From b332eeb29018551f79ff4db871feac674eb7df89 Mon Sep 17 00:00:00 2001 From: Jiangjie Qin Date: Wed, 8 Mar 2023 08:11:41 +0800 Subject: [PATCH 001/435] Add MetricsContainer support to the Flink sources. --- .../streaming/io/source/SourceTestCompat.java | 13 ++ .../streaming/io/source/SourceTestCompat.java | 13 ++ .../beam/runners/flink/FlinkRunnerResult.java | 4 +- .../flink/metrics/FlinkMetricContainer.java | 161 +-------------- .../metrics/FlinkMetricContainerBase.java | 189 ++++++++++++++++++ ...linkMetricContainerWithoutAccumulator.java | 41 ++++ .../flink/metrics/ReaderInvocationUtil.java | 4 +- .../streaming/io/source/FlinkSource.java | 12 +- .../io/source/FlinkSourceReaderBase.java | 13 +- .../io/source/bounded/FlinkBoundedSource.java | 8 +- .../bounded/FlinkBoundedSourceReader.java | 10 +- .../unbounded/FlinkUnboundedSource.java | 13 +- .../unbounded/FlinkUnboundedSourceReader.java | 6 +- .../metrics/FlinkMetricContainerTest.java | 2 +- .../streaming/io/TestCountingSource.java | 7 + .../io/source/FlinkSourceReaderTestBase.java | 28 +++ .../bounded/FlinkBoundedSourceReaderTest.java | 5 +- .../FlinkUnboundedSourceReaderTest.java | 5 +- 18 files changed, 353 insertions(+), 181 deletions(-) create mode 100644 runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/FlinkMetricContainerBase.java create mode 100644 runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/FlinkMetricContainerWithoutAccumulator.java diff --git a/runners/flink/1.12/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/SourceTestCompat.java b/runners/flink/1.12/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/SourceTestCompat.java index 1ddc2a957b7d..0b9ca07f99a9 100644 --- a/runners/flink/1.12/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/SourceTestCompat.java +++ b/runners/flink/1.12/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/SourceTestCompat.java @@ -33,6 +33,7 @@ public class SourceTestCompat { public static class TestMetricGroup extends UnregisteredMetricGroups.UnregisteredOperatorMetricGroup { public final Map> registeredGauge = new HashMap<>(); + public final Map registeredCounter = new HashMap<>(); public final Counter numRecordsInCounter = new SimpleCounter(); @Override @@ -41,6 +42,18 @@ public > GaugeT gauge(String name, GaugeT gauge) { return gauge; } + @Override + public Counter counter(String name) { + // The OperatorIOMetricsGroup will register some IO metrics in the constructor. + // At that time, the construction of this class has not finihsed yet, so we + // need to delegate the call to the parent class. + if (registeredCounter != null) { + return registeredCounter.computeIfAbsent(name, ignored -> super.counter(name)); + } else { + return super.counter(name); + } + } + @Override public OperatorIOMetricGroup getIOMetricGroup() { return new OperatorIOMetricGroup(this) { diff --git a/runners/flink/1.14/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/SourceTestCompat.java b/runners/flink/1.14/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/SourceTestCompat.java index 62b16eedca0b..8cda1341fd22 100644 --- a/runners/flink/1.14/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/SourceTestCompat.java +++ b/runners/flink/1.14/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/SourceTestCompat.java @@ -34,6 +34,7 @@ public class SourceTestCompat { public static class TestMetricGroup extends UnregisteredMetricsGroup implements SourceReaderMetricGroup { public final Map> registeredGauge = new HashMap<>(); + public final Map registeredCounter = new HashMap<>(); public final Counter numRecordsInCounter = new SimpleCounter(); @Override @@ -52,6 +53,18 @@ public > GaugeT gauge(String name, GaugeT gauge) { return gauge; } + @Override + public Counter counter(String name) { + // The OperatorIOMetricsGroup will register some IO metrics in the constructor. + // At that time, the construction of this class has not finihsed yet, so we + // need to delegate the call to the parent class. + if (registeredCounter != null) { + return registeredCounter.computeIfAbsent(name, ignored -> super.counter(name)); + } else { + return super.counter(name); + } + } + @Override public Counter getNumRecordsInErrorsCounter() { return new SimpleCounter(); diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkRunnerResult.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkRunnerResult.java index 885571a7ee77..d892049bce4b 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkRunnerResult.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkRunnerResult.java @@ -18,11 +18,11 @@ package org.apache.beam.runners.flink; import static org.apache.beam.runners.core.metrics.MetricsContainerStepMap.asAttemptedOnlyMetricResults; +import static org.apache.beam.runners.flink.metrics.FlinkMetricContainer.ACCUMULATOR_NAME; import java.util.Collections; import java.util.Map; import org.apache.beam.runners.core.metrics.MetricsContainerStepMap; -import org.apache.beam.runners.flink.metrics.FlinkMetricContainer; import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.metrics.MetricResults; import org.joda.time.Duration; @@ -80,6 +80,6 @@ public MetricResults metrics() { } MetricsContainerStepMap getMetricsContainerStepMap() { - return (MetricsContainerStepMap) accumulators.get(FlinkMetricContainer.ACCUMULATOR_NAME); + return (MetricsContainerStepMap) accumulators.get(ACCUMULATOR_NAME); } } diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/FlinkMetricContainer.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/FlinkMetricContainer.java index c05db9e0b156..7aa4ba784cc7 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/FlinkMetricContainer.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/FlinkMetricContainer.java @@ -17,29 +17,11 @@ */ package org.apache.beam.runners.flink.metrics; -import static org.apache.beam.runners.core.metrics.MetricsContainerStepMap.asAttemptedOnlyMetricResults; - -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import org.apache.beam.model.pipeline.v1.MetricsApi.MonitoringInfo; import org.apache.beam.runners.core.metrics.MetricsContainerImpl; import org.apache.beam.runners.core.metrics.MetricsContainerStepMap; -import org.apache.beam.sdk.metrics.DistributionResult; -import org.apache.beam.sdk.metrics.GaugeResult; -import org.apache.beam.sdk.metrics.MetricKey; -import org.apache.beam.sdk.metrics.MetricName; -import org.apache.beam.sdk.metrics.MetricQueryResults; -import org.apache.beam.sdk.metrics.MetricResult; -import org.apache.beam.sdk.metrics.MetricResults; -import org.apache.beam.sdk.metrics.MetricsFilter; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.api.common.functions.RuntimeContext; -import org.apache.flink.configuration.GlobalConfiguration; -import org.apache.flink.configuration.MetricOptions; -import org.apache.flink.metrics.Counter; -import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.MetricGroup; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -52,31 +34,14 @@ * which have a defined end. They are not essential during execution because metrics will also be * reported using the configured metrics reporter. */ -public class FlinkMetricContainer { - +public class FlinkMetricContainer extends FlinkMetricContainerBase { public static final String ACCUMULATOR_NAME = "__metricscontainers"; - private static final Logger LOG = LoggerFactory.getLogger(FlinkMetricContainer.class); - private static final String METRIC_KEY_SEPARATOR = - GlobalConfiguration.loadConfiguration().getString(MetricOptions.SCOPE_DELIMITER); - - private final MetricsContainerStepMap metricsContainers; private final RuntimeContext runtimeContext; - private final Map flinkCounterCache; - private final Map flinkDistributionGaugeCache; - private final Map flinkGaugeCache; public FlinkMetricContainer(RuntimeContext runtimeContext) { this.runtimeContext = runtimeContext; - this.flinkCounterCache = new HashMap<>(); - this.flinkDistributionGaugeCache = new HashMap<>(); - this.flinkGaugeCache = new HashMap<>(); - this.metricsContainers = new MetricsContainerStepMap(); - } - - public MetricsContainerImpl getMetricsContainer(String stepName) { - return metricsContainers.getContainer(stepName); } /** @@ -100,124 +65,8 @@ public void registerMetricsForPipelineResult() { metricsAccumulator.add(metricsContainers); } - /** - * Update this container with metrics from the passed {@link MonitoringInfo}s, and send updates - * along to Flink's internal metrics framework. - */ - public void updateMetrics(String stepName, List monitoringInfos) { - getMetricsContainer(stepName).update(monitoringInfos); - updateMetrics(stepName); - } - - /** - * Update Flink's internal metrics ({@link this#flinkCounterCache}) with the latest metrics for a - * given step. - */ - void updateMetrics(String stepName) { - MetricResults metricResults = asAttemptedOnlyMetricResults(metricsContainers); - MetricQueryResults metricQueryResults = - metricResults.queryMetrics(MetricsFilter.builder().addStep(stepName).build()); - updateCounters(metricQueryResults.getCounters()); - updateDistributions(metricQueryResults.getDistributions()); - updateGauge(metricQueryResults.getGauges()); - } - - private void updateCounters(Iterable> counters) { - for (MetricResult metricResult : counters) { - String flinkMetricName = getFlinkMetricNameString(metricResult.getKey()); - - Long update = metricResult.getAttempted(); - - // update flink metric - Counter counter = - flinkCounterCache.computeIfAbsent( - flinkMetricName, n -> runtimeContext.getMetricGroup().counter(n)); - // Beam counters are already pre-aggregated, just update with the current value here - counter.inc(update - counter.getCount()); - } - } - - private void updateDistributions(Iterable> distributions) { - for (MetricResult metricResult : distributions) { - String flinkMetricName = getFlinkMetricNameString(metricResult.getKey()); - - DistributionResult update = metricResult.getAttempted(); - - // update flink metric - FlinkDistributionGauge gauge = flinkDistributionGaugeCache.get(flinkMetricName); - if (gauge == null) { - gauge = - runtimeContext - .getMetricGroup() - .gauge(flinkMetricName, new FlinkDistributionGauge(update)); - flinkDistributionGaugeCache.put(flinkMetricName, gauge); - } else { - gauge.update(update); - } - } - } - - private void updateGauge(Iterable> gauges) { - for (MetricResult metricResult : gauges) { - String flinkMetricName = getFlinkMetricNameString(metricResult.getKey()); - - GaugeResult update = metricResult.getAttempted(); - - // update flink metric - FlinkGauge gauge = flinkGaugeCache.get(flinkMetricName); - if (gauge == null) { - gauge = runtimeContext.getMetricGroup().gauge(flinkMetricName, new FlinkGauge(update)); - flinkGaugeCache.put(flinkMetricName, gauge); - } else { - gauge.update(update); - } - } - } - - @VisibleForTesting - static String getFlinkMetricNameString(MetricKey metricKey) { - MetricName metricName = metricKey.metricName(); - // We use only the MetricName here, the step name is already contained - // in the operator name which is passed to Flink's MetricGroup to which - // the metric with the following name will be added. - return metricName.getNamespace() + METRIC_KEY_SEPARATOR + metricName.getName(); - } - - /** Flink {@link Gauge} for {@link DistributionResult}. */ - public static class FlinkDistributionGauge implements Gauge { - - DistributionResult data; - - FlinkDistributionGauge(DistributionResult data) { - this.data = data; - } - - void update(DistributionResult data) { - this.data = data; - } - - @Override - public DistributionResult getValue() { - return data; - } - } - - /** Flink {@link Gauge} for {@link GaugeResult}. */ - public static class FlinkGauge implements Gauge { - - GaugeResult data; - - FlinkGauge(GaugeResult data) { - this.data = data; - } - - void update(GaugeResult update) { - this.data = update; - } - - @Override - public Long getValue() { - return data.getValue(); - } + @Override + protected MetricGroup getMetricGroup() { + return runtimeContext.getMetricGroup(); } } diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/FlinkMetricContainerBase.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/FlinkMetricContainerBase.java new file mode 100644 index 000000000000..3a430bd6e368 --- /dev/null +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/FlinkMetricContainerBase.java @@ -0,0 +1,189 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.flink.metrics; + +import static org.apache.beam.runners.core.metrics.MetricsContainerStepMap.asAttemptedOnlyMetricResults; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.beam.model.pipeline.v1.MetricsApi; +import org.apache.beam.runners.core.metrics.MetricsContainerImpl; +import org.apache.beam.runners.core.metrics.MetricsContainerStepMap; +import org.apache.beam.sdk.metrics.DistributionResult; +import org.apache.beam.sdk.metrics.GaugeResult; +import org.apache.beam.sdk.metrics.MetricKey; +import org.apache.beam.sdk.metrics.MetricName; +import org.apache.beam.sdk.metrics.MetricQueryResults; +import org.apache.beam.sdk.metrics.MetricResult; +import org.apache.beam.sdk.metrics.MetricResults; +import org.apache.beam.sdk.metrics.MetricsFilter; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting; +import org.apache.flink.configuration.GlobalConfiguration; +import org.apache.flink.configuration.MetricOptions; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.MetricGroup; + +/** + * The base helper class for holding a {@link MetricsContainerImpl} and forwarding Beam metrics to + * Flink accumulators and metrics. The two subclasses of this base class are {@link + * FlinkMetricContainer} and {@link FlinkMetricContainerWithoutAccumulator}. The former is used when + * {@link org.apache.flink.api.common.functions.RuntimeContext Flink RuntimeContext} is available. + * The latter is used otherwise. + */ +abstract class FlinkMetricContainerBase { + + private static final String METRIC_KEY_SEPARATOR = + GlobalConfiguration.loadConfiguration().getString(MetricOptions.SCOPE_DELIMITER); + + protected final MetricsContainerStepMap metricsContainers; + private final Map flinkCounterCache; + private final Map flinkDistributionGaugeCache; + private final Map flinkGaugeCache; + + public FlinkMetricContainerBase() { + this.flinkCounterCache = new HashMap<>(); + this.flinkDistributionGaugeCache = new HashMap<>(); + this.flinkGaugeCache = new HashMap<>(); + this.metricsContainers = new MetricsContainerStepMap(); + } + + protected abstract MetricGroup getMetricGroup(); + + public MetricsContainerImpl getMetricsContainer(String stepName) { + return metricsContainers.getContainer(stepName); + } + + /** + * Update this container with metrics from the passed {@link MetricsApi.MonitoringInfo}s, and send + * updates along to Flink's internal metrics framework. + */ + public void updateMetrics(String stepName, List monitoringInfos) { + getMetricsContainer(stepName).update(monitoringInfos); + updateMetrics(stepName); + } + + /** + * Update Flink's internal metrics ({@link this#flinkCounterCache}) with the latest metrics for a + * given step. + */ + void updateMetrics(String stepName) { + MetricResults metricResults = asAttemptedOnlyMetricResults(metricsContainers); + MetricQueryResults metricQueryResults = + metricResults.queryMetrics(MetricsFilter.builder().addStep(stepName).build()); + updateCounters(metricQueryResults.getCounters()); + updateDistributions(metricQueryResults.getDistributions()); + updateGauge(metricQueryResults.getGauges()); + } + + private void updateCounters(Iterable> counters) { + for (MetricResult metricResult : counters) { + String flinkMetricName = getFlinkMetricNameString(metricResult.getKey()); + + Long update = metricResult.getAttempted(); + + // update flink metric + Counter counter = + flinkCounterCache.computeIfAbsent(flinkMetricName, n -> getMetricGroup().counter(n)); + // Beam counters are already pre-aggregated, just update with the current value here + counter.inc(update - counter.getCount()); + } + } + + private void updateDistributions(Iterable> distributions) { + for (MetricResult metricResult : distributions) { + String flinkMetricName = getFlinkMetricNameString(metricResult.getKey()); + + DistributionResult update = metricResult.getAttempted(); + + // update flink metric + FlinkDistributionGauge gauge = flinkDistributionGaugeCache.get(flinkMetricName); + if (gauge == null) { + gauge = getMetricGroup().gauge(flinkMetricName, new FlinkDistributionGauge(update)); + flinkDistributionGaugeCache.put(flinkMetricName, gauge); + } else { + gauge.update(update); + } + } + } + + private void updateGauge(Iterable> gauges) { + for (MetricResult metricResult : gauges) { + String flinkMetricName = getFlinkMetricNameString(metricResult.getKey()); + + GaugeResult update = metricResult.getAttempted(); + + // update flink metric + FlinkGauge gauge = flinkGaugeCache.get(flinkMetricName); + if (gauge == null) { + gauge = getMetricGroup().gauge(flinkMetricName, new FlinkGauge(update)); + flinkGaugeCache.put(flinkMetricName, gauge); + } else { + gauge.update(update); + } + } + } + + @VisibleForTesting + static String getFlinkMetricNameString(MetricKey metricKey) { + MetricName metricName = metricKey.metricName(); + // We use only the MetricName here, the step name is already contained + // in the operator name which is passed to Flink's MetricGroup to which + // the metric with the following name will be added. + return metricName.getNamespace() + METRIC_KEY_SEPARATOR + metricName.getName(); + } + + /** Flink {@link Gauge} for {@link DistributionResult}. */ + public static class FlinkDistributionGauge implements Gauge { + + DistributionResult data; + + FlinkDistributionGauge(DistributionResult data) { + this.data = data; + } + + void update(DistributionResult data) { + this.data = data; + } + + @Override + public DistributionResult getValue() { + return data; + } + } + + /** Flink {@link Gauge} for {@link GaugeResult}. */ + public static class FlinkGauge implements Gauge { + + GaugeResult data; + + FlinkGauge(GaugeResult data) { + this.data = data; + } + + void update(GaugeResult update) { + this.data = update; + } + + @Override + public Long getValue() { + return data.getValue(); + } + } +} diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/FlinkMetricContainerWithoutAccumulator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/FlinkMetricContainerWithoutAccumulator.java new file mode 100644 index 000000000000..00b1ea052e50 --- /dev/null +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/FlinkMetricContainerWithoutAccumulator.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.flink.metrics; + +import org.apache.flink.metrics.MetricGroup; + +/** + * The base helper class for holding a {@link + * org.apache.beam.runners.core.metrics.MetricsContainerImpl MetricsContainerImpl} and forwarding + * Beam metrics to Flink accumulators and metrics. This class is used when {@link + * org.apache.flink.api.common.functions.RuntimeContext Flink RuntimeContext} is not available. + * + * @see FlinkMetricContainer + */ +public class FlinkMetricContainerWithoutAccumulator extends FlinkMetricContainerBase { + private final MetricGroup metricGroup; + + public FlinkMetricContainerWithoutAccumulator(MetricGroup metricGroup) { + this.metricGroup = metricGroup; + } + + @Override + protected MetricGroup getMetricGroup() { + return metricGroup; + } +} diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/ReaderInvocationUtil.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/ReaderInvocationUtil.java index 736a2dd9da59..60b84e63263f 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/ReaderInvocationUtil.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/ReaderInvocationUtil.java @@ -33,11 +33,11 @@ public class ReaderInvocationUtil> { private final String stepName; - private final FlinkMetricContainer container; + private final FlinkMetricContainerBase container; private final Boolean enableMetrics; public ReaderInvocationUtil( - String stepName, PipelineOptions options, FlinkMetricContainer container) { + String stepName, PipelineOptions options, FlinkMetricContainerBase container) { FlinkPipelineOptions flinkPipelineOptions = options.as(FlinkPipelineOptions.class); this.stepName = stepName; this.enableMetrics = !flinkPipelineOptions.getDisableMetrics(); diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/FlinkSource.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/FlinkSource.java index c001b263340c..0b9fdd9dcd7c 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/FlinkSource.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/FlinkSource.java @@ -44,6 +44,8 @@ */ public abstract class FlinkSource implements Source, Map>>> { + + protected final String stepName; protected final org.apache.beam.sdk.io.Source beamSource; protected final Boundedness boundedness; protected final SerializablePipelineOptions serializablePipelineOptions; @@ -53,18 +55,20 @@ public abstract class FlinkSource // ----------------- public static methods to construct sources -------------------- public static FlinkBoundedSource bounded( + String stepName, BoundedSource boundedSource, SerializablePipelineOptions serializablePipelineOptions, int numSplits) { return new FlinkBoundedSource<>( - boundedSource, serializablePipelineOptions, Boundedness.BOUNDED, numSplits); + stepName, boundedSource, serializablePipelineOptions, Boundedness.BOUNDED, numSplits); } public static FlinkUnboundedSource unbounded( + String stepName, UnboundedSource source, SerializablePipelineOptions serializablePipelineOptions, int numSplits) { - return new FlinkUnboundedSource<>(source, serializablePipelineOptions, numSplits); + return new FlinkUnboundedSource<>(stepName, source, serializablePipelineOptions, numSplits); } public static FlinkBoundedSource unboundedImpulse(long shutdownSourceAfterIdleMs) { @@ -77,6 +81,7 @@ public static FlinkBoundedSource unboundedImpulse(long shutdownSourceAft // BeamImpulseSource will be discarded after the impulse emission. So the streaming // job won't see another impulse after failover. return new FlinkBoundedSource<>( + "Impulse", new BeamImpulseSource(), new SerializablePipelineOptions(flinkPipelineOptions), Boundedness.CONTINUOUS_UNBOUNDED, @@ -86,6 +91,7 @@ record -> Watermark.MAX_WATERMARK.getTimestamp()); public static FlinkBoundedSource boundedImpulse() { return new FlinkBoundedSource<>( + "Impulse", new BeamImpulseSource(), new SerializablePipelineOptions(FlinkPipelineOptions.defaults()), Boundedness.BOUNDED, @@ -96,10 +102,12 @@ record -> Watermark.MAX_WATERMARK.getTimestamp()); // ------ Common implementations for both bounded and unbounded source --------- protected FlinkSource( + String stepName, org.apache.beam.sdk.io.Source beamSource, SerializablePipelineOptions serializablePipelineOptions, Boundedness boundedness, int numSplits) { + this.stepName = stepName; this.beamSource = beamSource; this.serializablePipelineOptions = serializablePipelineOptions; this.boundedness = boundedness; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/FlinkSourceReaderBase.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/FlinkSourceReaderBase.java index 27b84910ac27..f0b93e0dde0f 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/FlinkSourceReaderBase.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/FlinkSourceReaderBase.java @@ -39,6 +39,8 @@ import javax.annotation.Nonnull; import javax.annotation.Nullable; import org.apache.beam.runners.flink.FlinkPipelineOptions; +import org.apache.beam.runners.flink.metrics.FlinkMetricContainerWithoutAccumulator; +import org.apache.beam.runners.flink.metrics.ReaderInvocationUtil; import org.apache.beam.runners.flink.translation.wrappers.streaming.io.source.compat.FlinkSourceCompat; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.io.BoundedSource; @@ -87,6 +89,7 @@ public abstract class FlinkSourceReaderBase protected final SourceReaderContext context; private final ScheduledExecutorService executor; + protected final ReaderInvocationUtil> invocationUtil; protected final Counter numRecordsInCounter; protected final long idleTimeoutMs; private final CompletableFuture idleTimeoutFuture; @@ -96,10 +99,12 @@ public abstract class FlinkSourceReaderBase private boolean noMoreSplits; protected FlinkSourceReaderBase( + String stepName, SourceReaderContext context, PipelineOptions pipelineOptions, @Nullable Function timestampExtractor) { this( + stepName, Executors.newSingleThreadScheduledExecutor( r -> new Thread(r, "FlinkSource-Executor-Thread-" + context.getIndexOfSubtask())), context, @@ -108,6 +113,7 @@ protected FlinkSourceReaderBase( } protected FlinkSourceReaderBase( + String stepName, ScheduledExecutorService executor, SourceReaderContext context, PipelineOptions pipelineOptions, @@ -126,6 +132,9 @@ protected FlinkSourceReaderBase( // TODO: Remove the casting and use SourceReaderMetricGroup after minimum FLink version is // upgraded to 1.14 and above. this.numRecordsInCounter = FlinkSourceCompat.getNumRecordsInCounter(context); + FlinkMetricContainerWithoutAccumulator metricsContainer = + new FlinkMetricContainerWithoutAccumulator(context.metricGroup()); + this.invocationUtil = new ReaderInvocationUtil<>(stepName, pipelineOptions, metricsContainer); } @Override @@ -368,10 +377,10 @@ public SourceOutput getAndMaybeCreateSplitOutput(ReaderOutput public boolean startOrAdvance() throws IOException { if (started) { - return reader.advance(); + return invocationUtil.invokeAdvance(reader); } else { started = true; - return reader.start(); + return invocationUtil.invokeStart(reader); } } diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/bounded/FlinkBoundedSource.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/bounded/FlinkBoundedSource.java index c2bd904dcc60..ab9a6cc03cd5 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/bounded/FlinkBoundedSource.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/bounded/FlinkBoundedSource.java @@ -41,20 +41,22 @@ public class FlinkBoundedSource extends FlinkSource> { protected final @Nullable TimestampExtractor> timestampExtractor; public FlinkBoundedSource( + String stepName, BoundedSource beamSource, SerializablePipelineOptions serializablePipelineOptions, Boundedness boundedness, int numSplits) { - this(beamSource, serializablePipelineOptions, boundedness, numSplits, null); + this(stepName, beamSource, serializablePipelineOptions, boundedness, numSplits, null); } public FlinkBoundedSource( + String stepName, BoundedSource beamSource, SerializablePipelineOptions serializablePipelineOptions, Boundedness boundedness, int numSplits, @Nullable TimestampExtractor> timestampExtractor) { - super(beamSource, serializablePipelineOptions, boundedness, numSplits); + super(stepName, beamSource, serializablePipelineOptions, boundedness, numSplits); this.timestampExtractor = timestampExtractor; } @@ -62,6 +64,6 @@ public FlinkBoundedSource( public SourceReader, FlinkSourceSplit> createReader( SourceReaderContext readerContext) throws Exception { return new FlinkBoundedSourceReader<>( - readerContext, serializablePipelineOptions.get(), timestampExtractor); + stepName, readerContext, serializablePipelineOptions.get(), timestampExtractor); } } diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/bounded/FlinkBoundedSourceReader.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/bounded/FlinkBoundedSourceReader.java index 7fb5fcc714c9..b015b527aa45 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/bounded/FlinkBoundedSourceReader.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/bounded/FlinkBoundedSourceReader.java @@ -54,20 +54,22 @@ public class FlinkBoundedSourceReader extends FlinkSourceReaderBase, Long> timestampExtractor) { - super(context, pipelineOptions, timestampExtractor); + super(stepName, context, pipelineOptions, timestampExtractor); currentSplitId = -1; } @VisibleForTesting protected FlinkBoundedSourceReader( + String stepName, SourceReaderContext context, PipelineOptions pipelineOptions, ScheduledExecutorService executor, @Nullable Function, Long> timestampExtractor) { - super(executor, context, pipelineOptions, timestampExtractor); + super(stepName, executor, context, pipelineOptions, timestampExtractor); currentSplitId = -1; } @@ -105,7 +107,7 @@ public InputStatus pollNext(ReaderOutput> output) throws Except // If the advance() invocation throws exception here, the job will just fail over and read // everything again from // the beginning. So the failover granularity is the entire Flink job. - if (!tempCurrentReader.advance()) { + if (!invocationUtil.invokeAdvance(tempCurrentReader)) { finishSplit(currentSplitId); currentReader = null; currentSplitId = -1; @@ -133,7 +135,7 @@ private boolean moveToNextNonEmptyReader() throws IOException { Optional readerAndOutput; while ((readerAndOutput = createAndTrackNextReader()).isPresent()) { ReaderAndOutput rao = readerAndOutput.get(); - if (rao.reader.start()) { + if (invocationUtil.invokeStart(rao.reader)) { currentSplitId = Integer.parseInt(rao.splitId); currentReader = rao.reader; return true; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/unbounded/FlinkUnboundedSource.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/unbounded/FlinkUnboundedSource.java index b40492201700..8ef2edfa606e 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/unbounded/FlinkUnboundedSource.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/unbounded/FlinkUnboundedSource.java @@ -40,18 +40,25 @@ public class FlinkUnboundedSource extends FlinkSource beamSource, SerializablePipelineOptions serializablePipelineOptions, int numSplits) { - this(beamSource, serializablePipelineOptions, numSplits, null); + this(stepName, beamSource, serializablePipelineOptions, numSplits, null); } public FlinkUnboundedSource( + String stepName, UnboundedSource beamSource, SerializablePipelineOptions serializablePipelineOptions, int numSplits, @Nullable TimestampExtractor>> timestampExtractor) { - super(beamSource, serializablePipelineOptions, Boundedness.CONTINUOUS_UNBOUNDED, numSplits); + super( + stepName, + beamSource, + serializablePipelineOptions, + Boundedness.CONTINUOUS_UNBOUNDED, + numSplits); this.timestampExtractor = timestampExtractor; } @@ -59,6 +66,6 @@ public FlinkUnboundedSource( public SourceReader>, FlinkSourceSplit> createReader( SourceReaderContext readerContext) throws Exception { return new FlinkUnboundedSourceReader<>( - readerContext, serializablePipelineOptions.get(), timestampExtractor); + stepName, readerContext, serializablePipelineOptions.get(), timestampExtractor); } } diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/unbounded/FlinkUnboundedSourceReader.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/unbounded/FlinkUnboundedSourceReader.java index 8f3595b9729d..04726990295c 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/unbounded/FlinkUnboundedSourceReader.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/unbounded/FlinkUnboundedSourceReader.java @@ -69,10 +69,11 @@ public class FlinkUnboundedSourceReader private volatile boolean shouldEmitWatermark; public FlinkUnboundedSourceReader( + String stepName, SourceReaderContext context, PipelineOptions pipelineOptions, @Nullable Function>, Long> timestampExtractor) { - super(context, pipelineOptions, timestampExtractor); + super(stepName, context, pipelineOptions, timestampExtractor); this.readers = new ArrayList<>(); this.dataAvailableFutureRef = new AtomicReference<>(DUMMY_FUTURE); this.currentReaderIndex = 0; @@ -80,11 +81,12 @@ public FlinkUnboundedSourceReader( @VisibleForTesting protected FlinkUnboundedSourceReader( + String stepName, SourceReaderContext context, PipelineOptions pipelineOptions, ScheduledExecutorService executor, @Nullable Function>, Long> timestampExtractor) { - super(executor, context, pipelineOptions, timestampExtractor); + super(stepName, executor, context, pipelineOptions, timestampExtractor); this.readers = new ArrayList<>(); this.dataAvailableFutureRef = new AtomicReference<>(DUMMY_FUTURE); this.currentReaderIndex = 0; diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/metrics/FlinkMetricContainerTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/metrics/FlinkMetricContainerTest.java index 0947ddda8d0b..a93a7663c451 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/metrics/FlinkMetricContainerTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/metrics/FlinkMetricContainerTest.java @@ -37,7 +37,7 @@ import org.apache.beam.runners.core.metrics.MonitoringInfoConstants; import org.apache.beam.runners.core.metrics.MonitoringInfoMetricName; import org.apache.beam.runners.core.metrics.SimpleMonitoringInfoBuilder; -import org.apache.beam.runners.flink.metrics.FlinkMetricContainer.FlinkDistributionGauge; +import org.apache.beam.runners.flink.metrics.FlinkMetricContainerBase.FlinkDistributionGauge; import org.apache.beam.sdk.metrics.Counter; import org.apache.beam.sdk.metrics.Distribution; import org.apache.beam.sdk.metrics.DistributionResult; diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/TestCountingSource.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/TestCountingSource.java index 5c54ce4c44e1..3af9062ba9b4 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/TestCountingSource.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/TestCountingSource.java @@ -29,6 +29,8 @@ import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.io.UnboundedSource; +import org.apache.beam.sdk.metrics.Counter; +import org.apache.beam.sdk.metrics.Metrics; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.values.KV; @@ -198,6 +200,10 @@ public boolean requiresDeduping() { */ public class CountingSourceReader extends UnboundedReader> implements TestReader { + public static final String ADVANCE_COUNTER_NAMESPACE = "testNameSpace"; + public static final String ADVANCE_COUNTER_NAME = "advanceCounter"; + private final Counter advanceCounter = + Metrics.counter(ADVANCE_COUNTER_NAMESPACE, ADVANCE_COUNTER_NAME); private int current; private boolean closed; @@ -213,6 +219,7 @@ public boolean start() { @Override public boolean advance() { + advanceCounter.inc(); if (current >= numMessagesPerShard - 1 || haltEmission) { return false; } diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/FlinkSourceReaderTestBase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/FlinkSourceReaderTestBase.java index dcab3aff0f5b..462a1ba0153d 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/FlinkSourceReaderTestBase.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/FlinkSourceReaderTestBase.java @@ -19,6 +19,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import static org.mockito.Mockito.when; @@ -32,6 +33,7 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.function.Function; import javax.annotation.Nullable; +import org.apache.beam.runners.flink.translation.wrappers.streaming.io.TestCountingSource; import org.apache.beam.sdk.io.Source; import org.apache.beam.sdk.values.KV; import org.apache.flink.api.common.eventtime.Watermark; @@ -40,6 +42,7 @@ import org.apache.flink.api.connector.source.SourceReader; import org.apache.flink.api.connector.source.SourceReaderContext; import org.apache.flink.core.testutils.ManuallyTriggeredScheduledExecutorService; +import org.apache.flink.metrics.Counter; import org.junit.Test; import org.mockito.Mockito; @@ -194,6 +197,31 @@ public void testNumBytesInMetrics() throws Exception { assertEquals(numRecordsPerSplit * numSplits, testMetricGroup.numRecordsInCounter.getCount()); } + @Test + public void testMetricsContainer() throws Exception { + ManuallyTriggeredScheduledExecutorService executor = + new ManuallyTriggeredScheduledExecutorService(); + SourceTestCompat.TestMetricGroup testMetricGroup = new SourceTestCompat.TestMetricGroup(); + try (SourceReader>> reader = + createReader(executor, 0L, null, testMetricGroup)) { + reader.start(); + + List>> splits = createSplits(2, 10, 0); + reader.addSplits(splits); + RecordsValidatingOutput validatingOutput = new RecordsValidatingOutput(splits); + + // Need to poll once to create all the readers. + reader.pollNext(validatingOutput); + Counter advanceCounter = + testMetricGroup.registeredCounter.get( + TestCountingSource.CountingSourceReader.ADVANCE_COUNTER_NAMESPACE + + "." + + TestCountingSource.CountingSourceReader.ADVANCE_COUNTER_NAME); + assertNotNull(advanceCounter); + assertTrue("The reader should have advanced.", advanceCounter.getCount() > 0); + } + } + // --------------- abstract methods --------------- protected abstract KV getKVPairs(OutputT record); diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/bounded/FlinkBoundedSourceReaderTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/bounded/FlinkBoundedSourceReaderTest.java index 6303a729652a..84cb2a72ddaf 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/bounded/FlinkBoundedSourceReaderTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/bounded/FlinkBoundedSourceReaderTest.java @@ -138,9 +138,10 @@ protected FlinkBoundedSourceReader> createReader( SourceReaderContext mockContext = createSourceReaderContext(testMetricGroup); if (executor != null) { return new FlinkBoundedSourceReader<>( - mockContext, pipelineOptions, executor, timestampExtractor); + "FlinkBoundedSource", mockContext, pipelineOptions, executor, timestampExtractor); } else { - return new FlinkBoundedSourceReader<>(mockContext, pipelineOptions, timestampExtractor); + return new FlinkBoundedSourceReader<>( + "FlinkBoundedSource", mockContext, pipelineOptions, timestampExtractor); } } } diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/unbounded/FlinkUnboundedSourceReaderTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/unbounded/FlinkUnboundedSourceReaderTest.java index f420bd8900ff..b7cba373cf75 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/unbounded/FlinkUnboundedSourceReaderTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/unbounded/FlinkUnboundedSourceReaderTest.java @@ -303,9 +303,10 @@ protected FlinkUnboundedSourceReader> createReader( SourceReaderContext mockContext = createSourceReaderContext(metricGroup); if (executor != null) { return new FlinkUnboundedSourceReader<>( - mockContext, pipelineOptions, executor, timestampExtractor); + "FlinkUnboundedReader", mockContext, pipelineOptions, executor, timestampExtractor); } else { - return new FlinkUnboundedSourceReader<>(mockContext, pipelineOptions, timestampExtractor); + return new FlinkUnboundedSourceReader<>( + "FlinkUnboundedReader", mockContext, pipelineOptions, timestampExtractor); } } From 3f2c5e76c07abaa4552dd9231a16f7596c4f1772 Mon Sep 17 00:00:00 2001 From: jto Date: Wed, 27 Sep 2023 17:09:54 +0200 Subject: [PATCH 002/435] Favor composition in FlinkMetricsContainer implementations --- .../beam/runners/flink/metrics/FlinkMetricContainer.java | 6 +----- .../runners/flink/metrics/FlinkMetricContainerBase.java | 8 ++++++-- .../metrics/FlinkMetricContainerWithoutAccumulator.java | 9 +-------- 3 files changed, 8 insertions(+), 15 deletions(-) diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/FlinkMetricContainer.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/FlinkMetricContainer.java index 7aa4ba784cc7..3f4d9617bdd2 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/FlinkMetricContainer.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/FlinkMetricContainer.java @@ -21,7 +21,6 @@ import org.apache.beam.runners.core.metrics.MetricsContainerStepMap; import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.api.common.functions.RuntimeContext; -import org.apache.flink.metrics.MetricGroup; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -41,6 +40,7 @@ public class FlinkMetricContainer extends FlinkMetricContainerBase { private final RuntimeContext runtimeContext; public FlinkMetricContainer(RuntimeContext runtimeContext) { + super(runtimeContext.getMetricGroup()); this.runtimeContext = runtimeContext; } @@ -65,8 +65,4 @@ public void registerMetricsForPipelineResult() { metricsAccumulator.add(metricsContainers); } - @Override - protected MetricGroup getMetricGroup() { - return runtimeContext.getMetricGroup(); - } } diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/FlinkMetricContainerBase.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/FlinkMetricContainerBase.java index 3a430bd6e368..a4a363689329 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/FlinkMetricContainerBase.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/FlinkMetricContainerBase.java @@ -56,15 +56,19 @@ abstract class FlinkMetricContainerBase { private final Map flinkCounterCache; private final Map flinkDistributionGaugeCache; private final Map flinkGaugeCache; + private final MetricGroup metricGroup; - public FlinkMetricContainerBase() { + public FlinkMetricContainerBase(MetricGroup metricGroup) { this.flinkCounterCache = new HashMap<>(); this.flinkDistributionGaugeCache = new HashMap<>(); this.flinkGaugeCache = new HashMap<>(); this.metricsContainers = new MetricsContainerStepMap(); + this.metricGroup = metricGroup; } - protected abstract MetricGroup getMetricGroup(); + public MetricGroup getMetricGroup() { + return metricGroup; + } public MetricsContainerImpl getMetricsContainer(String stepName) { return metricsContainers.getContainer(stepName); diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/FlinkMetricContainerWithoutAccumulator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/FlinkMetricContainerWithoutAccumulator.java index 00b1ea052e50..88d52273108a 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/FlinkMetricContainerWithoutAccumulator.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/FlinkMetricContainerWithoutAccumulator.java @@ -28,14 +28,7 @@ * @see FlinkMetricContainer */ public class FlinkMetricContainerWithoutAccumulator extends FlinkMetricContainerBase { - private final MetricGroup metricGroup; - public FlinkMetricContainerWithoutAccumulator(MetricGroup metricGroup) { - this.metricGroup = metricGroup; - } - - @Override - protected MetricGroup getMetricGroup() { - return metricGroup; + super(metricGroup); } } From 36ba537c6c2e90b13ac3dea7371bcf1b26bb5393 Mon Sep 17 00:00:00 2001 From: jto Date: Wed, 27 Sep 2023 17:42:32 +0200 Subject: [PATCH 003/435] Fix Guava import --- .../apache/beam/runners/flink/metrics/FlinkMetricContainer.java | 1 - .../beam/runners/flink/metrics/FlinkMetricContainerBase.java | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/FlinkMetricContainer.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/FlinkMetricContainer.java index 3f4d9617bdd2..eaa72445c3fb 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/FlinkMetricContainer.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/FlinkMetricContainer.java @@ -64,5 +64,4 @@ public void registerMetricsForPipelineResult() { } metricsAccumulator.add(metricsContainers); } - } diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/FlinkMetricContainerBase.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/FlinkMetricContainerBase.java index a4a363689329..a9a6db47c814 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/FlinkMetricContainerBase.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/FlinkMetricContainerBase.java @@ -33,7 +33,7 @@ import org.apache.beam.sdk.metrics.MetricResult; import org.apache.beam.sdk.metrics.MetricResults; import org.apache.beam.sdk.metrics.MetricsFilter; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.flink.configuration.GlobalConfiguration; import org.apache.flink.configuration.MetricOptions; import org.apache.flink.metrics.Counter; From 83355135b4206cbfd28b9488c60b5dedf429db1b Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Mon, 2 Oct 2023 11:07:33 -0700 Subject: [PATCH 004/435] Add raw format option. --- .../PubsubReadSchemaTransformProvider.java | 44 +++++++++++-------- .../PubsubWriteSchemaTransformProvider.java | 39 +++++++++++++--- ...PubsubReadSchemaTransformProviderTest.java | 32 ++++++++++++++ 3 files changed, 90 insertions(+), 25 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformProvider.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformProvider.java index 61a4cf68c987..c5e073347088 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformProvider.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformProvider.java @@ -63,7 +63,7 @@ public class PubsubReadSchemaTransformProvider extends TypedSchemaTransformProvider { - public static final String VALID_FORMATS_STR = "AVRO,JSON"; + public static final String VALID_FORMATS_STR = "RAW,AVRO,JSON"; public static final Set VALID_DATA_FORMATS = Sets.newHashSet(VALID_FORMATS_STR.split(",")); @@ -89,34 +89,40 @@ public SchemaTransform from(PubsubReadSchemaTransformConfiguration configuration "To read from Pubsub, a subscription name or a topic name must be provided. Not both."); } - if ((Strings.isNullOrEmpty(configuration.getSchema()) - && !Strings.isNullOrEmpty(configuration.getFormat())) - || (!Strings.isNullOrEmpty(configuration.getSchema()) - && Strings.isNullOrEmpty(configuration.getFormat()))) { - throw new IllegalArgumentException( - "A schema was provided without a data format (or viceversa). Please provide " - + "both of these parameters to read from Pubsub, or if you would like to use the Pubsub schema service," - + " please leave both of these blank."); + if (!"RAW".equals(configuration.getFormat())) { + if ((Strings.isNullOrEmpty(configuration.getSchema()) + && !Strings.isNullOrEmpty(configuration.getFormat())) + || (!Strings.isNullOrEmpty(configuration.getSchema()) + && Strings.isNullOrEmpty(configuration.getFormat()))) { + throw new IllegalArgumentException( + "A schema was provided without a data format (or viceversa). Please provide " + + "both of these parameters to read from Pubsub, or if you would like to use the Pubsub schema service," + + " please leave both of these blank."); + } } Schema beamSchema; SerializableFunction valueMapper; - if (!VALID_DATA_FORMATS.contains(configuration.getFormat())) { + String format = + configuration.getFormat() == null ? null : configuration.getFormat().toUpperCase(); + if (Objects.equals(format, "RAW")) { + beamSchema = Schema.of(Schema.Field.of("payload", Schema.FieldType.BYTES)); + valueMapper = input -> Row.withSchema(beamSchema).addValue(input).build(); + } else if (Objects.equals(format, "JSON")) { + beamSchema = JsonUtils.beamSchemaFromJsonSchema(configuration.getSchema()); + valueMapper = JsonUtils.getJsonBytesToRowFunction(beamSchema); + } else if (Objects.equals(format, "AVRO")) { + beamSchema = + AvroUtils.toBeamSchema( + new org.apache.avro.Schema.Parser().parse(configuration.getSchema())); + valueMapper = AvroUtils.getAvroBytesToRowFunction(beamSchema); + } else { throw new IllegalArgumentException( String.format( "Format %s not supported. Only supported formats are %s", configuration.getFormat(), VALID_FORMATS_STR)); } - beamSchema = - Objects.equals(configuration.getFormat(), "JSON") - ? JsonUtils.beamSchemaFromJsonSchema(configuration.getSchema()) - : AvroUtils.toBeamSchema( - new org.apache.avro.Schema.Parser().parse(configuration.getSchema())); - valueMapper = - Objects.equals(configuration.getFormat(), "JSON") - ? JsonUtils.getJsonBytesToRowFunction(beamSchema) - : AvroUtils.getAvroBytesToRowFunction(beamSchema); PubsubReadSchemaTransform transform = new PubsubReadSchemaTransform( diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubWriteSchemaTransformProvider.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubWriteSchemaTransformProvider.java index 11c3d18bd3dc..390029c8767c 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubWriteSchemaTransformProvider.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubWriteSchemaTransformProvider.java @@ -19,8 +19,10 @@ import com.google.auto.service.AutoService; import java.io.Serializable; +import java.nio.charset.StandardCharsets; import java.util.Collections; import java.util.List; +import java.util.Objects; import java.util.Set; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.transforms.SchemaTransform; @@ -56,7 +58,7 @@ public class PubsubWriteSchemaTransformProvider public static final TupleTag OUTPUT_TAG = new TupleTag() {}; public static final TupleTag ERROR_TAG = new TupleTag() {}; - public static final String VALID_FORMATS_STR = "AVRO,JSON"; + public static final String VALID_FORMATS_STR = "RAW,AVRO,JSON"; public static final Set VALID_DATA_FORMATS = Sets.newHashSet(VALID_FORMATS_STR.split(",")); @@ -88,7 +90,7 @@ public void processElement(@Element Row row, MultiOutputReceiver receiver) { @Override public SchemaTransform from(PubsubWriteSchemaTransformConfiguration configuration) { - if (!VALID_DATA_FORMATS.contains(configuration.getFormat())) { + if (!VALID_DATA_FORMATS.contains(configuration.getFormat().toUpperCase())) { throw new IllegalArgumentException( String.format( "Format %s not supported. Only supported formats are %s", @@ -113,10 +115,35 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) { .addStringField("error") .addNullableRowField("row", input.get("input").getSchema()) .build(); - SerializableFunction fn = - format.equals("AVRO") - ? AvroUtils.getRowToAvroBytesFunction(input.get("input").getSchema()) - : JsonUtils.getRowToJsonBytesFunction(input.get("input").getSchema()); + + Schema beamSchema = input.get("input").getSchema(); + SerializableFunction fn; + if (Objects.equals(format, "RAW")) { + if (beamSchema.getFieldCount() != 1) { + throw new IllegalArgumentException( + String.format( + "Raw output only supported for single-field schemas, got %s", beamSchema)); + } + if (beamSchema.getField(0).getType().equals(Schema.FieldType.BYTES)) { + fn = row -> row.getBytes(0); + } else if (beamSchema.getField(0).getType().equals(Schema.FieldType.STRING)) { + fn = row -> row.getString(0).getBytes(StandardCharsets.UTF_8); + } else { + throw new IllegalArgumentException( + String.format( + "Raw output only supports bytes and string fields, got %s", + beamSchema.getField(0))); + } + } else if (Objects.equals(format, "JSON")) { + fn = JsonUtils.getRowToJsonBytesFunction(beamSchema); + } else if (Objects.equals(format, "AVRO")) { + fn = AvroUtils.getRowToAvroBytesFunction(beamSchema); + } else { + throw new IllegalArgumentException( + String.format( + "Format %s not supported. Only supported formats are %s", + format, VALID_FORMATS_STR)); + } PCollectionTuple outputTuple = input diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformProviderTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformProviderTest.java index 0de998f11127..c2b32c8c9ba8 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformProviderTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformProviderTest.java @@ -46,6 +46,8 @@ import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.values.PCollectionRowTuple; import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.junit.Rule; import org.junit.Test; import org.junit.runner.RunWith; @@ -161,6 +163,36 @@ public void testNoSchema() { p.run().waitUntilFinish(); } + @Test + public void testReadRaw() throws IOException { + PCollectionRowTuple begin = PCollectionRowTuple.empty(p); + + Schema rawSchema = Schema.of(Schema.Field.of("payload", Schema.FieldType.BYTES)); + byte[] payload = "some payload".getBytes(Charsets.UTF_8); + + try (PubsubTestClientFactory clientFactory = + clientFactory(ImmutableList.of(incomingMessageOf(payload, CLOCK.currentTimeMillis())))) { + PubsubReadSchemaTransformConfiguration config = + PubsubReadSchemaTransformConfiguration.builder() + .setFormat("RAW") + .setSchema("") + .setSubscription(SUBSCRIPTION) + .setClientFactory(clientFactory) + .setClock(CLOCK) + .build(); + SchemaTransform transform = new PubsubReadSchemaTransformProvider().from(config); + PCollectionRowTuple reads = begin.apply(transform); + + PAssert.that(reads.get("output")) + .containsInAnyOrder( + ImmutableList.of(Row.withSchema(rawSchema).addValue(payload).build())); + + p.run().waitUntilFinish(); + } catch (Exception e) { + throw e; + } + } + @Test public void testReadAvro() throws IOException { PCollectionRowTuple begin = PCollectionRowTuple.empty(p); From 5bfd422c8d757e17d8fcdc7bfd7353955494d69c Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Mon, 2 Oct 2023 12:27:16 -0700 Subject: [PATCH 005/435] Timestamp and id attributes. --- ...ubsubReadSchemaTransformConfiguration.java | 24 +++++++++++++++++ .../PubsubReadSchemaTransformProvider.java | 26 +++++++++---------- ...bsubWriteSchemaTransformConfiguration.java | 15 +++++++++++ .../PubsubWriteSchemaTransformProvider.java | 22 +++++++++++----- 4 files changed, 67 insertions(+), 20 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformConfiguration.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformConfiguration.java index befb22ca6dc2..1774a4c330e0 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformConfiguration.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformConfiguration.java @@ -60,6 +60,26 @@ public abstract class PubsubReadSchemaTransformConfiguration { + "For JSON data, this is a schema defined with JSON-schema syntax (https://json-schema.org/).") public abstract String getSchema(); + @SchemaFieldDescription( + "When reading from Cloud Pub/Sub where unique record identifiers are provided as Pub/Sub message attributes, " + + "specifies the name of the attribute containing the unique identifier. " + + "The value of the attribute can be any string that uniquely identifies this record. " + + "Pub/Sub cannot guarantee that no duplicate data will be delivered on the Pub/Sub stream. " + + "If idAttribute is not provided, Beam cannot guarantee that no duplicate data will be delivered, " + + "and deduplication of the stream will be strictly best effort.") + public abstract String getIdAttribute(); + + @SchemaFieldDescription( + "Specifies the name of the attribute that contains the timestamp, if any. " + + "The timestamp value is expected to be represented in the attribute as either " + + "(1) a numerical value representing the number of milliseconds since the Unix epoch. " + + "For example, if using the Joda time classes, " + + "Instant.getMillis() returns the correct value for this attribute." + + " or (2) a String in RFC 3339 format. For example, 2015-10-29T23:41:41.123Z. " + + "The sub-second component of the timestamp is optional, and digits beyond the first three " + + "(i.e., time units smaller than milliseconds) will be ignored.") + public abstract String getTimestampAttribute(); + // Used for testing only. public abstract @Nullable PubsubTestClientFactory getClientFactory(); @@ -80,6 +100,10 @@ public abstract static class Builder { public abstract Builder setSchema(String schema); + public abstract Builder setIdAttribute(String schema); + + public abstract Builder setTimestampAttribute(String schema); + // Used for testing only. public abstract Builder setClientFactory(@Nullable PubsubTestClientFactory clientFactory); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformProvider.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformProvider.java index c5e073347088..c41d25197c08 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformProvider.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformProvider.java @@ -35,8 +35,6 @@ import org.apache.beam.sdk.schemas.utils.AvroUtils; import org.apache.beam.sdk.schemas.utils.JsonUtils; import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.DoFn.FinishBundle; -import org.apache.beam.sdk.transforms.DoFn.ProcessElement; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.values.PCollectionRowTuple; @@ -125,8 +123,7 @@ public SchemaTransform from(PubsubReadSchemaTransformConfiguration configuration } PubsubReadSchemaTransform transform = - new PubsubReadSchemaTransform( - configuration.getTopic(), configuration.getSubscription(), beamSchema, valueMapper); + new PubsubReadSchemaTransform(configuration, beamSchema, valueMapper); if (configuration.getClientFactory() != null) { transform.setClientFactory(configuration.getClientFactory()); @@ -141,18 +138,15 @@ public SchemaTransform from(PubsubReadSchemaTransformConfiguration configuration private static class PubsubReadSchemaTransform extends SchemaTransform implements Serializable { final Schema beamSchema; final SerializableFunction valueMapper; - final @Nullable String topic; - final @Nullable String subscription; + final PubsubReadSchemaTransformConfiguration configuration; @Nullable PubsubTestClientFactory clientFactory; @Nullable Clock clock; PubsubReadSchemaTransform( - @Nullable String topic, - @Nullable String subscription, + PubsubReadSchemaTransformConfiguration configuration, Schema beamSchema, SerializableFunction valueMapper) { - this.topic = topic; - this.subscription = subscription; + this.configuration = configuration; this.beamSchema = beamSchema; this.valueMapper = valueMapper; } @@ -201,10 +195,10 @@ void setClock(@Nullable Clock clock) { @SuppressWarnings("nullness") PubsubIO.Read buildPubsubRead() { PubsubIO.Read pubsubRead = PubsubIO.readMessages(); - if (!Strings.isNullOrEmpty(topic)) { - pubsubRead = pubsubRead.fromTopic(topic); + if (!Strings.isNullOrEmpty(configuration.getTopic())) { + pubsubRead = pubsubRead.fromTopic(configuration.getTopic()); } else { - pubsubRead = pubsubRead.fromSubscription(subscription); + pubsubRead = pubsubRead.fromSubscription(configuration.getSubscription()); } if (clientFactory != null && clock != null) { pubsubRead = pubsubRead.withClientFactory(clientFactory); @@ -213,6 +207,12 @@ PubsubIO.Read buildPubsubRead() { throw new IllegalArgumentException( "Both PubsubTestClientFactory and Clock need to be specified for testing, but only one is provided"); } + if (!Strings.isNullOrEmpty(configuration.getIdAttribute())) { + pubsubRead = pubsubRead.withIdAttribute(configuration.getIdAttribute()); + } + if (!Strings.isNullOrEmpty(configuration.getTimestampAttribute())) { + pubsubRead = pubsubRead.withTimestampAttribute(configuration.getTimestampAttribute()); + } return pubsubRead; } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubWriteSchemaTransformConfiguration.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubWriteSchemaTransformConfiguration.java index 57620c968c5f..cd49ca6ca07b 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubWriteSchemaTransformConfiguration.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubWriteSchemaTransformConfiguration.java @@ -18,6 +18,7 @@ package org.apache.beam.sdk.io.gcp.pubsub; import com.google.auto.value.AutoValue; +import javax.annotation.Nullable; import org.apache.beam.sdk.schemas.AutoValueSchema; import org.apache.beam.sdk.schemas.annotations.DefaultSchema; import org.apache.beam.sdk.schemas.annotations.SchemaFieldDescription; @@ -41,6 +42,16 @@ public abstract class PubsubWriteSchemaTransformConfiguration { "The name of the topic to write data to. " + "Format: projects/${PROJECT}/topics/${TOPIC}") public abstract String getTopic(); + @SchemaFieldDescription( + "If set, will set an attribute for each Cloud Pub/Sub message with the given name and a unique value. " + + "This attribute can then be used in a ReadFromPubSub PTransform to deduplicate messages.") + public abstract @Nullable String getIdAttribute(); + + @SchemaFieldDescription( + "If set, will set an attribute for each Cloud Pub/Sub message with the given name and the message's " + + "publish time as the value.") + public abstract @Nullable String getTimestampAttribute(); + public static Builder builder() { return new AutoValue_PubsubWriteSchemaTransformConfiguration.Builder(); } @@ -51,6 +62,10 @@ public abstract static class Builder { public abstract Builder setTopic(String topic); + public abstract Builder setIdAttribute(String idAttribute); + + public abstract Builder setTimestampAttribute(String timestampAttribute); + public abstract PubsubWriteSchemaTransformConfiguration build(); } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubWriteSchemaTransformProvider.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubWriteSchemaTransformProvider.java index 390029c8767c..e720a0331a07 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubWriteSchemaTransformProvider.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubWriteSchemaTransformProvider.java @@ -38,6 +38,7 @@ import org.apache.beam.sdk.values.Row; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Strings; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Sets; import org.checkerframework.checker.initialization.qual.Initialized; import org.checkerframework.checker.nullness.qual.NonNull; @@ -96,16 +97,14 @@ public SchemaTransform from(PubsubWriteSchemaTransformConfiguration configuratio "Format %s not supported. Only supported formats are %s", configuration.getFormat(), VALID_FORMATS_STR)); } - return new PubsubWriteSchemaTransform(configuration.getTopic(), configuration.getFormat()); + return new PubsubWriteSchemaTransform(configuration); } private static class PubsubWriteSchemaTransform extends SchemaTransform implements Serializable { - final String topic; - final String format; + final PubsubWriteSchemaTransformConfiguration configuration; - PubsubWriteSchemaTransform(String topic, String format) { - this.topic = topic; - this.format = format; + PubsubWriteSchemaTransform(PubsubWriteSchemaTransformConfiguration configuration) { + this.configuration = configuration; } @Override @@ -116,6 +115,7 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) { .addNullableRowField("row", input.get("input").getSchema()) .build(); + String format = configuration.getFormat(); Schema beamSchema = input.get("input").getSchema(); SerializableFunction fn; if (Objects.equals(format, "RAW")) { @@ -152,7 +152,15 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) { ParDo.of(new ErrorFn(fn, errorSchema)) .withOutputTags(OUTPUT_TAG, TupleTagList.of(ERROR_TAG))); - outputTuple.get(OUTPUT_TAG).apply(PubsubIO.writeMessages().to(topic)); + PubsubIO.Write writeTransform = + PubsubIO.writeMessages().to(configuration.getTopic()); + if (!Strings.isNullOrEmpty(configuration.getIdAttribute())) { + writeTransform = writeTransform.withIdAttribute(configuration.getIdAttribute()); + } + if (!Strings.isNullOrEmpty(configuration.getTimestampAttribute())) { + writeTransform = writeTransform.withIdAttribute(configuration.getTimestampAttribute()); + } + outputTuple.get(OUTPUT_TAG).apply(writeTransform); return PCollectionRowTuple.of("errors", outputTuple.get(ERROR_TAG).setRowSchema(errorSchema)); } From 0940e900681385545d4099838a95b6e80b977da9 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Mon, 2 Oct 2023 13:35:12 -0700 Subject: [PATCH 006/435] Attribute handling. --- .../java/org/apache/beam/sdk/values/Row.java | 8 +- ...ubsubReadSchemaTransformConfiguration.java | 21 +++- .../PubsubReadSchemaTransformProvider.java | 97 +++++++++++++++---- ...bsubWriteSchemaTransformConfiguration.java | 17 +++- .../PubsubWriteSchemaTransformProvider.java | 88 ++++++++++++++--- ...PubsubReadSchemaTransformProviderTest.java | 61 ++++++++++++ 6 files changed, 252 insertions(+), 40 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/Row.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/Row.java index 9f5546597c90..f5c6c7fcf34a 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/Row.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/Row.java @@ -786,12 +786,12 @@ public FieldValueBuilder withFieldValues(Map values) { // withFieldValue or // withFieldValues. - public Builder addValue(@Nullable Object values) { - this.values.add(values); + public Builder addValue(@Nullable Object value) { + this.values.add(value); return this; } - public Builder addValues(List values) { + public Builder addValues(List<@Nullable Object> values) { this.values.addAll(values); return this; } @@ -822,7 +822,7 @@ public Builder addIterable(Iterable values) { // method is largely // used internal to Beam. @Internal - public Row attachValues(List attachedValues) { + public Row attachValues(List<@Nullable Object> attachedValues) { checkState(this.values.isEmpty()); return new RowWithStorage(schema, attachedValues); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformConfiguration.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformConfiguration.java index 1774a4c330e0..cd957e4feedd 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformConfiguration.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformConfiguration.java @@ -19,6 +19,7 @@ import com.google.api.client.util.Clock; import com.google.auto.value.AutoValue; +import java.util.List; import javax.annotation.Nullable; import org.apache.beam.sdk.io.gcp.pubsub.PubsubTestClient.PubsubTestClientFactory; import org.apache.beam.sdk.schemas.AutoValueSchema; @@ -60,6 +61,14 @@ public abstract class PubsubReadSchemaTransformConfiguration { + "For JSON data, this is a schema defined with JSON-schema syntax (https://json-schema.org/).") public abstract String getSchema(); + @SchemaFieldDescription( + "Any additional pubsub attributes that should be populated as String fields in the ouptut rows.") + public abstract @Nullable List getAttributes(); + + @SchemaFieldDescription( + "Any additional field that should be populated with the full set of PubSub attributes.") + public abstract @Nullable String getAttributesMap(); + @SchemaFieldDescription( "When reading from Cloud Pub/Sub where unique record identifiers are provided as Pub/Sub message attributes, " + "specifies the name of the attribute containing the unique identifier. " @@ -67,7 +76,7 @@ public abstract class PubsubReadSchemaTransformConfiguration { + "Pub/Sub cannot guarantee that no duplicate data will be delivered on the Pub/Sub stream. " + "If idAttribute is not provided, Beam cannot guarantee that no duplicate data will be delivered, " + "and deduplication of the stream will be strictly best effort.") - public abstract String getIdAttribute(); + public abstract @Nullable String getIdAttribute(); @SchemaFieldDescription( "Specifies the name of the attribute that contains the timestamp, if any. " @@ -78,7 +87,7 @@ public abstract class PubsubReadSchemaTransformConfiguration { + " or (2) a String in RFC 3339 format. For example, 2015-10-29T23:41:41.123Z. " + "The sub-second component of the timestamp is optional, and digits beyond the first three " + "(i.e., time units smaller than milliseconds) will be ignored.") - public abstract String getTimestampAttribute(); + public abstract @Nullable String getTimestampAttribute(); // Used for testing only. public abstract @Nullable PubsubTestClientFactory getClientFactory(); @@ -100,9 +109,13 @@ public abstract static class Builder { public abstract Builder setSchema(String schema); - public abstract Builder setIdAttribute(String schema); + public abstract Builder setAttributes(@Nullable List attributes); + + public abstract Builder setAttributesMap(@Nullable String attributesMap); + + public abstract Builder setIdAttribute(@Nullable String schema); - public abstract Builder setTimestampAttribute(String schema); + public abstract Builder setTimestampAttribute(@Nullable String schema); // Used for testing only. public abstract Builder setClientFactory(@Nullable PubsubTestClientFactory clientFactory); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformProvider.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformProvider.java index c41d25197c08..b5e92d84910c 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformProvider.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformProvider.java @@ -99,22 +99,22 @@ public SchemaTransform from(PubsubReadSchemaTransformConfiguration configuration } } - Schema beamSchema; - SerializableFunction valueMapper; + Schema payloadSchema; + SerializableFunction payloadMapper; String format = configuration.getFormat() == null ? null : configuration.getFormat().toUpperCase(); if (Objects.equals(format, "RAW")) { - beamSchema = Schema.of(Schema.Field.of("payload", Schema.FieldType.BYTES)); - valueMapper = input -> Row.withSchema(beamSchema).addValue(input).build(); + payloadSchema = Schema.of(Schema.Field.of("payload", Schema.FieldType.BYTES)); + payloadMapper = input -> Row.withSchema(payloadSchema).addValue(input).build(); } else if (Objects.equals(format, "JSON")) { - beamSchema = JsonUtils.beamSchemaFromJsonSchema(configuration.getSchema()); - valueMapper = JsonUtils.getJsonBytesToRowFunction(beamSchema); + payloadSchema = JsonUtils.beamSchemaFromJsonSchema(configuration.getSchema()); + payloadMapper = JsonUtils.getJsonBytesToRowFunction(payloadSchema); } else if (Objects.equals(format, "AVRO")) { - beamSchema = + payloadSchema = AvroUtils.toBeamSchema( new org.apache.avro.Schema.Parser().parse(configuration.getSchema())); - valueMapper = AvroUtils.getAvroBytesToRowFunction(beamSchema); + payloadMapper = AvroUtils.getAvroBytesToRowFunction(payloadSchema); } else { throw new IllegalArgumentException( String.format( @@ -123,7 +123,7 @@ public SchemaTransform from(PubsubReadSchemaTransformConfiguration configuration } PubsubReadSchemaTransform transform = - new PubsubReadSchemaTransform(configuration, beamSchema, valueMapper); + new PubsubReadSchemaTransform(configuration, payloadSchema, payloadMapper); if (configuration.getClientFactory() != null) { transform.setClientFactory(configuration.getClientFactory()); @@ -144,28 +144,81 @@ private static class PubsubReadSchemaTransform extends SchemaTransform implement PubsubReadSchemaTransform( PubsubReadSchemaTransformConfiguration configuration, - Schema beamSchema, + Schema payloadSchema, SerializableFunction valueMapper) { this.configuration = configuration; - this.beamSchema = beamSchema; + Schema outputSchema; + List attributes = configuration.getAttributes(); + String attributesMap = configuration.getAttributesMap(); + if (attributes == null && attributesMap == null) { + outputSchema = payloadSchema; + } else { + Schema.Builder outputSchemaBuilder = Schema.builder(); + outputSchemaBuilder.addFields(payloadSchema.getFields()); + if (attributes != null) { + for (String attribute : attributes) { + outputSchemaBuilder.addStringField(attribute); + } + } + if (attributesMap != null) { + outputSchemaBuilder.addMapField( + attributesMap, Schema.FieldType.STRING, Schema.FieldType.STRING); + } + outputSchema = outputSchemaBuilder.build(); + } + this.beamSchema = outputSchema; this.valueMapper = valueMapper; } private static class ErrorCounterFn extends DoFn { - private Counter pubsubErrorCounter; + private final Counter pubsubErrorCounter; private Long errorsInBundle = 0L; - private SerializableFunction valueMapper; + private final SerializableFunction valueMapper; + private final @Nullable List attributes; + private final @Nullable String attributesMap; + private final Schema outputSchema; - ErrorCounterFn(String name, SerializableFunction valueMapper) { + ErrorCounterFn( + String name, + SerializableFunction valueMapper, + @Nullable List attributes, + @Nullable String attributesMap, + Schema outputSchema) { this.pubsubErrorCounter = Metrics.counter(PubsubReadSchemaTransformProvider.class, name); this.valueMapper = valueMapper; + this.attributes = attributes; + this.attributesMap = attributesMap; + this.outputSchema = outputSchema; } @ProcessElement public void process(@DoFn.Element PubsubMessage message, MultiOutputReceiver receiver) { try { - receiver.get(OUTPUT_TAG).output(valueMapper.apply(message.getPayload())); + Row payloadRow = valueMapper.apply(message.getPayload()); + Row outputRow; + if (attributes == null && attributesMap == null) { + outputRow = payloadRow; + } else { + Row.Builder rowBuilder = Row.withSchema(outputSchema); + // @SuppressWarnings("nullness") + List<@Nullable Object> payloadValues = payloadRow.getValues(); + if (payloadValues != null) { + rowBuilder.addValues(payloadValues); + } + if (attributes != null) { + for (String attribute : attributes) { + System.out.println( + "attribute " + attribute + " " + message.getAttribute(attribute)); + rowBuilder.addValue(message.getAttribute(attribute)); + } + } + if (attributesMap != null) { + rowBuilder.addValue(message.getAttributeMap()); + } + outputRow = rowBuilder.build(); + } + receiver.get(OUTPUT_TAG).output(outputRow); } catch (Exception e) { errorsInBundle += 1; receiver @@ -174,6 +227,7 @@ public void process(@DoFn.Element PubsubMessage message, MultiOutputReceiver rec Row.withSchema(ERROR_SCHEMA) .addValues(e.toString(), message.getPayload()) .build()); + throw new RuntimeException(e); } } @@ -194,7 +248,10 @@ void setClock(@Nullable Clock clock) { @SuppressWarnings("nullness") PubsubIO.Read buildPubsubRead() { - PubsubIO.Read pubsubRead = PubsubIO.readMessages(); + PubsubIO.Read pubsubRead = + (configuration.getAttributes() == null && configuration.getAttributesMap() == null) + ? PubsubIO.readMessages() + : PubsubIO.readMessagesWithAttributes(); if (!Strings.isNullOrEmpty(configuration.getTopic())) { pubsubRead = pubsubRead.fromTopic(configuration.getTopic()); } else { @@ -225,7 +282,13 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) { .getPipeline() .apply(pubsubRead) .apply( - ParDo.of(new ErrorCounterFn("PubSub-read-error-counter", valueMapper)) + ParDo.of( + new ErrorCounterFn( + "PubSub-read-error-counter", + valueMapper, + configuration.getAttributes(), + configuration.getAttributesMap(), + beamSchema)) .withOutputTags(OUTPUT_TAG, TupleTagList.of(ERROR_TAG))); return PCollectionRowTuple.of( diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubWriteSchemaTransformConfiguration.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubWriteSchemaTransformConfiguration.java index cd49ca6ca07b..987ec20b6816 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubWriteSchemaTransformConfiguration.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubWriteSchemaTransformConfiguration.java @@ -18,6 +18,7 @@ package org.apache.beam.sdk.io.gcp.pubsub; import com.google.auto.value.AutoValue; +import java.util.List; import javax.annotation.Nullable; import org.apache.beam.sdk.schemas.AutoValueSchema; import org.apache.beam.sdk.schemas.annotations.DefaultSchema; @@ -42,6 +43,14 @@ public abstract class PubsubWriteSchemaTransformConfiguration { "The name of the topic to write data to. " + "Format: projects/${PROJECT}/topics/${TOPIC}") public abstract String getTopic(); + @SchemaFieldDescription( + "The set of fields to write as PubSub attributes instead of part of the payload.") + public abstract @Nullable List getAttributes(); + + @SchemaFieldDescription( + "A map field to write as PubSub attributes instead of part of the payload.") + public abstract @Nullable String getAttributesMap(); + @SchemaFieldDescription( "If set, will set an attribute for each Cloud Pub/Sub message with the given name and a unique value. " + "This attribute can then be used in a ReadFromPubSub PTransform to deduplicate messages.") @@ -62,9 +71,13 @@ public abstract static class Builder { public abstract Builder setTopic(String topic); - public abstract Builder setIdAttribute(String idAttribute); + public abstract Builder setAttributes(@Nullable List attributes); + + public abstract Builder setAttributesMap(@Nullable String attributesMap); + + public abstract Builder setIdAttribute(@Nullable String idAttribute); - public abstract Builder setTimestampAttribute(String timestampAttribute); + public abstract Builder setTimestampAttribute(@Nullable String timestampAttribute); public abstract PubsubWriteSchemaTransformConfiguration build(); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubWriteSchemaTransformProvider.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubWriteSchemaTransformProvider.java index e720a0331a07..73953205df38 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubWriteSchemaTransformProvider.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubWriteSchemaTransformProvider.java @@ -18,12 +18,16 @@ package org.apache.beam.sdk.io.gcp.pubsub; import com.google.auto.service.AutoService; +import com.google.common.collect.ImmutableSet; import java.io.Serializable; import java.nio.charset.StandardCharsets; import java.util.Collections; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.Set; +import javax.annotation.Nullable; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.transforms.SchemaTransform; import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider; @@ -69,18 +73,54 @@ public Class configurationClass() { } public static class ErrorFn extends DoFn { - private SerializableFunction valueMapper; - private Schema errorSchema; - - ErrorFn(SerializableFunction valueMapper, Schema errorSchema) { + private final SerializableFunction valueMapper; + private final @Nullable Set attributes; + private final @Nullable String attributesMap; + private final Schema payloadSchema; + private final Schema errorSchema; + + ErrorFn( + SerializableFunction valueMapper, + @Nullable List attributes, + @Nullable String attributesMap, + Schema payloadSchema, + Schema errorSchema) { this.valueMapper = valueMapper; + this.attributes = attributes == null ? null : ImmutableSet.copyOf(attributes); + this.attributesMap = attributesMap; + this.payloadSchema = payloadSchema; this.errorSchema = errorSchema; } @ProcessElement public void processElement(@Element Row row, MultiOutputReceiver receiver) { try { - receiver.get(OUTPUT_TAG).output(new PubsubMessage(valueMapper.apply(row), null)); + Row payloadRow; + Map messageAttributes = null; + if (attributes == null && attributesMap == null) { + payloadRow = row; + } else { + Row.Builder payloadRowBuilder = Row.withSchema(payloadSchema); + messageAttributes = new HashMap<>(); + List fields = row.getSchema().getFields(); + for (int ix = 0; ix < fields.size(); ix++) { + String name = fields.get(ix).getName(); + if (attributes != null && attributes.contains(name)) { + messageAttributes.put(name, row.getValue(ix)); + } else if (name.equals(attributesMap)) { + Map attrs = row.getMap(ix); + if (attrs != null) { + messageAttributes.putAll(attrs); + } + } else { + payloadRowBuilder.addValue(row.getValue(ix)); + } + } + payloadRow = payloadRowBuilder.build(); + } + receiver + .get(OUTPUT_TAG) + .output(new PubsubMessage(valueMapper.apply(payloadRow), messageAttributes)); } catch (Exception e) { receiver .get(ERROR_TAG) @@ -108,6 +148,9 @@ private static class PubsubWriteSchemaTransform extends SchemaTransform implemen } @Override + @SuppressWarnings({ + "nullness" // TODO(https://github.com/apache/beam/issues/20497) + }) public PCollectionRowTuple expand(PCollectionRowTuple input) { final Schema errorSchema = Schema.builder() @@ -117,27 +160,40 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) { String format = configuration.getFormat(); Schema beamSchema = input.get("input").getSchema(); + Schema payloadSchema; + if (configuration.getAttributes() == null && configuration.getAttributesMap() == null) { + payloadSchema = beamSchema; + } else { + Schema.Builder payloadSchemaBuilder = Schema.builder(); + for (Schema.Field f : beamSchema.getFields()) { + if (!configuration.getAttributes().contains(f.getName()) + && !f.getName().equals(configuration.getAttributesMap())) { + payloadSchemaBuilder.addField(f); + } + } + payloadSchema = payloadSchemaBuilder.build(); + } SerializableFunction fn; if (Objects.equals(format, "RAW")) { - if (beamSchema.getFieldCount() != 1) { + if (payloadSchema.getFieldCount() != 1) { throw new IllegalArgumentException( String.format( - "Raw output only supported for single-field schemas, got %s", beamSchema)); + "Raw output only supported for single-field schemas, got %s", payloadSchema)); } - if (beamSchema.getField(0).getType().equals(Schema.FieldType.BYTES)) { + if (payloadSchema.getField(0).getType().equals(Schema.FieldType.BYTES)) { fn = row -> row.getBytes(0); - } else if (beamSchema.getField(0).getType().equals(Schema.FieldType.STRING)) { + } else if (payloadSchema.getField(0).getType().equals(Schema.FieldType.STRING)) { fn = row -> row.getString(0).getBytes(StandardCharsets.UTF_8); } else { throw new IllegalArgumentException( String.format( "Raw output only supports bytes and string fields, got %s", - beamSchema.getField(0))); + payloadSchema.getField(0))); } } else if (Objects.equals(format, "JSON")) { - fn = JsonUtils.getRowToJsonBytesFunction(beamSchema); + fn = JsonUtils.getRowToJsonBytesFunction(payloadSchema); } else if (Objects.equals(format, "AVRO")) { - fn = AvroUtils.getRowToAvroBytesFunction(beamSchema); + fn = AvroUtils.getRowToAvroBytesFunction(payloadSchema); } else { throw new IllegalArgumentException( String.format( @@ -149,7 +205,13 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) { input .get("input") .apply( - ParDo.of(new ErrorFn(fn, errorSchema)) + ParDo.of( + new ErrorFn( + fn, + configuration.getAttributes(), + configuration.getAttributesMap(), + payloadSchema, + errorSchema)) .withOutputTags(OUTPUT_TAG, TupleTagList.of(ERROR_TAG))); PubsubIO.Write writeTransform = diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformProviderTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformProviderTest.java index c2b32c8c9ba8..833bbdf43797 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformProviderTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformProviderTest.java @@ -28,6 +28,7 @@ import java.util.Arrays; import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.UUID; import java.util.stream.Collectors; import org.apache.beam.sdk.PipelineResult; @@ -48,6 +49,7 @@ import org.apache.beam.sdk.values.Row; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.junit.Rule; import org.junit.Test; import org.junit.runner.RunWith; @@ -193,6 +195,58 @@ public void testReadRaw() throws IOException { } } + @Test + public void testReadAttributes() throws IOException { + PCollectionRowTuple begin = PCollectionRowTuple.empty(p); + + Schema.builder() + .addByteArrayField("payload") + .addStringField("attr") + .addMapField("attrMap", Schema.FieldType.STRING, Schema.FieldType.STRING) + .build(); + + Schema rawSchema = + Schema.builder() + .addByteArrayField("payload") + .addStringField("attr") + .addMapField("attrMap", Schema.FieldType.STRING, Schema.FieldType.STRING) + .build(); + byte[] payload = "some payload".getBytes(Charsets.UTF_8); + String attr = "attr value"; + + try (PubsubTestClientFactory clientFactory = + clientFactory( + ImmutableList.of( + incomingMessageOf( + payload, CLOCK.currentTimeMillis(), ImmutableMap.of("attr", attr))))) { + PubsubReadSchemaTransformConfiguration config = + PubsubReadSchemaTransformConfiguration.builder() + .setFormat("RAW") + .setSchema("") + .setSubscription(SUBSCRIPTION) + .setAttributes(ImmutableList.of("attr")) + .setAttributesMap("attrMap") + .setClientFactory(clientFactory) + .setClock(CLOCK) + .build(); + SchemaTransform transform = new PubsubReadSchemaTransformProvider().from(config); + PCollectionRowTuple reads = begin.apply(transform); + + PAssert.that(reads.get("output")) + .containsInAnyOrder( + ImmutableList.of( + Row.withSchema(rawSchema) + .addValue(payload) + .addValue(attr) + .addValue(ImmutableMap.of("attr", attr)) + .build())); + + p.run().waitUntilFinish(); + } catch (Exception e) { + throw e; + } + } + @Test public void testReadAvro() throws IOException { PCollectionRowTuple begin = PCollectionRowTuple.empty(p); @@ -285,12 +339,18 @@ private static List beamRowToMessageWithError() { private static PubsubClient.IncomingMessage incomingMessageOf( byte[] bytes, long millisSinceEpoch) { + return incomingMessageOf(bytes, millisSinceEpoch, ImmutableMap.of()); + } + + private static PubsubClient.IncomingMessage incomingMessageOf( + byte[] bytes, long millisSinceEpoch, Map attributes) { int nanos = Long.valueOf(millisSinceEpoch).intValue() * 1000; Timestamp timestamp = Timestamp.newBuilder().setNanos(nanos).build(); return PubsubClient.IncomingMessage.of( com.google.pubsub.v1.PubsubMessage.newBuilder() .setData(ByteString.copyFrom(bytes)) .setPublishTime(timestamp) + .putAllAttributes(attributes) .build(), millisSinceEpoch, 0, @@ -300,6 +360,7 @@ private static PubsubClient.IncomingMessage incomingMessageOf( private static PubsubTestClient.PubsubTestClientFactory clientFactory( List messages) { + System.out.println("messages " + messages); return PubsubTestClient.createFactoryForPull( CLOCK, PubsubClient.subscriptionPathFromPath(SUBSCRIPTION), 60, messages); } From 868f06da01d115a9fd592bf34a89a13f2a648b6b Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Mon, 2 Oct 2023 17:08:15 -0700 Subject: [PATCH 007/435] Add configurable error handling. --- ...ubsubReadSchemaTransformConfiguration.java | 23 +++++++++ .../PubsubReadSchemaTransformProvider.java | 48 ++++++++++++------- ...bsubWriteSchemaTransformConfiguration.java | 23 +++++++++ .../PubsubWriteSchemaTransformProvider.java | 34 ++++++++++--- ...PubsubReadSchemaTransformProviderTest.java | 4 ++ 5 files changed, 109 insertions(+), 23 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformConfiguration.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformConfiguration.java index cd957e4feedd..6e665baaf6b1 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformConfiguration.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformConfiguration.java @@ -89,12 +89,33 @@ public abstract class PubsubReadSchemaTransformConfiguration { + "(i.e., time units smaller than milliseconds) will be ignored.") public abstract @Nullable String getTimestampAttribute(); + @SchemaFieldDescription("Specifies how to handle errors.") + public abstract @Nullable ErrorHandling getErrorHandling(); + // Used for testing only. public abstract @Nullable PubsubTestClientFactory getClientFactory(); // Used for testing only. public abstract @Nullable Clock getClock(); + @AutoValue + public abstract static class ErrorHandling { + @SchemaFieldDescription("The name of the output PCollection containing failed reads.") + public abstract String getOutput(); + + public static PubsubReadSchemaTransformConfiguration.ErrorHandling.Builder builder() { + return new AutoValue_PubsubReadSchemaTransformConfiguration_ErrorHandling.Builder(); + } + + @AutoValue.Builder + public abstract static class Builder { + public abstract PubsubReadSchemaTransformConfiguration.ErrorHandling.Builder setOutput( + String output); + + public abstract PubsubReadSchemaTransformConfiguration.ErrorHandling build(); + } + } + public static Builder builder() { return new AutoValue_PubsubReadSchemaTransformConfiguration.Builder(); } @@ -117,6 +138,8 @@ public abstract static class Builder { public abstract Builder setTimestampAttribute(@Nullable String schema); + public abstract Builder setErrorHandling(@Nullable ErrorHandling errorHandling); + // Used for testing only. public abstract Builder setClientFactory(@Nullable PubsubTestClientFactory clientFactory); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformProvider.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformProvider.java index b5e92d84910c..9efc34d8b502 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformProvider.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformProvider.java @@ -178,21 +178,26 @@ private static class ErrorCounterFn extends DoFn { private final @Nullable String attributesMap; private final Schema outputSchema; + final boolean useErrorOutput; + ErrorCounterFn( String name, SerializableFunction valueMapper, @Nullable List attributes, @Nullable String attributesMap, - Schema outputSchema) { + Schema outputSchema, + boolean useErrorOutput) { this.pubsubErrorCounter = Metrics.counter(PubsubReadSchemaTransformProvider.class, name); this.valueMapper = valueMapper; this.attributes = attributes; this.attributesMap = attributesMap; this.outputSchema = outputSchema; + this.useErrorOutput = useErrorOutput; } @ProcessElement - public void process(@DoFn.Element PubsubMessage message, MultiOutputReceiver receiver) { + public void process(@DoFn.Element PubsubMessage message, MultiOutputReceiver receiver) + throws Exception { try { Row payloadRow = valueMapper.apply(message.getPayload()); @@ -201,7 +206,6 @@ public void process(@DoFn.Element PubsubMessage message, MultiOutputReceiver rec outputRow = payloadRow; } else { Row.Builder rowBuilder = Row.withSchema(outputSchema); - // @SuppressWarnings("nullness") List<@Nullable Object> payloadValues = payloadRow.getValues(); if (payloadValues != null) { rowBuilder.addValues(payloadValues); @@ -221,13 +225,16 @@ public void process(@DoFn.Element PubsubMessage message, MultiOutputReceiver rec receiver.get(OUTPUT_TAG).output(outputRow); } catch (Exception e) { errorsInBundle += 1; - receiver - .get(ERROR_TAG) - .output( - Row.withSchema(ERROR_SCHEMA) - .addValues(e.toString(), message.getPayload()) - .build()); - throw new RuntimeException(e); + if (useErrorOutput) { + receiver + .get(ERROR_TAG) + .output( + Row.withSchema(ERROR_SCHEMA) + .addValues(e.toString(), message.getPayload()) + .build()); + } else { + throw e; + } } } @@ -276,6 +283,11 @@ PubsubIO.Read buildPubsubRead() { @Override public PCollectionRowTuple expand(PCollectionRowTuple input) { PubsubIO.Read pubsubRead = buildPubsubRead(); + @SuppressWarnings("nullness") + String errorOutput = + configuration.getErrorHandling() == null + ? null + : configuration.getErrorHandling().getOutput(); PCollectionTuple outputTuple = input @@ -288,14 +300,18 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) { valueMapper, configuration.getAttributes(), configuration.getAttributesMap(), - beamSchema)) + beamSchema, + errorOutput != null)) .withOutputTags(OUTPUT_TAG, TupleTagList.of(ERROR_TAG))); + outputTuple.get(OUTPUT_TAG).setRowSchema(beamSchema); + outputTuple.get(ERROR_TAG).setRowSchema(ERROR_SCHEMA); - return PCollectionRowTuple.of( - "output", - outputTuple.get(OUTPUT_TAG).setRowSchema(beamSchema), - "errors", - outputTuple.get(ERROR_TAG).setRowSchema(ERROR_SCHEMA)); + PCollectionRowTuple result = PCollectionRowTuple.of("output", outputTuple.get(OUTPUT_TAG)); + if (errorOutput == null) { + return result; + } else { + return result.and(errorOutput, outputTuple.get(ERROR_TAG)); + } } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubWriteSchemaTransformConfiguration.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubWriteSchemaTransformConfiguration.java index 987ec20b6816..f962e7185f1b 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubWriteSchemaTransformConfiguration.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubWriteSchemaTransformConfiguration.java @@ -61,6 +61,27 @@ public abstract class PubsubWriteSchemaTransformConfiguration { + "publish time as the value.") public abstract @Nullable String getTimestampAttribute(); + @SchemaFieldDescription("Specifies how to handle errors.") + public abstract @Nullable ErrorHandling getErrorHandling(); + + @AutoValue + public abstract static class ErrorHandling { + @SchemaFieldDescription("The name of the output PCollection containing failed writes.") + public abstract String getOutput(); + + public static PubsubWriteSchemaTransformConfiguration.ErrorHandling.Builder builder() { + return new AutoValue_PubsubWriteSchemaTransformConfiguration_ErrorHandling.Builder(); + } + + @AutoValue.Builder + public abstract static class Builder { + public abstract PubsubWriteSchemaTransformConfiguration.ErrorHandling.Builder setOutput( + String output); + + public abstract PubsubWriteSchemaTransformConfiguration.ErrorHandling build(); + } + } + public static Builder builder() { return new AutoValue_PubsubWriteSchemaTransformConfiguration.Builder(); } @@ -79,6 +100,8 @@ public abstract static class Builder { public abstract Builder setTimestampAttribute(@Nullable String timestampAttribute); + public abstract Builder setErrorHandling(@Nullable ErrorHandling errorHandling); + public abstract PubsubWriteSchemaTransformConfiguration build(); } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubWriteSchemaTransformProvider.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubWriteSchemaTransformProvider.java index 73953205df38..3f53a0701829 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubWriteSchemaTransformProvider.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubWriteSchemaTransformProvider.java @@ -78,22 +78,25 @@ public static class ErrorFn extends DoFn { private final @Nullable String attributesMap; private final Schema payloadSchema; private final Schema errorSchema; + private final boolean useErrorOutput; ErrorFn( SerializableFunction valueMapper, @Nullable List attributes, @Nullable String attributesMap, Schema payloadSchema, - Schema errorSchema) { + Schema errorSchema, + boolean useErrorOutput) { this.valueMapper = valueMapper; this.attributes = attributes == null ? null : ImmutableSet.copyOf(attributes); this.attributesMap = attributesMap; this.payloadSchema = payloadSchema; this.errorSchema = errorSchema; + this.useErrorOutput = useErrorOutput; } @ProcessElement - public void processElement(@Element Row row, MultiOutputReceiver receiver) { + public void processElement(@Element Row row, MultiOutputReceiver receiver) throws Exception { try { Row payloadRow; Map messageAttributes = null; @@ -122,9 +125,13 @@ public void processElement(@Element Row row, MultiOutputReceiver receiver) { .get(OUTPUT_TAG) .output(new PubsubMessage(valueMapper.apply(payloadRow), messageAttributes)); } catch (Exception e) { - receiver - .get(ERROR_TAG) - .output(Row.withSchema(errorSchema).addValues(e.toString(), row).build()); + if (useErrorOutput) { + receiver + .get(ERROR_TAG) + .output(Row.withSchema(errorSchema).addValues(e.toString(), row).build()); + } else { + throw e; + } } } } @@ -152,6 +159,12 @@ private static class PubsubWriteSchemaTransform extends SchemaTransform implemen "nullness" // TODO(https://github.com/apache/beam/issues/20497) }) public PCollectionRowTuple expand(PCollectionRowTuple input) { + @SuppressWarnings("nullness") + String errorOutput = + configuration.getErrorHandling() == null + ? null + : configuration.getErrorHandling().getOutput(); + final Schema errorSchema = Schema.builder() .addStringField("error") @@ -211,7 +224,8 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) { configuration.getAttributes(), configuration.getAttributesMap(), payloadSchema, - errorSchema)) + errorSchema, + errorOutput != null)) .withOutputTags(OUTPUT_TAG, TupleTagList.of(ERROR_TAG))); PubsubIO.Write writeTransform = @@ -223,8 +237,14 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) { writeTransform = writeTransform.withIdAttribute(configuration.getTimestampAttribute()); } outputTuple.get(OUTPUT_TAG).apply(writeTransform); + outputTuple.get(ERROR_TAG).setRowSchema(errorSchema); - return PCollectionRowTuple.of("errors", outputTuple.get(ERROR_TAG).setRowSchema(errorSchema)); + if (errorOutput == null) { + return PCollectionRowTuple.empty(input.getPipeline()); + } else { + return PCollectionRowTuple.of( + errorOutput, outputTuple.get(ERROR_TAG).setRowSchema(errorSchema)); + } } } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformProviderTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformProviderTest.java index 833bbdf43797..826d5b462154 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformProviderTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformProviderTest.java @@ -281,6 +281,10 @@ public void testReadAvroWithError() throws IOException { .setFormat("AVRO") .setSchema(SCHEMA) .setSubscription(SUBSCRIPTION) + .setErrorHandling( + PubsubReadSchemaTransformConfiguration.ErrorHandling.builder() + .setOutput("errors") + .build()) .setClientFactory(clientFactory) .setClock(CLOCK) .build(); From 3655b1ea3fb33c20b758b6a70a76c699ed767f89 Mon Sep 17 00:00:00 2001 From: Ahmet Altay Date: Mon, 2 Oct 2023 21:02:20 -0700 Subject: [PATCH 008/435] Update dyi-content-discovery-platform-genai-beam.md Fixing the publication date. --- .../en/blog/dyi-content-discovery-platform-genai-beam.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/website/www/site/content/en/blog/dyi-content-discovery-platform-genai-beam.md b/website/www/site/content/en/blog/dyi-content-discovery-platform-genai-beam.md index 8057374591d7..fd967e318a07 100644 --- a/website/www/site/content/en/blog/dyi-content-discovery-platform-genai-beam.md +++ b/website/www/site/content/en/blog/dyi-content-discovery-platform-genai-beam.md @@ -1,7 +1,7 @@ --- layout: post title: "DIY GenAI Content Discovery Platform with Apache Beam" -date: 2023-09-27 00:00:01 -0800 +date: 2023-10-02 00:00:01 -0800 categories: - blog authors: From c7d7896533ba50e3bf2b64d6e0eb18ea7313a13f Mon Sep 17 00:00:00 2001 From: magicgoody <131876064+magicgoody@users.noreply.github.com> Date: Tue, 3 Oct 2023 21:15:36 +0600 Subject: [PATCH 009/435] Github Workflow Replacement for Jenkins Jobs, beam_LoadTests_Java_GBK_Dataflow_Batch_* (#28738) * beam_LoadTests_Java_GBK_Dataflow_Batch * argument fix * description added * fixed comment and filenames * config rename * config name fix --- ...beam_LoadTests_Java_GBK_Dataflow_Batch.yml | 141 +++++++++++++++ ..._LoadTests_Java_GBK_Dataflow_Streaming.yml | 141 +++++++++++++++ ...ests_Java_GBK_Dataflow_V2_Batch_Java11.yml | 166 +++++++++++++++++ ...ests_Java_GBK_Dataflow_V2_Batch_Java17.yml | 168 ++++++++++++++++++ ..._Java_GBK_Dataflow_V2_Streaming_Java11.yml | 166 +++++++++++++++++ ..._Java_GBK_Dataflow_V2_Streaming_Java17.yml | 168 ++++++++++++++++++ ...GBK_Dataflow_Batch_2GB_of_100B_records.txt | 28 +++ ...BK_Dataflow_Batch_2GB_of_100kB_records.txt | 28 +++ ..._GBK_Dataflow_Batch_2GB_of_10B_records.txt | 28 +++ ...4_times_with_2GB_10-byte_records_total.txt | 28 +++ ...8_times_with_2GB_10-byte_records_total.txt | 28 +++ ...ow_Batch_reiterate_4_times_10kB_values.txt | 28 +++ ...low_Batch_reiterate_4_times_2MB_values.txt | 28 +++ ...Dataflow_Streaming_2GB_of_100B_records.txt | 29 +++ ...ataflow_Streaming_2GB_of_100kB_records.txt | 29 +++ ..._Dataflow_Streaming_2GB_of_10B_records.txt | 29 +++ ...4_times_with_2GB_10-byte_records_total.txt | 29 +++ ...8_times_with_2GB_10-byte_records_total.txt | 29 +++ ...treaming_reiterate_4_times_10kB_values.txt | 29 +++ ...Streaming_reiterate_4_times_2MB_values.txt | 29 +++ ...ow_V2_Batch_Java11_2GB_of_100B_records.txt | 29 +++ ...w_V2_Batch_Java11_2GB_of_100kB_records.txt | 29 +++ ...low_V2_Batch_Java11_2GB_of_10B_records.txt | 29 +++ ...4_times_with_2GB_10-byte_records_total.txt | 29 +++ ...8_times_with_2GB_10-byte_records_total.txt | 29 +++ ...h_Java11_reiterate_4_times_10kB_values.txt | 29 +++ ...ch_Java11_reiterate_4_times_2MB_values.txt | 29 +++ ...ow_V2_Batch_Java17_2GB_of_100B_records.txt | 29 +++ ...w_V2_Batch_Java17_2GB_of_100kB_records.txt | 29 +++ ...low_V2_Batch_Java17_2GB_of_10B_records.txt | 29 +++ ...4_times_with_2GB_10-byte_records_total.txt | 29 +++ ...8_times_with_2GB_10-byte_records_total.txt | 29 +++ ...h_Java17_reiterate_4_times_10kB_values.txt | 29 +++ ...ch_Java17_reiterate_4_times_2MB_values.txt | 29 +++ ...2_Streaming_Java11_2GB_of_100B_records.txt | 30 ++++ ..._Streaming_Java11_2GB_of_100kB_records.txt | 30 ++++ ...V2_Streaming_Java11_2GB_of_10B_records.txt | 30 ++++ ...4_times_with_2GB_10-byte_records_total.txt | 30 ++++ ...8_times_with_2GB_10-byte_records_total.txt | 30 ++++ ...g_Java11_reiterate_4_times_10kB_values.txt | 30 ++++ ...ng_Java11_reiterate_4_times_2MB_values.txt | 30 ++++ ...2_Streaming_Java17_2GB_of_100B_records.txt | 30 ++++ ..._Streaming_Java17_2GB_of_100kB_records.txt | 30 ++++ ...V2_Streaming_Java17_2GB_of_10B_records.txt | 30 ++++ ...4_times_with_2GB_10-byte_records_total.txt | 30 ++++ ...8_times_with_2GB_10-byte_records_total.txt | 30 ++++ ...g_Java17_reiterate_4_times_10kB_values.txt | 30 ++++ ...ng_Java17_reiterate_4_times_2MB_values.txt | 30 ++++ 48 files changed, 2175 insertions(+) create mode 100644 .github/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml create mode 100644 .github/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml create mode 100644 .github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml create mode 100644 .github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml create mode 100644 .github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml create mode 100644 .github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_2GB_of_100B_records.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_2GB_of_100kB_records.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_2GB_of_10B_records.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_fanout_4_times_with_2GB_10-byte_records_total.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_fanout_8_times_with_2GB_10-byte_records_total.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_reiterate_4_times_10kB_values.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_reiterate_4_times_2MB_values.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_2GB_of_100B_records.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_2GB_of_100kB_records.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_2GB_of_10B_records.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_fanout_4_times_with_2GB_10-byte_records_total.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_fanout_8_times_with_2GB_10-byte_records_total.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_reiterate_4_times_10kB_values.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_reiterate_4_times_2MB_values.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_2GB_of_100B_records.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_2GB_of_100kB_records.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_2GB_of_10B_records.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_fanout_4_times_with_2GB_10-byte_records_total.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_fanout_8_times_with_2GB_10-byte_records_total.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_reiterate_4_times_10kB_values.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_reiterate_4_times_2MB_values.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_2GB_of_100B_records.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_2GB_of_100kB_records.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_2GB_of_10B_records.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_fanout_4_times_with_2GB_10-byte_records_total.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_fanout_8_times_with_2GB_10-byte_records_total.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_reiterate_4_times_10kB_values.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_reiterate_4_times_2MB_values.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_2GB_of_100B_records.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_2GB_of_100kB_records.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_2GB_of_10B_records.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_fanout_4_times_with_2GB_10-byte_records_total.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_fanout_8_times_with_2GB_10-byte_records_total.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_reiterate_4_times_10kB_values.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_reiterate_4_times_2MB_values.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_2GB_of_100B_records.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_2GB_of_100kB_records.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_2GB_of_10B_records.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_fanout_4_times_with_2GB_10-byte_records_total.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_fanout_8_times_with_2GB_10-byte_records_total.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_reiterate_4_times_10kB_values.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_reiterate_4_times_2MB_values.txt diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml new file mode 100644 index 000000000000..b490ce69323e --- /dev/null +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml @@ -0,0 +1,141 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: LoadTests Java GBK Dataflow Batch + +on: + issue_comment: + types: [created] + schedule: + - cron: '10 6 * * *' + workflow_dispatch: + +# Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: write + checks: write + contents: read + deployments: read + id-token: none + issues: write + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_LoadTests_Java_GBK_Dataflow_Batch: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Load Tests Java GBK Dataflow Batch' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 240 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_LoadTests_Java_GBK_Dataflow_Batch"] + job_phrase: ["Run Load Tests Java GBK Dataflow Batch"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: java + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_2GB_of_10B_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_2GB_of_100B_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_2GB_of_100kB_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_fanout_4_times_with_2GB_10-byte_records_total.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_fanout_8_times_with_2GB_10-byte_records_total.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_reiterate_4_times_10kB_values.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_reiterate_4_times_2MB_values.txt + # The env variables are created and populated in the test-arguments-action as "_test_arguments_" + - name: run Load test 2GB of 10B records + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_Batch_test_arguments_1 }}' \ + - name: run Load test 2GB of 100B records + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_Batch_test_arguments_2 }}' \ + - name: run Load test 2GB of 100kB records + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_Batch_test_arguments_3 }}' \ + - name: run Load test fanout 4 times with 2GB 10-byte records total + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_Batch_test_arguments_4 }}' \ + - name: run Load test fanout 8 times with 2GB 10-byte records total + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_Batch_test_arguments_5 }}' \ + - name: run Load test reiterate 4 times 10kB values + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_Batch_test_arguments_6 }}' \ + - name: run Load test reiterate 4 times 2MB values + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_Batch_test_arguments_7 }}' \ No newline at end of file diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml new file mode 100644 index 000000000000..19cc12f91eb1 --- /dev/null +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml @@ -0,0 +1,141 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: LoadTests Java GBK Dataflow Streaming + +on: + issue_comment: + types: [created] + schedule: + - cron: '50 6 * * *' + workflow_dispatch: + +# Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: write + checks: write + contents: read + deployments: read + id-token: none + issues: write + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_LoadTests_Java_GBK_Dataflow_Streaming: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Load Tests Java GBK Dataflow Streaming' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 240 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_LoadTests_Java_GBK_Dataflow_Streaming"] + job_phrase: ["Run Load Tests Java GBK Dataflow Streaming"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: java + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_2GB_of_10B_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_2GB_of_100B_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_2GB_of_100kB_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_fanout_4_times_with_2GB_10-byte_records_total.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_fanout_8_times_with_2GB_10-byte_records_total.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_reiterate_4_times_10kB_values.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_reiterate_4_times_2MB_values.txt + # The env variables are created and populated in the test-arguments-action as "_test_arguments_" + - name: run Load test 2GB of 10B records + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_Streaming_test_arguments_1 }}' \ + - name: run Load test 2GB of 100B records + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_Streaming_test_arguments_2 }}' \ + - name: run Load test 2GB of 100kB records + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_Streaming_test_arguments_3 }}' \ + - name: run Load test fanout 4 times with 2GB 10-byte records total + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_Streaming_test_arguments_4 }}' \ + - name: run Load test fanout 8 times with 2GB 10-byte records total + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_Streaming_test_arguments_5 }}' \ + - name: run Load test reiterate 4 times 10kB values + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_Streaming_test_arguments_6 }}' \ + - name: run Load test reiterate 4 times 2MB values + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_Streaming_test_arguments_7 }}' \ No newline at end of file diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml new file mode 100644 index 000000000000..aeb8006dfd53 --- /dev/null +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml @@ -0,0 +1,166 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: LoadTests Java GBK Dataflow V2 Batch Java11 + +on: + issue_comment: + types: [created] + schedule: + - cron: '10 7 * * *' + workflow_dispatch: + +# Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: write + checks: write + contents: read + deployments: read + id-token: none + issues: write + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Load Tests Java 11 GBK Dataflow V2 Batch' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 240 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11"] + job_phrase: ["Run Load Tests Java 11 GBK Dataflow V2 Batch"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action + with: + java-version: 11 + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: java + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_2GB_of_10B_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_2GB_of_100B_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_2GB_of_100kB_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_fanout_4_times_with_2GB_10-byte_records_total.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_fanout_8_times_with_2GB_10-byte_records_total.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_reiterate_4_times_10kB_values.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_reiterate_4_times_2MB_values.txt + # The env variables are created and populated in the test-arguments-action as "_test_arguments_" + - name: run Load test 2GB of 10B records + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + -Prunner.version=V2 \ + -PcompileAndRunTestsWithJava11 \ + -Pjava11Home=$JAVA_HOME_11_X64 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11_test_arguments_1 }}' \ + - name: run Load test 2GB of 100B records + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + -Prunner.version=V2 \ + -PcompileAndRunTestsWithJava11 \ + -Pjava11Home=$JAVA_HOME_11_X64 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11_test_arguments_2 }}' \ + - name: run Load test 2GB of 100kB records + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + -Prunner.version=V2 \ + -PcompileAndRunTestsWithJava11 \ + -Pjava11Home=$JAVA_HOME_11_X64 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11_test_arguments_3 }}' \ + - name: run Load test fanout 4 times with 2GB 10-byte records total + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + -Prunner.version=V2 \ + -PcompileAndRunTestsWithJava11 \ + -Pjava11Home=$JAVA_HOME_11_X64 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11_test_arguments_4 }}' \ + - name: run Load test fanout 8 times with 2GB 10-byte records total + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + -Prunner.version=V2 \ + -PcompileAndRunTestsWithJava11 \ + -Pjava11Home=$JAVA_HOME_11_X64 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11_test_arguments_5 }}' \ + - name: run Load test reiterate 4 times 10kB values + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + -Prunner.version=V2 \ + -PcompileAndRunTestsWithJava11 \ + -Pjava11Home=$JAVA_HOME_11_X64 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11_test_arguments_6 }}' \ + - name: run Load test reiterate 4 times 2MB values + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + -Prunner.version=V2 \ + -PcompileAndRunTestsWithJava11 \ + -Pjava11Home=$JAVA_HOME_11_X64 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11_test_arguments_7 }}' \ No newline at end of file diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml new file mode 100644 index 000000000000..23f9f34d9747 --- /dev/null +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml @@ -0,0 +1,168 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: LoadTests Java GBK Dataflow V2 Batch Java17 + +on: + issue_comment: + types: [created] + schedule: + - cron: '50 7 * * *' + workflow_dispatch: + +# Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: write + checks: write + contents: read + deployments: read + id-token: none + issues: write + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Load Tests Java 17 GBK Dataflow V2 Batch' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 240 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17"] + job_phrase: ["Run Load Tests Java 17 GBK Dataflow V2 Batch"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action + with: + java-version: | + 17 + 8 + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: java + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_2GB_of_10B_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_2GB_of_100B_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_2GB_of_100kB_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_fanout_4_times_with_2GB_10-byte_records_total.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_fanout_8_times_with_2GB_10-byte_records_total.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_reiterate_4_times_10kB_values.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_reiterate_4_times_2MB_values.txt + # The env variables are created and populated in the test-arguments-action as "_test_arguments_" + - name: run Load test 2GB of 10B records + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + -Prunner.version=V2 \ + -PcompileAndRunTestsWithJava17 \ + -Pjava17Home=$JAVA_HOME_17_X64 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17_test_arguments_1 }}' \ + - name: run Load test 2GB of 100B records + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + -Prunner.version=V2 \ + -PcompileAndRunTestsWithJava17 \ + -Pjava17Home=$JAVA_HOME_17_X64 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17_test_arguments_2 }}' \ + - name: run Load test 2GB of 100kB records + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + -Prunner.version=V2 \ + -PcompileAndRunTestsWithJava17 \ + -Pjava17Home=$JAVA_HOME_17_X64 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17_test_arguments_3 }}' \ + - name: run Load test fanout 4 times with 2GB 10-byte records total + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + -Prunner.version=V2 \ + -PcompileAndRunTestsWithJava17 \ + -Pjava17Home=$JAVA_HOME_17_X64 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17_test_arguments_4 }}' \ + - name: run Load test fanout 8 times with 2GB 10-byte records total + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + -Prunner.version=V2 \ + -PcompileAndRunTestsWithJava17 \ + -Pjava17Home=$JAVA_HOME_17_X64 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17_test_arguments_5 }}' \ + - name: run Load test reiterate 4 times 10kB values + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + -Prunner.version=V2 \ + -PcompileAndRunTestsWithJava17 \ + -Pjava17Home=$JAVA_HOME_17_X64 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17_test_arguments_6 }}' \ + - name: run Load test reiterate 4 times 2MB values + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + -Prunner.version=V2 \ + -PcompileAndRunTestsWithJava17 \ + -Pjava17Home=$JAVA_HOME_17_X64 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17_test_arguments_7 }}' \ No newline at end of file diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml new file mode 100644 index 000000000000..9a0b236e43e9 --- /dev/null +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml @@ -0,0 +1,166 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: LoadTests Java GBK Dataflow V2 Streaming Java11 + +on: + issue_comment: + types: [created] + schedule: + - cron: '50 8 * * *' + workflow_dispatch: + +# Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: write + checks: write + contents: read + deployments: read + id-token: none + issues: write + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Load Tests Java 11 GBK Dataflow V2 Streaming' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 240 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11"] + job_phrase: ["Run Load Tests Java 11 GBK Dataflow V2 Streaming"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action + with: + java-version: 11 + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: java + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_2GB_of_10B_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_2GB_of_100B_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_2GB_of_100kB_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_fanout_4_times_with_2GB_10-byte_records_total.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_fanout_8_times_with_2GB_10-byte_records_total.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_reiterate_4_times_10kB_values.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_reiterate_4_times_2MB_values.txt + # The env variables are created and populated in the test-arguments-action as "_test_arguments_" + - name: run Load test 2GB of 10B records + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + -Prunner.version=V2 \ + -PcompileAndRunTestsWithJava11 \ + -Pjava11Home=$JAVA_HOME_11_X64 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11_test_arguments_1 }}' \ + - name: run Load test 2GB of 100B records + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + -Prunner.version=V2 \ + -PcompileAndRunTestsWithJava11 \ + -Pjava11Home=$JAVA_HOME_11_X64 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11_test_arguments_2 }}' \ + - name: run Load test 2GB of 100kB records + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + -Prunner.version=V2 \ + -PcompileAndRunTestsWithJava11 \ + -Pjava11Home=$JAVA_HOME_11_X64 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11_test_arguments_3 }}' \ + - name: run Load test fanout 4 times with 2GB 10-byte records total + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + -Prunner.version=V2 \ + -PcompileAndRunTestsWithJava11 \ + -Pjava11Home=$JAVA_HOME_11_X64 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11_test_arguments_4 }}' \ + - name: run Load test fanout 8 times with 2GB 10-byte records total + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + -Prunner.version=V2 \ + -PcompileAndRunTestsWithJava11 \ + -Pjava11Home=$JAVA_HOME_11_X64 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11_test_arguments_5 }}' \ + - name: run Load test reiterate 4 times 10kB values + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + -Prunner.version=V2 \ + -PcompileAndRunTestsWithJava11 \ + -Pjava11Home=$JAVA_HOME_11_X64 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11_test_arguments_6 }}' \ + - name: run Load test reiterate 4 times 2MB values + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + -Prunner.version=V2 \ + -PcompileAndRunTestsWithJava11 \ + -Pjava11Home=$JAVA_HOME_11_X64 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11_test_arguments_7 }}' \ No newline at end of file diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml new file mode 100644 index 000000000000..3580aaca51bb --- /dev/null +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml @@ -0,0 +1,168 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: LoadTests Java GBK Dataflow V2 Streaming Java17 + +on: + issue_comment: + types: [created] + schedule: + - cron: '50 9 * * *' + workflow_dispatch: + +# Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: write + checks: write + contents: read + deployments: read + id-token: none + issues: write + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Load Tests Java 17 GBK Dataflow V2 Streaming' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 240 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17"] + job_phrase: ["Run Load Tests Java 17 GBK Dataflow V2 Streaming"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action + with: + java-version: | + 17 + 8 + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: java + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_2GB_of_10B_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_2GB_of_100B_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_2GB_of_100kB_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_fanout_4_times_with_2GB_10-byte_records_total.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_fanout_8_times_with_2GB_10-byte_records_total.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_reiterate_4_times_10kB_values.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_reiterate_4_times_2MB_values.txt + # The env variables are created and populated in the test-arguments-action as "_test_arguments_" + - name: run Load test 2GB of 10B records + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + -Prunner.version=V2 \ + -PcompileAndRunTestsWithJava17 \ + -Pjava17Home=$JAVA_HOME_17_X64 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17_test_arguments_1 }}' \ + - name: run Load test 2GB of 100B records + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + -Prunner.version=V2 \ + -PcompileAndRunTestsWithJava17 \ + -Pjava17Home=$JAVA_HOME_17_X64 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17_test_arguments_2 }}' \ + - name: run Load test 2GB of 100kB records + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + -Prunner.version=V2 \ + -PcompileAndRunTestsWithJava17 \ + -Pjava17Home=$JAVA_HOME_17_X64 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17_test_arguments_3 }}' \ + - name: run Load test fanout 4 times with 2GB 10-byte records total + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + -Prunner.version=V2 \ + -PcompileAndRunTestsWithJava17 \ + -Pjava17Home=$JAVA_HOME_17_X64 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17_test_arguments_4 }}' \ + - name: run Load test fanout 8 times with 2GB 10-byte records total + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + -Prunner.version=V2 \ + -PcompileAndRunTestsWithJava17 \ + -Pjava17Home=$JAVA_HOME_17_X64 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17_test_arguments_5 }}' \ + - name: run Load test reiterate 4 times 10kB values + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + -Prunner.version=V2 \ + -PcompileAndRunTestsWithJava17 \ + -Pjava17Home=$JAVA_HOME_17_X64 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17_test_arguments_6 }}' \ + - name: run Load test reiterate 4 times 2MB values + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + -Prunner.version=V2 \ + -PcompileAndRunTestsWithJava17 \ + -Pjava17Home=$JAVA_HOME_17_X64 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17_test_arguments_7 }}' \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_2GB_of_100B_records.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_2GB_of_100B_records.txt new file mode 100644 index 000000000000..29d0ded11fac --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_2GB_of_100B_records.txt @@ -0,0 +1,28 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java_Dataflow_batch_GBK_2 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_gbk_2 +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90} +--fanout=1 +--iterations=1 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=false +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_2GB_of_100kB_records.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_2GB_of_100kB_records.txt new file mode 100644 index 000000000000..920f0a60d198 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_2GB_of_100kB_records.txt @@ -0,0 +1,28 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java_Dataflow_batch_GBK_3 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_gbk_3 +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000,"keySizeBytes":10000,"valueSizeBytes":90000} +--fanout=1 +--iterations=1 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=false +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_2GB_of_10B_records.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_2GB_of_10B_records.txt new file mode 100644 index 000000000000..7ccacc6c791c --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_2GB_of_10B_records.txt @@ -0,0 +1,28 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java_Dataflow_batch_GBK_1 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_gbk_1 +--publishToInfluxDB=true +--sourceOptions={"numRecords":200000000,"keySizeBytes":1,"valueSizeBytes":9} +--fanout=1 +--iterations=1 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=false +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_fanout_4_times_with_2GB_10-byte_records_total.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_fanout_4_times_with_2GB_10-byte_records_total.txt new file mode 100644 index 000000000000..8c6f6f1c89a0 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_fanout_4_times_with_2GB_10-byte_records_total.txt @@ -0,0 +1,28 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java_Dataflow_batch_GBK_4 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_gbk_4 +--publishToInfluxDB=true +--sourceOptions={"numRecords":5000000,"keySizeBytes":10,"valueSizeBytes":90} +--fanout=4 +--iterations=1 +--numWorkers=16 +--autoscalingAlgorithm=NONE +--streaming=false +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_fanout_8_times_with_2GB_10-byte_records_total.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_fanout_8_times_with_2GB_10-byte_records_total.txt new file mode 100644 index 000000000000..43f04dac1d02 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_fanout_8_times_with_2GB_10-byte_records_total.txt @@ -0,0 +1,28 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java_Dataflow_batch_GBK_5 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_gbk_5 +--publishToInfluxDB=true +--sourceOptions={"numRecords":2500000,"keySizeBytes":10,"valueSizeBytes":90} +--fanout=8 +--iterations=1 +--numWorkers=16 +--autoscalingAlgorithm=NONE +--streaming=false +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_reiterate_4_times_10kB_values.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_reiterate_4_times_10kB_values.txt new file mode 100644 index 000000000000..c8f9ecc34770 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_reiterate_4_times_10kB_values.txt @@ -0,0 +1,28 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java_Dataflow_batch_GBK_6 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_gbk_6 +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":200,"hotKeyFraction":1} +--fanout=1 +--iterations=4 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=false +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_reiterate_4_times_2MB_values.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_reiterate_4_times_2MB_values.txt new file mode 100644 index 000000000000..653004c6a04e --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_reiterate_4_times_2MB_values.txt @@ -0,0 +1,28 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java_Dataflow_batch_GBK_7 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_gbk_7 +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":10,"hotKeyFraction":1} +--fanout=1 +--iterations=4 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=false +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_2GB_of_100B_records.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_2GB_of_100B_records.txt new file mode 100644 index 000000000000..a6523c64edbc --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_2GB_of_100B_records.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java_Dataflow_streaming_GBK_2 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_streaming_gbk_2 +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90} +--fanout=1 +--iterations=1 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=true +--inputWindowDurationSec=1200 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_2GB_of_100kB_records.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_2GB_of_100kB_records.txt new file mode 100644 index 000000000000..422557e84a66 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_2GB_of_100kB_records.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java_Dataflow_streaming_GBK_3 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_streaming_gbk_3 +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000,"keySizeBytes":10000,"valueSizeBytes":90000} +--fanout=1 +--iterations=1 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=true +--inputWindowDurationSec=1200 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_2GB_of_10B_records.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_2GB_of_10B_records.txt new file mode 100644 index 000000000000..da6c04d0d284 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_2GB_of_10B_records.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java_Dataflow_streaming_GBK_1 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_streaming_gbk_1 +--publishToInfluxDB=true +--sourceOptions={"numRecords":200000000,"keySizeBytes":1,"valueSizeBytes":9} +--fanout=1 +--iterations=1 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=true +--inputWindowDurationSec=1200 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_fanout_4_times_with_2GB_10-byte_records_total.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_fanout_4_times_with_2GB_10-byte_records_total.txt new file mode 100644 index 000000000000..33181a939700 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_fanout_4_times_with_2GB_10-byte_records_total.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java_Dataflow_streaming_GBK_4 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_streaming_gbk_4 +--publishToInfluxDB=true +--sourceOptions={"numRecords":5000000,"keySizeBytes":10,"valueSizeBytes":90} +--fanout=4 +--iterations=1 +--numWorkers=16 +--autoscalingAlgorithm=NONE +--streaming=true +--inputWindowDurationSec=1200 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_fanout_8_times_with_2GB_10-byte_records_total.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_fanout_8_times_with_2GB_10-byte_records_total.txt new file mode 100644 index 000000000000..32b59df2a2e0 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_fanout_8_times_with_2GB_10-byte_records_total.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java_Dataflow_streaming_GBK_5 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_streaming_gbk_5 +--publishToInfluxDB=true +--sourceOptions={"numRecords":2500000,"keySizeBytes":10,"valueSizeBytes":90} +--fanout=8 +--iterations=1 +--numWorkers=16 +--autoscalingAlgorithm=NONE +--streaming=true +--inputWindowDurationSec=1200 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_reiterate_4_times_10kB_values.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_reiterate_4_times_10kB_values.txt new file mode 100644 index 000000000000..376d7e3244a8 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_reiterate_4_times_10kB_values.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java_Dataflow_streaming_GBK_6 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_streaming_gbk_6 +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":200,"hotKeyFraction":1} +--fanout=1 +--iterations=4 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=true +--inputWindowDurationSec=1200 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_reiterate_4_times_2MB_values.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_reiterate_4_times_2MB_values.txt new file mode 100644 index 000000000000..b7bf9db40e82 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_reiterate_4_times_2MB_values.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java_Dataflow_streaming_GBK_7 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_streaming_gbk_7 +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":10,"hotKeyFraction":1} +--fanout=1 +--iterations=4 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=true +--inputWindowDurationSec=1200 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_2GB_of_100B_records.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_2GB_of_100B_records.txt new file mode 100644 index 000000000000..d9b1918383b2 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_2GB_of_100B_records.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java11_Dataflow_V2_batch_GBK_2 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_gbk_2 +--influxTags={"runnerVersion":"v2","jdk":"java11"} +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90} +--fanout=1 +--iterations=1 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=false +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_2GB_of_100kB_records.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_2GB_of_100kB_records.txt new file mode 100644 index 000000000000..66911feb7d17 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_2GB_of_100kB_records.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java11_Dataflow_V2_batch_GBK_3 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_gbk_3 +--influxTags={"runnerVersion":"v2","jdk":"java11"} +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000,"keySizeBytes":10000,"valueSizeBytes":90000} +--fanout=1 +--iterations=1 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=false +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_2GB_of_10B_records.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_2GB_of_10B_records.txt new file mode 100644 index 000000000000..e265724ec1c1 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_2GB_of_10B_records.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java11_Dataflow_V2_batch_GBK_1 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_gbk_1 +--influxTags={"runnerVersion":"v2","jdk":"java11"} +--publishToInfluxDB=true +--sourceOptions={"numRecords":200000000,"keySizeBytes":1,"valueSizeBytes":9} +--fanout=1 +--iterations=1 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=false +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_fanout_4_times_with_2GB_10-byte_records_total.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_fanout_4_times_with_2GB_10-byte_records_total.txt new file mode 100644 index 000000000000..8610a9dff9f9 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_fanout_4_times_with_2GB_10-byte_records_total.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java11_Dataflow_V2_batch_GBK_4 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_gbk_4 +--influxTags={"runnerVersion":"v2","jdk":"java11"} +--publishToInfluxDB=true +--sourceOptions={"numRecords":5000000,"keySizeBytes":10,"valueSizeBytes":90} +--fanout=4 +--iterations=1 +--numWorkers=16 +--autoscalingAlgorithm=NONE +--streaming=false +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_fanout_8_times_with_2GB_10-byte_records_total.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_fanout_8_times_with_2GB_10-byte_records_total.txt new file mode 100644 index 000000000000..65aa56993294 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_fanout_8_times_with_2GB_10-byte_records_total.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java11_Dataflow_V2_batch_GBK_5 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_gbk_5 +--influxTags={"runnerVersion":"v2","jdk":"java11"} +--publishToInfluxDB=true +--sourceOptions={"numRecords":2500000,"keySizeBytes":10,"valueSizeBytes":90} +--fanout=8 +--iterations=1 +--numWorkers=16 +--autoscalingAlgorithm=NONE +--streaming=false +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_reiterate_4_times_10kB_values.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_reiterate_4_times_10kB_values.txt new file mode 100644 index 000000000000..aaf66ef03505 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_reiterate_4_times_10kB_values.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java11_Dataflow_V2_batch_GBK_6 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_gbk_6 +--influxTags={"runnerVersion":"v2","jdk":"java11"} +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":200,"hotKeyFraction":1} +--fanout=1 +--iterations=4 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=false +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_reiterate_4_times_2MB_values.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_reiterate_4_times_2MB_values.txt new file mode 100644 index 000000000000..8698ac90cac1 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_reiterate_4_times_2MB_values.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java11_Dataflow_V2_batch_GBK_7 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_gbk_7 +--influxTags={"runnerVersion":"v2","jdk":"java11"} +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":10,"hotKeyFraction":1} +--fanout=1 +--iterations=4 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=false +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_2GB_of_100B_records.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_2GB_of_100B_records.txt new file mode 100644 index 000000000000..f5a432b6d140 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_2GB_of_100B_records.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java17_Dataflow_V2_batch_GBK_2 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_gbk_2 +--influxTags={"runnerVersion":"v2","jdk":"java17"} +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90} +--fanout=1 +--iterations=1 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=false +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_2GB_of_100kB_records.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_2GB_of_100kB_records.txt new file mode 100644 index 000000000000..976bd20f7dff --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_2GB_of_100kB_records.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java17_Dataflow_V2_batch_GBK_3 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_gbk_3 +--influxTags={"runnerVersion":"v2","jdk":"java17"} +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000,"keySizeBytes":10000,"valueSizeBytes":90000} +--fanout=1 +--iterations=1 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=false +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_2GB_of_10B_records.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_2GB_of_10B_records.txt new file mode 100644 index 000000000000..89d356201a76 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_2GB_of_10B_records.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java17_Dataflow_V2_batch_GBK_1 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_gbk_1 +--influxTags={"runnerVersion":"v2","jdk":"java17"} +--publishToInfluxDB=true +--sourceOptions={"numRecords":200000000,"keySizeBytes":1,"valueSizeBytes":9} +--fanout=1 +--iterations=1 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=false +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_fanout_4_times_with_2GB_10-byte_records_total.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_fanout_4_times_with_2GB_10-byte_records_total.txt new file mode 100644 index 000000000000..0735c9bafc45 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_fanout_4_times_with_2GB_10-byte_records_total.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java17_Dataflow_V2_batch_GBK_4 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_gbk_4 +--influxTags={"runnerVersion":"v2","jdk":"java17"} +--publishToInfluxDB=true +--sourceOptions={"numRecords":5000000,"keySizeBytes":10,"valueSizeBytes":90} +--fanout=4 +--iterations=1 +--numWorkers=16 +--autoscalingAlgorithm=NONE +--streaming=false +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_fanout_8_times_with_2GB_10-byte_records_total.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_fanout_8_times_with_2GB_10-byte_records_total.txt new file mode 100644 index 000000000000..4ea7f402cc58 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_fanout_8_times_with_2GB_10-byte_records_total.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java17_Dataflow_V2_batch_GBK_5 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_gbk_5 +--influxTags={"runnerVersion":"v2","jdk":"java17"} +--publishToInfluxDB=true +--sourceOptions={"numRecords":2500000,"keySizeBytes":10,"valueSizeBytes":90} +--fanout=8 +--iterations=1 +--numWorkers=16 +--autoscalingAlgorithm=NONE +--streaming=false +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_reiterate_4_times_10kB_values.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_reiterate_4_times_10kB_values.txt new file mode 100644 index 000000000000..881640714df2 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_reiterate_4_times_10kB_values.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java17_Dataflow_V2_batch_GBK_6 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_gbk_6 +--influxTags={"runnerVersion":"v2","jdk":"java17"} +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":200,"hotKeyFraction":1} +--fanout=1 +--iterations=4 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=false +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_reiterate_4_times_2MB_values.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_reiterate_4_times_2MB_values.txt new file mode 100644 index 000000000000..a09c16d1e66a --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_reiterate_4_times_2MB_values.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java17_Dataflow_V2_batch_GBK_7 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_gbk_7 +--influxTags={"runnerVersion":"v2","jdk":"java17"} +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":10,"hotKeyFraction":1} +--fanout=1 +--iterations=4 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=false +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_2GB_of_100B_records.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_2GB_of_100B_records.txt new file mode 100644 index 000000000000..84349e3dc06e --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_2GB_of_100B_records.txt @@ -0,0 +1,30 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java11_Dataflow_V2_streaming_GBK_2 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_streaming_gbk_2 +--influxTags={"runnerVersion":"v2","jdk":"java11"} +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90} +--fanout=1 +--iterations=1 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=true +--inputWindowDurationSec=1200 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_2GB_of_100kB_records.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_2GB_of_100kB_records.txt new file mode 100644 index 000000000000..761fad3d11ba --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_2GB_of_100kB_records.txt @@ -0,0 +1,30 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java11_Dataflow_V2_streaming_GBK_3 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_streaming_gbk_3 +--influxTags={"runnerVersion":"v2","jdk":"java11"} +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000,"keySizeBytes":10000,"valueSizeBytes":90000} +--fanout=1 +--iterations=1 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=true +--inputWindowDurationSec=1200 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_2GB_of_10B_records.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_2GB_of_10B_records.txt new file mode 100644 index 000000000000..1b6624e52482 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_2GB_of_10B_records.txt @@ -0,0 +1,30 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java11_Dataflow_V2_streaming_GBK_1 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_streaming_gbk_1 +--influxTags={"runnerVersion":"v2","jdk":"java11"} +--publishToInfluxDB=true +--sourceOptions={"numRecords":200000000,"keySizeBytes":1,"valueSizeBytes":9} +--fanout=1 +--iterations=1 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=true +--inputWindowDurationSec=1200 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_fanout_4_times_with_2GB_10-byte_records_total.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_fanout_4_times_with_2GB_10-byte_records_total.txt new file mode 100644 index 000000000000..cf173fa845c3 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_fanout_4_times_with_2GB_10-byte_records_total.txt @@ -0,0 +1,30 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java11_Dataflow_V2_streaming_GBK_4 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_streaming_gbk_4 +--influxTags={"runnerVersion":"v2","jdk":"java11"} +--publishToInfluxDB=true +--sourceOptions={"numRecords":5000000,"keySizeBytes":10,"valueSizeBytes":90} +--fanout=4 +--iterations=1 +--numWorkers=16 +--autoscalingAlgorithm=NONE +--streaming=true +--inputWindowDurationSec=1200 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_fanout_8_times_with_2GB_10-byte_records_total.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_fanout_8_times_with_2GB_10-byte_records_total.txt new file mode 100644 index 000000000000..8e751d638ceb --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_fanout_8_times_with_2GB_10-byte_records_total.txt @@ -0,0 +1,30 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java11_Dataflow_V2_streaming_GBK_5 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_streaming_gbk_5 +--influxTags={"runnerVersion":"v2","jdk":"java11"} +--publishToInfluxDB=true +--sourceOptions={"numRecords":2500000,"keySizeBytes":10,"valueSizeBytes":90} +--fanout=8 +--iterations=1 +--numWorkers=16 +--autoscalingAlgorithm=NONE +--streaming=true +--inputWindowDurationSec=1200 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_reiterate_4_times_10kB_values.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_reiterate_4_times_10kB_values.txt new file mode 100644 index 000000000000..2e9c847cb832 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_reiterate_4_times_10kB_values.txt @@ -0,0 +1,30 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java11_Dataflow_V2_streaming_GBK_6 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_streaming_gbk_6 +--influxTags={"runnerVersion":"v2","jdk":"java11"} +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":200,"hotKeyFraction":1} +--fanout=1 +--iterations=4 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=true +--inputWindowDurationSec=1200 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_reiterate_4_times_2MB_values.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_reiterate_4_times_2MB_values.txt new file mode 100644 index 000000000000..dd410a81487c --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_reiterate_4_times_2MB_values.txt @@ -0,0 +1,30 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java11_Dataflow_V2_streaming_GBK_7 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_streaming_gbk_7 +--influxTags={"runnerVersion":"v2","jdk":"java11"} +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":10,"hotKeyFraction":1} +--fanout=1 +--iterations=4 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=true +--inputWindowDurationSec=1200 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_2GB_of_100B_records.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_2GB_of_100B_records.txt new file mode 100644 index 000000000000..05d93f213ec7 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_2GB_of_100B_records.txt @@ -0,0 +1,30 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java17_Dataflow_V2_streaming_GBK_2 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_streaming_gbk_2 +--influxTags={"runnerVersion":"v2","jdk":"java17"} +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90} +--fanout=1 +--iterations=1 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=true +--inputWindowDurationSec=1200 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_2GB_of_100kB_records.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_2GB_of_100kB_records.txt new file mode 100644 index 000000000000..2391bfe4e416 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_2GB_of_100kB_records.txt @@ -0,0 +1,30 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java17_Dataflow_V2_streaming_GBK_3 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_streaming_gbk_3 +--influxTags={"runnerVersion":"v2","jdk":"java17"} +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000,"keySizeBytes":10000,"valueSizeBytes":90000} +--fanout=1 +--iterations=1 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=true +--inputWindowDurationSec=1200 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_2GB_of_10B_records.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_2GB_of_10B_records.txt new file mode 100644 index 000000000000..3c4e04e02adf --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_2GB_of_10B_records.txt @@ -0,0 +1,30 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java17_Dataflow_V2_streaming_GBK_1 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_streaming_gbk_1 +--influxTags={"runnerVersion":"v2","jdk":"java17"} +--publishToInfluxDB=true +--sourceOptions={"numRecords":200000000,"keySizeBytes":1,"valueSizeBytes":9} +--fanout=1 +--iterations=1 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=true +--inputWindowDurationSec=1200 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_fanout_4_times_with_2GB_10-byte_records_total.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_fanout_4_times_with_2GB_10-byte_records_total.txt new file mode 100644 index 000000000000..d43ab9d84688 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_fanout_4_times_with_2GB_10-byte_records_total.txt @@ -0,0 +1,30 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java17_Dataflow_V2_streaming_GBK_4 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_streaming_gbk_4 +--influxTags={"runnerVersion":"v2","jdk":"java17"} +--publishToInfluxDB=true +--sourceOptions={"numRecords":5000000,"keySizeBytes":10,"valueSizeBytes":90} +--fanout=4 +--iterations=1 +--numWorkers=16 +--autoscalingAlgorithm=NONE +--streaming=true +--inputWindowDurationSec=1200 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_fanout_8_times_with_2GB_10-byte_records_total.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_fanout_8_times_with_2GB_10-byte_records_total.txt new file mode 100644 index 000000000000..fd5233180ff6 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_fanout_8_times_with_2GB_10-byte_records_total.txt @@ -0,0 +1,30 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java17_Dataflow_V2_streaming_GBK_5 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_streaming_gbk_5 +--influxTags={"runnerVersion":"v2","jdk":"java17"} +--publishToInfluxDB=true +--sourceOptions={"numRecords":2500000,"keySizeBytes":10,"valueSizeBytes":90} +--fanout=8 +--iterations=1 +--numWorkers=16 +--autoscalingAlgorithm=NONE +--streaming=true +--inputWindowDurationSec=1200 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_reiterate_4_times_10kB_values.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_reiterate_4_times_10kB_values.txt new file mode 100644 index 000000000000..50d17886b7c9 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_reiterate_4_times_10kB_values.txt @@ -0,0 +1,30 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java17_Dataflow_V2_streaming_GBK_6 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_streaming_gbk_6 +--influxTags={"runnerVersion":"v2","jdk":"java17"} +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":200,"hotKeyFraction":1} +--fanout=1 +--iterations=4 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=true +--inputWindowDurationSec=1200 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_reiterate_4_times_2MB_values.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_reiterate_4_times_2MB_values.txt new file mode 100644 index 000000000000..e28d4d895ada --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_reiterate_4_times_2MB_values.txt @@ -0,0 +1,30 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java17_Dataflow_V2_streaming_GBK_7 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_streaming_gbk_7 +--influxTags={"runnerVersion":"v2","jdk":"java17"} +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":10,"hotKeyFraction":1} +--fanout=1 +--iterations=4 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=true +--inputWindowDurationSec=1200 +--runner=DataflowRunner \ No newline at end of file From 2e0521162f6e23de29172eebfd3fd91f64329634 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud <65791736+ahmedabu98@users.noreply.github.com> Date: Tue, 3 Oct 2023 11:46:14 -0400 Subject: [PATCH 010/435] [Java BQ] Storage API streaming load test (#28264) * more accurate regex for table spec; add toTableSpec * storage api streaming load test * use io performance test utilities * only fail 30% of the time * use periodicimpulse stopAfter * use custom bq service with crashing behavior --- ...m_PostCommit_Java_IO_Performance_Tests.yml | 2 +- it/build.gradle | 4 + it/google-cloud-platform/build.gradle | 5 +- .../it/gcp/bigquery/BigQueryStreamingLT.java | 643 ++++++++++++++++++ .../beam/sdk/io/gcp/bigquery/BigQueryIO.java | 6 +- .../io/gcp/bigquery/BigQueryServicesImpl.java | 6 +- .../sdk/io/gcp/bigquery/BigQueryUtils.java | 29 +- .../io/gcp/bigquery/BigQueryUtilsTest.java | 37 +- 8 files changed, 723 insertions(+), 9 deletions(-) create mode 100644 it/google-cloud-platform/src/test/java/org/apache/beam/it/gcp/bigquery/BigQueryStreamingLT.java diff --git a/.github/workflows/beam_PostCommit_Java_IO_Performance_Tests.yml b/.github/workflows/beam_PostCommit_Java_IO_Performance_Tests.yml index 471782621fa7..b38651352a67 100644 --- a/.github/workflows/beam_PostCommit_Java_IO_Performance_Tests.yml +++ b/.github/workflows/beam_PostCommit_Java_IO_Performance_Tests.yml @@ -65,7 +65,7 @@ jobs: matrix: job_name: ["beam_PostCommit_Java_IO_Performance_Tests"] job_phrase: ["Run Java PostCommit IO Performance Tests"] - test_case: ["GCSPerformanceTest", "BigTablePerformanceTest"] + test_case: ["GCSPerformanceTest", "BigTablePerformanceTest", "BigQueryStorageApiStreamingPerformanceTest"] steps: - uses: actions/checkout@v4 - name: Setup repository diff --git a/it/build.gradle b/it/build.gradle index 35ccbba4c360..42a9ad9f4ee8 100644 --- a/it/build.gradle +++ b/it/build.gradle @@ -30,4 +30,8 @@ tasks.register('GCSPerformanceTest') { tasks.register('BigTablePerformanceTest') { dependsOn(":it:google-cloud-platform:BigTablePerformanceTest") +} + +tasks.register('BigQueryStorageApiStreamingPerformanceTest') { + dependsOn(":it:google-cloud-platform:BigQueryStorageApiStreamingPerformanceTest") } \ No newline at end of file diff --git a/it/google-cloud-platform/build.gradle b/it/google-cloud-platform/build.gradle index 0917ddd3e21a..4c5327b44c9a 100644 --- a/it/google-cloud-platform/build.gradle +++ b/it/google-cloud-platform/build.gradle @@ -15,6 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +import groovy.json.JsonOutput import org.apache.beam.gradle.IoPerformanceTestUtilities plugins { id 'org.apache.beam.module' } @@ -73,9 +74,11 @@ dependencies { testImplementation project(path: ":sdks:java:io:google-cloud-platform") testImplementation project(path: ":sdks:java:io:synthetic") testImplementation library.java.mockito_inline + testImplementation project(path: ":sdks:java:extensions:google-cloud-platform-core", configuration: "testRuntimeMigration") testRuntimeOnly project(path: ":runners:direct-java", configuration: "shadowTest") testRuntimeOnly library.java.slf4j_simple } tasks.register("GCSPerformanceTest", IoPerformanceTestUtilities.IoPerformanceTest, project, 'google-cloud-platform', 'FileBasedIOLT', ['configuration':'large','project':'apache-beam-testing', 'artifactBucket':'io-performance-temp']) -tasks.register("BigTablePerformanceTest", IoPerformanceTestUtilities.IoPerformanceTest, project, 'google-cloud-platform', 'BigTableIOLT', ['configuration':'large','project':'apache-beam-testing', 'artifactBucket':'io-performance-temp']) \ No newline at end of file +tasks.register("BigTablePerformanceTest", IoPerformanceTestUtilities.IoPerformanceTest, project, 'google-cloud-platform', 'BigTableIOLT', ['configuration':'large','project':'apache-beam-testing', 'artifactBucket':'io-performance-temp']) +tasks.register("BigQueryStorageApiStreamingPerformanceTest", IoPerformanceTestUtilities.IoPerformanceTest, project, 'google-cloud-platform', 'BigQueryStreamingLT', ['configuration':'large', 'project':'apache-beam-testing', 'artifactBucket':'io-performance-temp']) diff --git a/it/google-cloud-platform/src/test/java/org/apache/beam/it/gcp/bigquery/BigQueryStreamingLT.java b/it/google-cloud-platform/src/test/java/org/apache/beam/it/gcp/bigquery/BigQueryStreamingLT.java new file mode 100644 index 000000000000..4589f79f1aaa --- /dev/null +++ b/it/google-cloud-platform/src/test/java/org/apache/beam/it/gcp/bigquery/BigQueryStreamingLT.java @@ -0,0 +1,643 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.it.gcp.bigquery; + +import static org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils.toTableReference; +import static org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils.toTableSpec; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; + +import com.google.api.core.ApiFuture; +import com.google.api.services.bigquery.model.TableFieldSchema; +import com.google.api.services.bigquery.model.TableRow; +import com.google.api.services.bigquery.model.TableSchema; +import com.google.auto.value.AutoValue; +import com.google.cloud.bigquery.storage.v1.FlushRowsResponse; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.time.Instant; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; +import java.util.stream.Collectors; +import org.apache.beam.it.common.PipelineLauncher; +import org.apache.beam.it.common.PipelineOperator; +import org.apache.beam.it.common.TestProperties; +import org.apache.beam.it.gcp.IOLoadTestBase; +import org.apache.beam.runners.dataflow.DataflowRunner; +import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; +import org.apache.beam.sdk.io.GenerateSequence; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryOptions; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServicesImpl; +import org.apache.beam.sdk.io.gcp.testing.BigqueryClient; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.testing.TestPipelineOptions; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.PeriodicImpulse; +import org.apache.beam.sdk.transforms.Reshuffle; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.TypeDescriptors; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Strings; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.joda.time.Duration; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Ignore; +import org.junit.Rule; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Load test for the Storage Write API sink + * + *

This test is set up to first write rows using batch FILE_LOADS mode to a "source of truth" + * table. Afterwards, it will write the same rows in streaming mode with Storage API to a second + * table. Then it will query between these two tables to check that they are identical. There is + * also the option of providing an existing table with the expected data, in which case the test + * will skip the first step. + * + *

The throughput, length of test (in minutes), and data shape can be changed via pipeline + * options. See the cases in `getOptions()` for examples. + * + *

This also includes the option of testing the sink's retry resilience by setting the + * `crashIntervalSeconds` System property. This intentionally fails the worker or work item + * periodically and expects the sink to recover appropriately. Note: Metrics are not published when + * this is used. + */ +public class BigQueryStreamingLT extends IOLoadTestBase { + private static final Logger LOG = LoggerFactory.getLogger(BigQueryStreamingLT.class); + + private static final BigqueryClient BQ_CLIENT = new BigqueryClient("BigQueryStreamingLT"); + private static final String BIG_QUERY_DATASET_ID = + "storage_api_sink_load_test_" + System.nanoTime(); + + private TestConfiguration config; + private Integer crashIntervalSeconds; + + @Rule public final transient TestPipeline fileLoadsPipeline = TestPipeline.create(); + @Rule public final transient TestPipeline storageApiPipeline = TestPipeline.create(); + + @BeforeClass + public static void setUpTestClass() throws IOException, InterruptedException { + PipelineOptionsFactory.register(TestPipelineOptions.class); + BQ_CLIENT.createNewDataset(project, BIG_QUERY_DATASET_ID); + } + + @Before + public void setUpTest() { + String testConfig = + TestProperties.getProperty("configuration", "small", TestProperties.Type.PROPERTY); + config = TEST_CONFIGS.get(testConfig); + if (config == null) { + throw new IllegalArgumentException( + String.format( + "Unknown test configuration: [%s]. Known configs: %s", + testConfig, TEST_CONFIGS.keySet())); + } + // tempLocation needs to be set for file loads + if (!Strings.isNullOrEmpty(tempBucketName)) { + String tempLocation = String.format("gs://%s/temp/", tempBucketName); + fileLoadsPipeline.getOptions().as(TestPipelineOptions.class).setTempRoot(tempLocation); + fileLoadsPipeline.getOptions().setTempLocation(tempLocation); + } + + // Set expected table if the property is provided, + @Nullable + String expectedTable = + TestProperties.getProperty("expectedTable", "", TestProperties.Type.PROPERTY); + if (!Strings.isNullOrEmpty(expectedTable)) { + config.toBuilder().setExpectedTable(expectedTable).build(); + } + + crashIntervalSeconds = + Integer.parseInt( + TestProperties.getProperty("crashIntervalSeconds", "-1", TestProperties.Type.PROPERTY)); + } + + @AfterClass + public static void cleanup() { + BQ_CLIENT.deleteDataset(project, BIG_QUERY_DATASET_ID); + } + + private static final Map TEST_CONFIGS = + ImmutableMap.of( + "local", // 300K rows, >3 MB, 1K rows/s, >10KB/s + TestConfiguration.of(5, 5, 2, 1_000, "DirectRunner", null), + "small", // 600K rows, >30 MB, 1K rows/s, >50KB/s + TestConfiguration.of(10, 10, 5, 1_000, "DataflowRunner", null), + "medium", // 6M rows, >1.2 GB, 5K rows/s, >1MB/s + TestConfiguration.of(20, 20, 10, 5_000, "DataflowRunner", null), + "large", // 18M rows, >18 GB, 10K rows/s, >10MB/s + TestConfiguration.of(30, 50, 20, 10_000, "DataflowRunner", null)); + + /** Options for Bigquery IO Streaming load test. */ + @AutoValue + abstract static class TestConfiguration { + /** Rows will be generated for this many minutes. */ + abstract Integer getMinutes(); + + /** Data shape: The byte-size for each field. */ + abstract Integer getByteSizePerField(); + + /** Data shape: The number of fields per row. */ + abstract Integer getNumFields(); + + /** + * Rate of generated elements sent to the sink. Will run with a minimum of 1k rows per second. + */ + abstract Integer getRowsPerSecond(); + + abstract String getRunner(); + + /** + * The expected table to check against for correctness. If unset, the test will run a batch + * FILE_LOADS job and use the resulting table as a source of truth. + */ + @Nullable + abstract String getExpectedTable(); + + static TestConfiguration of( + int numMin, + int byteSizePerField, + int numFields, + int rowsPerSecond, + String runner, + @Nullable String expectedTable) { + return new AutoValue_BigQueryStreamingLT_TestConfiguration.Builder() + .setMinutes(numMin) + .setByteSizePerField(byteSizePerField) + .setNumFields(numFields) + .setRowsPerSecond(rowsPerSecond) + .setRunner(runner) + .setExpectedTable(expectedTable) + .build(); + } + + @AutoValue.Builder + abstract static class Builder { + abstract Builder setMinutes(int numMin); + + abstract Builder setByteSizePerField(int byteSizePerField); + + abstract Builder setNumFields(int numFields); + + abstract Builder setRowsPerSecond(int rowsPerSecond); + + abstract Builder setRunner(String runner); + + abstract Builder setExpectedTable(@Nullable String expectedTable); + + abstract TestConfiguration build(); + } + + abstract Builder toBuilder(); + } + + @Test + public void testExactlyOnceStreaming() throws IOException, InterruptedException { + runTest(BigQueryIO.Write.Method.STORAGE_WRITE_API); + } + + @Test + @Ignore + public void testAtLeastOnceStreaming() throws IOException, InterruptedException { + runTest(BigQueryIO.Write.Method.STORAGE_API_AT_LEAST_ONCE); + } + + public void runTest(BigQueryIO.Write.Method writeMethod) + throws IOException, InterruptedException { + long millis = Duration.standardMinutes(config.getMinutes()).getMillis(); + int rowsPerSecond = Math.max(config.getRowsPerSecond(), 1000); + + // The PeriodicImpulse source will generate an element every this many millis: + int fireInterval = 1; + // Each element from PeriodicImpulse will fan out to this many elements + // (applicable when a high row-per-second rate is set) + long multiplier = rowsPerSecond / 1000; + long totalRows = multiplier * millis / fireInterval; + // If we run with DataflowRunner and have not specified a positive crash duration for the sink, + // this signifies a performance test, and so we publish metrics to a BigQuery dataset + boolean publishMetrics = + config.getRunner().equalsIgnoreCase(DataflowRunner.class.getSimpleName()) + && crashIntervalSeconds <= 0; + + String expectedTable = config.getExpectedTable(); + GenerateTableRow genRow = + new GenerateTableRow(config.getNumFields(), config.getByteSizePerField()); + TableSchema schema = generateTableSchema(config.getNumFields()); + if (Strings.isNullOrEmpty(expectedTable)) { + String fileLoadsDescription = + String.format("fileloads-%s-records", withScaleSymbol(totalRows)); + expectedTable = + String.format("%s.%s.%s", project, BIG_QUERY_DATASET_ID, fileLoadsDescription); + LOG.info( + "No expected table was set. Will run a batch job to load {} rows to {}." + + " This will be used as the source of truth.", + totalRows, + expectedTable); + + fileLoadsPipeline + .apply(GenerateSequence.from(0).to(totalRows)) + .apply( + "Write to source of truth", + BigQueryIO.write() + .to(expectedTable) + .withFormatFunction(genRow) + .withMethod(BigQueryIO.Write.Method.FILE_LOADS) + .withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_TRUNCATE) + .withSchema(schema)); + + // If running on Dataflow, launch pipeline via launcher utils + if (publishMetrics) { + PipelineLauncher.LaunchConfig options = + PipelineLauncher.LaunchConfig.builder("test-" + fileLoadsDescription) + .setSdk(PipelineLauncher.Sdk.JAVA) + .setPipeline(fileLoadsPipeline) + .addParameter("runner", config.getRunner()) + .build(); + + // Don't use PipelineOperator because we don't want to wait on this batch job + // The streaming job will run in parallel and it will take longer anyways; this job will + // finish by then. + pipelineLauncher.launch(project, region, options); + } else { + fileLoadsPipeline.run(); + } + } + + String atLeastOnce = + writeMethod == BigQueryIO.Write.Method.STORAGE_API_AT_LEAST_ONCE ? "-atleastonce" : ""; + String storageApiDescription = + String.format( + "storageapi%s-load-%sqps-%smin-%stotal", + atLeastOnce, + withScaleSymbol(rowsPerSecond), + config.getMinutes(), + withScaleSymbol(totalRows)); + String destTable = + String.format("%s.%s.%s", project, BIG_QUERY_DATASET_ID, storageApiDescription); + LOG.info( + "Preparing a source generating at a rate of {} rows per second for a period of {} minutes." + + " This results in a total of {} rows written to {}.", + rowsPerSecond, + config.getMinutes(), + totalRows, + destTable); + + PCollection source = + storageApiPipeline + .apply( + PeriodicImpulse.create() + .stopAfter(Duration.millis(millis - 1)) + .withInterval(Duration.millis(fireInterval))) + .apply( + "Extract row IDs", + MapElements.into(TypeDescriptors.longs()) + .via(instant -> instant.getMillis() % totalRows)); + if (multiplier > 1) { + source = + source + .apply( + String.format("One input to %s outputs", multiplier), + ParDo.of(new MultiplierDoFn(multiplier))) + .apply("Reshuffle fanout", Reshuffle.viaRandomKey()); + } + + BigQueryIO.Write storageWriteTransform = + BigQueryIO.write() + .to(destTable) + .withFormatFunction(genRow) + .withMethod(writeMethod) + .withTriggeringFrequency(Duration.standardSeconds(1)) + .withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_APPEND) + .withSchema(schema); + + // If a crash interval is specified, use our crashing service implementation + if (crashIntervalSeconds > 0) { + LOG.info( + "A crash interval of {} seconds has been set. The Storage API sink will periodically crash.", + crashIntervalSeconds); + storageWriteTransform = + storageWriteTransform.withTestServices( + new CrashingBigQueryServices(crashIntervalSeconds)); + } + source.apply(storageWriteTransform); + + // If we're publishing metrics, launch pipeline via Dataflow launcher utils and export metrics + if (publishMetrics) { + // Set up dataflow job + PipelineLauncher.LaunchConfig storageApiOptions = + PipelineLauncher.LaunchConfig.builder("test-" + storageApiDescription) + .setSdk(PipelineLauncher.Sdk.JAVA) + .setPipeline(storageApiPipeline) + .addParameter("runner", config.getRunner()) + .addParameter("streaming", "true") + .addParameter("experiments", GcpOptions.STREAMING_ENGINE_EXPERIMENT) + .build(); + // Launch job + PipelineLauncher.LaunchInfo storageApiInfo = + pipelineLauncher.launch(project, region, storageApiOptions); + // Wait until the streaming pipeline is finished and drained, get the result. + PipelineOperator.Result storageApiResult = + pipelineOperator.waitUntilDoneAndFinish( + PipelineOperator.Config.builder() + .setJobId(storageApiInfo.jobId()) + .setProject(project) + .setRegion(region) + .setTimeoutAfter(java.time.Duration.ofMinutes(config.getMinutes() * 2L)) + .setCheckAfter(java.time.Duration.ofSeconds(config.getMinutes() * 60 / 20)) + .build()); + // Check the initial launch didn't fail + assertNotEquals(PipelineOperator.Result.LAUNCH_FAILED, storageApiResult); + // Check that the pipeline succeeded + assertEquals( + PipelineLauncher.JobState.DONE, + pipelineLauncher.getJobStatus(project, region, storageApiInfo.jobId())); + + // Export metrics + MetricsConfiguration metricsConfig = + MetricsConfiguration.builder() + .setInputPCollection( + (multiplier > 1) ? "Extract row IDs.out0" : "Reshuffle fanout.out0") + .build(); + try { + exportMetricsToBigQuery(storageApiInfo, getMetrics(storageApiInfo, metricsConfig)); + } catch (Exception e) { + // Just log the error. Don't re-throw because we have accuracy checks that are more + // important below + LOG.error("Encountered an error while exporting metrics to BigQuery:\n{}", e); + } + } + // If we're not publishing metrics, just run the pipeline normally + else { + storageApiPipeline.run().waitUntilFinish(); + } + + LOG.info( + "Write pipeline finished writing to {}. Will now perform accuracy checks against the rows in {}.", + destTable, + expectedTable); + // Filter our structs and arrays because they are not supported when querying with `EXCEPT + // DISTINCT` + String columnNames = + schema.getFields().stream() + .map(TableFieldSchema::getName) + .filter(fieldName -> fieldName.startsWith(FIELD_PREFIX)) + .collect(Collectors.joining(", ")); + checkCorrectness(columnNames, destTable, expectedTable); + // check non-duplication for STORAGE_WRITE_API + if (writeMethod == BigQueryIO.Write.Method.STORAGE_WRITE_API) { + checkNonDuplication(destTable, expectedTable, totalRows); + } + } + + // A BigQueryServices class that is almost identical to BigQueryServicesImpl, except that + // it returns a dataset service implementation that periodically crashes on flush() + private static class CrashingBigQueryServices extends BigQueryServicesImpl { + public final Integer crashIntervalSeconds; + + public CrashingBigQueryServices(Integer crashIntervalSeconds) { + this.crashIntervalSeconds = crashIntervalSeconds; + } + + @Override + public DatasetService getDatasetService(BigQueryOptions options) { + return new CrashingDatasetService(options); + } + + private class CrashingDatasetService extends BigQueryServicesImpl.DatasetServiceImpl { + private Instant lastCrash; + + public CrashingDatasetService(BigQueryOptions bqOptions) { + super(bqOptions); + } + + // We choose flush() to host the crash logic because it's called frequently during + // the span of a Storage Write API pipeline + @Override + public ApiFuture flush(String streamName, long flushOffset) + throws IOException, InterruptedException { + maybeCrash(); + return super.flush(streamName, flushOffset); + } + + // When specified, crash when the interval is met by: + // throwing an exception (failed work item) or + // performing a System exit (worker failure) + private void maybeCrash() { + if (crashIntervalSeconds != -1) { + Instant last = lastCrash; + if (last == null) { + lastCrash = Instant.now(); + } else if (Instant.now().isAfter(last.plusSeconds(crashIntervalSeconds))) { + lastCrash = Instant.now(); + + // Only crash 30% of the time (this is arbitrary) + if (ThreadLocalRandom.current().nextInt(100) < 30) { + // Half the time throw an exception (which fails this specific work item) + // Other half crash the entire worker, which fails all work items on this worker + if (ThreadLocalRandom.current().nextBoolean()) { + throw new RuntimeException( + "Throwing a random exception! This is for testing retry resilience."); + } else { + LOG.error("Crashing this worker! This is for testing retry resilience."); + System.exit(0); + } + } + } + } + } + } + } + + public void checkCorrectness(String columnNames, String destTable, String expectedTable) + throws IOException, InterruptedException { + // Need table spec to be in the format `myproject.mydataset.mytable` to include in BQ queries. + destTable = toTableSpec(toTableReference(destTable)); + expectedTable = toTableSpec(toTableReference(expectedTable)); + + String checkCorrectnessQuery = + String.format( + "WITH \n" + + "storage_api_table AS (SELECT %s FROM `%s`), \n" + + "expected_table AS (SELECT %s FROM `%s`), \n" + + "rows_mismatched AS (SELECT * FROM expected_table EXCEPT DISTINCT SELECT * FROM storage_api_table) \n" + + "SELECT COUNT(*) FROM rows_mismatched", + columnNames, destTable, columnNames, expectedTable); + + LOG.info("Executing query to check correctness:\n{}", checkCorrectnessQuery); + + TableRow queryResponse = + Iterables.getOnlyElement( + BQ_CLIENT.queryUnflattened(checkCorrectnessQuery, "google.com:clouddfe", true, true)); + long result = Long.parseLong((String) queryResponse.get("f0_")); + + LOG.info("Number of mismatched rows: {}", result); + assertEquals( + String.format("Saw %s rows that are missing from %s.", result, destTable), 0, result); + } + + public void checkNonDuplication(String destTable, String expectedTable, long totalRows) + throws IOException, InterruptedException { + String checkDuplicationQuery = + String.format( + "SELECT \n" + + "(SELECT COUNT(*) FROM `%s`) AS actualCount,\n" + + "(SELECT COUNT(*) FROM `%s`) AS expectedCount", + destTable, expectedTable); + + LOG.info("Executing query to check non-duplication:\n{}", checkDuplicationQuery); + + TableRow queryResponse = + Iterables.getOnlyElement( + BQ_CLIENT.queryUnflattened(checkDuplicationQuery, "google.com:clouddfe", true, true)); + long actualCount = Long.parseLong((String) queryResponse.get("actualCount")); + long expectedCount = Long.parseLong((String) queryResponse.get("expectedCount")); + assertEquals( + "Comparing actual table count and expected table count.", expectedCount, actualCount); + assertEquals( + "Comparing actual table count and calculated expected count.", totalRows, actualCount); + } + + // From a value, get the appropriate shortened name that includes the scale + // For example, from 12,345,678 return 12M + public String withScaleSymbol(long value) { + List scales = Arrays.asList("", "K", "M", "B", "T", "Q"); + int scaleIndex = 0; + while (value / 1000 > 0) { + scaleIndex++; + value /= 1000; + } + + return String.format("%s%s", value, scales.get(scaleIndex)); + } + + public static class MultiplierDoFn extends DoFn { + private long multiplier; + + MultiplierDoFn(long multiplier) { + this.multiplier = multiplier; + } + + @ProcessElement + public void processElement(@Element Long element, OutputReceiver outputReceiver) { + for (int i = 0; i < multiplier; i++) { + outputReceiver.output(element); + } + } + } + + static final String FIELD_PREFIX = "byte_field_"; + static final String RECORD_FIELD_PREFIX = "record_" + FIELD_PREFIX; + static final String NESTED_FIELD_PREFIX = "nested_" + FIELD_PREFIX; + static final String REPEATED_FIELD_PREFIX = "repeated_" + FIELD_PREFIX; + + public static TableSchema generateTableSchema(int numFields) { + List fields = new ArrayList<>(numFields); + fields.add(new TableFieldSchema().setType("INTEGER").setName("id")); + int j = 1; + for (int i = 1; i <= numFields; i++) { + TableFieldSchema fieldSchema = new TableFieldSchema(); + // Every 4th field will be a struct, every 5th field will be an array + if (j == 4) { + fieldSchema + .setType("RECORD") + .setName(RECORD_FIELD_PREFIX + i) + .setFields( + Arrays.asList( + new TableFieldSchema().setType("BYTES").setName(NESTED_FIELD_PREFIX + 1), + new TableFieldSchema().setType("BYTES").setName(NESTED_FIELD_PREFIX + 2))); + } else if (j == 5) { + fieldSchema.setType("BYTES").setMode("REPEATED").setName(REPEATED_FIELD_PREFIX + i); + j = 0; + } else { + fieldSchema.setType("BYTES").setName(FIELD_PREFIX + i); + } + j++; + fields.add(fieldSchema); + } + return new TableSchema().setFields(fields); + } + + static class GenerateTableRow implements SerializableFunction { + private final int numFields; + private final int sizePerField; + + public GenerateTableRow(int numFields, int sizePerField) { + assert numFields >= 0; + this.numFields = numFields; + this.sizePerField = sizePerField; + } + + @Override + public TableRow apply(Long rowId) { + TableRow row = new TableRow(); + row.set("id", rowId); + byte[] payload = getPayload(sizePerField, rowId).array(); + int j = 1; + for (int i = 1; i <= numFields; i++) { + // TODO: we can also make the struct and array sizes variable + if (j == 4) { + row.set( + RECORD_FIELD_PREFIX + i, + new TableRow() + .set(NESTED_FIELD_PREFIX + 1, Arrays.copyOfRange(payload, 0, sizePerField / 2)) + .set( + NESTED_FIELD_PREFIX + 2, + Arrays.copyOfRange(payload, sizePerField / 2, sizePerField))); + } else if (j == 5) { + row.set( + REPEATED_FIELD_PREFIX + i, + Arrays.asList( + Arrays.copyOfRange(payload, 0, sizePerField / 3), + Arrays.copyOfRange(payload, sizePerField / 3, sizePerField * 2 / 3), + Arrays.copyOfRange(payload, sizePerField * 2 / 3, sizePerField))); + j = 0; + } else { + row.set(FIELD_PREFIX + i, payload); + } + j++; + } + return row; + } + + private @Nullable ByteBuffer getPayload(int payloadSize, long rowId) { + if (payloadSize <= 0) { + return null; + } + byte[] payload = new byte[payloadSize]; + Random localRandom = ThreadLocal.withInitial(() -> new Random(rowId)).get(); + localRandom.setSeed(rowId); + localRandom.nextBytes(payload); + + return ByteBuffer.wrap(payload); + } + } +} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java index 58d769312444..ce8ddb683d1e 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java @@ -2499,6 +2499,8 @@ public enum WriteDisposition { *

The replacement may occur in multiple steps - for instance by first removing the * existing table, then creating a replacement, then filling it in. This is not an atomic * operation, and external programs may see the table in any of these intermediate steps. + * + *

Note: This write disposition is only supported for the FILE_LOADS write method. */ WRITE_TRUNCATE, @@ -3250,7 +3252,7 @@ && getStorageApiTriggeringFrequency(bqOptions) != null) { checkArgument(getNumFileShards() == 0, "Number of file shards" + error); if (getStorageApiTriggeringFrequency(bqOptions) != null) { - LOG.warn("Storage API triggering frequency" + error); + LOG.warn("Setting a triggering frequency" + error); } if (getStorageApiNumStreams(bqOptions) != 0) { LOG.warn("Setting the number of Storage API streams" + error); @@ -3266,6 +3268,8 @@ && getStorageApiTriggeringFrequency(bqOptions) != null) { checkArgument( !getAutoSchemaUpdate(), "withAutoSchemaUpdate only supported when using STORAGE_WRITE_API or STORAGE_API_AT_LEAST_ONCE."); + } else if (getWriteDisposition() == WriteDisposition.WRITE_TRUNCATE) { + LOG.error("The Storage API sink does not support the WRITE_TRUNCATE write disposition."); } if (getRowMutationInformationFn() != null) { checkArgument(getMethod() == Method.STORAGE_API_AT_LEAST_ONCE); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java index 17b5c5ebd99b..3d4565cb086e 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java @@ -154,7 +154,7 @@ "nullness", // TODO(https://github.com/apache/beam/issues/20506) "keyfor" }) -class BigQueryServicesImpl implements BigQueryServices { +public class BigQueryServicesImpl implements BigQueryServices { private static final Logger LOG = LoggerFactory.getLogger(BigQueryServicesImpl.class); // The maximum number of retries to execute a BigQuery RPC. @@ -549,7 +549,7 @@ public void close() throws Exception {} } @VisibleForTesting - static class DatasetServiceImpl implements DatasetService { + public static class DatasetServiceImpl implements DatasetService { // Backoff: 200ms * 1.5 ^ n, n=[1,5] private static final FluentBackoff INSERT_BACKOFF_FACTORY = FluentBackoff.DEFAULT.withInitialBackoff(Duration.millis(200)).withMaxRetries(5); @@ -610,7 +610,7 @@ static class DatasetServiceImpl implements DatasetService { this.executor = null; } - private DatasetServiceImpl(BigQueryOptions bqOptions) { + public DatasetServiceImpl(BigQueryOptions bqOptions) { this.errorExtractor = new ApiErrorExtractor(); this.client = newBigQueryClient(bqOptions).build(); this.newWriteClient = newBigQueryWriteClient(bqOptions); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java index 00ee815c3c93..fa5ffae0909d 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java @@ -64,6 +64,7 @@ import org.apache.beam.sdk.transforms.SerializableFunctions; import org.apache.beam.sdk.util.Preconditions; import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Strings; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; @@ -94,8 +95,15 @@ public class BigQueryUtils { // For parsing the format used to refer to tables parameters in BigQueryIO. // "{project_id}:{dataset_id}.{table_id}" or // "{project_id}.{dataset_id}.{table_id}" + // following documentation in + // https://cloud.google.com/resource-manager/docs/creating-managing-projects#before_you_begin, + // https://cloud.google.com/bigquery/docs/datasets#dataset-naming, and + // https://cloud.google.com/bigquery/docs/tables#table_naming private static final Pattern SIMPLE_TABLE_PATTERN = - Pattern.compile("^(?[^\\.:]+)[\\.:](?[^\\.:]+)[\\.](?[^\\.:]+)$"); + Pattern.compile( + "^(?[a-z][a-z0-9.\\-:]{4,28}[a-z0-9])[\\:.]" + + "(?[a-zA-Z0-9_]{1,1024})[\\.]" + + "(?
[\\p{L}\\p{M}\\p{N}\\p{Pc}\\p{Pd}\\p{Zs}$]{1,1024})$"); /** Options for how to convert BigQuery data to Beam data. */ @AutoValue @@ -1000,6 +1008,25 @@ private static Object convertAvroNumeric(Object value) { return null; } + /** + * @param tableReference - a BigQueryTableIdentifier that may or may not include the project. + * @return a String representation of the table destination in the form: + * `myproject.mydataset.mytable` + */ + public static @Nullable String toTableSpec(TableReference tableReference) { + if (tableReference.getDatasetId() == null || tableReference.getTableId() == null) { + throw new IllegalArgumentException( + String.format( + "Table reference [%s] must include at least a dataset and a table.", tableReference)); + } + String tableSpec = + String.format("%s.%s", tableReference.getDatasetId(), tableReference.getTableId()); + if (!Strings.isNullOrEmpty(tableReference.getProjectId())) { + tableSpec = String.format("%s.%s", tableReference.getProjectId(), tableSpec); + } + return tableSpec; + } + private static @Nullable ServiceCallMetric callMetricForMethod( @Nullable TableReference tableReference, String method) { if (tableReference != null) { diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilsTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilsTest.java index f4074cc1a556..d73ff5e2b712 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilsTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilsTest.java @@ -19,6 +19,7 @@ import static org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils.toTableRow; import static org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils.toTableSchema; +import static org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils.toTableSpec; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.containsInAnyOrder; @@ -994,6 +995,27 @@ public void testToBeamRow_avro_array_array_row() { assertEquals(expected, beamRow); } + @Test + public void testToTableSpec() { + TableReference withProject = + new TableReference().setProjectId("project").setDatasetId("dataset").setTableId("table"); + TableReference withoutProject = + new TableReference().setDatasetId("dataset").setTableId("table"); + TableReference withDatasetOnly = new TableReference().setDatasetId("dataset"); + TableReference withTableOnly = new TableReference().setTableId("table"); + + assertEquals("project.dataset.table", toTableSpec(withProject)); + assertEquals("dataset.table", toTableSpec(withoutProject)); + assertThrows( + "must include at least a dataset and a table", + IllegalArgumentException.class, + () -> toTableSpec(withDatasetOnly)); + assertThrows( + "must include at least a dataset and a table", + IllegalArgumentException.class, + () -> toTableSpec(withTableOnly)); + } + @Test public void testToTableReference() { { @@ -1020,6 +1042,14 @@ public void testToTableReference() { assertEquals("mytable", tr.getTableId()); } + { + // Test project that contains a dot and colon + TableReference tr = BigQueryUtils.toTableReference("project.with:domain.mydataset.mytable"); + assertEquals("project.with:domain", tr.getProjectId()); + assertEquals("mydataset", tr.getDatasetId()); + assertEquals("mytable", tr.getTableId()); + } + // Invalid scenarios assertNull(BigQueryUtils.toTableReference("")); assertNull(BigQueryUtils.toTableReference(":.")); @@ -1031,12 +1061,15 @@ public void testToTableReference() { assertNull(BigQueryUtils.toTableReference("myproject:mydataset.")); assertNull(BigQueryUtils.toTableReference("myproject:mydataset.mytable.")); assertNull(BigQueryUtils.toTableReference("myproject:mydataset:mytable:")); + assertNull(BigQueryUtils.toTableReference("myproject:my dataset:mytable:")); assertNull(BigQueryUtils.toTableReference(".invalidleadingdot.mydataset.mytable")); assertNull(BigQueryUtils.toTableReference("invalidtrailingdot.mydataset.mytable.")); assertNull(BigQueryUtils.toTableReference(":invalidleadingcolon.mydataset.mytable")); assertNull(BigQueryUtils.toTableReference("invalidtrailingcolon.mydataset.mytable:")); - assertNull(BigQueryUtils.toTableReference("myproject.mydataset.mytable.myinvalidpart")); - assertNull(BigQueryUtils.toTableReference("myproject:mydataset.mytable.myinvalidpart")); + assertNull(BigQueryUtils.toTableReference("projectendswithhyphen-.mydataset.mytable")); + assertNull( + BigQueryUtils.toTableReference( + "projectnamegoesbeyondthe30characterlimit.mydataset.mytable")); assertNull( BigQueryUtils.toTableReference("/projects/extraslash/datasets/mydataset/tables/mytable")); From 12b071402035a0404c4f6e4a83124b3839b5de40 Mon Sep 17 00:00:00 2001 From: Danny McCormick Date: Tue, 3 Oct 2023 14:02:48 -0400 Subject: [PATCH 011/435] Add notebook for per key models (#28327) * Add notebook for per key models * Wording updates * Add new notebook to readme * Apply suggestions from Rebecca Co-authored-by: Rebecca Szper <98840847+rszper@users.noreply.github.com> * Clarify return type * Missing comma * temporarily install from head so this is linkable --------- Co-authored-by: Rebecca Szper <98840847+rszper@users.noreply.github.com> --- examples/notebooks/beam-ml/README.md | 1 + .../notebooks/beam-ml/per_key_models.ipynb | 608 ++++++++++++++++++ 2 files changed, 609 insertions(+) create mode 100644 examples/notebooks/beam-ml/per_key_models.ipynb diff --git a/examples/notebooks/beam-ml/README.md b/examples/notebooks/beam-ml/README.md index 77bf3fc99f15..a1fe7ab19f51 100644 --- a/examples/notebooks/beam-ml/README.md +++ b/examples/notebooks/beam-ml/README.md @@ -77,6 +77,7 @@ This section contains the following example notebooks. ### Multi-model pipelines * [Ensemble model using an image captioning and ranking](https://github.com/apache/beam/blob/master/examples/notebooks/beam-ml/run_inference_multi_model.ipynb) +* [Run ML Inference with Different Models Per Key](https://github.com/apache/beam/blob/master/examples/notebooks/beam-ml/per_key_models.ipynb) ### Model Evaluation diff --git a/examples/notebooks/beam-ml/per_key_models.ipynb b/examples/notebooks/beam-ml/per_key_models.ipynb new file mode 100644 index 000000000000..b529449555d0 --- /dev/null +++ b/examples/notebooks/beam-ml/per_key_models.ipynb @@ -0,0 +1,608 @@ +{ + "nbformat": 4, + "nbformat_minor": 0, + "metadata": { + "colab": { + "provenance": [] + }, + "kernelspec": { + "name": "python3", + "display_name": "Python 3" + }, + "language_info": { + "name": "python" + } + }, + "cells": [ + { + "cell_type": "code", + "source": [ + "# @title ###### Licensed to the Apache Software Foundation (ASF), Version 2.0 (the \"License\")\n", + "\n", + "# Licensed to the Apache Software Foundation (ASF) under one\n", + "# or more contributor license agreements. See the NOTICE file\n", + "# distributed with this work for additional information\n", + "# regarding copyright ownership. The ASF licenses this file\n", + "# to you under the Apache License, Version 2.0 (the\n", + "# \"License\"); you may not use this file except in compliance\n", + "# with the License. You may obtain a copy of the License at\n", + "#\n", + "# http://www.apache.org/licenses/LICENSE-2.0\n", + "#\n", + "# Unless required by applicable law or agreed to in writing,\n", + "# software distributed under the License is distributed on an\n", + "# \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY\n", + "# KIND, either express or implied. See the License for the\n", + "# specific language governing permissions and limitations\n", + "# under the License" + ], + "metadata": { + "id": "OsFaZscKSPvo" + }, + "execution_count": 1, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "# Run ML inference with multiple differently-trained models\n", + "\n", + "
\n", + " \n", + " \n", + "
\n", + " Run in Google Colab\n", + " \n", + " View source on GitHub\n", + "
\n" + ], + "metadata": { + "id": "ZUSiAR62SgO8" + } + }, + { + "cell_type": "markdown", + "source": [ + "Running inference with multiple differently-trained models performing the same task is useful in many scenarios, including the following examples:\n", + "\n", + "* You want to compare the performance of multiple different models.\n", + "* You have models trained on different datasets that you want to use conditionally based on additional metadata.\n", + "\n", + "In Apache Beam, the recommended way to run inference is to use the `RunInference` transform. By using a `KeyedModelHandler`, you can efficiently run inference with O(100s) of models without having to manage memory yourself.\n", + "\n", + "This notebook demonstrates how to use a `KeyedModelHandler` to run inference in an Apache Beam pipeline with multiple different models on a per-key basis. This notebook uses pretrained pipelines from Hugging Face. Before continuing with this notebook, it is recommended that you walk through the [beginner RunInference notebook](https://colab.sandbox.google.com/github/apache/beam/blob/master/examples/notebooks/beam-ml/run_inference_pytorch_tensorflow_sklearn.ipynb)." + ], + "metadata": { + "id": "ZAVOrrW2An1n" + } + }, + { + "cell_type": "markdown", + "source": [ + "## Install dependencies\n", + "\n", + "First, install both Apache Beam and the dependencies needed by Hugging Face." + ], + "metadata": { + "id": "_fNyheQoDgGt" + } + }, + { + "cell_type": "code", + "execution_count": 11, + "metadata": { + "colab": { + "base_uri": "https://localhost:8080/" + }, + "id": "B-ENznuJqArA", + "outputId": "f72963fc-82db-4d0d-9225-07f6b501e256" + }, + "outputs": [ + { + "output_type": "stream", + "name": "stdout", + "text": [ + "" + ] + } + ], + "source": [ + "# Note that this notebook currently installs from Beam head since this feature hasn't been released yet.\n", + "# It will be released with version 2.51.0, at which point you can install with the following command:\n", + "# !pip install apache_beam[gcp]>=2.51.0 --quiet\n", + "!git clone https://github.com/apache/beam\n", + "!pip install -r beam/sdks/python/build-requirements.txt\n", + "!pip install -e ./beam/sdks/python[gcp]\n", + "!pip install torch --quiet\n", + "!pip install transformers --quiet\n", + "\n", + "# To use the newly installed versions, restart the runtime.\n", + "exit()" + ] + }, + { + "cell_type": "code", + "source": [ + "from typing import Dict\n", + "from typing import Iterable\n", + "from typing import Tuple\n", + "\n", + "from transformers import pipeline\n", + "\n", + "import apache_beam as beam\n", + "from apache_beam.ml.inference.base import KeyedModelHandler\n", + "from apache_beam.ml.inference.base import KeyModelMapping\n", + "from apache_beam.ml.inference.base import PredictionResult\n", + "from apache_beam.ml.inference.huggingface_inference import HuggingFacePipelineModelHandler\n", + "from apache_beam.ml.inference.base import RunInference" + ], + "metadata": { + "id": "wUmBEglvsOYW" + }, + "execution_count": 1, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "## Define the model configurations\n", + "\n", + "A model handler is the Apache Beam method used to define the configuration needed to load and invoke models. Because this example uses two models, we define two model handlers, one for each model. Because both models are incapsulated within Hugging Face pipelines, we use the model handler `HuggingFacePipelineModelHandler`.\n", + "\n", + "In this notebook, we load the models using Hugging Face and run them against an example. The models produce different outputs." + ], + "metadata": { + "id": "uEqljVgCD7hx" + } + }, + { + "cell_type": "code", + "source": [ + "distilbert_mh = HuggingFacePipelineModelHandler('text-classification', model=\"distilbert-base-uncased-finetuned-sst-2-english\")\n", + "roberta_mh = HuggingFacePipelineModelHandler('text-classification', model=\"roberta-large-mnli\")\n", + "\n", + "distilbert_pipe = pipeline('text-classification', model=\"distilbert-base-uncased-finetuned-sst-2-english\")\n", + "roberta_large_pipe = pipeline(model=\"roberta-large-mnli\")" + ], + "metadata": { + "id": "v2NJT5ZcxgH5", + "outputId": "3924d72e-5c49-477d-c50f-6d9098f5a4b2" + }, + "execution_count": 2, + "outputs": [ + { + "output_type": "display_data", + "data": { + "text/plain": [ + "Downloading (…)lve/main/config.json: 0%| | 0.00/629 [00:00-`." + ], + "metadata": { + "id": "r6GXL5PLFBY7" + } + }, + { + "cell_type": "code", + "source": [ + "class FormatExamples(beam.DoFn):\n", + " \"\"\"\n", + " Map each example to a tuple of ('-', 'example').\n", + " We use these keys to map our elements to the correct models.\n", + " \"\"\"\n", + " def process(self, element: Tuple[str, str]) -> Iterable[Tuple[str, str]]:\n", + " yield (f'distilbert-{element[1]}', element[0])\n", + " yield (f'roberta-{element[1]}', element[0])" + ], + "metadata": { + "id": "p2uVwws8zRpg" + }, + "execution_count": 6, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "Use the formatted keys to define a `KeyedModelHandler` that maps keys to the `ModelHandler` used for those keys. The `KeyedModelHandler` method lets you define an optional `max_models_per_worker_hint`, which limits the number of models that can be held in a single worker process at one time. If you're worried about your worker running out of memory, use this option. For more information about managing memory, see [Use a keyed ModelHandler](https://beam.apache.org/documentation/sdks/python-machine-learning/index.html#use-a-keyed-modelhandler)." + ], + "metadata": { + "id": "IP65_5nNGIb8" + } + }, + { + "cell_type": "code", + "source": [ + "per_key_mhs = [\n", + " KeyModelMapping(['distilbert-positive', 'distilbert-neutral', 'distilbert-negative'], distilbert_mh),\n", + " KeyModelMapping(['roberta-positive', 'roberta-neutral', 'roberta-negative'], roberta_mh)\n", + "]\n", + "mh = KeyedModelHandler(per_key_mhs, max_models_per_worker_hint=2)" + ], + "metadata": { + "id": "DZpfjeGL2hMG" + }, + "execution_count": 7, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "## Postprocess the results\n", + "\n", + "The `RunInference` transform returns a Tuple containing:\n", + "* the original key\n", + "* a `PredictionResult` object containing the original example and the inference.\n", + "Use those outputs to extract the relevant data. Then, to compare each model's prediction, group this data by the original example." + ], + "metadata": { + "id": "_a4ZmnD5FSeG" + } + }, + { + "cell_type": "code", + "source": [ + "class ExtractResults(beam.DoFn):\n", + " \"\"\"\n", + " Extract the relevant data from the PredictionResult object.\n", + " \"\"\"\n", + " def process(self, element: Tuple[str, PredictionResult]) -> Iterable[Tuple[str, Dict[str, str]]]:\n", + " actual_sentiment = element[0].split('-')[1]\n", + " model = element[0].split('-')[0]\n", + " result = element[1]\n", + " example = result.example\n", + " predicted_sentiment = result.inference[0]['label']\n", + "\n", + " yield (example, {'model': model, 'actual_sentiment': actual_sentiment, 'predicted_sentiment': predicted_sentiment})" + ], + "metadata": { + "id": "FOwFNQA053TG" + }, + "execution_count": 8, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "Finally, print the results produced by each model." + ], + "metadata": { + "id": "JVnv4gGbFohk" + } + }, + { + "cell_type": "code", + "source": [ + "class PrintResults(beam.DoFn):\n", + " \"\"\"\n", + " Print the results produced by each model along with the actual sentiment.\n", + " \"\"\"\n", + " def process(self, element: Tuple[str, Iterable[Dict[str, str]]]):\n", + " example = element[0]\n", + " actual_sentiment = element[1][0]['actual_sentiment']\n", + " predicted_sentiment_1 = element[1][0]['predicted_sentiment']\n", + " model_1 = element[1][0]['model']\n", + " predicted_sentiment_2 = element[1][1]['predicted_sentiment']\n", + " model_2 = element[1][1]['model']\n", + "\n", + " if model_1 == 'distilbert':\n", + " distilbert_prediction = predicted_sentiment_1\n", + " roberta_prediction = predicted_sentiment_2\n", + " else:\n", + " roberta_prediction = predicted_sentiment_1\n", + " distilbert_prediction = predicted_sentiment_2\n", + "\n", + " print(f'Example: {example}\\nActual Sentiment: {actual_sentiment}\\n'\n", + " f'Distilbert Prediction: {distilbert_prediction}\\n'\n", + " f'Roberta Prediction: {roberta_prediction}\\n------------')" + ], + "metadata": { + "id": "kUQJNYOa9Q5-" + }, + "execution_count": 9, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "## Run the pipeline\n", + "\n", + "Put together all of the pieces to run a single Apache Beam pipeline." + ], + "metadata": { + "id": "-LrpmM2PGAkf" + } + }, + { + "cell_type": "code", + "source": [ + "with beam.Pipeline() as beam_pipeline:\n", + "\n", + " formatted_examples = (\n", + " beam_pipeline\n", + " | \"ReadExamples\" >> beam.Create(examples)\n", + " | \"FormatExamples\" >> beam.ParDo(FormatExamples()))\n", + " inferences = (\n", + " formatted_examples\n", + " | \"Run Inference\" >> RunInference(mh)\n", + " | \"ExtractResults\" >> beam.ParDo(ExtractResults())\n", + " | \"GroupByExample\" >> beam.GroupByKey()\n", + " )\n", + "\n", + " inferences | beam.ParDo(PrintResults())\n", + "\n" + ], + "metadata": { + "colab": { + "base_uri": "https://localhost:8080/", + "height": 463 + }, + "id": "B9Wti3XH0Iqe", + "outputId": "528ad732-ecf8-4877-ab6a-badad7944fed" + }, + "execution_count": 10, + "outputs": [ + { + "output_type": "display_data", + "data": { + "application/javascript": [ + "\n", + " if (typeof window.interactive_beam_jquery == 'undefined') {\n", + " var jqueryScript = document.createElement('script');\n", + " jqueryScript.src = 'https://code.jquery.com/jquery-3.4.1.slim.min.js';\n", + " jqueryScript.type = 'text/javascript';\n", + " jqueryScript.onload = function() {\n", + " var datatableScript = document.createElement('script');\n", + " datatableScript.src = 'https://cdn.datatables.net/1.10.20/js/jquery.dataTables.min.js';\n", + " datatableScript.type = 'text/javascript';\n", + " datatableScript.onload = function() {\n", + " window.interactive_beam_jquery = jQuery.noConflict(true);\n", + " window.interactive_beam_jquery(document).ready(function($){\n", + " \n", + " });\n", + " }\n", + " document.head.appendChild(datatableScript);\n", + " };\n", + " document.head.appendChild(jqueryScript);\n", + " } else {\n", + " window.interactive_beam_jquery(document).ready(function($){\n", + " \n", + " });\n", + " }" + ] + }, + "metadata": {} + }, + { + "output_type": "stream", + "name": "stdout", + "text": [ + "Example: This restaurant is awesome\n", + "Actual Sentiment: positive\n", + "Distilbert Prediction: POSITIVE\n", + "Roberta Prediction: NEUTRAL\n", + "------------\n", + "Example: This restaurant is bad\n", + "Actual Sentiment: negative\n", + "Distilbert Prediction: NEGATIVE\n", + "Roberta Prediction: NEUTRAL\n", + "------------\n", + "Example: I love chocolate\n", + "Actual Sentiment: positive\n", + "Distilbert Prediction: POSITIVE\n", + "Roberta Prediction: NEUTRAL\n", + "------------\n", + "Example: I feel fine\n", + "Actual Sentiment: neutral\n", + "Distilbert Prediction: POSITIVE\n", + "Roberta Prediction: ENTAILMENT\n", + "------------\n" + ] + } + ] + } + ] +} From c01b41f9758392a00fd96965c91edea7b61d98bb Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 3 Oct 2023 15:29:30 -0400 Subject: [PATCH 012/435] Bump urllib3 from 1.26.16 to 1.26.17 in /sdks/python/container/py310 (#28785) Bumps [urllib3](https://github.com/urllib3/urllib3) from 1.26.16 to 1.26.17. - [Release notes](https://github.com/urllib3/urllib3/releases) - [Changelog](https://github.com/urllib3/urllib3/blob/main/CHANGES.rst) - [Commits](https://github.com/urllib3/urllib3/compare/1.26.16...1.26.17) --- updated-dependencies: - dependency-name: urllib3 dependency-type: direct:production ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/python/container/py310/base_image_requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/container/py310/base_image_requirements.txt b/sdks/python/container/py310/base_image_requirements.txt index e8a666e7b9d0..548c56371706 100644 --- a/sdks/python/container/py310/base_image_requirements.txt +++ b/sdks/python/container/py310/base_image_requirements.txt @@ -134,7 +134,7 @@ tqdm==4.66.1 typing_extensions==4.8.0 tzlocal==5.0.1 uritemplate==4.1.1 -urllib3==1.26.16 +urllib3==1.26.17 websocket-client==1.6.3 wrapt==1.15.0 zstandard==0.21.0 From be8053722387c5427f7c9f2c37c451a4bdce77b0 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud <65791736+ahmedabu98@users.noreply.github.com> Date: Tue, 3 Oct 2023 16:08:18 -0400 Subject: [PATCH 013/435] BigQuery testing suite that runs against BQ's day 0 region (#28397) * support creating dataset in specified region; * new gradle task * create yaml workflow * add file loads streaming test; add option to query with location * pass bq location to query operation --- ...m_PostCommit_Java_BigQueryEarlyRollout.yml | 97 +++++++++++++++++++ .../io/google-cloud-platform/build.gradle | 44 ++++++++- .../io/gcp/bigquery/TestBigQueryOptions.java | 7 ++ .../common/GcpIoPipelineOptionsRegistrar.java | 2 + .../sdk/io/gcp/testing/BigqueryClient.java | 37 ++++++- .../bigquery/BigQueryIOStorageQueryIT.java | 10 +- .../gcp/bigquery/BigQueryIOStorageReadIT.java | 9 +- .../BigQueryIOStorageReadTableRowIT.java | 43 ++++---- .../bigquery/BigQueryIOStorageWriteIT.java | 31 ++++-- .../BigQuerySchemaUpdateOptionsIT.java | 6 +- .../BigQueryTimePartitioningClusteringIT.java | 42 ++++++-- .../io/gcp/bigquery/BigQueryToTableIT.java | 9 +- .../io/gcp/bigquery/FileLoadsStreamingIT.java | 9 +- .../StorageApiDirectWriteProtosIT.java | 9 +- .../bigquery/StorageApiSinkFailedRowsIT.java | 13 ++- .../bigquery/StorageApiSinkRowUpdateIT.java | 9 +- .../StorageApiSinkSchemaUpdateIT.java | 17 ++-- .../bigquery/TableRowToStorageApiProtoIT.java | 11 ++- 18 files changed, 341 insertions(+), 64 deletions(-) create mode 100644 .github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml diff --git a/.github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml b/.github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml new file mode 100644 index 000000000000..952273e810d2 --- /dev/null +++ b/.github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml @@ -0,0 +1,97 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +name: PostCommit Java BigQueryEarlyRollout + +on: + issue_comment: + types: [created] + schedule: + - cron: '0 */6 * * *' + workflow_dispatch: + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + cancel-in-progress: true + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: write + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_PostCommit_Java_BigQueryEarlyRollout: + name: ${{matrix.job_name}} (${{matrix.job_phrase}}) + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 100 + strategy: + matrix: + job_name: [beam_PostCommit_Java_BigQueryEarlyRollout] + job_phrase: [Run Java BigQueryEarlyRollout PostCommit] + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Java BigQueryEarlyRollout PostCommit' + steps: + - uses: actions/checkout@v4 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Authenticate on GCP + uses: google-github-actions/setup-gcloud@v0 + with: + service_account_email: ${{ secrets.GCP_SA_EMAIL }} + service_account_key: ${{ secrets.GCP_SA_KEY }} + project_id: ${{ secrets.GCP_PROJECT_ID }} + export_default_credentials: true + - name: run PostCommit Java BigQueryEarlyRollout script + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:io:google-cloud-platform:bigQueryEarlyRolloutIntegrationTest + - name: Archive JUnit Test Results + uses: actions/upload-artifact@v3 + if: failure() + with: + name: JUnit Test Results + path: "**/build/reports/tests/" + - name: Publish JUnit Test Results + uses: EnricoMi/publish-unit-test-result-action@v2 + if: always() + with: + commit: '${{ env.prsha || env.GITHUB_SHA }}' + comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} + files: '**/build/test-results/**/*.xml' \ No newline at end of file diff --git a/sdks/java/io/google-cloud-platform/build.gradle b/sdks/java/io/google-cloud-platform/build.gradle index 560b27aae162..efc9ff3db9c6 100644 --- a/sdks/java/io/google-cloud-platform/build.gradle +++ b/sdks/java/io/google-cloud-platform/build.gradle @@ -202,10 +202,8 @@ task integrationTest(type: Test, dependsOn: processTestResources) { exclude '**/BigQueryIOReadIT.class' exclude '**/BigQueryIOStorageQueryIT.class' exclude '**/BigQueryIOStorageReadIT.class' - exclude '**/BigQueryIOStorageReadTableRowIT.class' exclude '**/BigQueryIOStorageWriteIT.class' exclude '**/BigQueryToTableIT.class' - exclude '**/BigQueryIOJsonTest.class' maxParallelForks 4 classpath = sourceSets.test.runtimeClasspath @@ -244,6 +242,48 @@ task integrationTestKms(type: Test) { } } +/* + Integration tests for BigQueryIO that run on BigQuery's early rollout region (us-east7) + with the intended purpose of catching breaking changes from new BigQuery releases. + If these tests fail here but not in `Java_GCP_IO_Direct`, there may be a new BigQuery change + that is breaking the connector. If this is the case, we should verify with the appropriate + BigQuery infrastructure API team. + + To test in a BigQuery location, we just need to create our datasets in that location. + */ +task bigQueryEarlyRolloutIntegrationTest(type: Test, dependsOn: processTestResources) { + group = "Verification" + def gcpProject = project.findProperty('gcpProject') ?: 'apache-beam-testing' + def gcpTempRoot = project.findProperty('gcpTempRoot') ?: 'gs://temp-storage-for-bigquery-day0-tests' + systemProperty "beamTestPipelineOptions", JsonOutput.toJson([ + "--runner=DirectRunner", + "--project=${gcpProject}", + "--tempRoot=${gcpTempRoot}", + "--bigQueryLocation=us-east7", + ]) + + outputs.upToDateWhen { false } + + // export and direct read + include '**/BigQueryToTableIT.class' + include '**/BigQueryIOJsonIT.class' + include '**/BigQueryIOStorageReadTableRowIT.class' + // storage write api + include '**/StorageApiDirectWriteProtosIT.class' + include '**/StorageApiSinkFailedRowsIT.class' + include '**/StorageApiSinkRowUpdateIT.class' + include '**/StorageApiSinkSchemaUpdateIT.class' + include '**/TableRowToStorageApiProtoIT.class' + // file loads + include '**/BigQuerySchemaUpdateOptionsIT.class' + include '**/BigQueryTimePartitioningClusteringIT.class' + include '**/FileLoadsStreamingIT.class' + + maxParallelForks 4 + classpath = sourceSets.test.runtimeClasspath + testClassesDirs = sourceSets.test.output.classesDirs +} + // path(s) for Cloud Spanner related classes def spannerIncludes = [ '**/org/apache/beam/sdk/io/gcp/spanner/**', diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TestBigQueryOptions.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TestBigQueryOptions.java index 3574c12ee3a9..4d8095c1879d 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TestBigQueryOptions.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TestBigQueryOptions.java @@ -24,10 +24,17 @@ /** {@link TestPipelineOptions} for {@link TestBigQuery}. */ public interface TestBigQueryOptions extends TestPipelineOptions, BigQueryOptions, GcpOptions { + String BIGQUERY_EARLY_ROLLOUT_REGION = "us-east7"; @Description("Dataset used in the integration tests. Default is integ_test") @Default.String("integ_test") String getTargetDataset(); void setTargetDataset(String value); + + @Description("Region to perform BigQuery operations in.") + @Default.String("") + String getBigQueryLocation(); + + void setBigQueryLocation(String location); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/common/GcpIoPipelineOptionsRegistrar.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/common/GcpIoPipelineOptionsRegistrar.java index 1ed9ed6cb6c3..f1ff827fc633 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/common/GcpIoPipelineOptionsRegistrar.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/common/GcpIoPipelineOptionsRegistrar.java @@ -20,6 +20,7 @@ import com.google.auto.service.AutoService; import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryOptions; +import org.apache.beam.sdk.io.gcp.bigquery.TestBigQueryOptions; import org.apache.beam.sdk.io.gcp.firestore.FirestoreOptions; import org.apache.beam.sdk.io.gcp.pubsub.PubsubOptions; import org.apache.beam.sdk.options.PipelineOptions; @@ -36,6 +37,7 @@ public Iterable> getPipelineOptions() { .add(BigQueryOptions.class) .add(PubsubOptions.class) .add(FirestoreOptions.class) + .add(TestBigQueryOptions.class) .build(); } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/BigqueryClient.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/BigqueryClient.java index b21fdd669596..0e9476e6a226 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/BigqueryClient.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/BigqueryClient.java @@ -292,6 +292,21 @@ private QueryResponse getTypedTableRows(QueryResponse response) { public List queryUnflattened( String query, String projectId, boolean typed, boolean useStandardSql) throws IOException, InterruptedException { + return queryUnflattened(query, projectId, typed, useStandardSql, null); + } + + /** + * Performs a query without flattening results. May choose a location (GCP region) to perform this + * operation in. + */ + @Nonnull + public List queryUnflattened( + String query, + String projectId, + boolean typed, + boolean useStandardSql, + @Nullable String location) + throws IOException, InterruptedException { Random rnd = new Random(System.currentTimeMillis()); String temporaryDatasetId = String.format("_dataflow_temporary_dataset_%s_%s", System.nanoTime(), rnd.nextInt(1000000)); @@ -302,9 +317,11 @@ public List queryUnflattened( .setDatasetId(temporaryDatasetId) .setTableId(temporaryTableId); - createNewDataset(projectId, temporaryDatasetId); + createNewDataset(projectId, temporaryDatasetId, null, location); createNewTable( - projectId, temporaryDatasetId, new Table().setTableReference(tempTableReference)); + projectId, + temporaryDatasetId, + new Table().setTableReference(tempTableReference).setLocation(location)); JobConfigurationQuery jcQuery = new JobConfigurationQuery() @@ -325,6 +342,7 @@ public List queryUnflattened( bqClient .jobs() .getQueryResults(projectId, insertedJob.getJobReference().getJobId()) + .setLocation(location) .execute(); } while (!qResponse.getJobComplete()); @@ -395,6 +413,18 @@ public void createNewDataset(String projectId, String datasetId) public void createNewDataset( String projectId, String datasetId, @Nullable Long defaultTableExpirationMs) throws IOException, InterruptedException { + createNewDataset(projectId, datasetId, defaultTableExpirationMs, null); + } + + /** + * Creates a new dataset with defaultTableExpirationMs and in a specified location (GCP region). + */ + public void createNewDataset( + String projectId, + String datasetId, + @Nullable Long defaultTableExpirationMs, + @Nullable String location) + throws IOException, InterruptedException { Sleeper sleeper = Sleeper.DEFAULT; BackOff backoff = BackOffAdapter.toGcpBackOff(BACKOFF_FACTORY.backoff()); IOException lastException = null; @@ -410,7 +440,8 @@ public void createNewDataset( projectId, new Dataset() .setDatasetReference(new DatasetReference().setDatasetId(datasetId)) - .setDefaultTableExpirationMs(defaultTableExpirationMs)) + .setDefaultTableExpirationMs(defaultTableExpirationMs) + .setLocation(location)) .execute(); if (response != null) { LOG.info("Successfully created new dataset : " + response.getId()); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageQueryIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageQueryIT.java index 692a12c0f4a7..d355d6bb9336 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageQueryIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageQueryIT.java @@ -17,6 +17,8 @@ */ package org.apache.beam.sdk.io.gcp.bigquery; +import static org.apache.beam.sdk.io.gcp.bigquery.TestBigQueryOptions.BIGQUERY_EARLY_ROLLOUT_REGION; + import java.util.Map; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; @@ -52,7 +54,13 @@ public class BigQueryIOStorageQueryIT { "1G", 11110839L, "1T", 11110839000L); - private static final String DATASET_ID = "big_query_storage"; + private static final String DATASET_ID = + TestPipeline.testingPipelineOptions() + .as(TestBigQueryOptions.class) + .getBigQueryLocation() + .equals(BIGQUERY_EARLY_ROLLOUT_REGION) + ? "big_query_storage_day0" + : "big_query_storage"; private static final String TABLE_PREFIX = "storage_read_"; private BigQueryIOStorageQueryOptions options; diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadIT.java index 570938470b9d..b4f6ddb76f72 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadIT.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.io.gcp.bigquery; +import static org.apache.beam.sdk.io.gcp.bigquery.TestBigQueryOptions.BIGQUERY_EARLY_ROLLOUT_REGION; import static org.junit.Assert.assertEquals; import com.google.cloud.bigquery.storage.v1.DataFormat; @@ -65,7 +66,13 @@ public class BigQueryIOStorageReadIT { "1T", 11110839000L, "multi_field", 11110839L); - private static final String DATASET_ID = "big_query_storage"; + private static final String DATASET_ID = + TestPipeline.testingPipelineOptions() + .as(TestBigQueryOptions.class) + .getBigQueryLocation() + .equals(BIGQUERY_EARLY_ROLLOUT_REGION) + ? "big_query_storage_day0" + : "big_query_storage"; private static final String TABLE_PREFIX = "storage_read_"; private BigQueryIOStorageReadOptions options; diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadTableRowIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadTableRowIT.java index 734c3af2c4d4..35e2676c70ef 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadTableRowIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadTableRowIT.java @@ -17,6 +17,8 @@ */ package org.apache.beam.sdk.io.gcp.bigquery; +import static org.apache.beam.sdk.io.gcp.bigquery.TestBigQueryOptions.BIGQUERY_EARLY_ROLLOUT_REGION; + import com.google.api.services.bigquery.model.TableRow; import java.util.HashSet; import java.util.Set; @@ -52,7 +54,13 @@ @RunWith(JUnit4.class) public class BigQueryIOStorageReadTableRowIT { - private static final String DATASET_ID = "big_query_import_export"; + private static final String DATASET_ID = + TestPipeline.testingPipelineOptions() + .as(TestBigQueryOptions.class) + .getBigQueryLocation() + .equals(BIGQUERY_EARLY_ROLLOUT_REGION) + ? "big_query_import_export_day0" + : "big_query_import_export"; private static final String TABLE_PREFIX = "parallel_read_table_row_"; private BigQueryIOStorageReadTableRowOptions options; @@ -67,12 +75,11 @@ public interface BigQueryIOStorageReadTableRowOptions void setInputTable(String table); } - private static class TableRowToKVPairFn extends SimpleFunction> { + private static class TableRowToKVPairFn extends SimpleFunction> { @Override - public KV apply(TableRow input) { - CharSequence sampleString = (CharSequence) input.get("sample_string"); - String key = sampleString != null ? sampleString.toString() : "null"; - return KV.of(key, BigQueryHelpers.toJsonString(input)); + public KV apply(TableRow input) { + Integer rowId = Integer.parseInt((String) input.get("id")); + return KV.of(rowId, BigQueryHelpers.toJsonString(input)); } } @@ -87,7 +94,7 @@ private void setUpTestEnvironment(String tableName) { private static void runPipeline(BigQueryIOStorageReadTableRowOptions pipelineOptions) { Pipeline pipeline = Pipeline.create(pipelineOptions); - PCollection> jsonTableRowsFromExport = + PCollection> jsonTableRowsFromExport = pipeline .apply( "ExportTable", @@ -96,7 +103,7 @@ private static void runPipeline(BigQueryIOStorageReadTableRowOptions pipelineOpt .withMethod(Method.EXPORT)) .apply("MapExportedRows", MapElements.via(new TableRowToKVPairFn())); - PCollection> jsonTableRowsFromDirectRead = + PCollection> jsonTableRowsFromDirectRead = pipeline .apply( "DirectReadTable", @@ -108,16 +115,16 @@ private static void runPipeline(BigQueryIOStorageReadTableRowOptions pipelineOpt final TupleTag exportTag = new TupleTag<>(); final TupleTag directReadTag = new TupleTag<>(); - PCollection>> unmatchedRows = + PCollection>> unmatchedRows = KeyedPCollectionTuple.of(exportTag, jsonTableRowsFromExport) .and(directReadTag, jsonTableRowsFromDirectRead) .apply(CoGroupByKey.create()) .apply( ParDo.of( - new DoFn, KV>>() { + new DoFn, KV>>() { @ProcessElement - public void processElement(ProcessContext c) throws Exception { - KV element = c.element(); + public void processElement(ProcessContext c) { + KV element = c.element(); // Add all the exported rows for the key to a collection. Set uniqueRows = new HashSet<>(); @@ -147,20 +154,20 @@ public void processElement(ProcessContext c) throws Exception { } @Test - public void testBigQueryStorageReadTableRow1() throws Exception { - setUpTestEnvironment("1"); + public void testBigQueryStorageReadTableRow100() { + setUpTestEnvironment("100"); runPipeline(options); } @Test - public void testBigQueryStorageReadTableRow10k() throws Exception { - setUpTestEnvironment("10k"); + public void testBigQueryStorageReadTableRow1k() { + setUpTestEnvironment("1K"); runPipeline(options); } @Test - public void testBigQueryStorageReadTableRow100k() throws Exception { - setUpTestEnvironment("100k"); + public void testBigQueryStorageReadTableRow10k() { + setUpTestEnvironment("10K"); runPipeline(options); } } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageWriteIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageWriteIT.java index fc3ce0be4b69..d061898d55c7 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageWriteIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageWriteIT.java @@ -26,11 +26,11 @@ import com.google.api.services.bigquery.model.TableRow; import com.google.api.services.bigquery.model.TableSchema; import java.io.IOException; +import java.security.SecureRandom; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; import org.apache.beam.sdk.io.GenerateSequence; import org.apache.beam.sdk.io.gcp.testing.BigqueryClient; -import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.MapElements; @@ -43,6 +43,8 @@ import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.joda.time.Duration; import org.joda.time.Instant; +import org.junit.AfterClass; +import org.junit.BeforeClass; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -60,24 +62,37 @@ private enum WriteMode { AT_LEAST_ONCE } - private String project; - private static final String DATASET_ID = "big_query_storage"; + private static String project; + private static final String DATASET_ID = + "big_query_storage_write_it_" + + System.currentTimeMillis() + + "_" + + new SecureRandom().nextInt(32); private static final String TABLE_PREFIX = "storage_write_"; - private BigQueryOptions bqOptions; + private static TestBigQueryOptions bqOptions; private static final BigqueryClient BQ_CLIENT = new BigqueryClient("BigQueryStorageIOWriteIT"); + @BeforeClass + public static void setup() throws Exception { + bqOptions = TestPipeline.testingPipelineOptions().as(TestBigQueryOptions.class); + project = bqOptions.as(GcpOptions.class).getProject(); + // Create one BQ dataset for all test cases. + BQ_CLIENT.createNewDataset(project, DATASET_ID, null, bqOptions.getBigQueryLocation()); + } + + @AfterClass + public static void cleanup() { + BQ_CLIENT.deleteDataset(project, DATASET_ID); + } + private void setUpTestEnvironment(WriteMode writeMode) { - PipelineOptionsFactory.register(BigQueryOptions.class); - bqOptions = TestPipeline.testingPipelineOptions().as(BigQueryOptions.class); - bqOptions.setProject(TestPipeline.testingPipelineOptions().as(GcpOptions.class).getProject()); bqOptions.setUseStorageWriteApi(true); if (writeMode == WriteMode.AT_LEAST_ONCE) { bqOptions.setUseStorageWriteApiAtLeastOnce(true); } bqOptions.setNumStorageWriteApiStreams(2); bqOptions.setStorageWriteApiTriggeringFrequencySec(1); - project = TestPipeline.testingPipelineOptions().as(GcpOptions.class).getProject(); } static class FillRowFn extends DoFn { diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySchemaUpdateOptionsIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySchemaUpdateOptionsIT.java index 611c691dca12..833a0a0829c7 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySchemaUpdateOptionsIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySchemaUpdateOptionsIT.java @@ -87,7 +87,11 @@ public class BigQuerySchemaUpdateOptionsIT { @BeforeClass public static void setupTestEnvironment() throws Exception { project = TestPipeline.testingPipelineOptions().as(GcpOptions.class).getProject(); - BQ_CLIENT.createNewDataset(project, BIG_QUERY_DATASET_ID); + BQ_CLIENT.createNewDataset( + project, + BIG_QUERY_DATASET_ID, + null, + TestPipeline.testingPipelineOptions().as(TestBigQueryOptions.class).getBigQueryLocation()); } @AfterClass diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTimePartitioningClusteringIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTimePartitioningClusteringIT.java index 3ceb6f0966b7..da5f396e8d89 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTimePartitioningClusteringIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTimePartitioningClusteringIT.java @@ -24,9 +24,11 @@ import com.google.api.services.bigquery.model.TableRow; import com.google.api.services.bigquery.model.TableSchema; import com.google.api.services.bigquery.model.TimePartitioning; +import java.security.SecureRandom; import java.util.Arrays; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; import org.apache.beam.sdk.io.gcp.testing.BigqueryClient; import org.apache.beam.sdk.options.Default; import org.apache.beam.sdk.options.Description; @@ -38,8 +40,10 @@ import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.values.ValueInSingleWindow; import org.checkerframework.checker.nullness.qual.Nullable; +import org.junit.AfterClass; import org.junit.Assert; import org.junit.Before; +import org.junit.BeforeClass; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -49,7 +53,15 @@ public class BigQueryTimePartitioningClusteringIT { private static final String WEATHER_SAMPLES_TABLE = "apache-beam-testing.samples.weather_stations"; - private static final String DATASET_NAME = "BigQueryTimePartitioningIT"; + + private static String project; + private static final BigqueryClient BQ_CLIENT = + new BigqueryClient("BigQueryTimePartitioningClusteringIT"); + private static final String DATASET_NAME = + "BigQueryTimePartitioningIT_" + + System.currentTimeMillis() + + "_" + + new SecureRandom().nextInt(32); private static final TimePartitioning TIME_PARTITIONING = new TimePartitioning().setField("date").setType("DAY"); private static final Clustering CLUSTERING = @@ -64,6 +76,16 @@ public class BigQueryTimePartitioningClusteringIT { private Bigquery bqClient; private BigQueryClusteringITOptions options; + @BeforeClass + public static void setupTestEnvironment() throws Exception { + project = TestPipeline.testingPipelineOptions().as(GcpOptions.class).getProject(); + BQ_CLIENT.createNewDataset( + project, + DATASET_NAME, + null, + TestPipeline.testingPipelineOptions().as(TestBigQueryOptions.class).getBigQueryLocation()); + } + @Before public void setUp() { PipelineOptionsFactory.register(BigQueryClusteringITOptions.class); @@ -72,6 +94,11 @@ public void setUp() { bqClient = BigqueryClient.getNewBigqueryClient(options.getAppName()); } + @AfterClass + public static void cleanup() { + BQ_CLIENT.deleteDataset(project, DATASET_NAME); + } + /** Customized PipelineOptions for BigQueryClustering Integration Test. */ public interface BigQueryClusteringITOptions extends TestPipelineOptions, ExperimentalOptions, BigQueryOptions { @@ -110,8 +137,7 @@ public ClusteredDestinations(String tableName) { @Override public TableDestination getDestination(ValueInSingleWindow element) { - return new TableDestination( - String.format("%s.%s", DATASET_NAME, tableName), null, TIME_PARTITIONING, CLUSTERING); + return new TableDestination(tableName, null, TIME_PARTITIONING, CLUSTERING); } @Override @@ -176,6 +202,7 @@ public void testE2EBigQueryClustering() throws Exception { @Test public void testE2EBigQueryClusteringTableFunction() throws Exception { String tableName = "weather_stations_clustered_table_function_" + System.currentTimeMillis(); + String destination = String.format("%s.%s", DATASET_NAME, tableName); Pipeline p = Pipeline.create(options); @@ -185,11 +212,7 @@ public void testE2EBigQueryClusteringTableFunction() throws Exception { BigQueryIO.writeTableRows() .to( (ValueInSingleWindow vsw) -> - new TableDestination( - String.format("%s.%s", DATASET_NAME, tableName), - null, - TIME_PARTITIONING, - CLUSTERING)) + new TableDestination(destination, null, TIME_PARTITIONING, CLUSTERING)) .withClustering() .withSchema(SCHEMA) .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED) @@ -206,6 +229,7 @@ public void testE2EBigQueryClusteringTableFunction() throws Exception { public void testE2EBigQueryClusteringDynamicDestinations() throws Exception { String tableName = "weather_stations_clustered_dynamic_destinations_" + System.currentTimeMillis(); + String destination = String.format("%s.%s", DATASET_NAME, tableName); Pipeline p = Pipeline.create(options); @@ -213,7 +237,7 @@ public void testE2EBigQueryClusteringDynamicDestinations() throws Exception { .apply(ParDo.of(new KeepStationNumberAndConvertDate())) .apply( BigQueryIO.writeTableRows() - .to(new ClusteredDestinations(tableName)) + .to(new ClusteredDestinations(destination)) .withClustering() .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED) .withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_TRUNCATE)); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryToTableIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryToTableIT.java index d6b7f8e16412..1abe7752b2e0 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryToTableIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryToTableIT.java @@ -46,7 +46,6 @@ import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.options.Validation; import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.testing.TestPipelineOptions; import org.apache.beam.sdk.transforms.Reshuffle; import org.apache.beam.sdk.transforms.Values; import org.apache.beam.sdk.transforms.WithKeys; @@ -214,7 +213,7 @@ private void verifyStandardQueryRes(String outputTable) throws Exception { } /** Customized PipelineOption for BigQueryToTable Pipeline. */ - public interface BigQueryToTableOptions extends TestPipelineOptions, ExperimentalOptions { + public interface BigQueryToTableOptions extends TestBigQueryOptions, ExperimentalOptions { @Description("The BigQuery query to be used for creating the source") @Validation.Required @@ -252,9 +251,11 @@ public interface BigQueryToTableOptions extends TestPipelineOptions, Experimenta @BeforeClass public static void setupTestEnvironment() throws Exception { PipelineOptionsFactory.register(BigQueryToTableOptions.class); - project = TestPipeline.testingPipelineOptions().as(GcpOptions.class).getProject(); + BigQueryToTableOptions options = + TestPipeline.testingPipelineOptions().as(BigQueryToTableOptions.class); + project = options.as(GcpOptions.class).getProject(); // Create one BQ dataset for all test cases. - BQ_CLIENT.createNewDataset(project, BIG_QUERY_DATASET_ID); + BQ_CLIENT.createNewDataset(project, BIG_QUERY_DATASET_ID, null, options.getBigQueryLocation()); // Create table and insert data for new type query test cases. BQ_CLIENT.createNewTable( diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/FileLoadsStreamingIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/FileLoadsStreamingIT.java index 012afed6fb43..678708062b8d 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/FileLoadsStreamingIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/FileLoadsStreamingIT.java @@ -106,11 +106,16 @@ public static Iterable data() { private final Random randomGenerator = new Random(); + // used when test suite specifies a particular GCP location for BigQuery operations + private static String bigQueryLocation; + @BeforeClass public static void setUpTestEnvironment() throws IOException, InterruptedException { // Create one BQ dataset for all test cases. cleanUp(); - BQ_CLIENT.createNewDataset(PROJECT, BIG_QUERY_DATASET_ID); + bigQueryLocation = + TestPipeline.testingPipelineOptions().as(TestBigQueryOptions.class).getBigQueryLocation(); + BQ_CLIENT.createNewDataset(PROJECT, BIG_QUERY_DATASET_ID, null, bigQueryLocation); } @AfterClass @@ -293,7 +298,7 @@ private static void checkRowCompleteness( throws IOException, InterruptedException { List actualTableRows = BQ_CLIENT.queryUnflattened( - String.format("SELECT * FROM [%s]", tableSpec), PROJECT, true, false); + String.format("SELECT * FROM [%s]", tableSpec), PROJECT, true, false, bigQueryLocation); Schema rowSchema = BigQueryUtils.fromTableSchema(schema); List actualBeamRows = diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDirectWriteProtosIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDirectWriteProtosIT.java index 93bc4162409f..3da93c42a480 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDirectWriteProtosIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDirectWriteProtosIT.java @@ -80,10 +80,15 @@ private BigQueryIO.Write.Method getMethod() { : BigQueryIO.Write.Method.STORAGE_WRITE_API; } + // used when test suite specifies a particular GCP location for BigQuery operations + private static String bigQueryLocation; + @BeforeClass public static void setUpTestEnvironment() throws IOException, InterruptedException { // Create one BQ dataset for all test cases. - BQ_CLIENT.createNewDataset(PROJECT, BIG_QUERY_DATASET_ID); + bigQueryLocation = + TestPipeline.testingPipelineOptions().as(TestBigQueryOptions.class).getBigQueryLocation(); + BQ_CLIENT.createNewDataset(PROJECT, BIG_QUERY_DATASET_ID, null, bigQueryLocation); } @AfterClass @@ -191,7 +196,7 @@ public void testDirectWriteProtos() throws Exception { void assertRowsWritten(String tableSpec, Iterable expectedItems) throws Exception { List rows = BQ_CLIENT.queryUnflattened( - String.format("SELECT * FROM %s", tableSpec), PROJECT, true, true); + String.format("SELECT * FROM %s", tableSpec), PROJECT, true, true, bigQueryLocation); assertThat(rows, containsInAnyOrder(Iterables.toArray(expectedItems, TableRow.class))); } } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiSinkFailedRowsIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiSinkFailedRowsIT.java index 3dcde8f39cd7..f721f57147e3 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiSinkFailedRowsIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiSinkFailedRowsIT.java @@ -108,10 +108,15 @@ private BigQueryIO.Write.Method getMethod() { : BigQueryIO.Write.Method.STORAGE_WRITE_API; } + // used when test suite specifies a particular GCP location for BigQuery operations + private static String bigQueryLocation; + @BeforeClass public static void setUpTestEnvironment() throws IOException, InterruptedException { // Create one BQ dataset for all test cases. - BQ_CLIENT.createNewDataset(PROJECT, BIG_QUERY_DATASET_ID); + bigQueryLocation = + TestPipeline.testingPipelineOptions().as(TestBigQueryOptions.class).getBigQueryLocation(); + BQ_CLIENT.createNewDataset(PROJECT, BIG_QUERY_DATASET_ID, null, bigQueryLocation); } @AfterClass @@ -217,7 +222,11 @@ private void assertGoodRowsWritten(String tableSpec, Iterable goodRows TableRow queryResponse = Iterables.getOnlyElement( BQ_CLIENT.queryUnflattened( - String.format("SELECT COUNT(*) FROM %s", tableSpec), PROJECT, true, true)); + String.format("SELECT COUNT(*) FROM `%s`", tableSpec), + PROJECT, + true, + true, + bigQueryLocation)); int numRowsWritten = Integer.parseInt((String) queryResponse.get("f0_")); if (useAtLeastOnce) { assertThat(numRowsWritten, Matchers.greaterThanOrEqualTo(Iterables.size(goodRows))); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiSinkRowUpdateIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiSinkRowUpdateIT.java index d5366fe29613..f8cc797a87cd 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiSinkRowUpdateIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiSinkRowUpdateIT.java @@ -49,10 +49,15 @@ public class StorageApiSinkRowUpdateIT { private static final String BIG_QUERY_DATASET_ID = "storage_api_sink_rows_update" + System.nanoTime(); + // used when test suite specifies a particular GCP location for BigQuery operations + private static String bigQueryLocation; + @BeforeClass public static void setUpTestEnvironment() throws IOException, InterruptedException { // Create one BQ dataset for all test cases. - BQ_CLIENT.createNewDataset(PROJECT, BIG_QUERY_DATASET_ID); + bigQueryLocation = + TestPipeline.testingPipelineOptions().as(TestBigQueryOptions.class).getBigQueryLocation(); + BQ_CLIENT.createNewDataset(PROJECT, BIG_QUERY_DATASET_ID, null, bigQueryLocation); } @AfterClass @@ -129,7 +134,7 @@ private void assertRowsWritten(String tableSpec, Iterable expected) throws IOException, InterruptedException { List queryResponse = BQ_CLIENT.queryUnflattened( - String.format("SELECT * FROM %s", tableSpec), PROJECT, true, true); + String.format("SELECT * FROM %s", tableSpec), PROJECT, true, true, bigQueryLocation); assertThat(queryResponse, containsInAnyOrder(Iterables.toArray(expected, TableRow.class))); } } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiSinkSchemaUpdateIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiSinkSchemaUpdateIT.java index 6931b7ac9b98..bc99a4f50f70 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiSinkSchemaUpdateIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiSinkSchemaUpdateIT.java @@ -121,17 +121,21 @@ public static Iterable data() { // an updated schema. If that happens consistently, just increase these two numbers // to give it more time. // Total number of rows written to the sink - private static final int TOTAL_N = 60; + private static final int TOTAL_N = 70; // Number of rows with the original schema - private static final int ORIGINAL_N = 50; + private static final int ORIGINAL_N = 60; private final Random randomGenerator = new Random(); + // used when test suite specifies a particular GCP location for BigQuery operations + private static String bigQueryLocation; + @BeforeClass public static void setUpTestEnvironment() throws IOException, InterruptedException { // Create one BQ dataset for all test cases. - LOG.info("Creating dataset {}.", BIG_QUERY_DATASET_ID); - BQ_CLIENT.createNewDataset(PROJECT, BIG_QUERY_DATASET_ID); + bigQueryLocation = + TestPipeline.testingPipelineOptions().as(TestBigQueryOptions.class).getBigQueryLocation(); + BQ_CLIENT.createNewDataset(PROJECT, BIG_QUERY_DATASET_ID, null, bigQueryLocation); } @AfterClass @@ -459,7 +463,8 @@ private static void checkRowCompleteness( String.format("SELECT COUNT(DISTINCT(id)), COUNT(id) FROM [%s]", tableSpec), PROJECT, true, - false)); + false, + bigQueryLocation)); int distinctCount = Integer.parseInt((String) queryResponse.get("f0_")); int totalCount = Integer.parseInt((String) queryResponse.get("f1_")); @@ -479,7 +484,7 @@ public void checkRowsWithUpdatedSchema( throws IOException, InterruptedException { List actualRows = BQ_CLIENT.queryUnflattened( - String.format("SELECT * FROM [%s]", tableSpec), PROJECT, true, false); + String.format("SELECT * FROM [%s]", tableSpec), PROJECT, true, false, bigQueryLocation); for (TableRow row : actualRows) { // Rows written to the table should not have the extra field if diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowToStorageApiProtoIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowToStorageApiProtoIT.java index 218aa7411414..f28ae588a5ec 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowToStorageApiProtoIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowToStorageApiProtoIT.java @@ -318,10 +318,15 @@ public class TableRowToStorageApiProtoIT { .setFields(BASE_TABLE_SCHEMA.getFields())) .build()); + // used when test suite specifies a particular GCP location for BigQuery operations + private static String bigQueryLocation; + @BeforeClass public static void setUpTestEnvironment() throws IOException, InterruptedException { // Create one BQ dataset for all test cases. - BQ_CLIENT.createNewDataset(PROJECT, BIG_QUERY_DATASET_ID); + bigQueryLocation = + TestPipeline.testingPipelineOptions().as(TestBigQueryOptions.class).getBigQueryLocation(); + BQ_CLIENT.createNewDataset(PROJECT, BIG_QUERY_DATASET_ID, null, bigQueryLocation); } @AfterClass @@ -338,7 +343,7 @@ public void testBaseTableRow() throws IOException, InterruptedException { List actualTableRows = BQ_CLIENT.queryUnflattened( - String.format("SELECT * FROM %s", tableSpec), PROJECT, true, true); + String.format("SELECT * FROM %s", tableSpec), PROJECT, true, true, bigQueryLocation); assertEquals(1, actualTableRows.size()); assertEquals(BASE_TABLE_ROW_EXPECTED, actualTableRows.get(0)); @@ -364,7 +369,7 @@ public void testNestedRichTypesAndNull() throws IOException, InterruptedExceptio List actualTableRows = BQ_CLIENT.queryUnflattened( - String.format("SELECT * FROM %s", tableSpec), PROJECT, true, true); + String.format("SELECT * FROM %s", tableSpec), PROJECT, true, true, bigQueryLocation); assertEquals(1, actualTableRows.size()); assertEquals(BASE_TABLE_ROW_EXPECTED, actualTableRows.get(0).get("nestedValue1")); From f6b43ce1736250c044ad90305351662bb95533a7 Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Tue, 3 Oct 2023 18:57:58 -0400 Subject: [PATCH 014/435] Revert "assign highmem runner to beam_PostCommit_Python and to beam_PreCommit_Java_GCP_IO_Direct (#28719)" (#28804) This reverts commit a4ee8548424b335cb3ffdf6aeda899fd1e3ba8ad. --- .github/workflows/beam_PostCommit_Python.yml | 2 +- .github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/beam_PostCommit_Python.yml b/.github/workflows/beam_PostCommit_Python.yml index a7a214c7c5a9..6f4bc5e2ef0b 100644 --- a/.github/workflows/beam_PostCommit_Python.yml +++ b/.github/workflows/beam_PostCommit_Python.yml @@ -53,7 +53,7 @@ env: jobs: beam_PostCommit_Python: name: ${{matrix.job_name}} (${{matrix.job_phrase}} ${{matrix.python_version}}) - runs-on: [self-hosted, ubuntu-20.04, highmem] + runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 240 strategy: fail-fast: false diff --git a/.github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml index 7c821a024742..30e8d6d6c33c 100644 --- a/.github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml @@ -90,7 +90,7 @@ jobs: github.event_name == 'schedule' || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Java_GCP_IO_Direct PreCommit' - runs-on: [self-hosted, ubuntu-20.04, highmem] + runs-on: [self-hosted, ubuntu-20.04, main] steps: - uses: actions/checkout@v4 - name: Setup repository From 95af465f0dfaa123364e5880bccc3dff32a19b39 Mon Sep 17 00:00:00 2001 From: brucearctor <5032356+brucearctor@users.noreply.github.com> Date: Wed, 4 Oct 2023 05:30:15 -0700 Subject: [PATCH 015/435] yaml label (#28813) --- .github/autolabeler.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.github/autolabeler.yml b/.github/autolabeler.yml index 5a8a22044da4..d1cc8296d303 100644 --- a/.github/autolabeler.yml +++ b/.github/autolabeler.yml @@ -31,6 +31,7 @@ python: ["sdks/python/**/*", "learning/katas/python/**/*"] typescript: ["sdks/typescript/**/*"] vendor: ["vendor/**/*"] website: ["website/**/*"] +yaml: ["sdks/python/apache_beam/yaml/**"] # Extensions extensions: ["sdks/java/extensions/**/*", "runners/extensions-java/**/*"] From a9684c5563dc9b5779cbe874d1fead05de2cf36c Mon Sep 17 00:00:00 2001 From: Alexey Romanenko Date: Wed, 4 Oct 2023 14:58:16 +0200 Subject: [PATCH 016/435] TPC-DS: Fix CSV format --- .../java/org/apache/beam/sdk/tpcds/SqlTransformRunner.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/SqlTransformRunner.java b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/SqlTransformRunner.java index e8b85f63b36a..6570b7fe81b2 100644 --- a/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/SqlTransformRunner.java +++ b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/SqlTransformRunner.java @@ -283,7 +283,8 @@ public static void runUsingSqlTransform(String[] args) throws Exception { // Make an array of pipelines, each pipeline is responsible for running a corresponding query. Pipeline[] pipelines = new Pipeline[queryNames.length]; - CSVFormat csvFormat = CSVFormat.MYSQL.withDelimiter('|').withNullString(""); + CSVFormat csvFormat = + CSVFormat.MYSQL.withDelimiter('|').withTrailingDelimiter().withNullString(""); // Execute all queries, transform each result into a PCollection, write them into // the txt file and store in a GCP directory. From 9efdf6be61e46d97b51161ea934706a6a18e9118 Mon Sep 17 00:00:00 2001 From: nancyxu123 Date: Wed, 4 Oct 2023 08:50:59 -0700 Subject: [PATCH 017/435] Fix SpannerChangeStreamErrorTest.java and stop disabling tests (#28751) * Fix SpannerChangeStreamErrorTest.java and stop disabling tests * More fixes to existing error tests * Formatting changes * Make the errors a flexible retry --- .../SpannerChangeStreamErrorTest.java | 181 +++++++++++++----- 1 file changed, 138 insertions(+), 43 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/SpannerChangeStreamErrorTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/SpannerChangeStreamErrorTest.java index bf2ccd454bb5..9ffa61c93078 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/SpannerChangeStreamErrorTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/SpannerChangeStreamErrorTest.java @@ -52,7 +52,9 @@ import com.google.spanner.v1.TypeCode; import io.grpc.Status; import java.io.Serializable; +import java.util.ArrayList; import java.util.Collections; +import java.util.List; import org.apache.beam.runners.direct.DirectOptions; import org.apache.beam.runners.direct.DirectRunner; import org.apache.beam.sdk.Pipeline; @@ -68,7 +70,6 @@ import org.joda.time.Duration; import org.junit.After; import org.junit.Before; -import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; @@ -114,16 +115,22 @@ public void tearDown() throws NoSuchFieldException, IllegalAccessException { } @Test - @Ignore("BEAM-12164 Reenable this test when databaseClient.getDialect returns the right message.") - public void testResourceExhaustedDoesNotRetry() { + // Error code UNAVAILABLE is retried repeatedly until the RPC times out. + public void testUnavailableExceptionRetries() throws InterruptedException { + DirectOptions options = PipelineOptionsFactory.as(DirectOptions.class); + options.setBlockOnRun(false); + options.setRunner(DirectRunner.class); + Pipeline nonBlockingPipeline = TestPipeline.create(options); + mockSpannerService.setExecuteStreamingSqlExecutionTime( - SimulatedExecutionTime.ofStickyException(Status.RESOURCE_EXHAUSTED.asRuntimeException())); + SimulatedExecutionTime.ofStickyException(Status.UNAVAILABLE.asRuntimeException())); final Timestamp startTimestamp = Timestamp.ofTimeSecondsAndNanos(0, 1000); final Timestamp endTimestamp = Timestamp.ofTimeSecondsAndNanos(startTimestamp.getSeconds(), startTimestamp.getNanos() + 1); + try { - pipeline.apply( + nonBlockingPipeline.apply( SpannerIO.readChangeStream() .withSpannerConfig(getSpannerConfig()) .withChangeStreamName(TEST_CHANGE_STREAM) @@ -131,33 +138,36 @@ public void testResourceExhaustedDoesNotRetry() { .withMetadataTable(TEST_TABLE) .withInclusiveStartAt(startTimestamp) .withInclusiveEndAt(endTimestamp)); - pipeline.run().waitUntilFinish(); + PipelineResult result = nonBlockingPipeline.run(); + while (result.getState() != RUNNING) { + Thread.sleep(50); + } + // The pipeline continues making requests to Spanner to retry the Unavailable errors. + assertNull(result.waitUntilFinish(Duration.millis(500))); } finally { - thrown.expect(SpannerException.class); // databaseClient.getDialect does not currently bubble up the correct message. // Instead, the error returned is: "DEADLINE_EXCEEDED: Operation did not complete " // "in the given time" - thrown.expectMessage("RESOURCE_EXHAUSTED - Statement: 'SELECT 'POSTGRESQL' AS DIALECT"); + thrown.expectMessage("DEADLINE_EXCEEDED"); assertThat( mockSpannerService.countRequestsOfType(ExecuteSqlRequest.class), Matchers.equalTo(0)); } } @Test - @Ignore("BEAM-12164 Reenable this test when databaseClient.getDialect returns the right message.") - public void testUnavailableExceptionRetries() throws InterruptedException { + // Error code ABORTED is retried repeatedly until it times out. + public void testAbortedExceptionRetries() throws InterruptedException { + mockSpannerService.setExecuteStreamingSqlExecutionTime( + SimulatedExecutionTime.ofStickyException(Status.ABORTED.asRuntimeException())); + DirectOptions options = PipelineOptionsFactory.as(DirectOptions.class); options.setBlockOnRun(false); options.setRunner(DirectRunner.class); Pipeline nonBlockingPipeline = TestPipeline.create(options); - mockSpannerService.setExecuteStreamingSqlExecutionTime( - SimulatedExecutionTime.ofStickyException(Status.UNAVAILABLE.asRuntimeException())); - final Timestamp startTimestamp = Timestamp.ofTimeSecondsAndNanos(0, 1000); final Timestamp endTimestamp = Timestamp.ofTimeSecondsAndNanos(startTimestamp.getSeconds(), startTimestamp.getNanos() + 1); - try { nonBlockingPipeline.apply( SpannerIO.readChangeStream() @@ -171,23 +181,20 @@ public void testUnavailableExceptionRetries() throws InterruptedException { while (result.getState() != RUNNING) { Thread.sleep(50); } - // The pipeline continues making requests to Spanner to retry the Unavailable errors. + // The pipeline continues making requests to Spanner to retry the Aborted errors. assertNull(result.waitUntilFinish(Duration.millis(500))); } finally { - // databaseClient.getDialect does not currently bubble up the correct message. - // Instead, the error returned is: "DEADLINE_EXCEEDED: Operation did not complete " - // "in the given time" - thrown.expectMessage("UNAVAILABLE - Statement: 'SELECT 'POSTGRESQL' AS DIALECT"); + thrown.expectMessage("DEADLINE_EXCEEDED"); assertThat( mockSpannerService.countRequestsOfType(ExecuteSqlRequest.class), Matchers.equalTo(0)); } } @Test - @Ignore("BEAM-12164 Reenable this test when databaseClient.getDialect returns the right message.") - public void testAbortedExceptionNotRetried() { + // Error code UNKNOWN is not retried. + public void testUnknownExceptionDoesNotRetry() { mockSpannerService.setExecuteStreamingSqlExecutionTime( - SimulatedExecutionTime.ofStickyException(Status.ABORTED.asRuntimeException())); + SimulatedExecutionTime.ofStickyException(Status.UNKNOWN.asRuntimeException())); final Timestamp startTimestamp = Timestamp.ofTimeSecondsAndNanos(0, 1000); final Timestamp endTimestamp = @@ -204,19 +211,43 @@ public void testAbortedExceptionNotRetried() { pipeline.run().waitUntilFinish(); } finally { thrown.expect(SpannerException.class); - // databaseClient.getDialect does not currently bubble up the correct message. - // Instead, the error returned is: "DEADLINE_EXCEEDED: Operation did not complete " - // "in the given time" - thrown.expectMessage("ABORTED - Statement: 'SELECT 'POSTGRESQL' AS DIALECT"); + thrown.expectMessage("UNKNOWN"); assertThat( mockSpannerService.countRequestsOfType(ExecuteSqlRequest.class), Matchers.equalTo(0)); } } @Test - public void testAbortedExceptionNotRetriedithDefaultsForStreamSqlRetrySettings() { + // Error code RESOURCE_EXHAUSTED is retried repeatedly. + public void testResourceExhaustedRetry() { mockSpannerService.setExecuteStreamingSqlExecutionTime( - SimulatedExecutionTime.ofStickyException(Status.ABORTED.asRuntimeException())); + SimulatedExecutionTime.ofStickyException(Status.RESOURCE_EXHAUSTED.asRuntimeException())); + + final Timestamp startTimestamp = Timestamp.ofTimeSecondsAndNanos(0, 1000); + final Timestamp endTimestamp = + Timestamp.ofTimeSecondsAndNanos(startTimestamp.getSeconds(), startTimestamp.getNanos() + 1); + + try { + pipeline.apply( + SpannerIO.readChangeStream() + .withSpannerConfig(getSpannerConfig()) + .withChangeStreamName(TEST_CHANGE_STREAM) + .withMetadataDatabase(TEST_DATABASE) + .withMetadataTable(TEST_TABLE) + .withInclusiveStartAt(startTimestamp) + .withInclusiveEndAt(endTimestamp)); + pipeline.run().waitUntilFinish(); + } finally { + thrown.expectMessage("DEADLINE_EXCEEDED"); + assertThat( + mockSpannerService.countRequestsOfType(ExecuteSqlRequest.class), Matchers.equalTo(0)); + } + } + + @Test + public void testResourceExhaustedRetryWithDefaultSettings() { + mockSpannerService.setExecuteStreamingSqlExecutionTime( + SimulatedExecutionTime.ofStickyException(Status.RESOURCE_EXHAUSTED.asRuntimeException())); final Timestamp startTimestamp = Timestamp.ofTimeSecondsAndNanos(0, 1000); final Timestamp endTimestamp = @@ -230,6 +261,7 @@ public void testAbortedExceptionNotRetriedithDefaultsForStreamSqlRetrySettings() .withProjectId(TEST_PROJECT) .withInstanceId(TEST_INSTANCE) .withDatabaseId(TEST_DATABASE); + try { pipeline.apply( SpannerIO.readChangeStream() @@ -241,24 +273,34 @@ public void testAbortedExceptionNotRetriedithDefaultsForStreamSqlRetrySettings() .withInclusiveEndAt(endTimestamp)); pipeline.run().waitUntilFinish(); } finally { - // databaseClient.getDialect does not currently bubble up the correct message. - // Instead, the error returned is: "DEADLINE_EXCEEDED: Operation did not complete " - // "in the given time" thrown.expect(SpannerException.class); - thrown.expectMessage("ABORTED - Statement: 'SELECT 'POSTGRESQL' AS DIALECT"); + thrown.expectMessage("RESOURCE_EXHAUSTED"); assertThat( mockSpannerService.countRequestsOfType(ExecuteSqlRequest.class), Matchers.equalTo(0)); } } @Test - public void testUnknownExceptionDoesNotRetry() { - mockSpannerService.setExecuteStreamingSqlExecutionTime( - SimulatedExecutionTime.ofStickyException(Status.UNKNOWN.asRuntimeException())); - + public void testInvalidRecordReceived() { final Timestamp startTimestamp = Timestamp.ofTimeSecondsAndNanos(0, 1000); final Timestamp endTimestamp = Timestamp.ofTimeSecondsAndNanos(startTimestamp.getSeconds(), startTimestamp.getNanos() + 1); + + mockGetDialect(); + mockTableExists(); + mockGetWatermark(startTimestamp); + ResultSet getPartitionResultSet = mockGetParentPartition(startTimestamp, endTimestamp); + mockGetPartitionsAfter( + Timestamp.ofTimeSecondsAndNanos(startTimestamp.getSeconds(), startTimestamp.getNanos() - 1), + getPartitionResultSet); + mockGetPartitionsAfter( + Timestamp.ofTimeSecondsAndNanos(startTimestamp.getSeconds(), startTimestamp.getNanos()), + ResultSet.newBuilder().setMetadata(PARTITION_METADATA_RESULT_SET_METADATA).build()); + mockGetPartitionsAfter( + Timestamp.ofTimeSecondsAndNanos(startTimestamp.getSeconds(), startTimestamp.getNanos() + 1), + ResultSet.newBuilder().setMetadata(PARTITION_METADATA_RESULT_SET_METADATA).build()); + mockInvalidChangeStreamRecordReceived(startTimestamp, endTimestamp); + try { pipeline.apply( SpannerIO.readChangeStream() @@ -271,15 +313,16 @@ public void testUnknownExceptionDoesNotRetry() { pipeline.run().waitUntilFinish(); } finally { thrown.expect(SpannerException.class); - thrown.expectMessage("UNKNOWN - Statement: 'SELECT 'POSTGRESQL' AS DIALECT"); + // DatabaseClient.getDialect returns "DEADLINE_EXCEEDED: Operation did not complete in the " + // given time" even though we mocked it out. + thrown.expectMessage("DEADLINE_EXCEEDED"); assertThat( mockSpannerService.countRequestsOfType(ExecuteSqlRequest.class), Matchers.equalTo(0)); } } @Test - @Ignore("BEAM-12164 Reenable this test when databaseClient.getDialect works.") - public void testInvalidRecordReceived() { + public void testInvalidRecordReceivedWithDefaultSettings() { final Timestamp startTimestamp = Timestamp.ofTimeSecondsAndNanos(0, 1000); final Timestamp endTimestamp = Timestamp.ofTimeSecondsAndNanos(startTimestamp.getSeconds(), startTimestamp.getNanos() + 1); @@ -288,6 +331,8 @@ public void testInvalidRecordReceived() { mockTableExists(); mockGetWatermark(startTimestamp); ResultSet getPartitionResultSet = mockGetParentPartition(startTimestamp, endTimestamp); + mockchangePartitionState(startTimestamp, endTimestamp, "CREATED"); + mockchangePartitionState(startTimestamp, endTimestamp, "SCHEDULED"); mockGetPartitionsAfter( Timestamp.ofTimeSecondsAndNanos(startTimestamp.getSeconds(), startTimestamp.getNanos() - 1), getPartitionResultSet); @@ -300,9 +345,26 @@ public void testInvalidRecordReceived() { mockInvalidChangeStreamRecordReceived(startTimestamp, endTimestamp); try { + RetrySettings quickRetrySettings = + RetrySettings.newBuilder() + .setInitialRetryDelay(org.threeten.bp.Duration.ofMillis(250)) + .setMaxRetryDelay(org.threeten.bp.Duration.ofSeconds(1)) + .setRetryDelayMultiplier(5) + .setTotalTimeout(org.threeten.bp.Duration.ofSeconds(1)) + .build(); + final SpannerConfig changeStreamConfig = + SpannerConfig.create() + .withEmulatorHost(StaticValueProvider.of(SPANNER_HOST)) + .withIsLocalChannelProvider(StaticValueProvider.of(true)) + .withCommitRetrySettings(quickRetrySettings) + .withExecuteStreamingSqlRetrySettings(null) + .withProjectId(TEST_PROJECT) + .withInstanceId(TEST_INSTANCE) + .withDatabaseId(TEST_DATABASE); + pipeline.apply( SpannerIO.readChangeStream() - .withSpannerConfig(getSpannerConfig()) + .withSpannerConfig(changeStreamConfig) .withChangeStreamName(TEST_CHANGE_STREAM) .withMetadataDatabase(TEST_DATABASE) .withMetadataTable(TEST_TABLE) @@ -311,11 +373,9 @@ public void testInvalidRecordReceived() { pipeline.run().waitUntilFinish(); } finally { thrown.expect(PipelineExecutionException.class); - // DatabaseClient.getDialect returns "DEADLINE_EXCEEDED: Operation did not complete in the " - // given time" even though we mocked it out. thrown.expectMessage("Field not found"); assertThat( - mockSpannerService.countRequestsOfType(ExecuteSqlRequest.class), Matchers.equalTo(0)); + mockSpannerService.countRequestsOfType(ExecuteSqlRequest.class), Matchers.greaterThan(0)); } } @@ -487,6 +547,41 @@ private void mockTableExists() { StatementResult.query(tableExistsStatement, tableExistsResultSet)); } + private ResultSet mockchangePartitionState( + Timestamp startTimestamp, Timestamp after3Seconds, String state) { + List tokens = new ArrayList<>(); + tokens.add("Parent0"); + Statement getPartitionStatement = + Statement.newBuilder( + "SELECT * FROM my-metadata-table WHERE PartitionToken IN UNNEST(@partitionTokens) AND State = @state") + .bind("partitionTokens") + .toStringArray(tokens) + .bind("state") + .to(state) + .build(); + ResultSet getPartitionResultSet = + ResultSet.newBuilder() + .addRows( + ListValue.newBuilder() + .addValues(Value.newBuilder().setStringValue("Parent0")) + .addValues(Value.newBuilder().setListValue(ListValue.newBuilder().build())) + .addValues(Value.newBuilder().setStringValue(startTimestamp.toString())) + .addValues(Value.newBuilder().setStringValue(after3Seconds.toString())) + .addValues(Value.newBuilder().setStringValue("500")) + .addValues(Value.newBuilder().setStringValue(State.CREATED.name())) + .addValues(Value.newBuilder().setStringValue(startTimestamp.toString())) + .addValues(Value.newBuilder().setStringValue(startTimestamp.toString())) + .addValues(Value.newBuilder().setNullValue(NullValue.NULL_VALUE).build()) + .addValues(Value.newBuilder().setNullValue(NullValue.NULL_VALUE).build()) + .addValues(Value.newBuilder().setNullValue(NullValue.NULL_VALUE).build()) + .build()) + .setMetadata(PARTITION_METADATA_RESULT_SET_METADATA) + .build(); + mockSpannerService.putStatementResult( + StatementResult.query(getPartitionStatement, getPartitionResultSet)); + return getPartitionResultSet; + } + private void mockGetDialect() { Statement determineDialectStatement = Statement.newBuilder( From 3cae5ada4f6e50b4763801232d2ede7f8193d0f6 Mon Sep 17 00:00:00 2001 From: caneff Date: Wed, 4 Oct 2023 12:13:29 -0400 Subject: [PATCH 018/435] Fix dtype usage (#28797) --- sdks/python/apache_beam/dataframe/frames.py | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/sdks/python/apache_beam/dataframe/frames.py b/sdks/python/apache_beam/dataframe/frames.py index a74ccbba041a..7330feb4433e 100644 --- a/sdks/python/apache_beam/dataframe/frames.py +++ b/sdks/python/apache_beam/dataframe/frames.py @@ -4594,8 +4594,9 @@ def wrapper(self, *args, **kwargs): return _unliftable_agg(meth)(self, *args, **kwargs) to_group = self._ungrouped.proxy().index - is_categorical_grouping = any(to_group.get_level_values(i).is_categorical() - for i in self._grouping_indexes) + is_categorical_grouping = any( + isinstance(to_group.get_level_values(i).dtype, pd.CategoricalDtype) + for i in self._grouping_indexes) groupby_kwargs = self._kwargs group_keys = self._group_keys @@ -4647,8 +4648,9 @@ def wrapper(self, *args, **kwargs): to_group = self._ungrouped.proxy().index group_keys = self._group_keys - is_categorical_grouping = any(to_group.get_level_values(i).is_categorical() - for i in self._grouping_indexes) + is_categorical_grouping = any( + isinstance(to_group.get_level_values(i).dtype, pd.CategoricalDtype) + for i in self._grouping_indexes) groupby_kwargs = self._kwargs project = _maybe_project_func(self._projection) From 9ed9c059fec6e2520b091446898365ad976d053c Mon Sep 17 00:00:00 2001 From: caneff Date: Wed, 4 Oct 2023 12:15:17 -0400 Subject: [PATCH 019/435] Fix method argument handling for pandas 2.1 (#28816) --- sdks/python/apache_beam/dataframe/frames.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/dataframe/frames.py b/sdks/python/apache_beam/dataframe/frames.py index 7330feb4433e..f51814e7f696 100644 --- a/sdks/python/apache_beam/dataframe/frames.py +++ b/sdks/python/apache_beam/dataframe/frames.py @@ -1388,7 +1388,7 @@ def align(self, other, join, axis, level, method, **kwargs): Only the default, ``method=None``, is allowed.""" if level is not None: raise NotImplementedError('per-level align') - if method is not None: + if method is not None and method != lib.no_default: raise frame_base.WontImplementError( f"align(method={method!r}) is not supported because it is " "order sensitive. Only align(method=None) is supported.", @@ -2580,7 +2580,7 @@ def align(self, other, join, axis, copy, level, method, **kwargs): "align(copy=False) is not supported because it might be an inplace " "operation depending on the data. Please prefer the default " "align(copy=True).") - if method is not None: + if method is not None and method != lib.no_default: raise frame_base.WontImplementError( f"align(method={method!r}) is not supported because it is " "order sensitive. Only align(method=None) is supported.", From 659b22089e64957f658b9c92f9e94c6b23708bd0 Mon Sep 17 00:00:00 2001 From: Danny McCormick Date: Wed, 4 Oct 2023 13:07:45 -0400 Subject: [PATCH 020/435] Fix arm tests on Python PostCommit (#28740) * Longer arm timeout * Use custom container * restore workflow * Set up buildx * Fix container formatting * Use default creds * Finish args with backslash --- .../workflows/beam_PostCommit_Python_Arm.yml | 22 +++++++++++++++---- sdks/python/scripts/run_integration_test.sh | 9 ++++++++ .../python/test-suites/dataflow/common.gradle | 3 +++ 3 files changed, 30 insertions(+), 4 deletions(-) diff --git a/.github/workflows/beam_PostCommit_Python_Arm.yml b/.github/workflows/beam_PostCommit_Python_Arm.yml index a77c4e96dc51..8be303a82d1d 100644 --- a/.github/workflows/beam_PostCommit_Python_Arm.yml +++ b/.github/workflows/beam_PostCommit_Python_Arm.yml @@ -18,10 +18,10 @@ name: PostCommit Python Arm on: - # issue_comment: - # types: [created] - # schedule: - # - cron: '0 */6 * * *' + issue_comment: + types: [created] + schedule: + - cron: '0 */6 * * *' workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs @@ -81,12 +81,20 @@ jobs: run: | sudo curl -L https://github.com/docker/compose/releases/download/1.22.0/docker-compose-$(uname -s)-$(uname -m) -o /usr/local/bin/docker-compose sudo chmod +x /usr/local/bin/docker-compose + - name: Set up Docker Buildx + uses: docker/setup-buildx-action@v2 + - name: GCloud Docker credential helper + run: | + gcloud auth configure-docker us.gcr.io - name: Set PY_VER_CLEAN id: set_py_ver_clean run: | PY_VER=${{ matrix.python_version }} PY_VER_CLEAN=${PY_VER//.} echo "py_ver_clean=$PY_VER_CLEAN" >> $GITHUB_OUTPUT + - name: Generate TAG unique variable based on timestamp + id: set_tag + run: echo "TAG=$(date +'%Y%m%d-%H%M%S%N')" >> $GITHUB_OUTPUT - name: run PostCommit Python ${{ matrix.python_version }} script uses: ./.github/actions/gradle-command-self-hosted-action with: @@ -94,8 +102,14 @@ jobs: arguments: | -PuseWheelDistribution \ -PpythonVersion=${{ matrix.python_version }} \ + -Pcontainer-architecture-list=arm64,amd64 \ + -Pdocker-repository-root=us.gcr.io/apache-beam-testing/github-actions \ + -Pdocker-tag=${{ steps.set_tag.outputs.TAG }} \ + -Ppush-containers \ env: CLOUDSDK_CONFIG: ${{ env.KUBELET_GCLOUD_CONFIG_PATH}} + MULTIARCH_TAG: ${{ steps.set_tag.outputs.TAG }} + USER: github-actions - name: Archive code coverage results uses: actions/upload-artifact@v3 with: diff --git a/sdks/python/scripts/run_integration_test.sh b/sdks/python/scripts/run_integration_test.sh index 6ad592080ae2..5ac3627a0960 100755 --- a/sdks/python/scripts/run_integration_test.sh +++ b/sdks/python/scripts/run_integration_test.sh @@ -79,6 +79,7 @@ SUITE="" COLLECT_MARKERS= REQUIREMENTS_FILE="" ARCH="" +PY_VERSION="" # Default test (pytest) options. # Run WordCountIT.test_wordcount_it by default if no test options are @@ -169,6 +170,11 @@ case $key in shift # past argument shift # past value ;; + --py_version) + PY_VERSION="$2" + shift # past argument + shift # past value + ;; *) # unknown option echo "Unknown option: $1" exit 1 @@ -242,6 +248,9 @@ if [[ -z $PIPELINE_OPTS ]]; then if [[ "$ARCH" == "ARM" ]]; then opts+=("--machine_type=t2a-standard-1") + + IMAGE_NAME="beam_python${PY_VERSION}_sdk" + opts+=("--sdk_container_image=us.gcr.io/$PROJECT/$USER/$IMAGE_NAME:$MULTIARCH_TAG") fi if [[ ! -z "$KMS_KEY_NAME" ]]; then diff --git a/sdks/python/test-suites/dataflow/common.gradle b/sdks/python/test-suites/dataflow/common.gradle index 7766cf3a377c..a713b82400e7 100644 --- a/sdks/python/test-suites/dataflow/common.gradle +++ b/sdks/python/test-suites/dataflow/common.gradle @@ -144,7 +144,9 @@ task postCommitIT { } task postCommitArmIT { + def pyversion = "${project.ext.pythonVersion.replace('.', '')}" dependsOn 'initializeForDataflowJob' + dependsOn ":sdks:python:container:py${pyversion}:docker" doLast { def testOpts = basicPytestOpts + ["--numprocesses=8", "--dist=loadfile"] @@ -153,6 +155,7 @@ task postCommitArmIT { "sdk_location": project.ext.sdkLocation, "suite": "postCommitIT-df${pythonVersionSuffix}", "collect": "it_postcommit", + "py_version": project.ext.pythonVersion, "arch": "ARM" ] def cmdArgs = mapToArgString(argMap) From f30f6c5e22046e2bd603ec47a3b9e38c80510fae Mon Sep 17 00:00:00 2001 From: Svetak Sundhar Date: Wed, 4 Oct 2023 17:42:26 +0000 Subject: [PATCH 021/435] [Blog] Quest updated dates (#28824) --- website/www/site/content/en/blog/beamquest.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/website/www/site/content/en/blog/beamquest.md b/website/www/site/content/en/blog/beamquest.md index eea893bf8227..dde6376b4077 100644 --- a/website/www/site/content/en/blog/beamquest.md +++ b/website/www/site/content/en/blog/beamquest.md @@ -34,6 +34,6 @@ Individuals aren’t the only ones who can benefit from completing this quest - Data Processing is a key part of AI/ML workflows. Given the recent advancements in artificial intelligence, now’s the time to jump into the world of data processing! Get started on your journey [here](https://www.cloudskillsboost.google/quests/310). -We are currently offering this quest **FREE OF CHARGE** until **July 8, 2023** for the **first 2,000** people. To obtain your badge for **FREE**, use the [Access Code](https://www.cloudskillsboost.google/catalog?qlcampaign=1h-swiss-19), create an account, and search ["Getting Started with Apache Beam"](https://www.cloudskillsboost.google/quests/310). +We are currently offering this quest **FREE OF CHARGE**. To obtain your badge for **FREE**, use the [Access Code](https://www.cloudskillsboost.google/catalog?qlcampaign=1h-swiss-19), create an account, and search ["Getting Started with Apache Beam"](https://www.cloudskillsboost.google/quests/310). If the code does not work, please email [dev@beam.apache.org](dev@beam.apache.org) to obtain a free code. PS: Once you earn your badge, please [share it on social media](https://support.google.com/qwiklabs/answer/9222527?hl=en&sjid=14905615709060962899-NA)! From b221d804998734dc9025dadc0d8354562ca79c18 Mon Sep 17 00:00:00 2001 From: Shunping Huang <133698626+shunping-google@users.noreply.github.com> Date: Wed, 4 Oct 2023 14:24:30 -0400 Subject: [PATCH 022/435] Add job name and worker id to traceId of AppendRowsRequest. (#28729) * Add new functions to retrieve job name and work id from GCE metadata * Concatenate job name, job id and worker id with ":". --- .../extensions/gcp/util/GceMetadataUtil.java | 50 +++++++++++++------ .../io/gcp/bigquery/BigQueryIOMetadata.java | 34 +++++++++++-- .../io/gcp/bigquery/BigQueryServicesImpl.java | 12 +++-- 3 files changed, 73 insertions(+), 23 deletions(-) diff --git a/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/util/GceMetadataUtil.java b/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/util/GceMetadataUtil.java index b853ab792e08..fd49b759fd6d 100644 --- a/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/util/GceMetadataUtil.java +++ b/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/util/GceMetadataUtil.java @@ -30,40 +30,60 @@ import org.apache.http.params.BasicHttpParams; import org.apache.http.params.HttpConnectionParams; import org.apache.http.params.HttpParams; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** */ public class GceMetadataUtil { private static final String BASE_METADATA_URL = "http://metadata/computeMetadata/v1/"; + private static final Logger LOG = LoggerFactory.getLogger(GceMetadataUtil.class); + static String fetchMetadata(String key) { + String requestUrl = BASE_METADATA_URL + key; int timeoutMillis = 5000; final HttpParams httpParams = new BasicHttpParams(); HttpConnectionParams.setConnectionTimeout(httpParams, timeoutMillis); - HttpClient client = new DefaultHttpClient(httpParams); - HttpGet request = new HttpGet(BASE_METADATA_URL + key); - request.setHeader("Metadata-Flavor", "Google"); - + String ret = ""; try { + HttpClient client = new DefaultHttpClient(httpParams); + + HttpGet request = new HttpGet(requestUrl); + request.setHeader("Metadata-Flavor", "Google"); + HttpResponse response = client.execute(request); - if (response.getStatusLine().getStatusCode() != 200) { - // May mean its running on a non DataflowRunner, in which case it's perfectly normal. - return ""; + if (response.getStatusLine().getStatusCode() == 200) { + InputStream in = response.getEntity().getContent(); + try (final Reader reader = new InputStreamReader(in, StandardCharsets.UTF_8)) { + ret = CharStreams.toString(reader); + } } - InputStream in = response.getEntity().getContent(); - try (final Reader reader = new InputStreamReader(in, StandardCharsets.UTF_8)) { - return CharStreams.toString(reader); - } - } catch (IOException e) { - // May mean its running on a non DataflowRunner, in which case it's perfectly normal. + } catch (IOException ignored) { } - return ""; + + // The return value can be an empty string, which may mean it's running on a non DataflowRunner. + LOG.debug("Fetched GCE Metadata at '{}' and got '{}'", requestUrl, ret); + + return ret; + } + + private static String fetchVmInstanceMetadata(String instanceMetadataKey) { + return GceMetadataUtil.fetchMetadata("instance/" + instanceMetadataKey); } private static String fetchCustomGceMetadata(String customMetadataKey) { - return GceMetadataUtil.fetchMetadata("instance/attributes/" + customMetadataKey); + return GceMetadataUtil.fetchVmInstanceMetadata("attributes/" + customMetadataKey); } public static String fetchDataflowJobId() { return GceMetadataUtil.fetchCustomGceMetadata("job_id"); } + + public static String fetchDataflowJobName() { + return GceMetadataUtil.fetchCustomGceMetadata("job_name"); + } + + public static String fetchDataflowWorkerId() { + return GceMetadataUtil.fetchVmInstanceMetadata("id"); + } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOMetadata.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOMetadata.java index ee64a7ab9ddb..0b5e063c0b5b 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOMetadata.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOMetadata.java @@ -28,8 +28,15 @@ final class BigQueryIOMetadata { private @Nullable String beamJobId; - private BigQueryIOMetadata(@Nullable String beamJobId) { + private @Nullable String beamJobName; + + private @Nullable String beamWorkerId; + + private BigQueryIOMetadata(@Nullable String beamJobId, @Nullable String beamJobName, + @Nullable String beamWorkerId) { this.beamJobId = beamJobId; + this.beamJobName = beamJobName; + this.beamWorkerId = beamWorkerId; } private static final Pattern VALID_CLOUD_LABEL_PATTERN = @@ -41,17 +48,24 @@ private BigQueryIOMetadata(@Nullable String beamJobId) { */ public static BigQueryIOMetadata create() { String dataflowJobId = GceMetadataUtil.fetchDataflowJobId(); + String dataflowJobName = GceMetadataUtil.fetchDataflowJobName(); + String dataflowWorkerId = GceMetadataUtil.fetchDataflowWorkerId(); + // If a Dataflow job id is returned on GCE metadata. Then it means // this program is running on a Dataflow GCE VM. - boolean isDataflowRunner = dataflowJobId != null && !dataflowJobId.isEmpty(); + boolean isDataflowRunner = !dataflowJobId.isEmpty(); String beamJobId = null; + String beamJobName = null; + String beamWorkerId = null; if (isDataflowRunner) { if (BigQueryIOMetadata.isValidCloudLabel(dataflowJobId)) { beamJobId = dataflowJobId; + beamJobName = dataflowJobName; + beamWorkerId = dataflowWorkerId; } } - return new BigQueryIOMetadata(beamJobId); + return new BigQueryIOMetadata(beamJobId, beamJobName, beamWorkerId); } public Map addAdditionalJobLabels(Map jobLabels) { @@ -68,6 +82,20 @@ public Map addAdditionalJobLabels(Map jobLabels) return this.beamJobId; } + /* + * Returns the beam job name. Can be null if it is not running on Dataflow. + */ + public @Nullable String getBeamJobName() { + return this.beamJobName; + } + + /* + * Returns the beam worker id. Can be null if it is not running on Dataflow. + */ + public @Nullable String getBeamWorkerId() { + return this.beamWorkerId; + } + /** * Returns true if label_value is a valid cloud label string. This function can return false in * cases where the label value is valid. However, it will not return true in a case where the diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java index 3d4565cb086e..04a665ac9947 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java @@ -1364,6 +1364,12 @@ public StreamAppendClient getStreamAppendClient( .setChannelsPerCpu(2) .build(); + String traceId = String.format("Dataflow:%s:%s:%s", + bqIOMetadata.getBeamJobName() == null ? options.getJobName() + : bqIOMetadata.getBeamJobName(), + bqIOMetadata.getBeamJobId() == null ? "" : bqIOMetadata.getBeamJobId(), + bqIOMetadata.getBeamWorkerId() == null ? "" : bqIOMetadata.getBeamWorkerId()); + StreamWriter streamWriter = StreamWriter.newBuilder(streamName, newWriteClient) .setExecutorProvider( @@ -1374,11 +1380,7 @@ public StreamAppendClient getStreamAppendClient( .setEnableConnectionPool(useConnectionPool) .setMaxInflightRequests(storageWriteMaxInflightRequests) .setMaxInflightBytes(storageWriteMaxInflightBytes) - .setTraceId( - "Dataflow:" - + (bqIOMetadata.getBeamJobId() != null - ? bqIOMetadata.getBeamJobId() - : options.getJobName())) + .setTraceId(traceId) .build(); return new StreamAppendClient() { private int pins = 0; From 3e22be4d310d93223d37620663bb270f71a7e23b Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Wed, 4 Oct 2023 16:40:08 -0400 Subject: [PATCH 023/435] Revert "Bump com.avast.gradle:gradle-docker-compose-plugin (#28465)" (#28828) * Revert "Bump com.avast.gradle:gradle-docker-compose-plugin (#28465)" This reverts commit 544745ca72868d3e385654a6cfaff444c012d00b. * Add buildSrc/build.gradle.kts to workflow trigger path --- .github/workflows/beam_PreCommit_CommunityMetrics.yml | 4 ++-- .test-infra/metrics/build.gradle | 2 +- buildSrc/build.gradle.kts | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/.github/workflows/beam_PreCommit_CommunityMetrics.yml b/.github/workflows/beam_PreCommit_CommunityMetrics.yml index f044b154c0ab..bb44ca0b5464 100644 --- a/.github/workflows/beam_PreCommit_CommunityMetrics.yml +++ b/.github/workflows/beam_PreCommit_CommunityMetrics.yml @@ -19,10 +19,10 @@ on: push: tags: ['v*'] branches: ['master', 'release-*'] - paths: ['.test-infra/metrics/**', '.github/workflows/beam_PreCommit_CommunityMetrics.yml'] + paths: ['.test-infra/metrics/**', 'buildSrc/build.gradle.kts', '.github/workflows/beam_PreCommit_CommunityMetrics.yml'] pull_request_target: branches: ['master', 'release-*'] - paths: ['.test-infra/metrics/**'] + paths: ['.test-infra/metrics/**', 'buildSrc/build.gradle.kts'] issue_comment: types: [created] schedule: diff --git a/.test-infra/metrics/build.gradle b/.test-infra/metrics/build.gradle index febe2849ef56..f1ecba05f84d 100644 --- a/.test-infra/metrics/build.gradle +++ b/.test-infra/metrics/build.gradle @@ -106,7 +106,7 @@ task deploy { standardOutput = stdout } - // All images have the same tag, it doesn't matter which we choose. + // All images have the same tag, it doesn't matter which we choose. String image = (stdout.toString().split(' ') as List)[0] String currentImageTag = (image.split(':') as List)[1] println "Current image tag: ${currentImageTag}" diff --git a/buildSrc/build.gradle.kts b/buildSrc/build.gradle.kts index edd10ee108f6..0ca748e3eb04 100644 --- a/buildSrc/build.gradle.kts +++ b/buildSrc/build.gradle.kts @@ -54,7 +54,7 @@ dependencies { runtimeOnly("gradle.plugin.io.pry.gradle.offline_dependencies:gradle-offline-dependencies-plugin:0.5.0") // Enable creating an offline repository runtimeOnly("net.ltgt.gradle:gradle-errorprone-plugin:1.2.1") // Enable errorprone Java static analysis runtimeOnly("org.ajoberstar.grgit:grgit-gradle:4.1.1") // Enable website git publish to asf-site branch - runtimeOnly("com.avast.gradle:gradle-docker-compose-plugin:0.17.5") // Enable docker compose tasks + runtimeOnly("com.avast.gradle:gradle-docker-compose-plugin:0.16.12") // Enable docker compose tasks runtimeOnly("ca.cutterslade.gradle:gradle-dependency-analyze:1.8.3") // Enable dep analysis runtimeOnly("gradle.plugin.net.ossindex:ossindex-gradle-plugin:0.4.11") // Enable dep vulnerability analysis runtimeOnly("org.checkerframework:checkerframework-gradle-plugin:0.6.33") // Enable enhanced static checking plugin From c2666e114971a4727919d9c95287a4ceaf486a92 Mon Sep 17 00:00:00 2001 From: Anand Inguva <34158215+AnandInguva@users.noreply.github.com> Date: Wed, 4 Oct 2023 16:51:02 -0400 Subject: [PATCH 024/435] Add num_workers and save_main_session flag to auto_model_refresh notebook (#28777) * Add num_workers and save_main_session flag * Add WorkerOptions * Apply suggestions from code review Co-authored-by: Danny McCormick * Add back removed contents from a past commit * Add workerOptions to the import * Created using Colaboratory * Created using Colaboratory * Update auto model refresh notebook * Apply suggestions from code review Co-authored-by: Danny McCormick --------- Co-authored-by: Danny McCormick --- .../beam-ml/automatic_model_refresh.ipynb | 1133 ++++++++--------- 1 file changed, 529 insertions(+), 604 deletions(-) diff --git a/examples/notebooks/beam-ml/automatic_model_refresh.ipynb b/examples/notebooks/beam-ml/automatic_model_refresh.ipynb index 67fe51af1253..9cbab0a14178 100644 --- a/examples/notebooks/beam-ml/automatic_model_refresh.ipynb +++ b/examples/notebooks/beam-ml/automatic_model_refresh.ipynb @@ -1,605 +1,530 @@ { - "nbformat": 4, - "nbformat_minor": 0, - "metadata": { - "colab": { - "provenance": [] - }, - "kernelspec": { - "name": "python3", - "display_name": "Python 3" - }, - "language_info": { - "name": "python" - } - }, - "cells": [{ - "cell_type": "code", - "source": [ - "# @title ###### Licensed to the Apache Software Foundation (ASF), Version 2.0 (the \"License\")\n", - "\n", - "# Licensed to the Apache Software Foundation (ASF) under one\n", - "# or more contributor license agreements. See the NOTICE file\n", - "# distributed with this work for additional information\n", - "# regarding copyright ownership. The ASF licenses this file\n", - "# to you under the Apache License, Version 2.0 (the\n", - "# \"License\"); you may not use this file except in compliance\n", - "# with the License. You may obtain a copy of the License at\n", - "#\n", - "# http://www.apache.org/licenses/LICENSE-2.0\n", - "#\n", - "# Unless required by applicable law or agreed to in writing,\n", - "# software distributed under the License is distributed on an\n", - "# \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY\n", - "# KIND, either express or implied. See the License for the\n", - "# specific language governing permissions and limitations\n", - "# under the License" - ], - "metadata": { - "cellView": "form", - "id": "OsFaZscKSPvo" - }, - "execution_count": null, - "outputs": [{ - "output_type": "stream", - "name": "stdout", - "text": [ - "\n" - ] - }] - }, - { - "cell_type": "markdown", - "source": [ - "# Update ML models in running pipelines\n", - "\n", - "\n", - " \n", - " \n", - "
\n", - " Run in Google Colab\n", - " \n", - " View source on GitHub\n", - "
\n" - ], - "metadata": { - "id": "ZUSiAR62SgO8" - } - }, - { - "cell_type": "markdown", - "source": [ - "This notebook demonstrates how to perform automatic model updates without stopping your Apache Beam pipeline.\n", - "You can use side inputs to update your model in real time, even while the Apache Beam pipeline is running. The side input is passed in a `ModelHandler` configuration object. You can update the model either by leveraging one of Apache Beam's provided patterns, such as the `WatchFilePattern`, or by configuring a custom side input `PCollection` that defines the logic for the model update.\n", - "\n", - "The pipeline in this notebook uses a RunInference `PTransform` with TensorFlow machine learning (ML) models to run inference on images. To update the model, it uses a side input `PCollection` that emits `ModelMetadata`.\n", - "For more information about side inputs, see the [Side inputs](https://beam.apache.org/documentation/programming-guide/#side-inputs) section in the Apache Beam Programming Guide.\n", - "\n", - "This example uses `WatchFilePattern` as a side input. `WatchFilePattern` is used to watch for file updates that match the `file_pattern` based on timestamps. It emits the latest `ModelMetadata`, which is used in the RunInference `PTransform` to automatically update the ML model without stopping the Apache Beam pipeline.\n" - ], - "metadata": { - "id": "tBtqF5UpKJNZ" - } - }, - { - "cell_type": "markdown", - "source": [ - "## Before you begin\n", - "Install the dependencies required to run this notebook.\n", - "\n", - "To use RunInference with side inputs for automatic model updates, use Apache Beam version 2.46.0 or later." - ], - "metadata": { - "id": "SPuXFowiTpWx" - } - }, - { - "cell_type": "code", - "execution_count": 1, - "metadata": { - "id": "1RyTYsFEIOlA", - "outputId": "0e6b88a7-82d8-4d94-951c-046a9b8b7abb", - "colab": { - "base_uri": "https://localhost:8080/" - } - }, - "outputs": [{ - "output_type": "stream", - "name": "stdout", - "text": [ - "\n" - ] - }], - "source": [ - "!pip install apache_beam[gcp]>=2.46.0 --quiet\n", - "!pip install tensorflow\n", - "!pip install tensorflow_hub" - ] - }, - { - "cell_type": "code", - "source": [ - "# Imports required for the notebook.\n", - "import logging\n", - "import time\n", - "from typing import Iterable\n", - "from typing import Tuple\n", - "\n", - "import apache_beam as beam\n", - "from apache_beam.examples.inference.tensorflow_imagenet_segmentation import PostProcessor\n", - "from apache_beam.examples.inference.tensorflow_imagenet_segmentation import read_image\n", - "from apache_beam.ml.inference.base import PredictionResult\n", - "from apache_beam.ml.inference.base import RunInference\n", - "from apache_beam.ml.inference.tensorflow_inference import TFModelHandlerTensor\n", - "from apache_beam.ml.inference.utils import WatchFilePattern\n", - "from apache_beam.options.pipeline_options import GoogleCloudOptions\n", - "from apache_beam.options.pipeline_options import PipelineOptions\n", - "from apache_beam.options.pipeline_options import SetupOptions\n", - "from apache_beam.options.pipeline_options import StandardOptions\n", - "from apache_beam.transforms.periodicsequence import PeriodicImpulse\n", - "import numpy\n", - "from PIL import Image\n", - "import tensorflow as tf" - ], - "metadata": { - "id": "Rs4cwwNrIV9H" - }, - "execution_count": 2, - "outputs": [{ - "output_type": "stream", - "name": "stdout", - "text": [ - "\n" - ] - }] - }, - { - "cell_type": "code", - "source": [ - "# Authenticate to your Google Cloud account.\n", - "from google.colab import auth\n", - "auth.authenticate_user()" - ], - "metadata": { - "id": "jAKpPcmmGm03" - }, - "execution_count": 3, - "outputs": [{ - "output_type": "stream", - "name": "stdout", - "text": [ - "\n" - ] - }] - }, - { - "cell_type": "markdown", - "source": [ - "## Configure the runner\n", - "\n", - "This pipeline uses the Dataflow Runner. To run the pipeline, you need to complete the following tasks:\n", - "\n", - "* Ensure that you have all the required permissions to run the pipeline on Dataflow.\n", - "* Configure the pipeline options for the pipeline to run on Dataflow. Make sure the pipeline is using streaming mode.\n", - "\n", - "In the following code, replace `BUCKET_NAME` with the the name of your Cloud Storage bucket." - ], - "metadata": { - "id": "ORYNKhH3WQyP" - } - }, - { - "cell_type": "code", - "source": [ - "options = PipelineOptions()\n", - "options.view_as(StandardOptions).streaming = True\n", - "\n", - "# Provide required pipeline options for the Dataflow Runner.\n", - "options.view_as(StandardOptions).runner = \"DataflowRunner\"\n", - "\n", - "# Set the project to the default project in your current Google Cloud environment.\n", - "options.view_as(GoogleCloudOptions).project = 'your-project'\n", - "\n", - "# Set the Google Cloud region that you want to run Dataflow in.\n", - "options.view_as(GoogleCloudOptions).region = 'us-central1'\n", - "\n", - "# IMPORTANT: Replace BUCKET_NAME with the the name of your Cloud Storage bucket.\n", - "dataflow_gcs_location = \"gs://BUCKET_NAME/tmp/\"\n", - "\n", - "# The Dataflow staging location. This location is used to stage the Dataflow pipeline and the SDK binary.\n", - "options.view_as(GoogleCloudOptions).staging_location = '%s/staging' % dataflow_gcs_location\n", - "\n", - "# The Dataflow temp location. This location is used to store temporary files or intermediate results before outputting to the sink.\n", - "options.view_as(GoogleCloudOptions).temp_location = '%s/temp' % dataflow_gcs_location\n", - "\n" - ], - "metadata": { - "id": "wWjbnq6X-4uE" - }, - "execution_count": 4, - "outputs": [{ - "output_type": "stream", - "name": "stdout", - "text": [ - "\n" - ] - }] - }, - { - "cell_type": "markdown", - "source": [ - "Install the `tensorflow` and `tensorflow_hub` dependencies on Dataflow. Use the `requirements_file` pipeline option to pass these dependencies." - ], - "metadata": { - "id": "HTJV8pO2Wcw4" - } - }, - { - "cell_type": "code", - "source": [ - "# In a requirements file, define the dependencies required for the pipeline.\n", - "deps_required_for_pipeline = ['tensorflow>=2.12.0', 'tensorflow-hub>=0.10.0', 'Pillow>=9.0.0']\n", - "requirements_file_path = './requirements.txt'\n", - "# Write the dependencies to the requirements file.\n", - "with open(requirements_file_path, 'w') as f:\n", - " for dep in deps_required_for_pipeline:\n", - " f.write(dep + '\\n')\n", - "\n", - "# Install the pipeline dependencies on Dataflow.\n", - "options.view_as(SetupOptions).requirements_file = requirements_file_path" - ], - "metadata": { - "id": "lEy4PkluWbdm" - }, - "execution_count": 5, - "outputs": [{ - "output_type": "stream", - "name": "stdout", - "text": [ - "\n" - ] - }] - }, - { - "cell_type": "markdown", - "source": [ - "## Use the TensorFlow model handler\n", - " This example uses `TFModelHandlerTensor` as the model handler and the `resnet_101` model trained on [ImageNet](https://www.image-net.org/).\n", - "\n", - " Download the model from [Google Cloud Storage](https://storage.googleapis.com/tensorflow/keras-applications/resnet/resnet101_weights_tf_dim_ordering_tf_kernels.h5) (link downloads the model), and place it in the directory that you want to use to update your model.\n", - "\n", - "In the following code, replace `BUCKET_NAME` with the the name of your Cloud Storage bucket." - ], - "metadata": { - "id": "_AUNH_GJk_NE" - } - }, - { - "cell_type": "code", - "source": [ - "model_handler = TFModelHandlerTensor(\n", - " model_uri=\"gs://BUCKET_NAME/resnet101_weights_tf_dim_ordering_tf_kernels.h5\")" - ], - "metadata": { - "id": "kkSnsxwUk-Sp" - }, - "execution_count": 6, - "outputs": [{ - "output_type": "stream", - "name": "stdout", - "text": [ - "\n" - ] - }] - }, - { - "cell_type": "markdown", - "source": [ - "## Preprocess images\n", - "\n", - "Use `preprocess_image` to run the inference, read the image, and convert the image to a TensorFlow tensor." - ], - "metadata": { - "id": "tZH0r0sL-if5" - } - }, - { - "cell_type": "code", - "source": [ - "def preprocess_image(image_name, image_dir):\n", - " img = tf.keras.utils.get_file(image_name, image_dir + image_name)\n", - " img = Image.open(img).resize((224, 224))\n", - " img = numpy.array(img) / 255.0\n", - " img_tensor = tf.cast(tf.convert_to_tensor(img[...]), dtype=tf.float32)\n", - " return img_tensor" - ], - "metadata": { - "id": "dU5imgTt-8Ne" - }, - "execution_count": 7, - "outputs": [{ - "output_type": "stream", - "name": "stdout", - "text": [ - "\n" - ] - }] - }, - { - "cell_type": "code", - "source": [ - "class PostProcessor(beam.DoFn):\n", - " \"\"\"Process the PredictionResult to get the predicted label.\n", - " Returns predicted label.\n", - " \"\"\"\n", - " def process(self, element: PredictionResult) -> Iterable[Tuple[str, str]]:\n", - " predicted_class = numpy.argmax(element.inference, axis=-1)\n", - " labels_path = tf.keras.utils.get_file(\n", - " 'ImageNetLabels.txt',\n", - " 'https://storage.googleapis.com/download.tensorflow.org/data/ImageNetLabels.txt' # pylint: disable=line-too-long\n", - " )\n", - " imagenet_labels = numpy.array(open(labels_path).read().splitlines())\n", - " predicted_class_name = imagenet_labels[predicted_class]\n", - " yield predicted_class_name.title(), element.model_id" - ], - "metadata": { - "id": "6V5tJxO6-gyt" - }, - "execution_count": 8, - "outputs": [{ - "output_type": "stream", - "name": "stdout", - "text": [ - "\n" - ] - }] - }, - { - "cell_type": "code", - "source": [ - "# Define the pipeline object.\n", - "pipeline = beam.Pipeline(options=options)" - ], - "metadata": { - "id": "GpdKk72O_NXT", - "outputId": "bcbaa8a6-0408-427a-de9e-78a6a7eefd7b", - "colab": { - "base_uri": "https://localhost:8080/", - "height": 400 - } - }, - "execution_count": 9, - "outputs": [{ - "output_type": "stream", - "name": "stdout", - "text": [ - "\n" - ] - }] - }, - { - "cell_type": "markdown", - "source": [ - "Next, review the pipeline steps and examine the code.\n", - "\n", - "### Pipeline steps\n" - ], - "metadata": { - "id": "elZ53uxc_9Hv" - } - }, - { - "cell_type": "markdown", - "source": [ - "1. Create a `PeriodicImpulse` transform, which emits output every `n` seconds. The `PeriodicImpulse` transform generates an infinite sequence of elements with a given runtime interval.\n", - "\n", - " In this example, `PeriodicImpulse` mimics the Pub/Sub source. Because the inputs in a streaming pipeline arrive in intervals, use `PeriodicImpulse` to output elements at `m` intervals.\n", - "To learn more about `PeriodicImpulse`, see the [`PeriodicImpulse` code](https://github.com/apache/beam/blob/9c52e0594d6f0e59cd17ee005acfb41da508e0d5/sdks/python/apache_beam/transforms/periodicsequence.py#L150)." - ], - "metadata": { - "id": "305tkV2sAD-S" - } - }, - { - "cell_type": "code", - "source": [ - "start_timestamp = time.time() # start timestamp of the periodic impulse\n", - "end_timestamp = start_timestamp + 60 * 20 # end timestamp of the periodic impulse (will run for 20 minutes).\n", - "main_input_fire_interval = 60 # interval in seconds at which the main input PCollection is emitted.\n", - "side_input_fire_interval = 60 # interval in seconds at which the side input PCollection is emitted.\n", - "\n", - "periodic_impulse = (\n", - " pipeline\n", - " | \"MainInputPcoll\" >> PeriodicImpulse(\n", - " start_timestamp=start_timestamp,\n", - " stop_timestamp=end_timestamp,\n", - " fire_interval=main_input_fire_interval))" - ], - "metadata": { - "id": "vUFStz66_Tbb", - "outputId": "39f2704b-021e-4d41-fce3-a2fac90a5bad", - "colab": { - "base_uri": "https://localhost:8080/", - "height": 133 - } - }, - "execution_count": 10, - "outputs": [{ - "output_type": "stream", - "name": "stdout", - "text": [ - "\n" - ] - }] - }, - { - "cell_type": "markdown", - "source": [ - "2. To read and preprocess the images, use the `read_image` function. This example uses `Cat-with-beanie.jpg` for all inferences.\n", - "\n", - " **Note**: Image used for prediction is licensed in CC-BY. The creator is listed in the [LICENSE.txt](https://storage.googleapis.com/apache-beam-samples/image_captioning/LICENSE.txt) file." - ], - "metadata": { - "id": "8-sal2rFAxP2" - } - }, - { - "cell_type": "markdown", - "source": [ - "![download.png](data:image/png;base64,iVBORw0KGgoAAAANSUhEUgAAAOAAAADgCAIAAACVT/22AAAKMWlDQ1BJQ0MgUHJvZmlsZQAAeJydlndUU9kWh8+9N71QkhCKlNBraFICSA29SJEuKjEJEErAkAAiNkRUcERRkaYIMijggKNDkbEiioUBUbHrBBlE1HFwFBuWSWStGd+8ee/Nm98f935rn73P3Wfvfda6AJD8gwXCTFgJgAyhWBTh58WIjYtnYAcBDPAAA2wA4HCzs0IW+EYCmQJ82IxsmRP4F726DiD5+yrTP4zBAP+flLlZIjEAUJiM5/L42VwZF8k4PVecJbdPyZi2NE3OMErOIlmCMlaTc/IsW3z2mWUPOfMyhDwZy3PO4mXw5Nwn4405Er6MkWAZF+cI+LkyviZjg3RJhkDGb+SxGXxONgAoktwu5nNTZGwtY5IoMoIt43kA4EjJX/DSL1jMzxPLD8XOzFouEiSniBkmXFOGjZMTi+HPz03ni8XMMA43jSPiMdiZGVkc4XIAZs/8WRR5bRmyIjvYODk4MG0tbb4o1H9d/JuS93aWXoR/7hlEH/jD9ld+mQ0AsKZltdn6h21pFQBd6wFQu/2HzWAvAIqyvnUOfXEeunxeUsTiLGcrq9zcXEsBn2spL+jv+p8Of0NffM9Svt3v5WF485M4knQxQ143bmZ6pkTEyM7icPkM5p+H+B8H/nUeFhH8JL6IL5RFRMumTCBMlrVbyBOIBZlChkD4n5r4D8P+pNm5lona+BHQllgCpSEaQH4eACgqESAJe2Qr0O99C8ZHA/nNi9GZmJ37z4L+fVe4TP7IFiR/jmNHRDK4ElHO7Jr8WgI0IABFQAPqQBvoAxPABLbAEbgAD+ADAkEoiARxYDHgghSQAUQgFxSAtaAYlIKtYCeoBnWgETSDNnAYdIFj4DQ4By6By2AE3AFSMA6egCnwCsxAEISFyBAVUod0IEPIHLKFWJAb5AMFQxFQHJQIJUNCSAIVQOugUqgcqobqoWboW+godBq6AA1Dt6BRaBL6FXoHIzAJpsFasBFsBbNgTzgIjoQXwcnwMjgfLoK3wJVwA3wQ7oRPw5fgEVgKP4GnEYAQETqiizARFsJGQpF4JAkRIauQEqQCaUDakB6kH7mKSJGnyFsUBkVFMVBMlAvKHxWF4qKWoVahNqOqUQdQnag+1FXUKGoK9RFNRmuizdHO6AB0LDoZnYsuRlegm9Ad6LPoEfQ4+hUGg6FjjDGOGH9MHCYVswKzGbMb0445hRnGjGGmsVisOtYc64oNxXKwYmwxtgp7EHsSewU7jn2DI+J0cLY4X1w8TogrxFXgWnAncFdwE7gZvBLeEO+MD8Xz8MvxZfhGfA9+CD+OnyEoE4wJroRIQiphLaGS0EY4S7hLeEEkEvWITsRwooC4hlhJPEQ8TxwlviVRSGYkNimBJCFtIe0nnSLdIr0gk8lGZA9yPFlM3kJuJp8h3ye/UaAqWCoEKPAUVivUKHQqXFF4pohXNFT0VFysmK9YoXhEcUjxqRJeyUiJrcRRWqVUo3RU6YbStDJV2UY5VDlDebNyi/IF5UcULMWI4kPhUYoo+yhnKGNUhKpPZVO51HXURupZ6jgNQzOmBdBSaaW0b2iDtCkVioqdSrRKnkqNynEVKR2hG9ED6On0Mvph+nX6O1UtVU9Vvuom1TbVK6qv1eaoeajx1UrU2tVG1N6pM9R91NPUt6l3qd/TQGmYaYRr5Grs0Tir8XQObY7LHO6ckjmH59zWhDXNNCM0V2ju0xzQnNbS1vLTytKq0jqj9VSbru2hnaq9Q/uE9qQOVcdNR6CzQ+ekzmOGCsOTkc6oZPQxpnQ1df11Jbr1uoO6M3rGelF6hXrtevf0Cfos/ST9Hfq9+lMGOgYhBgUGrQa3DfGGLMMUw12G/YavjYyNYow2GHUZPTJWMw4wzjduNb5rQjZxN1lm0mByzRRjyjJNM91tetkMNrM3SzGrMRsyh80dzAXmu82HLdAWThZCiwaLG0wS05OZw2xljlrSLYMtCy27LJ9ZGVjFW22z6rf6aG1vnW7daH3HhmITaFNo02Pzq62ZLde2xvbaXPJc37mr53bPfW5nbse322N3055qH2K/wb7X/oODo4PIoc1h0tHAMdGx1vEGi8YKY21mnXdCO3k5rXY65vTW2cFZ7HzY+RcXpkuaS4vLo3nG8/jzGueNueq5clzrXaVuDLdEt71uUnddd457g/sDD30PnkeTx4SnqWeq50HPZ17WXiKvDq/XbGf2SvYpb8Tbz7vEe9CH4hPlU+1z31fPN9m31XfKz95vhd8pf7R/kP82/xsBWgHcgOaAqUDHwJWBfUGkoAVB1UEPgs2CRcE9IXBIYMj2kLvzDecL53eFgtCA0O2h98KMw5aFfR+OCQ8Lrwl/GGETURDRv4C6YMmClgWvIr0iyyLvRJlESaJ6oxWjE6Kbo1/HeMeUx0hjrWJXxl6K04gTxHXHY+Oj45vipxf6LNy5cDzBPqE44foi40V5iy4s1licvvj4EsUlnCVHEtGJMYktie85oZwGzvTSgKW1S6e4bO4u7hOeB28Hb5Lvyi/nTyS5JpUnPUp2Td6ePJninlKR8lTAFlQLnqf6p9alvk4LTduf9ik9Jr09A5eRmHFUSBGmCfsytTPzMoezzLOKs6TLnJftXDYlChI1ZUPZi7K7xTTZz9SAxESyXjKa45ZTk/MmNzr3SJ5ynjBvYLnZ8k3LJ/J9879egVrBXdFboFuwtmB0pefK+lXQqqWrelfrry5aPb7Gb82BtYS1aWt/KLQuLC98uS5mXU+RVtGaorH1futbixWKRcU3NrhsqNuI2ijYOLhp7qaqTR9LeCUXS61LK0rfb+ZuvviVzVeVX33akrRlsMyhbM9WzFbh1uvb3LcdKFcuzy8f2x6yvXMHY0fJjpc7l+y8UGFXUbeLsEuyS1oZXNldZVC1tep9dUr1SI1XTXutZu2m2te7ebuv7PHY01anVVda926vYO/Ner/6zgajhop9mH05+x42Rjf2f836urlJo6m06cN+4X7pgYgDfc2Ozc0tmi1lrXCrpHXyYMLBy994f9Pdxmyrb6e3lx4ChySHHn+b+O31w0GHe4+wjrR9Z/hdbQe1o6QT6lzeOdWV0iXtjusePhp4tLfHpafje8vv9x/TPVZzXOV42QnCiaITn07mn5w+lXXq6enk02O9S3rvnIk9c60vvG/wbNDZ8+d8z53p9+w/ed71/LELzheOXmRd7LrkcKlzwH6g4wf7HzoGHQY7hxyHui87Xe4Znjd84or7ldNXva+euxZw7dLI/JHh61HXb95IuCG9ybv56Fb6ree3c27P3FlzF3235J7SvYr7mvcbfjT9sV3qID0+6j068GDBgztj3LEnP2X/9H686CH5YcWEzkTzI9tHxyZ9Jy8/Xvh4/EnWk5mnxT8r/1z7zOTZd794/DIwFTs1/lz0/NOvm1+ov9j/0u5l73TY9P1XGa9mXpe8UX9z4C3rbf+7mHcTM7nvse8rP5h+6PkY9PHup4xPn34D94Tz+6TMXDkAAQAASURBVHichP3Xt2xJeh+IfV9EbJs+8/hzvalbt3x1tQe70WgSAAnQDClySGo4miXxbV70oLX0b0hr6UEPkkZrRmuRHJrhiAAJEoYwDaIb7dDl63pzzr3Hn/SZ20TEp4cwe+e5RSq7+p4020R88YvfZyM2/rN/9zsMERAAgAgAABEBAADcH/t39Ut/MCABIBIRgL0OIhIRIgIRIgAgubPMO3MUoX1P5kJE1V0QQWtg6C9KRBcaYP+PSEQEhHChnQSABAQEgIgIQIT2eEB3O+1awMwJ/nxC00CC//8vc1bVPgJ/Ipo2ABARs6Kyn22/AAgIEY1k3O3Inou+D/Y3K9gvawTCRVEQuS4wrLXKtsi33bZKmxPJDgfU5I5V27B2Cd8OTVZkHhdE2ouydiBBbSjt+1Upk/vB/BVGMmgHnGzLDODMsPqPtfFAspgjcPK4iGvzHpFIuzv4MXCDY2/nYe0HAAEI0c4D/6sTKVENjgTMSXlFHnYc0UCVABmiF3c1Ydz40MooVB+qMas65GaWux4gkiZEtOCwLQRw6ERE8211LS8tRNDmW6yBD8HInqpbmIbY8UIrPCsef6wfXyAgI3QAIIsgK2QkM9DaSd53zQqPwLcEEQE0EFY0U8nCDiAB1qQO1a0MTJlFnAcUMjKD4MZ/RS5oIWxELTwy0FEa1DBOfgo5jJKdI4YgbfcJ7YXRCsl1zw6UQ4oXMZHrC/l54W9UcWrVNqBao9HPInBU5KYu1eRMrrNYURzWoWbpybUKyE2SCql2kLz47Im0Ipn6qNUmT4XBSuLVqDg025G9OPqORGsaDP0x5KBG6IjTH2PbSVBrl1Vz4JkT3fRZuZvneys4cNIwI7cqHKxRVG1WVw1ZYU2wDcKKIiwFoMEnICPSgCunIyKrtbDSUQjAABkBA2CrOp4BMsd/tdlee2deBEjEEMlCzoPeAdad4KYa2RNrbbBvCMxVGCIAMEMClhgJALVlZXSzYHWeIVYD7fSv/89+rBG5GQD0TbCnIFZIc+A17SVws6zqnZWQ66vnJY8Nx4U1jWwJr5oDQNWguFPs0NQEVaETERkgq3Wypp+djqxdzmCsrhsBfAvAmW019UzGgvAz58I5Ncl7QToGcx3XRhV4kwYAPMUiYk3otoNsFT1VOwmtfMndiqG/rMcSrp7lB8sKjFzH6iDwl6gI6UuuUGu/w0DtG4sL5i5DBscAjKzSR6ia6AYVEIChRTw4c8LJDo2OxNW2WCPJcXBl7dSGhyGaMWJ2grguERhTyCLP/0Z+9Kz0EYDVeL4aXrTWQdVpOzxeSismqYGPvVmNxqoJWZuZUB+Xumirc+qnrwiGKmvCWlDVvfwkdV/WXBwPGnO+QZmG+ssSn0Ukq+upisIqAgGOyFYnWgUXrJ1iIeJAzBz+aq32H/3twLLIxQasYhrN6FUidqNnNaZrngZS3iCqAYgANGntNQTWpofnYSBk9tJ1TNSuY8zIWjdq4+UsGqiEazkU681GNzLOdKumnZkJiMzCnGEFBqjo0A+Bx5GXniNjWuFdcy5WcPFS8K/qSwcuzwf1Qb9gNKxcyp5BNYVPABqcMVqHZiXBmuKtdcga60YoosYOq0aVtfB8c1bHxBmK5sW8MYe2gXXLFZmf+mS42YKPVgyl+ssZtbY1K9Kxk9fbwn4QrMeqrZvlrSkvddMCc5gb1pp2JyIEBpW5y6xBaZFMTpOvkqzzCqp56uRLTom4A9C2k/nW1k6z56wwaG0M7W211hZUXvwV8/pjV9w4g1qnaMFRwoXxRCIC7ZtcXRj8vZz/QLXWVoEC8AyP9U9o5zwQMYQqMOImVTVBViaDa5aAyouvjCBy6s9JcGWWgcOhJ27vXWG9LW7swPlADJCM2UtQ98l8c6GmdiuIr0qp1iEfgbH2KzmWZYCOL11YgBzHOF+ypqexGjE3sM6tQCsEz9++PWYK4IVGec+ZauC2SHFNNW8RVnmoenl311JZxY6VqnGMwLzbsaKjKvb1DSATs7pgV6LDiGs2eLD6sSB/oO945dza+9QvB9oaOwBE2uk7BjbmgH5GW22+QtXoohOWHpnviTV2KvMIkIDXBsTadVS11MCXMXZBzI7SsS4JRLQxCERC0CuTpzY2F6/jfqkrW/uNu4kdQj+RUJtfGTorwnXLdhGsCrCGkJ916K4LjsTRqW/ndtSaWtOkNaPMDicyrNxLE2IxHffidFJG31Nng3ioOWPSXdtLYGUaW0l/Cda9MWrOXiHZVWiRNtY2klUg1S2qLoDzkwDAxZjQAcO1zzRF+wFBZIjcdquunA3EaLWNKz8iAoiqM/a+tckH4AXqNYUNctcutjKDa7EJ86WdIXZeQCVuhAtT35zAvNCdiC2B+fiEH+GaO+mJ34LVaQFnTlrNhZVycghw4q8UGkPSNrLt4vxewyAjy40rto8lW9MjYoZcrUK0POx64Qfcwh2p0qoWqz7wSXRhUNz9nD1qcWdMCh+/gy9LW1zsdKUhLZPVLuvGzjWz+saLxKgCN6CWC8kHRKsGG8TrSuujuxR5htJfpkdsX4RVjlXTyPN+TQtaIjF6AgCZ0eyvqoNX57E7wrET1rBQYyQHYAepSjDoEj+V2nOdrIte12TvXWcrympkVsOQWL+5HWhy0RwN2t3Qm6FARMZjW417VPepUbQzvmoWoe2XYStHMdWEdKoP/HxzXfSZOh/5r+59AbXGeKpM/BWXyd6uzvUrP7tb1O0qrExFGzdwTcVqNvmsIfPsVHXFslLdEvDviFZasNJUcEThCLHS7g4iNcohx4jM36PGczUX114daxqhjkj06SqyQR8A67QyP0iGh4x0GVKlwoyRt8oPNU0F4IOFYJzy2pFkjRQ3jk43oTPprMZyagGt+nQhN9+FmhTdl4DMRWqrcUArTJ9W8m11t8XVy6ATpj/fDS0iAOgVR57cXK6NN9p0gxuzGgM64gRwwS+y2qyKpNpZ6uaSmYy1/tbEYBR0TSPZ3lZQrOYB1c6n6gh8BZyI9T8AAKKmD9AzXO0M52QAAoB28LHIsdqkAqBpen2ae3qw/OS8GXsNp0BruRDbM9TE0Sejq9/QBSMd0Xg95Nvm3WWnRV2a2ArJd5+qP479UFfBkUoEULu0U/KV6oNVcw1rJwEAECGgrvl8tr8rg+0ZF6imZ8gRmLt3pcfrkwOq36nGpeD6i0RU5THtMNiuX3AELjLlBS1fM4tesQqs0q4JrlID5BSBgzV4lWHvYnNGK3RGRKI+XLUqA0AiYnaO+ytWt6GaCqvBxxmgNdQ7M7QSi0sokzsbYWXW2ps6m94lVw1xIlgHq67vap1y/1pjzBCzs+XAWn3kZvuqiLyh4gRIVir22uTi+37G2hdDIPIFE2TJsqYM0TceEcEk7ldVTjWTPStWVloNgx6vaO+8gpJVX95fuvrtohPv54YJGFXM61T4xUvR6qe6mbBid5jGEhEgA9KeJmoNsoKpCBdqdzd4Jc3sQZ55EEyG0waKzYT2YPNE467FAH1I2SPFyxSppmfRKkHjV+Mrwje2o+VEBLAgRjchABCIISESrBSIVPzmbk4ezXRhSNALcsULr4ubAMgOGNQNa2/DOt1c6acVVWeu4SI8K/UfAACga+G1uv706MRaY+oGmLt07Ve40IdX4yI1qay6VmiHsQ7ui69Xvof6ePnbXRQjuFFAQOZu6rujbatthQg5KazC3JkBorqgs3JYfSJQpW78Bew/lp5qsTZnXTPP32hiYisT0ZCiV1Xo/JVKC62wEyEAdwl3G5ZAJG2igACrI+rnIDilie6CF+jKcZj3Ro1N4NMxrr9OHVRcUaUdnHlDfgSweuubVOlAZziad1QdVJnsTiDOu6prrsp/8a6tlznW7lgHUDUPV5FVfeUUXV04vq/1l6eDCgnk4r71bjuh+p77QIUxbCofAPwQ2Pa493buM0ThjU4TSK9ZPwSANsRJxrolBrZMxjGGLZmstDhYlFteNQ3yjXDq3N4OkQi0uWzFIC5XAd79RjRBeT9dqSZa329wISrHkxV7XeRKsLzsZpqLuPtP1WiaD1UcwqGEHLH6SVYB1dl3KxNtldouxHesjGslIljLHrhv3Mx5hSYNXrXSF+9Sh1pNBAgVlNHJz7d2BdarMvOjAwCgq7Pc0XaukfMxLWM7Dc0Y0+TI04q7GlU7ALUbE4DQldVlhepiv15X2hGx3qizlS4YIzWHtKJPcl0Cr+UulJbWJ5q5lyM/p4Yr0DGHhopRfD/ANm/FyfNdcCzlm1u5q/U2+0m2ah3WvIXKlPDMVrXKIG7FFFiNDXqS9WlPp838B6w3x4p21TR0XhSsvrT+EnSuQtlpFSJH5EAARBoqVodXX+hqzcw4sKrOrvJg3XBVpzhDwt6W7DiAK1oCXAk8+SGtcTMAEYmqDxVMfdKwrsKdTVLZ4dY3NCZlBdlKy1paR6xoHWoq3BzPTFn7ilqtTRKn9SrFWnWHfMvgIl+B9wncEXXZVxMSHHVZOXptZOMHqwzsu2475+ewvzr6fA1WCsuOjFN4tUbAynS9ACnvwXhKqyPVf3+RUJ20Lxy8ckylD6zYvM1jBeJrAF2g0LIMVQDCWu02eYr2sqhg4Fy7ipaZayf6Y5xgrEIC1zXjqDvCcgMGKydWB6xOdKN0kBGgdm6MA7EZS+ZPNO4XVnks6xIBkC05XXGi7d1ddT06UTnh2pwhWfJ0QjJmADq9VXP+6yrWNMrXwmJFcUZq5q09x5kPXiN4saD/tzK1jOZf8f8qrJi7uuGqtK5v2avodBem+psa7mus409xDm/d+7FOur+i18v+zFq62rTvS1tVNchb+eiJAXFVesYFcpFU1/PVqxHU7Sn72c8KAGSrVrVRWlUSuebMgbeQ6vaMMSjc5LOpT+bTJOTQ4sRUHzQvAxehA49R9FekSuExqCpMnRFoNbKXOpqlHd4CuOBmOlOdKm/Dj6sbt1VAeCUF1vKs7uQMLNdARMYsN5ATXcUdVf2/u8KKIGtDgBVFrQ7NxdJPU88AlqFXzMqLp1eYISedSpWBo7bqfJfErqYWwAVD1pv7VHuZ08kreGfXkDXDmL+NWbTkmmjzKibMY7GPCEAMrRqtKQV7J6dZai3yxzicWHFr2yUbNTAFqEgucWBZqqYpVl6OwKA2+Rzuq3nsZ4G/O1q+9DyEANoPk9c1XjsjIFQrCMyksUq58nFq96zcoMq2Ii9Uj4aaiFYzPW4OuLmHYIN3hrMtj7tX1aj61VfE5DBR00O1H1d65tpbI3iXP3Etqt5ZavcGja7BvUbD5gQLxAopXmqreqreNKehicjHeKAKNiIAOb6qjR4BIIqa3lztK75C7oh1bq+pM4DKlkJXEvKK0VeLOjDjWtSq0+sHoHWqEQz0ybr5RizMpkur0fISJHdkHVTgRtTTrXtPzltGJw7bPquf7aT3g1ybLOCMbrJXMHRkqyJqp2rDWL4idpWqzBfeNSR/WSunFZRWCueC2eoU1MUTVoQPlnHcLPT5sIqGWDVHvYnpPYG6MKu2X3R5V039WsPJ1toCgY2Dej0HFVycTnLfoikWqfcWoKYyyY2aw79BgWuxuawfbj+UK/asuZsCDVb5gvcevI7Q9SF3bSOqFj35yFRlezhp+EY4uYM1RK1jUdnT5LtGADWZO6MC6sPjpoO3n4zxg+B8BqiKOhyuSVu4VVdHrLwmB0E3AyqsYjXwFt8rKc1KJhUOakPmD6op+BV68KKg2lkXbk1ElRzd5az/pPUFw7F2DPnTa9qmIkxwQ+ksnKorPgLgZr+TCQIAc0FFEK+6eBeYyePN3xMdD1XyWJ3TlaJ2ROeNB3S6vFrJ6kspbHIfyCwt8oyLzkqoId9cnrmWubkP1veqW07Wd6rG12YTmF/uUo2UBQdU2DEKyPbb9qVmjFVThhBqJdhkKrOp0hpA6GJS6JgCnbnm4bhSmuS5ocaaVB/kVbCuGp2VpF6Fuzf0/U/oxnDFKnCX9YFYkx/xP5gBrYjDOXHeHvBWTsUP9eSGE40DazWVrTpCFL65vkHsFbyaUarPXy8uN1fAfnthqiFAtc7QDr5vnGkOA7xwRs1yBfCAqheNmFt7oiIA5iI/tUOs9C1sa2e5X1eaCbU2vaJnKwHUPtcmEVVXIQACZMwMle1Q7farI+Mv6zFa1XTWKjw9bqi+DgEv5NBXm10P5l0k44u2WwWymmtfb6qbpWiVD67Axk1rL4NV2brEksGcFxiBcwhXNYGLJiMC1JwG31Xj6fjjV9EKQKSdarYH2JynP74mQavk/Ky3NYJ+7nvmswxVK5jytrR2PeeA9WXQxqm22fkKzmi9AQSnVmoRUqrE6iRCta5UMnIcDPWWGvlWysoitBZPcCxoLobOCwNEs1uBu5rTM26kV6jXt6Cuyv0QXtDvK15szZFftTL9l5Ukav6WVXHkvNzapPPjWHtfN0Jr4jMjiL7pVawJHAwNFIicU26vsDJ5bBfd5HJLPjzmvKVQV+JERKSNHL1KcwNPNYvrQm+shvdHEgGCttFL5x/VMOqrRTx32gahNxaMFFxAEaz/zQCYn97GIHIzAW0b3ayo1I2b/dWV0V2V3Kx3CqNyLiutbgYXXQS2ukI1qH40fUiaavfy08DwsSdCN8ArPOSIFi6+XtXydVr1JoTx2KpBraz5utKujeBFVK5onrpercbFCMWYs1Zr4kW95ia87aphHkCwa/otnfkyBwtQ5u21SklYFq1nUXxrPIUS2dhKzdbyU7Am31pNgI8Je+WOZOHlpoanIDQ/2Wag5S9mDE1HUWbiWkI17VxR9DVtgIZQWPWxzlXuSGcuufCi740RKlVaE2sNtgNjruLfO4cLyIT00NGM722N3V/hEg9716RVZf2K4qrmHmKVz3SvV/Vv/XVB718wlpxfRX5kXm2qlfwFB9S/BRsTcL5TnczNZVfsX/NXGA9BW4fXmHLerDbmgzGlK/PIX1o7oKK9v+dtu4DTfVztSN31czLWCGgKNwlWnA2oT/AKwU412C5hRf0rwnNmqFuLvKLWqU4GVFWSr2b/rPdDNW3kysN8gsHalq4S1BqxHtG2Zdqrb9+1uklQp6gap10A1Kp9UiM2x5SWuhyW6p7QhVOsUn6lDLQaKYDqQr499RGpCbyaX37+umnvRw9sl8n10hhj5Gax1zaeGZFZ3e0miOk9EGiqxo/ckLr8kLkc+WSqKVGprDqvx4jcR2N42LbXZOwNn8qgIbfiH/xccxkG1xgCIiSbx3fGluPv2pSw3GML+Wvz8yJZ+Ex0pTRqNoDlL0f4Fz1ia0qYA1cBRH4CVzrolevV/oPK0LDTG9D99GUp9S9lshqPejXoJeKPctPnFXSuaH+yd0H4khs5RHmrsmq/+b7SMx4b1jAwhqEG0tVMAvSmEgEY9S5s+xEAkEgTugSvAbC5AQJpS3Lmsyt7RABysaGqEAkA0OYjagrS6keobghAzLhsrj++BMMfi64prjyOnEfFfDoObHfdncmbMk6MjhGhkik4drkwPABQHVYzb8zNwXrXNZokr7PsjHVhaAAy2zq69qObnA43WFV4+lG3zhNhrba/dhBRbbLVSGTFE6pfz+uFeocuzlCopFGnbK+VfLXkyglOvFgNvVNZda/DKQy7N1g1xGjULOnaLCY3ZPYvc6rLmHBAREgayVl0hibde/dRW7+HnIOEgKTRagyjzmrDa/4lBAIN5O/nceHJ1azl8GLxzfSdNE10BqsLzlnxomsfeHQ6nxvQmL61YanQuUoOjmepJi/bWu3bZTdQQ388YlV7X4WpEG0tgTEVnJIy7XQ2ghWDd7NoFVXkzsRV5q4gBMAY88C6cMyXMN9/5oXO6fTvjWK4cIBv8IV/fQO8T+J0JPiou5uermukLXVZmK6YdkREpAVWBKZtNNNaPnbDLbIkWs2sFceWkECb5cg2SoeWVhj5CDgikptW6NzeWtfQB6zrc86ebaqkzSw2R0NNIVZIBANp9PUdXiSGcR3fO9FDNWntHIOV+zp2cFUEPqFZ1xYmt4AVKVJVlI/1YcKq2Ssr5qqoUoUrK8RatmPldZHsoTZGr77oYkLyP/eq3F9HmXVevHCwvxdjTGtdk+bqbLcdqc6xGs9OWfM1AGlwYncAsjAQlXNX67PxGMjxW4267a8X+1YVR5oiOiRwaWhzTbKmkRGXsynAarAqfl9lNbyPr62B6MwKBzg3uwlMagxsUt21xR5B9urmpu52nupq4rR2e/UZ7PwGBFczapvukr727qYozDIk+Ru7s+uRf2b50Q6CBza4eCrYyeQA7eysVzyY2rh4WNCrduorqL3gZlUm1n/B5kFXwfAKWFeuU2uDwV8Feqd6nEjrtsYFPNlOG+AJfw83HOj64F3WlXaQb65T7uDu5t9aB5YIEN1ax0pYRITuf4AE2hp3hMhspM5ONa+pKx/IrSQhNzENrZmW+CVK2ql6oz7s8uhXyn5fHbkaFV3IirkRIjdCteHzDgB4ovUwtarK31Q7I9NJzImOagXOXl0a/sba5ITqVuD7Uj/Rf3Q2d2XJOBOsxotVrOG/+NKuDTUQ/5csCqrnzMjSkLZCtA1FdNuEW1E57nOWEBARCdAK3C5CRiGx6sp2fS05Brbt0y6uY+To4gz1lqJFp2tnFTfWnoXRF/LYZpvF42jNGPSDh4jgksCaKik4/8iBF4zjh8Dq0TcAQDT7770KTXJtqA+w7UG1pIGYLaomyyNQG1lcGY2aLVSjXqgUIDk8e15lUN3JN9j6XNWYr4DDHWbbUNUVOIvK//uqx4NV3Z1VIl/Cu5WAnMfpf/dBPa+OV+VpNZx1sqGqqGXMCrhCtuU0gx+HFjczAQBA6No5XozV8BIoZ1hUVGr1lzOnzEy2MKvW/1SD5jDqsOgCjE58bjaTE1dFUYioSXsM2oll/Vj0Q2VdJOdmVz2og6PqWPXOXsRbVDbTa4KmDvHAKhl5bK4UZa7MTfd/x6P1e7pgRk0uHjkeOm4CuJiMNVIc7KjWIW+AuS9r7rMRC658dBi6CKxqvC98U+t3bfRda10Gvn61Sjms/CE/92qsbXFRc3D8SURWxZOu8YJrvXFINKHPX/kZX8+21e1oo6S8biS7gJ385gLoM+4Ort62cSqwYiKn1ezmu5os3drHcph4PjMhUtMd26zKEndAMZKoPlWK6aJ+9EDxuoicejJjagREhHa5N9Z53k4MN5o1xnE/fhkJvEpC1cvVfLmTatErj3U3aq6/7nDvaNYogJxX4MDiW23TMPVz6yLyUvpyI9VpMts28AY6WIm4G1uGAoDarsrmFj4c5wNB5nvh5oibr34uVqzjw2PGHjJCQQDS2ulY1zr71k0yMl5eFTsyAQVA96ZqLAFjCGAel2FxAGBdHgsVUmaQ7MukZhCBMQC77WhN7lWays/4C+7VhbEEWEVK3TS3Ss4vgXCArcw7gBUGdy4Ueu6vq1fn29aYg1ysEd1MA3D7/FTfVA2tq2T0zqXjBt/n+sUJPCtYE2+l81hxR/1enjRIVxu0ebl5G4msR1gz0y9afpbinAI26t+FDKsIFPnpBHbZcZ26jNTMKDKs5GAeplTjWqiAau0OvyDVwMrcQCmjmD2JuPwokSbyMkEAZfvpLoHM7UNid+Wyjq/W5lvGGCIjAoaGMO3mPtqqZGdqVO5dXffXMPHlH6l2rMUSVOJmfjZ4jqX6mVYtMqpRu6Nc9EJ0NoNtqnc78BWjfqVtVq1Wprj/FT0OXqFS27Ia8znlAwSEjDkt5OC7cpIfb3BNttxYCbbePHKNdicanVyFhowEELVNEFfnVHdEBCIBXBipIGOOqJzRgd70NjF0Aqv+DVdbY8BckYiYk5fjXSs9czuTDTdJ/yrLhCC1BkclWmvrWiETnHPBOWOCMc4YcqZBo1ZGFsgYETFGjHHQhNzOZmYqQ1dTl25ymIhgDXernsGFjysc7DjLO8jVWebAGlW6UXLvHI5th91Y+AHxs8gw6KujXqnyVcX5qkJ3N1nRCeiH9EsDogY32kUi0ZGoOx1qgK5+81h13Fy7v7ODoZo5rArKVcrd2Xg1S916jE7zAAhVKjsa9kwCk4i3ZgGRX7pOdrJ6Oge/vgpA24y+9X+UUjVvirS2q9nMUVpr86/WWmmSWhtLxQCUMRaGYRAEURhGQRAFQSg4h4BAkyzQApRzLoiIkJGt5ajK1bwAqllZCdjHny6Mk51etVH+Eu3nTqupafexUqsVBH1iE3zLLiDEfFXxtgdkjZc9MtDbEDUurJuMF3SCw0s9gfwlx1RMVwnrS14XVD84f+kC6D0xO4vDuNB+LrHaQ+h07ZJGnuQmhZkFKObzBQGYImQyiXUiItCkwdWAGvCBVdlau9JQTaS1VlprAiBSWkmlEUBrJZVWpEmT1koRaa2kUlIpg0JFpElLJUmR0koRKSIA0lJqgEacdJutfrfbbjTbjYRRzCAEhgSaVElaA2OccwLiEIBWZtg0EUPGnJTr3i6Ck5gdBled5ATqe+fH78IoO2u3JkkEb+pRzWq3I1LdF83WHZ7DLcOt6DSXU6lFcNwlPNjJq9SaQqysuaovuFJtWPObLvo9dSL/z70ukHfVPKfE607eRSuFmN8qjaxtYztqL0FeKCst9d0nIHF4eqqJSJPSSlkgWn6TSmvQWpOSysAINUklldYladBaAWmtldIGjUprTZq0UkoTkDKRLQKlNQNCBK0tzxkAmC3kydpwIKUsZamknAZhmWUcMQ4CGQQ60GQmBihd5KA1IKcwAGCIjBEn0pqYzUe7YUXPGZXzawMBUE/2XBiPCpjODiQCt9ae6JURssis2JQcbNxlPOGhHQJ7nDXnoX5B0ySsobbGlNW4OUPMVxf4eB9Ahc7K79b/GSS6pNqXqH1ErbU1TAl8s2n1YzVZX+FRqBSYUwhA7umL5KXh7KKaTJ0yNH6c+IvPPlOktdbSlN0ppbQyjCKJFGnUWluKNY9DJQJQiAjELVUh5wztw7IY5wwJOOchIuM8YAw5RwDBuODmxRAZ40wwzu3WJoyIpCxny2yZLTVQGMWIjDSVSinTJIZaS1mWWkrOOJBGAsZQK8aY2ye6St/UctneC4aawCqIVGWaflCdYe1iG17KLvtSw2ltABwTOEh66kBnETjpV1WjFg2OiN2Yr7QTwJWr1p2h2hnWliB3NUMzVvXri0bzRRhdSLsbO9Gk170cTHPqhilU0Ux78cr4uKCAPMOS9dWNsQgMzVbylqG1k8bKfBMvjl8CczAB5IiaMYYoAELGGBMUMMZQMM4Y45wFXDDOGWLAOGPczFHOOeOcMyY4t941Y/ZXxgLOOGMaUDDGjW/ODHsyhmD8dNJUynKZ57PFYplnSymBMQVEQEprpTSg1qSkLFVRaMaACDnnSmhOmohpAm5IBd3WobWcvNfm3vyq2z6vDJj7vsaL5ELCZljoyx2O2mesX9iF78AbBubZZORrWQzIqhv40h034RBXblG7tYWUb3HNVrnwpmraqnnjZ9wFB3/1xEp3V5ijCtNVBM3WuGF1M3Q6xFbPMXc3++1FjVO7r7hz5ZrhM4aMIWOccWSMc8ZQMAbIiSFnDJEJhgyZ4Cg4BxcEMgJx33BEYAZ8BnsI6DeEcfEiRHNBZIiu3Aq01qWUQZgJEbClKOczBQSaSGkg0kohggZSUpZloRkHRC4DCjRpTVoR48Ye0mi8Om8K2j92eBkD68lZ4dVs1FcGcXVcycMCCejL3eGaWYn+LuDcT7COl0v2VosPnS9sdzJatRTrd8cauMz3nv/cBHIHuxZdPGPloyM/589e6FCVWa31EWyEa0UCDmtE3sjyxIsmaoPIjD4yc89mIaqQ5Uoc37dU3L5+gwA5IgCabClniGiZzizfMTFH5vddAIM931rkzGysY5G5kj90tQ5+3yKDTdd4sIar1sBYqSQXQgguEECTIq1ISyk554wxjVqRKqXUqBFRBaVSkmsbuTKz1xqZlUXoYt0AAOZhzL6itmZMeRtA/2ez0s7YW40xoR9cIlctZe4DaJW7T4IQ82ZFzfkwDXPjqX1hg8U6OUlWBkTVYIsEK9tawyqGBHv9L+P7evyoupG/qZ9Rq1rF0179ItafN5dy85Tqixc1gLaocUOENhpei4DU5oIhXjHoDmx00LAdmKQ+GvZjCIzZ+mDmaceZHxo0kOFK8M5P3QV2g+aF5QrJKj0LyDiBZgDEmOCcM64ANAERlaXUoTIxKamUZqQJlFIEWnCujXlKittZa29V1Wq6+2o7kljjpdVaAagNTDX0F1fzAFj146jXkl39flj3GzQhQxeItvnZuifrTMlajKC6mbtPjVNXScvGfSpqdwL31ii5ca3FLC4SKq7gr24ZrDRnxU4186xmkddVv+diJ0DzzpxONV7zt7GGgTvTm14EACJNG+TN3hW7yTGmITxrL5uLaCMTRjasiojA7EN/q+fOVdMCAVETWE/bJZycyQyaMaUUU8r6UMA0oo1REZlgFjA00VklpQaUQkitAyLQJsRkfR2sZaidbVPPeRos262ia4NtRV8HQT1iYgiSwC/EsMTnj6xsxBqNoRegCyMgoDeOLbNa/qxOdjTvFLSBGtR64gas1pTarb19XKnvFV6qRhhr+W+HVLTWkY9ouQmwMgeqnK6bRN6aMrVvnuEr46DiAMeZq1wMdevLhxcED5gpF3GGA6GjT6uyjUK0T1a1IvZPtNBam0chIkNknBt2dKG4+hYPhIxbBwa9dDSR0hq15pxLw9wAJnVEJJXDCJjnfJLW1h5FZUxPE2PV5GRBBLbFQCuKnhm9gs4Q+LJXHWc+vIc1vYHgRg9tssN71nWCcQOBhCiJSqXnpSIgqfSoKENr7iNn0A1EzFkoOBg5eHuUIfjgqysaomowTU7O+kW+M/WQ5EVQojdzbWFEld+yqtweq0kjOi+whuVagMny3MrtKiaqGyy2yhGs0jWttRh0ee4LM6ZSTGYmCIYINoJIYPdpRLNFqDMWV0pdjOCYQx4xxhCJMWRMIPpKOMM4HBxKLHAMl7jhRBeCI1JEzMSLrIvFCMHYpkor52QgIhKCVlqZ8KvWSilhph6ZJ0BY9XNhlBxp2VX2tW8qfq0FvK0qv2DZ+fY6FeDlXrsZESCWAOeZPCvkSVZIpV8ucwYwLItny+z1VmstDJ4tFuOs3EjChMHlNHq73xrEETP13eiKtrxt6ywQbxxo10nH5Gj+rOhlwzfgZ5EjtJVwmGu6m5+e9cnpzBqyqPK4qxiq41Gyyr5S3ZWfamdD9Wgh8MWu9taVavelw0RAJAQPHMd7i4V57kRE6zbUuJe7xI+ZWWicJEQAE2JyE4wUQ9SARJoZHBO5XZerdhs4Kq2ZQSgiMiYELxhqK3NGWiulgKEpvzD5KSWVVgrIZk7N0xlNAQqSsTeo7tPolUyF92+8PVl5AKb+1wxUXdlVgvfoQKdSwT5kWxOM8vJgoX4xXe7n5Wtp/GJZ/tHB2d+8vMMAn40mCpdf6YZnRfHT4fCG7r3RiP74ZPiz02ES8K+vde902pHggd2f0sKKKhihtTOqSWOa7cDg2mg1qOuy01dud0jfo5ozvmKlIK5eCirU1V5klXJl9PgfrDFji9uwbhzXo1D+3qyGh4pEAYTZU9Bk261zZOjT23FY2UJeZbKqQtR776ZPtrWmDWRntxWDcTYrNVrzlxmiAkCzAYrRAwRaa6k12YpVICDiDBC1Mll8pU0+n+z/TaQLyC8M8ZrL7Thfk2yd+KoPq06r0QMXvqyMeDMBEM6yMlMkNQHC0bJ4upTExJ+P57OiuJWmz5fFrVYzFfywoGku201xuZFKUqdK306SqVInhA0hNNGnw/lZrkNEjriRRruNOBGstuUuOKfMzpyVgIVtk1epNbeGISIoRUopzrkpaLzocrkhAa8bV3/03zqL3JUXrMz66joGZGgnjtM4hndr6HQ2qzuiNmbmhsLoOO2SMM7kQl+fx40NxGqVLAxtOIQ8PNEiEam2PR6iCSK4XtSMPMttruFgzWAExhhDBiYuD9oSu5tohps1aa2Mc6Q1aCCNNZbxD8y02sr/BRfPcErJDzm4GpsVNPgxc+LzysuVYEMm6dks35tn+8v8+TK/mcYa8EVRPsqKrSD42qB7fzrth0E/bq7F0X62vNPpXG00/uRsuFB0NW48Xi4Zsq93B52Ac9BPZvMfHJy/2UhezmdLpW42G391e3Cr2+LMrMujWqOdJ1U1zprH9aYSQFEWk8n45Ox0Op0mSYLILu9eardadeuxQsQF9e2MGPQ/ei70hFdX4vaCPodXv6iuyc+TKTqTtzr9glkmLNUwZ1YhmuyQUwJ+dMERKJrV6wx8N4CIbLmxScjVXDbmGlRvMtSnSQVy5meI+wxSKqmUIs21tqkBBBOd12TqWWoPEAULP+fLO6+0GkbyITHw2sTGSiuLvVLfgC6GVJ2liealmhZqoek4108X+cez+b3Dk6+s9UeKSlWOpD6dTIjzb3aai4D/fDonhC+U/MHx+Vf63QDhZZZ1gvDr/cFhtvjXe/tNxt/ptglxK0kvp8nxYnmz2eQIZ1n+o9PJ80JLwMtJeKMRBojCZE2AGDIC5986Kw2cp6K0yuez8dnRdDKcj89OTo7LUgVJIhiWs9Mbd95tNdvWG6gpblzVHjXYVuq/ApeNDVUAtiIj8FEj8H6pw/HKZKjF/ZzkLZz8FBAm2ehs4Qo3zHozoLX1ujz5+1VfjKFG4D6rROgvYAK0WgMwby7bhyvWnWXTDJsaJRKMG4s2YJyAFJk6FucEASEyYIwIbBWVUkbFO1CRBmKakNlyXnebCl7WiDRSXrHNV1Q3+GnphpAQl1KNc3mwKM6LcizpSV40uXhRSAHwm1cvZYg/G082wnCsqRFFqeDHUj8r5PU03Y7CoywPpLqWJNea6UFRTkspEa63Wv/w+lVQaiELjTjRuBbHaUSfnJ0d5vKtXvtUqqcno4NSDkT4rU6cCNZLEs44J1VqnQq2mYTdgHNnr2uCIs/zxXg2PJ6Mz8siAxbEcXrl6g2ttdIglUSZjw+fNa7fFSKoOMKPBlgoYW0hHq0ypEEsWTbSK9cgiz2H8HqMz4q5fpzHpfeuagreFCw7T8H+4w0rj24XtvOmmLml1lozEzdyKqHiWyJyQVoX6vMa6cKLuTCK/cgYmBQooiyVUorsniXGLjXsTsYGVUq5v1KEIeeccVZdx6ooqpQ7IFAtC14Tra1Xssay5RJNOJf6PC+Hhcylmin9YpFnSsecPVpkzxbLd9qtsZR3koRINxnrBPyoLG7G8d1BNyf6s9PhdhJfTZMux+Ns+cs76w2OPz073w6CS1Hw+Xg0kvp2q7WdJjpb/M9P9jbSxq2WjgVrJqkupy8X+bv9LmbZ3Wb6YDJ7kKnNNJAlU6DH2fLPjk97Ufhmu7mbhJtJ0Ap4isCWk3xyfHz4YjqbIxNRnMg8V1KJMMrzXBPkpQwD0dVKFpkBKFWYpAofiLWUhFd4VSmfobP6BIcqfIL2gZzWACCP75XsSC18BhVtgz/AkKbQCOgD12h4DsF6sgZAFdTJcgmgjwZXNmulBtBW56DddsLYqG7WVOkytLF3v1WYMRy5iyAoIKW1sVBtbIvZ+aC1zmVJi3khNVssgY04E4ssWywzpZSWstlu7Wxt97vddqtFYAIz5KkTELkLJeZKj7JyWsqlVO0wGJdKk95IotNc/fTkfJiXW40kZmwk1X5e5poEw6KUSymjMPx0PDs6PV+7uvtut3W0XA4Q16MgQfpkPOmF0XYcxYi/82Tvdr+znSbdMPxoMmkwvp4mcyn3hmNVyLjbTYVYRsnNgD85PN6N49NMjYpiLU4I8c9Hs0BTP2ncHgx++/GzQZz+rZtX8rL4i+mCR/GlNPro+OQjxot8eVvQr241OpBPxpOSRKc7KJcLqSQRLPNseHiU5wVyzrhotdt5USpdZ74Vf4nsGFUj4o+oRxRW8+ZW9TtF5wvoTa2f110VF1vDb8Utq13QfS/cvr8e4PalzdIf+zUCaO0XalS8XNnYzPkSFT2hjUFgbZcRtmIQg6kX8QEG43wwxjmwqu5Fa9sYIgSbONCkkUgr/dkXnz978kwpmeXlcpmVZQmAUqoSIGk01zc2vv7BB++89dZg0I9EoAlM6RYBHGby49FiUcqCKNT6YLmcSxkxcaR0qakjxHFe7u29vLo5SJLk0TJ7NptvxmEvDDnnL2QZhdFQqs2Qp+s9xvl/Op+cTGedZvP9XvuHwzHXcq4UAd+bTlRRrsXJN9b696ezo2V+pdFgjA2i8Eaa/Nns7Acn56+1Gg/my93B+neuXCatn0xnqMVGI2VIf/Ly6JsbG3d77YSx0e7Wz84nT+fLUqujXA7K5R//xV8U52fddvKd3f6vv7aN+WiumAiTBggpS8nEYrGYTmdZXsyzPF8uAUBpKKTsdTobeU5Nb15axemw4j0IN96v6D6bgrH8WLcNLQq9f+KDfQiepGq4ruH1IjqJEEB4uxacpqOac2R6YJxDi1MA+8B2RN8Td4qzPQHcNPLTjlbSW1iFwBnachPS2t4XgTPGEKV57hECEGmluOAAwETAkJli3TSKv/bee99+/71QYLZcaFmGYaiByrw4Pj394uHTn33x5J88evwf/uAPdi/vrvX7nW43iRMWp0eK/8nZ/GQpsdNm3c41rXdb6SBOzsvys8kEcsmD6Ea78e3XbxZaP1vKe0/2W3H8lUu7PxsOl3lZar0s85SxD7bXP5tM9ubLsdK/sr15UpY/n8wWSv/1zY1mwP+XJ/s7ofilqzdPsuLH55OlUkEYbUfBh+ej02X+br+FAf/sxcH6jau/tt7544PTf3F4+pev7WrGQ17+dDj8eq97o9V4vszeKPX96fgP7+3dubqNoH/2+PFkPNHD03gyvMKKv37r8ru3rz1/8UJK2eoMgjAOWcCV0oBFUY6n89FkEogAkGnSSqv5bCGVRO72jXMPr6+zoBkIH4r34ZaqsMIhEcGFgVfY0PmfgATKfOswfbFsqrJXa34/uYCa8EBHRKNAnaGADswO38ZZcfmsGt5NyNcarc7WrNL/jke9rWubYeLhPh1qVAZHxl31HwEoTcr4Ssb+AEacE0NyMVBEFjbbrThWOJxOJxzCVqfNGe+t7bz/3ge/cvjyP/zHP/mLB3ufjsZRFERJHAjOkJEQDalDScsghptv7DX6nz8tO+uD1zd7b7Waw0Qj4wutn07mo6xAwTe31iLGPz4ZZlINWo290SgbT1+/ujspJSPsR2JbiJOiGEupFf2d7bV7kzkCAuc3eu1lkf/7oxOYL5uE37117c1ee/Li8NNHz9/ovnW91VwsM6XVsNC3e53z5bIdCEySHKh4cTxpNG/2eg8nsz8+OT04n6go+s7Wxh/d++Lw44+bctmnrFku/8Gvfvfa7ubRyZkIEuRqMpk2GzoIQy7C9a1eb1DOFvnp2VmZZY1GEnFEwlmWl2UJvmbF/TFFyh6aKwxnQ0s1a69W8kI2a+n50cfIvQvv4vn2GBeKX3UD3I0qdJF/ErZN4tjISh3f4AKUAJ7MwNsnKzwKjvap7sVVN3fUT7WPAE7BGA1vU53IGHKGDEmTVtrsUmviSSYWZbP4WmtSSslCU5o2tNbHpyej4ZCIAPlsWfbXtv63/5u/8Xe+/0EvjXSpymVeFlKWCpVKApZyakzOwp/8UfqLHzQoG8/mP/3wi5PzaYzseLGcKV0w1mmlJVDAg1Gp749nu61mNwzm86zf6dxstRZSf3Z8dr4selH8cDh9r9O+0WpMNTyYzv7wky+y83FPhM20MRCiuP/gW1uD1zvNo6zsJMm3blxZSyJkvMgKPZoQgx8dnrJF/vx0xBgfxLGIo8eTGUh9cD7++fODnX7n8lrrf713//H9BxvL0eXl6QZkf/d7X799/dLZcEzIC6lyqaXWIk7iRrvV7THGo7R56dKltUFfkZZlQUBKKhHwOEnSpGkItD5YdVDWIy1+lGsaGS06/UhaU8GHP8wPzluqxanI+TIWMzVAmFONKWlDjT7s6ljScJsxSohcqY25LlttguvgCuzq8wEATMUeVZEsHzNykclVZWGSSQatZqWUTxeZ85BzYEiuyomIiLQGCOLm1sZW2mwWZTkdj4UIGs02D2ISjV/61rf/q19+v9+OC6nyLFdK5YssL0ogEEEQhEEyPmn/7E/ig6da8P3ZfCHlWhTebSRrUYCE+eHx8Wiy00rbafysKJ8u8tuXtjd6ncfzvET+j+7e2mq3fvj0YLfRuN5uX0+Te8PJ+eno9uXd/8PX3w04+3S6yB8+SvNlnjYbYfB4vvizFydXO+1C6d1mM8iyX+y9RGDv9zvnWfbnL4+HpfqT/WPJgv3R7F/9xedBFP/GzvpXOw18+fzwP/77wd5nV2LVbkZ/6f23X7t5czKZZ3mZlTJJG51Od2NrhwXhIl8OpxOplSyy5XIZxvGg12EcNcE8K0ql02abB6FFnTPT6tyJF1iqBlX71423c3TAsaP5T5N9KqXVzn6sL+LjwsVdXZVDPQiD+9r/apaEK10jUpaZHeTRwdZAyVycwFbemeXv6PS7bZq9FgBW6+YAwCxPNv4+AXCzWIQxk3GVSiljJBCgJptFsLaBJtKgyGa9GCadQbcoRsPzuNWJWx3ORZnnRCgl3X3znUYj/eGH9754fjJfLAVjXKowChWRJCBkLFt2P/qhvvra6Pqbe6XaXOs90jCV6mw4ub6zs9lMZ0WxjthP4stpdH883R/PfvXapWEhrzTSPzs+H8Qhj6OhlM0o/Pzo5FYzvbOzmRBcbiTHUuk0vvz+ezHC6WKpBH+a56+V8nQyHc6z9NKV0enZv77/bDMJi15/VJb//smLv3p1a7GYH7caj6dxJ4nf3N0a51mM1B4eru0MGo1oPYq/8vbbZ+enWlEUJ8V8UZRqs7++WCzCMMnL5Xw6ZWQWIOSqyIi04EJrUlqv9bob69uMcY8LeqWi2dNqhVTnTtiojVvg4egTKgL0lmC1SLFyVKy3tMpnHrXW4fLeC5FA5ySZiCZaRV7FjKByyVfAjz5XSWSr5SurwrXbTSi0hgn5yaSJfATU+PLovHizqQgjsLuEABn+RCBuwlCMEUMCWwSq7R6inIfRzuUba+vbwBgQlHmGZu8dFhBrXL319s7ulSdPHv7gF/efHg6JSAOVpJVSoDUxZEyw5w/ak9Hk6t3DNKFlWU7nDOnF2SibzDe2N2Yo1gi+GE6vd9v3xvOzvHw+nl5Pwm6aTjQ0EQ/mixKI0uTxIlsMJ3/88Nn1tf5RUc46a41AnGbLH7w4FEm8EQanZdlJ40+OT5iOwjR9Y2Pweqf1cDT8yXiaPTscb/S/c2nn/mgssvmTZ8//jFQrTXZvvrb4eLPTFDHAW6/dnExGRVlGcTNMmlu7V4MgQsYG61ukdZ4tVWegSZ+dHIVhNBgMnj2dSSlLSbnUrUYzbTTBaS1HiBVisF5O6iHr/1S5X/LHw0pxI9aQYpx3unjNV10lspVKzgy1VxZmkZnZVttSofV2NBGzsSJvWyKYQiY7Xep+FIDzlMB13J7J7PNAEMyWoOjMl5olXrMJqgirJgIyRSGkAbjLNzKw6U4FpJQ0G0Bo0JwxHoSNIFCl1EpiEAAAaaW0lnEhyzII0zff7W3vXPr43sM//cX9WVYKzoHZjf+AQDEmhsfd+bhYH3Ru3u5t9I6VHp+NFkl4mhdX0+gP9g+CIDgATNP0pCieFvn/5+mLnAevt9KdJJqXRab18nR4ab2fav2VW5f/zcf3ck3fv3ntvbXeVhp3AvE7v/tHrV6/tTb41lpv/+x8/9GTN7a33u20fvL85aX1XqIndH58NtqEnfX1JAl2d+/tHd7qdw7m8x+fnq+laSuCdhRtrq89e/6Mi2iwllKWvZwvNrZ20iRhnCMXiFiWeSnLOE0n08lskbMglEVRSinCYPfq1bTRrFtZjlIqrBBUQEQHniqcVOGsFj4iAl+D5QHnrghYjXi9ntmPexW0XHXRBAOosoA26WPylszkOc0P2mhZ53FXTOlor7KCtc3jOyOaTOBCE3AG9ikNjkqNLLwv7xIRIJAxxjQAkS5Lu+rDGMIMGXAOQGSynVqZzLwpSWYMgyAMw6gsC600ApRlQUoiY4wzpbFUvNW/9Je+uXHtypXf+dOfPD04C8JQSqlLCWADEIKx2dFxfOX6r1zZ/pOjs5dPnl+6vPsP7t4aBLwTit/9wY+//s2v7AXil9d6XY6//Xs/uHn71q/evLSZROfLbKzkj5I444It80QE5csDJnV+5fLpMr/daYYigDC8tbN5rRn/7pP9NIoHzVQge3e992I2/539I5EtMUlUWeZET6bzF1mOefbpwcmdy9v9gxdqMh0TXt3aIeRKY7/bHQzWCHkUJ0naCIKAALLlYjI+f/r0yXA2LfPl/t7L44PDIs8E50zwt95//627byPjX+I3rGh58t84twbrP7n0v/PcV71vMFrRee7GOoSV+tHavaqleStzxrpDfomE+c8kkJSDhMsTAXM7Z6IDZtU4e4/6dYlIawI3cQAdRRH4HXZqxGlX5ZkFyWaxMoIpgzfAJLNRjrkqAuOAqLV2G+5ocy3SWkolpVREKIQCKlSptVKIUmsNCEEYJg1AXkjc2bn8N7//S2/cugKchWEYhIFdGsiYGGyEb7x5OJysJfFkNIHz07VG/OB8kkt9KQxZXo4KdX50KoT46b0nSZ595/rOw/Hsf7n3LOIs04BRPJ/M51n2yWiqW+13b9/49u7G4XT+py9PPj6bRIO1uNe7M+iCkh8+3jt/+ODl4ctPx4uvbq31QWWHR+XLl51G+nKZX+40Xy7meZErhi/Hs9cuXc2v3N4b50sl86I4GQ73j49m2ZILxjiTspiOh2cnh/svnh+eHi/L/PHjJx/df3h0epoXRVGUCuDt99771b/y6420WaGz5q1QxZAmSISkfeVXVTHngt6Ol6C2kIicQQguUF5zhWHVzLW39HrfO2fevdEE1kmy8we9c1Zz4oyuJ22YFarl9cyayWQdcFvN5EnQGwcOmQjKWZwVrP288QFWH19AxhHNZk4m1VnNVcZN36T26XhZFEXAAwh80Ax4GE3nhVaSEZVFXpRlqbUQIgwEY+kyW66tbf+176azP/iTvWeHTAiBTGU5aJLd/vuba6WCvWUxB7z19a/95ntv/mj/6CAvf+/B81/6+vvv7qwLwf7s+Pzw5OybX33/g8u7+5Ppp0enaRz/9MMvysn8O3euPpjOPn70vNVIw36/1HSp1/7dR3vroeg1k79398bz4eRXblyJovAXxYTC+POz4U/PR7fbHX0nGvW7T8az5+WLcDQuk1heuvrkxREFZ4Moij/49otO74vpYePk5PHJSXl4MCuKa5evdjq9KAgF50UhgfGk0QrTZtruPnn6ZO/Z86NFxqNo48135O23KYjspgw2W2O1qkeJ2yTuIpDQ4bHyeIynYcfx1bOoslj9BHCQAq8w60Reu4r3rEVNv3vYoAM3ObSRS1qBrUElAPug4iq+5U60H8nRpJ2E7gD9ZR4cY0wp5cwHZMg45yCYLv22ZJaABWOMM20iZHZ/KEVEoHUhS8pRA0AQBoFggne7fQIqsyyKYqVkWRQSSGlFHCLOVbZoN5pvXrt8eHSuSsU5E0JEb73f++Z3Z8ienZ0/G03V6fHVW7cSxiTn/+r+0ybAd67unGT5jw5Op5NJ7+zlla++9YuT8zu9VtpI/69/+vO21v/Hb78fCnYljf/li8PtOPpLV3YDoJv9zheT+Z+dnK8R/+nJ6IeP945Ozr/91u32G+88f/T09NHTq/1eS2nJeLC+dnw8/MrG2no3/eHpmItoXGSUZTwKW7NpHqSfdbaXhyezklGmP/zi0eP9l+04EVHEBUdNQoggCKRSs+l8OZ/P5gsVis6tOweNtecvT+9ubtzptokqOxIvQAds0bNTxwR2IaTVZl6Tm/p861v4wn9vC1TDXvtYMyQu+PJOnVb+u6Er4VyiFXSu4tU8/tXu+Wkeu+J8M9srH0+rB9LQl8LXwhb2mNXb1Btt2Q/ttiO+np501VOzSZgmUqQ1kSJSWmktzaxSZZkTLAo5lZRGQRqIRpxEYSiVIq2zMl/mWV4Wi2w5G48GrealS7vRR18sdC6JSsQrN26/s7a2HgW/V6ifffrZ37515Vt3rs/y4loz+eG98d2ttTQMbkbBt9Z7P1jMkktXbm5v/+zobA30tVbj58PJBsCTefafHjzd6nfY1StPXhz/84/vh+2mPDvvdzu3+90XPNBad5upSkKlJGXZdn/wl1+7EnP+dq/5//rki48++jxsd7qN9O1ea6PX+6Nf/OLlF/f0u+9/9fqVGxH7BZMPT+U9GRXNHb7VvDzo6pP9h88e6dmcsmXQ7QftTsCQFblOGjpIZKK6V2+xze3ifHI0nf+Hx88Hb9weRKHN3VUJnosq2EdtnBFKDn3eUvS1oeji7WiTiXXMEUF9/Z0lvi+p5/8S3iISjrpNjGllJXj9Kmj2fbJGJHFCRNRoV4m4xaquYS7HQKuzBJz6tu/rFnFVr4cm1RkwxhkDxkyg3tZ6AjBAYynqUmqpSVkzlDQpIk5EQEqrZ4viRU4fjo6+1e8ca70WikEkEo4NHgQRB025VjNZtIke7b/Ms0Igiig+275+XyT3Hr+4FIpmp1X0Nvfj1h8enH3+4vDNy1vrN6/96MXx/XvP9Hj62nqvc/Pm3icP/vjpizIQv/34RdBItzrNNrDradR/6+aPXh6dHw05gztrnTc2+3+ky0ir6dHJV9fX/vLO2t1OQwL93//t758+2fvub/x6X4jf+/mn7//lb8Jsyc5O4/7g4XD06U9+/o9/4/stpSGKQtAfH5+Fm4PF6dns8/saUW/vKKX3gkZvsD3OKNrlMeK4VFkpRZr0moku5HyZdS9vnyslCorb7XLvxdPp4hcn59+9tMVtTBG8H25QZiOXThGjz3xXbpLRcuTYrgYRYyNcMOEMImrq/kvReeHl4gAgvCdk5kTd1PCAM61htr3Vii1m9+02WsEbl4gAJururQUfob3QCB9MZYhS+4fQGLGgN8G1RSFppXnAiXHidstvk+w0u0WadZ4AUEq1LPWS2EjqAtnPRrMNwX56OryWxO+t9Q6L8koieo2dU2qPqMiam+9/99ep29dJ40qURCI4ni3+4vlBvMh22o12EAgOT8+Gs9PzvNv9+2/d7At2rxUfzxeT4fh779zeTqK73VbIdh/PF//0j3/y1TduMYBbzcZrb9w6/K3fPz487dy8cqvV3H3j1j/9wQ+fffT5L/+jv3+yyH/nsyd/9yuv77aT4WL81Uubd/utjzb7f3R0PnyxH4fRP/zG+yWp//Hx4//13uOT8yEfrH/3+uX/9PnDPwQaPHkM52fq/Q+SjTX56Gk/ChcZUiFnaTvrdZL5rDw4WQQJRK0tmg9Ph0W3y0s1PDjevHa50+0eHpz8OE06YfD+xqAGn5ptZkfcfqtJGVPO6l+LHLt7j1PmGvz41gKcFWdhnUJrnEU2RLpySnUWgHsON/qZQC6uUKc6e1Gbf3W7c9U4b+VQcCit2b/abeC7Yu448mbVe+IMGWMcmbYhU1TalYuA2amXkDFiTCOQtIs7takc1ZqASGsk6gbIQA9CETJ8Iw2/PuhsCEFF8e+Oz3/rowfPxvlBRvtK/D8fj0927m5+8K0fp4PPWPy9zfVf3Rj8tWs7t0JBp8O/fm3ne7sbf2Nn/VvXdl6enuHZ2UYgbrebVxrptSSmh4/6ef52v/One0eHk/lbvU633fz86GxG9O8e7k2z4u7Na6zZ3Gk1vhhOf3hwdnNnKx6fqPm8FQVxt/l/+fnnj18eX711c1jKw1l2Y2vj9x48Pn3wRSvkDycLhvyDOzc//dnPijDavX6tH0bdRhw8ubd8+UL1ByyMmUaxs71/eJyfDyGKIc+ns6wcrHciwc+HmaIiiUPBTk7PdbsVhsHJ+ShupE2G2WL5o+cvJ3nhuIB5tDhWQmujGV+35lEAEIB23jmRSZsjOq7FOor8WPuPGsgZDB4EUB3nj6zhiLkIwCocaxetx49s8BI8z5nWo9bVkmp65ZZQY3W7i1BNibglm7QaRmCCm+0btNa6MFtCutYgIGPM6HRt/CSllCliMUvsEQTovuAfdNtzRU3OTws5HU42onAjivTp8OnB8Rvdxgfr3WA8+fMPP8ey+DubvQ86zX/6F18cLbPrafLffuu9S5e2/ulPP/39Fyf3zydf3dncefv1MghGhQwYy5ReRDEUy2eHxzd67buDzr85PD3Pyw+uX7qxvX6rlV4dtH9wcJakabvdRsCvbHTbcfCLpeSt5uPHTxDZB+u9FodZEG698VYnEA9H0504utrr5m99Zbl1WYH+83uPu2F89a03eLMzHo1+609/ErQ7jIvpxra6dpNzoYgKWYrTo+z0PBusi80NkGq+zKnTjjiGAStns7DfbbQaAaLotAVgpmHz0hZjLOS81PoCHlbYyBR5uPo7qP1nQeyeweUvYQOT3uV3cNK1KgtvIzjPujIQ6hjDGvz8PgYrYVj7lVuupMEsbwezkzdzS+bdXKP62lhyzYSay1WDuw2VoSvy8wUo5ktmQ7LAGeOME2Kp/SI552khklnbSaSUIqXIxURNcNRs6LyUshkEy7Jci4JRIZ8uSwqj97vN6NqlPaV+cjJaD8NvvXY1L+XhPHuz29puJE9OR/uzzOzs/+2dDRTiu9uDRIhJXkrGkzReiwJFdK2RnM/menQ2fnz/3z15OVb6H9zYPc/ynz0/EFm2P8/u9DrdQPzw6Yu3ttcJ4MFoPpplc2D0ze99zJOPT4f3J/MsCPibb/z4fPI//OLeh8PpvclsbzjS62tTJf/oT3706cPHv/vDn015fE44kTrttN5O02+8/0H/8jU4OysOXs4fPFbjGUax7K2p6aIoqNWIwpPjJbH48m5jNhnPFhPk0Ou24jAWotVrM6XGy4IYC8PggsazKMMKhmbuu81T/ZA6Jl3Vy+Doxptz/lcXQAfvalV4ALdxQQ0bmgg1OSIkYU09gytnEtdhbWKV9V1KqFYsooEYcG9rugZVvFiDOxprw+zuQGYD35oZ6tHsthdlHJHbBeCklXI7PwMCQ8Y0gEkjKa0lUWgr87RSSmp9nqmXpWA8OMrKlLMn84w3kmYUbiXxRr9zI4mutxp708Wnhf7gK2++M+g8HM22mumVmL34/PPi8ub+bLkWBn/tzpUfHp3/+uWtJC/WABln24342Xj2jd11CSR/5ftqOFwq1Q+DTiAixGBjcF/Rdl7uTedzgGBtcIrQEOxfPno+kXK5zMIo/vruhiL66qCzPzx/+vBJp9//W3evbSfxOM+CNEpGIxWKxt3Xfv3aztPheKrViIoG8J3r14so3Iyjm7duHokAGbvUbEqVv5zO2OmIQGLajHudxXRWnJ6VRyetyztJuz1bLEe5ZL2WOB8WgGvNdO/F4fpr1zXRIi/7UbjKSMYvd0Co+ezmLQJUxW0+mujRWSMjuPhaiZK6uVABwwcuAXws3T70lXlG9VPBX9/qc7soA2yWvE6HtVv6r7WbRjWvfsUA0P6JzbXr1IIJAGbzXUQT6zSekKkrZEDc1tvbpW9aa2kwqZRWsijL+WIxXMwl0W4ScaBBHHHGp1pfX+vtpDEHWA8DTrop+EkhoSx/fbMfMDzI8i/Gs4OiPC3yAPF6u1EofavT/tpG75Pz8c128x/eusw2N39v7/jH55P/8ZNHEeNfv3YVrl7/0dl4f5H9Pz57/J8OzkINnTQ+zPI7vXagFQ0n393dvNJM3lzr/sqV7a2ylM/3N8KwLGWWFXe7bXj4sByNOeK/f7T3clnkivD8nB8c9ZrpRhB8Z3tjlGf8yePZeDSaL/74/sNPDk9CxtT5SC6yNhNfu3ypV+QciXZ3eKez1u50Ww3otIMb13ivv7G5HirZioM4DHcGPcqzRVk2Qw5Ex9PFy9lcO78CaqNvi9PIu0CWusyxhggv2G810iX/zYUIY3V87V42OGWVradxz9JmxD2KvYdUXQ6rq6FJ5+g60SOAezpRlYKqgkc1WvXtcwFfwkoidur54xCRM/MIGrMdmV0IL5Wy9a8EnDEE0EorqVSpZFkWZbnMs8lkfDI8O11kPIi7YVCSjhljyP7qRv+vb6/1w0AA/M2dtW+v9wFgLQq+sdE/LMphXt7ttbZD8UvvvrW/feXfPNr7ydnkrChOi/JsWfzgbPI/PdzvpBGQ/snJsBEGC8Y/PDpLSV8PxC+vd9/tt7+10f+DDz8Tx6e/sbv+w8d7y/Hk1y9vlk+e5tP5J2fTm83GdhB8cOvqZDwr8+J2p/Hw+GwqFaSN33jn7tfWe4M0+rdfPM7ORpmCb9y5+c217j//6N5plpcoFBfrg36nmawF7On58NHey7VGozsanp0cP11k0dkZFAUuFtn5+dl4mknFsiwHGGfZQaaS9XVotobz7GWYXrt9c8nErZvX1pJwt5X0IqPlSVtr0ngw2i7zBk2g3ZJij0BFpDwF+kH1IKu7H1jzklcIjZzbpZ3i95lCR0/kaqAIiDlr44KfZC5H1RsCZvi0Os6jHLyn5VEI3qZkrB5YRbtDpMcw1qEJlW1qK+cRmV1YrEkTSU3SuvPMhKC01lLJLM/PxqMnBy8/fnjv40cPh7liIiyJSsJc67lSD2aLmZS/fTw8XGQbcTSS+ufDWQDw7qDz09Px77w8+1dPD59N51/ttb/a75xr/ZPhBJXeisO3+u3//vbu+73GP/tPf/FeFPyf3n3tb1za+Md3rkxL+X/7F/9OHx0nnO8NJzf7HQj47d31QRzuNqL/90f3//hwOMmzZVG8Nmg/HM8ejmdrnXbUTg9m8xzZMon+/cf3UZaPTs6fjmfrrWY3CbOz4+DJoyxtLAjTfuefffYw23/eyhYTLl7MlrLdUwHb+/iTrFSD996TcfRk78WSC9XvQ6PByuzs2f6Mc0piNpnAoyezrJCjcf70ebn/cjhdCCFUUUitx6UMhIgCAWgtf7LLZv36xYor/PqJilSdF1FB1J3wqsNxwQdy7pV9TIyJZnmqskhyH5AINAnXjlVnitzVLKCdIvbGgHHKHPfVjNLKgibfW5dltwu0XKmrPxgdrGsbhJvtyBEAzENqtNZIxICQiAEA4xRw0lpqtciWo/nyfLY4H03yxTJoNvnmtZ5UgWDnUv3obPi3Lm19uH/8Vzb7gygYST1WWnC2UOogL+fT+W9e2mgG4miZ/dbDvXxd/t2rW7nWv7139K9/+tEO0d/5/rc3InG33/nnjx4zJS/1Or/38PnfuH3laqcRhCxot17rtvZOFSP65huvncwXB5P5u1d294Lwn/34I0a0zPPtNBZB8NPTEWolFvP3d9Y/Ozj5leu7x+fDnxzsiyh6ucxn09n17fXzkyM1WNdpqvLia4PuT4r8cHJWdgd6mV1VmifBpw9eBpyJZmPv+Ly9vk5PH8oXL4CFEKei39WnZ3g+Ut1OO42LvGDZfCFl3EzLZSbiYDmdtgIhlWbEpFKzotRE3EACcXVraXRekUafcXLf+iG/wCweQnVN6N9UC57qUSF7G0ddTrVaBBIQWIBaiHl6rkGzNkuICIhh5bG7s8CWz9VMZnKumdtEh5zpXQEYa1tMoTufyO6IgnaFMJpdwH1ZiKFWAFSKFkWRLbNlKeezxXQ6L/OCNDUQR+Ph2WLRSNIbSbjfbIzzfBAFz2aLv3Jl63/67Mnv/+hn//vvffN6K/39e0+ePX72j3/tO39yePrtrcFoNP3dp8+3W43rrTQGKtJkPjwLArE/XbQbydvvvrWVxHf77YPR5D/uHyHnbDF7rZUMorC52f/5yXC8zJ49ePy13c020D+6tPGHQL9IgkfjxZ/uHd1KwmSr/zuP9qjd+3g4+9EvPr3/bG+xu8Nff+OzJ88PlTx+/mLz6mWxfamUcO/ho881KME7a2tFo6tPR/zzTx/tPWev3YZGozgbrbUa5Wx2Pp1yTXpjhzY2BGJZFuF0BFKFyKaAvWa82Ntb9vsyinvdjp6Nz4m2+p3FbNHvtYHg5XRxs9dhjPmxqVwiAL+U0iBgBbl1LluFKdbR45BX94+8ljdHO4YCJG2fJu/oi9zNhL9abSG8jxSAe76XW+RfD7JahW51/mqzrVaoJ2a9Y+jSAMb2dqf4qJPT3eDMU4EImqSUeVkIjkzKpSzOx+OD4Xg8nJgqJSklaWKADEEvs/zk8EW73+hvXum0302jaam6SfTZePb+dP7Weu+jZ/ufn493uu3rW4N7z/eeHp3s9rpfnA5Zv5PleRoGL+fLWSEL4NMo7SNdWuv90cHpkdTH9x78xuvX39teezlfPp0u7rz37pKxo0UWM3yr1wKiA33zt1+c8Cx/vxn/6rVL7W73Z/tHyeHZt7cG7/Y7R8vsoNduBuK/+85X//jgJBHs8qBbIF3rtr9zZefT8eTFcDzY2exzHCKPkuR6HAaCnutd7LYkIZUSAoECjj75JGykFCdquYT5XNy/B2EMl7YDpeXOjiAqigKUZlqzIOw0kjTL5kW5jONCiLVOGIaCEANe3wUJHJV4Zeuw5FVijfLqoKzDl8APtz/Gesyv8i6aZ2lbyJstUQ06tYsYESIKchvUuFbaIuXaRKkZumAJz6wYJjCbjqH18WstqJgXAc2CDSRA9E924ASIdkMHIw8fOK7NV2SMCeSAUBQFap0tF4UsT0bnJ6fH0/FUliUQMG4fOmsWN4HWw71nZVHwN74yThJkPOH4TqvxdJn/4dn4q93WndvXP53Ov7I5eLoUjStXPjse/p/fuPlsuvjTk/FiMn90Nvz69nojjjd77Ww+eTmZXY3CX7u0MSzLB430//v08O1e82ojaQveiG79wcnwlqT3uq1L7WYzyvqM3Uzi33zrllTlH3/+8MXp+dvbGxvN5NHZ6Z9//vlH9+5f3hzI61f+3cHJ7OQ4DgTb3u13msvx+Wd7T54/fNhKkmgwkGkSZhl9cVT2252y2BRx0oqXo5E6HeciDDYGRRgiF4HgshHxQBAXGkAUWUFQHJ9AGLSaDX52StMJIuYcIlmWp6fTZnuWxkkoGpwFgsdCcES0WUpPJaveTy2O7YbEbe+4aoZqupjHNuPnnSTnU1viM/FMUw9l4oh2h1BA//gd24R/9lv/wSwPAkeKZqGFybdWUX1to6jmxuaxG0ZBMMadUnahAADSimwUyHl5aLfXNUYCQ0TG7N7J5gwCG9Qsy0W2HE8n5+Px+XQ4GU9kWTICJcs8LybTyWQ6zpYZAgRC8EBwzrTSSmsETJIYGUPOB5ubV+++HW1dL0SERP0oWgBqgLIoIiH2lvl6FGrGJlJdMo9u4Ow8K07GU57ElxrJIAoXpbw3W76cZ3c7zff6raXSPzuffDGe//1r22tp9Mnp+ZPh5FK/myAKWfz48wd/sf/itUGn3Ujm58Ozg/3xwUss8ka7qRtpORrHHJMyu7Q5SDvNaVFmo8nLg+NZpuL1tZjxG1e2H9z7gqTSWqfNxmKxWB8MeuuD8fERaRqNZ9liITgvEHnS4GFIUi9bHb51ydS/8bgxyTIkAMExCBLBaTZfMCaSeBBHyzxXgI1QpEkYiGA7Dtth8Ob64Eavw6CSf50eyVdlQAWyeuiwBuNqcQRetBUvHukvaBCozVIgswlZdbvaM8SIRD2F6FW4Jm3dbfsYdq38VvBuXtnZhUCgmd0s20ZMNWkCrbQmIjSPmHHARQBVm2GGeJXSRk3LUipNUpbz5XIxm56en52OzmfjUZ7lpSyLIi+yXJYlaQ2c8UCIQACilFJpigKRNpIgjgCx0+5sXrume1tLHp3m+XC57AueBmEYJS3BR1KlgqdhECBsRkE3jv7j4Vkm5ZvN9N2d9eeL/Omi6IZBrmQC+o1+eyMKfnY+3B+OSqW/2u/eOzn6n/f2954/S89PppvrOk17XDenk6/K6fTzz5+PJnme72yubbZ4JJqv3bp67/GzIgXB+Vff+WpZSk302cNHe/svl1mezbOgEe5ub1watIOrW6dn40GvNRpP2kn/+u2rf/6DH22u9+OQt9bbx6dqluV6mXOSLYgKQjocZk/uMQRgjHe6zWxJpUTGIQyJCyjydrMVrK2HYSDPz4QGHcWzKOJpQ29uta/spmGAXnnBRQiiz6pXg7USbyGXOvIj61m2ArQ/y0HTRF5tZZ790u0pV8UQVl5CSmlvbc+ynoqpoWNIiKjcE2CV1syt4eREyu4Uh9p562QedAB+vZBSSkqzua/WpZJKSgD7pFqyTwWlvCzHs+loOs2zLC9LJaUs5Gw+G03GZbZUZSlLaTcE1YQAnDMRCMa5JI0auGBpGvabKQ8FRvHm2nrjymt78fpPDs4wf9nncEmNh6OzrNnLm102HZdRkwAeI1zdujRTellmAOzW5tY0n//kYP9yq/F+v/v49OiTg0NdZO/221+MRgfTKVvOscgeB2EjYleXc3r+LEK8EzRPz44H7dbbb91+vLf34ejs8o3do9Pz/mBAUr37+vWP7j86OD67fuXy6zevPN9/eT6f7e7uChEwzsusiMPg6s5Gq9GMEAJkoRBa6vffeW9UFE/v3Rv02u1mmhfl1UtbGpDOhtcubWV5cXw+4owrpbjAbjNdZDnMRpyxrMg5w3wx4QCMcyoWej5UaRKSzhfLQlOcNhZhYzIcz5b5WVZ8bXv9jV6bu/gLw2oLLefgVkuIVuJPZC3EFUh5C7YCcVX4bBKe5kTz/ENy+ScX90RnG7odmEkTkVBaItgHWDGG2vvfWnNEs7rKPT1bgdmdnuyGvyanaUpFAQg0mOL2UpayLAslVVlmebbMllmeZXmelfksW0opEUiTlmVZSCmLcr5YTheL5WyuitJuCWqeO0vE3Tp50sC0RgTGUAjOuV1HH8VxpxF306TdaoIQFARs4/LPoXPvi/tXhnuXIiXzJSjZYkzIZXt5Op0uWFEGAZMaFiePG1HcScJ2q4MHI5VlrxeZOi7uPdBRlO6enRXnpyeCzUaTq4MecX4+HCPATKrRaLLe771x5/Ynn95b3+xvb27MlpmSspHGh2dD0nBtd/unn37623/4n5SWTHApi2VRPHr+8uXhyecPnvV6nWKRkVJhGm31exu9XgDUabeOTs+nWXgyWzz47LN2I+m125e3Nw9Pzg5Phy9eHAVxyJCtd9vHJ2el0o1mYz6exEkcp/FwNI0QW712WUpcZKVWQnCOoMsim+luI02bDS0VIgSQqXJS7D39Qspn08VfvXn562tdtgowS4dWQ37JEmRwURrHrUj2gdeV1epVpFf97mrWojDhKyRbSufKUip/yDNoCQACEEWgNBBo/yBRbeo2NCmlbHDVTh2zolKhL00iAAClZVnKoizmy/lsNpvNZ6Usy6KYLmfTxVIWWZnn88VimWdaE2gt8zKXUpZKuwfOIENun2pjFijblDworQk0IhcoOAejLBiLwqCRhJ00bjcbcRxjHMet7iLtHcyzG7S4kuhyPGZaFYTEWTfBSOswwDBOwig4G8+CfK5UTjLMF7OlpkajIUqZRmK+mE6OD3Y2NkaylU2nd65dns3nnWa6v/dyssw5MMZYp9v5gz/5s2aa/Nrr3yLS//K3/8Novmg10q2NdRHHAYNOKI6ns29+8FZAsNDqeDjKtC7yXM7noGmt32k1kkYjPj4bCeQ6mz1/efTe++932q1///v/sdtMMylzonkuMQyhKN9+87W8KOaT6cvxZDbPOt1Wr5UEoJeLZRSF/Vaz12k1kvj0fBgGQVEWy0IWUqVRAKRJSS1BE3CBXKqCyTjSfDaccv4XrcblNN5JogpQKxissaFLRroPFX3W9h686MmjLdp3gDZWBVK1nY2PbAISahO5r7xkxoSxQIgbl98WUxLZBymCW1ZhlvdWZgq5DTdriUopZZZnk+n4+Pzk7PxkPh7LItdS5UW+yPIAmdRqkRfLotSlVFIVUhERN1vbMcY5E8yn7BEQTWYTNSEQR+QcQs5LAKlUzMNmFMVR0IzjOIyAYCFlg4s0TuaM3e6kR8t+fjoJgiDP1SCNoyDoJMlkOonDcGtj/fh8FAouhBBax4jLPG9FkSiyzfUNEYTzZSnl7MEXD7bWN7a2t8psqaRaLLO0kbxz906/1/3wi3s/+NHPVFn+5vf/0nA4BMYODo46ve5av9PpdS9vbGitHj58HoRBwNjZ+agoihfHZwcvj5SU/UH3vbfu7G5tkFT7B0dT4vcePSkWs1/79V9f73V/8eM/DzhLk+Qsy8Mw+OLJ8wihncbns/k8y5IwjMJwrdduN9Px+QgB0zgSnIdROJnOkGG71Yzj+PR8hJDzKJRKkgYlVZrGhdRmG2ulyuloWC6L+WjO4ni6u0FpjK5g14MTfbWaNzXrxOjIk9w+oy40qV2U08cjrdfjntMBmuwGCcavB7cxGWoAl3x3XpsWBv5ak0b7hHcwViSCBs0INZFyRUNKSSK7gI4h5lyAtSeoLEtZlnmZn4+Hh4cvp2cni+lMlbKUKtOqVIoRlER5UUql0S6yJ0JQBESaA9MAGkAAlQSAIAAER47IGBM+ikEEQAEP4igIo6ARRUkS8VBoLngYxs3mUbr2uxOVsPmNiG80wvOJ7idpu5EA6aPTMyC9PuiGcdxpNvIiX+/3EPH5/kvGeWdjrSjKo7OzxTIDxP76xrVr11SeSa02L106Ozl+svditlhub25c3tl+ur8PSEqqLx4921objIcjALx+4+pX3rjz4aefvQTYXuvPpjMeBVmWX7+0+/OPP3vx4sDE7zhjr1290u92P/nii9licZbJl0+efft73+l3upgvRcB7G5vTvJiNJ/PpvNNIllqNxpNlXiDptX53o9+5vrsxns6KPF9mRZbl/W47y/OAB6BBar27tbG1Nnj0fP/g+KwZx6WS86IslGokcRSKvJAh6AhUPhljL5Bau0eZO2Ss+jpQ24XG56Vx9RFt9lcHYiQDwirxCWjMTASt/fJfN/7WAYIq1O7PQgAU1jTVWqPSQFJKX3nKAEgIEzMqykIqleV5WRaFLMsicwuFCBBLpbI8K6XU2aLIFrPJZD6eyaKUpEutpNK51mUptdJSSjCLQ2zk0hUWasUYAHGJKIkYAucsCYTg3CyOY+bZx0ARMhGIOAyiKGo1m2kc5UBBGK5tbNDgyk9l+mL/SWM5/d7lpIk6C4JIcK1pMp1NZ4s3Xruxu7P9/OXx8xcvACgSnAiajXR90Ot2Wo+e7J0NR8dnw1YjVYCdXu/uzduz+fzTz+9P8/LF8XC93WilycuDw0iIACAnGk6ng7XBeDpdFsV0PN0/OrmyvfXhvUcPn+31d7azxWyyWLx1++a1q5eeHBzNJtO007i0u9VI0zCKAs7zZbb3+Pna7s5wOnv++OEH77/7jW99689/+vM//ejTfrcZtVMgkpJarcbV3S3OUANs9jvtdmuZ5VKqUspGmkZRIAIRx7FWMi9VnhetZrqzPsgLeXQ+jAMRcM4Qi1Ii55yxXEoFiKBQq3KxfDlbXmulfFWlYw2XUENh/ScPJheJtD/bA43z4+qaq+oN0ohu32W3sSi5oqpVU5eAQJgrS01EpEgXZVmWpVKyLAsOKIKAcU5EWsk8yxZ5Ns8Xy/l0OhmPZvM8y4hIA+ZFCVpHDJhSAoAxLMqyVNKkJ0mTlrKUCjUhgEbQRNzlOYmII0NSqJEYA4KQs4CzNAjSKNKkF3kBpEUgEJgG4pyHYRAGIk2TTrMxnc8ni2UUha20wQc0EOLNG5dffvHJ2cFep5220uT47DyNwlazubU+2FjfLDVurg0Q8Xh43u/1sqKIw4Ahe/7iKI7CVqs9ycvj8Uwg3Ly00wjZoLt9Np09+vQLu202UiONbl/d3b919Ucf3d/d2uAAHDGJgs8/f7C+sUZRMFvmB0fH77z/Xgh6d9A+GY4eHRyR1qR0Mwg/uPva9s7u6dEhZ7hYZFEUh4y9fPLkm2+/OZ8vijIfnp6urfeOjs/W1npr3W6v01GkdjbWpVKCiwBJK73W743Gk8PTYasZa61ms2WeFXmeh1H08Nl+lhWB4K1GMo/DeZZLwEgIpZQmSqNIKlUQKQVMSV2Wv/9k/7VuczuOtOdI7Vc9OgjWV1vYx6rYWOmq8iewD4Eh/xBEq/cqe7By7cHZojXnDNyzu215gEBkZJ65XqpSyflivsiW2XIxmc1Aa855EAaIyBFBlfNsOZmMFpPpbDafLpZZlhdKm4amnLMwCDjPQZOmoiikCTMBSEVmgy6bbQVAjvYhBwTIMGBccKbRBF5ZFIoQkTMslMrKclkUseABF0EQEJIGUAg5QBPZYpnNF0upVIJMypKVy+3GIKdg0Yjzs+zxZHQ8GseBaDXSjbW1Qb+viRaLRb5c7u3v9zutMs8E0Gg4C8OIAeRZnmfLVIg8CsIoWd/a1oTHJ8fZbNzttUfDYVmUYRA0m+12q722tsbZg/l8gchK0pzzLJs/Pzi+vLuZpM2rV5KtVnN9vT88OX7w4vH58Wk2XQjO37x5dWttsJhOy7IMgiAMuS7L4XB4+85NhtTrdD7++NPZfJmEwZ0bV9+8+1qRl81m4/TsfDJfXN+91EyTxWJeFFlR5M00aaSZVroEtczLl8fn82VmiQxRcB7HYTOJI8aXpczLJSKmOpJSIWeNOC2UEkmiFJ2PJqO83E1iiw/wGtmqcxPkAXBL6WoZbKOj3bN8zOoL7crsCBHJFSNTVaVELvtvyXole2V0qtulg4iEJm1L0ZUsyqIoisV8djY6Pz87z4tMgw4QQsYF50A6z/LFbJ5n+VLKZVGWUhFAwDAWQgDIUpZKlVqTUlIprZQiUkpLbbOnAMamRPNoWq21WQAPnCvGjD2Sci4QNUBWllJlpdICkTNknAWcl/bxSCiISMplKYGglSSDwaC9uf2UtX/3fCqnw/fzUTPkJ8P5bJFRGj05OBr0unEULbLFaHTOGV7d3Tg+Oc+y5dbG4Ma1q51uf75YTqbTHaL5fPH4+Yvz+WKZl5sbW8D5YG3t4d5LIhJCcM7DKFkqdWVnfa3Tevni4Hw8vHH1+htvj1/u7R/s7W9vbrz/ztvNZnN0crqzsdOJoiBgj56/DJP47ddvfv1rHwRxslzOwygw9f8sEMBwdnraa7WIi/7G2ht377QajVa3vZzOF/n40uXLN65fX2ZZyHkjbTSW8+l8JpV6tn80WWRFUc4Wy7KUSmkbrAEATVLpuVJBEAjOdZFrgABgmWcMIEzifDbXPEz7g3kUJXGUCqFddKcK3tvMYG2jjVoyp+7gk4vuAwJoDW6xsg2tG9iRKbcgALelPKBGV0f8SsDfx/YFIiolS1nKspjN54v5fD6bz8aT6WyilkuulSKSnCFjUsM8z8s8L5WWSmmtOQEiJIxzBKUVIJIGqaSSqtSkpDYBeUQMze4TyMCWh4LZlNY8m9Nk581i0dI9d8asfwqY27fJJHwJEICTJg3zPO+kjUaaNBuNKAohavTaXRyfyNMTORo9PjvMizxIYgVwY3vr1vUbSdJQWidxEgVikWXzLNte722ubwzWt4KoofT5YpGJMCo03r3z2vOXByfHR8046ne7rTRpthpnp8ODo9PpbNbvbybNzqA/uHNj96cf3X/+4uDWtfh73/zmk63HP/rFJ6DZ22+81Ww05WsyYKzsd5+9fLm+tRlE43feegOIsmXGGD84OjobjstS5vMFIF55685ar3d8cDAdnl3b2TGLsLWSN65cKvP86PhwMp4ss6JQdH5+Jjju9FsMdTsJIQnXuq2iKKeLbJ7lpVJKKkQw0z0rinaa9JtNwwpFKRd5vsxLGXDNGJcq7SffvX5pJw61UlWu6BXrE2pOknGRwBKp3/5I2yXJaBeyuWwmeZiaOJTP2ZPWzDpolUWLtQ3hDSsLrXUp5SJbzuazk/Oz8Xi0mM0Ws2mxmDOlCFFqPc9yM8OyvFBSEgBjTJitHAAJoJCq1PZBpdzEoZQtkUPGwkAEAdeMmXVt0ixf18SQcbMPqNbkHjuqiEqAkEHAGRMCAZAxpfSyKAutBeMEUBApUjFApOSg2Y2TJFeywdi4KEUcJoKdHp8kJKMoTuJod9C5feUyY/zw5Gh4fjYaTUqlwoALzqezpdLAg0QE0WBtu9vbmC/mWSkDzoXgw+FwNh1vbW5eu3R5b//Fw3uPs7KczTNCAk2Dta2rly//7OMH9548H4+n3//er6RhuLm5vrE50LJc6w8QkQueL9ud9kCww7/yy7+UBmI8HDa7nZCJsizmi2VeSgDY3lq/de2K0qqZJts77+ZF/mJ/LwyDQb+7zPL9/WehwHYj2dvfe/D8aDpfvP3atcvb671WNF8sc6nDMN7Z3l4sl5/df/T4+UtZlou8kEov80IWUsUUhKKUCgA0gCJQIqZGlxifa4o1dQLBkCqv24eWHFJ9vMnkkmw9vFXYJvLoiNvbl+DQqXX9OUYm6e4rSyrCdHUkzmytdvEUWqqiyCfT6dHZ6cnJ8Xg8KpZzyjKlFUPkyEops7LUAKXSpEkAcM5CzgCYVkoTFFLaPecEF4HgjEEhZV6WUgJpJMoBCCEOOTGmSlkqBYicc84YMvPgTRfGReSBMOWljLGAcc6ZMVOkVKVWjHFimGsCwEaatJqNfrcbxkkhJWTTgtJZrqVI+OBSSsvs5GWe5be2N+IoODw+/OiL+8/3D5ZlIaVuN9JGFCRx3Nw/EFEjjeOsKPO8BFCNMHjx8mCjP3jt2nUiPZ/P0yT94O23tJI/+/BTU8NbFosyX671un/7r/7Kh/cefvT5/WtXLokovnvrxkZ/kERhnMSMMc54HEVf/8Y3ci1J6vagNZJqeD48G40Xi3m3074bh5e3t25ev8IZ29/fD6KoVPKLR0/2X7ycL5ff/cZXEHUzDZMk3hh0XhwczMaTXr/7/hu3Nza34jjpKsVFoJQSjMWd5nq/vVhmjSQ6PR8fnY1Kpcu8GI4mCICCAwAyBkzwuKFEQI32RERc6ZYQdaa0oXVjHDrutBCygNKueNRp+1rRKLkaJVDOeaqVnoB90AC5bJF2BRmgNdmqKp9lNTboZDo6m4xfHh+fnp7MxqMiy3RRgJKkiTiXpAuptCJNGjVxhmZHGmOjmOdoEwBnwIUIozDkopBykeVFURrTGoGAtJQyB+DIlCaOLImiKBSylKXJjUoNSMQ4cIYIkWCMcxGIThIrgKJUpZQaKEIRBEGBwJQmZI0gEIhSK6bkMlu2kvhrnfjjvLgn4vHNN1U5vy2gODncOx91uq295/tP9w6yUs3meV4UZ+djxlgrjbVWy9lw0Gkdnk0Vss21te2trTt3bkkpJ7NlXmgmlFSy2+197b13yix7vr937crlIAyJdCONte589xtf/Y9/+kPB2WI66Xa7jSju9QacQZJEnHEA2Nzcvnvz5tOnjz+7/zCMo/lkqkm1GunN6zeDIFhmy70XL37+0Sc33rj7+qXLy+n4z3/+F8+PhhGot1/f3d3cSONeGEbtduvsfKy1/qX333znrbd5EDMRj0dDAAaolC6Pjs8+ufe00UhbrdZskQvOG3G40LooSs0QlQICBIXtpg4izYVqd7e31v/K9Z231zrGdkVbiWkxQ3YLGQI0z7T0vrvFrtP3jget2kfn/dfj/LVNGWv2g0/vO+MWwFoI1j8BBPHR558OJ5PpZCzzpZaSa+IIKAJAVECylIjEOSJxwYg545e5QDtjLA4Dbda1KT0vl4tllmW5if8zzgQyY5dIQM4w4oJxnkZhIESBqPNcmVAYY1xwwZmZe4ngjThinC/zfF4UUlHAMYmjNIkyqRZSKg15URZFOZ3NE0IexXGQrCXRb261QITPs3w8kk9Ys8NCHE32T4Y8ihfLfDqbc87jMOitdTuN5LVru2+8dqvf64og6Owd/JPf+r0g3rt77fIH7761tj746Onhzz96EPHiv/lb39cgGu3OX/r2t2bTyWw2XdvYjuJYnqs8z8Io+d7X30ta7aKQO5tbrV4/SVPOuSu+hm63u7W9OxufHR8dTGazZhJvrW2sr69nWfbpJ5/de/pccNgYdGfD04NnIo6Cv/rLX//F/ecaYHNtjTOBjDUb8f0Hj+492v+V73zre9/9Xqe3rrUu8mI4HMZRMFsUw/PR/UfPJ7P52qArlUyTqNVI0zgadFpKU1GWi6KYzpeax7y7RlECLJDA3t0Y/PLOhoGYtRstKtHuZmM4VWuwTwM0Vpgr/TXVbX7rMWdZevhWVigg1OMADt0O0DaGiqu75Zvovbj/+KHKc6F1yJgABEQym4JzxqQSnAkec7M1KEFeFHlRaOeREQITQSAEcpaXZZ4XeZFneVFK++wmJAIOXJg9VpkQvBFHURAqrRExiaKAs0VR5lJKAhOnIgTBWCMIQiFMrACIkKEQXAiuCMwDaqSSGjRyliRxHMfLPBueH7W77dvt7W/IJo7pxTI6ba2L6ai1PMry7Gw0KxR1uu1eM9FSfvP9N69dudzrDcIoVqoMg/DWrVvXdj/++N6j/SQKv7j3lfDNm1e2fvzhgzQQg8GAsRgAdacnZfHk8cMvfvzzr7z3/tvvfWs8OhsPT7O80MA2tna7vXXGeBTHrljWJpL7a+tJowUMtVJbmxvr65vno/Mf/vwXn3z4edzpNNstfXjSCNgkZHmccM7eu32Jc2SIUhX5ovz4sy9++tHDv/Ir3/2173+fMT6bzQBIiCBJG0cnJ7LIHu+9fHk6FEEwXxZxFANgEkdm6/RSaUCc5AWLG9jdhCBm7a7iYavbeWOtIwxyXBQdXRTI1GqC20jGZIdMYQcZ1BnvxkGTLE2iO9aF6O3WjOSNWqt7fTmfy1c5UcFKuTKgkIulAC3sIzq96kapCYhCLjjnJtOUF0VRytIWdiDnLIrDVpoKzhFQMKGlmtvqT9Jamx0cgEEYiEgYpS0EF4o0ADDERhIpHQJbUgakNAcQCIJzzplGKksZMJYEIZGWRIr02XzeCMIoCnkgBOPdVuvmlSv93mCR50maJnES8iAW+m7KZ6rxbrf127qM8v6VsLXZagh2QqQ2W+nl9X6v19tYXwvDSBEAchEEhBAE4d/7m785/2f/6uGzvdl8cXx8evvGlW+8sb6zscbsA/hYFHGZT8fDsw+/eNpb382z7M7dN9e3Lj1/+jiMk6TRFkJEURxHEeecITJAjaSkCsO41e4209Q8u0RJeXh0/Gz/QJLO5nMAGBfF6Wi2Nmhv9LutRhwHvFR6scxHs8Vksuj3Bv/dP/iv775+lwfxYjpSslwu53t7z/YPj569PMjyfD6bTybTXquVRqGUsihlo5FIqc6H42VZLotCKSUaHaW1DkQep81u52/fvf56t1FKiYyZCJHDUBWHh1qwqbI17XuPTct+9pkYFkMuFK+dEkdHujaw6jKb6DwudDEtF1k3jRECNDdBHLfujpmNQpQWDAVnGmhZFFlelEWplEY7A0gI3m21Ntf6RDQcT4ssF0LEQaCk1JqYyQYwxhnjCAhIWiFx4+4IzsNAMMYLVWgixpkg0giFVpyzNIqiOMzLEhHSJEKAolRZWRaktFLLPIdAsCDCMARCSbrRaCCy8WSUL5dbQqRB53Yz/Xg8XhMh27n28/sfbu49v7LWu355O0G2vr6ZpOn+i5e9bqvXG0RRLMKItC4V9QYb/7v/+u/8i9/+nU8+u392cn5wdHLzylY7YM+ePm61e81mK5PZx5988uG9p61WtyyW//b3fvzg4cO/9/f+wdUbt8+Oj0LGojjt9PpBEAjGzMIBk0ojgk6332l3ZFmkaTqfTfdfvMznyzAMpVSz0QiI5kCn56MHfF8wxjiP4mi9371x9eqv/co7b73xVhSnJydHe3ufnp4Pn+/vLcsyWy6fvjweNBMGsFgsu83GlUtbjTSWspRKDcfTk8m0yEulZBnEMNiUSavIchGlcx6sp8k7G/0AmfJbHxlXxlKd5UsXUoeaCieya9vs4g1vkvrYqFnDSVjVgRhAe1fJl0Q5LrbBArJxKLSPWAIgIKGkVIgISEiC8ZAHjHMA4EwTQVaURVlmRZGV0i38RSaY4DwUIuAMkTjnQSCyLAsED4MgK0smlQlbcIaCSJaSNEWhMMF5xtBEeUoli1JyZGkYlkIXWmuiQARRGABjQRCEnMdRpLRWtBQKAwxaaRJGYa5pXsrZdDocnXNOSlMcJePxKA7EcjBYUDJXwcEie342Srvd3btfhUc/bTfi3c3NWZa1u/1mms5m8z//6UeXdrc21tcbaSOK4m5/g4uw1Rn8t3/373x+//7PPv70+YuDn3366LNHz9vtZpqmoRDn56PjszGxoN/Lnj1/lufZaDx69+13Xnv9zVk05EHQ6nTSNDY2iR94RCzLQmmVNBp8eEZaTaeTg+PTQukwjERARZEHQdBK404r7baa/W4nTRs729uv37nb6fbm8/mDh/dPT09Hw7PDk5Nllk9n81lZAuPrnSZTajiadFvpG6/diKPw4Oh4/+B0sVwi40iaVIlRqnuX8iDWRKwVLqOYgvCDrUFDMLtEAlxhkSYA0OjyR+i8bOeMg9PgBGZz5QqeF7LzVsVXBSTg0KixngL1J5CvKfF1JNYJE7lUGiBENJvGRoEIAqGVLgoqyjLP86IoCynd/olIDAWKNArDKCzK8nw4jqJAK2X6RESkNAdExs3m3DlpxnjKeBSEXAhkGHBu1tAFXGRZwRDTOCq1XshSaR2HASIulzkHxIRzzuIo0gCKdFnK4WIZKcUAGJFZx7LMsqIoAVicpmmcKk3rrCCt3mrGzy5tb8fJrTTak4vR9LCb55PZdDqbbm3u7u7SycnJTz7+ott4trnWXet1WieH3e6A8YCQ3755+83X747Ho4OT46OT0/lyeXR6tpgvgzDZXONBIDhjKkrCra2drc37D+/duv1amqaL+Xw+m8RJHLrAjYkYSqnLstRKN5O00+loXYIqTFkjB9AIAedX1jtvv3b19dduDza246ShlJ7NF188vE8ERZmPR6OyLONAbK/3F8uciPphGAb85eHJ6WTa6zTfvntrc63/yecPv3i0BwzjKGrEkVJqHMezsCUBismIdbq6O9jttb9zbeeblzZMEZyBoS2udKGjClPkYkuVUehodCWf5DJGfim5NxGoxppk1qXZcGdVPuLpuYZUv/xJMGQCgQPEIoiDUDCmlVpkxSLLZF4URVm65w0CAeMsCEQSBlEUxVFIQMu8WGY5YywKhFSKI3BEYkwASKlyJRljwOxDwky6qCjKQpYI2GrwRhopqZFhgLzFmAnpGUMCOQNEwThnjCPjyErGllmeSdlqJN1GI4lCxoRUMJ0viIvu2nocJ9lizri41GhebTY30vgX87IkosHOsNFqqglNZ4fD8635NIziwWDQbaSCY16o+09eaHqOhBpZVpaMi521QavZGAzWu+3Ozsb6+3fvtNrtKAyBdLPZ5pw/e/rwj3/8IRAxVKAKxrlWeb6YzecN3mpxt+rVpCzKImdcrO1cSRuNxXQEpJMoVFopJYNQ9Nrt7fXe1uZalKR5nj14/HRZSMFwNl92O61eM+Wkx5MpAyqKcp4V/U4zK9XR2ehsNNla79+6emm5zP7khz8fjqcgBBdcE02XWSGCIu3LuMWJxSJcNNuNVvPvvfPa24OWts/F8LFO6747l8ijzmpdr5LNr7YcqQqLmrVrNude51FwgSpnD9i4fc30rEHfpuy9WUEAIBKzQJMzxpiUSipVFHKRLfOizEupTZEHACAILsJQxGEkhACGZu2Hto/owjAMmZQiECIQZHZFNAs47QNLqCiLrAQkEIwjw6KU56NpFIZxFJrWFaUkoHYYaKK81JyY4BwZkq1W5rIsA864EAqw4Gyt2RgMBgQggfqDdWBcI2s1mygEMZ4myfsxa0fF42UhARut9uNRHNJxYzhuPn/ywZvv337tjSAMHzx62Gs12u32+WR6dHiyKPIwCEopi+VCKl0qNV/mUlO/09pe61+/cmVne+ta2mp1O1euXH97OPzxZ49PBZyeHDTb/SCMhAi0UmUpMQg4Z1KZ5zyiCAIRBDQbHx8dnhy9jKKw0UjNyqxWI71yaTOOAkXs6fO90TSbZ3kplVQqTeL5fCYJDk/Ollm2zItBu1USHZ+cTsfTNIluX92+cmn7fDR7+vKIEBuNlCMCQ5HEQ+JjasxEpDUIpEyEZdL4jWs7d3pNpezTZRjaLb4McWinbpHQrn90j0YgW7hJFd25Pbh9QZOPENmNh92xRABGszNCMFstmZXyPk7lZ4UDNyJp8/RNJoihCEQzjUqlZ8tMKkVSyUKWZVlK6ScK54wLlibJ+qDXTOOiVOaAQDDGWByFYSgAKImjspSkNZAG4L7RpZRJFEVhsMxyAkriWHBuqvQ5xyQOT0fTWbZMomhRlouiEARJxBhH8+iPOAozKbks0yTBQDDGm41G0kgbjWar3Vkoef/h/fW1tfDS1SBthWEYx2kaJw0ukijuz7NhHh8tM6m7i8uvNYvpXM2/ePTF9cvX37j7zs727pNnT6aHB1EQ7GytS6nSJELGxpPZ0dm54GJr0Jgvl9li+ej5i+f7L/u99uWtjffffqvT6TaaLUbqP/zpj+JQ/Pqv/bUgbDTa3SCMyrIkojiKzJOdGGNRlJRFLou8LMvpfPH85cH5eIqcpa1m3Gwo4MDD0TRrNptbm51SytPTsyd7+804VArSJO60GuPZfDJfjMfTVjPZWetfeusOIpVlOctV1Om8s76OnJdSglRBFA7ny2f7w6kGPZ/xRhMaretrve/cvPyt7QE3qCACArMPmNFt2kXTbVCSmXXrZLFrF0Rax4WY21KZ/JcubumIcBVyaJU3+WfBgLbEbbzp2i4SCKBtCh1ACx9r1ZpAkyxKVUpTzImamA07oeA8DII4DJMoSpKYIDfPfmPIk1CEUSilBC0FxygMi6IEpavNShyNB5xJIRhiyJmJkHHE5TJfLPNlngsCXcrJZJpJRQBSSq2p0UjjCKMwCAMRl4EG4IxHSRInaZw2lmUZ5jko1Wq1ev21brvfSJsiCKIgYpyLgG9E4XY7PZ5l/8P98U/Pxq2g0dncGBfT2fDw/NGD13Z21vtrX1vbHA7PHjy8d3Z6XBTli+MzZCwU/Prl3WWWCcZuXb00mU4WWW60Uhjw0+GQiVBr2txYOx7PHz7f/+rJ8fa1O0EYg6sSV0qZrfUZwzAI0rTR7q21z06KstSAZSmTNO11O0kcD9YGu5ubWlMYiIPDw363szYYrA36Usmzs1EYCNB6d2Pt2tbabL6QSm2tD6I4ZoyHTZamjaTRjMJQCBEEotT044fPPj3NZixhACXwZdJs9/v/zVfu3O6kUmm3IsOqVJfStAkkF9m0jyFgTvGiY04bM9VOudtfccUUdTFOs9bXFX+a/UIQzO7AJrBktxAnv1sjeXsDzHVQJEEQBFyRVrJkSAhUSqm0AgNsRMZYGIpGo9FKkzAMlFaT6VxrzRCSOOIMw4AzhELrUirQmnMQgmnFUGvBuUkpcM4LKfPpjDPebqSc86Isy1Lm2jzNjIBIa5JSMc6QQBItizIuyjhWSimgoJMmnPFRlsWcN4MgkGU2mpwul7rIhRBr6xth3DBPQfaPQDMPA+GcDxrJN9Z7B4tsP5fXuz2gzo+1SJbTfJKvzZ53A7bd77/15lvHh3v37z+4cWmLiWCZZS+PToui6LYay+Xi9du3AHC+WDKkJEmbrc50kSuFUbP7/lvpjZ2dwfbVKE6iMOQctdbmvsiACq0BASAIo0arc/XGbRGIz7/4vMwyFoSFNlngQkkVCN5I4sFgsFjM+73BxtpgPJ3O8jLLsyjgj/YOllm+3usMep0gjNJGK24kcZKKMIqCQARBGARpEP3RvUd/9uwsU6AZQy0LQBknX9lZ323EZSn9okskF9OxhGdD9I7rwJdtvLoLQ5UNclamJrdpElZ4dlFL912tGMXfUbsKEp/uJxdh9ZcXaRqbBdFKk1IkpVZKAyBnLBQMAJCzRhy3G400jQMhgkCYwATnLBAcEaRSUkoE4AxKRRwxYLwAaboXBgHj5rEIyBA5Z0IwAiqVUloLxgRjRFSWUgNJJakEHohIiJBzEy4lIE2UxkkUJ+W5WmbLWbbsNZvpYIDIsjzvdDqETKkiYFwwIXjAAyGECIUQXCBiGolfvb7zznr36WSeMn5/NHsskVjzWr8/oTxEdV4qmc0VRrfu3M2y5fD8/PD4NA74Zq+fxmGr2cyyRac7SBvNQATIeRhGZVEIIQbrG+ubu5vrm0GchoGIwsCg02wdYB5Hhpo0IwKKoki1ujtXbk6X2dPn+71Wc9DvP3m+LxgOh+e9brcsy267vTEYKKVe7O+dj8ZxGGQMO+3219/pLpZLBGQi6G9t9drdMAgwEFyIkIkwDE9mix88ePCn957OxtMoSYK0PQui9W7rl6/tfP/yugCtwe164BBWcWkNg1U5nMv0XMSo2+HLsW3tdBsBUB5gRGif82qPMgVw1oNHxtzTFldaUd2USJSlYghFWZalKopSSUlaAxBnLOA8CAQXIgpDHgiNEEVBHEetNGmk0Xi6mM4WjSTkis2KQnDebKST2VKqAt22KUrpLC+5wDAMm3EiBEcAKdV0nmmto1A4W8fk1YAjk+i2M2VMCM4519rmscIg3FlbW5ZFTsS5oEBESdJqtjqdPiCejofnozMUQgRBxCJu8w22poUxdrnb2u0081Ii4neW+bSUgzQBikZSPl9Ot6L+7e3rnTjI5tOjg30M0pBjK42iQORSLubL0+EoEOLq5ctJo5PEyWQ8bLQ661uXe4N1Yedq9fKCNirerOIXQjAAQtzY3P7at/4SkCpm492NQRxHjUaj1UgbaQMZV7Icj0bLPAsCZsyDRqPBGaSNZpAkzWY7abYE4xwZFyIKglme//aHn/3kycuTyYITYRyp9c0pD9rt5n//3s3X20mplNlmwCfdq4Q4QR0ZxrmhVx7ZUYcsur0U7ctlgcgyY5Vk8rGnyktf5Ul0G+RSVUxSJQQsgy7zXEpVFEUpldndXQNxxIAzxtBYB4yxgPM0EEkUCs4JSCqSSgWCM4b5oljmRRoFSRwJIUaT2TIr7OITTZokAYtjJCLSVCqV53lRSkRMQtFI4rJURb4EAEWglUa3mC4RnHE2XWStNBGCL/OCcx7HsSY9nkwyrUnwZpJmSi2yBRCAkkoVSpZSlrIouQlMuIcy2NAbYhQGdze7m610nOVlqf7J88OfnZzvRmEqdNqg87JoBfHV196+eecttZxl80mRL7gItKKz8ZiLsNVpB2EcRclWu9dsdZqtVhIFRHY3NT9y5Ha7MMoO7eNNkDGWLeZK69u3X0uS9MWTB0mcaFVMpzPGxXyZtZqN8XisVTGbjJZ5SURRFDXTWCpM291mpxuHIRdhFISc80zKF5PpD+89/MGnD2WhkzSlNC2TRrvVvNZq/N07V262wlxKR2UWMpqIedq6sBWtz0z6b5zC9Y9hJecYgUelx1dlJfhtkMDFmHzk0yWrajEpD0p7fMXmKMwazbKUiohzzpnJypvHwHDGWKfZ6HU7jKHWWkq1zPLzoWy3Gv1ui3OuNCVpI4rj8WQ6mmVpyFuNJMuK5TIDIrO7PCJxZER6Os+WeaGUCoQIuQBAxrjSpVkehVpLpYxzKBii4IJxRRBFYRRFWmcAtMwWw8kUlN7udiMRnI9Gs8WCtO51uuvrWyJMSOvFcg4MwSpaBAROYJ6hDAAILA3DJAg3W/E8K7+fF5+ejz4cTr53aTvH8P5k8uenZ3ca6dfWumtR0G302o02B5RKNjt9IijypeA8jJNGo9Fstkw4QrnHibo13NVScbtyABFAm122GQBqzRgDoss371y5+fpyMjw6Otg/OjkfnXXH55PpjDMYjsen5+Oz88mdm5caEev0NgbdfqPZYkIwwNPZ/OcvDp+eT6bz7Gg0K1mkG0L3BnGv/feu/f/o+rMlyZIkSxDjRUTupott7h7uHhG5VVZWVVNXD6iHgBnQPIBo3gEC5g34BvwM/gGfgEcQDdEsBAwGQC9V1VWdlZWRkRG+26aqdxMRZsaD3KtmkdUwD4rwMFPT5V4WXg4fPvzVby63r9oqIEiS9eaDmZZJtMXqcO2jL4sCz5s5F2P5qWtdqypbS6G14F8fDwa6eoInU376zvk5f5oUnL8DzxOJ818QnIhIFjNTs0DUhjrllLNUVajrqqnrF9f73XYbY+z7fhjHw6mPMaOmytN2u62rqmubKoTHY98Pp6E377jUirLMlhISZRFEFDNEcMQIqAgFUlYDdq5sMiRGWUIFRrXDNAHYPrZVCNuu8T48nPrb02lb1QTWD0dE2m02ZspIJprGU0ZsmdM8ReYybksMZeaPCNakHAHRgdvU9F+8vm48/4eH056pYni92VxO+RZgBP9/+3h/ezr8V9fbb7rmZ7uNxWkae4QyEs1t1zVN45gZ0NCyloKddP0qZZ/kDIgqkkUkpTgODjE4V4y18lVVN5dXN9dffX3z6cPf/Nv/+e7T+2GaP9097rb7f/WvfnE6HrZtuHlx02yvBoPvP36eRFOW//jpy8e70+NxyEiz97rZpN1l1TX/21+9/a+/voKyurTcZX0yBV3ReAB4GkxbvSKep92f5G7OXPlnCOjq7J5DRmuc/4krPLtMePZry/fXTtEZXIVnwef54x2AiSkCsFkZ4wRGw+S93282222nhnePB0fETHlSUfEes8rh2BvA5QX1g51Oxya40XHfD+OkCxRa8ClA77z33gePzI6dihASe1cXWhoiOTYBImYkyQIAjGgpZQBm6scxZgnBv7i63LTtpqpCcOTctt2GUDVVU9WNCyFUlfM+isxTX9eNI1LNBm7dbA6FUbMm6GYITLRv6v/q65f/xVdXWXXMcjvEl7X/3eMJQP9st/t3D4f/6x8+/+cvb/5Pl9dX2yrnzM5V7abbbB17Wq8yEeGzNU5FNa3cGCJWMBWdx+n2y6fD423p28XH++bVW2YHgCbK7Jqmy+S7y5ff/Hzz1z5c7ff7/X6O0zRNYvBpmv/m999/Oc1D1MeYT1GOx5O1Lex2v7y5+K+/+UoRX7Xh266SmP/UJnAdVl/xyZ/8+MlOFgqIlYJqJcHhefao6CSWeLBu9LRzzf+svD9nq+dS60986tkuz+ntT9/RU4MU/+pf/2c5JiYyBO/c5cW2bmoiRMNhTrV3SbKZtHUVHKuqc9y0TUwionUVNm1jAHePh3GY+2Ho+2GcYoEARQSJC2Llq+Cdd84BgKkiADM7z6I6z6l4NRGZVdXAOdfWFTNVzjd1aLrWISFTHcK2bes6XOwvjN2YIhEz8367vb56udtdOKJxGoi56XZVVbVt530IvmJHRMjEZwNVW+5ByYzLPTKzJDLFPMQ4xKRAd1P8/eH4bVv9q9cvGMQMkNAAq1D5wgssI6qIolo0L8qstYquECOISBynT58+/n//5t9++PTx19+8veyaerO/fP1t8JVzzsyyZCISyTnO89j3p+NxOM2nByb7YdB/8/EYqtrU/vD+8w+Pj7jZNCFQ8JuXL9/suv/ml6/f1F5SFFEp87PnvO7ssZ6F1zOQRM8IdIsRPWHvBbSk1XHqc2dXhnIXaPOnVCZ4eob15VZ5nMX6n9niUm+cS66fBv6C3jsCLLl9XfuuK+0ev9+0TdM8noZxGAwElFUkqjim4KpN2zZ1gwgx53kaJaUUo2pi1F1XNZWTnOc5ncZY3piqxikmSiXfR3a2Fl5N8KYwx5iLwiIAOmbHSOSd984F7755cSkKv3v3IaesKlGaUNUxZwDYb3fsmENQYjH17HeXN95XROwcEzsih+eW7/r5yxxp+aYhlk0CpYBlotr7XVPFlGeRbeWvKhdTfJimi6aufEkVClP+qVwoUr9cemNyFrOyJTE1SPP48e//X7d/92/VwH/zutle7l++9t4jURl7LLO1atb3p9PDl9PxkCRfv/rmwxDf3304jPnd5499TC+ur/7Ft9/sN81fv9i/6OoXTbP1bCoxRhF53tdZzcbwJzd+ocqXHy5nSHUZdysx2sDOwNCyz1rPiaeZrH3IpTyHM5z6HMuEp9rombU+Xf9ztPmJZf8zVwoATiQnySBgYESsBsd+fDz0Vxe77Xaz23aS0zBM0zzN0zTFpKpJtKlHQ1DReR4rR7vWU+sB2pRlnOLjcZxjVkAFJUTnnBGqaYxxRGyauqoCIqpaElsELYiyqWNum7oIh1TEjsv7mUqqAGYxZdPps936ENq27XP2RNL3OUvgV5tu50MFxETsXXDkaBlQwMUsVw+6gEErOL36FaOl9c8VEHEOTJvKl4iDVPaewRnVFhE7a/QjYvmkZjFGU5Wph+lR5lHjIMPjXj/u6/z6z//zb37zL0PbMTsEMtOcc3knagqI7XbXdJuLsf90OPy3v/3+//39+3q333399n9/uctgPvhf77tvL3YOLcdoBiJalpg991tncOccggFWJQZTPa9dsfWzI61UtMKvo3WebEWOFttRe25ATz96+tLnNvccpTprkDxXznkOfP4JDQ8XSVCX8qKxkFIepgkASgU/9P23377pmqv9xe5yv308HL58kfEY72NOd0cz846DZ+/Z2uqi6rZdE0J4PI2H4TaLFIUwAiBmdmwAqiYqZWJE1Zo6AELOOaU0pVQkx5tQX2833jskSillk02oTsN0HMZ+HMcs+67dbToX2NfVwzTaPF1d3wR2TG7OklL0VeXY+TX4nj/tchWe2eh6J88/X442AhoCOfTK3rkCo5R9dilLVq29R1wWlqrJOW1YdH2ZQwiZSMbD9Om3px9/9/7LXR/zv/njY/fmN/+rv/5ft123CAIAGigVGFi15ImE+DjFv789pZR7F/76z3/19dU+OPdmU99sujYEzy7OUxIlHyQmXbEtldXPrUofsPpme/bZy1b1J5+3atTDsytiyz7jc320xnAzwJ9UXWcwAM4z9ef04lkqCQBL5loSvAJ0rK+x/tbyeFsxWlM1RGeiTOSYpKzEEgFEXwUzBdE5xpRTzHmeJmJu60BE45ymOXrvura53G+JcUpJhxiSjtMEZUAewExs4cIiOzYzzJhyUtMpRgCrghvneZ7nLALMRJTATjFuEKtAiuCRN03Tj9M0z/00j0kc04urfVVV4N1ls/E+7Npt8CGKfL6/q6qmabeeHRKKqYoxMCIy0Pkq/gnA8bxNcjZeW7zpMhZgBqpo2ZjAg0s5JZGCyzt2pfnMzFYmxBEBkMi53Qv38/9y5lcP+fcc/P/mX3/9s5//AgBNFRxIzmpFLVBzFjMTk2FO/3D3aEg/TvGvr/f/y59/TQWmAphE5iwofUQKvvLBm6mrUEcxMwNDwpVe/FMYp9xwNABewaDF2tYEAJ9V6XbGJ88t+OfX6yz6dfZ/9icQ0k/h0jPXCcpg8bNffMqPn52f9VeWH2HZ8uEdt7VPYqfTwGaGVFV12zZZdZrGmCXGRGg5ZUKovGubOmZzjKI2xtQ1gYiGcYoTxJRiytOc5pimKABAqsEbm9ZV1bW1qU0xBc8IeOwnUa2r0BJlNTXrnHeAKee6Cl9dXY7T/HDozbQO4fbYm0jtvahNKVUDfH/3eJqnfde9ur4JVT2mNEyjmb568bqqayT2zpV1dZIVmJAUFjbikxN9ss4ycrAe65VSbss1ZPToTDVlISRHJiIAEHMyfXq24reZCMBE1MjdfPvr/auv52EAg/F0ZOeWBjRYTinOkwHY0moWVf1q01SM/+U3r4go5bTyiC1QYdMyqsVpBEQXAjnPdQ1xKpsqFmVqhCXXXM+gLYNrslok6MpFUoCVybSCSvj0uz8x8sUB/7QGen7az50nXSdCi8iSrvSZf+Yd1uPzU1+7Pn+5I66E/zHKPM9JBAG9c+wYAULw3vuH4ynH6JhjKuFYg/cGIIRiMIxzP4ybtmKmrJYVH09zP8aYxMBKnatmKcasmiQTUhGzGeY4zjMRBl8jUZaY1Vow59gAKu+C9+OcssygOsWoYNttVznu+8EBTTDdno51VZ/6McDd5X6bU/qcZ5QkOb1587Ou3SCgmGpKhAjgGdBIi6h5cRN/0pb8yd9XCHAx3GJUSyWUAICZk2QoU+OmKUvKOaYcswDiRddsqkBoKpkBqiqkmFRyShEAytCgmJUiScHMhBBaz7vGm9nYH6umY3YiYqZEzAgqeTktiAYWx9Fs4KqqNjs5PKhKoamVxPOc0i1TFiWs/0n+95wyt4AbS2IA//zrmRn9NBNdje9pz9ZPSqP1kPynn/L5v89/ef4qLqacsxiAqjCRD2G/7TabRszu7h9TakwtiyCAqExTfDgOjOS9qypfV5UBJDWiZrvZpKxTfDRAVSOipmlg3XNvavMcU4zBeefdlFKMkRCDrwxwmOMcUx1C19R1HXKW0zAOc3JMSKhIWa1mDoSFGHCcJu/4zeVVUzfvT4/34+Cb6mevvjLmKcXD2L+Ic/SO1ZUYXSSYy709ry1dGymFX4uwEnDWS1pu6PMEoCwsKQ4TdV1wSoimwoxmnFUt5ynGL6ee2P3m5XXwFYdwru6ziKjYIseiAkk1g2RclpOjipRIG8eTbzbsHIBJnJmdcz6ntBbLiIg5x3gY0bmq3Uz9oUDITzf7nGqDncP6E1b0nAXyVPuvlfXT7z5Z0p8Y0/l4P9n3swv4E2N99n179rpPYf1cUK1GD4tLB7eoQSJ67+u6Ct61XV1V1TzncZol5xB8FeoquHg4nYZJsjjHAJpFUspV8G1dEWJKaYoppdw1VYoxiTJRUwUgGscxQraUVCwDFpkxQHDeOe9hcfJIRGoQYxpjUpFN21ShBbBxijGnnEWdtCH4uvrx3YfrrmWik6S2aVTs/vHovPuLX/z6Zbc9jv0/ffcPX7/52f7iCpC888ysYJBVCQtCdMaqEZD/tMW8LHt8usSLk1lqj+W/ts5IFAo5oGOuAUBVVP7N3eGHMX11e/jXl1002FeVI9xXTGZieppmBCPNc85q0PJyYM7VNxJZzpojgnnndZ41ZyNi5zUnWyoMW1DXcc4p1t12Hk45zmcDWe7xWiTBuj4LdNlZ9RRe4dkne2ay+J+quP/ETS5G9vxsPy/V7Se9ejv/FgA8yTTj+QiVN4DP2vSufKtMZgbvEbE/DcfjKGpMOCC8vL7Ybbo5ZyTynsvT5Gzeg6jGlEx1inHTNuQcEW26RtQAUUQNQHImxMp7QkwxqSqTg7KHE7BspSGzxlPFfOz7snq3qiomFlm22agIM9VVcMEzQMscQuhTvOz2L/b7u/7YbS7fvv4659Q/3k45TYCB3t3df3n56u3F/jrGRJSpiEgWimpBU5jOF3ptsix1/p/6jKcjjcvC6me3BwGQEM3YqPb+b+6O//Oh90T/+Hh4xfr9FBX5v388/pfb6v/w9gbRpjkxU60xiCA7pIAKBEiAyCQigIjOaRZEyoAUvOUMZiqZvJeUVESl7K5SAMhxHnKuuo2Z5TQv1c5P4/jySc8+71nyd/aKf4KcP/eUPwEvn2OZy3qPhS5z9n/nFupT8vDc0FcsFs7Txk+3oPxweRtODcCUVFRojlHVUs7MXFUB0AFAVmWmCh0SNnXtKM4x5ZwIzZTBOyIGoKyGWUSECTdd44NDs3GK4zSnqGZlVcgCthXV3PJeU85ZNXhfVX7KmR233hPgNM2gIqJTTF1dt22zbdu2CneHY+29ZiEEVL3aX/zFr//ix88f//7v/v2ubX/17c+96qfbL/3D/as3X19fv5qnYSZyzjdVretlLUwSXdZFPiVIhShz3jyGz6KkWfETVnrramWcwyGAmBVWChL8zcPx//JPP/51G+7NDlG/3rV/cbX9NMt71e/j9MMw/aL1O4ZCMDRENCWwlShgZlbu9NJKIFLJ7JyAGJS9P8Leq6lGUS1NAUNEVZmHU2g6U805PYvuTyX2k6vDn9aK/0nk/HmOuJjPfyIHff7M9uxXllPx01ThbNn2TGvkWSkKz3xBeaQ6RhOFnDVJpDkBoqFtgru62LVNHTyHEGLO85xyygjqHBfQNKs5gizq1LZVqOsq5ixJh3E2haapnHMGs5mVTiCYLSo/qp658N3LB2UiAFC1yrlN2wKCaFl1h0l0ivPN5cXLywsxm2OapjmpxpxCcC9vXjSb7T/8/rcPDw9MBETv775c7y9ev3rddFvR/Ic//Pbm5evrq1d1qHNOMUUicqX1icDEBaImwmXh5wqYn1O05YIut2BZn6cqktMiwAQIVNaRAiK83jT/x29fvQ58Efz7ab5pm5ppV6X/c3s5ioGJrt2dxVkgoxkSnVsva263JMdIBGbIbJKt1CIGRFSq/udmJDlP/bFqOjU9Q/e4OrY/MaxzWY3PCEf/3PLKd/SntOXnR/d8ifSnRmk/bWL95JntKcdYar4/qczMEMpaV3DnF1109wi9D0Qcgt/tt4FJVO4eDsdTn1M2FdWi5oPM1DbVHGWe4/3DQbRjQkfo2naeYozJnOaUwZQIELng82qaUo6iVfCOfYwp5VzsJYvsmrYK7jTNlXNNVT2cTuM0eefrphHVEHwWNQBP1Hbt22++3u92f/junwjx1cVV17ZieuoHR3R9efPhy/v+eHzx4tU4jff3n0+Hu+BDaNqqarNkMHPOMxNTEThjWIC6Mz0NaCmI1z2OCudpXVVBRCKGBa0D1AWp/qpr/3c/r2LOIvLtrs0iamaKBNCSLroyZb+UKi2eA1cMdgUOVo9ipoisJsxcFqmBQc7RsFRsT1VR+YuKTMOpbrppONlPgzucyyBYq/pnEdz+U2X4swACf2Jnz//yp988pxPPDH1F4J8euqYNcnamuLQMbM2jABDc+vaW9xB8aJqanTv1Y9vU3LUx6zTFOCciZOcsC6oF75goxtSfBhGt6goYL3abtq6c901dDcMwxwQI3jvvuESWmJJkFVEtRSuloj8OZoTIxMMcYxZX+kDEu64jJMd0s992XWuAn77cmhoEt7m8PPVj//iYRXzw1xcXTHh7f5hVfFXffvngiH/1q9+4qvnx3R//zbvvX7/86te//ksFNDXvfZYMgN65um7qwGAoubD40QyQjAgXPbbiHmRZAl0KbVVl9lg2qpSiCklsWUdmSMSsYAWHN8kgGQqOCbIM+C5ZAcIiH702s57d0eI/EUFFDQnLqkIEKEGc6LkHPduf5jyPg6uqOI5PdnNO+NZHPuXW///tcgm16/88fy148r7rz/75V7HdgsuuvBRcCqeVFbA8/RpU7CyKvLycqyqfs2jZjVTOlSoaSJbbu8eYEhEBYt02lXdt2xDRPI2HY386DZIFCduqqqvQeN9WdajCNM0q4p2TnHdd470Dg893j9OcVMqlpuBdCL7yzsxSFlObcpbj0Xned93Fxa5tmxhzziCqlXfsuGnqw3E49H1m3G12P75//2q7/ebtm8/39xfdZrvpTv345uXLm5tX4zwhNGZ693D/3bu/ufv04e3bt7uLq7svn9i5bbcLdU3OO18hoimoWRkUJuaiRBy8N3zyNabPTz5kk1ITMJKBZRUEICRFxKJVbFr0xtY7sCwJBwAgWgzCAEyRGJjOFrkUYmaABEiw2C8qgErGspDKVERyTshsa+b3VHQDGEAxXx9CnOcla1yhh+dJ5BKF4ZmbLAH9mcnCszLo3Jz8yZIk/BPztBU7sNUfnst8xKcmp9kqg7OEnnOGQ7gUS6aA4Lz3gGh6pt8Alu2DoNM0pRQ3XXtztd9tuxBCU4WU8+3dwxzTiupp8ME5J2qH44l6RMKymogdq9g0pSmmw3EYp5glI1IdQlUHJEQEWYhAmFXnaa4tBBdP/TjOaY5RRMukxDzHUz9mkV99++0k+o/ffQcASNQPQ/But93WVY1I7WafVU3leHzo2s398XD7+cMvfvHLN1+9/fD+B0kpSmqr+vLi6ubm5eWLr3KmmCMiiuSStIhKW7dmy+7dP7l5VpryOT3P/XPORGRPHnC5E8GxY55jMY9VMw5gqYfKUB0TIi0M6oWvoaZGCFguEK76mrYM8Zb9vJJTGbnS8zv7aUSO8xTqmoj1zDwqkO0/QydW612LFvhJunnuaCxpYvGChGtg/6n80tmnmhmu45qr08ZlKRIuj1yxvifnamsf6+ltoWu7ulz1OeaSJprqtq1D8LcPh2makXC3bRihDc4AjqdhGoeS5xJzFcK2a4eYYsxNHerghnnuh2gAqtYP8xxTznkYx5SzAniCUnk6IAbKkMt+PAZMoimlw0mGYfSO67q+vLzo6qqua+e9iDrvFPHh/g7MJMs4Tv0wusrv2o4QQ1XFnIbTQXPabvauasLh4X/x1/+ZIf0//6f/cRrHum1eXl7P0/zv/+5vfv6LX445vXj5FgFOhwcFaJsu5bhpd6aljWmIBPh86LtsQxEwOCvbqMhiAbgkfLisMUFCzIpMWSEX+KZw+emcz3FRElyW/5bKHckB6vMArypLhgqmAMV9iirquvZgNZaz5yv/TvPsQtCoiya3mf6pr9OnxPeMjuvq+nBFJp8kZg2QymPO5rt0n3A1cQOAZ0UPnh21mpbjtiAUxZSfoCU9E/cKVLtEfbdtAiGa6mmIc4wiQgRq1rX1OM0jWOUYNR9PhxxHx6wpgWkd2Ll6jknEbh8OKQsRT3MkRFObYxYRAyiiWVNMKWezMpkEWS0vxFoBM0dUh2rKcY6gak1bFWYTMgfvnPPOcUppu9lcXVz88cN7NGDmvh9yzlHFJPXjeH152bXd8Xg4PD68ffMN180PP3y32+67uv3db//+ctMN3h2n+ePtF0d0eXGlCGL46dO7dDoS49ff/ll/ug9VVzTzUYt5aBlqOk9xmC5pJYADMFERlRKZVcW7gIi67ODFcqOXmT01tFX0dTW+NYiDwbLaGaDQn/mc262RDgFW9TmRnJJIXlb5LNT9n3j6c7sop0RMGvMadRcLXbHes/tciO5qRgC2KC0v7Bo4O0GDwhzFp3pr9ZAGy/rDAmuu45rL1g9bX6BYvhoiLU9sBgiGeB5GWf1qWbMEjtillMtnzCLjlBCyqJU6pq5CcCiSEbya5iRMtN20O9B+nB8lDykP46yiVRViygAQvKvqKqUEgDGbyawqRUFHwdCU1yn4kp+54EUFVStedm9Xwfvg99tt3TaO0Mz2uwvv/e3d7b5tQXTO6dgP/TgR4nbTVt61TTPM84/v37VNQ879+Mffd93m1YtXd7cf97vdfnfhvD+ejuPQf/funQb/6tXb4+HxH//hb9M8/+qXv9psvzDi5usLW+jwGgBKwnQe1zSzLLlI8SAWxExKTV6mPNTUMZuirtt3yx94qpiXe1lo+biMJwJqWT9ugIrIT9j1spUFz65pdWUl2cgiWsYTnu12sXKiYDFBU1Xisk3oyX8aPmWOCzVkhYMWuYViOSXaLvSi8jA8H4enHGEtXlbDXtBiWIujMztpSTRxbW0iAFKhv5XsG5ac1QxBTMGgsMUwpTzFPIwppaRmpdSum6YJbs7qxFqgqqpTzqhKYCnb4TQ9nsac1TEpEhFLlj6mXIfgPRIP45xSyqkUFQtrwS/zzMhEazqiKaYk2QCaurrc7oCgaWoAPB5PhHh9eZEl398PouqcyyLBe+94itEHf+yH3/34fkr54vLCVC8urv7p9/8Yc7q8vLn7/FFNAen7H38AhJuLSzP75S9+WW13t18+f/jxu6ZpLi4uI8C7d38M7FzVtE2LzE23sWZjZt45Ji7WqaYxpUIzWLM+IEIClGWUv2x4fAq7Kc2LldiiGgyICFRi8AK7qqy3r9w/QeSnnK/cZVNb3wPiUy/2yb5K/H1WERdvVbykFZBVZdGkJVp7Rc9tbX0xfIrQa2pamKZPFVUhNNia/xTLfNYNwqdGkT0p6BWPWV5DdS2zSule0tmnrHRx6mbm5pjAICbJWbxnx6hmIjpPc+UdVt4MuLCO2c1JYlJTGcY55cyIGUBFDDCmOIxCCETglqkIK8a3MM8BHbF3jpmYmZ3LWaA0UQC9c9McY8oxRwyBs+Q0A8DlxRYMPn6+jTGG4AGw8h4BArHv6ix5irHThrxDw8vrm4+fP8U4v337zd/+/d+Nx9Nm2xnCOIw+BMt5f3n1+uVX96fTH7/7XV1XQz98+fLl17/5y5TTOM23/+7/4xG+evP2xatvDv5xv79SCYSYVyGgkuQtbgnRyrYcRFEEQJHs2CEgM+esxWmBKgEoMZqpChEXB1xIXmaiOWnO5D0RQ7loy7xlua1lcGjZdyo55ZSW8FeOBDwVFksNshbnZ1DIRBbB/GJuqrAKKp01vNbzsox+PquknhCf5VydE8dzYbg693ORtuTThmWOdAkBq+9dnhPxfC7O/twQwBTPTXkwxz7klMuZ8Y6JfEoiElU1paRgnjllnWOifhBTUTVVJuyampGkH89oHKEhUk55xsjOMaIZJpEygwtgIpJyxgLBSFYzx+ydA8CSpKac5yxM0k9T63zd1I/9+HgcmjrEnKecdm0nWaY4dV3TVvVp6KcYk1ntQ1XVHz59+u4P371+8+bd+x8Dgdbhy5fb4zDGlK4uLm5uXoSmiXP89PHd5Wbb1HWeY319w4C//aff37x49bd/8+9f3Vy7UP3dP/zdX/z6L+ebPlRN1bSIxMwhVLR0vJQdn+//Ot5UdAMyswOwLJJyWu9kyUORic20JJqw4FDFnSiqAvPKGIH1flrxj5qlCFLM0xjjbGoGhkaL5xFd8wAzKN3bpRhbK2wrKxFWwGyt/JDOQfrML1JYhOXxTBxerPVcSz0VW2cY9AwB4NnMivktaTTg+owAZ3r0E3awAvzFrksaUJ4QXdt1x+OxFGwpSXmrTCiCc8zTOHnuxjmmlLabpgqOmHxw6r2oVlVV1ZXkHLOM05xSVjUkMjBRyTmrCiGWJxVVAJQ455x8Dt45dIwOF5U8R66umIgJGawJoWKXY+rnSbKodE3dAKOqzpr7GBsfvHdJNac8TdPd4/HDw+nv/ubfv7i5dsGneQ7OHY+37z5+BsQ6eCYapylnvX+4y9MUmsbMfvbNt7vdxfc/fG+it18+OUQR+b//9//dz968vf386eOHHy8vrtpu02z3l9evgg/2VJlClAwAniBpzpKYHBhkSSK5jFUgErrFJhQFiZdweXYPhEhc8FPNCZDYLbtBwNCQSxFpalaYpGnOklU0a8Yy0KxqJuUtrcjmGoqX/yx3XmHtkRWzL4iCySo3QmVH0rMVXUuye0aFDADLiponn7omjKV0On89++vK+i5+8qy2eH6Y6TM4dUEfdHXYAADgXl5fmsg4TM4bO5UsKWY1IGZE6vtJRDdtzVVIKRECCQpReWNE2LWBsD710xxz8OgcI2LhmKYsVoI7MSFhSkW3PIuKzoBQOzZRNW2qGj1TFlhTEwLo5ylnGebIiI/9OKVc12Gapl23ud7vg3Mh+K5rs5lmOfXDu8+fVPLP3ryZ+z7PM+93v/j5z/e73e3D48PD4fv37yNY03af3v+432xjjI7dj+/fnaZ5mqdfffv1x0+fLvbb4+Hw4mJfef/+4wfHdDw8MuLNzQvLOV1eV1VjAN4FE02aHbEQZ82mqiClrBbNZ0wRlq6IIdHCMTc833qThVpK7CRHS8kQFRwVfsBaPpeSSCTHeVqTJVBTNCqJqYE9MZUBDHTF/FeqK6LpQrrGRfprjcugVlQQbdnDsVq4rdH8yaDU7Mw8xuVUrMZoAAAKdtYggLM12+pX7VlDy85w/pKYPund2E8M3PXDmLJ47wAw5xycY6JpmhHNOQKAnHWaEzOfevXBVz6ktLAumqZqKp/FQuVfXu+KUZ7GeZzigkQQEaGI0uK6nxcQsQ5BTeMsaujUERKATTE2REwcswxTFMlKdNbDceyOpz6myETe+Vc319f7C+f83f3dzX7XEL37/Onjx89dXX9NvO82X79544Nnoq+Cv/7q9b/727853h/2m+7icq+q+/32TVV9/DiMw+C9+/Nf/uJ3f/je1Mbh+Ob1W+/dH77/Pkve7XaHu89//MM/fvXV21DVdbclYnauabeDSJJYh2ZJrtZUzJ6xKwzQ+VBcqaiZChgsfA4AAEUm0lUxgKQUX7AgB6o5l7nNlauiALrkkiq2SM2fsaaFS28ABEuHfXGLRYV2ibrP5RdtdbdwLnpsIcoAgKEaPBU9qGC0ppJrNQ7FjBfuz3OEoER5BSRAfXLpq9z4OUeys5ozrP6/fAw3p5QlL5mAihKGyiNBirnsPDJQIvRZ6qbyvmLnppjnrJ4UoSrLgOq6Ct6PU/z45Z6Jm7qOqdRGwMw5C1iZ6CU1LZ5D1KYYg/dokHJCRF9z27ZVFdqqhiW82ThKzoominjqh1I4b7t2s+kIoAr+5YuXMaVtW2fVx+NxmqY0zxcXF/v9TsGY+XK/u7m6urq8fv/p83g4jeOYyvQSIjPfPd4fjkdT/dW33xCiiDw8Hl6/vLm82H+5vWWmHz/effN2bJoBTaahlxQPD18Acbe9lJzqpkOirIKIiIxUGCRWEr6zUKgVEQQELHPlCFDUhnSBFYtmTvFJarks1pWcJGfNKeU0p0UxGBFVwExtAbkWrpGB4TKVtqCturi/kmguosVrP9UWSsC5vwolSVzWKZg+ddLOmWxxo2gA9KxFCSuqD2e/f3aF51T1qWlUMktcIfw1xqyyPKtnXio+QNc2NQMM3vXjpKpgSgjOOREtA8kIBGbB0aapxznOMYJBTjmBNXVu6ooQx5jnmEUtpkxmITCgiZbN2yriRAUkkyoYnrnXMSU1YyQQUJGYU1a5dpel7eoc++D6EdQ0CUA0MCOm4EIUeXg4pG13HMemP+23G2badJv9fv/u/TsQ9d41VfDMMUUD3HQbQ3q4v/3zX/383afPn7/cEyI693A4uo+fReTmcj/N0+Px0NYhuMu6rgqmO07zpm23bX08Pswxtu0WEKrgRfI8nZCpblrvAwCknAByQXKLTFcp1WHp1WBeEH7IOcOKqpSVZ7jeYBWFs+tVlRjnNOcUc86Sc85pwXdK+DRd88mSZS4pxNpKOtfpdvZYxZ2XDNcWMPQpzyy1O61Li5dAfi5h1r75Cvg/83bnBy4fakl8n2eoZ7D2J+XQ+hJrqHkq7lf/C65t6qaqnHdlGKJoVDChGaQskkU0xRQBAMmFyoNqzFlzruoqqw1z9s4R4ek0MHNThxRT2SxjANbYFLzcH2QWWMDAp/kVMyNE9AgAIoKqWVQVLrab7abz7Jhy5TgwE7NjIoBd1znnHk9HMUCDz59vD8fDVzcv6rqAtm6/2e67TkRPp36co3fsvdteXH1490PbNiGEKc4Xm+7hcDwceyQCkctNV4fq/vEQvM+iZvrh05eu3f7+D98/PD4iwHd//GHT1k3TgGmOU1XtiB0Ano6HZnvhfAVYHKLmZM75pde5msqSc6qmnBaitCgYIDGVsXNVMy3cZFAoSnwqZTvIlGJUEQNVlbLg+sl4ylc5AWujG88LiwtOuVZPz1ua6xDdancGRsumw2clzLNhkTPiDmiwHBJYnxEBlzO5ErWW/AFxwUefg/p4zjaXRPWMiRWaCK4VfxnIdwgYKj/GOQRP2MwTD+M8z7OqFsQPzAiJmFJKpVt1GmdTBaS6aVmAHAFQzNliQiLvHSMGRwaYxLIXcmyTCZxR3+X9qoElZRVmt2areZhmDJ68I+eAqKqqrqouLnZJpAm+Cv7Uj4YbSXI8nk6E33792rHLOX/6/JmdQ4DPd/fjODKzAjim3WbXHw/7rmubGsF2m/bvf/u7vp9ELTgHYA+Hx3mevPfH03BztVcVIgJNc5xVJSV9eDyo6X5/pap395+nmK8uL8dpHMbeOc7TWLcbKrJTgCKZ2BXRKGIHBqpZl9TTJGV2zN5pXsY1i12W6geITKRMHeWcUoqSs4pITstocOGWPsGEz4J8ufOEQOuylzNgeW7hrKZ3JhQtRlh8KoIZnFuQ5ycv40IE+ATWL65vVRhbSiQ711SI+IT72/qEBbh66tsvD17fJpgV7VIDWPcuA7i7h8cXl7vah8izCSPzdtMh4jQnYfHgiLmqAiL1w9SDOWYfvCrPcybTTVsz0ZBSXdcxikj2zJuuEZFhnKY5TXNyzjFRybtyVitI4HI+IYuAgXMumxkii8qcBjftd7vdtkveXe33b1++OI6Dc3y13zl2D4fT6dRHScfTMCb5cHurWU7zlHPe1LUB5pya4Ku6mlM69KcUZxWbU2yrepymLNpt2kbyxXYzTdP9wwEM1OzXv/r5m5fXovrjhw/9ODimnOTL3UPK8g3xZnfx+eH++x/et93xuh+OD/fj0P8qi4lImhGZiFyoyHtyys6LZsrJl12jOSOAqgCgqjIW78kGqqLE3gBVcrFByTmLzNMoOYmKgiosY/gLYgMLfWNJDkzXOgNFtRTrstAAFoB2NaMFzCle8Mx2IwDQ83DSmm/YatxmaKBriVtwAFh75etbOhf7TzATLm7IbBlBO9t9yS8AziO25av0QRZ/utRPrj/1cZqK0qepwhzZcagqdkNNVU650CXnecpZidk7V6ux9yr57uHYT9FUg3dVFarAwyRZVdSISAEAiZkZ0TmHiIVejkWNSu3MNCvnuPTysmQ1lSyaUl2F0DZVFbKKY3bEaMjE2+2mbRvN8vv5h9M4xml+PB2naUazT6qO+XK/e3l9dbXfV1X17tOnH959GMZxGqcpxs1mc3Ox7do6xnT38LjtWlV7PPX/8i/+7Nc//yZJ/nJ39/s/vMtqwXERsDud+v/4u9+/fvPNTHA3zH/8dPtmnOd++N0//f7h0P+Lf/GXu92pcd774EPl62azu1yKJMkSah+qMqFR3F4pPBGBijwTqJosBZNIQftTimqaJUtKpcu5AqIFk8en0LuEzadypfjFUjTj+ohFDRPXqLo6UzrDkKuV6FK2r89cCCCLn1vLdizmpE91efG0uKpEPOWWa1qwUpFLurxWTctvnmcVzxZ+fofucOwRkZ3bbjehrjeAMaWYkndeJINBzjmmVAZ3QEUEY85d8ORDyhJPvWNOIkDUNaEKDgBENKoycxVIRZBwKTOXiqFcbQAQQkQiJSRgIlRRRqir0FbVNM0GVoUw0BgcAeAUo6l556qqIsLD6ThM09QP4zAeTv0cIwIQU11Vl7utqX748tkUvOP9pr29uz+eeud407XHcVK1u8cjItZVYObf/Opn37x9lVX7/jSOU9s20zSpalWHn3/z1ePhlFL+7rt/+vWv/+z1zXXvXR5Or66vQWZCPTw+quTBeQS4vLpxOTG7qttKnGGdDXI+lFOoJmBYpmZIlZgRwETLP5JzyjHnJJI155K72iKV94SFLzQ5WPpOVhL6c1xagRxcCpW1Vl6yghUtX/PjJ/LA8s3FQa6Gu/yIFnMtAfgpjUQAKmqMT4CnmgEtT7u8UTsH8lIuPWvCrVIrT19rgQ8A4C4udmYmIsd+qELYdM2ph6oK3rmc8zQnG0a30BwppVy0FOdpZudCVQXvCE2TpJTDflNVVYzJzBoXqiocjv04jEzkvXPOaZaY5QlaKDnVgtkSoTMEQyTmdrfpp/n21Hsam354PBx98HVdBWIi6odhjvHj7e2HT1/iNCPAZtN12K2QDc4x/fHDx92m27TtOM9meLHfFRIwGXji+2M/TXPbNsMw5ZxSSh8+fd51fcy5CuHNq+vjqf98+2gizru//LOfuVCJ5q3XVz//um1+8/B4C0D7XdvUVdd1VdUBkqo2XRdCVS6tr2qRhEhZMwiBmWp+DkOrgmVcrAELyJdzjDFFyVlywem0cMll1YkodcxSzdjzruMTxoxr4vjMFa1hdGXCwTlJffKAK3wL51D/pDuiazuyhLtzeSSqRnp2eM/B1vIiq09VtIXpjE/nxOA55o9wttq1HWauqsIcU1bLcwQDFQak66urOSbJqZqjqmAi7xiJYZxySmXpAswREL1z45xEJGdhpqqqADDnxIRzyqdhGueUk4TgHbsZ4notjBANsaDRSzWmQkQG1o/jdd7dXOxzzsfHw+HY18G/uNzXIUwx5SzjPKmB937bNBLCzdXFVy9fxpSmeQ5VuH849McTAoro4XgChMv97mLXfazC7eHYz1OcYxX8zdVFP8XPd/c5JQEL3n/4+GXTdduuIYK2DteXm4vt5jRM96fx5rq52F82bRecC1Xz6tXbaRpD1RFR1zXeVz7UWYS8D3UDCCKJfe1DU6aZJMUzplNOJROtJfJiZCnFeZ5yTinOWfLC7Suw8XqMAQHLHCkUKchFXn7BiZ5F1hX3gZWuX6x3VWWCBQJavO0y6AwGIEs9DmcjO2eZZ5s7HzMzW6ux5bsrkADnR5UibMlxVdfE5KliKz+n5+gVLpwqBXBfvtwX6K4KgRDLCJvkNKecY0pZnQ/OOWYGM8mLuLo6ZmIQ7U89EGYRJsxZ5zQwYfC+9IHGOYpoFoljTHOUp+0CS9auxXmUTTMZkAwAR+2/Tz++fHndOj8hSZ7NLGdFxNMwHIdBTV9eXKRpnqa5qcOc8vtPnzdt5Z0jtcttGxinOToCIqqrSrMchzGrff3VKwNQyXUIj4c+xlxX4WGacpYvtw93D4df/2JTRMlF01cvLt+++urz7e39f/zu8XBwRDlLyrcvbubLy6uUzYUgpmIIgM75qm7jPKZ5qrqtgcV5cL5iZjPIKSKUFbumqpqTEbNzCGgGOc4xznGeYpxFpFTutvjNglCVuLoMnVpxoIqIQLROL2HZ/XkGbQyfxM/PsXaxicV+Vj+p68rG59+FMxp1xrJWu7GnX129/9k/r8H8jGqd04EltVv94zn9XTz36pLPlo2lSLp/OLRNBUhZlByLmpo6orZpBrOqot2uVbWc0jRH0QBgCuYBJEvOqeQ+KiohnPpxTtkxbdraOY4xoYGYzTHGORaWl5oRoq3Y7FNubgaqiECqHjGLHA595R2IJJE5JT+ObgieMHjHzJu2Gab58mL/9tWN815UxxjfffycYu7aZk4JweZ5ZuK80cv9znl3Xe03m1ZE+3749OXu4dg7x8H7m+vLeYrvjp/3280U09bo4Xh88+r6+upaALJq3VSmmtIc4yhG//j77/D7P75586aqq+DcjDEAzIC82YWmi8OJ5yk0XZY5zgORK+PzKmIay9U3tSxJRdh5IATElOI4DnGeYs7FenC5m1qwQTUjpNV/FUcMJfIyGhHpMhp65rydgaTVONdBElsodcsD1yQBV+7egkQ+C/5P7lCfynA4t9SfQU0FSyhHyM5euCQNYmAL4HD+3urjn3lcRCwCz+XzuJzzMFNTV4QUY1K1ENz+4qJtG2Ym1N1mk7I8HI7kfFXXh8MRwBLxbHNK2cxMFh6dqG27pm6qlPIwziK5TNHTAh0rITgmM1DVYqnlzZZ1RmoGZRmI6q7pri8vMthpmmIWFT0+HEikaZqL7Wbb1Z65q6td11R1NU2zmKWY2rpJHO8eHvt+KNhKVVf7lKtQpZSHND8cjllknOZQ+b/+q18Tu3fv3t3dH+aUuqaOKZ/6sR+GaU43lxfHfvhyf59S8j7c3h8u99s5SYrx4XD6/t2H+8fji5c3F5vN9eVlcEFyStPgQx1cmKcBCR25lEQtRxNids6X1vpCvlvmmRSZUoyScwkvjssoMzAZAIGKSjYreBMCMpArK1TtfEtLHFomTc7R3tbYuth7yQfWNvqS9i1i5uXYlEJ29R3FVlbyGwKYLK5x0T5eQACzsoDG1mhNhnbuRuFakRlgoRCUiqPwVdYhu5KcnNNoOD8MwJlZnEYH5p0rWgSqELxXEQCLMQ/j1DT1brOZY5rnOE2zmbJXAxQdC26SJZ81VVSUEERV1VKSmFIxU1szKaRS1Zdc6Xy9lgyKCrFScn/q1UxyNsmaZeyzY2q69hTj4dOn2/uqCt4xA+ChH+7uHx4eDzkmBGDvysXt2padG6f5+z/+yLxsjClMqz//5c9evbh+9+HjtutE9OXNRds0x37YbjY5p4sdiOT3Hz7fH45NXTFxCH6Omcl9fHy4eziM0/xPv/99jLHfbhxgW7fBeVVNcSq8u3kYoKpNNUsyU5mn4AM7j4CGRkhnEDFN4zT2c5wJlIJDMC4SlQAm2cSySrQsksyQ2NkSuBdsbknxCtKJiMXLrpno+bqW0snW4I2rB3sGkp9L7OdpACwgFyCu436rokLJ0Jb/eSZsuy5OXvnNuAjrruG7hHRd84zFwJfS6uyKz87LYUGFUqpyruttacE/HE+OKcZ0PJ2mOXYxHvspptzWoW3rUHlHNIzzPcI0zRqFkGLK0zhpzv2i407OOchyrtPVQAxMDVTWDKN8FDhf2uINRDVOcYDBED1yW9UnHUz0dOoR4fLiotk0fYxzSpumxhHuHx4fjsftpiuicG3bjvNUlj/d3d3NKRFi0zRmqmKIuOmaH9+9/3J/3zVN1zZNU7+8vnCOQ1Vd7PZm+k/ffTdHff/5lgjfvLz2jkNgJP/4eArOgxoZSMy3n28t57ap9/ttXQWwMig0W8k0cyRya3RGydHMnHOEfHZRojnGKaVZJRWGPhESIROqKJhoTpKipBlVaQGmzFZl01L02KIYa7zQ5sH0+TDqkvHbOadcSylEOI/4LsTNNXdcy6IVxcSlsb7a8FLvrdXQalII5yUjT8nkGX1aC6sSNLW8h7WHUI4BPU9YAQDRVVUoW89MlUodENPj4fjier9p64fjaZrjbtshoedl5crFdlN5R3g8nbyKmBo7z46z5Kyy6dqmrQlJ1UQtx5whQ1EgkDPndb1F5Qroyj1bpNUWzAERxKSufExuzDPEeDyail7InoJ37ACwH+fb+/vdZvOLb79x3h37/vHhMAzTaRglppwTMHn2cV42iLJ3SPTjx89fv35FiA+P8a//6s8v99txnrfdtm0aM3375qthmP/ww485yeHYbzdtXfm23d09HKYYv3r1IqbkGLfbdretm9qlFEXEOVs6OqXmMwU2QCgk7JJpmZloJnJLLqdqKqYCmkspC8CgIKo5JU3TPPXTNKlmACAkYkNEIzV0WhpAuojIF20INSurpwDPFQwUJfZSbi9bFQCWSgB04evgYnDnGuicGCxhzbQoQeMZJTzDorbknraIvwMtNXjJMBcm6Yqwltuu5ZlLfxSXE2Nr6xPOR8AhQJG9iCk9PByy6ByjY3e931Dt6+CHcZpj3jTN3f3jaRhNtR/GTdfKwrsBZhLRnLKIFNLUpmvbtp3mKKLa1OWNapaMshSaz5gLiFhS0sL/MUSYiySONW0NhQyKVD5MznI89dM0b7q2auq7xwfLUp7l3fuPx74fpinO8zRFyQkVyBECDFNvIuxcCKG28Jjz61cvguMvdw83lxfbrjv2PRiEEMTADC4url9c0bsPH/7w/buPn28/fbl7/dVN225fXl89+Idv3ry5utyd+tN+t7u62O/3F03TMTM8u7trI7d4NTAs1Fheot7iXiTlmNI8T4PmCCCBUAkFUFVTitM8zTGKKpqAWRlkgqzeBSVRK7RxBMDiYpTIMS1usHBSy0/hp2o2WCLVApESLCZki7jf01dxgGfwH5ah5CUZhRV4R4RC7afVny5eeWXzrYo9oAhPNCszxDO7dC29znDX0ogwV1c+JhymGLOcTj0RBe+cd5/vDs6dALAKYbE9xMq7LGoG/TAhQlNXkiXGJBJTSipCRNM0/uEPP4QqOO+rKtQhVI4fAFJKwXyClMSYyQdvBiK5iLMvufQyLqiSNKt657pNpyLOcVkYXCj6GezUD8McvePgXRXCMExfhrv+1BfB0VIpm5klAwAi8t674Nn7Kcab66vLy4vH43HbNt5zP/QAVldVERA9nfrNZkfO/ebPfqkqp+Px05f7FGOM88vr/X7bbLfbFzfXx9Oprpu6akJVBx8KVAfPv5aKBRdqE5GBETECIaKq5JzSNE7DaRgOKBlNlAwMRHLKaY5pSsuwHq2VSMlCk0cgBWQDVDVEcs45ZkQUNUYzKFL1CGcwfJlORrVi17hCAmcbKpXKgrgv9m2ABFqmSZ+sfCnRC7BQyCZlBmABws7wPuAa4RGL7hWU8aiSDyMY8FJAL2+3GPg5IzAAZ0jEzjlNKY0pEbNqBYBjnszMOb/fdo758XAcx8kH75xjxL4fYozOORd8VnXqYozFf8SYRWWc5q5r99sNIk0pOuamaUacsxqD1HW13+1EpO/7mBIIruSpBYBDNTM5HY5FWGae5mUAHEALImWJRXLE6H2Y0xzjNE3lUBZ421YADgm9dyEE7zlLQqS2ae4eHtGgl3Ga59cvrq+vLh+Px4uL6+1mezjczfOYstvvL3/182/jNLbN+8dTXyCI3c2NITd123Z7ACPi0ujLOS83dVn+fTbLsscBl5mPp7TNUpofD7e3t58kjpAjqjgw1JzFprJ52kAAkZgJiR2wA3Jski0hKnPhvSPzqrhZcgRaQCiiMv1WOA+LEdDqHBcLWRuU5dqKLFjAcsitNIBWsGhFjojwJ2b0VKcvhlf+FITnjCzo2u4ssZtwmUQ/Aw8r/Ln2HswAwJUqpq4bQxz7QWOa5rnsCTbCbdfmXMUYqyqI6BzjNEUmSCnnnLNoXdelSxRzjlmKClfpMLHjKaby5bgMLmdiUtOU8zhOTVtvNpspzillSWmeY2k5AJGYMVoWmGNk50SfWhBqCgKFCI2ISXJCLnvEbf0qRo4AzAxExFRXfo5pnueqqt5/+Nh1m66p1ezt6xfMZAbOeSJMKW67/eF0PPWfr69e7PeXqe1Cu/ub//C3265pmrqqKnIVs/fOWyljVwlZK3xXeMJ3ibjIJgMRLNW3AnAJ+ExODWOKp4fHNE8oCSRbFgETRSAUIGXnmBw7YGVnzgGgslNmR2zeO2ZHVNZSo5qgQtn6DQaiBgDFDkqeVGyjEJ0WV1lo/kvpsoA+a7fp3M2Bc0JWCBVl+zzC8wnntY+12nGxxMX4DIpYry47P+BcTj0dl3VZyvmZiz92m+1mGMbgXV373abtx2nsRzNj79i5rmsN8dOX+xBcCKHt2hxjylKchKj1w1BOmveubZuUUk4JkIo+o5l1bTPOpKJoggjBOUJUUSYMzicUyskxo0HOIks9alBwXVRMqUiAr5XHCqcgEhMTmYGUr6KRuyLNpcvimJ3juqpSSnNKRpSLDDT7fhic4/1u808/fLi+GF6/vL5/uD/54367HafheDzst1vv64fD8XJ/9c3bt1Vdh1BhMS0kMyNiRDaRdTYI1y8oeQUgItHTfYClE5NklhTH8SQiTdXmNinQPPQx55yWvrshKRqiOmbvzcgoWxUMiUmNSJ0zREMEpmWrYVnnp1IolUtqsTCTDEUXUJwJrbQ0AQqdTBdjWqr2pUBZYKIzBL/8t0D/+GRb68+xwE1rG+n8W2t/vpwJwnPjdkVs16przd/h7IZNzeWcg3dNXTvPAKTwOI0TiAXvEanw5It8vZltKs9NdewHxOrUj9M05Tkuu7+c224753x/PB1Px5QTR8eurFxlJhYil5JlSWZECITTVFSbivSIOeaVPLMc3DKbz6p1FRAwznNSQUDHjr3zzi3D+yKmiiXpXvsiZZ0XM4vY8dQjknO8CF0RH09HU+u23cfPXx6Pp6EfwaypQxZ5+/p1XTUA+PhwD8j9NKvq26/eYMkg2RGxAZZhYCyrjvGcQS0V3QKcreopiIiFSwyQ8jScDsfj4zCcSvkfQiVSpG/IKKIsGm4lAOecUxbDDMijS84xO0fEwYeUXF1VWgVRVRe8d6svt3NvxgAJQeFp9ZGuE54GZdoTABbyUSlnVxTgGTj67IytBrloTZbwX6buyaiwLM4oTXmiUi+qAZ/j+YKDL667nOpl09cZDTVTMDcM4+V+d3mxH6bp0I8AEOp6HkYRIbY8ZzFlwipUwPzw8Ng2NSzPguycD5aHcZqmk0o1VdeXl+2my6ZxmmKKNFMIgYl16d4zB984NjNQjbnMziuY5YV3boBIRCKLoiQgECIQIyAHj+rK+u4SrpjZQJkpxlTaWk/HVoXJkXOBnUimBc5TNJCUsgp7j4D39w+FVgIAbVNnydOU/uLPfl5X9T/8/vsXVxddt5Wc5mm4uHyhgGZISLZQLSyLQEkkzplTwcdwEbl7MtBS7ZqJ5HEaHw8P0zRIWnOglGLMomjkAIyxiJRp0dMp4AgSZNGYiDiFEJiJjRVMDQiZCEWtnINyBdbeh5mZ5nOfU3Hl4S4AT0HG1ppmgTmXuaU1Pq9/oXNnaHGm628uHxlMQdath7ikp0tqscKquHbrF+9Lq/Nex51/0iZ1bV23TV2Gg2vvQURCdoRV8N4HVRnH6XTqcxqqugohlAaQijBzW9fZ+5Qz5wyIaY6fPn6qqsoHj8SQc8556HvnPRGBllm2SsxATSTHlJJkWxp5Aqti2RIW194SMzsiAUMjzwucR0TddrPpmqEfHh+PAKl8BFyPLQAyc9e1Taicd4QQY4opDf2QUyZEx06ylCGklNIwTqqKCN/98V0InhC+3B1ykr/6ixdd10hOMcW2u4BlldGCv5TUU02Zym5EMCvq+7S60dU4kYrcvHceF5AX1ExMk+gU8xSzKx0OAgUC5lLiOEQHAMSGTMSAWOAR74N3ntGZgUpWAiQnsDQwF3LQk5kiFEp4wSuhvJt1+LiEeIIiTrhy6Qvw9CdPhavBLvG4YOjwfBhkSQ7OOaYVrAuBVO1MeVk9CRhaliUxXds1CwiFAO50Og3jVFWhpGuA4IMn5pxzSqMPjp1j5yRLjFFVYyQEqOtAaCKSVcvAmkiWLPM8j+M0znPZRpTmmHN2zjFSUSZiJkRi76qqUgBOCcg0ZStjuGC2rLJ6ymxKsKtCUK8i6gCRuW6qtmmc91VTXyD1Y5jXKr74g65rry8v2bGqbruOGdVgHGfv3IFOOUuZKwcDEzsdTz74/W47x/jp9g4Ad9tOlU7DPE39zc01AJqh5BxCXZCgM+RpsHTY0bC4f1ySv2XestxGJFpq0zINl6OUoWRywQOYOWZYKjwzMzBrVKSUzwAA66aaEh/VRCQCZBEimh3XIYQQmBlpWaRbxgFWi1m9KqyHo8hKWonVpcO8QHLPQvozJVsAU80ATFTCMaxaOgvsClC6LeXt4dIseHqGwlx7KreeJHDO2Wtpi0KR/St+1h1PAxOlVNd1oIR1XbdN/Xg4ZREVTSKIGOqKgKZpHMdpnmd23OWmaZrC09l2Tds2p364u39ERPbujOIyMTKpyBCnsv7HETtmcux2O+9YqjpzFoNxGJd+nFkWwTUyAqD3vttunPcxxpyFwAxMVU/9oCervKur0HQtgKFaKZUMoGqqTdc1oRrGQVRR6auXNynLp+C6bXc69sMwlfbPlNLh8egrH95+dRrldOqnYazrutl0VxcX7z9+2e8urm++YhcQIOdYIratze51P5CtIX5B6UusffKhBoAgItM8HE/HYZhMs2d2zokCsqKCSjZAIvKOiYhpDdCqWXROOueckqSkZaifHXt23vlQOQAEJKfmPAIgE6rqUtEXGUe0M8t9yYJWMoQuNCV8yl9XJmTxEPhk5nYmSZfIvSp2L2nl2fqWMH6ePF4h0jUleMKVCjRTsl4pqmx6dtfgYoyFPDuNIzG/uHZNU1dVDYDGOsfEjJcX+7ZppmmepvH+/nEch5zySQfnuKlrBX/sx9MwZlVyblNVOeXCCmXvETHlPI5jnOeUEiLmlDWlTdddXF5cEt3dP07TjFjU/J8CQ/nQzOyrOlRB1nwsiw7zlFJm4q5ti9txhJuuDcxmFkplxuyD2293zNdgNs1z1zRf7u69921dN3UlYo+Ph3mamCnGrFkOh1OOiRD7YUw5i9nrVy+ato0pppR8aFyockoqWVWLCvOStC2hcyVglJoDCYmw/FmRxSxpmqZpGud50pwyEzMjgGdCY2VCAMfkHTMioRGqqmQpUgk2RxmneZpTUkMi77hyvmsbH7rF+NY3Y7bARes0CBVDNANaS8lzwrrkhmsyivTUOoJnDm+trZfwVip9Lfsd1rGTUhKVfSlECGtdf75Q53xjQbbsHNwXX6uLR14AcWeqQCg5IyEjz3HGExX1wGma1KzyjWQZ57mq3HZ7fXmxu729/3x7P/QDmE1hqurKee+8++rFjfcOAI7H0+HYRxEHgER1CLvNJub0+fZuGkdVaZrGAMZ+IMdF/kpXbwRqsAKZzByqwAj9sTfQlCTFGQAIIDCHqvKeYVHzQDLwTN57JCoMnKaqEGyepqap37x6iQgpp5xzjOnFzY3m7BnnubnYb6c5fv5y//n2vqqqEKpxnlXUIXaBX7140XU7YlfKSheCZsqQACBLWsH5pewsi2myZAfOsXtyKsVPIBCiqmURFUmiKWVCgDIpgcBEROgIVvKxJs1lNE2WYZCUYoxziiKGaHW9aZqubdqm9j5474nZMRNS8VNJFBEcczG9BWrEMsX5NDEHS3WyVtxPE55WcgpbqylY4UnEFapcLLeUtlIaEfQshVgvzrn0WZ5HVns/5xKm5b6X5sGyZcVpFkV0zvu6IqTDcbh/OJYyuSBH3jkwSbMeHw/eu8uL3cuXN8j88cPnaZrmGHNKoQreezSrm1pEnONQBTUlYjAz1aqubjZXSPTly21/OhV4KIswYt00oQrjOK54w/IxHFHwftO2zrlhGKdhLBVVoTUG7x1RkUuAnCvvqxBCCNM0IlLXdV3XMNE0z/Mc1bRrm65tX1xfVp6dr6qq+nJ35xjJuYvd7uHxUNfh9vYRifb7bZmrRDQi9M6FqsaV/6U5mRkzgREbq503Fa29uaVaV5HM4IAYgGDBtJGQQwhV8EsdgyQqKcWUpTgSzxgdES40ZQQtLKEi4EpgnjARAFBVN9eX+6vLy65r6yoQO8eMS0wH0Gc7txft+iVNVls2/gkAAwAUKjSoWqmTnhzbiggsQX/NX5dLYWZarH11jUimKmvgXrzgs448IJTXWoGwdRh55Vkv4JSBrWssXCpnkWIScd4557qma5tWVUSyrZV1TGmcZxU5nfrdbnt9dQFmX77cl0gXU5pjmubYzrEKgZkckyn64FW0H8cxxg+fPhFQ1zRFZX2aJnbOORdUDWzZBFj6WgAIRs6Ftokip2FIMWph7gEQEhAZQFKriGrvfPBqdup7s4UC1o+jqrZNNc+RiB4PJwO4vkhVVbdtJ6qq0g+9qe27jhGPp1PlfV1X7Fzwvu8Hx1RVNTuva1Ve9GIRUFUk5wWct+X2LJo0ZT0Nu2X5UWksFXkDREQUyVkkppyyppyhCCESlQEFM9UEEyzlFyM4QiYEYkBgIiarA3vfuarebrcXu31T184xlm7S2iQoyh9rzAcRMUJCgpVTVMBzpsXmVBUBDQt+hmuGWlJDKDXeikYt7nRtiJRyTp87VFxnoJ8wPztDK2CmgMtIdHlWXeP+uV+1Ml4BAEprfUlmEdAxe+9TSkmSiuYspfh3TMGxmn78fHc69TfXV6pWVpoSczkxkvPpeBqY6rpGIiaqEZHRAIa+Tyk5dtc3N7s9jcMIZkRUhVCc5ZKp2DLwh0xMbGJjnHLZnarGRIwEjN57JvSePfESIAlNCxk5jcNoiNeXFzFGJAzB13Wlau8/fqmCB4C2qXbbXV3V9/ePP7z/xzcvr8tNuNhtx2l++9WrLHkcxhB8FSrnvAEsx7WEJDWRnKVMXUqpLXipaNAWZSxaoPD1sq9RTCTnQn5bq3oDQueY0HK24hRK+1TAMqJzzKxIXHzUZts1TVe3XdNsmNgARK103WHFtmzx6LYAQFjE5stWkKWTtLxyGRp+FnlXZZ3nRmaotLI/zUpEWEqcxSjPqNDaKQV4srznF2F9agPAkr+u8PCaBz+lHWYG4JquNTMmBgQkjCmZQRUCGJhomTrJEhM+QQ6H0zBMMyM758ocZhWCimYRx7zdds7xMEXnnJg4dteXF3NM4zSaQRUYknG3IQTHTEybrgXV27uHRcC/pDvEQCR54d2pmiOq6trMFDQE31Y1EBWjAUQ2E8RDjNM8qSgifkiprarNtvPeV1U1DFPOWQ3meXbOp5y7pn5xsZWcfv+HH0vNXIUQk1xc7HebTYqxbaqqqgpSAQCqYklK/00k55xNZalGS/FROptES9JmRcjWFJQIkDCnPMd5mKYpJjNzXGyFAMAxxag5WUqiIqBSArT3DsuoFmIIoa6but003ab0uuaUSyuOEN2yFIWR2GwVU16rt5I5rdUPIQCcx0oLE6kc86LcXswUccF6V31aW0eS1gpqrcbObSsoYuWGAMtQgC2WVox1IR8uQrmrehmeSSa2NEp1ZUUBOO89GjjnRBUQnOOyAfviYi8q93f30zQXvFBVJWfvXQh+nqQAwq6cfTAlc94XHlOZD4wpxZybGqsQ1Gn2vq1rAGDm64vtHNPd4+M0joExeD53BQ2AiOq2qdqNSpJBHTtw3rFDQgKoHTNREiEzds57n2IcYoICkZgCuyx5enione+H7fHUI5GpXWy3vGmrur7Y7w6nU07p4eGh6DOOczycTkm0qcLxeCTiU9+XzNI7t6ZlRXRDRUUkiyQwO7c0ccXt1wTLcBm2XMBtUxXN8zyfTofj6TBNk2fk4IJjIlQmBJ3mFMVkTmaKCHUVgiNfusXOhaqu267bdCE0YDjOsZ/GApYRQ+V929SVeS5COrbgU2fnhYigi8hCSaiLuxUtK/LQAAqlgdcGREHH1kFkNLBz1X8GgYvpPqt2llC8+kpb6qTzFuc1A3jiJp6FzVY28xktRURXVVXOuWkrIhrG2cxc8Ju23W46M/XO9f1wOBxTSrTsldK+H4vYFTOHEABBshhYVdXsuOxUMFNGaOo6eBdTTDHWwVd11TYVAHZN049TTDFOU8r58nKPf/zRkhaYm733oUIiSUrO1XUNZR5a1FSkrNIgMoQ4p2kcUxYog9VgubSuRQzAN/T4eHg8nqrgzSzn/On+7qub67sqTCldbDdTnNnxy1c3CDCneH9/qCp/fbkfpth1m03XFEJCqc2hcJeWFa6GAEVMmcjR+U6t0anYZXERJWcpGfYwDqfTqe+HYRiDI4SKqWJiJqq9l7Y2kQkkpVwElACM0AJD8BQCegZSmcdhmPNpGMseFeewCgwMmjkvK0aX3cl/stoLDASFkBQAlzZBUc43QaRnB2ydVILzx1JVe95jx2Wnk1mpulYlvYXdaQBl6mqZiDz3mFafuuSwazBfsodSjZ1fBQzcfr83zd65lAXnhACXu+1u2wGBGXnvmqYuxA4wkJTGcSqfIaakoqYaqsCOmcgxVcGbOTFIMZqoGey23a5rDsNoprvtlomHYTicTjnn3abbb1tVu9xvf/bN19//4YcSetA5ruumaUNd5ZQgSxZhIMeWhQpqLSlLzlb2GqkhAJbtOUspBT6EJMoEdVWpKjGdTifn3f3D4fb+4c2rF3cim7bedt0U0+Vuy4QpppRyU1ebTVdVvGmb7bYtTQFmlsJaAss5F+9C5MyMaFkqe3Yr63V/lk2BmRbZkDnnWPRw1HSOyfFCzchqCOYYPXPZzEmgKiqE6ha37fM85jRFvevjnLIVAqggKpiKqqhktXPBsyy3KD6Scdl/XogjxfEr0ULFQxJEJHRET+gYPstDYdEQXz0gPCeL2ApqrqAbFDirLIFeGSnLA21F9Q20KE8U1kXpmp4helMDAueZfN2WOayri13BgcfpbrvtLi/2L64vd5sup/THD59jTKfj4fGRzCyLDv2gqnUd6roqSbMP3gCHfkgpzTGpaE65rcNXLy598HeHUxVC27bf//h+ntNm23775qZr24fD8eF4MoC6rlJMyNy0zdXl5W6z7cfx8Pg4xZTjrCKjiuZsa5timV5acnIzBTBBRCRkYgQUVXIMBC+vb64v9uW6DOPonDOAcY5EtNu4oR+2XUvsri53wzAZyPXl1W7bimRmMtUsUc0hoKioSokeJcU692bWlsgaAZc7eq6SDABSzvM898OUUkYERkCElHIpTHJKKSXJsuKV5lBBxRQkGZkKaB/jGGXKEAUNzDnHTLVDLmurwUTVs8Kic3jev2oAoFj2WayZcoEtzRCBkUqWSEAGkBHLB3ySOT1bGBaCvRGCyaq2gFhWJC4Z5EqJtDUrhVU3qlyeJROFFYGCFTpdW6ELpI8Iqm4YBvY+54ymTVNf7rfTFE/DAADOuU1bi+r98dTUgRGOJ+y6lhCO/dDUAQm994u5gOWcAVBUc1aE5Soc+/Hv/uN3dRUuL3ei+eFw3Gy64/ET4+by8qLttnMW5/zt7f2CZRC2TXNzdRXqKkp2jst1AijZn5oIwJPw3zIys+xEI1vYT+g8Bx9KzgAAPvhffPPm8dSf+rEKXlVD8Mdj/+7D59Mw1nVV1/XjYSCCUz+1zenm6pLYqaGa5pwQpOjDSE5lWXwxvTJBIWKAsJZTCE9Ts08VLgOC6uF0fHh8HMbRMXpytmx8N1NNKaeSAjJ7RgITQ2L0rmxN0WGYslq2wv6h4F1VcVfXPnjvGNk5dmKoBkxlXLgEFF1bkWVr3mo6q6vLGYAXGMzMSnOfzr0HW7n3sJZ9z5qU8LSgttjb8mC1Yr4L/LomrE8Zgy1lOwAA4wLJLWnnuco3AwB3d/+w6TZ1UyFSFu28u9l27dCkGA/HU6mKzFRFiHC36fp+VFMfqpJalFXe7JiZ4xxjEgRyjpwLRNS0bRHo+nz7cDgNL24uQqhev3q57br7x+P7D59fvLBt12mjl/vtl89f2LHfbC6/enV1c01IKaY4TvMc8zyLyOIP1kjxFDsQC4Nsyf+ICoQxp8jOVRjGOJ+G8f2nL01bf/Xyep5jGQkZpul0OmXRlHIV9NAPY9+/+eplynLs+8v9pffBlk2F8RzwVGXN64u/hjLHLuukh63wYBmjLc5DVIjIsYNSPs6as6uzD56dK9W+sYHzbCulgwAITAxMdU5SVoUTZCZ0zgcOm9pt2hCCRyIpI8tAWVF0aQqnnM/+HlciSzFTVaJis2vtwsxY8Jx1Azetqq5gcEZZl9q+2Fk5rCsmtQaQJX1d2vFrXnvOD0qHae3+rzmnlbwWzhSAAgE4RIwpMaP3PKf8+HiQ3KkaOvd4ON3fPVzstyEEBdtvWu8dO5YsOethmr1j7z0i+uB3m/Z4Gvq+B6BpNmZXVSGlHOdY7KYfhvoYXr9qHdmbV1fb7ebdpy9ZPn395vW267569fLz3cPx1Fd13Xjf1cGHahz6j2hzjDEWreEzjF/QNWLPBTMmIlMpooMqYiKi5h07A0Qkg9NpyFmuL7aaJKtuu3aSVFfe8/7T7f3j8RRCeHl9daxCSokRqhDmOHehcezmec650OaLyt+512K69GOYtJACFHGhjBGWaaSlri1Oixm99545qUrOmZAZUYAJ0TSJgCktjGcgMFQVkZg1GywUIjMkrIkdagBFzRINwLJhQmbnEZ0ZpKzTnOZ5nqa5GMfCGyd2jokZkFxZH+icqhYBVyIEoLN3LB+yMAxU4Tx2b0tBsxhSacevQFKh7q92CWVIfw36pa5XKLgBLV53wT9X77OKmxgAmquaJqV0OA2qaip1FSTrpmu2bcfE4zhOc5xTUrUUZ88MAHNMYsbMxBScA4QQfEqZHG93u5TSME0ppeAdmA7DmESmcWy7ZrvtRHWc5inGTdt2Tf3p9q5pmov9/vrq+uu3r999+ARMofJ1XV1fXV9fXbndxvi3h8+c+n6eJhEpnsuHyjHbKjKMgFkFwUA0RpBShaQ8i6YUTye32W3die/u7pu63m/boW8Ld+bVzeV+t5li3nTdHKdf//LbYZqiSDkEw9B37YaZNceYo4oyoxHTmkURAREvgt/ETA4KOYCWYdqCg5aStyT6wVMVWDSDFehOsJBcEZkwJpslg6krZsrMzlceagQzUEBH4GmZaYpqGrNaVjMgBjJnRIwiNk5pGMd+mPphlJQBrMwXOOaq8iEEdhzKDKRzxKyqwTtjXpC1QouHwpVHXuc0114RnuuedWPzMi1ccP6lolqxzhKun0DUdVxOn1WSxSJlzecXFMzAtU0DdfV4PPVDjwop5XGKt85tu3p/sa+rKqdU9qjePx4QoK6qzaa9vtzfml3utm3XTNNUenebribCH999FBHnXFZrm4adi/MsOeck4zTvdtvjaXCOHKIjAKQ/vnv/9ds3N69fzznNcxzH2aY4joOjFzeXV5f7i6tu8x9++49fPnw8HQ5xHDVnAHDeNXXdVMHMspqKOCYDO51OAgCZVLKplMS/ClUc43GeEZEc3z9WF9tNEmnbpvIupkzMnum2H+I8v371YpynOaamcobSDz2W/cEGZpbFQJSf5jYL/5KXze8/BfQWwOlZyVLgd8ccfCBUhwAAOWsZYxMAYvKrriwxe0fOccWEoFDm1s1ETZGQyJCSARM7Ju88MhsyIuWc5jiP03Tqh1M/pHkq8bocIu+9d857R0zeOb8sPfdVVdVVVVUBQIkcEZ3ppMuuxRW0KslWaUrpOfVZu6ylT7mw6kzLaJqt4MaSpCOWPTq0WnlJBmytLBhpGZqL88zOeeeD95IyMYlpnud5nh+PQ1WFtm32u00IPsXkvNt03W7blvp9t9s0dXU4nD59vq2bum0qM1C1qqqc45ur/Xa7yWIq0m66z1/u7u7uL7bdy5dXMemU0qaprrftw3H4fHt7c/PCeb/dbcVs6vsvnz7uu2az2Ww3u7/61Z81df0fu/b9Dz88PDzmafaEvq6NCiccm8AhhHIMtt2mrrKUPwCqxgDTNI/z7J2rQpVFjqd+GCdAeEn0+e4wTVMIXkT7YVgmNAkeHk9vXr2s64a5SE1JycsK/aWYGjM5ZMJlCKRgpEwOyia4Qr0lOkMzi1cycwx1IAACMFVLOUczptJ5J+fJOy6rugkNzUiTqZYOpQAoEDISOWQXgg+LPiYhUTZOOReGSE4pzlOcxnmeUaVM0TEzMlNx8lwmI533vqpC29Rz3bRNs2lbH0yJjblIQiwZZylo1rrz3EfiYlhqpR6Sxa0uWJusZq1rQ2lpMsGZSrKSa5/lb1r6+4DueDwhESMwsWvctm0FIOUiO5z7YYwxxhjbtmna2jMHR455Ur25vmya6njqvWNAHMYpHE9zTOM0Vd4H5yRLCOGXr199ur33pz6nfOqHz3eP37x99e3bF5++3B5Pp21X13X98PD423/87RCn4MPlfp9i8sTHx8Px4b4Nzabbvn756svD7Xw6BaJxnsFMY0opiTko/e2iCWaGAMxUheCCr9pWpvnxcDyqkBQyvzla7o9jqrwv7uHh8TiM8xRnRjyehqv9lphSlm9ef1XX9TkKFW9ScileSh+ErAbiEFUZCpy/gNNnhKlwiCyLmGRTUYPKMyEpWBbNKWdREVMxIlRzxFx7xwRQyHlZRFKRVzVidr5yyExNFXZt40MooLkYgFjKBAAIKlkkZclJcwaVUrcbrPMoi4myY3bBzXOwMl9F5BwbgA/IRCIFHi3EDlQDybl8PCIs43dSahxYkgKzsjF3CdxSli3COa+FonTG5xbXOq+9QgV2/pGZuXEYAJGYQgie3TDNarrdbrY3V4dTfzgc52lOMSJAVQUkvH14TDm/uLnqNpsY5+1207WN9+724RhT7vshjhMBsOOH07C7SCH4n33z5re/+66qw+HUG8D7j7evXr64udzPMY7j5MjGcfzhhx/atlNCQuy6hgDv7h5MNQT/6tXby93u9c2Lx9vbY99LSnmeQbToOjvvyHFgbupqTpnBCLBq6zevXjh2X77cMeH11X4cp34YPLskeY6JEczg9vauqirn3P39PSI1dX0/nNq6lZy3281ut51S9N4T01LWApDZuTsPBqqCCCLFUz6BjrZ2QWE1U0JidlVV1VVIKaJpYUMyITpCsIJSSFa1TEgelsoJc0oxDSklEQNgdlWNVeVqh23FPriq8kQIarNoEiO0lLNkycXsRVXFRNZeF6KqFE4goiILUc5sat67kKosOcZExLR+TEQsi6zxzPVcNywv2czyuHMHH/AZc54WT7oQmYlWqGrZMbLSROCpcaVLegSA4BAJCb3ziNiPI5h1bVv2Zu26jtn1x2PKGZBijGAuJQnep5Rvbx9Siiml66vLqq62G4kxHQ6HpmtD8DHlNM+3n2+/a8Lbr17udtuPX+6r4DdtfXc4PRxOlxcXVQjTOBZN4X6cRQyZRPXF9bUPPuU8zfHx8WG7u9hu9t++fptzDN7fO344nXKWGqBI3Q7DoBBdCGKmgMwURIfTgIhlXoCYri52++1WVac5TvMcvD8cj48Pj23bXlxcIGJKkdgR8uPxgI6TCjNt2rr2IVA4RyWAIvDLS5GEi6oblcGeEqnUgIsM1TJxVrIrIvTe+xBCCDmnMx0MAIr/K6BjykaQQRTAQAQkx5SnLGU1cPBGjJK9SgbLYNmUDagkuWiGpqZaCJAp5aLOvlKiYSnenpYciKqiQKmAVFVWPcKy+GaF9E0AHRMRmloRn4BlkGON13YWMoFnlNAClC4VUklyFBCXtc+2etelNFrnR8r/GQI6XwcAIGbnHDtnqqZ2PJ6yyPXlbu+dSsZproIHw2GMTPR4ON3ePRpCCN4xl/H2yntE7DaditZ19fHj55zyoR++fHkIobq82P3s69cPjwdTy2a3d/ffvn2bLvafb+9ubx/6YR7nuQqx67o5p007brt267oQguU8DaftZn95+eLPQqib7vu20/cfjg8PMs6FA2pmQ8o6TioCiOT9cRjv7x9LJOo2m6YKpbk0DsOifQI2z7OKxhinadZlcasgsyENw9jVNSGNU9x0mRMqETOvedgCvkJRpyPjJXxhqUNwaTItzFwod0MlpZhzLuMoomKS5yjDNEvKK4Vy8RyaJBGoWhJRkUJOMTMiqEUNyPvkfHRjJHKESOSyLbRTQkOErGpgxX+W5I/PZwhsAXnKxyAkdrSo1xqYZpGU85KflEX2RLAoZ67wvgEuumKll1foJvYUOgrAtLTdbQWsy4UyWqceoDSl7Jwi4DpEurhZV3QZGNkAVHWKMcZYBV9VYZxiQcjalolpTqmsRCrn0nlXhdC1TRbxzg/jBGb77abwmolwf7G72O+qJjgmEdnvti+uL999/BxjOg3j4fD48ubm8Xj68d2n4/GUUp7GqdzPh4fH/eUFImiMh8MRAJq6vX755sXli227vby4cm3zH/727x8fjjnmMjBUSAnMpGpxGBBwJHREoa58XdGEVFVTjKdTryLe+1M/ZjVDnFN+OJ00ZRERyuyYEVTy8XjabDomkiyZkI1XqJnKTA8qANhZN6SwnEiE2T2bqi1egUpDf44xxjnlbAaO2EiL85mTTHNcFIDBGNEzBbduvipebuUKC6AiipmKxBhHYgB0voBfCGXQouz/hGXAlxFhHYsuGQcTQZk/sXUAuTQ7DHJWp1r6SYgIzCbCAGVG2Vby3mqRZ/BywaHWLtOThN1aDOnZPMEgi66ud5F5MgNFKIt412lPUDCHZo6dc26a52mezdQ5Zu+K5pZ3DAApSco5WEEASUQNLHiHAMM4dm3T1LWI9MP48vpCzT58uq2b5s1XN23X7rcbVfvjD++7Tfvt29cXu+3d/cN+24hkNQvOGeAcY5qjqkqS0FQxp83n21AFU5VNF1Os6qYK1f76Zddtfl7XL65uXlxd/3f/w//j/R++xwTMDAbeMRKlnBAcmJFjDj6Kfvly54O/ubz0zAUR1Cwpp8qHnCKYaYxFb2xOsfNt1zTZ1HknotMc67pwTRacE4nMsNC0zYCJHLOacWkdFTdYeoLPaiRcun4mlj0B1U4ylqctNYtzLqdU/B0wMoKpeUeO0AC9Z1VX8FXvKHhGIDFMApANBWsHXApn1Zg1xmSSQZXAmACZzIxLMki0IOQlA8bSrSmsYU0irmi1qoFZFilbfYnQlMonRESwUiyspbgVd1mm5J7mjHnBbhefWnrFZcSq4BoLy3+1Wjgrh69oKyI6FUkpMVFb18xsZpXzxIwIcZ6D7+qqApvMrEwbIlFdhQJ7qVpK+XQaVVQBNl272XT9MKScry92bdPkJF9u76sqPB5OADDNcds1Kc3brg2e+1P/3Q/v+34owI2IxDir5FylL19ut7utqngmZjw+Pn5pPnKo9hfXdagqH/71X/5L7/x/+z/8j+9+/wfpe43zlBOxI+fIsqpYsphFzbxzdVWJSI6JmBxzVmFmIKtDJSLsuPQr0xy9d03beu9e3lxdXV7UdX1GAgUEoAhkoiqQWrFXOM/TFHIGmKrxsgeqXPlSk6BnrrwfmSVnXcIYIiJTGXgBoqfGtJqBISMQsQAqY1F+V0UzMqCshMaVq5grI1ZEE4hJxzmOc0opS15asqVmh0JtWXyoYdlju5CqSwZb2HIqqlky5YJJgRXxRirJI4oKIYKVDW24wqAABaVHW/v4RebgCdXHpRFma5vDZNG6XzT3cGm/L/ZdcAMnYpjyBNA0zcV+t+0aUe37YZrmcZKmCqPpHBMilgaDmbVtc3WxnebYD9OpHwhximmcpk3X3d4/Hk99OWgiYqpTTOM8A2IWAdPLi0tEmOb5y/3h/v7xh/efU8yExMxF2EPA8jSNw7DdbT27VXTTDodHHz5WoWq3F8zUtdt/+eu/AID/qa6///6P0+lkScoOVslJRbSkRwgK4KaoIsH7y8tLFbl/ePDOtU09V5WBtW0bQnCOh2FUkbZtmrqqq1AFf7HdOO/HccwplsLVIXrHziEXljygKqgYOjiDfcVSCQUQEFdhWwAids4D0BxlTnmc0zRHyQX21sLsoDKqU5TTJCdznm1tOoIYAAExsAIahZJKEgGwiKacj/30eBxPw9xPMeeMYExF8Q4IkLFMrBQneE5OIOsSAVQsZaUsRLlw82BJO4rntVULYrEkfd5kX7nJBTXlc/WOiCvzb4kqixkuoiVSvOw54KyggKoRmRNVVGW1nFLOeY6pwNCErKYfPn6JOXdte3190TZVGQgvcR8ARTITEnHsx2EYS8ibpjmL6KRVXe02HTEP09i1NTNPU5znJCIPj6eY8u+/++HUj8TkwK/NCairWk1Pfd8dT69eXjvvUsrTPIUQ7u9v67qu6oZdS4Sbzf6vf/NXu83mP/zHf/jdP313eDyMj8dxnrzjFFNOidi1bRu8F8nDlASwypkAtrtdW9XOk6myc5cXu912o6rzHMdprKtwud/VdV2CoZnVdZOZYpxzkR0yFikkUcgAZtHMiJDYOSQAKIkQEtPTyKNpwT+N1CyJjnOa5jTFmGKOsUhUWfkPIhAgM3lHCZSX51hLKNKsmhRb4OBDP6akAyGA6hjTcZiPx9PpNEzjVAi7YKXiXmaV8Lm+kkFxq4Rn2P1JwlJUEUnJcEXaiWjZu0aA5/K9IJq2Up/WLxErsSAvMX0ZIzyPdzACPBtFWrNYK52Bwo8jBWcqkiEjeO9yln6cJOdxnGOcY0oIcLHf3dxcXu53zFhCUzGlrg59TzFnQk051XXlmad5VlMTTWbznNqXTaiC907Xaax+mO4eT/ePx3mOwzghqCOXzZwr6xyEENqmnXM+HI7bTeO9c8xf7u6nOe62u+3mmGIMdYPEnvBie/EXv2xeXb14/erl3/7933/84X0YpjTNRIOvKt+2dVUVTKPIJ2Uzk4yio1mLVfCha1tmRsBt13318kVKkRDrqmqaJoTKAB8Oh9OpF0Mt60cyMBMjAljKgoDOeUA085JzKoU8Cy2Rs3iKJX4BQBYxUVWNMZsYQREIMUBgxy67Ocac85wTZYiJvRMm0rXqKjFymqkf4zCnOYn3wxKzAVK2aY7jOKYYJSXJmVau+1KLEBLAcgNxUaa1VYhgsdJ1sYfZMtJbyu9zik1Ey1KFIvVTxIYX4tKahK/D44Qrrrmw+59kSdciHp6p45QsFlZzBQNz5Yp67w2ggAumVrpH5NzVfvfi+qJpGu9YAWKcDscTIg5D1zR1COHqwo3DFLyP83zq+3GcmCirEmKc53EYr6+vjgD9MJpZP/Smev9w6PthnucqODB3sd8C4uPhFGNSkZQyEm67dp7j/cOhrjwTeucOpxMRD9MwTUPVdoGpkMSbugkvXrVNc7G7+Mfr33789OX+/jGXESUwIt50zaatifBwOFqZaidqmsYzg+mu28CicMRNs7m+anJOzBR8xc7N80SApjpNU5aMpaYWJRQgAgAkdEzOPfFxEU3VBISxeCtEOC/h0ZTyOKc5ZlXNZoAUPAXvRDUX+V9YFJ0KMxpMMz1Rh8tWxCzqsuacx3Fynpl96T5m0RijpKSaVaR0DhhtzTqXCkYLYPlUfy9qVnregQBwFn9U1TLnU6xcoQgzCSAScVEgxHUhaOEenjU4zkNwKwj7BKWdlZ7X7xReyfkcrkUSgLu8viyJ0zhNwzDUIQBAjDMivXpxc3mxTzn3w5RFihzSPCdEQCIXwnZTT+N4O47jOB8Oh2me26ZGtDLp4hz74K8v9oR4PPWEMEcZp2Pf90M/DMO42bQF5ri+vCjFLBM+HE79qd9uNvv9rh/GOUbnfcoCgHf3D0i46bZtu3HeEy4pOTm3313+5Z+1b796e39/++7jxz+++/H27m7sRxCZ4uyDe7nb7OoqNM3N9Yuuazdth2bj0JeCwBFXzhNRCKFcl3Kx5mk69sfj6RhjArAijZAKW4UBmDwXgkhpHZaq9KnjBypAvCTRZqo6pzzFPMecRXPOIooGa3O79ACXDmT5pXMxa2eUlNCxc75M2RAgRlGNySTnlMq0foGZFhmwFRQq70nO+jTPxjAUlh6DrIVSCfZudatgpgpFuhEWCYYyUmIASISqi0bVmYas69rLc9FjtkxaP0sGFnB0JSmYnhforGQ8h4BznEVVRQkoZjHTum6cc6Y2T1MuG/tirirftU1p5DLR8TikmCTnOaaHh8M0TT4UnVtYHAriy5urly9eTCmZWdNU3ofTMKUsh+MppUxEu91GVA+nHlfN4q9eXn8AIIJffvt6mOLd42PK2RGrChI/PDz+7vvvQte+9Z6ajnEhDDlHznVN017tr169evPN19+8+/DuD9//cP943227b796/ermRdM028120+0WEA+5LKlMaZKcoQD1gDnFnCIimshUzSI6TqPKIqirjOyYEIiQbZ3bRCvoI7GpKS/PvwCB610yBHLes3PFCJ2Dog6dRUubsNgnsitEOzAoQmLl8UXVh5kZSU1zloUwkXLKWUVKP6Cg8aWuIgIuk+eFf7EilbgO9JXcgxa6arHWkgoubQJf6n9bXKCaiepKgNflZCmULGoRTVvU1IqZPXXVAUB0ZZnimsICrMjU8m9Yl9CVpMGBQfB+nGbnedNtQggiEmPMIqdTP05T2zSd95uuqUKY5nma0xyTY/PBplm9D1nUwPYX+7atNMvx1AOY926aYZomZmJ22+2mqYJjPh57WE/vMIzeu8vmYp6jiHablpnmKb68uZzm2TFf7LY5y3EYsggCpDmq2u3nLx93P1xuLr0PhStUrjISMpBv27quL7b7b15//Zs/+02McwhVW7chBGbnuMyEcYlTRR8hS1X0wMqMmk8+zU5FlWVj29ev3w7j+HD/YIsXRLd0OYjo/1fWlzVJciTn+RGRmXX1NTM4uFxSF2W2K5ErM+n/v+uND6KkXRDkClhgMcBMd1dXVR4R4e568IisWglmAHp6uqqzMj38+Nz9+1idn1zQWEpRJAmhdlP8OftWk5mllOdlllI8x0VkKRKCqFoRLSIkAkgamIOKKVyHs91HQy6FAJMlU3XuQdEqOysiCEBemTumBQQI7LI+jfXzNtK669JqGSqqwaskqG2BZlIu0MhgtdejToSMykxaQU5rkRnQ84HW4L01U6tbRvVv1x+whoRWd65wfR1AcPVSJNoOQ9/1S8mlVMYbVWPmzWZ4erjfDN1lmqd5MdVSihO/Oxf9MPTvnx4B0aSM05RTZkZjTql8+vw6zfN2Oxx2m5KLiuRSxnHsukiE0zTNc9KisevLPG/6+Hi3/+Hjs6o93t+fp7mLHSJGotp5Q7hcLudxkpxU9Le//YeHD18hddxGN7w7gszM3HX9brev9wXqsHgtN82s0QICGHpbEsHQQohFjYOYZgQk5qHv7w53aUnTNJqpAZmq6/ENnfkABBiIKl5LYOe+8yfjqjAyT+Pn1+c///zL68txGqciDjeKk6aoqKhTpNZ3qdRHVbLFAA3N54mMauTVai6ga0mCzQIRkZkMkQCL2G1W556p9i49eiOymvhIlROnmgDwWtiLmvtJIoA2fK2qa9LpDrH6TjOvnHTd8GwZurYRE2g+3tYtu+ZzYXXYZgAQ3o7HGFtLvZRh6O/2OxF9O51yTiJlnufz5ZxyPy/L29u55GSAOZOo7Laboe/6vgPT4/H0w48fc8rMpFLmccIQlmVZ5nE/DPvd9vPnl6UUr+aJKWIU1SXnl+Pb0+NDCHwe568/vNvvt88vx83Qh8BLSkSwpDROc4yhjzGGcFnS88vx+z99/3j/sNnuAj8iYOCay9hNWIkc2u1YCwJoGb0nr6xKCFiggLgcHnMIkhEJQQARh34YhgERiVlFVKGYGUhnUkRCjNbc91ooYcXFEb1iMjVVkTKN8/l0fj2epmnKjc/H7dLLlrZ0Ve0T2qav+yesOiG26mFQm568AvBQc8/A6ImWeJG+Tk4BGJDnkYIGBmKgIkgC6rMr4lTsFfRZf6kBqgGajyJoLanEqAHvZiJCiC7S0owNWhy4LjJdO24tC/KkuDXuYW2YAkAIxKaaVZGZCWMgKdkMDrvtdhgATIo8v55jnLxNstvtzOWMnE4CcdP3IllVY9+B2WWap3EG02GzSTl/en65v79/ergvuXx+eSPip8e7nPI0L0hoqpfLqGp3d3sV/dNPv9ztd2h2PB6ZaLfbiGDX96/H03gZN5thM/QuIvrp0+efPv707sOXsd/2Qy/1Pq5OAqhlgQAVAGlnst4XJTJVInMw3b2R184q6uALEYUQu9gZQEoZfBSXsYAtuVAoUSSEQOi7H0De+AarDIZmagqmueRxPL+d3pZpKsuUl1mKz69Wxl2/t2ujUOvwbt1+YGifytvooCsg4+nlLcaNCIM3JwFc9kUdpScI5DcBtXLZeHPK1LCospqoUx2YqIooIXmI57oMCKZeBiliPZFXG8QmdLUS1xi0lYL6ceg6GVoN0LtcazIDdSTP3w3MLABiKYUCMWEfY8kyp0REXYw+R7Lfb0MIOWcA6AIV0Wle7u8O758ei5Tz+XyZplJK3/dfvH/69Mun17eTqCEAMavZ5+cXJuqHoeviZogiQwj8nN92203fdy5DP47jMAwcwsdPL2/n0VSnRZBOSLjZ9M4abgDjOKtoiNHMTufL9z/88PDuabPdc3hX0y5ne/zLzHw1WWtbyv7hGzRt0HSaPaAhAsWAhCKEIhx1v9v1fX86n52d3gyJTYXqHp96YlmI2Ut1NrMmpuYdRSnldD69HY+XyyWnZCpoyqAG5gsRRlghyfbItbJU1b0BbGABIJhhRTRbp7FB78hM3n8l8uFiNXBQBSITNXhStdgKdXp17WqctR9gzbZuzknzSczV7RVRpnUx8KYcXKWMW37piBys43kV62xYl7WEWGsIFFXPygAsUOAhUAjBzJ7fTmCKhkRUYjkcdtutSydE3G4A9Hy6HN/OMYanh7uH+8Pr8fh2PuecD/vdbrv55dPz+TyaqsvuDl3UUn76+dM8z09PTyHGIroZBgBMOe0326fHu58/PTNzP/S7bT+nwkgIQEy7GBDhMs4xhE3fxxhUShEdpznkEiMvRb/7/gdTtSz/4T/+5v7dB7AOIzrFu5/mFQJuRnnNyn1F08lCEFthQYjAaOjxrmZViJvN9v7ubhwv8zyr+Bqw525WREQMUJlNxYRVVEkNyZrFGwAUkbQsaZnzsoCUgCaEhAwVbgQDqNTlqutz9cul1nQxrOhhhSuxwuBQh4YBCTvGyAhIapDFzFtchMHFnRAV0GWDwMWV3DE2tQ7fVwb3kQ0tU1M0VFUAAjK6mZzzH0b0ee3q9lazXuv0m8B17ba3L2ogcNIGz3Bw3ZtFCA93ewMspcw5iyoDdF1AgBh5uxnAUEUCD10Xp2lKuQxD/8X7p4e7nWp5PZ5EZDP0h21/nufTecwixIQIMYau786XUaWMl5E5/NXXX0oX53kexzEQbYZutxmeHh/MIAa+vz/Qeco5dTEQQgguRCmXad5vN/vdxic6xmlx+DCGYGAfP/5cyj9eLpff/Pbv3335NeAWQsWr1nB/a6YtiGij3FAEcHYh8fBmLh0qJTuvTlHVwGEY+hACE5takYLkQhlkZiLKEQFBzLhKZNwsjps6WsQhumvSym5cGVyl+W6wSuB5kz+1SgKguk/w1LM2LeufwQDBZ1EdlzVA5y8lAmaMTIHaOryjN17yAAIYke8V14gjaiKmCiLAbP4ZDE1rBQ9QkInwirRjQ46qoa833k1wRVrMfbOfqNZeu75LS1bd8B3+N8MwjbMSDsOwDSGGAACBedN3+8N+6LpSJCX5/PLmyf7Qd/f3u/vDYUnpx58+vb6dnAT55e3y/PnldDqbKiFRF+72u76L4zQ59QO0RH5Z0ul8GS/TF+/fpyJdjH/7619N8xwDH3bDPMNhv0ulmFpOi+MvXeDArKrD0HdddxnnlAsiRGJDez0ef//NN1nKfwZ7/9WvzIYQAt2YI1xDjyG6Fps3bNVVNNR1F1TNtOSU05KXRaXUghiBOOx3hxi7k15KKWoaiVrCVyFPJ2VgDkxc/UR1f6amIcT7+4d3797P8zL3s5QCKqWISMEiRQTMyCmTbgGgmpkhNoYSJlwVDQzASbrALBBFxsAElZRBTTUiMFEgjETtfQEN1FX8GgZaTcopSwkQzKVOTU3VPKCDl2a8+mtPmB1sa7lTm51b+5a+Ue8zTOhQEa6Aa2WE8bH566vWFNU5DsDCOI4xRuw6ZBpih2ix67549wiI85xqbtvY8D03mZd0vlwu49jHsN304zT9/Mvn09sJEZ32qO/idrsRKYgQ2CcmZZrmXBVXMRV5PZ7u7ndE2HcRwAihlDIvqe/C4/1BDVKKx9NlSfP5TbvY5ZReU9ptht2mI4J5XpJACCwmb6fTH7/7bhiGjsPjF19Rpb+sYkt2Q0ForRIyAxHnGpOa+bkWr6mWoipLWtCLAyJk2u52T48Pb29vS81ZDcCHmdANXW/ZCsxUFEyaIg0S0n63+/qrL4ksLYsvNqioqlzGaZ6XeZrnaSpLKqWQKoAFdBQIGI3b4g9C1Qx22fM6gk4YAwYmYgSArCZi7JA/YSAicn5iED+S9VKB1ll2aPpETeLDTyY28NIfELtCcsud6gWtZnU9sjUFsAa/39ZSUAv2inf5oamG5ZVuS3v9r0M/9EM/IMA0zqJyt993MV7GKfbdYb81Uzf81+NbSnkz9OM4//zLc8o5MMUuppxTyjHwbrdlDt5q2++2fcfjuLgmZwisIs/PL0S0rmPPKQ1Lt+kjaNkOXZFSSpmmaZ5HAnh4uNvvhqGP07y8HM9pOauUlHJKadP3wzAMfX8Zp2UuMUYA+/zp8//I/6Sl/P3v/uvDh6+67trMWH1nM9BKoVNDjohlM8JVAAAU0klEQVQrvZqolmyiOaWcZibKOad5jl1HHIZ++PDui7e3t1LyspiqNXHJCg/lIlEkWhsG97holVWGiGKM2+3m4W6/LAEQCTEwRWZ3iCmVcZrf3k7H49vlfJnGUdLi9IDNkRpUGseKwjuvCSEGxuCDz0h+4hyAgNY+1iZG6KCStQwB0Nj/Cq6dr9XkzEzBGKyIsoFVeWFTA18guX5OJCKHe625hVoJ3DyCa24KrU/QlBoqCtaK2mazAIgY+mEAAG+YbTeb7WaQUhBx33XELAJD3xsYB96Fbd91x7fTOI5mlgmnKQGCSjFVV84khN7FN8w3zDGECICvxzMAPNwdfEp/M3RoWkqeTYjg3eNDypazT/rJp+eXnJM3n97d7fuue3l5ndKSMuVctBSVMgzDZujnJaWUWIRDPL29/f6bf0bmf/gv/+3h3ZfO/nzrO/0fUZ/WUfBSybF7KVKSqJaS1URUfTt5LsnA+h4Dh81mc3d3fzydVM2joWuEurn7ZISocuU0WNP8+gUhdTEG5sLeyHTRTQ4hEtHhQO8M8ldf5JzGcXp9Pb28vr6+Hi+ns6YEph5IoaV+DUqjwBhDna9KIimrgSFSdAYyX1ABUDVXAyX0IU5AF+9qdYqDkQAgarG6TwfjIXBYz7rBKg2vWNeDV/j1mne2fytE2tgc1vdABGCsaBo2LMKal8WbWxfSkjwJ22yGu/1OVHNKRLTMy+UyLkvaboa7w/7x4T4wPb+8juOcixRnU0FERBFx3h816yIPm6GLQUWYYxchMJqJFEHmYehcE5sQRcsyT8qMaGPf+XOMgU3AVMZ5joGYsKiGEB7u73IpqjbPKiKXaU6lDH3fdbH49EaBjPD6+vLNt/887Ha/4e7w8BhjXA+iNXTYP3xt/YmoFKtTOwUAfcyAiUwEwPq+rxQHgfuu326Gvot5SX43vdwpIszk3SMRp0EXqgU2rc7AzFJK85JySj73TQhMAbFGW0Qahj7QQR7t668kp/R8PP7y8y8f//zx9eW1LKkOVACuOyRdpMiIxISYRKesZtYxxUAxUDWOlte1C0aDRo8P5hu+qwFds1t16/GMYuXmNwKDtm/kS/FNCWSt3GGt8lcss/72ViN52ipqDbwGaA/Jv1rzB0MI5/M5MHd9L0VO5wsTiRlSEpVlXsQgMOciqioERHQ4bJeUx3H26biUMjP1fURERo0h7LfDdtOfx7mIhIAxhi4ENT0cDofd5sePn0rOgYmQci59F/vA87Lst/22C6UPS/bDDSKiJRFGUNh0PAz9ZZz6LoyLOgBNYF2kwj0nzKWAaWQoy/Ttt38Ydrvf7H7Xdb3vErWehqeL2cSk5JyWZbzMy1RymecplZxTulwu0+U8TvNlHKd5STlpKSpKIbhevKn1fTS1yBzIwRfMRTopMXSe4NYUoiVStgY0oixlmhcmCrEO+/SddV0k32NzbXeiXb/B3e5wODzd3x/22z/+8fvnnz+ntIDnnWgGFKOTgRERLWJzElGL7AeMAyIhFKgmogBeqRCBKDQYx9bGWgvX0K7XmhHXf1SNqxRj7QtVf+m912piCP/PxmBNWf3+rzm6451XIKBG+YoDXAslAAsi4tG5SOm6oKq+u+XEYPu+N7Pj8Q0A+j5uh76PztqDSwrjZSKSYdOHGEwNI3dDT8zTtKRlJoTtbui76NFov9vMaVly9hnJ2MWHIW6HIVdRoqHrY99Hn7rBSgBquWRE6EL3dLc9n08FYb87FHGiIjrs+t12k4ucx7kU3Qw9MlrKnz5+/D/dN3eHu1Jknqd5msbxMo7j+Xyaxul0OU/jNLrk2zy7Y1MEyaWUDIjEDKrMrAbe+0OEEHjouoe7u6en+4fDduBoToPp6aC2Hnq742YOULuMhnEI93f3l8tpmWeVUkrxOXaHG82gw87bWqoCGfqu32w2zCEXOZ/H6TJKya7KbIDR1eh8lUp0TiWLej3KgYNXRoBkVgB1PSV1J7giTf57nV1RTU2JfdXMTGpC1BDZFnDV1h+t5FRt2xOc56FBntU8bd3ywCv1/FpI2fV7a+SH1ZXXEN910flPOAbRK6Y2dN3Qx1JkmiZiGvqeENSsFCHEh4c7MJgOyzLPuRRABAZVNrW386Qli8pmGDbD0HV+t5AQUi5oQkxD3x12w9BHM8tZN32362PH0IG8XfA8zmYKpqbFFKdc5iUzcd/15/PRpyoQaMrLp9cLM5nBksu8ZFH10XH77/8YkBgxO5qj9TarFxzXxAjqkSfutwN2Xcm5C7zZDGYQQ4yBY+RI9OMvz7mUKRc5HqeUXt82D4fdX33x2K0TVQamUkphLoGDEjuICC3CMtF2s/vw/ot5mY8vn6WU5Gv12lAEr2gYzCDlUkQGHQKH/XZz2O+7vgNE393pAm0iIyIB5iznJKUIB45MHVMM3LEPF4MaYsVcnSyy1s8ebREAQdFFPAy5Si9ALZEcuIHKxlvb5dV0gAwIkW59J9xaFzTErNke4K1vrT93BQpuav+bFwJAIGIzW1LCXJjR5w4J0VTO55xTJqZh6D3zm5dEYBy7/XbbdbEUSWl2AF9FUhFULVIA8eFuv9/vNpshBEKRcZ6nSS7jlHMhMJMyT/M4TiaSckGiTy9v05J1WS5LHuekRZFIAXIWABAz8WxJ69SPj3m120yE6BpnDdhBW7sUBsgUQyBmcNhTxZ+CAAQwJCZmy0UAus1GU0pzssBzLuxxvGgMrKWAWVablgWILks6nS9/92//+uHurqiIlFzIAEMouWRiFqnL5j5LogYU+O5w//WXX0vJ57ejw3ArDsUUQlBELKLTNOWS+3642x9UrY+hC9HxUCbqmAlRDHKROUkRDUx9oL4LQ+AQAiCKT0itjsjvk88p+7QIgrpUMQAYtLUNU1UPCBU2p1r0GxgBrrdX0XvuWOEqwLrCVZVr3LPC+vv/vy9uc8/qpKtZ37hY8CIJwYpqYy7H0MUuRwQTNSbaxk3OZV4SghHRdjv8+ssP7x7vmclUcu7PfZ9ymqZFL+NlSSllM8spv7xeFKGksizzkkopJeUiClbhx3ppdX6mHqu6boitaXbTCmqwBDa7A+v7jpjHcfJDSE5I4LEMcf/01B/2m74//vSxlLx/fNrttv12+Paf/rfj8GQmZqRqpkKBCEEEkUrO5CR3mgWgeNXffq2ITcvCRJ9Tmv7Xv/zm7/72w9N9kQKZkIJHM5FCRCLA7PHNPwrGrru/fyqSfzSYLqfWLii5MC5zylkNRORymVLJm83Gw/C8LFnFAALjEJkIFUBFp1SKKBP2kTdd6COHwNAcJCGwuZKx943AsClyuHX6NEkbdyqiQK71B0T1IBOCoSkh15ebI/dOzF5dI1FFNFuqXQe5V3d6Dd14PRPXx3vbC71iyf6tUHLy5Al9Y8EZ9IoQIjEjwGW8iKgUQcS+6/a77dvr6XQ8zSm5bsU4ZxE5T9M8LUWtqFopVmn1aibTnH514h5dvPVgNXmutSZWYgH1DAZUCakWmW5+oqYCxIioCiLZcy4T9SBpRZylo8zz01cfdveHy/PLdLmcxwsEvFxO5lNkhABYZUGKACr5vTUwU9IqKaBtrZ2Q0JCYDbGIFjUEyHn8n3/419/9p39/t9+JFNMiJQsHQhLMAIgoCGrkhIcIiH3fPz68E7XnzyG5uA8REqVi83yelrTMaVkWIALDE5/Ssnz85WWeZkJwvnA1SKJLKqkoEXaB+8Aup+SR3Wyl7qqYe+X2rF4R18UkA59RMbvRj3Oks9qKGxgSVO7/6u+rHdY/tI3N6voaRXRD8HHNAW4zgBs3uXaQbssydzSuduqqJb6SYmVJvl4j1mZbmkTZZZqej2+ilfy/fmTC2lpoJQJWVj4lbAwpWHtdgCDSph0QsfaOgYmM6s42EiNCSdlXPZG5nTIjZslFTJ3iAwhFPBkidRUT36pBBIBlGv/0zbcqlVQ7ny6fz2cRicQYgvkcbt0R9t1bD7VcJ46d1c33exAICAjNVPyB+Z0xe7mMv//2T7/7zb+LwdKyeoyq6mCmzExgQAZWPQ1z2O/uum5AxMDRZ2tSyqfz6eX19Zfl87RkMNAiry+v8zgdT6cyzQGRyTvizvUiRLSJFAPHQFWLEcBXLg3RAVkD1FaA0NWJAiBBZchBbfi8udxUq+9xHQiogZtW4hv/ayc7gSY5fq1+/H0rHEa1adRstMX0vzDF1XGu7+/1QvCRQZeq0gYZqpo6S4CPmdSBNChiAE2/o5Ua0LABu7p09AYxUWMd8DlwIkV0CW0MoePgFYK7F4C6mswtlDiJMIhKyVKkpkn105m07BsQjcldgVNZYtVGA1N1/9RSPehCrFmA4xwtDtXM1dPbGLz1ylqRca2llvPUARMBB4rR1VOeX0+vx9PT450VQsoGUKT0XSche6EVYgQItfAQUZG+73fbfQwxhFCLNpGnp3fv34/v3j1/9/33f/rhxx8/firLwqBsQqaAoICmNmXJRZgxBu4jd13sYqSG5DhCpAZSR+zB5wZ8mUIrY0MNWW4lAUGrAyEidI7bhiBDS5QNXSCs3kqqVdJNZolVMQLEqq/0b1q7t6s1ri7zLxwnrq+oxGMAEFyVR1Swvq7CqvWtoYqGae00GFRx3CpgX/1oiyX+cg4hdhGRvOpUNS3ZT23d0DbQItOSKmq8TgqQ3xcFAAKqd0FN/fIIpW2ZWz0O6IeePG1vC0cYWK4pbFuaRIC6OuZsa2ae+SMGZKe1r8hKQmLWXFwecm1DOQRpAAURuZBqiB0xTSn//Hz86sOTc9XHGGIIoiZpURWLilh5AXzXMYSw6frA0Yc369YDABKGwLvt5vH+/unp6Ztvv/3lp5/yNEYCBhKDopZLyUUNoAu06UIXA4fg5Q9jZRfzR1H5u7V6eh+sqv6krntAnbOzSqy/XgggNk4cvHGlNTez9Ut/4Zo1ErVhwesNtxpTbQ33t2DcreOE1pLFduGIEJD4WgFUC7WVuKyaf8Vi1y0T0yY9iIDrgun6i31QB8FW2vaVMkpbc78+b0Q0tDVNrhJEgDXBUKzTWw51oI9F+bVRQxCb0/ZLVIXKKOd4HiG6rwVfloPKCOzllBYLTP5snTLJ3bKJBGJPNOv8uidrqGbq6bMVEcwAQcwuWTwKeoOVQzA1p05ELGZKRBy6GHo/vYEjVa5Or63FzCU+LIbw8PBwOBy+/uKLf/3jv3z3x2/H1xfJUkRTKbmYmXWRQwgUmCudrgJQMdCGuzeXYT4jpw3P/EvPtaZ+BFUIgpl8lg4JyYeu1dwH11vdkKgaM2m1O2gWA1Wy+9Ykrj1QuO4bXd0qrtkqXOslNQMIfoIcFHA/4Y8EW+BTbaOyiCvBxNoZa1F9nT8ARwOaL/KOAbpktyGtVJrNE3qeBC1/bR+hJkHg5Fa3rn1NGLQ11xBx3QX09EDluhpYf+paollTP6nfN1W1AoimRlWWDwCsqCATuC6Lj3+hVdm06h8MDdCMmNOcUirUU8qFQzAtuRTnIo0x9v1mt9v3/cYNlOoGRXsHADMrLu7BrhGDzPzh/Ye7w/7Du8dv/vD7H/70Yz6PToDXRRpiiDF2zA5OO/zu8TG0FhGs8xCoq/34HSGAutJqoADMFJidv5KZXfpbnaaHzNePSK3uK5iBmrfSDUypqsyoGQFBW5harRXbI7uxSAODdZMJrwG/rot5IxqJQDVwDKZa1XEBWy9r3SgADq3DBUbEiOsZrWRU9QJW4/c82we/Ddb2N7QawVq6oLCOfpEUBwV9VNHM5Si6vjvsp5cXLeLKj+t5rElxjVGK2OgrbnqMyEziCJGfKN9Qg7prVtMEBEABIzNP9LniDz6dWdc13ZzNFOsX6MvhaErABJCWdJnmGNgAci6qMs9zEYkh9F3Xd7ELMbDLDVQc11o9a35hKgZiVlFFzx77fvvll7+6jOPHz8fl9SSqMfIQOUYOTB7Trfmu5ubqLVh5ofysqvojaoPrHsF8x44CkEsmVTkGf8wG5s7K6tq7Qht2r12g5tFAa/sDKmiDayMNWm/dh0wBoA0A6C26szr9a0dU1cBC4GDkV9+yzBXz9yQFVwqK1ZFdE4LqJAFahmotCBgYmppJqb/MBZkBEZCYQgzU93le+s32/a//5vP3352eP/nkm98/DhyGAZlXuoHmQ/2Wo62p/urHGxeeX7yWQmui0k5KayZXdwrY+hmInjzUil4rhQG1zNMqveD67mCEqqiqhJRULuOy7bsQ9ZxzkYKEfey7GBFRREQzC4OBS87VigRJwVRFck55RgAIaIDcVkSRHDp93O52hhRi6ALFyEhciRPBRF1WGOlKKltp3hXWR2k1ufSPpVp/DpGIkchtE4m9dFQzMedchIpauas29CE8n1wz5JrOX2MgoK9IA1R2xTYnbpW/pBXWWjmhK2TQ7iy0+RJ/r9CYxFo9ezPLX61hHUg0xXYckdZThboiKus0YU05tCqPQisIyTcfgTnsHx9ou3n788+B+fD4eH55eXv5bG1AC8yAKA5DWhYTqQOD1bdd8d0GSHjMaFnP6rO19javiVe7EbWkb6mtf2y8Ge+u1r/CBgBonrbadZxOGwsWqojlnEpOl1HUlJm2my1VsFZLkZQyGDEbMleCFwaj6ziJFEHweX8hysxBW7L3eH//b/7mr89vx+V4JIDaO1YpAMVA1Vu8GJmtldV+gNvYKtQWsHtvtSKVS9DpJzgEInZtEiBSF7kz8HWkla+UjMzZmKAen+Y/cS1IoO3lwU0BtP5XVrTJy6D66JzpxLHv+hRW6wsVWGnfamlus83V/66bMrjaH4A76ht7lurq6oPFa+FVk0VCNB8QE6MlA6CJ5mXG1kZyX4vMHAIA2rI45o/12lrN6G/rw93tTNj16AEAEhN4U98BPzPTumpOgOZdaPVHxL5eiTXpqEmVrSxbjfN//WDr/xTBEFR0zmVKScWQMDCbWS4lFAakEDRnAcgBkFwD07nNlCpMiBS7IS2jSVYzJEk5OTMCMyPS4/3h3ePdz9MIIoC1/WMAYi7dC4YE4Pp0gK5600YLPexXIohKKA5iQOs4FDMyM3GFPG+Ki3rP1JiaMmqlGSOX4VLzHN3M72ZlnKwv9lu5GkmN0c3D113VenbcsNHaipQ/7v8LuhRpugWd6nYAAAAASUVORK5CYII=)" - ], - "metadata": { - "id": "gW4cE8bhXS-d" - } - }, - { - "cell_type": "code", - "source": [ - "image_data = (periodic_impulse | beam.Map(lambda x: \"Cat-with-beanie.jpg\")\n", - " | \"ReadImage\" >> beam.Map(lambda image_name: read_image(\n", - " image_name=image_name, image_dir='https://storage.googleapis.com/apache-beam-samples/image_captioning/')))" - ], - "metadata": { - "id": "dGg11TpV_aV6", - "outputId": "a57e8197-6756-4fd8-a664-f51ef2fea730", - "colab": { - "base_uri": "https://localhost:8080/", - "height": 204 - } - }, - "execution_count": 11, - "outputs": [{ - "output_type": "stream", - "name": "stdout", - "text": [ - "\n" - ] - }] - }, - { - "cell_type": "markdown", - "source": [ - "3. Pass the images to the RunInference `PTransform`. RunInference takes `model_handler` and `model_metadata_pcoll` as input parameters.\n", - " * `model_metadata_pcoll` is a side input `PCollection` to the RunInference `PTransform`. This side input is used to update the `model_uri` in the `model_handler` without needing to stop the Apache Beam pipeline\n", - " * Use `WatchFilePattern` as side input to watch a `file_pattern` matching `.h5` files. In this case, the `file_pattern` is `'gs://BUCKET_NAME/*.h5'`.\n", - "\n" - ], - "metadata": { - "id": "eB0-ewd-BCKE" - } - }, - { - "cell_type": "code", - "source": [ - " # The side input used to watch for the .h5 file and update the model_uri of the TFModelHandlerTensor.\n", - "file_pattern = 'gs://BUCKET_NAME/*.h5'\n", - "side_input_pcoll = (\n", - " pipeline\n", - " | \"WatchFilePattern\" >> WatchFilePattern(file_pattern=file_pattern,\n", - " interval=side_input_fire_interval,\n", - " stop_timestamp=end_timestamp))\n", - "inferences = (\n", - " image_data\n", - " | \"ApplyWindowing\" >> beam.WindowInto(beam.window.FixedWindows(10))\n", - " | \"RunInference\" >> RunInference(model_handler=model_handler,\n", - " model_metadata_pcoll=side_input_pcoll))" - ], - "metadata": { - "id": "_AjvvexJ_hUq", - "outputId": "291fcc38-0abb-4b11-f840-4a850097a56f", - "colab": { - "base_uri": "https://localhost:8080/", - "height": 133 - } - }, - "execution_count": 12, - "outputs": [{ - "output_type": "stream", - "name": "stdout", - "text": [ - "\n" - ] - }] - }, - { - "cell_type": "markdown", - "source": [ - "4. Post-process the `PredictionResult` object.\n", - "When the inference is complete, RunInference outputs a `PredictionResult` object that contains the fields `example`, `inference`, and `model_id`. The `model_id` field identifies the model used to run the inference. The `PostProcessor` returns the predicted label and the model ID used to run the inference on the predicted label." - ], - "metadata": { - "id": "lTA4wRWNDVis" - } - }, - { - "cell_type": "code", - "source": [ - "post_processor = (\n", - " inferences\n", - " | \"PostProcessResults\" >> beam.ParDo(PostProcessor())\n", - " | \"LogResults\" >> beam.Map(logging.info))" - ], - "metadata": { - "id": "9TB76fo-_vZJ", - "outputId": "3e12d482-1bdf-4136-fbf7-9d5bb4bb62c3", - "colab": { - "base_uri": "https://localhost:8080/", - "height": 222 - } - }, - "execution_count": 13, - "outputs": [{ - "output_type": "stream", - "name": "stdout", - "text": [ - "\n" - ] - }] - }, - { - "cell_type": "markdown", - "source": [ - "### Watch for the model update\n", - "\n", - "After the pipeline starts processing data and when you see output emitted from the RunInference `PTransform`, upload a `resnet152` model saved in `.h5` format to a Google Cloud Storage bucket location that matches the `file_pattern` you defined earlier. You can [download a copy of the model](https://storage.googleapis.com/tensorflow/keras-applications/resnet/resnet152_weights_tf_dim_ordering_tf_kernels.h5) (link downloads the model). RunInference uses `WatchFilePattern` as a side input to update the `model_uri` of `TFModelHandlerTensor`." - ], - "metadata": { - "id": "wYp-mBHHjOjA" - } - }, - { - "cell_type": "markdown", - "source": [ - "## Run the pipeline\n", - "\n", - "Use the following code to run the pipeline." - ], - "metadata": { - "id": "_ty03jDnKdKR" - } - }, - { - "cell_type": "code", - "source": [ - "# Run the pipeline.\n", - "result = pipeline.run().wait_until_finish()" - ], - "metadata": { - "id": "wd0VJLeLEWBU", - "outputId": "3489c891-05d2-4739-d693-1899cfe78859", - "colab": { - "base_uri": "https://localhost:8080/", - "height": 186 - } - }, - "execution_count": 14, - "outputs": [{ - "output_type": "stream", - "name": "stdout", - "text": [ - "\n" - ] - }] - } - ] -} + "nbformat": 4, + "nbformat_minor": 0, + "metadata": { + "colab": { + "provenance": [], + "include_colab_link": true + }, + "kernelspec": { + "name": "python3", + "display_name": "Python 3" + }, + "language_info": { + "name": "python" + } + }, + "cells": [ + { + "cell_type": "markdown", + "metadata": { + "id": "view-in-github", + "colab_type": "text" + }, + "source": [ + "\"Open" + ] + }, + { + "cell_type": "code", + "source": [ + "# @title ###### Licensed to the Apache Software Foundation (ASF), Version 2.0 (the \"License\")\n", + "\n", + "# Licensed to the Apache Software Foundation (ASF) under one\n", + "# or more contributor license agreements. See the NOTICE file\n", + "# distributed with this work for additional information\n", + "# regarding copyright ownership. The ASF licenses this file\n", + "# to you under the Apache License, Version 2.0 (the\n", + "# \"License\"); you may not use this file except in compliance\n", + "# with the License. You may obtain a copy of the License at\n", + "#\n", + "# http://www.apache.org/licenses/LICENSE-2.0\n", + "#\n", + "# Unless required by applicable law or agreed to in writing,\n", + "# software distributed under the License is distributed on an\n", + "# \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY\n", + "# KIND, either express or implied. See the License for the\n", + "# specific language governing permissions and limitations\n", + "# under the License" + ], + "metadata": { + "cellView": "form", + "id": "OsFaZscKSPvo" + }, + "execution_count": null, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "# Update ML models in running pipelines\n", + "\n", + "\n", + " \n", + " \n", + "
\n", + " Run in Google Colab\n", + " \n", + " View source on GitHub\n", + "
\n" + ], + "metadata": { + "id": "ZUSiAR62SgO8" + } + }, + { + "cell_type": "markdown", + "source": [ + "This notebook demonstrates how to perform automatic model updates without stopping your Apache Beam pipeline.\n", + "You can use side inputs to update your model in real time, even while the Apache Beam pipeline is running. The side input is passed in a `ModelHandler` configuration object. You can update the model either by leveraging one of Apache Beam's provided patterns, such as the `WatchFilePattern`, or by configuring a custom side input `PCollection` that defines the logic for the model update.\n", + "\n", + "The pipeline in this notebook uses a RunInference `PTransform` with TensorFlow machine learning (ML) models to run inference on images. To update the model, it uses a side input `PCollection` that emits `ModelMetadata`.\n", + "For more information about side inputs, see the [Side inputs](https://beam.apache.org/documentation/programming-guide/#side-inputs) section in the Apache Beam Programming Guide.\n", + "\n", + "This example uses `WatchFilePattern` as a side input. `WatchFilePattern` is used to watch for file updates that match the `file_pattern` based on timestamps. It emits the latest `ModelMetadata`, which is used in the RunInference `PTransform` to automatically update the ML model without stopping the Apache Beam pipeline.\n" + ], + "metadata": { + "id": "tBtqF5UpKJNZ" + } + }, + { + "cell_type": "markdown", + "source": [ + "## Before you begin\n", + "Install the dependencies required to run this notebook.\n", + "\n", + "To use RunInference with side inputs for automatic model updates, use Apache Beam version 2.46.0 or later." + ], + "metadata": { + "id": "SPuXFowiTpWx" + } + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "1RyTYsFEIOlA" + }, + "outputs": [], + "source": [ + "!pip install apache_beam[gcp]>=2.46.0 --quiet\n", + "!pip install tensorflow --quiet\n", + "!pip install tensorflow_hub --quiet" + ] + }, + { + "cell_type": "code", + "source": [ + "# Imports required for the notebook.\n", + "import logging\n", + "import time\n", + "from typing import Iterable\n", + "from typing import Tuple\n", + "\n", + "import apache_beam as beam\n", + "from apache_beam.ml.inference.base import PredictionResult\n", + "from apache_beam.ml.inference.base import RunInference\n", + "from apache_beam.ml.inference.tensorflow_inference import TFModelHandlerTensor\n", + "from apache_beam.ml.inference.utils import WatchFilePattern\n", + "from apache_beam.options.pipeline_options import GoogleCloudOptions\n", + "from apache_beam.options.pipeline_options import PipelineOptions\n", + "from apache_beam.options.pipeline_options import SetupOptions\n", + "from apache_beam.options.pipeline_options import StandardOptions\n", + "from apache_beam.options.pipeline_options import WorkerOptions\n", + "from apache_beam.transforms.periodicsequence import PeriodicImpulse\n", + "import numpy\n", + "from PIL import Image\n", + "import tensorflow as tf" + ], + "metadata": { + "id": "Rs4cwwNrIV9H" + }, + "execution_count": null, + "outputs": [] + }, + { + "cell_type": "code", + "source": [ + "# Authenticate to your Google Cloud account.\n", + "def auth_to_colab():\n", + " from google.colab import auth\n", + " auth.authenticate_user()\n", + "\n", + "auth_to_colab()" + ], + "metadata": { + "id": "jAKpPcmmGm03" + }, + "execution_count": null, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "## Configure the runner\n", + "\n", + "This pipeline uses the Dataflow Runner. To run the pipeline, you need to complete the following tasks:\n", + "\n", + "* Ensure that you have all the required permissions to run the pipeline on Dataflow.\n", + "* Configure the pipeline options for the pipeline to run on Dataflow. Make sure the pipeline is using streaming mode.\n", + "\n", + "In the following code, replace `BUCKET_NAME` with the the name of your Cloud Storage bucket." + ], + "metadata": { + "id": "ORYNKhH3WQyP" + } + }, + { + "cell_type": "code", + "source": [ + "options = PipelineOptions()\n", + "options.view_as(StandardOptions).streaming = True\n", + "\n", + "BUCKET_NAME = '' # Replace with your bucket name.\n", + "\n", + "# Provide required pipeline options for the Dataflow Runner.\n", + "options.view_as(StandardOptions).runner = \"DataflowRunner\"\n", + "\n", + "# Set the project to the default project in your current Google Cloud environment.\n", + "options.view_as(GoogleCloudOptions).project = ''\n", + "\n", + "# Set the Google Cloud region that you want to run Dataflow in.\n", + "options.view_as(GoogleCloudOptions).region = 'us-central1'\n", + "\n", + "# IMPORTANT: Replace BUCKET_NAME with the the name of your Cloud Storage bucket.\n", + "dataflow_gcs_location = \"gs://%s/dataflow\" % BUCKET_NAME\n", + "\n", + "# The Dataflow staging location. This location is used to stage the Dataflow pipeline and the SDK binary.\n", + "options.view_as(GoogleCloudOptions).staging_location = '%s/staging' % dataflow_gcs_location\n", + "\n", + "\n", + "# The Dataflow staging location. This location is used to stage the Dataflow pipeline and the SDK binary.\n", + "options.view_as(GoogleCloudOptions).staging_location = '%s/staging' % dataflow_gcs_location\n", + "\n", + "# The Dataflow temp location. This location is used to store temporary files or intermediate results before outputting to the sink.\n", + "options.view_as(GoogleCloudOptions).temp_location = '%s/temp' % dataflow_gcs_location\n", + "\n", + "options.view_as(SetupOptions).save_main_session = True\n", + "\n", + "# Launching Dataflow with only one worker might result in processing delays due to\n", + "# initial input processing. This could further postpone the side input model updates.\n", + "# To expedite the model update process, it's recommended to set num_workers>1.\n", + "# https://github.com/apache/beam/issues/28776\n", + "options.view_as(WorkerOptions).num_workers = 5" + ], + "metadata": { + "id": "wWjbnq6X-4uE" + }, + "execution_count": null, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "Install the `tensorflow` and `tensorflow_hub` dependencies on Dataflow. Use the `requirements_file` pipeline option to pass these dependencies." + ], + "metadata": { + "id": "HTJV8pO2Wcw4" + } + }, + { + "cell_type": "code", + "source": [ + "# In a requirements file, define the dependencies required for the pipeline.\n", + "!printf 'tensorflow>=2.12.0\\ntensorflow_hub>=0.10.0\\nPillow>=9.0.0' > ./requirements.txt\n", + "# Install the pipeline dependencies on Dataflow.\n", + "options.view_as(SetupOptions).requirements_file = './requirements.txt'" + ], + "metadata": { + "id": "lEy4PkluWbdm" + }, + "execution_count": null, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "## Use the TensorFlow model handler\n", + " This example uses `TFModelHandlerTensor` as the model handler and the `resnet_101` model trained on [ImageNet](https://www.image-net.org/).\n", + "\n", + "\n", + "For DataflowRunner, the model needs to be stored remote location accessible by the Beam pipeline. So we will download `ResNet101` model and upload it to the GCS location.\n" + ], + "metadata": { + "id": "_AUNH_GJk_NE" + } + }, + { + "cell_type": "code", + "source": [ + "model = tf.keras.applications.resnet.ResNet101()\n", + "model.save('resnet101_weights_tf_dim_ordering_tf_kernels.keras')\n", + "# After saving the model locally, upload the model to GCS bucket and provide that gcs bucket `URI` as `model_uri` to the `TFModelHandler`\n", + "# Replace `BUCKET_NAME` value with actual bucket name.\n", + "!gsutil cp resnet101_weights_tf_dim_ordering_tf_kernels.keras gs:///dataflow/resnet101_weights_tf_dim_ordering_tf_kernels.keras" + ], + "metadata": { + "id": "ibkWiwVNvyrn" + }, + "execution_count": null, + "outputs": [] + }, + { + "cell_type": "code", + "source": [ + "model_handler = TFModelHandlerTensor(\n", + " model_uri=dataflow_gcs_location + \"/resnet101_weights_tf_dim_ordering_tf_kernels.keras\")" + ], + "metadata": { + "id": "kkSnsxwUk-Sp" + }, + "execution_count": null, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "## Preprocess images\n", + "\n", + "Use `preprocess_image` to run the inference, read the image, and convert the image to a TensorFlow tensor." + ], + "metadata": { + "id": "tZH0r0sL-if5" + } + }, + { + "cell_type": "code", + "source": [ + "def preprocess_image(image_name, image_dir):\n", + " img = tf.keras.utils.get_file(image_name, image_dir + image_name)\n", + " img = Image.open(img).resize((224, 224))\n", + " img = numpy.array(img) / 255.0\n", + " img_tensor = tf.cast(tf.convert_to_tensor(img[...]), dtype=tf.float32)\n", + " return img_tensor" + ], + "metadata": { + "id": "dU5imgTt-8Ne" + }, + "execution_count": null, + "outputs": [] + }, + { + "cell_type": "code", + "source": [ + "class PostProcessor(beam.DoFn):\n", + " \"\"\"Process the PredictionResult to get the predicted label.\n", + " Returns predicted label.\n", + " \"\"\"\n", + " def process(self, element: PredictionResult) -> Iterable[Tuple[str, str]]:\n", + " predicted_class = numpy.argmax(element.inference, axis=-1)\n", + " labels_path = tf.keras.utils.get_file(\n", + " 'ImageNetLabels.txt',\n", + " 'https://storage.googleapis.com/download.tensorflow.org/data/ImageNetLabels.txt' # pylint: disable=line-too-long\n", + " )\n", + " imagenet_labels = numpy.array(open(labels_path).read().splitlines())\n", + " predicted_class_name = imagenet_labels[predicted_class]\n", + " yield predicted_class_name.title(), element.model_id" + ], + "metadata": { + "id": "6V5tJxO6-gyt" + }, + "execution_count": null, + "outputs": [] + }, + { + "cell_type": "code", + "source": [ + "# Define the pipeline object.\n", + "pipeline = beam.Pipeline(options=options)" + ], + "metadata": { + "id": "GpdKk72O_NXT" + }, + "execution_count": null, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "Next, review the pipeline steps and examine the code.\n", + "\n", + "### Pipeline steps\n" + ], + "metadata": { + "id": "elZ53uxc_9Hv" + } + }, + { + "cell_type": "markdown", + "source": [ + "1. Create a `PeriodicImpulse` transform, which emits output every `n` seconds. The `PeriodicImpulse` transform generates an infinite sequence of elements with a given runtime interval.\n", + "\n", + " In this example, `PeriodicImpulse` mimics the Pub/Sub source. Because the inputs in a streaming pipeline arrive in intervals, use `PeriodicImpulse` to output elements at `m` intervals.\n", + "To learn more about `PeriodicImpulse`, see the [`PeriodicImpulse` code](https://github.com/apache/beam/blob/9c52e0594d6f0e59cd17ee005acfb41da508e0d5/sdks/python/apache_beam/transforms/periodicsequence.py#L150)." + ], + "metadata": { + "id": "305tkV2sAD-S" + } + }, + { + "cell_type": "code", + "source": [ + "start_timestamp = time.time() # start timestamp of the periodic impulse\n", + "end_timestamp = start_timestamp + 60 * 20 # end timestamp of the periodic impulse (will run for 20 minutes).\n", + "main_input_fire_interval = 60 # interval in seconds at which the main input PCollection is emitted.\n", + "side_input_fire_interval = 60 # interval in seconds at which the side input PCollection is emitted.\n", + "\n", + "periodic_impulse = (\n", + " pipeline\n", + " | \"MainInputPcoll\" >> PeriodicImpulse(\n", + " start_timestamp=start_timestamp,\n", + " stop_timestamp=end_timestamp,\n", + " fire_interval=main_input_fire_interval))" + ], + "metadata": { + "id": "vUFStz66_Tbb" + }, + "execution_count": null, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "2. To read and preprocess the images, use the `preprocess_image` function. This example uses `Cat-with-beanie.jpg` for all inferences.\n", + "\n", + " **Note**: Image used for prediction is licensed in CC-BY. The creator is listed in the [LICENSE.txt](https://storage.googleapis.com/apache-beam-samples/image_captioning/LICENSE.txt) file." + ], + "metadata": { + "id": "8-sal2rFAxP2" + } + }, + { + "cell_type": "markdown", + "source": [ + "![download.png](data:image/png;base64,iVBORw0KGgoAAAANSUhEUgAAAOAAAADgCAIAAACVT/22AAAKMWlDQ1BJQ0MgUHJvZmlsZQAAeJydlndUU9kWh8+9N71QkhCKlNBraFICSA29SJEuKjEJEErAkAAiNkRUcERRkaYIMijggKNDkbEiioUBUbHrBBlE1HFwFBuWSWStGd+8ee/Nm98f935rn73P3Wfvfda6AJD8gwXCTFgJgAyhWBTh58WIjYtnYAcBDPAAA2wA4HCzs0IW+EYCmQJ82IxsmRP4F726DiD5+yrTP4zBAP+flLlZIjEAUJiM5/L42VwZF8k4PVecJbdPyZi2NE3OMErOIlmCMlaTc/IsW3z2mWUPOfMyhDwZy3PO4mXw5Nwn4405Er6MkWAZF+cI+LkyviZjg3RJhkDGb+SxGXxONgAoktwu5nNTZGwtY5IoMoIt43kA4EjJX/DSL1jMzxPLD8XOzFouEiSniBkmXFOGjZMTi+HPz03ni8XMMA43jSPiMdiZGVkc4XIAZs/8WRR5bRmyIjvYODk4MG0tbb4o1H9d/JuS93aWXoR/7hlEH/jD9ld+mQ0AsKZltdn6h21pFQBd6wFQu/2HzWAvAIqyvnUOfXEeunxeUsTiLGcrq9zcXEsBn2spL+jv+p8Of0NffM9Svt3v5WF485M4knQxQ143bmZ6pkTEyM7icPkM5p+H+B8H/nUeFhH8JL6IL5RFRMumTCBMlrVbyBOIBZlChkD4n5r4D8P+pNm5lona+BHQllgCpSEaQH4eACgqESAJe2Qr0O99C8ZHA/nNi9GZmJ37z4L+fVe4TP7IFiR/jmNHRDK4ElHO7Jr8WgI0IABFQAPqQBvoAxPABLbAEbgAD+ADAkEoiARxYDHgghSQAUQgFxSAtaAYlIKtYCeoBnWgETSDNnAYdIFj4DQ4By6By2AE3AFSMA6egCnwCsxAEISFyBAVUod0IEPIHLKFWJAb5AMFQxFQHJQIJUNCSAIVQOugUqgcqobqoWboW+godBq6AA1Dt6BRaBL6FXoHIzAJpsFasBFsBbNgTzgIjoQXwcnwMjgfLoK3wJVwA3wQ7oRPw5fgEVgKP4GnEYAQETqiizARFsJGQpF4JAkRIauQEqQCaUDakB6kH7mKSJGnyFsUBkVFMVBMlAvKHxWF4qKWoVahNqOqUQdQnag+1FXUKGoK9RFNRmuizdHO6AB0LDoZnYsuRlegm9Ad6LPoEfQ4+hUGg6FjjDGOGH9MHCYVswKzGbMb0445hRnGjGGmsVisOtYc64oNxXKwYmwxtgp7EHsSewU7jn2DI+J0cLY4X1w8TogrxFXgWnAncFdwE7gZvBLeEO+MD8Xz8MvxZfhGfA9+CD+OnyEoE4wJroRIQiphLaGS0EY4S7hLeEEkEvWITsRwooC4hlhJPEQ8TxwlviVRSGYkNimBJCFtIe0nnSLdIr0gk8lGZA9yPFlM3kJuJp8h3ye/UaAqWCoEKPAUVivUKHQqXFF4pohXNFT0VFysmK9YoXhEcUjxqRJeyUiJrcRRWqVUo3RU6YbStDJV2UY5VDlDebNyi/IF5UcULMWI4kPhUYoo+yhnKGNUhKpPZVO51HXURupZ6jgNQzOmBdBSaaW0b2iDtCkVioqdSrRKnkqNynEVKR2hG9ED6On0Mvph+nX6O1UtVU9Vvuom1TbVK6qv1eaoeajx1UrU2tVG1N6pM9R91NPUt6l3qd/TQGmYaYRr5Grs0Tir8XQObY7LHO6ckjmH59zWhDXNNCM0V2ju0xzQnNbS1vLTytKq0jqj9VSbru2hnaq9Q/uE9qQOVcdNR6CzQ+ekzmOGCsOTkc6oZPQxpnQ1df11Jbr1uoO6M3rGelF6hXrtevf0Cfos/ST9Hfq9+lMGOgYhBgUGrQa3DfGGLMMUw12G/YavjYyNYow2GHUZPTJWMw4wzjduNb5rQjZxN1lm0mByzRRjyjJNM91tetkMNrM3SzGrMRsyh80dzAXmu82HLdAWThZCiwaLG0wS05OZw2xljlrSLYMtCy27LJ9ZGVjFW22z6rf6aG1vnW7daH3HhmITaFNo02Pzq62ZLde2xvbaXPJc37mr53bPfW5nbse322N3055qH2K/wb7X/oODo4PIoc1h0tHAMdGx1vEGi8YKY21mnXdCO3k5rXY65vTW2cFZ7HzY+RcXpkuaS4vLo3nG8/jzGueNueq5clzrXaVuDLdEt71uUnddd457g/sDD30PnkeTx4SnqWeq50HPZ17WXiKvDq/XbGf2SvYpb8Tbz7vEe9CH4hPlU+1z31fPN9m31XfKz95vhd8pf7R/kP82/xsBWgHcgOaAqUDHwJWBfUGkoAVB1UEPgs2CRcE9IXBIYMj2kLvzDecL53eFgtCA0O2h98KMw5aFfR+OCQ8Lrwl/GGETURDRv4C6YMmClgWvIr0iyyLvRJlESaJ6oxWjE6Kbo1/HeMeUx0hjrWJXxl6K04gTxHXHY+Oj45vipxf6LNy5cDzBPqE44foi40V5iy4s1licvvj4EsUlnCVHEtGJMYktie85oZwGzvTSgKW1S6e4bO4u7hOeB28Hb5Lvyi/nTyS5JpUnPUp2Td6ePJninlKR8lTAFlQLnqf6p9alvk4LTduf9ik9Jr09A5eRmHFUSBGmCfsytTPzMoezzLOKs6TLnJftXDYlChI1ZUPZi7K7xTTZz9SAxESyXjKa45ZTk/MmNzr3SJ5ynjBvYLnZ8k3LJ/J9879egVrBXdFboFuwtmB0pefK+lXQqqWrelfrry5aPb7Gb82BtYS1aWt/KLQuLC98uS5mXU+RVtGaorH1futbixWKRcU3NrhsqNuI2ijYOLhp7qaqTR9LeCUXS61LK0rfb+ZuvviVzVeVX33akrRlsMyhbM9WzFbh1uvb3LcdKFcuzy8f2x6yvXMHY0fJjpc7l+y8UGFXUbeLsEuyS1oZXNldZVC1tep9dUr1SI1XTXutZu2m2te7ebuv7PHY01anVVda926vYO/Ner/6zgajhop9mH05+x42Rjf2f836urlJo6m06cN+4X7pgYgDfc2Ozc0tmi1lrXCrpHXyYMLBy994f9Pdxmyrb6e3lx4ChySHHn+b+O31w0GHe4+wjrR9Z/hdbQe1o6QT6lzeOdWV0iXtjusePhp4tLfHpafje8vv9x/TPVZzXOV42QnCiaITn07mn5w+lXXq6enk02O9S3rvnIk9c60vvG/wbNDZ8+d8z53p9+w/ed71/LELzheOXmRd7LrkcKlzwH6g4wf7HzoGHQY7hxyHui87Xe4Znjd84or7ldNXva+euxZw7dLI/JHh61HXb95IuCG9ybv56Fb6ree3c27P3FlzF3235J7SvYr7mvcbfjT9sV3qID0+6j068GDBgztj3LEnP2X/9H686CH5YcWEzkTzI9tHxyZ9Jy8/Xvh4/EnWk5mnxT8r/1z7zOTZd794/DIwFTs1/lz0/NOvm1+ov9j/0u5l73TY9P1XGa9mXpe8UX9z4C3rbf+7mHcTM7nvse8rP5h+6PkY9PHup4xPn34D94Tz+6TMXDkAAQAASURBVHichP3Xt2xJeh+IfV9EbJs+8/hzvalbt3x1tQe70WgSAAnQDClySGo4miXxbV70oLX0b0hr6UEPkkZrRmuRHJrhiAAJEoYwDaIb7dDl63pzzr3Hn/SZ20TEp4cwe+e5RSq7+p4020R88YvfZyM2/rN/9zsMERAAgAgAABEBAADcH/t39Ut/MCABIBIRgL0OIhIRIgIRIgAgubPMO3MUoX1P5kJE1V0QQWtg6C9KRBcaYP+PSEQEhHChnQSABAQEgIgIQIT2eEB3O+1awMwJ/nxC00CC//8vc1bVPgJ/Ipo2ABARs6Kyn22/AAgIEY1k3O3Inou+D/Y3K9gvawTCRVEQuS4wrLXKtsi33bZKmxPJDgfU5I5V27B2Cd8OTVZkHhdE2ouydiBBbSjt+1Upk/vB/BVGMmgHnGzLDODMsPqPtfFAspgjcPK4iGvzHpFIuzv4MXCDY2/nYe0HAAEI0c4D/6sTKVENjgTMSXlFHnYc0UCVABmiF3c1Ydz40MooVB+qMas65GaWux4gkiZEtOCwLQRw6ERE8211LS8tRNDmW6yBD8HInqpbmIbY8UIrPCsef6wfXyAgI3QAIIsgK2QkM9DaSd53zQqPwLcEEQE0EFY0U8nCDiAB1qQO1a0MTJlFnAcUMjKD4MZ/RS5oIWxELTwy0FEa1DBOfgo5jJKdI4YgbfcJ7YXRCsl1zw6UQ4oXMZHrC/l54W9UcWrVNqBao9HPInBU5KYu1eRMrrNYURzWoWbpybUKyE2SCql2kLz47Im0Ipn6qNUmT4XBSuLVqDg025G9OPqORGsaDP0x5KBG6IjTH2PbSVBrl1Vz4JkT3fRZuZvneys4cNIwI7cqHKxRVG1WVw1ZYU2wDcKKIiwFoMEnICPSgCunIyKrtbDSUQjAABkBA2CrOp4BMsd/tdlee2deBEjEEMlCzoPeAdad4KYa2RNrbbBvCMxVGCIAMEMClhgJALVlZXSzYHWeIVYD7fSv/89+rBG5GQD0TbCnIFZIc+A17SVws6zqnZWQ66vnJY8Nx4U1jWwJr5oDQNWguFPs0NQEVaETERkgq3Wypp+djqxdzmCsrhsBfAvAmW019UzGgvAz58I5Ncl7QToGcx3XRhV4kwYAPMUiYk3otoNsFT1VOwmtfMndiqG/rMcSrp7lB8sKjFzH6iDwl6gI6UuuUGu/w0DtG4sL5i5DBscAjKzSR6ia6AYVEIChRTw4c8LJDo2OxNW2WCPJcXBl7dSGhyGaMWJ2grguERhTyCLP/0Z+9Kz0EYDVeL4aXrTWQdVpOzxeSismqYGPvVmNxqoJWZuZUB+Xumirc+qnrwiGKmvCWlDVvfwkdV/WXBwPGnO+QZmG+ssSn0Ukq+upisIqAgGOyFYnWgUXrJ1iIeJAzBz+aq32H/3twLLIxQasYhrN6FUidqNnNaZrngZS3iCqAYgANGntNQTWpofnYSBk9tJ1TNSuY8zIWjdq4+UsGqiEazkU681GNzLOdKumnZkJiMzCnGEFBqjo0A+Bx5GXniNjWuFdcy5WcPFS8K/qSwcuzwf1Qb9gNKxcyp5BNYVPABqcMVqHZiXBmuKtdcga60YoosYOq0aVtfB8c1bHxBmK5sW8MYe2gXXLFZmf+mS42YKPVgyl+ssZtbY1K9Kxk9fbwn4QrMeqrZvlrSkvddMCc5gb1pp2JyIEBpW5y6xBaZFMTpOvkqzzCqp56uRLTom4A9C2k/nW1k6z56wwaG0M7W211hZUXvwV8/pjV9w4g1qnaMFRwoXxRCIC7ZtcXRj8vZz/QLXWVoEC8AyP9U9o5zwQMYQqMOImVTVBViaDa5aAyouvjCBy6s9JcGWWgcOhJ27vXWG9LW7swPlADJCM2UtQ98l8c6GmdiuIr0qp1iEfgbH2KzmWZYCOL11YgBzHOF+ypqexGjE3sM6tQCsEz9++PWYK4IVGec+ZauC2SHFNNW8RVnmoenl311JZxY6VqnGMwLzbsaKjKvb1DSATs7pgV6LDiGs2eLD6sSB/oO945dza+9QvB9oaOwBE2uk7BjbmgH5GW22+QtXoohOWHpnviTV2KvMIkIDXBsTadVS11MCXMXZBzI7SsS4JRLQxCERC0CuTpzY2F6/jfqkrW/uNu4kdQj+RUJtfGTorwnXLdhGsCrCGkJ916K4LjsTRqW/ndtSaWtOkNaPMDicyrNxLE2IxHffidFJG31Nng3ioOWPSXdtLYGUaW0l/Cda9MWrOXiHZVWiRNtY2klUg1S2qLoDzkwDAxZjQAcO1zzRF+wFBZIjcdquunA3EaLWNKz8iAoiqM/a+tckH4AXqNYUNctcutjKDa7EJ86WdIXZeQCVuhAtT35zAvNCdiC2B+fiEH+GaO+mJ34LVaQFnTlrNhZVycghw4q8UGkPSNrLt4vxewyAjy40rto8lW9MjYoZcrUK0POx64Qfcwh2p0qoWqz7wSXRhUNz9nD1qcWdMCh+/gy9LW1zsdKUhLZPVLuvGzjWz+saLxKgCN6CWC8kHRKsGG8TrSuujuxR5htJfpkdsX4RVjlXTyPN+TQtaIjF6AgCZ0eyvqoNX57E7wrET1rBQYyQHYAepSjDoEj+V2nOdrIte12TvXWcrympkVsOQWL+5HWhy0RwN2t3Qm6FARMZjW417VPepUbQzvmoWoe2XYStHMdWEdKoP/HxzXfSZOh/5r+59AbXGeKpM/BWXyd6uzvUrP7tb1O0qrExFGzdwTcVqNvmsIfPsVHXFslLdEvDviFZasNJUcEThCLHS7g4iNcohx4jM36PGczUX114daxqhjkj06SqyQR8A67QyP0iGh4x0GVKlwoyRt8oPNU0F4IOFYJzy2pFkjRQ3jk43oTPprMZyagGt+nQhN9+FmhTdl4DMRWqrcUArTJ9W8m11t8XVy6ATpj/fDS0iAOgVR57cXK6NN9p0gxuzGgM64gRwwS+y2qyKpNpZ6uaSmYy1/tbEYBR0TSPZ3lZQrOYB1c6n6gh8BZyI9T8AAKKmD9AzXO0M52QAAoB28LHIsdqkAqBpen2ae3qw/OS8GXsNp0BruRDbM9TE0Sejq9/QBSMd0Xg95Nvm3WWnRV2a2ArJd5+qP479UFfBkUoEULu0U/KV6oNVcw1rJwEAECGgrvl8tr8rg+0ZF6imZ8gRmLt3pcfrkwOq36nGpeD6i0RU5THtMNiuX3AELjLlBS1fM4tesQqs0q4JrlID5BSBgzV4lWHvYnNGK3RGRKI+XLUqA0AiYnaO+ytWt6GaCqvBxxmgNdQ7M7QSi0sokzsbYWXW2ps6m94lVw1xIlgHq67vap1y/1pjzBCzs+XAWn3kZvuqiLyh4gRIVir22uTi+37G2hdDIPIFE2TJsqYM0TceEcEk7ldVTjWTPStWVloNgx6vaO+8gpJVX95fuvrtohPv54YJGFXM61T4xUvR6qe6mbBid5jGEhEgA9KeJmoNsoKpCBdqdzd4Jc3sQZ55EEyG0waKzYT2YPNE467FAH1I2SPFyxSppmfRKkHjV+Mrwje2o+VEBLAgRjchABCIISESrBSIVPzmbk4ezXRhSNALcsULr4ubAMgOGNQNa2/DOt1c6acVVWeu4SI8K/UfAACga+G1uv706MRaY+oGmLt07Ve40IdX4yI1qay6VmiHsQ7ui69Xvof6ePnbXRQjuFFAQOZu6rujbatthQg5KazC3JkBorqgs3JYfSJQpW78Bew/lp5qsTZnXTPP32hiYisT0ZCiV1Xo/JVKC62wEyEAdwl3G5ZAJG2igACrI+rnIDilie6CF+jKcZj3Ro1N4NMxrr9OHVRcUaUdnHlDfgSweuubVOlAZziad1QdVJnsTiDOu6prrsp/8a6tlznW7lgHUDUPV5FVfeUUXV04vq/1l6eDCgnk4r71bjuh+p77QIUxbCofAPwQ2Pa493buM0ThjU4TSK9ZPwSANsRJxrolBrZMxjGGLZmstDhYlFteNQ3yjXDq3N4OkQi0uWzFIC5XAd79RjRBeT9dqSZa329wISrHkxV7XeRKsLzsZpqLuPtP1WiaD1UcwqGEHLH6SVYB1dl3KxNtldouxHesjGslIljLHrhv3Mx5hSYNXrXSF+9Sh1pNBAgVlNHJz7d2BdarMvOjAwCgq7Pc0XaukfMxLWM7Dc0Y0+TI04q7GlU7ALUbE4DQldVlhepiv15X2hGx3qizlS4YIzWHtKJPcl0Cr+UulJbWJ5q5lyM/p4Yr0DGHhopRfD/ANm/FyfNdcCzlm1u5q/U2+0m2ah3WvIXKlPDMVrXKIG7FFFiNDXqS9WlPp838B6w3x4p21TR0XhSsvrT+EnSuQtlpFSJH5EAARBoqVodXX+hqzcw4sKrOrvJg3XBVpzhDwt6W7DiAK1oCXAk8+SGtcTMAEYmqDxVMfdKwrsKdTVLZ4dY3NCZlBdlKy1paR6xoHWoq3BzPTFn7ilqtTRKn9SrFWnWHfMvgIl+B9wncEXXZVxMSHHVZOXptZOMHqwzsu2475+ewvzr6fA1WCsuOjFN4tUbAynS9ACnvwXhKqyPVf3+RUJ20Lxy8ckylD6zYvM1jBeJrAF2g0LIMVQDCWu02eYr2sqhg4Fy7ipaZayf6Y5xgrEIC1zXjqDvCcgMGKydWB6xOdKN0kBGgdm6MA7EZS+ZPNO4XVnks6xIBkC05XXGi7d1ddT06UTnh2pwhWfJ0QjJmADq9VXP+6yrWNMrXwmJFcUZq5q09x5kPXiN4saD/tzK1jOZf8f8qrJi7uuGqtK5v2avodBem+psa7mus409xDm/d+7FOur+i18v+zFq62rTvS1tVNchb+eiJAXFVesYFcpFU1/PVqxHU7Sn72c8KAGSrVrVRWlUSuebMgbeQ6vaMMSjc5LOpT+bTJOTQ4sRUHzQvAxehA49R9FekSuExqCpMnRFoNbKXOpqlHd4CuOBmOlOdKm/Dj6sbt1VAeCUF1vKs7uQMLNdARMYsN5ATXcUdVf2/u8KKIGtDgBVFrQ7NxdJPU88AlqFXzMqLp1eYISedSpWBo7bqfJfErqYWwAVD1pv7VHuZ08kreGfXkDXDmL+NWbTkmmjzKibMY7GPCEAMrRqtKQV7J6dZai3yxzicWHFr2yUbNTAFqEgucWBZqqYpVl6OwKA2+Rzuq3nsZ4G/O1q+9DyEANoPk9c1XjsjIFQrCMyksUq58nFq96zcoMq2Ii9Uj4aaiFYzPW4OuLmHYIN3hrMtj7tX1aj61VfE5DBR00O1H1d65tpbI3iXP3Etqt5ZavcGja7BvUbD5gQLxAopXmqreqreNKehicjHeKAKNiIAOb6qjR4BIIqa3lztK75C7oh1bq+pM4DKlkJXEvKK0VeLOjDjWtSq0+sHoHWqEQz0ybr5RizMpkur0fISJHdkHVTgRtTTrXtPzltGJw7bPquf7aT3g1ybLOCMbrJXMHRkqyJqp2rDWL4idpWqzBfeNSR/WSunFZRWCueC2eoU1MUTVoQPlnHcLPT5sIqGWDVHvYnpPYG6MKu2X3R5V039WsPJ1toCgY2Dej0HFVycTnLfoikWqfcWoKYyyY2aw79BgWuxuawfbj+UK/asuZsCDVb5gvcevI7Q9SF3bSOqFj35yFRlezhp+EY4uYM1RK1jUdnT5LtGADWZO6MC6sPjpoO3n4zxg+B8BqiKOhyuSVu4VVdHrLwmB0E3AyqsYjXwFt8rKc1KJhUOakPmD6op+BV68KKg2lkXbk1ElRzd5az/pPUFw7F2DPnTa9qmIkxwQ+ksnKorPgLgZr+TCQIAc0FFEK+6eBeYyePN3xMdD1XyWJ3TlaJ2ROeNB3S6vFrJ6kspbHIfyCwt8oyLzkqoId9cnrmWubkP1veqW07Wd6rG12YTmF/uUo2UBQdU2DEKyPbb9qVmjFVThhBqJdhkKrOp0hpA6GJS6JgCnbnm4bhSmuS5ocaaVB/kVbCuGp2VpF6Fuzf0/U/oxnDFKnCX9YFYkx/xP5gBrYjDOXHeHvBWTsUP9eSGE40DazWVrTpCFL65vkHsFbyaUarPXy8uN1fAfnthqiFAtc7QDr5vnGkOA7xwRs1yBfCAqheNmFt7oiIA5iI/tUOs9C1sa2e5X1eaCbU2vaJnKwHUPtcmEVVXIQACZMwMle1Q7farI+Mv6zFa1XTWKjw9bqi+DgEv5NBXm10P5l0k44u2WwWymmtfb6qbpWiVD67Axk1rL4NV2brEksGcFxiBcwhXNYGLJiMC1JwG31Xj6fjjV9EKQKSdarYH2JynP74mQavk/Ky3NYJ+7nvmswxVK5jytrR2PeeA9WXQxqm22fkKzmi9AQSnVmoRUqrE6iRCta5UMnIcDPWWGvlWysoitBZPcCxoLobOCwNEs1uBu5rTM26kV6jXt6Cuyv0QXtDvK15szZFftTL9l5Ukav6WVXHkvNzapPPjWHtfN0Jr4jMjiL7pVawJHAwNFIicU26vsDJ5bBfd5HJLPjzmvKVQV+JERKSNHL1KcwNPNYvrQm+shvdHEgGCttFL5x/VMOqrRTx32gahNxaMFFxAEaz/zQCYn97GIHIzAW0b3ayo1I2b/dWV0V2V3Kx3CqNyLiutbgYXXQS2ukI1qH40fUiaavfy08DwsSdCN8ArPOSIFi6+XtXydVr1JoTx2KpBraz5utKujeBFVK5onrpercbFCMWYs1Zr4kW95ia87aphHkCwa/otnfkyBwtQ5u21SklYFq1nUXxrPIUS2dhKzdbyU7Am31pNgI8Je+WOZOHlpoanIDQ/2Wag5S9mDE1HUWbiWkI17VxR9DVtgIZQWPWxzlXuSGcuufCi740RKlVaE2sNtgNjruLfO4cLyIT00NGM722N3V/hEg9716RVZf2K4qrmHmKVz3SvV/Vv/XVB718wlpxfRX5kXm2qlfwFB9S/BRsTcL5TnczNZVfsX/NXGA9BW4fXmHLerDbmgzGlK/PIX1o7oKK9v+dtu4DTfVztSN31czLWCGgKNwlWnA2oT/AKwU412C5hRf0rwnNmqFuLvKLWqU4GVFWSr2b/rPdDNW3kysN8gsHalq4S1BqxHtG2Zdqrb9+1uklQp6gap10A1Kp9UiM2x5SWuhyW6p7QhVOsUn6lDLQaKYDqQr499RGpCbyaX37+umnvRw9sl8n10hhj5Gax1zaeGZFZ3e0miOk9EGiqxo/ckLr8kLkc+WSqKVGprDqvx4jcR2N42LbXZOwNn8qgIbfiH/xccxkG1xgCIiSbx3fGluPv2pSw3GML+Wvz8yJZ+Ex0pTRqNoDlL0f4Fz1ia0qYA1cBRH4CVzrolevV/oPK0LDTG9D99GUp9S9lshqPejXoJeKPctPnFXSuaH+yd0H4khs5RHmrsmq/+b7SMx4b1jAwhqEG0tVMAvSmEgEY9S5s+xEAkEgTugSvAbC5AQJpS3Lmsyt7RABysaGqEAkA0OYjagrS6keobghAzLhsrj++BMMfi64prjyOnEfFfDoObHfdncmbMk6MjhGhkik4drkwPABQHVYzb8zNwXrXNZokr7PsjHVhaAAy2zq69qObnA43WFV4+lG3zhNhrba/dhBRbbLVSGTFE6pfz+uFeocuzlCopFGnbK+VfLXkyglOvFgNvVNZda/DKQy7N1g1xGjULOnaLCY3ZPYvc6rLmHBAREgayVl0hibde/dRW7+HnIOEgKTRagyjzmrDa/4lBAIN5O/nceHJ1azl8GLxzfSdNE10BqsLzlnxomsfeHQ6nxvQmL61YanQuUoOjmepJi/bWu3bZTdQQ388YlV7X4WpEG0tgTEVnJIy7XQ2ghWDd7NoFVXkzsRV5q4gBMAY88C6cMyXMN9/5oXO6fTvjWK4cIBv8IV/fQO8T+J0JPiou5uermukLXVZmK6YdkREpAVWBKZtNNNaPnbDLbIkWs2sFceWkECb5cg2SoeWVhj5CDgikptW6NzeWtfQB6zrc86ebaqkzSw2R0NNIVZIBANp9PUdXiSGcR3fO9FDNWntHIOV+zp2cFUEPqFZ1xYmt4AVKVJVlI/1YcKq2Ssr5qqoUoUrK8RatmPldZHsoTZGr77oYkLyP/eq3F9HmXVevHCwvxdjTGtdk+bqbLcdqc6xGs9OWfM1AGlwYncAsjAQlXNX67PxGMjxW4267a8X+1YVR5oiOiRwaWhzTbKmkRGXsynAarAqfl9lNbyPr62B6MwKBzg3uwlMagxsUt21xR5B9urmpu52nupq4rR2e/UZ7PwGBFczapvukr727qYozDIk+Ru7s+uRf2b50Q6CBza4eCrYyeQA7eysVzyY2rh4WNCrduorqL3gZlUm1n/B5kFXwfAKWFeuU2uDwV8Feqd6nEjrtsYFPNlOG+AJfw83HOj64F3WlXaQb65T7uDu5t9aB5YIEN1ax0pYRITuf4AE2hp3hMhspM5ONa+pKx/IrSQhNzENrZmW+CVK2ql6oz7s8uhXyn5fHbkaFV3IirkRIjdCteHzDgB4ovUwtarK31Q7I9NJzImOagXOXl0a/sba5ITqVuD7Uj/Rf3Q2d2XJOBOsxotVrOG/+NKuDTUQ/5csCqrnzMjSkLZCtA1FdNuEW1E57nOWEBARCdAK3C5CRiGx6sp2fS05Brbt0y6uY+To4gz1lqJFp2tnFTfWnoXRF/LYZpvF42jNGPSDh4jgksCaKik4/8iBF4zjh8Dq0TcAQDT7770KTXJtqA+w7UG1pIGYLaomyyNQG1lcGY2aLVSjXqgUIDk8e15lUN3JN9j6XNWYr4DDHWbbUNUVOIvK//uqx4NV3Z1VIl/Cu5WAnMfpf/dBPa+OV+VpNZx1sqGqqGXMCrhCtuU0gx+HFjczAQBA6No5XozV8BIoZ1hUVGr1lzOnzEy2MKvW/1SD5jDqsOgCjE58bjaTE1dFUYioSXsM2oll/Vj0Q2VdJOdmVz2og6PqWPXOXsRbVDbTa4KmDvHAKhl5bK4UZa7MTfd/x6P1e7pgRk0uHjkeOm4CuJiMNVIc7KjWIW+AuS9r7rMRC658dBi6CKxqvC98U+t3bfRda10Gvn61Sjms/CE/92qsbXFRc3D8SURWxZOu8YJrvXFINKHPX/kZX8+21e1oo6S8biS7gJ385gLoM+4Ort62cSqwYiKn1ezmu5os3drHcph4PjMhUtMd26zKEndAMZKoPlWK6aJ+9EDxuoicejJjagREhHa5N9Z53k4MN5o1xnE/fhkJvEpC1cvVfLmTatErj3U3aq6/7nDvaNYogJxX4MDiW23TMPVz6yLyUvpyI9VpMts28AY6WIm4G1uGAoDarsrmFj4c5wNB5nvh5oibr34uVqzjw2PGHjJCQQDS2ulY1zr71k0yMl5eFTsyAQVA96ZqLAFjCGAel2FxAGBdHgsVUmaQ7MukZhCBMQC77WhN7lWays/4C+7VhbEEWEVK3TS3Ss4vgXCArcw7gBUGdy4Ueu6vq1fn29aYg1ysEd1MA3D7/FTfVA2tq2T0zqXjBt/n+sUJPCtYE2+l81hxR/1enjRIVxu0ebl5G4msR1gz0y9afpbinAI26t+FDKsIFPnpBHbZcZ26jNTMKDKs5GAeplTjWqiAau0OvyDVwMrcQCmjmD2JuPwokSbyMkEAZfvpLoHM7UNid+Wyjq/W5lvGGCIjAoaGMO3mPtqqZGdqVO5dXffXMPHlH6l2rMUSVOJmfjZ4jqX6mVYtMqpRu6Nc9EJ0NoNtqnc78BWjfqVtVq1Wprj/FT0OXqFS27Ia8znlAwSEjDkt5OC7cpIfb3BNttxYCbbePHKNdicanVyFhowEELVNEFfnVHdEBCIBXBipIGOOqJzRgd70NjF0Aqv+DVdbY8BckYiYk5fjXSs9czuTDTdJ/yrLhCC1BkclWmvrWiETnHPBOWOCMc4YcqZBo1ZGFsgYETFGjHHQhNzOZmYqQ1dTl25ymIhgDXernsGFjysc7DjLO8jVWebAGlW6UXLvHI5th91Y+AHxs8gw6KujXqnyVcX5qkJ3N1nRCeiH9EsDogY32kUi0ZGoOx1qgK5+81h13Fy7v7ODoZo5rArKVcrd2Xg1S916jE7zAAhVKjsa9kwCk4i3ZgGRX7pOdrJ6Oge/vgpA24y+9X+UUjVvirS2q9nMUVpr86/WWmmSWhtLxQCUMRaGYRAEURhGQRAFQSg4h4BAkyzQApRzLoiIkJGt5ajK1bwAqllZCdjHny6Mk51etVH+Eu3nTqupafexUqsVBH1iE3zLLiDEfFXxtgdkjZc9MtDbEDUurJuMF3SCw0s9gfwlx1RMVwnrS14XVD84f+kC6D0xO4vDuNB+LrHaQ+h07ZJGnuQmhZkFKObzBQGYImQyiXUiItCkwdWAGvCBVdlau9JQTaS1VlprAiBSWkmlEUBrJZVWpEmT1koRaa2kUlIpg0JFpElLJUmR0koRKSIA0lJqgEacdJutfrfbbjTbjYRRzCAEhgSaVElaA2OccwLiEIBWZtg0EUPGnJTr3i6Ck5gdBled5ATqe+fH78IoO2u3JkkEb+pRzWq3I1LdF83WHZ7DLcOt6DSXU6lFcNwlPNjJq9SaQqysuaovuFJtWPObLvo9dSL/z70ukHfVPKfE607eRSuFmN8qjaxtYztqL0FeKCst9d0nIHF4eqqJSJPSSlkgWn6TSmvQWpOSysAINUklldYladBaAWmtldIGjUprTZq0UkoTkDKRLQKlNQNCBK0tzxkAmC3kydpwIKUsZamknAZhmWUcMQ4CGQQ60GQmBihd5KA1IKcwAGCIjBEn0pqYzUe7YUXPGZXzawMBUE/2XBiPCpjODiQCt9ae6JURssis2JQcbNxlPOGhHQJ7nDXnoX5B0ySsobbGlNW4OUPMVxf4eB9Ahc7K79b/GSS6pNqXqH1ErbU1TAl8s2n1YzVZX+FRqBSYUwhA7umL5KXh7KKaTJ0yNH6c+IvPPlOktdbSlN0ppbQyjCKJFGnUWluKNY9DJQJQiAjELVUh5wztw7IY5wwJOOchIuM8YAw5RwDBuODmxRAZ40wwzu3WJoyIpCxny2yZLTVQGMWIjDSVSinTJIZaS1mWWkrOOJBGAsZQK8aY2ye6St/UctneC4aawCqIVGWaflCdYe1iG17KLvtSw2ltABwTOEh66kBnETjpV1WjFg2OiN2Yr7QTwJWr1p2h2hnWliB3NUMzVvXri0bzRRhdSLsbO9Gk170cTHPqhilU0Ux78cr4uKCAPMOS9dWNsQgMzVbylqG1k8bKfBMvjl8CczAB5IiaMYYoAELGGBMUMMZQMM4Y45wFXDDOGWLAOGPczFHOOeOcMyY4t941Y/ZXxgLOOGMaUDDGjW/ODHsyhmD8dNJUynKZ57PFYplnSymBMQVEQEprpTSg1qSkLFVRaMaACDnnSmhOmohpAm5IBd3WobWcvNfm3vyq2z6vDJj7vsaL5ELCZljoyx2O2mesX9iF78AbBubZZORrWQzIqhv40h034RBXblG7tYWUb3HNVrnwpmraqnnjZ9wFB3/1xEp3V5ijCtNVBM3WuGF1M3Q6xFbPMXc3++1FjVO7r7hz5ZrhM4aMIWOccWSMc8ZQMAbIiSFnDJEJhgyZ4Cg4BxcEMgJx33BEYAZ8BnsI6DeEcfEiRHNBZIiu3Aq01qWUQZgJEbClKOczBQSaSGkg0kohggZSUpZloRkHRC4DCjRpTVoR48Ye0mi8Om8K2j92eBkD68lZ4dVs1FcGcXVcycMCCejL3eGaWYn+LuDcT7COl0v2VosPnS9sdzJatRTrd8cauMz3nv/cBHIHuxZdPGPloyM/589e6FCVWa31EWyEa0UCDmtE3sjyxIsmaoPIjD4yc89mIaqQ5Uoc37dU3L5+gwA5IgCabClniGiZzizfMTFH5vddAIM931rkzGysY5G5kj90tQ5+3yKDTdd4sIar1sBYqSQXQgguEECTIq1ISyk554wxjVqRKqXUqBFRBaVSkmsbuTKz1xqZlUXoYt0AAOZhzL6itmZMeRtA/2ez0s7YW40xoR9cIlctZe4DaJW7T4IQ82ZFzfkwDXPjqX1hg8U6OUlWBkTVYIsEK9tawyqGBHv9L+P7evyoupG/qZ9Rq1rF0179ItafN5dy85Tqixc1gLaocUOENhpei4DU5oIhXjHoDmx00LAdmKQ+GvZjCIzZ+mDmaceZHxo0kOFK8M5P3QV2g+aF5QrJKj0LyDiBZgDEmOCcM64ANAERlaXUoTIxKamUZqQJlFIEWnCujXlKittZa29V1Wq6+2o7kljjpdVaAagNTDX0F1fzAFj146jXkl39flj3GzQhQxeItvnZuifrTMlajKC6mbtPjVNXScvGfSpqdwL31ii5ca3FLC4SKq7gr24ZrDRnxU4186xmkddVv+diJ0DzzpxONV7zt7GGgTvTm14EACJNG+TN3hW7yTGmITxrL5uLaCMTRjasiojA7EN/q+fOVdMCAVETWE/bJZycyQyaMaUUU8r6UMA0oo1REZlgFjA00VklpQaUQkitAyLQJsRkfR2sZaidbVPPeRos262ia4NtRV8HQT1iYgiSwC/EsMTnj6xsxBqNoRegCyMgoDeOLbNa/qxOdjTvFLSBGtR64gas1pTarb19XKnvFV6qRhhr+W+HVLTWkY9ouQmwMgeqnK6bRN6aMrVvnuEr46DiAMeZq1wMdevLhxcED5gpF3GGA6GjT6uyjUK0T1a1IvZPtNBam0chIkNknBt2dKG4+hYPhIxbBwa9dDSR0hq15pxLw9wAJnVEJJXDCJjnfJLW1h5FZUxPE2PV5GRBBLbFQCuKnhm9gs4Q+LJXHWc+vIc1vYHgRg9tssN71nWCcQOBhCiJSqXnpSIgqfSoKENr7iNn0A1EzFkoOBg5eHuUIfjgqysaomowTU7O+kW+M/WQ5EVQojdzbWFEld+yqtweq0kjOi+whuVagMny3MrtKiaqGyy2yhGs0jWttRh0ee4LM6ZSTGYmCIYINoJIYPdpRLNFqDMWV0pdjOCYQx4xxhCJMWRMIPpKOMM4HBxKLHAMl7jhRBeCI1JEzMSLrIvFCMHYpkor52QgIhKCVlqZ8KvWSilhph6ZJ0BY9XNhlBxp2VX2tW8qfq0FvK0qv2DZ+fY6FeDlXrsZESCWAOeZPCvkSVZIpV8ucwYwLItny+z1VmstDJ4tFuOs3EjChMHlNHq73xrEETP13eiKtrxt6ywQbxxo10nH5Gj+rOhlwzfgZ5EjtJVwmGu6m5+e9cnpzBqyqPK4qxiq41Gyyr5S3ZWfamdD9Wgh8MWu9taVavelw0RAJAQPHMd7i4V57kRE6zbUuJe7xI+ZWWicJEQAE2JyE4wUQ9SARJoZHBO5XZerdhs4Kq2ZQSgiMiYELxhqK3NGWiulgKEpvzD5KSWVVgrIZk7N0xlNAQqSsTeo7tPolUyF92+8PVl5AKb+1wxUXdlVgvfoQKdSwT5kWxOM8vJgoX4xXe7n5Wtp/GJZ/tHB2d+8vMMAn40mCpdf6YZnRfHT4fCG7r3RiP74ZPiz02ES8K+vde902pHggd2f0sKKKhihtTOqSWOa7cDg2mg1qOuy01dud0jfo5ozvmKlIK5eCirU1V5klXJl9PgfrDFji9uwbhzXo1D+3qyGh4pEAYTZU9Bk261zZOjT23FY2UJeZbKqQtR776ZPtrWmDWRntxWDcTYrNVrzlxmiAkCzAYrRAwRaa6k12YpVICDiDBC1Mll8pU0+n+z/TaQLyC8M8ZrL7Thfk2yd+KoPq06r0QMXvqyMeDMBEM6yMlMkNQHC0bJ4upTExJ+P57OiuJWmz5fFrVYzFfywoGku201xuZFKUqdK306SqVInhA0hNNGnw/lZrkNEjriRRruNOBGstuUuOKfMzpyVgIVtk1epNbeGISIoRUopzrkpaLzocrkhAa8bV3/03zqL3JUXrMz66joGZGgnjtM4hndr6HQ2qzuiNmbmhsLoOO2SMM7kQl+fx40NxGqVLAxtOIQ8PNEiEam2PR6iCSK4XtSMPMttruFgzWAExhhDBiYuD9oSu5tohps1aa2Mc6Q1aCCNNZbxD8y02sr/BRfPcErJDzm4GpsVNPgxc+LzysuVYEMm6dks35tn+8v8+TK/mcYa8EVRPsqKrSD42qB7fzrth0E/bq7F0X62vNPpXG00/uRsuFB0NW48Xi4Zsq93B52Ac9BPZvMfHJy/2UhezmdLpW42G391e3Cr2+LMrMujWqOdJ1U1zprH9aYSQFEWk8n45Ox0Op0mSYLILu9eardadeuxQsQF9e2MGPQ/ei70hFdX4vaCPodXv6iuyc+TKTqTtzr9glkmLNUwZ1YhmuyQUwJ+dMERKJrV6wx8N4CIbLmxScjVXDbmGlRvMtSnSQVy5meI+wxSKqmUIs21tqkBBBOd12TqWWoPEAULP+fLO6+0GkbyITHw2sTGSiuLvVLfgC6GVJ2liealmhZqoek4108X+cez+b3Dk6+s9UeKSlWOpD6dTIjzb3aai4D/fDonhC+U/MHx+Vf63QDhZZZ1gvDr/cFhtvjXe/tNxt/ptglxK0kvp8nxYnmz2eQIZ1n+o9PJ80JLwMtJeKMRBojCZE2AGDIC5986Kw2cp6K0yuez8dnRdDKcj89OTo7LUgVJIhiWs9Mbd95tNdvWG6gpblzVHjXYVuq/ApeNDVUAtiIj8FEj8H6pw/HKZKjF/ZzkLZz8FBAm2ehs4Qo3zHozoLX1ujz5+1VfjKFG4D6rROgvYAK0WgMwby7bhyvWnWXTDJsaJRKMG4s2YJyAFJk6FucEASEyYIwIbBWVUkbFO1CRBmKakNlyXnebCl7WiDRSXrHNV1Q3+GnphpAQl1KNc3mwKM6LcizpSV40uXhRSAHwm1cvZYg/G082wnCsqRFFqeDHUj8r5PU03Y7CoywPpLqWJNea6UFRTkspEa63Wv/w+lVQaiELjTjRuBbHaUSfnJ0d5vKtXvtUqqcno4NSDkT4rU6cCNZLEs44J1VqnQq2mYTdgHNnr2uCIs/zxXg2PJ6Mz8siAxbEcXrl6g2ttdIglUSZjw+fNa7fFSKoOMKPBlgoYW0hHq0ypEEsWTbSK9cgiz2H8HqMz4q5fpzHpfeuagreFCw7T8H+4w0rj24XtvOmmLml1lozEzdyKqHiWyJyQVoX6vMa6cKLuTCK/cgYmBQooiyVUorsniXGLjXsTsYGVUq5v1KEIeeccVZdx6ooqpQ7IFAtC14Tra1Xssay5RJNOJf6PC+Hhcylmin9YpFnSsecPVpkzxbLd9qtsZR3koRINxnrBPyoLG7G8d1BNyf6s9PhdhJfTZMux+Ns+cs76w2OPz073w6CS1Hw+Xg0kvp2q7WdJjpb/M9P9jbSxq2WjgVrJqkupy8X+bv9LmbZ3Wb6YDJ7kKnNNJAlU6DH2fLPjk97Ufhmu7mbhJtJ0Ap4isCWk3xyfHz4YjqbIxNRnMg8V1KJMMrzXBPkpQwD0dVKFpkBKFWYpAofiLWUhFd4VSmfobP6BIcqfIL2gZzWACCP75XsSC18BhVtgz/AkKbQCOgD12h4DsF6sgZAFdTJcgmgjwZXNmulBtBW56DddsLYqG7WVOkytLF3v1WYMRy5iyAoIKW1sVBtbIvZ+aC1zmVJi3khNVssgY04E4ssWywzpZSWstlu7Wxt97vddqtFYAIz5KkTELkLJeZKj7JyWsqlVO0wGJdKk95IotNc/fTkfJiXW40kZmwk1X5e5poEw6KUSymjMPx0PDs6PV+7uvtut3W0XA4Q16MgQfpkPOmF0XYcxYi/82Tvdr+znSbdMPxoMmkwvp4mcyn3hmNVyLjbTYVYRsnNgD85PN6N49NMjYpiLU4I8c9Hs0BTP2ncHgx++/GzQZz+rZtX8rL4i+mCR/GlNPro+OQjxot8eVvQr241OpBPxpOSRKc7KJcLqSQRLPNseHiU5wVyzrhotdt5USpdZ74Vf4nsGFUj4o+oRxRW8+ZW9TtF5wvoTa2f110VF1vDb8Utq13QfS/cvr8e4PalzdIf+zUCaO0XalS8XNnYzPkSFT2hjUFgbZcRtmIQg6kX8QEG43wwxjmwqu5Fa9sYIgSbONCkkUgr/dkXnz978kwpmeXlcpmVZQmAUqoSIGk01zc2vv7BB++89dZg0I9EoAlM6RYBHGby49FiUcqCKNT6YLmcSxkxcaR0qakjxHFe7u29vLo5SJLk0TJ7NptvxmEvDDnnL2QZhdFQqs2Qp+s9xvl/Op+cTGedZvP9XvuHwzHXcq4UAd+bTlRRrsXJN9b696ezo2V+pdFgjA2i8Eaa/Nns7Acn56+1Gg/my93B+neuXCatn0xnqMVGI2VIf/Ly6JsbG3d77YSx0e7Wz84nT+fLUqujXA7K5R//xV8U52fddvKd3f6vv7aN+WiumAiTBggpS8nEYrGYTmdZXsyzPF8uAUBpKKTsdTobeU5Nb15axemw4j0IN96v6D6bgrH8WLcNLQq9f+KDfQiepGq4ruH1IjqJEEB4uxacpqOac2R6YJxDi1MA+8B2RN8Td4qzPQHcNPLTjlbSW1iFwBnachPS2t4XgTPGEKV57hECEGmluOAAwETAkJli3TSKv/bee99+/71QYLZcaFmGYaiByrw4Pj394uHTn33x5J88evwf/uAPdi/vrvX7nW43iRMWp0eK/8nZ/GQpsdNm3c41rXdb6SBOzsvys8kEcsmD6Ea78e3XbxZaP1vKe0/2W3H8lUu7PxsOl3lZar0s85SxD7bXP5tM9ubLsdK/sr15UpY/n8wWSv/1zY1mwP+XJ/s7ofilqzdPsuLH55OlUkEYbUfBh+ej02X+br+FAf/sxcH6jau/tt7544PTf3F4+pev7WrGQ17+dDj8eq97o9V4vszeKPX96fgP7+3dubqNoH/2+PFkPNHD03gyvMKKv37r8ru3rz1/8UJK2eoMgjAOWcCV0oBFUY6n89FkEogAkGnSSqv5bCGVRO72jXMPr6+zoBkIH4r34ZaqsMIhEcGFgVfY0PmfgATKfOswfbFsqrJXa34/uYCa8EBHRKNAnaGADswO38ZZcfmsGt5NyNcarc7WrNL/jke9rWubYeLhPh1qVAZHxl31HwEoTcr4Ssb+AEacE0NyMVBEFjbbrThWOJxOJxzCVqfNGe+t7bz/3ge/cvjyP/zHP/mLB3ufjsZRFERJHAjOkJEQDalDScsghptv7DX6nz8tO+uD1zd7b7Waw0Qj4wutn07mo6xAwTe31iLGPz4ZZlINWo290SgbT1+/ujspJSPsR2JbiJOiGEupFf2d7bV7kzkCAuc3eu1lkf/7oxOYL5uE37117c1ee/Li8NNHz9/ovnW91VwsM6XVsNC3e53z5bIdCEySHKh4cTxpNG/2eg8nsz8+OT04n6go+s7Wxh/d++Lw44+bctmnrFku/8Gvfvfa7ubRyZkIEuRqMpk2GzoIQy7C9a1eb1DOFvnp2VmZZY1GEnFEwlmWl2UJvmbF/TFFyh6aKwxnQ0s1a69W8kI2a+n50cfIvQvv4vn2GBeKX3UD3I0qdJF/ErZN4tjISh3f4AKUAJ7MwNsnKzwKjvap7sVVN3fUT7WPAE7BGA1vU53IGHKGDEmTVtrsUmviSSYWZbP4WmtSSslCU5o2tNbHpyej4ZCIAPlsWfbXtv63/5u/8Xe+/0EvjXSpymVeFlKWCpVKApZyakzOwp/8UfqLHzQoG8/mP/3wi5PzaYzseLGcKV0w1mmlJVDAg1Gp749nu61mNwzm86zf6dxstRZSf3Z8dr4selH8cDh9r9O+0WpMNTyYzv7wky+y83FPhM20MRCiuP/gW1uD1zvNo6zsJMm3blxZSyJkvMgKPZoQgx8dnrJF/vx0xBgfxLGIo8eTGUh9cD7++fODnX7n8lrrf713//H9BxvL0eXl6QZkf/d7X799/dLZcEzIC6lyqaXWIk7iRrvV7THGo7R56dKltUFfkZZlQUBKKhHwOEnSpGkItD5YdVDWIy1+lGsaGS06/UhaU8GHP8wPzluqxanI+TIWMzVAmFONKWlDjT7s6ljScJsxSohcqY25LlttguvgCuzq8wEATMUeVZEsHzNykclVZWGSSQatZqWUTxeZ85BzYEiuyomIiLQGCOLm1sZW2mwWZTkdj4UIGs02D2ISjV/61rf/q19+v9+OC6nyLFdK5YssL0ogEEEQhEEyPmn/7E/ig6da8P3ZfCHlWhTebSRrUYCE+eHx8Wiy00rbafysKJ8u8tuXtjd6ncfzvET+j+7e2mq3fvj0YLfRuN5uX0+Te8PJ+eno9uXd/8PX3w04+3S6yB8+SvNlnjYbYfB4vvizFydXO+1C6d1mM8iyX+y9RGDv9zvnWfbnL4+HpfqT/WPJgv3R7F/9xedBFP/GzvpXOw18+fzwP/77wd5nV2LVbkZ/6f23X7t5czKZZ3mZlTJJG51Od2NrhwXhIl8OpxOplSyy5XIZxvGg12EcNcE8K0ql02abB6FFnTPT6tyJF1iqBlX71423c3TAsaP5T5N9KqXVzn6sL+LjwsVdXZVDPQiD+9r/apaEK10jUpaZHeTRwdZAyVycwFbemeXv6PS7bZq9FgBW6+YAwCxPNv4+AXCzWIQxk3GVSiljJBCgJptFsLaBJtKgyGa9GCadQbcoRsPzuNWJWx3ORZnnRCgl3X3znUYj/eGH9754fjJfLAVjXKowChWRJCBkLFt2P/qhvvra6Pqbe6XaXOs90jCV6mw4ub6zs9lMZ0WxjthP4stpdH883R/PfvXapWEhrzTSPzs+H8Qhj6OhlM0o/Pzo5FYzvbOzmRBcbiTHUuk0vvz+ezHC6WKpBH+a56+V8nQyHc6z9NKV0enZv77/bDMJi15/VJb//smLv3p1a7GYH7caj6dxJ4nf3N0a51mM1B4eru0MGo1oPYq/8vbbZ+enWlEUJ8V8UZRqs7++WCzCMMnL5Xw6ZWQWIOSqyIi04EJrUlqv9bob69uMcY8LeqWi2dNqhVTnTtiojVvg4egTKgL0lmC1SLFyVKy3tMpnHrXW4fLeC5FA5ySZiCZaRV7FjKByyVfAjz5XSWSr5SurwrXbTSi0hgn5yaSJfATU+PLovHizqQgjsLuEABn+RCBuwlCMEUMCWwSq7R6inIfRzuUba+vbwBgQlHmGZu8dFhBrXL319s7ulSdPHv7gF/efHg6JSAOVpJVSoDUxZEyw5w/ak9Hk6t3DNKFlWU7nDOnF2SibzDe2N2Yo1gi+GE6vd9v3xvOzvHw+nl5Pwm6aTjQ0EQ/mixKI0uTxIlsMJ3/88Nn1tf5RUc46a41AnGbLH7w4FEm8EQanZdlJ40+OT5iOwjR9Y2Pweqf1cDT8yXiaPTscb/S/c2nn/mgssvmTZ8//jFQrTXZvvrb4eLPTFDHAW6/dnExGRVlGcTNMmlu7V4MgQsYG61ukdZ4tVWegSZ+dHIVhNBgMnj2dSSlLSbnUrUYzbTTBaS1HiBVisF5O6iHr/1S5X/LHw0pxI9aQYpx3unjNV10lspVKzgy1VxZmkZnZVttSofV2NBGzsSJvWyKYQiY7Xep+FIDzlMB13J7J7PNAEMyWoOjMl5olXrMJqgirJgIyRSGkAbjLNzKw6U4FpJQ0G0Bo0JwxHoSNIFCl1EpiEAAAaaW0lnEhyzII0zff7W3vXPr43sM//cX9WVYKzoHZjf+AQDEmhsfd+bhYH3Ru3u5t9I6VHp+NFkl4mhdX0+gP9g+CIDgATNP0pCieFvn/5+mLnAevt9KdJJqXRab18nR4ab2fav2VW5f/zcf3ck3fv3ntvbXeVhp3AvE7v/tHrV6/tTb41lpv/+x8/9GTN7a33u20fvL85aX1XqIndH58NtqEnfX1JAl2d+/tHd7qdw7m8x+fnq+laSuCdhRtrq89e/6Mi2iwllKWvZwvNrZ20iRhnCMXiFiWeSnLOE0n08lskbMglEVRSinCYPfq1bTRrFtZjlIqrBBUQEQHniqcVOGsFj4iAl+D5QHnrghYjXi9ntmPexW0XHXRBAOosoA26WPylszkOc0P2mhZ53FXTOlor7KCtc3jOyOaTOBCE3AG9ikNjkqNLLwv7xIRIJAxxjQAkS5Lu+rDGMIMGXAOQGSynVqZzLwpSWYMgyAMw6gsC600ApRlQUoiY4wzpbFUvNW/9Je+uXHtypXf+dOfPD04C8JQSqlLCWADEIKx2dFxfOX6r1zZ/pOjs5dPnl+6vPsP7t4aBLwTit/9wY+//s2v7AXil9d6XY6//Xs/uHn71q/evLSZROfLbKzkj5I444It80QE5csDJnV+5fLpMr/daYYigDC8tbN5rRn/7pP9NIoHzVQge3e992I2/539I5EtMUlUWeZET6bzF1mOefbpwcmdy9v9gxdqMh0TXt3aIeRKY7/bHQzWCHkUJ0naCIKAALLlYjI+f/r0yXA2LfPl/t7L44PDIs8E50zwt95//627byPjX+I3rGh58t84twbrP7n0v/PcV71vMFrRee7GOoSV+tHavaqleStzxrpDfomE+c8kkJSDhMsTAXM7Z6IDZtU4e4/6dYlIawI3cQAdRRH4HXZqxGlX5ZkFyWaxMoIpgzfAJLNRjrkqAuOAqLV2G+5ocy3SWkolpVREKIQCKlSptVKIUmsNCEEYJg1AXkjc2bn8N7//S2/cugKchWEYhIFdGsiYGGyEb7x5OJysJfFkNIHz07VG/OB8kkt9KQxZXo4KdX50KoT46b0nSZ595/rOw/Hsf7n3LOIs04BRPJ/M51n2yWiqW+13b9/49u7G4XT+py9PPj6bRIO1uNe7M+iCkh8+3jt/+ODl4ctPx4uvbq31QWWHR+XLl51G+nKZX+40Xy7meZErhi/Hs9cuXc2v3N4b50sl86I4GQ73j49m2ZILxjiTspiOh2cnh/svnh+eHi/L/PHjJx/df3h0epoXRVGUCuDt99771b/y6420WaGz5q1QxZAmSISkfeVXVTHngt6Ol6C2kIicQQguUF5zhWHVzLW39HrfO2fevdEE1kmy8we9c1Zz4oyuJ22YFarl9cyayWQdcFvN5EnQGwcOmQjKWZwVrP288QFWH19AxhHNZk4m1VnNVcZN36T26XhZFEXAAwh80Ax4GE3nhVaSEZVFXpRlqbUQIgwEY+kyW66tbf+176azP/iTvWeHTAiBTGU5aJLd/vuba6WCvWUxB7z19a/95ntv/mj/6CAvf+/B81/6+vvv7qwLwf7s+Pzw5OybX33/g8u7+5Ppp0enaRz/9MMvysn8O3euPpjOPn70vNVIw36/1HSp1/7dR3vroeg1k79398bz4eRXblyJovAXxYTC+POz4U/PR7fbHX0nGvW7T8az5+WLcDQuk1heuvrkxREFZ4Moij/49otO74vpYePk5PHJSXl4MCuKa5evdjq9KAgF50UhgfGk0QrTZtruPnn6ZO/Z86NFxqNo48135O23KYjspgw2W2O1qkeJ2yTuIpDQ4bHyeIynYcfx1bOoslj9BHCQAq8w60Reu4r3rEVNv3vYoAM3ObSRS1qBrUElAPug4iq+5U60H8nRpJ2E7gD9ZR4cY0wp5cwHZMg45yCYLv22ZJaABWOMM20iZHZ/KEVEoHUhS8pRA0AQBoFggne7fQIqsyyKYqVkWRQSSGlFHCLOVbZoN5pvXrt8eHSuSsU5E0JEb73f++Z3Z8ienZ0/G03V6fHVW7cSxiTn/+r+0ybAd67unGT5jw5Op5NJ7+zlla++9YuT8zu9VtpI/69/+vO21v/Hb78fCnYljf/li8PtOPpLV3YDoJv9zheT+Z+dnK8R/+nJ6IeP945Ozr/91u32G+88f/T09NHTq/1eS2nJeLC+dnw8/MrG2no3/eHpmItoXGSUZTwKW7NpHqSfdbaXhyezklGmP/zi0eP9l+04EVHEBUdNQoggCKRSs+l8OZ/P5gsVis6tOweNtecvT+9ubtzptokqOxIvQAds0bNTxwR2IaTVZl6Tm/p861v4wn9vC1TDXvtYMyQu+PJOnVb+u6Er4VyiFXSu4tU8/tXu+Wkeu+J8M9srH0+rB9LQl8LXwhb2mNXb1Btt2Q/ttiO+np501VOzSZgmUqQ1kSJSWmktzaxSZZkTLAo5lZRGQRqIRpxEYSiVIq2zMl/mWV4Wi2w5G48GrealS7vRR18sdC6JSsQrN26/s7a2HgW/V6ifffrZ37515Vt3rs/y4loz+eG98d2ttTQMbkbBt9Z7P1jMkktXbm5v/+zobA30tVbj58PJBsCTefafHjzd6nfY1StPXhz/84/vh+2mPDvvdzu3+90XPNBad5upSkKlJGXZdn/wl1+7EnP+dq/5//rki48++jxsd7qN9O1ea6PX+6Nf/OLlF/f0u+9/9fqVGxH7BZMPT+U9GRXNHb7VvDzo6pP9h88e6dmcsmXQ7QftTsCQFblOGjpIZKK6V2+xze3ifHI0nf+Hx88Hb9weRKHN3VUJnosq2EdtnBFKDn3eUvS1oeji7WiTiXXMEUF9/Z0lvi+p5/8S3iISjrpNjGllJXj9Kmj2fbJGJHFCRNRoV4m4xaquYS7HQKuzBJz6tu/rFnFVr4cm1RkwxhkDxkyg3tZ6AjBAYynqUmqpSVkzlDQpIk5EQEqrZ4viRU4fjo6+1e8ca70WikEkEo4NHgQRB025VjNZtIke7b/Ms0Igiig+275+XyT3Hr+4FIpmp1X0Nvfj1h8enH3+4vDNy1vrN6/96MXx/XvP9Hj62nqvc/Pm3icP/vjpizIQv/34RdBItzrNNrDradR/6+aPXh6dHw05gztrnTc2+3+ky0ir6dHJV9fX/vLO2t1OQwL93//t758+2fvub/x6X4jf+/mn7//lb8Jsyc5O4/7g4XD06U9+/o9/4/stpSGKQtAfH5+Fm4PF6dns8/saUW/vKKX3gkZvsD3OKNrlMeK4VFkpRZr0moku5HyZdS9vnyslCorb7XLvxdPp4hcn59+9tMVtTBG8H25QZiOXThGjz3xXbpLRcuTYrgYRYyNcMOEMImrq/kvReeHl4gAgvCdk5kTd1PCAM61htr3Vii1m9+02WsEbl4gAJururQUfob3QCB9MZYhS+4fQGLGgN8G1RSFppXnAiXHidstvk+w0u0WadZ4AUEq1LPWS2EjqAtnPRrMNwX56OryWxO+t9Q6L8koieo2dU2qPqMiam+9/99ep29dJ40qURCI4ni3+4vlBvMh22o12EAgOT8+Gs9PzvNv9+2/d7At2rxUfzxeT4fh779zeTqK73VbIdh/PF//0j3/y1TduMYBbzcZrb9w6/K3fPz487dy8cqvV3H3j1j/9wQ+fffT5L/+jv3+yyH/nsyd/9yuv77aT4WL81Uubd/utjzb7f3R0PnyxH4fRP/zG+yWp//Hx4//13uOT8yEfrH/3+uX/9PnDPwQaPHkM52fq/Q+SjTX56Gk/ChcZUiFnaTvrdZL5rDw4WQQJRK0tmg9Ph0W3y0s1PDjevHa50+0eHpz8OE06YfD+xqAGn5ptZkfcfqtJGVPO6l+LHLt7j1PmGvz41gKcFWdhnUJrnEU2RLpySnUWgHsON/qZQC6uUKc6e1Gbf3W7c9U4b+VQcCit2b/abeC7Yu448mbVe+IMGWMcmbYhU1TalYuA2amXkDFiTCOQtIs7takc1ZqASGsk6gbIQA9CETJ8Iw2/PuhsCEFF8e+Oz3/rowfPxvlBRvtK/D8fj0927m5+8K0fp4PPWPy9zfVf3Rj8tWs7t0JBp8O/fm3ne7sbf2Nn/VvXdl6enuHZ2UYgbrebVxrptSSmh4/6ef52v/One0eHk/lbvU633fz86GxG9O8e7k2z4u7Na6zZ3Gk1vhhOf3hwdnNnKx6fqPm8FQVxt/l/+fnnj18eX711c1jKw1l2Y2vj9x48Pn3wRSvkDycLhvyDOzc//dnPijDavX6tH0bdRhw8ubd8+UL1ByyMmUaxs71/eJyfDyGKIc+ns6wcrHciwc+HmaIiiUPBTk7PdbsVhsHJ+ShupE2G2WL5o+cvJ3nhuIB5tDhWQmujGV+35lEAEIB23jmRSZsjOq7FOor8WPuPGsgZDB4EUB3nj6zhiLkIwCocaxetx49s8BI8z5nWo9bVkmp65ZZQY3W7i1BNibglm7QaRmCCm+0btNa6MFtCutYgIGPM6HRt/CSllCliMUvsEQTovuAfdNtzRU3OTws5HU42onAjivTp8OnB8Rvdxgfr3WA8+fMPP8ey+DubvQ86zX/6F18cLbPrafLffuu9S5e2/ulPP/39Fyf3zydf3dncefv1MghGhQwYy5ReRDEUy2eHxzd67buDzr85PD3Pyw+uX7qxvX6rlV4dtH9wcJakabvdRsCvbHTbcfCLpeSt5uPHTxDZB+u9FodZEG698VYnEA9H0504utrr5m99Zbl1WYH+83uPu2F89a03eLMzHo1+609/ErQ7jIvpxra6dpNzoYgKWYrTo+z0PBusi80NkGq+zKnTjjiGAStns7DfbbQaAaLotAVgpmHz0hZjLOS81PoCHlbYyBR5uPo7qP1nQeyeweUvYQOT3uV3cNK1KgtvIzjPujIQ6hjDGvz8PgYrYVj7lVuupMEsbwezkzdzS+bdXKP62lhyzYSay1WDuw2VoSvy8wUo5ktmQ7LAGeOME2Kp/SI552khklnbSaSUIqXIxURNcNRs6LyUshkEy7Jci4JRIZ8uSwqj97vN6NqlPaV+cjJaD8NvvXY1L+XhPHuz29puJE9OR/uzzOzs/+2dDRTiu9uDRIhJXkrGkzReiwJFdK2RnM/menQ2fnz/3z15OVb6H9zYPc/ynz0/EFm2P8/u9DrdQPzw6Yu3ttcJ4MFoPpplc2D0ze99zJOPT4f3J/MsCPibb/z4fPI//OLeh8PpvclsbzjS62tTJf/oT3706cPHv/vDn015fE44kTrttN5O02+8/0H/8jU4OysOXs4fPFbjGUax7K2p6aIoqNWIwpPjJbH48m5jNhnPFhPk0Ou24jAWotVrM6XGy4IYC8PggsazKMMKhmbuu81T/ZA6Jl3Vy+Doxptz/lcXQAfvalV4ALdxQQ0bmgg1OSIkYU09gytnEtdhbWKV9V1KqFYsooEYcG9rugZVvFiDOxprw+zuQGYD35oZ6tHsthdlHJHbBeCklXI7PwMCQ8Y0gEkjKa0lUWgr87RSSmp9nqmXpWA8OMrKlLMn84w3kmYUbiXxRr9zI4mutxp708Wnhf7gK2++M+g8HM22mumVmL34/PPi8ub+bLkWBn/tzpUfHp3/+uWtJC/WABln24342Xj2jd11CSR/5ftqOFwq1Q+DTiAixGBjcF/Rdl7uTedzgGBtcIrQEOxfPno+kXK5zMIo/vruhiL66qCzPzx/+vBJp9//W3evbSfxOM+CNEpGIxWKxt3Xfv3aztPheKrViIoG8J3r14so3Iyjm7duHokAGbvUbEqVv5zO2OmIQGLajHudxXRWnJ6VRyetyztJuz1bLEe5ZL2WOB8WgGvNdO/F4fpr1zXRIi/7UbjKSMYvd0Co+ezmLQJUxW0+mujRWSMjuPhaiZK6uVABwwcuAXws3T70lXlG9VPBX9/qc7soA2yWvE6HtVv6r7WbRjWvfsUA0P6JzbXr1IIJAGbzXUQT6zSekKkrZEDc1tvbpW9aa2kwqZRWsijL+WIxXMwl0W4ScaBBHHHGp1pfX+vtpDEHWA8DTrop+EkhoSx/fbMfMDzI8i/Gs4OiPC3yAPF6u1EofavT/tpG75Pz8c128x/eusw2N39v7/jH55P/8ZNHEeNfv3YVrl7/0dl4f5H9Pz57/J8OzkINnTQ+zPI7vXagFQ0n393dvNJM3lzr/sqV7a2ylM/3N8KwLGWWFXe7bXj4sByNOeK/f7T3clnkivD8nB8c9ZrpRhB8Z3tjlGf8yePZeDSaL/74/sNPDk9CxtT5SC6yNhNfu3ypV+QciXZ3eKez1u50Ww3otIMb13ivv7G5HirZioM4DHcGPcqzRVk2Qw5Ex9PFy9lcO78CaqNvi9PIu0CWusyxhggv2G810iX/zYUIY3V87V42OGWVradxz9JmxD2KvYdUXQ6rq6FJ5+g60SOAezpRlYKqgkc1WvXtcwFfwkoidur54xCRM/MIGrMdmV0IL5Wy9a8EnDEE0EorqVSpZFkWZbnMs8lkfDI8O11kPIi7YVCSjhljyP7qRv+vb6/1w0AA/M2dtW+v9wFgLQq+sdE/LMphXt7ttbZD8UvvvrW/feXfPNr7ydnkrChOi/JsWfzgbPI/PdzvpBGQ/snJsBEGC8Y/PDpLSV8PxC+vd9/tt7+10f+DDz8Tx6e/sbv+w8d7y/Hk1y9vlk+e5tP5J2fTm83GdhB8cOvqZDwr8+J2p/Hw+GwqFaSN33jn7tfWe4M0+rdfPM7ORpmCb9y5+c217j//6N5plpcoFBfrg36nmawF7On58NHey7VGozsanp0cP11k0dkZFAUuFtn5+dl4mknFsiwHGGfZQaaS9XVotobz7GWYXrt9c8nErZvX1pJwt5X0IqPlSVtr0ngw2i7zBk2g3ZJij0BFpDwF+kH1IKu7H1jzklcIjZzbpZ3i95lCR0/kaqAIiDlr44KfZC5H1RsCZvi0Os6jHLyn5VEI3qZkrB5YRbtDpMcw1qEJlW1qK+cRmV1YrEkTSU3SuvPMhKC01lLJLM/PxqMnBy8/fnjv40cPh7liIiyJSsJc67lSD2aLmZS/fTw8XGQbcTSS+ufDWQDw7qDz09Px77w8+1dPD59N51/ttb/a75xr/ZPhBJXeisO3+u3//vbu+73GP/tPf/FeFPyf3n3tb1za+Md3rkxL+X/7F/9OHx0nnO8NJzf7HQj47d31QRzuNqL/90f3//hwOMmzZVG8Nmg/HM8ejmdrnXbUTg9m8xzZMon+/cf3UZaPTs6fjmfrrWY3CbOz4+DJoyxtLAjTfuefffYw23/eyhYTLl7MlrLdUwHb+/iTrFSD996TcfRk78WSC9XvQ6PByuzs2f6Mc0piNpnAoyezrJCjcf70ebn/cjhdCCFUUUitx6UMhIgCAWgtf7LLZv36xYor/PqJilSdF1FB1J3wqsNxwQdy7pV9TIyJZnmqskhyH5AINAnXjlVnitzVLKCdIvbGgHHKHPfVjNLKgibfW5dltwu0XKmrPxgdrGsbhJvtyBEAzENqtNZIxICQiAEA4xRw0lpqtciWo/nyfLY4H03yxTJoNvnmtZ5UgWDnUv3obPi3Lm19uH/8Vzb7gygYST1WWnC2UOogL+fT+W9e2mgG4miZ/dbDvXxd/t2rW7nWv7139K9/+tEO0d/5/rc3InG33/nnjx4zJS/1Or/38PnfuH3laqcRhCxot17rtvZOFSP65huvncwXB5P5u1d294Lwn/34I0a0zPPtNBZB8NPTEWolFvP3d9Y/Ozj5leu7x+fDnxzsiyh6ucxn09n17fXzkyM1WNdpqvLia4PuT4r8cHJWdgd6mV1VmifBpw9eBpyJZmPv+Ly9vk5PH8oXL4CFEKei39WnZ3g+Ut1OO42LvGDZfCFl3EzLZSbiYDmdtgIhlWbEpFKzotRE3EACcXVraXRekUafcXLf+iG/wCweQnVN6N9UC57qUSF7G0ddTrVaBBIQWIBaiHl6rkGzNkuICIhh5bG7s8CWz9VMZnKumdtEh5zpXQEYa1tMoTufyO6IgnaFMJpdwH1ZiKFWAFSKFkWRLbNlKeezxXQ6L/OCNDUQR+Ph2WLRSNIbSbjfbIzzfBAFz2aLv3Jl63/67Mnv/+hn//vvffN6K/39e0+ePX72j3/tO39yePrtrcFoNP3dp8+3W43rrTQGKtJkPjwLArE/XbQbydvvvrWVxHf77YPR5D/uHyHnbDF7rZUMorC52f/5yXC8zJ49ePy13c020D+6tPGHQL9IgkfjxZ/uHd1KwmSr/zuP9qjd+3g4+9EvPr3/bG+xu8Nff+OzJ88PlTx+/mLz6mWxfamUcO/ho881KME7a2tFo6tPR/zzTx/tPWev3YZGozgbrbUa5Wx2Pp1yTXpjhzY2BGJZFuF0BFKFyKaAvWa82Ntb9vsyinvdjp6Nz4m2+p3FbNHvtYHg5XRxs9dhjPmxqVwiAL+U0iBgBbl1LluFKdbR45BX94+8ljdHO4YCJG2fJu/oi9zNhL9abSG8jxSAe76XW+RfD7JahW51/mqzrVaoJ2a9Y+jSAMb2dqf4qJPT3eDMU4EImqSUeVkIjkzKpSzOx+OD4Xg8nJgqJSklaWKADEEvs/zk8EW73+hvXum0302jaam6SfTZePb+dP7Weu+jZ/ufn493uu3rW4N7z/eeHp3s9rpfnA5Zv5PleRoGL+fLWSEL4NMo7SNdWuv90cHpkdTH9x78xuvX39teezlfPp0u7rz37pKxo0UWM3yr1wKiA33zt1+c8Cx/vxn/6rVL7W73Z/tHyeHZt7cG7/Y7R8vsoNduBuK/+85X//jgJBHs8qBbIF3rtr9zZefT8eTFcDzY2exzHCKPkuR6HAaCnutd7LYkIZUSAoECjj75JGykFCdquYT5XNy/B2EMl7YDpeXOjiAqigKUZlqzIOw0kjTL5kW5jONCiLVOGIaCEANe3wUJHJV4Zeuw5FVijfLqoKzDl8APtz/Gesyv8i6aZ2lbyJstUQ06tYsYESIKchvUuFbaIuXaRKkZumAJz6wYJjCbjqH18WstqJgXAc2CDSRA9E924ASIdkMHIw8fOK7NV2SMCeSAUBQFap0tF4UsT0bnJ6fH0/FUliUQMG4fOmsWN4HWw71nZVHwN74yThJkPOH4TqvxdJn/4dn4q93WndvXP53Ov7I5eLoUjStXPjse/p/fuPlsuvjTk/FiMn90Nvz69nojjjd77Ww+eTmZXY3CX7u0MSzLB430//v08O1e82ojaQveiG79wcnwlqT3uq1L7WYzyvqM3Uzi33zrllTlH3/+8MXp+dvbGxvN5NHZ6Z9//vlH9+5f3hzI61f+3cHJ7OQ4DgTb3u13msvx+Wd7T54/fNhKkmgwkGkSZhl9cVT2252y2BRx0oqXo5E6HeciDDYGRRgiF4HgshHxQBAXGkAUWUFQHJ9AGLSaDX52StMJIuYcIlmWp6fTZnuWxkkoGpwFgsdCcES0WUpPJaveTy2O7YbEbe+4aoZqupjHNuPnnSTnU1viM/FMUw9l4oh2h1BA//gd24R/9lv/wSwPAkeKZqGFybdWUX1to6jmxuaxG0ZBMMadUnahAADSimwUyHl5aLfXNUYCQ0TG7N7J5gwCG9Qsy0W2HE8n5+Px+XQ4GU9kWTICJcs8LybTyWQ6zpYZAgRC8EBwzrTSSmsETJIYGUPOB5ubV+++HW1dL0SERP0oWgBqgLIoIiH2lvl6FGrGJlJdMo9u4Ow8K07GU57ElxrJIAoXpbw3W76cZ3c7zff6raXSPzuffDGe//1r22tp9Mnp+ZPh5FK/myAKWfz48wd/sf/itUGn3Ujm58Ozg/3xwUss8ka7qRtpORrHHJMyu7Q5SDvNaVFmo8nLg+NZpuL1tZjxG1e2H9z7gqTSWqfNxmKxWB8MeuuD8fERaRqNZ9liITgvEHnS4GFIUi9bHb51ydS/8bgxyTIkAMExCBLBaTZfMCaSeBBHyzxXgI1QpEkYiGA7Dtth8Ob64Eavw6CSf50eyVdlQAWyeuiwBuNqcQRetBUvHukvaBCozVIgswlZdbvaM8SIRD2F6FW4Jm3dbfsYdq38VvBuXtnZhUCgmd0s20ZMNWkCrbQmIjSPmHHARQBVm2GGeJXSRk3LUipNUpbz5XIxm56en52OzmfjUZ7lpSyLIi+yXJYlaQ2c8UCIQACilFJpigKRNpIgjgCx0+5sXrume1tLHp3m+XC57AueBmEYJS3BR1KlgqdhECBsRkE3jv7j4Vkm5ZvN9N2d9eeL/Omi6IZBrmQC+o1+eyMKfnY+3B+OSqW/2u/eOzn6n/f2954/S89PppvrOk17XDenk6/K6fTzz5+PJnme72yubbZ4JJqv3bp67/GzIgXB+Vff+WpZSk302cNHe/svl1mezbOgEe5ub1watIOrW6dn40GvNRpP2kn/+u2rf/6DH22u9+OQt9bbx6dqluV6mXOSLYgKQjocZk/uMQRgjHe6zWxJpUTGIQyJCyjydrMVrK2HYSDPz4QGHcWzKOJpQ29uta/spmGAXnnBRQiiz6pXg7USbyGXOvIj61m2ArQ/y0HTRF5tZZ790u0pV8UQVl5CSmlvbc+ynoqpoWNIiKjcE2CV1syt4eREyu4Uh9p562QedAB+vZBSSkqzua/WpZJKSgD7pFqyTwWlvCzHs+loOs2zLC9LJaUs5Gw+G03GZbZUZSlLaTcE1YQAnDMRCMa5JI0auGBpGvabKQ8FRvHm2nrjymt78fpPDs4wf9nncEmNh6OzrNnLm102HZdRkwAeI1zdujRTellmAOzW5tY0n//kYP9yq/F+v/v49OiTg0NdZO/221+MRgfTKVvOscgeB2EjYleXc3r+LEK8EzRPz44H7dbbb91+vLf34ejs8o3do9Pz/mBAUr37+vWP7j86OD67fuXy6zevPN9/eT6f7e7uChEwzsusiMPg6s5Gq9GMEAJkoRBa6vffeW9UFE/v3Rv02u1mmhfl1UtbGpDOhtcubWV5cXw+4owrpbjAbjNdZDnMRpyxrMg5w3wx4QCMcyoWej5UaRKSzhfLQlOcNhZhYzIcz5b5WVZ8bXv9jV6bu/gLw2oLLefgVkuIVuJPZC3EFUh5C7YCcVX4bBKe5kTz/ENy+ScX90RnG7odmEkTkVBaItgHWDGG2vvfWnNEs7rKPT1bgdmdnuyGvyanaUpFAQg0mOL2UpayLAslVVlmebbMllmeZXmelfksW0opEUiTlmVZSCmLcr5YTheL5WyuitJuCWqeO0vE3Tp50sC0RgTGUAjOuV1HH8VxpxF306TdaoIQFARs4/LPoXPvi/tXhnuXIiXzJSjZYkzIZXt5Op0uWFEGAZMaFiePG1HcScJ2q4MHI5VlrxeZOi7uPdBRlO6enRXnpyeCzUaTq4MecX4+HCPATKrRaLLe771x5/Ynn95b3+xvb27MlpmSspHGh2dD0nBtd/unn37623/4n5SWTHApi2VRPHr+8uXhyecPnvV6nWKRkVJhGm31exu9XgDUabeOTs+nWXgyWzz47LN2I+m125e3Nw9Pzg5Phy9eHAVxyJCtd9vHJ2el0o1mYz6exEkcp/FwNI0QW712WUpcZKVWQnCOoMsim+luI02bDS0VIgSQqXJS7D39Qspn08VfvXn562tdtgowS4dWQ37JEmRwURrHrUj2gdeV1epVpFf97mrWojDhKyRbSufKUip/yDNoCQACEEWgNBBo/yBRbeo2NCmlbHDVTh2zolKhL00iAAClZVnKoizmy/lsNpvNZ6Usy6KYLmfTxVIWWZnn88VimWdaE2gt8zKXUpZKuwfOIENun2pjFijblDworQk0IhcoOAejLBiLwqCRhJ00bjcbcRxjHMet7iLtHcyzG7S4kuhyPGZaFYTEWTfBSOswwDBOwig4G8+CfK5UTjLMF7OlpkajIUqZRmK+mE6OD3Y2NkaylU2nd65dns3nnWa6v/dyssw5MMZYp9v5gz/5s2aa/Nrr3yLS//K3/8Novmg10q2NdRHHAYNOKI6ns29+8FZAsNDqeDjKtC7yXM7noGmt32k1kkYjPj4bCeQ6mz1/efTe++932q1///v/sdtMMylzonkuMQyhKN9+87W8KOaT6cvxZDbPOt1Wr5UEoJeLZRSF/Vaz12k1kvj0fBgGQVEWy0IWUqVRAKRJSS1BE3CBXKqCyTjSfDaccv4XrcblNN5JogpQKxissaFLRroPFX3W9h686MmjLdp3gDZWBVK1nY2PbAISahO5r7xkxoSxQIgbl98WUxLZBymCW1ZhlvdWZgq5DTdriUopZZZnk+n4+Pzk7PxkPh7LItdS5UW+yPIAmdRqkRfLotSlVFIVUhERN1vbMcY5E8yn7BEQTWYTNSEQR+QcQs5LAKlUzMNmFMVR0IzjOIyAYCFlg4s0TuaM3e6kR8t+fjoJgiDP1SCNoyDoJMlkOonDcGtj/fh8FAouhBBax4jLPG9FkSiyzfUNEYTzZSnl7MEXD7bWN7a2t8psqaRaLLO0kbxz906/1/3wi3s/+NHPVFn+5vf/0nA4BMYODo46ve5av9PpdS9vbGitHj58HoRBwNjZ+agoihfHZwcvj5SU/UH3vbfu7G5tkFT7B0dT4vcePSkWs1/79V9f73V/8eM/DzhLk+Qsy8Mw+OLJ8wihncbns/k8y5IwjMJwrdduN9Px+QgB0zgSnIdROJnOkGG71Yzj+PR8hJDzKJRKkgYlVZrGhdRmG2ulyuloWC6L+WjO4ni6u0FpjK5g14MTfbWaNzXrxOjIk9w+oy40qV2U08cjrdfjntMBmuwGCcavB7cxGWoAl3x3XpsWBv5ak0b7hHcwViSCBs0INZFyRUNKSSK7gI4h5lyAtSeoLEtZlnmZn4+Hh4cvp2cni+lMlbKUKtOqVIoRlER5UUql0S6yJ0JQBESaA9MAGkAAlQSAIAAER47IGBM+ikEEQAEP4igIo6ARRUkS8VBoLngYxs3mUbr2uxOVsPmNiG80wvOJ7idpu5EA6aPTMyC9PuiGcdxpNvIiX+/3EPH5/kvGeWdjrSjKo7OzxTIDxP76xrVr11SeSa02L106Ozl+svditlhub25c3tl+ur8PSEqqLx4921objIcjALx+4+pX3rjz4aefvQTYXuvPpjMeBVmWX7+0+/OPP3vx4sDE7zhjr1290u92P/nii9licZbJl0+efft73+l3upgvRcB7G5vTvJiNJ/PpvNNIllqNxpNlXiDptX53o9+5vrsxns6KPF9mRZbl/W47y/OAB6BBar27tbG1Nnj0fP/g+KwZx6WS86IslGokcRSKvJAh6AhUPhljL5Bau0eZO2Ss+jpQ24XG56Vx9RFt9lcHYiQDwirxCWjMTASt/fJfN/7WAYIq1O7PQgAU1jTVWqPSQFJKX3nKAEgIEzMqykIqleV5WRaFLMsicwuFCBBLpbI8K6XU2aLIFrPJZD6eyaKUpEutpNK51mUptdJSSjCLQ2zk0hUWasUYAHGJKIkYAucsCYTg3CyOY+bZx0ARMhGIOAyiKGo1m2kc5UBBGK5tbNDgyk9l+mL/SWM5/d7lpIk6C4JIcK1pMp1NZ4s3Xruxu7P9/OXx8xcvACgSnAiajXR90Ot2Wo+e7J0NR8dnw1YjVYCdXu/uzduz+fzTz+9P8/LF8XC93WilycuDw0iIACAnGk6ng7XBeDpdFsV0PN0/OrmyvfXhvUcPn+31d7azxWyyWLx1++a1q5eeHBzNJtO007i0u9VI0zCKAs7zZbb3+Pna7s5wOnv++OEH77/7jW99689/+vM//ejTfrcZtVMgkpJarcbV3S3OUANs9jvtdmuZ5VKqUspGmkZRIAIRx7FWMi9VnhetZrqzPsgLeXQ+jAMRcM4Qi1Ii55yxXEoFiKBQq3KxfDlbXmulfFWlYw2XUENh/ScPJheJtD/bA43z4+qaq+oN0ohu32W3sSi5oqpVU5eAQJgrS01EpEgXZVmWpVKyLAsOKIKAcU5EWsk8yxZ5Ns8Xy/l0OhmPZvM8y4hIA+ZFCVpHDJhSAoAxLMqyVNKkJ0mTlrKUCjUhgEbQRNzlOYmII0NSqJEYA4KQs4CzNAjSKNKkF3kBpEUgEJgG4pyHYRAGIk2TTrMxnc8ni2UUha20wQc0EOLNG5dffvHJ2cFep5220uT47DyNwlazubU+2FjfLDVurg0Q8Xh43u/1sqKIw4Ahe/7iKI7CVqs9ycvj8Uwg3Ly00wjZoLt9Np09+vQLu202UiONbl/d3b919Ucf3d/d2uAAHDGJgs8/f7C+sUZRMFvmB0fH77z/Xgh6d9A+GY4eHRyR1qR0Mwg/uPva9s7u6dEhZ7hYZFEUh4y9fPLkm2+/OZ8vijIfnp6urfeOjs/W1npr3W6v01GkdjbWpVKCiwBJK73W743Gk8PTYasZa61ms2WeFXmeh1H08Nl+lhWB4K1GMo/DeZZLwEgIpZQmSqNIKlUQKQVMSV2Wv/9k/7VuczuOtOdI7Vc9OgjWV1vYx6rYWOmq8iewD4Eh/xBEq/cqe7By7cHZojXnDNyzu215gEBkZJ65XqpSyflivsiW2XIxmc1Aa855EAaIyBFBlfNsOZmMFpPpbDafLpZZlhdKm4amnLMwCDjPQZOmoiikCTMBSEVmgy6bbQVAjvYhBwTIMGBccKbRBF5ZFIoQkTMslMrKclkUseABF0EQEJIGUAg5QBPZYpnNF0upVIJMypKVy+3GIKdg0Yjzs+zxZHQ8GseBaDXSjbW1Qb+viRaLRb5c7u3v9zutMs8E0Gg4C8OIAeRZnmfLVIg8CsIoWd/a1oTHJ8fZbNzttUfDYVmUYRA0m+12q722tsbZg/l8gchK0pzzLJs/Pzi+vLuZpM2rV5KtVnN9vT88OX7w4vH58Wk2XQjO37x5dWttsJhOy7IMgiAMuS7L4XB4+85NhtTrdD7++NPZfJmEwZ0bV9+8+1qRl81m4/TsfDJfXN+91EyTxWJeFFlR5M00aaSZVroEtczLl8fn82VmiQxRcB7HYTOJI8aXpczLJSKmOpJSIWeNOC2UEkmiFJ2PJqO83E1iiw/wGtmqcxPkAXBL6WoZbKOj3bN8zOoL7crsCBHJFSNTVaVELvtvyXole2V0qtulg4iEJm1L0ZUsyqIoisV8djY6Pz87z4tMgw4QQsYF50A6z/LFbJ5n+VLKZVGWUhFAwDAWQgDIUpZKlVqTUlIprZQiUkpLbbOnAMamRPNoWq21WQAPnCvGjD2Sci4QNUBWllJlpdICkTNknAWcl/bxSCiISMplKYGglSSDwaC9uf2UtX/3fCqnw/fzUTPkJ8P5bJFRGj05OBr0unEULbLFaHTOGV7d3Tg+Oc+y5dbG4Ma1q51uf75YTqbTHaL5fPH4+Yvz+WKZl5sbW8D5YG3t4d5LIhJCcM7DKFkqdWVnfa3Tevni4Hw8vHH1+htvj1/u7R/s7W9vbrz/ztvNZnN0crqzsdOJoiBgj56/DJP47ddvfv1rHwRxslzOwygw9f8sEMBwdnraa7WIi/7G2ht377QajVa3vZzOF/n40uXLN65fX2ZZyHkjbTSW8+l8JpV6tn80WWRFUc4Wy7KUSmkbrAEATVLpuVJBEAjOdZFrgABgmWcMIEzifDbXPEz7g3kUJXGUCqFddKcK3tvMYG2jjVoyp+7gk4vuAwJoDW6xsg2tG9iRKbcgALelPKBGV0f8SsDfx/YFIiolS1nKspjN54v5fD6bz8aT6WyilkuulSKSnCFjUsM8z8s8L5WWSmmtOQEiJIxzBKUVIJIGqaSSqtSkpDYBeUQMze4TyMCWh4LZlNY8m9Nk581i0dI9d8asfwqY27fJJHwJEICTJg3zPO+kjUaaNBuNKAohavTaXRyfyNMTORo9PjvMizxIYgVwY3vr1vUbSdJQWidxEgVikWXzLNte722ubwzWt4KoofT5YpGJMCo03r3z2vOXByfHR8046ne7rTRpthpnp8ODo9PpbNbvbybNzqA/uHNj96cf3X/+4uDWtfh73/zmk63HP/rFJ6DZ22+81Ww05WsyYKzsd5+9fLm+tRlE43feegOIsmXGGD84OjobjstS5vMFIF55685ar3d8cDAdnl3b2TGLsLWSN65cKvP86PhwMp4ss6JQdH5+Jjju9FsMdTsJIQnXuq2iKKeLbJ7lpVJKKkQw0z0rinaa9JtNwwpFKRd5vsxLGXDNGJcq7SffvX5pJw61UlWu6BXrE2pOknGRwBKp3/5I2yXJaBeyuWwmeZiaOJTP2ZPWzDpolUWLtQ3hDSsLrXUp5SJbzuazk/Oz8Xi0mM0Ws2mxmDOlCFFqPc9yM8OyvFBSEgBjTJitHAAJoJCq1PZBpdzEoZQtkUPGwkAEAdeMmXVt0ixf18SQcbMPqNbkHjuqiEqAkEHAGRMCAZAxpfSyKAutBeMEUBApUjFApOSg2Y2TJFeywdi4KEUcJoKdHp8kJKMoTuJod9C5feUyY/zw5Gh4fjYaTUqlwoALzqezpdLAg0QE0WBtu9vbmC/mWSkDzoXgw+FwNh1vbW5eu3R5b//Fw3uPs7KczTNCAk2Dta2rly//7OMH9548H4+n3//er6RhuLm5vrE50LJc6w8QkQueL9ud9kCww7/yy7+UBmI8HDa7nZCJsizmi2VeSgDY3lq/de2K0qqZJts77+ZF/mJ/LwyDQb+7zPL9/WehwHYj2dvfe/D8aDpfvP3atcvb671WNF8sc6nDMN7Z3l4sl5/df/T4+UtZlou8kEov80IWUsUUhKKUCgA0gCJQIqZGlxifa4o1dQLBkCqv24eWHFJ9vMnkkmw9vFXYJvLoiNvbl+DQqXX9OUYm6e4rSyrCdHUkzmytdvEUWqqiyCfT6dHZ6cnJ8Xg8KpZzyjKlFUPkyEops7LUAKXSpEkAcM5CzgCYVkoTFFLaPecEF4HgjEEhZV6WUgJpJMoBCCEOOTGmSlkqBYicc84YMvPgTRfGReSBMOWljLGAcc6ZMVOkVKVWjHFimGsCwEaatJqNfrcbxkkhJWTTgtJZrqVI+OBSSsvs5GWe5be2N+IoODw+/OiL+8/3D5ZlIaVuN9JGFCRx3Nw/EFEjjeOsKPO8BFCNMHjx8mCjP3jt2nUiPZ/P0yT94O23tJI/+/BTU8NbFosyX671un/7r/7Kh/cefvT5/WtXLokovnvrxkZ/kERhnMSMMc54HEVf/8Y3ci1J6vagNZJqeD48G40Xi3m3074bh5e3t25ev8IZ29/fD6KoVPKLR0/2X7ycL5ff/cZXEHUzDZMk3hh0XhwczMaTXr/7/hu3Nza34jjpKsVFoJQSjMWd5nq/vVhmjSQ6PR8fnY1Kpcu8GI4mCICCAwAyBkzwuKFEQI32RERc6ZYQdaa0oXVjHDrutBCygNKueNRp+1rRKLkaJVDOeaqVnoB90AC5bJF2BRmgNdmqKp9lNTboZDo6m4xfHh+fnp7MxqMiy3RRgJKkiTiXpAuptCJNGjVxhmZHGmOjmOdoEwBnwIUIozDkopBykeVFURrTGoGAtJQyB+DIlCaOLImiKBSylKXJjUoNSMQ4cIYIkWCMcxGIThIrgKJUpZQaKEIRBEGBwJQmZI0gEIhSK6bkMlu2kvhrnfjjvLgn4vHNN1U5vy2gODncOx91uq295/tP9w6yUs3meV4UZ+djxlgrjbVWy9lw0Gkdnk0Vss21te2trTt3bkkpJ7NlXmgmlFSy2+197b13yix7vr937crlIAyJdCONte589xtf/Y9/+kPB2WI66Xa7jSju9QacQZJEnHEA2Nzcvnvz5tOnjz+7/zCMo/lkqkm1GunN6zeDIFhmy70XL37+0Sc33rj7+qXLy+n4z3/+F8+PhhGot1/f3d3cSONeGEbtduvsfKy1/qX333znrbd5EDMRj0dDAAaolC6Pjs8+ufe00UhbrdZskQvOG3G40LooSs0QlQICBIXtpg4izYVqd7e31v/K9Z231zrGdkVbiWkxQ3YLGQI0z7T0vrvFrtP3jget2kfn/dfj/LVNGWv2g0/vO+MWwFoI1j8BBPHR558OJ5PpZCzzpZaSa+IIKAJAVECylIjEOSJxwYg545e5QDtjLA4Dbda1KT0vl4tllmW5if8zzgQyY5dIQM4w4oJxnkZhIESBqPNcmVAYY1xwwZmZe4ngjThinC/zfF4UUlHAMYmjNIkyqRZSKg15URZFOZ3NE0IexXGQrCXRb261QITPs3w8kk9Ys8NCHE32T4Y8ihfLfDqbc87jMOitdTuN5LVru2+8dqvf64og6Owd/JPf+r0g3rt77fIH7761tj746Onhzz96EPHiv/lb39cgGu3OX/r2t2bTyWw2XdvYjuJYnqs8z8Io+d7X30ta7aKQO5tbrV4/SVPOuSu+hm63u7W9OxufHR8dTGazZhJvrW2sr69nWfbpJ5/de/pccNgYdGfD04NnIo6Cv/rLX//F/ecaYHNtjTOBjDUb8f0Hj+492v+V73zre9/9Xqe3rrUu8mI4HMZRMFsUw/PR/UfPJ7P52qArlUyTqNVI0zgadFpKU1GWi6KYzpeax7y7RlECLJDA3t0Y/PLOhoGYtRstKtHuZmM4VWuwTwM0Vpgr/TXVbX7rMWdZevhWVigg1OMADt0O0DaGiqu75Zvovbj/+KHKc6F1yJgABEQym4JzxqQSnAkec7M1KEFeFHlRaOeREQITQSAEcpaXZZ4XeZFneVFK++wmJAIOXJg9VpkQvBFHURAqrRExiaKAs0VR5lJKAhOnIgTBWCMIQiFMrACIkKEQXAiuCMwDaqSSGjRyliRxHMfLPBueH7W77dvt7W/IJo7pxTI6ba2L6ai1PMry7Gw0KxR1uu1eM9FSfvP9N69dudzrDcIoVqoMg/DWrVvXdj/++N6j/SQKv7j3lfDNm1e2fvzhgzQQg8GAsRgAdacnZfHk8cMvfvzzr7z3/tvvfWs8OhsPT7O80MA2tna7vXXGeBTHrljWJpL7a+tJowUMtVJbmxvr65vno/Mf/vwXn3z4edzpNNstfXjSCNgkZHmccM7eu32Jc2SIUhX5ovz4sy9++tHDv/Ir3/2173+fMT6bzQBIiCBJG0cnJ7LIHu+9fHk6FEEwXxZxFANgEkdm6/RSaUCc5AWLG9jdhCBm7a7iYavbeWOtIwxyXBQdXRTI1GqC20jGZIdMYQcZ1BnvxkGTLE2iO9aF6O3WjOSNWqt7fTmfy1c5UcFKuTKgkIulAC3sIzq96kapCYhCLjjnJtOUF0VRytIWdiDnLIrDVpoKzhFQMKGlmtvqT9Jamx0cgEEYiEgYpS0EF4o0ADDERhIpHQJbUgakNAcQCIJzzplGKksZMJYEIZGWRIr02XzeCMIoCnkgBOPdVuvmlSv93mCR50maJnES8iAW+m7KZ6rxbrf127qM8v6VsLXZagh2QqQ2W+nl9X6v19tYXwvDSBEAchEEhBAE4d/7m785/2f/6uGzvdl8cXx8evvGlW+8sb6zscbsA/hYFHGZT8fDsw+/eNpb382z7M7dN9e3Lj1/+jiMk6TRFkJEURxHEeecITJAjaSkCsO41e4209Q8u0RJeXh0/Gz/QJLO5nMAGBfF6Wi2Nmhv9LutRhwHvFR6scxHs8Vksuj3Bv/dP/iv775+lwfxYjpSslwu53t7z/YPj569PMjyfD6bTybTXquVRqGUsihlo5FIqc6H42VZLotCKSUaHaW1DkQep81u52/fvf56t1FKiYyZCJHDUBWHh1qwqbI17XuPTct+9pkYFkMuFK+dEkdHujaw6jKb6DwudDEtF1k3jRECNDdBHLfujpmNQpQWDAVnGmhZFFlelEWplEY7A0gI3m21Ntf6RDQcT4ssF0LEQaCk1JqYyQYwxhnjCAhIWiFx4+4IzsNAMMYLVWgixpkg0giFVpyzNIqiOMzLEhHSJEKAolRZWRaktFLLPIdAsCDCMARCSbrRaCCy8WSUL5dbQqRB53Yz/Xg8XhMh27n28/sfbu49v7LWu355O0G2vr6ZpOn+i5e9bqvXG0RRLMKItC4V9QYb/7v/+u/8i9/+nU8+u392cn5wdHLzylY7YM+ePm61e81mK5PZx5988uG9p61WtyyW//b3fvzg4cO/9/f+wdUbt8+Oj0LGojjt9PpBEAjGzMIBk0ojgk6332l3ZFmkaTqfTfdfvMznyzAMpVSz0QiI5kCn56MHfF8wxjiP4mi9371x9eqv/co7b73xVhSnJydHe3ufnp4Pn+/vLcsyWy6fvjweNBMGsFgsu83GlUtbjTSWspRKDcfTk8m0yEulZBnEMNiUSavIchGlcx6sp8k7G/0AmfJbHxlXxlKd5UsXUoeaCieya9vs4g1vkvrYqFnDSVjVgRhAe1fJl0Q5LrbBArJxKLSPWAIgIKGkVIgISEiC8ZAHjHMA4EwTQVaURVlmRZGV0i38RSaY4DwUIuAMkTjnQSCyLAsED4MgK0smlQlbcIaCSJaSNEWhMMF5xtBEeUoli1JyZGkYlkIXWmuiQARRGABjQRCEnMdRpLRWtBQKAwxaaRJGYa5pXsrZdDocnXNOSlMcJePxKA7EcjBYUDJXwcEie342Srvd3btfhUc/bTfi3c3NWZa1u/1mms5m8z//6UeXdrc21tcbaSOK4m5/g4uw1Rn8t3/373x+//7PPv70+YuDn3366LNHz9vtZpqmoRDn56PjszGxoN/Lnj1/lufZaDx69+13Xnv9zVk05EHQ6nTSNDY2iR94RCzLQmmVNBp8eEZaTaeTg+PTQukwjERARZEHQdBK404r7baa/W4nTRs729uv37nb6fbm8/mDh/dPT09Hw7PDk5Nllk9n81lZAuPrnSZTajiadFvpG6/diKPw4Oh4/+B0sVwi40iaVIlRqnuX8iDWRKwVLqOYgvCDrUFDMLtEAlxhkSYA0OjyR+i8bOeMg9PgBGZz5QqeF7LzVsVXBSTg0KixngL1J5CvKfF1JNYJE7lUGiBENJvGRoEIAqGVLgoqyjLP86IoCynd/olIDAWKNArDKCzK8nw4jqJAK2X6RESkNAdExs3m3DlpxnjKeBSEXAhkGHBu1tAFXGRZwRDTOCq1XshSaR2HASIulzkHxIRzzuIo0gCKdFnK4WIZKcUAGJFZx7LMsqIoAVicpmmcKk3rrCCt3mrGzy5tb8fJrTTak4vR9LCb55PZdDqbbm3u7u7SycnJTz7+ott4trnWXet1WieH3e6A8YCQ3755+83X747Ho4OT46OT0/lyeXR6tpgvgzDZXONBIDhjKkrCra2drc37D+/duv1amqaL+Xw+m8RJHLrAjYkYSqnLstRKN5O00+loXYIqTFkjB9AIAedX1jtvv3b19dduDza246ShlJ7NF188vE8ERZmPR6OyLONAbK/3F8uciPphGAb85eHJ6WTa6zTfvntrc63/yecPv3i0BwzjKGrEkVJqHMezsCUBismIdbq6O9jttb9zbeeblzZMEZyBoS2udKGjClPkYkuVUehodCWf5DJGfim5NxGoxppk1qXZcGdVPuLpuYZUv/xJMGQCgQPEIoiDUDCmlVpkxSLLZF4URVm65w0CAeMsCEQSBlEUxVFIQMu8WGY5YywKhFSKI3BEYkwASKlyJRljwOxDwky6qCjKQpYI2GrwRhopqZFhgLzFmAnpGUMCOQNEwThnjCPjyErGllmeSdlqJN1GI4lCxoRUMJ0viIvu2nocJ9lizri41GhebTY30vgX87IkosHOsNFqqglNZ4fD8635NIziwWDQbaSCY16o+09eaHqOhBpZVpaMi521QavZGAzWu+3Ozsb6+3fvtNrtKAyBdLPZ5pw/e/rwj3/8IRAxVKAKxrlWeb6YzecN3mpxt+rVpCzKImdcrO1cSRuNxXQEpJMoVFopJYNQ9Nrt7fXe1uZalKR5nj14/HRZSMFwNl92O61eM+Wkx5MpAyqKcp4V/U4zK9XR2ehsNNla79+6emm5zP7khz8fjqcgBBdcE02XWSGCIu3LuMWJxSJcNNuNVvPvvfPa24OWts/F8LFO6747l8ijzmpdr5LNr7YcqQqLmrVrNude51FwgSpnD9i4fc30rEHfpuy9WUEAIBKzQJMzxpiUSipVFHKRLfOizEupTZEHACAILsJQxGEkhACGZu2Hto/owjAMmZQiECIQZHZFNAs47QNLqCiLrAQkEIwjw6KU56NpFIZxFJrWFaUkoHYYaKK81JyY4BwZkq1W5rIsA864EAqw4Gyt2RgMBgQggfqDdWBcI2s1mygEMZ4myfsxa0fF42UhARut9uNRHNJxYzhuPn/ywZvv337tjSAMHzx62Gs12u32+WR6dHiyKPIwCEopi+VCKl0qNV/mUlO/09pe61+/cmVne+ta2mp1O1euXH97OPzxZ49PBZyeHDTb/SCMhAi0UmUpMQg4Z1KZ5zyiCAIRBDQbHx8dnhy9jKKw0UjNyqxWI71yaTOOAkXs6fO90TSbZ3kplVQqTeL5fCYJDk/Ollm2zItBu1USHZ+cTsfTNIluX92+cmn7fDR7+vKIEBuNlCMCQ5HEQ+JjasxEpDUIpEyEZdL4jWs7d3pNpezTZRjaLb4McWinbpHQrn90j0YgW7hJFd25Pbh9QZOPENmNh92xRABGszNCMFstmZXyPk7lZ4UDNyJp8/RNJoihCEQzjUqlZ8tMKkVSyUKWZVlK6ScK54wLlibJ+qDXTOOiVOaAQDDGWByFYSgAKImjspSkNZAG4L7RpZRJFEVhsMxyAkriWHBuqvQ5xyQOT0fTWbZMomhRlouiEARJxBhH8+iPOAozKbks0yTBQDDGm41G0kgbjWar3Vkoef/h/fW1tfDS1SBthWEYx2kaJw0ukijuz7NhHh8tM6m7i8uvNYvpXM2/ePTF9cvX37j7zs727pNnT6aHB1EQ7GytS6nSJELGxpPZ0dm54GJr0Jgvl9li+ej5i+f7L/u99uWtjffffqvT6TaaLUbqP/zpj+JQ/Pqv/bUgbDTa3SCMyrIkojiKzJOdGGNRlJRFLou8LMvpfPH85cH5eIqcpa1m3Gwo4MDD0TRrNptbm51SytPTsyd7+804VArSJO60GuPZfDJfjMfTVjPZWetfeusOIpVlOctV1Om8s76OnJdSglRBFA7ny2f7w6kGPZ/xRhMaretrve/cvPyt7QE3qCACArMPmNFt2kXTbVCSmXXrZLFrF0Rax4WY21KZ/JcubumIcBVyaJU3+WfBgLbEbbzp2i4SCKBtCh1ACx9r1ZpAkyxKVUpTzImamA07oeA8DII4DJMoSpKYIDfPfmPIk1CEUSilBC0FxygMi6IEpavNShyNB5xJIRhiyJmJkHHE5TJfLPNlngsCXcrJZJpJRQBSSq2p0UjjCKMwCAMRl4EG4IxHSRInaZw2lmUZ5jko1Wq1ev21brvfSJsiCKIgYpyLgG9E4XY7PZ5l/8P98U/Pxq2g0dncGBfT2fDw/NGD13Z21vtrX1vbHA7PHjy8d3Z6XBTli+MzZCwU/Prl3WWWCcZuXb00mU4WWW60Uhjw0+GQiVBr2txYOx7PHz7f/+rJ8fa1O0EYg6sSV0qZrfUZwzAI0rTR7q21z06KstSAZSmTNO11O0kcD9YGu5ubWlMYiIPDw363szYYrA36Usmzs1EYCNB6d2Pt2tbabL6QSm2tD6I4ZoyHTZamjaTRjMJQCBEEotT044fPPj3NZixhACXwZdJs9/v/zVfu3O6kUmm3IsOqVJfStAkkF9m0jyFgTvGiY04bM9VOudtfccUUdTFOs9bXFX+a/UIQzO7AJrBktxAnv1sjeXsDzHVQJEEQBFyRVrJkSAhUSqm0AgNsRMZYGIpGo9FKkzAMlFaT6VxrzRCSOOIMw4AzhELrUirQmnMQgmnFUGvBuUkpcM4LKfPpjDPebqSc86Isy1Lm2jzNjIBIa5JSMc6QQBItizIuyjhWSimgoJMmnPFRlsWcN4MgkGU2mpwul7rIhRBr6xth3DBPQfaPQDMPA+GcDxrJN9Z7B4tsP5fXuz2gzo+1SJbTfJKvzZ53A7bd77/15lvHh3v37z+4cWmLiWCZZS+PToui6LYay+Xi9du3AHC+WDKkJEmbrc50kSuFUbP7/lvpjZ2dwfbVKE6iMOQctdbmvsiACq0BASAIo0arc/XGbRGIz7/4vMwyFoSFNlngQkkVCN5I4sFgsFjM+73BxtpgPJ3O8jLLsyjgj/YOllm+3usMep0gjNJGK24kcZKKMIqCQARBGARpEP3RvUd/9uwsU6AZQy0LQBknX9lZ323EZSn9okskF9OxhGdD9I7rwJdtvLoLQ5UNclamJrdpElZ4dlFL912tGMXfUbsKEp/uJxdh9ZcXaRqbBdFKk1IkpVZKAyBnLBQMAJCzRhy3G400jQMhgkCYwATnLBAcEaRSUkoE4AxKRRwxYLwAaboXBgHj5rEIyBA5Z0IwAiqVUloLxgRjRFSWUgNJJakEHohIiJBzEy4lIE2UxkkUJ+W5WmbLWbbsNZvpYIDIsjzvdDqETKkiYFwwIXjAAyGECIUQXCBiGolfvb7zznr36WSeMn5/NHsskVjzWr8/oTxEdV4qmc0VRrfu3M2y5fD8/PD4NA74Zq+fxmGr2cyyRac7SBvNQATIeRhGZVEIIQbrG+ubu5vrm0GchoGIwsCg02wdYB5Hhpo0IwKKoki1ujtXbk6X2dPn+71Wc9DvP3m+LxgOh+e9brcsy267vTEYKKVe7O+dj8ZxGGQMO+3219/pLpZLBGQi6G9t9drdMAgwEFyIkIkwDE9mix88ePCn957OxtMoSYK0PQui9W7rl6/tfP/yugCtwe164BBWcWkNg1U5nMv0XMSo2+HLsW3tdBsBUB5gRGif82qPMgVw1oNHxtzTFldaUd2USJSlYghFWZalKopSSUlaAxBnLOA8CAQXIgpDHgiNEEVBHEetNGmk0Xi6mM4WjSTkis2KQnDebKST2VKqAt22KUrpLC+5wDAMm3EiBEcAKdV0nmmto1A4W8fk1YAjk+i2M2VMCM4519rmscIg3FlbW5ZFTsS5oEBESdJqtjqdPiCejofnozMUQgRBxCJu8w22poUxdrnb2u0081Ii4neW+bSUgzQBikZSPl9Ot6L+7e3rnTjI5tOjg30M0pBjK42iQORSLubL0+EoEOLq5ctJo5PEyWQ8bLQ661uXe4N1Yedq9fKCNirerOIXQjAAQtzY3P7at/4SkCpm492NQRxHjUaj1UgbaQMZV7Icj0bLPAsCZsyDRqPBGaSNZpAkzWY7abYE4xwZFyIKglme//aHn/3kycuTyYITYRyp9c0pD9rt5n//3s3X20mplNlmwCfdq4Q4QR0ZxrmhVx7ZUYcsur0U7ctlgcgyY5Vk8rGnyktf5Ul0G+RSVUxSJQQsgy7zXEpVFEUpldndXQNxxIAzxtBYB4yxgPM0EEkUCs4JSCqSSgWCM4b5oljmRRoFSRwJIUaT2TIr7OITTZokAYtjJCLSVCqV53lRSkRMQtFI4rJURb4EAEWglUa3mC4RnHE2XWStNBGCL/OCcx7HsSY9nkwyrUnwZpJmSi2yBRCAkkoVSpZSlrIouQlMuIcy2NAbYhQGdze7m610nOVlqf7J88OfnZzvRmEqdNqg87JoBfHV196+eecttZxl80mRL7gItKKz8ZiLsNVpB2EcRclWu9dsdZqtVhIFRHY3NT9y5Ha7MMoO7eNNkDGWLeZK69u3X0uS9MWTB0mcaFVMpzPGxXyZtZqN8XisVTGbjJZ5SURRFDXTWCpM291mpxuHIRdhFISc80zKF5PpD+89/MGnD2WhkzSlNC2TRrvVvNZq/N07V262wlxKR2UWMpqIedq6sBWtz0z6b5zC9Y9hJecYgUelx1dlJfhtkMDFmHzk0yWrajEpD0p7fMXmKMwazbKUiohzzpnJypvHwHDGWKfZ6HU7jKHWWkq1zPLzoWy3Gv1ui3OuNCVpI4rj8WQ6mmVpyFuNJMuK5TIDIrO7PCJxZER6Os+WeaGUCoQIuQBAxrjSpVkehVpLpYxzKBii4IJxRRBFYRRFWmcAtMwWw8kUlN7udiMRnI9Gs8WCtO51uuvrWyJMSOvFcg4MwSpaBAROYJ6hDAAILA3DJAg3W/E8K7+fF5+ejz4cTr53aTvH8P5k8uenZ3ca6dfWumtR0G302o02B5RKNjt9IijypeA8jJNGo9Fstkw4QrnHibo13NVScbtyABFAm122GQBqzRgDoss371y5+fpyMjw6Otg/OjkfnXXH55PpjDMYjsen5+Oz88mdm5caEev0NgbdfqPZYkIwwNPZ/OcvDp+eT6bz7Gg0K1mkG0L3BnGv/feu/f/o+rMlyZIkSxDjRUTupott7h7uHhG5VVZWVVNXD6iHgBnQPIBo3gEC5g34BvwM/gGfgEcQDdEsBAwGQC9V1VWdlZWRkRG+26aqdxMRZsaD3KtmkdUwD4rwMFPT5V4WXg4fPvzVby63r9oqIEiS9eaDmZZJtMXqcO2jL4sCz5s5F2P5qWtdqypbS6G14F8fDwa6eoInU376zvk5f5oUnL8DzxOJ818QnIhIFjNTs0DUhjrllLNUVajrqqnrF9f73XYbY+z7fhjHw6mPMaOmytN2u62rqmubKoTHY98Pp6E377jUirLMlhISZRFEFDNEcMQIqAgFUlYDdq5sMiRGWUIFRrXDNAHYPrZVCNuu8T48nPrb02lb1QTWD0dE2m02ZspIJprGU0ZsmdM8ReYybksMZeaPCNakHAHRgdvU9F+8vm48/4eH056pYni92VxO+RZgBP9/+3h/ezr8V9fbb7rmZ7uNxWkae4QyEs1t1zVN45gZ0NCyloKddP0qZZ/kDIgqkkUkpTgODjE4V4y18lVVN5dXN9dffX3z6cPf/Nv/+e7T+2GaP9097rb7f/WvfnE6HrZtuHlx02yvBoPvP36eRFOW//jpy8e70+NxyEiz97rZpN1l1TX/21+9/a+/voKyurTcZX0yBV3ReAB4GkxbvSKep92f5G7OXPlnCOjq7J5DRmuc/4krPLtMePZry/fXTtEZXIVnwef54x2AiSkCsFkZ4wRGw+S93282222nhnePB0fETHlSUfEes8rh2BvA5QX1g51Oxya40XHfD+OkCxRa8ClA77z33gePzI6dihASe1cXWhoiOTYBImYkyQIAjGgpZQBm6scxZgnBv7i63LTtpqpCcOTctt2GUDVVU9WNCyFUlfM+isxTX9eNI1LNBm7dbA6FUbMm6GYITLRv6v/q65f/xVdXWXXMcjvEl7X/3eMJQP9st/t3D4f/6x8+/+cvb/5Pl9dX2yrnzM5V7abbbB17Wq8yEeGzNU5FNa3cGCJWMBWdx+n2y6fD423p28XH++bVW2YHgCbK7Jqmy+S7y5ff/Hzz1z5c7ff7/X6O0zRNYvBpmv/m999/Oc1D1MeYT1GOx5O1Lex2v7y5+K+/+UoRX7Xh266SmP/UJnAdVl/xyZ/8+MlOFgqIlYJqJcHhefao6CSWeLBu9LRzzf+svD9nq+dS60986tkuz+ntT9/RU4MU/+pf/2c5JiYyBO/c5cW2bmoiRMNhTrV3SbKZtHUVHKuqc9y0TUwionUVNm1jAHePh3GY+2Ho+2GcYoEARQSJC2Llq+Cdd84BgKkiADM7z6I6z6l4NRGZVdXAOdfWFTNVzjd1aLrWISFTHcK2bes6XOwvjN2YIhEz8367vb56udtdOKJxGoi56XZVVbVt530IvmJHRMjEZwNVW+5ByYzLPTKzJDLFPMQ4xKRAd1P8/eH4bVv9q9cvGMQMkNAAq1D5wgssI6qIolo0L8qstYquECOISBynT58+/n//5t9++PTx19+8veyaerO/fP1t8JVzzsyyZCISyTnO89j3p+NxOM2nByb7YdB/8/EYqtrU/vD+8w+Pj7jZNCFQ8JuXL9/suv/ml6/f1F5SFFEp87PnvO7ssZ6F1zOQRM8IdIsRPWHvBbSk1XHqc2dXhnIXaPOnVCZ4eob15VZ5nMX6n9niUm+cS66fBv6C3jsCLLl9XfuuK+0ev9+0TdM8noZxGAwElFUkqjim4KpN2zZ1gwgx53kaJaUUo2pi1F1XNZWTnOc5ncZY3piqxikmSiXfR3a2Fl5N8KYwx5iLwiIAOmbHSOSd984F7755cSkKv3v3IaesKlGaUNUxZwDYb3fsmENQYjH17HeXN95XROwcEzsih+eW7/r5yxxp+aYhlk0CpYBlotr7XVPFlGeRbeWvKhdTfJimi6aufEkVClP+qVwoUr9cemNyFrOyJTE1SPP48e//X7d/92/VwH/zutle7l++9t4jURl7LLO1atb3p9PDl9PxkCRfv/rmwxDf3304jPnd5499TC+ur/7Ft9/sN81fv9i/6OoXTbP1bCoxRhF53tdZzcbwJzd+ocqXHy5nSHUZdysx2sDOwNCyz1rPiaeZrH3IpTyHM5z6HMuEp9rombU+Xf9ztPmJZf8zVwoATiQnySBgYESsBsd+fDz0Vxe77Xaz23aS0zBM0zzN0zTFpKpJtKlHQ1DReR4rR7vWU+sB2pRlnOLjcZxjVkAFJUTnnBGqaYxxRGyauqoCIqpaElsELYiyqWNum7oIh1TEjsv7mUqqAGYxZdPps936ENq27XP2RNL3OUvgV5tu50MFxETsXXDkaBlQwMUsVw+6gEErOL36FaOl9c8VEHEOTJvKl4iDVPaewRnVFhE7a/QjYvmkZjFGU5Wph+lR5lHjIMPjXj/u6/z6z//zb37zL0PbMTsEMtOcc3knagqI7XbXdJuLsf90OPy3v/3+//39+3q333399n9/uctgPvhf77tvL3YOLcdoBiJalpg991tncOccggFWJQZTPa9dsfWzI61UtMKvo3WebEWOFttRe25ATz96+tLnNvccpTprkDxXznkOfP4JDQ8XSVCX8qKxkFIepgkASgU/9P23377pmqv9xe5yv308HL58kfEY72NOd0cz846DZ+/Z2uqi6rZdE0J4PI2H4TaLFIUwAiBmdmwAqiYqZWJE1Zo6AELOOaU0pVQkx5tQX2833jskSillk02oTsN0HMZ+HMcs+67dbToX2NfVwzTaPF1d3wR2TG7OklL0VeXY+TX4nj/tchWe2eh6J88/X442AhoCOfTK3rkCo5R9dilLVq29R1wWlqrJOW1YdH2ZQwiZSMbD9Om3px9/9/7LXR/zv/njY/fmN/+rv/5ft123CAIAGigVGFi15ImE+DjFv789pZR7F/76z3/19dU+OPdmU99sujYEzy7OUxIlHyQmXbEtldXPrUofsPpme/bZy1b1J5+3atTDsytiyz7jc320xnAzwJ9UXWcwAM4z9ef04lkqCQBL5loSvAJ0rK+x/tbyeFsxWlM1RGeiTOSYpKzEEgFEXwUzBdE5xpRTzHmeJmJu60BE45ymOXrvura53G+JcUpJhxiSjtMEZUAewExs4cIiOzYzzJhyUtMpRgCrghvneZ7nLALMRJTATjFuEKtAiuCRN03Tj9M0z/00j0kc04urfVVV4N1ls/E+7Npt8CGKfL6/q6qmabeeHRKKqYoxMCIy0Pkq/gnA8bxNcjZeW7zpMhZgBqpo2ZjAg0s5JZGCyzt2pfnMzFYmxBEBkMi53Qv38/9y5lcP+fcc/P/mX3/9s5//AgBNFRxIzmpFLVBzFjMTk2FO/3D3aEg/TvGvr/f/y59/TQWmAphE5iwofUQKvvLBm6mrUEcxMwNDwpVe/FMYp9xwNABewaDF2tYEAJ9V6XbGJ88t+OfX6yz6dfZ/9icQ0k/h0jPXCcpg8bNffMqPn52f9VeWH2HZ8uEdt7VPYqfTwGaGVFV12zZZdZrGmCXGRGg5ZUKovGubOmZzjKI2xtQ1gYiGcYoTxJRiytOc5pimKABAqsEbm9ZV1bW1qU0xBc8IeOwnUa2r0BJlNTXrnHeAKee6Cl9dXY7T/HDozbQO4fbYm0jtvahNKVUDfH/3eJqnfde9ur4JVT2mNEyjmb568bqqayT2zpV1dZIVmJAUFjbikxN9ss4ycrAe65VSbss1ZPToTDVlISRHJiIAEHMyfXq24reZCMBE1MjdfPvr/auv52EAg/F0ZOeWBjRYTinOkwHY0moWVf1q01SM/+U3r4go5bTyiC1QYdMyqsVpBEQXAjnPdQ1xKpsqFmVqhCXXXM+gLYNrslok6MpFUoCVybSCSvj0uz8x8sUB/7QGen7az50nXSdCi8iSrvSZf+Yd1uPzU1+7Pn+5I66E/zHKPM9JBAG9c+wYAULw3vuH4ynH6JhjKuFYg/cGIIRiMIxzP4ybtmKmrJYVH09zP8aYxMBKnatmKcasmiQTUhGzGeY4zjMRBl8jUZaY1Vow59gAKu+C9+OcssygOsWoYNttVznu+8EBTTDdno51VZ/6McDd5X6bU/qcZ5QkOb1587Ou3SCgmGpKhAjgGdBIi6h5cRN/0pb8yd9XCHAx3GJUSyWUAICZk2QoU+OmKUvKOaYcswDiRddsqkBoKpkBqiqkmFRyShEAytCgmJUiScHMhBBaz7vGm9nYH6umY3YiYqZEzAgqeTktiAYWx9Fs4KqqNjs5PKhKoamVxPOc0i1TFiWs/0n+95wyt4AbS2IA//zrmRn9NBNdje9pz9ZPSqP1kPynn/L5v89/ef4qLqacsxiAqjCRD2G/7TabRszu7h9TakwtiyCAqExTfDgOjOS9qypfV5UBJDWiZrvZpKxTfDRAVSOipmlg3XNvavMcU4zBeefdlFKMkRCDrwxwmOMcUx1C19R1HXKW0zAOc3JMSKhIWa1mDoSFGHCcJu/4zeVVUzfvT4/34+Cb6mevvjLmKcXD2L+Ic/SO1ZUYXSSYy709ry1dGymFX4uwEnDWS1pu6PMEoCwsKQ4TdV1wSoimwoxmnFUt5ynGL6ee2P3m5XXwFYdwru6ziKjYIseiAkk1g2RclpOjipRIG8eTbzbsHIBJnJmdcz6ntBbLiIg5x3gY0bmq3Uz9oUDITzf7nGqDncP6E1b0nAXyVPuvlfXT7z5Z0p8Y0/l4P9n3swv4E2N99n179rpPYf1cUK1GD4tLB7eoQSJ67+u6Ct61XV1V1TzncZol5xB8FeoquHg4nYZJsjjHAJpFUspV8G1dEWJKaYoppdw1VYoxiTJRUwUgGscxQraUVCwDFpkxQHDeOe9hcfJIRGoQYxpjUpFN21ShBbBxijGnnEWdtCH4uvrx3YfrrmWik6S2aVTs/vHovPuLX/z6Zbc9jv0/ffcPX7/52f7iCpC888ysYJBVCQtCdMaqEZD/tMW8LHt8usSLk1lqj+W/ts5IFAo5oGOuAUBVVP7N3eGHMX11e/jXl1002FeVI9xXTGZieppmBCPNc85q0PJyYM7VNxJZzpojgnnndZ41ZyNi5zUnWyoMW1DXcc4p1t12Hk45zmcDWe7xWiTBuj4LdNlZ9RRe4dkne2ay+J+quP/ETS5G9vxsPy/V7Se9ejv/FgA8yTTj+QiVN4DP2vSufKtMZgbvEbE/DcfjKGpMOCC8vL7Ybbo5ZyTynsvT5Gzeg6jGlEx1inHTNuQcEW26RtQAUUQNQHImxMp7QkwxqSqTg7KHE7BspSGzxlPFfOz7snq3qiomFlm22agIM9VVcMEzQMscQuhTvOz2L/b7u/7YbS7fvv4659Q/3k45TYCB3t3df3n56u3F/jrGRJSpiEgWimpBU5jOF3ptsix1/p/6jKcjjcvC6me3BwGQEM3YqPb+b+6O//Oh90T/+Hh4xfr9FBX5v388/pfb6v/w9gbRpjkxU60xiCA7pIAKBEiAyCQigIjOaRZEyoAUvOUMZiqZvJeUVESl7K5SAMhxHnKuuo2Z5TQv1c5P4/jySc8+71nyd/aKf4KcP/eUPwEvn2OZy3qPhS5z9n/nFupT8vDc0FcsFs7Txk+3oPxweRtODcCUVFRojlHVUs7MXFUB0AFAVmWmCh0SNnXtKM4x5ZwIzZTBOyIGoKyGWUSECTdd44NDs3GK4zSnqGZlVcgCthXV3PJeU85ZNXhfVX7KmR233hPgNM2gIqJTTF1dt22zbdu2CneHY+29ZiEEVL3aX/zFr//ix88f//7v/v2ubX/17c+96qfbL/3D/as3X19fv5qnYSZyzjdVretlLUwSXdZFPiVIhShz3jyGz6KkWfETVnrramWcwyGAmBVWChL8zcPx//JPP/51G+7NDlG/3rV/cbX9NMt71e/j9MMw/aL1O4ZCMDRENCWwlShgZlbu9NJKIFLJ7JyAGJS9P8Leq6lGUS1NAUNEVZmHU2g6U805PYvuTyX2k6vDn9aK/0nk/HmOuJjPfyIHff7M9uxXllPx01ThbNn2TGvkWSkKz3xBeaQ6RhOFnDVJpDkBoqFtgru62LVNHTyHEGLO85xyygjqHBfQNKs5gizq1LZVqOsq5ixJh3E2haapnHMGs5mVTiCYLSo/qp658N3LB2UiAFC1yrlN2wKCaFl1h0l0ivPN5cXLywsxm2OapjmpxpxCcC9vXjSb7T/8/rcPDw9MBETv775c7y9ev3rddFvR/Ic//Pbm5evrq1d1qHNOMUUicqX1icDEBaImwmXh5wqYn1O05YIut2BZn6cqktMiwAQIVNaRAiK83jT/x29fvQ58Efz7ab5pm5ppV6X/c3s5ioGJrt2dxVkgoxkSnVsva263JMdIBGbIbJKt1CIGRFSq/udmJDlP/bFqOjU9Q/e4OrY/MaxzWY3PCEf/3PLKd/SntOXnR/d8ifSnRmk/bWL95JntKcdYar4/qczMEMpaV3DnF1109wi9D0Qcgt/tt4FJVO4eDsdTn1M2FdWi5oPM1DbVHGWe4/3DQbRjQkfo2naeYozJnOaUwZQIELng82qaUo6iVfCOfYwp5VzsJYvsmrYK7jTNlXNNVT2cTuM0eefrphHVEHwWNQBP1Hbt22++3u92f/junwjx1cVV17ZieuoHR3R9efPhy/v+eHzx4tU4jff3n0+Hu+BDaNqqarNkMHPOMxNTEThjWIC6Mz0NaCmI1z2OCudpXVVBRCKGBa0D1AWp/qpr/3c/r2LOIvLtrs0iamaKBNCSLroyZb+UKi2eA1cMdgUOVo9ipoisJsxcFqmBQc7RsFRsT1VR+YuKTMOpbrppONlPgzucyyBYq/pnEdz+U2X4swACf2Jnz//yp988pxPPDH1F4J8euqYNcnamuLQMbM2jABDc+vaW9xB8aJqanTv1Y9vU3LUx6zTFOCciZOcsC6oF75goxtSfBhGt6goYL3abtq6c901dDcMwxwQI3jvvuESWmJJkFVEtRSuloj8OZoTIxMMcYxZX+kDEu64jJMd0s992XWuAn77cmhoEt7m8PPVj//iYRXzw1xcXTHh7f5hVfFXffvngiH/1q9+4qvnx3R//zbvvX7/86te//ksFNDXvfZYMgN65um7qwGAoubD40QyQjAgXPbbiHmRZAl0KbVVl9lg2qpSiCklsWUdmSMSsYAWHN8kgGQqOCbIM+C5ZAcIiH702s57d0eI/EUFFDQnLqkIEKEGc6LkHPduf5jyPg6uqOI5PdnNO+NZHPuXW///tcgm16/88fy148r7rz/75V7HdgsuuvBRcCqeVFbA8/RpU7CyKvLycqyqfs2jZjVTOlSoaSJbbu8eYEhEBYt02lXdt2xDRPI2HY386DZIFCduqqqvQeN9WdajCNM0q4p2TnHdd470Dg893j9OcVMqlpuBdCL7yzsxSFlObcpbj0Xned93Fxa5tmxhzziCqlXfsuGnqw3E49H1m3G12P75//2q7/ebtm8/39xfdZrvpTv345uXLm5tX4zwhNGZ693D/3bu/ufv04e3bt7uLq7svn9i5bbcLdU3OO18hoimoWRkUJuaiRBy8N3zyNabPTz5kk1ITMJKBZRUEICRFxKJVbFr0xtY7sCwJBwAgWgzCAEyRGJjOFrkUYmaABEiw2C8qgErGspDKVERyTshsa+b3VHQDGEAxXx9CnOcla1yhh+dJ5BKF4ZmbLAH9mcnCszLo3Jz8yZIk/BPztBU7sNUfnst8xKcmp9kqg7OEnnOGQ7gUS6aA4Lz3gGh6pt8Alu2DoNM0pRQ3XXtztd9tuxBCU4WU8+3dwxzTiupp8ME5J2qH44l6RMKymogdq9g0pSmmw3EYp5glI1IdQlUHJEQEWYhAmFXnaa4tBBdP/TjOaY5RRMukxDzHUz9mkV99++0k+o/ffQcASNQPQ/But93WVY1I7WafVU3leHzo2s398XD7+cMvfvHLN1+9/fD+B0kpSmqr+vLi6ubm5eWLr3KmmCMiiuSStIhKW7dmy+7dP7l5VpryOT3P/XPORGRPHnC5E8GxY55jMY9VMw5gqYfKUB0TIi0M6oWvoaZGCFguEK76mrYM8Zb9vJJTGbnS8zv7aUSO8xTqmoj1zDwqkO0/QydW612LFvhJunnuaCxpYvGChGtg/6n80tmnmhmu45qr08ZlKRIuj1yxvifnamsf6+ltoWu7ulz1OeaSJprqtq1D8LcPh2makXC3bRihDc4AjqdhGoeS5xJzFcK2a4eYYsxNHerghnnuh2gAqtYP8xxTznkYx5SzAniCUnk6IAbKkMt+PAZMoimlw0mGYfSO67q+vLzo6qqua+e9iDrvFPHh/g7MJMs4Tv0wusrv2o4QQ1XFnIbTQXPabvauasLh4X/x1/+ZIf0//6f/cRrHum1eXl7P0/zv/+5vfv6LX445vXj5FgFOhwcFaJsu5bhpd6aljWmIBPh86LtsQxEwOCvbqMhiAbgkfLisMUFCzIpMWSEX+KZw+emcz3FRElyW/5bKHckB6vMArypLhgqmAMV9iirquvZgNZaz5yv/TvPsQtCoiya3mf6pr9OnxPeMjuvq+nBFJp8kZg2QymPO5rt0n3A1cQOAZ0UPnh21mpbjtiAUxZSfoCU9E/cKVLtEfbdtAiGa6mmIc4wiQgRq1rX1OM0jWOUYNR9PhxxHx6wpgWkd2Ll6jknEbh8OKQsRT3MkRFObYxYRAyiiWVNMKWezMpkEWS0vxFoBM0dUh2rKcY6gak1bFWYTMgfvnPPOcUppu9lcXVz88cN7NGDmvh9yzlHFJPXjeH152bXd8Xg4PD68ffMN180PP3y32+67uv3db//+ctMN3h2n+ePtF0d0eXGlCGL46dO7dDoS49ff/ll/ug9VVzTzUYt5aBlqOk9xmC5pJYADMFERlRKZVcW7gIi67ODFcqOXmT01tFX0dTW+NYiDwbLaGaDQn/mc262RDgFW9TmRnJJIXlb5LNT9n3j6c7sop0RMGvMadRcLXbHes/tciO5qRgC2KC0v7Bo4O0GDwhzFp3pr9ZAGy/rDAmuu45rL1g9bX6BYvhoiLU9sBgiGeB5GWf1qWbMEjtillMtnzCLjlBCyqJU6pq5CcCiSEbya5iRMtN20O9B+nB8lDykP46yiVRViygAQvKvqKqUEgDGbyawqRUFHwdCU1yn4kp+54EUFVStedm9Xwfvg99tt3TaO0Mz2uwvv/e3d7b5tQXTO6dgP/TgR4nbTVt61TTPM84/v37VNQ879+Mffd93m1YtXd7cf97vdfnfhvD+ejuPQf/funQb/6tXb4+HxH//hb9M8/+qXv9psvzDi5usLW+jwGgBKwnQe1zSzLLlI8SAWxExKTV6mPNTUMZuirtt3yx94qpiXe1lo+biMJwJqWT9ugIrIT9j1spUFz65pdWUl2cgiWsYTnu12sXKiYDFBU1Xisk3oyX8aPmWOCzVkhYMWuYViOSXaLvSi8jA8H4enHGEtXlbDXtBiWIujMztpSTRxbW0iAFKhv5XsG5ac1QxBTMGgsMUwpTzFPIwppaRmpdSum6YJbs7qxFqgqqpTzqhKYCnb4TQ9nsac1TEpEhFLlj6mXIfgPRIP45xSyqkUFQtrwS/zzMhEazqiKaYk2QCaurrc7oCgaWoAPB5PhHh9eZEl398PouqcyyLBe+94itEHf+yH3/34fkr54vLCVC8urv7p9/8Yc7q8vLn7/FFNAen7H38AhJuLSzP75S9+WW13t18+f/jxu6ZpLi4uI8C7d38M7FzVtE2LzE23sWZjZt45Ji7WqaYxpUIzWLM+IEIClGWUv2x4fAq7Kc2LldiiGgyICFRi8AK7qqy3r9w/QeSnnK/cZVNb3wPiUy/2yb5K/H1WERdvVbykFZBVZdGkJVp7Rc9tbX0xfIrQa2pamKZPFVUhNNia/xTLfNYNwqdGkT0p6BWPWV5DdS2zSule0tmnrHRx6mbm5pjAICbJWbxnx6hmIjpPc+UdVt4MuLCO2c1JYlJTGcY55cyIGUBFDDCmOIxCCETglqkIK8a3MM8BHbF3jpmYmZ3LWaA0UQC9c9McY8oxRwyBs+Q0A8DlxRYMPn6+jTGG4AGw8h4BArHv6ix5irHThrxDw8vrm4+fP8U4v337zd/+/d+Nx9Nm2xnCOIw+BMt5f3n1+uVX96fTH7/7XV1XQz98+fLl17/5y5TTOM23/+7/4xG+evP2xatvDv5xv79SCYSYVyGgkuQtbgnRyrYcRFEEQJHs2CEgM+esxWmBKgEoMZqpChEXB1xIXmaiOWnO5D0RQ7loy7xlua1lcGjZdyo55ZSW8FeOBDwVFksNshbnZ1DIRBbB/GJuqrAKKp01vNbzsox+PquknhCf5VydE8dzYbg693ORtuTThmWOdAkBq+9dnhPxfC7O/twQwBTPTXkwxz7klMuZ8Y6JfEoiElU1paRgnjllnWOifhBTUTVVJuyampGkH89oHKEhUk55xsjOMaIZJpEygwtgIpJyxgLBSFYzx+ydA8CSpKac5yxM0k9T63zd1I/9+HgcmjrEnKecdm0nWaY4dV3TVvVp6KcYk1ntQ1XVHz59+u4P371+8+bd+x8Dgdbhy5fb4zDGlK4uLm5uXoSmiXP89PHd5Wbb1HWeY319w4C//aff37x49bd/8+9f3Vy7UP3dP/zdX/z6L+ebPlRN1bSIxMwhVLR0vJQdn+//Ot5UdAMyswOwLJJyWu9kyUORic20JJqw4FDFnSiqAvPKGIH1flrxj5qlCFLM0xjjbGoGhkaL5xFd8wAzKN3bpRhbK2wrKxFWwGyt/JDOQfrML1JYhOXxTBxerPVcSz0VW2cY9AwB4NnMivktaTTg+owAZ3r0E3awAvzFrksaUJ4QXdt1x+OxFGwpSXmrTCiCc8zTOHnuxjmmlLabpgqOmHxw6r2oVlVV1ZXkHLOM05xSVjUkMjBRyTmrCiGWJxVVAJQ455x8Dt45dIwOF5U8R66umIgJGawJoWKXY+rnSbKodE3dAKOqzpr7GBsfvHdJNac8TdPd4/HDw+nv/ubfv7i5dsGneQ7OHY+37z5+BsQ6eCYapylnvX+4y9MUmsbMfvbNt7vdxfc/fG+it18+OUQR+b//9//dz968vf386eOHHy8vrtpu02z3l9evgg/2VJlClAwAniBpzpKYHBhkSSK5jFUgErrFJhQFiZdweXYPhEhc8FPNCZDYLbtBwNCQSxFpalaYpGnOklU0a8Yy0KxqJuUtrcjmGoqX/yx3XmHtkRWzL4iCySo3QmVH0rMVXUuye0aFDADLiponn7omjKV0On89++vK+i5+8qy2eH6Y6TM4dUEfdHXYAADgXl5fmsg4TM4bO5UsKWY1IGZE6vtJRDdtzVVIKRECCQpReWNE2LWBsD710xxz8OgcI2LhmKYsVoI7MSFhSkW3PIuKzoBQOzZRNW2qGj1TFlhTEwLo5ylnGebIiI/9OKVc12Gapl23ud7vg3Mh+K5rs5lmOfXDu8+fVPLP3ryZ+z7PM+93v/j5z/e73e3D48PD4fv37yNY03af3v+432xjjI7dj+/fnaZ5mqdfffv1x0+fLvbb4+Hw4mJfef/+4wfHdDw8MuLNzQvLOV1eV1VjAN4FE02aHbEQZ82mqiClrBbNZ0wRlq6IIdHCMTc833qThVpK7CRHS8kQFRwVfsBaPpeSSCTHeVqTJVBTNCqJqYE9MZUBDHTF/FeqK6LpQrrGRfprjcugVlQQbdnDsVq4rdH8yaDU7Mw8xuVUrMZoAAAKdtYggLM12+pX7VlDy85w/pKYPund2E8M3PXDmLJ47wAw5xycY6JpmhHNOQKAnHWaEzOfevXBVz6ktLAumqZqKp/FQuVfXu+KUZ7GeZzigkQQEaGI0uK6nxcQsQ5BTeMsaujUERKATTE2REwcswxTFMlKdNbDceyOpz6myETe+Vc319f7C+f83f3dzX7XEL37/Onjx89dXX9NvO82X79544Nnoq+Cv/7q9b/727853h/2m+7icq+q+/32TVV9/DiMw+C9+/Nf/uJ3f/je1Mbh+Ob1W+/dH77/Pkve7XaHu89//MM/fvXV21DVdbclYnauabeDSJJYh2ZJrtZUzJ6xKwzQ+VBcqaiZChgsfA4AAEUm0lUxgKQUX7AgB6o5l7nNlauiALrkkiq2SM2fsaaFS28ABEuHfXGLRYV2ibrP5RdtdbdwLnpsIcoAgKEaPBU9qGC0ppJrNQ7FjBfuz3OEoER5BSRAfXLpq9z4OUeys5ozrP6/fAw3p5QlL5mAihKGyiNBirnsPDJQIvRZ6qbyvmLnppjnrJ4UoSrLgOq6Ct6PU/z45Z6Jm7qOqdRGwMw5C1iZ6CU1LZ5D1KYYg/dokHJCRF9z27ZVFdqqhiW82ThKzoominjqh1I4b7t2s+kIoAr+5YuXMaVtW2fVx+NxmqY0zxcXF/v9TsGY+XK/u7m6urq8fv/p83g4jeOYyvQSIjPfPd4fjkdT/dW33xCiiDw8Hl6/vLm82H+5vWWmHz/effN2bJoBTaahlxQPD18Acbe9lJzqpkOirIKIiIxUGCRWEr6zUKgVEQQELHPlCFDUhnSBFYtmTvFJarks1pWcJGfNKeU0p0UxGBFVwExtAbkWrpGB4TKVtqCturi/kmguosVrP9UWSsC5vwolSVzWKZg+ddLOmWxxo2gA9KxFCSuqD2e/f3aF51T1qWlUMktcIfw1xqyyPKtnXio+QNc2NQMM3vXjpKpgSgjOOREtA8kIBGbB0aapxznOMYJBTjmBNXVu6ooQx5jnmEUtpkxmITCgiZbN2yriRAUkkyoYnrnXMSU1YyQQUJGYU1a5dpel7eoc++D6EdQ0CUA0MCOm4EIUeXg4pG13HMemP+23G2badJv9fv/u/TsQ9d41VfDMMUUD3HQbQ3q4v/3zX/383afPn7/cEyI693A4uo+fReTmcj/N0+Px0NYhuMu6rgqmO07zpm23bX08Pswxtu0WEKrgRfI8nZCpblrvAwCknAByQXKLTFcp1WHp1WBeEH7IOcOKqpSVZ7jeYBWFs+tVlRjnNOcUc86Sc85pwXdK+DRd88mSZS4pxNpKOtfpdvZYxZ2XDNcWMPQpzyy1O61Li5dAfi5h1r75Cvg/83bnBy4fakl8n2eoZ7D2J+XQ+hJrqHkq7lf/C65t6qaqnHdlGKJoVDChGaQskkU0xRQBAMmFyoNqzFlzruoqqw1z9s4R4ek0MHNThxRT2SxjANbYFLzcH2QWWMDAp/kVMyNE9AgAIoKqWVQVLrab7abz7Jhy5TgwE7NjIoBd1znnHk9HMUCDz59vD8fDVzcv6rqAtm6/2e67TkRPp36co3fsvdteXH1490PbNiGEKc4Xm+7hcDwceyQCkctNV4fq/vEQvM+iZvrh05eu3f7+D98/PD4iwHd//GHT1k3TgGmOU1XtiB0Ano6HZnvhfAVYHKLmZM75pde5msqSc6qmnBaitCgYIDGVsXNVMy3cZFAoSnwqZTvIlGJUEQNVlbLg+sl4ylc5AWujG88LiwtOuVZPz1ua6xDdancGRsumw2clzLNhkTPiDmiwHBJYnxEBlzO5ErWW/AFxwUefg/p4zjaXRPWMiRWaCK4VfxnIdwgYKj/GOQRP2MwTD+M8z7OqFsQPzAiJmFJKpVt1GmdTBaS6aVmAHAFQzNliQiLvHSMGRwaYxLIXcmyTCZxR3+X9qoElZRVmt2areZhmDJ68I+eAqKqqrqouLnZJpAm+Cv7Uj4YbSXI8nk6E33792rHLOX/6/JmdQ4DPd/fjODKzAjim3WbXHw/7rmubGsF2m/bvf/u7vp9ELTgHYA+Hx3mevPfH03BztVcVIgJNc5xVJSV9eDyo6X5/pap395+nmK8uL8dpHMbeOc7TWLcbKrJTgCKZ2BXRKGIHBqpZl9TTJGV2zN5pXsY1i12W6geITKRMHeWcUoqSs4pITstocOGWPsGEz4J8ufOEQOuylzNgeW7hrKZ3JhQtRlh8KoIZnFuQ5ycv40IE+ATWL65vVRhbSiQ711SI+IT72/qEBbh66tsvD17fJpgV7VIDWPcuA7i7h8cXl7vah8izCSPzdtMh4jQnYfHgiLmqAiL1w9SDOWYfvCrPcybTTVsz0ZBSXdcxikj2zJuuEZFhnKY5TXNyzjFRybtyVitI4HI+IYuAgXMumxkii8qcBjftd7vdtkveXe33b1++OI6Dc3y13zl2D4fT6dRHScfTMCb5cHurWU7zlHPe1LUB5pya4Ku6mlM69KcUZxWbU2yrepymLNpt2kbyxXYzTdP9wwEM1OzXv/r5m5fXovrjhw/9ODimnOTL3UPK8g3xZnfx+eH++x/et93xuh+OD/fj0P8qi4lImhGZiFyoyHtyys6LZsrJl12jOSOAqgCgqjIW78kGqqLE3gBVcrFByTmLzNMoOYmKgiosY/gLYgMLfWNJDkzXOgNFtRTrstAAFoB2NaMFzCle8Mx2IwDQ83DSmm/YatxmaKBriVtwAFh75etbOhf7TzATLm7IbBlBO9t9yS8AziO25av0QRZ/utRPrj/1cZqK0qepwhzZcagqdkNNVU650CXnecpZidk7V6ux9yr57uHYT9FUg3dVFarAwyRZVdSISAEAiZkZ0TmHiIVejkWNSu3MNCvnuPTysmQ1lSyaUl2F0DZVFbKKY3bEaMjE2+2mbRvN8vv5h9M4xml+PB2naUazT6qO+XK/e3l9dbXfV1X17tOnH959GMZxGqcpxs1mc3Ox7do6xnT38LjtWlV7PPX/8i/+7Nc//yZJ/nJ39/s/vMtqwXERsDud+v/4u9+/fvPNTHA3zH/8dPtmnOd++N0//f7h0P+Lf/GXu92pcd774EPl62azu1yKJMkSah+qMqFR3F4pPBGBijwTqJosBZNIQftTimqaJUtKpcu5AqIFk8en0LuEzadypfjFUjTj+ohFDRPXqLo6UzrDkKuV6FK2r89cCCCLn1vLdizmpE91efG0uKpEPOWWa1qwUpFLurxWTctvnmcVzxZ+fofucOwRkZ3bbjehrjeAMaWYkndeJINBzjmmVAZ3QEUEY85d8ORDyhJPvWNOIkDUNaEKDgBENKoycxVIRZBwKTOXiqFcbQAQQkQiJSRgIlRRRqir0FbVNM0GVoUw0BgcAeAUo6l556qqIsLD6ThM09QP4zAeTv0cIwIQU11Vl7utqX748tkUvOP9pr29uz+eeud407XHcVK1u8cjItZVYObf/Opn37x9lVX7/jSOU9s20zSpalWHn3/z1ePhlFL+7rt/+vWv/+z1zXXvXR5Or66vQWZCPTw+quTBeQS4vLpxOTG7qttKnGGdDXI+lFOoJmBYpmZIlZgRwETLP5JzyjHnJJI155K72iKV94SFLzQ5WPpOVhL6c1xagRxcCpW1Vl6yghUtX/PjJ/LA8s3FQa6Gu/yIFnMtAfgpjUQAKmqMT4CnmgEtT7u8UTsH8lIuPWvCrVIrT19rgQ8A4C4udmYmIsd+qELYdM2ph6oK3rmc8zQnG0a30BwppVy0FOdpZudCVQXvCE2TpJTDflNVVYzJzBoXqiocjv04jEzkvXPOaZaY5QlaKDnVgtkSoTMEQyTmdrfpp/n21Hsam354PBx98HVdBWIi6odhjvHj7e2HT1/iNCPAZtN12K2QDc4x/fHDx92m27TtOM9meLHfFRIwGXji+2M/TXPbNsMw5ZxSSh8+fd51fcy5CuHNq+vjqf98+2gizru//LOfuVCJ5q3XVz//um1+8/B4C0D7XdvUVdd1VdUBkqo2XRdCVS6tr2qRhEhZMwiBmWp+DkOrgmVcrAELyJdzjDFFyVlywem0cMll1YkodcxSzdjzruMTxoxr4vjMFa1hdGXCwTlJffKAK3wL51D/pDuiazuyhLtzeSSqRnp2eM/B1vIiq09VtIXpjE/nxOA55o9wttq1HWauqsIcU1bLcwQDFQak66urOSbJqZqjqmAi7xiJYZxySmXpAswREL1z45xEJGdhpqqqADDnxIRzyqdhGueUk4TgHbsZ4notjBANsaDRSzWmQkQG1o/jdd7dXOxzzsfHw+HY18G/uNzXIUwx5SzjPKmB937bNBLCzdXFVy9fxpSmeQ5VuH849McTAoro4XgChMv97mLXfazC7eHYz1OcYxX8zdVFP8XPd/c5JQEL3n/4+GXTdduuIYK2DteXm4vt5jRM96fx5rq52F82bRecC1Xz6tXbaRpD1RFR1zXeVz7UWYS8D3UDCCKJfe1DU6aZJMUzplNOJROtJfJiZCnFeZ5yTinOWfLC7Suw8XqMAQHLHCkUKchFXn7BiZ5F1hX3gZWuX6x3VWWCBQJavO0y6AwGIEs9DmcjO2eZZ5s7HzMzW6ux5bsrkADnR5UibMlxVdfE5KliKz+n5+gVLpwqBXBfvtwX6K4KgRDLCJvkNKecY0pZnQ/OOWYGM8mLuLo6ZmIQ7U89EGYRJsxZ5zQwYfC+9IHGOYpoFoljTHOUp+0CS9auxXmUTTMZkAwAR+2/Tz++fHndOj8hSZ7NLGdFxNMwHIdBTV9eXKRpnqa5qcOc8vtPnzdt5Z0jtcttGxinOToCIqqrSrMchzGrff3VKwNQyXUIj4c+xlxX4WGacpYvtw93D4df/2JTRMlF01cvLt+++urz7e39f/zu8XBwRDlLyrcvbubLy6uUzYUgpmIIgM75qm7jPKZ5qrqtgcV5cL5iZjPIKSKUFbumqpqTEbNzCGgGOc4xznGeYpxFpFTutvjNglCVuLoMnVpxoIqIQLROL2HZ/XkGbQyfxM/PsXaxicV+Vj+p68rG59+FMxp1xrJWu7GnX129/9k/r8H8jGqd04EltVv94zn9XTz36pLPlo2lSLp/OLRNBUhZlByLmpo6orZpBrOqot2uVbWc0jRH0QBgCuYBJEvOqeQ+KiohnPpxTtkxbdraOY4xoYGYzTHGORaWl5oRoq3Y7FNubgaqiECqHjGLHA595R2IJJE5JT+ObgieMHjHzJu2Gab58mL/9tWN815UxxjfffycYu7aZk4JweZ5ZuK80cv9znl3Xe03m1ZE+3749OXu4dg7x8H7m+vLeYrvjp/3280U09bo4Xh88+r6+upaALJq3VSmmtIc4yhG//j77/D7P75586aqq+DcjDEAzIC82YWmi8OJ5yk0XZY5zgORK+PzKmIay9U3tSxJRdh5IATElOI4DnGeYs7FenC5m1qwQTUjpNV/FUcMJfIyGhHpMhp65rydgaTVONdBElsodcsD1yQBV+7egkQ+C/5P7lCfynA4t9SfQU0FSyhHyM5euCQNYmAL4HD+3urjn3lcRCwCz+XzuJzzMFNTV4QUY1K1ENz+4qJtG2Ym1N1mk7I8HI7kfFXXh8MRwBLxbHNK2cxMFh6dqG27pm6qlPIwziK5TNHTAh0rITgmM1DVYqnlzZZ1RmoGZRmI6q7pri8vMthpmmIWFT0+HEikaZqL7Wbb1Z65q6td11R1NU2zmKWY2rpJHO8eHvt+KNhKVVf7lKtQpZSHND8cjllknOZQ+b/+q18Tu3fv3t3dH+aUuqaOKZ/6sR+GaU43lxfHfvhyf59S8j7c3h8u99s5SYrx4XD6/t2H+8fji5c3F5vN9eVlcEFyStPgQx1cmKcBCR25lEQtRxNids6X1vpCvlvmmRSZUoyScwkvjssoMzAZAIGKSjYreBMCMpArK1TtfEtLHFomTc7R3tbYuth7yQfWNvqS9i1i5uXYlEJ29R3FVlbyGwKYLK5x0T5eQACzsoDG1mhNhnbuRuFakRlgoRCUiqPwVdYhu5KcnNNoOD8MwJlZnEYH5p0rWgSqELxXEQCLMQ/j1DT1brOZY5rnOE2zmbJXAxQdC26SJZ81VVSUEERV1VKSmFIxU1szKaRS1Zdc6Xy9lgyKCrFScn/q1UxyNsmaZeyzY2q69hTj4dOn2/uqCt4xA+ChH+7uHx4eDzkmBGDvysXt2padG6f5+z/+yLxsjClMqz//5c9evbh+9+HjtutE9OXNRds0x37YbjY5p4sdiOT3Hz7fH45NXTFxCH6Omcl9fHy4eziM0/xPv/99jLHfbhxgW7fBeVVNcSq8u3kYoKpNNUsyU5mn4AM7j4CGRkhnEDFN4zT2c5wJlIJDMC4SlQAm2cSySrQsksyQ2NkSuBdsbknxCtKJiMXLrpno+bqW0snW4I2rB3sGkp9L7OdpACwgFyCu436rokLJ0Jb/eSZsuy5OXvnNuAjrruG7hHRd84zFwJfS6uyKz87LYUGFUqpyruttacE/HE+OKcZ0PJ2mOXYxHvspptzWoW3rUHlHNIzzPcI0zRqFkGLK0zhpzv2i407OOchyrtPVQAxMDVTWDKN8FDhf2uINRDVOcYDBED1yW9UnHUz0dOoR4fLiotk0fYxzSpumxhHuHx4fjsftpiuicG3bjvNUlj/d3d3NKRFi0zRmqmKIuOmaH9+9/3J/3zVN1zZNU7+8vnCOQ1Vd7PZm+k/ffTdHff/5lgjfvLz2jkNgJP/4eArOgxoZSMy3n28t57ap9/ttXQWwMig0W8k0cyRya3RGydHMnHOEfHZRojnGKaVZJRWGPhESIROqKJhoTpKipBlVaQGmzFZl01L02KIYa7zQ5sH0+TDqkvHbOadcSylEOI/4LsTNNXdcy6IVxcSlsb7a8FLvrdXQalII5yUjT8nkGX1aC6sSNLW8h7WHUI4BPU9YAQDRVVUoW89MlUodENPj4fjier9p64fjaZrjbtshoedl5crFdlN5R3g8nbyKmBo7z46z5Kyy6dqmrQlJ1UQtx5whQ1EgkDPndb1F5Qroyj1bpNUWzAERxKSufExuzDPEeDyail7InoJ37ACwH+fb+/vdZvOLb79x3h37/vHhMAzTaRglppwTMHn2cV42iLJ3SPTjx89fv35FiA+P8a//6s8v99txnrfdtm0aM3375qthmP/ww485yeHYbzdtXfm23d09HKYYv3r1IqbkGLfbdretm9qlFEXEOVs6OqXmMwU2QCgk7JJpmZloJnJLLqdqKqYCmkspC8CgIKo5JU3TPPXTNKlmACAkYkNEIzV0WhpAuojIF20INSurpwDPFQwUJfZSbi9bFQCWSgB04evgYnDnGuicGCxhzbQoQeMZJTzDorbknraIvwMtNXjJMBcm6Yqwltuu5ZlLfxSXE2Nr6xPOR8AhQJG9iCk9PByy6ByjY3e931Dt6+CHcZpj3jTN3f3jaRhNtR/GTdfKwrsBZhLRnLKIFNLUpmvbtp3mKKLa1OWNapaMshSaz5gLiFhS0sL/MUSYiySONW0NhQyKVD5MznI89dM0b7q2auq7xwfLUp7l3fuPx74fpinO8zRFyQkVyBECDFNvIuxcCKG28Jjz61cvguMvdw83lxfbrjv2PRiEEMTADC4url9c0bsPH/7w/buPn28/fbl7/dVN225fXl89+Idv3ry5utyd+tN+t7u62O/3F03TMTM8u7trI7d4NTAs1Fheot7iXiTlmNI8T4PmCCCBUAkFUFVTitM8zTGKKpqAWRlkgqzeBSVRK7RxBMDiYpTIMS1usHBSy0/hp2o2WCLVApESLCZki7jf01dxgGfwH5ah5CUZhRV4R4RC7afVny5eeWXzrYo9oAhPNCszxDO7dC29znDX0ogwV1c+JhymGLOcTj0RBe+cd5/vDs6dALAKYbE9xMq7LGoG/TAhQlNXkiXGJBJTSipCRNM0/uEPP4QqOO+rKtQhVI4fAFJKwXyClMSYyQdvBiK5iLMvufQyLqiSNKt657pNpyLOcVkYXCj6GezUD8McvePgXRXCMExfhrv+1BfB0VIpm5klAwAi8t674Nn7Kcab66vLy4vH43HbNt5zP/QAVldVERA9nfrNZkfO/ebPfqkqp+Px05f7FGOM88vr/X7bbLfbFzfXx9Oprpu6akJVBx8KVAfPv5aKBRdqE5GBETECIaKq5JzSNE7DaRgOKBlNlAwMRHLKaY5pSsuwHq2VSMlCk0cgBWQDVDVEcs45ZkQUNUYzKFL1CGcwfJlORrVi17hCAmcbKpXKgrgv9m2ABFqmSZ+sfCnRC7BQyCZlBmABws7wPuAa4RGL7hWU8aiSDyMY8FJAL2+3GPg5IzAAZ0jEzjlNKY0pEbNqBYBjnszMOb/fdo758XAcx8kH75xjxL4fYozOORd8VnXqYozFf8SYRWWc5q5r99sNIk0pOuamaUacsxqD1HW13+1EpO/7mBIIruSpBYBDNTM5HY5FWGae5mUAHEALImWJRXLE6H2Y0xzjNE3lUBZ421YADgm9dyEE7zlLQqS2ae4eHtGgl3Ga59cvrq+vLh+Px4uL6+1mezjczfOYstvvL3/182/jNLbN+8dTXyCI3c2NITd123Z7ACPi0ujLOS83dVn+fTbLsscBl5mPp7TNUpofD7e3t58kjpAjqjgw1JzFprJ52kAAkZgJiR2wA3Jski0hKnPhvSPzqrhZcgRaQCiiMv1WOA+LEdDqHBcLWRuU5dqKLFjAcsitNIBWsGhFjojwJ2b0VKcvhlf+FITnjCzo2u4ssZtwmUQ/Aw8r/Ln2HswAwJUqpq4bQxz7QWOa5rnsCTbCbdfmXMUYqyqI6BzjNEUmSCnnnLNoXdelSxRzjlmKClfpMLHjKaby5bgMLmdiUtOU8zhOTVtvNpspzillSWmeY2k5AJGYMVoWmGNk50SfWhBqCgKFCI2ISXJCLnvEbf0qRo4AzAxExFRXfo5pnueqqt5/+Nh1m66p1ezt6xfMZAbOeSJMKW67/eF0PPWfr69e7PeXqe1Cu/ub//C3265pmrqqKnIVs/fOWyljVwlZK3xXeMJ3ibjIJgMRLNW3AnAJ+ExODWOKp4fHNE8oCSRbFgETRSAUIGXnmBw7YGVnzgGgslNmR2zeO2ZHVNZSo5qgQtn6DQaiBgDFDkqeVGyjEJ0WV1lo/kvpsoA+a7fp3M2Bc0JWCBVl+zzC8wnntY+12nGxxMX4DIpYry47P+BcTj0dl3VZyvmZiz92m+1mGMbgXV373abtx2nsRzNj79i5rmsN8dOX+xBcCKHt2hxjylKchKj1w1BOmveubZuUUk4JkIo+o5l1bTPOpKJoggjBOUJUUSYMzicUyskxo0HOIks9alBwXVRMqUiAr5XHCqcgEhMTmYGUr6KRuyLNpcvimJ3juqpSSnNKRpSLDDT7fhic4/1u808/fLi+GF6/vL5/uD/54367HafheDzst1vv64fD8XJ/9c3bt1Vdh1BhMS0kMyNiRDaRdTYI1y8oeQUgItHTfYClE5NklhTH8SQiTdXmNinQPPQx55yWvrshKRqiOmbvzcgoWxUMiUmNSJ0zREMEpmWrYVnnp1IolUtqsTCTDEUXUJwJrbQ0AQqdTBdjWqr2pUBZYKIzBL/8t0D/+GRb68+xwE1rG+n8W2t/vpwJwnPjdkVs16przd/h7IZNzeWcg3dNXTvPAKTwOI0TiAXvEanw5It8vZltKs9NdewHxOrUj9M05Tkuu7+c224753x/PB1Px5QTR8eurFxlJhYil5JlSWZECITTVFSbivSIOeaVPLMc3DKbz6p1FRAwznNSQUDHjr3zzi3D+yKmiiXpXvsiZZ0XM4vY8dQjknO8CF0RH09HU+u23cfPXx6Pp6EfwaypQxZ5+/p1XTUA+PhwD8j9NKvq26/eYMkg2RGxAZZhYCyrjvGcQS0V3QKcreopiIiFSwyQ8jScDsfj4zCcSvkfQiVSpG/IKKIsGm4lAOecUxbDDMijS84xO0fEwYeUXF1VWgVRVRe8d6svt3NvxgAJQeFp9ZGuE54GZdoTABbyUSlnVxTgGTj67IytBrloTZbwX6buyaiwLM4oTXmiUi+qAZ/j+YKDL667nOpl09cZDTVTMDcM4+V+d3mxH6bp0I8AEOp6HkYRIbY8ZzFlwipUwPzw8Ng2NSzPguycD5aHcZqmk0o1VdeXl+2my6ZxmmKKNFMIgYl16d4zB984NjNQjbnMziuY5YV3boBIRCKLoiQgECIQIyAHj+rK+u4SrpjZQJkpxlTaWk/HVoXJkXOBnUimBc5TNJCUsgp7j4D39w+FVgIAbVNnydOU/uLPfl5X9T/8/vsXVxddt5Wc5mm4uHyhgGZISLZQLSyLQEkkzplTwcdwEbl7MtBS7ZqJ5HEaHw8P0zRIWnOglGLMomjkAIyxiJRp0dMp4AgSZNGYiDiFEJiJjRVMDQiZCEWtnINyBdbeh5mZ5nOfU3Hl4S4AT0HG1ppmgTmXuaU1Pq9/oXNnaHGm628uHxlMQdath7ikp0tqscKquHbrF+9Lq/Nex51/0iZ1bV23TV2Gg2vvQURCdoRV8N4HVRnH6XTqcxqqugohlAaQijBzW9fZ+5Qz5wyIaY6fPn6qqsoHj8SQc8556HvnPRGBllm2SsxATSTHlJJkWxp5Aqti2RIW194SMzsiAUMjzwucR0TddrPpmqEfHh+PAKl8BFyPLQAyc9e1Taicd4QQY4opDf2QUyZEx06ylCGklNIwTqqKCN/98V0InhC+3B1ykr/6ixdd10hOMcW2u4BlldGCv5TUU02Zym5EMCvq+7S60dU4kYrcvHceF5AX1ExMk+gU8xSzKx0OAgUC5lLiOEQHAMSGTMSAWOAR74N3ntGZgUpWAiQnsDQwF3LQk5kiFEp4wSuhvJt1+LiEeIIiTrhy6Qvw9CdPhavBLvG4YOjwfBhkSQ7OOaYVrAuBVO1MeVk9CRhaliUxXds1CwiFAO50Og3jVFWhpGuA4IMn5pxzSqMPjp1j5yRLjFFVYyQEqOtAaCKSVcvAmkiWLPM8j+M0znPZRpTmmHN2zjFSUSZiJkRi76qqUgBOCcg0ZStjuGC2rLJ6ymxKsKtCUK8i6gCRuW6qtmmc91VTXyD1Y5jXKr74g65rry8v2bGqbruOGdVgHGfv3IFOOUuZKwcDEzsdTz74/W47x/jp9g4Ad9tOlU7DPE39zc01AJqh5BxCXZCgM+RpsHTY0bC4f1ySv2XestxGJFpq0zINl6OUoWRywQOYOWZYKjwzMzBrVKSUzwAA66aaEh/VRCQCZBEimh3XIYQQmBlpWaRbxgFWi1m9KqyHo8hKWonVpcO8QHLPQvozJVsAU80ATFTCMaxaOgvsClC6LeXt4dIseHqGwlx7KreeJHDO2Wtpi0KR/St+1h1PAxOlVNd1oIR1XbdN/Xg4ZREVTSKIGOqKgKZpHMdpnmd23OWmaZrC09l2Tds2p364u39ERPbujOIyMTKpyBCnsv7HETtmcux2O+9YqjpzFoNxGJd+nFkWwTUyAqD3vttunPcxxpyFwAxMVU/9oCervKur0HQtgKFaKZUMoGqqTdc1oRrGQVRR6auXNynLp+C6bXc69sMwlfbPlNLh8egrH95+dRrldOqnYazrutl0VxcX7z9+2e8urm++YhcQIOdYIratze51P5CtIX5B6UusffKhBoAgItM8HE/HYZhMs2d2zokCsqKCSjZAIvKOiYhpDdCqWXROOueckqSkZaifHXt23vlQOQAEJKfmPAIgE6rqUtEXGUe0M8t9yYJWMoQuNCV8yl9XJmTxEPhk5nYmSZfIvSp2L2nl2fqWMH6ePF4h0jUleMKVCjRTsl4pqmx6dtfgYoyFPDuNIzG/uHZNU1dVDYDGOsfEjJcX+7ZppmmepvH+/nEch5zySQfnuKlrBX/sx9MwZlVyblNVOeXCCmXvETHlPI5jnOeUEiLmlDWlTdddXF5cEt3dP07TjFjU/J8CQ/nQzOyrOlRB1nwsiw7zlFJm4q5ti9txhJuuDcxmFkplxuyD2293zNdgNs1z1zRf7u69921dN3UlYo+Ph3mamCnGrFkOh1OOiRD7YUw5i9nrVy+ato0pppR8aFyockoqWVWLCvOStC2hcyVglJoDCYmw/FmRxSxpmqZpGud50pwyEzMjgGdCY2VCAMfkHTMioRGqqmQpUgk2RxmneZpTUkMi77hyvmsbH7rF+NY3Y7bARes0CBVDNANaS8lzwrrkhmsyivTUOoJnDm+trZfwVip9Lfsd1rGTUhKVfSlECGtdf75Q53xjQbbsHNwXX6uLR14AcWeqQCg5IyEjz3HGExX1wGma1KzyjWQZ57mq3HZ7fXmxu729/3x7P/QDmE1hqurKee+8++rFjfcOAI7H0+HYRxEHgER1CLvNJub0+fZuGkdVaZrGAMZ+IMdF/kpXbwRqsAKZzByqwAj9sTfQlCTFGQAIIDCHqvKeYVHzQDLwTN57JCoMnKaqEGyepqap37x6iQgpp5xzjOnFzY3m7BnnubnYb6c5fv5y//n2vqqqEKpxnlXUIXaBX7140XU7YlfKSheCZsqQACBLWsH5pewsi2myZAfOsXtyKsVPIBCiqmURFUmiKWVCgDIpgcBEROgIVvKxJs1lNE2WYZCUYoxziiKGaHW9aZqubdqm9j5474nZMRNS8VNJFBEcczG9BWrEMsX5NDEHS3WyVtxPE55WcgpbqylY4UnEFapcLLeUtlIaEfQshVgvzrn0WZ5HVns/5xKm5b6X5sGyZcVpFkV0zvu6IqTDcbh/OJYyuSBH3jkwSbMeHw/eu8uL3cuXN8j88cPnaZrmGHNKoQreezSrm1pEnONQBTUlYjAz1aqubjZXSPTly21/OhV4KIswYt00oQrjOK54w/IxHFHwftO2zrlhGKdhLBVVoTUG7x1RkUuAnCvvqxBCCNM0IlLXdV3XMNE0z/Mc1bRrm65tX1xfVp6dr6qq+nJ35xjJuYvd7uHxUNfh9vYRifb7bZmrRDQi9M6FqsaV/6U5mRkzgREbq503Fa29uaVaV5HM4IAYgGDBtJGQQwhV8EsdgyQqKcWUpTgSzxgdES40ZQQtLKEi4EpgnjARAFBVN9eX+6vLy65r6yoQO8eMS0wH0Gc7txft+iVNVls2/gkAAwAUKjSoWqmTnhzbiggsQX/NX5dLYWZarH11jUimKmvgXrzgs448IJTXWoGwdRh55Vkv4JSBrWssXCpnkWIScd4557qma5tWVUSyrZV1TGmcZxU5nfrdbnt9dQFmX77cl0gXU5pjmubYzrEKgZkckyn64FW0H8cxxg+fPhFQ1zRFZX2aJnbOORdUDWzZBFj6WgAIRs6Ftokip2FIMWph7gEQEhAZQFKriGrvfPBqdup7s4UC1o+jqrZNNc+RiB4PJwO4vkhVVbdtJ6qq0g+9qe27jhGPp1PlfV1X7Fzwvu8Hx1RVNTuva1Ve9GIRUFUk5wWct+X2LJo0ZT0Nu2X5UWksFXkDREQUyVkkppyyppyhCCESlQEFM9UEEyzlFyM4QiYEYkBgIiarA3vfuarebrcXu31T184xlm7S2iQoyh9rzAcRMUJCgpVTVMBzpsXmVBUBDQt+hmuGWlJDKDXeikYt7nRtiJRyTp87VFxnoJ8wPztDK2CmgMtIdHlWXeP+uV+1Ml4BAEprfUlmEdAxe+9TSkmSiuYspfh3TMGxmn78fHc69TfXV6pWVpoSczkxkvPpeBqY6rpGIiaqEZHRAIa+Tyk5dtc3N7s9jcMIZkRUhVCc5ZKp2DLwh0xMbGJjnHLZnarGRIwEjN57JvSePfESIAlNCxk5jcNoiNeXFzFGJAzB13Wlau8/fqmCB4C2qXbbXV3V9/ePP7z/xzcvr8tNuNhtx2l++9WrLHkcxhB8FSrnvAEsx7WEJDWRnKVMXUqpLXipaNAWZSxaoPD1sq9RTCTnQn5bq3oDQueY0HK24hRK+1TAMqJzzKxIXHzUZts1TVe3XdNsmNgARK103WHFtmzx6LYAQFjE5stWkKWTtLxyGRp+FnlXZZ3nRmaotLI/zUpEWEqcxSjPqNDaKQV4srznF2F9agPAkr+u8PCaBz+lHWYG4JquNTMmBgQkjCmZQRUCGJhomTrJEhM+QQ6H0zBMMyM758ocZhWCimYRx7zdds7xMEXnnJg4dteXF3NM4zSaQRUYknG3IQTHTEybrgXV27uHRcC/pDvEQCR54d2pmiOq6trMFDQE31Y1EBWjAUQ2E8RDjNM8qSgifkiprarNtvPeV1U1DFPOWQ3meXbOp5y7pn5xsZWcfv+HH0vNXIUQk1xc7HebTYqxbaqqqgpSAQCqYklK/00k55xNZalGS/FROptES9JmRcjWFJQIkDCnPMd5mKYpJjNzXGyFAMAxxag5WUqiIqBSArT3DsuoFmIIoa6but003ab0uuaUSyuOEN2yFIWR2GwVU16rt5I5rdUPIQCcx0oLE6kc86LcXswUccF6V31aW0eS1gpqrcbObSsoYuWGAMtQgC2WVox1IR8uQrmrehmeSSa2NEp1ZUUBOO89GjjnRBUQnOOyAfviYi8q93f30zQXvFBVJWfvXQh+nqQAwq6cfTAlc94XHlOZD4wpxZybGqsQ1Gn2vq1rAGDm64vtHNPd4+M0joExeD53BQ2AiOq2qdqNSpJBHTtw3rFDQgKoHTNREiEzds57n2IcYoICkZgCuyx5enione+H7fHUI5GpXWy3vGmrur7Y7w6nU07p4eGh6DOOczycTkm0qcLxeCTiU9+XzNI7t6ZlRXRDRUUkiyQwO7c0ccXt1wTLcBm2XMBtUxXN8zyfTofj6TBNk2fk4IJjIlQmBJ3mFMVkTmaKCHUVgiNfusXOhaqu267bdCE0YDjOsZ/GApYRQ+V929SVeS5COrbgU2fnhYigi8hCSaiLuxUtK/LQAAqlgdcGREHH1kFkNLBz1X8GgYvpPqt2llC8+kpb6qTzFuc1A3jiJp6FzVY28xktRURXVVXOuWkrIhrG2cxc8Ju23W46M/XO9f1wOBxTSrTsldK+H4vYFTOHEABBshhYVdXsuOxUMFNGaOo6eBdTTDHWwVd11TYVAHZN049TTDFOU8r58nKPf/zRkhaYm733oUIiSUrO1XUNZR5a1FSkrNIgMoQ4p2kcUxYog9VgubSuRQzAN/T4eHg8nqrgzSzn/On+7qub67sqTCldbDdTnNnxy1c3CDCneH9/qCp/fbkfpth1m03XFEJCqc2hcJeWFa6GAEVMmcjR+U6t0anYZXERJWcpGfYwDqfTqe+HYRiDI4SKqWJiJqq9l7Y2kQkkpVwElACM0AJD8BQCegZSmcdhmPNpGMseFeewCgwMmjkvK0aX3cl/stoLDASFkBQAlzZBUc43QaRnB2ydVILzx1JVe95jx2Wnk1mpulYlvYXdaQBl6mqZiDz3mFafuuSwazBfsodSjZ1fBQzcfr83zd65lAXnhACXu+1u2wGBGXnvmqYuxA4wkJTGcSqfIaakoqYaqsCOmcgxVcGbOTFIMZqoGey23a5rDsNoprvtlomHYTicTjnn3abbb1tVu9xvf/bN19//4YcSetA5ruumaUNd5ZQgSxZhIMeWhQpqLSlLzlb2GqkhAJbtOUspBT6EJMoEdVWpKjGdTifn3f3D4fb+4c2rF3cim7bedt0U0+Vuy4QpppRyU1ebTVdVvGmb7bYtTQFmlsJaAss5F+9C5MyMaFkqe3Yr63V/lk2BmRbZkDnnWPRw1HSOyfFCzchqCOYYPXPZzEmgKiqE6ha37fM85jRFvevjnLIVAqggKpiKqqhktXPBsyy3KD6Scdl/XogjxfEr0ULFQxJEJHRET+gYPstDYdEQXz0gPCeL2ApqrqAbFDirLIFeGSnLA21F9Q20KE8U1kXpmp4helMDAueZfN2WOayri13BgcfpbrvtLi/2L64vd5sup/THD59jTKfj4fGRzCyLDv2gqnUd6roqSbMP3gCHfkgpzTGpaE65rcNXLy598HeHUxVC27bf//h+ntNm23775qZr24fD8eF4MoC6rlJMyNy0zdXl5W6z7cfx8Pg4xZTjrCKjiuZsa5timV5acnIzBTBBRCRkYgQUVXIMBC+vb64v9uW6DOPonDOAcY5EtNu4oR+2XUvsri53wzAZyPXl1W7bimRmMtUsUc0hoKioSokeJcU692bWlsgaAZc7eq6SDABSzvM898OUUkYERkCElHIpTHJKKSXJsuKV5lBBxRQkGZkKaB/jGGXKEAUNzDnHTLVDLmurwUTVs8Kic3jev2oAoFj2WayZcoEtzRCBkUqWSEAGkBHLB3ySOT1bGBaCvRGCyaq2gFhWJC4Z5EqJtDUrhVU3qlyeJROFFYGCFTpdW6ELpI8Iqm4YBvY+54ymTVNf7rfTFE/DAADOuU1bi+r98dTUgRGOJ+y6lhCO/dDUAQm994u5gOWcAVBUc1aE5Soc+/Hv/uN3dRUuL3ei+eFw3Gy64/ET4+by8qLttnMW5/zt7f2CZRC2TXNzdRXqKkp2jst1AijZn5oIwJPw3zIys+xEI1vYT+g8Bx9KzgAAPvhffPPm8dSf+rEKXlVD8Mdj/+7D59Mw1nVV1/XjYSCCUz+1zenm6pLYqaGa5pwQpOjDSE5lWXwxvTJBIWKAsJZTCE9Ts08VLgOC6uF0fHh8HMbRMXpytmx8N1NNKaeSAjJ7RgITQ2L0rmxN0WGYslq2wv6h4F1VcVfXPnjvGNk5dmKoBkxlXLgEFF1bkWVr3mo6q6vLGYAXGMzMSnOfzr0HW7n3sJZ9z5qU8LSgttjb8mC1Yr4L/LomrE8Zgy1lOwAA4wLJLWnnuco3AwB3d/+w6TZ1UyFSFu28u9l27dCkGA/HU6mKzFRFiHC36fp+VFMfqpJalFXe7JiZ4xxjEgRyjpwLRNS0bRHo+nz7cDgNL24uQqhev3q57br7x+P7D59fvLBt12mjl/vtl89f2LHfbC6/enV1c01IKaY4TvMc8zyLyOIP1kjxFDsQC4Nsyf+ICoQxp8jOVRjGOJ+G8f2nL01bf/Xyep5jGQkZpul0OmXRlHIV9NAPY9+/+eplynLs+8v9pffBlk2F8RzwVGXN64u/hjLHLuukh63wYBmjLc5DVIjIsYNSPs6as6uzD56dK9W+sYHzbCulgwAITAxMdU5SVoUTZCZ0zgcOm9pt2hCCRyIpI8tAWVF0aQqnnM/+HlciSzFTVaJis2vtwsxY8Jx1Azetqq5gcEZZl9q+2Fk5rCsmtQaQJX1d2vFrXnvOD0qHae3+rzmnlbwWzhSAAgE4RIwpMaP3PKf8+HiQ3KkaOvd4ON3fPVzstyEEBdtvWu8dO5YsOethmr1j7z0i+uB3m/Z4Gvq+B6BpNmZXVSGlHOdY7KYfhvoYXr9qHdmbV1fb7ebdpy9ZPn395vW267569fLz3cPx1Fd13Xjf1cGHahz6j2hzjDEWreEzjF/QNWLPBTMmIlMpooMqYiKi5h07A0Qkg9NpyFmuL7aaJKtuu3aSVFfe8/7T7f3j8RRCeHl9daxCSokRqhDmOHehcezmec650OaLyt+512K69GOYtJACFHGhjBGWaaSlri1Oixm99545qUrOmZAZUYAJ0TSJgCktjGcgMFQVkZg1GywUIjMkrIkdagBFzRINwLJhQmbnEZ0ZpKzTnOZ5nqa5GMfCGyd2jokZkFxZH+icqhYBVyIEoLN3LB+yMAxU4Tx2b0tBsxhSacevQFKh7q92CWVIfw36pa5XKLgBLV53wT9X77OKmxgAmquaJqV0OA2qaip1FSTrpmu2bcfE4zhOc5xTUrUUZ88MAHNMYsbMxBScA4QQfEqZHG93u5TSME0ppeAdmA7DmESmcWy7ZrvtRHWc5inGTdt2Tf3p9q5pmov9/vrq+uu3r999+ARMofJ1XV1fXV9fXbndxvi3h8+c+n6eJhEpnsuHyjHbKjKMgFkFwUA0RpBShaQ8i6YUTye32W3die/u7pu63m/boW8Ld+bVzeV+t5li3nTdHKdf//LbYZqiSDkEw9B37YaZNceYo4oyoxHTmkURAREvgt/ETA4KOYCWYdqCg5aStyT6wVMVWDSDFehOsJBcEZkwJpslg6krZsrMzlceagQzUEBH4GmZaYpqGrNaVjMgBjJnRIwiNk5pGMd+mPphlJQBrMwXOOaq8iEEdhzKDKRzxKyqwTtjXpC1QouHwpVHXuc0114RnuuedWPzMi1ccP6lolqxzhKun0DUdVxOn1WSxSJlzecXFMzAtU0DdfV4PPVDjwop5XGKt85tu3p/sa+rKqdU9qjePx4QoK6qzaa9vtzfml3utm3XTNNUenebribCH999FBHnXFZrm4adi/MsOeck4zTvdtvjaXCOHKIjAKQ/vnv/9ds3N69fzznNcxzH2aY4joOjFzeXV5f7i6tu8x9++49fPnw8HQ5xHDVnAHDeNXXdVMHMspqKOCYDO51OAgCZVLKplMS/ClUc43GeEZEc3z9WF9tNEmnbpvIupkzMnum2H+I8v371YpynOaamcobSDz2W/cEGZpbFQJSf5jYL/5KXze8/BfQWwOlZyVLgd8ccfCBUhwAAOWsZYxMAYvKrriwxe0fOccWEoFDm1s1ETZGQyJCSARM7Ju88MhsyIuWc5jiP03Tqh1M/pHkq8bocIu+9d857R0zeOb8sPfdVVdVVVVUBQIkcEZ3ppMuuxRW0KslWaUrpOfVZu6ylT7mw6kzLaJqt4MaSpCOWPTq0WnlJBmytLBhpGZqL88zOeeeD95IyMYlpnud5nh+PQ1WFtm32u00IPsXkvNt03W7blvp9t9s0dXU4nD59vq2bum0qM1C1qqqc45ur/Xa7yWIq0m66z1/u7u7uL7bdy5dXMemU0qaprrftw3H4fHt7c/PCeb/dbcVs6vsvnz7uu2az2Ww3u7/61Z81df0fu/b9Dz88PDzmafaEvq6NCiccm8AhhHIMtt2mrrKUPwCqxgDTNI/z7J2rQpVFjqd+GCdAeEn0+e4wTVMIXkT7YVgmNAkeHk9vXr2s64a5SE1JycsK/aWYGjM5ZMJlCKRgpEwOyia4Qr0lOkMzi1cycwx1IAACMFVLOUczptJ5J+fJOy6rugkNzUiTqZYOpQAoEDISOWQXgg+LPiYhUTZOOReGSE4pzlOcxnmeUaVM0TEzMlNx8lwmI533vqpC29Rz3bRNs2lbH0yJjblIQiwZZylo1rrz3EfiYlhqpR6Sxa0uWJusZq1rQ2lpMsGZSrKSa5/lb1r6+4DueDwhESMwsWvctm0FIOUiO5z7YYwxxhjbtmna2jMHR455Ur25vmya6njqvWNAHMYpHE9zTOM0Vd4H5yRLCOGXr199ur33pz6nfOqHz3eP37x99e3bF5++3B5Pp21X13X98PD423/87RCn4MPlfp9i8sTHx8Px4b4Nzabbvn756svD7Xw6BaJxnsFMY0opiTko/e2iCWaGAMxUheCCr9pWpvnxcDyqkBQyvzla7o9jqrwv7uHh8TiM8xRnRjyehqv9lphSlm9ef1XX9TkKFW9ScileSh+ErAbiEFUZCpy/gNNnhKlwiCyLmGRTUYPKMyEpWBbNKWdREVMxIlRzxFx7xwRQyHlZRFKRVzVidr5yyExNFXZt40MooLkYgFjKBAAIKlkkZclJcwaVUrcbrPMoi4myY3bBzXOwMl9F5BwbgA/IRCIFHi3EDlQDybl8PCIs43dSahxYkgKzsjF3CdxSli3COa+FonTG5xbXOq+9QgV2/pGZuXEYAJGYQgie3TDNarrdbrY3V4dTfzgc52lOMSJAVQUkvH14TDm/uLnqNpsY5+1207WN9+724RhT7vshjhMBsOOH07C7SCH4n33z5re/+66qw+HUG8D7j7evXr64udzPMY7j5MjGcfzhhx/atlNCQuy6hgDv7h5MNQT/6tXby93u9c2Lx9vbY99LSnmeQbToOjvvyHFgbupqTpnBCLBq6zevXjh2X77cMeH11X4cp34YPLskeY6JEczg9vauqirn3P39PSI1dX0/nNq6lZy3281ut51S9N4T01LWApDZuTsPBqqCCCLFUz6BjrZ2QWE1U0JidlVV1VVIKaJpYUMyITpCsIJSSFa1TEgelsoJc0oxDSklEQNgdlWNVeVqh23FPriq8kQIarNoEiO0lLNkycXsRVXFRNZeF6KqFE4goiILUc5sat67kKosOcZExLR+TEQsi6zxzPVcNywv2czyuHMHH/AZc54WT7oQmYlWqGrZMbLSROCpcaVLegSA4BAJCb3ziNiPI5h1bVv2Zu26jtn1x2PKGZBijGAuJQnep5Rvbx9Siiml66vLqq62G4kxHQ6HpmtD8DHlNM+3n2+/a8Lbr17udtuPX+6r4DdtfXc4PRxOlxcXVQjTOBZN4X6cRQyZRPXF9bUPPuU8zfHx8WG7u9hu9t++fptzDN7fO344nXKWGqBI3Q7DoBBdCGKmgMwURIfTgIhlXoCYri52++1WVac5TvMcvD8cj48Pj23bXlxcIGJKkdgR8uPxgI6TCjNt2rr2IVA4RyWAIvDLS5GEi6oblcGeEqnUgIsM1TJxVrIrIvTe+xBCCDmnMx0MAIr/K6BjykaQQRTAQAQkx5SnLGU1cPBGjJK9SgbLYNmUDagkuWiGpqZaCJAp5aLOvlKiYSnenpYciKqiQKmAVFVWPcKy+GaF9E0AHRMRmloRn4BlkGON13YWMoFnlNAClC4VUklyFBCXtc+2etelNFrnR8r/GQI6XwcAIGbnHDtnqqZ2PJ6yyPXlbu+dSsZproIHw2GMTPR4ON3ePRpCCN4xl/H2yntE7DaditZ19fHj55zyoR++fHkIobq82P3s69cPjwdTy2a3d/ffvn2bLvafb+9ubx/6YR7nuQqx67o5p007brt267oQguU8DaftZn95+eLPQqib7vu20/cfjg8PMs6FA2pmQ8o6TioCiOT9cRjv7x9LJOo2m6YKpbk0DsOifQI2z7OKxhinadZlcasgsyENw9jVNSGNU9x0mRMqETOvedgCvkJRpyPjJXxhqUNwaTItzFwod0MlpZhzLuMoomKS5yjDNEvKK4Vy8RyaJBGoWhJRkUJOMTMiqEUNyPvkfHRjJHKESOSyLbRTQkOErGpgxX+W5I/PZwhsAXnKxyAkdrSo1xqYZpGU85KflEX2RLAoZ67wvgEuumKll1foJvYUOgrAtLTdbQWsy4UyWqceoDSl7Jwi4DpEurhZV3QZGNkAVHWKMcZYBV9VYZxiQcjalolpTqmsRCrn0nlXhdC1TRbxzg/jBGb77abwmolwf7G72O+qJjgmEdnvti+uL999/BxjOg3j4fD48ubm8Xj68d2n4/GUUp7GqdzPh4fH/eUFImiMh8MRAJq6vX755sXli227vby4cm3zH/727x8fjjnmMjBUSAnMpGpxGBBwJHREoa58XdGEVFVTjKdTryLe+1M/ZjVDnFN+OJ00ZRERyuyYEVTy8XjabDomkiyZkI1XqJnKTA8qANhZN6SwnEiE2T2bqi1egUpDf44xxjnlbAaO2EiL85mTTHNcFIDBGNEzBbduvipebuUKC6AiipmKxBhHYgB0voBfCGXQouz/hGXAlxFhHYsuGQcTQZk/sXUAuTQ7DHJWp1r6SYgIzCbCAGVG2Vby3mqRZ/BywaHWLtOThN1aDOnZPMEgi66ud5F5MgNFKIt412lPUDCHZo6dc26a52mezdQ5Zu+K5pZ3DAApSco5WEEASUQNLHiHAMM4dm3T1LWI9MP48vpCzT58uq2b5s1XN23X7rcbVfvjD++7Tfvt29cXu+3d/cN+24hkNQvOGeAcY5qjqkqS0FQxp83n21AFU5VNF1Os6qYK1f76Zddtfl7XL65uXlxd/3f/w//j/R++xwTMDAbeMRKlnBAcmJFjDj6Kfvly54O/ubz0zAUR1Cwpp8qHnCKYaYxFb2xOsfNt1zTZ1HknotMc67pwTRacE4nMsNC0zYCJHLOacWkdFTdYeoLPaiRcun4mlj0B1U4ylqctNYtzLqdU/B0wMoKpeUeO0AC9Z1VX8FXvKHhGIDFMApANBWsHXApn1Zg1xmSSQZXAmACZzIxLMki0IOQlA8bSrSmsYU0irmi1qoFZFilbfYnQlMonRESwUiyspbgVd1mm5J7mjHnBbhefWnrFZcSq4BoLy3+1Wjgrh69oKyI6FUkpMVFb18xsZpXzxIwIcZ6D7+qqApvMrEwbIlFdhQJ7qVpK+XQaVVQBNl272XT9MKScry92bdPkJF9u76sqPB5OADDNcds1Kc3brg2e+1P/3Q/v+34owI2IxDir5FylL19ut7utqngmZjw+Pn5pPnKo9hfXdagqH/71X/5L7/x/+z/8j+9+/wfpe43zlBOxI+fIsqpYsphFzbxzdVWJSI6JmBxzVmFmIKtDJSLsuPQr0xy9d03beu9e3lxdXV7UdX1GAgUEoAhkoiqQWrFXOM/TFHIGmKrxsgeqXPlSk6BnrrwfmSVnXcIYIiJTGXgBoqfGtJqBISMQsQAqY1F+V0UzMqCshMaVq5grI1ZEE4hJxzmOc0opS15asqVmh0JtWXyoYdlju5CqSwZb2HIqqlky5YJJgRXxRirJI4oKIYKVDW24wqAABaVHW/v4RebgCdXHpRFma5vDZNG6XzT3cGm/L/ZdcAMnYpjyBNA0zcV+t+0aUe37YZrmcZKmCqPpHBMilgaDmbVtc3WxnebYD9OpHwhximmcpk3X3d4/Hk99OWgiYqpTTOM8A2IWAdPLi0tEmOb5y/3h/v7xh/efU8yExMxF2EPA8jSNw7DdbT27VXTTDodHHz5WoWq3F8zUtdt/+eu/AID/qa6///6P0+lkScoOVslJRbSkRwgK4KaoIsH7y8tLFbl/ePDOtU09V5WBtW0bQnCOh2FUkbZtmrqqq1AFf7HdOO/HccwplsLVIXrHziEXljygKqgYOjiDfcVSCQUQEFdhWwAids4D0BxlTnmc0zRHyQX21sLsoDKqU5TTJCdznm1tOoIYAAExsAIahZJKEgGwiKacj/30eBxPw9xPMeeMYExF8Q4IkLFMrBQneE5OIOsSAVQsZaUsRLlw82BJO4rntVULYrEkfd5kX7nJBTXlc/WOiCvzb4kqixkuoiVSvOw54KyggKoRmRNVVGW1nFLOeY6pwNCErKYfPn6JOXdte3190TZVGQgvcR8ARTITEnHsx2EYS8ibpjmL6KRVXe02HTEP09i1NTNPU5znJCIPj6eY8u+/++HUj8TkwK/NCairWk1Pfd8dT69eXjvvUsrTPIUQ7u9v67qu6oZdS4Sbzf6vf/NXu83mP/zHf/jdP313eDyMj8dxnrzjFFNOidi1bRu8F8nDlASwypkAtrtdW9XOk6myc5cXu912o6rzHMdprKtwud/VdV2CoZnVdZOZYpxzkR0yFikkUcgAZtHMiJDYOSQAKIkQEtPTyKNpwT+N1CyJjnOa5jTFmGKOsUhUWfkPIhAgM3lHCZSX51hLKNKsmhRb4OBDP6akAyGA6hjTcZiPx9PpNEzjVAi7YKXiXmaV8Lm+kkFxq4Rn2P1JwlJUEUnJcEXaiWjZu0aA5/K9IJq2Up/WLxErsSAvMX0ZIzyPdzACPBtFWrNYK52Bwo8jBWcqkiEjeO9yln6cJOdxnGOcY0oIcLHf3dxcXu53zFhCUzGlrg59TzFnQk051XXlmad5VlMTTWbznNqXTaiC907Xaax+mO4eT/ePx3mOwzghqCOXzZwr6xyEENqmnXM+HI7bTeO9c8xf7u6nOe62u+3mmGIMdYPEnvBie/EXv2xeXb14/erl3/7933/84X0YpjTNRIOvKt+2dVUVTKPIJ2Uzk4yio1mLVfCha1tmRsBt13318kVKkRDrqmqaJoTKAB8Oh9OpF0Mt60cyMBMjAljKgoDOeUA085JzKoU8Cy2Rs3iKJX4BQBYxUVWNMZsYQREIMUBgxy67Ocac85wTZYiJvRMm0rXqKjFymqkf4zCnOYn3wxKzAVK2aY7jOKYYJSXJmVau+1KLEBLAcgNxUaa1VYhgsdJ1sYfZMtJbyu9zik1Ey1KFIvVTxIYX4tKahK/D44Qrrrmw+59kSdciHp6p45QsFlZzBQNz5Yp67w2ggAumVrpH5NzVfvfi+qJpGu9YAWKcDscTIg5D1zR1COHqwo3DFLyP83zq+3GcmCirEmKc53EYr6+vjgD9MJpZP/Smev9w6PthnucqODB3sd8C4uPhFGNSkZQyEm67dp7j/cOhrjwTeucOpxMRD9MwTUPVdoGpkMSbugkvXrVNc7G7+Mfr33789OX+/jGXESUwIt50zaatifBwOFqZaidqmsYzg+mu28CicMRNs7m+anJOzBR8xc7N80SApjpNU5aMpaYWJRQgAgAkdEzOPfFxEU3VBISxeCtEOC/h0ZTyOKc5ZlXNZoAUPAXvRDUX+V9YFJ0KMxpMMz1Rh8tWxCzqsuacx3Fynpl96T5m0RijpKSaVaR0DhhtzTqXCkYLYPlUfy9qVnregQBwFn9U1TLnU6xcoQgzCSAScVEgxHUhaOEenjU4zkNwKwj7BKWdlZ7X7xReyfkcrkUSgLu8viyJ0zhNwzDUIQBAjDMivXpxc3mxTzn3w5RFihzSPCdEQCIXwnZTT+N4O47jOB8Oh2me26ZGtDLp4hz74K8v9oR4PPWEMEcZp2Pf90M/DMO42bQF5ri+vCjFLBM+HE79qd9uNvv9rh/GOUbnfcoCgHf3D0i46bZtu3HeEy4pOTm3313+5Z+1b796e39/++7jxz+++/H27m7sRxCZ4uyDe7nb7OoqNM3N9Yuuazdth2bj0JeCwBFXzhNRCKFcl3Kx5mk69sfj6RhjArAijZAKW4UBmDwXgkhpHZaq9KnjBypAvCTRZqo6pzzFPMecRXPOIooGa3O79ACXDmT5pXMxa2eUlNCxc75M2RAgRlGNySTnlMq0foGZFhmwFRQq70nO+jTPxjAUlh6DrIVSCfZudatgpgpFuhEWCYYyUmIASISqi0bVmYas69rLc9FjtkxaP0sGFnB0JSmYnhforGQ8h4BznEVVRQkoZjHTum6cc6Y2T1MuG/tirirftU1p5DLR8TikmCTnOaaHh8M0TT4UnVtYHAriy5urly9eTCmZWdNU3ofTMKUsh+MppUxEu91GVA+nHlfN4q9eXn8AIIJffvt6mOLd42PK2RGrChI/PDz+7vvvQte+9Z6ajnEhDDlHznVN017tr169evPN19+8+/DuD9//cP943227b796/ermRdM028120+0WEA+5LKlMaZKcoQD1gDnFnCIimshUzSI6TqPKIqirjOyYEIiQbZ3bRCvoI7GpKS/PvwCB610yBHLes3PFCJ2Dog6dRUubsNgnsitEOzAoQmLl8UXVh5kZSU1zloUwkXLKWUVKP6Cg8aWuIgIuk+eFf7EilbgO9JXcgxa6arHWkgoubQJf6n9bXKCaiepKgNflZCmULGoRTVvU1IqZPXXVAUB0ZZnimsICrMjU8m9Yl9CVpMGBQfB+nGbnedNtQggiEmPMIqdTP05T2zSd95uuqUKY5nma0xyTY/PBplm9D1nUwPYX+7atNMvx1AOY926aYZomZmJ22+2mqYJjPh57WE/vMIzeu8vmYp6jiHablpnmKb68uZzm2TFf7LY5y3EYsggCpDmq2u3nLx93P1xuLr0PhStUrjISMpBv27quL7b7b15//Zs/+02McwhVW7chBGbnuMyEcYlTRR8hS1X0wMqMmk8+zU5FlWVj29ev3w7j+HD/YIsXRLd0OYjo/1fWlzVJciTn+RGRmXX1NTM4uFxSF2W2K5ErM+n/v+uND6KkXRDkClhgMcBMd1dXVR4R4e568IisWglmAHp6uqqzMj38+Nz9+1idn1zQWEpRJAmhdlP8OftWk5mllOdlllI8x0VkKRKCqFoRLSIkAkgamIOKKVyHs91HQy6FAJMlU3XuQdEqOysiCEBemTumBQQI7LI+jfXzNtK669JqGSqqwaskqG2BZlIu0MhgtdejToSMykxaQU5rkRnQ84HW4L01U6tbRvVv1x+whoRWd65wfR1AcPVSJNoOQ9/1S8mlVMYbVWPmzWZ4erjfDN1lmqd5MdVSihO/Oxf9MPTvnx4B0aSM05RTZkZjTql8+vw6zfN2Oxx2m5KLiuRSxnHsukiE0zTNc9KisevLPG/6+Hi3/+Hjs6o93t+fp7mLHSJGotp5Q7hcLudxkpxU9Le//YeHD18hddxGN7w7gszM3HX9brev9wXqsHgtN82s0QICGHpbEsHQQohFjYOYZgQk5qHv7w53aUnTNJqpAZmq6/ENnfkABBiIKl5LYOe+8yfjqjAyT+Pn1+c///zL68txGqciDjeKk6aoqKhTpNZ3qdRHVbLFAA3N54mMauTVai6ga0mCzQIRkZkMkQCL2G1W556p9i49eiOymvhIlROnmgDwWtiLmvtJIoA2fK2qa9LpDrH6TjOvnHTd8GwZurYRE2g+3tYtu+ZzYXXYZgAQ3o7HGFtLvZRh6O/2OxF9O51yTiJlnufz5ZxyPy/L29u55GSAOZOo7Laboe/6vgPT4/H0w48fc8rMpFLmccIQlmVZ5nE/DPvd9vPnl6UUr+aJKWIU1SXnl+Pb0+NDCHwe568/vNvvt88vx83Qh8BLSkSwpDROc4yhjzGGcFnS88vx+z99/3j/sNnuAj8iYOCay9hNWIkc2u1YCwJoGb0nr6xKCFiggLgcHnMIkhEJQQARh34YhgERiVlFVKGYGUhnUkRCjNbc91ooYcXFEb1iMjVVkTKN8/l0fj2epmnKjc/H7dLLlrZ0Ve0T2qav+yesOiG26mFQm568AvBQc8/A6ImWeJG+Tk4BGJDnkYIGBmKgIkgC6rMr4lTsFfRZf6kBqgGajyJoLanEqAHvZiJCiC7S0owNWhy4LjJdO24tC/KkuDXuYW2YAkAIxKaaVZGZCWMgKdkMDrvtdhgATIo8v55jnLxNstvtzOWMnE4CcdP3IllVY9+B2WWap3EG02GzSTl/en65v79/ergvuXx+eSPip8e7nPI0L0hoqpfLqGp3d3sV/dNPv9ztd2h2PB6ZaLfbiGDX96/H03gZN5thM/QuIvrp0+efPv707sOXsd/2Qy/1Pq5OAqhlgQAVAGlnst4XJTJVInMw3b2R184q6uALEYUQu9gZQEoZfBSXsYAtuVAoUSSEQOi7H0De+AarDIZmagqmueRxPL+d3pZpKsuUl1mKz69Wxl2/t2ujUOvwbt1+YGifytvooCsg4+nlLcaNCIM3JwFc9kUdpScI5DcBtXLZeHPK1LCospqoUx2YqIooIXmI57oMCKZeBiliPZFXG8QmdLUS1xi0lYL6ceg6GVoN0LtcazIDdSTP3w3MLABiKYUCMWEfY8kyp0REXYw+R7Lfb0MIOWcA6AIV0Wle7u8O758ei5Tz+XyZplJK3/dfvH/69Mun17eTqCEAMavZ5+cXJuqHoeviZogiQwj8nN92203fdy5DP47jMAwcwsdPL2/n0VSnRZBOSLjZ9M4abgDjOKtoiNHMTufL9z/88PDuabPdc3hX0y5ne/zLzHw1WWtbyv7hGzRt0HSaPaAhAsWAhCKEIhx1v9v1fX86n52d3gyJTYXqHp96YlmI2Ut1NrMmpuYdRSnldD69HY+XyyWnZCpoyqAG5gsRRlghyfbItbJU1b0BbGABIJhhRTRbp7FB78hM3n8l8uFiNXBQBSITNXhStdgKdXp17WqctR9gzbZuzknzSczV7RVRpnUx8KYcXKWMW37piBys43kV62xYl7WEWGsIFFXPygAsUOAhUAjBzJ7fTmCKhkRUYjkcdtutSydE3G4A9Hy6HN/OMYanh7uH+8Pr8fh2PuecD/vdbrv55dPz+TyaqsvuDl3UUn76+dM8z09PTyHGIroZBgBMOe0326fHu58/PTNzP/S7bT+nwkgIQEy7GBDhMs4xhE3fxxhUShEdpznkEiMvRb/7/gdTtSz/4T/+5v7dB7AOIzrFu5/mFQJuRnnNyn1F08lCEFthQYjAaOjxrmZViJvN9v7ubhwv8zyr+Bqw525WREQMUJlNxYRVVEkNyZrFGwAUkbQsaZnzsoCUgCaEhAwVbgQDqNTlqutz9cul1nQxrOhhhSuxwuBQh4YBCTvGyAhIapDFzFtchMHFnRAV0GWDwMWV3DE2tQ7fVwb3kQ0tU1M0VFUAAjK6mZzzH0b0ee3q9lazXuv0m8B17ba3L2ogcNIGz3Bw3ZtFCA93ewMspcw5iyoDdF1AgBh5uxnAUEUCD10Xp2lKuQxD/8X7p4e7nWp5PZ5EZDP0h21/nufTecwixIQIMYau786XUaWMl5E5/NXXX0oX53kexzEQbYZutxmeHh/MIAa+vz/Qeco5dTEQQgguRCmXad5vN/vdxic6xmlx+DCGYGAfP/5cyj9eLpff/Pbv3335NeAWQsWr1nB/a6YtiGij3FAEcHYh8fBmLh0qJTuvTlHVwGEY+hACE5takYLkQhlkZiLKEQFBzLhKZNwsjps6WsQhumvSym5cGVyl+W6wSuB5kz+1SgKguk/w1LM2LeufwQDBZ1EdlzVA5y8lAmaMTIHaOryjN17yAAIYke8V14gjaiKmCiLAbP4ZDE1rBQ9QkInwirRjQ46qoa833k1wRVrMfbOfqNZeu75LS1bd8B3+N8MwjbMSDsOwDSGGAACBedN3+8N+6LpSJCX5/PLmyf7Qd/f3u/vDYUnpx58+vb6dnAT55e3y/PnldDqbKiFRF+72u76L4zQ59QO0RH5Z0ul8GS/TF+/fpyJdjH/7619N8xwDH3bDPMNhv0ulmFpOi+MvXeDArKrD0HdddxnnlAsiRGJDez0ef//NN1nKfwZ7/9WvzIYQAt2YI1xDjyG6Fps3bNVVNNR1F1TNtOSU05KXRaXUghiBOOx3hxi7k15KKWoaiVrCVyFPJ2VgDkxc/UR1f6amIcT7+4d3797P8zL3s5QCKqWISMEiRQTMyCmTbgGgmpkhNoYSJlwVDQzASbrALBBFxsAElZRBTTUiMFEgjETtfQEN1FX8GgZaTcopSwkQzKVOTU3VPKCDl2a8+mtPmB1sa7lTm51b+5a+Ue8zTOhQEa6Aa2WE8bH566vWFNU5DsDCOI4xRuw6ZBpih2ix67549wiI85xqbtvY8D03mZd0vlwu49jHsN304zT9/Mvn09sJEZ32qO/idrsRKYgQ2CcmZZrmXBVXMRV5PZ7u7ndE2HcRwAihlDIvqe/C4/1BDVKKx9NlSfP5TbvY5ZReU9ptht2mI4J5XpJACCwmb6fTH7/7bhiGjsPjF19Rpb+sYkt2Q0ForRIyAxHnGpOa+bkWr6mWoipLWtCLAyJk2u52T48Pb29vS81ZDcCHmdANXW/ZCsxUFEyaIg0S0n63+/qrL4ksLYsvNqioqlzGaZ6XeZrnaSpLKqWQKoAFdBQIGI3b4g9C1Qx22fM6gk4YAwYmYgSArCZi7JA/YSAicn5iED+S9VKB1ll2aPpETeLDTyY28NIfELtCcsud6gWtZnU9sjUFsAa/39ZSUAv2inf5oamG5ZVuS3v9r0M/9EM/IMA0zqJyt993MV7GKfbdYb81Uzf81+NbSnkz9OM4//zLc8o5MMUuppxTyjHwbrdlDt5q2++2fcfjuLgmZwisIs/PL0S0rmPPKQ1Lt+kjaNkOXZFSSpmmaZ5HAnh4uNvvhqGP07y8HM9pOauUlHJKadP3wzAMfX8Zp2UuMUYA+/zp8//I/6Sl/P3v/uvDh6+67trMWH1nM9BKoVNDjohlM8JVAAAU0klEQVQrvZqolmyiOaWcZibKOad5jl1HHIZ++PDui7e3t1LyspiqNXHJCg/lIlEkWhsG97holVWGiGKM2+3m4W6/LAEQCTEwRWZ3iCmVcZrf3k7H49vlfJnGUdLi9IDNkRpUGseKwjuvCSEGxuCDz0h+4hyAgNY+1iZG6KCStQwB0Nj/Cq6dr9XkzEzBGKyIsoFVeWFTA18guX5OJCKHe625hVoJ3DyCa24KrU/QlBoqCtaK2mazAIgY+mEAAG+YbTeb7WaQUhBx33XELAJD3xsYB96Fbd91x7fTOI5mlgmnKQGCSjFVV84khN7FN8w3zDGECICvxzMAPNwdfEp/M3RoWkqeTYjg3eNDypazT/rJp+eXnJM3n97d7fuue3l5ndKSMuVctBSVMgzDZujnJaWUWIRDPL29/f6bf0bmf/gv/+3h3ZfO/nzrO/0fUZ/WUfBSybF7KVKSqJaS1URUfTt5LsnA+h4Dh81mc3d3fzydVM2joWuEurn7ZISocuU0WNP8+gUhdTEG5sLeyHTRTQ4hEtHhQO8M8ldf5JzGcXp9Pb28vr6+Hi+ns6YEph5IoaV+DUqjwBhDna9KIimrgSFSdAYyX1ABUDVXAyX0IU5AF+9qdYqDkQAgarG6TwfjIXBYz7rBKg2vWNeDV/j1mne2fytE2tgc1vdABGCsaBo2LMKal8WbWxfSkjwJ22yGu/1OVHNKRLTMy+UyLkvaboa7w/7x4T4wPb+8juOcixRnU0FERBFx3h816yIPm6GLQUWYYxchMJqJFEHmYehcE5sQRcsyT8qMaGPf+XOMgU3AVMZ5joGYsKiGEB7u73IpqjbPKiKXaU6lDH3fdbH49EaBjPD6+vLNt/887Ha/4e7w8BhjXA+iNXTYP3xt/YmoFKtTOwUAfcyAiUwEwPq+rxQHgfuu326Gvot5SX43vdwpIszk3SMRp0EXqgU2rc7AzFJK85JySj73TQhMAbFGW0Qahj7QQR7t668kp/R8PP7y8y8f//zx9eW1LKkOVACuOyRdpMiIxISYRKesZtYxxUAxUDWOlte1C0aDRo8P5hu+qwFds1t16/GMYuXmNwKDtm/kS/FNCWSt3GGt8lcss/72ViN52ipqDbwGaA/Jv1rzB0MI5/M5MHd9L0VO5wsTiRlSEpVlXsQgMOciqioERHQ4bJeUx3H26biUMjP1fURERo0h7LfDdtOfx7mIhIAxhi4ENT0cDofd5sePn0rOgYmQci59F/vA87Lst/22C6UPS/bDDSKiJRFGUNh0PAz9ZZz6LoyLOgBNYF2kwj0nzKWAaWQoy/Ttt38Ydrvf7H7Xdb3vErWehqeL2cSk5JyWZbzMy1RymecplZxTulwu0+U8TvNlHKd5STlpKSpKIbhevKn1fTS1yBzIwRfMRTopMXSe4NYUoiVStgY0oixlmhcmCrEO+/SddV0k32NzbXeiXb/B3e5wODzd3x/22z/+8fvnnz+ntIDnnWgGFKOTgRERLWJzElGL7AeMAyIhFKgmogBeqRCBKDQYx9bGWgvX0K7XmhHXf1SNqxRj7QtVf+m912piCP/PxmBNWf3+rzm6451XIKBG+YoDXAslAAsi4tG5SOm6oKq+u+XEYPu+N7Pj8Q0A+j5uh76PztqDSwrjZSKSYdOHGEwNI3dDT8zTtKRlJoTtbui76NFov9vMaVly9hnJ2MWHIW6HIVdRoqHrY99Hn7rBSgBquWRE6EL3dLc9n08FYb87FHGiIjrs+t12k4ucx7kU3Qw9MlrKnz5+/D/dN3eHu1Jknqd5msbxMo7j+Xyaxul0OU/jNLrk2zy7Y1MEyaWUDIjEDKrMrAbe+0OEEHjouoe7u6en+4fDduBoToPp6aC2Hnq742YOULuMhnEI93f3l8tpmWeVUkrxOXaHG82gw87bWqoCGfqu32w2zCEXOZ/H6TJKya7KbIDR1eh8lUp0TiWLej3KgYNXRoBkVgB1PSV1J7giTf57nV1RTU2JfdXMTGpC1BDZFnDV1h+t5FRt2xOc56FBntU8bd3ywCv1/FpI2fV7a+SH1ZXXEN910flPOAbRK6Y2dN3Qx1JkmiZiGvqeENSsFCHEh4c7MJgOyzLPuRRABAZVNrW386Qli8pmGDbD0HV+t5AQUi5oQkxD3x12w9BHM8tZN32362PH0IG8XfA8zmYKpqbFFKdc5iUzcd/15/PRpyoQaMrLp9cLM5nBksu8ZFH10XH77/8YkBgxO5qj9TarFxzXxAjqkSfutwN2Xcm5C7zZDGYQQ4yBY+RI9OMvz7mUKRc5HqeUXt82D4fdX33x2K0TVQamUkphLoGDEjuICC3CMtF2s/vw/ot5mY8vn6WU5Gv12lAEr2gYzCDlUkQGHQKH/XZz2O+7vgNE393pAm0iIyIB5iznJKUIB45MHVMM3LEPF4MaYsVcnSyy1s8ebREAQdFFPAy5Si9ALZEcuIHKxlvb5dV0gAwIkW59J9xaFzTErNke4K1vrT93BQpuav+bFwJAIGIzW1LCXJjR5w4J0VTO55xTJqZh6D3zm5dEYBy7/XbbdbEUSWl2AF9FUhFULVIA8eFuv9/vNpshBEKRcZ6nSS7jlHMhMJMyT/M4TiaSckGiTy9v05J1WS5LHuekRZFIAXIWABAz8WxJ69SPj3m120yE6BpnDdhBW7sUBsgUQyBmcNhTxZ+CAAQwJCZmy0UAus1GU0pzssBzLuxxvGgMrKWAWVablgWILks6nS9/92//+uHurqiIlFzIAEMouWRiFqnL5j5LogYU+O5w//WXX0vJ57ejw3ArDsUUQlBELKLTNOWS+3642x9UrY+hC9HxUCbqmAlRDHKROUkRDUx9oL4LQ+AQAiCKT0itjsjvk88p+7QIgrpUMQAYtLUNU1UPCBU2p1r0GxgBrrdX0XvuWOEqwLrCVZVr3LPC+vv/vy9uc8/qpKtZ37hY8CIJwYpqYy7H0MUuRwQTNSbaxk3OZV4SghHRdjv8+ssP7x7vmclUcu7PfZ9ymqZFL+NlSSllM8spv7xeFKGksizzkkopJeUiClbhx3ppdX6mHqu6boitaXbTCmqwBDa7A+v7jpjHcfJDSE5I4LEMcf/01B/2m74//vSxlLx/fNrttv12+Paf/rfj8GQmZqRqpkKBCEEEkUrO5CR3mgWgeNXffq2ITcvCRJ9Tmv7Xv/zm7/72w9N9kQKZkIJHM5FCRCLA7PHNPwrGrru/fyqSfzSYLqfWLii5MC5zylkNRORymVLJm83Gw/C8LFnFAALjEJkIFUBFp1SKKBP2kTdd6COHwNAcJCGwuZKx943AsClyuHX6NEkbdyqiQK71B0T1IBOCoSkh15ebI/dOzF5dI1FFNFuqXQe5V3d6Dd14PRPXx3vbC71iyf6tUHLy5Al9Y8EZ9IoQIjEjwGW8iKgUQcS+6/a77dvr6XQ8zSm5bsU4ZxE5T9M8LUWtqFopVmn1aibTnH514h5dvPVgNXmutSZWYgH1DAZUCakWmW5+oqYCxIioCiLZcy4T9SBpRZylo8zz01cfdveHy/PLdLmcxwsEvFxO5lNkhABYZUGKACr5vTUwU9IqKaBtrZ2Q0JCYDbGIFjUEyHn8n3/419/9p39/t9+JFNMiJQsHQhLMAIgoCGrkhIcIiH3fPz68E7XnzyG5uA8REqVi83yelrTMaVkWIALDE5/Ssnz85WWeZkJwvnA1SKJLKqkoEXaB+8Aup+SR3Wyl7qqYe+X2rF4R18UkA59RMbvRj3Oks9qKGxgSVO7/6u+rHdY/tI3N6voaRXRD8HHNAW4zgBs3uXaQbssydzSuduqqJb6SYmVJvl4j1mZbmkTZZZqej2+ilfy/fmTC2lpoJQJWVj4lbAwpWHtdgCDSph0QsfaOgYmM6s42EiNCSdlXPZG5nTIjZslFTJ3iAwhFPBkidRUT36pBBIBlGv/0zbcqlVQ7ny6fz2cRicQYgvkcbt0R9t1bD7VcJ46d1c33exAICAjNVPyB+Z0xe7mMv//2T7/7zb+LwdKyeoyq6mCmzExgQAZWPQ1z2O/uum5AxMDRZ2tSyqfz6eX19Zfl87RkMNAiry+v8zgdT6cyzQGRyTvizvUiRLSJFAPHQFWLEcBXLg3RAVkD1FaA0NWJAiBBZchBbfi8udxUq+9xHQiogZtW4hv/ayc7gSY5fq1+/H0rHEa1adRstMX0vzDF1XGu7+/1QvCRQZeq0gYZqpo6S4CPmdSBNChiAE2/o5Ua0LABu7p09AYxUWMd8DlwIkV0CW0MoePgFYK7F4C6mswtlDiJMIhKyVKkpkn105m07BsQjcldgVNZYtVGA1N1/9RSPehCrFmA4xwtDtXM1dPbGLz1ylqRca2llvPUARMBB4rR1VOeX0+vx9PT450VQsoGUKT0XSche6EVYgQItfAQUZG+73fbfQwxhFCLNpGnp3fv34/v3j1/9/33f/rhxx8/firLwqBsQqaAoICmNmXJRZgxBu4jd13sYqSG5DhCpAZSR+zB5wZ8mUIrY0MNWW4lAUGrAyEidI7bhiBDS5QNXSCs3kqqVdJNZolVMQLEqq/0b1q7t6s1ri7zLxwnrq+oxGMAEFyVR1Swvq7CqvWtoYqGae00GFRx3CpgX/1oiyX+cg4hdhGRvOpUNS3ZT23d0DbQItOSKmq8TgqQ3xcFAAKqd0FN/fIIpW2ZWz0O6IeePG1vC0cYWK4pbFuaRIC6OuZsa2ae+SMGZKe1r8hKQmLWXFwecm1DOQRpAAURuZBqiB0xTSn//Hz86sOTc9XHGGIIoiZpURWLilh5AXzXMYSw6frA0Yc369YDABKGwLvt5vH+/unp6Ztvv/3lp5/yNEYCBhKDopZLyUUNoAu06UIXA4fg5Q9jZRfzR1H5u7V6eh+sqv6krntAnbOzSqy/XgggNk4cvHGlNTez9Ut/4Zo1ErVhwesNtxpTbQ33t2DcreOE1pLFduGIEJD4WgFUC7WVuKyaf8Vi1y0T0yY9iIDrgun6i31QB8FW2vaVMkpbc78+b0Q0tDVNrhJEgDXBUKzTWw51oI9F+bVRQxCb0/ZLVIXKKOd4HiG6rwVfloPKCOzllBYLTP5snTLJ3bKJBGJPNOv8uidrqGbq6bMVEcwAQcwuWTwKeoOVQzA1p05ELGZKRBy6GHo/vYEjVa5Or63FzCU+LIbw8PBwOBy+/uKLf/3jv3z3x2/H1xfJUkRTKbmYmXWRQwgUmCudrgJQMdCGuzeXYT4jpw3P/EvPtaZ+BFUIgpl8lg4JyYeu1dwH11vdkKgaM2m1O2gWA1Wy+9Ykrj1QuO4bXd0qrtkqXOslNQMIfoIcFHA/4Y8EW+BTbaOyiCvBxNoZa1F9nT8ARwOaL/KOAbpktyGtVJrNE3qeBC1/bR+hJkHg5Fa3rn1NGLQ11xBx3QX09EDluhpYf+paollTP6nfN1W1AoimRlWWDwCsqCATuC6Lj3+hVdm06h8MDdCMmNOcUirUU8qFQzAtuRTnIo0x9v1mt9v3/cYNlOoGRXsHADMrLu7BrhGDzPzh/Ye7w/7Du8dv/vD7H/70Yz6PToDXRRpiiDF2zA5OO/zu8TG0FhGs8xCoq/34HSGAutJqoADMFJidv5KZXfpbnaaHzNePSK3uK5iBmrfSDUypqsyoGQFBW5harRXbI7uxSAODdZMJrwG/rot5IxqJQDVwDKZa1XEBWy9r3SgADq3DBUbEiOsZrWRU9QJW4/c82we/Ddb2N7QawVq6oLCOfpEUBwV9VNHM5Si6vjvsp5cXLeLKj+t5rElxjVGK2OgrbnqMyEziCJGfKN9Qg7prVtMEBEABIzNP9LniDz6dWdc13ZzNFOsX6MvhaErABJCWdJnmGNgAci6qMs9zEYkh9F3Xd7ELMbDLDVQc11o9a35hKgZiVlFFzx77fvvll7+6jOPHz8fl9SSqMfIQOUYOTB7Trfmu5ubqLVh5ofysqvojaoPrHsF8x44CkEsmVTkGf8wG5s7K6tq7Qht2r12g5tFAa/sDKmiDayMNWm/dh0wBoA0A6C26szr9a0dU1cBC4GDkV9+yzBXz9yQFVwqK1ZFdE4LqJAFahmotCBgYmppJqb/MBZkBEZCYQgzU93le+s32/a//5vP3352eP/nkm98/DhyGAZlXuoHmQ/2Wo62p/urHGxeeX7yWQmui0k5KayZXdwrY+hmInjzUil4rhQG1zNMqveD67mCEqqiqhJRULuOy7bsQ9ZxzkYKEfey7GBFRREQzC4OBS87VigRJwVRFck55RgAIaIDcVkSRHDp93O52hhRi6ALFyEhciRPBRF1WGOlKKltp3hXWR2k1ufSPpVp/DpGIkchtE4m9dFQzMedchIpauas29CE8n1wz5JrOX2MgoK9IA1R2xTYnbpW/pBXWWjmhK2TQ7iy0+RJ/r9CYxFo9ezPLX61hHUg0xXYckdZThboiKus0YU05tCqPQisIyTcfgTnsHx9ou3n788+B+fD4eH55eXv5bG1AC8yAKA5DWhYTqQOD1bdd8d0GSHjMaFnP6rO19javiVe7EbWkb6mtf2y8Ge+u1r/CBgBonrbadZxOGwsWqojlnEpOl1HUlJm2my1VsFZLkZQyGDEbMleCFwaj6ziJFEHweX8hysxBW7L3eH//b/7mr89vx+V4JIDaO1YpAMVA1Vu8GJmtldV+gNvYKtQWsHtvtSKVS9DpJzgEInZtEiBSF7kz8HWkla+UjMzZmKAen+Y/cS1IoO3lwU0BtP5XVrTJy6D66JzpxLHv+hRW6wsVWGnfamlus83V/66bMrjaH4A76ht7lurq6oPFa+FVk0VCNB8QE6MlA6CJ5mXG1kZyX4vMHAIA2rI45o/12lrN6G/rw93tTNj16AEAEhN4U98BPzPTumpOgOZdaPVHxL5eiTXpqEmVrSxbjfN//WDr/xTBEFR0zmVKScWQMDCbWS4lFAakEDRnAcgBkFwD07nNlCpMiBS7IS2jSVYzJEk5OTMCMyPS4/3h3ePdz9MIIoC1/WMAYi7dC4YE4Pp0gK5600YLPexXIohKKA5iQOs4FDMyM3GFPG+Ki3rP1JiaMmqlGSOX4VLzHN3M72ZlnKwv9lu5GkmN0c3D113VenbcsNHaipQ/7v8LuhRpugWd6nYAAAAASUVORK5CYII=)" + ], + "metadata": { + "id": "gW4cE8bhXS-d" + } + }, + { + "cell_type": "code", + "source": [ + "image_data = (periodic_impulse | beam.Map(lambda x: \"Cat-with-beanie.jpg\")\n", + " | \"ReadImage\" >> beam.Map(lambda image_name: preprocess_image(\n", + " image_name=image_name, image_dir='https://storage.googleapis.com/apache-beam-samples/image_captioning/')))" + ], + "metadata": { + "id": "dGg11TpV_aV6" + }, + "execution_count": null, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "3. Pass the images to the RunInference `PTransform`. RunInference takes `model_handler` and `model_metadata_pcoll` as input parameters.\n", + " * `model_metadata_pcoll` is a side input `PCollection` to the RunInference `PTransform`. This side input is used to update the `model_uri` in the `model_handler` without needing to stop the Apache Beam pipeline\n", + " * Use `WatchFilePattern` as side input to watch a `file_pattern` matching `.keras` files. In this case, the `file_pattern` is `'gs://BUCKET_NAME/dataflow/*keras'`.\n", + "\n" + ], + "metadata": { + "id": "eB0-ewd-BCKE" + } + }, + { + "cell_type": "code", + "source": [ + " # The side input used to watch for the .keras file and update the model_uri of the TFModelHandlerTensor.\n", + "file_pattern = dataflow_gcs_location + '/*.keras'\n", + "side_input_pcoll = (\n", + " pipeline\n", + " | \"WatchFilePattern\" >> WatchFilePattern(file_pattern=file_pattern,\n", + " interval=side_input_fire_interval,\n", + " stop_timestamp=end_timestamp))\n", + "inferences = (\n", + " image_data\n", + " | \"ApplyWindowing\" >> beam.WindowInto(beam.window.FixedWindows(10))\n", + " | \"RunInference\" >> RunInference(model_handler=model_handler,\n", + " model_metadata_pcoll=side_input_pcoll))" + ], + "metadata": { + "id": "_AjvvexJ_hUq" + }, + "execution_count": null, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "4. Post-process the `PredictionResult` object.\n", + "When the inference is complete, RunInference outputs a `PredictionResult` object that contains the fields `example`, `inference`, and `model_id`. The `model_id` field identifies the model used to run the inference. The `PostProcessor` returns the predicted label and the model ID used to run the inference on the predicted label." + ], + "metadata": { + "id": "lTA4wRWNDVis" + } + }, + { + "cell_type": "code", + "source": [ + "post_processor = (\n", + " inferences\n", + " | \"PostProcessResults\" >> beam.ParDo(PostProcessor())\n", + " | \"LogResults\" >> beam.Map(logging.info))" + ], + "metadata": { + "id": "9TB76fo-_vZJ" + }, + "execution_count": null, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "### Watch for the model update\n", + "\n", + "After the pipeline starts processing data and when you see output emitted from the RunInference `PTransform`, upload a `resnet152` model saved in `.keras` format to a Google Cloud Storage bucket location that matches the `file_pattern` you defined earlier.\n" + ], + "metadata": { + "id": "wYp-mBHHjOjA" + } + }, + { + "cell_type": "code", + "source": [ + "model = tf.keras.applications.resnet.ResNet152()\n", + "model.save('resnet152_weights_tf_dim_ordering_tf_kernels.keras')\n", + "# Replace the `BUCKET_NAME` with the actual bucket name.\n", + "!gsutil cp resnet152_weights_tf_dim_ordering_tf_kernels.keras gs:///resnet152_weights_tf_dim_ordering_tf_kernels.keras" + ], + "metadata": { + "id": "FpUfNBSWH9Xy" + }, + "execution_count": null, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "## Run the pipeline\n", + "\n", + "Use the following code to run the pipeline." + ], + "metadata": { + "id": "_ty03jDnKdKR" + } + }, + { + "cell_type": "code", + "source": [ + "# Run the pipeline.\n", + "result = pipeline.run().wait_until_finish()" + ], + "metadata": { + "id": "wd0VJLeLEWBU" + }, + "execution_count": null, + "outputs": [] + } + ] +} \ No newline at end of file From 6f4e2852311db381a622dec3ebf26654c9372806 Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Wed, 4 Oct 2023 17:52:46 -0400 Subject: [PATCH 025/435] Fix spotless on master (#28831) --- .../sdk/io/gcp/bigquery/BigQueryIOMetadata.java | 4 ++-- .../sdk/io/gcp/bigquery/BigQueryServicesImpl.java | 13 ++++++++----- 2 files changed, 10 insertions(+), 7 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOMetadata.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOMetadata.java index 0b5e063c0b5b..1893418dedb3 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOMetadata.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOMetadata.java @@ -32,8 +32,8 @@ final class BigQueryIOMetadata { private @Nullable String beamWorkerId; - private BigQueryIOMetadata(@Nullable String beamJobId, @Nullable String beamJobName, - @Nullable String beamWorkerId) { + private BigQueryIOMetadata( + @Nullable String beamJobId, @Nullable String beamJobName, @Nullable String beamWorkerId) { this.beamJobId = beamJobId; this.beamJobName = beamJobName; this.beamWorkerId = beamWorkerId; diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java index 04a665ac9947..1b6cc555511d 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java @@ -1364,11 +1364,14 @@ public StreamAppendClient getStreamAppendClient( .setChannelsPerCpu(2) .build(); - String traceId = String.format("Dataflow:%s:%s:%s", - bqIOMetadata.getBeamJobName() == null ? options.getJobName() - : bqIOMetadata.getBeamJobName(), - bqIOMetadata.getBeamJobId() == null ? "" : bqIOMetadata.getBeamJobId(), - bqIOMetadata.getBeamWorkerId() == null ? "" : bqIOMetadata.getBeamWorkerId()); + String traceId = + String.format( + "Dataflow:%s:%s:%s", + bqIOMetadata.getBeamJobName() == null + ? options.getJobName() + : bqIOMetadata.getBeamJobName(), + bqIOMetadata.getBeamJobId() == null ? "" : bqIOMetadata.getBeamJobId(), + bqIOMetadata.getBeamWorkerId() == null ? "" : bqIOMetadata.getBeamWorkerId()); StreamWriter streamWriter = StreamWriter.newBuilder(streamName, newWriteClient) From 161cd6b1c8de9a2a68b6aaf81cc5e13f827b6f50 Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Wed, 4 Oct 2023 20:18:54 -0400 Subject: [PATCH 026/435] Gradle 8 support (#28756) * Gradle 8 support * Make gradle help --scan green * Replicate defunct com.palantir.docker and docker-run plugins * Fixes * bump errorprone plugin * Fix copySdkHarnessLauncher dependency * Link to original plugin src --- buildSrc/build.gradle.kts | 17 +- .../beam/gradle/BeamDockerPlugin.groovy | 325 ++++++++++++++++++ .../beam/gradle/BeamDockerRunPlugin.groovy | 143 ++++++++ .../beam/gradle/BeamModulePlugin.groovy | 29 +- .../beam/gradle/VendorJavaPlugin.groovy | 2 +- gradle/wrapper/gradle-wrapper.jar | Bin 61624 -> 63721 bytes gradle/wrapper/gradle-wrapper.properties | 3 +- gradlew | 19 +- playground/kafka-emulator/build.gradle | 4 +- sdks/java/container/common.gradle | 2 + .../io/google-cloud-platform/build.gradle | 4 +- 11 files changed, 511 insertions(+), 37 deletions(-) create mode 100644 buildSrc/src/main/groovy/org/apache/beam/gradle/BeamDockerPlugin.groovy create mode 100644 buildSrc/src/main/groovy/org/apache/beam/gradle/BeamDockerRunPlugin.groovy diff --git a/buildSrc/build.gradle.kts b/buildSrc/build.gradle.kts index 0ca748e3eb04..968829caeb8b 100644 --- a/buildSrc/build.gradle.kts +++ b/buildSrc/build.gradle.kts @@ -44,20 +44,19 @@ dependencies { implementation("gradle.plugin.com.github.johnrengelman:shadow:7.1.1") implementation("com.github.spotbugs.snom:spotbugs-gradle-plugin:5.0.14") - runtimeOnly("com.google.protobuf:protobuf-gradle-plugin:0.8.13") // Enable proto code generation - runtimeOnly("com.github.davidmc24.gradle-avro-plugin:gradle-avro-plugin:0.16.0") // Enable Avro code generation - runtimeOnly("com.diffplug.spotless:spotless-plugin-gradle:5.6.1") // Enable a code formatting plugin - runtimeOnly("com.palantir.gradle.docker:gradle-docker:0.34.0") // Enable building Docker containers - runtimeOnly("gradle.plugin.com.dorongold.plugins:task-tree:1.5") // Adds a 'taskTree' task to print task dependency tree - runtimeOnly("gradle.plugin.com.github.johnrengelman:shadow:7.1.1") // Enable shading Java dependencies + runtimeOnly("com.google.protobuf:protobuf-gradle-plugin:0.8.13") // Enable proto code generation + runtimeOnly("com.github.davidmc24.gradle-avro-plugin:gradle-avro-plugin:0.16.0") // Enable Avro code generation + runtimeOnly("com.diffplug.spotless:spotless-plugin-gradle:5.6.1") // Enable a code formatting plugin + runtimeOnly("gradle.plugin.com.dorongold.plugins:task-tree:1.5") // Adds a 'taskTree' task to print task dependency tree + runtimeOnly("gradle.plugin.com.github.johnrengelman:shadow:7.1.1") // Enable shading Java dependencies runtimeOnly("net.linguica.gradle:maven-settings-plugin:0.5") runtimeOnly("gradle.plugin.io.pry.gradle.offline_dependencies:gradle-offline-dependencies-plugin:0.5.0") // Enable creating an offline repository - runtimeOnly("net.ltgt.gradle:gradle-errorprone-plugin:1.2.1") // Enable errorprone Java static analysis + runtimeOnly("net.ltgt.gradle:gradle-errorprone-plugin:3.1.0") // Enable errorprone Java static analysis runtimeOnly("org.ajoberstar.grgit:grgit-gradle:4.1.1") // Enable website git publish to asf-site branch - runtimeOnly("com.avast.gradle:gradle-docker-compose-plugin:0.16.12") // Enable docker compose tasks + runtimeOnly("com.avast.gradle:gradle-docker-compose-plugin:0.16.12") // Enable docker compose tasks runtimeOnly("ca.cutterslade.gradle:gradle-dependency-analyze:1.8.3") // Enable dep analysis runtimeOnly("gradle.plugin.net.ossindex:ossindex-gradle-plugin:0.4.11") // Enable dep vulnerability analysis - runtimeOnly("org.checkerframework:checkerframework-gradle-plugin:0.6.33") // Enable enhanced static checking plugin + runtimeOnly("org.checkerframework:checkerframework-gradle-plugin:0.6.33") // Enable enhanced static checking plugin } // Because buildSrc is built and tested automatically _before_ gradle diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamDockerPlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamDockerPlugin.groovy new file mode 100644 index 000000000000..442b35439cae --- /dev/null +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamDockerPlugin.groovy @@ -0,0 +1,325 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * License); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an AS IS BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.gradle + +import java.util.regex.Pattern +import org.gradle.api.GradleException +import org.gradle.api.Plugin +import org.gradle.api.Project +import org.gradle.api.Task +import org.gradle.api.file.CopySpec +import org.gradle.api.logging.LogLevel +import org.gradle.api.logging.Logger +import org.gradle.api.logging.Logging +import org.gradle.api.tasks.Copy +import org.gradle.api.tasks.Delete +import org.gradle.api.tasks.Exec + +/** + * A gradle plug-in interacting with docker. Originally replicated from + * com.palantir.docker plugin. + */ +class BeamDockerPlugin implements Plugin { + private static final Logger logger = Logging.getLogger(BeamDockerPlugin.class) + private static final Pattern LABEL_KEY_PATTERN = Pattern.compile('^[a-z0-9.-]*$') + + static class DockerExtension { + Project project + + private static final String DEFAULT_DOCKERFILE_PATH = 'Dockerfile' + String name = null + File dockerfile = null + String dockerComposeTemplate = 'docker-compose.yml.template' + String dockerComposeFile = 'docker-compose.yml' + Set dependencies = [] as Set + Set tags = [] as Set + Map namedTags = [:] + Map labels = [:] + Map buildArgs = [:] + boolean pull = false + boolean noCache = false + String network = null + boolean buildx = false + Set platform = [] as Set + boolean load = false + boolean push = false + String builder = null + + File resolvedDockerfile = null + File resolvedDockerComposeTemplate = null + File resolvedDockerComposeFile = null + + // The CopySpec defining the Docker Build Context files + final CopySpec copySpec + + DockerExtension(Project project) { + this.project = project + this.copySpec = project.copySpec() + } + + void resolvePathsAndValidate() { + if (dockerfile != null) { + resolvedDockerfile = dockerfile + } else { + resolvedDockerfile = project.file(DEFAULT_DOCKERFILE_PATH) + } + resolvedDockerComposeFile = project.file(dockerComposeFile) + resolvedDockerComposeTemplate = project.file(dockerComposeTemplate) + } + + void dependsOn(Task... args) { + this.dependencies = args as Set + } + + Set getDependencies() { + return dependencies + } + + void files(Object... files) { + copySpec.from(files) + } + + void tags(String... args) { + this.tags = args as Set + } + + Set getTags() { + return this.tags + project.getVersion().toString() + } + + Set getPlatform() { + return platform + } + + void platform(String... args) { + this.platform = args as Set + } + } + + @Override + void apply(Project project) { + DockerExtension ext = project.extensions.create('docker', DockerExtension, project) + + Delete clean = project.tasks.create('dockerClean', Delete, { + group = 'Docker' + description = 'Cleans Docker build directory.' + }) + + Copy prepare = project.tasks.create('dockerPrepare', Copy, { + group = 'Docker' + description = 'Prepares Docker build directory.' + dependsOn clean + }) + + Exec exec = project.tasks.create('docker', Exec, { + group = 'Docker' + description = 'Builds Docker image.' + dependsOn prepare + }) + + Task tag = project.tasks.create('dockerTag', { + group = 'Docker' + description = 'Applies all tags to the Docker image.' + dependsOn exec + }) + + Task pushAllTags = project.tasks.create('dockerTagsPush', { + group = 'Docker' + description = 'Pushes all tagged Docker images to configured Docker Hub.' + }) + + project.tasks.create('dockerPush', { + group = 'Docker' + description = 'Pushes named Docker image to configured Docker Hub.' + dependsOn pushAllTags + }) + + project.afterEvaluate { + ext.resolvePathsAndValidate() + String dockerDir = "${project.buildDir}/docker" + clean.delete dockerDir + + prepare.with { + with ext.copySpec + from(ext.resolvedDockerfile) { + rename { fileName -> + fileName.replace(ext.resolvedDockerfile.getName(), 'Dockerfile') + } + } + into dockerDir + } + + exec.with { + workingDir dockerDir + commandLine buildCommandLine(ext) + dependsOn ext.getDependencies() + logging.captureStandardOutput LogLevel.INFO + logging.captureStandardError LogLevel.ERROR + } + + Map tags = ext.namedTags.collectEntries { taskName, tagName -> + [ + generateTagTaskName(taskName), + [ + tagName: tagName, + tagTask: { + -> tagName } + ] + ] + } + + if (!ext.tags.isEmpty()) { + ext.tags.each { unresolvedTagName -> + String taskName = generateTagTaskName(unresolvedTagName) + + if (tags.containsKey(taskName)) { + throw new IllegalArgumentException("Task name '${taskName}' is existed.") + } + + tags[taskName] = [ + tagName: unresolvedTagName, + tagTask: { + -> computeName(ext.name, unresolvedTagName) } + ] + } + } + + tags.each { taskName, tagConfig -> + Exec tagSubTask = project.tasks.create('dockerTag' + taskName, Exec, { + group = 'Docker' + description = "Tags Docker image with tag '${tagConfig.tagName}'" + workingDir dockerDir + commandLine 'docker', 'tag', "${-> ext.name}", "${-> tagConfig.tagTask()}" + dependsOn exec + }) + tag.dependsOn tagSubTask + + Exec pushSubTask = project.tasks.create('dockerPush' + taskName, Exec, { + group = 'Docker' + description = "Pushes the Docker image with tag '${tagConfig.tagName}' to configured Docker Hub" + workingDir dockerDir + commandLine 'docker', 'push', "${-> tagConfig.tagTask()}" + dependsOn tagSubTask + }) + pushAllTags.dependsOn pushSubTask + } + } + } + + private List buildCommandLine(DockerExtension ext) { + List buildCommandLine = ['docker'] + if (ext.buildx) { + buildCommandLine.addAll(['buildx', 'build']) + if (!ext.platform.isEmpty()) { + buildCommandLine.addAll('--platform', String.join(',', ext.platform)) + } + if (ext.load) { + buildCommandLine.add '--load' + } + if (ext.push) { + buildCommandLine.add '--push' + if (ext.load) { + throw new Exception("cannot combine 'push' and 'load' options") + } + } + if (ext.builder != null) { + buildCommandLine.addAll('--builder', ext.builder) + } + } else { + buildCommandLine.add 'build' + } + if (ext.noCache) { + buildCommandLine.add '--no-cache' + } + if (ext.getNetwork() != null) { + buildCommandLine.addAll('--network', ext.network) + } + if (!ext.buildArgs.isEmpty()) { + for (Map.Entry buildArg : ext.buildArgs.entrySet()) { + buildCommandLine.addAll('--build-arg', "${buildArg.getKey()}=${buildArg.getValue()}" as String) + } + } + if (!ext.labels.isEmpty()) { + for (Map.Entry label : ext.labels.entrySet()) { + if (!label.getKey().matches(LABEL_KEY_PATTERN)) { + throw new GradleException(String.format("Docker label '%s' contains illegal characters. " + + "Label keys must only contain lowercase alphanumberic, `.`, or `-` characters (must match %s).", + label.getKey(), LABEL_KEY_PATTERN.pattern())) + } + buildCommandLine.addAll('--label', "${label.getKey()}=${label.getValue()}" as String) + } + } + if (ext.pull) { + buildCommandLine.add '--pull' + } + buildCommandLine.addAll(['-t', "${-> ext.name}", '.']) + logger.debug("${buildCommandLine}" as String) + return buildCommandLine + } + + private static String computeName(String name, String tag) { + int firstAt = tag.indexOf("@") + + String tagValue + if (firstAt > 0) { + tagValue = tag.substring(firstAt + 1, tag.length()) + } else { + tagValue = tag + } + + if (tagValue.contains(':') || tagValue.contains('/')) { + // tag with ':' or '/' -> force use the tag value + return tagValue + } else { + // tag without ':' and '/' -> replace the tag part of original name + int lastColon = name.lastIndexOf(':') + int lastSlash = name.lastIndexOf('/') + + int endIndex; + + // image_name -> this should remain + // host:port/image_name -> this should remain. + // host:port/image_name:v1 -> v1 should be replaced + if (lastColon > lastSlash) endIndex = lastColon + else endIndex = name.length() + + return name.substring(0, endIndex) + ":" + tagValue + } + } + + private static String generateTagTaskName(String name) { + String tagTaskName = name + int firstAt = name.indexOf("@") + + if (firstAt > 0) { + // Get substring of task name + tagTaskName = name.substring(0, firstAt) + } else if (firstAt == 0) { + // Task name must not be empty + throw new GradleException("Task name of docker tag '${name}' must not be empty.") + } else if (name.contains(':') || name.contains('/')) { + // Tags which with repo or name must have a task name + throw new GradleException("Docker tag '${name}' must have a task name.") + } + + StringBuffer sb = new StringBuffer(tagTaskName) + // Uppercase the first letter of task name + sb.replace(0, 1, tagTaskName.substring(0, 1).toUpperCase()); + return sb.toString() + } +} diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamDockerRunPlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamDockerRunPlugin.groovy new file mode 100644 index 000000000000..5297c7018139 --- /dev/null +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamDockerRunPlugin.groovy @@ -0,0 +1,143 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * License); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an AS IS BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.gradle + +import org.gradle.api.Plugin +import org.gradle.api.Project +import org.gradle.api.tasks.Exec + +/** + * A gradle plug-in handling 'docker run' command. Originally replicated from + * com.palantir.docker-run plugin. + */ +class BeamDockerRunPlugin implements Plugin { + + /** A class defining the configurations of dockerRun task. */ + static class DockerRunExtension { + String name + String image + Set ports = [] as Set + Map env = [:] + List arguments = [] + Map volumes = [:] + boolean daemonize = true + boolean clean = false + + public String getName() { + return name + } + + public void setName(String name) { + this.name = name + } + } + + @Override + void apply(Project project) { + DockerRunExtension ext = project.extensions.create('dockerRun', DockerRunExtension) + + Exec dockerRunStatus = project.tasks.create('dockerRunStatus', Exec, { + group = 'Docker Run' + description = 'Checks the run status of the container' + }) + + Exec dockerRun = project.tasks.create('dockerRun', Exec, { + group = 'Docker Run' + description = 'Runs the specified container with port mappings' + }) + + Exec dockerStop = project.tasks.create('dockerStop', Exec, { + group = 'Docker Run' + description = 'Stops the named container if it is running' + ignoreExitValue = true + }) + + Exec dockerRemoveContainer = project.tasks.create('dockerRemoveContainer', Exec, { + group = 'Docker Run' + description = 'Removes the persistent container associated with the Docker Run tasks' + ignoreExitValue = true + }) + + project.afterEvaluate { + /** Inspect status of docker. */ + dockerRunStatus.with { + standardOutput = new ByteArrayOutputStream() + commandLine 'docker', 'inspect', '--format={{.State.Running}}', ext.name + doLast { + if (standardOutput.toString().trim() != 'true') { + println "Docker container '${ext.name}' is STOPPED." + return 1 + } else { + println "Docker container '${ext.name}' is RUNNING." + } + } + } + + /** + * Run a docker container. See {@link DockerRunExtension} for supported + * arguments. + * + * Replication of dockerRun task of com.palantir.docker-run plugin. + */ + dockerRun.with { + List args = new ArrayList() + args.addAll(['docker', 'run']) + + if (ext.daemonize) { + args.add('-d') + } + if (ext.clean) { + args.add('--rm') + } else { + finalizedBy dockerRunStatus + } + for (String port : ext.ports) { + args.add('-p') + args.add(port) + } + for (Map.Entry volume : ext.volumes.entrySet()) { + File localFile = project.file(volume.key) + + if (!localFile.exists()) { + logger.error("ERROR: Local folder ${localFile} doesn't exist. Mounted volume will not be visible to container") + throw new IllegalStateException("Local folder ${localFile} doesn't exist.") + } + args.add('-v') + args.add("${localFile.absolutePath}:${volume.value}") + } + args.addAll(ext.env.collect{ k, v -> ['-e', "${k}=${v}"]}.flatten()) + args.add('--name') + args.add(ext.name) + if (!ext.arguments.isEmpty()) { + args.addAll(ext.arguments) + } + args.add(ext.image) + + commandLine args + } + + dockerStop.with { + commandLine 'docker', 'stop', ext.name + } + + dockerRemoveContainer.with { + commandLine 'docker', 'rm', ext.name + } + } + } +} diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index c31482d577e0..c7a62237086e 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -23,7 +23,6 @@ import static java.util.UUID.randomUUID import com.github.jengelman.gradle.plugins.shadow.tasks.ShadowJar import groovy.json.JsonOutput import groovy.json.JsonSlurper -import java.net.ServerSocket import java.util.logging.Logger import org.gradle.api.attributes.Category import org.gradle.api.GradleException @@ -1252,7 +1251,7 @@ class BeamModulePlugin implements Plugin { if (configuration.shadowClosure) { // Ensure that tests are packaged and part of the artifact set. project.task('packageTests', type: Jar) { - classifier = 'tests-unshaded' + archiveClassifier = 'tests-unshaded' from project.sourceSets.test.output } project.artifacts.archives project.packageTests @@ -1560,13 +1559,13 @@ class BeamModulePlugin implements Plugin { } } - // Always configure the shadowJar classifier and merge service files. + // Always configure the shadowJar archiveClassifier and merge service files. if (configuration.shadowClosure) { // Only set the classifer on the unshaded classes if we are shading. - project.jar { classifier = "unshaded" } + project.jar { archiveClassifier = "unshaded" } project.shadowJar({ - classifier = null + archiveClassifier = null mergeServiceFiles() zip64 true into("META-INF/") { @@ -1575,11 +1574,11 @@ class BeamModulePlugin implements Plugin { } } << configuration.shadowClosure) - // Always configure the shadowTestJar classifier and merge service files. + // Always configure the shadowTestJar archiveClassifier and merge service files. project.task('shadowTestJar', type: ShadowJar, { group = "Shadow" description = "Create a combined JAR of project and test dependencies" - classifier = "tests" + archiveClassifier = "tests" from project.sourceSets.test.output configurations = [ project.configurations.testRuntimeMigration @@ -1639,7 +1638,7 @@ class BeamModulePlugin implements Plugin { project.tasks.register("testJar", Jar) { group = "Jar" description = "Create a JAR of test classes" - classifier = "tests" + archiveClassifier = "tests" from project.sourceSets.test.output zip64 true exclude "META-INF/INDEX.LIST" @@ -1794,18 +1793,18 @@ class BeamModulePlugin implements Plugin { project.task('sourcesJar', type: Jar) { from project.sourceSets.main.allSource - classifier = 'sources' + archiveClassifier = 'sources' } project.artifacts.archives project.sourcesJar project.task('testSourcesJar', type: Jar) { from project.sourceSets.test.allSource - classifier = 'test-sources' + archiveClassifier = 'test-sources' } project.artifacts.archives project.testSourcesJar project.task('javadocJar', type: Jar, dependsOn: project.javadoc) { - classifier = 'javadoc' + archiveClassifier = 'javadoc' from project.javadoc.destinationDir } project.artifacts.archives project.javadocJar @@ -1915,8 +1914,8 @@ class BeamModulePlugin implements Plugin { def dependencyNode = dependenciesNode.appendNode('dependency') def appendClassifier = { dep -> dep.artifacts.each { art -> - if (art.hasProperty('classifier')) { - dependencyNode.appendNode('classifier', art.classifier) + if (art.hasProperty('archiveClassifier')) { + dependencyNode.appendNode('archiveClassifier', art.archiveClassifier) } } } @@ -2210,7 +2209,7 @@ class BeamModulePlugin implements Plugin { /** ***********************************************************************************************/ project.ext.applyDockerNature = { - project.apply plugin: "com.palantir.docker" + project.apply plugin: BeamDockerPlugin project.docker { noCache true } project.tasks.create(name: "copyLicenses", type: Copy) { from "${project.rootProject.projectDir}/LICENSE" @@ -2222,7 +2221,7 @@ class BeamModulePlugin implements Plugin { } project.ext.applyDockerRunNature = { - project.apply plugin: "com.palantir.docker-run" + project.apply plugin: BeamDockerRunPlugin } /** ***********************************************************************************************/ diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/VendorJavaPlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/VendorJavaPlugin.groovy index 061ccf27cce2..97d96e6cf1eb 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/VendorJavaPlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/VendorJavaPlugin.groovy @@ -126,7 +126,7 @@ artifactId=${project.name} } config.exclusions.each { exclude it } - classifier = null + archiveClassifier = null mergeServiceFiles() zip64 true exclude "META-INF/INDEX.LIST" diff --git a/gradle/wrapper/gradle-wrapper.jar b/gradle/wrapper/gradle-wrapper.jar index afba109285af78dbd2a1d187e33ac4f87c76e392..7f93135c49b765f8051ef9d0a6055ff8e46073d8 100644 GIT binary patch delta 41204 zcmZ5{b95%((seSiZQHhO+qRP@<}68$AGx`lh~)fFsaj;MiX97^(}w;JdeIg_EK=f{qUxfgB1;VggKi&SA9v8P52JERa$mc|$lmVvfx{qi69Qt` z*fZ)^M2W{1fVoSt`s`K{cHR_Ov(<|-{;q0kYpPXc^iz@|EJ@}9Rkl^$`6gZKo4=jl zD2P(#uTcD*y0DWmpF`Dtw^s9E&tgueEt9jYdg)|4^CH zb}H5tkozJO-=TYiQxbjRT%6PCU;JFRQPXAQ>rlRozg33`4{|4|Q{K zP#?n!$RS}A3u8%zoT~YzxuAqLNKTa(NWlJ4XU{EHZHY-(GB_2uuL{a8BMNcj)?=dUUT2HO%1h(mqG)ntl zN?SX{UvgL}$DJsYuN~%ASKh2fJrBPH#2??t43t8?^fOWdaMj%wk$O`DK4(tRuA(&E zog=Ry-f5N`!=ByQeZ>yqokNEb4OV)~d*KM!+n@>L3iD=%hcWl5GV|Tcwvb**xo{vG!%lw${AnQ~eWceyLLtO0ikN#30gs-w0?6D+m(Pg;;(saJJH6dgz zPtR9$S)TMwL6Y4dX7dYUtY^k@&mLj>shqlfVB>uT4%k z-sD&k5#S$1G!f+SeqV-O07FX!@mC%6H?4gT42hV?{rCiRc9Cr9B1@ZjfX@!wme?JN zAJ(4)af-zesN2Gr=Jn#7mg9j8%5Jvi=KRdf+^w(o&rhoFI@|08W-G$DW;^7um(;k@ zrb`3p^aRVime{Nq^@gWKx`2>bX7zjX*(w=Bcc4S{A@7F|ytuV3;DP(RjMa4RmukeWjWwVyaGM*D6m`mn7ZGF34w6Gb!;w3^St z3XgDy{pdd{y~uiAiiTGa2wO@_XU;qFfTIXAZ1RMapg5FqfM@t-DJO(?zaynola?z< zq^^3=9HQZI#n>+*T*@Eef3h6)^xyrwTYa3S(|cxi6h6LV6~ufKNVoA3J4aC#kWj^9 zU$rtnC%FQ(!JWlPz7l4OHcH%})DUBe?Ui1bJ3TXHGHytpNOUMTkK>O63oL1f0R~Vl z5Q&~zYZ~evszs-g;%QS1E$G6>(o=zr5zmFhgtr72k-dOTT1h;>q0$c5&3}By18rDv za{zTEQI@fiS&|kEha>S}so7LsIpivt5vVHE)F!Z@B(20{Xj&vc)i}Ts+cmgXxl^-r zfwK2C3bRXuS7T6w6&q}%IY$i(=8BOF%1u8n14+J?DQ}GLQC#%nt^E3w&nfRL0C{8o z1kK9$eh2k$?D=o(;&vtePX11A3JhkVfk_W3;pVPI(@owrFYG{UOG!MTtY45i(<@=S zN=P|BG2-(N7nJ4OA>%O~hs;#8d_UEH{2?oTcEBiMv{=Wp@WM@3;N z0{Wcat;=K&itA=*;~J(LXVs)n9~I^r%a4*|S$W0d?v?Q`{73J$KUQ_4q5`Sm3BFa? zV(eIUY#=VrrlfmrNrXeQej79wf^Or9m^X&%-g`DZ;Yi7Q4g&5ll^1FyD)W@D)KrVh zl|juxjkCEccS-qdV|;zxQr12&?HgSGVY)!2YK($@QY2deG%8D@JL$e1pta(|J1t#f zUVXCDHNM8*jvht|y!My(A~`L3>E<-DLux~cq>XNop^vVrBZxb}{nb5fA()O1T|^S(dc&uFNXX z1sOhL*eHNaulR)m@PE4EooRR|dslppGSL#gzaw^n>yc)xGtbqT1u>8fY9^T3tzg#i zEiiQ#AQn{r#eEUXY8>4+Mrqk_>>(;GWqA*KMTOm5@A`nZy=B$H-C?( zlF23*1jz}fkjVi@l)zFo10^&Obb)3fg9^P$h^iuJQ;Z*^a6Q;+qAQe?3Q<`pw}KAg zydA$doAnNj?u2d+;V1>M^FI~FysSLcf+g$@#ZKq8d0w`C6|MR|Uw=akFnT;VD^Hq9 zGI0`KoFoRj=k7WyyDED&OeSn42gbbM%*0-x1h2w>1ew$^fC2Ap-F{$PBzXEyrTj37 z|B%v`@t;f5{H-YoPp3c_sfip(oYcsVma9E!ya2Bhu7Ag^4(~_@9b)^=9|exlV;ye0 zkAQyjF?9L1MD~jY>AmX`ua2~kR*f1DUXh7s5&IJF3N2tvARh{hNrID-Z54AhWLP_F zFm8P-mQDjHL1nr5DLdug=(vCHfzCO_8J zEp}ADT4%<15ZL|G=AWR}ikGH-x5cS%WlZt>I`kA<%!uoz)|GAv_cah!+Y7h`@it zu8EEaceVwi{ki&o3?rRBTObvKrW@02mSpQ zD^OwQTHZFUnC}|{%QT@iPD87>s#7FYY?`8-wKN+V%kaw0n!Qwe`tejf7W4-uxMOEn z1(Vn|#vOg*_XsP}L^5eoi@55}u8>pdAD)HjFyCfJ{bvuXw4_?hqm*=Twu)n|8EDp} z+eYBWOMCAP%?4JsQxu}o<$vt%1F1M%3xE{do_y6?#u;H-jLua7VHFrsRou*Me7~Cn z1C`5D3zl~V=mjbZ4Z8f}+vsFcsxmOvll-$BZaszJJA)xbGmlU*&%|6w=(r4LQP#B8qZL4?#bw;rFwrUZe%Cl!17wp@m_D+-Nz5%B% zt?XT&zh679<@}P7y-eRZC5R&rPv$LF|5=J}QFnrP@w`@`o?^n3b3RXV&PMS=L{r+! za7Y9Oi2VBUB6J$%+V116)8xd1RLkFT*ANt*svo|*ApHGBmxixy&&1idy8M>im6v+s zJJG|fRAA@k#dU*&gHSrR)2GZ}tv0Z_0F(hXtyrUQ8WJo|a{ ziMWa`=&FqTs``tea&48@-F9e+V6{bmb@ln8%ChSG*rBYNCc8(^^J?+vcr*X*TlY&* zP`b0Uu~>NJh1l@*S4;+P70*S3Lo7CoK;d@+>MMofl3$_1TyK>=!U7BZw?Q zhMTsbi~7e-*pV{h+ncBqVqK6o#QCoWmO$-zcY?U8;xN7nN;fJ;JVN(eDNSd5D)gSH zWK>0(S2{&H?qD%k!{JNVuQ3tZ8$}9(;3+1vRTU|By?jbx*xdmg2%wem?uec@=pCKR zAWz(=cMrn>0cfkX0ZqqumOvoKy0-yri$e3lL`0*=|UF=Xo;nNB$R++7AqF1I$2 zZvCA#{pqrH;X7TditMr4QzKe8tYz#nZ6q5vj(Ua#bB__qb=;@iZ@p%Qg^;6D$z44A zh(2vy6$HmZxDhoJsL3VEmK6%>Dlcc!l(mcUV1(oOxE z`#HlgHd3=fPzjTlsAlQngunQjxoK8k2*4DNg|*W#mnx?YRFxRP7EwrMqU2oRtpMtL zxPS_E(=%Mu@}{MByzHWw_iUI8VR?cgJ+c;FlPE8-W=Lmm)AL@i#FS3`Nlhz; z;VGl5;Zpo)d6Bv+f3=idnJQ{bP%GC!Xo6b_hE|$O4RtQ1jx#G)d`K!=l&<^(h*HRs zB7rxC9LE_a62%!yf|{!|a{rJQm%q?~j7l6=D332v${pu@2oAIzsn#Y}EYi#wau{h| zT8}|=`?eb;fUz6p&v2*t%b~K}6e!jLKTlnsUkmQxi4+N7N)oF?5 zTBO*$S5&^zn+N6;KzZ&f-aD|>ic#ydB&->9(A|2Jp9(0;J(sv_dtpCp-h&4Q4`;Z; ztbWGaBh8)gv1G?HF&5~?h?M$s*1Sf28_pImEJRH-6bfwR%*v!e)#l5NRLD3#{KS_? z6i(c44`X6_Mr|W0@*mtFl(dVhZY50MO0@|5h4iNR^AspN6*hcRG`6RM{DAR0fzaxx zV^i-Je6DP8BQrssMCdS^h?JD)5qQ7|;!N=(g7m^(s~Gl)4Og&J6psP1 zS#-)hqqC_6Ngq1M9Xll{ONfA$!bY2bGt!YanL@r}o`q{yizFpUBV|F>HmDP_qul~O z^GDy4;p&fuVEdH)0HvGx#b1NSl4Sbqi(j-^TH$Q3^GH~D=O2WcMmU9To-E(KfTN=~ zV)vf^#83N&^X~F*ARv#BARzqz#m{|KLg1d#vNHO3zbD?#8U_w+git1$g-`;5xUe$X zC=rVy`=D`<64u&G)^*ypN5$PJmeHfpod_yHB+?h?m&%yy7VEe+t9F{(>2}@&kI>fF z+s7r>cOCjmY2tdaNYSK{BGq!Y5@&;41-eS~ae4e^_^jqDow$f2$l+U8qkzFuJKz*Y zyf0<&oj9~w-Nv@NLb^%a;e>7_rS~eQoL-xTuFHixV-<#6M&RBE$pouJrvuD%_jE(T zB~*fuRYWbf#*~#bP95h(cA4KKmjf@4j!Rod?_6o=<1O>g-bU)3M60w*-6Blln@L^m zRnr^#E+18zbKXAh@xiqtiJ8OP~2gfQxKj?}%vZ1~_f?(Z9C zL1*8t>OW|tYIseB)Gr47Zak-ZJu`wqm*TA1ys4|yeHr@+$MJ_R`zUv}tj6}DCO(?F zW|-yiM?*!7TyX)@e&&@7I0EKuwOJ;{&^GR!Hp1gDU=P@A4c{fePF)5E7=SV3jU&u^ zd#Eh(1m*qk&ErnQVr`y$*p%_iIR%R_(>q4QJNeIT4pq!2+l}oKFM5j=*+XU%j{wQA zT|*h4`o-juGLfXPh%Dq>_~hq|7U<7%SbL;I(L1W5?h?rfd~-sIc}U0&M7L+e@-usO z=5d>(eL2|aGfzk}5;vLEq`Jc_ zqs5Y&%;Ky-dQqvn2U=F zOw&jV-Ln)N*Q3%LSYyx$ROZ(q>N#foT1`&!0S8&Ft3nwRVG%D&0Jdd;1%i%e5t7B} zGzV`_Y(mT6gTebw*5Ia#73$3pH~W?>3^o3vR7h{gD8*E+DShpw=iDgybRfx-`BpQ& zuSGbVQSGry{Na0-L$+z_pusx%wy1Gh=C1r>-6SPYk7W{dS1`&b5t zEzQ7aO27)*RV5w96QYtLuJ>RA3jB_2a`#38E9Y*J{li9W$7iPf6V`^JoTBPy7DcT3 z`Z-Ny1zBK8+;Yb9V!Rr!BhXZPv?oqaTe-2q3}zbMH88WL=T^Sa5@wyQl{O&5q#<3e zYR58zpYSxGw<6G|oQB8x`KQu`T395ovNoz4-fYk{FVkk;Wd(rLm4k@ucY#~KO4xG; z&h#KT{d?LxwY#!wrShOhm&1UIjZ2~3>k<1!Rq@vvu(N->cGf5T2XL}eGRQ?XOE8_+ zUs?=17A7jj4id&@jsb&G#$gk46NK}NPIPR9aVY)DON&6w*03?B`6Z$+U_HmM@5@aIrq<(LqW+FVb*+6#Ws3C^mh`AE%ol zMtB;`f@JdQ{w+;T5l%f~wn!IcE#@Jt@qJRJNR0}6D}rr%nzpt{ff&dZ%>-ScVmm?g zzNyc&qQ&v*w1&`RS0(ia__dIwDCMU*6yhvkTIg?oSeRuZuxppe0mTN^i z*2P*uOiV8$ww9$R>NwH7toID>_M3muuJbZpxtb-f{!1V>F9X4xEUkhW$}*1c{PpI| z@9XyJy+2=1Tp)5CR7YSKXdMNI9oS=y^hZTe@S5r=jkHH(5O%cae)MKQgHaSv)drF< zIa3`KXPpT;*1D<*m}dOC=?d8T{E1g}Nrf?lutJUL#vqQc7Dsy)X^eIYR{UC>IGvm8 z>x^B7KC@&>E!XVr-NQVJwMJFgYdt|Rk7hLOIyDzhzU`yHOBQR5ahBewS^quqkrsi~mA1?%c`k2!r9qq@hyJktY{Wzu2NN8NPL*fCZqund&7Vh;v9i$|t(GgzQQ{bCshm$73o|_+C17X%@WIp!y1!r7O zBnLC~&VgBG`mY498zb+QaY>+(#d|TbvLv z4K`g8D{8%IE&1}RO^fq3Qv*mHSg-B1`+^e7A5Jwc;e>))Za=;WBN6q&4F`CR$Mdk% zX~mY<36Ag)EZeZ4C;L5Dn!YC#qIUebb&ywbGmyT`KrW5~LiqH@2b(1Ip`ofC-Y;Ab z=%z}uCS`64lmN?QU~iVV$y0ve>=t)of;xX&CeJL8H=jWVq>=0tcM_IRP1rXJz<+*w zqe*%9%eR%F{Tb$~3>>MX;%@4NV$hF&ywyaKa@E20;eJLdG4)rCN?c}PRecm;OJ0JN zn2e+@9+jyChBuCiFagZwS*E^(13zVyMvYzrd!g-kiB1UC$t|sJ1GG=c(5=&k zsQbbZgZ(g!P17t0l&&CA(jB0 zr!^m>UO@LhJ~xZ3O#f#x_i3j0mxF?U6#q?V?tdh6-U&)F96vJ{x<6L4AiozV*8dWx z&;@fH|AM482oMmq|8o-t@JbO)1zEuJqw%UKMHO6J1tq#gZd!Fj2|0#?QW6{uJ@^mf zRT;gbXZFtMS@A3QSExAgf0!OuCoJ$E;MPl>Y_3*wIhmbT`Tasbu#C}Wb{`XK@-lXF zWmHR^P#Xj;ld6oxf+BKpiHJ?~>&Oir3?y}a^9Yyz!34eCpTPXibLV!Gy3oEW&Yw6* zhIB0g=_wFft-}&wlcHAo<>ei7$n`$%(S=T%9<`~tKg&+~kFw(^XuUbEYh3p*J&t3- z^*ja&>ZgJnuvv{d>rS)?2*ELTvZnL)nDGSb`Oj#3ZO1-|ga_totxMKL2pn5a)Q6Q?=n`#eE)1NynuJ&48uPJwQ zj({)t^IEKt*H+@=0Bpa8VTQ0l#%zXw5~_JUhUr?g9Y;`tx(X}Y^9^rtW=Wxb_)j+s zT&mfmmKIC*)=fN9rpVY>?%A=g>8Xr@>F5)m5O4JoPd63sWXG0?}Iqw62ipHvB`4ka~b zzdcfVWZjRxZD!CCh;H&F*G;$;69D*;F^(36)rGec!BBJw(IPC=I-(~LEmzKUET?EJ zk5F|*M5ah3j|0$kpZxVhCHV@KIG>RInF(?AFZXH5{lc=q|Fn16y038c@n?A1Yna6l~&_zku`JHXY z^SIu0ma~1FY<fY(69) zfHMlon>SWBSih%>gZ>une91BIl%-(wi!W`{**>{JYG@~YHV;Oq;rI4a>J+>#SyaWj zeV+%lV#qmkyUo}Hz_U!bl+94RmVDC_l(9Ws8W^%a@E|tSm@f2fs_dfBy2pn!j&Iq} zrl;A{nq%l+?yC*xfxd`(v~rVgml9#y{RKGhmmbR)CUO@IcT}9U3m!B1hYoJ@eKMtw zUF-LFYi4?JY3H`>#7H>I39Jn<7zpv$Z9agSh=HoEz!OLh>wN7UF4HC`QI8x}<}i4m z*t_R0yxkw~z&iBkQB&+*MTW5R-9O+4sN;a_1CdAjvoeJz-e3tIPhM=uv(&16RqZH9 z8rDhHrpQjXThC)>_{FRg8Nu)DZ<6o>2r)?(s(Et<^UBmn_E__U6;1_}PKC7D5B-wW zw$JGA89rg)PFC|JzEL4RzCfNJx`m5?(r4z`QPe6&Fz@!?%b6#j5>Lc34s4F!m}ULI zr~T+jTTr>Kkdt;5wxgsSk2t+^CK$}{Ju56RdA2E(D3IKEZR{KCnYDW^&K+>AirD6G z4=|c4I}fR>A0GLnj6RsMN=#vGaFpDmYLZU~L^hv-EKLU}{(xVbWIbbS$KUeM^GE!G zmz;=G>m!1IPv)XII2X(+`%XI65BAT1^_pga15d6OlLdZf-T7@g!HJExx`Pu#iK-w! zVHvbJdTm%>VlcIJ;$ZbC3- zy4aR1*cMaCVWq*sWLz{i!TVf>$wxwZ6>l8884ccG$aX#X`e3xP!GwxJ^>9<_rZJ9Xa&|wh$)!wYRWh>RnHzY+lWWY638>&B zcWF! zqi2Ug7G}JzyXDW=u$EsPx>$DTh%qLF2lk+USORzX-uEz)0@L+e!6dLIB_trKWvCC4hfn#Wo~-40JnH9 z@;iegoEv`sfLR?KHn2;>hnQ!^T-;X&_NGd}TZRH_;U6sq7wA9^Fr^UnOtW`CL67kG zAc6J)mTpN?6OH*wAog0I$*Bd{XsH{NkPPZTM}6AUX_Avol7}^f2yfEJ6)I?FW2#cO&=1Nu!!TQ9s=`_n zs13DPWMs0+P0O1sfr_TRUHd1|Z^CkZ7JA-vzvQ^i3ru^?`}-ViJV1fSr+<1|50_uu z0W*Dm*GKx`v)*c}ee95H* z00=Ib8;p+iUh?o*!^riKDoLjpYQ3k`;ic4&nP(}3hB!r%VO8VI9IE@N2P+biUZo2O ziIkK_$XaPCVKJO4A!3Gijj72iN9Lm}KuLiyTF|4E?YoNuEg#!Hwfnm!d1WLP^P))E zO!EnIG`16Uu`+B`bJI5T?205*3HH?lRkQ~i2pTq(T1(o}?F3k~=+pru^B}FVMOr{s zR_1l;@+z&BmqnN@odS=VWz{adSk--_zNN249Eq!jXP&NmpB5(dYISduaxW7r(7Z8n zHHAS=d?UH}45GDp70^Ut<$HqP+KP*oKG9j_nWvRT3$~i2N;NI7!+7d@@{Lw3UAmWs z?pU`;bO`81!Y*CmWm7_$2IY5Qj zay`V%gBvpjimBNb4JM3t``oV@2vN)g-yTYqJv?`OFU)Q!b$Jxn;V^+)=d}y-a}74k z0>_Ywx&~p)RR2e2ZCu+K&2?!PkE2xXVg?Booa9+zbI_u8PoEjs(+_lLE|me%ZXVQD z-&njRL5uGfvw;=4jn@=GzheSHu;km*j9;5h=xozgYwN4)44MMGpQ142fQ%JruW63( z`j{1G5l~_nD&3>Is*U`}<(dbZ>gFyA{!c@ta%)uF=|&C5HMl;d0|zh=Lkt*kZm3AQ zBMzfr2d`7Y6~oU{kb?;k&6}aQG6Z2IwVHU=RCKu$hiLJLH=5Yfch)Kh?C_S}aJc{F+2Bm(sz z%4SR1@Rsjo=IHC@v#ITvcu?9)fAWk#%*>;|mG9D@&6sS8ER7n(qJ}X}$!<%IkivGi zfnQ+v>#uqC#94pGUMMsPt|wzcfGqSe*lbjQ@~2(3FT(b#GlJ(mLf%g9mrFfi0R z3gRQQDce@ITv&7Rcv!87&rJ47#OyI}wpD%)X%~bt^|{5Xx3{x-{5THbI zZZ4Ypa2LyVK3DA27$o51WR-0%8Oxt%Z&`YxA~ULiOsye~1B);GBbPBPi;l> zTri?_ILsaI=r>D91k^@!CQX*$=ajRiI>lOGwzd@l8G<5vuQE82sGfc-z)VsVcq|UUJ9d&4ioxIk^p4Wlou|i3+8re>zwMK2?38r zTqHX$rqplA7l`mqZdqQ1Iv*DI)AA%|(H~Q{!fFLDTZK8nU*iitN28LKneQ)U3;3S> zK1P!}CC|37VLfh|Y$=~jnW4WT1dIN_KDWtQ8}3?m$nQ^_<4+q=5}_j-;)%GK6TDRu z?@4^Si>(7{xJ6jndq$Ej zk9{N_fednv33xE^K86a_I(|@|jm9!DskL-WCfntpT207$sIihoY1<_!gsWT(L4}m5 zEJOY~^zGV!>TS!(t10BYw5zmj5>H(~`nL7tPRDxE&4fVOcD4A^vtGV-+i~~3z?!m- zvFhm9JlpRsQroS^>irf_O<@5pB#R#e7mdqFz>njvV@k9Q9LMHt)WGo?{DRxv7XmrL z1SqyPRf1SZUJOER`YiIwqvP#Y)^I_sNtiS^^CoGZ2GrG z*`JARasbZ+}NrixnS z7~_RuSD?$D>h!GW5N*}J4^2H6ctOZQ`w^SN<^V<={oW7#4D1w8sMlQ3f=kmFOqrFk zv}Zbpa~H5f7`gbx;PN>c@{~(gV9EPd}v^P^Xy>%Xqb4 zxDri#4uiV#$g;54=O0||Z&rP8mdCq^ilV2jp4@6qSQ5nR09V~?f8f*!q_p}pXXS3p zU~T$xV|4&|c{a?ak1_6;+%00fWpu2R>};m|Sd&K(;YEOeYwrA+U6-5BoosqqX;(*8 zz?O2BsPPOFB^Y%#MpilPp`y78)xP1frX+BYt$2megrVy#>d`^5TQ!^lIT0UnNqSHM z7H&Ir%$Y|03D84t$;n~>J~u&lFwR>+D#@`gE%wn?tkFo%`#f2Qj&7!WCG(;r_6n1E zKcDj;VO1lB_Ce}o5K?j_bZsI{OBHQRfCm%MG+%*9QF zdXZm)Okvrk-L>|p|LQ8V>Cq1;efT?&T!%(5rxl$S?b8+h0ebuhWWqCRqHt8A@5~k> zXk0rUT%7jehl0Y+inM#LaVoL1oUw!u^4VA!HJOPlY0(oMYkx<#mw2M+l3F0e^n2>! zb!uU=KW4_TZzTUo!VpFF4k!@iH+ zQb_n6imLKQdT+8b5X~`cAM@GOnGV=f{P4N;{)zbIWw8F)t-(++JyhI`kt=tQBhkQz8xi|X!@p~s8XrLY=>be(%_lACSA!#Va zH4~O3FtHi$H)yEs%Tm#X7mOz>Yrzz@qOcQR*rV0qSNDlcxk2JoWf-_m0bdVDf7>>| zZWPOV?1Y2uW(Giod6=mOn-Unb>k;lZ)WcC-4ak%{cFOnxu*THA{qh)NC^1HWmilwU0CeB(!;XA~g9= z*s&NH5pXK8soYrAuO2Ahtc{y$B&pW+ZEGxSYs?0WVx$@feVt(pDM#xYLHotX1jW$= z!()$t)@dTvi7M8qjnGjg=zs| zbjKX_sqHSYzhSnH)%2OdorC>CvyZVICskmoO#i^Z7aJ1(1`#@bAdD&3jBk2@4m)EB z&eEBqcXv61uj2%Lgp-dn5eyKIU->4LrS3o zRk11RP80O*)tS({f~WQ}!$c)b@kC>$X+_^7{xa~_v!teY&nEq3qTAcugN?1QPS2)3 zS1VmjV5xQ6vYv(A3$4+W`e%8DCo~E_kI#9`1<0VP>s;Bc|0s76v4Oi=AVQm{>wF8Q z#b8bKIGEWP7h*h*Jb`&cwKif>amK#_vc)=3+}-&3IAi#CWGxmx3Iu1xi-`lS9h}C=0E|umAtpQJFQ%9-?NMI*-GR3}R!l$FQcJ&k3;kTg+{sV^ZZT8< zQf*w-QGTOBdaFI5V%~9^R;6Zyq8F2-C^u5mn6re^laOP_rSKaWicZO{${BZ8E|d>F zF;l+wY!286%@sB5VDGURYHT}-;RF( zV0^OYR~IxGS3XRgDVHnM{^(xiQT3e~XUoBj|iLno{kdnZBhn-PCI$aGLh_lG+{Yz9-yG ze!0UVxNJ9LK4BW8gMI3_F!M@XLa6iQ`aAj*^D~|$(%O@)^*6ZL9h^dYP8R=M&aE?n z99h|*2ICl>m)c1)zWh%ZUts@`9{d&m8^g^POa|^Ws~=ik8&`+63yabu+&%spZ?n4CTS%I%T;FfzZ zV5wby0x5ZeI!9K9xig`C(9PKU5a$IH$vS--zvW=%4$K^j{FPUA{{}K!Qa^I6QeBC6 zdPLNQUv=(i&o=0&U1o7TxqHFR8R#|<*)3mE%PX#^Xk0h)C2CD2=(bm>=gwkP8(#P9 zj02~!b@zg8PGIS81wY-C>v=M&5sz&2qJWfr{^EWgTJ!_`CL~`IpTL}A=T7qKd6ZqyAd&IHl)*CNy>Y*!87YW(I6|zKP)lN>=|#^5HI@ z8z1=?zq^6yL0a;hLWL?@OC!uk*E^?mi`@yGEe)+Z*KkC*zZT~?xg4WFfCv%bf_`zC4p}RkirLY0OqX!jiT6jl2vO5zFna6Es%ZE zXkqRduoi$qkwB~24CqbK>Zabp-rBT3vgg|{lyu47S2#r3PQw*OPrd+4q+w-V+C$PksRZ-3e=6)`N%sEBD(sBLn*@_ft8; zX(q7h`j-H3!+PoO@-Mw%@_;Sw|5&%SWdElVMl?q%-~E>ci~Kui!Ipeyi=QlMhY!@# z!qY(i3In%<7GqLtgrJr536$T$@ez?LY{^iqEu-NSE|o9{ch1W3U`3nH`Og1#m`@BG zNEfU?qLE8~zm5lV&Xbqg#3E7J8bYlum?yGI%jaa?v zSHognXH(;1Ya;4?%dVBxhgFludT3Zs@Ir|LA!l`Oo(G1pd+ za_(c1>fF&KIY4bYQKP()s_Tc*-)d!4)ycLETg(LcrKab*@;D8|ZK~C|O8_;=gig!h z<~I-@z%Vs`Cnb^A%$$BPkPa|6ditD)$wj5+?^!(|cAAkI!tIUVoQFPa40!RtVxXD$ zNl6=OCT`O;5vZ8RiWdfopa`KJc(mYA>5zWN)7Vb1_S3P?S=VC=#ENx=$0|qS8eugO ztIC;RSB}ti;@-^RR>SpUhXO|>k?B`Qdt#Tp=Ev9pkd#JheOTZa;QQUGMI%%sWR+;o#qloiKrQ6ezta?;MI=Le8O%FdVgKMqOTOy%2HE%=1 z0<4Qeq9Nh;#;SfA+y8J*9!9{k({6VWx%}Y2Px&wcS1J(*}(Y zMss+gYjCAk|4=eYPw42m>!!9BjZu$-UFFU7${F$0CSTS*b{PmR*1Qn-mM3jftkVur z9SK**wjo*;DYK3vi&z}u#C6ay=L{$H%;NXG_=9;_h&0#7B|=bA1(a=KpH*?e(-U5c zCe7|K|J;;yYt;O^E}wfgukgHR_zY6Gd5#(QnmF(x^b+4`&_?t_@p}_b=ut6tYjg>mJ|l$^HB4 zaK&@jg|_U))j8g*Jt-+0#a->CYy(37h5Oan zjp3pNWo4kV=d9ei_!>+dHhOkC!n{}C!jUJvu8#br1Dn@%4ed=2DY#tO=4aTN*?ffp z{68U$5tINECcC06;XrWA80{A4y)!7DlDmb7q|l~Nim<{{RhiEq5*^xg9929KHRhPL z5>&n1;joC^hcKl)KZe^5qtirokNM1{kEYq%8ccD9a*Vq&rkuF6KC)EI#KXP^l0y^f zqV%R(vQ1)oWxm9IlnS!H;)GO_sY~2-QEW}QROOKiR<~%~G3NA_X{-b*xMLKSZeR#fdtJi99%71Rd`iJYTn~&6GmQ zRJVp$Hyv|qys$Cqc&;q~jR@!q{V7sl3nbolHf9|=n#2CapI2;Lku@7ZVD|Fm&R@u3 zN)Ft`OeS`s1q$m;6^FDfkSFLiAef41kSD-J7ce4cM+zvE0mc@+z6p(nAxP7|f(XL8 z{1X2rie)gfJ)Yz0e%<@L{>l4mM3!0K3g=<%JepaLC{Xxy3flsSO$R;dpqRUT*xK46 ztMV*4H94|rF`C~q)XbT_^_6KUBfd|ifjzN7VRH081PR&TQm()h_*R2fO6BuQ=&{$5AmgTr%K*?2azdM zH;G>jEYN;V_reljo0QnJ6FdjSS{vVsjC8A9;D0a%aRE%}Bgm-?LECuku52RFErNJv z(oSc{u}vOx4XmeN+k!K5V2Z2j<7Nc~(i6Zrr(f1StbM2C|Ds}y&$o*Ve=u>2G!&<^ zPUqVr?o*!!y-B6r9JJUyfEVtTl$))XpOg|`mz@H{agXDv_P)P`)75T0#NzG{GEw`hXny)_@7UT4*X9EiPhB8 zLeoP3LX^p1hBqXI6$#WXAV-#7{GnN(9!tGYkVeh9^aF8{Bu3G4DHBH2>DiHK-v72% zXs{1lSMy4*TN9|JrS+02Lk@HI%`P`{(>K4j|MRN(Wm4$R=NI+>&2MtD?bA z5pt-E(;34(@I;v8Jq$n+3bGgp94%*!F^(RxGzuJ%5)1utghqNO7d7c!1X8ktv=A1Y zT0SES3D*k~BsAFO<8661vP@xC_tH#QI+8nz9V=iBIPRjF&1@tC$5X5= z_ma!*X|WS*4`~OIs74T)uwADaP|_hDvT@h=FGFlmUutZ){TEyo6o1&pr?6aHvb3eN zHKtanL`^}7UPj{}tqcL45@gM@%&u0OldPac9BqGvV;$<<)3G~k0T;btI-lG)$8ssl zv7XSj@ht=O7)&-FPqvAecuTyN?;dnhS(XQ?w!g>f+$I$2nAiIB$kC&FtQ4OK^#3>& z)EX`_3^?fOVSIB)uVKKJHLcVk9Swtkg1}K{iQ|e2>DEt|4KD^TpU_tke#-nTIVKA~ z6?Z2a+(os@m162Vj8bouz?x>3U@xYdw!u)-V}wOc+9mb#&e@$cOsuZeO$W!14@uUBZXn&w3sJDAeZt8{>2z-y?Jj0F9Ok_W z#`sS2gpohTgLw=DS2)mAhRnDa4Pe9d3~8bDXx<<~S3A?ashy*~lYv^{dIlk&w)ekq zEW!U>-1#0td_z8X1MgVixF)8fRw@=pC^={{)eJ}HasKDz6NSit)>LR-Cc9{t(Pw= zv|Qx;p-J}ZtbzA2N7dKQbOz66Q)yu?YAzP>p2>-uHEsVk2VD(yuCKO5N|npuDz0Bx zvx^OZ9ibCAvZ_vVntyB9)VstI>$F^nZRN~$H~rbxakhzTD-tWP~! zHTKSR!y8c@JpKz79>G?+F7Uvg&+mk>lwM4~jY|VaGv*ZGN@L?42r@y}JI1RGz4N_z?kbMpbw#2i!r~O zo{VCUNvlrNc8LoyLt9%T4Uo=~74jgvQ(>!|o4B!~4+fVfx-D<;%#=&X5TC zMDGIZhLoEQ?z;$vo`rhy+x z$}^nt<&qQJw$7dBs7ppG|Rj7Y|XQ|B*t=R^Pf zH|L(xbB3D|;Q9OiJ&091JTiVRXse$TgUEl4(L_(!6n`&PV1nvDF|o?;A=XPe$?FZ5O)=SsAg7a1rGV2IL7sryQZ_PmE0BWKixf|uA8fGVmvXaj z?T<%!e$$Tu$2r}vFZtUaU^jw2OG$mBj3<4~IF>_+OEzs-BOZiYH!}3eZM2?_r)_ci z^p>pUr1iUM03oVBR!2MZ^PEaw7tWQn_UxyJlT2*Pvs&Yd6@+0>pH=B=IehekaAsp| z&#;UpPAA+sY`x3&C1-bB=Bw2v;i%`-xWG)UZ_)u{q3e%ifri%eld=+!#*sZXmXWCj5g{)USL*MG;}`s)fjIi&u_&U?w;}CI z$~hSNK<=PwriP*GJr%6F+E?w|!;a&m!^qlcLG=b54)n zdXC?i(mI#C0u}^IaQTY7+e$Mwc%Kg#`@gy3Sc!t+oeT>=7&6h?2ZKOndV3 zxJvc$1ZD1~^E2X+ze0B>U+XBQ;W@20pZnjEY6xm@UwJzR<)BeRk;hkVRB{`PRx&zy z16T3vFgA1KgCVq8qJP-)BZ=rR#F1&UvC?XfV{=Rr9S0#JnK6bWMrR+eu@%JJgh7Z? zB2AA)2_u0ot%|R3Q&AuyZC`urV9*m&YI9S}j@&7Suo<;_5EQC8ovt>FN?BoO7%Va< zJ@$o<%x?3Hs0ipu-BHU7pG$Q4Vv;;3Cvz#rd_GEn z--ppDjTWA0fz2`+UYc4h&Ln%lrY?!8<6Ez=C!m6JkbQTjijhK3Dw|X^pj_r?$D%8P zy@`ToAXg&mO>K&4adO(H#ueNmKDvEf96)@VpIwR^Wqol+TiUZGEVm;Tg+*d9a8ItE z>SWmAuS^m4()<7KWmxeVA$gx`cHrkgVgJ7a1<@Z5cy&UTLizY>@A}t5Ar@a+4Np1- zz9nT9sT3K_u%9+{*(Xs;2AwARew@d*+im7J{7vlrhvrVRMAcK2?7J2n>5Fi!I;_Xv zJ@FQF9(GqxE6?5^xzCoK*DD!9?+7~#ve5A+?6QM85CPs=mB&Ti$k+6z5D>(BUBRre z=WWI$KoyvZe(X+i7PHqr9F@C%TmeddNqhnVXeTQevX0(~_`I!fA3dB2*sP+uM}Ux6 zp+6qaqxti#bvH^@!na+T_twA%s@OkAcv{+IG?@#FXY-58N-SIZhcP zIkGj;Bvvr~Cf_iWa$yb9L9S&#Y`kWpa{vYaZ;u7s_5;on;%Jg#Kh{zeg?!J#-0yBA znyZf{kkxPh!3S9dzw~pK9NE&C$007VV`-sQp-BEAy7BAj7Fn`}#xS+%sn@*2R9Fns z#OtiODHOz;7!8@*(W3y9Kpsjukvd*Mi?)p)z45s5&>-8#ds`d}ZXd9${!A~sJlc9d zC;D`CT2s@a?-n}Vtt&o|5|gKlrPwTepX!MFsIkB-$qY9Cv~9RCb_VxljjdPr=S4Dz zTeCqn!oo}KxUCl9+Sxz26XDiVEPU_}r=iF*fifaeFb8JsZx=Y3@QZ)vU&GBhshb@T z+|83oQ&ks7E3=OdRycGhi^Ha($$lvSN*r7o6QsYfC@`s?>A1tie!8)>rsFF1R&AHj zM4b({lQD8gNzi}ao8lE~?V#yY*ys#(a=70o)Gm5~Ur1fDb!AsV%o5niUG+4&XXKN5 z6zgVz+fB?)THGprlw7bD@Obq1Dz7wURIId3#Xw!TaJIb6V}Hkrh@;%=^MXGI2C{GX z8)cBy=(<%4%C@O<(nNh>LqSN7b9z&IE*>$`a3X3M(Y((J^V3OXbN1%pL?RRRq#!~Y zJZS3+GK3p^#n~~4KYcP?3U1mSo`Zu%C1Eyo7H2R(g4p%|S!Fd+ccu@_;pJlT+Xhz5`9FjF`NUpI>#Jyf`Qah`b|38_uy)e2%O6wskSP_E9lA@05r>1Jv=Y_}IJG+8c8J02AdIHVSE0{@dl_Ye#TK7Q_dAkn0PLDUG{y#3-^evfIyd$RT z=l-NJ65?HB)P5}maFa)pEw!^yZ6r%!ZT-WhJHEk- zJ8qkXMSc(UI`5L&@Ba2n>6qRlFjsPW>FI%}C%S(58INZl5znal7T+T*%?A}tHYTNw(;JyXptLTrmXJkO1;zdChEz?a(tz& zDc$-;=jq}&&Vo6D;s866eY4;Kr3R|SGnh^mNH~)=jPR~k-$P$wc zvv?hOB2bI=oHh%ZWfe|-nRh^so8hz~cRN`N@;hq{5X3@+ah|>|DKivWIO6C=AU+(t zBfT0+U4*g|vaz(Zz{nj?24$5RR{NAE?XG3yJQ+=~ryStHLJ-@^@=J53;I)#V*Bw5k zn(>=?)J1>8QVPzxjl}C@E~%V80WQ0k-lVni$aD$4Al~;fc*}2C=8mu1vdgqZxtut# z*WoS@zyd>nqDML_8~l`9xlaKMOZ?TKRp&bob%tqD7Idx^|M^qZTlpXe?8|@@jce>r zCw_~9BmZNoSKozc`kH?4dl2rehYg7%($#>VN7l&o{L?00Z$#zmB8H%avVlNx=tr(J zH)gzA7g^8y4xk*Q)|?#aGX=~m(!pH4|8t@ND3ARi&z+qiD;;g2RB&4xbR);jG`%YZ zzeV`gu29wBfLG3c;Y`*y1DwiplbtbICTD8ghxYg1yA3LsO}@h+L4N-uE9J&AzuI0I za9!}}0>Y#!5;33NC{{r$ox(8Mc5oNV62hO_5>pe zSRkuG4rie3wM0UUI#ihsFED4hSS?`4z~G?pu(jfLlo%|7NsrW-`$e9vCZ6Q#NfjPZ zDvW9GG(5LEq+L&GnmR=8!nSA;Cbd_rL#-oLNxKE+SAM3iM?uPvd2~dRXEw{mCKo)m z!$g0jo9GESXWpN_|@@Q0qVzA-wUARtQx#pYU0>IMe- zc`IX%Q!e?OS@*0f4@Y9Qa!GX7K3PKwW@kuX{|Xn|S+=f+1!t7VDO$QPVSlUx4j18e z#E+O(F&wyx3k#H|eix0m#F6PINt|f1h#WW95T1pvmSXKx@}XwXIE6+f;z&CWE zNEc}5FF2Q0p@&;^ClL7M6QgoYsLj9WGKJJ3G_&cqR{jq20HiX4iRg_j_j+s zC#KNrBR>RQd19^;Ltp8v2tu37ecs7cn)uW(Da1e_KzJV+&z~6>+Yx&+2^AV3w#Z65 zJ~O@=5lJ%dhN0_BS$tLpxH#U-Nfo z%BIzC&Lu*`d*mXJ_RQ!2PCfMg#ONHj-J^sxj!EK#C&Ui|z%Xkx(d2!Qs^Ht=fl?Yq zFJx1%?lN%%gPg=ngzD9Ft${Rz-1(ae-|b6vlJlr(^angNkd)8zBqt0X5R)hFBk$m( z0X2X(K0S;=-nnef{M*JT{fHhK(3rgC2%O)kzqdmCfCkj|op6CShmZLF?L%a?XVt5_ zYgV(5g2hiG6u%=PhxMC4Xzz>=%TvS~Io15rD6|mN>%9^rnN^o4qlkM+e7ME?;M`ay zu??9+Cj&apADPje{Af0SYHCK}sJ!$smVKruSa*h$-;xHq;njCo?2Xe=a+ioF{X>R% zH?4rg{^tG-z6^NRTlxFf{jdJTbwX4mVL(9kd)4@p+eEBTh*b18!XQ7Z9SSxHP?ijn zqHgooEajgmCgKO_kQRFz8!0>D#}w&=It%P*2w0|2k|mC{@HnbxlJ$*jc}2SeDZ&e5 zxAhJ4vljHftS`@Sa~&e{QexF~I?(^Zjf82;;ZQ@|Wv2nIr`EU6ckZG)W}a{SLUa+k zF|=dLY3-3+pLk-Em_PmClBl`CzXOI6kJ`B$jdQc4Ue6kI9 zG^J`P@FNN(ZJ(ROF=ja$K8^p7X#!fAsB`4aFXVO(_YEud`9HzXbR6P@ps7S+x7Wn; zcl;LjLEy@e&y9oo$AqGdMte$jE*y2^yH)7Ffs53d@J+=pY&l74nd6tc+Ln?rLwFay zry|T&YwDC2py9@mPm{yLp7zg|R`zPU8>45ZF(@B2hvN=Vnn%C9&l7(Mg?jsZ&K|@i z+rDC3-v^nsXOEP%<9E-}aK$qjhb&i}@09o3!$8x$DR&G%Ja0hp+!)boy8!hyCKa~d z**XMGB+D3_{_xp0E^|K7CeXZUJr$s#nl`s$kqvVLEw__JT#4IAa61W)X*aR05Q`vD zZv3SCve^UvY`VS$pXsuzE4RCD)V_smQjeP!=6i!>dO$-f>JsT3lo4EK`|z@ylN<`k zM#lfmNe8M;QCOA%H{AK592Deo#XF^T0^C1&(7pRhvk$sXO`esR>DknnF2@eIdD6-7 z(cg!ACS2mdNt*p|LTfVG>zUM_meW`sPi31dY^I&)W1v3FNgrl>rM2t5+=2Zr%>S_A zDY5jktHRf>Ij{yB2N}Th_jh~se|Kgk>L8`5BV7As|%K&He z&%;z5wJk?{&I*!Mbcri)e-e*!@Z@WYd+;EPXcA;h#BefJAIE;^;C>Z3x{UBZDe<0} zq%BKzwQ{Ry8wxjr<0P1JsvE942RmtX!Cv{-wDb2D;8 z9Dxbk;l2D@+&LR4b z)e6Lt=l#@k&+3xcvr@CwjlWNeL97pY6Rwod)p;riA*X-Fe*Pg?qU#6~2*d&rwk1Dj0UY!#25u zxZJets$>7aB-4e5mr_02;fn`V+$1f=xO@()rEQ5VF!crcNo}%I*4(bZqf)>XRMLt zVGtbrB31S#ia>1%HMp^&tT~(`8}s0Ez#fc)Z_K^j=yFLpw8Z$8{{L4zQu7qT|sT=)|@`ribR$0-MMKqu7j_bkjh*3{NYp8NYe|IW#Mxbq|?~HBW z8OupwvtO(38={Hm-APsPYOe7+XYc(eI=M>qK}T1EH`{k48hJ|uI-UO&c}wzsG3Qqi z#>LUHXO#MFHh6k&tvnFRN61}LEV`ZBI0o6HodVI=Inn=%vh%V>vfsT^@&JyAch zsKJkbH4SWuNEgP5p=<@>yA~;^zRa=%+U6cnntg>#bvqK=kI=ZCxDXH^@qxH3;|c}# z7@uA}Q}6Iqf*rw6kVt|*Gkl34|LL8+?tn1R<`$&6XUI{zgR1*tKt$56^eeLt8`7=! z&+cFAOZa>^Hm9&jabZKY;7g?UUO1;0`glaP3$MY=R3OJ6ojqA8VD~|xV80|fPric0 z=z`UYLF6FW!clC}!Gz#O>`K)B+f<-L$}mNg!WW5b?kB2FA0r6;g%(p3+Vd}#3)_2a zAQF7q?KezuM@(KN)!!7_X}0Q>bDVK`%O|*Tc}pie-&5$cTuiYCT+j+x&@t(&epXYF zcBN+^wdTv5Wd4+%Mzd>%`ttThQ6(xN2!4z#Ju5HzJ7YfyGoj;6w@>CytA83$7-8*UqQ#Ncdv z>Mf$iX=imf$4@gT^}Q?ydgr!Di?!R*;i&q1l>2eMfRLkt>8XQ>w0vb#;L0k;@05Ih zi`)g^M6sj~1A@4;j2| z|Cff4!hC_7@_h?ECD9Wmg&?K#lN^ozKTpOkNPbeJc6|0q{M^nO{H&fxDb78`_@9)N z-C{2ru}>PL20o=Aj|NE7JpE4qp4*SZOgVqudONZ3rFcL+NyUWL zx~nYPp)7ZCDn`&Udk)IF?wWA>Kzd{2UP{K14xck`;|-rlHnlR3ELW z1e=gJCTUjE5|CS4#w5>mb3am>6*T8NwO*T3D~p}x_2s0`Juz?io6(f)U(4mt=CBFp zsfiY?Q&*l%bx+8~FvwTf2U|a)O{7LybMgZ5x++*mvglM!V}U3ZPATR3kLCJ;hn8IR5jkSfzvbZF_ zDY@(t576}0_FeFl9gYD<(ho**hvN^Cg8It}Q*1yI@(ylrBdt)08B9Q=)!^`FiJPM( zP4EZXWU4!4or&fSuOm&xRg)E@p=lR9GkjM_HhBFv?>L+KeZ*j{+w{J5Z>ix<@ax@x zVC0*^=mbc?5o+-KZ;zjJ@Gv&G(H=c!D`b!5Dd2Qd+vn9*Y5dwmZ^kph_RW>{!9KH@ ze}w2-zt*NgNAt+AzfL2i-gq0L1uibg;!v(9Y5y^!T$GBi($6*MC_H2)UR$u)7rFT&u4XP_?_VZpHGb zpPd@h&NhmwBM!&m*H;ovj%~sDhzC|i1HYznsHHrdve1gDQ|3!$Ruq0Rbsyns5{=+7 z`Ssf)XJ2Yh*|$?jsig|^HKp0<=y+N%%4M+!4nCNN-%%3!nzb;v#H*SU4O2hJN3DGQ zb9c+3$K({HfN8HxZGp>Hncu2@hC%1-APZ#8r%Y@Sm^v+gyfFxHA6nV@1bV6)6E#>d zE-Y6rX=wG5jS`0DWjeS2{+m|+i)Q%L*T>VSO9+eJGmv$Dkig~`kL=;v*7-r|fFWg8 z)QF^3c{V(2L^5w4to9X-e##aob-Ao(A3qJP$_g}yOjhT*oGym@)!ZWfcSbDAeG(|9 zo99V5lXkNvDwKAc;J1uf3p1tsDOFv^MXQr<3WWiHYl zi?@BW$){XS)>}xI(7{)cuN#DAe13b@F6oL{0AqAtja@OxSd)%S$IFjJx()$G6RxTSw z2tb0%TO{n;cbPX?f|#}gE9MZC-+t#sXfQv|2 zRneo9{nOXtW>O_#Ukv^~t5P}u-B^c_QJtCsMM8v=7DFBk4xD7hBFe@6vr^Hv(O2hj zA^Z|$;EkZHJX<%_Ho}}rYeqsyY#M1AOCgz3h|MN5!5U`wNtI)o!5q7VjGQ;vFRK|{z|w-NSStWS>-RU5W+RiT)7Evr{XLb5uZ+VIyS z9+~$bex2*#`bO=2Uy_fj$~GT+ZG^*m*>{st_pU*aDl1~XJaj<7B=aH|WK*SRS?mx& zhosu?JkSA?!zg9{twe6Y3HR<-DI%kw)SYwSPb;+BedSl@hC&|0Z-B9gUcqNnwwisF^I+8O_FP1j&no3I<3VOUJG$r#lu>Ua&Z$s-P_XG$_6)px}4dG>hK z=fbL}8Xtk1Lvr^ixIv;MTe7@!MN#k1BSw1E#RVf3u{Q3@sd`8k#P z&2@9Fzt03HO=ybF*Zbf4J#M;O{rJc87=66;JZ~o6tj;$fH>^*c;OoQgU75>u;@=Va zbO=y)9qh&VoH(YAQu*YNCQva8%uE?N?oJu{rnZjl%jfEKv;SB(pVQ23{hf2kdwrvd zdX2GmdAE94>U8U9Q<=M6{F!?!D0S@Sf#hq$|Gc2( zGC$l{h#qtk1?Fei)_hFdpJMw<2)-xRr;ch61@^idUGCbMUj-r?WzVeKKHRy4@;icV zzIK6#dt{E0eJdrNyyKkG^YltEGnON|{ON&JIn*A7p_j-Dm%hQ)bN>kUz(qOYLm)xs zIG3nLZp+K>nW$%FkT@er2?ioRH7ft^MuZv25+cV3G>>1q76y4KhyrY2Ays$tczXo4pUq5~A6J#o1 zyMDk(N}Fb8=9jVD5dlLMU~+YifIKATH;Gu-ptJtRW@LY=Ut0NAGdD(bPz5_E$4`L< zO}55@w-bcpN~o9YW#u>LNo8fwJYk--h!LI12oXSNX>W%TYMN(l#nL5!1OK~q^Hd%b zZoI6+9_*)uh|x@=AF|m?0BJf}A4;U`T+GsFt8Mjh>!# zDU%pgfJ5^zdDX5k6Ygmp9*{?8yhCME>8|~Sss=YeW*RWUp9_4yDr?-7pV-#Oh99XN zK4Uc^q5?he9jBw8Cp{r?$`T>`+z;)ly$k~=)!8S)P6o#O9A~seg8kRyv z)C>Mzp4Iv;>0{#aH}vn5+@x$3IWBL8DU2^g`W1T$poeny%WyYO8Er+ST799}O^1=m zI!z9;0So2d7;7ivuIc$9AYsUHl@_b4>;*~M<(6`o;%XPbjcV7%z;abSGkSX28c1m$ z{mwi*ge^O&=HebDhZ|KBmBMzh1j#AlVe5&2{Fi!vRmk!x>q208`Q(i)7eKo0C~j1A z3)Mce-lH^7#>pVRV{bt6IeKF0F&)@XXvL{5&hK{&20 zzjpO7D_#$@BfpahtF@BVJ@h|V>c#9^9t$(1;?y`ow}I=NiwTs?#}w-+GYn!SeFQ^# zvW9+UNAW^`G3*ylq(wfc-AtMoacK@D!N3CMp0RoVsg?_g?)yz@SNO3kRO=PbeRn+} z=$OD@rf&OF&_TjBLFe^*wO}jOYee)NOP;$kmCYMq^EcTb5n{(cB%9FWPCQ-6{`?C<} z4^KgKT*evGh7I^^yPt~gXtu7EP)#8ilGCSwGZL{(nT?)tEV8Nz#ZWUAU)hwd3^FoV z-+uZe1Cmx zx7@jpegtmgD!M6H^meG^omo-kaxtDHa)zo0=(G-O2E(q*N@^=pxn;$FO0lzl8Pi7r zG#>^9w?_T8$Xpiee-WZ$>#g!c6AN86D28-OLD;1G4r)7QuRjFpWy^5wXA=-!-~yHk zxK0#8i4L}dK43Aon@(Dx1LBETPDldQ3T9@l5IYjCxSqV;{Br-1Tnpj7QGfjG2X=*u z6?wyD(qYl#%^SuXnge{d_&b3+cfga4&?22Sw-`){{^w z#XN)H8*C^)t^BdTiz4&=WpWYW`;1BxZj;qFl}v+=N&bCo{lEBc#DNM#_2f10g>6D; z57)5aVg`ZEg=OlvZ{ZCupTD(k^vFsLF|~ed?X*45);yi|k@TV^LHmR}eOw{G>NnabvAF%%>tbweq1#@kfumar?aA5QIfNG3pNXRI z`)Z~UF17mCF!XzRpp~^;bLnLFdg&sZ2i@YmRHs|*ogQ_G4`ouaG+zV5aZb+;$(u*Q z={3t#q!8-GdB)hyH6o=iP%TRar*9VUA@0Ny9C2!tdC%T5yG@7oax)I3pV=Qqg_nTR{K{j-%i13I}HQcxhL`TcyY`PsifZ$NtLWf%hhhS&c zxKoe{gnLFqPT4wC_6xdrI7!i%hBBUI7B4yKd}J|v802;bs9>1JYtT1X^bOSEiRl&R z+FIKUhXwFdLE`^x_E@v|*2@_)qpktFib5!@yd%zbygFQ1w}h+Py=-ia5BWSkFZ_KR zYb&3@;r;z|O_PNg2vSu{7^xjjGss?UcE$cTOqFXA4XNB&_UIDRp2*%H)AJRtA9M9faR>cp z0^M)(a$Y=i)keuf&3l~V)ya)iod&E}w+jX~ON=3THCK(sGvQ#-35~VY3Zw__Nvm^H z*VCd#FPjgq1=#eTLj1FYpTiqv`wA-k~fvbW5$m%H{ z<^qPWMTpLQz&*X94uXD<>+=w-$&@B`iPOkdr%{%A)Ftf_v6p^>jyAW}(lIhZG2I?! z#N(oQ)P{$nL>I6zmF48-S@WDH&fae2Qde6*PG}l6wWMLnkGEx3Rv+cskoI>N*0zvl z8ch{OM`}N1>m3UkAG_4*4_CNEhM0NtsOz7jgMespbBC|$-x66E5q(5yCo^@ke2|le(0J^MKO9&ezi{WE zK8$DFQ3^+x3<*Q@jx< zG8QKLo28-a#ZbO1b{pRFF;uV-QhiYu_<aqX3cHw} zvO_uoj(=}s_Bfe^WpIc?31k$m#$~-c=A3ap+dGW%XqOxB9c<#ONBP?;HjE8~u=>14 zq7#XuSEv&(XDu4w1G#bWYT+e|P}Sc}=<&+akJ=Q}XgQc(&_~#o@_Xg995}8-~{&U*9Wyx<5`+us+*u6VtJxIHfVs9k8CVvVm4+kxTT^2$Xg!7p}f2 z$KM6vAzN0Z(92R1=vPXf8WE^YjhIYZf0YKxPUBRleA&X-j5)x^s8GC0;-Nk!$~%Zc zzsdWpW#GiJ3_Z#gtT+BU$K+6In3n5z**-9FSAT&&B_q6dWIbsQlN~Ti*)onT-Xip~ zzGl(2bV{0~#_@@v*!@*>Q1}DOHZ+p(o)=>l&Du zgYd>Sxe9fMnHhZYWo{pfdJJhYaZQg)tnQvQ5S~Fw`9Sd*A15yAYKo2+)6;xLIV9HP z^Cz0YJs5Gr)T}F@b%*oTR&b*crdB}=_(E=hrSL)~AYP(7fL@*MTw0lab0~a`f>YC<^#OotFa64{B`$s53CtnHU_K59z($cjM7Mh#JO|lHB;A2Ve(q8ok*^6rka@iVRNS40x4|lv0(cQG?kwV;) zoL&S?w9f~W3hLGXv5I(wJeTC`tpS;DVfW$If-QWN&R_7v)W6A73mIPY1b~_bcRauy zO4zKiD9|pOMVX37^s7U=7J;Za$2&FTG-c+rh38o@;@V;w4rQTbgUB>&Wwv2!>fuJ) zpbl$f%j;5>%i{A>vKJDA8uoQW+d%Gv1rUNygDIhaoRJv3u8hsAWvPTFqulyD8X=#$ zoc%7O)ykZkvnw`@p_tKE0j{>L=$H?XnwYZAm-T^O@bTpRe1z%vb+^ln`%s^R+JY8hF__?;xXXU zsC2gVI7Q$5;Ia;J)xs;m1?lo&;S-c85!9EM%4&(qhQGAV2#CvpcXXw{fW*EeEZy-F zIS(}6Cok(GnWjluhXFXB9{X#JA8d*u^=;Fi)gr_p+EyvB=cpe=XO~6y#+4S<*BVX} z5a2g1^dRPl2&3S!w&<)A+nTL}hZU2lz?!yrw1#WLj$r6h$&OG#u=R}+hE2HY>Eg@r z5XBlpN#@E`ZhYYz2VdM}Jn0?jBZ}Cv9Ye^ z$$vgEosoVxb{jt|Dd)%FJbk)xrkw#Y5RyKwu< zbgHUnnW$+w8~~|MNtVzonQOZ>lw2~nY&BXV2Fpe`W{aH5>6N88|j% z?_0C5xE)f;Nh(V4Wb&>(YLJ@q8yg6(GB1C9Pl4jn0f^c*Isr{^rJ8>(!1fS5_T?o= zQO$SmC28wPEu@$>$vBGJNwnLNjTM;Ov}V5a1+kJ53|o-wOrPaJ-g%9m%^33BM!Sn5 zT|$VUe+=H;37q5t4pQ?QaqhU{1M<`@@u^*eAiH&MLe9J?sp;pq0k)(CvA(iR=Ou*I z+cAX}AeZY(-CD|B%wz%8))B8T-IHQ(YAXI0RLUu;QXh|AXN*6lX%nNJ)i%#df@}T) zt6$_aKlh8Cx8u3UH*hAKC;bGmk`^pI)Gr5kT8-6rjeeCKL6Y<_1`|vIqI!&AYe!w# zcLCGCV1MCF_rg6S{wrlZK-wR;4Fj!y<^@1h^ z2t@?&O}fG0Xu{yqNj>U9@F8ym5wCfl9wj3Is`*1lib+F?NO-W*P-&!VvF#;i3zbWq zn`iKf>qju3uNBiJLbJq$d4@sb`8$!0YjA78NJZN_A_$vCXYa=Lh5j8eI#)bne*2Ud z*a;w4T@h+r4i;FM?LFBpAREyZeW8Kr#BhC+DqjgAtVjQtS_QKf5&aSClzTK_4xh+l zC^h|C9?NMq+w&qW1j?jHZ#WNf6A8A1HjFPfLDjFbxabn*l2^17+2nv>vBAsaM8GV9 z0D0k07G|N~F)Hsdt~{^TBn2)Ek?imx@S9Ge8?4aidmA(9dJjUv8`9n}CNCja8}5!P z6N>0!`^Fy2VdbW$x$HS(g&IVf+o{7)1%%!4JRe>9!W^BMP;0k08qXG)P?1OYygVva z!nZ%WFKC=-Z57r2;W-hg{)9A3LkYadW3A$PCK+`HbWV0&a9b_EI_cKeHU-Pa09gHg43P3Q&+=Y2?ICSx(dSra(ZTAU;{2&N{KaKrdVTO3}4rKQr6TCufWJsCZ# z&%T0%*<-L;wbkHs^y=;lb&qhw_PSPUG=_W|&*vX=NSgA>EOT_yu6(CbWBlJOz1)V` zmPcV8NS=K4=!QxXYE$VNRt+vXfQ+N(oZ+j|vDX)rFr-A4(*@$Pd-gS#Qhb%il=0@M z>OP`5JBqv?$Pw8HW(vWyWnx%D-*pj5ZTml{np=Z@mQ-qyds2|Or{$&CTK|d@#MKe8 zLVkK&P+dQfQPUHUQOiiACtgaO_9^COJ@FJ6_5okYiBPbawmUFwj30S^0@#vqWqu}; z&8Z1D+zxCZ#JF|pldHB^yTq@mJ%mT+na}j~whs|V-UzS;67rFN(V}?UK3Zl86 zE(dz;&~Jw2QsdVG%w?$O*V;218Na@Jc7^-XFl=4c{rhIbIf*f|`sie4M;_^)3{B^= z3@{{#OoT-fCHvisX-X+VY=B)Cv%yzwoqd)PT1gdmL{Yh|*3T8h>}p^)gro;)SAah_ z+M5jX-duMC3S&p~q#eeXC2abz(p^xgcw$vbt1-WB_lG;k)gXoP#OV2y1ONYuAztS5^1h-6c7If&%i5Uhfs| z@1Jk}IM068e&2o8*|X0)XV%$ky$5+a$t-O+vp%JXK>xD(-krsO6w5NMxh*E&%v1b$ zt-Q)&gT5*Cwkqo?uM8r=x$!EAj!d26n<|s9>H<_snaSfOyGL6tOTahq@4jj%-d(~JMyr)g6GR>wiCojoOH6>bOMw`qY3Wug zmbS@h0p%-ZgaauYHS)*Ly}lk$7YyB9yfPgIe~y=SyG1#2l;>;Kw#hi?b$wKPz)+et z)gr0XOpeIn{53xBsuQ>L!XrGaFhyXR4RIj`nNr-{BpR>b*SGK*3RCUT4 zpZFzhX|Tk7oTRT}cvKxeFt^DyC2`;d4>;YgOW>d6a42wakd(3OkE{qqI$asi-=lSq z4XK-ivV$+(+W_vJ%=vs`sA}s{0JRZ@=f&g(W9#5`P9b&mG-dCEv-1NDrS~}_hxDGJ zz*JW84~48vaQ!Nq+4VKURj4+KKz%OY`|(-FX%?Ynl=oE7eCmAa;JFB_)p*xI+oVe* z>oU8+iVxf28p7=U7=9bY42Y@(F6`SAM=4*E$Azs-WZx8$aJdb1O^SDs>cfg4Plc8| zZudrR_9iAgb;#(mguncdkLc?lr^l(N2sD7c>_K}`pCbJvac;M4cqow8hpajPq?brm z)Gq6=MMx1UICf!%UA?8{VF!4XoTcjK?%`78sJUbVdfXSZn$i9C6Wd+{w(`CZa#0P; z5SgLr!8%*3_F0hGn=C+Gk6@`@khg|rJ4>12^LhugHYdPAap#Ebtcv7gHPa97zJLsN zxz2v7#u>{jA8~Y>rPAA_ul{69x-N^aTQt&p0^tp~EB0ma>VNnUVBL^x-xB}Ylv0cN zzWnEYleOcYds7I&(W@S*9h65#3{E}#+lM?%ry6kHQsT2tXy zf0-YW(V!BvIxmt8VB+--O0Lzs`J{NNK-T+dDw+or)B4uE(M@HcNF|eRIrF$EhJmRL za`{8W-oM?`B0eC3@!Q!>2fwG_W!v?!&z=9}kta-6*DaT-X(gp|l9w*Fbl(>)MSd^l z^XItvkeYFDSp?9sXRnhCS|vH1M)#(^2W6d+b9KX5y7!3dH6)!&4D=xs;_VxF^q!bA?|5sv|;=RxXhZ=W~D*%B|eba9H7F@V?NJIsL{C&I>!e; z(I{6&*Elbqc5N=54ek4Ocqj{-kbZEcBB+&8uWl1aVfDrtHky9&1SDDFxf-5&SC&!q#iHBua}%fkF$vgm@#Bnusuh;T%zZOYsw zJKueBI9jd+CYZzih&H6JUVc(mJi3tFX2Lf7t)|;a@T2+sbMIsJ@0IS!i4+-=G3vvz zDJ|xMvt=el93ykz5fYx$s02H?rxMumYn8v#_ITD}L~EkLJ|8yPvPpMBzP{!2HE5sh z@Pan)O=b!#V#rxd>FDsNbBTh(%Xd8!?!|N2kvG8>IY2krO0+akyQitROb&aLR9(zj zom~}JhN5Mvlt@jF%u}0SI!SrBNKgH#a%Wz-*<-}T=aq}4OXI6Wj|Nv8nw%D^iV;zg zHoGVd-}0d;&2BPT3DGb^Yp@YJY;Be45G;qOPdYxx!dFY5n+aUp-q=PhHy>CFOEj zIt`hv_1TFDrMo$GcT=wjxxtC2wmC7I{CNH&QzyXFjDRPPI?mB>E!WCzXJvIatTkY%*rM`lmF&rMA0>}~AEpstR- z^@0;%s&IX4JEnHGIE1r|q}o+I-$f-0q`nL-fMa9J&wHC9Jt=IqEF+e>Th(l9-Bs>n z6s^B`iY(H*Max&ysDypE?{EF~^tCB_xnDKU+o#@6%$3ZqF5t8Ms(YKjFTQ&)GU*w@ zv{JCQB1gVX2Tf)+k;K-my-M6;JK%xESMgQ>zTnY>mDhjtneoxoxZoEY>;2o(y_d(dxpL zV&D7{qD}d}xu(fsudxXrwB#`{63|JU9McT&n1>UFOgb!lJNogOImMnb)e*j$H_6hz z8EAJ5+10+d8cg=vVA~WBO;Mf3ABz(ciLS*t<<6+5kvxw(NsFWHUXI&B;g|RxbrF|= z5fe9M7wYDFaTNQJrWfCHlIYn==n3Y)RgmeJsrg2l)!OIkmj`Nf&GIi|wSfPP56fpAJ(S;SmHSQ!HWTHsrm9*C$2 zt|E;?2V0EpJ7vLqKUsf{8Ghj9)Q}EB2hN19hL9I?p5h>i9(96R9mr0@D4B!kxSxkz zQd>}X+53ij!5A_LH&B2smZgEQa8&6k>UDwa7*$uOGm}EGU$QD_TGAK0CzQ4jj%J6h zV|O<%Lc~;mWL${KZ6b)CPVeQ<0K=KjP@~4Ks2I_LNIEGvG;Xj8_B7I%7V+shUri2K zR0R5CC}Vv$PYlL6qwTZaWL=Q<%VulddX1kDH{!xts|Wtr`VAUzM?6j4s#;$rTAVE6Ty4zwx$mW4A+NDIVR2Yj=-@}^k4#! z=5&e+qjKXS(pwyJjA3qxZ;DCA*xLm86dA3vs}d7&Zy^|cC~MT(*(Qm5PTWMf!EY$L zfw;KNG3yihy}{=q&Bhz!Otc8h-_WEW{F4po5z%}W9V#n;>MUAQO*vJSHw|$;(wCPo z9ZFp}1#8cyiA1HzJ;q>t2kBy_VSK4V`;lIYkBwp5g;zlI%66C2$5Vx#Am;tO8TehJ*9i5hDxM z;8EbV^VK`^1D@<*;9Ic;CUPl0e&Ww)v=>cj>AQ}6DlK@-C^_ux>|5*8ynJ`e*_e9s z{dSWH=4q!K#s(K_j{$&sJIpQ9Fk+} zB8N)VkCFm!h6K)8Cx<`S_r&g|_|@2Febyc`??l6B5x3DQtZ07Skv|dpvTu(nIEA!_ zcj*NZe+^wLMv-iJG{Yh&tkiKv72C}Rs4radx@j(7dwJF_I44+TNR)>HiB5K;s)SIV zRmc6y(%ksCo4(({wd}~vD@~gQxt9>kMb?(g_b$4SLlmIF5ms)}py3$JoN84PknAP# zc2$nuTRJ6Gi|XAdrR#x_kEfo~1`a*ZQ>s?XVP)+g)c&k~WbqARY}y~vaCGELzz}CU zfz1$CQCf5Ki^rnO3z)62CH=w^n>Tc_!^djrxMJJg|u>%tz9c zP_!?@bv5tH$Z7%miUlyZ>09-cm>C>|&4ORe!=VnVGF5@w(GvyGV;Gv0OQ@Mgm`w>7 zr=4-fi9{Ykk#ymD442>)uQn4M8ZdEM0!hP>*^$(w8HDGK7!4MY8w2CSo5ApKAXslo z=AcPxax0Ky^TcGN?$Nul6vOKL-{ebj1H zW|)9JoOm~wGXzFNYJPE7>P)G4779}ADUi_g%(3k4K1%HGJeW;CuDqb=F+Cmj`N(fBWLG>lDB)of8 zGh`R0NKKxYNb)%aNr8_>av1ZCic~y%^Qc1*s!i6RYGUBnxQ9my%$%NnY5n6bRWq+= z;p58k7|1~c<@K-Ht6vkJTzAARd|w&;D}5Ow;{6TJ>Sg<7$DMG;<;vxe=`+zQoB)7S zX@pKh({!AU5_5OCOq(813N=;%$B9_OkZz;izfHCGfPtc)oIYti29IbhB7$F0H-gTb z-kTq|GMZc12BG@V7V5dmx?br;KGMD%=XrXiy0bh`>-i1hK(*SFd->ftdf<}#`c~vR z;L)Qki2o1Y!8q*mE8HIxmzxN2kId)eAcwHxs%HW{No*f1WPO)_ame#a8vedPenMlT z;a9GDy`-p_r;3c?Y{q!G8=|=>rzm1{>uOo%MjsTf*r+z=cjMY zL;_;JDx{**1!8*Oq_|CN^*RA{4l&{CHB){i3NW4z}6$EV$2XpRO!Crf+nDf zyvVYe>`9|7lArr#UO;`!x3yHML!rEWp1PP$-|W{}Z2F%n8D>{TXVPqBlDQiI6O{u= zz40biZ5|PajfJTu@*V;Kw}`PR@=qBWs^t|0uaz2Y6vvhrUzrCcujJd3;ZaXC*ywP$ zQ%01sFu{qr0@VvU`3-VF+hJwBjkea)6UZ?k0evwlhJtzt6Ukoh4-9Vg64T0flAnZB zU@>WsPBS{yZZfuKZ!$tQA8>y_P{CG+ce+5*h)T8jofJu%U(*y0;g%d^Z642Y8;zP>S?GQTN~cqPwr|`9Vv1^nuXU4tJJh} z)%7!7>NNAkUp28`EzWUE2QbkEl|+kOjCf%o0nJM6Y%!ckSGzoAeV_G1-$F*8U%^^O zw~|SOCu*DD#4})TFp`R!)O+LIMhUWP4zNM+Ewo$H6BI)^niVXR`k&cH>zm(1_iah* zJ(iL$SRnHsUozU_wp%me+0+>=$;0BUB)={ocqS_Cd#~wnf z#+-tf`%?t7fc--n4KYGsWGRGqDuXW^Kr1z*eO!N(5liZQ~a^cI)GNL zSw6c21{LVjp^c8_rX3j98d^^BuCTa(BSrT3?TidgTa8qL!5y1a-F;9E{Q{uijh(yt zLi=BWd`&&?h?9Hw$5<^`aratEpMKcABI`NyptL7Dt8y|a=Ipuv#BxdOO*Lw9Y*$k2 z5OpYi6YTN}44T~gR%8;Hs4Hdp@c_v2BQNiHufhu7w9?gxAOF~hUiIDUm|U zx4W*kIfVtYM&J9Ji@S5`wFTwTHAp7qKToEA?@7YZ zco9O;rFG%z1bd5P7@+57j9i(cQchb@_(k<&OUjS#sxRt#t8045*6kzp4tegtM%(tp zIngTjh`IJEcUCyx_9>B0{tCcUwz4VWZ3tyAj^c5#7L$@S(o9YemoJ)ZPrigRQHKUI z>39JzI;08l(%}0-kP1n0wVh*k1mepuQNd2s2%>5RUidpvT^BU-9@rsIGW0Nzbi#|l z7fHFs=J~#qeJQbMWPw^ClTojM6(0LG&P!f746HNL64(^_rB1~5r?}g`%pf#4LxoO| zY5~uT3tr&!b45)67$6UEavfnK3a6r0C7P8fPwyjqqHCfCS1&S}#{se&&08sEuea9G ztFv6F%TsBMf~&Vf(pl_9%4}M~GccakUZ|4;&6!P0@l^_fQb;RmtSfh zGhg)7TKi-5p2*v45!rsoqV^2-La=<-ANdi}t)dgKb%Vy%tH;)WXD_N~-|J%6jbvBM zyAY&i7aD2njNGtpXb`^Hl;b3XT1vS)5y-^+LU4{oN+_Hx5t#3ex10Y8<1Y&eU{{8- z3E>MSwcvucs6}bx1~Li22t2BfILVJ(Rpl_IqAq6>QUg0%NGrtASqOAf`@)vz{(5(M z<2jWct|puEAsd)&8tM8&EB#ZTzq4nGwp+ut0x}xx1&Y{JTsAcNbMIv^8OJUxyrXuj@kr4FlVas}Ow&yq4PcWAjGc*XaCl&3vg_ zd~cl!@yiheHM?)~`sO<9YaO**D;<$OW)ITL4{)h+=SVOJ_C(koy=)Fa>6BQQ!poZ( zo>0v>8sI%j9z2960CE{|z}QASb5t7$tdwD`ftz5vDv3h#`($Zd@y^_Z?A9w`cAaFO z5!o7IS^Cb&&Ahg*;Uv4GXi6Lr7N(wW+3t)DL*IhYa-z$D7^errl9aB)BENc&P$X65f ztOJX)RyA})N(ur-i4ZVQms4-t9eYsVupg?1yuy6$Me~UZBF4I5*qkk<+LXSVi9*|ckezycy=zh0cR#>5}oE07D zF#(+vjJ?YLtA`Z)YE|?%rgoJH+FD#SgdT;~tf0rjwV!QeLQ*jK`cG6j5h>VZT?pEO zP>_QA*F~`YL|=O&X-vU58~XQ5^L*$+VDAl$dnU4$gd6a}hQU44PLYHXSZ34WUL@D@ zf<7Mu`q1kIZr{{p{BsVw^ZnMh(4?0OXwr)a%%4wD;35xrotX}7xJ7dh-sOQC_fxaI z7YY~sgN7*zTmpc*1X#f1TNL-;U+pvTGogQf6;WVd_#eR61)y+2w%?Qfuk`E#A^8V{ zNfGd*un-t~3+dkY4Lj~4uS7X!L1)=WvkID2StjOI`yFVvv6M+c{51tzx?LH>u-3^h^NfKRN1 z|26$BQG&qm_Ze|ujwB(rFffiTmYkL@#%6ZroKTGdJI;Rw_=!HXLBRNfrm+LF+ERdX z_Yv>S-ve!v2iKza6Al31?ce`)9>^p-fLl00;pA%nFV%$qE|K3Q@BglN|6o*jS1_(K z3okwQgDL%YZ9S-~{(y+)4kdE_J9izQ>EH}sLrsLxd|jOXS)-5c7Z!dnj-=0T($6}> z4F5xXP(btnaSZq`5!09Ce~5p|6~e&KJRr6Q|0bd&1|a==0i1_)aC~Y1^gt7iiT>@1 zg?2Hf|LPh%WP~oK6R3eb@0URp775IG^i$SG>4**vQUI2GO9YEKX@ei;HzFu_bGRQLS(_pKHdRIK{J366$;Q%Ip>WSr2!$xQt+%$(5O v<39-MJ>YA8{009z#z%UO@^{eTcg*1d1!3_wg%V2uMgS%YANotV{S)zjc9=jx delta 39055 zcmZ6yV{~OPhb)HS zzXM2kh6Ci(M*<>p!vPJ?;G51dAtC>%@9@tNVL?Ekq5pk?Fv#hvbszAIKnbABRZp)} zN%R|qR)jG*7+OmS^jL=qI%%)3ME^o$gM3Te4+2pv=w~V!8;ORZXk949Fxl|JiMZgkGrZF_RwgkF{t!MAmjs#D zoliD7MXvrj5Y*)I1<@Io6KOc03~+RrX|gJ+!xl+%}`Qk~+FEQ8NZyJH9;A zM!+on+=b(NYY^2k>JoLQ3O*NxzlezqW?*o3wbP_}{DM0PJjq9Awq`P%<{6?ua^Ae1 z>v>TvMtWsDYl`;*LJD#>rj&YAOkwr}qY>Bb2%8Tk;^46~o?fv%^By3Djq7H=k&M9F zVqe)g$~-V{S7)zT%m#BIuIVBgKtJZvm>Nr;6<%qMcAVs3`Nw>Hcz_;H+u7ON>+#ra zDpv|4ER|+l;qP;6EObJ5G3Uv#{e`L%pYb|h8l6)dyMuYug?t_Mfyt>Uw1#V(u+FMy zOfySG)T1?h(;15Au_yop%kQ=*91T!Ju9d03lkDcVR7Ei z`K;5mxsQN_TC3x5SvSUc#YBwQdxG=#%3|PtM{QyN@EFCL5bRLdqTn;Ek*}@dF znqpF4y+MqA9c92SG0u?ty+u)P5mso#pM`m-c7Xx3w-{Wk-U1KecY+|W+ZHD4a}m3r zPrgtItB8C0aFSLR(S$U(&vcE!5<-Cu;%rUi&GnYhjTq4~;~p+z=ICwn!sx^295s6A zPT<1Z#Muf-t3jV42}P@nOkJ>Ms>K*w+flg$l7_VT>vB4Yv;jv^MOpnBEwI{m(dy~j zk$+=#u=(5N9=JN}i!C+qq#fq{mPq$&18>(BBDo-me``9E5fSP-7cv-FILG+vr5ACZ zC4T)MfBql$YIirv1I7{y1MC&igptR-ed&8#=v&K)1T?{eXq^J1NJhnOiODeJ=OvU4 z>8_L&Yke(zsRF?Jq8Z7QrP=PlUJGKH)|#@LkNVRox>%c<`Z6|hw!S`|uAl;Owwdxx zhlOy9cF1DSsBooB2K&kA2vm7Waab_w3{DDlTta7GjDy0qF0rph~JgcM(cpmP~ zYnHjh?n@|ffM{bcw%`8M(+PTymEd#KL-!W`GpX@!nX;)xax@ zwl#lU0l!1uT&43QNM%~dRZBn}yIMB!iR3%{H`!zN8XhR@Q(0>|dZ%)aB) zDe)PdyAo2H2h%Ee01SgIO-HMEIH95#GLHqePnHWS4gI^?kPVbegw~ySbHmD;ukja- zBAxGnB6~(3i`J={u3xtm89^FODx%Za-TGNRQgaP8yV4p0ff-y#!-4%|l>eNv|HAGc z)(VZ>GSqfz=-=avmAaUD9$L7>oYuS zgd@Vc-=SiPaq*SJOIMvp_RpaRh1jp3xD=Za-3#Z|D_RxHiVh_3{!ac6LVS3<1Oh6(Ew>O&~F@kGMBc}3AZfdd94lC58* zy!Tv?;7NllfcJkmgU%_@_O_0Ur`L$GGvXVZ*MX0vqx>-nq7V@S)f`et`a&>aDgMBU zN;6ce=o@$eT0ie*59&&+ByB0RC?yd6K=xsMNOx1@iS0RZPx_-O z{d8k_fIkBPFZ)aE_u3lt^ zbcB^4yF zv+%tEalY`M&ubF=JVQ*WRocgD8!DbOJ~CC<6)id~m6^QUL(4Z68v1RG#XV-};XTt% zwu`!-?g*0IOND7Unv9CuCw6a4aYaE*?~SET*2 zTwFX3)G@^ZD3x9@hbwE)mO{KO1hTRm?)P}-DGUp@WR_H*Tj=8XY1pm7SyJVh} z3)VHL{e%6MT3s9xJW{JtisL&MLzW(aFm}PJ$4#YTcKnpG(YimW_c6g>HPE-$lGOx| zjxmHU+MMPK?^3W@&o+sZm@q`ZvlQ$Lal%|^gnhzw%}S;?T+#j^G=M3v|Eea8vIURA zD1gB?YzZHon@gD9WpP!pLC!k&)4h1_$0&FuBhG;&uU5lkY^B5I#T;8t1{78%1?ddF zw1yPp0fmv7Nw)y=3^I@0s^3{d?FK%;kXlI@m5@CgVlIIr4%({yZ&1DLh{;zUMuY)K z2ZLsAW$JIl)AEoGV4sJN+3zzSyMuNacszdfVuPP4)i#+1b&^Z0kL?Ukx4C6{h6r2b z#pT$)2IndAD>YW_!=r1McI3R>+tl5D87UKC-#`p-K;7ZknZ9f}RCfn0F6q<*e2>{4 zc~EIik~7s)th6tKANKF$2$HW07(B!J1iPhezu~;n{(eVy4og>)i1aKWx9HKKqmE>0 z1iXsTLk(>;cggz-&dAVxk@_`wnIL`LSld3H!BRMr#KQTHzE>vHO(u|@3?2!Dk8h-Bi27L|7I{gd~B_fTL>N8!%$a_cJ_a7hO zvJYWi)Xcpx&Bqp?|BPW32Q=RPDBy?x>Wp_{<|gS`^ttFjO9*@?>;Ts#N!MhaOl2Oe zB_60HozkeF*YnIb^bjTx(qs;j4yMykZN4X#60E>RHFHJhOc-WkFQKLcNbT;9yGY>& zH4!5RHO(kJ6w@wD6I`zodwvcwbsccUbi-0Q zTl-s7*&iT~AUFTYRSBRm1_3Zo1`-HH%?S9^D%Y)ms4Rpgk?AulU>t3>UZ$XsKbKS) z{M$@$zSp=l?GOnV`JTrzWV#!8y>uiw&DoJhz^sWx%HefA*>=6*&iM?uJEjf9wZTbW zpEVL@q~=?mB1Pln(PvZUP-a#(m*Om@4WSS%)Z#IdYV8g((mI#Y>?X{64CUG5j{vZ| z#jTyp5^JjD9h$LtQTm6+V!nuPJ%wFN3FSsP{Z%wrMiH(5ae6H<{g=37Cdep+9@C)zG%u&$+GEI zCgS%ryvP8RtFtY(BZ`m1#$GY&0tMXk0Y)SnPGxs~KgNj*md{zp*rUaZ-s)BSb-(Jq zGmba@Yd8-dtFoN-plGDY; zq7|W}{zJRMN|t%?Gf^k+)Zay9Lu@>G4ca4!?Qc+wuw6JxV`|Z4y84ZVGBoe8GtdKa z#%LeY0c}o)ZGoFE)mP#3X4v2}Z(cvxq*d=NTrDNQQcCfQgaof;DhOXIUOHEW%>-(F z1reR3-)$BB+BoDIZ5{sZr2%6UsvTw4F=2=NyfM5BH&^!+=0Ed|Zjmq65+n!+1qyIq zm=f52fe)C`I`c$RNB{2Om?>TYl})4(cNRNMatjLP)vy&WZx*k?q-B7gNI)!Rb+=dy z{@cq~{%aLM30zrCNw>N^r-+b~?+YXMaCdLN{AE^dkvyo`?|#bH@w)5pdy?Vz+dUr0 z%&VFNy+fLE1TWb(brCk6EKK2DV`Lf^BaDHY#14Rt7)Lk{Ty&JgE>StW#!p6k-VhQE zlN4dRd{qC#TGq*2&4ekK7>C&o*g)X`dGsj@N1KqB%txtfQI#Y2a#2d_ zQ@&@SS4b2k0U@tdX@rxBK{qB2TMHP{30j|6nIIdew0GwJb$-9sNi6BwYF{0q&R9zrY7?V7t_AjYzYdHete)EZR!;b#5(D7vO@sof z#>l{c>*52``KnPtQW?V0v@Mh$_W6JS;>IK`*xyTWARtam){^HKSIty=HIAov>%BSS zXa8SV#KxzedqdI&FH_zjV0`b z^8+Z@>6Q38(M?fDQkr3;%Mv%&~zt4t%EhDBe#*16J+1*SH_ z3r-WPM1PL6M5g&lvZUMQfGZ08fcvSyejO6#Fdouy3t@=ZU#n`9xRVMnxD`qJ)d%nV z<~na4uy*Mld4f5sx}_tBoriP=ttQk@d#xF>$=$ixHfjpt^QO_+sS^O;zxuiNwt5n0 zl@c%GF~K>eOPeCBkkad0eR=h`7o@7OVUjHkhB+F01HU#~EPKMTUIwc>YRIp2h9ybx znNBSX5zob=ew+N@feUc^tzQU&6|w{mD!eSF zp^-E+Z1Eow!9QUIdgjO5h=U0%cM*#$f-8GT=VmXYFtNS{n_`MJ++WS$PWjoRT)T%x z=X+z|W@o_7jlc3^64w^A1tmOy>)vi$hI_np=TYo22f+Za@2}$~Z9WNZ_Bc-u$hyvG z43g!{e@f}GMdvaO;4$*9N$o2l^BuqV6Vx_!z;MDV4DZMsHfG;E?HQ*ZdE6n?9d7_C z!0Rh2Nzp^)w}(o0^c{wGZ8S^D!-Ln}5GG%|d%M7RFa-3N!Y%M39E~c)>2SB42Xurt zTNjb08ITq1WKU6=Y?n~f@E-nFP=qX53T~HHCKMf?BF~M(2_;TiL`j8k`emYw zGy%cD^|$Hy)&;z*H$u%G=%=6s5}e*(!v%%weNms|r*TCVc?`U1ex z2a@8@VR;s*O0{XI-BlzZIafB4lP0C*D49j)v*~wT%uw5)Lm0u{Qk?jg^Z`#{j;Jw~ zDw)@f^n7!V&`s)NHE9>MORbYr3Pr?bH8#?$k9dMVv=68}>8;;hU>@EKK*HEX zOg1W<1;8eqN;cM+cGgUW{s5QMEmhZT2V0T}2`D~1G0S)#2k-f0S4EG&+6dKK=J})J za`j|diCa14R`mqC$CU3^UH}InR9hmeQ&qiF4zgkHB6{uaDZ7bTp#hSFLe-u^Zh_CM zpE0DG_#HoKT#b5>c#nVcPY-}-OSWe4rLx!Z8*0g#q~5lcMS8wuAhwb`Hxx1~h29;u zBR{H_ojGJG(zTEeN!pdFvBJX>RWY<7oN}2PA(BG=X)xL&VZdn*Spbk>!npR(qZr?l zeUeXScL#+wTwf8%;hS<97pbPU^$GbfUD0o;8PmL>IYtvU7j7_>D2}_kO2q9NSO?VI z0gFoqlETw7k@MgipOag#zb&JhBeWVG(e4N|OzjXk;+x7M1nvt|(JWu>xY)+DQ<0@$ zWj}Z$5}`Y>gg^d^tq!i!dv$EH6sg7gjhci9i@l8XYaVd907{nZ2>!JLt#U;4&SK(}z}|*?*p*M7jDs z^!azXlrc&ztbtgDh7&Wu6;>e#3C zT8xi#UG_6uRs_?@!+*Ujx-7jBo+9e~%}+QrpK9&qg-4U;@Lb__D+r#?l zmU=Qy^?~wVttLt2-Tv@H^I~@xfEiZ=hc#DnMJ*41>w9j|^yp5xc3LTV=Sze!0is_& zrJn;BJNIJZOE(m=&R0!)uQa7-x#^YJ*XXQ^I~C@n#0g-?+_Ois;{vi5odIY=G=JQ; z2ZN);31qb_(&47fj>9aY*k#)dD%9_W;6r-0RjxRO9`CqbF2oU*dF3aBn>Yc3xh!&@ zvi9oR=GaKN507@faYd=V^T&z#17)>$I#&aDinlC|&C`8y8XZ}cvN^c3Rq9LbORnCX zBN?XKvfK^2-^X8sZ5$W+HUP>KT!c|-(#MYFGf~{Z{A@>}jAS&$TqbnfLQtk`;wq_M zOf}M^U2qolbf;K0tQ0qpP*pfCJ;?_ZsAyt@b zlKEu%&hZ3=2D~|ed_q1xPxy58g%^&p_TtaDChtDvHs-72B_I9E-9f0 zT(2%D^WfLP_6EZ5z=p60AxA+oEYNB9n}rX@o3k|)+e&Yt0Kx@B83RB`Blny7#>(?z zPFfKZIH(!)=cfE;U9b619s@p~-rxeolgZ7PNDK1%-kE4k6oNt22Y<+rV2aa8OS$sK zQj%zJ`Kb@9qS%tG;`ngL0k$4Fagrd+R{f2eBGRG^^b+ga!ClN<$24>Gn|Tzr>H_M4GOZWnxJUY8ZG^}5Y# zq4IlEeu|U+NwI|U0IorLS!Kq$!~L*if>&@eF(6lUYTQI<6f!q5KSQU-cx_0eb#Bh; z1!6W2?M`nLv(@^x7VsQwOv_=e{Sp&(h0|*J{-4OQzKN<|8#2(%zyiVUYH6hHnn7_|_Ay=f6Y(^lZN9n5d zxBnhni7QK;AOAwx%9%2aTZ8#&XOV0agAgc;fQuPIQc02-896Z`2WYXxk-xh2m*PV7uz2Qw`jSlxvr#gB(|wn@dK` zPic~`IO1n8?`PKWPJ9+g?G6vDm*AZ0JppSzmI$cy{B=G8yCgQ^F8MBB>??;9jbB(9 z85?kq7-s<3H=UzQQj0+|?XM$}B=)4pa)D}Je+kk4V+GapQBobn1LwfGB4jCuHfaUr z85Sy)Wv;WwsnQA6VZIIPq$%ugg#Uzj23_A_87K%y5f}&v*Z;sAiC+o~*&nA#hu;en z>3=CcZaT`#3gEtg5a3(k9|`gG>~ocJXl09nEG|yBNJuY<0tmLMW@aWy&&U37v}QcM z^;z2{koY-5d21*Ijwyaf9*eYykF1M(ODMxWndK2 z4daVm^LfL6cI&hZF=^)Jl|0IP1uva$w)g(I0ayQ9LYm679mGVc0A`Sh^!egss3X?`ey*XH` zpk!P{2%4HW;a>-`GJQGW)SfDhyKnm2*}CU;}s-^R;2k&Vd~eUt4ybJSnv=q z&3WU0b1@z8_WVT}C|-RkWh@cFfQ0~o(8QxQJ#kzR?%g53x?G=M@Q)m8)92kbQi2^J zfNVLx0%*ezxo@@Pwjz1be|N0AQ?aVNWDahi^wJ+Xs35#t1rm4pqi^PJLHlpntM?3% z?nlo*;KXEJ=me~w2pgd|)Lg&OHyuawkFjLM*`q!ywfi=1up=Jj=|gbCY9ps?pq zFCz^%FcIvxPGf7I&7j-5jYe0dkpCs@A9*H;1R!7f21s!|-%AQ@xRIjIXetgXdv40c zws|5|TWI~4IW8ut)2{Ip1~oUJr$6Bvi%qoBxJ?hGwFL13V6ud0KlVZW+vnyIPffWW8cbH}pDP$QF^*M2S7+jBIT%GeH>~)QQoIwHjY)%Dn-^CFjB&HqH82wemg7LW~m(DqtDHO00g7 z0O?4x()pnPl89sfDvXRHX|r)o6OdZtWIEuz8xkN~#M`LK4N}}6Ox_<(eIxxZyNrYlT&pZFc9Ht147; zywp-QFMnp?1|BS@&nhX7+RTh^gT7HB)f2*86D~6r6zc=6J%V~u(@AjfxWZdYAjnN z2TQrd-LM3LBK@USIT{Ni|Cx9lbcN4hsF6o-fvnb*&mw;}_KW_L@T321a_4~1E2W4x zPc9i?=5!hkro*&AMDvLR#hK;XKt|Ku4<($T&(0l;-9q$nxyxB*+-IMhjC@(DDtiD$;|KSM zMS(CC#TWZdnQP{>b(dE*j>BpK+>tR6<47+cgL7`t7xoDKs0LqWo&0hc>w(YlqKHF4 zkaYPAO1|EAAPl$thR^@}f;BxgmjVe+J6Ih+!11gr4>!a2tkhhuxUOr#L3p+65%+#_ zO+Ivw79gPEk zX651J6tlQoo}htjsrQ?aQ;mv|x57wgRl&>e(3SKKefWe}kd|sI62Zt{rZ__lhuXH| zq3sS)oWQ))Ewdk{-(*d;mL@BUXwN4jQHC+D;{qEjqNY@R7Bzhqy+Ui+t5FYs9&&8S z7r)MMXvk&Jwl)kr2Zdub0zm%@692zZo2``ZARv)sC`=8WAsnZ}_9<3Vjbz>r^#9G& zh3~!BhNvJQXVgGueT;vDJ177KQ#)f<*BotcBej)|Z~uwu?THx(5u$#osd18x5Gs^G zTugSEAGqK{BnnAcX5&m~78D}e<-zV1BdayR?$kA<@tpYW@8irdh zezKZ1_88jly#Ko57D>^OpQhK`FWEPKC%S+WUQYvouSa7L*5eBN=0kwu`JLs{Yjm8C zU;e%*frDTEVco0L$?Y!NhyIYD%N>QlYQZ5I0XJ9SzAC|xGaJ^J6Cc)Jr(mQYibB%} zGjDR;F)C;4gH>h?K7)njE=_(H^sw3@S(hG@@(@Zu zDJywcH}nIrx)@_x(-nYew$8q4yefK#D#q^JIkWJDbxl@!^bkuV68_q7U6G2yl38Qi z28%ebB2#XOSyN$6nS2SuynsTS2XhftrMgIGMolJ?qJ$@vCfsp?tS)(hMy1w*rkh3d z$|YRWqVicze>O?GQnO-WqeF|jsfR@pNVAet<&kre@+R{LVF1uB|BYp>Qe#T*J!OJ# zshBzsC)3ozq5v#0&@NTul^ip7sXhN^jBB3%;NBsWdZ9w=ZlEn}e2S`FF%{;LsCm9} zL?h3DiOs%WkE4$YJ8WVh$7CTc$%hTIHdZv?bdfavCr*eBIbMuin?Ajn44Kt-keL-9 zZZsq<>oXGE8}NMo=xz&s$bi@uL8?3Sd^9PP4bGU=uiHh6%>*1~Rx9G< z``8lMSX_1qCuA`@QFF?*r5Y_qrBvByEZbLC)#|r81I(W))u$LL(Oy&k^`!~RPh+4& zF_fEDwlx}9kjvEeJAzSAbk|4pr3pzF%Wc63iYF?NTEoHaW~tcqQn`8O0uUeuYcM!M zI;Kb$R-;y}^1014*usJ_V0KcWhDzx#w;_(WS&(7HTUR05GG<_0VUo=ywGVN=4yMa1 zXm|=&0b7ekbo1CO)?#zrMqJp2MB@x}CT{JiVwv*ff}WKZ+uK$Wn$#k~U-wd~M9nZbU@xwf_y;z^E+Ef!m-9kes<~s4xQ{rVdQ=FYIDYtVv{JC* z@Zdo-!7Y{BoHCNX&bPAC1)-&4WV4qisX`5N13J8>omfq?;wp@h#gYDeL5gWgvQTByT_YdR1V$uk ztMaP$>r_sd2Oo--EtnBxOX6SGQCaS)kV-{gd_oWt$)}v)5%73!H z0h*R56R#@D^ZVbxw1!zg%YMhW4rFAIK7{|E#=Tx3%O?rJ9bt@v&^c2y?3AqJ<-wl! zKt*AQk%DxKixbP5v0|~jR3_JxT()b_OTeFyMRV|GDm|KU;OzQ^ng%73v%PE|eyIG^ zJM93+LhM?}UacX{b4Y_pKyEg87r1V>4{)Q1c1;>8#>pjS3v-RILTlZ~0^0&tecxJO zPAYp=JtZ@9%~^ID1dOdTK?-u>uUapbO9Lsueb-8*54Z)Y=OO8gc9Nbh78r7{SLrk1kNF0<{Rc=W9n%O3RSfpdI8fWO$Y zv>K_}r`3s>_=PHYyd|m`6;PR((^_`t)i|u+OR!LqK_r9=+FKTnHQ6;9x!lPqV3Sc~ z(cDpCo`h)|WgP}T&tfSs$$~XHsn^>dky8fY6?_0@!G@Vnby= z@*#g*53)vqXRnGuIe!xG7)Zc7rVw?|2!Bno<2=HAYb5S6Dr zg&b}qZkr;YtHw7{85woO3DC!2mw#7x8bzKe?7awOxym_rP9ru1b^&E5>RGoPI4FO_ zYjcbJ`DA1czf9%FsZ^6J0t7LHO~Xg3^aU9n*o$5#W`Q#uUAWZcuT;*9G6TA29v`6Z zBy~Pwn?1Veu%9Uvc3r+ZQkep(+@ju0b7GlYWhs$p*-_W+7B!dwM1SH#MfJbDh*oDE zmd}f|uwC*l7mfnJ>L$R#{fhGPD@+FxdT8qPmrRxhaqjF*YlwG~0qMlMNl+CFY_!;f zCf%bS0WJLr4k^>7*~;obN80xYZQo)HU9p)}p=>1ofduKDUAktx#ZFKXSAw zipZpcS|Hz^vf)r*{GMK}ACd$5hhJ(2$1v*k-BvykXlg`^znr^Q7;KCpjEINx5i2g1 zhe*&z{}m5GMetG1W2K4)?rcOPY0q-{3{1}fkZ@tQ^Nwu$0o4Ts&w=q^qT+F1dC=e` zc_$=n$)@BF4wOW0vgL7!dFfenf4<2%gf2(o+YM8}xn{7raAj18C<$7&-hPQ3|2vVuf zhb^@epb+gT1~4vKR{{yBQ~{dS|zB&##nUZ!dV=ziTR;DtXna z8zQ#4-E*BwCOm?}*g#^R8fqFIa_=seexw)8Hj7nbqOWpcP7SGXl~vTQ3WouXgD+np z%)Lxl&CM=kCu4}W*Z=xg&j;^YxMp>Bc9pBdx>&|G0c<)U1T=7AF?l=wJY45wamAWo zmBZCi8kti$4<)`xPx3e3jgnT_dK46$4(ZNt(Uu9^xp$qzX26#55wmXNIyMa^LypIW zUXbh;gFVnq7UNDO`)$;~>L_?d@t5__{+XEGvyPSE;avq^5qLY(uIrU9=gJcBQlTVi z{Aop|2;jG!9h#{WEtnI+ER9frQ1N_E#5uDC5Gz0H*o3@=un_?K6?spqH?eIg+aJKvqrtjhOeT@ zJ!Q*TS5x`8-f`k7_mD+dorUurI?QHbv?Xtg01@t0bTJ!=$sU5^uaCLGHRbO9*QU*( zB?dj}X`C529F1vfo6&!GoNDFo@e)VnKWjwBtvE~v$~Uq*%PPQcjzU+5Pn#Jw^AeX3 z0r-1t%|(QPcxC04X+3oAcoBQD8sC^-Ud(OpZIv93*`1>I{@B)Dx+f1Fz4SsHzS#@{ z01!=Q#%%RKFatvl-zmN$ZoDGpnG~I;T!O zcIS01{yQAs*^tmE)!82T<{*bB2Vf8{)@8_2Ui0^p&_y5ECe$${Y(2w{Ay*5?WE*y7QT1{748ucm3l;q;=BuKdTpddHMS)bMe#XN|+M3rGY zr-GTKaOeQMllOz`9Ij7j_DvM6z^&0{e3_AcjJic7B*~G$xok18c_!}U{v+JN7wgz` zqOp-YE%5xjXKb;}^#;}(0=Qw}EE{*EIw_j&8%sKKQ1*}KKMwL9=Z!-};Kuc^TK7b6 zVvD)YWB0#QMsM_lXZN!?<&lYuGa5jVIHcsFzkueuEH&>yQ2V<92o7l2x5AV}SFswY zElbe<@DfDd6yJB07bT`I2zOfkH;85A%&Wb<-+!RP7e<&B)?U#y3TSwObwo6>u$R`~ z*q71tDCz3g-`kv0V{n?QzE7rby`QGTAJo>?YFe6U2edsRTfm%8+940FH-bZ|)0oj* z4Q%XC{${+6KAw0)wZdYwXGv`546!Ho_artoMey%Q?bW=`6`!Ca5!|1Fd*clZITGep z)#}U*rlNadLCck31h6Z-&l7rwliHk6JLMQr`LH$HdT9(93xxf7#a4~4T=17<%8l+- zte^*)3C>+X58Zn(T5u-g_v0TpVBC2*fbF~f2A!^>l?DCR9?7ToX-H_*RXe6>npPIS zmBl{7SdrBlHf^fVu)_q#LKK8XkxNs^PMF~k3paHfy}F?40W_)%er&mi31HMhF>Rc< z|0omXikR+Xu5j!$+Ix(Q@caGYigdttErMc$zJ--wd-ofR{@V|+-H%(~6PaOL` z_9OV?_?fmyG>4-_20&gRDIQLKCbl9gRf*A#F|Lj;reBiPP?%bc-Cj)EVph^PPON7| z-v-Xz!uCzF252gG9M?dr0)OIfPoJEjmOpRJw822;UOO^0p*@`?a&#i++JsZw9!$4e zPm3C}{Urli;qnVqU{n!>6+lsBLH>-w{6aeIw=?P16^-L6==z7L=$2q~wGK!B=1}`1 zkyH|!XY3Aa;ZF46{NZ3InuU;Yg5*Oh_0U{9C=Yqc0FX%VTCp9XdF>k11 zT#v9mhNYv5#d36>Yok8p4b3uqSPn>geI~rS;?xS=HDL~r{QRa3b1bPw+#LXvvqqh~ z)wTc7xY2)CFWRh-xLMTCo# zA1R(0ob~6Zdjk+4sFU2H<2&C75XkSs?h3mQ3< zG^L(7T=-IbPPNq-KpyS4r-&UZ_~7;i2Y`D)vhL_>qA1Lsx$+G4{-Rzu zp(=XTnuIG_u*hL?m_=;5m&##@P%WF@AEs`K?p1qB#w3&uO6pLF&j|4t8c8aiE}mjU7pkN3b+FxMmN11vSnj^1NshU zSweGgy!#aOGcdJT0~z~zpoj_KYaQAf^ZJuNW}r|4e$a#6D0sU~9Ab9SnI<~jB4JMa zTo`zUT@64PNg&Z^a@iTA)lasMxzcNOZq2b{r0lA(YiqbN@ePw5OR=jv_^Z{b+0+w$ z<>5j&`wVASXYQLr3&*>DP4@X@hty|9e4OPq0en#W?*?ZRp*}qHza_Hn8=NLFb6Bu{ z+h&SZOaHZP7I2Hx#Anm-U*f-YHz1rH?*El|?MTu7x5b5P=)WERkF^p7I@|F9WPP{9 z(D~6e676XH%R?!wVMzSRG$NNVRmMY-4oa1rmI8sv&Bn~mrL8*JETr7Tv>YFiuSoT0 zKeXE4_Q0M$s06OZ9nLYEgrdE-eNS>-4c{+r48K8~2zHMW!_vmM^;Vo``rQd`j_(Bi z+|N6S<_Iv|7Hpm%&{1`XgUOx)z7jjP*=`$x%-#FU_t3z|!P*e2kDQ_M{@mhR)2y68 z_RcF?U>n?v3U*B7&!BVSu#{M7GG){KEzq)O_XyQH;RThhc2PSXph)ZOdee`b%Kb?p63%zK>ZM|s=BGHG4pq`wE`Z^ZI1uw@THsa%=oSuOm`l#P z<-dmDtM*P2dpEXkcy&Yp4X@~7m*P~B_If(w|9zRrdpw4>U&U1v;zY=jT_s^W_{r3J zC95n?;$1)a@`0=PL1i}nb_XW79s7L9CTo^gCq}Q0;prE~DnKQnxu4Dz-zGPmKhE@o z`s#rE3m3^@LZ^y&(CoMiP-*3nxP2yj*L*eRflbcQhT)wxWwW!3#-Tk4#+&2;+_Ou8 zN;C00d-|G1E37@tja1*(oVTUu9qxjkX1z-}=x>$hIx`*+gSeomrTR>e&26?{V{a&E z&b)9~IxjILVrFjYz8t1VuyW_4j>4qDU~p8hVl$pA*35r5s+}~F^x8+((=19b zi?wh0E-8KzDTsv>D4|Vc#$<}ngw_gYhPio^tA}&YjLmG75e+q?P-37skEmCwxg=Jwe&UasCWV+ptLu?# zWQFO&w2MY<&UWHv)a*Rc;6iY;mF`k4;)6};*MuA@n#~uQ`-XyZwZZ%j zE)FqGf`nyBbqk|cqE)-V=+GKJq4yJLY-4Yq!6P&y5&2VzAB;N`$wycw|+Aj z>OzhN&%K&cCGt$Gh`MBLLQVx0T?>&@sh@AwBuI)5bvo%?{~L;!AY(9$fi!cBu!w|N zW!L&*hBW(=5yF%pMi(u#a$d`jJIy*rUiR_}>_3T&6!N9gj(iUl8hvLJV{&?m_Evm&6&9IFg4jALw`-L=AgjuT;YgXfV+F zZp^MfU1wK1US{VEcL5+?Q9U3J?M=VBK(F_E^1g;0+f%bya$n>zZ_9I|5~7I2>)1x>lH*2^+1DbgYuU5E(@IBoz4U^i1-8N9w z0|czkn1H50_+);*O@LzQQv-nsRM(FyT22O~@1jTh8xivC{>@$a6ods-`g+fiPP_dO z4r@M)R~?onZr0W3%hU5FQncHM#Rd@1j?GRx1gpVAz`^}%#~&}QpAu`Q;?2Px0pEDT zY3JN~Jxb(APC1@jpz{n1Dy-+K_b>}B;*HzFrq{GKd6uWj;*JVLsy=5sB1Jwg4KXQ| z!!{#&Lp(JRTkHJ@KA_45BLn}>n2a?jUDZzr5D+pTu{#G~OBGEM`;GfiZ0 zgNQT;8e@w_y+$n6JOn+cee)*61aY;4$C*XZyvQkq980C@Pw z+a)b+(%nD3(X{*YqR;s)@cGFcSoA6_0fNS2Fa!aEC7PbhL~e?EGkGhV!bEDUApxIe zrZ?)#@+Sd+G1eV}jA^DfpzhrLQ$#_jHq1$X5DHa=g`}O>6rYpaZ#~k$k4zZOSb2Ri zx`+so!B~1U0F)@aF!bNZ*Cbm<4s)|hMjB3VnVYJo0_!X41TFkmjQSq@mio?7Q_3Pq z+ewRf2$k{5r0poi%<715?5Ss}dt;&+eOEferp^Fh@2sn}wQ=z3k$cKz60v96sx!l# zZEOFTOEjOZ79K3};+*~}E|r6=DSeiw)d6#iKgywm@f7l>=iR^X$#-c-I!)xuKv{!o zeCD8GaeCXnlqrAuka~8jK8B#%()@$cWO<3H189)0il-&gAy%%BIur-F%OKs3`GIpy zrwaiH*9r-oNrTVzTqKMq^W>y-urJB&ma5uN9V(OVjdn@t<1eadH$ioC;b_P_p^bCM zFo$U?=aKEyR%(gLF#OYsX9y@HcdwaHilKLgwf_5L3Gq1c&?!Gp?;ERUQ4^JO5-c5Q zqdAq#ly3wQ0;_OHFgd8f<%+SiAo~GWhere8RGFY3lpYTwOhyd-D>vc|-DuJ4OrsU^ za-uIRa`fgW%6hd>Fb_mSjtkI5=Td>P(QVK*p6B>O()#sB_Md*uyoZ@uUbI} zlNCkz!Mw27QKleOsH^CH!HO8$L;jNtEuNDXrD_DWlVH@Sg?p@i3NJR|K zJy{6DecSV5pGDSi^*@EZu)PCD1bQ{NY|(1)phvn8yZb@GH+O-hmlzRtd%uEz-jWW@ z)6y_K3e*9t?_0GHjxViqBy`ARQ}q|E8WkyWb5=(0p3XHK7yd;4UXjqO4hl4-pO#PD z6>D_1DjUtZ;86#v^O3wj4(6aJQVDbFmUJ_7V{(J`5$X-k8WBX@ zAoWeqVR$eUL8RfAfX3iOWa5*T;AHPm*YN^_tsA1>`JQOKdh0jeU_sfAY=5B#5TpPO zB*~;IlCz%WupLnfCRFNf_W1zc>2K(*qAwbs!mrynnnBYbE23^-F)6p9mu73!j_H6^ zf;Xn*ssG$c`8VTZt!UgkKr~r6@VP5r^|aWPQFtc;@v$H;fGdVH4qm)Y<%821nLMgT zWMP*S0cE=wONgG!nODps-BfUp1RreR@e^>X~d$0A|k5^?&`{wy?r=C8@b(;E>5x62ERAK_f>IHF~6{N3u3-{1hAfPeMj-yi%=~?5it}vvQ)|_maJk0 zN+6#ZL8yeH6E+Fj9cjSLDS4buz@g&ti-F*gF@SkMpFu|;IL1siyE){~ewpmwxGM+! z{5PrL<{mS_$7>Az|4CYc36vzJe+jwO-~WIL|9+8p30~ezfDBm)VWGTP5G^Z+#9TF$ z`Q>H-7M4UQicqyI(l*&+Uhcm{JCuhDT{*WG5NQs-r}eB|OOU-hMpSU9G}%V)|7`)P}J zliGR2+yJbf-<~rW5VvRb4gpdF%30fskm*1@*wkbJ;QZ7wHcj(>IIfqa4$IeOy8>Hi z82mXlnyt}v4bGPIo3L5^HcZ2e3=lpDv|YZ)1fjzjx_AVb6ck9Rrgz8Nzel}`Ou3YI z_Rl8MZL_URRt|h!Mzsmqd1+b9PfCfB7=(76nuR9ohQz;%UaNdzWnHrT?M~1!i9`~5 zn~rP)tgB=q>11UorK;*VJ&-}FENp089R;7nogAfy?b>=wOre>TTqnPk+qc2ho#rOc zvfG@;r}UfXQMnJg`z~zfVeS;PHWa38f)5?i_A#|5m}>xGoesPHy5g|Q{gnkSmpjp` z34>f=2#b%IDm*5@KaKIY*;!?{-3v=&p@EVOh&j`M&z*7*s$O?_Tx$zH(5CiTp+rbw z@iE#}{#SEg+F8A3629=4%jLh^nx)u3Fl9~eQa7^yjYmU0=Xtcu3$f~GhasL4i?>+R z(+dNv?aHN{fIoB6O>E^eI?>Nkzhxh(l-(`Mbo80qD;_Nbefe)-Q&4y+Fzu&ibD)#} zjQN}1(od?TLt@c3oF`NBDHBm2s!#KT1RrDTjTG0~qdAx#=WSC#?}7pXPchoRk@77q zU_!g|7$v^#GD*C{ziAiRg^ZaRAd@jS1qhfMfk_K&2wo+`o?>l7q@LBozhG%!aTz}a zB>+fxr{o87Aq)5IvQj!^<{+E`&gWAAvh5a#IdqI0+qNIpXw&YHd+0|Qdv6D1sri2N(qQ0@L6}-Gu2C3h{SrTE!<7l5d&`PJGqoUhiykNGn(CrRhol{9PDE z=;B3t2PFr&GVR$gx6U6%B48RbC914_;l}6+e!}*}GCQK@$2d^@A&ki4$Q1-|4GhNR z0J~!2Kv9NqElD91PP74(2x-cbP@Tw8bIHHR`GHX7bZQ$_vDq*RGw(&&W zvqTC?3Jkt=_6l15S5tCccV70LZFP_EZup&0qMuz87YI^Y>!e4H?dUZrT4(I+QkBA#3>S8s4b&n>kn9GW3H}*|)bmgodd5I#J4vQT6k2h)m2RDc?EsAi zOHxG_Swnir86_`c#!8{wF3p*)7EXZT!%ESCcJZxljK97s&KatWVwF_?AvbQ;=Ea}= zYkFxw@+7-X&0?c@8{EK{O1p=;53N)5qw~k#QQ^w`&ug0s5)f={j-$=a8%?=10Ys3u z>a%s2IiMbmBou(nHu^ETaz4^w`GImUbk^HN>Yjx_jT?PUuwpRs?_A7C&lbRO+I@IY zYV?(jsVX>5%^t@H<0Ez1C}R*NZUat^VYD$GG#46YvV-mjY_#d7$*J|sr_xlt^!$NF z9~Ex$mTcC6v#CeK0oygh;07!vocV?a*pagUD+lPZN+a2bBnf8yPYpgs^mX=ZGvjQW zWUzviB7TWodV>CvNm3*l8#N$&k&Sn0Q-wB!L_1a`^KfDp18!=1YE#3E+BcG%k z{Up3`5eKyjA)NgJX6rovlFji3uDfX}#@!(7oL)v*tzvABht=ZwU(9U7cdF$QXAAzR zpCVA$PE5%=W*b|!;AwH|VJoXRyosZzTv16xr(qzisbl_88;319^cP@`dsEz=HDY9mmWUEmR+sAha7B_^I9;qE4)YyY%M3ctUUGp! zz!=@|&xprgF}zW5xn0`yf$|1yyl+xk8E0B$@mFq~cW^XGUDwm0$(LC?f_sFyBU%h$ zR~K>)Sl2>zz9Hks6nZO8hc--5i5_V?g^`Fe91A6s z0vEB$iqC!^8c!HpA8mfVf5rUez_DUrkOU_jQa7`Jw)y98Zb1}+Es?@1wyG7t5@ahC zqN|-ekfpdGEZ@XfvqYi$8X`u~!!M?P?)m?qFCn!m?_2+DTz@EmfC&F12>#o}HKhgV z^B)&i_oS0O89N~+0YfMQmGm6||jd#nN(B z_0s%uk%cLU*brTQQ`P>wtoFxq_sX;8&iA{X?K}U;_F6pS{LA}C)8}N{iO&i5$wo@T z>-tTOcU)-l{mgw?Ai(b=FXr?GZ*rKCAEE!hnomuF8?-{-RBiFVxK4EDzt< zfb*LPJno67Q7ZlHrxC}Hu&qyde+gx0uW*ECUI^M95q#G`1YpQ54P#uj`gT@ZhccLo zQ6(Zz45kUsD?|R}7DET{_&yC!YkESLZwaJc zYY#kV=2?N5TZK>DXC*XG@rdkBx5BC66)hB78%2+NRhK?kjnM{SK#Z|5EKe5(7s{Qg zQ}W1}cX5DL2=Iq4=@%Z>v0ak4%68>F8ZfWCMdOO_SEuTcE#5n6PR%_Sls<*PhHAHj zscv2h%J$wK-?jQ0f;$WaPw8#!xL}fdd8hPI?D#;yNNp~jqNm7Fglcp-j`ErcXZ5|E zqeD!;H|q9CVN>vcSR-ZOh>=*N<+qlxKupMFQylM}0zl+u2PFzdEJ~x27K0gSkrge(-n-1Fgo!W_@uylt&DNde!{nSFxa?vd$?ka+B=9$fnmhHUKR~_u;ERx zF6qL85?=Qz;~+D#Kl;;JVAc)qM0;Bbb_Vgq=qaWY%w3B5d}APVt&S-EyH?HJ%ngia z5&(u<9%9Uul;dvP@(^SKA5gJmr{^dVRF#~IZy9a(>s8Cgw5J5Z{^cNG<(rKl=X#b7NhZV)q#9zg@8is9FQk=Zt>;mWvOE(6-p7P`;-NEL zfGYtsz-Z(J1+s)y&lHYS+|}Hu9iWL+7!gtO@G;u+3*IVzcky7;k31%93x$8~E!k6;YSjVl@fMv5T0GfuLMqyIBkY zZX~1wrq9G5lkrI*0}+oUgvpNSJ%M2R&te)4nQUAufjg06?=@2p)4Xy*1p+*nU$fjkxW^QXUFCRDqeEP(CG3lk28 zDT)ctzv8`Y*n56+aA+FP9Vt|j_C*HQ5|iic_r!HFyXh7?u@*z`?+A3z$(ep!%Cgg+ z$>*eG>W`gW97m8M`*VE4gWTn2NF8FkRxaeea(iD``XejWj|{x|bG$AJFf*&0S}Rp6 z#=@;iHaPRI;N6OQ87$u{JAleNTAzyHoV|_I;uDG!texo=%TLXBg#+%vOSx|xtU1cT z*;BRmNU@f=Caf(v2GcK;QE0wA$=?H?o>6?#?iMcWTe))$E)aaICh5ei#6&Mp&Q?C? zJ5ie?xpgcU@7dw^hsXK|9pZc5tlcp^xp!Zme}>_OxF12HtR98m;{e0(cLNeXV>pWx z%w=;qX=Slc%*-yw$*uYrp*^K{1)r&6H}l?DU%`BVQD6g0^YpCWNcgj_6Q8J2JygLX zZ;(+rhb1iESokWhmQgu-HP)XrJLxQG5+PY-xyJXTL60|c#^1RTi*v7`mX9kC=OHL^ z7Chr2tbF}5jB)+OSOESRhe-55w-bO+x!9pxWcsiN1Br1vi6o}Lj{pnP<;trlA60M) zLeMubF4tMjL&kE}H&fjM?BU#=M%(UH#;9-_WA$B2;EAs`3_Ua|4swI~;4Kw+<||yM zV>nJ!VTxHZG%IuJm8h}n{MHo*1DZbi+lOa z+eoPXX=^Lvlj-22LAjehfNZ-aB=0eGE(E+UXZk!qqQ3myakNLytGEZh8xQt66gm^@ zr#RN%hdz2JK%tC3n+T&+@DiHi{tEZqFI!N0Et00xfTPPeR09ucmBZz_c$4+L?KI@%w2@16k#7@d!{9OJc`f_t z?|B#p8ato|WyLO1gUD}UuXDwL8#9|ZEW;j|CwMz6_%VViS|>vv`%vWWiwHlB?+2z+ zf=#>32lWs)X)@0~oyj)So+9ssvgr=?IrRQE|Gk#}aXL)^yr>2oJo}dIRC7)oIyXyf z11euic135!)jp@j6(pVqq&t}$XmNUFx{>!5t6I6K^pmRVZKI4dGg0fEDiEF&PkGv= zp8?QRL^K%IvhEBJ4?6G!5{hr)tyFfBYWqOv(mcy7fg6KY3*?=QY&Hn)9nT_&E1^0P z3skPhAFHYUAVP?Pr4{rjVPP-Lt1t)WfOBN8wzVpiFVIgML6(1^f=bRIx1q)Ac&S3A zv^NIpT>DXkB8fg9UlKE|23d-jXBqTmw6fF9R>E)x;9^=uUk8ouYB9M{R_9Ny#~(Kt zhbu9U)4Wwse93aVilNF%*#6nH@%Ub^iHC;p5PPQdFPWhDz@OC>+E35}GDwIa09nrJ z$stFB?R?s#HW}}_Ibahn5fSDl`ZC~adOR=ER=GosiXS?N^_uyJKHOQ7!Y_!WrP7kB z2U#!9Z=mBh(VXE{X0IyT9{=**#Tg1!9s#4SAm05Ps|)S%M;Jv+^}xGVJ7H6I-0`d| z=7qemuk-Lb+I{#6)v?~4LDJ3!;EH}b^F9Kd?UtYZ?xr7WMqx4ns3YWh$OB((V^mho zr$kl7N%hHaoLeSZ(TR^1-WRyww|+jegAL|ZjAl@f9Q^Fx`x?&$5^NgtfH)pbCT0pXTKBu$fCu|%rz|8*H9ATbl>AOl)>WMT~9Mb`Xr z`>u-_onE5j=hfl}o*<5xo78a$V|ML1$-rdjQ4Sc=teyLklCn}8l6 zP`NE8{O{DM;P>T94%<}jHRP$P@`D=F*_3{{&aTA+_)F;vMIS}IWSm;Vx0=OUqU9uTG~Z`umJ?6Rv_BYd>#AOGqV^6S;xUvl9?;_l zalG>&??QH*TmN3@B!yTC2B0msjaVq&V9;KJ?bRz%-2iUekW~~*s)sG+2wk6LQ~ePOd$GXW=Gg@zLiKhxfp(SexM zP=a=DqyU2=F!9oD*(JUuMrMA&3b@FfkgBuoB}l8=w&=KXXVTNlbK}M;F2NI=+-ojT z1e^k~!&I`Hbx>AlNbSxGh6WRj8`~isL2RFK3R@m6cG{Pq@YY%OtJ8&903vP)yhYUO zy^qDmFFsK}TK86TKRFk|M?XX#+23mTQw6{#@c_bdp(sr$?zkj3LNDRl7sK=5qUJe> ztK5jaAVq^A3-&Y@(ji6b=l+HlLOA*&@-rrni=#)|ca+nclAK(ou9)%1mfm+Ts`=k8 ztiBQ}e-gfQ1Xmns{{`|-#q=cE@2J|-X2eX9@Lbq=LGa+kXJz44iUc2L!663SmOFP`K)+vO(KQr?x@cacRz> zt6>uO;uJDLLr52m^NtKPhl9d|cp;IR82}UeIw5N@l=nUkBo#+G&<}hChDhPjn1VRO znDSTOnk;*|kZe)73cP}5y~S>~;aUG2xlfz^nYPz6tfJn&X2$cgC%bB#B3g0N+>g0R z7xSa+4xqLSxuA?q`;;nW!l#Ny<1l?#bxtF5LiVs9`C@F0e|pXh&kIQmp{yYf1i&(V zQ5x)-{rs#JPy%Y(mBNCFuz`S}y8Vz4=i(olbFSDv8;(m7;=-3nuKe;V_laQf1A)Uc z&@3T5WPv=^9l6o_L);J7w`>=iL1H|T!-gkv~_U{2L@E>tj7HO7jj+m1gQxv zqfbw~$Xz`^NMQ912^N2FER=&Y-go}wP-oRu&ayf}^gFQY-r!NT6xQ%u$l8;%J2{#v z7gOj>Q4!vZSaL<+yCK)#8IegB!4abDLn`o86+$kr<{{62t@oAs-Fj0&7*H~|V{i;= z!iLnl(-$B-TL;zwwRCcGhR<*zl>Y{1)-P@+dG?N0rvA7mLG&Z>1Bw?EOigKzbBH@1 zg(%R2!iazu^1`7^07QZ!?kC#Q%=-pe8OWGnQWZI_1u5G&VM^0>S;{5$5GO8Q&kMm? z)7DBSXx4Nj_NJs`a$C9XWWFGo*M$~Q>_N%BKZSt^_CYzJo3^l_&CEi;7V zNpRd*U8t? z>griftx`(7Z#Yt*rZ)%kG^hj2Yl}}kSN*Q-SC^G(Zp*4}kYE`c4xo4fzBnC-I$bZs z#ooXV=h*{U@exgGWWxm*Nv2)1ePDT%MdF)&zW*V1^GDUH5DV;dr%+67!R+0Aji#vp z^#guVr`4SEi`?wmDCs-mdxOn}wws8kNW?OD6RDMAqx~ zFafco$iue4rfilb&=hJzf`f(}vVdR#7RnVR>liSLUrTd!OBUcQZMM?3zermC=%=C@ zGD8k$+y1R};Ty4BVAXYj|ExUp{xNk)){#LElgmgJ6+uEv-w~S_I3~x$8b++2xkq$r-B&mtowX4?+^9k~;-u6R!m>eJ!^BFPZPyYk zJD_wh*qT$DtOjp=RivSeL%mdnI|tMMZwh`TS>jB*F4;)Z+RWiHqV#sVlrn2a12^bAN`nVqE4e5tmObPpFgJN3F8B~q}h%`*;?NrQKX+VuUhztmENWNb; zfu($fscAWAfWjLN44ApQ_1yjd=`P)seytCv*{wk8F5bQ1DA~n+Ee;e%@qvz^dSxRc z@e%KHzvJ~6P?l=K1=|^cA1&npl5ppx{>%%+r+j5}{sQGsx@89AAE8Id9HB@3go>g3 zS6cfO?6y??xJw|It+NU{r)J%XBeq-`lbkw!>hl63}OtX-Ilgnt@hlBC>$2=o*CqjuW!2br+i>Cj8T<7sWyf6c6ezeG+vdpz>AR z9g70jQ>)6NwL3s)O}TB?@HngZ--tFzDv;6}q)ltOxMD4~ipZMgsWRJPHJT-D3mKJ) zujDqy`!X?vqmOJMb&x4no~1wBi?Z!zeoe8aysIh5d%H~&bGJN}&=Mb5bf}mE{(uw= z8)>D}l_$BVlA8H~p>{AvOlK-@Oj@Z=7B;6Qbd5b!8*SgW(adP4Pk@-b;7D_0=^pV- zHAB;f>K>*Ee_LAB_nu#9J``(O(*+NHGI2pjQtDv_o`4<23PHjRx8*4?p(iQwyXs5& zs9)8&wnq^U!3UZa*%x$bSLCn)a*v!hElH8+G$MMlItT8vnvFH}3tuzv|F&e|(qR^Df<$WXIJ5T&yK$(Dfw? z4eCoxOi)`Gf?}}y5xUR$v(pl`{Ow2sIM?C{$itM?;IR~{ z;J=&+Lpo?>64!D@MPxB^WJNSpyusxQ84CF_uN@4j-AQnhwcp-%q_K6C&!L{hg7`hX ztL2YqvtM%o%1bX?)rUlisDuO;pOMYVDZ01Ik^GeHVMavG87nG`Hld0CwXheyDHv4J zdTizmPt-PkMepeYHgb+(gAZe5;t zzv56+=373zS6sQU%jSw`^8Iuj!dR9S2>nY$sq|Q|ePeWCN0a9WO%%9;CS*^(*Al|e z6r->qZy_~1&EU4nv3KDnv?{q7xNtgLAo7Iawt(D}ItpV@z(p~pWsbul-ke)k0!^I~) zXk1Cb?TQWpiV(Wo-9q#8=H-|Vp;yTXN%B4{C#%{8b^Z{OJnHv?Kr%w!mCUhG1un%8}+tVyZy1Z4hBFtWgLICF(*e?WkFp@B@e z2U?ufgYN9HTfg%E{Qn?#{+Rraklz1jW0?PFW3>NQmFPSp1CkPT|HG*X-mz@aZQl~A z+IdwilF)|IhDt{m!jKFl6Q#FG-`%Q(F*URtv#a*5uRt+YORyk{YsyK7DUq9RoYrUI;WL6tuN@L# zSl>Rg<2+#r(8P|W&Ylg@y>7*ow}IL@fz^tGRBP>AtF^M7v1GGI!mBo7&}aQ(bKiR6 zHN-D%r0jG}`RKmIT)sD73bLB8*6@eC_&X949cUhuxJeLoZgEkCOU0z>n!fLP;-Kwe z|JGcg(`0kK+;%d`(^-n|CVSyG(oAWnF_m-eka>OwD3M#Rk0H!_IwxoM@|I)WR>qQd*-Y$0=8`WcpM{wuwCTG z{@;EQE~9b%9TXs-P+A}$g8zO`VF6W|kUl!9OZ?;wonw!#(izRd=-?LXm9j6UO5v92 z^NB39vgixtxAC1MzZwGB?bidTtVF>C71vA@5r(LUlyW7d{{#?lhqnIxLzGXHL7gkm z^V4Q;#w4j{d2wT%@Uh)7q51O<^=^;n`>z=hYQfioT&T;Vm~LcACdsUsP6-II5Sqs! z&r$xXGQ7Z&eNFiN)+Gpa4};0Am+lZNzl&VNNjb{gMk1m9Kx1iUt%Fqa+8LKSradd5)=H;>Ds?sYu2;p3^Dd)H48lez#Asbcgh&Lp8-@NmOu3umbAQ zHcUQo8&2SxDIYft57w%dO%3>~Gg1MfLr+{G`mSucpH3iq91K*GbYh9c4sHoqv(QQ@ zG0_W`by2iNFm)zmUO;0DcT0A*R(XdO)dK!H0Bjc*Z7ZlXwo&vZ5u1GwWA(`xH50qS z{MQ8wTTpAF3abUovP(gK;tH7Us6aS9bT~_JeNq=o{I3sM4y`PWgf^gv9Ix<-hqf2e zldLEVv@K%>vKI9aWg{r_7&m`@8t7%TK^>G!{Xkdn9I3+XZbW=e`xDY~df_!eF#Mq; z;L~!O*1lxoa>l}^dpHMHNeWHllT-7_bM7^Opo_NqH2&DOTj7A-!Ur*7J62*%j7Dw6 zv^lGa54l(somq*Jc?f`_rP|Z(=6q1htj5Y>zrI@Q^iUl9Bx`O{y44t5R(eNc9HNNJ zjFh!B)r>0Zvdj=wEe+Kb*fkVdGzFwTc^E^=%`F>YX=OZrUU}UZ`JAfRwZPQgAzRU| z2y0}T%iG3`D2;U`!@NnxRyiIConkj|T^<};8TAAb@6E=5J%`X2aRc`*s!M!vxy;_ zDRXUGX=T8`g#V7W6|;#j6g}czIBV+2zuja}pN_5}^sSv@g5A(Tm0gpW&|zIY=h>T$ zwlH63y0l5J=hoeiM2eBLsfSxAx~5lsmT{}r`wIXY1e6Y zYDpc8+$QQ1hxG%r8CH{{`#L>Aax}OSrP8z?83vEQ4FDCaTXTf9G^^e5V+*Fwx%^N& zZWO^_j-X>MAp+wz(%On0Enc{@kQUY9Zt0ZRjXby@fA?`#f<@fd!O0gwr5l@``#*wh z8BhZ{iGddq5tXV`;zJ8EYzwX~UBz1Z(}q(lK24*ZEk03;CAnp&*XJ_q`=Z83VqMo6 zs>G?T!T~8Ta^}Y;WT6b4_%IcdDjBF(Za$Z`F$Rm?ub7A)N3EywNTi?49>P%oAw zsqI#q(|4l?-~8p=sgXRh8QC=?B`2sC=2C?rxv69^I0eNf3HNIG(3@B}e=1O6A!{4w zw*U-Y--&Z@vFGu3t#JaMXQ_#eOo}CUMGTfKO|0p?Y~Z@QMW!3MYD`Ne5VuepQ>m0^ z%T9IhWL9O3>v{yF-5|EQ3L{LwmyjD<_kq71R!;w|8_VKghl+}Ek81FNf1M1peLBv} zg^4U$8Z~-i%6Bs`fg>+Ea}cA`mDn^ns{+hiq>xam6oAx^dAeL6pS7k<+~bkvsfQZ( zwXIchdKMC-J#y`2E#8L@gwgGJ{yWcK=eA?l+nq8aEatOJZXgulxfI>i0wkde_r zVUN<&oKL1r&t0~t8a~c{%J$ncc4`}xTAjw0G&O}H0%z}y$pMM_ zA*PnJd`wluW)(D5q(T+v8Wy(LI-M$LIesV+F|5kyk2DyE^)juCILt`*1CA=ei=AyF zm2_&qGnU}u!j>jGIhkZj$WJA+zJ+Q|UFf$;7H-Z&v+xQ-T;6GX;{-O)8GZy7$kRX;^%h5C7dQc&?=jd64n4y!Lf@>k4QUP$pH4c<| z*b=8hV2A0+G}Q8vlm4EDTrqz>ug!vSHz0?|;9Xrd?%MNg^Xzg~D1w1%>3*Vdx#-M+ zKlLI1ccbqNKh6z6#qDK*M3``V_ekbGZ280t|90(8xiQxAHi;T?4JH+FZV%B_^S_UI z^~bEqG%wQL>u`1!T9v=~%$NX?n>p zP*t1;u_4x}9Ugi+UAo0uLXszkuEt~Ij>(3_%(L-y$-up7OIkO34J#a@w|6AaNcC%u zGtCi$g$7lHBRya}Sp%NVK84h9AiyGx2f~HYE$wIzjv6z>*M>R^2_Vlgw27lglc`S- zqPQd4Pg)i)!j>0Y#Fa!?!hJH)ER%()s0Wa2s5kqj=qGbBa&!gB5u2To3ehZvoW_9$BA*}5p##PSy0xYp zR8vKrEi!hfS#J?H4d557^dDE|`bo@LelrJomApNk!ugNQPr7dvrO3+_}$(b15lb!#a4#p}5jFh`uS9{IPXP+*a@vnrg< zVf7Y48R^|+oxgu{YGIq!6{%13B;TVZC2N+SCW{^X0lH`ggTS-Z)qfp8stM7>r9|{zn^79JftTM zwX8m8ZOW<@#n;AI|F%dD6~R_x+#l#HS-Ml&K+D%<;gppmFGpqObo?pj>2$MO_~ph# z9fBsi1Q_YC&Gh@iwK7*AFGU<|j%lo|cx_wv&0f4F?+AB&JmImY@}jG}@Fy47pgA#b zsnt)DS%&~F@WzFzH`}PSx$D)S{J@HG0U7ddS{9grdD+P7qZm75=67z!Vf#9Dy~{Ys zrHjK+KR1$YLvXwn%)C_DNS7L(Z>GHCvyt@c1f=rH4bvx1)GSRWEqe$!1Y*rCmfJ)@ zdt+S*EJ})Z13)7j5sIfJNRUevjiXr-q;%=@R9FfJ=ZWmP3&8Eu8neU}#lHNf1cuD? z$Z|4Ma)Y;rTY=hdg`O7GWE_DvTG6LT-PSuuxUcsmdlEDm19ay zZWe1AhjNb2&et)Tp1)YBGFZvniK?*Uf9iSePg?IGOyDi!=kCcCT`|>|i3UaL)#i4i z%82qfejn$#2o=4_VSAOaHBU%XNo$$;0wyB|xD8mT44+riX7=pHj}T$iob5;+m<`$ zK(6wci0&0b`6P3QiscU~{xrNjd=Yi?0l=X2 zc+8Vulnvj2z_i1~qDn$OBzMFMCa9m|)b!e7K95vCEGG4a_M|Ohi22t&H8f?BE%{6| z1eh%dvqU0Jks=ehSp3C9txosopIvxmXU; z$8T~OJuva_f;)XxQ0~Tzg$8530ds^uK(aQJ@NdH1!>fbLl=Dtj&ahKm ziDU~dZjVbY>eeHYb_TICLO!mtVtq?9{@1@{Ea@-6ylss=h47@MnlUoq?afsXtC%L*4)M(|)-K z4W4LT(m#iEzHACOk1XWHwygH-c7C}Si9lQWy2QVDVWHkfWo(Euw*>pltJXjH@ZWGg z7S?9CQU~HdA7p|6%jao50UjwDZ1ap7xpSB0CQrd!rh_8f=k6XI(`9pVrMTcP+y+u} z`pUOdVotC)njx#Vpjw*y>$d)A(*wO&3S0?2>gInVyY?1}vP5*tviEZk&VIsO`BQe> zed}Uq71#pJy)aN!ckrB~Pn?lqU>Sl@?UZCBUFHUf`ne_Q(AM<{=Y> zbI*|9CiVpNm6w3ir+Nnl^ObJ4RF?P4Y_{c(KFaphB2|bSbwl#X_k^>5v6TD)QWSjf zEBAv)IEKq0`SoC@3SbJQH&Y~Bm(F`JoqctJ`jve2#{^dQS5-~X)1Un`pgmG4eNw|| zGhC?R%5AvbX0Ti*vMtG9wQFUypSl=N^II4zNv5Wkc#&aO!|^A&P}VNDt?rg&mEuG% zzG)o3D3OD_uBz}#ww|W^Q!ySDBn;>}vJZdeg0v&hETyW#j{#`(TxjequjT35e^eEj z_px(HX$kfDX?a`OL%75j>;?0&Em0I05%ExECIk4%Rt$} zlV`s?5(<@57P_PH6v7m@NhF{bH)2#7c8ZnAf6zf6w!R2>6X$LbvXR=Xe_LQ0(Grc7 z9#d&WkKB`4bOCaubUbE^$U!4~*=He<{2sDqx!1_BDY{~|iPJ0PXeT&bBsar* ziWo#a>I_jPJrO)07=@SOJ>23J zNYJJV=UbF-x@fihqt^U-Q{0XwKS$-!Rf@+Ka-q)tvQQ@XtoQdaYI4B=wF)n{iT0O^ zR)i1x^Fq6yRX9OLO-L1cZ}2xCR~IRgM=V)LV}6ueU?9kp?_wvarlL$|nI*ie^Qc`P z^>fJA;~s#|>R$!FpU2<^Vf&}~@4k3A-?}(HGRFR#xcTEhIDMWhqtY^1qb4uC=(&SI zYZO_YFUGPT=(9J>$I(IR9SOlJa`>!r?WsKDizVp9%$dzQnA?uN;nzd%BX(hj`EJsQ z;FV_Jxjkb346r&q&}`2y6@3-FD{!5bc^n0AsUUz3h@Gkb~I|jci zR@LGuXVkKl1o0!?lt2zpUj~1e$-J4&8W$H#{fsa!IdOI{eOMW!=&+wvSJe7@X;)qT z=p~>aI>VXUK|%J7*u+oMl7CDyK2AGiyGL12ivSQjhR}`PE=5#PrVz)1A2aQm1h154 zKJ+Mc=}MkRy{Rgf1Hb!}p#D5nF?Q1S<^xZ31tz2K@lacN4~6lCdZ}0B_=pv7<%%M2 z-wSC!uj4~s+~U-y7MupiO)=OeN~EZV_XN~d<4|>0iqwpybq0SI%FrF-%4WZmN)+%v zJh}@M3Ve4amH}T6xFaaj;pNh;eHF;APRt-3;^j7&LNgxG4l8ieg*iG&7lDcz@ssB1 z4btOpjgliFp8$Quz{Lj5N;U<4ewn-|bLA&hyc>#3T$E%XVe@)hxRB&-3;F-`h6CtA zMt3QRp^QtHB1o=4XSyqZIFbI5@I*8EyAyR{r()IN+-61W1UCm@e8+P_byFsZK%&bf zz06Zym43P`V8zL`#VLea&YQE4+D#j72pO5B!xPUrylcdsfxhhwjYuqG2RR{KgemES z`k)bfyGt)GXwHlF!L6R_r3HWalm+a4?q|-?7x$PqH%6S{L!AUs@O?}E1Z31}&S<{b zSE8&e#)Hox>SJ|1;n6Np!0F(FJbWs|3Yv=)v_P^7?QE#L;!`Pz3fV=h!nplYolM$; zD|6%ww3s)dlq)L08*(=pw$vTL2orHZzCfWWet$(kFeJ5*$7`bw8u&a+M)Q+h^~#eIi}N zfUn_**m`$CR)8WS4noaqs7X&8P(~G@{=_SnfEwtQ$UzeLkhq#>y~^HV4j4201asB( zW=;L(R$M(K3>+b3(~a!AZiF|6p%QR0=9BLQ7L2{_^EX6?gAq&;J%rm?;z8~(+WT)) zLW9w8#ecj9h%cB8zKF*HfYM25rEZd_S074nC*@3^gr8@wkvB1{EE+1lOPt zBzRLqEcy$cvQQ5IPb6FAZFpZYX26glr= zOaCihHpBsvU+;1Pk=^Y>8N>S>3;ax|huWYsArJmAkRL{W%<3B$p5CDVzu5&jksj2Z zDVsjz-V%Zy^q%W8KzJ|%#4cn0!O2pGl~#CaExFtpq%acjZd*z?eMbo$POsX+0M%I7E+Wpc$&y2dvw3-cQNAXngI}d&C*Rc*3CDDFOou)rJ@_LYwj+m&WO=I$21z#pUV^= z>A{vz6aTqLo(=D?bLK1g!MoX{L8i340n^t;f&5Jp-1_O9@fsloIPDgCZ|O(Hzmmp ztFl}SB)@6Co?d#%w8TLDI~$aqfgklW{>{0BCy9$SD!oUPmnbtAmpmASu;KeutLHLN@k%TDr@yhPWy_4K4jHp>oR zZl82Z$X2S+SuDvqe&oO;MT)LERa&Jx_h=o>^^?B^d(6yfm%5RB+`0ce)JhHK*LRa5 zG#KoLcAAYFMSr!Z<6dc*cjI!LNC_W24+Rq3Gi!dMiq6g`y)${}u&~`jJDdLjtBCE) z>4ls!fGf*_LCz`NE;A^itfzP>9H1u zc2sxpPUmME)TOmhhpHFd1F4I!l;hI4oeijq?5Jl?w)En;Gn?wFW9qRwU1^2p(rTU~ z{WFT5*p!9}w-(5KwDW(;x(=u&nyyW6g7iR;5(vF_kS0}1M5=TY=~Y02bOT6HDT0fD zf&r-(nl$NMq$ov@D$5Ca+_~3vtooBljYzUf ze0SWW0O7p1rSMbvnQD3kIpj@-bPeV#R8YTIP27I}y;^TPytNnR=V&Aw{knr0sc#|e zHLHgEjdJ`3^_=T691(M~*-lP%MC|CQ2g&|Fxr5i`)AZ!W4<%VMF)#~i4Pq5koJrQ2 zv_hyvsGV5slo|P;>K!IK){prlRH9yR^vjkTYOo7>LsaSUlsv9fdcS0-EM`m#QBn~y zeU--d)XR4G%Qq*U~gySg+GB%3`5&c1P!lLZ*yqP z!shAWLuw;>S@DSxZd!5|uhyimT>RD~%gvi@QiE2~;&|65e8H*nXa6_x{S4Ui_6~V9 zG6LTC2p@aGPT+_a4dMP|Hqb0fjoO-QY2e=(_Bueq8B&f)5RED_SM;WY@e~*OVrp+L zZxONZhueu6eKOi6X{gUo$oZ~Shc9k9R&3U*K zcn2RJk=Tkf(Nyy0V9Of+$^LLSVI7yLx6fKBPFN=-G=Y**N}Ke*dfa`~HK2Zz{02=#A}b zVjWETl}RfX<*K4d9LRo@HyRX8_?F)aZdIY}v?FC8>1_Q?IrhD>%@mh%mvpRCEJ!=w zOuAFkXA5~LjAt15eT2A1E`{pbHE7Rd#!t#nhu$%`O5OFs+oIl=sNSFM5%p!9ptq}Z zfI1t459Xqk1^XcbHFZkrd24wExn3hi$eJ;SYQ$qtcDiF6p_K<6ULgyXi$5PS!A!#Pgeo z@6)XFZ?@AFN1%K6#IujK%=hyw!I^(hhOiWK2pB<1A9UgS|Aq2~Jz^ksm>y+qG@wiu z)vM|cY6w zWKP_2s=Me^wph2WG&-==J9(oK*M@LMl`4t{Pl2MT3>MoGg_DLmlGVK^Y40)unIilo zFS+_TffgpB92ROeAUHnwS_ow+fyB$s^_ef@Sa~hP_9P~afaCeL^?O6owy)@@tl@dG zp)I-b-K%eVNjT--@jG>OjRod@;e-YE3tREj+z!&jv>{p#kuRmLWjoNqH1Ljubca_3 z!N5tHa5DuzL5R`HMlHf^dfZc{B(`$Vr>!paZd>YP31_Qb4_h@X**P4H-y&o&V(Ps zJO59eNKcX8v3lj9)(IA;K>0k;TiZgB`rFg4m)YNJHWyYcTz?;O-_{}HDSJFcK4V3o zKm#(dLFc7y`9~vPy=?q6)ReBn)9Ya*fT)H|vV?ZvUcAFA-Sp}Qn}O!vI#bAaG@q#M z&wXhg4<`}3Pn$ZY6;tCN5Jwc2_=e?IP=&gdvQ-2fg`Khd`~HPEMEy-(DeCr{su3jW z8d{$h-X9-UurGh|qI|u${EEVFqn3PDQ>Y2Yx9tl~uHWsJZf%kLItA2%Lt7Iwk!ms7 ztfbO4?+lv;f!_oi?}^5z0NW#Jk{-FKf_sKYvCmP{>+#(}vB1=aR2()mwdyg6T)vkd zQ;1UuM-AHRf}-Oww(A#K81P^l+M zur11ataS^{Lx0{%|K?(UXOi1y) zzR@IOOPYVM#Clz%lPATrhk{a5#=Gc^^Q@DdbHo96WF!1AdsL`zEG1bV{!w(OOyiA0d8BvM1+UtRP`d3%SMN7V@ zS?OQCg7TOUt^4`<4Y)ixL9?)~wL_6l6nG#nK&_ahQ-$sc`%(W}<~${5t$YwMu>vQT z)6WGL+1huM2SyKwjtA&6OSwqMJH~=(m>eu-f6OLJ zKkFagHM5BCsqpk`*A36NajCrN_+5WC)g>`KZ5vfyKU%WnW^urYRPS!iIRDnC7%ki< zhIh_gTvNNO+WNA|I^UJ}rJ+#imYS&AkLAgudFDjL6S|e-PY_4!idIbxWNIshmC4_} z6wZJ3ky?o@xLKT1Qm%?X4GWEPt7O^fqJzJGO&jtlBF5W^P>J`HzeJk`x~dEahLk1d zr(ZzviFRZ#K2j%q`LWY{#7WHQ2`NDG8FM`}G=_=H8rf&%x@F^iW-b>;!t7kL zL(t_UQ1dQt>ombJmCHP3)!{$fi=SXzpc#o6^V`Q&w}QUFC!F&IH)v-1k9iRG*8ZVa zMxhfIYu64F9$Q5}jTQ08b{WgbNM0El5&ncK^+mMtWtz|0dt^T%H*E2Oag{4QS0>%O z9K6wM0BLxwAIe-=xWGB85dC$L=$`9r>5}_=^8WWXHT1Q`ns!NFloop!T#VTU$exzz z7LF40RSRmnTy1pgL3})a<03nvF<05zCMold7Rj1pfVh&Twx+qK#hrJFy|iwfP#v8d zRF%=oP5v$i!kC1TITQor=~AIRX+POrr|PfPMZDXp^zxVhnjfaWt{mW@ZW&@Uf*&y5 z?OZrg9g^nWwq?=Ls*=;tnMxEh;!o0=G;tLdnMM z{rRJ`{0*O&CvVyUDXGu;mWUAa+}DDKn@5Ugfv9l-pRh%cX)f832 zgDSF355b&+93l>N-WLb#T!__AxgfOC1cji|9@ct~kJ{fJ7n}IrXEIlhCC) zG}rT!JUV*w2(q+yF0X#S5G@u>+9#^Fvv3`1S zDT>UZ1p2&uO!W;vvhNoZy&Y`fxn01}FZbR2hStgjg%z&yhW@gKq#P zZp~OEOTU>OYTb|#)}vzKp-S;7G#RR;Ch?N#VdR^9>QogkXC|qEghymm9;hu}#uQum zkZX%nXs%2*F=GPeXhG)(BXp{}cFp9j(JeYtVjrQe<8!YnKfgQyZs`?BF%YsJ93W{U zg@>6GY;@>UIj&OG846G!cjZ-cnOR8sdR~o4E|dBTdu--_gXD1_|Ort2jkKJ>{(3OGA)%@5KQDKLpd$_a7|Y2UZxZL{Rd_M~ExmNzT9 zu3{A-d0AUzh@0R$d2)w6s_g0;oFDPjyZ#11S{tjnbIWu*pVc-&l3A4qNpEkH2p4JU{i^slq<%m#PPmDNu@kGwMFiibr7Qt z<)dy5eK&0rBZIZbq|!`tp~ynCeFWe0G-oRL7JjFvJahLtIrvVr5ED z1fOE4YC5o1u(eNt;i7h1oioqyqI;_F9+0D`OUfmNwNh zRk9wgkD_aw!*x{Wycit5PpCJ3pv|nz686xr9 zFW@Swjk&p;{cFmoFbi>utj78_ZxexMYdKP#}Ju7o5L53BLVtp7~GdXdeEjB(op@mh2Wx!DHe%C@{Dn zb6NuhxE2-vVEh+(z*6Sob?|t+=nNj6mrlwEdMGfpbb@02ZUi1N%upcayTX~=c1wFM zjlh@hmS;p&zBUpz_GIMV!=!J!fHnx{04&$ZPlkszl>DFCoUk5CV0xVcC_b&K zTZiL~+kpZ}5U1x11w=N;@Qxh;`3+T2aCZZaH{cA&xyl2gKTmc8o($t(8|N?Nx#Hk> z-gXjkx_#5$-9f|`>EaFs%R7AzPBF5W^LRt9zy{`m;Qu}ze?!fKosWI5Dcw#8z-tWb zjEPv0uKzItJm-Y1-vVqt)1L&smgE%s|AdVHd#%3#mBHZ(I$8hy7jqFodN%we=5$_OH?1pr)xVU}IPI yj0UWUF)R%my@1!S$bUKvYaRfLU@!o{#`z)o2tshY /dev/null && pwd -P ) || exit # Use the maximum available, or set MAX_FD != -1 to use that value. MAX_FD=maximum @@ -133,10 +131,13 @@ location of your Java installation." fi else JAVACMD=java - which java >/dev/null 2>&1 || die "ERROR: JAVA_HOME is not set and no 'java' command could be found in your PATH. + if ! command -v java >/dev/null 2>&1 + then + die "ERROR: JAVA_HOME is not set and no 'java' command could be found in your PATH. Please set the JAVA_HOME variable in your environment to match the location of your Java installation." + fi fi # Increase the maximum file descriptors if we can. @@ -144,7 +145,7 @@ if ! "$cygwin" && ! "$darwin" && ! "$nonstop" ; then case $MAX_FD in #( max*) # In POSIX sh, ulimit -H is undefined. That's why the result is checked to see if it worked. - # shellcheck disable=SC3045 + # shellcheck disable=SC3045 MAX_FD=$( ulimit -H -n ) || warn "Could not query maximum file descriptor limit" esac @@ -152,7 +153,7 @@ if ! "$cygwin" && ! "$darwin" && ! "$nonstop" ; then '' | soft) :;; #( *) # In POSIX sh, ulimit -n is undefined. That's why the result is checked to see if it worked. - # shellcheck disable=SC3045 + # shellcheck disable=SC3045 ulimit -n "$MAX_FD" || warn "Could not set maximum file descriptor limit to $MAX_FD" esac @@ -197,6 +198,10 @@ if "$cygwin" || "$msys" ; then done fi + +# Add default JVM options here. You can also use JAVA_OPTS and GRADLE_OPTS to pass JVM options to this script. +DEFAULT_JVM_OPTS='"-Xmx64m" "-Xms64m"' + # Collect all arguments for the java command; # * $DEFAULT_JVM_OPTS, $JAVA_OPTS, and $GRADLE_OPTS can contain fragments of # shell script including quotes and variable substitutions, so put them in diff --git a/playground/kafka-emulator/build.gradle b/playground/kafka-emulator/build.gradle index 486a232f9b99..2d3f70aa9883 100644 --- a/playground/kafka-emulator/build.gradle +++ b/playground/kafka-emulator/build.gradle @@ -24,11 +24,11 @@ plugins { applyJavaNature(exportJavadoc: false, publish: false) distZip { - archiveName "${baseName}.zip" + archiveFileName = "${archiveBaseName}.zip" } distTar { - archiveName "${baseName}.tar" + archiveFileName = "${archiveBaseName}.tar" } dependencies { diff --git a/sdks/java/container/common.gradle b/sdks/java/container/common.gradle index bf4c122ca91f..cc427494ed6e 100644 --- a/sdks/java/container/common.gradle +++ b/sdks/java/container/common.gradle @@ -63,6 +63,8 @@ task copyDockerfileDependencies(type: Copy) { task copySdkHarnessLauncher(type: Copy) { dependsOn ":sdks:java:container:downloadCloudProfilerAgent" + // if licenses are required, they should be present before this task run. + mustRunAfter ":sdks:java:container:pullLicenses" from configurations.sdkHarnessLauncher into "build/target" diff --git a/sdks/java/io/google-cloud-platform/build.gradle b/sdks/java/io/google-cloud-platform/build.gradle index efc9ff3db9c6..c4a508680186 100644 --- a/sdks/java/io/google-cloud-platform/build.gradle +++ b/sdks/java/io/google-cloud-platform/build.gradle @@ -307,8 +307,8 @@ task spannerCodeCoverageReport(type: JacocoReport, dependsOn: test) { sourceDirectories.setFrom(files(project.sourceSets.main.allSource.srcDirs)) executionData.setFrom(file("${buildDir}/jacoco/test.exec")) reports { - html.enabled true - html.destination file("${buildDir}/reports/jacoco/spanner/") + html.getRequired().set(true) + html.getOutputLocation().set(file("${buildDir}/reports/jacoco/spanner/")) } } From 27a9f92d92f338fc57727afa90485d6bd01c77c6 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Thu, 5 Oct 2023 07:04:14 -0400 Subject: [PATCH 027/435] Do not rewrite project version in BeamModulePlugin --- .../groovy/org/apache/beam/gradle/BeamModulePlugin.groovy | 7 ------- 1 file changed, 7 deletions(-) diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index c7a62237086e..3a332d8d4d08 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -488,13 +488,6 @@ class BeamModulePlugin implements Plugin { project.ext.mavenGroupId = 'org.apache.beam' - // Automatically use the official release version if we are performing a release - // otherwise append '-SNAPSHOT' - project.version = '2.52.0' - if (!isRelease(project)) { - project.version += '-SNAPSHOT' - } - // Default to dash-separated directories for artifact base name, // which will also be the default artifactId for maven publications project.apply plugin: 'base' From 0b938306d135bbaadb3123daf0455f67baee4a9a Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 5 Oct 2023 09:42:10 -0400 Subject: [PATCH 028/435] Bump cloud.google.com/go/spanner from 1.49.0 to 1.50.0 in /sdks (#28838) Bumps [cloud.google.com/go/spanner](https://github.com/googleapis/google-cloud-go) from 1.49.0 to 1.50.0. - [Release notes](https://github.com/googleapis/google-cloud-go/releases) - [Changelog](https://github.com/googleapis/google-cloud-go/blob/main/CHANGES.md) - [Commits](https://github.com/googleapis/google-cloud-go/compare/spanner/v1.49.0...spanner/v1.50.0) --- updated-dependencies: - dependency-name: cloud.google.com/go/spanner dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 2 +- sdks/go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index e17427227eba..89f119a54908 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -28,7 +28,7 @@ require ( cloud.google.com/go/datastore v1.14.0 cloud.google.com/go/profiler v0.3.1 cloud.google.com/go/pubsub v1.33.0 - cloud.google.com/go/spanner v1.49.0 + cloud.google.com/go/spanner v1.50.0 cloud.google.com/go/storage v1.33.0 github.com/aws/aws-sdk-go-v2 v1.21.0 github.com/aws/aws-sdk-go-v2/config v1.18.43 diff --git a/sdks/go.sum b/sdks/go.sum index 71c1c4545c89..582f3d69f88e 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -38,8 +38,8 @@ cloud.google.com/go/pubsub v1.1.0/go.mod h1:EwwdRX2sKPjnvnqCa270oGRyludottCI76h+ cloud.google.com/go/pubsub v1.2.0/go.mod h1:jhfEVHT8odbXTkndysNHCcx0awwzvfOlguIAii9o8iA= cloud.google.com/go/pubsub v1.33.0 h1:6SPCPvWav64tj0sVX/+npCBKhUi/UjJehy9op/V3p2g= cloud.google.com/go/pubsub v1.33.0/go.mod h1:f+w71I33OMyxf9VpMVcZbnG5KSUkCOUHYpFd5U1GdRc= -cloud.google.com/go/spanner v1.49.0 h1:+HY8C4uztU7XyLz3xMi/LCXdetLEOExhvRFJu2NiVXM= -cloud.google.com/go/spanner v1.49.0/go.mod h1:eGj9mQGK8+hkgSVbHNQ06pQ4oS+cyc4tXXd6Dif1KoM= +cloud.google.com/go/spanner v1.50.0 h1:QrJFOpaxCXdXF+GkiruLz642PHxkdj68PbbnLw3O2Zw= +cloud.google.com/go/spanner v1.50.0/go.mod h1:eGj9mQGK8+hkgSVbHNQ06pQ4oS+cyc4tXXd6Dif1KoM= cloud.google.com/go/storage v1.0.0/go.mod h1:IhtSnM/ZTZV8YYJWCY8RULGVqBDmpoyjwiyrjsg+URw= cloud.google.com/go/storage v1.5.0/go.mod h1:tpKbwo567HUNpVclU5sGELwQWBDZ8gh0ZeosJ0Rtdos= cloud.google.com/go/storage v1.6.0/go.mod h1:N7U0C8pVQ/+NIKOBQyamJIeKQKkZ+mxpohlUTyfDhBk= From c89d44df1302464d85feffa5fe54474c8652b6ec Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Thu, 5 Oct 2023 10:12:36 -0400 Subject: [PATCH 029/435] Fix whitespace precommit after gradle 8 (#28845) --- sdks/python/test-suites/tox/py38/build.gradle | 2 ++ 1 file changed, 2 insertions(+) diff --git a/sdks/python/test-suites/tox/py38/build.gradle b/sdks/python/test-suites/tox/py38/build.gradle index 208a1d9d39ca..bc4aa99c79b4 100644 --- a/sdks/python/test-suites/tox/py38/build.gradle +++ b/sdks/python/test-suites/tox/py38/build.gradle @@ -153,6 +153,8 @@ task archiveFilesToLint(type: Zip) { include "**/*.md" include "**/build.gradle" include '**/build.gradle.kts' + exclude '**/build/**' // intermediate build directory + exclude 'website/www/site/themes/docsy/**' // fork to google/docsy exclude "**/node_modules/*" exclude "**/.gogradle/*" } From af491ebe33c8d9544df8a10e7def68cb8549a920 Mon Sep 17 00:00:00 2001 From: Svetak Sundhar Date: Thu, 5 Oct 2023 14:27:12 +0000 Subject: [PATCH 030/435] [Fix] Broken Link in Colab Notebook (#28837) * Create HealthcareUtils file with shared resources * revert * Fix Broken Link --- examples/notebooks/healthcare/beam_nlp.ipynb | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/notebooks/healthcare/beam_nlp.ipynb b/examples/notebooks/healthcare/beam_nlp.ipynb index 5106aaa607d9..4ba4a5e0a739 100644 --- a/examples/notebooks/healthcare/beam_nlp.ipynb +++ b/examples/notebooks/healthcare/beam_nlp.ipynb @@ -146,7 +146,7 @@ { "cell_type": "markdown", "source": [ - "Then, download [this raw CSV file](https://https://github.com/socd06/medical-nlp/blob/master/data/test.csv), and then upload it into Colab. You should be able to view this file (*test.csv*) in the \"Files\" tab in Colab after uploading." + "Then, download [this raw CSV file](https://github.com/socd06/medical-nlp/blob/master/data/test.csv), and then upload it into Colab. You should be able to view this file (*test.csv*) in the \"Files\" tab in Colab after uploading." ], "metadata": { "id": "1IArtEm8QuCR" From 5446776d00f6d3970ff859c38e26343274413c57 Mon Sep 17 00:00:00 2001 From: Anand Inguva <34158215+AnandInguva@users.noreply.github.com> Date: Thu, 5 Oct 2023 11:50:02 -0400 Subject: [PATCH 031/435] Support for custom MetricsFetcher in Perf tooling. (#28671) * Support for custom BigQueryMetricsFetcher * Read GITHUB repo and owner name from environment variables * Add test_name, test_id * Move client to the fetch method * Update skip condition * Run on self hosted runner * Update readme * Update README * Pass test_name to the metrics_fetcher * Fix linting issues * Fix lint * Fix formatting and lint issues * fix lint --- .github/workflows/run_perf_alert_tool.yml | 2 +- .../apache_beam/testing/analyzers/README.md | 54 +++++++-- .../testing/analyzers/github_issues_utils.py | 34 +++--- .../testing/analyzers/perf_analysis.py | 55 +++++---- .../testing/analyzers/perf_analysis_test.py | 41 +++---- .../testing/analyzers/perf_analysis_utils.py | 104 +++++++++++------- .../testing/analyzers/tests_config.yaml | 2 +- .../load_tests/load_test_metrics_utils.py | 11 -- 8 files changed, 183 insertions(+), 120 deletions(-) diff --git a/.github/workflows/run_perf_alert_tool.yml b/.github/workflows/run_perf_alert_tool.yml index 6946011f0617..1bd8d525c2fb 100644 --- a/.github/workflows/run_perf_alert_tool.yml +++ b/.github/workflows/run_perf_alert_tool.yml @@ -30,7 +30,7 @@ on: jobs: python_run_change_point_analysis: name: Run Change Point Analysis. - runs-on: ubuntu-latest + runs-on: [self-hosted, ubuntu-20.04, main] permissions: issues: write steps: diff --git a/sdks/python/apache_beam/testing/analyzers/README.md b/sdks/python/apache_beam/testing/analyzers/README.md index 076f173f9d71..91b21076f88a 100644 --- a/sdks/python/apache_beam/testing/analyzers/README.md +++ b/sdks/python/apache_beam/testing/analyzers/README.md @@ -35,16 +35,13 @@ update already created GitHub issue or ignore performance alert by not creating ## Config file structure -The config file defines the structure to run change point analysis on a given test. To add a test to the config file, +The yaml defines the structure to run change point analysis on a given test. To add a test config to the yaml file, please follow the below structure. -**NOTE**: The Change point analysis only supports reading the metric data from Big Query for now. +**NOTE**: The Change point analysis only supports reading the metric data from `BigQuery` only. ``` -# the test_1 must be a unique id. -test_1: - test_description: Pytorch image classification on 50k images of size 224 x 224 with resnet 152 - test_target: apache_beam.testing.benchmarks.inference.pytorch_image_classification_benchmarks +test_1: # a unique id for each test config. metrics_dataset: beam_run_inference metrics_table: torch_inference_imagenet_results_resnet152 project: apache-beam-testing @@ -55,11 +52,17 @@ test_1: num_runs_in_change_point_window: 30 # optional parameter ``` -**NOTE**: `test_target` is optional. It is used for identifying the test that was causing the regression. +#### Optional Parameters: -**Note**: By default, the tool fetches metrics from BigQuery tables. `metrics_dataset`, `metrics_table`, `project` and `metric_name` should match with the values defined for performance/load tests. -The above example uses this [test configuration](https://github.com/apache/beam/blob/0a91d139dea4276dc46176c4cdcdfce210fc50c4/.test-infra/jenkins/job_InferenceBenchmarkTests_Python.groovy#L30) -to fill up the values required to fetch the data from source. +These are the optional parameters that can be added to the test config in addition to the parameters mentioned above. + +- `test_target`: Identifies the test responsible for the regression. + +- `test_description`: Provides a brief overview of the test's function. + +- `test_name`: Denotes the name of the test as stored in the BigQuery table. + +**Note**: The tool, by default, pulls metrics from BigQuery tables. Ensure that the values for `metrics_dataset`, `metrics_table`, `project`, and `metric_name` align with those defined for performance/load tests. The provided example utilizes this [test configuration](https://github.com/apache/beam/blob/0a91d139dea4276dc46176c4cdcdfce210fc50c4/.test-infra/jenkins/job_InferenceBenchmarkTests_Python.groovy#L30) to populate the necessary values for data retrieval. ### Different ways to avoid false positive change points @@ -76,8 +79,35 @@ setting `num_runs_in_change_point_window=7` will achieve it. ## Register a test for performance alerts -If a new test needs to be registered for the performance alerting tool, please add the required test parameters to the -config file. +If a new test needs to be registered for the performance alerting tool, + +- You can either add it to the config file that is already present. +- You can define your own yaml file and call the [perf_analysis.run()](https://github.com/apache/beam/blob/a46bc12a256dcaa3ae2cc9e5d6fdcaa82b59738b/sdks/python/apache_beam/testing/analyzers/perf_analysis.py#L152) method. + + +## Integrating the Perf Alert Tool with a Custom BigQuery Schema + +By default, the Perf Alert Tool retrieves metrics from the `apache-beam-testing` BigQuery projects. All performance and load tests within Beam utilize a standard [schema](https://github.com/apache/beam/blob/a7e12db9b5977c4a7b13554605c0300389a3d6da/sdks/python/apache_beam/testing/load_tests/load_test_metrics_utils.py#L70) for metrics publication. The tool inherently recognizes and operates with this schema when extracting metrics from BigQuery tables. + +To fetch the data from a BigQuery dataset that is not a default setting of the Apache Beam's setting, One can inherit the `MetricsFetcher` class and implement the abstract method `fetch_metric_data`. This method should return a tuple of desired metric values and timestamps of the metric values of when it was published. + +``` +from apache_beam.testing.analyzers import perf_analysis +config_file_path = +my_metric_fetcher = MyMetricsFetcher() # inherited from MetricsFetcher +perf_analysis.run(config_file_path, my_metrics_fetcher) +``` + +``Note``: The metrics and timestamps should be sorted based on the timestamps values in ascending order. + +### Configuring GitHub Parameters + +Out of the box, the performance alert tool targets the `apache/beam` repository when raising issues. If you wish to utilize this tool for another repository, you'll need to pre-set a couple of environment variables: + +- `REPO_OWNER`: Represents the owner of the repository. (e.g., `apache`) +- `REPO_NAME`: Specifies the repository name itself. (e.g., `beam`) + +Before initiating the tool, also ensure that the `GITHUB_TOKEN` is set to an authenticated GitHub token. This permits the tool to generate GitHub issues whenever performance alerts arise. ## Triage performance alert issues diff --git a/sdks/python/apache_beam/testing/analyzers/github_issues_utils.py b/sdks/python/apache_beam/testing/analyzers/github_issues_utils.py index e1f20baa50a6..82758be8f180 100644 --- a/sdks/python/apache_beam/testing/analyzers/github_issues_utils.py +++ b/sdks/python/apache_beam/testing/analyzers/github_issues_utils.py @@ -34,8 +34,8 @@ 'A Github Personal Access token is required ' 'to create Github Issues.') -_BEAM_GITHUB_REPO_OWNER = 'apache' -_BEAM_GITHUB_REPO_NAME = 'beam' +_GITHUB_REPO_OWNER = os.environ.get('REPO_OWNER', 'apache') +_GITHUB_REPO_NAME = os.environ.get('REPO_NAME', 'beam') # Adding GitHub Rest API version to the header to maintain version stability. # For more information, please look at # https://github.blog/2022-11-28-to-infinity-and-beyond-enabling-the-future-of-githubs-rest-api-with-api-versioning/ # pylint: disable=line-too-long @@ -77,10 +77,10 @@ def create_issue( Tuple containing GitHub issue number and issue URL. """ url = "https://api.github.com/repos/{}/{}/issues".format( - _BEAM_GITHUB_REPO_OWNER, _BEAM_GITHUB_REPO_NAME) + _GITHUB_REPO_OWNER, _GITHUB_REPO_NAME) data = { - 'owner': _BEAM_GITHUB_REPO_OWNER, - 'repo': _BEAM_GITHUB_REPO_NAME, + 'owner': _GITHUB_REPO_OWNER, + 'repo': _GITHUB_REPO_NAME, 'title': title, 'body': description, 'labels': [_AWAITING_TRIAGE_LABEL, _PERF_ALERT_LABEL] @@ -108,20 +108,20 @@ def comment_on_issue(issue_number: int, issue, and the comment URL. """ url = 'https://api.github.com/repos/{}/{}/issues/{}'.format( - _BEAM_GITHUB_REPO_OWNER, _BEAM_GITHUB_REPO_NAME, issue_number) + _GITHUB_REPO_OWNER, _GITHUB_REPO_NAME, issue_number) open_issue_response = requests.get( url, json.dumps({ - 'owner': _BEAM_GITHUB_REPO_OWNER, - 'repo': _BEAM_GITHUB_REPO_NAME, + 'owner': _GITHUB_REPO_OWNER, + 'repo': _GITHUB_REPO_NAME, 'issue_number': issue_number }, default=str), headers=_HEADERS).json() if open_issue_response['state'] == 'open': data = { - 'owner': _BEAM_GITHUB_REPO_OWNER, - 'repo': _BEAM_GITHUB_REPO_NAME, + 'owner': _GITHUB_REPO_OWNER, + 'repo': _GITHUB_REPO_NAME, 'body': comment_description, issue_number: issue_number, } @@ -134,13 +134,14 @@ def comment_on_issue(issue_number: int, def add_awaiting_triage_label(issue_number: int): url = 'https://api.github.com/repos/{}/{}/issues/{}/labels'.format( - _BEAM_GITHUB_REPO_OWNER, _BEAM_GITHUB_REPO_NAME, issue_number) + _GITHUB_REPO_OWNER, _GITHUB_REPO_NAME, issue_number) requests.post( url, json.dumps({'labels': [_AWAITING_TRIAGE_LABEL]}), headers=_HEADERS) def get_issue_description( - test_name: str, + test_id: str, + test_name: Optional[str], metric_name: str, timestamps: List[pd.Timestamp], metric_values: List, @@ -167,10 +168,13 @@ def get_issue_description( description = [] - description.append(_ISSUE_DESCRIPTION_TEMPLATE.format(test_name, metric_name)) + description.append(_ISSUE_DESCRIPTION_TEMPLATE.format(test_id, metric_name)) - description.append(("`Test description:` " + - f'{test_description}') if test_description else '') + if test_name: + description.append(("`test_name:` " + f'{test_name}')) + + if test_description: + description.append(("`Test description:` " + f'{test_description}')) description.append('```') diff --git a/sdks/python/apache_beam/testing/analyzers/perf_analysis.py b/sdks/python/apache_beam/testing/analyzers/perf_analysis.py index 7f1ffbb944e9..c86ecb2c4e20 100644 --- a/sdks/python/apache_beam/testing/analyzers/perf_analysis.py +++ b/sdks/python/apache_beam/testing/analyzers/perf_analysis.py @@ -23,7 +23,6 @@ import argparse import logging import os -import uuid from datetime import datetime from datetime import timezone from typing import Any @@ -33,9 +32,10 @@ import pandas as pd from apache_beam.testing.analyzers import constants +from apache_beam.testing.analyzers.perf_analysis_utils import BigQueryMetricsFetcher from apache_beam.testing.analyzers.perf_analysis_utils import GitHubIssueMetaData +from apache_beam.testing.analyzers.perf_analysis_utils import MetricsFetcher from apache_beam.testing.analyzers.perf_analysis_utils import create_performance_alert -from apache_beam.testing.analyzers.perf_analysis_utils import fetch_metric_data from apache_beam.testing.analyzers.perf_analysis_utils import find_latest_change_point_index from apache_beam.testing.analyzers.perf_analysis_utils import get_existing_issues_data from apache_beam.testing.analyzers.perf_analysis_utils import is_change_point_in_valid_window @@ -43,10 +43,10 @@ from apache_beam.testing.analyzers.perf_analysis_utils import publish_issue_metadata_to_big_query from apache_beam.testing.analyzers.perf_analysis_utils import read_test_config from apache_beam.testing.analyzers.perf_analysis_utils import validate_config -from apache_beam.testing.load_tests.load_test_metrics_utils import BigQueryMetricsFetcher -def run_change_point_analysis(params, test_name, big_query_metrics_fetcher): +def run_change_point_analysis( + params, test_id, big_query_metrics_fetcher: MetricsFetcher): """ Args: params: Dict containing parameters to run change point analysis. @@ -56,14 +56,21 @@ def run_change_point_analysis(params, test_name, big_query_metrics_fetcher): Returns: bool indicating if a change point is observed and alerted on GitHub. """ - logging.info("Running change point analysis for test %s" % test_name) + logging.info("Running change point analysis for test ID %s" % test_id) if not validate_config(params.keys()): raise ValueError( f"Please make sure all these keys {constants._PERF_TEST_KEYS} " - f"are specified for the {test_name}") + f"are specified for the {test_id}") metric_name = params['metric_name'] + # test_name will be used to query a single test from + # multiple tests in a single BQ table. Right now, the default + # assumption is that all the test have an individual BQ table + # but this might not be case for other tests(such as IO tests where + # a single BQ tables stores all the data) + test_name = params.get('test_name', None) + min_runs_between_change_points = ( constants._DEFAULT_MIN_RUNS_BETWEEN_CHANGE_POINTS) if 'min_runs_between_change_points' in params: @@ -74,15 +81,18 @@ def run_change_point_analysis(params, test_name, big_query_metrics_fetcher): if 'num_runs_in_change_point_window' in params: num_runs_in_change_point_window = params['num_runs_in_change_point_window'] - metric_values, timestamps = fetch_metric_data( - params=params, - big_query_metrics_fetcher=big_query_metrics_fetcher + metric_values, timestamps = big_query_metrics_fetcher.fetch_metric_data( + project=params['project'], + metrics_dataset=params['metrics_dataset'], + metrics_table=params['metrics_table'], + metric_name=params['metric_name'], + test_name=test_name ) change_point_index = find_latest_change_point_index( metric_values=metric_values) if not change_point_index: - logging.info("Change point is not detected for the test %s" % test_name) + logging.info("Change point is not detected for the test ID %s" % test_id) return False # since timestamps are ordered in ascending order and # num_runs_in_change_point_window refers to the latest runs, @@ -92,11 +102,11 @@ def run_change_point_analysis(params, test_name, big_query_metrics_fetcher): if not is_change_point_in_valid_window(num_runs_in_change_point_window, latest_change_point_run): logging.info( - 'Performance regression/improvement found for the test: %s. ' + 'Performance regression/improvement found for the test ID: %s. ' 'on metric %s. Since the change point run %s ' 'lies outside the num_runs_in_change_point_window distance: %s, ' 'alert is not raised.' % ( - test_name, + test_id, metric_name, latest_change_point_run + 1, num_runs_in_change_point_window)) @@ -106,8 +116,7 @@ def run_change_point_analysis(params, test_name, big_query_metrics_fetcher): last_reported_issue_number = None issue_metadata_table_name = f'{params.get("metrics_table")}_{metric_name}' existing_issue_data = get_existing_issues_data( - table_name=issue_metadata_table_name, - big_query_metrics_fetcher=big_query_metrics_fetcher) + table_name=issue_metadata_table_name) if existing_issue_data is not None: existing_issue_timestamps = existing_issue_data[ @@ -124,20 +133,21 @@ def run_change_point_analysis(params, test_name, big_query_metrics_fetcher): min_runs_between_change_points=min_runs_between_change_points) if is_alert: issue_number, issue_url = create_performance_alert( - metric_name, test_name, timestamps, + metric_name, test_id, timestamps, metric_values, change_point_index, params.get('labels', None), last_reported_issue_number, test_description = params.get('test_description', None), + test_name = test_name ) issue_metadata = GitHubIssueMetaData( issue_timestamp=pd.Timestamp( datetime.now().replace(tzinfo=timezone.utc)), # BQ doesn't allow '.' in table name - test_name=test_name.replace('.', '_'), + test_id=test_id.replace('.', '_'), + test_name=test_name, metric_name=metric_name, - test_id=uuid.uuid4().hex, change_point=metric_values[change_point_index], issue_number=issue_number, issue_url=issue_url, @@ -149,7 +159,10 @@ def run_change_point_analysis(params, test_name, big_query_metrics_fetcher): return is_alert -def run(config_file_path: Optional[str] = None) -> None: +def run( + big_query_metrics_fetcher: MetricsFetcher = BigQueryMetricsFetcher(), + config_file_path: Optional[str] = None, +) -> None: """ run is the entry point to run change point analysis on test metric data, which is read from config file, and if there is a performance @@ -169,12 +182,10 @@ def run(config_file_path: Optional[str] = None) -> None: tests_config: Dict[str, Dict[str, Any]] = read_test_config(config_file_path) - big_query_metrics_fetcher = BigQueryMetricsFetcher() - - for test_name, params in tests_config.items(): + for test_id, params in tests_config.items(): run_change_point_analysis( params=params, - test_name=test_name, + test_id=test_id, big_query_metrics_fetcher=big_query_metrics_fetcher) diff --git a/sdks/python/apache_beam/testing/analyzers/perf_analysis_test.py b/sdks/python/apache_beam/testing/analyzers/perf_analysis_test.py index 094cd9c47ec0..9c7921300d9d 100644 --- a/sdks/python/apache_beam/testing/analyzers/perf_analysis_test.py +++ b/sdks/python/apache_beam/testing/analyzers/perf_analysis_test.py @@ -32,6 +32,7 @@ from apache_beam.io.filesystems import FileSystems from apache_beam.testing.analyzers import constants from apache_beam.testing.analyzers import github_issues_utils + from apache_beam.testing.analyzers.perf_analysis_utils import BigQueryMetricsFetcher from apache_beam.testing.analyzers.perf_analysis_utils import is_change_point_in_valid_window from apache_beam.testing.analyzers.perf_analysis_utils import is_perf_alert from apache_beam.testing.analyzers.perf_analysis_utils import e_divisive @@ -41,18 +42,18 @@ from apache_beam.testing.analyzers.perf_analysis_utils import validate_config from apache_beam.testing.load_tests import load_test_metrics_utils except ImportError as e: - analysis = None # type: ignore + raise unittest.SkipTest('Missing dependencies to run perf analysis tests.') # mock methods. -def get_fake_data_with_no_change_point(**kwargs): +def get_fake_data_with_no_change_point(*args, **kwargs): num_samples = 20 metric_values = [1] * num_samples timestamps = list(range(num_samples)) return metric_values, timestamps -def get_fake_data_with_change_point(**kwargs): +def get_fake_data_with_change_point(*args, **kwargs): # change point will be at index 13. num_samples = 20 metric_values = [0] * 12 + [3] + [4] * 7 @@ -69,10 +70,6 @@ def get_existing_issue_data(**kwargs): }]) -@unittest.skipIf( - analysis is None, - 'Missing dependencies. ' - 'Test dependencies are missing for the Analyzer.') class TestChangePointAnalysis(unittest.TestCase): def setUp(self) -> None: self.single_change_point_series = [0] * 10 + [1] * 10 @@ -151,18 +148,20 @@ def test_duplicate_change_points_are_not_valid_alerts(self): min_runs_between_change_points=min_runs_between_change_points) self.assertFalse(is_alert) - @mock.patch( - 'apache_beam.testing.analyzers.perf_analysis.fetch_metric_data', + @mock.patch.object( + BigQueryMetricsFetcher, + 'fetch_metric_data', get_fake_data_with_no_change_point) def test_no_alerts_when_no_change_points(self): is_alert = analysis.run_change_point_analysis( params=self.params, - test_name=self.test_id, - big_query_metrics_fetcher=None) + test_id=self.test_id, + big_query_metrics_fetcher=BigQueryMetricsFetcher()) self.assertFalse(is_alert) - @mock.patch( - 'apache_beam.testing.analyzers.perf_analysis.fetch_metric_data', + @mock.patch.object( + BigQueryMetricsFetcher, + 'fetch_metric_data', get_fake_data_with_change_point) @mock.patch( 'apache_beam.testing.analyzers.perf_analysis.get_existing_issues_data', @@ -178,12 +177,13 @@ def test_no_alerts_when_no_change_points(self): def test_alert_on_data_with_change_point(self, *args): is_alert = analysis.run_change_point_analysis( params=self.params, - test_name=self.test_id, - big_query_metrics_fetcher=None) + test_id=self.test_id, + big_query_metrics_fetcher=BigQueryMetricsFetcher()) self.assertTrue(is_alert) - @mock.patch( - 'apache_beam.testing.analyzers.perf_analysis.fetch_metric_data', + @mock.patch.object( + BigQueryMetricsFetcher, + 'fetch_metric_data', get_fake_data_with_change_point) @mock.patch( 'apache_beam.testing.analyzers.perf_analysis.get_existing_issues_data', @@ -198,8 +198,8 @@ def test_alert_on_data_with_change_point(self, *args): def test_alert_on_data_with_reported_change_point(self, *args): is_alert = analysis.run_change_point_analysis( params=self.params, - test_name=self.test_id, - big_query_metrics_fetcher=None) + test_id=self.test_id, + big_query_metrics_fetcher=BigQueryMetricsFetcher()) self.assertFalse(is_alert) def test_change_point_has_anomaly_marker_in_gh_description(self): @@ -208,7 +208,8 @@ def test_change_point_has_anomaly_marker_in_gh_description(self): change_point_index = find_latest_change_point_index(metric_values) description = github_issues_utils.get_issue_description( - test_name=self.test_id, + test_id=self.test_id, + test_name=self.params.get('test_name', None), test_description=self.params['test_description'], metric_name=self.params['metric_name'], metric_values=metric_values, diff --git a/sdks/python/apache_beam/testing/analyzers/perf_analysis_utils.py b/sdks/python/apache_beam/testing/analyzers/perf_analysis_utils.py index 0a559fc4beeb..f9604c490fc0 100644 --- a/sdks/python/apache_beam/testing/analyzers/perf_analysis_utils.py +++ b/sdks/python/apache_beam/testing/analyzers/perf_analysis_utils.py @@ -14,11 +14,11 @@ # See the License for the specific language governing permissions and # limitations under the License. # +import abc import logging from dataclasses import asdict from dataclasses import dataclass from statistics import median -from typing import Any from typing import Dict from typing import List from typing import Optional @@ -28,11 +28,11 @@ import pandas as pd import yaml from google.api_core import exceptions +from google.cloud import bigquery from apache_beam.testing.analyzers import constants from apache_beam.testing.analyzers import github_issues_utils from apache_beam.testing.load_tests import load_test_metrics_utils -from apache_beam.testing.load_tests.load_test_metrics_utils import BigQueryMetricsFetcher from apache_beam.testing.load_tests.load_test_metrics_utils import BigQueryMetricsPublisher from signal_processing_algorithms.energy_statistics.energy_statistics import e_divisive @@ -59,9 +59,7 @@ def is_change_point_in_valid_window( return num_runs_in_change_point_window > latest_change_point_run -def get_existing_issues_data( - table_name: str, big_query_metrics_fetcher: BigQueryMetricsFetcher -) -> Optional[pd.DataFrame]: +def get_existing_issues_data(table_name: str) -> Optional[pd.DataFrame]: """ Finds the most recent GitHub issue created for the test_name. If no table found with name=test_name, return (None, None) @@ -73,12 +71,14 @@ def get_existing_issues_data( LIMIT 10 """ try: - df = big_query_metrics_fetcher.fetch(query=query) + client = bigquery.Client() + query_job = client.query(query=query) + existing_issue_data = query_job.result().to_dataframe() except exceptions.NotFound: # If no table found, that means this is first performance regression # on the current test+metric. return None - return df + return existing_issue_data def is_perf_alert( @@ -123,33 +123,6 @@ def validate_config(keys): return constants._PERF_TEST_KEYS.issubset(keys) -def fetch_metric_data( - params: Dict[str, Any], big_query_metrics_fetcher: BigQueryMetricsFetcher -) -> Tuple[List[Union[int, float]], List[pd.Timestamp]]: - """ - Args: - params: Dict containing keys required to fetch data from a data source. - big_query_metrics_fetcher: A BigQuery metrics fetcher for fetch metrics. - Returns: - Tuple[List[Union[int, float]], List[pd.Timestamp]]: Tuple containing list - of metric_values and list of timestamps. Both are sorted in ascending - order wrt timestamps. - """ - query = f""" - SELECT * - FROM {params['project']}.{params['metrics_dataset']}.{params['metrics_table']} - WHERE CONTAINS_SUBSTR(({load_test_metrics_utils.METRICS_TYPE_LABEL}), '{params['metric_name']}') - ORDER BY {load_test_metrics_utils.SUBMIT_TIMESTAMP_LABEL} DESC - LIMIT {constants._NUM_DATA_POINTS_TO_RUN_CHANGE_POINT_ANALYSIS} - """ - metric_data: pd.DataFrame = big_query_metrics_fetcher.fetch(query=query) - metric_data.sort_values( - by=[load_test_metrics_utils.SUBMIT_TIMESTAMP_LABEL], inplace=True) - return ( - metric_data[load_test_metrics_utils.VALUE_LABEL].tolist(), - metric_data[load_test_metrics_utils.SUBMIT_TIMESTAMP_LABEL].tolist()) - - def find_change_points(metric_values: List[Union[float, int]]): return e_divisive(metric_values) @@ -175,7 +148,7 @@ def find_latest_change_point_index(metric_values: List[Union[float, int]]): def publish_issue_metadata_to_big_query(issue_metadata, table_name): """ - Published issue_metadata to BigQuery with table name=test_name. + Published issue_metadata to BigQuery with table name. """ bq_metrics_publisher = BigQueryMetricsPublisher( project_name=constants._BQ_PROJECT_NAME, @@ -190,18 +163,21 @@ def publish_issue_metadata_to_big_query(issue_metadata, table_name): def create_performance_alert( metric_name: str, - test_name: str, + test_id: str, timestamps: List[pd.Timestamp], metric_values: List[Union[int, float]], change_point_index: int, labels: List[str], existing_issue_number: Optional[int], - test_description: Optional[str] = None) -> Tuple[int, str]: + test_description: Optional[str] = None, + test_name: Optional[str] = None, +) -> Tuple[int, str]: """ Creates performance alert on GitHub issues and returns GitHub issue number and issue URL. """ description = github_issues_utils.get_issue_description( + test_id=test_id, test_name=test_name, test_description=test_description, metric_name=metric_name, @@ -213,7 +189,7 @@ def create_performance_alert( issue_number, issue_url = github_issues_utils.report_change_point_on_issues( title=github_issues_utils._ISSUE_TITLE_TEMPLATE.format( - test_name, metric_name + test_id, metric_name ), description=description, labels=labels, @@ -253,3 +229,55 @@ def filter_change_points_by_median_threshold( if relative_change > threshold: valid_change_points.append(idx) return valid_change_points + + +class MetricsFetcher(metaclass=abc.ABCMeta): + @abc.abstractmethod + def fetch_metric_data( + self, + *, + project, + metrics_dataset, + metrics_table, + metric_name, + test_name=None) -> Tuple[List[Union[int, float]], List[pd.Timestamp]]: + """ + Define SQL query and fetch the timestamp values and metric values + from BigQuery tables. + """ + raise NotImplementedError + + +class BigQueryMetricsFetcher(MetricsFetcher): + def fetch_metric_data( + self, + *, + project, + metrics_dataset, + metrics_table, + metric_name, + test_name=None, + ) -> Tuple[List[Union[int, float]], List[pd.Timestamp]]: + """ + Args: + params: Dict containing keys required to fetch data from a data source. + Returns: + Tuple[List[Union[int, float]], List[pd.Timestamp]]: Tuple containing list + of metric_values and list of timestamps. Both are sorted in ascending + order wrt timestamps. + """ + query = f""" + SELECT * + FROM {project}.{metrics_dataset}.{metrics_table} + WHERE CONTAINS_SUBSTR(({load_test_metrics_utils.METRICS_TYPE_LABEL}), '{metric_name}') + ORDER BY {load_test_metrics_utils.SUBMIT_TIMESTAMP_LABEL} DESC + LIMIT {constants._NUM_DATA_POINTS_TO_RUN_CHANGE_POINT_ANALYSIS} + """ + client = bigquery.Client() + query_job = client.query(query=query) + metric_data = query_job.result().to_dataframe() + metric_data.sort_values( + by=[load_test_metrics_utils.SUBMIT_TIMESTAMP_LABEL], inplace=True) + return ( + metric_data[load_test_metrics_utils.VALUE_LABEL].tolist(), + metric_data[load_test_metrics_utils.SUBMIT_TIMESTAMP_LABEL].tolist()) diff --git a/sdks/python/apache_beam/testing/analyzers/tests_config.yaml b/sdks/python/apache_beam/testing/analyzers/tests_config.yaml index f808f5e41d74..ec9cfe6f1ac0 100644 --- a/sdks/python/apache_beam/testing/analyzers/tests_config.yaml +++ b/sdks/python/apache_beam/testing/analyzers/tests_config.yaml @@ -16,7 +16,7 @@ # # for the unique key to define a test, please use the following format: -# {test_name}-{metric_name} +# {test_id}-{metric_name} pytorch_image_classification_benchmarks-resnet152-mean_inference_batch_latency_micro_secs: test_description: diff --git a/sdks/python/apache_beam/testing/load_tests/load_test_metrics_utils.py b/sdks/python/apache_beam/testing/load_tests/load_test_metrics_utils.py index 92a5f68351fe..01db2c114efb 100644 --- a/sdks/python/apache_beam/testing/load_tests/load_test_metrics_utils.py +++ b/sdks/python/apache_beam/testing/load_tests/load_test_metrics_utils.py @@ -38,7 +38,6 @@ from typing import Optional from typing import Union -import pandas as pd import requests from requests.auth import HTTPBasicAuth @@ -650,13 +649,3 @@ def __init__(self): def process(self, element): yield self.timestamp_val_fn( element, self.timestamp_fn(micros=int(self.time_fn() * 1000000))) - - -class BigQueryMetricsFetcher: - def __init__(self): - self.client = bigquery.Client() - - def fetch(self, query) -> pd.DataFrame: - query_job = self.client.query(query=query) - result = query_job.result() - return result.to_dataframe() From 94a579ccdc968c6b01a481777e3140ab361738fa Mon Sep 17 00:00:00 2001 From: caneff Date: Thu, 5 Oct 2023 13:37:14 -0400 Subject: [PATCH 032/435] Add observed=False to be explicit for Pandas 2.1 (#28823) --- sdks/python/apache_beam/dataframe/frames_test.py | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/dataframe/frames_test.py b/sdks/python/apache_beam/dataframe/frames_test.py index 6f7a63c29164..fffb5391c760 100644 --- a/sdks/python/apache_beam/dataframe/frames_test.py +++ b/sdks/python/apache_beam/dataframe/frames_test.py @@ -1021,8 +1021,14 @@ def test_categorical_groupby(self): df = df.set_index('B') # TODO(BEAM-11190): These aggregations can be done in index partitions, but # it will require a little more complex logic - self._run_test(lambda df: df.groupby(level=0).sum(), df, nonparallel=True) - self._run_test(lambda df: df.groupby(level=0).mean(), df, nonparallel=True) + self._run_test( + lambda df: df.groupby(level=0, observed=False).sum(), + df, + nonparallel=True) + self._run_test( + lambda df: df.groupby(level=0, observed=False).mean(), + df, + nonparallel=True) def test_astype_categorical(self): df = pd.DataFrame({'A': np.arange(6), 'B': list('aabbca')}) From e6a1062b5483f62fd1152418535f6746dd69fc6e Mon Sep 17 00:00:00 2001 From: caneff Date: Thu, 5 Oct 2023 13:37:23 -0400 Subject: [PATCH 033/435] Add map function and skip applymap for 2.1 (#28818) --- sdks/python/apache_beam/dataframe/frames.py | 2 ++ sdks/python/apache_beam/dataframe/frames_test.py | 13 +++++++++++++ 2 files changed, 15 insertions(+) diff --git a/sdks/python/apache_beam/dataframe/frames.py b/sdks/python/apache_beam/dataframe/frames.py index f51814e7f696..b7aa130fbbd8 100644 --- a/sdks/python/apache_beam/dataframe/frames.py +++ b/sdks/python/apache_beam/dataframe/frames.py @@ -2978,6 +2978,8 @@ def aggregate(self, func, axis, *args, **kwargs): agg = aggregate applymap = frame_base._elementwise_method('applymap', base=pd.DataFrame) + if PD_VERSION >= (2, 1): + map = frame_base._elementwise_method('map', base=pd.DataFrame) add_prefix = frame_base._elementwise_method('add_prefix', base=pd.DataFrame) add_suffix = frame_base._elementwise_method('add_suffix', base=pd.DataFrame) diff --git a/sdks/python/apache_beam/dataframe/frames_test.py b/sdks/python/apache_beam/dataframe/frames_test.py index fffb5391c760..6e32acefc61b 100644 --- a/sdks/python/apache_beam/dataframe/frames_test.py +++ b/sdks/python/apache_beam/dataframe/frames_test.py @@ -865,6 +865,8 @@ def test_corrwith_bad_axis(self): self._run_error_test(lambda df: df.corrwith(df, axis=5), df) @unittest.skipIf(PD_VERSION < (1, 2), "na_action added in pandas 1.2.0") + @pytest.mark.filterwarnings( + "ignore:The default of observed=False is deprecated:FutureWarning") def test_applymap_na_action(self): # Replicates a doctest for na_action which is incompatible with # doctest framework @@ -875,6 +877,17 @@ def test_applymap_na_action(self): # TODO: generate proxy using naive type inference on fn check_proxy=False) + @unittest.skipIf(PD_VERSION < (2, 1), "map added in 2.1.0") + def test_map_na_action(self): + # Replicates a doctest for na_action which is incompatible with + # doctest framework + df = pd.DataFrame([[pd.NA, 2.12], [3.356, 4.567]]) + self._run_test( + lambda df: df.map(lambda x: len(str(x)), na_action='ignore'), + df, + # TODO: generate proxy using naive type inference on fn + check_proxy=False) + def test_dataframe_eval_query(self): df = pd.DataFrame(np.random.randn(20, 3), columns=['a', 'b', 'c']) self._run_test(lambda df: df.eval('foo = a + b - c'), df) From 30c5e05ed4dd0478295ac0c69553a0758fbce9ff Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 5 Oct 2023 13:44:16 -0400 Subject: [PATCH 034/435] Bump github.com/fsouza/fake-gcs-server from 1.47.4 to 1.47.5 in /sdks (#28812) Bumps [github.com/fsouza/fake-gcs-server](https://github.com/fsouza/fake-gcs-server) from 1.47.4 to 1.47.5. - [Release notes](https://github.com/fsouza/fake-gcs-server/releases) - [Commits](https://github.com/fsouza/fake-gcs-server/compare/v1.47.4...v1.47.5) --- updated-dependencies: - dependency-name: github.com/fsouza/fake-gcs-server dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 6 +++--- sdks/go.sum | 14 +++++++------- 2 files changed, 10 insertions(+), 10 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 89f119a54908..d817ae549857 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -67,8 +67,8 @@ require ( ) require ( - github.com/fsouza/fake-gcs-server v1.47.4 - golang.org/x/exp v0.0.0-20230713183714-613f0c0eb8a1 + github.com/fsouza/fake-gcs-server v1.47.5 + golang.org/x/exp v0.0.0-20230807204917-050eac23e9de ) require ( @@ -88,7 +88,7 @@ require ( cloud.google.com/go v0.110.7 // indirect cloud.google.com/go/compute v1.23.0 // indirect cloud.google.com/go/compute/metadata v0.2.3 // indirect - cloud.google.com/go/iam v1.1.1 // indirect + cloud.google.com/go/iam v1.1.2 // indirect cloud.google.com/go/longrunning v0.5.1 // indirect github.com/Azure/go-ansiterm v0.0.0-20210617225240-d185dfc1b5a1 // indirect github.com/Microsoft/go-winio v0.6.1 // indirect diff --git a/sdks/go.sum b/sdks/go.sum index 582f3d69f88e..9f43e9a53abc 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -26,8 +26,8 @@ cloud.google.com/go/datastore v1.0.0/go.mod h1:LXYbyblFSglQ5pkeyhO+Qmw7ukd3C+pD7 cloud.google.com/go/datastore v1.1.0/go.mod h1:umbIZjpQpHh4hmRpGhH4tLFup+FVzqBi1b3c64qFpCk= cloud.google.com/go/datastore v1.14.0 h1:Mq0ApTRdLW3/dyiw+DkjTk0+iGIUvkbzaC8sfPwWTH4= cloud.google.com/go/datastore v1.14.0/go.mod h1:GAeStMBIt9bPS7jMJA85kgkpsMkvseWWXiaHya9Jes8= -cloud.google.com/go/iam v1.1.1 h1:lW7fzj15aVIXYHREOqjRBV9PsH0Z6u8Y46a1YGvQP4Y= -cloud.google.com/go/iam v1.1.1/go.mod h1:A5avdyVL2tCppe4unb0951eI9jreack+RJ0/d+KUZOU= +cloud.google.com/go/iam v1.1.2 h1:gacbrBdWcoVmGLozRuStX45YKvJtzIjJdAolzUs1sm4= +cloud.google.com/go/iam v1.1.2/go.mod h1:A5avdyVL2tCppe4unb0951eI9jreack+RJ0/d+KUZOU= cloud.google.com/go/kms v1.15.0 h1:xYl5WEaSekKYN5gGRyhjvZKM22GVBBCzegGNVPy+aIs= cloud.google.com/go/longrunning v0.5.1 h1:Fr7TXftcqTudoyRJa113hyaqlGdiBQkp0Gq7tErFDWI= cloud.google.com/go/longrunning v0.5.1/go.mod h1:spvimkwdz6SPWKEt/XBij79E9fiTkHSQl/fRUUQJYJc= @@ -195,8 +195,8 @@ github.com/form3tech-oss/jwt-go v3.2.2+incompatible/go.mod h1:pbq4aXjuKjdthFRnoD github.com/frankban/quicktest v1.2.2/go.mod h1:Qh/WofXFeiAFII1aEBu529AtJo6Zg2VHscnEsbBnJ20= github.com/frankban/quicktest v1.11.3 h1:8sXhOn0uLys67V8EsXLc6eszDs8VXWxL3iRvebPhedY= github.com/frankban/quicktest v1.11.3/go.mod h1:wRf/ReqHper53s+kmmSZizM8NamnL3IM0I9ntUbOk+k= -github.com/fsouza/fake-gcs-server v1.47.4 h1:gfBhBxEra20/Om02cvcyL8EnekV8KDb01Yffjat6AKQ= -github.com/fsouza/fake-gcs-server v1.47.4/go.mod h1:vqUZbI12uy9IkRQ54Q4p5AniQsSiUq8alO9Nv2egMmA= +github.com/fsouza/fake-gcs-server v1.47.5 h1:o+wL01s01j/2OdkIaduDogXw2bZveq9TFb8f+BqEHtM= +github.com/fsouza/fake-gcs-server v1.47.5/go.mod h1:PhN8F1rHAOCL5jWyXcw8nPfLfHnka6D9fT7ctL9nbkA= github.com/go-gl/glfw v0.0.0-20190409004039-e6da0acd62b1/go.mod h1:vR7hzQXu2zJy9AVAgeJqvqgH9Q5CA+iKCZ2gyEVpxRU= github.com/go-gl/glfw/v3.3/glfw v0.0.0-20191125211704-12ad95a8df72/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= github.com/go-gl/glfw/v3.3/glfw v0.0.0-20200222043503-6f7a984d4dc4/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= @@ -348,7 +348,7 @@ github.com/minio/asm2plan9s v0.0.0-20200509001527-cdd76441f9d8/go.mod h1:mC1jAcs github.com/minio/c2goasm v0.0.0-20190812172519-36a3d3bbc4f3 h1:+n/aFZefKZp7spd8DFdX7uMikMLXX4oubIzJF4kv/wI= github.com/minio/c2goasm v0.0.0-20190812172519-36a3d3bbc4f3/go.mod h1:RagcQ7I8IeTMnF8JTXieKnO4Z6JCsikNEzj0DwauVzE= github.com/minio/md5-simd v1.1.2 h1:Gdi1DZK69+ZVMoNHRXJyNcxrMA4dSxoYHZSQbirFg34= -github.com/minio/minio-go/v7 v7.0.61 h1:87c+x8J3jxQ5VUGimV9oHdpjsAvy3fhneEBKuoKEVUI= +github.com/minio/minio-go/v7 v7.0.63 h1:GbZ2oCvaUdgT5640WJOpyDhhDxvknAJU2/T3yurwcbQ= github.com/minio/sha256-simd v1.0.1 h1:6kaan5IFmwTNynnKKpDHe6FWHohJOHhCPchzK49dzMM= github.com/moby/patternmatcher v0.5.0 h1:YCZgJOeULcxLw1Q+sVR636pmS7sPEn1Qo2iAN6M7DBo= github.com/moby/patternmatcher v0.5.0/go.mod h1:hDPoyOpDY7OrrMDLaYoY3hf52gNCR/YOUYxkhApJIxc= @@ -497,8 +497,8 @@ golang.org/x/exp v0.0.0-20191227195350-da58074b4299/go.mod h1:2RIsYlXP63K8oxa1u0 golang.org/x/exp v0.0.0-20200119233911-0405dc783f0a/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4= golang.org/x/exp v0.0.0-20200207192155-f17229e696bd/go.mod h1:J/WKrq2StrnmMY6+EHIKF9dgMWnmCNThgcyBT1FY9mM= golang.org/x/exp v0.0.0-20200224162631-6cc2880d07d6/go.mod h1:3jZMyOhIsHpP37uCMkUooju7aAi5cS1Q23tOzKc+0MU= -golang.org/x/exp v0.0.0-20230713183714-613f0c0eb8a1 h1:MGwJjxBy0HJshjDNfLsYO8xppfqWlA5ZT9OhtUUhTNw= -golang.org/x/exp v0.0.0-20230713183714-613f0c0eb8a1/go.mod h1:FXUEEKJgO7OQYeo8N01OfiKP8RXMtf6e8aTskBGqWdc= +golang.org/x/exp v0.0.0-20230807204917-050eac23e9de h1:l5Za6utMv/HsBWWqzt4S8X17j+kt1uVETUX5UFhn2rE= +golang.org/x/exp v0.0.0-20230807204917-050eac23e9de/go.mod h1:FXUEEKJgO7OQYeo8N01OfiKP8RXMtf6e8aTskBGqWdc= golang.org/x/image v0.0.0-20190227222117-0694c2d4d067/go.mod h1:kZ7UVZpmo3dzQBMxlp+ypCbDeSB+sBbTgSJuh5dn5js= golang.org/x/image v0.0.0-20190802002840-cff245a6509b/go.mod h1:FeLwcggjj3mMvU+oOTbSwawSJRM1uh48EjtB4UJZlP0= golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= From 31e1c7a8aef905a54c5382930e43e2d36d2d5773 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 5 Oct 2023 13:44:46 -0400 Subject: [PATCH 035/435] Bump pillow (#28810) Bumps [pillow](https://github.com/python-pillow/Pillow) from 9.3.0 to 10.0.1. - [Release notes](https://github.com/python-pillow/Pillow/releases) - [Changelog](https://github.com/python-pillow/Pillow/blob/main/CHANGES.rst) - [Commits](https://github.com/python-pillow/Pillow/compare/9.3.0...10.0.1) --- updated-dependencies: - dependency-name: pillow dependency-type: direct:production ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .../kfp/components/preprocessing/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/examples/ml-orchestration/kfp/components/preprocessing/requirements.txt b/sdks/python/apache_beam/examples/ml-orchestration/kfp/components/preprocessing/requirements.txt index e902ead34151..706adf9de0aa 100644 --- a/sdks/python/apache_beam/examples/ml-orchestration/kfp/components/preprocessing/requirements.txt +++ b/sdks/python/apache_beam/examples/ml-orchestration/kfp/components/preprocessing/requirements.txt @@ -18,4 +18,4 @@ requests==2.31.0 torch==1.13.1 torchvision==0.13.0 numpy==1.22.4 -Pillow==9.3.0 +Pillow==10.0.1 From 0304cae8cc8f3d69a7c192beec071ff1d8474538 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud <65791736+ahmedabu98@users.noreply.github.com> Date: Thu, 5 Oct 2023 14:40:16 -0400 Subject: [PATCH 036/435] add bigtable reviewers (#28834) --- .github/REVIEWERS.yml | 4 ++++ .github/autolabeler.yml | 1 + 2 files changed, 5 insertions(+) diff --git a/.github/REVIEWERS.yml b/.github/REVIEWERS.yml index f472568f3479..1c260fbea5eb 100644 --- a/.github/REVIEWERS.yml +++ b/.github/REVIEWERS.yml @@ -52,6 +52,10 @@ labels: - name: spanner reviewers: - nielm + - name: bigtable + reviewers: + - igorbernstein2 + - mutianf exclusionList: [] - name: Build reviewers: diff --git a/.github/autolabeler.yml b/.github/autolabeler.yml index d1cc8296d303..57c8f65c6ac1 100644 --- a/.github/autolabeler.yml +++ b/.github/autolabeler.yml @@ -69,6 +69,7 @@ io: ["sdks/go/pkg/beam/io/**/*", "sdks/java/io/**/*", "sdks/python/apache_beam/ "redis": ["sdks/java/io/redis/**/*"] "solr": ["sdks/java/io/solr/**/*"] "spanner": ["sdks/go/pkg/beam/io/spannerio/**/*", "sdks/python/apache_beam/io/gcp/spanner.py", "sdks/python/apache_beam/io/gcp/experimental/spannerio.py", "sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/**/*"] +"bigtable": ["sdks/go/pkg/beam/io/bigtableio/**/*", "sdks/go/pkg/beam/io/xlang/bigtableio/**/*", "sdks/python/apache_beam/io/gcp/bigtableio.py", "sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/**/*"] "synthetic": ["sdks/java/io/synthetic/**/*"] "tests": ["sdks/java/io/file-based-io-tests/**/*"] "thrift": ["sdks/java/io/thrift/**/*"] From c2d7d5fbbc6a0d89131a624665116758c194e3ea Mon Sep 17 00:00:00 2001 From: Joar Wandborg Date: Thu, 5 Oct 2023 21:44:09 +0200 Subject: [PATCH 037/435] BigtableIO: Force minimum value of `desiredBundleSizeBytes` to be 1 (#28798) --- CHANGES.md | 34 +++++++++++++++++++ .../beam/sdk/io/gcp/bigtable/BigtableIO.java | 6 +++- .../sdk/io/gcp/bigtable/BigtableIOTest.java | 33 ++++++++++++++++++ 3 files changed, 72 insertions(+), 1 deletion(-) diff --git a/CHANGES.md b/CHANGES.md index 650b33c12407..0c2c2e3f79f4 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -53,6 +53,40 @@ * ([#X](https://github.com/apache/beam/issues/X)). --> +# [2.52.0] - Unreleased + +## Highlights + +* New highly anticipated feature X added to Python SDK ([#X](https://github.com/apache/beam/issues/X)). +* New highly anticipated feature Y added to Java SDK ([#Y](https://github.com/apache/beam/issues/Y)). + +## I/Os + +* Support for X source added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). + +## New Features / Improvements + +* X feature added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). + +## Breaking Changes + +* X behavior was changed ([#X](https://github.com/apache/beam/issues/X)). + +## Deprecations + +* X behavior is deprecated and will be removed in X versions ([#X](https://github.com/apache/beam/issues/X)). + +## Bugfixes + +* Fixed "Desired bundle size 0 bytes must be greater than 0" in Java SDK's BigtableIO.BigtableSource when you have more cores than bytes to read (Java) [#28793](https://github.com/apache/beam/issues/28793). + +## Security Fixes +* Fixed (CVE-YYYY-NNNN)[https://www.cve.org/CVERecord?id=CVE-YYYY-NNNN] (Java/Python/Go) ([#X](https://github.com/apache/beam/issues/X)). + +## Known Issues + +* ([#X](https://github.com/apache/beam/issues/X)). + # [2.51.0] - Unreleased ## Highlights diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java index 92a0af205482..ef868e8bf7c9 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java @@ -1326,7 +1326,11 @@ public List split(long desiredBundleSizeBytes, PipelineOptions o long maximumNumberOfSplits = 4000; long sizeEstimate = getEstimatedSizeBytes(options); desiredBundleSizeBytes = - Math.max(sizeEstimate / maximumNumberOfSplits, desiredBundleSizeBytes); + Math.max( + sizeEstimate / maximumNumberOfSplits, + // BoundedReadEvaluatorFactory may provide us with a desiredBundleSizeBytes of 0 + // https://github.com/apache/beam/issues/28793 + Math.max(1, desiredBundleSizeBytes)); // Delegate to testable helper. List splits = diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java index bb70eb78984c..971b91d89b74 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java @@ -766,6 +766,39 @@ public void testReadingWithSplits() throws Exception { assertSourcesEqualReferenceSource(source, splits, null /* options */); } + /** + * Regression test for [Bug]: BigtableSource + * "Desired bundle size 0 bytes must be greater than 0" #28793 . + */ + @Test + public void testSplittingWithDesiredBundleSizeZero() throws Exception { + final String table = "TEST-SPLIT-DESIRED-BUNDLE-SIZE-ZERO-TABLE"; + final int numRows = 10; + final int numSamples = 10; + final long bytesPerRow = 1L; + + // Set up test table data and sample row keys for size estimation and splitting. + makeTableData(table, numRows); + service.setupSampleRowKeys(table, numSamples, bytesPerRow); + + // Generate source and split it. + BigtableSource source = + new BigtableSource( + factory, + configId, + config, + BigtableReadOptions.builder() + .setTableId(StaticValueProvider.of(table)) + .setKeyRanges(ALL_KEY_RANGE) + .build(), + null /*size*/); + List splits = source.split(0, null /* options */); + + // Test num splits and split equality. + assertThat(splits, hasSize(numSamples)); + assertSourcesEqualReferenceSource(source, splits, null /* options */); + } + @Test public void testReadingWithSplitFailed() throws Exception { FailureBigtableService failureService = From b8eb9df44ab30559de2ccf23c8313604e003178e Mon Sep 17 00:00:00 2001 From: Vitaly Terentyev Date: Fri, 6 Oct 2023 00:08:15 +0400 Subject: [PATCH 038/435] Add Publish Docker Snapshots workflow (#28794) * Add Publish Docker Snapshots workflow * Fix timeout --- .../beam_Publish_Docker_Snapshots.yml | 85 +++++++++++++++++++ 1 file changed, 85 insertions(+) create mode 100644 .github/workflows/beam_Publish_Docker_Snapshots.yml diff --git a/.github/workflows/beam_Publish_Docker_Snapshots.yml b/.github/workflows/beam_Publish_Docker_Snapshots.yml new file mode 100644 index 000000000000..41262a58b1e7 --- /dev/null +++ b/.github/workflows/beam_Publish_Docker_Snapshots.yml @@ -0,0 +1,85 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: Publish Docker Snapshots + +on: + issue_comment: + types: [created] + schedule: + - cron: '0 13 * * *' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_Publish_Docker_Snapshots: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Publish Docker Snapshots' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 100 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_Publish_Docker_Snapshots"] + job_phrase: ["Publish Docker Snapshots"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: run Publish Docker Snapshots script for Spark + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :runners:spark:3:job-server:container:dockerPush + arguments: | + -Pdocker-repository-root=gcr.io/apache-beam-testing/beam_portability \ + -Pdocker-tag-list=latest + - name: run Publish Docker Snapshots script for Flink + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :runners:flink:1.15:job-server-container:dockerPush + arguments: | + -Pdocker-repository-root=gcr.io/apache-beam-testing/beam_portability \ + -Pdocker-tag-list=latest \ No newline at end of file From 13e4da3c2d75cdf13d738728177d370505f925d4 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 5 Oct 2023 13:47:19 -0700 Subject: [PATCH 039/435] Bump google.golang.org/api from 0.143.0 to 0.144.0 in /sdks (#28839) Bumps [google.golang.org/api](https://github.com/googleapis/google-api-go-client) from 0.143.0 to 0.144.0. - [Release notes](https://github.com/googleapis/google-api-go-client/releases) - [Changelog](https://github.com/googleapis/google-api-go-client/blob/main/CHANGES.md) - [Commits](https://github.com/googleapis/google-api-go-client/compare/v0.143.0...v0.144.0) --- updated-dependencies: - dependency-name: google.golang.org/api dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 2 +- sdks/go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index d817ae549857..aa147a8dc682 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -57,7 +57,7 @@ require ( golang.org/x/sync v0.3.0 golang.org/x/sys v0.12.0 golang.org/x/text v0.13.0 - google.golang.org/api v0.143.0 + google.golang.org/api v0.144.0 google.golang.org/genproto v0.0.0-20230913181813-007df8e322eb google.golang.org/grpc v1.58.2 google.golang.org/protobuf v1.31.0 diff --git a/sdks/go.sum b/sdks/go.sum index 9f43e9a53abc..12c43b056f84 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -674,8 +674,8 @@ google.golang.org/api v0.14.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsb google.golang.org/api v0.15.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI= google.golang.org/api v0.17.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= google.golang.org/api v0.18.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= -google.golang.org/api v0.143.0 h1:o8cekTkqhywkbZT6p1UHJPZ9+9uuCAJs/KYomxZB8fA= -google.golang.org/api v0.143.0/go.mod h1:FoX9DO9hT7DLNn97OuoZAGSDuNAXdJRuGK98rSUgurk= +google.golang.org/api v0.144.0 h1:01xgplvIwdMpnrlenPHMgRAAgAH9N5Zv21Qu6XwJxSU= +google.golang.org/api v0.144.0/go.mod h1:OARJqIfoYjXJj4C1AiBSXYZt03qsoz8FQYU6fBEfrHM= google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/appengine v1.5.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= From 6722ff197e80c8c797c911a53c0a518b91e0f729 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Thu, 5 Oct 2023 15:31:18 -0700 Subject: [PATCH 040/435] Schema-aware Avro reading and writing. --- sdks/python/apache_beam/io/avroio.py | 271 +++++++++++++++++-- sdks/python/apache_beam/io/avroio_test.py | 32 ++- sdks/python/apache_beam/pvalue.py | 3 + sdks/python/apache_beam/typehints/schemas.py | 4 + 4 files changed, 282 insertions(+), 28 deletions(-) diff --git a/sdks/python/apache_beam/io/avroio.py b/sdks/python/apache_beam/io/avroio.py index d86f59e3a411..9225acf346e4 100644 --- a/sdks/python/apache_beam/io/avroio.py +++ b/sdks/python/apache_beam/io/avroio.py @@ -45,7 +45,13 @@ # pytype: skip-file import os from functools import partial +from typing import Any +from typing import Callable +from typing import Dict +from typing import List +from typing import Union +import fastavro from fastavro.read import block_reader from fastavro.write import Writer @@ -54,8 +60,11 @@ from apache_beam.io import filebasedsource from apache_beam.io import iobase from apache_beam.io.filesystem import CompressionTypes +from apache_beam.io.filesystems import FileSystems from apache_beam.io.iobase import Read +from apache_beam.portability.api import schema_pb2 from apache_beam.transforms import PTransform +from apache_beam.typehints import schemas __all__ = [ 'ReadFromAvro', @@ -73,7 +82,8 @@ def __init__( file_pattern=None, min_bundle_size=0, validate=True, - use_fastavro=True): + use_fastavro=True, + as_rows=False): """Initializes :class:`ReadFromAvro`. Uses source :class:`~apache_beam.io._AvroSource` to read a set of Avro @@ -140,13 +150,26 @@ def __init__( creation time. use_fastavro (bool): This flag is left for API backwards compatibility and no longer has an effect. Do not use. + as_rows (bool): Whether to return a schema'd PCollection of Beam rows. """ super().__init__() - self._source = _create_avro_source( + self._source = _FastAvroSource( file_pattern, min_bundle_size, validate=validate) + if as_rows: + path = FileSystems.match([file_pattern], [1])[0].metadata_list[0].path + with FileSystems.open(path) as fin: + avro_schema = fastavro.reader(fin).writer_schema + beam_schema = avro_schema_to_beam_schema(avro_schema) + self._post_process = avro_dict_to_beam_row(avro_schema, beam_schema) + else: + self._post_process = None def expand(self, pvalue): - return pvalue.pipeline | Read(self._source) + records = pvalue.pipeline | Read(self._source) + if self._post_process: + return records | beam.Map(self._post_process) + else: + return records def display_data(self): return {'source_dd': self._source} @@ -184,8 +207,7 @@ def __init__( name and the value being the actual data. If False, it only returns the data. """ - source_from_file = partial( - _create_avro_source, min_bundle_size=min_bundle_size) + source_from_file = partial(_FastAvroSource, min_bundle_size=min_bundle_size) self._read_all_files = filebasedsource.ReadAllFiles( True, CompressionTypes.AUTO, @@ -280,15 +302,6 @@ def advance_file_past_next_sync_marker(f, sync_marker): data = f.read(buf_size) -def _create_avro_source(file_pattern=None, min_bundle_size=0, validate=False): - return \ - _FastAvroSource( - file_pattern=file_pattern, - min_bundle_size=min_bundle_size, - validate=validate - ) - - class _FastAvroSource(filebasedsource.FileBasedSource): """A source for reading Avro files using the `fastavro` library. @@ -338,12 +351,15 @@ def split_points_unclaimed(stop_position): yield record +_create_avro_source = _FastAvroSource + + class WriteToAvro(beam.transforms.PTransform): """A ``PTransform`` for writing avro files.""" def __init__( self, file_path_prefix, - schema, + schema=None, codec='deflate', file_name_suffix='', num_shards=0, @@ -382,9 +398,10 @@ def __init__( Returns: A WriteToAvro transform usable for writing. """ - self._sink = _create_avro_sink( + self._schema = schema + self._sink_provider = lambda avro_schema: _create_avro_sink( file_path_prefix, - schema, + avro_schema, codec, file_name_suffix, num_shards, @@ -392,7 +409,21 @@ def __init__( mime_type) def expand(self, pcoll): - return pcoll | beam.io.iobase.Write(self._sink) + if self._schema: + avro_schema = self._schema + records = pcoll + else: + try: + beam_schema = schemas.schema_from_element_type(pcoll.element_type) + except TypeError as exn: + raise ValueError( + "An explicit schema is required to write non-schema'd PCollections." + ) from exn + avro_schema = beam_schema_to_avro_schema(beam_schema) + records = pcoll | beam.Map( + beam_row_to_avro_dict(avro_schema, beam_schema)) + self._sink = self._sink_provider(avro_schema) + return records | beam.io.iobase.Write(self._sink) def display_data(self): return {'sink_dd': self._sink} @@ -406,7 +437,7 @@ def _create_avro_sink( num_shards, shard_name_template, mime_type): - if "class \'avro.schema" in str(type(schema)): + if "class 'avro.schema" in str(type(schema)): raise ValueError( 'You are using Avro IO with fastavro (default with Beam on ' 'Python 3), but supplying a schema parsed by avro-python3. ' @@ -483,3 +514,205 @@ def write_record(self, writer, value): def close(self, writer): writer.flush() self.file_handle.close() + + +AVRO_PRIMITIVES_TO_BEAM_PRIMITIVES = { + 'boolean': schema_pb2.BOOLEAN, + 'int': schema_pb2.INT32, + 'long': schema_pb2.INT64, + 'float': schema_pb2.FLOAT, + 'double': schema_pb2.DOUBLE, + 'bytes': schema_pb2.BYTES, + 'string': schema_pb2.STRING, +} + +BEAM_PRIMITIVES_TO_AVRO_PRIMITIVES = { + v: k + for k, v in AVRO_PRIMITIVES_TO_BEAM_PRIMITIVES.items() +} + +_AvroSchemaType = Union[str, List, Dict] + + +def avro_type_to_beam_type(avro_type: _AvroSchemaType) -> schema_pb2.FieldType: + if isinstance(avro_type, str): + return avro_type_to_beam_type({'type': avro_type}) + elif isinstance(avro_type, list): + # Union type + return schemas.typing_to_runner_api(Any) + type_name = avro_type['type'] + if type_name in AVRO_PRIMITIVES_TO_BEAM_PRIMITIVES: + return schema_pb2.FieldType( + atomic_type=AVRO_PRIMITIVES_TO_BEAM_PRIMITIVES[type_name]) + elif type_name in ('fixed', 'enum'): + return schema_pb2.FieldType(atomic_type=schema_pb2.STRING) + elif type_name == 'array': + return schema_pb2.FieldType( + array_type=schema_pb2.ArrayType( + element_type=avro_type_to_beam_type(avro_type['items']))) + elif type_name == 'map': + return schema_pb2.FieldType( + map_type=schema_pb2.MapType( + key_type=schema_pb2.FieldType(atomic_type=schema_pb2.STRING), + value_type=avro_type_to_beam_type(avro_type['values']))) + elif type_name == 'record': + return schema_pb2.FieldType( + row_type=schema_pb2.RowType( + schema=schema_pb2.Schema( + fields=[ + schemas.schema_field( + f['name'], avro_type_to_beam_type(f['type'])) + for f in avro_type['fields'] + ]))) + else: + raise ValueError(f'Unable to convert {avro_type} to a Beam schema.') + + +def avro_schema_to_beam_schema( + avro_schema: _AvroSchemaType) -> schema_pb2.Schema: + beam_type = avro_type_to_beam_type(avro_schema) + if isinstance(avro_schema, dict) and avro_schema['type'] == 'record': + return beam_type.row_type.schema + else: + return schema_pb2.Schema(fields=[schemas.schema_field('record', beam_type)]) + + +def avro_dict_to_beam_row( + avro_schema: _AvroSchemaType, + beam_schema: schema_pb2.Schema) -> Callable[[Any], Any]: + if isinstance(avro_schema, str): + return avro_dict_to_beam_row({'type': avro_schema}) + if avro_schema['type'] == 'record': + to_row = avro_value_to_beam_value( + schema_pb2.FieldType(row_type=schema_pb2.RowType(schema=beam_schema))) + else: + + def to_row(record): + return beam.Row(record=record) + + return beam.typehints.with_output_types( + schemas.named_tuple_from_schema(beam_schema))( + to_row) + + +def avro_value_to_beam_value( + beam_type: schema_pb2.FieldType) -> Callable[[Any], Any]: + type_info = beam_type.WhichOneof("type_info") + if type_info == "atomic_type": + return lambda value: value + elif type_info == "array_type": + element_converter = avro_value_to_beam_value( + beam_type.array_type.element_type) + return lambda value: [element_converter(e) for e in value] + elif type_info == "iterable_type": + element_converter = avro_value_to_beam_value( + beam_type.iterable_type.element_type) + return lambda value: [element_converter(e) for e in value] + elif type_info == "map_type": + if beam_type.map_type.key_type.atomic_type != schema_pb2.STRING: + raise TypeError( + f'Only strings allowd as map keys when converting from AVRO, ' + f'found {beam_type}') + value_converter = avro_value_to_beam_value(beam_type.map_type.value_type) + return lambda value: {k: value_converter(v) for (k, v) in value.items()} + elif type_info == "row_type": + converters = { + field.name: avro_value_to_beam_value(field.type) + for field in beam_type.row_type.schema.fields + } + return lambda value: beam.Row( + ** + {name: convert(value[name]) + for (name, convert) in converters.items()}) + elif type_info == "logical_type": + return lambda value: value + else: + raise ValueError(f"Unrecognized type_info: {type_info!r}") + + +def beam_schema_to_avro_schema( + beam_schema: schema_pb2.Schema) -> _AvroSchemaType: + return beam_type_to_avro_type( + schema_pb2.FieldType(row_type=schema_pb2.RowType(schema=beam_schema))) + + +def beam_type_to_avro_type(beam_type: schema_pb2.FieldType) -> _AvroSchemaType: + type_info = beam_type.WhichOneof("type_info") + if type_info == "atomic_type": + return {'type': BEAM_PRIMITIVES_TO_AVRO_PRIMITIVES[beam_type.atomic_type]} + elif type_info == "array_type": + return { + 'type': 'array', + 'items': beam_type_to_avro_type(beam_type.array_type.element_type) + } + elif type_info == "iterable_type": + return { + 'type': 'array', + 'items': beam_type_to_avro_type(beam_type.iterable_type.element_type) + } + elif type_info == "map_type": + if beam_type.map_type.key_type.atomic_type != schema_pb2.STRING: + raise TypeError( + f'Only strings allowd as map keys when converting to AVRO, ' + f'found {beam_type}') + return { + 'type': 'map', + 'values': beam_type_to_avro_type(beam_type.map_type.element_type) + } + elif type_info == "row_type": + return { + 'type': 'record', + 'name': beam_type.row_type.schema.id, + 'fields': [{ + 'name': field.name, 'type': beam_type_to_avro_type(field.type) + } for field in beam_type.row_type.schema.fields], + } + else: + raise ValueError(f"Unconvertale type: {beam_type}") + + +def beam_row_to_avro_dict( + avro_schema: _AvroSchemaType, beam_schema: schema_pb2.Schema): + if isinstance(avro_schema, str): + return beam_row_to_avro_dict({'type': avro_schema}, beam_schema) + if avro_schema['type'] == 'record': + return beam_value_to_avro_value( + schema_pb2.FieldType(row_type=schema_pb2.RowType(schema=beam_schema))) + else: + convert = beam_value_to_avro_value(beam_schema) + return lambda row: convert(row[0]) + + +def beam_value_to_avro_value( + beam_type: schema_pb2.FieldType) -> Callable[[Any], Any]: + type_info = beam_type.WhichOneof("type_info") + if type_info == "atomic_type": + return lambda value: value + elif type_info == "array_type": + element_converter = avro_value_to_beam_value( + beam_type.array_type.element_type) + return lambda value: [element_converter(e) for e in value] + elif type_info == "iterable_type": + element_converter = avro_value_to_beam_value( + beam_type.iterable_type.element_type) + return lambda value: [element_converter(e) for e in value] + elif type_info == "map_type": + if beam_type.map_type.key_type.atomic_type != schema_pb2.STRING: + raise TypeError( + f'Only strings allowd as map keys when converting from AVRO, ' + f'found {beam_type}') + value_converter = avro_value_to_beam_value(beam_type.map_type.value_type) + return lambda value: {k: value_converter(v) for (k, v) in value.items()} + elif type_info == "row_type": + converters = { + field.name: avro_value_to_beam_value(field.type) + for field in beam_type.row_type.schema.fields + } + return lambda value: { + name: convert(getattr(value, name)) + for (name, convert) in converters.items() + } + elif type_info == "logical_type": + return lambda value: value + else: + raise ValueError(f"Unrecognized type_info: {type_info!r}") diff --git a/sdks/python/apache_beam/io/avroio_test.py b/sdks/python/apache_beam/io/avroio_test.py index ba35cf5846c0..68f5226e64ce 100644 --- a/sdks/python/apache_beam/io/avroio_test.py +++ b/sdks/python/apache_beam/io/avroio_test.py @@ -35,8 +35,8 @@ from apache_beam.io import filebasedsource from apache_beam.io import iobase from apache_beam.io import source_test_utils -from apache_beam.io.avroio import _create_avro_sink # For testing -from apache_beam.io.avroio import _create_avro_source # For testing +from apache_beam.io.avroio import _FastAvroSource # For testing +from apache_beam.io.avroio import _FastAvroSource # For testing from apache_beam.io.filesystems import FileSystems from apache_beam.testing.test_pipeline import TestPipeline from apache_beam.testing.util import assert_that @@ -125,7 +125,7 @@ def _write_pattern(self, num_files, return_filenames=False): def _run_avro_test( self, pattern, desired_bundle_size, perform_splitting, expected_result): - source = _create_avro_source(pattern) + source = _FastAvroSource(pattern) if perform_splitting: assert desired_bundle_size @@ -146,6 +146,20 @@ def _run_avro_test( read_records = source_test_utils.read_from_source(source, None, None) self.assertCountEqual(expected_result, read_records) + def test_schema_read_write(self): + with tempfile.TemporaryDirectory() as tmp_dirname: + path = os.path.join(tmp_dirname, 'tmp_filename') + rows = [beam.Row(a=1, b=['x', 'y']), beam.Row(a=2, b=['t', 'u'])] + stable_repr = lambda row: json.dumps(row._asdict()) + with TestPipeline() as p: + _ = p | Create(rows) | avroio.WriteToAvro(path) | beam.Map(print) + with TestPipeline() as p: + readback = ( + p + | avroio.ReadFromAvro(path + '*', as_rows=True) + | beam.Map(stable_repr)) + assert_that(readback, equal_to([stable_repr(r) for r in rows])) + def test_read_without_splitting(self): file_name = self._write_data() expected_result = self.RECORDS @@ -159,7 +173,7 @@ def test_read_with_splitting(self): def test_source_display_data(self): file_name = 'some_avro_source' source = \ - _create_avro_source( + _FastAvroSource( file_name, validate=False, ) @@ -220,12 +234,12 @@ def test_write_display_data(self): def test_read_reentrant_without_splitting(self): file_name = self._write_data() - source = _create_avro_source(file_name) + source = _FastAvroSource(file_name) source_test_utils.assert_reentrant_reads_succeed((source, None, None)) def test_read_reantrant_with_splitting(self): file_name = self._write_data() - source = _create_avro_source(file_name) + source = _FastAvroSource(file_name) splits = [split for split in source.split(desired_bundle_size=100000)] assert len(splits) == 1 source_test_utils.assert_reentrant_reads_succeed( @@ -246,7 +260,7 @@ def test_split_points(self): sync_interval = 16000 file_name = self._write_data(count=num_records, sync_interval=sync_interval) - source = _create_avro_source(file_name) + source = _FastAvroSource(file_name) splits = [split for split in source.split(desired_bundle_size=float('inf'))] assert len(splits) == 1 @@ -306,7 +320,7 @@ def test_read_with_splitting_pattern(self): def test_dynamic_work_rebalancing_exhaustive(self): def compare_split_points(file_name): - source = _create_avro_source(file_name) + source = _FastAvroSource(file_name) splits = [ split for split in source.split(desired_bundle_size=float('inf')) ] @@ -334,7 +348,7 @@ def test_corrupted_file(self): f.write(corrupted_data) corrupted_file_name = f.name - source = _create_avro_source(corrupted_file_name) + source = _FastAvroSource(corrupted_file_name) with self.assertRaisesRegex(ValueError, r'expected sync marker'): source_test_utils.read_from_source(source, None, None) diff --git a/sdks/python/apache_beam/pvalue.py b/sdks/python/apache_beam/pvalue.py index 2e86c9eb51c7..90882651d0b2 100644 --- a/sdks/python/apache_beam/pvalue.py +++ b/sdks/python/apache_beam/pvalue.py @@ -673,6 +673,9 @@ def __init__(self, **kwargs): def as_dict(self): return dict(self.__dict__) + # For compatibility with named tuples. + _asdict = as_dict + def __iter__(self): for _, value in self.__dict__.items(): yield value diff --git a/sdks/python/apache_beam/typehints/schemas.py b/sdks/python/apache_beam/typehints/schemas.py index 229a8af20bb6..ea836430e8e2 100644 --- a/sdks/python/apache_beam/typehints/schemas.py +++ b/sdks/python/apache_beam/typehints/schemas.py @@ -93,6 +93,7 @@ from apache_beam.typehints.native_type_compatibility import _match_is_exactly_mapping from apache_beam.typehints.native_type_compatibility import _match_is_optional from apache_beam.typehints.native_type_compatibility import _safe_issubclass +from apache_beam.typehints.native_type_compatibility import convert_to_typing_type from apache_beam.typehints.native_type_compatibility import extract_optional_type from apache_beam.typehints.native_type_compatibility import match_is_named_tuple from apache_beam.typehints.schema_registry import SCHEMA_REGISTRY @@ -284,6 +285,9 @@ def typing_to_runner_api(self, type_: type) -> schema_pb2.FieldType: if row_type_constraint is not None: return self.typing_to_runner_api(row_type_constraint) + if isinstance(type_, typehints.TypeConstraint): + type_ = convert_to_typing_type(type_) + # All concrete types (other than NamedTuple sub-classes) should map to # a supported primitive type. if type_ in PRIMITIVE_TO_ATOMIC_TYPE: From 714d3a9aa047261359a3f599fe1881d12c6d830e Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Thu, 5 Oct 2023 15:38:19 -0700 Subject: [PATCH 041/435] Add yaml providers for avro. --- sdks/python/apache_beam/yaml/standard_io.yaml | 11 +++++++++++ sdks/python/apache_beam/yaml/yaml_provider.py | 6 +++++- 2 files changed, 16 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/yaml/standard_io.yaml b/sdks/python/apache_beam/yaml/standard_io.yaml index 9ad4f53ba1f6..1d6aa5548a82 100644 --- a/sdks/python/apache_beam/yaml/standard_io.yaml +++ b/sdks/python/apache_beam/yaml/standard_io.yaml @@ -67,6 +67,8 @@ 'WriteToCsv': 'WriteToCsv' 'ReadFromJson': 'ReadFromJson' 'WriteToJson': 'WriteToJson' + 'ReadFromAvro': 'ReadFromAvro' + 'WriteToAvro': 'WriteToAvro' config: mappings: 'ReadFromCsv': @@ -77,6 +79,13 @@ path: 'path' 'WriteToJson': path: 'path' + 'ReadFromAvro': + path: 'file_pattern' + 'WriteToAvro': + path: 'file_path_prefix' + defaults: + 'ReadFromAvro': + as_rows: True underlying_provider: type: python transforms: @@ -84,3 +93,5 @@ 'WriteToCsv': 'apache_beam.io.WriteToCsv' 'ReadFromJson': 'apache_beam.io.ReadFromJson' 'WriteToJson': 'apache_beam.io.WriteToJson' + 'ReadFromAvro': 'apache_beam.io.ReadFromAvro' + 'WriteToAvro': 'apache_beam.io.WriteToAvro' diff --git a/sdks/python/apache_beam/yaml/yaml_provider.py b/sdks/python/apache_beam/yaml/yaml_provider.py index 3a88f6074b37..20d4b096916d 100644 --- a/sdks/python/apache_beam/yaml/yaml_provider.py +++ b/sdks/python/apache_beam/yaml/yaml_provider.py @@ -684,7 +684,7 @@ def __exit__(self, *args): @ExternalProvider.register_provider_type('renaming') class RenamingProvider(Provider): - def __init__(self, transforms, mappings, underlying_provider): + def __init__(self, transforms, mappings, underlying_provider, defaults=None): if isinstance(underlying_provider, dict): underlying_provider = ExternalProvider.provider_from_spec( underlying_provider) @@ -694,6 +694,7 @@ def __init__(self, transforms, mappings, underlying_provider): if transform not in mappings: raise ValueError(f'Missing transform {transform} in mappings.') self._mappings = mappings + self._defaults = defaults or {} def available(self) -> bool: return self._underlying_provider.available() @@ -731,6 +732,9 @@ def create_transform( mappings.get(key, key): value for key, value in args.items() } + for key, value in self._defaults.get(typ, {}).items(): + if key not in remapped_args: + remapped_args[key] = value return self._underlying_provider.create_transform( self._transforms[typ], remapped_args, yaml_create_transform) From 784e9278b15cab1885fcfc86dfae38516796381d Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Thu, 5 Oct 2023 17:20:34 -0700 Subject: [PATCH 042/435] guava --- .../sdk/io/gcp/pubsub/PubsubWriteSchemaTransformProvider.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubWriteSchemaTransformProvider.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubWriteSchemaTransformProvider.java index 3f53a0701829..e1c927930626 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubWriteSchemaTransformProvider.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubWriteSchemaTransformProvider.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.io.gcp.pubsub; import com.google.auto.service.AutoService; -import com.google.common.collect.ImmutableSet; import java.io.Serializable; import java.nio.charset.StandardCharsets; import java.util.Collections; @@ -43,6 +42,7 @@ import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Strings; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Sets; import org.checkerframework.checker.initialization.qual.Initialized; import org.checkerframework.checker.nullness.qual.NonNull; From 07e26fd014d8c8a811abf12d354945b7865a75cb Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Thu, 5 Oct 2023 17:33:34 -0700 Subject: [PATCH 043/435] test,lint --- sdks/python/apache_beam/io/avroio_test.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/io/avroio_test.py b/sdks/python/apache_beam/io/avroio_test.py index 68f5226e64ce..c54ac40711b1 100644 --- a/sdks/python/apache_beam/io/avroio_test.py +++ b/sdks/python/apache_beam/io/avroio_test.py @@ -36,7 +36,7 @@ from apache_beam.io import iobase from apache_beam.io import source_test_utils from apache_beam.io.avroio import _FastAvroSource # For testing -from apache_beam.io.avroio import _FastAvroSource # For testing +from apache_beam.io.avroio import _create_avro_sink # For testing from apache_beam.io.filesystems import FileSystems from apache_beam.testing.test_pipeline import TestPipeline from apache_beam.testing.util import assert_that @@ -221,6 +221,7 @@ def test_sink_display_data(self): def test_write_display_data(self): file_name = 'some_avro_sink' write = avroio.WriteToAvro(file_name, self.SCHEMA) + write.expand(beam.PCollection(beam.Pipeline())) dd = DisplayData.create_from(write) expected_items = [ DisplayDataItemMatcher('schema', str(self.SCHEMA)), From 3405b31d0400a8bf22cd96f50f0a7121b11f8276 Mon Sep 17 00:00:00 2001 From: magicgoody <131876064+magicgoody@users.noreply.github.com> Date: Fri, 6 Oct 2023 08:26:13 +0600 Subject: [PATCH 044/435] Github Workflow Replacement for Jenkins Jobs, beam_LoadTests_Python_Combine_Flink_* (#28846) * beam_LoadTests_Python_Combine_Flink * fixed commented info --- ...m_LoadTests_Python_Combine_Flink_Batch.yml | 140 ++++++++++++++++++ ...adTests_Python_Combine_Flink_Streaming.yml | 126 ++++++++++++++++ ...ombine_Flink_Batch_2GB_10_byte_records.txt | 27 ++++ ...ython_Combine_Flink_Batch_2GB_Fanout_4.txt | 28 ++++ ...ython_Combine_Flink_Batch_2GB_Fanout_8.txt | 28 ++++ ...ne_Flink_Streaming_2GB_10_byte_records.txt | 29 ++++ ...n_Combine_Flink_Streaming_2GB_Fanout_4.txt | 30 ++++ ...n_Combine_Flink_Streaming_2GB_Fanout_8.txt | 30 ++++ 8 files changed, 438 insertions(+) create mode 100644 .github/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml create mode 100644 .github/workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml create mode 100644 .github/workflows/load-tests-job-configs/python_Combine_Flink_Batch_2GB_10_byte_records.txt create mode 100644 .github/workflows/load-tests-job-configs/python_Combine_Flink_Batch_2GB_Fanout_4.txt create mode 100644 .github/workflows/load-tests-job-configs/python_Combine_Flink_Batch_2GB_Fanout_8.txt create mode 100644 .github/workflows/load-tests-job-configs/python_Combine_Flink_Streaming_2GB_10_byte_records.txt create mode 100644 .github/workflows/load-tests-job-configs/python_Combine_Flink_Streaming_2GB_Fanout_4.txt create mode 100644 .github/workflows/load-tests-job-configs/python_Combine_Flink_Streaming_2GB_Fanout_8.txt diff --git a/.github/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml new file mode 100644 index 000000000000..165b23703562 --- /dev/null +++ b/.github/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml @@ -0,0 +1,140 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: LoadTests Python Combine Flink Batch + +on: + issue_comment: + types: [created] + schedule: + - cron: '10 6 * * *' + workflow_dispatch: + +# Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + GCLOUD_ZONE: us-central1-a + CLUSTER_NAME: beam-loadtests-py-cmb-flink-batch-${{ github.run_id }} + GCS_BUCKET: gs://beam-flink-cluster + FLINK_DOWNLOAD_URL: https://archive.apache.org/dist/flink/flink-1.15.0/flink-1.15.0-bin-scala_2.12.tgz + HADOOP_DOWNLOAD_URL: https://repo.maven.apache.org/maven2/org/apache/flink/flink-shaded-hadoop-2-uber/2.8.3-10.0/flink-shaded-hadoop-2-uber-2.8.3-10.0.jar + FLINK_TASKMANAGER_SLOTS: 1 + DETACHED_MODE: true + HARNESS_IMAGES_TO_PULL: gcr.io/apache-beam-testing/beam-sdk/beam_go_sdk:latest + JOB_SERVER_IMAGE: gcr.io/apache-beam-testing/beam_portability/beam_flink1.15_job_server:latest + ARTIFACTS_DIR: gs://beam-flink-cluster/beam-loadtests-py-cmb-flink-batch-${{ github.run_id }} + +jobs: + beam_LoadTests_Python_Combine_Flink_Batch: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Load Tests Python Combine Flink Batch' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 720 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_LoadTests_Python_Combine_Flink_Batch"] + job_phrase: ["Run Load Tests Python Combine Flink Batch"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action + with: + python-version: 3.8 + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: python + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_Combine_Flink_Batch_2GB_10_byte_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_Combine_Flink_Batch_2GB_Fanout_4.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_Combine_Flink_Batch_2GB_Fanout_8.txt + - name: Start Flink with parallelism 5 + env: + FLINK_NUM_WORKERS: 5 + run: | + cd ${{ github.workspace }}/.test-infra/dataproc; ./flink_cluster.sh create + - name: get current time + run: echo "NOW_UTC=$(date '+%m%d%H%M%S' --utc)" >> $GITHUB_ENV + # The env variables are created and populated in the test-arguments-action as "_test_arguments_" + - name: run Load test 2GB 10 byte records + env: + CLOUDSDK_CONFIG: ${{ env.KUBELET_GCLOUD_CONFIG_PATH}} + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PpythonVersion=3.8 \ + -PloadTest.mainClass=apache_beam.testing.load_tests.combine_test \ + -Prunner=PortableRunner \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_Combine_Flink_Batch_test_arguments_1 }} --job_name=load-tests-python-flink-batch-combine-1-${{env.NOW_UTC}}' \ + - name: Restart Flink with parallelism 16 + env: + FLINK_NUM_WORKERS: 16 + run: | + cd ${{ github.workspace }}/.test-infra/dataproc; ./flink_cluster.sh restart + - name: run Load test 2GB Fanout 4 + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PpythonVersion=3.8 \ + -PloadTest.mainClass=apache_beam.testing.load_tests.combine_test \ + -Prunner=PortableRunner \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_Combine_Flink_Batch_test_arguments_2 }} --job_name=load-tests-python-flink-batch-combine-4-${{env.NOW_UTC}}' \ + - name: run Load test 2GB Fanout 8 + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PpythonVersion=3.8 \ + -PloadTest.mainClass=apache_beam.testing.load_tests.combine_test \ + -Prunner=PortableRunner \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_Combine_Flink_Batch_test_arguments_3 }} --job_name=load-tests-python-flink-batch-combine-5-${{env.NOW_UTC}}' \ + - name: Teardown Flink + if: always() + run: | + ${{ github.workspace }}/.test-infra/dataproc/flink_cluster.sh delete \ No newline at end of file diff --git a/.github/workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml b/.github/workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml new file mode 100644 index 000000000000..1dae586de21a --- /dev/null +++ b/.github/workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml @@ -0,0 +1,126 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: LoadTests Python Combine Flink Streaming + +on: + issue_comment: + types: [created] + schedule: + - cron: '10 7 * * *' + workflow_dispatch: + +# Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + GCLOUD_ZONE: us-central1-a + CLUSTER_NAME: beam-loadtests-py-cmb-flink-streaming-${{ github.run_id }} + GCS_BUCKET: gs://beam-flink-cluster + FLINK_DOWNLOAD_URL: https://archive.apache.org/dist/flink/flink-1.15.0/flink-1.15.0-bin-scala_2.12.tgz + HADOOP_DOWNLOAD_URL: https://repo.maven.apache.org/maven2/org/apache/flink/flink-shaded-hadoop-2-uber/2.8.3-10.0/flink-shaded-hadoop-2-uber-2.8.3-10.0.jar + FLINK_TASKMANAGER_SLOTS: 1 + DETACHED_MODE: true + HARNESS_IMAGES_TO_PULL: gcr.io/apache-beam-testing/beam-sdk/beam_go_sdk:latest + JOB_SERVER_IMAGE: gcr.io/apache-beam-testing/beam_portability/beam_flink1.15_job_server:latest + ARTIFACTS_DIR: gs://beam-flink-cluster/beam-loadtests-py-cmb-flink-streaming-${{ github.run_id }} + +jobs: + beam_LoadTests_Python_Combine_Flink_Streaming: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Load Tests Python Combine Flink Streaming' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 720 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_LoadTests_Python_Combine_Flink_Streaming"] + job_phrase: ["Run Load Tests Python Combine Flink Streaming"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action + with: + python-version: 3.8 + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: python + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_Combine_Flink_Streaming_2GB_Fanout_4.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_Combine_Flink_Streaming_2GB_Fanout_8.txt + - name: Start Flink with parallelism 16 + env: + FLINK_NUM_WORKERS: 16 + run: | + cd ${{ github.workspace }}/.test-infra/dataproc; ./flink_cluster.sh create + - name: get current time + run: echo "NOW_UTC=$(date '+%m%d%H%M%S' --utc)" >> $GITHUB_ENV + # The env variables are created and populated in the test-arguments-action as "_test_arguments_" + - name: run Load test 2GB Fanout 4 + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PpythonVersion=3.8 \ + -PloadTest.mainClass=apache_beam.testing.load_tests.combine_test \ + -Prunner=PortableRunner \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_Combine_Flink_Streaming_test_arguments_1 }} --job_name=load-tests-python-flink-streaming-combine-4-${{env.NOW_UTC}}' \ + - name: run Load test 2GB Fanout 8 + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PpythonVersion=3.8 \ + -PloadTest.mainClass=apache_beam.testing.load_tests.combine_test \ + -Prunner=PortableRunner \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_Combine_Flink_Streaming_test_arguments_2 }} --job_name=load-tests-python-flink-streaming-combine-5-${{env.NOW_UTC}}' \ + - name: Teardown Flink + if: always() + run: | + ${{ github.workspace }}/.test-infra/dataproc/flink_cluster.sh delete + + # // TODO(https://github.com/apache/beam/issues/20402). Skipping some cases because they are too slow: + # load-tests-python-flink-streaming-combine-1' \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_Combine_Flink_Batch_2GB_10_byte_records.txt b/.github/workflows/load-tests-job-configs/python_Combine_Flink_Batch_2GB_10_byte_records.txt new file mode 100644 index 000000000000..8295d1c8aa86 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_Combine_Flink_Batch_2GB_10_byte_records.txt @@ -0,0 +1,27 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_flink_batch_combine_1 +--influx_measurement=python_batch_combine_1 +--input_options=''{\\"num_records\\":200000000,\\"key_size\\":1,\\"value_size\\":9,\\"algorithm\\":\\"lcg\\"}'' +--parallelism=5 +--job_endpoint=localhost:8099 +--environment_type=DOCKER +--environment_config=gcr.io/apache-beam-testing/beam-sdk/beam_python3.8_sdk:latest +--top_count=20 +--runner=PortableRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_Combine_Flink_Batch_2GB_Fanout_4.txt b/.github/workflows/load-tests-job-configs/python_Combine_Flink_Batch_2GB_Fanout_4.txt new file mode 100644 index 000000000000..82f8bcc7c0ae --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_Combine_Flink_Batch_2GB_Fanout_4.txt @@ -0,0 +1,28 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_flink_batch_combine_4 +--influx_measurement=python_batch_combine_4 +--input_options=''{\\"num_records\\":5000000,\\"key_size\\":10,\\"value_size\\":90,\\"algorithm\\":\\"lcg\\"}'' +--parallelism=16 +--job_endpoint=localhost:8099 +--environment_type=DOCKER +--environment_config=gcr.io/apache-beam-testing/beam-sdk/beam_python3.8_sdk:latest +--fanout=4 +--top_count=20 +--runner=PortableRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_Combine_Flink_Batch_2GB_Fanout_8.txt b/.github/workflows/load-tests-job-configs/python_Combine_Flink_Batch_2GB_Fanout_8.txt new file mode 100644 index 000000000000..45425b6bf153 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_Combine_Flink_Batch_2GB_Fanout_8.txt @@ -0,0 +1,28 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_flink_batch_combine_5 +--influx_measurement=python_batch_combine_5 +--input_options=''{\\"num_records\\":2500000,\\"key_size\\":10,\\"value_size\\":90,\\"algorithm\\":\\"lcg\\"}'' +--parallelism=16 +--job_endpoint=localhost:8099 +--environment_type=DOCKER +--environment_config=gcr.io/apache-beam-testing/beam-sdk/beam_python3.8_sdk:latest +--fanout=8 +--top_count=20 +--runner=PortableRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_Combine_Flink_Streaming_2GB_10_byte_records.txt b/.github/workflows/load-tests-job-configs/python_Combine_Flink_Streaming_2GB_10_byte_records.txt new file mode 100644 index 000000000000..12ffc1790e46 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_Combine_Flink_Streaming_2GB_10_byte_records.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_flink_streaming_combine_1 +--influx_measurement=python_streaming_combine_1 +--input_options=''{\\"num_records\\":200000000,\\"key_size\\":1,\\"value_size\\":9,\\"algorithm\\":\\"lcg\\"}'' +--parallelism=5 +--job_endpoint=localhost:8099 +--environment_type=DOCKER +--environment_config=gcr.io/apache-beam-testing/beam-sdk/beam_python3.8_sdk:latest +--top_count=20 +--streaming +--use_stateful_load_generator +--runner=PortableRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_Combine_Flink_Streaming_2GB_Fanout_4.txt b/.github/workflows/load-tests-job-configs/python_Combine_Flink_Streaming_2GB_Fanout_4.txt new file mode 100644 index 000000000000..c7d5552a03bd --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_Combine_Flink_Streaming_2GB_Fanout_4.txt @@ -0,0 +1,30 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_flink_streaming_combine_4 +--influx_measurement=python_streaming_combine_4 +--input_options=''{\\"num_records\\":5000000,\\"key_size\\":10,\\"value_size\\":90,\\"algorithm\\":\\"lcg\\"}'' +--parallelism=16 +--job_endpoint=localhost:8099 +--environment_type=DOCKER +--environment_config=gcr.io/apache-beam-testing/beam-sdk/beam_python3.8_sdk:latest +--fanout=4 +--top_count=20 +--streaming +--use_stateful_load_generator +--runner=PortableRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_Combine_Flink_Streaming_2GB_Fanout_8.txt b/.github/workflows/load-tests-job-configs/python_Combine_Flink_Streaming_2GB_Fanout_8.txt new file mode 100644 index 000000000000..bffdeab2cb11 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_Combine_Flink_Streaming_2GB_Fanout_8.txt @@ -0,0 +1,30 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_flink_streaming_combine_5 +--influx_measurement=python_streaming_combine_5 +--input_options=''{\\"num_records\\":2500000,\\"key_size\\":10,\\"value_size\\":90,\\"algorithm\\":\\"lcg\\"}'' +--parallelism=16 +--job_endpoint=localhost:8099 +--environment_type=DOCKER +--environment_config=gcr.io/apache-beam-testing/beam-sdk/beam_python3.8_sdk:latest +--fanout=8 +--top_count=20 +--streaming +--use_stateful_load_generator +--runner=PortableRunner \ No newline at end of file From 03741c06e0850abf56a3437b036264625784b8ac Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 5 Oct 2023 22:12:14 -0700 Subject: [PATCH 045/435] Bump golang.org/x/net from 0.15.0 to 0.16.0 in /sdks (#28858) Bumps [golang.org/x/net](https://github.com/golang/net) from 0.15.0 to 0.16.0. - [Commits](https://github.com/golang/net/compare/v0.15.0...v0.16.0) --- updated-dependencies: - dependency-name: golang.org/x/net dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 6 +++--- sdks/go.sum | 12 ++++++------ 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index aa147a8dc682..fb883c93af19 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -52,10 +52,10 @@ require ( github.com/xitongsys/parquet-go v1.6.2 github.com/xitongsys/parquet-go-source v0.0.0-20220315005136-aec0fe3e777c go.mongodb.org/mongo-driver v1.12.1 - golang.org/x/net v0.15.0 + golang.org/x/net v0.16.0 golang.org/x/oauth2 v0.12.0 golang.org/x/sync v0.3.0 - golang.org/x/sys v0.12.0 + golang.org/x/sys v0.13.0 golang.org/x/text v0.13.0 google.golang.org/api v0.144.0 google.golang.org/genproto v0.0.0-20230913181813-007df8e322eb @@ -165,7 +165,7 @@ require ( github.com/youmark/pkcs8 v0.0.0-20181117223130-1be2e3e5546d // indirect github.com/zeebo/xxh3 v1.0.2 // indirect go.opencensus.io v0.24.0 // indirect - golang.org/x/crypto v0.13.0 // indirect + golang.org/x/crypto v0.14.0 // indirect golang.org/x/mod v0.11.0 // indirect golang.org/x/tools v0.10.0 // indirect golang.org/x/xerrors v0.0.0-20220907171357-04be3eba64a2 // indirect diff --git a/sdks/go.sum b/sdks/go.sum index 12c43b056f84..1fcb968e118f 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -485,8 +485,8 @@ golang.org/x/crypto v0.0.0-20201002170205-7f63de1d35b0/go.mod h1:LzIPMQfyMNhhGPh golang.org/x/crypto v0.0.0-20210513164829-c07d793c2f9a/go.mod h1:P+XmwS30IXTQdn5tA2iutPOUgjI07+tq3H3K9MVA1s8= golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/crypto v0.0.0-20220622213112-05595931fe9d/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= -golang.org/x/crypto v0.13.0 h1:mvySKfSWJ+UKUii46M40LOvyWfN0s2U+46/jDd0e6Ck= -golang.org/x/crypto v0.13.0/go.mod h1:y6Z2r+Rw4iayiXXAIxJIDAJ1zMW4yaTpebo8fPOliYc= +golang.org/x/crypto v0.14.0 h1:wBqGXzWJW6m1XrIKlAH0Hs1JJ7+9KBwnIO8v66Q9cHc= +golang.org/x/crypto v0.14.0/go.mod h1:MVFd36DqK4CsrnJYDkBA3VC4m2GkXAM0PvzMCn4JQf4= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190306152737-a1d7652674e8/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190510132918-efd6b22b2522/go.mod h1:ZjyILWgesfNpC6sMxTJOJm9Kp84zZh5NQWvqDGG3Qr8= @@ -545,8 +545,8 @@ golang.org/x/net v0.0.0-20201224014010-6772e930b67b/go.mod h1:m0MpNAwzfU5UDzcl9v golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= golang.org/x/net v0.0.0-20211112202133-69e39bad7dc2/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= -golang.org/x/net v0.15.0 h1:ugBLEUaxABaB5AJqW9enI0ACdci2RUd4eP51NTBvuJ8= -golang.org/x/net v0.15.0/go.mod h1:idbUs1IY1+zTqbi8yxTbhexhEEk5ur9LInksu6HrEpk= +golang.org/x/net v0.16.0 h1:7eBu7KsSvFDtSXUIDbh3aqlK4DPsZ1rByC8PFfBThos= +golang.org/x/net v0.16.0/go.mod h1:NxSsAGuq816PNPmqtQdLE42eU2Fs7NoRIZrHJAlaCOE= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= @@ -604,8 +604,8 @@ golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.2.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.10.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.12.0 h1:CM0HF96J0hcLAwsHPJZjfdNzs0gftsLfgKt57wWHJ0o= -golang.org/x/sys v0.12.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.13.0 h1:Af8nKPmuFypiUBjVoU9V20FiaFXOcuZI21p0ycVYYGE= +golang.org/x/sys v0.13.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= From b90e40308e223f9a1a37a9547ae4f745ef495ecb Mon Sep 17 00:00:00 2001 From: Vitaly Terentyev Date: Fri, 6 Oct 2023 15:01:40 +0400 Subject: [PATCH 046/435] Add Cleanup Dataproc Resources workflow (#28814) * Add Cleanup Dataproc Resources workflow * Refactoring --- .../beam_CleanUpDataprocResources.yml | 61 +++++++++++++++++++ 1 file changed, 61 insertions(+) create mode 100644 .github/workflows/beam_CleanUpDataprocResources.yml diff --git a/.github/workflows/beam_CleanUpDataprocResources.yml b/.github/workflows/beam_CleanUpDataprocResources.yml new file mode 100644 index 000000000000..b6081e1891e6 --- /dev/null +++ b/.github/workflows/beam_CleanUpDataprocResources.yml @@ -0,0 +1,61 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: Cleanup Dataproc Resources + +on: + schedule: + - cron: '0 */6 * * *' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_CleanUpDataprocResources: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 100 + name: "beam_CleanUpDataprocResources" + steps: + - uses: actions/checkout@v3 + - name: Delete leaked resources for all the jobs that generates flink clusters + run: | + cd ${{ github.workspace }}/.test-infra/dataproc; ./cleanup.sh -xe \ No newline at end of file From 6e90adb753eca968e36cee3a7fb6bf98db5c0080 Mon Sep 17 00:00:00 2001 From: Vitaly Terentyev Date: Fri, 6 Oct 2023 15:02:25 +0400 Subject: [PATCH 047/435] Add rotate credentials workflows (#28815) --- .../beam_IODatastoresCredentialsRotation.yml | 78 +++++++++++++++++++ .../beam_MetricsCredentialsRotation.yml | 78 +++++++++++++++++++ 2 files changed, 156 insertions(+) create mode 100644 .github/workflows/beam_IODatastoresCredentialsRotation.yml create mode 100644 .github/workflows/beam_MetricsCredentialsRotation.yml diff --git a/.github/workflows/beam_IODatastoresCredentialsRotation.yml b/.github/workflows/beam_IODatastoresCredentialsRotation.yml new file mode 100644 index 000000000000..36e6b238cdfc --- /dev/null +++ b/.github/workflows/beam_IODatastoresCredentialsRotation.yml @@ -0,0 +1,78 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: Rotate IO-Datastores Cluster Credentials + +on: + schedule: + - cron: '0 2 1 * *' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_IODatastoresCredentialsRotation: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 100 + name: ${{ matrix.job_name }} + strategy: + matrix: + job_name: ["beam_IODatastoresCredentialsRotation"] + job_phrase: ["N/A"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} + - name: Starting credential rotation + run: | + gcloud container clusters update io-datastores --start-credential-rotation --zone=us-central1-a --quiet + - name: Rebuilding the nodes + run: | + gcloud container clusters upgrade io-datastores --node-pool=pool-1 --zone=us-central1-a --quiet + - name: Completing the rotation + run: | + gcloud container clusters update io-datastores --complete-credential-rotation --zone=us-central1-a --quiet +# TODO: Send email to dev@beam.apache.org if something went wrong during credentials rotation \ No newline at end of file diff --git a/.github/workflows/beam_MetricsCredentialsRotation.yml b/.github/workflows/beam_MetricsCredentialsRotation.yml new file mode 100644 index 000000000000..9bd795f0c2a4 --- /dev/null +++ b/.github/workflows/beam_MetricsCredentialsRotation.yml @@ -0,0 +1,78 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: Rotate Metrics Cluster Credentials + +on: + schedule: + - cron: '0 2 1 * *' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_MetricsCredentialsRotation: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 100 + name: ${{ matrix.job_name }} + strategy: + matrix: + job_name: ["beam_MetricsCredentialsRotation"] + job_phrase: ["N/A"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} + - name: Starting credential rotation + run: | + gcloud container clusters update metrics --start-credential-rotation --zone=us-central1-a --quiet + - name: Rebuilding the nodes + run: | + gcloud container clusters upgrade metrics --node-pool=default-pool --zone=us-central1-a --quiet + - name: Completing the rotation + run: | + gcloud container clusters update metrics --complete-credential-rotation --zone=us-central1-a --quiet +# TODO: Send email to dev@beam.apache.org if something went wrong during credentials rotation \ No newline at end of file From ad159eaabb6315ea3dacd14839a60e1e6f45cde8 Mon Sep 17 00:00:00 2001 From: Vitaly Terentyev Date: Fri, 6 Oct 2023 15:10:35 +0400 Subject: [PATCH 048/435] Add Publish Beam SDK Snapshots workflow (#28795) * Add Publish Beam SDK Snapshots workflow * Fix timeout * Do not use comment phrase trigger * Refactoring --- .../beam_Publish_Beam_SDK_Snapshots.yml | 88 +++++++++++++++++++ 1 file changed, 88 insertions(+) create mode 100644 .github/workflows/beam_Publish_Beam_SDK_Snapshots.yml diff --git a/.github/workflows/beam_Publish_Beam_SDK_Snapshots.yml b/.github/workflows/beam_Publish_Beam_SDK_Snapshots.yml new file mode 100644 index 000000000000..7aa5f6d4ceee --- /dev/null +++ b/.github/workflows/beam_Publish_Beam_SDK_Snapshots.yml @@ -0,0 +1,88 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: Publish Beam SDK Snapshots + +on: + schedule: + - cron: '0 */4 * * *' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_Publish_Beam_SDK_Snapshots: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 100 + name: ${{ matrix.job_name }} (${{ matrix.container_task }}) + strategy: + fail-fast: false + matrix: + job_name: ["beam_Publish_Beam_SDK_Snapshots"] + job_phrase: ["N/A"] + container_task: ["go:container", "java:container:java8", "java:container:java11", "java:container:java17", "python:container:py38", "python:container:py39", "python:container:py310", "python:container:py311"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.container_task }}) + - name: Setup Java environment + if: ${{ startsWith(matrix.container_task, 'java') }} + uses: ./.github/actions/setup-environment-action + with: + java-version: 11 + - name: Setup Python environment + if: ${{ startsWith(matrix.container_task, 'python') }} + uses: ./.github/actions/setup-environment-action + with: + python-version: '3.8' + - name: run Publish Beam SDK Snapshots script + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:${{ matrix.container_task }}:dockerTagPush + arguments: | + -Pjava11Home=$JAVA_HOME_11_X64 \ + -Pdocker-repository-root=gcr.io/apache-beam-testing/beam-sdk \ + -Pdocker-tag-list=${{ github.sha }},latest \ No newline at end of file From 7462069f8ced2491c3e5c90d2364c96ee002171b Mon Sep 17 00:00:00 2001 From: magicgoody <131876064+magicgoody@users.noreply.github.com> Date: Fri, 6 Oct 2023 19:17:53 +0600 Subject: [PATCH 049/435] Github Workflow Replacement for Jenkins Jobs, beam_LoadTests_Python_GBK_Dataflow_* (#28827) * beam_LoadTests_Python_GBK_Dataflow * fix permissions * removed unused arg * steps name fix * comment fixed, num_workers fixed --- ...am_LoadTests_Python_GBK_Dataflow_Batch.yml | 134 ++++++++++++++++++ ...oadTests_Python_GBK_Dataflow_Streaming.yml | 100 +++++++++++++ ...GBK_Dataflow_Batch_2GB_of_100B_records.txt | 28 ++++ ...BK_Dataflow_Batch_2GB_of_100kB_records.txt | 28 ++++ ..._GBK_Dataflow_Batch_2GB_of_10B_records.txt | 28 ++++ ...4_times_with_2GB_10-byte_records_total.txt | 28 ++++ ...8_times_with_2GB_10-byte_records_total.txt | 28 ++++ ...Dataflow_Streaming_2GB_of_100B_records.txt | 30 ++++ ...ataflow_Streaming_2GB_of_100kB_records.txt | 30 ++++ ..._Dataflow_Streaming_2GB_of_10B_records.txt | 30 ++++ ...4_times_with_2GB_10-byte_records_total.txt | 30 ++++ ...8_times_with_2GB_10-byte_records_total.txt | 30 ++++ 12 files changed, 524 insertions(+) create mode 100644 .github/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml create mode 100644 .github/workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml create mode 100644 .github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_2GB_of_100B_records.txt create mode 100644 .github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_2GB_of_100kB_records.txt create mode 100644 .github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_2GB_of_10B_records.txt create mode 100644 .github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_fanout_4_times_with_2GB_10-byte_records_total.txt create mode 100644 .github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_fanout_8_times_with_2GB_10-byte_records_total.txt create mode 100644 .github/workflows/load-tests-job-configs/python_GBK_Dataflow_Streaming_2GB_of_100B_records.txt create mode 100644 .github/workflows/load-tests-job-configs/python_GBK_Dataflow_Streaming_2GB_of_100kB_records.txt create mode 100644 .github/workflows/load-tests-job-configs/python_GBK_Dataflow_Streaming_2GB_of_10B_records.txt create mode 100644 .github/workflows/load-tests-job-configs/python_GBK_Dataflow_Streaming_fanout_4_times_with_2GB_10-byte_records_total.txt create mode 100644 .github/workflows/load-tests-job-configs/python_GBK_Dataflow_Streaming_fanout_8_times_with_2GB_10-byte_records_total.txt diff --git a/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml new file mode 100644 index 000000000000..fa3459992d0e --- /dev/null +++ b/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml @@ -0,0 +1,134 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: LoadTests Python GBK Dataflow Batch + +on: + issue_comment: + types: [created] + schedule: + - cron: '10 2 * * *' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_LoadTests_Python_GBK_Dataflow_Batch: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Load Tests Python GBK Dataflow Batch' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 720 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_LoadTests_Python_GBK_Dataflow_Batch"] + job_phrase: ["Run Load Tests Python GBK Dataflow Batch"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action + with: + python-version: 3.8 + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: python + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_2GB_of_10B_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_2GB_of_100B_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_2GB_of_100kB_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_fanout_4_times_with_2GB_10-byte_records_total.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_fanout_8_times_with_2GB_10-byte_records_total.txt + - name: get current time + run: echo "NOW_UTC=$(date '+%m%d%H%M%S' --utc)" >> $GITHUB_ENV + # The env variables are created and populated in the test-arguments-action as "_test_arguments_" + - name: run 2GB of 10B records test + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.group_by_key_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_GBK_Dataflow_Batch_test_arguments_1 }} --job_name=load-tests-python-dataflow-batch-gbk-1-${{env.NOW_UTC}}' \ + - name: run 2GB of 100B records test + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.group_by_key_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_GBK_Dataflow_Batch_test_arguments_2 }} --job_name=load-tests-python-dataflow-batch-gbk-2-${{env.NOW_UTC}}' \ + - name: run 2GB of 100kB records test + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.group_by_key_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_GBK_Dataflow_Batch_test_arguments_3 }} --job_name=load-tests-python-dataflow-batch-gbk-3-${{env.NOW_UTC}}' \ + - name: run fanout 4 times with 2GB 10-byte records test + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.group_by_key_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_GBK_Dataflow_Batch_test_arguments_4 }} --job_name=load-tests-python-dataflow-batch-gbk-4-${{env.NOW_UTC}}' \ + - name: run fanout 8 times with 2GB 10-byte records total test + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.group_by_key_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_GBK_Dataflow_Batch_test_arguments_5 }} --job_name=load-tests-python-dataflow-batch-gbk-5-${{env.NOW_UTC}}' \ No newline at end of file diff --git a/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml new file mode 100644 index 000000000000..da0cf8eefbb8 --- /dev/null +++ b/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml @@ -0,0 +1,100 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: LoadTests Python GBK Dataflow Streaming + +on: + issue_comment: + types: [created] + schedule: + - cron: '10 4 * * *' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_LoadTests_Python_GBK_Dataflow_Streaming: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Load Tests Python GBK Dataflow Streaming' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 720 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_LoadTests_Python_GBK_Dataflow_Streaming"] + job_phrase: ["Run Load Tests Python GBK Dataflow Streaming"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action + with: + python-version: 3.8 + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: python + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Streaming_2GB_of_100kB_records.txt + - name: get current time + run: echo "NOW_UTC=$(date '+%m%d%H%M%S' --utc)" >> $GITHUB_ENV + # The env variable is created and populated in the test-arguments-action as "_test_arguments_" + - name: run 2GB of 100kB records test + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.group_by_key_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_GBK_Dataflow_Streaming_test_arguments_1 }} --job_name=load-tests-python-dataflow-streaming-gbk-3-${{env.NOW_UTC}}' \ + + # // TODO(https://github.com/apache/beam/issues/20403). Skipping some cases because they are too slow: + # load-tests-python-dataflow-streaming-gbk-1 + # load-tests-python-dataflow-streaming-gbk-2 + # load-tests-python-dataflow-streaming-gbk-4 + # load-tests-python-dataflow-streaming-gbk-5 \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_2GB_of_100B_records.txt b/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_2GB_of_100B_records.txt new file mode 100644 index 000000000000..ad05bf1e85d3 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_2GB_of_100B_records.txt @@ -0,0 +1,28 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_batch_gbk_2 +--influx_measurement=python_batch_gbk_2 +--input_options=''{\\"num_records\\":20000000,\\"key_size\\":10,\\"value_size\\":90,\\"algorithm\\":\\"lcg\\"}'' +--iterations=1 +--fanout=1 +--num_workers=5 +--autoscaling_algorithm=NONE +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_2GB_of_100kB_records.txt b/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_2GB_of_100kB_records.txt new file mode 100644 index 000000000000..8d3358a12f98 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_2GB_of_100kB_records.txt @@ -0,0 +1,28 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_batch_gbk_3 +--influx_measurement=python_batch_gbk_3 +--input_options=''{\\"num_records\\":20000,\\"key_size\\":10000,\\"value_size\\":90000,\\"algorithm\\":\\"lcg\\"}'' +--iterations=1 +--fanout=1 +--num_workers=5 +--autoscaling_algorithm=NONE +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_2GB_of_10B_records.txt b/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_2GB_of_10B_records.txt new file mode 100644 index 000000000000..885c5ca61954 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_2GB_of_10B_records.txt @@ -0,0 +1,28 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_batch_gbk_1 +--influx_measurement=python_batch_gbk_1 +--input_options=''{\\"num_records\\":200000000,\\"key_size\\":1,\\"value_size\\":9,\\"algorithm\\":\\"lcg\\"}'' +--iterations=1 +--fanout=1 +--num_workers=5 +--autoscaling_algorithm=NONE +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_fanout_4_times_with_2GB_10-byte_records_total.txt b/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_fanout_4_times_with_2GB_10-byte_records_total.txt new file mode 100644 index 000000000000..1663e646f542 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_fanout_4_times_with_2GB_10-byte_records_total.txt @@ -0,0 +1,28 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_batch_gbk_4 +--influx_measurement=python_batch_gbk_4 +--input_options=''{\\"num_records\\":5000000,\\"key_size\\":10,\\"value_size\\":90,\\"algorithm\\":\\"lcg\\"}'' +--iterations=1 +--fanout=4 +--num_workers=16 +--autoscaling_algorithm=NONE +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_fanout_8_times_with_2GB_10-byte_records_total.txt b/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_fanout_8_times_with_2GB_10-byte_records_total.txt new file mode 100644 index 000000000000..4a1768c9d17d --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_fanout_8_times_with_2GB_10-byte_records_total.txt @@ -0,0 +1,28 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_batch_gbk_5 +--influx_measurement=python_batch_gbk_5 +--input_options=''{\\"num_records\\":2500000,\\"key_size\\":10,\\"value_size\\":90,\\"algorithm\\":\\"lcg\\"}'' +--iterations=1 +--fanout=8 +--num_workers=16 +--autoscaling_algorithm=NONE +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Streaming_2GB_of_100B_records.txt b/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Streaming_2GB_of_100B_records.txt new file mode 100644 index 000000000000..057f71d5627c --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Streaming_2GB_of_100B_records.txt @@ -0,0 +1,30 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_streaming_gbk_2 +--influx_measurement=python_streaming_gbk_2 +--input_options=''{\\"num_records\\":20000000,\\"key_size\\":10,\\"value_size\\":90,\\"algorithm\\":\\"lcg\\"}'' +--iterations=1 +--fanout=1 +--num_workers=5 +--autoscaling_algorithm=NONE +--streaming +--experiments=use_runner_v2 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Streaming_2GB_of_100kB_records.txt b/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Streaming_2GB_of_100kB_records.txt new file mode 100644 index 000000000000..57c1be11d592 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Streaming_2GB_of_100kB_records.txt @@ -0,0 +1,30 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_streaming_gbk_3 +--influx_measurement=python_streaming_gbk_3 +--input_options=''{\\"num_records\\":20000,\\"key_size\\":10000,\\"value_size\\":90000,\\"algorithm\\":\\"lcg\\"}'' +--iterations=1 +--fanout=1 +--num_workers=5 +--autoscaling_algorithm=NONE +--streaming +--experiments=use_runner_v2 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Streaming_2GB_of_10B_records.txt b/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Streaming_2GB_of_10B_records.txt new file mode 100644 index 000000000000..64d224a4663f --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Streaming_2GB_of_10B_records.txt @@ -0,0 +1,30 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_streaming_gbk_1 +--influx_measurement=python_streaming_gbk_1 +--input_options=''{\\"num_records\\":200000000,\\"key_size\\":1,\\"value_size\\":9,\\"algorithm\\":\\"lcg\\"}'' +--iterations=1 +--fanout=1 +--num_workers=5 +--autoscaling_algorithm=NONE +--streaming +--experiments=use_runner_v2 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Streaming_fanout_4_times_with_2GB_10-byte_records_total.txt b/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Streaming_fanout_4_times_with_2GB_10-byte_records_total.txt new file mode 100644 index 000000000000..8e38713cc66d --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Streaming_fanout_4_times_with_2GB_10-byte_records_total.txt @@ -0,0 +1,30 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_streaming_gbk_4 +--influx_measurement=python_streaming_gbk_4 +--input_options=''{\\"num_records\\":5000000,\\"key_size\\":10,\\"value_size\\":90,\\"algorithm\\":\\"lcg\\"}'' +--iterations=1 +--fanout=4 +--num_workers=16 +--autoscaling_algorithm=NONE +--streaming +--experiments=use_runner_v2 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Streaming_fanout_8_times_with_2GB_10-byte_records_total.txt b/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Streaming_fanout_8_times_with_2GB_10-byte_records_total.txt new file mode 100644 index 000000000000..35508480662c --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Streaming_fanout_8_times_with_2GB_10-byte_records_total.txt @@ -0,0 +1,30 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_streaming_gbk_5 +--influx_measurement=python_streaming_gbk_5 +--input_options=''{\\"num_records\\":2500000,\\"key_size\\":10,\\"value_size\\":90,\\"algorithm\\":\\"lcg\\"}'' +--iterations=1 +--fanout=8 +--num_workers=16 +--autoscaling_algorithm=NONE +--streaming +--experiments=use_runner_v2 +--runner=DataflowRunner \ No newline at end of file From 2be0605aeb79261739706e5d9befd1ff0bc48657 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 6 Oct 2023 09:19:06 -0400 Subject: [PATCH 050/435] Bump github.com/proullon/ramsql from 0.1.2 to 0.1.3 in /sdks (#28857) Bumps [github.com/proullon/ramsql](https://github.com/proullon/ramsql) from 0.1.2 to 0.1.3. - [Release notes](https://github.com/proullon/ramsql/releases) - [Commits](https://github.com/proullon/ramsql/compare/v0.1.2...v0.1.3) --- updated-dependencies: - dependency-name: github.com/proullon/ramsql dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 2 +- sdks/go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index fb883c93af19..d1c98259f1e9 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -45,7 +45,7 @@ require ( github.com/johannesboyne/gofakes3 v0.0.0-20221110173912-32fb85c5aed6 github.com/lib/pq v1.10.9 github.com/linkedin/goavro/v2 v2.12.0 - github.com/proullon/ramsql v0.1.2 + github.com/proullon/ramsql v0.1.3 github.com/spf13/cobra v1.7.0 github.com/testcontainers/testcontainers-go v0.24.1 github.com/tetratelabs/wazero v1.5.0 diff --git a/sdks/go.sum b/sdks/go.sum index 1fcb968e118f..1c39718964d0 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -386,8 +386,8 @@ github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZN github.com/power-devops/perfstat v0.0.0-20210106213030-5aafc221ea8c h1:ncq/mPwQF4JjgDlrVEn3C11VoGHZN7m8qihwgMEtzYw= github.com/power-devops/perfstat v0.0.0-20210106213030-5aafc221ea8c/go.mod h1:OmDBASR4679mdNQnz2pUhc2G8CO2JrUAVFDRBDP/hJE= github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= -github.com/proullon/ramsql v0.1.2 h1:PTtsy2iml/CW3Lsopyr86dlIs7JyYEmfLrfYvQVXD2U= -github.com/proullon/ramsql v0.1.2/go.mod h1:CFGqeQHQpdRfWqYmWD3yXqPTEaHkF4zgXy1C6qDWc9E= +github.com/proullon/ramsql v0.1.3 h1:/LRcXJf4lEmhdb4tYcci473I2VynjcZSzh2hsjJ8rSk= +github.com/proullon/ramsql v0.1.3/go.mod h1:CFGqeQHQpdRfWqYmWD3yXqPTEaHkF4zgXy1C6qDWc9E= github.com/rogpeppe/clock v0.0.0-20190514195947-2896927a307a h1:3QH7VyOaaiUHNrA9Se4YQIRkDTCw1EJls9xTUCaCeRM= github.com/rogpeppe/clock v0.0.0-20190514195947-2896927a307a/go.mod h1:4r5QyqhjIWCcK8DO4KMclc5Iknq5qVBAlbYYzAbUScQ= github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= From b87629928c92b3df4ba88fb42184d5ed505dbe2e Mon Sep 17 00:00:00 2001 From: Jack McCluskey <34928439+jrmccluskey@users.noreply.github.com> Date: Fri, 6 Oct 2023 11:01:34 -0400 Subject: [PATCH 051/435] Enable typevar-name-mismatch check (#28849) * Enable typevar-name-mismatch check * Trailing whitespace --- sdks/python/.pylintrc | 1 - sdks/python/apache_beam/typehints/decorators_test.py | 1 + 2 files changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/.pylintrc b/sdks/python/.pylintrc index 250932e79812..e649e07e5ef6 100644 --- a/sdks/python/.pylintrc +++ b/sdks/python/.pylintrc @@ -140,7 +140,6 @@ disable = stop-iteration-return, super-init-not-called, superfluous-parens, - typevar-name-mismatch, #TODO(https://github.com/apache/beam/issues/28241) Enable and fix warnings try-except-raise, undefined-variable, unexpected-keyword-arg, diff --git a/sdks/python/apache_beam/typehints/decorators_test.py b/sdks/python/apache_beam/typehints/decorators_test.py index 239c9bd57078..3baf9fa8322f 100644 --- a/sdks/python/apache_beam/typehints/decorators_test.py +++ b/sdks/python/apache_beam/typehints/decorators_test.py @@ -38,6 +38,7 @@ T = TypeVariable('T') # Name is 'T' so it converts to a beam type with the same name. # mypy requires that the name of the variable match, so we must ignore this. +# pylint: disable=typevar-name-mismatch T_typing = typing.TypeVar('T') # type: ignore From b9398e0e3e9e9b448fb082cb7ed5893af2d202f0 Mon Sep 17 00:00:00 2001 From: Vitaly Terentyev Date: Fri, 6 Oct 2023 21:35:25 +0400 Subject: [PATCH 052/435] Add Java JMH workflow (#28860) * Add Java JMH workflow * Refactoring --- .github/workflows/beam_Java_JMH.yml | 68 +++++++++++++++++++++++++++++ 1 file changed, 68 insertions(+) create mode 100644 .github/workflows/beam_Java_JMH.yml diff --git a/.github/workflows/beam_Java_JMH.yml b/.github/workflows/beam_Java_JMH.yml new file mode 100644 index 000000000000..07beb1dadba1 --- /dev/null +++ b/.github/workflows/beam_Java_JMH.yml @@ -0,0 +1,68 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: Java JMH + +on: + schedule: + - cron: '0 0 * * 0' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_HOST: http://10.128.0.96:8086 + INFLUXDB_DATABASE: beam_test_metrics + +jobs: + beam_Java_JMH: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 900 + name: "beam_Java_JMH" + steps: + - uses: actions/checkout@v3 + - name: run the Java JMH micro-benchmark harness suite + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:harness:jmh:jmh + - name: run the Java JMH micro-benchmark core suite + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:core:jmh:jmh \ No newline at end of file From 57821c191d322f9f21c01a34c55e0c40eda44f1e Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Wed, 4 Oct 2023 09:53:21 -0400 Subject: [PATCH 053/435] Add python wheel staging to GitHub Actions build_release_candidate --- .github/workflows/build_release_candidate.yml | 76 ++++++++++++++++++- .../download_github_actions_artifacts.py | 15 ++-- 2 files changed, 81 insertions(+), 10 deletions(-) diff --git a/.github/workflows/build_release_candidate.yml b/.github/workflows/build_release_candidate.yml index ded089667364..3956551431c6 100644 --- a/.github/workflows/build_release_candidate.yml +++ b/.github/workflows/build_release_candidate.yml @@ -20,7 +20,7 @@ on: description: Your Apache password. Required if you want to stage artifacts into https://dist.apache.org/repos/dist/dev/beam/ required: false BEAM_SITE_TOKEN: - description: Github Personal Access Token with repo permission if you want to create the beam-site docs PR. See https://docs.github.com/en/authentication/keeping-your-account-and-data-secure/managing-your-personal-access-tokens. + description: Github Personal Access Token with apache/beam-site repo permission if you want to create the beam-site docs PR. See https://docs.github.com/en/authentication/keeping-your-account-and-data-secure/managing-your-personal-access-tokens. default: '' PUBLISH_JAVA_ARTIFACTS: description: Whether to publish java artifacts to https://repository.apache.org/#stagingRepositories (yes/no) @@ -34,6 +34,10 @@ on: description: Whether to stage SDK docker images to docker hub Apache organization required: true default: 'no' + STAGE_PYTHON_ARTIFACTS: + description: Whether to stage the python artifacts into https://dist.apache.org/repos/dist/dev/beam/ + required: true + default: 'no' CREATE_BEAM_SITE_PR: description: Whether to create the documentation update PR against apache/beam-site. required: true @@ -147,7 +151,74 @@ jobs: svn status svn commit -m "Staging Java artifacts for Apache Beam ${{ github.event.inputs.RELEASE }} RC${{ github.event.inputs.RC }}" --non-interactive --username ${{ github.event.inputs.APACHE_ID }} --password ${{ github.event.inputs.APACHE_PASSWORD }} - + stage_python_artifacts: + if: ${{github.event.inputs.STAGE_PYTHON_ARTIFACTS == 'yes'}} + runs-on: ubuntu-latest + steps: + - name: Validate and mask apache id/password + run: | + echo "::add-mask::${{ github.event.inputs.APACHE_PASSWORD }}" + if [ "${{ github.event.inputs.APACHE_ID }}" == "" ] + then + echo "Must provide an apache id to stage artifacts to https://dist.apache.org/repos/dist/dev/beam/" + fi + if [ "${{ github.event.inputs.APACHE_PASSWORD }}" == "" ] + then + echo "Must provide an apache password to stage artifacts to https://dist.apache.org/repos/dist/dev/beam/" + fi + - name: Setup environment + uses: ./.github/actions/setup-environment-action + with: + python-version: 3.8 + - name: Import GPG key + id: import_gpg + uses: crazy-max/ghaction-import-gpg@111c56156bcc6918c056dbef52164cfa583dc549 + with: + gpg_private_key: ${{ secrets.GPG_PRIVATE_KEY }} + - name: stage python artifacts + env: + RC_TAG: "v${{ github.event.inputs.RELEASE }}-RC${{ github.event.inputs.RC }}" + GIT_REPO_BASE_URL: https://github.com/apache/beam + RELEASE_DIR: "beam/${{ github.event.inputs.RELEASE }}" + SCRIPT_DIR: release/src/main/scripts + GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} + run: | + svn co https://dist.apache.org/repos/dist/dev/beam + mkdir -p "${SVN_ARTIFACTS_DIR}" + + RELEASE_COMMIT=$(git rev-list -n 1 "tags/${RC_TAG}") + + python "${SCRIPT_DIR}/download_github_actions_artifacts.py" \ + --github-token-var GITHUB_TOKEN \ + --repo-url "${GIT_REPO_BASE_URL}" \ + --rc-tag "${RC_TAG}" \ + --release-commit "${RELEASE_COMMIT}" \ + --artifacts_dir "${RELEASE_DIR}/python" + + cd "${RELEASE_DIR}"/python + + echo "------Checking Hash Value for apache-beam-${RELEASE}.zip-----" + sha512sum -c "apache-beam-${RELEASE}.zip.sha512" + + echo "------Signing Source Release apache-beam-${RELEASE}.zip------" + gpg --local-user "${{steps.import_gpg.outputs.name}}" --armor --detach-sig "apache-beam-${RELEASE}.zip" + + for artifact in *.whl; do + echo "----------Checking Hash Value for ${artifact} wheel-----------" + sha512sum -c "${artifact}.sha512" + done + + for artifact in *.whl; do + echo "------------------Signing ${artifact} wheel-------------------" + gpg --local-user "${{steps.import_gpg.outputs.name}}" --armor --batch --yes --detach-sig "${artifact}" + done + + cd .. + svn add --force python + svn status + svn commit --no-auth-cache -m "Staging Python artifacts for Apache Beam ${RELEASE} RC${RC_NUM}" + + stage_docker: if: ${{github.event.inputs.STAGE_DOCKER_ARTIFACTS == 'yes'}} # Note: if this ever changes to self-hosted, remove the "Remove default github maven configuration" step @@ -192,7 +263,6 @@ jobs: - name: Push docker images run: ./gradlew :pushAllDockerImages -PisRelease -Pdocker-pull-licenses -Pprune-images -Pdocker-tag=${{ github.event.inputs.RELEASE }}rc${{ github.event.inputs.RC }} -Pjava11Home=${{steps.export-java11.outputs.JAVA11_HOME}} --no-daemon --no-parallel - beam_site_pr: if: ${{github.event.inputs.CREATE_BEAM_SITE_PR == 'yes'}} # Note: if this ever changes to self-hosted, remove the "Remove default github maven configuration" step diff --git a/release/src/main/scripts/download_github_actions_artifacts.py b/release/src/main/scripts/download_github_actions_artifacts.py index 181fd0c8b92b..5fbeb51a10cd 100644 --- a/release/src/main/scripts/download_github_actions_artifacts.py +++ b/release/src/main/scripts/download_github_actions_artifacts.py @@ -44,7 +44,7 @@ def parse_arguments(): description= "Script for downloading GitHub Actions artifacts from 'Build python wheels' workflow." ) - parser.add_argument("--github-user", required=True) + parser.add_argument("--github-token-var", required=False, default='GITHUB_TOKEN') parser.add_argument("--repo-url", required=True) parser.add_argument("--rc-tag", required=True) parser.add_argument("--release-commit", required=True) @@ -52,7 +52,7 @@ def parse_arguments(): parser.add_argument("--rc_number", required=False, default="") args = parser.parse_args() - github_token = ask_for_github_token() + github_token = get_github_token(args.github_token_var) print("You passed following arguments:") pprint.pprint({**vars(args), **{"github_token": github_token}}) @@ -61,7 +61,6 @@ def parse_arguments(): print("You said NO. Quitting ...") sys.exit(1) - user_github_id = args.github_user repo_url = args.repo_url rc_tag = args.rc_tag release_commit = args.release_commit @@ -69,11 +68,14 @@ def parse_arguments(): else os.path.abspath(args.artifacts_dir) rc_number = args.rc_number - return github_token, user_github_id, repo_url, rc_tag, release_commit, artifacts_dir, rc_number + return github_token, repo_url, rc_tag, release_commit, artifacts_dir, rc_number -def ask_for_github_token(): - """Ask for github token and print basic information about it.""" +def get_github_token(github_token_var): + """Get GitHub token from env or ask for it and print basic information about it.""" + if github_token_var in os.environ: + return os.environ[github_token_var] + url = "https://docs.github.com/en/github/authenticating-to-github/creating-a-personal-access-token" message = ( f"You need to have a github access token with public_repo scope. " @@ -321,7 +323,6 @@ def extract_single_artifact(file_path, output_dir): ) ( github_token, - user_github_id, repo_url, rc_tag, release_commit, From 14950aab7947c6a5819ea666da7aa298782726a0 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Wed, 4 Oct 2023 18:31:46 -0400 Subject: [PATCH 054/435] Update release guide somewhat according to 2.51.0 release experience --- contributor-docs/release-guide.md | 1092 +++++++++++++++-------------- 1 file changed, 574 insertions(+), 518 deletions(-) diff --git a/contributor-docs/release-guide.md b/contributor-docs/release-guide.md index f066047f6df2..189bc61e15e6 100644 --- a/contributor-docs/release-guide.md +++ b/contributor-docs/release-guide.md @@ -16,198 +16,206 @@ limitations under the License. ## Introduction -The Apache Beam project periodically declares and publishes releases. A -release is one or more packages of the project artifact(s) that are approved -for general public distribution and use. They may come with various degrees of -caveat regarding their perceived quality and potential for change, such as -“alpha”, “beta”, “incubating”, “stable”, etc. - -The Beam community treats releases with great importance. They are a public +The Beam community treats releases with great importance. They are a public face of the project and most users interact with the project only through the releases. Releases are signed off by the entire Beam community in a public vote. Each release is executed by a *Release Manager*, who is selected among the Beam committers. This document describes the process that the Release Manager -follows to perform a release. Any changes to this process should be discussed -and adopted on the [dev@ mailing list](/get-started/support/). +follows to perform a release. -Please remember that publishing software has legal consequences. This guide -complements the foundation-wide [Product Release -Policy](https://www.apache.org/dev/release.html) and [Release Distribution -Policy](https://www.apache.org/dev/release-distribution). +Please remember that publishing software has legal consequences. This guide +complements the foundation-wide guides: -### Overview + - [Product Release Policy](https://www.apache.org/dev/release.html) + - [Release Distribution + Policy](https://www.apache.org/dev/release-distribution). -Release step flow chart +### What is in a Beam release -The release process consists of several steps: +A Beam release consists of the following: -1. Decide to release -2. Prepare for the release -3. Build a release candidate -4. Verify & vote on the release candidate -5. If necessary, fix any issues and go back to "Build a release candidate" -6. Finalize the release -7. Promote the release + - ASF source zips archived on + [dist.apache.org](https://dist.apache.org/release/beam) (later archived to + [archive.apache.org](https://archive.apache.org/dist/beam) + - Java jars and poms published to [Maven + Central](https://mvnrepository.com/artifact/org.apache.beam) + - Python wheels published to [pypi](https://pypi.org/project/apache-beam/) + - Go artifacts published to + [pkg.go.dev](https://pkg.go.dev/github.com/apache/beam) + - Docker images published to + [dockerhub](https://hub.docker.com/search?q=apache%2Fbeam&type=image) + - A tag on GitHub indicating the commit from which the release was built ------------- +In addition, each release is accompanied by: -## 1. Decide to release + - A blog post announcing the release and describing the changes + - An update to the webpage to indicate the latest version -Deciding to release and selecting a Release Manager is the first step of the release process. -This is a consensus-based decision of the entire community. +### Phases of the release process -Anybody can propose a release on the dev@ mailing list, giving a solid argument and nominating a committer as the Release Manager (including themselves). -There’s no formal process, no vote requirements, and no timing requirements. Any objections should be resolved by consensus before starting the release. +The release process consists of several phases: -In general, the community prefers to have a rotating set of 3-5 Release Managers. -Keeping a small core set of managers allows enough people to build expertise in this area and improve processes over time, without Release Managers needing to re-learn the processes for each release. -That said, if you are a committer interested in serving the community in this way, please reach out to the community on the dev@ mailing list. - -### Checklist to proceed to the next step +1. Prepare for release +2. Stabilize the release branch / burn down release-blocking issues +3. Build a release candidate +4. Validate and approve the release candidate +5. Finalize the release +6. Promote the release +7. Post-release tasks -- [ ] Community agrees to release -- [ ] Community selects a committer as Release Manager +------------ -------- +## Prepare for release (~1 week before release cut) -## 2. Prepare for the release (~1 week before branch cut) +The following steps take place before the release branch is cut. -Before your first release, you should perform one-time configuration steps. -This will set up your security keys for signing the release and access to various release repositories. +### Decide to release -To prepare for each release, you should audit the project status in the GitHub issue tracker, and do necessary bookkeeping. -Finally, create a release branch from which individual release candidates will be built. +Deciding to release and selecting a Release Manager is the first step of the +release process. This is a consensus-based decision of the entire community. +Anybody can propose a release on the `dev@` list. There is no formal process, +no vote requirements, and no timing requirements. A committer must be +identified to be the Release Manager. In practice, most often a committer both +proposes to release and volunteers themselves as Release Manager. -__NOTE__: If you are using [GitHub two-factor authentication](https://help.github.com/articles/securing-your-account-with-two-factor-authentication-2fa/) and haven't configure HTTPS access, -please follow [the guide](https://help.github.com/articles/creating-a-personal-access-token-for-the-command-line/) to configure command line access. +------- ### Create a new milestone in GitHub for the next release -When contributors resolve an issue in GitHub, they are tagging it with a release that will contain their changes. -With the release currently underway, new issues should be resolved against a subsequent future release. -Therefore, you should create a release item for this subsequent release, as follows: +When contributors resolve an issue in GitHub, they are tagging it with a +release that will contain their changes. With the release currently underway, +new issues should be resolved against a subsequent future release. Therefore, +you should create a release item for this subsequent release, as follows: -1. In GitHub, navigate to [`Issues > Milestones > New Milestone`](https://github.com/apache/beam/milestones). -2. Add a new release. Choose the next minor version number after the version currently underway, select the next release due date (generally 6 weeks from today’s date) as the `Start Date`, and choose `Create Milestone`. -3. At the end of the release, go to the same page and mark the recently released version as closed. +In GitHub, navigate to [`Issues > Milestones > New +Milestone`](https://github.com/apache/beam/milestones) and add a new +release for the next minor version after the version you are preparing +to release. -### Accounts +---- -Please have these credentials ready at hand, you will likely need to enter them multiple times: +### Prepare accounts, keys, etc -* Apache ID and Password; -* GitHub ID, Password, and Personal Access Token. -* PyPi account and apitoken -* DockerHub ID and Password with beam maintainer access -* GPG pass phrase & 16-digit key ID -* Access to Beam's Apache Nexus repository -* Account to access to apache-beam-testing Google Cloud Platform project. The account must have permissions to start Cloud Build triggers. Required for Playground environment update. (E-mail at dev@ mailing list to request access) +Before your first release, you need to make sure you have all the necessary +accounts, keys, and access for publishing the release. The release process also +requires a variety of API tokens, which you can generate now or later when they +are needed. -If you don't have a given credential, follow the 'one-time' instructions below. +These are the credentials you will need: ----- - -### One-time setup instructions + - Apache ID and Password + - GitHub ID, Password, and Personal Access Token + - PyPi account with beam maintainer access and API Token + - GPG pass phrase & 16-digit key ID + - Access to Beam's Apache Nexus repository + - Account to access to apache-beam-testing Google Cloud Platform project. The + account must have permissions to start Cloud Build triggers. Required for + Playground environment update. (E-mail at dev@ mailing list to request + access) #### Apache ID and Password -This is your Apache committer user name and password. You selected these when you became an Apache Beam Committer. +This is your Apache committer user name and password. You selected these when +you became an Apache Beam Committer. #### Github ID, Password, and Personal Access Token -For some scripts, you need a Personal Access Token with `repo` and `workflow` permissions. -They can be generated from this page: https://github.com/settings/tokens. -See https://docs.github.com/en/authentication/keeping-your-account-and-data-secure/managing-your-personal-access-tokens for details. - -#### Register to PyPI - -Release manager needs to have an account with PyPI. -If you need one, [register at PyPI](https://pypi.python.org/account/register/). -You also need to be a maintainer (or an owner) of the [apache-beam](https://pypi.python.org/pypi/apache-beam) package in order to push a new release. -Ask on the mailing list for assistance. - -Generate a [PyPI APIToken](https://pypi.org/help/#apitoken) for use during the release. - -#### Login to DockerHub -If you are a member of the [`beam` DockerHub team](https://hub.docker.com/orgs/apache/teams/beam), run the following command manually. -It will ask you to input your DockerHub ID and password if authorization info cannot be found from ~/.docker/config.json file. - -``` -docker login docker.io -``` - -After successful login, authorization info will be stored at ~/.docker/config.json file. -For example, -``` -"https://index.docker.io/v1/": { - "auth": "xxxxxx" -} -``` - -If you are not already a member of the `beam` team, please email `dev@` mailing list for help with any DockerHub related tasks. We are not able -to add more members to the DockerHub team because [the ASF has a limited number of seats available](https://infra.apache.org/docker-hub-policy.html). + - [ ] If you are using [GitHub two-factor + authentication](https://help.github.com/articles/securing-your-account-with-two-factor-authentication-2fa/) + and haven't configure HTTPS access, please follow [the + guide](https://help.github.com/articles/creating-a-personal-access-token-for-the-command-line/) + to configure command line access. + - [ ] Generate a Personal Access Token with `repo` and `workflow` permissions. + They can be generated from this page: https://github.com/settings/tokens. + See + https://docs.github.com/en/authentication/keeping-your-account-and-data-secure/managing-your-personal-access-tokens + for details. + +#### PyPI account and API token + + - [ ] [Create an account with PyPI](https://pypi.python.org/account/register/) + if you don't have one already. + - [ ] Become a maintainer (or an owner) of the + [apache-beam](https://pypi.python.org/pypi/apache-beam) package. + - [ ] Generate a [PyPI APIToken](https://pypi.org/help/#apitoken) for use + during the release. #### GPG Key -You need to have a GPG key to sign the release artifacts. -Please be aware of the ASF-wide [release signing guidelines](https://www.apache.org/dev/release-signing.html). -If you don’t have a GPG key associated with your Apache account, please create one according to the guidelines. - -There are 2 ways to configure your GPG key for release, either using release automation script(which is recommended), or running all commands manually. +You need to have a GPG key to sign the release artifacts. Please be aware of +the ASF-wide [release signing +guidelines](https://www.apache.org/dev/release-signing.html). If you don’t +have a GPG key associated with your Apache account, you must now create one +according to the guidelines. -##### Use preparation_before_release.sh to setup GPG -* **Script:** [preparation_before_release.sh](https://github.com/apache/beam/blob/master/release/src/main/scripts/preparation_before_release.sh) +Run the following helper script, or you can open it and run the commands +individually (helpful if it doesn't work as intended or if you already are +partially set up) -* **Usage** - ``` - ./release/preparation_before_release.sh - ``` -* **Tasks included** - 1. Help you create a new GPG key if you want. - 2. Configure ```git user.signingkey``` with chosen pubkey. - 3. Add chosen pubkey into [dev KEYS](https://dist.apache.org/repos/dist/dev/beam/KEYS) and [release KEYS](https://dist.apache.org/repos/dist/release/beam/KEYS) + ./release/src/main/scripts/preparation_before_release.sh - **NOTES**: Only PMC can write into [release repo](https://dist.apache.org/repos/dist/release/beam/). - 4. Start GPG agents. +> **__NOTE__**: +> When generating the key, please make sure you choose the key type as +> __RSA and RSA (default)__ and key size as __4096 bit__. -__NOTE__: When generating the key, please make sure you choose the key type as __RSA and RSA (default)__ and key size as __4096 bit__. +Now you should have: -* To run the commands manually, refer to the contents of `preparation_before_release.sh`. + - [ ] A GPG key meeting ASF guidelines + - [ ] The key added to + [dev KEYS](https://dist.apache.org/repos/dist/dev/beam/KEYS) and [release KEYS](https://dist.apache.org/repos/dist/release/beam/KEYS) + **NOTE**: Only PMC can write into [release repo](https://dist.apache.org/repos/dist/release/beam/). + - [ ] The `user.signingkey` set in your `.gitconfig` + - [ ] `gpg-agent` with the key loaded ##### Key ID -* You may need your Key ID for future steps. Determine your Apache GPG Key and Key ID as follows: +You may need your Key ID for future steps. Determine your Apache GPG Key and +Key ID as follows: - gpg --list-sigs --keyid-format LONG + gpg --list-sigs --keyid-format LONG - This will list your GPG keys. One of these should reflect your Apache account, for example: +This will list your GPG keys. One of these should reflect your Apache account, +for example: - -------------------------------------------------- - pub rsa4096/845E6689845E6689 2016-02-23 - uid Nomen Nescio - sub rsa4096/BA4D50BEBA4D50BE 2016-02-23 + -------------------------------------------------- + pub rsa4096/845E6689845E6689 2016-02-23 + uid Nomen Nescio + sub rsa4096/BA4D50BEBA4D50BE 2016-02-23 - Here, the key ID is the 16-digit hex string in the `pub` line: `845E6689845E6689`. +Here, the key ID is the 16-digit hex string in the `pub` line: `845E6689845E6689`. ##### Submit your GPG public key into Ubuntu OpenPGP Key Server -In order to make yourself have right permission to stage java artifacts in Apache Nexus staging repository, -please submit your GPG public key into the [Ubuntu OpenPGP Key Server](https://keyserver.ubuntu.com/). -You will need to use an ascii-armored version of your key. -This can be obtained by running `gpg --export --armor` and copying the whole block -(including `----- PGP PUBLIC KEY BLOCK-----`). +In order to make yourself have right permission to stage java artifacts in +Apache Nexus staging repository, please submit your GPG public key into the +[Ubuntu OpenPGP Key Server](https://keyserver.ubuntu.com/). + +You will need to use an ascii-armored version of your key. This can be +obtained by running: + + gpg --export --armor + +Copying the whole block including `-----START PGP PUBLIC KEY BLOCK-----` and +`-----END PGP PUBLIC KEY BLOCK-----` #### Access to Apache Nexus repository -Configure access to the [Apache Nexus repository](https://repository.apache.org/), which enables final deployment of releases to the Maven Central Repository. +Configure access to the [Apache Nexus +repository](https://repository.apache.org/), which enables final deployment of +releases to the Maven Central Repository. -1. You log in with your Apache account. -2. Confirm you have appropriate access by finding `org.apache.beam` under `Staging Profiles`. +1. Log in with your Apache account. +2. Confirm you have appropriate access by finding `org.apache.beam` under + `Staging Profiles`. 3. Navigate to your `Profile` (top right dropdown menu of the page). -4. Choose `User Token` from the dropdown, then click `Access User Token`. Copy a snippet of the Maven XML configuration block. -5. Insert this snippet twice into your global Maven `settings.xml` file, typically `${HOME}/.m2/settings.xml`. The end result should look like this, where `TOKEN_NAME` and `TOKEN_PASSWORD` are your secret tokens: +4. Choose `User Token` from the dropdown, then click `Access User Token`. Copy + a snippet of the Maven XML configuration block. +5. Insert this snippet + twice into your global Maven `settings.xml` file, typically + `${HOME}/.m2/settings.xml`. The end result should look like this, where + `TOKEN_NAME` and `TOKEN_PASSWORD` are your secret tokens: @@ -225,400 +233,407 @@ Configure access to the [Apache Nexus repository](https://repository.apache.org/ -********** +---- + +### Dependency checks -### Handle Per Release tasks +Each language has routine dependency maintenance that you should check now. #### Update base image dependencies for Python container images -The Python base container images have static pinned `requirements.txt` that are -designed to be compatible with our dependency constraints but also not cause -runtime installs to occur, which slow things down immensely. -These need to be updated at least once per release cycle to avoid -out of date dependencies. +The Python base container images have pinned `requirements.txt` that are +compatible with our dependency constraints, and design to avoid run-time +installs, since run-time installs cause large delays at start-up time. Ideally, +we this should happen regularly when dependencies update, but it is important +to ensure that they are fully up to date for each release. Follow the instructions at https://s.apache.org/beam-python-requirements-generate -Ideally, do the update at least a week before the release cut, so that any issues -related to the update have time to surface. - #### Update Go version used for container builds -Go makes security patch releases of their tooling. Ideally, we upgrade as soon -as possible, but it is also good to ensure we are up to date for each release. - -This potentially affects container bootloader security, and at the least can cause -false positives when an default-configuration scanner is pointed at our containers. +Go makes security patch releases of their tooling. This potentially affects +container bootloader security, and at the least can cause false positives when +an default-configuration scanner is pointed at our containers. Ideally, we +upgrade as soon as possible, but it is also good to ensure we are up to date +for each release. - - [ ] See if https://go.dev/doc/devel/release has a newer release. Update throughout - Beam. See example at https://github.com/apache/beam/pull/27900/files +See if https://go.dev/doc/devel/release has a newer release. Update throughout +Beam. See example at https://github.com/apache/beam/pull/27900/files #### Update the Java BOM -Tracked in Github issue https://github.com/apache/beam/issues/28379 - -Ideally, do the update at least a week before the release cut, so that any issues -related to the update have time to surface. +Google releases a BOM that pins compatible versions of their Java libraries. +Ideally, this update happens as far in advance of the release as possible, such +as just after a release. But if that was not done, consider doing it before +cutting the release branch. -#### Investigate performance regressions +To do so, follow instructions at +https://github.com/apache/beam/blob/master/contributor-docs/java-dependency-upgrades.md. -Check the Beam load tests for possible performance regressions. -Measurements are available on [metrics.beam.apache.org](http://metrics.beam.apache.org). +### Cut the release branch -All Runners which publish data should be checked for the following, in both *batch* and *streaming* mode: - -- [ParDo](http://metrics.beam.apache.org/d/MOi-kf3Zk/pardo-load-tests) and [GBK](http://metrics.beam.apache.org/d/UYZ-oJ3Zk/gbk-load-test): Runtime, latency, checkpoint duration -- [Nexmark](http://metrics.beam.apache.org/d/ahudA_zGz/nexmark): Query runtime for all queries -- [IO](http://metrics.beam.apache.org/d/bnlHKP3Wz/java-io-it-tests-dataflow): Runtime +> **Note** +> Wait until the proposed branch cut day! -If regressions are found, the release branch can still be created, but the regressions should be investigated and fixed as part of the release process. -The role of the release manager is to file GitHub issues for each regression with the milestone set to the to-be-released version. -The release manager oversees these just like any other issue marked with the milestone of the release. +We cut the release branch on time and do not block/delay branch cut for incoming +fixes. This is because bugs are always being introduced as part of normal +development. We cut the branch to prevent new bugs being introduced and _then_ +we fix and cherrypick any truly release-blocking problems. -The mailing list should be informed to allow fixing the regressions in the course of the release. Issues should be filed and tagged with the milestone. +In order to run this workflow, you will need to provide a Apache ID and Jenkins +API token. Your Jenkins API token can be generated by visiting +https://ci-beam.apache.org, signing in with your Apache credentials, then going +to `https://ci-beam.apache.org/user//configure` and clicking +`Add new token` in the API token section. -#### Triage release-blocking issues in GitHub +- [ ] Run + [cut_release_branch](https://github.com/apache/beam/actions/workflows/cut_release_branch.yml) + (click `run workflow`) -There could be outstanding release-blocking issues, which should be triaged before proceeding to build a release candidate. -We track them by assigning the blocked release to the issue's milestone before the issue is resolved. +The final state of the repository after release branch is cut should match this +diagram: -The release manager should triage what does and does not block a release. -The list of release-blocking issues is available at the [milestone status page](https://github.com/apache/beam/milestones). -Triage each unresolved issue with one of the following resolutions: - -* An issue should not block the release if the problem exists in the current released version or is a bug in new functionality that does not exist in the current released version. -* An issue should be a blocker if the problem is a regression between the currently released version and the release in progress and has no easy workaround. - -For all GitHub issues: +Increment minor version on master branch and set Dataflow container version on release branch -* If the issue has been resolved and the GitHub issue was not updated, resolve it accordingly. +This should be accomplished by the +[cut_release_branch](https://github.com/apache/beam/actions/workflows/cut_release_branch.yml) +workflow. This workflow will also update +[mass_comment.py](https://github.com/apache/beam/blob/master/release/src/main/scripts/mass_comment.py) +to contain all of the active Jenkins jobs. -For issues with type "Bug" or labeled "flaky": +The following must be manually done or confirmed: -* If the issue is a known continuously failing test, it is not acceptable to defer this until the next release. - Please work with the Beam community to resolve the issue. -* If the issue is a known flaky test, make an attempt to delegate a fix. - However, if the issue may take too long to fix (to the discretion of the release manager): - * Delegate manual testing of the flaky issue to ensure no release blocking issues. - * Update the milestone to the version of the next release. - Please consider discussing this with stakeholders and the dev@ mailing list, as appropriate. +- [ ] The `master` branch has the SNAPSHOT/dev version incremented. +- [ ] The release branch has the SNAPSHOT/dev version to be released. +- [ ] The Dataflow container image should be modified to the version to be released. +- [ ] Due to a bug/limitation in the workflow, you must navigate to the pull + request found in the logs and comment `Run Gradle Publish`. +- [ ] After publish, close the PR. +- [ ] Manually update `CHANGES.md` on `master` by adding a new section for the + next release + ([example](https://github.com/apache/beam/commit/96ab1fb3fe07acf7f7dc9d8c829ae36890d1535c)). -For all other GitHub issues: +#### Inform the mailing list -* If the issue has not been resolved and it is acceptable to defer this until the next release, update the milestone to the new version you just created. - Please consider discussing this with stakeholders and the dev@ mailing list, as appropriate. -* If the issue has not been resolved and it is not acceptable to release until it is fixed, the release cannot proceed. - Instead, work with the Beam community to resolve the issue. +The dev@ mailing list should be informed about the release branch being cut. +Alongside with this note, a list of pending issues and to-be-triaged issues +should be included. Afterwards, this list can be refined and updated by the +release manager and the Beam community. -If there is a bug found in the RC creation process/tools, those issues should be considered high priority and fixed in 7 days. -### Checklist to proceed to the next step +### Checklist to proceed to the next phase +- [ ] Community agrees to release +- [ ] Community selects a committer (you) as Release Manager - [ ] Next release has a milestone in github. - [ ] You have your various account credentials prepared. -- [ ] Per Release tasks for the current release have been handled. -- [ ] Open issues/PRs against the current release have been notified. -- [ ] Performance Regressions have been investigated and had issues filed. -- [ ] It is the proposed branch cut day. +- [ ] You checked the dependency maintenance for each language. +- [ ] The release branch is created. +- [ ] The `master` branch is moved along to the next release. +- [ ] You have informed `dev@beam.apache.org` that you have cut the branch and + are proceeding to stabilization -********** - -## 3. Build a release candidate - -Building a release candidate involves creating a release branch, running validation tests against the branch, filing issues, cherry picking fixes, -making a release candidate tag, and building all artifacts from that tag. - -### Create a release branch in apache/beam repository - -As a final step in preparation for the release, you should create the release branch, and update version information on the original branch. -This should happen once per release. If additional release candidates are required, they are built from later versions of this branch. - -The final state of the repository should match this diagram: - -Increment minor version on master branch and set Dataflow container version on release branch - -The key points to know: - -- The `master` branch has the SNAPSHOT/dev version incremented. -- The release branch has the SNAPSHOT/dev version to be released. -- The Dataflow container image should be modified to the version to be released. - -This will all be accomplished by the [cut_release_branch](https://github.com/apache/beam/actions/workflows/cut_release_branch.yml) -workflow. This workflow will also update [mass_comment.py](https://github.com/apache/beam/blob/master/release/src/main/scripts/mass_comment.py) -to contain all of the active Jenkins jobs. - -After updating the master branch, the workflow will also start a build of -[the nightly snapshot](https://ci-beam.apache.org/job/beam_Release_NightlySnapshot/) against master branch. -Some processes, including our archetype tests, rely on having a live SNAPSHOT of the current version from the `master` branch. -Once the release branch is cut, these SNAPSHOT versions are no longer found, so builds will be broken until a new snapshot is available. -The workflow starts the nightly snapshot by creating an empty PR against apache:master (which will be linked to in the logs). - -#### Use cut_release_branch workflow to cut a release branch - -* **Action:** [cut_release_branch](https://github.com/apache/beam/actions/workflows/cut_release_branch.yml) (click `run workflow`) +------- -In order to run this workflow, you will need to provide a Apache ID and Jenkins API token. -Your Jenkins API token can be generated by visiting https://ci-beam.apache.org, signing in with your Apache credentials, -then going to `https://ci-beam.apache.org/user//configure` and clicking `Add new token` in the API token section. +## Stabilize the release branch -* Tasks you need to do manually to __verify the SNAPSHOT build__ - 1. Check whether the Jenkins job gets triggered. If not, please comment ```Run Gradle Publish``` into the generated PR. - 2. After verifying build succeeded, you need to close PR manually. - 3. Manually update `CHANGES.md` on `master` by adding a new section for the next release ([example](https://github.com/apache/beam/commit/96ab1fb3fe07acf7f7dc9d8c829ae36890d1535c)). +Once the release branch is cut, your job is to make sure tests pass, fix bugs, +confirm performance, defer feature requests, etc, until the branch is ready for +the work of building a release candidate. ### Verify release branch -After the release branch is cut you need to make sure it builds and has no significant issues that would block the creation of the release candidate. -There are 2 ways to perform this verification, either running automation script(recommended), or running all commands manually. - -> Dataflow tests will fail if the Dataflow worker container is not created and published by this time. -> Should be done by Google, in response to the creation of the release branch, and docker images are hosted. -> This should not block creation of the first release candidate, but should block approval of the release. - -#### Run automation script (verify_release_build.sh) -* **Script:** [verify_release_build.sh](https://github.com/apache/beam/blob/master/release/src/main/scripts/verify_release_build.sh) +After the release branch is cut, make sure it builds and has no significant +issues that would block the creation of the release candidate. + +> **NOTE** +> Dataflow tests will fail if the Dataflow worker container is not created and +> published by this time. Should be done by Google, in response to the +> creation of the release branch, and docker images are hosted. This should +> not block creation of the first release candidate, but should block approval +> of the release. + +- **Script:** + [verify_release_build.sh](https://github.com/apache/beam/blob/master/release/src/main/scripts/verify_release_build.sh) + +- **Usage** + 1. Create a personal access token from your Github account. + See instruction [here](https://help.github.com/en/articles/creating-a-personal-access-token-for-the-command-line). + It'll be used by the script for accessing Github API. + You need to enable `repo` and `workflow` permissions for this token. + 2. Update required configurations listed in `RELEASE_BUILD_CONFIGS` in [script.config](https://github.com/apache/beam/blob/master/release/src/main/scripts/script.config) + 3. Then run + ``` + (cd release/src/main/scripts && ./verify_release_build.sh) + ``` + 4. Trigger all Jenkins PostCommit jobs from the PR created by the previous step. + You can run [mass_comment.py](https://github.com/apache/beam/blob/master/release/src/main/scripts/mass_comment.py) to do that. + Or manually add one trigger phrase per PR comment. + See [jenkins_jobs.txt](https://github.com/apache/beam/blob/master/release/src/main/scripts/jenkins_jobs.txt) + for a full list of phrases. + +- **Tasks included in the script** + - Installs `hub` with your agreement and setup local git repo; + - Create a test PR against release branch; + +There are some projects that don't produce the artifacts, e.g. +`beam-test-tools`, you may be able to ignore failures there. + +To triage the failures and narrow things down you may want to look at +`settings.gradle.kts` and run the build only for the projects you're interested +at the moment, e.g. `./gradlew :runners:java-fn-execution`. + +The `verify_release_build.sh` script may include failing or flaky tests. For +each of the failing tests create a GitHub Issue with the following properties: -* **Usage** - 1. Create a personal access token from your Github account. - See instruction [here](https://help.github.com/en/articles/creating-a-personal-access-token-for-the-command-line). - It'll be used by the script for accessing Github API. - You need to enable `repo` and `workflow` permissions for this token. - 2. Update required configurations listed in `RELEASE_BUILD_CONFIGS` in [script.config](https://github.com/apache/beam/blob/master/release/src/main/scripts/script.config) - 3. Then run - ``` - (cd release/src/main/scripts && ./verify_release_build.sh) - ``` - 4. Trigger all Jenkins PostCommit jobs from the PR created by the previous step. - You can run [mass_comment.py](https://github.com/apache/beam/blob/master/release/src/main/scripts/mass_comment.py) to do that. - Or manually add one trigger phrase per PR comment. - See [jenkins_jobs.txt](https://github.com/apache/beam/blob/master/release/src/main/scripts/jenkins_jobs.txt) - for a full list of phrases. +* **Issue Type:** Bug -* **Tasks included in the script** - 5. Installs `hub` with your agreement and setup local git repo; - 6. Create a test PR against release branch; +* **Summary:** Name of failing gradle task and name of failing test (where applicable) in form of :MyGradleProject:SomeGradleTask NameOfFailedTest: Short description of failure -#### Verify the build succeeds +* **Priority:** P1 -* Tasks you need to do manually to __verify the build succeed__: - 1. Check the build result. - 2. If build failed, scan log will contain all failures. - 3. You should stabilize the release branch until release build succeeded. +* **Component:** "test-failures" -There are some projects that don't produce the artifacts, e.g. `beam-test-tools`, you may be able to ignore failures there. +* **Milestone:** Release number of verified release branch -To triage the failures and narrow things down you may want to look at `settings.gradle.kts` and run the build only for the projects you're interested at the moment, e.g. `./gradlew :runners:java-fn-execution`. +* **Description:** Description of failure -#### (Alternative) Run release build locally -You will need to have Python interpreters for all supported Python minor -versions to run Python tests. See Python installation tips in [Developer Wiki](https://cwiki.apache.org/confluence/display/BEAM/Python+Tips#PythonTips-InstallingPythoninterpreters). +### Investigate performance regressions -* **Run gradle release build** +Check the Beam load tests for possible performance regressions. Measurements +are available on [metrics.beam.apache.org](http://metrics.beam.apache.org). - 1. Clean current workspace +All Runners which publish data should be checked for the following, in both +*batch* and *streaming* mode: - ``` - git clean -fdx - ./gradlew clean - ``` +- [ParDo](http://metrics.beam.apache.org/d/MOi-kf3Zk/pardo-load-tests) and + [GBK](http://metrics.beam.apache.org/d/UYZ-oJ3Zk/gbk-load-test): Runtime, + latency, checkpoint duration +- [Nexmark](http://metrics.beam.apache.org/d/ahudA_zGz/nexmark): Query runtime + for all queries +- [IO](http://metrics.beam.apache.org/d/bnlHKP3Wz/java-io-it-tests-dataflow): Runtime - 2. Unlock the secret key - ``` - gpg --output ~/doc.sig --sign ~/.bashrc - ``` +If regressions are found, the release branch can still be created, but the +regressions should be investigated and fixed as part of the release process. +The role of the release manager is to file GitHub issues for each regression +with the milestone set to the to-be-released version. The release manager +oversees these just like any other issue marked with the milestone of the +release. - 3. Run build command - ``` - ./gradlew build -PisRelease --no-parallel --scan --stacktrace --continue - ``` +The mailing list should be informed to allow fixing the regressions in the +course of the release. Issues should be filed and tagged with the milestone. - To speed things up locally you might want to omit `--no-parallel`. You can also omit `--continue` - if you want build fails after the first error instead of continuing, it may be easier and faster - to find environment issues this way without having to wait until the full build completes. +### Triage release-blocking issues in GitHub -#### Create release-blocking issues in GitHub +There could be outstanding release-blocking issues, which should be triaged +before proceeding to build a release candidate. We track them by assigning the +blocked release to the issue's milestone before the issue is resolved. -The verify_release_build.sh script may include failing or flaky tests. -For each of the failing tests create a GitHub Issue with the following properties: +The release manager should triage what does and does not block a release. The +list of release-blocking issues is available at the [milestone status +page](https://github.com/apache/beam/milestones). Triage each unresolved issue +with one of the following resolutions: -* **Issue Type:** Bug + - An issue should not block the release if the problem exists in the current + released version or is a bug in new functionality that does not exist in the + current released version. + - An issue should be a blocker if the problem is a regression between the + currently released version and the release in progress and has no easy + workaround. -* **Summary:** Name of failing gradle task and name of failing test (where applicable) in form of :MyGradleProject:SomeGradleTask NameOfFailedTest: Short description of failure +For all GitHub issues: -* **Priority:** P1 + - If the issue has been resolved and the GitHub issue was not updated, + resolve it accordingly. -* **Component:** "test-failures" +For issues with type "Bug" or labeled "flaky": -* **Milestone:** Release number of verified release branch + - If the issue is a known continuously failing test, it is not acceptable to + defer this until the next release. Please work with the Beam community to + resolve the issue. + - If the issue is a known flaky test, make an attempt to delegate a fix. + However, if the issue may take too long to fix (to the discretion of the + release manager): + - Delegate manual testing of the flaky issue to ensure no release blocking issues. + - Update the milestone to the version of the next release. + Please consider discussing this with stakeholders and the dev@ mailing + list, as appropriate. -* **Description:** Description of failure +For all other GitHub issues: -#### Inform the mailing list + - If the issue has not been resolved and it is acceptable to defer this until the next release, update the milestone to the new version you just created. + Please consider discussing this with stakeholders and the dev@ mailing list, as appropriate. + - If the issue has not been resolved and it is not acceptable to release until it is fixed, the release cannot proceed. + Instead, work with the Beam community to resolve the issue. -The dev@ mailing list should be informed about the release branch being cut. -Alongside with this note, a list of pending issues and to-be-triaged issues should be included. -Afterwards, this list can be refined and updated by the release manager and the Beam community. +If there is a bug found in the RC creation process/tools, those issues should +be considered high priority and fixed in 7 days. ### Review cherry-picks -The release manager is empowered to triage issues, and accept or reject cherry-picks to the release branch. -Cherry picks are necessary if there are outstanding issues at time of the release branch cut, or issues were found in verification. +The release manager is empowered to triage issues, and accept or reject +cherry-picks to the release branch. Cherry picks are necessary if there are +outstanding issues at time of the release branch cut, or issues were found in +verification. + +Check if there are outstanding cherry-picks into the release branch, [e.g. for +`2.14.0`](https://github.com/apache/beam/pulls?utf8=%E2%9C%93&q=is%3Apr+base%3Arelease-2.14.0). +Make sure they have blocker Issues attached and are OK to get into the release +by checking with community if needed. + +You are encouraged to ask the following questions to be answered on each +cherry-pick PR and you can choose to reject cherry-pick requests if these +questions are not satisfactorily answered: + + - Is this a regression from a previous release? (If no, fix could go to a + newer version.) + - Is this a new feature or related to a new feature? (If yes, fix could go to + a new version.) + - Would this impact production workloads for users? (E.g. if this is a direct + runner only fix it may not need to be a cherry pick.) + - What percentage of users would be impacted by this issue if it is not fixed? + (E.g. If this is predicted to be a small number it may not need to be a + cherry pick.) + - Would it be possible for the impacted users to skip this version? (If users + could skip this version, fix could go to a newer version.) + +It is important to accept major/blocking fixes to isolated issues to make a +higher quality release. However, beyond that each cherry pick will increase +the time required for the release and add more last minute code to the release +branch. Neither late releases nor not fully tested code will provide positive +user value. + +> **Tip**: Another tool in your toolbox is the known issues section of the +> release blog. Consider adding known issues there for minor issues instead of +> accepting cherry picks to the release branch. + +## Build a release candidate + +From the release branch, building a candidate involves selecting a commit, +tagging that commit, and building the various artifacts against that commit. +You can also run verifications against the RC commit (verification will also +occur during voting phase). -Check if there are outstanding cherry-picks into the release branch, [e.g. for `2.14.0`](https://github.com/apache/beam/pulls?utf8=%E2%9C%93&q=is%3Apr+base%3Arelease-2.14.0). -Make sure they have blocker Issues attached and are OK to get into the release by checking with community if needed. +#### Checklist before proceeding -You are encouraged to ask the following questions to be answered on each cherry-pick PR and you can choose to reject cherry-pick requests if these questions are not satisfactorily answered: +- [ ] There are no release blocking GitHub issues. +- [ ] There are no open pull requests to release branch. +- [ ] Release Manager’s GPG key is published to `dist.apache.org`. +- [ ] Release Manager’s GPG key is configured in `git` configuration. +- [ ] Set `SIGNING_KEY` to the public key of the Manager's GPG key. +- [ ] Release Manager has `org.apache.beam` listed under `Staging Profiles` in Nexus. +- [ ] Release Manager’s Nexus User Token is configured in `settings.xml`. +- [ ] Set `JAVA_HOME` to JDK 8 (Example: `export JAVA_HOME=/example/path/to/java/jdk8`). +- [ ] Have Java 11 installed. -* Is this a regression from a previous release? (If no, fix could go to a newer version.) -* Is this a new feature or related to a new feature? (If yes, fix could go to a new version.) -* Would this impact production workloads for users? (E.g. if this is a direct runner only fix it may not need to be a cherry pick.) -* What percentage of users would be impacted by this issue if it is not fixed? (E.g. If this is predicted to be a small number it may not need to be a cherry pick.) -* Would it be possible for the impacted users to skip this version? (If users could skip this version, fix could go to a newer version.) +### Tag a chosen commit for the RC -It is important to accept major/blocking fixes to isolated issues to make a higher quality release. -However, beyond that each cherry pick will increase the time required for the release and add more last minute code to the release branch. -Neither late releases nor not fully tested code will provide positive user value. +Release candidates are built from single tagged commits off the release branch. +When you have identified a good commit on the release branch, run +[choose_rc_commit.sh](https://github.com/apache/beam/blob/master/release/src/main/scripts/choose_rc_commit.sh) +to set it up correctly. -__Tip__: Another tool in your toolbox is the known issues section of the release blog. -Consider adding known issues there for minor issues instead of accepting cherry picks to the release branch. + ./release/src/main/scripts/choose_rc_commit.sh \ + --release "${RELEASE_VERSION}" \ + --rc "${RC_NUM}" \ + --commit "${COMMIT_REF}" \ + --clone \ + --push-tag -### Build release artifacts +You can do a dry run by omitting the `--push-tag` flag. Then it will only clone +the repo, adjust the version, and add the tag locally. If it looks good, run it +again with `--push-tag`. If you already have a clone that includes the +`${COMMIT_REF}` then you can omit `--clone`. This is perfectly safe since the +script does not depend on the current working tree. -Once the branch is verified, it's time to build +See the source of the script for more details, or to run commands manually in +case of a problem. -#### Checklist before proceeding +The final state of the repository after an RC commit is chosen should match +this diagram: -- [ ] Release Manager’s GPG key is published to `dist.apache.org`; -- [ ] Release Manager’s GPG key is configured in `git` configuration; -- [ ] Set `SIGNING_KEY` to the public key of the Manager's GPG key; -- [ ] Release Manager has `org.apache.beam` listed under `Staging Profiles` in Nexus; -- [ ] Release Manager’s Nexus User Token is configured in `settings.xml`; -- [ ] GitHub issue release item for the subsequent release has been created; -- [ ] All test failures from branch verification have associated GitHub issues; -- [ ] There are no release blocking GitHub issues; -- [ ] Release branch has been created; -- [ ] There are no open pull requests to release branch; -- [ ] Originating branch has the version information updated to the new version; -- [ ] Nightly snapshot is in progress (do revisit it continually); -- [ ] Set `JAVA_HOME` to JDK 8 (Example: `export JAVA_HOME=/example/path/to/java/jdk8`). -- [ ] Have Java 11 installed. +Set version to non-SNAPSHOT, non-dev, on tagged RC commit -The core of the release process is the build-vote-fix cycle. -Each cycle produces one release candidate. -The Release Manager repeats this cycle until the community approves one release candidate, which is then finalized. +The following should be confirmed: -For this step, we recommend you using automation script to create a RC, but you still can perform all steps manually if you want. +- [ ] The release branch is unchanged. +- [ ] There is a commit not on the release branch with the version adjusted. +- [ ] The RC tag points to that commit. -#### Tag a chosen commit for the RC +### Run build_release_candidate GitHub Action to create a release candidate -Release candidates are built from single commits off the release branch. -Before building, the version must be set to a non-SNAPSHOT, non-dev version. -The final state of the repository should match this diagram: +Note: This step is partially automated (in progress), so part of the RC +creation is done by GitHub Actions and the rest is done by a script. You don't +need to wait for the action to complete to start running the script. -Set version to non-SNAPSHOT, non-dev, on tagged RC commit +**Action** [build_release_candidate](https://github.com/apache/beam/actions/workflows/build_release_candidate.yml) (click `run workflow`) -- The release branch is unchanged. -- There is a commit not on the release branch with the version adjusted. -- The RC tag points to that commit. +**The action will:** -* **Script:** [choose_rc_commit.sh](https://github.com/apache/beam/blob/master/release/src/main/scripts/choose_rc_commit.sh) +1. Clone the repo at the selected RC tag. +2. Run gradle publish to push java artifacts into Maven staging repo. +3. Stage SDK docker images to [docker hub Apache + organization](https://hub.docker.com/search?q=apache%2Fbeam&type=image). +4. Build javadoc, pydoc, typedocs for a PR to update beam-site. + - **NOTE**: Do not merge this PR until after an RC has been approved (see + "Finalize the Release"). -* **Usage** +### Verify source distributions - ./release/src/main/scripts/choose_rc_commit.sh \ - --release "${RELEASE_VERSION}" \ - --rc "${RC_NUM}" \ - --commit "${COMMIT_REF}" \ - --clone \ - --push-tag + - [ ] Verify that the source zip of the whole project is present in [dist.apache.org](https://dist.apache.org/repos/dist/dev/beam). + - [ ] Verify that the Python binaries are present in [dist.apache.org](https://dist.apache.org/repos/dist/dev/beam). -You can do a dry run by omitting the `--push-tag` flag. Then it will only clone the repo, -adjust the version, and add the tag locally. If it looks good, run it again with `--push-tag`. -If you already have a clone that includes the `${COMMIT_REF}` then you can omit `--clone`. This -is perfectly safe since the script does not depend on the current working tree. +### Verify docker images -See the source of the script for more details, or to run commands manually in case of a problem. +At +[https://hub.docker.com/u/apache](https://hub.docker.com/search?q=apache%2Fbeam&type=image), +visit each repository and navigate to "tags" tab. Verify images are pushed +with tags: `${RELEASE_VERSION}_rc{RC_NUM}` -#### Run build_release_candidate GitHub Action to create a release candidate +Verify that third party licenses are included in Docker. You can do this with a simple script: -Note: This step is partially automated (in progress), so part of the rc creation is done by GitHub Actions and the rest is done by a script. -You don't need to wait for the action to complete to start running the script. + for pyver in 3.8 3.9 3.10 3.11; do + docker run --rm --entrypoint sh \ + apache/beam_python${pyver}_sdk:2.51.0rc1 \ + -c 'ls -al /opt/apache/beam/third_party_licenses/ | wc -l' + done -* **Action** [build_release_candidate](https://github.com/apache/beam/actions/workflows/build_release_candidate.yml) (click `run workflow`) + for javaver in 8 11 17; do + docker run --rm --entrypoint sh \ + apache/beam_java${pyver}_sdk:2.51.0rc1 \ + -c 'ls -al /opt/apache/beam/third_party_licenses/ | wc -l' + done -* **The script will:** - 1. Clone the repo at the selected RC tag. - 2. Run gradle publish to push java artifacts into Maven staging repo. - 3. Stage SDK docker images to [docker hub Apache organization](https://hub.docker.com/search?q=apache%2Fbeam&type=image). - 4. Build javadoc, pydoc, typedocs for a PR to update beam-site. - * **NOTE**: Do not merge this PR until after an RC has been approved (see "Finalize the Release"). +And you may choose to log in to the containers and inspect: -##### Tasks you need to do manually + docker run --rm -it --entrypoint=/bin/bash \ + apache/beam_java${ver}_sdk:${RELEASE_VERSION}rc${RC_NUM} + ls -al /opt/apache/beam/third_party_licenses/ + +### Publish Java staging artifacts (manual) -Publish staging artifacts 1. Log in to the [Apache Nexus](https://repository.apache.org/#stagingRepositories) website. 2. Navigate to Build Promotion -> Staging Repositories (in the left sidebar). 3. Select repository `orgapachebeam-NNNN`. 4. Click the Close button. 5. When prompted for a description, enter “Apache Beam, version X, release candidate Y”. - 6. Review all staged artifacts on `https://repository.apache.org/content/repositories/orgapachebeam-NNNN/`. + 6. Review all staged artifacts on `https://repository.apache.org/content/repositories/orgapachebeam-NNNN/`. They should contain all relevant parts for each module, including `pom.xml`, jar, test jar, javadoc, etc. Artifact names should follow [the existing format](https://search.maven.org/#search%7Cga%7C1%7Cg%3A%22org.apache.beam%22) in which artifact name mirrors directory structure, e.g., `beam-sdks-java-io-kafka`. Carefully review any new artifacts. Some additional validation should be done during the rc validation step. -#### Run build_release_candidate.sh to create a release candidate -* **Script:** [build_release_candidate.sh](https://github.com/apache/beam/blob/master/release/src/main/scripts/build_release_candidate.sh) +### Upload `rc` artifacts to PyPI -* **Usage** +This step uploads artifacts such as `apache-beam-${RELEASE_VERSION}rc${RC_NUM}` +to PyPI, so the RC artifacts can be depended upon directly by consumers, for +ease of RC verification. - ./release/src/main/scripts/build_release_candidate.sh --release "${RELEASE_VERSION}" --rc "${RC_NUM}" --github-user "${GITHUB_USER}" --java11-home "${JAVA11_HOME}" --signing-key "${SIGNING_KEY}" - -* **The script will:** - 1. Clone the repo at the selected RC tag. - 2. Stage source release into dist.apache.org dev [repo](https://dist.apache.org/repos/dist/dev/beam/). -Skip this step if you already did it with the build_release_candidate GitHub Actions workflow. - 3. Stage, sign and hash python source distribution and wheels into dist.apache.org dev repo python dir - 4. Stage SDK docker images to [docker hub Apache organization](https://hub.docker.com/search?q=apache%2Fbeam&type=image). -Skip this step if you already did it with the build_release_candidate GitHub Actions workflow. -Note: if you are not a member of the [`beam` DockerHub team](https://hub.docker.com/orgs/apache/teams/beam) you will need -help with this step. Please email `dev@` mailing list and ask a member of the `beam` DockerHub team for help. - 5. Create a PR to update beam-site, changes includes: - * Copy python doc into beam-site - * Copy java doc into beam-site - * **NOTE**: Do not merge this PR until after an RC has been approved (see "Finalize the Release"). -Skip this step if you already did it with the build_release_candidate GitHub Actions workflow. - -##### Tasks you need to do manually - -Verify the script worked. - - 1. Verify that the source and Python binaries are present in [dist.apache.org](https://dist.apache.org/repos/dist/dev/beam). - 2. Verify Docker images are published. How to find images: - 1. Visit [https://hub.docker.com/u/apache](https://hub.docker.com/search?q=apache%2Fbeam&type=image) - 2. Visit each repository and navigate to *tags* tab. - 3. Verify images are pushed with tags: ${RELEASE_VERSION}_rc{RC_NUM} - 3. Verify that third party licenses are included in Docker containers by logging in to the images. - - For Python SDK images, there should be around 80 ~ 100 dependencies. - Please note that dependencies for the SDKs with different Python versions vary. - Need to verify all Python images by replacing `${ver}` with each supported Python version `X.Y`. - ``` - docker run --rm -it --entrypoint=/bin/bash apache/beam_python${ver}_sdk:${RELEASE_VERSION}rc${RC_NUM} - ls -al /opt/apache/beam/third_party_licenses/ | wc -l - ``` - - For Java SDK images, there should be around 200 dependencies. - ``` - docker run --rm -it --entrypoint=/bin/bash apache/beam_java${ver}_sdk:${RELEASE_VERSION}rc${RC_NUM} - ls -al /opt/apache/beam/third_party_licenses/ | wc -l - ``` - -#### Upload release candidate to PyPi - -* **Script:** [deploy_release_candidate_pypi.sh](https://github.com/apache/beam/blob/master/release/src/main/scripts/deploy_release_candidate_pypi.sh) +**Script:** [deploy_release_candidate_pypi.sh](https://github.com/apache/beam/blob/master/release/src/main/scripts/deploy_release_candidate_pypi.sh) -* **Usage** +**Usage** ./release/src/main/scripts/deploy_release_candidate_pypi.sh \ --release "${RELEASE_VERSION}" \ @@ -626,62 +641,79 @@ Verify the script worked. --user "${GITHUB_USER}" \ --deploy -* **The script will:** +**The script will:** -1. Download python binary artifacts -2. Deploy release candidate to PyPI +Download previously build python binary artifacts Deploy release candidate +to PyPI with an `rc` suffix. __Attention:__ Verify that: -* The File names version include ``rc-#`` suffix -* [Download Files](https://pypi.org/project/apache-beam/#files) have: - * All wheels uploaded as artifacts - * Release source's zip published - * Signatures and hashes do not need to be uploaded +- [ ] The File names version include ``rc-#`` suffix +- [ ] [Download Files](https://pypi.org/project/apache-beam/#files) have: + - [ ] All wheels uploaded as artifacts + - [ ] Release source's zip published + - [ ] Signatures and hashes do not need to be uploaded -You can do a dry run by omitting the `--deploy` flag. Then it will only download the release candidate binaries. If it looks good, rerun it with `--deploy`. +You can do a dry run by omitting the `--deploy` flag. Then it will only +download the release candidate binaries. If it looks good, rerun it with +`--deploy`. -See the source of the script for more details or to run commands manually in case of a problem. +See the source of the script for more details or to run commands manually in +case of a problem. ### Propose pull requests for website updates -Beam publishes API reference manuals for each release on the website. -For Java and Python SDKs, that’s Javadoc and PyDoc, respectively. -The final step of building the candidate is to propose website pull requests that update these manuals. +Beam publishes API reference manuals for each release on the website. For Java +and Python SDKs, that’s Javadoc and PyDoc, respectively. The final step of +building the candidate is to propose website pull requests that update these +manuals. -Merge the pull requests only after finalizing the release. -To avoid invalid redirects for the 'current' version, merge these PRs in the order listed. -Once the PR is merged, the new contents will get picked up automatically and served to the Beam website, usually within an hour. -A committer can manually trigger the [beam_PostCommit_Website_Publish](https://ci-beam.apache.org/job/beam_PostCommit_Website_Publish/) task in Jenkins to avoid waiting. +Merge the pull requests only after finalizing the release. To avoid invalid +redirects for the 'current' version, merge these PRs in the order listed. Once +the PR is merged, the new contents will get picked up automatically and served +to the Beam website, usually within an hour. A committer can manually trigger +the +[beam_PostCommit_Website_Publish](https://ci-beam.apache.org/job/beam_PostCommit_Website_Publish/) +task in Jenkins to avoid waiting. **PR 1: apache/beam-site** -This pull request is against the `apache/beam-site` repo, on the `release-docs` branch ([example](https://github.com/apache/beam-site/pull/603)). -It is created by the `build_release_candidate` workflow (see above). +This pull request is against the `apache/beam-site` repo, on the `release-docs` +branch ([example](https://github.com/apache/beam-site/pull/603)). It is +created by the `build_release_candidate` workflow (see above). **PR 2: apache/beam** -This pull request is against the `apache/beam` repo, on the `master` branch ([example](https://github.com/apache/beam/pull/17378)). +This pull request is against the `apache/beam` repo, on the `master` branch +([example](https://github.com/apache/beam/pull/17378)). -* Update `CHANGES.md` to update release date and remove template. -* Update release version in `website/www/site/config.toml`. -* Add new release in `website/www/site/content/en/get-started/downloads.md`. - * Download links will not work until the release is finalized. -* Update links to prior releases to point to https://archive.apache.org (see +- Update `CHANGES.md` to update release date and remove template. +- Update release version in `website/www/site/config.toml`. +- Add new release in `website/www/site/content/en/get-started/downloads.md`. + - Download links will not work until the release is finalized. +- Update links to prior releases to point to https://archive.apache.org (see example PR). -* Create the Blog post: +- Create the Blog post: #### Blog post -Use the template below to write a blog post for the release. -See [beam-2.31.0.md](https://github.com/apache/beam/commit/a32a75ed0657c122c6625aee1ace27994e7df195#diff-1e2b83a4f61dce8014a1989869b6d31eb3f80cb0d6dade42fb8df5d9407b4748) as an example. -- Copy the changes for the current release from `CHANGES.md` to the blog post and edit as necessary. -- Be sure to add yourself to [authors.yml](https://github.com/apache/beam/blob/master/website/www/site/data/authors.yml) if necessary. +Use the template below to write a blog post for the release. See +[beam-2.31.0.md](https://github.com/apache/beam/commit/a32a75ed0657c122c6625aee1ace27994e7df195#diff-1e2b83a4f61dce8014a1989869b6d31eb3f80cb0d6dade42fb8df5d9407b4748) +as an example. + +- Copy the changes for the current release from `CHANGES.md` to the blog post + and edit as necessary. +- Be sure to add yourself to + [authors.yml](https://github.com/apache/beam/blob/master/website/www/site/data/authors.yml) + if necessary. -__Tip__: Use git log to find contributors to the releases. (e.g: `git fetch origin --tags; git log --pretty='%aN' ^v2.10.0 v2.11.0-RC1 | sort | uniq`). -Make sure to clean it up, as there may be duplicate or incorrect user names. +> **TIP** +> Use git log to find contributors to the releases. (e.g: `git fetch +> origin --tags; git log --pretty='%aN' ^v2.10.0 v2.11.0-RC1 | sort | uniq`). +> Make sure to clean it up, as there may be duplicate or incorrect user names. -__NOTE__: Make sure to include any breaking changes, even to `@Experimental` features, -all major features and bug fixes, and all known issues. +> **NOTE** +> Make sure to include any breaking changes, even to `@Experimental` +> features, all major features and bug fixes, and all known issues. **Template:** @@ -755,34 +787,46 @@ all major features and bug fixes, and all known issues. ${CONTRIBUTORS} -### Checklist to proceed to the next step +### Checklist to proceed to the next phase -- [ ] Maven artifacts deployed to the staging repository of [repository.apache.org](https://repository.apache.org/content/repositories/) -. Source distribution deployed to the dev repository of [dist.apache.org](https://dist.apache.org/repos/dist/dev/beam/) -- [ ] Website pull request proposed to list the [release](/get-started/downloads/), publish the [Java API reference manual](https://beam.apache.org/releases/javadoc/), and publish the [Python API reference manual](https://beam.apache.org/releases/pydoc/). -- [ ] Docker images are published to [DockerHub](https://hub.docker.com/search?q=apache%2Fbeam&type=image) with tags: {RELEASE_VERSION}_rc{RC_NUM}. +- [ ] Maven artifacts deployed to the staging repository of + [repository.apache.org](https://repository.apache.org/content/repositories/) +- [ ] Source distribution deployed to the dev repository of + [dist.apache.org](https://dist.apache.org/repos/dist/dev/beam/) +- [ ] Website pull request proposed to list the + [release](/get-started/downloads/), publish the [Java API reference + manual](https://beam.apache.org/releases/javadoc/), and publish the [Python + API reference manual](https://beam.apache.org/releases/pydoc/). +- [ ] Docker images are published to + [DockerHub](https://hub.docker.com/search?q=apache%2Fbeam&type=image) with + tags: `{RELEASE_VERSION}_rc{RC_NUM}`. You can (optionally) also do additional verification by: -- [ ] Check that Python zip file contains the `README.md`, `NOTICE`, and `LICENSE` files. -- [ ] Check hashes (e.g. `md5sum -c *.md5` and `sha1sum -c *.sha1`. Note that signature/checksum files of Java artifacts may not contain filenames. Hence you might need to compare checksums/signatures manually or modify the files by appending the filenames.) -- [ ] Check signatures (e.g. `gpg --verify apache-beam-1.2.3-python.zip.asc apache-beam-1.2.3-python.zip`) + +- [ ] Check that Python zip file contains the `README.md`, `NOTICE`, and + `LICENSE` files. +- [ ] Check hashes (e.g. `md5sum -c *.md5` and `sha1sum -c *.sha1`. Note that + signature/checksum files of Java artifacts may not contain filenames. Hence + you might need to compare checksums/signatures manually or modify the files by + appending the filenames.) +- [ ] Check signatures (e.g. `gpg --verify apache-beam-1.2.3-python.zip.asc + apache-beam-1.2.3-python.zip`) - [ ] `grep` for legal headers in each file. -- [ ] Run all jenkins suites and include links to passing tests in the voting email. -- [ ] Pull docker images to make sure they are pullable. -``` -docker pull {image_name} -docker pull apache/beam_python3.7_sdk:2.39.0rc1 -``` +- [ ] Run all jenkins suites and include links to passing tests in the voting + email. +- [ ] Pull docker images to make sure they are pullable. (e.g. `docker pull apache/beam_python3.7_sdk:2.39.0rc1` ********** -## 4. Vote and validate release candidate +## Vote and validate the release candidate -Once you have built and individually reviewed the release candidate, please share it for the community-wide review. -Please review foundation-wide [voting guidelines](https://www.apache.org/foundation/voting.html) for more information. +Once you have built and individually reviewed the release candidate, please +share it for the community-wide review. Please review foundation-wide [voting +guidelines](https://www.apache.org/foundation/voting.html) for more +information. -Start the review-and-vote thread on the dev@ mailing list. -Here’s an email template; please adjust as you see fit. +Start the review-and-vote thread on the dev@ mailing list. Here’s an email +template; please adjust as you see fit. From: Release Manager To: dev@beam.apache.org @@ -832,28 +876,43 @@ Here’s an email template; please adjust as you see fit. [11] https://hub.docker.com/search?q=apache%2Fbeam&type=image [12] https://github.com/apache/beam/pull/... -If there are any issues found in the release candidate, reply on the vote thread to cancel the vote. -There’s no need to wait 72 hours. -Proceed to the `Fix issues` step below and address the problem. -However, some issues don’t require cancellation. -For example, if an issue is found in the website pull request, just correct it on the spot and the vote can continue as-is. +If there are any issues found in the release candidate, reply on the vote +thread to cancel the vote. There’s no need to wait 72 hours. Go back to +["Stabilize the Release Branch"](#stabilize-the-release-branch) and address the problem. However, some issues +don’t require cancellation. For example, if an issue is found in the website +pull request, just correct it on the spot and the vote can continue as-is. ### Run validation tests -The community is responsible for performing validation, but as release manager you are expected to contribute as well. -Before accepting an RC, as a community we try to exercise most (if not all) of the tests listed in this -[spreadsheet](https://s.apache.org/beam-release-validation), and those are good validations for you to try out as release manager. -The goal of these tests is to validate that we're able to run basic pipelines from a variety of environments (not just our CI environment). -Since there are many tests, we recommend you running some validations using an automation script. -In case of script failure, you can still run all of them manually. +The community is responsible for performing validation, but as release manager +you are expected to contribute as well. + +Before accepting an RC, as a community we try to exercise most (if not all) of +the tests listed in this +[spreadsheet](https://s.apache.org/beam-release-validation), and those are good +validations for you to try out as release manager. The goal of these tests is +to validate that we're able to run basic pipelines from a variety of +environments (not just our CI environment). + +Since there are many tests, we recommend you running some validations using an +automation script. In case of script failure, you can still run all of them +manually. You may need to have Python interpreters for all supported Python minor -versions to run all of the tests. See Python installation tips in [Developer Wiki](https://cwiki.apache.org/confluence/display/BEAM/Python+Tips#PythonTips-InstallingPythoninterpreters). +versions to run all of the tests. See Python installation tips in [Developer +Wiki](https://cwiki.apache.org/confluence/display/BEAM/Python+Tips#PythonTips-InstallingPythoninterpreters). + +> **Note** +> The community's validation means more than just running the tests +> that we have already run. It includes users trying out the RC on their own +> downstream tests. It also includes double checking that our human-language +> instructions actually still correspond to the automation that we have built. #### Run validations using run_rc_validation.sh -* **Script:** [run_rc_validation.sh](https://github.com/apache/beam/blob/master/release/src/main/scripts/run_rc_validation.sh) -* **Usage** +**Script:** [run_rc_validation.sh](https://github.com/apache/beam/blob/master/release/src/main/scripts/run_rc_validation.sh) + +**Usage** 1. First update required configurations listed in `RC_VALIDATE_CONFIGS` in [script.config](https://github.com/apache/beam/blob/master/release/src/main/scripts/script.config) 2. Then run @@ -864,7 +923,7 @@ versions to run all of the tests. See Python installation tips in [Developer Wik **Note:** running the validations requires the ability to do the following in your GCP account: start pipelines, write to BigQuery, and create a cluster of machines for running containers (for x-lang validation). -* **Tasks included** +**Tasks included** 1. Create a PR to trigger Python validation job, including * Python quickstart in batch and streaming mode with direct runner and Dataflow runner. * Python Mobile Games(UserScore, HourlyTeamScore) with direct runner and Dataflow runner. @@ -880,22 +939,25 @@ write to BigQuery, and create a cluster of machines for running containers (for * Start a new terminal to run Python multi-language Java kafka validation with Dataflow Runner. * Start a new terminal to run Python multi-language Java sql validation with Dataflow Runner. -* **Tasks you need to do manually** - 1. Check whether validations succeed by following console output instructions. - 1. Terminate streaming jobs and java injector. - 1. Run Java quickstart (wordcount) and mobile game examples with the staged artifacts. The easiest way to do this is by running the tests on Jenkins. -Other manual validation will follow, but this will at least validate that the staged artifacts can be used. +* **Tasks you need to do manually**. + +- [ ] Check whether validations succeed by following console output instructions. +- [ ] Terminate streaming jobs and java injector. +- [ ] Run Java quickstart (wordcount) and mobile game examples with the staged artifacts. The easiest way to do this is by running the tests on Jenkins. + +- Other manual validation will follow, but this will at least validate that the staged artifacts can be used. * Log in to Jenkins. * Go to https://ci-beam.apache.org/job/beam_PostRelease_NightlySnapshot/. * Click "Build with Parameters". * Set `snapshot_version` to `2.xx.0`, and set `snapshot_url` to point to the staged artifacts in Maven central (https://repository.apache.org/content/repositories/orgapachebeam-NNNN/). * Click "Build". - 1. Sign up [spreadsheet](https://s.apache.org/beam-release-validation). - 1. Vote in the release thread. +- [ ] Sign up [spreadsheet](https://s.apache.org/beam-release-validation). +- [ ] Vote in the release thread. #### Run validations manually -_Note_: -Prepourl and -Pver can be found in the RC vote email sent by Release Manager. +> **Note** +> `-Prepourl` and `-Pver` can be found in the RC vote email sent by Release Manager. * **Java Quickstart Validation** @@ -1103,25 +1165,17 @@ _Note_: -Prepourl and -Pver can be found in the RC vote email sent by Release Ma * bq head -n 10 ${USER}_test.game_stats_sessions -### Fix issues - -Any issues identified during the community review and vote should be fixed in this step. -Additionally, any GitHub issues created from the initial branch verification should be fixed. - -Code changes should be proposed as standard pull requests to the `master` branch and reviewed using the normal contributing process. -Then, relevant changes should be cherry-picked into the release branch proposed as pull requests against the release branch, again reviewed and merged using the normal contributing process. - -Once all issues have been resolved as in the `Verify release branch` step, you should go back and build a new release candidate with these changes. - ### Finalize the vote -Reply on the vote thread to close the voting once following conditions are met for the current release candidate. -* At least 72 hours has passed since the voting email. -* No release blocking issues have been identified. -* Voting thread has at least three approving PMC votes. +Reply on the vote thread to close the voting once following conditions are met +for the current release candidate. -Then, tally the votes in a separate email thread. -Here’s an email template; please adjust as you see fit. +- [ ] At least 72 hours has passed since the voting email. +- [ ] No release blocking issues have been identified. +- [ ] Voting thread has at least three approving PMC votes. + +Then, tally the votes in a separate email thread. Here’s an email template; +please adjust as you see fit. From: Release Manager To: dev@beam.apache.org @@ -1146,9 +1200,9 @@ Here’s an email template; please adjust as you see fit. - [ ] Community votes to release the proposed candidate, with at least three approving PMC votes. -********** +---- -## 5. Finalize the release +## Finalize the release Once the release candidate has been reviewed and approved by the community, the release should be finalized. This involves the final deployment of the release candidate to the release repositories, merging of the website changes, etc. @@ -1267,7 +1321,7 @@ Use [reporter.apache.org](https://reporter.apache.org/addrelease.html?beam) to s ********** -## 6. Promote the release +## Promote the release Once the release has been finalized, the last step of the process is to promote the release within the project and beyond. @@ -1298,7 +1352,9 @@ Also, update [the Wikipedia article on Apache Beam](https://en.wikipedia.org/wik ********** -## Post Release Tasks +## Post-Release Tasks + +At the end of the release, go to the GitHub milestones page and mark the recently released version as closed. ### Update Beam Playground From 7ac5c175871d469d2e03ccc19469f2fa93da4306 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Mon, 25 Sep 2023 17:23:42 -0400 Subject: [PATCH 055/435] Update Google Cloud Java Libraries BOM --- .../org/apache/beam/gradle/BeamModulePlugin.groovy | 14 +++++++------- .../container/license_scripts/dep_urls_java.yaml | 2 +- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index 3a332d8d4d08..061a63d1670e 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -583,12 +583,12 @@ class BeamModulePlugin implements Plugin { def dbcp2_version = "2.9.0" def errorprone_version = "2.10.0" // Try to keep gax_version consistent with gax-grpc version in google_cloud_platform_libraries_bom - def gax_version = "2.32.0" + def gax_version = "2.33.0" def google_ads_version = "26.0.0" def google_clients_version = "2.0.0" def google_cloud_bigdataoss_version = "2.2.16" // Try to keep google_cloud_spanner_version consistent with google_cloud_spanner_bom in google_cloud_platform_libraries_bom - def google_cloud_spanner_version = "6.45.0" + def google_cloud_spanner_version = "6.47.0" def google_code_gson_version = "2.10.1" def google_oauth_clients_version = "1.34.1" // Try to keep grpc_version consistent with gRPC version in google_cloud_platform_libraries_bom @@ -712,14 +712,14 @@ class BeamModulePlugin implements Plugin { google_api_client_java6 : "com.google.api-client:google-api-client-java6:$google_clients_version", google_api_common : "com.google.api:api-common", // google_cloud_platform_libraries_bom sets version // Keep version consistent with the version in google_cloud_bigquery, managed by google_cloud_platform_libraries_bom - google_api_services_bigquery : "com.google.apis:google-api-services-bigquery:v2-rev20230520-$google_clients_version", + google_api_services_bigquery : "com.google.apis:google-api-services-bigquery:v2-rev20230812-$google_clients_version", // Keep version consistent with the version in google_cloud_resourcemanager, managed by google_cloud_platform_libraries_bom - google_api_services_cloudresourcemanager : "com.google.apis:google-api-services-cloudresourcemanager:v1-rev20230129-$google_clients_version", + google_api_services_cloudresourcemanager : "com.google.apis:google-api-services-cloudresourcemanager:v1-rev20230806-$google_clients_version", google_api_services_dataflow : "com.google.apis:google-api-services-dataflow:v1b3-rev20220920-$google_clients_version", google_api_services_healthcare : "com.google.apis:google-api-services-healthcare:v1-rev20230830-$google_clients_version", google_api_services_pubsub : "com.google.apis:google-api-services-pubsub:v1-rev20220904-$google_clients_version", // Keep version consistent with the version in google_cloud_nio, managed by google_cloud_platform_libraries_bom - google_api_services_storage : "com.google.apis:google-api-services-storage:v1-rev20230617-$google_clients_version", + google_api_services_storage : "com.google.apis:google-api-services-storage:v1-rev20230907-$google_clients_version", google_auth_library_credentials : "com.google.auth:google-auth-library-credentials", // google_cloud_platform_libraries_bom sets version google_auth_library_oauth2_http : "com.google.auth:google-auth-library-oauth2-http", // google_cloud_platform_libraries_bom sets version google_cloud_bigquery : "com.google.cloud:google-cloud-bigquery", // google_cloud_platform_libraries_bom sets version @@ -732,14 +732,14 @@ class BeamModulePlugin implements Plugin { google_cloud_datacatalog_v1beta1 : "com.google.cloud:google-cloud-datacatalog", // google_cloud_platform_libraries_bom sets version google_cloud_dataflow_java_proto_library_all: "com.google.cloud.dataflow:google-cloud-dataflow-java-proto-library-all:0.5.160304", // Keep version consistent with the version in google_cloud_datastore, managed by google_cloud_platform_libraries_bom - google_cloud_datastore_v1_proto_client : "com.google.cloud.datastore:datastore-v1-proto-client:2.16.3", + google_cloud_datastore_v1_proto_client : "com.google.cloud.datastore:datastore-v1-proto-client:2.17.1", google_cloud_firestore : "com.google.cloud:google-cloud-firestore", // google_cloud_platform_libraries_bom sets version google_cloud_pubsub : "com.google.cloud:google-cloud-pubsub", // google_cloud_platform_libraries_bom sets version google_cloud_pubsublite : "com.google.cloud:google-cloud-pubsublite", // google_cloud_platform_libraries_bom sets version // The release notes shows the versions set by the BOM: // https://github.com/googleapis/java-cloud-bom/releases/tag/v26.21.0 // Update libraries-bom version on sdks/java/container/license_scripts/dep_urls_java.yaml - google_cloud_platform_libraries_bom : "com.google.cloud:libraries-bom:26.22.0", + google_cloud_platform_libraries_bom : "com.google.cloud:libraries-bom:26.23.0", google_cloud_spanner : "com.google.cloud:google-cloud-spanner", // google_cloud_platform_libraries_bom sets version google_cloud_spanner_test : "com.google.cloud:google-cloud-spanner:$google_cloud_spanner_version:tests", google_code_gson : "com.google.code.gson:gson:$google_code_gson_version", diff --git a/sdks/java/container/license_scripts/dep_urls_java.yaml b/sdks/java/container/license_scripts/dep_urls_java.yaml index ca47184d2a22..8a028f459727 100644 --- a/sdks/java/container/license_scripts/dep_urls_java.yaml +++ b/sdks/java/container/license_scripts/dep_urls_java.yaml @@ -46,7 +46,7 @@ jaxen: '1.1.6': type: "3-Clause BSD" libraries-bom: - '26.22.0': + '26.23.0': license: "https://raw.githubusercontent.com/GoogleCloudPlatform/cloud-opensource-java/master/LICENSE" type: "Apache License 2.0" paranamer: From fc93a36d869ad491cdf065fdb2b96dee222b4a69 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Thu, 5 Oct 2023 10:46:04 -0700 Subject: [PATCH 056/435] Make parquetio schema aware. --- sdks/python/apache_beam/io/parquetio.py | 99 ++++++++++++++----- sdks/python/apache_beam/io/parquetio_test.py | 16 +++ sdks/python/apache_beam/pvalue.py | 3 + .../typehints/arrow_type_compatibility.py | 6 +- 4 files changed, 97 insertions(+), 27 deletions(-) diff --git a/sdks/python/apache_beam/io/parquetio.py b/sdks/python/apache_beam/io/parquetio.py index 734dfa0bfe8e..4696e5ae7927 100644 --- a/sdks/python/apache_beam/io/parquetio.py +++ b/sdks/python/apache_beam/io/parquetio.py @@ -31,27 +31,34 @@ # pytype: skip-file from functools import partial +from typing import Iterator from packaging import version from apache_beam.io import filebasedsink from apache_beam.io import filebasedsource from apache_beam.io.filesystem import CompressionTypes +from apache_beam.io.filesystems import FileSystems from apache_beam.io.iobase import RangeTracker from apache_beam.io.iobase import Read from apache_beam.io.iobase import Write +from apache_beam.portability.api import schema_pb2 from apache_beam.transforms import DoFn from apache_beam.transforms import ParDo from apache_beam.transforms import PTransform from apache_beam.transforms import window +from apache_beam.typehints import schemas try: import pyarrow as pa import pyarrow.parquet as pq + # pylint: disable=ungrouped-imports + from apache_beam.typehints import arrow_type_compatibility except ImportError: pa = None pq = None ARROW_MAJOR_VERSION = None + arrow_type_compatibility = None else: base_pa_version = version.parse(pa.__version__).base_version ARROW_MAJOR_VERSION, _, _ = map(int, base_pa_version.split('.')) @@ -146,6 +153,24 @@ def _flush_buffer(self): self._record_batches_byte_size = self._record_batches_byte_size + size +class _ArrowTableToBeamRows(DoFn): + def __init__(self, beam_type): + self._beam_type = beam_type + + @DoFn.yields_batches + def process(self, element) -> Iterator[pa.Table]: + yield element + + def infer_output_type(self, input_type): + return self._beam_type + + +class _BeamRowsToArrowTable(DoFn): + @DoFn.yields_elements + def process_batch(self, element: pa.Table) -> Iterator[pa.Table]: + yield element + + class ReadFromParquetBatched(PTransform): """A :class:`~apache_beam.transforms.ptransform.PTransform` for reading Parquet files as a `PCollection` of `pyarrow.Table`. This `PTransform` is @@ -191,7 +216,7 @@ def __init__( """ super().__init__() - self._source = _create_parquet_source( + self._source = _ParquetSource( file_pattern, min_bundle_size, validate=validate, @@ -210,7 +235,12 @@ class ReadFromParquet(PTransform): Parquet files as a `PCollection` of dictionaries. This `PTransform` is currently experimental. No backward-compatibility guarantees.""" def __init__( - self, file_pattern=None, min_bundle_size=0, validate=True, columns=None): + self, + file_pattern=None, + min_bundle_size=0, + validate=True, + columns=None, + as_rows=False): """Initializes :class:`ReadFromParquet`. Uses source ``_ParquetSource`` to read a set of Parquet files defined by @@ -255,17 +285,38 @@ def __init__( columns (List[str]): list of columns that will be read from files. A column name may be a prefix of a nested field, e.g. 'a' will select 'a.b', 'a.c', and 'a.d.e' + as_rows (bool): whether to output a schema'd PCollection of Beam rows + rather than Python dictionaries. """ super().__init__() - self._source = _create_parquet_source( + self._source = _ParquetSource( file_pattern, min_bundle_size, validate=validate, columns=columns, ) + if as_rows: + if columns is None: + filter_schema = lambda schema: schema + else: + top_level_columns = set(c.split('.')[0] for c in columns) + filter_schema = lambda schema: schema_pb2.Schema( + fields=[f for f in schema.fields if f.name in top_level_columns]) + path = FileSystems.match([file_pattern], [1])[0].metadata_list[0].path + with FileSystems.open(path) as fin: + self._schema = filter_schema( + arrow_type_compatibility.beam_schema_from_arrow_schema( + pq.read_schema(fin))) + else: + self._schema = None def expand(self, pvalue): - return pvalue | Read(self._source) | ParDo(_ArrowTableToRowDictionaries()) + arrow_batches = pvalue | Read(self._source) + if self._schema is None: + return arrow_batches | ParDo(_ArrowTableToRowDictionaries()) + else: + return arrow_batches | ParDo( + _ArrowTableToBeamRows(schemas.named_tuple_from_schema(self._schema))) def display_data(self): return {'source_dd': self._source} @@ -305,9 +356,7 @@ def __init__( """ super().__init__() source_from_file = partial( - _create_parquet_source, - min_bundle_size=min_bundle_size, - columns=columns) + _ParquetSource, min_bundle_size=min_bundle_size, columns=columns) self._read_all_files = filebasedsource.ReadAllFiles( True, CompressionTypes.UNCOMPRESSED, @@ -333,17 +382,6 @@ def expand(self, pvalue): _ArrowTableToRowDictionaries(), with_filename=self._with_filename) -def _create_parquet_source( - file_pattern=None, min_bundle_size=0, validate=False, columns=None): - return \ - _ParquetSource( - file_pattern=file_pattern, - min_bundle_size=min_bundle_size, - validate=validate, - columns=columns, - ) - - class _ParquetUtils(object): @staticmethod def find_first_row_group_index(pf, start_offset): @@ -370,7 +408,8 @@ def get_number_of_row_groups(pf): class _ParquetSource(filebasedsource.FileBasedSource): """A source for reading Parquet files. """ - def __init__(self, file_pattern, min_bundle_size, validate, columns): + def __init__( + self, file_pattern, min_bundle_size=0, validate=False, columns=None): super().__init__( file_pattern=file_pattern, min_bundle_size=min_bundle_size, @@ -421,6 +460,9 @@ def split_points_unclaimed(stop_position): yield table +_create_parquet_source = _ParquetSource + + class WriteToParquet(PTransform): """A ``PTransform`` for writing parquet files. @@ -430,7 +472,7 @@ class WriteToParquet(PTransform): def __init__( self, file_path_prefix, - schema, + schema=None, row_group_buffer_size=64 * 1024 * 1024, record_batch_size=1000, codec='none', @@ -534,10 +576,19 @@ def __init__( ) def expand(self, pcoll): - return pcoll | ParDo( - _RowDictionariesToArrowTable( - self._schema, self._row_group_buffer_size, - self._record_batch_size)) | Write(self._sink) + if self._schema is None: + try: + beam_schema = schemas.schema_from_element_type(pcoll.element_type) + except TypeError as exn: + raise ValueError( + "A schema is required to write non-schema'd data.") from exn + self._sink._schema = ( + arrow_type_compatibility.arrow_schema_from_beam_schema(beam_schema)) + convert_fn = _BeamRowsToArrowTable() + else: + convert_fn = _RowDictionariesToArrowTable( + self._schema, self._row_group_buffer_size, self._record_batch_size) + return pcoll | ParDo(convert_fn) | Write(self._sink) def display_data(self): return { diff --git a/sdks/python/apache_beam/io/parquetio_test.py b/sdks/python/apache_beam/io/parquetio_test.py index df018a3a776f..1cd5f1208cc2 100644 --- a/sdks/python/apache_beam/io/parquetio_test.py +++ b/sdks/python/apache_beam/io/parquetio_test.py @@ -30,6 +30,7 @@ from parameterized import param from parameterized import parameterized +import apache_beam as beam from apache_beam import Create from apache_beam import Map from apache_beam.io import filebasedsource @@ -400,6 +401,21 @@ def test_sink_transform_compliant_nested_type(self): assert_that( readback, equal_to([json.dumps(r) for r in self.RECORDS_NESTED])) + def test_schema_read_write(self): + with TemporaryDirectory() as tmp_dirname: + path = os.path.join(tmp_dirname, 'tmp_filename') + rows = [beam.Row(a=1, b='x'), beam.Row(a=2, b='y')] + stable_repr = lambda row: json.dumps(row._asdict()) + with TestPipeline() as p: + _ = p | Create(rows) | WriteToParquet(path) | beam.Map(print) + with TestPipeline() as p: + # json used for stable sortability + readback = ( + p + | ReadFromParquet(path + '*', as_rows=True) + | Map(stable_repr)) + assert_that(readback, equal_to([stable_repr(r) for r in rows])) + def test_batched_read(self): with TemporaryDirectory() as tmp_dirname: path = os.path.join(tmp_dirname + "tmp_filename") diff --git a/sdks/python/apache_beam/pvalue.py b/sdks/python/apache_beam/pvalue.py index 2e86c9eb51c7..90882651d0b2 100644 --- a/sdks/python/apache_beam/pvalue.py +++ b/sdks/python/apache_beam/pvalue.py @@ -673,6 +673,9 @@ def __init__(self, **kwargs): def as_dict(self): return dict(self.__dict__) + # For compatibility with named tuples. + _asdict = as_dict + def __iter__(self): for _, value in self.__dict__.items(): yield value diff --git a/sdks/python/apache_beam/typehints/arrow_type_compatibility.py b/sdks/python/apache_beam/typehints/arrow_type_compatibility.py index c8e425f0e96a..34a37a886bab 100644 --- a/sdks/python/apache_beam/typehints/arrow_type_compatibility.py +++ b/sdks/python/apache_beam/typehints/arrow_type_compatibility.py @@ -311,9 +311,9 @@ def from_typehints(element_type, element_type = RowTypeConstraint.from_user_type(element_type) if element_type is None: raise TypeError( - "Element type must be compatible with Beam Schemas (" - "https://beam.apache.org/documentation/programming-guide/#schemas) " - "for batch type pa.Table.") + f"Element type {element_type} must be compatible with Beam Schemas " + "(https://beam.apache.org/documentation/programming-guide/#schemas)" + " for batch type pa.Table.") return PyarrowBatchConverter(element_type) From e94aeb71fffa9e9f94fe9a2242ff12fa7e7ab560 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Thu, 5 Oct 2023 10:56:25 -0700 Subject: [PATCH 057/435] Add parquet yaml providers. --- sdks/python/apache_beam/yaml/standard_io.yaml | 11 +++++++++++ sdks/python/apache_beam/yaml/yaml_provider.py | 6 +++++- 2 files changed, 16 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/yaml/standard_io.yaml b/sdks/python/apache_beam/yaml/standard_io.yaml index 9ad4f53ba1f6..af738a589cda 100644 --- a/sdks/python/apache_beam/yaml/standard_io.yaml +++ b/sdks/python/apache_beam/yaml/standard_io.yaml @@ -67,6 +67,8 @@ 'WriteToCsv': 'WriteToCsv' 'ReadFromJson': 'ReadFromJson' 'WriteToJson': 'WriteToJson' + 'ReadFromParquet': 'ReadFromParquet' + 'WriteToParquet': 'WriteToParquet' config: mappings: 'ReadFromCsv': @@ -77,6 +79,13 @@ path: 'path' 'WriteToJson': path: 'path' + 'ReadFromParquet': + path: 'file_pattern' + 'WriteToParquet': + path: 'file_path_prefix' + defaults: + 'ReadFromParquet': + as_rows: True underlying_provider: type: python transforms: @@ -84,3 +93,5 @@ 'WriteToCsv': 'apache_beam.io.WriteToCsv' 'ReadFromJson': 'apache_beam.io.ReadFromJson' 'WriteToJson': 'apache_beam.io.WriteToJson' + 'ReadFromParquet': 'apache_beam.io.ReadFromParquet' + 'WriteToParquet': 'apache_beam.io.WriteToParquet' diff --git a/sdks/python/apache_beam/yaml/yaml_provider.py b/sdks/python/apache_beam/yaml/yaml_provider.py index 3a88f6074b37..20d4b096916d 100644 --- a/sdks/python/apache_beam/yaml/yaml_provider.py +++ b/sdks/python/apache_beam/yaml/yaml_provider.py @@ -684,7 +684,7 @@ def __exit__(self, *args): @ExternalProvider.register_provider_type('renaming') class RenamingProvider(Provider): - def __init__(self, transforms, mappings, underlying_provider): + def __init__(self, transforms, mappings, underlying_provider, defaults=None): if isinstance(underlying_provider, dict): underlying_provider = ExternalProvider.provider_from_spec( underlying_provider) @@ -694,6 +694,7 @@ def __init__(self, transforms, mappings, underlying_provider): if transform not in mappings: raise ValueError(f'Missing transform {transform} in mappings.') self._mappings = mappings + self._defaults = defaults or {} def available(self) -> bool: return self._underlying_provider.available() @@ -731,6 +732,9 @@ def create_transform( mappings.get(key, key): value for key, value in args.items() } + for key, value in self._defaults.get(typ, {}).items(): + if key not in remapped_args: + remapped_args[key] = value return self._underlying_provider.create_transform( self._transforms[typ], remapped_args, yaml_create_transform) From 9602af91dcd05cc28db655589abfc42af226dc90 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Fri, 6 Oct 2023 12:00:32 -0700 Subject: [PATCH 058/435] Update error message to account for more detail. --- .../apache_beam/typehints/arrow_type_compatibility_test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/typehints/arrow_type_compatibility_test.py b/sdks/python/apache_beam/typehints/arrow_type_compatibility_test.py index e708b151d905..1e9ab3f27bd9 100644 --- a/sdks/python/apache_beam/typehints/arrow_type_compatibility_test.py +++ b/sdks/python/apache_beam/typehints/arrow_type_compatibility_test.py @@ -206,7 +206,7 @@ class ArrowBatchConverterErrorsTest(unittest.TestCase): ( pa.Table, Any, - r'Element type must be compatible with Beam Schemas', + r'Element type .* must be compatible with Beam Schemas', ), ]) def test_construction_errors( From 84b9336cffce7af17aa21437ae0b878984f918bc Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Fri, 6 Oct 2023 15:24:43 -0400 Subject: [PATCH 059/435] Merge pull request #28665: Add GitHub Action workflows to mass_commit --- .github/workflows/beam_PostCommit_Python.yml | 2 +- .../src/main/scripts/github_actions_jobs.txt | 33 ++++++++++++++----- release/src/main/scripts/mass_comment.py | 3 +- 3 files changed, 28 insertions(+), 10 deletions(-) diff --git a/.github/workflows/beam_PostCommit_Python.yml b/.github/workflows/beam_PostCommit_Python.yml index 6f4bc5e2ef0b..3fb650606b40 100644 --- a/.github/workflows/beam_PostCommit_Python.yml +++ b/.github/workflows/beam_PostCommit_Python.yml @@ -64,7 +64,7 @@ jobs: if: | github.event_name == 'workflow_dispatch' || github.event_name == 'schedule' || - github.event.comment.body == 'Run Python PostCommit' + startswith(github.event.comment.body, 'Run Python PostCommit 3.') steps: - uses: actions/checkout@v4 - name: Setup repository diff --git a/release/src/main/scripts/github_actions_jobs.txt b/release/src/main/scripts/github_actions_jobs.txt index 3ad16e555ef6..6f200454afe0 100644 --- a/release/src/main/scripts/github_actions_jobs.txt +++ b/release/src/main/scripts/github_actions_jobs.txt @@ -1,8 +1,25 @@ -Run Go PostCommit Dataflow ARM -Run Java_Examples_Dataflow_ARM PostCommit 8 -Run Java_Examples_Dataflow_ARM PostCommit 11 -Run Java_Examples_Dataflow_ARM PostCommit 17 -Run Python ValidatesContainer Dataflow ARM 3.8 -Run Python ValidatesContainer Dataflow ARM 3.9 -Run Python ValidatesContainer Dataflow ARM 3.10 -Run Python ValidatesContainer Dataflow ARM 3.11 \ No newline at end of file +Run PythonDocs PreCommit,beam_PreCommit_PythonDocs +Run PythonLint PreCommit,beam_PreCommit_PythonLint +Run RAT PreCommit,beam_PreCommit_RAT +Run Spotless PreCommit,beam_PreCommit_Spotless +Run Website PreCommit,beam_PreCommit_Website +Run Website_Stage_GCS PreCommit,beam_PreCommit_Website_Stage_GCS +Run Whitespace PreCommit,beam_PreCommit_Whitespace +Run PythonFormatter PreCommit,beam_PreCommit_PythonFormatter +Run Kotlin_Examples PreCommit,beam_PreCommit_Kotlin_Examples +Run Go PreCommit,beam_PreCommit_Go +Run GoPortable PreCommit,beam_PreCommit_GoPortable +Run GoPrism PreCommit,beam_PreCommit_GoPrism +Run Typescript PreCommit,beam_PreCommit_Typescript +Run CommunityMetrics PreCommit,beam_PreCommit_CommunityMetrics +Run Java_Flink_Versions PreCommit,beam_PreCommit_Java_Flink_Versions +Run It_Framework PreCommit,beam_PreCommit_ItFramework +Run Java examples on Dataflow,beam_PostCommit_Java_Examples_Dataflow +Run Go PostCommit Dataflow ARM,beam_PostCommit_Go_Dataflow_ARM +Run Java_Examples_Dataflow_ARM PostCommit 8,beam_PostCommit_Java_Examples_Dataflow_ARM +Run Java_Examples_Dataflow_ARM PostCommit 11,beam_PostCommit_Java_Examples_Dataflow_ARM +Run Java_Examples_Dataflow_ARM PostCommit 17,beam_PostCommit_Java_Examples_Dataflow_ARM +Run Python ValidatesContainer Dataflow ARM 3.8,beam_Python_ValidatesContainer_Dataflow_ARM +Run Python ValidatesContainer Dataflow ARM 3.9,beam_Python_ValidatesContainer_Dataflow_ARM +Run Python ValidatesContainer Dataflow ARM 3.10,beam_Python_ValidatesContainer_Dataflow_ARM +Run Python ValidatesContainer Dataflow ARM 3.11,beam_Python_ValidatesContainer_Dataflow_ARM diff --git a/release/src/main/scripts/mass_comment.py b/release/src/main/scripts/mass_comment.py index 3fa60fcb3e18..6d020f2af8e3 100644 --- a/release/src/main/scripts/mass_comment.py +++ b/release/src/main/scripts/mass_comment.py @@ -135,7 +135,8 @@ def getGithubActionsTriggerCommands(dirname): comments = [line.strip() for line in file if len(line.strip()) > 0] for i in range(len(comments)): - gha_trigger_commands.append(comments[i]) + parts = comments[i].split(',') + gha_trigger_commands.append((parts[0], parts[1])) return gha_trigger_commands From d26a78297f279135a35c8ab003d120ff482b1818 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Fri, 6 Oct 2023 12:35:52 -0700 Subject: [PATCH 060/435] [YAML] Add JSON parsing to PubSub IO. (#28754) --- .../apache_beam/transforms/ptransform.py | 16 ++ sdks/python/apache_beam/yaml/json_utils.py | 183 ++++++++++++++++++ sdks/python/apache_beam/yaml/yaml_io.py | 16 +- sdks/python/apache_beam/yaml/yaml_io_test.py | 123 ++++++++++++ 4 files changed, 336 insertions(+), 2 deletions(-) create mode 100644 sdks/python/apache_beam/yaml/json_utils.py diff --git a/sdks/python/apache_beam/transforms/ptransform.py b/sdks/python/apache_beam/transforms/ptransform.py index 28614c6561c7..fcff86d4c50c 100644 --- a/sdks/python/apache_beam/transforms/ptransform.py +++ b/sdks/python/apache_beam/transforms/ptransform.py @@ -1101,6 +1101,22 @@ def __ror__(self, pvalueish, _unused=None): def expand(self, pvalue): raise RuntimeError("Should never be expanded directly.") + def __getattr__(self, attr): + transform_attr = getattr(self.transform, attr) + if callable(transform_attr): + + @wraps(transform_attr) + def wrapper(*args, **kwargs): + result = transform_attr(*args, **kwargs) + if isinstance(result, PTransform): + return _NamedPTransform(result, self.label) + else: + return result + + return wrapper + else: + return transform_attr + # Defined here to avoid circular import issues for Beam library transforms. def annotate_yaml(constructor): diff --git a/sdks/python/apache_beam/yaml/json_utils.py b/sdks/python/apache_beam/yaml/json_utils.py new file mode 100644 index 000000000000..e2cb03dc96a0 --- /dev/null +++ b/sdks/python/apache_beam/yaml/json_utils.py @@ -0,0 +1,183 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +"""Utilities for converting between JSON and Beam Schema'd data. + +For internal use, no backward compatibility guarantees. +""" + +import json +from typing import Any +from typing import Callable +from typing import Dict + +import apache_beam as beam +from apache_beam.portability.api import schema_pb2 +from apache_beam.typehints import schemas + +JSON_ATOMIC_TYPES_TO_BEAM = { + 'boolean': schema_pb2.BOOLEAN, + 'integer': schema_pb2.INT64, + 'number': schema_pb2.DOUBLE, + 'string': schema_pb2.STRING, +} + + +def json_schema_to_beam_schema( + json_schema: Dict[str, Any]) -> schema_pb2.Schema: + """Returns a Beam schema equivalent for the given Json schema.""" + def maybe_nullable(beam_type, nullable): + if nullable: + beam_type.nullable = True + return beam_type + + json_type = json_schema.get('type', None) + if json_type != 'object': + raise ValueError('Expected object type, got {json_type}.') + if 'properties' not in json_schema: + # Technically this is a valid (vacuous) schema, but as it's not generally + # meaningful, throw an informative error instead. + # (We could add a flag to allow this degenerate case.) + raise ValueError('Missing properties for {json_schema}.') + required = set(json_schema.get('required', [])) + return schema_pb2.Schema( + fields=[ + schemas.schema_field( + name, + maybe_nullable(json_type_to_beam_type(t), name not in required)) + for (name, t) in json_schema['properties'].items() + ]) + + +def json_type_to_beam_type(json_type: Dict[str, Any]) -> schema_pb2.FieldType: + """Returns a Beam schema type for the given Json (schema) type.""" + if not isinstance(json_type, dict) or 'type' not in json_type: + raise ValueError(f'Malformed type {json_type}.') + type_name = json_type['type'] + if type_name in JSON_ATOMIC_TYPES_TO_BEAM: + return schema_pb2.FieldType( + atomic_type=JSON_ATOMIC_TYPES_TO_BEAM[type_name]) + elif type_name == 'array': + return schema_pb2.FieldType( + array_type=schema_pb2.ArrayType( + element_type=json_type_to_beam_type(json_type['items']))) + elif type_name == 'object': + if 'properties' in json_type: + return schema_pb2.FieldType( + row_type=schema_pb2.RowType( + schema=json_schema_to_beam_schema(json_type))) + elif 'additionalProperties' in json_type: + return schema_pb2.FieldType( + map_type=schema_pb2.MapType( + key_type=schema_pb2.FieldType(atomic_type=schema_pb2.STRING), + value_type=json_type_to_beam_type( + json_type['additionalProperties']))) + else: + raise ValueError( + f'Object type must have either properties or additionalProperties, ' + f'got {json_type}.') + else: + raise ValueError(f'Unable to convert {json_type} to a Beam schema.') + + +def json_to_row(beam_type: schema_pb2.FieldType) -> Callable[[Any], Any]: + """Returns a callable converting Json objects to Beam rows of the given type. + + The input to the returned callable is expected to conform to the Json schema + corresponding to this Beam type. + """ + type_info = beam_type.WhichOneof("type_info") + if type_info == "atomic_type": + return lambda value: value + elif type_info == "array_type": + element_converter = json_to_row(beam_type.array_type.element_type) + return lambda value: [element_converter(e) for e in value] + elif type_info == "iterable_type": + element_converter = json_to_row(beam_type.iterable_type.element_type) + return lambda value: [element_converter(e) for e in value] + elif type_info == "map_type": + if beam_type.map_type.key_type.atomic_type != schema_pb2.STRING: + raise TypeError( + f'Only strings allowd as map keys when converting from JSON, ' + f'found {beam_type}') + value_converter = json_to_row(beam_type.map_type.value_type) + return lambda value: {k: value_converter(v) for (k, v) in value.items()} + elif type_info == "row_type": + converters = { + field.name: json_to_row(field.type) + for field in beam_type.row_type.schema.fields + } + return lambda value: beam.Row( + ** + {name: convert(value[name]) + for (name, convert) in converters.items()}) + elif type_info == "logical_type": + return lambda value: value + else: + raise ValueError(f"Unrecognized type_info: {type_info!r}") + + +def json_parser(beam_schema: schema_pb2.Schema) -> Callable[[bytes], beam.Row]: + """Returns a callable converting Json strings to Beam rows of the given type. + + The input to the returned callable is expected to conform to the Json schema + corresponding to this Beam type. + """ + to_row = json_to_row( + schema_pb2.FieldType(row_type=schema_pb2.RowType(schema=beam_schema))) + return lambda s: to_row(json.loads(s)) + + +def row_to_json(beam_type: schema_pb2.FieldType) -> Callable[[Any], Any]: + """Returns a callable converting rows of the given type to Json objects.""" + type_info = beam_type.WhichOneof("type_info") + if type_info == "atomic_type": + return lambda value: value + elif type_info == "array_type": + element_converter = row_to_json(beam_type.array_type.element_type) + return lambda value: [element_converter(e) for e in value] + elif type_info == "iterable_type": + element_converter = row_to_json(beam_type.iterable_type.element_type) + return lambda value: [element_converter(e) for e in value] + elif type_info == "map_type": + if beam_type.map_type.key_type.atomic_type != schema_pb2.STRING: + raise TypeError( + f'Only strings allowd as map keys when converting to JSON, ' + f'found {beam_type}') + value_converter = row_to_json(beam_type.map_type.value_type) + return lambda value: {k: value_converter(v) for (k, v) in value.items()} + elif type_info == "row_type": + converters = { + field.name: row_to_json(field.type) + for field in beam_type.row_type.schema.fields + } + return lambda row: { + name: convert(getattr(row, name)) + for (name, convert) in converters.items() + } + elif type_info == "logical_type": + return lambda value: value + else: + raise ValueError(f"Unrecognized type_info: {type_info!r}") + + +def json_formater( + beam_schema: schema_pb2.Schema) -> Callable[[beam.Row], bytes]: + """Returns a callable converting rows of the given schema to Json strings.""" + convert = row_to_json( + schema_pb2.FieldType(row_type=schema_pb2.RowType(schema=beam_schema))) + return lambda row: json.dumps(convert(row), sort_keys=True).encode('utf-8') diff --git a/sdks/python/apache_beam/yaml/yaml_io.py b/sdks/python/apache_beam/yaml/yaml_io.py index 4a1d12490057..3321644ded57 100644 --- a/sdks/python/apache_beam/yaml/yaml_io.py +++ b/sdks/python/apache_beam/yaml/yaml_io.py @@ -25,10 +25,12 @@ import os from typing import Any +from typing import Callable from typing import Iterable from typing import List from typing import Mapping from typing import Optional +from typing import Tuple import yaml @@ -39,6 +41,7 @@ from apache_beam.io.gcp.bigquery import BigQueryDisposition from apache_beam.portability.api import schema_pb2 from apache_beam.typehints import schemas +from apache_beam.yaml import json_utils from apache_beam.yaml import yaml_mapping from apache_beam.yaml import yaml_provider @@ -131,18 +134,25 @@ def raise_exception(failed_row_with_error): return WriteToBigQueryHandlingErrors() -def _create_parser(format, schema): +def _create_parser( + format, + schema: Any) -> Tuple[schema_pb2.Schema, Callable[[bytes], beam.Row]]: if format == 'raw': if schema: raise ValueError('raw format does not take a schema') return ( schema_pb2.Schema(fields=[schemas.schema_field('payload', bytes)]), lambda payload: beam.Row(payload=payload)) + elif format == 'json': + beam_schema = json_utils.json_schema_to_beam_schema(schema) + return beam_schema, json_utils.json_parser(beam_schema) else: raise ValueError(f'Unknown format: {format}') -def _create_formatter(format, schema, beam_schema): +def _create_formatter( + format, schema: Any, + beam_schema: schema_pb2.Schema) -> Callable[[beam.Row], bytes]: if format == 'raw': if schema: raise ValueError('raw format does not take a schema') @@ -150,6 +160,8 @@ def _create_formatter(format, schema, beam_schema): if len(field_names) != 1: raise ValueError(f'Expecting exactly one field, found {field_names}') return lambda row: getattr(row, field_names[0]) + elif format == 'json': + return json_utils.json_formater(beam_schema) else: raise ValueError(f'Unknown format: {format}') diff --git a/sdks/python/apache_beam/yaml/yaml_io_test.py b/sdks/python/apache_beam/yaml/yaml_io_test.py index ab6298661c15..72675da278b0 100644 --- a/sdks/python/apache_beam/yaml/yaml_io_test.py +++ b/sdks/python/apache_beam/yaml/yaml_io_test.py @@ -167,6 +167,101 @@ def test_read_with_id_attribute(self): result, equal_to([beam.Row(payload=b'msg1'), beam.Row(payload=b'msg2')])) + def test_read_json(self): + with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions( + pickle_library='cloudpickle')) as p: + with mock.patch('apache_beam.io.ReadFromPubSub', + FakeReadFromPubSub( + topic='my_topic', + messages=[PubsubMessage( + b'{"generator": {"x": 0, "y": 0}, "rank": 1}', + {'weierstrass': 'y^2+y=x^3-x', 'label': '37a'}) + ])): + result = p | YamlTransform( + ''' + type: ReadFromPubSub + config: + topic: my_topic + format: json + schema: + type: object + properties: + generator: + type: object + properties: + x: {type: integer} + y: {type: integer} + rank: {type: integer} + attributes: [label] + attributes_map: other + ''') + assert_that( + result, + equal_to([ + beam.Row( + generator=beam.Row(x=0, y=0), + rank=1, + label='37a', + other={ + 'label': '37a', 'weierstrass': 'y^2+y=x^3-x' + }) + ])) + + def test_read_json_with_error_handling(self): + with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions( + pickle_library='cloudpickle')) as p: + with mock.patch( + 'apache_beam.io.ReadFromPubSub', + FakeReadFromPubSub(topic='my_topic', + messages=[PubsubMessage('{"some_int": 123}', + attributes={}), + PubsubMessage('unparsable', + attributes={})])): + result = p | YamlTransform( + ''' + type: ReadFromPubSub + config: + topic: my_topic + format: json + schema: + type: object + properties: + some_int: {type: integer} + error_handling: + output: errors + ''') + assert_that( + result['good'], + equal_to([beam.Row(some_int=123)]), + label='CheckGood') + assert_that( + result['errors'] | beam.Map(lambda error: error.element), + equal_to(['unparsable']), + label='CheckErrors') + + def test_read_json_without_error_handling(self): + with self.assertRaises(Exception): + with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions( + pickle_library='cloudpickle')) as p: + with mock.patch( + 'apache_beam.io.ReadFromPubSub', + FakeReadFromPubSub(topic='my_topic', + messages=[PubsubMessage('{"some_int": 123}', + attributes={}), + PubsubMessage('unparsable', + attributes={})])): + _ = p | YamlTransform( + ''' + type: ReadFromPubSub + config: + topic: my_topic + format: json + schema: + type: object + properties: + some_int: {type: integer} + ''') + def test_simple_write(self): with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions( pickle_library='cloudpickle')) as p: @@ -251,6 +346,34 @@ def test_write_with_id_attribute(self): id_attribute: some_attr ''')) + def test_write_json(self): + with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions( + pickle_library='cloudpickle')) as p: + with mock.patch('apache_beam.io.WriteToPubSub', + FakeWriteToPubSub( + topic='my_topic', + messages=[PubsubMessage( + b'{"generator": {"x": 0, "y": 0}, "rank": 1}', + {'weierstrass': 'y^2+y=x^3-x', 'label': '37a'}) + ])): + _ = ( + p | beam.Create([ + beam.Row( + label='37a', + generator=beam.Row(x=0, y=0), + rank=1, + other={'weierstrass': 'y^2+y=x^3-x'}) + ]) | YamlTransform( + ''' + type: WriteToPubSub + input: input + config: + topic: my_topic + format: json + attributes: [label] + attributes_map: other + ''')) + if __name__ == '__main__': logging.getLogger().setLevel(logging.INFO) From e665c1c73bdf35ddbe4b2fa79a7229549a7ba279 Mon Sep 17 00:00:00 2001 From: Svetak Sundhar Date: Fri, 6 Oct 2023 20:42:33 +0000 Subject: [PATCH 061/435] [Fix] Broken Healthcare Notebook Link (#28876) * Create HealthcareUtils file with shared resources * revert * Add correct notebook link. --- examples/notebooks/healthcare/beam_nlp.ipynb | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/notebooks/healthcare/beam_nlp.ipynb b/examples/notebooks/healthcare/beam_nlp.ipynb index 4ba4a5e0a739..c2061bc4d75f 100644 --- a/examples/notebooks/healthcare/beam_nlp.ipynb +++ b/examples/notebooks/healthcare/beam_nlp.ipynb @@ -22,7 +22,7 @@ "colab_type": "text" }, "source": [ - "\"Open" + "\"Open" ] }, { From ee7a2aea975e94fabff9a44508e4f4419ef11d2b Mon Sep 17 00:00:00 2001 From: Anand Inguva <34158215+AnandInguva@users.noreply.github.com> Date: Fri, 6 Oct 2023 17:10:36 -0400 Subject: [PATCH 062/435] Fix perf alert tool call (#28872) --- sdks/python/apache_beam/testing/analyzers/perf_analysis.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/testing/analyzers/perf_analysis.py b/sdks/python/apache_beam/testing/analyzers/perf_analysis.py index c86ecb2c4e20..0074625d3b47 100644 --- a/sdks/python/apache_beam/testing/analyzers/perf_analysis.py +++ b/sdks/python/apache_beam/testing/analyzers/perf_analysis.py @@ -160,6 +160,7 @@ def run_change_point_analysis( def run( + *, big_query_metrics_fetcher: MetricsFetcher = BigQueryMetricsFetcher(), config_file_path: Optional[str] = None, ) -> None: @@ -209,4 +210,4 @@ def run( if unknown_args: logging.warning('Discarding unknown arguments : %s ' % unknown_args) - run(known_args.config_file_path) + run(config_file_path=known_args.config_file_path) From fba098b69dbe18c84c007018f727f59aa077ac0b Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 6 Oct 2023 17:40:05 -0400 Subject: [PATCH 063/435] Bump postcss (#28866) Bumps [postcss](https://github.com/postcss/postcss) from 8.3.9 to 8.4.31. - [Release notes](https://github.com/postcss/postcss/releases) - [Changelog](https://github.com/postcss/postcss/blob/main/CHANGELOG.md) - [Commits](https://github.com/postcss/postcss/compare/8.3.9...8.4.31) --- updated-dependencies: - dependency-name: postcss dependency-type: indirect ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .../yarn.lock | 33 +++++++++++-------- 1 file changed, 19 insertions(+), 14 deletions(-) diff --git a/sdks/python/apache_beam/runners/interactive/extensions/apache-beam-jupyterlab-sidepanel/yarn.lock b/sdks/python/apache_beam/runners/interactive/extensions/apache-beam-jupyterlab-sidepanel/yarn.lock index fd09b6c9eb00..f72e23ebd403 100644 --- a/sdks/python/apache_beam/runners/interactive/extensions/apache-beam-jupyterlab-sidepanel/yarn.lock +++ b/sdks/python/apache_beam/runners/interactive/extensions/apache-beam-jupyterlab-sidepanel/yarn.lock @@ -6767,10 +6767,10 @@ mv@2.1.1: ncp "~2.0.0" rimraf "~2.4.0" -nanoid@^3.1.28: - version "3.3.2" - resolved "https://registry.yarnpkg.com/nanoid/-/nanoid-3.3.2.tgz#c89622fafb4381cd221421c69ec58547a1eec557" - integrity sha512-CuHBogktKwpm5g2sRgv83jEy2ijFzBwMoYA60orPDR7ynsLijJDqgsi4RDGj3OJpy3Ieb+LYwiRmIOGyytgITA== +nanoid@^3.3.6: + version "3.3.6" + resolved "https://registry.yarnpkg.com/nanoid/-/nanoid-3.3.6.tgz#443380c856d6e9f9824267d960b4236ad583ea4c" + integrity sha512-BGcqMMJuToF7i1rt+2PWSNVnWIkGCU78jBG3RxO/bZlnZPK2Cmi2QaffxGO/2RvWi9sL+FAiRiXMgsyxQ1DIDA== nanomatch@^1.2.9: version "1.2.13" @@ -7279,6 +7279,11 @@ picocolors@^0.2.1: resolved "https://registry.yarnpkg.com/picocolors/-/picocolors-0.2.1.tgz#570670f793646851d1ba135996962abad587859f" integrity sha512-cMlDqaLEqfSaW8Z7N5Jw+lyIW869EzT73/F5lhtY9cLGoVxSXznfgfXMO0Z5K0o0Q2TkTXq+0KFsdnSe3jDViA== +picocolors@^1.0.0: + version "1.0.0" + resolved "https://registry.yarnpkg.com/picocolors/-/picocolors-1.0.0.tgz#cb5bdc74ff3f51892236eaf79d68bc44564ab81c" + integrity sha512-1fygroTLlHu66zi26VoTDv8yRgm0Fccecssto+MhsZ0D/DGW2sm8E8AjW7NU5VVTRt5GxbeZ5qBuJr+HyLYkjQ== + picomatch@^2.0.4, picomatch@^2.0.5: version "2.2.2" resolved "https://registry.yarnpkg.com/picomatch/-/picomatch-2.2.2.tgz#21f333e9b6b8eaff02468f5146ea406d345f4dad" @@ -7392,13 +7397,13 @@ postcss-value-parser@^4.1.0: integrity sha512-97DXOFbQJhk71ne5/Mt6cOu6yxsSfM0QGQyl0L25Gca4yGWEGJaig7l7gbCX623VqTBNGLRLaVUCnNkcedlRSQ== postcss@^8.0.2, postcss@^8.2.15: - version "8.3.9" - resolved "https://registry.yarnpkg.com/postcss/-/postcss-8.3.9.tgz#98754caa06c4ee9eb59cc48bd073bb6bd3437c31" - integrity sha512-f/ZFyAKh9Dnqytx5X62jgjhhzttjZS7hMsohcI7HEI5tjELX/HxCy3EFhsRxyzGvrzFF+82XPvCS8T9TFleVJw== + version "8.4.31" + resolved "https://registry.yarnpkg.com/postcss/-/postcss-8.4.31.tgz#92b451050a9f914da6755af352bdc0192508656d" + integrity sha512-PS08Iboia9mts/2ygV3eLpY5ghnUcfLV/EXTOW1E2qYxJKGGBUtNjN76FYHnMs36RmARn41bC0AZmn+rR0OVpQ== dependencies: - nanoid "^3.1.28" - picocolors "^0.2.1" - source-map-js "^0.6.2" + nanoid "^3.3.6" + picocolors "^1.0.0" + source-map-js "^1.0.2" prelude-ls@^1.2.1: version "1.2.1" @@ -8351,10 +8356,10 @@ source-list-map@^2.0.0: resolved "https://registry.yarnpkg.com/source-list-map/-/source-list-map-2.0.1.tgz#3993bd873bfc48479cca9ea3a547835c7c154b34" integrity sha512-qnQ7gVMxGNxsiL4lEuJwe/To8UnK7fAnmbGEEH8RpLouuKbeEm0lhbQVFIrNSuB+G7tVrAlVsZgETT5nljf+Iw== -source-map-js@^0.6.2: - version "0.6.2" - resolved "https://registry.yarnpkg.com/source-map-js/-/source-map-js-0.6.2.tgz#0bb5de631b41cfbda6cfba8bd05a80efdfd2385e" - integrity sha512-/3GptzWzu0+0MBQFrDKzw/DvvMTUORvgY6k6jd/VS6iCR4RDTKWH6v6WPwQoUO8667uQEf9Oe38DxAYWY5F/Ug== +source-map-js@^1.0.2: + version "1.0.2" + resolved "https://registry.yarnpkg.com/source-map-js/-/source-map-js-1.0.2.tgz#adbc361d9c62df380125e7f161f71c826f1e490c" + integrity sha512-R0XvVJ9WusLiqTCEiGCmICCMplcCkIwwR11mOSD9CR5u+IXYdiseeEuXCVAjS54zqwkLcPNnmU4OeJ6tUrWhDw== source-map-resolve@^0.5.0: version "0.5.3" From cb802661f2aa34cd574f7677a64a1b9e72fd94ee Mon Sep 17 00:00:00 2001 From: Anand Inguva <34158215+AnandInguva@users.noreply.github.com> Date: Fri, 6 Oct 2023 17:47:42 -0400 Subject: [PATCH 064/435] [Python]Update state cache size to 100 MB (#28877) Co-authored-by: tvalentyn --- sdks/python/apache_beam/runners/worker/sdk_worker_main.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/runners/worker/sdk_worker_main.py b/sdks/python/apache_beam/runners/worker/sdk_worker_main.py index d3442fcb5987..1e44a998ba05 100644 --- a/sdks/python/apache_beam/runners/worker/sdk_worker_main.py +++ b/sdks/python/apache_beam/runners/worker/sdk_worker_main.py @@ -247,7 +247,7 @@ def _get_state_cache_size(experiments): Returns: an int indicating the maximum number of megabytes to cache. - Default is 0 MB + Default is 100 MB """ for experiment in experiments: @@ -256,7 +256,7 @@ def _get_state_cache_size(experiments): return int( re.match(r'state_cache_size=(?P.*)', experiment).group('state_cache_size')) << 20 - return 0 + return 100 << 20 def _get_data_buffer_time_limit_ms(experiments): From 7531501ff27f53bcb4fcd6942f34dbd45665805d Mon Sep 17 00:00:00 2001 From: jonathan-lemos Date: Fri, 6 Oct 2023 19:21:19 -0400 Subject: [PATCH 065/435] Fix interactive tests for Beam Notebooks release (#28880) --- .../29c9237ddf4f3d5988a503069b4d3c47.png | Bin 65527 -> 64875 bytes .../7a35f487b2a5f3a9b9852a8659eeb4bd.png | Bin 728485 -> 687768 bytes sdks/python/setup.py | 4 +++- 3 files changed, 3 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/runners/interactive/testing/integration/goldens/Linux/29c9237ddf4f3d5988a503069b4d3c47.png b/sdks/python/apache_beam/runners/interactive/testing/integration/goldens/Linux/29c9237ddf4f3d5988a503069b4d3c47.png index c748ea1a2d0a4f1f60273f7d4ff35809829f2c6f..382063f75092d42897328868efaefb88d744dd07 100644 GIT binary patch literal 64875 zcmeHwcT`k)x^3CE(niK^6>JpR5d}p+K|w$$TU+T81qBfSp$v@(5|mWrQfXz{0165! z$p9jfRB|klRs;l;D6z;nLs1k|c>CBh_syL(>#aBQ-d*pm^^Sko_Mz&8-|q{1f8YKN zza7%oU9#}|g%k>93GL^f3@Mba|3sn4{pp*paAcF=>ZcTnK!x_xk4N0Y2b(=|L!5=v zV*}yk8bg&E%+BuF{k@`LT*5lN?@aeEe)sLkU+%sCX>HCo#rf9D_ts~urKNsea{qht zn?X}U&&vDw!pt5<{y~mT&+n#&6+IoZy%&ZD@<#@I*=rG#T4(7sRBdE1R2^s3Qy!t5 z*?M`E>Y3>DKq>Vk*MZvjqludY#UV--nx}sMONgSmew2=nU?|19V))Y&-J0jd8I2A( z(%POM?yJU5TF-KHc%8w8;#BY21XC;RZzz=Ivf8wuA{i!syi}R(T4#}*J6@uvl=Ta~ zX&vd&jjxs3K8?`}74{i0pPOe4M~V1720FZ71Ua=y7G9rE`uoiE#fnq;qkiH#(_=+4 zGM>GU&ebI;#vfj^#eKBZGS71_m-wVez)e2pI*Az;a!a7>e?RC0teU(-$N~H?xbEZ7pqt1dc^rCR7uCzzz3?olCGoX6r!#y1zPds#x ztj*^aCmYl4+3p=TlPq#Vil4D|^?TV3)Fx`3{rK=zHCj&+nzNVbOC;Agj?1}kf8i7i#bIU~Yyt&woI2AjK^ zj2l{$upAF@-8i$KgQ5?HsTn^%!fW*tzc%gQn8&I0nCMPqS+(SP6kc7TF!`Ha$mXo} zYYN#eeU(`1AN}UXwAcj6;-YKR>^`>^_~p3tIgJ*GvCkJS!P-V_KX(*QNq29%a$nUt z-Jw~Nvpx|Y&ci9nH3t%KPl5ru=)q18Cwy!oRJQEi){b!J^1hFc_S7X=$hfy&;@1gq z!!w+LI$UqhQjK4*4U1N8mT%XZchUY}YJT=lOIGg~?F=?hx*~B|abq@3a0iE^)j z#Ecp4NR%w??X?x=sb_DC*7{aP(|F7cwDC{6GD9g9Ix7^8@~t|AlT`)-83%)8(`;*C zSy9)$F~++g%>~H}{+jsN$-aqt>nOpH zBQvKnNUz0*+brdfxmMHyJF9(eJn+b0dS)U@v=e*w@^le#OHcOQ>~G4NP|6upv8{gQ zw^UP}^FEaBn}xlpqceS)vt8@hq71gLSNDDEGiT1UVDaNlYsI~LUtFvq;!lKbHdXoM z{jKZD`R+55eQ}QIUs2Rv#3UwPO{DD$k`>HyD~gL(;S&*z7LV3TVs6-qz1LR`Pu<`rqirRWU+MT=g zv%~Pr2E~jf!+dWZ_UJ@;xQ4;bv7Zngt?@-{oSt%CDqM*cbR_AxNAR)NJ@xxQG7F0}t{;RCecHCe)H>p>%$CuxDW-|1LuGXFUI`1Awks0(NIWKV9}ZHI zGyd0MT*Je|LqkIYoAcZvXKJPq&MiRJfc3zX5Y=Liu18|ag+br^e*1%MHik!!)=o7# zMn%c6cJao$oSmKP*g?U;MtHD4>*V ztNEexW7k7*&JV6T!1E9gJ>YgoK=+qSu^;~ECnn=PdPz(=6Da`8m<6{N@hX^zMq_7& zWGLzL;y5*B54LOdKYL=xD*GVXeW$vzy(Zp7gm7S<=gGl7IXOG~FJ2WSTa2^iPq$x( zVYA>>*wWm=SEbTElb`tFn@+#Gp)rt<^8L26w`7Z-9e#3ET*h~*-g*Raj>t_`p%%oZ zSATmE9&}SmefaVsrDKTN7K~=si8A%rrRTQUH>ARBtj06uyj3`j^x07qNbR~~)L$Kk z%S1x6|N2=SKA@R5b}A*I3~7oI0KO>UZtcMiE?0PvG}ZUvSl8U z$ljelyr98scnFN+R{>8+7a|YrU9W!eJq(Bgf)>)kD&^&|a9;RZo-*?O8KwYvIcbFE`+1a_1KU%2m zHMMi%gKDs>O-18-<)*nDuEUC zV~x$31DEG5T4<3}_w-;Gliwl3>!BGS+I)Jv*S<0RV6z?0IoVegs~_<|W-{2UOZfY5SeEc(cZfu6RT{!(ihL5y(jJJI zT7xh4Aj&TCJ!)oVwk%LB)vn&@-HlbSBfX(qY_QgF=4lu+9mx%zb_R*7tK@DY!gK|m z@a=Vp*nl<)WkBJCkLw)0>$vvFg6}qjdyf~xR?im&tT9PA_R_BY72l^@M&_QXb@`hs zi@11xeoj($|8u_dGu;34l|i!g2w@?cO%1KxwRT8m6_z-k8f z6mj?Nb|e`3s>Nb!Tt4DJbG%71LXM82qW$ZOb6|3HgkNRN+8`Kzl4*LP8;JafkjMR? zT9IHJU;FEV@7@KckDiF@IvNx^Beksj?29pP{pK&387&l_H3ZN%^R|``HLk#0v-~V>SDq9dSnW0V5*HjUgRS?) zgsxn@7y(TZVV#HxIDkj-#fkJRsuB@x2rEHih|JEy@-1Gy<3%)kGlk#1JU3s{%q$mK z0LVk|s#K|z+IZ5>@zjS<&4?Lnbpvi-|8(N(5P1OS)YR1B=A780efB5(hRVX!f%iRy zWyB^3;&0fW_;7FQ0DJXL_tEa91h~6t!l_O)=fYrGnk)qSwT4owZmPFrj!p4huWcNl z&vgLycIS^3J<&~fWcvzO#Q;v+0x_8;cTvtzdLy2KqZd2>qWx}LAJ+&?SAC7HJN%ox zfq3(DJ0eN}S}BO4i;zl>BwK2yxt&Guj=SmH{lS!}T{+Va7y+0Ck7e|JT$|Pf=wp#? zSHFQO^}4nwIeyMXcKYQy6Nwk8?#X#mo4Y&+rS(}6MhUpsXOtbh3>+;^rA5E&F^ z2!<1g6M^*%cZWGDxjzTIdvr9>EN&vYP5s~&3Wen{FCkg!gv}V(6jQ*d6PHp;08Fut zaIH96<{XBMy2J7M z@)!Av_R^FS@prP*D{)?nTEAWL9QzYtI$2^A$`AG5hR7lhI0W~YhpzCKz!7@A*q(M( zO$x;>ZZi#pbob>s6v{UX=Ho}QlAv{m>goRAd{Wh!7dpDj!bT8C!NZonSvDWE@Zrko z#Ph!rU(k31^7rf4b19TReY==K+4VCXkg~Sm|LVbKlb25pHRMXrP49(;nb;<0(vbD_ zLX`+0Ss#GAU@eG|SUh52u4&cq%0@;>InAUg8gW*M*!%G=U;@QP8ijJA)MS6b7E$c> zk%OioIZBS>vw9MWN+|~Y_<#i%KttNOQ1~uxc>?+v5PBNdTHhW#!q?^qjebR;C?YwoES@7mc zD1Ga*J+XcdGCp&AhqWy|IGlqzc8KN(IA$3R7X4@3+@>n1qHyMSxZbhf#0vYj543oC z(WWW}gx00Mztwo_GUe>xFdS7rsA@Y|rIDqvq`ywoBSD|aYkk9zV`UT+0@Gz6fTx|S z>sH=VQd(*bG=mFVx59>e-gMV}i?{G?$;cT|`0d;2LXo#%sA53Id3yVqiu=LAVL3(8 zWtKi;JQKQS(wIR+>apxDF6UDou*J;AC3Fn@6qxD?q2Ofg}UBj zZ$o)S3)O@(PmJS_Djs|BgPfeT-w;BV1#qN_s;b`ULlnw%BCm|z$D^xSNB9i$5J7qK zdLm}u`*GJ?m>yB*y|@Mv_Wa03MMXtN?fdudgUK2$f(i1<#0vdoeD4GYr&I-KO=FjnGKWSrQLtFy{2}x8KvdQ@Q9c6m{a{qahi|l)Beo%CG8TV&Bz1a$JYU;p8 z4}1M`P~2eI`p$VPU%h&@$s{Q}EG*3I-VuP3{dYGJ50@$jS6~OQ`OB`Z^cry;P*hj5 ztv=uN+}KNv-JvWHhRYVKvwd@l#Y~XK5i)THSe1CQwC8Vsrce&ONWEDh-jbboJj)3= z5h&)f@9fY#IMVV#b(XA+Gl=*`Il0(VZ?COd)!5`EM!8s)R<`COJnP!9I&&#Lc*yDB z@jLlMPfov;c43GWE^DO?*SFhP3HvsyRppypS-7Ia^H*6~5I7T2AEKl$50S-x{#B2@ zwtVsFcRF4!+zzvhfzLztI*N2ulYZ6R^u9Ot-RfFSZ=PAYP_V2y($mv!l|F8v@$!w` zGGlM9D~o3d-vt}=*FO>nM-DvjE^}4ClYB_{Y+ZK9oAXkVqNYWDNzzXJ8}r0_KerA2 z7MzsZ|6qJYiKwmj$$sXyF!z1YiWb?at-6!Kjy_Cp9hAve4yADxnk z*?ncU?t+ng-no3!6>8pJJ9_ zdzTe+aNsCiaPADq-vaUU-)LvNImv>C^fGp(11p_gKK*=B4mC4--L(ZTPQ5LGJ04~G z&hiy&z|9Hbb~Hi0K1=PDH}9I_FT}07E&p~gLJvv^irPb>$U{mfaHXAd-tUs|*x=Qf zvIfIQl%Uerutv`rS5+=#9_ah*gjjPi(O_-Ct^4)7rSvqn!NswaLoyk~csxsw--utc5iKgW*T(f8n7GLtbr8#}2JcYwHi`=8n* z9Q1;I0ZgM#j-IA|h%jfwM(AsC?wrnHDExMKVcvQ3AlcJVYZz>SxS+>hroSf9t9wVZC#dcq7p8~#!a5|m&)oxE_xraOOJ;Dzjh{{%(; z>AC*#;GM(0>yW6N>zcXw{Qh%r=R}j_VVbtVj{KsRF6@#u+6kKPyfaZH7&^lDnll(!DYaB^~C%&`YO# z=sQ>=8aZnADhNL&7S0y?_H~bG2WEOJ~le_-kbZ@3w%Xnu5tc$oAjx7O1D z5?BuJjd|*I0M%`}G9z_QT^AUnEznXwdw$|(T_UwOwNED1F@xS`K!o3bve}U6qA2(A zBAK2j&ywQx+YWxx9{>E_W5SnTWfL31`2mKvz46A0dujqdt*e2VosSm{)HcU;&fj!x zqfzXVyrHQ%0_8Z9WQQ+yXP)tRej$j)Y$(gpt?6a6*XqVR*`AF}0H)4AvaWkYo^M)E z#FhVo%HmDlXUN8EKAzee7#o#!uIgz9AoFbbp=RPher3Uw)+WB08HSM%dI29qa?U`}>nFST;ZL$I}zI|;1lkFn7RUzW-6;4^WwPd*s znZ;y=#-X(<@pvRE%kf+mf28Cofo^`F7%giJG*gl z#--9^$;obWM^OwrN^1Dk1X2Utsqf;oyPMnuf|D`SFDI&m-x|Gc2Bfl0qt7Y`)gCgE zzcVrq&>D;ObRB5%ZB%@#vmd~%AUO4NMp>(ZSY&06iLD-2TCO_T#Bq8}#)?_Q#a;W? z>Pl`pny`fzk@5U!+^d$5YxPvo@i#{Ho z0Aqz(dbrMr$+T#-mmA(sd`Po&@9VOuqPmaf=jeR_7H-RZm+mR?9nw;yzx*>TsC0&2 z*>^Euy@ui1Df<+rZ4hfoW~y`S+>-M|YVU{F?#WlS{%|bUz4%BPC)i(0X&y>w@^|gkMeOaP8bCbP0WE{I3(Gxk^oUL_N zNaUI!`OfGbI7x3M?_!vLOFE!qbysCez$|EJiE*RUnEskDwKfjb}n^)t6GxA}=vJ^Q*$lv$I>pd9P94;4H; zC~Dm@9Xb$>}v>+DgMD~db!C|Typin@QzkZTdO z7K*np7!hmSp3zyBV*!Tex9|ZSS1Ou?GbhS2HZ=K+2m-0&tpTq6+s+I0vjX?w6j1pYzbDs7VZCI={P-j2>=&k@R1Khg!$} z3L?!uMVgI?YsFvh2!!~)=No{kf2#`e=mzRi0G;L`=wG414ugpK*&Njk(~40#=T4%R!@vd8=UjY?7EP+7O4v-q$gkoc;)( z?kLYlckezt8sNa6=M@@I6?=vLPt4KT6KN|x`PpEzWkrD3u%pRRVU-PYu&M82#)bpA z{A)jUDcqrUi#Pv`e-SyURd+KzKi4%UfZOESbIC&UZm&EZV;=RPSKxQ(%J5$2F=8Er zeF@Ghl`Z*$_dc!HT!D;Ix{&IBQpk@Al8wu$3OHArq?oF-UmvxPP}O`}kSLz{lw$^4 zf?ocWLqNWao-m)U-3~%?2CO4cHSBN@hpK{p`>3##ex0B_7W6 z6ow$$C!_g{l1&U1+i$lHS+A+d)j0l$4G{FOv~Lm5r@{kVg_(2e(8dGlmq6e)(1JoD zLs;wcj(Y~lf-WiwPU5Okho`ej*ED2=O$nn>aB~K8or?oa zGE-CE$iO1_Dtoh+?rHFhDlv-HfjKhPZ$JElf`Te7$Te$VJR`c?@_k9krY_nScICb$ zmn*zuKoty%B5#F6-g!I1L80iu-TPT5+TQRL5$->flsq?#*>4j_z_uH2{MzoC&{^08 zl*Hn~jGi9U3yQX>+OS@$ZeL|p6}RDB1w+oDoUOvk<4!fCcz9-CH%1kQwvk^o1#CP$ zJX_Dqn;idb+-F+IaJfI4miwP|nA_?d*K$H~;ul9V#TqbXP1h6=^}SMLN!0ZEgh` zVf}USbv=Ws20Qb0Uk0MC(5Ya!AX2q5j|JgB|8m-pZ|``jA13_#p2eKv^;l@SX4`&z{cS|ti74-xJ|hEa z$|kMZbaRKzg~XzmABsNVKEzR<5tq!QH>yd5GaM$gr{w$Yw{I0*05<`byWiK>C+1>D zKYIELSRXxcEY*DsKuSYUZKRajnrkTej51e^%yvYt5b{B9v@A zfY5m-3kH)j!it(4?ku^xsaeCWSHn7r*R)8v*`?GzW(&~iq3(tBAat0<^~GE=!5~n_ zZ%iI~{Pwy?*xAdP%y|goTThk72Scu%Uvo7#x7n?EyXa_6x;?6>v(4NTz4P)|htfmt zTdEU`om&E_4liCBzP+|yRUBb$-o~Ah4QckOV-XCKgyZ+KHSA|5pBq1~PRub{AWqz1 z9&WIj8W|oMTF2!tVow*Di|g=vPp=dCmK4ho9Q4h3EJppNosw=17mB=en-Ett&VMv! z%5bV>40QT#R2e8S+fn?;ZyUOVE-c@{Kxmn7YvX6}p%;75%63Uj94}TO?3EiDWEVcDK zuxOv3qLL)$);Ib4Y4K!MSJ?tbk)Zg~tk5AUDBQLxDQoJgbrF@(Hm_$WzcESVK9Q&} z-y-7nZIAXb?Y|4!XHp{1dsnoSJs9!@#=h=z2c+VwVT)J4*u)DO=)R^RRekFAw60$* zADQ+WBGKmy1J%8_nlC$_ zZp;>w@pTz)^01Eb*k>w^5NMuK_RUo}p-wiUq`$P&`Qn2{Kf!rC0U#_d@T^d1R~#^f zJ)S#Sed2w2IZ*NE?*gex>rPJ`-MS)&DViN{E}xx1DD1^@SZNV_gmdlt#T5ZeCCI5h zunJvJ!;g)siM0=_^-G#I|D&vO`OowASX#Dp80_TD?h%heA)6+C9z zG|X``JwxQCx7S?S(pAfK7;SI%cDxHv`gC|<{`p1}kEetd12;41&NX-Qyf~X*KEjiQ zrPrHyDrw9UBVy=GIG*5N4srkGF?5sR7QRn)FLiRqN!HSc92rJ?Z){P>&3F2Ph))a_;*Up{-m zmllv}gN}qdqdg3G!qidk?cZ@ts{f-|nE+p}b}dJistxbjitoUmrhaF5uAcQr#dI#x z!S?4S6>zt*G7E?_f}cb^Ku(p5s!sW9LAgy=Qh!HM5kz0EZF@tb9j11+Wezt7D7D2c ztW4U|)1`0}>Lyc>p5~g5FBcOv=tN7DM^H^?O;?uoaJ8+YR(cr#BoofV_F*zUuN4b@ zb3SbH^E&a(mbslA^_JYR?ngO;uO==DR#D?}y_R(e-Xyerr2$wl{8htv7KeN7<^={oS> zWjz%lo8q9|`Q~w-(ei2S+gbs7BhM^N%25*lgT(Aa7_&11Ek`jBTj5 z&}x=b49eDf;$NjCOL}v{9P=VrRw&erq*ev|?TZ@zkI-Q$;y-F+>)uQvvR$t^&1&kHr>kZ8qrl*u8IGo3)-}((a0r2qpO=m;8w;L6ooA$IQcpDHWO> zqlHTqr?!R{!UcIimxd`>a@wJPB619GA?v&pmbL8PX_Ed|(t{P!i-8;qb(^aC9`51V zq_JtS;ZyZJqRm%cE?m=huft2NC6R}~Q{uT2iRJ-5E>4}9wqURuCw}elZ*dVIXzNo` zDww|0kCtyxcn#y7fgH2cjmS$Yk(VGR%n@Dk7rKwrd@3|%Z~2L5i&~b^f^K-X7aQKU zuUe^O=7M0IW?z-P*cy4;{-?DC5Lh@%C|Y{&Dq0#xIF!Z<=U8q_l-ao7l_b_!w?&v* zU&~OhN;pz@`ZAEg@fD?L&ccD%ud4L5%A=^PU;M8fj8OC8J1ZYooh|`LfAxU)@ESBd zpx{7nl3U;}gIb3c13=0Wg;5*uguGK9YnVH;7%69RLIEKtFA_J(N5gOy-R8RXqN{VM0ZkiRA^4C;OI&TxvCj z#CMz@_ZpG;#Nlo~S1o^~SZ$PG#EWGv)YFxdjy>9~uV0Q;ggh`qNu5JP@o&FW#CrWu zQpleb&Vna=ukPS7IHuh=!X6G{O)5+suB95CI!&}*1mZfmV^h@CKGpWVHvVgSHvT||)EQXSP=pT!oCGrEG}LC5|&WD+4Qj{q5EV0zD(B+QpFfi z<@zEbSlJmzLI?%H%XZspd1Yl~G?dd`y7&sA+-QNY3F61QgIb#{hzqD?i<-i85bB)Tv&iLH4E~;QYLPHl!rr1c5?Q z#ztoKCVCQYtf%4PR}yqLDFuA_BrW1f!m4Xm))CqNO7-pw%ZPv28%8zPQOgfKpgLL+ z6v;fSGI#3Dd}?Toxst}^lX|zq!pw;tmEHQ|{*NRVTz!?{KGqJg@iPcl2~}Dtoc3bk zkNp`!4-wr)c)ZpzZ(5)(37LMHk%4v^AxzU^Ut#UOLN7Zs$sWI;$uxv zA9|H4X54971bb$DeE8$rv<&6^>BkT^KxtVd7v-(ugq+@B|454v7@>9Skk9Kz=Shx+ z=JbPQ>-HUomO6~%m<8zx1XvEijDr!HX@CrM$DPZdGRrNnfJSfl^UDspa6D9&hI*sA z*VG_}JtXwDEdK%GG$V#f>e1LIx{QbJX z4^Z?uGvgwZ(O*Y%7yU*BrV(^OgkJiURViN)8+8&(-!>w?M z`XHQZ-jQZ#+`MayP1e(#pgZfX`?P_*`x8Uh6-pn*7XEE&0~+YCFgp+23cF`?Ugce|j@21#8Vs$NUi zXkrRO1De=_wqBEr7KkFDb;b=8uMc%WsN{)CE2SYP#5_i4rJgcGk2qJNZlJWqs1W+0 z_TS_%MAU*|fbaBopgd(+KRxm_MgAZpfQ0(<`btHBT;f~_7}J5q?{n+cID_J__1fA3 zAP1Se-8 z_4V~_2{ji6e>(YM7Kn32tDQgK7SP9Qm@{{NbVy%CxW)tmP2#ba?U*7WLIB2G%;VNi z1HapSdi)mGyyZC@O$#0u9SG`#fi*zDiMCQ^!Ro451JPhg1vGx?csNW9sEQ=|qoDj- zVRHQCM7fr@^v>yreoJSzi)l-Hj}|yrXsgqL{G~m6p}v-HRb1X2v|cZrkc5QAMS;fW z78E{~u*D^z--aB?1rhPa_uHQn@@em8+zo`vcB7TpSv1C)T5qFoDL5Ud?baU`O9XI~ z1&KM4a8Yt@F>8Xw?^1@)(;x_Ytduo7oHN0lKqHhGy@?HX+P{oT-1t%m4Y5mm{Fg$g z{1(0im73ts`C62R(O|HN*yD-rGH5ff13oj8gnm{D0(9tz@fg_QywNv{LM=V}pzvj4 zN~SzQ3&-Kk+z~caq&kL4rH`=&QK((QM$(8A;6sLu&Q4D7mPXhRR>%>GUmk0_u!1SX zVn)FSRG3$Bd<2@@0|Gu=W2uZzE}i8IiS8H%wK7q{q4VZXsEKA42xd4C%TA1!#x71S zJcQJh2d9MBh4C%if@FzDK^F3adXnzh7sF%vBS#Q(DY&0I*&ee)8N^}2q!HO6Zq38w z4;0#RB30g*VhFC0tgzC1t}f0ZHWKqBSdFx^pH2zBSP>S~F-R7T`%J7a#PU@TMdBP8 z2!sFxrSIS448RJA?r3pHGdlW4Q;^i%_9}_Y466Y+r(0$R~rXGaS zsw&Y)*i%EQqj!t*9OY^VN72v(I#P!!zjwq6JHS&md9BC9gA)tEW{DQag=1C##!JWr zp#Wd)^nNXEl#$;Np!HPf%>F}ke2*266qyEzJ|tk4dD-%_Q7Tp?aQ52^Z$NkIi0U~+ z@eXjYPwaINpkY{nuw>z;tHr@`+}=;?(q8rcCKq`R0{pE-tBsJ|2#X<9&fO2r5y1w) z3Np70eQ$I!pD@4FIEA<+<2qdatsuVWDg`^Y$>x<^)v=G2n2vU zGkim}{1{R=rlSZ2eVJhU@O;A_nb?>ZJeHDZ@baI9;R$t=u-PU$6AfFMT*Nkv_dzNi z>DeDA)kI8#5ZN5AuMPOo^Wi=b4G8lGX3|}K;diVg?9~f#8`a%dnBiL}UmG!tLF8Bd z^cZe-0JaPM$Sl~>hw-52M{Ho#NOd|G>eyfq$&aN9;$Th#OeH%sw)_37#fGaLKf+)= zF&Y=EGe?eA3F!{z&wyV1QC@`*zTomaz>8)+ZvuF$2ELMf!4-!ZB-hB3=n zX%XNcf0)546}bl!BV@R_NcZ;N5EsuP6_jH{VZ*@(c$lZH1>F@ti5AWYU!;}&r4dkXqRQ~0Y!Vz{R9tlH6pTHNUiKOPk zslkv@1^zd)xTh1^kb;w*$#4O`UqmW~uVDpz649czAx% zbvk42yg?DB<&uxYRrY>*BID6{8=#GttIG4@T8eIcLpfAZD{2S=i;Z%rqzy!+m_cA# zEgqL#=ZKln{;IcdR5}0$)-Ro~Y_qf(oGwtK`(9URC=n5W-Y4fP-!{9DD%>;xw;qAZ z%1?Xs5UCRr4CtXGTIOkre!5QQotX&S(#dX0hzjQD%!n@cth~NQT-&AO&PGhj{XSn# zp#meD4A!iLr>AGX%vl7#C~OOC8>o8<%ZH0MBh8`@jqt6IL{q`AtcEXt+DKnxJ*RXQ zF^d?O14O|T+qKJU!9Q=@00Kpg=!D=Vrr_A;lJg9;Y^t7?5qUWxq8W}XOG9+oW>XoB zaF=0IbtXlqhSuEunImR{3w;;uO)YSbi&EcFYqVK8^%C+bYClTHGq*5HG1rpvMai2h zEX46UAt6&33`nAjKB=n+zM3~8f>G@RV3bfao17U)W_u1K;=Tu>BMCu6 zzyrfRjL9yjxUtx@pyNm@1C9q0e>#hsB&IHyLO!Nlp-#5xV1EU>e1%}nV~YjzMXEa& z33r6vzWoUfP9!+2`ZR2pFf$B352jY>hly5-D&Db1l*^*7?*$q0J*w)9z>>}o62&9Cd(w^;!Ug#?Da;fdMM=%CaOO0|x9QnlD!y}?^ zAd~+D-Ww-=z3S;fOt&5uULvMFnh|>dGInC3EJ(JmI<5*h^$>g*gT_87-RD|zd6-Qi z#xJXPN_h{q9FIF%Eb7@jt+@O`E^xA@mX;QWoD^D>2 z#Y#Xc+{4=1 zI@Y%_q9Mqv#1gAn&CB^D28d-F4i*Vt62ea~|3su~%Hf_1t9;qXZ8_7rfPvq1UM zJWBd8z}PZuG^S{jM08tY!~Pb;0vsFy^J(f7SsVC_h7kinA#iSV0g{o|;*!IQsn$-Q zX1sxHB455;Ve$*MFg8}*p=lc4c^rccGYT7hFJZY6kp-`)M!sXxzGa`K27p8Hy$RN@ zn`l9x(!R(5a1-Z#x^+(kBMIOM`mrF{5l|}R0m6eUU^SiZZ}Y}de*DHh>3=bhjQ-7+^5xs}*Cq@me6@{tf2O zGR6vh1&izl!RN}*!2)Oi*`tZm=L@zTIPll95HiGjPU{!^>lURVayp;AmW(ZX%f$%UFMV%J!i87?NC8#PP*a72D!rJTLOW^uMRfZUhh6O=MVwpDs zNJos>17;*J=k)z~#yIV;CJHr?`zvK>(s;=L@zw?ue>P4r7o%K8)QC_GdN*jaRr_#Q<=l(&&{0da%&UnG!3B3LtAwK!_ zf0%Y87ld38#D#xVluw!sX*Q(UAoG)sKso~H2&5zYZ%)UO;ereoWVj&11sN{>{*^mq zR3xJ!85PN>NJd36Dw35bG8_Ki&W6*>!MaNj%u_D@z#LehCVw6OT8|dQ>u-^DPAouQ zMkF&LnGwm1NM=MbBa#`B%!p)0Br_tJ5y^~5W<)Y0k{OZAh-5}2 zGa{K0$&5&5L^30i8IjD0WJV-2BAF4%j7Vnm|A-k~{G2=&a(@ctM`7)Ec86BZAvOQx z5Cl2yLC(LB13~2I6FCb;PQ3vdkdT0c1SBLNApr>qNJv0J0umCCkbrzu0{K!1@+}$U zYePt8L^30i8IjD0WJV-2BAF4%j7VlgG9!{1k<5r>MkF&LnGwm1NM=MbBa#`B%!p)0 zBr_tJ5y^~5W<)Zh|K7~#hpB;Gh}D!`w^*F7rS9w{b2pj0$=prmZZdb1xtq-0|GjfJ z;xzeY3G&Snuf-Fstr3tb$LH;@b^49^7OsD60DJO@dxI>0yX*TBuXJs3W-ukltQ8u5~YwR zg+wX;Jx~hSSSK6nWMiFdtRo|lTD$*0w02)z%sc$tw;^yX!A@y=^?yp*ed6-}05s*; A{{R30 literal 65527 zcmeIbX;f6_wlz%RQ4<@8u|Y*3DvF9wDhkpk8YKk+Dhkr8h=_Co0@7BZ5feqw0zsrx z5fMQ^kiIDql^z7?d!%m*fublVs@}OV_q*?R#<}nJ=8W%-`)B8j;Zdk&@8?--t~u8; zSJj_Kwbhp|S-(U;Kwvrbm!FRd2>jza0RhSHzWEwI*>QZsQvm^f9`)y+PFx8aZng9b zv}hXZRFatd_2J%oThA_BxaI2dpN{VQV|9JPwr8jGUZyk%ygd7p&FOck4<9xp#Xml4 zbj&K@n@?FwzWI7d!sK^fyQ!7sxh202`Soeir45X}1x^-@l?sim?XvvLM31e~6bj`l zfexB_z*emi_kX0V%4X(y95$#N6E76=urcr-qnUbhdXmMy5)de97|mw3#T%q(ZCdN6 zGS#3pxNqjEoRGlltUNw%Hd*Yq@|Jz|#sL!4%w(|z0s;+}zg;9C@WW)}>HE9R97r_H z6;v8dlCy2P-2b8Trc}cHJr){+>T0}cb3MH}%SbQ1nF23Xt$y}smi6uR`|1qEWzQec z)6#IvEbOjN41XE_fgM_{#<=*m#(pSudpE>|6@jnik=C$ z+DEf^cXKt~)}%SjwmH?>6@PtUnyZ+q>NegjrsBTg((A7li(NTg8KHt#ACjP&dVYR4 z$8J~S-%oFG)Wix4sd4XQao#MYyt1xODDwK5#-DBX-~Z_k=lVk5bxxC2>M_yL(RMBa z)mtPb`wE2AT3$=Gq}#W*d(Xb;D6wbE)g_uSHqmy=%e(e}cqAEi=?!m|No#Q^%pH5P zG?pdqJ-ty%%9=aM>ZVt>P^82iQ*^J0QbrMDm)7J=)!FC`t=s^JSr! zzsl4p_Gs%3A#omJcq?^}Q*YT)N=}k?xMG3Na*t1cNzKb#xGYkQp%p5pprFw1 zIiWS^DybcID&4LXyBuw&{b28Q++=I+Ve42k1~2QDX0S}SlB-?5eD6?wqS?!{?`XV< zNRe@y*{P9cEY~$5)%}ch-m@d=_*;fO;$wD7n(NBUWw*sdxDO7eRC@C`eL6}buauZM zvaG7OSV>3hYq)cr0W)uQv|PSj`Rv<2*livZDqX(q%4RA(E+K31p1vuS{=}0r-tNV2 zm2dZyy)xE;WHdi0n0NNwP19WWeOA@oe@aGN884P}ZL*5dl*v5z`O`a*@pE7O`i6L5 zdYk*uZ;QqDU+Q>`wBT|lD)|UbnT(FBVPRp(V$0<=_Ru91Ml-r(@}@sIbiH>Tsw1K| zZ?alb{r38M;o&A@-m_Rms$}@9i3+7ix84xv!5XT`_j8D@6S2pmX?z~;>7%yN$X?`D zOG`^tlseXj)@+*|uI%PWmslz+!~I++o_%@X_^!20udF6!-)rUSCE(ig-<6e>E%#jB z8mFJQM%wU5gt9{IvRK3Wc-pNs((bImlSLF6ldQ|#MRzU39Nr&O7p;Bx)3pSXY%%AW zW7x@3M#07#8<0;oG)VCn#Bq>^A3?He*5Q2*G7wAfaQ~`XQqZ5)%e^Arl617v%|O7 zn-}_(+xu`0*DAT#bO!(EiSWwI%*2M+^E7}OxV7*ZB?)RW*1w1U3=pCjw9|m;uzugg zKye`v$1$JfB61gN`m=@-MC5E55I_MD5p4h*je9$6yC)V2)S0)MG)fz#0YHR9%optqj~dws>|;2=qb5pU|`^{I&*=l=GE{cKb=A>!AMgome+(X(N*Zr z7~4 z>KzH{AT^xE<8bP=pYXm$cphnuKlSo#{%nkQRstgOw&Im*iR+wzw5UNC_Ohk98oD1n;WWd*8gSb*Tz21J#Owx5grr7Z+zDFo{G1sGoRt zv~wttkvEz(RH4uphC<|qJ7TxGwx85D+2=VuCP6I;rtq=$=PWIS#FR5|KiKDBV@K-Y z!|z}Jwm^2zo>8ELCM9>W(K1p?Jpy6)~X$n}P#GX>Q= zryqX+hAdpVhDcIuHzyAJAgXwIx;sE;3BL#T8Lb(-#@Hb1vek}_Q+QE;zin?>$jYrp zGWs9i+MySpfl90uq3i}=rgcvqKq3&pH+J(nf&Y?{F<4y@?>pS{ztGPeytt9Wr%Wzh zw)Vp-tB+TQfFjbkrtUezBz z>IR>!OE6KAk{bGOP{=agX7bHQ8ZUquHru`$&(`%o=_$1N$IG<9;^t z=4{#bc4vBV*iv9ANNRi3{~7#>dZB zgk_e?HLv28f<4XRe#a|&qz#hKz4u)gTR)1NcmDWP3sq6c{Jn3Gv=Oi=*`gF1I(K*J z+T((1+%bJqw=H~WF0x-iRyGB9D&l25d`L@8ZeHUvQ?v#G} zIAuA~S;48-S5!&PIen|vgUNx9F~%SnvvX?&k{XGuVtCI1?O%x4=j>*M9Ky?U-1@?U zn;eyKQ|HScL|@!UrDGRt0a_npbadzvva(JUVG6;f?gLfo;;pf|F~n0je0{^M{M-d@ z!0OdIP888}E(M#oWX?@B>oULHnjd7}P;<^ErCcs1+@&eXUo{K4?=sSKGSSp4|LVfT z+^MGS;sDg^x;He=+m$+=o!5nD*v$4Qb$`&DxpSaKJpAoSbwi50HWB08vx7o_ip3`Y7;~`@4Lt zK0alQG_wce5>2fOYkV4>F>Q;=%9Me^Hv~lu0kydC@^B?~LiXr$0CuOnzzHJR-`$Lj z+rNZUa{Ws=hyzQcYsx5$ycei9^WnGaMvrGn~!y;qoBsLXXuLH@i)Dx(S zG$B?Btop$|>!;pLsCuBLa{ST_hOaK{1_GJ5G+G=qjRm~*EoA)OiBhgj?~YHYSP%G@ z$sQhmFJfxu0-_lvu`JMwunXQHCrYvDHncOWzD5m^f z(;p*wli)v3Rr9wX-cfq%O+9S$+cj)Ey$7O$larGPl&tw=7%6RVwEV%oU}>YE;NW0d z^d@d4r}9vxx~TIBUdV5$doO%A4bHhxKf-%vfJ;laX&eXHGh$M(JijfZy+paR7LECi zxV4RUTfx7UtDSHtE-LD8Oo28~sW+r>nNFhwDvxHy(uCITdf3sbtvK+cChgq;Fj8t? zMYu`NGJ$ua1XJjJU@hj}|Hx~qQIh6>A`&alq)S+}<+u$Ne40@E&Jjo%YrW6O(eYw^ zLO7owsCwUri8ojTn>ac-F=htiYDS}&3_j0C#ICKwNTur;laJr<)anIL)A&o@2Q4$m5m^{IPdByZa^`83o{L**?hN)AH#jr`K?-dKlF3||2 zMmi~?M31&+$Hc@iA%gHM*FlXpP8AB4U;@I%mwrNp@Q1{MLcz^Y7A#u!i;j-j(6Q?3 zYNEa)yOCy!EH4fGrbx}=r6~$F4*}}AX;Y}*E3MA)2j>1NX=Bp7XdB(o#R3l>0^}s9 z$H$ah9!VHGmG3Lw>gdx6!KDgXTk3w3AGHR8(<0y-pSRYRq1z-f`_~Bwd^0aYCDsZE z{Cej!1onZe(6Zmye&1doAn@A9y821+U$Qn??04y_vdbX8&1=4nE?!5d*Y0Idvfq7= zAO85uEU^`$x-M`8*FC28 zX!7cr7U5f}vu8gGuM7@s+MH1TiBF*<`hHv}ARy-zy=s{npFS~Qc}qXBcy>195QFD& zazMF1%hAzs4aGDEh)2+$?(Pe?r)|bo|2Sw$_v8*dK6J||K*C^p=sAOymT|=_Ch^qE zPku^F)#*D0uFd9&e&K#sx?f5@{{Gaf3sp#Ox}>x^MBC%QE4u<;u?qpC z%e|xmDKsa3o>u7Sbla@C`1+ZmELXsEW2G8DXLR*af)e1u!_OcnSu>HeN8BDY-d%}e zKP^|~0UZ;gk=**0%n4jozz7cx2(vT82xD>Gbku1tya${IwnTgm|$Ip-}AW&$dYXe{E7Ta-79WH zejF)#k@lR|^2gnEJ+fln{XAK1r7gL}nfj(8xpgX1C!Ys~goMzlobaymRDAsf0fE!f zRz@|#D*zxBq4E?-Nl6uzT+q4IQhGaw{=}~XrqpOYXQIxXJ9i?{R7G0)1vmzX!?qy@ z{8Bgl-AG5G-iszgR9e$TsnENCz+a5&f+XsMKIu_;jrhm%Uf0@Mp`NN}zC=;db zK3b)T44z@#W~BDhO}15!4_T-b&`w|hz$KB-d(k$UbQD$n`6OA(lcapPs2qHT8`}`MLGfR$1)-7V-G) zo|`#^ODTDm*$-!B^2WF-59ImWTc4356xo!Evo4G8y}4xV=KXeQx=_w+vL>a?$zSEY z*A!=;S3;bgbV@<~h$iiO#MOZ((uS{|-4XBW>2X$_nZDMXo;`TeH1=McyP1;fXiH3_ zXHJ89EJCC+V1whSK0i^x(T}swYcvw25_l-ahnLZ^t`t7QoqrZXmyzeX`sEg@x_EKg z1EqAY>@<$lh@j%7l6u|XohF8FuDt-^H*oTvi8a45#gST^uGTMZrnP;JimfR<_Vcc? z`}?G*^chxqrJ2=HhBsXz<^0^K$L24FZiFB~tv)>T=4liV^&v#TNpHIjqSVN5)73Nc zTJ4nM&ehQy_;M$Y85tujqn7O4fnKnz(Mq3u(tA5n~cR27dy5BcA!M%&Cm&5;{Vqai~qRIpJtJE~DFO5A<;+syJj=d*+>{BLB* zU~Syu<$1K=^U^+ZtO2^%vm)wfR$tpzB_?k=i4in{EKk}}v~64rO^wnXeUphH20uI^ z!gR@k)vBPrCNIb8%}PDz1Amp5PiJaXqJ0XhZu1Sb0Mi6tHYIm_(7M?smsLvf-8MmN z=h9=7>wLhPa%L4;HN5GMMV)Kz_D$`NiH`w{NdHo0Uu$oaXu7@Zd%lMK`h)F>)FzPL0LMaD9chRRU7#zPM(;cd*Uy>4E_1GVC^5ZOYnDL1-=N2<-S znE>F;noZJ>z+)a^S4sw;RP?Q-^JdfK2kYDOuBdOvl6Jc8AAK7DRwr&P?LFP5^iCDA zd*}G&U4aiaT|+tCW|4q$@E1M#|A3=^#;Cy0UH)o8tYx7=U`TwQ*l?(m4mCG1OeQq9m?d2S*yYtyi$$yaak3J+UjCHVs-r5@3GGP%PgxUR`J6P0V2Yl#xn zw%}`l*ZhP{6}z6q#Hg{$0*p>3Mlg=!!eiS#b>HZ|pwW+QzEK~4Y%|TG*>>pO{B87X z+q9NI6{IUS6q2t?b=EU6NLu5{bCynZ#<8fY0o=Kd4j#{qWseFS;V_!1fe`X7H03eHy+Z!N*jz!Rh(rDBW|4h1M<;VNtHWJ| z+-E{M*{vQrU5A&Kx)7<5V|#QL(Ytyrojx}bz8k-D4Xv9EMETKiG-rgiT^X`JD4s=Q zcr^!h&L>#I!_yffe%WnawBZ6Fl@B)TRt@)uZf)$hXgRtA(GslzZrw z3P_GwPSh%%$}%vz8?LJ7{Z$@>dk=13SkG>Xv-qt2$pc0F2eTFVW95?rl96cV{G;+UnwIV#_|}ae6ZnBoniFRkP+$O(%WLzX6jg(*Jo^;L|tP9@WhME8T%$>H;>$ z{vFo+--B~UtMc!JhpUVw{Ipoy@ybMyfvBL$+5XM`JvgF583^xAQyNuFGAT59kb0q+ zcjtzW>T$g<5D``6!R^Ts!NJk3QcCs{^sKuikL2{V%|atmP(F^YwUgH8_cZCK%?&@7 zqFA>)s}k;#Y;j{hL?2zv?D9`vO3@#$(5pY-opa^$C&6&PoKZ^s29G1bH6_u(CJN?t z@%pA4jtxg)ck;%lO=6z?x9eF7ybSZ>Qo?hY^}YrvrZkHaFNvD=WR-)@Bh=BZLNDV> z(Z=qXzLS&m4co)IOTUd=ZA`8Fg%a?kWKIS>2^=0CMnTS-MgJvFeyZ-PInB#YmxpwR zL-tvgdAPQ?JFO!EDl~Wbav!liv*~x!-=Y`Fu=tTtp)O|nSax-S5qm&hUz$s$5oc#$KyFOcdWe?kngE?0RWFxRv3-Wn!wY866i1(1&+No+ zCUteT$IU-e|Lvj5A@@r~1qa2vFSit$$AJeSw!Et*vPNl_iWUeh4LUHuUH`mm=c%NzL5A_{=q$4@ z@kqF$w6wG>#D`R8e%(|lWobf27F&s#7s?oC)$5j?BU06ImBaFb6X&zVlla~X>@vN; zeY7B(!LY>LhW|{E+@KFKUmZee7b_1Hz4DqjXP0Gx$0+Tyov95>yrInPkJMIv^aV%M ztLk^{L@R!5PRQ50>jgBNEu+;%UCg0@)8|6@GkD;TX+aX29v-8j+`)eN;{h9LIe#qm zh>%3yJ?yrbJXE>G(wyRsoYF8{QBUYsqdocf^Y2oIl4%^*5bRp>%PBE#^TfFt@b`Gj zX@7K*?-3Q}HSOgVRJ4CBxBq25v;CQX6<5{XX>E0v?U`0`p7eu)%$5Z6m}IekK-W5n ztXPJ9wc&9iHSGeIyS6Vl*jpZIR;P$N|4t+Sct%^r5D6!)sRyG60w=RR*lEz-TW zvCD7!>`G9|q(<i{yx^bM zZLT9`3EThTBbRrzIP|r8jMP2`iQb}iWQAQGYfA6Ml;Y$aK?@bEl+E@%OZcDF#rbng zbfGu!jtm^p{X#V_2xX>FSuyA|Crap&Q0V?*8M|MN(1C#a zYyA;M{Es*H7aUDTTgiE_yFDHR!~O7#wwCM8&e*yrQ=DH7)|E31vRy_deJ8_R$2*@3 z2pqjC_bD`jSU{~6Msy9`PW*9O8p(U7a=ydz&yAUX87cq#!*%FKZW~%BC#{7uS%TY~ z*1a1P5TIKp%T?z<&buRgIv5Rf{NLY*^_zk|5yEqBpsq`J?aSxvc3dy6vu-epyjcJ8 zqfEKsW#!?bU{mkZ`1b^}{Q2-*_NlDP1~;xXyDCjga_=doS_gOk@~E@&c1BOyZ~CU* zR;6pCdAxxPsf1CJ7lyqXMU?`Jk%LKFK#Fh*P?ystk_GN?+#su_f#|Uw9Yx+BEz?i$<-X(#!7ePKiJKwZO0CZHtTv>ozQ_QKN$MUX;`eKx?!s~|O^o_00;Bd+he z6La7y0dulhB7ANEK;-!!(p1)vJ%1#f59L-$nVUG|JXFIu%l%zl%;Pb7Si!}MUzmEH zskELk*2jreF8d+}{hW63*-lxS*W<}TA`;XMwY9Z7+4ogEDq7isJZOAAk8R&{iR z6Ye<^8v~{(Ji84mMj2PtQIX}|EfkBCxA*Qkd|PIS)wfx2=)|x&P?>}HJDxJ%VGMkF zJCEpHHaHR#=!d_VBS+t(>3XV8*2eI`UaRAOHAA9sTK1pK5WF4l7LkR2qpf-2+SM~N zllC5Av@@X#nw*v6CqPvJ5htnss||3N_b{K$Y|7~g&{5IVlLbTvZ3f#Z3ztt+ue{YH zmc+lacu^(T`p5!iNzu0UA^I(iGa0GyA5ip-TRl1x8Bb$kf_YKcmNn1%Pp_To)?Gem zPn*#4!uiY8#m2LtwrOZTN+T!t&9LOt5)7P*7O+QLPiODJLJ6epJ+!#M!(46ukncx4 z;kZ1zIo?$ESSN0rra=Z2<*915-P zY6qV)C++O$5{t!>XzY|q4f>PD(QIks=^@#INJ94{C@(_BnXa&0`&=&;T~xd5Q92`8 zzcdKES*<+WDA@`6FZ9<^!e zGgnQHX2JdaPvU@EKi#gl0(o!q^-a}dm+aUn*{uBxdw!_@Xtf(iN2ECC({+`~Qhs1< z^(3pt3TIYY^mAIKgpW&gmU|0L&$I@j!ym1NCb!ME)oWHa*sbDRZn3<^q>W|Myco{q z4`RaI$3tQw4Zn{fT4`fS+<_Wd-e>>BI>Qa%^GG1G|5uHW+qPCi{ciqIMS`(+OMP08 z7(T7J_3w4`zDV5v9-t@6FET!%^rd8cHt%zTE4sla_iVndmuNcoHbCe0TF$3o1*f#u zw6@5sh!0(1c7)c&9)4*x%{1Gy>m|jcGsa8eqR<53=;%o z)7fFy_f|2+PPMCJ@7?0HSWks1B>Jv57>0ll*nTMN%&Af+%Z76|L^U3-xfvJ{p*nL< z*t+i7T}E=NOS5)}V(MQS^zc6layS#j{(o4^q-0txApr6H?e#tFW##2_(*X=iG`)`a z5S97uVNZ?fB+1C8*a5%A!wmakt(4A1ZO)bcDSgJCo#+!0q@_Y`?^^UgKBIkAsXX`U z`|$G%T2;pqB+<+Ax;EPO$RX-6bGd*3*Hk2H%K(~U5#ixpmu6kIC~DoUi#Ko@VafY> zca;~4i)Qh?L*HhimNfX@PCkxWWmTh@=(yqv z3*bn(S9<~l+pRgn-d*O{b=pV3+R+)gd0qRf^zGBwkuRr7|IAP3kC^`D4=up*{Qs%( zw&x=H8U%Gga}*xTe}}rD*R1a1G8N@7G~3*4?ro5C8qEt)uM>SlH-B?fzu8U~U=3Sk zS9f_jZ>sd_Lg$P1syJz=9B?@wa)f9D{PEb#w%ws#Mci}Rp?xT^75ll}OlQR7V>!%l zeUZ(%t9*ECFd!_ga9rvCw_+G;b?Oq}c()OYzeOU9k7oN&La_@t}ZyiNMAZu8r9 zKCfQA%E`&`^#!|HvU-PvmDLvIT6Z*==5Fbz?Z}`<+{ORITEztfrn|sP?*s+8ym2V> z^ZXpmK(b#@ujY4|gAv52|qbPBq_JhrwqJq*(1h*buExL@Y zGWMYl0^MaoOKGp&WO;gfn-kLu4vqaWMj^SWG?U0S;v6@lQk9QlLugk*)qc^g!D$z& z9v|@Ler(AMM)5#OEfX*l{Pe zS5S9=mHVnMKwHo1ka59SmwyVJC=Z4|?59QlbZtfgP7m#~2^%hmPn= z9L^e&!7)yleB!Hl;iJAH|0sA3EOBM`?=i*;d_$;o*oI<$E;Ovax^QXYfKzKWlM%_- z<2;DXcr}vNoEG@k6YF>Fz1VOXEvIF9{Cwn*oL$RUB%eF_Su#TT`NWD4P`V0F&I9@8 zxnJkl*_=8hg7aQ36h(t|3C8 zgP$mxA~~OOZGMO@3g(ptNsD{(7bG$`>4X$cA4&TUDLi%h((?IQqqtV1e5+%^e0w}^ z(|kiU?c9HzjsG%5=YJUZ62^_>h%+CnVN0F2Kg+GQLM9LtifF9Ft0VdUHiZJQA@pdJ zzgksZ*O_vo^P*nH^*Ay-wZwe2jFQsqL%|!P^=V#T8ibi>%<&fAXi=K}O}29XNO7UK zRnz6dTQUPIGzv7&Bp+C|p8Fc7{-rLcOby?!TCoh*=dY?7uMQ4^ypGFdXH85Qxn5|Q zt}2Y!7T#Gm6gto@I6I^(xxw|wx(eKY(H~&*z47Ak*fl<7sq!;jGabVPN2ZHox-`6# zSVk%5wIYY?l;>&!2rFqaacY~k;m|)0+KoP+_oqcNl0gsPFyxG+AKId%-x1PTSs5zN z#BtX40QJT5qA&jgESfl6@ltWV$Sawo@norV#NxeSwdZ-=dpu-j)^FHfk>zEX_;xG|dl z0U&L$V@8*rFju-Sdo+2%$}*z$@ySrj2(N@bjtcAsuuMH)19J0@;O*(T8XMLKmrpdFDD%8atPQ^%3m?wv$R^_~M zH-XAFdv(EqR~~HU#(H;*qQ>g(0vIPi61@~c3`>Z>(=f7|>AlaQ#M#Roo#?JVCHYxr zIXCSraY_>KBKn_nwu#Hs7oB}G&!_WgUVXaFOG|XnT!*~dAQ>aqPo2U6n#P$uV5()S zbGO9StjZ($G~2umMH1%?X{Y|J^w2{XCqvc?H2fTm1-ZwxtmJq72!RTr$L{Exm zbrVV+55dzh8&D#(6o2jrkQLc7|9*1va#gD~=Mzn#!@E{({&HqT_&99r!p>|vjNALQ z{PyX;`4>J%%;(#cTPFu=hmr~et>NE;-6bL8)6MO=PoH>C;<)9!g$2AomhX5CaK?JV zL4WJ>uh14a|I&xw9$vg#pOdx|rzd0ii&K)5BkL#RRMFE$e(|nTZ+`z4#z8j(5$sgXRfmLcZ%{~v zBT~ehur2n$mPW%A1N_CWI{lR{);#;^+9H!Ho!84l<@uj(QS5R)zlTKven8uriJQJ_ zcb|nH8%B5&wmMwn1FhMvb2F@4J--S3dVCDdAEMFAk%!|AOEC+l_b`0q?;N>sIZiU$ z`S8x|cpvOS`@mE!*yS?Z@Y{mL@KlV$3o6rgSMKPbT09KtiJ)|xTiA&#c&;d@zA(T= zPpX%|lnUE8;qT`2=5#eNXflL0yc4!i=fU3CpRaYogSg{;lnRW3#JC9d5WZa9qE=Y7 zr<4Ryp6v-53oyfW@61s3m>Ooxjo7=kxmm*RA0W{PyEq)}<1J2=Ch+iKb>7_)w$C0( z!_V11YIf*;AZWZXY7ha3ZfxOP=M8bfar$YA&jFqWf`$pc`BHg_z|o691F!e%ET8al zK@c#wF$O>0AXW;g!Jp2zu8VhruNB7FS%2~88C<{9ryv^+V&bAgiTR}MSD6AWEh{T4 zO3v6DF^}P6j}Bb_q}aP!-?%D9ClW=q*OhCRDM3Xy0MB}UH4dvKy8GWM)dH0onnh&6V9RoAZz1oy)XiQDGP`_0C&xudYNaSC&Kf{b9<^PcX!QBw|k z?jw_I7aWptjOeC_gM)$!=Sl*%TZTL9p)csCI@3RIc723d0>UIef|tM`2hVq%EMB7x z_EhJ{VvG^NMAr8tI(qfE4e}07?RK|`-X7+d0h2p~0Q!hjF$G&rts|`QzP`SMjo4Y1 z#k??ES4v^Dz)KE4a5`Mq=<&C{xxO?&VxTQ|_Ky{tiBTma(3t`s+m5aZc!`iJ3nw}r zUK5;w!xTp1tkEvsh&@fI%U4!VdF(X;z16wi1bG|F!q^Htrn9yBW<-#{j_iTS78-_@ zsPr(z0pWD*uZq$Q%|7hA)}jB=4I=Yzz%z*YSJy;eCF+Uy!b_-5emKZ(r}44Ken^{K z+Dd_km*GKm2<9ik0ZW)~^^9y;aHXv&%}2|x8V`Jt*7W?8pWe=Kxi-Rej6G4qtH47y z?WO(;r>h|SLKQUpYGV38Txfw>Ihw-ZWUE7!SyWE!JhQ z#tM~*z)`b2F8W4T(VzET7i~)Y<;B$mSa{cbJZ;LT30N?Iy5qZ1Z%7rMn=l}xM zMu>AR3F>=4QIDQ``UKz+Fp6m;P+$}fGgz5L>@qqZ!)J@zRGaR&hBZn_p=9+xhO1MG z{{wOhQp6obSy)b;VC$t}Y#^2e=eZqbA%x95KZAz%5p#Tm|C|MYM$9uS`K(Id9majk zU=gQF5T1BqWB}J-is?BR@VnH-y(Sk&b}+9zVu0wk;am@w-^x@y;IQx@Jb7>%anpf$ z=!mi1nMvo0=~&JjR1)GX@xY)gPgfwu+$>uh24G$*;hOzw!J;Ffaw8NP2YkqDX3`BQ zK`T`LKTa6&^&Co2=RB&Zk|XdBTSmB@4_9f7N-)wMrBG0qsng$=`>oqzhm$EW3nPSA5Mh$_3*EC6HS2B#x-cB3dTp%TT& zS7IItn2`hXET*|eKRha6UZHak`X^uthvq?wA~|;xp_l>N5|qvwV;td?VE5}r5^!}- zfp@T6yTAwzOm(h3sqZyWE)TE0#rgB+uP<2*=YT$bN2s`tE*Yb=omBc&VVfrv2cFa| zB}~|D9!xI`+sI$|&;gL0p1|&**z~Gb;T>d9zli7KuK(EVcYn9}VHXz{yewESUK!Q) zZbU=`On&IKpeBf646qp(utu4sUNGB8oQNPzJtYO3VRZ3zMxsSN}4GZ7_rfuz6vWE_0;; z{`_`~Nr2P!w`Q|}+`3pcyt|TnXDAaXzy(wjlVr#e7fgPJSVn{>IuDAwxiGq}d*e>< z!p9S*X|B?%Gzp8tQ(!h;>k1a6ezsLg;D@>&@qhuWujvF!IzB8 zQ003gg)MmuA`FnAIBdYFAd1F5zVS~u7sKucx<4WYPHR&4t(4Smfu9>=5CJoZ^_V3g zO!dy!Fhs!>4nzruKMg8&{*=YMQiWZ{=w^e{@I-a?953R&lY$-~G84dyW*vx}B=z|B z{wL>s`xpm9{`Mw5Nr>R|sqsUy8c{U38q~Q)VH}0G9bCP^`=Frrho2VXEMpj;!$BTn zx}X;FuRYyTP+*%(9G?Mj!4e&VcT5R`T677jre($V1fbgCeTbPVxn`RQBQpB;-LxD5EOQJyCU6`Pok!CT{Hpi&j6bR?kNbo0lB zbM1K#ky9x3GO5zu#ONvP^H?>+OxTRo;F%?=wBzi<+3&z1cU<{O>+|SssdK#LFe;{|%KeVH zo+ltqz(c8w3@j)bR9E{p9hdya@m7}7lM3mbCRuy=5(h8(rm9Npfaw{Ch>-T`zN@|@ zfpRM82bO;-ub1lSRhcEOc5*tOKOW533v)h^ZoMIvIA$XT;g-r|JnT^2EAVh{d4$R= zigFpl0EvS7D8&*yuYsaF&dh{wM))_%;lhW%0_O3=#E0`(7KDazH=yce3>)J-#i66iyc-0 z3dE{#I0VN7+X{DWQ*f4r6#^4KCH$o*8&v_4I(e)*BLYrvt7}RnaPA261hGR46XVNE zN(KR}*y-cb6wCAH6IhOJFxkZhcg$i81UMy!tc2V$MkRvkZb3EKsuQ8?@aZip!t@W~ z__?F^S^pffJ?0Ka{wTndn4HBe_cF#K6r9fU_-*4S(cQ-I(?Z>Itty8^wy=I%1cmw> zt8mc*Wf0qCG;~c6s2+Itt^-D6i7Bv!^+eLQvok=}aaRzVn5WS;$hbsI<`HuUAp8@P zA9eU${!0b8f`nedDC=N57Wo-SlA(B01b=i4u|k)~L@bXdt}myF;51Ybh|-4{&gOJ` zHPk#bJ6@!UZ>Q2>ph<1(d;vyh=+Rp5jE)L1c!kb(BY^!l^Q^d2+3q3lcy3TH#-xT( z<%spVl!(Tz3knt_UKk@8o6cpp;f$LY71THLV*S2Sds_ngqU>KYX(^Se*_dtN#K9~8 z7x+#`p<)n8RmeMJ0OrV4sc5qWfx|>$1Ris-*nY4XNSa|VaKkj45O^Jfq$P3E3bVvj zXQL;W7X=>d#{3S;tP!-8))m>s452L;0mg%g5PJ_b3$Q`^IVRFOYc#bqcFE3Z3rlz) z)#!Z2?g3YrZN0&X+GzYaF8?0J;9Q7i{Lo%qm~ULbt}`)CdgCqfy0CS(BJ1Qv}Ziu}o_`ABtI&3H-#6{rh^TBy2)bo3z-F#=V1J<@HV0f$loHFe(UL zWeQb@^$*=^l{{Md4w)K5-`q~r>ujn7-0J*x6h7jX8(wyy6J2;)gbmiFk4dN2kHR9O zLJ{KI63P$JKH!4~no+i(elt7jDJOtFn>TksB~Y$Zaz~`GbtBG60=Vab7E}#uFGg{>AhDo&DRzIZ`E=)?V2HeZ+ntRE@skM*HF_Z>hjNLji1+^D z=mbpHsLSG7YypU|Lf@KbLglT&L@ag_%e+RF_S5`e`!vFX`0Rzjc4TZkvINR!k>G}D z+?~IOd;&-v46tnLCE74g5)(Wyl1G3XO@HnJEv~51e(@&KdoqJc$3&YKN)EaWp}Wq! z0s5+fz;n}=FHr=8Zin=R(1NaTLRSF;j9%ENTP-dpfutB=(uq`6p#^+$KeSj!Q!^W@ ziDA)Q$d?-0C8!Ol>7LE#D8+$OuUoecEyh?jA5YxJ3IRUv#atWas4?fy!OqpsK289i z!9wC>ORLowbpxAn+@7KbOz7}NowkhOftB0@0p7`qog>6Hn%qE9VhzqrzP1R%jeI@IIQ&)yGiiSowbe??>t%Uy{|2hmGIjREC#00}?dzUkU>7DRw~uqa7O+S_*BI)R_*f8hgdn4GFcgK^>6&;0+0?~P5dg%TI_!ACI}MWC z>cvKy(}|BV;NderTt-_IK(~OI@y40s5M5AQ2vSMRiKBx-G}_Qb;Iz~31qI1L17c>> z8EAm_gD#&#v7BJ^4YE0+6hDIK)by3f7$i%u1||@B7d|{ZHRZ^+gFa9|j;s5w1)0NK z^ghsc3?C{DJP9DMiZ(Z7h7mfoPk(F%-QA14gsg12RInThgqV}X7@jhIhy|;OUJMvN zIjA^rJA9+9?*8WEe}Ge5Un<;drcb0THiS+1(y9_4VmvQQ8G;Z5-q?DjW*(85*Z_Bk zs)@vPr?CG-;X0J>b8w2P8V>*dc3y|Od zfY9CBhblZdIZ1pl2|ja#)w2i(3dA%u>I_j{K?I0CIMKM;d*LWbDh(lmb_K>s3_!8~ z;B^=_pH`pRyd0nIfpgCGhIV%}ypG#fthtO?Qrlq(l1KJ_ndx(Ez z{%K&k@_SGm+7^hW0y8<=74zG(SxUs>p2$GKYYz;K1%>5L$vThMQF4UYRe3k<$ zdkeOX05{wb(e}G7YNabg^l-z#(p9kQXd5wLxdUdWfG8juq_`sCUkNmvC9W3Zi&Amw zG>B^6FS~56%CY=2ZEz$3?Szl&h{eZgJchOdZeF1axED zGaZ?)ad;uQ_U5t7$4orJCYHCj0`9HXhvBhZOB^c6$zT1Of8mJXXSO+UPpzya02Y1#a*56u3iN9l{b{@EW3o@B=M8PPZB@LL?9&_*}EXs1X&HqYWRPu zhSRj+`rkmC1b+R2HuSCBwp+h$A$<1VW7ST7i(!GJ-_iU!^#%{vc?N$3;kZFSlOUSp z!^j08BZSNa5*$dhBAQj1>8*&PXoVOwe(8!5A za`uoM#v~~LNeRfoWO4+XoJl8#^hsqzDkD-Ek;;fvMx-(#l@Y0oNM%GSBT^ZW%7|1( zq%tCv5vhzwWkf0?QW=rTh*UTe*qoj*aG?GDdgA0kl&j_evuIP6Zw@%>&Or8J$ diff --git a/sdks/python/apache_beam/runners/interactive/testing/integration/goldens/Linux/7a35f487b2a5f3a9b9852a8659eeb4bd.png b/sdks/python/apache_beam/runners/interactive/testing/integration/goldens/Linux/7a35f487b2a5f3a9b9852a8659eeb4bd.png index b21d3b606a37a14eb51318e077ebd82bd55dc754..f3bf660dba0f5e51d311efcb66cf2757f6549cd9 100644 GIT binary patch literal 687768 zcmdSB2UJyAvo%U<3n(ZU5DY+rAd*p#927(Z1SBIlNKTS7M-&l7pe0IHqLOolg9MQz z0s@kA&N-d%YNP#yd+&eWKkn`E#(0h%4Vg0)wCkO}#PDL9Z6EQ!v4Q*72@h{NoSac17XEbO{kzL4!uru~NFcewtw zAbHj9Mp~379*1=q@p9=(m0NN?_#AktG4K+f-FQ6?dmxpZnj{sBI~c+1Ryb#+r;8?V z;w2PBw!XJUaqD@mHJqXr%+#sa#vQCgGR##UEC<|+YtTY_V|Ujkdb4%3Dx8;UgL%z6 zA1x1-T@7W%JV%C=k~UCcJJOS-H3E-Z?RBtEspHhGs+qOj#y3(q9c@Xk)J2QJ7?cc% z?T#L$tB0WKbeK4Ym_x^3x!CM)&M}Iv9ju^fuN&Th^&=vq zsr1~l$BW}#vtW+Trswy*L`)_Jk8}3S8D00el(rZ?)>q?t z{e|XTU@lUUl6x!i38G##_7kil(pFdZ0@|Oeb3X1i>xi30lJ{*R^qMgW^Nw~;oqurgZ3R5PeU-LvhPH;+Zu52cLgG{5jxjCB0Jfv&xyG4=1 z%-1Tg9UAvpI=VY6!&M4#f?HpOR@gKPKS;bd1s8*9IeCe9u-IBlT>NR*9P-OZwU^6M zzcFtN7&*$WXx~GaCmP+pnA|nzOHLD<#-o6qON|+Jod^WO+?ne#o%kFm?l0=Ke_7PC zR66u3->|DuuIZj<^#yK|&6SZ6)W!aI5s!Ewmu2kE2$`cJW=0IpyjpiXrj1(A@k>+F zX>o8UEcnZQk-F|5tp|$7QDbSlOSHnS-zpy?y&?;`7vcJCZ8^Mz! zuV@Xkx-B@3CvbW7#)?-TKkOu?yU1(B@3K7D5J0Ktu~ej0Y~?oUL-hIeg;3O{>}cl{ z_%aG{u+Ph4-oi{wO!(P%mf2+{Fjmn7g|sfGhOd2Li;!2z z6|?>68BtEvc7G>XEMapNI(TRR_D-r?3{$+T;X+T=*{k+d{_C@3h%P|b)43CVX|GgMXOv+m!-Y_to-eZ}K;(Bl3&70y!$^`7rZ37?UR zM@-e6!7p{vO7v!F?SSW9ggp(D@SGf+fiBwHD@(KJmD?*Zev$1TM#mydO-#<0Ri~$? zu%*^7^?pz^8moN{+XFTBoU&5a7B)1vYV}D0h&mr8k4R)Dc&;~vg`LB@`oe?nd(1s2 zPw?9A^F4mnrfRDPw$N_LhIH->4rbkjlwOr4BckjM0EY-T1Kp5QmS`pvm62o#iO&vJuz zqSW)WgAdC^^F;C4#`jqT)wdtLCR=`%Nj~#ofywUb80@yQ;`c(n!_=d&3wq#6iU*cH z#j91hJJH>_d-rag-$|_O4Q_5H!8p9uw#`tv9Bejlf-NmAvhwoV?p0AyQDBw=PV*`< zGR=~`^Es=FeFfm@b~vTwg?VzVEZF?tp4R$FVPYBU>fXv=fxiiV%Wm_+av_HB$Y7a3xcf5Wvu%n{`wvP8qoQuWx52uN#DBlBWP*haZ^WHPoZvr>l`%#&xN*8r|;GzXR^KmeZCke7 zToJplbp-?jL_D@h7j5eOPs6@a99=$kQjw5@vmCw`cv--Hx-|;mioCS+bt+Pi?L{{T zY!IQ(@yjY1+)Y~3!|(Tg|EO$aXxLwDT>;AtgZ&!C&6F!%Vl&i}ts`VVc@IAX0ULa- z)9UD_NF_@ZmG5t_8-Z<`@Wv=BDOF?ETe>sUb2ReC>3L!5l9aN|^E7B&M}4BWO$Fn= z!tz#VX9tV!f0@}}*R3qwW!j&S^j!T!1b3=lZ)W8*Jdl=_*35G5OqNc%8+b;XO{csN zJn*r*rbh^#U|wB5oCjZDU;F_Yzu{gLEQWK1XK`K`)|uSv>FVlgz~9x_eaLn`py5pB zYZB^2(lAp&y85JG(f)kn77A81fJc<#{(uWb*6Ppl4LK^WwIfcxeFn_9&ts*k_ucJh z%*v^b%Yy=|W?<9(MV200^HXR99eq?z(`|NTK!cOJa`mdor>DojJ844T5(SgfvaQ-W zQ}QqoKuiYMr)(q#b~03f?Yt}?48+5oN$7@6H(-nQ+M4eM`||=9KuRF$xtog;e~Rb2dgV&em!$)~n1~2xW+TQY zE#ViSQmu^Q60m!z>7Cw8+4>p+5Ax$4}K1udj9M7$B+b&do!NOJH<;-3L$E3 zZ4E9#1s?@>ZrUEt?)#Yk^08uDo{Lg&0k3_>(5COzULL5lI}jTZlaii=CtquZM_K63 zpzNeP<@*}~yH$Q~AVV(5&%$B>2nqnvKA_o{o0hx=wZ3HzGy4j(dNoTWBiZTm$WuHX ztBt;C8FcaHJc0~T<6)8z*a)uof~i`=l)Tav5oXgbf6{@a23eil3omg8VE=qF;ep-c zmpAt)kNiF*b~f*7r!nAGf@^4cF$Qy3G}xgp9Vt*r9td70_V?5$znpXwex0I_2thvh zyQDtygZ$jDJmGgjoZIKjiL;(?NqQU)XgB{tM^;!n&!*P=8L`~AE}NbCA*MvLAiv7E zVhxfq#;aGsUz2ENy*1igpIFsH4GDTq`US=a(**Y}3~g>q58Qn%?6~`Q?4}4!aJPl3 zYPUs^dzCTfEOONpilCZzC-Te#x}z7)ia#=OIr?lM3A3vl z)RlrUYVxzvyJjjTwl>-I;uL;r&2@W4jLT&4O1!v);}I-rUS3|xii4|HbIgpAGGuen z?V<^S)_$ll+)*N_Au;#%;SmODL67CZ16oghToZnmR?vO(NAAGN;O2%&aO1?9D=)Uv zc$eCRM#MdHlaoQ@iUhG@mHSQnQ1nfU-_H|XdleEFmvv3Xhq2Wg7NjPH#7%$KD*?MJ z9VyQ=g?E{J%Zf^ni1ILNJ&InzIqh|l`H~9=^wd2k#M%zzcc6^6g=~-6M9!|^FYV0O zR^V(0>%{$o)w4HUg~k0J7R>B=R$Jzj?e7eszk2Oh)76)ZcqPQg)A_y?-LQU9*HNuA zpfgaiyzgXr;B39*-1s_bB@Fj`*Yt{6ktSXzPM%VOO21@4ha+-R;n@CVWS0dbk!+n3 zbn%}Xq`UgvgZZ6$JI%$wREM-6T!ouk@r^z!`x*HVhLaUr5p%Xx)+Ivn2?7Hj44y&o zBB3KZHJj6IaiG(&6MmeNn2sTk!FP9WhF#o$K#H_LBhi=qiY;xP>56Ski4}XO;SodR zUoG~~P#I6J*O1@7*tUlY&8Lo4F0!b^-&IpnOY%FZ)Ya)r4jxpTI7`i=(SOY6W`Tpf zFL{wg&*sQhFk5r;R}O6Trh8TI!dAPg&dg|+QjYFQU=BOFYm zKdXpd?(5e3_&{IZbaDTS-5Dfw_73d{Z)Zo}*Abnf0mv>+j9+ut)GWupclh{G+<%uv zRR*0i-<^Ie{+jPSIXMb(m4lHmLa~4bL*AHf)Qzb7=L4n;_2}X1k9DU`lH&FVe2msQ ze-vZYRaMoqHQ9ar0|T>kDvHSymD5g%`?sSpGKyi=1E?jj$KRe@B0M-auT<_tA?}&G zc_n7ZrbH-$LyOLLw&P05Zq@x5To2^e`%Em3OVMn;8v7lugk#!FN>IBU_H=i8Ihyu* zBN>=QMMb~G(ffWg<2e=qPeDMLwoaoA%!UT>+U{hU5sddzs1`wB{o>}47`m;of+ zjA@(rEUJRhoo3f4Tz)~jvvitvXW8s+;} zm2n+XIMEkM)z{#$-cg0~aou;HN>O9ix)^vDUnV1K)Or-3u3Qn(^r+l?^&S2aUk^Ih ztss?Trpq(ExNn6?@+E~Pb( zviWB3lk>f{gUNa1iP96FX*snz(_A>KbaT4q+_z_|J?DtzVtAM_nWd%tc*#sBL7i}t z3tI@$bWUR23j3Q*#STa1H&mx8zF{cBJNJiJ^9Rp-%`Cw65V{$WNc1jKWwL-l=R=Lv zbc>3WRc7x(U_d~CBOSc;fuEQZC1n3tFIl#fA%p>(_1$uW0#vz1DvcN&xyn@BCK9*a z-T4&^Pa{t1{}d_z+c%|Q7re>-%^)N^oMP?_T;y@on@ly!WJ~_`M>izPeNTMV=ekpl z3{2aEDg^oYH-t`^Q>6K9?eRire9b4F@*NuC&0^Jr?Bfy-Ke4!Z^elR-@7NCH$>J{& zqA*SA_f(NFtDGr6)BE=NMqF?NN_Zc45Y1;<>|N&j1`Vt$+7J9|-oh{-aHq`FQyQ~PhU^(7oQFMMIw`Hn)d>*$m>@_ocu(HAFwwbq< z)*id}Xs`1IXTzf#6nfG!Ijqa*m0`HlHmBR;#Zu)o1yr%Sag@4};ssK_L-O-U%l+dL zAvi(%?^E_p3;m3_$x^|~xaV{nneXaQ)OKasU|7c@Hs`y$&|IzIaCd@mspCF$LPA=^ zr}a`~^rbyLDO6P*95%!?)eOKe#QoFP)_fR`A6HdXEz)$T(>@MgdEgdCP6bh!T)gw1 z52Ij~8WJ-l;0rT5yDoI6Gmr)OAk48|m>$9qUa6!JmLLPKJb9JoTl22>k}vtI%-zUT)uZ<^{ln>;pWl%oCS6-o^(CLXaKVT1;ln=7EMEARYz{o+( zBc5X0UgwF_gkJ3G#3tVeF^Umxd}~Uv_W6J>IoL}a`teqD`;CHpNPay=U>iTiK6wzP zgn2%I32%)SahXc*z7Q1!$?zpFyw@|^O1g95Dk|EgC*dlT>610&yo`i|+VqFGkK?;! zycz{^dE;C9@Z&$B%l~%I{MT=mzkdDtUhfkwX&ISUj>zasTkp{XGj-mKP3v8_qJtRm z6O{fvWH$c>*vVq8|K&@ocJ6EyfFj4m-L*f@t3=CyD|Kvpa*1B|aQ4UMWV9zHo0OGP zu`WYo3#zK&#sZ`_DU_QmZSU`S4+Z}+1qRrTFL3)UV~^#dgr$kse(W+y=qWuQBBOQY zkL^mWga5U=`*q(GA-DBM$(Fa?GB7eq{_<{)FJHcV zlRYLJ+H~&F5fWgI{;SKe={t*{${>|y{?DX{B)a^qq|NHjR+1~Z)9heXYe62 zH*;}OT{o`MZsKay?U6L~9K7(>3XAGEz0%<-N{3yh(siAi!V;Yl-AG`fc7 zz`A*oOGWJmD&7m%jwRCd9D69KrW=Rx8ya)<_+03(^3!WF{_dxoCT}}f=+;?(ymtpJ zPi)6=)2U~%j3&5IpUXF;5*;V(HlKDfCbd#@?z2xX3EkItp`D(Af?s!c3*&n6Y06n| z1YaIR-G8vslcg-Bc3LO8G=LiF53(Y)ruQ(MwL6&21a+3Gnqo}S@+n3XI66$iy z)I!dl4UMcTPd+JPzr$U~hA`DLGhHBqOGas4Tga#_E>7DajReKPB@K0A><8 zQQcyOJFxFtci6&- z(68n}AEj+2Z=OLN;d_n`uC(lWYjg8OT_c`Jib;NNvJS1ER3^SIcc)}zcH+N(=b@_~N?ImktIQvvC@_&HbNJ;m`wJMJ-)3$P@eR03h9tTi z-;Zhk+(X6^`tbMv4;`KWS8_^<*#0iMVr#K0$py`X->@=KQ=@7=q1IFvEIRzra)*Pt zD&BQ{j#%xd?T0i@Fi#sKw7azA3Juci;6>JR*;;Jx$MsaYbqN#Un{`ZU=0{sXe)PZF|sU+iGZ@a@KC;p%!0 z6+?HEuCq;Hmx8@e%Lg0nj4SMm9U={SO|qN{ukZa@32}Y1j>NqPxrL1ShP{>O`*)FJ z&f|Ck_m&8DZ|u~}`h#DQlAMav-K%@omCmM&}Y zY14Dukb8FJr(ZbSnJbWG@SVEA;ye4TdWa~1l8V$135k~p(MO5MPL>+vk|!Fy*rsK7 z z2a2UCe<6=CHtE37i%#=D4A#T=szej&#xyXv5QCCa;S9{e4O6|zc1kny)J)ae<-Icn z14|_5tN09mWS#!_3+Tkx{R#a4ULYTz)qdZj8^c_y`%<*t*X~4$UbEvBzkBy{Mov|p zcir94CjQyamuOI%r#Who@%Vm!F6VxVIh7OHKvQxbQ@s zLAle?nQ{x6x!sp%ZRJrJ9x^3=x#^UXr0MC^{04?C=i6*6jLq(mu*fmCi{h047~0Z` zvi(E|6!F<%3tqb&>tfs3vhgHF6n0A;KG|^~8-d$CptV%<@`R(5FF*IPYs$OByEgQiBr&8@Hvct+>m7kNpO82cUemZ4EI9)?57Yww_Dnb*KIc+xA~bmr>DP-)tz|7+3KT;R&t=UV4$9 zIND!qu2UM4Di`gG{ux$Eb6G_Yi~OJh!3aR@SA!Zx!&8S&{%@x`b>Oh-Kfol0;Ks9a z7r%BbY`x(?Kv|6#7Xz~W{|0adaPh+NyGnku>~R#$>9N-%ZPIjxw$Gk6o!!>Ayn0+dZZGj*ft9#_DXc)PW>R z;BAc;663u(U9GqS#+Arg=2f4&zcmjz&OT6WpBp|8<8u_pekNrMFu4x565A_9yI9 zfguAgG(Vd+;Zu>Geu7Trrti&T1=|~(&C-XP1hKWZX#KT8H32p| zBpqhhJzcg3Ylh%NQSu+d>fET>8QlAhiqCjha5pW8x)@(>v^1j!K$WTX!9#APnw_A zz>mL!k8osChzGAr9Qw1I39jApn&bNQ>yW#d-iU&lCFFL0WA3I34$2tPRhQs$wE9TK zbgLHmeDtfFM*w%FURMf%O}Pxmo&9v^kDG6)M~CaWPj%;1W!eDEKSMWLp!{2O@J#CyBL6c5-!++)&qa)70p&NeGT-MgBb9vk2nb(rS;$+QIM1q~@o091 z2|Nd%w%F*>!-5gLd>*zUY~7orMqhG>E{mUij-GEz;NJawDHXl7owy_V&(!}iVa82$ z;_=qXcjn(jM@K}3Q%wJy?hPeA6>uoN zyEX3;RO$62OF}~N)yH$s0@S?A-G`q_!m;#6)q?3Em%vJ+`rHR9{}!J<9+7YjyUI1H zM2_%>kKfBwcXA`9E)!4#KmL}Tb>YH=LwXSk2mYcLg=rLkUR22Gl{(D43Gg+4G9b6R z{qIThe>m^^zj-r-)h=*4Yk;y0Q$y>~T>kb9!%a)o$@MElJHJ@05ZD-=yST(70Djjk zpaMl)S-~!X6Z@_vgi_aCfc~GJzsw;G<3dIw=(+lBEG^+OZGOnp`H(>Y{1D)3YJS(* zKD;mm?$BUl#-Gk`v1t^R0!=L9;YcQVlk(B<)8{}B&vh3_C6gRFz)TVn(vKg_^*r~s zL|g(%6>n0iG?^x#1m@!_=yix;@DJYTXz1^b)}?fGMBL^(1)oSM*~OA2Cb3Hpj!?~s z!$+RHT0C;(+0e2^@p$OzIvehbf$v#AgoU%|6eCAiSAKFnc{;cBHP)JBB3qmeRt?@~ zq=rRg?f#guA98D4`dJD2VN~W4m8a+kk!*yn*l4*$_ZbPz2j(mfF~dZBq?gIpexksf zLZ7h=+iDT7=~fY!p$nZBG85oVMB&d59?L<8H1(|pCO6^u&PFPyy+g!f!-7t^sU={oAki4zQQ4t~4g0st&b|}7dllqXs+%e6rh5xa`WU71&hE|(Y zC_-xRpqb|IY}dfyle#p9# z{QCdE^d$9>EBxb-8p!G|t*x;lE}zoFtC>iRaKvp+u7OqFs`Yl z`xd?%XZ;vryP`TV@sDJboT|UXav;~>8M7^J2eMM<3ZsJ@p=nNS;n8x-I(&+E8}jc-nIo3@{-vcBW&9E7)l%Z+I9b0QosCKJ96-b*xVezRma!fU-t0kG3S+`&o( za5Z?1G7G0!64u)9=3QR8BMtydhHKr(f~rw)4jYlP%kbB(KOP#d5G%abRx*NK9pvUC zyTIT(3tOI(({{UZuVhG2C8OIfrFJ8TS`0U(K2mvOtU>F=Fnp$`NLE*OSg2rTP}ET4 zAF;umNz&|bNdbp|Cv&3AMU zXI@5BHl)o)uXMJ`w)#Ny%9R57rT&srRm^wJQppG-ZzTxXeo`OtgbEqm68X5XlA+q) z{Yb=10T=8WV`ZFFtRmoNVR0iR8-xzg*@xw%(~ z9;VI&yF`9J@!>1vDgHOH>qE%VXNlLiO{%+gnk@&M=G`~4>iDpg0;F&@@~D5a#C>zQ zH{ZJPHs4UbU03^?^h25f;l)=fg-pLz9s^Tx+>fzU$%x?Je=p^{+)oW$=_*jF!*SfU z^{WHJ@z){p)F05eK5qP~^NXfHpuci!9dAX|4u$%NL+=+U+6%0{e;V(7rYrfLuIlh< z(50mk@vfeX%pwOt;3*pMwz24t-}p#Bs%A`kkj#X;A;_KpBQtBT3nk;1Pc>3_c^s@* z6!p_RvxREiir$5jUe@@>yQ>B->gfa>yNQhKO2fUz1kCz|-wn=`UFmk5%w4g~FK}G! zOAZ0IY(L!_c-uUOfruN28PyZLCZA6$n{(VYB^AKp=2Y(YM-`ZwqPYuNWtp#Yo%=AE z63G-V>R^J4%@@-;H$NW?I84rCuB$Aro$dYsHDq)+uB@fRv3w28~MdB zx~7~Ug#r+#-5j*@QvXI%x@yUwaPkO+m$;4^7vKa@Os}4h^P=Wj6I02My~8M5266}S z_bPC1i2cqTQ-bTPeAoqEgy$XJiH;?OM@VVO>9QZ! zwW-TLBcyVz)8Ub?ipk%##~2$L8>Pw*uRax)oNke9i9G8A`Ro5>cApa9h-~p&eP3Ey z4xj`W%6N8NbW&u(!ScXo@nR}Y-Mos>6z`ty&L#yzjTRKOO`(;1}4%DgsbJ0QIb zqPkA+n8TrT$Z$P5QEO3z)O9MJQ&pvqJT30K(^k!QXgg3aztth!UX444*2dRKiMi=m_448we})B_5LA6faO5H=^E?cGST$KRNNd z@Imom-PZ^H^&Ku>yz}!r`p96OpLw&ub!V~LkLIw{jDo1w;F`nle)eNfntO>$|5lD} z)gf4`nUcfO;$84#mDF>QG%WvD!j*@`9*{R3YEwga8-J-H*9ai0B)S-_b0 z(DgBWAA}q$BWV0`#fnXT;m08xq-GrcyDA_11C7FyFBL(dCsg;}jJh9L{Sj0o?NC_? zGn<~?oUh#)z0jLOE9Q-Xf=o-OU{|YnE8s8 zN&B*85WidnLJGC==^TE7rS3d_DY1-(&Th-rjW8g=W^AMJQ>Fvckh=f=Wf# zt#0*`5Xh}oJ3pqgNOwNRC4`4>AsPxOHb)9Rq283kkR(#07@cBMu~eLqU7lW33y;po zz@R<8%^7V275G{uHeMTTd?005&~xloRW~o{=L2O;v}N}34&9p<<(?6qY<{E+x)(da zI8HPspL{&ZG@d!sI>MfkBXS{BK@z=9P|hgO+9xW-Z>?ws~b;F&%wcg?;NXpF=F4s zZc{RVg3DH2kqTPuNNUt^8FCt}@u{(ZDpzVAvjEyt3O4D9=6@eh%&l#9 zUI983>wo%6xyyzmC6EGQV>^3`ZyeD4}SD!kWN=j zx_cq~Auq?~H9U+KmJKTP6-Hb7?xh@~^Bq{yg<(Ck&bn^~^@It3{<+#I%V|AaDX_|6 z-d{KerP@bF+@Os1f2seGPg5qg15pkf=|j0T#kFw|5wKTc-8QD7x_Z~|l8q~3g%DRj zQzYva_pdr0_?;8S7#4zG?WOI(j!#Ues&1L)VqWzrzIh(LG>ljUC^^bOQv>I+)Li;C zVa&?ac-%ovk|b{RnQ9RP5=6fMRljU)+YSzh8Y!b|4kSQDQZ#Ol07yaszvG-9VxDnO zQ4iXS%F4>23YTCS5da<4ZrgpPBd!xC@ktPPT%q{+a}YHmB_ph!)vos3gDr4@|Awf; zbSq31lu$2>h-+#_qxQ<37UJEeqh8Rso7MUr2PxY+CtEI{Y!j1Ia@hZ8kxJ^#4kI_d@MW zQR0Vks`o8g-(GJFrlk~rO(W7T_Qw)z&e;Jwn-vS5KnemhF& zIRBj)FVilv6N{=*Qd8?mmJW@TPgBc)>cD08lEL+F5@g5e5E0h=_m3{aZl)cHk{}o2 zifMxhFarSu$fkgziIty$U-q|`DlB_B3i<}Y`k>%{Y6OSEZ6q8SpcMCn;gt;9x4^x9 z$zeWw=Nw^ys!A!D;cV@_guHwZc_P$q;co9ib;1wXYg4>X3`Q=Vw0Ce{Sul2a3daeldyjIM!pzm^K`Y#caxPJ|&=k)$V=*)!^~&ofRJ;B zRs^?#!XLn-LSEwy(yJ4s%q${-q zRb%--I42ol{4m0rcVK&I0DZ7rfk+&m%y=x9F$oBifes@P)EtBK#p{OkAg=%sVj89;!2BXz zy>-dZv~vX-yFCB_5Kdh|Ja7=nXWp3%CG_I{Agwv?<{j6$TM_vGDheZ~2ks!3>-u%Y zh$#?6!eXZ+C$s5RYN&R@&Jw|6HssSb(0FXlp)?LOaGZHpRaLof+&E!{Zl%DVktsNa zx4nnsrrfiod~5Z1PN+Pr>fqUkQLu>G=p0VIrj}!~O1E{}^81H8&>ie;%%E3mNpy2c zz|m8C$VY5 zbD+flp^R<$Tq?LsF!>=+Chob|fhHxUtq0T&H6gHMAblx)yP<)|azGRTGOu232RTr3 z06>D^SRw0~vL|%nh+cBUy7TC_krCU+b^~ROAg%!)rcq!bhdZ9#z$a&AwJ66ow7J$u zd*E3Oku`Ejz3^Z3VYp}N(jmy&=b*5b;6I%!M1FLotLJd|e);kReAauEG&o+EK9nqS z41=s7#8+Tznwpw)bNINq9iVss^jtF1(#&!Xadiu=t*tfNE!u0G?|C5NkXX^NvdU(3 zpPyYW+oaZM9>(ef{-p~$v!mO*1Y)np;MGMFY#{_7VF+9$1SQR@2SCJ7PJSYy-J})1 zBPpp08xAg0rfFl7O`f<~^0}<64C+New6zCl6NK0xZ9PXp(T(Wn79N?J2B6|)m^e7( z6~hcu8SCrog}1&dBg*O8=OkXDdVtzqoT7GrIYZWlUz%wc;uth~{Jw;92$4d=+=53d zvK<*lC-0`77Ka4EXsS6J7$@-M*Xm?^$px!)!G*3gHZ}%d7Px^A0lo>+E_ z_V@Ez!n78()a|KusPm5tUyFl(|13?I|SDef6-gp2foX-W&h!H^*jwWzESUtn@ z?PfeD=$1i*d*m5XvF=NL@*>X%hu3>x$8>;u&qm&a`~x1Q``zGB6SnN8Gs)9;iTH zWq{3|aPEO$!z}F7oA#4m02s!LdI>TzHt2JKx^M=x*p~ow=CU5$WySTPaW|djzk@Ok zqGmQ=H1MAgoR4RI1ABy>Kq!6>a4VQ#GkdQ*nB>(~#^;a6@j^=vvZBcY8E4YqMdN@``WjGL1?M+z(@}Juk%kMQHo`vcG5b1$`nL%rLc0qv> zNaoccZrbx-W>wF&0?$1vb1DUMzLe^JBA1(_%}2Dh010)g+>hz5LMA0EK+|OV{evP+ zuqkM#U@i?nvMcqP3h|M$dur8S^N=bWK#43GwhL&j&5HUFyMBF%j|2P}h>`;+IblJU zTR73ND0_#Rl$BYfZEp~vseV*4p6g5jW$nub!@1$AgQQ?l&GFAL=dT`e|JXiBSO?3J zCKs~}0PotXPH;*f>cu4Lu>lf28rtMI5y+RVS=7Ab45G&ViM33v5_ymSmvvM)&aH3t z>PbmSQHt)o(aoXde>}QA)smE$7!neKEDqQNs7b|%6#;h6&LW8!85P$bSFT*)I$I7y zzZhuJ6vEi6hpHkUhkPstQm>TcW9Sp#>i$3^3J^@+-Y|ZJw^@_h0Ga9M#zs9**X6Up zW;KQ_0-97b{w2<5945uOz@$wOG_|9xvMF@oZd5F+yfJN7yhLYek$wyCRm&)=5pyo5PiuiwNyO7 zF;&P1d#HMEI)>A|dVRVrM$c_3{20mk@7X#B^walqKx_(Vh*HP%jddOrmBCIV%DSh; zU!it$7}8(a%Zv~&p2#dKnQ&_F<0+K(PLhH*z5sMWirzdKTqs;>v<6H6`CNF@(yKHK zA;K<=ic`F;xcp{-qV-sN2=YQR_vb6&?Cyv6wbR~rqWVI5>`=w`-w-v4*Iohmyt4av zIQzpdCa3h2M3m9t@g>Wt?79cj|N8P_(KXAtf4R?NYQGy&hn0~m{-vd*AnNf?eh&X< zfzyqhDZiI0K6OZU4VG{~3gD&&EsFe5y%3F#5}kS-75 z6S5udcHy&gEaaLgR>s4m1P`LZ$*CZl{u~T6A+%9}{vbfD@Ox}$qRP7Fpv4HE@k0(f zq7=P?8gCL5Kd1x~5GM%j9hwj6f+!pY+>P8@uCbR)@wt6g2`HQ!bArc9qBqC9 zcSx?)1zUX@aiF{7X1#C&^Zh)Jqd+R_lzRw6kyYP1fT#75Ht41S?gci6Ddc~!lOdN{ z*xDbksbHt0d%qe&-wXTD9O?~27-eA;(9i~w;>ee0nIbek|4AnSixV9Z0%uoZMb-nn zfbzd|gIvD@;xE`G(qaPK1RAzFG(t#l9Y6js0%=|Wv>XHlQ%HRw2JVAWw|aLx01`HE z6+`X|S&-(iO_3!hvXGq|f*lS~0l_e~Bh_MW^v++f@tvOYe7QG zfJ36m6V-EcL7OdRH+}~$EKCDjlO=9jXS3tPy~rEY!{5#sb)S&?E^C4)cHoYl-CKG`>KgAz(fWGzbQ^7Zg)K z8eax|gI;N;g=Mjs&7$lph9KYoit=F$7TMN-C-Ef*Fzv*D8R=Yf1$u)3D~R1+$nkMH zA-uWKE=EEJ^%nr#USZBP;qRtH#QUVr@4oqgd>lIQ;M$Mh@lCu2!DO)~YXE@IK#^sM z{ge!yZ~F|K9FCE2R+3XtYypFC(5nqZ96%7J`)?lrt58#`KhTPodNzHE8`u?qW(Y3< zXYQbP+}#B$-KEup?iUah5n37)`+)W-%&-uHL?K7O+0f(1bRfw(!FKxg@bL!B&*gZk zu=Gc>gO*-B;yO548Hosk8#j=31dt6tTN$G831JH^D(HM76L_}^SnJHpO!~-kWvOf} zfI*jga}v}t+VGMlFM-iCDAELjragOBdGZ95d$qQ;L9Sp9kOPO=fVM8o(w*R8FaT7L zrWvU*Hv0z+mOLPe5)sjfn1cBqgxouYGijVz{bXENm3wcsV_5r~J34W`ILpX~nuq?UYlk~}~N2+y*4Y;UI(4IQ)eMLItJAg;X zT#wV8y3GF#9Pk75-E&5ZKk}d;7zDEL&fMIbIPtSjf5ygwKoB}606+X2vKR-j1UcpN zEOGL^fW`r1P?mk9$D4f{9OHxD0g%Qc$Qu%o;ZnOK)TTdM3n1VMXaoV&Hk@K$(CN>$ z(5|0`k4xf~Dd*@#^PiJ@S`aCN<y)-T*?OCx=QPG!9Z%5WZX9uOSm zoDDQehCxk4}q6m5evE97cJ+m>lwT}t@{P}bLk~5v}kd68|*tEK9oMb(GREvjB zMlcTGAZ)BHA!rXalYoj355MWUsvpyq+F_QM2*Ts@0VzN-iTfLBqVJ`6GM!7Rkt)8n zxl{Q?Z2G4oi60QeBc{t5+xgXiTpViiflP*=kn?5Y3yE=_#yLql{>!)$SLT$|R5-dZ zA1t-I1ihNTEkZ7K0KDdu-fhJ&Aa~%<;ns^t=wTu)E#SHP7`jC`J8wh&X3?9ygt`}D zEU5?`gg7J>d2I%FAx9PiPaTQt)T{u!+1jC&1hnq@1(oEwHv$c^q~+5bfK!90%x2$nrB)(h z3jF;LoMk~e(bCiel0Z$I6mZp8YwJ7>o{Md)IhBT7R2Z_U6-lel0`Y&DqbmwfYu*b!5Tf;WSXmrID{!p|K$F32dLSFZ z1DF6H-@TAYyccU|q;nZ7*-4#B5)0Vul$|+xXz<;;M6bb-FmrOkzH*063qc6+zk+eV z#sSPZMS6o7^8ivJnL$|`U&!4g^X~L+U_mc*&^#0c+H#|zq2XhCIgrx$m8Xppd*Fbo zbdj7gn$MO4A+~^k*qx5Cg@P@J46hRE{y2f{cvn`}WX1!PBVI5L2!;z=%aCh8)`95R z2fzizDGEguLuE2qT1WU| z`H^d7FFuLMs3Hyu4rt(fswAwt8k{W1ZC9YVPFWZ&3*-#<<>amxvG^xSTLBib6P#1g-!e6)Y+cq&P^naB1gU zuQ4(*3JFzw2_*8k0?^4BGJV{HqVJ2w3CKh|N0$L01>T(i?=Ecm^)kc>V1o<#7VXP` zc!8@F!&Si!e(zs^Tk7N)HXt#MmR8KL-d}%!Y#WJaX+Y-Mf~n)qeuG;b!_^PL+yJ@a z0VkKR)*_%&sLg^V1wfNv;e-uofrI;v4w@PriwK*JdypJ}`!bwo^x~5~Sn$26f-KoP z66yp>YcqMN$=@=kunZA{cA*eNI~u1*ewTj z77SyKu-SqPYe3$F1^xWa(ObvPHqrP& zFhCm1K|q1iw+nF4E*X1|DDv3FiwL(5p%?KWsWobZJbVNyTyIDbc{xF|Ks63I=Kn`o z+rJkZIQ{1HDbicm(P?rnWC@CJhGiS}xf_V_9w zx8X$smSRfpSRn%}Mt1N^+5yS}`*pRv%Z4p9wm-IA4@sM9(Lsrl58!XX2N?QmHgG--!EH56lXNlh{}v{Un;F7^Rk}$WJ6V|FXd!kHf_s z$1W?pPPbD(&+Hyf``gw>G~x zAJ6+N+xHbWp_ZmaBT_0E8S&CsNy8_zGx7G|YDP$W+=gY8)#a(%O+IuxGr^3{1j-lQ zajBdociFnOvo;b-RP5`J7Cs2st`!#*&_)-(#_0lXfs;nD$D_CST{N4_d7{BX>NKC2 zdeJha34KQ4bQ#e20Jw2DnA=Lv$cBaxipdKslxDNdMoq&xNt>x!nmTST2lC3T#r@4B zzWwPoM>BuJai{FrC56RPLmVu{!6<$UUJIhFbkxg%rXnapIv<+N_KKzd!=-4fGlNIV z%6+xWKcF(KSg+@fOQrlhC>8PADUrI%=e&p=`=0SEILO+~y_3AX$;;-!_ zj2K{pk-i&rq=GWdLf8h6t7I@D!*wVl0A&E&Ejkh#AWellh6yL|2WZyJfs8tA=Lm;$ z;U+CW6HGz)ger((IM>_Tz+X>v2i8qgxao_Es zz?<`6Q|dBCbk< z$P&^>OCu{GEwOZWcZaAfh=fQ>OD-%d9a19FAkEStARP(_(%iGY_cwFr{&8pSj5_bU z1Lynwp7Z4Ad7jUC9Z_L5vb$N2#o91l8p^0$Hk|#9zae4od)j0A`0Vp99hHlO<{3A; zAI;5<;Qz7HUhJKhUcRK}|FXSw1LZ^p{}XuB+ff_1Wqi~*l;+RuxaIR}LnUWS?9a*y zA0@Gx&5T+ZOjX9I%*#yk^btbxG+C{aRGRR$l;`9O^98q{U1pH!Nv!qa$*k_uMJs=&2k0agI=V~lQPM{~gLVmq*=GynhwPrzmLz+^T9 zD%6X&a{%E1NC+{lm=9nD1jyz~0A?x(Q%e9+HHaDrL& zmNOr?tm=Uoj=7mekByZ(&h~-W*i`13iN?PT(?DtTPNtvaj43G2j!OBXAgC+U5eZPXjawH_@&QR{wIWV z?vt@wdLni7uP?dd2D+1JUnJkJR5|bcef&+d=pw*>IrG^G|HpHSfs5G&NOdemVV>5$r#nS>`gXtv_%_$l$`Qd6!}8CT#9yCm=PH>oRJ~S2iMGway=UAMlT(j6t+Uk$VwHnNU{1k;i zzcfZhG7S_uy&c@Bknm@(`Sk$tRt$y=v-8e-d@jARQRta*!Z@M9m@eVI{Bx}&_@0~+ z6fRv|_btg&a5YspvH#kc>d?5--g%i$mqgZW(d#dwYo-x{F>7)q2}F#z@eu0rGqZ420XDUEvGBc0m6F+k}RP z$ZSsnJ8+#MW^E9dG-z)C045IP22VN7R0pmER57FyEkRyT^LWGU?n({U572JYgrmrX zg0Or*;Rcs+>42WF5QgVgca?ia(M&xCYeNgSk zOcm69xKGoZ`X%1Cx1sr2&xQu&DhF85>XIe-mnFcPS)VfJh|CFP&0hlqM{)HZJv%y^ zlev(;*}i}v#b2?A9c8c|209aROHlj6^HcQ!F8dOIIkVo8H(d)R-I>yNrq8!d)a#Ty z!JlRHMp#9Sv|-%ya!S5#Czg5i1Sf4H=!2D?0UH_?=_?hNsSj6bBwqICv)IjUkX;(0 z^o@QWw}_TFo&lc;5Bu`KE+ukmb!7(%2ZUok?;H|sAcDRHrX!)+9>u3%Zr_ay}B{`+oT1CGPR++aKGFY4^I( zD(u&T4lU^_f|`2YfjKG*|MyV1`9&v(P9Wyd1@(pPUyxBzA*2$l@%)Np6+2ay#>DR|GK)Ltk zf2#)9OgaF;tDJOrZ5{}d2GvvGmMnCM595I`nR2KBz;xH;2(TN#g5YN?&fC6jO;TX` z?r8Wl?$K({6Tk+IgkI!yDBdL}C;uGSdix15Gha6kXaPsKsOTx^kb#JKfF>dwQDrRS z17SBtc6R%(ubMCYh`+J zf9!8J`qH>o3wt4yvJ{78}Hqe01YQ2d97 zfetIw@4X`&O+mAHR=`UQ3c&@3&Ba2yp@+=mD1h#u(I77c_Cw`oF982QJ#THz2G#}) zeFzj=ba_6W0}Mk22cY{M1M&6I_U3^f3?PoLnJ&C5hJ55#?+`wqN(tq1H)>}_LIU)I z`V(1^1*V01K7AhkHBhs*p4D3%s@2xmcZFI%6W<2yrD-2F`C)@^7mZH2?bYHGaQ*sb zwFu}G^qzbueG*cQu|zi(m5_O3tbLpW z-ZkbC*w=BvRmnqSr@1YTyDn1#AJ=E-z56L9vk$+A7o)BmB4-i z3^OR7)7i#`s@|cUGA9J8GJpJd+)i4X6rL;Umcm3L7W2N{gbd!u_j{i69`%Z`5ocGo z&fmO;=M39j*0s%(l}4hw%`AWqE=+M_!qpGh5FhtJ)u>*Ti7An>Y;w4e_-b@>5rT&h z##>y{;dwKINW3TaKD=@FMUvIQ#ja>9Fhv~_F5l`eKYOk#YJ1otl)-SN=S9I|?uL^^ zq1g&(wnEi{x}f??&93ITJne5!dZs>az#+yecDqIE_KddKOOB$=#2n?@M?Iqj!4L`f z(8Q-G3vPdnXe#O|@yk*pc0=qG-1{PGxiVi~Ho_v3ZCkPMlgRS@`lcgE9JUp1i~*rP zNXU=xAA*<{*B+uYE)8N@Fj)lnt%ENMLAe72zUR+Agg`;lg$;k=wDZkYRx!uGtTV%> zd6)31dDPFp0P+VtZ3?C1pkInnLgFQ8Lwfx-ky9Ulo3)@$2!@9NzzJZc)6z~*YMRV? z_ICKYt?gKGa<@fK&>RNTZ>nuaIxa&z>SUgs4h>pnTX`UWYBUmE&gLUF+o?D+9~?Ch z4sWf#&8#k+Vvv?`(KX0r1*6+jdX~o@E&I(?)3B^I_8CK8MyUy;DhUa3R zXTQ)#S&DK#GSp;Ioujf^-Y@m`i60EFN7jQA%iz0_Gj+&rB_Qt{`S8ePWlx)5tvo;X z{b9Os1F>V#dE4+}hFSRgW>o|>3Cwwuod7lsDhhS8 zJK!w*#WDbF07^cfVsff21gLOLJOp|c=IMgQ2SHd1=#)=D1!!?$7FQhs9gA9^@d2Pa z0NVN!Eeob9FXFaS`c^=60=nvhz^w)#&`od7SqBvSz982!H#ZmcSo@7D6o3{M&?CWS zMIN4=Cgh51Uj(e#pn|4TX7ak*0+j!kKva$E1O(AX!hJPMzB?r8E!ot5{9HKQ>V7pQg>^2{f-V$Z$C! zaDG;xvWwHZV3LCe-4h|%AJ}-h-HbXvnCke8gbGGiT2{*<`pQckbdG%fPDih%1ummP zTI`Cb^$j6B`$?qYo3#%|8*3#cw-hB))N7TKFKOtxh#U=T>+}2?*MD@+yiB(vgngIY z{s`mD=Q!_k7|p9HdAXMBc@i#pW-puSx#Z+{-tpG|0he(makYWdeD>ozzJWbxd2Q0V z<(jCa`({ybNn3$yD~Un0y}QnxJ9`2}6HBd*Li^A z2ChrQBz6y*ib%UbW{#UVyiyJ@0EHct16*zrmJU)9yTTJ)Gr%a7HJi=AzMiS)sGzu`;IhY|pe zqpuyz_glIV=X_!EOWXd?nPM=vLCgAursdM^Y_#V?wG~OELbmGLl+?pCYVpr?mpf}M zoHMk?t-mhxj*eTrl+hQbhc%Bm#e!UOZx9HSQkfeZwYTKWKPdm`5hybHCsV2LIa!=j zf1J3nL7Dsa_)L9)VqQy!amt66u7V?zjW1{PFUJGFWgD8K9BtJkoxXoabPo7j+r?Ss zptC&pT3NeT^aMMmq0Kx#MpYG_?Q|@@-IA)8czox|U0A}zm2Kg=UjT&JP(D&IW(XL^ z_uRVR@?5xo+vu|HD`5&$1VASN;B0Xf&PDt--YQV-3Asg%yJ6$H${=#Dxa*cb2qb@k zoc961|L#bh4T5ovGV+iLxo5^)FBMNW!)5MVxLtmGgi!QUn1P(^4X*HHs$QIDMPEgE ze!D*D&0#&L0ZLo*7g2Uee{*tv)nxjbtvr?-_yTj$V>TF5*fHfEwI`ij?kGk$kNW(c zN1S#Nc`m6o{ojhodEMU3VlX^4~wuO&l-QbqB%k2Qse5I2LXXLdXrO z6Stunzojh9iL#F5CCG%sJviHUS@(*l$<7;A({n>2#?Yn-Anpt@Uw8iX{@2}gy>JkB z+V;Nm|Cy+j8*(#8lTYp5+a?#gy}_AGiO6k#xq@5w;1 zua)({zO5&U@_;4PlJ9H_{h9Q9`0{l8!MNai4^AoN0B7%NpZx-A1&Z>>-sgnY@*>UZ z9s%2(Nd>gp1x33fdW&`d{ot|Ifro+CK3w&SDZ`+;7_36T^b!+q`&KzNC5Dy<`_P4~Y zA{r|e4_J*d*?Pk}Q9gm_YzPd=#%ff`h*VW0jaD@u>{dyVEgfI;)N4}78MjlC4UF%$ z_C9eSE>+oE6OgSOvf-svdK5RAJ7gz7b^9)uO*Zg?5Ec9&(M=SWh{mpb@2WsqPYzcG zYBgB6qq$1A4!dYe+1TM~NHkkG?hi{W+u4&dMP&9(7?QCtJloF_+~gvAk18s62qOO+ zl#he-*})|W{T721MD$YydXZ}Hb`#rt6wMHZ)jpVR7Ro=k`iRRUjQi77w+7r7FMHIb zb47Jp|1Qkf64MUF#wBBNSA8n>#p(nnx3v(iS2};8(QI%sH$qTY|A2&bm_vD`N28Jz1nu&741sCD@Ed77-C$sN0wN zLPkTwUNH5UI*_xc$@JWx-u(VNYZN`cP}V@BVr`cwII<((r`c~=55R*lG3+LL*~_pF z0(m$DfeuX)|tXNL=ZP*!aX<*#J7j@>NPfq ze@UNxwXSMLpMyPh?Gq?LJhfil7QvjL&0+HqG=Khe5VuZ-_22@ zGz}J>Cwp5NVhvk|Xjn?l3o?@E0rs@Rk@Y#Twi~^Aus4}_g(ZK8Gq4wyR zLDc}OKhnr2kPFpILv9|&b27(<&ZFE{UZsNF*g(Hs+DO8d&#A>wD2?q*}s&cB(Eo5y(Mn zW0NBZf?>mXU?!s{;_ItD6$AoQx4}G0ogh=^f{5Rv)!Jus25*_i$<~K^K5iWf&C*8q zE%bo)p8YC43>bBgO_hn1DH?Ib;J)E5K0&;91;?u8vX9%|-upDSOa|u$UzX8I@ZklJ}?iaVh48q2+x)%6$x`H z<`jrTX=#yQRICR9s_;OA3TTNCSe^Dby0gj2yW`!+m`34TC8Zl0o1Br?6HZleA7#fF zWmF`vaoZ|bt#o{|N-s*QXvpU3jWskH9Zt_4&e3a>IlgpIFGl;elQh;&QwD*2G7#gw zhGbyY)@E-(JZZ3N(&NG2)0AIabOO5W03l-W)}~B$ax9ltzQ4b^g&Kj^ z%E`x8QGv(tt>1o>GvvTa9VOe;?hsaO!)b+SXXKB4p*gt_yUpcXGP~*FYrs`Ioik)( zflwon(IO41eU^;Py$NGCN^P}wUmI~Z+j5eL2~Z<*){M6p+$8b~vVNYtO$^=ua`jpd zxo%>UQ?%%SmAGGrFB{hN(QJC+gZksXSVmh$<@g~A^O}!J;0(~od`t;j%7zx z+VshGTR>wOH4&cvBq+Il&n{nL@G92se)}}LPPM(u6`~KgN;?;b<(-4Kc16@HHFXas z(rPN#SLy|_?X^cJZxiJ3kTie^xUOg*Q!+86z+YHTKX|KRdGWFfEVJD|xZOWrZo-oS zLUwaIrVhp@(Hmp%VLNHoj?-tfUDoi6@&%CM{HoT1G#S76mf`B2j{4G^F<* znAb?lV4sKCX=xkRrsNKV4c|V9sfsP}OM}u24iriz>~z(nL_KQgO?Dh$-%lQd1Mn?S zrIvj<#s4fVdt_siNw!@&=@>`6ag&bw6O9^4yJ@X;x)A$AiA|-xy(#!)EuC$oNMQUV zjdrD8LWW~xl>Me%hw;QCek#K#AMi(GlK{n(=#*A zros-7%a2N@a+wChhi$(1ZOT*0)2N4hpy9Hv5mzN_AxtO99Eda55QwER7a{eWQ0J5= zmp$N}2V)mc@m^n94(VNYL0KxIDo00Yto5|kych|yhMrsHu$Jh-4F&}BtvnxpcFy`i z#*~%7A+D$ER~0K^!2Cc6=7z+YYD-UoLN#+kMX^L}rva1JyFQnVsyDdffWQ0ts%!uA ztV-(aOTpHcf2uaubfX0{SEctVDU^BC$iSTm=me93xPo#6_&gkz|HxBQ=HPq*v+>UJ zvmtS9<@eR6w_qgE<1Ymb)vzp4UEvXevicl6JeFU`!Rk^bPkn#vpEPPUbbWVEe_ngx z$wtB#y3a=$IYg%D5ryx*85)|Kl9#q`JS0I|j%ak>kw_-6n2U1K~>=@Fl5DvS{dJYZ$w zW5FZO1A)a`sO#9J1oy2ujrIOO=9$7tFpYHR2N)30gKJfK!72~Q&5IG((a$yI%FpCu zkt><{gEov&+)s)rH(dx}4qR`ii`eWaqm}op*(SHUMO{^F`15n82;$_ec~ibyPU?dw zhz>#&#*sO|ENW<1Gwb!+qW|?$cqesz6&Rne6?l|BXTq&RHU6$HJKch7QG@_J&20U% zo;bZ<+~-?{wR@c*agw4MDT4G-aElGMPKA4p+a40yk$9kq(YhxyYQH}9?L}>h_knDa z)@njvIQ=``ZNV5M6wT(}SaW}MM%Y)NPd3l%b|&=SdG=7fz7k_GP^o(-^AC;qW=~hl zENvE&n-c<4@_XhH-)%wE=mi$@~(G^o9GUW~F73zntfk(|^kw{|xrHKafs&)x}oT-U8JMfeV#p+8+ zhi{;wtz}owEH2kw<6jH{#Jez+TkzOFBI(`tEK7`^ksI5Dm!bXM=jjDqaQ^j30nw%3 zEbSySJD7!dJtPeRNi;osMdxaDL%J8LH6t;(+M&t)^!2w>;3CY^+X|3u1i?~8gx!x* zf=qy8DD7K+-m6}*$2*W#y}S}?W(>$Wh5=tj%z{^)n68#`jiI@_uY-!(*vOk(aY#U3 zolk|x5n;5wZ+^M^5q%SJkk-?s*!R7=ddg6xl!Kga^Qju5p?7)L_qJc~%2z@JxTH*v z%&2eTbIsuM_upxvNWNK=u!IlnXjjDm?%_~_yLEW!TK0pW;|9akpRF7gM+7=Hd$bzT15{pxTD`OZ#X;XV>}la#{+o8&y1NQxxv6g?rp;`aIX- z*+({ZebQ)a=@mS6N(&S+MizE`b2!w#@E3t_?7n4>WRjEy(-hQkO#i!V@DO{ikuLGj zs5J~p-`oGkK?YG}6(yPYff5{gvI+6EZ++_?RvC&1mO6-{S#$R;BV!7$jpjF?X&(FW z54?!g<@mbr_=0fKvS(djMR9?`y6WqVgh1djH+Ls$x4kbt^S|C_!pQP&cn(IIWD^jE=k+j{@ef!{g{)5lj>+&JR6c=~mk&|JM#!?XmF2(|Qe;R@A~3<_oonFK~FUXac!O5eHmt#jK$ zn749vT{xrIz_~(?^*DM3i9+%#ue5SJK!d-{xJxD>s2T<2i;~(E})~9bAm$ zCg3tN`NK^|)94h(wwbKu-~je-_ZHp z@TNzox-qvDyZq22cL$HELWRgeWl_XpQcDESfD%AA|Yc5UEF z*SNU?_P(!-^22ugWbT><2W=p*V9GHXCi}}F51LZJ@bjaz`9J>Pw(B|~>yjvUCXk{< z$eKJA`b?wZ?^na#i}nkYF}?|FBazwt%k9I9$uKVrpG~+_i|YPVs*BaBP3d-cUoD+_ zyJPXrCX;-)V9F9_xs{@(LqkXYiYF5$|Ksj+I7A*bVgp99%h?kiT;9>GX-F$f?x$hL z@hBPa1C^zjQAuJD?~O}X-S=V82~|x59L>?| zRa^OSgwj@EN=VVe!=oxsOTc|vHbqDAT|;8k>SMtNW3&XRt*%@e;R5$?!Yc!e_fHaC zEdyHbn!dO=>Wh7b<<$dULt@m0W<4w4_y^7XhZ@9;Sw(l9ZRd3kPQWo1vv5UEuY2}h z#FYcE2aJ)5M2+P>`4}Y&F1XaFmDMaFI61*8R@%!K?XHHrXe==aAg*O+&^JZ_ln9OH z{RIrT0shr%aFqno9FBNgURxr|&;7ZH_HAoc?@+si4njAkD&;I$%AALi3S0HLDO`Ij z&uYTatDCE?k+yd6F1Q}nVsPU-fw_Z+Qhd8vp&@6DzM>)KXn04ov6B3((F;Z2>xa@- z*UUnb`?A`Dq_*xZ1=MXvtYO2YNF`}t{oY-1!|3VNgf!xup*4{VJFx&+V^OL5)TBO| zz#U*08G&0^^Jsn;_dX3s21q$K_rWI158HYI$~+OlNA>cC?hz*|KlAhEs~HLJ?hq5e z{2RkL>}QNhTbXRDWh+^W8^Fjq>DAX)V5*`(LGp}Z^65E&#bXfcn-iQdq>R~<1)#A%E_K#@Sn9(fpChv#u*Xz z+=24qtFP+rESVmhnyw!*k%0 zVz#+Z775_?Dp9^pH^GtNtas&z^Q+_5koxPHTM_{s2jj_&lF_1W?#gOvD;o$Vbi8|Xwk7HHBy;f zRo?qeTWRrz+el_T7r) zfxv#^p}d)PQz3zLA&#^bsEDtnF{2*^$>cU7@E;c}c`(?o{kq2Raq>yUn(H}eG(8Eb z6H~58TRLuAOHG`~uag~#FuyHOM76|)6DW}vRTJ+CLi3lz^h4snt8F*rzCXfQq7&kBd zBO52t0Qo=pxi{JRAv3$cDN~yitr`Qa5gfA`$+vzUD|<2@KyWD!w*0WWVtmv6RE)M% z`(A9j#rt>fco3y*lpZ0g+&(IKq8CCd2R^Nm{tE$<5&-(N|3BByiEkDr3SmVr?y}Nm zj<2{S?y>4cfiMExdNFY#(5Jkq;ODq7IXvgC+wV?kRI2@cQ1$6biA4agp;(d*@{)T46&PdH~5p*ZYuW>F-3^lUsQfDDU90YD}lu2Yl zAeOtf9snWV!97}@K^~qXz2+iJ!cV1K%*qCq0A*ff^nGmh%a|hH3y~#}fQ|%vZ{L}_ z!jiF>0Bh7@mrse+271kc1oAYpRu!&#HJyOG3*~Qs$2&U$T z0-8;8K_X*C$Y*%TO?ZZ@cKrT3vk)j8FqvTTzjhk|uc`o$kq;x3QQ;k{36=-Ezbrm5 z%A>FyGBR?>-xF;VJ<>PSI5%!#wAlDPr9Ayflg545Gy~wLf=@ zasK(A;$qYuBL_bVRD3qL77Ska0|*<`tjj*;m@S%_>+y#*V@et2vj=@O!6g~3oYu>A zyl!s$ZjM~2$JzTg_;H69_vfqv2yH|Q4?bMQoS!9J{U;@o9=>O+FBOZ!14s~R5$T4X zur{3Fq7@VVR3(rP(2zgCmL=BN8p_8?e;BtoJ-##j#hf^Gc6nF&rpSof=uhT0J-CEd=k$eD&WVespJ`uFEp{tMHNt)w1a~nhNwS!m`R(8k9|XgyRHvX zsHn$zYYNCcHo?QO$K%N}qeQ8#3%@_pPdCvaKU7g+hUnIBDOZ8zS+3(;x&3G_Z92;h zU)YqzdqKJ6NPu_%O0E!a_1tLyZdB{Q*HZV;TL9<)z;gkRBU^-B_Yg^MvEXl$N!>s^ z;V;fo4&$^UuZe0F{hL{qu2?IYt`e=tdWSgw2w65 z3%TR#ks`s97Vpti*0x5f#>`UYf(7V7EZWX%bVOE;k`E*CR-YvtTsCMYK#)FWHjH>D zII1!TfWZ#kaO&Zl0(ADo*e_37v1I?fv_32M&uI$Cl1+n;U?->!{@q#iKL_=j(j#Wu zlpBb)Y)-dDxZ^zI%7#iTE?Zzkez5}eGB&mF?4Ix!@%!50ACMmy0ai#QZzB*ZtKoop z^MX|7py&O0Cx*Fya;wI}zNR>yJ4Pf8XzQ5G=)c=W3`lSXW-Vy z^}kHb`=5%)YhQ&C5)S;RgTJAC`W3V{-#*2ovI0O)>&;3ctEBd|A+t7<-1K%1Q91%b z2Et^2Pfx??y#$HYPW$M2$?W=udbXklB`aAg?D2Cpgtdii8sL8tng5%%e;UPXiBRs{ z%FP2tYJEDbYZ*e5~#gmrK_dGeQm~;?JmB)OJWp*-Mc<@o3>g~i|a&c&< zD@~`X3KKa+8pCSsW&nRXZ7{w*riFe5J7rU1fk4ES8pXBh9-Y;D+a_=byr2o_($=#N zQEyGARcar5u@iM{LT?qqqR8c=mKahHS>BNmpYq&0c;9_mf+6ao?88H)2mhWxyomeS zH%^sX(eHl2BCEMh4{d0ss<_Q~$d@qH z3HLr-g+6Vl(GKC)=gU3}7 zM^W6txl`4B{XruPf$fTy>ITQXiPot89HV1jT|GsS#i$+2zJ9j{(oVsETxlx6RFCPI zN0!Ul_a*#P?|=4fk>KK^4GI42BXm0#Cr+5&?@t(*xV#T0_sPfN<ifkDpB4_t+13=tbox^epl_ z^L&KEDG{E}6F~j)Ro)@vH*#g(g34E__T2`dflaF8ee)Y42-|eoN`=kZy#k$NB_kAbpL&~*Db;l-_7G= zD-w>H5>r-|UCe%7J(ZA5x1qT{`^a{@(dst+mKFu>6gqw3;2E7a;1KAWtkqIaMpEB5>EuW?xzPP-vv4g_G4D|jxH2J#ioBic)pu>ZIlcp^S-!0+s+?>fheDM@Q z*zx5pRg8Ddrhq*ZR@?aF->byAgI95S2}L#~Znr(!wIwmxNUS5iu`3>nMBnvcpDok% zwr%;hY0nl`?D}2JGvp{!Yp7e)70J;vd?~#$cQ!_YAr*Q*AJ@7z%x!r7&uf%@{`!Lh z56NA^^y3>PaoBf7i9Ry%Y>hkgBwW?0kByP+j*zTdM7XV6q9FqU(+Ota(`S;)ivM$T(_kVK>b+nSkt0$=ulR?8XTebV$g+XOgXJ-~$5}?^ zLkw>Jw1gFy>&oX+B(c>-SDx00{)kh`O=c6u`?}q7`GO2Dj82)pG#TH1_UAXMy~*qw z%l~#=qw6|>bqse$$KMxxHy>kQNR+no>h!6T#_-z;yr&SJyxruyspU$Kym!wY-v8_6 z_3uHy>nKAjpHrCAn2Z!m?}wVHD_fhTuyPe^M$k^ZBY-hVMe}xS@T|^=HsD8b*4$V&XkNchD`a>fS)!^?x7#J`yPA zqMCwjrRkLqx+-|FOskdO?Z%TA=%7y{Z7Gh}w~vOY^w;@Cwq>IKQzle0_zn4m8!yXd zVq}-P8SlkLTbm_5_Itrh%-Ci>YJ(!yqMDn`E7R5g6ZYS!^Z@&oMw0RCF$SX^U1)2{ z&#w^Ch~H7)Pvrd+Lf1xnFm>?5VL;zlugS-0$o#)uUS9=nq5|J^qRB@KKL)Eun!v^` z?!0#RJ(%KdX1P|AmoMhxoA8t)hoAI+pM4*EwhEafq_MIHaZh_mS0DwEQ;3+NvhU*y z$hZOzqqB-1TXLyOH_+K^^tlUbTr(BM{o=lo3s4AGx;HGpV|zC_{-Cs<NP*Nblj0BNxKt}wL8xMy z^&cO~1qi*AegAYIlWDi<({yeVfsWHKpW_!gKh|WcxS*bCA&V zq;$i=BP+!eg(5v{a6m8atXSdidLa^9d*02D*%&LFZl{pC<9}u71 z$j~OO*Du0~=aq*0ngSZ`aNwz~^VmSH08Z!jp{dx~Df%MEnyK5=?-c)gHU|=S?sBY# zB9&Z8NsszMb<_1siZ0q%1O{%Yjxe6gR$rHf*fU0v_cIM-W~IyKjOhLD`cD8taRNY0 zVXhT{UpBOFZ9}o5@4wzra#um|VAO&Dc{dx#&0?kEgV_7A_GAs5-5|LF~1A8_g( zPa#-+Wy4ZmHJgF-$;&hQWfSyM+xVAudQ#7jWu0$~D$j>j?UTk`?}fTntx5fNti?Dx zrwoPj|6#&w$*X3{+-;F2LIsE|Eomy+RtpC%T#=H7L|hvy@WrHz7oPSy)(=Z(sZtpJ z`SI_SU2)Px3dwpHY6QivZu{b5>gCSfRGm6EF`Q8L)vk|R{Ht6jcSmG}EXK=}v#u;9 z&3IS;b{nK)BHisj?PbKtl_ec$vwK=51F623oAB_3b2Wo(VOnkg@BKI>chmT*%%0^} z%#bXW@;?RcPleesbVMO*?#!%?&(o=o&Xo9bzz#J(pB^*=|?K zKlvNpcRqR0mpoI^q2pLAp5>_xg8|25ROxCfE>cN@InWWmN?{6z!Xb5B{r_YqjSdJL ziWCVokK_?-!?SnglaiHkFo`;;3}2b~nehA;LbIaym63tZE4-)D0kT)4krMCV)c(9y zpQoi24QCt&^_~Ylq_0K)=SSpmLF)z#SzIlTq7<;l>~AU*t&h=8WX6lOnpwX%?9_W+ z+4c1mRF?72xtI7_&9?;Njj{7;Crw32%SUKrxjy1Q_tgemqc0V({IFNh6#!(DReYoH z$Hz9AU>+n~bt_cNQYh(FSV=&$^)U^0(0Vo4Jm+o;3vCGyzj8_&aqS+#RA=nnnddH* zJ5TJZwMOn31pjB&RIY$ML%kR-cGvy0#}^RqN}=w0q`pK^dQiV?Fl8Nsz#}1#KAI>Pc5D9AR+SsL^`I~$bP;6O+#M8{ z5IIhU9%dG+&+Dyu6YL?Sc=1!u9dj%6^z)+BWo>haxMdY>69g6Jbwgf3O(P+p_zZ7g zvSCo^odA5ZH!Sc-HzMKdWk$gV6asd+Y77U+F|+S19@WwRq)#{mh+l>q1O|acz!y(n zHW4Zgx(sdqIFhP<%Mt9x;MJgq2pQ=tHzaw&6*aHSyw{?Ir6i1tvN%BSnE z&5xK@rUR`KIj+x_p8DcTOTh5&D?woKW=L+>{|*)16@XfTyK{sW@F+}vy)5InnXr+4y{KDDw8d$-P#E-#MjKgr`Uw{d^mh zBHFg~pPy?5`kI`R%l{^IGVGVryY;7*g$p5z33n4#5=gIx-y~G=2%#GOS19{m;AFI!!=Rql8-#8Xyehv)D?Ct3i{>hhG{U zVZYQm$~sxhC-%cS3p!n(KCW$gQ8t!!a|LfB>)dgHJL)4ah#D&Pj#XrqYAd$MmX?Y1 zMknYhU$#fj4D&;i!f$=b8|ry(Tg|JEm-Fk)Q2w_H6BEPrDgql@{I?YL-{@drFTz-} zN&F@D0>_=cr1&;8@hMd_%Nle%2WRVy`m~D^*q~*Wy=h5Up2Sad2g2Fd8m@$UE%Dk z31hMfn5r0cR>y>4J?gy3u@jYBp4Wx6^(a{dWIw1YLGtnMzv1qnZOu^4aj9u~qD-$) zxJwzi#3KJjDE!UWP72dV^3z$Irqd2rp58(2M*{EItt>cnD(dE}cXfI!C{bNy@oQp?$=%IaTO!w@_>4ftykd?-C0oir4VPxlI{3kHs%FlpAW_{_>5 z`Axh-4#$+7+fDFIk@!wRbA~!xKJ(Bzt{i{jxS`7&i=YQ zV=5n;^YvUIugMycRU~AG+{NFIgF)%>@t#6weMzAmVRsj=OEHIwU=EYLPL(R_R(wg! zf9c*KQXtQxiR@{*@Nf>Qua!*CGZR@-pk}J(8JqC#o2MbeGbz4)*q^yi4u6&t9}0ov z(ZQjwj)l{YGl3Fc=?F8SA;{G--?nOOWK8t4s*~f%A_7(&?2Af-_1}jotE`FEx(RuR zjFn1Ovb<8ZIF&XJ#*RP%$_)p$k>o@gozvfcZGW9bU-9O0dq@Pgxng3evX@-E$CkRb zpi(`e#eoj%%w|-&_ZS@_zc>18uxzZ~YNSG`(jn3EzR4}T2>xx9tz`PP0Hu zvXWD`T$zOeC?I#g8Cbwo;xiI0gLo$?h(x}I;)S}-ge~b&jK$_2j-rHEmmL5(RC2yd zViJ5Ye7d`eB5Hr3;Gt{@iDGbw$?yKU#u{N!NV1xGrCksec2td@Mv zXn^Sg3$33-a)L~W57UTzojY98c`D56z0R=QUJ%p(Yp{;P%x@T z(yn^$>D1ru62E=0znwm`bKxPfWcvtMaC=z1x~#qV>+6RsZ|+x(DdbsKRYP^NvQ@BIpBgsBPjt$Lim}_94#7r{=}nde6^y-a@2o z$t>zhgg$;pHxW!u&DmE|Mfq_;bSOgQ1SxQ*HvyhHdzqzX@89nJ|?4C?@b)%0At)m1A~tCcxbO)YB-6 zO+r0ZQP-9;`UxjZ+Z0+MuQ=Rnc)oe%xR1`JwB!qZ3%gz^_lmf{>(}l)86*v->)6eP zRQ)Agws3WKSl*}t=f$>3RbK@f{nICQeu_?BNx$#Mo=o5Zjh*1t|I~;I7b8XPE4ap} z$W;zOR8IIGgwN`Go#uDvqrcCKmj>LUz`W8*(Z<6|LosHu7O~}^U4wKskD^p&Y)(wu z0xm}%bPII27`Ym$pmNr$j7dZ->l&JaEJf;ccjL7|B6lRAr-;deUdQ{o#MdGhZ-a;a zYCJbM?jaeq=uV1Wm zXxDhpv4l$@4ntVjGWu?1W$37D>B$$ixp;^|>P;u~4FgRB&lx&AcpgQT-)WABlZGT) zEgMPtZ*I?G(swP`tda}Zy6ZHVO=7mCimNS|az{zEQ>T|?qpDL{mKA%+2S!qGSu+3~ z@lg(Ti$eQu(UsjonJRy5qQ4WI9SW3rB2f2}`&35J--30_i|_7FysBjFvyy#x(~c+o zqwEtKPrImmawK!3Uv#yW0#9K!Tj1}+^lLj?;A#J!Hf(4#{L^j1Z^?UT)rLthc=eJabHEv|-O~vcC#`dqknY}{!f9a|re1Uv;|HUl) z-f*Z_L$IzUkD*Lr{Ul*G9pg!sU3`j+BuDIwjr9F_h$P{VD?g;0p&e7YdIk-`jZHi= zQ_631vRS0t^WvHPd{>{2O=bUp7i2@J^>eU!7J(SoV|t}n3iAMUV|)N^qB23Kv2KFF zDzGvYtv^@kJR2hoJmsvb)!lK_^}ne_O4~+-sT*s&SIN_7t#j+Tl2loVP`JHOJYA%! z=B%oD|8`qCaoYcbnU# z_)li(R!-4O6Zor0_vC{oY)@}*nToe3TM~sG%DFQOnm-a2>UAA_<=swN*VZ`WyX=$J z+~14)JPt1u;McO6S4z|KN=~v=sycNCC;O@1Hh7~h_{8|Buh-#1;$$>_Ge9+?zo?Qm zitERxispB>&>eP`5e+B5aEz%_OYVSMgRy1c zgDZu-dYxJa;SRvFwi?@piK(Rs?0aL%kvoc18LR7uorD)VuJ%EV+^ zo_R+HeNo8HTnUY)V3Z-^Q+`z|Qb*8L)-3-@=%@9g{4#7JurL?rf$nO^2_MzC36WM)wzjhGkStc6*)*~IIma^jYctUPy>vP6jtdSxg(~R) zX0a95k1elX&GzjOIZX;RuPkE#M2lXF8iwJ~0;_|`$eU~w5@(#W8!AB0XJ}$N0EZKS_rx$lw_FJS@`=|b$8w(;YasiP-!~NBH!z^N^%)RP_uhq zYzcw*SBZOlrJhi8wCCveN^Y`u`;**~QI>3=HxY`O_xzgkJYw)g@#xgTC}S~795VC< za(!I$ISMcNTA{midSd)1hwJLIHiOK(M7eM-U^xQg4|~?@(3_7nKuE&Dwpiewfxyj5 zS2*#}B%~l8E+P0OP;I??dXEIP`_sQxEzq&#p#-&K?|(IrWorW(q%Xa-06=4&=MaW@ z2|)`bNR#zo*Hb{gGlf!osEjNFWr6vBvvvLfhleQlkzC_JG^#K=6>B~11Q=>)IQ@-p`R$Xm$Gv_rBnYh%`a7fNM`sr$N|X#dm|=S>jX+A8 zuNKJ{2MTDoggRW58V*Ni7OqT98_YHe4TUlR z4{JzXghsbD@B1wriIo14Ai7_tWnsAdH~c#@>{HNpv~_tLRo8>eu^Yp~pZfFWUo?qq zWtR}qFfLS1EsJq&Q%*6WSF2-6%llNbI)KHO;$raK!0z=Sq`M_kIA>a#)zbcSCRVv$Ka@3MU2sn{A{iI1J$5DY`w2CEZ(~Q#Zv4u_nhndhn7gh0s*_GtW4D( zey2l*4Q$kJYBsk20-dwI@c#0sqWMU@;dkr&uLrO9_L7Y=mx>e?B-KC=IMzDGI!kiE z)&@$OVRtsuvobpko~6Y)Grv?#N)BS*Is51EZ^@J*B>@ga5zx7pFMvSS374uWgrwf< z8Qq3i{_c2XjPlXG3RNhpbWzBdn3y`9Ih`7}Q%u}?%O9Az?!O>CJ?UtP&E^BgtbZwz zbe0?cIes)0c(B;Cjq#E#%QP~nl36xYifV1&hJ(Jprwz+4;FoT!jc?f8JGE5pV=Ntk zxzUCI#t^VwLZw7xtHfh-XTMp8{t}T+VI3AN=Cn21wyiyr!%*>c%O=HkuZ#b9aG1fR z7xIEva6no=d%db|G$GoD5|+za_uqM{F;QkpepcA+;dey)^K19fxKNh)tBKLr1&1qs zh$RCkZ6mjqWwFk9&$C@hOBKd={#@oecQ6_4dF0ORVRY;Oev=YxN~##WcOoHJ_(r8A?mO z-z?;wY}=)qU9RD;c^*TI&o0U(g$H_4-@r%$);d;A0@8C4zTAsk?L}VF1_EJ_4_OCy zM}Lmtv?iMZP!rs}34t~=T6o}je zu(p~iKY{s*M#!ynNo2&0KH8JhD|kuU$(v4x0cwKE6Ld4_rdOsXIiJ)?PE0*;(5||? z*#E~~JRR4UA_v3>M+v9W~b}D23fn0;1WFR$YZN(e^lD7CB`PRTwNS2?hJQT z8)<)c0>}@^xADM*)}yl<6X@w=C#aG@#jn$_F&ZO60WO^mqMnsWTXgYZ9gW|Duk1R! zVMf+yA%5)=^sb42F$e>0z~N4SzS_h}A`73&&DU|hKKgonZDaGd0v!lIV|=K!RifH> z7I(7s{iSO+q%52K6qv0=+o7-sE}uL9sMfn2Q}yt10S$JoumU!|#FZM8Ivp7XbEy!rjFf@#~}vFQj_@6`5tr--Gf~Q}KojmQMIy8+n7mFC|}? z{E)@-(Ak@NS8ccBI>V3DfJ10N3Mf)L;tg5jD^}c0YRV$g+`4~VPOL;TQLD$qA)zoHht{*%m@ z6k>BQvv8|?a(FdUPA=jphPw&YTyN}_^uWFfzqmvEc(&(QO>wdAekRSr$D{Oph9qQE zrwThG0F>MILVUijSMwfxx}?OtNoh9jf^T=f0KnN4@R;fEeb5|2re4&EdToR0)EIO@ zQ8n_@U9e&z%eh~O1>1)*_F)qNT*cM6p@OyZ~}u`Bdq_T6*io%#VOch zXM8_P%Pw11O&ND%^KpY(f*r+_*A6!wK$Yp_S}^73dLth!8VkWbaRBrJ>1}}Wr0O05 zVL9hle#if)=a^*iR7oMyS%}+uDgn<%yNkReiboIlGkx}H_T9jl@b&IpXn3hJ<#(6q z^{Z6jmcAz!iokXw%#Ou9{VL2S5JUb_|HSp@bMXdUG|4eTb*mxfsN8SzrC0ICWXs*0 z6lxoY^1_-c%-h&5=TG!_O<+LNnTdHHNNGj!GY!H7+2gzR1BxT`R<8bXc_e%a^55_I4>Ji`~9}%{=+S;{4Uy74v0uBRN!kBgJ&Nr&34p z?0xs0R0q@BWU~Kj+#gH^o_ta8Mhgi)`w*FR)Y?zr%rQqAKX!CoP+i*4bLE@!EwNGQ+1q(@ITzkZuDA6FGes0=2G%-j5v?nj1huG^| z_dS7ObchEIwj1|hMsl)k*WHg0uTL)U&b#N)fMI+JTjtPzU8y>MYW&#gGG zKE{cL9)dL9w>%zpE2)0lt`rC*+kkD#F1NEZkiVW9iXZ2dS&1_!Uh5OsEtwAwh@${G ziUNqKOqm2+r@@}}0K@YiW_2<2d?0bd_g~M;zC03D8LL?m;4fsx@U(;Z-ux&S&DM)~ z;zu(&9(4U-=Op4e?H!+O=iS7t`4d-zL^!xGApzkmnqW9q;AH}3=#Q3!7gdTZ4D%PV zvS$OGTibe#T{Owq++{k<(s7{v;j+rYr_mN`FVyn($tg-xMGpC^jY|mB3{A@$HyA<5 zh)tYqF3v3g<5y~VvY)g0juayuqXCg5W*1QKJVu_j`^N}t+4 z{aSaFM|g%lnB6_3`dsawf?qum{^3v~PqgE!t_W}H2YlWj*vf$o1k+vbuHn@$ukPKBk2I)0Gb4VmLg#U6I8}P zUJlCblSNsvpZ(7J^c6n{HdfpH_rd0t&v8!x6|w=mt~&36O-#&S+U=A7H(K`0LcUU_ zr`y?3X#?(U^0wr5T-W!L5`r44uh_%wkswX|hlES2Gxx&|@(%vIaG2c_*US6SRCLUy2AHny-euUfq}l1BVcW^7!Ua|1;z67k(VC6tGfn2REUhi$8OO2s>{Uv(4 zLc~y%+hO}KK0=9fFI%Ayjbb)!7i2VTSW?_t&HvNdvCOJa@n4og8dhV3u*$^Z<*WTWI~>}@Ah=87RPsayD1G5 zuZ_K*Z?na}*9=Rzxd^%FMHBSffby}N^05BZB*dy*qGF?~Rd5V=KK6f1eo&+gl5IktRCOm|6!UIHhY+B9c zf%NVX)m>4WUaoKZ*5njtIxw-n|ngSn0%-o~Z0wsv29f}v}Afz+|-n0&U$P{L0?%tv!*ui^U*buQ*)XONBYP>()4Shb+ z(44>VcAQ-WD>5rDPYdnx2~O+#I-Uq9^|o&UGyiV9ejhHEq334R!TH0;{bqE!N|5%w zxuXDxce1LM>OtRr46_-Ecr>$Q8o4@Nwdmpt3=xY8ElP%a%?YPFLPGLlR6N6E0%82| zr^MACl`BcC+Er!8*EM000Gq2w20)oqLj=OySgB74&*gxo0C1F6D4qK149-e(@ks6` zApKl0mVF=WV7~iP7#gDy=^@itD9hBwNK6S2guULUb$fC>_p0jFGX@xStaWuGN4)Pc z@2@`vUuzZP;YoI<1^MO)tf7QVa{ez02Lq@k-X0Rh)H?gOC7u14M1LGkZOepeJe30 zfX6+IQPw0a(qmb)I#v)!k|-= z@f5bK|1d+g=2jpzLn@&;@^_3XJP+3A~0jx{Y4 zjBsj-9GR6;YO5nx@71l7tf$+lV_#b3%N(4(%BEjiTv?ScFD{})ywBzRIn4?gd%hkb zUdYzrDK{Sb)au(W199>CsFxiFpENbF@5u%+Xq%sTJ5#pH2;;UEi?%^o+~}qVuph94 z=?>K#o6cTx9p^jg7wHY)akXm*M-CNtfxq-YQvhT?xr29%FXh*)?nVk;6O~tpSFfygdhGh zsV|d%QwjM3$`4tz%ET|UtHM{od5Syz`ZQRWtq6EVU~Mw6IN`w2Q#9kk$fvy?U#Ou` z+u(z4IB|)X9=h@S1klCUu2>t@zp^V+?R}Sn`a~c)+Ez;@F+Py^dP`1&rFc|{`j4#) z7Eo4VSTquzA|!pz9wiN7`Hc;(p){IpAs8$w%!iZ)$6(+^k%jv9DpFbsF4}(-8534h z@}@^ey#Gi~&0EdsBRZJtK~YWV+Cr^D*vnlu5DF_f29(2LOxDL!%{~?c{j;cf+xL~kBW>`n+>oH=otk-0RjVoQC-=JMH(j49YCu43!WTf?D|t0# zElAwS@j0o9HP;?9k|`^HU+D&>n;(K7i2k6mMVrF#JlSBd&fbqOuwHz$e@sDyB@Gx zJo>CRv>@TOYcV6J_s5F_BjnpZmKE-!Vp%cclSitqx35QQncj*A0KE4_w@=w;y3|3i zVCo!+600v8!oeTW`*;$iZNl$Dh*FO*U(6jA^PEtT9uKM1iK*%J&&YqL&48ASDbJz~ z9T;n;m}PaQlK02_V|8tV;x@GLj)WTc>*~kJ9jYvJCUc4Ld2fFSuB;jXZ!*|kZz7b8 zSU*lkl<9;5)T*%)|1dx|anPo`&-?GPjA;Sr!+!~ymMiOjPfJ;%QgfeT%vs|{83mjs zeA5U%Uo3XLx+{OfDM0jLthQN~VlD;K6Tz(qr7qrNg)jmDp~@szvKn$0zjNo`;n_CN z*S*!z?>akrF&$(7>zP8mY})Q_jxL=(tM0?!j}|1S%V^Xu&+hj%SrDB=?0 z78D!`)+OK7b9?d1PPqSyT1cO@$ zqdw-5OqX%FhC?5i7cab#gkfmZ6WRw4s_=>PRQ&V9Ml1VOMI}3uY+6(xBnYNy6=t}Tl<3g08U6e$2A6LV%q?vEv9}d(TS)ZY@?x!E-aStJ& z;8^|p*Qhhe0GexnRpLgs9B)4`cC?JV!Tvb5Xo$#w(}!CFbOq?yZBei1%L_|yUIH}V zh47jx&ol4*=izUy=MJ!vY+(%e^)kj_5+Tj#E*{b^_H zRsA9S>ZS<(otrcerw2ks$9X7p@9i-+PMi|5%v@!IvkoTuvcHwcc7I^Lm;}8%u&q2D zcxt#l-gxl14z1=eFEd|NJ(DiX$4quolIfqfPU(P&^l1NWSOq@0z6Y3R1)w%J)HUWw zD{yh1qwvpo$Ep><6Tt#^zaaZH*2inwIcj95X7`1upq*$zxQgwFEmlfPIg2R!7!{|* zPRqh{1N@OdWJ?t^lv^uy;doZ1umFD{rqR_=GPj_xpFGitEG`a?p`#FI6^Bt{z8@q2 zt0O^*zo$ywysugj|AxYLPLl1P0qgFX8#TX%+%?NooIj2&31EB13MxMgmXtj(G3LXu z+ERkbNkTXwv0{dcooI7<)G8o=_i-UY#k?(J8`R}=fL1l8VO@8x+fW1il^{t$^5|D2 z8#3T9q;5yoHyP|2AH+p~wsv6KziGHoIk0xri3Z!aJ{4)WOV`mC?PRF2 zd-qS~Uv2K~p1|UfHKy0Jqo=|2tCl7h7GVlRdJK&F^&TX@J+j6MdKYqOaq3G__s&|1gbJu%GgBq)!mY) zwg|{sXmM+(u1E|xjT&$Jui=LY)>gAoGJ5EWUzIx9yAk zn+vUPNL<^MZb6&p#*C>Qb_o_bI3*@m7@2`Dz&zgomJ0HFpf|l1UL?}%ax?#~un=lz z^M~wBFgk$;1KVELJzRf^{&b5f=mk`>ah=Y+&q}#7^Gw12PyBc0=A&fZwb$gg_iXs~ zJC7oys+%86BXjFZlwcU~ zo2`1;(U3olF`kgLZEoH#)ISfWqADkWiU|eXo1R}eX|GQM4j*EQrwKOCZ>q2_c=jSw zGZK$uRT}6C+_g2eLRTF1O>Z>1@wP-*zp$5e8mZ4tFm?MagBRj*I+=O{0Rp7H?VB~Q zrJENA&UN&sj$&f>U!|7=y{Vg9f50iD#nI6t*q6bjf72{%X2qgpXDPhaY>C1C)tpWh z2H={1)O)_eHnL+wlqIi1VQfEkXr_Y4B zS6^IrxZy-iA$!!V@n3)hpc_s2C{INVxmVa3U!uySM7y;6Op*#H z#}&`L8}F#ZqnyiyJE^KZa@l~@F+ffDn<`+z;PI^o0hP0k$sf~W!cp0P z%cGNO^`rlC(9qtsxd z-iAzJb1NS#EP+xEEdvzZ+Ipx^J6t}m;#F}GAdi6s1EKKy&c?gc66Pgf>hp;mB+iqp zN}Gd}_=%!_&puyWGIHZkYgwu+%-36#iUh?+%OcqqU225#7k-YraI;}qYkWJAcTWJi1%olx#5Lxj%Exn8ycmN>-AIegBGr^h z24$EG5j=a>TUfxg+wek|rnyJIuwch#)~m+w1Ro{AW!dgpOI_5(^Bs47r+qWWDT%tu zyBvDU^F#@QUqu_dpwDW`RT{d`41;HCv|$68`yR3=Gvb*MitJ7 zzGqw(W+Dfwfq&;bRvztUE4;c|h_W(1;=HYMK^%~h!>(y4IK@dPe|yOt9zY%P*Poe8 z)6Do27lXvlja$(x+v5{E15#Q9Xs}`R35)Fsxf+#4d59!r&m#^f4~E)yV7o4ZeNDW; z8DIFXT7o`-=d=M}|!0x3n)a!ej*N@~BG!)^^LMYd{2w*%+@$hk_u!>^o|pSMl8 z{QGNrNIwdv1h+0XOxbL#-y5CLQGKFI1ge6$V@y>ia0O=%TAHD9_O_JY*t~mu3SBDQ zTD4G5a&oO^>AVIlk!@D{_EL802RQn^Ijb#x{{GKjXTaA$)UgIox&HJMKEmOzIyZo4 zD^|?%u)Jc&*D)jR)YL2HH}ZYxsMj{w5Lc7#LUj&$flNGCK&v- zB#c|QXmbh-_Ek>b^92L%woi{wQX%dO9#B%X9Rp;CT+g$>6g>VlyX>X&#RTw!^paSo(KMhxDN+As!Yau`O-chc#d<-Z^|G5r$bqbmjA=6Oblc;6ho$XwYz4`-`}1 zp5zXzH9~OOF@e;s7DN@z1v45XXyyq0J+!O#q}3)I)hG?lzZ`V@NZ{A(Vy|#`vK^-iZO|(Pwk9Thh<5nqVS}>xdFjLO}t@L<8A=nPjjG z_!K`s)BUOdK%PAfG}{)c<(!DO0O2Umu5rBq+O_w?#-*BspRH%F8$Vk!{ri5MB`(b~ zyVS4n-;Dm1lMIo8PN!1rs8`^pb8~Xg_p!@y28h6g=|g}uD|-yuzYgsH7Jy6lP}lL( zl}-I5ENjs{SZlDI;nt(C-1NOMcYJ%dgC&-DJl*mw`}0p)IIBFZgkAO2jzWFSP`Pu) zilT_pYV#EqMhs*-xH%?U;NO^jX=z#eeLpo+@VVHeY&!k9%K&j$sE3-2xTm}>t4;qD ztHFXAIi9!l{PVbBm_aFQ-7>Ek{FJ#2-_CvZ&Cv3jl6VNu`^q@Rq9j_7#F)`z){dtf zt>;r|cs`Z=fI;IkV@8`fV~H#&h55hESXXzkV&EKl(7e->0`_~_pt8{EzE!-O^94ufE!w(diX`E}cmjUVvO zjvJ$IAegT=BV92NaiXK9wD{hYgv~FE?kVtuy1SI-_^q2_4)JI-$8tQDpf#mn^HtV!qFBta>Z(g zAqF@&3;*-i%amb}8;xkxYCU!1$;VSFpz1nG8)Xj^n*L&v(4in+UBUgr@Ru1Xx?iW1 zK%Zy(YF?+HF6_4O@6w!{auP8hLU}qT;eH41_JSsDPVNGqnnuKyQ5K*_^R@i`+)Ab3 zV-ddSz9m+;pS@|E%)@4er)JkQzHi%m<6Tuf9c%?{S)7`F2Z%)pK2o(&DkMHS-M0E! z>+Y*`El<(Jb3!mEaG`YHJKk7)nezw}h5IU~pOQ{qpYZC~}*DCEKVl zK`9#jN6-Iea87we3|(KyvUltkM9cIfQq zWu1bOJ<(xDn)h!jskY|t+h8jU1>NH!9RdGhy5E0?j4_VIufoAKx&nd9iS~a%R!DoW z+ke6RWVcAb9&L^|h&6tgC^rTbO#E55m=-5NCM= zgMomk-Yf%EYpj7T{o8E4{39vbuE`o9#>QLEp5L!*781n>2#FCsJF5#b z0Xi%}T;_U3Ov8k$?6I$k`aNFU=@9Cqrfk>2DXE``xSun6qAK&+@~#>^4wI1nl0oFl z>z5$VzutV)_=w3zYK2MVWHGrKui=0|T?ttr4P1t*g2eDiXzf3r`~OZG>3S){pBaAUQ7wE~#$F*4SS2_{MzLxU;`1%;xyOfX$8Bh1#lt9H+5 zuEb@_ATx>5Oqkz18v;2tKkZ9i4~U`r9gdEyuEwm7Xy1NwE0ZE+-!0xSAzjCIC3}oT zEdsioLZ+jzSPGl+PkbKF4sBb=U|mKPyj{ou-F2EtoRlI$pW>OJ~Aku5LTITPyF zLDlQ=gv2jD-cIpjr1TLsBb0mmlTOos>jT(sIp@W!$9lSrnG0}{3))r}GFZ&V9`0x9 z$>sQ0yvh;s5DWF*DIP&`3g(IpQ87xYw@PS%Kh28QyAkB-DIxR*&N=5(1N*j}(s?Yg z1euw93B0Fucp^XKX;rNGeD}bb&CE3cVrPCI)sjsF65lX}h_;|9tE*i|Uti${cT}9c z-3PtIOLV286K+$o`Wvlg4e*Jn$-g&*jEa9V8A8N#3R|3}y#j{n2enSf9 z3Qk@<*Fz#*Bd=RY%5JC#wfdUcfX1Qvx=Nk1!wWsM@aa z0bXgzH-BS&5UMv^OA?&bnb!d2G4dS0SL7Ai4szPrjBXi2Z&PRJ?Vj?}Km9$;2`!mJ z3Kx!0br%GH0g)+@O-h16JwAGJUi|s4cph&b52J3b+dS&?P`E(R=OJjO7NpEP6L@EsQ3KO))lVuxzg}pg4xH?KT9(~Zh*l=gOSJPC(q(0tR7zJUFJ2O zrxz{o2`3;Jdt8n0qgH3w?(%4WZ8a7twqK#tO~A1B_!TxOqCnvgmV7|5&Wrt%(ST-T zVA_^uI`kG!JU>ug|990n+f6&|Rhx}L&968IMlDnwxrM6h`eh~Y)vkNDwW$I2Rvs9g z18okzg4+_eD`0jEF$FQc6^)_iy$8O5X}|C0v^d*1{pH96uW9^X$VD3cC?ejA`5xnkdmP4`&Wl>M6;L2?EheJu#LjmG|dx|zYFH0#Et;Y=KEg#ZTR|~{GQ^j)2U?QY#Pwn?#{0A z-I0Ubh>(Co2GRn=yZyE&-b@R7J1kGcr&qsuk$Im-<*y3G!2mzi4$XksZ7C6HH0@7ma}gbD`c}=-r=#VI=pImyYO4tWmDy^$^V` zv*93X%wK>1d7yMlXFjJ%=E~WviEPHK^UvXhaW7kHyYia<`btri5dOa8wr!9duzf%9 z^KO|m4?v(L4E|tUQ7dRHl z{+@~Q?H95h7i^?IlT!hk5D5=)Pk4pff5IlOQ? znYu!YFNq6FHWd0aa`$zNibll4b|s8N&KuIDtw)|13J6wN+)-{obFhWRxXPtjQMHT4 zn9Ku8d8kDd!(xk#g$B`7Q!^z%XPIs!HRunj575HZ}((4bV<+pklcT$^!H_`$9`{WfuBxhY$et zjf}JB$pxI`ITA%r{N6t~6mS-w4h(9+?v1S0Tq(mC`LVFK!uWq~i)tipBGf@SD zE^B}bM@>vkRIXF6*15@Q0X;9;3V88t<|a@{g;-BLRoqc>#w&lhH#k^D5`s}L*zwiu zZ)r`|)zxrdtX=smE#At19{#~HF&+*e>$#-v{wTjU`wU9zp!E@@uft65Fg6p;qgFnp zM@zz!Krbhy`{Gaezd+}p6mi}vwPm_#&@kU-sE0f* z$`;sIqaB~VDelhz^4{0<`ld9pGDuc6C`P#LfBrU%Huu3**GfAC1Gxf{#1u4^=2KJk zTEj8Hagj-(fcE{@Q{=%RnVRgQp7t4b<3%H1*X4rTe7n#$hAVVfLyj=lX*cWR#s_3$ z*{c_)01eT=GH)u)uDUN^jn+*i({T;j+q|JLLt#ls<11&2S>t}2wR`!N%GQeup*;_> z{?wFKobmwOckT%A?O6ZyX8@rHmkH#32=|*t$+hZ`Et19M#`~!B#F@0_*_$eB>Qo;Y zD)-9e^9Dn*6R4kup2OD$U#vd#zvm73SUfY=`=~bds*nKR;F{>|u@Lv!wRol=2{X6h zTy?QPk)I7+O&eJUq*tkZ3BLhGSOf!tYY6Z)u9sPEs2)5hXuy6&(J955nB1@R&w;MJ z>>tghq$2*@X5q&Q1}{39+wf2yaoP4pWs3nFFz{;b7gcZGofVJT93ka6%i|?so#g%>%fPv~FIsuEIN{gifV>ybo*?+xy(4MiYJz+IHLZZ# zX&ZbF%3gb>a`RR;VkliEZ%%41d*VxEz+UH1?MALF-#=6O!C=F%56Xwfo$fg~4!nm{QrV zFkG5;n+gQpj@{k2QoX+We(DW9?@ep$4Lh*;+;|K0iOKv6%=3w-PW*mZ|E4GygACMc zI2(`Q`Ih?*f&(vMiG$r{ARcNc4F2$U{7w5wJ#Ty2x4f^dAw^%-m?JrB@~WIH=W1C; z2URh?|LmRbOJ zTMNG@b_LWtK7GTqky`SNI#cR?g|BagvP zS0sb<^=IWGcs^M|s=??$#lW2E1RF!FhKrdRZot;d} zzpEqW_b8*jN144;`E0=E@Y&a&^#;$JS`w@xnOQyLp zsMH!jynEq6akJgoPL0RuGhm${{nI>854#wYyKU(;1jbcn=gj_3qSjnO!~nw zy>r3g4&J(uQ{^Bd=GJJm#c)~uU{k=$lsvQno1sSzv-G6@to-T1-Fo@X)T6GfV{#N% z@NBxqEtiXArD(H;7H!EJQ23Xhh1-)0eDlL`$&cj#vz{6p_DP_KVnMwXMBJA*_HOpS zYK={#yjNONx}3q;LEIg5G%`|}pAcF4O9`@`5)Ve-EagSms_vL!f8Q^3J8Hqo!M9{4 z`?mcB8L?ME-;v5`t}g4x@DCSqjo|6;=7E?;cnVsbRTjj6k!kGYNoQw&Vir(E@~~tu zxQa&`JK?AQYB-o&1|%aX2A=vBHOzJL)>z{)0XvLLjU1pzR<3{jQi0<^rKb>Z|K!V^ z*A!js(SBtLhe|QRhgL8v@{vLgJRS^yT-+2!^Be=faSctboA+TlM*Kf59F@F}7imgZJtY5F!R$Vo;2ywz+|M z$D4Jjgkb*D;GdVdFs>Y=oBuaIV4-N;ch~33gidDD2=d9h{?bt{e(7Gu{IwObyu7`< zo4(iH&|)H^0uARq4dDZQZ>WL7+{~jcO$!U0X`#s^Y@A zTs3UWk__Hc;6WM^SPBi$r~MCg>i_zc#wk&62x`I&O`rB_x`nO8!cOYHg;(i1CDzDf75!wlf42dkZI{T46PQ6U0Lf})k4FIvS9k&M+9c3rR!CiAFq z3pY48TGFt)vLJUP{FUEc0SwbtU|x*3zQ&6wp#f9Gyum$pRT6E_HQNDA+u&)mHWtNH+@WSe5xDnNv?Fi~JgNIagTbx(PklOJb@5X1w;GZ-d>pWr&zZRMYb zg}uKEHjND`=y!hg#QuYLrRy*q1Q`Vn$p-clwK7}@K^Y^7kR(E94~$KKcAW$`Lf!+y zdW4v?7@P7j)o-4yx}Mt+Ec{>_P!J&?eFu|rrQ>JHW#z0YU@=r-9j;pKm}P?D3Jhtx zL?Ijt1`H2^Y^1KfzAQHKubrBf5mf$Jq^h!afg6T-s;fNU(A$Qw#sX|A97IvxJb`9+ z#3#hUHw+OK8%2fZyrjW@yrd;$%tSwrS7k}uM})Zp$wQz}GiS^lf&vUB2GZ+w(;Nlh z#P@JG(pGM1YOKKCot`|Cz?uYS$j#_OggFz-80BwI{dRpu%sq%4M|7gZy#?+!5YnUi7ZkWuLDypcYi~3>z@FK{%KQ{3+lp--es%+f+Ko#(dDx8GYSFnTVO`Z ze-)I`+HCb}KWt#z|N7#kZ~e~-e+eM^f^q*dD&eQMa&JXwyc1-Rh z)vW&tI*O}@*V{qJQ-};_16}H{bht`d_M!>n0HH!`7?HlP6|4Xg3-kj9RaG>C)mdd(p zMJM_8l7?Ofh1;4zL_N$f(}?Yx&Xe5pz`>^ zs4ln(-79b%Y2&dLt3BU(w^Gx@6gnv`w;q3N&!l&j(c^K~vCsy?BTj-y7U?wY%wXI) zmk}T4FyNUhD@Jrc2l86HoduraZFvbJisBt$+`KdG3WnK^>=$Fqx|nnGqEE(6hE$9HVp;XhZECY^N~(^pkL@`P*rD$rnArARYbWIr z`HS_Q*QxL4R9p*tU@E+L52l-4FzSpCH#j-9SZ6g|CCz;E5@rftRy~`GIg4}id1bpz z%BI`OJcxb$`ZL>k;%|EApHlJLowp)v>wQXR_+DD`iQ-=*blr2FBpDq_FF%% zgGe$1^Cfz8W?d&ge9;fXU@J74+ubuSoe$5g(jU?54#+vcb2r#1^pYKV@hT=l;u6u) zK~eFH^5aUg&W}EO`TNf1r0KCkM2EC(!YD9KfV|RH9V$1Tu`Mhte#xUbjsx=oT{!7d zgkN=sN`Qif#*D>YrQf=RJAM>+=nDb`9*7^S3cQk)yGDk55xw8P*^&~=Q*MwUhNtUU zz8L|eljdK*d`G>Wc{$Lc#eM{{n|kx+4P*&z1NE$%Bm;YjD!-~fo@c~O#su(7tADo< zX{Ng_mc{LVk&5)3hP}TgE&98w?rkpBP40jdud~MKQ_<1W-O(*eQs-=We zi$HSOnQ7Bva$rx{zYpuDnd^t_XG_}aQyQMXP5b0IPDTO`^rv^}0y=AsBg+m>0ypkV zH*PH)^V{|^pMIY{zR)yp(KJ7Dz>3wy|M=1gRwH;!Uw|A`tis zU7nV(_u`RC0V6wb?&b;mXrMITn~hIE1|CxpCI}bEs~)Gzs-P1mTv+yb9KkVg1bs;o zw9brQCQKl&KB5){a2Kdk)u3)M!U^HhI4}iy1B8PUX0k64s8y+}+3brb-6Z@4^EvZ6 zGmZp;3kf1HU(Fex@C4y0uAfv#?dTJx;;s%{Wc#ef>unY(a83JHH&OfbhL+C2KwtF} ziLN`qWqae9)Msy>d;9RU(w)4q+%AtTfQZ-PKXUis@WzV#aoj@h7`RoiSk)O00`{);|Z5&n}3JV+DB? zgF0_97DdfK4!a0lPB=LF28b?3DQFYvAEZkIZVkc?TrE0XsH3muf;Lb$LH=vofra}Q z*WVMqWSvzk%~c*;e{bKvKI%lf$jg6e4C%AID+Ytsh&iq+ zUiG1m9rgoVku}FZjYSEen00}E0x)7u1OlEI-__U6nhS8kq-U_7SfL1)vS%0B_`IMM zg0u!X=<_}<7LAz;I*$w5!$2;tzo(HD>5@1B$e-P2Nm=QZOQ*(*){A+2Gxh&Gqm}tR zLcuz|Lu^ND@5G@J|MlWdCmlcnZeU~-7;C~|0Z$a)GL=2yI!x;mp6hoz!9uA!DBO5k zfJ$o_+-P~p2MD_O@0iX$I?fZKryv3aZJ`KRezLNU?}fL?f^tUoq=&T`&$-5@TTBS@ zXKRl$(I>!XQzKx`=_)3>@iGK`Ar)A&et~Sj4JpR-xMbZ2%ojO`HQLU@_evr-IwPj} zH)?}2)0cEPrv?!5GdHo>T^#6FAk|0khcbuAz2%(wn8>url6@;WtZ{GkL>T!Zz45C* z>soeo`D`#3HPmTO?KAn^*X(x{fVT2M06J)~6Ev>ey(ewv@}%$S&*gcdwHRV5hUELEW#t-~ z6e1iQO7)hkBxnsDdBGM-8xLB0o0#qG=do3&HL-t?+^Vd8&gfw`tAP%eBW407E@){v zI}0sBcVG6ni0N2vy+121?|jdXA?Ee#lVTvAiOm8`l9}K@NZ;wF*{Z7hh?8CSl^bp5 zU%rP5*?3wEA4%~XQ!+ff1rV~H;NjwW57>lLU4s`7P1Hh4AH7y>LXBGUzQ+pPh#=$4pY&ak24OnJr$ew%uiLby5`rgI7_HoTLF-<)dEdqhb_lkOaAb_P z28SA-iJ}1maUM4SSwvkJqKruo3HS*@cF-w1;Q*d2i=9HBN)cKp2rRAMS5*|)KO`51 z=nF%Ih@eA(AooD0paTkA845cQv=EV1cpU+UjH{DpF8*%=dZ6O6vk$7Wbf*|w`)_Dfs#=erU zwx(3;Ss8gB>JMaGe26cu)|>09FUp*|a{dRDKyVnQ)&bnV0e}2ch$olVsyO+& z7F~v-Srdi6A^YDN*SE>FbY~s*ds(esOoZgcahwQy;lZ&&oB>H8+nL zpPsT$#^AtTvF_%)zsHgQ$FGl2?OUr`mkE5`s`o%>4Z-*HavL#+pK=DEYO`!@hvz(i z2Sm@H@qMVn8HN~aGU})!2&Zv151cLZ?l2Jg-UspsUEXLiQuf)6a9ZPEu(xPj?wbNI zsOI40CrfCG?@+306|GTN@A?}U3+_CIn)Ijq_fgwztAW6K99cj{7dDf4{Ep=~q!WuR z;weIoZ*Mm-o8!(=+A?vJ3R>E>G|gi3)9LWU;gYx+p$l`uZZ`IkaIk;1i@Kb6y}X!u zA{?~icQxvCeuDBC3b}5vQ37E;4H+<$Pj2f`?@FVi`uZkB!vdgO(K#M`D-t0XLp8k< zpNN^qT0otI2K?fs+5wQEUDf?ir3kaX| zMavA4%agJayHAsGxJ&Y2Lq3V~1G#W#SUCHHwMf&M-_^=l;6S#)u^eCs-aZCG*^a#E zi1@F zkmXh3XbLzlaZrV8*C;}YrIqdni@m30vR@-@a#Funeedf-x$E3hI&)2dGUAPH_b5zH ze}ZTl_}l+yOj8AiZ?w8eeiL0cd!gB(mbfQw-S>7F}jT_N2OB>ji>ZM$Zx~G5@A2vJH#^Ibx+=W1V|dGg;n%TSshcD!cDE0)s-= z33%DHOoA>Q=8cuEpa9)Yp_$gH*N!#%naxT?J{82C9a%X z{97sSE>&d>gzpY(eqhpoFxbBm?RH&XV*=d!QviF$78)YnbR4N|r-r^#@my91G{q#0 zjt6OOo2hI&plTi%U+K7C`Al!IMiWU-TN;5CK<}tM_zlk0_-xR1_%MKEpu+!MVg-A^ zY(n%9ORUt~o|{#FVEIe{zp^hWt9QEqvBIF<ma*Pq3CdZT(eBmY95 zMW4_^U=H!Kk86lvEQ96;Ohak$GggsMVudG)h18HOq71H3Z`duDQG{5(ho1lv8PqT* z8v>TxTD|4G9tPUgLX4?_Z&+x`<**XlbOLM}+g61Xl_zwE0pku|qnN${yHn8tRI8q( zA%*22Ioq_44mI;$VrwwF`m1|1pn;yZKwaP{#Cnz2NYXB8z^8~d*2-bg5#tfgV$ClQ z^!TS`e1qsCW9GXh1D1X?lUAm=wnU#nw@vd#ng*>6>_v=qwlXQ&#)XRcX790D-?gs! zl5D)uNoo6HX?M=u%`d}(UhdNV(`|Ahd(q{Su{GC$n{iWd`~k+<=HiUT6;Isl%gjzGC|nL5uQsmlyhpv~sL+U1O&O~T(VBwxxz z+A0rEPaQtOBx+5+X(8w40TFrY8IzI*--^a&(TdVrB_|p0xf>bi0=ecM|4LXho#%9) zXAxyrAhkMIhk{C|$wy`Gs#_Wk_MgEZa{ufwB7MU`Lus^RMbc~?fUW^n?(iD;JH!$S zE3>w#GBJ{C%$e$TGz30GdNHeTx|LmSmEur`L6V30yIkewzbYNsKhVmjfLZXA(IVJL8n?>(Jnu4r~Ab3e~z z%OH_%M6iZk?MG79)`Ne(+e;VyDgrwCg<|xF2!<%8r0WfB2R1aHJ+#tK^+d~$rmJpD zQ>E8Ew^!leqtbD`iVY$rCQ{bW2;V2sjq`fmi|_?pRPk7msD2a^AeMO^SXy3gR3y6e zm!|6;HxIu`|9kE`m+p;V0KdX&{&SsUx4%Q!3Nq$sY%8Lb%LUvXK$KZa)wb$MY@QN3 zMp`SWl`fsu;J*UI>zknYV>fgi2Jw`Z3=tI^nn~zbbM8pKV&_eO?Do9Wbx2!G2px z@)Jf-j!=$~@${5WBQS;Dvt=WBJ4T43ecMx@1QAd<)G%sq6q)o6(|^W^01VGxs}*!q)qSH{yX9%q0b;?V_PmD`D5>NTgPPMu#6Erj-vALQr3kK&JW=^F zhB+ihDp~O_{bdRr$|dm@P_fAyy59y+bX(>XRng?~M%i>&=_xzFWZ?B3ETJGXHA>E4 zV@wEkcsPod-~X4EaoqBnvMMb3)S@;wSf9xMW;qsU8b;J~n53W86ty^t zP6~gy$)kZ*wvGV9MQnU`Igs^B9@;iuVeZ;4I(f${H0fO_=Yi_mV{YrRBlab6T?`3_ zYwmS2nR_;Tm9hbbr-E!LrFz45BIERZCrajxMi8o}MNQcz{;ok9&sEx*i%Z&xvvrLf z_D)9dfpusND_!oe;ey>=0FzBDtW=koKNFkm+YLnmCP-N=Adc_!IO6))?yOwB+z>wt z0b}JsUs&GQp0>JG%uGZg7(Rz7F&J9b@kRskd?s&CldwL!tf2sqfa3dlMZb#ssq5cB zZdtb$yGo#@ysQ{^6k-zHCV1yd>C{f*s+*sm_Gv3mD7N!Pi!i@yUgC=$dbjwvS1%&~ zqI$|`ZgW<1&3ZQ*o6Qb5d_5^Ci~I|H=Srvsl$!$~kU`^@zu-TTJbP63TdT0|dW8vC&(Gh59uODL zDEnsS&)w1Q6Z~eKwF$w#W^Hc)npHRJ7k=FXurMG6Iu>Bs#|RXX!k76?paG09Ce`luQD%IsI^P8ad6;@x7|8_k*KR3eJG>d``bo9aYp$hhd;TWAZ}nR+&Q^fJURJzzCot_Kl;2L6|$c(WdKYKp?hE( z5(Z+E4iaL4i{G_ItQm^WG&q;kG)?R1JDB=o=kEiCM?(lOJO$1Eb)%s?G(ME}2)~h^ z$#LqENfkeO-lz5z>rIyUl2XcjzfDPIo+e_A_?gS8p;tN;lF4uX;^+Fy(AbI~jAqJy zIa_j(RDA&5?Od-HzYJx`tAm5hWJ{Enn?Q;dq8?t+m zocmijLQLRy-ET^!NS}n`=7PDmpEVI#lWyBBBBC~qz%a5TqP*dmO4;8uxBC_`M?{-V0y-9R+-$tpRHL~w zd%MjLAxiyN#~hJa<5=2x!z9k^NYYphkdcX4@ZSp;Y7ZSvs~jTpwOj0YG^wjTF3?#u z{S$&U0}=|exjoyO`a7U^Dh`efqN+~!ZcmVeqzhN(!79i9)kO9Y%n((tF$({TZ+fpk zd+`OptB_i{3?n#tcl&ESFa7x>Y6927d^8CKx3BqA!vpECc15b7&r`gORYH^7wE9xZ znRAa?o9S*gXC}ycUOlHHU(js0PXiE}$pMN0K+{KFp>%&li+#g0eie@0f(BSf$og4c z3^HGmWx}PqD55@d*d2$2Rwdw3yqZ#$SnnLRe$W!9hOw9%Vs zyOK>wldrr;Ca#x%J%>wC^N+5c<)du)M?9ZuhXO?>5m>3!E6*u( z8Ay*55f7HL<0EeF-@f>VET6wUcK2JCB1T#*c4wyIV=lWkuI?CiD$ikHoTsrc&1m z(f^|jD9#MB+7FZf1T>m3!wnc26vhk0^4ZI%)XPzU^gWFaucOF>KH|jEt#Qf#a0#|m z4?_=>`4fDBAya+bO++z3Y+t3t-50h+G29}Ep2^-jvAV=h%{MKQf^yJ4WDB^iz^0Xh z`6#n>a!Rjx%6;4c+6O?U?D^UNPlBnvE4465byaYH9UQ^uVK?VE{9BRqr9aF@o%=)} za0Ljp6B!fOxq@O6wDx zB?kaL*>>+EBm2oxglMU3o`t5^!=zS~5=D!A5vU$ValFzUzI|NBGaFbMnj)9_ z!5y+T7+Vk60`&eLT`sle?ZU4z$abSxf2yZBS~V_a%k&X;poEdYSBu^95uh7c+HvNj zrV3N8bg~onYP?)5ys#yeL=`Q7D>wgz0(S}Qor*O_C5_5ApWkw%RIP%RbixXcSfjxd z>`J5@qwFb^55Qac3<_W+7xvg~%@1+8l5o+`1MWWo?f}FKwxp*ZiUwH3IytG??2c+4c|0O7qdN8%6Yz|FRXJI~*iDJGw%DxnaoQ*(kfe!F2nd(7U2>vTjh!eoYFk?yjw6-Id zrvQ_4t&@gaMt^5k3&TwoWTh!uZ-R#(8p}Qc+=V`TsE`{sG1NpSLqdL`Jtov)SqT( z^tkF#Q}w^^$NcFL*YHUsPu2d&@2&SS6!?O=GQY>^vv*Fp0bdh$Eq3)N zj7&7po`q<%#9eeRo-;NVgIL;qf+>qG_O7HpLsN8 zE7u8^1q=(YSCexEU^TvpQyfu2PMts?)Y1eVn+(AMXosSaix|}I458B)8*F@Dn_YF< zfBVn2b_uSH87Jk`bo!BeCkpAm;zbCCxF6Q8-*2d)IRAIC1}9+82-#P7)CJnB`URmQ zj_?%pIsPc<@+o0E=p1r2XFcdtC6#^AX?(g%IN877df=T^4@7s@O?aO*XVzyNo+FDV zE#r=oXa5C z$q51CcgD!1n>0Z}$WM1#g(gXQY1~i0Sz2n($zcA)p9)KH~m;0Co13#+S|#2XnG1ZLudk%K28lAqNydQifb>>T(VzEq&o=${ z$sl6llUXz9G;-c9FX@EGC2@&r?;asKLg1#7|JkVMws24tuVmbjhnpz8>sDoXwDv)n zhzOLz(%Uk5JBjC8TLdDajrZ$hNtlC~UlNt1X`9o(3A>Tc;**}cE~r69b-T>>Nq1bH zslUCKD=*KkLe8(kafjDc{~a5-_6n;&81G-MP^jKbkUg&qd7aK>^%1F;;0fOn>(ZU2 zL@mW1l3rHc&ZH9stjB_$_I5KyqlHvmM4+$xO_xhQ!~z!ejSkZ$jBx8c!xusLMzmql ztxg@ggLW%fJTe=@PR%s_yY$m>Hob8CP5sFi^o~ zf4_g>;m%)yzTv2iS5RiXm8@WU3-k^|^VyYKHB=aC7D3iZ(@DcgOhs<`RlLN4(a_lb zR^pxGKZ7<}wtXrH-9j-QkFIh)1*3jS1!q>c%2(y*|AtND@4ip8d)LfnrkH_IxYG%L zr^3Yn14GFWiBRC5B))?_%Mk&|StAma3`ip>- z*D3sJ+6XNqF>U-J#5Uf0c9u1sMy~7n;?0O-{K-gVw&lyuW}teXsop5)AE-0yLqIvI zJMn=EIXs)7;{sPw{aHvuZ_R1JN-Q5C_sR4XxLmLt>`LlMRya9sr8O6BG4|P3C;uZ^ zdSdCL@Q>&UK~DV_!B5ozLsg|!?;WZ$4lkk&+!qVV(ajLzQBcjd4K)n(zKt<B}d$wEpoU+`fkd{R=Y5UH?gYp2cIIAljwn7C3xjP>qk}?rm!|pOX z{4^aP+Idp2!K~!nhIhG(OO&K633(1W*=UiSR{iC885sB#Nb9j3=K-y4;!7wq==}TY zWb8en$+~7itc|9C(Gd3AfImIxpkDy*I)MR!_OazCuccbTdAoUbZIv+kILxUw1>gm7 zBPZ|5mcstmMR4@>JYCdAt<_u&_z?l63dbH*;k#{g|j;;WxvX-yzEbpC_B2 zIq}kaa7|RiSH-lUFUovW^}a`ak^ z9!l<(j{kX~pXghp=e2hXy{l3v@09Y|M>-l8D-7t~=65^E5IyJz2=&gTdtZX*|;J+cQUOc}*wzeoU|IchHTL zcb`q<6Y1XtT7n^E^EQ)OjDIARsK#vT^=!iA&EId+g(s>!=u&=cES(E`pdd%e+j|=n zEjyN$LKjdyWJ?5!=c>d0bj1COy6QmThm6k$Aoz1DrU0C999t-}?^sdOVs^vj@Z$Y8 z!Dz!S;@ivjMgCh7KAirxH+&e=0blI&mEKORzbRcrdk$1fvS*=lK@rB20+I!~`FUyPtm>2wYn!TbDf} zpO=DR}x;+}v1kBk@5RT=)VfVr9IN6zfPY@{4u4=J!puyVJcdaWBYxQD$ zg30grN1Xowk{S4IJ|K&+LlItE&Q+Mw)hCnP?PO$AS^E1%%WrOMz7slaXxesSU;5+t zt0Mh~+MC+Um`5IM{2%%P0{be7)3}Fvzgo@0^P--v(^Cl}3N3YtZY4@_O26SA5${{h zMa^#`?&%mGG^y3=PWRd<>Gs!@nER^dO*$WDgXKb1JE~|Tw`s?1+_4)|r)m|VbH@q- z#o=E6GK6|21(p4DtB3am?k#ihYyUPFIEvwoQ%Q4a%8VJ3v%9CH3aHbV=x+BvcC94br_qDS12flK|Y+8Si z=Wgt?S+u3!cn;^avJp_Es}-Fns$XE*;L^|_5;)jPX?_Q-GlF$3?+jJ73^^k82P0g5 z!iM!6rhfOwB&bl_k>uUH5O)!D^y{|=H{6cbu4e)*oZf6(N1g04;|EC~m;V)9t@<^! zc{G*-q(mj#0r;9mKlj#&z-6(Z(;pg)L3t}n7eY6&I;X+EBf^_vdS}PKK+I8DJb0@0}sLs5Ot7)^! ztU+FLICOb{jMfGLAAi8b5;mO1Lq?gamocZ4<+4?Aa;j$chf?e2M9r>Y^}XM)@_)T- ze0AISmg&!@#QVLATY}p(yjI1t9>gdF+k`yMHJtEv@En2{B4hJtgE|7~1LhW*b-$kmziA?S^SG`4Y zS@B7uydA5jGHhPl=(_J@Lr)QU@HG(klyJ6>-$UcZj6GkB^Nda3q=4W&(8qk_erGXn zC%qOHG_I1fFL#YER?S~$2ZQGcgKg*o2ed3|ojw4|0Kw!Uue#9h3boLLc}M{EunWh6 zYY_17g*h|KMynEcHLcWDj_1{ecYc7%(Ron#nb87>_8 zP$?V<1G^fy{M$#bw!$2rfzf&G-NtnjH%~VE4_^$c)9+YGZ*}Mvh!aMSdfe=MP+Yg+ z)X+Os@FHqQW}wj&L8d@_c~8vxvcgMUU=~89c0Q&xHr-iAl+Mg zlU{%%==^KYUtNe^_Idl$`GX!3vDKe<&3FEhhLrUJ;gA*&gI!T$e{E$2^i;z$=`DqJ z-t(wPcdN4~BbG#%3bZ9G-at7%*ns;0fcxsv7OWFwx=%H0-MONt^SpkWWKxN08=Wm z$mP}FJsEi2ZPwENWU5=f61JwP9F-~Z$C(kzlM`(6=d;4icU2u|#$G3vF+Or)Xxp;hr%)vnQ80LZ*<4hn2g;Wx@o<|`UoRMOsar^2L65v_ety)M|V zXpf_%!>YXi#B=}^Vxn&E!~2(r!~0El!cFPCUI3B8oV;D1xPWf$C|W1-MV65iUd_tI zXL2*Y%|&dt+8Pm-q|p84J%UK#9a~@3&J+BUO3|ZH`(DmZB#j!0(2PTAzHqxCXO6wB zEee{ihrVgAJN~A7>cfWRY`Cd;#9K*iNkxBr_-NJI^JycDzwEzQy)l+h69xB?W1OI@ zP(;?wv4&E&YZ^tb>#TY`p8_`sl<*wBL*03oCCr8YIAfo8^jV3;R8my}Cf$^-lhKC6 zbF>IAw5oNV|`D`QN#FRJ(S>c=Age!3;FWnOYgSBahzn>f*vpi?b@r!%T=8|PjmOG#N`R>h@C!ru~opcHwVN>{{^TqR; zvS&TdyQ5aQWE0k2Uk<&`Y`GARQBhjm(n({f?g~Yt1<*HfY9df)AZv-4n z?^>bYmdVXfa3il?^b6U*)56#N%rQM8{K7x?GNcN%bG}d<kq%A!Z-&41_WP#Sp<+MrZS$wY03M$F zt;T6`Hn9Kh=7*yu96u!zZrICLnMANula}%Iy72b0(Ub|Y(gC{1oph$+TU|inxuD?8 zExw!~to5A#;bgqSUe&ul$%=7&jQaG&NVOe?h-2365*4}|IfAplx{vebzW3zsWYU*p zPi|x~a@sxWO{MnQ!SC>J=TyFxv`HP>(wp3S;U4SX<@J~$XLqJk#f}fA*LTzSe@LYvFn7W@I(O_-bUmv? z`{84BwSBbUD}njf<8seZlYOA=HH3V6>l`8;2_A`eARFR`E%YC)j&fEnKROk8X*>Yd zTFRw7V{PChT*Dpvlw`;(BY)=t&R&y7!_xV-YV9u*C_Tep~{ihgt& zz4bjg1zm+lUq~xGUoC!OZ6{nbqA$vkSlcr_;oAa%G@yfj2^tMs9!T`*?Jx4kv!5tm zLJm4RGu;m}gWjBjuG#~~9x%C?irgPdU1b>kk7k9lb$uQAHl6ZAGMoZg1z(l!)|TvJ zo6vW*{cIC}kEqmd$v|trpR09~)nyi}YG7b}a%$t%juc8Wl3%9Xv#E}EpD1tX-h;+l zC}yd0R^CXLy$!wN=~()Lx4i0aP+dJPyz#8Qn)XKNSa2`^f>5VrhM*Il@XBXvHytk5xQ|9i6>>BSZhA|~T@O3{Sl0$GWq=7ZofLPra`SHK02(ZwRlpU?%t z{-|ld7|x&p1G@*hK6&I8_7;N@LOQb3y8e#K1`3!Ahxw)>Ktb-C1LZ_DW0yx}Gx_~$ zOJgy=vu(@0rXmT5-e2bsieHsm)I+u}&W#5GjyGe34Wq$I4WEj+sy_H)*3MMnP(9d} z#9Uqphuevf+X5dr{k_PO<=1VPkk_W->^LXLQBIh1a`MGu3z*8jHkLuFtr3}-9x(Q4 z`wc$L>e$M41`qRoWNXA1 zaMe_PsTya-FOa-t@+%P)x`-9HfKFgujE`TfPy`Y5n=mr%<-<=dc8&3?PX1?J=kg%_ z?EA+-R|{uM{vABoflDsY2kWR^2fx!@=73Xj^i@9ve)SRtCE(WsPkB(M+nk&QCEl&3 zC_-1z`Q_z(Ch35s_V$&P8PvJz>*a^akAoHhg68KYt<}{URx|hij4@wviB#cCKAVd;bgV08!SJED|Q%fpKHdc9TQ6>Fm_tUj`GprDuD8g0H@q&KS>AZmV2<|2(@l+iiHWV_J>1^z= z#S`0b$P76c*WRo3bF}6B9X?sj(wQ!cFnE>-yU-hk)HR(?UYOf8_H#m7R^z#JWooMf z&Df3NdSQS1{=x%#NjaOAC1OWXko~B@y605UWkM?dJ%FlmwR^8RRa1cSGh96U#Af&aI@OIzxrpxFz84{1^Qrb z-`QaOM;w6>Wa8>OjS0a-SrLH%xN&M$CIdW-XR!PgX+T>|z~xP(cgIs%?2o7}3Ut8P zgg;)4EgFT-4%i>FG0o?V43oLMt~@Z9O8Y`V&95Qn!G)<)P5}d61APLxK4Zek#v5H4 z!a+Lfgr^B>r;#aswPFQv!%sk|Zz=re`53^kUs{ya7xB;oIhAE-%Otn>y5<7`$KuCm z@iT+<9VN+6W*E+9IS@;wLDwn3tonisr` ztr9l;A0wY;#MVBhBOc`R1mAw|r7m(mT7u7@{SJ-O6VP_)qEt2F>^JblcH71|OCTrr z+q=1Y){v69LdzfQRXKZ->kE$bBc~xb{|S8fD($`UpE2%`S(P=ZKl?I6z#DUd0?T{% z8e&+Aqq9T?o|7(GJI1eHohg$H$*lEx(b{#MSpWVuDw1Czoolik`NrAD z+s6J2RhM=nC|%q`9&SFBtmO3i=KolDWwjWVtzc*a8%@+52BSHBR(NJuUylp${3~%win0|7Sx4TF`mh zK`t}kuu~{XF`ls%JIS!lmR7bm)q*X1TOQAmS@}=QDmXBnR@DV-kO3(#cliU!H#d}X zy?mi^I^{UpP0r!fLaqsAf?$imOPf!b&=vb(+)*U4k1uxe;uRcL{GG6rA{hD>AG-*NRYNV|EO~xUAp?Mu;zwHZ?cfTG9sJ7(CL+wb3zWk@u&f(cd zrf1|4xpB8oWw$-y!o+NIq3sEh-GA}IClo?K; zz&E$Hmlx~%$pP>Ww!Nnz6^9G z0~u{V57X{_NbUGHjDh{qpM1XMy7}p?tu9&I&++yZ+J@a-+Bjxu*OhLd>^gqqrB2}v z#Czi`aCriSWA^V`IX)hH$`YE?pLzKo$yJ?<#6_A|VE@KKI{GyB+A#;>X@h=_Wu3Qr zHBf;Le$_Obm@{7u?q78xO{4b%7FsTu13N^sV>Pi?5WoH8fH4RN)Ua!4jJu9Esj@GQ z0risgrVGBBL2n@5G`SHXWMpu_r3&PhuLmHPlUv`a{!;Ld^3HrB&8yR{t*AlmXx-73j=A&3KO8+sl?bNPI18#cACihA3h_ z5FPo-;OJaelTjP1rT4CS6RBb+#{sGHd{F?OM{;h(MD9*FiV8Y+M&(B*#k<7`cxas2 z=*0$jJP8G}@94aR>!t>+x(A@&>NYyxx+j+kD;hw!^#EX-jryN%y}_(jyuWr*=NDeC zYY*H_fJW>Bb6TKZA|JP2#b-tf`y@c>=s^g1ev4NZwwfjR7q)2~Hd-sGlZW-kqv?H4 z0FvScs9}>FxuZW*Jz`z86|?qmJ?aybeiNp*$^W13@2wY>Gh_|Uz2IeHkNf*!KP$IX z>)ddd5GP%-8UT8L0*b9bYFN!XQ`KAU4;zX2g-nq=?#ClbrUS>Cieo2#bmD$ucL@I2 zmu=!nP?gS_+?g&8{Pu3{j0O?G+ zIt;jEoL+TiiS>K;@fCYGr+agU^SlJPo7d5d%1Au)zT&$dejI;?dN2!(C&qn?EjT-r+0D)ozh4kQncwmmh~u03Tf9@vDr0bKJBN3VEX5#B+injOf|G;qeM@l&WCq^+e5{*goTnec_4050I4e}+g_wMFdCw*WRx|I%d{yjO z87~Q^hCQ`$wKqQ$wtJd1?8s`<6IpSH&Cd2+A;_M6Tix^hvHIn?M~JYT$e@Zc=TU>V z<;5{ND`+dts_)~pTNRmt;B2JF?7(|H3d|?by%Oq>ipH_P2XmvzR?6eXOoJsRLV1tI zq)aM;zC@VU#_o<<* z{OU})E~C5cY+QPFGV=Cf+=e<+y&JVVC1x`ExR5X3X#uFMt9pzEMUT6rH7UnL`Bj2l zL7q7O@P~ILj+RSJT6%W!6$ae356+-Q(-D*BJldTP)%JEwT(4y4_ zFW1sgkrW&BEBQL!{;#SoQ$KlE&5=Vd%uIW|{JT7U?K$){I^NMwo|||FRee@93c3MU z0ZKNhrOvPMk!$}Vt()YR$EYPr$A8bACnLMMhn^Vo`@DKISLKEW%ucjaJYaX|tP#s1 zfu5VA6^+7et_gOPh6(X1?~)2OC0Mom80e@#mEYrq`y-+6TnchAaD=aCypf8#Mc0Y52JB;G^QlGjthI-evEr z8{8d`543C(#i>a=!eg)kHWw7k||-w8!YtWzx;hO(G1$)ibBIq`8QUAMU{)Y-Ki8xhd)r@)GZs zTt2AEeq z7sJgT7RA*1)xyJ~);Ymi17C>|j_iB}{hgr(A4hd<_-f3&^6f^r`9Ppv#r?_YsZe=&~BOcbeo(4*?x&Olbp)7{rD_y4$&BJ|9$dMEh$9qlE6Wp#j8 zz3s&rG?b-yVMX{P0ydFTD>NX!F-yMPFjBT?@GBx0KKIr7Z^~JRM@vn``hc^(m7rGw zYg=x6c;j$(L% zZ)&!m{pxHg7fm2G)VfIT^CmE9%3zA`S;$yn?2Q^Yw96iWe)Jm zXtQ78RUUu_KQ1!?_6jy$M+dX(lETA(<}7QF12Pjo?dlU%EX;z0GssG03pNtx%-iM* z<#m%({$y5cEZKhPz8VHi|7TaH!cDpuWk3oPLi;0MYmX-_xqr1_1_GqEzs3nn| zQyF7#6ZW*48Y-wvQj8Lh7+<30mMH_c6#)cK_0#(zXMytu;sIk?f9OALcVI{6^3#(n z3WI6vaT4ntq@1}Y#Af1dpW+{8Owvth-@c6yeb@lA<>tvY*zU9^aezzqI01ccA<*f@ z%Lo?G0P+_z{#{H@MKHP8&CgnT+(4u3&Q8jN8E^t-Wv?$*!l&+w-)`j2HF`IDbngzse`9v0HziQDR<3(n&ER`W z_IY&0uY5i@Kl5xYW*Rv47dwzfHWlb)l1KRI8d?ETEp=Y5e?2P`em$c4$0nvp<#1^O z_u-EG{kY^6gH>HizL&?Y72;&t#JS02eff!HF}0b2M@Mia+e^j+2Ix^aQ%t^q z6-dn@z+ZqqX7C4qkovm&*m7(6i>48EB*uY32SBn2R;K|b?o_$_*xqXOpGj4+;wnEM zGr0gYTW2V^>nLDfbHKb`mi^o_rKBZR-$MEf-PcE|8KD1Y~i! z-qRyF8r=dcCQ9xDz?U&ooLYtehi+X$IG~VvUxBsd|MB#dVNtzbv_lUfGKhpC!l0CN zcMD2LcXv0^C5@!CG)PIz&_g$Zz>jW*P`bNe;2!?>KKJ2+-;g=;p8c-9_S$Qy&-DA| zKvo3ZCr&RkgBv?C2wM1;7l}Mob|a3HgIG4#UAx!hGabyl3{q;}fy4r3nF}>k80`~+ z0es+Ki5^_#cT&x_Y^eLn?gg@a+DjD(f1!rk^CJK-wMmgSdpb}e>1uo%NfeqOA30T^ z`2wq0&GArH1OF^HLeVYN!s$Fcm|Qy(SSGP%RTVUff?~2u|EbE-DHHHB6twcu%;6Nu zU<&YVt>?>)Rxyy}zW?s}HfhwFb(i^kU&i;`vTl0uo$y%Hjd^2Y7a)v~_jwj}gtuAz z#&y4gng}kqDtH<=HB>;HPj1?YV2@db+wkc!$k}_%4Gsg@4-n)m3DE`a(%XashKT~mpV=cNLf~i@e5>oAu zMep>jXhFe3B2)Tpygw*lj%|F5){_ZBEMbhw^Mi$C59Mq@l~Q)dDpCXPn!_PigVmE0 z!dJi*dD`DQ=8&>R!z|Yp9N@`97b@?=?_#{K#CJ1n=nYMpZsii5qr8ZsxQvv3`M1sc zD#3EtQ`5okRxin~W#Zw~)wWnb81!+XzhGL>O1)dB05FuO+qDMbTrig1?c~AI zp0yXQXV#d|-r1V0FwMpQlVyy3RL#HSQ+et8mH+Z71s}d-ng_Adtnqjx*J8h9uAPp5 z{zv^R+{2z#O86iEM`TGqAHT};_{`8b&B>3e^3o=OZneS?zE9R03ig8Z-HeO}qF zRT&5BaW)(;2*qEF1(^gfBhf)tur$EZOEPAEG&S`G~l@H^5Lf9{t}0>ff96d z;CMSNdb1p0hYlQbR!e}jM~c&@xu&DBuU%Jus`wSrqACVvuYQdr2gae%gMzSffZi+e4E}rAAj#E^L_bxu(<(pqadnF12v(@Z@AV>%! zf>aA0TER|<`l$#Px$NmJv78wxvX+f<$D1q~w)EFdb$v zc~G9`Cf8)Tv2DkJ2^Ig_DSf`Tw^j_92eI`gf^Yx7nSeo*V|(bwQIOCS|7rhvzD!CH>eI=7cJvoqFd7-j_S}7X*&jVuZ$RAi zE>0uSSBbu)*PnvPt}$h|VK|Ug>xa+KyLYLF8wG*b-cCFRncJjZ;$95y8}f+VU@@6w zg2;b11@>4)x&T5@;~gG;cb%JS+Wv&sZ6x5!#;=pEEwFF=`)LRW{ogL#PDReIhv|}a zLqTs@Ygot&^F~zgQ?;+60eeybp7(d`$61bU3v$(8n}jyw3|1Pt+JSnl0I(=k6erbM z-vn%(a|VtTgT2@RR-}AR*7p6c<*wAt%#9#_-|aNjT{$V`U-nE#cP3%xE{Jx4Z{1N8 zxlH|GWy(BE$*&)>>_^TSjSB^ok9Al+XaW{(Ki0{m4kxJ;Ph0-;m@zAmV<@RH=GTk) zbSgouB6~i52F%e@ObrgoYY$X3svOP;#Q*z{bBwl1xC)ja$WJjH@r+f3^X|EOF#DdX z-4en8%Rr~!KjxRpylhb3z&QjdIR5h4H271NPbpo<+F~n>H(jA!hGcuXYTgBa9>F!K zHjY^>#hdkK2DMKJUFh>8})&!!|HhXwHEtpE7Szm`S4 zvrf1!Sf8I;w~CS;f3*Ilirm4|(?lE`zpnkJI02Vtoy=5n;3WAMr`@IsV`1|1 zCrNC(>*gFAsCd0WSLQzFh3P<`G{tz%Y&kdxO$naXZ%)vg2HC>g4@D$oOF8nac~#XI zOA0j9O7emRofD$##R8}G{=~sAYOx7;-fITjAN$|gC5g<%jnHoAn!Q|=f5I?jRbjwZ zY8>%h3gEprBpy07P!u&pmygWt&0u(av;o2hQV66rq8s*=#$tCtVi3Q*Iz#PqArO>#@7b>#4Wp~qtn$_Om& z3rJh(Cw0q96C^M29$aQndM$9lAi&Wi-){vXq<7w!2pi$gsbC^BPjqa=046G)m}V5c+ybAd+lLT=4T z_+ak6g|CuI37ogeU+@*Jx%Laj;D0!UE^PDU-Z&Bu9}<|0e%Azh(hTMQEQod<=C=%D zmnnHJA>+KQ;$Il3@r1gdPK0@Fgyc+;F)N@l;Np}vLsMum#J0UjUzDOai=NiMa)|L| z+bL|=a^#PL_jRf#E-=hpzt)E4p>I_lpfHCcZr*676@Ps7) z?-S~diTR=wxzFP1TZ6gkm0$x?NL*LM{)h%5G9cDj!p9DaPs`KmhC>zdw6QF!W!W%Xs6oMrprI+x-_- zN)oTMVy;Dg;vmN~;Fxr%W;+x_sV6Iuu+#RTnraiczG3Aal#dI7`Gd1w(@%;vv9d#Bqc$F%Q9{r2)I!k*mca zh2PFNPA9}T;$o&q&%|$Q&opVgYaoiAVwHL|nwZ>r1nLffDxVzp@+=<<_jw4{yHy&b z;Z?cKc_=c%3J-*khF%uJBd&;5$&DCQ*_zD4Ry&Ncq;JvtUyX_IlOUk4)PuwHZgA1N z*=u!Z=!*pQfk+9CBEEnUWpVtDP%bEaA-M2|uaPw~)s<^;x`%pj)Fu>6@s#d0vSz=_ zxe#LHBgdhpceHA_-ixIb_X1>88df@!9SP`ajrID05ropB~An9T6A3v2vSUOx1Zx}&4dcqTPN3_0MW?Qk#A;?Ogfz}&S8mLO|6=_Y@)+7 zt{BUQs<#N-aUF(!ZILZf!21#%%=%Q00jsuYuIs-9<=^}go^uG%mie@^OgW71Pm73G z@C*U0{MGAS)pM~aPVxd^b!Xu<(AYAko+ikMES+cZ%3{Fk+P;`y&~s8@vM1<)sSCc# zMWeJ|`I&5|g%=a}O~sGcdrje1xNmnGMX*T|e17v+_Dh2J2EW*@^OW23_p#Z-6Ji_i95EQMLXHqBpSE z6n9TAVTD_urU(9`Wgr&qG9xqoF8?H(tA@WVypMoD_1c1G8Bz*Jj!G;&U4d zeXTMQ{eT>lzgREUu>^LA-L`qFUg1^gJ|qj|r!<@RaQDRt!!p#pVxC@fr6Ke)N<8gn zPFQYfu%SI9PiR|mS4JYVf$2xJnR%xRfUbF8fW&O6($qAKB%3s~)M_wHU0#@Xe>G-$-x)hiU%2R0tiD;h<-cbyqHrE$Aj{sI7F|Ja7G zOhnawuk3kh%WQp@t-ugQozp<-9!-&>wg|2zhG6aJ`YjWXJudFBnC4x%ueNW-EGNjX z^AY?rb3=yEaYL@@r4BvIQoK1wnl zMWQbJtR+Ssli=x+SuJICPNg+M@HYvQ)oU-HJ#r$w z@o%D!*Mwmko%2(+S*$L$h;>3{L+=Yn4+Kd@=(+Aej2@D!UqyRp%kxZ43R)9$cV(FU zTaN65QD&gHq1IWZD^1;V4=w+;bZoVOxb<1`rAMClEGs?H=|{+I)+*0Trz;G z-4=dE9AyCn7f9HR2Laaz+$1IIr7qkkTDIt5Qa$_CQ9`n{L8S}-z7$vU3W-0m>-Qe80m zSC*Jz(ATj~QEePjd-5^~cfF4H)4EIdaJv@e3E(@yVSD?uH>FuA{NM0?@y=LmiAr`9 zuLLsd*{i<^4hbJKi!`NHkwnt;+7Zvu|H#o^&q|Z^O564)7p%7BR|Kvlq%v)ift+v- z6S&aPYS|nF5FTC%t2zW}e|pst%|4XF&_RFB{&O1!Rx9tP9xm|Ow|opLmY3n~FM&Fy z2Q=`yES80nvP=Jj-@)n(WSdpR^{qo$zu{mS$IY7Rt@MYIg<@(r(M(aOFZ{)(FR6*4 z-rrcISRL2F17H-tDhT8r-cCUkGWyoD(?fE%x}zg$M~Q^94J8`h!~`q>AdhdC7@H_S zAZtan<#Y3xid$)`InA6szKe>xaaPal%ggqgT|{HnBbo9YAX8E%#Nv(}R@}dW#K54l zZ>vVTk=kU1M%^3~tF+3mtO}>B+}Hp{NiZN=~VUx_5Dw$OO z)4FL7C%a(q+rh37H9Ok@S2&q5eWA|KpcN7$m?g?}`552#`tJAJpUCZpi;6%Uzi z4+3X*jKhy>EZCq9(>T$ptQan!m{$Gw3MfX#5mX;M&R8M=14YQZVMg}Zu4c=42=Sn{ z;(Km*tUyqNQ8e_Geuy9<-y0Xk73Ec(2@_f3fB3UwL&u9kv;bPrKO13Z=H_?N~& z7hub;_r$CCK!^a~_*Z6Q?OE06UO8ati*!|{5X4>#<1zFiau_6@RYAY@K+!+Z_i{3R zTJ zD{P${+GCSMx}rK9!_U)lwX0efrO8qfm#UV$puLEJiQW{s*O>9^vtsND*ILpqf0}(_ zSS$~u9XXE_|EIP;zcgSTD2`?jvGna}QP<~j5K8d~R5k@;$Dcr-BC4XC2&>s11>bhsU_gGRBKw4) z5t78n&?z?sZsq~-W=~aw{k8aLDffZ>A#6T76z;2-6*V*RY$oKdx4fYB^JqV$IC$=j zvV;VlU-^NZQj5(IT;oCJtSZuO2gr%eaODR?H#x;SI5*m=jY+uEj_k z#!g~OYtT)S$7uq9*W({&yV1I)kJ#CF@Sj93JIt6WcK;@O^H%j-W3WO3G&3PC8WZ`G z-l!L^bZw}DJlhj`+*HPLx2GHJP8tNjbs$k53MTBrG?n*V&Vbh~vjrTAw%p=DK4^yg zOiINa)Lb#pPRWdQ3_&AYzJPp4u_eqnNcp`95d#H=GG)w?5dq0*`HpOupa%G^I)z|u zrQh`kaR^wl*aR87SYDf?q9Q{}w)!HPXYUme7_BOPU8gNHd75PY;H5W&+6GC*_Pp<@jKW%!D?YD2HE^@bgg7MV}VX9z+MPxora{Xia=r^WAj=8?`L?6omB5O;KhaB#x*K9e#IH9X zR=I>P#eNfj{oC{_ZQ}g`AGmQgvK`do@Dr!X$x_Vw8@@|CDXH~_DgwmKaf&wAAC_?|47=(Y(U-x3jH< z!-`;=rJyb_oBMxfZ`pl+eCeJLJAx4CyXJE0ogPvteT4%(xfGm!iMaj}IlVq5csg)a zLry2!cDL7P^_FEKffKs^dw47q=Vd4s5EIB#<%_hxt^?#gIJ6h-{u>7!E*QJ7GRmTR z8Q$m%-d7b`R#R`%B&$W+j@5*Ib8N;mM%F}x5{9g@c>78MWGaI6N*riUIKlJ&qbGZ# z)X%UX^Lgq>BJa|3m3tN($&NToUL^Jjd4uBAr*bPXxoz!uPJmly1hE$Acp}^VD4Q1E4ezcw%W+{Ff`*3 zW(l^7h~OMWJkq%cu7#t{i~A-byEWwxHPNZ^F8;>v!=}X~L4+Y>Kp_f^__){MPOdO} zQmX2oZ*z;OZ#sq!5@TSlz1zAj&e0N4UJHHU5Rd{5dZ&iKlrV zzbk$kqGs;PzR2`it{Y3PIgx(Cka}`bWl+lzUBd;a&XOxmrSfFmfK6>#8g@fNERdEe z=PwlsIl2FTBazO=N-~{M`|ZMt0xeXO`YGgDU@1}%TJRxE0ieemb|%n3XRBEgBWEOW z9wV08vQa8>+`Zn`B(#C6^FL;@L%~ZICC2rCs1cN&svh+OEndbeOT94x(2(zZ4FZ&E zn}FeJ%ny1j&sS_b!?GhFbUVVndPy7=D9oPVDZmdlKzwD(b%=VAXq~UK-nv3m_Zb|M3^sDV9X!_C6ogar+ro z^~%jr1}_yoxh;1$10UB!A1)`0MZE6Q=)Vn=L+tEs-^R@Ll=6v)hwzcmHkeYU4$ezpnr1Bb9yaGa04TQ)^X7w4o~pYh%WIzb`?=Hl4-8VA=}g6Sp`$qsF@1#Zbw?7Li=;XtI2R%F@up*pX_?78iB3)`tP$Kd=L8lO6Tr8_i2Nt+`q%X$OXakPS73*gCc;srlUh1-M>(tNY@l>FTfO=F(Nh z(`wU=E3{%1;qY&%W?aXHpMC;5#B*X?9Fe`v!Xmv}EFdTIye6o+1#?vaFTWHka32v= z8oZyo-5iL|9s-GV4IU$ZTga5)vZ}QV6O0fOtd00s8TuE00dGc@F%QrLtk3Bz@$*DX zo3KxhcR?fv=?sRl&WstPej^CKdL!i|tr)CcpZ*+`WoFQJQU>!=dg3sOBkSm1H(Mk& z*)Q(hiUW3&3JrV82<7U!AQ3_N?Cp=1*vhZ_q+T-)_cnj|-$9~yJV*|0B8|`8{E&5Q z8!pY4cq$Sn3FcJ#E54?)K9l_O;L_XOiA#k&zxA~|j5_OO7bZ_vM@)P!RciC!Z*BW> zwpm0b_?xrOCa8;}T?Vwo3(1hQrKu7SLizVSS-85{`>Le`>YF&_ox8ni#w3S4c0C(za6^tN@{jVVaU7 ziJJvpnvf=|Out4}MSM?)gR0vw(+qJiU@~v#^i}OP>~|nlaSg{x7ngsv!S(wf3zG;7 zwUQr$#n}Cp-raOQ-LB5*DKz=f$^!!u&PD6+*}<8|SLtyJnR(`!4L-0am~Gjplj`eJ zOk-72pq6mVF>JS77-%@o@qWCp!L{7OA3hGBZ*y$qq2-~@hb2mLXAp(}Jb&$Y{=*B+ z1jf$7+yuIEvhg!dM{DM7f|`)=YNi2>;o5OQ==*5Co(ubWi*Q&jNrTAbXo(iWaqOvU zp?lYtlHJ@EwX{i{VD%0c5xr6xz^_mB&m(EeqqVEEH=N@OwUeGUz8;}) zh@f(=+}G8JV1$k02$;4wAc>I!WPewG&=)4Jc@3OKKM!Jx(kM-;@;88+U9Y0HYFhu$ z&36Ckob3b~&VD40mR>nKUBy+gE#Xp+lqF_oH`jbR6386%qy( z=Spwll*Ln#VF^`IiH?+ryQfQ50RpfZzsp+{_Gvl(v*Bj_QG(b-#6e#XBgbRm^4JN3 z4R1RyQs!H=Wf@sKHOy4{g$A27FHiz_{pp(#vCQf6s@A-Pcra|nTe|pK@h7k5Rsev%8 z@C&9T7)p}V(&hTs@mN|Y$r3Y5s)o@Xr9M3IkIlU#xbds}Rh|22!~m12Y18(vz?Gom zO+j`*=0(^!TLnm8AtCHtqyp0riK9D|T5jF6m@PMTXp#iDSE|1jThLPIpA9kHj|xRV z;FlBx0?h^|_kS-E+TTn#JnP7(s0u$i;;@hDJ)pH-VL)gU^C3IQ7a|&kaoeS;L#px!s==^ z^O#yBFy2yMoHv`oZy6&58*`prI@M)c%skAqD*+cRgSrHD7*!W(7iWu^)kj0J4m2RX zt2RmY32f3q2Cs<^Xv3G3FY&MvFAX_RKJs+rn=aB1o_=CQm4^8>wwXEO^Tyt!;jk0WxGcX@wIxfrkvta=@yESFIu0e=mxVSTy-k34Gh<2@$%*NgM~!ff2- z8!WJ`(|W7deiYOZN&epq-LC(+$;ZA^eZIiEbUgOguw6Dp3N}QtVq?1Ky@O|-8MnkQ z;;>t=wI_MLI@*SW9e>(7rC;cb$IFpYL#35jI^&jw`rm~^1Oa+ai zh5PWoE`E8B-)>?$AvrGr2^u{vE*lcK@I}@!@2=QAF(K@8{aSX{w2R5pwZYp#B0<)e zw>qQ0g@&wS#l{Bb*=q;d&%|fLuZWR6lES*OzZF4FS%tE)G>QhNL~+9RJ?;1PqBn;D zE6blhe|AttKB8AFU~YwoJM>bmn>L&M!tAqP-R)f z?|+ZFPfxg@zU^oA^z8K?SRuNYKvbX2)l~3)qeh+k(Bk*I(F#S);HGd~j!Q&67!#uV z&I&tb&RYo=?u;HX#5Sv2Hmy&(=p-(%N|HMeJcBX~S31vQeD>Scu))faw_1dry+3^2(-43m6B!xctbf$K7Ra+LvPpQZ^KS(|0bsw#Tm$md>3No zQB+6FfTKc<0!qoPhp-T(!^U4b;)#FIR+^gYzM zeXIXOf{)Pi)mKxJ%%j#H)tPuT3C4KZpUqqzry8Fhxg7^%$W=Bee!fh-#_4+w%JdGW zTdJ?w>s+XTa@BU9`ws3@7QMXbU+Ol=$*yGzLn|dTja-5ME}+p19nQN5LF19VjJ9|@ z_BlJm?oQQ2v2HeF9ZLh7^CP~URet<5dPFZ!BgnxuAAv67r zHafc2n0k5D^o_qjUlF;v*Kg&DunGo!gE?$cjJSNVRc!TZn98)H3NdiW#ECB2F>Wbd z^cQx|);fG9Tl${`2q?|H3MRms8#biNyU_h`L0qBZ`d{yflj#EU{5e~f&(1^z^0Fg4 z*Y9oCByelC>enAsfiO^o#_>Pa+?$z3Y!2;*mpLhFgUMSU`4blk@>{taOtTZ@!VZrr z#;~fyA6@pno(-{rFDES*duw%V!@2t={oB+bU6JJSq3721Z4iX5zo)9<0Tz7uqMYP7 z9j@tIRf6&@W9u~deVz`VqV`9g2H8#;G{mNF?%^+JlPlgjOgS6ag)=@1&}aRDZEZ4w z8RsI{H~PpTl(gV4yh3_jav*yQ(|8KsqAan-!B5d=Lz;HTaDc#f##K|dOdLs6@?9^T zS;G;Z_D_YEIMP*D!?nq)hV-X$$z+hxFs*BHj2j{koWl@l4l}+Bx=q;KPs!0&w2BRK1K~Usztx zhkJ(!pXclIqKng*iC2(lU!*Wcx!YZ85;s`Qkvz}>9@WMC?^5M&^!n(tb|fP8)hv|8US3 zutN8+s)#BMXxGPoETqoByVq-7n*cq!Yn094Xc}Kgtg23Y7Um^zP_%DFLQ7<9>gc!Q z@#TCV4n(gEyKH~MBmrK`uW3=bv_F7_s!&ADIa1yp$u8l5{<|5fXx|-rxa(x~ zI7)siG04amCv>w-aKCQ2x7-d$#ug8+hXbx9MO#q#JiwpK|M(}5RArS&>oP88MAD4M zu~PB|{=0c?LSZ!`EabZN$<)7%U*>3|C0=;B#=iNFT#9ovxVQ^5Y}XaNxwU;tWZ~D4 z<_^*M(0metB&=-JXauO#InS&hd)+VZ-AUk%3<&Ad#?Zb%a!>!`5?@6%X zO=6zg+?Ter?d{?>w_hMv`2o*Oh^6xC+K!*l0TsrB3^>-ulHfVS;;+eP+?xe2lwVnv zM-MXP1BO?KRilviv7*F0v6B`Dj{9l$@(sf_DW&}oR9JuucqZ-UCGXJHZG89o4h1UOm zns%>K`mY7t?`_^C$`Me9uoQ*H#0)=N$~~YMAC^)xYK_Qx2h)HSO5vZ+Qs6cgsqW%C z*$<6;THJyEY)EJmlQ;4<7KCGj37B3V^RP1Osln9k`kp6jDx=Zt;hCd%PchAu%45a| zC-(-<@8wy~EZ*UfvwjVBG09BIkX?J-lgyJij#Pz-qF$K zJ_{4NHm4x?YgxEzQ~lIqKO?hD(>B)CoOSa;J4L-jO@|ZK*lbbsOi_&v9a+Q1)6U4! zK{l67fe7w@yT9FBq{|rt*l=jqw{ea~IlnG8fW7%)*Y04v(e~{kZ)^XZuKJ||~>02`1P?R_cYwb)}ahAcEY{ZfJb zHg6*Fz>(?-X>7#@fyjPFu8yc72}UUHGcgFJGL7VN;A5peh)n?l9{R|TWM|{j#uzF* zAL3#W-K_Qi_Y);#g*V_Oa=NcksG|yI{*9abvqDz2)Xb*=-W>4IhPu6U9&Wq5eOOn# zpN&h`mI*U%v0JncYjIN4UlgZ+rs@?c(;hnhYhAD5?eA}d$v?G;-Fud1V4pz z-0J$U3RkzZoTwY#5(;t*XC$k?~FbIT&JPQ7)|JmPP^m+@C zc(qfxe(19^iRHyHCb~Qb8!}CU@IJ z)p2np06#JFw6;tP$*I=HG*1!dwu{kC)KY((!D-F1=M1U$RW;UbO$}$A&H2ZNMMd9- zOFu;!AR@LgCjg0Da@if?2C5ATPDmy#>5`a zL*@tSV)ebI6k(?EU!OV#f9=5#-xiM;6a7(Q_8Zy#!X9-Oz!33x+C$|dRY>gSY%DA= zHt{_M*2(ue<7AosCvEunv;1}=?u_tdciU6(Qzckky+%%{qofXKjW<7n_kL|~8y{^aTi10DBM+xzXua^^sHx&1Be_`Pw3E2+Ql0hzwcB3_J&Y1^Rm3pPbM1|W5Tb{-Xlv}pt`B^D2JBQh!U!lbr^6drVgXHYZu7XLKz z7*0%t$HQ5mOfyfbfAvf5Y<)or;u7pvZF)_g*lJrP_d|V<09@W{wY5-$sUX9uLgiw5 zU*0tVUIR(`bio?Wvtnix-3=WAJIk69adhkafSO_B1E=8X+Lys26$i6@#Y?UPTo-#? zi&pkZ4DOzBlSk8fg(`wpSZ&NNuNkYY%%iiU;>X~V>xeG2;8L~%E)+a4e`;c86^s-&~y&2++>2R22VahjOe^j<3}$rf4&GKEtV^NLasYOSnNCLuOmq-x1< zc%23}7*%B$p#gJmH=nZeseUE0^wQVHI+->Nb4LN+5sV%^8n1DBemW73ap)vm9{!IT zA>PW4Psha%!24ATeN)+SD^FuVfWpXhWuOTem6zdQ=^~pcs~aU?>ztx8istC&Nie3{ z$D=tcm<2)Kfi?94sLLrGhD5pCj@q_Ss>v3uI^-S`AW#RwzS&$bdzBBquT^UX3i4H2 zQ+5vjjU7SN^{1@Kcp`^yGQFWpKL<8RTb5m4C-`fLYWdW6q$Xm$_$BsNjtyP2w~QtO z1sWbI4pb0h%;F%U(v^0TpU0)&HthZm@sRfYtd zhIJ*ApY@CWdh#Dv@4i#-lP+HDnO5?0He zzAJ2*eaiXANDmYG)OXocU*c1losukr&yoG_gdEyL{ zH-O-&W{Bo0O4`uy6>-zYe`&b-!1#`HWG^Wbk%Lkn!=}lRPb)@c7{;a z`tmxl&o$Q9QVf<#dKM^G@DKYD?I(6`O;QQWQNoM8pl5Krrw}o`x69Pw>2(-=(CK0_cdDxJ%g$I!!qn>d=N0#u358-c??MuS0C5cXP=r(nY7rV^Dmt~~DjqqJQ z1ugtR8JkcNT*$?DVlA#bF7uYP^1nI^b-^<{(bMl|;pSc`p>9EvA;W^hU&ntOXGUeZ7J3}qqM=7XYr)T$R&+JG zt1{p~za}_dPHDp%Cy&6DK(=I5J`M&$ZD)*ukO?v$jA(r{aVtU&UHi~K^QsxX=Sx1r z_|v4}0(`;^yd>j=0ycPYb_orSdD4lnelI#4~J5tz@ z%C{Cu`DnwLr`|afiY<5LL=&sA(kaWO7t2Q+d9Vs7>@e2W2>3uBIek8pHIHv$`{wI7s_mZ2zAr2iWVCNL=!|7F zp7|Ee#7bitq}rc0TjWALg9#mdHQNISw5ry1+(SLx1Q%v98!auC1268{rsan!AkiLj zmuJA-1c}&3=sQj&t7YQ0&$-$1oYi*d6-aDEazfKy=*4=Jj{kzA(}J~Ut8uH?aoKU5 zOats)0-Cv(t%i7+kn`91Pc+HZt-(U9M=jQ{P_BtbpxC zu=29_Udi;(Wm>l=c&fJj)n(ImWd?p4+3FjS??!6Qs3OCxCmE{A8|5=vM>>g97Wq15 z*vTEo2VdI=LmPD7!I zr20xGjKKjIvCC<`nokiWo=p`(jPzXo`e&36RC{slVWb3IqNb?Fb-mK?g&Eei+%A$^%(}v$HM{ zs`ZOu6F=a*mb?FIMQa>yo*|1|&)kn*_qk?TkD`O#S-Y1a1rygSDO{E$%2K8ZlNgz6 zH&m1v8r0-v3wa(;Y>_x*P;w~V+YAUfH1%(!{bzxTFMkhUGZ&NZae%7Io!vh( zK*pG;nM? zQQq5ZEHx@@CA?k^z4UKtgKPw?YP)p=f7Jy8Y!cS&G50gOij{D6(JzMB_3sHKk~Kp4 zW^GmmL6uIio~PrVd{J8K7NUGnZWLsHuSR16XopZ?n=`1&f7kfS6}SG2swqh! zILRFZ!mMW=S1Tu<{|Q{^!BndL_@};$VMF7?$kiS7m*K5$NhHn^I^2+AFl(H{a3+%= zT`k>}zcLU!qV)~nIGS&p`C?~?Yq%@>SJSUz11ICQW#(Opz(u&_$lbc^SRn{hvA-A4 zx3_56OrXtr!>RP*cl&jGz@?kv=~VmeZ9pYH^Z%5$08MZpmdw`T`C#y#pz0y4BP_XpIU)IXKb0@~8^ovj=RFtgjj=WzoP`*&mB zuk9g!(Tgk1N@_9l9qXrG{lY^s+F}91J8M{=%E$gSzJ2xt#TfYj(JcC4NW!Ep8k-}K4t|hCQa)Ns6B~(N(rd@A z6oO$A-+|aI*w+-&C9IokkH9zf)disMvex)qqkw9BjX7G{5$)*C2aO8+5r;j51OyFZ zo=Gd4+9>^}2NTf5vn)+c0phU`uC&=q>xKa{#pB5JnU!2I%8eIKHWw{a zvPS4me~`X>Uc%(6VHT|-jyEjZDLeej2&HDCUM@GrI^XPi;fYM-bZhL%u1N;Os^3M-&d#pQ=~fPet30#fQn zfa#hgC!p;Ia8U&GnULtw-B`Rb#W9~;v)3eRqH%l=V7A3ks4Wpyz{HVKT>GC^c2c8v zTP#$17V5qQbSB}kq<}Z@E%6LFinFpcOw;;ms)$HoI-TraYPVi$2FEK*@OyEkMz#kt zwYMBEi?~e8^+?#w`3qlKV&%VF#meWmhyrdSFF z+WR0@b>077anF?SDGb3UB|{7j`1ftVKn+8da+M~J5f&z?*}PR~iYADxHXIG9O8OQu zmUIIATa|SDgCn<5N#{Kl$cgP3w}el_N|3nk2~*k5EOFm6`{I@FrTbjM#p?n@a8>-f zv4KeERx_EY?MvqNSxd$C|3lPQhBf`ZZzF7te9P%Z;1+zV5ir^SsW>{kp(iF+6u{y;gUW<wVRwFQ1egS<_i-_7ct&wbc2ooQ&qW%V_Rgz1$#@W~16 z%(QtyzN!n3QO;ndDPL%496m_K zu^I-_47Q=R=iLgl*`gEKPis}0de@`>bz)nD1SIAcXXy;4Msi3nD7g1g1>H5JsvFcqpx^*&L zqRHCH#7W1`bP`E|N%D*J)P?%t{IFD8Q_`vkLwr2nZ)Q*0K^k{T8!0us$m%e3m&yh& zNQqbnX3mV)=kZ97m5(T`fi7MsD=oiY5x+zfug;;-Mx8eoz`+RK0{mdH$h=?Utt`F`_C5Q}|R*_4c!(E2$a5BX7?=voJ2GnUwRo%9{IpzhS?>t94TPg*+D@%2x33 z3ggCe>^tj5{((Y6FNup?lgkbR=K|c~GzsT;8`XtGg5Q5&x)6T$F9pqpX2XJPP@+%4 z)+mS)*k+|SevK4Lsjm+Xucd7^OTAQ?c8Z#yZFaRbvH5F>P5>$hxL+j=9RhMB*deT4 zNk4y&O(Y64s3FEzY6ul5Xu>z;<$C6*Bp`e9uQb0#r7ZPLX$q6F8fzbj0*@HD@xp5f z>*Ca(uJUs}2xFLbi)f<%B(`tF-BlY=3mcZQh+YmOc_U(Z{V04nY?Nl`@8;CsUz zjH_0+>!99hk5js;xBm=wc%`?!!lX#o|K=G;+i7054C>3uZQ0#bHvtu=xGFDp$b5Y| zocmAih4X}LdXcD6{N3^DhxU65@*k7ddS!XqMOqFb5AE~fF#FbxoTXq z>v{MsC5u%e@iE(`4ODQfFT-Vj!QhKh(1uvUudsRN(952V-D=|=Kt0sre=?YG|1BGy z)l?r)!9ER~*dIIA6w`{HB)M(3E=iAkk$Q`miHn(J-Pq$%ze8znr_a?kOp>(SX;oxL z4C0eLZd^oAnaIm)TeVD4-uk-Q9H+eF! zvo5TjpqTGDRbGjFTtM18b|ausqEwT^3Lyxei|aX+5ZCi$AG;+IWr504h`v(sV_AdT zEWhlk-MP6(ibu+h^yG%4K2IK1PrE#-V1_H}aCmS5Yy0(=9XNRTeX^0NM~ z`dF2cE?w&#HG#@0!g+e09485$mdz!Rz`Ygn8pQ>XAfvR_hxVOO>mjc+^Opgf=b;B~ z!$RDW8>mmvVB@3)ni$@3<#*4~CSnvQ8##?BXSom?lgnOz!#>v98{*+2}Oc zhZ;p~)(?Q@%4gVVu|!*PN?*A2N>)s&*EK0-K!LD1*UUlW5KCBID$ zXjeU-)KnbUZBlWbjJbpNR;+k3M z0KXk7k-K5k%swI@JP2n+?1Tscl8M&7Rv+1)!ucx$%EUpd={>9U~f}jJp%#R)h z;b`w`;(FiTzaK(NzVE>s6C8Vw%8d#2rnn5Fp_^4z6RzrfUB;6|3RKJl9R+qRyR3<3 zhRujQZ->v(j`UlNc4W3@$r$A+-f@(E0XulQhSCy*e|ENLp| zb63WLSs$KlRIIAa!&_9iFR1+kgYkXRD%UDJ5*$Jo!2aD+V@fzVjtEQPbc z(bC7!qUQ}VC8r*%RY743SvdNshQh@=crx?t78Rej?~sI1+ zr-0a>b7bWi?R!Fgr5rIA+vU%`T73{4W*9yjJzE+*rKINaX@_8TG$7X}6%^#M@(i3H zJW%rK(NT=u|C9<7|0xv)F%nano}pE*-I_MEzrJV9jlQ(bO1JLxudhk)Ox?Ej6cQ+X zOuv^_W54qEk57@od0&rbZk?;qYiAS)mNg^85XZk~d7ms0hJ4Fa_hF~i%>M}{O%VTW z^3dovyk^t)vY#`u-Gz@lDFE1K6|asceGAayH!S_|6(?0+?%v{b*$cg^$tDQF?omQUX3 zxv!<_mfN!Ctf_jQdS2@OW9W0)UbI=R-Xx>Xp2vF>+Iys;L-cYa$>^||tE2djX2gqN(=b>=*$!1cbJ{L0hw#YaqvrVoGHIxS@8x{r%U;OLk(?#sQ( z%UED=$Q|hVi`{2`Ty-uk_EK=uyYNV}`?oM^L8G=*>!;OzGOUwe^{(NLm5tYH8*3Jn z$e|cFK*DYTS5Ms-Z&mS}K*;q^j6z#ZuCd0P8`$8LXo}|O4r{?jL(IFZpjQQbY^C#0 z4V*gPp1394l(A3kUpMT`K7Xjg{CrX*z60I@OKIahFI*&?dSbTLP~*85{l>IudD-{d zLrY?yvfor=S#Kn{^W0%Mpj@lQ!qv)X+SHT5n}P0>(i}El78M- zU9AnSp;nOJ9Q0;N&f4@-n%%9Gxrf_G!*lOP%dutaKxa0R0KPOSuVNp=vnN#Re^7Gp zD;LHUxiYc36pWrJSyaC_a(1v`X>;FYpDEVAcP^drSrNkLvJq2{W`B)f%h}j6JWMa1^`yX`H`G-!j;aD|f1wF7v80kAf}!lQF_G-RVOh`z~6 zd(9BfK(51Q5+A*crc~Och$Qf|N@Z}!MKL^S!^hwAh@GpC7oj3#Z%>=!)9h$r0Z0xr z`P_UnE!yX^4!mPt5=#3zVu4ROd7pk#l=sx~{C>Q2C~|bp^RX{*jo$w=8mL-re5rzT z^g%z&ANxH7lYN2t57NfN1B@J;b*LWrG~Se2Q9L|3%f9*&8nmhntmdnL|8+|GpbsB4 ziadDNu!Kx_z?-gBDpwU7NQ|YS6BHOQ{bcOYUeoo_&P1x$+{{Z;vS+4G0- zECx9OlP8@39vTB51#2entuEaZbd(eWAXT;8WzW-CeOdvRAHS?l?_U2vV~R1G54%C( z#TaBKy6^(H0rGLn8T~@9>t6qedi_&b@ec>xzw712&YP>(xR&M{_5XNvL0|?SY4B;& zzgzrU%z%CxW87)VfV329-)lP~O_sDRv7{6vd1tO77Ujf^&TxVjsS zg>%Oi4OL9=iyIABX3=9xM0hAXd?LITffd{(j#YQ2O^M$7b~i7FY}U0d`YsqAo|DF9 z(u-8~xUTy6>L&{^yODL^#`aQ1<7f8>CKgh8Q`?hg@$tU)B^0~|y$kkWdivn@98Yld ztUq@`6R$_%Utn*|>~ZMdY=slWKidg$e;rpYg%opv8|=}4PMtgq#{w_K0=fnAEU7;l zHZ(YA>OUP7GH-qZYId3Od@|%~0W`M|`1uaWF$C3c6!z6IP_i?HU|*S}*#YaG%>r1z z6@}JD_M^CDU+ynO*}sJbxZyEKmbQ5Qy^I*`b<~>58IM<>%6$y4tN}76%of9TTOuuh z6XD}EE^pGqfu>QhlYvVozbeL>ls4$}4fvPIo4q>>fNCTtt+Vwn$K?R|V}*&>YC~P; zwkHq3a#dW3{46=6MGZ>l^0lA#?OE8;4?f1{{%dNe&?LNzHy(7^@;&>ypyNtrkAbf4 z_rC`Jjnkx*ox437Vj=dUZ}i#Y>_#bIjOI{SxO@u;NwN#L>9i)Y12&NpW{)_nNAzva;&+ zpTl?XteD=Q?x2Np&RNl3fLa*i?r@PNDSIL07R3Fdzi@S&8%4=4P+sj6SzaDfxR=Ew zzU`x>lPO^bSLP~5%hIta2PEJu*7Nkl{pL~^0X*j|D-r&Nx4ayO3|%<+R&mACaN+^> zB>%p7S7y2|2?JRVKS~K1rDw*l(=>_uP{ZILnMbZ|k&%DOY1(KskLMpK8S#*YZyx(R zF=jU|(J#OV!r%FE0K68FyrM_=bZm^-XTaEB5K2avSq*$9qhZ(&O`MEwf6QUPUBNH3 zn=_37*A7qlOSNVeyWwy;zdwe$8EpG|8m36&bD$tI`sMukNo9iSDYW)F9Iem7^}$9) zM7ReU=jhhw}$4&~2Y59R@x+Zi~Fx z9+#E$p)(MN6vMQ^#?-Z}IRB94y3J2IxS46!Z%Jb&`jxi`6vcwU$TW?Cfq}NsT&v51 zT40l{7OMtk*pot5{>VC%@m!8Q$oFgWJxc|m;Hh(Z9QvKL9ob8w9KV#+R_)0@R*q9H z1uGwAah2rhYwc^}mSZc+%Z8Q4*Lxc*Lj)|f|3Z;}H*rNywPJwe2GR8zN3y8O#nf26 zedo>0S;(hQZ9kZ1Yx$t-SY$OFAsjBO%a_V!V&FRl(HfikIG?rhd=a@gN;P`Z@yiS+ z&PQ+tP4aImz&xiY)c3L&n)LISBuI$7vVKR}oB6f-DG^*FO&aGLp~cIw`zEed8m^G+ zd0@0nE8@j)cpD!-?fc4jpq=m?=p86iqh4i4M1Yd0SRccs4!2u-K}pj7`QS=UuQRM3~Wj5#tqEC+evI?@+>t3&BI*&7PL!TTrwM3HlJQ{B^lw zW9xt^(bYE4ho#ZL49QG1dF`ugwGt~G^=pc;1nfg~w#2*dApLJ2es6hpNit|jzx<$J zr6kCrWS>y!>-_>;L_8hgU>SIu@tUL+z0uvdaed_Wa^2^6VfI10&Zqa+=gl$fmun}% zLDw-fbe6$o>|_7Ut1=njT8r%#sfmMNi(I$q)flCe;}3;iuAS~Xb+^xtEGwNVU{Iqm zPT&&RD&eNZU}x)0{ml1SVqX%NuxhEN@#1r`zN0A`0qV)u>xF(BO+~}V9GW63e%$bB z;U&7o)|WX}O=0O`8izaZ!zBwHRc7Cn%f>2*ILY7r3hd*YwN*9A3FT73JPjk-)3`kscaoL$ z%f||*=M#dp6#S!%VH8plBM=FJwCKen_MwuocLB!nHY0udL;6E1N-Ee4b`{8cL=J19 z&}LH3x)Q_)@=Qgl!6>qAP-o2fdeVT>7fc1y081_0&-{bu>48!h-}cuT3KEw8x9Uwo zzB`^bzdUm@WcH7M_%5E6_rlP`98}qWZD9Ci4V@oJK;Q1j`Jd)^t=p{e82g%TuJOkd z0y>(UP~PHtH>{w5VP1W6uD#0>n503a8sk+j2YArO+Q~t|Xv}7&oHn73WC&;gEHyw6 z?aw))ZkKqa;C%d(cb8aF1Isu|ES{@Db{P6XA$_c=#aGA-V^%lr>_p~j_5)F|&4+$m ztbyUDY3EXla3av)OD*LTsIL|{Z8Zh_UNdaRGl~}$KEDd*bvg+rQsVlZ2MTYPZaywi zEh&{S;2zVhtXz#5jP#eztoD;zJn#+q-t?MzDNfKUsw-B!H$#`Hg<9E2LVI@NL*6p9 zMovwP8}7~Dcl!VfgQybmMOUA5zwpkY&WMUD}jhSV^ zqltWOAfYYJy~ugFBBjt}3UK{3VY>H>7*yxgv&2Ten_(5o(&?(7>QpBaE397knX+v? zl2x?s6s#V|!tWid#|>_ zv@=8u2i&3ROrh`pgM!`EWU&#s=GF!ZJG#2AApg1lxl;UBSzfL}RApQT$m^Ku3GsS% zD^D2iY4H75$wx$=dHV4!>(?9(6mM}Nzl&QRr14Qr+l1xvwvgpm|*-L z^w<8?Iz}5#I3}hdvLLhWgF#n?X?KI}b(6M$exH6}yUv9viWrLjE}g)gl>i5n;>Rw) zgOtfM;xo;Cr$!}16oO{v7(&)+-o(V5$u=M@JWjcgxu>!5@$uPi-50gW-IDA{_;@|M z^Uppa(RG~aF@@V+Z8QRgUz#Uks2 zU`k{?f>0Ga5|gGj(40q`EvMepSgB5-yr$?Ga|4&xT5q?#Rl@H7NpT74l+AsuvetdlW**LKIi#97Ieq z%<89IU5OtBy)rX!tgoynw+EM5TtVz74a=QRzqoJpnhZQzad(~&zGl*6f@xtmO$>Gl z3{wfhSnj!X+o(@p6s(*&?lsrFUf84*>#2v4Z zl9vf}ct*rYwqIP#{!d^&v2xv_V;s%RP0ljqYAF|5yZl zcucz-Bv6H7Y^gU(>(^<_>c@(+rC`<PXEgT+TV3Cm7za z)c6>-ELY=Th+$(Ch#NIrFDNj2ASE7MDReMrNvzx(PJb{UwjZw}oxd3IHK&2y*B+$y zi_L63WlF9&Zo_U?%ZkBBb>`Ad@h6Xmj3y7P?oMw$e`~HNMmGX z!#{I{LPg*aIh(}N^vr0+a}yv9vHkg(Q=JH&G1uVnZC%_9ygpuld<6C1xi{WlYcjGj zIlB92ll!%A$41_1>|*474m1cuqrG@Bnxasws{`&jpNW@$IYZsuvHIH*zq)_T|5gQWgExOrzzl3zJp_T zjzYgn6>@u=m!E+hX_aj(^vXS+Jlkori_OK(TOx5k{CtFaP%ofmo@R;4P=<+fZSg$cQ~HBO$S%lQ5W(n9z7W%kNjX z+jwC_wfbzFeEB7Ly8`;iAZc+loGHP4lB1U%3%1eVOdaJm(8E}A3?T*VDh?aJ)r7%lR}F9X1;0X*Bm#z>V7HAk3+fO4=u9-dXhy#GgB9=WQ$SEQvJVfH&M zsedCg3sc#z%+guvHW%y=TDB&9nptw=5Aeos<;HW$DBTy=1Z#myd}NO&D0N!N8@K#vl~zG2t%F&imFZ1+55!Ic+9W zrDNx`h6P;bI3?|uoCp6H!$b{j3(GKW1J<*zSxv-m%(YeGi8@xXJjVQGi)JmOcimNFon2*sx$HO!^X)xa1sQ^RWufcrcqrlPTlDy8NBq`Ob4Q8nMlHXC2~|KIuTHAkfH9Z=Ax!7j zNMXRy9v`^+=xcTVQSg^#hQ?~*m6g6~9h9h~P*XcLN8G5IcXQk!TZbqtFtWDTUDE1Kpg~fuc$|t|_;ofg@mTC2hkQ?_3Q!4B_ z2y$6?U1PN^{5m20XHOY`>5MdDA?~NUx&ym^xso1Ld3zbZhHIvIGA3|h&*gVv_Y>2`<-^f0ZCXZPnY*c61U5-FI9Z&ge%94y zd&~!4(i{t(kAi0-F*%i`IWYO3G5*M0(jCczuJ{59K;UOaw zB>NBlUtgnZJlk856JCJZpod5%`8tLBSD@GauL6pVvp>`4^0WIrDr{JSa)Ya*l{fFgQyOh` z^!3-KL$IxfN`1hb*{GkO3KA))A3wg*pJV;ePQ=EQiDW>32wpF9_cd&aq>mzm9kx$b zK@76(c<0qeW&&R~GRPG!D*7{;T5R}-NXzvI`N4mYBfF8Lz2dUxZ3(%VlA}_33vGI4 z%td_317vM_v$8liHT8b;lN)RMdngahNFnD-$dF{}es8{lpHX_3b=gK^mYh^VWI*k| zyzppGHuFW-zqle(SgNUBEHMRIbj3dYk-a!Vf#u8f^i>f+UnIULk7PS`KrXqQ7Bc9B zH}oWU$$kp(bu-A~(ruFWP%I(C6Bk!R{nL4&^F91c~93~fzfpC@jKg8=`?wy&Y* zvu=kKI@OZNO;7Rz?i`*Rz9bqzS%v+$%ZyqjXGhn~{9!1_G7Ng3vwAsZAlj>6WZG0% ztUq=7XWb?na1sRX+>|1CZApQ#HTOf9>J-T81+z=k`F3*ppQ>+VgV?kE(-vAp94SqL zI8~8=gle|&pfRoa2AqS0IsaPXm&+d~%&F}e(YhKQ~^`RyK@=N-~u)b|w9N1Z;n zv!IhJy4w+8*{Iorj}7pS63}z8s-LeUb^y)$MCi%QF3en9e_W5*NjjvJhV}+qju4a`;;g43{jW#x}G1|P^&Z^7UafVd-?c+sZ( zNVAc!09j_)qSN={OOt7dF%|Vki^97LUjYOY9I~JeZa<`w!CeO{kj1|0sH0<30TDo> zjRg%$5l^QfjyY$ytS2(A^fhhb4*RxwFITa|5lgvzsg!AtyG*rB4z~*_s4RI6Tb{|T zoSXBI{OR?P0ud4SzM%fqy|J)U>2my}-WIqMwKd$F7K0Ug2A{u)c)ApL*Hqlp zFXlV8+!L*yci7&1+|<0R9jEl{K51;wMmlL&;_!7h9n2masq_X!2}QxTqY7AHp^Z9> z0hzkJvYWvke5($<{nLjIxbo`e>U|gIS#3ROscGoA=UH_!=}{!40Zo{>X`sr6f~ca@ zyj&Ozg62bsm>a!YM#koG=gnL9lv*jeVr1xYNQX%Ie9Pt>mjno|MAL$#0@%HvVmun7 z*3kCILl0ovq0tIioHXThs$yn)w2^E!m5@dOThgu6t-uM_?^!u1=VgcYEBDghf^DFD zRXP>t+yd&}xz7F~)KwBb!t_uxD z<%5P#IfP$jyH-10RB*UuVS9LzawlZ8<665?ab8Kdm`!OiWuiKfEV)3XXb`2EhcNqO zCa=@St0WFEscW`RbMiu3a6iALo}5P?E6Sr@sGv(1p>3InC@@N4D-Y~#yz=1*7Y%}a zxj=l~X$WvHs*i9}8us#g@2Iq0O_3DGS%LiMr1hZO0KUYOH!=?d2Ks%s+>`rCBg+~0 zl%5VvpmnUC0M3!549lI7m%O-`+Fx{Ks2A)bDKiHjB-)@0+rGkvV)4c^)IQsNz=rQx zYmF~~qQas;UY#m>JPi^|vy19v+$rjzr6KiDc9b=^r{gedD0}Y07pq`>5LVvGhFz8xWh@_uSy9cdVoNIsxni(G)Co{mp zMQLFayd)f0TIn0H z!P$z%vQN>BoNgdki9UuemAyCiiEAc^*ZDpLu8O`w9Ct_(pSgUKULJ;1ZGP0)^reEa zGybyhVSHo2CVbDgQP1GPvgzvN!x0I|BL8$sHq^;3#h9wj+u1hGGj9$^g~b5s@EYZl zv}^RvM>5S?MJyo0F0^8;ZlKw9ku-{lpf>}gLZqtFmr<=~FTnqo50coUJoN;%{tDl2 z83g5p$NMSZ9eYz_09k%N2|MKK$im&u(fVMOAH)05IMEd@V5y<4W%g2I= z8h1~ioz)?1FfXX!4Uq5JnG}|<4k7tYPAjQY;R8_$5^+}ODyIh+S{kU?zwc%26zYbh zm14BCzd5P!ui3VsmHR(wrMI2*2E^tro)-oqKW@xfNlOtdxoih;%myLO`v44pHMxnA zQA{AcOp@wG3w)Mo|6TCB{vs)VN04{sDD(+TYx|frM#g5mfu*Egiz_w4lKNi!p}m(S z=wVl2rfwylN};~)tV&J4PwpgR`1QE^4x(G)lVn&_Xpoqb+3gNSi!ty(;pbn?JvwQ& zCV%M}PqAf)^u%Hq0PV955JT8b`z+|xxQ3O*3Qa@MR>+sU`Fka^h9 z4B41ZHEEl1bc@fCPlypK`eCm_G+6}kI@9s?w%-{IHd?$n^q4$fvhTQ^?QyeC8oO4T zJNFQFkvgw5OmLT?-{IpWXp5x4=dITVO_cz-cX5C%+1&NRj34n5@@F6 zxe~i9reub_>Q8dM`&N{jK$Cj5`<^#C<7olknq}-=0Me^l zn|KTdN>~NxWpIKqg~974vq%2E(7qS?@-cs^bpZT5WvaoUOFBFvY4MxAp$gzMk!?v5 z_9t=0EoCI?5xrtI`6H2ole_ZZdX%=sjPGFWjOvbX$Rx?o+v;gYqPq!k!BnmDn)eUn z*a)1=7reu8ZA1mx)t}sYP{Q`er~?kD{F`%5wA5eIMq*}v$=Q-ux~gX%&1CUnSS z*46Ex%(qb#*bKFl>F#=)W+J{aeGB|nkja#@5F+12sD=;DWr360lP%M~EI5wbw zRYI4hEwQ?m*E+1`p5>Q$irsS?XPrjTXCBPE=N~GRPuHfs%L*w0OIu>L!f9MV)n~1B zM?`_#0waPUi#lN+z$Qjp7RoO_U?a*u zUAJ-)Y@e_J!{+~SabFb*R&8cnVynCR?MBd=<5d6(7&YxZcN21)^D`=DFS*dLL{PMM z?@i`GjMeG}$%lpqe}_q0LS|F-mZpu|~7+7#Q$Zv3NwJaC|qdo3coVffrwA;Cp% z)K1y`_(u~on%76=i+Z8fo^PE48M(6n+{Ix|LVs3bVosvSPq8@Izy7<#O!$3yhlC@V zu=FeW*LpVwa|C+aX8MtacumwE5Cg|rNZ<4;Y zhI)5wEME_j)pG=%0zmSuMQyC_UuoNj%7Ra8?$+Ow-$p!@l{Q~Hh^Br=ukp=mXv8jj zm%FB68tmw@G0)6_`Z|7cFfJ*{Zy>2E9}n8hk!$n&CQN?&kaHk|nPhUVIHNeTgr?%l zOzCwyb#}ww$LxD#9qU&e3J=12{Q9g2%s8uy9UP)+idz1yTs6qOzB}SHMTp_N{n~0% z-FcIx&(DVf?&%h$)3yF*>^lElR;&m-j$YrjG@Tlut>1S!|H=})|C?h|R-g1`|WR=(ze;h1fE;2mnSU1Hd#^$81XVGrnzShJIiHmGtuFyNEahTleORhHXUaD)v~^_Z`USnG&*Y=M znor6sT{k;@0UZS#QwOpNy3!i^FuI(e&xE*ma}*vPBnHSaW8%HP|AISzT+Dl*IUO7z zp5|Rt;)m45oHgA_B<7DTIcIfFJ_KBJ!EE88H4S&Ar9ZDO%Z?Es!&aE>fFq$~?(>5P zL4*l=TA68@U8*}pen7o$x7kYplJ7057A%3T?vMS9jIe&Ur706%(cs75#I|kW&q$#} z#R5qbA_y5^6p>GXQ3#`7@umnz`R{0c1_ogt1b0q%|4oa_5EVA&ysq9@r``1?!aidV z&562Za&y-XjQ?s*6p0dfht73JhnttZU6BE^q2Q~!)@g_g!%FLJ!>0R^JXdw)R!|kC zLxCkb^6icTi^VH^tg2Vz+rMDXVJZOhvia}-pCoOiQ#UcPJ$_H%E#->sd~CI!jR$yndmfEqf><)N8az~W~oybY?U%U)% zI#zhycfKLkwYGeDNW$zV8dSV^=G9#R@Nqbojq?Ispa(scUk&{9vt+aE@luZnIspYC zn-sT}p#yUa4GCFvs}wb^?I9n(q9&RAW&sw35^=Uhv*%3#%kU1-z)E#(0sSGG0s*0k zmC8^~nURiMRpo@HJ4Vrl^F!s~9KGRbblw7g(@xAcW#W$sUNix?vdwY{H`(u_hAGy? z;N485l;=w;wA1$=?Pp;uKSH_r@dkClfP4ZLB-JOQyTZ`#LVHhNh>*>A;4GlV_kp^9 zEi2u1W}WVco|IY%}Iy{PCvB|PU9%uRCXOF$8FDCnssCvXCsOGzT$ z1m3Mx3XuL89v!!^PTCKF^|7`HebN&Duh@?`x$TJvJZ;lQ{|cN5IkO$FRq?OBNA)*G5ZigUcgqfUYPjO=Y+h=MAMG z$7hCdp=}$u2}n71+j0VOBGk}fzrO=dsZ8#zc1KftuR_DRY^Du-y^vc;LUM}p&YNdv z5B}A67%nv*w@$kQQ-~%1FI3tT$l2lbr%41r9vF7xLKOk{!XP@;CCR{al#$)u4<{pd zwRkKF`R`#-bPx^o?fYsF0Q?~Cm9<9f1$ymyGJ7eMyLZtY(CTL%w7ZzpotZoQnG z2!#qz{N}DZ`9_pYrKEKV&1Mbt5Y||X;;`-`rMJro5GIBG_s7`?NTIuJf{hW;1A97P zGVK|ChJ?%#dg+;low;U}n!(nD4N{6F`6DmGzw>Uv3zJeS&0DXOjAyQwA_p5y%q~sC zMJ?NZBx48w0|T!6#CLfgi&!CsHGDHo9Ucy3ScPzrrBqije@Q_VOf#z&hNSnrVtdKE z1kR$^vh3mQO~Rc)c-6`9*arGp&&+9M!=L5E%}5tQxB>CR?6mc1{h6ys(v}%z6F#E- zzcuG9Wb!^(Taa#FKJ*UYLEaCAESs_KoIfC98_R<9Jg({fX+EX++W(lvt1Z~GA0O`x zoQ(3`e-ZsQI=x&S^EJnlR;6;PBQFbWm#NE-sQ-I0&5D3Vm#C#J_|qs4ld#odB-J;E zNo9Ivyq?fTPIHz?psWz!HfB;LwqJWk{ck8@%wh`0N*pszNs5ByMl#S`6;sR#tL`@6 zosc^(&z;vjydK4G1z1Y(?|OZIM%~5|;`6&Of?Q5V96AgsM^5~m{Bx~XqHpqgC+pzg z`%v?-{O!*M=?$OT;X%NCw217#rToK6(udLR1L#t81O@r;_LR>wg`+9D;iB}L4}mgO z<92FkwfrV(G%*&8<2t;21Y14l(D2yqkVms3cptF60>iq;*v&i?G@QW@F`p%|xL4Qc z`bX6qPJ|_iQnsL|}-;?jD~L_j=U#OJ*#vuJkQ>CnSTdm}#)&z#+>jSkPj zGoFW!_x^D%FLuS6ft!^VM4nS21@N5|gAcH_VDtA?o^t=xU`|-Q8 zOOYKMTL_5;pa0eXW4Oz7M#|YK#D02*>DJG4`s%jL9fq?4X0^Q8X~OaEwfBUylui^Mq=ZNg5=2W~;AzyinYEu2@?GnXXmF{%YWU zh)DuP8nw2e4=jx*OwV{Z91wz9|dS`5K$3Q z(C}x)&%uD)!ODbvq}6FJ3&Ax$`lvC+A{uj0L$CYg+HLx}dhG{lt!c({L8lpVgY-Oc z_!r#6tWwxugMjTwg(&!6bzMEWsd{UZW{@)t+|Vw7$dEes@ELyUkyvU&bOWz792wta z-&XwKYK33)Uj&%X=u1Mj@{(;_YTxo$qiGDJwHxFM)Iua3Oy_B< zjkh8ZoGN_T&R_db3jjzw`A6rr!F#nNfggCQ1sotCj92Jl%tT}`aAD8-BD}* zk%@RL9U~CH#M4oCs2>exU@lHRCGgV1WaW$(j{J8Q%bv1=)hVlWG>A24eU&luz3J~n z;rdNtt~wP5K^41`knF2I$X?J)YZ8>GBxSnKg@WL*TN)WRmGvI})0Opc>F&g1CpX7KCr5wz{J4U4LiHKsxXg!7NaU14 zxUTE%PNvrGNy5LaFE;VQ*0w<8s{Su>wTls)>{!iLnvya2ix*kTJD27srzfINs?iVh zzL9j0jjQ12TY}rc?-6IB*!n_&@`AF1Rdp;nYkjLVN()7CFsxdQBMjrYCs_4`m#bV$ zR25Sq%5n|~H;qh+5-q8R8l;ZigW?y=per?q77WTOFn0UxZpL=rjpbdO-XuwSA(cWI zprpkg(JN;$Yd2xJQ!??m`0wBJem7WsJZEHl>766ll{RFZOnc;GnV&8H^T)qI>xJ&$ zM;3qffH-bE%!N>^#tVG=<;g{kM<%CT0QO9<(r=HHMSSgu&oxSk4`x~XuE)P~*I?{4 zK=?$ho`JZ#02MYIQ@yV$RubZUU(+b?wRu0$lEPG50q#qfGkG_Muc#V9g%}qBTpNc@ zZ5AEhCAAS$*Pkt`Bn^`}*Q6xmGG64Iou(`+-yYxE{>8+nK$oOoFRV;NOu@D*H!)?G zz0q!;pjIGsGvv1O4axjqY28`ers4vVk6l88FS=_U$Q&>(5Zsh;G({N;+#wh$iQbjeU}@Oswd^_-ojZf@w8 zV)zofS57KrGWoa1$uPU2i()8T7 zkVah{h7ZWTr$oAT^8ST0na?|0QXp9TDGK%@4n&U%M$$msCN1D={QmxY$ji%+g|Dw$ z1zCRGiS3(tn1ljhI1BkwtOXrS9pu1NVH!&+PUk4mUW}k4L+XQGdSX?1L}IiZe~BY@ z3<;E>s!Z^yRA6t%8(I70ugs3I$o6H!_Ud%NEPZ$ht5pI87{_ww`VxNB3~x+0>vo?A zqVL7~a`pd9ajKFbUzq`k%udo@_1OQfrBl;~>2g>uU+J*|Pqn+B1P+56!*!|?KN^l^ zRCcYzQ}lriIQJIx45Yt?4s)uL^g~FLCjpw;!|R1Q05uwD&(1kE?fT8mwSMjh>`hPs z#n?@R24OEl61)x15L$fLaxFeK{rNJ%)XQ;QF7*QaR67$4QEzJp-DRNCNPQ)JiKT&1 zdTPzfcwI}!6oVhpBlS78S;74O%S;8HmX}_EWZ3# zJY&1Uhe{xD(|%g?KjpInMB7AcF*E+>3*Tr6?%6obk??Nl%EYu9WW}?gsG1a5PgG1-+l=$?T_s`oWy$Vp;*?; z5?#xJ=XvEi8LgqR%PD31YnKqiDl&14@$-cs%OAXyz(Y}IFQD3WrTXxm5;4_1lqj4U z*a_AkX87ZoyE9rkSafzc&rWhA(tU88Z zo^%Ucq9zunvBFvQ1{QX*)LjzNxT>D%bI3P}suy{F=N)@2Mtw)dEir~5LR_CrRYSziL8A zXX<9ov?4Onig}WEKk#|+3-Mr#4c>9qDHorscF`y7n;eGe9o~|a|fp1W#o-( z?$&|if3)EuEFLL6>*#bU&KVmGkh4X5GN(GEnDLKcO7upk*;jvX{r2bw!}&E&msF~l ztel#g1oV0VG0^CH~1D z`d-t(>I6w`S5l8SQa^;APq*0|Fd`R(ZWjRX(N)fn*_1_hLOMywYZ4_NwwOuWZR^kE# zf6B&ctN=U%W#veUE>!Y?=53UvEq|&6OiHN2?UVanGWXaHjmms&p%s*(_wZ$(HNXbj znDlwEq~@s1rW}~IbwVKvLlSgQ)=5HSA9=>%FYSu89ZklUf$M;H!%1YoS zICHG6KDlFDDs`$>;bK8tjhS)ma7I>!+S-YrW8W= z`l@3D$2N25f_`$MMcGig{9|HlA3%6uX`fWynNkM~C(xmcu=`L(!$u&$zbvQO!8PXc zDCQ|dm$On$rp@vnDojJw!BMZZT*!1`d;6^Uq%9}4`wAMyPN7^x$%U!D2oQN~J4`vR z4{NI2b_?P)kj%JTc@qkO6dNy6%t&{ZgO)#xIIW?YTh^zkBZ_)r`?QxXKnN`w11U28 zY`}y)UPW(;tzSjSw(E1abe)zg+YoN}<}gE$aUmQ?sHrXfA8T(N*L3*)jiRD}g0z4N zNOyP3=omv0bm^2PT#Z4fhD$#j3*m#eJ9!}^jF)fg`*gK5~len?y)`{b@-C;z7l|4QF~nNpDvtq z^=pp&ax8o!9hR}NpZH7!17dxIgE^jE?KzEx-f-A^#wC-w=X>YNhARy1%Uv_!tNHA!M>R+ zLd^uOkoc-x|3d;{>=$cjNpun)V9elrs-In4yUWC9Q|#Q#7nK`odzavEK>f>)0_S(+ zL)G6}Smi%f{?YrH>xS4+JgU6O2_b~PFsj$^>Ko^eF&ibsQ*DjV4{r8LVP8)8!x>Qu zOt2UWe9{nZ{BCwm2+_T6cu5+WcE6-ewaS{jFyg5zKjTAImeSQZ_F=~<)5}svbH6goK+6+V9@PKEB3nY|Ur(S0b7U4pk$2p!tPaXGnK^Ee`6ouCD%^ zoFGirmYlwlzx!@H(c@pjcTMJH;IEX7I})WqU+A%?wZ78l$=CZM>+Ty_uvmIrLkqu4 zd?Y?#u4@38pLY_5o0|)II-fds7~u)lS@y4(A@)Jw&vw8Xi)H^)cFJ)NVA)_0w`-4I zjl9fCt{Fx=ZVL)mD>hI3c8`sal>E%uNyqANDmO47dDXANY9sF#{blLl2hvFEur$sR z_2Op4+cM=awy-iwbrk@(DHaJDp`5o5+94_phK!9k#v_VEye4X^5Jav3!r*s1%H6m! zxS+w(Z$rv;{q<|wKDWEsN3qxoD5si{jy*S2-+(mak?oL4+$*KSS5?C-P<9Eq-`HUP z$w!r9KHdNYtG@2-9PcFFo|v)ufwe_j^HEQXZ18z8Rx?xbK7;3ZYGF2PwBRcchZeO9 zn&79XzK=|Co`+IL8__>7P6ChTC%BJ5zvUnO7-P|+ObX>O8$xq^|l==dt6rMtiG`7a^?FuQM8* z$ML(UV?gW8u{+I|tvi0v8;RC`L_x**&m0692o=JNfdjtcg8NOG<#>{e%gaf5qmMczH}5UponfbHn-?`CTW&h!z8?>xP57h&q`+o&KY3AT z5>R;aIb_VM=R~?O>ORg)2B67#PO||4)5}2dj}|-sa@*h}-;K&%^}PQE9rKmvkNcV} z=q6clb>~s8j+X}H#fRKZ5BQ;sSa$N9l(lgZi&_#hr&1G8FZaB4Fi}G^4 z8jAjc6eI8J=%*o7@uBLIilE9eWT+ax626=KJ&I{duVA{6U~lX0!@#N0)Ve*C`gPkN zHA+e$a~?`Aeb^tD1q5o5wi{h4$ytYO#xhh|*`m%7JnN=Dw^7L!?y;-QV#`;BuC%A% z3z@q(P9uOuuHQH*83!E2Pe@}Gl4G_q@3+uN?I_5rU9=~<-+$%js_5!OT=pyb!}qV> zf%5butopk83cC7gaU2CHfdXq8|7d9Zs2}K1D}JB~HRn*E#~u3WFnHQ%ERG*Z!q{A4 zsIN2|IXya}5xah&^H*imRs_X|&)mv9d7PX{OQv$*RCoP7UOZM|rOaW;SWs@y;ir zY<2^Y9>+Nck#i;?S=`_X&T;ajrA4H6gQYE`E=|?tPpkc#L;M^EMtcK;XWwV2G`_eL z)XpfXsU->54)>OX-r#g_g1%i%UYbARYx`q3?<9kb?U3uwNah4mm%%5 zU?miYFqjr6I!k%k%@5))P?xqhfD8$gR>4Vf)9pVXW@xh$MMZxVLe9||UL|rMRU@Y| zpf@D=R@PhYUM3nT`BG4ROM(KEzq%ZZqumq^--_TbjdFo>9fL`dlOcFl)nFsVAW&L8 z68?TXr-*gR*s;U=>Z3rKaMyNI{^6q{c1J0h(lq1>-4mQQC+9ZTUv+#_WgjZsRmdw= z*G#s0I>nr?3Xe`R&hFzf?oI=NDGRsyAe8C0Qv&3@B5^k?9NaCu%&~Xe{KkZnzJ*mAS4nbENRgB!`au2otjV*|-~S z5!C)}G~fXCZjs()A)vBSz8>I>$vA5_HG);wlNuc3ycBq)?g`Niqng&~*8Mvwymunb z{Ix=TDg>!nhO}i8-9Zxk$WO

z81hlS~`&%1B{n7{rfAlvvk7{VE+{F&Ty zT>*~lWLK~ZN|7%H~T!qODW<;o*6m*PlO=aB4dy5p?`>Yp@@B5hzFJl!~k|t~P=9=va*K zf8^}VzRRQ53)d6yb5G3D`YKEQKq&T`7kOTVR=^qO`gKN4uoV1bP8Wa#gg`2bYPVUp z!>qGQslPvj+u?c(Q zyDP+XJ0Torwu?VbBU5#yn=q2_uZpv{&|PaC&{Tzyo4?V-Md)XLEt(Z*S(jP1SGcSD7Px5MW5N=AWi1A1m8WD5DBUQh z*AqmrkIsGK#2gL;9mC<$0iZI*V%;$xya%=QA7pg&24+sTOq^*)y+AA}z0SUGG>a5H z%}UE7P$?CNINwgOk0d2VUbemU)e;|G-M32l0Ju~;i)xB9v(U44*;%m1<#TtO3_Q)s zwUo?KO?9c<6u!6dfB8m(M0DeCl~_9nJHyF`>=*RK#8C-!+7B5NdmK1oEi0Z$JpRmJ zAV$NWFG$D{OJQC{l#pg1vOqx76Uu9aRgXS_p52-{>5v8{p0cs1d;Q-x`cf&RljTX`H0K_^LW zH3#V&i2fjCa;=X1!HMz@a(hzka&uXo2=JlD$NT5_KL~(p5H1}u6TbS6BWl2CJDkBT zPIUX#hBhT0s;3j~I26^SN9Hw&zuJB+iPkk@GuA0~YJxn-`!yo8%oUu!nvuVA#G$w9 zv7Sbm_zbO_046);owa=`?uh$Ib`vVoZ!yTl!xj|Bz#C)2#lBnk9K$RLlt9~5Oc$zqijH6a-nN!}<7e!;U zTw6R5(bQLz{(=dnc6QS-+LyCQCk^v{=t|6GMSwc(V;r0WYCxP7yoJJ*@oTaa1P$(NeY6^~tqbR^J8=+xC z$qhjyK#D?}bke!}1z$4f*M&oT@4{9?qMbMkjW;Xh6h^_ zD6(s%xPwzoXxQo?TTwhO#RwPq)Bdfpr@J$;2Vv~+tsc=Q$P~`-H1(ZH5qBk?L;oh_W z%kn+uLoW-#mJCidoX;-1mCy@5a?Akt{C^-qy5W64XZ1)RpS~QmJ`)V3Hf11ss+^*3 zWVn^^)N#i?tT{ZVfnO71!1>xaGd};<_tXO>PUo29Via+WPY(}!77Ac1JgNKlbA=ADtL2A#EZ-k>LIvFmGC0_z{`vzzK%bp;E)-Vf-v{h~i>T_o=dyrjxieN(Ej{V_6OvH!&uW z&zx?I#lH%y8ykt+#(*#&GF@_^qixI)Dov__FH&Z1cA=MKGwt<>1ZfKpU$%$)8dssH z!W$m(PwA+8CMKl5jR-@E3hAz?Q8MBB=;PgEDjT#$!Y@8bY^klq6_Y{e{I!kMbc*cN zAR@(n^N0hC9t;&#OTRiKCh9#|vlAKUA8^50PGle)2W)^!V1OMYCgUckv zF#w12K%S!Y)U89i{Z!!L)YeOg;p>*QC$CMfKR-LJZ}azB73sLlWL6twZ7m13v?=*` z=2z-KK4Bd5SfIuSeaPP6XP!(tsxJyX`9Q^M0_8MN;be-Gv#|jklbgnJ!d)AsSjyhY z%oloUxj*ckk7{V(;vdB<-%vAY{`HlXv2&Tdah@1-SgaN;kX9O{P^uZUeHQuw*P}V< zsK1X|QpkzSU|IB+GyVsopl6xpiypHF!5(KZ!R~`o^E|4|f0iX~=6TA?dmsM;MAa~| z68;*ZA&g@jQWGi)&!d?XTn$@)+#s`)ELo%jPI~8Ml5La&R$XO?jGUk!7WOkv)fW?x zb<|MP*ac`*ZuL$2MV;OKyTX*A;7lv&EBAmb+bH5^M&Oe{7N(Qz0L_ zXasVBr2MmPA?>=_dc^ovhs+I32%*jAr3mPp>HzLgT^o|94q>34rBl$|#3LjN{lFoZ zX=Fc=X$dZl3*{Hog*{|q4S~LwHUxE})?YxWQ>=}SIL56!>T6lS>W0&VhTjdi!z|T+ zbi*Hcin!-b9~1Pjej^_u-R~%8QuegGyow-Tn1n!})R?mztkm(Bo`|Omz$|uGP^0GL z9V6bKLZMJ--~rT}W^zh)HfyeF|tMV1XvhIf>u1=0&S!oG91y-5{Ov<h;oE`Q99oKh_jQs@0MD4+Z)OPa$5I#`FAnMm);m@4jUvPc_Kw7$UldC7@ru%=ju*M=~I7<1R;Pddv`(UKkTT1LzHx%&_`I ziESc@^z>j}r+m6c_cx>lE#TU8G6!&?d87NUZn}Eph_S4*c_BIwwKJ>U z^224BoN6hX?#4}mR?(p>ajN#iV3&Z>>>(Fc*SjFq z7x~CP{>DwF5GL)EGOg^KB*T(Wo!#aT7iudt-B@NlkB4geoZsQa2})WDZz{v?26#x& zl91iCH%N3aU<_ilEQkhTFQRP!NPmi6q29bbelpIvJ$R;VXsug*GftTrUB(B0ncl+^ z_mY}PAy2QQJhg7_=TrN+AMW5udoS%9p|%>w1r$jpCd-P5P9BdY}NTH zk+M-38`-$smXIyd7R6IqtRq4cv-?b^$nFaZ+kn4KI;2mK(!4Jqm7hLj#P(Pj{{ug= z#F(~n=}B+BS(|%}xC7vpX7Ohu49V_{yClbp|Nfv8eh-z3wlS|Q9nwAf^ zkPY{bWuUph8#q1^!2l^dBUsy>;IwxyFx7ME*|WSAO&)tC!qkG0+)@)26H;GOP~x6H%RKG@Kb!vcs5YWZ8PV| zlo+0vD3O@xxa2p4vc(O@rO!UN&kBF#pbs&0X~+|BZZ^=fW6=wrLz84MQBfiX^h2OX z4_g|IDe>}y+7RAH&RzGP+F1PVc|fjNl$r&Rwc*Rx7SkQ~8dk*rPq3KAmiZV2wgo}%?jN@ z!y)p)NqtixqP&S5Ab?bmuq_F)=N9i4r@5$x&H-_rb5Xhw>8EmSV+4s+%v)=e2HeE+= z13w&2&o}mr**q86EkTjVN!0K+V8o<-P}dV1A&01AYBg3(5Y-}k5GfEV%$fdu(0QnEKWf5+NkBenk`Ww_2be`TLDT%N)M)@l6wnvlzxH7d zKmI?IDDTgpk(5fYHcg;63K7SCQQtgOMgYGsja2^}^vC^Yp9b!+q>z=B>MnfM6)525 z)n=bY7;uQ#4+~O2Xv1n;^GHAx5Ei&9oJ38%nOA_jvW;ic2`yv>mF1&eb=qYihO4LO z_E!Hj0)dkq*Vew*2azFcuLP=4 z(lm>)kG|shA8w zmT1F5XIR#lp~^XsZ(^J`Jw;RJWw3~AJgtmX;sn-Rh^`ubylOMbnxADk;VkWV<`nvI5 zLGDg7fK;$?8;OwdCqsrDsTOe$!N2MR7vn15Q^)Vh<#hf*Bw5G0u9V;2YyZ|OS!{mj zsnQZb6G@(^%~uYAf1Y}9X&AU{f=z!@O=FB1$lFImZHI@LKdh0rTKqBz)*oAYyRK5K)-)QX21{v8OWzuyCjP zt)}Caz<6%vsl}1x@=NHibhp{n^S7odpt3*g4-%z(zC)T zS>-e9tv~xO*_ij6$sGqG)y4iWE8|<`59%itBVL&m2ta;#8R#jqL5932{UJ^EG@P`v zVV$9mkSXeM_Qs+14@dG|ki`8E?inYPNP6XN#+)7VT*^JO|0Lxg(R0@p*eOdmi_@&B z<6OS@n*NBp${#@H4;%qLDzB(~b|z`2`6_l??xApQuI_M;k-K8UU+KPx+j3~KI_Nxu z((1ga;YTqgE19e1v824>+|n2Q;-o7<7w2APF#gw5L}pP{t2WLnBwh7h@<~I-t+VxA zU}yhzu<&ScNM#+b{hG@E=dJ#y4+I}RzFU`sbYCt;PXi2;zb~G;(tWlWkqrgXvad|A*n+6*Cf4eM#}b2Wf=?W0=$`-P`d9{`4~ZyX zz3?{)LxqF1LD6uhWtXspCvwYw*9-$LS(nXwKz1_#&WIq^OAWmJ-DI-Nz)GaVxZJo- zQSCG6Q4^vlt;8YCab;>1ZjB;DE@4)YeJ-ApvDh#O-o_Vec?0av*Ru5-3VYA6n7qRM#8G$5m;Dse{1b zV{W=h34`S7e(HSM+r^C)gCSZE({=tt35aTr^G};Us%9sPoTc3}H)P}G>akhNZR6o~ zggpQJpdJ4n9gy%1{J8y~vD19VO175Z%nCuH5Mpb^Z5ewo%i~g^#9o(U zs*-QZUH6uqjVZn6)t6WASbf>WsTesUtC)R%80%luyy6BlRRVJLqR23~fRa%+kXQ=k zCS`OzZoAJ0}XEB)Qa%oDiGbQr(=U z*a|YUME#I$;C~1ExEvjYho0LzX}O%?C7b}CT}fHLwa3ttDHZlL>e2yXk^|It@HgsJjHD3O5>0T_c8T@di3t{PtY#FF zPy#y0G0v^A8K>B1F?0de-&}d=Wj21+O*SH7_j7+2UrMUmNwnWHG+Kcoq+d%Fn7Rxf zaLFP=05_Ua3P3uMlcE}zt6HKA0gHl5XsV&Ap~cjp)@kb83~XfXoX&_b_}FmH=qSKX z@j}tFXN(94j{*wc=2}|n@>9Re5A!3SPa&86$)pHEpu!qyWYw!f5^~0rm-tZKBJ)>a zPOoOaPEaTEsMQr&tCg^D5EyIdlrt6^ZrhSsQS$-e2P8j`EaV1_Y_6P8p&b9^SfWmg z`?9}QQ`pV2I1k64`-#e1gW?`QmFIuj#X}GL1=`CFz&!qMJAL%eMETeLSKh25$(<1R z?_oH~96-X?f7}1Rll^br%m1;%H8{;jV>;T4d_w?Pj$_q(>RvvA1$RX0c(J@NP#*^Z5=QovCr1 z8&h0B#q|<)X=aTpz-2-Y=we?Mm$=i3Rbr7cQ_*fIn@CDJr@FE;-jf z?LY@RCHO1{ubybj$!<52!-7YYY8n={v2?9$_#dMzf{*V&>uH;Eg;jS(z-=^T@-xjVxV#=lM); zz8k+^q{n=*#bUYFXf~4oFUyhehB~d zSZeQT^!h-!yh58<5A-EDE8yyr&r+QA<@-;>>4MJ4?JEfO7PXqW=e%_se($lQ=47qh zvU?E3%ekk75rY4w=1dl8gev^_!uNat#$$XzljE7b=d<7y z)5ecnOy|DYU8TzM8yvCWDk(wBZSor@O%vxC5@gcSQYJ#A_Ub=7Z@6A9LJG6I+U-qT zTaPUb?fNiyw31_$tT^Aq)s1`j-L@m!9!sCr5HI`fT^2s*n0#F+ZD_93h7xOEtVSGd zpJ%DTvhuGuU-MPEhv(xmI zO^3tDS;mKnZ5_dfD+e8CbJ>+x2=vwp=DrRrF$y}TL)8GK`Q>((_Vqv7OR#_rg_xO3 zM+?~5CEGzOX4`x%5Hl)8j?hZBZaOSvKFeg$%C2JpZN?L08oD}x2~GHD8SOEvxd#th z)QFb5tR2uRWP*3|Cs1A{2%+ZPk{2y}*VRTYnP5#mJA2;Jw+kA!u1%AA*kD2iiU(aE z&3v#&D3k%k+HOR}ZYs|UI0BUsG*)bP>c8Doh}mn}S1ZhN`$-IIJ*?4Q^4mGvKl}9J zVmc3YJ=GBN?_{i+8OPauSl3(rDL(lZIvo!8L2obPdf{IwHIiSz4W=v81f7-LETnKc z@i}dX?twNBSG9t7);8!`S3~!&m!H-4xL8Q-C)ALFh)R>%uu$)MC5@Tz0H*Z3%f=%uiS@*dEa|@ieN6Cr<0{7$^I$%0>8m#-t$)h6 zD&m>#8`{tvzY`8%H%qTQ)}7})GJVVgRx+o}*`=54V}i6~4%FeVVAqnToAjLFqBiRp zriL4XHW9Md@0_oW&OUY2{5^?D=DFJ4!Gqa-QkmnL!<^UoPll(y9+BF;W@CW=8Stzx z%`=oJ9|j9;xU_XG7I2l4fXauXo|yu9=ABiCrYcEmpny?x3Z+>abS1;TUcKFZgkpq8 z%dV&tdrB}^k0hNe1)ddjOV+Ia+}qy0ty`uaxl}pmILR9yneYVkkjnkK0}Yg1TaT7w zO9cSODWKhRwxLW6*lnxanbDy*duVk_X70N_5Xj30H>$T@IhWdNb{gnzI%%+XGVFAs3Ll4GGC>^5Uw#oQCw*4$Sm}G(K9AQFmRl5p7Yb#7KMW!e?oq_2M4FaZe4iMx zK}oO7FQ$81;6Af0LkaE}@@BRSf9x(td-|>M5a0UTdaeJQ-+%8)5An_5Ej4O^%3T9G zX4?tUQ8ow*OBXb-uY7wwKiv>5y`R(K(!N={x_(eKq}6gDVt-X;zBc2nrISFkuO-F} z2NDJ7Hx%{o|@~} zY!)fk@fp{o^efc@Fam8W;PQ5 zj*m_`AT2|oj~PHXotyf9;CC2muVwMt4=3a>*Qn2InyIqf_m!bW#%4aJF|g6L{IXWd zPp{A3_RKbT`t^G6Kt7!|RL)mkPmYCTKR@cxb97+CY>ajQ>$2rjOo!a<-cDyvN5^f2 zMZiWK)l$IDB<$vFUm*qq&8g2xFJu3-C1KX@{FmVtAK2lk??T2kxuzPMga71ngs`BO z+)k#?XFCiHh2bPtyRjWhKK(w?X_7&A_%=djA;(fuQkG$gD6@AX z#FzUz87J7bqwQt-;KM4WcDWR?ZC0(|>nKCcQU`r7S7}k;#-7+W88Vzz%`G;U9%_2@ zc+FRm(j?w(^P=t0otcn6nGFYOpddn_Pko&xaJ`mH?`4<5#c>gK~|8201xLiMEgqb&bj*EZ3}-uURY75{r|*-H4mQ`d<=oaU9Z)Ioh;jW(vVwC0n* zd54%q&B^8VoNVZUmClb`_9xo-46@f3pRy*Wt8*1~5>PhNmE-$o6b#l&qVsa-37GUW zz!k(NhdrH|-{um{5+ zF+!ZqMxCpt58%J@?S;@1n^M@Io1A?XJp}4%b5V@G{W=Lqv_&rk`sJx#Pobkb&c}o* zCD92o?ZK}Qd3%>^^m1pdTs|{rj`y6ZvC!p~t&&Hih77ri~~`r+M!l>nDv zn`V8m#m#IVd3z&x(s?j)c8Chr-eqO)U=%`$cj&hapJjfPcIB8gvANO27d8nO=5b-!qf;F%Bu-#nPwT0f8 zV~g9)uaQ9S@6>{2ZE^b!qfO2b)%wWMCx;&bLzlll`9L>Rf=W?Q%>gb)*ge@Lb0WxnAEUYTvG| zq`r2l10?+Qaw#8SJ1}SF_WsuJUT!S!`|WngvBlYZrVltiZwd7J4fehoTuP;Hkh{FF zpbGYVeFj%6G;Oza}H~s_KYmA0!!ELb;bK8GUs9|Qi^qTJG4-7lnj3`F9FoNEq z?zNvSJ>%=Z+{CNg{AN>4!h_`l3$sygB+h1Stq@BavXW6=FR{*McRKeDb1v0!J9xqa zF6DCEejuKtE$6E;8D(zP1}sntN~CX0TYGz-xy3;ff2w-68^&fa2IjWUf}i7DZ}BTA z$ik>2AX6bmyt(5zK&F6w@D3imcV|}++r4;%=JkrU_Cb^Ne>TI(S`S_f1nsaGa|dVj z!4nWg@n)Zeg}!-{+c@N-<)IAvbhW+i+8mEA=y>b4oE#T3%NA(&c@Zs*P@(?^#7eIy zFYs&dd)G(%-qPO1Va{PaUspw7B(a_N9=j-HJmzM_9);P7y~8?*nO@$v$)NS0AXX6s z0QErQzmo5+GQQYu$J_JxwE`%O3fU;{`UaV^nPshxi)jOC+3pYjwKsHJYEPj!z1|Zs z!Anzf3mbaH)yJhSO&w*q4seBkxgr3Vj(&pQNqLa#YESp>?0?_dBnN;(|7q`o|HE7Q z@AW3t|GY`rFz)eMW4X=gKUV>aDiA4o=+>Hv7Z`jrNI#sG{SwC1_)5Q+g&#F&1L<)_RDe$i-1+7WDAJi>wqh5ZU5tC?5_F0!_E0& z-mBZ8WJm&>nV((K|DNKdec;uRifiln;6X))UyKZzmwWbl=UR{RcBR1(7KFXWQBXic z%t;@?)yUc)592Q_hCkOB~KuZ8hur)0C9ZUlFeqaPR zxwx={)8~@E)A3weZlf2uoLUwirvlu?zmhyQDE+BR!>_KNxn>G_nQ$hqXS7rVtni`w zH{3(cP%WF)pdCc?`FSZ{$Bf(a$fJAcC9exomD}A&4-&xMZ2yedF|-IidJxh4chGU& zxs3?a@3n|gr?!*DUVK8y1$Uqd*PZHyf%Q)}`woX%>A!vhb=$#-Em-poMqBpkZ~%LA z(z}u62kgMk9jjV~nDx2p>SNjInBLX8I>>&K7KBk1wY|sk7X+2eN~2j%YSAVAJ=AsrIkXD)^&w)eji-&2nLE$U7# zWyOq}eMkQ&Jwml@js+24AQ${j*fi6G4|xZ57iq*TBj@5qU|l3SqE&?{w$A3jo>!&c_~=HrJdNBejB%yQy&V>d|W1W#t;gJ7^2Q zF8Vz8>~DCK$IJi+N^g*Y(95?Qz>K#?1)XQstQl;1&PGk&p5w7)wfc^3G~6CNfPY;T zzuf~(pIK>(Z>{fCPI_5g(^7H2uD+;$?ZPcB(*Lico`2H6^FHfa=zlJrnw{x3vSx^N zIm!*8%7&f$+1=hGCTGdr?w*XaZC3A7+74%(v{?lo{^-CS&6o1VFPp<^b|g14 zf(Ii|G9Cko9a+MD+iB~FJKk;!1agu`eTA5v?ENO-N^3aE~-fj0suaV zyI-;P+?$@y9Ac)Ym*1_@w2nY-r_JfA=y3nvlJ35Wn0M4c-o0;s)6&!G)0#evNWUSM zC;B&@K>KF#4~_Mob_Fn`{P)%W%A5S-K#?gxDomUDn-?qKr2k#esO0s+$BD#M1@hf- z-jBT>0^@Y1;DFEWu`&s)Cq;KN&J@-1+*7Dn|gnjpr`0%4Q+3dU1R-6i5@C-W0IEk-u zGj6WAIqQQ+g?Z580W2A{A-=Ijxn`GfJ5=pCi!+2Lo*PCESZ{C=SmU{mYuk~(-#?bm zgA)`?I(S+JO+1uCxmEBIUj{R?On-gNRFXTABCu*_mr>B~ZfG}s2D?~HTNlYV(U%OG zlIY;(;-ddu6k&bqlT%(-H{GX<@3u+=^Sw<>9vb92UkErjdqndl#2hbpmA+!$8NKMU zWHa4eBX+Ygw^u(8Ma!JCFWouc$UQ+h+uL(Un>V?yp*vtd(J|LuS9iFo1ApqM1fG0z zmAU>oj`#c`N}(f2d>gy4d%z?k>AV{zZJGTOEKz=S&F)R#%Pw&c`sOR2MmFBXz7)3Q zV5z0D1;Yyey3@>t`JJ-k+Fa0@n>*15_|JQ6GuR0E+755RZHD$?hYXQNFwAYG+|s*S ziH*z{lB?5rZMxQjrDs@QM{YWlO^w~nP3?BccKP}+{kI}b){_Mf6b940!=mPYT`q?8 z?KzviIY+~qS4Z+q<<+o5>;q6`%<#CjOybrqXX|fR&>^OxqQX4nw4wV7ul*qSS`LF& zX(12Aw%tx?Ye|&*s))Vks%Sk=EtKW;!!lMVnvS}b`?Kakvm!nGyN?BW1f z>1|r27o-prYsm`Fy2y)vBydoBGBR6H5g&XL&xcvCnJh_cn`)qKsctSgyx9UcV$KEO=Qi805}izo)lDqpgU^IYo4b6z3}{JgoM8nFq7xnp;v)5 zq8G-rEE|g?gvT~p2RN|cD;o5&2l_L8;NfLqHl6!SgN7S0e_tuCwub&&ZY}<_44z#| z$rN>aJ1M;qEq_j_pDt(!hEf z$rKGj^0w?Z-#(VwuUh^1AsN<`7aU!bSCfZ8)E@#Qqb$t>eZq=Bzib(m3e(!9T|&pyhxbKlZ_ql};hX0@E$8Jo{y0E}T65Z_1Ez#_hTc3m^EBbE8}yUDDe zE9NHMvM+7(D6IWIP{$|B6`7_aGtr)~P?%wHY~C`cD)#EMaWmK@VEn zF(bn{u>3CM&dKoHZ#%Dq44+=1&O1P2k$v;hWFm)|Coo9dR%e`kUkAr*hKIq6Fj?TN z4~bbdQ>xyrY%C5@@tBI|xcr%un`$$Hl5+Vcq_4wVI{*V$+-+=SS>jJaN5}qDy6aQl z%LfIzTUmT*zXKZ;=xjy2yds|P?$#Yn8k%DUiVG=&CnPqY)QOC>@@!P;ekHohBs~x4xuDBOu^&LROdAB_<&AVz^Hy=*!-V|7+=04 zDkgS|9>6a6p@XKeE9j8H&+`839oo->e_ z7n#xd$VsArz51U111cPT>Q!l|`04!TW$zuHkUn6qJIxJ=co-8SagNu49j-iS0*r?2 zp<7HthwM2pM%v(X$-T8?b8hM2O+I2BhV(*MGk5ZKHfmRC7RkcDoadeLd|(3gSxQ3jof;2p`)z-N+6KB>_rt zCZ)UlOx$F@3ZU{2c6uy%@p_T(f(eUmII}p!7;K(&l7gfTs)t-*L6hy*dlqGRPvIT6 zXI(t?8S{=WK49+d+XEu5pWdDyH>lnC*-cq=+)0J+l~m`lsmG+foLnBF-Wz*M+d~*!vtdDGVkIbYu%QZ?NM##=77d4ph63!wgo#68*gM7 z8p--BwH^c?1Qg!+?KM3^CSB(zb{xcsOIZs zj%gdO|I=&dvpyZ#a@VIt2f;0e9}OW8G-rMX<(2e1_^)YTdDF+piMnE5<^@yE)4R4p z*UOCAvK!A$V1It7QJK5j!|tGkJDVi_#t2h?lwB?(3N}y~@H3PSqWe9>*JhN&JMW(k z)P4u1et*2SG<0jMV7yS9Uh~6HGLI5|+Zl9O#1)u+yk-2Y>eYIH+7_xb`m_wz>I(sE z>DY(#YyWKN08gUlEg$)Bj>6a${7^wMRMF{qJLG^({ZG?$u2*SzgK-l6T^E3^i7I{8 z?lU815qJ{xPeGW*Tp99p9A=cUFH7zok_Vsaf%;1lxp;5ZXojPrmM^AqV&tw4Jt{Gk z9lj4b^x&`T?D_zU<)rKSz@=vKv7za03t%>F=iu$PVORQk_@!^Y-3Ggj`2926yU{T- zHVumwftcB**TdLJ*NO->GWvGxl51sq;AHq3U^%q>j^na({cZzTvROHq3yiz**_Pnjg$m;;(mZ*9V7K!Kqghk1Dm(q}N8~T_rAl^+3Sa)2lG-(5;U)uxnHXoh#sb)i&(a z_f2Cim#HX~Zcm)WRCGMgQ853*oX>LC+t~9ZAFN}Asgx|#&hB$=$-ndGMay7frYkeQ ztWF!6H}9QG$zmo>()VbUagcyQ{?c{crQ;S4ND4j+x`M5*k!%c z*C;u9soPx-t0P}8@Se7o%x!jz3>j`0cFBJ=WmQ|B!UpU5T-x;dI0)POe^K||VNGrA zz9=pmiV7}if`F)qAWfu*lxR>8P*8d=(!2CdP$?=>lqS-YBGRRV4iS*9Af3=69Sprg zAUSVXYwvx(-R?f;e9wKJJO44nIcMg0$6J5zZ;UyeBt}eS%gXfp$&T}dL{y-`Jjrc& zzGP^tkFX$yAEKhlnk}zxuRi+jlSg+x!hO6a<}?yjP9RECBam-Xt2VIqJuP&@SuAd~ zVOP?iw3aoNK8U-o+j=cQ_ctGil^mWHi4Q)=pJ94fPN zDxWsAG0DjVS%oTZQ=^5g`7kq;ldaK5s`EA&xpVxKTW(y9%)0`-umP`@0~CMN?hUV1 z%TE{&4f7{g`@$<`A6$rq4Xe6(qV8P|ax;kROgHe&ny&OBdErAgiiUfF;%};RFn|-Z)`~`aRz$YjH2}}!{$VSzwIFPh}ULYg%8Klo&Y5`ch-$*rM>B- z*X!a5Z<4Qw&le1S2|7u^==xVBXxz--CWd=ee+prNi3k<{o#nOQGcziC-oLHv*ISsI z$qzo5fq?=u#!VMunnt!}wGA$BVHbauf7rA{lITg2{A4;`h&uo#F77HtYR4d+(I~5} zqc!wzTHx|L6TNEX5Z8F|bF2JP^RkU>p1-83*}jdx)GfxH4e|9DO|!2iw$u=Oe>lt; zonWRAs5I2*GvQ7jq-f!Ldo)rr^KH8lB52l7)ShROgj5r|HgDc^Du%i zF}~Wivu&&-F^GF#SE^F9;GRbjdFVve%|HIG&qE@7JqP^dA_!itof8?Hduv6v>DQqS zM@`mG``}3UgPS%0$y_W1-)Hv6O?Y1@1w~pu=09p*gy6c)BW}F7Dk58=*H47Y5S`A&<9!{KY#i$u)<)TSV$b#hZ1CAc%$FC zZ77GD(6%YH^{ejm37Y-u{QhJt^B>(cNc8!2fN;_=z#!hE4Vu28_jH*@!Piy&evkY9 z?%H0rif?E4kTbxXXgBA`n^)l44+aSd2@fRQ6E(Q^H`mV5@Cg?aoth)#PGcL zacV`e`L*rA1z6IISzqQ%%Ei=Q9F}0&Xo1)E#W1Pj&FZ7y2ONp|28ds8WZ4EkOKTRnH&^CNDa)Tm>a-^X z(P&q?H_Jqk1@Ssnm0OJv(9$CR+9BiO9~7@8(`MIpw-6*#tzL&lUlewQE=J$iG|!yB zYUUZdl$cwh-|!L1Vvmu(p<_l-%y_iqUng_dn)>K+Mz1y;a)V zK5JjKhw%Ry%wOfbQ8vvP(VUdzDlZr-K^R&!@SdHl2})?6HdfDGy1^QD(cf`<_m?QJ z>)|l`&h_JdyVv}7kAZ;?^wd||8LBrL;;IrQH{PY-yhzWf5j$J3FXf2gBLkfIB2IVp zs{hZ*oZ_3y6G;*qVW+^UffaM zzeXyA9*B<~ZCu#ssj@5I?#)|Rj=CK<>_jwF2U(Wj6}v>m)nL(6*fHidBp z_}33Y$D}uXJX;HN`-UTvPw4SW^jp@1QhtoOs=@A0&!=59m|Gzads}7ZW_oVs^ikox zH@0gN=;uCLGQqxG>+>ap3qM401D}UoP(AdlG{?s~0fA(!vKeffbPQ;WU&U44xb5Zi z0#VT;H#8*t&D&W1uvW*-Z#p(~*M{CRdnTsHH-!b3&ie~l&M-dik7#MmEM~Wh6L`{a zc89(3WIW67sfgCu77>vmTeo-_R8fpzk!{TtjraaWht~S!3(Z#_yuNms*`eM(*6eZ| zuSAuv+_v`<#cqarDPIn!Lx?F1Kn2mgc05vD-5XD&wTqcD$u;D++wG zr8{;~%ti%%(f)C$svY!Gi{FVveq>uo6)X+DX&2I2;<{8-5YNwjWyBrN>;JRJQiTQy z?`zoo^*k*33iM42v1gQs%?50qjO;?@8`*9+pY~12d!{`WD+vqn*uo=hJ=t2<5yY(s zffJ{Zi>RdoNQ<`Ykbda1!Qpq!Jo5614AtZ8&e$l8!{7kkDgNqe%7#k;TaRy)Te<)q|Tc1?kH{=qNAb#_vUB$R!mk_`+Y@Hdj z$eFKun-vqas~WkPm<4K4_r%RcX3`ftpZG=pjoks(K3$i)US5rCcBL6PIeYK3a<>LF zHW_cc%|h)BqEOek@g>yaaCN(0vCVb}sxqzk)6_`C!oAC9>OIL_D!&K4-Q7i7R97`} zd0cLQ$c}}*JaDErDclH~tVGp{arr5lN7%EMKTU=c1)|~CL#%`Pljzums$TKvUH{fQ zzslK3jSnitzcl5**9oLYdym}srq>>K3t`1L#vb#9x=#H}Q{zc#iHSU0c0K3MH@~qw z@36IT_N?Ah5{{B6mnAc8a{WSK8XzYK_?sA)Vxz zsVNM3mKRa?=SqKFdTw@hr*@awrj_)L&OR^vd^_N z98y+OQ|oI>yUTUgV&+WA0SXbFpFu_kVKu(q=1>JV)c)et$y#qBIkNPKh|ldrNup50 z>yK!o1ZqQi43?;OC-Vk*$uzlz)!){6F(S32j#Joa#W-}@B0-WtYORD6Ct?lAo-C`6 zO-3#AEqL)0I6;QF@{5cWqC=|CPgu8d9!)0>m5;j)^aYYUKCU!0jK+kLB$r&(z-`+ReFxTnVj9x*f1^S2U8H}kxxdQy#>v#Y*q zy_$x;Z*yl_Yg@(A7|_X>*w=9BNm!5c!^`fK-R#u$GB~uJM5ce64b{RBd|%=_HsIH?d zb#(ix;EhEXw{LEJsLY(ID#%N#HJ-oKrG)UB+YICG za9Fd3*>%`xo@*!1uE?kM3ij4XNAj}=^WCtAXRmhnb%1mCSHIUn1*ZFUj`SniJ*x4! zZ(s$C_h7Pj9U2$PTidGWd;3HW8Ij-SUYn%zJ+{!h{?(gMo6kE`)tTkT+1IesR0!CX zzQ9@g&7=Zf#cpRX|G5+lAF6cpQ~AJvJ)3LOjuIM;j){p1algZSJ32MauGDUWVG$DG z$neCJbg$|l8s%>EriIJ!l0=@l%%de#n1uEtt$c`lukUp;8~uk-SX*fF`o9x1mKklI zCnZ%8LPL{x2seC7%(|GX&S=l@-kMRq<0<><#jE3#l!yZdZojAHI&j-FuT%W-;SA+8 z?t2mR+MexZgw@6|scR;0Z@nE3uQX^J$5x3uO;?mD2qqj^~EOSJ^5;b-F6I zxnfee`6BTxhkS3&nO{F28+4%YDEooRb%IYtN=k}>x3Gl7E^+T@#pYZ@v#G5%MtG%g zRLu65goMOEv2W{!^I&L#{hphr^jQSv72O_2>&sG>+?hYo101cN#@ z!L56hwBsFn((;hEoQs2%DJr8(KNaA-guE!>!*xOL}c4NxBi!7dJWVBb!ZdZB$mc zvcbeE0x!M7@>qRuuKt_H^%rHsg8Tzr@dNdt^=9(9B$H3`XkyL<$6-uGjB?SvH}Z5> z+u}&`@aD9~$;M`Fr0Q)YThbYa z8$wtlms|OFuSVLhs%MQ#2if4tS`#O$A{AotUdIVo?sPXFrurly=djW}E1an@i1u0C z5x#1QEPm4QmX<1e@Jqfzdgy|k-3GTvq2nIsb=$mkd%8Cj#Nn!EAFu`_r%k(cZ5MIp z+1?MgeefRx5~*0=p2k7 zH6Bh-)JTy*e+_()u9)afXiIoxT>VA={Sg0b`ZG%YF3cal?0s#@ zY48s>;`a2!V!H3{qQlE`e0?avt_^LH`suP6+;G-%-b3Qd$0X^xT5oUf;>CWg;`?W1 zHP+aw;UFd6dK($}&4-bq@|o@mVIWbSmOocHgIMXBqnW>bk3$%C_M(uxb6YhT`2vq3 z=@q|DN(pZHToep*=md)FKI)B6vm7X~_S##-^kok&G5u)V8~w^UIF3i*7;nxMR#g;Z zwQFl@b&BmaTd9v`AXR;x7d?B4iS-JlHYBY-`$EX1A#6<-(8L5IYSz3@-=zr z2WOAtd+jrIIKR3MtXZ@F+lVG{dS*?NE6BrCOeX?!23zbQ0 zZ1E*izJ(ti);wEYUWS{T+ikuj%MkU*l7ZBYa3g?|$qNgsw~20@#O!*`Y$S*WP|Xzw)TElm?mJB_<{|*`CynL2fUf;S6C~)-o}fNakoQcU_qHtenPp zB2X#tl<3(9u%2<5=hC-<&+1t|KHk}j$#<5=CR?I`8nM%Kn$%^5#(g*FPu^xOE}}Ve zrqCQm6e~2lvA|OKN|)4pe-P~a?!+sUTkP6aa|DW5R^dHqRi?Y)8WvB-=zKGw#_)?I zJ#xI?lcF+|@bf;2?tB~Y-IO|?}f7NXkCurjYlkI%^GaZt>zXMetLi}k=b2Tk=2Ol zMC1FElEkk1TXbZG6DmG59 zsL;uZ<~{e&b84bS$;=UqCrY(SpR^!2i@GR9ee9%dPeDB?5cR&x(dN#bJ2p1i>-kt4 z`ec)V)`MSzP5AYzStuwbq~zq}=BKK5#7>wVZy_D?s1J5iyP8|dwEHTKZ{aF+sQY?w zVAoat?6=u4q zGvyPRaT8ys&}GN>3rK>oJS3N53Qv~ChbyQVPd4NZ-;Cr-aQw3)gt-Jo-2T#GXw!Y; zi++{!g0V%urj5$6?|JP%Bq=7YoHL@JIDhozix<0FDWd_$>96ROPlVT`up)fl@Ae=_ zmBiVf9ccy)D*NhyTs@Wxz^ z8s%vwredGL#CDkzurJWu%;I8YSBi1E2$J&o$WmQc&_J<->;};-WjMmeDtWStgH@O+z&JM7BkCyij_luKt3sUZ?yKXv~+ED zSWQ%tQ|o643ExUQ&RhJ{g0+SN~i`Vcv7TD&P_kVoNrde#H6Dq@v#xv)FO7;L}k4IfB$cd2!!o` ziQutELR9!e;j%Pj89BrQ%9D_A9T!$nQ31YwJ-^1V$vcQ*im>e|2a!1IzP&@TV>!- zfuxK=GU~5?k(YAQa){C3sqm}F4xggyb5RREw3n5`?N7tUt&n_s!wOHo6i(;cQ{WRX zjHGo(yS%A8O@7QDJkiqxly*+BidsZ^y1ALB*UE(4Xl;OLfyNLU9r9aaqYe+f+or}z zHrT%=QOZaA^TUtdBj1FF8v+O;At9*JP-xzMoc7B37|f1$$hU8Ip4Wso-#&KiSaS@2 z67r+_Bx{j&j(#4jus?m8T~3abDdh6?g!ASXfCUq7?coIng5*n_?MkN)PP_N!LhS4k ze7^gNw)uPF4&3?vWtGdVl*f*7>6Sh*+vt{-mUg+R2j5UpT~+-YuDt!*OU1~@=;h0o zu{;+gBq|?%M8=l2%YggP)z!V+`r_5AmhZotns4k=mHaM|B@aQ0Y!o19lVYcjV?z1TW{Y7MTprE)!mK_DfJ@PPMApdyb+s+OEef;`lG&Q%{ z5v1Od#lcYSTX}tp*u6c3+oDFM>PP9I-s1ta?4}aFyK6B9zJ-!Lo5P9nX;qsU7cN`? zdz11|==yZBHHzPOS}>g92^qCCX^P-%5A?aY@qKpnRQ71=au@L0JPSpcSj0C7*;-WR z)EX8!+s{dPOYrfH>?GAc&PSc3FXVWXg(hy!Y0a!m;@>*oB*Z+e^t%W{IPYWLPQ11d zoFQzjt{^Y(sCx2d!ia^CudqwJ%y=X}s`~ap1em73GLtrIeJ~7e(@HhCc-yw0`2K+)gknhugjTJX|qN zc#qB$%X}MGvLNBZBRT?=|94pL04#9IugQ>0S~BRfY^W zegldPf0*sXE<%bA!PjPZ;M#1xjR>xSaGw2IzHNp2k*d+E&7+)`F5OIp*9+G_)~dD{ zZag?rNRp#%65j%fs$ZWRad&%Tfv~VJOjukeq}rUtD5oj(mr<1I`p#|LPE%(0x9I;e zlGch!yy-&5)xJ2LRgdA*3keN%=H@L7`|1Z1*FLU?l5pL(q#nD(sd+m)&srMSu6LrHa)T%3>jUBXM8|HrYu)usH{hZnUS=+4r}YY1T$wd^$c_4R4GVnRW+ z_u5Q{pP%2aUxow?dM2iuZVN*YV~5L}8hb4SV?W)GRZhNhjFXcSg(9hCsbeu;cpUJ( zIVB^WSGl;FI|9hhVfjd#C;Ie7nW0jL@v6-cz|wS)$Xej41)gLUu^lMt&CwB0{p7W| z-~)*{VRNxWlSd}y+O~pm!)vY5C!?uy5mw!w^HC&X7=oyIIl^aaH0V^ubz9jmRu*Ba zZsHyxn3ij0EX%YtS}*@`fJ{n0iae9I8o2TdFuqgYW6PYT;b*!Bx|8ox1t71%2t;w- zUF^bo6|TJO>_+1aA;=f{Ri4-ZwDx>wnnGh38^Uhr2K$}CV%sDM4?z@hH~uVRCwP0> z<9D0iag6%~pbs0}xn`oMc+V_bpESh%2ra zu^$=AH*Wm;^=qP}7ps`#w}DcNj^v4;%Yi4(3Odg`9P9!K@M{%ZntuOtswNP2DXc(g)vm}*Z-T6h(7iskokh1=fF zy12{lBHKX`>lByjUXZZ&iIu}vE#2@+pkg+Jy{x?>+1;;(78}a z|87=@M#GzT4<0?`?BI|um4HC`wkJxOe}8wHfdLfglJ)q;{NVJDxV~~1yqMF}7J&fH zV+W>?26^bnk-(rJ90p}T`R3g_=;PE=G4N?_&OU$sysoYe-;=dzg$@i1JkGu{*=lah z_3(oq)+Q7JS<`RW)Q?5X(FCR}Ny^8{)?;VQ5_H@mt1{IeeEU_8RTT)m*z)_u9d@nTLubW#P)G^z{STkBE^$jD(eEPfr*4&u7xZer`r+=Oj})Ne%`j?P3b5# zta%0sr^7Ej_{FHsygkvj|4WVQhw)(k>U+y|4z8}bk5Xiuor`FamDA*#+l}5ycsPRG z>M^67^F;gMa=J5Tu6u7SIZn15V~6~w*lzeeRK>B>6cnl98?@E|$TzWVIZiJNlkfjR zUt&N;a;sU@zoH`xuP3tyQk$bE7zIGvCJGH~O87RHmro<_$jTzzm(Jh3d2@U21K8&3 z&N^Nu}C;tUeg|B7##%1hn+_^CK7%v4Um{%oJ_iSG+K3&K`R|`c3_DI&SB& zZ!ZN+TZYP<(&4m|OhTnj)9}_G(@-{q{^y768EB|2&a$xV_3GH4CDGB)JQ;5|If(7& zCqC)?T+`Zmb$l;|OQoT%&KQiZF_Z-=(FgB^v!+Q&97M`5pt0<`WnWJR=a)l4}etI#ZiLDU_?Gp!D;I zw|%t^dU^2R!JO685J=1=<@x<@Jo)t%`U#$EGuNDFej-S_PHpjGfR6C$6rZ4`W)re_ z2pk_yr9_^HX3+8FdIp1Ry^7xQXcs<*j^w)mG&&`A38MD;B;p>te8_{aF+bTFBP%1L z%}Ghg4&Dr5(=XuWwt1L}9*F#wpfzN-UgnrqSXg+`m-UvqvXuAM6I``VmFId(5&}bm zyd2S!XFy5`lX@@aWQIgO>$W<4oEG-Q2nvefR%;pf^YrPDfCAzQ`m!|muUwe{;|>W9 zrlO=2U_1e$0Kee7HuK2P@YmHos~mFXO~`n2ur=xi6j>A=Qo~C_2G3-*;G4dT%>EeE1W*2d zhrtXd#oVLz+tb~ArhmR-j$<`&Qwa0f{oK((p(S_cnIG>NSbJR)5~EH zkC~sF`%ZWIjd+t9cMk+jUowD|lb@gOSSAsYA;qB(y|%Kl1MLDE-XLnWT;;S^E`bIx z3sm$xr6$d9Z@T=xAtqKUSmiH-K-Q4(fOP4A=R!2PXIS>-aW=VOP znb~^2OC)A%q{^$KqXW``BHLU@Qy}2wOZsGRwS2|jC8q$@J_J6v8lLFs)2CMG9neK` z6zk7Vndg?{cXV{znMnzI_UswFdK^^Qb8Bfdn1P?qpn9nI7U;>wA`Vw#Z-DQD)aK~n z!vLXiT}PLfmY!e|vKXoK0A~w#MeHus!Bar!CLY&|;?ukNT%o+Y{1l^rA(#&ohrW9| zfV;qhU1JIXtyJ*3i}yGUqXy-6U7h-2@DYVTARxN{{|^}n6*~ld-C`RoZQ=C;6t^5R z6!R60|M-%Zw@4yxBA(X>JpT3~`EIa2Kzlv-oS=Y!mnd^h%`*7K@^bm8 z{28UZ-C;K1p?sHq-DU@d1(*Z_kO!Cmv^%ko=WWb%T#WGflm&T`zP^5(sQn0d z&#WwKP} zP2=9WB^b1ABaQOC4kM6uMBcmG3F2;wr+?BNub*fJTT9$6&@)vAuPp95_b&O`JXSJg5n=I5&*dcp+b%3Z9n1*XUs-@bjL-4hoRTVKRMqPWcz^2u}kcklB*!K*qs zIYn?c+lYV*DTO2lyVGz${5+)MenaQy_wT5us|yLDDEL8$xKaEDz5oDSRnPvSFC`Tq z<+G#76Afa>)hhf1Mg1OO69ua*x;{@s?3NBXCHdqRg@$@PUU6zE!m$Ag1M6O%#@=vU z#jK-{uE_VE;a(W2!p|Tn?Kbzp?ZIR9oK(R0)=9zp)9?{Ix#)1@poZSPII1x4E9kxq3HZWpJD$aR8Zvd`u_zX z_-~7y{|;LC-?-2O^VLv5^AwTRNKk98xPGYRaXv%^9iH7wFstG7a}11a@nK?KQX9MV zH!5tG*Bt?t$B~}giefqtmDRVfStE3x$v#I>)N$^8TLDOQb)V7{X^Gc8!Uv()HD=ga=NY1Xsg~* zcj8+8YC2SC(UFlw&e#@)nAf49#ddv$0-%E?YA_~>TbXmG85f58^hx418*5`}r9spY z1z%8AHB0l!96z*+b2ZUZ%)BEzo~{qQYf1KYvnx?9e=w_MX_>bT+23^Y8qD)fyKdfo zjD5F7qd-z|x%+Xx&0uw*Z6hBD#O`M))?d!x?j4!S1~zcz51frkwX(2v3f4xy$V4k* z%04uR{KDCcA5(K>tH>8_o|}$NE-p?kKIv{s?BB{(QK_2uvwALH(vjrj zMANOx&j{THw_hYo00Mdl%oq%8v-}n zCo_e?Q)!oY2GESf7*PL+CBRPC;Fa_eTNpMncFbyO?ZJg@Cg7wbHt4B8$ zBAXK#B+M5?fsZgeJiO!F8EJ{L#ai+HS{~=b@n~L&XbwEZ8tH1=91svh#eST&$sV9S z!`nJ3A^X)w!QIn33LlXHHUVTwYh4`-6WDc;9KHSItumJa5jRYbYYr1L>3sSeLCJnu zX?LZi-YK|x50*{serx{qOBVU!L0rjkNXPBg`X&*@wj=LqPL<5}XQ*fFpBzw#Av(ir zxj_Z(hpUZsK4`SFqV#W~CTq3Wzf}>aVHj#@$}IgAc(|{4u`g_jQSP0lpQlt&JaOX4 za~kBwhnh*+?IvGD1}?$D!O={-T7L~(0mXT1Q^M)M0pTI*z9;Ry54N_WsvRePf2_kq z@jR+@NC`y6@99yFjxO-|mwMo%X_6JDL*>&76mQL+aCP6<96yTMav-z~<3(*=2M3pk z$jHhL($Dl)CqO>7zMRpO9zX+#^mFQ?{q#Sr(A2C_JQ~F|Oz0{eH%Q_%v`@nWxSL{G z8RrZ~MV&BPj$_R@rG(wRNaRP0{tv{NM@if6UR3NCE~fa?R7+L!_l&lvSyv~rL?%D_ z?Wo-Q;N6d@r57(42D?7aP@Ex0gh>1cteEZSU)7Qr-B*Y`gt!y={YWy{DQG|-! z(;2^0|E|5NQev+~L*5u~XP1klq^1r-w8a-Ga&3-5C6?smI+<7g03G8{*;sNKEjD z?0U~7`VdsI(Y`-g8bX+~!$dS03h9<6zj5^{utSMrls|Lw2#RzZF^7wAwWZ7I?Vfe^ zaXq6C*4mJsD`Y^ye%;}Bn1Dn zdftt>Hxp`}J6)GUFpR5Z(44G@8*FM1;zVdwY{T z3W2~7$)zJh`Kb7QEQ^k)u;tLbG-a-+mcWz0?FTED>x<6&FRiT=jSOpLsCvSFK085V zV@u@_J!jNZsv1UJ=?ZEAP&TxY_W|=JyE0(SRYHLYJ#lwW&zt2maQ=kvp}@Yd`Fx=VQX_ zzkRz=9yl$;6q2IQ!~yf0q5%JKi*Fxug~q`D@sIs|VHz9b{mWeibXnRZc-3^@amC@0 zkt*{fmX^1fCLM20$)QNQP!Ye8;5s)n{}~qhQ}{_BV;BAeeHa1o5j#QE&%XyzD-((_L?vjrp59}5zd+^Jba^sV! zsi~sc#l&A#ZbyB0KcY>y%%S<{QQ$N<#lwF$r6W~O&M-w(in#QgJ?TVIvA3W}e@bH* z_1tjOWK`wCze(dwTVyI~VFbDZnHnXbIcc|_@ANj<@bRRHlZGC^mu#Btz0jGHwoXa7 zN)8Guq0WCutExuQ)DE_`_KwZZo91gmJFx(r&2ab59ZGiZ$|_sl>S`(aV2;a|d)xNO z!yd9XpWe>vqhdF4jLo^5@sN`J#1ZM@==J1jom~?0oH|(>!rdyzOw~*UwRWhmciS=m z_@!v_={$CVJ4&0&l_i~{Y<>-XTVwY7;l;(H(APpeHQcSSS9y(%g&(D5G)4eo@ zLr_ru<3o3McN1$Z5hO zuWL?4_m#Y7)v(X-P6!ic`tP31#2ed$QC#TvB=JGwzeqsT?hNwX{ZHT-=7!zQUl1&C zUw#aG|L7BnZ3|K&i|mKYa!i^F=qg+nPlmm?@+f%=6lAXZqmQfVO8Iz$Emf?GqY3{srIv$JW>XzJ|9fEj^L#^W&MCo}Qioc!VNcp=r&tD#hAe zF->_Nj;Ej?{j&_nR7k*I_D;5bMKvT}n`{zIz0U-dn_GIUaW-8eqx*6zOB5XDc=nv_ zN9^Vg5U4f}eMGZK(EHBM6L_rCDHf@*rN8Xz!rs>>ge26TXV6Y&Z8S)45i947m`k(0 z3F0qb4tO}39b*sl5$SyT*IeA$dHQji`9U0}LzQwF|FI5hproxWXfVV$)?A?zP_P+ zWVBa&T=pdKIPn#F-9G27areAvF1yUq+;x-H+=mUe`gsFlT31h|c+mU$;UJ@J*&I z)YJtzO`~)WUq)uHo3m(=%e-)W+I0>txpeEnot*c=nL0`ISGBusL^v;76=DSB=z~pn z{hOn%s?Bv}r5|jC^C?!IIfn{F@{Wrd56}*FwQA(j@@ju;ZM`0sCEptOAda+)AE+J{ zUl=MI|0)Ql`&Itp$bGx^tEO()Yh!(W_$nb&u5a%s_4E(YM&W2uG`CjmQ&aMNJ?_XH ze6E;Em?Fcs~ zC6dN#=RSBSFz`N6m4q{I<$~B>ym)<1c#6t^E$Pz(CWA_6?7+m0Bw4Q(C0Q{qM{-^Eyf4TiXdeBbl6CXbs)yaD6=NUyD35g3KKK8z2 zlFHH|ABD9^gtQdCQ_I#aG>%!|R3t9^OO*fAhwpO7S_ru=Zig!75pe0!4=#6s`~g-q$Xal|^W+H)(r%V) zo?CB@{v*VfM4GV*-oa0bq@9+4LPbSIEq(ccr|eSf>|cz!h>M%d%NQdhWkhrLC!cAk zzHMi@)BgH=}N3Q5a!o2WX3P~-xD`Bi4b>NxODllh<$JOj_SvU&hwqK zb6BS%&ufVFWC42IfykYs3#M;#md60g4{+&fe zZm&!_IiGB|6k?K%AwR?~1exUZPq6>g7^)NW!IO0dlDa7=y?3|dMCc-Op?(&|ixkcG zPZFOWZ65q#wz|Pm;koI<|3Sa5ULi(2Qv!bayy&Cq?oOSO`O+uL$|6y3a{;k%n(|p4 zd}B->TyG+n=I#Q;G>&aI-#SKdv5KHLDIaxCnH{bSZ>_vzNa?#~QIpI8z^9nkg7*ep zWT3d)q7_=M5R(XR2U^~KH^D!TtsoJeBXaf75+7W7gSyG`o{&GA@YW<>6ILdxcVKD< zhbv62Z(CaQ7B$0erGv#D3OVX&?v}$159~K*WXR!@qCj2dV4a&zv0epnbK<2ziwRb{ zJ?q8Gmq5{pZjkFr^U9O&u@biHb;Q0>3*{qMl1(QCX&G51PUx4q&;G1=E+2g*arki6 zvx?0HDs})>(9#tm)*8Hv-5>XM?HO|PFdZE(Wbcw!wbpI~yJ5kuck4h2P8gA#fB5q% z{1P4VKEX;E5>tN~7_JBKP|Hs;!;TVqvN7Z#RgQEbYQI>n_Tu|Ui2_8bLM+hNewYQ* z)Y@D#N7V^G zWX%|`L&8I#K4;Vap&C8eJ$oH`RM4h(urY$`zT)}zzCJ8duAfknMk1k2*c#cK`$|7i zmyc~3qtxgl*8JlKT5en>y**xXt4_(u?8g&lat!-wy*RnB7H{k_Kh*p`oy8GRppM_FK8*2*#28c`+r_qzlB`;!i9^>LUz%WX~=*Tg4<$K@!LyxW$r$B zFF4<=3Rz*48d{lZ3A_%?ulxc}sOf$qjyTz-@W}vo&foihKc#W?C-Y z{(-UIzu#8fmD}?E&d`@jl3dS{GmG$ODLQ-X<;zxjl(i2RkI;R}Yy1}K9T)DN)7b~O z;!@cdkx1zPng`6HD{E`tiJq4ZA4S}BT-y?LGKb|6`jpEPvt9P|Jl*>VqPht%mjAxA zH5GMA8Y)4V6o;+QQW35Um7oomKdf|;qOa%=L3ViQl6F#HV8EpR1<-##-OGDlX_%M* zFY;RROm|!#SZA(U{_!lptPp!DD(zvlF4lgkt@uH3`IF_x(3PsJs(Q;w?k;<8MbllPpvRaZI*hTPQD|0eimCey*k})USa%RQm@>5GRx9n6NNDt z6vtG5asetZT<{|!=Oz#fk6-0Jo{ zan)#~`<-vUl!B&L*C9Y}ZzK3LnX9U*TB5lw`fq(9d8Npp9({gT<2QhAOBBbrk)Apj z%MN1p#}U)~Q?A`>lSQ-2br{lfpxc+b;RPomo(jhDv?|CEQl|}kC50@SzR-s7>IfO+ zJHJ_9UXUcH=juBx5)VS^n`rL_oL^4tqkY{=uUu1mTcC6#Z$DUlwbS3>mV*IMtepDC zyzu`xp^U7lsUZVZ-`m^U{|2faPPN9glhfA@i+^_d3Sr^9*=p{v7Y)0D4DwKsNBfgVf0 zHP-s_?})mN@5#~4PzFOPNNMVOrk&&ZclI7R%a>33S_8K?-F_PYy3qs{uSX&@;K-e4 zbfvGMB>Npz=e44<*IjNV0LPl)XegT;2l(+P$&KUk~^U6BhcEi;wXtRV<2^D1( z6%S&!($LEbRRrXV#bTTJpSeE<)LO+|ED)r@yVia7&Z_%2%_W0$@7`&$e@i_!K2X^s zPx*-R!vgKbz9IxzgmCHLJ%(tDU`87)?R^N~c>BQ!<;O$NA_5O^HkB9d40Dhj8trBOy z+G`6Xfw~C<^})v>%)&s`egIoVlP!b)YQooWgn<svheK;Bb4!Iq!VOK1b{dnfIO`Fzh>K!L#LEt0Q|ydvkGH6U|-E4j-=^ zEKzD_9VVl9i)%do&`wDbHXm7zz*q|@&JPrym2&=~QV{#7Vq;FP%yWLcQRz4X_nJPY zdU0(bmgR0Wed+!tj|K99e?0d|Wo8V&y&tMgqF@wc$%e=L|22u=R?SVcBK)w{C@6my zW0tY3y!gjNvM|X+`*mV?5ED&aJwwiIt zrm(qd02M=9qNku$UBhBq0;z6eZ4kPid)k{?ekzQ9sz^1|3(Tt#t!Qc z;x-b1s96MEp6ncQ_oB&Kf9G5v04`CyutuQ^*#Jbf4&z3R4Bgpbw;fF#K#U^Zz3Z>k zQo*YPAUn_s=|Dm`d*X;J-M-eG;%9(l21!j^>LTpzn5hV#&3D~v$U+f?@%ULm>}qs> z{|$D3`LtqQm28VgRW&qpbn`F%AZI5-k%PAcWUHCLQ0u_R?NNqV1o5{#jy&+g=c_0%lf0-aae^kFJ)hy8L`| zTGZo$WqpSi}cipN(bLdkHZ5Te-`BJ~bk==>adu$8-IA zwN+=a*>E5*;;cy{dQ;QjJ0h@`{NIUt^?t2M+VIK4&?M_(3jo(1fik=2tW+=)`7W7) zo>7+|Ok_R?^7GS%iYf6yTB=u6>MJ$t-_GR>gNq~Oc0Nm6%V%JDWVv%jv%v$Xaho$I z6$-1#`0jcB=9$xhY$)R7as+4aj!#5RPpa+UR0!Z&VBL5vCv`p@8*47@q8Bt{Q#pg; zuJc+&U*DwdrykN!Oua|r@rSx|RXY$mS#mn)OtrTNpN+X#nq=hmN^d&I2%rW8kWmM! zou;8c8^|pnS5?2FOs@#UOF3=q;6&+p-^C2~m^2hjUR6rGIdAZBH~VbYK(Xcgj3b=s zH}orQaV_3Q&^O&2{-oGg?sW6qFSgyti-i6O2<-p&c<+lB?}+Pj#VqCB*8N2)eXLj2 zbL8aZ3#3gm1mk`Ty3PF_-mLQTJjnozgx`WtkPmMn*7B6cJ5m~5`SuOjG}01dBfz*r zy>;o@HHTk~Y=F?NwPSK~bgMR|a&(-`+l!QZxLzkgtK>Pt6>ulmGcyAcgv?f(9lKV@ z0sG5#-N9iKVVR zE089pMnl#+I<$@JrOtNS`L5;F`&TJGdta`Aoa;iTR42G3Mx&TMGSG!t6jnX!Fj>|- z&VQXG05r^3py@^CWFLIXE=tkxvwn_MhBC+W_r0m!dGTY%UQUxrzgYb!7qLedkVXUE zy?Y0wGRt@&z4FaW)f{zLm>iR%8TB~!WSj_TyAM69lHO$EnoA}hH6?V$zD-rmi00G& zS+VG4zqw#-o1vfc8_+H@lo8i(m74+O@^ExcU7)w1>sk{)=t-pc3!I$ku+vmu`PYAd zpT~fHwgKDs!(GX3GUSVG9G~FX2gb5t+~XQM;BD3O^f1<&(2^V}mUd`{X6SMDPd>sK zK7%627oIM73C~@V;lPmIuZALVP!8uS-zE9_5dM}qg94|31gtLJj*&itIoF4Fl4ZY> z;5Pupb*p-=XN9b20MaPtH~`kcG|6uQ)kmo#H>e+=xuAr7)m`9+}-a$ zjVr}WoE?-bPpeROof#;LfR#*;8f+v~P%&@|8SOH#44_@^aga4{zsV)gR?b!dmh zL;q~_ZtdW6pi+H#;s8DF%%03av}_)*KMJI^GLA#~Bru!RU|*E2i8c#Xw*}e;knY!# z4qvzcs8JjCCC>5{~5F=I?XJdhPWvhJEse8X zf#}U16j&Ok4)PX!NZC;&Hx4?;uV3!H{wmFtYmr2*c}J=U0IJy!q~_BkKdL)|eBrT* z0oJvXn;XVuTJdqR>qxv7@`X`SNRbn{c?+(D+-WCnb6&f)9P%k7M~UwE-DyP@Fk^cG z#sW++=?mF20dra>7#JV(4#8&j**aCpJ{2NX=$5v9LROjP&8O`D5plZ3?vmdIA3J-S&UK->|E;cx_gZ$Svyvo~M`51cSuEQuY?G>>%I=wydB?_}x!Wkux zHk!|bQKIL6Vh5BVL&9E9A1ou29M3DsO;bt(uboMRQn$dS9(r)b#>Q^*SaB#)kd*dp z_7#zvY}d_?o;}&nd%fOs9C*%>ywwRG!Ixf&4n-e!~OmJhlA-iebP&iAHO(VBg>HW4bg{jWxBn@ zr?mm+yS?0a&W3ks>YpyXzykorYMBl>p`=H6?YMA>0T*GSW$Y?mpL(pxvZ3r3?pPzaneoS#& zjj00tB3&_CKu~Z^3;pC_qG`fZVwf-+CXCio=uc+=$aiD1+^&@SlfIJ*OOAZ8?gZau zxlFGIB=^vX3n)zFsWp6&+0-;G=R1o+e{Wql?sdOA3`)s-b`{7Y+ku8pzccpKMl~WL z6&RwKH7a!XYWAST)Es!{M7Imn@|M@Lbm3!f>YhRq>48R*9er)`EA-sVLyNnD{|K&6 z$zN#0-~a!!_`iO>x?<4&C5S-uNDQT+qPcY)yq6j-`ANyB30wWAAQAtejCRVWo|hMeEk7tBD9 zr#_5%9}>b0RCfID$B9*f7ii{y|NYUTtN;1mgCeQqV$TDsi`HtqfAqy_(e0!C5Z8px ze@RuC+uY0pCsD+|B;nv8t3IzZI_G*?pnJiQWalJ4jfqBLZShJ+fu#XVuEPU${aQiC zUur#;|7Lif7KFRwO(uofG5mveZ1F_9MLU|~-qN7~pEkiNyywxgn9oE00&AecWFP~k zmqVB#iv8I$Jc>D^;9p@f;;zO=77kmDm1Vdc7RF% z|K!S*E5yv|#uxbwpBO2gfl)B(*xVOw*jq~}Q)Z+T2?DNRi`!;Ba1E=0%mGv=+DB8!eBj6v3E3pI_li`?mF->fOtq9K^IH~L^VF39Ky&o2_5QjE?2Sj z^mq6ikO4k}&KCS&GSEHnQ|uOdSOzcF$15Yk!$Hcp9ynHGUaP47a~p$c%qy4Pm^(nH z2PO~4lKwOB3qz}QKqFA;zep7gC1zA+^*FXKS52gPVA#*FSta`N<=J*83#4)hY=Tt+)GTc1IQ1_z;=Q93e7O%DmU(f`OmII{sZ9d zc4g#u%x{AbfT5TYP$>pY%e6fAqVRQreW}uKgu(q0ribCq@q|x{fKrKk4|FqMUtf#S zl6eEC(b?HqUwoL#y)}Bwz#1os?fWQK1oEhn##RciwfT7}pI5J5fxPevunvF?T&iyb z(}F3$H0VH>G{Sgwn|?{q#RbqbI%8Ob9abXOPex5p#< zXhVbs1`;A+4vyMt9nCA;m;n50ZyEjwjG*z)&%9xwChA-UDn?*J9;W_DNl8bbc%&2! zQNBmQq#Akp^9Rsu{BAukbsK}5p^}ImG2~OBNZ_%VX+DRBGOU;?&1jg>Pz zeE0{L1ay;6fiDQ<%m^SjWx^R?hH&=;Nc*)BwucWNu2+UNO8u^NVxgnU2GVjxH1NoN z*Sm2Ke&9ph3#Vr^tMpv@`32Yz!$2(s`onmIMJZRtR+CpT{px?`RjU54c$Jd#i#X87 z()vI^)d5xrJPb6M?_{HPZ)K3*`S0hw@XuUyNrqvP)C|f++Qd?}>9) zf8wm%&w;_p$_g@yZteE;;xF$6I{xs!97k;_3ZEmJ$1YnvAel zEIG-5U-%QYC$OS05oBEEhd@q6lf2AF`)j@h3cJJ5baoE4OpWiFSTQm1KKG|$=0y->MHu62Z+kCBt z^I-1DC7{4UN2$_sVr6<-w$^bS*l(x%<5mZoQ!E-aW%x6|9Rsu!`0~dPz04MG7nrIoGJMJhg@8az;FiFRw-@gY!EFu^%Aios4o&h zEd{=!9C(W0_-LjE2>YMQ!D7Sru1(h00^3G4e;?eZ`A}Y*!~Wr7vX>`(AMi8Lmu!wq z35vi$04=l}ULkrbnGUY(E)QJAz$FDH<@P8g46zN}5JF?SOG`@-9Dpl3?#SnhZ}s(P z0odWeBB`is;GpfpF7?HS33YSu^FV*~yf|}0{D9yC8HmM1p%nArIUxZ-_<}lkLrV|_%q24*^?|n?!wjVn#hx2!ruAZrwMAcn#cTC?WS7 zZ%`Z_9i@BRX$2<)0UcIc0YW=m#p7gOftJX%UWjA!mt*&%Rmm?g9_H)QQN!H=agsyu zg=n1sen&A*L^(r{Yhw5bVms}Ph_#yi0iS6S;IUch@p)qj;1IlUl7 zbDZG4K*SLM2MHMm`iuvlamf#Zb_+p!;WWUQhT`8Aggs_vW;Ui8ra**c zF5Y>6O$B}!e1!!Qyf}!puV(q8L4O7ijZxUU@m61#H#fC_pbD%#gYCqx&Ut?fC?p=( zwH?k%Ztv`v{rubr{LJv=KcDu0f|0=cGytX^w3)%vshr%&-v)0977s~pjKiYB|3kVi z>VHGm)fVX}+0Yby@nYn-*Vd%LH|X#FzI5Mnef>1#msd7bG&}b?x`y)o$RJ40$wQOOWnqtXx=3)t#Mlz!fk)8J$xOKwV3U8nq*}sa!}2I)c~c3oWSOm7Hov( z=@1VW*UL!rTL3^G~@F8JRG#sFlyTi3!5vSt$X$t_^LI) zON1>6$_ydU1_VL}yigJrP4yz}MqG?H@Z`Y%1?hwz#Hjq%`|M48SY2GF=yju&%Lo!l zJvuv>vJgxVbQ_wQ5^he&?XsffP%DH!rbwy_Fi+w2lgs=?Fm)1>h=5fsNcRvYP!49F)yNrlMo4*3qyamI{T^wHL6y+0Fe+Q@y=1kk-etHMwTK2cBpQlUg*8cb-Zs zmgtL>t3Dtg(5=mnA2pYL^{U8&QUziiERe9#-)V?$E}%{<0}8eBp~>y zGgnS3Ko}b$Q#EhB54lVg1RcnaBVnW-l!jo7tJXSN0EZqZz=o5)^HCcqVY;3lQ#YEl z2OwPlA})Mp`}G)Bs?_al*-8&f}B$x2Z_LS4kTs*x($dIR9cMh!+TLv zDw8Ime4Z)X<0IPm)vI)c%U=bQTPAZyp>68k7g9}TH>zrg%tH=Wk zpwQiC8uA{<<234B9e@O`^~E2U^&sQ16cs4rzz9kpZVO#^$2~sDJ@^ELAh4nh|Ndzl zA|#hi7ODqp0?`#n2u8=+p?rjtJ{y&7*pm>qQMXGn|NGltSPCJx{X9qoz`sJm3zxP4 z&UwBQAsPtLgoLyNG88`3Ran~|N$s{`+l|USs0`6T4@!sBl#~Dx79qd{?2W|G=uBv- z$GxP}x+`u%Js!4}+JCpV$)1?XQ(Nr;3G9sk)y{!V+v%)2^&$ouj{ajX|KHJ=>X@&2 zXc#;!V$^cPeDPvY@js}`#>co5R{ibb)Ja>%+5bQ^QFCs8_uUiJ{5Eb5|ISA9F}X{VmI_}y!&Ac z+^;LT$@j0YBwl?Yv%YgUT51Fu0JmJHMkS(achJp~?U4U9n)P3(Iom(B+W!ZIxc?m% zpT^A2ZVRbwNWqm+2Jh4^K=-ywfL#R$5fmY4zB`m?AY>r`gzB+^K~qQ*&Di7mC!V&% zNET0!T=1MhL?fh@d>l;%kLH!M zJj#uF$?G+f8}F2>T5M}oWLN0FaJ*0h86~c1x$97#3LJn0 zDpAK!kLLwLDFU8Ro|b62AZd$gKmjT_@RApQoA5v1&r!n%bx-@l!0_eLyJusB{@#9Abk?8$Wr2Pz%7VsYFsO(A4 z6ALY^DH1i5MhPGhAO!((6afKfd1@ZWyC9@Blh@b#bBqva#%K8gr zn_s?sfzas$wOS7>7U=tYq!j%mnNiqqUIQO{8AyB+8~YQ~mdRe{kfk&NC=Pq|075P3 zCEPuIJEzF2^ZD%T42Ou;8sG|Wu;)++LXHb+H7v~0Te8GR)}QT=6wM_%O}3-OWKy8M zg&f1{@UIR8d5Arb*n(fkcujld>Qx8^25MBz5&+r+!9!qUZyy{q0Kx{U7%1~cM>m!M zF#v6|1a^Y}q1kQ?zimqJ82Hc@(bad353zQ~y;C;e_ zfcT-4hK7bOK8OH;8GiM;@PwLS%4IQ;lTUI>!h5=+6gZ7w5Ri~}%-bp{4TGB-Z+S#d zZwjRw;Fa{Kv{xXB1zJ%Np-^=Jo^S&qLM$MOq2%kUt8S31LHbW^b(_VVAR#d^--{&e ze4XaN)T_i7|8!7VA>_fT(h1s5hs8zy4L(Ee$f!|cAF%J^a}DTiHZwn9{opZJ2Z^(( zKtfFgkSh=vvXFX ziewz9Nuc6i)pe~$SDp~96S*u@;ejE1MK`kB7T;(82?aT?Gjpb7k9`$*QNH+)I><;$-Uo;@GuiVPYV31}f)xg+x?MmCaM0pd zkldio7{X7zR_N~mk*FLl@BXfFk0CMbddrB zJ0Z)+lh_z5&wcv#id=>y+FJ5ub9(0*I7uz}?HnDM)hgB?uLK1%UwlZ}LARsgYZ({^ zKnK$CT=|f$KxC0-JBDS2yA0Z6{pfJe18MWiz_7!|X&ds=LA-~^vu9N>@d1=f1Ac!ue-tH5S1&}u-74eS^~xnFuB z_~<4`vSH!kMt07F#o*!K5F)b%SKhq26eSK%oC@0k8xbTj3!-d;Am9h!8elt+W2B8K z)xW&vgmUUwX0N$VCjJ)NPFVklqx_eC3oU;6PJ=uc%g-FM}!!$SeUk1IiE53C8Q|;D*3z6={jU6u|tU=n&V| z)`m2&zOGJPLj#Q&tcqvLdqT|#Iy;jfP6Q9oe|S#G(q?jX0{RGWVYGk{=Epc$ummqa z>F(iTM6^df;!y6D+TGu;5bB1^4g?5)Nx%#$>MZm48QOzGK?2J~{@?-A6Q;|K4`Vu^ zq5^$3psML|k}1$DY%2*02*Bu*58BUw(Xaf<9&*gq^G5wMV>$XL*aZM$jyCz@HC&vZ z0Ky1GZ*yxa5S&OzNDw~%kp}X`Hwp60VLudm^*lg$bjEjTG5egI1o>BqUxC*pFX94> z*S;X4&yLgwAB+`%5AceT4#L|8pl|f4djS-3S49Q9Ve(5lNr2aUqzGQw8W8fm@q|bP zoUOO7!&okqV9<)NxKS4=ZUEX`u$>^229PC5zy%!L?Ntft2oTmeSRL*)=St?W(S?`; zJ)gEyHMfa*pSQ&5#sYnmxS2p(*FTp5Jonn#6L=~MyH{V&_Qzp=t0aSzg`0~D76M56 zAY+%;8uI)1Z&aPWM_9!i#bXe2%2xy-;C4{I%5WW`Tx-&2&j)cns373pLPJ850O{UV zLf5K#>nkhtF`b~62?MYBEL6e)hbctp1$FCRwuh?KZ86aa_9Zy18vErE*g>$SP>jcQ z8|EvnLN3lp24$ymiGDk$h_C{SCxK>1pjSz=mshgH*}qjic?ql9>~&#^gTXiK@$^1^t-e&8Q_57@?MNoPJ0&s$Mh36-mL^#T!nZ2K$ zFwmlZ8GrY|0<0K@GN($7WS~?*`&USbN2T{;+VvqD01uM^rN_BjXL#j)O()jX#EKX) zgT|)~pCRoAFrZYLl;34rAFvX1xoqnK#R7qLl-Ac8!|Y`LpoO zdD!)QuDi4xV-KMABMTM*xCBr#KyOzYuER_}z@spc0QIF}*6U}Opw|Xb8vu4VX9At8 zmG$c2vz+#8SFN{$C?-4mHu!jPadCE1Txd+tDi?i5tDne2F4Fdq(y;UhGLnYV?asi| zLKv+T(F5I{^-A4<=$Ny|X@4Q!QBqQN8*+G+Yeh!1@>$^8Gl!ku##m(p{8=?0ytp}h zVX5gjl&5HQM5u?@4@Ku8m;vH9R7eXZ0tr_=ltq9efEvVHCt5j*YZzoeOTiyQR%2sp zn**56sJW2K_T16VVt%GGwBrD$Tpul^VP{7Zk)hGjmQ%UU*TH)=vEaYf6%r5Fl2A?M z_Zc%P=8b?lr}3G1Hh6tl?s#6i4DobqTwExy7OWc13^gDPfs{#BRyNy_dGA?{YWWJ_ zzxgMrMR@g^!lu7y#|J)teh9-+ki38lnmTjLFx=+oC@*NTPE4Eut^rqNP%X={r~@6p zVhF$x%3a+cLzz&^@ygaB!>trV%dDbbwdR{1n8)8|D1iS(YxlA0AfaFdGv1q70_%hP z#S7{{NMg&RF>UPZB>KWX${=$zk7V}2^Zhb?;#Dpe#nH;A0W1Ab;)Jhb^k1Kl9q|L2 zpj_r7VW)}@hZtN`j9Opr5ThqUeSnzsxkZ%VeYSqDn}DMh)Bq zc&2eW$dSX3jnPrn0I9Z;#x0}$5hJx%I#9bna!}D*f$mB}k$Yl}bIG>ygQ%h;Zw-A! z;###$-u8@U^MXTLcsOdQ(x{`N!LvIgY3EZRw`N@ZQLm(+TY?U zU#v>}YT)OgPs{ovd4j~tQtPXbmVwg~#p}ntI473x&ZnQ8nBU?8GZu8(P=~y6t+X19 z0xd8GkVaYW!rxDT=mYW$vd}u9`KMZDTnwtRqDnsIjZckQLd^V~)cNoKG93wIc~Ey& z+ndKt(IOyG8IhG%CF6x@DqqqPLSE=hp|x6(7V?kBw>x1&=&ybi7Y+j))_20RID|;8TK`|cmZz1N@0E1xMC=iM? zv)@%)kk71@^44VlkX{a<8pPEAcVJ{?{R^U}+Dhz7x}UXF5`=8lzBJ2y@<`}yX5cte z$ElAuenqBPXmmECP?}amq>^rXxxsUA{CqRhsw8cvj!x6;)O0<)*dzDImB%0>`5vBb z8Q;J+>2nAAh0gHrDE&JZ;YszXhrwFsqS5vt2@io$b#ihmZfrE)Qj}j_Z+5h#VWTm5 zEIxC+x=&yO(lVvoL39gfQ3Tx3(Vs?dh~;B;sYP()Uu4`M3>iRH#iaT?vmtXR zX_!>YlmW?5jPlR>{*?b`My_ym7Uu-RlU{vcwi;(>28{7C1c*=k;hSIDD715E3k|Uy?j>tseioLTXn)3Tn8)<4AZ@B=I~j;*Q;i-K)<}M?EZ`OcAhG z<3F0-W^uX^Emn6tp1Pd4aWnhSS;WZWufupaoBy!T{o&mW?F&)4vB&DiqhqoA^jIXG z&Ya~F6Noz$AzqaGZ5V2`{h9COmRGhGZ|RTTd+7@0oLZ@&R2&>;Qw3QiR4Y&r^FehB zj|O-rB@fR?#T=m9t1^02ruR`oW-N9)c(BlK5_U~W#P|P%x>p5Cd}D-d5Q~WQj&b%ja8g& zloH9k*`MSYa$f(2d}d+B?B+D@TVLUrtYY=G)~y6TrKQBl!?51OqK?cqvvr?Eymxb^ z^7E>jTZTZu%{wez{NbOrBTfCA1+afHuQjY5-+rr9vN`8nxpF6>(_kpmuyj7&!t44i zp4Y{_1xbXV>J1c$?$Z>~!lnmDEO$=UT=kal?s*<~ zEXUq=X-qwfDitq0ycd&P36f|vW1Sv9nWr1P6*y;?tvR&#ZL^E7hYAL4qtl!xn#`or zb7`z_$2ogSM{BPOpAY2)<7n`AjFGz8>;2d_v2@@kCM$cPkz(7njS%DRO{*9Got=na z5E_iUNxR#QI&e+=UMVeB^*7bxj@0s}vG|1BW;8Gr1pWwJhiy%hRZ7wzXn+nQ;Ga$JuH*D?&@(dTfjBEP!~m** zUuYp~(0#Tjv;+(hs(DCUh>>fcJPe-zbx|mdRUjLLzV>Moo^YB*kiv@>KKS2TTU&=_ zR#(kH6G&ILWTEw=8{1Ep7YRaJ;%$jzzGoIpb}|;X`1U5oqtwh-_|l*Iwn+4DmeiLW z^(LQe#uF>&b+i;jxXD|xE&QDE;0{&OI-X9nqUKy?@lhUk`Hr{gPU(Xw_I|C(Kt7LI zN36CtrtIWYN3zZTB*UPth=jikn^tw}hj(Y3Yp;t!9hjV^{a(q%MX&zWMgz(EajjiM zc&7-rxklm0r%kI-%`unV^USG)TYq~icX#{}$zozl7WzYt$W^q)_7zHH>vORY#KBn& z>$!%=_eD!-P8s)uB{;k8qMUmE@P#%J_;-7Kc!2)a9F@s~t|$B&PPa#y5>sXUe%|=5 zt2oaPUB`C8KcJpd2_lKJ-ec89aRgz0hZ7mC+FsD)Z)og1zKPEq?=sAk# zKa$INKXOcuGQ`(5sMZDe+hgcTu6Te20V65;@9DJZ~Z z9HYPsfB^Trd1e>p&6~jGJlZ}pVxC3&2Y{Ioa7M;&2n}#@5b_JmfIJ<@kOIvIMxQLj zd{t%T5$HTX^-WnGfXe<2BQ}r|Li)kv1IHPO0}cs^iaI#oReA`JXz^W1>ImeQwlVI# znxY+Ls1pSF_>LjG{5?1zLDwS%YLy^%2vrMO;~P?AKx7D!(9HeO-E9HB#>9SpZmtYK zFKD#`>We>gRZ9_OF~*FB&Xw2Ryfyq>s`q*Gr68MH7nJ~xN-{NBIC1jSXoI0YT{i|M zLiKpiMv*W0(B&(91DCWwNs3$gg5BgPNC0JMO|;tROeKvDWg`g@FSLv4Z;z28UUVbp z^Xp9_SMnz^qjs1!R@Y1_VhM4@C^nOC=Sp1osOG|(r(cwFciAOR9ZidD*l*Zb7Z}1z zYL4%Y*1fKtFH_Dpu(D}YKakTk@^#P$U&@TApNH# zvZX%eW?~pBaa= zrCdGtDNM#eX0HmPEO>Kp_jG zbF06y_0%$;=?1F3#Z=c;ML_0%-Lb?$T5qu!GWNR6%tdfY!k}vmBWPvNNP%(>`5qc= zP>q8e@MJ|=e7xG2hKsW^5rZN+6`+w$DC%Y{_Mlo@U$^kEhTd9MRu+JS_kw#bif>?G zY>bw+8gq)@i71$Ji!Q{$`1H0XGu-MwD-xanm*#es5~kz@_leoWF^;I3%A?fJ zyAo3p2}fM=QfKXvm`)^7d&?6vHZ~tH%rJ^d60yz>&X%%xJpBErf~W(_(>2pi-9m)J z#+(kU74CFz@_95xMJPT0o|@cyHR!{zQHi!$Hrtk~RdlY=`(Zw9DH}&}>(#?OEH|lVcZriDx9Q#a4tt?e z(Eq9(g{1TOd6%P>6SPN2#SCm2uwpiyFSV|5i=q{ZFUwEdqyUjx8n2E3Ch7#B%W)ndWryqZ`p zkoemNUv+keWf;5W2ov$0Ahq)(qb@8YcT5GdxN2hY{79V2D&{!j!gZwH?28lLZrpg0 z^OG$rqT9%}ml?NCjBoNGd@4)lw1S62CR)at6sUZDt|V`IEgtm4)%3zUh@@QHCts!# zA-l-cGu^i?PI1EFY70%VP&;tRc&5UJK)w%aTXgfND9zOhRx~0&I#*|N42bpU^iECk zzUG^vWe~u@L-r0?)b!pW)Z66}=niiE>CPTXVrO36@vzM15=91*vb9yfr zG{u(XUeG3c6$P<TSB^)*D27A^xe)7ukrDp9iW&CCQ>}A`TZd%Go9Fd(+H-0mCgiB## z=F3)Yob^^_+W2BX(C>sHdNS_KTTPFgdMlO)5*fIQmm+j;6_&^1zHAk0kf=W2l9C#I z6XPLV9sWQl2AW&Y<3_i@RiEUa>}AR4KtTj0ad&fbb1xJS&4Ca<;1n8tvRq@%PtfIr zDMhu+0cf796b=ItIik-V{D6O`=G$y*sMcu)M{h}!atwo#^q1E`&yV4zG-SbP+L#w z=cJcC6FKybqZw9qW&8O)e34$5qS{x~!$v66BGRf5_71Y>5uOQ$18dY*L%GT@o zwhWzG3$AHGuB38qXU)SOC&IWh8C8C#eescxa`lB_z4(#Z4dT)T|9CvTE2pIk zE!MoE9&OHdyxZwH0MT7J*GCaek70Xm@hw^ILp0$gQ+jFFlL`#GaqAa zlUvShr>m688xR(yV-UM#J^)0cM3yhJTnIn!(`p5%j*6nTNmy!x%D}ngCt| zCT`KEfdB_$Kr(k z2+eA9b5|iH1{Fq_AwBG5X}EU%I!wqN!+Z>jrTw=zmthhMJ<|!W18_OC(q4Up5Rxw2 z1;GPNJ&BE_^&y4HCO9|931|V@-`>Cgmj^_1*hJ+`cq;%?Eq%&T&1HtMmt;-xhMF&{s$ZEnVuoQ-|C7~G^9d>(bhKnQE82L9gH zz{nXJgv2aucAsgSkM?w|e{KxJ{32f>l+G`rNv`rA;jL<#Kw){+F!A*0_iXRU>4lQ# zRX)X7-mIQDGac`RQz6f0-$zI4(IKV}2lI19s@?6M=lAd!_iQ#Kc?lT3n&^M|J6Xh7 zgR{fxLP?ljT3CN_xxSSGcST(`$3GywPRLlva&zlO#IQO$SvTCssO@p#&x|KHMty4j zsVe=cZqDn;Zw+>PH!^DViN&jSJ11mvKN0Bt1Gu1=fy1KQVeUkb8_qFGhj}SZHnzT` zdoMB0PIr2t<#h`!+=*z@#NA!pFl~E~9A^dI`liU=KlxLEy{q^UM4L-UO(nnrqg>oh zCRz&{C;M0y$(1~uoXp=%_W3?E0|~8X8g=1niMKx18jU0B1&r85HsTit&IgEM>|R#T z!|VstPB6h`QON{@1OOiZbwQ|kUj6c=yGrFx<1C{-N%7Wn^!cVW8FNVqf8F2l+Z&ZM+FLae!mJ<&zL${Y=FYJ{;xwuetZ=w* zg57(X#B?%lWLJb%1OjIo`hedsxf{lgCZJgW7Pmio|I(ZyhuW=ba!0G~^s6DvkC|p~ zu`n$ya8yZii;`@+#veZGhK>96WS&BlBs|(=m1Xh3abrp-@ouMcFkriF6Lo<>Bl>+b zXlNH(sTY~dC^d^M(MApKbSO6szPq5H0FIOqtKs%~lM2P<3hd5ymgrFR4{v*}yC(?l z?1)CV)P;XzAYELp)G59=d1Gwl((i)pPpmAkH2mqDOmZHCW8GbW*#2 zx!W}rAK7MrO;fI(k23w$PC=XSOPnSOnW|V*L#vVarOF z)%*nt+DrQQZ!cFTjG9-(GPx!)i{%ck+Sfcz{CW2mMk$?iJQ2CYdkId{2uZ&BQEkq< zlEkTv=B#lyMM@iu+2XofQ%AMSVqvf8e2lw^5r;2Q+MLJTF(@9vC02lHR}l9Vr&FSv zBcq85Y>EB+q=kkolD%GhV{UCOtuG4%he|~XUboHLJ_2^0^Ey6hSZACRb;M*==!aoH zQEH!;E&ea>>JjK~sY-;y1IO=T1hzAF8R`_rb`uAVnzO|l5bJnoTigcyI%JzZap|Zo zD;9vZF<8~=)G;`5VeRv_t9rhYFdy^8v|5$vcLHO9H1koN#*Rzv!jA01fueDOzUdwG zgj$^=CT^Rry%?WHIG0#_+5mO?o=ACLN@#!tc4(jIi@2tgJz1OM5$6bLqvH25BVIto zGt?o-um??JAIYo9Y!c7n_&ymYwq)nmuEN&L&gb-z!%P#1ao?$ z0ndoj-1=kbyB-0Oawf?)UYBtG1D{0rWY%Duy{laKWEyuYE=C}g8ka?}vOd!A*9s$b zDT0k|OU0*uRBR)SnAx;6H$uYG>Hv9k^Q-1I5*w0J^K7uchZ46*!z7j*_`D~>Vq?Q1 zvj+WM&_5#BMciI6x?R(ztRb1utSwpW&eeT09BNKtu$iHIxLY489LsBSI|c?xGY3q9 zYeswy4|$ZV@QjqW%Ui_q746Dkb>@_>VayR=6>s^)9>)I3l*LPqO_`auQI&fKyl9T) z-S7I2_72};(>UV0Gah?$F{)OXb#3%)yxcyCq@PLXRa=-%|6>f3jBVMNxX!_#uu|F{ z85@=*CHd38wJntZ7Z2I(lzdlr_U6`y==))W=6On*?9|4-_}E4?<+-J7?OBB4dheKp zgr3b#FU+juING9eB5iq{MB&zWG2dP;WYb6)iv1j1bHOunPAOskj#0!cNaANJRjfUR z1Xp5e*c~v_XU2r&{&!~x49J;@@r)`T9o0(^*|aa_Z&};*gLv0Nn4rC&Wc{-TqjhB< z(ynTOdJ!ZbV&;yoTfHMsawjwJ1IfaDd--PpAUA0$C6dofC!o5VBI{Ojx175EHI_G< zSp3}Da)FJFVs;eLN~eZ;U15?h|xXL5GS&w`;+>M z2hK+LCmXrVeC4Vw@J3)t<0dd!xF+u>#@cU+?Iu^=EdBtyLNq428Qg=ky#DVR`?8|${Ro4avUhN3W5K;(9INx7cs#)uNZqHgIgR_V-wRbqYL7aubCX!R;&+xWw;2+DnbOewdoQ-a@Lq^Z+s%Jb$E6Su zSixw=Y6Tm_h@T*Kd4ps1rr5OXaV(F8aU|E{k~iu)TidK1`sgWK?HjP68NtYRn`r$0 z{Pi)KhGi48vbKN8>kDitQwK3wwqm@bxF~FeEqCS(%YRy>a2N!}+(W}sUN!_`ds=QF zF%w?jb9YVQioX@bEoY~SCm?Xq!_%2vzT5JF&2>2t>IVtDs_9$s<4h+uP@lv;@d z!~J59%J~HclxL&pi7-+TTx`#Bhe0|jX1zO)`?0`H4mCz*+uO^_3v&xAD;r4<4B#p8 zN&h&Dc=zE)JbW}0PVU!yy0c{9^!W*Wd2WjegmCqQ!6ZxR>mHGlme?%ZapH6l?UGv` zGM27+yozLIS0o`Rln_TfZV#qTt-tE6{zWz?A(0a#qCvi-y_uIz=$RGk5;#rPhBY;= zdr}`{C~<96jVSOWK3+=n*6RM(K-V5ZlSi3+V{Z;1Imp|X*48{1csnn-M8!Ceg7id*4Do}6b-m*vB%4uis1iKnZO*oT|$BJq8jdgj~1}_;R^Q_W~{65+z(qu+biuLFNc!0!z zza47*GO_3^pF_3AK5mNGJ-?u|>maW6#a=NNL@+$)qaEorl;x_^ev45>+Qz9f&)IIB z_CI4sB=w5fva9((G7#t6o`-j9V8VmUZ9W7u5X5n*OL=jwdOu;RGER7cUQ4m{Je%!( zYi5O43O|$-ekpCNJ)X0TLoyR>r?pN8y#KrS^6TSOb%uyZ{ENSF%zOo5r4iMyo~7cy z`v!MeLiYA@NF+W9g&mEu*QN=|xlAmRWMQ}eon!{QE_qWKZYL#>twVn8xg(Kd0Ci%)j)Qs1xKE z2IlEg@oN}7D0fNok^I`|T0~72Phfk(#`N_#e6pyqm$Pfr^D0GVguaFT>avqlD-Itf9r7Fi`caZMM7f-sl%*UtUhE33O{eYU21*RPfgX*vfPx? zEsNzB3uzNezXT4LN&ah>s_I&os@iD!JZ6no3u)oJ*_y87wsDdLVQw`Y34=ZJfo?Ba zJN4NSkoSxlbH#Np1lj+|f!?OA?V0fY`A%yAqZq?)bBw1mOUKr%0eh`r~ zJ$vl7wts9bW_T@vdd813ZZ$^X+UIs)G1(xziHn<*NuWQfixi zRkcqSpjLC=ngElLlo*%IRjBW~T@SCE{G=*Zfw>jR4ev7g`z}e}zJ~kfit}ctK)k?c z-#s_#+@F2SE8Me>Kg<*j;0mj^+WhUV`%URncWNpp^jo3!s$f?hsz-TF5c&Rf0Pk$D z^Ct;>_bO@r<$)WzlKsl+8PoIW&XrdA@kG@HlXYxZlwEi6zCUFwYu<}e(efL3dwaWw z*FxV)pt?b9t^ah4zGac@--%6s1TijOts*^oyea(q1>S*!=Kc9|pD@}WvYo`GF>Di> z!`DJ@4tjJvAL_B zFtfd;>e&Lr@iju}*gpc*Uy0S#c$LjC5%TAQi7P!K=i0ShwsZz6!yU9HyY`b^bc8|o z%!1i66uAm##$!@t3QGrhp&L%B(-l3o0|IuW^XGJnR=fjjb3Uv1Ac;tLexx~ELmH`yP0^<=H^q( zp{gcl_EMrBmCe>N6yD99jDd@Z@EJie9JhD7>LsrqkW#g{o*chvbh8M`Gv(;es*_Sv zV%XEy5*o{Q!Y~aNJ)x}VpGLI4NGJbTlNx9IM0>w&UnQR;W80Mkq@A{SW&-#o26o4IMp7guj>sZeUKDE!l3ZSMZ1`4}ixz_zrp`ULL$8wy z>UXoRt4bWp*$phr4AM$e?^%?wk?L_X|LnL=aQ?%CL=%hBCt(L;xsk`K=niVGAde9F z&=7r>->{y+YYsh_fk5$=`H3F6gzhx3Ss`1DWY_&$wkDZ`F!tN|*Q zTk%D{e186PQ0K(6Uk~g=GB02KPD=RVQ6njiFY6F(s+QgTMRmm(l-Xwe*Hl`)NZHi# zJ`oe9@{%Fd#RR7wH-t|&z175BKb&6Y#oUcHvf8qr$RU>?g!cb*GkK;FBqjVP?+-~t zuwYLyb$n+r`lqkTnlMLjG@;HemawuUTJSOBOd~B+D5C5 zBE!7$p5g#9vrNIHV74*RY=!K(Mn=rVgi0AGT&~Iuwz&=7BXRQx_ro%m1lM}$TRYm; zW#UNo_C2cy6jDEA&#w{&;(fcOhs4Za>6u-x`g!)jt|wJbvbFx8La~N>L5UbI(NE~6 zWC%~w{$xx}vdFz!rz{e5pl-1TH(&37>Y)_+qo!B?*b(}+X;S?e{z##_UNiI#EOOFYH?#Y4(& zze|%7O{l$hq=UE3^+K)7oX@sj;-N(uFKt$ETh(9sQ}#vh9__uBgjgqTLYR zMSeD~8I^nSOU%aB_FLbs4X|qCFcDn$m)6gB8lZX?s6xRbs4Nwro2!OPLsC8jN#CV6 z2N-#*Z+8A9lnxPOck>KK{C>dXrqkh8m_OK}oIa8E<8tfeL&*;=YAjr+7QCfziKviK!C44CdT#^IU^Wh57^}cxnxuJnzF%*MAnavR2 zV!bG-ToB!tcWjN$1kGZf9y9FBY(EnFzG!6alJ4Y}!(Y)to3AzHGrjJA+qUJAo=5f` zEqBj0P;lUDFTEK>o0E4Q{ph`4-XEe|{iOCL-BuG&^d~E9-+7Mvg1sxLg=?E*R8N(h zTtWvOG=3?-o{D;Yi?WLjr|Q*rTHL0pk6c1^L>}(JT0-&^?YpHbJEA4RLqog!rQrIwK485_xWQoa1WHa#EAFOew3ixJZwN-+ak?u-S@sf#h=^#?=C>z*!7 zr5(2Ed-rM^>IpMsdWjv%hcgj2F^bI2Pti~5Y=rh4-gzq}{I9*cvW6XAd){2s(2VJ+ z{Y=qz!tZXODKI7H!s7FkxSgTv#MNTi)V|42w4Y+~&Xt#|e5{o+O2@X+NZFAhw&olc z7;!0-+8d1M)AIZnN^PJaw$qcz-|&Vlycy_y%l)K=-!4@4CdBNbG{juiLIh3+Vt7%_cRjd_y6`Tje% zFa~;c|FEL(EBCchV{ZP9`2XxGDAXl_3pEhfB5{7 zi;tT{lzZ5Oj8fW2So_#4!- z9m&cf`W4?@4Z;7OpiAyM)zG1|__yKY-kvY#(44~2`E^`Hxe$B|>GZN>~I{3{i#(gsVbmEJe}LrGFx7H>^B5 z5V~opjQ@kB>kfqa|Nj+nQdz|rk&!))%#aUpMk1GyO_w;Ay^**OeSi1&z2EQGd_Et~=kxJ=J_EmUzG;4VaQpm4@+}E`Xk6y0 zNqUCJc6vF|tW@2exDP`imm@%7FBB#@-21K7@qH5h!HJR*iwm*OKOk?&5|R@B7KsQk zw8csDbvu8G)zdb4c(=f8jrWF}G71^95Dz^qN$_ZsEols)W1<+TbrUF=xX^ z7Ayjk7Rwjzz0xgkH$|EuZ-5yO;KvS8G69$AE`Da&c+wh^&^KI!Z;Y(>GZ(-3KhXc= zTQ6A)tkfzS~kt`hnFV{F*hl;9@O#KK*#==bLj-$_tpmDr&1? zmGHs93iLl|!uHVd^mt4gy=!~H(i-vezjyDNJQH~z`&qi?sNaHq`-F>!(FGwY)_=ZH z&hyd(KkwaFSB2x`E_W85iVp{2K#t#ZM$Q+f2SrXOWRwwWDDM920656-A@f8#LRwN; z(Xzp%ljo;!c-H-Y^-dk2|DH)+Blg&sU(K558Q=5ZYcW2!!GnBxHfU2aFi8L6M#%RI z0Km<_|3;mV{l1(^h~~Y((rwr9YBu52cf%S8-k{kATyhbR{`rl|i{>{y^ z>6(HQXj$*i-)PXX{1FeNBba-VcuEEqK9@PU;V?Cy2CSo*wHUHio#;;oBI$6KRoO=p z#3MUmMSU`Sl~O-6A}ZGN`PoG7UjkR_!<4}=SsmMZovnLASo0{GZYkg(xKlwA_ahHknHc!lk?h^Sd}p_I*J!vTi1s|}yBcx-i;~{tZ!o;rkElY; zm8Bu1G8-~r(cb*6P%c|LR*Uy6H+rL{uvL}?+pi2rPPSqjGhtdu_l*z|`Z93Z3-j+%mbecdHeGN;(*gGM z=_+;8%`pG%E`zL+CM`EI8+MWthzD;?*ZZUe=KgDb!2+GW{!B4fk2w-trUNoQ0 zu}*Oj6zG{6m0%FMj1bfsH{jPhnP8%P9ABULK6tSdE#car`YvueI#yl&zY!Pb!|t;* zxe;`eKfV7!+~)XGImmPAtJzWn!#1&|wuC-(FUSYN;9krRU7HY7cFSuGijYNOs}k&P zyIqY^xuNyNzcoemX-)W@waKi8l$`)!03|eznOR+PVr2gQ`KpN5mG)Tqy}1kat{a0Y zJf7>z;erw8?@Zm0U`mxui0Ehp(*-W-2|=8N{*qBe@=zDplwe}hyiV|WE&c1FQU6-` zo#{SxO3b)`e_Hj_w?dp6y1H6X?MmlL<)L$t$Zg*eP1S;3hb@FwbBq3o@6Df;f}@Oe zuj+M)ML4_0hL5vB*eMfMI#7!Vb8Zim(smaYdW;&tne-m3iFeOSleGAq$?u{~5PShAl0}*z$ujMo3$%8;o_5VR6!&6Q3R_E-*(;0F7 z@l`UvDWFPP8(to39FBfUdxd0t^!)a&LvkhS$ZPu#QmZ{@U8mQrJaz`57ViFw8w_uCUq;6myZ??SOO&l-#u4td$ezv_u%&@b;&kms^-E*9OxMXtdStog+ zVBF^Gycq~C&#K*=a_w|p6)#k1m3_29-ji2fiBX>yEEsed3<~sG1quF$4$PpuV2sma zeY9R{^5-p=nMA0oK;)^`Py%}2me$>NK(Fqrk?Tg$Rv zsM2E13s-d}R&M7qDb9a9WQU{e>l(d^o#!$t2(Gx8!eYFfh|BGJG=#+0uUgWn|E|)i z)Ty>;=Vpj&8@K?@MdL{|nzyrV@B`W|`#MOcSrm(A(Gq=L`|EFp5782d!y7Cb432=6 zF!3lQA3i5av6(sdoeEh3n6E;+pZi|aH0CBCvyA$e(!7ZSsjnu)E}s_VJ~Mvp^fMg9 z>L~|9nwOkgr0n6m#;8Clu6lbI8!(G0dvDSSx)JftS@lz<9WH>VTohY^m-F8)+KMuY zdshYhNN6E~glXB(S}h}D#@R8vr`6)mkoEt)Rm#6`^mqBDm={|M&FYXG*>4)bq|LUu1*vM-Ax7vz=RRBeKM_1r~%;&h&X~ zyPV>|Gdzr^*P{(w0Ri2**XMd41SYZ=XS9}5`wfO0J@`_?5nHAbZF9m&IzI@|$FEb%kH_JuI z)kse?^)K`P@Wm5g(KlYm=Az?Y#F(>IK4{(3iJMbzpM+Q~d=bh^jmfS zyRLnJYJ1Wqjy!lN`y}brS%gSS^7l^z-y)jVPi5N~qszehlUqfjQ)Uje7P$D01v(wm z7Ja`!9AbZ0<~aDtu2f$C*VC}M;6)9WvWEcZD>G%!zf#Tl?t^6&i7g$b1xyqJdykM! zTHXe(27CU0hse_km>xHJOJ3F0 zEPvL@52mXzynU*Sg7MHtB?#;Ihc1zbXMHsP2Yz&Oeypw{<^>) zQQtI|)o@cy3nbY2@FxPgc=u`JSjM8w=BLxqtTw)Uan|PZ=e}=Br3Spet$ujT6zed{ z@Ng1R?`z$Ox3fw+*rc;`wcmXG@87(DY_$0-`et|^_U3-$Xkw0yQ%!-87HIo#Q@k#S zt(#l_CqjEMCU$XA=n99r=;<@{QKxJ32DZ@p!Z%@;aEssf8CAcmfhj$K+#v|~A2RY% zQh<`TtAm$g?{xlMJXLBh2Iy~I^o$q9%4e61tq<|!74sScW-ctqO*+`cLjf(zc-{lM zHa$(!wXjsZM48K`|L1SsIW=Do>&uroEz?r~>b25(HYy7EIwEVZz`wznI|CcKha52|Qg+XlW z>?toHlzz)4w3$w1MilX)u_GFidzNX&x-S{~w|{+v-a;nW@bn{rCjgcrOPlb>gm~)c zys0K7;R{?<-R8D2QDTl+%s#l8o-Gn^1!c@C2|6jbc+DYPCBp3 zkj3ztyG1Dbv5MHt0Havp-zev6!J$K(xmo7`pPOuo6@fH)t^koEHQ`&%0y$*$yjZEmj& z;s3vE&xTG<&y5IX-YBlDIl{iT=wpx1`*4+v4#y|>ZBhfOg{CD!HXfQic5&uIij{&Kcmlm@wJP6y?|1`NON#rN+E&PH_PhQPx%Fyg|rm9j4XaAgrlp zpSxmdsZvR)LJ97bkDCA(d!;<%-zqDfE*2a@0K>oKi$ruQyjF1h7LGr!(6K3+nDd5f z+FacEVz4y_H{?cyPoG|xQF!$OvJs-LWtPh~>&`&x`Dgn=GQp|;h5|aBib({__!%-t z);4v6Ag~B*n72>px`R)4b-fSy7InP8wkTDW{rm(h?>PnL9dyIeo)t%HqQxl~!F$y6F!DmQQor2)BI@AP zk?=e3rO|oYrFYXhwLq!!)F^)8FNLRLdQp8Ecf{y|*2qfixAgN=0+tiXp48)`wsi-M zn*Pk2jb)jQ*gkIl4|GHFa@(GHr7}mU^tGU)*K?rEs zZ>b=F$0u(4z<>s7rCE9Vqv?gaZW2!7h$1Tvd;wBIp`7^9zs{vwG&iEX&sY8BtCxC$ z?7E7dtso=rKaC`oCo! zXyl{rOfyPcy|o-k-zs`3+_UkjRAtV0qpv^Tu#ZkW8})HzAS6LLMRPLl{4hNoW<57> z*Qxx#Zc6?K;-AX!Ar-zrwHS8jiMxn=>q6Ww_<6k;H$XyAa9aQn2HYXaQyGIJ+>7Yv zwHIE>W+g$QId7mJ8Gzss>PHN@XiqKLlq+}NdKSI07DFgm*8Q-$L^dq=3wGr@Jv)D| z(E@^@mqiOg5O`w=ezk$iB@rGX(TNvQE|?i`V5O1^TqTmlu*vG$C zD^Ws!B`>;wS4FQoHs`$138OoE>AW#K6DsT=%D@RkxCmfM2H;B!{~sWviUJ;~1FGdK z8dg2E6GyZ6!V;3Dviw{rAi;>T9WqL{c!LDT$?~VZ%VT>U#v&Xc+sb(r4I<`6#$~_z ze)bYg-|HYjh6ISa69504EYwTD3j^Nvjfl+))6~>q3F?W?M(?2{qd{|$omjDJworva zOMgPAPdKv{_yOx9?)%r6S%WzqvR&t_5vUaSr0LLp52G0NU7?5c=JiA7=7iQou^X}R zIejG0)z6YqX6oHsz_$72sz>%qL#Ekiy%QuTB}8{Q;Wa# zx?bz2G$9{Mjl}ZyE%?h+;nr}ARr1{eB*K4(fEv?(-<6$@L?p?<5SH8pIGB~egS$D& z1=uyRU<6puu_MyKu{rY7%fDJbuRFZ_`|^{?mCtWdks?JBIDZNDbL;-6TLgr=1A8rz z7<{34kwDRUedkSqqMxRX9l!EDin@Pv>O~g&laz4{3PuuBO9M^Zv6*HtZ!1UYyFnv; z!vd#sH=*G6BsL;~IpO-1F^ua${Nt>t_}2rXZV?Es8)DNLux;DR{AgL?aCS$hoJ6#~ zrS&o5%*ikzfUxq+@}MAA z*2@8TgnYKVezxf6r27Uk+k?P?_S7C93sr6jh^QbT=hx%M#`{p#wRJgB zWBQ7|X7jjh(sgQsPX~&st`)+?YsiLaILTcRhyJd;An^#&ntDA#wgt=$A_y$n^Hw zulSO}Up(M?Hbg@Q?=sk-8?+0kuht-s@)^r1Wa_~&eIym?q0fL!lrBhx8XvkpyBYiC zHzynzEx#cM>hdA)=20SZm~3_u#DMZzA^lNfVu@tY^o{EBZ(@{l{HMmlU)!~@qbRoe z_Gq=zjbiEYM{>D^7w#>W$8$0L`htW$Pml9cEVB}z#)TN3Y-&Vh+D(P1pv!diOwUUQ z*Gexudwig}MXYDh?>QGQKM57o`{0$cMyCE;B^wJ<3Om=ICCmy=-HqsyTzq1wl)f~r zQ!1(Jq~c5DrN#zwDc3U)BJLM>V|58{iwae_+%|(5JK90*h!LZKCp=2ru`?*X?+Bbo zV!ddQq{K!x>;tJkjxg=;aFc%8#b!naw6`sq;=17`rwQBf!SCE)H)CSp5!`y`n~_^B!m|J^qOMkpQ2JBE)-lc&9PnN#ml@&UTd*ZqFZe zJGx&qdOUsI+-kABI68ZYNjTi`Hh8{8F92m(`f)n4n{f}Pn=vNG=uebzPOj<={RaNA zE-Ld>s?P;R#SfFV$$xC>#$2KCuPM9TbO31k7pdLR#@t)VFUtH=#78d6v5SwZPcXWh zhp)~X>)qRZYO9jtmNs;dF6#d2vyO}{Q)y@fhXsQZ^{3F$cDI6IaZ&Kw|FLV-KJj|vKe2pKm!kNBdzA&Zz8!$+cD)ZyxNdX)sPQa z!Y#SVDH7D8!+8b!J@7i#L$QNMJhsUjc2#J*PkYw%`}s33Vs_r%yLU!eFBvN%vfgJg z-B9``KSubuMM|!8gRJgdbOYE(0%M$E?)|e3!B#yp);ai>;8s=niXmwB939fl%*nE< zsb;Ni86(qGZed}I?X;R1(8~9a!Xc5BiiH}1tU57X^_5lr2SAQfhV|zp27dCh*VS)e zx4McN+(Tv;$8eYlmtx_XLZlQ{(W6CRv?!2TWQA};A{48mmUB}VwUs=IpLxbA;Q*WN z%o#Er_4hB@7l@knA1)8ky8d6gr*}uuukU{D=uG7obSUT8Oce2>4Bh`jI z-(EMdcI9RvGfcnyeJ&D5(kdF9u&F=t(6%(s>65q6T;7S+W?R zeEJr>lVzcmD{g#Y8k4e4PQjXXCSgZ@a}`S|N#m+LLSAskCc*Zr8W{kbe0*v_bJy>X zz;Iu6-nq5=w;Pck|Ma2K2g&busLiL#?2M~4C%MW^eUGqmdMCk&#|t5D>TyN%R~N42 z`4FuHosylx*8-6q)QzK~(pPQmls=*k)3SEvjADmC*LJcrCNV>m|KW!csEO)DxWTY> z@}L6_Pnro*$V{?U&7qH@iF!SNZ)ILhn@Ys>J{783&dO>sy?+`+~E52gqb9XysYB+6~wcI2*@`O6F|9c+_F5Y*tgivOiku)Qs!RMHztWseoqL;&>4 zGiNex0yuS)sFL?N$%&u-z~xtR9MzQRwvtosMgTnLuumn&OtHLN&dhx=8a5fLg^T<& z$k6uM-uH5eEVQ#$ln~UY%B4Oswl}u2f+^%O*t63Ca^7*`IB{%IIhR!q7_O71z$Jkg zxf~WF1IH#6VtHaaC%y)6KK7+*h6Z?KOyf;GOs;)S+~7@4mX^RJUo$qKs?RTPoEem*?=;d#B$!a9lE3TbnDR3#Sn}WU{!+;j&1rwUiOuXzEV}Szp4`&ay&D# zA6MB(nlFaDw_9qC&gQxpBa$9oy0SuDac(oQyaG!PkbzIUF0|rYY;3Hq7{~nidOKio zrhoETd1VK`17HC9&;S7V?!83zXf+8DIKEmlcyz0+v^>f_7*1Y)gb|s_OCLy1N;G+A zEPA73xKOeZrauD82Em_YTFce%82pe-M?()1e)$iq6EG|vf4Rb>OwKNWhfem z4=m1-QP3dEUsm(xMe&NPQ2x`541**MZX{@X_;kQ|UARHkUjA{?qI zJkld@7TvB|M=6IPB@iC(?3-!>mkVDD8j>f29|2c9*^hMwe1EWrN)GJ-b%6)VG-O&lG>+GhU+9ZPI>g=zsF=!!Bv{`=42 zz3PK4m#=BFW3|UdO)O@ZOTb|iulR9%*xG}~8_&*2(`!n8Vmk3yp?l6t-HtW z-?h25#1>Ce625A?j#o-8uTt~;$T0VKO|_uY&9~-lkoq~zv%oqq09#b`g+%g8-a;kz z=uZ6@DsS>wTmXUxifd%?H|37_iY#>j0XIq9D@W0=2aWYOe92Y=A&RGuRouuq9^z z(scE@%^nuzxNEL(ixyyM(wtsC6N^EYgExPI=eBu^R^AqntNk7FS~&&VpU2$)oyx7y z?}>0jEU(XJK9b|~tBX5mPz`Sh{}E^*@tWOEtH@PtL{snc4}saIUTX`7hm-S}*+bk* zE5?dd8#n+Y+obJO547rE!Hz5%Wi$7v^MWZdIGNA$=ao7EDYe4{e1B49wpe!hv1EYK z;Wo|qczhe#g!i>VE{2q#o1_;4Rk<4gWmqoxd$pZlqmbFUrX_x%E^sU6Yg+7a5Ikia| zG2x`Y@)ukG^CaXBC>V$d(F8qQ@94!-V=_Da0`b+1ac&9BAaf�bmX49IlrV0(73h2uTKj6&A3Sr<4=MaLRyUT zh5$};Mj44q5L>+13SKS_LJ86VRBVcV5VeJOJ$En~~74A63I zuLOUg?o=~$*|?O zvi@dyId!6uciS|r!ip8mY1a7=j@dY#WBZn7q6Kb2vl_VYyoOu^n2g}H${$ufpZoar z*Kmeo7m5A-k^osh}j~SGo|E};ZqL1gF0$Wk7rYgH%t;(Z=vqIn3tFB z)lK|C3OxKWbQBEv{a@ZMp*Y!2aLd5;73BE-nI z)JR9|j-^XufKzj1?du<^*(hp|5*uRD!PY1`p!|~AK-39H(cl5G&}t?`^gxw|-tR3u60xHU4Uv^`k_7zO7a-4X!P&;e6@nLh%?Yc6`Fj zMIZgVq6-Mo1W4*}ds`~@()%j?iaXrb{>;5o*3pdM)CC3QJWsB8`mMde;S1l^PQn_b ztK7s+5B_T-c8RQti;g`Yepqeed!^r*t!(h*PXjKgc=6fK(A}vY_R9fO(=FPGfj1lw z0?t1}SQf7JdUbR|35oN`6M+LF5y1ZG#&JAEVArOom=(W4@%1h`&&zCg1W&huXIu)D|XBoru zR0}%A#U#e~6bY2z01dv_qCVwxL#WlDRnb5!Cz#SRV{@fF<0ijdf}DFSo^-GL$ET*9 z{Rb#23p8AFtxl$@voJdexIW6y<{bfb#=`-KQ2_+j@V5WawXG@lIe7WdmaWd9)gQ-y)tU3$OCospi+I-_! z@**C5fI70adpY0Un;3Ug3jL3L$ZK~-1#5x-cyapU3HO0gr~uxZTtt%wl*#rExHAjy zds>(5w|vxnHayA9WD`nhw;pYp-{G@F+BFeiyv`|s zpN&yNkA_0G?2UvYwMBrIh5}$8qioUF0Iy+w1TYFTj1#azmLY?J@892ke!LSKt1(7C zo^GQLoy=&@-%eH{H#ojudT#5CstZ8J<4CX~{Oh&;?z4C7Oet^3f6|_LIV<%dam(F( z=Lo3|TAXYG(G;P3lSqr%lVd&`J1aED%-dfraJ6`f;Su#jrpX)@JR2J{Hnb54zXO7| z1~QBM*m1{&tT%kOKYY3sd7wWFi5^@e#W> zB;Sl`xC?x%=Ji%&ztL4xYnu1R^}qW9VB`&;E3sw!az>p6?b$G4`cHqhllcPSpoNmKQM!WQ zDB26W3YpRb+cmDo)YLG^-rfjKNqVQuZgK^vsy;o4=MHk}R(*pZ+PcXq~E;cA8nP56e;otCK|gfE53ejzh{yjmc* z04Q5opCamtpS+92aSx5eFnaM^1Jab}F2et~>rlL=G7`(R=X`7ba{v@YH1OY6YG|b= zV#{}d3&8D~;U*!}?X{yNa2M-gMk3`TESMSI6*GiFF2QWg)nFezkbeKN&& zlyfrZ75DS0EuyMKvO6?sRyA~u;{=(*r!tH4^9a6IUy)l#I6wu&LLI+8>F%LCkuw|+ zuBt+P3?1f1i;ImjW9vKRe?-Bg!ywU#k$c5TJBCN0Pp zj_GFTgeeUOA>`Yp{k%5lA5V)<-mJ>nva+Hni*OxTuh3DKr&_*R(Fc14(IoCa zee9&(aPDaNW-!$^jV;HL)P7_e3Ats|eBV6rX}Q>RW|Ck~u%9UdG$|ee@(A5D=HR$} zTYqGQ#W4(sk%`{&L55NR!N%lqXPu0KyCRWDJl=Jm4J`(gzI;*kj@d};fZ3 z<mucIwMrDo?}dU&=i5VqpCxc2Qyjj1bvX~I{U zH^SsfyOM|GNmuB;W8GDuwJdeLfm~jIm7jqp{zmM)$?J_ZG30avqN|u4(tfrnlpATB zXV71maO(ul89$S>h2N88j7S~M-gvjAEul|ottPBKSn$NG2IfjK$hs_!_&M`|{ z0R*s*K`VmuXpt{5a*HGG{iU(T`5Fg4ds8Jn?NHKy?e4(vCBOmS*_w)<`L<7gn5glo z%Yxa-CAfzht}SNU$kQuj*wxEt$eOD8&9cK1a6sJJm>6XGvF^kpJh$r3Qz}$=cVm_D zXq9nbOmX;k{zw{P=7mU7Iph%{#_zo+xaPhT7hYZ%p&!Ic${=z9CEN40kU0kOfn*zZ^@RZ8l4dE@axMkw18TFtt#Mv}X& z__{11P)-c{Y-2s9->>xG^}BBej}eHY*VJ=bE&(0@0?BDu`8L8iJDnsa9j0u=9ZMK7 zn9WJwh<)|<@ILvqPh)YngK6lt6ZOYTpmIs>+KV!GE(dxop)!~1>}0HO#t+F9?)30% zNo;YwMkFm=XW$jAS4L&}+Th(D|Hb0k8&fsUD*JM@tGOx%5G^GDUr(6<*s7rPZM%+k z%331Im(nN}Gc3jv>p4O;*Y0KB1&xA zy@ExpJTI#(#=PO%+6?b4qW?HF{JCV&s?b2psdz_}wVp+HB9Q0pegvcK)$&4-puXM< zCkt6Y&Q77dT$F?$q!ho5%+t_0ezNK;G-rj7*t>C9Wj@r6n6b-c>{j0%kD(q%sj}B6CUg3*b)9B#jM0^9)tOK z^1A;8^DSFvP@6xvYKRnKV|Hg&vB-e;hQrT7u8Vw@F_hg}pOra}Z!30GVuRiLuC#QM z`<>@lbWENd%ygdowg2;0ko~ns_IYZ&kDa(G&k(ouaEg7vPHkdgp_8qIRC00r4)30j zn~tMAPZW@+{l5Htp=k6|xi6%Q&I>eHNl=dR8yTFE#^I;u>oko9CY|lv&!Y#E@5XNB z{Hx|*cS`CbcV>%5J(vHT<@98+vq^D?lYfduj>Nr~A?yA%E{9gv7+Ltc=fHo*Yp?4h zcWFgj3@MhZh$oHQ+IV_5LC&}u1_n~H_s35b=ZfqjLsyM|_we1;IBzt%v>tWXGFx27 z=lP`gX1ffVkf>8qW&h{Cm2Vav@x;mMn1&ArO@jfU5`5$F;Ux)pk`)418BcJ5$NF># zA}$sn;kcv(=BGF_f~i5!X6=0f%42baVgi}p;vHk_A^z#-78j0fZ|jnLyl&{YpRnRCQhu zr_@`RS&xqHaK}daA6&Yc7~Ig}0=p+?Z@yqjiKH~1i-56zM^p@3S-OrNequYH)OTmm zD#z=n`lBCJ9sG0!9a9gpLSifXpc3#(pC??GMJ-XYNhBxBdu)@*#TXwEnF>{@jm{uu zoxbF|K0~FI#(~X410(E_A07ucS>vq^d@#bMK$h`~G$1BIOfNK0{S#B}Zh7zV^9VNx zncEp(Mg(gt@S~d;z}uN~^dD#ItX0QCaIs!8!%ps)c)DpA(4)rdcTy*3k4$KwsDq2| z!<@FJ!y71H4To-&+gpvGCrJ@u?fU)4&L{BN?X&Zex9`4JhNSi!Y&DF}G#s@ACRpIB zRz`dH(B6X=&)YR9`MtrrKBoCIW(q#|jvcaxNp-C~F3%cl>`#1PZP%R2+RYi;x;v|J z@UgvgCB!ot(h~MS`r!xZXS)-j-n&qX!SH3w_%`am$;>(Nj#F-uoIzH(DJTnjBmAcY zh%z|%@!Cc3`;<^+jgvm|%4|;GJK0H0btwke3v73g`n#=_$K{%0q0gKDHt-o2jva|h zJhKx;Ix~zyGTBWomD<_uTAUn`Y(y@$afDXK-SnGvKk0*ps%07yXplX3?`oKt*he39>V^sKFuzpvV#czio7|GFJ7<707kFh|u)QLGDQxmb^)BwJ$dnG9)x zw&o;ix4CP*uPTj<|LLYNKs2XSv1d`7Fptoqh(CG*E?&vqrGMxCBZuvPg0_ZE>_Yas zTo1g`oXu>94ALKXT!~^{H`?A2-f03^{w*Olm*y|c1733=Uf?{tar`%)v{g;L$!*{{ z0!joVKciQh?LLC!apXT z^}Ev1SKq~8vP}UeamGZAktQ9;oA?{)9b-=? z7{L;qo$q%*KS{{*U1Oa*Y8>NZ*Z7q$v^^=RsN6M4D9-5pgN8}cGPQ~7vojsr%s=v= zR-+E=JFJY#_qWnrqixKwk21FQ?Uk%k+P-W^I($m%hr*?z8~XvLoR)4N9vvTk45zMk zuImmI?A=W#-4mQ^$#3-Ws22V8beIv$-xBU$v{%7bdh(a@{fS`UaXBikUbhbQJ;k2N zL`TW)*m&k^&;WfK3**)4`(v{)lI&l{nbGTwe!e`$q>X%YF8(QX*n0d9QN@A;<2)#Tt}Zjnq6Mf!OoG3h!}?D$ zqai?y)WDqZT_t>FiPi(0W#8$?^NK83u=H`CLXSiih^5i=_E()ue!&<+kMn6b4wQf25MuYA6qny>`gANxFjV5ttFrHx18gTfbr6r@aR^%OWM@@Qd90I zm$M(^_C-z8Ta4yq(zzAOAQgB9fk><9q|b7GpE9PTAMjOT0GnS~!z{B4a|vJPD11kv z2#z%grb@WEx(Pd}W84BGavLh&s?w~Jt>HGew-;8;c?_8|ZxQ++S|-=~l`oHM91b*v zhtEa+I`4N?I@T4=1asio8s8;i#Ox#6m8_ z#;dzj-3Zyq47#(%2_)l?eGS!zc`mL^2g5T2d+rCg;=?oDr9th~`o_T2rd7bGXU z&Cf|tzASFc4lh5>@O@lxL+PTcCnTtLxVF6Va4oR@&V2kiI%;vhnOQ6}N95oj#>Z!K z^lbP};}EyUGW(caCWrU?I5{o!8_{Tr=>A{VSyc6msQTvc*z$Y(KlA>mocfN9AD*v# z7l5>e2^$dqdk*fCO+MSWusine%KHPTOw|D*NeK^_fJn06G?e0v3gFQ-{c@QWAk+U# zpx-O#%>FZ8HfEdmB<-`2h^a`EnlUKX<|j$*GSfZRhCZ^@qFvwpy{iyt7PD8yjrTH? z>)M;?Ki-Xn%=z;T8LzMhnVaI}wwFlknd!t^ev0A&nO{Hc_3OK2#_b=qC+ZJz`HA

`PW8BU+~@lJ{iij zF&iUK+OPj*tXP3!I%qS=6h5nEVRSI{TTm!@7|4%Gl_Y+2%T)kO)zVXyq-*XpJgO7^ zT*6qY%b$*xjs3gtJnLKTD0j$+iu2pXHlU>ukm?7D|8vaJwB5kLRaw&@vasT@@WVy< z2hfIRByUgBA;MRR;lapbPC)l_20iGJqtsM27i`Yi)}aHNDpHHj2r075m2b)ke7 z`-Dch;sVMg8N<}!Mtfr)6`3xF(K$zP?cZ|7DGaD1b=8t!bUC*GseajMDsSEZ?BDqM zm@moQBzn2eoTgYIsH*p2|I=ri%hnj5Hj^s6ddB0ei|O&XV0h1y_3*e9k_D5kGN1E0 zxvg)8^i{I!N1*4i7?EC9cv7g*>-ogY7O_Qv89DY?pgLek00R3}Xhg`Jq#xIWEP z-q6QzYvsAmm06q*Sp}KwGZVz7qy?T$ib<6PVmvF;3i|dAn1Tg{-a68>pRE^B%n8I# zZII1^dD)8>Nem0ieyro&T7mLYK?PVU!DIk(%_ zpk5U$p_>zj>-|~D*olY7!}e~^pEu=w_9cBtGw9@v>{skslw?XGz{ClnSfK1Kz+fSB zpQcIu#cWvVm?w%q7^cNgmi9i~UzLB`syvh8(Z9vSJZ&76hvnXJyWKx>1$NCV@~ohw za_l)`_`B!3g5A4}K)U=`he%>}p|cB3fL4_>u2?mz!%gg(o#Fdrf+#!@CwsIgNv0^x7x?V@7=50v*2W8%SaTB-! zVj9i#eObc8RbZ`@2S&05u{vLCKX|e`+He7lJaW&{Xz6hmB|UhBhGJDiDqk>XCwa3# zh#VXVixwZ~cLez(Y8$*E9nk_73j!7!qE3#?QJP;GNA~BLQtU;sx461cO!P-B+Tzpo zs&zounPs7(81cbZ;!d)JfAFvDKLB&zu}yZD`$=iEzz1v>FLF)NoaE8o)Y2kq@lT4P z2cVT9m=arEwZM?j>Lz|bNqUD`bHnqnqH+EoUX=YuE04C*i}D|21hP)pz za^vk`hd^3rhz|u>CY!a@WHwhq)*kyvCd101ypfLGIvwD=BpASgH*U*E;AE+>vIH00 zXV{<^yX|+d53jb@pLu!dQl-RHjV68Zx^ao7+EKo^m_o@|1=_@lyQ^y*s70(i5TpXn zESrnchHkV0m_-esttc~59wxJ_Ul6$(^ycuR!j%f|*2f zbc-XSnJ?VQ6xOe+c%LhgRL(&|2ZV?@;Zq%p>|p*y!}ow5zc=7X8Wm2sjIaGHXePGZAEW95!{<+8nP1&H63~amq694lq}rpZBHt{{@Tv#^MjWx zP^-}iQR;Tif1k{Y)@JB|_)#Fh81du<)}z5zeW_+(k!3sV;i}&1eGtDgKdn0QXN0V9 z#OifsT+QG9OCgK8zI&Y;op~X0Ur@%sF!mNXnwDl;URG6(XfW0W#M%Ug{9buK^6WK> zy9fi^lva}&nyubDlQR5&Bwcqr)!+Z8#3gY>Tq8os9@#s$keiX4Ywwx8$%sDOq+BC= z%ecw7WM?N_GOi+fX5Gw-Yp?rz_x(LQJp56Qd(Zov*ZXx|KSA4B}3{GoV2q2)dW2|D@F9(_v(p{ zwT3ON&VmYq))p{heLZ@Y_8}V$a2V!xM|y!IH6;6JEVG)!@L*?Qh&6u1KEL-ZtWDl2 zO5!`@M#RTiOB0hzHKxJv5Y8c!C#?683%nKLcaY#cT&hF3+zivAvjYM@^D>*Txe}D` z7{mN?goBbfiswDu1zVo>GIs1SY*5~_ZdDyjq4x7h=bICIfaznVWp?8OAJo}w{L^%0 zaHLB}-~$eX3HY)1T~`yto4Tp=r8==ta-bWs#Qz?TA2yR+U7dO>Q+n|Bpfaqvxk}mq zZdzzH6@UI)z~L;iVD_f()=2p}K#!Tf_RkaFDXU&5H0u*KhjW%57I`NZ#qMK!{5+_8 zbd+X4X@d%A-%4nie*J|7YRG?yIa?mTM~}kH2r2@7J_)x@<($(X+%W3vJc&|PUw_aY z{{F1T?=Hn=Q^xp;g!emn;Oys>ydbV-{&8*8mC(EAPuT`*wJ z7t~iNbDRz?Q=LUmoO*r`cdY@;{?!d#soU8Fs4nk>F7IMrf1BMiF+U^mPul_Qobgl$ ze+3MuWw)Xy3ge(}^!5rRE~)W^-9etee=nBGCOe9p{`<};Rk+{5`h0xBN!rX?_3UfN zgu+VTB+5tBB_OmiNz14E_i>HCK2OYM*`I7u10Vqn=fr4!B@XfB(s8lcDjTLxvHR;Q zIQsVX8EtjmTl&4<6W}oZZeYPb(RES7_53=PpU>horkf{Qsm_2@u1fm!E`fE4=Y7)M zxQcRFu}X9{S05`-_{Zevr#iq5@ng54&XOD{tt!pcbp62gdnp0iFX@EIZ>nGVkMqV5 zk99g9#bcWU8M9q9eFV@_0`^bW^zQ*MT_(h2Eu>vUmE<77iKt8NnX`|V;Fbbu9or^7 z7`9x-)IWsya*NJ0sh;o-n?6i#+4tEaa((FbZ}OwoUnQJJ!Tb6j2)Q~8RDDm6bVK~U zP;ScQB>=T^Mc00}YBM9dz-*amvHv8vjX}(BY@9q^dGCfmLu&?XG3x=U)3X=;`ly&{ z!@b|{8`_fnEnP(%5QbL?vo+`7uAtg4JFQKe@fv5b8lRIth_O?1r-Kvk+GKWZjqU9X z7dtm;^gdNlO1_4ohVE1uUdyiYm&R2qW3J<8YW+M-rai>UN+;XtMh0g2mlt=;TRucL z>Y~7l+4Y%!=v7Tle;u(`@ zy7X)7hueHgSUEw?)lk*o2bWuEX9$yVME+2fb>>)rT&by!C|bFJtp|wN{-Nu#xMKH*qf5Jd|wJgFm%Y#My%?QRN7CjR{PwdVhA4LqpCy?MV zTWd#vsl);z7r8DI?%v?Q-rZUln;E(B#52=zQSM&Zc;p@1vhm|e5|g`<7w)7l@2<6% zskFnpY$8sIfDlG4J99n%}>M zp*FL_T=C=npb!so;myhC4>Y*mG2P1^((F|`OnH^W>DRJH5XQdr1>3N;-M}!?aLUuS zWAj+MlM|!X`&qmK4i;&6lRX5xBCc=O*jO`euK%B$!au=?tJ3RdKAR3}`y~grRA}u` z)cKoJN4@B*bSlkKv=;v&`8)P(B0s3y7b_oFUXTZ|LA?VpgiBc8AqvR>4dkVw1MJTg z_T)d&zgC2bTK_sY?CtJLO(|@|jq7^!u|{ThR}WA2BFcH1Dta`G#?~1rX;F!HsTc3M z$QS|lr(gs=M5(2&sq|D)v3-qR29RFMJ{cI0uT0iA#j6v36_Ol1wmSpP6o57|TU1VC zV^&N|18OuGjso&8ttDljx$C;D#rjMX|GkjGS6H4k_xG#bEZ8mm83)#&?m2!bL_4Pl z@U80LCaDK~3I=^l=nwF`<>}V|KaPSNz5kO+SZiYRsSf^CX0*7R#_rFDz|XX-f{~as zmkOuph7S~_kL4&?>+;Qx;fHndo0>V#eH{yHi~iFpizRHb9C04@ESU?)Z?Q+m8=V9M z;DRt10~xV{7v^_=#9WUAw)lSq_EmdN%~(p^vA=FEJ4<-P2T-?WZjK%2!fN*M4ppq% zlX}h&0=f#T5He$-#q;gFn*+x2IGS2pKtVTP^Lg?g@l~!%@$U4?Q@z+AI7~n~(}wZG z5uH5B!6xb~HG#XV&*ue3<)p$J>&&4!sNrPu#}G28JpXN}wZSm8`|?GLWQM)}D^7OZ z>?2dNd;W-9@m!>;BM&xl?Q3dhAzY-gH{J2up$DFzH3VYCe5Jex_lBFk^N!BQKL~(f zcy0Y*h`y9wuTZ$qO>JtM<6}wl_?8+|0HxF-$uT74PK2t&=sM+y+CAoHj*tlc3qpO+ zcyuS8KWC*E`V8AE78d3lByd4&4>LfZkLpx)lPGYJa$2j3u|gYzQh$~w#j_v5CtELU zXnt|ljNtZ5!FYjfDhDg{4&AA}Y+6vP8 z{JRd(=3N2ZX=4gdyUfcw6bx$PDxi&FcIDQI3qe<;xNr2`T>N~nUAlz0{vRh9IOa{tDvU8gMea7!Zx6IkJp!_wPu zHp$=qb*aEdeD=FQ%GM?kEs7-0)l>TVB=)%Vq3!+lmgyW1jm)qm9Lpi6^U@ylkjfJmNdT*>Ay1aT`Qq-*R2ntf2M!tIZIP0D0cj7%5ODW_1NOqr(kH&F`H3(+^Cpm(bn#D2UMzNX4$I0O>HWTE>~+>TQA-j3`6__UgcN~ znWnAyNc~StBaR%1uUFkHj1hUTKcuuh&Y)$Xk{6a=F`B2Ovv9OpbjG*jJKEw)>DQ-T zjJ!cqzkF9=*{#K>EL0R338G?X-xYX(6%g2HD)Ga_D%i@D9`$C9_dvrQcrCS^kCH0G zjvb`gm>f(-s}uToyQsCG?wG36ia;5N|Kx#Q?;;y?ZmCsOc(tBBrLu6GidT{cXbzBwCD`|}E%Ft=J7;EtVtIxR_RxlYxF+o41lhKRs` zItq6$J;N@-P22ZFWIfKG!{oU+xU~iH@<=bjE(>{Y_rWkiKt+QAJ6}XD=UR@aaW5y% zIg4z#m>Uio7as_O-Ub*lZKCYzFh$7d5og@xs;pvjNKyfRU#S70>To(Rr^3s z^Hel^?@A1l{DwhT1t;!LO~iJKxwxl`B~o! zFojKs=+|?bJ5V;h=<3KUHLs8l;RL6S&+M_3I(GUdoPd;PFZ8Tz%L+A_zZT z7}j3CVe0B$?v3NM$JF3PVY0Q}hpuL;)bzQOv?KrdGj|`6zR@*>{=5!B0DgWm?U&g` z@?#`h;RjWZwFwq)-6c6w^Om;Sb875-a+uE;Q}_1A@Zw(0B&~|yL7sBEb7o8HrYu2i zRl8=cqo=1_6l#y6Ww#!jGNbpZH+?vtH)+!|;p+@_#A1i)yR38Ir4{I^u^ea$XM8ph zYkaouX{f(W72nf-=|=dK1O{dA>FLrp>XUTDFny6zZ;XgYn0dU|mM*x7$s z<7#<%j{L8-lfndSpHodJ3j96YR-2j`pzba`W5xJ?$m+=bddSdzVAmANs;{D>nr{%Ox3ll-Ptto zb$Ju}R;&hw=2gC4vnu_8O;Qc1-erOJRU0^2zz*&y`2d8!nCinhV5s}qM*F2$YHC~f z+OXL7FN{%JvPx5Cc(=+U^~$4^UnxiRlTF`)*wj0SXD^<+STtq?+zVR9=)+U|F*&}c zUEPIBdt;5Jro`%aq(nvO;Du&k`|0T6A?}OtdnGRUBOOklYtQY3`DJ<&_TeaA<``=@ z<`!*8TzhVr9_EKDM@1Z?L);^bC-#kAlNY zSXk;+qwqL2NZ{Ko2xVl?a z<)Go%lZcya?y8G=ZJ&%h^sZ7OC3un_3*dQZkoC+Zc6xp-b%tX|2|Gc%ePEP4 zQh&k)W6^XT!xGG$=HTf|tUSHY5lHd%oHGvBXdzaQ%>^O942Mf*xcPDp4b2DJ0X$Th zPr>KM@v<_PG~67~FV%EcS5;62jrgNqrN5i%Yedq5>`^kG0Jq?(*Y5nMt?oU zKNiBGpNzD58Pw{3wmL&huyYBKCGGtyo4Ve+_~jFZi}ST)_q`Od25vyze~Rn!&y743 z^yM^1{OscsYB){q*QtTJVfUx*ASdwQu|SM=kz8*6>JmJYiRTLCIh2|7dPpq~)dNU% zHPwd){a0zvj^j>G$4BkcSJXvQ>ML6wV1NqWr+kB@{FnQ;JO-2Aj5>9GmxCDt6*tx# zG^|7t3tK}OmP#28Dy87FHS`ns2H$i~Zyi#J&AG!`wVnt4Tf|d6VtF2$pV; zk2R*?3u8CNQS$l~57Y=i$8pSp>&09s7Po3**-!y0q)jUT-%PyD#kUKeGG-`XrO<%_ z?Dvh~zoq=mh)H%mnLW4S>DDy79ZuePFHlewbS=Rg-1pS` z)7mg*SZ65q^eu#TV?MN=wBX%dT3P^^HX7$AaypCTNW<$0GX%y6&)L7)o`60-aw**x z$LatG16Xn%`id^{d(;7?ShM~Zu|rwwWN{6Mop&M)L(8!>3Y4Es)KyeCw_|!!p7CCe zW;*Ad>wur~0qkoJB;xV{kr6I-<%%VHSo4+EI! z49%g8v&Z{c|Gi6yVnCxfTd8K$aB>!Qk7`S^l>?UM@Q_0NWb>x|Bp|G4f90B49=iXl z-FP$H0&!J<=sEu%T;sE`$%z`cmhMvN{i?tXmGd30o?lk+*l9$Hk;96)ccSYK4giD$ z0(eG-2VS-94J7}}fx$rwy^$2dGa5Svytn>lntH=LDfq$Uk7`Q`BU5&aPR-=s0Shy; zGZRPv9jDMCbo z`-8s{WK9X1x@^u1t*Q`#ZU&a@(fyU#%3?W|LBrGCE%ZUk{&a(7`xH=ydFk&YHso`# zwM>qZD7;HEdE_zJz5^Tu?W02?msQi5I43Q!G|@pHf+^2?)wt(;Ci6c{1i62q`L+pN zXEzlsR(^>C12jVNJ6{}CRr&}+AG1P1F=Rs`Ng8kAeFFY-h2!*-yc^lS`o_B(a7C?# zP0bd}C;N$e`lak9_*$7yy`d8FepgxUM)RZXKjnU%GrjMMdLoppOKt~nX_b`wW1CogS6cta=dVWg zX_VFt!^YGl?9D+8dS+jopR@BiSh$F6m@=!YgKSW67#{Bl%qQ;(dZ3&;mI4R5s%-Ab zzXLwqqTE2i5eQepuOAI}0d`UyeBE@iQ}xKSxz$60Ep7DI)0;u|7>tYj56gi$>POgX zoy-1tDzM36=#plh@Zga8e4D?wifOi$YF|^M?dMrrZ}p;@n=48m z6&2Mu)(=%3Svni3(fzYht= zbH7a{dF{(bu@mdvwjF0+L3^tCzzXtg8`VpL6C;kg`s2 z1p^<<2s_a9I}+%_)O?w36Meiic%;fFnwb2vKaHeY6QtEg$|^_KRg~7so2h;Z>1Bg03SI=Ex1 zDFVk@m|RixU@qv@@&ohA#cXNp6NmNFpVA@jGjh*c!vxU6Wix*ZJ?yeH?!a2*0Q>Zj zxnXFvx0}Tb`35O|XKWhX`8w=O<$iJ(%6D&xOSrO9=5j0C!pJ{@A=Uq+DWba4F=u>g zpxzKjRpw3pmFn;YTeVFS+C7oA{!^v(zv_EuzQiQ0cDyEMfQf(7OwS1+)K!@}-xpfG z4TUI&M<30}dtx4FWEIwbJq%NbJovkl&1r`MWi$%xj8vCGTrErr9Yff~|H;rj<`8S! z#YujY`Y2^=?j|A{Mgj^H;vAcB4GH--{om$7?!^W_XVt+9INb%Qm(_i#-g7Rj(p{5n z-s(xZ4*+fS5gB6v_4`-j6Qp5DFj%Cn)0+;L!a5m*JDmqlX4rdF1~#|vw#ee%kjkrh zQBU19xj%#V4`Xp#>^Z6@G4~543X{1}PG*9{xje&hB{H(( z?f(_c%#7wHiRFoYOmn=&MO;U}7KLPY%aI)lm~g`Qn9kzU0WQCI`lnL{k9^^)SaB>Yb!;%V)63($R(bZs(DXf&v&SZ} z$f)A{@)T{wAsmMUuOGEdO zZT1JpM@8>GCi{p2%=fmSFcbq8!ib{2Cln>mT4O*C%)4wPiR%_qn!_N5XSboI0SMfIZ`#O=3!ZtzA0Jj zb6oqcPB(*hMI4M0xc4}^SQ?R6gLU4TrM1Y-i}Pi}$UC+8f`*jFGq=lv9NY(azm4oY zo@dLG{Uq~AFy^9VLTE3y?(jtrSMh7#K6#x)eyzkeC$2z-GlH8L#5cHD2&NpI9qol0 zjQbqEQ_+}{e~YTq{M_zOfiuArNjOTCG`E({(i9|cBl)`-90oPM$mxUkeB(bgBmIreR_eVGN!Pw|wmcm;E254KVA#9p!tW)2@08j}k&$&4IQLB7t zmH_`pG6(+)?>dn@sMc^ZTT0lMkI52!Y`25gD0OpLwsw1iD$#Yi&O~gNp{mYvnF*jO zbap-LK*kk!d#15tHFe2mNIq}8kAvIywZ0${ymf6@$uEeVbqnE+@s%9(5$8^Sh6M95 zg~U5nh%bEr2vUbmAyNP#k|!bxJQz9M+7gJHte(izmZ`5NH8qn)U(_0n!g0xFp#u`! zD6eb^Ee^Ev(lD9}gCPm)(rq;gzHbd9;k@>XN=vKD2m3QQIS$+a-c7N4ud>fy`_?Uh zI?d|p9{w!M-*e~JV59Rlap`et_E=+Acg%;%zl?j||3d6yHCn`3+P~Br0s==tuUgg_ z;C*zOHML&3QCrA`yNmjMfAj9*TWi{;E1O#P0<8iX0{Bb-r3Z9f_Nx;+@xP6p$1h?eYS<^G(vH{c=}%wGS9AYh76?^t0A?D?9Z ze`WHJrYIMHFbwkj_8ROn$Bsuy?EOkh_dfikN3UCdJXQB{MmCt2?yKgd^eYI2JNwr| zFNXV}TWo6_nKi+P>89$j0bs%bBNXxoBFRU)=P?G=0=Sw4(&{*=hm|u1%#-~5ds?b$ zW55;{r#8K~hD(a4J89)`Y@U`f9%u%XRhK4bZuv-Sj7S$c^Trv`M?C}R5vX5M??!4H zn@)H2!uC{k*jBr&l~O?yMDQw_93`HgEYTrApDdgQ`J+2FzG{u}j%>R}kW>z1n_ zq+FWatT6nHz%_7a4nUxLJ!aF-I(n{Cg^QvwF2t;%BZ4Z&c3@3)d1Ct5I{Q$VIl?YsYOdL@IdF-3mv0bT8rZZ(k`vtLlB@!%-uRSURI{6m@aQI-uc6RX>Jy1*^?f_oKc9r^fYPSkeW3?$_@%U zSF0mBO7iPhIeYY()MK&V`}&H3ax>`rqZ>TsDXDKfkIA;!%VlPpO;OMfqh?spQz1l(Oq0OXU!@5;Z-5!CMrq~0$A&03(b zY{mTA5n8P9a(aW43a(FE6cA@cjAq#1Ia=ua^07a>CI2WjNF`n~Fj%MWUjVa%bDe2jH z$;pH`O^BU)`2)u$-t%wuUL++OR#lOcD+58uPd&z0V*7$+6*i<+0n1Tf-mhcSRd>lNA=iV^yr4f zA_}g-pTQj24dTY~JJgo)jd3bO1G>=#sX;(*OUo&zsz-bL8z5;fz??0iBI5N1xj24N{QZSGw8^<+A&^4!gTr3d(9fo=Q2zB?C0h>8SQ^7yma=Ac|aF5M}{ zrY%Sd>|nmb4<3*&mQw%A#=e+RgP{Jv28lk19^Eo1eiIwFAZ_=x&V*FoxLw-#Z z*k|T?BxEU}<}Q;~qLk_-wS_S{X2VU%Yx3Eq5=>LsV%b#7HOF^o^$!&vEtMe{saNbvIQnp&z+LeA*?I#PLw~2Fn}WC26t40` z^X%>pWhsFIIqxOq8}{jTG7vuT79~HxsWVZvB!FSm_mKL10PBH?VlXl=cZ|2mZ|y$hqO|_v1-|{aA!ZO4wFJx4yOZy9FhhE33*17_1HUZ>eY_cv^9? zF!skOY|7)>$OuCO(CrOBEj)M|zvLk%_OxWMrdyIwTtY(ma=$FTWz{@a~A^Ql{JGc>p8ZJ$Rl{5(g zINZcfz&r}vlO4u zCW-G;`p^5^1C|&cv^t_-xa{-ssulno669xHg$E)DWs6pU|Aj6Fq)RFsBaJ4_!-9s zh!+0sSlVR?Y=XhiE_S+Xf3!OTah?2eid$H8-C<7yi^A5&Sz3PX9;vuVu`cBbTkuR0&h zOQi>@$W00G+r9yEm|w=D2g{df&WP&xu=iQFyN){y>56K4!920@D(g+9JElB9vF)yN z;tAqGJ8J3C_$)vW8h0dG1`%pPl~UI~Uj(<`V2mm@OT3~k81_ksHhn>S+2-z|TwB-$ zp?-RtnOS8~=@w6Sx;0ARmRL5I9SZ!eAPDtNQ^mb}ru}FhR+8;oEu&eIHY69jDDGan ziuaIWE6ctQ`qPcpi};cgSl?f&dV@>Bi*gC@v0NH6dm0npIrCxP*f+ro#=7_!L z5d)Mdf>#^Yt`8+W130TR>Hfvc;@Zza$%#u^hgn1Qh0zv`zpYU}uMUaQPOj8R#Z3%D zm2gGyF-Q;OPqyfQdVkDr=>6b7Uto&pO@Q{q<+)$lq>y3++!c=@-C9J_&|G-WkCdEi z-%0Oejp%Yxqzix8g=D*DU!_E8*=c|2crB5>FniP^+I+TMeZ8Na*)sJ@|5I_{2}f#5 zX1DAP8OQ7%MkOt_l~e_zil*JJqxeKA7_WarQnuOXq(UEI}|`2P;#wOw=WDQxZ)#-n?3 zQD?1kI-P!l&wX#a9#hc1iDt@=+5VupyRTxNM~-}5GycV1vqe+YeI*9)TZ5lsE*%J$hqc9|>GouPI;GNYOAv?EKw5oXhF1YBCy_-WgbBur_+56i)<>G+ONBMu$5P&+7diCA4Uztu%;A=>4RKh&L zo<1yUvn6G--KjGQE&U9RQ^`@j>|XB5|1wRzbQd=k&aJux-x#2NwBOhs7`S<4d8Yj3 zS!8PK>Ju%R9F6mZ@ujyaT+*qFcTJAA-*7+XFQB`N1poK0ZX0is}F%nVLyyB z@9zIzTYal4kQ+`|j>%JX{&m1auOpdJv?xh>5VC3SD-Nm={*&#l9q9FPJA8a+@-;Zu zB*pYT5`1r>TKk(dB&v#%&stoAZkn!M*h{PJ7$JbM!`6+dB~Iw6x)pQ#DgQnGd4sI( zj>VaIsIksFr#nymI7&bF(5a+zs6q~dXVoU?#H<2+QXX&vvwCj|prz!X^vcNlM<4s- z{6b$SF08Yfuz6L~7$2CHHYVH-mNPejAb;J_s&EJb9Go~dE(TO3{I1GhBneYC;xuO` zVB8*_37@aN0+MK&>K`{1yeA(oPcN^B+qATDa45sFRx8|qY91s; zF21@v4c+>@hQ7RcXomGQG#)%+@Nd+LM%j_LC!X4>C`@8tt47IbI1X?iw^zCurDY-__5o z@PE{e7O#umAV-QkWzZI#U2FcYcFY9JO%bVf)!}+lF6o zx;5xN_Ll93r&dYXW&c2U)LSxw?&PPs<2l~aRCSPdCV-WmGLn7UWpoKbtE!~7`&&88 zZx7=#I8@AL4y%)gQ9pWeT{sjcmBXrMx74)~?|Kge)&EC`geN*sV_$lgoYChgG&wo# zrnaKZtxq`L3_WPF*_kBNg!SgiO{6Q&ZUl5n=H!kmilaWbGmcM|qh&h9QS1@_Uff4< z07UJs_`pzV^)L1X@luMmG)GI0CX<3-ky552=8rO6kUJC$JM9Y@$yBc#RmpzRJSj=} ztyR>?sTw2>9D%cJq1^jLsXk-AvB@{5aET>}C04v{B9k1ToO$;OIN$b`Fy*ZQ@v0fOUnzMA(@{pMCEv}q3aJI+v~-@Hw~%S(Zxd{5p8!#VRBG;& z4Zf!_Zxf5k$ah-a6XMmPMcK<AaR@82J79j*ai&i~ZN;-!JtDA!%MAT+C3 z?^Om7!h`zoMx2$?@2jcAX2nN9P8s>fmI;%saw@{G3?}F#^C#x-ARdahtX5<;eV$~1iE2GoGwAH! z?~FNOWwhzy3NMvt__^hGdJz^+MAi7&&0j~`r``j9p9qNK^weOm(`Kg8RdG@_laq8? zncFU!wLD$U*Qk+xl6Y4OD$sot`t;xdy)lOKS!?^{#`EzVT5Rm*bsjWB?Of0xgYKisMlVL4{rrL3Y~wJWTNS;^gTMkz|iMf4nthdx5@Y zUq#HnPFIu>gkN@88<#A3A>pXCPX5GE8G|-B*bx~ zy`d~{kAKC76Inz=TjrsXOqiyzXqykZQEiKK%ZR{&07sIFCY9g zbx!!Cq$B_RgB|1ar+Rk=81ThBH>7_4QyKtnS`|0W0nKxPuVm1PYKwWXE6WcmIHn`K z(&OqncC&_I-dOpVC6x#eBlNTTG##3`17f3e9>ot7`{7yw_Sj*mcSHO7oD$r})<4N3 z^|^WgUpw*S-_3<9iF3-Kthb1!PVkvnK4#Ox{0FCAf(m?>*8tIVz9G_l&ZdUc8p58U ziww+R`N@`Bj#IJuwToBozSa_uz#hB7qgBM7Mm-Ixcs@x9gv%D%6Fn$iV55q};$0xG8;u=F^g|3GBlSF9A6#ke%(Mp+sJuS1X6z zgPq&3FUc-;OPw(Jp|_kdbqef{z0pX{lbbO%kSOog2Y_;^k70?1 zufy`o1Tf~-M09N^y7o%u(`zy>YWDs}DDfc{DhoR%zs{nz5cWs6|RMo zz36lTqMVtB8pdMX;<$(WZy?7(nL^yQeR-aoi}>s*q~Dbr46nh^qPmSU|8b$XTEj$d z733Fu+TIB@>Z;Vh2k5u@Z^j~oGa~_#3-C#h>HjjMAiQtEy4>Vp9epfd=}}r)`hZu( zGITR!HW6SAkn0vjW7Ebm1Luh@C4hF?3_(M^6X9G?K$Hz6ms&CElk3_>2Ye9L{~HA@ z*71wQkjS`mo7)Y`no{Zl)PU6e>Zzj$`1jpfLt>V72h~;j z;P1Icg3NptVu6Ru2E9}VrZ}SZxf}36=2Qa+4Liw+3Qa-(pC%pkjYUmbQe<89o9lAI zOt|ZiJu+=+GY3>J)@WZ!G@tx6=*+UjINPh$`Xw)vKacqJVVl!cOa~&<;NRw8nI*|1 zt5`Y4pN}@ZK!EFTQt~ny&JbE_Sua<{Z-Y5p;>OH}7Y~q-Oh$>|&YM~lMuJTD`46IG zd5}Kx^fv23V;M=-i`RfBvtySO(84-?Lq6>P5zRaM`dI$Ckn`{8C6hNk8s=Kr#Cdlp zKbjNSwvp;~U%W0+g9d7^_}>x}_ddV>kD&Vza9gv9Pff-<=Hm}OWC4a(m6Nd+QN=3G zow~#~Vd#-dDgR_(cnqWls#+R`%Ga1Lenk%Z&TfM8OE1G_KLo z#C5)2IrMNg3T(>k&*5TLYOv3N&tSCwpE)D|{z&Hvcdhx5h>>$!gh)-24N{8i_fU}C z^MDF628HkkGyqdD*fL>Ak0OjZ%T%in$?)S^imf*=4&sku__ih(K*oxCZH#6UOdS8~ z`1h9MfH2jKvp--4VEiD!(&U}MI<=m8^?YiD?pCK&SjZX)$Awt__zhzdK)xfS^{+S& ziJ6~{NOmJ!Prd~X&~#!?)2LL4!$vc{!5#r+RC}mD3Ct|`hTuEE5pCF1h_Pk$+_<9D zC^>kuy81Pxg~*#!pE4@{LbYTbTa%Th20JeP*qrI@&^Bm(#7~ULm}{Lb%@4G-Y1tsx z`VCqrQo=tcv_wi$rd}1)mo`VmZGd5t_G)T9-Vc2%(eXxnI`4+vBYP7{?popJjT7QB zQNRujGBVNhc|(5`Jl8Z1QNIf~aTy3Q+PLfb=K4#Iam;Jev{pY0!BL;aP>9Sbo=dOh znLs`o_)j;HKda4$gb=08e0+3ZpNxCbpg+gPXoH)?>p3?4PTXjjMy)9CK{-6>aGpd< zwg5<;g~MH3+#py=aln37fstzCqhbHW=`=5mbR9GVKbBa%C#(OhS)THEY4YrBvmm;_>1L8l zkCW>JNy;40HQ4Kxr9mzU> z;Axso%sV^D3*OnV3Icvl8uLg3xD&(pUU3f!;4+Y*2iyt;2Ov|trL)DC`@e1NE&c6# zwtZ(sp$lRu%0*2EcsEe9rN7VKp^F0xU`-l@T@?laI=XY)O$g+U17jx{8AuJK-?%*2 zxS8$nNX=dI_< zi-yZSG18b&+nRLv!8q50ATetvsV1M0?>m zkN4Dj$$Q;~@f%H4+>d`l>xF!88PBWjp@pm}qJVeoKVnW?^NZHA~w9IyqtfJckspqIo0l5Px|mrWMs^BVMj;dM10s~v9?E|jX7`4cWXD9@gw5W^HL>ppL>f`z2`UQA>e6Ix;-VM z!NJWRX=O%(obZi4^v*Y^ z+AmW`=TBAJ(T^li8@VPn;xg=j((Go6ppQ@Ss#dj}(6Bw=X)Yf13L<7pW}8(0;C;%1 zkParv!?w<$p`ofB5kvzTnRT1F9sa7P=nPZ;@O<&rvksT~dlU5z(__Z1U!7Revj|-h z&mHMcU(_m!kbP^v2vqg`wS@_&8Fe8eD@|F?%Y!nREy*ftUD^lWpmS;Bm@j;lFEce>TM^PuH*(`cD6+=%MJ< zgKh{P4{Y%#QrSUdA^+CY8K7*z+E1x`JNa{7k}$&2YO3iW@{5C=YVmbx1(ir&*Z$C* zi-UknuCVK()i%|t6MWw zLOYSfJj|?f{5pa)$_54a85G85Qbrijl#_`AuD$#acSiAYMr7CZCnZ>4hhhb_@rlp9 z{;!~)mr&O(A$IvRZ>S+-RCepZ=x0R^QNe@hbeh zxnfD>NdD-yV%UB>Y56FMeqWh1$$8-RHh|-Zs*RM>PI}ue8Ct(|Kki~T@4THiyaTuh zdH>~Ru0U^Xqp>?Ndk*a{^9=4t{nJz^2~|Z2=6)<(rtpU_LAB0k2`)ciKz(- zPKJd?KL%C&cV%_&a9s_=U7hpMbk?Mji=VltwJwdeUf9LNuiSE`fr{t%`$b_psDMMT zYUOn0d4rW4sdAoF&vp4Du>I(4{;=~6bnQ|2MF)NOa;3$1$RDEBUT)s;c&B2^1yxwG z#34P`g=+XgKK)FSGVm)tNh?0?uk`A^ z{(j+K{kGkEy#s(;+3+Ey+CPeIgtVbg_tL_9uqsZFHx)~^p`Xkr&MuDwNqdjD&X+a0 z#8>1z1?iP`0*~gyNuz`Y8mr5_G^_KOX4&NuTD$&?Ta_g-PI@2PrXi3u0oko7UDNhizhy2LD{(HUQ1?KDmEgU1`^;uS?--mdzYr2){ z6XaPUz=pe{uQP_jIzMp6<*Bd;rXu@UzhEXCztrn@S)hv~@X?3?VkmUF`|H zsrq*3lwAM}wUA94(j!dlhQi!{%($CR$Qg0RZ)S0SUzsuKh>2z-j?ud1a=r}65utqd z_t?Hg>WLjul_Y$MpXNK6%)4v|KmAYj{6h77E1+0fQkhtz3V})kojLc z2u$>Yre=x#eRunz>y(GWuh*JaGq7gLweB}~v?Do0Mti$NLxQ;kl0~N5Qtx*G6m;T= zAo6|H*J%rb_xH7|cKiylp^tced(VcJ$UQIFt{gw2QX8$VSbGda2jru$xSLZOJ+Y1z zqwhM!Q~D@w@EQp-&!0(>R$GC!)c-GXoR0hV+G~pR3m8K1V*@Hz3|_9ku)Pu`1WjFu z8igK6aIFU<{9oDu5F%qVhDjb=w|Gei={5*a*x)HccZ1!nva#v*E zt8M(&z@L!|O0;)7^uu?#J$L*RmHsbSclh_f%BU**8GLMFS+s(QhDwUgAyv?Wjoylu z2PsTNb3MKL?XkhZGni91Gv&-=_ZHoeOhzEMs{T{cQbMt<`+e zsth>kII50PHn+;CrHMGZ_|fao@4|LlII=h5gI^zP&;rMfH53Pu7Yn(HtgQ4kwa2Ho z2Zn1~2bw;O{^|zj8|G`TKMxKqZnkSr>-Rk7rg?J{dRC)NOX-eLuWaCFWE3|asGbS6 zX)&03E_nO%_z~WRow+{eIfh9%25BP)iizpBDU`MzNUi`*RJ{H*u$5a8aMMx)ZfMgy z<-;o&2M(bQomWHLAcihT$bXVD?u3?WdfVkXf#LO`TYwDAlk1t{GPiy8&D9~blf@_ThWrNnt>fOC!dp(lup|}P zf6BXL5hY8@Fm%Rk#au;|Fio(5X7JbDs?v>BU@Ev}K@rW8HZ|}Kz#eAx0q@SCt%*!Y zoMeAMRzJ`7{aE!`%U=;fVMNF7he&XKSHH=qM2JtCY|VYuIz}F;;mBaig{Iu(f^jqE zh(Lwn!mqrKKNUgNyc;Bu)E%k#w3s(*Rj2L;r}7@7i^e!{O0U|%mODtYJM)#>a~B1n6%PO{?-g{X&2R0b%O~;8IVQhvM7Ut; z!yS8kUF$9y&2Mv4Xom8DBTPGkr32@E15)}sI*NA};g!=*9|RCJJlZ_n55T}7dx!!7 zI?z7?`*SEs@a}aB+j(c&XbXRINbTEPE!}%3u2vs;u2iA-r$l)!cN*60sRoG;4kD-J zl?6q9{CgHOeV8capbdQjKu7LD?F%`-)kC9oL}Yhd-z?%xFAmq%LWELSkv~7Dv3$#5 zBo|x>ZA^Xyy6a8*9}-OaUAP~au;;jb72_Tw7l$Z&X{LM=C>m$Q{Ik0kUvXr&({u?{ z(UI>F!*aJo2Kb&5$!3HzAZ%pIUoX4plyt@(ve; zI&K7Y&qQWHcuaQe-D|KKNTk1J%N*V0eXdhxy|p`xI3TZvkC zA+2<{!h1=qXD#Oq@D+-D!eoWhR=VJ<|Fk`N6BUmDag)8K;yiA~aDD_l?BI96Jsgm8 z+po~jauZTRW^9AJ7N5P``(j&SFN@nYix#hUD}xUa+6HsyVmB0Dl(;l*1KHk{R0q5d zf()>!XAm><(9$X&6mef;iZWK>UKcP0Nn2+YA-pA$a(EinUA>9n9KWVCRPsUu`9(y^ zZ1A??uNP8Otc>tE@ef(`{B`_u&BgVBf&a(WR|Q1%ec=v>q`(jY3P?z|igXJqLk~!I zNOw0s7&=5+Qes3zx?5U67`nTL8oFWN9{%?}-WOhY;IPj=d#(MgFZ^Bpm1!9)c_Qrj z+wSjf>OA~<2)EzST@Py#xAWD$1WvOl9GGf6?@xNmnbH>-s-VN4ym=T=*io*`R3NG1 zUB_Qo3|Jt{7`}@5HBw2W@;<&pOK;9hNJ#MRm7e&YXf;T5cNFq*YYyjATPUO_foUJE zV}#F>JUnGxO9cZ<#jtmDxgW>rQBjMQ`(8vJhi=oe+1CL9rOQHbYeJ>F8k~=pE`%s` zclr7gV0gI&N!Qo&2ABZ+1`THKZnfONv_zEdxE{=qTvlu23gmJnKcK7Z506a-nDC2Z z33SXD?0MjW7TvGEU!B#FJ!h08dO`6$(Ierz<~isIj0)GmnOJC+_r(I?j$2%jb*cH*Qa zDwW`;nVw#gZx;Yyey;VuXW)-fkjAB*dR)4@dF*p>cUNR~bc<)~4dM6(dmWj)tc(A> z))Rs*KL>Y!$@c31di>sjwVCAzwg{%p*i&-ilEIENzZ^sOda{vC0HQdNXnucu{MF(B zK=Vd}EKGVBnL{;XIBT-%WZy-AU{oYAz_dMOL+h7sZx2ySEQ*eL?x4{-rcWTJ0PwqD z&PK*e4P-fZ9czf@RPkZgW30j~6!96tcf*nF_FcSZ_$#9o_C*-OozSdXsD* zYD4u>ruJ+KfbZ7AEHSay8nH+)gT9+S#oXH%SS;1^^3|!Pwzg%r6wp1D(c z`nW=9(sFWu}fK03zRV`;AJL_`>KbkDejGWnF=TKGli6tIR57!a6O)_WqybW zg~-Fy(HaOXURDo>L8?~01%iYLU+t_=rVNIK^HKCM7Ic?cW+3_M;u z1!9~pu8+1GCi~R6W2&Ky@{i4+?A-8C-C}M-`(dyYES&WbG)g>m^W=P=7QP_5JC@j8 z)c3k)YUzLP2m3Sdez5s6d6Ji5?`w$%#yrX_KG441It zW;*AnpV}pyODha6@_W*_2+JOavHBZ2gns$(iTM_>FQ=JWr+H{&}!QHB4$25s| z{dCB>PEa-Eqr0zjS!$Ttb5^_EFdTZ>q~$+#fP8_X+0Pj1TCVYZ!lpv^=tP*>gpA{% ztK$Xwi&EcC&aYRHw?;+5;&mX<^Fqsw<88Yuw`MQ=>Xd?{cnc}clCyI65Y6Wx@u%` zZnvxQ2CDwdhiDn6k_o;~c9%FMH7^Dusj?q=ZE^?zQ!+)|+F*U|9O^O+Pfd=dfl#??9PqB5!*W{=gs?1(X8*;^cLhdm`ss-Z{xRq_N6!J`tR>jw*cP?OY{%1 zRIBXC~I2TfY&o2|Qf*E7j zYiOp}y$nar@Z;1^(RsTR7)}_6t_`N(q~PyvZ`NC;Yd>+{P{?^3sZ63`rYfIw;YE{; z&@q{$o#9xMTV@ipj@;0IgOf<7DlnJ&H|GCT1^?k*r$xY%KK8!9YBu`odo_P^hZa=)#25$Lv@-}Gt9r3vw$_^B%o&;r! zzqMRCc@qe!i^$&)dSbqY#X2^9QQ+|SQ%fKXV)&2j&n{+Ke#Rrm>2*e6x??oJk4V8%@k!Gr(JgF)?dy7GcdR~p$UFB>v55LXSEsA(() zzMX^4vintDzq4qg|Mi7i7p<392=Vj3@7hj!4|tOYUP@E~Ye@L_VRt&Dxvp*=JMb8Z z@hXYLz9jMZM1EY(x~BRAh|zVu=N;4TB2i&X?g_F0ICNkavDNFp=Rc7qQ@y1h4CtCL zoW;$3`BMv}rJIDyywsT*e?RS^0%amnPp$|)+l5FA4m8S-0-s+BVI>!2jBfy2=lQD_ zaOGZG|IT;rasC~TBuNUv{nS)IekNmQ-Dzl|up6KGw9 zi8yo2R|9rU4#a$E(vC+b-=;~@`HgkLvCxWqLN}fQ=?hjJniD1y=MJ-Yb|5mbR!KkH zCwoJr_CRf_LJZ+TJ8<%IclJL=Bqr;jwFJ^@@v|k3_~8uVVT}&a%aXUpW%B`33#iX0 zZMRK18R7xA=)lXH1N2p(WL^k_{pj2w2l${X9DsA$0h*?|frU>hMT>g+)H2FKm#@3t zRIySK?}yHh`uNO8Ao}1S-Tq4Y8uA+YnsV{mkP%T?iqBgy4ff9WW@QNGU=tQi*%6l1qjQVh`bl(9XGV4Zu%( zmY7Y&G7tefWT2oa|AD$i;Gsc5E;ZMvCG_OdRgIB1;5R>Vt%x~s^zwtmVXLIj={ni- ztw)~oS8;T{7uo0=-WxludL*CsHcsHF!To9E7v3QR8f_^-3<77~ql|735a_kSxxx1h zQ2&vEw(SzhOQNY00b~tuC69R^DrG45Xmd=Des^ycZsWJQ7TgpqA5Z%S6#aGJDQ!HY)w^O22l$ zEQ_xKdPrYzuS+Mb0;a&C-R}*2SC7dU6U81Um}IlF(}O@{vi33DQnd1R@44jc-rJiB zlk0W46wyvg_i-vw608MHPrLjs3CNvGpA;F=X6veOa{4{C%cR3QK*|GUCgn0!FqdVC zi+Gfc|IO(uF1#8Wq^;dZE~4$5;(}M_ys5UzxQe^_gTL_OS5dP0rSRxAa3er~TBxRU z-}`F<`g8XPXK3MAsVrd7HTyo;HP9eU6b<+xj9I4}$JAgid2Q|9-xcND9kkuI_HBQQ z&bE1bal1^1W{~-N5wc>mbkWpyQWSWJ3;J_{hIfay-47cLWeeRAv|aD!`?dg3NB;jB z8GzUbJq5Jt=OQL`69;G7^%x)qk1VQO$u4{}WT5J)WNL`Ri`fnR+)U0S*KU_mEA?nnmp9?y zU;_3})3i?-)gFH2M!o%LJZdH{xfw|wqmT;_wAdI5NOd{kuml;7+yV_LYfun?q{r5S zpS{PD2AH2PV_7gGE}1MB#9&VX)$o|0{<$}w){|gU9<9?5QzA3jt-%{#72Cm3Md4kT zi7U9Kvz+$5LC`fwV6FNoYmr9LvJ6Kld1?YDa}ot42VJ!aA*_?J)V2ElEh!R=#ehlD z4K!S@@^ELcL}BY(*TZ@0dDS3|$OH%v8#34hye^(VUgZ@#UY<&!r0fl&Mi6%LmA(@^kj(5+l41J>S^Rg+ z>J!a6OVF!#;<}Iwmlo|Oh1ZCTku58#rMeR4)=XL=&kyHgW$-|+{Y#0S2ptfEFu$*S zYc=}$f}HiZW+6FJ+|-4bG{x*45XdR$8da49DPXj=Kwp)=nX=8=hYeZ{KVjBFiVz#A z#v*U^mfzHY@RC&NC!XuOG4eD&mR3|NeZFpkk&)tqO)5wG0TdoT*fiRLVuPoV=c-mY zd;F%PwY`Y*^dT^*r(s5>n+Clx!onR&CVGUZwi5Crz>4oPc{W)k*E!_(cBnWPMvuv7 zIfl2C!M=Vj--5I3KVEA`fXUYwWO8Y(hFBcArY82M1s(P{-gxx7V}O=!HxGndwzOK0 z{=VC7LJGEAzeVOf2K^cT_Yasfw4K9w<5ZLveV0xo4+ABDiXCG7&M+OQGbsQV>-=cv zym3&vY{s$fIx9QE_w}xMh{wERXp6__0gFizv*IzqtcJns8X-Z8mh}{ZRUk3>ZKja> z*hibJ2sdFnIzwRtpk3bE%rzjyPxfm_;M6>pyLs=}{XJ=?y3v_;-}Aa<62t^5VB7xT z%&Y8UJuY7v5viEHz34IdM$R75w$X8MHF|OXO)@iMh~sB+TvO3{6szAbu2x*_eBL9P z$NH*lNc`krt;48|4#Ku~;sceR%a)=N-^`45AeVOfQPX+h$bM7vZPkb9KW7e?LDIIg?s-lU4nrQ)tg~vI zQMwfgJGXIj`67fsr&lw+O+T1tH;fxzQb+PWp!f=;#j5d$l*pPeD+2_}L-NR&glKu7IMRO^>{&yo%d3l{8ka!eRXZ>MsrslUNY$ zdndC;Vd6LEfom(_aT5Mp7G-`Dhba$*=dQO5{cEj0@Tx)u6C}3urK5bzhB$E6K>!45 z7>G4FyAUgskT^05K)01~@rnZJ=bK)?0&GI8{HV&H9RpxL=Nb(k|6Ag4KpY!P0g z&C924cxp8_J2o+Qc|K$%OE+b}&|IWYhfU+KvGO+tu893_DN=8kb)7F@Z#J`{H2gqtjWwGH=8Xgj*P8 z%k4Ivs|>q&s+*!euJG<>c2?+X^+Y@&C3| zz`nTTBFZh`vN!OUaOrw8)o+Y5@KCgsJ1!3%dbM!FV-(s4P_=FV(igC9tSj5nCnL!( ziE{#coQLgLLlj-u;g|@;g*w%)QJCeRG}38Jqy3!I@J_0FSAy$YW?B0@mYlC^fk{6~ zD|?6Qg>GDv^&}#!7$35&6i2}W|4+Fyrkl$y6!&G}a=jeRJhuJD&a36?eL^z60-f5- zg7BM>5QVIeLCrUPJ~0IFq)8j|Vv%CjEQS7{6jfZXF+d9O9` zk6YJtCfzl$H6Y)5tVvl0H#{th5^Jy=2lh<`ZCJQIb1jTV&uGDGQlO9UK&c>5_XNIu z)kj?myHCo=JyFR%GlZGDue^+7KIqIw=3RZZEfq|o+;Pjy+B&CL<1*)ZYFdnQ`Gl}~ z{J9l<(kj#%l!ob4Gy)lwk1DiOQA!yg+emQKYi}I&qKDn@Q{L|`-F4^OpA<3!jk_3?_v1@4W_V0bbVxpJ&jCj)h{D8I#_x$_iOTWNNrk>!Mj&{~V8 zqzzLcU5=4ZHF;fPK_Y|!uKIET?t}s`2x&zy<20vmz(hu~yWi#zUfbpdr*~td|04le zs$Nbtm+HCu{-Op}Y7|@uRq!m)rM|q}U*!>qiV^oerSg!zdZZ%qmQqn=QBd5lYD>1k zZ6@6BvNJI79)00>EEJp39xid38+&c#e>GqfaO8Th7+?50hTBq;DrPqZh++clE2^h$ zGV!SitZ%H$_PsnSQs1{Q%`cwhS$<@OCwyC$kuNRip-@at)>rl096EG^b&8wl%#b+kVwk8AzlUkQrms;qC*jGcGhS7c=&saNthr$_RbjE&NPeyzPQlr~+rSw~bpu3CgQBm0oH5DUZ% zfKR}aW!OEErsi z-I8@x(P#P=`8a#m0d+`aCZF_V)+CT&Z-`KnNYIx6&GoNVkG8;;H8e>k%$^<)C4mSD z#-tq>|0etMPG7&)lusvGHE*4gBqSAzbF$_QjCW~Ft|40^pD+Gv2wlqpzzI@@^|JO3!!j-856IwD^Y4ZrY4 z+an$xs`MN1P(J`;-IMo)HRIMa2i)0IT#^$)-i=LMNi-AlxR?=OAiO#lFN}%-HpKg> z0KxAx31Me0Acx}R3W!jDupNYhixjmvs;i5hR=GL0wuq+xrsxmVCu$$$qP^?S^ioT%I^LWF;_hb?K6-6tQH1lgvo{!ETNKn6}oNuoKb^ZW^ctv z7NqM(QQ3RvU*2!GD<6C;Y$NT47VSC*p6uYku|Oqk+bl&Y>|H;%MW=L7Ccs#B7tj&J zpa4NewX&B-wfOyX$JY?fs$LO-5U$VAps(%(VYm!YMfNkLW;zt`@gFI2KSSAJsIP+QHX`LZrw5L5%?|$>KZ5!N(XN{7xmv4?66R+w}Id zyF7F{)>(T*DU1|v*d3;KXDHRBti`Qh)=3+w!8TO|X6rNA_IeGZ56C-!-I`{>tz7Lvan4|)xsgzV%4i$k8nvAOs7!3nsvTog^P6a?On z!^8VAOo9S%qXI=^Y(v1>QMZ{2HzIqVhx;`BrSJoAPUmcQ!edB^-T)2izkpUg9$VWw~3u{arSUYuns}^BIi; zg8MWhdf_v9nAqU7l_oW`UD-XQQNDWIk5zlf6#MJou}T>UkWifj{FEsE4V@Xp28|<&)=4IbLr7d_eH(eL zWz*nV)_IXK#jAFQYDO_z=6Z5{d={O57@&YxQdmF;!(8nL&tYY%?5M=T*@fN=)Rpu! zBCX!}La+=fGDLpOLKmH3cry#B!3J&B&gI-)e&}ppPI*)(esyUkdH3c(Tykdn zR+5+dpX6Z!!eb6)+Hl)H)mkbbuxMW=dD=r3y2azXgT@UD&rhCYFfH#-#P6C5`Hn`5L|gBNrkXe6 z9MgY}pM*A}nvD6qQP3jk=e&foIH{I&!st_I@vEr(%r{$v6km9_>2$VvFUI)P1uvLU zC+rMke7oEiDPMkb(HPL2V!srhhZdfkjYX>n=nklf{k@EBY4a9Ug~X~Tx1tRWYH4q5 zjjkC%-z6mWeeSkr5QZVwXeM!VD9}KRvqXsQ|8>Y|er{{U7iP9 zf2nOrrKz%0IAZ6EXRF=aJ`II~*InDODP!al-G+%X3g=?iQ2W~fqX>ucR=cT>>?LU* zuXT1cJ{X&%a`n2pT>Wv*iWMKGf8wj=Rz2gLHzG1aeDiEg(BbV0>XC`xfPLK~(0=pH zA|h~;E?jWGWg~^GHx9K}cRwCU=6il1>1i1Omialgt?{ddvia1)QyeJ_0u4=%u69r_ z-2%}={&n-$!&9x-cD=#i0Hoeh3NHnTjPLo<@nMLOs9>0X-6B57IA#z)G2=X2sL5QG z<3#U|*698R8IQiq&QBKc1k&QnI}|$k1fm{l52JFT8K*0bUe$-jlR}Y!?X7ceDG*7e zexWKG(BJqC0v6i8_qk^8u5*ZGf)95+#T`9OZh3RA=P_Z<}T;8b|O zPA@#V{Z_rlaXy7|E)jVn@ApG=eNoTh>T(I4MGg!{i&9&UKOy(uvHK689sMucaoSFK zbLet1{YDgwh(I3`uw?4f7kY4WrQ7aRa&Beo780O_jY7gSe{k+E{X5%CV|abLTEX6h`V0s$eQ0*s&+V7)BzaSf0eJYb#!PTwjcZ!IWkU zcmPqEV3Ek;2A|A2JjtMAW3EYn^0mWBk?%_jAo)+~oG#3BRAp#ZA7@+WQIAe?4^}2s z%s3o;)x7viZ^xw5rByYH%>V-fdC`nm3{PGKcb3vx8YW5H`~q}}oa~{hGFI%>{I-;o z(Q8}t1%Y;c!QkV+Q*>Mu3JrdD^Zo@%kX=E)-8y#o3-3eQoXfxAOXu$-_ZmwlMdV2@ zn*Eqy@vY`3k>FJ_x~m&OzqvZZ-u%pg$NdgM!fAvytU=|#??Pp?DyMkh#nC*P7WwU6 zuk)URd))_sq(}8Y(lhMQ=G@ENdmnIGO4N@UX2PU&NtWiW`gf)&&mx zjB3{Q`@Y}ye(?*o;F7~5fWMH|lwz5Vob>CjYTo#g{OfI7Jp;zO=Y?vlFMw!I6~V^B zYaKVG5lpOjQ<+E$y;?bL2pgO~ijkpS)s)))m$HXW9wh*V z?!tpb`D|d7N!go&NiuLq||__hi*vSefW-t&ICO7fh|Tk&vm= z3S+Job(JE{MXE>(p^@Q=0mEe2bKO{&5*aNr3EC@T;6?wS3`&(Z&mjGsI61Rb_bIHc zM_X|QO^Ah7WQyL4fJ57hmP0wph5P(_q@&SIk05Hks`muFVi}@4*)=|%ZxncV#T(E! z(y2mx0E8RKSnZD&T~AN1EqR%H<%9Jh+c*-#*IO%vys>O-@Hb5!{tqW2fGV{%S62&3 zdLH;A+D;>o({buW6WLy;Uy)+>k8yIm>}-1DV*Oo#zRdceFN1WJ5@bmqSh=ngP4=y2 zg@!tq!CK1cVN}q$V?al7U3$~w_-*JDZzF-$>(y_mW z(@O9E8a(pWUzq#v)#MKH6nEavuWXaeU0)espXM#Sa8o`PWxZ*+;CLnZV*g7shE!OP4dm5sOe-%>yD!ZppLlLGFZ`!fS@gjvCz5APLB1AT_~B*+n&h^| zRK?XpvoFMG_FAE4QmX9D1)gazTXz!aK(N(I$e;~jkS4^s2#cCX(_);$uP6#rv7iQ~n$x-!tt`(W)0{~00 zFxuItdhVb{9vB_ez4@oAyR;m+I}@<%Vq-=XFW3rdmbG(Yll$DnrK%KoG7%&UNPC4pxMYgB!c7UwGlnyqsJ~MO zY&c{A0XxA9>%{k0WFQ$3Jyu3tq)B~p$B#HAZp4ZBQt=)eDiA$nJWtTmV&dgwAx5F| zMf~``!pe|u@uq1v_y1JXM+~+wzbmXgZrkdA6$t!sTX1&7Xh>E&8E1*1$3*N9SPf#= zdn|^y*lC#0@K9Q8bFH)@`SSZ5^FAoLO{C$ndt=f}LCSV;EJ==GcxUu9tyqSN3?e>t z71uH*={vVyEV&%T4~mwr%{0%^v6EQb7yrSJfPVeKs2~`4uaNQsknW|1O(Z^0)MXy3 zAGB`C-~4<#ocaTW4>tpx#`yz>sao$@RDgqsRt;F1l11`N=hocze66?byk@4bgKa(L zB!-D^>Xg;goy9al42mXEIA#^*0ORp z);*!9BDtw*IMZMyzgbn*V*WsEGSeU|No!sqh^;*Zbs4~&6F6~tnellC!AUuJ-3S7a zw~IzM%;C4&ht97AT<(5(L7HhH-+?uSZ)g8+nPiVdzorDKg-@4lVE5sbrJZanN!gx( zB)V$X*SJDo;OZlA83h5rv|X%_5`}22x^w>5vnKh!>wkT!vmRlv)ITRT67n|Ys5dUF z!q&A$Yp->ss))~^E-IPSk0I>ObZx&@Xza#K1m2tF1RVXNrF{J4ou{U!`{-f&I=fK6 z%@$O2{4%loVf&Z2DE+*0Id0nCJqO0^hSXf$73?6ynjPcGdagLYe8fc zizBa_%fxq8-OeYiKyvr{Bbw9_eu6h16DfLKXSezibXW-L;#QlyMza=q)pxiZucq?Y zE`pC{U*d9mHi||57wKAsg6`d~TK!ck6}A2;*aE zf~@goHMVRT@FhkI+p@G2>K0KPL5A-}&qGB%_^}11(*7S5H?5rM_Bg zaqR%ZDS(PanEnOgI=jNgbXmm`bq9M(^0R~s+GX9jG2Tf6O5w5hJu(3sFYK|*=3JWp z%sDRsS7oQ&x5R!h`XB%r zz5+`XZ{@x`=4eb1J%~A)f(^YWuI`z7a(^=Nu3|TmD^bS>HRlKzlG#2E8nw+haV=?p zmc)2cITn74hsrn9M85!*S&$WA*y~BzE)d_S4AT1ka?i_%EB`PjjQi45O~$DYFwn~k z<+)tveP6=+PtyB$%O7FNn6k>qLTVU7g6dSiz=#yGRo#k2J-zFKOtOyJoSmWL%3a$_*K z0n9W9o>b~3E$XKHog`XQgXW=^r0p^RI||XU?@;AW;c6_QNl>DuDiRs4ITu%D2d-f# zbIb3#(J!lPke58!XDzEmTmRg@!`;5Kd}h(~26Wu6CFfe2B2_J|&AJ~fWx01670cK8MsR(fUbj6mN#KmewzyHMQ0D?e( z9!u1(xBDl>rM2eAJhqt#!biHcKls%-lA}s|auTq=Cm!*v8QFUfd*TqF14!k>HN#`EDaGSeIW|r2S4{|P9R7Gv4V$cWk?=1mtM-c%s>;FqvfxK27;r(bBm>lZLF z@~uUR7(Op_mc)4O)HB9UxF((1tu-CZmA1uAh?TGyLoN5!!l32CHKih^YHZ<-Y@mc_ zptO-Fw8)|<`=GHb^LLB)=e>E9{223j@Fez6`8%1(ETmdFI&aMFcp<<>(NOz0>b7yi~1Y=4RiPrn)!BiwNxIH`# z|Mp70og%LXV_wxX)`VJGl_Z;H25xqBlBi@$*3z{{59p+1Wm?Q-o;-==KY@hnV1m+A zbX9n>GXH-1(aTaMrbBnyA#L_2J(y5PrJ;2;RIi8}DYnz)ycJrFG-e{e-QTi0o|8CGT=+OrjAaYsUI2YNGdZqcjBg( z0j_!xvApW_KT_q;{Nyq}Yu@ud_<|t`S~k@JS1PGL?FO(rHFFKV_bD;3jmKlD(f`;K z-b8@4T6CR!Y*IbhNB@jV_BXdIV|SIHq0k48J}QN^YX7{tOQ%O_SPV&uQL>*OQPhp? z7X?hmjPa4^lVrNnJJXh_4oV_%99 zDgN&sI30!vGV%v|w;v6Ze4s`dwguMq&O9#s1UPSM1c&gOfC|P(btSQLLqEYes-Fo7{Rl7hNk+fs2kRGD&qg%vxOBU+xF%QV7f}*v zj?wQuXD^}ji48{*J&o`zOuz!UHDD@1*h}0;X9A)$w{BNl7^z(bO~S7)-B$U7B$(M+!>01^HkW`82EVW;j^dx6ZtJZx6OUF zu~Tc0%PdRb(pQk6lR~0M57T!V(&-;Vl4P@_Z=&o+N+XT)>coPml;@$Ac&rrfQyp%HMaaaG zZFnGbdN$`X6ADE{(l8c(WU1-CzNqC74X*L_{qN$q`RzfheKrF7w(#liyl*DYTHc{m zLzeab#OE`8h*V(DO(Dgs{I^H*FroeSfHaAPSUELL3C*TC%C_(+bg`s(O+A0&%E2SL zD=Z3^XzLo$&DQ5&hsCEl#*01G*#EVE$9fEw1?$@iTu!{tlkU->gSoc!s4MgqHXp}b zkHazEm5qJTOc;{k-DegfLrfN-Dhm>|VgR6j% z;w3}89-ir09;o-}SYc|m_V(Kw6aK$m0*!zU5U^h>`sicrnOo`C>;45ick}8rl zh9|l}clt~rRC^+xe^~dex(9#EId;GZBPb}8$+uuaa+H>@M>d_KqCfV=C#`@9cH0lZ zk@oUr_q_YUE<@Nv_+dvnQbYS|T=R7!r3lza`i+sGa;P#Md91oEmB z^)fDMAeGCsFjXS()U_=sNN_c2PCDVeX6VK>ahoycc(6wMVG6VoCtcPqjV=6BI`RUx zWhZZ{Cn568yiLDN$RMwzaZ7BCURR6Fo!tI1>PpSf)k$0DngZY&4z^VNn8x^K?8d~? zFK>}7PIr|U&~aitSD>jfa40)Ci=wv^^)L}Ln)Q}$;s;U=iq88JQHzj3Fwv;e7O7VX zsfe9iNki?YwtquFP@gOKh4tC2bz`l^XC0r$iMtbi-=Aukcw(SG?aKFFh17Bt+6shu zRrIi*R(jsMtARw-lg5`h83_6(_hY=btC9hmRag4^O@Nt@eYU~Y1=ou)dJM~NHP@4UdrPty}(@ijvIEAe!=_>L{iPwl*GHosbb&}0t<}@ zjZorJ~0L+HwgM_tx$)b5`( zAq@!bRROdRhXnMUmNlfEp77nblB0~b^Kr4N3pE3i-H*GHB*@poA%%a0FCGRlW)8UoDBNiZfi)C8aDdrV3|b=lbq;u#ZDw@36mT;8@#6ZzsAZ9)#K zf)V1YA}1RQM9?Wb28?GoxudSA-;i;IiPF!RUi=86xOuHt{*&$xuLyie>psg_IdK zTR-2O6w-E$e{-3iCp-)avLR~SHazy{t=M8V^68oGl|1UZKl>2qbYb9$30gcelROp- zyzvB<4A9^LLISflxXfnRa#C*I^GY(HQx?a3pr$dvHHwTOwr(rkgXtMd>X||lI{PoH zfB5&MncC54uwoynqYPh0 zl4O?PbB2Vy6X^hfCba4wO=QA=nBL`!+r+V+@h%!!`EjO6XE zJq}T(koWO(9KcZ8+2DV!hj~JVRQ##eB`}NUTke`K8?hlA?P>9OBp2!OWyUv1Pp9G= zI5G|;IxRM_?;c?<&rn@Ii}e^Y;%ml92>rBrwo8!`@6^f)4K-N|xC=L?(tpl7|Iza; zL(-wz)E^;Hj!jf%AgO7BL`@3!m(1foKG^36lhppSP#ZYr`S|PWH#@prUjO{B7F>f+ z!fM-wQj~6r5??#jJFM_%8K8F@KRgydimmA+bjy^M+KnVze}_f+I$IX)YSK)(fHSbJqfCXpZ-ahIeBeHF%TESX<+2Hw=V3-;f$SLcM|B)EtsSMu`x6ilQFAsO^N5EY>g8{O7rEKOs ztSCHp`%o1_$)OMiUHL%fZQ(8vN#nFb$Qf;Dcjjr}7YQvsd3hZ{j!<7=mQ>jg*k^9lV))~a`qN}AFc*Z0R!S{fHu@qQMrbrGo z4+!TTs{=5^jqDfK?DFXv{W%u^ihZ=i>zynv2M|YD#QKsx%3Yk&>qj*bpuOUku?X6e zH6cZ|vCrsLq1{;pP19p3m}^f#K>%y=i}I6Ie zuCHi*Az-LHolOk3i97ps)L-1gTT14x97hWJ)A~kO%?t$kq49SkFyM4MJT-$GSBD&v zz8Z5k`q^s?iSxfzKcoZpNRG(LYO+pD|BDuBB_6w)Oxm{q_G8Ec9p>38zs}a%lxvit zgTeC{Y-sX@!J+JwS|1huqZ0OBI-7^I9tKD1 zT|8wi^KpuLCDZAaHkiTSG1lI9Zc2-TWRV)7-{CpWbND6Df^)JZ1>@^UQ9>=RJ`G#k zmY$AbgPUzQ_j=GQ9UVIaFyihQ2Cz6Crx=tcjYqNXjC|j0U}n{i#EJd;<(6M5p9uoE zjC;(ySS&6mh-fCR)yGX!6&*p0rTyv(ETYX5`#1!u!TPf>CzG>G9_6 ziRPjfXjzeDWT+xw*XrW1P-W|BNewAPe_-E<*?c}$Bzihaxa!HJbsQjp5br}*8X}%7dG7ee^|Q8s3^a$I|$;?LwAF8Bhm;0 z!hi@v3PYE4cS<)u8l*%RLb^MoL%K^Eq@^3)hyQv%@nzOx-TT~o?mp-2v-fpr^Xuak zp_78X?Y4#~PkUP)%JUqUCqE)#Waowo^nLblD*yxZ z-p1>(9hu(q%0vi6o6sq(;~(}Uv~B@?l$f=beAn>_&AClZzGHt2a9=Yh#Iq|ArkHQf zKxm>WWvoYQDGL(0S`79&ocb#9 zd~aI+W|L?mAfJY8~+F-F)R`;xc+ z{YdV0SlHitlNBcB;vB4v3h{3>CN7~mNKMO06o1@In4DAv`4?bQE!De?evA3?|96hZ zQEGrB%xu9fq;=@><6=vk344e?M3`Ctq4Br##%mr;kK0#a9?eqk;U2Z>aM$|nbXPk-Th5@-0)Dl89|;o8j2eLL8>Ui?V^cZXD98Kwq#6#!6id}n8jaC zT9pY`M*}KH0aJ|KZL7rJwk9dLG7DGq9N6k~93UZKC9K8W93n~s*uAZ-hm<&et1KRH zRzwg1Udhg6F`?WYOZPi-Z7Ys9{RDq4xFx`vv&$ZV@4|{qg_IEg`OQ+^9kFj8ac*(4 z62E@qUNQf$Zo~&&&zl^F0k2A%I5BU@o(j6Z^q}-le*;W3)Z41cC7+~Dr@@5lXu|fzmV!|+IOSvuA>36&`D(_H|Ep%1NHqt_Vvjd z%5eaPpgtG>HkI6x$qkv-(v+<-{H*yIlMFI=E^BYz`>lK5s*4;o);jwd;dMi1mO8~Q z#JXIuu$H$L^X*$NPc61&Ucbp1nMdG-ehQHyHXF&XSTsbEuY z&e*EvSTxm}h_(DWWHu-KBX=tel|RXX2ZqkvDNE(-j3Kg=oLB_+PwGMw!;o)sRtQy& zB>28BDpn2b9a;}+Rr8YF=bxkQOv8C(4t>YfG*V&OQ!cGEDLD7dR$I*qHx9GCK$a6c z6I*ll$r&|UthHw15aIA6NqgU1U4cb#N6MKu3o3(4K^4IAX7$va#IJPRqluWaSDwqX z2$&5=Bu#)M{b?LoSOhOKw>tMwcWX;wTRq>2P|*XSP7^2}twd#t)-{@B;zs0bi&pCO0j{Gb%|J4kLWZeDR+O=5GnQocKv zn%agL1YM@&+p>+KObvhL?6JB(`ddA#|Mp*jt`B2gQKT-Qc zK@hxG#`~TANRn*kpMPzY>*y2KL4JI>Vk=<1LtnWres7U%mQloa7&F|K)*at zpb?<&Q=VSf)}rTY-$)oTqOC~+0(lVMk2kmIKYh9F=^7QiMAWZl-)ZzUF8v9DL+)q% zo-S&sdVrHEYx;arH9B1-q8pc}nyufgM)j#U<8gSKUB<=ZAFwH@Eq69%1DzU0c16cz zoeH09;pPc(O}zK~lxj=?ye~%MQT$}2-p|eFS)!~ZPlmS7o)az zR1;*%dJ8gAS;`7G18)^PvI+W#rgzVqySHw)&5@tJp8$%ag7W+E}4=jDhw=8`X zr!@7A%MKZoIKgga$fo^^)r8I{ef)sJD9WbKQEc=|p>g%*Vo+=g-c!}@vGziBuX1Gd zWMuXIhfiFWxAGKhZ6kAf3PmKw7PnhB22>Jh=m&DXDP=;5v;TB|k!7 z{?4M5&|jGrlSU0%`sKm=lc|0YK^m<2#j$m%`l`XE0YByS@Ytj_a>e#r?j3*qd)tC+ z|E;VSpeRD+KdpO#K(2=Qzy4i-#@v2Ej&eHBdu^;Z6Ix;*5geXeVW#Aod%QvFx}lJA z+uWaQ{5eDRPL8HYifgBsLQMftzdhmGJ<)4liNHDjl>K&fV>cdknVMSbkW zc6;U0{&^L{S~x~N)wf1Y`FiEXKcWe+sCW$>&+XrnRm!NqzXoe#MKvc zg?V@xXADH2J$pvmXS9)vk~HA^4D#Z>j%7)vcBv(qeg!%hl6=G4&8-(PV?eb^3kOIB z=u^;sA*L1!IAc*kUx*4|axC5S*K)&b*U3&69Th*BCcmfjM5^5FD}2DNRqCT_uC6+p z@TbLBgd(PhxY$Ca)~h$@L17;XlofM@#dLvdBtA$1O8#-{@e`zD)6M z*v|D2f`)~Ko&XPQtimsoi+2_z+?a%JJ1sw}Po0YwckvW!r(oqgJxW$A3<~b7xCzjx z23j(@sY}rnoTIeDr_Ha+lWnRs{W5RjjA+~c9EexU!4J-FOY=A3cG8u)j3_CUatS~w zZ7uBr1)))IpHutd-9P?_1#Jn7&$j`#q^l!FoHWk82$nq--NSvs~joeVM3@&3^ZhJ5T) z&5yGMwJ+!tkWe1@MR0p%B9qyLDSYPhNp6XHpX(D{bBe$pS%e8%Mm--h1qL=k(@2H8#sM;XP|A1Qzk8pb9R+ zDbopy;{MbVJ+*p60WjdcF~RdvFi7%Z7AiCn!GMfMQ zhYiN}p&n#~!*cdF8coZR&832~^m->+^c>ZS@@Z(kHPpxGOLT}XcP5&-p8ifNmUydY zhR-I+-m9FdT_s%wS4sJot^zbG3!&VIs5m*8ABx7)ZBiY7=4ToSu<^+mp)^ zWi6_ri#EtFh6)m4QvKDz!J1@wumFXue_?C5D3}nbFq_3w@4mB$#?>s(t!eyPFKA=r z(d_Rzb0QsmxUE>t)Pam4Sr4y#UB2fm5a9e1p+D&#`We62jgaTR#`5BYpA!P=W8AXO zV|@KRqQfYQu0Ump?8|qVK|0O}!SI8^$>S{Nr7mAvbBFtwg^@J4jj$$0Au_F~*q3MS zZjbg^>yH1wGAr=K8Ssrtg;vr(Aaf)o0-9(+iBfX3q{fF8 zSKSj10X&i!69B?M#Zt5pdm~11fv<nx=^sKjj;9NvaGVC^guDwX{agbV}}r5=EcG;DkEou^eMCgxxGa z_g_vA5gm~%_?Nm#iLPScTK-O${L7AzTE?$nI%$g8OR?Fr*O9@RiqraHDy_LB8Ez)u zYqDY|V@FH99U5H<$FVnKo!*vujb$X&^jGj0%c(V7$qkd;x%)f-JOuK z0;gv+=KDwMtY%i!PJr=8=+1HY%9506K7J}b?)$j|gS`1AUX6sbcF zB|eGaW6IHJxZM!HSl61Wqv&h3+nWK(eO+LE&%+xnbBChY^(Y&;;XxM&5PyR~r-|~u z@5{KEdmD#eMmA{N!lT*1NI$@;m?7l3nh%4tAoJ0UVDqyDZ2~z?k3pjtB+#rLBN`nh zPkz>pd)tmj{q*2UmGONW)WPka;+;{E%j@Yl$@`PWt#bByCATIA5>p5EiHq~BMYw&# zwD*M5pg=3LNkcN3A_!Dj0|wD-;A=LF9zU&7Kg6y)h>g1F9A4H^3<21-_t{cr69^Wi zn|1nzT5RB)E#b1`Xu?6DaP?Mw`QG%ZlN0ls3ZA(|ow^k}JZz0Wg%Lhbx+@?CG#`5T zdTp}o8TmUw2EZsUctL1SgEFA_pDIY@Z08@!@RybKtwaDb>94DM1jUYv05M2?Tge z1JoV9HZK*5wl)UF%=}CopNgmO*f29>(7T_t_ix)9&+FXY0liA|#kxONckTO4M{dhc zbr)DG&*=TRGqf4vA*v%5f%Oupjs>XXU;PX|ZQZy#Dm-X7*Za_n@u4{~x?*TT(ZpIo z0AN6e_*93aEYteiH`_YszAYt8frH~@N<1p?72SQ8Ujmd1h>U9tpiR^)9>9%1ZLlPCyUQahIxMV+VTa}hGTE7pXgK~_z!QWC}bEOsf?TnlN4DPz!#yb<} zMXGzrP9ayZ%s z=Zwl^W7J*lbqIe{YP~!3emrl~k**$pT%``pe)|3U>0W=VRdC~P=*o#JsDnP%*I0Ms zc@A>?8B_>i#)W`7$#)h6r28EyXPt_JPZy7UmrJ*HhyEOrM{np0KA5ETiVia@1_b&0R||ShNO{sjOml2h;q4jvvTDzKn`NWA_+?NvI%PH3KZzb&wdJ8FMVvdqFWm^3S zhNo#g+wB?J2cgJBOzbh|vWGQo-Q3zVbsw!?foQ4}9rFce_dTN_QhNaHsFs8{VMkt=XBF2%Zb7ogi9q7@pF#rKPacd(C#Ii zWDHV;Mgk+sk1{ktVFpIVp+;E#YQ#2$dW6y(){8O6&1G*j`rAErHtQ8@t6 zcN=B+)XqQrmfBFR<^y+`aR2u?|2j!zSW+6yfMOWMtpZ~nNlMfnEH04)tO$N&+1<4P zUK%hEh*-*gq)N>s?KdxDw`EE#P4`r1-QlgTIM|kDn+rV%`zn{K+=X7dSpzN3dW2Ws-bgWI!m2*JXxhbiGR5O)DzwrJ4{a}VrMphbk-g28%9Vt`xXlV#k7f+dC+D=9B zwa1J!F&n6j9aqPs<*jB_m-m*H262(DC~x^5FEozv3sSP-w$g_E3DWty)3Zc09mz5j zq1r?_iF~~iYG#Uso0capq=M$Y>T{*wBu4Um4W+dUGU{Qd8G57ur(U# zZWo(bw#@*$MZU8Z%ab1>qSKV3OJ6WJXU^?3UGMvo;&W?8lqg(mo1L&mpKE>zK%Je( zG0TRn6w+yM7>+{k;EPZLm9+&EN-A>a2WcfPB=6izZru)^{AWxTQ>5>gMwRcAe>v-|}_FSrlK<%)NXF z#3+Iz<(fdDH46ZN@7RYNmMMGg-HyL`Uq}acoW;A0P=Y~VBpC?3G)oVO)r$Ub0#3S? zPNHWrp&4m)CPR1XOy3_?H08e@ix!vL`Ml_ zm?Z*8K&%$)ZwqFT z-CU1imma&;Rc|(+^*)KcY;z2~N-e(bu}}eYg4Kz5h^hsl%5$;RsY7r0%m0JE{u~8~ z5O&zNuWw-UvwihKdTel9tUQb5kexr)6_w49h;O6;XzRXV50~b!Y<4Z%p0(vjxgxw@ zc!|b(0_&V=F)CaSAI9S8_ktkedqtop4M``5( zEDIsvxcz9n6)%&$haGd#wI#TVVBdg4ixNG}Mz7KgkZ7a8z+p_HFz2UV!6 zV~nq8=yFmEx`)=^#wI5ln@@+CS|u#&+@wYi6hCQdUj4r5G@tg_zbu_}D*kzVoOO2h z5b?`r+9;FHQ$-U$=i|>QDXxeg%%6-_$BVIcq4)oIKx^tuq%@l!V)UQJ*@voZ-wPO2 zppGDHU-u+Y4qsCJ2(pKgFb2H~de^>_q6QGR630JjI_>F8(FH-INsI$XcR1a!XJ5Bp z{yRG^9bMLxaIP)5zF$yfBFBg=8@~F%t$@dYAa%{w4hZXmqu{zQ8_vfdzm2Ns6I;0m z{2}4mCudrwB@W+2V*Im3O%IXUR_#1VdX;t)adNUVRgQ^l@>7~!WXJO`_|IU*hMKZ06kYTvO_n(%)8*IA|+a_OhPRfl*qy%rkhmDX7Z_@tMsks5|1`{?Q+yu zaP|SQTmTZIj3;fxFOn?gTc(J|z=Vr46~`)k@u|NVI0;HwOj6L&Bpv|S(8&I0NT96k zavcu)x1_ymze z3t&fGPfO0Ng}lR;1@<)kP~G}Lcl+hn9S?kj_A(owmLP9OZ_V^M9wUd7y%n&ch{Ekn zh3q*L>=E__?ya3(R}OAcPVfFyZRl0KznKTjs_q9g>J!fCk|J|H0QTGwl?tSislo)R z-nF0O%q6f5+sLoK>Px^y&QDqDe|yByMwDe30;y0VNl~o5_f`!64qs_2bYav8;upeq z`>;^BiRUj~0i%G(QOa~N8y7cNontxW-=r|tA~g=*fg>lyxyB|G|EuFmvPxnLqwVZx z;Va)hxX(?28hG?8cfHrt;ss>nDK-W8s`M*o zfp0zehpI{zqp?UzzMiFa!335_QNw7uQ-xX}C|4gTZt6tz9@oP~YItyLGH!-~L#Yq~80WL}G8zS-Q>p8Q6wC+kd zSZ)Y42MEOcn#u{>`5O^rgt9ggVPzBsX0-iPO34u|VL$B`ZS@`{=TO3h8`Qy4xz7KW z`@Ug6x>!}@Q1~z;J1R)Y{czt4X^fI zFaBY4-?#JvatDe&3kFyI*5@8QUPPLLPejm+b#tMGCJlK0BiB~TgQ(sCKv#U z`M!0B8?GAjh|j9qjPFrF?x%-~8QCXOF6z6uddZ!i#$HDNtUKErpp06oV&5k+Mn<(- zr5L32oF*jjLROIy+=y}N`+HCv)4J(yiuXS1yAYi0gVW!Uh2NG_dt2eaU}^`%0nFxD zzDMvna3M&aWz1$q&RRuwN89(H=#!t}Zn$eX`GqI%BC!*O4~k)^eE`qerkNSZt${?5 z>wmXa@Q#@6yqcM`evp5SLiTO(jXeh^H7g_(;=NJ%A^%i4ATQIod)pq3u zS0tLxgKK4*XCuADl1$AC3P6Mr_{X=Om-i!T2|*yr^moDQ!#ESb@?8y{m+##xlq1rh z(-H}h4N{Sv$LReeDtNb4@U{!Lo_{@AkGn1*@dJ7^92HqMdb+p^Y4b|r zdOXHy?6Q%&;`k^YhS##>*Mi>XIC2(E%0h7G;uxI`h#nt=r-LM+rple(X4p*xqjl@U ztXHO9qO^SSfx9P=ozQe7+Q+`6FoTT;Lbz9bc}L&q^T@L0s%LfM2C?;eqjXHR{{8Yx z0{$;w)O^YiiFU;0QvnGmXr^f5%&>}t>62^jnc->UbhHwe;)YgGYdV);2)c~z1`2TH z$W5MiM0Ah0b4=NZgl5m2%Q(c2M)PzbaXfNuTeTFi;&Yx5Ib6%|agsY(3SPW^$IUH; zu|yvRQR11uIq+BZ$moW2L>Gjqu21J82IJ^3m|fh~jHS0&V@V#aRegK91_vbpQ&o}=}luwq;MPMj-G4sdH2oc6;af-ecS2pwQ#?eJS z0lBeoBm84^Y^J`?m+IEJT^WmS=AV-@O?dI4 zV8*K7D$t@(b@QTG(MdB-gm@38VF)eIEYFQrkwq&F1_n2b<}>WB1Ud_eLshs2;uW_S z4%m-3NFkD)Mfep}$avxNF=iwHDsKH3JHq+2tldSA{B$n9awyMoQ7zzw`c5GSp$w3j z!>?D3bMzep&D ztQjqR52gy*Yg|1MwYW?USz8|QmLAo!iOA~^4Oe8@LQU7AzA+`@E)wIsWQSxD&^m*x z6<89*HPy<#=hxknIUqCFFFK$~>*V9K|0#+`3f(yBD2@N4Na@JJz3}T)SOi)x!Bs*| zG_SS=Al(2(pJ&R0@P0Z|WUbtgIU^2)z?7DL=v=&GMGU!~&Zh^jHSt8dk{=E<%Y$jF z)wXY3I8k_Iky&K$$V;pEcP~7LuoMAm+11V!vn-H?(u7p35KEHf0!Ut=H25GzuZds}Svb8P)>~hmtWrgMhbqiJDC^N`U=||Ne1zZF&uiclb=y@5|xur+*S&`VV7p zhnnsOb1d>(Rdlg9=W2Jp-IBgcr{&t4kp+4y^994V5kDH1`Jl(6J$Mg)oW-3iedCO` zHO2bOA0lfhGfe>X(IFFQ8_4#6uwer#E7B=eUyMT+0tRlP#monx~B2;GAFSl>84r+ zrW*8eo2IHnq4hRoe`r0}CBhUwArsxXuEW+9Wpb|BI|)_w=u$`J^#+ zj~2Y|G(?hnfth8JL9Ds$dadwmZnH9Fbm^*PJXckdD;XWc4I;^XAD3axE;^&-{sqYP zy;=!fs@Ukr6)LjeoKoUX7j>Mm;|89r!1g~4(4VR*K60_Pr}t-h5) zLjf`;ASvVjzT&B6Wt<0$<=UKc%3?r3(%fdLgjx>#9oWz$R)=1eIB2*JspSz_X>ETr z39eh|X+~`w|FiK0VKOeEI>_gNDB3Qe>>iI`xq;_UJzW z*>q88&4hLl*Vlpti)-nPdqsz-+zewVt}H|n#Lo@0DoUE-RplkJpg$b0*G%zBCvSC) z&zGktXq98Lnc^%_BV|@f?aRQ!u=E{p6a7NNrMx}P-YIoF`8n>_xU5w!B6;<-CFLPV z`ak>MqEKXb;Y%uR70$sZLXv1l8r!CWQD3tt0T?n3(ww}x)le0?RdxkZ;2V1|ovnQ| zaj~8xt5C*sJ<4B0+sp$qilAzFU%bir8)Oi?l?apO?u5LssZoZqvH%wWjU^AJeL<|w z#)$YG{HNUNKTcK0hj63T-=g>DrMg`o4W~4vT;38ioE&M+V84rMiPs^ysEkBHA)$F8 ziBvcc&`#DHH@Eoj`HWZDW;_G0_h2FaD0f7Y{%)R1B2H>7Ty(}~1 zX6JnB2R}aK6|;; zPa9uj`hainRE-o^OAPO$U*Z+`q`lW}<<|U@@FFxb% zwR6i34(j`|v_Zg*Qn1~At*=F_zp5q^pZ|j6@}-`XaHqgeBkKt(f%66EJKsmMwgk#M z2mdl*yZ;ylbXHChp4Q*2JT_B5wu(^{0*_IfF&9}$>uU3nbfdU-p-|AFZ7p=46%+J* z^?Gt^+);Vyr_rvbk|D!CD0cT)R%ojMsC$e<2vHe83{N$D2c8aCRYs^e2%*%`jAbdS z+|Q2Uerhd##<+@2%vqkA_ism$_kJc>`kk~Mr+>@1Pqy>Dpuz&6kX3sXALfjneWOUo zYMB0I>U$cE=5e&4CR7w$*}~0h{2uK%FCBn~d<^S!|7sXP8$f)E_#B9YgaS5*$b7*y z<=x5U{f91y_3JAZ4jx&^L~mhWf<))B*HaRXnCEFF&cjjY zQ(xa>;j%4`w>OYm-KA$5R$RXLHAnmVRi4yCg#MBQMuM$60OSKKL8+0Xt%X9AuT$+y z8eIpMSpMa>uC!f9qGY^q8-(-73;sf}6YJ)_CPvCTs7^jGc8>s6Qy1%@%en=`!N1_Ecywxj7 zuH5GdRcWea2v_dZ#ilq+qCDaYI%Mhq(MDzXdzrp!@)xpZGw_G-LVbD%`=f|kUKm5> zD9({4mP0`ey(RfGt)X0t=jrVizu1S`@0#13z9Uz&4+%3L#4}hOyO)V*{EXht)L6o( z90vxg?gbTpf- zNUMJZ(QIKjZFatqI-HF%g?Yy5aF&+g( z-f3ao3!!Orl1P4!7Q+n;K)*uvFB2c|S3dbVB=E3H?Ps#TpAakdcuReq6Z@FMk!S+I zDZqz(<`2s&gh=lWUIG`t<~J24RVh|4FR!V7M!FKa^y{Q(Qq&h-CbzYY zCxMINa%Sy7bGpDEdsr-?Y0R<~2~^F;hfJOgTqIv}0LUH_1O^=6RsN>d_T2L}*JaGX zB5}=puND4^FR9XtDc1Ildz)gT2B*5B9lf)HEzx$-)VoFe%3_-D84Zl(yt-r(yy)Ws zhU8LTst0_If21 zM#Ii`>~6CiGu{|aj_D;`PX-=y_Sn3scKdj%j)}Wq`wZi8gTPgIlp=b?N-1c4i$`XM zBrb9``MjAD8^ll=7n{rvb9s7B8`r+SPG1s4vKE1<-+)<>K z;eg6Wu`3~*43ATW9nHgIv(L#cL)E}t{__=L6c+ir-N{*C<_2nRjKv=+R;Ys>U|+?5 zdY>kekM0t1Qnw!m*k($F5Uv*C&g7PCPzQwzEl`98~i-xDL~IDOZUAT9i3y z=qyy#l>EqWC^-7z;>e>>E6vxmnS^Ox7st8XfwU zn(Z1`+71lUOZgQw3Y81`Qu^kWaZk9ma+7vVMGqGIP0j$`I^%0z7FbS_O`%&gkb&WA z$sIgGM6`qrXr z0zo^u_UZsL@C^m_&W3Z|nNws|Mf-aHzL;rIztWNw0zN1eEXSmZX>_D$A? zFk3BQ`_Z5>Yfj=JY6X2NP5I`n%WTp3p8JLDCuxK-^UWgxy=rg(zhMT3jcPVPQYm@( z1%wm(^I4k|41`yH3|h$4`i3DHTW&N5o5wN@3fL(2)cB%kM`@WyGn;NRWMv#o8>DQS zM|UiM$(IIrY*fvUz6Bd@5@d|$UfhO5J@TqH@EyM7UBWu&_B@$}jA#=D>Wt!_lN)E? zJcSggv5RxNWe8d$sw?NTUuiG=8J!&^dB=U4>SW}+%@!-J?1U~Nu4bCI2f?wJ(BA9L zoFmDtZB=U314{z%#u>OqN-Gj;iLPyJwzf4#DT8rpu{L$QMFbTVe}NsF`8Q92YL!Zo zV_GjrIdx1acABw^PcJ!Nw$$dEQAK}IltR(8qvhe*&2|%0zww?Yw8@ky)xP(rB$lek zDO=xf``iXws^3T4vqs!^R*rkEi7~WU#14(ZUVQHqU*-v+y3uFJs1%lIKoX%*{2;dS zgzUER)RJ*=s2||CRxx!y7>Ke*{0S%rzad;37YYEwu`9 zot;}tq4pVBTh|(3Tl)LsCX%NYOp6F#FOf|gE2LbEke?kO*&w){THe5SizRxE{MO!f zTe=~`33^aB%osyZPST-<(O@ZrDBJ zk6TUOLp`i9;2x|z-vh-5MbwpxeDn+U(d6Q4l`Pw!QdbIWuP=9GJtho=zmxi6Qkbw) zdV$;h3OG?S61aNfd<#L8?R9#R;9x*3+xqowEqBUG^>WWbci12*NmORfTcG~xJ5#A1uNeB7Mm)|akJf7gJJbe<%PdS|5 zzj!K zmaCkcoTSDv)FLxamp7wcM|O{X?X3k}0CZk3x5f^9;|-?m)j}u>Yh*Un3wt&2i>v&+ zTcW&dxTKF#p2y0MxWGq*ds$T{B;kgNycKHw_xl%*Sq7nH@F~1faG$vM;rpJdnH`&T zpz=YAnBNrJ$n936QI3W|#`5NQC*LckZkLQ**xFa9z4_`0QQ)^7n2CH$DH-;SL7ubw`zs3|a@YZTx47~@orOO1TFEYVRae-h4=;=ER_L%7cD5E3LY1Iv?Dh zr!xx0#qV6Y)wM)By^sLC`5B4}?qr;D{c}AYdnpZ5KK-Zv{{#6rbbbGt7F#yMA2e>qGB>y~WaO zPA|w-K{SGD5;~u_WTU(_{sm9aY@zbx$+z+8* zbiltiZ2IGGXz#*l)9AKugpyFD$q!l*@l<1Wv2=21tHg`sAE-8eTGJtr?Kf7M=AbJ6 zv@FEyRw68>QL>g^xa=XuoSP?gD$nCE3x55@DQA8WEW>K*3kV{n8f}k#KTIK}k zT!Y50=Taf-UF}z|p6=hQG$nl1PBvX;ExWC3CZxq|V(@*s)-eo>rnYs0NJS}+pJYpT z^+}s{eDNY{FAdYOt3H0 z$%0bR{HHk+8FhQCQ;yrQM&Rl`?G`@I<^d?A`k6?A!O|btwyJib310+^B}C7}qucHH z=losLvJ#E`=jVIC_Ow{k{^xT1xM9Nz-ty7u2(h&_SulFP@X8mN_WE$e3>g(j!>ty9 z2`n~u{UMd)Q; zu?WJV-Lph3oh_+v&mrLPXq3*T9hYS;v@E|st^XTERhooN$oe7T zqNqt9?eewu+zZ+c&Px|dopNPcUYfsY6^#8GOJyB&_OX#26Q9ov;lT`rEIEP6(X^3yl_r!=F za$ugnUvhwvJ(dd}YU7exs`DxXtEWI9GyGshFVp`0``VJ^#5)kVN&&80ZJQ~+{au0$ z)z+JKQ5~w*df&fUm0f-Nlam_>@56u42F2{#hw_CQ*C;V-yfX{|#HB)Zv%EzyUjcri zt0+sw4O?kO7he#bh5WAWHE@>0hw#5EiY%zUzpiMcQtQC}%EcbCZ*v)vCrIq0rQW?F zLxH?d##zE0ZLQb#RVsD<&{$y~h=dIK{`-E#fm)}<)4JWFK@#Olb}~HZ6oU%5^sy#i}{|U5@MF`cGIg^lK!~D zD#dLkp7#xa3Bs!_8m+nAaaPd^hXCM%L@$`h!~8rAxpIR!q`(M*pfOJQsyc(Hp&r=k zP3r28;~kIWV(-@3Kj3s&u?~y1KLjrbBL*nr#?IzblZbeZUawV3xx9NIvL!H%nkI1* z+QHu7S8xJJE`^U}8l|LU;V(m)JR`n>&0R>j=%PVeLP@{W}>*qWyofWF&MlhPuS*p5fr)%)F{ zGm7IwR-65I9VD88=u8xrg6iZC2lu(zVktcyWIMYRp>ldPY*OVz~C@D(Xx*~KGE66)X3p!i?)(f^7o zcwxcK@gy-72V4rsOFwayWmvfWfPZl+R{O;3uX>LQS9o7kt4C0B}LWP}X()BIH? zvM=AfepBw<`?!o@jCS>~O?Va}yRG=$1f^E6vOqP0Xr_7<6tF z*1h~7+P*E+7C7+OiWp|$c<=rL#_8ZYD)*q)?9?$ za}eB-iklALi4LoV$&2F$;D@npUUZ_Q)b2V z)h>NI;A4x!DN4YFNQ$C&ucuQBqAAFNPeJ3Hb*7Q*;)IJqijgz?=p*qC#1oWgkc}5R zfo`rInpv2rq&C#IH;td`5->)kiC+g)m^rHfaZOrOocGlJE;KY?7TSa!ZU4X47UwUW z+&F4xMrvnA`IAQlQamj(!Y~hJSrbqUvxNXuGk!Z|?crS~s_mvgctpJX{VxhZf{$-p z6QVsTv?-4tckD&kh}vn^7cMuEq2(FJg4QYAw5j>W!?}0N`JAOnNo=ZH>(V*=df$-@ zU+xi$?%{>4A_?OXY*ckAB}N)&EUT8670k~o9V-@iHO+tC3l9J9<{!~FJ@SCDoVV31{uh}G1K{0`o@q_?t%PYl2Id!9nb@+rCQM{^V)i` z9TSk=vN*D#w$(_GhS#{1>q@s603ksp$ssTfDSH3;YlA5f#ZRTBHx7U0d~TI(erytg zdtYsxo*M!1ous(hv9u(KsPX(~x>C60tQ@-uvwDdPr+QhQa+mJUspDdQG${gAS#TFa zM|WRR7h=5YV!;_9pqmqQ^z=EH?V_3gx@^DssDTHzt3eUlG|bx5Ht}_a65$CgfSJO~ zpaVH(cYqE(3PAVzPa!-|IcZ%bbXk{eSx8#~&M~ZBOd{FR&NTnr5?}+;V{B1CIo*DW zs~Pg?-zjoZBciY}u~wsV8(LpUTK6{Bl`~YU>lI!UdW_~pgt4SciE^eh8=tLBjaDgv zf=v6W(-l#}v&VK6xaU_`EJcT`+>JEtGUP{&C%K^=U(}-fQbefWE;8;rl19c^4xwOA z$dtE6pLY<^V2_c|zZbE}j*#47HbWGQ3`$N^^#G*??^ZqS{%NDvDPC)q1UAwRBEunX z1fFR=#&D-jS*92aYMy+q8uK;zT>7MGt>EIdd zXw)HgW(SG5v@w7~qd=QT0EGfvt>!k%XU*LosmhAXbQ#LBh3~b3iDg?RoIRZLg#|Pz zM0kaD!M3$WDiA4PZc?LcYETu#jL_KD?3!+9{Oy2LiMoZP3L`hInS!z=u`wpipr-2T z1q1RZ5E>MoU#W(GhXuY4K9KaF$ZHr-v~d9BwI0+6Y|wAlgj`+`u`p+F>P) zEyYYBn7+rr*4x`H`<86ImH@OGty^qxa`!Y;{&ZS?|8u7)+U#uouvPE)&C_Y^G;~Y% z*n3gTZRO3}GtfRH2abuYWYYI28mcV>e(->tDGwx*-8#Qjc4`@{lMbMux@L>SL3o<3 zUivr=j-_wqk>;Lc(NW5tFXAZ0`?px2&o)uZ_rEW(cbe~7?f{bpWj08R1tBqZBh7o7 zK#R9SUZ1BTx-{_WHyV;QU5=(C3}b>;XKvOsmHym};?On? zTlqU(e!Y5%*l$`nJ$zbUKcoi6!+!*M{|#&(AXyCL(E)Y`|9`s^Of%QyS6&kjebYIj zU0*q21LPga3*2O^UglDM?uB!!dpNTh4y`mmR|h(yKNQ;h z!IEJBOW-ExwLa;~GE2VVRWR(M2|_qls&QVnC&9=}o6=h=a9w7+ zGHY$kGeNKZ1^rYbe|h_~)JyW6D^XK`jHEPLpchn{-S&WmBn9roU_zz?(Mp5A7PR{V z73hk<3WY)Ao{2A&g4)cngpu@;m3k=PB_mbz(Fm9@tSCy26$CPlQf5tZM39$-tQi&5 zPwjHLj-jLEOyuyAk+`hs8f5trr~1Y*V);)jkQe~{%yVKp!zn)u|AG|^kmTG-7;PR1 zg=B_8T%GkJVPFx=;U|G20mhqAM)2VlR5a=5(}PboR!TxC$AFpZsoO}_Q~HJG{RBq1 z*NwvZig(`=GU$3eg{sN=?t}Q{$=kNI3jys4fZS7khtw5_BMt^#cQc7s?tiE&p?n9y z#e~MhVzH~*EfifwOrCsMB9sp@DVz}e^h0#WBsokp$%FM&aXY*p7H0}&pxfV(paJU+ z3guv%VjE&_VXtu-V;g4JhgSIuQMq9JM}&z3pLGlDI799`ql~2sSSzg%d^Dp8!sJ$p zBezxtW%6i8wR&m=kOr{Ch44Y(U3jBrkfRVT__}laq0|-p} zwzCP~(ou0ENK$OnKOeQc-liM=D)@q~_Z{6QHY2uAbcSq3e^>muGT?qY1Km7GEvR5S zC*u>^C$v;;PiQ$ADF{=Zr3|4%A4SK4&nR?k>Kg4aIu2QBZ{w^9#7=aRs|7VChBbqz z?i!T`uD^;DtkdMFx|^A~8abYl>z;1GN-h`J>(`R3z1gQtI2r+v7OSbB`*VdLm^NU- zS!}-UXZgfF{ai?MU*(9b`)IiMb0FDuLXQHPsnt8@sA2x5i%sKkL9UPJs*woCRW!1{?ib>4x`)1(e1~qPv;W z0zJzwgQb#J)T4qK1`NVG(8KZC8%@)F&_Q}XvL)j>;(FrVQ>G}!+NSWt+J^C_xHAS4 z3vv}fh2Cs9g^lFQZ+<8=5I$4R760$CFQ6*YNN~=}qP{LI0RE7LX?ym~BB7ec?{oGk`%B`$WJehBV zSoqAOFxr-)_cc^Q>Hku@QSFWvh$$1=Vp)^ysn2r6#KWpGC)I2p^e)gS|ZIp|G(y69wM7(0`oeWD)?DM0izmOsf zlTAt84zo43Rd{w~Q5>ww=}Pe(8kri5a0U&gsSI2uTt?8@{8KzhVm9o@#2CbI;>Xw~ zoXwn>RI6xfMtkxLTRhTCG`6gi8gp{%vilmFGguD&Cx4xzlTQWB5 z6y$Nx&u%?$c5X5fjHloH2Ia_k@`W&FP^I2X;8cgoit2OUyVibx0pAFvl?kcNR`ua1 zb@^2>MuT)ER%MP?d9U*95E(qfrZ_(J2lBHLPc<<9gv4mz;1#J7Z>4@QWx#|OyaR$4 zUV~(o-|j%5IKY)5yWWaEeG38D%T))?$E}A5a&o{9T_njA03ikd2#`aLjS*#a9?_@z z&$J^%ZsPmLs1febIw_;7=iT7!=rK6j?7-{x{j!{T3FqwZ#FHk! zzrELl@i1=SU}ICAX5+>EjhP8AR%6F5`?1aq?Xs+sla|gt68*yOmPoFTcr|0;Q|Q~Q z023qiXY2WtY=$b0fSLWiew)a<)l96BeK%@!uNl1#Mg;M+lIY_>fJ{JHNPC=ZuX+nX zknZ4H&^IN80-`N?uRJOh^Ee(maxUR<4fc4M@YdO~KNT+=$0BT!Hkf!U!^=O-jk2ln zzS1D$2pMYpkoQSKgInVi2(j%S=jGFYMzMXRWq{}hfqgKcK))FW?d3lFuI!gVr}(H? zLm}Y+z`peV;eP=uL(ev(qC%g8YTeeI!SMIE+vcka>}Ao+F0^0W*AnbBYkoLT^T$J0ZMw&UT18l z5D-|utke7*zxsG&_K#=J_KW^ug1jCWX_>~tKWkp&v>YsXU$E%bxcP?t7Gq{LJIb>oW4f9Brb*O#fK;M zW?E9BvH%8x;2j+P6qjD5Cj0BUwYxlw>M|gETN)66L7;c1&b4P*^S8-xlm+r~Rrhjq z+~c}pZ|eiFo1bMT-XbOozB`CbTh20=50p;T^~n_v{TNTI(?NF=3gMj%Hj3)hc^Ofg z5(7e;xvg)xHP2=N=4BNRVrCa^)$GtGOOq9%2i42wx)zxsJ(^nAgnJnCo;O!v#~20o z_1s9Eb5H9*c->Zt$H;d5_5AJ@_kSLV=KWrRW8$C&J1Rp3`lAeHU4`GDmvi~4a1PEt zm8Z35=Ublhtq~Nt!q)XE{knHlx~$QhQT$aJeYQWG2NRW+=4XK9u18(H0>%VNNQ3hA&tk`4&qg_Z+VudE3tPH?|Fox=ek*}DY73uzMEMWwOQ+m zB(`oMX}Ha^cD>e|FW#mv1!~lQ6~A7k-CX0Deqlh`?4~1KQR$pe!|>F`rYck~x5X;GCveG<F?pElK--w%apwpW##x8 zTX%d{%lzM7Mkv?j+PLI1FcN@>wCGXl-MVqXSy2upiTRF=%RO5|C7DBg)B2UvOMV=; zLV?Cy80IuKABM8t6C&bwNlFb*09Q~rX<*Vz(SjD{ zx7#IOj*DJe{>T;CK@)E=Wi1Zv6qa^IgErTaopO1^ND=WcUEOkq!Q=h($97Llh-48MrDR?0ZgbmzUs9fJSw(^OuiFKGBK)S-8Lc0z&RrdsVtB!Fid0_D}Tqvw)JW|y=^ViaDQRo z*$#3DE(Mn|s|*;VpGmR#p`6h04^s=oL3!^Zck%FK;lP0mFdrsConEf?9p@yiW@_#F zjH*@-b)`vB_a%!0-e@9-;95;&VFdHM$+E1!O z)~@0+?Au#SK6!9%a(0$xw>AMzBBt@q!^vhGUQC;u=<8q(iaaD$K9b_wLAe#e0GQ}& zJzz5=mPjSzs_gn&Q#p=z@*HO;Z{t2yCdmGg{iDhyQa3}+D%4?ui~das=XrF~^~Ijk zz3``YTPH8^+oKlXU7z)*ke~gw`XP&mZGzWs2%PYxksnXlKFlgS-Ym2KCdHa=d@ZHzT0dwC~^46cOB zTKuQvIe#)-BaAt(<7iGOKCp#qJfElCaOf|fCWwER!yve(7N)rNYHz9#;K!JUO_E}5 z>v6Yd#N#O+x7wXec4+ij&FvVs?|k}nFZLA0+izHs$YIUYk@YwZd6f(U@8tNrWk(H5 z@e9<1d6*q9a%JWjk8REvj9a@_L^=wal?-WW{h7Z8IeRt>()sSfJkq0{Yn&Lm@N3;k zSucl+YQ(x$npkq4`E|MFs8=#Ao&LFWvD_niTa*vs?F9%Bz#xO_fqOd7iqc5ZsfkU| z{{ABIsMqe_j-hSllM{Hx?pr-jCu*5#v#?!SWo-$S05wOpAyFlRVpq>CrlB+GE z0v;CEuk9p}NXIF*kJpc*tQc?S-!tHt@=wc)rTW{Ltk=d%&TV{F;5^nB=nGor#z~Ym z5#JUKFQX1il7?aJbRpgZtZBg9Nf3^Ws+=kMBB35fpQDar|t-B=} zD2*=73#m>~6}o3;StVx*j%)0c0gosyxTXP$#_d4u_5IE1Je;-uYL&~oW7;cCixY^i z-EX*W3TxP<)U2u7@aMWX*uOdZjs3uXc_8|nvF3e@364G;8+Kd2$^85!-oFeO#E%)j zv40F;ETpNw6ashv$63FHk?0?IAQ(9UAc?cH9%=nhkxq@8Shanl2U3_atoq`=G<2!M z1<%ScH7*t5(a4X%z+6m@4&HP|4|cCx>$Y-v0YHIw{R|XYDU21CGBd?;r;|Q=cioL9!+V2{(ET|9|$VwU! zcH`X^jechiv|jVwL+GG|<{kxONLS@4gAbP5I$GT1;o{ozbC6AEjq1XACiU$hEdB7S zt|zt6!2|uZqV8CG{~DEo4YIJX@*3@FIFh(Isky4*X?GIE10ZaSR-m2m`d7f7vbJ@T zd2=M>>c*I$8xn;D$LF?0qm_;Cp^Oy?ZHezNTX-5vD;GWnV<9vV?ox?capfF-{gDs^ znz#k9oxz4D{DreU?nR%i3K5ZBY1$pw(a&DHS9)DT*B1&Clf;O`&P>-&l)I*WD3Y1- zRK=S6$W))L7b6e>~V4 zHZRmOw8~sGWZ|{MC4PArcGFaMK7$W(Sth7m44l6~w3C1rjnjFAgm`$w>&Y7;fu;CW zAE1zW4{5AJ&(cnA+$t->lhwZu6flR#C4c`q(+dOB%aS@*E!*~!BJw34Q{}xdZR>~z zxl`EUmhnTsa*yh#k&DfJHxzM*eAx+PpdG&q8To@K<%jJrp>oh*q`xz85Jjc&Uoc*o zjU^l!Mu(1jw0FLVOhrZU4lJ)3Tifzz#qIZ{gvPu$c9L1WTu45_{YmqI$2d;m8_wJ} zB#&!ymO9lsU9@k~22iRPzoNTlQF8W5<{3VBA1Rsp7p0c1*nddl4t*TX9`B#mmSHb! zCPpP7XI%Fvz6@{$BDDRk^p(g%4yVaai_Z`^dD8f8I4nC^I|qF6;n4tbNY$cS9rSx; zoY+eo`e!v8n)et)G;9~yH#AfpP8PUYz2THL$?~h+uWPXB_`ZB=O^UeKoXqI!YagNy zv_2Ck0@%)`4-85!_~z@#yg^Yr{+lJ`JSPT`)WI58N-vxs?$ln}Te~U;hv5d3u`2r? zw&pG&*F!GxK%@;abCyrjS!tn}SaZ0;Gsafjs>r%SiWn zW%9!(l>LbzYw2NcAn-h8p`X&p?5p}LkqnmnZo%t49s9o;xmH85d8kYqe;ei>>5Pd_ z{<_z--7ftwRwi2-;p^URvRad?l7BTSGILnaH1{b6D_bSl>by5F!^5BgJ%Gb?2DD0k z9WEf|aJ6Kr-_xaYLr;+6dVd_MTU+$`rPuO!gWdKtKpLjP>z@lB(1nC#Vn#MT^-be6eV%|*=$pBdfBs(go0$(%vC&_tu)t~>1JUqJ0oqR5x+Xk{WBlH79pA*8l zJ&ozm^pQA>ezscxQ)G!A(SL$;KXJL}j2$El+dahgAA*}1H4n0U6l?JOn@@9M`~?3s zzVJ?N&w%sDH$Zk&ux@m}~7STXSUs`I_X-DJc4%>4~D^|rVBz2`YRyZ&~%es$;O zW_s>05;-gYj5X#4^?-TRlQfh35E{Shydxw&xs{}CM6&jmWUXNta$A9J?DGhzINm+i zdT+zOM||hlulRU1nxE9n#ZvhBC|@)qZu(iBOrw%D4jV2E>1F6xU!g{6A{5H7Y}v?@ zfCd*FV(8N+7qWL%Vu2%))E+6%ATR!+Aa6~7V55oix(RFx*KQX?MDgrZ%b}L~dJntI zRCAW0I6@S**g{Y2v|5ucx;n6hlg)vX^k3pUPDF<^+8-kJ(IK?tAXUUW3olMf@dW=I zwlEM0Jniz)Fa3GMFOj)c_KQ{l8V!miz5IE}qyw`1FJwM}B8xJAr*AlQZ-yl$4$qtc ztJI6+qN2xH7tC`<>9<^awacyTSM3G0_v64y1gQCXiHX^eX0L<7b6^O@TxQ$6%lfgT z&v^Yt!^DeEtbwz-NX1wEyB~nsRD02aaZlj~GxT#9ft3Ta^a5wil91crOk!%F74|gANuX%^{=wMW6k;o1636B+HfXT!ZXNp+j&JkvPn*b{t_@_c z_t5=7&Xq|gdyTTnfthbhF^5xNjews4j?q6y^`g%0^!}4v=N-(HU(IdL3~r7vXfXrM zg!GZBbU)IfjoJkB|FJOoYlyKtcRDu{-i7^}{xBVu71Ci;nrp~A^V|gNvpr06FuTOS zI`{pvW03D8#`*x4*ZH}}z4Kk-ex-}+aGTsO!TKV=w4cFOD-u{*Wzoy4z%PGkosO{Hypi) z&pf;$d(&BN-^o6fVHk0jE71!1c*X}wO&Fi|NUki(hE*HkVNA9miB=+P5dk0c-#<%}FF)6&o z!)X7^ii(1QcRwa=iVl>Gfk4P2d?G6WXb_8>nn3yp`=+wYq_LWz0gbZ~Eg)lGbMaDm z`Qpaic=mcNu4CboDwi%7jE3=GPMxaK)QnzQrVJ>1m5;FoGk?$H3d^oshTx?Ky-{T= z_l&))cRt1*7=9g+X5(7qzV+SZE_cI0C3*Hn|C(D0+!V3xmfu3Mp5KG}C`)`xcHn|} z&wwy8R~(l>?vs0Zba(WIL)v8tmi~%s=(ccu1N$YO^fM#rxYCqv{`#Lz$r0)u8)ERC zdb07Re96j|q^aB7GpJNg3*`X*A-CJ^%mns8p5^t1Gi9<3*DVlFW;C)?R8?!1l5N+p{BxU!>5FbXDD)<0p+D1KvS^ZkZQHXm3<^v^mN)H-Xu156fpur`%1Yn{ z{)m_!NUAmtNpU56q!FdqkoK*6xCKj@x&Cx$)e{cP{N-quTgFlsg4UV?w`my(@B1P7 z+_ISA9w1fas^8mfVZ(@=TFmumB<1oYXn{!g9AA^1+GXJ`?B*`e`;yrEZvMWvM)#OY zce}b@z~h$P`y#}eA$bgxc9*4A99AOoISF zoMe=)AV8_a1c#`e{JT8JUUevoK3|$4;r_k48OPmC*_*uGn0@%!DP(9{+Pn@m!+$Ow8b6Z>vF9B93B3?e(%x!*|EHbLse1P;b>&pyhjt|9Z?GU zfa}87SMtd-XM*YJ^tUksfLfp;+VW11-VX%I-+l)DMl+~EQ0?~24{}PV$O`x%VC&63 z$xx|d(cKQ>8aS8Z^qb`t&{F`~0w3-@w?`eep`phjS${ezTr*WOsD)xZFSTk*pE8=U z`76NDK~JGkO5GE~nJGOB`K(^*jf3ZD2#-vdS6#GUsyQIR#%GCd0!89DIx*rQ-r9QR z=3~%1Q*f`SGqqTiJ9sSzr-Z#&JGL?z58PGQs(Qi|JDc9I*|~AoD?5RMD7MNvy^wy| z@$a!A3Qymx1I@S7fMd#ay=blKJ}HnLp6P22xcWlS{b>4K^>6(5R>l$Qzxjpo6^_DV z=SF&0s{|i`dR^tBDKCDG0?W+WqJ*J~#22!q+v+RB{K84VuF~a+zPVk?nSN!SpQ48q zpT`$Q%`b)_a{QHiqL4QrkWuJ%bvn`cAPa|UYzL}OAw}cqy$!#ogZPf0bOM7qS+J-w zZv%weCYJZT`K>oq%wzIoH2*{1$5HIOfDgW#T7Nh<= z=1~JdW&hQ*h7!qBjgX6-$+`HfrBzhv=+vy#x_A%-Mq(fM{6`Q$p>5OL2$Ui3g@I^- zgs<2@qR0wXEc1{~mX*2sDV2-{bL8H)ZugqcDPKB7V=p+DL${H&$mo>Ehfj17*HY2p zY}1RA!11}-(lX`5M3QQbKcn zExKViBE^;0%YybxS&iZZV5i22RBbl}{T@;<=Qw&j5W!%Vk*k7XH}JCIrCuYL1~UwK ztV3a%9trXNN>srEVzG?S;>d*!k~ol1ij=n0G~|_?j!Eju4v0rPi)eIz&HcKWGV3|E zV-!n5Z#1`CE-w*{rf~8tJe%^x(=WM6pU6{WMY2?=%=qF6ZGA;!iEzJa#?ko`{u~Dt zJFf!UfR)Hu`aD^vD6!(N+_I%;__yhAJ0p{JhQj8?0{xCsee*+hjdl5kS}t5GVV!!BMDpmw;gKH*-NqmytNA1? zO2GsN!EiJuSdD8t2?!}0>EiiG zsoj{Mf*il&m}0!{&R0s2OQu;=G`1{RWixZkU}Ffq34++LStIU9nmR5N!5MloUJF2< z>ey5KXe}e>Xnp_Xa+hmIIIK#Jaj;5{jKXT?T6Qr{>3DuqS!!~x&YTJ%BKd?BX=5E7 zLV4}$X;ZY|(@B5E&4NYi$BF>-zg_W9nRuIR+zag|oha;D#1B>UlMRnB-u&o)bd)nm zj+U}<;Jj1gmAu!-;Su#kKD0ZGIXQo-QiVC5zBATi9C|3+IukuHv4zwM{akX0pR-ow zwIpI^s~bC;4)*i}l3?1!nZ#f1Zwm(iJprTCl$B)1qr~m5slTM8b*VnIA!Kw~zrP5< z$$1GD#p8lgF@-4X`+c(uR?~Iy2~=1=J?PqqwW+vIB)@s39k*c#gI!qt1{JPkHQib9 zmXFB~n*%ml*{DBnHOF}6ls6)ljh+d{Hk8IB9*u-C2a`+3FscvNwR1R)+=r>ElaZ#2 zh}}o(hK9)(u2WPqbd~8e5>XS0PT6VYl|^0pSWWw8iM#@V#znSzzu@xyBFWRapJx72 zk$yQc*&Pu|&w>4UM%pC>|Hv*{Wn9XsI zzb4|3?mn?nsp&F}HYpzOQH^zb{Hg7;k>QG)_rEwR^G6QUs)w$OX^Us3(@RyY=M;M} z7aMBf!(tHaBRTZ=!?(3!c4yO@hcP zuMb znd)5vb47I(PbluYJYU3d#B$_m&h(kCHPEv*ds6eA%vH zZa;GBIqn`qWME6dWXl!y;@vv3Tf{dJS?z_SgG1J53g-8b^>{_RKCuk}@2*n~SH0g# znJ$uiov5t3tpyma)t@X&7_uem3S)J zcwIJ=gsEM6+UM@*+RaAdmJE%OQ--grlLf-(-pFVJ417j#D$MyBwKd{cJ?W@1OQK_2 zRJM&ovt=q9N(^r(3=fgny`C$Zmr|@CEO^<{9*g@*z9lNBqHSp@=-=#}y- znHymxoYXrbmFRvbuC9*pTHKt>2a!Cs z30zj0!T6IhNCmgX$$mgD&)Px~7TYxgKAm7JxRkSy%UkJ71~kW_LjqAp=E%;G*WZ^^ z@=H~?s&2lySHt!aY{9hP#kUol0*zqQvaF@k>E@_MN#tokZvn4OdRhO}uA#n8e%=-X zzx$^qWpes5Ilm1fGcK8ym%;Cn*|STj+yaZWfv(XtGmS-8`#bSkiPJS%9oIEswR&kn zS-nNE5u-99p!N;<^)d@1@=S9sk8P0sv5zI2|o;Sq*DA-Ib(B1u#RMDKF^=vd^d} z6I9d`SwNCuWg7{A`PDuDr76Frmc}w0Y)&u5vdr1u>I`9qXkR5vZT#(cQKIupVm9J3 zfYS{bAq08@cD3V4$%|}!OWa`GAv1wGSbk2Aq-|~g^Hx`d6fvg3ny|0pTRe>Q#;3xa z@N}H=uUyUTKf=1UNAV~O%2CS?#%9j8XbM{)@??2v&}pv(()5rRHubGs56)eVubzb@ zI8YF+1Ep#%rPF&|tS)}BenS^FYY+bIJO*i8|P%kvq}@wo>XiTOr3SI*zS3eyWR z=jB{q&VhGP-7vNM*T<($35M>^xb>hc%$V*J;q1?nHB&Xr0q+ovS(_Y844)y~546#4 zUxd<0s3^0Db%|bxshmL4NeF-Ngu@7`RWf;gldmdmhsX_X8FBhPny)%i;*%#NeRsL$ zR$3azPmaM6>fk=Cl+lN^COWSrukmr|I@H+}{+)+O2Ku|2R|lz?WjHpC9j$_968EaN zicr=YBS(}Vt;fxdTn}mL$)j=E^EzZnN&$`(rzuDgeUx$|b`URtk@OL8h#PH^sYesN z;6kes#RjNwm-`aumkk%k5@H?~5YPht{C(Ynq;*^3ze2fQ;-{2~!KiDwO*d+31_ zt?L8cSvlSr12@Im%J?K*c@6#K47mwpc=`p#h zP-DB`!VhMB%%|PltMKn{8@iBa@d{IMWvW;I46KX4p@->SGU00GSs3wX==9cT$7i}n z*4elZ!BA#%o3NhXBzu#K){}BH$qDt*!}wBn9b@ghqg1Z$ZxBqXQ?WpIB2`cEvo%od z>rVbDl6by*T8sV;+lU_DYfuA&G{V5+CY;O5 zD<=hq7T99%=F`JGc9yL@TDC1qAMmBMBZ>g_#IoI@RDwpbU)(=p1>D&CGRV8M%*x&q zxdRjt#nw)KG|6A_vMi{Tewjm(-_d*RX_QtHLNFXU5`^p-%_->{nzAmxxt$dgwdOum z>*>kn!r8rFBE$vx-Gcw@E-3x2#oA3d{E;m#j%D_HS`${fRbgYoJNdAXNY~Od!?Zxp z6sosAPaP=Mu-|2Qe0#np<*`z{^np#iS3n}#slQeFS$n+b8?=`9^WM^>g+{v+7ZU~A z<@mIvL>##V);H_t_frD*CEmES7G+a1GCk}$IrA5P)fz6?b?(NTZw$qeTQtC7&V8U< zr0`5GVud|%@yn~OqdJ+0uRxe37-m>QT%(Ywdwp;GMV&0&J~hJQnw0PxE!31tGnfRS zQD>pgVNsf9iHQ3%vd5V$pOARuuj-s}Z{RkOsSy+QUiHy9SgkyY#yR{CdMCtEZ$Fy(wC!LjyNBofXo{m)8TH zcYD_<=Mml~pJuK0^Q-hrZYU=$1 zR{scrR1$`-8Z*!D7{*W3kVitGel$&xbW3BB3|@`x#GYR~QzWR9(HIj59xe2M&8)x| ztwoAcAj0aU|4(9hULZ+5iWhIWY&3TkuvvIx##-XUuA>>P|7V4W8Hy9nAK72qD=ew& zhC^#3h~v?&Juh)oaM`K$*1;?wF&xI|dQ6?xo^F=Fl-=3(^*eFn@TU1{APGjyfL7Jz z-SysWWZ3PH^-b)#4C8&v{dEb*=QXf2fsvN3%g|%p1Av_V9-#2-Uv_qgix<9qpj*>F zO!~g5u=V+atOUN!vJl%5m;gq2kf+E4CoU()RQAR`)D(GM*$caC#tjlQAW0gm6+h2S zf)yti-Dj1|5`!E0xaxG8mGxc1fhf^#Cb-3SlyJ9^Mvv8)Z%>#g^^)B<> z%7PqKK~Jv02`Lt){z8dT(Zci0A`u+3!(vJ#(xE!0ocgi*E{mMMC`(auzfh?C#X+j~ z%_!G>5r8d)d7Lb_$I*a6AU`cIN{&LJyr@X1m5lb`Lwhi%(}6%?-;tO<>>>@}f5mF2 zd_aJ{;LUU&x2b1p!a?ZxAvp+%ra|r)-H!n46qYRS%QC5Oy-*CT`7T#%f{qn$#J2O5 zW!RQs$&$@BY(^e1FB^U^~acXRFL%KIXM(w3y28x#kK>OK*k?ZnjpVqwgw6 z5OA>H!QhM*Ex7(J_%#*^XE4nuqr!m z@-n-$Oc~KSk`NgSBg;QK5pX?7`r7WJLl?tup#1hx*eRF!k`A-3wzTGL+lPv~k4v77 zBzeV7ITgjd?M$@>fCXn+s5^e-FCuk9l2nAv;lqr{1?SKu%OECo)e@^NeW%(91_ zUTLX2)&3rvPjmb?{kraz>o8UK^r8V68cR6uSgCd#=E3Z@Qmpn@F!Zf)cq6Vp1Z!*mqpZ@ z5-0zEs$0_p(5DCVKYVaOOBQ*g)=o#hDwVejj~~XADwLIFy~3N0>8q&P*|F|Sk^QH6 zhI3YdJu=28P;>n}$7lSq>$$xeEtKTI^V+zUOa1J`<_wi8(8~H7$w->i<@Q+d`FmSV z0s|nR^x((Bdv8jQi}gl&@AKaF>w5H?V`{YCD!cRS>@Ho;d48gw-0jxh+nRHCV-x$}}-M(v?wU!v=a;=#8u+UTW>R=@mi@An91s);wCy|}0T;Ao~|08x`#I6ZPXC%g1MNuzIM`9}^zabDpCEERJF;M__-%vT}W8<#3#KElzv3SP-!7dD18ce$Y$_@_7+Xm^wJ$5G^kWcNc zLhV_FS%O2$%O1&g| zQ!~zU%iS|=({P(439$@spVzT@IzMQ7g*oCgTC@ecr+M3hSl|ixIy&QQqhCzXFfe@} zE7ddSkL6Z5HR|uXJxh1uYopKloon=?R>~hf<}u5DKrw+4YslL(da#v4g7B$_dlF-! zRSsgg1@cHr%mEp@rMFK@(bhKqW=hgV$-N5U9(2to(21PW^yh(ph-LyPF*tew_V`a| zf0I;Eh=dX%X9moWWEzvu>|o@n!} z;_H6EU-O3`eW)7e+^uozm+8Hj3hGdDU4WX0lULTHh7)iNNY&AT%JHjcr6AC6i~i@0 z+!!Vgc+fhqN#biI^k{AAb4c?xl2~R;d2Ncn1w_A)+Uz(qfXUrvGadAn4QQ5HdKQ0n zXF&B)pA!CPO~{m|a<~7{`tw?mju)0d`v@^F{vtO~w4BbvZHI&B z?!fT#W&hOC!6yw}l*`9AISI#_k2(j3`k$B0id)qw^m z@42-wL6}V^r#m+l-sjn2UN>PC^`0FyUSG`2Hd`8wr`HEk?uV$or~j?2j5i#aWAiBh z71y%`*v=anst1%(f+?Qw7XoUia8v@6`t1gGVBz@yaP$fhC~ z@P~mUd!q<8+MS4uMQ%VZ5vj6T#N+l^2oL-g%EC1-{f&NnsxRqxBS|z_lFqeJunW=Q zQy(GFMg%s^w;(;|h*6!O&;IS}X4g1=FIZ`3S$0nXAY9V%qDP(B$sSAR%PvHq1=nTf zy^C_dQiKXQIqJnA9Z^xg%zWQwUVrN>apMli7Yoj_yw4==rvt@ZfI0u1_k3NL#Ni6| z{dV93$TYUw_;6oFRlRM43EY4KlM?hxYB{M-4f46uS$j>*C6`kf-J8|R67RFW zYPV>qcjXeN!TX-`z>BExIL-l7XiG~GJZl$4h}-um_qFSHYxn2V?%XcG>q(v7d>3J) zUBM6u#pl>mXJ_^6^oO_cW3agWp-_ObG>fF@HP<;!m{Y1%#fh(Z;08b!Pd zc4y|j&t{3=Kh~VuRrbC=-=92tPbGFa5$JW^B5|y|KQ*QFk_ZhHEF*Jk&5DwaSb^f{ z)jky8;D_QXV;^bh?(Qwm!iefbDNQ<|Nlw0BCh8`AYr4Mw2su<&;!!TgS>w=+!cmTV z%yA_9!{T$2CyFqO?O#r&(ED-2vH7h2?}v`6 z>h&j6jb_Np=u~DLg{fINeDX45lH_RYV0LLT@l!yOR)_mbL*8_2b+GfoO}=F8EcaNI zbMpTq?$3a-y{bi;4C+Qi<-CsN4Jqr>CwYM=h46+%2So*y)$XEW(bE$pDXY8hGm|HV zbQHwDHPq;1=KP;7kH^Kk66YoR;+|{07z6G@Id_v4wL3;!c`DsCqN)!U*|qkcZGYG& zRXg(wU|+1%r7CafP}W0JW6JbfM~X|Zy}==dBk!r~p9nGw&5vf95T!Mn4o6B}0r(IR z>i|2)jv6|hpRRu@ z@@#4c{-;gk2Em0Zm)m4If#+hsh}_L6^PPt$B3&cz?tK2<7x6yqiTgu+a(^&gYotX+ zkxdcV-(P)ZFPo92PRRPYjMe{Y1dJ(}^3^R|@YlK*Pl!9*f85m&OuL-Fe|`Fle!5Tg zrNxiSGAXcGZ(;Qqux;%Kl)}LJxt8m>`aD3}!VLqHBS3x?k_*NT#!~W(J-i-3;hnbG zwO5<~+UR8MY?XarIB`o4RyB9lDc_X8zo6IDP4U<{9KUZQ@H`>ldiezk4hajvp%uSz z_dXcVJCny<@7O1KoX8P8o~ zbi5mVNY>Lo-FK4AI|Yu$Yy@OOF3&zLpW#9w2uwU5a8pjjP8k+Hptk)6kp!2LYN!q* zE5-MZ8J!O~ERP(nN$OXO?zNd!k24zK*oINMpJpplPo52l?785#_ZyKD{7*N^TkF`E zSosw3V{_^Ye7}VX@5KLDhj9P|S~9^v1Z{|lTTq1xsyC8B+iVcFe-umvZgdEtNIV+W zGEz@Kg{nfg&9z6~fLZNH&nf!%<4;3BYUy-nv(Ph`IvGV|x+W)>Q&|pj!+sfn(QGNA z-*RLm8BMrDpt7WE6v_l3j7A%_oHeCD{c(@GT?lRI@f`ht-kl5gF`Ldk?uzUiJ4IV1 z4y0TJD$Hr%PKPXChSd+E7=g-uHr}vheO|lKCQ(#-lP#!IW8t34eVs;pe}PZA6(Vu7 zR(5(?;C?`4ebXc`fdneN6=5tk-mO=Kz{u$t6TT;CEZ?3a;7u0z+agErQ`eYQ#m>*K zX4a-|2U%`mRITxQ{yeE#WVaS*7-_wZS_N7*cU_OsW3}(>h@MHs?=jMaq>Fw(`ME`Z z`V>>B4UD!^sgXTx$kSVoLCb(C!kL-R^CriFCB2iku6&3-ukc>T{TMq}Z92$hf;2tD z#{u|OrWam7g^@-!{rjTHH{|7Pe|{nwh3AsoMo`)SkQBAEVzmfHGd$-lk|EKGD+Qd@0V&-CbLEmg1bLZs~< zIayn!Yem6OFe8xGl`+q)dXx&BwEdc{C9!Cjo$6-vrFq~YFNg-rB*aDFt^llG>=u5R z2pYfJ1^xYk@oV`n%YK?jNn6g>WLsOV9Kig4bBSjC+0gg27}D#rr@+Y>AE&N#nd_S5eL zBz-iPi>fcqhnzzhAE(1it#k zG#p@BD1WO8{$5aqdC&-?f}&%lu0uL#fh=agxD`5hFEMbezQ}C{L#uv$N^KBCu{CgU z7n{}bRY?O(l*QW@g?4SpI}OZ9++F}qw|dBEATP-ef{0~ycmhpczttEgL6Eb}7-)qK zQ!IKaq%aReu(aYMV-o6lv2=xlRc!_-BTF~ZHpx6Kh9oLpbVxIyRNz2_s$UdOY{}-+ zbN`OIs;9yl&1!c_W>&&Qb32BL1+l21GJD!HEZWZ1$_54;o^p2fac3mqlI>n~{`z&R zZP7Lzl%`z`hSz$o?%A3o(wk+HFB)pUVROo^v&Zp^FsU3q!cHZi0jKwqH3-j?e*O+P z0$f5cut2m1$VyIQs^MqL7|J&2Z-aPF#I~-f5Wj>TNoh%GfpL6%uLLt-oNiTnG;$^F;l=nZPPcf6p17;-V~T^f>F`)NiWkJS#AfC zcJR}Vwifm+C{HE{vDg3sdWF`R3BeFNI;Q)lz0^juKAxSH63b6`^z$ETR3TApP1@&2 zhtiZC!VPp^CrFY^gg?Hki5DPuSfto>G$X-5MUB~g_+-U{YPID#40#Hnj3W(b%&PD4 zm_H6q?@h^QqFCaS>VYVNXzXCpHWb6p{9ek1Q+FMzqy2cKcLv<00O&Uiof4_m89Ui3C zE&z=V6URU5O?4jwvmFJ$xb!wXEZcHQiv**TUnBmdn)I){my9OzT=%ECFtY?{z9^uf z0pqlV`ex_w`ENA#a$j(=NB_{dRM6kkl(xYfekMghgGGx8LT{x#q^YG_@ddvLsxu43 z%sukp5bj^gI$XMMb?Mtkekhp5Phu?ZOlkio7s{;(SWmQmJeg5+epeWUR-~NokvqYR z((NP&m1IWS-WY*FiC>wcAbHh;wrnsM9Y#Z6=<&|0xhxc~ag(U1kaZxpwT`=cSSn}H zgzJRon|Z~tNTW?#cS zvLz17mF>KGgd1Q%29ahZ2WAs|3O&$yc{)QB^^BRsd7En~{hmY!A#juqF(9a>vSUUA zL3rX{c8Ss{ivCUo%u>`biu)oM5O`ZA^mLe{gHt@Zf| zG|bdAdyRFqtN?WR>-mpXM-qhH2QwhqxR>P=qnVd6661A?504YP!|amk!r~S;AOBF8 zU7}Gq%f{Wlq8;GI8tsPcMkaIJ8CY+rYn^3`7&=f#urunnk-F^!XU`cMIR`fO|5wlK4^_3 zphYTd4hWqOH@76m00ynCmMRP6J+$emQvlyja$_XVGgH?Q93dgaAV?%32%_aJ(dA<7WF_uk*z?|I($Ti>_V z^T)f^hqdmRW!-b1IcJ|;uD#E-=^>7Xll^kkRVRbfvdos8PS!(~D!Gc!$=l{q?-f)= ztLJh3z3{p0q~+2~C*}$jX$L)Sn>Oj7a@puZnR+vY7aS9yE5j1ZTnAQD)>Z6Z>&P7l z?yS$b`8KlvtT$;p=LIGH9ozwzXh*EAtp@hn(2?Atr;do0kd)0^i%R>rZC$h4=~W`%MF z_;j^n-0!XbeA6sP0wrrJqWfKgU1>%@nYAwql=i&D^Uh}q*?;Qq?}fF0ZC`KDeI3z; zmm*bi0?z3xlmRnOqk^@HDTR_Opj2JN$UKV zOe_HG%qOo2$#9I&_xb*SS&9fL?^vWb%wH%&%tnoXJQ3xd!kWXT%2Vj)Uj(~)IKlZPavLxaQUfD4?AUT`ymZ!#7@btOgn1GhFfTj9a<-3Z02X1;0%*0f|~F z-N+F1&Z(tJ*)Ll@_ExpvybNA@EC7o}xR$ZjMVa^Wv8KS*D|OoVzsH)err|~}ovU7- zw88wmD=`BgP`u#HI^uitdc2Ekz1RhT|3Wn<3-kFuM~A7eoc|yE1D6?qoX4bp{?FAz z-1iLTUG6{QcT(Vcptv0#*a|d$i+K_&sH1oYJIGL3Tu@Tze#%B(4;)6i_l5QjQ1J2? z!H!@baE)8<|Gp~s;~|>@(Dpd>^%l#G^wh^^_ZZh*R_jMyIPvhjA92%|Qz@Y z)wv6_Qu8Qql2)wCUosL%LVkHpb=r${(dJfAgB8r+tR{&$)~R`~tf#;5vln;uvMqQO zQM@2DQz3zq>5LUKxI_4v>y%$i1cVY^CD}0etMhPyY3WgxVG_A0*HJ6 zcL$5g|3p;E22}iZ{Tx>4c7Ey)JJ)bJ)BWdSozz4j*Ce<V}=;P`cp$RBZ^cj$l=8q3~VTR3$SU*<)PP(NT!7XqWAhHY;IG)%$Ga~^S5tajCHF}EpDf+-$_^;sKiua!wSg)A3_Z_6e-L?1u+Pu5SP z`A9<1dcDN$>~!TTU;Nx)bsQWJ=yf9hHf86%ELzjBbO-U}A;z@ByasS*E?}yMHI8QJ$JWUUntHcxdVUFbI zJ-B)V@E%tPKtJ&(5L8b=A%`$Onhg{pamxFY0zAe66;_VK(Ico43GJ+p)r+SNd%U(*E-k5(;bUv&1Qu+&RK%J3O1v@AIQ$F@%|) zXseXW4hlTE8s>Yatc(LPy78Mm$e6vcy$YNY3)l$KO9(je@LOpixHSwJc0la}DQf>Q zNU;EI@>k)DKK@FP(HBhd4I4&f%sj_)IXA-r2eLik+J$W5su9OsK{U>C=H z#{tDeph?t=2))wHsK^o7KeL}HoUaP+g#K7IZD&fUME_Ub`1Hba{wh6WCSlpPY(uy}IC3aM5c(s-S*pmg8vmEu>Dv%Z=w}TQ7XAwn ziyAjt5DTG!gpr$~^du2&gY4JH6rtWIYC7IQ`l0<7yC5 z_SE)*wPJItb8v>5{ zyB_&9VAc)N)5V}3x4WX=`Gy%_L<%uEtlRMk(q7lPSxhlH!QMEiez}DvRDlA+?lN~K zAdRy~p|P*ljkGRSXW%Cx32yONwd&w{_C@X8Z=p3Jpq-t-t}jwsOpWsg;ikRVmnouO zi8M+oet4eJjTJv+I`Qsz{ex>HDn>l8_PG%zY-YPF?RBCLJN-m1wHFd9RxK)hbA9e{ zHS1wuz-&$2dAjB3*E=HChQT5~Gk26GfLGGlTpm-b! zsdYs7RP{3_Us%-l0u_#l>Gx+_n)9xXEQ)E)mHYy0_#yU>-uyn|mg_5MNO=^vUr3O? z6C>t-B4xrVmcj+fG;+h4y}k>Okkmnr7ltP#(|bAA^J3Y#BdMQSk^FJrns8{R_&~ya zS3+!ATghrFu$3=j7;!aOuZ_; z@ryv|Op#c8$gaXag25)Bb4-F;Od1{uuzLeeHkz+DfUL@F>)^n^K-0EciO*g*tRX0) zlB`a>)Co^KNp@xIhc=(8OKyCl3Raoog2V6!(L&`msx>sjAhysksQ*K$s{?G*W0aolO= zyV>V+ba0Tq@SoDt)%Bdcm;w0Rr(#!}L(3g_I(vYs$hBfh(FJ#%mxR8dno7P!Z5DF4 z5pz{SBrglhh(czDe@IH?!8Mf%9KH_RzaV%LjQ_^NY?LX3Q$?kHt4);9Iw;W_CBDApgg+gMU$Ps3uwAj8-JPB4@+Ao#NR2L+(XK&wrlKRln;#=4DuO*__ zt2U&3kAVhan<@x)qrZk1A09a3=r4`k){+KP)$RcC&@eeI&O_CxXF^X^THnyNqKrnN zVuY{W64G-GN#+qhHUBSvRbol8a;i2zsx_nSWmUg+Tk?PL4GlNj5*c(e?6UT#d&)K)Q93Mv zINo2iU?KEFL?oQO1zB=zjE_9QlH&}X=*X{}{m%dTG=8OuqrE>Uc=tm$9bK;J!cq6y zhB#%1I)<8C$x&9Cd7*3Opde?aH0Q_~(7%u}Cn@=LI1-)0lghNPE>UDIo%> zMLs8tVC6_$!sRr4|lkTI7O-P za-T9$<^&x+F??+oX2|vJZMj$-VQ{kIx!%~1(LblYJ(qwpL|ejDzQNRM9ID_@rbHq0wH>uhFbprV%VlOAhj zivhEmr|=wg+Y0t_)z`HxJz9Q2pB*+rc8S4D4`*IDa-q~W#A91GijC?;?M{2$7=*1y zi!c>*LLlMx& zInUKltjw{ZE%Nt`UGelarP;Ce=wh*?rA!K)7t!pTuN}*CIXG&Wj(x$Eo*LE_-r*(e zoWE!&a)^nBQu?YZ>jqXGAA$io;sIpb2>sSiHfNUz-}&##RI>S>{o)@)n|u8x<~u)! z&dihvaEW_ea>tHr9Zv{Z60&Hc^{QRZ6k~^IA2P$7XmA6ovDU`F`;A!#zYdY;%UXCh zs+5egv9rhTj{HIvrb1A>ST+~R^C>psc<8>6ZF|sRR#PFI4acz>H@n+RIH~wcFnxPl zZiS|0owB<`jo;rbII9ABWDOT5(qwKj`cEJgn?p4LGqcRxg2$ObaBH<)-KJ|E2l3o; zNKMM7%G8R?M6)D2M-CxC(8+AryZ(iMo=|ygK^y@wyuON(Iq^NyJKJ9ZP))hU!=6R! z^p?!=jj>{=gD^d_R|V`7kje2Rm8bou-1NXD-dPr1Eorz-jcQ+_f;6!#C9|?t$_cJx zmfHN*U!TVlbzcy+2mX0)*e}sdANiCQ6Q2$-fIvoLec(A%;P{zwzD#wk*>#b=tSql> zFn>kS)L-yk*_cVVOCFMNLg8DJK%UGi{4FWIywdSKc-p)h59dq7R!41U za916JSnr{s`vNvS@%-Zj)Vk}(4`0# ziPO=n3XXdR==Vd_>RR?+ohrMF#%x@Slt0PYIuQj_<~kcsxEGWLuyqBV-{ywt=!WBt>q=jRL|XTU zJuW?o3Wvr(Fs8DpNmhODzS+Ra^x$% zpQu{Sw6Q@e5Oy%Ud5SPlE1e+Sgokj0MU|#1bYLibp zalLuOHOxTbpm&^lnA#kI^TH$^(LzB*MRosv?1~1p;`H@hf{%RsNhEBzX!m+4mY4RG zDR6bBZtZPQTD&%TU&Qv+RR!fI=Q9eo%pL5Rf}~J($DmPOD#k*Do&7C5Epc{YDD^d2n8>O>L|O$@RzeitytK-Rf}F*XgUmlX+T9 z1gg{C7O3PTRbvX-xUXUPnx*rHIxq&2eKr)P&T4Pi-F}CI7gU2-V&f-a5VgbBxv8nC zdFcX&ul5U&={@b=IFH}SnVt!H$FhjBOe8Fi8aN>-X&v&0@^Sp!C|SM`nbgt8D;Dg} zImBd~JDHA$Vu{9nG%_a>BU7OB@c$6ua`(or)$prExW&n;ve z0?)brh2wo} zMFa{_Gs1awu-u;_=IB4JbVm~$A2p#_YZo@KNF}SgrD9trgl#3g12B~$Z*^#Ts|DS_g-65YblDL=Q3R4F$e7}4E8*{?3N?`snK^LorlKV2UOFCL=y zM631L`91SlO+;;EaY-;e=VJMf3bj^tK=+z6ABUH`DH%t0yq=<>vQt4m=D%&nW?Knm zhG}>fL_^(hxV5xqTSa1=pFVyjix*tTsxx`y0;v-G42(q1^r`TEl+3#*8Hx z{QF4~=?sJj^bH>em*}AayaF2E!KuY-KbD^>^e}b+rmR#du5N#_jbKGXmrRH8p%pPE z%inU}<$iXRKr6-%J}uO;sf>25`xl)lakA#ych7Pl$FR{(RiOzH%29BHpBB|YZ8Ti% zdpCn9{)iy z1vnHfKoCmL)09AE0+(BG7dKGpYGkCXkjoZ$_N!Ta>1SuZUvJ#V>QUik01F@~JWA4^ zVx;}qj+Uv)HdVfVl-#nOHzFhPhB+cYppyr{7@o!}3a*M=ml;5Go%PrPf@TJb!j%-G zlq&AAV7X5|C4{_0lI@R#o?Ab~c~i@;<(b)OH~L@^10AK0%n`H+`s_x7qGyfl;cIa^ z`X&Nlml9ul>x!CW-}pJ@&zSY>m=e>mDhyj4%4Tb&^cGyE zSR43T7)A?i{g66J9!d8u@nsB-HaFSj$`T7DGG&|FcNYOU@v-n{^6xWR^3zpdp1e}L z%Mg{OL=yIqSrS2;5eKv?t$En=q+1pq+?@$|xU5l-lciq6F8JhcrdYn_UGG_#0$dYZ z$;F?qnRXGu))uRJS5Gy=S(52^(QK)A@}U?!+O^5wp9ajptzD*BKuP_mSd2F3vmeXz zE;^oT^NP+tx*6)lzbN!C`M!RqMn2toD4UFpGCf;l4Vn{h-K&b zqbs2cYYNV@%NKqAw`+c3A$|}j2B(jdTUb~iy7v(&-vh2Jfo7i^s5W|cBvWuqJMUe! zffc~<0{p)j9~AQ$j?wD1GOCTq2#;7kytA8xj1acL1i=`l2Cgv9@ZaxIdzJap_y)41lw52+v4J0MoXrh8je(CXb<%t`X zwFAtEb%hD6k#hS&8*M@F%#~KTgRPYaHgJ7OEn>T;ash^CC|a8#QWj6KO#dysS<28> z5NT9v78&j)$>duII+J2F9%)^Hnt+kOO zY`q=ktX~w_=qlC$*CX5JC5=+@XI^lCOu4p3jf(K_+EG($7JZPy&Vns5pVv`n`}YcA zqzL-iU3|dXN`BY0)DtRZXA{AhO-?3~+@Fkd1T!;=ed8%ENf)EmCpwgUD8JSRqz9xd?Rl4O*&{czdo z)vlj+^EWU$W5{YbfB-m*pr3lUDc+?9tON%~hNK5XLQ0C^ire@Axrf z^{fHJ24_u*kVsQ8tp8vf5*Fy9qds7$BFS=88lSlBudIz*SIW`-Eu z2y(H|^Nl|5%I~%p>wY`o1C(X?4*|i{0;lRUN$p))Fa))kx*KD^7o{IK9tR z8qmER4hgONeHUNv0XR}S`#|7K=kQ*!4N=kZeFMYc)AfjJo60XY4?!*ZWn>CeH`j3+ z3ky#b)R%hV(^QZg{EJ;&L+QMQ7+`#+{Iq=*AFg@C6g+Ij*++83>6rLe1Je?kGtQz6L%7Nry>v7OAc~r<+AYy~oUfDn$ z>|xH&HZHG@vgSJBx$oGN&f>sm5qhEAuKL=jO(ai>7QJbm5{l+Qxv1`n&yaPv z8`LyyZJcRMp=?SotUU-o3bbq*J1O$T@bgJqZT!NYYV zik7D9WC!hrjE=^jQG&)Wqk|r#BIYGZJBcLu=ZjZnHEd5tbtQhS?qmyCnPtDUt9IP``L1LhZxIs|lNOGTaZtuA~AnV_L zzK&B%dIxXK5u1xFb7N4TK8XU{ ze8RbBqq7;ftpHwxpZEun=Y$;%ZJ25X`vM)i8P z4uDa~5Q8v#Y^N36&(A!(x%8c$Mp9Xm$sF)UpH>{W^V?m=lW5Gw4T@Gd-gB!rbhph^ zqTMPhStN?qq{O!@>Y)K9q>hox0f||v@B$V#M;*|Qm18->$rvas6s$WOT5!d^ip!mN z`PBDP$;g%Fv<{PA!f0wIuu6n z_S;lKM@oJnVM=2L%0Gelyu+ZngKc6{r6g@8Gl4Z;asOcGF4Q&a&N-mt@4>{UVKo12 z`RM>t(`x+3_S-RFYW2*$m-Lr=b-&z9(cOirNzpaF6BfmVJ;{$qB zhaV}uJrU};_V&34GlXkdx)IiER81JJk$Kc%&CBZZ(; z$#`jq!Sk<`T0ISk#(;%D3xA51O-fpwX88` zHu+;yU9tAnz3%lR7H*TnhHKi$es;+;hw2T;tn0g5p~)GtBEhem$ro4^!OxyF9!-vD zo9EzZ%A;cQl~r1OrG$t0#_9j?2*CoVQD2D>6s% z;c7G~S@lb#*y<5}(2y^pniy1rsyW2_#uD_>k$?JBZri!m(7{7p(OcHpJM|5XnR{0v z<4ZZNj=a*^2r|TC?WdrTCHGo$gnkXSY-2V+1u?Y+ms*i|E1Qpj1BPc5JZOS9SQ#-| z$RJN6Os{~_Ueh8A>ZDsm5$%uTU95e^jcC2@1i`Y6HpCU?9uds|2E5NA_N>$6C1QlD z-k@ObW7Wnz_4m=#)I41s5}!Y~T3g%r)NTH(Pg9v#a+XXh>*bw>RQIyg8>Hl!SQnH zwpjQOS09}H{XTEj5wFS5U}ExDOwN497%0>8F@I*_L28SmfOVzK#2~BU z-9o!H{1jg7a@=t07vcK1<+a0`D*AQ379LZZ05~yXff!+e++KnA#jc@Bgjor!K}P8D z!U*q3%bFOq62Wy5@3hH$osCKx=Acfq_bI~e*%DZ*KWlChjYgJ#HV(bfDyX{g;kJHF zn@WB7;q@K(P(B%w=f~souHYrX(2fVoau4JY^8#72#nxzhD{XTJm*ydC1F zy(}4F$ocTG_;(uMd;nC!(4a9|Fv?G>lMvn9Z&p2FZ|$x1LxojhO9A3j)*G|#tcx8G zMn1({?cgG%Fk2OMjMnMg6@?=B@ts9u^cebBsI9tnBt2RWErPmX^2iyw=m9so&aEW) z9^z-KAjj7tCDM$th2DZADNYWF?||MAf4b9=y5gwyy;$#ElZ8FQ-1rAJ+GEZ zV^acUhRJt>BB5*rtO{(kKul@i5ITa^Rsbsy0&%|r8tzITZ-g4Q#aIMQ=8q?m*Z?lp zAL~B+%8r~PAzR$O<;ODFa@i37rwcpLDkVC;rQ#qy8v(Gx|Yi4}l>H5*HM zM_;aIxi@ltW{r!}f9Dgp^4y5l)m2{;98Uqx?(VYM->ay(4G~O~n${JgK2B0?jy*%P zFFNNu8>y64uMbuv%z_(&5wF}O0mJ9sEla4dB;JszW33E;wb|7Gc@=$Q6m%hb=)%g? zANneAl49PGL?ro&?dyg)?h8se;}_R()|frz#`gj#5p!$}hIqmJ7m06P2Unv4hd%7l z{3xsL1%asLE*Eiuda)Ojp8)R%6!-+^_8sUA-fa;X=o=7Q0zcH0w6_<|CqR`Z2=pFE zm~U>;-@c?)XEeFJW`?!gUjA=BzvS^Ty@pI7r6T zVF#>q_R?tS?1^8KM)r`j$cN87z0Hi`(GE%s!mbey!|?)rqIr5lo2E_<$mKVW52;Mp zBwe>Cq4UQOeyeh5z}cSDNP@kC?E}za`Zx~o)mcO>R;t7 zEy4e{NUaiQ0_vQ(dIn`)BrIL|G+%Eav+2=QN>QP`Z?qez-R4k((u!f>q$Ff( z!&fGat-H%TY;0|cMh#9XgaSJl1yyoC@yGm}*(=Se8JETv4-mcaUVx81d(gpQsq#pt z0OPhHW;a#{Q2o%MQlDS1h}Ad0Q&u73q8zKeXosSYmj%lXqbZn~sh~eT9%vU7aP=+^ z(u~nd zIXu$UVY4;c5}riS^(6Vp?_Px`UuFC!8k}2QqAHljN;VLZ)s{v6t0^U2&#@QXk9fPJ z#!psWT?mb(3GHh2W=AQ;IzmI#Cpj0IYu_OU(-ZdF1RfZWrMjZ!f@EVe8dRTa zZG^2Kez~Bp-hJo3BeWVX-mCAnnqXck$2*xWwKpE0e^5K?0cuHmtE#H1qR68s@g#ex zd&P)Hzv(kY+nke?+hkTYyRhBldUG&*=L)_a{W*e zIsrN4NMzD;Ar~lxtY*IEQj*Ej^1jK7C8Q|o2zX8KTx4-^ui0P1n-d@_$6j|6NP0^T zpXBaoD9OiNY-XQ0Ko*uCnX$+lKH~^JUFoxqS9IIn5t0p4QhV6*M!Rozh3v(heH&v= zSgHMZ==YoJ*cC}v2$R%Lv1hDBSGZBHVfD{_+I|M8FzpOEMt6Vkvh9c4BKU|{t}mBX zzlp~NDFq+VpB9zCT81a?o0ec}?*7Y=xuJMqR5uvQXSQ|_ihChilK7BHdw;GgT^m)| zYoRK!*Ttuo0GREl@Y>}w1U$|)`&EcNZ-C!0urv&ncH>#{C9tSRzAoLi=}QFCGT<_2 zk-;Fw6Je!F#VP3Cd0eyqRo2KWzvSuJxeGs75lm20NrvP46o8tD=+R?B1w+&4Kh;Be zFd-BHSI4H`L+*gKiBd$AQJ9&1{xTJ$?c}^a5Rzx9$Nvh(WBFwol!4Z7j#11lgf?pX z8WdS3^=R2H`nv#($G|?anJPfFS{)u8IAW_7n47QT1atsUYjpjzMi^Ix#A989H_HE8n8dUuAyBnf&9^YVJDUh;B~>B?cJ%FWU2 z@;WUWzklZ>HN|JIZ2}hME8{!7apz@gYt|evM|GapJ5+2+0f)!<;Uzb*?VTu*Wav@y zJ4T6rEi-(NHcxUWemG=X3`V8-yJTbp{N#Q7%qE=;d{;GV`o=65*=`>u_4gDso; zOPuzr5;X5UcbG1``gu<%ghFx3d$knlD)FjTQ7g8DBUQ#{{d7lcULMdrgQ=$nn*|HC zblA9C;Ww;vDpo&&lFM92P_&K9@UMCMCis6SHY(7}R{5SnOv1|&99HWs-oieNSu?%(&@QXdUzef6aTb|mL(7cucc zAgRwQpW?;b;!=z8@!rLIO^Yc!UOwW&HU8djL}iEG!?|c za>VV!6`j?;1Lkp?sIk@fxYVGDnno=z6={A@M{`oWv$?U7VXlDaJ-R;TB|qMVHOc1Y z{Mum9=Id`?erEyPtv>IjdUNmI-UFHDz7(f=EV=aL?3VkD=jjFcEcVzbqswLFRY6!8 zV0S2>PYfLS@j`;H58F*TpWf++Kmexx$D51dS>M_XRMaf5VR4(`+MhM2;4ksscE4{* z1SzQ*;S10tmGipFmVeRSVodmIlVg)_O0N9&OQcmviTUXECI}oeP8U|D$4g#Y>l^w= z88uH_$uM0mBeG|ACY$!My3+R-biB1Ki3p?5(0c?t*GK24{Ktut4= zP5#Z7o~$BlY(-|)gXCvPwW%tI5$nC9So5>fjArv|5k=I9c-lbRB5Uk~eD3+++XbF- z0NX~_9i8}}>8)&&i!&bY9QzppN--jYO~LyF9f^J9U*Q{~$X*`hi(+nYm-!z#nR0TQ#0WB?l{)Kq`S4mgekfloxyZU8y(@hIzgDNt zE`&HBgGZxD*2$`dVtzU))|^|j+az8v#!{8hMIwzpl3fIVBrn&F=&S8UmRg@XZfq8q zitN~q9?f0LhB-E$FUnkSU*A`DZbkwssoF|gcP-4Jk-&B9=rr|fd-HVdQzhe{r{~Dr zW#8$?yle`DCbespyr37?1+z7JQ0QcW8+RJSQrEM<=!#&?wJzXB|5PT;Xm;;Y`i`By zhI)a`?0AbYf8LZ`x}HV3YAzU~(<~6M8$Fkvy-nwkj;R{K1t!|8fA}t7)*Y_CxNY3r z*n9%VueZuy1YAuNr;Xpg#0`@02sll+J3AdDC1nWD*{1>DV*zUy*PZmLKJ7_p_8FuA z?Hu_3Cq-T~+T^oSBJItQoof)%6L9Ifh>hf4_>dhSzK^v(8Y`cTB}PUjivWk;$guYp zOi%pwh5${%AY3n#q9q|S?Kn~Nl{l}-Alo6Exq0^Fq}w!&dDDku3T#-_25Y|Fozr*h z?2@~2UDam^=7=O@@x8nZp-9o;-8>l$hoVJKd#@h%d0O3H$B*zA))oR1u1{** zzc|yNE?v%Ff4^{En!n5P+H7g+SpXTxDU?;g22=G4RRx@BKPSNZ?rZMdxq#?WrI8&) z&vGtYG+LfTm_J%SNL0VkvjC(3=OK-pV^mNdSF|FiMWkiSI zBV10=2Cw*>zDIPs5wMn<5(2T^Gqv0&PnxHX96rbz1TCM;YU`YcI9yM!7C_MsfHN5& z?VCB?va&8_pLPX~)URUTH(RXaMo9l{=uA()p8usMvM$=^AHP|B)c5-<7o6GdhFx{3Bz5_T5Q>cv+aM~5>hgS47dpAv79Q{Rn z*}C*q?N-li#NY=02w_nGD$7E7db2}in%DY84C{_k%bnI1>B`%b5`5P}ESpXb7b47o zJKeG;wIHy%)eea}(+f340H@^);P1Dc{Kj{UXbXzUbwtX;|g3WT#sf2D} z_k$@Q2o!(ZotFCB7}8T-QLzlLpo20lrWsxMUlIt*C+sZBYt^>E>74}i758NsudZ?^ z<`?%&xo31wSD2d>;G>x)v53wHa$>$UI`)cxk@2=Vyplf4t{pvE?C-oP6)^yZLG@Il zMaOWIw5OOjSE+aZ!_b|^`xLY_{u|%zynzgb@ok7VqlOyr`}DF^ zDDT}fPvhpPtz%*BLW8A=hepCF{jy;ghFNd_(_Yyyy@Q7(Gp7j@PeetR&yWOpIjpLF zM@I!EP5x*59bZjh1FKGB0Je7_eLdOmrn|dah&}Q^*g?16`6`(Pn8?I>Pp=errkR<# zp^@L-TJOa=O;dda8{>Q`~YmGVYFSmT>2TbYW~-E{ADv(v z6H8nCj_6s9Q~`&#M6K85OQ{hO_S03Xc0cCMolgKfVNFDA$)K2d%d+&vKX_eV7|wD)jl3Oz5()AUm`6B3~8yU zeS4R9o0`~^5}<-NN9_d7QBmPX!t88pGpoF1pI2VKFwUfSvga+U{^5h7v-1>LnxOY? zUr#BwOhCY35b?^j+Im*v-s{gPLb%p<{Q@??G)$#Caht!rt&^}~YL1rF7Xze}I!D{T-E z4fYFVywG4G7GN6pr?o_-sFjj#N4NX6TxTvb=^ulZV+Oe7{Z7^N^wJkgV66*M&-nvYkNy;q z_BGe!{$(#EZC$jrDUAMnYX+-fIl!e+bygI+RZF zXi;2y_w$-p|MyVRyviV;TDmqy*gfmbkn?Nf8jYFrV^QNjv3DDuY(Hr_Y|r-eeH7jW z{4?(Mg8&r_zFy(C(`|M*8Hb%E`YFakL~Qyz%O^IiXRAF1^GF>FHEob10=wOo~ z6*6PhC;C&f{kl@7`(N)|g*FR2k93lbRS3r`ihS%Dt*EF7IaoG|_~yO6*4lV++`l?U zCMA%0<1$F@XL&JI(A`}YjeHkyv^i@z6L5k(Jak2COMcuRh36`oLpJwwmBJtNsVXP{ zP-rVl>(j0XVj_~BJgF3^Z(*EIfubR=y|y!Xa%Lj3d)6bxS6%helRvFD3@0jmy}`j0 z%v+aEE1^3ZZs*@_ykfSlx_dBS6Ra{@gz6T`Yzt}3H<-3c$fZ$v4}c(GApuP@4H*wiz>k&+rG?EEX- zgsshHtT0Vk=j+$6J}29HKdrvXWuMKh#wR^+#Ev~N(Q{K2ank$ycd=Dq(?V~^dCG2 zq<#E;v1HnSh~o?`UPiyAP*=5UmUBJxB94#cOm`Q!>%@lTYBqKq8+qm(ZRQhIf zM9QbhVZ69~QUv8KHHoMIWhQbhzk<Tdr2pnZXKhiXR)eSVomlq~Ag3QFiC|>eF)~ z7M`XiKuV`7SVCiuSy@P#L=Ccs2wCVwZ5Z7Iz!^5$1*57m7pchQPUU1)!{+P0=#`r^ zJMX!+GoD^#UhbDK@w?9*CJK+T{VptK(-$839d~!B%ABOG>;X(g7L2i6p0&QtDi3C~xrUNXgB$V>69x!*iR86&W{{Uw%jHum)Gt9imOQ!{2*aK#!sZTg%S-Gu1Z> z515r84{N--Y%#@t=Z6Z&5w758$Kx=qLhTWo%Cn#P)pM0eLUP|9*@~t(=>4RSb^LR( zHZEg&W+$Ce>m^7WHFfp)VO2AKwQs4jZ^;Arpy8v%eQUMJR!Y9e={oC$o#rd|`M;$E z%>KU^d_bVH{6uDxu*k37&l(`XiO;+WS6Vy5A4)ZAcx$4yxAvF9Cp^rBfn4^&4buF~ z;X$(3E4d(}lEFk0qI+ZJ7io@*`K7TO5m80&ORdW-ju?dP{`^!s-nyBMZhO?h@hdbG zaris|IGCU|yC=P*28dQZ@d}3uEkb~+*Ma4=uIDMiPIJBE;G=phB)BHs=}Io+?hzmp zr{J&~={ec3lnkfbyAHJrk@O$$)Mxk2H+?V zDDxTD@~4*%zTNPepQRM5(_n&lD{aP&s+`?4M?;>5lH1SJnEZ(il6tBhH|e5%dd)YO z&iXY)J_K!4+qUK0pVa)vOz_^0VLYc;$!AvQQ}3*b7G6N;S0FnLjcDug<>s5?-am8g z2L@+p^}cJv1LU)?%HtcFnX-Q=Tz;Yl*9o0l^-JOL3qF;%`bAY#l9qwNmY==^rg~`P z28xD#Td!U;AR;2E^FC7ST$FTdUh;Zh=eiclp=wrmc#S`fx?|K_S}-nnkSu}AV? zfm}X>f|AnQ`)FIpvGs=*cY^tBdUjx7OP?x`teV||@09JT=c=SVPs-i9z@3$n*?cn? zunzcdb57pb6$Xt&FJInxA!Zo=#yzN6`mR%GaCUj-5^#C#gIO18pass6rHi<9u9hy4 z;=htx;3Xb-k}_zKK0Ew7NWV!}yh(?+dj)dCx%Cf&(rG%MFF#_a=Fw|taoJ@-xCb10 zm29UmV2Yi40~+f*X09+R>1=$ky1#WFvGrvSYiF%q*7>e|m|7+!@~;OuXD1ST34OFy zX@OB3TmZJYwA+5%%3k0nRh~pi4mNQ;!WejJgYiu2CsnfJzHdCtXV_~;si;{mAANOa z_2ZOiR*e1CiPLyvvuPJ!^W~7y5&&($j# z_*-V3&8lEOSw%`p+V6j#Tx$P!%eCLdaX_CwM_pvvvyXGr;wT-Shq8%%dH3#fSOTH~ zSixsr^PbH!dn^YQXCyMDb+SLVMPy< zY)8h>Me}-y^RczxlbSytsOR|?vejd|kI=nk_29{a=mH0CCOmn`1DqDr7RAZ?0JClL z0K#PJ1TAoDlkYp#nm(x2Kf6$UXUB1~peUPNZCg?v96VA|F%ta4>H7I+LMK{^V{2xg zA98V>+fdRG5Iqrz$6Z=UA2%SqPj_=9!41x5QfATDR~O>_C~UJUGoO&&VDKFVqS}kd z#u$x+srBcWUkycHRg?Il5Ta&1zf;2a5q}oHLG`YV;QEhmqfwx* z>%?In6`75>nY7b2@*1YyMZrwxW%zPkQpgDEq)xQ*)@TH?a)Mp$TNYQT7K?^eMDA3_ z`f|8vy2_W2Oqk{=L?@oGRm>H}pa0@QfVf~nuO?wY9yyCMK|S65e)rB@nFSb63el4? zlb^5;A-MML`}gnEz{7Knx9MJ<*5zwAVJEqp)jIoOY#uvTjrro2=Id8y9La(5jKH03 z7OmXA{))}XLT6%Pf;ihv85WdVa`jl<$1ZOFgm&@)=Y1`l?ZgUicURO$SU*bUoWy(&~NJbfE`$T zUpjEN(6hIJxHW67n5KcE^fuP6_LAqv2l6^Pu7>f)NWCb8^hJ1B;JbG#jP%dN&KT;S zl5nF2-BF6Tjl^xN#W8X(lh~O|vE3xsPP$nMpNl)_9z08IolNg#*A+l1LuCO1v9?s_ zxJo&^y@Xz{F>ha2;V<)?*^N00E{7M^A@%l@@YO!3zHN;pK&4P?z+PgkF5>!he4`2@ ztHf%K8G7o+I+oG{aa-Z@Cp-V5SvR=uAgJ zRaAI01mQUzQ;@k-%JfJIUm6)=B(lhkFK@irx8g+V=rYb?@Emo?3T*hI`J-@c zI+)SHDh6%6y1cBYtXde?n5WPja7N-!+z`dgEXgUulFo@GOudd}b^6jyza9UfMmD56 zgvpU?p953p>gyY9vzP(Pk)KbVCs3oVxt7&8tio}2ETY_0H@s%?)x1>m9sTr>@#vec zJ;)!*r>0?M4dN2xGElDTC~VXAVHa^$QALGFZTqot_%uuHogFHVRNXs)%)%t^g{S1$hQCT%E_SJ5K`WF zNJTG2sYuf@d~DD?Mua5acBnb|_vc4maY#^*G1O7ZNkj zTAxBvOr$(rS~`ALw@L>Z8WN!w;%FYFnyXmTLKdUHhaQ=jJ}uXr@DI%#YuHb2zI3=Qtr{T5L!z>O^=Be*d7guL$ud z{eJk1IC)o20>as}>~~}Qy^u4@3v*wKi@%j3i|i+CMarK6$kbpy!*dUVwLYg7BC9;bw{W)o^Ul2jDt!+1<~_r5*l73D5n zL{~SrCpU`Ql%^acdIeRawbI%=#2tkP%fIb?^G&1f)QcDMl>zJs_hH`;T2`|*Ii|mQ zK2*imXzty+yHLci;7*EJYwYcIHb#%N#`LHBhaPfsX4RTFIeP}Z!1cKFxIn_(C(EQk zdbKB5uOOR6cAAKq>;J~i=6`u)CYJ8F9W1qv^j1&Q`aw6TmK2ASP(4YfWR0}s1WM2P zqEe(#k6me zhuVIZuc|B!YBDrXPJ8-U=!oKp5qklrYX=~fTj*>wcYj^GnpZqm+x$k-(Q>O*6BeIr zyk6Z}>WlARe}z2SG{cZ|M)K((hCq?=oE~2osZQ4+%>Qm={~Vg#J!5qKJf}*;$Ht>< z8s{So1QxGH9t#eV3WWa4GQzH4NCKtDE4I6DOlXc&hrWAzd16MJHu)?kD+mJl?(4P< z(u*yEDvWGv!_B8ftmfV213&O=)$kL#&N4)60BT(jV;~T&N|m-ccS%?{j)Em_nKl-7 zHou91)9IW^Z!AT$<84O_D+AQiRP!HI12 zyt>H3S7f_9iN=!!x>6pcZ&H5A+3@JR7@2IvIFXLXibX5?kDee4OnY-cB<;SnLuJWx zc;L&IFIb|BQ)iGjoLs5NZM@~QuR821%aMoL^o^K?IKv9U!fA>dZW_c8!A7R?1+rIOO zdLl-`W@_i0|wE6U0oCp%=~JZ7f^Q#6taje9o~!}t(gQ%5*%wUp~1Ib*FN+YXN6 z)0-#y9_JdYqCMjD*&4rj;4I&VOPlm1#GnRTHh-5gD!UqOc4lqy=oMu?b6y)g3~dsW z*_EuFbj4O{vp6}$u=IJUJ}e0*;b3k|D1>f_Eyei3`%-i+565Sh&S>SkHq7=4ASd9s z8D}t~8HNdZ`R?lUM^2oOuvj^cL@M&N=X6>!^3;<3mBv;CTlgD}?CsZo^jt0oGSp`U zPWdZ_)#rOkKa5u$k>LiI3@y|ApzPYM6E|*h+rJusVrw5C5RMs~zjUeGn(!httJj+4 z^_!z~);iw~g45A&ouQq^%5vE$Ul)*iF0{P&*Ny}GX@BOWayu<$hbiAp=vH~sf6{P# z|0OO@{I0Tc-%m3;T!5acS~HdSC1?RuxbT#SX)VP^$_Xr23-#`m6P8z2T>T?* zEt{$2H@<~BPpvOol-Q+!x{S5hh5A~35tP35dMMvTKD`3_O(cXMtI_t%%>3+>?hq`BWoQJKBTsjL zO;e$|9;z9kd`L5aQ$vG8f>@R4mB|S~ludW${KoQBT*Zg>spV4rlY?pJZQ6PL0#)vQ zp-!fT3;v)|Ry+3x|ClC{bC?EGFiy1iqP@Mn`x3DvG2__a;9y(2F6Tuvpqb!QiY~IO zlGO_JEu4$RU`A$mPX`ADR}uuod#wTaa%Ys;(7;d!Yc6gvYTxi(Ibre6x@eCrzZVl2 zS&t1y`a|?zWE@oeCtEX9;drDI$#^we9rl#-tM#>6=jSzVPn?Z3oWDI79geq@ak^(S zA$wj-Ozhmb;zcoOX^L{e#Ir+tiIxlxTrLa0ex~G^ttd&t zn!k}Nz7NJLtLwfwEA0kl0^J1V8HpSVywGO32hsg%4J4n#P+LO{6Kb5ADH=I1;(Tb7 z73(ZWBP~%8zVv5Rb8|nhH4wlB!A!q?(07odtV>kXR5&6nvJGq9DgfiCdEYs0JkW`= z>}jh^HpL4rrrzCDf9P5{`eW_HK`XF)9Yz8v%_e%`IBs^-mf zr_`TlcAqm2`uil$=<{-8KfhTh4LsVD*E~R>13EYfWnbN{t_&!sF>{}E7g#@ zvF6=RS9;d<-3LCeoaY$WfaSsH;=6LlZ$M8*h#6^zp3my6E6H-grZ^Y==edq0*VlfG zMDWu+(0##BYGsGp5`5jf$nG;;KlHPHXQ89lh3%buj#Zq;FG@-t7G zSp4Tl^%}?poHzYIsLiI+xcJ31Rc3z^0rH@ShX*80r|iZEQU+^bNRQr3C2-9P*eL_@ zHtLaaNQh-as??Bf4%+XIenW?vVTv@c$}_7@mfhK&X&1l8*JwP`8VQ*HMAm`o) zU$jKPNfG->Hxi3;tCmsc!N@vpugAI19)f++&au1>W|Q4B`)UpMJ)9Q(Dqq<9PRT__ z<}?DgR$oT`m7?(?LBX2&2E~ETXf%3*Kc#h&=0+=cFCo5{)IJm0&vA4MkM3WddgqFl z8M@}d5oS1#*j!U{b#?76$U%k+Idm+_!OzIJ_UtCLLdb)Uz=2W2umJ|pL^@)=eN-wa#gcXkKyI{cOcKz zw{C&=f}n2H2)azK&A(?pkJnM1H9{7a4m|#8TJ`WnC4@fYk5ZEiwX@i=WtTA$dR{8&}Ft~y}!Tz17>s&Db(#@ zx}}z1U<3Ggv*A)Gsx0V{T0%SbjPcWMT(_!aU$~3fBWiNb!nS7Ihz{qdMDTlSR0Um1 zpYe#P6(Qml$O|gPlqY{mdlaL zI(5x%WopL`Y9nghcJSi=qSg3ZZjf)g zY)-7)>zPGXR0xWVGc+y~A>(!tPy74uMN2B3l z0$u5#BynWC^qqA1F~HTlG^nu@G0_dKekjDx<0#TaDe);J-Yofk%xpy}U2^29+*I;p zVH$3ge{7I{VXalN?0osfG3o1t!Y#)~!z%e_Z9EOPHnCFr%WG8gh$_a3#azAv;xv$i zx8kFgc;I#Gn0PL;d3SYTq{cX$Ji&#$U0LSL)}E?`e_Be;oT#=GiL5hjFpR9@yw%t6 zl$FX(!ri<+4Dmwe(kRrOXHSc&JeP=;eSYLL9o8Qf@&-q2 zTcTe2=>$$(6d#_7XkT_8tfB@ojzc%FH$j66=LZHZLB=SLzNyus@#)h`0u9oh)8EV6RSWDDIBq?8^0F=z$bF1zLV0yoDjY&# zLf4{FF7C*)iYlf3bW`l@V17bRFxnb?;TVq%2ft7yFDCG7K+)Wx&v0wN`khGs2uy4v zfifGZE{hd?=`1(q<)474F`4xixZqtKTvt!EOntgZ-QfvDI45LzkhzZCI-~;}Xjk)> zy#wQ|77w7b&4(<48e{YJ)s-JF!WxD(8tR{sn4Va&#}NoI2}UbZYBcEWP*c2JP_SB* zvdN8?g!X&8X~Z4i+mD~55h5fCwJ(s2KGS`rqhSj!=K@<03}`H)d1Cs(!+JZ-9mYq@ zs)h;%Tw6n~f?Q7B{UrIwe)L7DtZ8e>r*=X7GD98sK7~wanCDK3Un&nrI-GAsW7WNx z&Wm#>DAx=XjJPjQIrj^l@}jRB@_Cr2kQuc~I16LFrg-=6Pwg~x<&`T}py#NefmDxE zQc(D0)M-#<@$z>}x~iI5uk~v0Wc7&c`o>0|izayP48B9@QZr5&SE275Qjr>(p!TR# zD$uN)JJ(+?c?*^YLJ{=yk!ohGP{mLl7tI1&TrZ?Z3KrAT^?@xX9};Bk>3VizXTUcp zT1=0Pi-U0R!uj*Fg((nl0v6(fUw7ZTj*gC&mKI!-x~{ISt*vdo?Ff7h`n0U94E7b# zMe}lY-o1a-p!0%(mO-Pt{Xd6}{N2j*ue*@^p~VvJDtG_>bQp5hV{=|!IvD1SR77qJ zx!OK!(_5&uSfPK#jMzsCV53|CGoVRA7Zrt;JP4^7>B?L`tMwaQ#V|wSKl`wHO5Lij zuorFJ@Fi6@-yo%}uE%l+d=e!={U589!P`#;Iac}ko@_iws_NT}`{B;D> z(Z@FzWyPHM@I!+>K7JGlYyPbvo{VFDgsMFo%INQV*H?lUb)|#w7F9m{B z{q9)8SqAwXn!Yq%Oa9H@DofdK3aP|YM-PYz}-sLk5 zA(gEujW+4s0xt&dDg^QAz7+`n@a-EMf)dyGxWIm`DmVeJ%6tGp)8xNzwkC&^l*kHV zFLGVd;gLaC8fPhljT zUE+h(Q&{4^f0v;w%`qa@zF&KkV?S|5D^wjC1P2#JQUI+Z#FOgPo^n(c@;`wHs%=f1 zI$8nHm9_-}qXoLk_%QvO$33@YAIn?#?43p9b#!H8v@_G}S36Da^l&mPv}XjlYD-tk zf`J&<{fK7w~R&+4w7yn-0-2ju1R~$R;emFSc1FN*_0t<3Fpy)*NH@QeY zJqXqc^;^GJ8CLj5ZY;Nx!HMb0wQBuVN(s8?g6HPA1yr`(? zN8W@WXJV^4?FUtWw4qy=*7;a9VXsF(1HQ38&CRna^USehQ&UqG*TxU3z#Qe}IluY! zL?rTnx;pmw>C+?cRypyMfdC&kwXZEtP06g7WKUhP*WfVxHAqutkccr=~lz zAS3l)m~~$n4aN8$O`)p$Ii?-SZD~5>m6`5LqZ*AgZOAa&?PFwHs-=!b+ME(HWnyMl zp9{GyFVFw@=U3O|Nt61hDpUJ>-R1-pezhMT^f4IioXG-*sRKnnaf2;MYS9u75SI7x z(Ml!10|0;a>4H33?1jOfuQx-u)$b`OQD$)@A4YZ=yV2VEXi2yDA1La&N$r>GIL8@S z&Ix;Cw6Jwg9@re^Xo(!_-U4{vbu`@MqJryWQbM7s$x6}sXk=rYJgF-u)Bxb3HIzW$%t5T}GHWZh_h3M-~-2&kvTcp3EGu%?GNK&P*e`9%KyZ zKJHStwFNwMLj%8^1+mD(ZFw>Tjh+qPq^uIzQJeF|brFZph|LXG1+)5Db?2tZ2eK(e z2u(C6G!%J|rToDcs&O|pfF~*SIb8E)x+i}T0DYO5Bps}E42tR!BjW*O*U=ToO~=Q_ z=hzTs>}()0lBJ!N-_z}VkVPNwDr{E+3k+^j)X;mJ6KNKZZp}ZG+*YR7>m^Cfr=*8+ zipaCL$e5_ThfjTAlZGU4oO~d>e#TR1*8Br4?IyF_qNY{h+M9{ufH8CV9Aa&P6@>X# z3KzH}BO?>$c7U!J&l$^^6( ziS7JgIbg8Dg)KB}Z5KcWr2Qowryu7%e>zqlJysW~nPc9XVOR-=B}GIjToC-i;Y)FiwP z0YLs1QgW%H);(**yBKSM!*H@CX<%TW)R%E<(2u`R@6>}~GXU~}&u&r|A?&_3S6o~S zTgnMC=&)GfED=IxK_vmZi za*WbKPP09NCiU6ot&=!X(J@)ti`sBO=BpaheRy1xHh>Et^p2KvTG92%by*yR5Tm!y z#jfI2!2V;L&s=Oj+%W_<8iOS8c_qTLkJ_F5p%e-9ti1C!YgI*CAOyg@8*uV^tZMu{ zS$ZUrIMa>owiKy%r=vUPH|_fvGA5D}J%XWY)}QwuR{#wTyQ&RF{bHyh;H)H@g#CD@ zNz7S+XLm;%`B|}ijE4?EmmxGnZmuta5I0CkB+B_RfV6LSG5zqQY(O0*i)pa<)h=G0 zT+6o9&Q&6c-)n6o3_013A+3j^fD`d^WO+$R2~FY;Fmp$V+x^hWf-NX4X*YTwkS;4E zBKwIGKeF4Zg1KN^eMxFuV+2rZ&j-uieaOnN@wClM`b8unRwzxZ*2(~qB&*cyL^#Y+CS;O5n1j<|~qo`Ha(yx8& zx;V$Q(t+RO!=v-vuvC+vi)QyJ@A7WjBH$plq&-~R9e)XslckAfE){ekth`S8g_?yN zi}o^<{JuT=_ThBXy}Z0Y=wY*1_T)Vb9oe;y;XYcb(>#YyJNqvEJQTW~$-G;4oUoVR z@TwW9Hl3HMr#Ou_#VeA!u^89^;7J8C#maggAi6IqIXgRxq&UBBhO46DrTO?~Lu+a> zqiPInrwO;-9R&y~5^LT2`HI>Z2Kf|?G|jAx&xlVx#Gg3=1V+)~E_<46ar{K{N?}O|fOAkTw%E`i^$w3!|VcsVaVmx72tEs7lXdFCzm{}Hd zOz2h3`sSl7ED4RKQiM#{zs}UnbtH=70Ab3Ty@z46y1G5U>cgjB*zsKdBAhvtbc=4de2UA32hN+M>cHFc*rjDO&$wKhf05Pg;*si4@@-y8F7)%1Wq#5_VP7 zdbBni$_4F66zCr6pKo%A&AC!srXIKz!LTn(1nC}Q+G*QdH{TYbu)o?sqQtI4g(%CQ z;_=|p(h{uNGKsjgJ{H3wD`eJm5lRMiqQo=8=ATWfA}ei9``<}_`*!eVe5<*rP2VPL z0b^^Wd-qh69(5jCzM_+xN+4|Y;oStaSEf6`Z&zxdCqVoacKM+JOdw6OOf7?EU!H15 z7P-5;J9T2=$#B!#?6A`%dl+2_0xHPRw} zJrP4u-Ir17NvfB4cz7!PT|K*h&%cxtp+HS|5X4^U#UOOVn*I6u>m?j%85;{sx^@Jb z@|4VEHHfaGd*WAql#Q+E`9KNS8DaAlQ`msAZ@)uxP7JHzJKC^85Mg9ehIW*VWjFE+ zR*=)i;;q1gY}1m`Qkr1}mq5S3K1m^%QzCj6AQj!&W@>?K(gNv0_JVkQniXk?K@bfx zT9VZ*8qm_9Iu=1E3!X6T$(v8t!?B?@Uo<=eZA&}Q07~hSNv+QW4zmh88z}L<6PE$E zje(f}m&9qh?c%HT0x}MBI{OGx@`b~1a``p5Ia`*78Fwp zC>Br&F5?IR0o62(25?5^`hUs3h4<7`A78H-zYktRC-1fxh+V!s9v`M}D|Y6Bd;pJn zatpZCt~FC>m1SjbQSy(SztkCLb*4V5m|=Lh_kf^3lR??rBM?e;gZ%^%=k|L0_U%cS zJ<#=^DG>lrbdZP!4~7Vj1`Z0%)`B+ukPi+=XO4xY-w9r+;4~d98e}s1gZ#o5NvHI3 z1DC}*QIiI=UWP$AJp9sZFPVW{+Xyx$#Na1y(AoHN;&sXMfYo_-QP+KrB zvtUEiQ#4%0kqOPbP|Jgf!xU{Kch972V)}l6x|Z~1 z?DT238me^f9eH_T)#LqN-xj=RvI8X#K7#Y&G?_#o+BtdaD9eCUD6COt zy9to0+M|D3&6mOLfX8vf>GghN16w%WctO!{MJ0gAVEFeZR^)sZaxIi{kmFt`k2s3D zDT!eR|F}-DOyb}oIeouq9x)&4rv_fwx=wH~9 z;4fOYBz{U5DP=_H!bWf-;logpdyXGlB4b3u9u*#*qE4pg6H|`&Au3@ zFvIH5D6_J8C{xQD@E+{)jwjc^3#znx5UUg^0=w?sC1}uqO84zsw}yfAe9y~bxZ(RZ zpct+mrV|Oje~^VQ3e5I+W878EbSZG9ge}^f`PgG_Z&4`#)gj?m+b~ElV%sl^Aim#USvZuU`ktURHL?H*O6f2#<4F z`@s2Yzh%#eeg5)g__%CYxg-Od0{6-r8;x{rfV52YYOap4Gf*~5zk%3wTjqwOwDlDZ zgGvVmOByf7MNZBW&hc=g8;)?JO}l#6X=n3_;ReD*WZn}TE>dic^3B(EIY~RqdQrnF z0l%SrZ_>Zch}Fq{@!EF^@vLckhT?M9vc9|7qY7L|wdNjmS(GA94bo zIcu7s@^pVR({2V;OjZjc$Ki2C*|(J&%gfgzJ!(hc1+?%`1}b^{00`A` zKZT2ntJ~6PJ)9;Kv$u-&`Pk#!h}d@MpGIk~aEX!nIvco@(Wf?R?FbYqv`A~FgJ(LD z)@aBzY}>1D(WY16UT&P_vC%oU$ilNA1PC*`fk9C)-Oq#o<5o~;TGhmy(&lVj=j_ln zC>k0~czzQaq~5;Q##vK-4)7CutnOQ;GMyI^DidVYh`n{kaPMDX#gW?r+G1qd#_*Ph zr$PkAqXRR|gj7dSCJmhe>~?u_a3F!BL^J?vwiesj+f#Kpz1AkDdZ8dDo_^Wu0T&USv=_Ubci&uv;*2r%Z!M3hXKVVT+!=TM`LJ_S&6YT@{ zVN!NJ=atZ=Kf=Ux&g(%+YvA&f8%}!n?lm{IvT}emanW-CW2mLB?x>N}j3Nu~muqI# zuSkH*qJD*L^S;!Hjz<1{dv`yZfNLerFf?@5%F+YTtk)m2obWKHRJ;oaD9Zox@gvjg z?H_zM$aM0amydpt?WHRd6_NR28&_h1mJjnFst@)ey99oV_{FC@_7e`qx z)(Ogo-@pGTT3(TqeS7Ed>1Tb}7ZrBw*umYph1?l(FC?DUhd#%bbW%5XY&fa8m)#w> zmm}_TrnGX`*y3`n;5ZUKuannpmnvi2IlZo(^4Cad&DzZ6Jsl}zeOxQQz(f`DnaZYY zbB_rVYFr07IF-VStjLmRE!of+26?bRt$U z{#Z=|V3>2XS7Cstwb>QWOfl&e>qo3l`ityOimw1hL0SU|i? zO--o?nkl(q&Fssb1yHwa&zVU;tU$u>=(lew45#v(e@Qi@DjchDG9}hq0=wrIt`Dcx zVsEu!hO$iRK>)y`1(4$@>B)*cswVd%;W&vezke@e^62n{_61568VyCFwfGr4JiCV9 z0oV{)CL=z7?!oJ{V?}KK02=bGqgD53UbP>x=-MgJ}dxy`-$N=^S?%dCI z3Bq$%oX_b)dx1mk@^^cQXrNM~r2zk1qy9hm>1y$7dAjMKj~zU}nw0+Fb(2{bhaR~{ zbVgUyGcC)Lp^>G%0P}RrTAJuV+GV3oo2;G-LgIWG&n6ZCqqv%Hf=*V=Ul1j)HaJ`5 ze*EssKK+oc+~_)LU@)yam(hcoZ?2v)OBs!H+dfT5oS9p+$yVc z)`qxVb#oWmZS;yt4Pb?l1@7y3r`+CURY(D@gb10v*aG~m_|p2A6mJ9LSA3xus(uEoV`W&J=`xbyL(2%=&D{0E5HdQ-E;*!+ZymshuK z)>-l0K2QSXxb>2Nx}yl?L7o}~g{pP!wmJZqc`?1(v4fTO=FL}`JY1Zd%+G1dQH;Of zTl@su?F}@8)uxS!t+MpKrJpcHz0(yZwvcbzx&OMaOJQpAMFnyGN9PWx&@Q_%A1wFe zkGTU7B-BJV4%!0oxU0v_7@w)6TaGpQ7@CsIa~ftkxpOD*kX+BmVC9{k-w(x!_I6QZ zjcW2m2~N(qOClo5X&Qj^{7+BvfFrXIqPXXC5AM_qT#yF{R&1;JV*bxmZuQhB*G?8u zW^df70WD_n2%|9<0>47H8gRzC$5`l;8)D&4Xj=D@gdSkh7~N6NiK4{%Xz8pc?`u>Q z0MB66lU@F`y*){93h)P{PMppB>-@i0jr4ng}$ees=Y|vCuuk|SpE5(^K5v6t0O6tJ>%{V|W=w`uT<+w{OUJBfh(zYwxK;dojnM>_XRb5awI$N1YfD09-v*QJ^;~zR2=J zEA8MIUY{<+XyCaR8)`5K0!J1fsnZV&mPn{gfb?_?aVhiY75?n^+tv<8KDcSDx!k*L!sfAXHUOxcYH{N@^#1BAjM7BSLBbd&c2q+LxNNZLR^cdcrUM z8YnKdn@O;;lT^!#Xe2&?hJHg%Rv*~tY4X713ms>fei~Pygi^>1A@O@=v{UURR zEl&^V2-CbrlcD!`-Lp@fR{_fIu{qJAfcbn?Q-p9()t0&m>G_!mVF7LP_H+k|tAV@w zYwX`wMC~et1rv9o##WM7XF9%l>^T1YqL9!n`WH|H`w$9k$5YwbHodnQ;Qo0m!`F%~ z^T~KFS(;AH=6V=X+8-wK(@4#jo7t9nnt7|w;mHQHAiwyPeBJy^2!bj@(`=Sq zvPHRe)1+yg%QxlXOz&#l@9RTZLVd)7*1Q1%F13z!T ztbEP;H0|gnk`=9Nf{_V~?`}rAzW7oz5+R9t8p0i-fq5Lt9uN4*X%^?nP7c&rNf|h+ z3|k}ZF`CNhe>zQ@>SNNzh@gHp8o1JI?B(l=LE^X(_x8YzXOD=Vc`D|(SvG%+OQb0O zhgooi>{~C$x_VeSy1EwVWB6gq@`GlFsDWSm8rEdhTJ|k8GVy3*P{+gu`k3 zsr8@bgK8&>-fk^6(uw=x%dSJuEU)V<^kUn-Pg4$^(U8gI0t*NSmhXr ziP~^N7?0j3qOcJ^y5g?>o@kNd^ z+E5^Q@9(dJ^&C?VQ*c^WD zHBjA}LMmFlsj3=elIfmrtC1o+Ppo>q3NbqL;f+OU2|^@Hl!6n>tb4NKyLaiy%Dz5W zB=XmTb6kViOF2S<0^y%4XG1i?2I>$FL%B##id98z=bM<59XX0)I(%~zV@TZscbBH>%Esz&0+jM@TGQ6ck=uXmQs_aGsW2bsK)=Lsepw=6BEr|ej!5p(!oUr7Mn zJyNRwJ7Mu%7J{GF^}xs~A*E0wEqu{E0sUPF8PmDs5gYsIKQf4Fni844DGt&6 zQ2DVW+EVtYrt_s7fEWFJ|G&5XKVSHXlbgG`Tlm~Q(A%^Y7|_d?m7YKUiM7c4_2e4x z{J=YOTfb-sJP>Fag7Ck}GyXePx>x_nN2B#5lNe0**D4767n8nDL!R|4sfb9Lzj^qS zC>WY$-mp(bF2%=G)Pa$1uv~GE?)HS%np#^-fWVTQ8wkT{SMsmFsLocAp==KLx-y)a zdkhU8Hu<$}*gon&Wz65}`MAKpn8{&&Fv#fYDN%E0NO!GkJ`j=V$)5^VK@fjhJG|Lg z{v0Zf8Yp>PZX;yXwDxD9$Gbpw2VD)R8adT*c($tc*O?hFYTD%85J5`X zK>RZ}Y$lwnIYq>(+&n>pFdWy+KYYjrPpL;%2B<^|C=({--VBi0xD|GkxEV9HxFO4`r-i>N*4LjYuS5LK5rk@9~ z$07b0l@_#04s9=04+jKD+7d?MHNWDf5GVyLpXI@DMky2QP{r0*`KT30g*y zHXkEw0u~kS-#;o_8+AQLr=3ssZH50S{X%<}p;2B4*!!<6gea&bYo9yRn$R5$(5Y@+R8{$;yM;;WH;gF}hYxVy?^U4g9OU?9jkV^{x(HWo1~v&e^++qY3bJzrW*(#7sI~b7P?+hux!?bxjiYwSftU!W z=;IurK`=wsdo%g&t9$XM)|>n=_ai2U46R!;%fFS%%TT8MT_&cIqF@U;TH;5C6a3t@mfBF>X7# zdGaGG6;|*K*wg*1Zv9eBVM?B91gTTpzy?ZmI)%AaQ z@6y5>KA9q|8j*0F#}dX z7sozxYc(|wrfq;4dwHH~tav*K+E`$85P=|UVz?v|>Z24Cv-qM!<>{|)jyzwMT*$ACTA)(B3ug{oCT^E=d(O>;p=`_w*o#`jSmJCKe3_2lv)Vs3X%K0z;% z?9GJqz)RZUT8*dR^jmNNq=O-81&LL2^9ZhMw<^ny2mRwog3)nthCnw9rG?f+HG-KR znT1zt(h+y?Yi3W1Zqi`v9y`H1_bE$@4RD_2uAQ6G!RQV4$<_o2E95T3tyn!*mx zaXH&orsN4E59sGe-3d<6@kw;Yp@5OE=tPa$o0#8QT?>OTsCp~ZSnvL!if7r*lO!TW zCH(DMJh@0K!`wdxLxK+66@zVE@sOi)W~E#EY1!}Vz#7)PN?z?-R)tg2_`)_JvXt{- zAd;8ISTz+EfYkss+z33h(Ii4#V~@na!>6w=j5o3eux-pTq-?uTd$1ntK5c2*APtbu z8kHAzGHq&^7~_vZ88*7dLqR%^;7}v0gMcwS=m5o}*^{v07gt z^gOO2I>b%dY9S{W-tFYEVD>3-d(>W_lBW84@{DI9S%c*RHsRFQs24ifvV+jI5edha zLJa3P(tPn{vEEFSeNvwq zJBswWg|CL~=U|)clyd~_p-X`L3N*GK;b%CzS$4xFgr+@?^I| zyKQ<`8U78g{@+m@LIB$8OWz^>_QQvzzx9&Y>o;%kdZmZ=zWKGCl%!i@q%mN?2>+cM z{&zmXWzAP`b}wP;SN+X5F=mYgy^}dMqfScq29F)pXOm6`mI5W6J_M``bhBphK#Bz% z#nE(WHVG#xSO#Dl9qL^rOUop}7s27?D-NRV>{zAXNbeu`jjl1Idutqd7gRUY@Y*?> z=?Ya|?}7tk#Kaw{Z((aF5Lv1WG9&)U#VAw-ege7&_9JIb<`&^8pM2Sutbtp< zqp2CJdX6WaEERktUJ)8S00OP@#I(6UJ+dv$i-|_ekU3bvtHVA+p$8%cz-GQVQ~&ht z{rgf5li4?I#>pLDp+`7}v{_uReEQ(`?X2K6AZiL47=Ym4lOlXR*8avz)QSb{NVJqo z^;c1#8L)wBp?0nr!Mv`(9m7l$F+N-RLu2(wCRt13ymgrJS%{>%(k#NDxyq`qAnk^^ zTKZ(qI(TS^xeTf}pUX;&o$Om|AJG54;yc|&JNUaBig#8W*_Vil+m4L)8J~;ri~Bi# zD0{{$CIra7g)kWszD85I*nKkS#js`fAT*k~F#dv>`w^{}a)xE41z}gwVy%Y*raExc z=9@;Kqf$!T^hOcO_NMTF0PUkUk5{60zU>{wMPns^9fJrhVEK%i^*v@}TfDs&_1jHt zWp!n&|w6EG@-&Re$%1a!pR9 z<@xJa^UgtVRbxC?$_vG_4t#$GxBvRPW?kqdx))cKS@^U$l+kgP`^YH?sIR0)OETXu z(%RgZvT0AheUM}CVNcGu468`8CXZeel6 zP`U3c*aJAv1gh1K_}Xb6;PWiVA_3#?GqZ4LQv^uqx?14*gl6%zaE@I2?o>?^vFLh8 zvH73fs*beK=q(%`<+Y*Rq+5^H;!<1(hV1ZUdySL}gyY~NePEZApgFU^WPxzpTmvSO zNMWnenx&cl20OeNM>6Y3;?%|gfgF^jfD83?k`#UCTLufLp+4@T2c1yKG( zpImd6Q4c;Gt#SG!6YzaOsv)V_q%FOBZca`bQVpXhXWOh`2I3?m@Z-qAP0Y3NW zKj~O#6Jck!3!G&fRudMNaGA6ZIprXM_)}^y?_yPvU7JM}ck=9eeLGIR=uwewXAg$v zE1WxxJ#rH04{}M)7-Wn*UJRX}3uXHvoX6;gqy&{1tvqM5JGkx{Hc4v8sn!%}Py@5u zbN>R|wWk2o3;PQY)!oE{Om21~@NgLRl`f=(haBqR-ise?jhw4dl{Y;V`3e^{g|p3)le z<@50~$1i$8=j}UaO53~51fF6TKmX~x%gB_fJGQpf zUJ7bZT$ISR;qnr5q82uq*5>W$hPvc4{Wb6(j2ptSj;#yRe~+hUuMaS23tK)QxaK6U zYb%LxYrIVbmYVRaF|nw6=+T1poor9QE|Eq)nztv1y9@x7=>SLaqjX(h(GaoNHVXRb zJ4+#G0SU^mtSZ?|=RD0FVm^|22~&p#B3q$1H;H47`WNI2zo=|~f9~U75yGoupWyPB zC8H~|r7#>wAImMAZM?jKX$Q!e`wYAqtE^kg%0Wy*0i|4|mADkO0p5%fU>I zqwK^&_3m0;0Etq^DQ8)OGhV#F$xxT@j54|z_AmYsv0#c;Hk^oO|4e45X=i`AX8nOc zK)!0DDZ=Ag;=A|ymR?`$9BXcBf*NKLBy1`3V@QYqp96M&j~$Z#%Z30PJOUw%2PKDQ ze@|p}4!bb@f4q*uwtwxPJT4BQrFw2ZVt6&Mil9PIcFBV12_VFJMBn)R#RZPCg^-ls z;9%H^H;;d*qP6tjnhNFJ+{|P2wFTW0~YLJKyjC>3S{l&#q0%V$N5~va*U(hkPQNUo*ujO=(zZV~@&C z_hpIN;drI-;O|Ve;~E569~J;oMp;P-^nUSH2aY5rKDmMFkC_nEvB8Y;;kkCxU;6k5 z_j2j{>vP+?T zj4XS}v*EG?V_X_MR;*_AoR1l%lI4CVhr9RYXgb+H1vg&>aErlrKOb^U zrHMD)1zASkL+@jI92fn$q8H+hDNDrvGy;TZ-Bu}SAL6I-t&)LTvBPApGYH4pFYQF< z#Q@ywyGnxu%CJ0M%DpZkM@4FV$)=v%`gQMi>FQe0l9Xc8VH7@m?@|d7jc{oyV?=DM zp~-B{t^-+I(at4IyRo0tlw~G92di*P^*MxzPo0XK#c%!6=R5=q5GSQ931$mVB?(7O~R%XDhQXxp_;t@wP5`Jav|6%RD zgQD8Lbx{mkFc1U<6akSeAP6WKw81<4THZxdb;<{@|2YqZyW|85LDScw0eZ96OjA~l@z$b+P);Bntr|`k)V>08; z_#^IPt{RQq)N=b@Q&K7$bq0VqSOo9i0zSOJD1iSq1k9a>ieOi{V^- zRSdicg1%@U54}MmVTrJs^20y)twQsZ>ic7I#>f+%3mCUE?>X(ay!66*{dNkK9_s6p z#eSaAt=a!^d|b+r;?i)Brt*chb2ZyU^5t^;i=wWsq$KoAsL!*f+X2wB%DS~xz%Pt7 zVXHVKn1CI}N$s|3q;+&MW{>vDn$nTVe2+8T22)iN^f#O%kd`;_)4dYv-s`S*1S66szWI95!; zK9j(G5wU9f2;mEjgeO~qL^qNQA`8D8F@=q+yTeKbSE?4*=I3ouO-@&KMs;shSN~a) z6Z$!0;!$d&S2n)ZUnlEZwc9tm{B>a@DXFUU*SS#XtsM;wxrNREmI03p<^W0P28xJF z*l!PGF+rTVb3U4Se>E#(e~kf0gW^e7zJT_2-OTh_u5_rC*F0lcyk&y%{*fHOy7?RJ z9?B}_m~^CiJ$ixU(sMe14&_+B6aqS@JEEN-Cp%WG)x&Gic_>ulV3Qa!D$C%j@cpkBt}pxov2VtL~- zX7YmT_lb&MgXIP2k%7XXtKB6OY>;bQ!f!`9*y63^Q0*#gys!xg*Hi)j-u-nA>{a{g z$`qPsO3hn7cBaaVYovF*I)6VjHqIcC)F`JxSk3Yav(e7jpWc(}y{F4ytY7kM6^dPG z-_i;PP5u?@flC`wB1AzeINli^t+y)`Jxb_3rB<%h+W>;<$(Q=i3syFfS{XVk6lx?uIj+NN)L_!%jL+<}Euq<)onoGmxk0T)kzQwB-#G{Tl^Q%APOw^zB7 z_I@XbkxNI8QF~V7s>z8ZzuTQ^h-!+8yRV0RohKAYKC5!=G6z#>s&rD5l%`Y2!^qfm+Pkyw-;0!6ogF`azOOt_3JNIr^7bY}q<*oy zY{JqW{dPdTzqC?MHOgb~Y^Sp^(RgRERgCCSZ~uZuOH;FYrCC|(PtD>Q-#cM$e&mj| zQJFZuWYdk=^PphQ@+JAqlr!bkI10(ne>+LqbFas`W*qD&Gdb=?0YWo#=q9XyGo7m{+E5LX|ZwdwPtG#0mEneh037M#$GYs7h8r| zFmjI(v5SjgHW(TjZYTZ_Y-#xkZ%P8wDkm!y&J!c-@pNJphNA1U-kPGAB)!_B&3(bw zzL7r%882B!l`-cjQ2Xr_cwZs(P*2X4n#B%zDwf5M*wtk(yl*Ka%j?9Sq!%Qnm6!X| zti56dp_qI@zUYf{5My8&_h~e8sv(R8&2{Hh?fuM)z4-U8rqId3lSMeL;@>P5vQrqA zzW=Dy+;g=5YS~%Z|LKc}FcpqKW(0YWk)6^$jH?{D`uDn4$p4D!->VKzj=upfvj5jb z`M|$A_f(MKmF-a9efIawrw+1isi{>HM)3^~3zOwGP3u2dBfn>4V?l3pLd~A~N1>ZE zN85Jq5-XsT3<}~%Y47h`-fB1foqX%BsScuvxs8-v`GwWF|B>It`yHnX>?*o?w1e|6 z#!w0gSr+lf69b5=O|SkY>60miP!fh27FH?#pTAFgPa5;9ATd-j)2SPELcfJ({}vA@ zMJ7Xs>Z0N^C4WsrZPPbKx}z_yDpfPTleP$b=i2;=QsD1f3Z>XTBbSQ|RlfZ1TMIos zlOxwf(RS{?MnUzZUg&p6oe+QJM%S$X7SGv4uVuZ(8^{we_WK6%Bsii_dO!j72IoO7 zv(fREJMnA3vDZ@7_Lz12lLn}6G92rg$zbUqqT{yyKB?rtlkBa;zx8$g=4&D zlLJD4i9SSKO3=QmhOb<2tEZB(AFK3eWOg5cnx5f+2$7gyPyYwt#oSo7-!;w3CR-*Ipy{`LqFe1u#KHDaH?{Jx>sY5=??)Ho;F=faQzv{ z>ptd@k+*Oln4O$F(ZJ+{Xk`gpWS8TXHx#8oJZry(bb!Bh?b;F7z&Jv;@(W81mGvzCX;S#me^UJK({9HIZ&>5iw&n5;eYpvvV!W5+vp45ZHrls( zPcw3Ueje(LealAOn8Mbe*4?RcwIL@DtESU?h_@hV&wDaaEuFZYM5fc};~Y zhe)qA!EQp62;;s>rTr(EpuBH}GLEZF0vDe|+Q%_#P)kgO!D^SLW@t}*LoRG~BfH|@ zYf|I8ZBV)htC1lG%Ls++dQd>A#83bNI_lD`4j;Q?edF3Z1^AId^h~#``Bdc#^L17## z@w4VxH=o}x$dP|cB5q_bZR#wPc%zyi>AhWg{TCFj;eNlOu7<7jzS)9X&IFMc?@8#Z zAZ!WZscqGscu>zud5+k*~fT(9lHDVL<1wPGCCBgv>e{T z`)pU#MHl-M712D_H~zRAZ9{z}OIXUWK&U5LEGjB$fr%e=A7A>~28Bq8Pa4iKk9&+i zuBRL)5x1cvY#y{(fZTp^!76Oq9gsrrCbj@W_fc07t6w`yPQCIh^Ib>R2<9Dr+K-6)&O`u6C{mlXjjR#T5C@|` z!;;aU^2%poK`T@cZq26GEjh92394|=#;>EOFv*U0dTfx%1yLZ@ofI4~3t*<_fAAPI zgfZM04n265kF}abn_2EpVf%~Rw&x` zud2C$Ot3W6@)kKrMYo_q5)iHQPO$pf>JJ9eoQCh^O6;S~6u!K#f8w3n%6I;}ld7Ke z-d}C>B@BX%nA(MJ>xCeb8S+bcLP@P!$IXLvC$##K4gArV>_cfUrb8i?4(f`DBhKxdev6o4N9e)^-6xzFzzWf~vxsxR@Vflmmd?+) zHES7Dji*B?tk#IAHy4c*R>^O34~Z7Mkny59DmM0Pk_{B~w|^LDP+J4$p>X6En28lA z<>y~4n4=yS3d&2tE*7=GQ>0KTje_1{(~zMk{l0l0<~m-|^)ed={W}@>Wyqu{a@zWh z&Qc6UEy99PX<|+aU+MhoYU90o5Z)Vu`6p+2W;%}EmYMlz4xKbM;b}h}7AAmkh&FQb z*zDDX@^6tRe-uzn)=6k9#)j*|_Ull`>vo0!9Vz5iztbUR5Gs-B1@v9OO6||<*2K!9 zJ$20eiUqOCA&mjrt%{K?4^~WbcBaF+ay*A;qsEUofD|kWnhB zh3Dm1!7L40MjO#XZ3!&t5auA?_rR{^V2f{qg#>a`48_4PmMysLc;+ol{9C^;Xaq3g zflEMeg?sgS!ZuWc!l!cUzeL0qWrV?Vtb5hfrG-+FOn}S>2dKpTI>+I60gMFnkUIVn zj_gL>7aBtNg+Ym~*M9_a+ylpqYT{1vE&@?uShf@EvDy7>shPUx=5I7NL%XN9 zrX(|P46jfnK>#J)HHXyFU+}hTi&P#`HFxNex70)q^ZW6zUxgyOzK>EG?Zp{cTVg!$`MzgKC&T{Rf8AVu!v51 ze@D`0*r5zg`(Ss6|1+3)htojUq3>N?UC`O4Zn6k`W`-MaR zPr}%a6SGsr5{PNV60$x-0v^hpNfEcMlut()NXoo+9vd0CO~ETDh#z=hY(#RG<`)(%9OJf&?F|S=mAHHLDe!@2>5mqtgph?gbm%1ZKDzk1PNf{pFcE zng}a@Bqeq7DjS^k2X-lAGBJXN_N~GRYJeJR`k3^?vl1xt_K*-3fmduf@VwRFaPxg3cie z0TtQf?R=vpxu(vn?!7Y|gvJ9Xpicwsb5M>;9De-VZd-@$-H#B7nU5fX!W1t-5t^IU zqK3}tr9%d3LF z#os9lnwnIYetQ{h?8zJ5f^_8BKG=e6gE6oC>S`YVm{rGf9(#>KP*aomd;x4ed$@}I zJ(O@vB!^AvOya+DDtgV>ZP1cg;P^iW(q4isNkV3IA9Rh+9sQP(x)=FaGOYsRYYCusXCHErI?zp#D75jy$*Ia9}Dt@c*5K%86==%El_rE+ieZt@B z*T=#M*{pkk`}lj3tIy)E_O%xHJreZt=BzzzFI*v$b`5&qz z;N#3gWwzdH$m)H4JON#UP8wkI!J7oIgJ_<{$yH{z0W;HA;b#>+R=$zbz5ULc5nnX% zR#9`&>S^ZcWMgz>WaZv`4zou)Uw&l~2rK=VHxPhdva_+9eFpA>mJI6MqUysSzfK^v z06vVl$Uwf|C|m+Xp|2dF|HyeZ6vbT-GLMIeGhdr(#^%+4%3$X7hUWo#7|3kbA!+m^ z6}QJrX!mbIMktHV&V;`}WEY$%a`kftAJf5CY?t~jsEjIj(|iC0=iq06!~l;>uo71S z;CnctGYDyiH2`vd!K)P+Hs{%<5bdCpa5$xths^?-$iU7)&^EM#a0lcW>}YkOEH~!% z=TvTw>v00!yQ8f})1bP1_PUd_)SxfTF0{5m@Q<$HlOHoC zv9M9{ye~47D#>12vPpucPB7M*PQiWvJu&?`&e8VKqmN=viP_mm2ZAHi(vJb3x|Isw zVEaQN)Q}9Ijgjd2Q4mhVQ-EW~#sLd6OZ`fa+6KJhrA&*MT!9A#ru9+z%X3%+?T&y! z#pbogh*(1#87BHMG5_sl1k)h88QO2-S7|-zrr<8y(6+=b7BlKLTm`cXtqAt6^&1v< zIv&Vr`nIu91%MLV0EeX+vjksheG;$CiWa4K2_^eAgf+3fC$6OiV$&M2qN)*TIr-Jt z0{E4I-_0P))#IwZj0-LnJ;E*fpeAzh9(4R1IR*K_>c*+(%y<^#N?% z-mW2`;Vg#5ufP8}NxxAfr}fPyBVop43@{A=rW86rEC7nf!bT&k=k7!M0-u#vtkQ(V zVgQ?KU{M8BwLsZ2fy#8`IfJxOS_bdus(FK<&+ar6mE&>;iEb0O2>yyK)R@8W*Lt)y z{Oc!R5taY|T88>NZA{KvwdeiR)!793*+U@qkF&Z$4Q>F@Oh|kE=;{XuLOqPlV}0`d zH1rp6F>;4SJc+m}n48G1kn&F0jL;CAFb>n-T^@QyQDU*69$I9q-sM4bCrDX7wABKy zncWZ8O5svC0uwsqsU7U>78s8kX=}3w{*edAd=&2!O|!PvA_$#0ut|f5&2Q8Ag$&*i zTX@nK^|DnUL(a@e?aU9bvvb&mJU;2AYFlK(DqMO34(468aR4pgn+y*liuFO&n2AzMR7-~jOkTuY-1fjNBnyTN z^OSk~wUcV)|3Efi$iCB1_2A!- zKyA|tKeJkgtrE4rpVjL%zHF7>$E*1N&0Z#Bl-Mh#l9yv;Sco8J(*Ku5QW#vJ@_~ol z`!80B=e}>+>3ES({~@g|JP0tW{Ws+CJp9Ol&?(TL$jybG2K{E*et5I)?~~ZcX1oYf=CCKtY-XX6cfjP=h$mUqT~b<_R*c*`-*K@XPaD7O#mttIzfe>Z#uScqs2X|Lyg^R2Ui7?|;MB{eKy}|NDRGD7O&y04n9I)Coqz zsxOw6K(t}3Aad7iu?F-(&F_~d5x)-1ryqx@#{>EUnc-&%nbnE1*Ou&p1=~K zzs%IO3w)0EF1RA#H~@~eQ+C}p(RJcr^n8VW7m8((jg5;#2sFgIM+gd$cIinEV?pW? z2d{cLOoHQF@Hb);*Nzu4*X%+6k)15{yPfq9+67L5dG}f>)irYM1t^i^F%wg#R4)W@ zPg*s?Q@V27YwHm}bAShkW2;I5!AC?p0yoANm|%#4yaRjt{l(&TtYHq+_+|KV;Vxx4WBi6tUEhA)4pPsc5s8M83Krh2?f<}V+lQuE$ zp*g1GN*q0#k834rU;}}3LICbB2@E253aq3ZWx>!Tocfef4T0%gZK(ar8k}F)g#LfM z&c~6KgQGV;0uY6D(0m5v^$fruzpzxC>x96+Bs1kVlHW3bpguW`Z{U=q*WBj>VS%PY2jyqa26 zvANk>f1J(d<7en(haIC{FzshZ`nOhfeU$?;5h9=9I7>pYbMu|A2 z4J6QpaNq)X5QsXKq#B$}2W%qrK@Gtg0KFl7gb1WkK_~BZZ}wcHimkkgvG~k)_bzY3w|n5EkDi|W z-{f*Ue0;+NTR`Wo!?w{@6a4!7C4zY@EiDl=2;y;l1a;kmoAI^gj^XUQyRST&(8T5U zTf0KL9+OAdLd4b}axBEmG)Ms2kiiI;9K_TipY4`l6~cgcO!zZMj&}7XB3vgpXIH8= zwCaUC&nem1%z+js0snTz0JDgU&9)5eC$ykKx>%W%JUY#00oXz3%rtNs5D5`e4nAKB zru?kix-=f_kHmDOc3$NgG|J^hdLUJuGC#`1brucuR5f_F5VJ*4;Z1Vpo;g`i4=)3W zCV2~1!ZFZO&((gnH{a(P04`J$Y)j*Zu zBc}+91Wg`nhp)|mu%3QF&XSpv!|umH`xd-!>`E6PA}OE! z9dHMpLjNq#MkAS9!1Q^8D{?0G?TIpX1k^t|Irk@_qZLnly|)#vzLGWSKVjI9-s$1W zoE1=OKmH2m!C?6Kn?Hl`4v_1BQ>d7#va{!5(gg$m0JaCpGs8e~^t~1qdP0IXs1cDy zS<|%G31%{}*#fvv=3phM(d4|mD{-yRe9RSAg#?U?!!Z5P2{4-nfaaj#fD__@lB%9* zMsj48mVVIkgasf@am&r28QI$lvS-27*U{BQus6(7aN}g$atA>lMo5DG) z5XKt;n?gYm5fKPF0tJPiPltW=h9)I@{&U%@xa#nuHwR#sNAv5UDD*J)gCOn42;rQ+1c^2Dd-?OmM0Kl(?PI;ki`J)>jA*2si+bNWJn||=}wV6K&Ia_ zBf)J5mI~y(#>lz#np4&0{p4+mbQkEOhro3XRe&ef6S}V5LwI9dyx$$)ZK>XXPk`ho1lfRJB?LB zBeST;NWc1twY9a76bn6BaBUI42EoD*v>Np$6<%X>Jc|ahdkp3Y1~12!CRcjG&G7= z-#1GGf7jaDdXomGb0Vkk!7ovD)Ek@Fph5&X7F6Y6zU2`geZmHOE-_Wzd)84!Ij#Z+B^5a!Gm$9aY?YD9V)h6^Xkg-CZe3v;&$;X& zEcxqv=v9X@3aB^JJd^+R542Z4Dd;c|_%$o|?_@?SU=dchnLCMJwF?>)Bm4_UAr4$( zcvjHWVGLzG#mi4%alpA6?eWRDj^?(HQkN@gCJ|-u^Pv)KN0Wx)&bPU$6SuYFKBXwf zx9(^!dyK(2pfl?1A&Ei`RY3J#kR>ABj<{W39zD<;5@~{`t?fKu7Jvp_i-Zf5q|OX= zBTOVXYy1xiegkKw2izrCXFxK@7ny;+1XtIk(bV@Q%`)H!II-aP%ey;hrm$PXU6cWV z%VSo(!Z#J29s=2fyuu0QhA|Kl!z*IY=T`F!;!7bBK8%C4a17#9F(=%gbpR`w)!`e9D49%|&>7DxTGTzA1aR9S zDkFCnhmb-n06f5TD2ryj3~?DpN5?DPil^TyAm|J6Y@h*d5x>*db};xuo&A?sLV=B0 z2fC_L)B!x~ph85hLH%DowR%PoV*VOvh&4nsTX=ZvZdY$ty}*_TbgpF0vN+rqpu4xh z1<*sITNIqunqxPGfVx8BTmJcxNPrIE6l3l3Aa@49k`xRK(5=3>MQSuS;n8a8k^+@E zVqKsm%8*vg2Rx@J+t=H`Lkh`&h3*ht0_9!>&Ygnh2<$(~`o|Zz&j;t}70&=G$Ry+G zqKro4y~e%3{YNYu_@rH`+^#x4y1t@!N9B>&-W`GH*_Wgr|lN*5#0P)Gn z%8~%SUMv@_8UaE0O@uuHfXwGY@fLQG!(6LWvNoF7gaov*vv+!ydse8 zBFWF<3~7#{RM6-H#2=i{y67KMd7{eP>{dYFF;o-4+w~#4{DX1Qfl1+!7s=1($FhG5 z?)DvVt?IpOh3|qM&`7J(n>3Zs2OIhfTzSYLlzF$)Gf!hw4r5c21+w2naz@<+q6A%iE2hJfYBQcPzz~IO|F|t|;?O*?KgI-H` zAN-rmVDujki|oI-7z*-Zdt@Z_?1_Dolb(nu_scsF9HDA`3ryte83i&jgIB$Cu0N=f zrE;_=e^8lCJs6G@bfXy66RE6!B|Ga-*Fo%I9cCcf+?Ai}@FooIZYQ0Kb^fLVaoQ|+ zO{&&%N7!)SKKnju`-e=Z$Y^gUVcn#rBFSI6cq3S^EM9!3UHO!X3X{ZU%xQ?sqw_Zx!v85(?wO^&PW1bz!96<@*8cA4tE~ zzJ+}Lv{oWlChPrHm(bf0PbG=4jX*R^Py=t1EHMi3cMJ3jL@TLU=RUO~arN8K)fu?z zBgVq%8vsg1fI9fBk>mr$SsF0h5uX#YGWRy37Mk;oJ=St`>Ai6y>L#9MKcv$8S?`YM7;_a`EfQ^hbjr@7w_5un39- zNw|P%c%2Lqwbf_fbnuPZ=P{_K_Bb&-#ORnEIh8r?ZG8Ia(dREXF6>*A^dUfPPrFT5 zAcuc^cX|YmzwTCX=(v5Vq2>MQG;M3&ZPEcj@QBt%6|r}uF_F|bXjF)rB;~`y08&1WHIicbvL*vPnqhbDDxlk#F@N< zI^(UeyNND*+u%%QW&^QQK{c&PExl3;d@w*IdmfDw^?sA$$ZJ3}AK}s$K_wxM=pph8 zkAV_7D|Lm@G;{XbRtJtm#k;pUyg1(eRM;_*hE_B91x2>AHU8;HhI-3*_JPaFovTGM zyYsg4??XnM4dq&zV_DcZzO)=UeGhj>%6o_*M+=XB`z?@Nq5Zf#@fGfp6wB?S;WQ6@ z_L6k0Z)0SAONrU3_mf15C+1VDu$;_ZR>tM|TPf)K2{NfZ#N-)nLHg@%zHiEBlQwlP zVCLgvN*VBR=%sHVOHXZruRn1k*5k+*QOVqhD;OfR%tLXQ4YH8sWF~_|kR%C6`Itun z%s{#?%+E`1dH*+2Wb^F+e*Kc!K)yV2n2amnlmTCyYwiVp3v_=+8WksF)gB=3v()eB zN08V7Tmx;Usd_ zeZ1frB%8Q`?}vnCz+YPcsvimRPiXSp|MK{PHAEL?2a16BVz^G#rsxz5F4wD!YR~DF zIj_LFJ!+SR!Ct3Px%C-*sQ$_*UUokx7)vHou@PLgEM?yF62JV6K#kYmuy5SzvZLOc zsB58MlNuq`!ylLJ_bWZ7w-a|pzcYozxJ(wxd0P3PG5+>l_{^Lu8VU5`T=&bJH-j>w zE0sQJBzk!*eRFPnlXCajPexM9q)D69!ETJ)vpqs|HO4~Svp#{eYkXiV zHL+}qJ1=O_b1GBBNvE;ZwtDh{^&nxCHNtByNRbEvJCeRcV)33I?K zoqI{NnZhE?lM5Muq!6#w5y^c*Xz-+iMQekv%y{?M=69bi7kUH4HTe1 zb;~1hYb44BUcYsMYab*NumYc<_>SK70z^U;N`y`o0tP4jk#Ir{jX$VV|gF6-qIV(uQND^tEn~ehk>nj0rk(Rr_ zTVsK|K)^P9f@7i|7#q7o!7CuJG@y0=euG52h42xNQiH4aHu^5&^pdXSkCb~VHAXFr zqbDVkZR; z)~9L=i?(h2!U!SNQ%T$;f0{0Pu2Aod{V#T7o&!#fGK*rRF*8ZB?w0r?!zHyZB~q=c zOSigxjMn=uJCU2@-f2%<8#hTep!Y$qcbl+QZSM^GRy49XH;xcOrPc!0(vC`R`oCW_A#N0hqVEkq%BILiLR}ZS5Qtx2TO-Z2 zJcI2FWJVlKM9VZJD6sXra;ka)^|cLq9#CE^O%W_|8RUCGd;|RSo4Nr_Mf(sg#DlDG zC%DkESpYy&=X#IOL7=b=#Vw%kIOL98PKQwDi>ge4A2j`?3s^Book9#PxSeoIP*LT1 zUI5^MbF9G=YXLm_fZ~R95YXhCo12jMHjHGE5D@OJF*QWe7LsQC!a$hI3JX_}nDVlF zO$8*qxJVv=j&cM_JiBCnf#8czUu*B^ZKt(pPMwlT<@QOxrN_tTzg+5vRkK(;r3fB zuIDj&@8gTRgnGkTU`N~<4EG9&BXuWGuo0HUvkOP7C(F|V+L+`iVnkAldrs(K!K^2`UcTAkAf$fGkGDkj7;9JvtApD zT6%6nLd~SF(znaK+9#+_)PJ&(ofgY|xNFf^CCkPkjRUH${Wv97@_Q~Jc3L~!DCD9j z&wjo}(n6n4?-P}Yw4NcgnwRtQ0^%x85b{0F_Ig#$NH-@Z=Ol8k^<8fu=Vg=dU;DZJ zfWSe{&4}#Eqsq$3X~C^mEEb(S=!1g+AUrQc#T(yK^yN^^Un$vyc&v8Ty@`;5-w5J9 zDOg!BabTDMsYM;DJD*-r>0|8mXpTZ6ifXzR1d5)mis(HLR9B-hqdt9|$RxWLD%6Wk)6%ZW0x7a6F&OFD=1P?#LCt z*m-w}`foJ-u97%fR0>=Aic2rnux;(6#EqMcDzo|hY1$I_H!=c4h(6wd-A{CMtvJ~D)8KtxO?8UX9R4(Q=rRI@azS0i;W`D$?+@{0eEqx>yg@Bv3X|G`FR2=_UoQD1T57{=AnB z&jL0p>#cp*&}r1Aa7eKm{DS(~(sItc*`lb!;J{jLc!#09w6qimX>csCbD!gUfSB+~ zbO$~MNu4uq*xxQz*F1xwF#lqHYXY5Va}axAE-tUg06QYn@YIdN#s5eaZt775* z$m{n<(xIlD$c0MM>`^L~DerI2JDh_kWwG7(*ds5yp{y$l7QS+(B==SWpIRSMDszL5 zzuX>nDb$zng>;CoNQL6bL7PaIK+E0x8!J8Xbr<~~`q3;8%0xymDpDlJ+_&tRs-ur} z;Edq^fsN0*)r0a23uGm!=EiKvS?{~8_0_4SJJp;YlyulR#@th8#dGG;_|}D4r3reu z_sv1)ORLcmajj|*XWsq5N~S&5Seaxcef|8|s(y+2eHs0o^Hc$1GHyA-7OqB_6%7py zKs!2YXoa3L55QoqYo4KdtC0K$>^K$@FCD+i>*<|ivw$=Sv608G@C4}hkA?6jLQn$? zD>Y9-Wo$)7MTMvg4GnRq4hgX{d)(~=YU$8sq)njdih9BoI`&L>BkT$tis-U+rMrRH?xd$zxz3N)bFi8l`o$i- zB;_7=0;S^o^uWlZady7C&`6Df_ig3@uf+~y(O=0P2>Sy`(HBQY4jk2bIj%-D2@>5K zqxMsL=jhxo8g1Y0X>j7pd_7*N&r~zIM{0CrkM5)KIAG#yDxuJS(1EI|iac$bL3wIS z??lqboC}sP3RKCZ@TS-2wvb%JJMjap2aW%gp1?xNMOewGD0<1ObdabM^x1}er=nqu zDHfY3Y^ioB{235*k{E5RR9i@JI0I~yIjK&}LeF4Mf1wR(> zW|MiDoTAxzxAA#Les zH+;XWleHe$R8G&9Hrac2X@R>shl2Y$KPGn}u|DD=E$aIG(e6m6&`hNVbLHJb^cIii zu|*2pOe3utEX?m-M>5D|;s=sD3T{vgx@pyQr1b`uFL807AUTN?rhS%9`P^v3ZYsmDZ7+zNj z)b(753cw8K$BiCUymet~eXB8_M*b?gD`e)6uN0r`7YTg$GF2!0TlO~fEBkcr0_J^V zBhJG4BaK;ddv%*RkzSN78@$ffw>_28*q)g{{^*ZM560HpOx0k;QZe zyL=DAi@HS3Ya7EdKH{X!3iWnb)2=HR{1|Z+m2^xn!Babf6E9?|?~yr0ojJMnGFEJT|2F@%hq{#>_-JbWKKqkXmv~xU zcH66%xUqtZqFbzzFP%KLeceyN1KkoB#dxC_qA$rcNjQkhLVusUv_%2@tKdMNc+DNYX z8{|wTs&FiWTdvIF!Eu2K$-#vDX*>m6cD5|!HzT=d5Hf3G90u}JuzZU4Z?yno8-QUH zIdj4YtTDt=k-{2C)PuBG5V?Ne?TT@MhENL&l%f+S)kkq$Y41B9iEV;^Mos)SntNrL zFtuwS=joX-KNr2c-aTGjA}a$2!yLCv-f>`+L638j-efi$dhf=2`*Tm$8(V4bk!f(E zO?{c?!gOs&&{yPiTDCfL@FD8i;#QWw3ys@@4k3* z($;QGqQTxwvbNiMiw{q>`i3-oONUjD5akEd7OUzr%jE1NCo|{#Gk`#<-<+)%L_6c< zqMYQ?H^k%ymW#al1w&l+!n<#!4pj;A`%b0nm>G?E1!(#=M@pXUo0jsaXk=a1^PX|V z+lwSmVV&KoR-5zB4=>G4{_#EVt!S;mzjr3?j|Dwh6$wq;%?Oagm-xldsrft|=;1cj zJILQ&ppMsO&!RQouM#w9leC(7PNMn5bF@v0Re1bcj)9hMZ@4+GChea3qEGuF3-q4W zdKE6XW-B3>>(R?^+Pb>#y*j$BNpdyW*|tr0AgY=K*+5m`_ADvUi5?=mA&Bl=Ypvc+ z*!p+s*PxQuD(;-6{*p(`tD=u ztd-7$3FHg(xwvGf`2mMZ6&Jz{cQV-CSjn8{NFq4TP1M}L2t`6lPz5;Vkm&TdsZE6! z4S@|Rx&6mT@*WyvG(9zm8O+bg-wggIlycIWIdP*zfy6GlJ3y7 zug~H;lM6MJMgFH#b)79EeMwqnW_nw}0pwD#FEQMCk$=vv9?@Wm?Yh1Tqcg!m8S3IXN*(%T??}F~)4UBy>(W23^ zG$w$wu3mU#uI&T0w|=VD$!o6@p&FaBQxajS~LIrq82yzqpAC6hH#ke(B!X^3Ob)L?3V*Gn<6f~3*?xO zrR_S>O|=#!crx*l?d#8T*z5&t^1BF*z)M<4%cg-p7tNlZ4$)Mrs%@EC!lVtx{wn+Y zGKhD|VR@%#GZW|hlfvB3xk21INQCRL*0Z|S<)^wm>n%&Fni-?ds7oeg&#`wap(Up0)kh$MY zLGbR8<@Bd_3>xe-3j0c;e|ii`yBf7MTaDB>x1CG&+bW2=Nb@4Ivw%MIs6>CO1gdR9 zBF;7<|BbDR_BA6Bmr)@O)%pIvJtI#Y=Qt&Fcv|FKnSeQ_1Q$E>sE<}4TFTh80$yTL zvpu7_efLv*tn(EbwgRI&F-{WqwJYo&rF@PE`&p--dFeE>&P4QwH>n);LdQ7&Cd3`` zhlQYMcSX0C@pl*6S4i-d#u+Vypp;`(vt92i)q%6DJy?Lowl^=_<=Cm5q z^OSk`+*~*w8#XYydRJUiPmz3gzB4wk>-aS#4-}6;c*hX@o~-^X?XiKEyVdPHrMFu! zxAO(qKi~FsIBC#3Wxeh*_NV@$#_ZjX9=t*hs5cgmYzHXfYdN2qVs|UE8ZRY3e$Xqb z7p{0ZClzi~EA-Ou5XLmmlKgXmes@Rp=fn(kHgj@`?v|LI&E1Q^nv4{`PI^C#qs;Yb zqkYK6B68{4*|$%wC9nChyn6)Q8ORR7O^7FtAF=lk$6+ibFiR!Y--DRHILz=djoxc( zojT5>lzYNxBJ$XEO8b7BTPgPWW>slsS_vYv4= z$ntz_cWJP(wv=7-v$@K+HqCx^?~_gR*X#Cc+hYge6E4B%A5+=T=~w6B$fNQvel#!R zkzoWL8>zO z15YUFXUMBIl^N@Idooi4m_`bvY!49p(udnI3{e(KYJ9*p zC@Zk%UQ=wgt$v|A{PKA?HO#kQP*JVhqn{=Bd&+#vFEtHU+b~FFUNTR_m!wzTZgW%} zY;9K4{KHvzgXV8KZY>Qm&F{>I=Z0K;jN%4+=s#ZH+<&MnYAB?3WY|?JW^ss%>(0P`nBHLaKFJrg`@eI!aKkE&PRo>Kx;h@M}A=Po7|^?V@V~OptTi)u1vj z|M)&P4xqT^!;4y5gT4a%jHZqru6L2#`hM)4u z|4?AP>jlA>)7qaux>r`Ji}SfD7+h|6U^*19C1Ku;`BLw}LC^Q^Vm(E^{oL7c+Bs%O zsUlCJY0&TL%b0rvwTpft4g0kYEfzId?U)jCV|}7YNVqnBN==z*=pF$OWV04YT{gXi$M={Aaf_hO@U#XmX&+e zr9MX26L{Ukk@2En1g~T51|1EQ?hK~P{eyYedGg3hk>%k$#-mRglGS+wiXNz52u6@k zhc9y9dI?<6KT(sSnDStUo~J^dW}hhhXd0f~II!TV$^NEp#c0m>3%*Qt5I}#7QPSV< z#XzFYcx;ZtsoNC;r-Mb~pMU{6qOZro9r@(u4R&PDZqR{^Uv2krU%-btJrkX>31dU;VLCGBTXUS=SFO9X#l%H6NHnRvQ73G<-c zLz_p&S7jQv_qNVfUuuc{VjGz9Gmt~o-x<(EWtdjs^J9tl;lzbwvfO_5ZF+}?+2Xw% z&J~v%18cmOiJj_m=gM#wDVo0nJhM`*fU~(-ltRN$-cozZC07 ze2&>ZQbX{&c}77esy>#5%HP#UxBj(0?T0$-qfEc2Gvqh9@Hl>MYiu5$UKG<#TEWPg zRhPxJhm5Zs87IQ%Dfw34F!m`bKFnja5D-joUXPj_c*%P-NUOoq_i&rU=HWXr-BCaD zjkxb|20egCO=~dH8{yLtWR^lKr{ji3XE;iY?s%~*zyzc$tg_E+;?n9)vhIBrkO<*% zk$9*a#bKmLsUzlM)GaDQ+r>w#uU;-$t#u_hu#+YuayYRjZ`4WcvRz!S#r-Htbsd4m zwjuh-DrN!GOnsGJ`PXKQuTQ^YOyw(i@5UzWe#%=`FWm z=Oapo)x4Zj3aq}Tgg@q|Pq=CvXK2)(=VUdj0%d z+J`7do))d8!+(rcW0Ko6-!{$L+m$5!!&4fYeg&#vFD<@eS-rk_# z=6C9@RPW|$(KBM^%q+^x(zppf^5FO@c4xLh>*i#>&`Tgzcv3AdlWTzw_7zho9c zf1bzEEHRQ?MUDKHw8TFtoOWJF9*tZ?7Vb%#b5u;J zSDet}HgpSHZdQTLbVVW$MMTiJ3%txM`-oEQ-A|Fi?fr*eOv#8@uYF<|Z(Og-mBmG# zh-f-TJtR!TrWy4ADRV8q9P=nsYoMb+=bWbe-5`|2xYfyov{OYK0irS%ekrM!)wTxi zpEKcQjJ+86%k5QE$@s?&ruU*wC$steT^;IpZhlZSI(tTsS(o?q#>Ec|{XZWH51Q9~ z;uLCZ4M}Z^^~=O}33KKfi6@axT82-CpdmjG9e``*1dYKyVn;VP=F} zb&KIrxuiZlI!(DeWK~w!+M?gH&Xb;U@kB)Cm3(cwtLQ&G=;7m9qG>?_)J|fmJ!f!q zd};Htl-P6ka&v5X&Itch3(}+OIj-aP1fE}9(wF7%+*Y0+2Nbfq{g z_UIoyZ*}~f=l7y)6h-SPr3Gip6K-Dg`zP{fq(OC38+p*>*ucN3U3

VEogM8O5Z% z|E1NBM`w68c>E^x;c|lJ%s*V6?hD+rmwz`Y{(00gI_;4-EtjF{%-1qvgJhF_;==9= z4L$DqFV_?){ZZkCI`ac3ZqmNJxI^hMxE=km)d6EUoM*t;!B2CO<{wVpoK}ZbOv$IL z`L8Ejy+26WM4D#K_uE}kYH~|(q&=piS)abT(sNbo5{eObs8p7~wK=FZTx8h4X>&uI zK|zzAGgap7EnJH6r}hgOL07c7@6Ay)pR1n(jb?YJ7&VCgCL0IO%0i}s@8Yt|6eR!2RzjO|CiNSbw)ZP8Hcm?9@*nK zIs2@PJK^kADnbaMoI`fT*{*EP_!c62JF6rsE(xh<{XhEs9}kZnJs!?|-kAmd8q%0=(-L(PffE(>;g6u#L6rs!FxDgMgu zaVzH2bL`mGH)rYMrqMe8h)-gmIPnny+ZSUN@?z8$^) z_|WIcIOYmD!@3ccLfm#tkXi2(8EHi3PKue-aA(HH${7a=XY#vHWc9jzoW&0F;L=xS)G zeNJSB|8jnta%*v0T}6aIMZ@^NdcBx;K&1L09XZT9f!OR0YgIWC3V)W0@dvs{HNuih zO}O@&v!3a&38OeLsQE#=nssqH!~wbGZf!pGa&Xit%P$ME*;eZLign57Iq+UxtjK@p zw5;KU^vql89ZEM-Pjubx5}~HA+7y%+;`nzH=<+r8kAzZ6B~44AaA8(4KEe1&n{j`{ z&!YVw?<#zf*>BI5rbu@B`}|kpLQFE7dZJw77&!7KH-ox2_L#J-OpwZvAKD>Arh<@(t)xpS~Rts&qwNnPGMJD-{xqHf5vDNTS?wYc=+N z94>buKlrot?)ljX)Oe%13&#eMuoN$T3z6wShA|2P_nwK4tN0sLd;Fwcj}HZ+TVPoC=qL3J{(%Ck}a;H+!P)X5cUa><{#E4lhX%*M#WYx2OYl+Jz%CF}p zij!a?CpF&TlN-4l5+_a0Vy8uBVGNN_m*2{&y)?rR&wFm9-=O%UI0ut!s_XH=Nb7hc zLx-e@DNcMuo8?Z21@G}aqw0^RJ)0n0IoD~7!5PxxbP3q09}BYa#pBjw8OeHClhqJx zV=0Bi&BBmpweUeSl!%Eex*Ya4V~FKffxmi@gelwh%CVNp7qLB$`?^mIhbGfZ1c{HU zCH>Y5te20YimjG>Eczf!u+3D~)<5BLbtCRB!(s0pnQ{ud*ecoDnw6fWHdsZbf()Jr zPfemt8LLT~YTf1#pwpb(haUk>dBN%bGkjEl;ZyI_+S@!_sYFjU(&;6YzPaUF4 zv4+oxp-0VxYgIlv$zY_0A2r(x0afja98z^WMsJcVcxn+fA_WSAWz_|N>P6cpx|2RX z88%lc&v~YLnP&r~_|n3Hz8F0pGb|A8!F6Z`)J`I_h4tqIjZDbl-6ZouOW`CzR$4)Z z(u$kiQk2dn#lCwlTFZ@B}WAv$Ez|ipJUSZKux6r?$1eZu->a-x0md zu7fmUSUX1m;t>MTwon#cEylzYx4aPGV5j5ru7PQ$23}e<1h`IsP5T({8@2TGcu`M) z7R6y`%0?J-K?=OkQYmB?@}1~B!-@Q;&*tu0ARTd}b9Z|tuoMQ5gAFyx1>@Cw;Fg=u z#5U*J=+m8^CRTl)VGP#;Lv!fjIIpodFA zLMWdlH*M_ONs%0bjl*_l)|`>jfy2`v5mB(9S>x3o1z;48k!i?KbiV&vp_F}gSZbz7 z{FCFMI9R~e0%kWd`W?rf$?E;HJ@?`gkbFHzKRI`#QOxSCDo~hV;Lhy0ZjW|aLM(e}v5Z7S z$rPJE8lfETcy11}>6v)X)5e0kTh$82>3XiKvI?6%if={|N>J<@DQr`de?&P=Im-2V zUPLm`R7$?kYIsC{wwy7afh<_p*rj;p&)fD}W|^1t$rzWcT6n)h68M7(;ZsG#>UY$& zQ{7ARF3L7ievma2sJ5pb@F>s0p#u55wztnIpDrDH$!89G@vc%1c6leVq)bh{0JmhQ zpDAJ1oa@<=>#1LxS3qdd6`%N}lb(?3KOnJQOKU3ALC`{ z6~h!=kLxk_KC90EXUdN5eLjCBl$J#mVaxK?vTUt4N?D*ka6&b5FcS;NFiESkmN?{5TO=0Hv93$$oT!x-IDtvS&M04bi>^NKE)W{870)zX= zbY|M92X97iTc=+Dp|bVM;4KYVapDTgo{`IKyo2HX6PWzfIst}KBFtTjXFvOwCB?U_NK|O z48v^Y?o;KPbLyQvS&=>E)xIcmE?c+=n;wV(R))7)n)|p*l!fPRAN!~I14E-a8(9@a zes#aE`|e{#rps>{{mgl4EqH}tR+Bg;?#$Tr#iE(~61x2C3JG5amEICdKO4-4XN7=M z?~ojDzUDoqdV}mZvU@mWb60)ywyVrCr!x9(KnYUtle+r^3-84B?cZB?-iz$0QmzrK zshYo!M9y{rkAVIK%E}SLc6{*GspPYBEF@Gr$r_llJJkOgEPiDF-?)xqE*LRAAObFO zIHk@e2cDe7pYmsk6FcO%dc;_GOSx9j5T}>N3aq6G)Y+CZPtAhQRy;9$>-M`mk()j) zTStMTe)Em~gGYB10!k#J!4CcI1$T>Nwi`WMJl#JCjjR0xnu+*Ad6&GptgOey*0aqN zD)xW2Qtsrd-OD|%_`-$t9~ttM;ml3W9xuF4b)Q5Fm+z!hh|LkKG|fQ=+?n4<^Dn=v zV86!kgK#je8cw;?AXOXrsC&P@bXfI+IdaupC-EWKvw0G`p*!YS@tv8MmA^?R(s(zL zMP&v@o@$s2S=_9*(_g4c{(Iarsr9y-5BOzW9{O*Y1aMmJ(lO7RU0JJB4B$$-##eHX z$`C2$yc%v2(Lg^=@>1)MJucC}J3arQZq1QO2U3Gz==id!FAL(HTdH#9z@)2tZ5Ik~3S*m+Z-U8yZMZZu|pgVq`eo!2L6|S__*F?pmYv`BE$Lb1{;Ne~EFZEMn{ zFQYRSr4`=3Y`j8l`RzLT7z!%oH&7JfOa~wUX$gkwTxEzlPG|-Q>J4|gDy?8YyX};j zjmd3&z2GGiGx^$we9?X(s7zeK9dU8T2$(tehY}yptX+Azy3oOkOHJb`^i4`x6Zx%_ zZ+oe6P=>gLzwd#l$xQ-C#OVuRW|D3O&uD`G`z@sl28yrLG_j=sOz{Mor~F{v{QaG5 z>~hiay2?Q*|C>^d>8~b=Y&+fC44{#M#!-VIjMS`2VDUQ9E9v;rr{KjX9)h@l5lipw z$)e+VWsPvDY)s4MbQH?&!*VFo|5}y)&-S1Zo*wV5htCr;URm1M+Hn{nec=Qgh?Ood zJo9&W{r(BQ;oa9|ISwgK#)o}tz40b++tL?>ceTf!Dta=O^4G$lyz`3wGku0g=VBU@ zQp}>7!u}6eAPYzX77|>d$^AG{2q*an$lIBy9UbBj9CS!Lhb^*Kpj+I16e5vhM2B!s zH(m~%uUti6Lls@W=}+F-3=e0kksKl!p$QiU&w_=s9WLs_C}0p~2&ad+gK2H;^zx)B zdGf7#ii*H>FT}Ssfh_m_%c11?q_8n^Wi;YG7VNBPd4>DQNN$@WjGqV#mrDqA&roeV zksj<@T7ZZ98P1eFw}cP1Q5yZbWLOQ5Ag+?e9rnIugpKcTSGfQ9aJXWi!nG$tMkQcO zST2NxRS&dXAfG+_Grazr3dg3=gb=I1^*^BC!6?WP&c9dtfnZGXP}R!7X)(bBv=5)(`Un0k>=u$eG~@1RX{$U`bIim-*zTnG7;@l zWPGG)Qd(srjGx+!gTfD9sQ36j5L7+ew6OJp^HM%{Q7!@g)*_h}+l_oIaeoj&Pyp4e zjuSktJPTeHe9wXxU?Gpu^3Oa2e)9)uO-yY34&Rf!dA=H8dlu8ze|+X2ySH=&$zM8Y z%f#q8DjK`4z9)0vFUkdLPDJ_+mp4{bRU7b+hWHFHI?hCg4hNdvt**;&(j*5vlS6om zr;94@?y6~;&f1v1`q%&bf4dy&lF1uL@|o) zfB1DU(~jOY7Y!y=KD$ja7SuyIh~bWY#u#s)Zvgfd-&P^4w2WC;cNUJLvH|XHQ$2nE z*gxs0!T^O=GJN^3rSAWnlYuQOx??2p{m+H@Kh!-7 zXPCLEk8&e+_5;6barQ<{e2*Yje>r?T%2S%!^U6`kc zi3TlunE9P0!*v~*;9e#(iLgwfL(k2hu4{gJ?yL{f%a{zot`0-7?D8+tprwMQc_O9% zx~KfluQiq=r)b7BK`(L4%YFQLn{J$o>q$%g-MggE_7OaqrIKce;x#`oKx6Kho|js* z%<4$n*RcWaF=PCujU{Gzj-Sqec(+IKcU2)_j+&A@B>uHAJ6@cKguDKf2I~14tM<$o zfZla04f^092K_0Mlu`pd+`C2G9PleDtyfyS%MWT+>VJWLXS3$lL6=FtO8D-JT9dWf^sbo_KizW~GNRDpz* z3R)X-2q_n&pfslJFlx0A{aqy?eizi_vKHapNRp4qFfT%X^$!Mqa( z;ASS%9O$mi#B^T91o4ps*#BY}jg{_KmXl#FFqx7)6f%nmd&RT0T-Wx8fQ0*4l}XY; zmF(n*A26Di^E7ABnil#EM$a-kLn#Zi`fj3^KT|&unG5b!|JMiS*#tn3TRyS11+iC1 zINPILJY8RkNkv!+af%w)U{JQX0!)eGSn>b=?3ZpKW;#%#00Qx z!>XRIFZdd)oo(eIoE zNuE4cq?LT6;*`=?V_>FZ{(abZ&c-lfZ@S7A#f>)a)EfKsY(Y0o5f1gUfo?QD%}!z{ zjsMq0=%XkQD}>SN@|VKm6{EuFZrxjr_*G#0>6sCSE=OvLfye1>f}#lEQ3aiVH;ehB zcZ&{U6~_+~aJgzLVlP~tBv8Zn3IS+VX>|c2g59ilshv@ZmEC}K>@5A6ybmNARXd5S zHIkc~tekUMF9gKAx40pV#GxrmOAqh4OC<)p|F#D+Xr=a*rL__QIcJVaM%u#bUj}&1 z9zVJ=%XuA4r^GpLoMnnRm>v%K*Yr%*@`jIF_g&ixDqGABvhE2B> z7%&wL$P@T92VxE`s2K1B2dG6%Yk)Owb)ZHHB|37K1?x*ares()(45UQE1eXxC2vY8 zECnZ(##J`VG13HK;(ZL(Qw8`B43_0YB{_q(xO3L|0b=V z364$v>u%$ljVa&oNYhAEgXh^iXD|at{xYeAr))UNh!Cd!=_Ej$f*Avz@Cnpdm(`Cy zRIQet0_qFc8a#|?foBNu0`^Z@(d9X}OPROtlLp8o2OX^$kyKAi;xTP3D6U-05w!xb zuTHD3$yMxGkORi`^36klFuE+nQS(%BT%osM`eg{9kV>F?+j=JsI!vm`9`Lj_8mxS5 zFAh)n$1eL_0*uU;Ql3O*;y|)dfz71ghT-0Ig6A!CY5_yL4qQWoj*a%OgOzxuk|)SS$VqGcR|&C!fxMd zakUjm!SQ>9YXENrVFXUoZPv3F;8dO!Tkoz3kp`24Rkvr4A%ue9#<@Bg7l#oP*kEW9 zKkeY{huLY(G^<2_4NdX=9>!;PML3=1*3JwDlN==#<`qRqw)y-Ic4pK5*Zqa{pE22s zQSy)(mf~h_s^aBa0RnIG-;vNin|;A!x&9`0we4vB0uu7$R#1(hrPn3GBjkB7F?E z9A_j!j3>P&q~y=uPGAg5DmmNy>eWJJJ24qCF&U{4H1N=Wc$E{qDQ?zbK}b{6zmAPD zmu0MjB(g;}d(p^zoFBmE-Z+{{$`j#NU8++haz40Am0#A72L5!R28ha}9>7u#rfJg# zfl|K5SC?52+_?4MHCZQM|fl0Ma8#Y*snDMY4FqyK1u}S}9`CqgEAej>zk|XQM zXLb*BJ{Ll5fYTCoq8*6HhLNk=c>l))K z3G(*?LyEEl;f(bxU+`Yt)NgWd`t#e0g)ZC7 zMdgLd?(Nuh4nE5X(FE7)0xq*0C^FQ=ni(;x^m-DlpQVVHwiRZx5;S@?1Qh!>p1s{j zV<|A2A^Ec=(^Lz3!0Kk~FSA}S5cVB3T5gOhFui09HoWoTEL{v5SGy^QDF}J>oes%i z5d1r>2L{#!C@mYbX_lcal9Bx?7m0TocwU(R;39kPR6!GUM}cpZ^#J18dW-izM?+D{ zJIYG8b6fd~J&BjvFC6jis;?Z;POxg(SER$*ZJImSz2e7)c+2MwBwy&R{U2MYz#NTG64DKhjyTqZ;r&GL~{eWN+6#)?5vz8DzO96SNWoTsfbCL6R@oW`!e= zwx-s=e-sh(Hip*1eUpOjEzMA8Epb=2i`C)-d%zODI$J_xF%14rD<)q}qqyxzZK~z? z<(AjDS(~-SptDAhmR^(*&DPHX`qA&8GRoES+9wer!drK#Z^oaFhSHgqHUdW-CJ|Sj zlp^Rus@n0zmURcDNl=I$RV7KR$-5aztm!5xf~JS_1$wl)FCj`}1(;291uQBOeiY}H z>m^@|J?`q-Mn%6z?ken_k2TJ{E{2&co4r_N?DLXKxco(QTOU{1>~~FLz)skpQI`O6 zl*7=@R;j!u16`xdE4`@+HO&Z8<-TCdA_V+nh{SEyf8XcZ)_kV;<$?3~Xnw1YV+E$O zXyd$|xWBM<>-I@H&yQ#wv7;z%%dMZfI+0*YI0`Ha-#HbYrTIWQ$MSUF+LVLijDpP& zMsW(8PTQEyWCg(hl@>dTL`rX%ZIVInR23`3Ll*rUz@(Lb_@f9UYB{a%E?O+WW#JwB zjra0e6;H=lqOoPgzGGWfqphmPYAgUgBG(Hv#{j#UTx6svRQtQ%RZGnZYofc(Oe+Wy z1R^oUIDR(3pTD64^v}8+hK2$-NkewMW`n9KJ8|QtVPBEtI54N757q(#V}XSunuK%f zuz}Cbgei!ovDW3dmxNKz)Q~ep^{rpOm~4C1ZrspFkZGke;yR9c#Y&6hPsjisk`fO_ zg307Km}#l_NS-_XVc#YJ<2f=sAi`FrgvfyUW3NM~eHWvhrUL@T6x+~wPV$QdIYXp# znAq!F>#4V3ze0laK#_g@f-s?9Dgnu0#G9qupQnoYd)Xl8nI69~o@*H25={?2b zgHY6NXZ+E-oFVex!8fNe9U7t_YEgq0#Ny~_`p(8obN7^wB-88*A*`<|cZyLz{f4y{?q;vQD7EulufN` zS0+>a+bU+l{rxxNA0~lQ41mf*2N=L2cnWDhI*I~rqsY%L96?3Se7iRk9I5UpZ0&V- z8h*2`YiA?m0;_T@A80K;#KK|iDRE153&3Sl{FX?VhOSG^l?4@B_Y_NO80TyR_#PPu zmY6^}lH-5gEd(r%_t}X<-#mK@Y|SkmiZ@9q-%NqL6+1y)d|sO=POq=mt$-AyKymVV zuO2P!uQJDG%8EuY4<>lWrE*I><{lZrav*pP7-&2tbFd>`6elNgb1Sj6K4RIYEcSZt z>g{ks#9ju0S z63IoQDs6cb?o5PKuKD~JFgNex)m)q7&6=>+*=Bjni|dZulGk#^az)xa8_DuQ>l|lrhE|3Y|K)GcBZM9!rsZf z!gXb@>9pKY^$~c!o$r&wX6Y~dUqA;Sfqs#Aupns7av*$JP320P3z?Fi&-3A+5r>oS zOM)1A_nQZ9{_MdlR`|4tHVxTm=YGrd`L9KaJYO`DAMh?rMa|<&9*TCLeXweKCLPfz zxpiptt~}jr8dce05a2*|c6792d(4%{&6q$h1-#!suuM<2LlV(;r$wT0Zp523Hd@)S zEx(HCnb^^JzTI_F<=OPI((av4wP*>9VUQrlew?fjpW=(e8JqX&h>dM8A9W0zo<8T9 zmkKT{Ul*ZSJ&t;4b>F)wVLa%ws@m!;Nr|^hhiMg}%w=^IUKKiCNXPn`r6f~`Guez# zUy;2sY>PN%srDOh^4q=Bvp>TR`}3TpOtbhF@8EU9SW9?_VkurERV?#k|A9woKOp^} zl3f8joYYf_dMssAe#lQ=tlTNfxs%+S{oJYVF!!ah#(>1d^QrXl^t0hWW3$}MF~65I z59uswTD0tlIe23@+Z|Zl?kT4VG)HcWlulU!8k%MLwA2|b2l)D+GEAQbuM|)=9$k6= z_t|VEV5J{{OFK{#2yuDgez~tm1Y!%yJR!dtn(40x73m*Y zs5)5M*a^X;@uxUB1MAQNG?F7*-T3~RH1J%icC5ikYa(Rjp1a_z`^fhz+&`kZ)_Ky+&J8ZGj4u8s7;+X3?VU?Zb#rZ1mz`MdE;s`a0&QX#EF~a=!0bM!iHq z=<>n6V)X#a|D@mGzj~By+N6UR0QRA&0Evruw zvlMf_9WA=g)l6Oj;7nHm|0joB!o`@_ZP4sUIm$Ev>BuKx>TTrxbmCo~+V?1#LJO~^ zy^2NeJc^@ZPHWrq7xk&@=toyvp38E}ve8D8p?m2W;RO+>L(oC*TAoR{2%pU3LH3l= z3x88yNNN;Hr`mpTTpCphUmU2o_b$wDP$nWwYrTN>9250cd$hXH214j2eL+=^Om;sZ z#k>N_Cn1)6S;S8&;sua20;;GNgtY=|U!}QRtljN2tdE(C+ZK();JxQRTsBm$9ywpj z@hDnvRe2ShAyHFHxS{n-cm!E;CwTAzC}fGa|vLgBGV~9 z|GWrzx5Kf$(&#+FBso%xH35R0!Bpv7dDA2#Asm#23h-cf^qSjF!Fxkxdp`1D)`8j(HcZIC|L2S}R_ z?YgN1sdSKMXO?DL^UKQ1;cdIH#s$kKx*J)WKRmv&h)9N$&-s1y%1+f zpscxP$Z&@P3nI@6F2%eu#$bM~=WM@LRZ#wYJQWA`_wXIp4Au3vG&8SveP zx)NWrlLBC``5y4l#!}P)@35Uey`zLkyP>Eg)wGAFq*tA(w(hp>FBSzlMWoOdNsd%x zJu)&j6co<0`{h%oIT7ltQrK$6br<1(XS3#8#g@7Be)~Jwyz|w6RYd{Vv~~ZkVrGl# zc1kHeyK0lLW@$n`mI~yS0CJo-58j6rDhqi8f59v^zSkymTfHq4P5LuoNC`AFNB!NDInN{+ZCW#9kV04>qrQ zV1Q7u=xV1-@cMe8=bAtUM0h0Q1HD&Xggj<`UNnd3?c+N{n9lYgMZefgbNDE{d_X;% z{oK#bXUw9=LERsYF+{aD>H*qUDp@qJNo8#z&2C$vcFOBY8WJpfzz~>5IPP!j+z+r@~(LM{Ac2vbSMwDFdJx<=_4@q zBuA#doi@2LI3iNNMVgG~`tTI!hiQ^1cUzR8801ph-A(NI(=!fmvVT^mZ74zcSu;b-&HPs6f}Eq>jzr&biO`RBhMc^@W-@ z4&d91Hg22U&gyRnn6XL01ODtudIk3&7t!LtiE?tC({~@UqmbqCAAdMCU;T}NjTWIb z7M*+Mf{)fLZD#c1lOE*BI%H>#laA9)p69eFwr!b@u*B7uDmbV}E;sp2G_TBm<6ZB9 z@&OH9k%yGKbY9u#g(8&8$6kBu_|G6(^21kUk(`5w$647I z-|aLm&XN+JJ1%FVrzw6Koe9KMO^EkFZF)tb(x;Q>p6R?Ibo4IXD# zl(BXA|(8gm3rQxg#B0^<}VNW@pNi4!zAPKFyI^_0XpVDZonJt zu$a1RGf1xgu$LB4Q@^0gVqJi|ma`W7e9#O9LqBSCvJ)^pjQRSm0&m5QZt@@6NC~?W zsZDmv+xsmOVZcbOQhdu039n?Ysql4F0KTnTG{qfQliGz zT|4V2Fs-|-3OfPb40XhPyx)-SxqY-(J-a(FAhWz=ry9`n7nAOno*x*1dIgzkX?KTY zn4q9+d-oeUx^hRQBH>*F^J1Zysa*=qstK5+dl z47lLZb5ss-ZL!yh+k1w)wjujVJzA`$$5Z#cNPezo;;^_6N*J>OGB&8*XQGw`49>|t z^?d&~XuGx+>FXktH+Ib?u2&<8xnWYhk)_S*Y{?99b(BQdw=yvT}UBk985mwWj^;-8Voqa5(gBuBGK_YqNAyp(N`QnXlm^~t<2E(F8*Toy)@UolwUQ^bN7LAKdoJ0VG4^Usv6joeVZpD zi`nVPEr=$mLe%n$&rRRHDdFk8w|At*uZWUYnt?riyGx$$IXHe;Ta-C+W*q(dbl?*X zgE;A0|H-kaw+|}F3>z5EO^2q~W+ewXLw$Hl9Onnm%0cm(Q#%_cCu{w_LBQ^#yi%rO6wJT<$EE6N1M-uYnQg|0M$T4U z`EW4Zb7}4c5%apn&);P@LUUMkDN>oqGxa<)cl`OU3gIt>xidf?q@c6HLI?wJHiCo3 z#Ry)?nZLYp+apZfytrP!86WFY69rMq#_}BMV}O;IpNqzg*EQ}j zdp_+64GlJ;Ssrgzd=9!-N>S34#e^EmTLOK_Stv_ArzEM|Ru>gF=8oPi_X6}@>jZhg7s{(;4=uK? zm$;WqI8RKV3(-sd*fuA}%=C2*4M8CK;&TKzROe+z+!&l>s5ldwu%q!aJ#@<{Qtr#9 zbLZr{4c1tE_VJ&fk@2m@W8(5BlF#L%s&6!e9)E(Z;jMxSXQD30ri0|WpM46b+K#y) zw2$=u9X{^J)A6IKZPbn*Ece0(Qzx=bpKJ<1cfQPK0V-9`ZLf@CLb3J-OIfokT{0de zp{X7Ih}^6jA@Jcr1T_9@1rPnUX0D&V_m9$+O`eG+`fI}(6Gw%u^GcX8Y!nm z2jt^x5#6o%b|sI%k;X*BUwn%^+hSUJ^)WozfVUe5=p4?}2Q9rzOef|d)}2;0KSf@Q z0yV&FK}K)Cd2!Q9Iva^yGHe*sDAzMAp8+=3jm;x$)C6CirLppAQ)aaip0Q-oV8CHt zqO`ihe{s0g`Q0XjBnvhlad9F`GrVh;o0{sb1=Q9b8G#7aH`~XtIs)g1uj38(OK%vb z0g{?451D+VmVpah3EXCQyKl0dKwJLd%_YCisqYF!fePul8T3@1v7Xx;gt1nh`TCzC z+)^@PJMIJJt*vPFpMMvB?yHV8cFKNriQLY-zPZaQVxo}8WeTFxi7`xX6^=t#7}0zF z_b_em?$w?FO<8B2{GaWuYaw!OJd%qd&Y21E)DOz1&|_cCw|9rb57jh?KmMrBX=xQ! z)O6O=_MMI!*Vnjgf9eKq%C)Iqb;bUUUJOD{r{0k4a(gQ-V1xUzX5!b}7!p}#C$*Sr z3Q!j+W72n815}C#HEo~M994OSHCu)h_X_){8C*Tnu_3nLo8@Q@8wz$APP1wKb0#&3L>2 z%63_JUH43tQ#Vzwt(6HUZ6~~ls=gv}``u1a<><}2x?^q7bFHgJ*!mWqTQm08f#zig zZ>?IuaR(32rDJ2CYN-Cj;tmsDOAEiSW67EZ>1xOgHA+%M7GhqJqEh9kAWsJGg$tn6 zMfY)=<6tV`}nYSKg{hCoysWy(-Xi>b?#lEl=Fg>duD#BBHvf2Ep-fApXxfJk6vO711KYxSM<#Ddzr!wA8(}`8)|V zgjOR)6@m8Ny@p9_gbn((XIsCFqsZ31b1l7x^ak?CM zZvB6S%~J(8eU|pVagygP&g(W=`8c0I>X!?3?^`x&1m_*bJ(8Lazt9%~$p#UYKmOz{ zOUtlg6aeU{>H2|PFqOG!D|o{R z-&Z8w=>Ik=o{B9_6Ry8{^k7oVzA(xd*^P=)s=k zSr;C+4Xa&szh766sO_+g9etAriN${uR6k18-;#CH0fT9^9wzqeM;2LEHzcFvTQ1I3 z@P{(Xgx$22EFR05s4Oc{Ru^mUh;EyP(3C%cACBK4-Ci=k0-}u7UR8#Kb-tbX{UahY)(a``q;RPU@iLhT3)%1{ z!nC#?FX%nYmFq&za=ktMgE>_$ZqHO@$C3K~EQ}*cae>Bi17sv)`)Rx_Jb#`*%fz-D_WK%2oSoN+9 zV$w$-p;szx4l6{dIu>0@uy(x3lsyKDO#`JFJW z_whAh%}4{~dIf5lrw6@F#4lP|9xrv&YR>%DB{eF4(Hb?Gqh~%dHUB$c1XEH&jUzTM zuq-o$m6>--ZtG{!_8rMohtNdxQQS$HY)xs%72t%$>BiTvzq#mxO$c`)QqZV(*QcFL zqJH;4bd!*ukh|)sR;0*%Jb0A%!>Oud2Ao~x=9jPMGDX4cR0As0w_@~PnEAX{E25~U zkMy$DD-CUE<`x*I`PkH)>y6nUIX~28=+Gtw!Lh;_Dn+Bn0IAf9D{U9q zFjN4sZEB!vQwj7J3%koiR2iN2Q-mj27OZYi-+Jgi;n}loD;{cX{WxK*c^!X_%1}g`*S^$d?fwbzAzEocXft(XW5KcD6I})7N|#Dr+MAtkL(^x5 z8WjPMe%%??{rdeNgzzJ1VKA8AfV8NS;vP}Ji3Tr6P=x9He_`Qv9}uk*019(mUmh5?8;2~{L~pd(g& z%&BkbO$BRT@n+G#szmu5k?#u_*K=n_+B$>abC)Gd#ik+6wiTqe8ce+^fiO?FajIfj zoifa#!&T1A&&*FvrQ>GlAa`WEqaS$7@N@Se^R-(Zw@(RrplW=3^arju6=SJ5*{{2q ze&6_WH5kooIq+2OF_tDDf82=t(?32Q?G^un$WHHmG8z0Gv9H741gkkWOyv&tjQMgq zj`6~>t?g^T$svt?;`6f=;S_?(x;&Sp-6f_dbprGWF=w3^Hf}66oh@o4og#uigVCUP@|M z>)0WVq#0`Ma=Gg09rR1qSWubA;@4T}kB!VOq{@k3f8@F>VTD42sr8h%J0*k(c*jB0 zmGQ%pX(vVq=oQf{?j=*-^vWlvn_jDS^?dYHCb@$LaJJ`_sE3z@sLts@K^3LmOW{~u zS<__7N!sS-1XtO^GNh|X{nyrzT|l`IccHB3sv?iE@0<{5s{Y#V!{6=F;m*5}??y+3 ztY;q2`3|!7QpKpc?tLFG_t6;X9Pp-kwr$%NH1M6L$o{$I>JfM0xal~1(zAI-G5=8( zPF$CfPwu8ZcN;$aNV5#@0XT;Nh#1B-N%LW$D{gZILKEP-QS)9LVOX14U-#Pd&YtWv z)w8Ujki}gng-0m(_w}psFT)?mf`&K0{K(k)4vTsaGpE+0-N2x9&E-HO0^SR zneTIvo`sDYH}ug-GHSbug|Vl#Nsf>L7lG*QO6t5(&~*lU+7E9~t8yWg?nn9E&H#tn zoSwCx?j7{M*~9P&BieC5%!rQ%RprEND%?}*3O-KHlnb@= zyuAKj)g7s^P!q_bpJ-xCdTpYM7+N5Hr56v$s;qfaEB`=KC7!x*aVXmgo~sMgL90w8 zIyryP1s!S*9(4^3J>-j?X_Mu2S#g@D6jCBAE<7ATy?aO{XfB@h+e+J^8ZuEUS7}i7 z_e6(K*o9*)Ne<=wh+FbDe-SK?+A}!G@YE>eO`?1!a zsMLgY?{ie(!P@$Hvj=|?RNp~vDl+yZ-&uF4kH9x1SzYyv*mz@aEB^Dw?LYr*0T<=! ziFaq&k3$M?>ccfb)?x}&FI52p1*%phJC{aZn^WPD0@`xvh2J^JJ^DSfeJ40%&(D5< zs-9(AFukb@cKC_@k=8ScYG~83(0T0sou764)&m2%CbuLVjlc?9B%+EM@+3R(UX@h@ zb+H~O!Ll%*wV7L{p1HF-75PaO`qo}Lhj&5<|EyytsBtaLBl-ih`vqtk-FzIQ%c#V= zY0Al%tY#0^1MGP=#{Uruq;6SY!bXj(k(rcnB$J4|E_%akoop=hvH6f5w?CXw3wfkf~_1c_;b5j zQBLi6t9rTYGq2G*4rmQw4h4dwyXUqk@X;*-TxrjzK|MH zxgs}UxqJDwY`XnK@y~^UfswM?LKE!u<_|l=(tq?omU-*uaA=c2oUBFv&0>^RII(St`?YDqNl7AD@Al!>0ntCc?$ND>>7rhF z;Xv$n;c6W}t%{>Q`B;Ef&u^z29V0i)k?`r_VSo$~?=leJRWJ<1r9GH4KLMMzX(mVh zSFg5uX96U&K?-Q!2WkA0eMzh1J>8{x&@uI4Q4#U8Yyl$cD#>`x4KEufVe0&inqDBa z#(S5l*5XA@amYg&>8#n4IW7Nc{QIttEvUN)m5R-TOplX-1&8cN#?)8o+w7zV=N^oB zZvc)r@$r8TyEnC7*eF;%MZ+xq0~g;`zrr%3IFh30Q;u$?Ff?#@cvD7FQz_+Pr0Yhm zY!0DJ59ARyEje^lVs$Ti^)7E)ZvhPC8>~6Igyvd3)uZ16U;8pb08aZ;id|sHF*Vgo zD&Z`=Rg&fi$+R(puy?e5ZEmmra%4tDLQfrOK-DQwETzfs%_AhBXAe0bcn7o-Iuyh3DJWKX7Tx}4Sv5n9}Kv9RL5zn1mY;`Lv}H;z%o zB%n`IeWbs+7EsOp;p?@bWP;JOg>Ny81eenLKb5w*iW&r8#=5_Iwr~mpz2NA;uv(}W zS0lBoUb|iESrl$cUjFR0U$H30#dlh-BBVe8kix0=x^zQEAN3l0y2pTRzk2uIwwDRo zm4>5r?LNyx13y4oNt^Dd?hqjhR|On@C0N^VL@x{S%4_5C=yD)&IZzR7%q!kU=*0q` zd=aALpf-NocsBDB2uSPrNp-wMBF$Kms!zQ52?tAX?6US1p>cbjGtnHK$8A0YL3XoC z8~a}~vf(zn5S(Z?<<(N$sgmkyae|;?x%m5X@zqv;uFu^Dy~<6+3E%kXOt!gF<&qL5 z*m^$x&_#|#&zz0wg8&`5Zt)>4P1Qq*zi~QBF!xlPcItW!d&Jbxy#SE$%2#iUXT!V) z2!tR2EeVfQf9-Hb&Ct+0)4B^|38gJ8w_YanN`vmL%M4NuhC*@}lYe?@{Qxmy0>j8c z8ze8|vW9gmibn+%$GC?)62prO?2(RG^`@ryPfx()o&7Go@uCD~V4K`Xy9(r(N@-R5 zw5jRq;v+*yX-Stus~rn)O4*BX<4B7F=JhJQZ$F^u6^=ma#KY@TKR*>DOF}FH$w7{Q zE?@CA{dKgXL?nwGPXC%6$7E6eZ5`d{pe&AxGNS16$b8`8cxv(?)=-(Y%V#Y(LHL&f zX`;{jT2fM(PH0>pwa8D;M)ct8&uJRHUo>k@{k#7E&D^%$@b!x9%jjanB%?eJ)uEc$ z#lIKfODhN36jelzT_$?93o1uN2OU9Y=BHo8erK=IfCRb4!@KF%)QZ1XNMjfomr$h{6d>f()wN1~l> zm*eTAaj%0~nSGzY$V?<)RN2KUp(8(tfASt2ntd!YK=dZFw9O zX9mIb*f}cWgMP8oCz-|}l?p0r1=g@-)&k~v0LZ5W+Ok7%4;jh4=~|~G zZEs&MXEnFSX0_d>?%k(QY zBRW)Nf(dLx&Z-$Zc^0CNj^b+_50$MLv3Cn!=+Ixe`)28ZYq(B@BV4Y8P=934hRbkRK2m&0;#g9xCR$I@YqQ*C&@-EV*!O8Iv3NgfDE;v1bF%>sp(>ep z*cz$KR+yGVLermjWsb0cGWA4iL@$87THs7z@i3zs>TJfwotxuEN9>;r5qgT5d=G%n zf_UOTJi7HKL?jS`!BT|6yY8TWglcvs7blGxlimS}HmzV zpflTYhYvx8nBK|M(eX}={6h7T?*Y<9pkZeKFI`CZHVDrdN)q@Hm2#(bpv(&S02$Gz zC1$-7CaUNOf*A7~S7^v9s@V_kV*3miVw>5bYW)lwBgo^CNlaC=HOrQ^4z1^H^;1f5 zf~sT8PvyZMkD~8$G>M=h1c8BqCQts*A=K&wlw*Z?Z8dOSPCxg`7>|~E>8p$H=pzbS z*y-*ik8Wrxz1K}NnKQF)Hrx@N-%EBjjL?XDGIgI2018*A9Q*{y9MF=TYbSPTx2(E+ zR+9nKB7KH%z94qc%$DlgCO&&%s&oFi%?Nrq-6~gRi0f3M(_Ekg z&oupamXbC&g}BA;yKVuFK&OR?GPiN?MTl%BSzdQMg%BklR{xlji6BJ=%96~8{UgjO z10~0{eH9crT5NA`1A%bWlT0t`>K9SnLf&MCf$S~OeK&K{`zh50*-SFm=QOw5&BxV| z@GZ-Rj)@XGdw^rn*YSIG+=`S@znThTe01{gq|iU|qkQant10hwES%LH37;q9STu z>qnm{{?a~36Xtmpm3&;(*|&ntt{}r(pvNYIr|f+p7OAVc8!WE^pT8IBPf7MRzYgF%uJ_C& z^K2-;sj&`KjOzVf8TXsQHSY0u(ss##fgw}NMA8j9B77o8<~T=2Fl*(5dMJxx9eV^f zSKd!_wTb@-5)7xkr|A_WO3kTqx8c;G8W+I3ko$@NjN?5%Ojx zV06@9FlyYZJ5T@yHImD1v z`YwB2=;FchsLmUy6F375B6Bz}d@$RZvL>`7Jz<=*unAT(B1`PVtcYvKR6CX$5hmnI zBTRbd7B9GOpfgBakO;KlBI&l)pG9)ET@L~I+u+`c!854mT>Mv3{I@s#wh;Of_+xZdJ!)NowX*zPF=9}*$?g!T=L)_1?BePiQ zZsU8;m1&7Wd-GC$lSK%xdXz8{;6BKaTa7mWSJ9sUWch9v+yEaW#TX1-53fzhT>vWG z@ug4TwRqs{?cnI;vqrz)_RD`-?q-`)9jJ2?w$(4{fg1;i@uKh$Tf!WobBCW)TY#uV z0ti8^ruL;(Kkcg&(>KmmliIA*;qq5h2Wsp*gfb<-bgEUF<69yUTV@{Q>m8a~89Rhn z2n(Rc-skeB1Rud@JWQOC`(6~NQ=!9X#_~t~jcyCmW56{Mhf3~_ne;Xy>u6Imbj~FY zZ6J1G<=%tW@8p_@8EJzZTE^0ps-0QDu)z3O#xQ!HL;OrFw>eG>7`7x#Zp?z91^L5u zdM_?cRZB`UkmXpXf`zYG0s`y^dtraSxK0Tz)XvHct*`4N@{)4!HtyZ$kO&CD%`Nbr z-p4L?-+x_*?-PUxiQmORzGT!78rCUFSp7VhbzbBy*zd%%@?FN@Yl^FdV1%xNI7=SI z6vy*MIh2L(43lB8a?9(wvryPXryKp`LX`EU}B zpi&7o92L?;PftK*Jj$ZQL^KfW^nh6X_po=~z#Jo{)xK6^mRhbH63AtnLX#W6Bqsq# zX>n9~28me2zwQ0HEIcvWI5Sg4)FPXCvg}?t=gBg#B-mG@7QZ`fSL{&x7_hY0eF>Kd zf2EbD?|=SSIj*zTp%#Wf7UVsfN})O-?Bdr6V;Ul~(0qZbdULV^;00-RM%FWRw~smF zb8LC{oXc$`X*%Rm9bkE=N4M%=a`m@NgCNK9C^+^|zwlQyV3+2=ScO zYcT>h$rK6Q6iMsia6sq&}<^Gi)IjsKPvUW+hM`oDf)NVNuB6Af88Eb z2lURZsiXSv@X*u&ArY0DDY5d}zcs#oB14ga9JbGkhqqN)p*H>+jy6Xkh@Z__QpxC; zF9$B4zvAMmcU$7u($_s;38Pi9I0di-ZF^}tf1A1y)}k^_xFQ^Vx5DFsxJXUalifQ1aDf4sUwtlpbc0Jthd| zcRCR@QK^#n2|2gYFN%wTa5h!`b-HOi()?$>_iGZx1njOT2w;e>D>aR6fqeH!pLa*s zr_lP*GQImNTmSE!!&!&3kBDNdx+b`ixTV*%x$g=fs*VSY11%uq@m>*+U6yZuI2Yc; z$eG3?FkcT!zorBo{##yIlyN@;4N%zmC>A&c634d|ZA&dACnD}Yl)@ZX6KRR#8|P_g zCuWG9CjkxidhaDSfX1la^~3{RtS%`5KLul;#+sZ1%!;ZUs~&n_mb_Yd%FYyJx~+5H z8bR$+NeBTY$lvy4JeXLOwdZ5E$hz2Ek3ci1jkxiHj z=$OgH!rLIk2#WXU1Q|B>l<&%PZR5gTBL`c25}1Ekq}moN9@_|_HzOLRrsukEY+8UxD0c66 zJaZsQ=WPNyy9`G&+*$MTFAEB%JFnAfk;kUnCI)DCU+Zha1iuP%fjA32M#0S2gH zh4_))-`{LwzCwNcJbCaCNI#)ZQ>9MqE!DlylJ_(MyN?a>Zx4+UJg%5)EL2C{7h#g^ zRYrab%8-SP8$!&qqt%Fh5rYw!Vc|J7yT9q%uYH;}&j*aVn!e8qA1_5-%e+W23C12x zND~u4VBOGS4Yre=>sCh(-};6z-^wv<<~OONh;V?Dqf$g+*O`p4VCl>*y}?Rfxt?obf%8cs`hRN$1zo15^NnQ}BIuRQ zWI>J+UtYBkx=`V00g=-`KYxlnFU@480h(`hPNIF~+ylYW#oNj2OhX;+VfaR2t0+udgG z(~WFoYPhS=dH-0E+2D!JjWg#b%mYE-ZI>kYyI5Qjv?=BsS{R?J$t~@<1u@TH(sD|_ojVd3o&AqK-86y); zFjPSyCMIMZ2Pc%mRg83Q1Udl*caU`O)_Jkn>tQ5qsLIPGIZhre6&b3aqRtn3p&{9O zMCuK$&guOXKYUz&U9g1&9px-~d^-=wdZ?uYk}u*G01A>O>VL#(LVjsLRp6SIg7mbc zNeSNY0Y}hx8aT8Vv#6~OE*|(!Sw7E=??0l0tZ5`4QkDZFhFAhrfi9F9$L;Cu)Gi1U zg0|}@2j$^f>-taX=+|--svQf@KsPqV`0myuxYi}Dd!X~GBx)SWz0*jb02lgu+l_yy zs}A{y8XG~``8Z<1h4l4!&}6| z?Ax>R)^~H`c^pCH>nWp-L<&%}AJq7s+A_?OW4)@%ihu8leXSk;jsV6AC7Ip1 zjr|a!Re~@hJqr4W7E>#xcT68$qYq%kc$P%+mQWLcgrk?eOeD?u9x-7!06ntZYL&_d z_BUc)d@N+jJ7vEEQxdVc+0Ln!N` zt9N5G_fCO=VENFO%%C}IXozVZxG(s45?iLJ-jA zJ;fo1`_ruc{aQFxgkTONl|RXw-|=ccGV9@ys6{55M@zzNvF-k`g>l|99joVCDMr+q zsUSof@eKN9+`D+wDOw&i9VGx9wBcM)bLW*TxD%YcW3n}Z%6DjHjF!v;&P=Ulr^O`Uak zDPL33G~+$(q6B>$I3i5?bIU5HGCn=>%i^f{V^_hsF&c(BXz|fD_4WHc_Ju)By*scK zfT(baLv8+P>5P%7tl1ns2t%1Vm^dGMU)NA;odp@in90Q~9CEI6vD;uiB*}dQB8o|x za=f6*ZNZ_+MWN?0KwngC35B`~jI;@y!Ec%Jrarv2p-@{_qBTT4 zyeco>tKJWw{VXVQ`e3RE2t8u73@!y=7>(NL@Vpcqn&PPV2x@H#JA3+9H+l(dQebI< z>_FZ3N+ckpV?bHXhsEP)J`fm!!5AVT)$Zu!miUIVJjO7v3AUz@;a{WQhSYLe({b{i zdRGh3ZJb9+9vt>AIr<4l5M4&2Pz7|lxz^a3hTDfX0T}21@`sd>txx65Mg&TYH123M zP9lW`WioM48-F_i=;L5R`;fPU#L<$uu<`*DHNv8S%jwWA1*PLW+4EOhu#&+O_6`D1 zGHhj!nPswrr|OMvfrZid^C0?iZu#%mLrcZP6~RM7JOi z_H(*=764nj_x%nf+r7^~22et?=7Ii`kzw-|DErh)+zr;s=%Cr_e9`gJ=&v5s*!BCz z_9*EvzRrX)t!P>xPpw@wb{||?vUn~|w||qF;4{QX4Nmm2)s?1O zW{LcG6fG|o!@+SF&C^F;YNRG6pl76==+gA{#$2$m^4mKqzFma@KXmBG#JlEBT z!bRyRk3Zc1MX7<>&csD2UrlTQF!JoYY(n76<}a-*DhPk5Hob>16Fj~L^Isk;sdX4I z1%u_|DU`o16GZFY0uT{gudR>kOFawWqTy3hQ(1m(osgwd-!=y%R1EUgm}))u*J7)A zMb6RD9<7WNn3W)kZRtfp-te;8Vy#&ec}o}s0e1Sy4=b5O^dh*vd06@CQs26jTS@3B zK4wujW&hI}0W~HCG`#Z@Vq>{jDjRL@I}wnx#=}EJtX(+^U+Z(c{G3!JS^yv%iIVkyO|*0|G6>e#3Wby~CA z3%+#4KcbXU%pUH1siW@Oj!g+_>5QHH1j+e;G@uprhfrW?k%$&!)31jL)$Ml*U=1w z;`w__M#Z_{^5S(0`UnH>fgmpmG_?$%9aeVJ(4k^w<;H>;_h~X#_9I+>ZARQ@)$9GU zv+!T}SM5|`K*P0K4;-Z zs3^`e@Lg;m2=dEjXuBL(`yl`&GV2nCdV&l1kt~FTICGHbvw<(n7s%HGx@M?-?*o8U zd;%d0Zr#56h1K-WB0|j=p1O-oh!j|;NLmR8w2n|HgXKmZ01_NVo&$v)>hK1>Rp#hO zmRhrwh*fKc!lsQXe%sm7t}n$SiE))9LeQl}#G1#i_QwIxoIy zM-Korc0`yVm3yYHjVPH!z~^H;4rP9|1`-yN!C;La+xs#Y2yFR$i~`z_A*%@BSUYbg zKksrOmqjXK;koaezP9=I5ppwPfQRt?)$4Ldx3szH9x2TRBru$7KTl3&QKMdXm`Kin z+Jd3?bQDMHYVUw<5s-HRc{hyvXUzIUj=8P(lmSO(Ua_wq{AH*dI?uwvB^)2a2><1KeC=Tsqj3pz25k$Aqc7^9`HTvwOXIvsWTWwEPdI5q$1#iNvEfXu4 zu{9YY&+&}VT4+`ZY@|%rm6B3I_tOAJ(BS#0q4w?_)^srx5+AMg5IBx7rufrLEzaOj z((!2eLMXoGRGh%N7l0q~(0;iwMF(aITf}LdkGgj1A=q)44+BT7q-_`oGJM$`+I0L` zc6(ncx%MqRwQMq~>*#_JOiE`har~*rt={F4^`|GOAfs=MG4~WW*6Z2%LH^r>rCxYK zFmSd8>TXF_F4qrFx~@t+NUigOo9j^%z<$rVOz$&tVku1fp6drXi2S*=aI#Ig`*=@w zjOdx)$0&Md7dH}U5ZK{)sba}KeFzwa;mgK|c!*<{nFkLl7MBX=Z%Yru0EG3PDCUDh z5`p#nT@!B3xbT_*pEL5bQ>OQM@hXsh0v${ zpX3@@Ig2Aw9FDuP7UN0jR>gP(V52SJ3Y6P>3FI+dC@wXe=x;yq+f6KlY550(Q8QG8 z2NMP-gFa?iI8!4-e1QMav#m_W7owK44psWZJwdSADQX z18MVsiovajJcpo;r%3DabY>L4YYTuDy)4b*K{ack`zzj2nT`(r6a!c-ak6%~| z&hsQ}9vwufJX1+rJ*Uq-+zNqVAoB5$uIwOb+Aat6Jh>Pfp>3cn4R1}p{9&e@SV?l6 z^r>?DD-Z)oW!fxrK=?aZGyi{Z=7MBRGu<4r3cJx?>iRN;B1 z`}P8?{i(c}wi};tQ{(Z4eKoTz>wO{siF(HYO5B^^b9D9%Wnlm`OM%=xdU=9bdtut$ zMEE!L)IBJun?J^63S7nB5iJwg>_qE-2iNYB{^8pP2$8h-?)@IcRI04t$Ghr zm@s_O(c6;y?PL^76iX)BY<;ywTv=9bDt9x%F;RQN#;st7T>C;zF;od{z6L333*WHp zOA@mc&>J8<+tyB0`nKbbcgz2d$+#1AM8_!rZ+ddcS{RMLSDS}E=w)#|R9jV=f7(== znJzJ9@Z*K&C3hcg(vy}k7%wg5?)s)ssM5)c6Zd!}toPV@AoTWDct@eF1Bhpg}{%mA#*ICPCo@6P2p>2QHIp%;YDy}l+@*j?mYNEce)!akcZYI_A;_rX-M5~x zfpsnmcj>{#_Sz@H8z*tUJIj4W>zgwYV}}?e-}U2FDsH&v{{%r=@^3`|=m>p=305PG z;BmOzC*8h_3A5OPQfx@>muXX%Uk-h6)@UNeobhGx1& z$t%KDd}z7l&WHHm$+z{Sc*dcSsS}#U1;Z}{AU=7kCxaQ6<*Tcoi7tY2RiLV|$qTmIx`F(7MV#8CAyqH}F79wd#42_*M7HyD=hAJd>-vL4l@A zoGg`|g-s`JMm$NQS4G6p`@N-c-mp^9ewN?TMR130s)<|-(xaGAYQw`Oh6<6!X)r->nrj^yg!5`G`p^*Y)08LI!!ldicGIJHaTnk?k>W^ zvtg|_;1_2rIiAe^q5AZ82RFebp%K;Fptpf^Q&O@8sxR(U#gUAgKgK;kbULi}r$`nH zMNe5OJS2Wd3}DTpI%UA*7(NgHxUC%3v3wFw;W1A~N_|kD_9-Jcn9Zvpfq8&1$hvMb zfKNUiwZJ>D=e54SybuO&VA|vCR4EW-Ab1u1ik?(cm^Xr}mD4zWoR(uSQsDD-o{m_& z30)^ssGKyPHqV~I#;CWCIgp52JgJ~Se6pD3iIC1b0lHBC&d#_T_#bsxr2f>HCS-&G z|HL65PPn{teLacHuPA!x$u4x1!jMZiPi>-k_YSCPwbjH zFnL~H9L~(Q2Je~+_Kv=*f3cVf##LI{|9)}lW8m2A@aL)IQ{&1)-J(WY!DeXgQ(=d) z1Hle9M~R5i-Va_5u-4TdkU-pr1jI8jI`rwk?R-tW1*gBkS>uW@IxxIk{^8a2JaV7cnfxyi}&#J zB-9q@eSnY)XU4AB$B)=V@d34JLXW}oOEmgob#%1cXqr9Y6ITMhx&>>5seV=Kn>#RT zEJuX2`n}PLj8Dbrli&6g{In&j+CXO{liI^q;OTPwXkeuH`}50#wqI|7DxB-;OLJU= zFnarB&~)a5lf%(1d}IH3PI{L0!RHNSfB&|gHr;D8t#swPYld9)$<%V0<3v!w`cL!O zL$qgm%50nbeEIZfNi}0D3|2enBy6Ec6-m#sdS5YH6N|}8}Ec=HFcB>nXF4FgF8dZ+W3N+O1`5%2K=f%f-phm1& zH^i_iDf4q{@~&94279-Wegn*OIC4yw7uiVTxcezwVB?+Z?L$V;lr7eHy zUhZ(AXF6;VOk8(;i~U>JEzW|RsmxYUEgmt<4eIs3yb}1*oqv{XYw|Hc-tthOv1RoJ zD?!4xCNq3}W88fQ>FW=hM6X9Dr9pQ-VNXkxaHH?efxTfI8WUGdS+0rBT1Uq|m_yf= zNfahFIQHDZ?}QR|t^_~os2A!!3-T83FXL7%=TD3_fcq6R7SB z0Mn+o!A%LE(mXFQ)E)PG&|M5$;?$9SZERUOHQ{4T?`$5g=_T(ZG2?-x!0a6*Rto=> zWpBt`B|-hcn3f`n#h?CaoGPyh-cfX|{N}21bjHPh;OBLRm44H!hVT#a7ieT=wNnpT zpG2<;KW3xOSrq5`t|O~W^Ee*qWLwb_8Hf!rHz7wSK0;gNR>mS7VC8{IHnyM4oXMG} z)Tt>DVeOcPeaaGkdb35nR%U8FQ)U{{b4Z2)1P{z3$7p(y1<+Y>6(}TF+unog(z?Jo z(+T7sPrNhv_w|7`z~6)#V%2r-iwrsOv{#auRj7mVw6o)ZS$xsP7)?e|hmH(Orx>8x z@C3J_7m<;F7;xutyD^rjz>cN(b8HT1#=XZbmN|n|RI+i^cVDpIRNKBd3TW|g1heNY z3UOMy?T5={J}eHV^4l*MnHUS5_lKKagfFKbAi}0@v!VVnKfD7A$^Eygl~m<5S-Jqg zfgi|yDt+a@MA&(n3kAxBKw9+de`%2e+;~c##{LOnQzbo6*k?I(J{1T@O&Mq82|w$8 zjvDF3YQTH#1V!F;8DjOx$XkYp@SjC~{{0t)Zi*AIR1#qKR~lm)4Xd#C=p_j4}$XwHik^Ys&MT$sXT#=_n+X)v@S z)9U&daFsSempF?Z(WSFMTRTk4taLK}(kaNOQ?P}=md&x;^)A5&wKd_GZ4`$c2Y`{g z-P!FBwm#plmIizspZ^-wG9^uO9Ey`FtyDF5?bAje#LJJ7fSB!Xya>h?Yig?lSdfPjL?tyX6!dW zLH8_kxw>UQS+UIZ7E9)~W-p5>a#i@5Z{zDdk0GHQ7s@YQ!)%hLVqS9ueYKP`66uxO zZ)#A^GO~Oa`#Ot%#kvGA*?ySOL-j|sjco{m6ZWQ^OA=T=Eiaf8Wt#RQ-<=16tWq;J#r-(amgz${+G4k7nagl7*n&e!5lrPZzBWqRJk*+dX z90-A7?>N1|X`~)sggMdv#9qEz(jG{HjHR44|T+p?ol;dC7c zmO*4w3sxi_yi{f z+FH^W|F-6L!UCFp^!lso^(n=Z`iDLNEO-fttye*^$?>#h&-5P zIb;O0o(TEQ_0X0QW51}?m+%mY4+fqETjU?JmTxqZyunLO>Nf*^VQ103Q7i2uE|mmo z%SM5*HSXm~;KiYi>hn>V$Mu|Libb~6n6Py%&IF5v$WCB%#i}FUNEkw_^@f){M*?&z z=ScfqGAx4vIPJa-KN;Wup6--{3ZzK*;f>B~E<#bfn`+zTRE7 zR6r*8-^|UP5l`1)Z(c&8K-GVq(DnC|i68ACslZ_iR59m4;d8NCao3(k@14LE(puTt^+G$fxNtV7S$>E)q+_XbxSAl&C<~It0t!Wa-QK5i;PtD#lO2BsAq4K}pTld&+1C)JE)xB!TiL1w z3Osd?c#2S?&{X8iiMZ)wApkJAtzue7Jh|h4$EYO9NWqKck^G10>|%WaSbK>=UuB?2 z`w=967fUO-^#4YN=)VJflwMIC6jgu79a^^y9bfLQZ?TiSE;NQr)j2FteMN%Bg(Hl9F8?APBf z2TxafpW5krp*0GWPkkqDzBM?GF&_X}`l>aaeNO9nordGH-vPG`p0%@GU+5cLY%Gno zqJ>tTF0&Sw@a(zB03AZxkcKAnISDj$Ddo~5CMN2>;1&%_6v}HZ--{%NioDa3@~ZLi zM^O)pnuFEeRZ&D&!=qNNs_vg(d-eo5Z%cGde+)vH!aB>%EZ1|7vqjN;?M4GGN$1qP**4>jBhDa z#h6pno>t9uKzC}wQXm59@KXov3+H`Xj8lJRI1SVmJH;1DCuxX2eP&=>c_d>AX)WmS ztmI76@?4G`LU9W;G7r#Pqi zaoYXB2HTMaLd1p!&9AH6ucPfHswqwb{sE9J;l{nEX#=?lz5P$?$|s8%$BQ*gx1t_r zP8s&MP3h=KpwE7LeL)|P(&uY&HD)9`dwCxk`~A&WI2T$tU$whTTR3}qz0-6(iQ7RI z_?<9Z?M(SBCpDgwHtYmRo;`0mKWAw4rWkpe^)%D`uz*eS>iNR;lf6qTyYohdi|=ut z(GRa*SY2{8ohcby4;UQJ^ZIR*U3a@>i63p*NDQ1OLYBJxmi8o$YE&hGpX-0B7y7Vv zbX9lR?(G}AH3Lxc16KzKISpq$f8kBT_Z!%vB2`%hde+K2VetA%Z(v9sV1mC-qf$~r zA06S36W$YxqWFxKKU%I+YGg`H=#@#ah1J`xmn*|6CSWdLO?(3QZ?XSLJ^H zHqsc*udgBv+!vDkj(9JYhI|isubeM(4bG3Q|72bDU0)^`>>o*K;aS=UE)SCf8Mao7CO%UM?`*CzG*Q+@|pSIc%6f3@oOto+tKUk_0GjU-)8 zkzKFth!31k;>>@YH=P76TsJUOz=KS7sBX5hm=V)YENhG;^AgR6R2AKxLhicM?Ro-J zaUVL0-=Fx^E`chX|L6J?GknsA`0dv4x{JdpMzJUAyFk0`ec2mwzR6S?PZHTb-%}D^ zS=?_6n+HkZd*$^I#B{h~og8>y>3Y|oX|J#8jNxKh5)b74OKiS;MCY8h>4>4}P-#BN zWHVIaBCGL~M)FTf(}6L_pV$4ojvdzYC-VAl=g_0Q%W<9P1K;_9aq0ygG-G> z9He#OsR%9!EwUhoGyBwA4P1cFF1I)(Jw|Gf+0Q}HS1C65B zpk~8igX*skw*6Hvn(OY_*Vn&4`z{p>`JC-t{kgWH1!z`{JGxEVL4NBD;y0#Ro^=-f z+$Pmz?&vo-$fb@P!Pn_qSz>xj6Jt_D%s+HotL`R=c~kjofL@StOq$AO^&z07X@L`c z`qU-fkVCeFd11$N{(#+~aN^W-*}O*lA;B-rL>6IEC)#Hs>LSh{?k6Xs@m6(&K5oul zbf9L*vo1-`j$@Uoxy=d?D9Y1Vcc^wv9Sh$%?mRj+-N|e^vROEoWNTbZYy5SYWOwo1 z?o_4Afx&lUz-xlDuU6`7J}nd$FIMads{dX>8l1(AtgEaHYiw{XRWy#gcvuBLV8MX%KB#(djuUiAEg4jlC7Z-a~1+?*&E zfY_NcyLmmnu7C;8rYS)j;gib#k5>30X#}u+8I(;#{@Sxoo6m^vd!VJjE^|G6$t&WM z-zxCh23nFmeV^WA)psjHrQ|q!QRh$vm$Y$`muUGIO1)PyTn^SfxF8TNn#E!pCzxmS8`I-caXMjeF6nlO0cL)>Mgf41h5&eH z=HBZ<%UyqX#|y64?|{0=&O5A)KN!~f##nUPjX+uX+^Bl)xf+rKm(V*tK8_FN;t=2n z#(OmV81~_iOnve+u{*vQDyp(M?A_=?B;Aepm0W z&$1WX|Fj4-UE6t|mtL=xGGe=i8s^JO*6nKGu`l1O^ ztsG(1;p;#^iNDa3(2r$%Bu*vSJl;YHYM56`0e=R0)Vc^U9?6HT_!D@ zRob04yUm&dNO|HLub>O1TqLRY%5>%UDq$D#*L2Y!1B*CSJYI|d8e@;4zT`3*Rc z1%O2Nb0rKMN*LbT`9AlKIX0$utig7cj)Y0MEle}*-I>3D8z$bW-@(=-6l#ohC>Zxw zbILqEFVczRra{K+X{7BbVrnzB3N_cZqjD)J723YGOh3^F1EIo4_4n-AK}NHd3K{4} zT{WZ6%%@!vkgHw3hYVNK@2}b#Y{PCM^e!NkJG>#eeIUeucF8xAnN>RNOj;?!L z7uD*o&M!n~IliB|Vlsm} zOqk@zHh#;&a-Xg@PORKEybISPH}Z%RTNS8+Q6@zq)C0r{a%O%j0m2ktD=n;E#9*UB zTdMJ?)n4iL_o`MGx+xb{wq8_A&f?rWxPy-u^L_cIur`SXuqG~_?Vt0eW->_0hCJ^- zsIMD5|2{sF0LlKM_n5NW)Uv5+FP|a7^#Zc$ct1N=4yKS>0tax!uTw~%CGE*toT@-e zpm2hSl{H#^Bs$n8NpQ=g^95~JZgp8LeU<3rL7Od3AUClh11zlel^U|>h)}lsULALe((yEhu|l)bxt zbdO+f*UA?7s{g5$W!y%9$y?D-df#b|*Mw{a*rA5jb2=VRuepln_;B26tr(n+-h)}# z9y`ej=>+ZC!g;|k5Igu05m{p{)dV$P8YKItY?e68U-E zqyR-FK0u7UgD-!-hI6-+I}1#?Qq9i(D_mRLt6TEW@MsZ^*|#O*^>{7QA-0&prnKsnBZ?a_Mysi|D2P^g1CPFjaLHX+#B z=Ar+&32O)*s0Pi~K&HAo_Ub$%C8)~G`j_Lf1L07S%FBk^*_KodvwImfRYRnq+l0yI z*}0Y=e~_myPF?a%0d1kidsP+MKbNyMP~B=;gqoFFsRtRIT?k&GW@|#_$S9IG`o90# zRoKIO1c2Jzx@qAnN>WfpgYA3Zq)M_-wL(voC}3Qkx~3nczXF0f zk*!ArK*%HX-)pjM{Dzq=sJrd{S-H6SF(cD3{b65NMg)vh*U;5>dFX_TMG~X)5G$Zt z_N8brH~nuXj2o&_u5WtX0!35 zHt*QE*6I~th}Vy60ejazp}aY9qKFqhZ_D(G$lBB<9-%nlTSeRn7Wc4l`8^kWH&Wo@ zkfmD+9c=&N_eXlK8UM>O^}@;HS`!dxv3CCOP_IDaP8$lUiRmWJyx;NS&-hOOkcJ?D zz3X>N&z~0-Y}yO=ytK#h(-Hha3jsRcNL;=~L@Qsu#@j9{hJuybi8;v#aX&3zwXXQwZ*3tpn z)1muKZkh`S4BY9BPmc2+X|r9%%xQ%pk~I&nxsoZP{_q?^5MWU}swE|jKWBv>%lh&@ z3F=IySQ1PNH8oq%E1)mYBqNf}fSZ;6I1+8#n*{XuMDINj%n?}EDL&3$L=||kItl>Y zwEbT)18qcXrmR4dhMHg=7reITcYS{H-DmRorVPW`@Xg8H1+w9*|7^%Ow0 zz2MvzN}ygjs)2OKG}&b24ME<;;cAr3BBjB2?a55a;#Di2t35qLr1ahj1>)2p6Jk#Q zNdrJst+r$xP#zNjd&BbJtSS-Q%P31+e2th^k`#zBFy0X9uvpISOh#ueeznoa_c6iI zon{nuV+mS-ow^7GKq!WiAhn_YN77k_MfpB$cxePA1Z9yHDd~_7L8%3a1u1C+1f)9` zX-VmBmiW;n-3`)6cP-tu93II@7ar{-?fhOr)+g<{TUL*~ zwyKuy6d>05ukm0VM4O4uBb^*xphdvQnBD&iT6HKx>qPmdwnzq0rVh8WvJMdEX$$|a1TY+1s;yeeY-2L;prY^lnhjC3HN#p7x(4WdYRpXcW#b64NF zdMA+&l>ug$uNva`eYr00+s^JRoZbvZl$K>^UnvpoTA;ZSk&28#-m!&GxtvCS$0qV2 zr1Rx6`D?~nz>lw_D8?!1=4gLffQaK{aB*dfgGw@=c)3_)Pwn+0I2`4PI>9TiWO?*# z>EW-HpbdV;Sbo4%+6&)j5-q(Vk;6l3_=jaj8D-Qn(_LU9dl?wT(>HQ^dUPO!v>@#N zcxl?a3%95R7#j8}=Ldu>r95;u3tz28gp(06^x1tc1S%y>E8i&w12r^5vM;-%xI_%K zS?u%w)Bk10in;Wdlhe|?962?~@HMdRfqU@<;>}oQ)s_g|jc1tvZUxEbIEZTL*Q+LJ zl7CDWv`v&kE1K z`D`^V29XZ5pE}%%YsRGk-l)3AvQaRN520aL`TD27%>VXe%*?1Q zc=10gP?-Mxh`qd;gQG#NVIC|gj&onHRZ>HSH_;6rM&eS2g-vk}(jy?2Ik9{_+oDuf zoP;0zdr8rtAzqR3Ps3OL{B9YyOy5=Uybn8SiF|4!iT(xRjHtna?tdNTeyO0TtyGC# zId}I-;?7b9={Hob>&>cmX+MD^b}_mL%+<2sA$b2usrmhTHuJs1;-wJ#r)}pco2BRL zTI&7+h63>SQUyeTy|^v3t}KcTb_>H*63NdmZB6m~u_ZU$Os2zq%ZV_*^e|97mAnP& zjDDGrU>?PS1`W0aqV+dB>GCPh338x<+_#9PcG<`mi`2P(5>$+C$tTU0R6c|%zMXB) zQ1hXe@>!}O1~m%g;gB4v*f&^)JqomSD{}2QqNZITNB%@F7GEUa30@Up#IgR_s-@8F!$k@+Fvt(-lHi}i%-mLUEk zBRZf`_BVJXJp|?bVzXa1O%R)X?{L=JD|dhNWUKsn)Yx(~Jgk5U3hW%~FaoR=>}8kg z(}CD%CGSBgEM9-Fxxb^WSnFV+p`Zi712E!B0*22SC-aiE_vB-f-`q?qMMPg7rew^W z7z5x0E#b}d8+mgrV@0)ri()%Z<#flhx-JLhw?g4xgkGE|cYRC>Q%ir>-mYk76p&~z zP)=P@@X7bZGrCvta?0@$DTbuzv`k;j3;(7HvwucJe8iT5GkikN`42%gUR43*;t0f0 z0-RoOKi&f~=jG8zVjl4P)(!^)d%YXp;R1aaiHOacjDEW zK(PugSPI_uS3w})`zs7to^(@tfq}I;aK!0bHviv7`O|_l=*;x#gB?CnQKUDlct8*O zLKO1Lo>J`{#%8pB$YHS&-BbPX^68%XLEztrKDhb9- z|6bDOI@+6f!s3D41UyZyC*wMvL;S6nAk;dycJ>Fr+pM)xUJgww`UgNq>Mfl9K$C#h z8BvFeee&euioo#sdDl5${RIQVn-~c|{9Tlr;>X-8S(Ub|Ystbz2Puo}dfhifPc@@Y&*00p^H(Ht!hgXFmBMQ#ED`?2k<;W680n zF5-Zwa33Q+rwNEgD^UP*;PMFBl2l(`B63CEx*rONZ)^JZDg4cw6^oWxz?IeAc(~~_ zUksx8MEG$9GjoBnkITDz(zJcyx2d@nLerKGr`v(e|5b;qb?&GyO9e6@N>W~Mi6EQg z1HuM(& zb4fB~;kSnoMk-H(sZqtTSqwGteeDTjnb~OlkFpnJ8iAH9GkXAu#I#AJx;_x()odqy_DyfC1P{0!|VcTFeE(mWX`;A@Y9VQ)Ah)m5cG6cRU>Cxi>I> znBzl)k{7iH%BANn#tiN+1YK{w%CA6^bt&)7mc;E!U9d68wor{FPH)cNOl<_TH!+%vWc_!=!dV?*Ky0c5J3T@qAsSZ&o3IBpI ztL!QdFga0fac`-(AC*ag>+tCzi6wu`BaGPs#>g(=|+oXK)Gjn4; zM&&e7r(J&UP4TAFal~^@K#$Hlx_h7&ay;LDf932a6T!%Fe{gjBGxBbXx|E&c{xAbZ z0ip@GK5D(l5Wh5o%~{o2T=WPpok>w&&oA#F9}Wavo@8PJpjv8CwN00A;oD8M;wTGm z@Drih)M5yce3it+-HzMOqq33RM?dnZz4u-XTXeT>((c*f9jY*}rO<)0Lu6n;PtDn{ zGkn(XV%A5fWqr19wa0bXBn#8Ba|tSb2$#}1R$yXxWB){JlJz#BQuz&5h_T9ddRh>O z_!DIqS!U@Qd`1Q0Ak`qw1S9E0nY7VoMcz)O&~((U_P-_`UfC|Z#tPcUngcv1+X^8G` z%oo2>(tC)40s5zjsr5sMuGMB#D-w3kEy;x4aZ=lQD&Km|_&9Qe2D+os{HlvwB)Grq zsIL-7R*AQIAS?l8Z2sx7TF;Gz_Dj6DT9e>7bP4oBQNnE4KC4Rrwwxx*>&eo_kpR=u z{?QPYthhJ!m9WVBFf zQ(LW6Do$I?jw#1~VT9J71ju@(m9X<4Q|n z*R?Thkn+0(4lCMNy%zV?TJQVmQxV~tHZ<{z)<<&VzmJIC$A>}-G@W8~vX#zUFL zz6DF9=km1Y;?=|PvXhR>-9f(CjS8rZ;I1#y>w@2VB8K`gG~*T_j@(zx(AC#Xqu>yE zBERA;w!#ETZv7|UX=Ll-jXYN$ancxej!(EDx6HAIewza~YY`jp6b{0DM|%=Oy*yma zb0%N-bm$DAp|RhhegNSu6yF9gbPPBax|NH0$qrLUM%UwiBBXN^SF%%w4C!qelu$`hRyjFvwbOqp=Zdajia+>;wNY@?8DY4H1xV)=hX-X=Gc zqO;T19w}Z?%2*D?g~9b?LwWo*6kKfo@N52Bzj)iIMuPUkn~QV~hs$_;OyW1y zFujxS=Qu--FGTkn<#$eTM!p{}`Gy<^%L{hg8m;Do?CIz^y_U_3va^3>kCys6yTcaW ztL1L%Uub_bnhfDrbZH8~)p;(WTq?EcKFe_%r;SY4&(b52^5ERh)w1N=eme`w{K-}F zcJG@T2O(lPQvJ#GwV z@?}^u9119potX_ePI#S6ub&6ytR9cQDyfNmA^CwB4HO1bv*GI%FPYi{`HUIZUX=cM z=#zg~e%w=f7;n8nBGSEg>+=DAt@uqB@$!k6xGmB)=qc}c66{3_5`c$sTCpZU(VCU@ zf`4jVj{(;T0F$Jg;O!SR-Uu5v&DY5ir(1Lj$F87Y%wIo`yIfJrZ%uxCz&j0)8*8}MEe)2KMC#1f&E*ex_e_!XON4a6 zH0RjqICoy~V-yVP59oAtRVbZVbTUBn-I*wBIX&cXJ8~WDeb`reh|IXvn!fP9C3paX zR?XA|1bHYt0 zeU>^-O?oc5oKRorr{MWnFyLS`Xj0Su4k&gI3G8~lib;9DF&2`c4Bf=S>Aa%2N<>`F z*RC$2%YX*3Akf*lrB$+@tj@ z!le~UQa~@~u5VpQr@83qvQa;-5L|jRA3d~FKZKOs-?(^>CEOj@+%)jYO{vEdP}MuX zVgLfXa0Vb0cyw`wlFA$)_!LE}u6K6}7Kfw*UO+LlhW%(nLnC5bp`lkXai6`&J z(fWVZ8o7Z{69liIZND$8T-g>~>uYZY+~-C@4Y{bk0Oa|t@2eJ=i6ZQ-1M7-moc4pI z8g}h|qhskw=z1%QJSZRaNcOb6?Gr_6ri8~1tE-IK0w%S|b6l*{%=TjWnv1s)-D|cR z1`EfjNi?qHZ*81JL2V^ic=~snN4H%vk9SEU-doh($3Z&NgJ*b8+3xbg12tHDX=CY~*rX(RnMiBp9q-P?Hhh6qWKQIeE?+B>PIYv9=jf}2&PSw4d3G&R{nVB8!BOzSY6ponxLVlCEutuH5jJ`+11sB+wK8! z9e^W$GgHpz9Z~<2Unh>roqyDX%M$`($!#f zxET**5KM!P-_FjU+9FJfVUv;Kwzr@bGq$6Bk*tJlUcOztYJI$~txub}UwIOZ6`N}w zcFP`iaj9Y7WSjxF=rFKE$OX{0_}op5t!gG32y4|(4|k@1YA3j9HC__NEqsDsGMJFG{>O@?^9Eter`gR*J7rfB7b1&&!lXAR+R=nIIVfogv&<#LDB~l+kLo*ZQ2KPHP8cIJ{o5r>KFgQpwTrP+9_xdqNE5-ZPcn;t@1t$g$k90x~sN&~n zj#}+O!b1YRL`rTphUF&~jb5eG`cx41xX~@eLSaOB8^LK^%GxzO-KUjb6UB&VPiE{XLhil4Sd$Y@*rq$GbrVv zw(;=W&Vk-i6P9y|Gt zIDiX8TkxAmx<-02sXz~RAdFk`FmnVLrnimi(Slix>XZE$0d|$#uezt=R|4I2wy`-7 zu-ij~7@O`p4ezIy4ZM`n#95>b{?Wn~U;JW$MUD~c-4DlVHph}h9`2ml5fk3&M@|#g zzb0RU+TucRy)ljPfAEI)l>ITvM_u1f1Ra7<{ne%DB8y65R# zXi?|zZh>m>qBdm(V^-qs1eXPP%317$^eW1y z59YWqeJ*$pNTv7(6`kCV(o^ttYH>jHKpPHtGizrlCy6SGEyuMZ*T!hb)AmZ!ssHF{ z3OfDQEb=Z|CH>H8!Z~qVXgz=n4dB1MmYg92#!(r}-i7xOc#X`@Z`lo3y(M>AUxh5_ zzqU&KL7*NerL$n}_S-rw8Y1`7<>v;DBnXdnBw)-D{0~)p;g(ljD?M`jha0B|X+C3v zAL-ZTZbOkix2(g6m$+XwWVK|U$xSvFUPlXZ&62cI9F&{UHh8yYmV-w#SLG6>_-)MuU>qu!=%tS14hQ)X zp2aupA+h968l{>uI4aomaU1sKVn3)#N5r{}W82Y>+}a4uX2+S_N*5nD#s_AX3iL^m zVs{U9geqS?Bd!0f#Ozc#aqGJyhcZg*bj3CT@ctT#o`jN*<|XxnC35kN=nHTqvW{ZdL z-`t_ zy&R^*&;gKQ1jTY2M$$1=0ps^l+c=gq+ z0l1a*PJAh43bR?JFwG^ft?2flUO{Q z^g$2j=OgOX6Xy?jV6oF)nCtb-iH+<2N#uP$wK(!uBy3j0``?qX7^pmf1V&iC9SJFO z`mfeAyXJVHCqN}C@9uDz1-2JEWf!#>jo==UQo+scyFldR9UZQ5Z}Tb!S=zZ|EsM?X z>21G!mT;#h-%+It1;&&&Lh?~TgldwXlzCqYMO1lU9dw@r@OsqG!zvbask%FT2R+<@ zPqxvh-9DG7bcd8tY`uXr3!c2xzjZ1_SQVGt9|n2fdOQqcJ%gWoKTmm_=71px?tW9F zuLng2c0Q?5)%<+BLt6=6n;K+R>_cMiiwe0#&bXky@pC%ZITnzisq$384vJcP>=t?Q z6}@j%GZfbA#!|1dIUY~lbKDAbo0#Uv$(N7$mz1#Gj)x*W_mv)9N%hfz!f=OvwdXab!uq(DSOqn0-u1foj?e5fEsFHVH z7*dhmJhZ^Q1lhh&k@{V<-xP+&O%&y{f7c+Wo!axVw!46JxE zSw2A%->4fl%ea#VwRKI#BCtYn5hx(i;f$2zq-GMpWGi-ppiW9nM=lXO^zAR)>EF#} zSb5cgd?EokQq7{~=pl=mJ#H|=>y*Py#dCw;ab>V|w+L`li*vu{IYYqiWD<^O2f6(T z9RSqo(XE>^>@$8ftL6Vz)u>|A&Mgji^bGWe~m`08@m^-3Pb$+Nt@2ZtJ}twN*2NfdMaemlT7m*|f>~P54-aNZm^u7Kfp= zHA}-)Y1CNc-6!KWH#^QpXC=eP2U|_sECBfCOx|%l<3~Uwxpdz5wMPOKv~pZwj2KQu z0qF?19FC6VBv7#i)Y(2PtlM}9OKwN?W(atm&rG=t3hE2ET>h$UMI%0K>$GXGeV_p7 z3%uJG)l)XG>BiO@@|KfRQE*9nx6~yz?X6-t>z;F^YIL!%RI-^FU%aB@E5(Zhj`OLr zTaU$#3v7D+wbH5ob~LgtwhI!&J1B=a1@{SrNgMpma@yBy>%?Ak--{9NB)j$mk<3KkQ(YyyV0H2PvFbCSd8y>JyRimm$s||bkFo$ zrxTj4#*N3{{v?b*kh--z=G8XAmk+G%YS&?aAs%MZL}{$ReBW~gTS>}KcRwYj&H!|g z-o_;dBr&b;*4lHDlJ}CA-Wfndjs#xL=KR=!Q_q}b!P)v_N5B&3M7}FELYSsH<{tnIzZz7wwEDGv zBc*v;!~_H(_fxnI<}J%a%YO1PL-2>pK*Od~ltarXwaENV#_R5qV%aXQ<^@lsTyuNo z=VKd_ytEs$Bcfjl)pDRa zpU%R+m{Z*T5vOj^DO^Iyga&*EGGObZ zBz6|Mu|GCd5FQ|f#t}ksw2t2lMeA%p%uFf_V2{B}6h*B1?SyMR6G@o@YAy*9qBZN=bod41KgmL;Z z)5pFT04%M7_-BxG4LiR;L~J54mp_qDCitXDziZEY{Bw*1gwrq}W{Wu?VBDg&mrP_! zc(NT^-$5^k3lr3dp%kjNOFq;Wt<#eRCYltzdNey~>3lml+_943dRDSf?{WHf_=jG= zmk+2~FQP%@KiceF70N5*Fwx>GxL%DJh>@o-eb5B^p`@_B{otbvqK${H$9_+$D!zEs<1^@6w!=mI+}M8(Lg(J#i#w&`O+qUP_l4V?-A*3 zUpy~7O^yd{HBQhvluJi&u(NvJT2LWFCMS zoQnkStR=A9`q$;aXKwf8%jXHy*TmSXEofLa-gEpdn^x5KEBbf4jj#$e_ST&e{Xj17 z`Q_Vz$j5&&;@9~wol>>s>#15)kbp7lq4(%iuJik6G8F$lrmq%tJ12k)P~T#!-nUi}pigL|*ea| z=B06%djs22zXhs=<^)9Q%_#M2Xosrj_nj(vjkRdcieiK)BpD$Wi~mkIW^cFm@I_t- zO9;Nxy%MR7HFFbVdZomgal6pHmE=6GVB^5^0wB<;;SBsFb!{Qlpex&#qHEExZ~kwu zCxTpcpJ zLYZDQ`**Xc4ZJG4;ym8W0gaLB7r94KqrRA^o$KFF9r2kJ=P148OR~qkDpE0fyVV~q zoZ_-|m4mp`Cp;eHHn7|I+OwDQB~w_B=aG-V1k}27OF_-O_mvCGrOLZn4B4;LvSmVG zGf>EC**p6Z(~@yhifdX|Y$?^N#clPQgZh78hj3}%Qqe~cf07^x43fF>ykdE157sZO zqtzMgKnH<*l!mEqj{_LRolclIF#+CFULB`)*$E#zI_M1v#_HokrAzV2OaA>wiu>a- zaU^Lf?$0tFFH#8Wc?vTJ!vI}O$(SxY>Nyskxi}?jUfS8>){e7{d1hA+VUi=<0XE@LS7zg*S0hkzT9Q5tYOWkizPPE}x$?zCPx)i2wE-kn|g%JszA!R)sz`rb!u;*ioZ~{t;TKB&U zLd(pWDxQa`83K+Q4{20SR`cN}?q5v9{UI?T5@xu$;aNr>*C0jo1)ES>f;lAH^=+w` znA5T!okx#vhzv#f)^`+8c`f&Y^znDuW=2w_G)uN~+cGeiKDU0J!K?>V=+qNBF{DLG zEa2=qGIQRe_)ptN2fUX?wW z_>Fxt3Fjwj~2n$41IhbY1PXzYRXz@S4D3Sa5w2zO+_o%AHUdW>dV03J@<9 zIG;Cr}5)^skE;q$$)O%=>Yo$MXDh-e~IT?|51nLQG#`M0c42ET8TNz zYCIn319^Pqu#Ud*rc#|&vPw`XP*E8a8{B#3Ujg;8y&h*_Z2jPy8IKu$7{cY?oDh&X zoTGuvB`xYDQqpWw@=s;Q&7#Qa?yE0!w32!h{)PfyC_VO;5{}p+m$yJ~c8}P@y~obc zo{fwEt2MYnNLn`0iYyUmtyz)=(8-D5LH zPl_5$Xjh21Z6uE=OL054^3bcevCI3E)wOS6!fx&qnmgi9>OYACV9p#Xh?VmX1ESbk%F)o#|~>tEtwXNAV0|Fk!VPhgFq+2#47xG*8N;_!8%SH_^tq$(fxjTWDh^u zXji%SVxhS#qx6tX7m?!SQ9aJOUfo-1GNmv38+ZC8C+h`PFXpTct$P?x$~K@Im@LWx z%F6-BSV4S;I!)B8+Ud{Ng=`se|KIKr#I8})Emn^#)CKywUY_(vY;O<5g)V-XB@90g z@wY3jnbJlB0Tf<3Nm=^f$gq9y?(3{+1*)VFy@j``&Yiaw6G6KZRcR)0we;LtJg~J` zzzDkhrUdQWf`|YkhP~WM-<3j9Bl3aQ)F15a<;Fw~HRumUA9Et-r@S*j` zV=3)1&Waa0ssP*tpJR$;yGztFrcfopu7NI&dm9o3s{J! zTmedBRmo+3yfDJU%c*vA97v_Ku0((?=KNw>C)2?zwNH(C?wf`?UvRar-Y4fMpEZCMKf zCow>uCPNFZW@dgOQb+@0eeF=J8x~O5uOj#gnJ5&P-f-Xrl19UfR%fbM5K}F;6cWKS zsbRa3k;}EUBA6(8qMd5mmL=i!PWl$`i}B2o*tw%{d>%59qgV4k1rVuc-VWvVrAh2t zj&k)@XQ6pz+dl>0g9)`xNVvP)-TXQ~%8m?NyIf9{{sxwquxOD{G?} zpz@!KijRaACy?i0BuGUOR7wn=%`4qeT1%hmaD+Yev;V*-`5&g`?PtGI$maTziGQ9j zy!w>_duE*a%)4S9=3Mx1^+(BO18`ojS~fntbk=h5!!GK>jDwU(c}>6h%fiq>pZe?; zwRHqFQyO-&aPQ-In!A?;Z#2#hOnT}J6rL6O1Faw{5bxZF{yKGtd^cGB%#r>eeTUt< z`=OV*_^8D*y%}Dip|~_Xgk_!#stY8)?O`fWav_!O9B@^}0D(cKF`ZRI*P@o$s|5;UFeF3WRHo4_shtb%a#p1AD#Q_Woo`NYr5yj_?ep- zV70tsewzScW(zveXvekUMX-+8qlHa{$kWDZj=USZc!Ime3Z(CaQ@^H!mD{B-HVKyS z4uW@#{&b{}YM;pJP!}$4AbS!wtzE>1UghpM4Chw%|Bt4S_LS29;qG}W<($>(DfZR8 zDwnX9J>eOVBL!kJCWwl19`^6!NcFIn=>xLcx@vx(iM_g^lBxg&6AZ6jgMPyU(GFtm_)w`qd^7jzYA}%I%0{aWoNHRAHmRxq6&Z!2`N+d#c`4Jw2MH3!y^?(iCcqV#!(i?R~UQEvZIF&Q0t4}WJ zA4J}}SKR~FQqSK6TdGU(+#2>D!$}E0rLSY?DU&gaUyek$ez9>{q|NzUnMf}`yFeO0 zN80Guo;hJ7PhU1KoYe2z|Fz`c2caYO1pxSNqSLP#DJ#wKpTq=Pxw6YQda{3)Br*~gYq`_JfD@nn4W*qiK~Y?&62{9EG>Has1@!QD zU&I;w2}e|Pwo?={!>2HeVfLpnB~4c$6=-u{^T}RzvD>b@XFb;qyR3$gg*Jx*)hl`S zyz77$*`o33+kb(8Bgg!b)H3WtHP=~l9LHZVeTH^FZ6Xk1gW_i>&^u>1A_-+wpR3aC z=Swt_|3w@Ght^CAXr5VUNaOYdE2Lky)pzF%Nw@&SigujB+@(us7IxNXXa{GQOm^m> zZiTMJpls<}73DTf>y~*5tv`Pqp|pi)!d6C_e(C`u`F~~}Y}m9E?7F({&l^nv4ztPP z(`vnWnRnf9kILJVhC61b?lSAXT%rQAVE|)&dn?7qp$7ZL)Krr`^)4-OqaJ}v@0lUW^T0&AM)n?$@p4>c)IQbtWwe%dul8V{s z?CN-Ue3hfGH=t-XbbHT?lS ztG+5y1uYB(Qi#w-xFkql#rA+e1KVaPh=@Pzxl%p2dDlY!ZG)|<$4jcO&Ie}4GDTji zv5Rle_!3H@FIK81as?s+#%_Fni_antR=IivSa9l>USj&{PE&IH>-F8;U9a++LcCIP zo@milO`0A=LsK{wlQs2Vks%#Xz^Kv9QJq!{@JRD+NOjrr(BdFfu+CH^Z-499IZDt8 z;k1XbQQ)eJzk|B%`jNiM(A}xIHE+lZH~Wf$x@|D+B0BAmPeo2GsMEarOn&5#dc>ci z+kuI)*Ym{;)*q$GRr;DV9DyUjEM>I-W_4s{{MRVofeS%;>nDZ?h}}~-3Iu4;z`nzM z0e`3@6NA6OzH)YT|5@s$;0RBW&D%6ekPoK>Kv8uRiF_f++zmKrX__WYbeQRLNm6lp zT;tz~{qS<#bNMOCQOdu@*+!UE^RM=O4;amUIDpVCeb!4Ih|}}^zyWzlmfK$+hpb8q zP@iK#*WXJZW6QnUxT`=JL0dxw<*EZMC1lr{T#vvAPWdIW6a@q>ablmj%lv1C+{I$1 zQVbd`VL7*?<7VKFUq{EUO-@{G9&u7MqWR$8<$c}Oci6uj@@&&+F6n@cG^~Wh00gq7 z4^Th5-x}+7sI$l;C!V$rBy)@z4$y^6z68!ko*|FeeejxjUj#Aqva*t>DpEMcjH*|v~5-@T@}gCB6BkvfMciM zB&z&jEz>EhB<-{Y#c#?zYaUjbF@!(U`3p=Vz$>{>aS zbi~L^+ZnE;jbA&mxYlg2I|(^~d0fynO4_{BRCNImlU}rs)BU&^dOvi5I&G>vYYfP) z=iUMh=08g68>I80RcV`^qP3|t<680FD@Y~pQpdZbxGE6Hd@`S%%8I~UD%~cNiwE%1 zLoX#GNB{K>9d8w&Sc>P5bv{0C_4K0qi7gq1-JYXzv1yDYZiLNJc zi$-vEwUpV&qhjV2} zsLHe78j~#8ZQTzBMWkR%xl=#!^Bfp*H8OVX+Op;52sZ8VU@5sTo&X4Yj+0Qanld+H&MIqbb*nEAyCA>4mkSp_ zWPnGkE5wl#MDCX$H5Mq%AWZN$HB$%3*CkZvU}D$O^~w808c(-(vwyJ|g{lx7(=ytR z24N`7gW+7Fy161)*h2pJBW1bKl5YY}sf@`QO=_n0@W_oI!0Wv@T5`Z~Tf^eZE=DLX zlakiWzAhY7%jr*puOogeDSpmCWgY9%ts(NGEkNlnhe(yqMmJ~XT(*#8XF8r+Bu${^ zFlYN@mO&yKok@r5bS9j4&|lLy5h5xcmg7#+{G<56HOIY44we}-gR{Cupp66juH^(!DftOl_qlTSZW~FEJLpUOi2` zV-U{4T%%>^DaO(=x|>G1P)_K}E&bZd@b~*Fk%izb&tT>^ay z-gA#44ZAjJdNG)>7Pskq~k1>6s;4gMJSO<%BL?^_$}+hPpyn9N4my& zB1wEZCb)OEWw9b1-v>)o$Jxvz{Ah`GT!AL7=X%#8@AbYXH4lGDd*6l)>;-hwGiO~o z3uL*uCo=)Uy6U2JRWDc-V%5IadaONH9y=lSUNE|zTMUee2dt?7_Ma1Vb{sE{v3ZF5QF^xKnv8Y^dyu}n;6maqgReCLsYyA8 z?P>8Tg#_;QqB?^{(uHeF6P^|z^gfk&_~!9L1j(WI>cdGWrhX~()@WVWMEijfg(TGA zBr%;7t^4l@aI;%^Quo15!J9c3)dhToazs5byk{5k8RQe{7dOI@zK{5|H0qs2i|vDw zm%xaa51mjEb@U430@3WyyM@9R)#RH?3l6x^qdrnX3W;^1P~If%rXCK7S9FkZxElJL zq_w$t_1a-GfQ^G`B)@BhsalrA#Y1pt!DeLgaq21Wt%5)=6g)W!_}*RI$NeK9vuZ$K z-Oqj-g;cV}L9E;=DAB4XwK(=zo$rBPP=OB%mX&3%%v@5&OXJceXV)^h^dPWWvN zDT9vP!o=;Z{}2V(Bvai+pWI|Y>sZe9MWJ4K{|*R5$oxaN*ld~|3Y2etM0;h;iZ+Z> zz2ygx=zuwa8Q^u=;eW85Fw;9o5bIKx!A(fBNp^uHdW|ILRw&2kGK1Q#?pF2v+7h7l zv=ops(We#*B5+nd8BjJ-54#E9X&K%iVX=1y9A-X3izMz&4b0van3Jg zE_(afMgcHu-XhXFeFK{|t1e2~((9J1vz!>EeUjkrqs{EsMVIbU9^(%%J}t2+_R4IY zZ9+-PN@V@8#&cPgG^f);$q|9qVbj2%#mnb&Na6&v>x))m1%2xdp*Hu zU8i}KuheJbQMFeRf^i;q3ksS?{xQqX2JS%GQ)cSi7vkAcTRCIHQxj#yS&L@m&v^v0tw5mL)ch zIi6NF&W4GY!_5(B5J7=pnY&~qIOCS^3A~p9rfv)O1jgb}KC*j|-f9Ww$3{%|=uTgJ z9O1Zy{{npI13VSvs4o`lbHpT9lHIp+0F7;u7)F^h9Aq8~DQybA`%RBIt18!bWs@Bw%QgdCexu%M| zz82`9p}7G-Kk2=BiUUEcBU{3Q@_x>=wW0AB-FEy)Lq*MMWxIx;XSNQXTH$ zxM^^a!MK@-O^jf|JTS3+o!Hq;A=Yu6*V>$0hC@%KWaxajUW1l5EDd*-8cmiNd$MS+V}2$-Kjw{u`T?< z`8{K|M(H8~b{wX~H3u%-IKaRIN>WZ7i$<0U78U^%Q1#^EtX|plpjOM}pRQUt`8cPE zVbn`Ab}5cdpVc3=%l*>{@pPdIuaGrxi(k~Pb{hiKtz)?f@-%JOtxguXYs%=WJZbIA z?C-Gv-`Cu3GOKe=+*pVSGsxgo77xb;i-zv&21Rw{V7{uFZfBeh5NLGoa*YWU6CXJHNTuF0#PbuzDyMtEMa?BmN%hD3*w>GW917Vn-_V&i?+%T?x7ip!VNQ)TUvVO z69-8jjUYNB;mzf%3F@`PUx(e5*fPxX@8=66tLHLhdEyDwO2gE(EwgB4%y~F&x&v+2 zI#-j!l`1^CZ3t>N_T3jZWJkeVznJO@f&^zT3^B3uSxDvjSLdGdYI4>F-6g~ zF#G2kgl!l*L+%`_-X95(VzwKVW#D0E_t~gpu0Wd%&O=eqJ{SzGqJr=xC$g?=OYlIN ze?yw#$5GmDlMcOJh`%%`Lm^6RHSFgxQ#4Q8+^W_7kEyc`is}#7_!7d>i-2@DNSD$q z2nZ-0;!*-q($Xm`-BO~2#3H$bbceKbBOxWt(jE8kyK`smKg=-047=xi+%^NJ_j{oJK#VZ;AE32NK?$0cj)h72bj`uZ=Z>G5)Og_yY zwtjt71|m5x5s&S3iW~u4ZGjx!NuHV%q3sHmYljinXQ}K=+hv&B1OAFQjjj+R!UMNK-D+ShL3r# zCMz&v0t@(^T3H=u6#q#-;bA$+Sn(5j2QMgyk${uQ`ot;Y2TgVSl(@N~ZT%S`4(7O) z0d6n{T|`Hy(ha$PJ>_?J|DO+AmwHz9KAb*g)RX`q-_TZCV*%kmFzUxE{z&_ec8iA* zKIBCLS{vo7%5)*NNLU1QX}2uU?`dL)wAFk@VqC;23=t%O_EqyHXkxPa5rH6&=Pi6a zHc@e%mGb>bn3!?bye|$c{?faplMSl^7<7{Qp?M-uTW#~~ZO5BQHKac?3E73cqkEa( zUEEzS(OnxLL6f--4Oo6QB;$WQ?Y}yG|MRrv;ofR~p!lRT&KMcL77tl_J3+T&7;d^E zZbs3vHNw4ExnZefD`Y9Apuc#Ou5OrR^vv1CvStZowb0&OyIDfdR`(=@o^CAF3 z_i3%RcV|XY6U0%2q6G%th;a%I&Acz=$45w-yf)-@?x6b|GSg6QKZDp4%zf36^n?3y z-YRu&qd}4+=w&!V#|Yo`dS8Kaf%s*=CVL(TpM)y#tvlB^DYTEyG~a%;-)^+os+>&4G2>O+3$MFt79=gq|mC)?TBdv+)4HAK60ST^v?3Q`|- zEarT>O9B>W3=-8skRI&C5%_@B(q-8Its2I^?K>3j~ zpCs6T-%Qk;Ia;9hE|b#!Gor*TN8)i9bmHtg}l+bh$`tNRh3LM&r9SRRz5byV>9{%x$x%*<&6 zPT5M3nKVXd@&Y1fVBCG1B?&M*Z@3i6fh)?9Dgx@GHC;V>MXmhJ!=W@|D`YB5-496peC_7t8J5r*5mK!3c=l%)1qJtJJ&Qn+}&UJ_@1j z_xi86lb`I~r#Cs(8H&SVU!RW83Y#eZG;U7=xN7!QqyI3w={pOaU=d9gmr)h^um7H@ zL#5CP8b!*sRAr$40Q5K3p@+?E4uBti+mlY4;BAZhQwVcSmib)p@X<@~(~3#%!jfy9 zYHCjo>o;O~GweL(x)`*f4*h#))yzP$_4X*Yh`dy~<%_#?>&K!*f}|vs-ocD1Tvpd1 zOg%|*+gq~tNWpn2M99=p_qd!5kb9~Q3x6a3Gk6xMp~YdwyB9vKrJ8`7M3k4bo%C5P zX5NGv3txWH58?F7M0{%ZV*@4$^WdNmB{=^Ry&4T2t>5<+Zy}&EXt%y_94y(_csq1f z;=BOVs<<+vpp3|>CW${-@&NC}0WwiM8Kz`+?}VDjoz#l>+!C+)=z7=13c2tS8zXH) z?7qp4GfGlseEWnVOZ`}AK~4rRJmIU#46rxd`1LzIA$N zqQ<~(ol|@WeqckcytPgtpF%TN>*XjWR3>}a2%FrGdUbT@7rR?_t*<*02dtz&p5guY z=4$ixs6@jT&*OLr%mxI^TSv+Z+@;d~X5}{%V_Q|d$vWTsNnp0gs(_c8TEQRnYLRa1 zPFKcRH?~8?w47n=pM7bq4KrWut-Ml>VbskH#v3l>AF5lU+Fher zp!i<*Sj__crTuNEqtfY7zgQ-udlR+!I;qz)JZ82!VDcfQQFS?v#ZSRLRHAdAaQ|h)iTCg6Q zBiY_~^Vx?VlMVvyNqS;);G~MAh>q;n$2wxx7sj)cSoodpj>di2gl=^M&l*S3G30)E zU<@B~Ya>aEX_Ld+Bj=|jh3 zu2#)Y%f3)bP=v9nh{JwccJ(~@%ATrUDZFe`MA2x;flOX|$yAB|4xf?cj7->&+E2Ib z0|+EZ87}ETbN-Xth&;Ei%`cBCz!q<*7+{IIi2x;QSKrr^Iow^eeL^x`J!h?Mr!&^UuBYI$h&vfKVl)waAHP_e90?WbB{ip z(4H_i?$oi@h8F3EgQao8EJ-TQnZZ#ot)oMT`-P^( z)8>}|np#7V69Fe(sP}j3DBkC;N01+hGdMw2XaPXWW+-P-QhXbcN_AaL?QTj&F+G8@ z7t?#nA4!UGANNw_F^Qqc5tSrHnr{+Ag=tO)RuMU-nh;gaG*e2|kU7u%%27jHR+j`9 zUyn!l;gIkM>MrBI`neZTY>LJ*fQwxuKL5tpt{-{I-Fvh3cRy0CD2D}Ee|;6Dr!r3x z#tjtlRjSsJ<>FjFbO?CrWU^2&HD6p-{QzM_{1+?(inpUdBQpRUg?YFK{6u29@bg7|S0cUiaY%@R(AKHf(8+r%a0zh-SRx4mtiNLob9H6jQ^u(_D;#>T4Z14^5#8-47PD|DFqGj1>-MS!AYE@4fvmcU6ScwKFZh9uE#q2LK)z)`*m`n;V8z!_>`FL-q%m&)Wxm zMaqEZP@ayrYkmhrE@C@=HyCQ9@?))3hp5PoawmpVU7Zk z^f$;#-&ZI`v{GZsCB+@+pHc8lePPO}PSH9{GO6x~c=qKPCgaD%*G^!jX^(C?Te$gj z1ew5#WGQD$^`;t+n(Zn~#-QzU-HyYyV($T^x2o&}t7>+vc^dEKWq`0VF;g2U4Ei&k zvL;6t4f(`H;|L-VzZWUQpvA4tXkb@_|HS6pwj2FqO&GNCc{gz&qcy|C*uf+6X|fk`{ol!X9gcaR=zED`1|lrLx}fv??w{i= zV4!eL)WKhON)9ZNg~tE3kg6YrgDV$4S_cL3uHvT+uTz&(o@?VfS@Uy@(}iLQ9w*kc z$Mll-m+C{bJ}!4U0V8~sun+FsRRj!h%*@teNyAfQ-LhXx>&fq!6 zKmUqPMHB=I2_jhfBPxr-ikpdp531`dW+9egK<2YBSpubr>Cssz^k!b0HpLl${kJKZ|&A7!~*0EnbbXpx#QSCs&|fPG|_VpE^c_X5qWNK)czBg zgSb;mS%2~MWq0-bWgjJ*XU@Fm#}5Iw{ejCX-=8lzx(wCkNRP$Gr;dqw@>Y%2dPV8k z4L)A9u z|0sa78&IzvwFLj|p0ZNY{OEr!b`@~uKMO+x#dj-j4YX8Vskfkw?c`}RB9AjY?t@U* zCdX$1E{*aJbeQW0g@=n>gRUWcj@o?2)0o$YxQCI@@Jh{B?5|HFk{}#!BE5GlCSyi zvvW|r2;e}%0xkqj&}pB z4v461C?0Rif}=%)gx-)Es<^IB5zL&H?}#H^fe>4VkQUCHx8u-h8hO)E>)-sP(en~8 z0ICXD4PXtpT~713d>3)rG;6 zFSB3`8HI!8(qn1Id9uk!b2PFjdc4=O*|4 zdEqGf#?uAimM^qPSjoig*|I$EhXI0vn8Fz`!JHVPUnH9?LygjJIbv6>JrFmOK0jiU zawFvhf*^r$JT=mijl$D=thX(_u3fX@xsN~$ATaT8+`xLmGez(dXbpBSKGU1Eqi!^e zQw%ZPjOY1SHb&X6m#Q zJ1}0A2F&0bfr;DyXCc`lP}9(*J2lJEhs1KXD*Ocu*MHRwE47Z$ z&1jk@uBY11(%Vu9GX<=9V3v6pGsKXzDRP~4g&-0tk-2AXQ$`W?>fRf*mh18QyyRU(YwL z2k5hWYGg{{@=l&uN@9R$)TCCJ!V7jlgI(`WO0`u4hOW4S@8-!kDTP_a%V=E9*{^@X zzzJkquh6lU4m%leSDm)kAA}vq?aXehbl3Jrt)s8}Fw}-B-LdhpZeyD9qQM;;D6&Qq zvHGjYS=p0~v3mAYy?^>HDcr0fpP`G!uK`?4(dh~Gk8`puZoFXBn! z1WWgog~a$~T$7))JS$Vvx7oShB;au-(Y%jh2%fdX?1o(6<-Jy^LNfD5jCnFy_)i3D6*A({LHsY)r37Mn zk4E!9f7RO7(6=kCqH>rIgFJM4DVUdlRMmR7G*((8ip@AW-ymzlKBZca5tDqEtuYY( zxVfY@1x+3#wf%g8@r$V%U=IL|=I;O}77K2mDS(K>aiA2$(P*H4YbT#RdI%JdRV2p0 zvlM*{3^JGRc$=+V`-|Y(W$yf4c4z!+rsI~;>|Dp*aAKc+u%I8SGeBtnM3VS1;PSm} zNK|J)X$sd!fOhFQt_M+&47<5^EvH6L9uyH6QlJNoSZsRK{}nS}!{O-Fw0jRjNxl?JKiCJWGYOSk*v z9^9!^ai5$keggP7$NEuw!(u)g4b=F=Ku4w|=3%yU9eo>@Q*6WzhiUWKki7Xd9o>Q| zaf>EK*;XAI*hBfLskR}HsbsU@>vs_QG# z(Jr)l%RC1C?J%{>UYZ-bQclGRELS&*hMa;2M1Qztj0u`kLZ8`Km69xMXL?fb11QuW z;RU-GRiX;p!El4Lzb`&w?5NzL`++5&0`&!W1a(@`3paVtg=C{G(vqNe0bf`gF`x zY|^TNO1?0t4a8jarhj(nsi?Y#H&5xCR-di|1_j7xJ84UT=|chJv$avfmx~OQZUVSC zyb@LWV@8?~AG+1_W$qRuMVmfcxs3N;n^89%sjp-93H<<8)nS5*@Ny#tqjq5u&=QS| z57y{n6iE$*%Yy3!L$m^1%npU{xNlh=O&5OnD8Ijm0qUUU6_nlFm#B*!2jb4=lLvi? zi}Yz4>9i3|UHx)H`|{QAkMng`YzJI!d}sU%MpW8{rMFMY!)#^=jMRYH^9Z!z0F_&_ zpVOQ)wqItFfT|5(A_1S*&+@Xia9o3Z{cU4jH5&C&qEUz5#!l!b4%4Sq=I{cgh)z+& zOQh&m97X=Y#GCo0-~GOlnYfHp4&vCqI!XBGDSh%5C8xI1_dEa;(?H>xmU9R{lp-YN zdrcYak3L;}t6~%R`4@tpF>l!$er6f{9!tmLJ>64$29XWIVU0yyCVSTl8h!P?`~u9q zHell@fFEZ;?j!viW|&)oe;A7xe~lo|gxO@;cCz+9*Ei%UJqh(Dlm&+e7bsywRtu1p z`h?MeKupT~4j22}+w1xGuqM>%K-tiff7C(be8kWU;D@wbg z(1S37lq5H@`@ed>pN3I2Rs{<*9Wgsbi*|tZEWFOPF4t#SuA#*e%RbuK(@q(oJHO%n z3jA{Mvc1*BhUax3o~+DDzTusjl@m=}H!=8dh84^-+~#d~DmNUpbYmIZdei1c&*OKg zFL%DpgbTzUQ?~zA{x<9Czl&{8`&`4+1UkAm$?ew5&=fCR#4+Wbx!ioE*H$Umu|_!f ze8mNU;(?iub_)~Bclwep^|aN3Me#Wq=g+O0Oy4$l$O@bl%0u$$x=0))vl{XDmA!! z{*>0gPO*d@{h?=vk06jJEICq|tFh+g0($k(UtKd@5IGQ9hyL(qMof@g>xwD1W|q!* zQtbUDW%rkf%DLYHo!gDDietHD{)NZ4&B6({(eeXU+t zVs`Jm5@v6$K4EAIK15KpObbRxtq2@& zhzW#v>rNLpn4XuS)BtwC#GrXb@`l8q&>m1GU}}C@IOx_M4uRgxNF3A{+1*ls!_~vp zgBrwK0^F7()`whDOETj_B1e{cC5XZkPT26YT1K`{Jttk4p9~vl38iO|# zs{}%{GJ=NBuDmpropt%&=Ox)_N>oD{4RP(-3;fFBNcrQK{bK_bNnny>Mj;D`9*Zp|9kN0l2AKk1tf)(}_9hK_FR=0W93u%NKUGco%RH2g;CD0FmL@3l3h;e2lM zc^w4i_gqXEJ)Dz`_YjB4Fp=O9 zB9bgbG^pa~%b{$y`|pIuL&>C(u3cqdifGZNlpKs>EN^V9#rSImfV<;mU;R^ud3yOB zvsTf*$aF~fNdJMWhfM`GA)WvLEdvG^awpeq%c<|`zSjvKGT*-bW~(r10@Vt05HP8)p+$&d2=8w%X+kGm<-iOumPBYWxN$3E&Tygjl5+*lnC2LT)V z#M(Dgza^}$oAH7}atDI%7zTc(Z$Rx|`-=jjs65GQlv%eYS|bNJqaX?JbFsPm0Gkgg z)?K=b@3&ksYXlKVV{;!kKA{Dw5)3S~MZkLVa;R}Bsvw5o1bZj2n59K#pRej`zYKDr zIWY23YHhM&3o!?8y(j<<88Br3uWbCiMV_Y0@;Sc8Q=a6}(2A9^A?exQ-X>lLBBU!o zfhr^$6f1A zmwAEEJey(Y=Ae;Kb|y!1JauG=FF$)c2(_qMhTcqG{>U_zzTN<83uNYR_Y{0YtwOY1etwmrM#PLgwW6KB4w4)l z!UpHSt?(A!XTE&BDg%5F$jW|3p7b_N<;n?Lei`=_6YPoKkdWX`fmC>@q@W#xWg|{Cz-zC`iqgT zdlm~#t??RJ&lsDR(ZkWOKxwL(5$#%^&j4kpV9HNwvFt|Ha0cf>^5sY6SJsnKfXkC~scH}ioQz4kH**%_!GOyx@P{1Udy6ZxfOZH$N}@)C z56q}i0TNslF4k&NE#>UI$6DZaJWXZR+D)IFnAgV&DrtFn3%jsd|IKPtFDMt#i@LNs z{Ino}$2a`qyQ~J0YF)+NF8o&j6WNt9oJygN;KIB845Yk_pq(!eDnyinR-gs9Qti49 zqfSK{aqk?#v66gGaTqJPgc;8P6#E^RasI@s9!=4fv-+umzz08B{zICjY=Ce_WS0uH z2A;DWB3Tc1kiM>kgP44OHr3L8pp}yUbYV=t9w(fUAl&3k5yOCkQi2eDB+!i%{h?7* zlJPL%Htb@?U17zT3E`9Yhe`1!1U$UuW#YoEvuoQHd6hzwK8#dYRkVAqbb^m zl=LyPC_xzEpg@SMYoSiul%gsk@A2HGkIxuBnk=R{wnT+CRZMvpE}|c~s>PK2_-)qd zgAv!8+W*81cJTEg@-8Gwd-x0VgO;=3P6})VGgjX{$J>Cg8AP?&J%PiaS~24NQ%eM? zN+gfY(?0YRIWNZ$^sDQ&tPOnCn-rAjqUoGIlg)_@b+pfN{BJPZPxNqT&IPD+|800% z^r#{jK-c>9`_~nV>p7$269We0)-aN|P`R!!YFnE3_Ml%J9M^}|;ePlGZ4hGcyYFnP z{O%bUbQ%BIIe*O%Hh4?OxP~V9IEmV965$?N<@Zi39S}=3h2#5-F^2n5#3}mEDgh3; zykO?m-P9hH*UCT-Sq@z`)ZoD`R6GH{VcRCuC**H`OLLoibPEZ)#rA zasdy1U+v7jteO^mn4WK%JaYeohs}?kqsLljom>3|BD_Y(=(P}kuc{}b;HiI5 ziYwMKNy8&(fF~x@APV4H^RY^#z;CtQAGv}+o2dF_lIY7R$NM2I&bxrtL%ZAdzt;eI zlVU^W&|Id$p83IPY5c+B+xJ1MU z($)x%ku>Rc%z!gGXh5mRtg4tM)yTuD=WHxT(0?WDC(*D4{p>eC3&*o%O5pcTEw3G8 z>gz9mWOM^-&MqTX7TptrT+=RZWEY0Im}~#268g)!93W>m&&2Fx-*=?2z%@j0^H=M3fyPSouI`M>?xw_phB|GHL*fdB6t4?dYGGC) zygwPf^G&fF)Y~Cnx*ae%J-< zr_jF>^L13e2y2A$crY2m=K+v#v8jxcWZ2k%m=X>v5{0ggO*TwG`eGct?rRR(vO#)+SZkN2D)hcOg%r+r+X_Ga92wIBO)*Gk6XDupiu3xcCVddDYZy>j`4%+dvv1l?vM1+k1t@=oGGf6qI(f zDNl0EZks!8WL7XSu!`Md+sNfV%SvupbicLgwkpMjGVS1UvUy16sC&M>4-xe>h zzMXskbh&9f#kTfYTBKoSUtZo#%=bk#TPCw>6=u*)G2t4M_%BRNz+=*xV%Gz8Pp?m_0KV-9e0Ae)e+Aqs-LGXmk3CMQZMwW}Ij8+2qV@3p z6CTD0Fc=j;dg*^n{oc1ltqyG9jV^A6~S(p z%SW^MmD-c9UPx(W6#^62+ouXtc?Yg8h&p@$cP4Wn^KcdiSt_aQ*oX)tUQ@s==I$cyWGR!E7rlrBLyL+Xg z$p~i<*48}e@2fyt?a`ZSb4Xeeui9>2X_l>W^QQv^-Yf)8xqGVt_n!y$!|WH&!A+s3 z1>z5-^MI=HkN9-eit*rvYdMGppL_7jT9@y_nsNQL^DcWzEni102x2*;@WLrBJ!xaX%J* zEs@z;2te9yBDQ)rd|}A@%er)`V3U0)z3@6ykd1sS^go;c?L(2-Js83S=R%C#YF~db z&&-s`sC@SW+ijW|j6%l92v<11|M;Pz>(0rsd&^=#68Q<)*UzBUH}B#>pQO>;uCCHFt_vNUEys|VAR5T{r=JG zxJ_gj@uDtez+`5adq_<6bO0QpM@8lhM<<;7x!`NcT>IyR-`tGRH)%mS(GC&g@a1v$ zxacqZ`~Plc*HeIm_dy22Uf~CTXqM|2tqsynidK#5iIa`~Tzlm*z$?gOUnD!136hBP z%T*!y&gJ>6FREVOeBJ1wt7*;>!zJ16$uv2(&cGyCB5HQ?^P2VW2wThx?!hQqYY;1P zdI%!k{24~_8%T7zE;x&i?VNT8+cK9Zm!_46c#ERT860)=jcw-X&rW_Z7|6ez4B|iV zbrm^>^lk*>T)wr5}1TbF`rIDdfveH1yXRbt18G6AA&^M2ZL|+ zWL1QcJ3UM0&ylxjdrvWV>`@!lLRgS!^d1Re@#Cx8xQS-Z&L2gw6OxY^UWvOlm!wjf zSJfO(R71f`(KnyP+?_sNE#s6m9p1V%Dp8x&e^zlTnKjemG#wv#D5_@s_fMX~^=`&K zPIx+f%o+hE`28!ED(&#?*0iil{EkL?e}C#fQPw!iIJN(3GI+>Q5NO>&EiunPT5Hw# zqGQ)~*iaS>%r(B&S3H}}!CB(-iB%W#t?6|m?g&#^w#)C1tCjfirx~hjI!;-$V9I|n zN`D{GdNxatWOL#o^Q6l1M zz-r&sNQ}JeT`NI$M&TkAyp4G%p`zKO>3$U8iMT+@JjQQqV^<;2eiDQcl_qrD;XOB7 zX@Wp^)I#BP5BolS3_G*Tk;4Y`0=c8*U8l#G!~}ougOk|5uQCVQNF)Qb?OU$_t#bgA z9#^WjBh`As3tB$+yO&tHl$SX|X59iaT1)=B%*9z}O99(fSiu&mNMy1 zn_jJppnKNWUcWyoId6P$x3yhNUVt*9sFB+oo14UEV6~cIu?<*uq2x>)j}*V775>sTlcWnpn8v)~cd+=UU8L^F58!C|m;!lg%)M^xhRQolO|saX}kMr_Pf7vL;l;H0jM4( z?=E*~mEK1G{U0cak4_i*JNTvQttDwc31)TZMa&hxtUrDVoO$s8Bcef?BQZY1x)PmO z+Ef+RN@GH@532SUp>~>a-OOfg`mhE4OiBKIuWtd2-L}>55*~CRSLMhP`Vo^ZsBFvccb@1(Kp|Z15OHA_kGt$@w?I1tC_K*qiNq$nVH|89)i+B z;1FBj~$-pc#YF<0m;zx5};}@J3Y(G*8ZU$aN{U*Xf^#*;z2%CJVnL-!6-wu5pBXS##9SO z1uGC+=^PL7TEuF&`iY?CTKe9}h&#R_hf~v66Y-HR;*youvi9C3Iz6?TjjlWe@-LGQ zUJ5u{V3JHUjnI{_i%>52TQ21_luo_=zrz_YC0xeo6prK!w`km`yU*F!fUi@?PTjw^p13DB)yVkX7 z|64KofOT$Os}~PX@7UqO?%V}lKFN9*w$Vx>I&-niN(f92ad|Bdcr1u%(q8_W{Qa9~0U__* z^nGP^#lyErMw0O3TOD+N83?G`&K|71qMuvNvY#32SZ(7a%rh$yE79DWAHR^H0=f9PU2&TCV=N*5i zaRB?!3EHQLMR`_`3pX8oG?36hqImVYx&A{fu~)n_v^YQ=laLPPc1TvV^I-G0XMoP9+T}Gv2?deA0Y=S;B2#~bRIv= zo!6o;D=_VS`=MX%YcJm|ol&v*Z* zy=>+*(&S>rk)W(F?qriWF8#UliQhA>2-WLk$HtZ7 z+u95aRpunct-hFa%u@TEWIsAkez=R1cI>W?vy8Jhr%$GL!buSseH&S6j z&_}{=650aF^}MQ#Tgo1r7+AG|Kn2g19h)y{X;WazUqOLJnz`F^R7ybPY3K5y&B;qj z&Mt!`0`kX$WR#-wxmfovXGxeD$ld(&BVfKwtHI}EB@LWaEWISo@2sO@arYP(Y;rZClg4khpdZgoJAH*ZBO3g0=4IEB5~!<~@K~BVt;VHVVN| zOjYy4-|~s;M$R_kT+^y%q`Ma(1*ro>T88fFqgp3xDjVZJ@jk?JeMe3HC6Egw7S}0j z_V&Ip1qKBbk~td6M}O)r^mgL_pM1wHgXd-EddOb%6J8Q12Z90auKL-Hvb#=;OGz#q zhc|?ABbvKKaudm7UpNWy8DzmY9Tnu!3dzBzTT0z$GW;zbPm%VTts$o_z||~)L;=Rx zbxCA2vhi+5C8vqn}ZZ_Kzk-aD8PC{8X&t^C&>w zUb1~IU(YG^6_-?dtFFph8_`IWS$%~{JU^Ga%07~C=-s8PiM1>9F&yQ&5Y-*|Clm3VI0-qtKIR<05YleM@GxN|lp4On0AO9! zx5!r3mwPZP2E@d=*~|;99YL@e1!l_zF?9PyvOqVcV}A4dId!j;Bbcq97#r|liQ#l^ zOkx~9W>N(-CF&1x*WX@bCHx3)>G zLk~c#VA3u*`);QiweH5yeTb%YNK(uN(h*my7kH+1e^;L+#Xken;8Siec5{1;T1uKl zzw}E*ikbq#nS@mZLRy@rP*k&T)m`LaLZm#mruF!rhrsaHsXm3f(~5{%{bHeh?Y}o5*2K@n#NIX^D0}J+ z7_Y@kIDGb|WSfbCg}2d(3pF&3X70JaZ`_MoSbS&VRQPMbBaxa|frB@$XR^@6twHM> z63#4VK-4tSxi-{{*wZrnbf8hK<-UNeNcE$kc|2pW(A|Tx&&B3pssmkF0GH#r1~;lW ziD-%o+iqQ8$^HVVTG$+70lWllV|v3b13e}r?v$M6+B6k{#0E%OR z5dq&v;>D@CLTXtc*7S@W=jbxm)N^|o@)w*Ph5TbyGW>$gf(eSjKeueg zZw+d-qXQlHw9}iNE0P| zKYPzQQ4^iM^kT{pKo06$%6{CqZoc%lYMIX1!C@0=bd?Z=L>8*sX(w;%AcToid(5eb z+4Ie4G_qtdQ8{>W0!YuHY!#c|ZgkcYDR}NRV$a5RvK-F;VZG#KS*nl87Yk!J;VAUn zzq7RrsIDp|-RJGHj4q;!2@ZVc>jA?6zL3w?QW3L-0lPb~=sy$VHYc)Vo#S_K&kMZ&wqfjCf7?Jx*- zA~+GEf1aXrF1kHFdG#?_{A9>Pw{EIjRetk28x=>*oV;%_Td;)Gq~W5qSn@8+ zv7OYaC+*JOS;BCA@cRzepoxL^Vhn)`X(FS@7!7U9KjVLWIW_q6bxm-TGz1NvU~+a_ zNiLRCKqpSH8#@9V=ejuzxL{Tf^C~BvXwzMBA63nm@A55;^Bg9BZBzO#h2LO}d&F2-#jNr8ViDs$UlSbfDTb25azFXOUDB7Ikz zb&-31f2cw9GPll1A&_rc}Dn#HUX?#!sEuk)O1u;JS^?i z;x_?&qHVK6Vm*MIOq@b(R!y8Dzl((x7E6z<$r;|1+Q#z|t^tdSh_WH*@$qegn3it} zNt@cRtBQL~4OYGe=KC)HIqEWzO06vFFZZo~ z>u}8|bfM9%oMsNT6$U+f^gZpTdwa_G2i8IT%96D85S^~~e80>9#F1*I#N&&Y5qg}U z(Gf>X|3No&9t`spevU`}&%t8nZe^+hem5S&8MjfFO~CAef9JyWRA1-4CsEGyC8g)^ z{nb)Z{L;;-i5tzvoYJx6y}Har=lvRe^X>iJ8S2Aj-9tkQ1PGM&<(x^4S^~9&sG>(Er!KeLRDD7Oe0D-MX||aE_uhRKzbv)6mAWigN0rUgtL*G#rqTGr=JW z`ZvHNAY&*PO8S_+W6`r-T9-1TBw8(ry7S%x%({vEtu@0BHmld-(pTwhRpE>gy#2>u zgJ@otb1YnX%hrbB@LD%J2EEgHD^tk}8(s=D@YxSlfrfLzeavC|ckT>u ze179Tk(uvcZ}Y08M=^Q$ltGUY>i^cqtG(o9P0Wc0hK1m3Dx_L2oL_|dfvQF51nV!x zU+ojrcb_!kG*aJTYsYD1J&vPLr_#SpKw-DgU-|!)$i^VXZ`oFJ_kSGt9q! z{OV%KWBF(utx@8c_P5Q|TqYb{=BO$Dv0(IN&8YVM@!Gcaa%FqS$<}$g?`-SyrYqAQ z;~C$+nHa!sj#|&p-|n4#TRQ2czxs5))-Bw9y9;do3--@sX;L;2H@*DctDVVOqUkM6VyLH5j8fAQF9`YF0MTfKQIr@eF@e^;$htis^sY zdaJ0ox~6Ni2?V#`9vW}lU6MwEyF0<%-8EPU?h-t>yF;+x?iSqL1N{4W$9Hiq&KP~s zm%Debs#P_s=9~y?A-5h=ELXzYgvKSCeC7~*KJ&EAv`-;Ds06k+iXf0I+if^DZ_GI_ z%q{fWQw%YG%y~16D-#=JFzZUfZ%`^&@A?!H&3q zIqWz#GeRb{EGn-8uR`+y2YPhuGF=4>h?xckTq8p{_+5`kS=%XV(Uk)Ds_e9yem*bc z{@Yo`h;_7lv81UyRruQ6naAs9G-picis@VR^IrrvOHV@3#num>n{WOXS>j6n+TJc+ z3VB~KHlEReX6p!;#b<5YF}+!xXRj#;F0DQN4@39>;7J{Ta8(*>7EGo=(U^Yfj$OP4 zfrc#oI4)*7;wegKvr}k92+_KdIA$}t8^jIb1c9O_!|kJUG8++4qmRE^#t0`XBvTXS z>NF>wSr0Olt%FvK;@9Cv%6~z9Dy1W5Kx6J%%h`XQvEOt5n1v>IKPCEI7Ak>^5o-5DYsrC6EiKooCYHc9i5%ni{Es zJ&R0p)hfj*O)KSc4J#GVH!rZa3=c$ijkb(8MGuX)jP@u`jENL^dhPnrXdO-isIX%n z63;Z(<k>pI@VoVY<2U}7 z@ixa}dz<=Snx*?M%{uno!a3V&rI^x^W1)M~lBNaI(u9S9 z>;tIobFhXWPIx-oL~dnbDy|R^GHL*4q6LU89_r4Bx&)AVfSBZAiPE9vN{04*UKz?r4vp;D zQZnRZg{Oc+%I#ow(mmzxzJ-wLBSGob{*xRF+UgxI#0ZX8cCb-4VNY(NCZ-csq~Iux zt(}aw9IO93AZc7Mg$43G)DddO)Q0`_Jy{zGwMISVDA&q@3J&wDxNkNWTK6;@rgw-U z6nHLUCt8DWx=4Uxr~7n&IoSitz#s?MKphnOpXn+88PvbD0(Co_UYh>sx@##G?h4Ka zwmO&L|M%LIArM#hSRMh(vWx~VDHaY$u~aTX%U4}w(~wzd>%suCJuj>LdZj<8eZgC@ zYA_XPTkr6EG$B`^qw=U>{;qUIaHRJRIFqq|hLjp=MU;__l78u<34T+&ieE1-C{WET zKAr#B5?`M#RrfxgOd2Z<-6WlE0M}d_Qc<8pbYL`D6y17*P%-P^#`mx6FNqN1Imgm))&?11I7wl#GNj_16AG=hw! zJwzcn!VY@|S6t(=Q(afete#VwC_gYxyUeKGu%77k%^|zW)Tln>m*P87N%CII4yCeR2{yT?ocd}0U zWhy~I=w-gCoe<>k`Y&`iB4X>?J*@w~^$`UDuan2~2cC`VFaCzt*-Rs?_rQ3y6#<}d z6EM^N3bL*omR1<87sQX%L z0x1PijC$My<+o5>zhpdX4}L06mwZQ-Y<06A@4Ru>2f--=U#Pl66s6f{bhKNM+br%g zXvY05wpxFjJ=d1Z0aQ>Bbcn~VZ{cN8;h+DURX@S?b=QT=+rg=UfRDw**;-1@{GkgB zsLjl*(Q5U-A$Ro3%b=4cwk6Y`?HZN!ic4#-8H>T8$;fFBIRdP_N)bYutTW$`*!VX~ zB+Txya{<9jnZg#p`AfFKB}&ElWvCdO6jyd^YG04XMSmt)v9odPecUj=mXwq?M^ess zo06i$|4E7>K2Q=X?nXTQkKhbof6d)sxk8Qk4j^;DOF-gK;~)8m#5p5faSmCTzsDpE z40LFBaC5r>3U;jOHv-qET6 z{YIMni`9?LU)HJ~lI>qpbol?bZLr$m?>^=gF{%BZ-6S-!aNe*JWt^qVIm)mlp~^)- z*Ul-pkvOOul;E7?Ja^Q^D^SvJ*Ksl)eV}TUi;ViYrGPFM2%GbgS5&|bD-kEiIh%3x zxL?cdj6}e{(W0VYWJHIPp(7cvcHII$-1Mga-)UiO%1}L~^^$b` zoD+?X5P68*?PJ=fg?C)~7;bW^#JGSC=2_YtJjAqX?dzJ|aqr{!iTI>tk9@{`Tlq?o zXQOx&k~IQ&Qz|b>J-Sc^<)?Oz_g=~JXywx=bFngH?&w~2v|BR{E&b83rp}4xIGfgq zEYdWahx%AL?(F6ufK7UbG`|5)YMGDr@^?Is3hA1L-AS%~gKDLHf7Enp``tIPJ=i!V zTu^}FsPo?xJMJg|7jIp}g+FHIF2BzMhEYw={mK}EB30P}*4n=7a`6IS`DUBuy-70G zYB-7?iO^ONo&h!@2RJmifObWxLpxnt0&cTp<=VU<4fUPBf+A0{jwf2{=XHMW#b53z zjH8E?;|e{k--9-j>E&BS;W5|{6B8F5^(Khdw>pXMA!x^nmKBAGTGPhE;#rxXEQPW? zHud`V$LBX71Cx53a#~H6bAy0b-p^}jgALF0yOt1lKKTGvGW~G5H(p-0hW~Acc>R~Y zqR`?Q8Fu>n??OY}oIwJ7T8B02Ii{hjO9D$ggP~+5SO31IbNJG8-hNWx4m!NF>Nxq5 zd)y+RUg>9Ckmwm-?PuRyYN*ZdbLNjm@*_a(t(nPhBkV_Bhs{zMjIu+RKmN(BoRwZV zc5)?cDiU+v>SF=+h`YTuDd}+GKhExRg1HLyVP%MliFdyWos6f&WWK`_&bZ&l&C+!! zE~WGyx$T%`2vvHVrv=upYa18!?pX3n(JU((=B363Ac8H3+5BkgUgC8+zXbRcJzzA_ z_q}4HP(~~v4o9Zu{&t`_M49Mrz1eyr4-p-Qac{kv_TMo=8AA86Q&xCuck&pva#Rdd z44PdlLcY^gT>m73N&PmjNa)M`!*OU*ns>w40RB#TY09Oka2vuQ`D zM$}6tV|ops(Y8+~x)~T#Xn>eHE|~Q${#!Fl`43|lAuvvy#|La#@V&2XTNYuXvc47$ zPV26|uj%DVRa_Ix)FUVe<>mNBTV~pAb&g^A9qV$LdJP698e*9QJ)H}&5^7y;Dc z*{^MHPs;hYQnE8i588~>wj@#&Wq*waWMzMhPV5Ngs0sL~3r}qd*lW9c^C~214KQaj zWc#){AN}6^`v|O-*~W>}L^0ZP^<00EX}eR!Ac>zm@app*YY*ng{SL^g^=o@w)N4N4 z)LzNP>2|Gj>-7*YX&O}ft>=WI{NlVrRKf@(c*lW8&bc@KeHHk>&ulY3vMxSJ&a1Ny zurk*|Gjq=MG>6BTP4Lw<%O*zGNSJh~8Fo&biW;O)YOH_fkX=g9cWxW*%OA;>Ct^kK zuG!u9w(R_Cx%rpoa`*0hz+cJOsPEr7Jk8Alr(=CMgI3{Q(=ks?|n{KDq1wC@9?qswj0fPSbgn61c= z9Wr_e5TJ~++;Wai%gRe-WcUrQ$%kivz zyulVP9`ow(ws{?R7MefCJ)JkagKqbH(G{VJi5i-Bqts`i4#k5_%aq#_;negDD;|lb zU76qeszbo7X)+u?lst2HeY<8KDUB8y>-e9&P_n5l$A5?(f1zqI&jI?wQm9KK)5K`; zDcMMBYneNR-bXli5VNZY_CB5pC}m%xU60bG^g|-0NMvsoDJer_{lbWP^31h)Ey_N^ z{zSFD25!GY@N|dCgP&M-uv?q=RWn}iMuR}=5_UAO8x`3$Bgaw8IGo_NE->gdsCEBX z?hC~utyt{91sI)xcQOIxVmD|;=b?N@h-3|aHiq}Jaq%}Zt>oKjV4>?+zBuuW`fF!d z4u6W@md7`w{=&|MqF-eWxuH9j%=jyhkLmXhar}koJwAi5@Z$vbO^c5+3^}i=`@h+k zLh$~sXB6lc@fl}cn@qMjR61>TIu^1*XRqA^4Glf>t~}R0Yn@+3weHK+L)EXyIx*vE zr3>;7Zj-Z1$J*DOs{>Xp%@Bfn&fm7Y{jZx|H=Z}15sV0Qg&*=oUa|dOYJSK-)-U=E z{so`^dzCIk1wCYKyu7?UjDKwZ*YKn33b*S;=gpiL-Vy-xh5p+J$gSs&jtbo!2X3^l zDxWX^+iFaTB6NE<0v~+W>*H^yKAO#9#$pn6T}u}* zKX7nLIFLjRSiOQ^;6N62fF~NDZ%sf+%}S%AYax6q)-~sXk550tF&BX(61l~Vr$>5c za_pYcHWnp*UN5guy)Ko^Y^i0@@i_Gkprjx04Y{%wrpC*Rov8ER`Q$3B7Vbsp(lO&~ zYyXiuPqhGKdDz2E=J{ZL2KI=9%(#78owZ;NCuZKF8&MkPycl*VtuVw?=|o< zu9mK^ZlGRvY-&Vw4WX7Oh(<>bmHTbk`z`XD({$vW!Mc(4+|ab>!^{$@{miI5x=0cb zh+70$2}RxSfpSkuTd>t6HxS6x!lcOe$Hly$HAEmaM}iLCmDqN!ONR%%DIs zW-h+YsZE&Q_;;5)1o7Pjb>Mk(uqez#A&;2kD=_~!QS?;C#;YlbtZyZ+o~COhKI!y_fC()okl zg6Bk=f6x6(mFKfai%<2Oo&Q1KN6$BJ|F<55GX#*&eb?i-|8vJ%4*@LLrcpZ~0ULXJ zo7n%`e+4+}e|Z3}f(zd%77srqVJYeFDAdi7&;FJdTgQom7#z(C%$;uEEx3{nUJ4N* z)Z@)qg_nIm2b^kuHGDv(3YB+SBWmtYb;A}h|FE6uJRVst=`CSx3>avX0%Yi)1yby6 z3yk%Fvgw6x+*Ip$2_Oo6QNu`9bTj-9S7UTBh-1q1@A zia|)7sD#G6`|r02@k3XaL(z?+m9Xf#z1Mbn`Fm|p(ZvhsR?QvByhX4VeKeYfw?PSlE0qI~5OF27NM#anX zUFDk)FausA{oh(p;`IB3tlxsp+@60;iZlM`Q`{7tJ9@x~r78|y_emGF`?LEPrnr0j zaYS;VDqP-vQ*#Z>JA_K%h$x)hd5&;Y==jakUA#+2*7qu z{$YuRui~Gk6Nd@0l7R)Tkv|-b&DB>+3a$zy+iR7Q*cFXCes^e}5J8_8^Jkq&794?3 z@MPH!IPbeFx3ZU24ooWLW2oIQ#YBTg{E#5aaz)Qc|DT>d><6w&{7=zMoQ8_=48n$M z`KQhwNBN7zJIhTglU8r=gFv4^us_0bKMu@I?+{BKk~5J8QUwjc4h4pT!eM?pG=~5% z2r_{sCr>1y-mf0Zb#G>!SCi?iKHdard!XVcoY4MR+sWcNs8Xx}&?HDnP2SE9InMo- zdKxkdROx*IWnQnxs;+DAuDAInzr%}(F|+H{mTUiq_506oT_?8ozJvEZ3#4kRqW}WQ zt@m=i>R?wNEPm|f1DYTI8m*^$l%s=(pL}g*uoJrDBtug|paRuf_z^@N4fWqJdANFc z^-XYT^~)dLz!gat2!N(iEd)lzybE9(zGggaYEYLL8T)Qu&!$Z{2DG`xE^cg>q?~$R zw&Zw>Hi`-sMwqjSGX>+u3`|ZNhd7uT9c%*u{(2s8kPo9!kPwY!>A*2EFWr!owKz6= zuYnDJR?j^tmw$`gZ+WZ$0wNnvP-uziroe?0(leUPBo{32HNwu9)bOMBl`oEkA&#(P zc_u1LH4S)m;G3r8>!|p+9o>F-}6q**&TE_E1hhIK@9p>3tH~??Dv9 z9?1-KCC>vl{9Uju)8~BNeSU7|d~FvhKTa?%p00w{pTC^H-IxQcKL4xEErXAK2cIEx zD{os1nL_SoS-=znz3}`Y`&@eg=|61Egk}6Yd5F9AE+H{@JDjPfqg$n~i~K}|$xEXB z+7GJUS;t^<)O$I#jRyx#7215$6F%#gxM0j?oX)ENZ;wAq9MTF*AqtrmHG)a;kc!4U zlkfB73?^k7ndve=NLFfkaUDxs~^aP!;H>e(CGm$(p$b z$=n=hQ>SRc&2_jV_`4hzSHYyzFEF?p0oc7-m;{(p}6h#ev5jXi7X zDe5}KYgaFt`9oLIfSRd#^2GB);9A;}fV-P}z4Kmj!Y}46gt!w6TT-C zagIXFaP->~BeW$pBIoI!4rWx{+QdvY-}~KM(&FnoE!{5YtEki+<#%2DyGF(^lg*i* zUv^u`@(&1r0kNjzw4`aLBIL8iTBlm8EJA)G5z{W*E(@rP=qSsiukBWR=AZDe^rH9x zg{a91kRXRLZ09o3XgSjyG{$h5u626AeX0nQCgyA35nKP3+f}xKZ&E9ibHT z#=!>VLio@Mjf?UG(et`@O6mD6S+?QFfxxJ z)CfsBmoQLS8n^UUx>dS1Ri1`AvGIVGO3~ph8YdQY`g$9Hh;HnXp(`Vk?Krg*NmL$k z>gSe9H|MeApkticDeIB8$Y!i?O`N6l^95~%U39BiuPq^aQR2J( z(iiMr?I;yiHP(i3rD7Qx@qo!=<;tJP!)6__eg|zzN&Q#53^5{?JH|2mwdVH)*+p_0 z%k%T&fE&t^1{w$yHe=;K?tl@^=DOHs$UfkrcV5}YdQO^{vfawC;*6p*6)8V8yHz_K zjqoY-P=%|Hwfxd`N~kehwo;9f2EwiMr(3%37q`SV_mbtz1^&p~ql}_(!cMI!k18xP zKL^=%9z|Ygh7V*X76G+ImA=h{O{aBIr)JdB8rO!d>Y7yo&RAuRQFOsYZUIe*sNrbv zV6H)1t^x}kyN{|!N-jzhkIS~)(t~PO<7_~eXJa`Cu?9P$yb@cS^eOw>oci-;C}Id^Ar3V3yBsE5OZ&=dw$9>?U&pky0)oC``Y>?d z5Us;Zzfr_h3pr=DEP0GgU+~}QpUJ6Imu_lp-^5C(H}cul_c~wO34fQ4E&3UY%zs?H zYfDDW_uhI+GR4+vEuwJ#+kUGZbBe8Q+^dtu5u0n<*`&H)Z!!6|62?6JoyWP;jhYjo>0wJ2T4 z4&8WLz6q9uH{KY#isy=dh_132F((xvDlM1e0Pjx1p~h^xy1{*CU`N?AO=jAvYP*FokfJYYV#H0$$|^a#99#8m#{ zqxB9xWD>28wre(C_wbIaS!9X3GKa%6Ak^n_H^si|ni*=5vUY!@&ucz@QcUK5lf1Fv z&~INU{4m)kH?wFxAvDgvJ)2v~2gs_!b+l38zLr^P)8X_IC}ePdXH!QckY=N5t^Oii zl1CS2wAff-N#r86?BOsjBw``6*QuZV(mG`;|CH9wvKxC7K^H`yUP>HhG^>%SBhP-k z{U@CVk0VBNy(gvdeU#3>J5Fa`r2`UiL~c$j?!yo(TZv$G+}ydTxz<0aMBH5M!p z7DzL#ctETeqH!Kpsftjg=B<@UYjUd0C>w!F$eX)zg_XIajd9_g^J~O8LMfI5ScXT* z$p9<;YUxbp<4e!KVq$?-{4)as$MgA_)6NY1#zmj;y!QrfB5OY5Rtnywm6h}bKJcIb zrIH~GZZI6!B)uq)E>}?!QR!$YfpwsO6;n+O2^@+X@EyijFyUH9xo2O7-GF(dsIF72 zFcJ?F8d;Q~zQ`Iq-0&0)KQ|q90!hQpVY(GINH8vsg(l7=zWsu|3!%^?B9ESd(CM0-p)vrTQi( z&y_D9caoVdXG*|Tjk%@CmIpPde4eG+YC`N^Xi&ov_4$)B?@ePT%GAUAjS0RWkg~$* z>%-~!9hHq2vOF!(jIS!gWfBf&!!*{0n0uKVH_81F3z$ELeTie1QkI5FZc&g`tXH$v$B9mGhnx+d4fRzk4ewpa zWYih(crpmI$0TaB3JVe#g~XN;7Fntfjv>ui%^vw`&7|ROaK3-jJMpWb%J{=yx$R*m zd?7UcPYov6t`s2oKHL}EXk}0)>|gIEzxuw;V>9`*f^Vn`(#h{>LX z!kZ$alO;waz#VA(!P`OHG@kUaKZ+x&bDXErC1E(1^guTNDK|uUO|kbwUJ9#Y&jt49 zO{xU38D|+hrIFeJ1y)g5Qn3{o{-q%f1HNIFtQ+<-^Dqa$(rmiV$+_u8xms#gUzrCw z{1HPInE&0a#(&GYd*e^T8ytT2+iVW}cmZYRj*ctElq(GDc{h;y#^-F$wzAGj>{l?* z*e!O-_mM$;+sa2q=&5u4s>Qa2?jszv{P^0BkRq!3&wl)XD%cdg@IlLd@Rrp0q;;_& zWt_Bmm*ygOl~ldEXLL#<6N+(OskZp&x>u=CW$h487i4%{EB4Jwi+L$&+*Bp=eI|mk z1LQqa6PYmt(P!bFgw~uDPaz~RZTm(a29`P;Q?NvRVn-B;Oe>>=MAHO`MRWv{=CD|g z8$pJ}*iE6Qn~~Cu^)w=nLwf%ja^#T)q~pGPoy&>;kbGhqE;*cPQ?_)%TAv?man2IM zIAy?Vi3@<&QY7pAxN{{h!@eWiKIJ@<;cQu5zR5HpX zAZ`m@ypSg#@vlVKVa}X*X3=K~PM8Kaa(7iwA6Q_t#v>YAeY;ua_a2aKC=E%Qn$JnZ z0uSrbZY+WluW~ zedcL84D}iDziO#eU3T;(!x0c@Nz@$*0EZ}cn_vP;iSsx1c_BCH=L}nfLXG|5zh-~U zm0PV1mr6?wH~`Y8bXgQIeNFc;ruV`6Wnt1&%u?2s@ABifTgONr=TT3cBx&#y>AvXj zlpeKO9;d4tyi~t9pl8e#ZpoD1ezb*jbmFgaV?Rz&F9&_2DkVpxU97!tS(&dC^Lewd zxy`Bn11wDlwnchbk^0vw61Gxx3<6AO_<@?2No|gG?$Svt$$!)UqZt81x_|6A@cj3x z)6$!BAE1BY&52NpZMeJInYzlEBzId3YI9N6kV}@K$?C)^P>J&@bz@p@@9es~u}i^0 z=joT#uPLE#h-WQ)5OI3rt}9PJtE|SykU97k^)Sm-8Y3Awq9V*7wl1-z)2XI3Ld@O@ zNawGm{&x#%wD5jUJFTrwGI(Bc4C2DHi<1g>8)N3Wl83zV*BjZKrB*ksUb5jknfp$H zIVjU5>S7^g)qw6oVS_HZ%jt8q=HNT8DDPMtM_|g9l+rd9xmEo}QT8D1J-@lOU9u5AhZkSLoBtiU1Le2-SrU`S zxzu7R?p7+1I7ub`mKoca%LhOOpuR#$6j8ADCh#CF>GN{t8_kSIUUv-3^FH*S`hwzzQbKWF2>w$!Esy@wgd-tXXG z+4gX&3J5d|fmDY7vXx43KI79;`YZE^SN;YA0 z&YYfCMD8)6$uJ#BK4fBg;z7ZbP?p4lV?-n{28>Biz;G$?C`O0CP@wu#vLT$?A?w00 zqYjZG7hY9tLh;W)U!b5<9dufMs!Y*R&kG!1W0fGFgyK@yB2E46DDAzL3A$ZqcEl>YmLr6PoK4O?ZL>fx20>nG6r?6fmJCbxY)QTc-p11&N*K zGLXJ40Mdu~JXoOg=lY+G@J6;%MuMMpd>gg=j_S~vYzY?HVS@Fohed;CqT?aFlPSW; zj_$%R#|H6`A#6l!A_?2Zb^BfAzv;$x!a-z8!lS=dD&Mv|g_(UpjMUQROHOIT_zOBK z{GyG7$Q7moR11!oy5qLiBV1fhSy_SySu<7^NRSWlgcHt$b;0Bpz9F%R?1rN#?|!iT zsBq)aDO5!c3?a}&gIEH#)MNNdLLV1V&^_4lz+Ovt@KBU>dK4+d)8q8KoU9=L}}d^ z8hMx5Q9f2?+S^$1s*tD4pJZ?Ko79#f!Kji_W`hkC{JZP)?BEFoDoFteF@ZX0O@BV` z`{D#N>{Lsim$mB{k|SE5USWYDyYL#(PO??|@U4oVgAC6+AvlDU0!}4}jfQ~1I@%+- zSAo6B)Zs-jG!X_HiU<)L)|367%}28e9sL(M#?>`K>Y4=KwkdTYk z)=X0wO_@4}t6CEdk2{J26DaTJ*vr-EeKR4Jj}OA;y5U%IR9vG4IIH&_%=r!}28($eR6_z6dCMot=cg}gP$+cn2d=DJGV zBdz_-R(j91)hsFV5iytC(4g{SP>miOeDiO7UZSMrueq8uU4N?du#B$~bD@n?tBDl- zhe${82Z{PcDb)k_Ar(D6gNcc9%>lutf`(;8wh6f^%`nxu(ux!)fZDurB`8bwG}p1u z-E?MXMMy6Z9YX_WmU%on908n=Gp@=<5zLz z6E0?UyQpWEqA{TnfVXj`@A`XEQd0K~M~2>qe<&=S548qsUNdNjxdP{{`^hV3)v|T@ zb#?o9X|j8pq8JLlx4T+T{-$NvDgRl91$`pK$x&BJteSBUao_F^8@~_E9O0iXFPX7F zbt1LbdeGc%la<0h$2Ny}>cdn`zp9Q0q%#HPQotECy6S~74Nn+Zb zcTIJ{N39#{M^pm1E8OMlqS6J4*$Lu;V^<7U!}h5ugxTe8P-$=hX7nzeNz zz66>Ajp`Ivq|f*x|E!c+xFua=9wt}x{16$FXUuL2n~2(#dVl8wv)3-$&3Kn$DJeK4 zFUg{Ol^$@ivn}|@bZR(i zYPh9k;+-;D8U@!d#Rz$;m2J?G^P*rK;v2~JkD2`fw>A0ujQG%@W&U;-;{QdO4Xm5!k~@*E4K*;6RQ_~4e~rxKS+M{%n|iVi#3ssDg0yhm|6 zFTU&Ppy$4Mh>SGEW7FO5Xa8PoHVo{*EYY{Q>QchSlCRND=)@j{oZmxi^cK*paMBdC zCn)_uFd&eW*k@IBILlS{GW ziSpbj_BJD&uKb52*DabU-YLNv1efI{Vk(hvsGbU5F#DNke)MD!aQKHllSxeLyhZe4 zxrxfA1YyFI(W$Ze7LhQT}!qm;OdztUvgTvCZL@EA=2r+M?7`9lF73GqfL7Y?eUQJ&?6gf0cf%nT!q5kCiG(MLXYR4oq#o>R@z=#!b(y>og z+FH81RWtb3I!M-btK!DK)Zk=ltm1sV{(RDP8Y5ufa=7cqXNCp?3YdmWQzHdnIg)gh z2(7FR3;9354;w-eX^^R%jekM%#Mje3k| z*CxS3i28Pv_0n)hvc}zNJ{R&XZXyn3j=QygZ7H|bbgoEb&#+1%MQQhWX59eLL3p5j z+=rOiP7!vNADzaTH2aop!&OcMH&A=e<9sEGG$nGY2XmS!aPgN7$#u%<{zfSEhyT1nBiF~k>m`VaxFi+afJKk0|at@0hlIPV!CP%8;y&enk+bX@sSy-Y& z`nm+~Pj1UN0SJja_9Gf90x)}|*sr}y+{nIaH+-;T1MbouY^-@nnJW!nW8JXfwAhN5vxbl5^&L)ajWBgJ>s;1#8u-VWhCR)9(P*D2&672cqP+>vb1w2>z%X_F5Lbz5`59n zYN_tJ1S{tah>I`y{NI?s-p|)lPR>=NzpXMDv-P%!c;Bw>c6?*$Fwr3MVU}Z}(%?G` z$j~GZ51y5TVBfDB)})9ubNuo1$A$UAfDZ$ybcvH+r`QaVa1CF=mSw;ph?F? zdTtQE;)Q+I>h$Tqq|NpjRr+!U9Q+r1LgY{YajHGKcUhi=*xH^#!?$cS;6MTKK>;wR z-^dO5zU{w@zb@Q%gnT>w>#6qp6B0$gHbICcSH{K9=S!K80x_uM`jZ|*X-rj|^qsE66 zzH5?^<^z5;bf~?L^$seZvhpv8C?j9ZX2BUYPvb)eJ?EoGou8@OR)_TSes+qA8sKo$ z5-a&N1VSANb(NbLizUn;$-dyII1)%rFPr#56_>cGb=LU0BFA(mD3XEk=c2GICxrE^ zE)K?cmbGP})`i~}I1YAoW9sHPo=@6QIjnv@yf{G0Q@CIB)d-oKE*v>JJ|fo!B7p{z zHDD{&y?qb8is|qvkYL^d1{{K%MHguhKRox!V}W(Mww)`B?P~;48bDHS+%?S%VXAO= zjGrj@w@1UHbr20V)yo>I>`=v!W@$BRbyO}12c`-cGUqR-DB z%wHn@BlL*K`#tJ=u_)s_&0BAJbs^{UmWBTTmQ+vzqdY7tMWLe9{{Ty@CAhft7W~*m z<%iE#2{tnsOS03qLBjE^NvUQ(E8IoTyA*v+-9*H!CQNYX5Q7rr`La0t&X&x%n(E8x zMQ7--BJ$qTMP6M~Vsu>{S^vFEO}qk%3Eqcd1hhlo2Kkisxf>J~yvAKbX4sLWk)v>yV+PS$C3+8cv>iA< zz;N~SXq20aNc~q{wOF`rk)9deqr8TeJVFE}K7h}xJ0jWTANh|}D*RFsRi79RNWxTy z4y3|l=-~DWp72r&ww)cGsNV&=Z_n$ z(dOA!qdZNPweF$2D-wEN*0+ew(etA}+?9!QWz2)2nDULHMR*)6TH}#Oi6hU-(&Bn0 z%*|uJUSqhnE^-<4DIou1+^3^n6Hj$dcQksqwQ!T(|(__ygQKcts~@c>CaTyxkZ63!d1^ zn0(OzBRSGXM>cx(?=zqDX%eh0of5L*`w-YcS=0t1 zksx)Rdyl=7q=hdW<*D-zq9|G4!+yFz3NC_fG#)DpTXeZIuCMG`x}4MrNM9Hd74pZ` z0nUahC4=%FJq)ND{@s6%0fQYl+j^D9K%8hK(f?g@+|*W2e;frVkPknequ;qb=8in$ zGpopS^#_U}Glwraqhm7Qf*nb(NZH-m@1qlmZBCMiht0xZOQzfJhJ>=HvQg$#nzZq4 zz1G|ED8_;39NbQ|5jr2ju|Xh1J>bkqkKcxlB-T^T@EPDJ*uhKp-nQ&t`(R$Zi$PTL z`L;tUl6F(4RZkXi)Xg6k^t{BV&UL~+dX0&MIiWPQYnJgWgU-r`Oo$s4@Z32OO!D{a z$e9E!J2B9N5Zn&Umj;vmX+qNjnkjBpAg<%BZ6@3ce5+VRuzFefCaOO`zeZn?*_~eD z{K0CGV3~txl7|K;7%fUd;&9^G;>LEg(<73${USZ}lq0X;Q3)vJsKbhW=eyCxSC=*R zN)vo=wLmVWVYB*PdYb=;C$5!e!cKSk1cUL1=Y$aG=YLk%cUK=*wxK(Wr09ED3f;fw zRvgQ+)0n^anfj9VPLV!zNk8k8TLIyKti3(zv}LMd14+S7w0WJK4vLhElSp5?>JqUh z;y_r@VzSvcW86A3t^fX1GX8 zWDhuO0p2ex924r#o#{vN{kx=j)Wpo~h?x2>)#dW8?;lA_+?i1zVzDF5q^FTnJuf5} znOE}&#s&0LL~MO=sd{WG-)*2brLX$r04@lx*LyCeOfFl4MWaoE-H{^|#l{t}^i#uz zaGbHH8EGqen{QXDA9%}#Lgn_|pi_9midy;2r|2KEQz)Xb6h;A6%rnHBPmF1fNCW!x z!z4{7FMB-l%74gHocRTAG|M(CH+NA9SS67ATP2uA3VGH7t^#t_r#%Ix{`d# zZ$!Wo(IIOSd>ZB>yY|uVoQhGFrV_h+!H%9R<(v$TCPcoJFp@Q&LJJYAEe{rgW!6=| zj0`zP88BS5^pqTi9nkyYpm6~jcDftQ#To2vwap0A%cXA2n{Up`aBR}zl^g$Ko)z;0 zmWS{m2D4qD7y)2F<e1`CoylHBJY~e+!QGB?-MAE(d6^oc$l#)qmE7>OMcK0DmLa@&nPCq*$H`l*%H31h z(tgKPm-$ z3lt!3#{c5Q*>{gasghX=OA11@-QRY`P~eb{Q>8H)zy=m8Y_|B_-JA#=y86ClsM`ut ztn09_wye<-Z$eVeskvz%{rja(Z$xmy zS$E<(^&=wqmFG18B_wsUEif_O`K;xwR(k{G$I{rZm8j+I2%=*(8m?nNfvU%=mYcdn}IG7Ore=l%k?-w5)MAm1+Uoj6bUAElXhJ)KbMEg;kifj~CRVHEka zpcf}!GPri#l#T#ddHb^4t_?j{yP^b6OaIsP{d3&GU5i z;tMc!9ddAUK`nidMSDcdz}(c@3V+B{F41 zHez*e@sbvIrggdv_sY9nwKem*&SfBkV{ zKjQ;5Sxvz9Nx^JFI8M1@A(?@gcmt5DOUZfL{~dvi2`$d{nfAojU3G>XbYyP5xe`Tx zJ*500%pH3Ql;WZY|0ONKoX5{g@vM>%xP2IG7#3w170Om=2sHu5oo;uyAr^a5xOilv zvI8^UJ~YvGk;l0AHg%t4kB`C-lQr>MA_c%P;T^do*QmsUhgRo8T8m00bdeOOwZ+f1 zpY(v4r3{m;<*}8*o=^^kiG&Gr!G4tL!0~~sd22c#>7?B3bo&#L!HAk)CQ4QXr+?DI zfXRuvSMxQk`2O)BUk4djIHSw4e+9l~`X*nd}OGSBO)TR46EuE(7HSXu-# z@Vus*LBq^opb#H%A!YSe4D(aFA{kZ$`DP_;Dnm1+xMH%lVKk03U7=>pye>oGMf}HI z{=MhUbA+IZZLN`J7u_?@z>H&;mG#{g3jBlcqY&j#<>f)n--X>R0{*YYz7<&_Z)^jR z1^pi9FBsfe9lLGuMJ0bB7 z<9jSHGo10~&3cEfe98CW(&13ZA%ndX#Kmp&*c~oT)@|It`S4`S>}5NR_RV719w`&FPoH-AdViwZN{`MTT3G=&EqL7MMAI+ z&@}3oSJ){1%5$PooLWkZ?A4q@U1OCKB{Zmr=L!?#p4FThx^z zC`+XaO^Zg*c&%nbHU9e>37b|P@!eZ=-9+1!Mv76iefy*Oz#^WG{Wnbdo$3{*Gjs!= zD+Fqn$}^s2o{aXe)|HDaJ~Xh$+oBfXR~hO^m=8p!?z& z1^w}YKt3F`1+RJ)2pOn-a(}v}=wiV_qIw=`(`Uc@edG57iI(2%#;4QCbo*}OQQufa zWeZZMs+QP0_a^doH;eN2v?7x2d-ZWb*z=`ffjlJJ?{AG?BM=5tF#m-CjJiPVo%GngDE3ZS5`w&> z1m6+mT1dB-2zkD5aC^E5K0G3=KRQ0niyJacdGlItfBfiwHGFR0>DsjB1}RSmdffZ2 z)@ALG$>tj4=Or=_NCE*7D2536=Tx=VS8SsnW45_nJxJ5muQ!ZC22|-(h{|Jaty*(` z!E5Qr!$;Q?=@gN2x%lq!T1)Gld#=JVUuU1latylqZY1RiCY<3bO1mTx@v8w-JEGMG zMwDnaHLiZvUsg(URH{reTm^}`h~sI79s5|~h+tXHwsBS_ITiRR_@EEH1SQB*u6h_; ziglfm@0)3Pw|$xsxW?A*pd+PP&Yrx2c4v+FH`?KBuiJX=^rcQ0Mqfz$|3aM2COZF3 zzh2RoAKwAxQ_rDLSgr$`(tu1P08paxiRKd^r5LIfIZhzvb^{U(%w?8YOB4%d*rjtF zw~Fi7X*T*wt~8+s_gEi|^MI(oZ^fJmx*!y+p?8RV6)r*-~R1&3X^w<_g1rGo1gidfS2`qn7k5fag9m7Fw?cqmMS?pev}kdMQrwF>oZaVr=X~?d zoSAdZ%=hQJr$5q7W@q2JFIj6{ThHe49_|+v{^drL`cTq2IPo$xCh2SFQUOj`)@kTx z$Cwi=S(sVBHYAxEdq&nNiabN=1erZ2zucc2EjpOVJ8bLpy9NjMrAqXE^$u0`RMMaT z9SoA^ZyfjBgX*7q0<}smfIQP-UHfZVN$QR5*2Q93JdOv1d-EHZIzv7jcbsR4EDR?j zxuZ{TL70+FQMR!Y@iUupI9MC=KF69*Wgb3NCYt#5T$Hlz@Nt)UV$9$2n9*VS{W5EDa(gTa-&`-goU9Z(Dv59QHhu=5?2^a~E7ij?|X$qF8WwF&{pPuZ@9;Tp@z=DnN9E^~xaxSm#qmMQ*016Jbj^dn)2)R*H>?b_ihFTQPmwltUUBHM6njhnu-s= z&!H;3bh3d>)H&s{f?v-b=QB=nBy6OJ?j1DG47~zb=?DRXwWewHix;h@L8LMUWKQE5 zmE0RCxehBi3digAun3u8-R3eHxqjw}+HWi7_>*h0r|1UJagw$aCK8AEBgF(nsE zthwq$`Bp}Zv3dn;iV}SCmf^|gz8#Mt5d@S~=pT_XRVS=47rZ{mMghG=qWA-ba{QMP;IY~GOp7eESPZO{GEx@*~i!a z8Vc#-{tL7rMBzz^njz`n1&NFEPVQF-5$>AhYQ zirG|j(*+~QSD(oDhv2A9mip+>XPEI%HZv1I;zhe0&l6}wx7Pfa4=&df44whU%`bl+ z1-2XAv08N&7-qSoTQBy_(xBHv3;TYd$JYOG(e4_T{muV;_$wk-k ze6rPk))NM5x;Kx)9zr|aGMq$oN@G|{TK+zK=?Xafk7q=%$b_i#!)7-|0;{KQ9qebl zjH!J?YK~$}SaKgzuoWh%cXuc*LAkiI*uoIAdFtKN5`4Wey6mM9E@mp-Fgi^JT@ce_ zb~YX2G7|e$hOoSe$#|fdys8xba2e{^(6Xs6RoKkWU2eZs8MyXIW94%>W zWtIGCy%@q)Y<2;(OsrBh$1vxKmT9A^c#@k~<;|cAu;3;YzDVrD@L`|r)1&f%GaFJl z5wDXkH&&;J$*PS3Cr7CASxb8=#A_=43>yBqfkF!$ErmMJN$;%GZm2&F`GPQ0>8#>M zGoR@4KyWfbtoNKWw8Rqz{VM1$mHe?;eyEb zu7R#EodTvdbkir6xO-`$xMfOB_bu~j^EG-9lNX(P^4O-;nM;S>E;9);XI#cpUVVb| z{|V6%Z^bhdGK7vqz;Y)jNWui%H~orgn%y29f#Kp|2)k`eOwu-DUsY91_+b}QD^rDC zRMqQ$a~d3;uOo zBYaTU9?pP=T~K;Sci}BUOPrU7jhsX&Q}3c$Yu0 z0g?Y6#{kiVD}APfpX))OeE=~X7CVlUK#c%yltG#2?ma}@v_`dhrWp1u<$gk0eGvU;TCQKACWS*8}`ZRXP(ov7j zsi=`jQVlZ1|M2i2P#C&oeylH&*Rw89g~_y9q|~C8vPv<>v!y+Hp7mv{PCeI=!3vqK z`234H2hPe-Ac1nk2k!MIyNiQEC06~8pmwn2rt3i~1|gEdfB~7FSjq`~_A3qf9hT@9 z&J942Ki;fUSSu=>d2X)ah!0!OTq3|){k8XU}& zT$|2|1|=Yq!mxYBzadV}pRYb-^l0IwZ*##;uDd)=-q0dr&KX{-G1g2LRxwn2YzhBT zlJ6_e?b-8-?NlCS$L3v3d3I^?2o1!}#*E8{?GxBG%?Ue&vP%ouKdkDs`fPfQ4?R}qa8CPfhw%xlyFIqZ0u>PS2aR*jCw8xJkN zKm6!glj@UVGqqi%LKF}}J_Aej8ZS0{L+oj4jm02*KI{9QY?u?bI{Ie(Ay>`7&msU0 zE3$I+(JZhalhm1t6H<)KZ98{3lb?nQu$7#CFsh@fCcfrZfP&{Y&nwqrx{h@2b3&-(CZ|2#D#qx+koR4V|}=-2PW8;=vCUJkAKlsp~33H8Nz@F|hfE!=uN z!1>-yRNQ@(-FOO6ByoE*Y-{pLnr!iGQOxvarN_Fmbs=;22p@8ode^$=cs^sf9+k)80&G zV8jta2M&Ge^0{*kA2i&Oe$D8hp~c)3MI6N{X*VIHvBjHqEA?I=6L z$KBXkJlg`-7>f*9ZaJs@TB=9BGr3TPe&8oB?PHqBt-oo%mH)QvSin;cnzN{)%+S8C9S$A@3^j+E*Ci zv@J>Zy}>Fo-~qExgp72D9{xq!Se`(8MS3ukLuxUtvWoI^vW{NK=+Y#gXei!R{8{YVGj!u;Uqxuxcgj* zsqiGoRFUjOIL>QaOcACSlE?H<{M;y*>6$h?a$<@(?+6sW2MTf(Zivj3jgFKckpLbm zUtK0I@pqT&9;AtV2nZsf$#5Q$JceI#{PkG8#I5h=%q;N4MgKq9zxVbfzr&nP7Azw(U7oVs0Sx# zMV#;fp(HqD=p;Pn0%2SGtGr71{zn2%oVb-itVkgg&%;l(oJ_Dv^HR5OfTZaxAc+?g7Sv!)9unCAoFG8OEw%DeH@S zET>qj{)*8(X!)v?^8^~of-T@BqlMGS+xf;Nv;W;NCc1&fb;|Pns|$HhVeL~1{L0pe z@u}IVMF)}HUM){&l+E5u#-IiK{)Ab=q{ZrIPl`!J$u`h=@y|hTPETqM+RpDhf`e^> z*a1oG%4i2|xlcxh2PoKu(Bjqf;y~?Ry1UJJ{^BT-7 z7nCjib|xi;aF#AQj)gl@@nPJupk+wGYnKu%QxQk>vwqJ#MF{est2 zTsN``STVR?BP$$}xHIKFpgr<0yn;_@tGA;d*h4qZUy-%1ao?j#jIiH?y75v^dL>zi zxvx`EwK8;pLX}Yh=F(C&J{@MEU-M@Dd8wJPLyw(6@{!*lr*`8NCA_=$PjGdLyWp|9 zF+vcKYPcrF9RBvkJ|Ws}K2?2~5dg+`)Xmlv7NY)o{){h7N=mww!`bq@+FVhax5+4f zI_&{Gi~z?y4oLR5^%Z6r0f7IKTnfU}#QZbQ@1Q8UAcp?Rv`Vs5l zcx7>2=3 z!|7MlhB?bhlK%`lhT=y}i|r`Y`H0AT+Sge)zF>0yTevzwS`&c)@h4w|toBv=7mzlT zo3_ox;ShFWB$34#K>a}jmoPsJ;=r-L!!+^yXs_WVz|tg#V5*_{n$!J^KW1>egUYpb zzsNl5X|dv&2=F!WDs-EUD=KTeecj%^9JOQ=^fxa&15q`V9gyjTF1aPT>q#+fyBz}f zUdG-_r2)&C!ZkVmHzRq)q8J#7roql-7`XpI$f{e7Giexl1?@vR)OkSkbLyolveTbK zFP)ooH2?*-AJs8YifR2%8!5>bop33Ll}kt>$&;WzswduQJt0*`s`q^6bAVA$vOcWG z5!p`C{-IORW|z@SlYYNkw6}m$pKk0of3I@DFQF;v%j#uPd<=}IkME2^mMzo#jSH0< zRkgyEu#yR|HUk94S;9#X<*~ zf%5ut#oeGc9mf4a23ZiXvwSF9yx7klFcA9XAJ0TKxAQS>sNHg5iV695N1p|+DFgx` z*#Z<@19Ao5%JZ=J!zh0Eabz8vecw~315Gdt*jsi(Un*kK1*wR~aWlcBEvs0^xB;dT za6c)fk<%a3RbNtbfW5ayN{97El{w zo&2I7VxsJ~QErn-eczSIs%kSSE^wR7%edg5OW5lq33b4A@v=^>mC@raHVjOKiR?OT zBSJou$-_9Q7GA9{A;kTf`EL3e&f{_1(J>~QSe8!HNz@4OTL?E><5JIU8O$5_MGShZ zf%wN^8P@3tbI-z2sK#tskr8H2=i0~>9cT`rNi|jq@ihusGx+)h_L&M>ziz>(Rk{+H zd`Wp8^yBBZyKniMUy3!L<&)z&lW}A}p1q7@vSMg+Kh&caL-)(?Z-nJ#13%Ky@m!?? z<2jC*u}pVfiDhB#N7BEZ)b2cGT#9Z%h5R0n+B_0-qrfr`u25$S=)^3dMica;k>h-R zbjmYPOPQC)+`G>J5RZs*(_9c=P`(X}UIjk|_Cp8eLmTFR1{ z0WtWCoo#aKQ-ltmJVihw2}B|+6W3bMeiqT145;M{-P0xfH2Qz=9qC~4KPzxJ+fGjJ zTV6)O;zZXs|EV!hq5vqE=08wKctyz}SV@n=ygw=;+b#7GFVk~uMj@+`!_NeQpZdU9 z`m-XfM)gkm8a*q-iZV(2O(smQAGS+FsTLygx0fR{|6MnbNphLlIdLC^Uq(ExHNoKr z7xBAl;qxX&*s0OO0IA;`=f%L=q-?dy=H>G=WfEG?Ge1&KPtWag3ad+a~z zr80%mW-}&)ig>`>dQTf*3Y$|&(y%|p<&)6OiIMB#?3)%xM4pQR2iC5(=6f?{Zfe!%iRtlp!@0g}wR`yN=51=Km~S({L>0 z0n7RCCo$aIOuFS?<8F~;D_^193%^(xKVzFwnZdQAq!2uL=xkp8k#GBhI1Bm2NfQ}E zaMd(qFQN8Sv<=hKpw2{=u)JQvD*G`98yAw&GICFisy?^KDp7+&GVN%5nebJRRFB_o zO!f#waAbVzW-aEu2}@z%Z#6uc_tStJsysvWd3HeYHgfaNeC!svn%2w`K z{1uv2Uiqb;hA#{4(rcv@Rg42`UK5QezrH~$G3H4OyfuTLy7KY(DOzpgq7bnL0?_Mb z;{^#Beo)lN-`ejN#_oGLwzJ}-10V4$f-VPwZ;!=w+wK}KYYtwOUsNuaR<OviGZYJU{B;HFRGJ>zC>v0~!JR*TZ5D_G%&D-mbu$p~OclL_j%lj^#Wae6 z9L~>rl>ZhHOJu5xK1saT5=tJ7X9*hk zW^!?Kb|>Pk@I9q|>?cY_ck%vr&Wq`HOMka%cI0S+-Ip0tjDz?1`T0wma*WTLOXDQA zR)r0RVr1zhgg1N3ot>q00st@dfg%Xp!D0)3n~p>~wKIhv#*s$S8KPOSwYvL?F?Prh zN5b`KHcRD)cosGVZJ@f>Y;1{PRbT8g*wT|7U5;wfXj%PgM|VFP8``R6<_o}~f3OCP zD!=Uo_mFFtVp|MjPSrM*yvrQt`MC^d`Sgy2GjL_F{_gbL)}Pbpf8x&!=oC?Q@CRuP_BPHv{4G&DvRZ)qj6Y7lz#10a} z6Os1iN;8$46*F*E1UueDL0_$9Bga3O&#_G*PmsKOEf=pt|>Syk|^2F(0o4J(n$znXUQ?2@*ef6Pls zb!42Hw=K4$Y*j5=_Gw-B;?lcw84e;+(WuXP*}`9W(U)aQOx5vOu`t_zea1030D2z( zKW+Z%?X`5BFY>QoIsyWmMh%;IKKEbnZ^m)}>@x-k3Os+Ndmronvi}qkn5aAkxIg*Q zf=)TcGKEy@;b&IpL0;>$A5tCPm}F=pBLwRDQjJOnGiTo_g0nOjNO0u78uh)JQ4q_y z=6sCnsEONMRw2-L> zG3BmF#(kD$c)}R>=5ESWsP?*|*%J^k?B`l~zhMmf#o2r?(ds&sIivRGjs>y_uNiW- zcU}mZG9D7T)dn5nVt}lHUhFok#ER}WtbEpe0-*DL)FEOjM>%5g-2>Ns2XJ1|VgOu9 z7>GP3JZZ~vH*JOfbb`F_Qw~W94FIcj35U}O07mCr;t%wE*WEA*ML;MBmU7NXCDL0Mn6?JlCaGp|js#SO+= z)fX({{l+V%C!^a|FD&DP@1`eQ-Z_ig{?i6ILq+$}1o;*_-~i^*bGSJMeKe3;LQ_Ph z<_!1_W>w%5JPdmn^2M2L!p$<_u&P|JT*uzNC_6?+8%A$#Jaz0`?!vRh6ZXmss4FE6 zWN&54n!=ES>kNpY+}U4np#!BBcaA7s!hvIBC82($eK$ic7rw#JcR8-ewG1#Q7v^=V{joN^{*x|m8OQb#do zH5UKL%lt5A9p!0gy(6!4&h=$-hXQuhp|_f5?cZF}Xy;Fm|D?q+A3g7l4RV0iC;&H?P_!}hO?Q7%DExTi$t(aA*x0njB6W8x$#oCVBE!KebN! z92Tk-7wUdJw=823(bRTOdsRh6M2$jWCz~#B7^LRR>8U;gxHGcFvh}53gozK`rvOu0 z;2+uKOYyHEGqP=_1SC1zAXiGkALl>)j3q<0%!Sev-Hh?lAo~7XLI)|WotP4@zJjWx zFc*AC7ywiN5kktPWx6w852QB@gkW#%JPCR7o@wm7_U!9(S-oiKuL(tIT8OZs;j-e= zvFDHQ4xVY)*<$vhOKiLXia!=1=ZE4nC6k|A#;kt&OkfW8Y8KFJeuDk3%6&`LZ5%M` zASW+C*b*PRFQtS=*)U{*BBR#niJ#s0A27ZkK@_sTRw!cJQjCOAX6fRBXhW);Sy5OX z%7x-pvH-|z5*pT(r>;MQ^H0%9k>z_S5+jTJbl> zofGbrL4}09sujzL32zo{#H<-~j0k%A=eh9rRP8SLsZpGT6ZiGPLZYvypE>}v)QRgS zGO|ip!NQTjlTGPbPszp|KyrcN3lP!)SqgVmbi8uHz@$|fLyzk)n z`@!SN4z}^T0PdP4t@GLH#R<2Lai)$gjNeNyfR5NY95P$*Q%8)tqWFidZhhcIc8aRP z_X1E>&J!k>y2He{&b!7Ke*0>TDg3A&{?-ek%x+zFxaA>Rf2`FmHg#!#eb6+ukk}6e z+7Zm_N8(oCPS|XueG=jO4+B4S0;}z(F#);*AbLJL80|y6D5`oh+<<$rndX@UxQjv#j$Xu*7vw?iONzB zAp4xCIVR2WvG6UZH&6|=Yc!@<^ws#=+ixShY+340*rW9I0rqM=vkE`6O5Ha3u5TAO zT)gt^6~L}#iTXXk+4ErK%ZDwUqzsd7QL2zLlx$CV8%cXW600@Ot#5JW*EJgGxcTY8IXqK zT(jGX1ZoOs8)SetTr{~07m>ILdgD4tU-*B?kDba}qb&6NS^i8;2)5s=ejes!mPmj2 zw2TF5EyJW-HfIyyu`ASCJzMK+4O`piFaM0((&pZCMU9R8s#_%3(PK!M8D__*ssu;?wBbLbm1Yf9e^7>H zS!SP(L_!G+pZ4244h6eOk|!rF)gaBY=2ZkRCfc?39X?1?=*_|g)nIPGJ*f4H{yz}jRl!uMb#t9vF)mGCc zPfA)=NGQi59E?)zei2h^_kfr@kVS@PxfI)yLzBkNyFA6&!o~2nCyxaClcnDV6ks%n zJ(UHS_G$wx9jG19Zz;!|IoZIs7ZaK26!?XFOx0QEbXZGV%We zf;>2RV(FBYju@HIB1q^(r|?;))UJ~9uagK`Q^p-W{}w>dsi$QxBb}j5ldbNkR#sDL znO?5dKh8?k=%ZPZznOvphYv9Owz-#0w2yCWUTw#dv8i~~E zT49!EQq{3^dqb(g7ZdwUHalBeg_%$F2a|X0(D2ZZ_dvhTP=8%8Z4WlJ5QS04D@NdL zYe&m2z^E{F6oF`&`ISAK4b{qilp+uzvPKC`m!^S*yOiMB6Sw6HLz<}uayxB#~ zS{RiN3NxRpL)iaf#5$3>)X{$R^HI{{ItOi8JDwGvHD8h5Q9lW@ki##cL@5Minid&~ z)7o@(n@|ZReBM~+ht>T*i0(9u5d@&LU`~32K$z$1vikO|%Oy)8iB2}Eb zJk>4y?fHZH^O=U@EH#&J<%KPzo+c6Wt2+>)3!45;J9sE3k>Rk0(E$lg_{)WFzm#Yx z_yo#byY*gcL?Tqz8jZYXfK*puiA?lk+W|LO_l~#xDyZFGsfSg-`V6aDu3Cw(Ht!x) z(c;CeUiOUR;KOya5!18>Em!F|{aly~JlY9#mjo!;X$E~a^AS{YAGq_P6i)RT9Bhg$ z2Y($U_jIy=>=2OVbdJTyc8 zHsv^_{l?mt4Tw7C4$tE(^oMUeYr7`(|8!mQ^frOuvd#35u{`Yaj=x{;plIxhuS%AQ zlB~Vzl?s)MVanVW?;xGJ<>y3}5Ldne#x4Udqe_)j{+Ifs^oJ4VX$X7rusn8(p~2nv z@~olwfTUT)%KGu8)1+bJ!+QR;^z@^_7*#pBwsI8wXAd|}TH{|R<=^|YY}-Ivt5OhN z1)=S(_Us;ySynADrqfg$O50gA7CdOY3`9I?#EGnDSVD~ta-4g-Hfd`OM0JPSN!FBl zH4jC=CwAS8ath>_KnQ0Lq9UWoJb(0_#4f#VIe6SmI^cS*zJAHwyV3DM$FI#AOS?+; zi_y^MUw?0qCF(A-TcwBDlM}Oq;bYg=$|abLCvq=TM%vlSc|2U@-MX5`fo*t90Td6T zAGB(u#;K=2eii3w7#D`As>G0sJl|r;KkiG_%=+bMVPNcK1+&rxK|--6ch!qOTjy5@Q5}d6q-@$*` z5-b?@Qme@Ywqhy9nVT@7ND6aJ^=0aC&8^z#Hq9_`O?7>`0fbsH3G>sVsCm0j zn~Y1tcRZUXF((LLzm%uP4?|EixquLFzF!0O7r4M|8w-q07Ulvft@r^zLL)pUm26sK z!6n~hGKjTPg?89D1s}!YU{QidjpR8$byL;YKGr8=fH@C5KX1#inMKy&2tP zo>UYkQte5~WF!x3EhT*90wrYPpczQzU4beAM(&(CuU9p(wHxeJSP_YA*a#v*MHU&D zg(7C5v_?e!E9go(=WC_PvT8QA@;g>!_V45GRsG}{% z3J$S7=3(D9{9=_BGUI)G#VJtGGk)G$+^ii%y@5|{&ZZu4DLVeIdf&M#X zDcY#tCrGd|Au&vIK+Qv<8=%Wjb{=v3hiQ_VOT{}?@11|;-lF=-)SMm?>F3IgN8I!P7dDqM$* zs65s>$?jxw)I@``uC$6w-q%Xo8-F&qYc9e*)y6y$AL5#=Wg?j|qJ`Y1yjcN$j)`ib{z(p{D(Baf(61W&+22;M zAY15PTXtFa5PUiWF;BU=Io3wzamOH^M`cW-!qV37k3F9_jrVO;%-f&Zud z|GB8AsJ{>BP5!6JvH(QI|7s^#4}oLwUwfMa(BJ;6{U38tfz^0qxTQ7MGxXnxP(=Mq zl5H%-=&%!T+-`8!HBV$W@A!a^EUS2Je+m4(19NI|*2qv-ci#X5?7%O`pVW-~mBv#6 zFmC^~{|~*M)~n^KxacLsBQ;(Vxs<#2&A_)vEd`|EfF|1p5COvvPz(0q2P><`3rhPJ z3)cVs?>f2Mm4J6YXG%@^BN@10Tp4^Uao_MAMWJY%MWvf{9<6OFcOL)5^!-)%PUXdl ziY`5%KSI-bjn3I`XbU6;ZvR)p_~#HsMN`>43dA}A*y6_aMbJId>ejt2xjWwF3?!jApe;eA~T0VvBKA&|7KR9q3^=pTeF2zd(R_?9u zpFEQIyB>6=!J*?eKt|ejj%I(?cESZ6I}^T$cyo5Izr6OY{p?Qy-oga1dW@%B{|*ON zt(}RInut2~J0D%%05kXIZ(P0q-da!lo5T5pI8X1|iOIN``(Iz{7yPCkP|J)2d3kw@ zxu2`#+@HT0x*T|2C;q3v!y+=Oi@F)_+jYd1$fyZw&(U#IG;95lQ`Ds~QrIPIXOgR=dxMpbL&mkC>IytKBcF zN$)O|tG~X{&__*)KidvF>=kuEmXw=TIblIIXH885cYoTV35gdNxo-UL65{S6Jr+HZ z4z9T*&d=Yi2g|T51eQ8`PhUO*t?Z`^+;1OGnfQyRDCka%?>7#AL?SEnV@wzsL`6%H zj&%!e8%CS~J6^h3?n9w%grth_C_D^uWr7cyWH!}O#t2^&o1U;8-wK)p{XKImb=biZ+tpK?Fsy9zU9_b+ zokbtF#MUKuwL6KlB0I#-yyo~x+pn*bJlbpeU9PE54g=oZ9wL^A>LmUY)G({e75p_T zmzyoE3)^fG+ zX0)2Po+AMK#K6TG2Evuvx2aVNO0go#inmxHC)p+*I6b3Tf(&834`Ce z4kgcJy4_ii31*$%NMy@mK53Hxr1@0EZjDsNhFVwqvH`}k-O>0~ZVsUO^4k*ui&U|0J6c_A`zD$WB6J5?3zVRA#DANy-2G#&0W?s)y zpbDmb^ib}Ei zds;bKm_Sam)UwHYu6zHZFTU0j!tHGu8i`asGspP}_Xo#uiy2ijByZ|3|CUdQ2kxM{ zR-AnnPZM!~!pEnm%119mWyj!HTDDeA@k*qxFSf^^%e^kwcdHqGIa~&RCxDMu|2;pB zYt@Wef(yIAA@hQkD3Wp1IIFeK-q9sp#AC?7C#AST5);KQSmC7v%!yQ&*AWs)aut1+ zZ$En`8iYs7=eK<#Xc|C9>p4Br+e?@P1GJ4?&h|4ZN$b)CENyhF&vS0uFlqnouobm8 zb6}NjzV_=+3#P%RJ>~Vr{GKuR5SG1NyUJqZqF>5FR$wgmWGT60@?RjwX3a~+x|aWL z{T*uWMRZ%5gvZ%>414k(YU!iY-=_~SWW#u#;d-+i)mW-F?u402ToyGyyC6^*!@=WV zhvBoq8NnPBwA!g2_LuRYjmlTMUxVjwE^Aw>U@f>;ZME8D;-p#DLlGO%=BrH{^qd37-v12MC81sPXK1MWc3) zYAsLVom{YMQ9WwpfUuB9lV<-rJ)W-vY(I?`Z;l%3?~iBuhi_ZWcP|bO>X@1Ccf!<* z(dmq<9>StT4vxG%-P-tS%N-w^)t)gtYF(?APH(%F@iff)wbVuD2THW?Dm(tXnZ2> zb|WD;`+O$&naO?I397Oc@2b7r`(iQtUF+FMz&r8l&D&>`-pYBXybPrMBQjy*2dUJT(O^>jApEVAtC)c=2|IZEVxPWyBPz3i1GxMBbuS#ai zTAiqKQh3R$rIin>mZDK#7vi)XP_x|+cYS(cjgv-{M+vngB-hxH* z!F1jPIW{0@%SU$ciKoqONB>Ekmn>%M=ic3(J!)BQzHy#CDlQ=mW$ zK6d3HJK#v>de+UkZKUK}2k2HOc-(b1zgw%GnZ2SLJ9k41SuV?KD_%})F~#J zCl3}(IgRb3qVTI{)(u;aZX}F*e->!n`8Q z4t%5pSb`sDt0wA1SC20x7CfGPX4D6lT3d0IrZLj0XD88C?g?qhTfGjw#O_)k1IOy3{nxfryD`2*HJ3oHC^NMhP~gJyTq z!<-{qXs656OG5nkrXL@Y@s7X#`l{_)(7C`f%0NBSwR3|)`&pA~>V~Pdo=+d$2$`sk zqj~)~^=p8GIhO+MmvV;)sM+rY^fVqRu_|ew@V?_JE;Fh3Ov)t^n-X66aqE56xfq`1 zDr-7-(dM&1A~EA%bU?Fc2?uQT52t{a(XwLW+Dr98)854)o!`r-(UH3^DH!(=3;zN* z*n6Oa>;5+KeZ0f{Ef=19C!Wwr>VJ%E;PnH#d}TJx66kXtj@&DpQYR^gz@0It+&HT4dB!J0?3u4j*VA5WN?7<9>?=L&dIXEl@Ox4ri_zW`l^lPKbTZZxkzn&0=4-o0;QLxrWoj+HIG zi)2M&x0NY76zD|2W$Flf(!>B9su*jug<>1E3Dg%W#sRl>?MsWU{fxRsP+iktG*@iE zK27k>;yOy#BvUlNf2EJux{MjajQ*J);FDl{IJ!7G9$xg@iP>>!3fee0(A2B$cL~r$ z-rUWW`pkK}Q?&Oh4afhJ83zNr={}tQc)Gb_)ibyd7J`^lBVRzbKznMuslsvYY<^}8 z2j_grkrehTAZJJpbMSt1e6vR-wd6<|27qOwqQ}}+$1<{(O<4*tN{Ck;1y(tgD3fKuMGDy!ETyj z+4vBlSdT-l;XWIl&^@k(Ro&1<6`<&V9;Ye{)ZHPJENQ` zbcuKV=A_3qNBn%a{q{O31U#F11Ii6=1TyPtX?}_GY|J%}5ha2*{OiT6ODjXI$=(?tU-{Lye&ZXH` zH%S6hw}wvSMsF9vMr_e#!az^juKsPqf7Q=`nNP4*B3S~aJe1uX;OYBbQ?+~SMl!cw zpJg;U)om(~yl?%z^W=EAef4PJ{Azh%&?b&du;SiWTL(%H6A=}8YOF262v>Y#`4qS% z%YWY!4?1?|dH;s%W{2htDMHH_pa;7xvoaFVG8A%%4X#yV(n2S&)lxY1=kGbuYMDgX z;ii6Dq5F&rDR+-JkU0dhrI7#hdch3$UpPgb+UxXrIF;W?d3$?>+gg-!{gTE+t(y4N zfzt$Re&cA(kH8t%IpW;#dsD#Z>-8oEOY=Gcp|3T?llrWT z`F~#|jpxn3QD1W#Tq6Z=F=nlq!Iu+j6%R4G0Wllk54mE$-3c+iNq*o|pg>Bzaa{tp zA8N_kUMDF z*$-U{zujfVleqFd?{ChzyZI}nCc0AgY`+~~m}Aib#20<>oc86}rE-Ou6o zlg3^CSC`A-*{-8Mv%b!Au=Ie=G)tay6}1UxkHb6FV9;9-((C z`)zn35q-$MnxR{3nf&~mwOh)O>8k&Wy7!D~f@>EAQICRvU;*hMiqfPgy(=h1dKVB7 zQIW1d=n#~qBGLq;1VnmAs?-Qbm)=VVy+eQiAqh!lN56CD&YJsY?#!QIt#`d9*?m9z z>1FYvvOzbfv4!wMHCUP%9}+A*3DXt*Gnij0DsHcC`U+lD?w2Qw4C^XnIO4wW8|uhy zvTGaH$687eh;<1oSGga^$4IX>SeqCb9l4L(EMLhTm>nD2bi9$>u)KnV6WXBy0)o+EF^DL=dn&A zAm>S5Oa0?TWjO`mT)w+eN0X@KM5`G`6c^|2Vz3@^^`*7-dW^lL?vf4uWl6vN=(y<* zJfCW(j}+Aze~XWtb5MXBb?IlJi`q5UL)bfHj#k5w9-AET#>UHvw*XskH&gzpH=)Wj z@wWNFz>O%IPb6#<9O;SHUbTG9t-RJF(&W7a`nyfT=GA%PbkZTX{3?m63dt0Nu71AJ zM!cz26R|yo;h_(y(aW_So8ws0NTkQi_)YW&?uYyBaJ0yJK_CQ7E4^#O`qfW+Aa>Qg zFUigylfiJz44xwUAcK<2dn+cK zPH8_6d4e+yXZM^)uyD4Vn5%Ovsl`og;$${1^W*pra8nJ(C+Pg=rt=q|2QlGwb#=as zL*wThIgfYzwy=K;bv-F*xf)2b8ocS|b(xzTrKNQ_yWfhd4vWNe6z0CAUoGC9SK}Ow z-+J~5aoAxDo5KUAL0mki{@CFI&T7{cT&js9q2zNk+U4T8<7x(lUqUrgIBU{LM~?{n zXOD0_@N$Gv5)SMx3IR25lK$+?^)B9Y>G`av>wS^Ejl#3M@85Mq5*O@odXA* z*YgfyEAp93=}jXXj-y#@sWL?1$Ioz|K8{io7AIkzbY~R3 z>gPxbE7?To3Kj@d1(ss_WbqA}!qU*8CH`qtS*wYKYsI0YJKmZI=Qw=x0M^w9DqFPJ zg5*|fzFfoCF6;vA7*Q=|K&8dk%cK{=}HtFx8Ku*x)D+rYgyRMUef>LX>s~# zbHW8Rs$y&yrt1FSnRGd?dJNLf3(w){k8NYZOt_bF%O~9T#&z+wN3y}&5-fnlu(+&b zGjy}_U0`4D+lHhY9d&7)H?~Ez2I(D(k)xN~W-qi*AQPJCL>cI~0+;l2)7^#w=oO8q z!~nxC&a!+e_(1^i_8~4sr!#D7qI_~|vSxNKe7PsF5#Kir;0GC_#$mY@Gc>+m#pVNF zCbTjaPYQ0)d(8+-leQsJj}t<>r344>DW3f_=H@dve(oW?KSlxLyVW1%1aJME9=Jc) zl?#(xOD=`C$AcQr8z1CQMC%I?Cf?csu#{lZqhN^XL96+u7bGOrD|@*ARCP@aTh8QY zynyvKpf5&}BZW(+3e=6nrKBKA0bBj_Oq`!s#9k6ld=NcMf6|q5Y_yfjp=Q^6eJDG_)sGQVpN?%&Myi(?jdU&wxJQVdPAP6;?dl;STy z7fPXK=nE*v;O~^+eY@bdO9|V?^{5|q`JWvqi2G|dK<1l@@du{^jDO|=0KE^*Zoc+E z-l*mJ0VD`szfxeRt5jIh6xgqTX~Lw{Q(Da-YLMkl>kCa|!WMPMJJYUa{jyQfu+tQh zuQD4nhcM8x9>DL?DIGlRDc{`K?2|#gYuQgqaXuPWp_9R8E%F;n+o`cb4sNBmm_LSe zuiHI2+!Q1p{30+fqDXU;qobn))Ity!>G-`0^r%9GUU2+g4O+NvEeuIGqOU=QT`YIn zd>0|S4)EM|6^4GbAHNZu&i<7wcl611T={T$)IZM&U`_ziQ3@OH_4wE$+CXTQTgZUS zDvUH;2yJClJz@H9f{TfK5M$rC*G|fRN9YBG#=8k zb5~At`YGc745#$&xQ>+?)Nd!G%;D1@=xSv#Ul5#Mp*JFG@?D97&OUwi$tqN^bXCW)$fx_w&i2n@ptrqLub` zt{tymV!DdN?i8*AqbLKI#4^$ljV6+_1a2YUV#Wo5m9be zbR?}>irU5{sgaP))2N@raBy%dMU!wWA~NRH!ElsP-O+|%f{>Yf;6he}aJ?JuR*GcF zlQ$z7AdbZ@ToIe7CSeI*+ZSAQsW}58i}qiT7Zcx;h&mxyS_NQ30}C$PvrbM)eyPo2 zmT#$=4v6S^+7`fPpob#*nK+dWegz;2L`;tbNHRj9fH{s-{oQZ0{?MFZZ zwl0MXNWhz%QfjA3gzWLw`HZb%OuZmf!lmW+s)<-TOu1JN>M%SP^?1cjdDJ?q?=6 zO(&OLVQbfI9QZo?5=ccuiQDU9pq4B9hmo<-bdPQps2}b;5~Jjeg0)7;nhaMeK!2k2 zwtLVGP&jT#zXO)HiBrUrj<=<(I+)Y_nzKQ9p`J{r?_4TcMA!)w$0?~ty_i1AP&+w+ zNs)pt;#vvx9IpqOKYl-XIuHM1$@R7#)H<%z|EkouOce+1iRW7#l5pbVuCfB&0ea!N}BLxbm^QVkLg# zk>}m^BLu-OG&eNqoFuHu45F=lJB?Y&=2_`>VL?HCtj0rqa|`p5pG)uBBCg`naxq^8 zih~Tg&x(90wKtbn98&8(MRDyZn1(`gI^wtFEcyo!}G#jogKVGS_NuXDi$br?lF(gWctw(g+`lJL|Z8&Ibx?USn!baDzn z2l07P9zSwLrAkD5*mGjN-U|^}vFgJWa7=9ULi91$9~B7d$SkJXf~*E!@Y)7Z*N=~}ohsGdi4qA|Yu7)CYfpRPIl>#rM@;$GJEM$S2HilT zGQWEtzx8G&COi;>uV)7rMwJ0@-xff*6LuI>byYR(e3Nj)$++fKktss`G~uYbrv~|E zRgKH{7hR|)!nTw^ccLMjLYOwxy%yqO3Ehkw$4nCHOE%d9W;8tcJa$48(|k|#DYqI6 zSFthb;&D4-nS&xCO(*0TyfCO`bS-z0B*oKL&6Nc=e+=qelBWLkA4!&NdyoV0%miMA zuhRDWwz?$uYkrrM8Di^!;cAsB)g3oX@~tfu3V)sZ>JV#nijU^234|GBgawDJ$Js$x z0C5%VAkFE_@dxsUQ&%C|#-C{u-gzk_+oYM>36pX3L+x>LbvfCUWNpMYQ*JrMz= z-25AAcn*SovLKrZ_ES2;2<*E{2Ml+NkraE!4)ik`#WI3>Dq zKauR<4*u??KayLkeQ=Y!5U>=GYFD#3wc zQozi@7yFdH-f{1Ht0c1&r>n@Zzfac10P#v&)#aBSbrh(&ZajYC()|T+)&2&C?!%YV zxW)%3&2}D<+hxaU-AeI&ZCNuTahk%(Te2i*UL;BhRQ>fqA-vM|Bd$RGM&m)zFHkq_ zZ)gui(B)Vp^}D&vQn;`eh*D!qv$?=b1Zk__H9eeUeb6dBh8*N7czbK|aP3b;e~P&C z@XofJtlVe0d8P_UBo1G{gWWGIg@}-*k)&EscTGE8#;&F@@L;H?l;LWLlao9#%W%a> z#%t5rOMWxe+7lF^FnQWMj&zcAyp_$sEpB)8k}9JVPR!v62Z1&epD6ac8;BNwBr`9P zQB~PIOHgtvrt#DG#OU%Fm~A4~t|+#iOC45Putm>L=ekho9=49h_T48kv+fb>Qe*G@Yj z`ZYa?eL7M|Zw2L7JwlR73wgHbK1ZkycE}P)SrzlX1M1#Z%3wU{lM6eV#RTPoyC#87mk>F_C5+!wxD3X<5ruB!iRP=p<178Pz2p) zZVIv?zB?8M2e7g z(`jyxZVld(qdK-DAZe2GYBgeJY~!?FG6+@RU0=+f%3vGkqXQGN#Xk{F?0QZe6?zel z$1pEi+P!`VtEy?zP&j#Yy6g^XQ_v}H@wK!#(*v zEiFSSxu7TKm0z`l>``T`59dfoN%enKbWTJ;UBQ2))j2PRwI%OU!JzXu`xQtFZ#p;$ zAozTza*+TK>C@-nrQE-E@KIzv&UTUS76r3Tev4AAhD&;Rk+rdsi104-| zZBb|9O*luIA2C!{k+8Lv&h_D9a_#ypP&;4+B6ZS~UyEHfwc5tUj+4;s_I7CsAiP>h z3CCnPj+w@&QDqngzF7d0vB%oH5~(YS-pSDskG73Mf`WYYsOrMax^cuVzOPTGr<{l% z>@dvL+7W)f5OUtOFZ})cpIr=FWs8A76{z?M@pCyBZ%`0zVmllKn)b{ z|0N{A3r|uqXL!BEmixby7s<@uuP_jXxJ%kXczG~DTHE&O?f=VQtr2s*QA1=XW)^ST zcKRKaK^D#RN0v|#P8PTS69o!8X8%V1 zyl}6eO@?-F7mc>qMYo)bOi>7A-+lfaP6~P(teIH)B-4|?kJO^4p%D|Ek0@s#CT8~t zm=s&p2eu;oj#yJfZz(Ckb;AQqs#IYgjSHc7U%iitavkxYv9PfxzJd!|o8>*uUn>W@vA=T*`%_ya-X>StBvkX4~$nK7SdgyxET zWW9IWCkLXylM_ZWVyJef%PETb_Df+uPefNcSSp%y6D7$@_&Tf>#o|S5X}cK_xWBi@ za=bnuZa;KD*bT^3ZpwSy>Zjx{?Ku;&mmbyRy$9+GWqO`8nhoXjS`;*zj}8wH|EfGg zN$weCPn|yyMv!gFnadgP!_Z?`Z9|kpEZ0Of={9`3oSXg)rnfEGT=m@)J z8`o9&y18MutSrV>kxPo$Q_357iG>AA4CnIk^}TI*FnmJ^o{=6&OLrdYcCz|I*k^o4 zp>VvUVrPtJgoa=~xuyFuwju&*tQyJTwb&l{0)G2of1g#}>A}vE+GI(42KRHCO5T!( zeoUnZIpSxm$Nq?YU?8#lgoX{}a%YTy$7GUzUgYEyejz4FPf^cszGsDWfZkE?qvw(_ zFMFi{C3UQPXl`z9(~L(n+LdD0Q@K{9T*iieeiCUnIL=gyV!Um>KbWaTNyjPhxbpdH z_0H@b)8wjG91IN4pHAWFIT^t3zoERnx}+0F_xy6k8`rPjb`H(P+6ucLtS&0C$xJz< zT%^>rFGc>;FMHJmkU>hW>r_KFxcHcu7%QuBTODh-jcx6t()PNgpr9bxQVCW?H?8LH zHzNsj?Hi$i#ZG8mv98HBxIh%Af`C!c2=NSMuP(1HY%JTa^sCtKyIP3{PpnH>M++6i275Mr z)nC2(@$6x*u4V$?8*DeDBd_Ld&H0MePT}=~q>5Y}SYIlCTto$W?MHePi^O%`r9$ph zSnA6_y!IYRcXMH3VQuc|-K78Q8BHWmn%%Xlo|Sz3lHa+HT}Fpp#;EM+DYOSK@9M8# zzdAcRzs+6K5VrqL6i<;tnXq6ecJZPY-3(LoRB7AF_tV*V1Wyb^r~U*8y*(hB=vmZ zj~jeLF>y1Dr%{yD5p^>gqU{)%ENnSdR#p~0pvPoqr{aF|efF+Qxg8Xak+bwW>F?^Y zUtn4U&pbuRnpj{gz{YBlmX_9=BKaW@xA`$1AduMQ;xcil76yfcyDe{^3dg>(YxO2e zvPge}>m#K#?K#-lXJ&?ycxM~^Q1x6dYaFMip5J8@Hp+Y6bs4FZzK30lIra7}k|0_S z>ABjE3|n8yAC5@8jkoPZDJm+CO?}~cYOR!{(I-qa-qS+yuj3-wfPOA|X={s}nVFg6 z?hcpyG`7)?8$gYc2rk>S-vwl*9TIVxsXt3$aE`i_CgT13_t=q>4?_FvA0!1o?*>x_ z><6N}Qem**U=`n{y|LJ~YRJZH!KfpKOZs?~%kfK@>-|4iTLFt4kH?R_b3bp3Sl8Zt zHp0wgxBB)Bj@SL@SRwuBSF?(11453EFYi^%_118@nCEwsnqacE4^xf`O-ds_efpCk z)&I`NJ$;>=uSsOk)B-FSua=_gOVPge!J^!;N!A?+Cv>3y)Ke;~7#k&$8GYm25R zQ=`@JwV}Hem-W6gkCd3w5h5);(UJB^t|2$qZmQCrH&DUn9w`v&JoN-JuAPHFGX zyvo7P>f}C zLCDaTkc?dI#d6=4cWN=UPT~E`f_aE_djYj*T3W@rYv1ktibaS;B*W)c=jNhkQBBT` z2O7V10Nj3TJSn`3-4qeAsxvBiHk#l(J)WnJIFW+foTKbN2Sr_8$PJB{s&-kP-{Ev$ zK*P%tz&@!iT)1!ot@fD|r>Ks)K`U|_W+AK5q}a4Via0(#&e4c#pG+X8F@$@!a>*aK zzJC20yZ@qav`Eh7Pac{{n3nbKAAft+Om81c6d^FZ26=Fe%o1Zf?(3&Ce3vtOZ^4u1_xNzPwzX2sH*LSRK$~?S(>sr#HOM zC>X&Y?36}SR`Rai^-H|SBy2TP<=5Yy8)eXrqj9vanRx#vNz9H&x}yH>-MgRkXM!G- z)xw`n0-gpfD<_vO?K5<8FtR@UkecbPq(Ms?hoZQ+N4M0l8ZFlbq*w6v?V`5L-V|{c z4}cuNa}M6$00`RkFcAJI{?Q#u(ceE9fAn!}&o<(h1sZ)<`z+DJf~F-PT(|9)uF#L1 z#y5*E52p@YX6s%AZmg}XZ7l(a{*&JNy;Lo!eRR|xoBNe!rp#qDCvtPFln_9eLgs2E z^VuJ|!p-e%tv(|N&u5I8X!ietevJMyWPn^|R0ShzQLUt)yp&kJ2mj zbaXmk@UbE@a21mNBWw55X?Kr~(y$2n?Ws!1=fex?DT1bb2We@U9iP~wzd`k04^+C^ zNji+&GiYk?iMYyt*K>yHj1pE_RCG7RUW~QDq+fml(Ey31Uh3y+JPD@{_=74Z=G=T2 z0Jbj!w!hroo5Cbwm8{^E795;!*g?1QAt7+rX%{2>);}P?wAf5;z3}{?;llCi!>eTQ zrlX@HkQ@Mxyj{zhLK&oRx6Ri~^|!~;+K(exU7EBTA8tw_kVw3t7Kvy$bY)^n6o$$A z%f1e}@L95&=j}Yy>ZdpEkD0yYaNr#mt;RY?QM%#Kpl!z#WDED_x7%`eiTRJ)V*o*u zyu@-K>$8sd1i03%VMIMfk+jeQtM|;+-kd73kFyD;WP#A-XsCqbzD%)m$zAJWPVR=%!0QOEJBs zeNY-Nbn^Tbr61Nb+ct5>YiHJZrmVSxpmdk?Sn&pqvL8RM!gkWA`AtTtFNHMRVmV!- z%60z3B9qDa388a!KIj#|Ui^sy41=OMOlb$ZxC=2Ie{_~3j*p4LPnpcR8dFhI`vy2H z^xJ$BrN~bZBAx}25)?s^;1k7Ds7cXdcf_V@L&ZZ?Rn^~5L!VL@aO(J;qxhBfKvh-P z5;y&&OALxzub206?@&OG7MgM?`h$#nKc4rVyY}!qc?ey_4LNB(X=#hi$LN->Ggyx!am4Y|4Bm}akCdR60O`AH3onnrU= zx~Lq>QQ?=okS>ycHslp%vg`r8(epg-`0v@V{gKG`Y_lKaWfc9+g_rxV)WzP_2jYhM z54V*+NeX_R=*79YxkR%OWNwS_tj3aBSGf;Il;6#LTfaeeIxtDfPs}6SOWWPW-!7Ou z&yq=n3qKEkw|2x_8NW$PF}{^<`}yF*mJHkHFcge*?%i6X}eC&_|t4UOO{;7z~gy=;;#|YF*Dz$VUO^s(T-o z=jUT?X^)ic4-Dwpwo;TCG60d-pvVK*iZm1ZPNezx-n%nX@SI-#8|(FN%&f=sT#B^R z)OVHWXlNXbcw(efqQ4Akhss~Z&TggUY}sF~-OzHWg9!epVT|6mN_GB%?vjM+aEL6T z?!~VFW=_g{{g~fTTxh>zzPs0AP(pKQ$6T>m_LkXBAD-NCO5wkIcaP)=c3^R|JKy^2 z*FCf?6iV9N-91~2*7&+{K3Hyw{?!{(xpq79YjPE*TYx< zg4DFMf4<(ay(~Q3->(}RuJ+8)%!yOpdv0fkGk{Yww(Q`-Bf!fI?OdFo`waFL{pJrz}Fs*L+kLbM-dXSp+`by8+ejWO*KE9=JgHms6uXK@id z*BqQ>xzd{=rG3k=FI6V)by5_U68J|2pzW81#k%14%K}nc!EPviT_;}^1w{nkUzzjv zLY5u*ZrIP)12x#y+G=U=;K6UMO(G7pw0`{|HDqf7@O@@7^VM)KPU&4&aHg^zPTTA2 z++9c{wC$rrjSC{*wB}`Ga%r+1dcSTDkJ@ZeZk?)fH7k1>C>VZ%L1uPDrHd=|#ol!9 zNzk^9e16>bU*G@k?Dvhkcip;TCZ|&{G~3c91p%cj7We^tin_yH^B5Lm=|t)wyM-O(lqg;e4z%(^s*=6aMV;# z!g**9=H>UIGkVs6+Z9nK<}nd;_r;^p0%Pap?kdw#U$R6#*WB71jMvydI7n68#*ed( znAIHhn^zS;c7}9@0Af!d)j9m)ERHyRZ1adbPGby!wnsKfZGAl+k5%$rS|ySU|6C!Q zoD?S;#E#VTe>-ojWk2&XFj-tUFzV#HN?mPs#HUaGC(_qm>Uhyh35GX#@0AA!2X{4( zoE*0MVK`5mCZsj`)yiXF=fD0%)(xU+YHDbW)3Gf8FrPeO=iq2R~?e2)iK zvFeiLVyDE&Z##%7LCo$*&TA)$3t1Z=rPjA4_W&&chz4Y63B&IAoLicsIA}Yy*eF&(79X z#N?z5AKw~QJq)FF?H2isd)r7j8e7zRX7aeA!oiO7%+RB^&F(Jur`z?cY{wD_m{(cX zX&8l{_NqrPiAbpdY&BJ}dT-J#yU+-f-T7d+5p zF}&7f+hL5>=dio#{uwnKUMZpSO>2U@j~hTwg#RqT@cHwsK~SnBh+}PHK2ksPTq3{4 zqoGW-z+fegb#bgmw%209LH|Q-qKC}VcWr}rq*%!W(43nX5{a6 zIy4bHkJ>L-l-u=hEH29J72dh|h#^uU_J`b+_aDd(y_dT>I;6B&69M?Pr+Sl`yuxVH zbac+8^0Kh7_@HJ3bai!2$HfP)fO!J^{l)D1n9;L#Z{NNJysN5Zp18F1Z_wYO`u3jc z-Pd-f2mc$Jvj2aCKL7U-@Be;JfU<+K-t45h%!L`o+M)V($^g zGf-c%(kB6aJf1rffPoEilTkdqLFc7&8W-3c&@VGpCW3K?kviaO3VgH7KMF&B$z(== zeCq!<^7H@tM?OZzun$+_Z$)m58aDbSi}5-vV337++}XONx3oQ$yW^6OK5B;>`EDw3wnE9&f8qcDk=D6Dpc?t9G2b0`*l zxB{6AYQm~&r)-Sy%SfvaMJYWR8q6%vn~Xc~tktO10>fT20lEI=TStxES05SV>QGY+V33aoS~fRp)ZPQ#f5jS!^Q`+I>Xm$+WV~}*()-N*;mJs`1#U~s zxRhvQA=(qC#vymh?MMG~opGk>&B-CqDd=bRM$5(F3>DlwrA(X8?-zaSFS@gCs6F$s zpI1Ihsl@W(;8P%hXMF|aE6LuPgsgtv7|}0zQs=?H(P&whr+Oh?xL7;o`|M@rygv$khY+Z?S}kEQpLu49P$WJSx~6hTYb z(9m@}>JWo8uQT#H%6o9W%B8r_^rrps>+PBRTKB4Fp&yyX*<95P-kvu9sTV8jF;M=( zVNf~N_dqY^I&PVXLq1P4aip%QrS-}WHu51n&N^zDTg`p5C=My0W7iuksPhpkNus1r zsHp)}<(9MAqg2Q zsjikKsXMcggoaR+@JhR=S%1S=d4rMn2y2U#4h+lQJ^_>^P3C9q&C5 z&Tul@p3J0Svf8r5>}U+}rt?`;+gX2nKaizS_xgS?B@q4{It`07Uwe)J{veV@=I_*l zMihDYlY@f`OrvBcuU(S$O2`^5w1tj#sLjs)(9%t}imi6`bNG%WzCkr)67YwMQCIsT z-dnYV($obK@;|IpTmz#7C92tYFDyK)a^>omr{`){Y_w7U_om=i`Mj$_1l(vn>I=8Z zZB&8Y{hb|FdFc%|jo7~QaTL6#Ul%~qN5=GInd=K(i)+Kln z{mLJG?7pm@Nl${t(Ef6FqHZLcpy93Usce;S4_u}1kx|9Vsh|3#4d2gQSlTWg7|81> zCgVmCkwZ?K_wNVt4SmttjJNYlikIdtLfVF;pLbo?)vc}XAQsfu6R~Q@z`_e`MwNMw zz6>~)+V`>s5Nq956S5>6D__O%G-A?p(mwf)M|0zjEWk58+=LM7#!8IAn$F(hQ$Ir~ z;OE(!%CD;n5K`EMo9pO)$U$CvRFo`dxmz~wASOM1tk^K&wt1a6{}I`!dQIalkZf6D z+~*H=ihB~PeB7U)wNmqv#L_u-xN0)hj5FSSW(V69N$gJ;DJ_k?nJGL=KIz{=U^C$V z6cI8Q(QMru)K+!#~gB=9&T>tbsqVH_sPv4Ed2kOSpToj=Km_T#l#JsGWY_R zJ6|84hf0lJsI<(?#iQ-%%-p5kl$zJ`E$4X1vf$;=mgp~l9YprQydH9Q#CE1HdnP%mjMdLyD$GRGcYi07WVb^&Cbs5?ChB6fVc_@9@bS+o|~qoEe!6srU! zI*ylUbSH`+fm$f?Gx_{@&Z#lSM)(-T#>EZ$p&IQx$U%kIVx(K5p9otuJkC5aQ#orjxy zTuMs$bN%Lb0+01Vsk?@U*ABn{_fq+2MgB}%SkyRdns7*e(@b326x48W0gWar)T&NP zNxkR=G}kv-OZY7H1LOC87%({S7hCS3ri#v@WiiT&whA!5z_w8}{)Ia@gN^)5sK31L!j1XGZypOvQom{7iBCST~{j z_W{OEk+dJa1(VM3$Btx%&-$pHrMb0i4tfSLJ+yg`$W$}0)qX zYIT7<4@v@JkF2lJS?WABdYH7$V^E2jB1@U!oJO6GA%|?9!OgR0?U9PkvqgaC?PE|0 zJ~vgZE$7ueZu+99z|j=Za+=DK^Q8XlUeY4cK=qosprU3Is^LRlvSl2s;(MlJX zCu;16t$>Nva1Oux%o>?rR9`nZl%bXL1g<9)qzFF!xa4n4Y6nGN+*$LblQCx*@Y+(w8jDiL&Z$lqD&*beO06iietJn|!XGrA& zK3YY^0&DBpSpX+kG#l)e41lz$s(7hV;)x;4UFN@2ABik8g;~@X!JCwo0Tp*!m6X-q zulF^EY;^W&PuB=qNM;U$75?hW-gRd#u`-TirKTU>LF9iIQtwW;uJ;lY)st}Cz(M}I z)zS@GFVx1U>GDPal&ArLI>I5EkPY>r*~aeJwl#4uRG9^TwdgT6-&{h$Hw#2q3MLNP ze|nA15im75+M24;ba7G51se?tc$B0PtE@bd^#MFPB~sQ904Y;pP`$1vakM;JS^1L= z8E@dfoTgFn{y7D5!)D^}u8acDJy=y3MM|_&Dxq*$54{H=gZ7}=({O{QHq5d{;Jvh- zix;%>U|`d$Dge+Lc7au4YKfrRExq>Pad zKPfA(<@I4fcf4V1n0f8c8eerT#1XJJiyekN%OH(CMKq6rjt$iX zUft2J`=XoA(PSGT$jkfhpmefQ-I*1MkC}vDA!MhNx8&t3Uq} z7MqzlNaiA*q77Q2vqaz{y(!6eTG!Y;pY1#WAdqhu*$3vJ@Oz30oyS2Uxk_Vh0H7Gu z-+GXt3_v_nwX3J+-;wxuD)HoNT!AT-R@g0XIe9SPFR>S0e`u8bF_47#*TnfKAjAgm z={%L?gMLLLa!Z@F_iKi6p~qVEzcoy6;i**OZc)>X9c`J?iPevau7nmDI zTxH=PqsZ!Rt9~14s1=-$)2|2`D2OGAKYyPfJQ^?f?<9-M$6Iy@7{H5K)`1!N4mW_L zCSmt^P=odD&klElDCx!n(yIVS+$P? zg^4BB<0Y1=tDt9)c6m=rnfH$Z6XP^_)m8ON9~tx~%K@6sbO*owx;1HSLq$F03eR^m z&2-PYW?Anrc{eni4Mv99>JB1x>sYbx$zwzv3+%?(^;t_r#ZQmOXyx`+sptth3SgUR zCot;${gqL{(pS@x|3*EUu(p|KvV?3fb6juBH&mOhVf(9FsP5>a4u-?&I1IL@K(99r z<6|Y3K&vlH31)Np1Y->2kYixUuY*MP#Vm{z7w*2^MmYYv0A@Ks+Uc6Y=Nd1%pL={} z;58LyyxnO}RyuemfPSsv0Idvi8{|+jy6n!85py3b?#uJg3||Oj~urt_ZE5N8lqou zb8pOA*0`{+-$?pd--AT{JBFL!7)}UrMWS1?#ln^&B`}`QaK}Rj38%c68$dl$U<8Qp zIr`7Z!yRgdxdrMmd2wmbFu%lnX(kvWv zXL&|1g{egWj4@WEAABy>V=?UAyGlr5DvXh{ghfo;HSLx*P_z_0F|+Gi+qJQ5U_J!7 z&cFf0@m4vR0r9}!EK%6LpL4mt)E_GzPzUrowjBaM?IF28X3-t5shRi)kW7aWbtxA> zN}wkh87fs!_{@w4wp_quq%C5vR9NFLcf3pMC)*Nj6xuv&2zX~1KWXcxyv@Ru{?hYT zjEa-P=;9e5>KWz|RrYaTs~AK|b`Y6Y<=%Xssvg}Kb@Lkk!vFEp2w(>))my_J_MXi8 z6&PvUio1S&Bsa?UFcw%;fjmjCG*SxRNQYWI3%>G+TLWB|cbEcW7!P|aj{+pgh>}u9 znolz!FW0Q%k?(%Z(F1F}(dPR@aW7-}Q5VWEaeAR!d4Q=NmvrWUAYBzVO&y_DpTdQ99Q$C_IjKK(0bk9_x>yNfSVAY)$j}9 zmOk!M!$w~W4Hpy_-^j0}ePI1(g{^$Kq+i9J@rNqW&Hzde(=hD%9Lp)^;o}uw2&wcpVuCtc3ce8s_B>QY6>s?)QG> z1{F-!NBGI&biCkZE4^`+`bgEMg_XH)Qb;4z?8dO#=2*V+S&h3%fU3@VtMKYN%qx$A zG-toj$CZ-l!b zC1oFc^aQwLNmQ21E<1eyVAm1~S!_4FaqOImtqfl?yy27zhXZ@&-}oMid$B;?w7M5_ zJ=TNR@F-DI_9M^whH;gv>)z;9A&;XX9>OtT;sN4{gj>r*GeRo;kdN7##g}jLrz(k1QZDzq{=@c{{a9naNEQRC5=sLIMTYPbz4;b55AlD5 z0xa&F#>0?rdXhG(%+{ zEe43=;&$;BUZa|le`J+RB7jb$Q+<2J9#9oKyl0YDfnmoQr~?b=0q7|&Z&Y1G#DZ`5 z`;vg#mJ0ri0Fq;5~MM3EWv#}hAXq+j3j+F=ItvNq7NBg&90 zSYM>U3({&Q^J%jRnLPs&Js-Rd^ik(7eEQ4aOxr?3n-P}IQ+mv?L*E~6OvGIIADv=- zk47?7X7*+5*_BF$b4M$3|E)X@Fa+Yx1x036uVMye3_`2}_4`tNwkAEeMf5-1;*ab4$FFF3nlbYnu4kr5$#m|P15_PMiQ85A*V`ls*rOx+ zqSZO%HjdxGWLm8|uJB)12GXIYvo90a&h7H?8?>>iQDe34<~4nQVRLMwwrniW&-}L^ zwwE1h%sdi~Id}|Y_uHV%mf`vkVE+nDSpFT|pU+XE>Oi5*A^ZD&*zdGLzC(`Ss!zHN*Q?l1pim7FV3VEUK+WL>iKAeep`-M_q9 z=n9wk2Ont&rmzMd35PGYw4xIe0l)Y=TQgbSY_UTS_y}c!_5WI#Bt-LQWMl|fJ$cdN z@Q*3Q9YWAK|1(p%MWhM_+5*7O0>zzE(xAT6Y!s-!>?!p49$6)E?_E~N4ceDn{@>D} zWTDP{1hCn_ZeU!LN3S>_g=GHA_jz7lW(XV51GubAltbn^6I15!L|PC*A7k4_9 z$}&+v%`?>ilWye$gdJ5mPpvA4d|!z3OlN!yXtvt|q>yM}*QkR)4hsP1t^#=%vNXWC=#*ya4 za?TkcpmDglIVO}j{1|b#2_QCGyHLpD6{z|J`W{-sb9!8o%cl3S0OLSfW!?k6o6j)hKmIE<;xyQ}}ltQDJ zsAaHX=6TVRqu^)sJ;Y&jsM=XISL3Bcb)Jb(yhy2nAK3-=uiQm42m~9MCr?elISf4* z&S8Pwngn{(9PQDp57kckbsm4ky?Y;{1;E1>x3Bap4!bbSo1SMZh{dvS@M&Oi1!iIw zxJvHWR{B<@!Xp0(?AP0@Oh#jcEJpIO~ z3}1WS2j5;^KJfvRLoopG6GvP6pV?I@$$a{wFRV1dJWNah_~L38fJ}Xt8G)_h+qF!w zn~(1Wh0zWQmaZK+6VOpPI`MIFgGpeDXyxoWcjmN9m%ry5O1Pw!KOdAr=2^;Ro@uk2 zynBHam@I%1>C~A#vTUbrm485`BF@Bgfulgo&L}^2s>XD*0JkkQ`7|6@Bg$U@CdabA zK=lI90q81alVdHfO=cgpJWY@)4J29mVSuh1vi4fWb=I=NPEdMc?Qv|@uY0}8*)z2@ zUfVdxY=f+~UQeQ+sc@z(ARpHt>f0)@Y2!_VX^R@pU5rwUVxz2=NvTzVN$*#ldQbbo z>q6GTP^fWflRs*92gpD@Nn*Ko$QK+f@FrxQU&?j%NxkQMk(o-V^>~@hRHMacVd3LD zkz67oB7!e|`WoA|=c!*BEh#=jDeji#GBa?r%|U(j_Hi4N^<#cAVZ8R?25mpH_^Scn ztkl@hz|Q`?VC2IUV4s4YdR$Wjgkj^ZOwF}u%DAq9T0!EsF2rC>aU@!~h&xpQJ_@K5P! zF%o>+jJ&7&=XLv;=#qffL{k9%Gp7QsZI(>ed8k9VyNk>!6bT2zY*H@I|9l->yZt6f z`ptr=bpQlJ9Q~vZ2F)lm-WBp_ke!T{y5f|8&6>@XCKh+J&35&wByb$aB)7R{{7qDS zsQg7wi8WmD_1wM8;NSfS1Yt2siMUZ%3PNW-^?^qd^d*X0Bi1NKD@nk(0Hl0Cu3bC7 zu(%ipykooKc%jGh;i`6jIaxIuV8LZGuYC0ec-%c}`^W;jnQ@Mmxy)g-fQCs}T1JKi zMi%|~Swin0Zj6$Beu28(#Uq;OHBgtoJ4@3CfYY?p>fdOO5~gTcGgQJ0jPl)pQ%^Tp zX}H|xzuQb{QRAZEcT~imza#xu^~qbs6gCE&HI6|(n0_TyN*^0Ah>%epcy2wio?EZ} zs-&|X=hum*%6XMM_;49|vZn^lDY7q?84M)vw$uqF5$rt3Lhs*Z4SVfuz}<{I$>9CV z>nB)rTLe=LvIU?eis?jOnpWUzhF&>r1cn)3y9QdRCmEn3 zr$PMC7a@ykVW7a^Sjwj5Qal0PbkjAiQnF-W!63)TH)MXkHN4h+l?bv+;4vsykAy^E z(OU$e3t$R14)P1**Aq}^5w&fn1r`PnBf!SNbKd`G($1pVNyZ;Y_V+=P=+oKlH)w%> z26?2l%qWn5!_gxlOTX3)xSs77Sv5}mTa>-Q;-xSwdX@6**?^P14pmBxyCCMxvf^bg zxW`DkzgK;0lml4w-t&Rv1rb+;UjF%N14bg>ee6w5;FL1+ZzO#K#HBFco@fOR445dh z0V6%Tq5A1Dm=FyAZzP7BU?e`n9Ia4l#;d|lUU@|GT)lKjKTj{C4zfuEH}&tDyJPA; zixyeKi2zE$5Tdf`L+-C%Gt0OwfeA~j8Xf-K9((>KmCE*1mDtNaKRvh2GlO{|Ku`v_ z8DPrZ0LmUISu(*icyHtpGCz=^Qt!FV3izGD`>_$gE&&__SNqeUWFlJuT3riOf2hW6 zUhRZk8_a}~nb~Vye4?VFpjxhL6Lw{UtY`as7S{j;k_Pw(ceoi&$Kl`+y%aC$Jas?# z9AUSW9t2D@-_jUF;=gZ#j@srnN z&R;xnKX;5V?u8m4)i>7HN_`kjH{YNo_YMp$P>Nvk(2F{Lt-Tcgy|Xjdv^%dpT5bWj z(9IVZQ%CmqAd76}YZf_BXkPBqVg@f$N>cJiMz^!^<_c}e^&`+Sh&nD}m+m{lUG)8U zziGTO;DY28YgiWvwC3hpe#?tmt@+vQNb&+~Ow}}3*K&Ux-TC`(^7A*S#8x%_Tq^_a z0FZt9Kuco#)4&@`xeEYNrv2>DIb6%vcmJPjc}Go=lX2bduC7kx{l4?uH~nxY>y`O& z&;e*S*QqtfHEK5?ldIpHI|*{dDlZ4e{c=B6r`*Q0?4ZnuX?)ufVR1wcA7E3febMoV#Oj17 zl0T5joAXZgzsKAVd=cvae*$X-=v0Z9pz`+p09(LjvP;CcDPDY#`vwqufSWEa4TkUh z`|lfX&XOksxX0!s0>9wqMZ{kS7Ca;k$h7fU@sTccOqe0n{MY`b%%UJ>W*c zPIYBkM7wOP{o$nJB&l$kAJqM zD7S_R*|_ba4~9v{K3Q2=aewv){$L>X~xYQxxAJ`KVei9BIFkw1NV`4eq2nYxup;jWV-*Z_CSCo9`cbisABxh>6l+lG$ z#ot7wP03-k1E4}O`i~4^kky1;6R(Uu^*RPa7n9EUrnv3M+qZ8o6yWcV{=P2YdVKlR z9SO`nKa~&6T0Yg@xa$ zp8FrSup{Xi738PSo-+@6&ixxlqBzA&&GP z#l^a)cENzxTMRnjYg+=|AmY;&ootl0UTz*NbdR&MS7evh2|jMzohP#UFt_7Z>q*%s z{YA%5oaoEnP!V%n)K5?WQM8_UvYc4(WINm4pPR|Um^bx2HB`rLCf-97?R%E1xHj2p zysa=kzw35%s@AnSj*+6>B95QZOpkgeDHrK!JMt%4YJ^-JO~Gvq9kQ;I}l zaJ;^p8~exYcQFE9SV=aYHRVisdLgzGmp5EIJ{B1_ULUSEuKf`CoH`ntmXR_1e(M+J zm0nyULEDbj!y+|$A!DBm^-gD0Zu|w~p;&g{xN$ zmdyDKKMYhgrYE2n(N~7LlFXa^HAj^a>gu9qNYX`m7{_no0?ZR8;m)x(T^_p2SysU! zs(eOQMG_RG+cTu?AE$qI%5F`X_qQfF3Kg*3b3C)?>y9!P#!kN@5fpB;~*7qVP@c)6K8)a`*k z3iijfXYd&Z7&XE%`%S}bNTlQUHlif9rTb@O$cD=jAF(#28X`8I8)f~wI4g{+XR!c=Hk36pUJY$e(7UR zp1^6hpMAd8*sNY(>=~=3$|5H&BlTK5R8S=7*SyKxL_)fWLVh0bUsfYSKaY22M|0)- zMQAs)u$W@B|J~`%Le4OU3dOm&V3twYn*PAF{cnT$`@qyyh0Lw5%*-}S=1bQlsE%~1 zh6t#!SbW zxz^Y2ThGPP@!o7p#q;Uchz30%2$69I*JBH&x?Kcq&(5dNh0!paF*WVXXIzOpDG_UL zMKADyg1R9IIDFqo)Z^82ZR8N+8!IKKM47brjm3uwF|!&g?mhbP!?gO}iEr;j;y z;T-8s4#?LJ*Qe=v%7!<6)|Ar<7misXDP*N0oA>bZ>MF{Ps0UUAq9nk1@6+RL2a~1a z^?NRMW=5nLH-1eGwor#zLRQp>U}{0Fo-n_Qf!JG^Za_{hc=u8`X-YuJ#F~l zy1pPb-j(oChWZ#s*}63cUdNBg_4tLmSKJIrUOuay9QYWgG*`-)oNpwmVAF6~;DSNP z9%|MPlld+)BR}8A-kmPfSkqrWJTW_ajic=QB~KdTZ(G);o{D>;Eg87kE_F#0yyM2-y>ACSYOLv5UKn5{+67GFp3NlUH;h)mz#Cp z{VGheOF93bA&{W*wzQ0Oer@3QiPx+n_VLr!ZWkh@qj?QKFf5El*N8Pw_Yy-V@raM^ zu%U@3r?FXL? z;;E?TF?#NoyLs0J*4=JQxNucSxTC_S{Z@2l1(Uh^ffSJwF=>@AbYta~FZC{~zb)Ue zRG!Qoq!rGv|D8(OY&km}k9ZJw@xgzc?8jTC8~in$oO%V=zkU3g6?-r9YF>tgzLZL? z;Jgx-+6#LPLoqQf;d=lZNd8l{ZiA(Cam4R$*Z1loeD@do0PF)k*ln*ri%)UJ;!3G= zT|#7iba>X||3Ox|C<9w_TdVK}V?bY|{sr3_#>;cunKCpUCB+tZP}SBPKKslG6A z=vK71Y?$MStbRDCZ5cEzpMRBo4!`T_rXp<7^IFc2Kk1`f^Af2h&yw`AW zA)VNIYrd<}r+2vyFrbGy>!QQU%cR`*0Zyn}aY`ye0+Kqj!_mu=Vruz`u9Q|wUkVDN zqCF+TMFAPhO1S59tiDOy-WAub@`jDo=@rfcFOHDY|)Ze8~5M(3qhl!;HDPINbW9K6-KgQ}^> z;}%VOS`s}u2;tdA@iZlcTi?6CNuzcXd4XInEZPG#oMOzP5H}RVM5Lw z)==RXo37f$>GA*>YW7$=3z(_h2gB~|Evs`5;MIKrt8UWv$=L~T7~m(~UAi^-WYhDbTNe zNuixbC&!j-XVFmYGGH!WGqr#d8gWyC4t_6bcEF-H{oT7~)ymoF*5lWU=niB}p^0&# zYkR!d#*Lx36BA`$NpKK+ueU7x0dbbSAUmi3{jmKXDE=NH*-ifzw5k4d)|%tQ5?M?UuCCAkBI3}-+N_RB@B#jPe8C-W{@=)eyQ)bZx&IyR-2YF%BvbM{_Z_?N8F{&(=)f~- z6uWlO^61@|^+R@w&!ny45qXhZjMdC`{a#q8J=T(3bE6}}lrYbnu9xJK)N{2s_sGi1B70UI`Pni&co5Uw!oqT*szZg$m!zE5>ToaZ>5dq%4OmkOCn)8CUC}syjOa%pso!iOOY~W*61jwxm??0mlju7|ROF?07X*pIK z;a;mO|B)YP2@a02rD2TTczXuKr_(k^ujQ|L9vaRx8HhO?;nvx;YZo~K)RklB<^7r) zq46-1YBh3rd?)1e62i&Qne>0>Z=DvOCe@d{hWk8NgW_3dF1wi6(n>{5jljR}QY`cC zx8~qXpYqMlM&N;RocvYTwyV2!EA%Ywg_`4(lJI)2?(U^;hkpI4jg&eVaRM9U+ru4< zn!ENOuZ%Q&w#{g_i`95A69A_Nm{Crd>3^NcU~3EFpp4DL+a%QtpYZTXLWb#R18b;2 z1et9ieK>bH)~Mm8KU=J#3UOwNvvM;Mv zQ!=wns;Ipw^Ph-9R=YY;En90{5-7bmy{h0H;+vO`SEGH<)$mqed6)&YO2q^KQPmw} zlG7d)kx2FP<6xl}H?ye;&JDd^=vGSBlFEGHXM|D50`h+T{H$2jtCOW*>b~^CQrV)d zJjo`tu@)UBg2ev(SnadjGo<=pzLA!59_>$ZQ zL|t-*APR>S*!jlTk=iT$<-LSUGr_q0(ArAUISFcxvT(v^Gb+Rhci-1VS@k`yeSE+0=}tkb>+R{l12#kA_qzGR zq4`oqK}$(>b#ZqX=tpM6bvCKd<~UV;e~uW1I6of!x-&woD6y&Xn{>vq-bR(H)-ay^ z--}o=AW-x@pXl(&dUb8j(MXTF+bkaXKi(Y^z#5|CZyHTH>XJjwKR3Pdb3M*`XWLGr zfg5{|e3nr--GyTo_?aawT8sj#y9{gMuBC`XDX|bX^dmnm+k-Y z&l~`tD2N1bXbM@!C(1_^j+3bO)>@aY<5?s^v8d0y)f+{!nH7=KpS5w%0B z=?{wWH`kV>zDTkQSl<|LldAhD!*;I)H9*c2mmTH}*6i1&-Pc#-h%cz?N0Z{NQ8p%W z>JQSWzY7cm>XXq0n7ZxT;UCKc;}&TK?~#}m7R4?01n|24s$=H6e_CH3BO+|4?rhgI zFc@9p1rTsklG5A=3Y~)BRp|pyhyq|^OvGW%hFE`!!#9FmckTd~Jb{ojH`?FNtgjFF zb8STbNHA{&jL9QT$tMM$UGjs^EBq>G%WCwNnEQKY+-(+khljZoOTERT?F`|te%r#S z!XzP)yOs_}S5ti1*DQid<7dq*M`gR2p;E@iNGa|_vUk`t`vr)Y6!FGbfCt9gbV~s% z4(7!NC|$~i&W>l`db?k}3g(fx=4n)W0&jRbn%BX4CVjh9sL<$`vP{s2M_b@XT8~?j z^kx$OAzyM@{y&&gz}Gx!STvcQqNX|Bo2K!&%eHXnG*@*u0iJcQI8bHEDFC9SZ4!NrBy_n#?Ta83M=$!5KVNH-HUxdMfb+Fw;Y&UJNrL4YOXWqSVU|l zI2ajC+WfR&gjiS-lp{El8$Nm!6ugwxR@RAh+jGl@fsXGwB~6>6s_&iLtbGiKVPo&! zqB--YNY6FlN?ABN-1hlW{+ixS7aJET1_o>e7Fh$7$|COWKNNYGb9Sy7;88DHO-njT z3Yw<(SIYgQ{Mn6x{3OG#d$#Ac_(lSZ8TA6i@x|#fnzD3`yYDi(>P8xkI)PuV8Lnd< z$ulG)LzzXlZXNk~3CQX30veSUrKe54Xrf?6CU zjUTgT?JX@@e%Z~~2Q|*xTL(^ExSM7)Ja0lT`gUx=6;E(c-tIBg`oUAEYI738vf7?R zGCCt~0;vDcv2&}_?&4l`eP!15hnb?JBK96VbQmZ_YC5AoH@|UNY=w{)&z1LcsJ1+W zUzybM!E%__;^SeibpPXvr9N@m638UTiJlWIrz{UvILbegzkw5f6O?kXfXPx)KG#0W zN_u0BMSrl*TPK2!_h)~FphHuS2w<7zflA+>v?Uj+=hGNBBcTRN=%uGGy~#(@BGb^W z!V<6LK}OEI8cE?dUl(;`W|>_<)j|h@!S{eeNy(%|qqN&oF=2LTkWTon%bM;coTG^4 zk_0Wz$$?6h{4NT+`)#F+qs;{DX%-gkzg`}TiMb8x*o(t_r!F18cmmr@x zy3>rBE$K2V3&tH6*q5ClC4$>mwoP_n^Dd9X`Tac1v^M*iN#>2lLo&{`h;#kq zp1pLZPZTItFYt9_Pa4&qQp+*;cFCu`iUgUAvLiw9@ zkBGg&!UMIu`)R_~J#)=-J&)(dQ%0_^?hJd`Md9$dMVZ?gs9tH+qUx$n(J_p0&P^Zb z?M9kopsSc$)PK0vq^jMW7pqhg_F!c7G9Z7;b8TrAo8YkLGWV_!k#z0Mb|iWhA2l6< zYxMgWUlRbbfWBq5>T)qy=|8kIp;iWz>%C0!)h}H5q$jgRlKYZ0)WnIiT(64gbgjhP;&!Ou zCT0siv6a2&`d*RK_*61anvrtE+12?FBmknWe@XiBuqt?Xp?ecA{b;aAofhyLi)f>! zRC8Jf`M7{H_ax`*X+cAbQ{)aJ+Hj`7Lu8#5rCqv-|9TW%tPUV`eQ#;(&F|%!V^T{q z;QmT-C!RQaQ#Czuby`vnBomtl32J{(h{BxnWZ*LsHK$SI>3L@9!*;eqzl(M56)(>L zLo%Ag(#xg=>};%p{hR`)`u!+A(cZac0W#$I+W;QzjJjV)2(DlRWri&yi#r zHzAw&wOfr0z%2>c5Y3jFxb~E6Rux*_wNPHsc-PTMu&$+I0KaE$x)iSvbfYbU%gEl~ zQ_0I)af--hfNh?T0nYM7XMG@{y!Mt!nQ4HP3*%lIwso8kr@VZiR~%8!iWM)MihPRPp!!djC;|V7;xg zV|H=6;tbP1E&?rlNP3}IX&QgdqgHH)A!3zppL*@h;18e)cIu6fpXxgzWWp22TKzdg z@)m!`3{%qJ+hAtlp2)&nmF^y*K{9PKaQ%EvHd%J>F5i&$iZk{<_QMB4YCkf7GvX8u zLNy_Um;+W@tC*z$NCQZ_DW2~94Y7@5e5xPOsg(zPS9A==+mR>&FTS7vz9r6#nSN|* zP`h8V@VVq6ebtpiu`eJzJearki!LfV8|!g&e1IBH=3`ib1r^-)N!9ISG#G0M?7Ya& z@A~XC#|_!!$>P8{qO0cjzhJ)7@4re(5r28JCXDUoMK8MH@7u;(5Aw+sy3Nek&!1e+ zu>-xS&~4bAkyq;(piT~xQg!Hnx8cmG#s(DtMmozirgS8kXcs0z2Q;0ezO4GkA|UOYR}v{P0Vz^nhH zrQQEUMAV%JHy>mRlnK~rtm>?zuE#1!TMU+W6If)2eqU3|JTO;cKc{ZJtt$KU*(u$? zgKVnM|VM(xx8v21s=;E zTKh^%e*Tjuvz<0XS5x$5{c%YXFYuW&dBa5b?AhHES30wT&&!m0BmLbNbCA!hmD`_{ z)W>67uyNjH`opnfmcwA!{ahlI6Z>PB{Wz9%s`EFRA|&Vpfwy*dk5zyLMCAyUo1l&( zB*F(NNv;gL4V3J~0^Cbcg|Xv*AbH(Uvn4~0iZK?&qG$jvn-ibq@dIkLGl&P^|Ejk! zJE>4M$;sujdC(xebt`amvG)sD9x9?bbqqWZ2nSY?xWalZ=rOo9ih9Ac(<57wPT+jlU&W$GuMm^d8IEt0>%(Tm`YmR@QHK0vW1e}K!ixNqmX?jBu)K|%J{jId z(8c7>8qr&t=H$>I?CdO;TimI(FexE2zYv(8Ur@jV-bkom6qxvxM@qZrMjL9&Ueob_ zUs59$=;;ZN;QK4)4}XT38ug01J{v0X2yFYN!@1|o8DJZxUEdoiWM&yHTZ-83=j%_( z|KKJNv`u1iP&I?;+W6og=&AeY`$l3ZsQV@ozdyAamT!eq$mK>|-VQUToYFTOBjSJ^4;2PI zFY|t4WNid|jP(1pE*nr%z>lm=T-~MRXnaJSc=f$Yyug%;mA?jO?HXm2G>g;~cca1` znO~tGkeMAP9~Ibk(ZwZ7hTXkINa?_jFp+83<5zY#Eyt92-Hy6GQgTTQ?blWNPnt(J|y9X*f4$0upF%uK_gU4TEL*cKmD7X`*A zzVT(sOZM5%n|2PH=`sFX+pM$NK}&ZqhLy^gX-6?>xOfUKs7c$oENX@W}veD|-($Q}5zjdT-`_aD29xWRet(BZ>#OMp+2g#p4l4*zSRH{S9d zkoyro{LG|XRA8(9sABHS3(>`k+jfrlh^fz~#N@k})2p9#2rBp&SCM)1&Ub%R(8`LOHG_QD~Kgu>v_bxALBSsA1#3H4jmVb4E?T#eGCdQQg*`m@-(jJ7)k2{#x75qO% z3&>{W#2fze4)aB(K^m4xI(CY$LG6g09W{rP>qOxYHL-rPFR z`^C-iU}t@c3wp*m8N>V37K%M@WjLyTxjNd!Jefr-4B{zMrDePG)0J{&PNqbm#qU3K zGW=?a*YgT7Qisw1&ezY*iZ{CmbsYdt)QpUtae01+O6{b$#*V5hYC_Z^q{FcKC6d4yCQi znCtiNm*z&r98?KKsU{)U172>V*K-ZeHDmC#vV9eQisOQWS;?Aq)xD920-|8UC}l=8xAB z;2*5sl0qckEn0FCquFx8)Aha9HOLqC7kkEQzzcOBO>2%%K12Ex>1Wa@C?FJfY1<}7 zzenIOCDgT4Qa_heg^kw|nIZNeV18FyEh!%ig*0tpM7P>MjAJ(7M&q62exp2*9)I}EotiL3HPbH?dlkMvK^a~;kZu!pf4Jy2EmN9tG|JsIVohb* zqh0pWD+>tRw<&h!W?PT_EcRp_?CUYjZ~XRH&_#Va#UEb4;UWSpc#!^VU=s5gtFREq zv+Na6g{?hdHE#5+jfS8iV?@8*x=d1ouH#pV{{*Ae`O+Wxeuz26n zk4}s4;pFyLxP?ld)p!Id$NMXwrM}rB^~uQre06puGRXZF){KVo6Vau8M}am|RCZs{D<RRtY{1@c5Dmbg3*ZEFoD6Ryqr`u|{+S<86B4bO zG25tAMpt6JHsd)z4k`T|hiJx@?~UtM(*sT)v)xVG26(y9;g^5+?Fde#}JDM z@-r9LP1=G&RQ~HeHg@HfY`pIbhK^9WNtUJ5aQ#twW;Yxr58i?g_WG2#gw&fJ4R}ed*yKIk@Wy4IrpxNa@H)aAu2&s??0C!F`BUO`2c8x`$>N$2nS=_6KNTRU{IM!@!_ObDB{I5KuG4*huz^_yB0S8n>UDf&HqYCSIR zwx?`=`Hp9xaw9m`hL~ce6M%`4`;*@( zW**=gOiaWZ0ti#$MSaMx@6}71BQe#XVEU}D*6%~Y#hV_)64`J?R|+z9I$?y0T)Sv=aG8rl& z=iR+b%brJc=Nnp*A#gz&WIs{RcI1pY2=;Kem6Ap#^T+2V$j8I;d3hF6Q$n0VT&j+T zO4{{U;THVi)9qOlkbeuy32YD^cU81 zMw-&`@$wgP%bf2twbQ`7Ol zB25ZIPH(W9xp(C96+Yb;6x4N8VnYcTprL-0>D43WQdzHRB@$_7=lznNdt8S1o~R8NO}^lsg$**;JO)TS*fUyc;7NuXFo$C zy-=!=zhV+}Y$*=itOx*;cW$Jtb;&gl042qwrYtLX+oxWSG%K~IqjZ6w`idp4Ps>?` z>pLtoq{XG1HZM+tynpJpJm-ZYYpW^CvyeQ&0^d%FCAUN`Olw6fjJZujG&nP;G|>UK zHq)HdwChvS{Z=pyL2OSmM(8$D(X8~~fl(7O>Np`Y)!C5wqX_gv`A{eF)D7hCP&bzj zgkm7mJ?jIMyM7)uH7KTl^3q2+rC?t1onI0?Y35cYG0iLUtdGV1YW)(Uz@BCj=eD>1 zH?eXFkF0T&`1R}?5M#hEvHc2+mTA6;r#l_q@zK^S2PJ4(?W0FpHMKpXf1K0Q6b|v+ zzo#XGxPfp$^#$y2ffR1t-x@Fj7mfTlTrUxQe89Hx5sSSi0A?ieT;BdnC7_EgFg&Oi zM{Nb=Ju+cvOUviL-1u1LBiulc(F>nOZZ%F#h+cqrBkuaAia>~(u-&>$!v$zYnh`bG zNN#y9VNSgxy)!Gu;1&u5M|d!L!Mx}GB{P3D zY={|WCAotd-amysn#>=aH&IARq@bRy;?Hss%WUQl69bM;&RO4+ zWNRKNTV}QIDRZ{wODpzl^=G=mh znDzptKD0~D5fZNA68I7;0R`lj-dl>2?!?HK(6}AH;?~ljZ>d^=;C2h`2Yi^6GxK7a zEW7(}BU&jFYya8eS=syOf59jSNK)+Htqh1Yius73)qs~@=yEWv2@@Fzgrul?rnxRy zOrtE>0LG*B^L2kVD}b?wJbwN9h1zaaAQu7MU9S6`o(4i#MeHsSNdJ2fLxoHvs2WHbk2-JV60Z-U1_ z9}2`gp3^QFYfWJ}aiTyt&7k(fBz7?tSfw0>3f$d6bS#2cu@{3#o3I(C^>(x@xM0JP zQm?U0&e=}&TtRvb=SLgVw2)Ok!TRsNJpdnUtgj?3@#F>uHsf=G(_CYbE{f!dkpPNZ zl`9M2R0RjC(4Few%sl8$&OP5AlUIb*XngQOwA{yTw;pQdNC03(Nd+7yJ9hYW6 zf_Tsr--=tO-)}vzHbO&ioL;S70%m<>cKD>>?;?+La;bN-txdp-1CipbZ@-xR>?gT! zQD7$s&P)9gj82xg6Zli7bfmX#ku<5i5)V=L=w^b86FgQjPXh!MK<&ZAk&B4fYF)#M zSH+}C=cs*7v zdmii!{ibn06yM2XFAxF96qNIdjv~pyf!lY{cEmoWTTss3~!F9xQqV6S`+N!5Q{VL_`FDrBTqDSk9jPjG^mx zS|CRcj|Q1_o?9K;_DF1bC<270{P{NHr#q?6`=~qXaU|PSgW!^mX9RQslLOA-0|Ntt z*~?*!`A!oZnb_AXjy=2@DPfM*Tx!@cbx}L_9s3IM610c$=;i>eB^6*#NdFa$6b*ycw{RV#}PzdhPobHin-=TbaLLR|dR*&8_(F!?K>f|0o? z7&uoqwG}x7ZET)*1WfbD)61lhtGJxtVHki*(Cg3XH>}Le}@u*%yY^0s9Hk z6uQ-KGqHrwss}{A1bKQ1q`SYdfTY15YlV^SwjWKPpk{k3 z=Hfg*)(Ta?fvtZLVG^5?2x6cHk!>s(f~{Y4ue~@#9#k-LyjmD9%9|tG*%r)aOg`QC zj1{E-n!?DV@ojaH$3aFh=ljDZ_|a=iFv!qc;Y~DI7qG0XY*lHV%<)85;She)c@hb$ zz#OvTn8$e_hj1!ksF?-h1b)G=yrAKF5N3|UYRLc8P2hHH##(OW+FP-(L<3tj2*TgQ zQdRz-cF9W*QKH8DWpUEVjb**Y?yE+>iG#TYQN{B1MAUTfbrb z1)Voh!KT0|w;x0rTG|Vaiw|AUoa|^s>4BX)cW!`ugJWx@=P4BvjtKi(34lY;51yvc%HxCvR_0OWgH;iEUm4b-}mFR z)W;FM%j{t_7%Hwpnm>9u@xuN%ZF?D7xZNdISYtNVoj7i{(OCT%coCD^^_ z5Q3|tbJ)z>oXlbXhQYq*+~zPaFfiaec`ImT1fKzCf*{5~3X{#v#beJgjrQ#j&Nga_ zKa!gT>=h1a-sNG+y|;FZd6HWcK=>Oc(Mzy*Ap%iVT-@K1tSz>&YJtK2kc5Z^9oDcE zgm4JVPfdYCS&4C=ViXm2m>(NDqK*>=mV_)MxtfY)L_R7yv*s#vWCG80;wTie8fm~7 zE5<3B6uRxjQLYYSc;c`EM)2?I;emlBi1(vI)ZaUHfG93<`m_OdGu#%jwb|Ppro7=a z$v7C{Eak#Lc-(b{s|GcAmK$%Tnp908<)S0?>cq9MBcZz?(DcG;)1p8h96OH@;+D=c zWh^+!MD1q{;+s|Tp}D#4^2{|TWb#PFXfEAoyY0H#?PNr=DY~ms~ z_jK`ahdEUmV3q>!>E&Qg`C~0HO9#Z?%dD%aO27v&{5}h6#V@E~0oq3z<1+5ILM zEsufBijzs*@)j2$UGTHg?De!dgd`we?08`9AF+ubZMD2eY;l<|;iZ1kwk^J2g52-w z<2M5=uZNB|)oy#Fnrfiw%qlG0(cIh&-B8D@Crbrv#7cgmz54ej+j(?;e1?q%v@|$6 zUM0;4CJLPG8+l%T-M@N(@CyPM-fjk7#{K6Gl{UjXV6N8IUs|u^$dfx;my|Nv=X}Nv~KuE`={D z%&HwQ+3HRvbv`mEq1iEggk0qQtgtB$_x}Y-)>(BTi_FMl!BYSPIO4CQeok=LeyU9Z zBkiN~Ee_xR;L`uwyM_GkBc63~3-;5|Wv*(}hfuB6NL=Ndbvox{q zS=qNdA+|WwqbU}b@Q>odk=bGXt_ig)$9|=9g^{@dp^Z>sT|-&$J0-|R)$EVpY5a1E zc%x~_bF$OnxEN>N_{wPxi&KOA_I*G6YFjDTubF2*oYoq(4h3@60gU8=4a&i{-_|H#7|B`!YHj!fS@t^IZi@ z^bBFQu+_5JY<;1`YN8tD^@4sl1?yRkx+Fyt0;^Jt`1tZk377S6?W*!V8@UQ-vH2^a z^=lI36BJu16i&%3eUw>|b$#LAma%cs(wx+*A6D{8tZ7_EV{C2bzUw1h zX`b03A=x3S3yGo3^cQT0f?6@ijHN#&2M1{m`T2=qAh$@0%9iCm>G6GTva$I0j%&hv zx}}8-Y8Dp+f{8lRXc>5pl+DOwXKUL5(PK0r^*$z&{$Q^Bl-JSm)@+2wP6Ms;W9?iq zQoU}FETZ42sM941cXXO|d|j%VJpxZ;(tT2b%c9QD^w_au!otG8)m_VU1f7;EvnLmJ zQ&I5;-B3zYKPqf#TX}yv@w8&NPsnGT)}#R&HPTG*O7ALSh&5hQ??{=Dj2po-UVvH# zPU@G=WUSJ(T_c0$$?xj0FezOgz!|q>=nYyYi#S>k+AdCMR_1Aj5l>2Z52e8s|E9;h z@{ikwtVrGnbzh&(izTb!>o+U4_i+uQZ;jg9(9)N5W@FuJAshF&h8xgWrJ7*S@%eFN zTytXQSG)R+>0USxtTAR0ZfMQAraR0Hz`IqPi;(r=w->tklFQEX9lpN4L%P<(9n(dx zo|dk6GA^(&RV_PP+uGW`DsWwxTp!GjaMyBMMuM7BjrD^`FRPDht z==eoE%%yxPMs{3fgI;8Ilz5FR(^IRn>sjf(y?g!phK3BBjYMC67e3e4Tk3;#d_&G* z;0-yzzpwa(QS;NCt_XQMA04&g^-iHC)0VG}{gYx`_xc70O~ZzIdvE95t%di&AY|=W z)8!PztxG{g#pNsJGL4p-EG<$aaxbyV zndAbZ!mB$G_I#x78D`8qe;0LeEEK`u3w3{)n_OL*0mO^+bG;Kn{=!V5A~v$}?dKD~ z$&S5~I7`evZSVG`J!GgQd1h+rx#&Z(LCJoNq8!D*`h^YjV-*8EeJ=W=*HL&@j#yR* zM~DqyZ(Wh<>Y`!>eZ9kF>fZs?%6ft&HF})+`yn$h*zNzEI;d(V)=;Fc+m0iaG%-aue6Ys`2fRbGo8nirORLpzFm>iykT)dU zLa;9e*M7FwIcO(rHS2Q;YH9tVmup5E7#Fgsl$-3xJjpt`5niV;Ffb5+iNM+U<{(lW zqXPqnONkggjg#>T^!v)t_hF9_ zpBo{<9`hQGq=I;l&Sf41xh3tjlA?qUj!R#NhgVal9%tCxy?eJ(>0t7SlyrK*=EUZy z7LS8!*_O+iO?#x!6%R?V*LPxdq@?nbR1O|KeAwv{6A$UlC+$)>$+mPSiN8o{y6y z)X@y)xX0o9)Y9}4$h&;#poL4AScY{%F!7`8N|U4fC7%asVvIa>t0T{EX;%dvH#al8 zrK5wku^tDC`Qz1?%T}8g&{a5g-l*|e{BP{xgmx=#0&^Lf`7Ad|56Twjez*W@X1x4T z89_(Pypci@6e85J$SN)gE+!An%$D>^tNFX_CWDv;QrEod-?UHs+oj!1VYmJD%0+n?j??iFTKs+LQSKoWUDk9zF&QADVea zR7YnF2W|wx3h&%E#7I&*>9*IT;q&7VRY9~u(!3%HWLTN|%vYOQ((mV;J!#8Jq<;2$ zk)I<^xKbwFP5%l5grgRjA3ozd8&+i>?#jKC4HwN_njn7$%FJ_{|9Z)jd)!pNj5TA! z4b}wKz(`(t>Tc4O!+hNRACxQBR*SYd*yR0;)fN%e2thwaMqyiDB=9#;NAyy=SwxN+ zPv)(i6!-G)8wfaa_UvgS(sa1@4mI<4@rzpQ{cxxTjdXc=dErKi@5GY-M)~e+PW(ut zdMK}1!Ea`1xp09Hv;?e6@*7$CwUfF$5c$7>*T>|k&_Z04lgw+fezqy|xrO68?nbti zlBfQUo3aJLvo)~Xagmq<2%|Ty`A9zqFXvUgzqqlnacI|BBBlX3X1F~ZZcJo5_utU3 zC)#!p-Lt}lOuvutjWXCe43_(oTtu?S?R_cA!Uor-etqRS1y{GCwK!8YL8Z{mG5Giu zdy+Jg58-obZ|Q|>I$=Ncy~h$%(rrh*)Zu!c>xLhbtryy?CCjRw(-Pbrpo%Vn_~&rY zMe~iQc2gZeDQ97#z1WkwJx{xIkl@JrIz}x-QE}J8OU*)wFi{K0{r~#s*FS_CtIK8= zzf}(3W=67P-nMrf#)_(YGL&B~te0hd1!rTues$%W^LzvClik@|!=oOAJ}8?D7guqH z&Sj9d|L2oHGQWHGuAkk-WiWxi%Wib96q)btGPw>55>KGb$w}Nu(Qb}RI+f~}n;D5d zm@Y!3Y-m%!Qbd$`6Ee-Msuy@h8xomg^t?uhA8X$94-O9g`t?ulGa;MF>#?fYib(P$ z93#ulf4)8aw(#L}T|aK96sPDBCLbYgMJm$SHy{75rzaqZKj}{LlO@A0VY?G2PB4h= z^98?>l@%$mS|0uKH`7bZ9oa73jg9{4Gjj$vZKf7$pe1MIR2{|-hb&u zwtTht#(~+C);gC+OpInNPZOk6*9D9^FcP5*_mjG_}(IVpB<^Mb-l(L4-mC5 z>o{7qq0A?U%WEk878@2&gneEQB1*u@J-T#-g62#|nsoG#$LnZWO3J-Qr(1y;p_puU z(U|YBLz{VYNyvC(I#!!&>@%TOfKJ5}Gv#Qaya$V`SZ$A_eGVQGjOY%#+=sT00RHc#>6Mhd4@{hW=c4k>y zgZ7yERxr&NEe1~!2!v`Q^0;$&KzIZ#<`yQpeZqc^ul=m8t%bEv)>i&9+uVoufqIqyAwDT5x_8v{TmwR}v!7Y*drZXIv7kt5W3)Bb@O25Ov=spgbwL(`Xy_;6 z&{fmuUje4g83{ysIkpP21XcbZE)Wmv7`sB%QoHX;le!AY0GnY zA?IX>8@;6#1J&(|%_!+j_LqfbOGob;I7llSEkX1u>-?1WqyQ-XTRgLOGE%qFMB z#~bu*8EaE(T|2^Zz4X6+rM1=~N7Wo2`DgFSbhvJ1Wgu7NzU?Ts*hjWHj~ZV7DgOO0 zJvIL~1Y~X?9mOlSY08f3zIgfazy4qR|JqCE6||nn_mBKFIJf}0p~dB86_-XFa|T^G znbFbFb4sBd7lx1EE38FeMIh~~_*vr;sUDqhaLk03qVU%RVlUY_G97(cuz8Qj{TOC+ zu?0W(`OB9tpFj82Nh>9Hwe9=iM!hL^?Q8zVy@qzK`;Z5ku_E|1zv}WZW571@jn8d{ zO$8l!?>8>P{3B)*g@3Mi`lZBmmCaU?7t1w8X@aVKpZ(D~I^>$}o4Q#al@!oo(EPmK zduX#M-=jmtzAI;G)6>exePN-x_jKhflK{Ch@0N?@-<@cQ$0TUu^^w@D3g*L01#C*c z$;bcL&|fPRH!(N2@I?jXi|zDpWE&fhbVTkS^{p=Ob}PA%1w4Sj}u*L6uTR59u(YqdH4SP9*4F^UB9)zFYYwwzpID#|9vg%^j1nj z?5^j}ANg)!`%C`n{{4GpSTE)M_5L(dWY_p?P&eab=t8cIp3S7WZbj$l(hqGOp$s7* zr>>-$?93}^-=LpIPL;be*bn(8joe%-f;1oWgID2e5$yW?3ug~*)BfQ@7RlM7@`6U) z_LuGVGGxJ~i)oOStiO`BYJ)PyOP=Z}Cznr$FWBxlECUgN-$D~#d!-4>b=i}7?S8gQ zg7QAXGm_2&sD1G+_EXXFND)n)Ng+wdWNxh1Y}|*eAF3mA!qY{p(QnObuhERB)QA?& z#vL_Dt(|Ko5I7F+%>RTE@+6DcQd#)M>L8QUhtsj7GO@K`x#)1`Wv7KsemJ6<%N~NO zql67}wFQ^;L6`Gw1`+s3rJ;N3xG-6SW{ZKcx5#{pf}-*@*2emJQk^tcrz5GV6FM5p z4beOD`_h`#S5ObhFmagjq&u+Vr~5Cm_~k^_Xqf#7>zAU_KAFk_d-B)DO>J5<-OYMS zAkN??&y_P(z}I}wqKr8(K5Ha)o9dILa`q%)QYB9y@}5vSGi(ZTm@uR*=?^4`G7a8T zuvCL==A66EEpq9azBIGv38kMrMBC(Xf5^^OWo?9~?I?C`%ZJn4d$d1!{~7*( zIV5_$_?q4g8`gi^eral@d&9X#WT9cMYs=gP@BVh@4~A{Q#)$$t?1G1GieK6wU}8C& zt;o;I9(4v!xs~Jik*s4oYDg0(dnY;evtRj=di)Y0S<6-4XmxSgd;%3i=b`uR_F)pH z1pDEr2ya27Ks9Rt)TNO6uVKs{PjBCMq302dg5xhrg6&vxslmX;`sBv-<{~MUP)p>P zN$pPat%H!L9FSxQw*BqFGE?F6VaR1;!G%0mlV2*%l+xRDblP-_z!P2Qb`eY28opS> zl9)47F6XjbKW8;QJB)JPnq^eO)}6=B;!DdRXFrF3N=a487kw0N3yC|Q3%{R-+H@^_ z+eVes>n&uU7uxw|!*F?u%X!*!89AVAq)JXiujK4g2f$e5_dln@s?6;K;iyE@< z+u9r%zH*VB2et1~GjsyJ^bZ%6=wgq+;9F85uVoT8gDzDJL%-3*JU>ts&vn+SafB~qE$E0B)wIyd$}oN z9ZQlu3fXURtpT}ZX;uX`e=OfjG&dLUj_|V|47EWTKeb*~NLTUqZ%?jjt_^*$mDVc$ z?woGt6*UyN$*Y7~>E5}|oJIu-uvuV0`ph6l&I$W;b9c`w@(%T=5CS-IaxvY(@avReJ zbK;`z^ww`6g*&-|Quzv&*=M}o?OkO@urz*N+Q}M)JALJ;zM!oUrvo?bo=vA!;63v) z*h6k~B}N9vd4MOp^+EP{pL-rDbn@o6I#kHGGW!mcn(Zn4in=nD$m~z>>{+8kr#<9y zaB(Fyt410A&eW>&?U1{uL?Phm-t)E)D8(NXO3lKsQE^$(ja3poIuFZ&hoGKGpq`{_^ejQ>t)&`^&4|Pu}i2vyw+JKDgua z>aujj#`2b3g;QiJO8w>dxx%~mEC|q$ZUv%cj3pV^;ti5MYR9(|G zi~$e?!3YQf3QCrdAVH!?RML3luZq~)C>-E6Ox;o&61F;qkx&sp!Tzpoa74i=pHC21rQ z!6?f8F>d85(SwtfeJR9BL6D*hWosxSBYqkRj!|fCXcdDQvW%_d$Je6{mLdOfn0b`1 zg49=FcZ^Yo4$-G@1Q0y){QSJo)LW@O?Pu3;>afczQ<%t~{Ualu8${9WDD@m)+3rH7 zP{BXM;=8r5)DSJv#({8v4BOUT2lfcO4KjdY;5|X)1D@$^RnadQT2-S1B3didI2}=Z z6@-C)*ax&N@n{nXWAor<#8)35+Pb(14CZ`?J3+vRFrhwFwvglt z++FqVpP+&mUZvE3g6L^0xRI3*z$S6r&bf-&`gOTa5OwbUhuD*z92$U4&<%p=j0 zKSK6@&x-9pgb)Fc0xo5@a|lyeydSM=QwC<30Jq!$0U9Jr$0Ms=JLwtO>G*hNd58h0 za6J|oSwDESbghe613@g25B3{CHkoie8y(`vF3US* zO7an$I)$tS2iLzG-Qsh>Xj~`R39WpAqNb z_GH%@zx_4&TB`zjpAPL zrN{XGYT!;|?uzV)r%bow$Xk_|@xTBVs;Q-3H$ey^6QSp5_`lPC`45NvxrQN@iC4*)kuj^gJ zMV5hj2o4e4f$eo!Dm1$bkqBmKxIct@Z!px|t+r?xX<7{tgZ%Em%NdW&U$Uu~u7Pc( zd&_)K7ze`$XUUZgK!7>%HqoZm^gM@Mujieysl)6uO=PN|B*bSu~gx#7uQ zxQo#n{}z^IR?$;otOgIQ*vn}|Iz)WU5sq@LG>e^q)l|KSiW3Z%Y5{MtkNAGLEsLtPNd!m2Iu3lu0(6oTSryB z+W=5dB?I#kovy;_v21Dh5tJXG%P1ML$dILXohqA6c&EL5Z@I7=F6+{xD*J0#+2hYe zD0j_l28pp<2|}(5&|a7G$`|FtBUplDVx|e-SGciztuT&WVWQh}4i_~8Kk4gel$Vzy zftVxbn~NQxEdivn@((1{yX@nT2XArbbJ(cN?+YaGNDV-dW`k}2wseP6N9j!U_SMu zS(T-BA4Cv$z{entQHtT2hUeM`oZ~cv`2xW}F_sTt86dG)UcgGAc!j^w<6FS`9rddr zGXN0Q!H%N`&j62F0o=n|-8&~m$Q0Q0`w_(4db=Wcv`?I`8sD39>(PhY*#vhP)lEot z#sI}M(z|ndh9^ry68K5+?%-2^aI_K`~C0{zBIXGPv0q4?KW^D;TLfA8bBR0YMDdD$J+kR7!4!`LDteR{Bw52 z%EH3J5CB+Z(XAe_>L_F)#)_9=($)hxvP;+M&Q1!B!W*mh+HB7aY9cd{6txRRDheZr z*dS~Q%fC)V9_F{I04Qz8f3K$gXPbrZAOMBFmk)>t}5r$d*F1Q@P02{t~$AY1Qc*8+V z8VylyoMg6Z0>Dt%Hk1Ye5LsoGllc+2@hVuiu-7|}VmDt~2MjF|PXP?`K!Sqn0gPM# zvn2+C7oZqmqwFCGw+(V@uOF0Mkh%W)RZBVnt97q8$7>6gA)Db-5C9=RIK<}NVOM^N zfqGO^Kcb1Ut0QR}C6-cNygK&Yz-=Ca!&v>fdsdM9aaV43ah3dj4y#8J*Ue>hS?+r|%@uz%&kT}3* zy*_DSZ0>FJ?*#!9@w~D5!RG7JKFDOkmdmVv1eGFQaqh|}g{d?C_Y+MgJgNQZnEw?g z2wsMt!lTH9DF4Y%2#ls)ijhg3yZ=uLk6in&oYYy7|KoT6^Xe5Mv;zaoWwV$7^VTzs z(zsetMt+~R_6frK!FRJGNmRwyp$)**kmGrXxvRi)0O+thP^J+0?>7>>jAdm0?@Xv% zJcCf8@FSu4-?!8-XAX7$%21|Q?U ze;N?H83Paj@`2+i0PJ9_QX4=<9IUsdur6#6Vnf|a8q7_&^Z=of>4<`OpJacmxWQG3~Sy^QlxL3HMd-7p%7;>UX( zjtmFztgu9IfROJU%hOGt__w|x>t{)L@rRH~FBe=cB>Up4$7(Ff5zOn>y%$M> zfoB1*;|S%UjO=oYBa=h%76mps^JbGUQRk^}Sx6uj6w>rvMz(CR`@^tZKX^i>u^og9 z1*&Wig#aNdu$+UapNZfRa{$x=;Bzl%yXy`kQ@vBUYU*`AK@Rc19bVf7FuG%?U_g0? z0@ZtMz}|E>EfYo{X%Fc6K)FvcuF3iuD z0a*d#mj#+d2G^-@rz;=IFynRt)csEH;d~H)f(zxNy}d;Mj{fF724y4|_BKFQrC7fC zAj5+{kU6l29RbjS=NXexb%c!9Yf*Ph$EjBXQmUelHs$i|s%5;NQc`>q&K6~RHrLP( zDzJ$ZWpw2?x)~{3`<)um+!P479y$pg-UigeF?&FSreYAr1rcw;K9Gh^N?FRIKrkR= zC_sW?sLpiset<(|fHAU%350WBurMMSTY1Pe0L8TB4PAvN@)39a?ma11P{O*n-2UkD6Z#8;u|p?B1gNEaKs2gh=^~cDQg@) z5)aF-^UnUh@eQ~VuxvMxSz5@(WN58FN0zTv?JMBAz?bX+WpKy<<^ueT%h975Wx!X4 zk?5*bTbNUg#2f1mbl}dchP_}ift#y75*B5VPW<2a;&a}Ozt8}3-B@&@$y>OGdzGbas~3u;&2{*@e$opy!=h*gLpWf;_&lUE@1dC z<6xVQ4UpLi{~kWGVAmzj0?=Eke+%+OF1=C}Hq+s4FmJ9_gyI4cI46>{ek~Eq38A!O zX8MDiDt>ZM8`Gu@Z+0*t8|D-dxvMZKEtz<2DxjX;}_9*D5PLlmPIfPlf_AbYWe)+T>r z>pkrOs`$84g9MIJO&YQp+ReT~f7Y=!kLZXsg zHWLel11UEC4=I^;NgVTYb7y=Q1CxWzl74@O@9Z4z)zq~HMx*Gc5RP}yY0YuFfiM<; zllozm=7HW;$CxDP#9TS-Zkmt4We7xP>O1o}_xd3R-P+t_#ksk;{SmAJ?27T?8Z*j5 zhST?sq61_!NVY`9`FIV191yu72{i)oN%kTv423kPuOgPkrZGFGg-d*P{~Ax2Ra8Zy zR%Oh;QB~kzu-xm>6zpUQJ(@wW;;e>8VWoldOw=f1(u#&p3x+<#U5Nj4I0d)#0w_Rb zq3K8@#=!*dwP;oGKnL@lw_vE@iwQr#0M0LBQiWjnAoQ5P^8y%jg{fgtjD3I&OlA$8 zObHVc4xL0k54rH(j6k)}uA-$2gb5DZ4>)iq_ExiRE!{th&3jA-x_<})g8oy0TjwG7 zd1wm_gXLhgtT==9O97u+0{IOy*>&mqr}B6~PEL zZ+*1NhhYZzfB(n{$P&lQit=*Ej9@_FfOBzq?k5R96;B0Jguh5|0e43S&feb`p>=^J zp7ny&#{kI~JlLDb0sA;`C@3-*awJvm7s6{aCz5bTABAVJug(_P%peYYqi{V>GYWCw zqJJL0(JfouM{*IcKv8CKE9xI`H|x-Av0rt(RM4@#p$maA2KovhS=)yBJl?N9b_Z&L zmw^$Br|??yMItpY8!*8u2-gQhN}0nfxVtKZ?nAOj+7cP+kS-9?8Q(CpmSCm%NGT#~I%pBtoG7 zB8HVtkOL2v*Kh^OE{T-CM)m4YWc7BC3dl!@04MCowOfVIwxVZzpxpNyvV0ANnA9@h zgG6`7eT}J)AgE*~)+ap|Ag5cAL9<@E!fkK>gj;Dg?de*GKEPxs;v$Wy5fldD|4Jj$ z%z4x)MKK*#Wj$;R^5Ckt)xa1b4)Uf+VF?O1KtdBr97&RN<*itbq9ZSID&{-m9 z$APhUfSkwgzA?l`dpe|X0EyPD4Ql;cV6JZPJ8<( z8*o`oxJ|mjl%zYmRY;@X#xP7b6{#Prw_N}w-AVOK zFXh&+Kt%^0)TQnfi7cJ{gRJTPawOyeofw*6H)*>1R~?Wo9b@!y_o1xxxxv088X6XsistP~vb z5$3U}k#695AeX%|NcxK1Rn?Kx+{G@*8@+&(`w*9w!n+bDi4NcWK5+T-{=_t9E0mhAcUYV1P1zS8bUwBglgmZQu; z<+eiyU!u<9;hN!rySBkFp3C;xiRpYa($ESZB&>0~#1Jq;+0JKuk zZa4s2^%JomkoEA*%)XpL04K!s+oeg8R9nDpryv9$cXe>k)!@^~toUYSul2uS*l&Xz zU9GkTUQ+_x;ynA^U#5;X_pyhje<;L?j0Um$dQBHx8v0igD;M#@2Q#@Rw*TVB(x>H1 zZK?~D+~>Y`W1?kUL&}C* za=J^q^QN9^)J@66>l|hF;wTVX0d&us0DjSK1R)#`3zWtH92eaYC;qUqvV!EK2=n9A zOQhfP5$cdg`5WAbz|1TGwI;Y_1_Vox)9TK@yX-Ze8IU84-7(Qx)H6l3y6)9u-=Ey+ ziLc7tDAy%#4m2%n>ETbM6J0JiUiD@mZ?2PN8BuystJsm~)YF#{%0Xr(lv7#x^KbGb z#c!*CHZ9tS%dO^7ZG9k;NlT2%YpSzWze@Bl3 zc#r30j^;iuQfgUPXn2}dju zmu1g*p7md;9=&HaTp5+hfpaqe-BhFPW4HNVF5F^kZSU@4S0j86Tnr|ZZ@W*W5ujP{ydj3T? zQD7K)`4rigC z4e{e4^(zP4Qw&&}7UgqCiDk3Z6$esLLHE)xud6$GF9z$H?1x!9?{I{%p%_;@*Ox0@ zbY%Pa-Dn$|!&Y`@gD1Tz_NZ2m>dL|@*Za_W6Z?r?E@Qm(LkB@F*!8J>Ti2u=zJaxk zY9ae=HHEYoma*!IS*#SVp_jwF)@|1(!#TeTM(*simmj3Jg4V>=3h*xx1FyjhR^&KU`0_$9=V6 z@`Po)T~Xy|4@LI9=J?}BhdG%q7P2X_kKOE3<}Kr2v<<|c({tS46WutfF=SI3?TZNu z9w3@1Ujcjz2{z|5gku0aw-0$E6c3>+|A6-x2W!~8M<$|yT>0p_9u?w@nH!GRf87&aex1w z63-jm6A(KNc-GQ&7<$%O{P1;FB>H&2Eq=As$OYF^hg%7sJ;q40l`eh0#{IFV-u*0W zfcN6&O79-wg<+fg1zv7}uPZmGZSZR{gbY<#5d%8b*MuCln79|>ir=Hz^+39lH75Tp zg6C&L_2GPJjI5H#93M`7w$J_5St<|5j-{fGZy9}BeBJoDm4}T>FKR4g<3RWSd;Q(S zpJPtXwesv`aVf*RzSs+5ZUuq&P03PPGfiFp;e(a3bnFmtN5#Y03dP+6~F z1wQ@(wg2Y;uocpvyZbj-d%`S@|KH)UBWFl`peFeiZ2^)P*bQPr6!;h5$@=htm{=sY z1uymj%0R5RVnFslS;L9Qz5rxJJx#NU`rH8dr63|WbAfby^R+yXJd{id8a#^+48nu@ z2uaQ((ct0`ot6SF0uze}cfy1hz5+p_$X0|v8aTolxR?qMXIOeR1m(gahRzdm#j=SwZ2_df1ur^cKY!RJ;%l90&WcwPxG~ZWFk`d7#{oYEC2?M)0Gl zc446yl>uV+j+?~5UZL8H+pchajzoPHMXCsyy@nkM4Q(F{6}3~s-E zWMo%ZH0}z%aX+~;uzy>7Wh-9g;S`mem5BYwVstzCWK+-ZQNq|<#SXS&Qc(L@$Zfa9 z7r#2NJNWR#(Y4H!Voim%-)31)YIobWfRB`=hI3JWTC2k_{Q4Fvx$uI)g?k)-RvN4pIdJ8v z*zJ@tz2xNFKCR@-3=Ja>e=OySV^d4#l{gY&T1maV_}IBDALlzgl$DWS zS+?F^s^`n`d10Qxa6w@6hwb4=Dx2c1C^Vc9eB5U1Ri9VAmjg*RXgA#<#*TMbmCvUl z1wM|`_+RL>p4_L4nklu7Zp{%l*+nIO%FssTs3`av0G}LqnFC3jV>&hsdcZqxvz2GGz3Jd((>U4NBl6HYyo zKw9uXz6>6G6_IuzYCMGIgS_+M6r4;Ff7J@bcQZ)ksx>nXiI zsNin$x1zN&;grB4muG=nKRe&wFp4AFyn=(bZ7vj(>mx7vw^B#`b_7@su1uaa>AO}v zBNV2$|D|NcbD-dr*=;8+vgdvnjrXM-nhbh@&K#ZNCY6qUUm~v#V}1=|tE-<#voVrO z*xlO9u;tRvuNym_T=Y{F+O_+_rJ1-tTY_A*JhMn^wR|_4D$!%!SNfrTY+2?o!xU+E z`t8MSt!v_6FD<8bkDnTe-Lxlulq|u9+r%?Gth;%vst_~TKc~O4%dbHv%^{XvHa;B$ z9=UR1;3q@!^w|&+1~mnYKQ2Ib`cB7GM#txr-l6Ae>@483Lks|8(g`3MNv@O@c%GON;+KdxFnP<`;1 zTBG{D^FUss8TSIn&mtzwwWAe?&1G3aL^>rYJwTX3?lX>L_25)@AkZVVm@neX5RQ?O zKcYoIbQ{pf`o#{!XE})vQ#FIvjgOH>^Vpu|6owgx$ zYJirbBEj)0XohBIuev2os<1U&ImT9qqZP6SHEuq!Fz&d3>pAa&72rMgO=z$T)302J zpr~HFCwa->+w2?K%=Nr4V|$HdVfx~`my5~N*rxrMro=isO|IQkVB=LZ+O>O=5~*Wy zKZ|0eat<&zQN~v%Ekck6~vZw$M9WcO+Pk8Wrk zNRufK`iWGA<%NY7tc&`uToXm&R(pKu*Lg~A;3n$HTdB4{ck!q@bCNE}_XVPw1+wY6 z7?myL32a5cTavw)mL(v+*9k|^V*&Jd9WENKf>c%^4XB=%$A!TA=Rp32=y4J0H&XZd zOusMU;dwiTb5}a(%3q=SP3#2C-QRZgNEt?~^fcIsg1aRIs5G+vVGbH3lw0!VCp{amisZork|qH7Gvo zj9YQVvMP(|OTuw856LOxFYAN``x}cr@bN$Wb+xsdUju^joFWc6pR~=2p zqWsO6Twr~{#7o*65%Cm|tRjFfAlS0C#R%Ar0$5%wy#+NO&*5SOSO6}KnfY%2ySuxd z={J}I$0gFw95O;7LVZuIJfRk&Yo)rHoB(@ly&}1Y;9?ry7OE|z2>|lG9(n#jD4IZ> z8Ci5v=hzH;R4;L9=YKB-!vL*FXK%0cf+Jt|hf{#fvm*b*9|P9VJ#qq!bBM?cAzReL zdb<8B?Wpgcjrj5?hR1nn0HDar;O9|zI!%j-2&7>E3Qi(k*vn zJM2UNy31wf=MhQzfTKVdmlUZ3dUotKQFr3jZ_v*KHvB>I+4nMBR~?f2ukxz0Z6#g~ zD%LTP!u9;P*ndrH4!?ZB($u1*V9)ofu;}heVW2Ii*UoD$aYct@W`iZp67r0N2(1@4 zahHy_3>l)%j%AJVdtbR{S~sMSCXF*K>TBzC#Hpi&HmXM$Lh|mFXZ|6$TWpHy9C43I zMkc=f3TD)wSSbJyi6YfbE9 zmwTVkfyemOo$j+19;$3b=4lnRIwq0m$ge7wrTW!Z9%D^| zM59d+FYEH#UKX|}fQ<@~Uamz?);j719t| zR{W8m1*Rqskg8MPW0Md>-MllhsNdsve*!7EgMt7B(Hlso8K7(sRW@?r{=F7pvXvnD zL&ObG2?jFtvw}7_g7|yy#nu3%Dkp{p>7@D7&1Dr2S%44#c#nrbaBM+_cQhv}j>Z&6 zpe-OuSIop(f_}K^u1jJ-PRC8e=Yz;=33U0tptS@H3pVgU3i}3weh)#bWKxf6hXOjt zc`>ouAdK(;zW*l0<>OA!TH!(ZYGB`Rc{2);h5-$YSSh5EAPI{HT%9TI?weMSU{A;W zIM)ai=5w$eB4{0vf+kf^gRRFT?N-?RQ|C6_Ci`_Pf2QXEMC_(r`5d`W7^V}eTL zXhSKZbkVKbKzDN+r+nliCe*ib(AQY)c$8=}_cvbj%hLGi(IRZ(#Rn4%VO9AH5hdD| z?|Ox`_C`(mu@1A`M;rCW`$re6RTP}|+h;PA69(?A)QOMnw#8TPZg(B)?e2yY^;yT; zTe9M^4;JGY3~h%_eDw0!tmim876+(Ts)`|2iD_lj(lP7Kr(OKLR+q6A6NR);ANxAV zkNVET;=-ge!D*K8QB(1%4O9HohjSZ|^{Gc|THH~;?Mkf0X51zecaAyEObWeU(spXr zwH7q)wfB2y>xt>B{aB6fL+dDo(c0v%l#-Y%4^|&FW_DN~wFih6U=PD|7Vlz;Aar%C ze5~%ilCjEba;=MEgNpNG)u3GKV_HA7_gn>q)NId4oL0WEo$HU)ztb8Nht4~^4#)0o zB%`|AS*~@x4T{d`$59$$p1&7l4Htf5XR5UlbJzMDe!h9vCp;0mJ{B<$bL==9&AXpR zVr2dSREU|WhCxuzoP=_WC!}IXc@yCUL3%y_USt6~-3|4QU$P9MAQJ~N9}jd=JO~be zrI5fXL_9q@*weKUI>|tv^%E382%jw`zK!r%knU~64n@j>6R(i!FUZm+!Y6g;UC@sEdL6 zR6$Wy!5XqMAmX6R0p)jy)EPmT_fM$;1#?wQ0y^H$y;ntHu=o&U z8cz@rA3^{NP6opUWqIOHLAj0)AaJk9;(@XiqDLTU#37PrR#7KwBevTmt!N-S*>$8uY~{tj--Kb^SM%RF$BAH; zuwu`JgJzWBCLU_;vgrTa6vcY&zw*7CQ}n7$;PEuazKsX)(PFDVX6sR!3Jvz_o8sbe zW!7!XH#^OkJtw$c>>Y=Wbsn!)?;lK2c5NI;9UtK}RWYu>XGa!*t{5p2BK-hNSU5i{ z5s|nXut#0pOBsjeIV3DsACGhf;Xeh?4?!e;&)q038Ud;bS8D%*Z8&LS>`4%p$5_sB zR&(#;v@s-}vnVGVKt4nE6gQmsUhIX*A6z27U)DNzuNr&Ey=ahk%u`YUMUCul2Qv}m z;@H{hF-u?Cd8v-7lHT_-oU8Mb>%`0V$GlCq%XNbXjgK1~yh>-@^l$D>e7a6bHRj&% z{je&5ZI;wO3egk%r^V-7STi|L?jyh-L;7Zz8plCx1N}cir+_|&P7z5s!D51utiF}I z`T$Y?7d1G@k|22&(pd)bOT`Z;W7skcA;gq&jV$HK61$CA1>x@|l#Wz~SS+TT&(C zH*%9k=I=4fvlR_dwQr4ntUmZ_NWL>f8MN}PiMkpOr3TE;juSIy4R`Ahr9Uh{D1;!@ zM-VuwZE$xQwF7elVFT&V2-zz`G&RtSI9_@lg%$$H1ricGv^A|kjn5Ng&B&XP9#2m% z%S8)5BWu|U?^m3lJ3rWnGWxrjXqrXbZogD z*fW`I_$?RzD%50qk?9Fd;dz}1W>2g_n%&=h;8;;dYKF`EDZXaSp5i*DtCM`e5`4Zu z9d!U+R!hXIpYmG#FN3ejkf!|&;xTr&o}p^HD2IQ%B)jxaAeb3_gTo1Gu~Ge_iOP** zkFIk8a8nk~n=c9^=66&57_eibhz92PXzgc(p|!EYfvn5HMk%DWS0%T?C9s75TB4x~ z{8CrWe_EkIu{!s^>NxRDvO=JQr7fU0T0_)qHDe9lHr!AxB@ygQIZZeB>s06#t1$}T zv1Au=P#63p-8v(G@;`t5zt@I-M>5O7C8z&4CXtTjEsEfAZz0 z6aU%81C*e#692XpfVWtjto`q&zLK2&?~^6n`u~Nj|Jxhi3@%+J*g8|s8klUBLuhW2 z%o?Z`!TgB?7rDA$wOWzA2>*9am~~H}I_*Vy+L#)u`_ihHr0wbajOA=61+OQwDqJvl z?3%Z!fK68ZG0pvbUH3VE*BjyGnHXTb@w>=b~=slk+74a2;c#(KL$Wn&i_C$WvY_-g z=G(L)U^-imdUWv%B|*~pn*WN93I6xzWehm+GWyoVT4mdeD$RJBvw|EFY0PKtcWkhF z*Y2De+o?yb9Ws9Y7Q>p%=`R>sj|w(v*Og0VO*WhqdcGG~Q)4T{D|w5#i6pj%?`AiB zy3LFD!xYR_+sy1m`G~o4P>0iT;94B!Aoy9H)YpG^S2?HVi;J+jKD6bgd~V2@?kCJXP%SH0rF$PxgOUFh$4Uw}>Vty+$m&r^c)1wc zhH1Rqogi|Z*M;-Eq}Q1)@O-yVZn@A9eF7cI0z{bga9Ovo-%wYd%^1rO{JSw+p^erU{VM%VnTw z)^lE6$R_&N+eUuM2PU7%Rp~)Um1S3>o_kqoSQGT~xJZq8K8adPJn5bdQSkE+tkPr}R<> z`$YCt_9FB30Tv@_e@v{uF}m_$YyNH)I`Dn0rl)C$F}1&O?3J{Wt@-9ce2KSQ#pPIh zPv?W?y=Lu1E%ff7tzU@UDx%C!Q_l8gWDJv}mYE-BC7U<7PIDON{`U5lCGj(UdPy;# z$s;U`?|;*LeuFphGzk}TyvqC&4xZaQK`q>6RXXG1H|6Voh$^zA1xQx0-w{#n!&*Eo zKz)_FKHg!j<6N~V^7+cQXbU2J9ei3!l0d#3+sby*Pae%Zaadx!XIkEECStblro6RG zelzg-lQ1&9FVp|opq%Ano#DD>O7yxUH5xCeEwrZFqCEb4nvbe%Nn$*PnfZaET7w+c zX1v;E>+yxJS8sWl-5UJs>BT8t&~dE+EvZqky1|R9`MQy}NyXDw)>3+n$5tqm^)iuZ zhtxtO>le|_=M{-&g$9bRo#wM1bdF=TkJYrXpB0SOxc_x?1!)a1`tTC5Gx_JAC#0vT znHAK|`Mgz&PFqopZCq08Womaxy3)RR_dHwEyyc#(x;2wnfsI1&+y?Rb5aY&)vcjKr ziylcGp3)a%a*uxMYA_K?wBk-riNDqMNxy~aBAX#SmWmzIQ+g;jLgAlHn~E7KzU=R9 zI;eU{SF=f$MLyE>=2A z9Ank2YZXK1gVijREfaoTT9xjjJo@~{@7exQ zBN!N!nilZz5=>2D-Tkpf&rZ%VoMn-CaOA7bd^%wkhyKJ#YwpA6yXuaKL(lC?>d>%( zxY5=$l&qrQa`6prE2bV`KOdb3r794!sM;$Wxc=OTMxkSp-CzQX`%ALsYLj^yu?-h$9 zGJ4D08$r5zX2Ffl|9r>nZgQy}nLxg^!h<~Pdr4mBFi(6t*D$JZZ7j_>rT zk}`$k>7;1+2(JW8k)Np_760oYOye(KJNf*&dLM~M0G{QoCxid>;rDtIk#*`{eJt=CtqZ^%3u}*iLIg{~>Vs{BA$C(4Zj!yy`wJ2~#e2Qx9 z6FS*Tw`I3SxdXTpg;%NmTxV+q4)p-@)qs z30K0dyLQuVz)-)eklOU83b(Dzw^9hnwN`katG4>UDwc>vTBSMl*6KNUY->UQ1NG+` za=ygY&iY##Q-2<8u8KS7^Ri{F5fi}QMRvp4PmSKK;pK~vy~VxnZ+-^yTP6m-Sm#&P z&iXr%WY7~?p0qgaZ$l^e8FN+9B31_PeMs{8in+frxJor~;^AVe;dAvq*4^3{8uFn1 zmhR&>v7U}k+7-}aEOGDOv)5aAdz`JyqkKUhb$4APEIkRIy%=#UId$)Q+FzF8TTNt` zn-}OWY%|*>XDOr@KE2j!wi>1Co#~^XL?m|lDgIrF_tGxkUq7?7Y4vH1MG+TZ(9Ne$ktW9x3Wt@yj5-Oc)mZjb`~!@>n$3?TcltvI{r;PW zUC zNF3j1rWIiiX=%IIg-N_IE_shf9o>c^8!nc=ZugKFeHKyYbrt51j0Sy7#aBT zktyRu4X=KGJxV=|scxes^z|lLa2=|Uvm?LdaQ6Us6E z6m6q6aPx{tl#G&@p?Nni-?Jh;6jvCo1-pM5#v6K9hVXTwd!_y zOJY+1fWS|bA5xp;&-ojtzWGJ?TrDyqJJ$c}9}Dz-!e^Q)uP>2uoFJ$%N8DwU??ZYL zKb0#KH+Vnk?Y<7iMx5>ux_-u|FhnQfiXuBUI3>)7aXj0G=U3U_$8)jDL$}NI4CL7^ z`&a%|;$$w>%fZBA{BB-2O;i$E%-LOR-CbyvT_1b*-OOFP&6Kz^zkYJTzPFts(2MmL z^I81*@+*Ld-wDH5qhg+45b0nFI3b{yVrC&9VfXPaUTvs~D^=4{M4haNfH{V;h*>`3 z!EE+5)k}V#Fw|z0ulzrKOl>xn^*4STja!sSzd~AU4ZPR=+G5TM6;_o7uKLc&fylOz z&N+}S${_t{nS5{Ne#aGCht{DsMLL|W^>69*-%J}XPna#EdlJd~+nH-ZAx-bZ8*$3n z@sqMfkxUP#yN3Dp88z8S>YGrkWNCVjg@c#Q$Hy*Q+^ODxHCSM{P+wnx6iY*WSlmV|P#ZoIvGw z5IH%iq~=kA zo^ftDW8Bq7N3r+4g}32;@aD@;L_AONLh6bSqDfjJD}JKEqbgXPkqR1;=}rXi&M8i| zj0%3F=JyHUDfL2msIWd<6vl!wK4Q8s_#zsU%D>5Mn0Qs1PBq%0Mnfa*X2Gnz zaqs$RVy5YWn2(jej0$TkybUxKuxET;K04s5x=mK!v>{TD;tM$$GuI0zIz~wi<$(de z)1!>MCfR&}^TJyeM0iVbh6ptj{!5m};&lsyDv{4(IY!4vR-Tm3|GO#r9kY^danwbc zE@!LDJ}2mjIxm$@Fv^Z$Vzbdg=ly$JewSG5cDI;5?%p&H@*wfM?_KxKssh#>G{9fF z~47_E7VpYgar5~}u`uHRlEbn=1fqrY-{eWWxD zgw-0DYmc&%#-^-^Qi!fDJ!_MQLN{N3+pIetYU^*jP_C(@je1m%;%uGLyC&>j*6fRJ zK}E2#vxqW%0(G(sJtdh@p&A$co3_KZ@Hia~bS3Hwur~Ug>QIkNLP|-b9`P_GW>9Kv zW1^=&#Fmd3sl)v6pHl7?N|e`BBJ{1b@J1d*M)hajqsY;>(a(A#d7?-)(q-&yP7&0c z0bfjjdu7E~As(WTT(eTMAu_h(G4^Gq4f9=<+W&mn)w!&~cMD%neo#vam&=RYyV!p} z%Hbzp(IBc+Ap%2vYv(0Xnvu37^^m&11{*FiP5rwVvMu<(>DFlq!*^R=b-VQWDsWaP z`9F+j+}+&1=AHa30k|o%FlaBbbY= zaT-x%t@&0JC1e?nhHD(7u{jqSNsCo+f>dPF^97yedlc!Piko|e|}T5gX2T0DYe_xx~JOFn5cUXhBu>!_}_l? zxfR(-!de5|N)|+LYVRor5_6vj^3oI^g!mwod`omS3QlP}JAJlm5E&8+u784c&5O~WM!W1D0@p$7>NjLUTn zzMxsINW%Xd%N++_LNsB5>~fz-`pfTWKP4$;RON)dhwc(fGU9wPEhuWvyws&>!L(S- zbk!SEzewm{`<04`Nrm_TIr=mesWT-+b5tmks&6k!@m=8wj?;30v&_#jQrEn`srH4{ zy6p3dxtHYo^`N~e+)FME}M8?$3 z=P^bVLlSKo^ZAO^F#9D^J)xIZxW}tkY&o9%Rqr<9rauk*I{!d5JhtL|&+oQkAV{q5eOAY?Q6xbElcW#%i}j=4 zsTrM+=xY$XzGi`!Q(Dkv%z-~6o5%OO-nvoy+_JtxplF8MMA01nHo0zH);T;KqLB8S zY5wP|>3brBTN!7X0OFYwGEz%aM9_dZCG!2x_$z-(=}0F8mC}0D*o8u28!5Vl!6gX9 z(X^bOFTbtB&zvV$~l{%T6DRk0<*}~eB?6{M;7nm_5`KVw=(YNGRUikFhr+nq-qj>q< z*DoJq?z*dC+#Jo4da=2vLirOb)W22#oe~jQ;hl(`dKH5G<^=HMxoY6 zvcH~@mFnU2BD8n0vQwGhTHA`x_T+hAEgvk7XTHOBTEAbYDE8LdDVXpqg43tWO&+t@ zmB|!8gvh2Zo61_6{ZN~<$1mhYxKEAp+01;2XROCL^21BuKrlovm68 zFuv4y{-fX;U|;l9?|nL2$r|!stwq7UN#pz8))jvDA;4S-BD^pgn@Zy^*>4!Z9EXpS z_k}Ri`2gmJmWlt=MgPl6{s{>YlaX$23O%u0XX+SPORUe;v!3ywxp)062fzCyfjNcX z8&)zY#qO`}zS)cFE{w|`Y}^xUhi^Ms&CRW)d=1Lx0eX@5jMQ+x+kTAQF|pno_y$r~ z_|7-wKcdN`=lZG%su4s#}_Vjf@Z?Q+?`h{4~}dgI87W;R}hl z@>Mfp792!OY8ocicrG@z`#>3#ZJSicm9Om<)c&fm=`|z2V=L_mf}eg~RPJty`QkBF!a2O5XY);Bn$)L; zCGBmW4nKMQXC8&fpKpu3y36`?-wD8O5HjSHWJ$P*OMa~U+pa&mlowNpBvnEIN}|a0 z`zKbcLe-K<(=YjdxA0c+EXpM$Afw(E=_LHvu3k=0-4Jdu^-g14uI z)tE0lJU6&P4s_}icOkR9kA!Q2tc>{k`{z!MIf=INSi@p0P+=V?mbR&9mM5#9vgy3^ z`PE4Um}EIHqYSCL#9!zhs8UPYQ~M`qMW4=IM0@eNLN+m*YLzhC^$|t(+4h9fTLN^|147Ty_6pOU0706#d`b42~1h-rI@B1?&(1@liKgN3z_Yb`~?S3}!%ukE)8 zfy`~r3+kpmJZ+z6-o&*%I!y3oe4Q*Pe?hUEO}{qZhw;q3Fz%0>F*UlO*USciZ9K?; ztMn$Z_|*ut<%g!Px>qr%`1+yKez%uZxD(Ew7Q9GcE?gApdPzFJ>%4$ligNf16xj&S zEj*rMp(bhu4WIASp{8n>2V+jy0Mwy=4i|K2qNbdZGAbcZR?GI8-BYK0BNg#S@Wy+e zr9#Ta@JM1x5`sd`CBC=e(wEXMA=I6^Oq7^WY^t$Jgzs{>bKc`k&kiU3K+^~@(uC-0 zLK46rzEBY2JArBG=4%CF`z!0cEfb(4pT`_t+C~W~y}3ZG*Pn7})?+T*$BH1yZ=yBP zf%fb5l2bu6JV zW|Rrpm$6hrD55F*z9dUg_N0XJmX_b0KHtZ$|Eh-?i1-38S znY=ZejyL&43u(w`ephLE@elpdA=>EOOgEKuR~S0@ofLRiNG-*S7n>M(@}ox1mHWc1 z9gTeTE{db3)e;-C9{NLjZ2y^c%tx7^T4g(zs^WYMpsKAx6QvL)A#Ok|B3cBWLQf zB@G0c`CR4^dEL>>raUU>aT!2b0Ubg2ss!jAVEBzv;p$hFllF7YrIH1}6d6Hb52mz- z88IVVBr2Z7!+-oP0SD3yXpE$&7~M+n&Z;%c7O)6lB!nC(@mF=3a->2bjMM3+)3gD)f4~e2!}mOcgo9pCK%V05ybh&i8R9+sf>Q zmVg=2CFcQbYQxbm7J71G?b&Dce<#c<&?lb4a;b$zx{rkjn}t;UH03QO_-DylMy7z0 zosHoquXLp4Eb_rsvlmiz3}gG$3<~(tg;vWc1>S3o4~)+P0?j_2&mZi_PjEP)Csnwd zS+HWeI%6_kx*~HmOGz6yYVCKEx})6HcUBzOjaeuO$@lJqpq7BukK#D{Wkuuc2=1#q zeQoOHi6?=dHhk^Mv2awTk&z85oGw>H2nPi~IUXEJEQ6JMx}6`%m<9zWn6dk)=0%H* z=uVgjV^@!F@`7q;$c}x%1TzH2_9#t{=zcc@W?Ur@dfHDkLzWjn#6UnE@lt6W{(+~! zb9<%K3-~==q+qYqaI4)vRW)rb_n+f1*1J!ddqw`S7Q2ZKb$9)p2}$=Aiax6`~#d8pWQ^ElC` zV48EDdg`^bwB=tLKPW8@LVniGH6>7RB~svaumcVN2i#WG4Di-LmrY0MXCus{s_j2f z6QZrHj51lMq(HnOPXY54?RPY=Kx+~BCq~5$I|ld^5|iX%F$GPsvSpz1Y(CDBBpSZH z$7MC}Ke|uO0x0$mXegxfZip$S37XifYOKZ77A{(>;a?)M$}}?d*&XtaMt}!T_!Xa^ zT3%z096MvF3i!m!ct_c{jRE(o$_bNznhggC*WCI?7|&S{#pYSSY8BF1$meGf8D;$u z?`&%gNr@Hu^Lb!gi)p^Sz8V~SjQ@=FikTvfszUCZSIdeLzRjzThu7Lkl(j+^M}R5l zih!^6NLQSG)yve!fD`+Vdb8%Zx^y^sw3c(NaB1=L3=^c8$rgcgwUsl<7@yvIy$`qq zHiOv+olJ=#6*h_jFN>MnU~qT$|37S#A2$hg%b8Yz9v3eT?7MyHUItYgHo0M5&(t&; zi1n#j{#9WLB7RBTa%f+BPq8AEC;<@oLRswX#sDwB%PF8N_0RfQ4|uCq6Ae&+8Q)0?2dtR1 z?IGVDD;lfzC7CF_@%!6ZEMy2sq+i5ZAEh9mYL**TthF81W|%5WTcgiH{tCd?paF?g zWwXlNGnKsruB-Rc17E*nSvtUtmndykX?yV^N2<7H&h~Nf_75_H|ETVj*mQA52vo7G z+#Ixh_ynA_oQ>!^V>uNrJcNwBwjgcO^O^wFn`)&J8dUa8*84?j^Ep?LZG)k%D+ITv z_55)cO1M$>{2Ax(p#Zy?P?~gq#4dq5Rz=NH6-q(s^qpa&G%Gx*?yDkTL(SEvR*Qp1 z-h)AAetguWm6(KnLNZTwv3S)^=e34a-DDM2^8!p*W*Im)va5}(yT^18EWhB;@*@d; zXh$PJ3EP6T41JF+rYgAWrppq;N`*7B@(44u@xGDO z{sFxxBcyv=|4_H20mdq+8%n#hui&n(LD8Fym_m*}03QJkVv==*IBHEauuFL@ z@f7`PWI&hU2#%c)kB4SiEu_&T`6#Fzu**_Zm6CCs#~f!5B#y-=htL>Lfgvk@Ba3OZVC zJh^>Nd%gnk)^IWotHDe=b>QxJb<6<9zu9Wc0zfu3Kf}i_*0|FNcF;hK#i@@A{^%)v zY3?w7Pd>VMqoYDl*eqt5%WHY&nvHNZ1YMRQ;X^cmK8C zh&()8uN@8PmIaaZfK}eVxJu(tVn`y_F;Ojf*jtxT98v;?RorV7T9q}$!*hnz)hl_A z_+_(u_O+*wTMZbJ0^j&J0AMWOgZ)lo`cU&wRgkca_T+T(YmJC9;D^%pa8%u$8o!b6T4=p29jV5(BDr69p$Y7vj}?% z!tA)^sOxKc<2IG(%2~-b)0v(qU|{4}rh(}4i1*;Clwi{j#GwX6MoH)Vzy`jeX#Kq$0Zj(1XaX7D-Uv7yz2$N}AxKFjZi^*Ecw5d3 zh9UXF07c%q!=NXgsZVAZQCCvx$UKGt*%;LT7+8UN$Vt&UoV(MO3q7_^EtbU8EYY5{ z`v|JeeiGr5AGyZ}IM0Prug%uLwFgZ$YMU#0jzG&+r$VG3zcypNsQu1xwUG_o>SKs5n12aXt%N}oLgAsS>`p1X4HrE@~y5S(Uu z?TU8+IUCY)(5WN{I^EY$UT*LmT;}qI5?`eX(JS+$Jy3s;;$#iXOtW0N@aoobFa;-Pb5Go zIJS5HwVYT0B^3%FT{lyIpD5(*=Rxio8{q&Nu>v%Z%`5|mm}8X^y@I-`TVqcOI)?*nT3 zB9>7P3Vcrq=4Vg|Mm3^&miiK~Hf)#aNjnC+t*FADeUhdFjvloIjT{1)5C2 z&3SibWHXJ#U%qdw0Zhc;Zc%1X6j?#^BXvGF2kkjDD;5YSc9>|`Zok5oerzk;9M*Y9 zY7+5` z;E%GKE47q$!d1867I<00=I0wXr4M8co~VNq#(}*t$dKp2?4#aKN1d?<;RULbg;FXs zq=r}mpOt`-tUPEi?^BWw&>XDoD{@Z?NQ(4{7r*C%K4IWX6!2e}3KxeZXl#LBd*eM# zTPbCIKKUyX)jJsZ1D8+L6M`U^6S!yUlt1%X5jc|%KpkMOTdA-8aIriaJWbC@9?b4K zutoqBTCnPSGxZwyuxf5Pa%{F`c#g!+kOqukM1m5peyAY~E64f@usPIf&4Iosqx^$U z9qg~g+8a-T)G!zrF%Hgx)`xLJJYRL&>RFA$SW@8yRYN<4feTGLAg46LOpmZ>QcV>K zOcl%?Lu7$fTm!E4Yb?mhKuDTjy}@Dd#1T}S7;^|L_NOv}GP=Ir2MS8T2>;tFZx44a zpRvFjP7yF9J?jLJCnA!7pY5y=Wz2jNVeTV578bu$y$fGD)J#icmH2Sx1gKu(?8Vge zT~w5ywD0z{e?mqSc=bEJ+DxE3q|SzQg7G~X;0b&<&t2`M{YBdWFfAV-$($8q8>dDR zZ;BVp+HNCGo0~ufDG?dq)%fUI05)W>aupBceUE_1u?oX?J5`L?+D7pI)?$4M{FPcC z>x(D?%;LLuD;?}=>e;ftFSecBO`wNlLCNn&OcB8oGsw!GGAzc49oo44^Jce1B6h@7(Qu15Mj6E06WYc415oWH zWW)4|AcGz!Hu38*wNmH}W2|K$A)s_@?;wyJc6y2*9Cc1>>a^w@3K3mwg0`#u87?D^#Xlp*l2`%T1q0hIw zm;BGAPrVrhYCphfpa5++a6*oc6HgDoH2{u*{e>)z@}HgLkbt86plSliIF5Mu~+UEEO2q{mm)JBzG^kGluO2PXcV4lfy1 z2mSYq?z01Dh8V>CcBPo(Y?)D$r(v>brE^q;N-}Md7gC};)L9YtKs=t__#nMP`vpQF zZD%j)ekZ6#jEiZ&GbC>#Emx)=PP3T{jOR8oq2|Xer}U?gAr$caLG9jHu5iRHpBZCs z%ln$pA(km))YPRRdJ_z`rQ;xY<2l=s++1{&tcV6Bwo6Hk0iTNlN9RG2l#`|hA;`6p z21LAZOZx!m&@LSc_kGh=h9)mw2l2rqi#lRmwX(74+TzzhaN-R{~ zcnZlz0dxLZSyT1vsR*-aShgwTB*XtcFKgt(sjwVDWsMwzCUNGtz7P<;ev$#t8MWd2 zPz~4sKR`22fLUpesaj5jgafJ0xb)qmDooe)`Buf>&9aRw5I;MxC9I+8x9=nV1J%1O zYEskDm=^3IUvD-%fv5xy6J9`EK(` zU=LiORZM!z$Ar3#d^0J4yk^aLgtV8ce~eLO>I(r8Zop(rKPwxgbE0^rj>Z6&?hmMZA0wLz zf-gu``Mg8?835<#cNM&{3Q7vem=8ZuDQo?GH}viXqT+AcUrtM1@XETPUA0w;CkIu5 z24<*JfL2}|oF{53Vg1a{eYVAn^jr#H@+{7s(8+;8M^-={Cu&9;UuNqbtFM^rDSTEc)U^cT6F$@0?>`h>!_zyIQL8oO$ z|0(0HRN2g$Iek&8EV(9?7m75QBD5`Gr?;h<jk zx3#`Bf=gCtlDy!Q8JvcZUX>CklN*&D$OfC(7VMAzTVcNB10j&nQCWbh73DMNhgGhP z+n)d*=idLB<~t zWnC5~YbNn}t9tM|jjFldAPLYHrwx=+hr2VYFtwhhSMX)&pDgM1+Uw={OjInV8yF$Y zJe*L;0((ZnN^aZ|YZC6)imm2&G^`ka-SJy{>cin#Ri8bWX)iFd*t-#balirAUjFsi zw_XDR^j+XbGk?)#^*)`Pya~D>FacNlsG1B4BE3u&wYx~AjKxs>kFCGF12)+x6e4_* z#RktSmngn-qNo#7J}3_<0n@=@nyaj(p+DVW*y+IO2=~5__@yu^vK~ybir=JuQiSY( zjpl1oP|6Gy@v;1|04)OmZeT_-JYF=0>megp!FI8L#iTrUS>{F|-vA`}KmV}OQM|1+ z-Ni=bL9v{r0&$e_+9Fj9o#hk|$p_xPZShFI|01xeZAcHItbbZxYCzWpgRZy@N*%Ug6-7PKXhZEM@wj7zeYfP z$}NcIo(b2$QU$>#aAJN=mEVrZi7DOy^h-yjZN_sJ0!I;M4xlx%u0&wQ@~POe!!e5Df8 z)h9@0x4s@@4)3RvFjb>z9(edfgb2{0jQfKp(A%o(533{`ByV@r>N8K~v-+fp&1=h8 zgKvareic%W5xgo_@qRJ19sMZ-c_fh_f*uaH|_+zs+uN8{m-uLj(nf(4iLbOVgt{efzqjM1E~x%J;gM^ zl>w;hcize}4jk6P@ekG!{m?Y{5>gV44z>8h=2HET($T5b9$AE=ATI{iW^93L!M2fdo z(*%G7gkIb_BDxA=ZEDL)e8;NinQtcmBxw*uM({Se`DjI%)AS&f1?cvHn3;PC!kKW$ z2uWZx;4)iXw!D%DI(yoj?_&h3Fd;G{z7Ii$Y7Y)9cgbfhvn$M?(8wRRWPXgWj-3d7 zaaKDMY<17BJIP<_sOvhc%4-vIMhX;fr=Y=ifx83UUXnCQb%I9XKHA4?aT~yCH>(=F zQk*Asc7X+VfX)t?T2&1w?S-8Q04dgws{&o!2xW8I;0yBodtcNman!0uiPqApXG!SI z5Mgnurzg=G!6)9mJ*4xWGxi~Xcry|@w+#oV0n#i?{f&{4X9**&_t34 zMuY8fR~Znx2Giq&k=Ao#^=7m6d#cWA0)+>}g%cPHB(`HJYeOtTnP33;{fZ=VG&>0Q zcBWTMB;7gKDB}^A<E63#H z4^V}Y$xwk(4i!U{#{-2aT7ZV8ODY}d59dJFy2#F6kWcIHbxW6=VdbLsf*?S&g_C79zc@3G zuEU-IKi8|T?gip{#16m?6a;wpXHnDVT6s)tUga<=N};&@82?Fx{8YIU$@R~3Gid~i zGC}2;B8$4L^qCEi6A;T35oQ$rUX>Z%aF&kyL|oLyvF0}YA1$!F5@%gRv%oJyn=KrX z9IOUkoy{3c(RO-GqMePZc7dfLE8$$t-J@P5t=Fk4g#4P@C71z!ybwFk>T= zI$6ktinGViK~3Q@#AnO!Dh^4Ij6p~YOEyp17Po4J zCocvafGR3An?gb;ivPfUT`vp@EUo+{^7x@tt>BC0TA=(N+x>x$#~b>2>t-$bv_=S z$1w797d)1wW^;ZFDm%Pw!VK@;CW{>m(RYBPJc!y329VY@+c1#8ApEI6tWl~NAx(${ zMbEFtH!CyK*PfQ}1CurC4`3vxiu~aIuj^tCgXqr>E-3K6(PFUq6E-xppS?ls%DJ+{ zvFS{)VvMQppVk7OT`?(3xr}7W>kony=YL(NqKmIm3*`Aewa8-uDzU3T{c$e*^!*l2 zqt%z}FA>WJj0}KAgD65Vj8>PHV`E{)U0)Yk`TC^a4XarirSGPuB2ve5AEX-GZmC_w=_gY<^lDs9M=A&~| zKlH#|TYgN{N0L-kzmxFc`_;pwuVLEwAQJzLJ77F0php;i9ZPVuaz;+{ z1}RZLdZ2CKMH7)SHTh})_6l9vIrt+j&U`*g$SvLKFI6P(?WFFjlg3iY zUsGiS@MohCK(i}nhd##4wZ#EPM?-aVtd`nRcIwHe05A6qD17$b20fXLV)k^+pD$X! zByjWoq4u=ksS&8hV5wP*xs6Bj-5dH-%^Eo^NVJXI>+3?DQS=2gSBk6R^;=5{M`1~m z7wm46SDu9k?~e{VpZt?mmJS`EXQOw8YUwBp0eNaGXbHCD5%7|${4e;S?H`FUO)oZD zng);4Zds>|@qo zW$ACFE@!w5-DOE6^6fM>d0HNFN7T$j6fwmQ$f#a6{G6Vs4XxnGOv<9&$Y6X}tnC`@ zX88W+fA!_8dDri-d5rZ?#Br-yXGcI+hWjj`EV=^p^-{yw2z7}ii8a$#eF$zIn|Ved=tL9I70&oWTKwHgUA^6 z*Y}h<9pE~5DmF5OHO8E}(Y1mPmFeW8H-hJb8vL00Lf-o5UCKQY7k+i=f=SsKZ*A!3 z2yl^G=&yU#l_!&YjEf%eob^)?V5OD}wmXyxh|lTyZB%3`wqj41ZeDvF_{>RxDpHG} z{>c_lHSVZb4Zp&EX?+jw2I!iLU6Y2!ia5tdG^aJ2`)m4vyXXd419e|OpQD1E??ts( z(12IYOFcC4jfPw4(cQIUUj3rjTxpn!C}Td$Ye8=duVT+*BL5JNLbr+y;9m9-+H84_ z-xZtt-`Iw?;M7;wF%GOc2bZMsMRJ5~BnK3>w7h5)85MTu&GnoC?NM z`GJnwilCiQ7kW)NY+|lx21EB4mCE_#B3+ga}Ot$7c&Z70? z@(P0%;Jipx-#o!Ay&AF$7}$fujN7V0brF6uEVkfQkK?Z9n}pr7yTt-sA`GAM=Lt91l5@|tMjT5SvopsN# z407rax;&Xw!QY0rJa^vSRPmHO^!k(CIVGxcc-st))gJ9Nu|Qoxi{g@rx3J zxsZleN+)A+!4}L+8Kz?qu{w@Ony0535<@>kyL*jgXgp4StKWGd?lU&>vvzsCMti}k-qovD*Jro`Pp=xHEoUC!;D4O7 zq{&Zsuu;4+iL1#&dGJz>-Rn}Azvsr4pif=;pIg#fI@f)WM%|_l)??^MJIP|C34L`c z=sgKA)XLxkTYeFkJ6^Qte9OgWKLWk}bj2Wb$8!taze#`OFn_?UnC79*ho7NM6Q4N0 z>UV0W$-7Be#(7Vr-SoDHqFNYpR4%XT^M~o9f~&06zL#-(Jl~`2@mNeQB@rq@L6pfh zf%>E?(LXU?_c*0fU0#iN&>>RQ<7c2H@M&@I(<_CRlW`()A=!Q;eym=Mqw&;XemR(!;}1+glQnxu%`x51ewnr-bHIyyR@*FF8!)~Qy+$WE+W~zG?u=cP`$l?>mN0CAYs|Cj za*&dEkaS{H#`I&nYnmt>QJ&|Rli(&(4lyOWHJYr9DmV7K7)beNgV3?+SFmrI%+$D4 z2mwTCc-0M4Y#X)hulnmA9;KG*z5Xqr_m;#VaZxe8vHGA; z)~n6b&RAbb@JLZ5UKWO`Jn?&xMjgDm>FE&r=m@=D(S z0*5(e>&dVDtTUB^YX7wfpN|()-F=gy_C;NQRV`1&oSdMr?1CB-SSil8``%F4@7jA~ zszBOXrP+=7SXg7LQ0Q{M>F*D)b64+}86$!x^|9dQoAXgNZqC@cKM$`;+gEzpT|U*M z@9EWjPEisT&F&YXvi^)5PZ1a%k-6nAgW_iPj`NgefAacz3>t+`5eX_VmvgjJt8#z8 z$j1oDvI0j|5R}f?tmKT2Mp^`$TZWP=KKVseY57-WC)D312|R8z!zT}aB>!PjoSo9G z8~!r#5MgrBrh`G?Oo;H+7?tteqvfc(h3f}FLF48xOP)M=axb4ml;eMHK>kwp%dp`| zy84B@eH;fg?^H>rp@-zW5lZmE2q{zl`LefP{3c4JjoeV%??pSN{Bk z*kTixq>*j$g>`IRMdgF3BHg>e1UOKrcb#f$96F`Mf7{a424pw@vsQkKzmOw&l>bw) zf>e6B=3+~-Z;gL|Kc@V0PXFCv`m;%uB8B$eAqV+et2IkmA;Q(2w9#tO5)aOf)ONI# z{jF3@8R4zuHQIUNvh1QucTTAwB7@<0%RRGW=kx;u7ZqP$^oMCPG@%PGxY!wzi<|K&|U-L&nc zC-6qx_6CBlLX#K_)`^z87DqO%@tCqtj;laC?)cKUx&vj<)EBL5tGoog|#?Qoku3=w7bzs zUrh^c2X(i`z7H{b@%o{*_;-rw9%8qSb00-|ysjQA0`ds}CiA74f60D|-~4%#(+H$U ztKKD#wSU^J3(O7Pu~3}d>_%i|x!_28^Hl;IbOJ@i8;{>D-|*=D(EZRBVLHrxeitJnKzQ-XF^wF%5_p`_H z>N^Yb#5j9fX>M$C+~x^0t)38Jb*U>X`^{Ntxj-e#ABuY|Ig?b(&y9W=g~{mp}{&mCqRZ3>`hi$zGzC`qOs+Fwo()ZjowAnso?($36vaW8Yh#J2adbXX4|xr! z1b(pny@zxKlawSy*v+&NQLM#1>+|poBHu=auf$7C zS$?j`P1s!GeHrgi+KAwlclvem)dP=Kh`D>%O8uL(&9t^0=cp}UKy3m3Xl!UumUW=e zt0w(TrlC1;A_v&BiCKAiXnwz{=nAUH=pg&l~eH4V>Exv+FoH|FW4(<|oI% zoX*<2TS-s$h7=a_vitpD@b2=P0Sfg1e3Q$vAEY&zjZ)HS38pSd+ui$PKf`<0P<=CT z=d`BH1-M*Zfnr{9rKhJ4@u8-{A0}s_`DLKpMg?!lD{-JtLo(ht{E;&|w`3D9lyi(n zjRl4X-m%&f1>I7`V9tA64H;J|d76Zvv(m<<+>nwJ zfZRa}fP3^n_t0woT?W+rHqAh(nyqTTb#%d@n=G&Y9R0OtxWv(Q)@r1;@%X+tDkX41 zcuM*9(fE8QFK0V%G>^rU_u3Vbw9df5JH4t$3TWiL_}4tP0RuG^0F699g?&n};;yX+ z0!uK@8v3y;qAJ71xEMpKxafCi=9ne@GTXaSaMZ#SpF*4Gx75)%G!uG5`7s!N!|pqq zC@#bE-dMh9w2~~9g|$Xx$wxQqFCLmIzM>W{k*w4sYlnfaWMBW7$%f6!>oFB#?Y~@g z?JL=Tf*GkNK7H-;pjW6>68v1(pEk+4sluw^{L|ZK(3-}@&ZK-}l$?#3tigNM>01F# zERaq?TT$5K>r%-iy~;boacueE(p%w6 zF5Q#NzUSI<DcOk4)ileu{s$`p}!+mll{GAXiRVdY7fyGQcs@slh^wm zNe{A2oJ9MmCdQRU&1Y)ld~YswRpc<}Ri1c2_tdsYFNjbm*6=!GSpnA1k$2}cxkbgc z^;=HaL+fC|#tp&@PPw*V0=l<3w=qR^U zi?{pUHlM%I(mfOEZFnwTgX?HS6{q1m~s%?(4U?tAagb`Zwc5H5~ODr>O!USB- z=MhuGJ5Fz2epXwPx7Zhu*+;?hIXm4f9fM zif?-E0TklLPfgx?Db_OO&Dox@XUh58UP`<69^A;enR{hb_zppi!>plCMUxI4<%suH ze9~P4gSWm+7V~V;<@#%j*u;!j9}%Cscybshblm^0`nD=Ruaul7&LuE&qXjqSC6-NC z{WSB&`PBT6;fsJZZ*fo+Wo#GMF0wgS@Q!HT)Ktw6s4~Td8K4Tds!gE)s`lT4cwaY} z*-CFY`)8chpYJ-t!MD7|$r0SkzO*++bZ=!lRxK-V{JQ$aT8@vN2H?23QR64gf9e?D z?z;W}Ya$!mVjNxbj5C(slsIlD(|So+UqT$*QXYusL-WtxDN7F_W9`#&qS z-+oK!K0PJ_0v3<$NNsE)1y3yrSwcXLma$Ur*?*`V`0*1PlpnJAQ;a9n;FM(D#Qt-9 z3Am`hSi(ntW1DH_AS2&=l4xI4ZdhT7R@t8VJ7X?y$7MSCDk67-?*6)QqnhZ(74NZ% zE-RS^!9|}t^JmSo33CxGtvN;Fr`NtfvQ2nIerik=4c>OnH*(i#n^LU}IC?icS<$tY zL99F@^j=}rw`$hxxhy|gGSW;N#OleniqS9 z#t5jfq91YM2RTT?8a;h`IEaUjFr{j3gT)#DbSDGrx6JayeKQTz>NVaT);#gckkG2O0CX7htvk~R6wkIFf5i@s z0cz3Gskqi3SO1AQ#y2m(a=M!AnI0zcu(I^Q1k}ZaKVJvB>H(M%gr%yTRPCQkjKoe# zX~k|%4m&rrHP9~H`8~z@);4(%`{%R7xM?%8;lXPe_wallACi3Ny9TnOnQgMutM~n4 zg}*FZ9&p_%_Ksw~v^V0R36`?gg3VZErBjY5%q+NUNlF#SpaK@J-L7QgPu<*7)IgjoS#t-JM zqg!L`TKIl9$?~fd8b|{p$ToO3LScBgcz~Il(ZxL9hJ|iiej+(s15WH>n50vH$+|i^ z7!1DUNj^l>bc_&N{CA(KfJ^W`Y29jIxP@0U0B+P3J+dO6mI4kM=o6Q>5cFyp=CbrQ z3_i&9aemE9k_%WN&wsi5+&(=_6MDMkp)TEWSa(ysekraZqPI;x!~-A(kK{5WS{5k| zNz`@+@<(g&R3etWww^=#@BXPT7x)BIx>5J^LEYWIFlSSAC2j7+iL3)efBtYs<8haA zr9Q!d*wA4Xu~)5`riX=pm#b0f1^xcsqru}6yIY0>M&SwJ?sw1V7i751n*R~5!sMtD zcx;m<=k)(Pmj0Z?C=un`J21Y;NB2K$NLpMe)ef5NGUnsZY05P-csE@yW=BWxmwo(Q zS1WO;cLp}xF=NZ#^4{THhYOGYT4j+N^Y6l}!~T+tH=NHEDE0HN@ zW`3~OmayGXXz(z_0DoI5>zc~pkwy;Df7X8<2!EKUY$7xExe@wm^fRAQlbeP7^Bal5 z4y;t=`SpdfY)U`X`tKF{2aocf4IyjslLd7m2nN2fOrA6{WnB03e>l~UbIS(baFQpj z^PI`vn|88}9*7jLnre~Sx-X|99fC(s&$@k-N2zRSKB*!5XQ~%$zDJb4TFI&!N&Gt zZ=vn?6Mg56)wHp>Od_`43#LNSTnhCvt7sKh0TpGReed z!AH@{MaDy8;a=rVV$RRgNhRNmbBUK^dE5TfH~*=e5+i>e>J7fBy2Dv4*3)f+s^^a$ z+3z=y6Y7*AYC?CSW1dPL{dU_Rl_dwjjBNp@WSVMGQ2#t2hmpD8v3u&dlhkzE=ahSw zRD6_tU?}N`bu35E|HTouIUC}2AUc}{u{4|X6g=dR@ z+;GRHwG?zL^51rMcL#a0>HT3@jm{+$hv^X;~+mK6=}swrJ%*S`m- zc_m`iDV*oBCpZ6z;*E&#@ZkPv#U5Q2vsO&GW2lNZzPNIA=rhnm$*mg>z94@jKS<+7 zLboJJ?9Bhjea)@2lh;);Q%b`JI%Qk;^Ay<^+}t(nwd+yG<%ayR4>I~XGu_Nb3(2}n zb*FKr+(KTG-8Gk6N-Dn_tTe}Pb5+k}UrQe<)6LkmIrwX$%4O){v+^TJ_{f zo_1Xuwc7hpb|Ll|x#3bvF-&I0ds^RD zb5Awy=VKSo4UhL}`2v<=#^f};MFZo}H#F1+L6FEx!nkehTHY~T25DU-&EPM`vpuT$ zQN{{v(HxIMU!(ZQO9M#puzqUr!`Abw-nW8&^zM4t?%WEx<5@v`XlQDS zzNr7cvP3>FHTq_uFA}wsq`*`dr995$bR%>?*=4$4fxJi^y0n$XF*?ts_6kC90M z1uuWA`+@#Fe2N`KXY5J~4t^rLVUOQNS#itDn%Lv05@gmG*1$J~%$77*t+Pm3#=)Iw zRJn-{fvJmP+f*j6dBfs%OpA0bp%uN~`x@%cI2L=VGu#o8&9yW9?z%-9WRgTV(gjRw zgN_zcgZiH#n`CP~IDhs=NtXI$Rsvz4!=T62~E>4OQtiz@GRf zJsu#!K__=wGgti`^Beo-U}yQetm_UVUbuyYFB_3Uxl1z<;pJv@xQgY6YsR=GJWpt1 zTx+mZ7_U(U|9`{vlCV{@F0pGq94>Wnl)^bQ1hiX%peAgd2Yl!`i?xsFd1j3FuB#~5 zoztfqrDysT{2``1X$JVW6a*;KG&U*;l-jGo#&w_=Z(>5y#^ryR2Tnn z)p7X21N2OicqZp~L~icEPLA|THC6Rnz+oV$neZqD2OsGFhL{$Zt^e6S_I8m@@&c{= zZRT9h%}<{MRl!BWXRdgAT(7G4_ode<=id`YC0`0d7f{4Z&=ds0KXIsvIq}+xSyn2b$~mZs)`&KgMy=fyoJIbfTe$eoeEBi?K}b_xBYlkFUa0;b2-xMe)C^ z6I5v{2h!jfdR5|&id{252)X2AuYqi73i;)HK4@PL^*euGPFP23$1~?Cs9XhL@p9U$ zk&k%gZRP_l|G4#5+w={d3aTLC-6j2$(GB?Ngce#uiT;$CiMi1!n?|>-!LB{!y>Jh& zJwxPblkjUU2t!ABUSAHO zR?m8~K2$Kw)hp(q<=A)yJPmk2r%0vkK!p1w^coAtwy&9eJbu-R!4t{gyJEoy)WLb1mg2G3Kq) z<9)UJLieu^)!!R#5T&+ZP?aQ;hBojAA-s3sCb{>Y`YT$VFLbF>*jD-ggAYcm^Ir8& z&p$zwUNPu>J6_XH9l7`8-~i_-M!CI`gvAKe;pcX>&Rz8zc#xe@v$cABen*xcdV_-! zT+dgN9IkB72enlMBRfv{GCBH6sEXWHVISutYdIq zaLmj%eD(Y3J4}B22N}>Iz)4*p1g!8?h?^WN=KihPvvL2a-LrU`EKXAF&w0IgJ+VaL zsnyW7I1bJ;>KAh|Iaz63PrSj{-iz9?ALN&gf6)oa=N z=W~}5QIRIW>%LldkaMa{r4HPxSJ@uFUw-aCs%HDrt6v88U17r}K_e%^{i@yls)cXZ zf{;1XH_*D3P0e%%m>F++_Wua9bZvJqs^b!EsAd}iC7??KkGn-+rshuhQaqwg2;AJO zNQ8N6aj_`N_}w?VyCNl8EP8*`p15K&;_kWmZjz2|@0v-Gs&ExQM^1GIOYU2k(oJE; zlJu~Q-(>cG6YKCNp$bB6Ttd%y{Ya}?54a1nb!Aj@_QaL?kF&?ZR9fujD#bw4kc+z{|4aRmQY3OL7xJSLkdX_tsHbfqK;_eGnH@MGlTk!acTXE||DE~vpv{WbM zf#|n?87Zt902dH)X_fQrq26o*kkKH=UguiBV zp7-nh*$ry#@eT4CZ#%u}n3!*WjM(PKgTIVOWjWX7QRj_$7@`@>y9ckgM(?L z_589A67P0zsG3M|dBHjwT^0C6`n-B{i<#bAjC%PE-V}wCHs48q{R@OgV0?xP2@o2U zNE>8%5#uoSPF6_0qkfo1MB>L>4R|fcOC$wUD(!E`3?(77Yvf8lr;N-|9!si`UqJ_Hw)fs;uV581ToopvCWngD^C6S&?YDoWi)bEuKEo@X^zaM)|m zNz=urL1r%0Du_3b>$+*Ljzj~eCS>?}Km*D+Vc;n`w{Cu7qUVN4VYhBgGh{pAkQ)Yc z3mNP_zBhaIUTw)n(B^5xikI`iGR{0XM|M zOl|ige?W3yih>qyT^=gRs`-`_?RqoVySS@7&xP(i@32^!&{o6er@F>xPcSM?zLZWM z{O*eTf!nwNY_UW%aIyK`&`a;>m&@viCyxud<+tQZvjUwt^SgvL`lI!K@`%8%%&|@U zP_i?-ASv&MzWrK4CIXJqw}ferT`@ny?Vh4X@jOgwev#+qKzCAX@U~7tqvU=uWsvc` zz_($Kb6X>xJ-A#J!>XTTGk4d?jb8sm2lR=S?w_Td{pr8qa#5{8KVpFPN|I`*zONkF zU8f3df_w+Mg^GP+U4FRY*3}MOJvowX=B_86no#k(e_G5Y`D|nPpicj4&8KBP_ud<2 zuV60@5aE%LP4t6NxN&5*$qmXN{zzrVsmphm%j1^&Y@I`E?jQi1?Jq;o;C3*lY%sS+ zBgdX~^(utT*q`qF&f1oq%!^^ue6|0_)tiSy`S$PQW0}b^WZ#ov>^no0WY1u%GqPtH zLX|2PjMfTkgS+Yfuon%kaX5Uh_BB{RD^gPe!cO1Vt-haH0$$ig#UH5gpmh*bO z&J(xIa_zA{pgghCyHubJ8h{Pr8LwT>Y*p=QJgu35XQ*JVAfgU)J6>E%(ZW}kcZj=9 zzZC;bgCy{0m=>mNQ#lnk;`#!otvP7u68SD~6N zy8gS5D#R2tKlZBKVzyjLfJ^MlAkm5qnUA}O5rY0I@TrHR905W4DQ7~_GI)1qy{8PZ zh-A(vSnoP4-#+B^^rpj_y+7idSr8(3crncm`ZFu5LTuTSXl}&6GG_eou(_FBVx7;s z1yxUgx4E+iatHl&&87ceri0OuK_+OZ5Rd|_L-19Tfh7G17II~_jZ8q`H;K{XK&dl$ zT{d=Ku@TF}clcJ2Spt0MvCD*yG91Y0GZf|jFn z#G#CFAAQfxTf8TmRH0aQURhAiPZ!#Z?Yw$E(FLsN-ox{!Fqamr)na3@mW#|=893)S zUz7o!Tj8f3vq^Ft)l5t_MWND-s9DCx-19dMS?SS)=TI8!lG~FfPvj|49Mv9f8y)6$ z&)~D^XS)8>bW)btr(TiFziYy>x_!?3o8k$7HT1cx%X`bI-tN~migajxxeI`wEz|wQ zV?cfY+TpBnl~T(0*alhCdfaE&^+sx94Bo3y$2I^JP@ryTA<>9NOltqcwmU61zeiHT z+}91hr{UE(a<oEwU=(#goBQdFNx7+MGma_GOQK-YOg$mrGA~1a4>BA@!0ur2r!A?0-S>5SaTw zmFHBj0^QGO)S>B_m!~iuGq|u3htz`-*WmF9q~`e)7eO>3fo9D%&&cTq{3Z z9P7$(T942vETVaV{MAWUmnd(U)3V?NuisofZ+2w!(X_n*ZB-llCOMR)_7m29D~(z= zjQd?fo?`xa2yaZ~ohZ&uW{>U{HNXH|;@d44qH#x|3Vb7IZ{bqJ_ z`5@tz$(Wt!e-XRkw?!UR$zo%v)Jy9LA*Mn-vbuS#-7!&kFAe@Fsrj$ja@|LvUt86J zXnBIMPw8Jx+>#qbY+iBT7BL&oxIf!{^XYq`|&5J&`nkFysrgRa0#-QFr;=SWjG3iN=#pyhjdm^@saFdp+ z?6zPj=R0~#MqtXa#hiZ-U?llh2n|v~ zgi~<}%vz8B{Kz#XYMXEIi@CZvb}m>tH2px6$;-6gOk1tph(A+LvRX!E++c3MpD0!l z+2UV`_S9sITNibUeJvP3FT4+|dvh{TBIh3A!&?Tp>j9&_$ei-B1E^6-aS~-+L2X}% z^wp0I7>qR6c{ga1a>5&H^)==PE4=o!Jo?tOC)vNwOG(VRv%OV~D-v_PID>mWk1M*}&CG_c2VSDt0X zsXu+3KlU6N<~dFGzh@a>8Vlrz9k%jRkgI?QUl@H?ZE)=x`!lY#F?nj3?abGVU#S3@ z5R-{R47)(_{G$}I?|EU>Y+HdGc%S2MZ73qt=hu7E(OrRH#N{xJg{@ncrplv zR9Kkciyn8M*`Ut7Ut!3FQN8%c&)Hq4L!3?tB~l8U3yl;~QqXnR1109l$~cctuf(lqdxQD*7XbcO za!o&YO@g-Ff2udt?b$%Rzj;C)+<{}Ogon=GuGC3AE-IA0dAuDjcK>pd)dl!S`r7#t z0R{5f%TG)JwloVe4X~C>&^sc$OeSym(KV^FFLVL)4D?1oJsb_}CzTs8T^UKYBX(To zJ8NrdZjy4-!IX1A9+<9c)azXC>o({LLj_N`@X<@42@Y74%NqFIx`k@oa}j2?s)dJn z-qO2ooM{9BqkwTrgi6ebO8&7&Mr`&Ms!38X$GGC~**EiSi6ysKMn;+XLhxP11@b85 zy_mK81dy+rN+vx_MMmB}wtBZy7upI1oCYAmWES{kk`~m_1U2?Y0cI{!Du5`0Ak&5h z9pLZWq*?wPy3eFr6(aWCx~LQS8djHkV?Hpg0_Q?L6e@VdTwezJcnzi7lM~HC zG5(Wkz)gGT)ydp>Cf>Il5dp%!jw_elg6L9LG$@(VY-Tb>1C+Y`$^KJhiTOR)I}}L6 zltZTBcGWZrI%1nOoza@=)>3pWK(Zk}lx5>4=vJ#*WU={*pTHg6u#Pug zZaR{n0odlR!R{RwYx98VG8bb7estNq@V$!yR0@mg1bGMEWQ{HHVPddcJ=<30#pZJ% zT~%qSE|d^gXOdSu6=}>Gcp_Bdj`20WRfV_|-s_>X6{}aecKu+C4ZnlsXY~FFJh=^} zs2L}^Ii)vtBY z1m(6`QbZ^!qDu*f&CzO2bF}}hL?WgLK9I@pUWsB5Y0hV2byu*q2USJH$9xNncq^%) zYd>U$_N&JTbez+cHOPa-B>SN@b^zM!->0y?{KpYjtZ}vJbzO~o7`(<6m2N@^(X8^_ zuNg^y5~{6%le-hXRtJaJ`;m}`nP5dE|M#H2qJ&NhbJ{HDG{e*}whd1q!vtRr!UzMj z#qpp&5>I=BdR<1hN#KgX$Itoct*k_MU^s{si6f6W2$DoGtgZ}TlA6_@Habi3s?L=BYx=b~rP>gb!7c6D`a4*kAI{)Om z{P>g6{mdUXE9S|c3V>xazxz-~1c_di6MfW?ot0jcz_`A!F=+&+<}cM^o~-OXdS&L$SB;O~jGK04DhR*Ex?QffV~of7@Gv;=&d( z6I*nT^Sex+30`Tln7x%MbT^*|rU182TUko8kcYqRHNGj|LnS0270Y;1KVcJyJL!mGk@2M+qDsgq)jUST$p(;Zp%Rm7C;D*k)s*zc-t zFIxwg(ND`t)4W@wL8aq_4OqQgpg!Lkh&Q^Od0$4yTy$Wr3TU`jkqj!f5^T;pGaAK! zN3iq8N~-ct%C->qRE=q#guWo!nXCvO)-CJVpK*R4GqSf!V_yoS$@VzYws#_=c5a*6 z9FnDcA!xhN#*ml$)wu8U9xRlI?XW6i&K=^psDHCzydJc_di!LCBPZFN(d@xB1!t9PFgXp(>~KoZKz2>0fh<_Z4ewzKIzA?Eakniz|$sc39h>(Rw> zCU_c4O2(8;b@i|%6>4mbs;C3nz-G0lW};2)e^T){hg8`+86x3Bu2DCK#$v7Vgcy9A z+Arp+8yzGCh1rQQoXUnJPb<|~Ez~6p`Gl1}k$uLM^L)nRXb6+eOYQbiVf5vwg;NgW ztuX6B*JaF5+no$5Q7l~somKMWiOOlE!4~Zcj22x9u`)S}`}?~8P-o^=j4-?xgdlBj zu581#`XS#0YgxH{N4L5^JTf77(^Ruu>RA9!7ilxD6wew3tx#qe`LgS!hCn9Rh88nH^?O~ z`<1hI)@&YkNKSDX613L3@uutLSl5+I{g%MBlJslWT?}V43II7spnY|bn9=^}n^nP9 z^HOrAWG;Lex$Q5%+_Z&6{aP_^{yj_NUAcYt)_SWUwM)eLZRmgNdVcO1%3XS^;`t_# z4rYrN_1eLX;Huu;j_{;ngHPEE!6S7{Kx_Ax2EGLeT@hsUl{L4Ig++5zJGxr8-Q+4^ zArJR?VVG$UGR)c334`Qef%p{=J8k?@m5la_CnjNt9Y1mXcDPR(16`?mGApfyG_fv0 z9K=LTxu?#uQYrD%K8CQ1U-6YwR2N2N+861b`^=B5+?4d)%pnivo(X@z5`+*TN|v(8 zSzo(Z-?*Q_N-NU&n3x{%TEx8A=kI}K#-0rRy|*Wk6DzeSd=KFW^!dfu%h*VjLvi4fyNg`IeEgfp^Grn69<8t#$WaGKgui}Dm~4!JxhDB!f{jMA)#%~Wwm z@1&L`MRUKR=Yg2A5%s)BZ}53qt%+;#4Enk64K(Jijw9)$@7J!=P!Fjv7ku6{&^rGD zl!dTVm~R@@ywK*H?-sr>u-t>2+&6@Y`XBq5{@#_rX1>{p$*|zk06a(k!nR3&Th+A< zG$J29X4OC@uwy7^JTy^e6QhC`#}82=hk8I?{BUnKCoSAkMdf5DszQ#+TI@{wYZb(7 zWJvV-)rm7fNo9g%ZdMs0sRH?< zB~+k^$2od?zMLI5xgjO`=io@N4oS=5qXRVXqDBCB-GK@dyx-9z;fwX=VAWRVQj?)o zV514lx-h0WH`#N+s+?J8oVA7UHD-+%^GZnnX{zkg;Y0X(px#&y)iCbCj*5PslahDF@=reGa2g|f_!g9X6>y@da?H3= zMu1>aQuyDn2EZT#rCdCJ^((`#zWLj(FiwGDpS3^T0a*Si#Kz-SI%C{#IK5wrr|-#l zMWa8uk6$8_yQ{_(WP`zNMZj^x=)2c)jq!VCfrKQ> zajk^|)B_{=kmI%uPs9!Zapv=?VFh)|y4+LgM>TA6uNi_fm-i%YTFT%-i>0NwDQX}S z*NKld=*@%gcV-S#gf!@;UtaE9lK+Ild^*}}PCH4@hcji;zDi#&6HJ|F`EqSVYu$i~ z{{_Lkm|661F4{hR;YysksJ3uL3LrUGlmFk6WU`%0*Lom+zovyz#a;Z&C1u-UzIXC_ zZ7z>JSF}w&w}EQwdJFD}8x}c3R%$s37jIf$m&YGc%)iI?G-V1bSIUE!FJ|fw%x`hUbDh1IvAFD`u-%gYF9${0d!tS>5(ep82G-YeTbL7#}*RC8afwcUFQ2 z+_x8IZU*SbHvnv~0L3r5_dyVKfjh<+%>JRoyvu(SlCp z8|L=$QprKm$=OTS2-F{MUVW;z>ri5=d!}k${C~bLSTQa_OXJrjCH12xa#Ee3D}#?K zY&P-qwX`04=t#y}rzvA0VOS-7&GRyHv(fj}A&EzZW-G0SlE$xZ4%ltJ*m)T>eJJjo z`#y&qj=ybEn?}ScoC*tSvEbrqe1EsQTQ=ZVhK<=;4)9++1}-7vW`o+Y-M`^GZaTlG z#q7!Gj%M1)+o~&f#_PMHbTesS$_xdmu888$NK$CKGlaYmmz(1!ZMsXtPo3kxF~Wp&%Cp`gZ$8zv#sRXT8Tx; z)d)LFf$LfI)V-<6ogs(uhmt5CtVKbV+!wKWHqcm!3$l8{2=5pY1zJ$krQA>4g35d) z&fgbQ5dO9CmNNSWtqD#B|HEi)e>gg9FUT1ekofr*q>zI%^AR>`Msxh!pOlIO%xrSNHWQ7y! z0*&>&QS>#?+}=fWXLQ-U2z2M`LuQ;W7}}C7Rx;T_U+b{l*>dWbcG{3W$i2F{bN1y! zWl#d#NLLm%@$TUPW}wri%D4CLJL| z_ev_MHB6BJSon(Pm(F@szwWR9$3{-ZJ!@ibR{FXGyUJ%nKsI}EcJU3q7W~Zu7ZV@ zFR|Fn7%d2vdN!kwXseGco|jU3EqScCG+(~Zz5NdyUmGk52pd3+FW@+vq@7&XJTV2v z?JdC`QzoN325rj4J^?qJ?0;F%`-~1a@7F&-^*NpIbYR2N$~T}&S|nMPf7 zEB~CfeE2i{>MR5NX7XcO&ASV{5(3knXepFL(wA-5qst4l@dd?_MZkFdZe_yhx3*~ckogiPm{`*#K8O>Sl~F?wAerF>;D5jQ-tvS)8(JlkvRe-;Jn zl0%|f$TwcZA)G4LJik`m@{L_AB{mdf)9`py5S@SF4GBbHqd!_Bg);s^!~&C`9<_C* zyEVi$8N7Mv<5>;D;m1=1sUnwD8!_oGi54?34fD`4`Be?osNu<4Lz%VvcYM4<(D`U_ zP2p7}*#gbeX7$I)d97m+IGwsm>%xW1t5z;Ni<%&={u4LoSFZg~cIm16*XlpDM&~4Z z>QzzejmvDhmC3Q?=C_uK^FGu z#?+O|SrS>gTxa_fq;DyI?b5okY1?;iANt3x@2b#IS~4@vK|m7qo&=N%Zo-x*?UEy+BW;8?GI87Oju;^ixl3{f3$ZdQ9Acn zoWW?!vpBd&sio4LKKTK@N#gzsGNxzhfld8)?jNUTzI&9B@%BmNC>vh! zyb#Mn2nQ`~ir)LcH2(ePXbn|Ph~+TC!m&A(Lwrr-q>9ABxTRNCnZ@o{I8584t)S86TB75IB zKa2HG-pYT@sFfNdj8P=qXVd%UP}Qip_Kj2MP|>(Nvg|j)G16(zlX=cf=VpjQt?d*q z><`WT(S&dOysh!J7em@Jp2$lro4@;{&noh@gCZ75rpAQ;3mHuj`+3xYN%~89b;PFd z&Ml6qkDW@ZN=-&wl0$+`$jsa0eP`Xn_FUC*vu!0-OPkN67@CA9`<<>>j+~a&b6hDm zdr{mGp@>-u)!K0-bkyPcUO)lJ=NR zU@1RS{wUTVo?;~Q6Z?Ic*No~`rj8|~0ljP_51)o0)WJ%kqUVeSh7q0%w zZ+I|q*Y{c5!yfj)m(6osLC!x4z&`tw5%~S0)N6-fcUt5RnfMp<&(Ek)6BpgGVnb~I~@(bM19s}2|ISv8hUJ< zcJElg9unTH2Bvz699E4_?3EBKFqzjNldE1qjd-Y-hN+`&?V_OteaH{E_rsP;)fSFU zeQpy=<5)y8;Ek2p%a)$zv_m37F^)f>&sDz?eTCv_3wU_0P@}GqrRSNj!8&!2X=kaKJES`xqI$=!sm}N-*OA+6nN93e^M{! z6U)E+)};)0ry$eR5qGAKLOM?-gIyKh(zy^R_1hcDypmkpH>g8C9Ff6;Iyz5)-l{icUwQ?(tP#ksq@QiOU~n~MbV(tF4g=$rkF2Gu(H8n zRp$l*B@#4OXY0qi7G!5)lE#6IpZc(JS*T^1UvB4(WaX(v-Xu>!TdDk zM&Ahhuqh)Paf>`k$s}_S=1p6kx8gj*S#WHXko|Gf#!@2ns@^XF0db-}SPwok%Kk%7N2?P&xO!x6K0MQ3?}_jWRd zzm?Olv&SO-wr+30tdJkeu{)Es$;eLraXY|#x zi36(2{n__Bgj}P$lwKFuk%hU~ESAoUU_!>@Zw7t>Jh6u@#jl7yG}RiRqQ%Yf`VV^< zj_iE&*4C4Mxn&z*xLen^gukYz24@OVlTkp*$jD=pghu6@fbLae8x@7EOp@||x$q?pziSV#ZfCvf zUWQHIkUxMwDC|2xXxgQk#Z!OZ7vYHKe%7CTd7Sy`jRaq-CBqc?MeTT6jy|e%pYMtN zfBI78_HGOxyqhA2{n7HiEiCJzdn)?;e&3>J&aRxf*y2-(l!>ViB9N*C1{pkR2V`9RHY0ZpgTwEIWNb|$fU%AFMo-TH31AOLjBcSnKF5?P*ZKT;} zMlk(xKJL8s`#zY_!$pSHkNS5`X0UtCAgzwH11*^m*(f%(ZJ;=4GS z+7;Ie)V>@LzQ?4eRi5XG{jy_cubv%=L^rF+>usJmJF;Z>4?YZhoF67s;Candm_l2K zfNWR49MK5pb9fo^>Vqnj+3?t6xL6Ne&pG&beC0sJNF<|2;LJyAAwy;(|EV=$gZ1P` zyS72M!+mz6Oe|)yHR5=VTGFEFtFX5pmNABjl2Paf()PzNRDaa>{#HW_OqNr#35*RV zgG`YFvykpQe{}x^yPFK*cr|~ZCG+f+YQXjQRU+$ldh($5DLyKok6FOnjl!od{Y6=& zUxIMRVRzYFl>R+_Z-g6_$%qe>*9XdURxiX^#46RyV4W5HGRfhOKKmyMiGKZTozqrL zMwh8OwE9#+>3^z)p5N0u%VF`drpNR}w#9G}|E)(`7&zJI=cA$!9a4YwITjW@cI zK_DaX_l5w}K1$%h%j@5H1}PHKAo;<&9A&0vfah54X&UGlpRXc=5IIO|9nIv_-=IDv zA?W4-IAgI)i_EHuA1qDql)&b&k!%hKuReB#gF6?UFgIc+$^V4|q_`FwvIWkz94gR2 zAVl)escBs#$?Kv>zlOKmSFvofEE!-cfkDEz5~bk%tbN_2IE|JRr)4=0BtGL<61rRp z~9xN&iMj*Re!lA_6D z6p(Pe{!1k0T^5|l1ZY%r3kTk^nz$9j`vU!mji&?(ew?_H^z?1g4P1Y#BG}%l!E(7J zRgE@0GF3#Ph@Mvi^(KIgiSiA0+{st}MUwxX5QqE3ewX7o6{!X2+h#3y5!ZPnNf#Wy zfrX>GuNvv-gN86#G723*G6>f9W#%qad^uPgmgHeV;NmMP-Ypi5$vT4N&)gu!%!{SM{{|rS0)3rR#lpS{ zjLH}T%`CX@1lw{j{J;066w>tWrgEtm5OcKus$(pMaMhsGWUm%0jfQzz6>U=q4`ZD6+;_hV z%>R8qbKlz}&_p71zKylOGV_jT5nuu2<4%au|7mlhfhwxR_`Ll8Euh?r%)H*B!HXPB zK7y05cp!JynD&q@A@kJ(B#`;GH^$+ncOTP%mr8n=?T&4>GjR9t4))_RFx!{M_49_q zpMaaEC-kQFJ<|Br(8o*!A2$y0{QuU0TdvS(W1!izxn9p_5^FTnlJi@OZZtq`O6Z5C*STMFWJ1Z!79u$<2jJW4-m%&(Q@zk|$j z-s`+hz&qHB|K&Hq;=ZdqD1lnpKO6wlPj@_I>ODpNUC2yKWl;*+f02w&64<`xK2)`L zc7EU#d>m+JmQY@uA5hG1(0`9 zqK{!x!FHp-AZqynBqoe(=YkSxwR?2IE|dpbN%EGisSHm7G`lqLEQL`y7cB?ga-a$c z35O4^OYVM+u?T8;%L8j~nuKFb$~`opw=)S<~)itm7AX}v>AN{bAFR|O$IE~vX` zng{lAxceFVZVx@gErJy5=J%@AxH;yP3;`lYF&yqY5YXiz?Ly1a6;LZ|g2kADa|hf> zYGu&4+zb1RuRqAuFwxhCfwILAz50s=2}An{ zRQ^O2^5byo*8y@hx)^W~kl-0*dO@RoT@nDx&#(G(U~x}9xi16m%=yR%dP^T@S%fJM z3UGEkbmzbP+rW(pGVgHaU}a+GX5F?Ti5H1mZ(EW$eMlE%FO{iM<_;7QNfNRH@-_V0 zOpr28@>q|Tv?>P$H@ED(fZ93WHe=E5?^6Ox>t)UMZkSV=#TU;1-JIG87@1t+yTp>M zj`d%vF|>(}&}AE0AQs3+OF0=CD62dD-J+z-V#@=U?wr$txpSASC>9u8=V)nG?Kyv- zcwX3ES4wPo7=HGXIG8f^jf+`Mdl7)E*3e%$iXH})FCgPf21|wc#IBgv9mpL9bpRX) zs16ZDHrip3A-N?Bv)n_2d^k2Pq7hk5OgtL^(G{SC&m$imN&v&EM63W)3#g(NQar6$ zbSH$4?ZEWXc>8k%>fp4CLF$Lg9FSOyItQ78l4&X~0^-(%cSn~$9SNdKzCuk^qR6gG zLLfQGs#QVXWLHhfzsA_VmN%+NBza@7WE81`IUFqsK({?pU5JiT6PV^k{o%ws8tH`! zB25fkV#$!4>RU3td@zKZ&l(v^9mH?H2z_7(ZcI_c5jp9vQJ_XUyjIP2uu)<#7TF(G zC5CY_#zB8JlJRws#;uARK48J&e&)Jj>9&5c9D$BDBZ{o45AG0%ZsZXYvn&OW&|T-9F@`Yv$>o~jk@B$j;c<@wK; zsR7|~V#)DyomLj@hFX!;>CczvlKP(~a*)gkTp&zBpGo?Oz!_%_tnFM>7*yN9HSMDd zg<3LcIquS$__-y@3XKxhf1+elj%>{M5UMn?fT=w;{+Am432AbGRY2KJqqGxNt@=qG z^%8i>XwZ2~)ElkamRq}TNg#w(^8x2ML^)g%vt)vH%tHQ&kWNv+IE|7O34x>-bKIvC zW|rkVeh~PH++RB0;h!7Br?eTlWTi?!nzIA(=q@MNk6>Gk?%p&}fl)oz=Wu#;1{+hPp+3>RHZqF6wZ2V8yexc~{_f_-w(_ z;B0WR8mb?;%0KB}y~L%~_+`=nCInB`z5=epF%7btG#3%Y{?^iW@F2aL2nb4&8@mW# z5mgjC0cZ|Ld|RU@whyCthy4X{&NIE?-?%|Aw3OdHbJpPeOa`NM+gCF=KL*EGzYFIf z?oy5Q!jS`}%fz8_FFMDDZ>8ieZ0CG{bXM9s1EFj^I zR2)PKRVl@)Um&oka8quLI5Y~QwTkKyli0;{f(Ug-21#rz63WG7Lj#IBa$pZ!% zk`W$We6dJ`kAqMK=ogi8w+9lz?p*JvTUH8rZriay3o&lm<&7=6t4_NV{WcJZn+=>@ z|G!@hA9pB9@SExS+`&6$<7I1P!lU+(($KDj>>CaiT6oU!jAOBo1_3Ei{8Pf*FaWRHS4@+g~W5{(|q<- z^dc%ndED5^oHsH;K${Xs$GvUYsUm8hTLPLs&y5(#|B9lSNH(df2jH>CMpg`4Ppf0q zSJ~myCI^Bgm_XblDOWUBn$2|I`zHE z*==SO^I@se32e>rg9)HjY>YVOK=O*o3`7%ixy~n%h$N?1=67cJo{9zkZ-Af@g6l(> z^Po3LVtrIbVnMOg`-NV*A(^x7uandmN*D37tgjsa7Exk;M3?-!BrJB~iM+`S|d zKu&#Ev;ioR01IAUjpz24j_!4sMg^fU1?H@z1o__^3k<%UC6lSll^m;)rdy^v=Z~Ak zDg;39U&*>kyXme1QS&FDTs|w~{B74i(9ZOCUy1mMbqnF6P^$pLzzj7S*wUbibDjr+ zsVm6q1p-qtKvK(&YS0_-@GF*m2ZEQ4b~c7cZ4{7`V)yu%2@T{B zd`2S-o_WCndOn5KkjI9_PgR7dH~fNb8U<9b{Avg+&oWkjsrrLG)%}s7k=Ko=_mmwq z4d;xl-p(ZwQBLZoUwe=EAo#WnGKwq9wO_nU6$8nN-gG+ZAAadq} z@Q525?BZni@8B}}K*nwJl`fI3?<-VRA&%%C>!6gf(*c0fG~F-vM}QE6Wu=s?Bd*tazLA&XEc_HZg6njMHwUstm7^z zq5j}CZ|5#QT#~r4W$k6l9;6I*9dF?h4~zALW^t-ptiiNQ(pwY^Q~JJIt}cU2twndL zS8XzKyCWthrp5CbhKNBEfGYg(aB;4qQqTu8^i@HqGpA#*#h0_%YnB!_)0{2cW0b^ayKg}TN7V~pdcql5i@ zu&>IY3>uUdYGy7IbLs^D-@ z(0Hes_4_%V=iecG@`8ZK=>si}Q_`!&^I4sEN=?LN!g0x7adXDiObsf_ETFW)Gw1Cw z+F0$YB1s2>RJl}bXc+tS0QMIi`ZFJB-6w{NkaXY-P|j>0J)Y%^3pI;(wsf?VeaqeB zYPIDs(Z5MISykaiEa{ty81<>`HeRAaW!!^7bTb0$&X+Y*9#7~9?eio=*`|I3Q? z>L98ykp(Qcjk=&580hQNsH0vNe4ZM$Tk_Yz;k<1#U$oS1Pkoo%@}d&b85C2yPu;jQ zB?T}K{i>z`c9Fk~p;h~SXcpxvcMll?QS5Ly>6;Q{h;8^T$CO3y{R*4P?fIJXTo}-W zObkAq+v+GF4s@^wZs8I+kCAKx9KZYODIk1|MiM=d223HZDt9M|Qz0;}gfchOjU+wU zBH;KMb-cS%Bi3U!@Ne}`e;MF#7k^bMJpTqHrHC}MXA@T`&MKDj-$E^t?QuiYOb<6l zuDH16sOkbSN9V1UOdO83v3h_UmdGX(0q&){e-ZAca@F821(BJ{c=Q(?5?#{s6v#at zWl5ObX)h6RpGcQ7Y3}AI9U>!z!b|ZnQCciwRK6UpUn2ohpNv`BI8gjB2-J>+(6LDM zr#;&;Pxu18c(hGy#t&Ri@+;g_tc-E9-aLCH*%nK55QvuJE50Aoi9c&a?TYxvLx>#X z_++DNV2Xg3lK+Fz`Q&~G1woqshoZ>R9(InxRsYk|O8U;j|72YlJKB{w}fMlE{kIm(e zrwlPa#Z_)}eC?ojHpa0Ca^X0guwUj|JaJD&p}M(`=pm^Nc*`tg&YI*yFtIL5JTx&c zV|xy&rj*K=pzi=`qLu}`T+{CR#l(^-V#T)9m|Ml3Zf?b-#Em(BzYmR`nMQM8Uu>mc3^F`OOkXHZPzCt)} zESK1EOM3OSd|ANsUDDQAOZimSnP{b#`9VZ?7Ya|KfMD-*Mn_r{SQ3z17mZnRgn~`# z+)oV3X>KJliT_CgfG9Ddi;jco7oBk&LVV&vVTs7;mqOXKAYM!?&|0gt*CUHs4>}c= zEIVQVVLPOsq9-#0bVl!qjItn&3AoH))~Z9_DPAi`xJNWt93oOy3>3gLY>ea(iP%q= zX_Pb{n(4Y4=#$3YR>8(=Mk!VcM+^v66Peg0clUHjdCLOAzN+|mj(66s=(g6UBb>sI zn5&cI0IAf$6;?Y^fRrsX1WK;|>#{ymY zLuut+gqg2`on2)bG9d4qn=*!t*M>JVgT>1kTfBV{3g5gX2SPl}vlJO>T|3~b4o3Po z&I7mL>nQhXqPu05#8plwl-RI$Uf*SVbASs}m(9+zXq({1Wc(6oP?!sF+o~CmCm71J zP0f0MBU^B`kT}^|S^-vtgY$s<%zr3o~TalqCuhd1`TJm znCMbVIPV9}3wyz?LD8L%EC`4(QJBh$w>IuT!?$S2z_VQjB|?uX)CJO5v?u-bIij>e zX{E;2OFcz&)$#!RloCgi)L>uNq;~+ALP6kNY^=lr^v^3F_JBZKhJ=pKa+AYE*lY~V z=z4+H>QRowXfE4RiF|UHDiazf*^xJ}Zg6k{-&Cc7g!?cFiyV+=6N!L*NLEvX<@GRi zT(5F&g{hH;j3HW^v?iUXIEpAc@+!73mTQzS>UG?b9Uv*z&w2BVHsWxr2rV1Y3w;1N zoDF>2y*&$h2>fIof@ivL;VMpnSDPcYSVU~9l3Y!JG+TRAJqI=e4<$ykoPn&85{#A{ zHs>z^v}Z9{tS6Bx3MAN8?FZ8`rFq`@1RXc`4xpRPOTWIdwg&#&V?nbhZgix8WTk3= zK}L-l)%&1>RwU31OsFka!irJ=%6;=Tsi0S1pVDgi(kunVLV2M+W~H&8r$x z!&~nrR(pU<01B?TXEE(iNpZYo%l1X!_+150hB-=SfUmWTxc7Au{emggS&)a`Oc9wo z55&<}Lo3jHxdRVj_z= z?paqdoXu_n`e-unkmCx*!mRFRk6a@<*h$M2GY1H=_Cg=Xx>&Y&S|e}Kqk2)UkRcE6 z`P5^$3-NmOw$-g$x0HAk;qonp(^Xd@dRti`Gn60>b3N_Y?r4fX59 zT+oA4{KtMIny}r`Q+{Dg#$e0ZK1no@DM5>$G4syg^n1WVPN4~=IOpnV$>MY?7N-D@ znsb`=kJ_(_)e!236N!_4&=X?Gnf)O|avTN3xSC)PoP|t~YGz?!k+}Z6^U0G`Sj9D7 zZ4)qTt7g}}}Ti)G<%XPnDmWM)~^q_}LB$-wxh;s4g+4gG=NQR%A-3Kv|f3Z+U;ZK@w!~wA_2~ z44!9n7NTIebR$Dz6gxRj^5tN0dMSbImo6%j3RnlRErWB6o%#{uc-`q@@UCKzngmy5 zun3DOPT|6Zxr5FT)rDnFF%+hhRx$I$&zR=V(@K%V^F>Q=?4O)lrsp#}*~hKH=J$q= zHWKK?Gy#&y$-jAqrGCSM1Kfo^PP4r|sJ&fzW~a!Fiw-yCpYPs0d9^Swh!%j5SSJu= zPI4lQ2m%G`xG&1r-!qtI8M`tQd4JO^g-t&HwhiJ7sqcywSws;arJ3Rc6G#L4dXl3$ z-}xa!Ix~}=94O>4JkEE2-^bVYKqN}6pW5$3G#k?|k*Qf9pplvi(3m!31@_qG0+uOp z?R#C|Ade1B0LRhJ+TNv;e8urP)ERzTFV-)g^}aGW4gjy0P2K&!CH_!N0}VF?V2Sh1 zDSeA^VlUO^w7e`SA?~?IG72^RL{3+`B7mqW0&MOCM)%y^zgNa%A;5OLMN+!ENvv2E z9Q?D>-CMnqP_bETJR}1@5i!GME>ZGIR=L;yAg>UIsEuVH?Tc=9qXP4BKU2&4p7Am7 z=2oy!JmHHj3Pi-S`OPVu1A*bEM#r2WC)N)X3;%2sg70hdmv z@Foqwh-cXyAEN;e!nfS)yb)PO!ooM`}ulvXLqE8eu{wEk1|)FOW*75 zF$YdT%3pPYc5=X}(ZFIoGn@K#84}Kwd!cXuKHu7!$WL@I?%+aM4&Hz*V!#1 z<2;2Fx#&&xG9)Q_)6&wC5=dq6_p2)_6AB(l3}Yy7G7uv83!(}Bz-4}Dgby}&E)=DP zx0LbdO}b`Wr9CczhNGB*IZzlg(QU@>DY-5~C_11e)ozh%;S&KICS=Co|DT}(^usdi z+pnOYpqbuo85z;^-GcK4Cs~dVAQ>@P<*q78^8sIx;p9jVND=lPwdgv|J5 zyilKZRCq^{W8Jcj?}s-V$Ty1Ii&9!dxV>|Qgu6?VYO?LBMqDF6IZ&OZldfq+@zm=K zt;{UZCw@GGD9oU@&D^p>LEyQRC7Oi*FK2qll(_E4B4a^L+qvoz`R}C2HW$%bVS47P;fS-ckg6QQS+w5r3YJK( zvpbomY#M3_+Z^>A^zK8T)fFz-&Ur3K87jP#uSrbbOuI-85i_@Wefb1^z_76_Qq`;D z9jWe=YA{Wu*R+X>L*x$v&|H>qBvU4PT@}-AIM>u$K;zya3eL#7peNrqJ8mvy5v{O{ z!!qze@2Y!+AMHb03><&h0R#-O_MX{nch7W;bRMiD&l#wJq(Oi;iZi$cvYlcj`=M@m zML#*bDpz=!2mpxvei&VURj~f9Wl<@`^HOewa>`APlfNwNZZ&Q^XQe_@( zmEI41CqjpcIn$7;K1~<3c46?%&D?<+sj^>u029#um)u6^-=(mBFmVLba-W~y&ZHrS zjgzb$U2g!8H@BTrP#JHMNYD`b0bfYDM*k@}_@O;V1@K|JZZUroz&9yF4os|O)P|jCgO8<|w zzmAHk4gZJHK_o=N0Hg#JkS;-x25FFH=n#-@VdxShC8QCgdngG32N*<3x}_Nb=@@G0 zI5*Gt_r_Uo{BhRV>sdZ79f!U5UDqeB>tX_&NWoWg12a(WJrExYdy8yCBO3<_1zD_` zSzdUc<7uK4%L#eC?mdp*)k+h$Zk|NVZ+BiraZnceD9HKgpXPf6U%;e#eG`s=fP|_} znm*1ND@zFip}B=S$<~w_2Zx(2(i2e7gb7j7z_I&V0&I9e%TW^S5UlK=z1Im%vealp zZsmkroDJjO-8T{2j~_o=SdzFwFKLO7fmlZb*YBeZK;vaTst8%BvEIL4V&+JO?k4f4 z%%t0q7CADhxI_?(xRdq!|D-iomUw0ElCo0Z#!~)IDA~5){E9|JsTU9zIJa<%N%`DT zaqkMp0Mbcei2lCv)=rVwolHGw*;D0nNpMz|?B?2=ec`8~Rd5L!%6gk>`8)qwnVyt@ zQkFfK)%yVi;tuQ+S_D@a<(s%<#5$n3v>Oy^bMN6@Au4uB?%7>e> z&y1rgc-+AdgwDlB)JweDPQ-v&}QUf1&mow|II}d*4ET_ zt8(~;(+`}Ro(arxLLn_?-*-k)`kWemmRU3Sg6v??&!z}OP=Q06y|=Dko z8ZNe=@aBWtfp+V46iRKLo zA0MLSv-qXCfu>I?`|6|a3#hWDRF0a8ipou}2HJ~5$Jfq6PFz0T7@4*J8cJeZ7pP)T zhEj9A{je(_u zj>R|21x(J*VDBuJz97v)5f(ClZTbw!r4W@HVm&s*XEZ_I=w1Sc?@WLH6h`d^m(VP* zAu@pkrUee#7gpclgG#EuV=a11fs2Ey)glPs#BIqqY%*(bRuK}OtQ8(Ueh87^mjpAF zbH^vGkbw%<97Nx_n+S*WWDA=#gk6gM3!4Zu0z|eI)Qy~LuZSNoV~z2dkpP!dWG);I z{sW(?2m^u!p*ny5|Aecc0WH{G+dv3A#uvRJ#k6e-0n_d^_#q)AQf9x#tc*25M|y#z9G^wy>ZKs0m?wDq`|GUOu7@Qr%uW`Y&#QbGtJkbr=Ixs-G?$} zIDEt&um4z7Kw>G4!d$+d;B#}ki?oY*WbwaSJcPk;&LKl9qgbdF2!t!xpS%WMJY9T> zC!(HTpNv_3KE04`<7Os&8&v`?fGhGTUJ@ib*wr}*Vo^0oKcuzxf&A7;l!-2+DTcH5 z5?Dx$@I?#J+T_+$*oB&|A)W#W1!h##RS2&Ktk5#R|05DxR)5=YPhip$Fh7)4UlFU* zur&iYn=U-JIL&gVZUd;M1^{U$Gb+3=C+|-HjS0MTu&Rlp)k!;P02zRjkUsubBSNeqLelxxL?Cf;egL};W+3d_RGO`RBq!HF|Dkz{J zAK%2ib?i{}+-i2YJayDDa2qH`#3=ROhEOV9*cA+7DTcwjhDdRuU*jw1mE6T8lFjwU zHMW-=<>PtDd*e%p3<|`8>yMlfy6MFU;l0z-hKATS=qDF!HiAHF{F%|jv-|zv1gYjz z)=3O!SBm0E`zbMZ_u^l0q2!Vw7O=1l-@0to=#y|wTbE-dCFrDI56z(tr6lO+sU2>| zg%8WNZ`jM{5feGe1}q;C$AA@70%u)OAp_op>{~aQ6J3Li;OOEV zSNMt7(}&?L-Y$x@6`RA>W88$sP9J#4$0FTXFs%IvM}S|DP2Y?5ioo6b@q&XNxA>@? z#69zOxRrdKjZz`yrUZV-n>Ox_4+?-{iNZOH#eKrK1END#!-zo1i`yRHtgK~qj9z*b zYWgIE-`VX&jt!M+ZjY6Wu!_qD^!c0g!h63CM5Z&p#fKDhoc@8Rc|8#C_9C_r>i|)p zUa}UHDCBh_`#ATTo?zet4FyMo&C~S;daw{g0Fgl|lQYV$mkcwi406{^mRWUmQP|72 ze1P{GCoXUUAtZCRn%y9I5kU)M`Sx~z&qBM(i439g{#`!U7cmwhOVL)h(F1hnU6P%Qq7SU=1NWF_2`!8A2Bwa^&^ z96l3Mk<2G@5G?RLCj%W?Ve4=?A+8m2w=FgpyaoqoQOEvjgV7cZj=g*89$xA7?} zCQ4J@Z$B(KoB&!C^rmvw*%RU$3jqi=MxJ~IGOg#d+@ma3%=(j0LlD8i1LpWd3q7cj z(S&sq4G}a_n$M^nW^|)kZsk4$RzdGa>ZqV4W)TX@*K42lQ9nmb9!jRCJq7{p5QtXQ4X+aQ%CE2{o(W3k6O{A{KPjN{Z#v|a zn0S0B&+ZEcQd8My00Y)->n1O}s@p`s;Ym^Cfcjiph`?E*_=Ot?|CNVxwgg^3U-W;W z;n#!&YT__bnPr|}Q?)x*;Amv6kw_u&Yz}dwBLYl8{Qr<2MVcu-$5k#Kff9W`&cTe7StA(s$?|m!?B!%Ja zJ+8%SR?scT^O1yxuP>O)NIIV`&{$pSF=3nY+@!Bx+B?cSZz&KpqB@J1Ix_)Qk4O50U(Kz?>sgStS9RVzz_ZP$3+x(>iOO?sfM!ya#gE9mU&(P2#Nw zO&P)-8WFSbg>$5Z9umE}ybMgUZ9f^9p@y|j02$sG`Rw5IPV21VOR90@(HPl08JYXa zpQN7=(78!B-}7Y>u>^2KO;)7Jm}Y~A+uA)yazV1}-z_p205p8cX+Pfr!E`O2;lTQa z43=1tf58wjkPCkSx})g|(%~3jh=3(EjA8FIF)|l4MqI+9;&1>lgMQ4C^fPtkQTJ^P zFXUVAtOLOzk!Ik=3#?wsnrsS_?8|^}B1|a&Is}8)6r%Q<@a)nJ5Jvjk%2K0lnJ=Kk ze?LJWPaZO49omCbaDfortwH7wJAluUSh*W23?6qmQ5}S3G%Py$ z3Ud1Ybs8qTxO)q{J=M8}Dp`$gefvY5=e-2|Q1U|d($E|r=`1*>zAv%`;3V-?l7}So zqKRD^F*vE+xx5~S@@1aoH)E-Z4z1%qoL2&Z8yu)yOQcLjH$f9enuDt5SCGlM81;d>urF&lZi#DR>Ai3FSVm&I0DPjT^d<%ER{o}A=%%)H+ z>zX^D`EW(GUtb1|3uoo3Zp&*Gf+n{Wa9i%rC9lZ- zv(&u-K`s0#!hYS0HY0IYQW^k{IuP>S8A{}2vDMS&vyF%gwvd62SF=+|-&d-XPN39P zD14zS`8I;6rrh@-!5D73{*heiOZE|u%#H5m;NNn0gYyk?9*juz|XxK zhQ}~wA}QQe%uvjK@}tp{pYDx|b}%PS#{2xV!d9i1qft0**@q<{A~X9fDTv#C)*m4b zjtSl*rnlMFrhv~__OwSK{U*u{bYk#sB^r%r-UHG&6!Y*^OdfM1^W%ItTt=pfozo1B z#bv$bMQbjWMJ)LNFH62ZE4*OGUBVD+J^nCXWG_~xxJld0_B}TR%byPtl7K1VEB`_c zdNSQ?cB!GXj8Iq?cy2N*SBmWCi>;7B$iuI+EO2;31>8oc5-BkQEV}T>3hHFCWfpA~ ztzD6<4gTHNL@wg%3QHR)^zQ@VPmd_ReZ14Z*k?}5y1Qs+CQDC129O*^Hz25@tB`ZZ zE#!$Eyszu_i>HHtFnd=C>;}M@!Zy!tQjaa6KQxQWW0DU%J_!#pW|bb~-7Xt0hTC{E zzcWad!OY$GR8)&6B@&*Qk>zzfBldMp{D>g>Ongz6A@@6qr>gi?1z|RS%Jmi)jdS*NqOI@t2$t$V~T(`Q#PkJLOW|C#dsO{l{ zz-r5ZXV~Q;qS~;HkW^dX?kp~qBqY1XD={A4*vIAmyqye+UtI_{YqnG#wcdEG=J{cS zhD9oH~}qK7FIHgB1xl>6zrES76x||5O|Z`rDS!-UwkO6IYI<_LG+Rvlz*0 zUF9F7JDJ_#s42t0VXGFL+=z#q@KW*V3l@(Z`B^aDudm6V|sHVDH9Atc~z$Cpe!z4z%&G1iSGw|Z_K zm(YSTO8Q|QkNuqyPDkzs<@pKJ1k@+*f;BX0PeJ?u-+!LyGnzMCoJ*>{)%tOSsG>9+ zR(K~Dr}#yDpvoqr!eY{xXHOnD)k9u4J)`coT=_ovX7p`Y!#~#%LM0dRrN!GHL9jk` zSCmj~i9s}WAunEf@r?!YFQURm7Oe2x-0(sfH=|F|e9&mP?ba_=5N0w_o@WJ!kYhB! z;OwsYCbFk;eB^|&OZ{c^i{g!#rG^R+Hm&{;;fZJO1v2jUc|!s1%jgw}PvQ?Zdw8B| zOJ1m9GiJthWmMP1ukP^O3m+6*!47{XSg}}bmS)NPPGcGHEqu|IB3=7WhVkhmId!|e z@XaJ{0g+CU5tq<~kqT0+H7oJl)HUFXSddMC)%z zu8n$wo-fz+msHa`wn6#BuNND#-;Jh`DVQ-%>=K9SmMI0t-RvQ10zr~~cW*4_15DhX zmSQ}q0Z9!RV+LUeb2I~k`SRS|)(aIV)DMJ2ZTGP6P@yHenL(8rSy-0~vwpS$5fXji zDOrXFej~ErB>|gh?726`Y*M-hi6Jt|qV_R1SM*7Hv|QNQpC-Hdh!F+USD(N`4FL zg0-J6!2jT>_48Acy76-;V9}05OaO%!oT-&;kr-al?OCg~Sl04R9w^6^dMd7IA-TO( z`LeB-tc*&_I1_Fe$x*t`pwI%kL@X9geTib=@gB_%!+KKC@alv18FKJ|g{N~Zzu(17 z(5m=3-?%MMsp~SbW6~XFs)_@j;SC;mm9M>ZtOZ!5-^=-{Zc4~a;QyDsT=sszoQqVT zVkl8nu>{dZDGLEqCVbp*iF!Av9|)=NPI%p3x@7H70nUojdlQ7x#9%}fy>5ku5>Y!& z*Ots`l2Qpc8Y1y-nOiIt(eH1G*vp*vYse{!3(hp%S(IF7@i;MD3(=HNn}EWQ`H_k?57Qlut6 zSPCem=_xc}+WN^*EiFgw?trG(4s)8AL7){(D0?$8%`Ql0@_w0nf zy&EQe4n;S&*Gn7nYs4^q!aKQTG|c7o!6cp`Vo*1mN1iV*T!U5}eT!@4&1noUNk11D zl{#V0gZt_M2vQ2F5)`}nM#AB&6{RC;j6l`fd&?yBGAg6jCQbH$#`B2bMrdC!&w2og zW1c*mW9P&?iAl|ks%Xl`=1`Kf**HW>k4L3ygM`c4z69ck-~d1a;qAc|Ax|@(C(Sb? z(kHFZ4sE?tjAJMi8Q~Mfe_+-f6ExLZWmIquI>!at1DY^qF<`z3wi_Zy+OHVUh$RsFp^b@!)(!S#0ZUSy!ZJi;< z4I2X)>dtattF|wNo6P_bA!ca}0od@*9^(g1ligx6cgK_@*le(kpQXL$Q;);j_~g9g z)l`T55SL}oZlS6E>0G5Yxsnhv{P^i@^ADb)w`Fu$*di70zwX8;e(sRZF~(7dFOGRd zH$~b`?R!*1p0Bbf@F{_1FHvsYzXS|G`jeXhAmvu8l~|1d$%HEG)P=phJg;I_I|%BN zXf(BaBUJ?ryoT4_P8IDbnQ>0M9ln3WwYM9%YO^e8aGyg9-=C8<#gX}yo=&@~2Rz~D z>eKL-6V^4^BSEi4vEe!EJ3_SP(zS{gsbp-bcYNb`+WUCs<&nrwwx z%(#F$?p~2hAStEKv*MM@QvZoFJplo8WgLiB;5Zgz!1@HeTS$Fd6jyObCL5>dyeaT) z1Tj%gV)}z93s%mepTZ46dnzY^fG-wRyD@`dWPD2{F+X|d!=aPyQ}`|**>s4(@;QTW zE>N@WjivOPY-adF?%c3HQmZL9{P)%?;=8YJ4IaFkdU(uo2d8}WgcV8>!L;X_MgXJ2 zA0_nkvoIRfGMZrF1`IotEaya@z5jmyo6m}U1xLZF?8GV`t=Nsn7R3+ zlTGGj0mk_b4@Sr5?RPpNqSiV{Uf+Tz-1~H!P-?Z;%)wHW{CWE`yp6DIPJFR!X3n?F zOLib@S&jIAnBA)3|HGoD63UGve-h*A&B`!;Xr!Y}+HoKvv-dK3LL+-OFsp9lyMqk$ zrHndE^bbyJO|^#SuaC?jcb-O4Fr0`Tgdm9GwVFPhI3p!zO9J%xV`e0LJx5YPxEMmQ zT#glg$!F$M7fnmVGq}`2?nYXE1NA%UV4ZZFpe6$;H@9B$`eD|Fe@cfOTEy!AJGNy%1FO&@rCSV+v5)&Eb_aH7CHlQ6Y}2z0>S zNMmXd>Oce<9j&q;HR9#y&$^WjVry$YOHl2jZOG=sbCq+XYVhBs+*?oe!z&+lDRz?i z-+F%damWj~>LpSuU#A>_abeRdxg=aH5j2Src8Q&ln0>ypAu{+l(QQeKZ8;m_`YmkH z#AH_~F<;l1U2N`)94=q~$H8~uCQ`8)q|gT|4Bz4xfZtexESxT0S{(Sk|2IxKF)0M9 z@_@`+RuN^IW~rh=+`rK#@CHdoYL{MV zNu)*c@AA5mf1Txhs=}=Ibl~_C9=n2BtBv>VEVI^$gJEUk_>HkQzV0sAV)TZyQY>^h z7i9NTratX;BcsT&h7OWk44#WAf)NnoF@M9}#?Beu_Bkk&Od$4*(zAT|t>nWOS^F1o z*2=HI1R%(~@%5xNd=S_(i>Yp`a60Dy%XlZ?sOxI+IjCAYt?+3u9whFg>G8$bm%*>( zD!Py1M+G3e49TJG13a$ALmavf8&jU$67pM}*k_E0I8AHsaWUI#XlH&F&Pidi^5Hg$ zo%1U`?OJd3V5+Jpt%zl|n}(T=)X0RKNN=q0`f3gSgZlfb$6jxrY<*yy(;y`LNEE5` zYBVi=O69aHbg`^ZgaZE=ma$V#0~PkNM&N3inP*8!zIK(4BM(o9Az z$l!mvch~ZS^Y+{Bt9#4}#4xU*AF!bXiI+5Hn4FSK>`JtRX8eV2jl`nB zF&1BCVu_B^G5thW#*+fGaeMWwYUh?vj!vSk#IJLrAkp0%wLP*2589~p8f@t+BF;ay z<2?uRzz1L}EUEs6W#S1@=m;S=6p)|z=2c?({K6qpK=bn7UElY?n?hz8>)BktmXh*q z@E?+3GpFGTu|?9~o-b10pI`hd8O3QhL8e(1(TZ(OAkIN<^^zezWIpf-#b?FL(JqxW z%B4qJ%O8o_@1B|ws?mstKgRzhb^7EP!JWtn{QDt2RUb*lwZwbhgY8rTsz}M*-day8 z;SJqlF;_c|4kKd3<@(P9l{<9W)F=F`{h=T7uh$E4bPCXsMfJ1a-HB&VdcYwsd3M{I zJx_t4>|HCKo@Ir>AJW`kWXFr^1dIqm(h%(-n z@Zaa~Dwaq{J-*Vxy-DUyH3sJ_xL81db#nzK-e4JUNhnh2r4MUC7sh?DSU;Pttvdod z1$l=DN~QeIUjr<6b3I|aIjO<*`2onvPVHvHj-yzxmn!KYjWC^%4I6^fk82+h@UEN~}(+;BKdKq1e20%8=rLX}a0~M7ml( z)a>U^3cI0Vg@PSA@Iu-g<{OX-qlN2^I1=B#`_CcV1ai(mAJqSz_5bbVfQ%3p{xYM> z?#4>SW;TslyXgZ}&rF>TkWmf>rZ1%~aG^aQODehXjc5?p3VPHG@Q#^Q%g6-|HUD zhoch`fSs=MoB5+~9IfoZ_M(6iB;_ZiYkzl};de^AIy*`=X`QrU24?MV}v71|0OpitB3Ex+GX$#?Z|c?`2vr z#FCO-o{e76HpfooHG<%SvtKG6dGfb^ob9S~kqKs$LOxA5|J~)w;CcF1O+ZVl)qVhM zQ=qsOH~FW5fg@WA$EDX!eNl!_NiK6#RYAbDUZZ2PlHB}r$I)uo%u$$!9~!fIy#E6z z08^y`1!~eUD3{k@K@7CWf zm#b5p%nw(m^(c>)X0u@eUZdo;)O&hzkerr|RB;sZO!}X?IT#higE@dv4 z#?D_}G*8q!PL1VHA&Uqf|0!2Ry`YsDS)E40dN&8f&L+6-8cvA*`^LFyA_!M$K0A!M zfSleh5E^OtQ|MhGRbKNL0@k8}w4_!)^+fajSlg$o6B+e!;*3U;<_m=W_swvQujiOY z2vgcW+wusn5xqxWy$3p?s;rpi=pFxnM=|l-`T#4{=9{bZqKthE(o;uBTrD%)ru8sX zeD9)Q>wgXgQU1pDme6e>UR z;eB&(S_IlF=1*-T^-`KIp^@A13lkINI$7vbQ~-I4z&63x3YIBCtgp~ZBfc-4U%_y=-OP-8s5VfB{d^gSOob$%8u+N^Jd_0$p zLIAbh ztYG2@)K@35fQ8}nrK6g)!K`D^>?F92n z{EnCGI1v{cD|oeL!1q+XKD|rrD3;q5hZ;DUJ!tYkdHz^hkTVpp+-lvP4%q8<>r8df zGxDMJ>i90Of$(42J_e$KdD*xmo{5mzyICWXfIWr$;igo@>^)2-%-O@8ZF z8#`f8=r(Njr$JcO`9$U2X&Ud;hW!ns znaJzpi%U87i#36hzJgCzXHMNHFjhZi=%ei%;H&yI?@uMSiAA?(l_<@>Z;bV#kS`s@ z)BjWouYHbb^h0<@Q}K54mzy-sZ~*e?wREIAAT5?;0D_-i`khB zSmxM_Tg_f?C9k?kzX(DMtrbQD%} zSF^MA{NYI76tEv#ZMc$YBzW8vVJEvti_F+N9SFLGI=(hQG`QAhUUL}v%7>}g=~U9V z?c0#;-olcwlz094Il;j0cL#mp8^n3DPA12a6d=+=oL{Bz$hzL?b0#*Ade2#|f8}H< zc}3KwvP!bR)VlDsDgWH?`yxt82v(qk277?_MnBiXdK=fZgH?>=Y99OXDX4|QYo}^F(U5Q+T}|}n!38#n`e$$oTLL5 zQQPdlxjcfgz|IWp$8`=9u!K%R>3D|0<9B+TzXz}1oe8H4IJi9y^Pob<8HEUg*S zY!zM$>s@t@^N9&5kBCzl_jub9csU`CQbS_?|4wIiY8Ap&?sCRok-4M}n)7 zo2O}KL{vLR>c1!k{%yRp_Vj!$AP-;lK3ed(Ho9o3;iMg|o$}8@9(RHT&*F2gsem!; z%5rgB&6K+qIi4|j)OYGUJ*GMMBh{PwmGVUI3)eI=>T*Mq*Ecve)COe6l^fOAwpR!cc*Ion4V`aR8 zLo8&~tLOryHhAElCcb)EZz8^3=Gt2w242L~(gm9OrK4@Cg7) zbA=HcooF*;%RY+Hx8Od(qixoFiz;m?y{(z1@w$HMWAW_q1u#@iGjU*O?dLcC&WXIc zPQK>Q2c}Z%2uZrtrcKZO`i6vEPZ{6d0!5@r51Pt2G_*l;4z)`eH zi$r$`qG$p-f7FW|YMr+=`tw|tH{h*|!)r>Nk1uf>-B)-!_a7h&faasTh|}BhlBF6KJ#J?}eK&OIz%}RkucC8kLflrG#Nu zXP;E*Wn^$pg+Xuu?P`ntORW@tcx0yu7K3zrzOzb2R#}+HIxVra*I$#C%aQ%;KT%c5 zcy(9XMRe9wxPVhxM}sYC8?~Q}WXd>Q#&g<#$)W%K>L7(~KJ#oWxt}`V()5USd;4Dv zR~p|==y@k0dK8`i*gzoQ;1rME5m}@(n5X3R`{(F@{{_;3DW>`Avx-qb+ChKIr$@wu z4>*(1KaL#cH^9bra%DH@G}pAZ>V@Lz1cg7Is{Z)C$=zw}?>#d0$fVpxe=3%5+*MEe z-*!qS=Va9!x778;(@+za=jE|`v^8BrsM438PS8ukES|? z4^#Gq_bSZszH%QN#1_5DEVk7L$FJMQY?2du?>@`5z+tJX_*bN#L54R5FEi2i!RtlZ zGHb2)d~E&4?G(b_VCj`DC+}KT@U&*xKvBY!`ygLw>+pc{()3T%;~BGp@WGb-P?m@2 z)5W%W65AQ1Qe_@^GY(T8Fx`6Nlvt{B5YH+HT%JzDHnLor$s?^k)^h#H-IES(_tq^S=|8R~rwz>Sk)uZu6j{3GHN#>oGWAESC^3 zL6oc6Q^kI92~rqh(l-EfNm7i@7wbCObhQ_EvB*GRWar;AnIQf9bUSs628>QmMI8p~ z1u4M3V^Q8ym|4v*W5Xn(8(0?D7Z#W! z`wD)o;HN6Igt4Mi@=w28W_L&Cu@E=y&2{`Fz9abPP z)U(H9r5I}dibYWV5FV9McC!$lYrur<;x6&&_ArkJbH()F9aMv$j?4=I!@FG!? zDf8lQes?_zW-@BoU(vWRGAMo;?sTv{VFKHB-kp9~Iy$KWDPB+mNgI$SqU2YY)r%h} zRnNn2u`U`pyO>P@HT5yMP87=N?_OkXe!W)|&jgG&2Vf=R{~+dw`Rc%}PTlfpSbLIH zhO2*Gr}DEnd62;{L5w%rqlx>ti^5>H%dB&9vE`B2>0~|#PK@I7Ipy41b~<^$8^|+# z{gBRQW2%Vm+G$-JuTJb_MR-T7LD`@)b=6BV^Xo;ckJF=nXzK~2)G9qmUj3M;s`KX}`+Z{*6z1syPKf>Z1CJec3ZG+abF}~4?dkFQ7-S}T7umS;%Juh; z>AcQrF<+!p_a(RwYWE}b>+ZD{G6oK=YVUe7H|%yZlJZ5M1| zL2U8=BMb87UeWlUg`}Wc?6zP!M*1!O!Six-!fWlVVKA{EC+EopBX@gLCxxBI?;O6i z$)V_oge*rFiCzS3(<@U)OUrFWncjn(bm&L`<;%h#ls>)lR|2%BuK1%+_+)q=oBjvp(PByA7&D`8)(~R4*E1h8($uoT%xi9e6QB!P=2E3XN^uUvD_I^5@onai{!}mf$g~MANwph zCK2v{ov-Bv{G&Y=7mur&zuU{)1yP52)b=EjBcLlL0|%$;i2CQTgvYkK4v0x|LXvt@ z3Z|uAN$|DNds7YuGP3nqN_f_Jw4BcO3X`WQTJJj*u;fUYVXHJ#Cl}M;yI>L! zP2<=5sj^vF=TdXeFJQEjD5?sVh&tIvr(i4R%w`m6o_>^}SRq*pxhb&#d7`Y%X>xSZ z_HZrgE6;NUjacJmt{-M6BnK;`JF{O7pI%?p{L5ne)HN$sah?5;Sg^JYd5$(Y)8wWQNKp^%29*0DE!sN@@Y)Y2jvx+`cl~J`|E818`iD`HYkxLwsw5i7o?}?}X`wF)anxZ@g?@%t1SilB&aO z33F}*;@{YtAU3XGet67@FMuyVH!OFi%NdeKj}1+c|&m_o|;{@|(XAi>gcOw_eU4X6;?0sK>DWB$PiW z<76j0a#oFI9jAq4hko|tvI;rh%u2f1+wwE48vbwhHfVD!@ssU8l?D@6*>i7JJ~V#> z>0LM1|2swW-$(vWEZzT3acOyje=Z-3*E#of=lM@#O5U9-Hvz0{*8eq@Umb5rX#f12 zZ;$Cw;?my3&awT<`>(Hg7SuA*z82lDH`AsAGRhYDtzH=X=)YLQj=!aFf)6h15C^zUQ-|Q-h#jCC4tB1ujLvpuoIs|B@~C68pM8zr(S7PV_x`jn zCY8w_H9rrUz!HgS+ABQ6Y@}2p6X~a&S}rNgk!bN3Hot;S52*O;J^Nhz#LsqvADMW6 ze@$E-Z%oiRi8qSMy1Hr%b)@n9`(YxAroe9c2m}y*?|y}1s6_@#4Jzl0hXuwfQQ3)k z6$}wAA^qw69nB8DsH}@g`U7N^QHx}od5kcp3sZspJ&(T$bY2Y;adPhH;e;K(tvH1+ zy7}ebVt%Y>h~|&WWFBe($NXh7HtnxIJ6@~C&ZcI;FlAwXo{Lp-bQI!yUsjD@ zS^fSpD02K7r_SsAXuIX&;-D>7C7EG8?DRm+C}2)qj_zRcA@E}ywq1B7(D|KQg*^|S zL3s=s>6rLjwmM96cUlgf?X>!!{j3-BHU6adOGi}=il4h#wGIf(`^WMeZS-;h61C)3 ztYd8AUcm__B`e?@v$|^Rsi2;yW8%NwKVc$@4*Imyd_GV!D7d;iU}PP9heX8r`s7Xl z4akZ=GYI#Y@x^pJ60UP7dgMF3S?P#EiaIit@XL$rUSZ?~XFI~3lNf=hTrODWq-mIk zar1L`H|UWML*pgL`w%;G77Az$BNP?j5~z%#W}rREc`S5sFlg50LScUQ`nV-{rgogK z`~k5Y$*j+y5xrbAs5Hamz-`{g9bGd(+f&cA>{P?fWw|mrD*o@y`A3Db-sx?hmr_?d zJvmWaN^tC^BSc&!k`DujF_T*71(u?W%out=SV@mJg_#}{DmVFL@VIYoo7q zzSd?2w4N9*K(pSMQgRk&kS1|NWLeDbs6tzv{y0raaACeW{UgeAYVms0tBrTGHdW)A zLXV&s{(gw*P)%G6uXCK4uC?LS2Pt{#NzGz~?^&0Ws%)1d4V;h>s6&vm^e5kD^myLy zJpBtVN3U5FU<^g;2H|I-@pD(?y4{by@+6-G6-_@`{{S|u%Pt)8!?h{96&({IidV(A zVhd;rTaI_G&IQD`b_d*eZGwr~vJd1ndQg^n#(jNNMFGFR9NkGcO!+=@Y#Vwpmn21L zlB|W7nx5h@kMqb#c(Lb^iGR{BLQpI#A1qrjrPJ@E$6q*mr>3XgTZa+HieviVP_}?n zarANtm$CCOH&tA`?d6Weyoc91Lj+?CfzL__ljww>^DmIb**B7sRLtIj6S2K=_-q_k zfu!mBr=$~d&pS4=84!~b3loN#JIUzg?{iM>Q81Xp2g>gemvQ?9 zmtK=6%J1mY9J5#TqL5=j{d@Gq;nJP8gZj*q+|x{* zVdaXq%Ewby-HRPAY^%QF=RNmzMpF>0UUl2~c{Ubj9AlLxi$-Y_-wJMCJ>3`r6G_&4 zeX$hFzfG)QPO)`<<##;$()N&Il<%}BpD0EXyz(EY5jA3k0KNaUM3)s{f#7b>!G>v) zde)N{`zC)N>_yt%vX#|WzHfG6!3kHwOZgvokrz>QhK-gO^*JHvZ}r4Wh0+m#4Pv*) z3pTPI0rLYGn4(Po&cvdu+~qI4E&A1-cQ~>U{wp_jLWz#PEO(aWJq~BuDmE}2dFsPEV zEP(R#JCAwO^49%m$dxYh6|%S(1ZW^U1j8p}UIrXrpRTUbc~)KxXvV5?%8#n&pK*jc zr6=HK^_lE?puE`ja8$obj$m6w!O+9z7s1CEI__`PF>u29v?K2s_po-x2lC@@)mZ}4 zm;IHb1Q{+ZwvU?Az@d`FG>#>9_|s3vM7Y|f#u6Kk_WGCWV6tD>pqQZ7GO(=e82vDt zs!>&$@8!4podBYV(iN%r7{Q~4a{?NT&%enSAnrf?TdZVs2 z0UKzs)&+}qt>XTKu?K3-qj`TsOX23l4vo%jX$M23LKTpH@hSV0+cv~#03N@ zRWgL?=Xhdx_-zkU@z5=xs+B_gK+A4!!SxlO7oN&CYAr87f?! zOP$J#?(w$4`FH~qn*wkDhS9wdua&aVYhj&vm_R374BPDGipbbR*Q2iLqaUZhTXQw8 z_In8%(alB>e0Xe)fqi~OPl~hU-DJ!4)eIb3TD|2w6>$Bcll&sC zX|)_oI-B6gq+}+Xz_RD3N&IgwD+9Cv0DkyK)dCAi+&Po zugysxV#gWB?X>YnM4|^#Po+>kHyKp*F^xrb$C*)8hXi*Hf&) z($H&hG&&D{QCh2n(d-XnkQBV&4!DW}{P41liT7Tyd5nUS4?wnuyHdx1_uM-Q%Mw2y z*+lhSY-9+SSVf6CA5YRFj!rx~%bJ=ZY08g>L_Ky;i-BCojQW5BaxRhcvA>#w+;0w+ zNah2U#!K*O?P2yPtf(r0vo46i?s%s%W@ zZE%>;p5qqF^moUHBCe@Q8ijn(K>Ws5?k^?KPQFzmt1Di*sU6OjBRJCg`Rj{y6e?9rM89L zJHe{u#h`M1Na+g5fS_?xZd~Xb26{CF!cgU~Nsc;B=l_&MG2PV(VKJIWCDYL@%k0gK>h-$0t4Gz^m}ek>lNq^(SU6VotOc`+TA)n) z_*zBb`!%>(y=3S@KW-pF7--)D%2G(FCoZW9Ef3!zA^*4Pq}$@ylMqWJh9NYXXgn6# z2a}zhmGx}*(t~Z|hno@F>z}%2rY}2HG!VdJx!!nF(baemm4%l%dz)x+;fm>80Q$G~YR{QP~L4V$k5*8ewyN@9NaF98=2GxHh(Q zE}N%auK)e+zNR)GV!h5qyzkuK`8v;Io3-pG82ygd`s0e~=9;&)vHy>KWk6{8WaJYq z%~17t`lvWZZ;fZBT*ut?70N^)7U6zo#jEL{_d&Hsam5Em{%$_B{1$M%-{4NAN`yD+ zGQ9}lalL|P=siAeF*|;8zH@ff=#)FWRy$$5;VWGa_tJnAKjmHr<&B?+o*mKMOk#qm z@&6BP?;X|T_O*-Rb}K3fx-B48MMb45y(@@_2#AO%Rg5(0T?hdyh*GQ|U8#x+(xpZP z0)q5TNFpFLgh&k}B)RiNfBW7$Zu!nR-#GchfsnjeWtL~o`K-Aj#8a0(YoI);oUg~_ z5%XU;3)9c?4^{(7^7^u@*#k!`tC%xrB|@+z zRSYfMo;mO0aSmVnZi1Yr`OY@arW_B7u?KE_bAuz%r1zGkxy}jBuDl%A(_JZcp4LhP z!#6k4zUMuvnC>-UTWxMGO>Q^0p8bKLK?}QJR$bhSUPU;vav|h%2_=-ZB2H*tcWc_z zZ(U86VPc+OAS%wiO4WbHzq`8^NbT{*{8!A(OI0#Za32`Tl^M2G%CfWqGi_bO)DsO2 zvSOd)6z#|-z1rdUeKp}yi7R1d^^vsZ;&-4QcH}=N9LGUWy8J6WJb=<(tViV(N$5+b zwfB-_%b#mx7dTJ`nsSBOXMg2dmoEyaOl5t*m&`J?ezn!sL{A{^pWPvCHJut78#_dUHB>kQC)W};2q@e* zpnsv<@^6m*?5V%P-sK6d%x0$wYdQVcFrGZ$Tv~`53=%}SUS)^Gy0x;0^l*LC)Nxtd z#sz9Uf1+Ffu~N_XS0-|1gq1Q(1M$2FzpY=^vgt&+nvnIy9dXdM1J!VNg7dW$W zm&etQ6*&GqnO!sBM{cBTIiT^TK;C=2B{{Gxu*mA&B`MtHFD=OxZtD@j_$>0a{A_El zjmDsSvkqRgz`m*Hekj}UNO<8n?)A)h*Q}bEQHo@&1!hNUD zt<8Wf$t;~83bzhu-h{h+m%gav(f<#gE!a8L{edxMATv=OGFiu0tg-NMuIjG?Gp`yu z&Unpjwmvpzm<*uV67b0VXDrZS55O7yFW zf%Y8Lhu#M#Pp*?fr-a|Ki&59;qz7J689yh|eOiNd=EF7o`p42{e6o?%D?OwTHtPF? z31^WLYmmz{^zycfYE7pTY)f}h#!8$mlsGXUKt&w|@0m0_bqVi2#_KVVF)KppnP+B* zDKu3(dQTp#q0^|0(KtDWka-Hoicj-{cW+8hUh7DS{@bv-Yfb@jR*rfhno`uhC#`pG zBM!nU9gwKPjZ;le&Rn0o@ZBH3o?MD%yGc4yF%_6L3U9YwxAij?Ge#?Bx_kbq9iK?R z^s(W$ZrG6eZw0Gu%lbo}pJ-32oJPNVu*|c?RyTk75TiCWF%qp(u0MjK43rSk2_@>U z>SVC1XPF=G->+DPY+-z4+2~R$c6qA9Aj}X=85Kh>2G3@iUt@R#euntKpMf-$qRSnjv*)36LD$JWABH@a<7`9KCO%&( z3YqA4jDoq$%Tz8m*l3It@C$e3Qr=U?&%In|C|@KyhA^99?8_N%1yo#?Xt1>~aK>?5 zizb=vF67gs8)rqVvh3|8zPwc7Pg!@8jU;Wogu=(C$DV@`u&>Kid`dAZWOwbvG(NA*G%;pL3A zD2+`r)|Ed~<7`5T&09qz`G-?)v_d&4f}zL2x(rgXQ`DCUv@9+5*REVW-@t{T%|)#5 zoyHZz1qQpNY`)T334!%)oWeJMbj8^cik#oNqxF<&{A2*TGBe;QMuUXtBINk`a0v@;x1kN#iD0b-`8z1j17!O2l%o|-RXS^DK zvP%eMAip3b*8anlXO&kEbDTJ`h6gTMhS~9-mNbt^J3Q&e&2jz6ce}&cckbMQgzm~G z*4-^!0htw2=&Y;bEo2XVLbYjJN>xlexWwp@HabX{|IJ?D6Hcl*&Qvaxe3aa>WeZ54 zL2n9%hdO}z`6jW%gLIPI>(M7WoIqCdnGHBRvA1ge9OLQ-`X&wcRE2{R?r`S~L=EXPS~9`?8& zButeCm2t9>fD!Wj@Fi;0YHCUaj>VvcDv|u_-{)bkMPgREBWDQZJ&^Dr7#aZ;W$4dv6-5YR&541JN6Mui%B(w~@+$nbI=4>f(xiBnI`dSy zU_rfP6Mdnu)nktjDqIIoo5cLIkF!~AuTMRWyY#_oRJrBVIwk|m;8eGZB;KYZxK1oa zqeZPit}*t5C4dupOMYp0!-|+XfK5(BL=9{WOMgg1HP3 zzY&p&z@}J-<^)Bzj(e=`JaZYT@bh~vFue=^BF4wFTWZ_wC)+2>MYdcY6Xk&`=oVz3 z*X!~nwem9-OG%xC#~LM2pKB_yEY*DA4qY1@`sYR?^KphiyJFcCEmN_4@JQp{qWl1& z!b~*&_&i?Mw7Xsuz5L_HVP9+W>sOv;oRR;^-x<|6T6O-gP{gTV5}#>|2eC5KtZ98t zb+g&t|I1DbH?^eiU<~hn($FJ0&ChwO2&&-C1%)3UAKQIg#}_WxP>DwM`#-})<$aod zTG(3kqnfjxblhQq(W&KH9Xb!@Mc)Nw3JHOp+4Y>z_8wAbjO2Ce>Bn#u#jgF(b|PFf zxzeRKx7o5pePuyTJk|G_)$6q8M5WeMI-aOUqxiUD8KMbS`~3xYUU_HGN3)7e?C%FS zjA7-xeEZS9&=h{N2DgHN)kb&vz`RDxF~y*+1_|j3Xhz}GrAqy3mxrZa2uk=APE2JD z@p6R#MT(an_4I&-qVv~?>?9UzffoanZY;(&IOOc$k}!`ORe6S;97r!1ZStrCoeGz zxtSTXn&0%V8OuEE!q>uoo*mL+T_4nO-y=tl#=E=wc#==9r`xJsTehTDT8@6mmCR&& z%Lx(VuPlJb)M*^3iD@aJq;bRm)L%N|pLFho?&k4(dr)e3d~f4Rsj}R6Xo*-A9$$f_X(~^DXNzUAZo^-3QP3Fb zFkDos(8OX;vYiJ~*$h#F)iRC5S#skWwb%_kToXg|C-a{*^cmh<&pSbixzL&Mw(1D1 zEpNs2L$T-yNq=VN$K5HNZL)@~v^o|I+;+h9pc@3=zCJ5+yS56*pYOhLTsCjP);+97 zO-_Os57C{5D)f`R&0CyMCmRXQ|0NhiLjfgXna_AvoK4wOW7A3%G@&4^j3=97Z8S*p z*RQbV-tA#aS`QTI@6f2py#Tmt^nr_YebsE(6W|%XdQqNB#gal0mzW$tLf15qA@?K_ za(xcqv0+>%(lD5RtDvG2Y>_3EU+u=vMaxom%<8vnW3|zrMIvW)k22#DhkmtuUkx}E zEo~4ZEf6(u(n&b#G}1V;@#I%Wy)q^pPt9hN`a@ZCQzIi~f?NOk4Qc_zkosiX=2eT{ z#eM7mT$GYks_#pNfee;DF=Bl0ds|gNj2iuorWvW_!s-%(nw=czKiC*8tX9chDl&U_ ziR8n+J3r2b&PZ$)rhf3Ij*Sfw5K4HG535wJ`tbp0NCXBD&}@iI?rD#`y1m|AMB`{@ zjwB`OILljZ2QhvR>$frbU|pf8rIiY|nfeH2>xA=6x?R}4hvt5L=~`)UjK-|&BK>6i zOAcsbwC1$BHX*pOBlC~@hDL`RPZi8X*IeG9&+D2; zW~_@QQe~=NPWH7tsj3Jdmn5o&Rm&RSo1T7^>{XwkZn4v%ZD}j7g?Y+oDG{k5bM}*H%lvC~)Z2BB_{1FGwgBXk3b+Sr zWum4vM!kS>MaKvg={ERj@`Dly8}id^l}N2l_GwXZ|*wSebO^PO`pIq7t>9Mq(`A;TkUd0=KLwuklB&Z_|ZFKe3IS zepWSSE_4qy$yAGIh0JAyQ(fAxu{&sB`g34i5dd+~j|Ui|d#?hnXxIySLHuO5Ln@E- zF<<`~&P%Xzt})TW2+f{!JXCzc@5G*^o+7W2J`qSyJlrL5w50f72F*s5xrQ0r;A3(E z#a6w{g=cxu(}ZN;ATJJpL?6RBV3cT4y(EYz>l2hnBrhu>-xD~Q9CQM@{U}35!T-;` zM62fXLg7?pK9Xyvu5)L}VuU!)^+=7`3#Ec~Xrh)qn_zK?d}Eqz7Ky?)S}Eu2SqsoYPYZVd=hvv|c{ z?BF%c6TN=X6%ChJuHmsg(9#wPWmP0&8=DlnS|dw0ybU&lNum%7s-hVE|;or*#VaXzC?z!=NB$Y5L$w*BnU^G0$A9HX-HIb zy8+iHt=zCZmEJx8I2FD`D27mAZ%_uk4VHR7STMw|2Mhr-|xK8NkG1-VG{&cOc1`!_qD~hiu6^kgdmhPt<_(Jb&puiUMJ=| zjnqtf7i;xk{Z<~yRBO@(+3)6-a0e9Cw6H&tOSSrn0-Ls5^QbL-($p6%hS~Dj2D|-i z7&626@i!G{F8?SDjbrs@=#O-^@+ZZ`L9(FQb(B*yWU7q#`F4G=TfoOpOmkaI#kG~Q zA#yS@wDv7LB{?@- z^Cio9S7~g8SC=q$bq<{;Rsk4p6#5e3ORC%nbeLxln@Mv6RqQq=Er6k_rF?i({O-*I z7G`NBvdo7s|eG6rtXMv^^N${g|b)S@y5;PACU0X#{ zL$2$7ke81;hU<$KUU|eP!(4vmoRhOOLqPq~ca*VMXf3dp7%25XPrVgCjtgN^TKfyf zKnWAHe%{-`==_=*D!mFeX{M4qwjeKjz5! zT0eMMmN@%tLrv#lcVEJY?U;3NN${U3V=LLM{se7>ZhbDfWmbdVy$020K2c*WPlsv0 z_EmUM=*Eamq>3p2`gF)pP%YZ$R9wyJ=t^jFHr7m|_+AdLr8u@E*;_7sK8bl({q}s# z@6ClPapx;R*p}(!eq(i&cG(4CwYj`1hwruev`PlCDX&^_w8e`z*Q!jF;C60*ZvXrk zb9t_*Gb{Ah0Hf2kYN=e}G&i<=UcGwu`4LlIwf8k&I7P2t*^QD+sJ^YgVS`#;x%Wki zYB6pwSOYaZnQb$8Mb(8K6m9-A+Do?LC>G1A% z=5fb*uiK}bFjLjx3hw!1Z@1S9oB(2;S1nO#kUdVG=Dyi^WHK3>E)XI$4xS7q(vfu;IjK7#_!TetpW3 z0kee$F!ABA{7?I}uGySA zHS9_0R({XHU)$rL@q=A3e#x2QzeEeFMaZM={}A}|2}8Pixf1VQHU9I9&itIW&g-_jywRx0=A;$E0!=G$vG_=2 zK16^;svqqI61OVE|73)e;dSD70t#86c)0VZ=Ura4>h*@l#46a~&hfb>nN+m^X#$f{O7frV zu!mkgW~2hJM@C7?`HhHAico9yHj<&A%<; zR0m-j@>RDw-c(a#gjBrS({%Z)Jinae6ux-*vNC#U8VuK13VLRbvgcKcfg;!b0zr?; z*&1H01q=x;c+5?7FL$2%`3+ScP3d~u87>-fb22mT>{H*_I=<}87Kg$` z6DKuCNS5;@%$k66G#Xid@*G-SnHM)oy4hu-*Pb&Ht!8BMJYt9V5kgmKZ-HB% zb)AWofgo(i+&&6=L-ro^9!&o5z&@R@Brn_^xM#+lHCr!=G(l=j(P-6TuR>5U-)s}g zT>NsEGk{8DzAtQ{k0p9IA^ilU{SCQAHG^v(Y+oJwe0zOr@Z7f!yzg*?c*s(}qhqpP ztR4R_{XQ2Tw8DP}_ZtiocS~DczIgE>E# zL3!opGg9!pp@sfPu|sQ0^761RY-HCPlgNBz5R)DEJytsmWRl_Q0Ks?-Ue3#4fIqh= zns_LYt33549rn<(UO2!NykMjrkX+AqK0jVtej(FqRq;gkV>SQj%Yd1^MsUr^YQ6L7 z7U;HdwM8+ zQXF>~Dk;!lu`+nY8`lR1bKn_3Yh=Z@OqxTDHgJB-olyQF?*sVj>yZjhZMX8AyWTt* z4@BT)Q%!2n)LTLKPp@H7!MLR>z&~U{1#Pj{mOWkA8awwnMw9OJ{pGP_RlfvTyRWZK z-o?d;YN2r4A;3E%CN2-F(*}KN!g+x!@$2*Y;Zu)zwN^c0E7J91q?=gt^Yh`nqPaO! zOptPbMlzQ_UWJF*t6Tu7P6!LTY@DpRcyEUhzgEaJv(BlEM9Srs zPaVZwrV^{n^K6*4JB%xrhabTVAZP(u@`3nrx8&vXu=-GH-@vt7$gj%;>K-nKcTT|| zRtC;$h!j{Q_3HyD0dj+V;PTt-6ndKuuatFpk#VYKp<}B<=j6$dFM{S-a2SEvB!m5~rs`h3GWLlNA|?p7oFPdYnGdoa;lmF>LBPc8 z@PfU|M=)%ISjm3Kdl&cGAYyO~2a%@&0O&IZ?!@zgmUx{Nz4qa7BXK$j_mBB0xpf6k zzw^7ar_}*O$TCVEz=pD6o2~u(?ai&c?kmHQ2v51`l>M%%BpW5@6E+0 z?JJ~b)(uw&v%y3~=7U|UhOO5?FD1!2!V!*M+fd42nOyRy`Ck2}jRjXApaR(^Ku$5! zAZ}^8ALgC{w#%(Mr~CbtLJ-HNQzig3iPad!t##{@PQ*!DL-1bfh!U?>&MbNjj469k zD5U0KK8vt|AkP61WMSwFfpg!)w@bm)4jgUTM!v9-!xppHT`%PGJ-=Q=Eoi|>ph`$7 z_GL-%NR*P18l^}?BPhwc@d(7V@7?gh;t?aIagtX$VBQ71NgEq3*Nx9WpbpB-+^RGP z44+jsYUMmww@|PGPc?BgCP2`SPt)%T$UFTgp8?1#08uGe>bIuZ^lAX^BnWCKFfb5a z*|kO3C4P*0#JYD6DwjN`P-L~_GXX=VC3;k=fB?oyS;C?M~1b z(b5p;)}~)d!(bz$`Y9~XxYe&*GKhu*40?sT;bGooV+k%#$E?8iy%}2GMs82ngXK@g zi3$0(o$NAzwDUxI%uMBc3nKQQMo`utU!ENS`$U52L8{i)t8N#kkmu)5ULyze6OP+A zHY<@VJKPxM5PAMYz&qFjDcr@0_K~Ls(t$s2 zb8g|{GSzr*k`8xz!gxxZ+VAv$_kWb-8c88yZ$*eyrKD zD##DaR1@LSjUfp2c{G==S`^yVY-DBF7s1*?6#Y&wMkFT(wxN_UQ{gd*?#aFGtU4>xe~h?34rLjcJm=0V__ToR)f3=&J*pNj;Vy@SXi}cV9JGOq z-Vz0mUYx2+odNqE0_wHgtRaxb{n)w-lE{E1%AScE+Y~w&!fyFlBdTE=G?@twZ(dSA zg1hF^&!+~5&b@pb7l+ucmL!$=INOlGv3SSjjlk4kz}#SytjhZ<0;nJeN#GmNojslq zgYPmwmmm+V-2K3LcN%LYYhvh@9#v-1*212j*Vzvq7Z)EdXp&A;5A-%MF^Lh;fFcNB znIYwtuD7NJp?jMj-5h;kt{N~C3~@p%iY;YXa&hNqHDd+`y&qm~3pgeiSu5;F1TM zK!I^BeiKiX#eo6Af^Qt^GBuWBKG6x|_7ZXgoF-GXZ~;WRK1P&O=HnRR0V@SjZjD%s z`croia3wnqo$E`SJ@fN*xJYAD6B^_T1eFEY>KBNe%KUdZns;i2AN(4)XyY*Q(DCE| z=qn$zFyTch?uEnOphY4eqzm9;zQ1hGHcs`P27=o+*q8+Q$(ai3k~$R@T2#gR`h8(2 zd7e=CBmFYg<F!Z-9|W0R@-;_0|M~%Y&l_!dh5o39Ur@#SfL5Ih zW&}YxAi1tz?`=ktLSV_KqQWjC&6U;VhIHPqaQxU|to~%i+d7~=S{Sai^4)Ou{a5~U z?F$h+%0F(Y`b~XL)xts{s%Q+s1t2{{5a&jAMzdhY9b~<>WU9aK&mVTqNjEp3}*OL&QpYsqr^*eNU6YHc64J(*& zGhrMVr{W)s`YIu@H(Y&ZP7gnzvHat1<=Jb8PSDn%FGn2f*6kr7Xn6ubgy~x58h*<7 zH{F=vbzZ)Js-uPhmUQ6hoy7u-9 z+G`21|Nkfc+mq>kDZcu5VE?_xm|1_mif4(ZdU#o58&kF)eL-cw$$%xys`i>F6y|X8 zkFh_Egp1CHu4cY!Ov%rJVmZQK7-N6D>fvATi;4w)NWoHaxll*`{0g7-TrYcf)_^z2 z#LC(WJrA`@Xbd%!bNFiCr*8=~ooe!m&p5TOyg4^P{Juu<0x7gr$b05}KglxI9=}BM z8Do+9$61BG&cm^>_LRQ-8X6N!ZJ%PuYve)Qm(2d=)hJ1-=}c6|yZ(MZAMSq@@lbnBSzDo+EVxU*BS>uiuGYpk^BaM;wiWIfQDWB<26;#5N z%ZC+XV${wGg~4_UWl^@QLSk|mJ4!#^aT3K0%9FEAqvTb}Pchy+;zBu5#^6*7^{ov6Ow^HNh zU1@a&Vb(!q$bCTXEwQXoa~n%+NKL1)`lVR&k;x;iljZJLdc;#hE@5p5UIBxbmK0K@zBcg=GFDGkoTn@nnOhe}CW0vn>7Bouw`|=S2UEp65f$|=h#xMY3rw4 z^woElw7##*Ia%KgzJ3)Eg3{4QaxYa}vX2Ah!Y+&R`Es*8}dzG{V( z1uh)M^;J-fOC{e;H^1uN^f0Z>(9CS*Z0^U8vj0S!Ias%-f(Yx0Vq|W~YNLKL&&}T& z&(6@6e!<2bdf1M~-S+3oo{kqK^nFXF2DvOh@p11>{o-8hCX~GDzJu<&+evGmd z@svIqN8(3`a|^sV+;~h+)Q^@NAd5RtcCMF-UYd}G782sT)$G@sfeHsMKrMi^I;SA^ zwz5*4IJ;(byKOQ#xy^p`kK25LcX>&?)%^UK#SCFHW1wUWkO2;BVQp=3eUQ@RM?n>c zSfCrc=>w@VyYHiy7|Jyjv!ow^A{A0`r>jIG^WoUYi>f_Q0*Y3J{uKIF!HF>l4){@9 z?;hEDVC{NJ$lV)Q!R7LRLK^)&k?}n?%*pk%9ECdS)kwdIwrYQh1*tV&g6MbR(HK&h z2Mon0Wij~1q{c{CZEcIb@HfG^1g**Ky?*KK6YF^mSC^l)Hqv%N*nKs!N_lY<43=uN z7|}9Xd_SC9jX1Uh6yCs;(jWDy8&hO&wnd10;H#65onD_B!X7ck^(m2B&ITjoQVfax zv$J&%3NHA=EID5ak~^d^*OzI|ogJ|S5@~jkRV?sxBq3xPS8c&JYc5;7^}EQ8bsymqNi@m%;UA=!vB3Zy4v% zVB~t0c2<=VyvE6$#EfuRQmJc&Vd7ma<}VD6e^I28)z}3Ts@TyZyMgTeEt@H7uMci6>?!K)uDs=_3Dg;9g+DideI1%U=Ql``8RO*swBH- zE`*oZFyErW08UdJF|s6_rG*|B7T3kF*{xj!3111t6BoB_FxmOhPA0#%?^+}-%p~-5 z`swuT=#!`jg%W-gC;D1mt-!|v{@9~V7c*?|lX6%sq8gPk{e6buhfJFf0kMhLT7F>s7SfBynLY3NFy-aOK{6Rh-qBtpu?>Dizc%@(q?R$0@@0R#EDxh2)rGc-r0d$-xMSh_u* zt+MMx+qpyYg#>?%k%i6yLZom&Uq~_}Q34jqwI8&Ta~Z$Z)7&eXdyp#ztGi=OMJuTu zx;xT2w+dY6-$Ov|tuhWL2zc8ytu4`fW(b9A3AcUex(h^yPRJe(UPmFF81#|9$5->f<>7&!*q!6gW=k z@BN*^=ioSa`+xq8r}jYjUsk&AqBtacq-clMzUfwOn!LRHbF|>x^}>E{kH032&UH%u z`?ouPdQtv&zxuzJOgr9s|E4?GL9bp^j8+-2UxsST*`wRTfi|RB_0irrR@&0P{a_rV z^n_~UOCSRPT>~p1jF~+R`(x|S90I)bgD&rnt-eeT32eK~C}kXR_|}nW=~i)9-SO_$ zp3vYPovM$fznAvnaSB%$+5OWji&IpS)rSH>Wt!WdDPIkwcRVa$SEFfPaR--~W#gE+ zNykui_Mq%0XeiMbYc#>AOR=4mZ#ozCV^Vn4E`?Za~yr{|My)tJg>%aMz5aeA~`+gA8psd#n?xDQq@!z zXOOLIFoO~R^~CWU32A~$OzP%+Aa(c(0{_5ywO~)hEQ<;;{rf{*t1f_qN%xNapa?j2~_kk*bkgf>|WvZA+X_qqiNSyG?D<8fP!XPrJdBr^m5NGe! zmb_AMYSS^_hHWhJM?0PL!#A9qTt`fgL+199+2RkKm}P*uoIL!VuzRUNbNV?CBiGy( zM;pmm4{qC%xOe_y=HmKRj;Itm<6% zhR0vDw3a^?_gKxpFt@E(lnLM#Q2sSx%Pgp0`aOYL)h8MHt-2;VGnw~;Hr9I?sg1n% z>$Rq}>F~It^#jH5tgh|hqCvGo;T;RR6OHMta8YU{L*zetI1o{OjN5e6_Ko{mH-A09 z>B+a32emqDtUTKfa#dS`bxR)hvFXfF>~zTykiBeHmk@wd+|P>Exq|I*6{nF9fRUt zK4dY_kmz3K)-zoXJ5sPT;36D4eT*?(?1)dL|Lio5QCX_xOTmQS zUzpAR(+B*=w66_`XCke6Rz zne&RMFFR^6`k@AgX1_xV9MD+HecU=a1RV5KNL4@BE(TYZq`gtH#jcDcA{9cUs6z8- zWA6QP>h9v^ty{N>osQh5-S+aPM|Cv+u{6n)C>xHoqm}?ef`0tkBfy|CC#Tl`@pl(_ zy(=+gS?{?2Xxn*>i}^z~(tkYPiA9btZIr#pu|RlT+*Or2YehJ^;SX8ryv9AdH;2#i zhwX3CohhF5rg$5#|C2L$bM!h_wNm!GUH?AzxrE5uhYIWetj&qVTbkSi9(C@jdPuNi zk(71SV$Ue?cfB^};KNDg+Vj%4?6c~HLK&JDQZ;TR{SJ7yQVpnxl5UP{rO!*`RUU^- z6yzmK(AY(I5DCnR8-6Btm$Y^H0ufC`YHAf2(+aYLfPjDlRLHV+|y2E9{w2wj;oV0=^xLdzsZQ zB{97}S#dB<%5n(`-5M~;IGdo!1ee_YD$DNe6K8~je|-T04(i%iW}J;Ga1x261KRB6 z5$H2V{s7hWAV|H+yFfv;OP(_IE)Pi=q9=e@Vhx4Ta(gqVAHA@@!e?{y^|s+8pO(p3 zzolApc_k&TeK%ZO&@&RMC%65bkmc9AwrD5nydjUwBcgtJQ+r3p92CHbFxKYgX2P}U z5BWF&$0V=Q_up*T|8T#qi7JbZWizQze#Jnx{N{weZtpffTb`O{#+qdy#=JtxA1!+h zmFsxFF*G#PegFvWn-8+vknnBu5(;9#!$rJmw_CcGjY9EdrQb1i_8OGINXYiuH$G2g zua3jvt>ewB^QmmfibF=KlNqGdF{M@iQi6ZQWLBbbN5mnd^1VR6yLTDhQ`sb4;aWBt z3o9`QInG0d2@8-#75BfI=vGoT6eb8&z_=3QPW@+xlA)Ha{Q!|Z;2OG7wa2T|qKaS8 zmyCkzxMV6lA%PE75kARoZ2c9gqtsnNL8@e|4A-`%*n{4)U~y2T#V=;(5{z zH-cFnFih;<+Gcalj$hY=Gkd%>)#FMJ2p?LYG;=x4=zX?j3CSzzN}+>aQ%*UTA0(C` zR<8JQhl>cBLTp{^v7~}1j1T>C$*Wo#*J7`LkPLwJsn=pXanM|7)9JPHX?Cxm_;;I` zN9~at`!3hIO2e|H?!5%VGhFuwERNv5Q_^veSR@XX`A9YWqYdz>6l#|#P;nZS7l$1LNq{oQc(=K26{Q4k1BGYn{#GajFf*PR{iAIKg(0Z>q45=MDg&1-+dZg% z8*8rogl(<*b|~?Cl8O&(E_N#Coz2am4J?=T)ch`Ut=ZZg*p+6F7AV9;zsKh1&%&cR z{FqSjP(xS-Aj*ZUniZXB1cTiwkaHhYZYoGfNFYKW4+IA|UX`i=MZ~B}thL1uiDJ})-xIk-9hB778GO9ts8@ZT6yrHjdWkQV^OhRU?y>kMT_DdnpaJw_x zm=9oYvm#*zfz8wN+09h5`W7ldSJ1^(jC)-qjSOlnKAo31>g(}QTImhTq z7BEz35sO^=X8sgmpvtjb`&_cBS?^-JoTI415@>)p_RZ+@gHy&x7f>+>*?xxPM_kE{&W>A z#BS|hnRpx>hz9dvVPU4m8h+pzc4%`8DgU|+n7~bNt)hQI)E-KD30d7;U2XVKCfOE* zkpwIn$$K~987z7))*bX8`6X~&7ok84pZA1izz5GaBNaIhK(tWS(tg0Jx%?_C%b{kg z(kiG^(>n~*$wY!CUW0|fKzRce3BKOa7;5NS75iRwIsEi40~PXzxv0#18W9i_6ooo4 zl)PTt5mPnn8kOE+ZL4G62g{F4fB_PPu#&p(Jj}}EX%2(+0h0%u9Q1H)M65j%_IwaZ zwnxTR1F#GN&|qf!P%&VaydoZ_MT74<9m($n8%cn@17>X#)XxJ4z3P)azi#&a{_TxP zy3@6Zw%6JZMw{YVQ?)`@=0;s}w(RqVjRdjkqhYLjmGG{GKyBE(Z|r+KGJC}Ev0-uU zZA0iM4vB7AW+(ewT+=L@?|46P1vJCl+OPW$KLe35jaEEJo3YC1 zbEAs0u<#sh49`ym+q8QZL7DRnkKB8cmzN!ZtV>y0;C!>vIMC=^KYjs`)xCcKB%%-@ z&|#TNE;nuut8-1378YkQt}o0kZP%8xsk{k14td9KPY;&ocJ?j7 zZMeW2UV(mRN(u@<==wHadC(xX64VN@a6sr$Ex2+$5Mqscy(CIe71yY>rAdSH%`Y4H zK8oL-$jx&?GXWrf0zwl8B#TLdhXr=%i)%oIlrUF?tlz{2BlX!@;L>eEpbs49$Z@u4 zuLDiLzirZF1c$@HpIm{Igz)N&!AsQf*>?5xMDLM1ZQ}rS#k7ylTY%>ERn}EZ$`0~( z2oDXMIR>iK^lB};ulLsTD8G8Ns8ez6Q{lyl(`2mAG+0(d9f2Mse(vQ7p8-vI?FT^q z^M(Zc-HlrZu0Jy@4?y|W$bwf*fiD*VI5fuxCw)(Qy#bTz2sLJ?ssBSxL3niOQHp8L zJixTQ0}>bTGK*kk9FNDJJ5<>3kVpXI2*1C?4Vc;GeV zkx&SrVk^j|T3i#5j9dU$4t{%m7gg`}ifbPNe1HLkphwVyqHWh89iR#yp#j({(F$zX z7rtE0hV5Bh%nL=L41`sva@`FWvOw>tg@B-0P2FegI}c85yPhbfh+Gavw-13Ez`GCz z7YruG(R?Gqsbj*jf_)c358dZ0A^2Mwb`B8l@5)5?UV0An1#eIQBqD$xFPNJv0Z$N6 zLovblAR?_whVzKq1)L@g;v>tg1axK+YpIk_VvZ>L8~%d+X^7qw&`YlMUcf0FFQ0+w z^Z&ih{35Vdn^S?D1MBNc5v!410_O~V=wnHVKOh63urPo)W~9aR-B$~SkxV9wqNno_ z+yDkVX|qP~vZonbtVtuLGN&E2A3$hj7p2fMk!G=}=Df+yT>o!$NP|QZrk4gRmOl#b`ut$gY!t`Di{Gvh-Z+AoEpuHe&x%k z_Tz)t3qafZZ3tTu3*i=mb;}$6L8i!Yc!L7HY9j5?!>|Lpm}G?F46M3U0H2V!0-_4g z;ci*k&c=kUj?l?@p+F7gv#x<)nhoq+$V|L1pUMG+q18t0aLz~U9iXtw<-U^wF7NXZ z#Gay&7{m+q7UApAVL{RkTRz(KQlO5_9^lp*@L>5|BjR7jA z^>*%E2BXnEsSxD>0%w4legq<3agPNV!gnX7vp_k9*MiSYd8-zQiRQ_vbngu{Q1WP1 z9*LALq5mePFdyd9>?Y{GFHFk*WK?eL%6zz$=DqIA+Q&I^I5&m4JKk?P`a3^-{+nmb zm`%usOF!&>|GitsoG$C@Rc=)=*!j8U&u6yu-=Q{7-q`SOl4kUwUe2}VTYu^U=QUpO z>Xfek_Zt@MCj9@+*qpBmLzn%hu+hE#Z=KS+e+~|wH`_Z>hW<|)%M2kS!;}^VIeA5_vY0bho=^4YtPSTC`~RM%0um@FDnD@MsiA(x{}~0x;m!X=-s?a4 z-2Yu!x;jWmPC~3^I`e@;!+)VCt4bH zL@>yRyh>O8`U-}x%a6bl0?mO|94vi}^d=_tNC(Q1ktT&%B@r4izfq=NkJxG?r$7ddFZ}*p0|MmyxtQeci zXP-!UYVm1boXVfUWtV)uoqKhGM>_j(BM`;GLz@Lvz@Y#LBU3;LA#3q(jkAf^<`Ab|NMA*_Qtogd)|k+AXsU<`n_9tf;ih;K*0qwcbH zof#}c3>{*xAY5<-unq`#2}$MwV&^>R2}vFbM9*NuaNpzQ06uj<9!U${?u$Z=Kzb|$ zXyg#7-Vk0rne*bTavKU4dBy&P8Dhkc7{0VpAAvXmj&JD_YenRiTmjLD}>AS{dW=5Wx zf~2Cc=5Amc!@C)PFDiy=!bO1r4QZrjF)ScNrvMps?C{(632mK%&o3s6rX zw6B9;DOJHGcH37P6PS=mz)deME|#hM^_P?iu$awYFF}itoOY{ZvC&G_?HWQ;S65f0 zhz30mG92KeASOP3y#Y4IAQTCA0Z1nUR=Ejs1LEL6Jkbj0^kJy`mA1zRfWYd1-+r*= zp5pIK*^n}D@Xb#YAp+6D8Pg&eez=wnV(=Eg&;XYGo*63YuXf8fLz3#GSvOb{NTC-^ zB&8W_JoyX!tUG8=Le{tS@st8^>5B65=YT^FcwU!p8JxACpkOZql&~?Oa4sR%{s>fd zSqv@U>VlOVfY7hEd_E_~c>pB#&fY%I+0eyzwtm5KIWZ4a+aP$i06>5g0AQhb1>_G` zov2^ikppo}d%~XhFNfJJ!-md){sYDVIxUiHMkvD z@L^p0U{-|yBNrDm?Z3 zS9&hl(lx&hjG2Y6nXZR`@?ceKyPl{9Tq=)L2Y{}<0#?MRUF_txV9;sE*5A!?hPf+P z=S-%P6f;}yrS*oGsLnR**8cI<`r$*3p5&W1xP&G&bs78gvz#xS4=0x+rUq0P3&uB- z&vfU(C*Pl^h_!k+JGaY61(PId!SO)WKmz8AhP+ef<9#|nl+9Y1ALrWFqQwrjzyr{) zj}n9?CGe+9U`G*A23YL^02}}cX!9it=Yuvca>?Q;Hvy7WFC&>oC-?R)XPh=R8VVi1zNyOEIr9Z1QO`2pLRi;Df0S|t z1bFS37o3ZLlvCUm1CFX&IXO^r0&e!_ulKnyBqeSg78wdy5W(NuCM6|h$8Y1?t}Bmy z?(wn9aD8}1T)Q6zNy>pp@JU|&g)qb6m;-Eqv%C8xvtpzS2-FXN@E7-;64b*U7FORe zuou?yEhDf!v25F4#+0;+bNR8Uh+UbQaN@bU?W=qP;n!PDX-@ypY9rkjic1x@WfxMKvWQrNSD6-bFYC6by&Hp#A{%j{$#x zLsv$jTkh=TMp)^`xoeOR1R~)Tibp7Z(V9%Y`1ST~ZEgYiGrj)@XKx)_@8YA7x5|5?Hd&APLe#mnv+rb6`}VSDWUjGAb&{ ztqdNjt{#Ho?+@oo_n3AHzDma-_HddL{f^WIkaz6T>Yzr3ImmvHs=HwS>zJ7_gymQm zz?o^=L^{WSh_IPoregDDFCzt%!P%0P(v(Wbv^Aox)6->0{vWh4`T?`vWy~G)HK3Hvy~Q%*t#bbTlE#3nYk5sL3u4{Vxt7$Ffo~`X5s~G`RQKV4^3i9_nrGRcytulWzChTRgYh+Gs)OT~lmy|$}_Zxy| zQ24>l+)3N!2{0!Sxyx{d^6c8(ry)n6Gne||!h(r}N#&NC?tmOKQA7A{i+qB8e=33Yb+rnZ9w!)%=dNL>1aU!%Sf{MDl%e+X?WK z4M2bcT!|vSqYHVh9*TULT%hXEK=&eC075#CxqGJa){O@};RPe7ApCqli-HL~5PF$V z9b&G3S|HCQ&J~xeBElFnK0(NKdzbV;d^tO~$!_eJImtD9(&B#>6~3>&rp6cw$F@z_ zR%y^Hh9bJL4(o$R5J-f-_7&JOcJ;Uapziwt=mhDGg28eT6)hvLYPP@9LUxJyt2HPA z23Y{pTa&PsmbX(uJG+2VUQf?GlDDp?xENZNAT1!JGbk@@Uuo>Nwvp>WD0QeQ{h_LT zdELvpct7R8%Z;?G!9P^TUWPG`U+NoReGr--Aya`s@&gVbKc=k!hW|t%1UQ0t3Gj+5 zD>c5Ybf;=2s!LbFkWjlc_RI;z;%`r;`Cz>RhMv!C28*D9AJJO=)1|z zsxmP_mx%yQA)QS?S|xkS%Qyve5Mf~G-4&BUArTE?MHRG8K~EYI;5QYR=m$AI1>(Ri zJ9u;JNwnmSa{i|S+-D8&s>xo?!bktD^qs9{!I<>h3J3J`>tkAEK^DCbY|VzjZ!jOdw1!^poE(;zu?ES7rPQM zm2TsqQmsOrTuNru%bQD$gm+dmDldOSG(;fH=>V0s?z+QS*h9!7h`4*-$1bHJ5eC9h z^VVn6rlJdA)e%wb-!hBwNVb(HFsHD;Mk!(*kYfE$ol%Q$VGZ-qNbWpzFd^C5Qo4EP z^hczChZx&gVd2MrV}sn!O$WE0G;@3HHc zf5TSa_i{SYn2&9<+hbJ5|JKxo{k=9aF3P>PI?!h$LmT&!MVhUqPd5^sxzX<5>oO)$ z#}(R{A$IDSJ~L*2{Py;D;PbkS;B4u3G(DXS?I0F4qUA565?01DuMo4G6@JOnbT~D0 zBrY>vcLrGYI<$OU#z8G-IDuv>R1dr^@H6!uZN&V*0wLumqIcTs-!vHk5C}@T{CB8| z;o}(M{Scp>0k8tdGY5XT4-UK^1jdJLmeww=Rv(|uzg~|qDSrO!;m_S48fJ?yp1nsp ztLf+0=_+&l^4pF{T1x8weqDI~(U`8Nb5A|xO8+|}+*dau&EZj zl>Fr&K`=27Kq*tyIX9P6E*vq50Ek1R>6;+6BkDDTX~>jLo=E%rHqaWW&=dHYS^)7y zr2RTj`tv=*`-dVGvKWwQHlRBFwhLm0IMiT}nhWa>uHZ32Yq#VFupXC`aLtiMBk}nkYd{%4ACM}i+6N94 z7kns(T0IKgf2X=d!q@N5El-B2hIrVRHy;pMzgo_di`4tM@%Zf zK1)EVxL0m~M~2l7&3=tg;KgdJz{SuC*^U6sGWNB1auJxq3Fj{QuGiFHT%I(6j8?H0 zp}S1hx_HwI^sPhSxk0LZ1VKyv&546k2i%tVa(#a|JQ(FOl;q7FzQ;Ur!lK#sS5*LpTu;s$l z9FM+0bSOw=;5*&Pe(5K@GjVp;r2gV2L&mnryaqFchc4Gb@iM^}T|=*BX-w+=b|0o& zMAyLxa;#uag@u7Gj<% zsn^_*ziWRJRyGKm@NP8Sje!<8eF(CD5+n7qipOucZKszTPVFEZXJ>G~ltCp34u;ad z44lVt2vGyOiWW+Kz;vQumw-tCkASo}jf-PZ`rI*?4%Q=hmC?mCh+hk-2x|uz#}ia~ zL1E!c=ey?wt+1(Kl*=3pn^-DHcccNe7aBenS@&}p06#1F?WUsnwA(N{Fd_4@hNlYw!&U;CPs^Uo#M zLTMQ;{t6urOA}uVdrn^G4Qa4L@18sV1Isd@QSyfG1$C5qpPKjSMuC)HeYW}UTQO!+ z+eE6Bv2SX=j*l&UM8EUhs1!6hhs~D7-?V?);(Wm2pSj=DHd*1?muj>w?Ab4ob^P{X zubUhF_OwuhBg$aq589!2K-g}eVYk)4E$QJ1iD|Qg?C>3K9CDn>|5`! zY8gu0tq`|~EG=+4|ESogQ(+S5?d}!Y*}4C+bzke|s0o&on#2s}X5kStK~&PuM5_v) ze_v{^KEEROPc4!Pee$xq2Vc$Xcwzr?A^jaU^l&I??Yzep6|erz_&1^NUil+RR{*sgh?Uy8W>Qp zqaYjYB(SHM6 z79ITG7F`N6y=;MfgK!N6@Ht5A3D!P9BtHkWa9B`k!0u$LAAAd7U6Bh189dkSr{Fq# zS{x)%V3F*ZI^jgyUBf9apH8Cs;`|mV$e<`WhoYG~WE{j%x~?OHKzk{DnzQcv5gZj@ z3X)*zgdBDOROvL73<%AqR2Mi!wP4M*Y`ia7Iseh;Cc|=NiZ8?6-xI+P^6JF~+P0^a zqB9%D0IFP-`zvU>OCz;AX;n!f_aevtysw_V;WzQab)PEI$4gp9rD8Bk6$eM&eop9Ce|zE%>wdk}X)g3q>%-ivcZnFD9ZI)t1uKsw4rOgz-rEVRxIeMYkHrf%D_b>L z%CZl~)@j*-%h{f(hw~53ceU&GLrL8e{Qmgt=HQ)bGdQ=MvD%wH_;_T>ytQKXJhV~XVA`(`i2)`-^4v~LAfq*tE6+*gxy$uW!HLZVu zKY@EOxMTG%Y;dDDat8`rKBKzV_x9Bfyxuz8_sDWok~M zZnyemrx-@bg$Jg#BpoacZPH{U`+luSZZl|ffn`XC~4n%0iPqk!)g^@W#JV#=dCS0WOAq? zq2=|5Xc3`B$sr6ZD>?t2{v8Xo4O64?bzHKu^p@Yf48E;XB)PeE*Y&N+KDbP?9s2Me zHE@hcEW}-SOdH9ChgsG~(%w&>*xyggfVE0ek(l?L@lJ-IuZk0~qoJe`mUv9e12*1-rPss>EwmkCnLT@rP4sgXNA6R2l9xwMlQTwAc`Q%iK{0v~J zT)5r?o7SR-phz&{aa!grD^cDwh`03Ss83s@Zj~1SVxVwBl#li9P;>8Qn`__ zv4p>WMY+psw6BFhL&tiz@Zb%aFs6&7ji}Ub5qkM6qnGKAQ@=3q!m-hmHCV${n^?lY`eubbI+{gi#*HO)g!F+A3D{*k-B|~XSw<@C&#>Y zbmx2#o{ict4l$knD$8EB6wKEF;)HKtiOlhMsHv%G&+3Gt^?BLKBZCJLR6_J~@Fsxr zX2F~@>brQ(^VdJpRuAEMp7*jveDG^oUW@TNUhV0qn6WM2W=>|7!_#xS(bBm?$J7~{ z7OxR?M`?4qbqxhy_h)@G3D+u=G?MnO`jEGwVfMc7jC?l{qknzM+E&gnI{8(yVVp{@V-{}S%PO_07pezZ(GiiE+Hp+KGItDgw8x@JS14<$XFM<$N#m>oi7 zg}A#41!T*5-RT?W=9iVzLNUNz1J4IMDGDE-?1b)t_2O+%%&vnz1>E&_i%1yk6)1hS zqJijJ_+zl^wDu4Pv`FFy-%6LX2b%}R28D&0yFI1-2^VB9d;-4cT^nftbp2u5w^YlB zTkrh-fTz{%@cLT{IUZwM2hqpxNgq!O;Sd)IW%`~-RP8AK)hCs+7qnGYyL)o{vfy$Z zt*mPhzv0Y6n%DMA_v%GLrhk=YzF9Fe`Zk&m@IJg~lzLq2#e~JiMXsnr5hwmrnzwqj z%F986(rMkoYf0uI`#PPX?|9low%*jHbmjDm6Jbi7*WY5L{Hhn)BS$6ze`#Gc`Xlwk zfBXB^@TT1w%dcZ5?~E$9gQU2GQtEn#g$bgYGZfe}G*(N^iOm-$JQ7|*5gQtMLxf9< zFmDC8Vx*(Nsm0mo%nQ|lB*yFDYn_;w0BIZ))J+hCvh&Bap#v34GCQ!sM9;sb75VOt z7?5w_Vcv@*mqWbJ^VTm03A>Pia>k&1BO3I2z0+Fm_oug!X-(iS09WTHHQ`?iK*l41 z95ILs-_bTR^It74G=(`^fG(h<&%0-kxrEp&Yd*p7)H^m;_-vWH7G*%3|BAFIfCUiY z20%bKUD5Er;#?O``WPTDV(0|6qW50gbjR368q%Er_i+I2F2&{L82}l8RecVHTE%j& z#qCtosC7kg@h|YX-vL1x@n<7C3K#?-mq^1@Nu>Sdcy9$EY4skOu=#Hxc55(%d;owV z4N4(miRNUjigLi~cm}8%Z~-vcVxTZ?0znAEj`Lh6($)j2E+PR>t>g_7;(D#imX<$| z{+sXx{kLl4DU(8Z+?l@)apTsg1nkG3b{hZOwom((PLq=99{tR09KC7T;^G}?w=Qby z+UM$cbZCEbD@%GOEZ1)4Y3=^Q68h7w6sME*k&g3|;R}<4W3y1w{tBVzi*fRHIHr3O z86Pve#!nZejrUeF-qktu2bSV7Ilef_DgT-I{t#y}*6-hoA7eT{r-~;1Uses%dTi>g zKA?_2JDvQxE@EPWrSS*!W~nsphF~lwYsxe8uU>OR-7x z^Mp}{u8(UBW-t6iPlo6tD?I1jU!rK=G@j0o6mCi%Bse7xdl}mMpVi_JQ$PCNav?o@ zNPDub!7H$Np;7EW+ooMkI(JjUN7%C?$s@gredXl_EhMZL7HyL zppDtV|7nWJbC3=E5nBvs&G1N}N96kN$lPVfaEQac$%;Xy>kYWd5sS#__PX&1cvcga zJi!V+02Tgi+mRx0-=``?t2%?aMnrvk0U2uqW`fn#RbZ7S5hi-CkROEzXP_{84BQDg zC@L+5@@o<#+f$G@{D8cG>gf;I1Wv$x6zl!$@AwDEjv%u{{Jm{Ezvcd{9}I6EbW4cc z73PFO1IAV3DogH1OVT5aD2?Vn#eESd{_u9>e_2 z`&oB`ihBX4k7jE#;0 zUn7rTe6V@ALZGfU=`Fr8?!1tn@-fqUbs=kbQm62A>R`Hg#(Ss0!^)xQ7-vaC;?K9G z-L$2|9$b=Adbi!?>J6{WVy|k;F0FK(Tkmp5(uMq`XFsE*JkD-fUO3O|DWPkYQXhV6 zT=G|o6gKu;pg%eN{deiYXCzPHYx(Wf3pJ^yU%BhSgm|~K%zG~X?Bd|JG{ePS?X;4{ zsGGp+&p=N2PmIVnpD8VK*f;msy59M>{m*wt{e1qAdcL6@optdMmh`Xnb_LdH%F7!+uAX~lYjkeZm)p5C%<9?UKpNwatABE{|{VPeRYuNZ5 zmFl7u(re?mQrBwUS?pF`p213GfxfPaRME6B6)gvM# z7+aT{94E7A3|?QK87i6U2sI;a^;x=^>|?Y+WRq1WeKBS@_ILD=NQ$@oF&9)6_S3-( z$E*Invv-fDBrYcZQB~6#O*Ed$n#sP(ofE2i>en=1(_Q;eYWt5VDZd;Z**~;N%UD41 z3CRST!FS9(P9Pn?zQB<=t{R=eh=tU^aMv;qJ^r?X8Vll1>r}-v&80!?o=EpS@5YrMTiV8(?}OC zr)q=OgKwI_sNNfeZ-=&nII^4dVbUM`oj(W-h~2(*%N>~U0j;8QQUD2zhF0Jld8P66&B8{*`xPEp1Day{$tP)TMfrQ-wItg_9iY3;2;O~t4IW|I6%KEqnS?k)5TZR2wd zC!(BDBE$90^c+gBEbt^FF6Pz;>>n#CHh^;vxur!3sLurmeC>>3(?`(W4(C{4L~c|7 z796D43q-~T((3$Z!2devKMkL}U*-s1W{}?F;;S%BMScH?u0~^P5}G{i#@L1h zobwR7p)tFfvH~toYk>7gdRG7NYLiPg_tpPtaE4F({~3I-hZuGiXa09sM`%!0$$zeC z`WpmlG56u^#Tf#OCfpF@1!>5ydzk;@mj$OP)_={r|Cj6kfBlDEjUpH|p_{m}q>0|# zG7Qj}z|S7RoJK0wP_foj;V_B(cU}~JXD)eVE}h8!xA}=QEO7H=PZ-4l12;J5k`^DT zR_<{a-u~v#m?+KWbXSrr)SOeHJ!@e!d6goMGa{F>q-2O&c35ukZP3S^&uIlWJauh| z2+mYYZ=XMEFVq$K_@-j5q=c-bWVOliTA2A8=<7(hr6%ws%~CjBNNre9jVYvsI;6m+ z)k`>FRc=_&r0WJ2#xl|WnvMT23Uh1+hKyoD_-tXlqFLm36q_iwOnCRMPnR=iV7ogE zp7KSznhX@Sq%vmnfRWD^O(Or@C_IQU+i6~G=lDioAlk(;=xr>c#$bLfp4fWMa&-On z31`G-iP92Va>bGo1ugP0x$slEV;vai5>gVush!KIt*3{lr?-29Vf)?=L;hcE8@X4^ zIVCxya(eO;Bx=LUcw4=Tyr_-<7rrZEsJ#bxaW8nq6Udrl?> z_K{5J=jU~?iI+Vw#tUL185h^RGp}MDt==_%!+=>RwCxr2=@72e2Y#auePdQx*zQpL zSz`(15w>QlQp?iB!;gz)M6tw)Y)A6k@Ktp|566DFKm3=HG@eaOpofE$b`AS_n4D8^ z+6!7c(j$%XyV5F@IWk`_waMcZ=?`Bu$>U7R`bp}}G1r+-n8(XF`WZvQb5Bdjg>*^pbTW{AxSBnZ zHBOdO!li?v`u zZ(y~&&ShQd%X~Q=^zdweo(>1zxTE z@%#<(Mc9V{xe)U-Uv>X{Z||V0j*}Hn;*&K*veO%g*7n0(;_GVc_Qb^;U}a)b6rp&% zSBP_NBof0G$Z0;uVHr=4-gvx{x_++7L!of%8oI23=Rxy(?VhACcHLD34r8j>3((ty~|} zm3KG1GY4+;=^k=^fAYL|m262ulhDeGR^>+bw%xg^f(^@^4-RrPKbEkECKx{iwA6L> zPU>e*qa)a_n4i6>&s87}h`)!gd^1)*n3J7<4)q7~LjEoKt%hp&G)Y>CZBs>h!S9ny z^Ea}jG+_fu91$Vr3V(Yw_1N$*H-vc^FpH4622w;2W0RCWo2KZ=lRi^-ojtjvf?X25 zDjL1i7(L%mp>)+bBIqU=R-l<+TT$?Vm)EUK%Xy*gcJ1f>xKwtB59h@2*BTRVnb>~l zwDv7t>HOCD*oFzVk%dU-EoJdmAruQrz?oJ5S{U^N1wnK!2~Hp-mh81vEnU*|&WbRK zvmeOwp@0U8fEh+9bu-(b4E(qLJYttHcNjXE+g<56k_kM2$z08+HXeI5pXf8TKi8|A zx#VfXT`m6i`@QkVni@^=FX2A~rv-P9@QvLgMBcVi;cs5zj%6H7{Cux6tpHn=u{5#M zvgQ_&Ufi(Y1bNZjNGr-V3X+umfWblACv>-iKhcYwd{1LbGNy3;5o^q7&?c_7^XT)p zIN6a3U*ZY=;g#4fpIjPP`j^XZh7#ddyQcslCeM%9`EjuMwF2YzS4MZAnLK>=KJYSA zi1GzR8WL5ypnoH~3*X2nr}jW2cIHpt)gCFz21+g>xybQ?*A_B%5!EB=ffRDHo{o0( zY|^pXS@PDwRiEbI?6#qgIWjdkz#m$$LVdR_OWv686&E|T{aTkA6M5qeEDGz~`}mEK zmv4SLqgzJa1-gs}>3vUh=$~^NDc2fSru^B}?zThUtaa6jXZOsxbY+Fk6E#v%ANaEe zymqKU%@wE#UC8qJEQtf2tB6wPHq`Mv^jH*&kL8yf7;_)vM-8~nOTPU14OeZLs6&%n zF+AL^v8q|daA=TlaITHCsbWp8gbV3>b0bBT^DSpYceqtOMsbh=lP;H!!r9e)xgsWu z=7^TUAbK+GrMdtqiolUJUNeQHcbHU4T!))K81SLBDv=Q{gcNxbDvcsps|y2)=G?<1 zrUlcPw!ug0-MHSTX(q#t$nN?_*Vy0PeZ&4v_00zl<`MU(1m>crI;sB1ydYnK|2)nJ zvlyY^KB6%nbYyjciRZ+wBv%K+SNfmI0?Hq_-|h@@;pC6SN@uFSRkqh}^h+q=FxgEU zrC}gmRa7e`EJ<@({u(F`Lm=~WIrACDvt(=oEg{r#a_zhFt91yfp$FcuXRyg$OVc6` z&G|kle@554UyUq!B%I~J`S*s5=%5AtExMQYXvL!C^)yW;>v}F27bQYIY;)AB#R}wy zCT@{_;0V0xph$PdJf_8aSUGevwlq6FA&}%!>iL2Lqr@Fz%3OO(CJs(2;pcavhhT9H z-76+6u)&di(;pTgXI2=ypqc;uXGw{zVeE53y+5+seK`>#>6t37HlY^zjR?)emYLaJ<*G-6?or%s zuZTtJAXN+wz_EA~s#32MTe!Gz>9ae1UzNk;bCZ{Jgl(JHvdY(SChJVbTCm_18H&Zsyhku9IT?^uadA(YWte=Xs01_F zd7||{&5bjFo0`JVYo%|#Ao=9>v^BB8(bFQWquhqR+t@S|iORVyMXwy~@+lY2@r@&| zV%+8O)?3h6kl&J&PS|!cSz?8~Ki|2GYP_+d_t0IfWUfR<4%_O1W;i(k_azJ&f0$_$ zKoP8ZU4?+@=JotknjgU@cZun%)kF2=^2c}xX`JBf^*o+By zT9LV&5xN@_5Vtqi(h9sYD{V?D*65z++rT)pl9DgBO#eZm{Z9cS38bR*riKKCQ0|S5 zv^1@7chb1X-Lc1ui*z}pUq&C&cYSc^tD~Q+Q{ft~n->*+c}9QJ2& z<5|2t*9GmyJ>lqH?Rv+3vAL-?SYU1S5maIumgAcA(17b&*Z>D@|4UOOiMC^WifF$p zsS>7t^M&BLXSpv0T~~BzaZ{Dy?AUV_sYgYDSg-tu6|crEVp;T>SQcm7h_ea*Ym;@z zThXW}r43sB^~2VV*yT{$Cu(MC*Nx%1Wc1?p#XwmtXPM-BV( z|F|Su7$hggspz6dlJHLzHVAIVaL+V3WDSEgVa@WXi~0e=ibXn-kBUyw(s#W&Dc0&F zvk>z()=M(HEkR86pkLz|zqWXKyo?!az$XUsg16(n?BKLpjZIAdR# zx(^E)h;`D_-tQRf!Iu3+mdMl2e93bN&b^upf53S(oiD2rk=7AQugrS~33b(HoGO7G zu?_ca$wQs{>U@~xr=rVc_@R@t7gL08VOc^^OOc1eC+j%6xxj|&UK`5p?RAIPo1oli zzKFN6^g$0iy`B2{(Pl_Vi5B^X`b$wWP~W7%*~}hIj(>H+rRftV|3w?N6Aw3@sIypN zFGHwvIqS^>^+9dF&-EB2mzUKR+K8U^R$D&GC|k>1T8W-tZGtz4sbF(;+*8Lf8LR$v zUpRgL%$_tvEAeK3en|-_nJ4Q%?!jenE|(xvze4_+Gk(0p22=mmJo#Q>_4OJgI~a)d zRRuBTzsPs93k``0LPsR>}B5K2WdOd|a*03L7 z2Uiehv`#I^;^o+AkuD@;#tp*TS4MHME-!DAM=CvbMHl{K?sDeo`@U%CBGy;Oi$0v! zXL0;Y8vjjsi7O=3H*L}7zQwfD;LS(1pV9&gSKex{a(4yEKB<*KZY7v;L7)AeQ+QuB zm)x@4KjsH4qVrwrZX!M$R#TvY0KyUp>q=UID~V`4LD=u3HWdr~Q0NVi!}{wA2j#!dTh41F z%XEH_<>pTeXe!Vm_sWD>DRNO1qEUepH#<#3e+myMGGVy3Q z3@Ry!V9w>dt;SMOl209Go>p$5Z*hqIF;^9d8DfmrWQ15I<>S%WwPP-%SZ$k)6-sG= zP4Bm%Db-=JchP0Uu-(12A+bcGb$kIM|Mmtu#&q~p6+h}SdVuyYb7`WkS24Co;mul8 zlVhhNZq!<)$OnFgd;I&^9vp(8fDHW1CG91lf!zi&T91c# zL4&)JW9~&(0!Ne(Ql<&9Gk{f+5sKr??iA>eX59JS$HtcK^HZoH7bFzY z_mU@sV$*}zL-Ma`1^__hV!*@E49$1Nw+@OYYRC%$3Srr(bt^GF z{)N#GkAQ+0>vECh(^{D@Vvbv`!Pwf^(emp*=}H1Ri@#+q;Rb4m2TllkXL6`wH_I`?Ip~xRBVU-p0ss|k&x#roK1+LRQOT}+A&JP za)ScPeAI>^Bf0mxo7@m+=SO9X1liD}-Vfu(9VEChZ7}BEI3S2j!NG#ff3q_nKhryWvk^i`$|VYhSM8RNimKmPs{W zlDdCa`cOaIh_B|kXP$P+l(P7TmZ^7vV?A7$E->DV>k=N$;6s(yXS+tds(;i8&EJrO zzK5)6*R#me2fLhd-yX@vyF`?81j>Gsf5VvA3X zuT@3{gQx?lketF_c6A{ivv@d&W8dk(9-?Dk)sD&da*{DY;xe6xzRWvaj0XXne6oK| z0!zGHg0V9n)y{hQu|LuN!n(Rr(A7Vq@b-Gf8_r=zxM6Fk68H3SJxEs1cc70-b#V<0F^t_*Pu}(Q3kr;U2&8FBEudCpx`Cyq zCsv!r9=iZfCJ6%`dB=P8S(hmLlySJd!s3srk-P0Rt3w`rxS90}B za^E87$6r-03r>uopQ5pWp_9hz2vDPY0l<`L$d(BuG%z2wd}R1FGsW3tKyV$fY!Ar2 zs8_CvT7qhMYd0g9)qU0TWg!%CvLKI}#SZ7|A>msTFRxf%Z*9#&kx=^5QD%zMN|_qlHnqktr=FW%tEpKqG)yxxbZImu6f(>fFKyezbuPj5aOFPyeo3>xyFyx# z$tv@i$r%6rt`Cjdm`U7Qk}3Ie#&&vRf*HM>TDRq=j@1@Toh)Xgpzafa7#R_!Lmo6f zzI01fq#}LQ)AY{w?n7U6y+dSkwZVgEt|IMgS4y$vYQ{cOVhq}h@5)fwOX?JrY5QoCLg)wgnTc zw46@uyFO#?Jbk%;vHPlk>ST`7cj;aS5F({;EEk#Ey&Ezw`gG6r#?t{;hsj&0X+br$ zVS&`wHZ%*uuXB}$-8OL4WbbiiW5E&ut6t>R{bx2Ym~ktZ5qX8;PW-1jhh8UkjiC94 zslQ`d69>gghSaUUDTwQN%E>*k+hQ-=XE4%q-8QXc z4JXO`-9WNKC4iy}C{MZQpA*I!&JMD4ecd1<6?d&z~^nBL35$U{rOM;@ckJbj3VC(m19@HjjBn&L_2Tgb92O|95R#yyNft}i~U$oy1sRXVy9hG6L4ZF93c^n$= z@i32zN=lF&N@iAGkjp8X?R?dG0n`wgO$b$7GhA2=SyOLQI zI$Y5UCwKIWd6(LddJ5=KT>?cSFKxWH!#rbJki6WbY~RQBEl!F-Yipc(fWW1IDxG_& zxU!57VFtTX4f!ivH60-}5!AGMKmW*L=^MeY%@evetR9vDld#L1=wXjszk-8xL^{s< zX*jWTR+NuIVJcZK$)Gtos=P>))PX%3(O@H0<6%_kD3&QZMP;KozQP-(lKbD+9^fn z7Ga#bpP(y)5CF)J7@TCi@7eg2;1@xr(mMIh`6f=o8d@%s#N>{-5r}hJgvAj*av1xg&AiN`WVLvjF0e--z z2%z(O<>h!4LeJ9Q`Uqf8x=Zy{Sr?SC1qG0YQq>zjD`v509raaO#l%Q@W#8Ny%lmA9GKPyfAo^Ol5c4-0-qws7f@I*i8k3;^2%L1w6vAkdIc1?;z8ykevn!DPM$+frO^At;6bAYHpU>^Q0AxHG&4NesF}tu z|B=y(Nrj|Pmzf*$O0597YY#;`;<(%Fh#(mJUEklV?#sn z)n&$fR$sQ2DMOi#+LoY;7JAPQoS${rB3>ay*W|)|ysDOd#OA;z51c(Q<+Fmay03P;PTscN*}ks|&|g0G z>YDCG#$}8>VODC%fxL*)oh)TuQn%(oclm@tplnbb{0cehv->hI&R%8`79dlOn12@Q zGP=;ED0;L2DlGx{(GhBaoJK&ha?#KGJb1mD`_ta43LNqDt> zR#csbf({A%(I*Xy!Erw5dB#63n{*i24|AGvX%Ee>tu_sKPy~(039r3j-**)#){-Yz z{8g4+#M%_tO-~z(f*SKbYl5Y2f(JU!-AgL03oeuv0tqh|`(=J8(Shq-S}3;ga;Cu*m4nsR-Ly!Su&X+;~z->wJC+kaRypRVw@2WJLcn8odnCU6wBDQ zgNXOO-={Ck-ob%LjP=IYuU!VgFr|BtJ*P2TJjmbDe z0whr%?6-WR-vjpNUH$#HNnKJJ9T&Bm-q(7B5zrRMbFG{AMo6}Twe8UyFilXSmz2<8 z1)49X6^>d=J&xCdb97Aw+Xpmj$lA^U5r=X{u+-#pvXLWXv%yt1a_yGGfqW+?p9P7(=pO zjkn6QVe2XHC;CXs_pLSkX=2BgPxj`3BRzzi3^{Ve5vO)%V|c~ZWKIQPh3crT6eMhg zD^&OHRp#&*!B=q)ngkiX@9}s&-jnS-%}R+H@e7e5uf z9U&K|iluRQz5#Iz8v3sboFX^Ovc>%Zo9R95mdWEVSHaDfpZeAxxWOMdv3ORu(e*CaM zRW)ZYZ(QoiAyDQURey)U+G`Ll(E!DWvyxRjQ9gBVD+9bol4-h^7Y}m?BkhlMn&q|3 zrK5`UF;q!OFAG&f?y9)B3c= zc+Ps6-l3jw_l)M4I}YQa$cBG0jfXZ@=EY-b7_P2Zt5~B6lda7#85?z%9}_YR3yMQ^ zeA0*L(hciH%#{9b$F2-8leIL!Z5pN$GtJ){3sH`|0gE2OVxG z;K&upddo2}j+tP^lEe140OZr`1~@RH}jjUTBylt6lv(YbpR+>%6-_@MKa zZb8-$xc&L?7W(jUEvZYNXPn1PFb57g@Fw)cY6L}A3>6|f1Y;1v62!6ts$Nyv^m^c> z@N)pzTjwAFAh6Ej7gVzDZkB=VmtOz@dRLjhvzU-(-)hU(tg~GiL6)-RYf=M*@@iwd z+Dtzx;M^ROAUF4{qPmQ-=s0fk1iVjeN_z43z+v(Xxc7u1P@Kto`w1Wv98%Q|LJRd+?#eUi zs;CSLHs;WNOg8t2NWTQTwEVqvU;?IR5Y9{yUc80Dh)ugY{c02NcJt%93PQZskdfRJ zigE&-K}_H%7_Ko^Cv)ZmCa}sN>*{quH379?$JZq#Vdly2O1W-J4Q|?>_^|)M^(Em`sZwa0 z;Ognocm3XCUFDDuW=tzsZBp++$21wwv~lx11VYJ+7!M~lBsgxlm|D@?SV;(2 zmD^doeYTFsuYxXH(hzN?&p#`QU<>zbWkfi7FyFi`hdy+l-j@`3KnPN0!HU;)qR5am z_RsuT%ipA?dmC8bWaJ^YobN}N_=!||Olauxg2sCun0s4WQzIx+j}TFP5z)!5o^N;1!<^EnnZOuU6pw{b^Z+i4dm*~;o26~ttS8^760j1W`&B7dtU0>~ zL}CHpk;!7uq2%NZ+yR4(D-`rZ+JpeX0Bw4aVwqdVD}CmF?Xn5Se-FWh7+e%{qDR*q zM(o>nPmL?yy?YlL6el^L#wxPQ4HbR1M#pHv93vao(JTZ?3dJ1a!PM6p(J+>M5z`r- zkp-;#gYFKQMIDF9+-t0p&n06dQ6NQ<$1CLA!;{Cv&?VxbSiq-|&xvI7uwXw1Q1X23-aKC7`M(oIq&pS>I*bf{69>+bzIZ$7e0)X#3Todl#S6TB`Gj! zpaV8K1*Ahsky1JZ#u%Vp=b+ zKy2z^Am1_Gbt&_5RL_4I#J3dc$Y2`em>&%55@Zv|0p~b|MA^-{LI~{JsLNcPq*PSM z`M%X>sBgWMXY+cG7D(2>5+SH|9LTv|wgjs-R8GLtEw_#~!{1F!w@9VW()@_diF}qD z#s$0!P@^20lWxBLMZXPTnvR_aDK@XScV|&7rsZG_b6M%LRdM30X9uCnDVmM38{uRk z%hwi6*(s>bwkxB@fK7iOkj}?CZ)WL%L?uF4RGl8I0IT(pusolY7$@7nm;eBi6M89F zoFI?{(Up4e&@PMrv604?_GbcsQPF2!aAlSb0NWJN{B%jT88j&m#9sl8;%%0MKwr-Z z@|mg>gT&$25~Mg-fN^g!Pda823OV$@M_-*$gzG`;-ernFsCryLxacWiLCqY>K|&q5 zzyvUpPRzfr<)KUZOUHZ}N_zA`&*O3W6I&Onvf==>0abdo<&x zlg+xk#{<R;c&1 zfFivxMg#Zy_WZyBM~9u2fQcoSzxxf&k0WJVxfa53z12=oOZSuxvdL-`e!H)fd1-wj zFA|WaW%(cR$V@35G|~L2>qRMd?X&gU`hmI^^}hp{?HoIW84daUs1&NovaZyU1nI;> zbtFol!#|#X{|2yy^S8>7*$K}$)>gIyT^{ghu>R(>uyi^03-1sBXdgYe@s!!y!(n_c zL-}6@YEFQI-YWGn5a^u>7>VYQvEm%t3 z!4V1bs~b{@Xaf~8+o;{QXaGI~5O*01wZuYaNK4hg{ANOWviBXKt)Of)1SZQxs=Tqv zM+;W@k%N|DU!o04#o{W*R3}*Qn#4H%{Ni+!l4Xx)eSO7F1mKqD#}4^nAa+i6#sG82 zD=Y(!waP&ggpdW0ftnP({n#bbe#Za}@_A^i#Bg^bs2|04-)euMVS~VKp801;bEXji z>6#GIIlm=@C|F;3>G@j3B};pK@`p&D@JM|u|I2cFVWSkF!?9Gr42C``+0e)KZ58fX z#^n5;pT3GVnsC{@QdUcHkRfYOAP4uO4yA_Pf-^*dhnIoNy~S7J1_KldO01_*)x68;l8=Z)!i#4WVii!DdV!L!`@6pn&mW(!pa+;K1$yw}T-Q44HYmn#_g#!P zT>ngt4PZ2c57)!`dK~)xF~EA&D5!OF2j6IA1ZBT`*9y{ahkS#lcE59zF3?wDzZ8Qr zkrq!)#r;QzMK3$0Vt9fb_&4x2yYk#YoYljUym>}wF*6pHRh|Dr+9nGNWDdd{gMC5y z6j!LcFHNnZJEkp0I8%jS1loD858!z$%R@j2^rw3N~bgW_WeAci(w4k$W5AIv>qT?vbH<8lAEo zwPg!S%je4+6qD95&b1z0JZv>$!*z^0EVQw*kT=lD4N`>1|Tnu?} zk@2%W8htF9wz>DsadeRU&)XjXtJ9TMu^*i~ki&97Pq7I^z80#1p%z-7oKj|rbu)EV z7*-B-G+qHOac+}oq#(AHRm}7*U&q?>s>r75xpU9h z9bf#qw*D4ugs^b|A5A-?y8EJYW%@R#IDqpW3*;mB7BI#MrhR({;P2~#X~Qk}ugldt zKxGWlU3nd-yLc@83w`lp@5)gLiZc0F=URd3L^IVtuYpHU_E`c2OXCjEj`c9P>%6=a zLKXvUvPI2T?XwklfWuWUL376-8(>-U8C{?C8m?O3)FFnFohmR4OO%~-Kq``FC&+Q0 zjLH6US{>3pA86YFP_>)_UKGZ>u2{jF3b;GRi3H-MHzk4WV zACMyzdDLgEsermGFZYqu(!st@|1lLgB9+;f-MnA2A)dx#{~wkG?`#&bT-FBay1Lnf zQyp-R+RrKb&CWqH4A~w&LF19VmgU2a@I(A^7=D^(|RDe`V zf&1k_Yv0$g40(Nt*fYSxnRx?qg12s&X31=;0$nz_2;2x#-Ip-C6jI?1Rusm(?!SL+ zO@>Za`lgCll1tb17l4oY#@^{5Mh*HJsQS%>C(Vu9A~mM5V(YU`Y(I^-Vqnoch_--sHL}4TMh~!G0Fe z>BK-26f7SV22h9wOv0Q4k^f4X8ehq|Gi~?2%sT770wxo(~0ifN*Uj_%#c%zoF2~L(5U0(ZmDpjt@k^rNH4QAI)c_LOsf| zz6qTJ51yijG0l(-7e+J6RXGq+vZ2aEBdi)2OgkvIk9j91QTHwq7$6Ai7F0MiVWVg5 z@F?zC4Ik}9aJb049$TO+czBWrWCV;u=h}s$@!gkZOz|7t9sayYb@gf0TWepH#zEdR z`+$nVLXdnxoecYbVisGl5W)`52zl|f3toY*BBb&n_!Y+ld&y>J7Sxc^4w@V*)iMi5 zRMIM7NhZf`mX?-tGok#Q?qKuh8DMV#j}m`k*Cy%m+!quv3+G5;wf6{UBN~kgV6f?6 zsev9ZVCjHF>*jqj?Rw2Ni3I_{*cr?yIwq5I^i=W<9R}5C~);81q$EW1wj5`SMBT97XLF;Iq!iH1JSg za@Xkn73B$>c9(+gGwH?AC&1-RK27npF>FzId#rKs%T#r3{Yp=dH;)6c;@uJmKL1|e zA#w$e43ae~z7Zs6rutM4f(M_YaF0b0Hm~pY2(5KO~qy1db_Q z?mLgq(J>A?0*4tZ*5hyHZvL%GtlR7ATRT|f7i%Ah}_xrzt(!*fUvH;^Zb*S71UNF`fbZwn z=+~Ue$S_cl1{=A_Z}K-W7%eZj=Rd5OhAT%CAo68TDvpPq3zD7m=NnW5@)fGM@~rGl0SVW3eA68w zbN0Dz7Q+Wz|5`=FWM}C`=_uj*e*!El&rvFvI1a1F)}7R|AY^(9uEHEcC@9v!DVNnj z#15uaBpB1#i`vLg>->d7_dm$p1Zl|=#^r>OPL}z0^Mr3on1krjFYk4B-kN|FF6sy( zsEEigr283g0jx$T1Eahxu(GK-Abe6A73hl5NKtiX-rBvUDOdMJ=(JL8-__KXcWJDe%I%{`;VB4%>g@OflJ3g)g`bhJ98EAbu7d z`#=x3ktP$83|aC4-hPU zIEL!GD`GB3foC1TPM6e|z~>81egaaY?#9anv59)FQVc%8F-pYhsnf_9>Rh}zaj>jW zQZJEK;X`g2s}O@0TD;FVOL1sVzR;pcK6UneO-_x8P{&R){e=A*GQ3rWLi^IzC+>Bw zfG^FY5jgo|2dKQs-{2_i@-Y<< zGtSWI7GM9IhKSRybOjK)4^U^&kd1bNgFxT+)!en{>sio@c)Ki_Nhy+(+;wNxKXjYc z43K~D>CRI*UL;0+dDOI{1{$Zz9LQIFOHslkOXwx)l2N82Rh8yyHzkOg%-#Sa zUA=|O`m&MqDXH9~j;SE;D{Tg^`v3Wm7*f2HJ^xPQEuD`qRUtYK)4W;W;cQa1Sy^QG z0xSI*-nY^10%KsUtN$Ec(kKKr2b-CYpX&Eu;Rf=G{gO9Efbe#mEejb3RIHeE4*DB-LtWPs_MIeKfbqLG6r07%xwm; zU1~y{?(UGq2d-r=AQt8v699wUWDi(W=*|pXHn+rBa=^s|Mh-NX!5(Ana8T!6p47J3 z?LJyDvuvl zZ!?B@rM~J<4chtq!XYy&(fs|mvu(0Yg6@Ww-25GK`8BMqi51CGfv%z^KW=I{!ST*z zah!c$511d+7r1Pe%x^o@u7R*+aghbWfmxx?-lUh{*fZ(jI$5bqJjW{Fyy~r#RKsqS zF4k7BPj61RQ#7O})_-H&f2aJs`TPE%*LjP+*a4DE)w!R3*>fV7M-S=WJc-O#mR$d1 z{M7?%?C)ez&`kZ9O&=R!&S`Zvq}hwE$jKYk=dqqzvcz`DXS%pKrVYiJ8}VR0C$9+d z-Mh(5$+FM+SEJ<@3@Awq1z{2e3F|ys)^8DMU0o7&G#$i~ce=v<#-p6{LQnk%`|hTl zQxKK%2v_qD0DT3y7flai`gM4V-nI|Pi2fxCjIR4j)~Bd&yw7)LN5bWL!oDUTDcyYu zNBZ-j?{x}f>Weqy_5J(aj0q~~{2psk)H!eT0CSshF)6dSB+m5pP;Ns=L$VH(m*(T* zw3CBqrpVeNb2Lg}GQ6SQg(V?vg%J(~Sk&P07EZd$>7SMU1>RFX=avREf*JAU`7`Bm zjh;6fC6XGq@5k)14a&JG2x~TuC>t+AGf5%~^6!)jGrU=~p0dXuYf|sY+d8(=0_t%)FpSQ7w$jn1+C#4SPPj^syT1|duB0G|FM0p+z3ah zNv>t=2Im`bm#@0QlDrEJjoG7KR3~1)csmmHW}12~w|aV7QAf0+LTtY_N;s?hn>ROg zEN)ZdZhN$^czPNrskP@o+CK$tzfsY}s|%y;I}eKY9;bJc51FlvT>J6OgbY}#2~3IR zEJ`nxwfkrIZjIn00>-CR`R6Boj-;lL-(DfE`5CUmkr?^9TJx&fExx33cEEMrH%G{H zK8LH3G9y-s#J80Q;}~wo`I+mSw$y1BuOg-bplW3ar{9NaAH~?&=m4nrr3_Gn$b-(G z!Vxz&cS+N*N@ui}{_S?4CjeND)EJ^ksNw1~1VyLzL~2GBK^~(h-aqNHH#FFAhQk7*w|Ll{%)qQ)^U$-^9pj_B>#hS4BI#)ccNRiK3>m)4Hq|e!Kumkv|dYDbe=8nugVSJS2yvOVS)l9BhT?~i-{q{ z@hXQJE=xB7K@!bXIx+i{(B6j640UHSlEEy>s)2$z{pR`F-JL4} zSE`!R^^jp#qZ*r+O?_ftmXADmEr%6BludN)3imj^_svGP!M({UNE{hCtChU2? zcV8fwJT0iMgI<=aYB5rEDisat=Gzne?KDg>@iSr`fiDa`ql}@ZvM*B19`~4m+Y~L~ z0~o)-^HBRys_kM?S%Z$sJ%``I_Pa=ZUf#QX)wk$#TcpRH{8K9esztH4lv zv}3AR_`*AxEZqZ}YCretU{88m(P0@bHu_BJC%Ho0aZ!)vqJFkN;EotxM0=60hzKoB zralpQnr0^@ZDZn%laZ5jn5^uVkQupFhw!uC6x=7x(i*bTNmQ<&N3 zzr?pBR4TpPeqq@JnMgD0>G9x{@$`ktH4q#?x>-@ak#L$<5{+8({niqSW*o+q6G|nG zU9Ok1AY3q`nTZ8#P)kOCMp}k-Q)cIM3*u!{sGWGve5-}mHP6&5B9+8iAqxxX0H%xo z@o2zIP3ToUD(&UD*P_>;8wnI2m)GDv5suh?7V22<0`GpH)pBiG_%JYI+~n3~%^fr; zSlR-%3x#-8^Skym)-)Bo{`0Z!JDze1jfh?530WCwxmnR$hzb$HYQR`Iw!t#O__%_4 z_#d|QWoHWQ|ER(B2RdHFr$yp!Ke}3>V_$j;FZNteG9h_xX83|?P+MwI$7P*ujW_q2 zC2pl;4@zu)mr`eEgDWPv1X#tKE~b7^eezTh{xUA-lYWz5Y}K(fme?k&*FkdwxmUoq zMI857*YYm-tYXp=f)krdLa4PqqL07(VY|mk(MI@ToS3FA=M6qngeRfdFK3SbC0Dl9 z8h`4_8oR1seRK0Y({fuEm-Q76%`R;N0h+8decGEcEL!DD;ft+OqtozY_;C|)OzZHW z*m$j6dy>Y83rr6SRQ5X&9sUOc3w|Abyubl?k{tkfK!V`YAJL@+zo`?~%`5v>A6hX; zZZwCMWU8;I@V)jdgL|sp&-JHV@WY&nVEMXVf0j4>wrElJMf3*2;YzB(cXh6}W+C~O zmh12C>IEG3lbuC~N+&6AQ*Lmxy9fUwO76HK9v=^a{3kJl9w)Ng$BqiNMp6p)H&}Q6NV%twos?HO8s1G_J$a?149|964FLaBBgd8+}Q#ZPmvmsH57OFKOsY zVo#6SC?a>Y+UtPrT2(52mE3!yjz#zOXV=e47~HRw){W(>Xs)#FEzmLQPrLlu67R^c zC2wXSn$L_-ViXBlUwgh2Jp-(Pi)piG~hqEi;TD1%=LYT~g#*6+Jr^iVE$N zgrOAzoSjh8$c|=$kr!=d9pj+bYn$M)CmFyakOSpv_;r@HOAe0njFb#b%&U|<&r1_? zPrclFZ97uk#|ZI@BcoS5#hcGaFHO>~3SRg){z1k*uT5?zcIcbdxk1qtHx);>MEo43 zuRz-8e0Q89WG;p<$5{9yRgAIZ43yH33m9RQm&V=PMPBJF$Pkl0*oo@#W#lLer@y{H5(sFOG9u{(ch6m4@QuJEBe4r$ z_KOh7jY2=D200+D@(}etNPko)|DopzZsMSJE0*V2xXi?}RUWQZ!3_np5T|2-#hsk7 zEigHi878Nlzi8)+%ij@`;e5qO;Oi&+UAO%mSR(l~?g0Qugfo!&li*!{ayk`is`J`}+DiI`S@Y|4 zX&KYdTMd{;#cacOBK@)OxT<(o*OY6;TA@zdf9e1 zPI+w=QX^_%0Xmc&Qy?bH$s@-4W+WdNGsla)u`}sg6Bigcj-EIW2iVg8&X!2=AWK8y zV>k5Oja+X9YlwcDlxc|MHD;=F`rvt%b#y3aKfMQ2hg9Lp-DJK!dn85SMz2$iWIoDq zo+Qx*_=PcGCwJrcnTp@LOJBc`{dOk-M_^POq1xL@325Xreo7mSB^pz{3Dx|R2Cvfd zXanJ;oJ68LQSXc2mz7G^ntFc@tULx|oxlRLJpaRl4N>B#Q%kNmBkiyF2UP2Sz#P>( zo_TzJj~I-k@dhhToZuPMYLuldA$%juVEAhdjV_N&wuNG{&YLC8miT4H8zfT3x}ats zuOPVur2UshJA+=8V@)2N%Wucs-wkub-BnN)TW@6CB6%_x8LP1970r#lQLuf&cT`p_ zUmq`6cIHRI1u6UqqDL4mNK8Is^+B@7LbBVr|5L z!&sb5p2?NX_{n5i6PNoe8pg&wk8XV8{2C`#k&<+Bf2Y^6y{s-gwbW)``8N8``6 zZ6$_T)!hWTh0Soo9-8MVk8Y*>e2}Bb=Kr;NSAOLcHCSjMa6E6mdU|FdbAvwIbKUwx zXIcO-Tx*)^>@fv?DFe46KsEP{UNtA-R2A#e8v2r1hUNKoF1@%bel`1FliaLrQ}7Oe zfVzEqrApETdM|}(uRdxK+*&wRH?IN`>;E~F-?yP7ad7-Bsx+{DtT|byqofXRMm~2i<9c~@=q0$w~E&XtYsbpLKwEDbPr^jpWA0**-@K`0kIc}$E9?99{&EWa?`|f_^d-=I*9=&UF71l}zcG(PhcD z(ih&s=pnu{kMUF7T59X+GDFlRVVtH~2$lOXk2Iw>&IEHk0$q1$2sVhDGhkRJGwt() z&oYh(h~gL@aCaS04E|-uZo1%S&|+2zl=l-XCa~TL!9uJAqSwnjynUKQs4-_ObkRxi zNJpJARZ zpGO{@rSZM58V|kglndSa1tl{&lyGWDqxZ#PX|r9cN;dGHcKAVWHUV0-H;%TCPOSu_ z(DR1IQ5_TT=R)l#=Bz+*BF0EINfzo>=d#X2ZQ{fC^4DEN@tTmlzolNfh+O8XBCaroeO@+2@b+nwA4CVtgf45Ui zhPTXvDY|~iBTXo}G#&YCHVVjD%4A1jAMU_1XerjkAv*WYGaRzD)}2`w``_IMDZ_Do z;MRS7f~p=|4@Z0-9F2?Rh&wG9Mwff53Z%{y+bVn;^&Yty#@!a{S9cc2PP=gJ%~q&L zwRuoAIXo3CTLjmJ1h+C@e3 zUxJHH@sHZ(j}eGCeJnIB$+VV+=XSNK;&>x7sNvy8#Yb}A=sJrA@gr`tD7_2JR9}@C zlc-U29k1QTwe_#fbtU2UDwKA%ZuWr>s5-L>On5p(cgM%MEF5Cz3 zi>*Qcwbq~4eo-~%OkNFdsRk>mgnZzvgrRWpHapw1)w+b?!O6F8!PM^{Yj&gCv;7KoH*Q03An=hiEv)q8nI zW3jJJ5}c&btfC`Irn1#$w1}= z(nR2}irV(;21wzWBk48P5J0i+-j{OBW4Hp<2b($l9D>?Ir!em9>r>I#fw=5^@1-H5 zrex`w1P|wYeH*D%+s0oxa^LmK?VJ(fZ*1S2sG5m7);eV)hK85G4Yu+|;gxCIYK>V)Rc>xD?@TNf$*QAjL=Xaq9WyO!e>!zX(P) z|DkZCEe9IzUkEnGUQr7sLSe%XQ6RjfAkzHr+NU`!OCx0r60rJqG;W<>tRSj_J*%rH4Cn*635Z$2IHkWx#YHa#e2>4omyF@|#oR{ccaA z154_;@gZNR{=3^cKq<-N{&iRxnoGTPzv`6MpNF3+)rIF zjr$H3s>;SkK&tIHIZq?XiGD0~Q=3B`qB@lw3?~i5xrOrev>9rS{2(u1YW9 z=2ROi0ji&kvsqV$44Nb(zq82hoV_U=*dks~b|p+biCWk7qIxZN+I9P0!;V_J7>iBB zR(d{^a_)#Du-~=*&2!Nb+@=MQ-#jR)Ad0@QE?OfgmY4_T-$qm#kFQl&b?ss zP|n_UTt7K4-6wS@#G)dw@bLhyAT~E1%$y^fPu(o8P=)yKhSr&*GLQ1+UlbZI=_c_@ zKSc>7q0{1B(=n-oSQT^q+v~%Uw$!74ID2X-mXuG@W$4={9mNtidm4wb(XG7?HaW`0 z(j2TW;>#$Mtr(-Y(@-^ei?J0>(R6*PdbqosD8m1k=n6QO;#2pA7RQ4Y*KDD)P<^;U zHY5diK6@`IbGHi@2ViaS1oh&Pyg;#BHy8E7lnK3}KiX<~m^$`i zc6Qe7k?pVHea_Jh!c{cFsX-`mSt~os&*EM%g2D9Z5DnUsre7iali8>yUJG?&OMC3I zJmYf3mUgOwS;?k7^0ph9q5Bt3ha<$JcyT;@Q(Wg`QKJ}W;*a@sL}4I56r$lFo^|yN z_^p3wC1kNv^&(>MdL5?m2_8zX>l3zZIzX)e~WILSfFjx z@}!<1cjn9(idGe*pV;NGypzzoJ$C8r!)DfA*&k9H^i%WK?H!WIPaU<>^vf9=jEw>w1Gv*rAZTXjY*oeiJWg6Os?qveLH-6Gohh!*%g zJ15mUG*1o0!JBA2zk}l}eEhi$`}#n=_3sd$TG#1JI)m5R-+{Tt~o2_(-z~_FrisBGA zGk-{&$BeenpE%#4{neUrN0k9Yfzv^n1+pS-YUh}_9ypQOACP0g01ZzP6kPPid(F*UGKs{53O=uSeDUMNw z_;V*i;e}o1BQvSNvP2*q=^q>OpB&AordboV0$V;LZ-etNVV-RR0*=L7Mrj&!a5+)(7cx4J6%za?@{2VM)ZO zcLvSIyq}pGXu=WvG4TIHu%k~V_gZzFo<9C_T1gLQD=1&x;hLrkgmS&ezGLd-f3HHQ zKqu3-##%KJ1U6uV+idSl)L9A0UaqdjhA3k1w@7PIZ2A6OWQ~pmdc{07I7|B0uXvYT zf<)-2W(T4vuW>zKY4(wL4>QtGlKiQgqXCvE=EJY}DIKWd|ADO{(lEC8;ON&6KAmH# zqWgU4G+Zu);Gmg0vK9+Ovw02?KlvtTCc~(u1k9Ko0qh|CcOv#-Hy&87uDv>rJb)ZE6o)7N2=6X8g`3hB}_Fl@K6{_bAIeg9`fi zGmxr%h^|fQt+9V9Lf;&OpP}lzrxHs}-ec{OB7Or7Lf%9(Mnp%88W5W-8@;l?eG;aJ z&o+k2{CBTtd-O}C-izSq=xBfchmXxqt3ur3yP_U~t_(BANk|p?d>1t$w4%BSC1{B` z$GAGP_E_jd<@sx2i;GWN(>^NYK#5H}1s}L>ZG$Xt|zj?=#+lzPge2mqQshSY_-cpf8bOckH(*5gU zO}`aV4Lk|%&BS$NAgV3SM!UO%Cj->3$4wG*z>U z@f#$+*>(BiOAqi2{`lsx?rWWVdqTccmX_|OnY(X|8ffA!Ly=D0tXaL1ICo2{EHK~V zk?oP(FzmCN**J* zvl;V-Bs?`ejqs&y)fszyFqh8fgD~CfDY@Qbm?^X`8Zbt4lT%ntS`I9&2Lpb(9Wpb; zt#?et#Q4@L^l;AvetfGwuOa4#Ga(e#bBtxYuvH#rfM@$Q=EpQTeCb4=O=;cMcPR%8 zI>zS38Iv*(rJA(aIwQ!_RLGhmeRk?tzaBFU0$)3>4c4m~^`u5CAO zYiq+hJ2O*x%F=ly9pNznjhfF@wLoWArcl^3GpH~&PXg;Nm0<+|d9|RPX+=VxMA&i_sE;IM%=C#oZYX@8T77K&0VvR{_Ce}7haSK|#j ziSb+}06l%m0SNQTICJ{*Hx7-5kxZ{y^A!rcLsX^z12x@(`_N^=su*$l&4RDqYlH!l z5!>^)?32CRIoe&~D(Pkfm<3l9DY(H)ojAV#^P%70QSNOA$i)lUN7AsQ{@`1?Fzus%fMX^n9bTzieH>yv^231DqAr+*SxwB1KO)kX?5QXtofnZ`o$vz1xKnn2ad> zpSfOvM(`R73#4>47$JUWNIh^8&Ou-O)G4*M)=-jDxG>1_yqK>-4AG9S4VdgSTj3e1 z`1m=Co#TJtI8QeZL9F&g@$T%BDc+Z5M1XeG{^Od5YCz&uacu;@#&}$;F2P2QG+zAo zDE?TaH5o8Rb4_S1EKJ`gLBa@} zTXLC1qM_1~JIRVHhtluF^8Ojsg6IzZ#e^J zui||sMT`%x8F9_6f2$e#Ve%d>b5De%2s~2XuFGcn#Vh*T%OW5C0}XfiWPljz1XG2k?l8h#_T zZw_=lEyrg%PPula5k>TdXZ)iQqdpzzDHrDQdf~&sB(0uC{Rf;CVk@_h-NQQPEg8da z1c6ZyKy~>7$YI^d<+1p-3)&`hPEXA`c9(+5Kfgv*G0~v7JDDp>9Ea3pxoH1!vyvJN zLC}hos2$zPrH|}!TwA|i!Me1y>?Vy&z8>bNicHAUfE1~ur-_swC1@54rUa{94#`Bg*~ z)5QiF>|25CuN?bTvrn{8Uj-$@|K(5QYb}4lIK6ZluiIfb))l^d+4bXnk03tE&ggY| zHhOuttF5+1)esuB9P8MeLhm+$NS6v5fcVFt#XKPZqEjwk)3utNG^z!4atcamF7<=* z9}cOswTGC(*=0Irv~Qy2(UuY%wq||Awbj!IwK{NHdp`)ZFlY}1BZ6Ec#yTRyntNQr z1=YFT`i(?6ioLOX=oyaUXB!%&r9u0OiCiL$P~q1JMh%IVYPcCp=j77%J#YVi*wtIV z_0H4ND%E|eUIuh>-7xE(yqiOHgFgsO`_i>IYjM!t0?Qqueo6=&$*Z#Y&ble4FW&yF z_m!2&K(#fAq1n-zaq9;oT*Y4z4(nO2w_=DkELxD7Um;SxTBA?9RIw>96E-+yh&&y( z9a1xssgkfZ!`#BwZ!K;`Pd^oCpy^9c50nuOq3o4kJILWTM$s9Ei99tQNX}znTkid> ziMjAZ&)xF(+f2|EN<}U(rD1MV7SJSWi?gX0LXYGE9MnyT^+`hw1ViCOmdw?F3%9$o zAGzBG)v`sx^ycgL4>}K0Zt*IcNBq=s`4aNwrC;P9`gEieDR~ymJA=^Eh%wUIDh&5; zwkE?Ny5J1xCAKlG^+g8VRzMMY&=yiiqrc`rpVa-8Epej;WPhu!b;jkU@(;Oty2hSl zA)ZG(4`btq1Il`t|50}-2<*e}=mgwdy*-PC``QZZc5(WYgbxgo;+PfR& zwaMl2tCd%x0o7fj1?Zf){rc-$^M+%jEs2_;iB zTKpK4Tbf?ph(8C7lTF3=bE&JjHsL6RVWG-p7=bv6G<5}sO1 zOW_4ePnS;=4%t$9Z^M#Mha>!FC=Nr(Pg4%E{t|35PW`r1-*W3`U`fYyRb(o)Zglh? zunU|P0@K#n3$kH6zT@tzLMV_%p0r-bTt!wa^%9;5rm1ophr<>FWW_Y`heb9D4XgkI zQ-u`&ovtW^Gw2fB_-%~II&aQ5`!$0Soxz{K_<>Xjfc&YMks5H^%Qm0JviHb9rZMAt zp4ItRo;sF)CCpJ~Q3f1hQF72w*+bQF!4IY+XXL}j_B0fmrR1cT{%i2S-JP8dlYuHc z(=hy)lIyeQ7~$lmFwUbpg6(pm+4fIK!=B)e48K74+1{aw7tgQFZ#jLNzILk3+S?)6 z;X_ybJ98n;IT}%guiZA^nZkK}_Ngz^s0J6Se;MgVLu*K43{{=`Uq^6o>_*xX1MeI3 zQ{DV$=oeLZBRAg=Sw8J|7npTKiT?=wK$$HNpMofQ0fteK^Izl@KIMp$HV-s!Hi~YV zHYgh-Ljp}^OG)Pu>J;gHB8`+y<*H%yAPTk6s~r65EV(E5*OIyF*!%WK$Y_*AA_i}d7P}#ERihG@>~B6 zq<*#?^9CXX>*5MBRMZvs#mVYm-sKhcdQSRQ1W25p0X7Kn0E))KD7~%fNK0oaD@t#E z8=@GE^oP9y_@_vs>^a-c;mp=c(Uika{b5NV3n&m2w&JyoOZB^v$F9!nAq zor?6x)FY$|WxL;R#{SHA={IW|5MkB_I9-Lhn=a8OYAk-;yInAOHrPhX-}Kvm3%xr( zRqETBKTf2>bvE=f%OV5Nod$SmzbScRijbC z^3-vb=f;}bO)wN1`Ka);uk`lxh)3AbtL^a5MH?DA_=%0~rcxVzc~<1`)wwT3wcIT5sTa~`@B7W1xOnaMO{exS zPV;%=wzAGwE#-Oc19j{I0Y4fqGszFfz%S5CvkqG1*VaO4EL}usDMFa(U=ntcLJ9sZZ(^f71hNTcQL9UF`79CUT1 z%d&?be6&`%_#k(~;!kSOYkvK`+-rOE>H?J|w0pB<|~V^>}0!wGWstrO%rHnFsMw zRZoOSjU-Yn|KRiZsvta&?a9wG4nJR-FURj*HrnQqEog-Z8)6>!pP@3{+~~{RHOP_o zEh7HZiAK+T(>6K8MOwFRu!?uTPUTCY;dd#k$e9e96}v#d*A~L>V6I#@f^d&Z4pcJ8 zSHUj~n9Y679?iZrC%h8H9Yw2Cc5^|!*}RZWZ|+|C&t>>k)#pMG%VVAtVn5JrxlFF^ z3IX;yDW<$4QQv67Vm(^$$Bx$bJDp3;MF0<)g2+EdpQn$>T0!5Wi;hc)$rN61A&Dw1hyF|U(i6*hz z#Rn<|5CPskqjIO?(e{Ro^7SS@N2G{Eh~H)|%)YGa zrTO(E;;tVy>Wfv0vD8K3)D?wHJM1O0;fQ25Fcf!5|~qQl1z0SGM=-g18-=xa+{?-~2I}(E;;q zgOLr&#hRbLFptCM`iq!~6wLXgk+c`rzM))=ajv(@BRqvUN~0%NA3gb=xaDfE z!o}Nt%WL_Kx#?^5@BFd|0XnxPrzfDQo13{GF-D`-uA-|gePXW>mN=qN_I%I#Nk8L> z3RjpVyAY(cxebi{qe65J%1H@u%dPl1|Kh=G)wxr~LtwLno6kMn>S|kcT#X449etrb z*0>J6Vgl(>P0%+z57by%By>V2{~>`9QFnoA-fL_<2%AlqupX&VS?)BIY*PDS`tV|F z4wFt_MeQ71=a+(y?Qhkzio_!St5JhP_mU#mbes&MkK!xlLdEwyA4ku2c3(+1?!{6} zvcsKgMJ$gkvsDbydO-!ouHQlHW!Kv{2ld?FUBZ!_kFgpfPUrMh=&~K1LPzzCQeT6q z_sAEB#fyXRw45+^<}R@6qr<<%xr`Q-8hil8c(b>6pPeXvM0m2M*X^6SWBqL!cX|u= z_*FyE#2`II>($o_gWHECR!p|(GaPKiXj^UFJX`$ECr;+VTF=C}Pn1et*bSA1j&VyV zwm?+(?efRS?_I)~jVjYS+CJL3u(a&xz|cxJ^~yY5HhcI^%O`aWnFE4@(uQZhqC;Cwf%K+WMzY?i46i!_CH8jGAo+nc8X z5~m5aBL~FoT_L5tTu($@>TCIR>my#b8!@IGxrNF?>hFB6np<r>Qs((QIS6}>OKt6CyHmwF(UQ5J38TnT zsmVw0)Sje7j6S}1vM?*%h-JGl~xqVoe{ts_KlN)v2$Agd3**#w1tudtQr_1`wOG!lOWLCOY;Xc8goZ7( zIrT^In(#Ka9!}tFduRC()vQd4+Q2ie?Ld*h?l<+CCb-wSWA{~q*svrix|t)xOStMk z!>qJl{@4d%=rQ+mN#Nf3Wic}2wjgG3QN{oblz>We@Hm4FSU>d0}{Rlzp7l z1QBm5qITd$8lsC=t3XLtCK4)cCB&9m^(9n3Sh=Tj>2qpL%vFW$ik4H0{h7tOsW`PG zcd@bG{|{Af9Tw%jUy$c zqJpHtR>}ZH_gUlqedk<<%gaA>nD>2Z#l6;Z-ch&3;a^CcSJot z2$#E{p^Ho^&tjdo5OO!Pr-{Rmu(cqVtTA@zK8w} zjs`J_Ot)9d*M@>tZ)7Kv$1nQ{{t+E+VB$x0n^{_}9zGLQHa~Y*Ubt4D&O}a)(5C_~ zdG5>ON;%W$t69sD8slOg)~wcyW0uvjRrCsf5X$OuxD8*=+fJ+90Cc)r+R>=fr1s9ifySNdG}=qz_Ue$h z!EKJ2PlEm@@%Qbe%I&SJs^f13788}5q>mpSjDQcs%oZ3%4A?U;B;U%75#C&i4?iBq z{bcCUa+Iaj%ezYZr!CSo)f`K{WYx<)I}vzA_e|^i2PV0fxfHy4Wtc>6F=cYTs>f?B zzh0$NaKQ=&>Z5$?-Ci`LsRR4_WwUR;1%>BBiX%0AmGx61$q<%Nk(EUwQfcRW@MAPJ zH)jLbg=GW{0H`I_Js0H&jHzQ1{JERd`VTl|-R@U#%SSVD?lJ9PWi(3vYt@=w%G0|G zdlCZV)Ya~59vs@EuOiKA_uUvmG6q844b*VH=4c`6zYv>R3CWAjGnnu>CC+&J#gC6O zkx}_(YuNirksp-quRh`!HDT4^-N894%{5Q2m*>jsE#NO~GQ&3-$Jo_|I-~5tS(R%ypLy3=> zp;Pg@^_@end&*gWy&ySWp|yc z_tqMAsUlYS-3_ze;A4v6l4ygYE6p0+wqJnHH}v5^|B zrrmDduO1(R)8B;62J#PHe=u-Qk_r^2LM3T?9+s^sRi4v;03hH+Mrh^pd~wlt7ecg3 z$!aeNJ8R0WX^kys@S_}41vC36>2xtO7swD@7}B2KbM#wFpb=B>a2YHmzJGFKes0dE z*8J^*l_xre9nC7ddQYT#*u#DwcHkH#N9fK<(PFnP$|^5R)~SDc`^(jf7K=0p&xq8puEfy6OIX}7n?aRB8xawoibT-ny z`g8>$sLwLI&(H|us2)kocDBy{2}(lLPhNX3g0UC*Bajhb1-4(T`?b) z1hb^sDCQ=CkbkOY2IWz*2=zvikJGlrCVO5QDQkPHUmJE;oCCok+|9P}v@rr{*@yUo zC$dmgx+xO3M*fNGu%e)Dmv&I?tN1=KQ&#`hO#E6Ued>r+-)8XDc7}o1v`HLCZ>8(< z0vOYWVh!EBUdJUbzdqEk_~MO;|Mu-0qp)l0U3=|S!xP;@-HiT= zN2uqH?45VSrbC>lDFT&5G7W3Q#kCvwm6INZ2l!!E{?%{zIDA(YRsHDB4J)f`PTUB( zR<3N)G&l5$$W06*4xujgcl^D*XFQUFZne|!_=x;DXLVX@FrVetX|~0@3CewyvxB!} zlf|BPWL}SV_$(E~=RFhH_0&8ow>r-^VEl(vvS-Qu=sGxh#(S-!x7z(rPhr}b3E#(K zuuD7zs&Udk$_pOn7I3_rB>nV8NQ9{=F^aYA&e|NQ`sf6>j5?$W6=8b4A=la0t1JAACfsTjl!!?- z(mCI~7E_|c`&3^{(ZfYBCMwIxSS*O<+-)|dUo$zaZ&WJ_^WQjP*HnDUHUz8U!quct zpYgfXENK>1;@2-%DpJOq{V^(_T>WZN@<~b`4*tLWj~{bP@CB=f7VBL2kv(8m=)FKO@Cx>Ys&ZP{@K)huijlP_09P5WL1BIiH}kJ z>miXUisCxm;ypa$b4$uiWY$#RXIF(+{O-|Kd?g|`!>qdR_$&)*YgcBaHvfUH%;o$5 zJ`ZzZQhgz=^yj0HN6QQb%J0b$lY@U3_g>uTu{~n--qU~Gelc?og8>rdMurGqCmwDk zT{aK%us3bAo4hJ?{a;xZ^IvZd!=2B>Nqe@A%yk4l?bdoNllwHV7kk=ZFJNi*w%cWi zsar2slz()ee6JsT-)-U1TA*yw+k#V2bZzi=BNa`Q_2zV?sHawYo|SL%jSqWQ zl-3PWC9fQzJFc})XJy8OPclVZ{1#w!A){3LOmd)w#$tlTt7$&Q%84bCR+?wbA_DYEEx?KG(&J23X zlqlA_Z*ywuvnL|2-U(#XtTL-;Pvu~`pq}`A}r%;cqQxOViQAkcm9B8c3_O~ zt^HNHkl3i?pRf0GKZ>%I^PVer56fOI)wUSWYlM{DxXWkMG^~ErfxTQiKcBc!6>;s| zkWl;D&!n4IN7bwPPc-!8uHMrna?b84+obF^?A{H}OK~VbGSz<+%sfy?p64)hY)QVqWwFVuB*V>m^1 zcKHgz?d`wsSA917;L0X9mo_I5jAv+LnIXi-kCM=6dlm?mpkJ<^jZ?-9yM=RPJl0 zti}cUlMM{^6*G$;QG)#ilwvd}CVb3wyr;aZgnj;@GA){p?`E;U8_r0C4;n5Ho_kMekSo{ZltyHZnq<|OblB( zw;*4+Vn^kHjC7BH%>7;`GbuV=X){&+QhX1C-%OxtrDN(;*jTo!$#usvKAFf0HSLnq zHRpIzmuofx&o+Eajbz&p)6Zo3q5Is>ee;D1J%d|4_LSRtS2(MUYn!^H+2a8QEGfJs z16NBBFYM7~iSHuBT_@65eQIwuu}?xXJ$ZlbyZTz)@0@?wuj&&!TV%_NH=ei1u4LOw zO)D>YYB<|D^F5p`)b~kOw0?e~N#|`_UN*Da4(>Gd0z0V#E_%TtX1_A8rc&ryiBeNeGW|VzBFQ%7 zn}-C8)v~|Sd?vIkep2r>pJF1Tm_R4;osqd)E88$dKFCcT)_3E5cbL!XH(~qhweQ{o zv&H&&>FH}<9Ge4%nvNd^#7T#-UHIPgCV)Ltmdt=HN~%Y-eByLkHgWBd{<&qxdnk;X ztXIG}k$)wcJ!-$FFGL_!J4>pm=>L6mW-#iu>f@4Rk>q4%x3zvtF6OKMI1y#awL>e! zZANE4U<<>u(nqCQ92IlQhsVn<^Sj0e-m_6QubvBr#)3eN>bISoI04;PGXk*#oEJ9D z4$XsynDeDfGLB_2-Z&KRjpjr+?`kETrxt=6iEyZ?xPMV8_v2@n_l#*{56Q$Elzw%*GX0(tOZN))s3Fw z*zw#G!U6&psp&BXT#cNPJDl2|6x%WBH|mM^1!zhnkG`^UoEUjlrQPOIBy?7(0j zGoa4u>iyM@e;gan&VGoCE05yW!-V=pJ-&7r-s9$%eJQ8dw!U$(Bg-+u^5s`{Qv!j|4r80(}aT?@8PkEj=YD z>SM=Ryez7MkSe#{3U9L9hO5>VVA5_J4>wRN&v&JDR4be)t$X(#6ns)n=l8S}nJ2uk z&m_~Tc@bdy*bxRG)*2Zhen#?jwdLb;gJfpE==I7JY^$l1ZzMmz)3f+(wex78x^lfX z*tyQ?`~9_7Xj&zGVi@K;73^yL!P*`HmoTXI9sQ7yWOSs;aF_R zBGVepul@N?eJ9->KJ&yo348OKe^Ymh!EZ0sZMbmt)h}6jB)ufr?JExiYE#)}tg=`S zBt?GZ4%wd@b7GbRX}pX=8? zGihjcNxvCbF;|uL{gB{jn02~NN!wVpb8yvUMo6$sBjj;)QIVITsF@vx-4nSI7VP6< z-fbcEDA$l0C3s7Shw_`U*v&+hl}h~-w)_f<1kRV2SP);eNx|o6B#^}|GBomeh`guI z75%j~e)?S6@_y;y?~%AqO%0{`Vi*2&43zStx9t zJ9#?}_3!$Pd5dl@vfO*#BB!1!NbI&+El4;>Jv{}Ta0rvhQO#Ipmt#F+ z=d^>mdWJ(%+hRZA?Ejz+!j?TtJLXybg%cX)qNVreLR@}yIc!opm86~FSacHX>X@xR z(S5EdO{EW=E{AnNM6zI^{UZF5!Ec+=L1$Jn3Q(v>SNRgl1-^|`U^vHx8<4YO1h$^w zTYD5It+EhU9T;!X`xg3`lAUt<7~oSP4wfGkHLvWSL|8@w$RtQk9m1GvIL->!)8Yiq zt}=^z)xMKLd`u$61F+a!0`e37I-Gn6^=nmk)Z<7`I!SsG3r@cLijR8(T=b~F>eJQw z!da^qfoLEZZ3q(+i<&xkflle;(CS9_(T?8#{)>q$#y){}5ct>C^DVY;Oa1T`fPGZ8 zXZ2Tj7op6e5F-bh78a;rsv}Df91ZX#`To z^UFbHdbtX8Fda3BzfoFgeW)Q*GIW;(-!dGDRDs_U`p+UbNoV$BwWtPw4((mmPvvyd zaM(%1kxd}$2l}QnvFGHUXIxD|R0wOrNP?ZxtIYpA&(55=&pX%wFI{J;MRj=V&F%m5 zw-66+4Fwfg@aE^or`lSaq1pP4ZP{=-SMd?zm06S{i;mpE35 z8U%)tO~IXtA9Z&KPWAoS|L=Pz2VvZ1JHB!Pt&&sJ*-v3YOo8ygzgwZ%7hf;_RME^} z`5#Z1*Sv#)3@&`4r{Lch_kXh&mdKm=H8lXX#3f@w4O(4ZD?no z$+elj#Qxu*Dgt7_VhU=0LUUfu#eEo$)o7Hv(>xsz%8+J+inI^mBU*>zw;c2nfxN1# zDUZW9F4dEF=Pn&Wv{%neM-ElFHAWVL2Kto)H|9Jea>U2+nd|n*rr%5lqcWG$;`>x?guW-=~=l%$Wq+0llK zC)j*_u~<4yOrCq~JLCUOF}xI0lfZHOP*cE(KzAGEn#c!K@c9~3soc(SA4cl_B^eK6 z1M>fD3&FeN1^zw_1xm+PaF-PgY+lynoQ>=-Q7zWKx@Y0N2bUxqhCTtdSg_HU89iXj+d-- zT;0*A`UBJd1DU&xeU{s@9p(8^{aB=%P>hNc)aY_0#{Cpm zqhUK5_pvL@4yf)}P1hwyKE!E;b!gJZyjR_f)Bu~sK(bk?ed=+_cZy^uW^-E8`*6$B zxl)%!!6RhfnGmX`nr=Hd>wK<#^%}|6YQSD&y1*p6QkXx! zdgBbO-7~tA|NM0Xv)YB98)_;q1RO5TSQJSE8RY-8O zV>J@^ZVp#;UsLrWRh zB~`d&5o9>8Y6zKH_&pc^mHWr@(gVi`-ga#BYpKUDC^uD#za>8AKh7T*eag~V;`0dJ zWu8R@94gfjC|)^2nni`r>|-hB@k=&YvFHPrW05SRa%jNIR-xFP2z(KqMW?GdtZX|> z>KIcv>))ruX>(I|RyE?Uv|(e>s-DOH-+3xaT*hyS&{T`Ol)s(8em;nrq6eI+*oj-_ z#3@&2z1#%$X3;6qEh5|~8NkLi6PMM%<|688RD!<3}IuVzO|5H zok){glEo4qkJfa~nn-!nW44b3GpULf{d^9N_wm4-f(G#=sIl%+Y=Wy3gowf<2P`+^ zo}fm>K6$vRha_$Eo-OS~Ff=qZy0n{O27wtGLkvf+FJY+FUi`Ow1||I!Al#`V^e>R6 zZLNo$p;Hbtu|5#vBYA#S-QVZu_pq2$h@*TqHv69^X^|um^g4OP<6=syr2ORUz7)$j27p8SxK%bgfu8FJX&+-Dwxh2f{FWCL&%Aw<% zX4CR(nyg39UX*_hIP|=GAOseOflNywbCF~3G3dIjiPwaAIk3eF@i|{mKxt`1q~mSj zd2&=A%fm4ze|sY>OO@Gr@8^Hirx20AvkxSiO75w{anNMyh zPmSQTrvxlb<`ooNdUwbXHT3W-i>Ine2%rj%A#-Lk5K{iwg)iburkT- zE|x&>B;K%}qGXTO9GTb*9nrA5Hzyn0nDYg8Q9 zQ0mX3J8Fis?o9Q8&9mJ;@#(vH?d8?zPu!XP?w`by^AVgKQaGXCJtvZ&7Zug9_t8E z#Sq-v(GJLKT~bjdc+>Doq!3jc2;PnRM$@F?H~EN_=%can;#mcI$4wuNtebEi_UX*$n&<#vv~Aki7cTFxOy|7ZhU&up3PSd1=Ej|pl6K*wpLs! z7HR(b7y`>3D0!Oam?99Z7!L;+TkRlY?7u0KUZo9EfzwMO*s?4)h*5uepjnb}+7=kf zsO?{}i&CGz!^=tp#`5ww<5rX^aVHIsR_7v2A3uPu7I;w#a1u$us@-TxUQeL#^}?u0 ze>2(bGlU>9cX=?A=F@mUo6FunP3SWLQe-cv**nQSGWtLkm`*5&G(bxjaRIsbgI2hC z$A%J-LE}Da-7g>KUDV+ZoE97W0Rz3hg0~!0h+h9dHN9$+MF|0Cg)FbjNBWuC$19<{$o|O|%|!;)iR`U#S34UxG{R?V zJ3EAK89T`Exb#92Vg3NV`&bJyhV)n+i-ASsBDmtV{i|amEL@kkugak)kj0U)0Gd!c zOz$H8*ofx#$I!wHc6?S!^DoNtUkF#OlOZK&hB2gx+=-ljyOca0ems*cUIPu9cVT7H zu|$exP)D#BV2nO$@a80$uQ=XcKLh^=Ul#fHCe}abkZK{oZ+NtTAWD3Qk0nEh!RG#g zElamXViFT6C`|Yle~`31Kzd!LQG5 zAg74x(UxjX?yLq%c+%hkBLu1vo7BK6#*;CI0l{Ol`C`QS+$8oW>$TNovlh8`p4V(?U|Hw+(n=^;!*By8){#<3G6%BNo%Epa*i{>Y0hr z4W~b9%W+CGAE?qe08K_nsG9H+-8ZjS>eEXny4kUr89aBBRXqSb;g%MXm#LQ^?HhVb zEINSdAr=*bYpW6obeag+hYH9|*4=1>y$&SMyQ<`IO5*yxnzQ>kCV)`i6qK?)LIM@f zC87%%B`ZLl*LpALtG-JNDpE#)?8N?Cf>3eLKcS`D%Y!yR5YY-ZbJ;9+cQm=cpFLTJ zc)mMh%!7M4Q&3^8rEvrjgTL#QLA+sQAr_=oKH&zEvDxl+ngyCH<1V8M&%Jo>4pAV2 zg#}%yI|+@@1|AxffLRs3Z=5@HOcr6y35OfhmsD$dkB=P<^25vxD*KUi>3&4;#O*|s zUmnM6HR=H?@HZx3K1eaP;?eQ2JY%$Vduy_>=)L$Fi6I(LQhURR3x4m$8m@FzDJ0NB z^}q%BEalch+6wFz;$0v657y_vL5e_tN~>k}ZGyr$qf?!~qXw zB#o`2OU9Y;@v`~xU$!alf;ZoV7|CjypQ;aYbKFR3)htMJ{dF^5e_z%1rfWc3q#h5# zg?E=yfM=#%!kKPV)*BEni=oOC0Edgzd7ST^16_nG+JK>dK*io*|-Js`RH z>Jip7kdH;E>ljd@-p=Rv{-a;|26Ov>}rM#g@UIXdrz0~beqKV4{ilw;V@e? zwzmspf@#eNO2dO}cStw4IEEkiKh79(DeAu3m?r0rFqbq;#a(GbQ))nUVI$0`bp}*F z2;NYDyka5~?96UGYG)8&7vJ}iO9Q=Ae(>RTl?V!s(He~roa79*UYDYlSDN_PsHTqN z_`SMj6~W_ph`n0a$YQw6kp1^7E1&ImOZgs~7f7qBZ*ET0!ywCxX+<)7)yMc(G?92a z5AgPk(qD%uXu($Wa7eF36_6oZDo41ed1h`Cl|5HLF>m}v9@QJB3qc2S6p_!ZFB&p2 zNC&&OVZ%V|C`#oXHJko!j8NUnJ(Wf-5HI4@cDtIj1a(g;K+wZaM5b zs?M1$>yDgQEF-0+S_e$sp(M|hMCNK*){0rG7A7JoRbrjB%RbTlhw)bZ^R+CcKy($3 z!vGm`ws4pG7-F{SD^e?2p<8K9%Fp4hh&?RY&GD#HVs+DFW0 z#}Mi=o%bmR#SKQ3%b)8tErnB;CAk%q$PLwEaNQ7XS1Y^J_uk~C=3vV@Qe)PnSwTF{EU?uk0z-0tNaJR1Ms9??i4;Esa`LI{#*!xm#tt2T?T$0!* zGDM^PCl!?FcSftr>eu?1ebyd<&aP^wB1-Ge`yGfoo6Gjazm>-Mu+2TibY$w81r?o{QGLx((+;dgjn%&_#?3Cz-ytpgcr-4VuOFv6@r}gYUZt7zkR>G$#xK-a z6N`pKsj+1@TEEbEEb(H;m#OcH7O)FwMOJ%9~pwlExFabP&KlbwbXxskJSm>E6>zU$n$vpRJB8FFb#`b#nkeu}OO z-S=aY)~v`XPQa}4voOelyk?QmFen0zp!s+lDnPS&aoCm{_UU2asGUpNiks|fV*>8J z3)p8Rg7+w=f6@YB)(;W+9ektJK&HC(*oBaIcRP)xebU|3)&1ziN3U9er#ZBnQvbQh zF+@EMK&hhfFxoWNg$~fYmo2>GPYkgRE3-@?_Xn?6VF;0n-RheUz)*>|Ki8`~rh0Nz z-ef$Go_w*zae`Ua3G`+JOr}xQ8U4`hv+^n}yW#F##=g}-fk$(6`GeY{Q~=2$G1vKc z@Hi7W6a|knS~Ik)d?E#ljOo4RG|}=lTrFJ`poPI1QG+~pPnwnve_DJ2VX(;%3z{BX z32}6oP?MSt$Fz`av4o>WTCJ3P6IqlNauJ6Dsl)jPcS^jA9NXJMBN%dSQAd^$lo+Cn zP6u^TIdcJ5uzlWW)7>H3yEFjTPSUx=Rwg7r{T$3=BU~ktjN+dR;cwPy44SoU{%1%- zvj(e8kC=Qn%0%N6=3z1&`mq-mZ2EP%z41^kSNsu&Zsma-fDIq1*%VzwcuuKv$2aya zhR5Sa-k)WR!;g$kc3rAxHCA>(_M87%y&cQ|i305vRdl(5=uyte^saFYa_JVaQ0pk8 zv-w<=Fe$ni{Kq7wsc25{8NHt4v1rG|F%ShHco)6xMS`2ZENiX5WKgV{)fywRT#^vCagTAOfP^slqvqM0U?nDC2Y$hnw;D>@+P!e{Kg z+rI)0nW#As6(_Q{%!$+DIx)w`PUdzGGH;^Wkap1 zHKiDA6&9PkU#4uY511=Lr5wubqYS!#GUOov=6X))m3CNUikm(7k6 zUf=8Ok{yxin@slP%fz!(QIS^bb9+ga+eqAsvDL`&l0Dby0Iq5IcFUJ)plcZSa+i!^ zPOp(h(Cj*7pm6RNjv_{sZvZdcm@93U>aDr8yCBnQ2?&#Ik(8Sl3txO+i$1~oO^3f7 z+cA{)DHz!bM78!yQ&$&2zIWWCRhzNy$J%}~x(~D)Czu&04EG|QhBA!!AR@(VT2zYh z7!rCDVC@3owS!@;B1nplpVx1CjX;zF(l5VqSr0Vhry;DwV|IMtNktw~`Ev3mb$#Yq zKni>mXcI%}rb(ebD@xBQ!sEOYcM?R)BuEiB3aF!fQOeta1r+jVkgB4H-pS zFF&YLDyNMBX-ofo3ERS$0aseGjs4vaC4Kc0Fe|3uJOh~4pv|Xd?6vz_fs`9a3L+W3?*ukyxg=^A|+pQnZqz|jmZLJSoU0< zXQdqVY}A89$&Ipm{}Ok=&q&&Zw*@Jvza1HhSzcB*V>5fjs{L284v&e&1KwS>(b=32 z7YLLgDiJ0U%sA@aXQuV zTH;>(K;ak&15bIH+Gjk}lr%zod~axMuskg$o6s)4Te|Fe$|*28lMBz#zrIElhvOdp z-Yubr1PN{k`AX=W>R&`c18-wW$;APkFLYWPh5Eqik(Lb#Q7wMcZ78h`^=N*|Kcck} zXxO;5I=vM)cMhQfEBGa1@uB~TlRctY-p2zUC7lWl1^!*4tR!*V?MQ?sX(R-4Jv+W* zMH$p|h+N4BoLMhi)_>;axx?_`F73iqn*2hcbYcC24k$%cd__`BTwoxnRuNmeD!7}s zp3CA7c1$cvHtLAHLMGGE*hI4@g&m)FUXe3$#Dhe8pfYd5it;IY6AvtD$--u&ISYFP zWAFPr%(=Eo6wh0Xxp4u&fCD~*r;P{kvVr=$SHQa1B-MDDOdn_)!IjB_-4$Tg3%huI zLk3k%Ab5;q@B*idCbHzdK}srHA5I<>QymS>^R%+(d$yYdIYasG5kTIng*^DpFd5>$ zOR&=hlYEE2XQ;9OoF=7`7U1#c9)2LPAP-ff9##)QD7$2oWDNjN4tI=Z<^4ZSR;r#Y zRoPILS>!K&6TnqRAPluggzpiIhEP7Tdhv?ttT zH51>wd4ofpe*93_wjO7aZT1SXDh=_~c_5y%NKKKua_IRdceq0Jp7P~qKVU(f<|UOI zP*A0?PRu%K-cr@-?@(y0SUd(yHM4VbJNE%zP_-Ftk!YS8IvKI~NfIE!$=kUFSp#P@ zJN47vOl-YUyKs?4l$G^OZ1b?0#&S|x)kw=7EHbq1Sx6i{5eqVKNuGSL^E#UoHgyo@ zw?iuyI>HRIvd2hg0=C93O(U1^P8joh5uR%OrZs>8D}rCPc1(b3+Y%ot z+?29T2T@Tc3Kd9}m4c{!@_yWMr-4Sr?kTS$8{gL*pFxy6wrnF_rn8Z06TYb{^k>u2 zs4>Y#14FxBAv$`6l0y<{QU4?|7`%F+AtfrDzCie%Rlb>l*dJzVa=YmqS46TFGL3!N z>lqFC@DG5%93S47y?C6nFUl>S3qEqba*(?`{8T-uihH zpCrlSqYsgNg4-FOY&7mKPyIp*dYO^*sfPBit~Q-NyJxUL0!!9YoY(k zIP>~liabnA=*s(tP*aB40M&>f!7Y(&C^5tz#D^x@xjhda*`>;pCWz=wP_mL#WEEcQNk&qlo}(WsDmo1$l$@1aurbK z1HnYMme;8(ZW6>*CE!T~1Z(&K$k%cu*f)=pirwvixl&qz&-R>6_NNEIOo-q+14TEEviH#kD>)lQGpr)^t{H?EaanO2!y2kF%;`7S7H7 zY>4ot(fD;LSt>g8XXi)|S^8En6e|FcLO0NPk)vFRhavB9CLey0rQ;6N(xnbG&~L8x zRwGHdk6u81rIxC5~aDu}ca$%a;T4m%TLRDb&~tz6U@P1h3xdurS5uE_4EI+=n=0 zFb?2XNL+O@Gdpd9Wk~}k4jjo58E9OuX69gOEPAs~j|*OnLH5A$nElOv_hz9~sb$PU z9Z4@lYx11aKk&-40IoE)(9lX~m;I}OB9v#{2UQ0IY~u4GdqjnR&h1}^;Zo%jJ0e;b zUF}T5r$AL+DHiT}_evZ680yFlkw0G(LEN!>Sx%3ZOYBugDu*oigY1Bo*T4m6QVaHZ zmYj?tn5InSMRr`oS!{AB<5$OV>x5rn#+uD{8s{?Be`6uD40N+lY&Sk(2C||W9A@{q zg;#d|O)DLB^LusT`yI2*Pp)qzPLtJRr3`#eX zD(zAy7(zC?E-4BtEkOL6^tz*et9~y>pm{0YE)#*+hly6vaV&p^B8PUuakVxHGvu=i zy8)b9y;~xKqIA6TSC)QWg3gtS5T7qm5|cu-FZL-xoCQMH*4LcP4|`=mB7q=kei_m* zu*K?iOUDNY-f?(sZO0OjK;<>uDO;YD12toF27ZXMU$d92c2FPNz z!!D_n!`=3MRpmVN;EoPG;_gy9vMg%=&>YQan*QRn{=+d?Fe`|z9UO|sJfL=ES4{`8 zEMmX?K3RVvBmbMwu?MchQmYXftRF7B|6$wkigHEp`1}>4o(Kx;RNdeX?DSd`F8rAa ztw8Hdt4In4o+~eU+e?7gX=`&+wlMX&dyg6b!W z-K$8l73xWq5D&%)^X1kXBxZw)xCLc<2#JjRL zt+o$WR4IAPL?j44MSR(#aD?1s?S>yn5}Eo|_t`gx8A&|=9sTgCd0C4CMFfs`Fjnpm zVY;=d=`j+#1JtzCw)f(jeMwmdQ7&iM{k)dMLz+p7fQ>R?mFFSWlEmaK$@AULVwaWD z;C>g(DwiIc*Ys>Mo%cyPX!6kLMU?VA;1m0ob3!psoQJxksn@6o_`jyxjOFr)tm|z1 z-@?cQ3uk-D;=5|V*k9i4YmnWA0JtpBZ8#|XQ5SiB(6}3lmNrh`J$<_T1fShuRPnjWk*T;&5`{}-jU1^lRz;_Cou`&wB z92KLiSRt4Xr0MVva_Rk6-1=HV56z`~B}#ExezsSA{F32z0d$)jC#U`xoV4w2YS8A- z2auQqaZhy^l+juC*OT&6^0YP3UPWw_ar@Cd+pc7Sk!8BvP&CHGqx;aBN@=ck=M;-W zN>KOWK}d&$3N7Th(~ux`r8MuN5(cM0I0^q9bv9+Xq?{Q~J1c|v5H*dDziy%_%Qp5@ zgjM|`FZX3b{tA7=%fX)Xyf4qlBR(3+Mnw&?8Mm5iK5Lg!aCdiaGdy&_8p@U1fe@fHe-^AbF$=}}}uAm=i>9RR2Ek)^JX1EEV z0HDrN!BK1;c-V4h)bP1%gZ63|j~;m-CQ4i*s4Kb~hoW2jv1`IGSuHXRc2RKRmzZwx zBL;#ry*yN^O{UxPbc))na;|?uC-;Ar0$YPQ&yo-n;8lq&as< zh*at3223EhVIZ(+=Hj5uI~+=CYBQ{3CA4JO52LHvs(uOurI*tM7A({j8#@6G!2QSF^pi-X(_Vp$23T3TYjMP4}|H3*hGBNf_ z9Bwa+$M(vZiLa{j=}=@h4PDxzP|sbqnNeC(hjumUd+}~XLJX>bRRDUDN}_jJ9PUei z^UiAmy?Vsw_((F#iZr4PMINU<(M;7ZNupJT2Y{~Rbn?Z=ztYbg{mYT6XrAz)jO6r)vzS1DrO@&GW9{r z#k)t!y~Ij;s*);~K2le36X5a87-Tb^w$As^ZG#%I1?)Ckal7&;Gx5EpNMP>8 zID#mp6R`-oVbCUd?(2aBKn1qIzF?G{<`WZ{G0<>!RZWSGGlB|Q&2(vx;@1kIf{G6H zgtC)1`0YCUUw34WvOI4KxJWVYv8+H+??=SY5I5~@qI@b54 z=5Gq|alpceycY(rPwd?8LU#Vy3@9;FL18lY0Tn4^At4+yUpHrskd@7)jh8@B^OzXK z>SwQ5eexcq3hFwzS;>uo7HQob1XbiKka&D!e)}V0 zTK@4Pg`Vw&j@yE58q4Fc-bI0oHifL8e~lU4%gRz5T%L2PmyMMyGhGiAig1n@uy@>KQ=p(6^eS%&?+@s>!tuA*vr%C$x zkj}ID=UEu83xz~Gave{Eo5=Q)A@4$NW07I{QOhOX&US;@TOwgM(FY)0JGFXDIWMk& ztYSY0ddjt_`q2c@WvVn|sXB-|++UZx1^ExhpQRM>FMaEvp>RVq^XrT9^*s04`T6vd zXL}wzc+lFk!5MF1WhLh}k^!PLW!cHnhzz+bWek#hxWOLoknqhtvB;;xR?(?D4S*}GT zUtUnX0W34LQoxx}!$~c7@f{o`1rl<_unY=PvGsTFt{Gq<32t8)7A{x$_N5|uhq4og zQqGzQaZ2oMF)ouHw#o&E+M-I7jX1j#aj(Gu6Tf83m@awH!yALQ)V%OQkNhKfk{9ny z{u3Qe=M@yPM+2LK9B+)Z-O)^LT~R7D)7$3C})}l?3y^wUq^#z%W2J2IUHT{r_{VjJwKUhQFD){vEx_b09YpAG=Glys{ zKdAgRX!ll>4b)1NXIUQ!M`o=aQTC%PtV0KViiWJX{TAq>HG9;A?bi1Xu}dP==3GVd z^yk=Zb5HXEMCVT^`+&uY?*Ty{xspoy$nucGVlKkS*9f_S5=Ip9*;)^!q&B;Lvysx@w< zDJ!pjy1wqljM<)&MU)=u&Ox2GosgM4U^i*mA1o+&QH7HduBbauM>XgD{}WrL#jm@o0?V_srv>5ylnWuh&att^iR6q)GToj-3ucIbcd@g&g96A zIZAns80e|e@LuC{rG|oszX1E3(Gnv?aVxVQJp)7fA1oxYEoSU zky{P*-Y+^KP`2M~3sk1<*g63{qoxK~hdJ?hN)7DFu`~Yo0i`gV-9@c za@Uuwrg-gpHqiIr_JCnSC$Z^do-B5lZ%>$C2%c4<4ESXb2lXRMILuVP-2RK)*=5c^yc`=Ohqj+b(SD(L!mBA%}P>>HFe?RKnu zEOs+UP~|JjwXM#rlk=`~VTKk$*&hsP3sLuySFdntf!FusVLhjNk>Gvxw+z^RhrcMy z_A@L+o7P3VUKj)qfVVZUO%NfF)#*kx_50rc*>l7}9aS#!Md%^(ZcKR6xuw&{3T~$I zuB1*~d!oJHk<@fZOi*rajyrzgoT3BdxGitMZ!ko(bQ(*%RasU2$AwAJaq4SwsMRIlF+pU$0C#Y;vd+2O-*QdG9LvC zPhRgM%b@XjjyaR`$$<0=QcRca6pw+tyLfBv5q9UWD)bdElKcXPk=p=k{8v}5l+mj;c;^jmABXI`CD;s6g zGA7f3Tnqc*^C8N9SDfg%l^#D^S)MH0_|&x#7Wa})RmQ+-Q;Xjt@{;atwwY5b!_;{; zBJW=ixsKxp1WVk%%ifEyK95Kuw#iJl#iDn$3G1~Pf17D#p}rKR=I<5GG8Al&b zO+c}WxgxByrhBXFu)9f@zPXqkii2g^7U!LlvN=Sg&c$h7o|F{Sq^fO;zJI4rtiJmZ zIckH@+d4>jwxPXN*6fa@%en-Aah~t z(jBj7Q{<4&CF>I5_zUEQUDi;iv+xj-mmvpDui$0Vwc}& z=wEyr`feLA;x~L0?+I%d7+9U=fo|S9Bg0nz;mW7Q6Z}5S8V4^0MG=;Tb4#HIPG6GA zO=!CLA>S>h1bV77er3Bv(F~AO%=CukC z)KINX`~1BlC&9|D(fy9_lWu5zkqUP99lIdb_UM}r<<@{dYxR9k@R0kXNp3Dq+3)EG zUmRID7uL78^!!f7_d7KXKOPZe0Tz|R>OOhcVy-#V@V>(@goD|u+ovSs-nHBV1gL!L zgGbn|--MCY(#1CsYxuBuLTmR8gNI#=GOU@7z&K zi+(z87C-$KI0lsn<@mO>PQ}IvJB_=n>rP^IU_tp$i}of$&F2#|YgY9yPrHd9Zu&Lw zxWf$9Q6}D3vKgm<=$0|Rd-TDay70}(BcR?52r|H0N^EYP6uoOba`LOhjk95rGo9=q z>^FzM3|xxJ$jah3^fhR$HQqstOC>67o&V;geuCE_#Xi{XK-K%K371+>%KVgQwtgkr z9iVr4&5gtDMQ4(xhcr4arD+%ZjM+{IlH!a&TOo+sV_s=QbF$Vut*1p&`u2yb}iH_Ma5o`CpU z*G$f)1dLmgKC5;2Xq-*k5&)w%;2P-nADciO(G$>*KSQAUPkamF<|TK~QaTp52Y{5u z(dGTB&HBl zdi*!O>tbIOFbSaDY~OhNJ`t>c;t<%pK^eXVJEZ`rwOu=|-OUX$xOel1V;?xm z$wZh~h_LV;|0SV!cfU;7QGZk}O8j${l#cdgsoagomwfwd;>{}{ z3TvD?^x7HcaQ@GRs?UuFZABo+qaY*CDCB?);H>0#@)#&`zHnPN)P+x2_W57@k{g?> zoZWaN?9|KBb(8SzFAb}ACQ1BKZgFowoVW7#EbC>A3*gd#_WjGs-&Vf>b3CVXK;eR{ zz7mueuSrj^(3ntS%gd7P3@rZUMJXMF!RzzMuf8pKSm|eOliu)X#~#LB4=-P?yYu4? z_}l|mxy1JkbHN@Q6_1{Y=uWU^Uin10%Ebj6L12E)v0>lNnXsct@KhnJ`-V#4iPLT~ z0I0}2EiiuR314%r=*Ae)6y92c)f||%GZi4`mVD_NzxBij`*aCA!PB|2`h&8|I zDEj>dgaFR#SOt?0hAsuUAMB2ZurLJ(egmf^kU`GUNg5=c3VO}VQbr19vcEs$=a6_A z2=lW77x(ejf`a#nQ>_ZkX{qy#P(Bs(`}!l1R{)@2cN(;6Spon624ZiSC~eKQgB*)$ z2S3DV$eTxA`)hqz{LbOFsdsJSt#TVs8;$42W|RN;TJd` zlili$!SG1R+hg1#?l_xx^R3Vek#XD*4Z1{DzGWMvyp^O*@g zSw&iS4V+>YzoL_qShmo3f(n9FauLXHD| zVPrk|%%`BCuW`o|J0RQ<_X+NRa#?^)4X_&eX;mN0PFT$XRHD4A z1qh`L`f9SW%zy@m3UC~bovIA->GwDRs=I|o>=mwWw;+1dwDA;mSMGmX|nAgP?q$dN7>lCpk;3WsL-6%xglRJ~2`lD;T?*?ei*R-&RZ~ z(&A3HI{UJM8hdeF90?w0bNKeeJ{ICJko5^-ihty&{3yqc<^BL`;r6AEMiEak9#aHhk-7JWQv?5Uhx)W!5H$awndn=40kr(hyh4X1+w{kB0tECBIiNkoc_j3QR2BXyC6;XNDVYoy?V6n{Oasz_7v0kRQg|xT)f{S+lOoK3W~lm zyE?ktFL?=NxZE{=cJByNjX00MV^Icg@E`e4@j2XYXKv=~iA>5qR-^=HGUYf>y}=%0l;|%r`IQC?>c0t_?PG%3CQF zz0aUBi?#urkp@CRu$9rmvS-Qf%tHua%iD3GbFzsQ&CF4nhwi86t=ww8c*$47EK=?( z&F<;1PaEMgymwT1?hUF-#jrx_Pw)$u)QpB2-ni-WUcd>J{6=aCBk+=W!DpL2*w8}3L3PgJ^QUBCYrY-NjAfL@ zBYo(fs^T1L7R9L8mpBwT&@uF_#E&lgp188hf3$pS&A2wJYQ`N zIN{;;MN{NZlnm%CF&RC~kd)K_=>rq%wX;&qu3u!%8E~-!$Zzc;`3_#<=o_h1cfQm7 z6$7tKOdhB{Mgb4e|Wcll4PR^8o4-WzTDUmhuzn)V9x`TILG5?DLx5Vb%947B)-lobV|qq z>^f%VwxYLEF;k8~HjxU_w;exf3p7xM`IE0uhH7rR&(+2(1%k8eX%xn+qILJssaL4jq(1%AqdAwP%tywuO$F4cY2-QQw=Y&G@+<#HV)(@ zG*|_+SI}*fhtGi?6#OVs~i zwLUH-1%x;o=xt>)>}=6$>k{lE7{yo6FSk*a(B2sc$Q(GV1raL4iA9HSu zJtJc-4|+Xr?g2-pYW$Ct=2PVK->fu&#!hSAyE-}@AB5GHmX>_RZXpEj=w+el0|ule zC7KoQasAIJBH3TtN_Q#&E9*F3-naA9ye}X-9TC;l(*GMh!L9%Lr1(X`U;VZUk!yEa z?;WK8dVEZ}W)7jGLd9LURH^fecgL)E#&l}M8^4sl!@iZ?kM_UCqbOFXCb*sd?v$v0 z-*Li=7BAbgXWbIR+F#zYs4Kt8mVQ<7SpQ9afCercY`J=WudgU?WW+?LFPKB1HMi)| z{Pm1FN^=`bArc%l^~T8UyN8}HCAjK7ip&hUW0y}*lKgya8lZ6iT5DUUoeAp!6PFLz zON^X{&z*u1?}u@lJPQram^-GOXyTUWDr|6lX4b>oz$imWHiuGRar@QVT)5qA<}I~@ z58`VR9&TN7Aa=L0a$~QfW5MdH-Qdcx`8>c5J@tQ(-aD zX1m)_;YvIxsVPrDI;grE2JU|V41;}p0*sOiv+qc`9o}Y=kQH`Xh3RwF!|Hf*+XTgY zFC%2Z)}^du#EFp z-ZOoV{pzc6=k9qALl4ddW-j;cPZvq?t_4S=t#}<0&jcg<)F$d@)LkyUdZa}BB;3UE z;8S^MHK8Rb7b}j=Pal1+1)W0sTU7)oZwBihy*E>FyM{&a^fhDQbcLFp*kZY*OJREV z5YGX^#&d;B>Zb%N-)a!cqsJmoCMf6A>f^p8-uHF=fr+DDm4E4>eq!4pWzpxlNDR2A zPU;^Gd#CoKjpo#r23D!+@o<3f0sLd8 z#ibz$Ycq;YFuV9=kpC3gS6R8CEaI8D9JDN5-lj`XX`r>$ubHpL=uOsQVk4hP4dNg^ zPT~G*lQW2J=2L=c7+- zXV+x0omL3*olLt4465XOEg+YGt92is+rMmtpa1(AP$X4cmT}Cg*#XTY&|H>ZlC=Nw zz=3_{2)G`d0ep}B>k44+{QD&aREYom^1t|~{X7UclC6?@($~fQk0zW~wS4cxH5M<+7J&3a``&eTF6r zbcqhW+O^TmoM;wy!DMeSrmFmw=*!>6-V!-Mqy7K+CeyP&uVd3BRd(*RKxPx?RK2^z zt$4lNK|d8(rS)%Tr1(`lGNU=hEA7Wsd|R9iBh`Fn?wSt=dqzUGQ4HUo;#d{%LKtcq zO)H8v#eRI%mFzzJeR{4vSy*yst-6p@F_!^%sYnjZxUxHy2@|SXN!&aKs+%3Jx)Oa? zY%=>?G{4C(Gd;#uw*4q`8Qiq*Qt|7eYo&gE#>}BG1=vPBj?=O60#)m)`LGWd2_>Jf zt+7RL8~%)l8=SS9R#ca9H45J2X<3;!zCKDc!&Qy3J&7x%I@Y;rl(fX*xb!cZdTyTO zd9Swfvs8|^Y;HHn%cdS=;}uulsrk%DbU}8Sp&QIB;$4$vh za)>{5r(Hw7)BN>4Xu^po$8lj6H-t!qQs{{UQlKA1+xf#-cn=C)C-}h zTLJ1(D!%7Htr?X8Wd!1G4X6*7n7)_cv}t!a1+%ay3_ zqaQDum=e)hdMTUT&RE(ic|Q1rV_tR70j;#&T#ASTD;|Y>Z=>w!s*sEt^Tn&fAgf13 zst6<6{khIkh+Md9&5Pc*coxY@FTN{JUHT;av#jS_fF_HJf>Uv0V{lR-dikw%Cm=h}R>#VLl~>X_ zt6}8KV@xR*I=F%_#T)zQ3V1WxOH2E?A02Q)a1?miJMo0P6pup=`COWXfMNu+{2Lm z>qI`zale6p+u=%UZE9nw2q@~6f16+OzFW{%MW=G zoV|w$ZBcm4e6aS;ZU{eow@BS^E@DT}KBIJ&NEj58*v3dghFvH~`H>e%lyRS(aXtyX z<@pd-HPaUfvv2L!3-824o{ZV%ZTY=iT@cIsdHmVpd{_AVa%+E|cGrxq{ z(B=)5zMD}H+A>*QkUlpQUoVAN$8#1`cJ5{;%lb7uea5#@C#^A4Nj#Fo2AP|Y7H?D9 z*wgN}dmS2QYSoabt02CXjcuy=;OL;V^tO*fZR)c-o`JU&LY*%pRS`GzXtr2Yit=n; zkM?!NnFd>i>FE0@RgF_R+tw;vw5<)js)8uStgT3*!eo?in*_cR*p*j0*j1r+h3ZlC z?G&s~E%mN19gZI4D{Dw^O1rOtqOTF+g=%5FzJ{BNSPUa05LjIe1jVqKc%iLA1$1gZ zagU*6wrB6;p*!!+3!+x53o$h~dI(pK37=El{5rRV-%^hnjV0NA4SFZk%G4D_Tw_5} z8=A!O(Pe%qzzjBkyO>#Fm*;b@8;(DrUr8OI{l0L$?QBuq19Gt~jZTQdqs)&&%7$%7 zX{hZjtPlndoX$M_YuI31bzvCly`@#`07YTpipWywJ$z-c?d2_L=k>3rpP752VOUtt zYYiFab<2&~%K(NCHS1I9lkr8{y|FRo1yM=`!zde1%Sg0-E zQBpTFfo|o^H2Eu9@%nGf+M*z9UZVT>ZAbaY){4A^xoiEMr9{vfW!0X1qZS{aDUb8R zW%)*+Nur7RH6ij%=+Q@(LYh5=kSJ4Fy^cT4cl3v29vKU^-zYSS!zTc!PJHE;>hIqg zA%$`iYqqAc1*>u+Cq{&g#4rRWLO$^ho3O zXV?ss%1s!MkW@)O+;rh7OWSV)*t3=M2WrO3@d0)0z{&$&hmXPgPUDd~i*OwKER~m`-Bn3WYr|Am|Ft!Em#Q~| z$RhYBll_yuv2#oE+)$$vHXcO$C4nmnvZYXzPV�$LiakSV(ASI2TUzaSv$Kcq1wV zw=1vZGf4(^TEV)Z3&IIfS^ae65^c>%uh3Dj= zYt3eR-6*yY;$J>8!ngdh>42kaiM@_<`0IlaCT}l0LwY?ZPzp32xjxRfza*80<7MDT ze(yU63F8|io4mA`-fLo?lWo70QI)YT54NDf$BtVrjZaHcbp-kaY5tr3*6Q_N8XjE7 z^KAQ1P(dMIOMjDAZkbb$&9VW#YL6i0J`a`GM9r~=mX2erbOp8__FX9{^X zR#pc-^^uCA?P%%rfu+;W;adL$H+nTZi=nM=*ckFY&}fr@Y_{n^rexTd7UX(`tj=eA zi!-T#B9Wj}sCgai z(LOV9Dq~Gx%)US`5r&%&FtkJv6NZb=EaF_LDcYjKr$~P~=4EwEfsuyV?WBvXQ_FJO`~M~j}~%sYe=#9__t6_#jQ={!pfKJo_cvd*BuD#xT!wMxD&Nt zaNKRE$M!uPj}r8m>w)`8_PQ3TM_Rz9J4`Vx!AUNv^PkU3La0L(g(UuxzmG*z;ln>1 z^YObnCYZuZU1bW9Ac350(Rya&TiQ`RHk)slnj~k_JJAq-F+*I7xZ4myLaa8nQRyp2 zH#^FN>?^5Co7>V4`~2nyco1Et_NAp9YFj0L*|A2jOuccW_0rd{a1M)_SnC~PZrAcd zxsOXgG$G|dUr9&V$gAS~GJQ$2uX^~t&YkR0HF92$M#*3mUo&0R($d0~g5l zz3m)`jxNBi3b0+5eK~v~QdzV1D;Q-0|0!VGe>lE1x5XJ)T*;MpO7_?;|5a;-CQqK( zC9hya!D5U$OaMK=!6Jx!P)Vq8p|{slEL^waSi}?ow()ZnM0qMs&}4U(h3nCxIwsCR z-+Em%<>UQ1n7!n1Ez7XKVTG!UmtvZwX^gJG+q}#Ezq!nZ|K(`k5uo?}Em$C5VBo5L z*oaFECCA3i8(n+zF|72bxLiXX`G48kHM4IsQnzfCCrF}tX>ye_X}g;Q)UY!rg-_~d zcoKjm0Xv5iXc^pN9I!EX0ER+=>lP4P+W+~#O8Ig#8YelCDvT!G{d5nX!(@toWI2y zs?Mxt%AG|U&zH`3U@Zlpt;f1*Hb-8BOMV;_1affCfhIoFEv1YrO4DDhg(~JdKPs>6 z7CO2rF*4eKvmUzIBtTwgWkh3rH^_$aM%E3O%k+d7$_Ie%r2t5K1H2Hx=?S43!~yxy zs^K`(jZ3Dk#kE$Ugu*IB(h5w9?ZVg`7pNY_cC+nEwEp`qGBW+#`i0}+-+74e3E^-K z{+PB|&|NHNWUYyIEB|Q9-P2?lbs@tYkAe{*u>RBz!^Euoa{AruWZ%Irf5O*f(j8rD zRtj*gbn<*!oKV%&FppWy)B`bY&{_=}_v@Y^p8z0 zfmXl$NMpU6>iXwt$nHdKV{q^%`JLHpA=Iw>sPVjI#Y(e)p*b?*Qk=;Ta`gh_xxPrE zqV?Ts*g&gb)g{?iw@R3P{{wfv)|cdk5HZx)c#B7p$;!561%$8md%5X+KwOcFF){># zvMl?xLQE7RpImL^mtX@rhb)As?`kx>X}F5qXItIo?(G3~2X5T~gj}#S!7ki>9U|xt z_8&ubt&3e=@Ri9RI9lkdG#IG{ms<1SBo{k>4KJ4}f}u(%z)Z$nC^NT@9oXGkf%QQR zX)Dc{sNJ;?W!K7&p`a(mkSK98p3J&2Z%(f3d0I$B zdNJiBso3{Ah_#C*u$(#h{|!NVv&kGou`FM}$1O#~u2E}KO9SSJe*8~8sFSs1M z_4IelR|j%8GX=WFp6wM_W@QCSqNd4M!3vU~ zxjX`|JoB?&>-|uiB*YCo12Usj>1$2&F(XSbWBLz7l1>KEB4R2Gq2`)6S2-VL;k4E; z5KNPGpMqk|#=0K~1zRx=(DalyglM~4jAKf+W>iAT`i*LW`)v_6;w8lt@f%ij3sqEZvk=9EL%s-ewU%bf=@lTz=XlvIzYo=_-?UM}`cw zh(D)^aH8fcK55pXH`e|(zS(@M)$DWm)5`b^m3PN|@uli~LUEs1ws&mP)HWw**fBS0 zE?~F=X|R7dRX8Q4ijgxtjX{|vZAL2idO1aP<{L8T1%7An8!4`p&7J8 z0EUN8_`~t;lv;tIx#4146n)qh+(UGhy%2JH@@#f$(assuai2DjdmuL_S>(su#%2vI z@+}xe+7e1@GX{oc_LoHb=2yz;(8X_t7M{*x@#X#+8k_6dl5+j>d(^P((1l4Bq0eei z6Bfa-gZ9(vYYp>h$ejjT7@itn#mfoV{yiO~hFDDxG51@8P#~+np@Zg8g>K~2hGzZ) z1Nr<=(W^VxFIvbIMxK?n9dz>lEd<}31a~Ux{g`;Xfd#$dje}4(4Q@W)-i(d{X7;yJ zx$2o2V)%5;4df~;F6W3ci(R*M>I-p z%SWd#)GkB-wvLFBM6J@{dP2}=VEr|oU~{qdw)E-O)Vb)A?2q1i$3%|_huYED<&szD z-|9LP7w~D(X?89f!FgwQCG0Lvm2@z zJZTL4M=hZa9E%_#ls|gW&lzz=5Imx3J%wQ@Hjxn+gL&MaATFXr-jv?fP6-{fupNVl zo{21IOVyIY+rAE^Q%0rN58z1wflCgA5uczbW=mv*tpBckGKs3OImV#j--+crCB72n zA3RvG`6*mi+4_l&PT9-Aux9UNBn5KI0uxJlWh&U-`s<79%meD%t2-|{D5WOCq-t-w z>H5b+COx&;k`SQ^2B8qWJ6TQg##wjCM1r`I8@j<24E9;eBveF+ve#3yHlK!C_kB>- z+bYHYSVunTT6D+Dn+bFH%aFBp!9K5P+nL>CVHkG<@{ zVq|Iv1NNMPsQX1x-IfXg4t?ytJSVTnHEL(rH;~}WYD-0*J~R4UA*hf-HkNxe+Hr0Q9t2)utsOV2aYppzv@hSMQ;d)CV$+7g# zI4mVQmN{7sSry&Vo~kNZI|#;u#5t^7D%Z!LX_xgB2BYT7m;3J^$cuf3$nFfO<%P?z zpMSz7>C+YNVXSyKCB<4#;UlKMUTRYX5|LH9XbgOCLvD$}b1_)ZCl+*FcRGflMo%il zSlGutH2_L4SE`mjg?I8of%@or-6wWkyV=@dV!TX=`u|*R(3#7k;Jg0S(87OhoGMf` zx$Ee&njX=JUhlO~cE$WTCl02OqRb*e4E zSTfm#y9i1xqvtJPJh0)#0e0)&WBI_o(dop#e3x(dE@*JnrD0s(d^Fd zFDu6S*EunmIjV?>xgfp`wcB&WsL@GUVc%WxXb6a#xOUJeT!;KXw%4iI)Wc+(;th%F zR+haM20eTGfaLyS>1yyf*Q`BdR5{^R%1C*c4uK4nt%cUyyhNjWU(5oZ?wcp=Q?U1y z|G7as4TwyIHB&|0dTP$AFT$NMG!Du*(hvvp!RE`;)~K$t^xY;BW(-QuVI7`3UTMJe z-C+#zpf}FcS(RRH$*J}n?^O0`^+jM8FI3JFy|k1|DrV4|gs8Ck6SpUXz^aH979D3S zzKzzaT%HOS^qTs_8+~&?~0V zs?^a9Qw#!q7o3$9LTEgSq`JFBz%0H>A}`{~qfN&32l3DAF_(2!)_hy8!;H)5wbpvZ_YpkeSJp4?5d`GBrIh9ByaSYk&`Tu4Cx!utH zO0gIfH<6K7)hlW6t%L!ro%29pr+1rM>xPa~ZYYF*oOJeqlMm^d}Z9*`#T~;pS9(3_z zIs2f|$HMMkkN>TvNxwb!AcnRWbvuv+*4rsII&&Yz{`fI-Zal%aiQjj-mQ&@c#gKDk z!NvayD+S?Re0je_N=IPvD4OF`8kO{`&r}F`Ujt-SSURB>HjWr8$up-65mqrm*@Rso zOcaD9l&wQ26bit`Xv5j5KD1iBw$khs*aMB?R~)q>7KIc0cDBbPp8dS2Jd)?cPveDfLtcCrz zVW!!KsWSGZr?2IfXjA$+vM=6NMr=-oSX&Cz<2WI1_&HOT1Q$3+G@xU#@f|7B^I`KA zXu@kPywA!2@C{eE%-z=($nrzmJHK491JD_#S~1C;`N!&UXE)|pw8d3t#;aX$Hk4YY zCSUq+HiNcGYr9@O`s_m4g^UF0EKZzLIHPtw&Ssc0>YGcV+NL5#C8|l}5qO$Q?aG#+ z*<+(Lx52!GIA;iCYlWFZaXF6fc#r}e+7jegSWPds9$)xn%X{F*!GlNcnZU=OnBDbU z5asTDuV=`?h@;FLDhexS038shm`Tz}2?m`ixdn!Ww*z^Ux854iDech(1b>kHun2j7 zwK2w#Dzvr64As(5gos}u0#MMEqf9EB2S{?h{YZkXr5n~+UUjRR*)|y}Xel1u@_xws zCEy+%2-#byr;H0{C=*dWAgnok$4T}@HN;;?ve0{B5+%=O4f1FjfFd+NiB3zX2G%^O zWm7yX02{8&Ok`&gC`g)@x}|KC<-41Z+3Ny2T4JOe=iXm=cSQ2pl^Z4fz3N#y2{&!G zfUez{#lCE7lH*oh0-8iBS~;^PRw(nc)gf0>$!$Uexho<2GT|?oh0VxoE6?~&XpTXa znHPN+7=L(D7q0cOeLD~cXY*mi*^n0v785*}{$(#;96+iKgyj=>Zdla`F|%+IYwrnx zX>0ANWAKlfN?T|)z&%Te8mNM&`R`4>U+fd5k+HxdSh|0U9U$l>$w5D=AyvARH)5tG zF}uO+aw9L7Vp>>MicKqNlD*l5gerk?C%{1jeXxKFAa=koeV~}rSXOAZh=|M^<-$kPaYUQ)w{#xG%25vofnbN~tWX zY^&W%utGue4GwWB# zlmi6L&6)m@P8JoGD!Ai~92PxL<_?QCkxInBa>S({Z4!QzeEIdAJ?kiyp8)yt?npU! zi}UqY+kKo481qc1vzB*rf%gbBN=)J`k_gc?bq$Q@q|Sdc_snMMR{xiwe)r2QKsTyK zLB`0G>Y*v^=V~W!m51!hjDqXYv-!)au-;UV&oVLbeo9p~6+QA_jFpxB89e(JAiZ?| zi4&-5&B=~(pmdPX6uMQCb7}7nN8&FE-4aQLvdI1WKVJmEw15wysfeW_u>v#yompK4 z`q2GoQ2~!_ed$-753BXTA8dHlktll~NZsv^mVT1vzz+Hi?^PcYAVKt06rbN>z)4lK zx6_e3Sey8MEEc0O7rQ^?h-pCloo_0oPY?2KjGyVdnmLcu!z4LcOkby&YrPdn$R^!I zQ&pTtyhN0CE4V>9u>)uWi6pAnw1N_|B5G3;L1U;grUr45X6^&B`GHhKlSiW~brShXBHzuMAi2!d;9XEOh5FnhuK6^Jw`Sp(IxfGE1as^ zzvagf)I86w3mGfrV~yk*PRQ=t&~aais>p~7&N+0UmI6`ODUm*7iTKs0rnN; zE&UBct@)25NrA%J;^Zi4T}PX8jm^K&dSf0qfj05#WnmT}kpBwlCESdabhYWaRI4&K z(8H+l3A=O#vh#O3NJspY&m+>Vb@QHk$LaqG?Lvu(GVrpW=+cVKYM z8|a9Hen+RLJ4Z?ETg$ENg{lW#AXTWDR>&H6_&)&z?!D z-5|ni;goT^dUDu8mZ=K?m)-S^hi35kMe$@kY{-U^orWX=GZ4-YFCSX2ax8E{g8OgI zj3O@*?d(+ZtjAp%uPiG=mp{N>88)udY2iQk$e%)GAfUt;x2QGI81EqJPYc^C^Z zt*YUyde<7_IS~`vl~QmdHQT1a>7n%;&TX_G+N10DzepYBS=I+o_9GcNIWRI4S3r0n z4rQ_nDPZM|GkX!{*(pS$vj8;3Q?$ac(1OpkWKx+_!>8E;C!v0?F>^&w*RmBo=bX^w zvh_sn%X`KZ_st7;h;gC*ROh|Be9j9B99%Xx&-(g`!6ocRqaYl(m)tG`UtTjq7~6q> z{*b15W-;S2Y!%(IVuAG`>f1HT^v^RdB{>>7b!;e)}4J*bXETTH36 zt;a^}-`n*akr%d=I3MM|6KiPO zkyuMB!|iuyNR-Zenoon3)$B~%X6CUgw>lM@oHM+os=boBdiE|?gE^yY)YVz1Be~Mk z7v7m%c}njguvWV?5Quv9X!6Xim`PAJy6#1oGDX{lV04}5A2F_lTSqc^37NwVQgaJ2 zXVsjdFK{w&gQm_oTY=@nuN=*>g*8hfPIA?Yt5%4~cj_U!sVctTPKUVE&Ym%Kt!x0R zVQB0-dw67}sEuuBZ$r|DzwDQLYE8`w{FS<)(PVBRdJR-x!BA27Oiz>A);6~WP~+j7 zIusb(c$a*+_*K_j1PCDCg$A=3S4B{iy%vt+{i&$bd9I##OQfcT7`wp81E7KiE+td2 z$;3DWc0sx`>a&q=n+E%nYi)T!gcw92;9#{-><%poQa0pdfGgW+73GJ$C-}1P@0BOH z)*?3{lHw6HGi^d|H7;hS z)^vpC11o@&Dw}?33CR+h(gb~0C#{m49=faDs#Im{%>Vrh8E0_dT;+FbJpCEgbpT}k z+&1e{xwxw%Hy#>F_xToi zl7W3H`B%G?SlQ$0{|t{@1Crnrq>6-{|W+;tE>yagyMP#11+npK^cm&z*} zaj=*F>R`70l9=MY^3ZSG-r0TWUbQuj`9oZRwjc3@-X0|!*Xf&j!q}iCiY5zY z>yC4P!45#y1RT8JU%gCeyJW&x^&{qf=vWxHXckToXlwtgOIC>k-Gu5#Y9ONY;$vod zdR&oh<%y*w3V^;KY;SIC^tCUy`O_~ilkB(qePGbUHCm^<2AqSSZ>>#ehhj}|`ni^6 z8tFr#arQc=|3IVZ?ZutL&w;)vI4Ju%x0{xSJ&(u7E57r<#tSmKP~c#Fw^xpT9pbgP z=2hgusr~e~Sh0sif35Rs|re0q$d&(`9Mvx%LuV(eJyKK5Kd)|6EX~ zbfv0NH>JBkhz3r91VzU|h$I>y8Fv}$ZH4>$6|nFgXcX2cd1tcAFAU@wzROod_76+H zYOers-~GN$rTLnTyNr!Fz5;RBseCibL^u`JCaZ<}o2~2uEHY!EO%h}BDkyfoGljY)nkG?P*XoON@PyY(XK>b5k=u*^|B?q5QVs3DGL&xR1@VPBCsRo~J84 zu`;%u$vMBkQ}qYoynXri#P@X9zGv}rB_w>QWY*4&!`;86>beC(1pJP2Awg=-Q*1sqEHm?mYSpLyUlnE&^2Ntl z8V~Hx+wzM8{#c|{;VfM?sbd>W@*4-9J!*&gkt*c%{lwHeZAt&#g!woNheh^5OUZ%8 z#~^<29`aY-A%NWnt6nKfU)Rzn?*0IoQEkMyc)HQqs9Ty=a?CU6|Ih z@TVI)kgq(juqdE$*8CMQ-$?I_q7qE<)?IHs`HT!n4 z!g04g@w1|yEt;7I5HPtp-ZBL|U&n%~7a&-%_t(-#AO{p}Bc>u^pYba>FO`_OU@|k% zBRq`=p|}UZhOZ8z|6`ihx;6mH<)eJ`&A;9ILrICdTfa!wSR|2Kl2rk|#g8k~vDw^< z)`PUoZ#d?cHY(0Xs7(%8%i+9+&N38~Q_31IKiu9RHSweVT7|TR9mp5bTqC!gRq_h5 z@5?g8j%4P4CPkaJthPRRS_dr7uvRO+yU-Hz?L{C9r|!k)fUR}-U~sBU&ZT%6osA?0 z6+$OhlQ5S4l{T}Ms$JEGYGy{OXK&C>}`CMF^UEq zkoL~!*K8qZ_hso=dXBMYKTK%FL<%wB3_2}n;qtUoKBV$AM`8OrKbxtAACv`fRx_*h z`<=m+!G`>+L6~pV{rfdLJ3AjuZ4)eGTec5p|ESWJ3!9YF`(GQ&8qg z{voxljY!qWnctk9DzuUogRIpckVzA8B>*kQgK z`Q@ZvxPlyh6I(l4OTOEGZ9%XghL6?KC{x`^X1$m#^(Gb6McTg?2{f=r#z#@O=cQ1sqX zuIkn}#CpP`S>N`F zN$j_vo<`B~o0GA|!i5h3+9hay9|b(aD2UjU+0^kAtdVtuSJD$r3B*lGXNC7=|K*Z0 z1?bB})?3V8=l^vjsNRu?r|!(B8Jaqmmhe3W;(|;+e<)G`X&zkWoQnH7U93&r+b}&< zG6cUt8S*U9I zKS{KVcZfi_vyJP&6ZPd$S5FRgQ8@vos;mFL?dcgR2{r0g5A}t|K7Cfjplc0aQr)KX zJ*e@PC_bq#mho~NU<@8maRD*+dC&Pl?BKw4FV(vPHqT4Ir~dYl8&mHYC;&#G`az(M(LZxSMR zhRN^He{SaG>cne{hX_1cc0Kt=6c3%DcI!JY9V+W67 z)LNUe{ttf3lMj>i^5>-IN)hAiZzdXeECj$)(AWgUAb>hWgo5%F4F%c7SCcB;7r(`? z({4p-#$0&&b6nLGIPM;Cd3@3)kb4pGb4^tIHXV4RbSOLd-`uk(Ls>84M!i6c)Hmxzi z!FNWyP8_NP_XBBkQ)Q|wG^Lm#`ehlnt!G31xGJt@~ zoXcH*abuz2z{X)vjIB~D@DYJW*c~i~D-xt_fuMhrJLq-?NY(kHe&nTrNZc+O&Hiw^{p)|%<@A5ga{GVMETu65 zn1oCL{S^h&{J_B}m)GE6iTEJ^E`Bu?DRp|cU4UMU&OryRFzxQ zE{w+`DiQ{Wl1HVHlG05H(%nc4NOvQx2#5&iM%oQXhk(+Z0@ASw>5|yAbaUs%`<;95 z7~dV^-v1wWe2z0b$h-G?SIjlzna^Bnv#W-T$1@p&Hl^4pnGpb#MnzuGW4Q1(xlQRR z_u%Npfu-$@9zP;MmwC$FypogQ7EWOrjZjOLUi;`z-cliKbVZ2fLVaoA6%6m!MBsXpF0lwqwtNKf#F z@%a3wT2(}p+G`5~QF~b@!g5Y`H#%7X?pSK&XS|P>{6bFd$adpbrt7G^4{O5Oa^!jJ ziOz8Ubu6dpYN@@D&}qGq$QwD{F}Enx?&(xu!W*-ucbduKC{fWW_iacMjua2O7nF>9 zu2Gw!^5n-j{YW6y zL6?0qby)|GwpBsLO$n+Oy>PKvX;W-Ai4Y?|26$^Jr@r3wC?_u41c%l~64b}M`}Dq@ ze{>*Kmg|}xTiAZy)mLv!y5cz<=a$bMA9?`{P}%>cp1pFld_?eI9j$F!(Nvs6Oy2H&fEk2+u{8N)XT%`T z`pu84(gnwli@)j=0RH)+VY_6)VX1$=l04uu_21RhmkgX-3iNMp9;j)IIjuNH5!}uv zCJ`7uQdH>)N3J)L>O*j78W;JS=<)m+d$HS54ux3xX{0b=g_e^??3-cfBJjir;SWqj{yki`G72 zKium|%pE5O*|y*l7M*%aTvcL?BIn!5F-Mzxg8UO|c=>;|*SCk38CBAL>tr&9r{|XT z;;Al5Nx4}sYio!7{4^K<$90}%Q@Wq{8@L^q&s&#_4Kyd%slpAU)S^m-CkIQOhaIk! zt=*^b9xu>xymU$Aww1}|dg$#z{Z;IBYi=9~LZ?TT(xVb<(qK}AHs4)gE!!GpDH-kC zJwHM8#%ymrr0LxYmIPP7f82iiYXsVsE0vj??#*<8v#f1hHKc7@Q+rxG;r%^`o~nPS zr+&n?>U2%w)MHnGm$hn3xte;tw*KoBxxntIj3}4m+!rFjJ-44<31kEr_bU&_TxUv; zMYsAjCUVOcrBTCLa8omu?ChSgJ8`;KvDW0ZJ66hPQxnEkc5=|hGiDX8boy}rO@qR+ zP-4Q`A@A5$9z@N}C&#K+|8C$J3e9ur7C!vsK5Sny>N%JcO!yG|x$@GXFu{$9GRLJb zz8(A*E#U4J{%0Oal;sZSz!!BFOl@G_>5eY{PhQGt`9|v#YVTj2$)SkneKDlVB@@>i zO}Yg3-XBM5Sp>OG9Dm)D(sXE{B@|p(fS?^NMZ4Mq=h@8Oy6~8^JBdNM0&ATVH9dvE zX_K$<_L7zl%6tlL!hU51;8JLtAFpBrJVLQ|$It<+^AV1ChRE<;gz3LxGC2uBCE!F9%%_I{AyIdyMkcD=I*d5wU&o98DH_SN1Q;McxVc3WTFH$ ztsJwC*@#o;BQDFE9kKoA2uzejN%dt-QT+yd*y;sehxZNL$t=}wceMVwtD9alkmNjI zemc%t>U*;1Cmv=KdNakb6E4@-S$i}!9Sor!;RreQ<%Mu32J!1V$2;V@Ue1<-Ewg5+ z5etca(npL5>48*9R>fOfp9Q@ZCjA)kG5(}x6)Ov$92ZUwGZm;2Gx&G!b|r6iCZ8ut zA<;<5DM?H4VI5s8nJ}C1!u$$14`wmT&cTZcqtuROcJ2`?obcXS$`ch0r|apFKnosr zUzh2g3ItjTAN5)Jg(UFD|GAr$mmV&k^!bVTw??fBt*hPbFZVlK4P2BoQD+=4x@##d9#wS zR=pTfT^5rOQBeYd%`!H*dku*Z8aCCxwg;sK<2a+~<&q2<@a_@S9Ep_6*Bp!Qj!6ie zNLIN=jL$FSp-0O)JTh`pLwM6NRWe=I_PHQ_`g7d@@z$c)?zoH$ixdS!wA-fc-jJBR%g3JC`CD3U-tQCbWr~urJ%3h!HMW+1wAce8K zZ2RdY6KVS1j9%FtJombilS3`^h8J4sF|c+yB?}d6W*yvdd5ll~(kte^6uY}V=IS)% z-!WME>-vL^fl%xlaE9T2>#&_6Q>X9OwpJ@dMRR57KRu4p5s_ij-8tCaF5K!ZoC!@m zg_9Szx}R{_P*|5URvo)c)u+*wFR>o%Xb+o^0|z^aJ>}qFPTRWpb_j)(QByZlcK0=j zZYqy%&@ThpXy0?9qNhjnA-Az}7p32;?*mapZi=Hf6v?vXz0!s@&79u8!1_FuA4(P{LR9Mt@#HrWg98A zOMmqi7LsHA2}J_9hWJLjgs9WWzIjRV>nlM!ceH3%OlZWEhR;gPGH|tCd*5(^>jO_V z6Hb?_cE`XxY5V4lXTCazMd)}+W zExo@#WL?Tge3MbDQ1|M>1+j0wq8Pci#OA#p#T9 zk0=yeTy{GP; z?Yf$xqoYj|yaptx;0rXj{m+GxQc#SR+8g=!och>8NcbXIJVaIu+p%87Ta(1P6h^F ztp;<)gd`rMC?uQ!Zo^KFr3*@p^TH0 z6ChK0I%RjwH|IJcnA9Giux?^d6aM*6jS~|>JFoBovr5HEx?6^aLL?E*5C)BKmLAN- z%*pu&F3rhFp;Fm`N!RvgqogCGg@xg-OciDi7ZMf=1eSq9^_L9ty=aB2<*YO%BIO3X z)AZ|5%HRs-C%7I0m+*Q|j?3^!vf_HBui!@5hc`-xy zyug0STVng4thsvq`Nlgw=?Y8Vn)}KM+_w~`>Ip|Kz&DY;^}dHOyb9s;mYo#F9a*82 z{rRaD{SF86yK95>{;`i#_*^oSX$3all{+aNY{^fs!Iw^g>wVpHD$??52BP#^C-if) zv*pOu@>mD;C8qiqJEA`i6%=`tAe?&>g+o7~;S045iR_cCJ|~$9f}p806z>38M62jQ z_l=XK+H)bdK5G{Aef^9}s5(}jb{@oa4?nj7r-Gc9>zVJyEV|e_mg5QZ!r8LK>0qHb!bx%{Khxkze+JyMA!`aUw@yHe`y-#rJ_+w;-_d?7w+uN?hSEKB%(TkaL^|2- z&rhU3h@?tq`uL2M8X&;k^({$e-zy)hQ2Um2u&q+#jc`+B(TXPA`e%RBvJtNXmF<5; z?Q5dCscz$bdLkC7|N7z#A_Pz!Xs%`vA;}j^QlPmP8(R(4rR+XYv>D#r&1FD-Q1i}_UilLMKRVe<(=7TqlA zh(B+}gEzl*oL4Q&&(%eTQjQPZq%rQw<+kiA{kGWGlW&}cT{E`}czy~GU1Uj^VTEf> zbDOC+QB5&NM_lH}au>CkrdYu-Co=e47wIzu&L^8e*8c2t&;3F}|K3#i;Jy`)*KrY& zevq?kp_g`h$rN!FCY?T+-L33NplU@im5+{0Bm=__CoD9 zYmJm{RSxZMX6fr+xMG1(PL(xBgtS#{N7_sbSmrU>>J?d;Bfu^7HJ!dMrN?|%B!YSA zz!@&0lZ}y0=SWwf?}*Rgwp2}hi(!BcQoX64L>U_P6=n|P4eqQkJ?(ZPOTK$gt)iHa zvbQOONvrQ{910DZ$VzlDo5jAv@kVf7$pxvGFSQCKV;*FBd!dAE&)%2*nceNPQu5ZA zYb=*kkwr}CG*xf({H?%~y3O`I@BT0(-L}c&xgsy;?sF zIT6G2UC^kuUtbh7g$4aBL?nE1`ySc7rJeN8X99fCq4EkQF z-p7UJ;Y=DlXG088y2U>ZB@wXYU!HG`8+0NLGt|sddNMUN{Ot)(9EJ;Cvig=*Dt@Jv z#Ovsr`)(QwO>9RRTTNfCZmht_Stod#@cm4obgb%kfPB1a=clw}w=^DWr^S!ORvc%e z)P>2O;v zq7~%k;r(_R?d)z}VY`(%4ws>w2#I(-iieBP-;7$Ly8g^U1HNB6!bMgl>Os{cPL39% zsXT}^2CmS=u?ec5%&b0;*38dPp+`UBPk_1K>S>QjbA*Sbw0M3JctnG*FT4L1yYM%x zFG;ymPo~On<&YEB&D?!rurApawX|bhwS~k`j8A`SR?S43w6mX$>R9m>e{U>zmag{g z2hp8VVzIoS^U|JR+ogtfwF&~3a34MmzKL*^C+CSiTM!cl{CN8NvK20w;soeNl`_Al zTcZ?YqGJDjXu55&zn>{h0uSXI0uGK{(RZM|QGEZp%`ckCcqAY1`eP{c(jAPghmxXi zq2b6joYBkh&<@8hTs8O0(xCIC@7cIlU)N6>)Ylh!@m*wpDE7@zIxbQTD^RYmH=;Y@ zNNyy9nwei$rj@a1Cj9dfXs6-mci=0(2$nc_NtT@f(g3`)xYLTU}CX)DO+E@j*NAcZzl$_pE^Lu zNDiv677dar`!rfPo#&ImtTX|+$wR;J@6{!|HhPM4K)sEPM zg?mz<#%-%o?`t|OMdA{vA8auxiyrO_QF|^&>v_n93D8M>{(SITLQKqj$I5vj(f`U? zNmN5%e@%^aMDJ{y#kfwUO18W6$^`YUZ(CG)_9Ci@jnjS{YWF za-SA_qmI~-^icDA?1dc@bf@v9|J3)({s9)p?uR`M>GI|yJi3(`1`Vp;l1w`LCVZB* z2Umy$og{0H)7qkFK`S3`h0_$7L%ym%rR8|X;n$l;7W8Zz8ILR_!t5uz^+Yaxe0Kyh z(rGhou}>+S?f^r7e2{&URW)6Hhn|ouMJ{d&tq{wruvVG8I>znt<-Fc@gJgI>JuK4I z!1_rkil?{mNXnP&<`?_bin3^`4zjdL2ErAx%vG$%ENV*ZXK3`a%cX9pWhQ~=U=@vU z>08Q!J~~gmqLV|Wd^3u;9~18`jePYAuz*pH=4A0k%F)rvGK+z}_jh)hrmgDg&V&k9 zQ(?Nc{vXB{8x=OVe@iN-$?n9(a4yYt{7rqdesuJaE+qf?w~6I(U1YIsREk*eDne{I zLkVJUsaOGR4vyA1-qqZ>4(F9v9vfN10MXA_`rMJWF9t{Q^@5p{bYwfD--wm#*w$u2 zp1Q-A(X!dKJna%1%yi>kTUVwEa`j*!p09)kC9t%|e;RSuKPgG)+dXI^tQOyoKB($P zEY!`6n5e|_BR>jO6%zZ#+n(%7-|;g{=c8(>VfzFkl#~QUk)}U4r{;7Znq#LTDT&MG(<{IJlprgfj#!f2 zFWri;BZrG9uOFSu)l&<=&;1D}m-|Y9WxC$n$^P9ttj4j~o0DG$dH3E5Z*npTNosU- z=|u^#Mi%AP_rgnqmWM8KF&mSVy5%X>Bg;ZQ8$nt{${n#tK+IY{Y1&q2x~u?P$)wTg zjpG#aU~yPYB^{K8RQU^)8}y68Y|Er(!MdGWYu6o?26TC;dx|XoU7_pRwrF+90=tBH7)xPdP{Hxw44^b74i2M}o_i|{p+7l7JB0>1Zlr9?^{L*3uIJ$+#DK1_~BNGZ8Ig^)QKURa0drAwSGE@z^7^&!)xdgTJj^Bu4&8P z`W3EKu1d5&!7X6FfM7JLG zswq(zG!tUuoHMm-Ace%flU~7r(a?0LOeKOTh7fD_GW+5)iiq8^TKmIY`*oO)o zv8&Fb1%p4!RE*o0*t^dxq${|+{M?onyEa&lJ#cX9jVa9NX%f3ze7Q5N`L;y^M`e4C zwpK|fRzcXY;rY!=p|ohF&89@4V{Mf$%AJGnVQWuT#lY0c#aRs87n6tsW16O*5zntv z1n%Rc9x-kh7?fO{p1hja&M6kKYHr|&@ce6pm4mLkAKA;rnIHm7q93XB^ehg6mjLg> z+HX}9-_m>n$u;PmMzo-A2syI~-5bQoXQ1x;vx^M;Fck6k>BRNDGgrrEn@_P(`tNU= zk;aJg$Xq=ETA4Ij8L+ZX;hcqPFXxSv{O25JUKb0PnURfIi|)tTC0QaD&`SJ@&r>$$ zHRd}H)}NrENCsz=#J_GvEhBNXY_o_4}wtv59kEL@uB6873$&V+Q)z1a4S9F zH?C3B!cPvbgG?BQ(Y(CZZ+`$4i=q8>9PqhEDfdBD7H#VF0EQH6F4b7I)Y={}6@vHfD+&rQ&I!Y1 z8F`bX_p1SnAe=?JuQ(?F|Fbkm91WoiL=vtOkftg_QusFy|*_csts$S;M`r%2oI^zomv^n&GpTNJ)QZo`PS-rnjh5Br3 zjJ(m09PKQ1IgMPiX2Mj3KT*j=*c4^u&P6^oJl4u5M`_8?-6=slHhqyS*jNWUA6?&t zyYy_v`HfqWveb%f6EF<9aK{xHS9g+FtWW}WRo*)hTPagNCq8IzKwA8j2&0)8}@dRB$YMDs+_`bE8YzwtNU8nG` zqfe2|6bpu+68zUNp;-AD*+8OfLoskxmeeB{49+5|c?iSb`r?4XrV{8Y%Era-M)9Sc ze^gibB1=*DHmAjs`fQ8Q*(Ws;+4ru-$xV2ruT`T@nv>@XK z-~$`I`~$l0iaOLl)Z>W4ek=->Z>Nb z-dP5OINMs=c{#&zUiPANRtAID)(7d2imD$$TkWo%p%QMCcz$7k2yvR2op@UROuQ?$ zH5>Idhy%s=fyYL*1qqH$UKO~RYAJ8dV1RK@wdh!Gbe-Sgkji;naI;o^8X&qG_782H zm(s|%KXN1!Qgp9RNym>m(LtZtKv6jh3zKn6j}$X%K7cJ+Y+dA1fI%ZS z<_hUzAa$XEA09C~3J|f{Y^7WONEx@9wcUgPT%cH;ogda}Nja{IV5Kk#)9!$JS<-R#mxFnS+qv_D3G;2b3}gOKep~`5V7_ zz6ZH4v7iJ|Q#mDir;~mLytd<9l%OZf+_XdJ*Qu9 zvW;}gk}lr>FvnqjU<-3Faq6^@GVU&usbV=<`w}U5vS*}mhSmu1wB^g1`g)l6BsBuV z9txjWz`X=1XbE?odnX;K8pi`dsFuz2@TaE7#J4Bpyo+Fztzwi@mBD6>X|ZUH9)4wm zLjIhRafJ7gTjYJsPb648ix_uwCQRsrL91|Kx-mEb4gvA8ov4J97cG149;!YW%Foi# z%HF1fd7W`S*`Dj%{fXwm-a3;j6_2qfSnzsrvZV? znh!2qU{qr|^g*ZCKb`cyLdpY`jzIUDZH;jJnFw9lBL#WM)hR>XlL%Ex7O&cy4o)ke z5rIleBSS+TtDnyhfU)|zP8Vi?P(OXlOj> zHm; zr|dy!5Jn|%&_E;pC&eN3<~XtCD~dzq&L3OK+--MwY61{p-tV z1^h_33<=o8yys5pNof)*FZf+_!!f zm?~rfHfD-lWL#q0i6S_k0tUgRTfWsopTLD`G%z$Yg!zc%G?lltzz!8x>6RzB0w`CY zbOJvOElgnrd5VrWUj1UL7+#V)@DUXTd_g8Iu0qAnBG3i6Dk}$bKNLLSkfp)AwV{G$ zCkSS7!kYa@Jo|th2npiOfgJ5P!f9forguXXu55vV;Ir~0Om2DhGzODy{%V1`nP|N# z_n6M<`z+eUqfnVS)ew08{CT;vem}#4ps+rFU%V)^JL;I`_pnET+RF(Dsj%r{&X#%z zu%<`6Hlshn>AT`=tM?C<^5QzDb-IAH1cxTcqYxzjvDoQ`0d@19hbtvYN=on^9?I_+ z@prebQBesx{E|FdYyMmQbtS-KpWo~f@&O97G7J$Bj5h9wb*}Y2mjGNc5R^YXQ-vAb zKr0AA9wXD|^d!x%^v7hKpGvCqb|>H3A0fR(D-R)%A_En_xNpy#%;g=}Tr$jq}$i8 zU%ihHjN9PMMg<`PA|l?{PL`*UMbGN~5)!J((Jl!H3~c`XT~a#o{!pFYML7xBe|8eY zxOomINBgH#jZ~DBULa+><8JqWa42}E3J_!zbtY0{iSD5mn)RW(49O-UAefH4O>q*Em4*BR~m!uNdp< zT7!uN%~BD(SsDSk#(u~ne%`zk7!Yt$A_imgt)El_+jcl%dFyb0KZKHRpn2p~%H-r^ z1hWQ@&FIgGiPM_hrtjY=1l;?!GvN!P;6!GB67U%4TFaftOh}l>P)gSG{4J*PVN@ED zp|H*s&Py&`6~s&HQw_Oc*aXNNb^z@$oTqvkGiz(WgkA&E>V(eS5~G5lYWSz?_>1}Y zl@=r|pWUQykSM&wCIq^L)gticumD~Hhlz2!tv@UHFp37H{X|6Mq>7e~F5m_&?5QJC z=+Tj5Bn?b`l%Bi!-ujfuW@!N&m%~a!^W6k)dFGLV)$`b0<)(y?t#;+-b%Z%Szfvrh zxy@+l6y#Q+@~J0TA})019M+O#sbw!LEWjeef;d?7!5jgmSt+5as=CaGEO%bo!5X~K z84j49r?8VjY&0W*xakAZ(BX6!de19jP&x>JId7NK8=5kQ**#KyVg~JE4|Lj$QPfZ= z1h5t(13@!2hq<42m8$FvCc=aSuTgpbE}K`M?a*%4tgtUUr6|J&!*0X@9{#U0w0Y5Y zkY+hhJs{=atoCvWRmw+h(>j8Lx)rOWIzhb~*95~lS4Y2=v{|_F>HjeM#N@w&w znm6^>-~J4c-&wcNCewTGY}*A^YVz zG3K%M%?L86%<4aFca}2|&H#8?@vIDQE6#x16v7L6rPe5o){pr^&kGpux4+D*x;To_$cM<9$9aWGYH4tUI`UBgWRbqJ8X9srrjhZeCW~7PDd%Wm!W9Z# za>U=c)!vxz>@PGgx4-u-59`u@#X+CrSDS0sszSuCC@C$D?&q@Y&NR#PR(l2QEjbjJ zv?9eU2N+&B93PkHxd}VY@moBj{(Z2tIJB9|5`2@<%Uxji)A1a5+p5U{$Ag1uAxTM8 zF6)+qTkV_xCUcr*nJFDMl{xk&`6FCBYaI`_(ZfZ<`LhPQF!7Ug4ETgRo~2A018H(n z)CNoHIXanMhadUNXpGx4GF3ENkxu$PIWIOp^5mQk0Q8ugXJOsIxPz?Zo3RXF5{xQT zM_bQZDBi>B)iy#dQW8E;f1${HQ_@eT^dG6EfktboNLdu0P;a!L<%-tINV;uxPtc7p zL|>PXW!+!f7)K55H~ppG@JPM;enTUpH~i}ia`gw_TqEON1W?|)^1*Y!n0}9{rM^{8 zg_;cvfL`ith1Qk{c5jNjDu#>I+9Dw`1<20LUP`N4r$4)ylJWP5E&PrrE|jWBXlW_0 zKNl76qu2P>{FmfmlWs*{RmI{!|ISKlf<6QsbmB-lIpsh?2tR107Y6lj9Ga8{6AATY zs*v$Ny>MkQHEI>{b*jc@Ot~lNvP;2QO;n0_$aKx_Y!t-R<*rd3X+6obm`Pz+|G187 zx)thlc}O6p%4U*rJFPz(-RX$iZs&xQT$uxi0M*U~Fy{}+VUPesQhO#QCI-5%M=+Or z>+;$*IchY$i+!lX69*V)s*L57h!m5S>dTiq7r$tyhf;D|e4Oj-&r;L+Cej=x(+fbj z&uOWByqidu5Vfl#%;ie@m>)#9K+ znWK#ITjDGYw$-51k8e-MD?K&$H%Fp2q8RTdo_p4oCbiMkN0B)50{1Wun{rs z!-b)9fPTQKroA+W)aiM;;VFBonP*cP9fh^A~y4QEVJ215tRQfMAB;m5hYII|tus+r5Yu4^#;vFvqNpuIoa z=7g$5`=bM%LF;b>CiPGj1(SHO{L{Iz3kK*!huP zXc5ty((}z|ry@(O!tw%S?R`OMUW^vdmWg7LiLBr$w#g|pciy&0lHhZdfBCXCTFKiR zKnv^P3`>oUguFXXxV*}b`HV#-EZ5ee^a0fJKIU@!YSJ#bZa~T=-RpbqH^taePj8CS zOw(_+^B#%MTU1ltKz6++IqPW;$BW)H$Q_ist4&Se5ov7BSxk&`+yhPi@Tv@wBGFo# zMW1SSZL^COGZV(`n@I0L^iZ`{>6!7<2yS@1tG$^!Tt@m(_hymGV)N z>-dM1(X#%`@PS;lW8i#}1L{La@y5e2==#R);QAC03J9paSdY+{4|9RGZI1zDx%yCE zCCzL>QEK>Wo%eTZr>&|SqaRmMh}3-J{?C5pt`O}l4)=ev^<}miO8V=vh;LS(tdjQ8=DV3@Y;zv-spY#l1W|m20fte zUMFS8E63X!hit=L3Q=s3fRQS9mH~CKJ5iEw-F44y$jYKmQb8z(R9`jbfyMZ!y)6v7 z_Ff`;^asXp5S_)o%Yx6^ZyC3xNk^!ry`z)w1MpF`#e1;SesNSYjtA4m)}Qf#Nh8!l zeP{V#nR8_*`Q%tFIUtrxDdquW!Ts?t$Gd77?*j-S*8<7X+1AvR5jY{I>0QrS&C;YW z0oijQci~=gTcE2(-0=fQH^B$crjLx_=vpgN%~UG>c%Z+L zf`_tI?wQV({W;k?EBzHExJ1Q5+2RC1&&f9~axVfLDp^A4nLE~dG(MvMPU?w>^p&ig zCk&x{q-@l@*d=s>W4YRkNllv1by&fC@v+}U%Pq69^4Up)8>w>S`}^i|vh=I%%#=xg2ce0m$XO8A_%}q+)o{K|x4Kr4(-%s!~JgowU zye2S5JDrRhKBu1NqJ(XisL_7}X}CUF+Mq^lHZ4j|hxys#?S&&5^zePMiCm8UT054c zGydxK|D6{H-}%3C029Hl8dZ{gr=G_{S)Z&^nPD~bsp3joZ=K(f-tx{$IE~POAebk? zIAhH;&eEv&SDlwk5Z`rKrHoR?k>NE+q1a+;ZpXA6&AXf7Bc7@?A6uolWt;zJFSvek zRXtBnx5Q?QStB<^CaTlvrX={^w8KS~yEUiBU1!LN#xps;iYo*Ja&bH?)^%PfvM~=J z0Cj@Z#v%R8URu%J-HinY^ZKn8aR6Vgi1#uL3kv1hH+rkWaN)Pg0uSKZKm%Mc1Y)-Wt_1Gs~dC^V2?aulI=I zC+N!d?rv8Ea}?&EH~%(Ya+8bv)j;ghOad@u6QV*>b_eT(WBX9j#E2XP z75?Af$&KenZisc;vF#qd1?neYERV#H75~2M5aD$I6iqJwx-npR;^27V`PX?h>b-tR zE43Q%AMaRY=HlW4YA6P1WR!oqO2s@#rNNd3u3sa@-EF+qwaO^=miWxoPY^c1;<3BL zd=0e**bz#=2sO;i7$o>B?)*CykQns%w_9=LP-$IcIS8I<05(22n-t=CRcGz{*xGzS z|N6-^zyyA+ttBvPu%d}JNt52B6}$#o*$nKl$oC`1h$(>fIzYhS1iGm1SHB*@cpkb)&zCy~nwYjNX>Gm7&g0;~yBLwAYBa#NznC~yyi@r2! zU!E%U0#YekZfkqwvH&;0ao{Pp8EoSQ3jyoDC$y!%J4TwypOL;1$mJg{{D z(Skp_qaMJ&1>*4S%K>gcQii3zD1#go0pQ~bF~ik0U7olVs1n7;!*j5w8I`cJvwQk9 z`b^Ms9~;<_LPB77GkAXR1D#xKl)m@S_6s}eGg+U26(=QORxcc~+O>ke)+#sU-x4#v z6BPm60#*~-6)}H2lBDG1wV`N06$QNxZngLwP9{RmZ<3Dze>Dm4q}_J*@A7OkGLh}7 zG&HZz7LxCw46*`P$M+#^0CRFWganKPK+D_jUi(z+pVrsccP0q=03;6RV4%N$qYWUv zMp3M~vZkOY?f60RGu3m3A+9%4$hIgL3Lv8H+0iU&lnx7uJ(XCDSI*XIPF z-{a%sj=8#;s`dISk;aW48pDO=+3Goq)BG^cxdt)|d{=eYn3$O0%mhFp0Quo_-&*iG zi6ml0_1VLfCG+c z7|he3Uw48P%fUvYFY+VV216-_OS-CQE!9a_vJ4x+xqN#QTuP`HJz=xfI<$JS{JNpY z67O=1-M+(cbTm&OA!g71@KTrnHtAZD3#_LA{4~AWG>kqcNA&Wa_U6-SYT$Mi7C5%f zm>o*%d{vZgQOwky^V$qalkJ&pgXo`x88VE(C0gviu}g9aaIEHuj|lj5+9eify2F~9 z6JVfQ49E(2EVj@$B5Cgl!k)m2&SC5*0BbA2%CD1z1(6psa8nJZ11}PIbzJ6s)m6iV zN`AQVLp_}+U@3>w12J*7JrgpR>0!Mo_N~?L-^)a?8bcK_Ea6TXz)=8>^{!h&Q{%xx zD)dTjrGlqAj~ugPNa1Vx-oS}DSmiS zcJdX{t?6JgUitkEC|@F`S}n&BgadQ+PTScFtw#o|uqceFk(BcE@_5ebSdnGaa3Rzm zGAVzzi*JkChU`0^x;YxCmf%!Q@mJU8e*vWQ2+%7e6eZ_@*vH7j?wJ&AXXpO{{5YC!~CUY8e@_Du#V2#iU=&Zw+GWCSY=l zsx_7yC~XD?kbw2WZ2|q9xOZPGC0@W8gB08sJwA++pK&g>MuS(^OA8FPnP!C$>V$O3 z4`Cbg)E`;m@=c0P+v0UK^BFAFjoZzzM1NmD#amQXza$B6Cjq7T*~_;ioT#~KVABQ? z9&BrHTW6W0u{nqL^n9LAK_CuYAc4qZ8<|pQ9>ExXJkg0FAmcq@wH~hYIZkz)Up_rK z895ns!t^EipCLe>TEO1B@!&NakCuK03e-{_q%?(27C0tqBtn3}#H2vQqW49_UpD7 zxRsX%fWqpW2S%+{i2?s|xt%aLao;b8Q00*7v7_id0CB7>vfTbjTO^hfYR)e#`r4#J zS@Ml~j>2B3gogoXK%cg3@K^0raHI(RuOXVn5nr?e`L_IaNUY)f$y_U~#kR541*P zDb_i~(v;Z=6Ob=<-(k_oaNm|qw>OGRy?kS9K>O_*#dP__5sg&oN8W7A>I*1W$X^2w z55SH+I(ler^qT)2Dth@0cW3ZiXQfLXk|-m`Dl>su?_FimrM@|rhAzcA?8$*i$>1)- zUS90$92;wmaG$5l(a$sK*c$hcH^(N>LmSQ2)=0>*nACt7#{<}1@jr|VlG*B+YRpAu zpD$d2UU32|Mg|N_dbJm%)(>`^-L>BXD`OxBZCis`R09^uaB=@sgZu;|Ok8pz`{?X} zGq=r`EliOO%HbC~s!7Xb!n`%8 z53c@9QyBufRoei2U2s z`O=dtm;RwCLzBMT~Y@&qk^E)M0Hv+|0k%0aqI%BSbwfH zb`Ej;4MrbsFQp530ObS2z)E}#vRA4_StcD1C>vYkQOacwKtW9Kwm?W5RvzX;pccoI zi6sxgi@P>4^>!zfc*Kem)xgU|E9ux^lwq)Q9h-B^kM*mvz|bVUhJRIJl5=&798!yI zwu3bP0sYqLSK9pPplY-g7P}mrC{}V!)Efe*TDFVX~_QggwRWAF*rtG|3 zp=o?iPx5enBbHz{>EUS;F!hrI^yHu(C#nHexrMYD>;2V|=E$G#by(}WG^T|SWnzp<3-K%6x!y=M8R^xa`Y5tTJf+v zeTKLQ(F#Yk+AW(2^|^MkhuwUqmyQ{LSk(~=d$&I;CKRdq`7@K^W6-^N0wA?UBRpzZ zDSj8R#36`&y7OJak6ZpchB}MaMi7>h zU)#K=Zt%25PqSd-;_w~-3y=X0l+0Q%1DOdRz zng#5xiQG4r(2tPMSG2?=vR^o&AAR~_@FZ0VwAfjGr22ax-V;^qrbZ#s97|!*h)HHM z=@0cw6%_>GlnW=qd5`JKEIMXb;^;c9pz19!EfC|~5%$Mp^l(wojR2C-xaU@V4d}o9 zO}RkWhuAjHUu5ThA4t^%)l?!+Kow#rSb_C`@Q7@~CjAG0f3E-PoSmi0L0Mm=mYo(t z4g;zrVgC3a_3cd<5G|Ak41CclKjTeKa(ZU4QwT0IPcVjmF;f~V{R||odC3&r3aF`A zaxv8e25P0{-y2z8MgPGQSD{-9HA3-)V&WbE6V#Dxg%o` zy%b7n7njtILL`=~T(VHH1a_SHLhqS>R z`!qQq`NGCF?_J1DpN5QlZm~6x;j+~GYhNz0o|6d3ooREJ3f2LN$Qox=;QD z_66)$vU6$YOMirrOQQ~yEwXALJt>8 zMFN~#XrBxRs&%Bftn^iufi*rkg2NfW_D8bN0*g_?>*Ypmz1?;e4RA%auDS214t$A4 zs39!|Al(6ZH34_9NOCcTKC&e?T2)%i8sppF-(Lk@T%F(cN(m6*fbDm%6Oq!10$B$j z5sbIz3veW>JfLvpEcF3}WZO$wigz$C(PD&nx zZ~hKO6b#vD87-h86|9daZn+I!I<0=OpAS^u;X)+n2pW`N4lyIJo28=$8iUCMb)Y_L zWds=d0OngoLz>!s-x$EYylNz5YQVeksvXoT5SmnS(=h;9V^pC#_O~7X^8RYoe%i6g z+PHnQ!o<8QX|~uEul>}6-NB|1ND?#z%JddngHi*J9HiBK^$}2lX=R*d0wB|Nnh!aL z%`SQlyDv_dGnj&Pnwb-y*A|h;u9}eu7JxNJ*l(*>hU}ptR8kCnxaW#Wov)d=KLc4L z@bXl#>8($Xki@smYncCe{9=;{SZW<(6_8hp zr*Fg-kk=4ua6+Ayqf<)Rc7xzjJG{$4x`AXNrwPs8*I-gw^IBiGo1g(dK*F+_8p?6N zHG6;c!bN)87NMl1U-L?6dDy)hvnfKy(_+nER)PfN(EZuvZc4(!_7W*aa2j!z0yVsZ z+NV(AHtkRdxzy1QP!lwf5?;KXe+vtjzt~i|81zPQC?BdvZ_qO#w5*50)rUlsk!6F= zm}zEvm{69jzFe%+;Po`O#*+(P{aF<}0RhszD-oK+!SvD|b~w({FfvIa!3%!*m#Y`wsp)Hvk1W}f&!7_U_^E|da~Lqr zr4IItq@cLkaXw4wp7q2@Mqba2D}E`P^Wl42!$q_hpVMHEO^MLLu-n8bNAf3A6^hVEA9 z)3n^-#7E*14NNymM?$F$!^5W}%!^P>of%H2pniM+wZ1SNwJ%GXqZsb^_z+NoRki;4 zEa7G}j~sBT98Fj>lh&$L6yK*QM5PdrI~|up5$%7^yg+s!-*|fq4cuaKUSOk2K`9Gp zUg7gntsffBMdcG-QHi`ChIqr==xrhCe%XZ#Y4zHl)uA4?W4oN zu{fbmKxT$f_V$OHd3*4{jv%DsIb)^1n361zc3p`s*X1Ik#bBuVBeq%voo z=Ts;{DJnveX|c@nSQI-#GL<>X!ZOA(EsOVDYd_!b@B2G`f4uK|yzg^7$3807z3%&S ze}?Niuk$>wf_9=hsS)aouZbvD>=C~R0)oeq1wIZGrm_NAz`6T7McNyp!9C`bI-r9gT-=(L~x^NOC>JKsP1Oh@>+DCRE#>V zKx(oY(^;gF=ljOPnqm#GTN_v7%&2*Umkb1G@pmee->Vg&Ty0SzC|YRbIJ0;(G_I>FxD{y4d&>tywQatu8I%mya^uiP;V4CQ6ANN8 zP$+bB%-{ckQdQ6{ZAc{}8)Pm1#*D)3N>XmQyAEHnfB#&YGO00DgI~<*kRpxR<$Lk& zw<`D~AQjpYO=qxHk3tv_6nEmrq!JBINWnaswZ9I;q zh@@#8e^p!`I9EP(ccPo2ds$d-bSfa;&R9{>;e!*)ybHAe!FK%FBu8Gg_H@NK4>VwheD2-wR8Ix-7s>UIDF(%>`BM%`YF;GqPOZzjS>fW8e;~T=;x(m zg&I3A+mW9^y954Lg=RihzlG**Zj2(Ol*5$iB(yS$u09=vlFho zdTath#IL$vvpuKU8)NDsls-L;ZhS(Q&QM)qWUM3um6mCR;ZM=SB$DAFkWn`r0r2;BmSDXMvgf2y>zmycMXF#wjBy*U>raFmRBf zPBaIxZDK`W{Ar@QIsBmISBxN@T?s+pMS^B<)#KL`GTV-~UlWx{7qNjr29~I#OtQ;~ z5Wo%}DZ~)|R(vSjBtO;{&vO7@tAxZ84O88R_zqa>pb&512ze z8Tu8Uq|2&TAIEysIfW{GNeJXc^n&E3;lWnu#_7yhe8TIIE!%oUPASVz+YE|tNCrl~ zl+@&uJPa{Zr~>%Mh~Gg(_^=V#LjUL8SS@~QDc^1~awqOVNB?*erFDOrFT3GAe5&y7 zENIB>;Q5GUg_`Mpc|76WczL$H>4m+=7z4}oZ;h*Svxxb`wHTSSJ512W$d|fw7OMxc zdX8>wTHU$rjZIIU8PpEQjO|wzFTQpaemsUKf1Xu1|RGZ5$ zUfc9sRMAk9K1tXMC2P(Vn3r0WF8dyGY-Y*L$36lFMWh zKoS?u2N)uLWm%m~;E3536eHh@2pz+#CjYb5x`$Ba&cIG8>Sn?mlnJT{Xr(2)h}niV z6$X92nH#rDF9UPES^(4c00v z9NjOeXHZHwzqyTQB^l)SOc*jnG1rV_!@BWmRxxJ+=869%#y$59l`4hvA|t!4i%PR* z8iu4eOs@S)JmSD&2f!_tp;uyLr3kuPRJYx1+zUm=RvuVZ{X1Ty7CiF`?TEe>5QE-w zD4gPfdt|>sAb*}|MFOrT%Xg9=MO^LTU4xC4nY8E*&!Y?bf!HWaN-dIz53_W*&hptmb@1x zjxw(fcWEm}fmTG9@C7M3jisa~X3AJDSacwWU)dfvZ0q-3Czy`DfiWjv3d0Q=gv z*x@|%Om#byW7ETA2Sx|+5aoZqVCffzlzMp)GYl~ieFfEQ)XLNX!cMi0dvQ_mAAMW6 z4}fw9z11@Ctbd&LK}#-X2?9{QX$bOHx}QbA+zaCe>Xj9TRNx>2$1A+dJfc2G3rNCB zC!??olCJ zpv-k4ufa)l;gKlk$E{lFS9?kT>ExQhA>zs(Yw4;yPJ}72S;4Kg3+)>BXxz&tobI<4 zC_Xq&@BD;2VF5$8_!&6r@D{?n>VQ;j5%*b>BR9h8QdH#ywCo0A+rXaE(n^!H#xPMHbx(Kjee$}Vx9!I)%%;eQJ_!Yi$Kg`UU`jP~>8YL3^sMb{2!9XuH+NL}- zt^EF9O@9B!!^{tNN|lQm-$khVDIC5AT?NIG)u`zFdLF??XBpC{0w**?%5lt9(3#@_~3OVFj{GVmIb9?33UK6UeZB<)M&BnhWUlk4$60l zEuUYa&;nU|Ybs|k5APXzVS7wh0}o!d)lAXhEt|jzlOXVmJ_ZN3B-Vc^vX3~VtdW=E z@bl`>=ZEa1PB9G0U~Rrvu@6<*a~{04w#>{>&bIWIckjNM`{S`2%$qyust4QN+YE(- zI(>iA#NJ6;3fIW$e=Y4EWmvA4s4YU9=^#nFKjR5x(5L2nkzWoe0~gyP0YNKX@LYQ1 zt7bWYYXmLOWwFt>rCSWK{qWPFw`D8?8+BUvT9Mt@x0T5X>cB%j-jQ!dIb~o&p_rR**%nl;`}z)$ z^XdRo`S-dED4;jwUM#0Aiqp)*Z`0&MTPw60=KB~93aWUPTTl@wkc_aJmo*-0ba6I8$1GI`$E?ge-^6Da*c{E&3-UE=cc zayXxg&tT0KhWO~!-Ku_37_d#v%%mBV7T`AXK`U0L-%=rN=`-X#DCBc^{w3&j$<9vH zj7~}E4Z>%j@A<*BzHi5l9UthTf5!6;E=&)?;5tck;Sgi4?@ACoS%WLwyLd$4o zC;Ho9s(&s5f{-?pVi=a)|Lb>u)jyX*(I`p%9s7UXf;0gNC);mXew2$9j~LVnEZs0m zd`)w6Gvr8cV#DctV@H5OzSrV!+%DM2jku|QnMeQmcZZ~1UV9l-nP63Ru1|Q|bCZ24 z##8oZ&YTIas;*|Jo4~rL=2=8F8G0l`NOoGzTvxNR%i=wRUfy@K--t?w65Fh$LGh|yyYj-;Qf!nvXR>J5+)5u$J4m8x)FKF3-=mMT>gxtXZo?)N_W~ z@SDd}U$7|JgG)ltb0G(Bna-r$wjV|Q)n%{gfsc@stXaDrY@&qIet*z)a^RmZo8AkC zBc3Ig1zqB=rc3#wI= zpzg8UC;co0jZ{p}$Zx9=5a2CLZeFIZV(2Mx;p5DXq{c*{ee?ml>3di`iJ8HH*u6>q zz@O{aL*}OTwad2Bpu)?&#lBw{@WdX^!%C2pGNgreS$8(b-sln+x;A{5H+KKNeey$} zN0yhNhvGzdoPJ9&-(v7ZylDfHhS1%028oAN-;jnH5}ICF3{kir=`=*8UOPp%bGoA> ztzqmoZ_t)8epi%hzgchDeya}I1@9Si>U^4^OdwkT-2W^if_AxnHlN@>l%$e#>o_Zy zp!d}4k;9FJrXdbPZ$S&~7vLHVwhds`EARYtM*73txKV_NQLEFYWXG|(s2A+0;_Bw^ zJ(Wb|KXJ3hY`iP_%8zXpEm2o6FRnef$-XU6e%Is2bFyl?a6e%Thn@ci!XJG2{}p8v za-)744oa&`pg;^1V}rwK<8OJ2;;^jILObr3+KTBqHp)nX4{f&7%CbwVtFWsuj1x67 zjpR_o(jHC&E>6?}4hf&xGSgSGL>5jBV$`KbxU}6C*G#!C`Lyf7JP)!^NvAJXnJC@1 z5~Bf8+RlFud;InVd&8^LfGtQ8^a?c6w3dEuhpE|C9qxlDY+dq5^w4yMAz_HlX5w%!i6G%4~-XMzjKo}+66)781DkYeeY~(9f-Xe ziU?IkS`J#Q`sCZ%NYPFYr5vjbG1JAt>4hzy1PU(e|YL5wpi)zn> zQC+BiUJHJckiO+tUW4M_{mFbvLuKal)pjaTw7Zx`QR#W7?F9w$Zxgn|Qx|Ve^g_#^ znU?|}k3tTM`tniu8eFG3E9GpI{Qe*qbY0t5=t>HOE28H-Oo7T$p7BS>P)FKgFX}jZITAR3i1LFlfqFs zG=IV&-ij2G_uNQ}RsU&4z5Gi;`p_6@6r=+mc600Q-6h;Q(?srZjgTt5Ee<@#;n%jk ztCgc|>P~r3EtR?3?~!s2qMkaYhD0w(xh*dh45Ox4Ix-$bFTwE}NK&g0UC}7G4JRO< z;|I<{6@o%uMhwbA10Ux-ou52`u}h3(>cy>i?+FT%^EP`rauU^&wWxF8wS+@=02r5f z_cpJxXs`PfD|+LdBa0->^cNRG%d95D7PcaNQnYM6yy9u(qf4Dj^>qDydu~t?^|Ou) z;QBtx24T|9mndWb1hXmj9giBSZ($kA)_FX(A4~@qasE!9Zy_&T2O}OUZSkw#?4XJb zjQU^*$hO^aGWVEip6sf$&7Z!$iJCLNcF? z&7>0LueJgrjl?OH?(i$!ssl~ZE0*fQ(?2~W4%QM?69hk5etE}nvVzNERgtqE?{e)hIC>NY zB+YHW&fUDLPT4I_uctJTG8V8|GekvGVXYj`t)b~#B0Ur&y5Op3I&mL zC+4;Sb_pMggAs+`r_S;&*?$sz@1xGeOI}|5)9AH{?k-^+wQf&`<)VYSeZIDp-&3DP zoXUmF4Gy7Nlf8J3Uv(z_fqVz5E321}3nBvZXgtCkX@UX;n~04fQaND%^S#tbThB-n zgG!%DgGXG}C=9O_JDMrwh~Iut>e_rjDQBpuuy3zaJF4z0j)OIGwx1$FKhpgauKTMF z#zY@K=!8S@+r2N4RX zk*e=a)dPe-4vLCNgJn+z=e*W(sc6~nIs^ZBg|MQkFZpRGL(oX9a(TSe7pvaR`h_(z z^*QNVc?3V{W$k;PGI5n%u%6@8SFjFqzNgRkh5b8RC!6cyRx`9=<3$_d@wso)omJQ% zT=cQC}~~Plu_>12hi(2Ga7B%n*NfPHWv+yCR|Vn zK0eAEoKmK~xAHA7#a;`)Ext%i)5$I}*dd&rru~bQ9I^+Fgg0)`3!JbojgjK34{tc6 z`2Ky}Q_`Bj+B5?xzykF3=bMpP@wJ_(S9s-9xW$rFgy3%1%cI`CiJr?-_jrLVJpNlq zkM~pTVrAw;Vc$;AePESC$pAh`z>jDJRn?SU&w09DIE z8BlPls?z<@hAujMQ#h=9?ECE;L4SXK?PDS%Z;77%*xf8WsEb1N1?MaxE2Q}j5=$%QU67WXrv`dQp|o3is@|! zdHJ0!L-wZH^q@2kz8NfpU2Qp~B0t>Ix_JX$7fXYzTR3>@bGVOg%M9Mdw!>+X;Neik40bWQvB+lP(x8mz2fUu3AMsir4HdzhHKNs8yrXw*F8XJZGNJHavj^V7n7j&CKD zwqN35IN7`Xs>?WG&DuPh7_bB6R#J%iu_UeB={1p}U zEnMhx4Z#zDX?omyqpl z$AiB@c5hz|J){gt!JPXYoufzRxB(=_Na1{v4*-#j?Z~s?X(1&!no~#Qw=;Czom!$( z|LN|+AD+Usd+4QG#89-MV#lDM@V8TTs2XzlE@y05eY(h%Z^?g5t=t3d-PFiaj@?V= z6jRlDD2tySpIMwKr7DbQeDzlTBf_iwP<{c2{yiM}mK|Czj{zQ`Hgpx{`;^m0%cijiUml~2j*D?1wPo~_ z?}1G5smf@W+?c?}n}Agthg z#>G2~>A=DX7GqMA7?Kc<#I`hU(G}kwl**Jv&hJ)P<9`+*Xpv9Mc>;6nRzj@0Z znB|ta{`fK&CU6JqkyRPK=`x>Bxsy19BUkL!PWst=n)FR@js z(cQsS?J@pGyaGx)(=M^4x!I>Vnvuu87j0Pn9gG8au| zC?A+^Se{-JLNuvI^pSKq`3%F$dl%Jbj%)_Dxk=qE z=Sds6K-06`KLiz{CG@v&=!leEN6omcFw=Qyw!F!qFzm}jk5=;=tw=%Gg6sRPTH`ba zrB!b0D8hgnC>LT|$LoxcEr+B6fDDAj-Oh~zZD~Ylcf?rf^S!xO;tBvGAoeM-dyjZ$ z;5@laoA}4NbV=uQ(}why$GN9CChyU)yMFcC?Q&MfwoNricu>bn+XK6Awj2TlpGQF> z>-vG9OG%dzF4Cxc1N=#Vg+aR7uJX;cw9Fe2 zgH~C#=V(H{wSZ!->A?ji*|V_+t@T1UoIB3^L<}E%ikwFXzaa9;6=q zr&*+ii`DtsQfK-FguJHxAukExC~;{}^>oRA$#eKYShxp^5FAWU%k3$pRI>ta=P#A= zxn^6oU$G_0V#ODd?m&AqEzOB_kderq{5?cB?`ce!Xpuo153 zIc5@^c*3Hfh8)3`-QBIj&@F{=adL22O8zBX{Vys}VcABIbRK=da2pccTmASotVdpk zGO2_^*4DBWH@i0sg?9YoG~3_!n`~&6ogBl5W8vg~n{;dgT6ysgS#jJnL`?twjUgU9 zwj+SbNZQ}k0=M}P7xd(Cs}cq_0!oE9qWd6Nz()YoO$27%!NsTS|M%Yy=jC0)jVesP z34VVdnmhbX7-aZU^s5>x(i>%Y?e+iRbC~(!GdLebhvFalukQ%>GrQo=hv%y?guQ|P zxpf-!RsWxFUFXDQe1Z4Bx1Pr*%#wgjZOOZeJF_}`7yyo3MdZlFa>&)20l4iYnv6du;p$9}j3G*Z*Bh_X^9ED+3~ZxANlX=xb|gb0xMt19 zE5)ADu3cRe+$UV8RtA1b-q3 z_)^|qwA05QrE%F9#QfdG+js9?glE^!&o6UPV}m|1drzF(z(c_QQK5kNVSO3k)=JcO zg+eNO5;nGpb7S9cN^b3Lh0#tA3IbH=>;JYBanPlIcOc{((krL5LAmz@nyOsuM;!&` zA7^xTbvuMxG4vX?G$%wYM7{havjP8lk$%~)$ttU@>MHBjuU}z+sPG#qmZb~N(CJ{{ z$tlO7BbtPZmfzqdiWUa7e)^th0rf8=dgRDx?Al?x8|f#LT-A1?&#uI^_xX}XH4*q* zbqx3vxhN;A_&QHruUx-nNOr|77#bwu7}R|_^Xi@{7CQaj{q{;s&gzsAzxt7aJ-6!K{`{m*UEI=> zpEjLiGR+#zs3~S#1MANHcgpECZRsPe9cv^a4xA5y2N+sqQu=0?*Q{A%Q0gMK3zCa> z#$}EH`vorpAEJ2>^%9k%2k`Y7C{9Xbf~txOJy+R%?3>crq0jRy#~}L+MK^*Kuaz?I zO#^jooKi_G$uTX02pd%_H$}pE?S4Q&loI`H5#=(1xB*Om;7agX?i52Z2j>VP z9qdbsynD|ehVbCiz)0?>#vcEK>2nAY#Yge@_iX z+p=)Wc019mkTBjVq zEXub`bS6WYQWG+Br(hrGQnaai@J;^x3n(AFec_wGS!8%vwdGU)KA5L%`iuU*{?UHi zOSD@Jb9Igh+R(xJWZ)xZG|gj9mo9f{NqG3+%l(b0_xcAA+#O+DLC_M5*%&s2ipA#` zDWsMPp?mptvR;pz#2AJCX!qZL1bixhO>lT9ai#!q?f&L;C@|<@^Q%+1JOq}OZL>>c z%idRO6M8zcZ-VQHSYgr-)%Gd7o|41|e^Gri4bpaRchv2SI~1b!_9_yi)rQAs&=|(e zKH_lvwu5xxkv1NG%C5Um^m9yFhf&9;r^rhIl)^7+gF})td3Gcj$s?Npy5DGxcP6d< zu?jkqGda)dISW^7Z#ie&{K9Ko}qI*ig zz=9A1Xz4vPFywq?0oGlI2#_ z*muw}&6-Brv_T%(v)`^Tl>z1JgH2Kgk zFz=IP6UkV=rS;9a?aKHDp&}(?aEHFZWZM8d(HPg((!yn$ve_}xoHcwxK$Lp_9zNC1 zBUOG|8go1gX-RLsvZHWSg5W|k1 z-cuxkojL~-DfkrCMQNe~kq^zh1B1-=s6q4*cc#NHOy;@q1R-giI9Vt=v@;2}6ovF| zNrosCd+Yc4q`f&053sxU@^ARNjD0!YfWmU7r6s282U_s-n7a{*tY{|w! zRe1a=dsQ_sMGwl?nI@|)@m~$(rN6kK4;k-CL!IL7LpWGHc>)H>U5gfttk*DJ(oC~l z^%2q?m`?8X7(0p!qLxxRX*qF?O)zDJ26@wn4sO@8*h3LDCs1x-4pYM{X5L>X;~A#- z6w+htPctO-!fha&fNUSI#j^Qqf`TXMywCw2$|4C29@uI<;c3ekFl)h@4JCcu9l$04 z+90^Herxmm-+IpJj`tFwG~iDV(uev}PQnJQFY%DbCP2)YxHfrvGrSqa%pU`9f&sJw zt7dviPpLB9qS&z3+26Qs2k0R?g`h;uF?0QHQw}ywXMu9WkyLQ6^-U#1y0Qep9r|M( zXdO@sQs)*w>_KywR-vnoABij97HUGK~+;wWlQVu zC|Hi^wChMg7zY*5WG{$-bi4Hv7liba!VlsgN4ujU`?|;Po%U_NEe>}nBow6ufGXE+ zaQH!I%rZ3!F*i43cf(|W&|^A4-St<(ZE*Fz;;Pp@&Rl2%`aAeyeLip+ zr~-AfaWB@TxHXt_5$-c+nFXtU>pj-{xy}cfcL+mA6e&ch+6+9G?*85X@|7k#bi4)D zdbtNuP@UP)-s!Go-r*A|EkRJVb^4f!fdezyp@V-JV^Te$O02mp5Ew3f~Hlt z1o)&loKJ+MJ6lbJ} zw4+_RZuj=xX_%7bngJ&U+xNAvFE+=vG>grl=;wi>x2FWESO(wN)A6*i8;KKM?dvu_ z6Yfc1UB96|S{q*Azz?Xzk;96=-)~s@u>l0aDu48j5?-Mz2Z|k0#ikQhkyF~;3@hQK z3y6tnB9$7IKA}N>*6xwk0ixC=jx&4Y@WZX!-r$%fXcDvhL{?^EwN@-J{+4w_olQ z8Co`lBc_j{2s3tvS{ASe`>$^p^2@!|KyM8AH4!#)PR}-A{|7DCP^SDhzNliq3K!Ov z-&#nT81!x$ue~B#4nhkKTPu+YJ2&2ws^oTLpH!(c=7IxkJ<2FCL!5i-itO^2Y7vn`8O(#c^h!d0&BbQo;vP{~7RGlT?z0*steW z)G)*TveyHv2A{jKO{V(qt?^!q8s#NJpY7FnPd+zEJ-ZI-HiOcileUr#mFLdm??AEa z2hpj_Sic~}QWK61=nqNnnNbvYu1S=claM8f2;q3uJGR^6*tA%maFrN29R3!)0OIdP z3b$=n!L*(^Z~&)1wk5teMi-6l(*v8(e;1t{H?aI8GLn4LU;n`a#5k7CRYIYdX=)>1 zVXKC6p5!K8IEf;E*?9=r4Zo^VIHdXE2RV-3E$H8(mh^A=7z;l106N=PbpYZX*}LE? zW|?AZ^q)^0XI74e3LC~otk>HhUIuY4QfQk-G%u07wxxN1JX#Bs-Y@-=_PKD?MC{PyIzHT z`OX>$Kz(T+ziX6hxfLM;n)(5;x3g3Tjl*N=bxCLlG3T15dV9jy?~ip1BEIna9o>72GYFPq zeS5`>uV0RnMM7uQUS#eHPV3+WFn5Dp0u@?I@^fxA?=oUE6T9qo+8DC4EgXj(O*`)U z9X+TtI$aCSNU+QNCq7U$x9lk~cXd1G;{y(c|NX<#-HcV_FfYJtYzz)Qd2%U;g%fVI zZxUiE66&8rm#{SdykTh8`@O2ieEWFUri}~@Q&Bfq>cZBq(lvFH$HFm9|4>aZiv%rOB?Q3yobcEO(F!4C#|ek)$F) z7q%2>-Ssyfy1Z^Z1QxqY{U8Z}hf(Fq=m!HBjtc6+2xzJjMMOI_2fKvbZ$bV>OUYtAEDd}oKhtx znLm@ioY}ndq}3Fh9hOg$pGfO`&jJ6RZ9@qTz4@)abV0t454Z*&jjtOfT-da^C06Fj zZm%UL`1Xvp#T+|%jmJgvtGCGC03pjRE&Cgf>uILE^WZglCRGTmUiZ1z%CtffQMxy` z37zqX`eyVgWEZipxf-1TRjWTL_6u9K0mBI8o|`hJEhckFxGeu;mw4769b_W~|9wN3 zr2T)i3t9JmF4@`#M?@cL!*q3q5kOVVjIo6$KW>Ow9jiv9!}>5;ba|!pO#0=I*npYH(tj4m_G} zlo=()DShZsimtkXjhJ0$<^p1dht5bL0C&S*-!+8UDjoo5SD3?iRozf2eH#oR_ z${E9ZaDbl~Tk~z*4nDs<{OT8^TuwT?>+^XU$9fPG%D`v*wgerHhnqTu%t z7$wzgq4$oVk7tGF=IZ8z-!tf=aiQv+!3Xi0Z*#+cL(_NtOkl|R!5eb>6~Y*E_mxQO z?tm))5MIAMXwj9bE|o5d&NIi&=vM^I#@&GqLQp!#zq@f7v#6$)|1@q z89(#ez5YX{opE3T8mAEjwLTwBc(1(U2;JKH9Uq=*ZX6zO;TkEAc{a<>jiD(F{pTmA zw$me^?*{$=(Of5BF{gM$1G%`Q>ETc}m!fD1G1Es5B$uBr)3>GB8B=c@$8p#*afDgI z=Av4`ZPPdJh)@WwN=u^-0AG%q0cX<_l~4`85-0w0zo!fAMt++-r4wd1e^6(?d-wA> zKmCU^_WS;)uBYsgK%=|0HSgu1l?`MGIR5sE&?jReAj=}6+a`H}2L0L}cCHWf;(RN5 zQaug{p=u3bhK+rVF<+s+uD_^^T7qwye036s5C3{e~MlV14Zxe$& z<>;|49v;d>DZGZ2qY1|Hl(AGq;va*7J}`=_im9dUsg>!Qcsoy3;Pe8; z2fuoqsd|MMaz|=;v;-Zpc>55i7+;)p>*j|Ko8gm%e@cHz&;;-jjwm?C1d-KT?d4Z+ zxqNnw_NUfyAVj zpJUrs(iWZ`CKu}YGYvSCt3I6?WD_B1G5-CgZX||3Lb@0UF8ZIL9E(;|_z477K&TKf zu(3t-FMyP!o1;@732xB&b2^9+*#veLX1a{0Pgdj#)}t(k?o?>l74*pK-R6Zd#xbnsvACElafU$0f#D7M>HVnVm35~l|_MH^*6W8wvV4i*ue zPPOtfTQJT3CdLL0mwje_o1hI8waLiW8ZRD{rbv@~Yb&6Y;PF`Z-lhy2E8_j4>ot-Y zou_<#w#%-4`2@|})#Z=7B5yUkFs>}V8HfP@%?lJVZ6n=qXdvzxDHYnZ_rITuvue%y znv|bmgmDG~?NYS(gt0-14hR#zD=VH#IeGP;Q{F+{Tr(** zKqdV)6EVhiqaJSswCM&9n5VZCV2nvCmTKw`Ri*Vx*mi}CBXfQUNL{RE8~QTF$sgu@|fw%ZY%wv{5K z`~AndQ=Q3Ua+PyYu#N!nNxxutyb!$H6UAhHP@{Uw93b}Z^c7aWfJHWC2V(!4wPSvP z%*|gDmqycgla!;rJv`(ZOUD-%XAtTr&h=5{%(j4RDPd!d)=Ukl>X=Mm;u9JE6g2I* z-}NPq3NWp>6fw44R+9j(4*ZL+#7s!66xq*(J$-B1z-uvYLv4tchbtJ3QZaQm{9ub& zoJ_y}?f>Chfq-pg##>fA8{Su0#;9FaF25NzhY0;ytBUiD#YMC}V(d&lYgQ|^?i}jo zi4Eg?8W)EyOCKJ>skyY`+}s3_ONMSuuK9J~qkUfid{iLxg=<7Z|8=Akn|Hwr|BpYO zNlY5}ZRe14*}w8V7iDGHi*sXhH!t9qeSGW(S1Hd4+S=zntTnvYQv&f2(BBV#L3Go1 z(!1nRpPo_^RR}FSuyrS3ncH>`<_u5B-{W7xJ zQ7WbNzcIH~S1S}fM)~0uZd;M<8@)O%LG_6Lk59&fdad#jn)Xhr$12UH&up38F(|CI zb+4eJ`|pe4CKhc+VDxjdc5GfNZLVLS9Xb8j_d(eV&_UIc1HHGe=w_j|0gZmF%v=XK zs!1{GMChq7=n?JAT-gGCeu|-tA8*gM0y_LXmmHy;XVzrU@d~yvnH2=v3f06*V1f+R zd@nRAOWDLWp(h)Ae>z&M?jL<*vuC<943dIP*927!ME_w+eGgjKJ$i~q?>P4hW3?Eh5<4Dx9!Hs zi3jTx;a=XRBp-O>ga%O0+q%@v#crzM>zT8(Yk9-^3WzA-{HR7JUF_jFH2*wLS2VSi z6$2RmkXGb|-YbU&rZ>X)d5=Wr2B!uR@3F6jc{n6@m6EB~?XtG78nSPVhsW`o`{)r` z^Ju;|`bY%oB*#tB3jqeqg~yIq>!A{aR;UVwl^Fus+a!FR9EcKE-X$_RUP$>7$RX2N zar&Uk^3B?ba;EmRnX#VK0$A6_%LCtZ97^%^Rdl5pELPs=c=sXAOa<-t01G$2Vf|+N z_3I(!vCpT+;+nY6pS}}*(6Np*3c6fjCOQ_BdrWtCm6tBlD71M5;Dt7GA=%_$En*4z z$;NWO`Q5%_I#8toSH@w{%5VMsM%)N^8h(EBp}xO=VqPxI;uLBp+Uwi&6=d3%G}R7O zW*FoesdkyfJv&?665c@^hk?;)6aCp!bE|8rshXQDREG-C1XsZ=BSK<)X=$F8>yk} zuK2wiR~?5__=oOnI%&o7`rtm6)L)-Ap8NX;m3oiXGBxGU?U`9F5muWok}oBz@ztI( zcjrEDYd%Ryl5>u^YD1g|JJV~D{(#5bkE@e$Qibvpw{#pE2!u)ROhfm?jM5Ar*pnVl zZj{O__?|Rw-5es3VO>ZpcC-o>GF&wX>4sXH+`gE>5cQSK9lm z^3&YvRA9-%faF5|(o`U%F{3@D>sd1@sipYCh0t^r-;#_Hf_P_&%lE`s&kRGKVr%0T z343&EZV=BWXJ~42FJA0=={M;`|Mm-E8MZ#T?bl{PY=> zS}wl5@npgh#fs_}EnSQ*L6m;!;nx^&tIYDq7e7-HM&nje16YW-Q7yybq{|ki>u>+4 zB6ti69I;+~anllZPh;7577q%cO*_-T#lmyCo}<)t zUQ43;i_6MSqt*O^PQrMrE~)H_p<6s5#&AE=>F0BWN{5tReIr=X7MnQpd0UJF^Lytj z+NXXH^=++bQ-Q=bZQB_4p>37@%QO1PF>^J2VveTd+&F7unS)i2vp!!;E0_=Y*Uylj zsq2+{7Dno4I4}GlRxbY%DnP$1VuQGq_fH-)%G9qH>4ODe{)G@rs)0y(>r!Wj9{0lO zG~d@@;r!2jVl|5u+IY>O)MvY8_t(*mOy{ZhM6)=98@j-; zZg1i49R)RVbp_`Ax2;Po$gl6|wIt>15!*$4n)MtT+8^&H#?jK9>xGE(<&2Lia9eY_ z=%f+ymqv-JR9~Y$O6iNap5pmlElu+K4PD0Sdu+Z*E)BbUYuwZAX-gjDwk76kk*BKl zeQ8!!G0u7k7lfuq4f0aN9Db@rZkM67wwI6#v~&|GpHvD~7HYN9+Dc{`+igno33|~s zol8xtUgYsO`wR5vT;k4Vn2vd%uArV`g~8lCS$?JA^{OXK(N ztkp})=+yHZZ-;IB(%r1hJM$_QK0kbl%X%TJnnxTbQr_RU@ZHXb;=FUK zs`cu^Xd2vqC?7@I&+iiYWfL@a%+`=HuFv;sYUzepPdygJcmZW$(AG=sTWMCCmYBmW zT32O$t&YLFDu3s9Yn*?7%5T@RHgX>Uyc=&*6n{k~zxk6SdavbA*HGTA#ebE<{$O%? zdr(+W&ns=(szYWTLbCP>s&MvSv`qr7omL$lWasNbvrNGvjU#o>dBuM9t-^cNr+g=Y zU{pEU=lf7;By!!jTQdjKmk6PQ>>gjU4Ky2K^mENRR+o$@A-1ha(d~5t9fM&Vi=+B_1;1?iO5fij|9TGB&rUNdr>!l?9sTc> zS=TT*l}W6xULto?vq~J>Xi#6x+Ob3~Y)ab6tyS^qX4Xik>F2+hnX)e11^6X;F8pwj zs-B8x)$FS<)Rw8M@)xtxDlI?DbSL9B|NE(hLG)DNss5t*ISP8M{H`DU4)tyQ?;AyD z59gY;RrxoHGQB%y%>Vvs2AA%TknEl>p_AOKyVWXJy-Te_YJ^t0Zf6F!Gp;o+s!{B5 zOIO0%1FV8r)`C5fc{U}fJp~sA4=Lj`wT`QGVq!Ar-g7VBb+YsNy%JLFg@E|9B2p}e zc)ErN)agT=>+YZWfm1-tLe10W zY0~FtZPz{_cBbpP*|XPQU}N{rp9zdta+~|^F&yv05+Lhs`QfkkU*s>xUJKs2aXpic zmf%75g076Arc9~Hv1Q}oZHKO;|BAFCT~8rA;aMlodelMRF}@wk^M|}8grg8F4a?$ zHD)=#hka+44GSh5qn-923uYH$eGV(sF$3@Kk28rcthp5?=9!C}HLIbL8u}WO0 zeeKe?ZI^GoQ_>#W^u-yg+2JG-6O&~9*L%j)ajV8xC6g6bxB`Zhqcn(G<3%?EZ+rE5 zlXL1vBeW%N4LXHR4_e{Zi1|_U*(9jM8lWseWcA6)D9d>o81GI$w{NRX`uYamUfyh! z0!2!Y0R3Zk{Wp`}1Y7obGbZ6VeC$_b8f`9ZdQi%poul2dZQZWc7e_hsX`0%^9@osx z!ow-!EzLgjr5@8CS-kx5Qonq?7f+~pL|>*OEmDr`94<5p+F0c`$#ND2d(RW=MKYfo zO}sPsVqX)MY2Zo}Yd7~`VS2s&)SvfxAoQE zyt9=g+cCSB=`_hB_d5IWB-76+rIQnR#^zt|Z=`%x;`EgHdc$bd!@&N$%jn7PUW2rQ)t1lyNB@fZgQI5Y_j4`oB zdJnMU&_GC0@-w_~cR(0)4%70aj{k! zWvFt%F)ocSHbrV;_?5f)yEo^>%&&Tns#BiEIrUyqkt+|C&cS}BrI$M6QdjEu%~;GN zX!S3w;!&YSQH{33N(0wk-KTu6RiY;bF?E_1q^Q4VD_5xmf@9-IYs_z9>G^9h?xrD} zUGy=O?_FZXrnpDTD%0S(Vdut~FL#6r)1BKb`X}dK+#D^Nv8;KdXH(qRD`wkgU)!c@ z)&|6OS--NJiD~wM2RwL<>G!@L$E0=bY5V+Zq;J_&X4Nak(`IeykhRCk;!>e+ z+F+>qW~Qt$4}CNCok9d`dg^h67M02SPlh!T6gsx6w8prK=gzmNWF~}VpzU{?X7uhm z60XDX&K(BMWFlj5NY?A*HDi1{5I-PF%CF(zl-U+z1{YY{@{-xF95N*@*XR+KxK|V4 zhg}n5Wy;k1L7uhk!k)Gs*A70}=GntcryYB3ATGP)J$29do3Zofqny%PV}=EcQmryl z6}T*Z3*W`6RPo74Qcl}1&k^mH*lAO$=Sy4q-b7z;=B-^@0tg6BmVU5Wp+TQx0}JQF zE5{s7t+US17HXZpoY|t1(_tv)>0;ohTzzw@P`D(8<>YJAhToG?OoHUzGoqhk4ah`o zJg=^q^YBHd@EF$&1+G-N<#@y;KNPXsh<|%6T>Msc=4Ak%aW=Mxa>^*#XmaeM%bGjF z!<>TNIimzIn`<%N@SC6K6laO~6(lgaL?(~rMrZmHI8J6&P>7OaS$e3-tELbL+pnas zX{XzIr;VIqx+C1r`N~-5_O~oOpT&~#0+y)2#;oh;eWamp^W8s|c+}R=xkNx=V);vf@N|>+IWGL#}1~~o84-I)8f2cdVESZ%(~dhEu5kK5}{14_RjA0O;CxW zX9mBXl}wZ*Dd-j;GVq+x4dl+)$#h42askte)9I>#gJQ!gi}7U}Fi9+@`Z=TuLS;)v zR9B)~uB6=zP2IqqzCw%UGSQ{@uBgb&UR930CXoJx{?0n4Z0`P8Bq7QrPp(#6kuB!A z@aE=tsV$~kOS1N4FYfk49PM^Pw00%c;Hf~laISD!7K?NiUSfWS4gKEh-D;onYPKQ2 z*vQw)pWHe-GA_%?Y4!UjHkT_>#NhU`nlZjU$}yu?0@mY{G+I)N@yyU;Vv3xy!Gh9E z(@eMkmRM#3=S{Kr#CxcG~gj& z#!j#-{xw>)nfxPAsHHb%uia_QHg}Skbd5g8shpZnqRd=MEti$YGZXnVP3k1tNL@{i zX=gn2jBt~vpv$ia%J-)B2yKsEEeX4!mz1{5_88Z8#$>jweseyb6iey9=iFsY@nbz9 zqpUKYpV@{0%(}3xR<8Ho%f#M|Csa&_NRnQ8^sniVaRs-#Z78}UySDvDpoC0a7hpCq zFE`4lrbOR0+LD-zauX&Pr?ZBECj_A1>DPEZYk2QRgtp~OgVnDHa=X6OqN}_2c228_ zZiH8ua~b6Y(Lyrt(G|O`<2`m$%=oUpiR6bSUqgiV9ShT9rk7h1aD=pZ%G4NHBw{;^ zlFVOOjO#Mt>g?Me2yt`Ax?N7WdMOaUjYe6GHcDAl(nibug_P_mZS?4NEZcGoliTr0 z3Ul=h(kpem9o*Y^*^Fl&`}j!JW~aWuA* zg{jPcZesvpsZN{zCJMSN?``Qc@yxr&uWp$fGwRuwmVGmt>sR?=Tc#zx8Qa;yU^G{v zLeXGor*&bcWm?J7XcB77C1o!ES9|CB)zp=S@zdJsazq*1sVFGUnzlMqi`qdzg8?tK z)}@Fj2m%4Dtwfa#e1DBt{D1nqY0A^#VdJ$R$C%07S%<8wEY8{I?ESvK=h^SxS(do(DpSy)sp{4!ytKF02c|o{&N-CriL|jW zo6DZ+bV_ZvJnY!8V#nP_3Bw8?A>8$(9S-)QP z?0x*ROL$LP+UQ3aHf3Qw!d08gS^V*H&qY;!yN4yQjOg>(;XZgHl~R<_DQRs> z@ZAG*1F?!%=^bX0rz^Ak)%}ml+HPr8QD{bv_cFK2ky>-W?IVEx-c3hp{J6qtOE5#llD7^F_li>OfNbr%k_vo<*MojKe@lTwzTGU(`% z$mdtgZMZ(sLQrP1T~K6Z3#<$DT3i|9+-|5#sWfy=NWaWo;_0|>ab1L8b|e#YIk}74 zC6WunSD&p5#M$v9<6gmxR7X{QooAR+wx?In2{*VP5528i<%4bPg}KW9Of)Xp!hKz* zMA^g$mv6|D;ocOcD>D4K?^4g`>M^gM>^g4;NgBA!GM;9B#y&Fm;Y|puzWc1kxY`#f z7^vz1^SdoHa5YC?r&Er?Twv>!PNuS(FmJWK0-lJE5ja0%WKxv#)k<1C?w`qKqq8$% zYn{%@+*$K}SatdI<6vgeI;NJ#dT=drVJ0i5$fY4t{#mo^w<}nBh8TcYKc^EowaRLc zjXPVJfBaxbT`2i5wfsODGn<{6vy>WKNegI`nnveAH>~{7I0&yDqrtiFgq)SR=-#w; zSVqY%FRgt209R*oR>YuB`>l(K08k*l=oaRC?_Nk~mz5gh&6|gneTnSR23^Cbo507Z z^&M*RZ`t~5mt*EM+%k<c zR;L$g*5U4d&CPc9;Zz)mA_o zsWzze&|0k7X~VN678wQIKiiZ|^;FD{CDGozq)k3C28aF@UI^;S4woW*J4Y9!Y%dUx zC3uRz^@tJRl0g)ZAcfa8af>wT7ler>0g)T1ykgc+Ucj4I+T4X241**~ZnB+E;)pY+ z7-H=(t#tTeL%6bVl`8qVJAdzrQpb7WW=Xxm;}-OUu0vqmQnDi?GiN9;Yr7B^W2>r0 zd`YqALwFswce0|W?0liMiQwW$jk$FZG>JLwmzdXS2+OCN)t*81LSv?P7BuMjAuU4l zqchcd74S@3Kf&d~UzpPiAZWy#EK63vNcFAF)p=k8%h-CL&3#h=M<0mG_gf>K{bmiT z+@-_Y=+)6D4k79-v8;Z?%agdmYw(@kg8*hFi8jWTh9*zwUjz=QLze8GN#500OSEV3My%$Gr_EZbc2XIh!B1n>Zx zHB40l>j_TZL{kJ3$y68c3$IL@GKEjwl&Ws3vw4Rdm3NZy&(*7$F*IP>DcL3T^@YS@K#n#M?3hu?HPp|&>ZDf@2bB7$G!r-%5pZHO@Kja+wu zxkJ6xdsV8g&X2%x>VkWBo_^pi1IED=du(eHc6L~}U%H4Lhb<9jr*ZE_>I4|>=>6GY z%*YKlLR;woBv_@!|LU*6mas3=0v+X|VAyLML;d`pFs)|a*aZ7D%wFhD-6EMaTsTz5 zH;%)+-FSLDkq=K2lf1fb1QdBt+m5rtvU`4o{pxyU+ZVd*QQ7>A{)h0B1FZt<*77uG zezRrSmgszBLp&IDMnpG;OmQy1CM5k#{O^wZ>gU=I#=BC{)#4Y|G?`F?F}1t5emd2- zi2dtQ-9ujme7KeSW#Sn)mD}jjI&5;v?h1f*h0@l&+S2jkensAHV~C>;N&_#Q6yjv( zUucRQd1!nGgO$rMCVD`IrFQsYUxLy3o}I4kON3`DHpmSM5%aN)F(OcRgI1Ybg%~yS z^)~Z90%+z{e+55txWXAnV{-cX;6(dDwz?3WJCqE1K2jhsZdjPKspBnbaB=N7TkHF4 z2D(%gx!bXlgO3~|D?u9k;`e|AbUkvnQTu$Q9p zYNNK+pB}Fep-2Dg7#HiYk8^pVpsPukix6|{PhtP4?!WXkS!hScf zy?EBBZ;hLh@4eJ#)9T=5r}6-|+VvcPUDa(SQq^Jt!nWs*!<5i4-OK{#`THI#RzL-= zU+^$>@W6XzxQN3k<1u9=T_=;!Nj=%}sZQ)7wXLA|-TS?n^g4tT$YhS4R2C46gp`|*A{$a#L<*`%y&0*p`#-5IoR>^#ccJ2u+Vz7( z_m0K2ebk-HxHOMn@zJLzakycW@du46=3_uy<9o<{9 literal 728485 zcmce;1yt7Swl(hFZUq$)gOC&zQ6v-u1qsDKT9IxEX;8XRK@1d>6e*PwDQRgH5Rj7Y zlJ4&M&llb2-h0OV?sx9}?)W?QIC|iHfAKu)S!=Gj)?81nONs2>NxgH+mMyzQuU(ee zvgI$bEnAq$i2uY-SY&9Uw*0YWi|FNxvbMpa?KX}#>ovS>iVZut^5luf2!3q z&U|4gEv?863!CCg;Tuj=t zxHH7@kuBy*OVfQPW#7MG{__%kIbA1cuO3T;?~0x^>nI=FH|b3y;7XB9~D`7HCjDXGECJfUUOSy6Bg}|fH`B~P;K<1 z-8B6}L%deQb>t`I9apD{@7}%ZO0;KvAk4))DLGO+pdsd_NQ|SSZ&8s4CE*J3%4r(T zOS;RmLmm`t-h2vul0C&#c z(~X-IG@R1RI`al9Lmc}26wG6jPuz?$Zu;2$_2Diqy-JBzL8FGao#ZU6%Bk0m`EVQ5 z6zZSxq~;zD_Y$<9zI<8zPTzOG<+%}|xkv?dJ0?mYb;5ga^$HWGG#m2-)uv}tdM+-rhn|12CQ@AZF_yGG zUZwGLRh7^N|Cuwh4e=^9wa;nJ@)*_~FL1u>vc9BO=Fb*p+?nsswM%l1kL?`rv`8s_Ods%0zkGm@n2zDx@Vsbic^$ zYuB#n*F=;Oek2j-z3z~|GM?>sQel3q)dou@U_ZM!9Ge&+o+x&5dahApVZ40>PZs7h z-~4Oej=MGGI%JS(jW#8J_ww>N`?dEs!i9x}{n=E~ zwThm~M2bD9I(_)Wi4$BWcWv2HlYy5$$aNbl?9vdY1e2k|l6Dt7dPH(ii!;zX`^=#| zFLyEWS&yD`T%3f3FW{_*`W!nV;@OsCm-0pP+Ih9Hmb6aCnF`p@!f^e`Lx(O<@5{wb z$IG8LX-UQX@F0dbCKMjJ*tm-qFFxE!&gZh8Lw1;*eSEw<8zv-4LuFY1Va~BnY~_1s zXs7{AdwV-q<{kkYckD~Bkc;|v5*g)lF6)lC*45|Kew8#gWn_4anH;Q2A1rUvlaoi*4k@P@%Dub%02S6{q3et504WJa zNo(uMkGGdR%e$`ag3A&urfKFreUd5%lm2h^;7Tc%H4!R2Q%2u^e zlJI=Kq3W=n#mR0sNF#WpSEscPA2~vIbG$hvTEvSUZbZK!u6bo5-({jhAZ9IzzC`$O zQ&ZEE{meY3ZJ)3KaI=p+J*~@oW7P7VU)9#vAB00rahSU=v7Djt;%c}-9nA6Sb2l=a zQhN9Vy!1eM;E~+n;o;ZwCI}?NZ)=BE*Vby&vokY?EcqBO<4N=?UNgC!v*?$Y8?G-4 z;9#Jkp&=)?!g2fj`EyrS*T6}ImR2bV3Cq+bk%ETw=zE#HoA+BoSYkO1h&bl`ss%br_;(<#wB+WrF`dH<@h|yp_<-O zpOv|hg=W35AlIvdDdtl($700$qw`I$j8Feub3pW;fykd(HiQGPwOtJ4(sM_g`}tEQK{az` zpejtjVg8B~$EVE9ZQHg*(k0y)X<+jym3=Gxu)M19KJ^n zghO!GQUsR;qnnpk)#hGOTVc)QV&$EtOPLuN84;WuzC8SM%RLwc%YmH+$8X>Na~oWH z5F-4Wi(7}tI^ef;Ik7fQ{oUP+Zi}t5EmW`2-5%S3Bbat2> zte%>h%FD|We(bO?&Zd-nneV&Z>f+Q$V`8FOw*2x*LVQDf{#ICcZQC`qyoY4?KkC`m zCW?xRh|IoXeK0YLf&G7kUZtm`Y|OG633FaPVG`SJx@XUxg_#-gjf%dqfPC}5r#-6b z=H}A~4UtYZIIiAI;`Dr$gV+PS8@aG-yz(3(83*URi__m|%#2&pEAA4zz(&T$$6+6~ zQGwS28GQ^c65seBd%{nQoPj}Ob;AyS931p+pSz!uvO6%)?(*eAgp`O|=@unMQoAe9e85kB^TlBXYT=TB#y201!(k|FKH_% z0_<{pt*S~yEWv+iimWiU_31K>v&Kz7vTY_1h>aQ(>SJyepN?!c_I&=FTzIrS+XjGf zZDoPsymef5HUSgWc0A)Y+0z>l5n(wm6WY>8Q)1*R{CMw?t8Q*?*l$EL15r^?>^Dpt zH&Ac6^+}riz=7Gdh0dyw^9%7A9Wuucz*y8i-FfTf zmygDT@!N5ZNm9K&V{q7Q5RM)(tIYRA3d|dUCuiy#&9Lp5FG2IN)c4s`=&`!kQaeO}HjDbvxji}nE4CCf~)YN=B zC1f#ENk-yPeLX$lwL_!U+1e3@C&Om+e^>YG-QYL4xVVJAy?CoiFO+u^zx z)uVJ9jMHuG1iVZMK>QvdQ@hx6ZK{|7SkrlZbtzsYgOuvjg&0}uu@*VvYPjyYx;mgU zRUC`TknQwrymE<`2g=l4AqAFdEC{%KLhRSw?-xDYNzT++lN-4LIt;8xMHjK|K&~ow|~h?NmYJ(B|+e>e5^p(K--%sBgV+GiS`Vkf)mBh zHcmxYE*aG7>DcYAVl+9uFn-0&mz_Z-%TY$A#^+|#pe2LU^82gtS-K}>ho<{m2RKx( z>7`p|3!FY(t;4CS(2jR4kv3;PMJwcI`%|_iCZ$wKa3IUd@T@`G#|!uWJSe4z?#TR$nm!JVOz%D>-S}$lZ;7oRMHA01E-!})*yJgM{ZU=LCs;DBb_j9b z+sCKkwuKbM4y=EGH>V!O%%N|VqnEwwZ*R=lY7KcOQrs_hsK`eS1XsG0)ifjFBd$wJUY+tm5KYK~eP<~E`;d*U79?cVlIL<#-C=wj3*?>EfG4^&gZz-RX{POxQ!u z^YwO0WQath{M^u-=j<}zGs z_U4Fz&bN^41vj#XJGY5RZ{50UC!^`3M?$=j)%p^Wl2&ms>WOEmNW9|D5#B%|S>P0v zWO^ql^Wg6E4FlfkAKtse!yc0De5gu%cf@*e)a*vj^;A?4WBhyO1w#M^PG&V275Z7)hEhWW6{X9^Eg77x4jQ5b- z3b5+7*)0~yI+h=lvD=`ES0_{W{(U(Xg?+-l$LUAA$JyN;KY3y|)tuK5_hj}>f9zY~ z$Dn9}%KP^-OK--Cu89F=xA!X1^Y!ACQtNh?BW38;dR~#evzxii&dx6U`16BpaJ|Re zK2cepnAzpGo$;Le;gZ7I;-iCe_POe28LheouOy<7z+Dvrr_+_GyYgm$T28=FcCKVR zlK)#-YuQv&Q4tft-t7P2?HTGmA#lgS!iWCov7^V1G3{jJ7Z3;wUWd;WCe~nJ5b>B@ zbk@=7ryTC?W_SBRGh1_0&F}-+*V>*bueyqgiW}s3^A~dFgDj#Pj}{jfQ}=*t$;eR9 z{PIBfv76iFVx7~clZ1&G7|hHVCMM6@&1MN1f6&+jPQ(ZdE)4;WLI!UG+bxqou-zmgT+p(hNnFv@|#J5)u;DR_2v5oO>#QGWe{9 z)ZaR07UXZ3@&wV;B{sglK_BPHAar_zKRzh`Uvsf#vP(YSe;+XXOubbaKD7+O%hNZPm0r%W23w z%5!drz*P}c<_!t?JOhZ~?71$S~xULmk zjF(4-EJ|s)Pgws9b-etDErHJDCkpIg3BD%c`YrKFTYi4m3NPF1eE-(8h+?gz!f)TA zqN2X{nF|RCojrSYs3r+t_&ihEsOC*e&;K}Zs)$uD(fPEty%fdFVPCPxM~@yg8#mzM z&)rk;$$ONSHOYh@-@Er`r}LVwslrQckSRr4cde}CSeifK1GYCTMG35P*8w$o{Pn*O z!F|h%i6=cFU^X4<-b>e?uJq z127P{lSC*{tuf0fT4CgjerINDrCPp@jpUGG=JG&4TdVAc`C%H97PHnoF3OXH-Jjm7 z>weR?EwwMG+2YY2I@uVlA~;6rb)65{*(yC(cU!#qR2-+0Kjop+nImLw+b8Dd8Fr4> z`KJyt&nQXi0Y@fWYjR1Qnchdj1-3jXF$JWHu z9dfNwq9l(=QJqYCwyfEP+@$*<2Zr>)M=$~1uR@*B@cvW~WAS}ef7-~njao>MpM z$+qO>Wnv22<)CW281i^!A&A>#RNtWHQ?~|Xfq5QbSr2WL96HsOXCXqo_ke6+wal60 z*NE>9RJI$ljfiG^EqF+F#bZ0K@>6o4QQ|%MJ_UaG&Xy&S?e{N6*Y03!Yj2}vQ;u(_ z4)u|dk#QrVA0PLmfQ5;iCp^;VXI(kMVx9B5s5yCOflN4pj`>A!FsmE*)slwf&|Y{; zw`*cj)~QFCn063crNyUzJktRVb&1u9#}8Y(ZF)k4lf%LDHi`ui^EN>g*{cSU$bA7EAdXz2I- z5h(+&WAETJlZSca=0k>+bbT}NwxZ!#OEt8zoORW@;z4?l|D#<>RPzn7<+G+faq20i z%}>MLy?f{CK~6K(@uM*)-=7Y9_USGS9)OeM=6yyzonxoEiZ%`Qg`KKA!TtXOIR6jK z_&4PK{mn>Z_!VQ+F9;;C`R}7rB5~{{>NAhUBEN4Qt1YX0vW^lq>CV&7_}?v`Py9)! zU2KW|U4A)C!0!K-U$Wf1|HUw^GL$P(;vq|!i?Uz)HeJQi#!hESnXJ{CKA}F-O>1|4 zQ-0?uLdjJxI);tOx595r?mwtW)G1tHC@d--u>W-;UQQi!r#o-6w6tuhn4-*B)weF* znm#c(GuNJV*C|1P;AMLE9}K&tUqAeOeQjCCh3TMCZB+f|Ki15DRdpCj7q2_L{rs8s z{SmHdsb3ep*hx=#i~l-L(REa+7Y|4T2SChdY`qc!Vn3Ll$?ZLYpofQey23*KS9UdP z*?si%&Z~u@jF;0s$o)P-VJ`wDqGqE9m*uPS*RvKCd+ux45N#tbZIO{>kz3wxIoh)J z^>QXEoS;=Em!}h@Lu>+gRjlq-*t+EV#=h$WF}$rf9g}>&HeKGi)k<+d^_-VWFiu zPf*44*SQ_>;?XybxQnxC7fV0;ii6?3g&trYVPPu1J^A5mzpfoe#CfhRM}KLp}ZR!~u!w75yJV)l2(#8#(6W(y;-D z_ls>>)%weenlH-wik?zf@dI2HO9#ivr-|_>%*~2DB_fKB@2%1?_(P^4REAS#35Wkh&>i7Eh4L@&K1hdxp!nO=sPQB}3~*UuUK;i8afG zPz}oUQ;AYz%9)CU;P`?;hviiH>h#Fgd^+XXf(7#L0cGP4YR<0BaLm)4`{Y>a<;Nml z(k_ON-djD?^ZBFpz_`hIYxcFf7Zai%#zi0z1N+nNVXTy#J4JZkbOJQY8GjdC?9X=uP|-iBS$NMf!H1|zR1f)eqFi<6^=C&k>FEF>dY?!#*|8reQrL^q%XTl?ummNCQ z8!@|eYx2%bYYkj5Ubux{YQAlxB?|I)e4TV__6$| zVowSUdJ~NIf8r*8AKjQ|TY)EE&CQJ-OU%8IWT;=;&qlfasZcAa7eOapr$0cW(X1{G zu5>qbJ|RD}i2W6BuHQaa^vFj@@b1sL!3B}OC{J21mp7Z`2N@+Py6-;dyR}H*cH1K# z6dX=lm#-ALx=M!#&eWexVWTK>C~)un8*$)`hH`>^ed|)M#FjF6PW;G=uj8CvJDoAn z)|_tO}Htl(!QR z>1eCVWZA*6J}tsw`Yg_oa{t!nH*bOFr)FnY9hd7G;yX$jUe`26Z)7t|hs@)LTHj`o z8~?j6bcc&Ffc&Qwx@-Qm&Q!2NRJxxM>(l=2dqjI(?|HqIelZGABXiXnO z0P9FiX86_c(!ga}7P+|V`ReU4Va^G)Lt=iccBe;c9$fLrwi*lm?OQ~?X;WEd)$z{c zb!Iw9YiVh{{Q6KO&C%({=k?~KpwMor?IG-QtrPvmotY{hA|yxr;cvw?@z7h6tGM>q zR|;0~4H9^q3)W?tB!uvPZ}r({{Pq80=6~Y|Uz>@Je+D7P*Vfi7vfV|_P)lxt*DYNx z|L+jKiVSY&(#U*>a;jmFQ@u`}cVmZs`+Fr1u#I>X+fso`nu;IwMFG`DPcqH)JLex_ zVd2Y&CA_D)bKq{0#*MtGnHHzUdsZ9>(A4bGY6B&RawX^jr6 z7;T8Wi2}YgrSB~!2I-~Q__NvmtV*1}9P_lR#5VWcLFpj7HA~*uq-4J)iC4;CBp$>u!mfd-TF{_N_CpF2B?i_aUKw9AaCAO0JO91Om7oQmxFsY9Qq>N-0$ z-pcy2ED_@7f&1HbZKq%zv{im{;g5$hH?j^;DiESgay3`m(587h`&`|V(>MOL0hzfw1_xp;$|fod%rVAEfFrA9iILV{`lj!+m(iGS@NkB?d&tEcjPAe zQ2NnZjQnI>f*jXxi}83?d}`1#Mme5`NUZd_*i&{d*dFC|J31 z!)3!M{9e9GtbbzuS>tQgo7cYJ{e^0KP(_qX%(%!Ekj}B17d{834D?rM+}UvMRnVv` z30w}1(*b?6ZR?*>VZ1BtZ4(onUaRvX9RElO7fA}VDcIDP=Q29!gzTFLb|qdqN%$eF z@+wYAFz=M!Z^u1de%AD)N>nP@VbmyTp4##BefQrheEG_izF#;?{G{vWJ2U<9-A6<= zoO&2U%9eJyl=e-dN#?f2E$JAY$&Y*C47&As9maE1+4qpL`~M@=)TVmy;Hhf<`hr9F zRWC;~#V_~n-HfhU{iLH5rY>NyZg}P5#VZMF*@8NYkN0#iD4jag)f-d0%SE*&QJNzm zh;W#6!m$2r*_2ZiHWpH$d~1`pgu8_w+RqPm%#39m*m=c2z`L8u-C^PIZKs*Bsr}56 zV)hHe)vk8rQirkyW_}f6mWckxF#ewkHYd;{6V%FmNaUfL zTkod9FIHV&URfEi-tDD-=Jl^*{CVSzI02UYw-UvFN?zbk2}^}KgYq0Tw?pn6a4vv+ zvzk8_TC7IuZ7%pxrwyNVHu*-GVxF`Kf`o7mE?>Q}n~I z-qF4v65$hj%sj6sBHZn&W`{mJq$Y;`zu%8>Xa(k(cYDnrvK|v&5_M8*}|&Dm6|bW*lyeZOu0`W4!!u zx(}6fg+!&ZmXp~#A0B4jx!Ng4&24;2U(<#0a^zETE$trpI8_0y#UHc@znneixq_~! z`kjp5{n&c+W8~95e9`>Ga4;~}iZ>?0F*DiIA8SKH;_k3_FR$wFyOL4uq#(H-TfgJU z6HkiR`onJ5S-9xgLel=+y6Z#z@S`0&yySo@&Hs(wK|>2DTl&?p7vX)#>wqTV{coG8 zNykR0hqRy!Wtkd$&hC3zS?qO9P10Yv9>c-TPbTmYjR_k9x-EKoRUsP&^;IK0#>1^E zbj4|s7Bw#}ZcS9p+!3VwCA0*uw-Y~Glxcc z5ah;l?ejwDB4OJk(Y(vWVaUs77M{BhNZ^Ix=8T2&%{NNRB;(}=ejkPsS4qzL9fbxlRW6>d+K zTw!jNUcGzt(J#hxPhhEz*gzku;vTmGYSN8c^JW&G_&8XbDa6VusXS$AKM0O_-Rd-} zNtu6q>juB|>gPO-;Z0I`NfG|WEZ|OD}&1M>DKYD0UB(& zKBr0x9^0%jHxD#_J5|2s?0oIXzIf%lP|13oj!|9{ed#1Cfy6+F4QFMU}$xXxC8I@E(if? zf{``PM$7ex_>g)?`WU-XC&mizP?V#hrvq! zujRuhz@$@=xEF~BY-HKENEjQx6}}G)?i^jLBBuMztlWIN>#Z;Ein3jn<}OM|pi`R9 zO20E&q3RzjQb+6h^oZFW2423ta8ahGM32<6`1@+BB#za-xxYpr;vIQr$~sBumW*3` zU+gQ|$fDvI>N#;W{BF~Wi@)8bv<}?n?Uc1-y$KaFYm@8wrq{2(JgaWDZa>p#y^+kU z_tYWIJ0#nAXvuov>h2LrHtM>pdYbqSTc_emwSTx?GAdN-+D#Q$$UQ02l-YEWjT62b zFtOC=vrWSvDFArfL>{ll8VF1%o#xii(SE>}E_NWek!O?UCJv(3@>C)BH#=-|(Xb;CScAgW*_45t<}{C> zARZAG$D?5Tn9AfGd1q3P#ghs9vZ)-C3s?q5%V}S$_(znXD!8*SE|A?$FZ1qGc%**y zA)jX2wL^RkBlcfGzK&Ttlqv|*vZ2d~JFwzL^^JcQYyRB2^&we?S?ArtHoc?`ydfu zeY)*X^0x~0L`CjYwPN;sIzI2My1cPE)jyf$sCn3Kz7>m2rz2grK9})}D0%X1YXJ(x zc|kKyi8l6eip$!i-iahMr-!wgA_*|8vtn&M$ZTMCtk!5wx(tn4y!wNa`Rwj}zj#+c z#!U*#m@`n=sBxk*V*%{&4n|`hl~z(UvAliL?HL-nzmgKM4|VUkVZG>Hx#pK@T>G~7 zBqRFuLi#ot*%vx@+!O4*T+{&=gTz1;t zqtSESj&;V|0SSj^1J%ge}k+F|7TWhUpmynhg1WK>VlzZFo%p_VtQ zpEu4aHxpaY zZ}%_mBd23?LlJ&5JF|15IeQ|2P9-JBmt<>!wx1)HL*=+Y{&;7~$+FGWI8^#cs4Ah| z;VJn^jVqPA8&Q(PRx&TOc2lc6ufGd(*8+E|He> zQvIsD9CI!EKmZdZRsZ=h`tR4m#QJ-Bd|J5)F}NgL2TflYMudOA5UKu8Yt)hDV}$$P zi}MW*&aoJ%L@^^+BQ+z#CRRS4P!DykZhmyJ3ZJlzL2qFcdK1yFUbS?kGtX%S9eQTD zbMloOd@H}$GY&s|y{k*p!IYSkw9=1@ruiwmDB-)!Kb{=hwFv#Axwwa77nQSx9TU62)f0s!R8hW{I%Z zc)^|i@<4R>=dLf0Kqq6^k~&1h;7CS6aV*yo`W5J7_A^WAEF}kV8>QP${bUWkrKRZw%^&ziOo@7^r{*+?ZTDc&Jd{Dj*%r{&qn z?gEum{nJ*%vSwxz@87>i|2Ls46)`3;XIgBYZ3PXz97@SL@)c~Xtj1s6cA!OeNO~T! zD0JtW;|5h{W`2ntz7-drJn*zweHlIMRnVu|&+4PVRuOb|4-JnoL|=dW@ke0T5_*zN zJ93tARU#rHJ?BjH4u_)i9-@wu@^OT|RB6GbLiAW;*{V|2E8{fs*U=ITNs!%iWRmCw28GX&EaB| zlY2LAkaD}?gL2w1dhPwwm~_2UlRBIh+Nrq>&r$E+zaJ8)v-Y!t=*aGM6cuAEL|?I( z@A1j4QquJsASSt;vh;|-xhcae%yJ7K<#}!-^LSnP53-{1@_P0B zLW^Cl5GbG_UZB{a@1LY|f@98a)f3UCZSD%;2)E%ZR6Q#3@-b_xOEX)ylf1{1t}V}@ zmDzEqhKaEd>pt8NKk{k-!VP)|5uu_O8iL4yhTBjlNs(5~8`_w~7Nf+64G^?(U=$}aZFs#gpytL9D4cRzdtUr zCoH;QBLZB4<5Ur)a{PSKJd`~we&3;M5R;B9uFNun*fI?|t~8@YQyf0@koWxj=|+ZT z5$xC+O!LyEOX$B1E$!>bv|wgoVKL#?htLOmfL#ey9VcuYR8??Y&u|6Zi^|4UhrM8ZBIxVQdQqYT@hfBbq z;8JRmYuFWt@8ZhEQT&#JKO#hZpr}j3afEN_g=o-o#z-u38uokZ)-AL;cHmrOWGp~) z@{kPTJxNG4;1U5Q)**ZOAj^V<#gRngyX9C5w+~ks^qgqBf|8~&QKK{0VF5}adLif4 zm=VM+^H8x!7MX}^5)#kIe!}Nye|we*!7Zd?u>0d~AC=>w^IC*32^!n^Y7quHK}SX6 zTc@btZqoIt=!+#-Dax>1(8ccCyZ1D7RnRbA8-w$Hg~oOG>Q0A=T(i8Ter&4D3dExQ z6+r{GD>%G3n>uHxACjTV4vq5EP)vNhRY1@`bh>P;ZBv-!wVvHY&T__PybbO1LJPFX zET=wYWu=?8FGED^1d-cK=4(RFaUuuk_q41!RY87wB$8;0jP(L1C0aJ zLEqs_j=Q}&clRgOFf5!NIS-Tr1&{V%6>OpXgP4QUytf4B>pUsBxM5?$8srTAuyyzf zM1h(g9q>MeX)Vyzz!V_|k->E{6j`y^Bs+JqKoM<8#`n*HNJTmdk4Wh-LCCIrOLcJg z@C9rDr7$$LwT+E7&_=-u;W7#8y)bPiI`ry!cz8%0o0*@thSE*mrC;xrxA)Kfe*g3< zvGv1q%gh#yup>C8IwVez&cV4oqdPy3pTWo6v9VEztvAnXYi(6Q5@Vjp7a~Rc()zQz zC55n|v=P(|kPk8lI@;nqjnv03bT~|`L8Rb$J)&_0;=r-t;SbgN%IU^-EAu9Uo=EN5QW0eD?!02KSwC6D?u7s ziH9J5d;R#ct36*_iB@KZj?5)@K!6ZuyabsPLFEbk)p`5b_gBMXWMjOwy`+fm z_I-y}&_LWvO?mF=xjx@&hRPJdePN%_SqVe>*6f*B0Ia%i-#)^o!?D7FCqN60pv#eL zU6m9*fk&^_j}TZNklLlR59K};tS>He)}&>&PA&)s2XO{aXCRV~GhNg48oCaD98W0WYZ^yzUjl&PJVne%d)^mOQ}{7@+ZhXC zF((xgMyaM-;SUBBMfGZ8gp5FQ=R+_A)ZpolH{QjVtgU3wY@Ciy)+MO5Rv&&Ix&*~h zWHj8rRm6cIZD}zv`{B46im<>y=&A0&3_6`x#!7vd(E$w16qZ;;3s26{uo3X$#YY@Z zASxW~os6JCN_7mOD#nLU>V*;+Iuqqo{TtCzA%Hbd(n66X!k0vbkO_b0=JuC?JJc%B zf=y}{@5j!>RqGSvjt2;T&S-w#5?gN%rSq(j$K%KEufG+Bifz|{<3Hd!rh1C=yU69$ zb-XsVaTR#@jvZHjdmWx@4)*0a!$XRAmj_sVej?{4ybKJ^YNDgPs%ioXTj9spKzI~D zM;>l&D};5koT;nVufGs}d&kO(To~F&-StUJ<&F7WHTq|uuky|+!f_`&2tiuIrRO>O zA@x;iETbFh_Ta?~THD*&B3#?xx`Mfl{L%It$c0mb2XI6(jzEYw0^%YphxP^8*P^1n z(kti?kD|NaljhWe-SOe)=cgYb^A8Rl?e9MpP*%$`JZGQX9y+h;W+{B(yIR#nzKj<* z(dSAJ`As!lc}KF{wcZeVvu-GyUZo2_4r)Kyq{klYHDNbBeRHU4wP24Bt5R|=q6E&} zSbu*50zCp<#MmR`KIS!Wx-Y(vjGc=@XRm<u=mKR;&pa0svU5pLx(!Z?FOKlMJ&T+r*SWlb5AB-m1iHQ?4Y-eY< z6j=O_hBc^|<&slUX5j2&wE#+Iq3A3MfY3hl(uE73@BOhgq8O_3Lx;MOJkrm5l8(&T zGZrEkjZtwApL<3_Ms_{ZybpG@2^iCi_(H~$)~^W@yMCE$+N=Kj>Wres0y6}h;n1`* zO9eOOyX0FQ$Oc}NR+ksdA8JV(!OE;7(xy5Iz%q)o4hlMc3NOMP1ugu<sikNSf~s*h8Kf>J+BO=&DPe|-RT7`Lh5LI5#e%XM9xIIpFbibRv#i zdsWdtIH(`9aATQMq8@@!CZ;~jg#hHtl!*ZihoICr0q54c1F825?K;D`o7uMfqGF^# zzb9dmy9;|GAZFoHt~^a5N9 z770R)soo)va)&Q7Tb1fF2e)Acz(=#xGGctNZM|g zAg95NpxpLnudu`c#y`a=cQ`$J^hm_mcoaJZU4Lz&P&~k(*bEv$fTR8BX`X+>F-EQi zWiM=4t=Nkp?9?sHCWy&6F$kqT+yQ0n0yYC_s!i{+OP6+y?1eWO} zbL1^*;o7*j4^JCBMNNkz070)3Ac9yVCy2JmjZDZM8ZYk^mW@-q5h;VDis1iZ&Rb#1 zL3ejIg~t{9*RNkwlan*isaslENwzYqAU@V?%~f~#S-4~9!y4ivn{p~K%xzBgNJDPc#${>P68lU z`qim!$jVo6IB(v(iHM&Aiaa`@i^ePmfE=!t+pL2hl58ZeUbMWN1@+Q;nUFnKkIdn- zsvk8k;_N_%e2|)YxtI1ekIC{-bQpHl5xEONM*KmRAdS;ju(p>$CL#n+1mU;%)*Hd1 zdi^z4wFJD*8Mu>@ouB}Qk=_23hu($8%&aUP{U4fu0JsM5te1y}3MMH)L0w05HBv?n z!zn=az!6EeFX*>@nnHHzf*WIwj0&qiXbF6uFr0_@Mx0_2pjlhCP2#VaAJ%OfV#j>q zG0z1t2S?h%$|@I%FI(GBJ%S2V1H9Ir7&pVArpLp}tCY2zL-r2)t5g%Pl8Iyx*$-yu zbW{^2C15-ZJ7OrNGr-5JAiINWs0AuR79K3<#K+1y+T$hUU#f#GS{zNOk_r=~qo!ur zE$|5m@)@aRC8HO84^!BNKDLellY8FX1_p@+Cm*R@@*G7Sh$#*qM2q60B7ue{Mcr`k zSZoYE$sS8v#k^MU+&-~Rmq0s^tJF*R>55fvYL1g|*)lm=Xp(=(`txdLz+-#qD5 zkmE8IBGuqA>!78lPq=w98$dN$;Kf>8e^uDT&!07r zC5w1e+ztXR!js?yBJ)JCU$8ymUAwrbp92>WMkI`-HG*S+2EOLaMOS2V;&5?yd+>6Y zhXVjIP!VK<0UtmNp7a9CO5)!~?so${5hxf~H8MPULB~aKO6bu6V;AirLXaKzsvwmf zY)aBLWpl#x4vu|P({FN|1Iv)uNI(#c7beW~n4APk{Dv__q6o!Xu8GhK9b518Zrtj+ zDK)RM>^n@jlm*DnqgA$>wd$pMm#)nw$VmwlnE$WD=Vr-2$C#Kz>G5Y=>R!O^`~QJW z9sk{+R!J!Bbm?-=-PyT-;sX38q$Ie${(OX|n4qCg{% z3Y9Niaj>&XGi`6Ft>qe^3~~BRTmNJs@Vr}s|JM`*|B?8XiP}i6xD5yxyqV6y9iKb* z2}v+^7r{dfY%g-;SEWV#BeMqaib+4QRctWN@ILHLu_qmpPG41aN@2oyB9Jrqk=5P1 zcB8p5F)@TeT9^uyWj!`*ybgR}kBLo)x}o9e0axkUeqh3s2S?r@KqAmlgg@d1j;1rl z0sz>I(clPxC>A2{QucGhmACVNQ81gL;r)#s-klCh5FCOwp%rrGM<)*Ien5m6NMS#E z)9Mw!-83F8Hz$Xa*Fvu|GVUZp;9--;UV_WV3rqq^B^^OBB9}=O^22G+NY#It8qZBF ziS*~Z)o_ID?Mc0DO6-_OfmhqRcdvDJJKzyc3Gj7guY(CsD~@y|LlsU>?(%SKRjU5_ z1qkd@;+JSbI(3Fg3JRa>hbLXa=mEe)u&q~!Z$w;siD45u&Zn#RoR;xAJ^lS~)cHFY zQN%`yisf1E^?JEECl)1cvz4t5sdPXbUGZz5Kz9`M_LeLJ;bo71Y!}vw{+p^)vKYr z7FUUHWwa95_@70SM0_fJAtn`_gRA`qNw=X96gL-jSpjVwv9%Nz%yikvtGF$G?b@k8 z31lVnQ&WN9Ur_*o1KvkTseqwD1aetu4gQc2J2G0~vK%J6zJP|PLbkl7g>~RT=7HW)&n$y$^$-JgK@&v*PyG945x%~W0uw{!u*yskRkIkGi7ML-~n00n`wgZ~U|NQe$ zPg>r62M-?HyY~jzbz~K5rMi)8>=mH`*~ph_D00AbqV90ji$3J$$>$=Y)+?B|1Kz_h zrI`>U;#JNLotgwN^kfh+JwuIi-uC{+5ynDnGLE+iXP~2#6P9-fvxU}!5sSqASTN)! zv2g_HJRg86#_2E?!mA^ZKq$A+9Ghm2t*54;QHhaZ3Y0kJ_8yD_o-1M_lVJB?)Y&LJ zQse!^`oQFYPdE#Hs;6fL2A`dk1>Q_Hv5^BaOv0-n#VQ9H1u8z#oPWY`YhF^G) zpQPBkHwlReU=U_Lfs&5Y*yr*lMO3}DVfA^XLg+0L*J^!!03mpm^?^0>C7{Wcbd&R7 zN73Rb8l=P?zzk8qM5ha;c_I1bh9pA5ETtK+i%_Zn@Q+5XttH<>;g`2g3knJ%JgO#X z5yt<8&^cqq719J?dca$o%dOyX!Hbd96}noC_{#LUU6m*97Le3D%ZhD_kL&Z447fLe@IqyjpC2P5D4u#VRR2b?g?2ICijKm*uRZl@_@ z;2EeFgbq%LT@QqQG#}ezYP5cJfFY2tU=j%$S~*etM{x;GQgGf7tWx9=Q52F0W7X=8 zM@k?8^Aq!fuB3AI^y$+eWZ|rZAA`M$Y!?yL0B{GPtc)plsYDlRaX5YJ7!oc&*@urJ zUenXlcXSY#U-gvRl8%n6+C=9B`1vbC1xAL3l)zzGX36FiPItMJ5);42R)SQa<}qHZ zbDl)r<~$Og;YJ3gRuQ}?eCtCp+>6hmzZ}RD1JWXy#Oe&w+D(NV=8dt8sOoSnCFSTK z?N3z;B_w=5eq4{4xqw(_J>J%wWo0xJiCxd1{(chg_CmcS%PRH@^xOy7RTpDK?!nJv z*~YM!r>RI_KXBxgUj?6*Z0ggTKl)Y_<>CAXWNrG5^eiA)R08m>-A)qlLXh)llTP%=pBC7u1kxD$vpl)4gC zGT09tdIR_eVqo76MzmNA zqi%$n8#W4)B$h^t83v8L8`HU6!e_i7!a$miDO?~p@%n_k+iUHt^{gcLbnq1Rqa_~T zyD`OsD_1?S(J)QMLq4`*^_R&W%h9ONy^&Y*xGo93_nQ`PJ~c z8&$?Un4>1oSC2ieK)r`h@?6gE{ShwW1^aZu9bx)7zLG0UmCV!q?xx7@z_>)3FuM^3 zf);7rVDV#cS-op%Ig7Z5>iLmeR5d2rvq5|w#n6h3aiH|=d+39$=O1?(?!MOL>TBXc+u(jh_Wi5bfAsM@>;eJyb;L}~!6d%Yo7{)wfnA(GQw>HTJw zf>R{gR1kinJcV-uQY6G_Db)39_)Ai`os6^gPJFImHCb>jSU7|=HB4^=Fml0|C9%k4 z)Ie)m)0D^X6G9c}PS*v4WsJYVLy2h&5_nH{_fS5o4~QTj1U&8RZEY=qxq#PDAwvST zijsb0NA8kjXv+^_;+O$%ZIv8^XwLvdyLuc!05UOYB+2I3u{UoHWw+;n4u)44>&RWn zUFZ;BDZu{Kf%)|)n#N=_+{Oz0A4i_y1{>=47;u94dR!?vqE0JG5gw!^LG3(&>RIGP zi_rXKw?~hP!PU{#236c1s-N9>AF;*`6cPiIN1U=M#Of_N{y@h_MVj9)2G$uYw>9?nxCx$`JLG@+htmhCu<< z)ub7rD-0PJ=+pgs_jdR9_m`~N=XU6yL8W*A>;brsM*TBzP1e?R&1Gd}MMb(b5uye( z;qTsY{))ZyZ4D3dtCzl|thr<((^gdtwhZNqU0hjFiEqFpG!f#vz|`fde?>$^VO(Sq z;L*;9?+}eq;3A~$WMAv+>k9~fF1@`TjLpHXJTnfQkSWwezyH;a{Qr-$w+^a$?ZQB9 z5mZ2s7TlDSbSj}+LD`}RNQZQHhl0vRl$MYNY3bN>gLHQ{BHfL|UEA~Bxp(d#ckbLX z&Ya_!{i`?DyVkRw^(;3rHnt4dRS-=+ciAKfYUTxBqOR^K;wgL-)Y-X;vV4EWza!^# zEEd^r58ABxswHp_2)H2;42ZF(zdu%M5FmQMCh!|C0bT@plA$+UFG0}N`Eq4x33OS5 z;7rFVz8dmESs`e@s-^UTy9#=|5bTfWa)TuSgOzee0qpW3KweNPgp0s(RY2%cY&Y+o zH0rNv5t;O^K0 z`1&t_3MMtYq$8wV|Dn@XBBai26EQI9FnB4$rr-S-Y5xKS3NLsD4ir>Lm=~R4W1qD# zko#npOKyWT1G543X^??Q)K~|%g~C@Ndk|}&DC{7g0CF}!O=Tj6OLe)yEFK;nVoTgX zh$k@5dDpmcpPuL7V)Xm1kwRCNQ;Dh!ZoALZ@BmUINT7i79K76%(D}|h+~5}QMj&T0 z0H-i0hIX)Di5&oMl0`9#ii(O&1RF|#$gw;ei?rfi&+j`|oI>>GuO6BJvjP4hxatZp z5ArPX5Z%9u5nd#F2!cphJ`XrGL#F&8s9i{({`qBLtTEcPSO$>}6yHC3^dX^sa8Pk+ z9^%8AZ*DBwhzIMgK(b?RuVW);GiZ_lflg0fX3Z{-?_C5^ zZJy1^MmE5z52^7%3>F+cfQdmg`VAOkK-a-_q@<#P3<;#W0H86c(3mVVQ(eqii?pbO zb%=`@h7}gk!RH6H7@%&<0re3f0Bl0iDQE-~0HLlFh9IbY`6?j+0amvVtQ4FUkJ&^2 z9vwjfJhI1f8je^jzz_3hwHOC+fh*8Jz?wqRssW4=uz$L-Qg4Zg;PGSiIAH<;*$c+t zhv0;Oi0d=8@mz<((>o7rhp?H8lfS@f zB1i1s(s?p(0N;;I$iv=^R@W!w@P&AKdG|>4{SlUt{~_bsz5n=YO_5tqL+P=&hZE8N zm)a*N@x(83bFnafQPv!g*`Ij3YSG&zwg409D^&Ot2LbWW1IXh;+)t$n8=CZ)FU2mn z8dR#*Jg~%1K>{_Vj zHVt=D*XN&+7CU(ZzHU_Nd@9wbK(s}&M)ui4fB^nvK#YaYDc>(4suR}I?u~K1$qC!dKfmVF<0Pugw)xeaGB&;C z-3lnHIt}wgCxyQ?>alPxWS@rTSXk9K%(wIjIW#hWh*8sTuVTB{KSQ^^1qbH?XHQ`A z9SBWfef)tFe*vU?J=L!rJuYOyYDaZ2gOW+-WGZMu3 z5=-BK92z8XgMzN|of1(mP~RLoJ)F7dH~7?+zc5PSBD7DpMwtR5m7FxYA$!CKRYi~Qd z=lYuTrduxa4l;-7~+{Y{-q<7_eSCnIJ|Fq0$$QWSv1HLW zd$;ogBHKP@Yt)}H{jh#Ms#Cmvnbf@EL8*(lB5A!%Cp}mFn}T2qk6RNezFU`?i0;dMZBr zlgzB%n&cRQeoLy?KSFy4qA0+o%(~+}q!@{i;04^I;9Y>E5y%(Q0N@TRBUnZ{nBl~) z*n}CFV{m4w029JWlEZ13vswS|}kS$RJ zL%TDt20aX0MdZFcYzBEc?!W15Zsl4*MgG~3%7&5 zY0L4~WC$B7f!NYB2=b#D$~@pnZ~bmII2P8>_wy|VXaER;@aYZ^HbIaL=^YT{t>E=d zc!ABhfJ<2nAru%X@M=>JF0)HU;KUH+rJ{Kb)@lUYvty(tdn*uduC-0{L&?n#P`vHwtr2V zg`zzO-WR!29LTdGo=O?lPDQ^BKC5(z%U1DVU&3tckSwmw$7lU@?LJoPtGc?C#<8;7 zUE^Uy%C)^9{lwwD6UXAmb<%b>!3t#@F2Q^qQ{&pttyeLda!oWO>%}As^G6bycNdLp z%PS}6t3sD^Q~y-2PG(jG+qV?c&tBbrH9(VxdDqz<%}6ci$_x-??#SE4!s(;&MER=I zZB)^X`K(4|vptJJ4?ZUghZ5?@rJECnn=_cy&(FkaZ%+;%&+eJ{k-t*1EImP=7?D~X z%SyS&m7xiIKd)2o%+VmfsLbKLVO}gV9PM8@z8>%p^leJ3Zl-w$%N*VQ7SlqE`sLf( z5IOJ8xXyIi&+Jjw>3(}QDOA?a;;Zsu+_P0baj`=N%O?DWhg#ok{^FI1oH*2*MTF-# ze3NdCdmMW_v#U}yWEaawqrR?GTBx1Sm-D1YDgcX;LLdPyKfQ=N6EX4&79a*CVeBpd z)&Zc7wtFkeh^G)_0`R-Mx+(%}P5^Mqii_bF0G` zb`ALj)*FXLc`>YKZ0;5^)zDo4W8BZu*aQ4!C}E``B>c9`8WWrPCJn3~@GV>c6NfMX zFkEt2k~+P|0DdCB09}a+K(dn(Abfx~flY4$I~B|(_|bt0*{zURtAsEKh(%a34&)X9 zY=S?xk?(f8pF!{*3*8Zyhv=8IoUe;6)6yzsi;;AnE`O#$3{eN?yXf)8FR;DfQyD<^6Jkd2;xGnlp4k2tAk+<1ziwG9 zH|8-e4&vsxs7YQ#WRE0R-$1*H|L2~P<9FSz*+4HX$vBG94Uf78WV&YWi*2eRLZZjXjHHT2`2oLSxC z(w}Hvb|4CUH#9XbP0@IdazDy(N1#t1v;X&a-)3;Ol1jnmZ`&YC=+5fj!*3=Iw?c4- zQ|niexGuJ4Dl>gE_8}TDG}}kDW`m-R={+28D^BOS6!&8*&vr-JVvI6m8Z0_~jv&9R zzU4hTs!);DNig=f8SSl`y*d}PIm`sRAWP3U#VnV6*L9rn#Von7v7KX%sn2*~XsSBV z-u-R`r}nZr0t7lu^V->(9Ie4V_pyzH9CzPv*^0u8nK*gT`<}&vi}dK%mx%=?uDKK& ze0+9d-jm16@PuaVpo3~x`jx|%7BjG{03bHWy@jn2Jgccn0^;TpnwmcVG#_bzBq0DN zI0$Sws~J>ZLkMb(3o2nxK~RPK0)EfxSjjec z!Qg!ZERIbm!0$Y;7$%r`6~OG^uSv|nenJE_gI&gEci7$*2(!-@*{Sqc%KA#sp&@TM z3)K-n2A4sn?hes&%I83Fb8#^cEP`m$c5O@|Ei97;*8DB%ha z`tAWx%3<=oB909BE|4w)S`MNP9A<2x1^_*{2y7(*Bw!#oj25zC*ZV zxcI<5FyoI@iy`IM3EieVg43hNwfsi$e^FLkH!W}!&KrT#EU;>bMwIkUB0mSR&# z{``IS%={$DjW!T<#YumzWj<}}WVeG3PK$v=B@e|%BP8;Va&&euYRe4hoFqmTVw7B+ z{J!F-AfwH-EP6EOcp2^G&ncCyR5s6)+Oy`CA5P4vl-h4~@7gE4`ZL3be(ii22f_9g ze@j)9Nxd|mV5XVFaPK`I;OsGDbU-<_79KDz0Pyx-pBu2JlF+(fdS>Pp85z?ZPS+K6 z5EMs#0l)|uY5R8kN09{`A(a)2$n_0TL@!O}7Z+o1rEG~^d3 zKg$0&9S((3hfi+>f7(B9CN0QPU_ZA>Ajei!C?W5nnTV0=?pOqytvmbr(WQ$dl!Qtg!g*jXPv|EX< zL~YMl`HZ!QC?+b!-?R14uW1@~!8>=d1ys#aue9RbG{xB%z(G*clW8PZ2P{`jI6ZE7 zix-IyUpo?;U6@O|mldL0H0tB5Rn*FNMtwEVWDHHf@6KM~uqcNn*soecP~96^;YXku zxFl{*F`@%ZL_7%i-&^Y|E--BUCgoMFitxXq_#K)9k(GaY{_@S)`7tkM;gHmsW)vYy+#8Cb8+F?AA*r~XRa*?xK5 zLG;e+3JGO5Ulr(s(!c5^Jl9elKEIeuf}~&8C_KMb@^?9!6jXJ7Q|z~`WVfoTc}SV_ z9FNjGaz0iXay(Ygf>HQ%QYAxr7|V`9P1gGB2Sx->OKE5uRs9$~=?5xW|MG69K$1qb z8yiwsB2$5t*b6{pY`X+Amyz*@6&kVA<})wY?iJbXNtWR;&7E&#bj`yAsS^dK}B2_qMjI;Q@xc{;Okn2#{5tvp$p)u+{$# zrN7X4LXVf?&O9#^p(z7~OseLw@;!)h;ohVOQ!T}q4l*;O`2 z2cIRq?^i!Lz+VsltH{wfHnO`XVCN-D(N7W?!DJ^L__dLoExC=DS$<)&E>GivP+?}$ z&%5sg8McjW280A^%`(zQ$^L$2iJ4Vq|w)I)(r z1_t`@@)~4d2R(Yj(&e~#`o+}T!LF-0IZd35dX#7P41EZlenLgkIe~@Y<1H`;%$8A98ZmP55HwA_CpR>i*(UWgRM%eH`Eissm%t zyVlS1Mp{~0V!2<&{9yKEWMmfK4!&g!0$-2~5?&i;kXB?R#(jo_dKKu7<#fDZY2TV_ zc0$GN#Crr}4ZlWUPQfHX>c{f=^DiJAP_By^E_GP1goq2w8Z)t1Z_3$8x1h^j25>At z?|%t^4ikp3ul#BOEQ2mRZ{P3sOJ&KhJ7449feg`hYwuWKI9;?~C7;g$m;?j=hWTjbdP$ zQ|vRvsQKOqjnVKc*?qLXaH__K8(Kk5BU77VX;h`%BUU0sy>NQ)^EUIvgJRL6$D-s` zhp)7QciJ~Bt%#8Hr)M=aVSVU((yxW(-=Lym(+6AoXqYpjh6heHC=hk-p8g&4p2Q%x zE%<;U{C?qVs7_*|JnAT-jX%Bp6h(-?7%P#9v*U8io87Npq+}*a?H}*xHIa9=^M3QJ z>ik+(OCs;#w6WtSOl{ApvG8-KQ;2cqpJfOLb*sRrLbb6RH?P_C=bjGR=Tk}DUW^sh z-6D;vvS+g&Pl=G_OZs;G73uoY^zaXt&0GkR8T*wtXBUN6P2f^kHO^2=FIgqP9frmisys@{p2JCEnvEv-bxj_~R2z1ys zNAMR^QZG*|t0HW|6fJOikI)TiSLWa_2ue6w?&Ku~r0>NDk z7S(+E6gb$X(9i~W&QkX{qaHeB0Sf^@g4@`Pa53c0p!^g7gI0j8plpi~ZE3%~Ue0L1 zZ@c=#YjAe0VVOq#_;gp%xGY`0aW&F~e`{pZMN4O|gZanqca0_0Oy7Tw6V>osFELS>csyG!* z4nX0=;ZLKTctVJJF5grSn=U&pd$nae#^c&c>LBWR-+7qqWpmeu)5!-KFK;}|$Q)8? ziV?>1VuM8>RH7CW{!rjH)9Xj2K1TqQC%P=V0;EO0$Nm#IyFdhk6biO=CB&jYiNzN^ zsLpkP=7xWkmm~2xzU{NW+yaRKe6cz(Syff6WvkR1dn5|lqpfQ5v>{P`-Dr2;Gp zz@#=I^@44n1=HViTP6kQnh;1pB!DHnLm~mo!UPnpH~b7(^>-Y+Aib{)RW*>4!d67N zogQORCmK;q6sxY^*H^kxM~`mW?WzqM83Rg0nJ#g2@}z&N>lH#&JkjQvxCoxP z+j9EVQcsHvs$tv1t`h<2)3Lc;MZ8cK)6b_W>rR=C)WvqCN3|MnPfDC!m#v;?tiHR4 zNIQw(V#n>rH*1J3aZ5T_xq07}qB~V>Hn8$Bf7xoTkS+6Mez8N`O_RpVBNKlD+UB1R z$*ofBSqt*cemOStrOGUHl`u7atsbT`_8{JK-1km+pG?ENlD*o+Iiz2Nhak2d-jcl0 zO1w>R)F0bEImu(uJGcKJlxAp+IrPP;c2hck3yQ~#6y+4ho=Ec;*@_uLZSs%Q6GrQz5Wa4tz z#4UQ>tY|WF040}P=(aCyotWrGeky8rmcI6wX5G~keKD^Y9QAjQ0)jR0{5$4s5GuLA z8jcoOoW(BapEH0u?g?;W_D0@1Ln7oQ3Wai^x&6D8%VnObigeGV$|Enuhr8)L=2WR* zrZ$Vm-PKyA9IZZTBdZ%#x#&`6;Gc2goLTPpZnZCDXf5`*@NYUvkDbaUcXORQqb_$< zR#4!Xl7(Ms_{nafRzaOv#G*_Ur4gbr-+cT57uV*&W>jTS2c*Tyq2m-K{*7}@iClIr zPm}60@kV1e6NT7mh5r1NqZBjZ{gA&ggL_zV(@^nGq@C91wi}N-BuUcW|4G9ekNAG~ zet9srU4i=G)^0^4{ud42@+Ne1`4cdjJW*wsS_uEH~i+KY$O4)eN%r9cc`YJ$Ke8#a1klt{%XGz0h{k=b>8c9ArswqATXUe>q|C6L+EE-zR8OtnjK4TR~St1Da4;O%Ni z_Zy$OUwFhwNVX9On2UYnI52W{oP(g}1er&72LEVAv-1+U8B!h#MomIDCABAs&Je?xbZ)+h!D6Kx8C&0dfa;8F1eQ;R3zwcmuT}qvB zKK;^DFAcIbg@k5f?$JzsEzB_RK!FS}yuu%ugSYr`+_-4etX7{Ci4)#tju~cCK=3sG z<_=e54{G(0YWi(mvZI6uVkSnB7L1ACA5&j3XIF8f5y1X=t9PEpqd`1b0V@5i-1TrWAE`D&S8Q`LoXLYZ%IxODQ<=&rR$7{I-Zp(;C{m6O|Aqr zqY1bRQX;)W%NT|YrTCiyZ|vz-^owi3G-R|eCs^45PR)_y1?xlzUHu{SHA*|4 zi(h8{E;}>J!1rJlX}*ygjNTK5d=VK}uyY&1Hk@x*rrxAm8!s@D$#0W};A45UxJ2t9 zGg@FAsjx~iwJekKVR%xN`X?0ewI^d&AXcS>;~GpGf4H3819y#Ve?P^#MX7G9y`O?Q z?*h5FhRt1t5xR1dcf$#4&5Od@_=AR{0+$RGor+T{#@d(}6DG8!29gr#%MKk&OH0!E z*o_M!FgKuRrIVI(iXrX=itAo7{h04N$N9=tEic-7O-U}L8xrZ=xFY$b9)D z`!)IN=yC9U+kf>ZSe~Dp@lZc`uUk2R+rbZY0_9%SAI>;ojbT*e>{5^3L^Jb0KIQDc zMQv6^ZuF`zX-Re)`N}dU{60UXKE4-@1{gj8MeiDIVicw$EGph+*LJY9BvoB4m?cTA zV_XI#La33N)ox*2@)q5TG!uSb%H9vg1hBm5Qw_9BrjPE|EgW4fQID4652|O|Dg8ob zW1ZIa#ALsx_%`mweFY`e>SwcyYw_K`?LWf!oxxXDHJTSK%gNnvu0XKM#QZEmKO41p zN>0N_PGnd6-Lr#5P^U=czH_>`1#cZAQ$7VLheP^=abPNF0CiESRS5H{D28~MOTFr+`Gur#jA$)w|-`%Olm;c&)dc!$hf=MF;{yE1LO|uL~@_+Z+`uX+?TCgU#8RX zmDx1PRz13vv*4ThEtelN92}f>gXPD4lAkv=%RvW7u$(V&rJSFd?P1|`p)qJ5EZ>M=~5$%6b^#Ozu`4(3oTM85HQD#V&170^c(h2 zzDy2&^W7eHP8yEY8^!^B{SWVRPSCvkkjvNv+cPvnS z$}f}I-UcAq6iHDD&^ayb4Jrf` zr@*4tXI+5GQfMK@4VOj8HR)NGfbmr|v-j@TFC0qJYPph7uUi<_rtd9o!AYOA#K!N< z;2r2gfE+TjkL0|U!g%`rS!_Te+iwFL=LOaLfM@5$%A^|Fp)ubhKLqF9&fL*e1@C_7 zf*oY{y~%M@>|3fsye%Pqp7E)5xDCAv@naoOyt68O_2`?=L7G!s?yicnM z2#W|Tn2pP1v@y0VA2;SpT9BjFQFF&FC1w9%E0eCSmS@QfuOKb1`Dj8LGs5QVz8lBB zyY6D8fq#%>Lkmf&w!a9D=h#%r28GJM9ZCU52NX#4oDcJ<}teiGQ4cM@@TLYRNgT>iTY z{4~-Kq+!9F`y4xqxMQLkyENYd%~BvH*`ntR4p5Sc?hQ^ZX5p6PH$T0OOhm>omZ$&w zNNT9_l(j28c=&CA^4rmZqYdhNv}!=>Zx#<^y4CJYwkhQVACvl-mPfG}5hZQn3^p=O zyy<_sGa|J6&mZnS%)9h0Lz>TEwSvOb-g^St3CgWWsGwhFe!EPJ7uY)IqYIB21~Xrd zj7@VR+r7&`J3>aN>fc-?3M z(GLC$#!PHu#1i#MSK@1`qF zEuWntv!TBrAofoP(puU~e?m_nj1B)58@`& zt6HdxUC(k(sL?IZU9}4`WyGwx`t+MS1bI7hIFLFB2;lp(MTvo(zWDKcNvC#U7?EAx zf$oz7_d2&Vz2=j^J8ExO1QZp0%OX`n6xGrzX?4R(=fc}Hxt}#et~Dy(va=6To{p10 za>hxb4f+Tln_-f4t6{y!$d(yQlqp8vN;5~vP9_&Du;N>37p-WSCT#uO-ADp5aBuW6 zG8WIjew8kDedhMChb-2xXcT=S&Y|}EE|5))Ni)kL3!k4)stSF+f2El}y5`nN*TMe# z@&1Pm2Tw80x&=zB{E_OVO@V4zN9svErN-X3`xM@blTT${->@%LE0ARjEje&;q6g7w zZZJ+QJ>RYa3!M?&Z}_T&h#{zz58=^I;VU~EZ`Gca3u5@w;?OR5UPA1>{AE$6a#Qi>L@Czo-RvQFc#OE9L;1L*EPOiz8A zUao1*oSj0E)VHYS=SClHI9o;aj{v+9H6qz2x_(XH9H!s62sY(HZDbQVXVjxz7hXnB z>iJDuyWHFChJ;9LZ$JuiZdJ6eO3Fu}Hpfn#QUE$A^5 zc*l+kusD;0>k`A31y%(TeDQ-;6e9b28Pk6Tv%trqGG#9xG2uBT**r9etkL}{*>`3m z#m06@RY8 zgy&gPxf8%t4VjX-GtX({V_EyA;;o=ufmH_oB6Z{3<-DlJ7GM1Rxa#SLsF$R^GAkq~ z>l&#LAb*CSd{r{7t-h5Rp^@xSlT&R}grU~QUhLchhTIfw$@h9c)tMKmn(-&pX9$qr zNKL)wrY3c=tpDJ0Qy)u7SvqPNE32cfM!e|v!|^6JVpl3JQciXkf5yzJ*=nCo4w$?4 zV6X4S&R)N0?b)r#S%0sEX6ZNc6*(`BA&j+D&ePP_kopbrR9IUIJPrQy({X(K=bPvb zO>Vs*t0rR%#eK$T4GwQ##feuAZ6DOYrwP_D2OXh$?QZg2lJs361^V|uH_hM~%H$W1 zuelWjxu|gw9y{iHut(dVNY?rz^z3{o`Kp{KwTjYAasV&#K3?O4hY-Vfppv?bq1f?L zE}x8n6d#$-m7w~laIGu+kuNQ{6Df9*6Nz6)#lJMLe2^;jvwXZiD+@e}m=e}fQP%J` z0mIr}FP>Kd+vl;`fJW_YY=Lf+)}yuOlBlm`cG6?cnyMhmiD3KF);6qPltOU>UR>28|Ebs;l8wMJA(8$cJNtbHPK$E3!^C9NA-V-iG{X7A zE4Yd!#ALc1WtwpM=u#Vy1@uB~f!dpV2LA4tpFBy4D(%Wn72WyZ3qkx=R@y|2YtTE? zPGS<@5n-I8Vg?KQ`(hb`k{Le+$|aPp3(&H4Ut5#{MG9Ny-zjfceh-&lwdh{HVRdcK zlNlNX-(#THU(SO)b^2KI(JP+hCEwhyEuPLL9lG6=tHWKHZ)toU3fs*%J;9If&8CR*-GGqiq@GNH*-5m6hV&=r1GOAO`HFY?owidFsx$ z6o){3^@EDwyDz}9*+2(xNEaniT>qBa(4=INhKQ8lI2H?6OASbs=bjI}m#3O(U(sxy zJG{DQF#f9$4_s_EW@l9|(dg#iYguWZM6Hu#Q(rUvUR2Lm6mNnAWZAejU*o;xF(1x` zdn2a&+;CbrTKQ-_>HxpVKJw+l%hY}C!1bppwu`Fldl|>POZDcRaOLERf24^D`KHJ!ZwBjPEO>-bJG;M`4%J>nWzz88Z06IeQM? z;>IWN7KJ1#Y69bV8tB7jf=>M_#PEaIT5n$HEUO$>m>k!6Ym=VFmWpyYf8mk}!H>hI z1%`&_Y2rO!UU-c0S`3e?Z+nOQkxY0YIc_>q%Q@onp4J+{IDI<(HlIIIgj^WAFJ zl2ifJbZBRvpnIkbKG))??^QY+e+#~aWO5ow@@b^PrE{-^NnSN3oZ>zEZDMMm_c0wqc2;HL`*BQMEx zUKv%3tZgnNgaSwVSdOX2(U|pm)0wz`Wa6SCFy;iKjX*Y`Rr9V_-q`zwEsbN|e&OW^fJ6{cr2Ig}EI z|5SGCd1sD&=MP*kTfA??l~?xDC%TwL3dk|srNOi^sMVd_(*S<$l{?25rA-J+N=a7n z(<)+YI&{W#)QppSayuT(b2bnYNu)N0b&5=G&pONg9lCus!=3CeL>Rkt<8-rLN+zTX zgc4o!^&%9n!lSCaFI~98c_W@}`y^&1Nrf8oaN!bjLbMGk)fn?@eeGbdv+K!)&sIwq zH@i%Re23YOMdO>O?2I(zn-(TBwuE3glEcSSe`Scpx7#;nPzNKIW&M-)tYbd4d16>39$f^zv7<5uFi8)m(QKQ7$#_tVG(fWwsmk>YUlwmio{4_ef{b9_VL&Z;WO<|-x@10 zczpZTIS)DA+lWJY8GD>~iNW_R{s zE&?5eF*4frqDKg2IYhb;b4YDI)J&3XAc%V-qlG%*3>!(pzWGM&hgxRMnY4;Z3S9{% z-Z{#Z*>u*B@8a_E!3;tF=6LT;VG^ zDYe7gFC(4O)Crnk3z44#`(8CanYY0jifRt47&!!;Uvb7a%liuAa5S+UbICH`;z z)AQx^N#y3Ttvi%;F?qH(C7J7)ya)GZ{NG;Fc<^jyD(AEL)u6GTS)~ zC|J%X4-&i^=P&Y^T-=A&Cm^>&DJ7q1B&3$4RyS$QC$DHdwBm7u-gr0X&T-eazj91Y zhw^ehSA(I>7_SW(0TM?@CnNaz%d#K;T4iWl2mAjOBP*2oE!wJ@^G_6O$Wtfw5cbv{ z)ye6P>kQupv$hYfm@wuO1_@UCj39@~d4D1CxrfPrYO!x4aZ`hdWxpfIOqdZqT($m3 zR6o3ya{* z3Q@fG6i4(MoD2#~s*-ZJ>erOt$hM@V<)sCnu4NJ+(E*<4-5dRjqwb>@eKQ77CT98p zhpRq($d^@m|C+P@_aVI@c_`^Bz0vL{n~^YC)N)k7Cs1?pzz00+Q{%mZd48l&v5cI={X3tet1l&j(g#MyBrt0*nI-_c7Ahe`ViKV>7cnJ z$h3TNZ90@p#)*6gbcA1N;&8CJq}x2BO7$#}!qaKH?p%mKH%NBMsy_dhWT}?>h&6nv zXudt(3{C2q<-!?k*r({cAWu2FeDqu&UTM2w11&g3L>kG8=ub%7EWZ0U2@ujH&DBQ!}7 z(Mm$UMn+oQ|EkkK76+v^;PTv(xRy|-6K1otZCquMDSlH=UZ!Q%<_9xEl8m81CFfnA zK7FJj<9%BN90Ut=r$=Fs11ZP|Z zRCvu}h3!Tbw6mX0I#+*J$tS0Ay#B|Be`abVyy5P)u=Jad*Rm!$E5H6+o82*tbs5`^ zR}d+&=PxdIt`j$t8AiT&SV7zBoyN`6qCrOIBQpu7E%pPXul^y>oKc#3sKXiFPVJmz zCm%fVYVi8@FTWb<1dUs6voC5|=hL#sC{J5P^(=lSg1X=}L+1j#(X38rC5B|mqhTwZIUey(e|R%@_fAMrs4d_hLPqp|0o^@D9#6WLm7DV>ee zm*1*2eDfbh$QejT{@&)g{zh@}RhSy>6Cm>!jZaypqU=Y0I@rr!Oq@EXG*9 zWhuE_UPkT9^AvxLLX8+>G{tY{I{pSG;+;4 zN(fya5HiaM(B9b-xu?52_`a>m`OVS!TC(mG8k5bE$o}^BcPVng;>>kYt#l$!nUX|3 zss3}N?=ZFBghfZaJCvb77G={A5ueu6%~V#D>EKLMp=M{pQ(;xrutZEcem*f zQbaw*^Lo&9%27pCVa4%oG0M%zgI|YAxLUJIuHg_kdFl+VUC@^+R~<_J4Xz*s)*O8*`Mhu=bzMs$lgrV**JwV9 zKRNhu0Y%LTi4Knab6#owXcD_`kjOG%|f`9aDWTSa7*31MjM2 zW`t@?L6F8z<| zMN5vazqqCuh{Wtu81UWI)fWl)I%P%1p8s6rqv3ys<_0!~a9tR+C)%}cn|fyMtVAsC zyhya$gz%tt0aq@|{`mH0l99VtPbxOEO*%Fsc}sbBwl_`e9=v9eK! zb)-NuH`CK;_>PiY+L34#j^q8Cq$=g=_$qc%Od0<3V=@VEGlAG4mFn35Yo$LA$k{fOZ!JkIWV1cZyz&&Q%XdzY zkW=wu@1H}_{qkuw9yQ}gWN6s{4&U-)k$ujZa%#_3Gww-4j*ch!*dKog0=FydypNS^ zcAU4tWo;IYdd|q*FEK7*0ggslyDj47-2Pmj|2x<2U4}QH)S^o07pEt#sCxNsE#K-2 zDnsAU}pnUs>Ed)hu^(U;n2AQC@MBB621d9~CC zJ@Jn2dAu?{mQr($gIG!H9HvHSKFWV&*V4(4G%rU%@fL({k-J?nls6GloLX%b{l5yt z`3EzKR58@V*o!oHVMQu?nb^IXE1#-NO(aVmAiK0#tMiLY2l;co-r)O4wtC!)c4hzn z9&s0(3N{a`x&abAbY8R{v=?i)WhOx8TslT2yP@U&0U@=0Cal%R@MH!ZjQC=hfP*Fb~``z7UJ*yl8g zbehnEkoFjN?$4nSj)(5b_)Bl-sJ#{c{NnM}EomR+qlw41@I5DW9ait2;Cy+%Elw(% z!Iz)1Uu4#<+v*GANQB8!d(^b3Sv$3>hFf%WwmVBPG+UFypG-l4irPesrfm7phmRh`3-0LrsXiK*JJ-wzn4lQ&NHanv1iNl6xjUH^(=- zuJCDu4`D;!{HYq9aM41nvPy-m>wpd<5g=5vs{t#(1K=<$Wo)<1%pR&DH+a*?{(jS;6mR9-Zeyqi z+Xy5ooL4#pOhN}{wpnX>s1U?N3+7khx(`O+(?qdDFp*2+IT5mLZ8V|9mewL z#`|{tTiLj6Mv#GS* z^HI^D$d9nMTL?(syE;H-@;2Y@^D`4XG*`B=iH6wQGW+kH0+HLR8p$Ie48r(mrD2uAtlQsd~PDwGCNu|(ypeG`TV}i zhg8n&Hc~gO4kqZRP+2K5E)U<(M@I5ZWaS=`yrliJz3Cmb%lXep6M%~+oPv|&SKQEp zyz>nrprL4 zdZvk?5+z*PMf(EJHm-Sz5dVUdru@HmN`cdEssvPnI{N>pTBT1^T1rrgjE#xAxLcR0 znmV+cRDNseVArg^q=Qrk5@$f zpHO=;{%Fz3S5C;3=t>(L&Af}g^6$KIya~~y#-jp#I@HY3NMQGf(G#EE1SCv957R?; z?QBK6qnfD=_vZ| zem-0cVg3ZjCKVIkZ=w#Oz31t^K2trffjXAGLZPLX!L49q;M3b+Y*vsvLTlpF^NKRP z{g*tB&$T49OdFFHu^f_&Ze^M-jJx*QX7LXtOEjh3*mFb7t4KSA>`e)zh+o{^KU*2| z1ojIl72=Z4NMd#!B@&NUHosV#nD=cvgx>`^Cee7Mhb`V5o^1u;gg`IQei`Nj@MiyG^QaTa`|u z=^f2?Vn4>DE|u$7B2OmV;}=5vUw=_+K`o^2tN=(4xzLh0Ev9UG7`ylPik_ejk zH&V~;#RjK?jl|)SS+Z@8P9%4jx$p-Wn0QWH|D_sponK;ah+3vR0+39+PohktVg}Yd z=J4DGol8*FXyxPY|8D+{iiPA|&|xJr&ZQ!Dt(gT%snldugQ#1bTZV?VWLprY&v<7~ zG(HI*+S3Da#NI^$a7#Z~6pg3$JdxMG?KX3!h{nd|XN4HHfwx3ac;8f?bWR_Op5>yJ zja1Avf)4M%^bd|OlH%8EP_Cmtr{frkqfKd#W3s(Z5>3kj4@Km;20hxQC*bp0kz!8R z?ly-Rd=FB^h6vz!HrRkc=(DOz0#1cJWfHTf8g9X>v;F z9hwArHv?f36~^}r1u6jRLpA&Ax@GF*{YLH?{I{Wh*PZifBuRCWM}C=x6UM@K+PHA3 z80sxrY6*jYy(Da5YP2tav?c}{d=mVmG$E62$B$e-Alak~ z|1`^2{df*;P`n)ym`Hcgrc~xu*Pd_9jz1pce&+lDD~I zQgENceUxX;ghQ;bqQcl%h%K&CeE=SxFx!*y%wiVrc2(dn1vTljF?ABfR$ad-QBYjk z*6S~qRm|jp>V3nzYa3*Fwel`cn}-8IvsLX>iGO?$Hj^?M-UL!=X-}mIn#`Hpg^aDf z@YDBmYxd_Gm_~NSEE^Jc8`pokQgE=QV!e`=coZG&5;jV1%%JzgqcjDTma)QlP>;L0yKiP82%NI-{#KyZZQb=A<#Uv18NBP#?npCYTDO6m zPQX7Dfe?=*9#c|ndnpd3-|K?BgVrs~ zyxQ|^O7EDy5GJccr^_Knn5#Rl@)8X(ggcu5JhW)17UY%~rB zo3NwOImgCx2Mqk^#&~j1^oJL-&7?}6`cHCwG`DM@Dm&tg=ezG?Ka_IVg?&Q z(aCwWY^2jJlP}DIU5kweXy9JwoI6MNO2c@um1%b_cw}yj$L@E-P`!>d;Tuu_({qyg zDK!sX5GPgq!i7#to?T%fDXO74;ZNhGFDs#pAgqiTwX}GCp9X8zb`da^!F#o`1U7W@7l6Ve67JS#Luqcx45dJ!jn>Cm>=20 zkcYp6>^hyX!nGIY=Nkt-8nv`1g_N65#p%vZ`}DjB`wIcoJJJE)Hk#4UQJ~k_rEhtv z;hsU2JMG?ZCA#DTcuV`BE$59qHy2Kma4o~9ZS#XyWpU-wy7%&y2Iih)*;~=wx04mR zzevHz;EDMCtBX_Ztt>AwR-h9LeF@pkbYktYe*IaUk-UM4MS`z$}*T}00H79 zJq{fT+{Bv^{_3yR-qI6EEO*+Y8m#)32`69Hwr3CL9qfxlbpU&=6X(U5be69cf+Hh-tEM&Gx%6*nM~|)p4Y1w1QcCjw{#X=?K9ynho}f(lv@LZzaX>a6OwF{Izay8Y7!xhG63^eH6P?%rY^f}w~Ec+43=(G0EMH@Y{Y{$y-+Ko9P zvApehUA{7uBcrWQwnaV23!jfQg||DeYk;IerW%KbbwR-GRsRq=wyI|g+>#&U^%Rbd zbvciY4h=3{(RRn@<;TYTo@bXRzCl)cY>EIIG1hz^a=BY=DqHRAbI_w@q459f|4MpE z9^?OJHj^_o)cKk9ToQ1~(+w&PTV-5VJnku*6kG~-qbiHHSkrZeN$_w9i@*H&Z9O^4 z+OKx}<$<+FTfv?p4Idk<;Uc#*f8dqwvbtU~wmdG@Zf%0IQ>$!wjw^extT&r$9+oM( z;-1YC>sFmCR4~(jky6up24mM!0%3hm3Kib_r5%q4cT=6?X&Hrmh1%Y#wD(j~v^k^v zO-1oW!|ZFjcyi}Oa}aY1&bMR;#Q8F92ap#{-oFH#=y{fou7r|ab2HjsUuDeF9Fho{kijJ&mswzHu z`Ds}^5WpjHe0^yDQYtc75c$RU<&>I!!f+C4K2GVn;PzNZfjm>1_nUE5DeV z{4AlQkT*KAbG0GOuV$xnYIn^QZ(71P+>2qZNg4TES@RT}DVNb<-%%i3*c|}~MFMhp zZvOMXdOe_XPQOY~k_?q9-rhAIQgMl-#ILO-i+y@>{frBOGF*F^Sp2@p;+wBb#qp(Q zO1i-d#uu0Xs;teU)NlQ?Xl2El z3jALTLj_G_hq6~JtbS>z{SlN|y%i>fh_y_<}G4>{Ku@tqY`NwVM3vp3~2tu+H<{OayM;_vIglC z6EOI}3}99)kum~Ty<4XCI{R8wdV$7#NQSne)6e6|e4QW$CuDI&U!w8Ss>lA!)22l) zy`4M|w*iLWE@@!m0}e6p#b;w9JY#{GmEQNSD2uI{%je-X&kM#F0v{QQXhVoFl&zUYXGA)i>NO^>K4{tcx+d)pay7;zINBc3x{+8eWD!b0EcY>DsA$QTRe_IECX@Oxi@&_xE0Cxntxz5&PArW@wzEk;GBri; z_yHFuj-M`UHd9IMx|g>$rvC~53m|4+1dtmz?jy&hge2^KMM-p(dH+~>Zx9v`>aq zY|MLre>zOs3&I)hWJ|acY*YJ=A^z)GPM1w1w3<7lxU(mn;8g%JU1zeL-rE*3IH0p z{Y7pZRAmM-+7#Baudd~;`q{bM#~tbyuu;tF6RmiCt2sj7tox;RHN9_^=N|aZ$b*FOSN{chxi;Gdt1<>sKW+o_;Y##yq*VKDSeudP9mx<#(nJeD=ZAy5H zU5e#KuzUzZGL^~E|48xgYEbauLo2m>p%-)03j#>ctX2~VHFD{EG!4?-`~YD9m| zvJbG)0Y#VYFSzJNF|)zPJ_kaRzCI0Vx%We$jNjPLT*OZR4iWrm(?OHCbPS1XL^he< zqdoHtO$&Ng`1DobcaFUFi<_70Zt{(0Rhi@8!ylM3=HcYJylnabd|K-AGY$zJLqTlG zTUq$G%dF};3`s9{gwRA%c79|r@3!Ca(UHrFx3h<`cd^IWJ4*1i=cxy^>Oew*N?^Nz zE1!WweN1Nl;aO48A-(90VUbtpRLob2aFi1MPKl|j^V|34;%Y_L!mbH;WLN`=JWvV^ z72i(Jb5eCRY?-I2b5+A4wcZYbl@Y+Z#mGuCKqvMxMW0&aTUP!)Bo~?M+~PaGvXL{b zl}CSnW0$yQuAnmF=a?=-sdx{K1cWHqz<68!Wfu3Z>Jmpy#rb$+2DXWvPwyap8R*8Zpq!u+0SEk4OtK2VreLYO)h`5FAWbz>DMTxQQs z9#jQT_XN9AtQ4LTN?8t1eQEP5e&!JROz;9LI1xL#@8OBFP zi=kUYbV3664ut6>iIzzRK--rBYu@&1#fV?PGZ49 zB;Fm*4|yJ+d(Um%S9e_SZU1Cje^WCKH8wRlPNf#Du8vEze?fG)y*Xo(`I>HQnkhF@ zq{*QzKLMEG*{*;hGq|xo$zHp=oi$@8a?&AG5eV&eA`3hi!hHNWUP$_xd1WBsMDo+q zfOyOu7)a3VXBR+{##9Fs?C}Sow$l8@+zw_ombB-x zznR1fZu#%Jq@T}3nM8*L6FZ6Jc(S;7wJBO}&HX6tH0>z(=&J9DhM`GYB3lFpgp7CV zx(Zkp8=uZ}AZId*q@<)Zy8`F;%$`PA1UZ43_Eq+(EhY@KGbLoHiEL)}@8OxePV3Ar zqvISIe4*kREUHJ2z;Wcy0@R372-(;q$x9^@=QrE%uzO6_#HTpnGtfsopU}+^b-p`e zaC-YJB~UzrJ?+75sV@F&jCuU*)(`Oy5XD@0<4(Gyt@GFlSWadQ6Zv$J`;`J7ivgmSD1%B>C$>(0m=+kd>Cf(#QIR|)6FTcIS>`IJ^2h9RzvNnZM3nskS(>|D_RFGZ zU48Sk=(Z3yW>kfqY#e94L|hLQi#d>aj(3G+MD(*tU-AJxf?qwA3q5CUu9JE{4q_R| zrUQ<8i}bsr_C@@Q*q6rF1ItQ;()jb~G#T<|A|O>y{Yz%9fQ6$pELUBOuR6c;b15)9 z`zfsfNWe8U-pe63;syqn2rXvjuE@&M+8bVDxirmp=!XcGkMpX|ya}rRGBkFW*SycV zPkV|qT@orwSAUL|@S>7>zH`nJ?-t3UJE<^8){D57fp?`WktUCa<_TM!?VKq@!7gg9pY$`Qy+c&c z`->X4P@Zp>nVJmUjwn*@mE8=CD59yP;v&%(;QX!s%2)|*6SHDnD3`7^r=?ZDX6lK~ z(+DHhLkpJu4N}bcx;5`f-p{~D}&?dLlbhr0V%jtRw2CyL>=1Wg&wn> z#3r1N-cUoq6_B9b!Q8-e+HAtTcCPEgKiDe1L;@UwqhzIDwpinVmTb90;pLf--#IP} zRzwojZpcb*l$UvZWqo34k9xx$@PD)eZJ$a0ipTi#m058WYdr>Q>;;YA+tp3;%M=j7 zcyOZ9Rj}wnYt_(>ki7N=l^k(}~uOB%0Qhc0}Ex)tyXQE2f~#YsMwfSfwu#9&+i;Hoy^y!%6FayV5YV{QH6IVFpR} zpT?V*sx$~OyOXjRT5s>6fAggqAi-TjSK)0nRWtwFmAnMcw zs;*q*KV2y=|LRZu9{_W-osLdk$(L2!;NqwU7})xt`gbHn<& zaT;JKKdlX ze=PTv>Lwl|b%~uiV?%&f#5NAtyC!$HmmXGCXQ%otz1E1h)1}fhbwUJ2KCgA20)rKt ztG?s?ke>`(HS%wmFq)95(SkOO%s`Gb^?i_yN0A@e?`R7ctyVsI_}+`gn8dBEjs8Z; zaYy0E|B#~$Za{2(lz8)StKBIIUOjpaDE+(OPb^D5$NkHfX%xP{13jq>IJtmyhztOe z_#>*ne-rztM#+F-KJ}}Fh&DYF>1WEPsNO<}_Ky#vv%_U+vafxK0sY4|l%ZGI4}EKxEX9#21>md>_3*8310 zSu7TUpGVK@P%GeKAuL?XoziC`wl0Dd+hW;F-|y3e3#m;1Sn{M@_9;$c0twjT^4gv+ zpW77l*!2ytUR>1gUtD?V9OxW$hRXnf3ewbmxrw3PW5`dy2DM3a##c6KVz9mC^SlRI6tkO=)T6X*TWcMGD-jIV%10)N!IpCjIhBu`%^Y zh{4M}_P`(BQLYrJG?xxi^e|6ZmF3FH9Ds1JhqBCdJF)RX5s|80PssxrHds3p`=+z%ABDHD5O5s?U3^A{}c~Co#NvLo755$d7v(OR_$GE zvNmb^DCTxF9`z)%-TgPWl`8|6 z3MjsOeNDjlt5nP7%b54SfS2{3WQxbcZ2lyA*}1#AD6%j0>EX=L`P6~N`mKPc=;!2^ z+{_nLm+N)MF2yyg@JpzeVbU~9?C_%ufFZ2=gDEBb%SnAXT)tR;Jx>y2e%;jARHuN@ zff2|rW}Vk`8L6|Equ{xgRzsU=8mM$d)R(0+zOx}SWo|-R@u*ZmdhQ*x2y5BH2wJzV zWaP|`e&wnk+aH9?4kl#~cID2e?KqsJR9m4KLAfIG{<;lTEJ|a3b-tN;dJ5w$U)m8_ z_}|$2Rr)6P#9?b>e4Of#{CgSbqKzHd`)H10VTbeS&)KSwdfuWu_| z`DY>uoi%>XW9B`kmAho?^TJBDS^Q%te}#9ek_gF-!YI(@r|`-<{ED*OploxiG2sIO zjViqh(2H4AsMwdCZkqoKirF(;(}Kg?s7e)9>usTP_YI?ay2DtipiDn8Cfe6+R(vEP zGY$?gG->~l;bDbS#|&x{*xLn;>W2!9)&}elChg|6 zKzZIIIW|Q4pIcYCy{$mnOxycOhps|lO-md1N8REi0q{Ie!iX;is592}SM`pRt|7R= z?G2(!S3F7{pRc9=EBgD-h{vQz+w{PX*Uj*a-t|%LM$5FRPT1p=Sox8fJt(>ojHUB< z9D*YbA2Wa$P)@i`@H{G=5a!uSE1@Ef{+t1Bi6`j7e)0>O?ROjxlpW;CmR6Wtf-Z2$ z1+CqJF1W2tD${&xPC7fK*Vk(VQ8L}<(j_R4AYV6D>I-6teVv>0+97?63~bl8+`CMD zVtL%4q_9vwdvB;4FQJm=7jm1@92JF*IRNk`U=VN_<6T*c;q!u}{BQ5~hBmhi>AOL? z(Ja1LE05cA{kLGDQQHf(r2ewK2eLkA8sXRabHAGwoT~z|r6$dQ(}2ZL>QEJFJ0#vEop_qDad-Su1zZ3ZQYf)>@1?_WR8i>9sY^M5RXd3fWfchhCb zEYRpQ0o+7HgH_N;1CN=P<*$!Kq^ubhVAV4r7UoC$Ef!~V^n975g+1r>wE%+Xrn5rY zze|nh@cF(qN5hk%l~%-nV}Akr0F79^HNepIn$G%kuV;+kl|&fDv9)=^);WxY z@qb_`#HWg-$zo8iZ_?`%?|t$+f4N*pVy2~DkJG0=&F!u59w{gvj%qR1<7Pf`90s>L z5yB90yRC4=9D0F78E%+Gs*tYkV@^&>l?73I;;&-hl8dvQ)6<%K)-|Ead3?>tXC`3U zSn=TSuy2lOip$-Yyu`l6?TeZ2_{t1CVv$X1YFjEiRc06@nKeGtj((T$GdzAbQjjln zoK4sXP!f^fbRn-y>@8S_#}URy^^cl*^hF7@XYi!z_nF44s|h%=MU_c-B;c$0^H(?3 zd%L}H+q=o5PE6x0ouB^cY(kEV@Q_3m$N8C);?^V$K z=PMi>F|`CFdlO)6|D~1(>+0TY{4rj)?sIvIvvx7%Op}Eda>JWqR#xLN$@XWwGt5O1w;4jQPOm#E$@YY_h=7LP5 z*I}>bttgEr#?r=CR~0I2{NQS9QiVfJtyf=VruQx3Dh>*2EpxVCuWq7-{AjyfMMXhn z`EUVzV09LrEz7olb!Y3a(h`!q z!_S;i;16xJ81JsFf577C?QTJv(Y@pPt!kkSMfH5_qN`Au7tF`-o<50cpTN0!>Kh|F z>fB6z0%exozXa^FzbeT^H;TaQqa=WVIv_t&+wicT#=yV=Zdby!$DVprq!wkYaX@wY5V&{J7=T%=YH27xBS_+!v{8*#?d{+`C^3Z z55eiQ{zI7brJ)^k!AywDf9eMC-x8A3AuGRNTD9yx*o&IFU42HIx{$HdY?&|7z}0b2P0?TV9cdVZebMx1c)(ZO0H+inUe zt1Je4yVg8Qu{@r<)#T9EwRLi8l_`3JVDHl1eY;*Fx;dlxLe4Zk;a62uG4XKH1Vh6JBYKsZ&wG zn5WtjkD-30-X-Bg85p>!j5moW3<84k?E9eQ_CM_xZgt}xEzsX#dX7_=5Pp~W&I#OZ=n#;NDpOoPjX^@ zX|VNol58Y&UgkfquIdG#o8QK3S*$pvoZJx4kc4k249OjxNR$TDH8_t$rJ(~qvThccYD_^s; z6{ne(2L{^HB*9rSy<#|zn>6f)%%T+ zYR0b;!}6n>x_cczUks;9KpFRPf%78jvBlZo`sK{M31W~?m$%AMd2e^FM6;0`tD+fs zL1N5C;_HCL;pAf?=|rQlqMx78&wf`Brxa2sgD^FV=e7|Q&UATF&|gh=`HK`q8$+BN zMXGhqb$TD}DTFl{ny~jMX+oIqkH5Sq9`&nvtF!q4@=h&=KXoLH^M#Fg4W>vPl-;CZ zukTA!#Jr0c0e;9x_#Eq3PMH@U8T%XdT&>?vZ|R&yeAxjnnni4;p&Y{G2$_57zqm3yAlmM z0|nuC0qm>tXO={}g-P`7KEair2Yd#xww`mO=H{43OZ=~R%#DN7(*E+ztWevJ%L&Iq zz(A-pE%7t=ccmr+j_zW}`yLovT%L>Rl1j6~)Yuy(usAV}R0epQ@z(W)p6i!EWPr(^ znIpWaz45}Pv#Q+=|D4Q_8S*_?jI5kR8X5jocT7D!v~!VzZ@tVSsg^;Pg&a^Y$GC)d zJ;rd{|2tOF|64ox#osd1xxLZ~Q{{V;cmXNm?TAMqSBthHWEvAYCAIu`T9waw3bv=) z6OURrOqCam&l`rIC7+^w1}s`e73SsJKkwpL3 zXU@`+FmyYkfaQ(r_8}!RJnAa6GViDn@1s3xxJuQK)e}6*gwZg*0w0%R2%-Tf2ICj{ zru>mEB<}ZC`Y|0~Vfatj@5rL{MY+oT_jiOo#tvxa@YBJ`2?09{ZG#odq}(tDRDxww zpvRM=w=LSzwLl8v;LF!*)u9_V3w|;wt18-Wepq;E)-k=?sA$!?-)6}%(r+o$Kzsd zq5C_f`x_&z48dx1`D&F9+Q%f9`}qj>XhYuyCPX8@mh?IY90L9TFXa+w^F{O; zTlGzC#Z@&Y1w6}jz~LyKmpwjk;6O>vSlB({;R{MEa3bDdE?G^JXm2je2-Sw%PCZKJ zJg)&duaS8TsGB(0ACD^(2o#rG{(~i_0zzhm0{tTdEKZO}cfZ>;favk?^C?*)rO+n~ z0`t*-U5K19%Ft_X<+Dnf2W_u@-+I)wH9<+6bUPYNQ4ZNSS_X5?M1E-3*1t5%>nLeO zzJ^-({I!Pv=huMLuNHW}6hkv44KHI<+mc^wSW)h{6cy70C>;iuoo3Ct5*tQ%q$P<&p~q5kO|NUb)q4OHogj8&h=eSevaIQ=EWcNYsjI6} zkl6f3AdBSQO8JH@GOkn*<$mV7#v~yP{G=cMbM(t3=fSV~y%_xDTR%B|LSsq131)k;KC|7w%-lrgK3QNfBiw-%3VY_`r5Lf-+*GF$AUZ$HPebty~%*t&_ zv|sAzvdqm}+W@l=(YV71Boy75_ZJ28|$-ybiQ3-TNX2rmJrU9-}36QTQDk?2hK3~wZ zj!RwQvkM0_R7)*|N6ve9`$j&omDd`t6eegHn%*i9%fs^0Gl7iWs=;yJhe3XOPFvEa zwLv?Xr#ZEpMqvwI_=U^dujw>+E2xn{-Q5w-cHyX#)`s4%!1;~L+&7|;UGkJH(iH40J zo##dUOpoKI+c}dl#McN_{Xl?i{&ex4Q~6s$r~^ug+5hkYK}J9;T`2{LUhxQGAbP^N zzz@C9Qy z%&=NtkNm>k-xH6@aKS3it#U#dL#)=m+hG=`nm^cMXpLuTwfL)bLwnAA^7VPt`v|^eVjP@0pune=WD`Ob2X;S6#iE ziie^dcRbKnRKi($$f%^@bvtyy<=$!_^5Q=SGn>fajWk@UOt14REI>+e1xHtXSZo`; zvfG!Y&=zRVE1A`Qu0Ius4iK;*^^lcwa91wg*hupVr=;;9RbTStR#2BsspGi;M0m0=1zf2DOCR$-|=%?$69$&h?A}zHc7yn> zu~3&qa%yGg{Lr`thz`A&!A>PlJ^Y$IqyZIAmjW<;{rqKZ$;p-M8&AZvD$@#;c6+3F z7W`bIh<+$D=3@V9`o&h+D!3! z)F8g*jedX|sjYU^1ZNqF?JCa!!_fG|IwLqM7u!wcrmgJLS`=;L`jKE^u{O=?coTVe zyYKWTGabd8ZMq5;>=d%I%KuBncln^bdv~bpTFM&($LGj{a%{op{sWoL1AQ)=eg0_c zi^I^fGz=xWT<8YIQDr5UUSZDRW|r9OhLJ(|`|qYNgt8>U>1xZl`1aU&e`4Lp3r5BT z{xDM=qpS#)MHKv$Q_ljBmfUZ1h04hjlM&5sIrPN6TR;i>G$q$fj`}?Thju=YYAe ze(HwnIYj|eu)W>4x>03&A_-PCW6HGhS=?J8D|g#XyLSB;+LkZoIjp{G#TtUgz!zk_ zKLkFffCNJTXnrnLY3}F)3pDk+kR!3szc9X#&kM`aq)+Rda=i~WF@YO+D;yiSQ6GTY z^LwC)i{7WV@fQQxL#zw9p4I+#D38ux+u0>j#{7;@CU9_JcgyKd5HVC#(`0o8SNmsf zjF`6aP4MGjRUpB0@oDeO=v8RjzEMqbj(wpK94L&}ffS-wfen{{N-ii?u(RF12}1wP zck&oDb$mRREmlgM*U2wY2@xYOet_`?m9zStLt$C4Ikiq>FHpAc7(W)P&T)0&S;-8v z>Y*p*7J+J5=1z;&t(w(UX>5-gaeIqJ@@9A{A3YP43!QEGC=TEvEm+QDR&zZcJvjUM?nX{qZRnr)Ll2_V2yf_nCi$>wfsP~3W18G@z&x@m@>&)MA zwO|3-y|~dzOG~u3QVy9m{c6g(GJb}fVi#MZ-ON+t4e|jBTI9c^ElId|^R{=RgH4O)%rlc)L>AP-x1q|CnM(-JuP+sDAwoRBZF44f>*NFD(R69mU_Yj1U9z0#GYn+(&inWyZX{I9S;8p%g{6`ygo zxJNm~z=E%-YJ6l21>$|`#GnvsycDHnm1@YPfI_tac~ig!HZ<5Sc}_HhzaX~H+&$jE zr~5nzGC3Qp^+cpYNrdoKKOkNe5UTu_7*&!{{DMQAiK;kf!cG#(;jbG(ukXB``;E{BFAEa7S7h zYOH10nKk;m*OkIeumy({Q-|_`cmY*dck{syGj%z?1|>N-D(5i>O_Vn{Od?T1L@cQV z=9aEKJ!q49O?6eZ((S0McZeNG2ho@OTPSfDP_$E+>*fFnI7Er^OGKf3>5QPKtVfT?XIRz{)e6jvUx1aZA3hIskf@}wT!()B+u5?KvNUlMM816gW$vR zxE>Ti-xElGb8Y1+AV{>smBKri!f@a(TNrCz{y2My5PjLX(Ox+rZwhX2XJ|%@7Xr_C zjky~vU+%_ypHb;AelGMG&idcx`Kg!PP7y#8*H%-ADqq{KsxZ+DT!F$q70!cU5v(2H zhQym~EJ{ngv!$g-bCux5%Ko;9wg`AWqU%=DteX|^j`2NqHcmcsxpq!k-9tkM(D*e# zDx&1|!WXHCCIRwx=nH{9rltDlnhg5SyVP@yMg*x!=yVV7@l0{m6;1CI_zi)!~e!=ZKw|UBOj+t5?w$ePMdHB4T94T9v$2DtGJUABe{>EJ!s}O$*bV6j@PscLQ>2QLEs=+?8Ihb1%7k?Skzi1k#(hSSZ6FtjOJFu4-qi-synu+S+dJps?!#83?ojWYGiB zYJ(;ODx3$MqizW>IWyQYQx$a1IQ{f@SN(vAtz!t&2=+jGG*hbG!jL&DwoB_Mw5YMw z?C6_(L{fvnIVc6G!uzgRb!pz0eY3*rU$FJcJ6+H!8Z3a;3LHd zB#+7tl7Ybny1|AL2h^MYB~(rEviWXvoc+3nq{Y2QES@|Z8FHsDi<{&7+ST2ES)+za z+j{mWwMS6wwly{YCOEOBrmpChlvd;3$585lvTKKm3FFfPgstVdlI!KaC|YYg|E5{;nIiJ0NtB!;Q**h>rvfUz`RjG zXckP-5i~Yc?h3{CEI5krvq4p3F1%5gTLzp$m@BdA4L=Rl|84!UwC~s^gl*S^6Y{NN z#(jc`7D8JE^j@g!%l54bL7W-Jb7uvlc%z00IcjqF^~JFt%l`=eoGIkcHcUNlFSQlp z-!4NpIML%h(U-vr|FxJ?c}o){l5W7T^U49+INj$2Zv^(#(oS4O1zNjLyQi{aYYd)6 zPNWeUwgds^(T|q<6E;+2o+#AF>4Fu}1U|3T&yOc7yW0w66(k_ZgGJTC!4Ol#g4>%S zy7DdD{R?EJJ-P|YXD_Nxho`5Hznlvu>5Bo|1TG2=x!b!I{+e+*^^os)j25YFe-k|*1LBi;9u$5w-w(1KDL7!f=YCY`g_h<*4yFogb9_GccBO)*e6mj4pOVHz)|jo|dr26FLtGPpBQucqt&ADJ0Pd9=haUkM z@YtS1&f}XQZ5gdw#$apgsa40nV4IGp457%G3a{1SxZNIq_Wkg$6h_D0-6-N1uuC-^ zw@d4;hOkf~)|(~*fF*f;GT}CmhevNLF5m0(0vzXAL31N*Fptz~b97BE(3x8@H1nZ2 zEx(`>YN}i|Elq$r^iy2LwE^z1jkREZ~o-EQ@8b$+O5Qb7mIjf^S#JG1BNg5 zeKJ>!U?=s~jXpo0?Cw|fOUx-)!xwU-WjOpX~?!o7GG?)LWS^KW9`Ut3xaHm*#duOK?I;cDDdsr~j$0ii#f{SU4FF z$ZG;B%+PO*{~ghPSFVkQL;mx!En^$v_gs8q`<>0nyXMNe!|jUs0;g|AwXxSww$4K- z=7ERHK>oJ8xp%*cohoL1qE+=6o=x$=w zN!4EXSYVP}4>J5(svSyjTPO!a$LMZMT2&B)=;1k5wf@8+w2D=m^yAulH6dY-O+vKcU&h zq4hu_s`XfzY_N)Fvo&Dkmmw<>TCflEo<@#7kv-`t6?uCn=KJq0#qQlfi&nPhM5%UO z0gv@_M;){>sb>xrz|gg(lgLsI;k@=XfpaFKkE)C{oGcNV2cm6w2RyA`de`j}x?Jiu zcAX9Js3f$fvFE@qfe=@i8jyno^dtZLf1UqV?dpwMg@6~A@-F2;V1?xd&H*2UuB6Mk zf->1sVdlYwGDqfRB__^e{Lx`%X1~mgjZVbijx4NX)viZNX=V4A@L0NwXZx=;9>W<` zJ~|I|F|k44(T)r!Wgo6Gl?efp{~a>>!2|T1&p40EI@*bIz7b-jU5ac1353v)4Wl1^ zF>1f>v8|{KC_mdJo&>5(MpcXUd{0@lj5-qTShb9fVe{?`V>-!iszHQt5*T?YY?*&V&kK)sgy$KR(da)^_6v3br|zYOIOL>N-?I0#%zf4dj|(xp;szGWO0)QGAUVT$<(H0hUV zAF@y1mpLiLN_N(gKu@DkXSKBtuYrn3OVhjmqfec-`)C4#^gwdLKX=ox{V~$%|CaE< zbn5jz~ znslo$Otg5V$0GUzrnW#%TrK1$F2T}=pfcBe@8;WB!#Ve<=u7IH5bX9{KyM!o=~I0L zD|K29fW!@c9C-={^iRsiH2BePn8aSaOHO|O^IqY{MmBNL#))%Amv!g=I~)eg$o<2-Vm&C_w%9>NCu zyK?RNdY`MJM=y{83(R=}83boxHdbW&BpF(!XrCay9L->9+hI8@!C}1~FVx$nktF7d zCA1RV#K?Vt2s2~JC2Y6__@nW&D>#fT3KlC+Gxe@3sgRHRT}q4W9Xx!JMZ3FhKu|+l zlN`9GHE(Gs>OZiwxfBGhKPk$8iSPL?sflAEYge>{?Wlc*#2e=V(R@vCrhK+;To)_z zIlf|P&a#}e1E`T1iE3XC`xIwHCjvR*G9Q^8uc;<`9R2C#n@A0f_p##Y{qwP|);Y2N$v%ySK8ddR zj$<6&DD!M8(@yZs)I!Er3lXFM;UCU0P9dPWCkF;vY0l)9fQW5wfi~mB%v`48iOHRS zlZ@$-ONgbTQ_5G3rwXM%JtlYQ@p9=!@17LUeYuky2;s=APy?ts!NvdDg`{B5R5Xk* z^(hP|Z`a!8F-%>WpN+_!W8xFmB^H+08kZoc`wy^P5bSFPfa?h~-77KXC8^K;)~fRm z>|H}n&xyWadb+mDJ`(2MRuVHbHo~R9RjMS=1+;PwjGV^gA8{w?SZnAck6ma-UkTiX zntW-|{0JxV()}@!qlqZC$_)v?f>sJg0+{pM^9j5C? zyE-tk%8dys+~2ASU2U3js*rb2WJ)!sdV34#l`t6Yb%0DR1|fyF`d4ARA6eo5*d9#5 ztzJj`wuS6Io=yd%x zkD`UWX}`~ zuHTpagk$F*T2!t-{=_J=hiQBrHCmDKKV$8!H+aPv_XxZ48FT>DS%bBJoAy8Z+-P%t zFb~wcYn8#gO2EEG59}AyV3udz8GxDth-8nW*Pu@4av^cs$v3k}dwh<(L9SoWXR0kF$CHxOzxM@+ZIpC zwsxh#rPAvQ{xQ;HK5>d?$lI4!0p|_j9L*=Z`A+eX9hVEekbsA@*Y*M$CZj`wS&Y^G z>Hekt1GJLNqT8Xj=8oEA6vV9rRpMVp_SM_j4_Asn*|1sB7C+?Eku|I(jRv-i)$YnpYccqpDliViu- zoiX~HgSvw}%BCL+O&ApD&BnaW6Pu>tGf8t4dHg#~nABh!MWiL1(q1ht9qcY0^s}5v zqwqQSBgEBTWz(xMdN`3b=wc3W!~sXs!k26jn@hwM+0#q-VI?v7AoU`25pkjm-?{7{ zHo`A`0=ro*9;0warf10qYs8%#;u`jR7qt>2hB}TQ>|@V)mri(>h}W0!{KVp%p!Zz} zf+z9M|NGxVT4FUzTL3Q0zGH~C*1gVcdn(4T(dBACj_}L3lmxUNZ5BqNLJO#dgoKD2 zwZE|{ln;{F0#sG`>7qX@nS3+#qpqhHC_Kx>|LY@+^c4TNp{r|J4a`3CDfn3|H9^m4 zOSqa3o1E1BVwVl}*JFP{E+1KEIC{d(t(lRvkQa80p|~_eTox00P*=r>4K;k>w8`aP z4E!XO(7SX+G1893XJhZeuFhF5@22vZoVOw_S=Rkd#o!kl7x=~cuhX|-Q?s&oTY?<` z>YUOO4zI4RWG`ok2WrF_*$W(Dk9c%g$enr6_LHze+$CPDAs7d`zLDNVJwIW~6z?cYxSm5m!~1Y>;>Og)(v_jC#C9qnZ=ciPtO9#wg8YRhenp&j z%U-;!8u{n*@7;vHgIMxMZ(h|n^UpOYhLS%E$OfFqLExPyMKTfe=mY=lzjKq2HBN_# zZA_?J4~}u3l8xlzl^Lv={sR7(T$dq+Qe#5=G{#P}YjVA0#!Tan-@`EL@c#ZofYsO| zOOQrg;t~^R_PYf!k<&I`QzXDKUnA#FUUIdx%L%(INKB;uY2^pB2KffV@HK{ZxV8 zrK1IumSfQ6sVedp>SCIR^$sA!5ms2FefGM<5aY%0W!dvi{q`-|IGj8sVw3;s%=Cg% zmSDVe-u`bbKK#r43?Y3d>mqX|a`pG)?snyUnEUX3h}KRswf=l|M)~CQs`}Ey+kE) zKYTt5%ImT>Khdb!!}%)0zx+S2QtfTn-^F%Geq%)XSOm*gT@$$T7k+A1B;hI!->)#* zb@oX?pgV_HEh{meN3uU3`nlC?7Gjw+Rw81ngO+@kGKceeHv$L?%fi~GU`^hmycIv{ zXV;{E~gNF&mmp9C3*$%a_^M0OhD*W%^&e zHYm=5Fs9@FpNCXz$BzM%id($5%>*l%n`5Dg7@f9xH@Q80!1D1?suI2zd-f-v-`;X# zV}k|%m00ifCCigMuVx5?x&Kfan9xnp0d~ukJ?KW9A_(U>m%t|5E?qAF+n?+oCs;zo zlFU~Pl6?zci-thG;-z9|VP6yUfPra?gl`JR^N*8mMtzNOWO=RZ7?&d%CE(IX4<&HQ zJ3xPzmA(HGdIjBYvvWV#O0Q*ctiR846a!&tsF8|OWKW*?{Satxe){;sUF0vsxg}!# zu<$E3DA3-$=cr97a5yXKy)YAzE%q+@h_JGB*@wk%RU4j#unrwo!gI&yWQ^|m>~1Mf zS9BbA_ASxF02OpVXxL2j3hLph{D`@wpl2=PQ}R29sZf7Db65#xyeXW4GVq+IygvKy zru)3xo>65ZGt!yOqfp5)Z)$g(73uNtrn%%t-Ot0G2Xe!+vA^tIK@u;wP1^5U0je5? zCDs086-Lc_?u*+v#M-&?n!-isL-`*^Z9=yHwESX*$!wX!L_MxvHce%9=K!e~mHZ&}34q#C zO2)ADNX|}7$#lYZT!zT=>5uU*gG%$v#%eh~uGXTCu}haf3;*@Oq;%i{P(&;-G{;`l zpt9|1Yh>?q>1;Dc4{&2&iK#h+DOW-g<{YsGP{dh?j}Z<~Cc^O&!7)yhIcRkowYim? zN;ot{Bnj+uy{I6bV-S0&^Ua(q=cS_{ zMn)WhfH-gNZ`T8&7hxcJaZ*>ZhlCBMNO*Qid@`_n^&!*qzNn=X3D8NME!KD^3l&1I z`-RBciCe@oUBrbezJ7^FCL6RlHu>*>>=Cc-63NhjByQ3e3h;(U zjqr_1Vh?d4r>xAwdVL5>>?B^1$>4LCrGs71{xn|wWFZO>@x*;vVh%|1vNMY7z1RgiRXSs}If_moCKG7h9@lJD+#2bt5&WTyg?7FI=Ao>}@;N zPkINeoY^v41=ynI=yQmxp{D0o?L2o47|%$9a+gk|k8&<1a9KMN5#8qC8y(ok$bzJ$ z^EsIl1>$}joJiatdwcq9Iq9;;^lC;8aWT%}QHBoO5L&wOVv+u&MFRB%5J;PT%4_o` zRhnnq7}fMaLI)GIjdTg-X5Nq^{~6_1r}Ioym3+v4sai6-SMQr_9R2-DeT;lQGn+?K zEnb`n)y2k*G{RQ74wA}{n@QwLqHNg!oeMJrNKW1P2g(JZ7;#3*7 zBW)cm7uE7>PGI)q2mZG&I{F}n#ubtgOD)YpKxb_=qbZq+<{b{%lSBl@mavDwE5Ofi zw_%k=9j98d=RQk@lEA^183G`o+E?UkKPb1U8v!=oU;s1fjaN3q-yt`470xKfw)*d> z{Aacm`|@)~__Z8waV4(8I+GlQpGx{vHzlV4jW>rUZykpC*+=kDJ7Ggx3Cq z3U_ut7+78XdU6qF;5>>8y({6&`$2kSL|=)}j-Zz=K{&Lp=?_8+S5_ph6j0IMw8s_D)^*YoQ@ zhNTbOOPTlOAX7RcGy>-NGRf~I=aNGg|B2p%d)wtd^HwykAFVWh)hLIrrrHS`AdT+9 zitW)7(o8O&?_3MV9ko?0>bcbO)f@2h@|l!RzU6TWu!2{t`0)(Z2s^r9Z7vtIl{k%o zs;=2h;023GF>NAZaTMaG$6W&YIF)p}1)YDx+p4-b#pDoinfT~Ux}LsLjaPrq~` zrC0nf2q1Tv4gC8#WytG4tM9awqrnyv(|{V0#Z>5XW^!|lo%;K$l>8zl@T>{LyX^+~ z0)6M(a?84tZ$5r0lLu520Z1kKkhP@Vz!nfvxL9A;O|+^Bd!LfJ(Oo4^&P+pT>10Cw zmCX!DC86(tENQ&wC{Pr|BQ3kLQ)Z#P>l(F~Jw8xIi>-hF?l|+ch%3C`v>lw?s%_dD zOp*Tgbv`FW_fLp4)1E{51^(pAUq{|=qj%f)?BzKvC!$upYu#9n3Q*YgwcD?t5iU*_L4kXi!6Hq8@^;FFbN5n`y! zg&;$k7R?h|>ALjJq@&(UR8xY{xn1B2gDd|XrQ6v~=5U~Wl06o~Z*o;+;JnuvipK8E z4;{V*dKLT^#)sC-6NdPCjHCfqcz>gEYAXtS0^VUV$*c@oU8A2XpEVR~Z14#W zHRt*hA?6Wf={ofiFezGk;Cd>3NxmZYLU&}#UyEE{J4ADX&Gwb`E0OI{)q6$J7bG9H z-@C_@8-L_)pYQOEQGdR%zgVkGI$`Cd&o_kh{J1;KQj$rmcT+S{X8%+uomE3EP%n zX{_2tenqbNmrpG&(n#_Dl4tx{z)2wPEcj$v&AfvU7){~HU!UXtq^r*A+vi_XdK1hZ z0+kQ$w2CGP?Yu$!*$=ti3v{GckytY7ze;=sCC>}4OVIf`EF>x6_08zyuyqrsFdS>c zas2jg#a4Rr650VMx3p~OpvtlX7TYmQy_z7BCS$c5$QLA`21T1GN>N(lxCsZJ;h}ls z@+l6FN6>Iz|AOKFt4^XdJfR~%YX=ORtYJO+sY&p zR8a8ky@m-rJ)fwin=*SbNFd zRipEFK4=~8sblSA@HNocUj?G?*!2lv)mE*)!E&u4tVk+U+T%YvvUaJ@I!o!h#}!^@ z4_ux0cZ88!b_c04)a+D2l;F9 z5if6(KB8f_Ou&FZPh8uh=D{!!>v!_I*Y!ocg95yZeU@`us~7`b&$)9*e^=H}QZ6i8 z^R3uURrX?gm0j#E__HQ&S>;WI%jALu_{AG&4>F_3b*`thwAFjp|C@RyuI0$(AF}sLq{7I(9BooxfLKG4%#^CPAsx1Ri~0e>ov+b_gQmLuvS1lXh z0z;2oP?e%BieGG=@`}a@RDTNr*GhUz>F}X(Etm^@VW*ijRFD1<{p!?qO@u(_?|sS` z5Bm!3Z2shps2);uPCXCH+U{k*Snfpi`c7TML06ZNU;8~#L%G^}RNcsRqj?7wGV)GN zZMzA|lQOZ%{bGhN|H%VshxA^nS_NU%@ja_K91Bdfl?nN)#@TGH!BoBVlp38KnX1Os z{R#GwolcQQLM!BYb9>`NoC7Tzl|W|-DfL_lT0AOX;aqyLi?3GubsB41JwDX+PVaAb zqdD(^b@_xHLsQw74tS=z^0Rnb^ArV5H@4;k56R+JFC#(0kh{@bB23+G4)qIKT0Uy3 zyIC2tu5+Z({h~Pqno68cVl7qG-udJAk7wR?ia~ESk*>0%1v{I;njy~Zv$?W|98@Y8D%hG@uZdNg4f+iNp8PT2+ z2^&=A#>*oauQ7ecCfHDb)Gq23m!h`9|3u3xaGLCAnKaJb-1En)E)B$XBfpdb2DXeO=Ss3KwGrFMOefX*e1h=_2UFjpL5m+V=T#N>RBaY0?Fv|QqLz9$H6+9MAPvn78Tj+vwk z+jMYNsBfuV#-^EyL!WIS=7Pa6a;82HCPx?VTOcUYP1}+BX~Tm<2UjoWNNBv~H8+`Y zg_yUY+8~!6KXvd65J?F5%cyKdD%?VIRwz%-px=C(|g zR2c>EuxFU0&EVCfX1=UE?gdExoHob!x%vwdFZkLmEhcWY#k?}eM?kG_4?To9maf=$4CKPXy=w$)fko+ccHwnm#_sRe+f9uTW>5aCrcd;=6bDpN6v);2zVf`p0~jqfY1pDS5+7-jIMVw4J-VeU8C+Oh@S)2l0z2!W(w9e<9o&TYQ#_G)-!My5&~g_Fd`X{5M&zU0U)uU z8bS?og7FqIkuyzm`$A?|N9q}l}mkDu-X5Uc!k@%)a;3r@P zK$A30wXxOn3HPd9i@CgpDCfA}G=j`q^d-954cMf;$QZt9N=qsbAKkOCuE8QlxnJlS zcmmaGQ;xM<%4>TD8~yk~jcf(``!@P%g}@Ot0w`sj6HueU!;%&K6h~kNW^>V|VTx@= zto=oUu}5ZteM%roI6bu`CT0s@R1P_m8sK;EngeiID$D!wn{H`OxuWi-ujeN73Co$kqLvRVRc_IMUZeqTUW7t`Q=rtG zAzUs}u2iv9uGE?##r${DB0>Mr?7emU)BjKx%AoJ5LF=-9my$~&EuKrpK0(AdJOBqT zUOt5Hc;yGQ?4AY?`&{7{iE#jQ*lZBAfN9(0M(r^&O}Af>JQyVI6WegzEN64xo(tad zm!|lPN50*c~-WjU=q0GN{n0U zi)GgkmX^eknjSyF2I5j;lHrdabR`({^3 zt?75wAc3BuyD97We$RqMMR*X}jIW{x0>LLLe=-ZBtut*y;#R{{4ww$8C;@%lG**j87MvCy4%aM{ zEH)(I`;`Qt~pwaP`9i=$Gh_~6SCHDu?9#oSb@P#$H7+7spY zNa}3!PkKBf`p;f<`F0g*-vXJLx!g6oy{T@lWwAPx_e1`-#YFf-YQc|h4>tCC3hl|A zqg6v=G54~h;8JKo(D%=ar+)NHDBC3h_VN=8%K}X}u*4&-j$)rZ0ayY4_mgdo{~?<2 zmo}D3=D3!)Df{@nz3yfXv7ZI8dJuORwz|D^l_JEU2H)vM5O=iH2z{o+VS{|R5)zP} zq5ehIlD|KeaJ@&EYM8vgqMYZvSGPmJ}~3 zfn1KJWb1~5kMpZoZs65>Z?QukSoK5AKP*e!HB#ccks6}hpPZoS(cc_zq{Q|ga6&7R zl0FK_31;5CsS!^8%7DhR_eDreR(ipW z5F7HBR%Wlj1rQCSY6veZFjS`5BAJU9re9RB6>x9qtFK9CHCOKq)8`^fba)e=_xUoD zO|t@sQw7wLi>U3!25}8>soD*FzB|c~J;Ocz3knCx#EHOjjAt^}eyj1O#hA$RRhT0~B zgF*YT{Dd)D{4Vk9WaaZaK=@$Tm32v5>>aVJSrAfRyHeV@Rk}COKXRyvY8KrI_Y>Uu z36f9tWd-rydv%RJd0oO3u3+rTNh7XKzx6#h)g=?9(5aCqO3rBFSERo5E^B7k*5|Q{ z-qSAf$H+#T@b}7K*kKY!mQxecY0Awk&-N;2(sFgE`wB;3$JNMd*(3B6#;?@r4)}#2 zMexA)9*)(kRQ?V!t(|FU)ZY3LjQ7Q(Qfqg!+k!>h3v-{61nH&z9I9kyL0)mRgeJCk zDuF8$s&AN0&|S}zy$v=hG4v`7U=41DB%8lW4FrGhx%ZWIpX&KqbJS}@rFtvLqOxkY zFe}KA4r=5}`-ZLa-N=cMnhXWV_xQvWN_8EiUOtI!!F6DA;9G#NP9>$vxe3}?Y(&(N zeE=W+W|YB9d|2y?Yx7-6gH?-v1afZABT8@=eRj}BM8~Bhi|^l+OB$K|WK-NpYNdf4 zkQEvH%ZkD;LgA8-x`>65DlxNz7dGuYe|x*%ivNAu4ZjoamtZ<}d48DkZfoq4FMZl1 zaIGs3e68?>EOnSBh1zuY-snbinIWWHV&cl8UcL09&>_!iajzUwkktNG_Ti82J-1)f zFRs1#m;h&>UA2mrZ)cAZ%yxgm?@aGqh$>56^4vrdzZ|)lKc`I6c9*@aQiCKdukvfN zOyx0CGz$fel}^;#QLc9%8-v3AQ>=G8tL}1?%c1T*xgJcWRR5(Qu^@}xzHbvcD8y!R zKbgdBJ?_UK;cy0JaxCY2PEE=RGN;LKPf(s2{9U@ZpMIhRRsf)xlTOyq!s~tOq?5^^UFW-4r$(6otJ13(DZ=JV zGr+7s>^CCt(=^h!ojgo7DM)2_=<0hMaUwhDQfz$^i0W05G?2PEb$*6yh6-j!erD%v z{VlQSZkI04QeYB#*(l8puN^NH$k#rD&PFk*csMr-&|P^<{`IZ-?AN_=G)sI`>ph$k zrhyicq^ZGwD|PL36TbbPW9OAS$0wM{yt|<>1CP^U?e*LOrBI%z!`yqZ2Ck2zGs|}0 z88;|@H-4q~j%UdQ-@u>mj=C9sWFef5WUge85VGXCxl%p@O_Y2x;jN8)-WCT zAEWsZ*#R5f2wWpa9P?!l;z$d1A&aoBHOU2lp0j`Olqyxo3Y0=ea(D8rCzn&@AN)zl zG*t_p9bzLuZ;=`LK(%|sXP9|ib26$~*SiwwPP z`Mj27=?dh(kz&61mgMEUT33COS4rdHs=*k##$43kUA$(Dgh5m0wP%z<_c0nAuORxb z2l@1uUF?($RY~URT|Wo24cDBoOFxg!*4t)w@tU<4{rMnOvuv<2ST#7lEY%7xEGb5% z8tvF1vE4%Z@-W`|I><#3IC*Ut$Bxs#c`t$DZNKSdA#vXjb-|q$&mKzx8n5*vu3o}{ z3d|Ez!ac-gD%MoS=k$`;)403XiWO1|Oz>Pj?)|h@F1~ z^4F)zWK^@fJb+Y!vhM6Q_wLDhQ*Cd!ZHuEV*|0f74^Ohkp?epp5?5Xl=ngqwvW4ag zq+pMlSgP&Nm{X~!R@Sos(rFKQ{j^3f+bf1*#k9KV)`fbL#g%V^mtRAhvaAFwH}&{! zP>%|ez_-XjX&}&>r=ROSD}?A&=}T+5Bz@JoH&^S@v&@FpPbzHZG;r}s3GWn4#jNgV zyH;fsn2%_V0H`cl^94m=@HHc)rv=woQ$ZE=&lR65A6%u^)+P_A6z*h%W5mVw&ypBq zPqFY7SK=WaK@=lmcQ&>BQdH4H9S5FEr%$lwx28`3nAVxtYD$a$Hh6~0b5q( zqi2Vegkn>Ih-pC2a~tAT1Y!qAn07@RhazzQ978r1b-tTJ6j?eO?$O-*sD@w8j|*Cg zyE@Cc>crv~h?m@*uj@^SHI)}zm1o7KKG!Q@@BAS?#sPl6BA~k zlv+7kfLKzC=B;Ehi3)I^9lSvwGRQlu+2jWXMwgd7{HoP3pCRpO=6}$k-?&FtM*6&> zgXHs*=rx-pCM+$BZ0q}yN#m$7lazH>^!@t{SbGrY zX!~rP8$Osr*vx4Ip!Ekw;`Nf9|Dbi(DZ}l9J{H8TtQzO*xcSP}E3GG#eSY!i3;K>GIhWu_6P=332W_6;GkddN3=KCQl8=2?JR=7tclv1&8);6?Q}lRKrQFze^YPjSd!-%1o+Z>X?x!+2TOO=Q9glI+179;Dij zlDmfArtoLHx2+Y?;sWX#ki6Y1_g_Fcqy3+^=;16x#@pW9R;yr^n3p=T*j5=hjK<6% z-mQGpOc5F(Q+!|3E~QnnXuRZyQc;;cV4^pk*f$3RDa8lY+I(z&NsYPWY@MV=NMS-0 z>$AK=j3rY9vyX(|7!=O}(2BEe`n)Aw)N2RL!iffHU}`bx3$gy}O`2>2EYsGpPM7%I zNAwnwf1Vlfmrg(jyEE&*{MiJWLnp`DPqrSXh`iHK=W2^VhXAf^R~0y-<#JrfLMa=Y znvKoEb}}gM!?3CJb@#Pz};W{*3(I?R1_YUwC(|bd-8c zc@2X&Mgp@7m6~RGxJ{ky>=j2FvJTxV-Ykgl)@n)u1sg>|(Q0N1*VzSPn=?Fz!oIj_ zIHB%_L2(!3+JfiLKGN9Jq_uZ++_L59Z2tB0rGVo>|!zrXsmR zr(W+6lmovP5>xN6tD_ulpIzmfcB)GztrVnc}tp+ye|h+ZmuN-&2C%#&}p<&zE@%L3(j(@D#7aKYGQu@bd@LH3J}Z z7!V8EXNcO&***GVughLY)nyicqqya3aYVVTZY&VPkm=JE96Xi18$$j#@^%;Gk*`eH za$)m_YxJ|+oP|k$V7EvLG(E;e;pAM2Qd$n*NmDgpfK@gFN}#8 zzdKLl#F#3O_`+PjI%&NJHs&d=wa?#(skyN=fs9!_NZ3HP?qj48y+_;?YpNa5?-GjS zGI?mji@VO;eD;mL`nW)7#-#2x%J+8F4~nD3BS<1WY%GT8ey?Y=Du=BW$Wj6Tg%1Oe zde1Rq+U`=1P-U=4;^*&TXlTdW(&UX?PLs9CSwqt9IH|)ZdT+2HO>QwJUAq+xBbDg# z=p8rm$P*1E>S>VkuNFphWeYiK&G#V|A*I%7DfXs=qP&H4mOQq8y){Y3J|Dvpl-_*0VSgXcM1{SfWA7&9 z3L5iG4x~88)<#eMeRUj~JHwK>M;oWGxw5#dhxiqJhi(02L+mlE++wt9$9-={;{^%n z4U9cq4QJb_>mN-I&~uG3HO&akNH^fZtDf!t2Q!KP+r?44HMOp>wT1P6X6TYFF=|LL zCQi@fq}gZ|DIid4ZMdzX%Cg&^82ePXc%c+z8y7*gNwRbViigD73*qf3 zbr=e=?&9k$hxbb1RS(zQ`Oa=s*jlPMjUO_ho*Sok6w}*TYu-uk_(6-_2=WhT?NpyPSZzEb?yU<14L;V&g*JI>) zW+9IPxhK2C4Y~Vjt%Vt;0^|#0R(5PRFpn5s$#FA%qneKM^0&PiI%)Q3G9D=Z!=f@O z*3MtMuH>Z$bU4ul7S8M~J3+oF*k_u*Uptum+v1yDBW3-9qyU_BgIhc8!$R2_-A;K$ z_74eLSY3Fh2ZVX=I0x{jxSO`&a%rff=N>^qaB|ickk*v*$2ja`^4M}6ds$nB@KIN$ zPmT+V)Pb$nlt{oY@>G zKqyU5zcf8s9vDklYbiWTzWlWiO?ka(--sS@(p;S5ztOKI)LT?kymWYut>S2})c_7> z5^y+oUn`~m+4TQ?`D2DYTMD;o@Y^}OJG`RT3bdRe^(tO=H@&FYOKq02yVf00V?tCc z7}Io_ryPCmU-=L8i0ZaH_=};C0SHuJINQn4g_b%feTkNNTo^d~IBwBr|3BF)S!_F& z5bCqw!vbQx)r|xA$AJs7mzW~!7#dlRzhOu8XFzZrW>Gr5%kKUNc^1NdVS&cUUj2v4 z%1FNr%arnV^71?XZ>77^e|a)h7DK!|l0CW=CWlJ3eb9U6iCWHz-QMsctk7N!AI{Vw z$H*QAo^_A`x>ZsqKZ;|So);u}NMlu90|KhGXO|0^Bo9~rYsEfz%|$DHI=gf}Oym>q zn=4hR1w7|1;Y|{|*})%*flj+aTFMIO_qqX!@dv}3b)eg?=nN~2d3CWlOSm2~6Wr^7 zhYMPn%`0{*9ycr~8L_`leofEg8bXgQiygfBaNtg}#3F0h0Bg*2$R2=ts%1&mOxZgR zbepI`$#vo?%UK+0rX%dr`ad@n1Re1>Tx6S;oSj@x?rKvoe%{&nhHnhY_>FcdPDD4h z@@A-+a*j*+^yP8$B)vh_2`7!9cOw2XTQHan8NZhiD8hsOPt){Bb?Gpc-w)R(bBxn~ zJZ!>bKaMbk1fBe-&Zm_0F^Rm-ca!aTz}Cdv;z0#J{$nLi@*sdIoOY#};P)RVJM6}| z2F=~e2RM+hzw?{pIm9s2t36zxb+tAeKqs$*WX`MOu7OZ{hNxX-)aqB6le50r;SC@w zqy^BNRCtdrd~zDmeHF{haO$hayeYLBNi?65m|IKxY~Ag_gId-m zFRzj3ZTp!X#Vn~%ubtawFZxU`zZ}qmA3#KdBGQ>`sZi9IAc%DgaKd|5*4b6rjZ#UfR!=EoMT!)bhwlgs$^aC5|Oi8nwyyL?vr5aL?<#E zvn{;}Ys31i^&YXIi(12g)e7V6KixK6X;dx(REi~skkMHZ%AYE$wA%XT@*;hBxP zw&46=lDP8cW98vs*yi~Yp23DO;d-Y4cT7co1nq(sHfX^+pl@$M_tNA-Gma)QzGb97 zml7{c)%~-E4isNhYM-^VYm@dA#HzUJQ0~0~B{yeqoj@gM4)D)*^gPIa-}(n`m*MRQ zFx*7_V3F*qpYbE3BQuz};?$5+*5pF@oOkcjic76O>y|II_GS~2fM!_|m#X4WvkbN$=un6_jWV+Q2t=z&Sjg)7ku{d3o zZYXO-^0Ojyy^W_sIu9)MGfqBtpZ3Fv$IvZy{Wb~1E*H=ZFwBY}WxEO9|GA}72Nm#* zBA+o4R-_Rw*@W|bm}g~ zWhkeAP~EP-4ZZE&ScUD7s2qyrjDe!s|Ir+0~- z0yFg~ntUV+S@{K}>T_R9-RrMW;aYEE4EvP~keM7}U2p-$h1z*~^B+4M|HnfT_0!Mx z=~~?Eyr3KwLVshL`(Z}{dciY(kQuU#vhiQK9<$KnXlS^s`F_0W0WP)clr_r9f{w~o z*WJc8@4xT$r>X(I{-%?sYTmjQlA4+|+Ri$rf1)uQl7sfy_f&k!jJ~!Os!~A4#AE#3 zo9>jOe(z*4+Km4A^3S70vRb6;<3gskSgyzyxVmLDs8^|N)8IS*NT7|7?AtFj?nE} z|LOV|l$%ukAD+HCD5}5hdqH9;>23sM$)!PhC8R__=~_CaTR>nzQgCT0g9QQUk`@q< zTq)@mknUXKIsEQ<-x-;4`~y#X&sCqeB;{oAL6Z$Z=(jh(^3j`PO1BWq`rKqE2SRlV z&%kT*XN-ZuRkWj#8Y$}s6F{7@JzISiX`h>!W$|S6_^#;j18f?m8|?Sg6wbpUNaY=j zPQ7%3?Owsde)Ii30>+JxaSHA2&gO#8@@|J&=lyO02yLTR=d~4(`+w`}ACHLbNE85s zP^79EZD9MNf6(eP%yG zAI=FNO~cUjZ9})BS^jMybkIA^1N(YiQz1Y|^Lmmbun>Pvud`i)&iC^m{Z@iV^ioEd z>y#d*L}o$C*}#H@9O5o7VDs9wy!_3&?6y0?<817Ag%&|RoF0+H*a{v5j5YX4qJ-{u zc3v-SDACzWaX+nC(&Qs%d?=@h;sk}hezo(Z#I;r3nb@>9BgT&EV6abn8e3PR$Hyr5 zcK7zuHLV&L40^ftV1X>-&vq5uH-8>nbTLbw{%Shg!R+JT)Qvi|oF=$6E@QfE~Q6?ghTQzv+`5go!S~~4FqX$+ACN<-)3` zLqUm9StY+3-J@LVfHe|)Vvk(X>Z{17dlB{qIwVn$bqzB&=emUU=bbX)5|^JmP#G^m z{g-DqPSTn)dCP_i92H$aM^6uMu;&e>{&Noe#QI z0{sBWfFUBk*0m)yRY}2PIse5^Z||TklHBRD>xs6FLE44KJT2$KYWL;Of4AeaHZFq% zWhYGLZ!TzC1JNbBZR;=3-1eJ+c5?y-%#k>`92;ldSR93z9rI-7AN8OnfA*4dfc0RI zmh#Fi60b9TfmFsTDKMWHm`Ob%QrRkW)G9)E&R;Il1|^0LeZ{EmxfBiue2>433B>k@!_>!{R$Qy z4bzw5hPzEO9pI5EgA3{$X^U@W@9B-VsBDIuN~Th1KzljTjzzdqXDUiZXzG-{)5xw~ zr(%IV%9?p=)@69QuH%{5c!%%5NMYH;Mb|#+kZuAV>5Spx#(1+IZAAF>oe7&Zg)W7=N20d;=;XoY4K2N(s4&$gIser*1f_`Eii$E z!`Q@V>PMZ=TJfC`Ee9g@@+dic(i06La}{X^`Rexf-2;^=q% ztiZ_CZza+VbvNyHx9;_{qTKn@;a8SYFp1Za_dQ#)^ChQ_mg)|Q5<|E5kI-_Lm*|Fz z_$45x2d~BGn0eZNGR(I@mZ|_>oB4a-DfBL87aG;g@39yuc;yGz}33JvQh`s)~_Y08gd6-GEeV zkrJ>QkrT+%{pe#no$~q*>K{BHV`<-x6miI5-ksD@>?#i5}^S1CD6Jh3YE)G#0MFW z?rTXD`w>8*x}Y(edn+_0^%nlaP;~I`A7y^v zd!80hD~CEHlXo3VDIJZuwAzAW^1znsjK|cc7?{VhuMh}tF%W6Dz zON{aeb<6;C!Zyh}!+5Hx@A)U{b}lBg_5!z>ms+<8TP%@1&3>Mf7R*)ISRf*@aM#f{ zT7;Qh(#*f}Gw&Nbw&~Elc;s90jWnWDsX4n!Db3!B$i(?^G#T)&-mqJK&e^2K(;gMu zkp+fflpj+VqByD`F@Q2yx+MSNiE18eCsE;ZqKLLni}C zR7two|4PNOM^rnX#cE>JS;<`X=c)*vWa;F}EsJ>Wy$c4JYKBUGOxdgr-!`Z&Tokv*r#WieCdAFU$xmnrZj)y1>7E2cvYqUBv_yA6L4We)NB5LcIeK_P27vp zEz35_Xy~p6bQhc63d7qu^u%3hf`@=2k zf8_gHVai;7%7~zPYo@_t2WK771hS9tM;Np;m#65)d!(%{UY_l0qN0qL{xY^8GVJF# zCqG~RqXk1IA_Qrz66fe_MGXJ&)p+(F$)b5(rNEr!f5eOx?l6Fx|+_O zwSE~b2w7QAiB3J9vl5aUMhM|7@NdE;mR*lHQDj>ZG<;4)-_3UxGE^mNebI2Ue{u3u z2wm4$Nsl0F8zFrB>%#rWxDG(||3`dVq3IshDEHxZIrXPng`}I*VqnnRFe-K&^5(!= zeO#&}So62=P4Nh_z^EO1#_`{#B*3)Ko_^g#_F5O7h7C$j9(rFpDjMPw;^e#1Khw@e zY^*fD7VyKck2_`eB36&tG;}C?^jqe%W z9iy~QKzRp9Vc-mT%o6CIrLSlTT!b?>1AL2@6RtR#Z@2IMP$k|z>#j>_G8TPI;-;{rH6cYBxu7cc>C|HF(7bR=$*qs(rAJaKT8t4#p+3)c9(f5p^)RZPpY!%E%P-t^Nvj zL}k759U+uzOkX2keQfn!>P%e;q-fBX#E*KR!PVlu<51Exwm$V=$=?;FPCQdpLbTd| zUpRY7o#y;VEwE6baaMkOu!`ThC~4hKM+v8#OWpFyU$9YA&VjATN*%(EFpwq(roBwu zI^Vtt0CJ;m2k`}$RMuWV^hLy_#68z1<)C`5Aut2ghC#S+tP$iLWb8Pp*t+WMsVPXJ z_DHg9(X=Cedq<9=tINZ1AR^bessq)@xjVZ^?pf8Mjo--&d#)$-yg_hh_GswmH2Ads z@ackBsDjdYEIuJu16ARr^Ug;!ot{bX0UZeSH}JRi;gzDRT1ugWu$GglJc`D2-!#AF z8z9UzWHh%Ho+<@liABRr-t9~_EmVEqpQPdx%KD|j9l$-UhAE|-}@4#DMqFz|w!N8wl-p7<*6)>wu z{)&7<#?ko`K`~mvKjX9AxW&Xn?MpjrmMglF;WPRKVX9M|DrJ^XZQZ$l40Tac&$gBB zWrH!nf%!eGm%#5%V>SBft};0w=vbWn=+j&~8;gObxtN`H(7=*1@M?+CXT_0+kE=0| zj(@z`Uie01>QRLl=jTke>fAl#&^Pg`kI4i{KD_<&bVxe2t9y18tMj8|{0H?=TRKvT zG!N-d+mXXhX(v46P*$!6U3tE2t=`|B^KG!OijmfL!hv-$Lk|}Uc8qz74EvImrY-2j zbZY0Kf&=#KQDesCeuBMdl1_MTVYZtqduh5NSOE5oq@L3Qg?JGKr{aCtDc1K+ueuZ$ zZM45V*E&-XU+j2J)kG(woJp5T{Ah3TsLZEqu}pcx7%8!mpPsM&0p*C~BGA#xbX&T4 z41Bb%ngAZln8$Wsr{3V{iN4G*)ON;g4Igb08ayQj%3dguo32f$+%PP#mCzV>3)+wK zqb)4|-prP&tC+(gLFzN){aJ~kD~(J4TAC?8ig_f z9Fl4?M=Bvwfbdmvy+qXM4~gzL09NBb1LJ0JW^BGt<6*O_R{Wd%<+EF!db=vuWY3)Z znQuKVC4o<@MxOjU->3)Rr(1Bz4H?4uHn}?}nwchkPc|0tapQymH*= zenw~Daeo;YdJafF77TKGd%8vflARfUT<9mI=b%BY^E z>sKPJiJcDAgIqtS-6NNd?6=%<$|!Kk>x%bd`|t!9k8eDew$6rbhcjyz0uRP`5xU^_ zC=G3?yrm6MY?!~h$Csp{ai`l~iMM|#^KNP!KjfX&p|9VwUf%MwbiiVP@{IYe-;#14 z%3LJ>h2h_Ph?ShYP3|k6K`~P)*K_1rX!!`Fi1R)yCM-sUc)@)QaQ|2y8|oOtBlT=< z^ZeX)Y2U-5Fl= zL3s{!^6&B0uyhGquf#|Q#Doe_?IUe`4zj-pvh^7K!(aM%72n}c2S%*BU#(_oS%D?%O=C_S9- zJy9;vjmy@U=wLFC^DeJLa&7o~UwsTa9we&f>>E|ICD$R~?$)GLlctrWX6u@QqCdx= z-e-)}2=qXt@Bv$?#y0O>gVbl?{X>=J_6>}xaAcQOg*x$(9i*s=r|@wx#V?977e$Nd z-8^9yb!o&`>Mp0|JUkqD@5kLJK4ORDOOtpSv0Wj|S^BjCZ(!SHxPt~)tL#?g9Ko7ojIyL@r zH14fY8p^R7sZ~+x`lk*H9ERwLr)`MZD7<FOAE2hH>u3$fOnUFEXb`abr>UoD%C}1GuiwbFa2nG|R|l&HGlNyHDKecCu-R3f`v*R^3HTp4B9z>V#| zy?LX9)FpPSL0GT~CkDoHsHqm##}o>u7Z@VbNjY(+h%nVbXxY*u5|VI?xwb0uUn*bKB>?^2IRVZw@P*%z=cIl*Hj6lvk~C#b zhK#>(qe@d-lM67A9kv9c-~D4jQGHuRh{=x*upN(leM z7p%Qi-}=05*IEO8EQa~bKWt^=x_qZNF^KB4ght~n>$~ju(f6v%CI<0MnoG51ixAR1 zJ;7_SxvI5dS9R|bDVMV}j7=_V>P#%HO~+FjY9OGD&CgXW-J7 z48a|Rd3`3H(D(5Buc{rBhxNKe-c55vp*!AaI!>E zeOaW$29w7Mvml1w>#v744moe8N@^LfmeH(d^SxvaRG$-JDN9ZNlFvw72?us>PaZO* zlIW9``Tr={ZE_24X{c&^BqMqcL$r;U zL@(#S^RcHR@JBZsD4JT+9)TCv*L%|&V>_A>4>5Oh0L9Z3Xh}Kbbi;~I zdGI!V=y^}ba!-hj6x|i(c3;y1BYtbM)dPqK>wz?xcEy*vcIm#d1GqrJnU6g_?V!6>*v6EjQ2SqCJW+ZujbELr` zkBDD<%wu%ZK27WR|`NmgJhVzIwWoQ z>-T>5%n}8E3R9f=hzIZoQ*}jHd*<^)uT}n>T|M=z3fL%_oKsvlIOR*(yhK?5ri#}w zd6#MyH>vMjLzkN74_p^61+3}vfhwd91tgjNoGr;LxX|jwPGzT!wM?AeQOJ(CUXXR# zb)R17DNsqye?%j-v_8KAdT|K#fw%ZSZ}*e5>L~8siK1; zui1AVw4?&$2P!qkl)neQ>WmAV8W1iJkit}%XG-8uPZ<5yDLB&n_?;sFr#6HVqdN4-op=HYXwos5nc6^ zSBCDQN8={w&Fe{Tw}cu!SL%#QPc1@M(|bfEfR=Gb2ME7u&QWu42GPp7a{9PzT=@R` z6`kO#V=sjWbf}nWmK4xcQ)U4uwoXv$TvsM$Q-;-LTZN3{yowr+n7sSf&l;D&@|+eE z-V=lrKT2ywED;&=O;kBxgjj}Kcou6J!st~4Vn23W@R@_n3GM9fR0XKlX1WX9*Z+Ld zx$}{gQ@!JYZiXfCeF|^l(ML-0N@6dT)no_3PboJ13Dy@Htg7t|i`n)?R1+hb$`A<3 z3gD1Lb+$=08H@aSik@F&d>tE?He5)RSjmjA56ippBx-&kwn0tFc-ImW0ZGQS;i(i| z!P2Q)TNhIH7lu6?l)FdmA+JKku0_BNC1w66*XX(}N z`)`-nv^r+Kh>WbJBspf_?tI+{M;zJ23xEZQl^|S(x%UIZ1w5)k=T{0~U9B4b5b8Yf>5jU30;H0a64dc zm*U8v9C8>kWZ^n!n7>@e3E6DVyteR}$RaAx6U**rMWfO*OV#XlH4Ja~71Fw60704X z8qq(keI89OU^~M4#kp<;`=<0w~|Q($x)U_k!d``|GA7wT76gHD-%LKBUeN4E%j zmuo{a2=olJZ@Mp*=#@bL>CS6&4`I?E`1C_x*;Yd&*WL)3nX^q@tUymkP3%NQYyt5e z1|ELhJzyMZBDj*R&fa4IESfDEM(ML7QsTWb9)l^tTd#B{3j zbt_>JCuR!@=X-EE!tU`&Msa96CBU3KLTCzh>#tG69J1c62ckBX-$@RU6LqEw^+cQ6 zbT)f^p4dD2!rA9GI{0KA!qxDLGS~elV$ooLy|qhy$dE#GhNW58E z4tFkRaC|zC`)sev>7|e$S1kUx_LFt3U?BUU$JK||ctop0^s$TCL zV2e9AmiI^>RhS1>J(>RZv0_Js5h43D1LY}U)RG* zY1a}61lL-(4-u!OLCD^yA*;xC%p-AEFn#ciyW>Rw2SV6`&qck_mvA3=zE-fOMO@*~ z?}0zwu*1lPFtwvYyz3l^l)c>yP-t(mY!|pX9wY>Xpl zq0Vr|Vl5_V`4OQyJ&V8&Kh>J)rs+v~h7r`a#kMuXk88(vYTq`?X8F~J`&B4fgQx>{ zfOde?OgqSUAjs9kYydzk8tM>RZe?E1j&mn^c^suYinv9-R^huZa7Xnn!o(f%+g=DX za5>)7Rp&Yq!v)oRxnYab(h~4oEtwr?(;tqO#S^qOwfj7KCE2$zaUpxgCI&;SH3>YllON6~5WxFe;R4vfUF*N+o+ zM(gtTa;1AfcJh^ay|t}OT~8mayj+hv84n?f4?fHW8VN(t|l*XVuZK3u}@i0y=(9I*Tc}pJZTRgV}z9 zO#hq*jOJcnabIe(vXUd9*anv)`p*^J-#_HRwdFna>JhWuMu3^SgJ=dNi-FlYb9yx-Je3Xs5 z3)tWY;PqFxz;#WJ6_!#GtxgRrpf@i7;O2%{K2G!fJ{9(_5z`6} z03TYFLt&y|;tO9>x z;bK(lE=tM~Ljygb>L6~fs17!G2V+rv0M}7C8t}UI5Tt> zp7AdhMd#W-h4|OZan%VNnu_?k?$H7tK@{9rf6!~qGzf4q8wzq9qDx*@r8#}|hB%`7 z_R9IlR0E}vMgbu8b@Q(2N*VYTlSA3CGA<@tx6v~3Rf;cP<3sxFQ%C*Sb8UJW9 z+j&S5{N(iIVvh!Q(|sOR|2^EH{573d1`a0ZHi)yV-@QoHh4uMG&zFMBh3V=&n{W%C!w|f$97kthymm2v)Cw zBJ-r`?*4jly?4=Lp$|!~4>7$%rtn%iKDtWBeE=j=-F-(IkgmwZ<*uOypwVW{owukg ztJng878!?FoaRktF078erHI5JWuICE76}ZV>~*ErGac0F4OJjGWMUxyE$ll$b{_HFP)%&lc{kdT}UR z3N>w}+?Z|yrm6)A|HTK1TDAe*w1hnfH3$S|e|n~`Q0r@>m)qgyEzv^h|L{6!stbOlS zwyle739)XU(r=y7>3q6(G?n zcT7eWTjVH3rH;x->ixdf_dr)8ZQWVPo7aEUJ4YxZ?-X+VbA%V|g-4lLU@-F%%#NU= zf44TC?N@uj_5K=h-)$DROkX-KnCV0PHI37rGJ2e0G zF46^%RAq34$BqHZQuaybV%MLkwPj1-#$)Mw$w1(x+>rs=onR`^3Hc2_KmqCoWSG2= z@kqOxFC!x0yTuD>uC#Ku?Niql&Fhs3)wF;pE!J*KXEc755=;l~eh2435NACRWJ9bJ zeJg$wTgB7&xzvBZ>f8CJbP}&zGoNmVC;izdfA2EqD^e2^;dp0!_McRY5*`p5r&*v2 zR3vCWlzlpi;$aba1+M{|e;Et z_OtNrZ{Qz3=2IDUgEzW*F;4~CUKJlUjE>Ob8u8` z_L{GTPEHD5h=3n=Pi{^#fIdDiU)h4h-W9+yceanKrkz@JbLS#g`bpWMMIsMF?RLIv z$A>SO3Ru$7uZj+QPeo@o`Gp3QB@@35$R-*>y&DR!+|+6iqcmVsaNW2-^*joS|O6T zZ{P^PP${tUh3$OhpV4nQYeh*~8epub-|YDgNzy&j>rFoW73LeUilT!GHGcxC#(^VA zbWghM^2r0eh1)Nh3mDgn-j(!Ox9xe8+ta;$fB!u-S;X^4yPWTeiWsW@^t)oi_tyy^x3l`j{aVr9vj*qQRp|_U-}tMUXTO`=`l@B2Dt- zksm8Uj-_v~k(FZVm*C)n@oP-&&{Oo(s#&c)b`;rTe}ndxt<=8!SNzmfdDS|#7*<1b zi(S%cu_bkpwQbzL7e{U97UeA-GopDHV{~eTz-7<Kj09vKkS8; z84JALmh+nDGc}y!N7p?Kwp!h9m=C0i{Nhx)5-_hxh7ho#h>uY}ChYvF7!Bd(t|Z{l zMOOBpA816&zI&rvf)~8di;_f-0n6eg`rjO^U$(#1pDBFQ;x09?;-NNSK zM4^1J!?ib6k5R^cd_u$A2`&}oL=CYPaTZ5-Pph~VaQZ=51wXxs(Fl54Z(XbiB!{*L zgt9--t|iBFsv_Pjx4LHmnb?_hfRYOSJa_xc3G=LpR_*zte|*wQT!PHJ0D;a-=&sxw z`c$VxELIs)tgCD5F>P@DfSRSo{Zt$iX2OOdygzSL#~yKl5sO0Liy4g;$GTP(tj5rdg`;ELa(ZR>7^(8m!`eCk+~3S9Q;%NF9Lo=k zgmbWk|IAObPvK8;Z&o5j2tU*P3;$6~#eT(s!)~ox#GKoL_Up$h%`kX{@Jpaze;Z%= zbE*h>V^ebWqI!n714>sA5r(w7df9Q;J7yK$UFFL;E75HvNmG`qG%7zOARAA2)#wk& zTOj5&0es8Kh9*EORQ+$l?Q|(Du+?jA>Q8TCP7vJnw>r>%h^E;g<6`Oys-mA)$wfRX zw9)HPq|7OvuVh1var(BGNwKZYN^svc84$cjV6F3OS1-`v9YuD`tQGE-Q_cV8SB;Dw-iRKU%zwrs8mlTZnSlYig-qnId57*SO7-QK?RQmM?C?M6f0!nw8T16 zEr6m*%tk$RCx*oR18tf**)c`Ihh>U!sG|b7!*S&WH|N{M^5?Qjq|KR2UK) zCD?wy9WR`U9h+V08L&#&N_SP)4psoWPtOXK_;tay4(P!r*<|Iwc+WGs)oCJSV;&we z+?|@iJ7Q4(U^)X|M4h!El{GhaXqOrEeOJ*Uhvw*wySw*JjS|UBjYgV*pJsK4Pj{(S z16Sg-%4n%!uA7u;tm&Z=4N3L)yvR8*Ss9nIs5$*H)_izUw{$=K3P5HN%@u9uZYFWW zd7w;o8Y8~)(zVl@xnFR4EI(j?lQx-kf%zU_VJh<4*uj4iMzF3j3WY4eD%90dH6ZV8 zGgHFjIAhWubyM~`r@>SBtL;pVa^vS|xx*J)PG;~$1UKF_MX%?jr$WI;c2ujh)_JTms z6i)PGB37MrBbIXia<|L;3wE;JsSv;Yz*5WXbjPb9*R)RS;*T?$*6EPasg||QUN=g6 z2ccyE zu(eV5h%o7%71@JA2Mh`A44vW|EDo}B#evIRV>-2&E$n7w0ASCUy^dgue@mwn%?A5Y z7X`uwFJ6##{L?{={R4KJP?g~WmOOP8@S&0*hl zO+DK-_fI0o)#)zPc0oh%)@HGFIJAaHzw&v-?2%k5JAxW*#P6@tpS?Y~tM&Paed>C@ zyC)miCS=ljGD9kV{;xB!Whp3h@pkJBe}z5^$Xcg&5t_-RKUeS%qj@jeBf$POd_ypW z!M5I?-O@d{Q0ip`QJrAfBMC5jp#yCJVcELr_rmSKIF&Ys%{TCPLxt<(bk$I-QJrS4 zfUCh_iyz0M7KDbkhxbx-7p~B)%#?>c!2F>NF!2ca_j@VtQu5Y!yK@{UFjbG!I9-`c zXL2Fq3`*l1QP_l=YjBgNSfJqk93gRyr%N4zp)-}1%+e}ER$!{%>+pUENVq0W-=nX- z`RE#p9QG1DqyA*-RmXGBczJS@gp-xTR%5P0p;n!5x9W#6qo>xPqz)U)n3&}VL9iiKG3+5HWy8!|-kRn$svrehPed_t|0xfLy8S1`$1*Wx?x~um3tc;N5&g;KflHSSX}Vj zB%=5JNnN`9(Mh^8(F4zO@5=#6L(=PmF26rpwHCMIx38DV2ko3gLSB`q-N9Zt8?FS? zQe^Q0R^Q*G6q)!E1zL~jduccAzLd3`ocu_;nqV0o6N7@3A{{T*Y?|B_PT8>2+>E*O z=(00#C|@sW;&lr~sjwBAPKY1(!24rP6k0CRZ)_smWM7|m-QF;^oygrD%1DN4-L>ZT zb_zzFhoS?Kjde$QvXBm(~E ztr8>aOicmNcu~GzT|FNA@0q%KYy=4Q!j}**D;$R%#EvI1p&?Ick*}f%!a^892!~U9T$i1A?0xP*#mRM5T6P?V7H(8l&(A7LEW$Q5`{X#3#l2I8?#4PGU zE_=ddplQu^WZ~B<2d4tOF0UrrE_e5M)B8)D*MqjS#TgpuDgkQ={=2iQxBq&=$NIfO zwdeNiM+ov6K5>>N#PazU%Z4~q5ysAsQn^g33-VI4tBOJ5ag&2RJlB&TU!FXR5ay&? z=#ai1JKDrZX2$OF7(A%YOHyQRxJRY^7%SByn5Hh)XTLtVXMsP$Ua*1n0)H?lWCJOq za8jgGZ@j~LQQuDX%mNJL{PTlAXd z$-RV}YIEwx@ntmt&_g$9F-Va!Z2)u%I<=9x#jkjs0C(Jm)_N-{0_FW!ET8~Zie}I7 z`}TIdfV5twQ;ae&3?BCzc|A>vLchN5+9LMf)ch9F7OYN*2l`>Ma9EJ6Mw;N`51hG~ zEFjw)gaMrxQB;tDwQ$VLP{-d2hpxI%RMEfw$upHx)lo-|olb`wga6e*0NQm>*DOjiNepm<=n9A((92UCd zvo{#zuw|##`uApOjXhqgr$Qs@rKFwjQ;Yr>hZnrH$k&^Hdy8EOP!!!h-+_*|z?JWg z0Yn0eCgY_hJR8^p`~LiIiK=80UDGXK*NLgy#Xm+#?4+zHSrVY*-=}HE925rbR5dk?u{wUBm-sWLY3E zyjN&mwEiu*d)b2NuH-|x&n7IY{*_|a|H=#m8f%?1^WQ0gHEStavcj3X(=XLOKB&)9 zaW!5r%4`ejE^V0b=TLpS^K~M1>n!=|!tO2bVD2dWPrY9{x{C0R0?cuv5?NPjXY=8) zmOM8q#P9l9NV%A6DowNT@Il$ujO&D_lSC;+lhssfq|e?uVn~{lzkak^=T64~&s-BLCB=(yEHw4fH0_JufXPW>D;ZYI zPQk4I@Cy8~E32g__hIm{V8(AQeK%t6Uy$ingLTCX;>MCG6BE^QnzeF5=AQ7z1TCz9$_vXvEv9VxNH}VpQdY`9FpWk=bUUh$H2dd z;~UWcGNcjeMeis<8)qRc6ulm#hR$e=*oli~x{Cz0qWuL`FN z$*X6pfn6N;k@@wq(fE~slt@meMW0MbIq79Z!Rh$=US4giWNX_iwJ&Qln&h-^XIkQk zu)G->fj2D?aCbMoY&ABtYc12k33FPpfb3=*jt|0pGymf{Hbs44Zank)q^o7Mp;50U zVQWS#RJil=N1SGQ=S@Wty;+mZLZ#vdQO;Oc6{gj)7TVGzAoa8XszXLSIq(iJw?FYT1wI54Bznh?CST#H?L4jwC zehL*OQZK!)w&w2sM?l-umD#a;FZvn9*$U?qpT*NYs%K;wt9r4+Jm z&m}7yZ>+vhE>SUYy_DML9i&hOs7(Q%MTaG=q=2rBS0?VJd0i*yReM7>H=TiKpUu|> z2}SqP#j>$t`?pJhRH8-on{(enSb}@hLUx3PLM}Vx_KaWuYimBpOD8@3nR(@6;Q4IQs-4LwIH$)4wRpC^Y9D5(%Yb*nWto$!MT6L0`X*|S zSHt93wx?>Bh4=w3k0WVS{OTji?~rTRz?HnK#i(*%p#TDp}0=-KM+d4CA7IxgtjfA!-XKUJFr_6r(cWie}#Nf}Ps0Z)0@DfX@k z@XMO;$NCC)m5=uM%Euq0AvJb3SDXyNxhqXUrz)4;)yL{$?X}1Lkpo88r9sb+>3XK zSJOcYr!BM#Qx0Qot68mZZ(#0-y2U&Uy=-s`?w3-%8>^Y!&8GyJX6EHuEaB7l@vwBz z$NE6bmX1~LEZ;62m@_kfJ>C%#bo4orh)`t#3Zdm^;e7T(FdcY4OtoJv{QBcU0{lFD z_iAiiZ(=fvUl5its(SEQ6Lugxn4z;3ZL*lZ%8@>ky(t)_^Yi55-{vO+*+nnA-G;wI4Hpe<=he4Cr`rvox2z!>Z$tj} z+|IRKZwN5nId1yi4Pk-~OwV#7TS7iTwK~U@O7fzBex#l3qCiyds~og`@1*HIO{u1+ zxTHaYaTUpo$j(>VLZ!6(qo#M;z(HhPb#7^PL`P?X5~qK%!&^`~k`^VGk)M?v9O-Pr z{cMX(Qsdt$ZM3!oo$bt-2&Ug27+fXqVdnSBUX~=YqyX1oW9s~)!sptLeJql(7R`~7 zk{yn{*xH>7U_?2n(VX!NlzOgqpaW$DxhHXn74511`mWRVq)BETul|Ad*lsh;*vEXW zLn8?}Z&;V`_1+yl{7BUKE zHSu3^%pa<+T)msmwXv|*uIM))m5^V&7kbtXkmTB~ZWONDz9efxa4++Cfvqks2G_Mg z?6AqkXG_aHZTn4zx4NyJN?>6aZhx@*!4Iv5n*%-c6O{gCsf@HG6k6JZYN@?IPx38EK2_z3hVhhfM3jGxONiKFGI`PY|PMUuoJw(b;i_ud$YQlns>F4 zcTwPWbtx4adb%F6G)~I&O`lg%L?1{zGab~}QNzUM5TB)PE{kb?+S|u6jisGu_XrT(s?LQXh*$1*EetFqgEvLlHqX;$o$gQ(3=T; zw@UTxy(};MFDIN^-Rj+4Imcljt{L7B-LKGii|;3bQjv7@=_^x04g+XH)6 z%(Uk2w+Uw;CS9H7T}?_p_I z`H!}zqRzc?_)&zOr`=wrh(^8v30~pbt)3Y_;~$o^-@0?1JPIx5&Atwf&sNFgIxJy> zvGCm6&#sugMC+u*<8!x&Eyy@%dviwcjj_P(HS4nzUt9X>6@f6=+GO*6FLXUI7^8Nx z1=P@crmy!67S3JMC12eG$-3_^vR>^5x%mzNQKLSN?;AA!GcLO_l}dnoYn^r!{CioP z@SovuJk>4Y6p7-K@GYXRTB9&iC6LVUM@mvuvn!>$TAb^B@sd}Lm1Ez_>yGA&%rH(z zN2Ar07p8y~X+}g~v_dQX1HuXmN8;It&M0(lIgTJ12MdSHFn1$mX9V`Cs?c=GD0_OG zCpB(Gf&5Dx1Q&f`hR=a;Oay^ZQm(&!Q+~$tfvCRA>EPjzVLvTe+KqYugrJ|m79qv>XX66$RV%p$+B?&$I_Px9)^+XE6%f0(oV}CU z+@V&JL;7YHYTe){1yIX1Yp)jpLo6b_n~oB@xVRAbJfA*M*}B%1@{71G437YV>X${i zxy@itM{y#KPk(n9n%Tin7^Wn(XRYw8;j|TLaM-L_Ph&>*Jrh^0O zA~r#|7dhSnLc0*s4Z>2<|TmLIsasMCb+W=FE$) z8q|GV=r|D}?)=J7;QFD!HbYF)3#2p)tYGoXiX;IDK<~{enq&1AOiEe5rrP}%1!$!L#N4L`8i7u zNUP-#ebnA+5maN5;eD_G&oFzP%Kjg{h|^r^r(d>ol0edq6w{G_^Fi7>;VNI-lyse* z7KaUr;+MZ?GWf=(CAD3$MD0Rqtc{2;IBV&nw&G4t6=a!iiIbAxP8O0zId2z29Ql4o z592w`A49>a{q;a!mViQoAF-rs+#Foh{QdE3nWi6c4vsF&xz3}kk?(mHFWwek{%)B~ z=?FsCFL7Dwd@lT&xA{I}agTcYjpX)W@{vi}--rJRz1~!{bEs)lJxK(w_Xz!B4SjP5 zHQ~*$R0o30He_Rr({^%vC*s3(nN<9GFzo=f6KH^f#n9w#QSvwKqAv_~n9^5)T7Xf5 z?7$+`hXz4q5QIsXxVM3CxUrs3PlqE^l+ng7d51vHV|bX|={^XeI8jrIq<=FBz*g2T zop-BD^46p}72*ya^S_%6NkHnr-VO*0jnpl7H|G@H@&P5^4FtCz?*mdASZv~JopGVv zc!T=NhxD@%MSC2fX)-?LZ5ICl$A(^8kHZPH+LfX6DgJMqvqHhGX_~@*zH5h!vl$dQ zsVTsFWlJw7RdQS;b;aX|kvK`su2=A=YV@FV?4W9dpAv9YHEWOwb!t=Fy`OFgXqMlV z{s{|wUD-5q#DCtl^Cz?sGTdU>WD$d>2rg=tG~gbm9zMgIz8x%=cA2d;BT6|4P+KZ& zz+e08QEHcdDef?E9ZJ#Bec}=<8%q(i=d2xqhlpi_U%Y^~srLt_PIbbOLs$o_J)%qy z0;D;bL$B|JJ4Nnc?=P7a{3T|RW(d|;Tkxudv&XtJ$NwI{djJm$ zIouB(N*LyZlI=sh#zX^*#Yc=sd`HrkaV;d^=fn_>ub95P2B)bydxjG9g&{n^G4(c% zPTp>07WH4@Lfx!|v==`#B8v2R8wU#r3aJk(i}+zxdor=U^7_x->+4bAtI^BZVgVdd z-XECQCbR6*&1vLei^ZFaGA;PT9p2tqX zpDq#yjPhJmS{9`BC>(GD*g~eeZ< zCY;oNY`4mvCKQYO_Fi{7&A3yEFn$1!V7f9Dkh!Y0C-|xraVw*xlQq9CKLwUMH3`L3byrhKX-LxvNit3d)I){xOKGg~!q+AlNM8J@io5&LoR(x7=6 zuq$(1s&dxm?GM}UKS8fl>g*G)sR@Hcz#?0$8!{9j+M-Jj0k))&P!Y`%+6ZL8?P7$J zon1GNTdeYMS7{6(>sty)`Y&k3=ihs9kQTub0uozv}SSobk;f zvWxNjObtyCM8FQZDl&B^(zCD~svcR~y5F|Qtv?iI)qQS0-AS_jpHUZC-q-TBFL#x5 z(4@cZV`}3;Cdi3L8$slk>uLgjVxuus{B3sEV&mnl$|PCdyB`R{l6QO`!IA5AfdAJE{Y{*RKQi40bDNEGE4ua)qHS zn}C^-8>i$47i zQqeD<&aY`xl6gO)zMw2&+c4FZtB(&s-|Z?tKaaGsv9w~bA={^KKu+4vuj=v$P~EQF zcOCfkYg_L8fywv<-n;W_^?8qDpwm-e(X0^FjP0@#zgcPXGtLdoSPN zD=?h9!M?8w7k_bDiSvQ6$pQ0%+4Kf=Q9K^hkdWe z_(`;>?o{-~o9sff!mJK(1f)So!06RVtz(tBPj{NhBCW!wO1LCnhzW?%y>l6*es!r*tfMMgk*f<*2800RT)R9!mAF$(K?)@Dijn z2sj_o63<11a^9g5q7Xoo;?vwV11;fqp_pVilm&wi$gc?mE4%}TfS&_b0N(+md_nDA z_gF!~pQ(Dpp_-4Q<6wD*sYL)2lZyt_>c$vwRJ=(*u7{}y6A9k?(AF=iOCW?08#3Jt$=pAyfq!H=k5QU$O5mvOTjX&)GY zCqrDGCS}Ms_PD(z|C!*%1_S_0U{a?}9qZ_~GvL0{R9-(#r*rp$ldbA7uxBef8I z1%}-mC;aV`2;P|gw;p_>y+Kz5B#`2bS9f}sRxiZyK0lN@ey*HE%Xo2KB*jCh%m>)eLjS*Us~UA3j7DAdmQf77|dyF)zbguI*bj?q+B z#Z*+ta7(yv&nfCe<2d^)wjNEyX@v6ye*DIaaj7~$ff)i=T7iUx8kHqLjUX|oxKJag z5n3Br*DQiLSio8X5mp+zMhEU*!tnnl2?5TRd>V9VKk>xEPJ)&|F)rz^6citT2V%pd z)SFeMv|ChZw55O-jduT4fO5IV17&f@Hn)2QI0hVH6$8#9r=a*iKL1G$*7+p{UpnpQ zQvA->T-z3OqV9-DxuRjO>&ZSe~Kh2MB?gEvL%%Y!< z9~_7Sls<49c4KkuN3JgO36c`KAumDlM@>khM@YMukFbtib%R^GhcD`sUDf&u!K@Gq z^bn6X$V{JTEInZyce|M+8Gb|3=5%datRvi^TpL2xNpaC?_FX zQlPi<(X>tamV25KpY|!=&>8!9flLf-v}feIWH{a(yw8++F*I1DWcZrSuvm$6dKL0( z_)nc)v7CI{KSjo(H^~tbIZVaok03TIHacBq$10bq&@l8Ua$?L_P_0dkYSSu@HZv%aRwy!VlRG$xNMKjZ zh|EA6ONzPMA&2{l2sG8xsp>2G1WPbQk}_UChiU<=gpAZyz5+9{o>Rc@|}q-24~;Gz{7w zqi#0u^XAu|q{{@rYNRbo^J~Te`*_paKMNh6JF8Dxo=ESs>}2n(RZs&BhKpF2*XxH< z7Ct{@IGgs;Jmmro%Pme$G3r#Vt96(RfUfxN-aUNc?=15)+TYFGy8U`mPy=TD$*Ftp zvfqQ)+jUR`2`#a}YaQWU)vS|W`*tl4UycS@iUz~M{b8(TRU`!2w-PH66-C?s%T9k{9ajPf72R-Is}0q>Pf( zl}XB0A9J@BxK+U1L!(vIDhf9l^x&$jtg!%b0>Y7mNC%IkzkpJ#B!GLOx3qgSQ08(1)+#ggmc2(Ia&~P?hrDVQ7P{3 zL%x<|f@FQd5Fs25L4}Yqjb5H{?XyzKtuMG48x8AB98Ce5f0O8B-0^Q?P;WR<=gI6F z34A;}Dnh^%UOt|vyM=E)WH8x~@46q}v0m<9;g?qd+=UXWeh(%Q0b)-jM-Y^vbPz+h zaX1o{_#DBL>PCx8N35z+sR>yDZfDT$R%J2ex%8RF^a3J7imQs*j7B2kro?*99D8EM zA>q*PW_`611{IoGGchUWgREr+HHOp{aRJ_B{jBkY;ABq@b_9$tuP^?e>&?UubD%93 z%^EL^K~3TQv1tf%poC%2SrUgIN&^@zxRe0gb~zqaRxl0qYQDNV>s@5;{I|)D8H)KX ztbMoZ>GmwHeaJ|M`L##%5P1R*YZaV~hzU9f+yh}nGa9B=MQIqvTxj>}>^>=GGaX;# zRkn}@HzzT0)Z##ql_AS0w(XSZ9Hq28j0BrAETm^`i(C1co|oW^Vg&kJ3LXZmtTh>0 zh>3{!?{4J211BROD5kToPvp)@wdJmjUrSU2b<3JW`OkQdV8F9sPs|f3lvZm)k!oWX zLQ35~fuGAAk+KB`6JaKn(>@=nY6)(x*3Vd@gkV>(x~lYnx%)(6g5>Wh1|q#%tw-|Q z+XA{*Epk-Ul0luJg!jk?*1qUSHpnt8GQ|GOEh~4O=XX=GmB!+|DvKMR^_spI$Bnz_CkEVeX>L@ zJ~4AI%-2qkg90_-B<21XMx!?>zGCr2jAe%WeHvjaA5Pei3Mp<2V99XfdOgGQoH^lX zk?{W4=ZDy>aSzr}Tj@>ByZm}Q z^PIfhii6C8FB~CN{Ub5@qdOL#9fQ_YSjo=Jf{?|^;Pv}vP)bFse1t%nat)CSupVo_k0SAhWzVal0+0yxB;KPt&e511(rRW>KG82eO zA{Qy0MT6BkUNAFFnHhTOO?hKd;6C2PZ2=>1A>UR4CrGaqb&dE*BF=05{Uj?Agbi%! zvK1e9ZglY^?0F_oU623}0(707n~;eMIhkcrok81I7qUsoTj+7C)jIFi=6O|h_3T@<1^m?TfM-hgAB}}yQ`E_Q`~|RXN*YP#{zD^y4=Tyk+7Bn4{rqj@7<746dD2Q zZc+6Wc~L>~VKU9V)(=1K!%Y;N^6V!o>ecf4fGUKtk2O-MmhNCVTb-8CKCni8rbSgz zRd}c{3?~YhnLacKd;lcGAzW$<1LsLdsaDy=cB#cWVQ#HI8ylkpkBSP=2-XXST0;fW z*WKN33_lqBh@=vXQFs=``YJOVse|LjJk$$(0C)+(2$PQc%2RuXUTsw?q>5QN1b&ZG z{wwPDKJzP8Mjw(O3{!)(rpOClyT~k+k22)=?IwUfJ(8n9G74G~{UCdCL*aV68`u{A z;#F9(Po6jwm%j;bi7*>{U#exG}s@Rw0j-BM-P=HM;2yjI8JNNy`xST3Ur9JW{#* zMj_+oil|hc^=LD_6vF7Q2Sgp+?_ix`A00)k<4gSADlg(f)$PRnkcw_$dxmQiFYLU9@z+_h%9g+Jnv(obt}nS!@Q6 z{+4`R0*wEf@Gda^^4Y*=n@h^Y*hku$bM8<}iY>b`x@$7mqn!PP{-}zpx5FjWiXI)6 zgz-NPY90o7gvA~_4YzOLH^2S6mi_SDU%!;H$r^&XbVj@AolhzO@!Tp8(bTprh2m#jjk zoJ{_BA8>np_$a!+L}nIXSrR0`n8I`$%~os1hNwqi#RjIM-u%juw8{luxI12u{E3Y< zR7JHO=3nGVz};ZJD4qkC#Z@SW(t1=M(VjJGxj?$(RqM;2#nAc4BvFkWJjUZYfFBzH zaQ46}K$9a}=yR9FO=xrB%Dlb}lv-#u!Oe%yD{3ORlWYYSL=l%EHq+p};kEZwGtYel zK6nekh+3L>FtK-oJ@)2?E}T!kfSKt}aLPSXGk2K{($&_lLE33^a^TcrBcG3x5{dLQ zBc#N1*f^?VnObGrZj(kyu}N(ic_@e=$=rlPTc>lQ=loe6{PZ0MrW?C=gwAab^WiOZ zeg%}Q75e+#@;;VH8DkDbEuTEN-E)uGPk9v1*IPgRvefZssd8RBIrM(;$P8Zkh28+J z*3IUMZsEJ6#6gNJ8AtW5Lyy*%`>aywI*SnuR0DY~faL|+V1W@~(F+D+8L zd{XKlDnnbrpsG8!^)(sX@agUYC<=5>PRzJtXKb+T2&As3i}6T2|F-VHQKsKhYUO-A z7he!B#uxGP0aCMCxm_Dy+Xk_5u6e#z?nrUjxB6S*eEty~Pg$Jud2Ju_Wf2n}pWL*j z4+w^Kfqe#+3IBsT_l$m^z!-R?S&Vqx@1^43IHYSC`BhT-Wrpg1)r!1W`=@zy5WpYk zWGqS^K8X8=q6|4p=hC%y7?Z(1hfsPNpQnr#F!gNoQf%ixs_dE`YfjsHdiEy&mOL!a zu=F>@#%fyJ-#Pw9e)7R6VE4MIuXV(Ok6m za!)>xr^G3q$1(pj^zV!bQ(9rQWsVG0~rCfjo^rtD6^itN+NGK$}>6=`?(;&}ae>l&7ppxiL+3#*vZGd$dt1z17 zWoJ`-G79{;t`2p#n^~);!|<%UUkYU&Wjif(2ziBlpGXGMPI^esGM1nlRLXcK6i~s0 zkPF<+=I+B}|66}6Qx(&v-UGMYuUh_28?KcSdT9>a5)lPHvE}<@gp#|6hy@*8wNsXA z_>jY1`i_bjTujiKC~m0`P1|bk*xD_cBdiRyMLW>V(%Pc!h@Zf@w2yd;YY&?O-&wkdA=w*jjgn&%fsLJ`LYp#a7$uW;|G8_e~urf=q< z&nWo8u>|t*mBtVLqixXukwG~_IYX@^JSv=^SFsnwi%MLFdq62TnknGt6ZCjDg9|_v*Lj z$J>hP;PdudIW|C3>j#S^9+B?GoGndEWy)Zn>F;nd*Rv8Xc5)aLGturnB{c0d?ZG!Z zRJh;2vRx);ZJglp)97`&rJ_&|&A~neJPZwku|f=HKX7ROb+xW>mdWG}kb{unWFGD8 zDOc;mp=MS6V$W!EXJT%(Soo*qTZmuqXLofJc(_&4Ih!2Gy1lgRZMAe#B#0`s8z8JmwrG3D+a=A^^F5nd$%H$-kKjCcJyErG4n1_VLuo}$OoRIH z1ufp`O7W^tkc!_G)^2_5&t`|3P-&qKd}JO9l8?0-m~vjK%c)rLYtNN}jekUCgw}=& z$)~d@f^i@uiV{HxEGI=(p1~w|b5ochGcl(t(GY#;XN9lSA}`>Be)a|AaMiAc?rNtu zr%Q={GXrIxORZIRIFBBM$4EEHR9Cy5sFl|(ihc{7bvR%8_UF&Z0OQi{S!~6vGZ3s9 zi2s}57_^9%?Y2Ss<1FFa)0zVfjMG9+!+bWe{pPqK zi$LS1xvYZN?+ZuYdBgH!Phnx;At|pF5dVwVR##IjeK;T*%gw=b+ZPmu)=_mcAWhC2 z)Zt<96cgXN+-l4XDcGwhjmeYRk%_-89bCL{KCd)&$dd3?SlaMV>vKQTYUNF_PpdL6 z`uXd;IN{lXS*zJFbWlb%4}(g3&>R+ylXP`nv8NW1%eBJabsr;VPN-54o?>#futB z?}UD@jdXOs`dd00e5p>KY-x7-oKWuNzyM9w15YQml+m5EuJjgl<6XCWjF37cpRJuV zDlam3fKMrB>($W{8yRRwlI5oBY*H4q5)SQE9ho1Hyolj$bJg}6t#o`7Lq&I3;rrji znOa=A>TTU4$;nS=Isuxc_12u$#7wDwSW-7kd{h0zMF+6>)bFT5%{UdecPdyq zkqX%Fiu~}m&7PmX*1J@h9aR$&Ab&Y{6-5Q=j_>D7X^B5*rY+tus^5Y3{a{g^4`L$I z(*U$Brab%hi!R6uQR{awq(X2Kwu%7Vm5&j);lO{?%?^c<(+X``2#4c)U2Sc{-q}Q* zSB;)_>pNc zcu2IjAxEjBWq>imWVZTzJ3Vi-cZt~AYh7gHSWb3h;vtqcD_o6oMt{nPN=Cb!j{Q`@ zq5`!Sv1{bc_%r6uWs|kU!*e*M`Hg1H#hqX$(W-g0)1N=vIihfWa|wRl;9s?j+5ttA z5)*ZL<6#=fMlAprN>jMrrP8@BSpaBsDU5|?A#Ho{MXOM{ z>Fpf{5}ns%xo7x=3&?yVzq6NIE+#<;S2cB1XN_CIy> z2xVrMInkFpF_ARIaT$D!xyp({ktx&Ar#ZI$)L1m{_$v1%%0g9F{bQ3vmLvJR;sBqx zQz&J*d$Jd*liGL7&*ZelT1|l_Dhi=mJ3=Fq#WeOh`>hMsM({e%qI@_k+s>vBudOHH zmy@=uW+9H?>>$~&yGAoW!_7wa_?=@3IK%CIwx_wORC5}Z_lIuXu#+120Z7ZS-;arCNCL#` zUW~;XCDtCr&sHs3qG=wOBnA`u()`)5js$aktfqf}Tx@?xd_LrhXn7cLIWX((u9IDz zp$dX%)Y$7RiDc``1NG}I`yEVMo zZrXXfOA+^SzREm8`0yx8PK_E^3xq|?$WLfO0BVU9o*OxT+eR4j$=w0;_>q`evTu;& zQ{!(!3{~{;oaJih5zk=Qv}UiD#ymSgMj){S zg-C$!4zy1U(^zyg2Q=~LFfw+voce?#+qY+{8R+SyFV%NCj{oJl%D*ze`wVf)R2rwT z#-&&7HJUh&W{TzRL~#}?Lr~3CZ8n3+`aJzKp=)6Zn#soqSte*X30wD~^EpfAKhwob zmjx}Jl)88Z(GNb3+pDiw`xzcPTG_#!vNYhiEHt!q7R#jtF8n*m9RdN8E~T}F>Ps?o zzjHBYO4f{T{WQYVOI$at%&(W@b9p|%N3}nL@hA0#X_?BZ0zY-y#zbzn6?J=oIO#a5 z>|5hED4%+8$>PidW2KtYB@Zp?mIUzJJ6^2TyyuG2`$1f9tnsBCpE&8|R(DNBb!q%+ zGLTA*JAMr(v;JCFq4CdpwCahF@Q!FW8!SWLH&C|PSw}<>q+4&OUd{T7bgItxk!OaT zLs;9Utt#QSwy}v`ZmLRRnjnwUGA;|BdGp7vO$P(7vRxV4-ZsUpte8{(JHCE-b9&X) zyg6kievK~9zFH$~uQuQZR1(ANE}?31|K`0{yv<`zhEjQ@b`WsXHpj~FA>C_sln{<@ z*n>f3-a9?n>jxO7=q{prI_F$o$r!J!?mnuG&Wx}kl18G0A(<}C@r@Pp7E`5C+rLz( z$vI%oIh#$cAc?sCeuw=CnYFr;zXLSihDz)w;$u4thb66^KSP$jKmNr(D#gB&*coD6 zDh)4{{I%m_m8{PlDG76!E^(z>9I{vu_cniBgxvpj$1k}LX~aldRsB-WyoocLM*FR6 zXPTpgB}y1aX$=!%1ElS_ZDSo`g^M!MetMl3CK%-eEK|gF-cdY$+ewhAOehd2n4Miw z6%5uJX|as856?R|YK@VMPnImvu1pyA0k+Eizs;&Qe(hki<#1t@Gfk1>xm;=s?7-4C6m!K3|5af}peUg7_h1oL?uzrkAY@#chN*{liV@GVP`CIRQe_?tEp%VEG9OTP@cYmsv7iY3HH}N0$ZUXoga)_#v2Y06{uVzNcdRJE|(gzlPIo);{&%l*}r;8SG-mAwtMgpZ*WBrUxQ0&Pa zgQtJ$tM}$#JeTwiHd^6HUX)4u=lBUfE8Ha$dnmxFyt6MIq0n_X@YnofdzVG^mAqNy zfZ3Ovmv?C(s3hxClqZIA^$#^u?9-T+71UZ;YrZr{_=wiZ&nF>uR1H==OARdg0D2NA zcALLr9&D%S9=@VQ_=qgk;kJ{6f;9fiphtt_OB*KpR^&pw?QV|!Hcfgm?KeyMf#jvR z*=S7b(d@lyR{U?b+f@h{yL?9vz#;YTy*HDPZG1PpKZkE;YVUATYy~W%g--u=Q}`e3 z06wcM@UCDth8yzzT)k;KIrpCAD#3t-;Yf!6;_|GUvRoCO(;kUgR+Io z^TlAM_;PGpveS+}w{|YQetDwk>kZ7DS_Qpms!7eAdoZ{=lVSN5$sTK=2zF`VS#$71 zo_J?+ig{G(Ijg-*ea?|PQ;3*m{ZycdL%)U3h00%JrnYg?9tYW2D+tpS0U)Im)??(B zSE!Ti+3h+tUHg;pd))`@cyA~MqIMoya|GM)&_O)} zziXP{sb#p(C<|bA^lXY`%8>ayxn|Gk3?1Gd1t5*IlUH1W%W(gIdRrcvRrDZ9Ewe5? zQJ{NTTLW&)Zkrd|%SoaK8cPA?`E?@sQr8(Cmy;pto;k*8r!fqFExteT{wN}sIyP>u z$u*7(=SwqY2rJG3aiS5u?g17zgu7#6mv%Kl&gbz)%C@FF|~!D-hih>wdz04;49&*CwUvTU{2 zoGJg#)nF?RuRECo?tFOiR<1?I41`Kdgikj#pT7I)%&tuRYC56TGHq(vHq(z9(wp%2 zF@S&C#o6Bt$0dd!9Ck?CWJO;V)kD184!iCVLpWUij{c_Dl0KO21% z#pxQ|{cI%W*OY{@;h zdApQvC8m`u85P>WWDH1h-GO}}6}fcPZhg`qA+h0~jwX<`2y%K7%3o{`e$9;B>Gf5rU?#X_ zA-9T;(?zh$K-_!&H9=&obUt!e{ur(8Y&S0Z?Yn)Gnvk35;l*Rq;6&j+gS=<=sMjlw z{!Z%P$!X|iM3%&f3Njv>Oz9Rn{$s79yNs@?c*JN#V6P;sHH9 zR{NEe_bNJYwG55e0c*DIB9iWl%2_eenIYR}pj~94W2VNU>|w2-O|oKt{*0#e+X_)B zd#QNaBir!wqxpMXQ~^sVL&{`^+kqs5Y?)jJ``T$oHxv79ekqrz_sj#=W^Q2X5P&qe z_L_t85^d*y-yN8PGrLUSpt%-Cr9wmkP!S+>rRES5Nuw)y$)~ijvqt>+@{}<}nqqkx z^CVfi4)mzWg;gDk!u)j4+%JLI-GX81W&J6$3(1FZ5l?enX)k*%s z5^LYc5NB%r4PZL4*t$4&1WmzUYVhn4bJxc>9p>9Tfn7Pew_kg09VA0E8%R?Ee zZBZNP0?V6G-zO39D`b11$g;&|<8zMD{X+5~QO(z}nNM@2GI(I1t0WEb?i9tt;pG!$ zJ4TdR|HsOgS+$V`*5(Ch5m*Meoz|L|uvC$SHfpeJS%~fl>vaGnWSVqZ?2t%t%cI8o z2{XOH`|9nd)O)&E#Gnw`o!!Aew~|vYWl&!@H&X+?Wog6B_u=lh@_-=l0J2BviED#I z0s_&O^sC+1tM6NDh6#3mH@DO)&{{^~R5@j;gNKevM!(FM8eq;C#j)6a{%Y~JhYQFp zA$?{6=do6CWH_29rz6JUjlW_?zvmufBkkYBEv^ISqkoTBguw=l zK$Mxh!^S?$QjrmKa^Xq^!=cl(vkMlhs=BOU-_^(uHb9}|038ZJ`Z|D$BksHiB@!oM z4xel{Er*xZB2>io%mY(BaX@@Sqf#?Y8?Zfp+Q7g+R3^1P|e^s(wq; zieTlAZskJopF)@_H)LBvTAcO&SnF`Z6BGa0{riQsW^jkBjV~LOUHp${CacF(E|-sw z(+adz1aDdDiU?eTes6pY$zBmp5hrBgNBV%=S_=Uswxy1>1NTN;Z%t`AH6=_pSrwX! z{#)5p5}_;t)b8K{imZwwJ91ZBc#u?3Z!_XNyvY#jBJBCq18Ozos28{(Aq5TO~;{zXBu;r<&uv@K5obr>{iCE+XSDm){jBPcC(} znvQ%z|C#K6b~W^&%Xcb@iv>0IZ;IMv0ReawzALKkoxmL(VkPiIwO7s}xNee4n|Dt* zXDz-yhNL5`exOF%xs_EP_wj7UNjq!h@vjzxuqW$@YvLVRuNMHH|YGb6~QU z^6}%WU?3cRK@n*t6a#Ox+cWAgIxULp=zzDh2AT?qK~2%b&I8X5A>{n~>{6sfoaIYR zeyOLq2|SqH)IooD=S^*<$R<^xZuC0Z+NM53AK9|tN7qPgEp83xrq%B6&+3_y)n%By zm2^1)^>+NrImt8U0*v#(bc~E_sGIYnf6V~ue{+@4m6gpG^vn9AlMBEWu|#IQBDsfm zR#m&aKy+opky~-hZQnntA1ln!HzSlKC7{(@_CSP;bmu9+EnKnG;6fc5XBk5|d|7?@ zd|B@s!g-+Eb#GI2fy5jo>VrZ)G$Z}(CTuim&rVQ`K4h@5(;!(tZ+#|Cm#SV#NHYO> z%mv(B!Vrh3xWn9D$Qe%BazFtAaI4&Ss+_iU?{93QckCX;lHtCJl-P<4U=fTFE%(N! zh>Azs{pBGXVO?SjZQ?+IeB8K1L_UbU$$Dmdmco6#xbTeoZBd9C8Q|0e25~23*EMK_i`D?ARp)f+${;);A7{h5z ziJT^LK*fi9zd#fhg10)>Q%JQ|dF%M*>b z()&BC@v)K`Lwaf*RVG~%6ykOAhAA*4_go|LYHpoY7kc(D6uE)7DiZ)-d&&*%A@&UPZ zm7_G3=^1yj1&LqoFIgt>a%CDj9mKIv;rY{38H;i_7eXFQW`()>v)QC-3fMJ%TYV0wxo==PeU8%N}&^EP%} zWulYeqfpu(@rpNbbP^;k`c#3AuJ5$rNRDW?ypfO;4rfz_YU;4T}`P97eoO(Xm zF;s{U*1d$#yD1qMYmQypEI!1_&!2y2{}J}`_tcfFtZbTX(A7<~+~vaf)cFEoE0FDU zll=Fb$%A7*^bEo&`|AuQ?*fACv0sU2xWD#gc9-l-2zxd znT3c9tr7mE&p}E1o}uH~7dImT_$qciHiB+Y$tbrhfjA4h^rfB+^UEQn3@iL-sPbA; z<|*0|UgKhe(+7A?FOF~QDt>6E z6||uF(gmA^L#^hdo)bb!CxYgQvH}*FV}<3LZLN7pfI@!euQSdC<7nX$gS}n__ZB{- z;WTOH-NNtgEoSsg2y!l~YdO%eEd#Wq?sp!a%#Pb!Hn zdAH;59Ph4m(;thD7o@$-fk#h=m5zSXwQjbn#6Dp-H~Ljas%hFhDik{&d@=p;-+r;_ zJPg=$J_FTWO)ku@0-JmW%5oV8cY%H!`zK?zaFxboudzH_{HunGHy*Oz+4?PY(^MO+ z4SbBs{R-HS_VC8evBNw#0a;qn@efe>tj|nF66>Sr_PPv})Hh8oM>PFttARPEVXx`1 zxM;k}$}jNM1bozgcXgaBb@-RQARTIKwsp?U-O8U2rpk**<~;C50>|Qa z5daM@Q(#R*S9#%stzeXCvW_=bV?F+7Vj*Spr`xKyckb4!)61D5CLIo+iLMl-PFYpp zd?@`8M_n*A*M>idd{ALpl1SCOn>#E0A2=L?k08@~JWju@+qe?9Bt1XJVCus#FnVFi z$YsA^9huwjyX_DOES2QboWEG!y^W^X@_rLpP)`_;dgk55!`o*A(LA3Ht0k5ye^9d9 z?+G;)dPzq_@#>}3b2qjn=;F7mf1acMg^3PHzX5|A^cZq>-OevT0oil*g=vx68IrAhyx;On5HaJ~?hPSsbE zq>6uO$lSG3u9eCgR7%%aH3=M=Gx2~m;?IF3*c(v72R3EDRr+`yy<{%cMu@+Z4HP4Ui1swD_OmYN zyHGS+(=~>+Sl$a)#?R5Dk9U3=zegEg^Hdx2>YK#_DzqY8abw&xSnT-<$cNZ&Wp)%eSLnV<`uPrlxyGzPIe-#HjBVyrQ9MxzLZx@q%gU>v3S-qp7zhTcD>YY- zW*xT5UY=9=$B6@D^t{7H{OM;|(0*9{*5q9TAwg-rVwCaS2O(bJ%u+p{(E9RxIG(o$ z$A>F9YfvW7QX94kJA7hKx{u#sgUWpqrvXFq>@l;Ema_&>uFi+EIZH6L4By5Y$-&rs zRhcCG1DDBPKr6(?+c_`i;Wzaz4w`+CAwp;=lA#6GgTi(w^#4S}Qp6S%00}GlavTH;JU0xJ9Nj!J%A` zwE$*WcaOwcDGj%`dSEy+1qg8lVXe_ z@7z<4oWy+%Nj)&J_xX6p7d&l|f1d-r&Puc0*!cJB`E054dF#tc6?uF>7acGIv_#zz zH3vrlb)usp9t=uHm$iRHfw(o?^U-8uM8J(qRA6));}=82P7Xmp>pZ|!XAqycgo_2SV zSV8)H%-hj^-j{xB3=HVp`~M5?A5nYR&cmSehT=4UjPkAs*r$OZ#ia92_S~qFz@Y#c z?Tc}AKP~Z=nS&iEsqis-g+WgX1G9Aq0R%nOf=_Iw21wRio3p!g5tQ~A2}S!+@7K0;f#3;)gXq**LdC5du9jfaJ!%#Xf zXh8reU~nAyl(?)HEm-pIy`={9@QX3lxg|rO z{F<0QU}Tk-AMKLB>H15#>U?>!ueDXMpJL;eV}%ZX z`i*Evn_V95y}S;WIKrM_vs}OIcxbxN0Xv?~|L2Ew!H8>l9q@W+2J7;zcZ?=Y&~le_Zud{l^nRHYYDTBL>EK|>RyNKv{7q4(Y)Dgi_SD7_;sNSEHb zfJm242na|Cq4%D%c%N_Y^X;?GpFPg6lQDp>U}UX1*If6!%XM9MB20lR2bnxGcQaJ{ zOB&~go)S&lCv10NDv*_DTrsy$Y}H}fk5qifKTYYWKB|5>0U-yLNuMhZuuMGDOD&`4 z>`lJ7{MBq1CpHel2%!ykv$It7Yr^<*nrr|+Kd6OtGy8fUU6Z`2 zZ+D!wz~g_j!>8=kP|hOnVY!k0xOTbj@x@J8FBG`OGmQKCVU~%Q6UOh*^y=c1b=qYF z#+=Y((%;{YUJeilW4St6+&1yJ!Rn7sy7q87v^sNWF9jt zLZA=;)}E6?VMwk#R2D$fwDOPE`++f9givdP*cEZkKit5b`nM=npFQ-QJ$p4tz?%Cx zMtbF08%&d}#hAdJmaDzKYUGuR1E;08-0LBxXh^bv;Y-TiAM+Qny$q%Xb9={z)ql^; z(yA_Wnp;bO*}XJ@zV)U*Jz63 z4%=(l)oF;9HtPrLCRaNKg-JRX@^drKK>(W?SY66+E`8I0>Q#9Vn+7E-yYmY{bw1s= z=g9A%$H{Eu{8eF4K@|mB73N25@iy=~7OLeiTxqJCRFoT;zZl1igFkncE+z6ZU(l;; z^)&BOD~#j!4RhQCeDGozIga4qF9V?gIW!Wi zQr-sI9Zvinnq?9fqta2%Pf5ngj;04zsblO+!iu;?p?ftDPNF7|{@melJEA5V(N{%p zj8}bISQK1;qrCTn=$Pt#%cKqtGAdS&GOsQY^W~1JmirP#@=XW`T-pGLF<|CgLuQ86 zUjcZ`TylWN)pu*C{)Z@vis7N=nf897h|3#P;_xGq$tTr7M z(a_iFKB0aNX1-O(6!MS&`ka+LM&WKd71`slxvIjrhke6= zw{0K_BsnWWyMXjd##~1Zc{k7r5A;fv6^W*-GedB7p)^b;RV^ipJ9fu5l4Zi}AO~od zjrqgH;EdJtXN>!zY!II~6qC#pj+Cs_pwSXH!6~+ zhfRL<^t#OoHD(gOrS+@U`jeRr%NI{aomyAt{41x`BxP_rfQQ6Le?5`b^tzKajednH zlp$s>3%hkZ!OI8Uo^)Nd#Qf{SCX-&HzMUSB4B3u$kWQg}sH1@w>na~kfGb(R@zUaz z^mmzpu~>1^SaDrPdH^UyOD}Toms;%l*_+CZU{qVux}=bL;k0fgDBUx}brhmh0TxR4 z>enqxko%&4vvQ-q=r}}P@2txocbKxO9O<`7WKs8Sm@Zn5KT$4VXtVxeAwTUb+%{8C zKwx@$Wcsfc@I=2i1L{t$H!C7*S-1;J4(pK z2hB4S5wmJD^=%Zd@S5M*MkTx+E#3)O`;5RN{$rlmiJh9-Cag33D)5EjH9!K>X=+Hj6e8M4 z=O;=;)b|zo=r^hed|Pu7q7idmONgPX@q5Xii=xx-r1i916Nngim~e*%X)7I0mBGcD zOBs2^Y)KDO6H^<7P*0i^8%t7U8rYV?fn%NjIa;r z$pm>CjwCzDN}fUq)w^G+{1AZ4OIxls08wk)SCH|R<~ZBX{4&CJ+ylS_ON_b$l*CmF z5PgU~@h^9rN6=c6qt+RRCjSch1-7T#7cBFui=PJ{wF4Xi1p3=MPXwVH3RJlo@qm2rb- z*`i1#m?Hn(POp+THgAzhl6ka3qiWLdGS7f#?r;06y5rc`Z-*N(LEe72mrgG!v{Br+ zQT=1;+=3xlSHU<>*PYQn$e9zIwkz{uA;eKj<^X45cf(-dsZ=;@J zfD{;~72)~C2$tw5xx)Pd)$k5fsA0CUw%m}%Z*o4@_eJ)DLpe|2|_wyY+<41rY{Q0}w@pGDd#o(JagDb8S|;Dv`%hI5Mdwt$HI^xT@QX6eq}gYMvqDYO^Fl>w{;A!5cdmH- zB+>s4Ratwgzg<2k1M2J><6EYNkw)i80h39fQjsEJFFiz+T|ZxdAfNP1 zX(?N3Dd?B=3A`M@~krk*~B6fkhM6<4z-AyfT^7YD@Y1_PeAFCHtc8e8#gvJ_sXKg}=&@?lRmWZ4UL@eQSofM(sBdN=mfCM}mjx)<3D{}0p} zFrY%5;}+d>ux@roTjrC>M$y7{Qb)yTrQwTtM}A+oTsG0DlRP&G66xu`Z~ghW+EGei z_Az0B(HT!xNC!`%_4I6u3^A1&_>oVIEMIq$E#hT1&Rtazkfgc~L%8A=Zv&PN=xb4E zpl!F=8-ozwxDWd_c?ZZK&#UbZJOe!g86(L?aVDyPG@lM4Eut#b<{)6BZ|Z~jdBFxGI# zua|l#Im8O8CRtLVs!K?30^9S)7H6H9l^_>2aJGj=nYs$XI`V1C4Ok&Vilfdz`y&ny ze!5R7<9CHMoxOpz9u{MZVgu*VKkkZ*Ixhv{FD25(Iqa-N-3?D#Qd!iXM@h*-hrAD^ z3YfgRb7?J9VTuDgq!YPV?W5#H>!GYQ}b1^fH@tKvY#Zv zhU6^5@x1`50VTmV`nWyVYupStBf(MUK}3f4WtK=6otG3?JSmsyb)oI1N#`(N_OyRw ztx_S$&wB8~E4wExblxv|0Ja8omGyQ>(u_#2BeM-k?{u#}@OUmg92p zjV%)8(9Jb6YHMY`dt;N3nLEW!V$wrz7 z@YKXrp1ct@aHZNRW*$2RYHy%BSr^YSEv!w>_7F*f<0eDK5ZAxlWPI zalTe)TK*(%*XQjBdQ`@s4fO-V=hf482q2$DK>SPihCJ-~GjR2O7=}58e^f(dW}nsN zEAD3sRs7Fkx~@7bvH8kVRX$iL5RklJ6;&}P?iMQUuwLy{p4U^L6v074IFf2+3L%4%ySOgy; zKHrxn4P=cGzq{@UGygS}i$xV!GB$G;9n(}by^zwb`i!!TPzc|9EMXAal|swA8?Gv7 z_{lzqQPGvLC#^??ahEMY+JNB3NGC`MNzwBZBhe}7sdKQ7F5Y8+6=kLtcCTG#l5jBp#>z zk(yMnw2M)`p2Lgx^*&WOWUKWB`g(<)ktPxs| z06hqR16YTo`r<;tsY2P~%Bc9BQ{j;hP8RHLy*^eg+|w47H;oFV=&UKR{w=*zA6HR* z`%yO6%ta(WY@!BA-uN=e_Q^sT@$`oiQk@Q-7ef|tn`>Yi*bf-WMoDNN(482`{$&X* zWK+4ANt)dYbJ+ugZ#}xZe>`ZF1Wxz6u){Z~P421-&6(d}A~qLv{I`A{BzKi@&HhnR z?%{~&qG{f_4JdV^PB*O6FUQN^l_Hzhb!jXGoZUN$^EueQntqjcQ}f6F0!Lv4DUm2j z5Uq|X-H!k=+(w1k0IT(Obk1jr4p1Ios0F&EMqh48>SzLOwC`)Xu-5(Q{gD1@(a=zH zPBr`eby#cQH4_Jj-in zGbO2ey#$(8)r-6_Q}gq+>)+#BFUBpm1I~>)xi0oiua5q5%9Q^*THF5(c|!BN=O1Z5 zT9{5J6J5^QLV=ww8QS>NM^js+7gqI9`402FNvr1D+?#KqUTi!Z36`E-`*!SUPkvEj zX6CXeL*rdin9?x>z?Spg6o=6!UNLV1qC9yxKWRDieL}~+pg6VhlSbjzD_%&L@7AaJP1NNul{6d2%K?gu0sw?19$G)?B%(~$go!|8`$+;kEjqF;-so+^s3|6q7Y z13Tx~r^1>36ns8W%MXw{bf1{IEO={(U)4o*AeH|!;w?;wOhk3HKAM3b<-N8!UH?O~ z87Q6JB(=u$YH$oRp?wdGVf&It75O{;#?5hIJA-mFY<<&8LsDCa%_FssD-DLCGz`=t zYQMaYYv#ygEAthlCq+^wYTjt{pnQ9(spm;{koC}J#96vti%XXi1M%sFCj25e)v0B9 ztVd?o)>@7M7SJx%IuT<9*!X`$2IANVR+xbLx4B@v>+|lB4t>NOU3w>KImzr7+g$X8G`wj=E0%LWHZx|@s)=D&UeY0|QOgNg- zTvFmb|B9BA>J zZ27yu@(1$PwB%NzoeC{6_-m$QCZr=pi6{N&Z)7LoC>?I$;(4Pgjif_&j9qnw?M*WI z2cg5S(oJcptWx|WmyXE(>J?jHH!^@kpkT<2W%_5HjaGA=OATuO+@^{JJPsb|^UA-w zP5XoBR+}5F!lI&HSEsAn^X~2M2WyZSA4W~TH&iNhk9+8!NzX%+qIxAOt-J?mH{9F} ztY;_)#bSF&NFevuPCj@(B=e$7@*Pg@=y113;$wU5G7Q^6t{o5-6*vti?|*!+*lSdei&+4N?!lsr?T7#*VQ{`bOkC9`Uwo zGoXX^@GqeHt&?u63VI~u`Bb1A@mWo@(fk4RhjUN;UPx<_nH3??DEI3T2U03Q9kzD@ zT1kTT4@dpoJU60#W-1gqinstneV5$E`({`&@?CrN)YE*4DO(f|^iD>(4)GC1mp&|4 zD5r$}xu?}VRg`JlJ1@2GoJxGDhAp!jIL-t`u3$8J|0WDdSI$#eCf95p@}+Olfd3Y& zuJ?KR;F5^-yO&I`Z*2fV`)#{#l@7&qrOO14X*DBqd0x(w!`agpZ#bcCYjoVfwgPcJ zAL#%AKpFLv+QoL#6fb15WPy|cwpd%$k;ObY4fM~}Ii+^7tR3Q|6`ChDHiWOzxh=)% zu%i^xjT)seqaQs_nr2 zd0HM0-ZvPVET07K?%%r7P4OI))W|`;W&cM}Ssa}?VRWOTbUBqTwzsfr3`u<<&#m{K zq%+J2+n@-uQz*+PKX?Hp9|CtVxf?f0c-{rZtJHH7#pkNTeJmssp=0J%bu|rZ_ETwS zB@){>1V;u|q???5QGNwcZEE<~u~K{Ru!o1c(!%y^XI#U2vc=?M$D_RT*Gzfka;RHu zCBP?tf7b@OX>$|l`ed00$H@p_xYl!YUdAqHz5Ga7Ebu#3>@~lNckarVienpXKJjhX z@)NBxJyAzBJCSd_-|-L*gk8-NDPJ#(SDhBu#%RdT`)BdNkDZajQdt~;JN@qVZEBg| zBZvoY;_p4tA*sU8q9jZcep{TX-|g+w>13?gnZS!P-Lf*6?@Z0ZV7J?^*MtQPI;uFT z_rEG9Sy@yRUmt89ww)T3l`*NUm|}cxleu+dZ$XImp=Z$qN-sz0FzhUm%+EcAfc@!=LE11pz+boeQ( z%Z=3DJV8xpS8*X@B%RVucoNI6==IeLyXzGgH(Nd;wO#vbz8TbuIi8g=;0ZcP(9zbQ7#3PFl_(sjX@yquP=ZUDTzGWRtmN1>Fwz_QzZfVfU2bgWa$by z`26Nd6mAmuXUFLig2mT9-e7<=nSB%0CIM7DOg+mWP{|zbGo8Nkv@WI;7`Z$eZ?LU4 z8NXDqH8(rbBhdH6S)V2T1<2~-z@3RvY=6ZAY-iM4nZPlxq#8W#X_A#&TwX!vUc}(R zr{jt4?FZ8Dh zc#2-O2PzC}AO$9C8uli>yr}5m8KVA8*q+oT-Z}Rcai<%lk#HoD&p2IP*Q3&yV77K1 z_*8vzQ}@>jw$a)4eRuL>QLnh!uk+7Rk0O!}pz}ECiFw;F}kj7aOa}%z+m=LqSIc`57O`%}HYJ z{zE`&qJBXPDoSf-#HJ*8E@cQCjugF!r%S*4X&htyAlSr~J%h z1NW$_MONDxkx)_87)d7sQ#fw4e~~~6TZ%h4n0cW$+;W3AadH()TIFQ*h`rq3r_frO zr;H3}*zrzN;YIpsE@#GRChpg=>XYTGO(&qpQ+01v@W!S!tzTjm6|(8& z+XRL3ckj%+8kC3{)5c%SvSS?N1RoFNeh~nxnBONP$@*ppG*$NM_ZoeXkWVTn*zDul zlJDI@ibGX6J;C|EN=L+fn_`J2he+Su*trUvVbzFfy*PBXmOWLOb|7T+*z}ytbh&Vi zgJv56w!j|#4`Gk1^bmTR{jN#$qY#kz$LX5sM6yM3zIYpQ^eg^?+b?hUdDFAoD&v0H z1p8K3#jcF*%N5M1S}rQS_yGC-gdORs3*IG9@ZX9`PWUoN8yv9~led!1{`!f~cK-KS zK2t_sdru*w+^eIh%(#DKDbB;0YCTYvBr`CwdB=!S+j#(6jCMMoZ=Bn$VQ^~NU2Sdp z+eq_f_(BwL}h59lZQ6E^IMCGaPmMK>MI!<##x6x!-j9GKiQ|*F*8r|Hw|At zbD&3l2{2^Ys7g3qE7Cji_I|K(c^~WG_2>I5Y5rY>*YY%y#IWl zwKTT4EO5!_8ika3k;^IE8VM|Qm(3wir)v+BcQ1QlzrPN_LquIv;MAf9h8mRhaPirX z$9!l^>FxIv=Ig?V=0${Q;}N5QCVsIi&w)focXGfpqd*V!(fH?VtB*(&DaDpNh!{JB ztx{fVnmQc{Z2fvw041{3pf*GboaWHUu-&M2<=DAHol! zO?v32TwiQX%31!$Cw&|xe({U@>(Mm`2q>Mlbu5Y>nMDi)i1Lkhj1)Y5a{Gq-&Gw(= z`g()X=B_Kbj~hj1X8R?Qp{(&5bfF8HNTNJ&uf&IUHvmt1TzEho-F(Re7*tbVasT@g zUsfJX3Vd`kxkcPsWx&=FbG>X^-=baSPI(SqYWxpc;vRQ%E&QYrwIxMW#prh?gDu@z$q@reA)kw3#0_ox`hlpX?$o;1m?3Tl>xOx>wLO>3#UrPbJv65JyEZd1y zQk~#niw<<ROjA|Ah1?{ytLM3r(28wEy(9(BtcOc z#B@2_IrmmZVwiuYPBOl*x1{6$Maz-!dTm&b(aBs%P{Q4VjY~tJdY#pc0#G7keH>qN zm&wq(O!WJdeoM#NnxI4DGz=gpBk6?5N554fELi#!x<`m-=b3$BkB&&=qX7~i>*%Iv z8lOdpLN=xZP+pSiRZ_xx3pb&f@uQj^T4*_7!;k!)F=zqY znG*eE!|H}%w0#nX(`JYbQYg0~Lb%BO_a;sJpvyYxmoddfmPY{P0wExpmfWKJ{5g~_wE-4T^xJ-;k8)O)XyggpA(PPcC=S>e6=I!x3wCvk7=6l6;VqyZTou_=o zeb<$pY|#EafCmB$mI-H?1HX~e(NW`cnv+`WhswN?4)Ii6Rl3Ij_@hIaoa?y-<;Nd7 z+F7@chyv!`ez@bg7v{<1D_D_$u3t{TKD>3^g0gB5UROkVvv+7@8^YA}Uv*a!T#xxhinJ7l1bWaM< z0#^G}`gsu%;n8ugQRtgfcyKmlxHi$yMi5k3%~Ps!>KBi|UC!@g6e*Z=Rm)V}_kiMV zn;J6(4sIjK|}|pZ`OM6fol$Pt%>; zWqL8Bpp=M<5I!wsrj4X*^SMyrMoiaWAJ{vHj>KtAZE_ z6a?_E6Yl(D-nj5r0nqb&vYh)y56bwl{bm-4PIf(5CSG9gc4dxGwL^WgMZv2z2$Oe~ zBJnqYKk{JhQ+-(4zOHGuRPz_Hj?Qqlq(xT~71y>%QDDnKCE29JPnJkm`g!C}ZivtQ zEPiO%UBP(b1sJWGB&!j3SY*zBSD%=#c=W%hPuzx3A11#$uSQZOcaE$C-j>5}si@nqyVzgp3o{8&(=Zmczzt z9EzRhbTLg?HICe5@*V_)Bp=D`k~B1(t+1cQ7TK}vebvT;&-0Q?M{kS%tmIY~0}W8P zA!4zS>pebQM3)6bxY%QGy$M_U4H^hn2xv5UWor6?Q_sTdt&h5^Uy|}XxnF>Tw{HV1 z+(b-1TUMCUa>M_qrWvb-liPL2taa1fVMPk`RWSC9cru~b-Wz-mvtZMHG@`O-W2O8l zZe3-X>Q%JfCD+2Ih>*tniIl}!i~2m4tvC zo~xTx>WN1vs76hsKbo^cJ!h_))qt21T(1931tzTO5PSG{z_PiZozMO2=%8f(Se)fL zAOT54?KS$1Fp^hlT<+Vd)bOwPj{S8bvB_E|u=5O;_^R3W&&mwnO6`FNm}ijB$my62 zMMzE2(~WY}PjE{4+m_5cO-nmbV%*RvtP`xC@vy_SwQ4Za5T}mWPOSa-!|1y`1B4^a zVC*MKQm--Oaeu!XccU%^gH8{GM{OpP1?ZC3_8+B#$iF7+qP=PmHoY> zK2?ANK~E^8W`*vrcqLh)ZD2oQ?mx2Vq3PU^pNRE~;I0^mPzn}NpAPME5lb_@}_?#bvPK_aC_W3J4o6vebud= zJ00zQx|~>xY6pysRdMrocYphqKi)y|EH0+wZ@pBr#G_IJ?uL-x?fgjwl>>cvNT2+v z+s>ite)Rp$a5w$pPyuK39^oeeR<5PB)2fkpA-}!X-iC)(DtwMJ36=&ZTl$T*`eYg6 zytp_)k`G)Y*M+!48yl$%+Fd23i;;z|`UjldXai#;IW#$$Tc7E6e*Wav`ZG+R68_z0 z1Im#BWtmjMmy2Oj1+xOQdb2gXGP*zLexzz~KLSF1ClOlc z%TJ?z2X!k|QpD5F=EqpFCAm(1Y54EZjh_{zO(W^!{*Awha~nFO?JOkRjcc`t4L|}| zNiDvj-x3)EBW}~B39lH7#8=GPI-e`tAd~N^k2BARh%!?Hbs+^tb3?n-f|IEQ)CX#` zo&@XIHa4O+*F_r|q|S)QT5_)d^&Vldcvl@d#OLi}-F$7==BKVSmNyfGY}s{GLsf4* z+PzJtuqgKWIk&{neh-9$R;Poca5~9QS15Vqb;gKp8q5s#a?u~LN^*<&1F+Q;2!Y-7Za(_4&i^VRQOWOM0tDV!~PiR<>7bV@!J6Sntly>;meGHg-?I zi5I=QZ{4JMG|s4~!nb=NY1?0j)I3ZonHnocc|7!)OkQV=uCGCenbInfY#T=ENG7tr zoBx9J>pDHG*z-GsWFSaR@B5aE4F>Oilck&p>o|MDb>Is@);YjK%S zw0#}$P5Ce&yI&!t!HhoC)^Jcd6Ht_DYx*!o_2;2QN~rDD+81I#jd9#hw@|T5_od5U zMEZRmV=@D-Vue3SH;DLhKdU}x%cCT_>;ZHqMJD9l0&u*zw@NB7+N>{qA7hwu?jp@1 zDTCQKmQIDi76!^n+dq?;45SikNJ`3|FOc7D66mg3ni`C1`TEEB&Vu@bPPaQWck70; zKE(Ns+1FP^_At6bcknvMjQ5bw2@uW#HbtKar*kVO>dY`yp8fyCet_+(AP1;ZA1~ zMF(V@tw!7p(zoOx_QI0^`=)QRLK_nr&mLzMu%m1bfHGYGP$uJFdg2qHGq_fKxfVPF z@r5o_>u33px|~uF*F{p;7Z8lDTl%99Z66o)Gqw(e(DILz@5!?PGOUO&P{A21$kOV2 zn9xp=ZU`J$Pw5ve7%-1eN+@7xaH4InT?@^^(o2kGhsaQ_C+sv#<~wwsy2iEc6zN|Pwka+p2L`AqcwFC**N&vrGE*kqKIfr6MwUpA^s zEjF2>+F`);+ol_u5(U#o0p~DVM`bZ^$Z}15lBNf|NUm|Ufpu+R_2Or|3^T5@VJ**| zNEYtYJpr0N06d16^NTDNg^k@GvC`z>^FN1GlU0)BMIr=K?LkF-P$_-L zsz=EnOE#~2qrW|`6>!9ooXk7OC3E=!x06x-;adWtdL>!*PM=Od&L*AFQ$vx|~fH>z~BLZ@unMCIfQ8gk?;?`iO9a>?B1VA`aDf}lSt zO!~YSjDT*xY-WzS3RKcOUC*@GYM$Fk|0YL8W;Dlq_^Hr)iz*ud1)2S(eFOOjOoWxv zCp*4TlfwFbm*k9*ent_g%GZg@76PFG)4O* zf4#N(PYiSY%UdP#|KA<_k>b5pX3 zYJBAz8GiGq7VHB>>mt6(-m2)RXp-ioJZmyX%-D?5f9}=0o7(7qLdjZo6}Nq`({Gv^ z-21w#$(pvNt*icwh1$Y5`e?kohQPfTqWP0-J=vu(6TJVS5KAKqC#J=6Nv3wbiA8gk zOb@?sx9qj=1m53rX%dcKigrF}<;0)m&pTesjHB@%^qR39{BTv(=@g*p0YOr@_Id*T z4gQk2#p`NSc6YNZXqN?kD$m#h%0fF`ZL--nZmfemd5Sb=umYe(D=2; zbT*n3NaD&4lypL*SWh3wUCtHFU$s+bCJK1{5a1jP#9Lps>d7u(ho(=tm3-th{FbWW znu{lERGQP#t6Q<+QKe_&U1jiNWpExSYdZs2MCc8sxrc2LDqv7S1 z`DCmIweu!^hnYb}j4ZefY~XV0y!Y6h=x3n7#mzF=!*FX8_+cg?HLpLWQEo2*e>}z? zu$mbBA|aZ@@f;hnTGEKm@DTMrB7U)dGvWB?L_7%0a_AhS! z^X1{A5~nAJ7c|Y%u>so-Y@GEt=nH1MZ*8rd&L(` zCl`nglRFQdf3qJ0oXI{j2tT#s1*C4LJ>i#^MRR9YldY>cQe<3t7v0u!Q?)kON%&b~ zZNQ(=M3aDpS?hZP9@(Y=+qgr+!@l{(i&VMBU*C%}16R30Su{<1>OC?mu9|^^5iM7v z+|~h}DX+_3cPobt;un?!Z$?;4UWJ%m{EZz9nmaD7JFA?(>J25@Zam^|MQ@6e&3c|w zC}+OgjWYXOiQl$Fs#8%jyY}C~ZMRBA4h2l(nYuH) z6FvVJQMY?$Ps4A}5{RQl4A=0co)Ho<$T`n9`0iFb;63HMMNkEmx=ENfw*Tg6X7BIc zYIw8V){x(>Gy*$9o1;LLu6;P z%x2Z6vub>DJy!i5*^@2% zkGHS>CTb@mH*+-tJ9I=#0$sNF2aa(={#e}Ibd1_4N=9~78Whl@F3>G0=|6b75}mo1 zhOb3G3$1aY*RKdDTQ#h5Q0$0RA_vL`UwN~++>E_Z%0$$>*tHEM(BUE3M4=^O*naL} z@Nu*(HPFyzde5z3YD!(z_lH)`#-VYlSgC21g+WUW6|2X_9ZtaFM0dEX=0v5Y(6XTG zoFd5=do^YJU3MuEheyC?zCsL#dXh1FZrlkqdy9;gS@;{tFu`Zo;NWvE ziU1ySqbND-Se9etJF~u+JbygLKLm_iGZO4~SS<{X`sNNwBlu7DRPyC)qcnRTNYcx# zrDP1Q(97AUWc;`TwF$6TyJ~bgzvR2~!E0-De6kgt)g4-!uT6)Z-t9S?iq%LxLMC3F zboBAH*sPXazDR&CbYu>BWNW)FRxneF=pASlC+eMWTv;Gh+8J@OIh*t>=b4&BBbG! z*vAevVMF1jhp8|0n0&`{GIkjepG&uG0H&M(&BL9*J}-I(-o7Il{?tC=_Rfd2Gd0V1 zG=tU$#Ed!f7-w1z>4r{Q1Fu#AcSG0lTFx&&&2{B^$?dV29{jbIfzL3FJwBgpym~e3 zepzm{R_4>^pLsM(v+c{Mgmp54bh{_tJ3LrzJwWD5?=R@AovCpyre)&8kD2}VY**oH zZhc-(t-B4|@MZT(mVik;Wo&063`Mh7wc2tr)|2VkGaz=cje^5BO8EU(m4{>&s(Q9h z8;;JhS6Tr7?;QtreJgUwrVn*5f~A3FNTT(OAMUd^no++xn4yPXa^{B*q|T3wOq+lc zTh0gekFmy4vGg+yo~s5vF?CLMlT1!BbIGgh5Nz z^T+g^kAou;9gJJ4S#uebl=Y5Nh_=+N1*POE0H*5OXJckQ<7io*&ep32+s&JirXnWt zAa>W;6Hm7QkFBuvErzL@-GzUMW0xLRB5q-tZyp-vY0tSPUasKqzC(DYY4=(OKDj!J z(?L9aM0~!CmaSWpxWE}O_&V2{R-M7eo4!>NEe*EOWpRaZpOTFi!U1!2K6v^oHeu$yjT^8hUo0zmhtA6UiB#I=JPj39K1)bHj;(e7URR)_RA_6rx40l$5lSP0LnL zfa&C+jQ5d@=47Np>DRc1{Td^nk4DT!!%cQwRXVwY%im8ycRHVft36IXIU0$nKj5T* zde8dl;QyEhu7U`mwHS}iLwtiF*mBxSGB&xOo%=PD>yQaPVo6`scXG3q4=DFRZAN02 zYcT=x__jxg;kG}EM$Q*Qi+SJQ-FO%L>jQ|^@0@5TsJ*ZBqKk~7Rnkc^zE|4k7*7rb zW(lyh4fQPR{gtLW^Y(bJ$rAi>nw-xN6AK(&G%@SHswTxR7vMSn=eOzQj!|B3xXAI` z>5_ARmg=sJMqKZ5GODZ@&zHegd$npxHs^0md%HzX8H5d$6%gGRFRF=6=_^V-I_X4< zwr*GR*B!%aT?VW0I}cKcnzr~mHdpW6ZqBT(7g1vd*ana4@M?%$amod3R78G~bTFh8HkLZ(Vlz`LE_mrD#$07So@ITaUWBvpe6XlioNhQXizq~U)Gy4)=6yE;R^=*u&= z*4%b~bFki%?kNgg8{H1P1n0|PH)vY^Vf{N^BsE&75fvS<&7En4|I<7ju&{ympU$xV zD0k9LBZIXavXyof8wFN+D}G^8?$n7=u_xW_tZ)8ejz944=JM~qV(_b;^R2o=z(c-| z`uoB33JVvz^dHjmo^%v@!c1FSoM@R%Hy79KZhAH|Z*sO|K<#XdkCrnm_1{~{nEe&{UjY}zEfS)v z%8W~BkILo3UZ(o_+K;uJ2Xg0EXxRfg=%4lLh>_r3^EoG8g~8{`*0FooHDJ*_pAW|9loe04of?(YZ%jnO|G$+hh7uNjcQt z?|cUH$?CnH_n!~{^+nG_mk}6-$9VM(!S`~Awi;`-`6NpHXLmW9SME8>t|!4#$1?U? zcc#(K&8NS)@nc89fXKjo2s;2rCa_7pKFZ|m=g1K&u{z6ddeLVJUldx^P*-o+yjM2! zJ;($W7Tto!z)vvFX!LVH?Yy-9-vAd9!Ohoffvs3xP4j=VG}#$#9JmM5@W13X4OqoH z8MT({7=T=T}}9sxXMYX5#uCAVtiGgrx*$f+|psWJUcb5 zTM0#f>gomJ%Q;>#jDAU6qyPDvgP6qDljSnIjne~8*`sRbfWEc0pY!-V&iUggx!u3f zfoJYaOzEJk@ZIvE%>8h*b3Hg-RSu7DwwQFsz_(>+0&({C%yn{6IeC40O^4~*GRsw? z6i|kVGJh;y23ItHHdx!b`=GA|?$-ODlJORS%W&|&_Yj@7`-9VQu4>`&w*O?{@horN z?Bn&$({-nl}FZG~aqw?R)_YrRgXIE_{@Wyxdr1 z;WMdOENp$_j&;VLv;`httu@k!GE1&5vCP#4VJ7)6t`z0YZ1arIR^_fHrkE8+8t2al zF_4za;7sFy-m+-Pl{&KK)y*YX4u~~U05N>r>fYXGdU+O|>AuB7bOTWI2c$Iqb1zq$ z|0u?*o-VQk9H$iFSL(LPdooWDoav_r*0MbnJcySZsqz5zE4|l>0=j+q2MDGwZw7<1 z6kjA!pW}98*dkY^yl>u^7UG`)UWlxoSU(&V)F;{Mot&o zG;-$!EP>nE&Vk2aeFVd@d@yq~S*_Yo z5#%^i2Uz2?Z!mhaKzC8Jq~E~JL-ALxo$z6H?xH`3S~g* z1~F)IxDB-#O1hDKo8aF^bf&+exJM4#4}7U1h8=5X4;GG__ft(}wo_>?-%bX`u=ySv zy%~~#&xgsiSYQQzO%css)|hxhNcJg+UfpH=d(I5~31JEPS+UVgW+dk9hh{Jh{N0-$ zbQH1b?d7Qg&OD4$E-v-L=q`l;SMYgS$m*TV4T68uXv0deu~Z3`Qc;!eeJarshbq>x zhoQJKwBwNfcD+4jMr6M6gr+Cc#|`$v)~>L&41TmE=67^x=(w6-STb|6u-=MSbh$d+ zZ?U2!gxXm6*vOH(?M#bYpWz*${sB;&BdAg?7eC#|mtI(sY5cWRL;H_?v{gT&3(4h z32(T^#AwFI)B-?S8kNIeO?o%&iX>RV*WFDUu&sFCt7+@Ylk@r0vDKD?-+C=pK0a%k z&OV15`6si9Ee$@szsZ_#CEG!LxJLi8jNRhZYi6&t^lopN&j7CV@^Ihups8}ZVd=1 zVn`kM;l23Ai{GXq=isj*vWL6TLA$d<11&>9Y#-N(KP>ydhr&|aRER|^uHYpzb_!GfIEMqheQ zc2!IsET1|B&-6uiL?H?vHt*=@EO5yF(u;7g*%E(ebe?J#TV_w#$J5T0wqf>$)thHO84Z&v2_@z^2SiHG zvb%RS8YJ;v`udx{d0d+;^hi&#jZBTWT~Wp;4w zzK(LS&eQE_9@7?j87j@u&vyE?thvLVc6iJI)syRT<7n{`$&D^bjQg%mKqR;4d?n^E zw-gOz3_;37eE?rmQNAvQl`#?9aex){s3UbMJV#1u&r9M7OnaT`X#4S!9Idg>hbdb% z!vmOLs*3&8*qXyQ8=KMZKkdFmR^!9-UiW2PfX$)QGwU$zPp4rX?%uzz)+ep2M^ z;fy3J_og&+4z^9ilsWFLJ^&Wlm9*D@anwOg$BzGV!P=7Yp9w8J(lCxxldGANEwQwq z;IZ_D`1b8U)uvHeyz_z9_|8tf1g-9R&WKGVu0fLE8m2q3a%7!E91m)IpA#{YI1hTX9Uk|6PCY;@C9^4QUAmILip_DYZ1HwYv*;K3`j1=2ql4!R1bZs&=JmpT*2_n1 z9QMPsZp(Yk%IZ}RcC@BhB<7-~B{uUFwsE_9`>~u}+wZV?#1YI;iB6W9-d@#w)ggiD z?qPI?`7Q-4Ea~bX{gL;q6^0l}SxutO^IC=Mpup-03Mku8u=>$*J z97Y(w4V#v^W5*z2qUN_6#qAAfd*b}r;_xl9D9=e3^lo8-Zk6?_rgtZ;1c}bOlZoVp zGwC37KgJMgkmO-IRiMTqPqZNIcVO`oQjsJO2xFY)eI?dw6g+nv(B(^%(a+|(^>qHB zq93jyJj9|p0(9}+7P*HVrRc4(79K($6N=>S-KmN`TumU__C3ZBh99oG~F!lq3;i=Ft;A;0Z*r z0eX8AzLJ}z7P+ERjW=K+DD1T7tZpy1oSmk-n}=HSSSuz~%$X^m+`@FnbmO+l9Z2pitEB4L9TFb3 zvnIlwxk8%WqcdIzx%m9_Z+VsPf3+4@I#~^9SBeg-X^6sH)wL%oD~}!{?~C|O?l!Er zc-rqF1?-29jfWT`acSS@F+G2%UXn4MeqPk|UmYNU+O>mjidAilzz5OZm zl%C%Ft8x4MOq@>GC*j%a<+6pnRk{H*trVxv4Hc5s^Dmif<8bpGv0UD!U75vk)r3^s zvNu(hdLD<`x4+Gf-4_bGalAH~KkW<+NAGtqjHt~zJKy6cG)t_L zl5%vCO}kjzw6v>ix%a)yEBckz+#PbCBUo5iZr$Qd;_T4-dVk1%7D)Y;&;EXXrah@~ zRNtP_vH2i3_6tkR@04pZUZ8uWug@ITWNEo+IbM;Z;~Rvrde)83iPH&;$wm4dKXFEL zuAM*}s~j90j2*}$jptNmM4SEm=Y&?NRYKQD)VF%M*d{|hgG*ufpde~a^?csa5NVd1 zvf=xCU9#kMOEVkHk3R@m_G)m1i-^#ekL8Ya-SG9M3XByPDJQ-0B;qS7D&(|^$j_bL z$1f%o)IM`(sI5gB4G%wg@?>~;m`k%*!fWZ{W@Sus3dQ`>2oBZSJ~=6^$`=6aw!3tYJ-t1?QIg-~0LmT@zPJM^$@M0r9;m>RiBE zDlIL2*M9GLo&;*9DNJ2KA_Qp1Wgv3D$)_=GRP;_uK+wd0oeU{Z2JbeNtbKqZfEhoB^D}9QrXq zz?DR-rfp9Ub;S=3Zd?+qq$Uelv#$$b!2*#zSExE>)tl^3_3N-F?RZd7W<7qN;+mv0 z%f453wmqlK!qUiOAn8#xK%VpJAj%qK3^;9OlH?} zkzo!I=xl#@Wq2PW90WyBi`LPnko>5&Fd4 zS=Jui24gPGcu|)UyLWEk9O`yXS*mH))hrZyhxSL-ksv^7Djd8GWA zLv(C31>!iKU|?WCy4ESY@o9W~yq%l+Dd3F#so&f>l{yu+KZjEOFeh!`e|bHHidCVXN_PYO5kvngX2I z7400iz$byRF*D8K{|L5lwrJ=SrO89tLFAp+)wc9#7+ZiUvDrPlCHloXyy7u-O z_h4NEslY2j`YQ2yC>l{Rme3o=-#uc!co{>-5dUaY*uBM@%0D*R_t{VZyflp}r* zY$;cZRBEA~r(5orZ&2A^INzSPsJ2`gNLrONlWev-UOJYho3zkb<|1dTCLj^^o!OQ#O*RKNCfC!4n+ZJ6FZIcgu zYeu5<)s2nkkmNR4h33lU7w29?=fsUI50E57#CdtaM4pgu>`f-I%Ei9W8X-rBi8EOw zm-zWLrl+Ud&&H9ii8wCjj>QKvp^4ZiciEOv35f(%7fBDSqgI4kxn~tghxPjP>&(m` zMS1ng?4{}YQXdH|l2A=zz$le^d3H1e$4j_wuC2yu7B8nBK1qq!=1#cl{4A~lx6BqJ zmAHm`+WMf&*Z==PE{j$2l{ zN`IC8DkYQ1t;m)HaZk6IiMP;2rx7bFtMHbdj*i`Xu)?S?C`f)YZh2wJ6TglS6BFYv zc@(&d#gteM@9pHo+Lw#*-dbN>9@w|ELLWT`H#fK0W>MgK1;xbh?;-~h|2kCi_+Tt( z+0DW`(UKatoRpL#>F!QRCt$lAA%`T76?0{jbi2aDM4H;)BDn818y1j}c%rpOvgwpf zp2c0;7Q7`PFEtpBK;K2H>;j1~rMI+y>R}Womrl+QM)`rGXl-*}zJBNo;A2ofFrQu4~ zD7WWUmJ1quQLO*MP-Q0Jp>Gq34S{$twj3Vyh}mrT+blRB^xmP={z6Jg%_sUwX={)A zY|V>}V&N)2zP=50b*(XXtbTm`=@W2yv8R-jP1l2t8%u1TaxK)Wex19fn9wo4Ow>)R zB24WUss)Cfw|MrhH}6i^>S%@Q+~?EuF9hFQp7le>fU}T|$onE}va^{IutjUD{oORf zryO=E*r_@Jen_+ErS95jx%D7YE7geK!6Bezb$GrhR)IT`t;e-R zEk`{^!+E}a^=6B}mXsQ~;@A5>yA0S`6GTULw@hx_m^K$Q-&vLX0b_ckzIR*b)zplc zGMk|F^StMY$m)Y_hJ*S0{%ZXUvM*m*-$8YKGLogS(I^y&jDGR%WMJj)3YYg4F~_}a z_Av8-Y_(+RE*a&t*r=$FOrqwCAJF`TC7}lO?a}uN<+f76ubtngXpJuYG1@vR5NVdu z9PYlmFr21GS`&9W#~@+5)6+erE|frn)(}p9Vo>JOQ- z*y80l9S)CO+*AD@KYmiocdYHWrvWnRHfqk+tds{}^0=iR!j&g!fq{np(@1lsCyYev zVt%`K_Eb*)6ArfAg^`fdAg58RzYh*KYasoe&BmvZ?)PtYHdYB+v}8I8lO>Eby(v<-#8d6WuD|T1V>y<`*&(B@EqRdu9ZYiNwjor@$^ z6L$hFc;Y1k1N{jHg3=t}?O!T8`AYk41fSX>#?a7RNX*F*H^&=BehAkSLkW3KMVSj) zh`VnN&Tq|MFkn;3P`km(YFs~K7c9o8p*3=uU6+|v#BEC89CqV6&V?e-P(4OMI?{qP%)}w@)B%`4CvNB zI51^mmM?unpd$jf{_6@9)rFzdHs_ zPPh>Po8cNc&cFV8XLspt+Fde3>VZ66^WGw##Kc7KMO_tRIk5v0ud4$B0&>)9Rmr;+ zch@#HHWrJ%wfvU-%%Po}PW8=b?lbqi52O0CY%GcL)_K8_WsF}@Jy9U$U)J|us|-4w=LQwO>OJ| zuzEc!t2Mp2%wKP2X8rD+-9tn~L{QN1bU9L!y@8eQFB=}9IbTLa%^c8QzD$2HhBGpw z;AUio*@yWh>6GuCol=ZzZCCN4%F+sYk}P{!M%G!WT$*_r1z%-%$6>TI`+k%oiZc}@ zCMuemrc+Hdr)*W0+Jv`EPfbs&Di?0!&V3dxDk}P%!FpSucW?&dz#lc!jKf(xuTVLE zee41-e0xSwS{mMeqg2PwpFS;A`khyR?+hh{a*4Ok`&8mJxu(KaE7e4$z7s8KH_i0W z#zsl0TtN&HofSH#X8R=Hy?dvM1E9#s%lmTJ`wnSE;`xvKkPZ`UL|`wY2rr*d5IL!I zGuw5;hV1ECnd4*=k&0u*{ToTX%-|ZQrzzLQbaLLuRzLJ|t-=$>Mv#7SBPGc#f1gA= zO3CaKV{-L)O_C?ynAu1=QD{Pr(5-pxwLi5#X~62ZFzWPkCA#^h_$bzMhsff6*X6*_ zHS&YryMLg`n?u8#mp}aF?#zDq!@@%Hz@9S4w8|PjS{ZyKSY;$nyh3EbF~h%uKxVq$ zlCGTha|8$)C)2o)kmn^;4g>L?j*gD?@?V5JyYB)<7Ge-tzIw&O9k(X%{rmSXcdhyd zFT6`K(Es@HF?uX)JHEkB4qK>`c;=r=X1(kcTac6cmdCy(k#>Qia6gx$alv!XWFzXyziNFQ$?B4ZxB#mG_C*MHT^jaagZzKx|PxX=yO#rgC@f;T8iB6 zpzcn?KR&%FK91Tq-dywf^VjdSR^6Ou=nt#Q`|ViNrerv^-=g<&-r-=&9tw5;jj`C;+==_W_?ufI$O9T()A@cSD*hW~L~ z<}E@%pf_LIpO!~2#r3V@K>CXoREf2M$I;*2zZmTp!2-jYK}&dKXim26I4zG(6ZR$K z@5I)O)yf(B{grYG`Gk8{X&7~Xrr=u_m94UK1T{cnYc?@G9rk*~ZoFz}=eR2wpFAtW ztB@wca8A`{b4q;vv>L^3Z6mu_{xn`EHDdMLW?vR(1eb`tZC+?%*AGcw^AcE7s{6^N z zo_3gO{Am%{F6mD6=vyjA`Xx2UTMm}F{w{btz#S*t7H+ZLc>(EnDNUY7qsXv5n~rjA z{8}#>+jwW1QGDlFjkKO5`pu6Yd?~V=*qG?(;gKP6H~b!3DFZyNy;;+_+cFFzrF{bf zv`CcCVO-8#Hmv# zLc+p=sa)h=6Oo@~!s5fe6ET15Gw8%cEkp@yzXbemYPkY%JpH1~ucRKYwp+3WGWRv;6LhbaZjVI@L|B<+Q&UCM>p`=n z)h{xJ)fcRC6-)e`Y}dEnnmn%8Q`Us^bE?Mm;I$2nVosmE*wo>*9~9Q2S~^X=(R}{q zMSo=IW*Vmux6uBH*0weus<#gw)YsQHN4)uQ-{=Mz89D?gsB~)-5x+mJo^4rD>E`W~=MbYYI~`TcjrCrA z6s5oG>h|{cIc-n}Ggbx$T;AVHOWn87$K;Q_eaoqq6BHO&Tw03SSR&1HqFoRBe@gf) z`)a+IEh%Uw{FqMM%uPr@m+sCN$QyN|jwTAc$>!obc~VEbdKX*$>sM{qiziZ2CQX~v z0Dp9uBvvGC)4i%Ikbcfqnhau&GsC!8XxG*pj|ghx6XZ;XC06GdMK;$blgi4@cAXOd(p=4S#IVT9F}a+a@TlG&(%9+^>G;(nrhT&4~gKep710Lr~kA;Z7<&R zHXTh*Odv&0Ce=TB@mWKRY8yMkCLrM0Iq@n~Q4!orl;m#S*L@xRA=)}tk<&m z^N{Ul-F4U*9XuU2iXH8BM39l;`6S>W9?ogA^Q1lL64T`I=)PM~{W-38%YZ)E20wG{ zq-a1<$E#ZgdANm#^uqfl{hVIbS*W>E0)Jis)fE19t$-x$ix+CwK7qE0?0P`shlig& z!$`$fYjxO+1O^7ODO`)($H6hOd(ZpNoi%zI>9*h{HI{_Cq@xLphexEZuP<@~OCIL! z{Wqdpg5~u6pm3)nFTE!tFE5`tk|2rde|eM4OXqKppcLfPsXry`XQdu*jE?$K1-{R9 z6h%X1!P@a9m}}DYzKIC!Tb|V8?W~|bJ}gJL>=WvqaKb&H|;r*GSl%( z36%mA=g-seTkEHo3&QtO;vGzFZTp7C;;v|$=!@YJBJ3GJY zidW!Py5wBdS=;QH-Uc&C;ueL!W0YHUb9xnlcg zU2MzjahmA2?-&>;?uN5_5}JKj2%kBZx7+9i%;s~&v()mq)QY5SzwvP2n~#hBsJET> zR3E(!oGuWzLk$fsE$^N9tcHr{P=WNKUSmt)G2Ba}#pUHJXA$UB(~>J>=vzU&ENo)T z`u8!C#vNbclRXcEbZSphaxBlxytI)t>i#>}H(0mIBPH|tcspMnv5fEl&V7*SC}u)vs@A;f*(y3l_vj>xv1w;sQGMnqNdbcc#-6tgQvn zho%}D8Z{oSTZ4nQQ1$bd~o zCXqfx#l_3Lxkd(71fZA^-sY8?gM-4-IN5WAmEG~J;t$&m zAgU_v_er{#(a_Sm5PzlINqTY_7G8b+^B+`Olr!3V+E|ElDh^vc&_t)f411&Txo2ay5sB|%qZEuz2p^@HXjBdIAcvDoSm=`nnr~L zs-8QN0x>mBt*O~PSXL&ZoC++g;l#u%F3r-Bl4cjZ_ZtKe@87n@mG8%HGHVLNL1GPJz;Bn$}UTbJ$2wbj+t zT?d*uAWt9Kny2SFL)K0AcGC9*>KqL+DuzhP$d+q+hV~+4=KrI#YGLbV#;x3^!>`=) zHBZlnYPMNqtjraB#>{MsdWvkMcD1oi$_Hw0?TQucn$a}|tpTgS?X~L2`VYhh0W>^W zx;f@2Xic0b78aKqnr4+*1NDma&7M88oX5C7f^cN)BWbuhPRP?muN}D0v#bVQ6}0Ti zHcK+*3J7;x61H7miSAilUAf5f9ol-#lhoPY-(IKR!~LJlGES&u`UcAs!MDdaT|LTW zrl!=gHMzo?YEGWAsDQmlW?N`UB&aQQ{()&j=GQaoSy-gKAN4%Oe+NhF|A^T;O30|Y zu1T4eOGm z&N%v=yGw>Ok51KB z#hEioW9QDEQxV9-KRnoYOH2$|{uUznt5=is?0deqE`K;3 zmim!VaJ02mAY9>&5MMz_NpnnEQnZNYT=7-m-PPe&7O~h7sZx9)VYfZ*Y4~k~clxYa zp>Z4isq>6#C*B)0DD-A&9jf1}x?XT7$}i|OhUOSraflQ;ab`x+$2RSAY`IO z&8AIkwt!2cP*+raM}nlqpQ!Yng}}?11$B^erq^OBs zx8>@fH|E;33~t;2-Y#n( zcX@F!{I;TY&*NkF-I&3XGBbBK3=A|3sFQEtmerZ`^k!wUEC2VP$-H9lv!MB&mZ`=9 zjOA)cl!VvR%+H5XUf5AZoaxKwodwicsdCZjRY$fMSLp(F&iGm^TP6RW_fLt(f=+De zq3tL4LfXX$M*hI}ukl>L^L7wPEeIXcGoW)=%? zNCiVJL4!Ph4oeN;yf5{kiT|UryP~8*Gf%av6%JFqi&0Tg5mv)lnrCP^91rS3A@>5x z>@0m}M_rv^=MUl;pKl6i!0Hc5_BY$2gvtRIH2a|c{+k?8uwur!ENpp^hKRlU4cI~o zM6O8<$=QuXj)RVkal$3 zt~<@Y+-p6aZPsu@N}knKKupYIv~00>G)Bm>(|NgnjD06n+WXE4vhMeQG06-Ou|lhU z!NHPt_IYShB;(kr2u}6-_BE11REfus2$%m%?;=4Dk9J%g#{5pv-k5s)X>qkHNhzP3=G4`I1O423 zq0A>FJ@%qJUf$9w$zBoyx->-S-}Aw0hek;5ac4x`M<{g@E=p=E}&l`{6 z*5NB${wrjQ7qIS|Y$SJN%j|}mTSapv@ONEZT^Mc}YHC5_*4}oV;n}uWpInY}*DI&U zk2=NY8I<3X6&1UhDWavMq?WHPu)P4!;Mh2k-2;Y24Ba1h6bZiYl7+e+J&#@Yr%UH~@o&1$K;^pPF zsKxQ-+CN8}k0AoL)Hl4@ksB2SlM(eWWp-)Kt1(EwN*AUoxBVyN`F7JKpKg9g;o~Al zI9?2WnnI@`{AMNK#-B%`Q|T0S$I~n6y%o^n^&c6`t>U!E?Ou{XnIBV7b1SW{PhR?| z_ih&}nW+)~|4dA{l$DkHRvR0o+o+FE(AFm(QBGv$cA#&5W7*QsAO*R} z?c3S9KL-U|jLfo)Y`#j5ION!y&34I5JK!8yy7Et(x#v>-s=Ke$DiEkpD%4ldFK;j; zS*W|=phe(EuO_&L!^Hh7#5#viCm9u#6y!HTz+fg-FwoFMgoM1bw8V!oXliL$n5B4n zBMl7=wY9Z#wIu}!!x#)^bX1#F_Ry;X#TK~PzdDdq37D; z4o#ZLPoM6B;uJ5hv;*bo)2Bh4=(;(xAIXQ}U{iDzQzaC}&GkO1cz>lN9`7a-QqjJx6QyXV~t|P88`6qq6B&x6a@J);$eX-ft z>QY~942Qu`fgm+!Z-#b8{O0^nVI)0!l53%SCOwB*I={=BQsC#emI|qhFq82I&&!b# z@wHthXsYS=_mFd4sAW5QZLsoHlAEygD4nqL@Q{PAqmvUK`qTIC>!CMJ6+xq@l`FLd9*Nvz-KI@g4roK&ld(y(s-+xu|t9nvB?&d|AV)JkS z-p7u;rO#jy6Z_TJ`20o|F_%A$Cn`ERf(M(thzt=w;@a;wf1u%F36&PmdKVvW9ma}2 zjQ|Rh=qa_bvNhXekQ4e8o-;U;o@6`UR5=mL;Ub+Ig6V*e`m%&^Sd4O^(eswbt@c6rvLC)Vxvvkr@pKx42evw4| zMxb?w$9G~l$c7lD;9h>@C8t{SjkvVS#q+s=Lm3~JpwCD&$>UxfE(Q?+yT34vd^?1?4#OEIP1<#qrCVIh4)8w;=I&eK7Kc1C)_EAuk%>K9x68TRmEPX=uLT6sy zbG*uxC$c3)iOc_yMSbgO`g{KKfy zQj=s->;?yFa67A7aHgevxkAEGV1*I>AFW)*{IDev35gGVi8~`U;T)=&I+Z(q7jB4& zi-%Ks;`h3K|DN4h9f{T(oo)yQO_Z^@c}y~4wA_K4n;Sfj>((5DsB^Y9cL#phYaGh` zoNC$Wi=eY6doyEt8UY^&+fLlSbSccNC)H}Gz-V)(>FwLMgae%W)?DC8&XF=Z1NctE zg5<*6jFMiXWp>l{v(0g0uJ&`S(G~-_ZAzC?!>;ffw?a@yABF@3v<&7O@RtK|Pd4G( z=uPFmyTyPclo^ZL}AE>jS;vUQ>&M5Z!`S+I(!w^B)19 zrw83rY%%bZ<&$QyIS`{~8AKerlVvcBevN11#og7a-Ede5*e>`1$U%U&dehr^5ebE4 zc-^!H%sN_Vq5ukS6)s>qFR{sID<-{%B=MII1q6!lRYpce__|!R98HL?hq&e4nWjsK zIwIjPOD!jJX)6y+w70W!-Cc{@T&wXs)Yj2az9!{3T_5=V+*Pj;FOVY`x5a>_ZgwnH zE*7ZZqkW>nuZlq59yHMn^}zsSLn0M4eZb`m#WE=><1h_wZA%6Sw#4 z^@#`A@#?YBQS;vPt}JyP7nfZXYj!9{bb&&q(gmwq>AdzcinU6 z_fL9#j-dMXB#<79K{LUc@PNIMiF<4I@c6iNDz>3*a3{qM^X&jPDG*gvRVUA0d1+)+ z45lL&D@1qY$~G2*>FCH)N_h%s9g;S6i2>N7V0if8=etwQm6VKPw$K3QY{iQuB_$Z4 z_rAV|dwVS7cWf*yva+%cp&h-JtGyY@X>ze@S!!IW#^mSE=c;D)AAKYr7lU^IM^WLl z+|LM9^LTY7+z18D?YoW(Aymk61O+YkN`KDC{?@!nTTHP@M?8E$j*$Oy7vX(H(7YEK z?n4E23JBg_z{3|9wLFdFj*Dqe`u6Spd4{Ehg)*>fF0GQw+}hB6KcN<#F$BDDNY%Nk zLap)QmC$l%HQ*})OpPb7q@!pQ33QrrktYoinn=G0F0J$Q0t@{hm#=imgL@7^JUoBx z?h#0OT@EEz07f;^q0Bm!&cy`TG-wDK8Xlf& z3TvpXeMo-c%9TB!BEf`M&!2_C^;E088QTnq!2fFC$B#<;3~mdZiNO1Uu{O7UeMSNQ z!@GyR=jG=7p%ex~ia^-K#0r?{o0tq7T}3X&a=hwA@%^z1r#Tw`F63c6PAxTpk; z>pfb2U3$DdPILsUveL!I#AN={P1zFDF62F`1K5l-hWlJ2k|-{#w7WLe@#S72Kv=kq z*Y55l+yJ2W;-?;hRI{b8ZaO$@0mMSo6`FK>pyo^!QG59kl&}+D@gG&QH8$YsA<4mt zzda66%T$q}yCUIvFoVS=%>7sodnPBxqf?RGd^WSlcCkBUChdL|(A(UR>bcr4`leP^ z@QXccv)BjZie}Ka)$iWb*7;u&wi(Cc_Z>1+mR44> z)6#a|jhq92J3XQ}WYiKdhDM=)Sp4h)47-GB=T{LJ%Goa*W*S4G;Fx(+Ax@+ZoHsof zT86U8;P+Pv5}vS0Dc#%K-sI%uS0ufvVq>)+RyPDOEYHuUEzMLutTE8n2iV!4r2Yv6dQl@sp(~-gWg~9CD6x#*RP^*D)G5eHOH&~qJ$j^}smW?BEFkdF z&yNbJlC5C}295MP+{Hp{tbvFE$vI$A7;fM5^X+VHrNL}$MnGDaN#lbAOc`-N*U{WM zF*lbgM)oNNcBFc4tpY}^>cI9H{T2*hR!2u?tgeNVi4YMQ+C}MPZDLXi^KoEsToZe3 z3(i?cNQiK_YXd%hse~G2C~4p_PEj%n+__^mUbSCpkm%{@dAPF*jBpXSLEx!>2hf6| z4%HDa4(JZBZ}eR|eRu-{mHNVke2@K2#iS>zqvguvAe;fZc57i_0s8!&hzJY`a2+j% z1u(4P=i>mHb98)$(2LEg**cXa;CNs}RXZ(5*w@)ZD1t?>zzeRfuJRua97{)BLjL{t z-$4DqD_v7ZTibjf7pjD~nFLEi!^@0}@ntu~#I)%U94hHl*RHM1&N@N(eEZj_rKKer z9^KK}+S;)(g2{QFEA;f>I9TKJVK?_92|3tzYRx58V=81aE0-V1vn%1AgK;9#=VGar zmKNK%LWoe4Beyp%KmSz`M;M?Fp>Hd#MoQR~Q$ceNSH`oc!~CcEU?!gMr{T(Xz>Td= zO(}uy9?a*K3(P7nRbUQnjS*}C|3ZEToIDI&fYvpyYTutfe_pH9nhVs~_Fo=@Pkc|w z#CZ9#p|v%)_a68X@X(*#@7au1;GntyJ;P3FQTTG#4&sttjfW!5*K3f!4uX57WOR~L%uGq3E@2OsxWar0d=ty64%$Rt}iq+SV%J$Z$1@s-+AVAn+I|b zc<{~GH6Rmx&QMo4E_Q>}LEPf>hBSNS-6={h7v6SG3_uGkA|{#Ygd;Yes4olQ!}Po z0xDTU@4;7sf02(95j1MPJbNzy8cu9rLf!Jc=I-VOO;C!a@umX1+~1sa-3fo-9K4CAMN9F z^ua)$9upPS#L|+vzWzljD%4zAN*zkZbpW;~4;ukqxJYzfzkv9h!TKPO~A z(+DsLjCOe!gF-XkPqxk_PV*5bWr!83PubQJU0TL=4tY90ip541cr`oBNa_1L0Y`um+{o5Ncod9Z`| zNh%!RLsno1(AL)0e|rl?a%RB2yZ7$x9~{tdX$lBpdaGvc|%4)5dc?*xXb(7#0fj@ z36=>ub=7!QoRp}GvvWM3Q8PS0wLD#}@TUw6479g3LgwhM1KhreK>QpT5#8uL5&f(m zMh76_TOAYnhK4F7mP4m6v9fqY*pu&qD*`Wau#3&Jd}eRmR0qvqW@f&4@uCVp^;^Am zbXYvZ+HWF!A>MIuV_+T^OD#uAeS>#(fhF(E|*uN6OV5z_UNC1*WFjeo&t;jsih1Y~6Yr=#uU+9|~H4 ze_LQS>gmrP^j)E$NgZ`%^S-5;`FYcDBl8l{&DE8C-W=i`kQG|EY=BUw1shI5PRkM5 z!X)XHea&+l(t#$hl31a@u;gjlZz)HKIGMnYzP`SuCNl^>FdwQCoGWN~A**l$gdWy3 z3@bu=n*x9^G@Joj)vJ1a6rAD4ODzW0;9{^IdCJoxe9_a>)6if77TtV#1KMIkDF~4& zA7DM?0MB#v`?Fu%xpSw9k|v~O_QsQscU{)B-&={S8z_~H^1%ZsVFAws(F~j;{IkN; zh;wkjG=X13L#~iRIy*Zf{lr{1vtJY%Ln;DDA9j&FK|!NkWEyJE`U=LNq~}2|glrIm z9YOfDu|qIHev|gkPg7+2nM0+)WAH&GA94}Syfguaxo3Td0PKh0V6?|RpUI;~{fMSB zJr>Ztz)@(+sc(=9G>_Js^M{CZ260i(fVPnw-13DNGvu|9KZW-hUPy+utjX@P14a4M z#SwnJF8e=tm;az){`aY9L7fw1_hcb%;E!%X%J-rmIhEo;&<`j*gYN3A*|BKC<=#NU=iR^6~JFR$R3|v|iBJ z{r*gAI`1Y_ibuU=+QTzxI2U_UJ=ubNDZ{kxiiyGWE+D!RpBrlWK=T^k^0D*oq5P{| zMFn(2ZKu_wI|s^IUo7Ss?KR%1++7R)@Igpe*abDwRqSpfDM`X|)6vq>f=gp!VuGKS zF4aDO9wRgS(n@~zG*Y{g?&3ubTIAm+e~*!D z;lhZ^+QHf{!1|D`0D%XU&**D><(hD%nQPG(u8da)yB*v{!o0r+gT9nqi-V*{o@3Lj z&oU>~N;PECmU5kpTVpO!Q>S>`f@an&z| z9(%)iaTM{J3x-IY)qi_&W;Dkn!38^>RUz(-H*67FjcZoFt(=SWZP&@uLGQ1!j#fBG zkt^0{zkn(k((fd0`7|OIv)BF!#v1U6nW5655Udmx&pv~Ze*a{s(e@dAeQH{sQ&NvP zi9PnS7Lh?hd>Je{YIi0&5W8bcW##>r>sr+y-x0RjZ|K0Fes@1c`kCj<1v^CDDB4)aY?K#fWaX^cAj$n%X204`*K>j9&)z65RX+a8rAyN@ z+KbZ-3JMA0JnH>2vY9slmu>Xy^RwON;_{*DF=;vb%^!al_Uyxmjn-&1uPUB%fb!%z1e@MQeZ?={E?6z00Gc^AZx^pZVLR zzD$muf2gL$35Q6(Up{9J_YTe=K3rzM?Oldl8_$v|{pSP)IfAHLxc0jxG91Vw&Ef)! zHhN61JRO>xcS6*4gUdX@G*{eaes?;S;$he^i>$BAZPri zwMxx5a2DUGkbtQ5^z_P;!8lZQ_Ja&Li@-SzE_ef)S{|>3oqNOQI&ZeeORQf#; zC?TtTslJ8fWgdu71-~KMl7>pAx1$}vMlT^%a#c34)Upn<2MRaq>z^R~(ic!^x_`>T z&pw8o%z^Y1g(BIQ+y;qsOKUI`AyKfARaw?_A&G3ty|Jg?w}1~l5o>5e1aPncZWn`dr|cI z!ty{JXJ7XL$!5OXKH4w%n^Ax<$S0o*($inLHNfY!vM~AZ6sD3i(LPq`zO#yUIy{;_ zXCjvnQ&xKX#3lv8!^0!o1?s;v$ji4Eq6}qmI2@Glp|*WViWYJ=Kw-0U(8Xkn%aNpr zMcicV2h}VW>`)};XQEDQ|FT-%?GVg^#OHf}xo$6{k1l#mPTfR2Qc>w$ESy(SDB=~S zMb_2-u6846l5eaGZ2`y4&IF^0l1ZEf>9{sn<5tW87^b=Cs~?Hj3tbs+s1VE* zMUH^6x4ujI6cS>KuVY6%3S8>@lr7C1dL&ohNVf8Sq_3|J>~vU+|Is#vKY|ZtgL$Dl zb9E~vU>uI;72vS%zB^WGjY$++ zHM=NY=`x0zm>6=HhxtY(;i9|#VQI|FHwN(rS)xD$#ocwq*EIz(440Spx5lz5fTR`VlS)~-e_gs{ASsEWaM)@GMM)($JVD5r zIy;}w&eDEvxOTS+8vN$$W(%sTsObA(F(HAzFDnjhYX)Kkcp~OQ--b%Ku7;=4#Ys5X zgA~fnZj%!XUkzQ_6(A(X8;Qm{QEsd|n%L_DgUxZxs^O+xeQ$ff>`kinviR4z@ z+z*IMYyZL^X}$f?OnGyzX=}Y}dcbDfL+f32+*t0aR+f5}+x%MfJKx}$4(xYpT&AYA zpbu4&)Y+>duS3M-Yi1Ly+;_3U{QPm^7EcbHt!h>c=Ze_lM2ZIb`c9uepP`wu2~+9% z+RAwE-0AA;gB`V$G6aC1d8PwxyI&v5!gwxWl}o$I5xSjT*u2c62`M+QsSFkk5_HHY zqLu*!y*Z0(p_FNkhHM087Q# z{Q>X)>&31Ag~aziy1vEz15nF!JD!G%HJ)byJ|!l1QV*E(7;eNN6k)plh07b74Ewt15CsHyhHUwn{V8G+Sr zKh3^7xy_}M*&A26%PJpjY-qSJr^$cmgRK5;yr;uF5517#^XTpro3R>xWBelngQVf& zQ9d-Jhh*1%L!jiwvSo`zBBzaW%*)Gz= zy`=N!0;2`LF5tt)3mvp7)IXgA*2eTt^}_HJ`&pTT^~q@yCMMwl`b?UEm_G3Yv2zrZ z9TUGkZEhY+5cVA)@?=JU9WjaTwBnz_Exh@1!{^63vGx?gJI6{wH6Si0hiaW*qdl+oHp&mqJind%DUVBSM}v)byIZE#8StcLq^PCa?G8Fi!0~&J_BF*pH7^YM(_>- zfjcs!Q|0)g&wg>!2-NWh2M2W88cOGPcHrMUkG4wj7qo4n{)x!eydzN3cu%RuRttoW zsqIllR#rNxju}}4xq8*k2CEoOzQh6>9CpOy*Gxu#f4@;&vo=>z|WYx=y2pvQ?#AT?bIY0UQ2@i)#u%p#xEs`}gUhwtrNY?<*RnRHQ z2>tLGf%8MSm8h@qNtqx2OUeHSyno~P(@gcqs?s07gr8(zX<#lhe5`p?^oJt-QDyYM zc*`1_pvUwP+A&3hEpsoo`4lk4mNgtpo=e0<#h#Z)+(Q zCHKssN{k#Iu&Wi=ZYTkw7P2Nv6^0*u!M(S)f8WFt&}W$hSr?YoGgM+upSY>lH1P_i zzW*bj`^TytCVO~KF62+s%+0yGSVx+hy~cM>-jL#IZKGTrN!s{h8P}L+Z=8h*nhSk} z?)B=Q3Rvanaf@ThJ*t1cM7p_h2^6b_NQR71Cyqa(l)V0EdG-I7nj+_HPo{+H!EGQW z$rGie_tqzCL008M1y3HV#2^CMfU(fy;a{Oa>ZMhc@8B=vD878=_e=)ANQn(Bbjr<1E){@LaKVC_x7sb1T* z@%C=-Xrwt*QlZEYq7q6}5=x|u4P>Uw<65%A%tXkz%reizVp;1u zfA)T#@B7~Wb9~3~zVF}BQ8KJ`|L)-$&g(kQ8;$j_`ot>FeJZqu4@UP7+Ma)C5c7Hb z>BJ3VC2e#{j#kOKyG)nAHhbPT$}}pe>Dl%BucTn$N;zcOCO#s-x~B0PzRxE7yx`sq$-tmD^d za~mCPkd|(mT%+s;Sf>(%-@h}o>;9fK@U>vOB%ep1@|KT~SuTNqyNL%Su2JijfFSf*W1qN>6xQyFv zmfbn9tA|G^n2$zKZVC3|vg5gPse?;rqWHr96U6y1d{cP~JNxT?PG1I!Xnz~NwB<@@ z!H6-R3qnnFDTY}##=Z3Z{}a=D2`BsLQ5VK1-pm2XeW0p26nL7~TLNnfKHr#)y@z}= zzAjNOBkCe3qQbiQiOv6ikkMw*tL5*hK5I59S!pBa&(2ce^RXZHU)h_ z$lj2sCuj5J9e6}pR@Co~A6_rIIt1?56PWxv9AU*1S%|Sc@bs)trkZrVKPdE4N;7K> zhvdZK1GWdVqe`<&CdnE1$JdwSsKfJlvh9Fu<-Sd8UuEnAC9Y?Jw|~hHFN@yvR1|$! zEqV4rw;u;;DDwr}G*qI_whf*{S7$(J07K}c@?iG7eYQnw80~NKe%g#*2Hh28+J<-L zRc*-Bria>uPMXgP=Nwa|&+hZMpW_Td0jgrXTM0|h&Az|t+WCQ2#}C*@D(B8IULHxh z9l3%-yVkL>x!K&*)LGWKvr|{Uaf05X%!A^a2U5}U6c+qy@k({BZG0-*&_5moV~p0> z{piiQ_+W;JSe~p|`|6NszS(r{a{zmYv3~@o_f_h#11E`VCEWfpIf461RaEUvm*IiT zL2JD{!>d<)3#0eS^|*35Z+Dp+B635XOcw$AK0?Gx%W^|SUXsL)N?!a>|eZ8G#jn}Q7p zbB1V3^taaD|L1BKS?64<&oSt+9q8{jQg!zVm31BIh&+9fUrj>% zkRIjsMGZrZl%1T14WQ>syvDHyJP@>Kd=&6aHDuMwl_fz^){&l+>ET+zJ-?rPWV=oN zX2V2XC(Z8Pf43etqqot_iIG^(HQWUeS`}lu9KKZ|Y zlXqcmJdz0ZZSCxS{1eg}!Xp1G*e6Wt2eRm1B2xIDghg`f+mO+6Nm*-BF4R6r;8TPX zQw~hpcPV}-Fsa$^*c6@iI9bL;`iNHi$R(f-G<1J(#KmKT{us^c#%`JKT8MZ29Lh ztrW+hrq)EINOLKxnLQzS*S-o&x*Xk4^jO_7 z&@VmtyjSS5MsmnSdgNIfULKmC0gVto-w>w&Q_aH0IW|0Hf?rw|7xZbp zXF>xS>Fmvi-aZSysM55Ja~mtq*-aubX19e62(ZB~JG=MMk~6yp>x9OC$g`v=J+3d! z115)$9w@aRaXS;cZdkl9Uju1&{rB%Q&h3e2UOwRb5E1XCrGp@Z)MdrY=z?pXYJA=| zwUA>jLth3o%WQSDafIcL3j|zBP^Y!fR+sW9$v;GN(_0##+`89vX_)@+RurZ%_t+4}JaAk`J&Du~680MTP)~_!3LeeA+$k)oVBeiay-=v4`-WyY!+4 zK3NhYx8eJzVeh}-H2-~b>hnMU_`@W3MU>o9D;-|?&d_m0zzO}FjOe7Kq@_e-8J*=H z>iT~_nc-Ew?L}J7%y;TP+Zi}B+;lEnZ2#x8>VmSeq}sqF*O?J!_i!k`k4m0olI!dc z#5eD=7c9(BZjIY?-j{YTkusnEQ7-b&ky*E)@?h11LZGY~WjsRCzuPzbAkn z!EBfvYRj|^*Oz~IcC+9yGNbdSmtW}z_Vw#8}a#TX%=r*@=D65uQs-{8RNSW@y}1ztzUt*lXLrcoq>zdc5K%mqS4GjDcz7V z!O>F+>_pyhXDaB?5sW7+Os&abXacZuOJLwyQi9ZLDmU!*xD_GWj0*?9IYDM zE}szTY!%jb8Ei5V7q6WXQq<9rE;54U@=&@6OfKhU-J^I-fdn0k>89z)EeCSR3#W3o z5--rbPtL6)>$>7&RvtQ?F82C`ze=d=;8de+GqW+|oHo1sCXq`TtQ~`cuddiHS*eSw z?xJ_Y@t)@z^7OoSd{I}&9)Q8Fu6b*OhoskSKbG&vyj#IlAAmsV_5$F>`L)~W6LsdA zj~AA+;Rjh*7S5%t#~T-4Qh)Io)SFT8=g8}J$aYuV#p;6}X^?&G;%u^>7}H+?5%B6Q zLbq*`)^l(;56;{(|8*5^~XJ)Yg4n*a&2eZz<6_U7%g@7Bi1V1e`33* zT6jKKYz{~zK+BozEvb#ReIOiq^lWsEod_@O$QqO`%xm(evxl=&3`#Eji zbw7Q@FgrOk&!InLy&_@$j+C;gxl|sG3tplY?-E}e`x|dN!ty^x4F2)RB_RC2{ZG+y zaCCGu5iN&>h5i2*Eyw@<`SDf}*V!HC<6fGFQaPcfuoqpik!ADA0JUhVgMK78yaViJ zKOm)@M~c)H){>e3&))f^z~6`5aKCUz*I;uV^#f;Je6oH{dGo>~thkZbaKt?pq@zk; z+M*9P@3SYP|Ee{ay7&=gJ{+t&Br+?H?82PgaO1bkQ`6pqmq7BDnM^q%7CiHI{Yi1k z(y5eIIwES(mR46ONriqGy4iEmR8au+njJKLiOvkTu5;U@a}_61l|bY~6vAys^2J0U z8qwHR>ChH(WrNq39JeDNpn7jwIws=#Sv*$<2|F$nJYI6w>N(d}1Wmc4$-I*zy!to;gff(d#K%59 zRUw^)g7B8`+j_1uVU$~Or?a8O+1}1hW{RL{f@)rZZfjkp>SB)|v)h7X=bpGhCTI=?|L-m^#FPh$8Sg!f(anOO$ z>O#xxWWAkEP6sY$fIq>;H4z*0h={Y(9w6Ke(G~W13#Mtx&Q=e0Ugb?}Yu12{K8ra& z!<5>-&rfB$+zdJdfbw%lU9&MUF@g6EP|U4Q{GP2Q!lTfy&b$2g-;rda1((>o1{B`4Js@fpbnd?DBld&RKTz6VsE@Zrzs^6r$pzFQ>ji2^Sb1 zawhz!!M%Hao8pp^LQ9s8V&gg3P6P~b8sndel;Z~mG(YI>kpU!W|{)w(&Q7<)c}}m(WC>incw5384LDj+cec*7wI}$crmuhtUe>P z&o35Lc<$;^wz`$)#8u*+^Tul?iJ8yj|D6H=v9f6ORI${~{!v@IQ64G_u@@V5Zsb$3 zM0ElzOKFE;j)}ex|MxY536p$wCTgnq{W7p~B*_A$2_cvJdgMZ7h zC@SsZS9{>$(Ouyed*L$`L0hy-J>1V0p(`^3(_E;RSn1_W{hDz-wTYJSf8g&+cqZ}# zkh6(Rm3!tIaT5q~+b19J&;O_n@K5Ywhz84-$-nhiegA%83C7wb?F&H!vs_QIskyoN z5(d7E`O$YT@qYgu#PmPx-*fU@PeKEvWf@tEP0^4J*1363Y5(J=Pr)KKh1_06LXf}E zs{7X$zpCgk8h5g9yIh_6p}6C-S%XcT263g;@vLWv6f1&e!&L@1W^b3Z8LA(ahCHc} zq2~U}X%^9c16=V&{eYC~lev#vzkDTD47uzt>8sU98_WEKqdzWrXA2^>dNm(?@7_t0 zd5c}?mzCa}jKGm-xa-)PseK~)SthN?Mw2l?$-0_5uRZkkwucZztdz-pdNs}w); z39Z^}2Of;lzr3sDgKM->&8L5^GE~q$!+T%;AgR+IjtC9dm+R*KmErq4BETgj7|b?# zzv7Hv@{h8lU7~aIwBfcqa`hbsvY$tQxIrM1wkU4@;%(y`GB^6$Dr^tvKee~}mM24C zXj!p7ehG#%^H#)P$X+Yo|BaM={8V(2PeoKh;!KN`wRK8#G|*MA_#RqWkUVzw!*b!J zH`GU-|K`mbVC>XP<-RpZe9pBu%j3^~qyc&G%i!YRlr*n#svEF}kM?=`RMxKhSB~vg z4|cIDt1jjep}}5}%)1=}nOyMNnJp3y$u*BLO*syhP2EaVce{x%<3*1*TT=UHHyD*# zw`SdFzf%Y+tq~b(vy^4r{V{Tna;sk*E5k`&Me6(!_weDFhNIH~jvLW>wqe7n&4Pz+ z(_f6}k);k*WEyoHh4==Khj->Q0?SpKic`>9A7@5HSVKG*20RaH&BfK~&!w*a^hHtn z?a0ofZidXeCv~zdGD-;o;MC8REG(`)_qT%X6tU+tmw5)!5wpwnjONvcPoB_+x^AHS z5SgIl>7AL0KLb{vJ4B0_e&5Xu)15|ugsSArqt0)Fm@Jaog1W@CiJuOV8e3W}wA&CJw{j30Iv@QxouM!zLbg+!=3M=ucs zgM0DcC6shY$?E@$IR7_tipFm~+=uj**Mvj0^Y03x!!G2xz^$3+#;#q;|CcfD5md|n z&9wRV3CX|j@cL&=XOD;ihEBPhnQQu3G9V!@)1vtr(g&!EZWLHZ?yEAe8~)x5Fw8zX z7ch~}iNagqt8A-Bqr>DMW#NI<6*Oeh(%jeC@)N##>570Beq$Lx7ejlG& z==W6jX8HT8OrwrrAiEY2)MQvSULR%LOQGg(mOth^KZKCUQaiKUzlpKu@Zm31=HuYt z{^IXl0$PstNf*hqQVM7~nyZIe3L2`atjAv84K^UIDfZQiKK9_+T(#W2zy4~<7)29d zw~aHTvOtkPxKTj46D`m34~vuo9ZjfNNL<;D9ouq18~uWvsDghj|Jfh`4tDnUMuRgS z@2!A~y^`K{R5~a@E8BrjYY76e1_+H9K_|;996SFWbzmhA8ZtL*=E#JK&5UKU;Y6X^ z$>qy=6udf%G;q&J8M?a0{jz5lX1V_c;0wzW|A8J^*Py;~_RN`E2Ye0ct{WS3aczZU zSl{K>$G-;782l?t&=eXUx8d=~%9{D#=)8p*K=?Cf_FagfzdyK#Jle8f{2#}^>Gylk zM_}(>95k~PExVK`CLvKCPGz8}m`lpiwB~)pvxsZG zKY}hZ3*C>}4k<)Y~18RsK-* zoG4aP6fR%>8GmRHgs`pKH(bAVO+Q67^zI@Pd|c+rHOM~q3)}u1sDd#phWV$ycP{Ih zG_EQ5WbhFC^`EovYjd?&B5viHGrUW#zD9A?Rx2WS@b&v|)II-iT=5?SBJjYjUB_z= z?OH@H>EB#GLl$JJmsJ|ZL@X^Wm#Q4g+U5S|O$7eS??FX_a@$WQlzKM6-n2fgqDWaw zEFy9A(bK2s%QH>NU3=Dlzd_11ze9tP#XUB4kpLi9ZxyjWpnkopmn@;lLp*pzvAXVc zNHq=CTW^=Ndj}CDKvA5Boqj>dML9?ycx4gqLCanPl(k`nhxE z?I)Idumjm!!N#?I+ort%XT>Q?U{K!K4%U+v7DMn3uCoK*(_OXtCVIA%hwAMjGp4yD zjZ|-YK?@QI0+JG2q3o9q4p}O%B+P2wcAy3u;ImWK=EtWNdUc_Uy3mR%#3AkHyPwj4 zZbJaz8u6DqsL`1j0}Tnbi@ixtL=NotQb1SH>!d~I0|n5B?<>5?Z;eulMB^FyhYU&A z@cYLl1Ld#!mUVd$1hM!nw&Ll3hYnAY2DvtPMcr6%z%!F*1DWidh=l!ORUHSa^=T&9 z{O6MYIgQBJjg4cYqo=$3FXCzZg+%lVHV|B$7cYBD#|9YY^q&{NuUN||1&Mx}oP5Ws zkpdUHvTjbi{hjT+_dh`V?q}46&sd7Uefs&p*o`ZJb#y%cCmG1qd1q4Mtplp8_W8H3 zyZnr+ZdQd_TUoV@`m_Zb5DAIz$*5zBPM_W%#KG-v$$a2v8x5`Y>A_~5-Omge9e<(^ z5FM3=ueJP&QQ_PyqMxXe5O{LMTHQ1|X9WdAJ}n~F=-$4cd!Hzxh-<@C)Z^q?xy%Ov zte|vY0P=g6`2rg{F92}@ZPds-eE8?LuSK-E`Aq_sMT2qiH;7>A{yw7HPNR$=C&Xu0tRv9AA{8?)Y1@@aU zF|V@d8#i;*6_^K|dwakOM%PRuMG~+W`l8TfNT6Gzb>=!*CN~Y21j8pFKCx6TADfuS zMg5iijqa7$a=8t}kPP_=9H={pb?w^Fqc@2Bf#6+*ltDfz{_2TuPxkSSm*`Lg4;--g zUeoi!d-bh zqEUO6ENB|n@BfuA&d(lQ1Ev@^h*Q>-#Ydas#U4BDqVq&H*(9PIW?eu1%hPo%a>^Gv1-fDK{WzQAn@?7eob-({}1e_vng#LS%KkVZfNLF zzi!=@59(Sm)`s+U62cE4cC!5WCwh1HvQ@k|*ad3w=Lc-r85zgG&G~S5Ip~t+Mn(bi zm1x?agQo#!dlc$&qym<8+b@IpeCN02VDQ@G$G|Jw#=()3o6D$nhy#5x3`stSQq#>3 z5)%`@;_c1NUkMB1ECgSzi`T@Vcp~U*1^}Q`R*ovyqvLv!HtgNWVUG_$>n4q2}xqaNPjB^w;%)QV;H)^|foy zJp0U6@}L(H9~-hcIXR*6!aqLy3AZkIZRD?id*O3koqflr`*5qJ4j_=V6}W1FJ`N@^ z!J41w_F}(|)2EqmNpqUkU_j2n4tAYYb#??_FgZCHWQ0R|P4PJh&(l*=;5NTXNHB+x z7D27ko(50xrG%+EXi})v7m}H6VD}F-rcd!f9~r{Ne?NCI3*r zz-72Gy$`HJAfjkiQSEVouueO0@54Jmft|zxqT)nNo(0K=!eqD-T(*W3xUNrV2Q*qB zjM38_t{}>Sh*JkLnR1XY&es-s`uI0)g02S3V{@wkTLPptN1)i49-%8~drN%x+J`R8 z&*KbdVF;Kd%i5Rw6S^g=+RNz-tsEP=uMk8O|@Ek#YORYU~FH(X2h6= z`}^b0?<>q}f$_m`3qZUiW&s3M+RVhn1gGfLI{^W8y)YP-O{m&egJ>itN5OF{`uY_D zp}I6(M@L6<^FeT8_l?RrjeRI8Qsi4;((1*6(b(mN)d#694+Ls#U!2$BQAm(GK~KSe zyx21^kc$ZeZPuGxPM>wWRpY9qWoTzT6qS-d>_!hD_=)DNxtY217pXLyz{I4ajSu8- zOAZdUTV6M;ig9H~Bh(kbLk9z&3F7QY&o$_}xtwcfinoQ?CnI0STEmu<+VkqCuy=y# zrxciv4eqgWBV;2outQ)}AVZ>W@20}%=ZqP2ae>)g#0G;)f!Ww7e?U8fV2^|NB?pca z*smyTfCC_5v4tI;ZnY#h3V8#-pJh%i(9y&*Ve)Zz?0}`BQvvjbaItWpHfg@<=LS2^Zd^H=d<_;k~Y}LY@8Csm6e3d z1(`wXgVE58*axj<3qrIGuuNM=BdC+>A)&{W#2rO_) zS{ybi^@oJSNL#@I)*kN*KX=JJa-%#L&Ibep;?YLql?PctoqVU%xVV18dOhKYOi0Kx z`M%rUz)chWaKrZP$FS_sqySUY74w31UkpxC{=C(F&z?OpGBQ0<42jtLb}QFF?hWSw zCkAGPIYOvQ48e?INa3Q_{(|TLL@4qL3P4)VwrT~z*8k*9=$(vp7Cpw+*t%am3NI}G zkYJz6*%^eAtDX}IC?GL{0Q^qLj}N<^yf{sG6p)G&w3E*Ti^VsCn2q_9e~7)eS@6<1 z`@|3UfYbp^k%C7n%{heY#A{uC|BxuCFIrgr-vvUGbQ+xYWqX&2@ zLWCquZX8<;t19C<1AByzVa&Fh8y%;>FQuZnInS`CNO9 z|6JvtB%m)40Mi9tbY6UyqyVw7Xuut%F!f%6@l)aq? zvYHLQBjgnjn2UJ4Co@U#hRXi?)`N;ZIJnUMaXAopYc-a=9@TYqiw>L7ui?dC9X0y% zz2SfwFO)9#E(byO6MEi(m~W6erKrdYKu&f`#Iaqk%k@lOAIoCjxik4_@#_P7N|fH9 z^#}-B0~oZh$X|w0(J19oBK65H(!;FPG=44)4qjUBc1#Ilu88okfSJ`-yfytORpxZ@92ONInCLQzZ!9Fz;JShto z7ne~PUJ?uPJmlALbUpL=*+UWnC(x_8qN1WUtL=xQuwMz%FT&DblyJo<+L@*0<%yUB zIf_1v+KNk;F5%!`!liZJQ`6GU+SEAIwRM1S#`6Uat(ksGGw*g%Vj^a`3#0Aroq+*C z?P71&GY@uD9F%6p(3i){F~~@QN=si^&?vGA1Q8cUif|@r*|7mtrXY1o!?_`DB ze#Z`>pfWhAD0D3OXf0Oa7$Kl?gMWs+O>9G<_7!NeV0*Yu{ra9_Ag5^#iLam_tw%S` zfWdqlPXcBkpe!B&p2Nd)Nm(865S5Y=!ES=Jg)K%_afL?##BFYvJNWM}{gR3^Gcy=H zu!Yj>A@DoNZd|qMWC2ZRxIj;|z$FvARQ@3fr-(pF(vDYxf8S^@Gz9*3CwaMF%?HFq zm6aoS@OZU;ya$q-@NQARz`#rJ`e4gxoY*alYIv2}k@^;}|^&J@JuU~V+4T=z4 zFzP<<_2|+3$gXF=@ain%G_;Y%Kpe)y)3XhJ2I?QEbWnE|o+$`FChMzz?=eXoUK_a$ z0xRbD|FYps!Q%eu~3AnjERJ9Y*C*l~nK8eO~g9ojMoY@nYri6|LS z5+D%zyKA4}G;$qB!{~m^S(aQocF4HT+hGqQMp1hq`Vo9az+DiTKw1)e{O<3iGzo~R z6p$8kv$JQjl_WTb#A13{50i^8;NYPqHPKpxiU4*SEMV-hL$%1!mOfK5*?{MOk$|Yo z*!Z|oA>#`aNs@=;cz8OnWlNh;oQC=a1SUe|$P5Ib&>%VFFm##ci=)UzV%5zXl98xY zeEZgiEC{Mi=b!#KOPZ`p(qe@^RmyUfbr2?^!Ms55CI=%l%RX_PP7Z#%6vFW6!$YM%f6if#K=||gT6MaxeJI2(FhjtBC-LRdt$1K#FA-U!ylDqOAPqi; z4vCwSa*#1DwS9wTz5oGqh#|(_e{{T)s?=oWyOIYf?a;4ZP6&Y@w*+sA3+aH%xxoW? z^`(=7Fhrzc2zlC4Z=A-)Lg;~X4kCs7xsxuZcH&hvs9~WYA-Fs5@fM-@rm!Dxk*Xu3 zbc9p7@ogQHn6MCVxU}YHvq);ae0{q~wC9B~5GgBECbMi}lS)Pk$wD47HT9Y?&VjnxL$JbBx8z#Dx3#)gnH4 z9@5ya0|mDiP>jP#^xt^lhf<7fk8|z(%rML-e!LC2>mkb~iK1bYQaEE%QVd|=F+PAF zJ;-!KMQ_0i-@JLV^q?qUTx@k2=sLgzIZ|BT=j7-CGeAaQ+VVbCR~Yv%Qs5vEPJ`~VM23HL`0;{KDmZ(t-(|5@Wf+P7NCz! z3{A?jN=xV2ag>GLT(M#W+)s15u}^$7oCm@nh<=zry5s)+A`IVQq@@u@7MK;{fGQ|G zA>jkKjBSgYo;bz0@5*N2BTiGmWE44Lh0>+NP0yd8&H+WHy8OS8OCrHG#=+!pS!qTf zaE6G8k)bQCNPVlD=U_?9^ra;A3JyYL7a0dao|E$OXp;{K3DM3liFnyP2FwO9QsBM| z@|m->P{)DB2&NG7N51TxFfT+9hI4Q>8sLptZH)Is=L&SMLT?D-fvK>i$m5W`Ksv68 zS2++YYYNN+tT%E1$daOHng|10_2jg6fr~vxLYle)xf%755ETpHVRG6f$lW)L7Bep3 zdo^`+!*vNk@wnOzTap+G4nb+E5|p{{nm{|AczK}=3{AdMfaxH_1f!~rp~4mjhQCJ+ zhxt@}C0X&|!&RIXEI#2|aG1#%QHeXZjZi66YAFSd}rGUEjvUn z$DiVA#Q{jRnVAh^Ug^OfeE&3>uGKv%Elo%QVkYson4~0@aHp6dt0}V8WhKR(3tz}W zS7bz42@w|hNn>Hym*eYj1HftR{UQX^yeoG8&m|~ynViR-HUB42x>N)%wqpG+eerMZ z$@wL%-etvBhd=o~{m-vs!3z>C7(s{-f+=QXG8t9LaszA6=j;@W$2ZjD{(UIf96%Td zp;Y*AB(1PvhKzXeb^r9}h42!!ga1hz`@gMn{Re3udm2V2R-)HPETXIu0ErB*K8QaS^gq zi7nEszKT~`KtW-8;s=>E(aO*;gw<0%Y1w#ci(3RV?h)Y-xgkOVK^-`cR3t=@(n0{S zm(+>i6&CG#yk-Kjg@i%3&z@`X=JpN_swb~Cs>#cL!YM(#u^Gx*CMK6-&ILo?jF*-a ze-2t&z(|2Moi*wkXjAUjl2uov$)9^EvA?K6I6!0^zXxl)kr@Z@pa+p=KuTslkVG(r4T{=Na;Ntq^1vZhTtDwai0eoa0taZ zOfY5f#$JzGj*dihgZ#P2CMUNo{!&sC0{^M_;h?~1eG|y|0dLwN>vmXHR-bh}H#aWg z=)eMy7Yc|>2-6Ppq%T$gT&JX@WWV03vK3EJd&Yd@2&p1R@{wcAVJMjH8pqAl!XRHE z2#1x2N;|?M1Js@!AIFqEiV-aJlC*Bq#j*ixAz;ah3PRlpE+Eb=86AU=7Q4@mSpm$? zq+Oq535`grG_PR(3h8BW__P1lFMnY@Z5oBz+^mzchX&UO>M=GKvPW1cmup8Y$WaORDC_=Xk9^3^miX4*;0nKBy>sX>}WQ5p1R8#{HY#=aS z9JvmEFp96;y7ewL))-;xeRf!K=$74reMdG!Y%Qce1Q_9X6-9?mtS9`dSa2e;^7!hI zTEoz-O(iA$2paG+z^-SI3qdOi5xSpYFv2o08}L<#a0=lzHQZH0FtzA3U@J%<-C^P3 z+OgvZAKugU7=E`k-x+S(&=7h{{E)bMvY>KP1ub%rw%T#{)@!6u?4Q1VjX{Wo$Q1b( zy0!7#fZv}f>`u+hv_uY%ZrK4ZB5s6BM6?bK7+5$Y(+K%Mne7@L*2Ah$Nejcx_Pit! z;IC1z@}*j12u$!&oU(3CH!UR?0Z?Z^ypH@7H};`@1COPe!!-w4EC3jfyu5DP+Csk& z{Ts+7AgG9C!3NjMwZjc)XA6xX9gBPgH4jBaMM8<}lgCQLluIE743va*+p&#V!p5lH zu(7i4)630J*u7pJ%3wVx8q!J%$V>(XgY7$!8CYUS$Z0}v0XKXH2VcPmaSJX)qPK0{ z48v{)l^;k>AkCkJgceOv4%ZhUt+%G)%(cPQV1f~T$?6U>!8SXCeN^{#eJx z2C=G)^mJPwB${R1M{J_-4Ww6tHPMf;{|3n7jpqVgB6KoLZet_E*+FwF750iMjI~(K z&(H6rpq{LI1dubNV#4$h3$ELF?Ctq|s1vm0+MCy{!~N0^fq7v0q0oOD&?RMFp% zEfpf(F@N$x0tU$MnP_#?DX_O;dF^27u$i#fT0)bE4gjgxK~fDlA2dc01Z-fs z20$EarZr<1V?kfN;JExl99%&P_bxwY7nefk$(#65K-r5hTQC9v!!Xf=VC~^s?Lhxf zKjYl6Aq~+Fl2X(Iu$rBK(v(yCAo_^`5J3v`g&>)VB)oKnUoC7*+jt+MV4OFY%=dYD z2$)y#U<-qi|JPrC5f2M!7H9=<4?U>u;&NsH#bi_&0BS>g2ar!}LP9ejU7!j?ri*k$ zJx=4uVf<@}*@U08c(85<1oAd<6T*z>_l1sK*mZ4o`n^igD`T0xWV^~|@q5<5V)>of zi*14kaRdqI^V68ylomgK6qx`R?dJ;EUyC!Yqa%%-5ZnVrvE4yskXz>SxMgpzLf^sb zjQPVm+1S{e&EF`Iia=Z7+vm;u&bPxnV^`u#05OCt9A0nDs#Ryx?$x4z4H;b+3P@*` ztcffCn%2JB(#S|N`fLC|KB5Sd*VR4>I2nlok4Jj?B>X(Ye?8cdc%d}Jzz)U?1ej~o z01iOMkRy_^m?!8l6CevUo&)4MVNa_^v{`Du8DannH zvP9}5{}6LXa%8K{9*zE6;aTkHQEe?P9Jbj0!>88AX+ai}aJ#V_-Po*9!D)p0u@Mwp zOicmUE+8kt^dM*kp02~nuQ`>Lru-|5<@_xKkDltk|UsT zrj`BKojc0_b|U#Blq7}-)y0ukX@nB@<)6(DeUtmO=6RjIIpcKS_PAAR5V2Gu_mYor z!cm2*@pF5kd1E8m?O{A8vIIn+tB4jA!$9bgiHVAy&w|2e6zNk7>d*)(dOJFZ>$0)W zSb4InTIUDq1bM!2lj+Zp@<78DwTGVL1;9vfVtXQHkf0NK*%cKkv@7cUEjg1&UJ=#B zrKC^^NZ-11`)>2XYGi)-w>tis$lNH8Vt^j5r7%pxFqDZ9|2^(A-?m z&6<0j!u(<) zQT_qABWGTJz$g-UnS#O>M`;OM{t(u^OHy(MvxrP^=TSQo#{$&Rb71Pfu0MwTag;QN5r~zx`r&T?CCH#rZDP!b-Vg&Ah6i&%DvWcB>s_HjDoYfRK0knyI5V4c z{5rsc>GAOkVR@%?bn+1T!hz1t%t%7t&xnCyIEG7v9*}MaUI1_&i+?e@;RqfdWv7jN z=gzbsnS3?>91-7%w03i$IquLK^q+mhsFfhUE$#ZzUj%?$R+fRBK>i`pn5bU0<3Jlw zCd4KOi%mH|0KzHmz2zJ{9)NEUci=-q05FynFff8{B1$2a4%*yf3hAQi zuf!1WIXZ+8%E(6`(Lf()5FidDBp&R*`SI*p1#0T|a1i+u3hJpv%wIzSny0rh<_AT1 zJP?t-dk0T)Z(@qxf9dYIdVr)jRQ&@3P<&TuEot8G_}(HwDC z{k}K95Fa=8#Wn|H(KbL=1?m7#J2KC-()y5 z-(S69x=e&Z-%F1Y&fF-Ij=bk(Ok-o?5!xBh8NF>6dql{z`s~8ZJa+xjL8SRwWK2`F zI$VjmIbz{ed>&D?{eBKsqpL;ot3{vaSmi`(C&kqE$I31CSGl%LvdhPYljfml zyjH5^wZArkuWm~okMokR3g{DMl$p$wl{_{%Es5FQY)o9YX3>oOQ82ndb0f(ba65gX zEH+bANQ<6Q`5pAhD&|Z!#l1>q76q-&=gPvgO2@)<#LPSjuC5E;|sy4n(_slWrRyWim!t3{-X&3a>eQ;-l9J;`d!#UGdN@MOLgY z9Jy)5rx`QW%q;ZNq;mb--@{wnWH_=Z;j%t>?WQh>sZisJjs^W&A8urhI_H zWYf!gnpTCRF{;Rg1N!gw{O(bWoRm6WpN)E!@Xz(!q`8jlf?-Q!fU6x8ZgtK#H559a zI*;&*_Gz^U{ksn-<7dThlRcHvqxs#>m4QIp)L${yL)kfGvdHTD%62K1otGC&C@oh; zusC)FLyrOC+gA6OmX<8mCG0r|9ScS=AczU`uA1}k!KCI4n2JR z-Nda&&>#m#B`+^;>(FcBEIIdu&fXVs`z?9phw{S3VQcc(J#M$U1bzEj6{*D{1A5V+ z2Hi;XN*TsQ=Y`3v{TMJAt zHpk}-^Q<4Dl3lwVONBWO8j$Hfo70mrnES)YxPNC4*pm4REIx}9y_S@tGp$$=W7619 zr$f0TnQE^lcgng>-pKL4JxMWmUnu21H0<2VWyP_5cD;N9y{_N?IeD?^-nT-vcYo|S zarJyWy@0G7zQCLcTRTY&yLUcm*l5(Eg%)b))c4oU*}V?Umf6v@{jgHTcYPL~2j#uUj@`_xZ`*d=`n#bA6gsZjB41=^ll5H>i_JRv8;eO`*H5PAwojvX z+;H_&J`sKJ;0A@rxzH3+sUs=Xv2PhYOZa3ui)XtU<&398ci60BlJ&PjyXl`nZ8dZX z_nQ2^PwvcH^X(La4WsT?ZMu3xsa)?zoL#9IMohV|BaCq7KF;XIa9*cr{%}sGQN@Ke zx6%);!WQ)2SAMSPlA%V)Mi_Vl;e7TYQo7^H-h!g=DQ+GY!b_9_kDZqBL#D%EUFaX(=&rZkjCT^zyo zqhC}#(o-&8l|K-laO%`4Z6B&_l?;~XF>w;VD{gHCVnbM#w#5<0JsySh9+7Z|CR0B( zZVrwsfMi5SpZwXTZizVRRNU zgfMu0UxzbRDmqC25qoC{_15RLTlJkfFvKvJSW*`k^+l!hH1fMs&7^tK+F|Ft@t7Al|uss<$pi3TK9;wz+EIs*{JQ-#{8gD=8HdmQK>Cf_6Bl&f8g8`l9N`A;bzhJC; zX_Xn3JQOrJCmgc9aKdj#tL4nnE;!BBme%qxg*Mb$ zRX9+8(O~g&P~pN>M;UDe*4q*Fl+Fcd7EbCMRc6p&e38W=@V-VkePj=rG_9uQTvySD zp6}7c(!~=J-ainwK3l>kW189J`zSrJ*7V}h_43A{ zO}D>#ehMPxyUS9SzdL-BZ4D!wWIAP8-!!jHy&kWaH|jkr`@YkFl-7G=wpxQkr?h>a zf9BZW^GK>d(0#^8Fl8ZpBte#YR!(dpxM5H9P0}S=FIxj?CYrvQn$eldEYY9Jt#M># zI`_@;`HWb|$T@X4B!%8_&a`ZntfckFhPTZJsy||%$B~)+YioY{#_+i>Ty0~X2F&>e zL=6^e%2nf6Oa*+t|VRa#z=15f9pC zva%Ox;Y^wjncl@^o&SAyF(ElOetM5y1Iz^D=MA4$1VYAP&N2QInJMnWXjx^#0Z;KG zHsUZ2!I-pw$}H!ZD#EcoEjR!DfgHUfd~wJa(GHU|n3PQ#xEOx4`#6k}A}ciSzubBZ zWMq#A-)Dit3haMaJADR-{qxpjg%tDGX}Ko!o^96By^zwdUf!C#(2RP>cpdJLg5|my zr!n8bMt*CW-eU5kYwnR?vZ%@=7j^C{nK?_A8#YyQ#nJjH;9%x{#_D~Cr(jC{#Aj}X zsQj;!-1F5zg;UW|uq?#%3}R)-iw$HLCyKPKA0MsI;J_9g{_N=3*l(gw5K8SBbu^*P z;W%ylD6G#+tk3rT$sBw|)p;jIp*J(ej_S`wXNNA9k{LZ) zt(xtlLO(EX)eAC4`h#Yor5zSOv*|DV2-@Y5c5@)uJ(e*#lJ}JPSgwRVQ21%QrpZ!mz1N6WZ4|>f6hE2o7Ra( zRpr{zznW`qxc)i0bv(S1V{4DT6M~FU?Pf;ltar2FEUKv{DDR=Zp(E7 zAi&4PxE~+cWEcC@46?0u$eW-VjJtMFHg@_d9yX4x^EEB8pHIj6Z6K^#;AV_H z*jn&Qk(Gz%%W6dXZn8e?M@RD~tqP{=k=*M-cw#sky{O}9Wx^*}QG&z%5m_A2qPR2~ z1!ZQTur2h}34j3-(r=5=Lbq4DC8MLk^kr|E{*0Jj14D8_khZ^WOISuZ!t+Sn5Devnx--5}?hlfAS3J+`T z*2SfzXJ4mmFt6@WjrNJ1&fU*M+&Wex-J;p49GyF!w9%!Qd-u27?Fv=x$sbQkW+s`X zrzcsHv-*58-#RZASvS@ca`H^&Y-QHgS1oD|o;GlA=ik;PVlXk17%b1~KF&LOtZOo* z&}hNhfYfx~S=yWZEQiC#%3U6^P92*5)pd_cO-c*XN%_+49LAXI@0UqM_pTk z9T}#H(~;6liJ9WI0jZR{_MzIN?DC@popJcikhxszbOeQbisy#n>SqljZd4}M_-*Pn#a^ev zImOH|$`9r0p}Qv}&Pp^pkL_{k8z>nt?lfF4FZT9gJnzZoi&btV-8Xv&YS+t`v}awq z`eNcr;!mGH($(B&y3>bLjHO2DVUOG`a#?wVn!V6CG43ZMipxJ&^4QI>*fC^ZdY_84 zzvlRkXul;5A68}mgEa#EO_0Nn57@#kA7kp^=m`0d?To~FUOo0d&c@-J=bl;~XSD=D zh(`jfIuLmzhzn|LQpWeDe(0M=$XK)u>)MmzR3sP%i=P5Dkb;68JQ>DSJM|iLOn>9~ z(a->Cz(a#=HT_fQA|%`cs;Q?h{NOyRkFo;!nnF!)7w-yASH9#%XxGx zOWaj5yXJR!6U7)?mE^vTaTyW0;ZG-L`Z;ITI=c5cvg|y-VjpLC$Sk>vyS6oCL_=r) z+uQpk#AZLvDp&j-KBz7v&uYJUs6N|u1s7|Cxa1}TD^Dtao_De^AUyxbqljnes>3FW z583JELc`574eO}JvX+c7DVDbB`>x*vcs7bXoIAKh(ysS_N41lXw(8w$9pt#6*Nt*p z#3mjtMjdr|!!4=xTwQG4GxaYnomuRux*ony=)qx|U05SJd?>3_9Nz6Gw#l2e{@orA z{Z6o!j2Ui^al>BGmwS0_TT_;M@x2F6&RjUlCJQ&Ijbv|HYwK1tmg6G>*YTY z2smiEUWbiDZ}+&`>>%hH;<05y7wW$wL;$drGFLQ++bCNxqlgm5VXsD?h*ypsOW8ce zw_{@1pD?uKxd&t?BGSiqMM71HBqHfIRUsBfR;RF_y(Rl&L&RvTefg1*%X#czc9Auv zkQWDuu;4!XRs*+j7FelE-7ox04B6$qPu;KyPr|)UUHp&kAN%fKGwMF-YA|0DR5%$r z>W0_{9-#oqk8s|Q7}(Knoi_r#Ghr%r!Za8 zSR*tSG!AS*N3yt)-#Tj5$LRg)XQ{It3RSN;7C0uo{C@G?jh2a{eytn+mc51{k(H+z zUHzO@av*@Mk;jj2M0Jn#?L4`)!1+e=S?_ACR(jhF3&Vu_6?=!3?7bcmR%rv0h>aESr z&{?tANw%i+%w8-nckLESzRPYlUzYU2{qE8}SiU?}{j^Pi>N&1!yCeBsymbt?H%P8? z4-7rUYG1(Ec#7J~e<-0^I@Ry>!9mr))O&KTCl|kGSW2(?oMXx1*}=2Elil;$`LhHp zB)ia9B|A5MPvrMcqITQ+OkUZM%^)JN!qQFaJ@?T<@RZ^!o2fH|bH-9+zX|YPMhPs`PWF{i?{#ALUdy z9xyrzteqBa6uD0yB_vz94f87*ui3xVNAorJI=3A@8W4uj^U(W7EB(+|?_hoGP*xnmO zTcSALT&d4%$Yvknd?pgqYgq|52t@wthHr%&ropzb9L zicxpk$`kBQAl`#EP0!e?PqdP>HHCxnxEtEqRQHFTW0hyEnp91E_3&Hkr;tGwZnt-u z`KtS61{2fU5RGXi)Ll<~kFKP>N)DrKtxXm{rK4-U95C*;JG$oNS3U+$W7e?Ov`gpV z|3%n)M>W-aVWTR=(4-2X8+s5hR6&Xosi8?gI!dpCbOb~cLp2m3^rDnR=^g1(1Jab< zL=*(+iZm64JK_C(_pf``mxT+~GUv>hIeT__p1o&#(uunN1{mo5y$1&4uwnTntbjI# zVT0G%+{D0u9=RH?xDDmvQiCYO?*h-_S4zc0a2#2besvYdi32p_Ux40bJ^Tv3B@c$a z@SWB259^8=uvj0xd%4RhK$^v``zHtYF^Q8{4M?6QZ%+Cxb;!scrCNY5J$<_PH70s# z+B@6C;)ZnYDu6F~5Li%*6O`{uN{U6$+BXStS4ZsvumcQ`{_*$wM+efq-nqXAop}bD zt3+5gD*|6-@hZMfvjN*!==Z^B_d~~oi-&PX%p1PSt$_od`ajG*{7$I9U3S8x+hi1R zGhn*^Oi;)o{m=QE3Hwu;mf!e2njm?9mad+Ern>dxLM%Kkcv()dIh)AOeb08-KV%XK zi8B?p``h$M2I=DSl5y#0d2ei4#rx@7TInq4vhO-?j@{X7o3}o^O+|`IIH)M^glt9p z{dRbvpf{|{sD4sjIc|$GmZw^7|*tAlFN?-cx;aVbaHSC-;4x zy1WnV9$K3hHa*(~BlYS#!#f1B9e~X6XghwPV6#dTct=9t0KEk$Tkzs8*L%Eg3hp97 zMdx`Qdx~^+QtG8`p@%(hYmlFmq{!RI+<+Mz=FoBJZu-Z>GKQ!=1=~~@VhlDK8 z%#U*<00$sdf%GcBJ_8)+?cag7wf#Ii09Nj5<{*K_Mh43JyeB)r`R=_AYRNzO42Y9J zDa+mW~-*{|874^EwN?q^|%8wQ@g@#(W@62ZjIn?|WDns2KuN3zA3#lx-qrc>;tUOlb3qi5b=8>^dPoOaeyqvi*HRi?B5K$>-d$TitkBZJOH8^ z51e0Yd-;^{wjQ`%*)jjMGNL&?%(*%Ipf`T;Y~ofbkQ;X=#k8N zN?7CfPhm|<(qU~YYt5o$JumC|#>^|F7lOWCkWg1X_~hN|{EEvCSBJL0vrJD$|Aye9 z4*BQ(W8~d&tfTAMVSmd>tFfG>z#}!$)eW!Q`l7Af>$3;VQAD1(yGP%y%saa@9rDk*Kfacy>0$H zn46il$?fTrS6@D`;C*9LsTvs4KNJ7>n<<4=9vymISw<5h+Z;oV z?%O^h`B!95k8t0FZYn--d@w0)^uDQZNtV|Qz}v@%N`ad?CpPz0N{;mIX78JCT{-!- zNl{V$@nPsm$Q6#}9Sip^>1%gS-o1N4nedl56MVKa>M%TfX%obMFpSdAzRlete?PyL zEsy(`Q&7CW+Rs~DuDk=x+xY~O7o#LEmK4L1u%4rZlVeWLh9|y{fYj*TAqUs;M6yRXOiqF33PS1*a0HZ ziq?Z!gV1#&gP?w+Pk>Kr`KJ6`HDSRTU{MPX0lCup2f#3EI(KJ53)Fx6Q`dn@@aWW7 zo8F+o@417$3INXXruAZyMVEf@dvL<^Zs>A~DF`KZGv1R@u=VZ-kEyd+`vJks1@z^V zb)fYI1HJOzE%$AG%m)xLj*_BUQiOkZH*@MQNmBroKbPr6rMp&IGyDgjLw^c*|E&sM z33_+;c=;{~5GRENp!($*LleNxK+ED2Ak8}f88QO}G+->5-=$&4-@=BSYQRpbxzh0Y z9B~>1=+!0^od9y3&XnB;E7W%QDFhIV32)bLS2O{p48$l2JdaiZFBldC+>(C)b85If zd_3r^vKz_inck=k^1kVn5LNQ;0Y^;YT>uSp7rk4jB=>jq@$Bl|M@JUP>o3CI zDQfN!IL#}8|{ z{_ZZQ$Yx39?h-6d{)%6Bv})T7mqqi;7Crd$=kITWt>e#Ayuk}$`hLd8D;*;=!<$b^ z>X*+O%)I(5fTDZ!S$%S)X6|m#e7MXS)kt|rThE8KRZ<>w&SM-xM{|K+~YUhKE>h}udqz_s1zQ`w(se71l%z`uFzFblsG z-`t;YJ59smq!yJdHn1UA3$FEa-$CRh1qAT7XpBs zCjC<){T;uoc0~`NLpr|}d~7WKJKG-oi4gzq_&UHwyQ=}{5_)pL4S z7Wj@$O!$9YgwDj*02@gEE&DgF$CKhr@A>~)Tm1jt=Kn%(Q=FD9fU6PW9#x!YQWfyG zk_4X3#|nSR9>DbYU*W2#v`7wu(eN<}^EvUws}HITI%NM4Nu^@*=401{z`~32n7clR zJ+(*(vt1vTDJpVL&1;NbnKjY;v?j`6G(|^d-!ffs@eD$?3-Ota1+eoBg#00~zq+F}jzV4ii_&}xDKs^MHVTfh`Gdd&x<%Z3ffov9} zfKpM2ZP6mzqwvL+VP* zO`fZv@7JC_JQj+d*M(HfL$BqcT^HOvtLUc-7`RjS&`1VM{Z1jdolFn;W+iNHY4>b?b zutdSA74crG{m`l>6r?43TODaqY$A9Ld;%AK9 zxn;P?Nq5xS(ji%k(c~R+tFDJ}RHQfh^*@v68}zo1V;XxIsSl5sCSChjog9n~#i6xX zb}togbMIZCjpD~YsP*~Lf*&|TzhqM=z^=pbq$a=1d}7OwRR+9B`>HI`fnkr6*}$$G zhvVhHiD@TA3cIV;-fuYgAu$O_ihMJzr#P&Zjqy%;pla+&+!w+dkIAG~8_wc0DcwPF zFL-7{u+MaI81l`F1nZo9zuPF8(e{dJ&$DS)Pj6*jN%CGf|LH@+V!kB0*3`eZl{)~I zJnX>Q>`J;;o*m3rCk>Y*s0}4ot;&z}5Bqjwd88GkCD?6u)q@8PYvBEfi0}6gX$i;Y zV<3V>1okF|FA|Ku=01NJHqOg& zcQZg-EFv*+kS3jNwZ~#M0pI=6J7dY_<`gZrJ*_p}>BVQj#iM)}8*iqot$*6SczKCI zC5THC%ccttjnz8C!9L~NLEbNPsaQqr2e=Sy(^u@So?bb?3zoZ93l;>8Ddyee5W8Oe zEC_bk)#EwNmrV5xHBtD9wzuSq89w<|%77R4KOJLpmbA(`ERl7`wsNFd;7UpX=lsKA66dT zo&@3J-ER5|bH;jq>0yOJsgWi$HEJ?MC=4r^n}0hw&Iny(o?4n38}%IB&k6ma#lgC5 zRTs{1F6Fd!cDgB;H=2>s*5B_VC&#m>*n6deT^XaulCM*eVgm?Ty(yVE1|+-($~@X6 z-wV-GBcI^IC!$esa_8(`th#o#m&C|D)Ae2p3>f%}IpRSCu>Wdk|H27rxa0(mMBak*|0>D0{jQJBUNZ@%?AC(+$A97j+ip_a5GF zSl!}`J&YIFcdf{hBmxznHzp-MCgVT$!CKQmF&L)j3olc?Id(yNu3N6k(FTh%#xUJm z@0N0zobZ9Su4kkOx&IoZ8MTe<@y2x)kyF3_*xFK^@9A1R1>5FXg7omZO1Vq{+0wr9Q)ya;PUsvV#V6ZOGtX;B z+9kx~dp``x(BWcd9&UnfS8F|^0htj(Qv3523Q-Ef@QKEwG(nAP|GX_jj~=0vgnoI` z({kWbJ<|yp${Q?4Sbu?#zvsxB)y4?h@Y`3#NlJF z6D_Nv%Yx4{^RJ~dBn6Jfdnm>)1+Z?`DF@%@)Sj7^AyfzgWGm+Hk6&!bb4*xSlN6X%gQ|o?49J~#?#q4JY^zHX^vbPqh?sn7ajWyqp^yU^$9w5k-_T3GZn3TP@ z3Neu79ev&rso_7R;#*MXANTKbE|K@m$OseLVgos49NK7%yVp}O^r`PGN2sG>l@pDe zCK-PrOA^)YAI&9;nmIYg`)ryJu;Sp;zC#Htve4VLi9mXRBaKD0UN^0_top(^p?KHE z)Ma^nw=U|CT~tzl)2Fh zXChtKH2S^82T=Yjn{k)-ay!-P70NA+n5lXnt3x>K?b{)}i*7fmW;k4ZA;?O5i}Ee`oQS`7bD!yj9{lrxb&)>^=4k34wp(f{Jzdi7Vy|u?$}uvY0BUSmPtauX=N0{SbJTq$JIltMWvJsn=FP z{lwq$r|YfnM3VmKP7xQGWIH{RP?JNo!4JlODpQjL!aw9x+IAr+qy9cD*deSoM4nwj zu#TYBz>>cYnqIGF=K2ig%K7@IH5MhXUMQm{;%s_lg6B5txtx{s?fl4fh>W^2C|RVR z-c*s(Y7@Z~?B@8b()H>$rG?z)c5AD@QY5%$DaSc73e(wMdvB)B^>^_*fxRoW>v!rW zWn+LL=wZ|}L>DC&8?=b*APc|xSu!DM#|H0NyL)VA*+z{F=dIWug+ot2I{AIVy#_&v8t15@?=sHXmfO}?R>TT|Xzsw+pNtz!Jxm{a*? zt%0RpjfA$M8UJn?tB^K4gBK2SRbK3gL{tCg!^iW3y}>AgKUXoc%4-XpUv@qPJ&Sg; zuI>+Da%FL$7S50Q%cgHUr%P+yI-i;1235ZU?auawSv*l)8hqJ=*3yC>YKd=MigU|R zSWJ)Fj3RjW>ZP?gr^jD}=b^Rzr^zdz_Lf({+w2YtOm@+JZha+J&O(dBbcHnD8pks%*yS*U7MY|>nE2_ZWQiMmvya5 zZ!vTF!-XQV#5;W&B{EwKbJ?%#C^3mFd|VZ%qStyB72Hx}Bq@)l0n3;dVY@oD-ePR( z|03v#&BG>f$xhrY6!6ohiGEpCBX06&C#<@d?cd5)IG({ST>1$UF{z1-9?N-o*3fzT zgE&iW)^kVe_^(h%*&eL9zqbkb1qwT% z|JwtMz`OBUL@eg=vbk&HJ2;NPdWloRBm~)z0%9|Y$X(Y=1MaqrRBzgW@qC8&_1RJ| zwK18Yx)^_#9%-Z&B_nT{5I&P3TnCm=`AijVNbk^P5Q%!&LX7pF{L$qjPF-+`Hq-W{ zdnTG1Uds@37RIBYRn%a-+uJYT*oZ)o3h>YLQUd2k{>eAZuB8qw$s*YYEbp(azVd%{ z{R>uDY*xXcd@EN90d4gNn5S0?o4jm~im|nH3Vh6{rS#de%pIZwJ&jtNW}0{bltBW zOF`yX8Xk-iDoI2?Ng&?-3N`aEQ->v;S$oTmUzxNhA=&qwQ2uM{X3ail6G*)q9y>xv zi(?24UUIxsyd$sh)0qn1DI(BAJ@4#@N0R3W3sbgMJ;_Ip{&={#gougIwJ=@h z$RiM5aZmYMy%GO*&)VK|Q?Pu$&|ahv=qpSErW1 zfcN;0!Wk(d%V^3J-PP#+oD)| zY?Y)%@x$*|C!2US;OrJIkCS5?m;rP^UT`!_STDZa0tx>yMsH#iI zo?>RdDtQ>xi0;uC>0AngrWgBJ)^Ko4^x#)$koL#Di`QcZ+>4A8)sWP2Za+J6GEiM+ zW9n8k8wMOE4jy;%>YO)su2sl5Fbt%_@dypGv1mN#K3L~mcYvKdC`@5sNt22$;3lg( zsz0iW|{n4A)YR$-qFR9__6jJ ztPpBS1@IYcl}2Sbx6=1wR0EdDa9^z`7i&CiCI}S^COyj|E=kF`k9sIL{;Tb?8&|lY zuy+)T$Ox(u*+!NoSKw}gPy0vkn+A5FtHC%%l#a2caXvmxi^99RfdQ>%=sbtI8^w>t1@InVi#onF2T@(32&5%WX=tDx1g91KcG6WVb!gbm3S>HwFXCE(-W{_P; z5;BIUyt}0WDMAK>GGP@ob>3vdCD`rGLf)cu?un0h>bw{@7U8+C%AlLN9G%lm6Fwp} zY~~Yr+C=!wV^Wm5t4|H%Pc_ccE`ud2eCp|iW@UQ1G})UIY%NmpukK|AR65$^@aSoi zcX9~EYx}}mbETHlw6!6UGRnI7=~c_)@7te>h)rVK9+t?3m`C!F9&PX{Q1_qUFuf-Z z{8}7si@q&f!q8Xudh0x{rl88!Cu!JQU%k9IS_`Z!Gxrc4;jLSC*+A^(_u;voYnLbU zJM!od>KC{19SQ1C*m!qPzg3soU3m8u;v9Ae})JGuFuw2jU@kRnY@ zed)HzuG(NCTJvh2C=%d>+Oq*xbK1+*jn4NZr~GVndZUJZGjDNFzuAwvxnw)WmQ4e< zn8Bcta9Oc|?LNhgkR0i&ZpsnDxsj5S(N%=Y{DMJV^t0EolAe^0>>Kx_Mc{_rzzG-q zACTz_0GPq%N-LerT)T5ag5owe$G8Rp#)|fc7iZG4{eQ`|n6WKKp1e{%ZZ}&!4_q&8 zrYbej2u3OY`ElA_{Rf?DP=3@VW-MOJTNhw~P}@@fmD*u0IXdp<`(wD|yHn|IWDzLsB8{Q11-ze@WUP+R3Zf;7Y0)gzV=h=i{T`WXR9O#!1ue=U z9x~*_5P=>_((V)(d3blM)~`sVCjmHXAFTTQ?p@W!;B1+AJ+jg+2?``I^~48JoT3QX zIIt^&gzbbHvvq9RyFbv)`C|`0!hHGaM2NRLnXljbbtk>sr~amz*mIOcYL9y^N`6v= z@BHoCHdv8KA2@%ST$fftL4n7Q(Lzk3cKEaue2Jt++RrKa6|3TJ7p#~P@gS^*r6UGDj%u`a-?DH}r3Z*xfx&;~; zB$+>Sk*5Dx;Ys_Mm7Hk%=Lo@X=&l=Z*=s%)kJj5SwIs4_pix2?rSN~3@jw}BvkT_) zh{&nOob%RgMvEyuP{}T7M?rih64GHYOFS5t z{`(ri>VJ30q{(Zwh9wC*a0&geIMXSPX^O9@{$8qjDzZ<&@P7<~FMxvw6&ZVzB!l*D zf>s7l=0!7y;&Z@-wnGx{U3exaQpCdSQmjvpxVch^Q1mD#p4*yy|w?9;d+l@oN4!z_PpGz*&&mg;{&KlqLK zu9n@WFo6)k;LuV|V|1@z<@h>B2n=eE*?I23gq7?_x1bzqGJCPOTY=sD1_Al46!5la8uN5481F|C?Iy zuVN+2E||>I5if-%R zsAA?P@e&9K{GUiSMw@Vzk;&KCZutCXUm*>p(iasWv@#m>EAW}>W=<4Lngr}j>+i7 z7oO`?0Px}RU9XO1Epq6DpsCVOT(@mrVxY4)Bjl+O8l*0VwYTsSV0bmnw?5U#XdQN6 zRtnYS-nW4UmdS0agKKhfQ~M%Cd#b%fa^y=%G!ZOl2s2WqSTS`bS9;2Cq<+vZ#D=VA zIR2B>NTG8x2{}HmXp<3@)~^3<$b1mhzl{plQU#2_96bc8>RGS zl(5*3&?`Tl%2Ox*(z-B@l!?ov{NG-pv%2{>=#@Ev>0ELF@68I_Ej}l0Xry0*mbmmk zs~8I%2`=Jn6Zmmj`^r~Pi)&6UJrGw+dQ zKOI3ydiP^uPdR6^L!cIgxQi8r^nV{_$LEVMt66BW!qsWk%r&|6TDvYe6{OgNgoU0| zem@#&Kg_?yvxR09K9_IgmPsBdKoTSyebUx`H6t`q*SJr{r3)7TP}^bRD-oG^Q7t+cgDH))F+S*Xc1;W^{Ib!*DR%~DE!PU9xM8Q-&03^tz_yDs&9?D~+8`?(~dwb%rz#j4}or^LHt z^yZ|h;We-HE*I7zBb}@>{j2mZc8oq#n`cV_7= z7D0t`t5+Z5w~!_IDQ7-2kPNzXjVJp4aOJ%jLqp*H3nXb>x}J|jJ*|91;a1|Ca5|Dy*H}EiLv{ox7aLV3bunVYMMi|i>)RG_j7W&VxLobz zefQ8JoG~iTB=VNeh5aFvH|!z1;Bvl63D+A_A2m%AFnrI5T-JJ1uFq@PF{x3~JkE@r zTIPw=xXb=-9A*V*$iu(HV-Y+C4H5U-dV&`Hvgz`D@5(tOIJs4-)(RE4)#G`}zUXW9 zo)E5_iKJNpNFio5-m%Ux(ZkM~pXC#&jog|4gdYKeCCNK-vZuBGw58GV6?;|l*qYf88Q&{* zoAf1lst#H5^gyWnkkrsaVjltYEu$!GuAhH?RT{2!3dp5PHw2T>bkypZ_vOVV$dVpu z$UA#DAUm?1bc4r@PfM;=GUyI;ZYdagGTe06OY$onPuKK(T%Dq#9+h+K48^0L@haQsQqbtDSUs>gMvAo!Y*q6?wKG)H_ z220IjzxF@n#!ea=K{OtVF4{1k8pKDJUkP}YsxVZwIoz@`YdmJ7beb0tPhq9~IKqB5 z)u=DCkww-hX*Ttp;mY&VOUe+*Z4$!=F#i8^eB{Q}1sjn)p z0k%wM2b3QfZEq3EEovu*HIpW-)#Y@XX8sooEf)6&Y>5 z{Q4kfq%))Bo~6vKuR`;mh3%!VDtQv&luq!3?$>9}-)1;V-&+_xV|R5O6Gb&CT|RGp zX=?`QtXN1vx-r)k@X&ni&{tBfwE>P;nUyH-dCN`KoF537r>W)fcwNIQKxO`y|IteW zUYh*9DfQMI5hq{?B>j6Lm=Hbd*)~|&Vg*(Y9mSGh7YU00JqF`A0NeAjA~SSiO5=RW zGlp*6ybEqziI76q8}>z)0wA~ySM$HHsQR2Tt&R6;Q~5@4L1I<&+}|DO?^E0EzlhlG0WcmPCya*v)Qc_`h zmF^CGVSCcEZu;lOl~P^gSVrRb_)vXmYw?)X$V9W}zTKNsJmEjudOCir%Ja5eGa3Rg zN36QL>YHR9f?5r2LE_N8)d9~tWr1G-ElnCj9uA-*oo9}7E5)Bga$<|)URx!U(Dg(l zq247o@5?@=Nt}8Odj8e+QguvtmRiAwFbI z=J~Y36vq(x(nN!mvNr0aWd*uMx0$~paOGCuox3yYyRNI2q^E5^M_PM6tR!LFZ?jEr zgZgqXvHm;jT2hi|k!&RztyTN*h~2{QtUVS&h|jT@aV*=anmjltw~pBH2^}Hb!K?|$ zhajDODQDA|IRxSzUHx-y<#YOVD>dpoM+?e0NhCBY12;G2c)(iwNv_~c*E>8+oOI#; zGB3(EtS_5ZZkJoA2#0J?7ri!1Mrt*F{KGDE#2O7^Uq8>?^*usLaEQdz9kZ{gBzgHP z0G#z~s0dzzsF&z-wQQWbmzTr+AOS_BSQ-$Lwn4ov&uG+jyPY;MRaD{j%6!VNw8xW; zG^D;8&?+%c7?Q-ls>|6o#ap!c{QaaEj^?Jw`)W5MMxh7EBiIJ6%;_)SW-7x8+``Ke z0R^NBy{f@7=?h>O=BuE{$v*xO=K0DRhZ=jPr_#)}4Dp`OP6jnk;qebU)19vkz zGcQ{G`11kEQ4ztvY*Wa>TEb`RvvZD22+EwVkUEiCr%0`Dpt#lt%$lcAnJdnX1JClm zJDp~Coh6!$$rQRq#(+K|wv71kajnVPKh&P-FQW%UzSxR&M*n6^;A4~)D7^M*K;1sh z4&~SDxi5TcVBFd=(8Jcy$welO@VhkSy#sa7KqSQ}pXPOrw7v8EBsK~^>fTNJY~`(r zS&X|>F1Lsl7n=y*K^Yed^|N`0#|QS3jfm9gmWaE*`Boa=GM`&Bw`&rjzh=yUg^#?} zkS*RoIEs=ctxpTyg;b$yCc#yR6nx-W_$2)u-9C+SjTu^U*xPbYs5;#vuv@xF!?m^@TcSvRJ&x10!mo5>>rm{ zy$8F}HDe(BJUx6UwZU5@UNcqEAV*SL+~|#-T}G3KTfM4YvPERB&mL|_kFGJ5;A&g| zX3?Zim6u8RpEVE`0h;=ENK5Bg_P+Fo2-_9A#}kPq^!R9tPrE9XezEYg<%1Gcr)CVmF=XicISqOcGd8DxZifPC35xI?_-%{+~dMd;*k4xJ|1Q6OpZs z)kJ$?81gd9%`-jOn>?U$6JlMJe)E$u0|>D(KXK`$rih2Ow zIRfscE64yU&#K?5-5>z*2>5wCZv-MG`K2Mk_uqNn&p%=yQt3g*kt`HM~ob(B8>u`NWJ&&5ANs{omqN0BPN&C<77X$i&8bC`byEM$vdV?rBvrY8hRGU2+&*8>}|)225br?$gSN zl0<}f?Cmlh4xz3xZZ%k1#n286=68G()Popud0_?AE%X!tfr^w zvqwtkk~vEk8TV`BvyJOqmU~>mpnt@x>uVa2{NnL{`(EhNf8Yz7ze=qI|I$L`< z<+G;$^I#UhZ|r^+RKRluUcrD4p&i<8j>1$s1^H?`rNmCw6XYb49pn&FbK33DdR9&h{u%C}andV^kU z5K{kzZp-WUU%X$sN}ph?L3E3A8r@?erZDhnKX4ba9gpo^ZEzMYO|-L;0v99&uV(!utrn#4Q#AC>qWd5P6WUg(pTpMi+PeEnW*WBxTij#u;MeMEDp;`?O zTAzsU6isQk?SDbM{;hhkC@A*45jtG>oFD|o!Uq_pY5#RYE0exwi`9;F@o{vHH6D#t zSst(M&6&%MSxOSaFYWz%Y2|8MJcNofbt^S}M_i`0!)a`tnPOY*yuP_TV{&jP*2Y*+(x>nRKiY`7owI;GOo zNLwr!aGc#cci#VAjS<0R@3cQ7o`0UK;h9=)=3X`*t{4oNNY%4pKt8xL;~r0Y!IP6C zSS5pC9goaTngoWm9J?nG7c*v$%$5`*{%QZ@1-^^nQj^e_>Mu z!asc*Z>}AA*U8IqD2l1r;F@eK&yB95!YPKl>*f_LELt7UKa0f6BggN%J$ibXN1NyN zvt`}p6nBH=pQ%e6ss5cQUs3QS`Pik(J0;^AJ(swsx<+<(tn2=?$sa8CS#DCQ#4zYW zQU%T;`>>qaRwJ@cZaJ9G7ZfPZ+zXM*dLz~=Q^qWz7UXKvbFof%e%l|B7s<|GvgJ7i^h8pyzJeu3C+=_2& z6NlbU$$yF-LSSbCt8fBT?!jNoM&_z~{k$5`_XX$@)D4+>ti+z^*RO$8cM)4%HA!Bs z;+1L*73cd_3$31Ixr|ICenE^vcp6W2rff}?v#$7VMNEJCq)5-dkPja`M_zlsttt$@ zNGK>YmU8W(T%VAwfSEDbGES93{)%p%;de@qaqp=&{Z|iL}Wl&y}6pQ?y)bo+Ele?pvHiyvnW9Ci`F@` zeEt6YK5ar!8*db&<>us&KD@i}tCfai3F9V3K-Ut46T4((vhjM#cSctt%|maxX*!Mq z0AMWSlwSFUC@Mzmgm@3mR!yy1Xqao8Z#`#P3Ecc(ZD~l<-8H6My6k?Oug7D!TDBcLA)_Tw{~YIf*o4C z7l1e2Tb3*BTJwuVHyIlMBtk5V6jG_e1?9hGtM5~nm@bhutDRhslY_?7)v;PBs>0!F zVrpE1amCwLi?rmEo@l7EBpVa`#xSFTvfp2U=jG|OtuZf}iXswuxOo@ul<~Zt(f|qe zy$<>Hu!F3j9SjAU*d#%BQwJY>Ptn=@0)95B*v+M^X*0;Sm60}X{9wC+mUcpuxE|+J zJFSFhz5H%rRM8b?t?ot}26*MmMayDEK#rrADrcZ`VV~PZMm*|f zUrup7V-BmD^H-u!)r+0?-kjJdu(q}|DSseJv`$@?C~9wSjLvs>P}>$dwc38mzh%1g zYCb}jwb{Bi4q0rmTaf-+RZ8xY+1vUpoZfg99qP2W>Xe1U>!UJ2Y0)olN3Mz3ohRI1 zK4}JhaQGA>dF!}OdpOBSuNy|M%ou@V>S4xtT&)QjsNU}>?^w?6^X_V31QlWD`+2G0^1B>b`@0{;G%4rzU-h;5OD#MXXJLI~>&-Rc zzO>wboR@cArzdB~2@Vmp5_@4e8}(W2;NN;*U%gkAkc0Iga zYW(lGqyNBqsbFcyP5j5d^d063yDyLk!_J70jUf&#(`Ikw-nxdGYBtZ!x^+T($)k+!X&&^eZGD2X|DeoUepoxZ_hgQ0XQr64&KBc+OcOcpAu zK2GvZAB-JUBoLD|q){gK$z{*9q*+6gM}qZ`x?^HAS9Tv-SIi;*OsOJWAdg@~%omCM z`Fq${)E>HW?gcokh5@7Zo|m^GF0CN+d~x$(is;3=5Lnv#h88$Ng56*dlh&V$jzxFT z^QBhalgmLhb3`No>zy75wh_7z#L++He?}e19vxOc|7Z6wQMNd?#?iP2!3l3Ly}unQ zQ<}P$d;4|cC$rX$+CL0@E_vl(%cH3mD&n!5P|kVtqHWLd^Pz9W1g__~4<%&d&~THA zgdeTi3W^^m<+wDnLXPCzEfk3k9iBdyLGjpf?3=)o>GoTWF5Q<+|2j!C`2jX>VVR`Nm8JDl<-TDljU9nABj%w(lcVsd?}{TJulA&X*p~ z*-lU^(bM%f>dA`i>*$E2s&d2dloTl=IC3TMKmd-%@x_KCN^f5#>sc)58I4D~R!oTv zFQ}#o;BnMl!O+R%?qsP+f7_WRT#T)Q#~chZ?0ye8|FF>Odw&bVDw`_asvyjGIf{OF zRCCkH2^KM+i!JC&_7SmWoeV+*+v&#A6#b0IJfl@oSF9kC42Yxq1*nrX>e@D*e{L@? zl<5~;QgV?9Xl6~;&G&kU(KH-=O>R#%oGI?mbIomby#Cwwm&hOT7y6fHMz0sK;w@`Oe89C!MuyJ`Ax3Gg+pAZM zSb6RbX_ky&FVGr?CtiFuBwMnZH5uqAUGI_g>`npz{01)m4WiJYxj)2W%n@^8b1&dH z=17l`0)Y7HLJ9^Pw|O3vnlBAyI?XzX4oC<2s*v5}(#$n2pE>{?LgZEIrD->?Iql4U zdD+Tlk~?3I?8O>xHtNsHaAW#fC?hXLyTv{5>Yi)!eONlZ_cVPs*$y#H8;+pr?51C| zYlNrtJB~r^tzVtGE4l42Oe<^EY|XY7q9Ox&du+F_vfQn&fpLq&8zBE0EDp{eXyre@ zoh2P{@LSQUz>u3KxYjJV^bY5Ru1I;wsr$L~;lhT~ucsoTBHu{HhXn8WT~iE;x|~`Z zLO*_C&$PJi5@6(nLoCv9i1>#xQYH&kAp-Hz@$>fPlXh3B0h%jJLUS3AYKGHdX&g_F zv`>Z`lqB-KR$ko#83Sb}!cbDjQ_fH!ia{zdyuKO>@lDO)CD97TpPHq zku5A1gHK9!j`DpMgcg;B!^j1>J>qRwQys;NA}*XgV-rNeqGCm8-Mjwnd{9vcxCD>? z;YC<(w=i9Z&$EmKI7Rse&dnGkN+V@;;X>pvEKo3_pgs?bM*c0_HxOSbng6RU6xTMy$>!rgv-RUe+Ifpf6S z*01qKCeHVrzxOZA{n98f5#td_3osT9;oLV&%Aodx zr6G31-Bk_>;Z=@ngrbeea(8B-Mb13ThyRD8s|;)M@4{0MhtxpSNk}6jN4LsAgi$h( z?n!sIAPf)~DV-7nk#0%J(cK_QcSs9}zR&-?_`Zv$&hMPNPB@nFvda=+ZXYY@&{e~g zzsB0s^@ZtBP1~}7AH(WpXKJIPr|j+NT^)WUAaVU$UTBk9TAHGn9j511Bb-c{Gb){M zKxOgNNE(M3omP5E2xG%hA@0G%9|gJMbvgiXaqg&N_w4F5!ouncV0STZa^OsMxBT%{ zvuwup0p4slK6qjCZl6x2ss3lDPaO?4Muo?`@{XRY`H08Wu=NSGJ}&6$_m3$nvE z>v3Q8(i~Q&rF}C|nWz~7l?k}=gxiDP)h@%>ELIEaANXA2cAyqtMP zvb#u4v>VSDM-&wvBgWf821N(OsTRqH+omO1rxaMv*2VSF8gf~;eq57OC?nXE zgtJGmeNj)EVrHOS!kKD4iEIgd@)84)Ry7-Lh()M4Zx`g73^lF{_CLg|V&ozhBsiv` z2eyDjcu4aY#~aQjo7zkx*WQbPo|;!!V)-O6?9{kFwS@kHBup@;8N4+iLa3|xOD0`6u6pnRm=bWn|=FX@khBjHcy~-))#uuC^@|; z-d+E~5E>^hjHz70p5#w$JMHnkEf7H~f1XB?q>H+JJ8~RJ@xH$B+Y)V^p$=lfQappH z59X%KC1v&fP8fIV7a*0L5Ym|jDCYm$T#opY2x~dH$mMUER&Ew_^CT9Npy-}adks(e zo;v$Qk2B>gi({b~)p73(Sghv5!jsvWSW_lUEJ|)7VckSjcD~Vi=3a{t|E~F1$+3Lm z^^h1)UP*JsK79N@KQSmM!6tzdpQ*H8sNoS@ke?7wv(20H7@t=8PFkli>k+%ojHxhX zRw?wjS9nrWtyYdruVpUBHbvR&`M!gA$xsW37~cJ0q6n90d+bRzF!JcsjW=_EL1|S# zm*ceArMA)W?bx``sq zuzhNjL-MXi0ctTcRb&WTCZ1+-A0a$6CN$tI@m*6Jl8_d~G{WF0BHvl@knkM@qfQ;h zbYGBsJlxFD&M!H(R|q(^3Q@p5+pAe)rHPaDAuFr=O7zUE&X|(L^;$z=cG-txCU&|KG0o;i0l#c=T-6=gC&g<6fBmbW# zY1nS=@Qw03fKdYrlMXn4pW<2&u@{?uGq@XNa`A&d-%G^y{+IO=>s>F3DWLuM_#Z`| zqPL1*{;9^f>f-kyW{)D2Lb-uW{sSskBdtnhp=$iHZc^}H#(C+?wEauLpGYBlEJR{2 zNG|DB_vpda-=Fb+7YWgr*Tu0WBj%9{1Mg z6-Q)LzF@=K7svd+$=~&;bttdDk}lY?T$19AiR7qQkbJ3M7z?1Lm|=ez5YXI=pV&lQ zy>4l(*ZKaYl$0Ys+FH*SOBS3~a4>t2YT3%0dBCK&2!y7RTH2@)cJ4qz($+gFf}|65^rFc_b~-bEv4tMl z@A|p#9Bn7Ohaifd_Sc5I{_lt;6$&`JZb9{PFe98`-BM&~9VuIl)_PjJyRBQfWcDFw z9OI z&r}(RuDg}PucmE^MOW>vGiIdls%ajjB5K}5B0OlTZ1r+o`B^Z6PonXx0*(lkCsW~i zJ6>crY0NLhhw0Je4JopU6*`+oSTB-g z+5j8!u=9-0tN+Lv&0KEqW9Daor?cNH>S;>(-?L>8h~# zEduq{vN^@8pJP#~ty(yJAMre^;IR&-e59V~=t z_S0l7w`ihj`C`UZaey4=)8he4%MklV3uR-q<6^tsc*dgi+|B1OgQB4$xbs?rJ%7} z8&grrsvZ~oxD7f0AUnN5wY)R&+G=;X42<<_th|2p0aCI(Syh^KpyKa8qFm@}`E@5# z5(_vvRq1<>IF#*lhX3^3KZOsfy>ukR++2>jmj(`t)1}8iZZ*sG#SJBFlG8QSV}%`; z`&nG1G5AC*eSgqG5`K{3YVzw(#SfEUmZ%;NtVm0WAb;m|C}pmusLFyz2HCCz-?IRF z?XZuEBj(>9;lIXK-xDt`93YeZ%V6A+%tnm(Q2l#$qc|;hCqEo+;QAY(0(*uQpi{w1o$l}*)r^ZPbG zfY9m9_eT|t5m>!_fqcLkuBoA%fuTg9K5&qRUdL^JN=z4|cWo$*hHxgzhlL*SJY*mX zN{cnsq zeCArYeH9p9FbUv1v6YF^_n-2qQ^I8Y-Q1c`dY%^=)pO$H{R}iue%98HJ%3Fm%N`w=60A*Eq?-*f76n&-%0I*V@qE z9Xa^VgeBT%`N?|fGRs;gx&E5Yx87lkLq)KCU{koqW^U^xnoIp3+CF}|Ex~35M7}ZR zu3gj%4r5|pX@a6EWs1zVqIWlpj_dPF)7{E4n0Yut5;VDGxaAQXFbE!cKc7}dloAL2 z_*6RbwcTs`8?7FXvCMk;Wsyu2d%jq6ybO#L^C8c~E!H`Dh!l-|V$LROJL=aUB{2H$ zMQN)j=`Qlgf@U>TJ;;8*iN0{OZf8=%*jd>(of6koMY>Su+o&de*zBn~Q$Ly}mv|C* zqJVtneGllLg(4wQko~?mA~bfUq{Znh?fJi{@*nqmD8Ni&A1CId>3|7667$9i14EOhJ~0plQo?;!_dBXsJ}tEqTS3XO05K?Np9 z2>>N5grr|Udb*_+iw=#}e?k~vlC=g%GW_7WDTt88@K3v~SXi|Ndz9vLi7rnna~;P= zKWoZ+APNdhq_7 z`cdiK*m?&*b>Gp=knjCdsr;=&=Sxh&F$SJ;ZFpmtS!n4BpiS;!kKWc_J(ifS7uxB( z7sbwxwku#ZVbXz4N}&fSLL<9nW`-;UURB<9tJN!?>h#f=PB(?JRYwyDs4S?MqhR&r zPZyKwmC+ITT_LAZB3h@O2c^*uZa7vllqNovp-XfV7i*ZZJ)#mMyFXF!k-Ayg zIXMprqtJV92(T#@Hbpw&VcAcvHtyGH7l`0m}u2a}mwSVgRH^{9#t1&WAusfAIs(1W# z)o#02zHwLF5}m0WcuHAmU=3>l1WL9;(?!N>U>bM#jm)#()nR3y|H_t{y4JMs{6J-= ztJT;tVBpEQZNT3xk78tc>uwND!QS*FBJ|>fgDObPK>Q6pE}=2ut0kGLhTK-WHv#QK zhr_X}Y5gV!K<>s{KthUCeBu>ft(=GL&m->Gvbrq`5lM?`7kd)9s^>R&^4SdQmYGs; z(6|jGhV8abKot9MYo%vN#PVQGExKM{I_Nx-{@G zr`^#~_g9m;cQ-pbV^D`YsG&6_zki`)M^9LB=!H=`u__xnO`F> zhvZM%jHbTn{~p>FVZr1pv#jeh?#CXyVa)_a>DJYEbpiTcua_&dma)#f0A#2_s+fz3 zMd~sfh%&VF>=;>RyYm|cNK9ipcjGHR7WnVh7zT{XhbpU}5fRo0U(B#7QS+e=X$38p zMe{p68V>C8=aF$YV+7yG%w-V_f?%hox}`g?z)=os$z5x#}Cwn&N(~tBx4A23GPt zxL0i7^vPOn@^5s^a|2$-5eu!_TE{VWZOJ-Bg4S7(_2cMFd)uFOk{W}|oNTrtVVvuR zG4|K$H~}DID}xn46xmf}s9&D>cmu6QPL8jS&xk?&fpo`V!HOte zmKPL<&-%Ufb-o@pn;?OrGhEqlsC)O7!S+HoE!Ro_%Stk=zc{sg>ZZa!{jHG8u!u-8 zgD#m#I?bVAqQOm~wn{YJ#oaJl0joafd03p-Z|dl2WA2EY*=E`{cStjG`SdNHGD*eu z((j=>+bvLAB>>wz)U(~cQ*Km#aacg$>$Rj-&(=LF;65O9ru_cAEZrfXB)}4Q#`xb` z3Jo-Z&(=v_Jzh|r(-KiUJ$O^&N$*0;97g^d=%&klMC( zFm8bz!}#``|9RYDpaG$z?c{o@;;B(8i}S&7KW{5L$%0b!fAb3H?@Yo z{WuiPs2n`pC4+-~%nMFHHC>oP5cK;Zz@-YQ8pdMsL%bY5GRsf4yh_&t8b8P2NomhOlH+sTNW6>=P3eWi)SQ#wu$$`{oH z)W`KblCVtWFET0@NyowIOiQvVUVlN;xFR^54Ld7&W`<7BpGjvM2RJQ2P@OCnG0_JQ zX5DXpFB6^s_+9B`5$Pgad0<$jUInVo7R@CipBTu7IHe zzM|fhm-18@&K?9;G<*0mN2*T3OGVIb!Tr1Dv*h`j*3r4Ya^F6pz8J-L?r}aCP%$jr zl&;p!>)EWz;$Ob2nAeAoG!>CF>a{e%vjWVE;D3*i&6M#A+M^nZi4se*hLW}ey1_*c ztE7*}sV`oKBL3Z>+qVhEg%BU)Vciq#kAo^r zp-HPYA<;$kA6Vu}LeaA8HdB!+YkmGCJSZ3}fuS3UX2QgyxX2?pAd1{#_hVKOtk}Cp zb_0E|?mSR6>te~$ib+uA&Q``{t`vAD3J+4k-=7vcKe&Ueuum=9t@E^-t+^bRr7e^z z6`8dYA#qSzCAAtH_`OL9_Vkq-+UJw;VD_4 ztN|=w4x2fod?ez+B%OUmA#8JIQH!DuB6i+dZ$&65_UY0QwkDvQauSY%S)=vQSL9cx z&Xhf4{901BX%2I${q7=VwvxMp2T#BGX*%N9BQ}Lwj^b_UPwqK0zfnfnjjux!0K;J%dKB0CPTxov+^ef68jLR8H;+5)$;SD|mc%bKKjTx8DB_ zZ;?%9LsZ)L*>1pzuCUtHADE=F{J1Lve)7HQd2%nvC*G+&Rids8+ZRlQAnP29f+Z4Z3qsC*akB;cm9BS(`%~Hb7t<1-{ni{iE+y?~-CRodxl1f%i z?epD6G*lO+Z7Ox@SQpT zH17k6OmOUUA+(Dx5rqupN1H|H|JFE&b!E~K&@g$8Dw@tJ1OT>@-L%^E?fHH(4TrY7 z9(M?QSH;3pmjr(Zs(v*=2<1V64Jnh#F#(?WB$hB-ZogQJY_Ts}yjdgLS^}W47yE*Z zwky`8_+`COBMoFk8s-%g6;50NJz~1KbdjV2l3(I{bad$kaElS&FHPi#evD6nn59|E zW}B_^!CunMNCgO)Dppet{XK_uQypAK*_az=2$3^@l7~v2zel(D2E6QpvU`O##OsO0 zaIF&{SIQ1}N&_*4X%D_6e&U+4>N!yVsb*1XWR;Upx4x($Qp@m6oC;ypOnoqCZ0&Ok zU77N(*?4zLk&ayvpxv=>n>V~t-XA*|Bsih>RYXZA_V$NcPG6>zKbD<3ji+xK2xh5e zjee{omL1l%^ok((LC>^^<4$Hq-Pu=$I{*hJ{olRDwD#DnTiG!dX?sCG&mE+3g_!Z? zqX_Q{tW#eD+CeJb=wDytS>$GfZFKIhcZV%0cfUW~LHtT_el`Us1wbIVLW~W~zJ8di z28H5EEsBna82t(b{ls40AMNeTjg7^!ynk&#V}E|wo{DH}Ir*g;DxbG=-ECtJ2Nx^S z(P^CBmTy#@cnoxGkLIW6JkkqPzMpF$LuHD#qp;C`iuAbf>QZA}tIqAA&bDc)={6eJ z)#UiQ@5U~aY&OP%1f2Q$VqU@v~;}OMZH5M174JRgO@4rB9=Q@0VLrXBOYBFpBZuXV>vG zJ4gTM^?6eYYQc{~6ICs%Q&0O_?Ig^%uDV{uIqN!hHJxi-TxiooS;TjhUV6m%(tdp{ zlb~rk_Df-^vKBF{SQxW84;b&9;{R*CRJXjxdO@_bOMv>bQ&&skg#u?HJnL7g192U1 zP0TMMDcpaFK8mKk5o3c$r*+}{pGyXWiUMbgMp#%o4Ngw5rnXSE-c9I&R-O9Q#QxhK z95{%j!LNL&hopO2i?%3x4Y@R*7F5w*+0<2M1>Q?&H$3GQkgf80Rm^rKW3hfkH#9Cw zN*`Io$e~s2n>K{ReShI4z)hFk70A!;X4&rohUr3*@+y4tvq>=+2-X#kZZF(&1cG%C zZi@v8I}#5N!xYKLxz z3NzII4S}e0*q)hq)hy4Z44fUr1ka%il-=s7blWd&b`LiB$s%5WP3?Bm z&@`|f_hb8wd7k{z>t8(UzfF#p4s(|YGF*#B6L`A7fEs6qX3Sl#@ajyRN)h%^%tEmv z1l2z-tTdx+DC(ApLg?i!Jv}H?O;A#RH9)XOA1YIhFp=9yfhlix7tMzB;NyHlg9(mV zn^XPk1X1$pzt(idg9<2ngbjOC8!oec6mGiMviJDbOl$vWUG3?Vr3g{)g%&0Z>hb$g zQ==4!`wHlm= z)Pmu@)Z&+%ssWrJ4*Oa51L_Mcd(NZ~OxmvUHV_NR8S!-~qf!Sc1DPPl167nYy1TVUP4t!O`E5?L%WY@JdW$Xwx)FVUrV zb@j4UTlImIU%5K(h40W&YR}h^2%Aag5>0?suQi|gviaKQmv1ig1TW7}su^Jlb!Orh z$Jb_3j5Us%qg0Qmz+-fULzN5BHf@b*I^ibsN}h(ELSL-s5%|HMsYm z;!38|WILzi1=HN&C=#Z$GFlp3=rebK@8nb!d&I1W061Ns9Z53Rugg@F3~64RiUasU z1H5tymf5H*G*zBJ90W+373E|!r}Tuh>_l4NqzKIGpxjkzC~1$gpta}3Ro#S0y)}(@ zbDRQ9Sc??UXY8E$P+uW_p^Z09Eu06D=6vf*aH)vedfP?Qk)+#9)m)DHid?!SNY6X0aScpO^rMnsAirBWS|U*daYzn*I!1$pUM=0`mnQPFuIr(Q~8uu~*0RVj682&f$cg zucPkqkXx??`&dufXfASKSH-YsK5Q^+s_flB6jDyc05+%IwhBS#Tl?&e>w<^HlgCe5 zP*CA6P3?tOx9^wO0-~}W4~Zrb05*9!lg&Y5$}Q!zq}|zh;bSBW&STxMYP=(`0i@vM zD^q2kz&f6?PgWgAFUY`zRia$QcinIaRE9tmWOPg8_yCzwM%Jv zNX?M3eA6Sx%je2VL+vrRT>TH9A!*K2`<1GE<%7Ol#eGOg#rVL=jCE`!x60jg{RLSd zH$j^3Lx;)X!F-k>TaPz?v$f}OVN8kC(V$tHg?Cm2&&?YrHiOSH85Cy8B)!n5gy?2} zTbd+u-Fik^xXw)Q*gW`v>wVQNhST=iME7sh7)W57D~Y;L^Qk21LrD7vu^LTh_Vd*^ z!>=Oul!T#)x8#$Rhm|cAGrTyBQ^G&yPcf9F9{4TifaOylTO}>^?X3PS5|&sQhvbqt~2!|qS(ej3CR6Geh`R;;NF9YjIJifyr4_(FB9**{vMsq-p`-VLRuP> zvO16K@8w1V0HX+gsN%NW)cr@(Z;t3;0>mN6p}S9h*D-o27GM(k zzVB^rKFaE=l4^~Z?oNk)?_u-eT%)09g>+7Ds=6yPN$cK7pJ4}1@y zKnMMrA|l0aN$qN^$i^a5Z-IW1EX-mwGi-U)>(h6NBOiv2klaTQw|d=scMJVxsxOrO z>F3p#Asq_Uehcka{+=Sz6u!JU)RaN+AaMI~;$NF?jW1q3IW()BigM347;Yv{|Co7> zzS2zsRXbQ*JRrSLX(*GdExszNKj29=I#$%(e>?K*TUEvEP&M~JSzY{|a6IDdu6O)P zgxN3~W|W8&cu2Hn1Vp<&z15l*3%N4_QlfhH_AV@hAjDRq>MXRq49JxO%P46IZ+j#?bb3NLkw@DW#%}pCnO5g#Jcc3a$o+|outY0(Be#Lkz3*s*nD;G8c~ zPEz`r_YNXfreFTxu({Ovx_v&YsP;OJ02Efp^NSetVg?)AI5kXnH#?IE(!s6v@oT`D zzhG#4 zcDfSd55-Q`0|u+O2YlX>K}&YuH#%zCv<2?vKjB+4EAiRxG zg4fAkVGH*lN~b__;7^yv_EF*ZJ3iI%cWKVJ>2GVqBgu6bry;B*2umHhe$Y$E+C{$r zY@wt7EJD;$Y|Br=^a84~=Q-g1J)=u4fQTJ}*h5lmc3 z_eayXzf}3S9bHn8BHOKIvp2k0Cfuz1;$o9=we8MXYd|-C5KKvU6nZZZ0ZK@6{_aq3 z$@QRK&pycX9N}0|lfkC=?riL~m@$owRTP7IKfDb2($g-Vl&Se*cq8~Ciahvz@q36H zhtKc7V+#~-{vDx*x3|yxIs3BGY^prj zAvsrKPkz=9)2k$_(e5~ACJp0%6nfPskoHCLKWWh6e z;`S*@fx1V+o-&T5Srof;Kh8ew#Q4zLd!sP08?6X=uI(;p&oyrFKO=Rr`B0zEUYQG6#1z_2s~cKgu`} zvR6!HlH)6JX%^F@iH`nJoB&x-Gy$QdMnW|86r&(=U{nNgck8A^nFg;g9?!s$d{L@f z=nllVLEh6gvpduvC$tI$cSlbW-_%aG7&TWwdx#fw;&j1Gm`>@J$pI6t&ZR51_7QiI zA#NtdI8@_`Ys2_*d^ZLJ?CSgvl#Jdi1lzZITC!U*qU}+7nUq zMi0M4i(R~nGxZ)lsD!J(8TqOEIREu}-&+SJ>z9CGcTa#~1?O3T1vVZI2rn574W4a2lDK)tH&V{|XV%4_*F%)2_Sa;- zp-~0*z&sQQtZ?a7xsg!CM2XJfphJ&I*h) z-GQgxeRn4J_Fs*a)S>1(dz9j?h%-G2dT4u8eCMBg21OI${#I}AP2s1lSYOfWeEC=; z`(u!pZCIdPJZnTU^Z;5*Pf}Ea~DDuMG`NUi1G+9Axr#tuAh3 zvFv(fliiBksIX-h+Q)Cvw=Z_6%b7vmCW@ zIDaP56TY#TotwakOV9!{e_&*b-o2{*Z{6@uE#AYU=Hqqs9wlj7L^ogqkZbIs>?C1FVlRQ|u9>VZ^N>?X(HN~wp)#tc!?rx=s+)hz$J7b(g*+ zZ_(0JM=pAonrp31`bUS`SEtYsHOs}ik2p|10!FqpaF75!Uhz^S(m8AX=r!?1e#3wb>LmP;T_KoMzpl^dT=V$EZs@ET^gk| z&jE@T(^dUAJY`vB6v*!)z2j|!6L6kcJdDY$jQo~vUYfeC#>bc4UoAD#JtfS%Ace-F z^qf-7`x;V5B!rr!OvIkstwcvl>lE!}r}(+Yb8~eku7FJmiQR99$(xE%J;1|`mIp*5 zW=%?*-hGx(k$^zZ0_Yk|dBrv{9eXyE1GgP*8Jtw|Uul3EPC0F0d+sk}2)y}K|JCig%?kq7BNdt?tt|SK zMi}BQLLa6tF{>NA0+kfmK1A8#2(4re-1|-T$4XFby>h{y{yBMVjcRiD@>u`<37b1A ztV+D(Rt#A(=w1HtGzh|`gMUttW7L7H-gAvLG!f zdY$Uw%dQwNwK-!cVu7VZn0rqdq6D-@LH6a|~ z`}ZfS9J_X<4o!?$Zud{@t9(Rj*AuQ|pqeFF*Es*=t#!X{YdEg7RKOr{#h|-r>pUL* zUBM*|I6Jx0fzHLgN9&PSeW`BiJKDuvQX)#4OqX1$IsbG?&;D%j<)KWO%xKLB#ZOc7 z6Ep-MxP@2yyUbwPK9!`mBhvGk>3zCcYsw0^LGR9$6roM(LbhR$-)%ExX$Ipb@YjzL zXunDz=%1b%*bj!@CPE91LaO9Ek{%%F<8junV3lL%B#;-FPrKtBoO_GcS4ZL6GQZ9E zNQPH|4h1MA3KRJYN|Qw8X~rZ>lb^D05zOybb8(GK(bku1P`mppR!!Q_Par{{2%KV1 z_%s+=GpXm`hK7RLKzWIf+@HHVZa%`ini+8W!8hl&zT1`9X?bJu=W^B``Q7(4uX|4- zgWG{=>cqg-e(;i;zSlL>DTz&SMX{(O_Q$)5I*T^Q%jnHGxSRCMgq~vi(0VEvwOu_#?<`Ni$Xqezu6;IEvoMO}J9}+ zw}NZ;bu8cD{ZCvTD2a}M$UtcC!;yFN;=G_u4Kl_K2ztw_m&BQdM2xCcy-P|LuKx!$Asv+u=8~h9-Pt=i-WzLf$;h|PsK%C8 zz6?_&zQsy7rboe{HspDzGLj;u^F{fPFsu<#f@P&+c^rp}I!@EtMZw^{pjCiuug0%ac%2c!WrEk8QH3^z{ z<1a-t6&9dYq!3gOv2E={<`XRNt`GgcxUaTEE2Asy^(jMVto2)D(yEseEK)KeC}}K6 zUL)zBpW;2leGfl5IC*DkKp*}p3ob)JO#L>zac&uc_JBWOB}_xaewqy0wQX6AR@=}` zg5xm;u0EEeptgJf>X5gZ)6Z%_snc$~V&xW7R2G@a(yHV>tYg9ib6EHZ-m#&(`$}Dh z`;EN(!52*ml+WKglX_LnGd9^)wH7`t?~)8J9_QSmfhI*7IWl4r;u{>C?e^Z2-63uH zF?f6|xM{(#>!Bwz%ag>;YxLZ^FRujxw`2Y#+_q*1hnA48P{OExYNCESg&?i-%9=YW zzGg`IzxBpVSzbEXFW<8@$8+GSgl>Tbhs(7Gigla}N6fM9N)oBS2;pS#4f4Qds^fi2 z;+Vx&17BU$LC>E-Wl!^3QRwMo{XC|N=+{UJv+4c$z;<8~Cjd&fy=3Ckh|P4~KmV#i z7cfNPO`}VsG9MUOq3T#eBpuRV_+^%NDVw8r(!n>9!v^|9zWLY}bUIGG2c5By%tBqN zM}DHK%-`&^EJHtvl)l$<>?qWW^#0bVWHVZWMs60I`QnGLYz~A5(M<*DKP`ts7l-CZ7)~2S|)m zvG#Slf5)CO6XZ`S_ScF)J);?}M^`#DHg~c)A3jxDmsS^|31xxgMzS~Ip&wb5tQ*Jo zvmT#rCMRXno+c9o8_^X`=77z1xcJPr_I{EeQ-j`xp$*nTiU7h+ynB|qOEqG+s{1Nu)mN3ySWoRC8Ua3&D%Xr4 zKPXmHdC1GVCh;v+>hIuqQFSTz$O-q4xgmu|6Ua?*wXy&wTiqmg!Rr6&M-4>2$iB2c zfEbOX5uut$Qwn)pm_ivZr~9zLdDijT=y!Y|0R4eTnWdZqRkO}FMjm2>7+a)(5zz`= zGi|ZxqQ-gPI*du+)8|&O&Ln2Sp%jfhwO`|i`ADFjR&yCUFuAq7N3Gm>31*}w+(4?j zy_F;xQ!jqZ1}|d26;UbszTJ zcv#lySYfe0gauq;Hs8Pt9^DRIzNcR3?*hO8At8lVH#i}caUpZ%HRYtJqh=S#WRIwR9gwZ8Kp3G z5ziK@*jgHc)oVV$uG`u-h!9zoCZKlfOQwUAaQS-WZa0^{GH%O%1PnbfAJ4zzjF?S_M*^yg*Z$6N9q(U7f+}3N_ z)`l6k$-DDrgh9?0mylqS*-gh(1Hf$7Ww}#rmvs;%4>ud;lTv>Uxi{~Tbi}adCof8- z$+PA*&x^yNy;M%4V~xr7=wNze=jSDF?UL_+#JltC1+sYktpDMgewl#Rr#COG-CI>t z`RA>Mb+fu{7I}gm3F4h{PIlAn06Uv|_5WXJ(vhe&Z9)Gz2jW#EEw=J`Kxo-;-p1N` z@`-vS@|8zJ@Ds)9*>yWCB~l4?cJ(*|Z)a85BqCpFP@n9C{&Rdtu2Ij#PJ8&@OD_)_ za>0Zrq{f@cE?C25qE`uJV5HL=Fb&?rmeuHhvF(iEYVysat1tinlLM-+^P*CujG@!k zQd4I|swF+{H@xGmU982Tk*99@fyAxOJIGXS^B^UJ3-&IU1v^ZJPF$w1nMP(8=4&BI za%uzsBoTt>byrG|w>_A7#QX-hx6?3`+e;!2SpJ|`GOEZu2|S%s3s~)n#J;Wk(V=zy zyYJ~91LwY~t%;{58BX(bqT7?OJ*{WQe2v!Nyi)%ve?DY?%>`{=PkbBu%UvEwr-Ybb zq>{gTR5T8^wEjM8ipl1LmvxaeHCz!qd}>!a^)ZOcy;htGfYTVRsabf8VyUPvF-}cRwcN#p9?@4+xAs^liK!KJ14#pZ0T*M!MNi=w`I< zpRd1yw}AnZVyNJ!v&-QTTe6FFU)z?`HhEa*IM<%r{8GCSQmF_!q2rW=>=sSx1X0S* zy0h$*sC!JDaQa@f+8_QES1Dqmp8FjEA$PAfHlF?aGxm+;rhSU89ol&rd9uAs z8)Aa#yZl1|t4-)P&?1P0tTj^KUm5*%9Q36V^7?}^C#=ijcIYE5=y(P(4x5l3o}Gw7 zxZtQ)ZFI6Pj<4HH!z;*|&9VIK18hwlHGlnVPX-(T^*$A0^lnMq<)|Iu5ME0}!fPpO zgW#f>HTkBua=KC@Q!Rj3UvPO1iRI52$e4RT1l%pMivNnL&_M0if(N0?YhjFlhYV`! z_BQ}9X$pNrg%`JcvH^|C7fZe2a`mzsp@~^1`YbNf!K}yCsYBbApKHJY3ycS$E3|w~ z6ndR7wCNk2TtZYL^9r{_Zf@HYy(DL&lA1t%7b}IA5fG4$zM#R?5P@ERYDDG(2aE|3 zmAmRlI_2-e(P->6s@FiVJ7hAw_XS*@i^&V0GCIGdlBrUPTT+H~r=aFdYVgWg@ymq- zvcs30_S+{Y21^HBzzJQ%$U`EE|`Dc*07TnG>_ zrazo~CkPNF(`Y24A=)!cef*a}aqU2LYsqpFD~NJ4b=*ae?VSTrXEz~+F$j`l`s{Wr zf;b0zYiWI&kfXh9)yHTVF*X~?Kdm-ljwb?yMAI-EpO-{X9NVUD60rApBhA#dGzv%$ zc6e@VZ=M*xX{0sN>&}nEKwPF~76BDB>F3STo6SHHX@k+(u0Y2Zophmt8*rT@b9>Bx zb4#I4*`0;adjyzR6Y8(3r3RcrJ&f+aUkAW5g#BZKpgj&)cinG+T7-N6TCS8aiH;3U)Kp1hjexCKDbNXp0kPCSu%U@ zus^nh-&)X{dG#-R&ZDK!6E}u5bpbK0alg5G*k}3z@;V}Tl;7b@VE04yxPtO@KNLW+@N<`2F=+Fp$dfGhrnPY40 ze%Ql3kfK@NTb6H1-nsEbQ#Z-H#S8#!=RGWRsNS&zd?EuOX^U!5D4y(KBM{B~+^4x}_^~AM z1rEN%U~)JS-y&Z${`}K?%5_MLT<|T>3rl4q4|AWClx1JDgBkEZn2_qIec0Q?hpQlgxF^ZV2Drqovfx8w+!P{s5#w7y=Y0&*bG zs13cou6J~07RoH;G`^s;-6Zblblij(2v=arhrBsxyyhyN0;0=|u!dM;>{l)Am>Ru8 z!bhNdY~MgVl?)*zrgii8ds@PfzP_2s-FoBksv#(@mgaQxW6g+|cKR0wKr#k_mI}w( zd|h!PJ9z_pIjmEkUDwo*K(m^xDaZL>QIbO2(!$r3#C{}3C(Aj@Q18CY*Zbj+8Is%2 zKPLnWE9H?(ns)JTW*GnQQ?cd2=;%g0Kf7$m>(NgS0H9Gq!$*giobZ1K83(G_ zyZmMO0-bTCb=4lC_YqrUXI}jBFjwh?4xq3t-v0j?{sA61niACwFk1Fl>Yz+78c4*& zk4n7sABgFS3Vn%!%(XPr8{W0F^io*+Gqw;o2X#dIno-p*h?QKY7>f<}b=l5PSamOB zv$Yq%QwEuSpKzwQYG-rxSKtWj&HQfMPlfR@Kn_K{+b$uKRUjsp66H(^YN=m7lD3tK z=YPTVk(||1eAQ1Q83!!0GzEd=pfm&`Rg;|!${&^QiGT3j9F}ege-+>ig~(v!)2?l6 zJ6pfWuiOPKO%?@+`R}Y^E%kPn^POh3(_Pz*^``L|-&9LT;bj_5R&pQi8tYZ|UlarQ zfz=I0WI(YaLF5WIlSnL&S%M#PCE|boZZQ?R&D5H25R1w<5mQZijP@QDnaKg1n}e74 z-$MvNG@pz`D6WzDYu;InWeYZz*6ak?kmHpU%G>1KP58(mpE)Qz@b$fd_pM?&K@c-# z4)?jn(}IS3WUA4AhVKqj2gFb|)9k-W{2zC|K(77;C*%aU`N9Od{mkEM4rgW` zG@p)0<@0_j^W7|d4M{D0&V^e+fN?d?zAZ2y;M~lIs2vF*aHrM3{e#8w&edy>(aDgb z+xPqX=EJiR$tPs+R0t>#mDZmZvvSnpHL}m=KMZA5rdXs!e2#g>i#z4mM3GlN4i)|c z4A{uq&%p)cjThgDMW4=iF=nM7L>L%C5c$|hl~VW%V4oQXfuJMCPSYx6Jw$XXM!QCj z37-W4^PwP#fv0Le*ZJ&@+i;gN-#|+2_||f?zX|qnop#gjEPRToMNop2Vp4907$Vt- z#dC4|V9_yGsJA+RgZ?>cBG~EZ#&^kPG*j}OHy2#cVrU6 zpS5W=p&t)~XrU2w$Vz^b`z^VAP@}!gcVgef#JmfdJiq9`zNPemm{;>57T#pDa*L2K%Fh?{)Fbr?TGuOD?m9s5ayv`waZ9o^ zZMFN3Z8zgx>LdJKnO&G?@};OV)R))gAZh@{cYP>qi$?pm9U5`9PR^ zj^6X5j*E?VfZr{u6*#`P*i`m}=>sG$!N5}VL%acu2ebCaFSPz~9SUwSgmJ{VWfe{c zX0uN^3LP}BHG#)a)}k0-HN*dC`tEo(-!JSaC3cJ!r3kfmsXbHGuTUctwMW$|irR$M zsI64cmY}4lTD7S?QoFWVvG?99ypP}ezMqee|HPB~e(rOhb6wZDPT=*+FO?WVp?IXNK6uBJ-~n=2_9P6kY;6Uhl81jP*a zJ$8@*L$S~}o9koXu5u2>V;r32HvK@1A1(H3_fhuQbRF#Pt*V5yl|DFf=1F>;P3t&Q zjCAY{LzT4K6Z#Lz`U{XxmjJVDjuno5Z`ZI7W(q74ExOa>R zVZN2mDci}kZatUxXxS=K-WiNl&r^(BU&xc47ffI1M+#@iarzdaU|<`pnrvu7mT882v&{(Zp>+ zKtayt);B2e)nxTq#>URS=ZEqzky=Mq;&>AMQMpVhch|za;eO+ zOd0x5x6%RHBEg?DLKIc5>9UgwZnf=9DTxK$UcUe=N0)hGu96#^B^JzUhkxZ|>`<7L z2!kVq_}C!?@KUG$S8g;)6gq!k(H4k=Osp=sIfGL8?c}wI*RWSD^;-ldlsol<>CmhW z>b+qrDc;n#mit5?FeKa@)Ow45#LCE&#~1`!Maf(Z^OUVE0Jt^zo;#g^jb`FX2V-u< zCYvvP{7!S7YvbjqD8Eevi_IHnRl7(U7TT!apyyBjED|s#&wgQQLZk{Nm*oz&|C82w z*f+-3Bg@{=c0{0>be^^{_RD`!jT6&_2-h8S$k;lu3(1`GUDDVlgzjIGsrsC3OC`hg z5U&sq*F%f+cny!ktv*45dIc8UW@WHbKr+g;~77{foOv47PO;d#w65&o3$^r8@EPd4bPf6ho}wT>|}E?f6M?`QPgTw|W2+an(VN1pc2 zpS9|Sy7ymu1m?SHf}!w_8xxIzH`*S{AP3_~sv6d0*y{kaV+@nILH|LTV2?6v^_{<^ zet6jay^{_i>-RjzbCH3}tluoTnf{bLJ<#j7#hZ8dbXXu2b7Lbt(Bw?S$m31B>74i< zHcbav3+8Fk zG&e^df9FDJ>4l4}QzyUb5A=I^Xg*d?0y|9{W8=W`No!^47-jz*1&mJ-%iS8<8DH^J z?As62S$R^nXK}n>c4*RIlV`pd@q*LzV-YSr1O(2Ia-q%kqM7lG)s00}SoR?%&-nO< z?apk&{&Q9Xr|j6zpP74ljwjO}#9gE zz{Q27hloJFngA$`sVwv&|sYn80g^J z+ae1Az(=)li!Y;82Q`Oi^4D?DpLP2%WN|+#`v5yteyo z6teTDdS?gm7y*rt2j*+-%oFp)v1<<;(!@hc4tneys&2EZhLOd0wFbq(vI9tnq+Ob+W%LJGbqbFJ+S!G$D*-- z{PJDB?{xV~g&7e|R2~T_RCT^WRvovQG$_abX&GI=Tv5Dmva#{LFMBz|dyo8v$ZBIG z3`S>MOl?lB+-1m!CYhG#HtMS?X%^$Af;KGZJUB8y9Z3L0%D$j;)qst5X_Q_Rp}V|zP+fAEZ{F?l$Q!74_9jb>h&%x2|Y zLH0~=xumafjyQ>vY5v3~w$fdMsSUhkgersbw`1hfru3p862R5m)u$fqi7%k9(h zL|)+_64SnN5u#Nr_s1h^xZ>z#16UQziS;Dx=c3(k!X-*byngc0R?LQpJldEq|4{Tp zt7qhj^;k32698tND@o}nArCcBb7-xZhb?Jh@_n|AoL&`GRS$2S)TXTnYP(h#-Jrdp zOfY)p?Ehp(bM0`f1S*%7zc@SVgM#xe=VEMgC&KSfRrppzkNy8MKd#xBAfQ` z@oE^z1_ZUN>7+|Pd^R<*WjpdBVfy)_)hLO(f-9zP`o#PGJ<{&25JRj+g-zQ?_iyfWKdk5$n+Fa2J-u>7Qam zJtNpaq4%HCX$lv!`8oZa5ap))6Z7I38ZS3?)RM`f7VGZtLx)i$`MZ4aqVJth-VTZG z3E!D#LnuLKPubjykgx~lod=<%X>J|veP?f`?YozIAj+`&V?sWxDEIDL;Ri8dY97h$ z5O-*8eKVbQ|C21k{8P?Y`*NmaGOf~8HK^zPv=#o|cFU%^z{(Q(;>tk-9*so#Zbsm; zO)E9!bJMs1m}3$)XmNG7Qzn3yvNHY#QTyI?sr}=XU95V z`67mzvKk&g#+`f`U#m>(Tl)UnLwa z<)IGCsNWFlO<9nQ9mywjEP?R`3bpdOV1JU%%#3k1@G(&!_4}ktJlOtyR6@f^fhU`pFei z$bSBri0zQ-1X-5960`w~YNP>H8JT!m|88tMN1(u}bT(PT9O@pBk(2))Vql}pJfrYu zN63TzGC(}oKP0i{I+rJTOc6taxUV!)GpLZGzx<67VR5v5{z%@GdUb~FdavgZOLHws zVgC=OoB~mI?cD!jo&^7Z=q=`cWwQ?i#EhwKRxw$oALX%l0`nK4BGMK{bW|b@=p9c? z>?m9S6PjrHpD;?2x!b534s&Pv^0 zyg6NFRt&W1IeQez@FhzfdYi%epkA@a8PGho`_8y{UthQeZNpExekGNeb~0#4*3VJy z)R{jnH&;>Ii&Vm`L5)UZ|J=kt54i1U9Hbf2QaC~5LLu7pAcn`#?ADPVEu0ini7d2S zU=RJ{rmhc5QZt{GrFlQ5vww8Y9#e6vpa4HIvQ;1B9HTqLL zX9eb8L^$_my`g>YtE4fxFj2qclyRUPBPhn8btxg8HTovYki5w<`U+!z^^ti6Mv&^P z8#vzb3<+T*jEG^Fb5Hj(k6dN{iC7my+ZVdj-;$Q%1=F-J$^nYZyV%RjrkW^NJnhdU zgUJ$nCm`?mRhfKqcvcLU%XOZZgGj~HLY#!?9Lpe6<7?Gjj2f+B-I=CZq*yW%E1l?_oVhi`=KqU9wOJ zRS3LBUdsUt1@JW@d91{}u%;@u|2-DB<&7Bq4Zf*Z(xKFmFnl92G{KPVC>leK(GP6Y z5sp1W-gb5v;CH$YrcvTe>9%=i!8mgZDu`-%RS)?e=OVcA>i>R*e1G9^I^vFX9oO19 z^h(r{H=E&$WvEq_DJyl$Lzq|CoU96%L$OR z-ywhZk(a{Mk|MYXP4>Tq2FQ~Qzuimav9G_LKOScC(wkhPnVt`C1|4%YiV2?z|Eg??)G)UfN_O1;l};7J8H zh+hIre9gGRq4 zDm`RM>T)u#J;uXoB-Tmp*gJ7K^MtB18WfClvJ0ABnSrhH86}wkwrZ2Cb@aSS>6j3A zT4+%*lOorpg$vL(;^W%CsXS z>!Ykz1(W7|5GsLut&c$fgSkC`g1sMXY9aOP+pZNaE99sqEKB$2?-&(`T+3( z&ho2&ojJC}wX)2+0{_opC_~6egMv5^q@1NQwf4?vRHy|AQ%7_t_{2fU{8|s`jCt}4 zYr=3LV(2$2Dp;Gqb$xYJe6cUph@U$N znyn8yW(e|`GtpMSw@C#~a=IcXAG4i?p9CIzWL);`oONVeh+JEp7HU=DgRVcY1>p=E z7J6NSMyi6w#uTpFvRfywH$|>?r+4u4D{=DpE!^2V5#O1!3Io+CL(ma+ z=c3E|_t_S8(8@_b>u2%YqLrZlE@m}y9xWC*xB2lC{A?9v$~_=vsd!I_<0`f*ms2Lh1sMi9Ogm;RtL z{(4n=Y~x7j;y~$qOlg0*ac?82BnU?Uw@>V7JW#sE_Fo?MOEGgc9?3Nxdt5Ksnd5(q zJ!0n6Ry-OHx;o47z2>yIUfMb9syXv$JQPr>8q&t!nS+(@7I`<2fOeZ3uQzrsnqA8_ zD3U$5fa1y1Oa**j323M?yFX&Z01?)%Q(qEU#0TDz($vg0516PTv)u zefO?EP^jbfZrtf+^Ib^^__uR4<4OQhJo+3D*ewPfJXZ2ropnCJo&2RnUN@SUnK4}r zXIq^1lrB8I-lD!_(YpD-)Nvtf%VU304BFO8U4Psn zbb`Cy-JH_=@?BiqE<3NpyZ_zX*2&C)a=tz*5TZ|dkM$>K=W|QZ`f}p0xi#@h#h!BNq4MhC1j_vy0 z{rg{Rrw=#V0S_?FP|KJQ__Np@NsjKj-)+%bhnX%&una0(sdP*N+X9<&M7FrGz#VEH z%F6nEwB%DQdVt%un%U=d6F_zPE$HOS(}oX>R@)U$X@`p{2U!IH{GGO0l96O_b*JigrM~}q6G}v{iJlocDy6!v$w_BD~5G* zKTZ^22tm4@K4DagTe$3Vy_{>u1^yEXK(AmsquKJBn~>F&(^(-^_ho%1m#QpiVMfUn zq>`2;Uf~-g-z8g!73iMZ8|lc8!8U<)hC`s6)zX5&h>*DLmawjHq&$U2hi1hjIoiTj zkcMMuVzhReB2*O9uwoeU=C>)DY|L@AG7iBP40)E}5p^4IDY8RM-YYQKYI@g7h1t?t z5vn_(bX|Kp31P>&aq|GWx0%hUp2aqL&}@Eg#_Dlj%FF77<2vaT_}4)H6KdBNrfDK6 zW2=R3rTO^^8TifI^3hgv-gx9(+2qmBn+!7p$S~9As)c_J*QST_jXJosA5KVGjz0ja zLlWSR>3^69#>n=CBPUKC!}3;QRp-DfrEwTaMoAKzbrKXW%iiwK?hYq6{bI?+%D3iHOL!JyOkzi z$6T&lA9fc9Z85W8@$DAp|90?2Z0pByTC$>9C+~(#X$@N@ua%NXRge-W)41>& zcYURFymsXHRhRCDF%kk^8$G+&;*k{Ng zsY|wF1xvx`-(O2 z@->S%9}0iF{(G^XH=fef2RF)EHn!o5Ua1uZ1$$ieiTJmA2dz*eml}PpM-`(yLoV8Q zt}YXvvi?ha9Jnl=;k!k6^13ri&ONJR!El(w1wYXHWYkL^XbC#WsxR8$rtKEF-x z9c$&g3uG`nYUJIYM%tH<1?gbEEsj=C2+tm;#Q}y|;qZ&=RrpRY7!NF{bIib3++k%p zD*z;qQr-wWwOUex+1r-XN_ME=4kxR0zj3!A6L?WW0uC&t_m&8214HaKw-v!$uM~nk z^cOf}A z(rEP1Mz6_!!rXJ{ohES?pv+)UO%1(m?C^I$VSACl_=jxK#lDi4E#PP8R%%3|?1A>G zi#CoH+);0?bnK~m05`)4paklGPTSt)X5JrieUH+?K+2wJE}{eX$G`RU{OiKw){-=R z(SYBsUf8y{YR65j>E7rIB7h5YOMYG2yrv`UjraXbWvn=lIJ4+aav3er+V05;w)ky# zr)Pf7pN7fhMS|Z*#E~r6n7tnu9NZfxrr6k@f|dDZ{`*~KG^NKYVKz3I*n0Qd->jGJ z@?HDOhc$hl*OIL(!&7f6Y@7f6qZl_xIQ-G+DQniFC=!X-JKYr}u^&Wk)4<|BWYaos zyML#WiQUVqF7>#~Vy*8xg7~HW$uWWJ6IPU-< zUOGZ2SD&kwRJ?XFLza;~%H#@yLEO#?iRo<4bdK3Px~u<+FkC|>&O$KOfd7IYHTk7C zB&;PF@FE^|;=)4RSg7asTM)ko@4a!+FuW*!KwI{1GVZJC0bHz()Uzy~Bn- zngsfuae>|aDR+JTW1j$To7bP7=#Q=VHmsbb*&)0Dg7zU_4Wb1d0jQ+Z+Nk-D&aZ$- zm5`ti+p+2;uS5Stm;T6)G=+#GamhMU922tzat8jNnH_L>xd^B?+0*Q%y|Yw0PGdcL$_+olm2iB3x;<(x ziLl@6n!ZCw0}D;eoE?5TwWn}3+%Vt98u7PM(H^4Oz{bnvU(dz&B-caunH=I_;bNuY zb}WSztb{If#NFZ7zu)!$q^ImPq2*_em4?|yF=Sz)S_fYK!D7bd_aBfh!0HnZn7}hL zQjIqcK?>XhQg;E;1bm@eeyvqyw0%E_Tk$}Cr?Pr=^YE`1MO#v`ql<^2>>KUs6-c2Z zO$W%u)d~rQf_N!506FS+oxI*2`G#^Ic`bv3pN)m_{+u3pY)_8V7G{g`N$wqmgjv%; zDXPv?*@_aU7Q#QgaK}A40rvf%{xv7q}<6L+vR=$s@tQkn>PIL5Xd+52dr18 z^=C2kGKI}NDBGyWs9`9(u85s`{7A>Bt(Wv+smPSKzvm7#bk9pfB-m<^*oq$2TC!TJ zf?i1z0F#FPx2xG@Xi)PUX1ac=JI;&pmipqYsWoF*@QfpLz*Ckc(?G~T;C85TG6up6 zZf1WKqnC|W$yh=P6~oHj-lGqoyU0yUX~3nt>Pl|_DPC4G%&OO z#GR_^Xky0Q*P_O~`ri~C(a1(g6ruF!#ED#(Y@y*+jr<0?!tOn7KbFz>5D5Aq)o>t8sEZY3}<+Fl!I7lGDCPx4= zLh&YVrPiuO{?S9fr}fCm@$sz+W1FX1L>)IX=6)>k?zfa^@GZmGi4IJ`SrpZ_t;Apg z(WZm6Chrbx)Wxwrz{h^?2UZMt$n#Qy5YNXUH+?}*JM^$e`5T7odxw!e=R~6CchR}9 zh%{@i36H-_IFc`)uKuX%{4l>(U*`%TYt1v5{7||hZIv(atNKmOG+Nhq`iGMuL^(pY zMqW|U%MMcu__tf+$AnpxFYtUb<-bM9LYDnkK?CKmnMy+eb0~#^;em3T>f_J-h^<%#Mg2*f|*$_7C*~HX@X%U5j zVXM;bHtsqe1=f9M{W)?Bz;afMTZexEUerjCK054Xq|CTyy5NHG*9W$yQ;rJv*onX` zq5#VG`AgnZ;Dt}&dTiVPcZB+M!c;lhiN?jQPzHL_68DwmMja1jHuKS;HPnN`hRU=k80kGBJ#l}a!@K`{#f z>INV5iZ6P{zrvW0g0L@4j9bem*^RFP(KVI!j|&0t_`^i?m(wiNif!Up} zmdZB&<__z=3BR9}!I5nLf$IUE5%}|l+9t#0gOT zB%Ur*r@QYj{fRqwIuwe7_hK*n2eTx0k527F_lnHuP#BbtuWiZihpX7nvxw=}Y$V`j zQGI*Ks90l(LBSY_=vZT&h`be9xvvV=K!AjMEnGp%e|D>Y6>uaCI}A~;cABqx^51U9 zxZNr&@3KZ=jDe2bzf#A^=T=+LP$>8KLFR&7Y^mkCvUH8oM8%lU7=#<_>Uurv6e>pb z{_jZ$9P`yDG2D1#;Js1(FE!9#ppH5m!r+9~%sV4B{ zV$+{LP@IUhrpvB6KPD7GOEk0kU&T;te_wgjZ4#ScL**}S9l$(W2TGUK5#kW`PzddN zQk4d6!g#H?l;iE9otfv8;v-FrH13asSb$w{@x6J+fwB+;<{yJLsg&N^mf@Y~{#Bn? zEc|OUu}6=+jgncMUGw9@AY!B-UUr1LP{OK|>>DlrSSA3h7d^J9EW8E%Cc#V39ZO0U z78x1o?_Yn049Xfy-#NaUv?%57LI{i>FV<%jNZ}z18?bg8_Hav>J3j{mVO@8u>6iNi)ms0DBr71+|I*LlA_od&SAU*WKWgld4N#H=>1<-EKu{C~Iz!0gOn#0~X=zUB7VkJop2)=orOW+t zjbD-X7M!v+xjNt5e^Chh)Cogz|CF-C+%lu)c&l(B<@#YxSO7M~%QEtXLqGmo zYnljo11-;%w0Uot`IjRvcRLDIFb7V=*BlzNu509$_I)EZsk;|$`+!0qw7T{E4>`$} zLebZW+y3RcCu01`WUo#_anVQY1JmXc8%u79xlFj4IRY>Z7cW`8=-X~gCSy(et)EHu z_2xxmEr{kPL{QZT%XwAZ zY$KM!5$`ca|ERF(Pjyt((dW2_za-d&svrhhQR3T=oe3Zytl%Q}_ZWrgXsGacnhNsq&^kcE%q2|-{L#$^UhRM&i%zs5&?6Z3YOC`b~4GG;jE+k`~ z*p4qYHo$b?m;{8Fu%Q7-LE%e>nyT_a3-dJ*A*XN<$g!GO>QX0|p|53^@K(5#xfh3U z01G=2L;5`sh@I$6Sm^F_f|^^-i`SA-05h4b%ggXM6gtyqA1i(=6?B!Gak;A0-`@v^ zuE#EKG@kZa?ENuhh{B}c|5;oHU1!7xtgJldtNQNLe{NTe2&(uH-mp;Eyd|8* z6XFT_KJEs=M96-u9rqO9yaZ%}iwJkVut8He_wAI#8>bD#zLm~92zOiac=I@P7}^GHYyShdUar$_ zLgwk#ndp>((Nr=~a4Q|MY+BDK@Z8tD;{iz(ux?Z|&6B?eEH8wzBWY-{4LYj4Evbv9 zWLed~0KxnuNR$U(Xfn`27zVT+SsC3%A>=pHW_4g-U_g8n3!Jw-?lB7)Z*oICnXVD< zX*bu^lykk!CzbC?)0h#`8!dXH&73BAekcf}Ohn*WzM#BrzplJ)>3tej!1ttx+?1i3z-!uYxFOr= zX}*)@0ArNfqIAuU2Q)P)%|n=o87V|W2eSuE_Q7-jq$v!g;t~e*?`~i>UQa$x-Xl}2 z2TR7FiIPT}F`eBeo6@N_C^+cssyOf>rNQ54p?T8fVlLyd@#>QLdUkH?w~LR4zvs~j z@F3vC5{7$`tHm)yzPzT^I8qS@xSpBc4ktkhCU1rRCRdm9~;7LS>w5xN;X5&%sI!Y)uSlvO1cy??E?FH<4YU-TS>c z`8&!PwXB}<^FG5sB}OxrE%8P^MOx7lj?7vNyV#)fccO!O5{^|Rw4VF)ap~2!Bkd}g zy%BfClS$h#f5I#g54k?<-4aja>?}zH`X)#@ggN4%MLELkMcl%v4Y6d>&rS(Ylx(6E%9mq_!;qiu;U{B& zBzaSENV*$AmAB95CPvP?M{`Vc1QoJJ+?`1UQ_t@6hAMUxJudDSmkr#10Qf)bey#`QwrNu~vL0S2?FZs2YJba@I;~%QFh4q^i^q>-T9@et$HUs` zjPtpm^p$^KXO53WqZCi_wej&5=f#j_@A?ygBJ8IspwWT#=K661`jTIYM9R4p(|McV z&Et6#dTZrfa`CB{nng{r{2U#)BUh5XF==5AVD4U&BY6*$Nzg+?XT1DgFTo~)w1wU# zinKYDW0Q6)`*3yetq?E$M~hIG3Y{N!;UlmbT9iB%6Z$*xF_cUCG#{66riN+d$1pz`MA8=@K02}A zuP6LT$(cnu2m#d*n11Xx>sh@R(uTK=4P3S9$Gg!T{~f?j&I*+9*&|C!rIDQ$r)-UD z(ppZQOL7g{PI32rw-12HEKUuplnl6od@P&)p?HQG7~crcw+o-@a*~uTr*Uq8uD|hg zLFuw`=d8i1JHlgVNa>6%==hys{XG!CGb??4u$O)^b%9sf&jtkpyB5fX-&GoBHe`te zg7ULD)WwG(zp#(SOYon+u0GGSd_KNh8=ZRX?OTeA^OfBE0}4m^u!BVCRd0zTEy-9Q zv8K{VM6{{oo|aAI7HN&`O;(>I!C!T4e~yovFi-MePoQf)!YyFV&?>v|w|YA8Zqrvn zqZ!e1W%Z*qTED+g7+7-+hZ&v*oTW#bA24hj{ML~q<;@HuZW4X77b$(c746zg&u~vS zi)K&V)=TAE{)+l6jeC00TUsA;Fs`dU)@I#=#Q{ZRVMCA&EsLD^KoM=Tx0!CtLX zu9qGj@ z5QBVd=Avk$Tv8v%8`J1fY_g*{nI%2e3akn_#DvF7LZ}Mt;JV z3~#k30Evp#ki$-=03LmuIsg|)tq?F)e^P(V+_<*?oE)gGzeWR4i-}IvnA!b9gNrSL z#vwWf?J_Fn@*$+ac10OdD%u9Ur{Sk$G4rd-)NhjOU?RG)`i&v@oml2qY*(u0vWDBm z>3)r|5lfNGFgeD!)L0@|5FAytzN++^PoKCRlWS$OF^qd z^=uJmhRRW;UJcip!=uJkdDGOsTiMShdXxTg(1oCU&R%wl_%{tTkzgFN^aXBXGEIf1 z_nFpXN=M8aQLhB*T?9J)KqoOT)PETySN(eZsP`g99Ai$4A}Be2I?r|OvzNgT2r)ko zBY*^_hAc@e7<;Em53MOxlq1~iN!Q6>v4~Q0!QpQuzdQpeHhK?I4pLH*y|!io_o{M| zS``EF=F}h^!FnTW&(ou>{e7;d%%*eb0s)bkGE-l%_@K7|^A`9?Hhj9pbp>a?eBk2e zTmlfMjg8X2o>KkwY?hF(7f{N`-N|?b+qql@b+P5^_onuCgzs^x@ zI@=3TfPAdJ(3c6$o$Y7F0!bDp*h$4r;>Ue^P1aGpi+YFa~T^}n-=7`iMZ#TbvLYw_8-@V{yvVn!p zT3Ab~W9>iyPgmFnODuX-L#*sj7|VnX>CXODXbqZ$7jwUm>+XMh&Zt6^rM+vvs(pvN z%E5=3gRxnNg3)Vdnn6vM7Rz?_7Z=o86@(3&)XtFJ`=oRk;(C?n9Wdk_;9Y8Dng%$; zTrf0DZoCD7+PlNWHYY__PlTrix4cBA6yK`%oVB|KVVQB~$H1G*=DVC9k01JQ^En4f z2SbhL*B8@;e(!-*8@u6lO;la~?%0KdjjqF-d?n4(SEeyJW%BmkWhxmMME-_M>YT)8 zz}S>kpF}7bql>D*M*7~Xc&&`IJc$lR&XGKQGV&_T-OQw@3%7}Wp!aCh`e`}FzQE@Z zQ#m486dl!_>|B{5){)1+Mso`+Hh7DXbkIvpJA3G9|0woH#lR1?GhMM-+v6CqzPgZV z)PsYqkp3qSb2?U)HB_!G2#tE8_?{K%#;j{1Sj;5QaU}^sCYOznNLzD6e!h zYw>d||5D;k3fVw~JPUMDqj5ek=(6s5L*)9D`fGN!#s0R4oQ1)W-BqAGqhL?3RSa9e zkyD&f&>;{_he;aFOh_>W-QCzYc@j|dN!ABpnTe2MWN!w32dCtD**4#DxjJ5*v|pBS zDGi!!AQ7ow`Rk#rbac9+bkYR)&8;Y$u>RhvlxkRV#$EntGEE>)v<}sNu4p;k-5IM9O33 z74p=-(Fyl1Bwp!UhHnW^t#H}YTHLUx<2~cCa}yUc@QXaiO#qC0_rW4?{j?NFEDDa! z4dwT*9`1V4Flp(aKm+*k zaB$K+%$CK#gzh`6r%MI#9YN z)o|o&arMXWgB1~IH{eFmSx zfw$sb|Mem8If44T0eh3UJx|=eGsfMw_XBcwr1(rqX-_X`4;yrr`7A;CWc<*0eSG`1K3Gv=_7z@Rh1z>V;YFxkakXIWQqam|{Xhorl4cH$QxG z(bbaC>ix&Z_Wz>QTGqEX^{BO`OB*F;BKX60jYW#h?cPV57i9c=Dc;}H%Vwr2=P+Xh z=jS@Ae8p!f2XJO@jN%{V`B+V}$;r!lj`@s(qbfM+%>xN5;c4Hy)xCaAUib8DCZLY3 z6TKN1;FJt^U8Vqo%5X4jmUh$X;l1ilVY|;iEl@v;d1NYUQ26tVGw3WOXzveayy=RY zaWPTQ;nSe<=dOkQ0n6gWfwO2KnIzjS>ADun>6hUZIx8s*48{WDdtCuDiZu`Yf0X^T za}bGk-Si!ZPBMTCaEA@U?ei1VQK=3hT46_KBMAe>WsItj7Umk)5V7#3*Pi(!X6xTI zd~4IHUe)d)T4?)dD*={`zmj>dGcnzBS_q!fD>wdk8SQ2>c};%wez+8Y0Omo zTnr|)kG&y>MsL)>+%_?~aP5<1Rug%zx4!3Rs5R3!+-I?kKT5uvzJc0`XlNDp#olqM zk~*Hs7MG&<#;(fnD4=?P!ii-s$@{y1H`c(g|81>DU?+xlV(LG+<|6c;u*HXLF9qxj zy6M`_XUJ+=`B3m-Oh+p7F<@>$0osaD7}-`R9hMzj@4@ovMG1IvF7m%llKkH9)6))5 zRH&(Tw<`>=%Rc7sXU@N8PtowM|2>c+7WM)Mv==bd3H#%#IBAaF5>l%3Ij=#j)ogdc zr>57k%;;|+YTF;&4_A8eCbrqs8)_iT~)fN}=!msubu5?i@hVt z+v|}~`Ku;3iZ5o!Z`e&&A z)JFWEzp#8|q5P9LGO(M(Re-Pw@x_bD$?jjI!AdkY7S>+?QYHlmL0~mKy$GMF3OYSA z-cXTWwG9#93M<2-a#ys|jMAA)UVh9qx%>~z5yP$qwuzWMqrU~7JM((I!W_#~STShK z$64jG6FVro(36VRM>tVJUsL+kMRP7J`9ULY;cyclCVBgOlV65-O=ZN_m4z%V6x5$7 zMl&@wud|!UaiJ2)%HPeC!9Wy(&(y+7^K>Xf>{q&tY<(5~E2}ZT)=Xn{K0&`&OWVNK zr-^%dyc9H5;O6>7OQqZr7F{Z8>AJctz@QVvkMcDW__ms}p=p(rQcbw0$M-nX*wARxKTlDz1M9yXfAWDQa$On@ zDQ?p7FE`9*6420GI_MzeWuSF(SBpA1t|(1m%q-E^xpxYnBp}~HtCJw_MLQU6-jj9^ z(Y+^Ci?EELQG@UoqF4>co+J1$D0&23hJx(@!^0Idr|c?ARvW-Itjp9^K1Ho&Sp7vq zVG!%J6<+(#D;B?_xWqg1N|5D&p?xnSU^u{V+`nQSMr`!s}(-G32=OTG2&_XWFBF zNhLW){R@8~M#7|K+bU1yXW{nRGIvDehV2KK)gGj%c zz?^9_cp1S0+^LY#=l1J0!w!2}Ew5S)by+>(fH3^zjZowMKa1MY*e6axIhrpdab};l z3gydu)&U5zW_v!s)zgU;bZb**UN&w+2kT~5b??rJArC8yf=`ggwUK^dwMftw!toJ9 zRUf5+qTBwiM->Q6brn_l;!Veb4ScfUfxU1=f_}rUmT4)_en}adgRyF2%vY{?c z4*|y7Oa1I15nTd79`a#8iYtK$d#Dbuni+3PGKs**D4klP2?4VK2tQ2Byt=&4#xN6=B z+&Brqd9hrc|1}P}%(!wp?{NS%T|NF_as6GSQf3~|=E(g|ty1XOae%2l`1g5*Rqo`{ zddfUbE`R(RFqA%!sq1#dd#e%8y}LOUuu)=BIeX$?ent0=^qiq93?$ zV#C}NaW7;BKFifU#9wyx1Y`JcQMLO3gPy-4-LOJ}tT1Zue~a@!Zd`tD^MSO@KE!aA zJyg*xaDMdne0oLx2Q{i=pM9rY*9>F%k@sdcF@I`${@ry(--2(X+SV-A^msJ?u*#af zT}Kw%G}$4sDxq}|?dD%w8>B*fJk?@fOL1~NDkd`5C41D{Um;2t#f$KyZ3(f6W(3PJ^2fc zK7IPcT|;7*f8vZi#+?@C(Dm0YSy$`Y{}C zT=ITPCBx(H*JQ3ie&cBd`bqmiU0znIq&p`q{tvUmJ>#qWnUs_vz$YZoMe(nG^Eypr z%*YC5`+OY*pgz&@=@VUi_Us8+i2uMEpS{9GrhC!U)wP?OZ=SMc#+#I~i2|Y`a#{Dx@Yea;B|4W9)bE$1= zX<_<0Z^+*aBM1HxmrP$@O4eHZ@7RZNiF*D>f**_~KX~ecyIcPJC#WM-H!d-P3urLcG>QXs?&n8%Hb%^&t47)K57%@-rQ+E zozqTCy4xl4^GDxdhN5fNrbkdA7#;Z}-R^rcE(`!mLhLeF#x}CcyXB~$ax&}YuVhIf z`?cReN|gU@LCZ>-{KnB*)_pqZBh$)F^8sZQ$@}>V{-Bk(g^$l$4V6)~0 z+;TAU@|(<}u`Pc&`}MVzq;*M06+&NIx{3^h(-OK@mt$|wQGpa08nA%%u5{wLWa=I`UVd6`hr=akgPF$a1F zP`8XA@X$03u3d=E52d|Y}vS>TxV0YrKd~ee7j}Yky`Ss9Ko?rXLE$EZH%mKPF??1 z;hV7-o%(0pc#Tk=BOmmUsBwf8^?B_2x(#Y3VRFWA9thdF*OLMhO9OfA#0+4@FbH)u zDF&z!LQZ8qsBJh3cr{mzsuSMB1m=27FKVY<^HB_y85A^7*+kp2Nc}YgkNeU+Kpz^R z&eyb6z&pkvS~}Z`^8GRl&^|f=qyvyc1d%OChW*g!{ys1?`2BlS%VCitCWu<}$iVzl zr6wmP$f4yrbI5l)PTQ-nXQ83xEUh_M^$93BIW<+n*UQ5@-yr`DB`5%xSR*<|FVge%tnijW6SaQ#H>#m!xxrB3LpSG0h#z0EOX1A6YG-OW&*WMS3ds3NX) zb){vastPy#n*_QH3J>XKl5-^R zYXcWy*L5FefRZMzK{tdE?$@n`eaZD2Hcw|_PizlJ${MkNoScnK;%0(k}TC#;Ve44 z#!R$#9qWpKKwpW|ZFn^&q3TfA8s2BLl3wf8qG+3v;{C-M^y>A}`SLq(wu=8v&Vn*( zc+b7@F4D30_@+Ir&A7(*+wu)x*2Tr>>Lci@p>p-QqxQATd)BTaw}#UezqNa+JB=qX zu$eo<(8ZhY?qb*LarZabEp$K>!J7V()~n}Sv@JfSeTk63eLxTa>~9HxyUQ(8{dM2* zhQp4B?dHrxMMcnr(R0U%Hsa3Yr$gsmY>|TKI&{@qjmk4hu4@g0aKeadb|oefy~{rY z`T9>eUTU}4ab0EIAp5t8DKP^ppJenWpTmz=v3_Air`=A=LG6;Y;P zW56M1?+v!=$DL?~fV@tZz#nuI9liTMw4H+L(c1I>ZL;RSnX+c<(<>AdC1+>8?G)O9 zDR0(}TLf;1k;%Ve;xe`wmPg@saLkL(b;m=Zu+)}o2NNq@`zZQ^v-Zz2tz1>`R}&CS z+KtNi7+7Jdt6qOx^Vd|n4}mu6f<&#f2BQ4ou6Wq7R(Xa20!0qIUSku^k}IEiDat4N zJgzX#0#P@A`~f06Xpu=(Bv?5M;^~6de?Ml@Fqd?my0i6vT)fV9e>q~V%XuMiw-Kvu zR4bbWoB7=3|L?nK^FfM=XmJDV{;R?_g12C_?{b?rod>s4EYaPBF}3HW^^~iw-<_9C zVlLM_fuW8a>CH{|_Q;Trh{D*z;_(Byj@5h`Ktft+Bj&QUb8wlvpDL~H>NU> zHQf#%_yIKA!OiIF^C;SjAK{{DZ6ECgqdxdmihEbr)TW;C_7wP;zAT(qOM6|Xp?$pO zm|0)5{p=2?8J~=vAZCv0R`5WWsotxhcmIiHbv_l0;Eg>@%M$ zK^Q;bUf)rdi~HIt<3!rUzM522AULT;l2`CANQkK>Zh^OgQO_w`rnb}VZv`=^IKoiT z071Yj(aL#LU?fC&AfYnnIXUQE;nWy=0<@DJ(PjL6(py!qA@&tE*qnNP>8?z$P$9Pd zQ*QiGJJx|rG*ecVaVp(UyNpy>fFSjY4l5W+j+GPq?=34OLE}iPt(WqGt2`kqWYu(? z9*bcZ8;&I?4tsra4J};vNjU68aKLsoxr~%59K>X=d^?jhZ7{`E0DCnl@Zo@Q8wu27 zeFnDQjJ})fmCqZapveM(c4untbLj4dfu`joV(E@4Y5qRz?k>ptY;Vc`Uo;(-DLDp6 zx|>DfOyz#szNT+tTO&$Rpiv(Z0wfg41#*jd-b^2Ky$oloqHGsZip3K&vU5Hk@4mmJ z(@uO&#h4>U?Z2Hv=Xor;=y~%m#;@i&Rj@HeRqwavDBv&;;JZ>-GTmK73x3^kQNE_m zHZ|(oK*_B|w}sZA|Nlj;OzCF$58sPL8x>WUDL#>sg)VZiF!Tif+PU!KRxfPwx~Uem z*-<)o>6v5C+~*Q+5b^!ZCn>12w}G1kwcayX6?Nb%+~tU(EmmGMT+%rFag-n`AT})a zS;lpnaZIJt`Vm6SEK*^73TnNed}Z*BBx9!6`B2q@_X+fy@w|MQ8en;C(IWmYeXnZh z#QWH3US7Zpf<@f+^5n(RdBkb^VvVB63UhY_E|5_H=~Vs~+>9cVANjk|HQf}s;7DzJ zj}s1PK}SB|uHVl56yy^+T&XbG^ywK-=R-~jg~#^QG7)#}@%eVUyczzEfKS?F$=hTDoJkJjmDiy{zcm|@aIGaY=xY$a3rv`NF?wXL67|9A zxC}YDRx*o!QY@13*)6+;%Ow?~}Y_g#HyL5))En zIP|>-mXudq_v=yLI!DJBo8an2r@llICsA}a4Pz`SqzcHXro)K)eXhHkIJ!%e=~;U; z>JoiXe(rZdC)rQ&j(q8Uxg&lQGnCMOy@T$`xaCk10u44h1kNBKsP$MTlPMYTJ#xu+ z>9$E3b?%R+#b%tg8BTejweKW9jhXzTb@nm@vu5ibkdx3{l6yD6+?E;12pHjeh|{;b`YY*iD@l zDhu{}@Z_p4zN>H{mEZ-K*scAS&TFOk2qOO9CAM1oub#bvr@gufW~gl<{YWiOEHb(! z*<30Hibkb`t+_rG=A<0BjQ(nON?Wm0s8{l^tO8s(AO3=|jzCsn_MAh@O;wR#+WnEO z#OI#KbIFr~TcV7X%KQ8%zfTvP1xDvr+D-eX9VMYXy;9_U1A@MN!C661Xfcxgc)V$>K4A~JQZkzn;|7xn~ znMvh2PqaxgqY^0B5;ir<)V_V=x_7aayXrCF*-~`ZR<62=z+@7HEaJa={EcSmRD6i- zV?^juT}>_9i@Ve7(?I$ke{h>#fdjWaHug(Ir~ylj<{o=J{+IkOjb>>4e`WhAgrneq zJ8OQWo(ocTvs`ou6Qn7vU^=7Cm`!I_{z2Lnk|lS{+whTQYjj&2X~Cjog)+-_En2E&Tirq3RoIEvsjUW@lIbM zXAAV7gSej*tZi)Wo%S~W(69BY(xr!!n=S#N!Di+_y=6M&J_jW2Ec*hInvk0tSObWW`i-OQ)lt*uU^ztKLey}LR?yir5(4&*cln$NHP zy4j)9&A&?1rN;6+DF_mHu%MCiap-oupL&Qd1(WIG%Fz&r!vnv2@7Niz@~}S~0kF=~ zoh4Ovd-Dzg09B+)0Z_#Z24p{l{&sZ~hZ3EB#+K#b(b6Y&gP41aajEhSi;^S%N9B}9VIT?sudFA zrSl7B7s`nBY?&_&9Cg{M`ijohzA9u8yrJDpcj;4KOeShJf^*O^Q6+6`;Q)cD&*s&lK4a&|g^Bail$Dl@9xF{Z^&Yc|$hd!}lpL&7>F zs_2A-OJkXSYy3tvF1Bl?IMSJP%n(Ap7UON>H=yJ%sSpcS=wfPVBrL=LS#>;}Oxdw5 z4CE11Qr{jXK*p~lHV`WAUceN<4ik>kOR171hM0zxt$CiBtrRs#FNtVLs_Ha&XcwQn zUZdf>GRT#;#)M#Dv^|DO`_!zA_PKMJf_dP^Ja4X;)*?&xPRiK<?PdRK7 zi9v3E5dgCnIRB@9{Ur(pB;we95;nAC})T8@{EgS~h>hd>HWp9`-a zN5ahzxXkhDQ|<~mog)p~k#nz4=eU0VXVZG{RKktht@=>@;B#jOLoxdLRD5ES=1q&& z)@VY=SAbN7(78zZuV0g8JsNxPA>#s)#5e4D?Pm(J~p2C)tGB! zbS%Nq6?L-8Rd^;*Gj9|Y2Uw0m`AT%5$|7F`lu@xa`|^i(HZkxM zr_g{_=4q4G1IlBQQXYYeH(bn=rLvE%om#t0xW5zmDOCbX8Xf_;r{Lg)1OfEm{-4Z2 z!#W3%%gY=~nF_uMCy{~vOZlwZlkxdtk%gLi)ZPANuqh=`&5f2FR3SfjSRPOZ0Z0cu z@Q(;qSF)6thxJOk!k$8t4_6*1uq}Cy%jz8yTJtv#ZOOBSu-NT-jZz{v$65mgN@b>_ zIg~;Y@=NPS+BDRvo<1KFoBM!H-5ZnUI+@JOlq7p{bZ}S!3fOB*=gDnYG!&_s%s@xj zbtusyWS6hL(n$yZQjOql`_sw)<)&Oa9aq66_`kytaq1$aSs+~nkIGM|>{@=FgCXoy zHm7UQU&*t<{N8*XvVVI2&TMZd*)saF45-gS)a4n11!k@Ir6Toor_6Aud>$R_fIH+uhw=bIg z3rPn;sHEFfY5)U)`3;2yzf@EpPf^u~*SI6@i}CJSKJw<$Nw2Z@YcfkWZ)0L(`8IdF zG>}&K9-d7EJRuOk)Eq%U{}v)bE^MXflgzqc`hoj5>UAe4PsNxLvAok*|aw6EwB)0~wz zF#ZxdgHH_{_m##R@2h15MeqTOI#bNdbC2^;dR>K`^N(kOCU3(;IFXe?YC?y1`rqX6 zhi6N(b$pwvj#jV!6SQ)jLdFWZ%5;W7;25C3p%T`7hLAsudR9!8iF$ZNKb0qWcMX^-(Gb57c#q*$o!_|s{g;R&rJasu=1*S3{J)1a`r)^mrf$nS%JzR|V1qm-e z`Dcn>R)3O$5>C=5owGuEw(DkZK4vI$VAI|1RQ z34%rj7stV15fkG*q|WMw5OYdI11KQ{K@T!P#)s64ue@DA{@JvRr% z8BkQkpBwFQ5Qo81(>6ND&C5*%4CFmzdKKlI^i9)vr19e1XG|E3e7vf>tGJSf<+}?+ zX~~)x$%*ByRiFIWOh&kIoO~8Gl$=t!l51E>lt*MV=0;*;e`0?OvO=B;6W~Xy1G3)H z92=aXc+GEkoa@h({15_SWS?4f74Vaut`chqre%dy@q1fDbK=E-83fSJLLe^*yDk)2 zy}X-t5#F=G{P!nb?&8WiiQ$n?un{9rSrEB7UeHRx%7qu`8{|W)6xbBOJ(XqY)kJ}skarwVvo-wA=bh|^7;6G?G!GI7d=n%oSv^Dkj$z``<35;DJTMZ7BF z>p{-&6MT9nZ5wwawwY$}@!2mT(@3*G8Er%PJ&k@KilU4CbV{=6Kt3ER%IX8??b`l@ zO;IF|uAs4}Yg5H=oE_1R;7{TWC*63qTr~{^P3(Purvd8%dgmPif~@pv)Fc-iNnvqb z(O?5UOQ%8PQy>byamAzuNaSGaq5!p86-ACn8(+U=vp;ziGCkP-QadA<^XK0ZFFjlE z3N}l-Y!TVoZVxgcSgLV@mgoh%mXlfNCESjC#FMFyYz^*IJg)Uv7)U3pi+)!&Q{D4x z)cpACH^?J+bZGaGDMXJC*`I69^l)}^9?ni%j66?IgPliNg-?DHfB$)b*NO-}@eKc= z0cK9W10NGUw=^GXxe{6=^gEddEfZloxf#}p#3JmCbP1s-1?qe%ibd=k*OkU=C#mFT ztiguiWZmdq2|nqPR{lAakt`KQ9g7G;So;ED=z$(pa_pAx>RxG*zkL;GS(caFK z|GWx?U2I#2(S)&8g*E)L*C{6I=teK8O`B54IVaK7kSlf{z|hng6k0=|DEbKLKhiu3 zBklr=W@?BSeO54MuIe$cXhLi#+I{pWMIi)UB;Jw-{QRGMr*dASu?MIar-SCldewdO!-rD4}YAn^7` z8*)U99+f#@yrcdeoEz`e{EoE^aAyH@kpKw==owgJTi6BORdeQBonC=wv6dYp5*o`T0pN~3OMGZ zspaV~Mov;O%~AmAt(M2-)L>%f@BPGt6ubg{-y{?B1iJW>#;zwwsxNcCJOf$$LgFF7jxkR5h7{HlsB zXcUn7748A>hwEDek5eM&*<>P4k)6M^=+e!%+0U#Y>}yoo{vu$}ajphpX1J@m z`EnIM!cxZ?YlDM`1=wqzxsn1#K>-dT!29!gl|DJOziZ^Zpmq~YSiYz(>5ymR?ZOhT zg+S@CaS8>>%5ug-4SnOjF#|Lmvz_{T_g!j#qTLUriaW=_LArmOUd}dg+o% z1ul?F(~S~sQm)>k3sv)z9Ns)DP6rgX>Z^5+m+m*OzpGrVv3zJ0kdqMTpDeWnSN>t%h7`ug+_e60QxdmzT zTPgw!O+D5oi>|)AKuBb{ZkXt1QgRHlEy|h=6I&>mH1G%dG86rwio5VJa^_Xx^@!%b zXRI7T(iAjLcj-$Mt=q!2f3!KLyH)(mDo?lY6i~!5I5PZ$4@`pPg(~##*v*qoLPY>} zEq)TOWg~3ZCN>vJZE{TyP-5e;k?IWJXn!Rz`%uOl-b2J@wk4`&SJNFmiy44qM2l_2 zf@y6FX_N1d=IEavwiR*iL@5)Vx*9aDoQlBdUwX9XWr^x?i45Y=%%0u8l#7!xb}ANL zGVF(-ma}e@@5;0#sV*B)5;sl8n;+Jgsr{~(u8W=PPL}-kv}5^iGtt*~Syx;Br{YVe z=Wd5TfCktd=bon7q(FV^HHU;@p3OyS zxR4i6u+&(j^PUUR11jS8%T>FVL68S2#c=8SO)QeDs{u9Eg#KrLd8j@o+SMTb zfO0C;-+pC`dm2a&7$yXU3tEOsvBkOTP6C47&|z*cRrk(V)El}Rg**W-Xv-!Bh?%Y+ zN@79w_Q=t&{%T#_k;bX*XJW%NjC-~eE{fUxN%hm{g$XG)2$UQhK8YDN88=S-N2TKp z1S$*8b|8lvdzhUykxbw^8Z_1NsHYv@*3z-0m)YZ=2GMps+a7i@ZDriBI?9eyPfOh) z6`cO)(cyvtc}o5CMq6dzW+`!VG0MMY@MtgpRDl0W0~zk#FDCSf!o2ROI&ZNc#itN@W}#Qo zYNQ$Y0~dc>tJT&bmUk>Ba2`QT0=8g>34^_Mns|nEKyK-TRI0fch=iGhb5n>QYPpqdED1 z$iXXJ$2#mE4{Q@N6}J0tDaKa=o3VAv8yJeJ@#@m%Q7% z$+a?FZ~uMA(h_$+t=+<4=5g0amY0_+4gA*SvToWvH*Y+=2p5-%1|7hEY0~h)JVHvg z{7YFwvxwbBAe~|*(EqACb)_u7i?KHEimH1Ah(M~vTndN#v960ulU#sGlCyr7#>oxr zXkNg%T?6R>6;@P>`Sw>6syRH9_(s3~YFHSs{!8ji5@pL|=9}nIYmFuxyF23Ev((!X z71FEKf<7XKh9NF_5j?4EimJDoRmr5eOY>sao!#yS>tHBwI!ja}jJSM-Qt#J~dTnim zhf+*L2WbM|vSqkcEXps{QYub9yv9lTI)23l3en-F$B6Kq=Rf}4cEc70b+7R%%$eh( zyoj5|qa>?mm5;=BTu;wZN4=)hW4nCBvD#Mc`3F^<<7yIvw=j7qd$`wKJ{#`l2&lxzMMh_kQzQ0bwmc?!3u2~wf?nE!IUA`<~x zE&QT?`%(khgY(-VS6?7kyDubZu1|@2Yl^Eh)^t>VpQTB$K(yf0tK*d=6=udK|8{ZJa>|uTp5n8NBj$2NMY&^juO&Lq0#o0 ztvfeyS1Kxt8*bSIl0MhxHp+2uD> z^X{&v5q?H|nfq?nJs!4+|8XSCeWK(4W{I2+uR7;bQCoHFNq~>SmMW?CA64l8ZN6+*mMu*kU$5;|= zNF|)sh1`0D9nRav6lTzJ`XuPj^|{`JP1V?z!M%CI0tRUO_r$IDhleqJ5O@#GYwDj= z3t_j)v#{0wVEgh6N!~kj8UFiUXXneH=5dTX?b&M*I%(eByFn!$$3;K04h;agniRwg z?z8fC=8aumdf@}^s* zkX9mIzBI`*pyV<$xGE<-L0f6ja*{0?EPR&v7vubu)wga6N7TBqx@W&_8k5#{J!WE} z3c0bdmCp)l9m~2@Y%Ff7 ztT_pjdRBkcMthH>{C)m|LE|qZvD@O{I{u^?{?hIydHWHGN2c^-nVgg72mfE zlq+)Z|2L}4%AlLb!hgGfmj$HB!Zx#Kel_sRn2IGaZmviQ)C?2r0$(*RVyU!mk)UdSxeQ^yY`Rz) zD#>!pb4o;|M61hY4^)*6l9LpDxf9zl*vrdkJTyW28K+zb-kW`Sp`J|G6pJdZ|4Oy~ zg3PAD+X9=s=xbss(MLzu;asoQ4}vgx$A!W#QxK;xy4tS*wWUJGW9rkb9Rs@t)S<3W zyQ^^1N22HMwvLUfrvBEnzd{kOc|h>*Fa2lSmQ3dU^^-K-{qkLzXu?d#^G)wt{yxvl z2(c-3gd4YX6^5R*`3Vboi&)q-R6>ijxj?P?G@lyI&Hrp@!MgRp*T3!ByCvDspg~ww zV&c6&3a|Tw<03!nhK24XhVHtz<>nsO=f1D`B%C(S z_4(s(-0TC-90?~9u=Zo|4|u*-3OcBx{p&rald)yc9{f0_3x$C!wmP_MnBs&?-w zMU&(|FZ$vkqNywPfEhw5yRU`Co=*Ya8F^6^1fYf<_y2M- zzObs1r2=^*TA(K9NOK&8MQha9)_Y7u^#cSW#167scE3)gG281-KZGtA0<)X#Qq|fW zeH3!Bgze|vvNdNNcUNPr8|%--PdZHl0V~^|Z)Z*;l4cc}{O&)xoA#-UT%882v~o?6 zWR?&#kcw&qJsRp1$UR%ckUpi?qpEN^6;N5zi6xtJQJT)W-eVe-3N1ZrAUEyuGe~edU&L$2wrK(+WFBvJsLJNrc^;1e zPA)?dcSHSUA{xU9o&ngL+e07UtngtFIgP#=8~}7zz)2SUb*^@|+qr*~#xm3hyUNC~ z?S9maoXdRuW_>1$v-x6c;yBC5ujtLLB@pTObS^9yo+xjfL&!iQ`zob0-Mq(z^O8ow zf5mmmL5{6xBg8nm*J8GZ+&t$;@IaR8D^sCpXxEPJB8uPH!G3xh3yw`F?a{%uHt-z( zr8X~$HB`~MipZ48O`K91NO_0|W?|k&XZ)gFT5hi&+Mp|`maLn+U6(H)>9$V&%RVN* zXXUu!icM}3&+A z*;(C!&)rq}k_6$>RlD3FcF3K9&xWu zeEyhk?7z~{;(wU0r<#g*J*MXS`nOgSbcWzvHzODp+bM=FtIZ%v3MrcSS(Erhbfz>3 zpQ5jVpF}w4YbY+WM@1H6?t4y8&bnB61=j_V9s2_qn7LsWSPYdBUwykkO!0p|Fh}!C?{87#?*fag zAnwAXZ|L;GmIht%=ZonZjv$C0*UwAGff7IGBHywxl~H~b^#lv-WJXeHN=(Fv9v5=v z5LnB|)k?(zzTJscD*t<4P3~+$VWzDL{AgiqX!%Qp^jh?(_QDy{P(euNmKpH0$}&{| zlFYNW-k~*VELYFw1oIrG{Un!(Vjn_6V#C#(iQj|GS62xfHy(kUdm7j`)n73r2*Igt zpnV7wxhXJ_yI`Um^S|iO@iU|t-chlp>A`-eTTglpjA+|22$#-`X37rmHN@G65Q|h#z zQN{K^Nc>ojr+?7s0Wb*yEdtK}ZeRuBzQ>a5@jt1^?GeC7%(4EDvkLN}uOh*S${Y?T z@}$6(Jb+j%L0#`PYcMGyJl!_GQWJj#XkXghvBF_;8`l?-@`~%|w3N@GcK&!ok1- zNredrIqnW-X*d8!69%l|tI+B(d_7UUgdMK4Rym3eq-6LOz!R)k;ZMM#>J}@Fjcr%0 zPY6EtLCg@WPT%&E`FC~_SI_W@*j5j|;l!)enSdE$=(001=YB?R#>z1bnlp^VCtNpn*^**xY{Nc<)Q(yvSA7uH%y;gNHhkR#ZlCL$N1P^6p+|+A~ z5jgha^X4mMMGRHq(CO-Io#3$d>4?q8QEvERWy_RCbde=>A6an$7E3$D<`4HiQp{X^ zasJW$EAG};_M{*aZQCG_xjO!j9wpIAPL_AUv;hN2xRLI7Y|Q|7kZmfD_*+W>Qw&$MDRO_t17dnM^u9d- zd-<$8mVVV_>NK#FG&oAI0J!Qh8<7&pwhO4Ey8Qz|R8p2c`Xt0D@n!6Wp4~ZBIg_H{ zswbEP;OqRGT}P?S6EJYUWn)4@w$23k=>Zq9FJe!aZ+H%gK!E*l0c#QP^}Klq(eh1l z8YG=^|G?ZINgn`T9-G+WOUPFu)K~DfbyCTHHwSF7EYlgmk2U0X1(2IzJb+7Hyq4J7 zPRSR@*BWg6Kb=yY5u*G_<4&cFoKz|DW?1#-Hv45@g}f=u5^hs~kbrpHAG!KMZiV>rw$? z$<(CmZ>Zwyx-`$nQ}r(oHBCURdzBx90U2#R9bAoHpR>ua{7wh?d5kBgK3JvZy2Z`< zu&zou_NC`jwyqN+B1FgcwR#7hgT74tpWjtJ%gU}t|9ID1%(r}0<#%)A#WKVfhCA#P z;Yb7jb4)XIp6~J-5}$r}59P?cwB)(cDu-oZL#`aX`Zhzwk9RAU4l|bKtj6zmO72GO z8&e-Txd&jWS;4ayp(SGyiU9=!ieB?j6DXuvnl%CAzmLs(tbZYDtF`=}Z->&n?T#r! zv}T7W;-Iuz=DG|UuVj}~hC_LT9qwNh388KiRz3^YojPW;ruh8`2XC{xvt~nao1Dtn znfMPL$y-n3&%X%vP&^Z(a!&lrG{L#p-$l+EKgqm732Jp@GieP|pE@2KFYw>!pV!|p zrxd}pzcTPCpYXQvwC5%-=TFeBp7?KLp_Lq``1*6v(O~#?{;U7MB49Q6t3c8monN7B z!CxT@D-hA|PaoOomEZ$$flEZ~ztKt_oHPIF`LfO^wf?uZvMqV5dIX~b0E&zKb0wc4 z9JMblwU0p;3C6cvlIO?wXJ4<;JEzy8{;U96&=C3X5x83!E;7KuBFzfoWGf>f?NwdE zQoFcuI-X7is#A0YxAZQbUfllrqSLiR-Nb-6v$g7 zGQUvjM)SQ53wviUh1c$!{n;P69vLxHiGiKG!9TYp;wqMYKet4$o#g1leOVF3>2qE) za95Gl@W!$g>wePlX3F?3!T9>G%>Adlk_Q#?`Ty>7?)TV_z(iMz0`DQjyE0NpO^5dh zv1HE|xPLUO{EDFYMw28&>m~-35N_qbjO-#%;2GwWwKjn7avFG&>cacpQk1 z<57y6H>K*akoQ>TCZP9_MJA=#Vl_QWc9*a?H@@e1l){3*XAf0WQ$0_E*-BGPf0fI8 zRTSyj!P*5T+4acOmefk~Y~0fBCY_};ujIW9aA~7}siQAe=RhEa0I(St1KJC+aMM+I zYrutQH{Vzd+JMUbot``3I$e`&tS_J6LFqCd`$(QWmyVC6Skn$U^pePpo|P(n-!r$p zaU67o2R;R}0jMv4%G(1! zT5ZeR_b5gA4`};OC>yr}b%YZzys%yJ{LU7y={@1;Cxeh8S4pBoOWsZAoRYW%OLs;% zWK>w<>rhuduhLRy<*b@C30eSZ`Jv&|IaJaEwHxG^Db&;x(K6h86pqy9O=M#@S*c=a z2iL)&pm2qks;c|E&hH?m@a}$5^5bvovmh+qW{%A&mOs%Zp|Ly~UB!HW*znD;$^4H5 zH3ME|oq0LV+mZG^j8$D7j{kHX6aRb+I~oyruPVV7$)k?T`Nt{?Ln@5^2cavq-m7Pp zS;=-1ny&3UEzicZ7F#G;_B6q5OmiO|OTr<e_{NK^^=?;qCFzXh4>=-xV#V_%dUh_+kwCZc`7`|er`;Jp{V~_3d zrG|rbG1LoKK;NM-3*>F!?JLBO13UA51?8qDig-cYR+tu-Slz`FltDoU`(^I*VORvc zv%wR7GR0jp9zdkR%|yCZq3Fkr0I`+VXP&eQL4=2ed#*+jJ4o{ zPGx^nNFIWvAG2G_^8R~=U99NDrr?;A(zDBMT3d@- z2i^eC=Rqbiz`Xhm_tbSeBVgAm6#PnLEZui#w&-P+;B0tg)?Y3d_T_kg}2bB z?NHD>EX%N|bXF+W=IJP<0b`_tBB289nGqv$< zLTXH>u^C>1(mCPAPIl#;wHP(}>vs)!1za`W!sg}`BZ$dX^0sLk$AcDzRNNbV%n;&l zCY?-C@pzA+S}uvQomE{Bb@=P?eD5z_K|J=13c7xN$<4D{r#QZexvfjm%UAUr?nJw1 z4{kr3|3kW$4pT@yf~bRTnS=k9`{XaB(iMs?3T+U7TNz+u0L%RKFLc`W$iDn^JB>E< zIg)pOvyl=#<(2(Z(PumJmvWQ$X0{9!3!S5#)(#K`H{M&P!rmk%Yo53fd)Tq}Ngjgx zD~jG(EL62RqLtuL7?5p1$`YaCbKK!wRl3DOIi|udAi3G&e+-?ENKUb-prrlzq0xwu|sS`NwEu^M0B0!&F5 zbu*ElC~}tcbt1uj8dV-U6Ow_$-P7lXiKDGORXNfj^c>1wQV0U2Mq)QJ%>ftm4z1}!9Sj~HfMdGHe`$1o zNkSoOCeL-MJ$w1zO42RCZ#x>eg2RQ!TD;1?)zSr++-qkGeh~NP`4c4on;YDiIQ7MY8sQuI6izeLM!2|Kjvll`mz*7qj!+*bm$Y)*HZliyQRGX+3v`|`l*^33q9cN*o(ILiu5Mx4EMfT{o%D=WP{{M%ps|t&%4Yz|J3_S?ajYu~N(hMEaQqo9EcXxv* zDN+(cNSD&xsdRUD*D!E4|2cQ(h8t#{y=U+5?e(q&^QscM^l0X!Ak61eh$fB3l@eJw zbY8p)eK=FFaU^H3>X0mR6Y*OWK%sHuyv+D34z&@m>uY;UQ48z>|AbAbaYIdF3{h56 zK-vRmGJ~P&p}^5Ry09EG9Ew5;O(QlegHPg?Z51woWzP zpLQQPaST?ydVe+_O$e&{-6^KGhA&z3I-EfCm?d?)2uW?g0ws&8>o_g-CfXUlm#+2b zh}E|bL|sFIaqAtupQUGOcmy0?Gtbx6iY)c}U~;L@e@&sPA0)QNrZI|N&~k-@HB^Nitb!tHkXZ>?(Hbt}KR z;csQmoap*-CQx5AJRbskmE+KTiiEqOp?5se`_Tea6&`Ho@@0mO%`Ywo*xk5JIo%*m zUp0GVbL-UfDg%uqGOx8oRu&Zy0NIW`_L|4{Egkl%`B(sx4>1a_LyFbw%bP2--Jy7o zIr3af4N&SB^Uu4y=M>fif06PVhP|0)QZLwnP?_S=p^0NMHi=)8wA>slm zqWVw4pZZXg$7`w=T!r}<;i2Z*?^MrN>{PCSU(Q@6`$}1I%FHwFvJ5MXMQ%a<{MB|0 zPtSLwm~CjJevDP)OOybW4HRB#ki^bH^JcX!IP;afeDZe2!wN|XYT3ur=2T98%I6H2 z;^WR~BBd$}ZTzatgigZSHt$-=3ozx}%)ZuCvyPAu>49lTQqp&A4$;00j3xqyv$^N# zaI$zAuXu#d8%Zj_U;~a}5CMxV|6rez-gr3kxVkZ$gI<)|d9YE;?s?57-%j0N#I3+W9 zHmMR#F9w2NOrO%~UWvVSPng05{grp50iM+3?*09vj_z{jrtMm&n?0KnpdtVH$_$WO z84$97Y?N(>;W;n98k}C+m}F;DA?1JNQW3``6=5@+s033CC3VPj`UWs#Y{yi-`ln`D z2^aYd^c$xP{!5#!jjh;@zQ)d*P!jh`2`JW#DG#KuyCp98@h(sk2c-@~csPqLY3rdjxs3^|f->K|*~lf$m>o+w`E~_mT_g-p*0{q3ZCyT+nKp#pI0wUkqtl5fpE`r6YF~dZx zGe62TFeoHAcT0E2%G(hO80>gZX0yAIhX5KYP}7)+D4SHd%qX9>Y9!^x+lnM?gU|Z) zbA}%eN_re=oqE;}{necCOdMXt+cV5zCy8N3@5I#0906vyiR=8r^n=mZ7fcJ zm76}Hk;JRw4|SN5*nRvomQmTKKsA(~B;Bm@`DcooGDpI{xl9uiXWK{I0cP>4tH39V zkER{xqB`eX-pFZIeVW}m>R@4 zZis2I6|=ezKl6*~84{Nn?v}InW@s)bX!pF^){`B1zAjH*ZwDi$333SZP*)u;up{>H z>vz`+FMukz)?A4nS${V=pw@MW8U}kaCRA;xP1C3`IVM&;MKM;Jr?qi-8D%oFfrf+c z@@Y(ex1Gi2(M!78L_Kq%@PN8RRD-Q8$K(?1Mxur89J1u5iVf+&^}7(-kQgqm(Qu)| zPr(M6SXF?%JN7vDIKPg+cW6z2Ul)Qp!Sb(=Fd51bfZFgp92G~Q;_sd8OWlQTNKl9g zsTu=qs4f90!9g{@Ml2%)*|^#cxSJBXC$g*-2}nKa^n%G7Mp=4XVWS&XXWKvGa{VL< z4r3CQ=(G{iR!0E$YwA(1BD3iQ0MCvA5ag4XP$7dk#ZZEtsf>p@&@187;CKfDW|Lg|{OLj9&}<-$f#okN z#zqrDsIotvO}mrfc+G<`&1V7W44K-Ineb5aE)zWWF}m*(u3AvyTeDVP)jS%K zjOWk9Qdd$)Srm?21&b5kNtI8qdS#1xp+L1`gG{>%H|oP1e|slb`)j+4R1F9F-^hr% z?6rh%j0i{q7g)4CzzKGR_>5+;zWE`S_JWL)JUxeOBq=Sa!bR8!j2V~;S`2Mx|52>l z7z_b^zhe%ubo-?Yv$2a}r`CM4g#Mhc^u?J)t6b^B)+O3WmzGByLbQ#^{(@vwc zCSzGR00vx#!tpAnT>zYYtemtz3>~I(k@0<1y2uR*2qGFuPTMT>Xtrpj@(7hCNEC|5 z7RE(u2SsEXp2_2fQaAkW41oNS`WGo*wAKhA_nNQp4&H(JpKX!lJ_iNpB8Cs}v(O~% z4=fmAekZ}AXT$yovTJ`w;MvJ!0QM5R()b`*IR_p@x&#XieH*<715hx>e$0ZXS|{zg zmaLU(U~107U%2sCxcK<7~ z_indSy_YMg_p|n-aHiIi$H#8#+IyJc+4i-&8E{$mXKC@DC{ThbEiA#Z-RLaFj$aw{ z=C(NDj4p^EGU>?}PTDWst(*-Ej3Vw^Z!ZnuNxpgsfX~k8JOVJSNTvsk z24KrcHjSWXwXjh*Iuugh{;5*8$4^XnkAGS{9%%Y4KhMM&$Rtw>IkXS(4dFbasR3TFkPM!FC1auOb*8&y=--8;{FghFf9>E_WYaSXTl(-fQa|F^D+@XD_&^i%TE^* z*`d&|qODe{wO`2#SU;S^Jpl7G%;md34_n3@T|h-p_=G%n@R?3yoqCP&GOKRA%(PS( zL?Z_N5VW>?r&y{@I*-j15j6IRixK{v+S}ecOAJ&Q+-a?y${rr-FoXor%w3Tz(s0v9 z2^7YIqfIZh<|049?eC|~&ISaR;q~^O@Wz(4;ub>C$(6hRi2b~q9AE>TpRs|f4up;& z5_)+$w;Ne9YDN>k4vgT;1F|krMaO?<)z<$NjpMi9P=q;^N(K&w=Q!80gsSOT+X`MD z&;whzGb~=PU6XZ<#d5FpFXGO%%orV${GNaSAt079OKAc}QTDl&HpBXKRla!9r|h#M zQ!Pa1+f-3;Q)_tbkGQhG|6^TS)v8bf>@GYBY*D0}Xn;F8Kitp4rpvCTH3;VejO|H~ zg~AD&PqaPPzU```e5(}1+SIMr;0mst;kaeps-F;>$FD4)cF|*CyiH0S+bDWXK8=P> z!>eEVw${_JR^O=zDNN3r@GAOxhT{{Je=DVk_xO6@$#?i)<;4r5dOhagu^rPMVvvzZ zj$8y{dkpC5q`a4_!odlFh->G!&(19%hS%vu1rA;=+Cn6&H+w$D3G-4+aspg~T^teA zl7HC*@hxm7XSp41={qT%_0-ae71K-qy&&=Z(@Pmf-31vzU79z4Z4v#=w!9%Oz3QRVed z1j3sO8!>_$8YVnUjvi8>8Ad{HsnCJ$ZU`~zOZ&CY>gElN6Bd!S`}Ha0qeVlB3?pwl zf3?c7Fo!@w%wJ!hH;$>fxsL^0sd2$Alh zui5!Z8kPUdU(Y{}OdL#>Z4ct#Zj)ZYB~}itNy35;{3kpnXa+$4e@Yn>-}tpo+k3&{ z4OOgwxb+7Yx+7>e$)Q-kiDs@G%cfr^SERNkqVNPurPv-FEU-jFKcwbqlx8bpr3UVZ*6)+H)Fsb%L5!3tFXbS-{zB*Pd;R zA8^-AFYg2^;Rn{mX*{-RNdH~sNjg&kM8irXk2{=Bh@y8`%62t{<1OVb6mwE}wjrn_ z;>B8GzfJ23M53~k3^a;1af|E}Y}*aDM)GB=^uiGl{+CmowwV$CjC;wj@d>?1{EbW5 zfYbjJF6e(N#JlnfP1}TRT9tF}Che*A*H&k_V?wo0IwA{&c9XVqyqIQmj+sO+f*Ot| zA`CC@gTfCFwii24;Lk_Py_8~6T$gcshYV&U``zQhmw3Aq0<~NgV+Hh{qUuz8wIJ5i z>WQBFnAHUw(f?BocTmmd_2=BoMKK6rvqQs7`qe=1p8CApk5*L(YUVN9baH0{hiOrW znj2|bnNd(sJk^wM`IZJlv7j(s5;U72j3;r30As9+iD)ia{HHzFs>xBo z5Xc3F-MLG%pCN3a@Ze4|-Ev|{icw!@pC=h3yQY#N;VI*hc-kZ3Op5C6{gOpbsee?? z;W8bN>~?j2MS{%PL*sgEGvqt6+iPuLJgxk}N<;Vs_4|fsL`@bj9{D|8H2)k=62(&r zup85>_u~du*SD^{X_plmN_k%OlV)m6MuUH!7J@qeyNK#qB4X{chJT)?yrpY>bB&NC zvJt2Pec>ZFyQA`C{*ev2pn9qm3VqfQtJQ<_gtHilr20}9!u4R~2+BdV_10tt2OL1u zkTk1?aC!6N3=yw@SVY76woe;!%o29|Bb9&pNPp+bndVUWKU=JOa)bc2P)N|O$zkLy zTtCHJ$dJSh3-lJ#la`J0=apMBHbH1F!7f^ZiZqqorwXcBwN9O{n z-AAA(SRq{4MPn$SvLnpL^xIe-i9FGtc1=h7Wwyo-UJGsiDhD60=zaKfOAlW4W}sTw z_^xFr__+Rf^5}OpyG`DEJ6C?Y_up!R-#HiXEE?mUUM%mgPhf;mzP3dW+vL=X%<|qv z1462+u`j%63DO$xG&A`>{t(c~EI|#`e>J>p=|UT1VpJ0eS9LAL!c!C=8U0LuXKpFh zX{{ox$~+_;+Rw;}I!_){`AwBxLihqHh^5=1$RLfXt5}Y?=KW-&zHsJ!M%=^wW4iy| z9i5@qL8&ObrBy$F^}UR_&eePW^^YRk&BwRlRCoWnVX)(erQ>qVV*mBDhq?B$X&|bC zfr`%cMsH6^V^2Ems!NRKWN5AUtD9+tq|U8{uWyy6_AFR7XQB0~OU3y$Fh&d^KxCAftBs~q#58C=VDL#{b^tSnlvHqk!u(U3FO2dL&a*mYCKoA zztG})%@o_T`%i`(`Y_xI$6W?eSzmtghi#yT2om$>1X3e2o$IG^OXhhGKSpO*iLj8f zns*+nKqdhR#$! z=Ee^|UqLZ=`LBM355xCrd6MQdU@9}d67%ot%Jv(Mip+(9ir2DZ)9VL!QDP;7i3MOF zFgDzq^rG;x$4U)mM6~~dvOkb-->W~KS;%jW{qZ?pzUdTwII&Gef2u3~9oUQQTSj)A zWY_bAnmB YkCjPy%@jiiCzkaFGe(G`0Hp-K}y8hDBbgDxa^; zWWb^mNDSu+)6TPjAKvG3t|-iSbgaKU>s6}4Y866=Dv|$5X4_4pAL%A&)ccX>epkk; z*)JP{nC%r6XDUq`3~y$9@{QGt0mhG8$8!d*Q?cOJn?Ef<9(kTW6kM~L9qOCYqlDHE zDpK4mIiX0Gzx+-WZMOoqPI)G0Bh|2%rH@Ar9SS9wLzJV7(9$A)UirA!QAXeO$}j5zvC)yms-5Z|zJ&@EuWp z-UhLEJ>E4xBA#bGdTu(QgHBfC?)yJ3-QU<=wF1M+hFseln`^I-3jgem*inDs&lRVg z!Z^?fJZ7&|{NAIiU%U?oW7TutVM}+-aAv>2gU4D-XrCDV zWDvf6D$7*#mqrj0c?tIMW|D-BlNmtpe~2xSdysz=-nW(5T4eL_sU!5pGKbzv+EE=Rll| zJ|2u7m}fs#JkeYLmKkG-*clO=_F!GUaD#ieV@mc`8w3K=&?{AIpBM2FUzrU=&770Z z4X*jUZAvkZB$2nIXXFh&D;k>!jx`0A+J`?doIV)leE}A$mSOB~+OWc;aJV<*R zpNstc{MPTj*YDqA1-?H++}-9Jhq+aHo2Z_dbK;Hv2!f=9smr@J0C4`4~R+5$z8W zh|=dIaQWuauyOGXkd-X5^&D`< zTeeSQMICUMx2%MwyIlLcE_e8Mo1=~Kt#BLM_&l1e z#*#y^pE5XCdoXd?F@t4kcxSfI%n@(8v*}Qw_-f&ozhT&fpf<}#DgeYFE+Adx%ljKyD}=O^6x>6qv%5>~r>P znlNEN6&<*+-dX6K)*9Qh9Wj<3Pv@3(I^j^uHtNic9#vH!(ME8dMH^^flUi8E!uL0Kp+!1Ew9bvsmH|mN8YKt= zF{sCLHoU8bwQubip5~ zKQypd#4y#1WoCEG4&$laVqkB*DPm#W^c<*e%*XmcO|D{Jw&1fJDMM+U&To)zNmvZ$ z8oYH=k&mtg8;ddiQcNrtX)R=D%#3U-@_zAp=2RBp;d=C*2UcIm`!W>P=a}AUFij@e z!WT=8n!f?BK0N9zXE!{!dcl5_8PE=zZV!~8%=*=8VYlV6lF3X29j$&8d%DG@Qzp5+ zlh?>huriKLJ8o^jW;&@)gySG;gDk8TG3Rb`_VmFa1?Z!BQ|>0l0NtPX4j!qect56% z`0Z!gt90i$iDi{v<~7ME|7gfR_|j`*Q89q_ zZo<7#84u*M)p(!h?YkC=sVw1Ck~aNfZbn2nsOiBgW;l+w_b({m%8emitZ# zi;^>hI6FcB4;I^23}3<@p8Lo^X114@>(E+Y@Ki*Z#x5K4cVU*R_Foz|H@J3*gC$1Y zIX&kWf_rV$iRKiD<`swl-!-SeWHD4Bc+?zB;}kj0s-ZM#p)_r-^uc0FKGs~D$rF!W z=7{!EK(T_T9ukN@qPH&D-1v8&{X#&?8<7;aq82G=FA1Ur3)b9PA7Un1Uhc_}(GEtCRIC03G@^wW%uxFGe zk7`Eaw`g7KPdgs4J@tY*&t?Nw!3O#B?*R=T2s8yCCGZ{1Qdc)CSIIa2cM4r|t2X9~ zR{z$^s-ByEM|b_D1r^YtKO-kqD_=pS=lj9*c4W$8l-FFD&)g=oD!3o7H6aReCN%20 zSV+7%oGy)nj(7D9U*PliYV8QUlNcM@FHW|dg_?l?*SDaZq?PO-Pv~5wlWWKZaxJ9>SaLEmJqt8AxRfbm|g+l z7YPI}i8FMx*B$(W4|Joai0GA9( z*Gp*l<+d6peH}qKaS#+7gOmtC!;hjE&3?X?V&#$}aJe!OseU<&Ut6T6LQTBPlu`i1 zLRERIjAR?L(wY^g=D9O@0uO1U>=#99Y;_uXa@jb1AFOy4De*(d4*+51*LbgK%F5`7}cSDwo1o$+n9Pk(t#@$TRmR zCCl3&4^djsK*2~T0KFY&Hb7kheocUv;$gP;p=sb#y@39g*TFy^%3UXIFyaYM_# zQV=P~h?s#$_fVnpqDrAmz;f16HwL&*Bq&pUqRI~z*yYlrkQhkD`#Av(8eJ5+R3BZP zSVsP~V%Wo})2?=r!!Kfrtn`KXl%=jaXMK>-`uojcT<@(}i`I3))<6FWAEylwuzg${ z)L^RH%B&kv|C4aybq++l=%a+_4I*=BA}3$!F|q) z&Ay9tU>c3Wml4q!%ti79qxajH@V$VZmGFA zz1)veY%Mf*=!_o@q7anat9wJ{aPw!vb``in*Zo?rRt(VIW3i0`(ciCAt zrH`kAhVVCr&BqGBP*LDBfNnT~hB7wzJsMciCt3F*CdIuy{~8>Wv0Z*tm^mIZW_3ma zxN1m%y%VpAiC)W6USi*$m;1v908m;i1_DIY5I{^+9}c2Lp#g;K2wspH+gAbON4|o= z*djIG3qyJGprbzKQ2k&CaFW&P`R{TQ#~67mvDe+k49Tvv&=7B)39Xo`ZO$=r-no&P z2B7NNJ|*C&*VLSXl$hd5hf1dR(Qg0sZ)NU?}lJ;q!ro>NIy|Hoq=ZvA05KpjWjnAz$O-XV%pc>-?T~J-+rlF(>3>Jp^gx7r!!9g!l7c2bf5xLUljOhn??g!}9 zOY|{r35qE{Y((=(VbUR@0cF_Ru}1%!_k6++ZwwJ1Ofxrjl|>yM5swd|4;>s2XXTJJ zpG_^FTX#gWy{I)^8_>+W3SKm>IC6qOu^e~MvZ-PyNrv&LUOBQH0^Wgx45x~J%X286 zW-!e2#|f*sPDizI=HB!+u}O$Nlx{kDf!YbthfPaLAWh}rCyR(OEJW%4MfOJFL(EC@ z*ZnHG-1h+$kg!mjTZUYcZ=D81cS2MfvjL}xm69hh)gk9G=ONV^KNBTTV%}mffv((WqyR9(K$aq_fd(exiX0whr*gIsw4ZF5O-|!d-fQ&-2(lrC>V*WDxiU3V^82;z z72Q&NgMuVvPL$5?HdxI~`ewgIN=Ar}1O%#)!QOz{x?fS}tq9<`VwT|xDE)&yPm!Kr ztILyJ{B=Af^k=oco0Is1Ci$9H^Tp{@kAH2;Kf}dOW@ek4S_YoS-bKulwp`>%N>g9IefCW=881HDo#ouX^fa zTQa}o_E!9Awc#}}m1j3VOy`+zMxM8Avv}U!B%0t3>x&(L$<_|l4kZpH_SN*i86p?} zO#lrECRCAl${>-qBJm=Glmot!K5Hmn$c+CSgsIngNyjtE`t-i5X~?+TZum&~*F)WG za2pW!5I@OSK0O2l0HWguPs{rc+mCm`K!{0p<#-xcrjQ*n$QI?rtI5*5N(r+Wi}p<+d!e}ZO1;g4|TvCF)Riu%^aK;1`!KH`Z{NXOfCHaIuqx) z^DU8)Z)$=j1hNJR|7zoTbzNb@00jeuqC6N$JB-)@K}m)q^Af?kM_(%ROEXm#4q(#Q?Y{X-Xc+Hf9VUV@dp{8z=G> z^%;oZ?;J6W>j+Jc(*)}aaWPJSqy?oYm?CJzlmARk_QOV@^{jnju7Gf-?@n;|bA0KaDp7m2&}O`;J=NMUCi-iKvHB@pId-XJ9LoLROwKq9}+V1rGEFyoSSo&j&+GL2?5Bx~5Br)wX zK;M;465>Xk9})89V~CPe4n|%rQ27%45YOI~c@shb#o3YkT<=|xarhk6)`=@zS64G} zFs~m7JXSJ`Yua#@l8GF_fv2aM}79%$u$AJ@c>(g^?3*xY~#1=E*vd4}M~UY0bMg`&*cJo+Q3VZX-___X0SwPA+#{NvRlL#+KgIkou&UJsYp zXwajF<-i_dnh??q^mb0!K;n~jsoB=gcPTE9XPi35r=%eBMuxS^+!#Voz_8aEDTumD zX+=uiz*sXCA0u7kh9F#zl!}AhaP7bBe!O1=xb-g)^N3!j34ANTxcL^j zsleTIVH=GT(SjU zfIH`x=2hBcp@AFOWRCxz?J1j5@BMwx#y8p@Day*TKSiu*cS+#&(o#me`etRFi0Y7* zQY45HsH%$Ce(QiJy(&w-K!1`p}pTyqc zuA0ar^r@8T2hp!WOMW#KJP;75W{uW{rC(}ENno3z97R?qJBp?jYWA8{ss5iNY;FOr z)!IJavcHP!KRS+3IDH?Nq9h&ncnG%>eu!?l84NqE4PHKP8ASv69OU}*zIk_BEs9VV zJ!z+U3V~Hn0>q*H?N;mk2gCaEL4a^mfcEI2(Pl~*YEzmR4@?6z=R0APbyfN79Kp4A z4NDe!U!oyb>x7~xr~xs>F~wH_`u=rl4*OC9(;Af;&`shXHPv@r?Zn1DTEiA9@=UNi zNcy>lr(8K}#bC;8QnXHEQd#MtvIFNAfPEzs%aa==>o5p8*nGTva`seRNcp*G*SX$w zcZgBUasDk<4>74L_t%#VX;NbyPA;AW9hjkAC-YO*mycN zzo{cV>>BeXKqHEO>9RBo0bWgyXnM>NIR&y|k9CpGi^e^Bcv*Naj-B__cor#K2(X>U zUD1a|VZ$f-4?Ooqq|Q<}svCfKV)Q~7i{eGwAc_YwF0*&d^Rf#4SmX6RGbCBIZYusp z6O&z}j8F(HL$$6VBv1P_G3ZxMyzr(GTWvtRU>FFC_en<`AO2c(8iZeamn)<>O5$mc{)#c7CDGQLdJ-xLkcJ8%y?zODfE15d_@jb9oh~jtEs=URa$y6(J;|0Wq z)~3ISLCR^S7`cb>n|th(E>^;lS!97`bG`hzMv-5~DRZ})t*F67WFL^_aGiLK=h;rN zJvENn0KAWTjVhf#EK#k2r%wyMl0<*mAvI2NWw1FH^#zug&^}XCARY0gq|n zzW_Q*YS6|UXR%d9{Z56;tnGu1wajeLG+Nw|jIMf@0vDbNrHz$)u?$ruj zcG^i#d6)34FeQY0Gk81e?HDtcm%Jhw-$qu)oYWBho>np;FHX>q>1}zq=Po0k+Re3p zJ+D>nLR9wSN?_-A_2nBbV#c@TzmNW9YmdNxx<8QbyBe3GGRK}Be6~IMuElyA+Cia( z%Z4&KE>2kPrpfWDc~FvFq9q-hkYL}c|M$baL@s?CjovPYXexGcB2V+ihz2RcU~R$| zEv(!zV~$avBlJX@Z7~M0+pfpBX^0qdQn7=m1 zTY#tSA6OwK;QUUr)0o2<*kG9ra$Y))tbKWPhPfL(YIgW`-xbH(YMxeyU1tQfc`0=I z-g>zB&HZX;yZ||9ts(axiTmG3BTw3MSBh5i2LE4EnWJML+V8@fYf4FSlVlTR&pcjc zZ(H$izO=ydO^MXQso4*8-wBHY9`9lo0D`3GAr?xy)?8=qN7gZi-sWJmlj}Pj{gK8D zx|ZS!yl?rS-wow0LtYfzT!%`dn&#WTs-5j10d(mMZ-21zlh+8^5LG`&B^apfCVl4Q z(3a^JJeDNReR;f%JpLxDFATFyaFRK_j8|Ecl!1C7#l-RnJt_zc`4^3)*OOBO_ClXK z`*Hc&8W$di=!eE~W`9>%q-m_+{hZiT?fe``ya**~V1>@d-PLtqncC@0ozkUnW|d1APrJFNSG<#LdFrSMJ##-97~p zee3I(e?izNWE-8&>tdqt@rdaE=!`(@+ zhkGwCt9KcO7KcbY?oKrh3A`zoHbqmf`@1W{MCVUP&9c-K9O4 zxaG2b=jnlhDO_RnEIq`fuM{+8U0#=DI$Fx=FNZx1KhTDrtRK}>apY5anzO<|0*AXG z@)8NBDNaf8qqwY7D)%f&1x(Y;?RS724;C+UMfN$CE>n_4)i|lpqvC%3=*A+>{`GOy zKA*@=wV&v5O*rQHr3|Gr;;RbLH{UGjCOUZxwlMA%0uDsG1Hv=2q*=uOu*`VIA+)=g zR$oD&eBz70xJC&}2-f9)a2?mSC#Z(2e37013|W=J;4W0gKu=!4H<`P3I~9;!1LJjs z@DmuL(V*Z3op_mi(aPQA^zG7{c~k~n-aX;pG+h;zi$L@Ah!xGfY43e~g0V9# zbIM{T<&SUqj)d-24`x%MyXA)o#C%~;< zJ0&m2L~o=bXB@C}uKG#OlEyQ>KS{|Aw%3ek|FNzTG@FfvmCiL~wW-Bk0+j!YT82|l zOPZsW=!tz^ix_80T=;p4g2hpm+7Idtk0s#4tkEBScQuZ>l--L@HRgRNOKOT%4%w)D z5!nYvOj@uwXRWKT9xZW90jJaQ$r(vn*%A{C+AMb0K1{k+&hVTU%MIjBnJWX!%Z;et z>n1SC_g;j*fN}zR{=MyqMnT^ssN2DG0IdNCY?U_#wWfJPBrji-7%z`L@@_ECDBi-m ziW@m#EX;8-wmUinE%;SdaZde~!!Mn(WIB}9z}je%_seWnavzup4ZeuO^%P#Tm-09&ChkCb_5Ye&#-(7H98P-MM@Cy(f^U zmVTZd`D)Xu%%cRlzANG+% z<$aig{hLXWRzDNDdIE$SWrMr2GpX)c44F}S8Jge)i|Ppz1z%r8016gj;4(veYoOYa zM}`ekPcBOO-N_H&~g*1UA!9<(m{q&GQ)N`*NTpz2HUr-s)@X zbmGv)o}p~uSLwf6``LAt5xYj)@r=<>{Z0YtaoQr8wa*Hnxk*JITj;n})@u z&VW_Y+*t10JLdfo_+a#=9m7{eB8=By+^WUB9ic)WsjEzr{n4&cBFQOkU6ShP{p0yf ze#VmwXts*z`OAk1BBY&w_pP|M1IO3p_ew%Lj4=0?PiPCy0mckBjy%Mm@g?)m<+^c( z9rWAx${Fxs*wv*>i%G-PW7FkBmGgOPaL;%9hh%#}*S(D=Hh}kI7v7H#xc4)0kDdMx zqYtM>{2`Es{_w|@J5P?31HE^^rW-m?1AZYaeM98*eM4({E%fq!&8eXx^A9{da@yW} z`RXNU5V0||`7b7P)D?NTIB8-1{q+HX@S9b|51ir-=YlsqAWd#LI}j)d()v4Q-j`1a zSv2BIXn3uKYwrplj?Fm$qJN(r*{xZIy)jneHn#hVM+cLM9 z^SFgp)wOx+ci%oV@5BnN-AtScbEIc7wcOe+^^YsnzP*x_jX2+9`To_n!Ha5f85JYK zk0Y}ct2SPabchnu+&oBr*ipYJizhpDMZU_|o#t4-uK{o`DKicRRhgP-F0^(b0L6-Q z;73+zRnHOIruiZvBU zS`Pn>S<#1uUb`p~-O{W61coOVLY+Y|5~yXDtaY`(e`^~A3U2?Y!lnmhdz5P=lKh=-mY=lHsOc-r={=x7T8<-W;vZVo|JfemBG41y6 z4n1*vlx)BSRjlvBDM~jpc_Dyqvl%ZB0H*o}OPVRL{2kU|L{)6R?kC#p<#{t#ygT+` zYBz#)axK8lN)W6bL^m3Z5xUkt-rmo%Q?VZ-6Qe25g5N7`#Q^r+-pA-LIs*8##Cs8M zNV1d#IV#)DzLhe4BlYD6V%JZIfa(sa_{QYYS|U}j((cOB{=?F(`^ul>c{0RgtepHG zuiH(nhsD;%xW}t)e?rjD#0#I}NWRuvfB~`fbazzfJeTS&{QFAzV?%4zc=yvx&{2ep zQw$uZ>nut3b|H9-wY6r((@K@2tROh#`0(Qk8bxw|CB#%c=~_gi&c7%17i4-i_zy|Q zN~HP`@^}rwT1t6CN-|_HOgd&D=(_Nicrb6up`V{0SD3V#Sllr0^+sxM6dZ9?yj`Vb zsHCOkA;jyoyq{L?%z4Fh>C<|*V|poi4LiCx2yDJ^v!tmbea)gUbM6GP?v}5uk7LN0 zHy|u!I3I~-HBc2oT3J!*Dt(qkh|yi)MpfEMCR0EybZ?t=cyn?`82ntcY02N;RPjwm zujpxiEBp~Yf{U2JZCyVEB(QyUQ+!TS{Q8OF9`=TI$Is8T{4b~d7N7XF7BPDc@|OMb zcgsjjaZV>oJCn9cCU*_>4-F0a%gZ9gnJ1UbXvINDA!St0RVy`OC`BlR+=8YsW(vHo z%%g<}JXhZLF1AS>A`#_e7RV)E#4M#z#H-3lj7)g(^{S-k>9DhZP%y5o>NQI_b>RQ} z$BSqbm&0%r6ktS>2m375u-^f7r#j_rLR}? z4u@%CPc^&Z56Ad%ZNjS8k~h75D{TTjeYFIWBju^yf$D*`owmp--%@O_DWpEU_IvQ@ z$5w>kf8(B%$NMbb5$|Q5tAOzWCB&Tn?Ip4SK$BttkJvT7Rmpnm+Ij(>jogU6Gilf1AwQy;MQ}|$7wGc1b~3$ zi+S)knw&P@{xWry0{m2B!HWx_@hzXg0e8vn_Ne$AiAIY8i{-VBIWd ztDF+!tn77Hqu8rA*ztTgMSmPFXez?U|1{phKG0Q9V&0IplV}B|Y5z&OY8@hkW*6Vh z`^(znqOJhjBMf~y&Ikv#;r*t{h!;a(WMk4bOchGa_R>3Bxs$UF!C>K;$spI{;_C3A z=XIO2bN!M^ouT0mBm_*GCD5{dlevBC$NTPsAG|RqoKi|0jAOmlr9@Inpp~<%%x=gF z&ZR!_S;p)XF0$yW3Ndq>@{TPP&A2v9D6y}pLli7h#85*yu@)GaQ8veG?#(wAJ#B+p ze181~nrnuFB97y9j`8KM8EBb9HhfcP%+y-xLpvv}*;9&7gJxTQVV9 zc?-E=ea?)*C@(RZ_=XXIrvX8jajAjl{(Nru^2RWQjB$;W`#fDhZ|71drdmYBRNtUk z#NSL5ELMuX&vLzh6kpbh*@k#H?k;X$bqdP8N1j`<`1N106D z-&qS6H>R+Y0!l&O@!%{Vr`+E!ed+*Wej+>gTl#yy;j-XF9+JOKXzK+eaH zfFiVl;VIFz>+I9uq-&T-F85JQK9?~Fk}2%1DPL5JYNR~Zyzu!V3QPlyIEEV&tI6BS zk2Om$b=PShzb@COXldxuZ1DMxVR}_{j)4N9-B97fK&Y#?M~2~ht!L?5Q75L__ZM(% zq-c72w~gyz6v-U>gI5z!yxGfAb|UlPE3CeQazt&XETtiZFLHK&Ovg!>VvUYBzgHYX z{iOKJh`lr~b3Z*JCs2loKC^MK#)%1K^t}B{&F5iZrPlfO?uTqba|RXGlN37`F9?Mu z2-5x?asU8p)^81F(pg1r95$Pe-3u&`g2KoAEDCDk1>T=&+BiJCjL1%mf~?h*RnIqI zj-;;WEW8`_9))EPc{b!rdbI_o+rJ1^BChjOxp*xyT_hCfa1qw}Hh0w#n8&>(?)D`} zq&Qn0^*tcV5^@JRdMV12eyg^osDFGPwtbJfwG4r#o*_J?bvU!rzD(=$oNbkWnlhvvphffz!;A#2DJe||CuC%MVF@@yS*-F~O_>&ui z*Cqmzx$8=zI7>VBBW_yawHoX|1-dNz4O~oeJ;Wh@cW5b_98Hbw>B7mbA!YVb@_)uY3o@U3vYBv>%Zkjl4 zBDdC5l{HspafcJv!eRn;A9&s&lG>5Mn(KO#Z5zArNB@Q7MU6uJa zmGySyXJcL%$96`_S$orqPKnRaD4-MJ1JyOSfwupIO$*VjI-i>~H!~mezu9=ja@qp1 zC|nu=KZ0vjQ;Fw5Cm@5=@WN*u4UH##HJ_;V2oE8C4-_q0$*#q{d1LTX1LwkhwCv-5 z(%(V=YOGGaneGp#6E>Z0d`b!Y3U~7TY4%5?xru#iEd=Do_mwG*108@C;X1ZjE(MBm;flUU5)c z>tA%G74}04`jsVUj46zTouE)DRAX%C=YZUL;>!niQ5H-Wj|J$}>$LE|Py{MU2b-{s4`d($Fc;4QP%x8~eNSdoB~+rNozJb#O8+oC^Y=#~=z&v6?h0ke z9-x^Vw_~%b`D)ahI!KA$!J+0S&vc0Pnbm+5@%srm9vi?$sUhGXLYCch0U^z#j(rm_ zyK~3fbBsG9GCZWYIuFqnBW5i)H8&Sm0?$PsFr@WB8kSs+TWRNvBM) zyXxt0Y6mD!Lti3yFRY~e+U`6B1RMKZe!6$xS6%t0fPEG2f09?hj=D~m{eHyyGgL9f zWh)#o22t5HZF`4fJ-9D7@7@B8OoxX5)k)`8${v*ON`GNJAEfC{`0G;H<3TUAJ($GP zkpE&nu3(gIsirz$)gqhjI@etezQ%IC^LR z0!6wG$GQZ-*{1}kds|zPnlInuk7Ucj{o~5yP(O2vbt?X5#c1G zplkvHqqf2sGWOOH=mK{kTV(!OnzA^5aHEy3Us8 z?VRf4k6V;iy8gW0TPeCE>1)N&3*@N3_gND96qTKR9z1pa)Lq2V@?lZ}8!0GghVe&7 zX?A*tND!zu^5~mq566esOQ#((tG7Ca_Snt3&cSOz;#L=6$k2~(L;qFaEx+$Z*wWKt zZol$ss!NRULQyj(g|00!HQzC2tuS7kz3`vbVOU?e_pFWu@nQ&UoOpJ|Epo|12KrVK zwlC;@x0TxB#AD3);ShN?A;C<|*-TU}|3=kl?6PQCy+;ui#Z1sT6VcmT*wIwL zkyMVI|910-yVJ_&g2>!z&K1{!&iwUiLFMr1-}~*XoVKD%<<&!4Sl_~BW$s4m;=8PA z2Qr4cOF!)_s$33?n3KMR)k34ci-YAWW~w!~SKUpZyY6x=Pm zHvDVw;6Xrpzw{k>9o`?db?&zu3h&lh(JToVAMT#rd)rB8i`j3f!#C?RZ2VheY&)Vy zl}7Xm8CUcrys!yp-276WDfdd`=XZ&qj#O@e*V@HA%Aygf@A`oH)mWS5BdHJVD((ub zcXq4-G?tn7R>21R^gV}fET1hsH@3qJyM8*@6SC}DK+3GtjRBID0sKR)rkoj$9eRYs z1jR*_R^u{E!6Agvggs!YtV|6nEBkzIIlz`^?9Vv8!iy|L6i0+jxIO|yc%@8Gnqs`# ze6CG@sn-M_E|UkOcSaT$Gj@2^_W8Y^zqqy511@bgvEHg4TYgQs-1qOx>R7*(*hGW0 z^PG2+^wG_A=hHb7gkkPvxV;h3oyeE?i7Kia)0PI9Rx+ zI(z)$@HnE0Xe!C^iVQsQsYr|I=0)MzEXE!~0HK~Ol}7=lykl($d;N5k70O$zA{cUx zO#RY@+J|PX=C4h=W=4(T96mfBYW>t=NTu4^ zEoAvppNnd!qc)Di$x_-FQUs#|&IDI%K_7A@hni+w!d@XX*kXHkpD2$M7m14X{%T{s z<5-w?r4Cw2*Bez!D}3@O^t(i~uNYsaRpFk1{d?-o5A=o}MS2=#6i5EL?jlyXiw3j# zw?IR=es6ixg$zaZI_i|e;$Nk)lfjJfpJkNm+xjc)#*V>S6X{DVW8eW$KBzklj5FYc6G{7BIonz_F3or&~n4#Ir zf61sq0Q*XfN{FVIH~VHyM`%Cp_DD77?|fy=+0EG6OH~;?e?o%rV}#B+_ljKSqbeee zX7=0UM8DQp7^&g()XG&xN0LJP`t(BVjEai*%CoM{3hnZ+aFWHVY=9zaS2y{7{9Jm( zVRzycf;UgDZl9b>@XQ}?W@Hv@Q}lP+n@2YYw0wD6XV%a-Pv2tuNwC_u?7m!>p|IfX zKOOq)#r)_R{&XA(J>2NfStA*uPlP+%_8x%5qPQ(*sW~`k$quJ{5dNU zL-SEzUWYQOHyw&;_U|b}2TMS`Iv@SLRn#)Km!qsV-}GhR2yCnWxnOU4_lqcs#%;9= zi@d)4TeL6sakV+5g&iDhxw@72!~T^vAMC@94+1OxEl$c^0Nv&`#qbdkh~2wz8wHGz zV@nR0$wYjJ(@EMbRNan?t<=+x7_vB9(hXQYh%sD6UTTBM;ReD=(0N`^TQU*47LVoB z{A7CmXg;kRkH(Re9ZWJ|XkutI>9FMj*s7Fe@O7e^s%TxAi`mn>`T;g-6OV^KE3EMv zVcnaj8Zj4ZHFS2~dN1!LmmOenc-)x(&#OoNrh4%1qkyrnqV3GY}GDf}8}&T2Eu zNfetNmNRh>PCu+3b8($+_sM2EDXTs6Yuo*kkkkYH4!)N>x3VJS^sQflxnOUndze8T zHrf!UwPb<&8#jObr^+}*blDj_e`Gpc{0r-@R6`rp6VP8tfx_-y`te@+_J?bJ`lY;> z0*CFmzJK|69nrGp{rSXJ3=vW?z7rhWH;|A~-f=$UPF^xR{1{H&?6a(2_wmt1jH{6G zO9i*>_~-Wl2O#Bv@Fx^{fq8ebKvbBOLpf8hblS1dj}KvbRo(BIq97&wFmI5p(u)@h zer2w0xUbl9SG-^L6~Z53fNb8(GEfsv5qD%`Crg$lq@<#xD+`&ijs^HhzZ>2Hmure59(BW>S!D*Ps0IBk)1%aiN@Ndkn6U zJ~Q-l{KkWE>v(F}&_5rB5c%OZ&pHR~o|220U%LL;yhRWZj}N{dOMV@oDo>yrE^lQW z5Ght+;uDk=j14K_ghAjm$vl2vwW0X(zdV-uQP2G58r$qGcVw7kVw#MzTy2^(Tx4xrT z?L{feY5Jn!oP6_uqui$unG6HLH>IxF;jHSME0G{y!@&nty=mksL^}5ZRj4u9oe>*DsaA#Y=tOQD^#-zm%o>41K@Z(Av7xe%zt( z=MS(?tZPk{yREHJQwz^l%q6|9um8n7c==NRA+dOe?dO1f3It>TuLs=S-u5U4vWNZ) z6R4QVFh9Dhw(*ra8d}2N2Q9-AB6&!&f}0~)J;+}yFg?xnbh9tY8%jpdzTz;Q+S!~@7Eo)eHRcq zqyKjI&uy;2L=?3L^vA<$)3@c&Uy1xmZm);nZJYB%7hN4xjxnn8Rz+n6MQmEXJU|!v zgDM&ftTH}LrquyCzq)G>)kbuM_v5HXCnm0(kF8Ks9}mMqf4^Dmwy5`+j(f1E(+Wg! zOLkj=jpD;LyqlGtfN;j`F(N`4)<*`=A$kt^=>4e&zNXWbVF)9gB!4`YGWo<$m1uz> z2XE{%@2u;*n);~eAs;fdUDV4sf03p0Q;^GTIzY2rXI=N`htM)SEJr8J@wHtJH)7Th zt|x!r+RC%IiQ9sL`0{sM$YU$lg~D?F$QfE|tJAM5E!eXhg8P5CnW>b0IP;&GMw5gE$wn@oNMo_(P5 zdd@K=h4>Yy<_cvw8N(IX$@Ddlb674i$P|xQ6$+mI`H5V9kj6{~dmCAiL}o;-wcEsj zX+~sKUo>_HIN$7^1}W=h@5IhRuP`RDuAj=(XL%8Jc6KV5wv7?Kp@AXUXoEyD9zMRE zy&Y8u4Cta|Q3ESVe`we|ueQnyBZkmhVnWx5%g-5}rL>b=@xIR`QgNzMxe^UPejLhzEYEkvUdpTl03j zYH~q?y*TLehr{_$D>!DG)s~6K(tts)O#c4%xu9o`lcdBA59qO_D2GO#aP~pl{cC12 z3=P~iIugoE(wlAjp=0aJUlOWAx&1dWAY_&2anaYil^RrShsveA^GU_Qm$%&)YV%J= z-L32Xi7(yoEol6#%tiA zJyBmr=LDYKA+^1|!Sx^&RjgoEcB^m0m8bFZv~t(SFVgDH;#v7sXhs=bZl4M}N`h;= zCt_NykhQ!Fo+qqUUtjD6cZAhhKklwB{ba%V^R;LW(rl{Bvc0C}Xhcj)E^KpPt@Cer zw|t|0+^;+Bol1Xyy)kf2(v! z6a??;n^8oR3s(vOqITZ%21nPLs%o=(Z}J@%Lzf<4pcGbvacL`j&J34F@&Y>Gf1Wvf za%7P30s=w;0(Ir5hMIyR^K)sKwiv$1tS|Ex1ymq@?BFiY7$Kg7LJm;j%wBPVlXw)6*~S;N52`FasrjBRJI~s-6ElD z^SuqbNlx$^N(MFw!8Iubapj8!+=#64!K_AS1AR6ja%_LcfB+Q%{38B*l^lF)MPXi; zp8mTPF%aVUsRD^r5dC=kKE99SpT1tnws+}i`_s!H_$+q{3lqbVa#5{oTP>%8;-RCJ z*E`zhF*1y3o5TdEdqKS}65MynbROJOU-Yy5QE0I-xw7;7FAvHRG*Z9Kw2{u z5)sU&fvubpec}`J`uQ3j6rfevl}OmzJFTtv%?Z-*+bsj z?6)PS)e50^y+clY5_$o52909F#pdU{AQJo7Ua~?c7+5j$$L~~;_4w-y>&u^y{&YG% zF5=cWh>{K-qT|kY9ZzU??B2RffUz{6%u1s4M322UBPZ!l?7s2L0u|nB`Youtmw`L&n70 zuC;1eo%o}>z68QbrXo?x9t?J7G?NKcH$|Dc##s$db{Z7A(xXLlfYMI!9RLd}zV}6d zi`NJW(uVmG({UF2z@`PvqAxk4DV9X#vHP{)*V}ZsmxQYYgajXm=lcD)0afZ zNiDJrbU*^8SvXEVpFh79BMA@Ks z?8M5ol7v*t0}aF9?QOdmPuq4|&q?NzumYY0H<7^Zmg_FlK-b20Dm2v|V#`S#-e5;I zrI`nF`I9f|&`;CgFY-(>enNS3sbPAC2wF1uq9kIr>Et&jQpz&Z5ZkArI3va22Id5o z$`a|&{owau6_s<*%P2UT$6rfo2XLDtas?Q}LV8=e~BRF`)UzJ`H(@p%k))kFMr}M_u+Om^0 zHcje+>0taSvvb%N@Ii7K7!4X%C1GtzN{>Jy5y-yq&f3agMI#1=YMgb3W2J38x#gmi zsS&)pJkZ~g!nse;AlDJRR?wHrg6t$4q!C?UUAeQR&{po|Z1A`Do*l?r!9`Ou^poH8 z4XKT%Tl}HM*C(V(k)QALTh|_{ElMHvEbd!B^=_5(2R44C43HXi((xdTIH$PPXQdEG z;s)NLhf8`gQ8b$Nu|P8(Z2EGUN;ZXpgPubY`F>|+-vMq6Jmvg}b)68NS3?jUMG;NE z+ZpYH6|TZrBSYKDUCjQhk;BOQf5!ePYfmrs=2i6hsD2n#2RaO=6Kr!Zr&dpdW{xcUwwS? z`^nkYQ@W!4^}6G7ME+y@PAV1G{!M8IfjNd942BOmGkBT#;p84NsMj2Y@` zU9rr+@b_8UoQZ59b`YMqm0AH{8N`juYQ`L~PvUfAnYDAmM7c+reFmlO>lh8j45!px z{L0U-Wj)i&l8MIS>3Fr<4wF}$KE9M{-5!=Y6g`;xQO;7UN^U9V@gWwGpB;Kl$_*lj zpzO^P^COT}y^i3l4kmE}8=7VlHQ?f%nGLLLC=dpqe%b?JeHS1B){`GL0aZ|Sn>`K7 zYx|qat${Dcx0T0+|GNjDRaQU0ML&4FNKdi*J&BxcAIm8a*3B7&T&{?zzpF&wFZJEx zWo_7%qs40vj>^_QMb_a#_)!F-==BHB7E7tG=$}04+}^Cl*OHG#zaNMHS_rBHKwb<0 z^3Jt0&lIJuk`I&cAJV@X&?7U?4{q^^KSw#bV#_J2zR%o9=HmDh)9Ul_EBKz*_s7#5 zOy?!xi*qL_CRM<}Tu@dw9*p1}OBGX&O_nq@8jL89#~v)skNOft1^fT|^Jz*93C-8l z_Wa(_vYz+vpDzQG)oSVA!E*Nx~jWcssgcon*6^tR#s1QkRs~G zyW#Pp`0_&j%)5Id*NPtqYRK%Q8bJN}Qq{IS?N2J{$IpUE)om@zSq+}YL)jg`5yyf)D6l%`^+Zg-P>ni z`qj}o+<$j1eE9S4X1B%t64eLo|9(bF5>EsZ5-^ec7Zpx#(VHgpKn~k#UGcaox2Jib zHHtsfIpppxAq;V^cBIR95$V8z&;=o^S79#cx|V&Z#`x_lphQR!ejLLNpB>BRu4(`w z(m7$Oz1knc;J@$NMb5`flDI!imtP*dd%n#$gZ4bJDIeaCz*WyQ0-?r7C$b}V8 z@^A;pRWo#1bcyrvwS?4Z%K3kT`u*WY=`JRQgrA>3A|;IhR*Q{;n|COd+U#bm?v+?u zN_-10%&$NTUZ~2m5Q7c2$;ROz*J)} zN$|PFhnbnnxPzgovXQqMn&q94WwAmm)8_U_jozcxfAc>b146XCy+?BShAyt`J{Br4 z3R{G3{z;hTs;gda1wx|ie<6|Wqh13}2+x=nveLjx4G&L96#3BvS9!Th1ea^-Y@Y`DUC1qyrwr(uNhmL#8ZUTal#3~C^s-a z`Z*{DiVLzDG(p-p%&Ya}+L@xa_=h4=lXd^77-c!kA9}mr2$xstOuDhY#oADL50ypM zoZ5ZLO~}LZN$OhM#LKLcX#{Alu_AwEi!J(F&eFrP|Ap0imdaWlYdVFVP9*URX5ErP z*SCCKxLVereD<)K>J_$CmYSR$wYjzBg4Nb$+#L@M+AVlb$NPA_Bd8Hhxt5Zc_~!Tn z@aD{h*FVVDA~0W`eTctBaKQReAMDE6%YSrLNCQK2UqW|-x(2H8ZQ6%O%(fb&kKQx= zLz9-om9LgDz3JI||1zYvZ4wkN0UBmW>dRtU7T>}jcBUTB+;Pynj$oyJ>C?k>EmIin z5+xH0_@Q+aR|yV*f5x2jgYB{9fsuA3ma150@|D^vLaDkQgx-D3;dS*qjX@No7nAoE zYFLTt?Oe?0Trj}yoL{c40k><)W;cVF&wkW(W2#L)re-g6|K^dv=e1IC zWTq8{1>UUh=-jgQ-}zEqTQjWq-<@}r1F^wy`9Gz*T}DL}-J^$(RYR>I5J(60Ooxl( zt?NLc&$3zmxz&~QJ>Wsps(}3t>lp|5Ygd&t9@8{`e9D@0-^(Sd z&kQtwZuRHX#AfAb3YfeST?4NimVm{VLp}oTG-lM2>U|`tqS=&8yt#HJX4L#cmF{>! z;~cjI5yULKsten1I)N{bSEDkNc+p%-CMi{ncT?0~U5h{emH}jhu$d}s=rcSxfZ^{JF zecl#9Alo&$NOYZ;7RYr`g1wDaVS8fMxG%X_vy@ir)%0{Nv!~PVz}_ZJ@d# zI{I)?ZGA)JJfsh06IuFeWtN8*yOnda@cQe_d%afU~t6|BU99z&IWv)aSU?u+#+p9p4(+{o$=76R4*b%GRVFsM!`!`r8 zjIZT+gFA2^57UXe)cjxE!MSoiH+-v3YrCvQ_0{!Zo__EdC+iC%^ZH2>`|5Wd7MyoF zIi5A=$p!4D>yNA69fQX>3md2gUzbx=HqqZAN~f*0xMSbf(8R7=6X~KsRY49A5d^rW zKsgdRB#CGgJ`8C8{cUCMJQ&#=7H&Yn!J8q##ntcea8{CPo{|dmaR&5S8LzINTCPT@ zcQ?N8d;fjZ)ay4?6SR4PPaNr=s@!J@5?l+8T?XT~X{1^zz1EIb!uz&TH8Z#~y{63R)(YdYT#R$Dwv zx0H(?gGbpr#Z(_M4nicwber@R9kq&MfkI4AP^nKw*_RvPF{~CZ#(TCL!2qYwmPS6X z#6^d2qoyQ9$fKrR984Lp+)@yE&3X@%Gr^Q3GRJ@VSV;8s3$_J=S6Ue)vBhoOD^{EH zJ{oiVGF|C2`qRhr2QtgG6lj~m5`MIH2-< zzIiA1b3%Lc4mdi}4r7u<&d{LwUl^c%hIdIXJCx<%4x~xB?zSuOl!!yWkR20xWOF|@ zHqO~KJHe&xL=1k%1&fAEPO7pv%=ctC$PMbYI_2cePtyqcRrB4>vW%0D8GMaf=PNom zWW6Xs-@}r#B51UHiEO{d^MIHhVxWWKm}%ukLgWu6s{b7pwfR?tLXXKmZiBJ{ZZp#+ zripq>q>^v;Q8e%2E?{?-c* z&?_F;`4Mb^W}PzRk?%h)piwe#B1RaXnoV4+A`FDZA>~1lQu(e{z2d6;k7o#h-9jKm z`r^G}XXCwP7>A-NLP^4u7Jyz1@bn#Vuj04k+(JVORY%LbAXyk5|Ci|UozkZD@YVa6 z`*n*Vq{;$-;caE5Ir*lE9W6M!Byd8T1q4=|kn?`pi`ooauyZ8Qa8yRqaIyiz+61}= zNN!NCAgSfz*aO4hs#?Qk?)|fzo{LG>Ey__MR!sx&Vn|#r^OPh!gWl2riJUe(k1uC< zU$*|c(c5Uqsmx9*^Rx0YM%l*uzwUQyXEspJn*fM9bsLZHbnZ&lfmtpc`CWU0pOBiJW(@ z?VMyMFAD$VeKicY*`*oF*pgdKlFG>%8`eGja-&L4p?zUdCr4^a=f(6B4JX!_mDHn0 zB!!MfP{XizHJlF8QQ*t(z}6_c?*K-Xx0h=0e)w>?cL|X>t0g0x31=rX{Y}~gYcbk3 z^3}|Y5|?RF_W4_;sF=>JuE%Y9o4=tZY5?r56MEkJz|flH3o#x)8~qR5_UW2+%Q$?+$}e>KpKa6Lm2&;?sL zxd?g9Je~zpvy6i3bDLGvfV_-(a>_@~?>~K+&M?8#3U}}^Z z@2L8?Tio{YU7ub3X5X`R%8E`-ZSZWD{$%mt5;iy+m0KlvCH4-Jo%!ug_Z0 z?<;e!_pYcTV1Y|df1jWO(|*+?YSoo~!MOpISvZ81S)a&Iisn3&p$?nC<|9u#QDKXOJc-tD=FL1k*xQs5wxmyyYJqqEtP7wu@rVC+G4znp*4Nn&VtmTar4 zPD+brXei%+e*fPjBQ@-?-g&~_)9~DH;fyiCRd!sji&t0WVIN*R`PX`ukwT9QI!f@? zc_Q(n^TjR)D(sK2X%2UTTF6`^~(-I2$R7ofqMx8Oz|J9@A&JI8K(8m}qKr<0p(a1zF?8I*&4B@MPBz z+YFa%q8xm3OXH(kj1A8wU5Wzpg9kYUnUdc}h8M4!){cLNbanv-V>++?$&V>3jmZMU zU=bdGsBQz_!Bu#(`(%dVl3>T#bSfS`~T620?_&{ z7|`YL^~Ac5IwogdcVx=uM7a78oi{9>WP;_#-{*<0erO0vaz%q>=!TFu zq7Y8wKP1Q8(6r48?^p;h6oXT!t%Q+*Pade9Ys+NQwHjKsX(RMQ7i?MFd!><>3IlW1 zN>5N{+n}*oU&GPYG%l-9S0@-Oiv2|FV}Zi@((MZkRU*}VQY;@~Bo7Fl z#RsDAw}EkM+t?okD`r}V=H4p4wt;~&66y!l3KYHNFIv{gf&3^CF-z2zBsC-usc$Nr z;897n*4fgO`)hP^%XCd+Y&JdVQHqPtJl-m3(D9>AK?F5A@vqpU zw`MhjhC!DNIP*nCO@kr$^i@ zT1Mjd(YVy!4{`-Ax+11IC^Z(y*-@ndm8uNr>tI&{~L+6j!8 z8rOk{6l}HPUN9cNG5I)E^=50$+{{qMn`PMBphEtM|LAGSZB{9tfyy+l10Ev+g80V( z3w5#~1Dq)Okk_;+80x0pN>wAe3|i+VQ+(k#>JjsNYhZKDxV1^KM&wD{i%51*|H{$s z0C%rL?IscyXb(!CVHn|Q(?&h%piFvN{Gh#s>hgtFnM8ocf2qYCfR}wt7OClT;237D zU0=j$vZyD``>C@>7i>T>oSSYnqb`^|zg!8n;Kue^P zQ<5ZjDsCH@XH|nI>SaA?Pj+b0(^AOAl20P6>_`C3G$Qj+wtJJWB%CM5$HuA$-~IUU z!+tpV+f<2?$UD+Kx$42y_!Gh`){nolS{?QDr5I`r4rx_#ZR30NDFSLu@q>b?f_v-D z(gkGLY|GjPt~h$VIXI0d@{n-|wHvfxC&g2-lrQ;OC~KPI44(<5$4vo?Kl?8j{BX z%QIyZO!ocrT{I}&8i8f-N*c7B8a)Vh`T}J8)c%vr0)4>i2tIye*-YG8Oiq+TU7DHw z{#leK^sSX%HoZ&B%UTFbUNtXpR8*}>d(>`wQhL9{rhdPzwrCvp28v&=`xD9xst=wd zFL8;Ed^hBhEd5vjUWxw=gM;nYzJhEtaM!q?3c)OV`YoqfAl0Wv7Jm~be-zD5E^lqN zi*D8*e5^4qsUA62>I;WFq|hozN|MTG;&w)qt3ASMk(ELpedlLlsrcK0=k6^4?cJG` zbg%cR_x1>lk`Q>!>xCM0TNHtss_F;%YV^FR*54T!1hz`V2Yfz?P>``_xdDqn82Bfp z6E__E{O|wKxaaA!Q?mW(1rgDct?JL&%sp6F@Q8aV4U45@1D7dHk)@=)*fdwy=}%ym z7Xe-b1|lEslbxum=7 znK_#j>@r)Hk=bHI?%Y~zdVlaElTAjr{WFA5>6@(<`5+imJ_iD1iVFI@jl59vX*Dd~ zJ^8AYuONK?L7_^bSA|aVD)MqctESH; z?_p}VjFP;kl1(PvxFij9ydMUloTRIh0tWEVI}0>p_b|@QGD1s`B2EMK8iBlxH&cJ# zU-O4F&b>69uWQ_^>-K8Bco)Fd?zg{om^NI<&(AL$zMm+T)3QGm^lR1lP(}*~U7UMn z{kj=LYQ4$RQyvtC1Kz^J^^T6UfkKu?Jm>=^V!?4U{{=r~eh(?}ggc^>u=LGu&gKy| z5OOjlPmhAs)cBWHJnRSP8YP~7(>}0)@Yvu|3`&{zp(yX;YY$CGUd;!GiO1Mc^BLkc z9LXxV>xs%7$7iOKkJenr(iUB){UxPe zdib6QYKqQ&_<>xQyoeM}O$jl9Vi`ap$8#qxFUj{W^UgB5Z-JQ&APsxVJ>;|W8>Z=L zHb<+gd8`i%4smvyo134SQULJNXaa@XgVsULg!WP7gJw1F$|tryy-P_*toGOY)#~Sb z$;0bRiIuR`vL|s?{yg=P@MD>dkQ30)6Etjlj1lv<-ceWc@jbnrh*n7kPo8$nZnOrU z6a&M3e1G)EvPB;M4XpnhwvY#8#|4XD8VHor2R+=dPF`uANzL9s{ewg_Ws+z}R>?j? zv4Ri0@~D}QZ|<6N=-~NKv)i+0Gnt9?2CQ)up%Mk8TI3TyEVQ(=W}SY#SuuN6-z+-1 ztJPTXOLCy8*o3k~CYQCe_Rr%#dO|)$T^0&@UyH2Y|B{oDsS0W=e-PdVT%HUJOyQ^B zeqUYwqUiElWNVp*NXQa==`%>N4$UfeA^74)4<|{M##B0@3GGAq$6gt1FZz@GMt&sc z8kjN6aPZyYwr|UT`k?IH)(>7enSOS`ZPD3(3MeKmIh*QLbft)5w3CW5O&t7Gnef_F zQD>q^lj);Kju{83}Y`ESN=-h0P-3{N^}_OzFSxaTyp@NCpDR4M?vf={2l z==ytoi@ao#$v=3v%KMz<0T2%#wWg4BCX~BoJdJOpd?W7hjd_KYG1*3|xNt)Y(nn%n z$dji&Sf_&0S-c81Ysf1{vp*E~OI2T=|(KQJ_@7g99J9R7YM0IQr9TwkB z$wX8gN`7hxT1OR)SrNRP4QOsFaNw|MWOFxQKO5H)~GuHe_%m?X-1PR)?6xcB7 z+g}*Ud1DhsbpHy?p#w|xe5Jo`8@itwfrf*8eYtfQ-bgFTEI5esax;tP>#7TAVO}wy z+61y&$o#1C&2s5}4$ee}EqYtVNN|RRT|HBJ8rGYudY==OJF(06ApERBs7_Q^&_|Xv z?GDg^!%6(fF6zUU^uuI(3Kc88!6w#JLp3mdSlkd8t@V!;;3}*g{Ge7sqN5ZP zuh-8`(IAOpVggaWW_2(|rF8M|jXzUNt1lQvB56ZLT;u@sUmNPfE_kBHgxaql#D0z- zxUdTE4>MgsRReiruP186=e@_j3*x0#^m{_1-=_;g(- z^mwiPn0~Ma0Pgcu_O~kfImUc!^?>9%ojYYQKOsY|InTvDdJ$BR`R(a{b z4<}z_!#!5ITz;s!R#hD?hCPV=127H^ZW$22uTvWtI*+SPbqUx}xtLg|{rZ}xaPkXc z^pj!|`fnH(z}gWOn2ml9F9)8Hm;CznDeTveO$|!(W{)6%D>-KYlxvlFzV;K~5~p2U zvL{!b=_iMV6iRPwh@C?NAwY)A z-3Iq5<01m8@nqZ}_BI>Lz1UpRRn9>*gY7Tsi}e;os+paeF(GLKx&sb2M^5E>Q6=gP z9Lv4+AdYkSIo=X}lAY2N=Gvnrnel=mDSa`Wa%u;jBm+Opru zy)4>Jre>z;%avfuBQ`OJs<27f9?SU0(-yGtPzSdx;b-@_e>YVSs85>$Z=%eEfyUI2 z^fbF3sU;f+W~b9DRH$_q(~)79COn*?O3Z~+568jDufJZpVhm}9a&a!D63LIu$A3`d zmP;Xx7%pJiVDMAwf_mVVqHQ~GDs!A-RXw;#Vn=VUyzQbk7+d2PIl9D)OKgHMhnm8 z$epSzXHTE|Tzd;XEy+u?^}GRLc9P4x0Dk;v*xHc#dAn%pBSMZcu7Ny}sQ!@wK5>D4 zi!I&)iDU`zLAz%O>rigFjIu-~GkeAI#kBHA<3eN@ z>fR-nV!9(3I%5n|F}szkTruJy4&UzR0nbVz^0nMNY8YM0=tg1F1@pTT6F+k3IS^j0 zOf_IS&Ly@Xl(Hy~rOVt>F4i+KQ39()gC>&@)|Aj<=Uq;uhgVF*+grvdFH-P)nel&9 z(nSM^>r*&uGYaoxni(9QjhRYJ*I%V21JzhjcA!oIwpSRf?O}gLK+civXh6#vP`&Q< zi+8Z}Sx6#!lnBjiSu`(;g7T8AXnC{`K_?7A2~fj={&8{JX9Qg z^7cE1(J5;F~nG(!iNy|R+5huxiL>Wql0g)b6`Zn$fJu0>K zkY-gbUK?y*xCf5a)|Jdgc34H?3V`O-BUlm6&4+lDl^^5EY@XzS>rcJ^QA*oJzrX?$ z^N1M)TzwoU0LqgPa>y-6q3RL0A!Ma|e44_7a}K5kEZTK`u<6`#Qs){0=*UuK)l_9Y z2Xq;1zZcV1WZcf^0*um_8Eh%v9z(mOF-%e)zF^Dv2X7NR4DNURtV13xuwa8=s<~HC zO83lj1iU%nWUerw2sBf&XWnRnpOZESOMvKhUFhK((zWujkYhGwfTaEGCQczfoH|F+ z2C4=Bj!`hDm{z!Torxa}w|cPY4` zUrX)7R1~4TmP?@>X16tYa{M}8Q{2P=99v!C`V+YB3`>C#Ei}#$7>JPy6cUmz`Ezys zrIrIpP|*)=6dr?U)g)7OZ2lrASx#Kl*UPim&9bmEo%Tb{e{A25?90Wy?a^Z-27n10 zBk$md^H5_#z?y@NOl=q+X*DT~e9Rj~jirXo8g&udgg%5^yq!l$C>Gf*I41gsaJyl z{FgfKsD1`qS6a4xkf>J-M@^|c%ydD{s*|5k5TmD%K0tAlKv@}}E2Ces0_s-npa1^) zb{gF2rm)@sBuHB27!0U!S!-6(P%J_hGDH^XXo`PH7m_iT%Oz82AJo;qyv8Nt8Lp@| zf9j>}0)uGM#l^U1I!vlToOqeHJZjQPK}4#0(;<9WIM+!BoJlq`k2NX(eF|Fuo5H4S z@Q0V1t(@qu@&G5!K|jnwNBbrZF}{B<8#AP(1#*5jC9DZI< z&6^gDI~EQ8nqmNwD`eAA4g=q9!Ple$0`lVTJ8^7Ch(~ak&fH)+8}T$`o9k)V#dZO2Q#9 zn40OJp>0dTaVZ8U_@Um?!w=iirlQKkLEWMD(pU5_9ib>t8F?v83Nav&Tk|n=!q#tdu@ElLbH?q7c_Yhvj-t(+?eRC zG+j_XH4OJppF)`!O&xU}*xqYbEKWCpgK2Mv=rG2s`gF!MO14-pNYjHZyh<=k$ZkAVg@uq(I(Fd(eyzIsQ9~ z^@QR0H{;DWA%*hMwaRZR5zrEs$W4#kr3GG9&9zS^ZygT?A_TaGDRPy=LjTjxLrSX8 z_$x@u7@qWX+P%|l$x|9Eiu^fBl^aJmr+iXs2f*m)Vm!9?Q1~?^W{9pk?F+5gR@S`~ zYX--O2f70h%)Q%$Tqv4N6SO^l;cJp)@&BXk&7+}y!@qHDC`2K$CPgJ%_PvyZkYryb z3EB63j6#&P>{}+g$iA=hkq~0CP7KCA_Q7C`G3L2_KHu~F{&~*tJm>t*?|qz*S?;;t z_kCa2^}1fy>+)<;d&bQD!t`_km*WRtcJ0Tp7jTbWdY`j?I#@IK`W|~sQ^D={yy2Q& z&PVOn(n~X&6a5q;ktGK~k++IZiRW0$09)GWE(x*>XN1|kqSR#2VKrV4O`C4=4aR?Z zQtgy#_qv3+{q5Co8+Jn(6G`o)BOliD@hqJw>Yt=={FvaRjz)-$c2Q!$5x1T&k6Ga)3j8LNJu0KJa zp}27ENqk0jr{<=bQ+203Ob(rT^4>qNd5>5zP}r#3`fqNfBog>IXRFBg1FQn zihdb=;_d0_q~&Aj@0LAV*pd3_%?B+&uQT_EY=1q+Kpo2_t31hUk75Q{c!Nu1Wxd^V zkKjm)u%8caxQ!SEJoSQqYVkh>v0BLK08k%Yrg)ab&7I^IW2prmF|u0JqU`>bGqUgI z&%K}^&fV?HqPR+ZaLAB$ULE{qk(W#L)g{b{-E`$b+wo91^%kD~y3oO;%V#Z_{AjB9 z!hZX|AB%r$@W(lm#ns9J)+K7KF{q}&@XPc*;1?#bYLD*9!$s1sr(RFByPcIU8@_c{ z*e%TNHgnwF%%z$^cOQ}IE`ctSoYy|@gt$&U@)mMQ4bT_0OXoGZ#RS5BNNVF1JI_lZ2V3s@fx3cE%&Nj!1v%Z^MLf(0+?}*}T)XXa1`jms%lhm+AwNF)2MGa`C!8E8-3* zL@Cd^G!`9a^;E2YwqGy$l>X_?ElKM3q40K6qT{(RrS#n|nrKt$4(~DZu|yYi=a7*_Tj_TY&wI~R2rQ#F6GRQlhr*9o_Vsvh+`|Jq=VrQ9;|%(m zIjvHGY^+Ro-r2JXj`jt=uE#)Gb~LA9>4+1LCK;3!ySp;tP^aV-n$cRa${2neVJjf(Fs z`+ohk?#o_#I&__>LOoB-;FAGY=06Ygqzr})80^!_j!C+DQre@TWL_~?6;X zTne$is<-f>^t85h_;s$&{-%{?A37c@lzx{8_nn{@zw*vHh-W3T5aa^$MhvtnJ!{(1ID2p_HJDq+&bWUjY7zoyN`Am znO#`auEyBr4n~LccfC+EIGrPd!406sZ=`7mc!?C?{TFOM8R^?tqp3-YqeoxT&C0MMiap_F) zE&H6jgENu0>`wv0^9wz|E@t?3NA<&xRP3!-zSHXXyPe{R|6F>PE0=s(7M!-sx}1K! z_q9&5GWjK2o5WiC$1ci-+x&W;ESwqpEHPZ%E~-7`ey40$4ZY_zrmd*N<@_MnDE4!! zrNOQc41S&Z#bEgdm>P?LUZNV}3KJ^=qR$fc%#Ggm$F?L=BILet)}^zNY;Up({j)C9 zM-M)^`ZV&vGk<=;EdalAA$Pd?wi_PovdXamzvx!v>{ULU!n=>2*f`}pxO{W4`}4gk zNw?#*k|vAS4Qvnv2Z_PFMe#o%7h=J6A;ayEL?&= zs^7+cv*G!;XRV@LS)+RCsaTisAgJ4E`tHEeeikI)_`m^`eZHG1mhW-aW!Z^;^dDz| z0Q482=2%PJ6MvlBap#ZZ=$2_0JohCdjV6=d16q+mNl?-5s!?hF{BE=PbCZwj_)Fn8 zqjkYjyTlEc^xFDTChcLW#z3$|l}}awu9l}_=Qfz6ZR)Op^mfA|;QuD6`Id3qdiL{j4ysrK7D{Kn6}W?xe_s zt*hurQ>mDpLG=|hFO-V*s`lf=bdHr}%+_uuvZ^)fhmjN;CpQT=r_ z!g?w6&wcF;+$qzmrk9Eyc0mvOdM{+Ex|V8C-Aa?Jn#Su=96TrQW%$GrYE9 zsYJA8v#K36#RWa=i-Taioe@a3R&FRqN~+>fa^N(SD5pLAgP9SyRL{oBi7x15?K3nM z^Mth{%6&fCw%o=`c7w#uS<}d zQz}7g*sITnZ4^od?+0CBMJ1|q(xP4!W{gp;X+1|rHI2SPS6|n7mjoASzz-JQj#Q;> zRr2(J4fHgG^7k0rxcf06MJ#!??|F5CrmpcNq69iCuhMZb?*g zGH*f6TbT^3?X^&rCO^wexl~RtsXo89)ma#&Hb_~?A(SKb%7Vciup*mUs8i=|N0vm$ zXSy$bE-F6k?-i}m;X-*U$w_VWI)?KD*kED{Kk>4(b1^hR+F|G^Y7a6-ZouJ`*yqFY z^7^P5h_e|<_E%#jd{^hT8!)MRb43}rUrv*fG764+kw_f$<e&tKh!}($)+<(h9>=OptO6=X_L{Y>RKL+wN!Tve;mv7q0fceku&q9PPpy2J|c;(oOj}X3O@$GCg9M=T)ou1KM zsCD?=zN8$o^*l1HK;CPO6`DTQup%KbU1e;2r|jYIZhi`+)n{)fy;Tp~=~cC{ZEJz* zo2kNQ-MLVep7k@_aQ=vkyPGmI*@9Vh3YiM^&T6zlgwd2Kk+y|t^{n;j5+ngq{)qNc zvpG8d_O(RDMDqM+I>3B(%vOXU4 zhN4nGji0e#jMX#n9xh)Smf(OZ!sz+D=Kg-pOB|V*Pw0*-ka4)2tinfijq7@v*o_~r1cq0aNFUJgFracW$Z}w>6Jc$P|Ec%=~9aO zs9h9~pF?jTq>M9&6zZ83d|i}uL-oU}6*$7oyN|3pDYvw;b6QF}(5ClqUd`{uD8os@ z0$%G%iwWNFVYbiipn|883#t!yH^_TiQ-|paLW+)0@@Oeg-fP2qT#6qna=|}FM?AXQ zAQ_$^g(v50++NA<4fZ_z*%lh!hVM+j?Rrx4c(uRnS%w5;`A zK}nIi!+6cO{_tF(NP1!$k;Fec5XI^NCZwAl}7_Ur61ALnJG-AOtY7kMY($>X<33+KcB^vrk5 z`)p=;gUVj6zF$t}eJU5It{pzQXZ6QBXqllt<;s;Zii=Hu{i@W=o%Dhj0j*a7k27>E zN~IOL8r<>H{^Q-3RM!w@d`%{HbIB^Lcmp@}@Z9j62BajNBrUfN8#s91&YAZtiLm63ne zaBe@lhUxWfh>qcgtRk30c_PT zW?5c%G&)ifeQadDM+%2nX2^N(^@@gYmz(UrpbGmmQ>uhO3bIGHzYlr2<=GsY*ED!HT-*qaE(6&BcKO<_(`F?C)aj{rM~E}V1I|8@JCsPE=@eR#%C+01sD zyB(%Zg{Bi?^g>$qRBx#WuHh_#zZZ+$e@2vd*XT)#?Cn#wmo@_dgRk#|@21`&k4gq* zDgGS37cO8Y3ES>xe7I4RSM;*y$UlPdsI(+K=hiYtbT|F$)vimNyiju+qNH|Q)BHPM zTb<)WJR{nBbI|c0gO5Gmk}hmF^gXrhyGpPTOTV4&lae%|s0ilETS!D%FaC6QESDyQ z0Bgy)<=93p#w;3rX{$o55fA(F>!(PcB&6HE*DP!p*=3EkML+f&+n!$x)O$Vt`ks+_ zw#}N4uvtCczf}lv;MF~b;goWQ1@KYl>g|$3;)0Z2D*|pvDa~)SlL0(FpXt6$Iv%&K zG?mLzDcJ($fOmfYZ==WzcMO(Mqt|f@EHuKPG31znnq+m&&77vB&i>8V3|!yg%heH$ zC@9j@66=z(XYlTM zkqJIw@}K%5qE}%(Z@DvMoG9cJyG_LGY+<4Is(a5$w|7@)G11}VFg-6yI~8bInvKc8m3go;u)Jno7Ud%`G1n6d?Bj13OlC4&kAzvE zBsgVENMux}+Iyb}PcRb+`orTp9Q0ww2SK429B#A_@lwqA``EKnGWOFR^P(!Xh$12F z;pRGL=gFFLs<(u5w|O&UZWQM6g*|^hT<>xyoU1qNNtp-PZ}kNb;QkS2j1F2l=9I{& zrB${urse~Dsx5mfm`i_Dq^_o%`vKzol?!;!^#MQCVi$i_cWm?v9maBc0Zy_ zioM@i!Mn63UAQYNz?2_;$M?naRLF}@fy7UG?HuTbffV_mLMg?jJAK{k&6-iA;<3s` z=EK-XM{pDdk_iw4P4=e(QE1kxkmgL}njyIRm0#So!MyDis^i+|JIctjoJRH^2Vc8P zm06c7lS|Acq($2FhHY`J;@vMLEr)Pb%-yljmYFXl;m0Vkqc%ds5UM&V&K-R+)8#v2UmNHw={|}t6q+eSMb_gx+A2d z6uu;kW?5Ji4j^b-p!A1^nh8xTe2H1(Z-B#~;JfybNmEA~f}cVGU8I)SV@1IGVPZZ1 z@Ri5v*IAjnlx6aaes0gJI#a^tbXNLcgP4jPx(nVIoMDlrM|scz$=LTwu?#ZxT@ToT zt6+6_WR@CJ%<9%&EcV8j)#zyUAMKJ+kEizU!+1Lmw4x-uJBn3b8F#BG&ouH$=ot&Ea{Fohh{*Tmtyk+9J zcUTvr5)`MnIWug=%S9NcaXs1^#h&l^mcqc@=&~2dkvyEe)|tEM_i~m~LQ5<174v1+ zHeBVi@zbib)<+9;)tI!sAM?KS_2b<9F0C>zWWv*N%XJZ5u(quYT})u>!}=`itc>lU z0~&aX=RpZCY^oabwcZl8`BC9<%klPVI+w+-9bT^HH`Bj!TFl$uto|uQZDrufYOMP9 z*@THRU=fISnRGL%kp6t*eM=7BICNheY6x2xr%UDwJ05Rs)hO{<@$`yBI77o!yP-#T^@g(B-p!VfG@0C%}%y<7rN^kL( zZqPIQd0V2+9ZF|0vo1 zH`@Pm`1+hY(BWp7Ao)MhZjQE+0Da6CsOqa%#*%L&)H)q>vTDg?I=enHGE z4?r5_#z)I5$Z^8FB1bcD#C~E@GP`!cMQ#O*531F9qPD!y&$dn020hjf7lv?es{%-Z z4$yqTFdqG`E6w0-!uphq%G*!$(J3DvjrR;kM$z#$tR+9BVLWT(clc2qVY+SeIuyh@ zGpqF)mrq7tyxnOBe=Y5xXg?7>Gkm4g)OTfKejr1AWAa#iUO*c-ibt*3x#^V0l8-@3 z0`P3XbFRI*p%HJBKX*B|=f(6$p=0wBZ8lq0V8G^YHk9*0Zc)+Jo}5J8QCp9JAqPBF z(qlI!*QMoV!xC$dNE>;R_yWy^XhVD}L%x4hrXp(70vwoxi$nRb(%o=%0BlMW_> zfk#r~G#KFZ0$+HWdK?!A<$yn!+-Npjqgf6k&m(EG1>so@&^?Z@KON`yn z1^n+ZU_RXZuQB=U9yc4u2F|y>r&A`jXg6P+?@~;Kr$jNmdc(oDyJ=RA@mu@CETCOd zzyv0d%Haf+c>Ce1JoL4R$w~-e+OmErFt=xutgZLLL-P(WMf`aDO*Ph1L6x9(&HgSN!v@do{RORHF#x67walGRj98bH_)`D@K}`jFJsk8qu= zvY+3*jSSivXhBWG5PI_kNn?`iF;U8XzNE4K&d$y!sJ>oJ{o$cLi1W~Qf1-8N=&DB} z5SYKnt4dqzGAz=-1r)@z(Lt1>7^YY3k3TP_4HvFsno%1*R)D*1W0Sx5J~3|Og*G39 zMnJ9rZT7Gy^)JvI^&M;3G|AKvd|m5rq!RK2JI|yl zH*QjUD{uB&>dDLBXYvX-*<#mEcG2p)6j;LyIg-xXLd9qIL&wT@;2+;OcbYc)r5*4& zz>TY_)}e+JKMr^-p!$Z=7965$V=`Umt+nNT?b`VezEt6BVQpru#ZZWpXx;-&*O5GH zRMsb9K$rh-1^#lA&6Jshl8$__tgKhehYI1H4nlW-Ls}g=%ec{&nZygAZGmj;ypI!l zsKut__v09oWQ(2c2wx24Px~1X2I}oo&!ZH_tMvTEi)A<2ha1<$Z#O!VE`ob6BPFcV zwQ0PweLdMGRJ7I}?b06TT58(1K4;k)Y)YDk1RvYiD_mShn^g+TM!%PQu@aMhwuA%N z%>RZ)8*s2)iPQ}u9#rNI3n&F*xMf-;7yZ?eA%UdOQz?+E=Cfl8I1^HbFXL2D#x?e= zz}b9^%(1l7pSMI@+ukIds6{Z|d3@Z2aoXE)ui0-le|+U?ny#`3C6_RD75>RJvdWRzUqaJ8~NIE_1a4VBOI&^e!a4>!R_|d+y)!Yrc@6~NGr@Ytj z`C8%h&B->YPiRhGe=5d49Y7%G<>fu0@hF=y1DGI(bdi;;5gis0ynaymWTVw!ZOq+$ zd$09;(L)#_c=P+!svGL^Oiccg+K`dPsP>m1yslv{w1KLMuU5^^9N59zuC>TmD#4EY z9@{RZcEekWJlVqj>^ldy9IsCDt6xNK473H2RJOJ{Sk$!P`ym)?#=VUf%|q{kMANn= z@j-|C2x7RNk-7iq0JlOcufm6ic55HOyoG7AMb58*ZC)EPnDph_2On9acV zp`pv~WSlVj5q%p0uO{`?&Ygn<*A{>M>Nc`GSda0SQC;pDL@`_q*!VSwqEveKm3Z3m zDg?Zkc@H~7tt{=s)hA{kau>6ZBv;+rt66G@9uv;c($p+#T3w~#^=ILGVQhS`KCHVj z-YTRdfuB|94P0m^83hxpoy^Urcr3L2TBqAuMPaj@DQ(6(<2}rR>9udReJhPM<_jx^@~+*6nIzjG{rt&IwU%%hehK_wch(zJTZau<#DK--j}5s?@pz$PmcQ)) z+B0{KZqfzCYGj$eVw&{+cbeRMH!s4@WPdYovxAl$B`0||CU^O|8r4=b`RRs?`Wci8NTxE-uXOC-l}-}*_C^4))0 zk6ygvN!i10x}v@_t8K>^VL?3Eio`ii)Qoq;DUUU2bB6sHx(N^38JzoZ1hq4tWRRIw z^w{0IBv->BKYx4Or}2=mLhc!|(N32y$i$ckQ?)y^+N^lnOu^)zscsTL##p z>q_JA);glBjwFx*Hw8r@`<*nn*5=(lR_f694UZW1fc{)vMY-kT_Rq@KfTJnF_K4x( zSWm)yx03sCX?-xMW1=L2K5(v5T~kwQ2ao6RpJ{UprzcrWiHi28?)9Telt~No4i~bo z+OEXw@&?ZJ-0Vxn_;h~})y*hgJ2bQDE)_Tj#D547`<_DGX7{L@#94lu&ZeB?qg{9aO1_ryf>> zH+Y|z;fkpC8w@!^BJhV3b&A#<{*x zNNFZvA9T5<1Q1z6vtSP2Z}fpXXso+kZhXABX`!HOxhJ1vr&+=YDd(7FILo;JQdfUUhn7en?L6k zkG$7%ZDTbgOAd>T$tY|)7((NxW!+}(=f#!@46L$T{ND@8W6*7S)Kq)AT&~(Sv9UK< z#EiHnqe2@Z=Go;H_UDgdL}5X`updP!><`)%={Q)-`kD@iV=aX|VTT>eHg{>Ko69ZV zNf=l37+H4L)qcjhhjukAw6nd)sT^XHYTKW|$a8PE(A%+C6dSKaqQ_%Eq$Ow^(zg%uP^pi*jjEz0Gcud?ms`S{#tBjC|o zK|x7$LP_xW^a)bD)+N=nw3siImB8)&e8YY?oIwT?kRjYP*5E!ri;5}6Su5t2l(mHy z*jIPP4;Ja6HkG)1Xu2A_GkgSfxE=x^dSexcAwt0LBXc;U9PU9Eew7=R-lw&YI!M<| z2}(=;-0nU(`Y@uYx3@|+i}nh89bs_ZXOQ0dUK)EiI4HM9F=VC|L!o-uR5~~*cH3$g z`@~T6+{y9j0Rem}rDlPPNud&e4MiVL!vlR3e*+YM_LU$^y|GgS;BL8d`67b|Y^UU$B!C{NK7E*j7U9L4BR~RAkklGdMnalxNa&=g*;SaKm3H-f?;%v z<>Cdb%+n2>k~=Cf6eA~=OkeC=G!8Y)oRuLSSSH+MdoP2=(!FyUar~IpDjJ@4Fj-{- zFE?vmLXSe?f|tgB@c`pD(ny!1@qFxKf=176Ac(`j6V22s?|-jaFL&qt)vIYG2up(? zId}KUb{ej3zOd7;zHVBRgLUl8i9A=s#nbu$B%DLbcWMREIxI>4!yl&;;PYedJzaqL zYsCS*9{Ex^>r*7>)s4j!Qv{Yd(9qJ(sNqJ$?c`u z1JlK0>F2~1wIj{~F)rmA{f^@(0$Jq9{Yh~ZR2@LESRT{sGKXRae`r)PJJeJ5@M z-@s>WdVWYbh{VWlZ2$hKr6ueqMZTq=&@R*R2@H1pw(oLiT&Z-4Qdca~o!`im8!NLk zo23o5hK6HmUz%%N(EC_8tZ60L3{`IC3y!OSzki#QimGev=fjaH&?dL-?skMv5rK{` zG1Fu9hVY%c^k+}U+~cHybHsH0ZD`0`Uk4}d2D+T`Asgk@w49UG8o=1J>JT-`F`X;t zD5dy5^aqhazj?U0SJzq%?9VSz5^a!k=loq;4ks|dxIqK$=dX8Nl{6e2CYzszeC+@#eCFB_Qj1}^(M!h5otcE zhXj*t$hxRFE@kp!TFUN<+2Rk1D8fUo-RJZ!;>^L!Fp3s&fiIJXv3mzN9B}bKo&3e& zLhYX3-T>d@kb{G>ap)=BkR@v`-S7iBu7NmZlPb^deGH7aw`a5@LS9esRWUYpKWCCG z#G`kg9u%%=j| z@96K}h*Tl{KClx=X5_!JN>-&EeCiERkH4YEB4snNi^esrq2R$p+dUMW%E8OyxwsIS z(p~`eNawwzaWHpMd&**AQ7aUcdaWwgJ$EdnW z=K^)dw)G{>Ga%SoMV=eHaa&GmZ~1e_qOOW*%kn{`_3hi=B82FJVY|zp6Kf>HnEE^x z_tq`TQ3i%rmfko%K<(w^s77?iyo_sn?=+?tleBo2}xz!2{is+TK1-o2?Xh9R0+}vK)yc_^x{mc35{o560t_ zf{*>TDCkLrFO_m~5+c%qYa{aS-@mskXX9INlVy0^@$L+zRbkwuaH5{R zVZ923WFQ@hh~-iqsmHvPS1C%@XIUKI*4^WE8@FLS)C&th-0_h8qKM>G^jh8B4QM?_ zB;Nk^t?F({a-yxRblYMbnhkdFk$J631hp^e+2S(f~FlbL9JTWmD+PID|heIk@Ln~_cUwMpIK50h9G_oIgH`SabM!`yb4XRVANwM2q;WDW}}8@IBbnB2|$ zULbrFi6zE<5ji}_WmMupf&ykkV4bhZeEcp%!zgm+uCW*;NROqux{4=gxgEUnecGEh z1dba6Q-Ju({-(t~5i6<9w!{`DS7$H!G{9=ev-4(Waq;G4+n0iszX!5ei%hEBaol@E zC>vpQ53I&AihU_+jRakZ&-roZddtCF76yvnYY(OA)9hPn-A=zo+rEg3Whb@=;y|}u z6%?vkGFA#nlPGZ{Y#N%~FkAl}I4=dq5Vlgg6N6Wpcvi{9wARqQ zKUN^_fP8}E2qbLladF4&`eZG-IH?cUhOe_$kA0Kwv+(Ql+>e}v#G*1w+2Hp98LB}X zeQF*4M#1YBY5J;~eKr$q{HG?Q6zW~^Br;E&AQSD=K&zcIj7mc_Ce_Wj*5C;`eI4A? zrQXS_!;I@_&ehS(_3wn~0R96%%ye*BYq3XlmuV4^r(=DXkxeHEruxW-fl4;_#413c zKZME=a`X8s>t7sLMSUag)E3Y^X~k$7cojcXunIz~Vun3^26q%vfj zxy08wny1{Se(8PG=mBJP{eud}2xQ=Vt+1IiV#oA#+v|UgeRm9848OS2I6f|r&}fI# z+H9qq${qAO{E?n0Jv)vS&`tLnWj&K2Y3~6{Y^?vmgQq$5!tXjZQ{}KpD9bcbyMn8k z^{cXmxnlHY10CP)uhec(yI+a9sgxn*sO&z(64kl$`=GMCZQD@RI&kOPwy5?O`B%S2 ztL_%sy>64VPFCu2sh()^#=%e<^^-vua{Uwx3!=Boy`4Uu^1we|OK8k}8huRBjb<** z{DOkm>1G!YT1=C8{^GIl_CX_DBLN2Q4kU2b_e4(C*yB;tsdo3$LNe%E)^~var4wLf zD$BU<1t~Sx1Gc8nC~oJbYb)s(7@ZeHn8D6Jp>e;nq-J-7 z+u7Jm1tKk6j7*%&CyO{;TWWL>VmyD&-59ZFT^%+S;TUK>?7$y26bgbR1!~L&Hb+VE zS3M#ATR;`Pb*UJbBPp}H56ihhL6h9!VYyMMA7#py$8UKyJuxi-q|+y=0wGD|KF+9? z^eEEAc?!=dDh8@DMCHalmu%LjRAb%u>W!0`>yckfkfP}YJyl%msSVbox`<5#0i|vB zb)xzEKs2^!24wLbQ8{ zv~!m-IkU&U@^}W&$JI6*jh3(%4!kHKYAl%Q@{~1i9+?+sA}yvt5lf>5b_x z1bNlve73CS?PP^EbMkzt?=)$ME`dpvMU7jrf|4U*0SUG{4q{X}JVt}ozHO$JcSQy5 z#!)K2{pY_3XGmuS{-$~xaGEXTYn5OY2h#n=2R@@X4)@abmwo}Kqi>&R=)dYHU!$>5 z8hIo6DmvmKFOgJ$0YJp{zURnkz=CjTAW9n;ti4fJsdpL$E;sRNeere>xd`Cm zQgz+1XQlzR4`9lwZ%DjmhZh^7t=OwOhXiv%B>}SU`(W3K%C2+8YapGZ38>b9QT#rB z?iCs#@zVJ282HI3t@cjJz#cku#v7DSAJZrDlJ-P$0T(JK1EokQz1FNRC_oohwol(*970NJGk5lS9-RZ>xBCVorl-o>+$rBqiRymPTaFW) zZl+~0c!m{(GcDe`ZWbs7;OizUY}%Um991V2K7~%ak&I;z%6%!8sT}AlEXy@uC#cIw z`N{w4+RJ-sJe-8+0{da`u!ZePMPyvlcCta=Nu10P_KXk}*bnocXeW~{Urw;h+>7j( zZD}CfL|J&gpPFsJWs57+Q`~VIGCG;}$h#KwSCu`m`_4v?F+uU$-yoDekRw7W{X0r4 zgSF0c3Xfi%p;8APEl5jQRH(}=M8euXUQ)H6_)hGGe3zdW2~k;Ow>B^cv>dUW>T;$I z&9)>MZvxB5BkxBQ{_xzTvmM3Zr~W4oGhgYhOB2CY{QT8(58{(Yqs=zEwL;0lfcHe{ zN5yc+oKF%SnFXgO8_WM>u^z)=XGpu8ew&Mm+auKy1)$(^8eNx~dWVxTLj3oE|DvXE z@n#V^Y(Z7)B_Gxg4>Me&e!)&r&7Rec2h)6V@r}9`Vgv46{r8u**Oqt9F{|QM|NaK# z2XJ5i%YB`YNdF%Yw*8g{WxUNA8ztKRw>~)7+^mJjlf(WwX_^Zo_Zy&uj(^V}8436a!oHl0OkX zUdYRX1&MNtA5G|{+%3f+&Yyf{Lrtx|uGpZ>AuVc%)SCqWC&9Y*n1#gBo?3liqWZHH zcH8Q2@XQRDGQYJL6d)Ut0n7qaxxI01^Ptf(=L}w6{I5XX--Roy=}&}- zM^QEq5sKe-$d}+FWgxf2sJxSUqmPy^+$Q-h`PGgmvTb*cF1=vNA4&f9?Heb|k}y1h z^Th9DWgMMWU9QfDfJ$(3M!DhoblYOrn!3v2fidXPb*0S?3E7yayB$2yW`Uk+CX66? zPcl|T`Qz2oZt~oYsM1uz*t##qq9&Nq{zUyO)tiGKiDC^?fFj**n}4eq^nO9tyEqKC zjPG+RXlZHzQD~858wKxmH=>PQ>UY>dMK&;42DY};-R`KEhC0wf=I1tZdrFhf3e-Jr z)3%xz=ZucyQIG~9Xvnp{)~~u0cV>#q4YjJ)JsR1=#WS?&JED2KrwSm!#fh1~%OC$< zw^JxCZRW0_VL>@rtZzMs$}u+y(}A4Y!z(M_v@@iYeb!%})Ne~dtyXs@vxAz2L^%Jt z&LmAW`D~)GLyi#_D@{w~8XLu>GX~$#mJR2*=dLz0{EEC6xLJZY+R4D+y4m@|PCvhP z0M@F*R{J)yAd8y)XqkQ|1KtDw*h1zg1J{O|F44tVD$!hAmRXMMD`7N7?ahds3u#UVv9#dEzrStu^_Iit`c+0{H{68xT1vUG6t2`aCAjhtYA^Dk z%p22D{a;Dt5>|8gZrx1t!*TVG=g+&;H(g2!H03)Rui0QYT zYO(46%-@{_eU&r#W_`3IP)Z0lQCo7<1{ey>T0_EuixV65>iY77)ZREM^ABJ>f*mj( zqQ={%fQyNa-VKeo$aaOLAiuz?ttesKDV?ydK*0C{b?x9%l6~tSsAac#{Tj^3##B8* z4R4D$uAANZoC~DEK|jyT%$lI?0ieH)Zg!=p!=aCdy(}#slq3raZ5BCajDf9^cJw0( zbm)6m@U2H5@_S%_xX|%_UhwINl=DvBN9hvpwJ~Q&ZNVEq`NIUX)8w6gNo}C(IlUGa zC2Wyta)Fx+sJMwnPt*=;$(8`;3WMsafR_cN4Z8RB!5P0S*O6NQSW>EY8Mh4j z@u#3ac;0_uVF7x;9rx))bF1LQlbjfNd;maD1)qM&QjcpX3r4i%x;E9?pksZf0x28y zQ}^8N2PfNL{Jb~g9LKyj$MCf>OHv(ugcr^f~E&qT+$2a5j;WbV0Q{_tN}FUSJh;RE=7w35jgBL!v0pJ)edk8 z$tfmNn-3N|#G)`z$LY0l*>#_=KXzKmD$w0qs=r=U-u1(M5jLYVK>tXUw<)_$Bx+bR zdtF5G(trb7 zq`{+L6z#k-C+w4)kxFYpfSKE2+Co8g%w3Jr%@h1}$^O2N&wf8$Y3=mh$l3vqc6Ruw zw{&``|4~q8Ci4pm+sFfxPt;FQHB{W+xB*@WT-Wk$6mhF7MmP?)$$6b!?TjF41Ox9vzjkRtHuHn z{x2S8iB|^|>O28d*5BHa)Y}oEI`hF)EXf@Wa zp!Q~?TXN;Y=MwxMk3u^ZuxyINkpqmh(*%X!k{(;eJ2o;hveSL;+&S#?kqM=pHm*`1 z^u^WtD&ij$Qd-YYU8fx)uY!aM1GjkZ{NClA;NT*F{A00@;Gp?hxsv2ymwV|vP`{^P zrxg|Q^KE>mJ%iy|qC}Uy;`8(H0Q;`*kYMy%ru2JzogbOAj=}zih(OYkydD0uu1MHg zs5rd#vP}3Q>%Ox|;#o)4BN;JJDI)PB4hgB)jS{`28U>7RluaKNpi#igcPkP9g0Omh zf^3)XY#~DUQjckxP4R9IE@?Zuepwyp-ZqtCypd;Ev_(NHFCq>BKrYJDw8K|h`JgK( zkS;b7F*}_7!MMMHMtF>(lIj?|^9DV=vZ06LLlL||Ira%%tZdS$x=W28j$MII)YfpFA(bk4hXj#KiS>UGV9g(sc=iT6Yj@L*0K?-1#ucKV+gYE~fw+0d~{t*_QY;DbpJ0uqxTz(vXbxy@guX(=0vM2<=uxzeujp@ zq|)IPzvXwN_=Nbwy;iTETaGFgRV;ipzL?`H{TY{3?@0_3HqjAG6NlY(7R^4GAneBH zi;WI~)m3Fzp`-kaN}n<_N6K;eDGcltHfx4qf8c?~tAn{_pkW|;IOw_J3Uo0+Bf2}z zKy5rBA>lIfo1>WPlqwxxqG+w_wQ8sMdIU_d+0j*Ad+*v)A%9Hza2a{Wt_A2 zcA{c9WgLJuQ&m1ZOZiMd5r0fpF5UZ_Z%KVBU_V-PzGqlU@E6IUnB5%lCu|pLg)?#n z5N*x?0|#)j{lvtD(Go=I|DH3fM>7@y%ou-JjMrNbHbTH$>3BRJI#K$s5t9)uuP7?A zK-aYwii_9PjEqbL8@45Izu#XcmRO*r7j@C~j?IAt9#Hu9NDoL_6gz}Hi@yu{iR}Pj z{&0prj)7ZwCs$WF#`q`S?(+0TJ{#m#aC!dmZ`eUFP?A@}FoFT&L z<&BII9r*o((ZcP9FcCWbE|dlmXd|tQXjlemU^fTDj$TX|OjpU*aYM{=5B?%V^z zM6954sJK7q1>)L5+~-A*nw~Sh+ah7%tvC>_PSs-`(Jn469asChmS-ser+Gg6VZ>%5 z#j0@F9jKfcn5&F@evBRbq8EDfIx7jkX_i0dZ}i;Mq1y1nq^&}$%Yq{Lwc+ywS8(FL z0Tp*ebWO3BV!fE`v{8F6BRYGKy3qYb4NuD;a!+@6gT{!VQab;IH{t>Lc_-MOcGap}e*+TuG<{!>3!D<1FnBK3oy<`>T)+WGqbU1xE zyxO)NeMwcVv(vU(t|70Y=vISR?>KNXS>^e!u(F~z03q8}JL4{SH+Sa88VJYI3cJ#@ z1>(9n74PQ`F8c&tzwEXLe4#^hmo@~tuTgQ6QS}Y*SY6w;!=+2j_`(kN8yDiW0)iW% zHc~2B+l1T7$|5qNqE9aw*(@4CbYWH#C7!l>o1+qd+B;@)Jg7?hUwcraiDChD4hD6j zhYafpCUn@r`rZ}nmku6%^CO7O6TR)L*Hz+_(~>ebWS%C<`tRhGt+jW@-{AAI^tpD5 z>d}>5D_KX7V+3CcKARVm`}_Kky}g@TlrOTm`pweLy&aEfo&WVsDWq>Ca6>?>veQR> zEsUNz6oq|A>#xXDJ-Y$}ORG`kP@e53b-6q=!#Y5eN30)m1D6BWJuF?G2m zjp4gJ-@o7&&GLKkWg8Sk5ZZVBI}4FW>iND9jS(CC=8sX zh@u{zmsD@wya@sv8yg$i>(&jaYdgaRXFzQ|0$tk(Nq4M=sF9^>|Da4dM>V0)u^3Iy8r=>pw-)reKaNao{ zNiZ{V%R|9nD(40B@@_yTa|0j6{+QoW#;Ur}@qO z>|x-P2?CA8_`?|=0@Ap<#}Cg0a}gtOZWrg$-4qp75tKn~Z2zj_QhaS-!1){ST^>>D zHH+|@;cM!p#(Sg1^JYGHw|0l*8No>$L0#S$ z%%@gEOTpz#m*XFu`Vi>1x%kig;Rq057Z;`r;P zwE;A)aqVLCcI=nt@VFLOhv*tGnX26OyIdBxHWa+Q1CPtO^|n56bp7wCyU16w-p7h- zAw2NW(doO=4-c}1l6;)?YO;K>z)$c+k{L@)n#<i2CO>o-21-Z&=K;*MP;=AOF>cfi&) zUEI1&EoU}li_I(zUiWz1D0|ap&e{>Xr zb`7imc)+)B6m972-eFe2>fkJYjysD0)%}O(Kr3eeAKa?)=|{?7?xoNF9T{0@3koFf z4wth+_WTqDBx(+U@)d{%PJ#v&Irq8OrW4=ND)fh+1K8YT+aRu+G+K-+u^tMO26^@;T z3(j?j(efGSUtOX{{4eU>0xHV2-5W>UDgp{B0-}H*iZs%aI*Lk5H$#fFbb~=CNK2QJ z(hZUWA}!rHQqtWy-!*La8|Q!i@3+o*&pK;;thM)mGxI$6bKlo}_3w92K;I6dTXal} z2I1;S9C9IG{ez!Jfv->Pv#1`Z+_M|9T57D0GKpL9+z>mmL zd(K@ls%T@;s^uo3unpR!*lwYpyYqMizw@gXL-mp2e4jhN&9VwoIS0hX?g6c*m*5?o zs?+C>J8!Y$qz3j_cmYiL->@ z4v|pn-Tg_zU12*^*I7u&|MOl-jsuC0EOQOFp|!0o^D?g7Yb(5K)Zq#_+TW(oH?enk zp#||z6II!WnMPmB^Ys$ac{p81p-`D=TJs}%IgGZ^s4&(^CjQRe;Zb-v9~mDv5w5)e zrB{u`k_e>MRaMEaU2|Fzq4$r0#Skl~s;bJ%%hQR#aSbY=^NDxh zIPp!#`NS{xZ#+{}RHTC=j8v7Cm6iIo@7=o>z`$wJ?uC^l`1$|gC^(jW?EAlG5`O!~ zO!NN+7eVwuF*<9uXT6EbxHpV-;H&m}!@Sw@(Da-a<#jdXXT(PV*wn7COJ!Tq+p`)D zBL{>+cr1|O+m9`)-zc$O*-Tg-%$;akd9pcZyllPLiE=7_UF`-zn=pBNq`>UdT0p4{ z1>0IFI8(1*aaxS~oZ2I-a9o*TtyI2x^rdR|jUV{T@bk`=%B8@1zevNEJSmKI)n4E- zT_vxewEtC%|S;j;sS*)l20iX2Rt zRx@uKJ=mw3sFzl)a6BjJwP{qpxFxVV`S9kG^XXsN#$^PBXH%!?1g|(Bv<~63QR99F zCU@v5w~x3P>aZF@N^aA^JJB+)*H3q|VTbmUpwxPn)2UVdZO5}py{=?o)gqJL zN|$Sq$z~<`-K;2+kF@kZWH#sR8eO(WTG=FIWfAF$E&ddOGV1DicFRW{30`u_7At`;WC_vnR==|wB`DX*y*64ODw2z^dMf(6CC;9 zJQl5=9M*4>O?Zog)(Z@cC3iMqkNELiPEk=RJYjAqyLf~$VScYQ#%&4h_Qfj9uiigS zLQD*Z>%SsYRhv2z>T9lt3@xfu?0Hoy`BuX8pT-w@j4Sd`ernXoyu+7A6V6dXKp$PO z;VgXz(Uo_W?X=juK8}GTv522;bKIK1^tR6!Va>$q#yr2f_Do7FdGN=NFAsJ^{U#V5 z)k6C(n^$+LE`RUovItSa34K()6u+N{kfN=eVt;bOLpA9dMj1L-ytU$ z^PBVIH^&otG0bb#U-*{CY}j&>OX-n7+Nb48wW0kW%4!M!WH=S`kqkjE_ain2qQl=c zoK|R7AAJ4R>#aXNzE6dwxuuv6?HKe{O>$i>h9gjiZ5 zW*H_^CgH=&$56=!8DCqv9+}2YKyhB?^x;}_ou*(}m~czOQ~sAfUpmM~T=d3!B>ZQU z=w&Z7J2+3SNoxh3e3#R`%}~BA$4CgkYPGh zcr;U!l$Lh1a>yq`JXC0Wx%JDRivDFaU1V|$Z(yNOPjQ9Q_B+onZ?+Z%15~uXq+_jw z=0U`rGZ(0YO2LALonN>X_~rgaO=;E>KCFX)^QiEe`PcpS&k5e~&xotTAxZVE>a~k2 z=Fne-0|^aPW5GEBvt zTenst&Pf-TjnMp%q2RaG8SEa|-=Zi`mgqyXjeA~ue8ZiF@(KdtD2-01_H2}k+UN#Q zlcmZTnC_V7SKklr8^hjqQI!^Zj?}Wd2?W{J)R#z`vr*FMaI* z!|NLJv)%WdcYOc**A#3qX14X+15z>foaUbY2+y}F>L0mh{ENxtd$@=9E1J&fmuuVg z1KS5=j<4uH?>Gu=^}pROowxdWacSEH+2?|_xz8^i=Uc;XwJl+RY%6GcCnp2Db35uE z85HJp&k-a%_HA0a1mx4RUHN5L+~19;5C(-b>)B2!5iK$DLz4_i zvDt{x@K*~Dyek(d_&6QbCiR;`UF1(C%VD1dFdw7$n)~|fHfB0h)Pd3kXCIZT4+>Zr8xG@BY$-W7$9&Gz2tE}cf8;v zmt0d+Qya=uD=sqZs_TxIXQ=nR25BNwVPTX?{_|ODKHdGz`97!3IfeY^EpV01#UaQb z4I$paM(X||Q&Z5ADwndiwt1e~sI@?65yq7mm{keo9-{ z2&XQm^~`mpNT;lyk`8C1=HugY5VyA8SQ;toQkO~ZVbLfpfQk%E%!`4%DY%R8qCb`J z*5>AU0)p$@Cdw5sseRd)A#>DKlrStIt7dsPuSGlxg(A{{u|Q;^nWrZTv)>whFSuMh zsKghaI!mLhH(iO<%?&eHXuQ2Jus+w5K6{V|awQ2zJk5fY9x1g+SIivD(XIjF%?jsT zcKodL-FB)ZA!F1A({jOgnCQj^|N0&f@baPWEbD8B-Q>>tq* zT|hv4>vdW;ohdtyLVqe7F07UowlTmToJ)HZd_#^X*#@GJJbuqoAPRV1F6awQfi!5zb~Y{++cF>bO_q6B1B6 z<36FxYRM5-f3-w`XqCd$nGfV8$)!lau`QfgItMpkndSxy=T|4{QzRntU^*|;N#Rmo zfXb0O*&>CeQL(;^Ax+c|Jz+CuYgRbIf{FN$P;kC@ zW@2LED8vMEKhhM!m?9aa2vc>H<9P-=m2{j?ic}0NNStbB>r&?{|9A4Z9WXm5PMuNz z`tq|{u^G~Nwf1NATf3#t!mIxnW#kRsP}2e=^yYY!X+UQgiD>JS)p2PznZ3yPJXIpeYy>5k-MX>$4-Ik9wXq;`SGfR;n-wj za4O;JuWp!1#q>zj{1g!9{BW!2XzGcz+d z`#Bq#JJC6)x0{(Y4w;&oUcPkcco8G+>Ut85gQQ+zRAhoh)zj0nx8H%m2nYzkbB<=K zPYl>fNPI{Y$?mJKtxaTa@hmfWYQ>=&s1!S;7EZlz}`@}LtJc*rsOeG_ooz0mWT)n2*}Z@esl5WW0S!G zQCw|pZHw(CjSw*|wj>cgpt_*P*nIeF$5*ILg>spXmcy1J zWl?9_QZ{S<^jhL}xyxMR_J&S`Q{ML`7x(fhdGg$Da0^Ghv4anhcY>UA_L9k+OT+9O-&89<>XBgN@3Rw z#dNr3%#m+6`&%L!LBj_J%a7dPoCOhYV$so9Sgy&wbEDD>*GLh({QQJiQyf4IkE)7t zJp_$eQBeVx-hIsvvj*oq1vcu{eGZ4)u5E5+!UREf!8dHB$Q;cQ_!K5b$K0INeP=2R z8Od$xc(AkTdH$NCg98|?tQQ^eb*>~}i>~uozmS*z=;PxcsX0~ta_rmT z{x*;KsL27Nu-Wiem=rJ~P0Fuwn~RC!m2-8Xc+4;0;Tge{sk6lK+nH@SKY3*rSm}CL z3f2{bSaQ^ejSPW6sB37*Axq>_WkH$U9chN!IIT^-UmYF+DS8@6Lnj^Uyf!JiyEf%P zG}Ro2vRxeXBG6*ngCq;s9NkiPl+(5uKYtmheoTrl8OKAut~W&SFnG|!GBPr!G4&lC z9gU6jf;H!UjF>)ct9$tHp>(W(q56-ZA*UbVcV$yKozVU#aPIqoYT^Kk0H7f-jiWn1 z;-LiMx;j|z0+R4p4E4hJXrKPrW-G;T^R{{eAU0q!-T(|(g zKr~PWo(`x`S!9XnP!VYFI8fbSGC0XwV0$huEJSh|>O`x7u7cGeVUTaIm(_%kfXR6D z=uzVIW*s~&EC(?{z;WYgdJnmDq@aKRXu7-3J1cls7&gE_gRZWynr?MJcNxrXM0~xx z<3Y+ccmKcuxN%0aok>zsQt+4;@bRIETM?vK4uzv-c>Hs{Hn542RCd4~Ha+Orv**d> z(-oQDKeC)@r%siR96u)xG4aE%rKJ!j!!snSX4-;lggV>;Z)&L2b`c8WZp)(;pwXg2 zq0bc_81W9$Z6B_3B!l?@Ed+B3&+oVvmV0~o_4RAlqMB5ipcrt9>RJhrPWz_^qobp+ zGqKjp#3Tc9V~s(#U{8iSlP&Q(I6AVZ7QCo_a~c2wz7n??k(CF)jT{pb<7<^JpPFYe z@wlt`x-<0C*VMpj#Kv-L!uGo5cCLmL=}hfD1Cn$9{{7YQS}@$aFj?UF@qvpAazF*K zq?5}Q=9d?hYff=-oUog#o?c>lx^WDj&EEEM+%Hw|aMWP&;V?B>HR-wx2Fh9|c_o%8fkkr;dSv60BpcZvAa! zbH9fZV2;571I@s}oorvEXW81?3i=ojsGPDFcP0-wW=ceZZzuDgqMrn-!^Xzu&bZ)Us*Ot4mX>0PIKruuruKN z4()7$O$%m)^o5gdajiTzi~Eb=uXhuoPvA7XgnmG*Hw_M$W{IV~oUXch^y+vFQM^P1 zd!1Kp6eaG}OP6%6r>82WTTM0TtImV5`u5}S^?0q1ap557;;T+nf{q(A;Ks-DTF?-R z*3&(T9Kp7i=$#T47N(+#1Fg&i>-Eh8AEpcqh7Tlu($WFM#)AmtTuMSBnoau~3^Z2A zrBtWZ8@B&QrHdM_4CpR=QkFkXp9b^}1k5erG;iL%MOXGbgAGC>;(f5z%vz@b!@NaL zPruoE5(Kh8M;k0`bAkVb10KV2Gsw)|coNh+g-^oN4L)6VH8Lzdqf5(zI=l=Za>-u$f1Cf08YGtvGEW%4dC8^3N!*u%^ty2 zt4VvQak#|F0CbGy*v7W8`Ir!J$xVJ4a9HHwK zAyT<-3BrzeH@Vz@n!N@*qVG>Ey|ni>X6uZEE+t~;u4*O$m*9;RHc9p~;rk~8c>P=O zC!&S@ON-=~pXQayAIBm8a~I-eZTRQ=d*E7y{TcHOXy6Lr4`csz=X(H2K*-nMVcybl zev8eg3f}p9)Q5XaNAQE@fAh^e7Pc3ho~OK!^VXhBL%MUZ!i1YFicMw8sbFj+21T3^ z-~4sm(&P&tzwZ<>9keTVop{WEe~rL1Je;ghHAb`+`vl9>^CxROn(vab)@xRhhv_^0 zuzl#|KbU9gC7Wt#VNv3^=on^JK|`Y%sm2SAEd&=VEK(_5`uc(J<@*eIci76ElZ4#v zDyHxp*ihe|^Sy0s8(9IqnDxTWIY!3x;QFJT`5$qcs0`A$fE?|vUv?}R9WUTuv=101 zhLAHg^_EoFC5S7@z$G|C-5Trv{(TP~wdNDUqf)Y{v6Pw<#(h2J{fUnZsZfsRlO^LW z#sz4Z@YwRJ^N2y)Lw>tOnLHB@@x3`&o#!dyL*zf9jeYpO}~phvdL3|3`{{c-BjQthT!(5v-J5E9jEe zRJGU^%kS1XX~=q0liO&g5b;^L(sJ|F1K~us&CfMY1_lR7SQR-1XvXU;gwbCe#Bnc@ z@5Q^+n@b-!q&+T7GgT`V*zVi)(@Bu z6U(8wX~2s5QLF%EohYIIFyb8@HMKcuz{igt!^6My&HDzwfB&8#q=^%okkq^B@O=4( zXeMS|zCbn!qtrah6N6d}Xr`sAJe-*chgJhT>gPw?nMYxpuq<=krm%r))gBZ#htkKy zGIN^uPVQHtXRS*^O$X18To}zWEt4+EV^qq1@8h4XkiR!P!12K+Qv+o&zZD}`W+w z=LKi7*;7#eAY7lHDXXZ6(q3I*#%4W6)soM8e){PiV8BpvfTXMRkNxsynf4>2?)KOa z%5}nXb*?fF3AD`>&>r2qytJh&wVo{LP6U}=bhcFc;?MnU3<=^7%@P{-JF1B zd{RumeEISfJP(Q6`_ge(J=35ZK}p6CQk*S6Dzsiib@>{jdrn@Stg>=Bngk4iFOrv! z5085E(Gz@X9XJS-D7`O3qvCm{VdKh57~-9Uu%Dmrd%>gWM&=<~lYyd#w}Wlw`n3LIPd&#wlbn_w3e#eAUl^LAX{B8k`4`5<2R{W~u8u)2l^5jfi85ibVfRT4ar zQK1EexfijHGBdJt$40(8srfZ{Cg@75zCoL73<_V`3q%tpqnPEtd?KP@VjAYXx;O4Y z6#r6tnb=7h;+YR0;O6v$6xXjuM@LVtNgrVi`Oo03M$~MHeXb6ac!1}|EtVJo)%sr3 z6`P(&n=JHK`Xjq7DrD&K?7F1%7sZIT8QI$(>VPtS{u&(YgO(0oNZca{4ZtiDcEL&IJ2%X!B zp~l8sC4Vz~Qr4?njtlRFrAI3%u2WFR6^c!zDKOwu9~{n^pi7MFehm0i3SS^7AD^4s zgX>RttBwehZt|D_;g9XgdVBj&wOyjDAvEE55DAvr4CGq%b?DYRZ7j{lGWJzJ^tycM zvJ%r_gVX(rR(gV)i~0Ha)LBB8uFo+jWK6lk zT7gN2N#9ZhvKup4T`fd2o|Nt*o?_0I_ytK5$ji(7MLrFhB;bAKVBRv zx`-e{TYgeBb|;cewFVlKwpd}*6>EMs*S32cZj6cr?}*~>A00&TWTY$P*MYp0Z7nb> zw;S&U)xt_$L}aK}CYsJGZexy|YSUibeB!^Wn3iL^>3ylzr$VED^8EzA?F)0m)OUV< zsWKIVo;D|>r2NcTlb_u&F*TLkzWibP`qv6rkXQb%-NW$Q2ntjeAqM;?mL$($V`Y_} zn|t>RmhJh5lgpY$kmZ>Ybu)AG>>i^`2;4I=o?V zbce-*S*=()?(V)<^{J*1W~Z_7BpKbhfuW(E!*9a#*fZU5gf0h#hlk5Q2d|;7Mb%|& znMbFeNVoZ2hc4t?db3qR#*#o=Cuuf@A-S@YwtemQ)rO1vttj2A|@uawzs#p8<2%KH#k<5fhq~4 zBIzR8!s3DlQDgnN?^xrzu(-%km{EMMpC6Z6XJTA}8LolAg~U5k*53w&N5XXDNKj`? zv_URaUbQ{_{;yvBqxv$vn(9wn?;bS*)_<4bLS7?uMucy}&7y#RgpWDbD3_AzIF7N6%H(Wrk!uqsl zMqq5!%DtOd$M(kj>hIb5XUY5z&h&rb%}@b$cJ?yk`=x)DV&EP-3s1X^Y;5~h2%+Y< zz=D;*DpdO|tRwU9Y}r{n0=^itr2~&z@%=wCG~xzMC#f`M+*>dA2~oY0!|n8^u%BJy z471-|>uJB35IN2jt_SfJ<~Aevu->5Q$| zSJzOtHAhD-{<(1P3zpe9jW%rBC`Z^)K?PifonP#-DF-yLnrG#z=)SBAC80mlmz#n@ zv44u|A1+St7EbuTR6`+Kx(eoJ(eM5JNf9&WJz*zlmRqaf%8aH&L-%>I(zf}7;cn_bMH$g4?u4g?yX023@;$f3>L5llx;V+PgTAb)h=i+JSo~!2yBf z>vQkAE;_&G_E?d{`OtZRK;?Ez754WVK6gE_3-RwP!0$low`|;U+gebu?R#aty0z81 zgca76fxt3oM6^4P0T&9&aM=4E;A;e7U;p*2{L3l3C2e&sMyeR(e8g!B^iFfkP=!c- zvId{sN9^jV<1vI7HRfG8OL(}t){|cDuwDQVHMOu-ONT+KW7*pm-8XUSSQhUy?cZ=g zFE@qKyKD|_0*+ApR#27(pTPqQdWqf5VqxjnSa!oVcAwwyn4k7NbU?_;%3@qy@+yCC z#K&6gFF2!$&6YPL_m?zKR}edjX*=6!!XsWsrukEEHM7xfKcB^o3LegNUqVc_Ejuh_ zukUYB-8haXBj|jqZUi@Sn=4Pu{M0g9U1^|j-Zs0fw0Ysu`|rT5tbE{B4e4S8P2=8Q z%I|G#lC0tDgIx*=3g2MW%&l*=;#ZpX^<1^-MQMX>8^q~M$HF80y;UUOk~zatZGuY(vj=Px!?7edy;DnDxm#G^xIlDiK1!W$%X;8!`QYsS*9N8z|L;bN;w( zSy&Nzb)9vwD#sjCEx5nMs>zmZof@R_EZ`p0L@#2~f#18=c0}J1(LjcEKmogA0TW3b zw?vaDJ=>d`9_(`J+Qr7&CI9 zT;(#q+E4})_8E6Fp6cl6$VRwqk5*iWDXkoFU06B6!_)VHxwff^Ve>RZTfdmqA)yw? z)6rpOmWAxe7S?;_b}gz+ALlLNbHnd_A$pTdFNyP1je%bk_DM3rvL0a3KsZ9u|GcOE zoLKRsM2UEC*ndj%RM%eO0LEFCD_3iaqpD?c@wubckud5gVbEfvG$7p|hKKOKP5}VV~ z!Ikuro)@mlkERTRuZn7kW)6AkxIGt$=0MlkoM;RiE_7N(PP6jz-FxnGoYat_fr4=A z4?kbu%%M*mU#XrHz8LN9XW#J}la3V{$=5aZ48y9;CUxJ(a-S_X&G}t8jJ~t!iep|s zjP>$d8Yp1~7O;-HHrJd54G-D?F!^JJ26V}*HZgKLGN{j-WQ~>_r3n$8Vpn+{x&e=8 zf{CSQm6A#h1^x5K>9eSc$>CjmJSV4MsO;gqA`Bb-jLQ(hz`}C06X!-LY+*S#DH@^L z#V?xF`9@PSSu|78X=@l+8N!gP%p^dg(-msU7{=T|Nd7Xvmn5vPQ?nyG*6M(h>3LPE!$7OFQ0O;TvZ>?KkaB(_B( z;AYb5c+twccY;pabhNaQs*t6R{rHiXge11nqhiQiR1VapNzLG^Vkl*d_rJdks*QJ* z6YI#`cni)V%}u2LfyKhAFov`6kF)5&r?gNVX_@>ssy&@0IAps(pj=X}KES@L{_|kh z2d3@%892ua=Vt}%`BaHnSuSJwH~||+D7^%J^LT4^mK9l2Dmfntp{$Br+Mv{=8f$L3b8TxB0v)OGOc=(X8Xp&#P9L#Dy zN;y)xF+0M6&mcb5F;PEu@#5GMnj>CICWrl6s}!p#%gcyI^$@3n`i%rQ2(%@`a(8u&(4bu#xm6&r;=)t|R@~ znr}_$v3)5G+D8wyl=DpVAAN8B!FWrO_1iaKeUBL$ap_sRb<^|0K#6hN@UD+6Q@&P5 z(8|iY-`{eWoK;^sLKAYEjdo^;Cgi)tv~|Cc&9zjx|N0;nb;u_|6DXgiT;0+lncj07 z`4cn@nE3uYhpsMzHhrZ(6+uuIKY?S#Q5Zd{$hfO%Xox=_U#zxcf-aKVkSXx#hhUJD z^>n#rPNnQVjKEl!`Cu`#Y~oYyE4VQ1Nlx%NLD(F3*Yr_BXnk8QY{cuZyB3n4-?b7b zte8GEt(0B;Mg#>U0bM7ficFwogN*Z&8#f8o4o|Wfp7m79rXgk7bD3QETvWu3HCRw= zy@>b`QTFLY2TgMr)6zi2{8~u*G^@Hv^#0mZddWJ;heHebt-r3|uf1dXYXR?yGelKw z2RgC|9Y!8Rt~N&7vmA*=;oOJ&KUkMB4)&9ZcZkj76C|1WVib>S&QLdVcmKY3 zK8lt#Prk%k*pK|kG{sLv1sk;o)ck2mDOT6?fnwWmt*RQJ;8f~)_77NicxZkM{Hihb5-!HD>Viv6wS(R}kB@1zo-6pM*rFKRGJe0C@AxY5hm zw)1OEz|e?baaoyRCy2!Duq({6yus;?#jY-nb_<;@3(}VX>u$?9vjrjeS;7U@z^4L0 z_0bSBRG^%tUZQ8Ik{d6eeg%fQ^K90|VyOO;dP$x*03ihJjz%@lLUaHos2d!o0S#j8 z^8|ygp=V4QxHZf1 zUc#PFn-7&-t{E@w?1Xqdb`@_G@Vk(?3|cb*52jq2G)k}U4E-38eV+(17UYoc-nG!e zxb~zga51YVJ{`+o#zNlZQwFKtZ>?V`c#4#@sUw2S+$2@K#mixw?Z#2t@X4ZH@ESP zY-e|(Xu@cPWBCY6aO2kUz}Uj1o>5~IkMuL>>VYh`ChlrrH&&H7TXx!bRSypG;fsmN zK?G=5B>o?5I3D$-@yqT++*D5v;R{uCbc+w#R?nS#@Su3ewW|a%H3!oeBWPDCmelm7 zX>{m=idC#2A;pS|F!dv6#2X@={nnAqN+bEf6xoF5%z!qFY1zcaF>!vJOKdZnju2Fn zj*UT1vTze2aD zW;ge8g?-U(EA44l#Es>%pPOXJ4s6%;h$ri(DP*n*a9VFh79KmYHw2H?eTKw{S#P_) zd-txvT}PYk%X)hDWMa^KXc!Od%6;}lv=ljo((xOa)Dduwt_j^KZMo()mq_yIRTG!6J(%Q z0dhh13*X~hP*nX1YDy*QMNVy%;$f%4ST&+Ik4GFw^IkPm%H{oZ+e3r4D7RV5pHVy= z4(pS%!>JFa{m|uySyO}t|DR|a*ju>`NB0(|SY!EP7{i9ZgSGm_{MBkZRxF7@ro5l< z?b*Ga)EI3M?4;aN9ehpQoOt!P%~dZKBQS&IJ*k?0TLmxwBMJq$F`EBFpkT?vQ@oHh zMg{KCpRvO$x6k6(7z!ueNr-V4_kGd_UDZ=51|cxJQh&A4oMu~x?%*Zdj61!R^<1cVa? zEfI7OFjy}$xMNd8Ie`htCnAk{dBnzt4r|>xfx;KaS6q4+LfT^Y=cnJJCR-Pmlu~Sa z`+o2D3j`;Od#AR4@JxlByYxO0;XtLNq?D-`bvh*JQ^$*5^WovmA*l*S>&pnp`#CL- z9=`R$u$}QdJbR&(m!DOo;FBXWikPuAk^?0^?dRgS4Qgncema6RuFNTmiIocbP_&@g zq@o3SHiudvAHb_pp&BB=GCQKjKG13!6&({T!3?p~ALt6>gHr99><^999(2^wR07Vt zx_iWO;y%A5IZ9;VIya`D>JmJZkXT4p3T$na681YB03Ym(Z{$&>wZmHADV4_zAtW1X zme$ssQbvV?s8%GlRBFNQ?^Vy=SL%i_r+y?Gv^QiX$f%UwlVj*p<959<$PpX(MBnhO zTKUy-jr#i=4-_fAERYj9XrLAg99v3j#Y6Q)O;aPlOs=<+9_~4Z?diyv$3ttt&U;jW z;fQ5zUM*Le0tLItWC^O(KoaYhRi&`_vWSvrD&oqdKlhCtDZ`@{3uD;bv+4GfRI6Ox z9~r9g$^QWo(`Vjapfwu(M)R~RQZ~JVoWN74!imdhCxqx*ih2puXdv2D%747clzObn z3X&|ms(b4mxL@sSdda7(kl5 zsGK-#e@X%S@b|Z{vD3Y$ka@NN0?Yx;@E2PE17d`knc9Ya# zT8e_-dUp6Ohgfc2o|yhpOqrcp(d>%teuS|9RMS*%)Ix#!GxES-hUU3r=0RtK}!c0s|e>#dd7{u=?(tjbz{nrhCr)wJJ;#@GxXC>7N zT+0p@8YfCDOZ25gpQp;Fm8_FOO=m?xFVQJWodRWuE;FzzQIFM$fK6z`8M$sCMXm-- z9x*~zTORnNx)bBx#PkARFxyD39d;i^tm`c#WBA9JHfM*vEX}}j{kG9VQWi~1Ys()$ z(nd`u{dL>oP`o-jgr_UYSMUxu*ZC6-i7vJX{m5~A z4p1o?DO%*;KTlTRP&3F7;*wwK3U~P#9Q;@d3a6>6NqL6se0;wkFX2bUqX^ck(Ibf% z@Pkl>Se>XVFd9(Dl}XAof4LG<1DJI}0=wlALglh7o9171PR>a%wxYGzY$ zm^n}TQLO!HF~%c+$Oc_vvxrd&Y^1$J9oCG^5xuhBA$%KKbCU+jAa{T6yBe}aMu6;t zj=RQVe$R2?Q8wj>>Sa zhA8K`MhA_DaqpKq4`WYNLS_Pd{#U?|m>-|(5g^?TKE9+mCvNfUGx-@dv`UA9y(*irdFS*ac4QL|7wAFhRJOFHkUN?gwwaMdMMXfS)0-x;?h|%WI!@5Z9y2Q6o3@}Pirew&C=8YGvM<28S@Fh$6Ga@-O zs1!n5FW>7f8#@+q&5J|Tm)_GNE6ZzK$(&zcVoy2QUB&EYDG~B0^}+94=7SrvoK+5*v%N{^ zEg@$qYw8~9i&;3@5y zw+~n_nC2}!55!a22=$(_+Il+Z%`3N3tnox{&Wvs@Zb4A9jLxq~zkgyPQJuxEZHnab z<*4@xzdZ?EGTKX`^P$WT%xr>HrT)J7lfT|C?nPIcerR{7td356NmW37%|y&1xZfgp zq(wP7Y*LVg+~2RR@xIb*AnO57dH+K27MbVghi4v(h!h&4Wmv+X_mm}u-*sp<3PN>L z2rrAxP8ZJaEW*J>BkCHcqqL)K7gRo&?ddtALdWl4?!x?Y8F%u^{grkVNA)_K+foSv4ghzCg*_b-&2e#1uWk ze<%29QfKpYTlj58#XII~WYAj7^L*o}fk8@lpReP3vNY-hJkESyR#Oz6^e+-TxbWl+ zNSWGiEtFJit+2?qcLxchQdAS(+g}YVu}ONksw3h-bn&90JmAlO9Q9M*JWqy(Bvwn~ z90hZpeCy@qMKs@>creu!~hY45f?tdoT5|EK#$-AV`g zP1zQ!tLdSv78bUxljO(F?`?2aHBYZ8Wh;VmuLXsC8lPlmfB9^>Vzy!D*UvGTroG(; zL9ED8XjP-+)q|YIsbQ-rymCt$=$^?!+p-Js%-XKR95*kFYC-42Y^``0^!mFr!J$ri zeX$jAZcUlfa;2eMucBi4F?#+fp{Rs}#GE*EGB9hF9yA2%cHF(2mBq3--!qS1=C|_a zIV3U^TAtZmZ@;%r3RZaMm!$QmfCHc1T>EU)g*qkvVvq)adC4-lEv&qT{w=T)kG;3v zoUXXAj1GPsVsQQ+nmTN?8rWyu?$7<1xPk&z9T_(NjT6W58xr~7j{|>yGw8|V$ItD~ zKP5FcU;eWvAdUi}>DDFkBS5SC*;rj zKHb=*@~n4gX#_I_owdanr%eV~I4EycF$3>KdyZkM<&}Ato)@!gA<%5}RY%?V0D5jA zG|ki-z(d>4@6L2gn4}(uX~d$?*f1yl1Nqf{*IPlD;OeAY)`F*G6d}gKy1C8<`oCIN zWWm#vjfeu5d|mO`TBt_Z_7MopYJwNkKA?oE@{c($Ub~}j9XCCJ@8}_J$7i`P=)U&9 zD)?<+z@aG{Rq@`@YGdVunpzY=Humlf4xP9J5VaMnWmr9xFX~USs{h;lW`c;$x@V)0|hQBl!Vs z4?b2x!CM~IZh*#hd4>v#>n27e+uLZwJIM3=$Yg}$ZcY<3s$#3Z%hCLQ7R$~$qor|W zzGyvwirez2%LEhj6@n+fI-q(S1)YORZU4X^w_*3o*)@gNN?B?0X%+i~(YzM2xYS`O z7REH|cS4#Z)rxH9*B1GwrT(m=;s0s{tpp|ntPfr0+^bxl-s;V+h~g zg2BFOQyKJEpxB@I0^Xg?T<1uNY!46}Z7+6-(*)}2x+QiQv_cHwXoGs%k<~Zvc74BV z&o@djQ+dzBjY2m;odsEBKL&2E>n6GU7j?ng+(i}%*&6E6`Q=7%b;yKm`JRMCdPuV? zq5nG~g#Bczpm{a)VB!IR$8%THW!r`C@A0NwMx~hGu+80xncVZn*ru`I@`^|A$Lahi z9k&ijf)n6Jci#^kdPh?LfBrOfjrj6zAG~E_mrJ?0nxQd89=fw9f)CB6* zo1Z@IdyQjopwSSrh)s|5i&R8&?(5<+sx1_?z*M<0w%>)a*TOZrg9Ha;>raaB9e; zTIzVX%uPJft=12+o$KR8q}tkr2Xiekf(a_Yb;)%<`RyV(%@4P}{aK-2gwob!*9dSu zix8i8_R{xsANvAArmdm$*d~CY3D7_4io!N&;^@6Ds6X}^c1-^JQRP2>GxYnwz`ZhK zxH!&AU=SIBCj_?ed({)~|2*06U`vYqtX?9>9iZ0gzkk;|%!-MQhQq2RfBp>EENc}Y zaD**$-!BiqOGZ6uSKv64dAmaZ#sJ8~dA)^41vUTU;c;v{dFSrkyPBGso(n&xrb3wQ zxY@59xA6pYwZFf8>Uf+1T|{K;M8BJRYAKGZrJS%8?;nTAlbnBe5!dIBgU=t_;% zX--{M!p#MSH9$ox0YVu-wr4L~-^t#05eB%_%I0PTptgD!u+4@8;-D)5PR^!v|LvLc zx22+ICMO~4rX(V22RtTl$q=)s^R90l)({F4)63E(6p;`Ub2@Ik;O8&vCUx3b83RD` z_A(k1$4zOCMqfc+665vtS6iaG#HcnHApN=r%>hfDI^Fh>9ova++2#?)bf=fEfc z$Xf>oha@)?^a@79f7@s`BtV3k!&tFkKEP15wJi&~uvitDjgUjWN&B07u|Y{$8Nen7 zySg|Y+CnUyR|OLTz)$VSg8Y2=A`&pifSZn_9wIz{K8Dk<6HZlQU+G>tRk2Os3DEV{ z@Vg1lyK8dE5{>;qosIT6KuS{v6~+j1h?xLjch7pJ9j<8LB5Fzf9bj+(!h?&@>dL*p zbc;bLYZyRH0E5&w(s}`KV_7$UDjSu*4-~NTAhX6REprbV)>V%yq zVD10}`U5`V1rW0GC)iFe6y|6j2I))sDL+>(_z}$jV8;gX^snHFJT|ogoE6|sEo^LD z{B4pDxW11QI$i`7LsvS0>bZ6upe6n~U~>V7dU$XEc;BS1*$;~V{#u5l1)$@Nd(wJ9 zTmb)wPi?!1zs4L?7>zq6`4F$wTB_+o)-W~z86{0>Cr&7VGf0yy2??k)(K z3FPpTH2{8@4=_1^uFZ6%w84xxySTtw?`_dtEFQ#Cxlsfmef0kYIulqIYWh!}Vc4+hn@Ge^qoim}kfLL=A&6tH~&NS>djCA0876R zfa72ymq*Hq0YF;_e1gLz>wtT6Cqe^p5EiVrj5Y^OMYtoN{Nng*(f}k15KhUCTaCkY zH8nLr)S&^mUr=Nq2G5~NuzU~;#xz6Q&`{+7z)`8}_A6RXp43m&`x`XW!Gi(z8!Qfo zp7GUhD`-LJ&VuX0aykJ38*PsNm`bzU-``(0Rlstx!OOhq4ggsl7BT-QC z0)QM)RFv5*llNg!a`2!)yYW1_+;%ZCJ3Cwb2s4p6W$6HO1!#5G{l#KfG;XtDdw`R& zA}fgz!ysaF-KhYW1*pDBZ^l6QU0rvgWr@?UHKf+N02f&J(CY&H0j&4RfWmI+i93Qr z?e{pL>8-IQE+u6e^1W!yH8^Mzt_*lkh7rTDg*(9Q z!K(gsrL_$L?6Bf@4ROMBILjcU+3_EBtVsKmI&^E`+rsEO^hHQ0GrdjV_@P7`(K`T6-h5Y!=J z;Bo+n15|{&`ybSg)=pfjDvvjgH>mL)r z#8BL`)C28`J?j_DHlK|VcrBEIj(V!l$6}cbA%Xg(%a=WfKrPVA#JkUGzV`L?4GRlf zeS`wrU6;!R8@Kj+_yz-w0#6e0 zM6n<@_f|}ln5MHMwdbwYRdp}O#y!sHoA2%~I0~Z0S(w8dt z1eZDx3nBn5+Vqir;@7Wac?MuK82I$yE%Y;%m6h%9?kZ$!j@au~gFgsr3kVSE&Vv@g zYB%~Ux>!D&CRkQjIM^WY1J~)CwVrN^lddRgZ*K?aJSlj;fPe>K2F|FxD!L5khO3yf z&piq0hvds8ZW{yb3kaIzwy6ZHY#h)VEEYpZBtrwE044U)#zxR$+@0uKb8|ZEJcxP$ z#f`O*eMq;(*;)GFM$i@>M6h>eQ35f;m~H&LygWEAnZwqH5Zp&#(gD~s497g$cvy1H zpW6ND)2A=3ti(snU>}q>XaK+)NDqLmMZH8eSEm-}S418^hN(f8*|yZwOf1%)I7iLI z%#0otfh5t|3C^oSJS-u5U5%roLLy!{U%&DK*9LGFl*^>bL&*1m340O*j*Jg7j|XX^u@Yiz{qjWe&r{K zIAFW5{5K$^lZWl#Qz<%vK?Nt&JQ8|nX5rM1RlMZD;^JaM(HN;PXGIuyYph_@$B*(L zMIHzX;J9*gbBmF(o2c^zgPhVY0}l-(BN9qZKotT+4x>w4F$iGs09v|Y`o7>fR=5XS_3w(0_gv9DD&}Y-e2T7 zycX6x1z4w^4S>n$Z60ee@K88zL4mj+YI1Ht)(M^LYT~gT-Zm z{lCzEcS2zSq8`+t@@|_X*lRk;NNey~CS3%fh6t5KHmz5wA}L)FmD6C^L0t@$*_ne{ z;+Rv-JOGaX*d4IiAE?VDGofpXmX@|fh!$jb5aot?oB+_wV^tyueB8i?g1`lOB3RLA zAZCIc3%iRH@sSmI1l$y$`ov?+y5v9Ie+>~CIl-9;|AjcWqg}M{#C~_26x3?Xx%G|l}rxRa|US^3Kx){~(IpSb;Vb~ZxT?MN8T(%#74fVdWTd?YNQL>WS` z8<&ac2Qe)$mi-^n-aDS_J$xT;qOC}xNXZ_N>^QxTWGh7WCMzU+RaA(SnU%^)NOpEq zW*G_D*?aH(yWXA8@AudD@%^pyIFECr_jtXYujli=ulu_1`?@70yfc<>jQx~rPPP;k z6WjV8aS?Zd_5bkUgLc6^7^m+Y0>6kM!wnY7>50Bse#5&X5PkD2k*u(I2-ylKUdc*H zeR5fRpO?4Xk@OTTU=>yR;GTQhq$N-mQW92eMAS9HbFj3WK$h2Gy4%Xy8WnO+7uG+o z?>^a_KeBtUI=Hf@#}mM;TDMfs%8E0*)^lw%X+SozAW!1jH7H7CL)ZZh@;%_d<9qcl z3jQxi6>KFjTqM^18NK)xB25Ta6=O>$r*}e3ADTaa2@n(m27=dUL1?q!@CbScmR$uo z*WVB-X8Ex4;*L{VP><@6<~vS|i=CWSu+<6w{!7%P0z2)}{-dJGZdJ*7h96nz7ExbB{ic}E!-qteTo)|N&7J=IIeD6@MKmnxK}1VBI@!6o z3f=VUaAXzSKtJD}o(NE!bhNR#4ZR=KS!4a4xcM#9J>Uqr|MAL+idWkG`p zqBT%=k&k7*i~xdYq5=2-q%+_-@GM60yN=7#D8HH$VZ{ESBH;B!n^kepfpGzf1a`!! z*2FnhP8QGC_1tV;(PQ}=&?B;D^8Sre4>SvJdKv%62|=BF7`xgX2-Hk(+{epvWD2}A8F`?n#xpLR+8-*KcQsH&Xvw=c;I)m>;kT$f0-jAN1-u3GZ_2Szo zL>IOn>gx74c;P&std!nvrD3H1(iNrx`VaI$RK;R)Qt zQ875ci0c3ZvG-HpU?LtcEG)tdc|#J!P7ijILezzxf};P=pFjQmO%Rl8I`afO17bq& zAV-y$68q-On_wn+8Xg0Q7iX_}lY{X34M7*UGl&eK5FTNmKexd3lx8eyq5XK63L7Vk zB;aB=hV84 zRWot22To*er{FkgYG^D?v^D)go)-!OlUdvtT z9~c;WL7xC{j2pP4WebrXAgEYKID!zu)`RwvX1vr3P-t7NL1IgvZQ#(cYZkCyu6U7P z<3f3K^zg<)9(&sbrucIG-{(tjJrz6C|&>dNS)obu(k^jZ#+yLKk?(A7$ge;F% za0!q`s#)7KVcR!LY^E2I#{Ngd49+goqZJ};@Ey=UtXk7HrTd@!o|DY~-+IsgZH90l zxxQt55xfdoHjnvAaD}@%J0tE!o&F@OIY_4M*-HAR{HW~$x4RZCsmXvb2rlF}@qlrz zHl8#58?Tg1-iLxxkY@Be2Ht3v^|MDhz!NMA(0ra%M@#1tusL>SEzWoGH7lWtYffk) zTLjky)T#{oV_1t=3goD;`7ajaKHa|IP+%p9YzI`AhWJZxPVsi=Ta8}cU+a93>?w;aNzXedKGz4eQ3?Tm7>3S zTU{HigUV{fF7{Am1rAasd!w2Hk^4rN@jTCy`~IpwB36W?q(zQ~on{KCM-ZXB?{gK~ zui3u_w~0UDNX9jddXu9{7nB$Li066{6gX-FXfEwPcu>yR0%mhzcDDL?`Q7{f(JQ$) zLogz1Q-#y8lsSmY{ky((h|5;Jfbbod!maW2Qlfq>2gd|frcJwXnB~lw*AV3eV}~RC zu(Z?@>SSoN6mC6q-~hx?{m9GpE}Rmy`SS(p+xHUJoSt3sj1w2w?p1nDc*CNZn>uUl ziItlr%;5>?7C9jg3h3T!ZVw=d@Eanu;56evkr!bp;Q|52WZhR0Q)9PP>Rmug(2oO- zi@#@UD_#5&8wUIoW=3U#A!32f8eC~l{%=GY!j4loS*Ix}RZ85P8NMR;09&q+q;MKA z3#NsLeM>!ioObr(C)lH=+kAKPi-_7~4IBBZPKUDt1M6Yu;B;=nEQ6jA77@Wi<@&~p zc2uOjiFng>2_7TW%gYO0X*`CKK!lesU#1uLC<0_A>N!HC(PNBHaio!8myu6us955ARz5V?_EOh;)oy5}s zGog{z;*>ggcNGh}y)myCqcA)%(V1a*4Xz0MGHYnfdD$>TS>Gxv>yf>Hv#g-lgj`mH z9Ms>|ckSKp{rE9h-EUvMoa5s90Kv}W@UQeEa&sA{i3dz|>kIW`utxZ<5H9UC-Er9j zpiw}(QDU~`HA;N+!(-W>3Vap*Z0NPL1r9`Fe3|`B@+&M96cKdyy6x>hC}ILW)i0oeh+h8N3nc zVVmNKB(#nrchpp1Z_%_j5PJpZ4dZwgKw+lp6tBS=iVe^%`no`i68n#blN$UV_-t*a zV8Hatf|v~WoB_IpsLh9u7vKb~8wH=}fT@=QRWAV8;D@QUMQVt?(^P|edrO`7^cjY) zc!~coX_Wy4B@s@X61qD8`X6PB4j>5)*O7F(7C_Tn>bVUnyBO;JOiU(#L?E$fE}F;z zG=S@TH57*9j7yRYWiz(2(uQa%Tm)?gMz#IOs8fMJol$iVo=_3oGtLfqc~_Sf}lFOrvVPriXuEhE~Y}iDrl%)rvEa&I3HeMuDfYM-1|<6X263 zoe`L0oER`j8rC4gmi6%?#xyD#uL5J5<+L9=FGI$FD8SVK66P%kX3;hlWf#<6aQnpxhIeghKG> z^mJ?CQcfX%nFLGkgP7eyLW~WT9LzfmwPD5AdZ-T~+ii$#-0+tX2 zC7^LE@nomM5L3IJF5+ZkGY1_FB7v1P8QuYFK=8q*M+3KEGs9lOFgOceHG!NZagpJg zKN{CkIXW&Qe1Tb=0>lBO&DZ@6IVfvXi$IiB)HxHiuH}FXRS=tyIcyI6H7w4`)Vtj0I4eP+Ik( z;J@zL+oSfTSJE!`I$J326TbSQPH5`Ql(3AhU;VgPVICe3jv(-jK-yMF0%JXYK2vw7 z;L|5+dWcb_z}P}CD+qGiu&pkjo3OJhJ}^~GQB;Y-Z{it)3TnB{hwo}Ha?T14R-V(% zuh9JkegXau4uwFLpfox#uuzxAibR6~(s6wK@o34y)#I>bWJqtn88~u}>3Sc42R&rC1O->HV6a7yM{7Ww`s5HTQ+O>9qoGz; zMa2-d6!I&h$X(j5!a)KJ(R#jy(eE1{cZI#c8bh}0Wv-dEbr0xooH#@+Fo09xx*j}B zQ=6S`Dx?}@`V;v^!cxu5-LY!Cntwu3U>>zm=8JC$qR}irIb0Q4{GlHZt7T{t)ESM=N zk8|bS-CH0+5(h8Q1lZmg^Hd4i<_HC43j2bbE{Nv1Kp6)ix0R27b!VXv22Lw+3DV*H zgVp#i$YV4aBGpvl9>oK-ythbU0|9{I2GPr^U9bQQ0otSymh#{>rPLHw1&6c3v=z*b zO6sjyz$Ao~Aiop8<8*)(g0QIzj54&{;h6W5kVF=aO^l4(HZv1K`gM@|NL`*w(kit z62s0ukQXOv>?6M4LrThg{yfo`pE+{|B35s%W6KL)xNtp9b$oFR)|8Q6^%ndZ91fWM z{S3r`T15I{K7-j>b}MV3zkeP{DlBu+N(6 zybYp%XpY)?ot!>F%qG_=DxS}9i1>rwxi3K1lD+9IR^}k_kVaHD|on^yu5A8*h%eloy(61 z89KB@_xRBxgbl5btASGWJ#{Iv-jR{tz?k>%?*J&pY_K8Y{yfhx zZ*p))@X)YQHVzJT-|L@j3&2D+5laM+5+bHluu}p0p-7$UJa59y&5gGY4e?vFa3B$i z*tOrd3!6P%Jqu+fAgq$+{Num|1=3ssoCvSQJ6#t>^er*Z!U?6Ch;=Tm&R;=}z;UvD z5kl6mv%xY6K;a-l1P-6T?3+nMbBiv;qmBP!hsQv4I5@Zoi7Q;s7#3}HeZ8`xLIG0T zNQVQhUlKDXrXdh`i^F25u%+Q~3a1~YhP#}HyA-zH2&mKFD!u5!&)ah5g%omuh`msk z6trPEH@RH?hvM6V3S2(Wqb9KN1lOL3DHP(2!`6eM%_<}}I_qKWDJIwwn;-c0z#l{% zxDgibPY#nB_QjTXC)NuO!*$%F($0 zbTR()|E*a2ziC8Mb=mzTs@U$8qUG)DDZDw618*JVJjD_st^|h|V>kwpqd*P*;Uh=t zGcR`oc>_a(kolf@xvx|MlTl!_dCmG|66U~UqYetrNdh|<*7Sw62*Cx`(-vb^BC=<3 zim?GwC47&;tq`A)YRZrJ)9$Yn9%;=-Hf?N;Ct26O&&X``i97JKv%K>_)YI?Q+PCl2 zF0-FmX*LWWUE~%$BGdh@d7S10J?}=<#&QQ!cVZK*u)L^{i}at4tcvuawZ#T;ytggt zsK@TZVI!aVTjB(!)&5PObETU46#Uk9Z7u4D-&()FA1L!^i*9gG-S1S7L-DlzZtX-yKgcob4pXO6Jm2$qb zk7P|!6X4RY^Q=^-*Qt-GVjlFEX7qk8Yg`)@&_?HGP#GgG?v_PP;P_3qF&AL&w!DA3 z_|YSghcOeBB8k7)qg%JWj3;+mOwWh!Blzg|goIYdm+sNpp0VmX7xL`0-i4xW;kFV< z+NYus_P(=SMt56V;tcEWsFr?;O<3RbKMm~E-CcrXqF&U)Bp)ep^(rD;uBXdSEtf%T z;Ot;P7B~DX3Iv9LT9GhCMkBM3%cuw^`~;^4<}fDXHmxy16ooDzD`=qt-(^2YesN19 zwzeYKu#E)Q>y)Ep+R(bV#5p;&*}K?Dw%xIiljP-ET9d}Sk*}53fn2%fU6HNT&itIwU~9*aaYgsXIWHT!MQ-#Hk1i z4?KkYg*4r6F5MDQ)S3Yt0V%nGm_Q}`?>kmjlK@SqtK``H9Gel&t$Ub;ud1>#2|x$+ zHDm`4P!KwcoSQm2*5U0Ezk!=T34-~C#@QD{A|2S8I3NkI8ZoroQ$RO+RLfx(gU^Do zvCVQ1m@+;GP6B`b?{gn46o4TCiq}Gn(;-#2O6&=Z5_PIB$-}ldt0yB z;n-|_-g5c;>-LoOfme+m=Q8iOPRB@>%mnz9YFeMvcG5}NWf$pQXjh)~n~dRWHM1Hl z2&eZ*$@+=FhyZW$^kZ@BQ?WYcH=c-Iq+R72&*htm^_*)hpDvV3_vGBz<{r4I)jpU+ zAsam1*P3NfDn8RO|2v$kX?&aRlQLOjYo4)@!9wmIiij2M717k+j`d%f*T&z!s;Ks1 zF%C>Nm2qglk+9tDtZB`xJ7;hC^W&p%^TiI`_w#kChR<|AZOV|nX+F6>uft(mePT_V zj_OX{Wcy~TaIt<}*lTVQI*vB8YXK$eo;tRdZYZQ~%6)Jl zf4MMHF#O4Vuc=O0n^39N#Tr5Ot@00Q;~v@RExg++|I%%GD7w&4=Dtq4K&_-ZzW7Wx zE|A0S&Pws{%D>NamwY-d(`*H~Hz) zoZg?1^#Sga!5|dO>cZzv0(XNa5tuAS>KLI_qMf`E9Z~Dgb(u)Q0vKVFxWm{YktKJd z;S>_4NEHLkfQ4!X9f4znP=bO0a_KFI5_EL{vsV~`6EHQT1dn3`7mP350#k+avW03q zq~xVD+Hh21^I0hOE6WngIaVp?Z^O{ueBYpQX%O885!~9 z1A8Os&CKAflMVuK9V>xJ0vXmlVi=l$DyNg4Bw#Zah%6XH6<{!tQun}Ffx%F+yvV@N zi#GtZCWfJ)fQ8Zzmti`>yd@XocK2pl(=TqVZ-OVILK$y}9GZ%1av{-(BZ?Rt9 zALpO-s4~3O zCz99aWnOwd65X!tGFT~>U5@>hx{FqmmzHH(#7=5iwng`Ja;1o=KM^$ohLh5IPW=M&B}bFtvXp+dau41PU|w?d`eGAx*NyLx6~h3 zOdPkZL;VWt=#}KjG8|nt)pw`R&~CB~2sv~5XXQpSPZr*IPx_$cTgrLsMrKy+j)05L zOh2Z#_YIEqlH@48{rF6*;Ah3KY|`URO!mq@kZ=EzoImgBSnu(avn4|%&j5< z6Q*j#&|D}l*8AD?THs0e?^biu&hVQ@dj!^p%*yFa?4P`wAl4(ob(h+yVj7pcn$g7u z03j3vfM^a9P{oj}jt&g9z`Q&VamckJ4-6!UqQk|J#xK-tFk_ccO%75>!2FjKu|2C~ zYt6#c)RaL2@d2@DexmIn_zIBP@TA!=iv%(N35488Grq?kE6yJ{&c^sc3OWwYfw&HC z-=@#!5Njb8eFDBBR`ol<&E@?PCEdVu{_n$@=NkeSrj}bWTh-QB8v-ZzY)$U+^Ar&S-+ke}P z+51+zQFi@w%yReTuH1WFdky!;}hoT{Eq0 z6PK}!jE|XNny497mL~X^zcSwSfxfCLaPw$YpW&u3|ERRnM#Hf?71c8=gSuC^nU_*; z#rYSK8xR8hOs%)(qc$df_`JSm&F{aQt|}Z(-Zy+_u)f=$OmJm6yQGBca2Rj$*Pn#3 zK0Q|N+czpx6r2P5_MUn~zIXojr4KK*f8{uKGPp4x$p2M`(xTqm7W$@}^{esL`|{vZ z#195=_KiM&K`*9hyeU2wc)k4r5Dcr`J;iQ=-YH|6O^zH}O2P<*p%HJs@WKb}L8ioS z9K|*J8GhAIQ}6yF()zQ=h3y{AxKfUsh5CsUDo2MD2G%PEX6)Y2MG6iuWb<%~=Y778 z$!-i^JH%f{EXSRsr5#7a_m*e%9An^oY!A}hOa;#>joyx0Kb6Xz?d>r zI}#)3lUwRyoa%0`kuyqHO{ooKH;u-(fJRWL_enG7%$|`2yfnLEXF?d zQA*B6a)QXf5otv!#vtJQJQQBw^nq&xJ#%u?Ozgb+cE`GLIqeY3EVaXn`4*>I1947b!iv65|-&5+l8w1^0%-doEvhB{7Qq~~4V zei6@x=2J&RTC2X^PSaryFYz2+50s~(AFsNNsS|lmb8P2&=Wj81D;_=3A1`s%&F|A5 z@n=*7El=`<=dvkbHOA5BLuJ^)Z3^cZ*CwrmJ2a<6kI(f7efD8VkW5lkF%!4Z^C}Du zCGRuNW@Cmz_V%^SqcZE2gmk}fjctol#lQN}OwXS@xW<@pfjVAw+jFP1xiPzUL&`BF zP1%#&-cCFz!56ypE{ChgnzZIh_twrIzv$(m-)m(R^R^KJ ztgG>5C>VIB!iGMD0b-{&IwLwY2xK`LX~@waX7t?N8knr6c!&x@p670lGv{vMsSz8^ zEB&guk3{`61iykcm&I7W$OP4tfFsqLm}mx*hT)J?2y@$!7{tgeq>=BgmU;|>mHhf` zb6nT)zfn?kg~fUy-#|sqE{TuvwRLQ7bG!AAf%Av5S(<7%hx$)G7{j5OP+79x$;o2a zOB7xypzIa7W~3-k=6s=BT_)qlK&3`{D?|ZbzBM@<{H?UOKj_#g4lDWK*)a+B{IjdO za;aLjHF;-EzV7a4*?TTWfr4&Z_#4S37prm3ID40_Z0_|F_CL78YlJ6PR(C(;Q>9K) zxR+aVpLyO*lz7MI&ArK&__sG61*n-lA`kQ1KBAYqEz%mO`0HKHno^9zOB1uj$65;0 zP7Q-!Z>l-QIJBri^?`H1{17emq^oBBS_U-WOXcYFP+ z;krCLt|V*0@M7%dHgfVS_I@8<+~tIolbCvyb$r!tJB1^>XxoyJhyO>wfL^udKNr`p zHMO^EB&(G41Yx1BdP{-CLrFP`d|Te%WOsI*YucNjwCcf}DfDmHX$x>$D0ufO)1&b) z4>3?8dm*}qhB+;DYt=RQ`SU+xV{G2&9R%Auhoa{oZnL5+BT(`{W_a|dr~}gh{1<@| zM4&>Ml6aJtp+XB;f`o*Gj6p=s;9-8;uV2lm$=p5J*LQZS^#sc!%1YsH4-{T?1n72P zqjYqzQ~K>D-D*zw8Zu6tiDp`NDn{W1HMJl<8O7nz;+|rw4N!KWMV5FVP8kg!iH*1Y zdjd|evU|FAXtq}{oqx$>#gW<PLxqJt|kW#1gqEECN zW3vyQWu**no2qCU8TeW#B7*Xhf#c@ zi4(}{+(DlX5@Z1a^!}hWVBX<~07mX2-9)n`Ks5)p5D22WP<&8SeW|0Pt3~K+oA$%E zA6759Ov0e|8_lFta#L(M|GJq_be`_&BdWV~ngAKI;>i+Y`olp6UKI!bSZ>yH56HeL zQ#33OxK`KwQ2Wd^v2eD`+^`W~Y}eASz9v}euey(++gacoX;tR`9Dm@{0iH9*6&793cduO>8j+bC zzdl$z|3cFvgH7X`U{Kk_RNCZeb4QwHL0>J5T+^_e(z6q;enEcjnpSYAetI$U7@GW&-PX=fNb%E_^hw6RU9af?7l` zMFoC=PB1`BT*+it;WAQP6?8M8pe!%|c$SgaH(}MD9f3h~xM|g&7Z*il zc&Z-mMhSBd{t9q{SnY?mBSb|0-UNweZ*suz%jl*#QaV)@|_g-gU37iD>HBE+AN z6XCP`-GHVHVn3uz=vwa+deoq=fP9A)BzD1}C5GUGhyYE2j$n#h))vfcY-~^%gp@7B zq1e5tQ50>vkxl*S`n{Y|A5>AF?74JCHGgzH@rv}2IInnkK(MXq@_zYad^KC7CY>&;Grds_5uR4>O5P5KehiT= z@h{9v_-@b^s*TO6D0=u5-hE;(`{wDziJJ!5NnV9dbG{6{j?*aSVq*39)%a+7!eHRG z@rk4K_FHY7{f6*9Ct`c}yV9#(Wd6BTD6B!X@bN&=x*}Uucx@tqr?tniOUUlr*%j%9 z-^OZ`bDj!g=3%wk?_W1H>^;FvQtGnBt3TbABri#)?znQr?-1L-Po=#v4$&@id}n*_ zNXez^)~~32r#PMNv2a!)I)+{;v^tPVHi-ZuX69~v&p)zY>fxoq|8DPMSY%1@5gye5PCJ05GwT|UNMsZZ4w zS3JxjE%I|tkB{^SnO80(iF=;RaQSresP6JBJ2^WFFNL;{*%Q|GVgEd4M-e5;D)-B* zJDa0>gND~g%E)MdIDUNE{4`RXa~}26DsKpn;DgZiW7hZ87x1FlJO{007h{Oi4f8*J za(f&oRpWBZ*9cv`NZ}JJYv3B;=`xWl&)52Jodvtb_|(@0M8!JM&G#GJ9%#TE z>z*x?>VTHu7yEs0;&%or0#W8Lx4hhXn-A$N{>y5x4NFs9A}D79Fh{?uHvS%M<1F4F zb`a2kejI5{?*SW_kBb1gi5F{S%ry_fYb8mlixSkTUbyZ7r@o`AK@g3#&cdjIfa`Fdi zdCo&Om!Gv)X=IY*QHsfyz_};uZj?Yh2tt)7zX0^ zEHj?9TPx#VGqh_(+F8v#OShlyB)z2JZn6E)#l0peA&hfkLap0yteuuZRp(M}bOGIz zF3%BDE1|%z&xgL;J@j}um`440`cR9$TUf}#wu&bb_gP-U^`y%FHvcfSVWtX53=gzs z zD&>-fy5pa+e6JMun~TRiMKgCm#6eo(ii%-TiNmUb(UVzEy6CpgE77V0jH6DJz7l=y z4wjwyX^}J6B6Jmbg(t^Pm$tW>U(=}KB^K`a`bux@YE8BM^LZzz5X$~HJ9muo6-g$| zth8!Ae3nprJM2J!YtG2r(yT|Z)B$>~z!dv*lW(Th`;-!_$?mHD96MP0p|tVC9fqHT zXCWU;iA;9S8XKnL7GqYspnq9=sOs0JFpi>(5vS*p#>dG z2qHg$Z@UHX#RK}o1x0NyZ(KXEbvksZwkRt|HGMqCRAxL#w`lV{X;Cc0q)egkEd?!m z&cXxIW=lhw(lndibjLH1^Ygp+-zbWi{F(^sYecE6=M4YL3gT40SF#sz#8Z=Z)87F^ zo-XDhU27`O*pjn(Elz>NAZg{=#aRZ@wf?WlLBu-gmskIv$U9b}nHhrl#5d5h(xpIT z>UkMT<#me<@+Us*s-ph?Fesxst8mAF4T%?u<=eEsmEJ}Nh(<6aMY`L`1m9;(U)eAD zvS|ltgvU3YKR+78SHKt`CY^Tk{AjZVew*I7pTS$@tnw|Xw`R~XxZ|q1nf{yW2*4+{Qg?LAzqvkhgmTAE-_Q3?p@xZefy7=ZNG4eY zr3cAD5TQ*!%K(To1DZ~ zC+_&MHu_<=U+zcg<~{Qp82XtN}ezB^YdJkqzyjP7fK+`Ww197xRP|j zr5I|1BZo+eyI;2+@Velk-*ECENpS;W1;SY#$XlbD=hv}hiL;?H@k$Iwk0q-hUBmRE z%IL1_qhCr46C8(f!}#yiEFOX@|JeBq)KlaEgEg>$3pNo}M z;k>W%$-WR_*@z0l`;WOQs94d^R9DxmHOf4);)Cx|_r_|oh%qs-@-p=fj{lyKa<_cT z$5Z`U$s$y`fH6lQq2;{UJ##Cj$#y;RW4;Y+!3TwlnQnwqZ)x1IV$D&Zwxo2SeUw)C z)O|{eMu0i_$#+yv1rrr~h&m|))_6ClMSha- zEe=s#?YJ!ToNV{g+EzMgq2UW3f zH_lx6LlM_He#pgL?zY52(*^q#i_KbU`+L!B!V0nRM;|Mm;f#l1NOQ)%42IsZmwWP4 zyhQV|{@v=V+dEgS9pg=Ig!O+uNp^6>2-`bGHA(gTJsD%A5%#Wro53cZmEog(=ce34 zn3{f*G}p4oGP@Q!g{n$j;?I@78-X3Ke5=Zc*8nXbRg!y1+|V28r4br0uTt#1ASyQS zh?j$xRg`VjPS$Qm8ZRpkhf;duJ3|U9QM)$EnUA)PFErKJDE%h5@Zj7^a0S0{*3wi{ zpoyC?yEY=x+<&4mZ>!}i6OKhzVfMj9Sta)1Ut=p0H&wKh3N*kGPvPj2V2>SmIc$d?Lo4hh_1*+neVyBP}Ex0A*VqlaS4y_7HBzhd2RKwit>dyishW5%552ex_vuGRV9!*?%9F&1^=bBs>n za)eRQl+0-SUb1zd3wSz=g0DSBnPwD zvI--$e|%sPx*SbHdaCZ=<6V6B&;0XK_sl<@UT4V27hW?L?5#`&%HU)M#nCXNO!7_l9ZyzBW%;`T@`XT@K&<{dNp|Iyo#9xhokTk))ILe14( zQ#$>JbKr=5Z%mfbr8sF@&-#ls++6W#8%3l<(Xjh@^m|BVXlA=xZ(i#BZJ*~bxYZCM zzLfKbc}35dg1{o+$T@em0trcS!s{TSu+7L_$dx=z)py5Pf zokkb_;7@~J-cQXo18aTf(f{)cPT~s+ZpW*To-4Y1aNW@Pg6MQ_QQ|}UA2tS|3I$d@ zQ?imr{}XJEK2LV=&xTook$R}elff=4ds8(2Cmnp){K)V#*`xbK{DT#*XcX0gs?s?%y~+Q(jUr2O>d(&a z69)%c5=#mpNS1VPZf6VCGqWYHyX}24svS=_N@ZJ4%;^kAKl$$&9Nt4>^QuPLc4acg z##A@J@6U#1(jmb_@?CS?`~{uuc`dwS+#WM^&PRwUn>(y&u}p4!naNL+#ZIl?FlI_n zCb^;^$lz}x68w;L+~$5(*@m8}KuuOrm)JC*C%nPr&IdcAk2LeS2r|U^g#>V&-@jFK zZ6R^4JL>(ux+ZDEBTvb9{m;cq3>1;>@bD*)*$c&3PSpHx7YVI$eP=Ir$=~En*;tvA zi)o4=^&Y`+;>4^S+(%L`+@HTeHTY+Ws=!sRW~``kj(?`E+Uv#g112LX>I;ug6UB~q zIPE)g;l7%@jCufbuK2#-nvk4}OSpirk4les%3K@lZBgi~5h4kR|_?XcdB8@#EU zqq4b=R$9r7{kRHu5oH-EduZM>d73-{^Jr^Fk&jixSKdECdgc#!F2pC>gw!nW&$B`V(?P<^t17hL9s!R#ktDV}*uQ0KcTV-G)9{#kq>T z!_mAltPFv=;@&fQM=OL3<>PWPE1m>%Qr>ENO|IgpHF-pe&h#VC-)S-2B*gC@%QSa5 zJl6H^WiHD&4?Q|LGHHpBBs0@HALa|EOIBnRJ?w**);|%tU8i52re|;z7@aE+LTZ6_Av&+ zLpvQdO&MjCf*|>(hJv}Dv?n?-NIcN}IMri( z_S#d2er+{KA1^U&5%c1C>#Zacm+vrKy5V)g(=5kX>RMde>CI`6(2HSyf&qOI?}95+ z<_+w1Qn{bsk(}S7f2wG?zvfw@ON#Zdzx7_%wK; zcX6IiW6U>HD5K;vReOAlzRbgnJ4zj8UoDUPQi!F0CNykI*&2{aV-{`KeRjl^K`Wo22AbhuZ-18NGnPj)NRkt%891m%P2BQ_Q<$Pu90b!!3VKdG^Q(?m`nf*yeg7SQo^SCnCX))x z+GFQiTUTWQ9e9JjF$UOv$xJ9O zXS^h*lN*YC`Oig}a~1P^KQdG8ZH2a7rxV0oPw3xU(3&phee;v$jQ3IBy`x6&f=_Ep zX|~I(EU*MYlb&Qn_&;-bG7!sls}hEC=W=^ zc{|u;^K*yzCI*>gtXo`kRjW{^FSp(G4_Vo-L&B#L7Bg=|>dBqlkR}Q@er7G>+f}r` zPAFa8h=(mFQngHM+r;_hW~-2Z*cJBMa%5*;G9Im~3VzUAwl{Xe}XXq#%4t4Oks5}Tb8PKhs)TMRAk6e^hx~*zjVI+%jG`nMU zj<7TuFw(wXy8p@U{dMba`3t}3o%Y>pm2|Hp;N(>@{kq#C@?u5)(d@?O@M4aiyPXMcI|J8O(}utQ{t-A?E9oHA;04|f9DImTl* za`|H2@0~BmkaQIyu(0$eCiPG;45vogcp?6E5 z?N9v~sp!{VdOgfIt&GuvPd6&b_aCyh;x`*E%ExcUh5l+^W%f%G>X(i*3Rhcg-)Q`o zVbAzNSMjW_+l^;T8WvUu2@mwjg+N-9Qh4Y;$W{7|fk}ys&f8~y^o?%a*E>a5doViF z$Cg22X)6JD%qXs*Idju+`C1cdr|xA)x1UzK6~!)XqLT9t8ya}fl!jlKVy7;Bk-o4bC{XY(S%rq7 zmHou3?blCwqS6L(&-PP4w7=3Z)R8v#v@jq<#$ucz3?4rrm$wW{GskU@rJsDm@cK#N z<2}aGl(uDYDeQr}_7O}9bv+&Ox-WC2WWIY}F`|COj$OKVnQZUuWZ0*V{|KLyK6}cx zq*K1-gjHPB&?oVLwj*uL0iDxLFINtvRJxrdKAg|WMCW%!TK)9lMH}bI6vocBMWHU6RgF*sJ6_*&{si&d+^@NeDZ9FO^qAbk-&?6{#VZ$y!hPb+if>i z*S*m3qu_mlQq9rNnil3TjWg6Ul8|%w8?qYqv zBV$?oo)72WOFx4BRTu48CmT5cN- zWI6n+@4t_sD6~=3d4aOuEv|UEqP^0B(WJ-!z%Rjc4kXxTNx>GG~TEOFJLk|8KzXp8@QJi%E$$-`#v(Q!^A%{9N&f zq5pNN^XidIF%IMO8{X+Z`h5y}e;wZM?3o@JmXk`GQd-E&d}hG?-MzS&gq0gtK3*#m z8ZPpDFG^#0%I?M0eJ2R1-y8E=yqNbCit~!>HjP(^E^oIP6k@XnslD7AyZP_iN&9^xB4E#%0clmg*k6wt9B#ZM9 zr>8$p#Y?5{uLN7k>||I6{ViM{3P1mI+1!6D(4FkR$i`+rY(hk^SH`QbGQkC-LFGA#Nb$J{1%EB*V=&PS5twxefT)O7@>DE=-T-QT0~DkRtZl{k-h zzNqv(hpda0t#ibWJRSy}(d#KDcAmnoRO}a3-pr5M)D)<*%Qm^aX+y)K0g-?5G@BnKb2K zv4V?zXxfip^{Cva9(Dx!058XH0 z^`zD~ikL{)$Cu?1I*`r@yP)eicBnT3R}tt>zvbGIIohn6Jy{%l{<)6 zsS~BgxmI^*MfnR^f;7_5eS>tqzf)_kJg*547z<2!|DJYKq41=y!7;m6-^@hnvvin? zi-J7dLemzojO`yY@UPPY!bdZSj7X7fw>`6!d0^e*Eo2BdZ}!|k^&py<>;TgleU zH{SLFl`P{cYZfuG9)Z*-*YWDpqRgzhx!yT=Ot#HqO zH1Wy;+YhCX6)r#y^!^Gy_F;-2w04T8=5SWVXD_%#CC1nfQ3; z8QI|gelNlirn`J;DK#3htmpTvONtlB{8Lu&C&s?+QE*!Gp5d6E)c?uQDD3@vFe3C~ zrySgbsr&i-<~feXZ1lF5`_svdy}uK(#2l@7&YLC#@l+MCImBo9Ki6Wm9b`^@q5K88S<3maVY&~Td&ZyJC#D*o zv*?OmuwnGK{pv)+j1?09^mB9A(|hlikUjKnZ7*Z4Y_=7T4s~a<$gncoJNNMz%F*#% zzb?scxNCoRJ=@Dzo?kjGZ$su76P@oaki;BODwGi(j&I!Zwrxr% zqAfxklUUhSxcqqxIn;IiTTakvdtS(SV%RYvX<_xlSLg9DHfoFW2ye`^wVw5;S4Qg- z9@=}bMWqWcdv2+>(Mz++veL%IdVXVmB1`V~$>)<^2rpqDL&&k=B3_M?oQsYMx5{7M zpt*V@v!sejB&*V`ZA4SVJZZXUzB{&Pq)jMc-~X^^2+eJVqI1jEPl&79x41;ug}J-L zq<_Ec)_Hip`}Sv=CTc4MCo1wdzN0rzRZg;76mPE`k!Frryni5+U(SsoJ&q{UbfG@JUWcK+DixlMeMDhas1bhgMulOg4xoZqx7oV=#+(PCdVE&nb+{x|U@m2DT8&>5MU&sjy9!olFS&qLA99uHw zdpgwe92z`VKT51PI^kgEUxzjrBD zDi>vsE(&F4T{|};J7n*Wy|-%=SBZNmJMN`$$;u`>>&ocLy7oolW?quLen+40&maBM zIrp6N8qe3``FK8_&*uU(o+L9^V>tPpQB^+} z{0G8h*DFl^_~MV^VGO>G7FM14rcbl<^p2K#qX*g-1c8APa_#*U>MQUZZeM&pEr+4+qLAv$VGki8!5VBOoV$6Zo2hD%Bhz!(_wd_1iKGoS zCmMOxbUF=Y?pae?=+$dP%qQhYIncEjc{`hQI&>HB7xZ4b9ulmoOUiLJFxGgWv4E1# z=yt1k^h|UW|AGtuY-@A>srXd)uS-|k67Ks4mbJ7RYsGf-d}|o4v+PLz@wi_n;{4FR z0OdblDT6Vx-+xe#P2(AuTV%Jc6JO8idMghVix%5|VpZS0u^1$CvKDXhTX{7uv``Z+ z?T+P1pl$y+=wUJ0@rKB{a`JISH^X^UTzlp*2@~-yP>t^m#p08W^WT3le10;rp|~eM zF7NNrSXi)6@3IDU(%3l4@@CueIJa;u} zJ6r^V?7hfo$e{SAe}i&hl&Q3Y+0ft*hB^!HcJ05gv&XPcs8rPj7Q4SeFGBWp|C5hl znEyroeFWBVzDYZtcz#I#&+@Zrau9Ax0b2|U{P1Q(XMlCAG>1P=9k8hSi7bl{!x)2$yJ|g1$Quo6EyD~^(x1? zF%04Jl2hj|vp)!YnN>7Ydt9z|o1TFR!P)QF#%$-DD6r$j&~f$O;OYQSKIm3D82U(I z39=**-&EUHzs)gXe*d;I;5+Rj=9SAIfBpZLZpcO&ztr#L8WchPNhdMSdv%hS+nKg` z7-oz*sNP(rXZt_o@$WZ9aLJ-{-o&094;`o?nYMY+goRgAKJ`wr=y<}vCBpch7}e&A zvN{^?4_tG3En4=`f0qY)D^sFNs29)c8(s*~6Y;!Mrh~rg3{6@-Wr5zFkalTn5{0hf3k?tHuj0!2XiA?IUA~*MiwxFomD1d~1;Lqx&oAHIRo4E7> z@1_R>sc>=Km^skh8sW=00HmnpWvZkx3gjWz5N zB>~Ae3`+jkEP4@^Zqsfu;>x?K>DDrpwTjvMaQ=kN+H=roKyX=LO}mv~kvnR?f_e0o zG5)$8lIr8TZ$bhCVWCk#mqNJm`PD(xNEpS!>du4z&-SRXmvIHZBOa-#;_n@Ld9&5pKK1W zDFjs9GynK}QHRa$^QK=HggN^~z6R$c4|M@?$~n-I41kaNt> zpx`i+TDy)5xx7xtnC{>LhB@8_DUwf{uq4}q#r|&pGg;F|lD~WYR!t|s+8@(Q%#Rxr zTISitPnek%PM3U~O5n&xbiw-lC#49oWXq+vnQac*^XtQJ{<|K{Gl;FpPm#*aVb&V? z1R`AKj69KXO|q>vMnI=g-1T5h!p7vEzK#d`$qttecZ<6iGZTGhc2GIr+G{ z`gnHVOyOVM<`}DoAY%g<1pUm%2vvQ>TYuyrH|c+SoCjN882<+tux1-1Elw{rhBa;H z5A3Ungf^&+e;q@u@PSjC^Y;tzUEl~ziW#UGrq=Z3w9ba_0b;c$KE&K}3-~qR^7)vw zgVfl{zKazkc`%S`R-@jrAj5g?{HO}-`N?OxwuKckTi%N6s~(H2`QI*h22AWK2*A#$ zA^VdVS6@b>z1EgRvy-_f4n`;iSuO-V#{69GKfq$-J11IO#Fu@>8IKV@Tea`IhYbTy zy$}=ztlKgPq$s1DWkxZTx_S(}beRxd;~R4k=IP8Vjsw}U)oim`J5s!0}dV-r(8E0M)p~qi3|NWWtlEZ z`WLsIQv<+#kk}Iq71?1vd$QFpXES%QuGB672*>O0irHxJIqKD6tj*%b()O{JCwp?l z70BNIwu{=iJr@JRWPIhvoYhQOgskQErv_F}DMp9P_XyRIcM(4x^Jq37WH(KnF_@&7~ z7_Zg)g<|RPt1ptL57-GrDDsl(mJJ_3J6&L7B{O(-WMKW{ysJ5M~61sbL0G|HuRY1UedGf)W zZ-1<6N*C@AUM=4q2J|Z1AHFn7h!T!opw;Fjf@tso zJ#b(j;{JgRY3Ea`!&$GC{kmb~M5hhG`@8l$oZwH`>fx3YDcU~oG{ z74(^U=TN$X^Ed7!>V|RdzBs>%zDfQBJ1rZ-eP~q#vG~F3k;Oet@E$joQwnp z>I#z`j_H|}GC<8$I|JyD8^dRn-Iu3|+CkOoOHZU1RcJ3HE8$ajQQ^<4{Loeyp2d-?+L0`RrwpGMd7f(cgE0fZ~7G1Rr6Mn63{ zsaQ)`jj|o)Tz(>r9qh9Tx{Vxy#*$4dy^!FF8mLQ_Rsy$z<0U`{GkBTCyv z`4by*4M9jMEJClJ&-SJPjEQ^gH?0=)JOcEY&xmbahc(b=(tX$?v&OtB$)HUX9rn*w zN}Z#LRBnU~QYEz#a4cor?MnRdW@X}^B@!o}))d#IpbRDnz5EW?N>o5)1}H13?g?cSHkdC1@euoA0V>TCV3W*k)6kq^w@i)tgftfzXlbZ3Q^VhqP1IiE(FK{-G%Iq&*N`M(a<73g1 zM*Q?ZDOgd}pE1-*xiM;yRBhZ!IWcOq2^89id;^^pQs$P+ezJw&Cv|}R_7nlOfFKFI zjdwQfbGg^2pK6d}@8@+g1Ls|dd$7EcujQxj`k#-8&N}m8ygNcWEmOi zjbVN7;QS4*_c~jz>aR|ktfK?)GQ}7%yg3O8$XeE$-rq_>OhiNm;+?qpZIgX^DX>WW zw)bGJTU?y+l2vXBh~y!;J~LB-Mxs$_4d^5C1n7rO4UoAd6~oSkMwLY?MX4GF?wd{) z2{?UMV02Lac+-UvYMhB{)Haccg!aXA);y{EsPs+7vm{L)py9DddK_P_IGs2E9vNie zNk(|-y^62FSv6+rSq|!S1{!==($V>;$&q`IptCuH zhkOX|#j6RY(iA|l?6{q)Ruom6lMypuq(G3!o;8Ep2s$n%dgsb?2WsDKMiNPQH)K_> zlOGkn_Oh+PEaUXa2B*$0&D|r)q-RClDhqDqgka8NH%F`V%T;q>4?%#eCW?|4c7xkh z{w~8n2I@nu3})ht4kn_SOsA@zSgs?RL0-Jy7ZTi4Cg9!s<$1plsNim2rFEa4 zC8{})_vm(fW5tXe?JNm7bqXRuBf&B>ziR>fW#h4!k*RJ&3aecj?y+XzoyjJcaX>8D&m;@?s#8sbsI=R5)4h50X19%r^ zw(Qji3q>3sy;Qg6-bYqVLV%bV9RvZ!naVhi-2PCh7X>9w&dzT5Ej7*@@bo0{gt^Ao zJ;06BloW-C2)xQ&_Huh|=R2OgOF3LQ+ax1(rwaDu^mK(CqwAGYd0X2Gs+;S(ol zUA)AQx!r1|^f=6!+JQY}-X(M=tm%seAKdj~l<#K&%0<6G*VW?nbzxSCr!G#Y2l?Lk zrpf9}0f_kbk#tqULPwOW-n3@oK}`*f$l)7;YU*4GZc>w?G>bWW}ihtAuTaaWc%Jy>$ zl6~VZF|68mEG;@|v?FA3>}4Q>blg)r+MLm3!|3a^TeZyWHzujQ10m$ypW^i1$GDVV zA>l$284NkG5OBK`sR3tcZf)UoL?Jy<=Q7rE95vL&?h47p5Bpxo_5Kc9sGv0-tCWvhHXwPK2k^D6tlxitlhy z82djRy|M67|Kki-VAy^Ws)#NV4}1I|)N7?s~k_zl#Vuy<^2|mg&sDN7Luo zGBY*FtaH^eIp5hYw*-EjVWLGBq$D}XG{)ChBfk=$T$b778@4!UNDTPb{GjOuvAmG# zLe+vq+}5WI+NAU421i=6=ZTVs%Wyb2-6W&_Btw4Une6UN*-j73nXy6Y8rlgsTC{E?DZTx zWLld_a;+M@g6~>AQyQsZONg8Q#1_A!)Y?#|x3-pMh_$!F!~F&rFIE!FS6Ll!6lwnY z$qcr(0viG2KkbLss~czVYhMZ590C_E96vs{LK)yl-yPy}e{w#tnS4W1+pr;I1*LE% z_=lHNQ#q_x$|^t2J#(ibn$z2%pswncFwgP|>2YG_jaUAL zu2`gK+0GxlSTuOW%-KC}By*~xW|0uRvw?E`GTO_JEn9sJ>_$&T0BC#%!@LM)t{FW) zv{YA4wd=M}6SIj&Q&q2AJwMH14Df}jjNDe*g~JR78G2)UiFLUpDtuw3_>;dxS;v&b z;bpG}6U5oGpL27ogTGJho8ZJpaEMyX^`ZBFMl2vwBT`Jx{W5U&NRAjZm zBGpHKWU6sNcJEc@RMO0ex~MxI?o_~dG33dV&6SQ`CP#rBfoEFPK7?g^v?wlobanut- zT(7?ZYq9Ja;F@G)+!B1cSS@rn#FL&TAfwS*mmGepP|P@+=8 z_cT0Ebqv0*L5JKbo=&8N{V>xm=n4vba+5W8@O$&jzzrHQ)UjhV%w>f-!%)IxIC=3S zA<^6AJfcummrQbpGZA4*@TKXZ4fzU8m2gJ+$W(U;Jb_RQ58g!(&}}Z zUhqY1P7!OacEND3IdkrlA>dsdZvaEd9u0|uv(DqMTDh=CsgJ9&M^N2DZw7f`7)ni+DEc=#cIwd|sN@zU0Yl_h_1gx?!DdtcYeX4%gx zbpc}IVh1mtfO6lkY1;z`oC6i zkNGvL&wQ$~Q`T&|t!QLmp1^eGMC*Y7w3yXY3nvSduwafeHV!Txm2OOvT}#-Q=A4~n zipTj6Gm*mA0DXqLK+%tl(_fk9jT$w>_7d?6|ES+4%sB}f?fcpJm){TJ+KqQ8&(XQB z5`UT~cVwvAoTKx(>l>Xfa&dVImg2kxa?(QKQ%u9+J|g2YGe^%w_G?S7;J#9B#q8;< zhM4Wr(0IzIMVq+Q04T=`Z@odykIhhBU#Ib$WnWv`}$X_tDdD9*Vtg*t{&693qUq(O`P!F zo!DogK|v8e&YnY?477HUuc|k%srK~*F+$!s#-Xday85=a&o4VXnI|e$)Ejt^b#U~~ zvqT+g;T*uw+rH~`+Bp3&>17)R85pS7y8<{>VZbPClbQnwgf8WE_g5HM+wx-76Ck|g zdN0T^KgmXW`n|97Jt~9kxa7*TO868IC=iT2^$PNpJFP`~7>~|%2_FGL_Y^D_BEDOm z_6=XJS*av;4LeP&C76+J91S0t4`bw|nFuw<-k3Ljp{NI_2kQM(>#_Qiy><`^yK9pT zz`oWzXZscH{Ny>_ca%O~$Zy!I;a3G>7GoH(2I5pT?m5@z5Z|UzcYnBW_(c9Y2WckM z2GCT7K%RAZdhl~Bv{9mHTJg)WTx<@o(1R7SNu`XfYt?{Jc@D4JUs^gnwJp59rW_Kl zt~c9Il@Ph^M+Y-48;iBxHIo1TYCZ!{8+`U@3J1r2lzSt*j94E5#2PM9xSlTx|H|@T zOKE5t^;G(h!&esoRx8XjKCWli-iHLB8>cVKPhQxqN2Keg7)e~U_3)@k2^4wmjRyN1 zzVaZ#Nz?ZFoPSp_Cx^hEEPDFzKgd2`_+!(eN_}+!qf`eTWge#7n=%0P`1M~rh}RWa zbboaAe#=Q|!ybxb50HBvKMomd`7nq2Eage*xDcrDrV4$>IT7QtS4${z?rodfz~QVp zIYNiOQMH3QNSks13-!(|wzPECrBvARs@_dCy$qh^Wg$KXZft$4^zq8nN$Gk8fnliK zo8qR|*ABg%iL|MjlD9RMj3U4H{ZVqsE&vMRKVOdG7-V0j^*+k; z@9(eX>3Lb#RQZBPET&|)4}2<&v#!EI*&F&Nr^MaeSq_eHy)iqI`=;0U85ISYy|yxI zZ9R{5`~X7n3vHm?T<1{Ir8mX&NY|uCWJ+9-Y0bmq z*hQ|Mdyyvy+OnM&8EDkob9!2b`v5hoV)R5;CjvpW=2_x$#Fk+MG*!c#47ra+(txBD zv%Z-J(ts0P?@s{TBA-40CO$4IiuCosDUXdzMzq~1RjtjE$;HE!Jxjo7f$y6ybZ+2e zS% zNH}yON~?i_H?BGFDj~aHXR=x+FNWnV7BjC*eNP7rswcb7@O|RZLIfCobmHl;Q-Hnt zq+#>Hh`(=y@547OejjVXn`i*e{OcYN{IsfRC|Vi3a6Qmj-XDc9Bc*X6<_zr$7C!yn zBhS#S6Q_rKb+j_HREM3ddoAtmPCn+Gv(HQW3ZD`;z*ANdfv&qVE`olI1Ja^}vz+nN zrdp2#Yb2|1)N?A#3q@v(2EJZ=vY4oPD^?w?i-1xiD#vlN$l0`UF#LghUS+xVIz*)D zCS*Kcvg@(d>wPax-R}8d#}UF#9-kyi;YYrRS^NBB;pjb`!b-hI*xlG?RiAkkk53>7 zb;#b0U)5I~ur&c*wVRr@CBpX&lLMd?QMQ72k`(}Su-tYPNZbQ7M@P4dLsmaeHmvfv zsfJ8M8b_V{+)WdWJ6>GfhwJQKgCORLUCJAex@>|5n9?AQ5E-I%`~Jj{ zZPOm{^=bJjV0-63gnHmE@fwZ-_V2Bj+*lua8(}k^4iNYS7tXWVAQXs;iCAK@wfjuX zla1rbXmG~-OT!0N^)XG4X2qngw!NM+6q}wpae+2?&TCDjeK(U}o2KCqE^Fe9R0G*+ zwH~=wQOL7N%uY`h-@6XvPg)vEWSVz~+mf3b8Gx#tCjZ(b{x9e6TaoyJQdMgq@1y&x zb2{aeSM1ot?m;yXF%$)W`2%G9Y$r#GK-|I$Fx$wQ$#m5zGuFVO{H|mhC1;j5MyaZL z2(T)R70%2RDTrlD3M0Tm! z5&6~__2}`=UOsD`H~e!=`QI6pS2;|Eqw`KphJEjhcBqONMV=!&MHwT#0h*X1>GeW( zSl?RICUDX?D&EH;lkeXC8;NO3LXd*FfM|#OY5sHV3n&Rg{(@mrX==YhBqA=x7Z1%% z2S+BH`uZu|eDaWHHnPUZbxIs(7ST`BPFYq{vO4cnT~QZ!zmK7DCc2=t`{>l>LDr|! z@)Fn5>R=CR5}(Eb4@$W+8medfpli-A*;Y-O(UkG zW92Ad(t%u(7sl$T4poiLyHf6=J{nn0hcUFVP|aMvzM;Gs+8Gt^Et%#1BqANxOs+`ts>PsgHw5$N2%8%c3V~Tko6Bi2hT>1zi0QNrfkO* zwQU|U*`YyM1?uQL@4$HBl1i-qqP0JjeVAtug=C>8mgd8`a|%^60S2_MRrjPt7_-5#j6GKcI{S= zr&Xrvn~^Zv|FN$ofr!rBShj@jOiIeSrKR72K$`2v*=#qm;p>yil99F3JsNsFw8O!% z#ZkiP+#gD^TtVvL`+(ivh;7Hc-Q8{oBI)uKTy%_i@M^+Yn=EIGET>8`7zsXH^xSLR zKlBRp{DhKmwz-`y!qKId_}sJrBiQedCgc|~6ZDyp9!aYj^^r|?1h;E{&TMOFytyN( z%C0B^C9y1bCxtkcJ*L0Q1OSKszqKnruS8h}ej)8t@D_sfU^43HPA=_4==Y#TUL&Ew zE`UW!`NclH0zuN={aE#|D6+bty0vdtDo0O|KTb}2CI9_XeP%fFV0>*+q(D3&Hy{fe ze81sX$h0xG{$Vl%@is<1kyeAV1j@#E2SGd3^HV@Q2O4kAu29#RDqPv+|DZL}T%N7q zo9VP=o^PMu=?iwZt*BBbxtN8bnNQA^x_PkfAnGbl(P8iA#`)Y`v24B6g|@kHJvN!* zgl#1oB>x@n{wjg^1*p!DT&8D@aU(|4xi;NFQ~d#Fw|u1f!^g!rdJE!~eECxggUyw4 ze8qaHSbfQ1XLn5CZYOg*HA}bL_rw7BR(o`|a9QZ&x>eXQ>(4KbsE+1m?k3aqK5k6u zRA*E3PTlzv3besVe6DF)n|ya$z2Px`cRGs!zoGqve(^X<0$Dq5rjjiW5IYtX0q*AN z@pc_eG*EUSwxeyX1v~()xQ7s7VKkQ=&Urf@N}o==Skwi%ooMxQDLl-oNzn5=>(%s)I|n=VxeWqAhSd zLIF`zlZ*zzKZ52u4O&JiMUWe+`Lij7#P}H-@n`lYv`a6RsT;`+14Dq!KXClB5lf!? zY4s9qe0eFS-?NLSva4pav8W3CM?3d7kn~P8);l?nWY)TuU0%%jlTX@KCTO=2j9hY_ zZ1$24-!lkbx0^O^x44ZHb%+5YM6bp?UoZcn)2xE)Z16Uh8YM*-&bBT$3A5Y+D-e!3 zAYzzrGuOw=w+ANQ8K!jElT2sYzVQrM zk)(Shl9LQ-`y8|;FHX<{DG2kNjU`s_HNtVG$1I*$&LiOQRupAR2@M>-YYpbjW(fulobe3c#I5K=iwK4!upd zI`zTFt?!N0A#@;zq5x*^o$TNoK5zDySm3wL(!Osi==s*+?X5o?mlKlM90mKKv56CD z2dM04ICsl{(nX|Bk?KSNsrx+LiyMfXGozph6pUQ4RRZ;YI2sQj@6AtJhUV=uMg{$b z@=_d43s|ks(r5SiO?Wfq-(RXduhE10pH|0h+f7!u5-%(Tw>Uc#rXzDO@6 z{Yc%w{Enz9a%)1+hE!5RII6sNRLPH4G_r}|B6v8OW(f-rzj)2lgfzPb zw@AYI$r#_FLY!(ELKUe3t44g|zZH}FyX}pZbL|h#=>Vdkq5G@FtvEPUTk<=eH-F@; zO`00M?7st-E+`yqt=dex@S3IuH1;tHpV$`oVu$OIqDr9Xxu!r3?TqJ0(T`6-@2Cf# z3D^n*i(C`RF_NAcRi%Z+<`k`xBEWd_iJMJwel2b+JwOZ_(A1wxphrH`X8e*#aVXD_;EgKD!6g)MmFR-L7( zHqw|Nh5dn-09iXd7nB~^_3rUcDBipW8wk{kDu`RP_gHq~9imPj70gqgHf>A}-4r{W zCnwcAbjAy@mQeZ6cZVr%*x{Ca9wyteeJ6sP2euQ!<4YuRACa2&PN6UVgZ#>vEX*hI z7-`$Ei7LRS&0I@6l{KW@?7jOrY+>a_#bF1%Wb3B=xRjJx>{GVu3KZm0f}x&UPs4!9 zJ>|wxGy0!{lHN$~#xT(Vn~pUknRTVg$`7?y{SOFRpSgx>{rt>UrnrZ@?A*N z&W)#Mf^DJ%pqe5@r70lD;oMk{NkPq6Dzdvq(aP+h8uc}0FAdIPB@vwN{pJe8&;$1f znXquGmy)$btgfX;#IFze9gx(xAUa-eI$@#eh`oa1G=e!$41`Tj!}=4P1}h#vEUqqG z^8I8xV%H<{gU#}4;C+uO|wtCi9cXs%R#$LONE7YTOD7>pEMMAyLwQQ4b&;c5MX^H4R=LUos{yk43XM_?*_lN$6((bK4~ zUly9mRm}R9jQTh1<^6Py=CY5Jo@gLGc(XYiOh@lz5A1yQ&Ei7jv2m~+@6OF_M@0!2 z`^=%eiM9wThy{=Okg4U#FC6NOD|S+EB>(A?d#R}ru=nG$tv%4||J5%%&9%GP-88DP z-BeTkSE(!zv)ymTzbHbH&EpHy!sb-OnE=`%Il3)uT*P>0;Mi zshP(#W`P3ot2vut$1A5-LzZ+&#sdJn1~9~jwYIAr)kEe}!@?~JO^t(N-Jm1|3Rk4) zq(Xh~YuX&70dBh~=`Phr1YHJpaM~hdM}Kc&j}g6{IW-)hDIl1K&Q=h6(whVIsH@Us zoYbV|hw)MTYRcC7n|-oKScm62Pl$gX99QlfR-XPMv)A4FB|Tp*-h{`A=QpajIlsbM zQ-!}~>7Y>R1K8q8Z#?gJo2R=nZ@|aFx&SWN1PGqZd!HI4Pr~Fk;H3$zn3r^HDtaO7 zkHQx_asht>|EeL``uMxOry9Ru@AH66Ia(Nq$WI+GFB5OjpKHqP8?Y>&=$}z4i;M@U zo+?2uX{N?2mL&onfEU|VKrwu4 zXu!xgfjRa+|h-=S^aD*zzC6&@X);NOkMkRO$Ct8R!@Dn+A3z z+u@RXNG>5RtJD*o4HdoDTL7E__im3uH;4^8WlS^k$c#BO9B+S%`&W}--@MnC(Nz^+ zUri^YW|8+BoZpIEblx(6+nMw&A`? zNY^en8919zL(^-h(^ZMvR)s0e5}9XOWUb5|%dY>veCZKgH<@VC6i|QUrA&C$p>&cd zynA?}2UP7T)Jwn9oKCGbb_guBRFU|c+>R^6Mq=hFZcp+G;t3+bU_6Reb@I#Lmo=J1 zlQ|udI4h7DF{gGO-n*4tl6jiN%yGED-V`IsmF}g%%=q)L+kUxykP+=)rzp19;)08+ z{EQ`FpH}rmdMi{)#@{>H5vFZF?r42I40!*Jg6&XRS}cN&F5creSLoBP566$`&U%3M zUdtlvjd1JCsq7x}Y;L0U`LFoS+S~OMAG4cn|#WVtW`Bfs)F&NJ#?~Y zRcSwqonMz~{!?FD+o{yq=>XuOaT;rjzF$)}9jy_#zZjN|VL^!2;^Cy+AU`s()zbXL zw7@rbH`!GEg6jUK?`_-4k$2ua9^_~5}o+9E=Pu}#*{XPwLMZwz$k@n<7zL2MXOR6)CCeux;Vi=dlm#@TD(2UE?M=B+=^;QrDPE}1_nC=*0JCRg(PBwJ-?3JTvw*e|pk z25cUTov7;>`eO}qM+RSZ;Z5ySOz%YOWi|(v4TK$T(akeKMqH>Me5B+j8lat16O;xr zAj)>sJ+A&Uv}+%TmY-)m2Hj#Y?Q466xu5l#N~0Y{oU-YUbumG3-fNRVuO$=wb>N_PVzYqSf3KuIY>Nj>F7v9A;0WEO0us_{i$Kc7{M%UHWx`snI8el0}iIc~1 zm2zL13MSg8r)9h0)Ze5%QKhRPz6*5;5$evPkFnJfOIy}=?2*2B-Zw(ACx9>{HBHgE zMo2C|i4z8*87s=_mNrP~nD@Hh7k^cxX)o8izE1IdDlht{T|bldGu(Iw%N1U%KasPF zZSm%Y`_1f#u%L)4YptuU77={HmeA4lU?e{xe5Q+G~IiulT zyPZ1)N)OLcIvKc;fcU@}Q7&RW4MvM-@L^^sKFk__J-O+==4BChGYM+;(Dv1jB*zr*oIOtG1}ya$QICD1J0~jeGR5;406QE>cGZ(>k*#5v;YooB~BcmB=2d;dJoAjaZ$m(nI%+Qhd6C3 zYdx5h?1ip!-m(P<#3i(kj@YUM!tTQTMx*tUt0ayuGi(SOl8$wAYY+BMhT*-{M&eyJT*VRG z{xdTvls+nS1w9Lnq!yAhxfZ(rWzT6hjxm0CW8**f=^%@QNiF`p{)JaSusf9Vd{QsF zayveet{r}NMm*>p*7Z>kpG5LTmvClRQZAq?HjX4o;7I4hcSbadP)3k6WA!zH44E^V z`4zZ{h9D!cax{Ay?V9JDXM`FubDnW3E_E;H88&a$jRf~QzQ&hW-YGT~O^8<=nKj65 zk;EN=sZ+ZvfKZJNsB8aseoM_O@m@;zX0-zL$fOjh2CU1kuuVMW>nDarl>QPS#|j%X zmTdsnR}n8ak#U~__)i5ng{|!M@&vmq9`B5rGK&+n?F7M#Y-?u$u1B88L0zpM*$zff z+U~*0NnN0)b_Bu0y0W&g_$2%+Glxl*#I#5;{JNX%q#PUxr0SL>hy8gg`0J#D-GzpD zu|3@eIJM&wc}%TNZBE(;hW|R<9}Eihv|CVox-)LFcN|MnyR_*q(H;|h`_U%hvg5m< zPl}(`sM3%ku*hNY`8A~=F@o#52<{yi7Q+)?1)N75R7+t$CJk#6Q64PVj4P18%`YKr zqmoYU;X6|^W-M5sfE2$&HKCz`U{dTD;i4nH#2XGK-oL}Upl4{9py+|AAuye+ox~#E z8r^rPWyV~l&HNf_TMk89n*VXiaw_qvn51^bj_zya!x1`sAW>LgHp%%U)a=O(~q zMLxWU^lnK;jT~-uWA^Zrdze-GcaJF&a$L}w%bA&6Z)c8m3@!6YVZb3oh!mNzk7RGJUy zRGf^eKjbC(PmfFpH1k>Bx6Ceo0r+&KDn!~l-~HC^Bz1~^?_^mK$ zK$VfJhyPCC5^1(0-Oc@^3o~`r8r5O0o%{Sdr^wxIasq8j*#L_w)4b7ro#CL=*UwoG znC)OO9)V%NUGQdCbb(5R1Pie#jHJ7_*HX0F`+MOpQ~2ge8)vkIJVxippWfBIc=Dcu z&T(IK1z14)pySpn0ATkQ5DqJUBIdfHL;a<*$#+a~n4OgBsE=}NPfwI21i5Kd^w355 zM%wH@hF^BeI+l3kO%vJi&wr}AVp&{kL|1-i>UupogNNTFJT`sDYbJGtJQKKDE(+t) zh>pqOYxqL2GVT8z^|icSzPV1+IeCj1P1>6d-@F1?GUxSPU_TnBVhbX80W8k_ueQnP z?C8QrsNE{x=joN{N_N+qNZcqq<+l_a=d6P7jZp+ zKu}O-qHm|rRT+t0@W|*7Y)CUN_16dng-4LbrlulfnLM;{D!KhMTAP4ue2y=EX(_eL z;vlyxnnpAqf;c&u^>9CXE+d$s>WzOcJD~Ga=y)yd5tH4Bi-W5xS=pPKyFZvqnWGEC zvr#j-+DxqBTLHJLyZ~1UC1a)8l2={hgpacTz}$WDyvBjcNSy=i zG}Y;7Jje4_B*X4!h)NS4SaqcwAmHsI57hv2Nfl6Z{*6lUz}Aw$4JmEAEV+wfh`Hw} zolEvftqa`gA{tcK7`Z{km-sT}bs<`=a1uR0u~zy}Ij$FEvt9fXKXb$)_L3aN4(Pic zyZ=%jV+Pm^@rD1|zaTY7ZxW{%O(mrxX;ovublnqboM!`$*Ao}Mu&P*UT!v`%+UmIh z9`JMMmzugW2h;eKBB5g3$(&9W^rt%4(7DC@-QK0xUy1GcMQl5Dl$~lVDb^sM2&r3X zdvy$|RXYgRv9m8@&9A6XIRR{YOqzVR%+j>QCZ~=@Kb(Cho|2Nb!Yz&cM;{mJHVN`l zkS)0j7XR9azd?CxOnCX3jJ>&n%#Cqiju?Lp8R*3NM~cRC_m0!753(>~bYu_LGy{fB zNS4DRs1lB|SEg+*h6{xCpz)h)O=lEH@0(aIpi)eBHW_ez>lzZjAIgsn8Py!@;=LR_ z^O9wB#%jBT*)BdHq@}Q2TV)IZnA88;W%A4u4a(g@dh#t3cN2~1NTz^4xzu`=JrqY) zK!?6|`AJhdx^dzogCW!?%!h9=s)6nToZe;WaV1avwe(@JOnQY>c3oyA?(zD|;)|=no0L!ibs|Lr#q_J`&=QwmgQO^Z z^b0kVVF(|gGB5@zps1a`p4{)9?0dWMV}DUR|5inMhDZ&MQQ(n?D>_otXP2cdh(I?A zP1SMJc_2qmA9XF>B$g^d+FVbUAwD5O+#Qbg;eKiQtj1!~yzR$*ALHNpD~-0>Hz!LKB?ygg z4tb)u1-+a> zzx8Zo!y_XS1P~54a0ma(NuRr4D}v+zj(?Y+2JW>I+6~4$(V_ zTHG$ac14w2S2cR`+cKWLT`4j*(#}|a^;k!3p{2_pA5*+ApQ>+PNdgl999FWHxM#Z8 zBljF8W(fQ>LiFl{5vO9i{BNq!`tDmXU&-}UcxWwLjA^rzea}E~qe_V3DJxWQq zy&Zp?vOBTCgxuUIy4Zs%DWO@llK=jRTt)@#%8%Bs0r7L_zIb%Bctt%=N!m}_CeNMg zPn!>{>SE?bx%255LL_OHXcK%&(Dsou#Z?XLw}N>4jZ^jZm7aH>)Y%$_MIIxC;6p_e z6_uyn&Yc%+7~ig0zF0NK-JD+N^MQ{XTaC|)7e@3ROTdU(kVhIpSBI~_2#h6<(re2^y*V%t_LKmy(?c98g+_eX$RzA62OxYi^*g%>$ehGB* z?f(NtCvS!TpuO!_*?>u4Y~Y*aNV1QqZt$0?bc4D}4%XswRMT>Qt3~*AjYNu&cyPbB+}FG%P;1Ax68R4Ihzh)O7SIiXG;?`p>KzUT$q;#NnT-iyJ0aeK!QVaaLIq>t%Mm z45y|-m$uE$mv(mVw$5@xpKIVZIeikbO+J4>2<5E0`@?_74}T4X-7^)(zEl;O-ZeLV zdAHfqgFiZ&&%=*7IXD~w67IVuoW-*PdtC0 zo=$E~O=P>*ZHAvcr7X4H-oa56hO8Uvd)sh!Tw|1Vv~4Jr!QV&*4^L@I3)0(=2O3Vv zkZ?)!>FKpbfj z%mY9BY-xHjBh-W#-y5BJw2^q$D&fGrh8-ZpbJ?BPdg|Zcjp6__FAeBZLEarNNhCe^ zh%dd-68b*o*}be_@Y)#FC4}iG_rJmK*k`dAbB@blsm9#7qWqlv1z~V>=5IF<-cVWY z$Aw58fK$OexvJWw*<@RH;q6DCmOxRhfAP)_Njn!7uG*{h6{cE?{=x}t;s@Lt8~xTs z(n4o8O+B<6&35+g%0s{Crl$deE*X@bc>xqKrliNKdKIoV;=2UEG!>s8>z~$EB4qvA zns(9W?BY?cr%aUfCx#{_UG-D;yE)RT4@riMt*U;_y8u-s(sJU`~~ZxtL@Sftf9y1ltG!eQPeQIi3hLY}_i|>@- zdbR0J$W=;l>s}Gwwh0P=im#4$Ezl;gm%mq=9Lw!=TfPQ(SZjj10o7Tu16q_H8q@-+ z%-z_D2V$OcaX8!B1M$ZK{46e1Eq!WpOCL!_IOM#17oSJgz_j2-O_VLWy zlOH>ur&eSGgz7JTznN_;mX#8S|t#SIy6>Sq4}M+=--LD2)zfZA_coiw;<$&P0GJpaxlIX6Ie=ph8H5lGq9p84`e&hf^cTQDj z?8_u)Um)RA&ARLPsXz!ya?fWXSwrJ$?iGo}0swH7bxRi=%&x;M0y4}49XL0e=)TLpq1qL4g^&fYm4qh({%Ts6mF1*c4(3|pij@M17d7LDl z_<EO=C?wQ3Jn>8s+47Q#ixa1!l6#iS0CNhwu5n$y${*SEGuXKCsIch7@B%rmr`!dk zU2o2BzXPMZS-#ksx(+a4jsdx12fEV@x~!StJ(XYZP#R}HenO9PHCV#F>~Mm!SB3tZ zf&Z<4ZTx8^Z|IVAQIlR1hYXKOtfKg!b`)i#&DNul(5&-Db*up1# zvOd{oRhesGhA|IHd2^cHdk%2~UJIlAEAEb5w9C9r7P!hP?9qv=o(Ung( zbmOWmQ(LaCuMqzG-<`%sR(XK|J>;tq0ob66ThQxfHUh-QCaO;FQxtQONt9W#Phy@JpEDAx>Tn$eu_X!>Qx>z&+%oF8EjjR7Tfr6f zUaa>9>@#Z9Tx^UiV?F|Wbh(~a!dL^&$i#Jm6ThOz&T7pj!T^;el86s-Jq#h9*X zZg=XstG6a7%tZE^1e^mJnbmfg9cs4J#b&Fr=IGfu|Bk`7(zW(*46kBB*Yg#C984vH z+cr$?%W=ybGcw*Sr*sLsCS>E{vFsAWse-#-HG4;rtJzB55ImjCNZKm(l^OTh-M!E|lKDa>h-sM=;jxkITzh8YM4{tLSZ!o&(UrRPu1fP9kIRsb9;! zqjOTFO@hj_*L@#unMaB^ioD=j9)W;isG3sda*iPV7l{i0wgv1ExOmaE7h*u=p%tnF zIH5k(WN)e6{Gcyad#9~F>H=zTuo~+3v%6Uf7Z(@O;#6-lnksXHmkOj>s#2R=^#3oK z*&d(`j?EQ-!qj3cMYR9$Xrd?8H|1|FfL`09p$(mkf1_Q^w95vxEu~L&MZbSaUpH8} z;hDO&4EG#6Kfn2+{Zh710v_{;H|w95P{2Q6T27D+Y89S0XR#OT#)qV+$;DZ32OyYM zpBk=|#%*#~-^MU^LyMf!p&V?#j8=0E=OiB8G`OTkL&8G`k1^lngQGor zI0Og&E56$qy{keMRqg(6yyhrk7Frxt(%UNX2h4xR}BPv1v(+5sbhtbSeY z@%d?a%KaP4I51f(9IXF44skmjDz!N8LSy_(kNR3F5*KMB?jGzm_Myt;Q%jg@kdO%! zMpW^q$6IX17Wu9DT;`5P4Qvb4RT&!7G6|;qigJ(H7w`6}2uOOMj{4G}^)z_h##wCs zQ+r+p^Sh#O+qCtdA6@p}Ga7m z&dakP5kQ1|&eeHw6lzB^X?a`w;MzTve2lv2<}g7I4D7V2Oka7OvJ0t1D@K+G0#HSU z3Yw>?$*aJd*ZgG4DvDB+^8=_Jr+SXzjtkvgtWE{YLLIB*Bks24 zi79$oWBxQwjx2}Tt`(g>!T}iD_n+u)ec#;#+Ss?fcc0$ZSC2MG^w}Hqy8V1!e&1{a z2H+$-yJuIlJ%5m}NyHV&XePJzpJ?Bil^Q}Bja0>e6|TzaRVA*$JM6-k8xCTp1y||$9er@Gza4{ zX3bGMSIO#9L>0OFMMI93@b+z^)#xbzzPt4G<7wBMVbuPeAt_^*_qN4DmFxj%e4Qsi z{oDT;`aD*$$ejm3%ly17<@>BD%59e0Bmjn8tgx?M16>BGfA64it6#X4;(%2fQ8sIR zTGYU-+pjhyFi-%s^^%H*1V7#t1R~{(FIwGOn@VAt)x-&4idj72x?}WqK&_nr! zn|TX*QAyqMhopS6+~g3MSfyVh=*y*|Z>zy}sVP4J9B$y`n>P@ue*3IX6{CxoKibtz z9^Ftyuzu6!QFiyE?fUL%_)fU^x=PZfeIO z&g0()GnRXICBG$ZSwsjQiHFK9wm%1|Jsh4(rLbE8=SrT(PS=SXL?T_WkRIdc%6E3B z9&)*s@e_?)9S>nT@K2Q4Ydb%Hw!4^PP(V`$akJ?)z>M*w?J7k9c_lx(I<@; zz1uLIx2&ux|CqTa=Uq10wCp}cj5oh@Su&A82!P_>LquZzve-J7eqKf9H%UKY3^hQ< zrhnHp$;N3`mUE2O&B=Z`EoRV{zoU z9%oWs)r7PH=I>rmHX6yd+che#WhEE;=~@13sLdlC~=bIl&WT# ze&Le4WD)A}@LDT53q0Rl3bxXyN|5|*hR^^bh!Nj1YdQQv!>L-w0*bxSzksRKYEq<~ zFcpxlPtBcfhxYjY*}KDqb0j7R`lM?jSD48=_O_xK@5Fb{iprX&060OV%1e8z#DSHL z>6fG`^H1Fmg01{wA#X*K>tYW7Dq89QE6@Bc`p>48^(P^)zd4h0GqZfbjE!O>jw74h zUTqX_O_fx5zww#hJWb~fKD_KYSR&}Qa)yJ!RshY6%(%2+_02@TdT6S(3L~;9VZz#Y z7Xaal?Xh-H+N|LKyj%9*jZbMDOD0zFqEj!&B~{4>s1?CxoijcmxBS7iV+c5~T0%Nr zpqK{;vox#C^RD~%Ynn_{5GLF$b+R47zONfyXf_v$WaS&U90qXKwevMjA=W#;DMOXr zXr9IkJyyE8*q@9TpEjzK;9UyX9H|Df&+&s*@vA; z$79CZIg+~5>&Diprn5g1$jj2@9#9}l8hg;$yf{DF~86cc^Sd99{* zr35QGhv8UE9$5+n2VF4(hzlj%Pp)@(gPf>4iX2}EeCLm}AiQxg?p(~2rQ;l_m;NsZ zq2G={y>=l`QJH^sW2t?+wLJQLeh%t+L}4XZ%o;2B7|zOp<9K?Q5fJZPZ;(?54EFDh zE%Wk(0O_wX*gek+Wk^PtdN6U3XOPxJ2++m8RRX39ir11*i;fc!5lIku%5cZ;0|Q$zcjaKT1pR%2gU zoU`NfT$P_ddGW6?{BXf_k@2G9%bf_{b;hpu@_fH6?iR;dTCUX`*S}GS5(6OQV*_^e zhJD(B8`FyEU}pC7w!_Ft17j{^xM)%y)CM1i_IX-lDjPgTk804EoPAO2svC7oc?Th6pISo`z9#;SKqX1bo*tVn%z?EaK%5FfQk1N@_|F1 z9O4)s63;RQBzn@?HNop&>EQk2cbQ%1sKGAFTUSh=*EP{TonzwI*dUlJp1B*%qn&P$ zFB{LgZYbw>L-q~FtTX08PioxnrQX3O3Y}#g24nQ(@r?Os@d2@cfA%ab`>zJvv(QbB z5?aF})^#2qx-boXwd-++&!rMtn^)dHP2j1_8W6Z*g#i65%3|bt8OXUBCEc;G;YfF8 zX87lox1M+v6iy_bQ|h)z9ke}WsUvhf=budOBRGxX3Hyn_c^U&vv zbT>AhtNbPC5cFr`-J`?k1OUoMl&2K#zu!~{&Mo*jExDbeFSR_Me{-hJFc{z9LWdSv zvmco@I0`+$-%f$=GrUlnWwr6=3&`bAXJkVZFIvt7d3(BDFV$vhGzGLhqXF3W-WFZ_ ziHx+(J&%LHGx)s%b4wpQ@mkZvho#!S52~ci67AHr5h_b{cn5IL!8z8$iRuLM7r)E} zZ;coaU?!HkE*iAclQ6=Up?9O%=X;D&>tNWRg?BHGyB5HJcc082WIUGeRO(wYh7TQ< zwl%a8!Jreii`RqDe$J31Jd7tF2Xnk2JV}c2TVnC+)EF)PyW!=~vxbkzYO}yUFf~k?csL(}nMLgjsMc2~oJ{wTFm4KuHlMHF#(x`9 zA7Xl=K!Zd6fXZkg_G#Dw5h}9<)x#f-%i!W426=!qiAN&)o)u9iobRG4Mh&mN)lPTH z*P}{H+vZ+)x9)XzxprQb18~Q0LzS@aewsKOrX`u{Q4@8|zp- z{7rt2{6E$iwJd^U9K-_=P+M+e_lJUFJopMqF;DCX(^~@q-I_X=_qDZTQzjtHrd3OB zF8{p%y+cTV-*-#PJX4Ze->gSVE;-Q4hU_fQ?FPtj1f#|z+0|5GMI3n_WXai`=SR{; zucM+|qYsL`IicAQ96}C&9u5!0Z&FYGl&I=oVUdcwth}#C>_;Ax0h&xSQ{ND4UmedA z&f59`gmIRa)MD=-@(mft@;cpA&i0M)E!1=nTHY2Rs{XjUA9X=4XA=@!;9RQ&S<+V;%~_Q5R(kDQ8laIXkv!;lv5` zUTEW2BM(VP_z!R5UZa6%H0y=NdaJbz5lo5rwegi}eu@2@(1u0UT-KBH1WBt=8SSSL zk9!}*5Ej^Dd%6#g#fA0Zte>hzfS8fb%50X-r$!m5q7=#So{H4pTDNfbC&@>bB>gTL z=bufi%70B@4T7)sDfWM+K6P`ZYr|?Zvz!|`#p;WoFiBJ9O3)`S`#i}0up~M^mi8Dy zDIPE}GFA^0tF`8i0lSm1n0oj<%CI&adIo+wfisX#a^28Wh&wv+!Nx@(KkVI`L(9d) zTYK^EZNkDyO)m#gi#DjmqImumo}Kb{YLT}h>eS&I7jOB!P$|?T&L~S0 z|A;ThFJNkLN_SbQUidNcIQ3X!^vZb3@t4JC@rPnt?ANy;UHP@vyvhJ17K9~243BX% zR^p62RGa!?Yy3?T8^Rzx3(;$q-W;mkNA=Z_jP~>?MCK5{UO4=)S5zib*JJ3foGxW& zNAWY5Z+KnlQl4G!7Ds`x6#4}Ax?FdTGA^W>f6pj4?eMg9ENLX>?r41Xn)twAAMrFS zLX1UyZ=oLe@l~!c1QV#+1KSekjXh0z(+QW7CR)ThqX%cP_Wx_iFC2 zUJhN4?!EJ!w=6b=@TeB(6BBo5;(wz*&cZ0Uw%-2B$;@79plB4#j_eRoP2oxSsex5n zPjn2ScB^F_{>T9vmF<2|>PS7zI?S|VGwVrotg=X9F@)RO|0gjB;V9R&*c_J*&|9XG ztNzqVHb4bePUkwcXh$JfRC`3!(X2(OM6Dt8Q1d$)t#dV^-k9T}?*o7T{#xxc!hvpl4wbUqG|G1*yp{ zhgLn$Xg%;{7e52rd;o?=K|0x90RGlYca>4n{Ev8^%)Y(y_yum*3q8VaX_Bjp_39wv zb~zTw3=iF&+GM@MGlv~GmTo@PZX4v)uo!*>Vg9bfUY}~W_XD+Ek-DBl;6NR_$xjB2E1 zg^$x%?H_=Wzf3sL7m0LUEJUWNsZH$lLHb(Tfp1Jkx-Yo2Xf*=l{-u`lmx^B%?B&%1 zZ||HaMT znp&v-@ZDA=8NjzzYR?DiL7Zmt?>iQAO$eEySd613XtiiE8-K+Wkr;b zkWhTI&VdoYdRZypdI9y~=h238cw_#`_VC%+$9Q^JeEONJb{^h7g{jGUPgR-uf-a3Y zK?Q11Y{4NYi)wf_2{JKftOj}_l;YuL{*-zgYT_~-c?^qSdiA7NfH*OM$lzN$of_wu zHMwngoXBX8fKvfAG7tNpc;}tszJR3Pf2INhoNErJ0n(+^+uqmq0d6+`hF01QIGH}J z#19@m9g7%8s?3u?^4M@Q`_fqm*U!0 zmNX8BOlf+0LO?8kf{cR5s9M4L>sDv`Uf&Y7kb*4Zb+aaT%nYt2kr2UJ`>ilVK;@~u zz7|wTI?c|AV3`h3lV{XteCl}Z{IcT#NUH^5u*+euGRm>sNOO%#U`=4AI<^f&$IkfF zQTBxKnts;n;o+m1XbJQEC<|L_&QXI@!5i}8pQ*)$U*21XgAuP^raPM84SHCsCy(iY z1e6V1cC+-D`c=u0JkLHnfd=6gs@Y_`?c=^b`@X>w3}&I zt-TsHt()Zg?B8|Q?kop@2?@2d5vFM3IpmXpVpTQK z%w(F6fSNr1I@5kM`FM^{AqHIMJm0|EQ!7w=w;1?z={j-bKN%F!>+_U+nI?y#iE3U_ znS-y&q>5-U`dbxs6VJYJigEmBp)+E5Vwz~u0!7oodZ&HD3|4z?W#YkRqRX7#)>P=4x=kFgb;ZsNN-@uR+dh9BwAfV6vGAdsMdC~0n zK`wmSJ9s`J93sX$cuWYAOjVs1g3+ZW{5;GC7Tq?_Zr;C%fxm>~blsYQ69|?%7uIPu z%^o8d_H0&;??sIBI>}}L8+<2TSugg~r6p)F90H54v;Poj>eA@<^}#!so77H&Thw9JZw4MP4}#=mzI4d`vVrlx2eLcs$MRu1MT8S)nLeaE?5 zF^$2vWMRP4veZ?7(Lvsgm#W)%u$_Q@Qx_asIe?of22!gb@Zvi_dpc2czYC*L9nDSx zL<8W`A0YFkZ);=|lf=`cK0zocVmd|yHY^@uFme`=Wwj2O`arym1KVlsha*xUq;R0n z4xT9~H^82ru(*GsU9JLbJZD*>#O+iER>PlIy#Ycp%hklvtv9oJ<%txOM=XiYmD=nk z0~Ckc7T|>ff26ussK5xh{YkDjHUG0DjTLVmLK{yl=bH5^GcU>e0#tuq3Fat%)lsm` zvh*|8;fQZD$AZcB)@ng|s!e$Qs?du|coL1Q-@FucGozPx1lAK9Vg;w^nvkZi(yYpK@mW7BuHU#dD1!e7sJIzM9aB5;ov#(w~R7K8XO!(9H(mwxcq z`~O|d51VLQXIaZ$0cZB^bT5xS(t^U$<|zmgGXHQmLkKPCG{}ULry3we{AxZ=FK;!S zhg-pjF-8WfcVG2$rPr2WM~ULVL~h!JMfUy?shhX? z>z0__9M5Xzy1HcmF@Zg3Wg(g<<$F6PA0o%^OiM(QF9kOtl6mX(wRVI_sjsiv)*vck z_ZlRa(e7o&V-UggDC%_cgDsjTHJQ5*J4zq7&|V6-z!@z`@jK%2Y$d;#kiAceU+HR$ zH=5aEfkq;A0-Tde%<4kA&I?IH@j)x%A*bhF-wNz2;QXpTr$`5)eoK5EO(}G@^;>Xj zPhgm6!h_pUao;t|e%9%4R@ZPG4(k?(v2IO#US)7jNm!(wH|Ag%4ngTaD_;1_YxRrJ zx4-a^ME(%boE98q3miQ2s9V{JRNW5qyt@v)=vqF1Q{#V|y&tlCy?i z%JLOGY5Sd~ZwDq~Kjh*)f5`cH@F@T72tV;2u%b==oJjt1?>uO7dpT&_H)I??&+llW zdpGn}Jl}5+Wqy7`G#<3>+lAqE4O|OcIzQ{ebmdE-y1On>+MU<37N=L@%eRZm=iLP?Rg|0BUw})(o1@5JYl>`?0yjeDEG5*0lRH{I2iPxc|CqlLBzU4Y_~}~Kxdi~ z{9eC&&1V@$O*BT()IpXU1cH5{^jaL**dWpS)UikR{i8R-R#O7hptwEQ>jZDQiV4Y2 zNx+1IrBM74<$4Y-e&y21?R|AjbNwB2TM>GF9(wtn@un=>rERTlkn!?#`GN!XPkQ-q zk7qPwhaGb~Mvn?z55=fbMt0rRB{k1ibp)Pafj*_mpS<5s__)CydSlC-HewOF#UHYv z(s>R5y;e~^dynzn{y9jxyYLOYvkh%EpudWczuL99O3Uv$&oi7Vf1)jWd%=HqG~TuU z#`P}Cx6?Mj=14r`I795J*y7QcGLdWGwlo$9b21)^0q#X~p6!oD?owN5qc<*0gr3we z%G%y20M`lCR=-{z24P0StNA^wjpo%n`r|CBqabZAavAeB(k$s|>?K7M1%|jvnVE^! zYtFW;`OAB4m+A~L@Fi+`A0q!=Z%uSxKo(*bse4h@b=!B{b!f7DHs$L=y?nM;7~1-oZ})DQf$!OB^sl{rQdmul5h9k zJ~LeK#aws(j-Wj)uVGwQCXzn|2CSaT-yPqd#i>k0NfjYIa4EhYwl5rH@+5>6JdE5pGkr6~C*~M8*&1Tc<9uCx;XE|Ni}37VlqtaX7k(;h$&{)Synn zyln6LqbQm2uAD}kQ6b(@y7K>gT=1WxcO^>n)(dJ91PHM}%b4H$5`iy1>4Cy*DgeCq zVur2$nnz?esXLwSz|z4#)c8_=Bu56fEylj1U8_-#5cG~!x7buRuxR^B#Ra^P?RK5? zb_igpTE6{yju~ndzx$PhY2RO4x@-1z{`Fa8ckVtUz*Yd?n&vqzKY9PUgUW7FJV41% z7FlOT`+dIympH*JWRC|~`=T7~M}$*}{-BD!%X)YeV43Ao)odFyTGjt*SBSK41;Nt$ zUySKU-!CWpjWC4sMII^WVl)3H(oFh1C-7Rw8JJ2C37k!f3#pO5=spfYi(od+<=po= zGE94H=e3|6Cy`s{rx}jlcNZH%_w#PompeHC^XH5UxF%=fQFncRj&0vG3s!R`J1*+^ z4MnpwCiqNa?W3@uG-B6oMR#NE^Y(T<6O$_Ac}{ENJ|pJhYHt)d7y+l7=WJcFE3BO~ zDL0;f`uCU*Y4_^qH(Vd6waBX8w2_%J%6DIwz`kkTb*9TkV3%G*vrz+gf};4vvnJ2M>TX_7PvOWO@($6lCO`>*wn%cS#}A^li9rVQ z3d3T+PoMqo$ios&a+ESx4RA}FekF|Gn+p0 zXI-I}J0FWsd+n~{d$;n}MDlm}7<3XbX@bmok8syb1!f$78MC{5lCT``9;^K_!S#AD z^a_PJt+Tk=@@)@vaQKsdx3)jE*YA*3QY8v#H!ZeRJpD8$W&edwaeXHxvwE1G+Ezbw z6hsD(vQJl8a7H1V`%Z7yT;2d7DJNxAbo99ukA${_E#b2Qepd@QFiatR&5&!}q3JfV zr*@kUH5g-0j%Iz*)U%fhDYT)qrC!rhTfFwSwy`O>H`n_MCLw)~Q`K$S^>$r|U z(i^~B{+sVyrA_^1!L5m|R@2!;KXB&d#O3I)%nlO8ZbJmA0WeEQB8fue6bW0SVx*x2vc6Ytp(Kf&1gKBm*Nyh~h zq;{TG>|>Vp?+yYx_kt@>kDOBi54yEw558ho)R*yxUXIp$SxwiUO)YVj0ID<2tmtM$ z8JE3Hf)jptsHwkfj_JyW5>YWap7@M^Q~}=&`Xr`Ph_{*Id+>qz zQQt-H<(A+B6car3Y&|qaFyPj4xp86@NWJDM?{$SWo@6E9i;lPNk^*?M{epGZ8-xsy z{bn6D)Y#68^0;<++B;9oui&?qKuH`Sq4jPw!F4g91(6wFA`8E zz{fF2d^kg6wCx1VM-21_#@4t%&(v16Y_-(VKbeiyPtqKVzJ-j|9Y_92SF_fxY92i2 zF7bbg-y=ZW3rbT1Ui9owq;l6}Y!3%T!0z=N>UUJO{MmL_*F+Hz`_0GSpNq_^ET2LR z>W1c>#|lxUiI1f$zi__yBl(r-ec-^UprdVh7( zZT|0<1In?9*0Xh2QKx|?P+yGpB2P8`P*+~nks@W6fv6CV}Y0UM(MW!7JZ2lHIJO13anRwN@HhJ9zzRyf~s8ozD?I_pU?g8?L#P)uqJy3BOjG7Rot~>ldNK`17hAd&0|Rc^K3Z+?bF1A1cw*<@UU z^I$W*8pYUr6ijklq7*I=%!J3(1Eo$hseIux>2Mt#)ofVN>sZIKd>*{K@53@e5);pA z9sB`Y=!I{0pZ@0pdD^@w-SkuGvzV`sjMGo&WVYf&mNpcfAA+f?ZK*1j&TcQUTMA9x zHP6$>GYDG0H&==5PVi`<8PHfj^B9CpG5;{OpVy>iNm3@LXurl8mixOfb`=Nw+eMrT zuI&D}_2GSWuhF|0aN(YUnxmhIpQAl!j(PTqPiHHe3M5i?uFU?!CWGS%E_|&dHZGda zt&;Y%nuoz1+n!IK(hvR@?u$CkY48aYN}SKJ)jJSf+_#CUMOdOw)go{Fs(cCz77eRh zp4zQVI`OOUt>eFWG+q+{CTn-34(ZKj_O}D)`d9jU zg&RBEP$yc%>BVR0>GO$iNo?fT?16en70cQs+)(A@P-Y7;xSEu+EYbE+>Tp)^r{aMPU6G;dY! zLQg9sTrb>XxD{bBd9G8pGXCwf)MgIB0czK}Qs4$#mTe>@QAImafxag&Bmv6ceJ`)a zkq{wXlvF1#$MjY+i)<1w){)9y_e~F!+7{*5uMk5V?i@|&_}Zzo$va85)6s6U`BdhK ztnj;TnI}uf9*R0yxlNkrf8j*Kc2vUY55QlW0+^U^2s8Vdd1`cKq(p+A6xWBo{;SYx zy7&(JjbLSJ+?rnZx%2ooB%IR<+$~}qPk>uJTZ;t;QPw)6y}wPoUl`jh=C$hXBLG4Q zz!ES5f3j(i?=nloMQF0qpYW6#5wIsBDOAFerw{zOkVD`8yWI`^Hj!H(wnv=MdinDz z{}Ufg>iXr{kyboC`sC=cuyyv)%c|$_SV!+M{0Cq`E=pw;Wfz`wA1XprY(LrfvG|Pw z<&k7Tm}gV8EDm@LXS9UG94yo$vs;_8bnbMq zfY%_9<^0&tkRPAn!EMg|BdEG?)@$SZ2_w$p>OyQRSU4oB#L-x>wcc}dN_uq7EV3tN z{gPHz1Oysd3peFE2(Hlq+?fVBS8?p0_KC(aicYTHOpi$R-6&ao3Zn|upR*g zh2b;a{8A>>0g8SBh1htzTVW8P2$`oti-8r-IZn0dh_xB0MAA;?hC3s{R%oHoriDc8 zsf&lE^J=mGq20v-O!=*{3?MIn5Gp(<#Lnm6Wx%b3)n-#FUV@+?amC9;NiiSK!Y>s@ z3v2$?=*IicueSLZb9T2{e|)Rvw$lgim>B1H^e2meI#o4B^`JU;(oEW+=!tpNAHO_L zv5A{E_zW{MhxXH-x-R0nW;S4Ub2gv;?_(Z$`fuWwtbHydBLI;EGH=Cw;ipc?c>qdn zzvA%mhu{dQ8`P4p=D4M?wZV52)c_qL9Wl5RTO%VR9P{+q?`8fPlAs4bhBq7Bm z{S?O*2)vuBsUkpdYzimHC?OrYyhvl{VB)pMrTa3!OS2soAd+V1`9!OXQU7sIPHVL? zfsd+i$qYEy+uJ5X0mKyE_Z4Y5E{eT8*V^GRSY;BNM0s>4m0EmXGaf4ip4#C7OoT4u ziQRsCT^9eu08iEUr-arNC(SqnUNWY za$ZM57n^rGU%9t0X6<;Dv@L#c@ihl?^-WG&lUiY02@g%iM%KaI+jXGh>n#`EpPlic zu1m`)k73m;D$&95El$+jW7EBn0ilIU`;i1irl!so_~34)7oYV#C+p>35Jfq(I?R#; zy)yYXV3PzWmD#x1J|kRHni2WKhi(OkqMl2iooLa=m1=XK)8MhJl&UTQh!< zI*`JM>)triW+fWaySMC}4ftM~_!vbB_2Bh5&dXl_T@JHpPr4SPO19y=%gf&5V-=ED z>bQ1PHX_M@QAkV8mSm%WYw{ zUCdGg48(Bn;RqgZlxHhqlndp;23A9UftZV`V8L)eUS^K3hkSViUYtrVR!$JpR@oIU zZF^A+;I3z%Zi5Q#X%ejrxD~6&Q`KF{HgFg?|K`HG2cXQ2WmZ7bq;dPcs$K!ant{hTtqJ%n=_44!PyOs2QGX7f)PW)!HT1A5?voUANuYb z5450I=Bt=UA#Gbzwokx<15oMaj$#Aa3_YE!Z{O+5rjsmpUZ=WK>C&iiej-=nWa}SD zf--}dl%On5#&7J`M%HAlti)e%Njo-~m26gesx<2hZv2E?xU8U7t@P9;cl%SGe@`jR zf^u>`P4-X&I-%gQ)MQ$j`2waeFzG`)RvgsB;k8P7lQ`+-$pK}e)grd|n$8-aA~+|i z0eBiw5IkO22gySCl7S<^q@(1&ChM;60lW;S#xu`@dJQI@#DI`}r?uPJif{fBE^yl@ z5>PqT>P7Z5H#)`q2sm>!CyLNFh(cU0)u!SU#_sPIdA*qkfHw8izJI(ZvFY-61R4WT zN=#|l{d&yZX(;kN%bN5*!lqa5L&vdE2ol0_ml?2OZ-Hu?RfBTSen!Owe6m9Ok*bwGdOv??j<6M6ib6nRR7=24oE7su*y!;9`iY5di$n3a#DJ`w2 zy{v@DR35lN+tv~QL)!V4C8=d(3}8f z9Kom|0gtD8pCy_EeP0t*#^bF@-O+ob9PN)grJoiZB2%)Q+ApJz~K)w zWUb|Jvf-=uUr|C*zV=@twtt$D=)K#TCiG?r`rdGxN0G<*05^38~KS}zxT#)x7A z`HGv_y!US11Pr85Go7F5^lf@|bM|bRRV zVazqpPF?L_QRg$OZVsbewMqzWX66jZHzr#@$zIZ!eDQXfmL1CZug8kj&piCmb6(a` zhV*cbA$^#T0GHI6mLwG+nT>JMPY%^N)rEvrtz;i+%}Ap)Bn6mk0GZirwQkn3BH=&J z>1WQJik(f+H{1_^ix~q zRQcp#!{0%A>ok{zBfcJ*q0e`NRwVcYNf0I+8OKMG;*XrFZCLK}0%pLQS1Y!{1r0eK z)MA5}a17zKF5Hg1_cmNEo97$rsWEOrxQ;s>)(bU7(655LB)td>iOU-h@e!kWmXwYH zmUqB!LZm{jbOq?$row0wSFewEUv=V48UiJ>^CFt8j;ZsCYS4lJsncr8o#7Y}>M{0- za&gZIurc=9Ez-}crcxag!B3P~IB|`-GL85(8^Vs_R_5!6r`u` zga^$}IjCNI_zKZ(M8{1z;BiTYJh0(QX`|v9rJ*iMhzIxx^zccs(0;cIDU`O@ z=lvCUOeFLK1QN`Qpbg^u`k_eRB0Q`+VVypv;oen9MgtoKh_NN90R1g=aQU2Gq3`)* z(zJ<&(u1J~Qe)ky+_1wacWP*HbF*$!QX*lM=CG?@6L>oM2wiiioy!bRD#I7Z{*V83C37kq6O$8X4G0CU-B#tZrj)Y$;xYgqk~^? zl?OKhQ$9#QEg_B0DH()zPD9`)(F73$ETa;|If9BGEH&I

NKK0oL^#_u)(^4sit6 zXXJ|FI!+2S>J3;BtoT?E++AXyi39mYkB{5=2uF+WApC1RFE*^UWI38OF?1_BWM$?z zARlynjQ!a6v75<3%!#_i#Z4&YH0eL};--bxdn@|v-io@^{_7*~(0a$6)1V=hz{243 znzFu@`;X2ivZU3FyZk6j$NJi8%9#;3yZN$fdAMFH*=zMniuEl|+)t5j^&#oXsmk_LHpZF7 zuwt@D#2`z{S6O+^*L^1dK0sr+UUrnwX}*PLy8gi+=NFgmdO}5=naqr~wYnLcmL!Dg8Z ziRR6=607WLb-KYLS?&YpWtP^jK8vU3?kAYryL|@fpdqHes7_3;YtXg{S&zy?hMP5u zi`mu>oANii%QxNm7yR2Y41hna$OSC*Hl-NQX(u32a9X2cElReDef~jk_;bOBKQAUU zg!rJq8R1|F7Td#Lsl9@~dgXW!!A78%j>|)2#MfN8?XWnJ#?q(9RZ7`CBmVuDJBJnZ zIxq)qBzEzk4%iuriw|OQ0_=7jaW5;78;*)-_6iWxfdvWbc+p~-(eE7P)s;-hzt@M} zmWJYhf^M%nH@~}v4z3Ky0)4B$rI_tfxx+)$e&E_lxSE4P&)a{811F|klbXFDH0(U1M(J)@0$GPV2;&~R0s9^yDG6=dAFy;yjd=m*SCW18 zB%_b}M4d+k=wJQIWpWKfaal=^STWR@j!wMq?Vm_J((qaj*e%nVNo)`pC4bCe%L$y_ zhm{CF*z+9bJVZAP1fm_PeTU%c{nPg#ij5?We3VJ7y96QGcf{MZK{ZHpiLf-9Y5blL zQkb2k$2Dd*bXiv$TDj=VZY(}ZKJa{)ob^?Ttr4qO9H)BZGpP2LR5dJ+#{VJds-vQO zpYMVqEh!-)ozh)`gv8PdN_T(gT2eq7DWw~UB}BSgx>c|usW;mCWhSdt;$B07_quUqAK4$)R#;iVRcEmeAeCps@v)CfMqf_<0>&{{Mvu6` zBp!(mJN|d6a_#qO--p_P&DhyOe)mA2)>~!y-fQ{8uin+BNh*Q{^USk7G`2(i@k0)R~g_h#>)|mBZ~B6!g~6Z zP>G7E)TA?^s7)(m5vVv2vI=oxX@o$)9Kxh=iY)Zsu|R(k;~>tAQ8*ewk_j}=OrHUa zlV@hn$Qfx~LSG82EThIqao}LJ8DRtl!bwr1en^(BtoVRgdOr?Yi{hjsi^xMux%NQ+ zG=36TjnFHEC<$*!@vx-gD}gzPwXFP-z9|UHhm0nuVX4uWama^q$uP>k2z*W#SYctj zz=awwCT>QGIUub=FDrbk_|@nXnX!)_h1+ud&-attd}#;XRL*?J+cOXU3)1M2kRO#N zfV;+upmOzGk^c+;Ise5T^WVw|JM_6g!o^Q_ncn)J58=@SKfoRDw|A~dMsI+@Z?no9 z(Obdn`)&{4m9Q7=bc{@lez(AYAjanq4J7uE7?#J}a+0Li{wU}726SBK(Rz25eUl8n z2&5YauqIahc7LJT{pZQFuYUiyP;s+Tt+9*|EG!hkc5VGEA1^8Qf~%9v;B-_8xldC)q{lXPOx1|M^q48?B^J-ABg%k-H!?%-6&pojV36gqRKO(sF4d@eaL9LTh!y~ zketH%B~D5DA2uI-rTt60F73j1OXq;plx>%oIy0?q>Msm<(hL&uR_gmk%{ryxO}{eIx39 z{Y%on?QrmS=7pYfIXP5Mp*%jAMtLW98EA`tLn0mZ+zfpV@vNw@XwT74q#62?U12V2%y6vf+U0MO;5i5Ay2GOFMN_G`JV;)BLSe%p!)>xC z(_Fomw`_AAYNCkLXuVJBIeHZCTRYfgQ-kzm%5V)3PM3#a%cZMyyr_Hr@+q){kG7Yj zSo}wX{X(OM z^CC=@!Y0zl47sUDbYcZ!9n7!& zwe>}52nyhD7vISiVcqoCkp_dH6Qi>#o#Jw+xn$#Rb2{_eU5w8NIu5wFh%#i-dBeQW zuZmnCuOU^QW(?6#N`j7EQttsu(6o^lqMZlUuud;v(A6JYTf`#pox*bR2T99o{TARu zWa%&S@=SaymPIG*#$=u86c|8K;NpL)WEoV6r6g#eS5*WIrS$K|4^5sE1jl?0dv9gS zG0VB;XKwsli`#g%&a4oAr%v{;Q22Ngv~pu$Ka77oM#K-zsWIB@X)0yS%yuPDdDFS)RdUMl4s?XaIgim`A;>>9?DwytSKfV5& zp;PvsW3NuacuJ zvvu8V-N*-F+oWWoy4I=!5KMs7arCg6eedsoH=z12G;HNGrd-_M^9B4-Z}wxj$`x= z!f$-mAYeN1?x7wM(>*_e#M-&bK8SGdIEd?q@1E8_au!+M5uBZ%0#m5>_;47 zmKs1aQguWBelhZGrfs{nqrk*(Me=0Gue0O;dl1bh(E$+~0f1^9^)7~g&&xx4gJS^& zq;K*ESiH9Sj7sdj6I)f{X%K4FJq%iK3SgU1fw@VS@ETB>560iUv&MV_e#l;|8*qZ!&CB$X+C-nz5d&MvH=5c z*;09~#5JqLZX|CnL9R!3S|{xKOo~k8{w{1@kMGl`+=(cI@SL^%>VAZJwwDR&~om$ zf5&$Dx&Ry)=4Ia8fV(-2OnZ z@`$@~&;HKy)f2{puHVexo%R9*6eQpo>kVv#EZy#V=w45GhOa~2N?PnZOP-Pww>D#Y zgh?=Nuu>4Qoqzv3O0s!x_%sDHQLe;~`?&1b>35*g#o`06N76 z{Ykcvcj&;B?B!l%)5nDpA-cQJ`enYoPfPm3a?>S_fBwVV@T399K)sE``7fke`AK02 zwmc~uMiPM|z-CPPIZXwQotfEeZSS+AtdsHPUfEwHRNV{zwaFAW91M!G} zUi{L7jFjqmULImplgwk1ZBD5giu zBDNaa_H$fci`xkVy_VxEKtvpN=E!cyH1jTPBw#a6S?Xd!Z?p%6wrnJdv#kLRs9vTdSw500+K5c)rtEQu33}Y$+;+Pa8sdpkysvqe zo3IXcVj)ao0pUxbvJ}X)uDr$_WZ-lpbX61fi~kPc8C>Vu9d;MxZSAf6?2|j0jBT#aMIF?s$fco08zIQm+0#KU)iy0>8Pzts$lQ3?~Gxm`*dkL=?580g)k z$)dt!^xOGYYYs;VQl;6 zYw^`#R3bMAcICngj+S<%RdfO}DLsAZya@V?-DxJ<1*66dOQ~T{=g2EA3z`5-P8tMmOUv7bto{q0^XW?U4)7ZpuB<%`pq6OB8f`p z;#+TddF5jG3hs3NscrxtkCh0*$fxmZ2rU^Rg^`XttXuqRokQ5rwl72>Dh9S8WI%qN zO~(|~K!$WhA1ZNT^EM6aQ5y(%gsh_01pGt=W{|_NcOBtW31TxV*F2vU(k?g5n-)X~ zdO7Qw#-rj+-jfTQ2Xs2YMx6*o@C0HCV#kesk)~A`BW~-lZ>A#E@xNThP)$m^|S&89;!K z$g?mcg0;PC_>sfIKS)U#V`{B#ZIsZ^DVEqhTbVg5c=yDG-K~GUm(W;R-d-y;H(I>| z$>Yq}a3g=dhRmy`_lda5k?1msAaBBZHIHqDEq%x_^C3v@lu#Xq(Q>|Ttg>ZWx=EOS zZxbsaU}qMqyv+gc{nU_Y?9x%+jB{Sr`~=9?ePe}oH87TY%~ERYx<<&V?Zpbm(?QJl z-~BH=G9k|St<`{1q4-p(gs8_s$W6q+=%P}6<+!w-763*NZ~Y$78Dy49p&Ovtu{x>H zDtgW^No|{k!yde2R_CV>C^QG5HXo}3wYVy z4<`dbt|qoLc6|qX*3au?Au6i1a1jN*8CxKkwwo>}*X>?|e!KtqU^btQP!;3Xk(fT5 z>8eStvgK&^(y3y$RjG^jn@``@@_1FhC}K%Q=4q__EHvsQypl966J2%W4Z_XpR$VZU zrQOoilS7U!W`@~#_CCvR{Y0htLV4wuDj(win4IkkbSw@nThq~R6AEl*#GekXqU^3x zvtRyur0JJ+W2B};q?xz-Ok%r>)F784L`5U3fbE@^q;a!e;4j;9lh*n_RP5STVG*^k95`>e3udIuk%J`Nv^kdJ*U<~dek_)&=nQ>%w-(-@q9~`_}7`u9DWg8bOJspI!ae1U0A){il0M`(|!M1Uf9bo zU4s6RlKm@>Q%2%f|KjuX>h!mh^rCx#QtdX&@it)J%gAd-G5c|5tmPu}$a}9+ZfI(+ zdHKwcWsCF?;d*h~`S-YgF1lu+)U=ZCx>1b{s#{v?%L`1{ReAQ*!tu+oG&iuyIM_!d9V^)nRjb54omLsC3rKA7f@%l`d` zaP}_hZZ1rLX_hl>a1YBC`9{Xogmo@|*sI^h{?=s~SFT^%2hq$X=n+?)z2+r;EmNrn z46O$)w^foL?#>qiICjS7#=!(q$8IzU~eIPnPa3o3qeC4L5ZQk7GvMVEU2`MZCfR;arak32Ll7hJKLvtWM= zz}M}r3b)ruvPom4DKIk$j0mZ;!KY9xPEZru2x532#RF!6@r_I{|Il}-jhPv>pM=5|C)*#y+X3xAhAh=PdJZ4SICr=aRo< zpP#U+kmYVd_JYRzxEe8QlQthy?S2>2#o1Qz*qg2!-CLI_I`+jkEq@Dqaqc=McEVGi z0oxrglLJ64v4$ThmTC0$OJ1Z>J30Q3zJ*K6KC7Wxt=qNN8Fma^T-T|ubfUPc#tkHa zYCz>FPCbF`1j?i$BV8jJWBe*A7ZgkMEvtDnApF-P+)7ex@%W}XpI1F!l~=TK@sViO zZH4|)Xqb=vEC!?VbfY{cW-} za^Ej+Kk@l|X3$U(OLc#L^!_1MCrvUqKS4c-6)gs8$u`MHQ$`FW8f zJAd6Lni=0(lxoi1+P^^qXi=V-L~1iuvJLS>*vuuxS0jIEEhf2sOVWcL#BC+nOTvt4 zZStMpAyh)weE+56Zuprp7APozs}o>Rt~57kD*QN%Id6te-nTcE`W{^^n{IJH2QL~Q zZ=KqK{aV}@W0c9nz)t(c?#%f2N;q8t`YZhGuAk+0@NAMqY7t%>rE89oi=AfPW}`SxJu#@I`nwDEJ!25h8;w2v-8bv@rnO@AH+4V(9hglL z2U)E!L2@WlSJ$J9lrl>cSGZ&^gr-M#nD+ySa?X#> zbVQbo9-;iC0B)DIbB|sx8IkF)1;At7=-^xh+U1vIfR^2F8v^8RG1=Yh#}g{T1wC3;3@4BNO>%}9<4lV8Ev}Ul!`*h zg5_C~7f-M2>;+f+?tq!2b8&+GLb=}(;Jg16Cr?Oc*aH$9KM)w5JQG|&1$kaaKKiX7 z%Q0OVpAN)=!{Zzk5=F}T=qttLis!~{yCOih3i}^hzgI^PtMC@vDAo62U@q?r$q2$K zg3-MQ$7c-9#v2Toh*|I7uU~3v=K7D~c}HT#+0=3X8@Yt5jAZRxLPist40WsRUjP`~6POGN5It=4 z7&q|4KUnfN9>5!>yjpDDIk{>%{q*z-%6>ZLAfU2Qbhz1GaG(-f{S6>efDZBDW3%U@ z|8-0U9)9QD%5m@0%c6}3Xdm9?0(CbXgN?|Nj?ljpk$)*-y4jl${o_rHj*dmC)f*#^ z)rWM}1w(F^RWLM@cPV4{uphV8X2IV7v=i7V)q|dh`egVJ=R%2n`J2o$`=|{~~W))qQd`B&H8WF3{SI{n(#_q-1c`=i1=ju#@$Xqow_+H}x%@ z{Oa9w-E`3L*UinnezDc0si@zu+nVBW{^wRfU1x@ORD!#tU8g9}S>nn#ssDi;Z}wzOdxXQ`%fWv8TH{&%jCV-oK-}jBW}8XBewsdLzt3aYI(PfHI0d3{W1p!t;B=~Qb>Sm( zrqQkCb!LEpxDZy=awKG{T@4U&T@%*5lg67db~i)2TC?6@{TWF0e2DK>4rlB2Vl^=B zdeHa7rEmavhxV3FpYH`+3_z1r_W#~@=kyk<5E^pn27pYX*i9Ekm-Jq?m>&EYNkov& z+lk`p*|f`Th$}Y$1Uzh_WbKclWG>h*4O^b0NrqFm6<66?`K4f8`scmPOZb&=hP`>m z`V;WsfKD6_@p6`3Z*dZ89~Nm1rgDVt-1Fl>6wv6^_U_L-SuA>Y-8>|*DtlNjI@>-D zTkyXvNSKI%Za2HiQLMQo^k^E?=n{v{3Q*2Kff$T7YVRu-hMu%6u~0VZ;!ny5GUv46y8Xg+@Av!eao-JvT!A~#Zqfzu=VG}98AEw(b$qR>7bUz#0Wh$45=I$eW7%^hJAyq}9?ytzW2_~GpkC8D+ zeKCuw*|xt#L}eflUW&ovN{{}feZ5gx3(T^S)v%5#`D79isU3reS`rd-)7ZKnM-}eh zXv-Bw@35Fxlz#@poldj4d)2O{-;1_uSuR1*qoKq%$u?YcxbyqTo8ax!A(JKUD{R7M z@;)1>A)P!o#rybuwg4eyb~Bh>zvP!ANBN&BC1O}y=;mN;+nh6kH%S3#QOL|pziTXo z%vHGG{Vmfg5e;jl4JFGY5}?LVwmV>nSr%zIGf69OMy1yF<_^~kbPly4mg5}F*556W zxay&K2AtH(;|dj9bUp9r9L;@&~N z@rY|kO!6R=bts=lix=u`EZptp4mTsNd!D(=0T&xsk1}fVU+Iy=0P21FnIRuRKcvGu z&*B4{>AadT915aX>J2y5y;t#4mfQ>macudmbND(=Iw)Pf<&UNkSt>e$d8vQhH32eO zjq;P=;e{IbqH`HrqtDPfYbT!+(F6j?^w<1`d|&zWrnSH8;q)(QyW3u)!#~wY84Q{+ zdJbuzzg#kO{3U#qxJ$a?)3v``w^7!zo921q^24Ou?Wq4@12Jb=h(@FDf^30&v(f|) zkSVU4H*((lcGD+WeEdwCaE^oB-XeZ5uzWA^#%J>+wM!?hk}saLZIL4}V~{8#tZ z&|1~aN6N!(8tB9Y$boWsKiV}*eSXIyMzA1O(5jQIZ8neQ14e3hi-qTmGO4_QHGQk) zypPMFb>1X+T?p|JUS0xld$bD@_I{)WEx6QD4C%dtyIU5`Hr~nNiSZ#j*HYv0>8Lje z6$6Oz6R-!haxPhe@g;g4>=qxSJO*XC>-`S2~FB5-6ywhnV6eqF z%N2nT>FU{|fG$FR)@&{MNGD|-j7W(1Sl4O9c78XYE?j@Z3g~)P-6q8NJODGASiES% z`%`E;#pV8&`PG^+?Lm&LeH_rA*$Vc)D)#ys8Rp!4Q<{CH^TO;@N5Of8G_H;*FPmnF z0>h^?*u{Y5g!0B`EYME^GfXoUQcWjDp{an_rn@q;KFPC+y_N~Jo$Kr#%U~7-} z4Po(30}^=PLD3q{8T3nPh7GF+0TAj9O=j+R=t~bvobbN?NcHgY62P|#{*1ZoqeLB? z$8DKIA7+826i42luI_!%;*qIE&P9~&d7tYC&`1^C4dcP6XzP4$iPPiIVUPe`KaouFObaNZMbvymhI0q+T&eM1D8jt&#pn;a~ zopy)^6c7ht<(GH=4hA5dxS(bGWZsH1NeoQJnB9gLQSYw5e@7zU(T#|=ab4;2Mi36} z?7np_`KB?R_MbG7FoPZ#lcwd*S2g{?T`n&-ONU>ihI?lvz&&Y0Aq`~s+chckjJoQF zlj(zqZ6%3T>&vI+I+&m{9P`03eoTKy9iV2}q`81Oy*`%$)s-{LtZo^96MZm&K!n9Y zebm820fk|Ey^v)FuUDMYJot9(U+hdXqKIk&o2^PW4&u_b0%l&)nj;zi$E_5}=I|Z` zgs<(4#f}Nn@LN5Q+$g3?`@P36T;dV7@kD zHPz>S{^O%vpG#nz|B&FJX#Hi$CWNQ)*wl7So^FU|usGrLbY*Rbuird?EasqeJeZtO zc6&WQk|ohO2}}4D)7UV-_c<6`EENt&I0#V3qLql%z^pX+tN)=TCQ}FEOGQz4;!A-D zf>&W5ywylPWI8Cj+)B6?uaZaw z2J5&~9smgx1ZI)+dQpAzdBaNLv5(YSJB`sMqaw=S)D4>^7}t(yh@WG2J_<7@pIHC* zl0wApti$T7`^D%fd%eKD+>SR6s{Xf|8>%C3N2(MxWks?f)Mhp4oyCWp*sgtjixboH zvo|Qx^-GZ?aidC{hm33+r;x%SpjP6NopqBpu{GXG>R;c&R22bM{As0lI&D!0Jdgi_ z?Go~{r9@urS^OkZ0r+cvV$VfBAe}pj(Bv|q(-kMDBvw2j{@>K%BkeP0h z3Bcb5y!126#NQeVt$a3e1hc%_PZ^h zC$M=}0Q7js^NnM3vuzolI`Rt$bb!qe@NSG!q!ES@7>ayF0Z}*NN0q={)3oC}2a-6M z*TRsFVri!q4-wmXaRS7pa!)H0MaY0WV)ipl{|a~0)$$Ld*@Ghd2ZSZ3`^V+E8E@DE z(lDt&tTD=CcdKYs>07GdSx94iLs9QLIyh(Ou1Du(v4dk-b|`s!)NY7mp?Os0Zt9JF zM_jB)L_c9#0n3KlF|iPHJ{EzP!TN>Lrps-MHZBf_@;lgP-zMKONjtOGNgE;yM+=bV zN|_8-zafRGOq_H%5p~Z0u5^R)Z)?NJ5+E6;e8b)Vcepr-?+mzmFIQ?Z?i zs+=D%Xc&WqV)nhS94YQn7cOsJ)N!GLd|~5A2|3mG+%v}KkU6PHfp1*h{S}mxb(Ay3 z*AJJzPFAprcgN7{7&Bv>HT^hc*#Ta}a*oQQdHqUcJkAC&^UI%vK^I>YTWHP6k`TnM zjrDdE$X&gw-#ZLJ*}m)4Li0Lvu20q8t0%3u-#TBa-H386Y21*QkNO7%w15PCgZyVF zsnsKqnU0b;G`-#Qu%Avd{mbl<7x#TN6A#!U8bp~llG}(l`^56^CIqJGF=j_)QSC1G zp#`nCw>!UpdZ^(!U(UZ*R&S`IDnBYswpduhXAZo!zH10Qa*yO+ zop+vu6tOp``6^G-nZ(bQ#h(S%Bbx3qMUM>J97VD5>I!1&t=5HNnvs#+@}%e})_tvJ z(Yq8a ztwtRmXSi%7-deQ`RaM5oSS!`zB&qzjADw-h`gplx77I8EBnh^j6aq#l7&?o4>VVYu z7NASmzXukVtP=dBByo=uZ`Ony{1*t4vi6Rlx~`SF=^d=g-TDDj`K5BXO_eGM#|uL>h0jW7QO)`O**T}=Eo2_-)$cMdnnEpnvOM~nE)u0-fBx)b zT<99~rjzGEz%Z9?vZ_+ou{$6@O2P)Q71EqTsllD%vB~|mVHjy-*g^NKlqdoS87SV6a1r*EN3CZ+u#K?vp4zmIbTY zjc+2~(j~4#l+H;ko1^Igeho4J1y|&+&U8uL(*5$6^0|l80F&>nP8IxKzl2HUxW|f` zs-&__l4ST0CBU8X758koM|~7c6;0lG8h|z$F$wW$qbi)glD`8Wfd_m2HJ9*YJ zUpKgekiJ`~-H@Vi8xjB+Ur%F!K48HbeAmijRmH^@Y}|NcLpC8CK3OP|b_7Fp2lr}+ zmyW_t0^{VMpTVZwdY(Bt65~n}`PrArwOg+=AQl7+?{f+J3CrGaVUU49V62uXu5PearJ85m-4VJI4_O+9{Ml4|8%mf z!3s!;43CUA;UVb3B-JlYvqy{${Ia-xP85#+ycyL)YZVJd$f3gg$GT4D4~QnXU%-%r ztA-aU7DbijNQp3<&e`o9a52(K34FZ1JMq1;p!&s0Ng{DwI8ffQqC?F4MihGgqg?E^ zKl!0mw-f~OTfLJwCwklh#5w5xtLvHgPm(qh0E?NHChp@!_Uf0X{#8P$z5Qg96iglb zchsv4P7-)e+2!L$L2v<%4EA~ zfT&@1p-2tqX@%;c3|Xn}KixG!8Ao?z7KFj5LvJoJkRhwRQr|J0D=17cuHZ-lc;Kr) z9Jr{HRo@zo9l7O3H_5n)ixQ|rJx{SGu}kR(m^HvGAe<=WuGuuaHj|laWG!Tw?}ZI! zGwM~AZZQFqdNwFLz+30??59wPnep8)n-S~gn&ElR&9=r_v>}<{C>I*sIAWrW^i1MW z+<=%J=S_Mt6*BDHYUQjdd~FV%O+`sgE`tpU&~t2UTsl-zR(71@X=`Z(45|#?J@t`c z%n^kP7dL)%7H*ljA(9_ypDo0+){5LVA?jPdz?Vc_q~Q`uDEy?g*(nktQwH+!e61}X z35Gthy*m#2GF4jt*<*UKGuH>6JFS7;w|$4;Q>cD`&1>kBV4&>KuMJFRut!%;98(*I z-#L5KQPmT%Y;IA7Npgw?Q|nhzRy_{{+pX`75-MKrP}VLP)w+{E`-qRq%%6UtE(Jo99s@s^J0hyh&DXj#QSUgRkSGUT_jwEQ=ZW$-UO-kE@s`^!-U;sj#FHS z#e7jk#MQt;flL1dKrx|c-I0-SV4Aadix!OARrP-5Zh7I-|L&08%25jo<7bw`1@J=+ zVu;jN_yZWKE1%VxbzNyU@XZKKSu?69T=>NCf&xrLyPb)^R3kTcXKh~? zV{bxRBo4I4YMZb}NP^YACR52`PzRTp2w7Q@L1>sdujLC@B6CNF|82#%_pMv{l3xA& zf&F3)&H;J7<0f*`sYu{hsRjbo%-aL-Q5tj!pe@(EUFXZ%orZgtDzO1`a;y)4i7n!f zyFon}@!^{#A4PY$4&C)M^X>`D`MZPe`0V$W0F}&QZVwNSAS%LSZu;`H|BKmO$tYp+ za<;zE5;Q&{!WnI$dYyF zMZWWWM4uxd+Yn{F$@~iJIGhMND-9#jt#0K~CQO1^oux3q*%Vn|s{Hex`h5GY|B9P} zKp0ZOrn5m!+JNnlzttKL1HA)*%81)*}h0 zpE8s5|1whoJuf1p(eK^XJ^>eX6mEP&na|xc(s9|juSC|ds40ZualTJ19aC>ikWr*n zrG1Hr6d_#O*+KTAl6|3SrSu+z^9~5H^izg0!ZM51XC>YdP6Z10+KmQ4c*l)Oj zI9&IF5e3OEndr8#b<6pz-8zxdr~)m+I(IaBQfDTIF=MNtU3p7!B|F^F{;?3{XX)K11~cw*SX0e%R|u9U z?{4u&dVw_^!N|vl*{(<5gUUvg_lmp&!rDQW_625MUnLUW;re`hKA!tZhpEYROhDzW zy^4C;Nfg+*=au;w!Gvdl){&-JPQE6@1rtJ68&~7sc=1Ce2><1Lg#mpr2`ZP6oT|(` zOapI>KL$EsC~GFtZz6ypphPFQH{-wzuYb{lK%rG+cMAO3K(~?}eXZcYNN;gW3;z-H z*1inU*e2>LU0c)abl9qs{PreUqk4=DsOZPfT})$ZsY8_RbY{DGG$8Uh=8P6{fxCX~ zs+@v;pTP<&!W-F{hC&n3-U(BaHJRqEe2Oe16A1;Fohe2g=-LV_Xh&gddJjiIa6=_k z9x%D`hD-!QK|8)oH`id&?rY8wGBp|vn*0QChJ+)CKn5*7=bu2Zbp+bf$~ ztMODP0cKw{{C%)+6vzBl+1V}ljD^vsaIyk>G{56R3Zt_ocEO|JSkLZ2rwF3S;7Q5k zn%!;vSv;M%fp;qWhJ=;FYf@XwmB>6p^33{QYnts3I|n(R|K*WeDAYB<#P4k0SlJlS zODWL6%n2dH{BbqO9Cr}W=NPRC3ciwr6*4jx=9j5}!a{Q5y|{$uHk=`3MM}k!xac3WI(GePq7FyFEG$BBqRa<32{5@LPZ|tUStE-+jvFmu zw?PY^iDRxj3zhjZw2gVp^B=!ZomuPqJ&H8JRyeI+RSxIP#q*z#ljeK43P-H8t ze^euBUwldojteyr%7TlGdWw^|WFo#QhL}UL4*D+WOT$d!@>v3Cm9?DcjT+I7n4{O& z1vwu?8_e$NwA$9&s2&HhAIc>FF`UCyvftK>`1Myu-Aj(AghTU`a2Cyd+TPX?B{{Q< z9^E(T;`GrcZ2_hsH$JR-hh+2o+gfkBWZ)mM-@g`&JCp#_muEz(W6;+&ai>unwh^Ni zQUzZlOj5Vo29xw`n^7l)AGtaC0Cd2IUd6QI`d7Okgayq!8^+rjny#xdqtsMvECsGT zsJ24OZ<-A;fF<;s3z4=T7v}-AX?k!R-&zLmXg;A7m_w*1^gFGOh;8N`ed+J2cI{<7 zq4xTq-FV|+SC=FUotLf}X~$=;Jw5xJ(u>a5!gKSI zcXQbTrM{qjJMy8DLb;}U1|efrwsZu%ACeKO7s71>s#bUG>E8!`Q+YrJl2-^-SkMw) zpKr9{V|ucBPp&Pld6&!$TECw3tjOzlM^#+Xr@IB{AyBIZRVMhBb80O=Ykcw(N5WBukNuGqbHDrLdd;Pv_%Ha_{ zd!WMJ2!p4$@(~0Eoh^G|0I_06a5ao0-NCmk^0~DhfK3j_XFL0)`aj@H7lA-nPOxA| z>i5)yY%$@xW8bq?U&??j{>fwe;}R;T2lz@Me3ZAv1@B?pgJY?o6jFSD6pJjkKu7^S z`RgY^sG3)Gq);Kv9CJgn#rI}zZdFqLN4|pVFEkKYvrqqM zdZYjv4Twxv7-+twFb0#f102Ycj-I!{;bB3iSuGmkUw&v~_7EO-HlkI41GTh=MWUv1c4Y{fov)7^-niYBK8Sq+~4~%XE*y4wQwHcHdS1Z z8OiS^Z>NW)BZOjlCdgm^{ZM47cbH1p`XZPi{`v(J_ke9G5P6=%0L;;hYK=`bV~OzD zoA0t5gR*_faYu=jo6JoCE1@)K($gg{gVWr~r@u~n!pcqg4kM5b;Lu*EG?YoSrkMB5#tp zt(FE)?ygk3y(!NGw)NmYa8lztSVG($ZI5=fc)EYvc-kJYPC6r1Puo=b0}5!JFAfri z7I(psaQUx1`B((4RTdn*8w9kxSY>%=@3JDI4m|8689P`dh^AFhZdP_^iB0Hw7phb7CF9r;aW z@(9kGp~^2SE1xLaJr4eSJ)LgZ<+$lD-0!2x6us{mfHUg4s=t9b5Aw2x=;v9R{KW{w zuWGNlsGH}=X8Y0VRP_F$|2)z;O`c;qS7cI0>6lrOWt$I{!BPI$836g9zz~vfADCS4 z)wuzUgOS7U;}io@i~~?dJ!|l-3|7tuZg&55GZfg;Mv-j&rs(I2*qSe+Xy_8uR1|fE z*V#mp#&tc4abMj_0XVdB_iLs(BXKlPVPGL&0T&VE_ZEQ>Ha=~@_uZSM{iZTe!~P{v z&BoS*@;OG(Pw6>d@ZMo-Sh&!47Ir_W(h`GYF zOeol_7ci&l2()_1UsN-DhF*1snU7pm^dJo)-3UlG5&{AO!q5%Uozf*C9TE~FAl=>F zA>G|Qf`q_(`Tf^=*WwFb?%aFMdCpV&*;~TCp~I4JC13UrY^sY|QU}Y= z?DMs~zX2T0c0;d8E@3FOg3-X(kXk4}lmqDmzubdI;unhb-%R-aX(+r&RiXzN2)iNF zVQ0;o>6ORdDuDV+IcgL)zFb);;iE%~UL9a}Xio`rV9;w4{ej6F%z*X_-QN}7qV zV#N#LfBv}|7SE$lRl_ju%*H0w$&E~t$<>jRXyuir4oDo=ukE9T{tAjy#b9L5HyWeT z)s!chj<3&jCp7;c4O}+7xE0u8YdL-$Vj-s5Nj9jGrD}TrRu(*pIFj%az9*$3*e4L~ zRT>0Zf<#>MrRM+OTu*&8h)8R1O43mqb2&#riO?a;UjJFP3vZP|H^oGSAqeU zze%4cbp5NF<#BymB33ThX9#b`iHfgf+kh;VeRtK;Dv% z+W-nmNAq^Jw06fRiL>IDLL61KaQ-)TvI3jC%*8g(e%ACXRV>h&>Z^1S{>!6e`pg%( z94E3YaK~^Z)uOTTu)}&nmiqcWk*o`|OiFt7lTU@?n*`jdVEB58$1hRzik%In7@ho0CRFrQRO1Ls$A5$WSLck%{oKLn00)b5R znN=J~MpPfGMC zUv6-+#%jd%feKr~s$rw`B8d(;1b_L4o;bR8Gt&TuV>Mky_gYGd8`&_4V>C$nq3~Bi zfCE##0i&h_Qni#PlN5v|Y+UuP?i|${qXCHRH#47y@7#Ose&~9Yr=+|NwqLUrH}oH% zY@8dJ>y)eg9Nb0gjScdg&k*GQjLeKK8d(Q>G0uQ9CUT0w1EO00 z#C`n5cVnN}OzD+edPw+>&n0T8qgc2gP`Kd}hC0!h&s_m7NO?Fa!sWy)+`Jw?u?=ycU=Vp7%_@(Cy)7YDqnx^vaVb`W30- zs;+lD(Alq4Y)?kT%E$bxx2EMZ$d&F|6{*Wi`7{Dr6#eA;iGRP^{3m}nE#}86!2$^2 z94AiP_2Q5**t~uG;FI>p@?Rk^i)C`#gCDieA8nC-ar7^M)f$xqir>W%Ny@b7x)nO$ zQAG*zEA+irH}~hNbu~>J?ctC46hW1Tj?c_QD`=5PmDew^Ahd)olk|?Fov0QT|0hhG z{cjyc&^U%7vmK3bI8;)l1Fv=KRo6#3qz=^gT2jF%2FD9=#TZhGied4XLbHpyY`!D4 z>n;AteAwCQEL&~EQl*&C5G+ zgA{0JT(=iw!~CAT=n|;Ki*L=szV!Dfu(EBW8lcC^mm0Kx|MpCtqk{|A37%l|y{Y(M zzX4U$eQf3UuR6wa$5wL#6vJWZwQYEYF_fD`LQIso0Hg zo{&$q3r`-TAqB2po~W-73Mf~Guh%P4C$|B;LS~s4>5m19#4`eiR!xWqD(Jrv6EqkChwnG{!SbJEdP{+^oN8V&abo{ z^W=L}yjY98ikSEToM>mx0uicA3TiK<<;xLYE1rP>ia7}uVLN`fQHa)fZO!?6;H` zCtgwtT`q+O7xE}sq|5ite!&hD#~P{}vb7!Pe2?71Vw6ysFzA8~h3*DY`M>^LUT$o< zxGte0-{f@v0CRq5`Kc^(`U?rX=tTaTBh`S|!{s$6`Oq8IjcGD&^BhDP2MAOtGGa8? z17B>{B?OxK=!_o~=|FzAlYU4v5?H^Tg-?rIZClDT$Q)Vl#ld39=llHkyJ}8-2T?rH zmSpJia4nfTSE;b=o9N9gAvd8jVI!38v#^PGm7(<{$?L`J(MG;dW9cZW_h=T*)exU?#9FLT>%wK z_+{Osujtpen;R0-^qPx5-XnfwAfW^jQZZA%8nck0!5i%pS`RZYX?D(uQA{93eCp3& z7r3d0A;EE#a_!NW%MYz)5JzJ5rRGOt;d);odud92S#XDq`TzMV+|O)cM?R=sPA z52_A_oOSyyt^6O`qbFE1kBx$;0G>FrpAYl@fkdYd`G4$oH$(-orGUXP6 z{jOc=#r@sXi!Z0wfY8MaCo=CDKpGeScGCJ(9}s@350Hht69=thl-Wo| ze(q!2{t7TCIR0!h(Ex$gR_U{Rybn~!UgV(G;*1a_1#wcK4=!FvzSiCt`WVynTqhoV z1kgugqHjC^ux-6!NMTgX0Y)ETXZ%svU15vp*s1%-&PtCgU~1l2zOAXNghcfp*c&W( zjt;4`k>&p3#)37+iLr}Tu7(ax!W<04?CLXPP*alk#Hd&a6C5~q(387*&w4CgRWvc7?kcvI2e1(#+OD0x<^&^o9`*lYe^~1Oa<`YW zNa;Q{%kD-D81^j3lVOj(Cc=jm|F{rB>2L4mVqI*tG5LEe*_1pCu#3q3;OYn>082BLkuEalVkvAl~f#~uG}cQ@H>Hx?m^O_h{p}x8ryzcE0pn45cLft+xUp4MXO(7*a3E=4X3xasc?i6D|!# z$9TA2?n?BL?)tRh*LztlI3J^pZklk>vws1*2Rn!ZBbSI`mrRv7bLk%$_<3+HBm- z!y13YM7ScNN9w?`IlS&$+;C(nv0EzD_^;_HT15J93#fU6J`GlGi{MIFX#L#R@|b;W zY$>BaB$r0MO4yQ5NN=wpi!3*tXV!HFQaylgkOa;WwXuzg1pxu3$EWFfG#rz@s)Pd? zf7h7r>B2VirlGUDo0kOzw?dJMacB+$kqO|#RFpOuGQFMA@T(bsh}ZyA^98|job=~h zkDT9jL6&I%(lG&Q99ePrw&z^JRM}x~CqfM-;FT>npXK{m3pM`FhsbmSBBAREfwIGq7nd(KzbDx78bBX zDX%2|jf(9vh&OZ~+euT?b2Ge6)as|3>(8cqub)MY=2zZsqKJKuJN1_Qp9jYxV_u^k znV3rHlIYVi%e&g%%>reix@YML@bN`km;XogO$EdaZdtSugjjxxe1cJlM@HZ+n-btz7bz?lQ6t zRc2~^Za270KjmMr2;$HhBa#E){bBr54omChj);EKq8b!?WslFiR$w}I`&TRL14dI3 zhD`kWeb^=-Dv9MmQ$d9;Nndn!H$+pW>&`}qs!xfm498V!AjF7bLRB0*#{QuYsTWH7 zYOJ!jWiIuLiNn#}4;_c}4P&<&+>nlMyC2Cl43O!W=ugsYGA$Uh6Q&Yy4<;5)E=Cu7 z?~%q>x!pD3Cag7hp6e4kM_rho(i}R*^y-Y;=LKm;@zuc+)wfkHnBo>Cc^dL%r^x^6 zoy5Z!%_z_rG|YT}Om7(-O^Sq+g~*j4xy)Gjph+{CvW}=tY`_!_0YM;K-$iQu6rfu~2(Od^9c=N?p)Z6{Q>FsVhLM^t6a{XXVec_Zf?l(|W+9NsQ znu||gq4MvDmzX^a*CkxGg10fT%#D)}1!5Fg_ABX&^Id->t9;}U7NzE=r7m~Py^laa z2y5YA1%&S7L9E7~j9|0-Y63B!UNSD98YU6IC}G`JFCh;JfMmXXk;fmCJ6b*Sh&p^V z{>8CG-D$57M=YdWYE6S&9ybsCHs7Tenu~OO@uySs_3qEK5|{Vv8BSO2O6*!mrX{0j zSQ=vGAX7-b9xO@gW- zOd?q~1_c!6o)!T+#Ei0<38vcMejzTUE3f72-*8l;k=kz`lI!Zdci2~@F)DFDz<^CL zgn>$(hm?(k9@6|V^T0v5C^E_O#J?x??jUMFBaTikv!Fy5$3vG2C5ZY!HOt5bOM16T z;ss50(VnQ6bItXQGeHMNE}@K@u*kPP@V)_0hDnS~=-!N!pFLdrIYIXJ;9*I>xW)V% zV;Ol0X_@aH5+dGfbH`2{F4L1Kr_pz|BZePkbR(^BKB6%gr@^FD-{;@Jk|?9%3i+G7 zN?V~lmg#=m=ASeOgpO+yub2oP02UG-dskN_(&ca(>h3?o0Nb#IJ!Esc%zCJ@SRoS| zTAYcQ5|80Y%{Gy|yRf}I9UkKOXOD~sY`7ZuN z0=1sku*XERh}Ic@N^%*fLJFLA5^?&}J8B-Xo)Z9(0^AOj)a@A==p&M12AF&OTlIAkb-8aU#3WBur>7 z-NRkvK+56eZp^BoH@X-EFCtYIIU;E0a|Y%L2-IuA*u5L0B)SmER?a=SC)2wduO6nR zFomT8&}T%~2(+D#+_4Esi7rLwu*9ooejHtb+7OHp?$XGaWK#vbyYOaHIx(u=;%Ad# z>m7tzDt1dVePkYTWCN7ouf3T8Dvtabdyq9hASs(ktRAN9xXU8>wng1T}# zaQp;w7Q=Zb8A@Midgz?-j%j<$YoQ6jCIlqpBh8)uCDd8*>+Fk}a07|$!)m7GZFDbmYz_k;cW*7p1v!zOQM|1vXJ)))wr4nCiJIJ z^5oNiVUqx6thy?xkAF=X<`4)81d@{ySC5x~h;OLynV1si;Z-U4D8!fo#>rUmWSliW zc!*i!T+A3zK^BDo4~c}UKt>lgz2`(-6^U_zs_MJgQucE?Wv7zrwGG7aqGn?hdl11V zwg!G{dW#ngL5VHja1WA&54fkkXaC}qd6!-DLEg+G1h@D7(ulhk7fIXxgLx#9HZ!~i z$T=9NQ)|-iWGqc1EnT#qpAayQU^;YcvWPUT{qYnB7fLqgXkG%`>3v6IdP9{ghdG8{ zEI$eJL6h@6K70lc zN$`4zvIEp0FG+O>X0~hF^2Q>!W@4m%B9gR4gt+_LZ8vj+q{wgr3r7fuXPILV*vHCR z-Y1vO_kkkKiuHOJ4#kA~kyBq{%7jtDs7@@j5GnZArJ5H7k{&eG>Q^>3Cr!w4-L z7`oX2R=Kys*-}h&)jpKKgNpy>4Y5(+5u`+9W4WcAz(8r0*B_~M$PRa1cc-=r28{CM z9svWwlEOHxUv@ zc|DE~6r}_a3aPP6?abJOEjeG$==6w;55=IK(sexQ&d6n6C=y!O_s2cuTLRw~>*!y9 ze2mchcW_{H@@1_f>)$@H{@=WOq3tB|<}K5*lq~%{<<+OXc@6B%{aVAyJ4<^87H%Wt zkOxMAj@k@mbD&$NB)nhscCgjXuH)(4juj20oJd z=_4iAJaV`Fvn598hIe9*Fp)tF^&L_k1!P3Y5m@f6I#8_pI84s!Q%O-Mi1UxUt!4ch zP$GgtBx*R8J~4h2dj9M?Z!x5nHllHWtMwKOhc)l zO3{74)0b=i=adIJ%0SFt`xJxTuf1KduMmJH`0U`-Kg5mA2vCPH^wRM`S!V{t4J}1F zFq=EY_6VRqp*gnVZ;CXj{2=0r6*~49HSL%bm{6u{(GLs>P+G8|$!c)s(1}NdTuizT ztgh|*GNQ_yjEs`9>0hM}<-grW;yyVahx4keySmf6mD7D6%BMGAdwK&g4p($cP~rFw zJc8rSJUvV@JC5QIKaZ4&-&auv0pWo)xyzNCMCq5BtF<#oG;2W1<%r>RxFm-Mi@F)~ zIW)WIRMU))9nVV>SA2TF+*g2Xm8WY7C+OFH!$4YwXL4u-liz ztFkX}?UIL`A^nM_lcy8y<63WU8FT8<1)OLV7aPZEJuWo$=FaFr3py6|V2pf(WEsEu z*^mg0tB2ML>VSXViwdaUnJtTMZol{%``SLh*>5f-vN?o={^+nj zj<7%WX#zVva*_-L2tS{0UD3?VXaei5;0W3D?cRzRG(QMc5RgQM+;T0Qe0A#189!6SSqS0dc*=W?0y}Tn5iOp1+Z%&V*)$H?M3m1!dk`Tudp+>iQt`Ar86i$rQZ%BB_f~K;4 z2Z|z-Wn5n*7Qw5&Qw8<@nx>D6K4Kx1$3Zacf8opGUpick!I;hRAtNkxE4>}!#+msw z{l$HC)fb`TwsaABdgD*|fD2w7oVlYD>84>jvdf zXuz5&UE&ZU@N4hCjGMGCSf>_0m41F7LXmBHI#XuyxB2>yvz+_2$I0dC^J;Hb#9>*} zRTt$yvBySp;5x3E2Z9GmFm*_Me*kkxEq$7D-Hy3GHk21Hz`mHreY19`4-*+>({z>= z7_AF}NVaX@oRY>B+A7vur2K6tO6EP79CY_^UsX}L)R2=7h6Id9hCZnEh6uj>V&RJm zCFT5s#oEN@LzX@m?mQZW4P4wacuzLH)3oAlXx};yb(XKPrHxNlsF=@aB)A@C<$p*~ zThSpa35F-;1k>s56@GaCI#$hyCYc6bdw-hJCTl9ejI#iE-C&Ib zM3HSj{&(T6a!~d3xYu9^2JlOVrpRbTtDRJ5!e!H+u^ZrVnksU?(XfbdB!E%X145aM z3qt&+D5LLK31L4o;_kwgxG{1r>k-CB

IW;hK9*aas_2)E@ZGi79pG7XGn%x zWfn!s1y`>J{LhyGi^m6q*99;Zy*I6b<~u8efex8AuK8@9DC^|YvDUJ#c9bu>tQAD3 zeQ22iO?`vYWxvvVPhR>N)2uQ@4OC`3hJHk@z<^OH^(U`+2p)GWmEno*~?C}eh{?|FB5n`DWFBhLW1wwjx6lQ2|kh^hutLW_do zH#6g!!MPOo@m&A0#D^H1lVT;<+m~1<*50&@Bd%CDX+OplZ|9aHw_h-pM1`h|iX z$6CW1ZSum>zflDQ7;q}6xk@W!8j5#`< z>F+@oBuSbkgU7mx@ysAJ68>^&LC4v_uSq(ZYz%lL6|VBFO@7A|=rZ^oi}*3xrdyb049j3yBO%!% z=W~5{Ij4p#D_pk4arzHAWZVJLnM@d{(-%rv$oGO*JN4U@RXOl+yid8zh!hw-R#c;S znctydvpm6^J3|`TPdB&%Z5;k*Fm}1_xYbb8vVMzrrpe;LIT8?sw5+bbN0VZZ1N0aM zKCijjqCvsw0o<atw@||7Qy-CcSHL%|| zKqb=V=ZR8CJEHF^azlTjwM(nNLt zMxA|aop@~(Jl()0#i6I@Vda)n7IMQ`PS|3okPHVg-FYri!1T~GFP`Fo;EdDkDh z`O0J3?s4>R<*e_^gSp+qwEj(-DB|~{dExxw>h;gq^;nV16~OA$fNOWb8^H8}gUQL}cT0tqPoUcl^0Ibh#j$JpInTSo&{Mfq>IqAA+v>N4wg zeN~j!OiQcabfE@;UFLNxR)Pk)o+o}BUR-%l0*O&$VXhBU_pCmQx&cY{sqZ=>Yk!s* zI2;R#fD7fNlnCS0U`)8u=u3<%DIiBwXV5tSGXb-;WRWTz=b>GLrtk@CtHnO%z*46N z9pX`M$4JiJU~THv4v*1XGlS}ZT-Y4OxH&a}pg%s=P0z8ohirukGlzMX6~I^nrpAW{ zaJ-h=2fa`v{2|9)+4t$no_ajftq+=SHsq=YM>|| z(ua3D#ZoXVkwXwy^Tuvve@cE6GiDrThf+K&{%t)KKK1R}f8O?I`_Y0Y?Gf?i-(JY& z|6)nNm_Hgsc2PkSYKr>YEQv`#0-pv6cPFDYfXtYB3!;rs(8RKBGju*3?KpR=nEHzU z0qAMeOr$lCHO>*jn5gMA7l_OO&Hz8o&d4C-j&W1unwG=`3%CmZV^D38Jz1Ya`m!W4 zrxS-MXEH)AP5~b2Lp!s3`1?g>RR_0hnZVsOpx`wF=<*EdKQ_*aT<+wj`~F)#^*9~7 zKN?e!=c`P9@-iua{g=>T1r*gDvl4tgW=XRh;j8LL!_tN31Z>6-J-3Cf{{5pno4myB zW@QV|u^n-axir_unq&xBn=heZ0q+Ot(d%H0A*3%QAzct$BW}sev^Dyn!H~owLBEgz z8<5VJSEai@4_EfNjnRATZM#K?dhKuno%{3s)Kt4KyG?d?Lv^+qeZXaOYgd9Kd3tav zVS&%59YdXxaG!&;mX<9H`xSVJFMIGiyk9lb&UKa-Tc+RKcAEL5BubyN+Sty;s9wE` z_*;_`D?@Ic#B~a^#Sx6FE;_R}EK3uAf!-{Ht~LyNiF2K3vz_fzbKzd0)%4?Ytu=0a z=HXroyt4$KmM&RX+ivPN%ewO3w%m$foc@}g_e z4rWQ(^|<1-NFtlV|B$hI7nt1k2jKoT^)&nGW$$iLKuWga^>{B>Yo}xP99VHSt05Gu zW#GOs+yBblTJu5aQF`%yAq1NU=t%a1$(d$^c4`G#Q@5m#8WZ09E=Y9Ed{@(rMO(Z< z9P;ntWO6StaFLyIuC+uUz)SSqaR5Xhvy05k&(bY|EU#bEIod2EGC$`9`J6yX-%`Ng zi+XtYkH$Ln#Sd&RlJ6j_erlpv3ayl$wxb@CwjQdt(2<`vMsYi;!m(^%m8+$=F4s^B z(BaqQ6_3R-p}$pSO<&IENczeOZNL0x6?N+E5Im{3_VrzNdt7^=Eb2a6*ZQFZfHEQw zm`$!HT5h{QBFScEj)t}TP0ZScqAWkd58!d(>YYnj4=)xRgK(Ut#V$)0^Kg0nSokHx z?^KW|g9g2pBBAN{=Hd0cR`*T${lQ@LkZfHt23%yAHN8}DY%Te|qA283DvwphOzhQ> ze=xmv`x@bLws6hlsD!DTgMl|y0m^+iQIZ&#!*3D8(6H~iU>B=y^X}mIdBO&hPtKG- zFGzv|?{*do8>VC5u{704p=;Iw%)`F$y|g|R@X1^cty{ewUG&{P>aqQce5`qKZFea1It0Y8hZh$3Pc&oKYtT)?Y^g@N(1&moov z((bwRi{7S$eIC1hd4M-dpyo$&i;(k3`)E<9+cBUj3ME-gID96t=LwMa47+fo(GoEK z;E=TB9?B{1R4$4#>8q58vjA61YjKdYi`^tk})>(+W+g8$otUPDIzR|jbT zh}QX2iRVyve{1`qCeDzd`_5#}WqCFb{Lgr@R=p||Y#HCR$W{RQfdP;OIW%kZ-g+2I zu&*CF8yeE?TTSoM-|1h85;c-l=Rnl2z{IdG94x0r?_r|Ny9cqlUBOl{2du@!#*x!0 zn{T5xxk7hHo?KydCRTB&hn|%lX4#Gj( zcJiyjks_?T1nq;643l!edXQKD!jd7XLrfk?yXmv>GZIVD9(V|KD<4kV&j7J9ShrJR zmc2oIg#ENAYL%nXZFT%Aj`;Q@sUGelAOS{4Hy(D-|`788Z z;)z5hU^sW!Sa0B5{$#w>f1MdELNZ(bjB!zW;@O_kHHQp3QmBPUA4s!S&|_TG2G$CX10CtEUj@WlUIxaNF$`rHQI zk^Mu}QBtOMiOL22^&B>NSi4fAjs33Jrc{+fPJ?^%e3~UN_*Z`|k3to|(F4drnC7Ub zGHAXYT>n8i<;`c$Ip0`UO)ftK(bHf|PI0;%Z}&GlG{|)^sCY|CSJ-@=L@y3v>=mK& zD%xJ&o@}-9(68Y?YCz81RI#$7ekwyH^*sw=ub z4L4O>iBfVh$~jTf1tfSfPH@sM2Ov{pMPjU*XiSb!RJRV5*RB zcw{OYYT6DKJ%-7eL&+b9yq^C`#>R#O-KM{v14`@yT^BA;UQYM9KR8;%s3_EQCdFEq zxXXGk8!hh=WxrSTr8$KjsD&QJX9&qjFOXHprDk0_sFh#AEh%)h{|@Am!c#a;Y+H3{W<+`B{&UJ8WRY^G|kpW5B3ss1>IgK z9u-F={d?ANZA6lpi9_8DHAXImF zAUz4VogC<#==P?XmKL*OHaV)_3shw49*R5X8$Z5$HU za6x%<)os-s`GNP#$H+)h5ROR3UJ5!kiH{MXFxQVI&OPJprSg`tB@wvYI$m)jKm!K7 z2MudR`h_?OE*ZukMR%^!&Fx_HK)pyR-_P8heODi9E4J?Utt6HK%Kj+X0r={H8}?e{ zZ0dc@gS=(D>ctuna3d8k*(n~BR2+;e6SQH^A;xA9nCIw0{hc_cQ~>e8oLWB3a`GLo z(`5d4mYOY3x-t$(NR1!1f%!M0!5*Wxe!-MNWD)+(5PP}s9SBzmk8nbsaoCk zEAqs%D7~%s)5;K_R#PQ4t)7x+8c0l%w?-aDE0YKU2zDR=VFsNXuc5e#P#PSEq2ZGH z#8^j{VzTIKpzOs~!IQB()Zs)n<{%@64H^&M2*kYX%Ge5%j8@}$zb}!ASpr9rD17PQab2910S6_~znlk> zfEI8erh-$xNlM01wiYTa>P&8KJl2Ft9fVQy%z>k1qf_a{7I=%pdbc^gFY1+5o*cK_ zOLXPnEz&miaFt=915NL?qvwQU>|Ww@XG57k<#BNdo{9cx9$($~KExMvSbnaWEjGZG z?962Nd=2d9nZ|B*14|$7+xL!Rd1d+fW91!-HKJwu&hHMb{J(`+uF36Q+wZ5l`^+x^2c59On%(FBpBBZpueIH0E>q~1a45J zz!F%vK@3zZ-6!t~dG#n>GQWsULTmiOC4t_^;KrEF2R1B(u_S(7Zj)z>;yMmiQjgJb z%9#1w^hVYXGhO|cA}MO);}U@P@Sj<*Fvc3!4urEe*0lkYrs80LM~^xT1hn;0lzfnI zgP;b)hA|1DU*XwC4je2o1YZM#H^gwM?8DzPrsKn%%T-B=fq))pOsl`%b{0$v&%rMl zH7WAR3I0hPmJppV(eA9U&S0aYi@LMrg5)>305t$2O=n^|!>JaO_CjmJeQ-BK6(oUW zUCOP;6h+*1t_rBULsf+{){sgMk0lKyy-M);fo`4(Ii8W=leA;>yvvRgUeE(;yT}<@$>CnHK zBSAaFdrsXAd7eq>?)&#$U9`JkG*R4)(TwHK3|C-y;_F3ye0~N?xW~(F%i-TjNqcH8 zSe%_WhS{4T(Ubyud)L$m%w={MyMbO0v-&@yY4lL}kCA8ih9yId&h$tD5)cBVQqldH zup^$_=AJ6oq*|xUEe{s=K|~570gVsW9T!M$uzbb1a88Ye_E?P|CX8qDGv|n4x-87qonkwu z3!%hjM7{6{2Pw3AgS_ntsc`{_yX zT(_pM%U$Q-YMSU~|L`!2#2(`DA^7oB^6^!o_dK)SW!mb^ecSzaw`U+gnLuC2d#hh? zYzV z9xm~^7LzN4EV<#ttZkfi1WN1i%XEwWxm6T$Pw|_~H+lV$gL&_&hpQD66%*gp4Dcut z$|FqyR4*iG6me_;6ucC16hv%;LQ+B!J_kW_jDj5ur6-9d1b8u%?-%R1zu0Z2W22Q7 zvE0{?4z_NXw!IY+LIJr?+qL|ofB35mxBOM(u?%Rcn%jBc=jzm9GyyFQ_NN=R+hrFY zU>K2O%!9-!biJ{Z5t28JE8_tyBPC>*0`o#3 zV*W_q77UQ7Wbh>BBHY*Ff*9y0Bk?4_*+F=Yjv4p(a|3)ps0(l(fRQc;wuS88Ae3ag zV-?jB^SQ=8W-_xy5@^e_(jAq;ML!C(TVO|q$XGK{_kz(AB@M7R(4g3n{$Ozli=yz9 zTyiwBn8_1CaWMS5Z;{t;7q^J?W@th}fH6L8{MLTXuwaB+!n^9hg#PN`n!%dED5BnX zxp2^<90i8nYw8G|=$!xC+t(lQ`WSqvz(I>$!H_k$W6U8>F)ACb0#5C06Zm_w6$|XGvG1OCJ8%)&v>Iti-OnP(ucgUa@-X zfDM8e9W<>M=%lEa8u~u zPg+iYXZ_X|@%od`ZwprW zHPU25uQ=tA>w66k1jG#zh%%x-$V+8dSjB}>r0Dg1 z+5P=0DhLZ(n20nXf=+(lg7R?WDd*2ocsxnFM9HoJxQYsc2&yhe)$z$P)MS5lz-UwL zwqb0<)PFXJL{ZTE5{eRvJO%XyV2u38(zQ14q*c)q4Z-r#(&#-4a88$A7TzH!R!J8w}DZO4TsP^@-rj5_GLzi9d zdVe;Idr^PBY4`w%&Qj5~5!;4DEC*7A9sx6fhatZ}w=-K9)- zA6@?%pWS_3x!IuW&hjaQ(6wl{ziRd`=`yg1Fe~~U(X$0Ry#BM{-N&Ok8S#-B&*Ak< zWkSJhC2VFfkx7>kT+Lm@ z@5W$5PUnab_BNKSznD?FdAXt?VXhYJ#ym4`2JjOhqOcF)gIO?eP2#1m{-}|jSF(3a@>`f;gt(fxrH=kYk#!9?Ib?irV-gENBi#mU~Xm7D>$>byE zYqGYcK`8EgTCwGQaq{=qn>yY-m(rw2M^M;7^@eq6;Rl7Ihy{oNE44i zva}zHHbu%d&OS6YE3QTh{XrRSzK)0QXN67TJ6MIj)x<>jsP;u3MRCcpE%wjm8#br9 zDTCZEr3_U}EwoG>m_o#6aDfB9X-O!sy+tb4&Y%AIUjG;(#Fe@)F*-`xQQQu9%67^o z&$wIuVE$|Ua845fNr3VS`?JIgO_@asCRSdXiFU56o6P=QWh`Gp~`9wjc)zE@9;iiQ;Iu=Wp3!&aEUtg1)Y3@CzXgP6Xv#+k` z^4jVw540|IZw%nVip$iZ*CS;NC7~jtiq3DIPHl`)8$cIOzq3jjvMTXB$#p& z6=BC`4#9P{{pW3(FbC8GHNtrxawaU@PCLf?BHnshu?wySs^cG`_~#tuB6Pm@cJ^jT$YbIHi!N)I4ibGhk%_D7w_{%Mdi#} z-gpLhPFIyHgS}Y10F5#UO`^7Rkq>&#Q!pqX;Q62XtXOq2{}tm~E9A~AMPA@AUlqIX z7`6D@r5h7!fb12)Oqu+f5nt=zQK#P&e~nJz`s|;wisPRQ!4s(A$ZAe9o}+QO z$!49v6Lh9T0QJb+io}^24L)1CFBhSu%-UE z3Wyo>j?3q%H=ZnX<>)J>eXj1Wil?}fl?0)g5ZB>5LWNT6iqiO=!5n^WN(+d5okYO9 z>H^+da-2K8o!|^vk9%I;izu1#W}VC2;bMQapppR$ktR(50aWDJ?7EI3>h;SlM9)^2 z$v9q0jDlO9tZ!d=*2BbjLCm&JrQ~rQ%km9W=5s$S=H8pT{Lta}3~q1Ei`>D-{;F*5 z==D>~Gbl^&Tb5wbpkg08I)b8d)VQOPShsfjHbE_fFl3;`%D5^X4jxq)cdIch3`BB+ zp!AGfFH=+Xes75!+ywFR>Sj)Bl2M(vKiM}Pn>bu|>7WU(_KiE1cPD-7m zfN)5HL`g+(Met1%T*{sT8Wv*1um~LPz*tEc+V3FXf^+v z;8f?-jd&c6iKlKTsByv|VZ7zq=vxB=2g(53t$P>!RQosJ%-0Ul&j^;9wlb5O?=*sg z(P^%av6Auq)>`Zy0jX2h13FewVn2@G{xzW7)enXEt?NtN>`vV+?!?@ml8VA2(-^o4Y`cqfgTIAqXsHER36y)}%#88^)w-eR{5G+e*ruhV9+_IKS0Z5x13l>r z^n=E+gyFQ35J_-ScgQTrIABcDa$MPkesW1XfjVnoE6p+JJ%0B3jO@bp1n?edREr9o zf1VV@U=?1>gFe7V{yN@Y^!2x8xcOa4-i@G_1mBL&SmyAJ{FK;>_32?D!2MwF;pA|w zG2A;{VS8FgG}O>MBm7H$YL5cSm2Og{`Z6tRh;*xIr_afFG`a2-F_Pvl?}F8(@B0~~ z&0-?TUjXr2KNm-q&G=84s6=Q9N*_s9=RX`S)V@(~(oR7)mhz zi&8mpaSeUpQ;BXXfB*3f)B3@qHvfm?$4fx&wDo0x)4GOwZp8oYDnkEmL;rR||0dME z@qm-Y)8)~=`R)ZhJw4~$35|8Fh)Vs+O^e^0K6^DKIi-&MB{0IS;Q=(QpCTu};mkx- zt~rlJTAUKng%CL|An8n(~3o}*7W`k+b@)2XC~n^kMPQ=&Dw-Z-3# zTy+ODnye0jK&)ts5>a~`OmH6*K_$CFP7?QSJLv_5K>`~58}o~=IH~-x?|2(hkQzJk z90th0F1%Ghs6{6GD;H(Tm}sg4N+GC9=JHV3YNI=oSjM{_JwLYU+onBE6jU7D53(m_ zD>z`VzoVbIVPfP>?nn0MPi2b8s6?Jr$2+d7*j)DX+{2@U@efUvEMULmJqsM^tf>_# z2$&)XHZzQQ?5$k0?J+O{ATrqO7}0w?J&u$DCT&17K1id~y!sO@hva_AeLOEucbl@j zkBsz~*K|a_uLSF+cG8*Pio#?Dwdf;0g#nU85+_a(y{5T`Kh$T8^hi8jv~=~nr(jpE z*$d?_x_47)jH!6cBz_S`aDVoAyF1m~>~h?>z8)if+0wE;<^KZ2B`$t(kMbM_`0dy2 zG>_$O4Pt_Vg2FsbnVf?_*&X<&o~&lhtJLAAb6#567P+Mu`dC9(9yrGg1j@h^Q@60H zt*&>eR$k9#h|&LXk9`NDH{h5cpvAxsiua4cOIUKI`Jy-#h(307BV3iKprGv(i% z=pZ2|6D zx{Pt23Ao-BuQZrBFjM+j=zkv#LI)uZF527MBmEzeseN4@FAnDX)~5V6c2pW%eeWv) zCE)Dvo!Ar=o2}7Uz0y%Yb=|3`g!r5|4^#9|9*%9CS^NMVMJ`t0mDs#5MKz-Dgg)aw=B0YiS=ZaRmx#ZXc7b$*V0>KxcH%D$7gr4z--i~> z=S{2B5UI>OaCHd_YUq2&&79kX3x&&{Z(m0PO-|YxePwc;;S93+7F8hFe;7iz$&r~r zaJ8c#)R&}4G&L#9#Sj$0sfuD6pf7^KF#2A1%k1~XhkoK5`W-)20vQ^Q&Bf92FQ1BF zX>VYfUlffF3RI;ktFZ)5lGfvS`8M1sEqZN=&Ip zJ4WP8`)sB7;TN(XK~caYgCZn=@B^#Yo8{ZYg~M$E1AF2+F>QZ>a-0)zC{*&h>1T5) zV@0NZMBH}Q&dnNu{$<$XUD@2@MhDHe6sUepTjoN8x!KmuJR7hYnO-`rAQk$M|9tiN z*z;T{%?k;9#T%;Y#@N$z!U={0i6Vh|q)?q*TLt(|g-RtpaxDI?br`31{}F<{c!iqI zBos!#dN!}6yZJzaBGG$(|D}yy$JV4fRoY7S)U>DIrxRA|g5e{01kmr3=@tuEYgnG% z5-NM7&CjtxAhZC8+Lxf#GP{-`Dj#wdJ$yrZX`W8jVki2Mx=oG{>U~YkogFfi!!Xe5 zRN=hq`?PqIakO|=28<~3#*7I(=k#07ZuUxjCv8#t59g!zxAva9Lt9fC|3(v**JPN; zav;#>Eg04`k8-aEh+YLX+3iG85IW_G)X?qa4>kU*`WBHoHpd*c{W?8(I#YAp`(l>0 zB8WC*@dfwSewoyr+rR< z`n>;gK2T?&xW8bV9ZdD272AR=V=GA>Gccai#p?jr`?`i zH&jmgrPd-BQwaIHxsENMf~ESWa{ds~3#mq@HSXAPlIQOix z^Qy(~?;3&l-(&s8a}F_IA)t!+)(qyGgJG}`?#Ue&YR~mJ!Dx*b^&HxecCPt4XY=g$ zv?0?wp-kj36Z+M0C1(d=-@hISn-u=<(s)n=&+>xYC-~h_T&}3<=MnGTAN}Zi>+17{ z&nT4Efq9<0x0S0TWIKJ1+HQJdpYpq$*%-qGaugmBwSa8P6Xwp6!R8D_r?)MPnn?+O zDV=rIr+l-KxOwmL_eZ}#eSVu0M20wL#I=JKY=xefW`|$aJiJhY{&YpCV$KdtbeE5S zIzbAfx~*6z3~Vdk8p~vxs=TuucQHssm!aj<(aw)FT&|=&&S=L+c(Zmc{v4_vr7K&? z9Q5Ic%s*q_S(JYQk_17zLm)kPcULJQ(^6F-TF;<>@R`=Vlj{CW;*d}jePA*g{S6sc z<0DJ5KT7MUuHZ$MYw|YomyWS&@zSo?J1D|W)Jl9AD?pS zSk>4cE&AW?Gx_graK7&TXF%O{`R;pr@vwILIIL+~$-ej8c{4uuFbi4Eg$oP0y*T)) zW>tbV9qYnrzMDx1lL4QIa)O_9r|8E2?aWT-uS>D-yp3_E&geHMZ~O{2=KpWH{W+Lc zCY*-}w{aooVfYRC?*WO1F-c};n>UNTTq2?ad$yS-YVr&BchcY^hH2Mh$%q3=r_yu- zmBkIvZyyylw4WFj#hW$XTcI75CdQ;MEz!qxo*3Yx7)qrtqj;7~k%Uu@!$;V85Yd6z zarMzax!GHqzRfL}=k|Zx4qgG_77*!h2m_cf1l%2B00+bVoX;Crm;eA#KM-CMT2RxlEd*aZo4f~o zdU)S3`lXV6&C_hK4X?1 z;78N*`|%XKUk1~C-`S7UlNK3G;o=WfMQw$-3VZNK6CiU7qjoT!!5CNShe$Y9OxDNg z>E4yULLqs5c%MG6U^9OCn!`K#u;WvaIWd>2X4YZPX1vPPW1Mh6Y2+K6W7=_F{gQ@@-Ff>joWqd zay8p+mX`^hlqiXiz*BRaEvuonMeRvwc8trXZW2-}n$ccE>?;HY_3sdl6dw(eC>_67)WfCWrqLc_UoiC3dy8u&v<=&bp; zoWIk5{&jRK;^wvDna^x1kpE#T+vcRL*jF)P7&GY+ePI;_ecqoQ_({R=h?T>YzoeqB z*p(|fN3d@$PLR!X#@Ig;I2?nuWv)b&&lRd}5+-h1`}tA~M)}&zOYwIp{tuO6>NG7= zKBzl5)w-Pvt|FSn*~{ zdNn@t=#W{1Bk%|ceuJciy`u0C{@$fQK5!Lov6xm+;>M(O=B$&78Nl}y$_V8o{@zjK zjy_6<=RUfb1I|#XQ~%3?9&ZY0pb43FZ0J+<1`GChlsCFx=z35prmP7jJ}Dv@ zfqQ=7HAIw)b9H>m;8R#-j|>rXZYkS&FhoL9Ct02Ecb@7ThmXO61~?U2J$@_)s!5Vi zYO%1q#c##jA{gnG$KE}lgo^21Y?6=%ZWUXjyL@BNrYxxB9gQ&d@zXS$k!5s21ktub zMcxRWwr91kAe>|Jv8ZkRw_X84R_7Sra=bcfgRQ#dpHu%jSgCn^^T_XmGtCX!Wev?o zUPbmSe(8dT>z184IofNg85|ZItzpfC5Xm4w1z;G~R6apbv<|fwXUEz)_IwXFW-Sp zqCp11TXXeu3EZ-fj9c+Ic)pB}4qJ7%GSEfLPMm~Sxr)c68~#=kw_7(bhl~f^!|Va2 zETPbuxt9RLww;T(u8Ig64FT*A$MyX9S40}ym-w}_zH-k|IPuanxov?8h09t%2JHe+|5q3js6Ms2zpV?i-%c%nPxDW zGHIudK2HzGa~Ym(b>jWpgvvM8fdXD&u1r(wp(B$wYOeAGp^$L{#28N*F0f#Vb>lwgHFC(b42SH z^9h=)AK&SX8ccAq`{x?Teh(^8Ync7D5}&Xo3Xc!f%{6$;nm>U#dKlBnH)QE*?HoRroB6v&Ui#y4|3> z>akho>@!dKC*fNAP#IjhXUfWQ#XIP8P4%>$Adw@=6Sa+@hurc1xpq5w-o@9C z9C)Q3Reg?3h26dXD~e9ByDg+xMvFTIdoCh_50~upz@03u7L4ul8IP<3iRVW&qmJv+!r7v{?lRt^u ztR_y(Blq=^COj>Xzr?Ml@aF_DB+GSPO<D!sdz&Eji{m#9xywTrPC```R8ZNr>Fr zXJy>%`YSG857p6s-owEea$_1k)9KGlSl^~kH9sKgLEO@)mgJ9mrbl$Kof@EYY-;~A=rBc|_qXVjL{e?h4VyI47m%LS znI&yD;X@%WQ&bhT@SsSvy8T7Z4KLkWCA<+p664f7d;SF03gHR@!%{}{J{Wrx%VAa- zzpBo(e;TEH$o&yKUnxzTna>k(-L50*Df+4SQdlUS2bdC7&oM3D;?w8$tBvl2^Q~C+ z3!#K!DvZ-Ak1jTVzFTfQSF4o1DE?(F>v|^ZQD5R@&E9t)OSJy6)ms1Z zraNXiT#ZNHB+?y-~b+|NqU3oqp@s$YY`lK$GFoN%EQ&H2%Q^Ibr2yuC0 zt%x;x6~7vJNwua=#;sD+B%GyhqmHHv446+8y+k`ePT5u5yiq=n5yKy!hBiB2kmZ=s zXNeQ4WGo*<4_0Iz>T!YAr7TLKCi^nnQ`C7=5!L|S3(nk?r}pU>Yb10;)|AY?daaPC zB9_%77{T7uVyNr6^ z{Fv@L-*(%KMq=Jt)^v_cF*M9>ww@XmXgsV&uOAm; z`S1L_$h!Mez|`HNe-p2d1yWQ*`Y&CI7Zq@JRj?O5CY5#kifCyNXhPRKl5ymDou$Zm z>c;Nsj5o})6gWX^Agl{=VOeBVMxO60n*S#W{A9P z)yO%CgeVm^s5VA(MEjiPyBV*L7hh^>oWh^mS)4F7js)f3AX;&4u3I&8B){4K1Ki9E zAs2I_C&(r4C`(b|`X`gS1j|#j#AG&SUEcitY-eErYC*eI9o2NT zzx8{s)L!55dZXnbKH2x0O`nR4`XNgHF>y=cNKW9_5l|=74(AyqIIMGWS+exP%}6wc zZ_CNiq(Zozm=TQGtjYh8?diDKXOw!=T31aDl>HGT>H_Qtupn#ZG*@7D9Hs1M!%yxo zT2bykZ{?;;sX|KXk`DD;N(A8WU~J~B8fnQ zr?MwIQUr-+zZ^ba~5&GM*;I$8gL^gN>!I*?t)$B$hgT zZpIRlhKLNLe-(Ip*03{}ER2H%1ZKzE$!5;qK3aF#XEe+2uoH;=Jzi=COdp<^|HB`>2GkZe*$oRq!|PI!oK+^T zZX9`ENQDzRu`sYDD?rAo)H(hb*C`Xyg$RXkg&faRr$%wZpE1_OOR-rZ`4Ky>i%aOB zK){VuQ-+}+wrpYg&jN2ODi}kzhh=ZU|EQ`&h#jhRr+Fecb0i-(F!_&>rU+>;ZhIRwmUn_cl?8eZnoKxxth)3tErYoo zIVK2zOOjOF8wg*gnyeiD7E5NZOjuVdO=+|)y6!9xzn3fXU(>(8&JsH#Q_*kQSY9@O zifVcp`PkUl*){hx0nZD&Z2!1Sv>{0K7dXVuUpM=qBe$VK3uKtR;7&FampmSiq4JVn z6t=6b`P};}WKlcK@0GzDbu6>a1_`Y!P53i3<-;Z2@vG0YBOLRSSt&5mFYl2R7`;ET(k*R#pOM`xJmj}_u-l2-I`q6Dxn0W;7n|6j{#&g8Ij^g%JJ~Y7X z4JA=yv#n5s8q;1Ck9&RWmir~397GkHL|DY0Xpadovd8r{w)V8+KnQ23^*N?e4(u%f zx*f$F*#;tk*}5?^ARL9lzHTOXg2HDtOD8A0(_Y+#W*M%q_x1F-j z_%zKMg#u+Q|6^eMW8nD7kNM5LKkNdx`U;3`6KRC3RFLZG!6m*&ASIW9`$48Oew1P@JDo0Z@jXHAt%E6!! z1XD)60zsq?;a@R?N3l7kLv|G5FRG1e*rZ^a+2QfCoVMOd3t*~tT$A`bxAZij)ncZe zGvmew@n}afwCOo4gmNCuIGr7|A(;}j?XtyHxqjfqmvOukX}QGW^xt{Mn%Mt$Q1Np4 zcqCTMDq9Z=F{u+A6PN-x1VWQY(61nRRm&b`F4Gtt0=(Qbp`Bke&@yEhc#`4K%b*$n9*l zSBw`6{SlmrdRSDBRuj@ysZEaKXOgqhE+9fFnn%6O0T_H)(7cLny7bhS8QUfqWF;Pm z7&8MkHtHY_Pxip?G>=zXw;)|D{T10k$G9WqA(dxS|U$Nh2q=MluwnSyl#uwSv># zZ85+Ndzq7yFGh}Z(H&oIrFo`fA*A!FRO10{JLmBabQ`~1ATdym>SGqCdBo@}^r@n< zB3pEU$>L%H^&1RbEbmt=Bu^yRh1!07Y7R>(>t1;!pUl$hAYSm_Q_6iO%q(n$-fsjR z898s}!`FUK^a)Pgr(5cT;k^KQC6j&skjbVx=J8ePBhj7`&bclEFN>SIKy7w4>JH5U zk5k{ydjphymwa1mA%^D@3BD)jgdB{Okx);N}$* zsE++JCwjCg_vDv>af-%)3# z{HwTI-P2n29ok_2W|3zQcekVgGC_E^vy!(r=7oJ3bG=9uTDh-_$5~+R8 z0K<0omPY+)RO&?=vtvng$j+;vk@`33?+C@j1w=f~lhdb>`gf~OGY9JBhuF=A!hyOY z_U_B*><^6sCvgDd1CGhE0#O1|Gd*&ZPjsqicF0z`3D~2D1;W) zGKP$4r#wHXNg4yDj}JVu6pR|gre~Eu>VJ61ZI&pduD5|b@Ucai*kcGp&vnB|U@Yw8 z!}7)n`=acKOD5i>aKyNF5&&;^p5PddCNv@D4)Yao8m}4r6pjHQq}&%T2>?*$w}R6L z!W~s!L}GVnj>o)uFF}woyWDaB!YmIs5R0~Ev152A9@D=7jU1&{gRGa_ginyIJt!_{ zj#B@G%{y^n#$gMIYUb-cjJS7b9p5&Ya(NbpGVfae`$qlZFd?6GNxr<%rv*=SXvy38 zzFnNoLN-FSxQiA72uRj3dzthdy5508I0>3ll5JsyP{vRdwWs+9e+HnlJ>XpG^s0fz zc!Wb6tXK=7AdPp6tOqRP)`U)zN-cjO>?R?$9^QDkT4eucZ!Za)fR4qb)t!rQ{Z@Ix9HGEQyr@k zl}WE)$oC1N+RWU9(Rz`Y(Z>v2r}7L#j59Oo{F(gXv|50m9X;Olv_Q1XBwwSYr+;Bk z9!a-EFmUVlO(V1FZvF|dwnH+A}N?ZcDUTRW1QRJDGV zg*GbzdUMDsZ&9wa46LYpcPKQ14E!jDBA`#1$-o6ZKIv1y>~OC0t;nYiO|Fd0IDw~b z#A<^DQ`V4C5-HBVnjFOB$dGGOy(Slv%{yytEVw25?x_k|2-pb8CAEbQ?NnL}!)$kv}(#w-4|qed-694~niNx<0zIOd*Luof+D%Y6Q5Tmgwi7NyUzaQ;&<{4b{< z4`4WapgE&=1-gclbt6T&4ldvyA=BA0O5E~su80{#CXEyZp%iqj8UuIC0&o*-=`tKP zve@#Jg2Q;WWZ6VIez71YWfkJgKA6bjtpy}^xz?J;8kjnOo-MYh6$MvlAGL;{}mWabK40o%-c+(2b#h?QdzO-il z>F`Vj47lmUFI^dORpFebdHYO-F5Md4iT!FE9$6?mj?thZb`2g?y3zV2dkm>qwOx{$ zuMSgI2S|QdcxH`=c6`5U=(W@?`&N+oH_pr)-YS_Ua%ZTJch>u(Aw_>{v(4yd6ubx61dk7mkJoFDdyg)UKdT>#YFpOG0GqPLl-TuW z|8;+#)A6hwjVWL%%pZ|q2rTN5t9zD17OJQ^9=@S-l87GmN|v@g1!7h;X|1n=kJgSs zv^wUDg{x!15*iQa29y#p$)>CguOr(j9c?Em4+z8gH#J@Akf)$Vz_r|*4UZXCn7JKcOQglaq( zLcTy*q%!oWG;MGB-2n$eCVL;>q6NR9qlbs1$CbUs^C|mJR)AE?f4TF7C|TF7Vs@cn z>1;9PZa{A7rog|#cCM4O`8Y4RX;tI)V!saPWZWEo2lmgKj$vRYN_Osh`xwauDoKb} zlW$52K%+;p1Y788VaR}0r9{o~cyZzHSk}!jaMgC%lZBFbRH~;#UaJC#F=$R`g6B_@ zVwqx%7IP?V2$TrXNZCMN#Ha4_MivX8)~BB9Rl><$0p0N*ca#}zuiUIc>cJN~?vSkm z6`wmfl^EbndZK+#oU`s(hUd)rqQ75h`CljN-S=w&Efsy=iA0HqAR6!OY`LUnk=J7P z0s#Fk(r>dMDSCgV=@#|4Kcv!h&0?IB!+DdM^)q3BK;p(iML+ug_fHgUjp=+*-*IKmUFLrwiCV>i-qXZl&h_=Cw>!wPW@; zHPiOu%=BJ&h;ayj@2%JTAdPh7{%^veT5hOzBGB?x$QXjWIyi=F zp+klNBvPhzNvWEJs^*gz`>(;E5U@~nB^!fi^k{dwJr_FKei+`?7j)WAh&OoD^^jcH zAGko48}=OfO54XP^vSxe3zeDo_ftBq6M&dPV<8x@s&i6{JFO@Coou1(9=~WkT)fcI ze~fl-x}|w|XfnGlKtR&$%yl4$)b);DS?1jktQ&kp& z1RmrRUx$O!ekj*Vlc`@K+>%oTx+9(oKTNtvAav#d7!5GdlBAVyFfcX{SO?Z>paUQj@;T2-ZWAi4&2Bzz zjT-nE-SKAKZ7((7Qn@eqULM^oTBCs+(;E-*?7cc@nvzCM^_GkL_BOJvw&gMblj?mP zpf3MRQ~oX<_$Im2j0&2lygEJuqOihliyWk;RWoDs@BlpA$Nai5z$#p>$fjIrboL4G zcn?TG6tO*LUo09pR+ELz21f)LQ4a+}|L^H`h31;(9VvSr zF~Hfp13Ep|p`1QDeF952>uL7B_cV`v?!Z?8UJC;i!QXGwT;lFgyyCAz_f72|Ad#U+ zVjmZXDu7A_#7=tgPOgC%P2*4dycXe&iLY|V(~neDj_BY*fðOQFi2ld|wW5xI{iS9P;eOq6V$ zcwobHcvDy`np6O-3G#R87KuB=AAlxCZHL>$Ilt1#diBM8@gfQ3ck#$aLL}N= zZhzgxix2$Wbu2v|0FU`eM~`h=H-Y|uw*BFL$rlfZJF;*nR4V-VJs~<82?FU|q_qS> zynvqZf?ahtQ}@zwy+q?dDr)-#2RKLX7V)Bo9pd#WHR9FnQ2pXhAK=0emWKTGUE&2v zp0bCIxGu;RCvbPRR?=f)pmjPNF*iARUyZ2J=k>-%qj{wUz79!wzx(^m^8HBEhOTsx z?P0q0-o3i$&|ARdta78jRu@UT{T>NvEiFg{pn)FQkF+7NkbSQduWpcUDf}r#DFxSh z-}f}-jLPy)4XOh&Kkvbe4tN@~4bDu+=Ve^ak&h4-J-_JT-spAiEuC<2r3P-zyA_&; z<*Am7pW|C(HTM1&z@%XS=oo-N0y_Tp{}@Udna80LCqbkSojA&3Lf#u}dKdj#jW))& zHo#2PRg{EF5-eaXS^1oTUx!>2iw$q&Wm6hLp^f(y5G*#f^1RNAqUc3ZDscb=xba;@ z<`b-8FJK>@y~^c+TjtcSi77I}LWD~91VzH`i(Y;l)wjKD_mw+qAdu0{WMldJ^1uTV z1i{tCGQ^13vK)p5rA;L>y$YwzM|x1_2Lkd==XJta2Y6~a&bI=dR36sA*e6v+G(aZ| zwJn-SLH5D2J0}IFaKb4Ly)E`RvB#gYk5H1v-Pctt&TCdK$6r3nc{hcLV1RF=yGGORf za1OyuKKZ9V`j7m3+J)c*gn+pBq0l8e)lB<=R*fZLtD1jWgx6)QCsJ(G`Sy3NH zNDISYy9m^XdOU|m=ccV5bqma@K&;uc1E7;4eQt{}_E zD$f3$9vIycS`j^~`acHg1!_{R)^RXw+o>?$8xG$Q99E;b@o_7Xqm-pA6xh4xYXupH z^opof_&-0OaD8djnRmS3ghed9+o9WmAz8M)_E}}wYi>o2B@Y$=euIK-#OhTqG@Uz@ zEx(b~^ik;9bE+uK-_`#1QdH$U9}^cU_1OpsaT8VSGt<>ZH-vgDdgZ)8o!uXZDAeK3 zd@+7c9CFyn?{BDQGDGffwILK!iDc`_+f~Wb2E+0(M5tc|PCidBQfmw2opmTZ)!B3u z6F#N}CPzT;sN@fHj4uDJ`F1KuHT>{D7rke&#Iad+Q>?>>gqOh7L8puucWE8pmjM6r zO{0+;8YloB-PI#O%{kvw7prDpu5J5I*X22Y;xa%2X7|EF&fo_ZMy2t?Lfan>5hs)h zjxNzUQexeee3%d-9~;^mD=Pjx@a27``^i|z+xcPg@Ykdn*otz8{?&L50FY9{O`n>C+XoNk{m zD*=?Myc=BrVTK`{4Cvq-#pW+9J9_E?>)P!D1xwQ8Idd;1Wk^wvP&hY&qBS(0w|wZ> zxqzBymhbQy->Ozsz+AH>tRzKk)hh*~8~pq9A#iaN6P!KHE>|L?XlI^?T65sUzAXJi zUCwHx@q8b%wf7?U`+aP`nM<<;@(IXDv;7c^?;h8l7_Kqw>6WN`a&yoZp;F*TuRJEY zw7%C46mCQVZ!&}HqX{z95Dc5Iwa-dmQa8OcdgpS91QHs;w>;lKxVrb_bAct`GEUP0 zCZ(n!ePbr&3Wq`_HFzUU^u&QtEpvE~&!>jzvN#gHA z3YHuRmTo*OFS6B#2b1U9d=djz7{WnZg43btZ*q8Y&JJ71WN?iLjA-HwI0`gR;#_vm zRPWmu6h39>z(rdAQ65J>ZYGmRfkDfj)G!8I=@QqnN+tkjCXoTWwL>#}lFrX&+{0_Kf zV~>o22v2lmw#_&cc9R;UZG7LwPd=;z-|gg^UfL+Alsxe7laLglKbPEi|E+?{elzA2 zRPlS30{iIm93p+96YLx{wZ{0QGW~N$Nc?g7d0Mh@z$mLw_g*d2xPJ8JQPdi+32pOV3GtX27tVG~R zg_GsIk^fj~Qm@SmIJS&Ir1dj!!DKi#SYji}W%vbG`ePq5?)$xg%bohk>VtB@xJ>~+ z0{Cq8+w3hn*MB*0)YDgxUlGaiMv~-I^>yCR`YKHXTV@w_5Ga7thk`P7O$#aFExRp= z7J67;P*a7bmU=(Fy4~1#DKD&nqBvlVo>f~%^gZM*2{FMP+L$otHQAp}RVGhw2E_iR zW+M(~%XWac2^7&|f=U$A;e~v#-V`iP9ySEQ_^LRf8O_`HO2_rvl+*JJ)WG(CHWIy` z2u!0s$6vU_Gh_(HJa2FX02zC$m6q;x#Z1vott5@-uEu= z+S)Xs1*1u_d|lr>44y&n8A}aijusI`d1OMX88=DlDZ`{$mdYP~65UVB|j+S~A7cYVm?*$gCw{ zc!99?WhJ}PxM*KU#;1R)D9=H}J%V~+3={2n~15FhD=LHQ~o(usAo|m zn@b*N_0lOnlhHDXLE(?yA2?Z)MvXHa5DZ5BG%@x!zT-@QIy;@L;TM70q+=3T(D^RSs z1R9`tptz)NF6h^n0ChQm5w>NMS;Lm1wq^azr zVUS3}Q2Q!JKHrl&6X}!+b4neT0!(m{+by&ddQvdGmAo;A(@lNY`u*Bh__m~%e5h^>u ze*PgJbQ2KHEc+j>^6g~b{Vj#k7LqdW>PW-^zEoUNpJwDd2&#}l2Y#MWptjb}Asoi! zVgSXO`hbX@wBq~y1Vf)?B`3_;Btf6U)wMYnM(W%`2Lq}=4n;T9U6|^Z>Iw7}D={AD zbWveqN(Md&Wr^A3){d?+B~g23YF7hL-&3Ushhs|i{|d>G;Enpq&z%2}!~|BuX_FW> zESE-`sQ;ICJ*o4hp7}1oZN?qjK(`%>ACUC45eP|_d`TyDe){O$PkFXfVWU4loFh~z zrsuLw9wswEAiA86Oxw>0Y~R4h{P=w&FL_Ll2!m zP}3pvY7NL|@;-ja%E#n$GYNY z|LDB)u;=}t&oo&56<3ZtQ|RO1H(KNwp_pI@lVlQ=9PTfmIJIV33cIlWgA_CS=Ov(F zkE$8+bHZICWMa@l@YHJQ=g-_0MQW-)7rKRvqcS$7H+obwLnm(ou3+-~rMSPXl}2<3 zAmH&>a69hX6|WKvZvUTH)Surun>t1`XaYoH5g8Z8norT#d5MI3(9nK*gzOkkE=R4? zM$20gCO+eG!p*@|3ulr7y|aqqf)Woz<3knMfSF8?l4ttd?;O9*y=lYp`?-MvbHIhN z2L~&xfU@akAw=Nb;=GZg`9A8aq%C>J(f&7)_wVHfJu?8Rs1+b-r2Sv1uZl!nUESqz zO3PZ9|I*DPwAM>(iHmz6h&sP`74w1^km@AS{erdsoo_2YJ$Zz_IY+F{Nbs|8tc>5T zju%)%-7Jqv2)_Z-THD+EnGUKD&j3CaeG3%&a0OukWZR#z(%jBY;ohA9oDI;D(!mMz zs6Do;q+(+n~yF31)cH{0)1_6C$wnHy2sQ7%nMHKqCYlmW1RMut$6xGzGGw zXBh31Mj$qYQ43&r`oqQ(oJ^n`nmjFP;UqS;=`^KG{LZ*>{t?>kYS{hNpvXZoM0`hg z=(aSanHTf+A+z}=#v$i+745z`{u}SsLR!_uLX~VBoubS;bXg5_nNS)m!71rvnr?gx z2VC<5I-i4ig^S+7yL-%u`>a67jA5Gp0*6scLf^d0qV3bGHLna2n(c>spCSH{M(v@J zS?^|XJ~lp1s-@|1G_)_}WTCXvjYL>NY*UB$qhfi|BQGm8yzPKQjTAs)M4<7iz9wT6Gfk1;ikx zP_mXvA+gX^bK!5jvuWBz8NMZ4WGurye2)m#hr!q~=9O54#m0FwsevcE{T=AmA$e*MTn9}YUVN0<8!W#E-ufM z*M!OI=Fo4weed&!ZyP^remAgBekTo0szbqXwH@eN9YdV)8cG`dTUxqa?$s``s|%an zPH{EgoU7iLG|AmHDNd3^cXG)2za}Sd_FTTzE+#_D>b`1y#R_|&Pz9A03H~x}!DiO&91#qC6O>utNGrHJ6LxduYkpmrg}7RH;HMhfLyZ#~(1BBISzDkJRfLszk#s zbg6OCO+8ie$BUZ$){7n<2Sb2?daAfqCKLV#gWcyq@}pLr>s9a6(W2#iv(qYlJLkQ> ze+<7s+Q6MS8L=9(@4_y^x!x*e4d@Ih`0zr3Iml>~EAf|gHxeRQVYL>O&cHMTEs??# zsxPWeI0OwF89?!22=zT)ZW#cVqafII(VT_k1bKgDZ$y{)E!R7~yt730Arn50A2Z!bLXh z@sc4ve4+Ru|JI<^{;~oXF2g)^y#F)~!h)i{)XrjZw*ntIw6@d6Dk4kPdpRxE`2gg za`TM2rh!ab%~}*AKZ%Pcbe%S6fEn|zI}0cr3k{8fg(Gkos5Ms4SMs_S+8{eRKc^7) z^&BtKJgHzK@PHC26SD|Fr=K|?6s%#vJvP-VJ~sZQ;qmf1AxS4riSU=M!^~49CvPn_ zSN5~lyj*N-dgo^egC#kM4>kFKv*_zvhY*#{Nc=3-KtIUwaVJlyfeHGc|JKtua*x^`%>5<;(Sli|8QY z92^p!FM+8~zZmOjHn0dZw9)>?+(q#erB(a_Q{(EoPpmnQl&vqffGe|7w5!-)EyECt zQi7-zO;d%T--K9hZ>7+kgKZqWZ`>arfxVpP2>dwh`g)|4zCIb=-a@ko&g>7XVVWV6 zCtpNDO>~mgdOqVi)c<1Dug-%JbFK1AaDlGsn843$L=6O!=_ZDWKK%RC__iyums6{l zj)$CwG#D!=z{4y?j{7W+Y)oHp_HQptsAqs-<~KpE}hUR;SV#IR7L*Ixp$hEO-v|`MIjb z!st7?^b!f+1oa)2NKs#$cra;#H;T+MZe%K088mN-MT-?Q9sU z?@FwZcQf`yKoxABn}lDD4#OQ?TTaeHrdnJ6xep-py&zOGJg;YE_| z<0$K(@zc|e4j}*GH(Vm{5jpl@RSl_|OqTI#zg#sR->XS;PffC}OD{&2&vlnKobDWR zzguDkM-`zhrdZM&QLe?q9}eNdLWbFYLO(Yom1Z5Ede<|wiOI)}DMl|VAu5AjGn76? z`$U5-2BLZ#YQjsZ`BlO;yZ?Rcs2RSKa;P>HZWlLx#+h59CZ^NTpS5kpB48xg_xG#9 zxhGCflNvqxt*z_mr56BEaK=Z1$%$9A8cO=-?~oaorH!pwrcor7tR62X&4HF=G#H5) zl4RV4zct@FsYB;9f*K>0=sWa>L|1g=N|QWokY!laVWFBV796O0rNV>;AE?2+mxn4s ztLr%7ana~jWs^+)_|blbmbOP>V}Vkkci{jMi~KDPr;9%X)hbigvAY>$R!?Z%OmUU| z`X!ktfmXw<`U4$39o}$!ZkMq8zE(CkT22{y%j;7{g(+iRJ-u;vY@4`Io>HoFR$3-j zrr!*efBzWmgN`98j9MT#e)m-_m)-NK#GX8E1TP)kxLzLvtb(w3!MJ9nNRvaED%q@t zIikF=zc1wM#NB85DFGNMRTy=23c0-GMvxHZN>+TuaOG~I4K)!g)Y#b0KRkkade?jV zwAx|YppwUCzWW3^rx@oNR}zng+j-I)$H!TZam?6@`!Lu>pFcbaeYo%;V}6U3DK2*_7v^I;h%OiQ|ICDy`McbK;vH9=J*-Qjb$+y>zM?~ zjboQOZDOmMlm<9Or9I!T;ZTo$6?d0C2h`j~wFftSIT3ee3uc4{^V!)^y;sJN~yw&4;`${D@^u_Jnom}*#6f=B( zy0P`o%NJ-_xJ$N=!h2+MV=$dr-jtk;^{hc+%#P0f(s4#nEt3GM=3GrP!tvz7YimD* z73zUge*C%yNn>92FtJX180`eDF65^Lc%}g?1P=YzVSOWJcJTG zVpl-`qn$H@B7+!?j;4~2#vmr)7V-3xr4w0kT;)D9Gf|c~@rJuO)HTQu7qhjv1y`DO zy&iI(>L{UE5hg=tW|7bo>%9q>VaSbUKLZxc_19n4W^1E*6h7mb*0;$U=+~Ga^}#8t zez$_iFDYw&6qp?Gdh%V?;6@C-A^Ex_LXRGs;nfIFw`3O&aJF{oUQ2`@yu&-t`$bQgK~z8o2kP4pG`0@efmR3Yv{zbN;RX^&suOXBMe=;o!X3$^_$lakSy~$@8**TxV|3EMo@P2EUJ&kEClbr-0XOF1W$S1aw>22zlCg zcn_$&+j-ARJ$;Ym(BdxBpyl*Jf^@8$h=SsL>^LMcnN`2Wb1Nt0V_BID?^5u+$Nqxt z$ckH`#!Q9eU1v1uv7i?knqbOf5a{bVsGb%s?`o46iw!@9&!F2Y<%mN3g!7pGnGSXMEXBqgkve-qQOAl5X_r z_qcMACvxiCsM@(yt%7U2iZoMD`k2jc`^m2_9-lFPe4G`(tX9o{3cBB3gvC8dJDd5anjHjlL(6Dg5iQoBhBgPczEl7Gfbx0QMcy;lJ zGgO(EA2(~fmKvj8@TC}xS_{96Qh-%QfV?xhzov%%JoIgX?BU~=G3hPP2~G`k3ACna z+Pd0Av-7;w>?IY{;Wp%Dbp+Xp+v7JuhitZb=UH#+)s~y( zTBCsBE!V{;a-(eDdMKYQ!EZG*v~W6(K$E#Tr$h$Tn8?V!_8)*Mdn*^!-5$DhnuH~5 z6{o}$s|w~Zhh9GWp@bf4c_g~CLG$7?DeOj`Mdq>^!%_Gx(KnUcaVW;G^rUW5Wv#?V zfy=s0`eld55cDuBF#I{A0sRG$+n|^S4yCpLM=*6TUst6$*Bi+5Kn)Gm(S5NW8I1_XJISI2D=w%VhPEMBTQ#89>uoNXmuIkku}d0`#C=L>5!TtK z>+FPyH>J3*Uaw+XLMZ2INkLjCilVV9QR{TTm11EYe}^cOtI540+gM&zcWm?_bD2%> zek-IV#bRNg^W@keugt*0)b*=zm^363RDCZ|#PHs2)VE&P6;yYRCGcHK$zx}a$u1V{CPc|6ZX zbEH%c!KIdeT9Y6u8j;w8w22;>RDDHS_MC(5;xvKaR(4+}nIRc&B{eUtCqFK}70ZF( zHdDT){iWp`#&84~V6l(+j)r!afJ>ZZWkMWkMvQ`Jam~#y&eEhf(?{Zdqe}Yw^Jn!( zQtR)rRVrQ=SlMOVOkFNst#W$ofZyZP)Z4CD<}~bi!zt=eomxuhqVN)gA0oB9To}o? zr(O`K_yLn6u@d6}FqBy^qh{EYO3xAtn-nULVn1}15EBVx#OCZAh5!ZEF%ID6$+Gr! zTQsTQN{JE|XAJjPK;gX9<6q-UsZ7DWZdn)!ib5Wx{faElU2}0MCnlA1Ao5r)rZ`Gi)Exag&{YxD(i-@ ziWAg+!o5P5&ir|Ne6th^ADd6O>i&vn%V&|4RQn{JDo%foP2DS=Cg?Z9{2AH@DQeDl zYUT%)AA0H5+eGig2(cq^>QfTF+7O@zg4?ud1BpYEoY{ZBCtqfWJzQ^pEVjli z)K?WkLQEWLqN}pCv2TP-ZJQO?3vImLc#+m&2UcVeA_|0s$2VA^mXWWKG`tqfbvM_& zYtH$OnV3Ej!E!{jYMvd%ccn}jp}(uOTpGsHb)_;eC8Cm8Vy3wwL#_;>yPg z8qm$e3^gGMg%rkpBXCBk;Qn8!Zd{ii4B0KcGN|jRyD4r|>+8^{3dz zG_CS+^GmPME!*eMndBl-R?S)`cDdgoK9l`Q42RYZof5sXKCma$U|e ziu6Y`_O-q_Y^;tR!J5a@si)AHO(~bIfYVJ@uZN#wTFZTLMB3I#mC;BUqoFa13V{^x zkG7u$Q7z429}y!x@Wh(!5T9XvO5wUE*P^)fJGx9SAp05S4l1c0;eFimJhE*4Dc79^n)#h03xT5 zu!5OM)z^gCnAqCa*TwqtYhP|#%iJjrff;HV*tS?AuI|`2{n4G2aXofJ?-yjuhaGeh zwQs5Wyj=l4V;sUSvtjscGAWUMt!>V;@;t8Dj2qR_b6D`I+ zbo+$mL|NzxltN26`)KX5aPC}y+WGb*>k4Rz)-PQ-(+bwNDURYFzs#)9DPA#&@s_9* zCmhP!=EQ5zyBr;^tb-9V9ZNezCqqsyu%~!J>c+;NhB+DypFdwu6cu2C61+T_G?-id zNZ@3uW@{+samQ$X=QX@Wf9UeKm|}ez8Al)PIxyhd1-RIKLJ{Cq&4ITyLOr)hGiVVq zTDaN*&(JyjEfIZeD1ymBH{NZ0CdkHeb9aRutr;f}(cCT5=o= zEYDS7{uRR0Pnc$m!s=ibbp(`ElvHOuBV8G3VPhoiPicy6Ho%oEhC>h~#vEA5KsRBq z1$VNA7w~UbJXsA!QQucG2@QWuuv1YbDw$XzrqGG86x7O@kmrLU(i-R5)n5^ofu3vD z`E+Y;=Ohz$FD$n^(bgeon>tK~FA+#)BQv?0j>D-ERRDUyNJCAX@dGM{ldK&~izQIl z7Oj0Lui4@pYr}2Nq?Mz|sBt;@2<;t#kdpV0A*UoT(%Ra3vfOB>ySsw$P*sbyuV{Qb zE)F+ZuKh%`f#@e*s7Vr5p+>JJ)4CS(pRB5EYSefLx@avXYF_I#i!D#>!* zz}&zv*c8SGJi)>wv~u+P{UpsuJZbST}pC)-|9;X5Q;jpKTEl zlOwAf*gxKDRkuuGJe8zMPC3ShL$twrYFf)sl~vu0WrI8{@~)#x58x_URdEz@0s?w9 zi^oCjGBPqctb}O;axxfPbTcjQf$`7KW+Y&SgPGx`^7x|99ym&)(TjC^Q#JAH!$Ot8 z0t}Wq>O8E42)Sz63Z2M(4gDAkTC!ruAt}Z|PQqV}^aAXI9rm-6xJ$2Y0|4c*_;Oi? z9cn_0|9t2KA2C(}(beswQIG4C-B`C(0`nHoB1T7|3W0tVwSmi?Dm3v8b@Jov_@ydE zqYBNw1pbe}5)!C@Pcqqz*G7>*0j1>~J z>4k#YS5ISI?<#=IBq*OV05(A~+lmo9i*X5nFVP(YkdX=4XXC`IyX|#R?_*zhm~rpR z%#57E(xNK~#}hl{1+|zq{pQw8T2kL}czr5>6s)Y{OH!xwuFE!9VQR0bS?J&HhwC{)0?#ORxb|^UUUPkPbW{p2Y$yM1$Qj za_CUrr>jC^d3%LA#r83Yo?9_BJI@vjwh|rnn+4HxQMwUDBT^e^q7$R-$o;LeI%uXj zUH?tlpAf8)gk=qGM36~`gap7ixW-AVfQx?9k-cj)S6vWttNe2MfuwtT?x!0umB22? zYa}FFgSkX2jS3T@@kq1%;)lDd?CGF>L9yaBK%7Sn}c+#|q&fL!#Qk=0! z3gVx$V-=T+5UssEy3p`$EFNPf(XJzU+{XcGcN8s$6vLU}#?`t?^nshD&Pd5@UaOj_ z8gqWJ`2>W?uq71qw0M|myFQI1)ItammD1ydOweY;AR>~?RY%8Q2@W#w^#%UD3TxuZ zyOTL~F%)Sb5i-o*4ea=f6%Jr@AqtKrcEZWelXO;fZF4%3O!x|BS-p!psBBQI?apN7 zOSE9EoE2Jj*hFyrm1Bb=Laz=&^nK!LHbBPPU!+*OoIO405!%mX6j4<&(V)bk?t>Ij zV?fdJ6?z~Le0X#qMMV?*>uaHa`SzK2Gneo%aav~=Eh6U?C^qrM^TMZzvNc;T$l5Dd ztu-M6M2M8){x0R^J(|mH9c`j|NHzcjI@d>Wh)cjE!l*ETjtyCpZ(evy!(cG)2Cieg z6G7p(!oocC;LNzVD?ye1$Vi|h?8sT$BP6V`w@Z6#U3$Tv`vDCJWz%N1a1K#2oFW=w zrwXFdDJ2XRA2sggCFSNB#qV}Z<6G!cYH^VFvFZ)ygLOt~1F#%xAB#D&17b#dAD>YZ zL)XHDI|a91iY|r4oI6CREGy`t_8l{v@S%Y8OU?gScC)P&tYH|#xktE7;el3x!9Z&6T*T;+9kA++6WK0eV4;L=dex^V zGI`cVrLgGK>*r`B$n{z2CoL@4a5x}RYQqAuPQHz0nhEP2=T%QQuPLr zE>9=lPg1YTIT^6{gN?`a1`t}23pRr_Lve(5SEBw=COVDXxH zR33dS8B;7eOKiOC8w;c`yuWI+MMLX;U;j-ZwEXNtRGkeUfMhB%FzuCdLWvWjRYzwN zMwFj3<##c4ekL%}PoOq%;Qx9Y0IT_IwgwrTm)*!tY_zq=fYvu^Z4@Rd>S7OlHOJRt zPh?u-uAd?Tv7@v0+02O{nNee1M)sVCuveD+B(Gi#^e7;XoUShgrKNl~R=YO(L219@ z7R7KtEO2lVGqRvyZeDtM8R>YT;py2uFUCsdutEnU8Cicw`b?tEjnH2uHP)z8FaYcI zKh5A4>gBZft$XSOcyH3)dDLPv|159I4Ig8wPzyRN!+dqCTfo8v)=&OW#x>W)-UmgKp6f8iQr#&vG>|ZF-hwLhBn_&fXVKbAPp&L)Q@W zy}UX+%7nqgHk{KS`bfa<|7s_fsOeAxC~qL3#j6()Zbn=m1;}McREnYy;zKGG@G_ zXHN&?epzmpBDOZ7K*f>{|J1o0@XYm>?P>71G3T38{@h_YTnV^gnnMu!%qa>M&YesfUz)H&O^B0MX`Lh5 z6qodMWV2)}Kcs@_rSqed;~`%ZCo8F4AJa&YD7hx?S&C=DU zYuKScKvur??;fy3$E*TKGnPuu8m*1t22BO@t+jm=lwMmN$z7;K4e#|VOgHh$!M~FM z;y4_bpry6lP4$GH`$n6p0a*2ODVtPrrul>fYh?uig|n!P&Ag@%Gm>Nz;!YY&bJTNK zaWwFXn#NQYSj^9ddTC3P+KSfFr8-&G)4_v%7ERzm>m>YA@JbP+-uAWVkkDfHrM7wI z@=qoL=a9CR->~ZGl8LB185c;75Z`!Gg85*-IUPhR!FmIb4h$NaU$E+#e!|f8@Jf8V zcIV&6=^DL#Yw1etYK4OJkQ|L;FjA&4;kyPTS}SK9tUs?&_K~#<{#o)pM~H zOOnkEK9CoJN_}C?rL}4nN!^pDT9FFZa!KuAqzkAY!3|M zD)BKG)Q-7dFsPkC_<#p)K8|tlT3Py>dch&&yDKR8#Nr7Wff**~Lj`eYRZAfe6Em`H zQg4^$lRQ#O%{XgH^OIDr6E!oNI|&eWBv@vk1R_^xXpWeHkN(>s`{&U8+rn`Fu^|Jw zOVsE<#S*anzkGyP68Cr6{~Y&)@f2X<1GY~9v&nxuO8^q|-;UttpZ`6hy`=uX|B>HB z`lD$v^Ne+!ziIngXnUPEOqTuoTt6A2`71{O;)LxV*gnVu0~! z^h-nxVI%rx7&QRIivc@f0r}sB+)B70(25tlb@p8kIRm&y_#GCJoC` zA2Z!jsOfP;#C`-6Oq&;J5o@B{LQoPaKTcTWR)oXGI*xGnVcPv3G-w(jom+FWhq zzgD~#YN}OPCB!fQ#$B}QeqN-^pk=DTfZ^d&R#MWMpU4BW_4uOJ=b}$qJ&1~IsyWvg z^>FiS-^s3j9ov#cqbyZo{qeg9T9~nCwO%GlaZ|v**}$r>YC461fgZFy?cVEP>wEh+ z!fJAZ@8*8tc6Dy?3_(Ka{!!d5H z^r92qSX-(Z9(i_pw8yt}@@3ri)$nBIY2j>MNA~>eck zFX(nFKh*N%_VMG0?$N$R=}C_=eNjskG@=_;ZLf3oek~Vh$eXWTn*sf1yVm|Jz+u*k zA0ZfHJDu<6Xys|=I`%`^R;;;+XJizJS<1;>x7G7BG|x`T7QdxQ9IkWk;0~5-HQj2X z@Vho@_FeA27Ylj^h8MXt#_UrV>cVwOCtCSSqkS*m;i5nM%o3RKpr?shjUk&dJ3x8f z4aArr!Pr5Ljt6Bs+5nciy1F@8q#X7wz#a?hc`>9}%iDVpa;LP93Q$|T+?MJlj zwV$X=lms){ngrJ-=jA0v(?a%5LcO}P5P#^Z7mcu=`2!$4%Fv4A33#N0pMcFzqD3Wx z8m+yRMN!+1+4meM7X=(@9JN5^4gC&-OtRgZnD<@ zz#Bf>qpb>#`8lz{Z}(y0?smFH1^%m#GX2zQO%hvcKhn@7uO_^D)%LcxWxmm9=lq*s z%+5p42OlUEO`+06liTditTnQ8rQcTSgXN%^O-2IpMYxhj&KGC**95gJ;m}x_lm8>?eMk_rM@sqP0^geG@F* zMk!XODuZ*dc!rHxv*>@iWay||jRB9Xw zhSL`p&Q5Z4$%yZc>_`)pmML`<1-D#F?rbObag*(RKgd!G?ot}FLV@A=`?hQc3w6mi z(tZ6(2UDwl9;yz8G(DAmksUdcmPKQv8Jl9Fex3I<-z{#+RQd3)mOj3dzr?`ZM7a5Vy=7wAG_)Ljr+Rr6inibC$$oRI* z+79K{_o> z2FzY)SKrV>VZLcDg*7h}x!n`y22VY6o?Gpebj#8{<68A=X2y~T-p2yzh77gDa=7R;4PJM>g{$vNwIZo7+gcx5 zx?lH}7_M&uEzmzO!p~;`kQZM;POd?Ty_VxS2+Mk*VejbeGl5b)-F)AUWQQ6Cn!;0s zM?iCKk+c58V*x({^t=0R1ND{1a9!EAD+LcT;P4J@g1o z7Tz5g7{1MG(-P2?KOVWk=Tog7!-Dzn@J-~VV5RybmX3R?k@AIKv-E}K?6S~)AswiF z4aTdnqasF%vb$Mi&x5=dsnTIpZ=rx9wVIjUYOy30jds1f6+Ua+6v(*yE4bfU&^{ap zs3)KKPPV}eLFbLjU*%6*c>eZL7ERjT?m9IeW;wT#0;Vo5hl5dKkLSbw-cmq7Y^@G! zW37Yrx%aH4_F%XCVuSyg&lB;p!&4OnmFx8aCJqz_9Q^8PNpixF6;=%`?P-y!$Ab51 zeosflS6c6VseT~CiDW#ndkcp5W)0OW4)*s~c*LSV4V$ZX3l0ZTXTnl0&WcljsVHH5 zjHZ$$uOF6u(bXQiJL`1Osd-}%p&=y$OYHF%5ND^qQ1-gP5N$eH#2{Q*&B=WAyvgs$ z$S4Goka^qdzw##|(o#jdF?Dd9jgwY%N8-kOoeb#8K$&tVxN-J!eox}(T#NhF^n|mQ ztI!V)OeruVXkxrsVdp$E_S?*9bj;1e!tq$B4N4JO)V^3;We&^J?T4Wv_d{#}#rG(l zg~06%??ThZ@zNb_WjQm`+tst;uiiRlX6R|FYw^Gzv5m zN*jP(Un~-dBwM_A+DwI+y^N#~uS@%it14spAhb_O%sR0Bi%Qt-wmaLe5knGKW9-n< z+#p3blgH65Nn#I>gfy6W&mfw^iiZ*MIaAo_w**yq_~XaH4?^yOf&{OK0Zy!VOi;^x zpuKQwwaJy#jRAZ{Hh0{x(J?d}7$Q|lvACQ3)Y>gy*N4T;2O$7jEHe-lxn5nEMLO2Ni65x9`QZLKHFl9HPJsm$W9-SiZpfBXWmG;jjm_fQc%@yZ)`wf_| zt<|aTQKuS6aAI0+U*+BRunX~n*{cymB4H#QozT&6vxs4fYHal}4pwxNN0@M?(uKG|bKaH@ej)<(c#WwZ|8-3*!EJ zjZNQHQ!@+~Oz@CI{fHL3<)*AZ6`y`CNe~VVwFD`#<2+~K^SDO8$$Ko)`YG$QrbcL# z-b%m5Uf6Y^!se>Hx?r)ztJ%JbeY%!SyAZ78y4FOJ=znGXrN_Jo%CnjHDTPU!D@W-H z**a(-=#j6)UOs!sM}zh9pLY8r!TB}(_DOIqg&64N3!e@NY#@~{>X18l*Qht&OhB~B zmrP9fHu&TGjlX`i|En8=#9>b7cd<#IFJ<*!;tfYg`)J+sho#8--acW`DsJ|e?Nn$j z<3vQ=2{)bJr6YeWTZI1&!+XLq`F*m?$AM<1#`I_$+fq*XtFlah0~BR;*SnU7u1dLh zK7Y+?Hyj$?LOxTj5}a?}(4;5>SoOLEn$fY584_oWdXwy1b91vzXM@!);b8%PO1F0Q z_TGrob&-=#)mscC81>7UjSIiEVagrQ1D3J- zw}oca0hcz*p?*US4}Z$+)3n(X!lTu`k7@4k`g;6~D_PESe}8e;yj@$*6R`N3v;S^( zvh}`3_56La?`;8_;d_}k5Bjd_w_D}(0|WDkK={~;fRbo53-WzR6HT!6>)*fsoE+`t zH5qKshj&W!tDAnpXK%P0px>UxOjJTkjhYtM_xv-LgaJDd<3-@3tasF$1VJ=hw9k*@ zHJEq)EO|G9_+?=e9t-(?%wnc=X$Z3`jsYe9OOy(%LK z3a%K7quqZ_;7@F0d3+$c|EMjF_S54KYK0KAR<8fN!~ZXL@d#ta+gD%dKW7a15{x3W zC(?Ccb6u0QOEVn3ra8I3zIVUNaRF*JsfD7!N6H-GoOn3ZurNBe>eQg{sTLJn*NsQ6x+7?o;?oQEky zKIojB)luTVT9w#eyg4sTJAd6m#_!qJdA36M9F0!@*UNtgecIFkvWfmw99Bvw{a9qv zYmq9QIlDgF_TTR>JcQuMxSsVQucHlLq2$mQxBhddWz4t{{V>III%brx?RV9^USJV6 zI<%y7t#>uQSD~Z*J(MASwRU7y$kemY;y=HzKizEUaFbe^Hg<6H>pd6Out=>#Uw{91 zt=tLiMgPs^Wd)hoRnu0=o3pLNE%r(mP|e z?*Kz$e)ni1Z4`DVYPq2i0;yjcLJe7?qoq@X(gNOac)nU<{>At0{9-0D)`9g=%G9Yc}up+4Vq zz36SIFFw&5X&L+sh8&}SjQMKS)?u>%PnEZ5AMmrnfYW=&OD^J}sGwljw3O1^_@Grd zu30qhS!#VUVL!_P1dT_w^&{l7KiIel(~t_^D#ondK6xC$$VlrtP$Z#3CN`3%tLUSi zZL+*4;zm#7K)HDR;*C+Wb42)xsoA}dlij8E;#iS-YTO~o#GJLj^D1pm#x%8epk=nrw(S%iL$?WCv9bVQoZy?j) zKhT8h%pU)2ajwToG(B!VGi4YW@~7<$BtHl=0vS+K4x$Bk3@3K5cThB{{{{dLsP5Nd zOk##J+bRsqxTWzsxUKyOESP9=J-QuuGr7rOXjm%8c9W3zTgFJ@VYAlZ0hXDOoQ$Gn zV;lE3yBOTlid?_x8MWXh!IOOtAoO^fiDEodY-m#^BRqio3F70g%%XhvE;_KQIS9~~ zfu*=lK{1_z8{cC^OLJrXl%~OP-%exg?Tz#h9vMDZMxe|HwaAw?Q1BpR-D=O9N`QqF zW5XSeu}`Z$hu_GhLjc#r2K5H`1#?LpKtFvFoSPS0*c$g3E3+bvwTga)w=*xmVKu92gu>Qxc0rQT~zgUd|1FSYrX$_=fM#3Fk#9+biG~+ ztWguAY}Hnri`}`St!D>J1icO|X`Z)#msRYctk-9oz^kd;Ngp+}_{`ITlUoB{0Vk-V z{9mHX7i70;5)PED0t>sw)z4+dzAIFFEba(cb|!r!pG}wW@!rKrT-cs4o85SE0#hFH z`zM~99J>Sk)rta?09Z!&VO3UN$u%jqJabBufZ%_tWk=VJ6cgg3V)pN=@+Wous0oa;ZUcZ>e7GZ+6<)D#PeYpw)!q$G;|? zn(tqdV}U>*tmk+Z2f1BQfWgi9SD7lHRJXZEx}Y`>KVmB6Q#S#4eW;cF^yK7KW1Mz~ z>T5K#%}|O^ZIt0DU!Z;H76qxmtBHq>$kM zIZg}WO&mie%@f8uD%>+#jPl4*dy>;h&qsQEwXez(`{ZST>W~h9* zEaC^-xTMU8VzHmDIr{6+Vn0&jRqoMH>z}(#k3;LYP%ggdsGOw++tEE4GB!k2J&wGB_i@19Msp$v9>vF(0DtFd;Hye^9-q&EX=RP| zxLxt8@tN3Ce_(}P`#d6FI`QMV?jxUl=xyk$ z+C5UY?D=)VXEu{mQ!_S}RMSVqVYk>kY0TMwW!)Kgcf)gr0JQhB7ZlKJQ&}2lv(0*D zUkU&{dwF^_k5+5f*7}4yTh*Vo)jC3LY)G z_oKM;EBuuLm2}VRQwjR&Lg)3O+?MN^p~Zmq22?BCrQ6wcG#JUv#)neuIu8!>8_wTL zGo{1@FeN5DF;ILgMKmeb?!35P?ZMn=nRJ1Pv;X5cLMv^m}HW92|^MoN8Ro1eWLy1GgkXVJEs zZ$gCg+v~U5_vL%F-ZuC=WKZhVUOBHhCQd88%;8CInC|TUV(rD*L@D;)zlK)3 z9uj0?Ue|wnM#Ews)Jwgi-N}U$RyP&f6l2K6A;s={>&26}!@G#lp-hR3ot%zY_g!X# z{9PhxaUNn&t;1plu<-JjMsdi*j_zV#1t0`AHwMy7$qha>jp^0>F+`Lf1B`Jt{Sbl$ z53l63P220S9;QO3;z|3N>q(UmDK3j+beio;t<9fVSsmC!xWgQ+{VaMdq9Z_&ECL^o z*nNKv&#-9hjr~j!U+TyCMG7wGnsL+l`^g`GyaVGRPnR%a(c3hgz1q^;n7OF3I+;2X z;8UzttOW*xO--dFwJGpPp(rRHO_4^uGoTJq*yqE)JFheIr?D|JbKCMob|l0jmRDBV zPMTt44-yt1b^%31MkGG&|IXMrx3e5i&C04StYEXb(LZ!w(U?)G=(r)|! z(x*N}`b59W;h&#BUshYH2sn&y16{gtD8;FAL_9u<)diH-d$z6ub*Co)>;(ugpBUR& zBwlX!(TS`K7P87G)&sa049Te`YqM6MvFq7cV&D3cr~c`w$=>&K)2B7Op(gq@>||Y< zB^q9nEx~wrc)-N7uOGA7X@AY+^zzYLzqGk+g+>0n+1G43yt;Ni6-X=?_*1_b2f%3r zq=d;RPFh;m*@=N6hpxW9QGaRrM7;Q6(QS-~TU)`Vo|Lq+5`d zR6;-+=}u`uK^mnUQju5)kPex`tuy9`y4)zjN-nf1JC1cimYoUCy&- zKl^#_z2E!we!b(7k}xs(IBto9?RwB(X+3R!+wD_r1P_@UOJrSLt#B<{One9fE%a!8 z@%(VjQoA&&vMD^A0|Fqe(dq!d=TV+m=^GNifDoi-Epx?M8JL}V7ntR2ooPzlv7r?> zGlOH5i|-MWZ!N(WK284m74e2VFgVz1xM$K+uW+=kwhp4|uAkifn(S!bd)o>u(uq8s zLh)=ljFTO=YL_&G+bQ^>w{7%^7v82Rr6BZme*vL=c)Wbo!1s}tQe>m9-ovD$h!R;0r?fF8Ki&$!)IL&hH=FORF~r?okg`lrLY%sDN&6LZP9 zk`$&Vxkzv%>7siB*2810XyxKOEfGX)x;sGe&CPq9>`sqnTUEOYeTgF?CqLYoGzz`2 zRG6^%K_O%$`9nVS^c?`*Ps7NOZJ(MDaB6|Kd!;sdahE5v8K%8EBD%ynwO&~2G+wRC z8~;UmgI-OnM7_V1h4UA_EY9F0X|jxeK)7{!JYm=x?&^}p%s%+;F+2dQC6%{wKO*Rm zEiG;?u-9`2P1YMe@Y7p2Z^B*+BF-D#TyA~*7$CzEV|rZv=Jo5WoGb$S%J&cMh>MFi z)YnVVt=8=MD%RG41sT~qys=5% z6FE9J`U=_9===5oeMDw1bEozz=jV$a+HZl?{K_N)a1fL+a?d5>*bRH2>);RqBGaug z-&hd55^!5}+@1Z|?4mtbLTRbZ9=+s6smnJD{(nXFd=UmgOBM~hgKbw~0sr;gNAuB@ zf}E0Wvu%It?dCjgOL!}g-V4^)LhTak`L9>~&O3ojoJ{MRF^>`w5+sk{BlK$dN9UpP zrU(9bR9U&X#2t}O-@@ioly+uPpL+J0l+n>K0Cu}qYDh`q?JfXbrfA{o;z}^_DQq@6GP2g} zxZu+n1xt}p^LHaJn?|QCEjjz12L5saOka<4J%c(#)vrk)ulW8|oJlx+)qs@u4rZpCPcU#uEzjc~+zCr9-6rv~m@FHOdX7ci*}idn@xI&( z%zvIv{lKasx$yewJfhykhIbYi3;0X;m$!l(PsR6F)-C=$8u z%Hcl~MnT~cA)z^Q?yJ4Ins5;WYlhiaEPgq$*@tH^W=|UKh=_J~oI5-ues;dY4rX3iSlR4qJq%`Tq>bc0} z-Bl$?d8l@QnFa!tE)EW`nd17=wB~k`_ex7C z^PA7?28uwESYI;zfIln5*UvAUga_J_b(c$3)v_{2EVBa~cSUOXsRr>$6H;D?{+hB^ zGgY=a1seID)Yv4C*YW+>>R#Oir(~`|8o$+Q$igZz!^z2Ju5Jh5RmO}#KX>7)S+0IE zJ|=^@zWA~w1AYA_GM>v^ei33b8>0{ko_#p1-qHLGZ<3FMI7rmn1X;qPo10_Bv(0ue zJYljk!v#+XVWi1smh1+ph0RQb)$pd%?2}h((B@N5N5>=YD-S>jc!vu&Iqqj5whO6o zN#YYy_rJqo32dtY0gua9`_v1^PZV{lpdJ@v%&(f^>6~~cI(ob|3bL{mBVP?CqSCw! zUSygTQ>2yVx&s6;nxn8jxkD`Q;;@o4mNIGWdV5mn?&N8a zbg5DO#md*g3AmZ3{g zai-Ly)lio?x94ep9PvQP@yQ9ZK}o~#%QI+lSF@w&I|6)411?&bZI)#HK$56`I82Ya zAV$~-RxV~P8XC`iJFKk3Wsrk*eKW2Mr<-Itcd|w`D{@Z2YXd7{LVN-u`M9#TiUncqE-Ou7A zKFhHJA-AiHaC98QvtA0?nQ!k5tR0W)iUo5A6TA+lhP6vc(J=2c-Dw^53#FisX0CHI zpKR>algmEn9e$eo6irn1-SW^C?~VWfF@DA%W#9R7Y7yfUme`+z9%}e7!v)$1LP-uS zq%nG8N=a1fupv&H7blSta1gH-Ub;~4F;5L256JY#_1xQ!HrD2`8h3IzeUMRyLzXJy zmKYFQ^x_nn#21*Sl~Sh~sa(>Yu2y(WU5kBB4`2Enu-=y+t*BQKB|`^RD?>(V7JxdX zl6}VBalaQAxtux_$_fGhLqSK+uzA&Z(=Rxfmzj%;haRLKdqDt^*8-)mglYK4zm?oHBO@!19S6LMu;c;{QQMf#ac-b z_e5-ux=afv{QY8r!h*KPI*cbj=w^1yc&WV~ZaCuu9O$)6rEGYwGH15Ry05RVLbsTF zqR4V6Pgdn^K)WSBV(>1lZkfA9lfugf8Nk=057^M}jrEd_=f8dP@tYoIkWZFUPF1fR zMrJPTPP=V>g6&v=Q;mh?00P;$a&tRQtc<>oPl%cAB^uFhrq>M323wmXwW%Y`V1;G; zWc#NvZxTK}KJNBxG>7ReQ87^zr6*c8++eyTSZ#a!zP|jy9nHeYb;K2hTE64h-uIFlB=e*{Z2C5BjSt#wA1>P0HntSk1F=ab1n~qI-Yz9xWgWxbGGu%IFQ;ML-{E2MX2U7AH+}g#40` z+<5F&G!G2pH4xVZOqEZ}bv zf#33QivauOt+h0uKG(JoCwk(PN@0OwoOXK&0_J7&3%8wwBN{1y+o}t z)~@0GS(hrj^;M^G#nx1<`^qNI7B|V^j%)Pv?>ZPKabo>Ee) z-`o7FZFQSHkAs#Nl?GDbParna7n2QXUI>eeF%ZKSN+w){^uGlYaTS$J!y%nm+OZ`b zNdDRIZm0HeO}j)tr7=H#YU=FMr=A#=X;@v|=Es}2IP@D{yzB&pjX;>?GfZaAWH7k- zMtBfg<6&No8M>{$4j69OCsY$oof6tXij=Zl8n zgIQ*hRZ>#Ai2MHXUOfdGQ!J*%$RPp)Pv1$Pzd0qnl*d+55)^(D)HPy>JcElQ#1r28 zMNNq>F9{guXU>;Dex|LxDPG;(?Kr&bT1@l0YVR`gG&s{;lCpE!Bq6(h`O)&94>YUe zxF7)P3kl$PdlHX|Cbv3~J2a8&hZAF8w7&I-(VJ{OM0=JU0W45X;2k>OdMy&_qrlL6 z6`{aDPn)2B(m!d`G+95XPLBFJ$!3149M-Zs=cZe2BjPdN1}H%Wkd-7!MF`hPewY18Q4|j z60A)+Y#P+AHUGvD5)#y-E5aruBzz@{Y|E66Y!VihB)CuExHA)4#AhL9UllN|VQ$(vn#*E;TA#4v?NdN(vFROc&H4u<@>)EzKU;g%M2UPVEc~DaIa}IXpk7Y|@!&g2vg`B!g2^5> zPvo(A?AY~ywhnCARkm4S>H|^;4#l3BrR8RB8czS%*49KmkLgIbufrIS-tRhK5KROa z1-FZTg4Ckzjl^$10Yw$iT6CU2e}1f+;js_JbzjglQ7ko-xhn&0|)w#sqEU_rpY*raVTQ9EBX7wP?h ztE`Mx>r8DRS!gmFE*Jw|&3y$;645HE;CSYAp)gtP)-+dsn*6@`b#X#s;!K`JFdUC6 zKw|?Rk4pIRWlN`~j?A<+IqO5BG*gOnZ|m-7|s2ZDyQ)?w^*tA*bLq z(EvlxK>|`K>n$#wbXeGK>5kw`#nua;TqhH&6TuQ1x1^^{>gV-XEduqHK9}kvj(KuERD0foB1JN)|GrZ2n)u} zUJpeS@!3Q6_ZHhQkm3L|EaTj4JIu<;a@-yf5_;%kiiF|uoUWZ?;8G?&vAS5ru^MAJ zX7uq3(D>C0wA;q%Vhckl&wQE9;kxQWV3a}E_aL-nNJRK6DX;T2nEZs@6ane^GV2_$ zH+O~`_1>yP1l-2@ID~8q=TBYuYPOQ$dswPpD{uv;PIGvmS8bE_%%)?H!*6${tWw?o z#K+Iwb8TQQcSb=Gl-sKwjP!BaLE0ELYkj))khW9W<)UwT&_Dd*%L3fZ#Rh4+&DR@? zti22;l&r8mr?#53c~P+!cbKaja%Qp~z#*O1{FbXTcJ#?(28Bf0xL&MdSNk{ZJ>x#; zASRD6)x)^V&6Ma<1`-%^o8uqSa6RkY;)Dc|7u%!Z((e2ht?lyD+9ZNkZmCQ5KdaUo zVD?K(A8vh<6S9A6w>7E9n`abG;nj*_Xvz8P*bz40bB+tuM@Rg^FXXl#o{0omyHYJh z+<3rok0g~*=;6}R4Qiz`&r9{*|EEJVL*q=X%4T7Aaq()SllmMEu+xQZ+<;y2i^{}+ z5`$*dDg}#JQwa&5QiEnCNo2Bs$JM1T-o-It^{=zDPeVfyuud;>A|l;<6W=D-vGDm| z{u+ghiOKIZ31PRTV=YkWlKRMNp?CD~Xu4dvG9fm06$Xy9rB}$o(NT#%@&$Rm42ow> z>YXP*QHPTASifkVAGeweT>qWjd%QUw$fPMp8_8`qG1fT?pQ+s9uL7H=k0F%Ys*>Fo z`mDBX|7Q-46(9hT|A%b9S)I<4jTZur12S4%ESC);zx#M+-$ ziQHUWU9TR&-OUr9gk?h`>N1AF0XlOJQo-ETun{qjT=|X zXQI&4&FNA*YPfi#VOnA=8OgmpjU;Af=3J$ejj?PupN@WX&x>gve>|)u8T;Ss>nz#@ zQ|WU_l5}@kMRJv>Mc#eW8J!Hs(}|u8qZ0JonQ@AqZa}tt&stuF9&bLI-3Il2s$pmF z;6V)H-{}pXYz+kJYwLt7zfLtiJ~Svh)8I-_07YN6N~+@~V+tp?tHA;X$_~87pvg2L z8L7w&L>b2eN-jxYc;%=Tc7JCK0T$p3Rvj-=4i077I_OT~i`yVk>7Rf6;$o9OD@Y2= zdF6hasNIZ;ijsW9G#M48?x<@pVW6GD&cz%UP@q|Ce>AoK`SWLTK3hLhs`|+(XRED7 zo7AU4hULX_`Q3$ZJWOo&0?NX10cc-!rk_{(I8m;8G4U2qkxW2E9&e2cdH!DB$7c*& zA4sY9vO&%o@fNY3I4Qty4ARi>si+@z+{Ay)3u+Mh8u_L zG(qzCpi#}!cLy4&O9IW~@A~VGy>`yr4*e=2*{$JJf?TfaU%k6ec;&%QflJeHN!)p0esneR{5xAm_-J+aIw5fOX6AAM66HftQGsT-k9>Sf~ZX)R~6 z-Ys>6g7fLK5a*8{9Rmrcy@`Zg1Seb#-%Kug`lQ=e)K^BsMJ~=V@c_awFmO`O>EAov zubQ^b>~t42Ia@0!snh30`D$w+l=u#-+qF7Ra!rlmmlzey&AW1rt$^lHnmK;?bYZmb$0Rfd zwdeLMb?$rFEP!{s3z2V&xaOIKl2zL-)Ph23;rNWRMozOXmyK_SqdBw}q7v zunBu(*_M)a3Y#;P<)~c_8}1vFPCB zR4D=(=HddC2!=nf8rHEg^Pi6@qjz+)PireFD$-~8_~R<4@dpQRTOI>fwX_G%>dB5E z7zwqorK6~~7Xs=OJT^RDr0Ox}7g{(`6dV*L4BbBaP2a0qWiBPjmdFQx)n2g0XRFH^ z3m$$)wl3qPZyzRMKn?rQC*Y57Fh&vxUydfat0WUAR00RS?&>S$H@un>+ zQQ%LE=^~8wBY4X&r;2p>1aQw7c+{!3q7L8_%D%EQCU+u$IWYhJUX1q*-_;L1+r%QjApa;Z?ePWKhq znzp(0=Q)QkTVRD##3$p;)0XLs7St@5RFX$Fzmo*@LqY9LX2a6Hk1DLfny8|MrWYYo z@l%gY<-!rGTe&$y_lAdzAKtuq>5oIHfJFANqOi)=T(uW4-)ihSU$ebCx*HuE%j-V2 z4kj6tZX8Gv2%``#k}=r*)e%argB&6c?MT;aOKZMzxHhY3*W_(L>bk^BV81XyGBPrt zY~66?FhY4OgV=KQPaQfS2D;@iMaaRp z=d+GxfkxNL1u?sRSI=GEq)-`cm6#jehvL`W^Zpe2>8Z!7G+Z2w=eLfv6m=7$^CO>#;quct# zOrEMbMmD~~^w3ja62H1A|Ki!Mz|SO{!~1Kw<35u`QVvwT>}!L!Yq;Le@(L?HbsJ2q zwh7ELrFSoLRB|;0W?!-vsIxP%Gc`4(ZL!&f(|c^LH(ymu(q#aR?<`HXwaj^^V>(|& zpV5w?$oJ~tg)>Ylij%`4)dj&~2IFSPS>xiw_K+a@72C<-%J<3|znTODcF=G><#<*K zgr_9Wf`GxQrRBG6|iB(NGH=BMYi(&93-7CRx-|ZqBdX z)1(?y-t2ZzRsEf;TVNcaW7qd;+LD}_Fs9S!6MyE= z2Zn_W)TPA6GTiwnmD$~S5*-x#sNP&<5ZyHcyq#9 zI8H`Q4cpmCF~YeYIc(ks;t7Ozmd8PLOHlWym{)|cJ~u0C50t7|iNLwMygYa(t>Snc z2Hs+EesN(oSA~>}Z#LS%M)y(5?OlM?ba!{Brl!i7&T~wh6vfAr7#SJG_E$$vF+?*1 zM%@b;pP!!(Xf&_cklWlxvMEt)>p8vn)*9Q>Sqq=26Vc<@8goO3>5|35fR2)fY4y-v1y1g#zD?Y#0 zNM%sx_#i_%JfkB#$$hDV&B!PX@(@G2P%ED^7}O3u-}m2OR8G}mAq4GQn-}WCaK*3HwicBu!#+%w%)JKi@v~g}yhoA)D|2Tvr#BNQKny;Ez=M4* zhWrl6ZY$@||8|Dw^Zl#SY{7Z+fd1E))BQ;<0p>raWJ8S#o8Q24#C(KFcr1(BO_=)x zauriD9maQO8jTn2%C(DLxLjRkQ^*TAZEpL8&c?DM-jA+|Pzwj>H`_WG6Uj<~vY1{a zFWz4MC|P23o*oqsqhC2AM!$C(wHwf$;k9u-F}s?odFQi@<+3xgP(L|qU)wi0xOP05 zv@_i(!o{^QQ%P>uJeROkbAhwzT2r4Q^S|xAR>)pEdOfH+2a|oI5 z3nH>GF>${x;9_FdxWK;qV1!>@x_|RKogyYOOHoYXv3GDF{0CIA59YJ|S)x;@8%96Z zjAVMswAW1Uf>kL)3*cnhV^gidSi44qb90-2IKFBay%3z3eOchfx0gT-qM`CQ!xSEc#zr-RoOF^6deoI=Z~)y*ZuA$vtIR5kp_}keZ%_oWXRojfP(PtuaOQT1lF; ziw>+C)6H*keU! z=sxN8D?z?mVBwJSS?e{N;Q^;zaTdTfW zbP&?eoNy5;=gTu@S^@jzAN}Mm1t5&<&w?md^4|VWrHh~v4>vrI^A%UId#psklC*d4 z-qo+Q1VMrqSMYpk+%y#yLTdM+Tn~S~045fe)mb|RQ07+fLCb03a2tyTlU}@&u zCw1;Zjc4#ujafe#9yM3M+Bt>1Zk6>)kjSJ?@dS8IZ1`{WShleRWe}gm>##bwACB*i zaQE}Lk}aMY|97YUM}z#&=kZnuKg-=)g7EdS?@D)CIIs z?EhenYufjJTk7OLvi|RGvbb^tQ^veqX)R0br2s$0hlEnfLm(PuwBtpxngv4@7O%^m z>71{nIlu3BV;j1)UC?$#lrgSef>zO{y<8< z8J{IP?_9uk7u$|Nd|(Y1WOz6e{L|h!`suEuvk z1*%Kktyy#z8;mn`F&C$UolDm}x^f+HHn6z3SdOiCgIeVIVz+dwSYgGDaQ3+-MesVI zIHG{A@ffLa2417ILIm8RE8@}`_-p{ix_`5+P`5IOWVN~8(2#&w`eQ^~P8eloqg%c- z9_Ag_3!1=B49Wv;k%K)86R01w=+JO3mt)}NiTfgh?rYhO z7^!gQheUQuQoDr9!!|>~RILTNyC7Q~n z&3AO`95>#J%wyd&Xi{*2tx}Mb7(C1%R&ZUP2%V|S>-kJWOU4^Q#F66x zUtgBp919e1&XNuf2snabVr7ozTEGmUTWnCIhnymP{%0cA2fiQcTymE>En3jNP3I|+ z2ZiOx$E_b(&}Jy599;Zam1`ecB1Bj;9^m1TJ^r*hn5$JxQPRQbxOmf`R-B58|5+la z43MSK=|*%fzg48B z_V>7B$peWrMBaz9X@}X))A@X$rA=ksKc)}M1mPM0K@_MgcH5^i(tgu*F#nN?^G)|F z2A&%hs#BHL#fpOSd^P$+{^&&BY`?NJy~X5HE4#(h z1?1LiR99$U+aj{G?-NV=pd%Kd+)hDu_pP+*ZUmfPM5CU!(n|V~)=-Kd|FgQ6Vgq_H z?27lhqq}>iv$KX)ddG@LWwZIMV-U}jPK@OTtFF8AZrPK2{dlG+MNKXBr=rZ4(q;xP zFDR@rd6M{_0{w)Zn9Q@YWbGHr%|@ z31oPw^K+~e8IA>ag##nlwvBEesQ_uSvMRMU7~L*TN;;Nt8;mGZYPdU^4wSqaY4d+w z-uPqTdNPv~`sa$nAqr9LjUFQq~j{~`T215LX@Pqpo>~4*;pM9bBspQ zv;NeSPAgDw0cANDe~Q{~-EbRZY6ZC%_%O9+EH5f%NAIol#=)}j)4qmyZtUOpm?}LH2J5`o>CkFK!(Qcm> zmF1uHE*#w4NBoa|HoEcA(Pe$@xfbtz9Eqr) z-yp+h2Wskn{wxVaL?f}$N46~B&ODIA4sr|3+sp)XanT6BB=Q)R=Mc!}(c9o{PvCQa z!Qp&^emLe)g&nm)jVS1#Kj~iRf%oFbq) zSpw>AT*0nm5t6BHN_?n!AMJ@uxjOUN{h98KwmFwX#*=}LBa_M8K&Ga?(FGB3jfiE> zUN;9oAXO+|Iql}%>u{`JMfXS!q8|)(gmT-EQSysh@;L1(cwDGNGyLGB)vmOoX44Vh zngEA5kFA1{yO>=Qv%My2*eW1_U>UD56<`h0C+jCRmLs1oj?!)CWYX`|7a`*!#jGZV zqUas>9su#Ac`12ZfS?;^o$FtIqTy^P0FdRxI*~72O)XCv?_!mlMN@7&(Qx^~>0(z< zw>IkM&-G~K^-<;SIfLy))UUqCh`;O-%#37E&I!ctHc1!uSl=pTW|9Fv0{N)i#y7!X zb^ophf(M6O3S`9cTyHFhPfLV!65r@hIsv63fLxhUk)@?mw^uYBxfPRTu!thU@z`F7 z_9q1!Tth?x`l6}Z1R^fGDu=6M>9@c-ssvPx73?Gj|K#@CjUNnz&{@r-7ie0HG9+>bI`7hH zYnNfMNT#M%SP?76ixz6DA8$Gy7fjv0)=4Cs{@>Fj{PUcwz>ytFo-c({RtOmcn7;B6 zJ>zJq5Pi1KwSM{b#h>QQY_)nYkwz7(L5)fV81Z|y%kB+!_-vAGfo~0@nXHv97Pjtlqx~yLTSI<>C>ZJa+r7ti%GFH~oHbwz*=f1dfD3fP~zKHd_N@#7(bFq_C z%>HnzogIXaffp-M-E8wR>-#b&l8Zy0_evVHZ1uy-{mQvH;;XY-1c-HN0H!u5^3(@s;XlJgaBEX_5>J!_alNbP2Tr~=r#r@3-sdVp~0T#&$rcr z$~_V0aAyI};6U7p`u9~d!Ta)Sv*f^xL&djLWMJ&jfzkJb{2zGubIck5ZC4a2uD>jfn zTpRVnF^L0YJQ~`00Uf%Z+v$4aDc~6L4TZ|&-_a^2jnupwtLGDf(*5npOYSEHdU=B7 z1f8ATOszWTDeQqtx!LDPG3oI8Mk9mGsF8yz-THWKrwi)t&x#G%oGAwajDbBA6arN2 zzmF!{y=h$I7AF@6AY#6SI=Gpk+lk6-mE$49ZS)|6>nO;EZ&8FKiQF_C9F80WHA*R6 zFTd=}wwGM5D?^>TnAnfB)fSo@Pub7U0t|x6-OeJw831Nbo$>FKft1-*&aB+g2pVo% ziV<<+JJ?xVufPTy*=qkIVM9gP4P%mL@y^>{9IcQ0?riy#$Xjf~FB^y8(?BVUaP2%nRM{#nT_y$%Od zehE*yN2BLGw6_K1zMd{HRwl0YLF%thoi6-0xY)ep{VDR9()feH26A^h4v)sQxn0=# zowQL$x%a63Dvc~Kb14PYOlX-2z%ajU90Nnp3-Mer>ECIgd$Vi&qA7VEie(|@j(NNB zoPo`V-I)?4ioJtmUQQD13Nsuh82Cz5QgW_-v_SJUZ?FL>LJzHhE*A&eY7zi@y9S!; z>o&7d<8tJDLI34?Z=CvM30-7>0meN@q_|j$;42MK0SH7kFC~9i+iVV8592_-jPe zu73Kk_)F~fQ_CUIZGm5;IOO-+hSDYe>(xDDU^^|_mu-2!G3VAOCX=E^X$S&FB@&W> z)NIRfa65`TtALwlQKtd;|A0yCR?6Uvg?WvHUiAW+atxR=}OI zEi?>T+Vug-Z@Pa+Z{In6(+4qWb;L=alPw?JFI8>#SJum6_~_`d;8a5WNG^t!Hbf4! z(V2LYIXOT1<97xcU_5;gt@@Vp&m)&DT7OvV2yIP5E%x_rz>cofJrCw!VhXsAk5!XH zTnmkn3VBeQ3~cHECbrOyFwir=T6>lO0irKKzo?GcKxjflsU6Kuui3-QNQE5zV(Z6| z!!o>jHztp0S9*q*yR)=JY72*lrph!Lq3IL9w&aJ0ee0xAKks_-1`B&*BTJ2Nq)A+Y znK`iOoI(51VB&KwuI0u3Wm`*@#uMG|F79+4+vU&K>2Myaq_4maXEUB#2r5f>ZAieI#~*k>mqnb$?+EQ)O@K!ql5xLe5gWt~qp#-@FuJ=Q;a zcnmWWpn%aeN?Q&CnL!5|x$^2p|9@D?H$%ph&QS9KAv9!9o-kaX{Wz zj-l0n^+N^J03V!!HJ>-${*be#b{GLCpjHNEj(X|p$mY1?F-s_^grr6_d!?-+z>4y* zDrO82Rj5FFN0i)iAN1iMLPw-9V8)d38Np&1t8=3C|VxbSGA~^<6D;M zr;GV8(fgjjaygnoj2&a{^O>(7z?6O_AQMgwFyGResSfwN2#jTWNK3{Obaj;!06@5G zX8Ko7%LQR={LHn1Y^`DdLNE5E0hgM{=k?8foMm%1t|Nq|IJTi7Po6j^kcJjqC3_YR zPAtIn=<;CV<@P591^|G$*aqlPT#7IIOUyJMnh>!{X*uiW;6A%QyHA_dasPPyR6;mg z5~^Hkpj{q7uqz5!J2x3_T^P|Lp$QkqH1G>F~dLe)%Ra@y?5C`i#3u z#BT?M^qr>@v6-(b0In|o9u~_>kPruipc3*_@8#Y}&eV&tU{~2V4iVU}bU0>VSnGXq zd0}CoCs9*SyJYP^(%K3PnzG<7;MM~NnxteiLP|Y1$UsrJ)+4F473f-2OQ0&DB&}^5 zGd!r4!on5S)6sWFm6j&hF!6~0YGF#oZDURnCSk*EftC+>Opg=(KA|-j$ycP~jzVTs zatI>-y2KNj=sTb@X_x|@;(kb92fHgv?ckX3?@>z$RuuM<^|uxm=*M)Ox$8>#xdH0=iryADDoY0(8!ldm$43fJ*O)11BJ=%j+|RRsMKu^t%r@ zn+Bh{Mv(Cl{drPXQlL5m@>C!Gmjk0oqs>FXeVIySn^!Rj}^tS~bB(3xUlK%HoaQ1+Y zcidt!D3Ln<-am{wSJ350$*q8Z^%))k2S4Zu!QTU1R*9U1j4#I|l@bD2F#tp5QazYJ5)R6x@-YrmEf`CE+;G&}q6by8~V78f)bJ@MOwcL$@lPwnf z$kV5(xr=QZYiZ2PvfFBM@n^0V$J%6EH$hD{VAYT~Tv4IZj%;-ghC@%X_#J+*XVabU zNdUD#`%spJ@IRn*f3{RyY^ihfXRYJnM=)HHT&3ZUw~r8~fB2|&y})(j2ee{9C}7CT zvH-ajzH|Nv&ar3>xm=~iQk1Ep(hw5>Ze~KF+i=!BIIl7U?A%A%z}a)nM|qo+W*u=( zd7O~4=}DdLMk)bVI`Bw1Ra^r2ClUoT|03v^n4*@5SSbi(tcneBk~Ee5wA8TCd_lPG zeKY~)zgS&M&5v#xSjTR>Y`T(y&H&Lz&wTTZ<;VqT(z;+IXm9V+7y5rMktd#MOd3eZ zjJOAk26nJO#|y?91qI%snj}ku!ubga8krr@4relSCfZA^AyNgpF|MgfDYOX+f2_g> zX`hQ|WS!;cfjL@jH~IyX@Bb}%ah&OoXEDqBhi1!FyMOBz1h5H!z`s1NfL4Mb`#uj| z4;L{XXTik0|5wD?^xT~Hz)8*R$aK7`$fz7-Ajjo$#_4ec38j#=nf;5Gm#d?m`qKoD z%h}A1>@NXUD25qW(e0zz@*1U7Jm#WcjahexrlH2KR49poPWzZ7&rm2RG+RBlYfyRH zGAIdnw6xeTadUhyWZ8-9$xtns(CCH0y~S^imnc3{W>orxImB_piw|7!jzaU1>6%M9 z*~wgfZ8;u?!|0evAbg>BV{$NKe4-kBs~%$+;sho9>6Rv1fb%w<}N)``64i#%)9LyTt;M zOhja++`pj#G+hr2OrJDFIl5qm1-hDloBm4oUqosYN--*_*gpniueJ>|P=q+Zd4bOo z{TAX74^#&N8$}i{G!<&uU{yx-0xToorUm2>;;;tfFk-ne#*9+ff8+oHF8KG3!M109 zmb^KcTWQq}MtOFqT%aA^^En-OEJoG;-ufPJ4l3kk0v5!@ZgVCE9hVM-4}O$1;Z{Rl zjg>P35J>q{6qr)ijEU%CYt|Yu=x1{>9C$kvlb8T;Wr;aCR}c7~xmpSNIJG$TzxKC@ zC|(;@!y#@8zr@CdM*e-91$Jj1U-LsV;Im;R5^%x&vu&5fU%HmrudM&%x15fesYq|~ zVCL7^{)qWP{O0lIb4me~dS~GHcrF?6huR9j_&}qr!J=CqyV{>Ag3MCO209N=JR~1Q zo3G>`kP2tE{xoH;D_Kn6KeU#29!Ml=1&-Opo0=sG9&k0pg(IR4aA$>ElR$=KoZi4S z(;K(Z2BOBUO^zTm5JIn>gALdyz!$!z1HXFJF@&6|AN<$t^;$+gUELu+BXi+Teg~_LW)= z&+XI%{9~1TC`ZGCgWLRY&fs?LS0Njtps0e1dbEoIh`y85i03puW2T{r z7d!>@j-q^wfUrf>`KP->!;vlu@9`lYnvosNNLpTJ*Ms%H zDc2q}G_0eVuhyEU%A;K@9>WqZKuAM#fcr=r^!XH`S@3dBAcWLsXVwaSVXG+PN)M~K z9xJB_UYqh=JUl*5&KKbAxF8oPGvli>+toRSs;c=Yx(7T%uUh=be0k~xg>Z-u#=G;! zj*gthH$CAv+48r%D=RB~ecR_7-Bu_cg0e~wLIDL5&mKUHgi#4yTpU}09JX{iE=aDD z@gP-AowzE5Qs9f7k>f6FH9Webb7_njBV1XTRNs58{a**n06ON|l)YO-EuedEfc7&>y zBs<=MAKm4%`+Z{bvnkXcFETl#Bb4hU+Qrd_65D~Ksp)C()>R52s775Rs&7@*wof!j zPxwyt;o2a^KmRZ>E%OpCFG<(bexIYXwRMFPp@oDDW!>GJ@XcpdPe8QfvA^u`xC!U{V)42%lf(1Vp`^|s@!$(@~@#>U3tq9RXTGSoz|NX2t5M-MfA zwVAEDaSMIfqcDyGh?Cu*b-eHn7F^BN8e3gGJ|W%x{by#M$EII$~n6jzW88b4$o>g8e$q6*ZrG=Xedq1bIj^EM2$<=*^Z@9I!#zBNu2oQER2nk7EULMmQDahD@<)I1sF=}$x zxnD>854Es6pyy^5ypTfBBlEwHXAhRDGL;bYwT;cr=H^U^f#Ca2TEWp=<%LSC$=PbV zn3$Lq%Ilfy#=!S`27Ji9{&>{8^No*SqOVz=J^Kmv&v3@`Xa>b77gm(K=%^J)nv~ZK zYh3qDwkC>KD1(doQv_XrmJ$;aOH54s_ZhtJww>||2>AK?$5rg!7Squ9oSwc|VKHhp zn9BC|0dQU85HPTau4*0Dt}YLi-(%2HgGl{U;A88Fc?4GH^52o&a3g8T1iJtPtAc?0 z16;}fi@CP|s(StYg*PG{0@5iRN(#~-4GIbfg7l`PL!?6`Bn6}dq(P8wR2o6LM7l#t zDFq~iyEfN^S zwW5PQczU|Y|9r7M9I~Gi-LLKC6cmWsjur{ojs}K=^oWzAj#aGC0pD^SP}0J}!YFu+ z;@q#fBqSy2g6i=5dt7YPpf_8N5}p`Ueq{98@%|>%>Vi$GuB-&P*sBE|3eHuwdf$^8 z*&y71=V~5;^FsdV)2AOlek>^|*(1zRhos@chsDX+*-Lq9DgT)T;niTeZ<&S&urE~P zWwcTa>pjY0QjkhJIy$-sh|+ku*%uI*OX%nX`1t>M3ZyD^VqzkmTOW>UC44y%ad6tQOk1-cx5yyohOD_iC)_8 z#`bNLh$@qhw?)4NZ5|Ud1fyQ2Q(myJu+YxE6MxgHzf+v(Vxh&@@Qp>?Hnb9X3?UWS z=G%MLZCza61#->9t(nT|>iB;J$`Axfv(~}T%*>2%@Vo@dEEeYHLoG{{f5-C#PTu05 z-6cuy1JLoDpC}qPA(gbIrUqiLwzKsg{+$}9Y3okVhYugBtM?myPhdnt07{0ySiPZ= z>AxdlV#B9`n`RQq*sTtv(@;@$i6>+&08#nj!-ozI4iF@c`FBv74 zXN<5G9*0|W;#8=~hn15Y(*{GyyicD#Sv5*NSyR7#ySd);*D{4hOqF>T$(1XwiVk($ z+}+cq{TokFOTm7lGQWmR=GOfuz`{YUE|CM+y0SJz*Pk<=gqYO~H{lIWQDJD^++ z4hZq_X(1VOqS6W$I6FIgg^uDvwMLL6^SzX3LIA@Cva%Br5?~8Ge|=X2zVg}xO;+^ck;m{zw7i;HUvF;&`=_~FI5b;Z4mLLLGosrL*#QcL>_hmpE2^kcPS4DI z9Ug}EI$T`TG&FN?1Aro@mm%YauLhyIDJ-n>;K9jL&Q(6t5hkO7ug1o~>3W!zpKonx z`E;p+z!J~%U3RuB`~*wo_{AM2pvuFD8i>ZT>!G#UWLpKvp%bkGFC!u#8Tw6nx^d0D z6p}v;s7>TbfXI$q&BA^!xnJxc6tG9O+5N?*1q2gTOCqi=ka07#C@af!d$!>X_bCe< z-P;sl$AhgIeb|f2=wE4}DA@@iN#rw^vXoI!$OW}>a`4lvM*SL8uRQGGZNe@kBO~*y za9nE(&<;m>^8clm1C#I>%$^EN%uVd`;@GG;9UX1( zs5m%SkKK!E=?q3!Cvb{!C5`=i`ZI`-w)q?jt1BFjkHUj_?iy}j3JvOb5TRzf_Zo7) zI+b(YQA{=ATrK+g)o__!G0H!LkZjsnJ9@iOWP7|^eJg$S{+aKYfl0^TB{0Y89`G8|LX|B`1RzKV$&Bvb559CXwk z5vzO-e0K3WET*j&kIR$rgBy?fgeF~VI&qBr|Hjv_6bH79Q?B&1b>Te7o8RC3aaY<` z_JfO)Gg`oB`QEh1*#?blOnIijpp}BNd6Nh_JbnG~iIx#ka`GgJ4>qW;xz)?4w3tV>johtc~D!dF30PVCi|DQZg&GPEz-!WNR>w6or0t z1H)3vr_;XhaGT$jT%M`-&FgbI7MMU*{exJeZw=mNe~(isBI!<8`PCn^#j+yi7oRiA z?P ztdylEB%_@nz9z;Ctkp6Oe&VeyzFWc1 zWC-tG*9u4=pKWN%kRb|nng?2iZx}K2H4=E|3|2r^xkd zzu$Vw!E~vm#p75oM4{A}S(NHq;p%AgGc?P-<{|p=*nY@80=RJZDWT$E9Z6x zKUzu*%gSY(_!78ZyrGJs>(PC(g1a@{()(uc>q$9wPu!8qHXRL3PL7YzMpqY(S%=98 z2R=Q%ySsX6vkfBdgzj7OE$9(6A)F5eYfC@J+l~dN zGfYpr=(0S)Fu(Vtzc!34vzS2pTg1rLkBNwT^qFJ%hT;gAdR`ut@DfTnWSzzsw(>bM zSq>MdUzipsF+3O$q|jlD40!FN zG-L;x>Vkgtbp#3WN#FhcWu9d&9mN7o#5IzZE*xg1ML03uY|ej^g8xgNj~H(_DGZ-z*VQC3HEpD(XX_Ojs4upkeQcfKW?T1hv5!Wp zIN5i-C-Kh?S|rW6?%URzlJB*iDPLAotFj4d@b(wsL+wm=OhhnCUvf~PPSfe%Lk2`| z(z8Mx*0d*nr!O-yQYj>TUY;B_ap4eR28g*K%<->(^cG&V*Vc-Ti<9%zB2AN0=t=Mt z!y!*5$`en#EF&|K{$_p&T|r@_klp$9>cCB}*NsB{Qlw-z9ZHh>s%@#-QUkN2mtu}b zXFjPv=&3u355ka;{8dGUI_S%tlu_5dedt0%s1y9GdF?boZp8Q+uiI;H1|W^TAl`1tf^YF z>AF%e&kC<&QX$(cwUpGv$w||br*(BE{i&szm;C+h=F)ZpFwoJhtxd*w>b-)59p|@r z_;206Pb(6j@>(EI7cMM+f7SFf{vX_InCVh~=VwV!I*Gi(k5G<2R6buDFF06q?%q3J z?~(jxJccIrk--ED^9XVARPn^!-^L0GaJ^6q4OVgDr2--LNR$d!g9-bD{i%?tUhH;ukXobM^G$`T8+K^z(-oA2& z8J%L%n;$>s>ybTtXgyA}zk&Rrd&4Moika|Y zdx42uQcRVKtgz51q4oNG`}XZ;zJw^TXN~5N?C~rCBc@u;oaIj9*@RbQZo%@r= zJQNi8RL}3Hv&`NRniG2}FCX|>=1$O;g|@YsPY9A%=vaeI{{6Kc-Qn)r>bbat42lD3 z(q-|LHzg$ImtGGj;$?O!uWhvULE0A%YME8kmAxr(AM(>*ILz`a+M4xuhco2>7h+?h z6ej%kUd4c$pE#Ui>B!Z?Lz$QS7+Glv5|fl<-m8(oat)DkYvIUs`UA7`oeQIf%y9S) zrBQw7X0nZqZoTdu4McM+tH*C_PR{WP3}&S}<(Z~!>5n3U884_iY|4%Auy~!kWFx z*E#fKB1uwzRt9Aa6O*a2niBqFMy*`=zhzkccl(Uf!k3u@t+`kqs@&WcWKAt~e5TDx z-Zs$5q~VhZ1$|4QWVak_HY0FbwL>B1v(%^*Y6$Z+l$gK(Ah= z>u|?D=Rg@NcW!8S@ryBvxqeq&?SC8om7dHDK#0+=UtOJ@W8ANyRv?b1|Nq1?{O=R^ zKQb`?|Nb?cUE%@hB$Sl8;KbPlF~xBFs*aD32M1C00wf0TG3@CTNCfznEciIK2N&?u zim(fCDxZ>T5(hnCp)F*!qRMuxqyje(wIT4=dpL4DoH9Qrs|&RY**>YG{`O_qTc&ir zwY9Y;>%$5voG0e^tgN}nw%{&Sl)=lCWt!c|j*s~G^d|sE$;Es^Lh)DsaRK}^`Bf+d z1qCa}p8gouKsk8}|Hv>yJ0o&%XlVUgVdC>vos;209bV7&3yIgVBMk{eyA7Z~yQy;x ziD>(UEvn^1njQ$v)SH=(J=$IZ-q8I|Df^V=VgCt?fd@4at8G%dNIv|WsQ#D!$O1?sHfEdO$-ZiG(h{OaM# zz#M~y#>D6zCT&2cU2Z&AOYe_DX9MBY9yIRCgp$DTd;w8K1dCDpAsyp zYTG(5GGRw1DSCv>SnvnY-ae#d@SXUK0vv)!UjG!~QYtAQQy&R|^ToxTJ=W8|3rpzI z(w8wW-44Qaa=P7<;N%ogP+)La#!_BY1+eB(je&|v*U-@J=g_MHK*EAgV>TZfzT@nC zGnzqTM2aw4hLNP8O<*tq68S7y$iHSU@baJI#aT%YFSyMlJY!=ds_etlBsJcT420Y# zDSlLGdHj35b^5m*Y+{SI{)G~f4dQ@Q#LtbsA}*JWjMj!R=)_z52_HTjwijeXJKSEH zm86xzzjEct59rc5n2mKOF~Q*z>+2j16>4hgmX;RUSEv=x!X^s~T=?;*$SCr=K2vQ} z_{74>ZnpU_DaCAB1Uw3zQl~?nj;Q$iHHhYM%DV~I_=rkMh{BP!cuGnJADGfj!U`Q7 zy{?moA5G(4U2Ti}Wz$HE*Wr)k53uvC#K{FP}ppDZ>4>LPB1irH_w9Kp5HeMk`E3 z)Ly*oTEuNip;1)>Q?tadttohqSBKI!_ z%pp9iMydIL-wboI_RDe!w*$iRfXEZpzF1Wy z_|LKATFwz+S^ug37`Xkb94SclUQ8Md9xL?~GgwdwY3k zu&qAGY3Svrif?L0OuYNehK()XkcNO7m@MNHGWz-n-WS+~u`(-Cw+j2$cjWPt8;>+k ztuX!n1T^|CtcN~$XX41aZN0pD=4&^FDJ*;$(=u)(AMN^e01))5O-#Y$v3~6PXl`l> z0I+v@)i!TfH8gg1TPm$Q4)D*;b`H)2?tbpY|2btg&Zn!}Y3+`ZG-dT>963GeVEmS| zzi{{rKa!fX2kCiy?buA zvanF+%D%L{p^dN}1bAU>?24F}wEhK(x1u1!hQGV(^dv9@t3Q=9bZnA>w}Zjoq>H3C znRQ#;e6bx5ghA3fIyE9$h~JJf7++?hV%Ua9gmS6|dGho!hKwOyTZq~56G}?uLQ#~0 zCLz^r-C01Ntddd2o}Rh8RTp!9)|Ri~^t8kOoNTpR*T_h>20!EqdUF68r6@^Xa#x9) zu&~{1gxMDac+NMivMkjms!>rTtzlBt)}1p+YT@7t0Ujgy1etZ#q4lp$F0$|Lfhf#o zD$7`Nu-)%C`^#ZS&8%oL%%b(4C0*I5q4mACle3H|lVW7+a4u~QN-$W+FrxY76|*#$ zbMh8PL2!((X^`1(ZCMXpLdP^Ybnpz(bQ&HmbD;a)2l>t&5iMB?7<>u~tG=f%scx3J zvUeu{rHN94k}2k&SS**3-$9H0?k@OOu(&Qk6SA0CSg8`B4c`2o74>qQ*J>TJ@6o9y z2Yz$|^LMm_n-t)fy_Iq^EP9FC-`-OSTqdAd+%@z3f*PDw{i#-KbaGe}-R`MqiIGA) z7Y3Q8^ZarkEZm&G*^!H9$e!zG)86EuJhhm(z^m-@TTu`%rTZ!jnZYlx#v5q1XEXhL zi;TFpApr^Q?wHdnDkR}Ug%tg1?U6LrgBd2-Tx@dr1zgXVU(u46n!3uoRzp?6!B|ysK&k}SgR<4F1#mv!H#4_tBFDTNc}s#3^bWy2RrQiXXi& z_HVCxEb;`qyL&I{`@6^1^NR|My?!m}4UQD3?0U~V!rY8V`bb#kg-h&fS78Ky%NUel%z&I#RXd;5 zXvuNa$jX}7gctDioog%!^3-0uy1#IY?>MJe_|WOe=Y=)}Grgx$EG!wGuJwKd_wQ>Jb8TB+ zh*VTP8$J@5Z$y`^WVUkWafnuGZf;BKC5vv-ETiOklaUevTU&#AY*toJC1y1w67)=1 zSe<(*$IK!se5%~Im&^yt(UZ7)?8ZpvZyY%N5rnzil^v5H)}7zwT)*YSx=2SbC#8`~G=XxK}M8P={bijxE6jF=@JMuy5&5plEcsx0h1;-n@W_<#P) zj}(x^a3lEmTP7;&+zLJJeM9*svB47ie81OX`2VEfeg>%UA;I}sI5G1wRR>Csw0`fS z6xB>acFu;8B?(`VlFHYRo6WqCSg*PJ;JF2a;zThhjg<(5=9`1Af((IuTwLi>mWb!X zRG}N+hCaHf_~bv*qb@4WEQvZfT%p57*(aHYf4_e7{AFSn!DDDnq>3Y8OFmoQGTY#S zXR@Ki!3kRP1I9I_urMd*XCcr~>9sotN(ltbx_hBsS-=TZm(I@w@CtI)<`}Nf&rjSG zo$mVGPX~5RgZEHrdQl3GL`P&>%L^FHz6ClRCMyw1&CWDx4MIls;O&t~A3PkRArp>1d3&mZEI?k<5G67!R%phyPQ zweqiJlee2{l}6Rr@6P!P*px@x4qs*>c9@NGr@fe^wu3L0_hFWsttFl&P;|7k;e7ni zo36dGJ&Ww8ov7CBEY=UvuXS3FtzWwEE{j+W_yBdoCByAO4-GGG^d%;`mThH_^A5lm z0_vTu+s5l(vDVi`o+!^cUu@kS5^u_bAJ5)%c^xKL@UadxKik=B4X3EE-s~i<_X(Y- zv>19wEMQN4FZH%=(!Vc>V`0OGQCZ#3FX$3Tip}@+j=LUW_{n@|G?>78mX@5n!nzQ< zyj&)pD-{EEHN{n?UJZVU?cvqhrP9$7Cm$BJHco6caHk>j6$7Y7O#EmsIRyw~j+e>& z-)%x{a`Tm!;;=}e9=YCIq;;N~w*J#R9QD!{4X*a>NErL3uxasgr zitteVhX`iQ4>lvW+r#Rd<-*hggIn9s(XsB7hh{m<-%b_7DA)aWsRTq>@XO10_MSI1 z{CL6nL9Z%$tFmo+CNn3HkS=FjUSlj1orGlmC-=YK|Av+%c&augE$U&hIfH7~w|dd7 zX&IaMRU7W^N^QaFnz0Z6wg)^z>rVXgb80W@B6bwr#fSvN!=gvl*!Z9&&cJPxi4$ia zO{LJ!-CfX@t++cpl1(i~>NUz0&F@70@)3!UO{Y{WN#8gn!l)tk&Pryc$3X<-Buq?{ zhoC}89G_qqI;8^r#wwQGaWd<}0J2C(W@bRj$cddjF=eNc6iQ+G2I0sdkv3dl5FWW| zl@fAu7ibx!3ff^~U>JO2`*%jK;j6|<-|%(HL{fu2iyQg^Yp9G-o{6)6r(f93Eh*)$ zGus6+q!A|x&QG9juPD9?9PC8iT+LsC@AMr(NMf=!(VZxkie~96yC;?m2#8ja z`F%<$A4Ic0OP5R#?w>yzTtXD@0~uc{vutC#WEJ*^UIfaFyVCeRx1-H_xQh-BcmLVJ zGq{NGz?ZU-$Y^NH$}G$4B>odMH0}~b4vxfW{8CW?uR_r7WuBTwCELOF0LS_NOK2tY zVR6K_(V}M`M7tCEJZR_%#l=TTUnOuOZNenIfx8LV08wa}GjokYP%$(wphNtx(G|2( zV=FDFI+HB;tU+>PBxe0>kf2aZlI{-kNxLZvo2>q)fWp4G=m4slCdg+Uk>FQ~x$XR3 zcPG4xu&@BT$L3o`UhqdZaH10x_9S`!iR@LN<}eto@vA3cp#>eTfE3|!3%uvg5w~vy zXEQs@w6=xR`O+lvWNNbJyh9JDylH3OWGCg!q*rsgcZ(5CQPJZMkxog5w>v&v*qU`5 zPvrh48W?XmIVR(LIXSm)D1M2;#(DM%%7tE|Uuhu$3kY{{i;n(UmEgPb%i&6-;)M%j{PpjZLk0J1i z5JwCYgeq{KFm)?(PMy?=aHc^Qu_0z_8_S4$<+UL}7zl$8&&A!#Y$<;K_A9Cpu;8=0 z*I|o_mCUc)9Vt%HSX>U|H$%*KU9J1nHboOcDD#L4fk`0SqdP1`WX zC%ez<);5>~we;dql24`}`kSxQPJ$#rG1BQL^~a|{DKwuKBQ&nRqDT9)LE=XL*xEdY z77-VRxW>&L^6duq!U_h0jg5*G6BDs>DjE?940G@}yW>61&k8jJhuM67U`Rzmq$OJK zN?TtFVz6#EG4?w^76)eWYnyGKFTY|yk3M%E`ljGgQN{N9LJ2EOcvHr{&dg)`^!=Tx zj1AXl>6fn8`5NfnhkzL{G9uLTdasO;lRTF`|J;)}I0AR`hhi!^0z|fw)ek!t0ephY zd~7u}z%Cwt<#T1T>}&0e(XA!K6_qH;n#XBtMsd6JGG#gG9%P;@u40uH6flZXaa{hQ zu=e?7SnT6TlryN6WN{ymD044QZ4Kf|OXPP|gL+8P8LZ_sYU-!_N<#e1Wd-)?mc4p4 z*S*w$Fx-2oopon#nHR!Nwq}fZEUn%@z@p%}CP)f+@0>re~Kzu|IeUkbRNF7WZv9`}tK*|y4CUliWc3BcRs6*LaQtCEui)F^oj5C{y6D%+5W zN@B{FpFc;qe);6?IxDZhx->18lFtFr$pr4qti8WagV+KeS3hd-MECbc432#{-X9zr z@VJiNT43{RZt&q9=C_ykH_@Aqeu>_QeUj9;es%my(bK{rwVKUcq7CrK5=;gIz@_zj{|w;4sC>Nkh65tq<~d}_e8^XI&)5FZ|#Z2*{o zmUey{z(|Z3cux23wu~8^s=f{LK6F|N{@30RT%K-=Rx&NGXO3gX)9bSw3w-G!)$O(IKVfC52C{u2XDyZTcH6{4+u_h)`K(Hw{3``BD=Dw8S z_x0m-?pFXixX6Xcl4T1C%ruhBf4?{>G=cTN#qblRw>RhL_B_JTk%bLq`0(7o-=1~8 zgjxcPryzcJ?pTlFkCpaFCB<&;S*#NVKc1mB5|^`S+wJtGL|Ed7mr2I}tJxUq%F0R{rM!`Sta6 zQJ$BfGDmykRX8uOdPH(!bKM=~6&6}CsD!K@hl;t0oE%+OOKu*F_T@$CJJHwUZ5hP` z?Cd`#n0rj_q}GW50O$ADd~EUbNW>}NExs`X_oiw^FK~EeCP)rLfCXOb0G|CyPtQL- zLB1RsL$YB=EI6&B8$3H5Rpu5^*hC!tr_%vDu8+f4#^ zd7mFOP#b>ietkzw zdYP4VB2bIz&!6UD688JxJ4m6O9?(GI#pW1f|KCUKu|C=UG54ww5~wdc$%TWzP{V{s zQPIh{B}1kHr$Bz~E0&bh&h9g1mUM)v8PfC;M1DmW(-F%YjA(#WYGSx=b&8Xs1WT5; zR%PJ1&9}i)20(re1@O8Hn{PvA|3u|V0ykoCxD|kWUuxUYPHs%kH~i>;uKt$?g)z5{ zL;&!bEVQuL8d66VG27t0O>fi`+WF;|(ZS zHBQG?PO4*2xK^Ln|XrX45UN10x3295ZTYzHs{pk5A! zYnw}3hSoOS;;7X7EGDK>Gt(ze%DjZmpqC82X5_mh{ff((nd7KTP4momSUNf$o|!tl zFWdK`6m7DyAzrPe<+M&xiGg~WA$p7+Qzo1o8}CJ}FezCsg=YEwv5=}NK%7+^w))F6McaV3iHV{rdnUCj;k$03~KIduEjmwf1ti5#I+A@7-wEv*Ch- zp<=xxL6zCe3)L-rNH`dvud?I(7bMAGpowJxZ6xg2Iz~ap zrOcdT3N`jbM3ILM_>|YS)f-ov&iQOe!M59)Q8=5J4GmqJrAy>l*g-y7XWF&~X?+=% zcglj#&!4Xk&(C)d97(HW?BL-(55j=_T?5y*uFgLwj}H{W$wizmZEWmCu{C974Sp2z z@pay!{qgfZOAwzPh!#-FyYa0pj#IZHr&*5cp{6FJ6liMB{{#XDKwVZkrCF|4lvE4L zDq5vH+tz;5ui0VX8X9#zRH&5dU>Hh1M~@X2I4_XERAC|FQ)jZo>2SQ?R;VN5`;o%} z5207p`~pi^H87s+-rHDrLFu)obBJBg(Rq0C>!a?F<3~pf8iLkDmc7lqP4nIHmm=Qj zf}n#~_EJ&O^oYkXSXpB7^1^=d81i`vXg*k>Q>O>$%ilkOHZ)TXVj}M!#=BqO{R+lX zAuJZulGLBvAtm6H4MHdi-?$C{bGmme)F&WwQIdbf;{&FgiQOo&mZ+`jAWkR6uCPV8HOAC z0Vaz90fbsd2#?W9rwx7Eas8`_!NyGhnS~D?fXgKp;&deNUJJNJD6QRow`H1ae}t2g zDpTd^WlVBqUxo4Yuh!piuj`op&;?XHL!JqoEvtd~wU?Q4fIdFmlD`u+y{Mwx+;k2| zJ=ireFF;ioH7t|kqvW6ep~))^5m!vNjpkAE`)Q$jY{$mh%t+=pD#R5>$ODOA|MXjIVHWzv)0Nzn07CkDS$_;jjEI zEQGWHh1Zh>IrH_xBXaNZ`}^T|rjp5a;js zh$m|SHTD4VJ)05QT#y3~503)YZsZ#z&MR7G6+GhGzqnbpaHWulojHHXo0t$0>Zg4v zk74Ay0RF|<--QJ6h*KYhMz-%RL6)9Q2^dFfo7rx*pv*K`;X=6+Z(@RQ+n=_1 z8Q%8E9hw|!Y3({-Du}?~tosZ=e=_?rSXUlL9rF(I2~V~u zrRjSTWFRa$L~7dDh>d;wHUbhp_0{xg^78nE6#9}Y!Y)`zjY?rS=!kMkJPOlw%^L;~ zMhP3($;s!yr+rBy?6h-jvL@bL5G8{0n#?nSGN1@y>-IMw)6;E3`IAt=RPB`q@3L&P zX{Gy-7m-e_Ub1mlN*AQ#6@UMc(knDHaCI^8o6e)8d^11`9r){-vy7IsPgcWbOx0%l`Fdhq44g%-qVXON33A5-a79Kr zK3_3aRlrk7u3FG4qy+|=bdv&r;JRAocD)|*%29r?Mkq4`jf*rj0Vzn}Rq_ylh;4s= zS$T}k4Pynj@Ayx*K7VipS7*BkZp?pzL-H)_d$ggYMG?q0aFWzgV2ad&0YMlKi-6~} z7rMG22MQ#NY^ahwF3xG2etJ6T-#4)h&E9)^iQ7-zg;Gx z2)Ua$zY&HwdXM;J)JvH}fw@pyR>nb!(-uK>6Ze~_7cv}`9k8Yk zF>}bnICK9pEb6>&EG^z`SCA3vZJwkU*Bpn(l?2)JGLc6M7c zpV&|4fhM9^_>db233e+$#;GVT*Ld)tP^)0L_sz`*4<1-BObv*8>|_)d7uR3pVlj0` zUP1ti1(-bvT-r830)nOrsp9U?D)G!xL|i0|=}sBk+wJ4*-*FL_+7mfuT~fVPk;I;EiDT7zUYfP6(Q zXw7HdNyN)rN)5>E^$MWZ04M3sP_~l)*^$-gUi@_(JzZV9i3$Z4l4_vOaB@Cg{2c0k zy8ES;jyUKU5H+CgnyZ0UwjzP3U;V|7duAf8v0i*^3-a>50~^Te;K!RsvC+|dR{g1# zeaRD3Q>A*3e&9Ty; zTBRu3^4i+-Es-o(V)wPdg{i%}i9FCOAyl4e!5*^FUcY`_YTT6eg)U9n-_P9Kd^8rC zG+brB=i}ZsGBN_x#zR;`;E@66PuyksrTQ80UZG1StT*R_yegnc0k!M;!y+On1{#`_ z!|gd<-LmGE7PMxN3|PVskH6*tscaVdE^bD%sKlf1?CnuY`P995Ay;YH7fe9&eSQ7S z7fCpQS=(ZsdpVHK1e*cUA@*dg43-Z_jolfBqJ&3m#~QeZWFcE}N=nogfzXuerY?Eu z*VsWi&~%Oc6wq4^w&(nh_AC%U?Q1$awmsTgfky9;%#)d!$q!6X5I*3>Lif;?E&_>J5Vk&y(z!|_7SHaFka*4~HB^_-Z3f5Y4| zzz%#PDhCS{PRIeLW6=?vhrx)W6nTB z827_1Lo2IQa&A4^CR*~+|)D+EaI8Pu%sKX@$AV+17ab4>Nmjcb)3UZsrk5_hSy+Fi) znl?K-E8Bm~91s2rf0AVnU@0#z2bw0mh?5C4u7jq1!gr;H4;7`QyaB_&y&fDKBr-w(i*4pP2Hpnb&$%JAdE!HYgFD*W-F}RhL;pHOMJOC| z@9%25zm}HPUPUFGqPR~eg6)q9*FQT8K*9wjwFU`(d_-c`uHBx^Bqn2`a#wKy?xU-# zezlD@4CeWZ7ocJ$VfTveqZ9WBV_0;@g}!=)HNUzFy*7$_8!tg-ot&K9!dy~u@y~3f zR|pR;FQ5f|?eD**ObJBYL!jgNu;WXsC@FoJnUTC{)!9o9yf4@EOT4ADp_e60(H%g@sl#HzXzLZbSPs5Wva=TN@kL4K&y}kVMu1*y7Pb zoq_3Ti5oX=0GAz(ma*~N;2@v>0^5x5Ej543hDzE z8vbBdVXk6eFbx6-Dyqi++z$b1YOwlnNMOc70s?o`)aa!B>SJS}@YnVDxHwd5BeFHn z_v8Ra4U`kGe|cVD5`6SNEa5Z#0Ou064UQ#T5ATbl37K%>5P?BKkF8N3^MuLqaVTQ6 z2fdy@I2PsRl7Y6YX1R6ip)w=fB}BlgCMEq1zX!VEn>U``-rinbXR3$K^YZeZ zJ(C%w!UH`GU7u^p$~wJS--x0I)SKh|IS)Mp#33>=vNAy@U!8H&?fGdJ^r#1NFK7|C zYZnW3N`Z=d({WaUnHj~NO}C4KOAW4*T;OJ_w%^bxNsWwzQQN~RvKh3Zbi6W)UWIvR zP0g(UPDvK}6FK%_(! zzfQ<|_YMmK1CY3t#&Y1t!cA#uY@`Zx2~0Aawv6&rseqH}?tTcG+1Az;0Zd9@Bun3X zCNl_cf)QC7wMsiL1|}pt1cCR zaHOlN3*=LTukT+F4QcNK`~Y~RlhaNh&ehb^)X5rqaai$F|2)=4`e#2@SKno2#d@lh z!LkF_o1z3qwl&%@t5s3u{{2#5lhQY2W1a9Z%^ zzZL2L`LSEX)P;V%f;(=u=?ezPaLBR#AaVK!8g3HR>PP zo@+|qXT;~$t3vPrUzLI)5sm>6k{vci3ZN_muTIJINWp_uU_F-lp7Vpf1*+TV(^Gq0 z-K4ZMBPiAJC5rB3ra1NOTlcMLPq4nG9jCYo_CVhc0Gk%d9l*iT)Y5|M{CKmpX-^Uz z74sL!1sTV}vU>Y51G2n<-HlQx?oLC%6|SjClk|Q%S{%qY4Fw%)A+B^22s=qXXJa3& zZ)||Jg-5OWwOF`Z1lT0qp)^FL)wL8Z?f0Oo&AEYseg8Y0icL6+D|JY)Fe@GMv$J0p zrx{c^&PgS*OtrKyZflc~!r22$^%cFOn7-#$`Tzt76(cAa>%kIs9{mPM(9HPyy`V%u z22xi5G0~bYG58<=IyaI=_?GqHwMsrVHpMy*tc<5WCtLd#Z#rUw@?z=z27Re`bV>}q zKZFI39+ZY!vwd)zK$|M;>{n3jLI{=!1CM-%ZCxvnqf;Dl;YyT25`RSnSh17-=YLV= zilgH;XfAvPDM;@w0FxU5)-5pugS7wIDM(F&=PygB;{f-rAAoZKPzqqK!F%k>7j_mF zbGVaa$rm;^PwaJGfOAk@zMG{G9tbIbk77a0PE4M`V8CWYarJFJ-TQ6U_^ZwW#*k&I zqN4H`%v&f%&;*(PaH(Gf0D82kH3jz2gw;9)=kgLxJA~nXA z3rq|U9S#!A9on}j7bRbPF~eK*`EM8TtN-uF%>UQ8CS5#8Wfc|aP3p9S1fEKA70eQYhVSFK-rRWCr#ku) zi~n2~64D+{hN}h4q4o9COBX?V){487TT(AS9$q_sW);z^$MV?w_n)qO<1a@?M~T;U zc(}MWnu@1%POJ5XyTOJTy7*K?M5ETvFBZs&uhL_;r^D}EeK*%gL$)iba4Ryl%6?`m z7TPqL?_|(yB}OrbB2328e4U+l=StqBrqYs;#iymw!@px&c@Zf5AciqKkddy=lbwr8 zjO@0DeD_c9Aq;UYRz^l-yK$j@m4l<@qgw4L!gKp6#t85AiFfHS1xL`GU0^KrmTLn1 z{72;h!sm?k=;Vj=5B=(5?WxJ~yN88aV?l0dO&&;d8>4p)=&wT3`e^m8(D92UvLP-@ z_z#^VqE+80(yMatKkoft?4M#{ZnITkTU$0VcRsPBVP;1IPX2pokF}R$rRcvZl&t#G zwnr;pX-iItxzYUoDZTc+So}>mbh%|2x#e)ma%ejisoGa;PmLQMq1p#O`u;r)tnJY$ z5fBhMw_hY(uy@)y34Q%e$mYBH)MTFifYjEQ%hVhwdxDgjD)%P>l|@s%nfnRFQ!Wv| zW*bfrleMo|}^+ub{B9B8LLJvs!PH#a>Q;&+^7AvgK!`ki~c%AznhFZfpm8ghZSC!tirX@?K-CF&=dfm;(>5gU2<%uFfz z+K+L5Zf^UX!3C@<4BW;g(=U;7vT|}tQB<2Nf3X9?!f*rr1O`ff=gwEp^dZRJ_#haQ zoGfOyYz?ak9_0Rp9e3e}$SjFNT*hkJ{^~dnt+J_wt2V!7PZdwO9}~Dlg&y^1 z6s6Q|MWhR|iu~~%<9RFYwz2kW#C`3hhM(UV0-AG|d-YA0m6df6{FG8s>`%UVC7>d9 z*j+j+CnrZnF8HmOYfVj%0>!&~ujr+zL@f=Pe^JC9xUZ*QKi6wuyLHRqa~b30q#pFx z+n*c#IkBOttc;C!1xvlaR3S4bt5GUo?3v752^GcR3LeqbtFM!Eqm3`rZ*1Cjy6ss` zj`{94+ZdeC#nL+^{`5QHjj>tzh>iPq_Wa3BOzrA@))yzqzuVZ@oEM#pefzn{>+9`q zH|IALD~+Dkgzb0U=#qP9_7#1IlDxI8?bMWTlh4Zgfd|G=!^cp(2Hb{}l$6N7_KU%4 z`e#$WOS7IcAE)4t|I~SspKAeBnBgol@(7zIJ#wJ#^^{n)O&-HM|fx6!rEK+V9ON ztKC*lL`MV-!^VM(^>z0~zq31cy2-y=(u|#F=j1>oq)qQb27^YIZzG@oO#LNz^>V!0 ziZ{Uj?5xnQ;UoIVp{?nhOWXJJU_z>*{QRUxR+fYr@AJ6o)gL@SCqminkIQwz+OYxz zqFCP8SeiEm1&PP_s7NArb|dk;Qu1?te=ArhY?cS16sxZGYxL9B{#MEU3XLed9WEO% z(=fo)$3l>C#nl!V4=iQAJE`9M>%Z>3zDI`mIa&2kU2L1UeQV+T1|s0YqvGy(F0_1mq)~S85HxQMB#?w0FxW5>5^CJ$kr8=o*2BO| z#mEiBEr+x1N4Iaw=}!SBFe&WFR&{pc8sc_g0(YR0xcJlh`KO$`>6(VNg?Ov?(3I8G z)b2p1r$(ReJU*YAU}N9q=7!2=K?7yH7#;rnQ!&1^_8uJ}HMK^&o%650L&CfjVd3F- zRaJXyl-FDbhq(eSv>LHW{K(bd+H;5US964UFlBv>u@!lnHepP$56~w z&5-VG@0jd$@uypp<*quGNb9vP)9&w-YD)&vn_wsBFlz70N(~P7KV3TZN95$@qH!30 zj4QJoed?N+8e8Gh{vMiN2L%OLkXwFgd;I+jS6BdAUOUcKsEv{Vh8Hb_96UQ;_cN94 z_>=YLj|U?6j;~uC)V8Uv&OO0E=^Uo@7_ZiA_Bu1A18L&M3XeRyT8}z9gh~wD=IV}` zdUNR}YxT=K){Dj6jq81mPTTPs1T3h9s@l#6KJ@kVvZdRv`h`mE$8G|7hK!fR@&Y&W=dO6<^6j=@2^n^24xESD2U zlKL5fm&PFN3q9Q9ygbE* zkE4%lzmAM(>FC@)M^Qn6WVV%#%CaFQ!&GPtkr4^;JGf!eRk5-?7lM8G6J-9^*oM#p z_WPIT$IILPV$qM4%pnH5%CpO~3lj(}Evup4)TjLo{U(PQTVV&N*5&ZKb3u(-_&AfT z9AF`>YU;l;7s} zNcc{d3FdPvG&mz4{eD^YkP~utaXL`&qOA>KufBzO>=I6iFc;3A9d%`t2nHnVMeSYcr!Ou@%T13%FKb~r4 z(??kB|32RT+2FnHx4OVQRpan;b$fe##jv!*4G-%iI(qP%?dYYouZ8cxn=Z@jhjySY z6V)e+D?PIM*^2l&+S+}3DatIv-@XCR*-P7(n~{-GSXh{$zWiDnb-l-Cbf#Ci{TTFv z$pS7&C-OvB<Q@E>nWc&N}8)H{-ff?-nb9^$}d+bUk z!vTXtTw2=M3FK7fsU8opX~o6YI?S3lSd!Bd?|%Mbb;eN1^{@yVD34UPzj0|3{+OAm zQdv5=A_3B-U7Wn&pK`0p$vW+XUM*eV@!}un0?q zeja)9y${e_t+C0C2fE{+FkB*#Td9w(`q30ZwZr+rg|TPP;9mG--_zlEe>c*0Wc*-` z&Glv_c-`yqoHsoe|3FmMS3e~=S#NYmNj6<>ivg-Bf*g`$+1)f=4*@{<)& zP1|Wh{7~Ph!Kk$y!qS!aKX`i+XsZ9WZB(gLDox^7p+T9G6qR9DDpC%aa#Y47jw8SeYK zulu_1&na0^1&r-*CEtQX=4=#(`t_UF6O)r>85e~~!^R>Jxve7; zcB@~BF5m%FZ*Ol0&hoZgM?Axd!vE~2Bt7=zZ-c7aTAu$ajgZ*Xubz2$1Cg$&0uBK@uDE{z6737brTzfXLqoUzJ(vu0qjWtwwy@*7#l z25fcf>fHIJrq7p6RoN^zR|4R3FRgCDdNeCd%_!BVF{GgX!OEo~JA?~@NX9_Y`RCfb z-}Kjsl19VS+Z zvaha|{B0l459pSJtzXt^$}b7=QK?= zttg*t5EYZJkdmUmOjZRj(>j0=D6Ax&_V(-{_WR!RJm@BX`gW6h+7*+s@Cx=SnyQwXq>PxkH>g-;>j<}kQ+A+?9%vxWzFw`yrc^) z%Pcu*7tdp@pE-7cya5kYDWfZ|)fmwDol(LyJn6J0<9306_TBra8jJlf<2dNiDsw}8kc*4O=RZf=dk zbN-IOcZ|V)v!mdXC$7v$5I?mn?}hioIONHnqJoo9+mcRc5oK25~4b z%qZ>M(vWny)DRqCzM>%B_H!zfbFVq3*RUt*Byl#We?i{~!0C z6PJCzaF(;)f5jL7N~@1=(N@z<){*v!JWx{AWWqP~^%yY6^YnpmWyh7U@N~}2%CT&> z#Q1-VoPNI@?ce2yBww7d@{)xf`CU{eix+zrxu*A2S|Ge&mL`N1-i+#zH(xIzV(n}+ zP!*m@;ie@$v>AGLd~JpA-=gT-cjtlD{ONAI7pd(eH813GPNi!BbI;8~k zK^+W7OUqnPdGb`&rrCGVXh8jU-cRBCJ9h3o7RrZ&VAjM1Ex~{9t#|3m{+;vxqvxNX z@-gAX%m3@4{$9(Xh!6Beu0owQOOCBGt&v%QCwQ8d1_*6$P6JWo?g(S35@^m zY&XX_M#6%EyDt4Xh<^ZwDtJ2m20^>5Qvwq|2$Gu;Uh`s48UPZ~sd6Lb(YlHmdo9jlK#oM_yIDAXX)Fer5*6O zAD^~H3Tg8o&0vnKi66hvVpKBJIedHh8L9c)++2KSV_wGir9YAP!0m&zZ7C_S?fv#D zE1|omCp56UCW@3CAD@FHjZva$mwYY~WIPzcAK$-Ajg-4Sp5?3=+=vDB`anDm3Pp-* zwWa2vLl#tDpSn0hWXiF2e%BZ$VW%Nchn>gvHk`jjOkx;bZ5?2}$LMRMh&T?Cl1P@I zcQZPoqA}=v!yddbk&cUr2@P}-Kj-A?e*RQN8&jkZ6psRsp)X5qf{CJ+*DR90 z(b3V1x{zMSB5?Wf_6WuwF*AFj&j95hqE0x9-ZlcM$@G*IGMkw`GKHI=Eim#xVI5~4 zuHS>R(Ht>Z=t|qv)TFX!PlPIYM_M0-)eQj=lpaW(E?%se2%Su}^Hj3LA$yLRrp%-uN$`Nr2+U|3m*S-M{AaV`AvWzGet z4WJ?7LE=fxx%rc~@i3iE!VehZT`sNOxaY==8xPDND~&Bm#$TZVQg*DrfL00U7&~*7 z0s)V`wY4iUzlac*uHCuNO5jM)Nir-nay2lPIA!lA(xymt+0ACa6zR3c#KoD8bJ6Bo z;@Gnj0G~w8O{)tnSuhMl()Q8s#Rt_r7g*+d_o0h9UKxO?GueM`-x;3MHa1TmJ!E1JANaL$q9Yl&u7~DM)caS>;J$b@rv;CaZ)YN27zHjU5+LAjtHb$tM^qD(orbmuQ zDJfk=ZUbT(^rUsaa>X2xACf?Gh=esdUE|r&e--I$y!D!E>(g#57L@jOci|HdZO(L| z;V*>j2lF5Fr^c#_r#ezC2hFCTO2BEjNZ8z&E~O$YlxgMzgLHw_c#`w zwzCVBdkE?Qq8`ln?SrqkS!82;7R;N+vm4#T!gknf)r~aA&ceiFfp%Li8| zw6hS2I6LQ`|NgqIt*w%u7a}F(+Kr5jAyFAXK_1~^h6cnkZVTO$A;W(h*K&YgflXEA-d`&$+v!#vKif7u`H1>kK^yLU-gv zU7OP(vdKr2{l-Qzv=h()Kn}8W*)pS(C)cZb5E;wu=StwokZ;8^L!N>3DbIGVsln!e z;iEy9LdDT97z#-kYAik}_%Q5Vq}3^t{f6D1_*I#=#k*r+1hwev>kAK6J}3iA0eQu0 zoe8A$#ne3K71Y+k$vuZusjy)H-xVYIvXo$*&Ys({jQ4-6nfyT^6-C?ZSbLvJ+{ z@*$5NVXbDe8R9dji6|p&pOHlO#WN%K4t?K0YuB#%DoQ{s*=V*EMidJgDH7O74#b!PM_ZHJv{}x?~A2@6D*j#;83V z+_D$#c7@eTEcNtKkRuj=$nL6iR&p{OPqkp#`cu$9L2m$UoUyJh9pC3c`~9Ym_O38j z)6>(ifa#EfLPCOso`r?TH4u2+eSZXSS_R1ODmd2GR?%w#0i_Tl5XoC8F3!Y$J67Mq zL&UW(DhXZKCg_mH0~dqgT+{}mhkoOb#R~BA^DkWryZ)o0p=R{H0n~#{TVKC^ovQOv z>Q~AoVuG-DV1;pkuq_9YZ#*(*=l%~tM4C7Rpt7>ZJR*i_Z+3?5-E;L)r#=}PgI&9J zVX31pSo6n^J7JOGEJLIWx+UBR-R-$PtPn!+*lmVL5`LXuNO?Zgabb?{ppURP=iuN8 zzYSCYi;EK(JiNSAcJGGSMMtjpQ2sQoh3St&Tc++hOBe`9T3~OwMJ*%txh^;Nf;ykq{XeLox_{+u z00`Hv|Ew33^t?xR+i1^CD4h_8*L!PtM9>wq8qeF?o0ypaHK{nBv3#e^_3k>bHSEzN z;#3vvN0{VZEbeWOt|CIzQo<5N1a+}J9oK}&DXpbOZM^^Dfdg}|;j=c?YpZhao8^1Y zVA!Dg+q-+Ws@Ie(JbROeIKj6xCXK|!#@ehK;3NY0Vjax1=6OL)xH9RiI~oYCb$=mm ziVR^o{0am~`1bKy%E{F0IqoH7AZ04Wp?&PvDw(9O+F zwaug%LgJ{aTZ3q+UB(N`Q;b&1!umT9CsEnFT-BE)%oTe!G49ZiBv&a=raA@H?2L}f zm}M5u35bog#}@rrSBFi5M*S7gYp&6H16MAt2+T@ee^U0`GCX>PQoWY2)oE+%G;ID) zpLXl&>Ow{3qAOHaSLYYJ+2Fu|0}pQ=yww!H5N*_;xCBm)$8$sR61n@*0W-7Nrr~?J zvt^v?H`ba@JOK~e+tzjwB2pk6+ciiW%1BAEM@z*_8cv@0`iv0Ck>Mbr`M^m9t<}{@ z`}}((d;9t(A^Qbx!s!wD1bWpB+-8-pYX1YL9=pIzHY*PU86ql&g6igt8_>8Ycuie^ zf~pd?|NUEZ@0ItMq(bkt(9K|)=JWAU!tZpyhT;$^La28ZFI#ri%`MND?fEeNPVrfR zzkk~6kHK`KT%|DaP}=wU`epq5<6~osAsSZWB(Q+;n6S5uGufsO4tlFvdlDN{4m_Xu_^}>@Y&C<2I5F}plJ0a_ z6~8u=T>4}K4K_Ail@dM(rINLuN&}JAHZ%xDZhADSSZ1|En+K|8 zmviT?Ks1L)?69V$Bi4LaSfUj*AM&#-mka-EVy*5>-E9?z2*K6;e+V-U$y8mk6$R!Q z(3^WT*B8f3)Qbi|OwIePp@BOUqKW4sL4gP)3w0JoIpQeGgbl!3_}~@KdeB73!1BO` zJtcP}R+eK+K{kwYy***<`1k^FYKphJ*!U?jI-mi3)-2HV=wKgJ{|Ir znsp_zQ-21~f)IuRrJu<)0>q*E1QTT;g6lBI5YHkI4o2XI)sJWha=o`$*DrW*s)3oA zSx#0~aZwTMzjsd7&6{5^bGT7gZ!gRQWKViv;Lb#z!>%D}3jiwN$!3{MTT903I|rdw zVvLVF*2O)|&St_k-?(uD2^yB7*Wn{#fdhZoscWN zs$2;jDA4Z(Otx;6I{0;Yt9%?qgA)-%pj;e4eBaTbh#r)nGNESQax?_;GCbtMj>;%i z@}|lsobSR1{a$5m5rQMbhr@VXU~(WKyEM|aKUVz)L6g1T-T2{*2ELcEUp5TK3T1(j zKwNmP`X8J&qXMi1APB#RN3*f9sl$s`9$AL028ThTR8>_e%HPLYmpA`(5eGjN`ZQTV zH8uDv<>ku_;JN`RA(C4hx*w$r2;$kDWvg(I5*#mfu$);29QI)g3pVuA2%ac7a70`C z?#1h4b|sr}U`i|=gOpu|Q%7zMWSD@3=+oT1=j!08dj3X|11uV}w}f4j#i=#0CxD}< z6-nK$B(-&GH`a@~x;i2Y*g7YyJD40uQ4ulFa7V!Kh3iCd78DeZh>!s1PEmdHaFPvF zXq{zGC_CR4Ms8PyuR(AE3+0Tk8Q6c=f1 zJh|NFId~hx%B97=+i2>UGCp6T_68Oo8vmPi4KO>wossC2v}P zo5Y#!$SK_x7%&a-DqNolH-5>&g}+j5?p;g(7X_6dE|&CmA|gsq`Xc@vs%~r#9bH{{ zgLlVZI3VsG|CD%27Bs~g&bT)$0i^N1@9jDJDo(ZUsb@XdW{=@-i(%10z*eO#G>ojri6J;0Bi0<65VFSF6z7wMC ztCufJzyEdj;{msqh9^%#bd7?Vu#k`i=+l%6fj7Q>)k2LFA(2Nk zrl_D`kE97e5O$vCePNWQY%nkIz=DD()j(e#IVND-;IN&cJN*-PXsGasVhYHO+xz`;a%N%X zd4S1bjR6jIJ)_RGBkSdkS3wjB22k)?em>{r{+p_5YLK#^-7X>y#JaL8@1PPZ8w>8; zyO1n`ZN_O&KWb|Ma8MLy0!$A37jWy=JU%|NQ>W0_d$=hh)i)GYxi;12X!}CM`4~_v z2oocf<_o6A#!$H7tgjz*^XARGJT)HdCu~*3v!UQ)K|VqY zpVeR@sS3Q-Ykhu47vg%~hI!C)GR(EM!C!uEqCX=!bQ zws6h}M8G_#quvn;ekdg1J}wK~A$`-=|MhSe<;~Gxv1ea!x&%r`H2C@NC1)6)7#SZQ zXwDi3#zaVG*uGsxQu3gtrocL-!3Haeo2%Tl6 zu!Tj-GLgL?`o|%%588(CXaqgk^q}U4l?jV=0&xd^4#FTWyc#TpF$Tz$jl{uZ@AnQk zZ?WyCuLTC`wvximzB?h(i_hYF#Ic76(}(I4n=)lQ$Ykt72+H8*u&U=mRX6ZEeE>E9 zIbPh<3oI1*^X6^ahOt<|GI7A~fa0CNzKWW8dG~!mK>_%KMU6L5nwO5kAtV+MX6EPT z69q3YY^J8aVKo&SW{^UFy#)A!sdBclal`wvxRT%sc9&k5V?_zwA+Y=!Ax`h%6SszE z2L^fpeh%&j(=yxpee)|;M`o*g?7 z?Kd0(J@^E>bo)nGw$hKn2Pc ztpyYn6=$6@csTox9^a@+Slm6ZxF60IFMh0r9S&(%)EDDZq+W_(42qdsIc~AzH7Wt;Ly1cv}4h$iMiTn-QNgQo+M%5e;7;Nt9Rjb}W zX^+yKwzdx#7;KXZLc1Z91IJAeOpuS5`=4B6?KiS76vK}&Ym7M4sJQCgKgc-CZQ67|(&g8OS6brI0Fl?Pe@3k%$PbjC>8G4Mbmlu;hMyiE4X0;5 zQ?%2;%7y8G^M3qz#Wk#CRJao5LvR(?;3jDXLH_VOG&q02c5o!JNMazE1oORW`SKH( zfLB89h+;szAZ`KD#ON+Kec^(v{cSDO|G+Pz1eOPPjs^$^^uzH%#T6K?ts~#CZ-sTyGMqzXe17_O1AtmzIFeF)=aU>YN=N z0ohIX`1sms`tPv*0m#wnlxqlP2MDKL{?a17-2zt-p20(HTCc*xFSHGSz))ZRImkTZ zKd{m=2^vig3K7^r7C@EIG1NBV%Q)iI1T-H2en>&tC-A!HLceDl z^raBot+sSTC0!Ly+BP7t@=5_k{2djZ%1g84?~CYNDFy&MFC1ZzD+DjYT2s zuxUxuhCV#G1N;kES=<~^!DW~e5T^;;uEktE3=&uq#Ab6*h=~Iq%2G1mFF<7?R#@bM zI0fh7;cvw7`%2Cbb2viG;jQWGdzN4gAU1?}Y}w+)K@(jSeBk=p^|xHZ1L7|jO=wMo3Iq7Vh8ihDAff|@i8M{&uV1Gm zkD&Mq<#-qYguvY_RQ!HQ5Lxm2_ixNW*Z?~EFFfkX)hlCKq<;LEfVwTLhVl@(3I*x0 zKmGlSPot_6!-M@(DH|9VD6~ztq_h-Or_wSq;m@9VH&s33hmAlo`BQc_@h|7kLzt!t zvkDd;*jR2P?_~lGqFBFj<)-GG!zLzK@I|0emn>UW4@6rzgE}9Omas?T|JJzUZOAy| zCc7ULi#;UU*z$IAppL7&v{=_ef)02je5UoT>?q~r!KdN4S@YhQZIYPc<42Er&5Uii z2J6<-MZE@=1$G?Ow`TC+2rOm!y+?rX$^Yuf}kCUB($K65hDIAGjXRj?)kRo2ZsdxJnH5rB~g&ngfd&%HMDHE zqsGRW05BjJQQ=o^z7d6t&stTW^qO+rb18!HY+>(iaRnGI7 zy-GyryB@!l*s{e2;S8|antR2d7T_|x$GacAdUXag3NJ4&WEX%4h#&p*=)h1A<6$#L zMn)DUDB=OIU*Ijk4?vdx;sr5-@IvzP@;nd>1M*d|B0J{;mB)M#8CXB6oc*cXK!DJ0 z&A}O^YHE3K`HM4qR%0_Bh!f<{_eM^~#ki z`Cg6FP4ty$z&?rR!V*Afp|&fxm*70?@)1CM%FAQp(eXM+%Cvzr!f?RE!h(V8NKKsTJC^05Ab7DuS1`uuu#2TJIMSQ1eWI_ZqepJSeP{-x$bT@Ye9mTII8vl2-XF zEd7?|W~ek0UcNkmL>R8Iu+|R@^&~tB6lIlW0%T85khtbJEdtvZ5NEd!^CMbDT)8p< z78ljHcF2P@JCK3W(cj{do15W@6X&swMU$5;gojjvhwOPs{a#!d`nmVAc+rNHcd+Qt zkb)=$oV+mB)Khq3MKSMjd^Z+ji(sTnWZ{t(p!NjLvC;Vc=)TZ%AX2@iN6rAyiMQB3zWV^27y#gI6_pC&&|hg`zq4Az;b))^ zb^cF3{!g3{`d?&4xg*Xx{7Iv)xuT`36YQ|l2HTD0ZBxH#4saxPHVpvSrv5Da7;GF=1TtxB?q~-)?zi-(xG!2r86SFG(&Vq3 z^JzF|S3;IofrnFIzhicnJP%DbxkA>(l@Xy^UwHD8X@J_QnC!$VtB8rjotHeh^M@6c z$=*5VTa{|2p2&M=`iBRSeg&!+$J4$`B&+wx`!h9klSuuci5F)gOLY{tZQT;nm8>J( ziMD#;)3k@VpVAAzx@{6{RJ#;=tU+qi{Nj*em20<1Z&WYw5xwNl_oJTryx*|OYH69h z)X^mR#hvz--_hct0!!s(KU9}+c!F22=DI$2D`J%EQR7Kga&u0(7cn||ipU!M-8ue~ znm%qHPd?mcZ}=u?s;0E|^mMGdSO?FsnPSN1&OE-ysA)LjcS3Zf|HQ@V;s^jVV zW5=x5PCTPbQae8eM@6&-cI$LEyy z!7C+N%eNf*UOC5YnYpjvh!RsGKKelK>>>?TM@8A^nk&B_u5R`HHP#d$*KyWaj6ACu z-_RjPle?tt+4VH*a%Giikx6yu@ZdEfQ|!ba`lKjM&rmb^B0tY#zcHLh z&kIYB4f`Eyly*1wL-fn3kjoI0y8=bRl0mS*12m1{K*Zk&4FQA=sz`UkcibyJd!`GA zp(g+?LhM9kPc^|;!9n!=4Z!J`Xq$BN#KVA`}SQ!H9WirIG!eW3v+2cJZS(j709$SHZ~G{Vvs!m@4aHpkvNh&fYgPC zq=>LE6YOnhvVea!1(U!nfEok_b--4Nk9UM^BGgW8R4ykCy(XGf#l;UUwDj--phmu= z?%$9el83J>rD1ee0ndk|C!w-~b9gkMoONQ+y})L6FU#d>-l64-wz$(~b~Y)*6_zhs znz1{|^Xle-@<4S}`lXAu4=CBj>?ZrRlo^*dB%X@`6FV)Ncncn_x;Me?U)W~g-Px;H zQN zN)6DJMO;g~3sQ#Dw$Xf;i}pE~dOm0k;B7rj&l*WRO#fmzr``U~1XY0faeC;53HQwd z&8V=PIz@GbH<{F_0&8xBXNXCA<#AC9Tg&p_MJ+K=&)J@9uCYJ4zA;F))FoC{HX0kYWc7Av)%<++O#KX=}C?HY5tILxw$7Mo;|rw?U7=d)|nrP0sbQ9t6j%W*bb~s zAL{UF_S4piLs1cg<49-uWr8>00ju*9AP2cnAT@NTX{oJURe;Eh!omuPZ5z8*-75ykI65(bJ>Lb^GBF_m%ubYA#dMuLf_A(i z0p*LSg|sbAw<5Lnx~3{9^eOnkzuP3@?5}kQeZVBkbxj_o2g}fcfDTPT{hJYUz<_b)cf|6uVzn(o{XE#2-WO`trKIf z$l7%qWj%%1-|cB956$W&S?b}iA-lOsQQk^jLKWkbbhA;%n@2_I#v&t+ zf3X_+8-ALpq-?xqP|=jYtkRMEm~8gFMdp^BQ*jN$>&;3lkM|81o~0jr5&W{j>*4mO z{Y^;&r6LBkPgWh2?td(lX@4rczHj@2YnT1{ZDy+lOcW(YyEJxLuRCHE9TR;xi~fCR z`{8&!cb@XD_$_&(GlSVja_n7R$?dWZuZiIvS>z)%Fh_4}&GO|JLDH@lf|li1QSmK%QjC^toJ&xh#F0GkP}=mM-b+O1&&{HtY>78k?if6n-hFv+{$ugm5<#PPEu|fb)~_C2Ix4&&a?{;zq0NQytBRM8A6=3xOnH|3 zymz2)rS)r#xeGUy&)a@Ec)yO~`tHV@oVa>ZL%NuILmu6-t|ceRj^0w&GCP~q-5)y~ zlQR`J&bDCmJt6I>NOjNc5D_4M)u{Ha&3Lg?Y`}}2$55k_YsRCv-ZSlQL?_?dsgH+g z$MUK5?$`DDUE4Yqk|`OjK9v&4?oa0QGbmgJWhRu8rQg3I(4$C&Hd7tNor&T`(q}}O zbu>;BW%iD)_iO!r{`@w6(Wwss?8Yc|BSf8?=MA~tNjbyJXe(y41Zk#BJ64SON-TT4 zTE%MguJN~Er^uc_3af)Q;;$^O*erCt%Ic@QcRT+B1p%*LUxM|cSY=VHa^w85Y85)` zhdlcSbuyDvAIRwq$4c^^F7On*D>movD&dJ!=E~ccaW5G zC}tX8DCW+PxG|Dhp6n#+2I`eA$ia5F~0pV($T(uBz*rQ zb0?|3f7HpHo5f}vsZ)-DK7-arCe&veYkaF+BVBRBX3v2{laQx2GskHY$BCg1)~8Gd zP(J-RW~tV;RIIn5#it>rWo*Q(MEOS(&8KI8WNGO$8tOf3?d)?)zu$_+Q45@HpoDVz zdbpk3P@6`yTJdcw`ID;O-;>XpjT{+sVNGUo+Z9P2_nbwo^t3IiOBw5EFN2vW@}5cG zyH_ZypOBh-KdUxD%j;|4%;s>Xr*zpj+)7;A4qKa@Gbz!~*H)W(73ftQ&PeS3{Eptr zFr?44wU3Ls%H*-zG-J2}NdnxjX;IU`rE2@6CSSYw{&D0CYE0-R&!h>oOeO`gOT~PD zKOuE+h`A1kntV>0{U+l(AkXO{F<;EKoDhx>)S=P+4@gZurF5*Ibfk$+_Xg132WJoM z4|SoGf8n(0s(9MjR;H7B*}+^G-deJQk> z31mgz^=0+xc3C|j4V4*NjYs|Nmz3Ei6+QSs;DolHuJ`ooMgF?8A9W41Cw0G#tL&oZ z(DTEcv!!N6UJ3X}%7dIcwslqXPcPbRYXFJW{&r2U{vkcpk?R89_)P=14orWr{*$t| zPECw_M&3@pBdW~rfkxUbK8zM|;e>P$=$s-6^-IglGKMZppECILomX_byB?`qA+Wt` zolEZHWkpS`QyIb?c9f}04lQ=R-F7-LwT#>zHFBg(wCL=w(sHX7cbboOtTJ_`iX2Jm zJCfG8HF2`Xr?n?iHJ|SNNS8>xaeEt$z@)x|&9_Eh4ft@766e9SMJVqR@sX46AWbTa+=y2qv`wAH4|69w{jG-WGu>Y5^_ zUur%femUXK*L=11X6snHyZX3dzNAS{w6}NZfTdfH7tNdAx$1b6cS0^xN;%3wspopA zOP1Z(=(mIM$)suDre7l|`8^%c`u$Ial`GV|dRiD#(UBwXXTLYe8t8ibWVbSh17)IH zSq}rdjYxg9bY1RSx2deatVt*Oj0Yt`*-FZHQlNE~&Fy|5#_auBYOC*WuvJO5?cHPj z{x4$kj3e7Ga9+0_&{3CrK6WCoXRJ4UpibMN>D}|VjJh`+GxloPZ-X7m-xyM#i%Lgx z`nhb`;$;~bdc@i+Qfmv+*A{Dj8SL=RxS{2x+H1;<$br~B5jwL{AMQ}7PG-?3O+u7g zJ>IaqcdmVElQn3p9h;pL{%&T{iaSZ0en5K^b<}UWXyB1Y=KGY$2@t_O)i+M6wD|ev zdvn(jzUTQ;5jyAUpfG=YTJrAM_T>KT5YtV!E^=pSQMA^JUzXCndsJ#JcwN1FT=YDh zO`7K8Zn!m~L7LEr8k34To5-$rVHd<~|+Q?E` zLAyh6M$z0r-EcvfnB@nhcs#bAg~46uuc@1x-sBbqQrsFO0T$fUx zJVNq&m&ZNMzu1EFwT9Y9ZJi0_vdki<8$?|PB$Y`FnC=0t>wcTo$*gw$q~L7Nk1mUb z*X6&@eDq}gO;HQWQp<+oQ#RRB0yC>jb&3u%<3|Ofe@dv$N{k1ndODwCc#T}x@?=)W zyi9a6jFS4U(vHz9d-L*LU>Xh{5bq>A-zPI`qjdBZS1wgvU$gI6Of9nEj{l5VrleRO zpL?yN!FXTDsxa`fd2Bb-YPM&k3tTsMwofW(8xs1d=QZ6f0 zb%WCRuRcw52B;0k=)sexm!8?8qED1X75zWY)y7oZCF=(;)x?p`izo#xw z^%-uq%%{AeD(rP61>;>6y%W8~@7Oh5wi<29FpynNRT$SSOHcVAQGemXMVekBCcN*~ zh1VA6o(N5@xtw}>yK?hdQp#ZixsI^_wG8jlOA?RAB3$&f1vAxriv!03)DJhRTIMyV zjTrPk@tACttu;|rS*bpHCOSCpQsR+_yjwco1iagLdwd*-z2rFM=vA~?bo6&azS6CI z`wO0wb>0+XZSb4fwz(pea4n2hFL`&5g&VyGt;MYUmRqBSFZb)!x4(a;y!&>~?Zyql z8LYK##Ry;CP#)eo@lFP9?g-37^I@b~skJZCeGlxe75*mJ6C0RPAy+!OQf%g{X8ntI zEyhZ&FXQ9%we`M6>GvOZ@vMbUVNaLVIDgG-zFc7tITlRgvT0OM&$UsUD?7TxaIbDM zftpUJ2kG~>deQ4lpJ>=uJvBJ}BZ|`(6(qe|Rn@!7(Ng2{fq`VJv2JB*mi==H_0c;e z&m`3Pj?@rFeKNt$2k3}ENG1)FDy^drHP!m{v}4_=vn`Bf#rO($5PN-B`tu}QdYW3V zHlK-$ll~`b!iqaUF^q7PZ7))sIjD4%;adUIQ!E~v^#uc%B%IL-Vqn}*Bp}f z&VicA22s{CZjT&wwoIAytFN(^DbTS?MpN|`^74__COxxTTzR(+^e>7s*rzg{4-jgKT5^CWciV~kp2(ty>ppVjm? zxMvQ7%B8V$D9WONx`VAFHda$Mlrag)m9_QR*~+=L~Bm#WKxFH zp7k?@sn@N21`;z{vG!a%zO2(>bstx5nW#iq0q8LkZ^iD%rP+T?JAvoJ^ZUZJB)9zq z+v@CE5<80Zgfhl#pmJc4IE1G{Zj${p*?SAy(W1eHTw$ipfJ?ZbWNX)0aDD3lOEB4^ znAjjH^xW|PAZWmgnEU|?`q_Bq6uyc8gF7O?ee~oWfhvgWxi??ptCRKs3Suxgtu(LC z{FWDw%TT7jh;jaj8vTbfn?{;_r<~UZXwowt$xy8iAl5J+WrUB$Or$XhXheiSGxeGI zlf=z6M8_WSk)}WEQkb<6cuW)u<&WrcCv>g2%mxy7riMym+xpWxL>arIe^NL#Tob9u zB=@{~x}@lzc0ToXE##hvzXlgbjoT)`*>EQyGq+35$M{RX;>>J5eHQLcD8gg4k}Jb^ zOhb%88TO%R#whYtg!+^}k?-lClP7C(`*<> zsk5rxeFxd{Y%A{QIJYH@JI!m~7gMU%_So+dcPfxu0r$sdyHiKKEZK@y9!*xW_X6|j zo1X-d2;}7A{Uv_b)9w5piMTIxpUH|f{=zN0vstgc1m;iqxV%bD)Q)YSv8jz#DWMnI zPSGw~Uuxo&r;ZM#^+g6WbKBBLLm{Eu(9D*Z7%|TQFLh20XGpB)s_jpo&mBNaE;)T< z(m3XOa zB03#N87{JFVOCPEcGTvNzM+o&p|W1dbG}hLKKJTHjbF3luI3JOCXse~G}oT2XWY1< z*qsvSlTgo0%F#)RxeANR&7scb(6f3X#n|7)d`9nCB?NnN#-pfvkHL%(K{Ab1MPq4F zCNxQyx67p>?zfgl!XLW!vMdm{!YI<`hG?z56+>e8aXToiK)T_I+cV?m*;b}>#njTKHAw~_a1Jc?i0k!BkSKav(XDa$Mb2DmTa)2pE! zOFRoJ9&xi6tHgy>BE}sRzV5-YkDASj^1@q~50q1wF%;$#^}Z+h?d$T}i4C9M zdoRBi;l#?AoP;djvP_?*Odoawg&oNL73fnI>Rm?T&H}(TaYw{B8X-2yH1F{WE6x}} zR)p1fcM~iGsb^13dr?=Z+VCZLkI(#^Ix5qcAP}ewFDe7W@v6UhA#KWuHuV-BXB~BP z9ko9`zn?H`_pDfdh(Lw#X*Ijfgw@%N5gZ28mr5fan{wcb|@#AJDu#pjqLFqHs*GflBNZ@HT;V^EID1Y*;2}6 zM$_{BDpX{1X=563^`^s>3LM@d~r zdZ>z1FXUN{0!%T=*#3MnZ8SpHv-4)M_rN`?aRF{J$$KP`&uT_l^!vv|{qo*HLGGZ7 zDzjag(}Bm4pOK)Br&^H*+`T7bNIi7bY-*tp5O?@Hc4`m1+MV^&k=ssV{uE-h)ckuL zA1dRsmjriE%)91=cNn^aV8FNImai; z#ByhD>U#aTc{+Vatg-B{MpG;M4F#e=2^w>fl>2z2hykZ~9j8Cqiu+BBnJE5c&Wlyl ziKEnshv2I)nP)N@9ewsja0&!i1q9Vo@UVG_*mMD3{s=#HCZnCoue8XI_tLF)EZYPG?R^NM?P_M}zKx+ijZdOjonkYO#rDuiAN_LM+$l^q zcbZ#|GS@s~L&a$i%P|sVlsg_nohjueTS-=m`e(QV2{fztFx^@8BDBd)d9NQz)yaA7 zP7H6h0GC1Y>Rx8h+El-PL4<^q% zPbMOpkzt@DVvlfv7ejt);oI|d$@PR8@5=F|cP2Z(U|6xg*5tQyXTMO!&0en2sk!O= z52b=ep3votS#)_yeGL~cnkuD?>QblrqL!q82u&~?IRh#rukUDnmj-vLsmar|yI7Ml zBw+@_H!1)7m(@JPk{!fj80YmHI@*$=77>J+qviN>%0wXNd;eFDiy7VwcM3x;vpKaa z>lXoUxf4;GTEZ~I7+pIZIMWAsSPC|=#%I`-;OVSpo?4O1j#_Sd6UodaxQW7*FL%~= zA_$+Ma;pWr>G4)mLSjC&GFt0~S9fS*j@&=-VzYj1BY`&N^l&o!zL~S6Z)XqpT<`U_ ztj_pW{h97VG*+VoMR0I~<1O(yJUl!-B_LZe6E?-UI!9c+fL~M#E?j{B;7@^m{m_2i ze}DdTZQu5Pf8q(5`}se9I#s^N&0c5l^P-G^Py{XwY_s*fEptnmH?Hye>@T>Uzq{>} z)p8xzmzk@?)&Ao;AvYe0|NHF%{SCtZah;>rPyfeHvh!~c4~0J;R{xL37=(%6HSwRH zHd|aCH<$Rx3x#B#Ax9GTPN10QYMvKM7asZhYaX8e{wn|TxBJG+6Tfg{iEx1E{9k9z z-aoF{l=1OpaNIz4(m<*#@&Bl}M!24w$Jki(&i?Fug_3u7HqP^G+9{X(%Bktyoe88S zg*#ZJbe8T!dKc}&Z=+)2rw+vuuE`ZREzl&yCagZ28+X$7e%X%VsBbqE{HesR4zr}q zrjB#gQfh_Yq-WGSI4w*}PcPZiV{Wy19$ZVC8|;uLU}&f}SHog9 zJ4s{2ZHTw4sz}RZ$?@7-%b(kBV_X|sdQSgVrSk1d!vd45R(f1d6G=U`t7_C}`v6_c zw<V7N8emYiQ3UjN)GN6^9aW_`AOYKEby zvbI#>y5D>y!fP+rZ`Ay?^YP_vLF*TI9_@IX%unhx-b&`jdGUim*LzAI7ateS3~6(^4*Uo_DnBtCjfB z#(Z-uHZ3+h*g(gjKtrlGVXFZJd`erD)xq|0vfXG3}T^l494s z+Ej&iaTwY7xGUS-tGmd6Ja&kbC0f*96Dk-oGpT4iM{Ww;2}FFa6Gg zD%2ppYfj9hg+D6udvwpfFjRJU-ne>q;+=+7)+4Q6V!4cs^XLQ14=n6U&Sq+*{nnn4 z|F~~Ko_?HDpp#Es?AkubHrkkQZ^t3-UOsw&-)w;3TuNc#wUDx0=O+LAW3u5{B3jlT zHf|g8;Hch`t$0YSXLn9#iVK`lU0!eMDC3yunw4I^r{{%R^*`2rjD2nt=ZuxH%J&s4 z1h;PHK0Q%b$CHl7J#CHU@WE|j$Tl)kBA1~MB>9XT*sK29&w3!LXUt4*JlI#$?PN}R z)z90{CXH3ux=FmdT$*c}o>(fH4m&lT4caBUYRQD|k?G~%ulo8A-#mP9v5oej#B}}Z ziCn>CeZ4DUej90 zykJBlmWZ%%{rkiWIL=lq(gER%^L^(9Zr`)byFm?hb5>)hFKG652$rc%>cO z`N-z(=FlJkso~q>>&H|Xks%h5%1%vJW<(zzh|A9HZ&dFg$uTfQJQqcYsf+%o_Uora z!<=X-FR|S=Vf~_R6#bSWUX5`7-8yMFQbr5Q|?Wky$O{at)r~h4SCcct=gnP~-lK+VvW@#RcE*buct+ZGcoTJ;ra_q5VbM z{e-Qxe(-t>vFFIHLtB`|qs9aMR&5C4zFf5^KQ2y3y7C;a+#}of1Lb0h7491v4vxhzN`{;L6CbHOd8!fF z6Jh%7sm|qUJyWs@|L|*ijc@3CA)~Dd&x?G{+<&ZhirwrlNZ6c$y%^|=!?(}Bo}aj6 zWj4j((~IvZuCMG5*4)bCENb<#9KP}Rq2Ju6rccvP)Lqt`vwI7Fi`1WsrNV9@^OPWB@iphQPApuf;jrMb-(}YP z>iscKUL`q2diBIri|p?TvTa?YH(_toVQ6b=aD6Kr#o?=$+T<^i_&n$1R`qjk%(>>O zSt(X?dC@6}5s9Cq>p3wgFcaR?lWvxIVK{|7O->9%7@NZvN_N%ltp#)zxRR^I?eM zZRekx+>WC%cnX{W3<|PpGvxjm9bB`)@X_i6(ZZYi-+!yzb=>h+zW9Uv#Sb5+1dUJd>KFMxLLobkV%LTbxQ4$UCQSCpWPeFO0)P34gHqC zUqX_aSHEP_)vn#2{79>lmvYU!4}HC|aEtWNl02`eC+8$2Zd_j$Q2*RO=wtOKYhI}U z*^2WD6+3V4A@mkPnb~`snbX-LJ4D%gbH>@(GvQ>U4>BUkmYF>cXP!7CA{jY*gru@1 zg_5Lxuj~8sw|l)`@8^6xAJ6Ce^{Pgd@;54;0OevR0W!k#ForR8^VhU_zOkaf0yrW% z-YRY=nuQD*Bu#WnnY(BDJmGeZ_oTS<2McHlAxmd9WxpxKNlrVk=e$2kYIPSKCCez4 zGISg{R?$&b-C^*8E45-a9Zpw1wNZel*fN8sn2gIDKf1Y1tXSSYeWycqoE*tF(cB!N zk$UGrAsPOuthwLtQvnGc?@ft%8cZsj>;X1WEVOQumcuNR6(A8(V1<6w2Uy+oahZsi z+I?STiSgEk{S+1riW|d;7VKlK4SvoD0Wv{zOESFwmn5cGY7^PT8QA_gKQ2ouk)f!* zs1RX07IHV#ibY$R_sLPudb$w)vEIvPnThn8x37Um48#BfD9DHyW(m}KmcAH6BQNSa zgeTg2Ep7;xH?9=-%Kv6W&7Clm>9!+5zC!j43ac~`;`nRL;v}d9bvss>1nMR067fXX zG%_L_Bm}I2JjoLD%9@!V5C?aVqB<2iH!{p9WWV+2Q@Z~Qd1sYLI;S)cWt#A~p!y(1 za678nTi=i|?)EJIk(;a14JOe+w%2CPDQ?Xw$GHi+u@(b#)0pkJ7 zJI~zRJbxh5FD_vrPz$nlZjC9N^l&BcMkb&1N!WuA`bg#G=L1ix0}2_+rCAc?#HuT?Wshk$Vk!$wQ7wZ;=`CFkyWGuC<*LP9 zcbO{QH*?WS!+MVMT7h!U5l^p-*78PH40P~KQW@OYA`i;${M=ikwRu+LskuuPj}27e zl{fx8CN?qekKli#Jp7Op6e^wT1SgzmzEV1VyF{Bi!nB~Qz+`+kR3;xZc1GQ0zTMDU zB&tz8;i~`MEmkWl`kL|wxg91iAX3t+5|mLGaiVnO19Vd!k@Ns_%*tpE$KDu@I>F(UuCSgqS zo`^Y7mCEEJ{+r6mnFpNN7W(^%35|6iE`7elK(k(c#gdFELTwEa%qeFKcU2pmSX+b* zVFr z0aq^U8Nk`FI|F;n!`}LWON`1J$bE?mc!$}9O19(FywdlzERC-YD}5M|@wXue#F$&P z6x?^oVUbndJ>6HG?FPSCz>e{ybNYI+NJOzM`0wQRuND&+ZXpoEPsJc2u2Oz zlyYH~D53U+fR)zgL}_avlRP5?lBL|t&}h~%bh&Va-IIh}IgzX}MqOHM$vd-K4Ezx= z_cW_)eKM{@u{(7bFWG0Y4Oxxf*H+NzMl8|`&Z@;>yfSC2b?<+2m^s6wan-JJnvImkX~)sWSU)6g7J4KUPmZ8)X0y^xGgheTcrG9 zVERP_ahU|~!CtKWP0#;N60TY=4JTed3T~A0w#cyE_OFAHl(0fdRfT}Tx{~?(6d1vb$~l+;7~QF>SJ$Ygw1FmAaxLh$&{B44%CSIwN zUHUN{0fw{*#=2am*@Q+X;J?lCOLXi?hNlMZwJYx6X*>VD}Ib$V9z)JXKa1cVR zjqIjO_GZI{##1jrPGLckm`3+9o)=#S;dc4aw@?9%68ci16O}8~K0Qr!P6}Oqy^A7O87r^!D=LhUHH+|a((q;=;tYWPb>N@h*h%7ugf6uA{C>S{P$WuOO{|3XbF=A!a4 zwN_kgG3DCg+9f3Tq~(!l0m`N{Q`gEM=Vf{)mY7{vWAsXMUeMzk9*V+Mda`q@j~(>M$oUA^Ty zXH`ZmcLaywC@S=LiU9+GUOsdpgiw+6Bk(3!0HJZTavsb+-k78mmyjatXyr<9)i zjU4v~r>NY{E=VR7of>kawu@Dh1- z?Y%r_aN~PTTt{};o2sS3E4w7C2v5%IP9^(t#HR7p!)~+d(}}mAamhA*6F;w=5#`LH zy>ub$=8&j(y1o62D6lFv2LqTVr=+E6Pe(RmxSid=Tc;UcriQQff0j*NZN_tw=+| z?%><~ACUF)s%!$=(g6gK<`f$d-wxNe_3eDjE91CLPUL9F$_L^>*FiH5ME?UVbB*f9L4@QpB;TFi0#i4rirB@I4k|NWV}S3~m)+Ax%;=ms!v9ic3- zk1H^#Ia!Em(lQ=}|9a@G)OxFwmTWzBSNH9-zR4Ml14|B@@RKfKolh-K?y!7nSxMt& zjNt8|kKiq+_F2&xBiSJZi?f2o>Dr{alziIX#A<>_FOzRlV~NgKK4a_Tv7fJ!KgSQQ zH=$@i)V|<%tADW?!6h>t4EyAZUd6@AaOlBDN@!NAVi!k6iGu;+?H~dcT@MfW`#bk2 zvujpk_KgOtdUn@6ST5g8ti#6+U#V%AJLgg+md{yA1nCrXw7Ey$DZAPf`aMhOCmuv^D zq-#teEzJd;)G`4!@+V=SaY=G5?%(8mD_N#MEu&o~Za8L9D(TOXyb=BG)zt~A+8K1> znhYc?8S~NCOyg#dUF8>agMN{v{k83*a$&6sa<|lsqnNdiw@4u|0TAF^qS5a)UN>9{ zK+I>2#vc{d%u=3l>A{-a$}sy3w`R2z?yS24Ou18^P%XWdWFqy(mm|t~4m)|=3epBN zO@}P`SbRKHI`qQ)k6fdI{sWKhu z-nq%_`=|c^*gS-W8fXczjPXH!bbTmoFe)VKw^@lxu^kNZF3b0rOagD3`3@Qn# z>AUaczU_DihlYj&Q5ik1ohF{QqlFDgB4QV1Plky=#9nmH&5$*nM;$mgAq1{Dl?Ico zEy7uG$od7Sp+~lGewp7zb=Trbf8iwvgq<9?#C)^L!{V07kX)3_wfU%R7bu#@$Lzn}m_x}In}|fYDocng=JRLR29Jc@ zhQx%F|5t_$*zW}=kEY(2O!yq_n41%#Fd5?JZ_X0&Wh7v!qfxredTJO-WuZYN0Ad52 zusW>pfhFB78fgW&%ldTjV3zMiz6vD*5n|ba7Tq1Cv@hwbk32jDnmh=U3UIvs<7L>AQCk#ktL3P`07 zBImPd>2lru5;z%g0a47cP9)TQkwt#nJ_vPC+=lNehq7urEmnL}>xdI}JP-U&qaj{jhDcY4`Mn}y|YM5^U$XYXtXJpK>v8K)>Sa!5v*Ho=7TDX{H zA6Mcb0g4f>H1xM{ZZ|75!tou#FW`Bm%6Kb-+_hL8I)ZEePj3V25>NGM!Vl{nX*k+= z*jdCKfB3-2s$1?r)~e_JGSng`$A}gHPCO-U3><~{s62D98RKViOR#1DOpo63U+$QY zX#{lnRu?_&h#tFu)Rja6&AG^e3>^TI-r3a*rVBCwpp-jD3z1nn`_df*V6TGM(3i=0 zQav&VzA&sWnTpvhJLhrdwEgTGZfc|-Fln=@>N`iGf+$9f^?vxLXo?{~tWt1t z(ldr7iT?4Tdb33oB8NMZ#SP5!7${4aP04bxrISMZC1%E73*KK*u#&DrHvm!;jDGBc z!VGZuED?5F71IxU9+2LZWMnL!1ttlA#q+KbpUFJNo;-dsJs?*By=jw54bmP+UgoE zSaf?!;+7W)jTQ_Ea1I|POQ&2TnEtvd*7B)(F&i5FaZ)6!OT5NIAAjjfJ3~Z3kPNtt zdMZIb5)j%Q0X~oWpCZv@ha`1mevT#Gz@9@@w`l zKU^f;s`Rmc_N;hRON6s!O#s`3pRDvnqDw{5)h0e5Zq%nRl>!_$BC}pH z@f-KG31s%9`^^m6Ok=egV>+b?ghT@yF84vgC_NpY_1?wX`Qk_J#`KxQ1OwJ%pFEDp zKf6-;ZDHdYkGxHl5LKF95lD>X^eL(2+;0OBbmbcf>sIku<=JIh zzuvpBpU_8NsA=;IDp0*T z{fV4F9&wFQ`KTC#Jq>d=idjQy*k<10g@OQQmzhOYh;^OOkMu1zTJ+*3EulwZLRgXd zfIr)!?PLRU#`)d)fX-!)qfwS&4>7~zZSNn4*C72xIAgm_?eE%c*w~z%j={v~= zl#WXy3a_+1*JHB&{U2QQ1zr~cBcV|oh{4IsqY#VPwPsPyNeuPqy^oJQMetUa>oC!d zHpRma9b2z}E|>$KGQH*-nPLz5|Aa4c8-TTs9j`(I8pn@Uk@hS~HYQh|0mDZ3Q4uN+ zAyCNRsq-I-lRY>^A?9ay+v}s{_Me*}5JIe?hoIfpeIxKjyqf%?(!+?IG!lqE3ofep z&U?8uZ0w;ZXW3rpi+%z-=M35-wu^Mr-w8vEe)39h$&%Oej=S3rGu^%~7;xrzBj%H2@@*t;?P_JCtxzJ=4N)M$R5q$AY*br3t zwFb>VR@oHA*VIev8Uq7q?P_8Fr6yJaNWUGtJywKNV5xZUKw1edviLSiH&*|I(o4q7 z3V2$~Jcybhu_z6ur*m=vi}k<^hL_^6V4}6c@_!;m@*7OE!yV}(S zZZ#{HQl9yfdN2^+qtB#(C2(L6!2?y9YfBIBh8ldgQ0mGb696QG z|1BEs2Jz!U7>8+>h*dWJ!XnzM3Xy=WWhI?Q}LN zXqb=Z0d~jMt$jgxc&A8ym`%GZyPW2dkr9Zp$pp2Oi9PWZMdKm5r)-UF4C0=`5Xia~aHcLb z(>p0t>!}HZ>QZvtj#Cpc4{F$`u&ZjKAY-uBap1TU?HcECCHiD3)+y47Qv1Ie2q{)+ zOpXN7Kl*Po0d@D#O=z@uJ&-k38Afbx(K=V))Vtz@JTfKZgO$oFB8bq-Yg7e(rIB@O zQEm-b>=(X)DLLa3fT*9UOe&=ia3>XEH&gPc5?0X+Co&6a1R}>{x3}g#wZ1W2d45fhxa_1v#^m%DcZ4HZt`upb#b10L6bj{l!ar$2;Xx@7z_m_kf!ZBCiJF3K} z8f83h6^%ex@h4LYd|#9r86kpn6TcIv6?)L)7HUF1h|02+GCL;=zG~J*w(9%;QmIRS z_jcpRT?Uq+#xXg&n_Wa@B|Uon$-&FYj0{qg_~+e&F`;;dR-=%b1QSQ$Dj7dxJDkeh zSs!y2X1{=350);iuV9iFz=RG4pm@>FQBHR4Dq!FM=4!d>&HC6aBJ6nU$$ar@AyE%u z$ePaq?SOJv|j;~x+V#qpHHE@s9{6k z7Qx+6i(FZ(*sBTX#4suMj<`i@5sJ>dI{`IE^ok9 zTO8VsZvf0zC0n(nS7Q}Z{akr$Na+VB@I;V)il|D4D?V>Z=5$GpqcLKURg>Mhldm1w zYj0437#!1<@Wqf&*0n3(5s1wqvEmS<-*#|VxP=wd7enxtz@9AC zBd9A$(J7*Q^H$e$h|`6=%J$c-lh(T`WJk$oXf$tMg1G#*^E~*GbSAk}wal>Mb}v0j zb(RL{rxcRmv5X9jA&11c0c)z`(o;W~!1k$%<`NG`vClC1Kt ztx2IrRG_rE1kf9sOx!~z`fLm1Su^%+A6-OF(HA4f6=A95R#qvuOAu(fDGYxeee zRa=|}FqTmJD+w;CEnqUh{qi;6#hAoEz`=vCYfEjyZFH2gj}vSHgvne*qj*jB$(!P3 z;Za~05GnC@(Wxpl@^@pZqmbLKQws&?U~Thk5BR1(tb$o0IeL4MSLfC){W_T90))d(y295RLDrhH?e`vJ`p_T{c7Ap5+pxJ+#GE)H zh$Nv-mI!Bh{7X5BVPs+*u=`#!G#jo@jjBO8-%AfmT;GkLp(la3sQtXb7B|^y)ILVB}$)pZ%gjj;NsV9?rVhIgTUp3Q!~u6C__vBxkz|5UYfS;*}!_v-PUI z%D9ELAqho8A_zN(8tMEJQGs1*1KO*54?{I5GPtQ>;ReVZPMPe_CpF*szOqB46z{R$ zs(5Dk>`}Rkm(OBoD_{bl5Ku~d*Hf*)`nFzJzJd)OgZv7X0kN$XD@ zgDwLoeR4hYRhK*ExmbuHF5|G<4=hsnv-fbKhF*@a~rg_Udf5}z~K?-svlw z*Q3KQmCJR~!zy5!M%z49r>%|wUFn{hLHq6}D&~L)5_1elPVT{u8$x79ai>_}I5~m0 z>(Rz2tD0F6lw+wN?Wix~6H>^M=&x&4)jqZHnvOPEqYW1!snU~=roFcmt17ou?uNR} zA+H&0`b@1Z&=vr7=K)nJEj?H3ivDd-NA00B?o<86AH-dRhrrI=&_qxb`g6IzJ#I|C zu4Z|@Ek3?%ZQh+Y{cw4ty^<}pWsMz7a2Yr+Vvql+r zK|Q1NF5g~q3cV9S1F++gMy8H`_FyR%gHFla?HKmRNS$6l8hAdGmP^}Oo6H`JiQ<8{ zJOPZD2zRQMDLCK*hoNr4)T7ZNdj^#G?M&O}KWxLB&i{dLM7WTZv?ayDTJn2ztxmbU4&3PAEIdAX$; zXNgs{iihB7d%=7{kQ2c`YNN~%=FZz9Sij>NMuNKF&q{0l9QS@Y-x&rv8{O14rsZ$b z^;j>*HKE6lx>Z(f;uUmL+KeNNu8o?%*#4)`*Xc>IV>JNJ-Ab=jUm-9GH2^VaW_Gg5 zJr^?=1J2b?m$5qF9a}E9kLy^lv$wbfJWW~cc|g6D&b!t0L0~=qyi|Ia&;*5#YEXK$;o++$J|mjDF@kOB8B7=Kh)i}GzV%LN_SV8Bu4 zTKWbqZ~jfVuQD?QQdW#*%hww^Syxz+=Q<8P(?NregxvGP@C(=^LfC$6O^ny(fCTA?Wzxn1SN91*Y+3tN<0aiUO*Uj0!uy&5o)^tR)h2_FX zznlw3_0#t90ELV4K_!LDi$i8#?GEEGVD!Lml0UY3LQg{6D3teqi@*j-v4I89mklop zv5&uG$vxM2wOKvulbT`JG!rLHe&z^?al91wf&V3zcsJ+X7lM$EN`ecnM8^jIsl=(( zNWSnY|;wQ4frA&l;b?s#(=}Eu)1YI6nH}DN<2X@ zvjo8?!t;Fj6V~gd!V!p#+okIj&@0i8sQJ~wbgm?)g_Kzcxq%A>EK?}Ahz2%G=AOH zMl#O#HTGlJ-h9h#^|q+6W>7-Mv_XLEXF4QyNe>Cm&URiqfQCI|4V{2f_zu(VI*v`SwT^ zo9HV{xiz5;&`q2grC;!xPe2jc@u(x9+hH4r0dyp2U%KHKH6>R3@vQvAseN zL9SDg4r;5L+_}RHI>7Y=cLNhK!OIa)1A_-!eRvW&5P>=vB;~eyoBVDY<#;HM%WMOG z52$YymmQu-%RVemr-j7uki~t_NQYkVCzVN#gB1u33^rv-bMt6~)_Y=z+T~zQGchhV z(FKS$BRGav#-mXpWi{&kvoQ6`*C73Pf+}HN%EqaHOUd1Imk|Liz36(B~pseZ(Ntj{t4mp&5KcqA5!TG!s8-J9Am9T-^IVjq8s_) zpq~2`q?frMIPPmrZFPARTDXnDfS47)h1)l1A}#_&!8_u(54T{MC58BWHFN!}e}TNv zGk0`Rjy4a$6FCQpX2*wXd9qjrjUc?dahAt2w0Qe0{PU(G3qP>`ZZ)aTvIi;~lAzqP zU;*6TVw>b7Z0c{PE0=_t1c2O^aTQ_L-hdlp8?mPf@%-^>n_D2J?Jq;KJUHY-$G12l zEkWP_V?95(JO23OK2q8G*Xdf2pSYg@y~4_}z3p5sm$(La%_Yg5VZCc_@H>+~dG}R~ z2f~9tfUt-SFihhlsKUE(_n_Gxq&h$1f#%7PX5>-otQgkk%6pB}i~@<1Sb#NR63lXC z-???0mRl4Aw#l(;_6GJe)uZ_nTpHzv)5BeE4>ofFs%#$u7I-MUVzIgkAG1t31qWcDM<%C_k+4`fUQrM zxijhtOefjpn!8eB;OZuc&OMa>kY{EdjdO{c3e|=)1EzIS5F97*PJrlbUYj zB*h5mJ;y~5m&EIFn4O%~((9L%0aXGiqUzhXh@f;cXq4a7d>$~{0p7iF-`9lu992bq zYYNzG3>h$LO6uZIMHAd{0Og>;Ox#%Ta&NAc7$I>(@Kj|SzHV~Z5W#?Sk9;t%%fp`U zk}%~AetAF?=h~#_TbAj*V{m?k`+-21)Ckli;2sn{r&R{fZz-a4F8}#APZ6Y_FL7mFvKJM$%nxl7VKSDN6&& z;;D(?1dJcorj%87Z+`@}%xU01#^YULU{Cv-Q(NB!>%|SHbY^hgD%osD;{xZ$|9>(` zF;|%@huW{$Otw8-^-CxG$p*>)#Q^3`5;-psvv0|E!nO2s%r$6HQCxushXmAZ zxkG-KKm>G9Q6xskkdmA+3r{`ty`Kl_t-0G}Uw%oD*0GfW`(LNYxZ#vPyDCR}##UR% zwVTi?ZYOT*hh3;-YhNG<1pa?Q5aGy|8eQ`mssiYh=OI7(0N5GQlAuw&2UK|3osB~` zC3rv?eN&w`aW}NU12l)BJb)w=ts1$i{tn2<#V`gevC}d~Np3+48@2^~iyQ>PQiVxC z*366i7v;?vKJZ`b4W;350i2jwAC$Ga9@v?ux;{&ydm}1v;cZ%ZX5t#y%Ttm#*UJnr z{gN40J(!tZ=BfODBXwcueSvs7@NG%ZRuCu20Iw5{Im$WDW7olCf+HMoz7GmekQfP4 z@XK{;b=Xj3?k9(+%X_%-2#gZ2_EkNdh zg3$H_WR#yo;0Pe72+!ZSW?$=f7_C~I+irBnU@Z!hSsBSs^aOC1iUO-P>SN^chLM4t zj-G%uLGR9qR22w%fiU(Jz@H9Af--nbVCh){HV6O}*hTCS2<>NCwS4r0Qi6Q(bXU!`8N-{$sRAR-WViP37gh++XTFbe}(3adg7>+8yUz?`W+ z6dkQawU$P)NmN>HLsQm zO?nV2wtx{bcm5<#CIzhOiw+z{iI)--ZuSDf#B_5^$V!I+)~>p^jpe!lZ(`Xc z5Mo6@@K&?2mQT-$Jt`R;l?su3NIv;*IZ1wp;d!x+o9QMj7@w?$W0L=Mw zG0lm(0?#NQMp$tYV8Zy#BNp=2M=DZax0lC-I;_SElYqxX;|0Z^c0e2G*Wfpv|VY4V{DGGJd zptyG`_(3f$mWR;}d;vBE0E{|#OKD_mK~v9kGEqS>DAHj=9RZ3^N`Xw^h{8Vwjc&O5 zAyzt&QB0I_ZJMVcb&{1DF#rt^p8I&flx5!7u1?8@MAhGu*-D!HvDvNn#DqNI5AC}M z_V%U#qoNg$rNajFinw^}q`VtfI_8mLU^uux2!u(dnU8rUB!MSaxap+hM-JG-$EbXO zUrJHjJFnuxmbkjl)zZibV5j70eaL^o2=SSL4;q%L1#NbVya$Sd?)!oV>S0v2$HO45j;MDM6BaW~A z7$K$yG2^(N;i7QYEsvLcwU-D&0*8l$p9cQvT4sOe)~}EjDscUbizwA5e$F$oTEGY_F;#l5|Gd)IlEk3# z73G-a7F%<_(;Bpi{CNpavr;Qi;^;Xwo}I(ic+p6M=v5}+h!O`2be)=CNa)a4OuVk(RW`g4s} zqX8{C>6ki7uAWE?UR1B#aWur39y%692QyoiEdf^De39-6 z>a&j--h#ClzwN#1mh3&3#;U-n&K8pY{irk@er8toIYBZBWm z?4e&gUyl~z-uqMwkNiQSx+DwEV!=o_=7MVC zv*=m+f1N>Hrbkc{oUQo19BO^T>Is#(EQ!6tqJ(qZKg}-^ng8zmx^9qMJp0~8Y=IL0 zAmKGAcZ_qcW2>1H*~mFReeYigYi(|3oCq3JiN%zeWPc&^OA7(5mf>(W8tkXltwvB= zg8`jx(rqcTX!s!r*mCTf(GS0Mf^dIDRkqUjioQS3Ss4;8Wc^gV9uX_2Ug_AH?jd$_ zg{L)0{i=GwgJkng4MTBvtZ4dm-?x$bH5b#-17;gu2 zub8~vv`?2H4y3^4DP?YT&qSTLe{)JTV)A4roagYqWEn?J&|Q-(E?;ec4DUblW+ZZ#s@)}Zl<;u`&j~bgWKqp7_uoXO z;`YePYOy@b4dsb~>N4giKU3MH+{+8sb6uxd(JGs9vUyRId9NY^v{;0q16MPsA231Xd z>KkW#LwsMrav=>i8JgPTJ%s5PIQay~Vw0f&8ANTgeMWT-iRG7P5vu!6N( zwpdi!?zWwE?5!t({D2Sw+6}9Fx{fv|$0UZ)fQj7IsRydEin)nonqa%lS#H+SHdRKz!&j@wAZHgrylf@gK=)A_r$vq%I~xFo|-$p zeo&Uw1Bsxf8pzU>9G7X7^^4AXrcg-38QBc_Re&dvoX)RNSvlyg^g4`wM;8A^ysPX| zlk`iSNhy;Sv_Zi;OfDb}bEiFOHTgr{sr)83kIg3ad9@6`321V1)f!I6<;pft??~ zDG7bM)rEn5XIsE^Xj65r;7)*Lbe*+|H)8c)(iGp;qz@B$S2Z_9_A$EA^YysmHOqDB zau7LPK=-ht0N$#HKTPb;LF`LjDiHMfs$I@ye_df1!Y%g1(C$yBB_7M8(wf`ZhdbR^-*AB;#J%+a*x^_-51jf!^V zg{xk`kKl3C*3*)CJln9Z>47M8`0_DRD4L3{xtr_6bO6ws&dpu zUsGW?@=#Op2iK>W|3MP#EEQOIM}v~&J5BBG2(~?2^Un(v0{c`C3?4k?Z?xluQ(w5i zC5y9d`>7=sH=6u1THm~HUl^m0@DI+nA{EI^DEoau(y~>Sp&Sbe8kr$IYt4_UgH7$t zW?$jHt{1)VG?PCv2Le>T(|-s@C1v(zYe`Ono}TOT!NB6d6(fZw3qG{cf0mm+^~8+y zltQ@sE7})nnNcObrI2$`oB96e{Z~yVx}&0FTZOU>Z592= z?#CJa$yVuJoX`{#rXi^8VXuV&z-kqt+m%;8PCX!lIDo;5j)MDoGv72TitC#=Uy4J0 z?$*Qau-v~zbyb|znI&P4wTDj?vbg7JR;(>B2yH!Mc+5cj>bRLv{J;fsX;OqX7|~c; z(q8SE9tYbGR`i9b0#>A2`E~f%p>1myeD-kmhRykzNSJ zryBScK#8l0$StU0@v@W!?^W!!U3csU8#pq_WZ&DnxPI#RzKzeZaKAKbukZ!CAgzPE z`Fd5TZA-=ZMzM*QcU@hz9t&T?RZLR&zO~v}+(4D_^epol zG$3RWlu$F!tZee~JHgWkr}QwXiPn%rxr;J+&CFjNJQ8~JBzcoBfPaIj(afR5`!UvW zs}iF}t*~!#a{IC$Y`wQy79tOiRq{BfKbMcj-;OQl#_|_DKAlWHr)n_$n;M6Na+i>l zSEwaq?gXTkbsvaN0VitKC#XHal;VDWD9VaNiEZhauW?O1F*+K)@U`mkEz10$E8-_Y zC+uu9KgQaR{n~0C1 zT_EDe%j<*ACWn} zXVllqNL?;JIs+Rn^R$obTlLMXP50lDkq(N!Zg97EDU)aZ&0_5)*{>Vdu#IS6SCKc4 zEWP$%jg$EbWbi>u+5j)P(+CbT|E?;8+dGAza0-ILEd=TceSNbF=GU*R$c5SkTZsA} zhg99nZ;a|q@gctesqf;bj`Z)nGG?9R-WEJYuX+XHb?NklTAX^doZd)_8Dc(Ec~d;D zukhoLx&ELDnoQEf0P8VI8FF^yaVBpkgZNXXMc24wF2d-q+liMR?&mzLOvP7D zb^hCik53%M^(2Q*_hvQ(t4|cx{DvtpC{0x+^f%Q@ z`)(6@ve6%$%2_4FCkrYIXV-QPg$4Fne5Z@le`W8O27N+X5BWH}A0>adVS8el!(2r? z9rhvY9{!}}2ebB_lENy&`N13DDhzN_7`R@q$fU+X>O_9<$1}Y~;TwX(s~rJ`-=1uk zb3{G`ohAy>L2JuJtelvLtNO{Gq9}&Izn{`=p-O`~Va`fl|Oj25)HzZ*)8N;jwP(8nyQy?*pUWF?SAyG*$%BiZGPWs&##Q!QR`tEWO= z4m!ryz8&jbDsWip=kZZ425koL2O6+_NwJ?I!Zaipl06xGUsR1RoJa0R1HJGAy`-I@ zQeTrhIG#hjjElR8Ij^U;@MVN-XC_$0vSo^|4Gm7wXO&tZF~268jIV~oKQxOC&SK~UhbeDB;p8Fn` zkVQ=ov9X0r@joXd9E9jj;~TjlnC`bysyVpNY-w@RO(uom>%}Pq51frswzQ!_CxSez~Vi%wI%PDbH>ct^@Zl z3tCh`iyB@pI$=E^{V{rCGxJt01g0+&_UKfJ)|S^h$lzy#Z8h7*zRZ$;)j0YyA6Id2 zL{Br4^%-TTpkbwpxAq=h;XTt^YN4AMk(X%#I{Ov-FYW^NpHMibC z5I=hF!<`vRXU|*WZuv>E)B*xs>*#C6lNFgT{e}wf$rC%z4W(^xk+@bh%+UG|^U>3z z?S_4(3BD|yjmMeGg5Qc_X8BAPS&KSLtV7`&mOEf(ridn5fe)I?qG;>DaBKW9J^F=1 z?x+uA-zyk_oPZFb7C)ef=msn)88}M1H|bx-_a(X74a?@g8eh257=-yT=qWuJ6RZKh zgb*qiK#mj9sA#lW1F)N+;A{S#TmTIhD2TnJ?+Y`(LVed<5Dg!N*7PA=2Ach^>JRky zu`kO;ch=~(qUKf$s?#EzN0WW&1&t<9)UDqK(yk1OEC;9co*zJ;27d%y65H78pOYZ?_y7rq9fSKd!bn2N_pJSux)kaqz zS1=`J;|I$h5hHbx==Hzxo_F`3%!$_ z1@>8o9J~Y!;!8s4NbFGUv+lHG?ny#_vfvQ~3xoDT^Dlo5zx5I98Tg(dVc=rx^-M_p zhRhvrJJoU{((4|Neg_9{2D_GZIIF_t|9iazOK46Fy!pJLD&vF(0tMZUM#1>04ht!* zGPdR7`aflnF#T4e8h_2`2kS|VbkN|HL{t`54Sl;j z74OixL|Yf>Eru%Boz1x4Gl0`gc8nA6UB2~4jQZ+upQ#J>kCB|w(Td0IyToh4xI3?w znwe2MWne6T5GRdixphYCV zc$e&tS%H>8Eg47W3;)?lyXtNGmyXU)Cqx>3MebL6#Zcd}$Tt7zIuaSuKN`7*+e-WR zK)8O^VPEGvip71VnqwzA^XPd-0u8LcYrIU3u1W`Ac|;HE>PWLQR+ z% zT7Xf9+6$@NM#m%9KbH9gx`f?;=`V^ZAsG#Z5;7GH5!U_LcLFaa{I@kTHwMZFg=|`@ zlgeInWAv$G{mO553q_p|MX%XV*ULXp8IsX^vREBYqK-spPq`0!gDnYzLc$bF}WDql+F50&q5ZSN5bQGCVwVoLH)L;`*!q1=?^`L+B3Su zxp*S|Nk~i?VBmctqx|pRH<2S8y<(A2!&>=5 zD{AhAtLKl>bSjv)@l|K@cdt4;OZuQief#?|5Z|2XSvqRphB`59ZIJI;lBlA-=_{0f z0b8=zqWg=u@A@YAeTwv~-01hcQ>RiVfBENG0;7bhI2k?jC$58gLT!I`ylC!j{=1@}phg*A&8YhjJg#e7zB zInxAWcTS?wz6QRo#`J?46aAtOjoX=Nv7ak_exj_)BlRa+?{XdR`|LDbfquMagB2fF z#O)Sjmau9dLnR#bhoxy5hY(#oR zONGwNnnH#H!y_RW-q4bGede6p#N!(r|3}n&2U7k1|NqJ;S%o-8GLAj7GE-LP*km3% zdu7WmJ3_~?vLj@V!?DRq4%s^+B%92H((gLnug~}A&*Gft^Lk#->v~*|$Nh1;&qS`@ zf0j5Ree{D$mXr^|1qH%Ht;p4}tD1R&CKSX(;t|9pH#ali`#7M3wxt+~QmYDWI1CCj z%IVUTerOd0gwzFLY&Zz@IBZ|k@O^*wZ1hR>U}4rmI3edKtp)F|Ln&#Q1TyU;y`BEZ zHUz-^6zk3k@W3z1SNi4|;BKt728O6l73j_3?^pjjK^LY0+E`thAdB$ua7Cie19U;f z7U(b_Qp|J}(5L?7B{gsuI8y@H$88|u!v(b<^OdyEcYd={U}2QA+JQzHUheQw{IKlR zjA840O1dXgVr1}1HpFMCslyG{zZWykf6AwoH>mTH_e$&~-EWqEbeOxtdJ7=!c&o^D ztZk?C*tw}NH-4{!$hF)%k5NKcJ!N#6upAiiL8Q(C$@h#eggOHvj%eY@rsFq*z=gyv0*fy~+)l4FS3D;cF&N75)&=Z%5JMwUiu`w*Ugy=b{D zhAeAwcFEgQxpZfVmC^q2)h7m(G5V|-Ol|~IVz-5om6?uXnBtUMCOq2x{XX0Cr+bcN za}(s#opEzXo7>QtOfrpPtK^0D(oY3Bn23r`CgprA)bnMAThY28GNUw@0LOQMQ6Fs?olt6G zN9`X4iY#uwxMnG#aeq$xNBfY>Pg#bt z%$RnT`+ppFMxWGfKjLnQMi_q1AN2Q|7XA{sv*;}v7Fcm&r!Z9?W_+tz={%}OS>r8q z-{N?ZJv2l|81<2|S$Fkz=p%m!Yy=D#4XMXt{<+MH!cV8nwGr5{W?z&m13V>EW~tI! zHc-Ibfltbiq?HD7WQOdY$rC4uwt%hE%9aA)k}zeH3=zh>jc{U%jk^_8sQ;G@P!#eS zA}z2T`ZO={VY=&~=5xC2?hd)ycGuK$m2`5C7%9Jo3M(($!(_99s3j?RlrNq$)nSsK z$lcnY<@n_dJ@QSinB^lTqMt#__?AkwUl@%fngLM>#rJ1m#_8=92+m^fN9j4%u_?#J)LeUlh{+sY^wcca=^&lnwqEKaPX_(p2%D`0ZzM!GV)pcvBgR7zTRfaU$}3Gb zNj$*$`;E!~a~A2=b;w5HyT4DSyi;synZNr?PRTp3xCa?JV8+BpQ)w$c{Iq1o2+_w$ z1jib%KG%H_?jaya3`dbIGD8M4w^DBW@!YU{#X6+O6G-!i1V4u+n;|fvPgPk!Q#zlv zG|N>{p{ZblUMXiIn*q=7`|`Q6N#%1OO3ReQt*%41D*TIC0lrn-nv*YMb=EZ%D#UH3 zSv6^Bl%>m<^vwC8%2k;HrmfyV7^EV+ypqmPXm3HNhmViFcD0bbrgOW^ZE(U{J)_b2 z!XwuLhRU8QMnO*~oMdL`pANL$Tufh7jTYo>Em9fX9LpGDU(1v^0pzTN@D}DQ5A0o) z#rJ##S99_}SOHu=*j`Uu9g?L1qis&+wVg`pNMd$LWY?x+mEXHxQ_tz=z>nt6bbeL#4V7e zFRc`zE#og2z*3Q#eWfD#Vxu~qSCiEnBlE(L6N;6?I!=v18I7!CqwmpMr57d?OAGUT zmVBHVCUf%p=>1Ppk9tWe;gKcaMyc?I1kddj?g-iIxLm)TY$i8N1e35(j)Z7Cz*ZkJ ze;6uwH1|s@Z!R(E20HW!W!A^DX>0>W-J1Om8zT;pvlfTHP^7YM_Xi8gVv*y(k&J!c z7BwM5CpfhCj#4^LHM!rf=a$a!*nN;!7?-jVPOGv2heEf=9Ac7!PR!uvLkjGx4>xR& zSZJA3eq5=rl6o%(JImD2eo(CYA>_lDgU1=>a2|qs?Ntv8O)!)sl@-8iR{|25(v2u6 zJI$t>!mO

@h-WD|E(7)hd|9uci`hCVtAOfTHOvwdxGQ(!8+j1Qmx_ZwRy<#L`r zbdrMOtUgCZpYRiB(**oS-YGP6)^Mr{Fu)D@ea-hG@sZQF70w^ipmg$X>R-L~$h}gQ z8-k*lZ+2R7k(>#f)as1>XakNegio`sb~|lGwIO<&7^CyJ)&p&i`(Nmm5K^{|7Ef$D zE8GXIfBc~qpf~`IwEBsLDK4{vsXxM%yvblxX=F}LF3Xlk4WjrS+YAXR7PvZvKBx&< zoUz>JjpW%01Hp;DbiXjdWpw|Y1AUs;Ba-r>$~-BC*R<9e^4iogi#GToJ#O16CNTQh z1b>#-IX)3NA1b!qla~ar$&wD_t$4fn=R++#TLf=w)NUAg9G*DHD!b8Rmoi!ktbm@(wE*qM+`*HwB?_gc8u=yB?l-VspP(dDqKzNk%nLg)O84b_E>?sz<~p9}?$1 zi(h|2o37C$UVu0RaO|$5}7`r|Y-=JCF>-AoFRo;ZraUodBESy9YeI zNT|kUviFbk;}<}SKv@t(0r)K@r*;E~B9Ff<4{79Xi>ZvuUk^FE4+0X66E$aPy6>Pb zRQ9IQgu(7mmGe(_vSZ)etk<>r>Qm5g$Gg~I<1f1H#7x4J#7vPj>?5R^Kn}1=Sdc_Y zmFJ1F`+Gj~$f?3}#y(lAML0cP=dgH4G;7I)HoBXiIiq5or}{moj`ZC(slPX@ZoWZ zvvFT%W0;12188G^i7MAeh#0I%o0l2?Y=HkQ#$%SLb|#{V)g&&0Vpa*W*L*U5{t6P*mI<2Qfr%Yo9 z?gD`kDekEs3e~#I)(U;7pru1WrHQL77Udu6vcl6Jj25Y=xlED@L*b4@gPkV-I=&4C zAPrg8%}!zQ+$!T&C3SQKXN`N_^?w|#I5L7$Y3(M~dAfI31Uwg>R}BrZLUJS$@1#IA zqK&;yo?n$9i=J4h->%dX>MgbS^sQ;PF*brXS0oNU(Xcwvtf2HI$*YlAJPMuXJ2{oE zGWaMpiiGcExXxT=+?LQUk4i7pb~JeXI_Ex?k(2F5$T%^&6z{_oSVq_qXT6#M(k&n) z*!2o~{CIj)S=Mubk^hXc+j@WhHN*D;RIX|XgZ z!;;nUFvG)@q?+3#-#Emd3CCK){#F+!kLn(Gm#cNyDZjs!m8etnGt>5FfUAJ$tyJRw z0>Cy4dw=i5NXV4F!r>O9NLPPS1@n_kzf<7feFPuX(&u*^p7c3>;p>x`T8!ZO{G&!! zYpQ}q-<5gn&0AE+hDYeHq!&lsEZA!HrR_2uYjGImHo3T39?fL+W2#$RFM!xW;MGNA zLM8<^ZB9-BEr8(OBBYD|{Jp6{Z|Vo~_BPtTV^+P^oENL5Z ztIvaiDKMs4yU<#>T!Qq#?zYlG?&}noCcAY5v(;?JR~%34U7VWQr7#Gb0gG zs3W+*HX>B}MHQAU7b)-eZ@PwhvNx*v`l(ak!)@qpeI`Lmxa#E9|9gClqHLYxO&_E> zi>4`P^<^)kI||gNEGFDEW(@+`zZYkB8vIt~@e=~@US7d%gOnY^jH&O87}*2w!TB`i zy+hQSVvh?#zZfG&Ljy>l@r2o$rYVD}Iw3NoaMptvreYhPQ%7)SsJi2m&f-wc9V%lP z5pX5l690(od2tMT|0EKc2lLYs3{{_XUie&qDSR#9a1~+&%^I@mRWyBXZRBmUOi6)n z>eXUeX6G}~)m~5D=l#qs4b%+472%HP>wYW?b)$GL+WOuJ-u+BOlf=;o7`>odj7XOx zDapiDVj`Ynkt~g3hkG`HZsvBvZ>PJE5X%p@D0pr6PpOJ&Ce3Ry`(+AjVLcqBBJAmw zv;z&MoHI+EJhGNoo6~-3^!CQDQfEI7ze!pAuo4h;2=hr2)1#opw^T%*C9-KQY*tM2 z5EGF{I5z7B(DQ3v$@~#4^TKyfeqjF3Jvad#1n42k3vW#z$4$Q18M*TLLMSO2LJO`= zJ)%EZOi%qWlLr&^!?zyWX~d3rxm-yk zOQ;~)0Fm;$)gTg&8y599LHC_;&@xCTzdQs&*wPIWqF{^m*5Rl)RCkvf5CV83X_hm$ zGRA{EOVWO>e~~1mMtOltJKt@t%sFWs?s7XD*q;57#+6iyU^tR9muPX4?KtbRwuDjA zsw~6JGkX}UB4mt3pwVmz4FlIf^X^rR)%^udT56I=V10%SsO&2ZN{U5*%A6GaLI;j}EsnaOL}jX5+y=2sxA=s^cQ~v!_xk44blKh6=THg_zb!8h-XFr< zkIJRV|B|i~ZEm$Zh0O#mh~&uQc&5K&rnNhm=N}*#E)yO=Uo^pYBm`q~ZOC;=T7lcMM z>IwKyXg6f!DD2lf12y#YY7b}YY-S#sh30Vqyo$XjqOK$&ZGaJ_I76}`x=6yZ6_bPR z0;kH2Le+zahLX3tDzwniJUO0XI+%s1y26w%Yt*34zibhmp)Sxr2mxZ{@<@8YSM@d5 zqH3M4F)l>HR~sd@L(Cmmsz1!+gVo1bt%@!|=FO}CUi4V1Lx}ksDb$S;9nQ>y;iIt# z%w(;%$CxjE?AD_lg6`k*36XFPx`WmD<~eEnIMwn zinshNMZ#@L^OB*Qq#HM*sfS8bBNOT#kP!Xi-;+#>hOXvu`N|%I!xzxD@2Fy*y*Z#a z7g=^DY+W5&)gju# z-q<_gdqTbiGLmrO5eoA}@~4pibZqHwxJ4a8BG}4Eo>Q|Boi(t3V&`GM0+r2JZ7Imi z=wM`Ie9Mj;nS2%ODY^*ljWOpY;JT**%Vr;B+?ZIBpiyyn_mlUQ`7>iqEfQ-?(XqQhZFZuZa#bO<}ktekB)#3HTz1j-~UtSko2_`r(6YH zp`bn?I=bqd-mAGhmXR;5NEzE5`Y4?i*xt5z0CzuLgO%F*oRV74>#WHLerjfO{kVBY z!+^P?{ZoQ3xt<1EycP9eD##0c+9^1v^)wfOliYQ5JX#KcFN{uB-#|d@lzO374i+;N zGc*jI9#~83<&aMtZ_f z01utaVpN^2OpRRab9X=SPZ&{Jxu@kNk4K=eR|h`M9An zoyD(6Ra5?7@*iR_(uHpspFKkA{0eHft9}|61Ue*{TWbHB+&mJ>&C-goAySm_8$YKsc2eb~ zKDJhCi5wB)|I5!h$+_7QH`@Ae_S02v_}A5BSmJ9!1)Z!K2l`*(22!HpC&RF0i!zKHrdM?B`&i|B|I$8&$;w zz(3`c`zA^*#!x|k4zz)DzH^VlEV=f#!_>aH^)dT3?y1B9gKw!dos2cRnItUKN+Wgr zrG&l^0rCSGqg6AYJ^GzmLik_?^|D}Ik6hzC9*zJ=Gs@2`T6TG#l^`VLTSZCXDoeJr8^W* z6SQ==Q-@LqsU&0T42C(VvOe|-c66yW{W*xNeH}Vz1Ec(FD#a>n>it|@wu-#G7rtIf z*fu}1BK24COt&E1J|g33E^K;)DmsYdPt$wQ)H&^qQvtOdrzF9oKG5x2VhKjOfRC_E zDLTK8(|FWmScDtdIkw|0TSL}b<)&qn0)JD}U9$Tzdx@zbo)HvEz)ALe-cS z^W&|BtV7(T(3>6(T!Lv$yl%OA2o`Be^r@nr`0C2!{!4g^HqDJFXx5S$;cQr@wE2=H zvdGVJ*xYWiKW4$9BMtVfAc-1K7pg3A7ZyBP(*%)X*%y*nYoMo0)@%(t6`s;azSCVq zuQNHJj}jme`^FyQc^k&$)^t@+>&exb*Cjc0fx(Yis!NQ7dN2BTQQ4=^C3TnwD6?zC zP?u}?529n+hDGD5}ZDnuTep~jW1+B-l8J0I?Oq5;_|kfW|1G|{0eJ_`jvfFjNV=!oZl z5$xa2VnyLbF3!dC&=3a66KN>}z2xyH%LbBP4^@R&8vkQ|7FH z21(~U{PwMuKnJX7?i~31Pb?sxCCbyf9vXAjPX)+jX;AUkZcQ>7cE~^Phz^cyyVxd) zUirG_n*C2ai+!i%JVM|a&b^8VbX%4haY0tttlzn3$&U67DgGo(4^O$1h3x{@b>Hn& zw~_g~;mQSE&5n{(2MRL8?0{#5&2bHmI(mNp|2;uNuAy~ zrB%3o?b@}>?;T_`l*U^0erYiOLQQ*BZYFN5Wy9e7HJa+d3zPk<6&cCW*R1JxMmsaGR;~2;j*Z@zK4(R@)WrKCyVsB)0)*{y=37Pr%BD%2y2a-JVS3Md_ zO0cZfOXI5h@jk@OE2_H`0ejb1 z2z@wUF?ox-&Hy7*_=m&sk%n!58NQhrAVIxJg=xeYy>>oWU*=16u9DoTNp^BYsx)OtEfoAVt=J3VtSr?JhG zew;^#N;Dq`sL(GZr1N^-dhQCN*^~lJJuA{9>0O~PX)%v2^y!Tf#q2d8 zbKe;3DL}3;=GS^l9B~!9cmH^U`eVCUjU<@p9e!o`YY-(=UY0u4x?K;~8YYTt8jLzH z|4}&77?k(VTBK~4cq>*#dGJV~X3#lli>0zG|4N=r*2+wBS;%KK2O;WNeZAXRR$i^W za`hMD9P$x@n{5G>$I&eBf1d3|*tR_%AWs;eMdc~r4}buk0K~wyOYiT^hD!F2jn!it zehTW%FENqi)1Z6?8NUzyBB(6cLBzT*=?c3SsxCU48r}9XBnf*SFIr5C&?pt_D(q=K z8Y*Wp>y(nW$LD)O@;4#^YGD8Lp?RWgkp5mWd-L5@zcxNyhZ%paov85pF4vELz;17-UbU6X^8clUunY@e2{#^dK7d=083^$je7|>E?+`B1zw$;=Lv1<%B!K6l*Ek2 zZ=u_R96(GP$x*JJBG0pCH0=y}HfApTR6QVPt0TgR{sE+~#0X>meC<>>wx@j-IpO%I zt)kF$-0r0T23x~Rg^6xFXK^^_mXfPZ;-9F>Yr5((m1ew~clqOFpJ@1 zJZ($miDW5i*a${dKs{knlvzXc+kLgk3{iD)Etd|hM9VVQTFl{{ZlaF}OfPxfZWNH# zN=ir|m9W+h!V59hvPr+%rwl-T_|xw*aiIh-ifi{^t%B>#x_CltSA{FZFSTV&{KijM%hgmZ0`W$ z1Dz)2RojdRfh)xAHGO2RcPbUm_VX>_q@28KEd8ziS$n_uR_Nb+YR;2+v<3&s6lkTs z7s;B?5J&uh(zJ9|B;iIW{MTt`^Ha9z3`#ia_?QR8hQN!$Se;T+?Tu{7qhcSCD~PXc zqt`i{A{E|DmohUj^b5bPt%EWH`yc1<;D;jrOoiV(ja}QvGvl&du|NN|0TOZ%<`Vs( zJK_sDf4}unVAd~`rBlk3H;?9irOQ^n$}{raK9Dh=zRPy6;oLY?@lEg5hKx5iwp`~O zJ$+^*6AddU8>6n>v!=aJYK$vm{C$q4Zy*7V=N_?( z_xB@P8oeL(`?mFm(jYI1M!SByW@qt^_powAW2 zFk*3jF03O2RnIEeVB1h7ba>Z)r8evHD(U^ItP`fa9A-4%_vq<~&lOoYVQVu_7H%BG zL06NKZVBWsHvE1}!I<{kdFfb|)FT|A%;ClDtMjY;G7orBRTsg0 zjMZQ4ozj)-E|8uJI`ku{_XB@a=$GsCoVU86wW6(-Ec_P2t6QS=dephR- zrqOvbfyJhr9lsv}phq8znhX1}t3{`qnQ-1`8{rRw79Oc=6eT3yZ02etKT znAHCMtADLxWxDujWA6nL%m{kbaV|K1^`rdPBTS)yCVKEXAJL1BEj&CAdUtA~GR`Jh zNV98Lu?ST_jjLdoZ8(2=&NKvl5DP;j^@<+7`Y&B#YXsBbMPPRsg-fzsBD5dNLE%L4 zH3;VXM&`AWM0;O(;$|&kwhhvOSNa6Oy%CW2lj#G0y<2Unw^)S;mf`&xM+Hx*g_ zJ@Wm9EdjZ=^XeHhQkbGBwX(kF9n*oo{qRl#Dmp!I7=3(0@-%chOsRuhRBi%e>WPkn^B>_ zk2>kS{@WUHFWPm9Q-EKKaggqr1?Bc568{I<=ZaSM+|_$ukvL5A9}01?aG$flYwaYu zRWEXF4!9*9M?1gcr>+Pc-51jKcS-GPFEj(gL`?g+5#`M6!bZEDr;~4W#g2%x8&dr9 z8ip|GAbRLZ3Ou7R_y-+If^MbF^cJcc*&ksQr~BNeI%GGqv?NBqvrW2~&OSsYh6LWn z1lw5Vf5H?bcC6Y!!nWDcN7acW= zO6dDSI%WPgI&#bON|;$PnO@tWoPPD;O3i_hWk)a8qEUO>mF2y8MaaJx%C*RBP5)J5 zgSxJ1pAX4`oK<8fs*OAK8Xr9jna%v z4@CdzTzA&HzbRj6%N9g8KIEOj{xU|2Q%$^)6TMcxI+0%~^WBg+n4dVGySaGGf=9E$CKkrri{ z_+iY5WkUXb|2xb=b-)8W3%N_-;H2@GD5BW*&TCKU1gbQwvw#tkGgq0_bB@6?s4Ir+ zhJ$9#TX8r4X#qHm{jx~(Kai2o?>4F?Qyi;rkz{;`Mu=FmelqDv+zU<&_##QG$RHh! z@oCAl){vES#O|GCC>!6;(xCn7H|2&`LCfUkA%%T-XFQf5ea4do@bi*5zWr&=s%iGWGbO3p42=eJHeD!$JtJ~I27DK}>|TMAkzz6bv8b8c%W8P{gi>^zS< zkvsM4LvQoaAs9Yo^&fp(aY1>qzR2J^inyutN0Wo9?>2|DW=~((R^%tn9XC1=4o#?( zf_=SRX8_#5i1(oIx0lN5_7AxcI7SEx=T>4O@}t4V`mvR8+8r_}(ZQ8KgD|Bk;qFG; zRkpX!GKr+oJhOWV%Lb=_ zi6^2zbi)%b8swU|zVO#SouXa-LO!Gvp+rCN2~kKRVb5RvQ?D|}(Bo6}x0h2uIcVHfAnvbI3=(IQL>$>|jCsFCY2H~&JCz2<@hmSj^FM@;rVWWwAt1QR zw`eHzg{Bo^$6TYS6zXK84Xke)G^(*do8vhA4O~ryd7W2h4Y2BoJ;>TG5~?gzmeATr z>mhKoKQr#PPM%D!Q8`dDt^Yj@_D<*0{kbZyY$O7_)hHh(knen@_GHAg^mm~HiZ{W< z$uig0zs;3COGAeTY6+w6KM2OnhL$bOn{pI}G+*f7BYCPNC9?nfu>9R6B1 z=|i!BzJ&~2#t92h=?)tT05)OjbTwESuC|cR)tBxVGNoXN_>&?lWX;;|{56g^^1=zg zLk)YQHG{~HIK6YFGT{EC)=Zw?y4p#Of-o>=C|{ypZDRSlJ;(b~;PzSVw5AQ(%6teJ z{8lWHR9({a%4nC~K*(<6qjUWd!e^g*CcID0-uutgXPUNqj6WE^8Uq?9p#Ho3zjZ#r0X0#wsjS!ddT3-`5i?yTB0>R~M1D?t0HoM$+f*nXbG*frLb)0HkdKrlhgsQAqVwmVZNYGmleP;dy!Ay=1C=^n-GrguJHA+dip}rmL=XOrwl&xB- znZM%X!MhyzIoU=Jwb`_r6D-XM>CSwmVvR$DK)``od8^E2tt9u*imhk<35lApX;wFM zkNP;8f$L)xS%8Amo#qtmffzTP-_zRL8B@-!39i6}LpU}j4>Zxk6-*Y=JRTjZ-ILAf z;gNLL?SEkTz3i4!O8V&+*K8J!7gsqy^i8(>6+&f+9Mq9LNS!uq?}t|iNuQ}r5rxpA zDy}<5D<<*Dr0QqNt3XalIOR6khZ_|$+U;&#)8{{bFRi#dk;||)9QL2snLgbZ<$4}} ze#d5J@wxx|K^ljJ5l+$SbiiKKp?)K<)6(5*6~XbX#7urO`n&dPMfv527L<2MJpWhp z{kgY?w8o#%PR14^Z`i5w(gz%x+0^TKNCCe{?q00C$Hn@oyfStxri*hZ^P;m8kMkFF zkCz|hrNB>uEi)(F<6yhVd2zm!P=Vsce|oyJ(A)su?K%WtxA!FeK`LbABXH=Dk;HVR z=rxv%N(}2xi`Wnl?Djt5NLa3YDEw{+bQF$^+-g1OcFO~Nm%)^XC+BlFtfaEgh<8=@ z1>W|r_*JALGgYH+nEw8dny{jFXq;Bs-VbzR9aWlbgdCvtf9=eIRF?HWiQLFx5af5X z%=F8_P}95rdXOR~B^jKBpkbS)cUzU7Z4Z4btLnGdz6{@yBp<_<8*GBV99Pbj)Ci)Z zt~f54BsA;#gLKAqmX+6_6mzlCEA`8U-7bM9?d0#rQhk5Qd;=8$T~%b(*9Ovu(m!v^ zQZ(^;aWDbRX4j==(_(#Sb0=-T)h@%6etLZg3FU^M1?Z%_%HK7Lme=l~e7z7Hp6bd< z{biB2ZBwX)Jp6JaXsuR;6A0?|?1AwA;_QA+{Ykb@8--M>M>yy{7|%~T%q5nR63z+f zbJ=f%YC<(SG&MMj@19*0wHd^uIv~+LJRg(YRY0b&rs(6Tbkw*7&c+`2J$32VXbE}g zAMYpa#Xle?eii;8@FASvI#12fbZXle`6;z%lFZl_9EF@u@`vbWb;|91KQyVW>k~Ae zL?x+_9(Hy%yuW#s!6(RCmpBTF-6q5;`m#%Tbbl!oUG z>2g`9X8mtN?VKpJ2)j4eQsq$n+iJ)OWJv{g!;!gI>p`)!{|(|d-%MmE0W%QC7OG3n zwvfnnL)>|Ye!V?bmq|R@j?y5Sc&+F$829;mNlEYE_X|?%iG&2s?}5VF<~ciW%>Ecl zy;bC|DyCI34GjIuL}CO1M1SNB%g1poJBrzIe=OPPgQ^hkT4<$GH=CH=d@iF&FT@^Z z3Nz+w3lM(T9L!r0^y6=!O8iYTZKT(N_5Utp$_FM+eGY;MmD@7AoRg|gl5;}-si*bf-^P?U&NPUv zv>8>J&h&jTF}+H&vV`yaybBHPanNMmI$*4*~YR8rP2;YvnR! zf4=(Akq`?vO2TzvtTXFk`zkx;LJZ&5PFE@ZFSG&0r@!wGP4neM)PMbu$M1xQ9Qjs; z*u>2?$h4^K$hy1y?Y)tWacQc1X8jO4P|nUmpz^XXiHJ#Wv?7F7KO73rVy1X%&I5`( z_szgsRXvp%@;I2|@sbp5<>p<*_}sPPXX#yx-49Xl8+7K2~_4xf`d80~F9m9j4xiNX#3frB-HY%cxw9XW3j5$g`3IC#Lqd=MpuFhbV z!aZnmpmPz(kRLkNH0`~ivHa8MO#)Ej? zy1Q~Zh8RJNJ8Q4&r3 zQoSCN&t5rnB(24_GLKHMS=uWhye_z9v{A5f0Tq(W;&0!l#n zPL;}bKx10J?dzG<6nWOuQ}3VQBH_0~({rAD2uoJ?!qyXq-s(%i5CzurS92S6X7UIHXJM|J?>e1YnxucNf6zpZMy?X+e$j^M;!P zo6ml7R0esE4>lX6pLVJQNxL$Xwbu0$Sz;ba^Az9jBkA|i<#l}17txmfm8cpz5&9=w zV-P%vMC{$^6DVKO4Xo3_+6r1G!q+l4tU^%bN9AD9eX_VYEG{^`Y^nohxB-mY8Xd6c zfqG-4#^TM^O;X*EmgjN z=5M_<@&DwG-u;*2Qg-6ciRQIE_8}C19Ld!CEb*-hZHI@HfxxTdM$$|TW0?7$!=t`o z1^oY-#XzTx_LZ6su-VS%4XV2XxqPA$Hn%SnC?qu zHsQ*D@U_wrDJOdV1rEaFYmA`D{iSC(N2K*R39v8ufBg!z zlPY(bA*g)2-+&Hk@n%U^Y{n{J9emdR+ErO0)-`&7d;{q4=+ns*6C>|+_y7Jc4pZk4 zPZ+j=oInvVf{5j%r_BQ0D5L^x{fujJEMj8RVw#@7*4F7fKK z%X=WD$JG{9lk`?iSdJuBu*e4~|8jco3k?U1*2~&v^<-W7k&qjVavU5Cw?*dFFATxa zA+F;I1L&%5ICd6mMMC)r@bggM!iR484I#Ej|DUos8ho;&i{eK+@D$FYL6ge=u;!QZ zD)ycZM2kG^R`k$yHSo26!$|DS)@m`h}n+Z=gXWlL%d0utoSx);JEM z@yoK;b4pgE!DQ6^BhDY94(C+xaz1oC)JxdDye3u^I$#}!;TSr+@|hT~p$Zo<0#tz> zs?6I|>{(Z`ULrBi<+Ed%E^Y4%2_!)ihuvGTQicI2)a4Uen+qgYc;G2?_r#pRTXTFr2N1ke5JQpSC)WVc5zmM47p3awt2Tmr zM>pApm~n0Lm!OqpOSeIKg@1Gcr7EK8xRNAZGi`wMJ3iufDt$Wijwa(vIpFT z^p(%49!U<_YbMK`5Ji9vCnCaCZsbu-ZqSK>j_Rlo)#5>dut|DtI9N4)cwvb@hjtLh zj+nx9h&Vy~a9Pf7i+VKh)KD_21FR^)*2FLDmd2OnNRHRzBn=IJ9aL{gIH5}V8L zARnN0F8@SP$gM|2T_OR|>-m_8=zzc0h03aX41gx>>ZJ z{%c8&ALJaH>p?##$y4A6exLLT?gjRVJgu!w1N1#3)%$)5|N5wu72eSP?~KN7UBv5G zaVX=OUgv-_8m=Q|fxK5}R{&q*6E^5tcG~lRM5WKuaVKyt{>oAz-pdfM4zlA60N+R5 zP*;*%!+B=rJ|G45XR-ikhgZexwjeS;o%Nm)UeRy)7C6AX%E!@J8{XlVqj}kpb(2zd zR$sWiJHA@D?3poGP5a5>z_)m1p~mh$f4>6L9i;^6&&hgXINWn2j<)$$^G!x0ZIYwH zt9y}v9rNSb2W-@##0DlEd!Q5kl#vuw|2v?h?}X{;efBd1sEw>Ej9wy=pBWgbe&O=6HDg( zhxdwbD9p=WUPon%IJsP(Y%Fv!E*JS3b|OYhe0Jj!qe%q~f;8ln+{>pa!=yY-aRWvc zmpIuDD(#iJXS0PavwZ!a2Fz$Dv^WB0?JU`R(M*7vw{GLWVcE~avN9gr-Xet4X$*D= zfd=e4{-9~S(P7-{`*Qwi8{3DaI+sa$6j=|X+#UDXfD#-(qOO<*aha_%DnzRVs64IG zcY3yqTk?V9-HOyCVhDI12~O(Jhhd100ZqFL2UB!|>5M^~_+RMg)ouOs+_nU;{HrOE zOk&{GBdkh&i@d!TlgIq`__`3l5{Jv`yl zRw=G45ahCo^%Mwy*JP)IgUz6FqSXbQ>3Cm%> zr@Vh%(#OS9!M=cc!oVf{a@UQNSH4?Kxh~TMn$@<8mebxvd;?x5kHG79Pl6%jE|1Hs zW6)f2T(;CW-MHkGm)73pR!d4~{EyMzI(COJ!4_esBh=mslHrV(xJDTm9>fYps$~RC zfZAo&xJW-Cxz_`NI`(@#EcM_G6o3k|ggi-W90;W*GT9KoDu>zruT~S!a%U{Bjps4z zNXxi81#>w=pS2ci2YE0qvVc18*8u75L2HwzBx5tRKjl@N0|^Tl5?LS^akPrubb(X>--?70$Z&Q@o9UnzusvB z;=XKf8RnjHfjpS7lT}|9r$OH?UPdk_k=N8zng`^!`I}PxJ+hAH0S^SI^+#t9GB@SKtL;>&q3-k8-5db zi6~1n15o(kUaiKOnOl~|+fQspTMb)>IBwueY8*b?6T+1cKSy#;+-0^KV&zqd=WTkT$>@WU+(~RMV~?w zvzrNsIOj}9uCqOZh~ma)v9sg3!)7_WexoIf zk(8?xSWqB^Psyl1Hcz`R}#CA3q6zj066@n!rsC)r!Av@YmnJG_K5aM zqm)Mgyx;C%VVpxFaYI*hvQ|l3!tSsI5*IUKuSO-i;(F=*hI2Q-heBdENFt3!YfjpL zWpoJ}1%6PX*=^jW=cYU?n__0uYo9u|eFI$0`+Z;M^6E+Ka>3cR z{*>!G9l)yKxv8WS5$askJ^G3YvlQh4ecV7W6u{C@#&bE}h;Z}mN9#l&51_c)7{x2w z7-<~J)cfey3_-d++F*pn7r_M2K^`YGT+!fU0DCZayo3d^Z(u@h8+!hq$mss}Z~->W zgIh~pq;&_;IKWX~l=;nGH0BLtd?s+x${6*=jGg4jq^H~io zxU?g|vRoJAP1toxSh~3CyQ<}uP&Sy0*=UNP0V&WDPQRZu&f>`vb^MO(;4R=zxUK7u z9+~p^y}R}oQG6S|J$}TJ2 zp!Ajg7gD@u_f_TuH1VZrehXG2Q-%bJZD3h{j%jYjm5#{Ks zueAxUswPKF4I-hB#KGXFNp`efgcmx1yjdT+m$-uIl*_DHqlc0=4gYS&lp?DAUDFy( z@=C8KE%19U2$c5V3%@D<_LwFD>`5=5$S)vNyaqiV|E1~q9+tge}0( zi*Rv8kMJY1dEwA-%DM_ua_3h3f1G`JIFxS~?%4Nz%}xl7 zh{0I18*3TsOwri4QQ1OvB71gOv!=n=DI#Pi`%ZR3lp;zC=hg2z=Q`JQ{yFD-e@yDi zH{bhy@B6&ZeLu_XHg8Uy8&x$R;?M4xwS3ErS;tNEDtZqUQ5gJ|aseoPsy=^)P2X zpB6}6?n#$?q_P)s$OKdO()ziKbwRvS3+|y>;S0BZNhJH%zQ}W1})TFwwi6vx{t5# zbqCUzTwC$RS4m&90c_J%i`ta+*vW6PFz9aJHR5J_o;kn;a#FxR;2-aAHO%rw{BmBPrm3m|5T(REt zN)KNW$UY^okJMD8kIKgzSf?$9kHDej`P2l>qc!6=4_#?{XE5#OIYEnsZPUTkOzrp0W)}^2lhHnwuf)a-K9x$vmUq4KLvWnYNx8S~ms|w-mxk96C4Ytm; zoolY~qOsxkN(32W1tem}{gjGEja9&yc)^&+e;JYD8@&?011T#Z&F$N_fUW`|j}HVg z_T(VT`!e^_zT)wptC-wv5-lOnJx9!HsXFR$ju-^99g2A^;;mO;*7sBayruXsv6mcn z<%43SFX!ujETeAdiWHg|h40Qp|0&%u9D)*uY~hpu0O5mPigDU8*}ZgozU^aX_78KZNHN8$r}P2~ z9H?qdLaF^JCEo+IM%q^@W1%)>a>*MtmU9VM)v=n~GzB^tcJ7kBk5~5|rEQmbC5pxW z*b6B5D-BsK8XwgtaCD!#7si-JM@xJ7!MM?^90!{IK{|pVbX_E+0ShjNfYdd!9y(Z!ClhW%FF_1TCi8C*IZMfcfAI{jMsPDiyFbGlEV< zJZlUPt`2YPWq&iKK?Sx6xe_&7vZgpt8+Q$J9O+*(;)!~3c}U9opR=@5V~SWUFnnu< zc4KVp_=W&j?w{&m_`9ddpW0M!0*w=Vp7Phq#_!}nt!MA$p0p44mR_TKoDLrZa{zQ0 z*}8PItTsbstRNHnD!MTv-}!@{vtQ`mt~Oj~7XD=D&s(U2x^aMv;mwBtt zB5he}8n5T$KU%^X^#P0=9#ZloY>o5sl3+sZ$b6B=%O9Y;{2KJ1GA12ZmC=F}fw=>! zVa=PCY(@_kVg76}CGlA}5^@kJT$bidv48y_*DHeve%d4tJbJHrfv9C#a90*53HDpO zE>|q!yAD7!pjCwO5f5^N$3dyWZYBKVX??`4piciGb|~40e-B58fLi2 zc*(%{iBBJ-Ukw9i?LMpSU%8RPBGq9fU;q4nPB!TOAfDt1?8L~{R3A;S1*GFGA<%o} zqtgH-5yooCh~8&P)mqeyr>p{7N8`Q-LDegOP+>a}8@s(`Qv*!RAqK|5%N3tDq$?eN zu}%NI8%`i|ZXlQE7*|Bj-+-S$D_TtZ7=?#jPOikeC?t5j@DKh^g=;U2DGep>O@d7D z+O!b?$G?6i@GgnU3lw;siUxGb{{tIO!!Qu9F>iGKwkc(dK-yuext@$6V8i&Q8eh%S zJx1I)y;zia8q1w>mz~2Wd0N{RCEw1^xFB}$$#1={R76k%ZbaA)78g=LF zmuVB|S)aV^Yme1^!ufk|&H7j=;%Sumny(Q3vodR#l1&*Ppz%Tk{JM5=)|0ivt*dtg zM6cY{$nWX`#`2#^YucJls=FYMZTxlZSWQYGE|s@!%BVT?_Wr(mDE8v*A;rCiXJITx z!$-E@k@|{3Jw5QX1|9_H+wEkKUx2Bnx+D>;R3evFewroh~DenNdA$}9R5QExN+2- zeeAn)GKTf;>r4Z@LkHGobD(BNMEx0w&4==-!HQ zh8;b_6(50C?Er{qT`w{k6%?Ce>kR)-jP%F>Fu3cpcJa*F%~^#8WhTESU!T&FTkviR zM4&Q!oWi|ESbCxZL3S%1*!s!cTE+Z4>T(l~3++sC!JPYY132<}Bdj#=MLZrnhyh-* zu%&d(d(4*PJ}UqZr?q^$z#xA*4giTZH_nf4+yE6E2T9)eV^Gd||IKHRMo?wxcHKsr z`64^vdT)~0juLhKj24idg}CEORjutDkImZXVm2c#9C~*_xzcOCLv?yUxMLMjG9IlU z8GqEe`o~f4J4lvLYKA}p3^-CWtATg^Hc)=3?&jaAn+`a439xk#e@07Ja=w`i}9**6KyP)l+8$#U3 zleU|4MUp3PU+hz5M>PcFrD&9epw6=8=*-8|R~B#A)<+bfLAUQ-2*c!WgZ833j(CaK z55G|vUgVv>TgqMM+sFmL7i{&5G6E(Cq;Psa09BK8pb>sIcb@K*u~rIkK{`>TdJtk7 zUN3QKuDO&WaaxSA{57gh$?29IrIFG)+xx7gDz2!!JXb7kY|>eM(TZi5ZtVa)7r{lK@WioXjwn)OTJGVmuKTu1S- zJJUXMij?C@nL58xL-VyOy)tB4mpuE?#=;gq^MzvAUdxNY*05hKJ5!X$;5pv`lUc) zmS@)G0*rQiPs5Qw47%;dvYR{n`mff#Be3yZUw1@#$-e@T3f`wWTGu~Gp{rlfpRJ7=RF&W8PjmRBHF$Wy~shefd6T3}#2g`B7p z5cb@WE(}hIs4c|^_;%og8nQZwRDYo`M#y7u@vg#?1U)6Nmg4lw0ohNYQf(FsT+2}z zGKLyMf=~@|zy~nq$)j~gx{|9TdtEWR=fEZB-Q&2E5Mdq`W2q7dz z#>X#RgfV;UJtmHXFDEm8YLV@wKA!EAf_FXUY!L;HTWPR6&;-o`9cwxUW&CkO-d3}I zV}y{XQYPx`Lso6jFCPZPXg0p7S!}z5pZYewIh%qOdgIO|AKg4xg-Ic=pFwGLIV9!H zGA29m{sERJH6`2k@_*BZgJwK^>g4|U0S8a@6COC2!2njt<<%Wlev}TFl&0H&A1DsD zP13$Rg6S81V$<&q_AlP%0!cnP{f8vUX7y$#hvC~VCkeUVDm`7EotZHWFA<5QCp`VJ zOT2;23O)uNKb)rSV?MOA51);19O2$pj-wqZVLChCU-G@fw9QNQ^?u45sOl|EEiL9B z)~|n5F1E~$;(ca}ou*sBA7&LZ?n{fVZcD}JSFe9$6cX0)H|rn%-r(P+AQmMg_|tkf zXnttxOY?Sr%YpJnT<0(Z!h1?DS<-b?7A92^Qc1+9@7Y~-HcwYSk7XoI5Qz=vyI5Aj z3EHr%_m$t&1~*>Z60yulX&WViNU@-*2IyQC;E9>r8YZ=oM*Q2^fwXZT$8Z)y{mJTp0(qsm`=^#-3c==eTP8b>e z_eLCJrE!>cHkWR-FD5T-$~Aw(kJYXqh!dh39!j7>^;>3E`i-E6(kO~d9FesEbM?W3Ru9W+)w8!v zWmn|fN0Pim!}4-pP%rV|0cI(~@}^)@=z$PW873tC2}Md>PTbgJ;;ju;V@6$$=s#4| zZ*Rv+gBK@TTr3GGkrW*-L0-Co#?0r*DCA|xNPr`4=lGG1P9NMvtWcj363TUPn~A4x zxyMB!9MsoIBN7{>o(;1K#HkmiEihHa6d4{ijCrz?(m-?>&KJ8Bt}ymF20D%tQnA1| zL>0lKV%~3Ms+~XNoCq@N?YgbIW%%3-ceE72Q?vaAn3{VnzQ@sO%Q~xz=ULgbZ zc&%3|#WBs*S8HUTDJW#0RHv!$|6xu;lM0dq_qUdvnZlWhozfBIfr!1#b1+FRtgEFuKvvMY5KEe5P|uj}`dh zg6b+Ksu=|dc9Ugwv zFb}TL7P@D{j;XqUCg1ii9`EpPV;uKAiHN8Zv(mThsf0J`_bG;Sv0_~cV2?3*WQ=N*a zgz}W;#%XE0dLf2}iZo&bBk2me&XEwcJ*{-#RD_u_5}}Fgpl?qm;3w|Lgc|+e-~IGl zhOIxF)(<@Lcuh=RvKN`+$kH!w%9P`xW*te#vJUiZKGBO=9vNrRpLe3_G z(z0{E0X~spzgP>F+L3TNd|QIq%q2QQteBZ>sjh)Rd?#k}^w~3ZmYlsL_t6&?wzpFn zvj|ZfkhffiF(##9WWa>_$WK`Q=BeGMgyz*7l}+z{hoU$~kMv6Qe(5C-#shZ@(sZ#; z1+CWg>F^qi@(J2@wU*43w&UTz!LBH#iF)=@WMsi#RZMe0fY_LMp2LLz@|Jj_-|^wq zFdDO4xHM=U_@6dm1Vby8lePJX+GHaAvEHLrCCQ{mLh}c?U;vC@0H4N9*W#%#e^udL68Q6ai>z^%~4mw&+LE&Z(cULH&;M% zBB8(b-UB-{H$HZ_HgcHq*%!R@OQeLP2(1YO>8=F|XIf{V3%Y^XBlS0)k*M9m!9~#+ z)N>LwdaYE`=y|NMh`}=$_bQdWyZtSksoulMunoTi?O{j5={;_c$A_Md^`p8Z@6A5Qq0a*vjG^waC%AA>{#g~u^`w+IkFU4%Zv{% z7LP;C;UAWb$_V!Tmp;0LbHJ|c;qH;n358?Sx8b6N|0-h$NE^? z67kHPK$6sIQB!kCL=TTg9W~bd^JSFk!bV(Ws>hwDTxoJH>4K_4lcu0&aDsY%0!RrT zF*Szl@F6K~G1QwEED{v--%)o<%#2jS09%RKG!;wSCQ)0}92X?lR-*^roO+?4RGr5J zWn_3H&1*eKzsTIHu)uZGnS*|X*ID(;dbjFF5XeLlym`f>OCvhZydss5bn`Ya=N{zj z9rrd}mMSTh{A6jvdg!pMX*I=+;Fn$rA2;Us%9oIu@k+A$28f^wjy?(H=eEYI z-Bw|WJC7UrLYpG9M~F&Qt`DYQVBkeIaZqb#{LYHDywmBCtJl*b!IVG^*){+36s&Ah z5tVowj0W@fM{SkvY88{a7sWGNT$bUoryp>KAO?s~qwv~3ik zO}jRBV#b_d$hRZ{bsuA8YjGCyaF@6q15K>7i=C)e#2?FDaaq#-@0)k`Cp5Jus9msg zDdwDup#_sJG936;JI1(6bI~LxH)L)3gV(^;SXZHB-&|P@=liPkytk1-3+yxurF&9M z97XO2T$_oF6*>p$=`0$dzGS zH4v7GV@=a59e*hW9CzKYO1l^H@5RDlPgD4^4T|{4nX8W?rZOVI3OE*I7w;z%duhrw z>NWFiP)Wb6!_o zD@?_}Bp+>$9&Bz3KHngGYmaaTA9FD4N4)sy6ZJwJOBbj-|H`>tQft8lYq1PRJ z75Y>XqPh^hC=^rK)(gIpg;HznT_lQ(PA~6_lX5;STOkjuCpF;2=ri`~N>`;o)#GWjfQ9vd_|6E;Kc-L{HR6SkS;8sbN zlCeGiz$v4^@WSe8>SDEtPru{#CutxMm&;*awD;pI%O*Xzws&^=6~anfg*G#T_CK1G z^UzAs&*%?hW$O`OA$sQO;y(geuqiW9;V27GVV=#Vl`29xJ`R>_tuKSyZF;SAX6jVv zcdZb?P#FCcbWL!Q8HeG1B~BqPKf$Nz1#)%z(;WW@@X5fJVLC)TSb${z(<9dHnup9Goxxgphe+~`;Qpb7|yWSM6;k+Qv3zn)a z%|{@ZgTG32np-}ToCZ!CW@7N4*b~rd;KmqZuvZv~)%39cfQ1%Ik1ch^{7pkjNRs|! z><^&4zF&_suefoR30%#YaRjjbjS4fd9UfvXs`bov7+S#IcngHFl0N)qFO?;5`7-e4 zvdy&c=8ACmOH(mTswC$9FP8(l5UHiHfN*zB5PoU{U>R8IwDgmazopZ~qY}yr^ucJp3 zSj5Dt;~#J|l#ua~P$L7th-m5P*tD*TSJ=Wh#98A~Q<+d@4|{Hs2nrZA6qo{Im8w^o z)_cXW&)kzV0$ssEO{(GTO+DJz-Tkt}NT*0dPpH37!ww5mWz=Qb>*+*OTb{@yL`9O- zin3WEodAStM9YS%U644gk?NvaROBG2Jn@vcMX>f`BrK7YU5&m(W$R~NXt%CaO3Al0 zvGhTEDcx^Gd*b5c9QYZLSX+bmu*louj|Ih&mfP}=r({O=}LtFR+>gS=c!im!5rnxYt3fLg;IAKQ! zqf?W~-n2@NBJg=#eH7Y?i-Ar6z+VwYWlqJ4pBOQP3!l(p+Du?sLt@FRHTlXrd5w6hd{6J7Q#eqErIOo z^X{z1cMI!fn{cCIW4A^Hw>pwCC<-7l<+!gXb!|5jqS2$)z!s%IMglOek8iW@&xI+D z%$Y!LuN(AmZi@m&CYEcc18fYGBp-RW2xU0mj(06pa z51d_6@(UC-39Wd4p8K8Hv)p2P9UX6d^_B0wFC1fL+BTgBqj!zt(>xeFLmdY{| zzHa0x=OS1>_pyne#S4~ z0+*&V3eLzzG+0n`B2K`bk2_ns%9b8>^*j2%$IOC+^U!X=F(C>*4HHYR^PXm{LB~j^ z2zz@}DGX3q0D_UoHmRXA1jtUFwq93LnII(~vR9!vQLJm?e}K%FxthOWisb^f?36AVS4>Y@Fxno9CmZ4X-hnO3slgwy+ z%+t9KI<>|``jfq1TeCOdZaTt3Eg-4fo*YmlOi0UAv6?tW923KvF$;jeE?1koX&TdO zF~zY`1C%UMNmc-jG$zahxaqnmPN z73|g~nK?D~<&axaTO!1_+t^J%ptGR1s5uDS!^F(NOASI7@o_ z*>yY6mNhdzh#)20Dm)1rO0ZYhoevT~=!x;>mt$)IJ}q*V?C$OcRxl!&gjSnqloLOx z(koJl<9rWT$x7P|UdxyCPJSY)A3GL?H>FGKLQppa8?(0@9r2{KRIVpCvHIZv+14df zi?MKS7uX*0>DKf2h;oU8q(z{KK^WESwf#2ex2%63c&g!6|0CJ^%ek#f1S+~5we(Kf zrZayRF02E`X(h0I;5lqBuA&Ps+9$rVgOgf2G0qs5rdi8Rum?a%5pz<_38gvwk&xPp zWXuuZA?-CX{PUe^334u*RXALah}yc1q4NPeSj8ILdO3lC>s87 z-%{$teXhplFk2Xsa;;OJ9uQ$3xk5a~%x2{x^PxuRdOfFEILNX zD1x_|NJ+kgj}0SVU#`Q`2fDwtq6goq@0u*cESi)$|VB zP3$L_i29yk(w*}Yp^~Ax%-o?sIZ!4IyejCEmc8ZzC)0ET2DsmHSIQ#2Z-NRJOs25!%_ndP(jefP<@eNpFY4!x|-IMaEXwXZeRNitcwU$v^N)>Hm?2&6T z48^Hqbp#rwY^i5AzT3Pmj+0fk{|dbBSRz?SL7^8TS>t`5BLO%e-N4VjZ;C4_4X8kR z^T*Z>vq4Q&8z3$Q{9V`fSSTaYbPcl)f!79D)xAm_N4}H%+>}zaqsa@#p#D|XvT60g zpF22W9i_{F_v?QR*u}fK#MO2s*1Db^QKI!EsRJd`z1`iKIy!DspV%Y#0eabnFJq!@ zW_9Z*%!b4HMAF4h!=S8GisrevRK~}z2^nqn7?nnzo%tyzr&X@x`K(GwXi)*q8H^u0 zj23r$Fjn_Ky<)oV2cDS%!zB#?&BqCdI?4tn<@JBKqRxekfnxy6g4d(OV~Ym5ClEQ% zj5&4CNZk5!)$>$HG3Z06>WrvKX|M;9OrTWC zsMjGK#Cmocphwwg{F=Y)y8J^P!I`dTEjEut5jq6OG}2FsCD6!arh!~C%}q3@tKx>{ zOh1{T!3Ih%>HdgW39HXm3md4ioI3UEGX zCoXA(zdG9bOfl%UnKfI|-=ScDT5hFRgl3UYW77aD3UCuP%P?V(RI&acsecE3S$!(^ zIUKC24CHl8hlo=@tVCbvq}u!G3|6*cy0sz7DytUtdD^wuyaEv30Qot%)VzpHQWtsDjeRZmu%7Cu-s;u|yfIsB8_W(GDR5a+kJ25;Dhp#RzkC{-wi=Np$p_PmW0M zhl8doO}YogwhZc*xkDccsm=S zIFWem%YB#RjeswNGZjtOE(J^`In)fMzZyzFeYf)G>`q-g8CPErv)Ie&+P0WYmz?>J zNX?Ez@K+AmUs)UQy>LCf@+6Ek2{G^#?p6X%Ve*Axj-v0E#aH)JYwss=$@eC5=frap z>A$wbcBwxr31j<|+70y)2PalwIwN6of+am>! z5e*&#bd6uEbI^~qAmLkz_44n&_Xb&Xlsou57F?jAbYdd-jERY1ATDmKNVzWDTY&W7 za*B?L$!kYi1FY3#X^Isjx;Fk+HO5=l)}3m`NV63^TyHvzT#&(=4^33n8desD17~p0 zTcORGcUz<>5oF4RUIz#-v!5mXk0_3aV)O{GUMJOP_l%5EWdP8Qt7_29BmhNAWf7~v zvNR)oW?m4V8wRT7FNQJ3K9qd<>!>G0zyCM$Kr+@;^$*EIox8zh&xfanw5M-BVGZ0# zDKU4m+zU@DL1iCA1JCq<^j?$Tv6SA>4M0`eI|ICi%!54_>cbfbC;FHnMTmcm&Bq*j z#I|K@_~$1iWjNjhe3fvj$Z=3CFU0cAjoK%?Nag+jgW5}0rD?R-EG<5A79{JNvCJRB zDJ{Pcsk8%%=K$%7TK0YPp@o<4XTd+n9i|c?Izou^S*h5WC^$A}mxvCtVwyJC;z{*`b5u+%yobhc$)M z9OUD}X=ZKmuL?QSMv&C+fcjzHSn31j|qPy_X zdLBJj*;vR{iqXw2bkr#A-Do zZ(L<_>?BS~wTWDW7FG2ahCezpYGn`^!9utm)V$Of{H$7Mq&Ly_0N2>L9D=8YcAjY= zIm0UrSG$Nc5ARTlnPGY8WZjdIxU@&~x^G0%^;o^?%n4a#!S75Ug(oX4q!f3K%SFZlDwM_>Q_)n3Ssw_qb@bdd;WAzwnQ@Ch^LB8GboQ>RT{#Mf;!J(%x6v0*A^#7wdm`_oRtBv5KhGNkX@49qpRQjJe`JG-$*bs^MgFI`VGjISr| zl1hU*!k>9lIlB_ix&vBMK+cCR80h7@-PIiSv=i}ihfwUH>|;0;G8ZEo!tU6?YN^y= zOXT(@jIT7!(FSmt%*4r7k0}9m64NpkbVFcdbJF$d1F0j_GsYIILy8P8pJ}zi476k! z#5f)Kwk*tVKjHm4p^1Z@a`48bY6~NDj769=S?7yigDIo3=k}MVUE~Ffl~e)XHFUWj zOy&n78A1&-f|poB%qNN2N(uxXSXVg-KBa_~pb@B#cr3CO*R(6cklb3)lMB5acZ z*arcDjAb>PsQ?;*Ivn!F`K!gU&fNe99^TxGD`ze`Q$pXQ05tLd8@3xzQl^DDAstxqvL z+w}s^zu(1~vQtwwy(gX0t3R$0HQ!3D);p3Is8E}c%vTvXr4LaS$_AXK2}LjBIdnJ! z9&#J^vP6mHd9V^23ewMMi*WRR8hcvBwFHZ5$IX#Ky6ZKZl>OU!Xl zC5bI0W-ZKne(;NX1w7FL#K07qDAcpl(<844#vu``Mg0-PagorvRvOBlCp4RH^f|lz zV^wHL89%_!uGuSkCf$A8M@O`-RFUMMg?M0hn-grOQ)hxY;o*X(8nmU*(QK?T6*~e1 z2!}!*E=fH9nife0X8J6t&2okxgk_OJ)#Xb21OTQGQBXeuXK7_U*3a)}1Q{JT59gKQ zaXL6tkhl18^!dubK+FdGgJgt6?w`LB?Z?-im#tuBz3yNSm~r|oFu5E0`yEHQB8Qcu zDSYG{3EAQ`VUrhO&hV#9Ed521ul?6|HeSDDq#Ho1&-tjt>8@i9c>4sK%#lT(sSowN zT;xY019E9L7rUDk&pqOXuTZ9$16t1Iod7H6SYO!Vsn*M!U)~Q85{+cXqg1-(Pxu%0 zOV^p}u2X#XfP>AEM7sV^*Y+GxWFQSKQr-CV0~^_P#nKT$u?afyK9B#&!S^)}kbMt zQ_lBesTMUjZ29qWSJKiJT2T9!>$KS>NGE`p(;DMmx6oe_`^+jdZF$T47A}?e8*h0Dpqx}3yb3*$5PrM`PG!?R`n)Il z0A%JMI4Qj4Y;OSHD{%vUks7mPYCaz`L&h$I$Z5lW|T=YPl1(CuV1PP1s$RQ-4DzgKD2_Xy9sc&Fpqj#>F!sHUr2O$MJ&WNZ#U zf3RP1+EQb3b<7b>jD*3(9+h=c)vK=|xZww(h-DJ18RdKPM7-2hc;3n6Y;2}|A{_R* z)1Gn5{{`Ni!BI37w)ajjk^q{WlucDauI4)V^XL3i4u#4g*3^OS1?}}$kLhtGL$;?c zqkg8<%|P5F3x590RyhgJN3_^f$G-~uqy1Ki^e6p|eolJ(H+p%|mWZ!53DBZdLFNPf z>6vz2e}KaI#D#nYP5e_VaLnw&Pbk9-^<=3K`;3Gb%ZYX6It5JLZ{5%C1usNzIHc%o5-lE|J`3|lQ zVPZgcfeXX*Xy%0*S6NEkDPaj;g0}ksSf~)y-cVNH@Pey`g5ck8LgqkL*1vzjf<&)> zmn=W#zaMhwfmZ;o|LyLPJ+`&J-2eWSFAwAk{rlH`&P+QwzZZYjW}vU7Jm^x5869W zY|xip$sC<%LSEc6iGN-k!#>%*F?MZO;PSf?nVY=Jdy{qM99-z)>6_1v;%<)!8hXmT z2UMrf|ib#Z<1 zUpW5xF;hNu*wJfYiVPHx)mlO`N1j_Xdat!!O3e^|7WBRP1^{#nR?U8APn8$X-CX3V z?fq~vnHUQL{+Zlo{P`YNv**w& z-1N(yGyI;!xmS_i=O~;NFWBncIJ+kYE6vj-4mnExN0Cucxk`uI`R+LiKX!8Bi=eDS zyLL~8($06tzAWX0D#}%7LQSqYlv6?O3I>o;iz}R5Jlf}KG^>cI!CMtU_|D?J;cfhU z*40;J4BW~!N1H2T!G9b~yiGUdK66lW$o-w;eY5=gWs-Q5?;JZZRtHiZuJk(-KOfS{yb7}@Nkrgx!g$<`Ic_sbzo=PZ3_}Lkvo;% z2lxA$w+7n+Cd(rfWyR74ht6|z505?(kA7Kbcap6}-+AS-*OoJDVAUXeH%y%(BjiVK zI=)@}IN+~(@wj=FS^HvWveM{~o6puC7ojpSi&Ei$XVsXs;LBG=&TUEpITJTN1x_FS zENr{54f`TuVuyo~UQ3VY`txQZqCa?UVwVc{mTs#Er^ZCTcRX7DoD+1n-P_yKb+R_6 zUf+DhLgHMd;ie=92uKz?;+3eF12+zQozJhm;*!67SzyZ5(B8iL)7{`NKiij4xmMBr z&XW5JE!TyGAMHg=TRl2?WSunW~nStp<^OzJTP663ho$XC?5J zObM>Mx*F#&05P`ODr$t+oo@0>m#;k7d3jLZ?D(VkQC2kVd2N~A@d8m)LKV9H%L>^K zretI(B?YaRpS9l-bSkh&RT`$>4T8@WA+DlMKj>y8^M~bVKX8K4a0f)2`FXzQ5oMOK*aX z2D$>2+b>Pr?$*}tAbnC z)Ah{Wy5emeuI4L#V_U3s#vKX`nSr&YtEKG!;7Z32v)^A<>^&ZeqRT(LVseys8qVl; zom034HXYsh+LbF#PHhn;(+Uas%B|18o5aP&{%q0bwRn}wHG0MJEhvy1H>OQA;_7qo z9g^|+wxEqy=oan%u~pxCN)>!)dD==zQPdF*ScDp?h|bm39cN&bPY<1ZhjcZYkNh4# ziKI8KCRoW3vC4=(3}B)5=k1o2&Kd5}Pe0wP&A$H8nvcS+x1z`?7$n<+In^ZPC`R|{?gxvHM0_%wLKkf%vbd4?5*$pJMw%ve24{Moi9orv5z)>EDQ{w4(NY7Sk1ga zhwFT3I?4RJm!VI0v?=j(Lm~&uy%RL{=CfPojUew?TdBAr1gk#dXk0Wr-`8hoAcG0| zeB-rI5NUO;+~?I1LxxjqOy*R>y^+vSfyv; z?#U=F5p&UD|iq;^qxOOY*?I=Lkg7r1E`DKWAsezST*b)6;QLFHtXc^u*qyqT`O*-#bekR2JAnlg#;V) zPJHy?mW1EddNc4*7>upyMu*PNHc${Ec8FBEH-qHs?QhOAReQ@T7WGeP6H&Rjc?T=4 z+L~ea|5oSU$opQz0dw;`n_7{(_8epn0}qIa?1$fj#lD$dlDPEupN%nlpM(yGR+L|V zu75a=t)KrcPv$w2ov<-Ycd$%PvUYgm%E8J>%ymsLPI~atf`9Z%NC_Amtfr;a>(>Of zTa32vEqys@)Grc=iXv_gK7&08_z-OFf5`GR=qybeGi&+k`T@D{3LPx>4RNvaZoo|mTn4XHa{c>-Mby5s$4XWWX%Jo{E{ z*8)H=FrITv2qyC2Gp|$3&ieeu;tX`dPsM`gDRS%Tb9lr3k#f9tapCxsi70?vfO6NJWwOnpDnoX0#FrJniR5A5( zw({?JvFXN%ggKsY)e5>xB`Ha{{>NR^G6}qvFbl-TxX>n$y;P&oQ+Z85>^ZjuR zvZiXfM@$7?51AE*S(3Z&NG2E{$VNe`T@H(X(0 z8nVFIxN)Wqz*=tp9tnEF2ge`ixA(zJ0pDiyc=0SpuM_ zn0lsQsA=RiT^GoYXa6AR{{~+F4{X?7>be_tp9t9)c=0_Dgc*)qlKFkr@Ya|7M_Xl1 z1ZTm`AG{Xdu;NIZGCkJzrLN^#^@4$B{v@5Py0g#_Jlv-EN<(8=5w4JN^(ff6P~&Neo!{`@6ke+?z9DZ{14{dJE*Ds?H6U+1!*dxBA_5rq$#~y zq99dJid0d0?;WDDX(}KhARwSfM|vj&L`0ep0qIR@fCNHKfIxB{e9!&ed*?lK=lpT! z&RsKmCR>uV*0(+N^L(CpDRfFoN_JOLnbUgLyO7qJiGCD~;= zqCH8(f?2gQiq+l&HsYuG3~YW?apynGzlqS|P+IJ%Y<|JhAZ#RVmOW53QF=Az7R|-p z+XV;S2ia+T8g`Y z8Q(*uN|Ap@e0ES{X>l~Js(q&aX)jF6VZ?o9JY~dQZ}(ejlC0}!Z%cF##j*tFJ8=zWkTRt0~4I<&@u`)&U#!}n^6rZS>S_)!F;`sGm7Ox>;5 zCh{NV$@?mV^O;c&$W=Ml`%sL>b|MY557*IF+Vw_-yUn>3nU)dJsH%whrQ~NFtWN zlHxiNz8Ea&7Q1{9+GJp0>;F>gwf)4!Mf8#2r%#tSw&gRe=~(y$gg_lJ(9v9TaPeVAEvN#jTv^*LYuxkq)6bI)B+7$Z_|wY$)m_fW zLN5=GCv*{~-?n}K`%tF8hgpvHL_juF0bIC{Au%48-*5kGYjmjBwX!Y{_v3RIqh28y z9nh@m&`JLDm^A8pQY6h}NcrC|HT!Q(O@rS`xRHqn*SXOUaDu{j_X1+IRpz+=-dyzT z1_ZnwbKbo9T!^i2sNAU{AhY?2@9J)%JrTbqfv^xi8y-$?bum3zabPdNtStNFWMTsi zvqFGNw8H5_lZ&%z4nvPc7d`yW2r+d}-aeY~ln4Dry{$L@{&u?r%$Ul4$sOFkuy6?F zML>J?t$qCJU4iSzOyN(S-6H;a9w=34w&yy|#RVzq^O18qeQI}JCGh>xyK}5W5 zrNa=e&t7zE#93FM5C2Fk{d0O2=H-(9aE|FxL~Qo&f`O0z1DYC1lBAlTz1dgTYuBz# zL|qsvHlMyZv4L`J!jelqvxz&1JI+P(8)5IzVZ;8d8?fCRC^S7z$H*q(W&8`}eUvJ_ zvTX7=yDPhHCy1B?P$_}yb3$P!*fbH(pFfYS=k$hsjyZ6+o?X#x@XgII22Rh87;a09Fh3;U zaD91-(=BOUjuXV4zI?9cmyzccyc&0FicISvFtuyD{9@F1jj6ktj#Y2B7QYb73uI&_ zC{LBr(_hi%gB<+on-@QMNl-oDhV~LTt{*bUR3ESaOhikkBt>1jzqdE4B3iL=+ ztB)^EI|RbYy-j0HRdAOT6fC#M?TNjj)fp@0T(oSG56B^x6{gBg5w!$kw43izs79{G zi-=_TRdN$@kAEN0!CGj{%T_dip__wegoK1n)OfBJJ>>GIvKUja2T|Ut3uk!OOULEcK(GVuihADlYHLC`-!=ez`ejXyk<#ewcGV2`#3C zeQJpt;|t8s2*a3^^2}b9;^T9nXepVX{f79?cD&=43i_4-^M@Zp+jlzWd*O$m(z2=P z+c0v>ZY!s|GpVYWbR0_a-ixG%G?F%Pt522pimV9@Q5T$tYCCgcn#RNGkYr+KapOUw z3McoO?}=%1UDWkI&mD~hu=wg``vm6CpHACsy(iDA-he^NNPMJyA zzE!m*-FsmDQx>_fxUHs7t#PRd2ze=Fgw&`s6G@$>?qQ8H+vAWEC9&c0!iGN6p_sf1 z90R+eVTMgfH+N|B_wVw)YZh&=@qQZl?CdrtDpAUxY?mrWa}43|P=pisZ!bP&TZZ$h z5#jDVzpYUoBOTMCkacWmXvp#)n))YA*+X`$uBI$v9#^>}zcy=BwIju4Sn|ZTvvwBaqQlK2uBo; zspgXG<{-N2Wt;$n`i~qvn(5WK&dPVcFu<2fz$8oE!oq?~WRJ*EPb>xeHwL3WtbV$= zHK$eZQK`#?gsTbU9StK{ujRLY+NEc)n=3iGColsIlJ4SHi{CXhjmy__@0`oK&!-wMu)f$w$0fTv@QLfrOG(Fm z<4hf2UlzN8l4;JPL}u;vGX<5)H%_xUq7 zX5FmH(R+S{*{k4+jOUaAp#SXpa`Wf5^~|c;aFfgpZ(!6?j>BBYe?-uxNv^=lhJ6b*4`l;Kd{+u))AK%JOMLzv`si3u)wtRh&i1~%y77;Uf=f@3XAYgbq zEgd!8zgsB)G3SlJRSWVa7gvhuM6R0tfCEH5X1`Jb1Iftg9g51-(k}=7n^?XZdT+c%pwW9;xj8EsGlt!o!ENq+ zZ*yne#lP)q?rzJ~j=e%6jPh`W+`2Xy)`s#X$h+cfTa)fbs! zyJGeI2EW{)<5~;ct9_P`P<|arIS|Icp;GtZg3Rb3Kr=6K-F|&(7PnJ^A97bwp79;; zZi@&9>8O({epcy1%5?RorckXEao6b`&%V3?hw*evMJ>!B{U7`j8vRQKpZfuu{~|b8 z{YtFz_~6xnC(SRjm(N{!{+ms;Gr!EL<2>riT;vgMW^~Zr;t%>XQ0l59*!EdXL^@yL zmgNv^>Zj?SS!fS4pZmfpWlNN;+^Ov>d8(W&W&039AF`e}37bK_t|MT)warq@zvgFK zQ?^3t0?C!;PrWPg_^A&<_L5w*0pxxg@gO;$X8~$vW)m;R<5c5?19xWHsGIY?i{B&Q zA4*X*nfQBq`qWgVDtB`KCYw}QLuJ9htMIj%$h|j54j(yx;z-ef!$Y@$T?~xp-MwAI zN=mKi-jSsSC*4m^n$JtvOW2QJe^ur_oZ5+u{~j$G8$Ti1MmFHta(bu6n1;fvwhiFM zP*NRdL~8s=mCY8CFAqE#xhF5bwz)Xwz58C9TRGAdv-NrKADXExt^-FxUI@+|@(XLZ z^c`h%7qO=2p4;S zetpEdqWyh>YpE@dsX_64MWqw^h&!v`>#`u?mxcQJ`uzUFjzlqigpbdjtc2xkX7|0C zS}ST`hgTYIfFk4WwUD0X?J!bm*Xg%?=A!bX)3}swe`I| z3W~#3Gm{kUy$C^jU8#!e*iXT@fiI6xtKMF{vSX?{%I|x&*seU{%s(^{>9#QGPLer^ zL?g5>C|am&o|o`mQUDGRm!01cwz0@JshgXD1-&@4`r?rbkTuK2zM6mguBB;uyDYn> zX+p}$XM5_-cpZn9WpA1ik|O9_AAkUmWri0K3UG^m{Sq&9ylx;E|MBAu{Q-PIgHze# z?zcrF6%!54W-qQPJ$l&xWhlILfF$AHdRHRD+E+oILy7V$Mge?+$D&N#1#D7AxCZg~^V8vqsuen>r_? zypq3u{o2~9!sX`KS?K)>RM2GK>U!kTL{G8n^hlM{INBXM)XTfMD5bs{t17E~dSAUtVxQf2@@RAIO@&0E5WfuTSqVs3MV>CmN_K$YhQ4*PE`qJw(R=vU#hHB(iB zPi-F<@=H7!Al04;rpBT#c~0ZTGbFaq1f09@d1Apx&x3&?69rfN{9*+gMR6CqX|Eq^ zI+uG&8RcnXQls8|h-yEUL4p5&Kosnugwwn^9>Dz1ZO|J)`iizi(oknrO*J5?+U zno$n3ltBMG+I8Nf<>zRB!*Yi^2(ZP+Q2tT^0x!cQ9Sw@~@=u8v7j)PAZC~f&;u7lm z9*V`JRoV{iPK6-vXrG{E@Y~sPO|A1iWxU;%UT)~PP0J!ht%qFV)a)x(G7-NsBTVco zzw-(_J6wMupnO-?H`x_ETC?bJ;d}`wazU~bBrb!=ByYlKo2&7k5pP@Lsz+j3UGHUO zwgQXWEFi`cIW1lFL8jb}o*5mmDlZ_A=hvC!?KCFqx#nYG@f$pa-~*>}%r_S$D7))_ zrdy(K-@XOLeR}W0;7;X|T;aB39|T(t-yE!YN~YF&J#qf_LB>6f?~xlA)1an2y-KX3 zQ)WBj5qFdW<)AUGWr=|U+SCm)W3vN}{Y55w%S$8k^YgQ6tf*QmQlT+u0rVc3=(mbb z9C83>;gQSZ`>AycwZ508OE<3%`AMnvG+~7|E!;h!)=?@T;LuHJ*WLG+ihKa>F*7qC z2{!M4Z5tfMaMd_Nh5AphRp)ySq(_*L5+-GDzA?}UvqRpd5OLQlcmGh(m7UXG_UfS- zoYx{9$I64|-bHxHh|>i0uy*tCTXz}1ilNJP%K*vhiG!0ztnuX=;!KA*XDd*%ftloM#H#Wh2Ph&jaANLPs)md#2(*&h_Ik4q7P~F z8CTE0wd{aG`XS-S^T-MpN?op)RpxBl4^j&y@YAPHp7wgN{kz~^f(v-O-qiI3$90Fs zx*-LWx5W9#%#L@D>KCOXFvzj2l%gWSGv}R*6v)RD$r-ylg9&ZIs|@kK(q0H z_)G9BH(VMd@Cut%dVmLK*Po|A!rr-Ezc5g#yB=TA;MM+i7L2^`9Pl-cGUAQUcC&vz zO_Qb$THl5&jiphqajz6m?UpZf(h~?hWsJ)c1^~qn-;g$uvwAAOFUL~! ztd%;20^f(j$--~?jp>w_ZBh!s~I4k11+s_m6azD zrM>fNZ-Qur>jg5o2&tJiSnNoO+C!d?{IQv@pQN6Zd#WY!tXR+hXV^pol5q%{WgT~dCif8Q3#Ne_SDW(RX791SVLO4Ofys@zjC+M zlCa;dfEsBiovJlyp`%hD2qkt?v_D5TEWFl>V0k@JXZ}Vugu`^JRkXS6dOxk#lw}{f z**cp_Mk=Jp>4eF%-(W7C+6x71T&ejkrwy{rwoMBt` zBl`TAr=rlUJ?erS0DCd1l%`_B51Wc%%`v|VgDCPw+f&5D3ct?7?p6vxTNe}jmpQQ_ zVDgxv^_Cb4l0cuaOMCnaZ+_6vn{2*=-wLxCEBp+4O^9v46O)1p3jWd9_&m+bkw%67 zy`F?_K*@tcRtG>9(LvfMu*7G;$yK%6i!3Ei4xPv(w&PPn^`wuG`(nWO48Q-<}NxE^SE?ja^FoK?s_y z^uNLS*1>k&oN7xFrX8MTz9shDa?g^KL z$A7-_A2g?^n$8gj=Kaaw?-e$yN$<8fFXbR=`wc4VmrVGf=8oUg{4)dpC0zqKS50xf{WS=#*Y@MB0M(jUmJuBVpMoplZDo3w5T+_hNwn!@$4*et}7$>JMj`v+<`*tdAsl z;)fs{A#Sz%uk#W50^@=+%!iL0Vc7OUGz;{pC6JbfvZ=U%lA7frQG8 zrlWFclTE@BcS^ALjG0oy_}-Cs1@~*zw^h+QLDN`*7UJ&bRaE!1>bqQs#E*3vRY`2+ z4UGpL4bN%R@2n&YHhTrEtinO!a@-6^<8?~9>HoyRtrk>g}{Ej=4s zK;IXig|*0}$}Q~ZwzP!r+RQ?;T0*u~qVdNY6dfM5*Jp`KmF#lY9s(%#?KhJjU_Xub z{qyVuf8>E9ubQvB%oo38X*WarI1vh$s1$$KrXLevSm>19t@zv$RpvNeALu=Xy_KJOQ+RwFjM}iHo(*77lt|?#2#zL7yp6=fIA4RTxKS#x8zN876s^qRy zG@y~F7##lk#V_QaKR+VR1IDV*y@lab0ww*vKzOTOV&zYYZEC)KC6X%R3CIRx8TYQ; zRbH;)TE~_KRuK-_V$$HGlJi*YG!~0-l_Y!c__X5%#~YB;wZ8mQ%2a`Q-9OR6NMWjH z-&DxDGuil@lGVco=ObvJ?QW0qs^!tPrRC+h_!I>}qsk33>5`5IlqG=J&FkTNe4p?1 zQ4-;t%&t-7-Ra4gD$S)NDd&kx>2n&%913199M?(RPY=`iv0T;jn0T(FdRm0&nIA6@ zm*?{778TcTiCtE3U_+Zul(kSxQMD(#zntsU6EsVb$Oj^m2OVcR;iYdQ^b3vY8Kck2 zT6{DuavuB`6lvEt7!~m@@ni9d=uxz3(SV$cY**4X&+VROgvVBEw`744ODkdF&qq$V zIl17*H?JRDNnZ&McMdz@hKYNH-3Gv?*YfXA58F<{i`?Av)6Lg2FF5n|%^N0bR_g_G zJyupO+90Z88xo4#x7ZtDi5vx`XvC11qe-uk4n z`wNx}u>rxzTvmGzk8&UYaSps5uT&UuwD5S_$bcBit!E(VT>;}WbcXiVm6h)4o3}$d zlde^0;PfuEbiUv0%^xU=Xl?5*(d#^~9L-K@O(wws70c4N#8q&2q*}X)r9m*li~-Z4 zXK=dNG_XF`D4V#-jg*mO$tH$c#)}lzTkuY(I7Sy#gxVu%O;;GwI^7Fsh>8!dhI;{5+fk z)o~Wx3srWThMzBfkRfBXks_DT>Je=gc0*Zvd42YOsYpK;O;k_DV1mQX_ZL+WzH%~h zDlpK~r-(T}{|((!de>p~5f-9nD;+1c+aAQ@`ZGHyN(`y8fp<@X+$<8eADUZ>%SU+3 z;g#rEg)XHZ*jC8b&)+I4yk97Mm}BKzxRP#v)K?W_fa`Ox_XY0l}u$aV7{ z!s!eoN13pRJGHOx?xqC=Y%l79gBsD=-S|m!$uObF*l}Fmi_(iPEX)rwA!5yt3Hb(@ zw-cNs>>qNZHL@Z4g2^_n^wweYqNL5wRfrg@6FIj&Kbf^d z5%EhCRaH8<>Q53+#;I038F~v}hI`%ZJMXzXhHCJcl;>LPMjI8Jmyq?^Hv4=1Q&+ka83$m%gAMd6a&hIwS(+hXKqSNK zDcTo0pB}S6kEg8RGCTAo3#x|RugK)}<;j_k4#o>5W!~K{ci`{9V6;BnujZA-9&L4$ zwjcG8Obkw?!|D{y2Va_dbI+pkl1w^Zf8NOB$#fpjUW^=y!}U+~$7C!1S_NQOycSpj zBZuumi;!8Z&m3L`C+X0>IMq{XxxQTKzspGR+NyD<{xDOr+GwSEF1=0a`>O+5&t5nG zizIm*5D<{jMiM-u2DwoP2`oS@#j`etzZT*}eh_Wt&?AcGjr`vVmg_qZrU?oa0%MhR0JD{zWSa-Kpa<0!BV zB%Q~8=$Y2|72L);rJyzqDg#2wEy0c{@S``Y3$V>j3i?*FP7MM8N?R$5eV=2s&w_|x z3NVhER@>YwsHInXm%E}{rsq#$4{w;xDVavUeD{U|xj`pGeFce#bm^!4S`s*MKBE%V zhJS(&jMtTTN4}*c{5-MR^?k_ldznA9PEflF`~WxBKC0=sjaIw5|M_KJVfVY(`wk`f z&6{8K6DJl&P}r$7tdh@)&6fA)HD90Kd5@m%{N%LOOf$^0hzK>4aPF^Ai!*^gA&DOscKheAX za|n5a;oRJ;?nj5jp0&WHZ{Ns7FvPsIh;JTP=0$%MX(*HrB;reX=*p}*ch=K0Z`ZSz z__yf!>o4}MbY;9=*&ocWvfc#`olw2t#>W%wG&hcfoC%#dM#pG|vbz&Tbc8fHb?%oL z(h{>Q>#_1xzQ%o^NIyFeRcTa3xR?$G*qN(_h}?%~`2P4(8xL~T2Fxx-vPqFiOHj(h zi?1kh2BRsAp8W%_*7DBgqPWxGO3!Qu!UIQ4vFpn_E&MPXyM1I6H`|`LPT4!^N~C9G z#KiJe`4<}Fm87chMCst)rM?}NF-@WISB?6`%-cD$pa9|4tJ^-nTaf`=SQjen`uR-j-lOeT)5hupv(*=LdYVctRVV6vlf&Ps^56`> z?~PN9(`E{Kf8FtS>*Vdw$X#%cC#o_~M!);=^tKmT&h<2bE#x=TsfO6Z^~+ERA!HKg z*~62+7sLp+Z{_LcpR}I%_3Ib3%c!uO$i2I?6CBFm50WlPPeI}^1TxQ_ewgKfM`OM` zGJ(|}t@G@tuTZ6}6XWNu6La-CN5a$4C_dP+lL>jasd!mjYk9P%>5H&%BVpo{_O_cP zhU)+6t#@-6BWIkjiCc5~MBc+fkE!1aV5yQ8R~vqG8usUyqHhFgk#*&-jd(8yZEUT+ zy+aqC<6D+hMHqGL&!Y!)^`fuR-g`_FcNftxMU$x+rtd!AJFq-^3j7HG=mG9gxrYjw zfSkO<#mZ`J8>yz3Kq5?x6x_NO>V3ts-1|zGrf^Ey7^ISbYda<;2J*zj0!A7O#IdvW z#(=>6bf2la9PXMV?>RY@`MH6Ond1zkjq4!}>oNbk!hb;cd}Njz58ZFYCp`s4+V$Nc9Q5~Q5q`L`+rSzYeIZWDIdyHez1@2V<(^z#w}st~%j@3- zyFQeDG264RH?%qZVhH-`YT9Hc!i!`M}lwCp1s~L$WHSn=;Gt87Zx2rKqhVA zclPw@4?;$ITmm}PzoH5n%Hm8Y1Mc6dOEjaRn3-cVH8l}4fBrCWaF{31@;WVUBvL<6 zy$33VeAjW z@4JZlPloe}@`V?uV^_Lfb1O9LFYq#cZLagG5P5rhsN@RG8&~raO{^K}xjF-ZPAZH= zrVXZrcx5GjId6iio7+yF>us(c+-8inHsUlh^W@CC<)mU{66#MaCUR^ocX_5Oi@BIBX!!~$KN zKPidYF@1eb>6-Jnd|$q9#rk%}2T)X{D_9o`viEao{L)(h#5{M(ZIQAR$)^;zE;$e z{Ui5=v8r<(k`v%GFzoqC3&_^m#i@#AYk77aGkJTA@bmu=#Qs|=pX3>XlarHCC{$qn zyXOx3mVlnhczuXFzy3ydrb8eN9>|LUp3BB#SVa~9SSR*b-jJUpDxkY-xs)RCnj>o8ot7n_-A`70Y7;-kHJ2LMvejq7*3 zmZ*-TR6uOs8|_ceZ&v?0tR*ZWIbL{MBlrFVia!ZeMey4=JF{d`@3YDk@y-jFb3bO`87(j{*!5ZewAMr)o*CJ&XiS+pmL zS-%`OM{`}_FUD;_g|fTNq2y;+f9o8V-^R+lh_}r%m>=H(8Ou3Q*R0PjTVARCRT(n& zf@{^HT_X{bFx>yijsE&<;8L-|RG3*BGcpNyqCm)3kmS zo&OumqH*|JTYMvHmOv1RG(X13sLv5(u3x0(F*#ON)>JLc zoKqYe%UEpK`%4f07EQmgZNnm-bl+Ml#KJw{dK)OzDr_F>eNNyQAQf(}J&A)5LfOy< zyd9ZL=Jhu_lXIDnP6gf9MzN=Ru|nA+Tjq{-pCg zwYmesDdZF-?`srOqM zt#NuLN9iwYF2)rFkPjZEy|6AaDRM*6Cxc!rw*-R8od6k zV_X_?zrRguDWE4PHAy}nSL+VD#$?;tb~s=YQt6B4X{nao36L)C_LATE!c64DG$uo(=e*AYmYB7uBnHC2@7`- zb7cc!{%d9*X6BKKCl5l_SeW#INA-+91;39krtDfmPp0r3kX0924j-zEA?W1lgdC=` zUS!|!%Db*zny0ZJoFc8$<(T9rDdefYyI*0}KBF=>XN2)_m75_V?0-f34Z+|>Y1ePB^$LHTbM94wm#L z2btLp2%KupCw^AeN}!fnQJ;4%FV|?MN|TqWroEQNKLYzKY7|ZW>LAH_Rc}yJH&lf6 z@3y`O!w4AMx*Lh}T8J) z?{>-v1b%|V%e+TGvTLO%`fLo2e{LaG+Vy$N|DxD(3SBmEEP8Pmtcj*8YG|L;m!?G> zsbji%lJ3;T>5X5pwDmV=Uo}4=><5CLrARpUUlSEo`4oq<7Q4_I(AAuIW=K_hP9rnt{%{qw_Mj`rBbqa`fH)NB41 zZY}QAS|E0}oDWF}NRh45n3f&MBMwV65R_I}9|=Z9F>p7q;g?JlH7ihF3sN@s(+n%} zvjamW>B9EaDqIdw&GYxe{|9sq|H)|lcYcch7ew7YiALA>e)y2)8TCg;N5{DCGm$$H zCwY3*zKS}HO8e_i2Ftk54Fb%3|MJpHM?^qiHtW?hb8yxd7SAi7`+#hL=3+erLucz5LD;InvhZ^c`(=a&2J!(nu8uo1yWFzi*`S`fYfjam{s$ z);EBaR8N+4Yzn1^xG*H5&r7@f*^dw#lvUUd=;iB0PESRZR>K1z?%a_eLLD$ec0#1G zFHd*D?yJXz=^v5g>1d^s0qr9r{<~YNP|ULbglE)$xzkhRywC96f4sA?;GheyC!m}6 z8(cZuKoP5;!7o#ZWLeL}t`s@v(dyM+Ev2>rOtMswMcdnay#nB6gAj+<*%RD%UvEu@ zaZX7gGN6cMEdNoN)e9{a_;nN#TCc%lGVVj*>=>ZM24tnB@q;Cw1IfgmB!Icxi@gG` zn6H^4OLs;PIQ1RS(a~BBL3?(;bF?8I@%-S?@f!EAa4*6*fS)~ce{v#syGKUIU=H9X z!P{E=ox9!^gaD9`^KiNC?q*9AyM13?+^tWK8I39(fC3ovK_>S5sZ?1{I~A4i01XwD zR*)BpHJ|xjf=BK!P-p@J7%8_cvmX$yq`+Or$H!s*&q>%l3_e1CUcye?d)b&sS|t*R z>+3N6hGVs!5jL|0{W8eP;xH$-L-)K%HO$2PFv+_PJK(N>kw*zDnU{Gy-eUx$@f0(P&Y$Fay zpR^_arQr%#f76Db6otB=z1pfO@7?;lt}~s_@AsZIK zak$)XbMfA_+bSw5XunOE)AhFZ=Brc9F!seU6p6Sr0-IP8L`x-|T+Cs*GaLA!b=bYI zRKq1tTOv;JhRn0x zh0GS_=ih&~pJ(R2`%cKHzojLt`Glb1%IYdaJq7I#74vX$J;8RRfEnoAoel*7%4Fh_ zQMpYC>ULLw zxji*C6)&b&<2K7C<;2ttA_Z&`!%S6{GorA0c3?lYF`QM5ef6p%_d#~Xd!W37%@Vi8 z6b?^)nQl;Fr)#Ux{F242A*hqWfOrV>WefbgyxA)&D~CE(H#XkA2+KH~Dq^OnrG zhSydmo8S-vU$q{b(7RC=ws$t|D-|^vEx2e;ocJJWehdBza|1#4EjSS>A(sbdM;?p% z`1`}k%2sG`UW`!OpzE=#@C4cMAe#XgbKER{zQ6Vg##?G}+#h&7+q}yR1QDmtolAsp z;kF-u8}`wddB!zsa9qOYVXpxcjZZrr@%ZuM8#lgg;s)Gj+h-RR66+RA^Ot`8x^El# zw_}6WPPp|MvEKzo2@=mP^MKptr2{+p%@57uh8y?qGx_Iq+surVSx*D#ZC5MA-^b?* zNC-Fzur4N*4v#?ag9!Q!&&)`O=;>;xT4CT+3;^p1&jfPJ4(1akU)Z?%tDm5h(`rv>9zSDLOOH=%u zJN@g|8~X`bTxJDGPnZoQkeO3*aVcl|0?s+1WWV zg3|tcx5BRPN!5zi(s1T|KC8~8*_jyxLd@!?%~0v=>U7J#lv`M+vF*+J7D87RbjAeb zwYQ8Sfa6zY*>QLOsG62p(W5$iL4Lr_cN(dT+-uiL61N2*MHojvWqE8;ACaRH!!=Ur zhzcZof!39Y0yz*9Na3P6t_}H23Fq+u_}8@xfcH1QVl6Bzj35|;*#YZ%&(U{sauPIv z^}sSL>D7lhzhHx=Dj`h^jb*q%c#Q7Ptu@WT-i&*(a5mJcR!B=D&@jgUR*CxX*sF7I z&k8YV!>jqG0zj(Ec@_^C_o=tT^d!M5$%6*0_Ck$pr=MU1m{WCy6b|M)*A(TyI)$7oz7SKYROm+uwuK2o(#8v^9ajC zCK1vU1N{sP-ox9%c}Wm5y22Tn{1IdVWvv5>(}b6LGWY91U~_E#fht)S-C;bT4LRX4}7@t)8qGzS^?mjAZaD7z-$xE1DnpCSG|?j-$Mb zs;WpJvuHpf zE9Kqi)VHltWIdmZojJw-AsnzQQQBSbw*1TABTnrb%9{gbS9S9)7o1>Zw3Z@-=YTeH zDD|~S8o_{v zrqN@t>3~j~Njq036U46z_!F>6!ALFRn}W3UYVvtlcjKJC$^z!(l z8>gE=(6P&T3HE8S|7qAp)LOzjJX4RDuFh0TfHSx`Sn?DME7*Nro}L8;B~zm{?h5kq z;jLq1fxw*&IsvTBBG^?1#eYB!4yk-o4BP=T!0*?`n;@qhM=FMKSQa7}u?xPb!Q}_CFx$2O`0LU@CY+@2~qI zBO)Tg^%gsLScM+u{j*2UH&OV4Ub4r5?!Oa96Dxe+$Tx|07u|NhT3^5awg8_}=K zSCEQ^>x~EONFLR9e2+>q&{q$b9fEu;To$6?XD$l;jb_6C{a1_?V_7 z7{k#-^>}ePt_;an%_q;EwJG@{0cwt>wY*38akYyKAd!kY2EO&;!_ATmJ=y$N_+4k_ zOE7}i?`^#Rn-kJ2b(meOu*Rf_yi6|;f{Z#OgW*zK4~Lu3ym86|To5s3b>fxzm9p90 zkY9gDw}vmAemnW2)9Bz4MzlAfJ~`RQ#%3O{!(Ay2qK!fT3$(GxX?}8?o}T4UUDp0x z5+4{II&mf)Nuj5wKNZ2pXCRHdcEZGB;$*V|AvNUdEVI%LR{3yxbt-d@4GSPAJfxhKhOk8hI@@`fLg(JnHdzB#{x7( z%4T$hb0skWCC0fAO>5cZ+9x|-K#j@5noIi zVie>9=f1#A;`00664;)K1}LqDm9pdmzR`E_^&_1yE+Z^kOY^IQ@SHaN$nYG*>J$eD zG{2p$zzs}OJGr@)>axHlxORnh~{3p24M$uRHvh0uD?j$in^E^wseXcW!sk`k#_K=lT+4k>TF$5&a#bVojmuWRD$$4X4 z+;LC?3nr{oaHuh<<|YDI3&PXWKO^5?7lLzm7)^WG?U?pya*)^x%oW%wEbR%&tJ7Mb zc&~6lJK|PIAU+sa=G9CsG_6n#^K$G;p;p;Pn@&!qoo!x)V)HOf6Gm!a`Td(k;DDQ< zz0k^ppafyFdsAjl7epEq{dO}TQ3zZH(lQL#C5ES_O@gi6I(sEaSHMac@ z0? zD$YIVgN)P997Qzn06~#>M9!nwD~OGi?x$u-N?dC-u3~)BgKcfOnkmCAS|92Hv4x=U zet709M_y2L-+36@RvyqSz@@{li?EPP#`Wi-?aA>G6VYljOfcTH&CLh759KQN>K~75 z*&~f6_hG8g&@{aN%Lo$(vo?T2p$cCkMScNkA_sXMua-iEh_voSPh9XsgNSbF_b$ie z1Ytu_i(fQxIyF{fp_eH)N8M+KYaG{|uLdoQl^tbv9H|f9pAebc?i-)`ilH5<8UhRo zSIJiq5!wNQn#q6R!4?e0cAOud$Yj5q{h(765P8?#hRVTpJ-HLw?R5&t?PCr@ffeQF zq)Ev)*u9!5ioFGD*rzR8Ox^mC=kt6iu&{`kS>215#lol94ob^&RiM~`?%^0FMYaHLU-vB z0ul(NR-X6Tw@=30{k)Im#S#cuLK?4@6Ble&?(`R?rfZ#$7K1KvuXry1B^1B3`e1(# zJv+g>bh0Y*Et{0{N$tj4hvhsS%t~(gBB?Z7&;<_v2p~3bbyuNb&d(%;gzzMZJ>Xqu z2Jp#j`1}&eyTb0F^Zg>H+><<@NuWvEu9v?KL!;lhF|>CYeQdQaj@X-x$zTVIMj5tm z4Esv+-cAV>tg8uszf3l?BJt8y+wC7?WhY(P4(<= zA@&4yn+z$nx`i`=lyA*YAq|l57B75ne#^3ge>8{OcsBLx1t%vY#Z&safZb-h8Gwu9 zThDD02CM<ORId&#f1re)) z0JmbX|8rHDdwdg;lP8_(iF>=X{Xb}X52&iLbxT;qGAm|K5zK%nP>KQ)jsZlDA|)t@ zBvAp$k~2rK1QVb{$qF72NlKI~qLKt8OFZP9b2y>rw(fne-}w9W_(%6>$Guf0VV}L% z`quZ&Z_c^a#>RUgjg-G1pR&G-ZhUd3Kh>7@0GpL}OSy!E7Tsqu&~%58;pV||i!peV zDt=&I_MZUQUqIVzT9+apt|b)`nrz(k(Wf)xTAtuwYy2~EOf#$SOxl_1LmmyGX0sX@ z*#asz1?W@(zNqd+a+5sZov5vj8A{wKJP(KbpLCTKHyh+ajOvbRxYQ(nc*1H zrF*uvyeJq?_`=-_)5?Mk>)yG*shYFM-c>O=-}JN_)|t!n=SjL{na-tA4%S8_r@6kl zCqCUJ{tX{&tO@INAK|x{8XfHMXY^Hes%dJtgh=~6URP(%VZ7wKFV7*!*26XZ$uLW= z+xZiRe_UquJFVSzRnU{QUZLP;IP%NYJ}??{I`3_&l?Qlw3od}_iF)c)~ta5zUwy% zz5nBIfvtR!P;x^vrR@bG;28Cyy=vS&JRcgC62I)&s|mhETDvGT3r41*t0EexXt>5< z@VWZp;Kl7{Hg@^Q5O;9;(8KN6+&?t`><^MquGKvG`rW%|xd-ZFC5-!Sa&TyzBc|%K z-u-{8G$msX!+ziTp6NX|@h$xMKRkHn^Y35&pLFM~U>{R$ckiiQvtdJ;?Qi|0Et&5u z9GIH!*KfqvK}Ojcr4l==Nr2BKb4UKa8&v-9+E$cDE{J-G`{ zvt`dwE^(*3fmKu?jcgIqF9&#e3*l_UcF4U2=jRp{Jh+%9o?NCdChtCG+%;3S3un6Q zjvm8y?dFk@WPOEzy1tY~5hED3J?tky9k>4-UT^ioISa-nx&`2s&$Kp_ zHyWr(uwN{10c^XPa9!iDpJ7FLJT@r%*L}+BU9%qNnjx1iEX<#Hr>*MnUaO@M&ioBt zkNLFX`tliBvDf0i)LC)`6dZNDd-pnhV*mdA*y*b@%SAlul4C)N%hy$C=N^av!*Rh# za-!#8M1YVs4s+;GKI}IyB^4SP+Hmg5R>JmpcHTl}YUmpdrx+^|FbSzBc=%W=+b*Wo z5X-)3Qc@k&kJUHsvu0b#HzCr#W8`PjctINe@GQy%y#2ks-nYe}esH?VLE^@zyTwM@ zc?X|u-+V?%s$GWA|cIKp@HZBBRU*&oLI4K6E=TbDo)NZenUX zyZoZo^x!dr40!$KCwm8LQVRnq<<8h+Fc69tR}o#s?33+o^!drk!u{I;H-O!IT#QJ% zWpwlM%1++8#CZkmpSV$sc_kZ#Nm1ThJ-oJS_CN&g7oUTm#vwWlPq%jnsHE@_fb-Bn zBX;7tFDjk(-5e_l7Dd)a6GA-!Yu`-5^)u1cREuw$KbWz2vSvpXO}5@KR3zkiXL?4!&mbJuVzR#yV!&KJ~h|H1c*?^X6F}rr1<;N>66R`h6v%{ectF zzr*x}xzj0gbyjXpboncsyHJjG8OrEwZlIKxI*z^#gx5{>qJM3|wf%<=51@OjTEJmk zG4?J$?=!dXD&n=K`mp!`9a}E^AmPY~eGzc+4z*C)n)W^%A=&W_M^qcCal6U;L>r_8 z@f>>IE9Ut)>T>vw?c22?>^>qy8QFFQSnjB7VLiOR<>VKhPN#0(ZHQ32YBlwpCMu-; z@i4}rIb08`@YRYF&pI0>dzYHEF3+~NocgtTej%CHD z@o}p$2Y&tfk&FIE14g|`dA;oq7oYUCcNT7NkWv>nDS128kc_v*@4Jig# z#1On_c)dko&2Q%yuC#uc+t$DT`3By)@!JfzWKZ&$wdelpPyADXkL_B%j(^oZn>6p7 zT*ayM&*6wmii~l>>Yjhx3ojbry*%^J3q5e5&o5kWO$uv3$o~3xleY8pHNV5x1|U^I z>;_Gp91Z!WdwAFxMDJFT4b&}_M6FQQKiu5h;4Sz9)f-|Z|MerciR00KK-w6#7_Tj=_S zn*+Ngc5sY5xt?VM64E3mCkJj!pCUaF{1?x+Qy9~#kd8Ty-lzeY+;vR|4#CGeMeepi z1L=6gyKj6_p&@`6o2sKYr~(9PayP;Xsp1H(43QeSkwyPPDA}n-s`!Ztp~KVzonJIJ z{;iEm_fBqX)kN z$oFh0OK=?w*2Gl9IzcEw?yay#V$R;HCQ9G@d>?iQM@L56udd-ekf}Uq69l^tPmQZ)o@EWq~zt1)Csv20veClz1Nr;sB8-~787T%#Lx9E zb`@_iGaJP~!yfepUV|y~fCK7}o<)2DbKX-HtO^kcfO9Q};PXjS1k?Q~#mLI7sf6AN zh8-0jriZY7j|Qg~U_=pJvnhNh$Py6r6ll55>833uty=$w&I-{& zXHK&+a}61puMmgf18{`4j3>h+EkZh%vBH;NT=@vqi*TjXzOVgz^%fjw_-+r>%y7x3 zFzC!9&EohA;a{mw2#sIhO}w}76vhj36sUSg1v8LGTOeP+8od1(6*G#}bSMHnDQl`) zu~-1M0ZjZ-G8F>4AC#C?>(}oVx%;B>?g8{&5ZJK5QJH%n@nnAb1S1j7D4dW~l)=ys zwB|^}Dk`nP;$~oiU0~#aA?Wfe;r61J`(cu~ME-60ZRxO=#)w1Z+#dmz!h>6C0nvl& zwWb{u|`FY91{L?a+K|*4L2LhUL)TKa+nW2UX$^1JS zeK5a6(t14=u18u?jk~_r zB+YKT12rGo)9B)89&ZFRxg*-Sk~^laY`y{!*lojZ2k5BC$7t8gFyyeK;Eka#%~TOb zn8j`UN^cR!;uc*u&qdfmOs*eg` zu%wW0wPUh~V=1v(V3biudW5)|V%%`7fwB*|^a3v)?Yg1?Hc1O}UJenfL`sMMVsj<( zD^_7uia|BxK@sC$P_Kqr^hqz3*rS@6DOR#_GRWG`JU!9lCT0IZG|=gj6;Sa|E_g&u zL(WI4eY*E1eyw&f+%sp-p2dvdVQV0mVNm9A@o-C<;!i(qt*n4&hMtldD$hP$zD_S0 z=c6v*Dwx;HV>SLq)Fv?F7$6~HWwc8FOTZHUd91ce;Vx%{$KEE^p<*4T=*f^B-?j7I0OxsaH z34NQkEJtg!zrR>;3M&Uf|9YHmGJK;lLN@kl%$HUz7zkXT;?Z$(2X+Ab*bXRjRt5yW z%oNb~G2N02kq-uIV*{T1q#S&?!26KzDU26~Fz>(e^8DF|e&A9jgjuiM65F?L_a{+YV87WazMq)3 zi-%%gym$f5YWeczH3^!*j}cpyca5R&L2OLYuYgO-8A%LF9O?Ne3O-?%!Adl`J~lJs z!lc_@xUfoK43!dUcexL*7o86apw2KAc9UWxECenjeEnmAOvWtj>tN5|pgNuv51Eyb zQ5&PIkBWluWrX-JLIju<&v!^iH!5}G^+b?DvEPAt z8VapUSATy4w&G(VWMySxc8P?1jjLV5yBfJJ9H5~O`VBf=(Pw0^RSjxtK%(uvdyk(#KM;Y;33H>@;{>go8*k1&+I*0_e8q|t z%a?29xky22U9Dvfv)r6Ja;qQvYx#y;^CvZ=9oR>5bo(Thi;L-6;6A8RD5e-Y*1PpeZ>)a47?`jQsc*3#VaO4n3a1p~A zzVZiXf?vXE)o+>z9@tNy9-+NCQ7iD6E=)aK+qPYU2@jb7r*5@bUiMeIRb=s0V}aty zO15oXYT?_>Lx~<~=@tyVz-x7OZO$5*zusOjTXTKmUqfpyC@){i?;6D8y3F zSGn%HcG~;rExu)YU%Xhd`d~&=|M5IhSZHsUv;M4w{D|r7NNzZ3MsLcN*XTl9iWY-rB2ykT|`z3lMM8qIG3Lb&5jsjNY(d5R(eCXQedi{qnC&>y-s~ai>wl@0ysLP*5INe?x18^MD5oo=4Mu zb;ywxUOIggjj0mIwNUa7>y={8b>_`|6DpUQZPsQCs5(#NV2YM-avr+$>Rz*XVP(=_ zCcHF12QnsyKJ?m?W3=fV93dx4sz2eYznmt^^kMVRE%8;=vZuzzUVJ8}M4r!0EE`Kp zfjMwQ*7MZByIK|X;r{4wW+91rCi&M&u6w5`S3`I-xNLsu^oCfLZ&8%oyI;nM=GA$< zu2SCn@7at_F^QoM1)8=k)>9oR|qF*toPzHL|Mqo)13^-Mq9d8R=> z+;6}pCN85V2-c{gr016#D2zevQt9vA4b1NQZ@G0%HwtBsxt6f`sDvnpw)jq*oBIVE zsh`oUFsjNND_|!kH8vsB(8y>|UwSeSUSvkAb+Ryr@Hq@Y2$4#shMB5y9n{ z_^6Ox974i$Y;`Fv70SCbSdla|oKB03PyJEBr#z42(J?r=QCHIIZWw_#;q|=j za6>QvR`Js<+&9$Kb>GdvPGXQhnGMht^nQkL-&RP?YUqbr0}rL|i@eU9dZfV{*H*ar z6yqsrypYe z{fgY~YtQSwA}%;wD3=(Oq}G%OBx)S7PpQ{(Y_Y76!Y_#CPWUAf>rTozuj|WNp|@ZD zBFB(e^A!~4WVoLa)+q-^Ylq)rE(IT@Vbh?C;Z%;gG~YU#n_lzhj74{BTv7?{&%U1; zK*XNEZhY2AW*)s;$VgAPewP`0xevFBOa31m^of8x$!=d|KCMBYaQdB(e!ov=a9f8= znKhJDiU+90V3>D&m3_9o#bGcWle?Ggvv~-S*dDCpzK#lfvdLJPJ_nm5w|&=35_8;Z zaU{!(s}QCnMw(S;kv#-m$55;iNct}`mw)B$%#tiHOsI?-0U+Ylxt zhEuAaKEEaYyc0{&ec*?nTGHQ!bpq77>$g3=N13LjrVay+BgqKl9m5QrsJq@7WRxJ@ z@pwN_#VkzAG^=U_$I+ZwtS@n_10)b;7cc40b)D}bJr^1(Tu$PPCTR_B9Y3_4#$`)< zwCYTRgYuiI*wp-C346?D7KRvye~^iK(qF&5hRDoI&OM>gdqmL-SGit)RS3X_4`P*> zxfF=A$$K~B;}oUmyYO{M_@fa4Ja%mlLb}E)()TGvHU7Sqw=ApN(EX_PfnJ?Aul9@@G2&U1a zt{&cv5TKKTqIyGZfubfaOODGtRe&|Qn8-&MqZMa_(Q9kkG;|}aSn2EcB3f@_9cbzoA%+Nf6)YyTT%ZL)x! zl-=ch@ltj6Namo!NZ)VU)DpL&sv73s&aB*B^Uf@<(^Eo4Mp$I8v(Kp(iZRB3#OPPV z(=Qk$=A_zMY}7(k@Eg^jEJ`6c0+`#1Z}?fKU` za{pp;{onr5)W5F2q~;Iq|M>m?_H+2pd+eOX{i6BRY0F#4nQh&L&hsX(Ip?2s^vBJO^x3A)XMe@=9jxBKyB?a>;4Uzy9@_*(HB*{O1?{b=ympSpWM$ zaby2_%Ks-nnAJ;(HgG86kG}4&%jss)WE7uThdC8N1YylDy6rL*V@y>$&Yvh$vY$ zF=G%=0?W%5{DDwm+WH+qW2fD9>?&nBc02UFfiE)x8Lrz8|43 zl|0!1LAC1hC+anlfkcP^EB&Z*&g)1iH!59pn8UzBc4^K(Z+!)P$wghf>*zN>?nR*& z@G$RT0&+C$b1Y*8(w9zBXB zK8{Z`Zn~Hee_vW3=gHdwPhz1Z9odDQH-ITQ`73r!O(YuxNzXQ0T8%sP1nQ@?*fi-b z7&gCN`Km{}?vot5s6)4oSm*QOg4ttV!s%VSMDLA7;;R%JpVnxbx7gB3DND-D)3*Ue zghic>Vx&ZVq+^Gzk30Z&-=4ixGDJT@{3>|hS~1VTc$4T*n?^u`MZ=~QClst-Yt|D$ z0qZpHkbHzM*}E$dAZ+PMuD?=@!tHwk377`FAQd-UwBIX4(%vH6_}9iZ+`cn?ypa7= z-)qZq@m?eC#YXTOl;PC+nPJtf6$gb6?0|(|mP6C3{Sil)UB?s9<&>#Pxm`K31uq8^ zvP_Z5#0TS&%K<*nw`-tybz5ZMRfizg(MkI$srk2HgamT`{B&kkpv*GP+>6`vNWs+? zgPFl5X**gLd&`35wiE;2^Z|tVMD@$@IB#`p4?qINjv)G6A*mGzQwzleHOd6@jASQQ zF-qz7Q)S{w_0-a;FH1-v!P4f-ZNGc&Zkqhg1>^$4sRbAjfe?c}fvBT#^J(yV6NsPq zKUlK#U$@p$YlsM9E)0%l4kqM|YFkyx5)}Z5fV_p107EUdwB52VO`5_Nn|rT|u!{!u zo(88i4IR+iYelJPl)E`DVZ?F&CbA_nx>a>MARKXbE8(0i^ zQ&gd{SAXu8-&`#MQ!|r8%PwE|d9A88qwOKPsewT|yEG^hcFD2R{sd@n=XU4=aY(EQ zkVEKH%`jW@L;56jx}d$F0vcV=M$!Al!1Z9#SpcA1@6J42gDEpF+%1$xw+M=9QL%)dC1A@Kx1Oea})AcOel4Qjq>dHbS z593{An0;06D(5GjBbAdDuDtaN|OGQt_c{;i6>(rUt{f z*dv+;Tw_$Fb5g+);tDYjqIvUWE4l6VtSjJd4Ug&v=*tFm7>)o#I|;+PzwS|MEb1M} zv#pLS;LAJ%*@PW0$CDEm$*1E#vA}?T#@I_x7qVO!H{MSq@MOAQ}KJ1NpWah zOp#@2Eaof`qKD)s07`vmlL!IaYP|M*iRxwXZeQRabmnD;NhR9Mi2$Sbe_V4G9ABzL z&YA0Yc8BNwQa!XgM78IS4zXx7h|U3S2L~aU!$i6X#gHMU(@_L-i4p)jmWxh-=xmRd zi`~W!Bvh0prFuHX0e*cW0$Q7xwA_5So`8K0j64=Y$kRISmRcQlwGi+VX7(dm+1L3` z=zLjTEkZ)OmdBh<ksYo6V9ugD%iyR1`w)<>#T8_h8 zE{nB)DA8Ex#0s^O0UtXm5l89t@NFJXDYdfz(P|DTR*Y_Tdhz5;a@8p(p z>hp7N?Pm`1eaH* zU8KiJCGs!|WYzipKw>=YdkKEW-{-YADw11AJ8@UDGztZgi^IW*trqB1(L7I~7~S0x zg0G?1<^-dR9=7V-h3<_}tJ$Zp+KCE6G&hK-_#G=rfW$&h?Q6*~SjY>r7(A?AUnS(; z2Bhc2oEd`gt`^;#FsP??67fqkGx%nIx9N^sbGeFz)S`<*IVoi)oBkU6Jkw-GL1#9E ziJMAP)poXS!GboR`t@l8&qt3gl4B6dsXgod8rNvFC_OrwH7ZS;y-2M>nphA%ziBEA zmodit`sxXz=U|tfyRSt96j)cl2{8!0G!q{XP}&XLf7|bx${Pw%IJR2<)rDFmJokrgKg7un(heE){1YSp?vMxQEehH$)@$mT{fB{T= z0O;nI*`El;MWGsRRLGtzx8#!JPeo+cuH6LsxE&uz^!Wd@B-j&C7ABRIlOv^2-D*Vo z&%D9oaFCN&1^5A~!2+(Ug-)Qc;??af>kB}B_^Sy8MFmC~u#e74CdS8mXc>UjaQ@N^ zc%Cs?EHe{2sL}vhktdzM(uN7JK^BaXGkGO~3~~C88H2KJLqE-lA*bo|gExB&sNq*C-yyrSFaVhJriIm638pu|L78 z#R=WmgL}i&$jE2SU(C`plR9^L#QAZJoe01iM)A`f&xAaYK66A@irF^SF~0X3&O z4h`D8yhG@IPqg}_-ATRnX1Vh{6)rQ!9V)F`P~(CGxUlhLoA{~%f^5k_&y&kyj1mc% z=!UVH+18wRGq(CC#wemJ;!cg-$XUE46qxNd*`d#w` zxOZR?^(J>-@6;QjNyG{R5DaTDhz`Z3)8fIQzCJPJCd6_Znlc=hGL2*_C_B4BG-%+o z0F}?;iPtx3Zu`1JuyU-h3PgRb>yL8V+@;77RzmO_X{}@d31>cz zV{J?(ORw2853B>JSuxd!tFQrd z0f<*=F#o{2A9`jbg;u!|^$@?p0;p)Isyc?;<&azsz5K8rnY2*p)%6T}{`(k*+E=tF zinM=JK@H#KKRi40eU}-w6FsKaIDdnt>~p{#@WW4upv=q!mv&naZHNTo<7jCk2n5pj z6Gh3f&uruv!~+z1nDk1M&;tY##3U)H(J*#C7vPmn(1j>t;8#B*?DjoN|69inl!gls z1`?f$+Bf0n?UDj@2Ucbt+AVGnRQ~GEbKKoTenx3lsGdT|aAF0!EDnp_`uZ63vnvf- zl|lfSJr9vU1`A8FnLfJW$_hp9|pAjJ@|k9rM0`zADk@QN>ZdWgZkb&HFW6Fe8tt3YtnJW*y5x@R{2 z9K555QR+h)QbSjF^4Cq`C<_9_kYk8FW}?;!RU!%YC4Ai{Z0Sn{I{-_Gc$sZV`*(Nhmp*g{HpdBTJNOWh85U8k$#yqK@Txjko*6~=! zF6v~{gONZKxm2hEuY@0H??9INmLW}a8jG#7zDfkvq6TPXHs}Rc?jjg0v@05R?T?}i z@Cwv*O^vOnP40u4zwF^F+>?e$wH4(mShZY(POZeimo)0_+-R~ zpu9Kou8?)(Cp$rYD$*t@Igu&x0aLwmr$xc@a0~I+Hk(7iXd(jc$MFiHwICoo0ZTC_ z2j!2UVJ8Y54tU4pio`p9-6Tf`EPWjDE;TjPeWx4Y+_?Txa^ffI0m&>N1-pqL4B56Yx_ovo8WSJqn@< zJm)alCMnSBAl}2#Rf5T_1F5D+^(Wdqf^b3>5>>Foa}9S7EqSoZidAV`n5loyi~E+< z%1F_TjVlU-t3;%5dz6{vaCxJoUmPpeDcyUW53(T*X>A0^%%gSV&rv!ikOh*H{zTC! zqH@==g-sgi?(AZ91tC$ZmWDvb!}%lez7FVoiB%sr*L?i$<$JH)#fj7kj=cF?vZC;v zSD+0+$GpK|2{-S4SSMe>CDxauT#3^mW9}(V-|Bz%$?q{C<=VY%XO9qzZ z|MT4?@769SyxRElTJS%Aa7snk$*w-JhiNr>WNOjc=m1Z%OJ#HTaJX##edqmukqMrT zh()N)!Z&HCh(a!CDJTF7D~5JGP&zE{jdpSZL$&LJSqL(kin@t4zPaI-sH;vX!QLYv z&HhmBux~|~${>O=rSiNxc>l=?5{E)0dhh)R;4M#4`-wHY8NE>7L@O-kGuUq}}fPqboB9HX&Q-g1m zQ0ubiIhW~+O7&xNYez?BhVx9Ux`u^v$J-}N3hJ~OgSzQ?Z*%M_*9#3%1|FVyb%b{6 zj8C;dep?QG)QdZNVT<-+YWoI_Lh8qrVRv5DMW~nW{$REo8zK|$ol&j4#&lx^Y8mLb%a~M6xio2brF>*q~W-+dEcx0H=bi3u6 zgo=}6n$$=_#;7$v_42|OJ%8OWCJleo#Ak#IbpA z^idaj&seDH$|*AmxRusXYICPje4pB+?$qr*s%`_Z*q~oHbD% zph%}j#UC`vP@OQ>|MYF)wy2t;mHM+B*QUwxFB2=D3B25srW;~2)g#K9^iZW~WxK}y zp-@!y$GQ1O6t_vO9XY-Aef?Rg&b=+Mxj9|5^c?3N?&Q3z21Gwk_V|%J`{K^@T<5wl zGbhVPQObu}$9no6wd{NS;p7hXQD$B`Wum`<7Bn>z=y=4XOEEjme|?&eLI}N*mN*bw z>H1sPAlgmKv zc!zSB1d(P{B~NfVZLVu`XLCkF&A;zZT7f!qoB%$=Py3)9!8%)C@IuQBZ7ed_2#A}d zyT^okh|U3|4l$7qnaJf@C>F%8VM;%tq7f+`DP5W&u&w|=_9j910Rc$Q7b|K(z3W=# zfZ9Pwp9Jel5O-MMskjUwa>y%<%Lt*^zxBCH%q-egtDk395*d|VODIqTDnRvLH}u$@ zA3i$C{f(Fmm?yjunczZx%ArGE;Xnvy znS{26n%|5t0#EUoTP4BQ0aqY|SC}kJK&GI{s1m7D`rg`g4vGCrFHquT`(R4jGVk%X*|Y)m`9%OnN$u@&SLJq;MzrT?Nf zs(d^$S2Fr{9jG7Q%$%?SfG#o4ef&p3!gb~}G(OPRnU5R{&l{C!;xXlWsD)hhLoU&+}@Sl#&fjHu&T zlEO%-eKTuG4sGxMbfj2LCy(r!?oSlV?ruKlMJu7TruCb42eMa_Pv=ckT@?Ceo=qSt z<E|SDww1UM6UzL2!j4sg&+$S@S?g-wSp)?r5d(ooC%-A8c;yn?fXWl zByCtAF`HmDDMKF%ndr3gP;UdYZmbp~%mR=o>cCwQ!h|HKT-KO7iGYGro2WGae8e4q zA-)Yi6ouZ&BR@ekaKNI%mpW0ik%F^u#uFWfVS>{>Y#Rz>m2cgGI&9Y_C$&x-4+VME z1)WDLbO(af!g`+l%*M_~1znPdwQm82Xey03*9oj?EBY@#B@T^y`RMMa;c6KK)*>PI zBh5F1E0pE$7V}gx#Jv!bW+>Y<7VIe~It1g&gjl`^iDZn>X)r{8mX+j0wJtdB@$9_q zDc(a6NN7AAahtgyNeIMfKM}$0rUSk~oM;b;VE~hbY*cKuITy{K9Jl>jT|%3Hl=E`TLI!x6QCdQH{byC+$ z`Hy%^Xb0D2wr`m56B+xyn|ZrLzwa0G^K62Y_*$s~6TF+zsPPcRVXBq(7g;>~x+A@} zB8)VBkvUkcUe^@W8l7~2IoH1H`w+upW;eSES$xZ(6=r4hF{Z#NX?rne)l!EhC%aje z>{>0JzWeXaPog*htVf<;r9vK=%vR-@AeaO+mCzL?z7doH5Y-j32{0pgf^e9%k%lKx z5ka6h0Z1IYVh$oQ6o7=_*^DyB{{GjJ=fO7Dq880RkMRMxh#GktyG}%04)%vlSdc4Y*n>T$E_Z z$uYnTZ8-CXf^zK0?Z{O~EK$;Qj3Pg^oW!E9PoMX<+r7A>CT?A+Qht-{j~@X(BpjTLEn)_+9j z!gisQ^9nB##kSOS4CyDE_$nPfN{RF43z@YjJ=LNnMom4bOnzjkJA>17FEueS>9$bg zKLJt+BBoD`tJ3?chlvGhmdCIeSB821FS1{0_mNUWQ*!a4;4#O(R^p&F2g4>CN5@5^ z|K|FFoy*saeT=#nxq8ccmm0vUO+sEfPdwf56@lb?px9sChV@Rg$@qB1q+OUbisghd z?m>l&Qev4fXqxA&n^r7RMu-^DjOgWlDmR{-3v=6sao~pb4lW>wZ7pjFd=T<7XRc;J zKr|-o`+#OBre{ZT;h{H%)-$y66Z&x*=tPhr@}{aTUfDVfmN(mtU@l-A($w@@nf2vrlkD;DgzW%|NK#EZ;qXlP3E~9_ z40P2XIR**G6gv*yZinm+>kVYp`KpT!IRV>r+SsSi=p{5udxqy`-@6=Keb(>{f2v{O z+%<{Ahk8vZ&8({dV}?l>j)DPHx`Vks|$l`0>w`(t||d+;=K zh&^Z9noHUl&dpyY=z&qC)jAo5*pYocSx}p|?+w*-2S#ZO54zldETqZRwh#QgaJ(p_v6Hfvt~Kl4IsNXG2%-_f^tsx4Gcv5<)IXFby8V}lYDej;W(7BuZj@I<)fN}@9bf>+A2#V zolY@kv7$>77|_NvMrxw72kUoV3!gs@`*|8c$gujP#XzN}uD+g<6Bjn9Mi%jGw{vQo z`e5O`-s=5E$t{%y3*Cz+qyhWksp|g4Hc`_(?+NjbAjg&Vy6ap&G6)~ng$)h+{&524 zUY%X`=V#T?gJ=ezh|mCxNXz57faUD?TFQEDI` zq%qcS7Edx=-<0B+$>0_E^fKkq#V-@D{H5RhsFqyU^-)X7(^qM?iJydVG>-t4Y-&3* z`B?HvMl#nU^UU!p>63XKBI*WK9%lnc490T_9>>Av&a_5{$nm2I*4?2(v}qZSlI$G) zu2ri1sc((m9sI;T=9RfxE%TNQ>4pGK%fBFP*(yF)6b8O?zg7t{VD2iHZ}OB`f3v)TEPd z-W9%C7!=1>#`%7=n)`uWkArSS&hgh`fnXn zGVV0x+j|mQcT+b#duS@85L}&f;9Zc@yx2SXQ#t?hLm$3J-zkkI$9&5O zY_071;}0J`30x&xKmXag-hBz% zJdyJSIpbb-`n1X{SM$vJet?xaa(N$f5Sl#oN$~90UfXF8-{C_Sw~GrTUQ9f2ar+K8 zgRbQLTc6+Yx)C#S*zJ^+Yk4>+FHf-yTSsP+O6CYXt;tG2xY`RxP7$?Ow%C)k9x`trKOU zi5f2oJ$9x4%U%Nz&;VRi?2B!F-h-Lo4Bk2&pw>jlKD$)yn~`@>a~>Nl{FDflKgJac zj0DCRktj;DJRZ;W>$WBFbhn9VUG1 z2TLX|ZMrai6Lq|FZSP-KtKw$$^XjklSENqA-_I$;Y7}kbN81nUJ>1U zaW)|>F5Sa-sg&@-@j^dsdVL|g))t|l-kgc9FY4tcY7~i*JMR{!<9F}3ou{v~RTvJj zQOq7QVfjV(C^D)X>lrU-ZF$lyTz@vI+1u$#8X3c3&vws{K+50-UsM-QzdEK%=5!-k zQb5KQxVi~t7^?Ec;np#jX$e<<4%8vyn}qd`@D4y7vJBfqxSf-Zs7I~N-dntcYY-qk0csTDAG9~ zb!H$LVCut|yWH9V-iz|P3;dUg{k#*Z4N?rObt}C5HpmLjz~RK_Zx@WanniTPga-hg zG2emx{M1vYPD#T?t%#F`4Yx_6b0!oJqW#dlF^^)O0Te`-cCjk#0M5aU%G9~%Xy@G_ zZV5P85u!LzWAG&6uhDxBcXgm7OB*!+d2DCMj9)9V&Oz+eVOp$n`UjERS}2TkKF zhoFWXIyntE1<5f;WHG(M+o(10lh4a#lZU^x7&Jajt&~{h#Tib zrY{qgN}{m`YW@O}wv%9p;V~ho-8caPeG^I<7tH=JGX20}_z1*qc~eoSUI;#WG#+7M z<{bn{4*s0j+Yhxb-lQpEkO{%$^?@Ejf&j)>2W8IBzwc=Tr2P=kLBu8@5K8l-Fuk$R zXx3@kYEo-J-4fPeoK!jns!a+`EN)mgi8i(CTxrq}*k66QK@ey}=MVxgflfH2*Iu;w z9-HIUAP@d6=}8KM`SsyZ(YMv*v?ybW9R8%nLAO!oMY903b@_wO)jV8Ajh@rLYHEM# z%1L0K&15=eY0s8Xvd6=Bu}j%DUE?(n;G&&eNOGpkRWwu+jBNmYQ8BM!*4NC~u}Gcg zs90N+&qe>qri##0z!ZlrOWms=xy-i7E2Bqs2dkDLWtDSwI2jIJ<94xGs+}F}V!LRqGCbEP>qY6#mQ&qLvq7^-Rg3aXy40D; zGo0BT6q##ZoJX2`xG0q6TxYwXVQ!8gdL&^i!pt~0szF~dnwVwI~&^ov)0KNGuF|k)!6uaNtu!Gu_5!eP1CLJN!A>_q5j8fRc?FJ zeBXGL2el`sg#ArL?ld%8&KNY@a~}^U8J#{Evx`Nk$hIu*Rb>^^Qr4~fso0D>O<2T; z)V^-Rh^l0T1oGkrv9=|@Ua}QnuMt5f5H?bR)@e(W9(WfC!QT?f`uee1*9TuGkrlXDYJTJa{<~!#g`f!k&eHYG<#YNiYT}N7T*4`40!8Ai95ia7*r=p(^zWDSBf43t$!XfqAf2*70PWk4&cqiz&=xjlcC!y|KsZKR^(G`rxvIQP|JKMR&~^|iBkP7CYE zZm_a+eM=nYi9e?`H+uix(X`~hQPBTaQu{N)TYKK@-+H_J8MSM~weCyzVE00^d2HFP zoNY^{*8hie|9`pNV(7ftlD;Qmm%R@dOsU*Ia|!>x?5~^h+ILNX@VaxfS^TqL&#Etb zRz>hVkKpt45mE3tAn$!Z!AH!q-yldY+wDkcmB^cDHUES1N{qzkELpQyR=G&&Rqq32 z-44I^g$_Bl))ChbDeH(-a?@3Fdo#D~#)l)%XnN?{i;&@-3DGqJ&)2Kh1k&*IBxvVR*YL19r?~lBhuh+oP?PwcKHT$qxts9wFT&3|`PN0gn$Yw9 zJBhPO#G_M8A=ZMOk9PWyY?0H$12GE`x6UpTx|jA)=bgwf_aD`Zi~=t=ojYoJLFa>_ z+;lk4&$tBr8$qcQeAo+ zT{4XijrSO}iYQRG3M@6~5YYW<{{7Sg<-pji*l7V~EeRNq7!o??$F4j*^cw)HP7|H{e_dGJ3iu&|0A&TBmL+abDW?}KyO+|HE}@cn=KEVQ2+cGTwUf13R$T637E&s$D`vQV;bkTMx@u{|FAm@1 za<4m=SiNtQ%ffrB?H_j#zW14Iu1oJ9qTZFSb|TF$KOE(9$6$)OV zhy+f*&UyEUn_uVa+L)g?CUF8qhV@hT4_|num3w{EkK1Oe`j3>515Z+-m0$K9@ff1! zzDmqkl%;O{V=I6Aqn}|G?=~M4=XC2RIyfx*RH8$2WxaLeWB=Xt3du*xCZ)L6#FR|F z7g!9*yD>s+S9?vme{;!e`QkMnViXhD$jY7(uax~`My`Yvaw*mBAE~?aM$Q1QuM;1f zIdC;BW-fCZT@J~KZx6R|P?r=L%F^q-SC)>sMGO=jf0e~xKYwlE z*dLtMwofom)}dmaT!jkWHRiWA+oh;-@oq5^_CC=5)+A%_HSxllDbe4f5+rr>+Bxrg zmn}CJu9pAfz30y5XSO~%v+-{onYH#$T_5W_^*Qh)b!v&0qdD7viF%IiNqxt;$5X3o zoYaM^U7L%7>XLh$_;Llqx7YtHkn4IT>lg8DZD5pq!+Lw)mkNrz3+1boxL5l}89e!} zkbh+=|G-D@1M_U&Ecx+h@G zwuaMd-|qM#{5UhFOX7vm*~Njqw{1@r@O72n>2Aw3P1t^z>cds|CS@XSbF=JdqSIw% zp}0%lU#prq{B?`@;%=FqV_%C8Ic-LaIBip<%!wO2?sl=r86j8FE%z$OR{Z^*x%$f1 zaSQI}HAlC;t9c*y=>34in$K&Lszdi}u=PF=rStWF5cTDOP_NP7V@(W+!B~=IFxDo9 z5M?(rmdx1KOxZ@3tfj@Z4k=UgyL&^V8WyzfkD-Si#op5Mh}n-Ao4 z?Qd84P`CPN;Q^H}d5nxgItl$U-@B`rkH@s|*Ydx$k2n89+bl{9&GmS?98y0aaeTj# zhz^BKYuc~0!!H^}-rNzd3->NTnM-)dHhIHtk(58X9rAdnZ#g9Vblz~JIsbMH5Z0Ls zmqxziTlkMD9C7bT_5RoT?@Q~0fqrTB8BP_Gj+H0IC#>d%S^i~xRF-Ou&Lwv5E?ELR zRrB+7YHp(cU)*(kr+S;T`f%ZMI)$Xh_Z^M0)?l

{h!xnbmb;bC$p4WIB&=zQEwm z7kFsrMw%N+Rw$HB+CT|`#Pr10+4Y0R|Kp8T7-|SqM?90tQ!@#089FE-!7Rv3%=4b< z@YVIn-ETw`i&mdFAMkxJ%3i3`qP9vU_q7=|?bCzF&N5u!!EJ-L77Vj0#I9(|Jz%d} zMgp#Q;8G=#ArFlibh$X^t$g2+(LF?~me!N75Kh4foO!)<_UcF#{RJ-h?Ik7Gms8i5 zomFP_Mr}J#GS(ckK4eP@a!M!)X*@r2=$o~tZsB4I!Kt=Nr=#&QCic{Xq5&ZqoiN(@ zv-xH4PL4d;!TvnRYsH6u8cX%;tj_LgzT#;x!A*l!lHv@EbPqN80Q=B1E6_8}%Xnf$ z&rRA#nkdLW*VbWCRR21c%i0}u8-cS%P?N=J-cGIjZJ9Wv$@tdUgGe5VK$M&c5u&Ac zKW9);ya)JH$7i zuvFtIbkBln5LY?P;ujX7$)jIq+pf5KD4F|5y$>0@<@>ZXd~nIQAwiE8!DD-P-e+0K+l|oOYFjH=|CJqm0C69JqLYOWO*|K z*>&2NZh^5i2s4NQ|qmDw0 zcCQIB44nAH$rh%_hwz&X_|Zv2N3HYsb2SDE?58SngctEcr+YmCaz)ftSEB~Wa_wy` zRV3G}JF4esb9LeSeOkAhM*(PgnCy@6FV0Iz7MGYeWFN7;`}xP+m4waJ!cqCJ{_Ibq zM23^QexB@DxJ3PgZoRB&$b{BVa|)%TlS@uT2Xf=jjm#L0Nmgr$wtZ}AYpbF$irnk# zpVwPS#xXDie>h002;}yTBKLEz|6C^ZlTmp=0w8(a(VZU>{5pML2`%4kCI2{uHMo6f zEc;$7r-s%YioO#z6=S4a*q*k)XJ_`CZ>mLZa9N*N_r`GVjjMjPzd5>stu-b^az#|Nk#aIK-H*gLUb1Tq2G1J<02m)p-F4khSq z3u#dw9z-3P)5sC-!_7a#kMYKJ!Loqi%Ca`Tn{16g5!<6i6yr^N6|7wlNEr)w$#hj2EtW(+1;X>S=47yaP4F%)o@wfvXiAs`WKi zdT zSW^MYLNu*PlW+u;nkQp=1Sj{Fo+@`X7|kA8;_V#0kBjwviqi=6*w_roIJ3Q)a_)Y< zPdjQbisPuEPOI5ZJ#4ygM>vGj3&Cy4bPIu zQuVpuT6e)j?5D*4{0I_$q+k=5`#MPs4W1l0MJX$VXAgeqhAOUvP@b9n6~~nwB4UdU zWiMRH)^n|m<}v%L@1xABx+_T~BZ=AGv+K3isDVQzqsgD#m?ANFJ?h1kjJ`N8;l7&< zS~-Q3th)l+f598<2ME-6#p`cE*zZ9qAy20K^R}gEXy*TX%ar7+K+8LE%8?i}BuLmK z7_naj*)flX{_h*fkd0S;c#Yla^`F3p1OT(NE1jf6LHxz6N9V3eL?@$Q-wRBYelofbs0iH6} zQSR%Q3Hlz| zqo6wDZjat1ZSftfumik?G*&?OIN8iv-jD22AKkYr2j~rUI<89N z+D@b0Er=?95%Hu0Rb9uZ_br|^H7L}g%t!u{$lpjlBQpIy8(Zj;c;#q`p{>PlKLs$^ zW#2&IS>_+n65Y-puWCg`TkqB8Gtae!HeZ@f!^#@fBr4&BSMqt~^2MEK#_^m3fri*> zf?gVRC|s`gqT(+b!|~)Sb)%e*F2+{38{bC{hJ3G{ND(vU;`8A8-^Fu}=x}LIq=lgQ z%|$6O`l0o?+hnXUit7TEX~xDGR5ULY<$6Hry+U~_H=5yeBsOj&aS5}+-Cxyp_TbI% zb$P>t;EqCnu(v#4&dV+0|hmPgjY5~Xf@Mgm-K)5arg7-D!GbG zj5&&KE;m>(X}_kFHIF$re9J5pjH2|pG?O4)|C?Nl-y=QAj!|#9!MxAg8s8s3)e`Ov zrWTxWYTcp@xy~i4)0e04a8vl@W9jA^pq&)BK|6Wkoex}<&h8`iDvuofCnd?tFu$*y zVpWLeSuq)s4|fqh7sC#e)k4q_u@U+E6h?0w#@Mca9U85A$oslo3vTskR2W65@qGbF zrBTjIvnn=tt}XB?r|);FfB-+vXXrJf?Xwfw*R7>=i{{u`k-HBSOyl*82xQL3wH{am50-MoPl+3hbch<|iS=JQ-jNicH5h zYHdP4dB2GdGcSE2wl%-QKWr6v=&-U)_ zlcg&Of$Q26%AF|eMvOt_ddFOr_RLG z%R6M^CM*{VE)&0|r@ma0%ZC=TIF~x1%b(qq^01!>^?dbZq~CCZh*R5z_@^#Z)-oph z?o|SJm$qnSe{kPZZSo+0=<0G;1hcwI)PAa}!oE8R1B0PM_&H}tkK-;n7H~UlR%-~w z_j*g+F1hO_8AAa-=O?8)N#lC`V`UzHT%Nm6cHzVhA4&E9T-JOzQtUWIT|<;`A@g*H(7$>&7+FQK10VzJXL%fw6bBItMDQ? zYi7;+;9NqbZOQ6L<)>fmK>1HkPEwSjBG5n{ZvLqJN%OkSp0)1+O2JLqHaoJAn-QvsKLgC*e$*CtxEH1)kLH+14{UR9?0|dXsPg$1=a%!Mwl-%Ggae7|(k4DH9`UrS( zQI%-_xg~tR0QXaXUc>Ga*Cl0)RpA<6!ajn4z0SLs@U7DDx{IU3B;IbSK=#(s#WX2i za~~olQ)P7u*^`wkA#-P^ZiXW9gP~*CT+>G!(F@sER>fCeh~jNr;B5<+rBglG)JvPE zSZp6tq6ACNjLd0H2S}NDYg^iF)uzt=0 z=8)@~ulQ5P5y-%*_v5&s;u^5|a{C0%M0A@i307b+b-7QyTD3>nx}<`Kx$TL1olm_} zGeB(m_yzbUrf-@MqK{r((dz4t2!&bhhLwgn3(vMpKRsGb2n2bg5*j20(hrOwHf28~ zlm~~&_=CjqBp!4RU&m^?g#@=-4xEiy6M@+xS#;1ziIt1tQT^1JqaQv5& zpjt&b-k~EvW0-%$JF~(N$uquzJ>9!;W+aYH0FR~Vr7WWCD)#q8lh%1930^MgUp|6H zSr)H%$PLE(K5YW+wAdxmDcr2p+(6O{8EwK6w(}K$Q?3AaJSOs7t|U4a6UC^)?E7xt{kpax6%B^V(6MfsqR~lxHaW`*@Fa2fHEBIcbW2bLp~{wnE>TE51+-an zADyp`&(Tf5GFtcFQg>AoK)-uVSYc+BukjHpw`~L4{G*(F%Y|&LzP;}zqtzc=oR$_| z$etiFu!VKqpX$TszL9)5$a+zAVu%WkKr(12ynfhmC!ruQ9x{q7m5xK6d0oT1^5~I{ zMvhgdU_jnb8tuA`9WyPw8pXy1dpH=FW(qt_eg%gFub>1`>zxvTHHi;lcwE9Sm+Q{5 zb$l8HR_c!iCY1S}Bg~p9dhN+3S!~)ZF9PFPQ&_bAzjpy1+=psaCX+c4h5fHLK4iG9 z&KED`s~yTA(MP$!yoBJL{h0QOL{aN$$8zpCFM*V#!!`CnueZ)zE!pvcrueqCcNo6i z;-LK`PuNtjr%enmQ-ahgYQ49PZ!X^F>GqT|7qE+IR5X*M^6B%kq0aSuN=3r2St(Fn z9wL9vW|~`wf9*BOg{A&46`fFOyN9Hmhlvv9on~jhvfo8jAN6D@2+OM>%92`MRy$l} z(-{3!k8i)$F)2W*FBpSBf-{`8ka?l zIcyLulA<7BF^PvZR zWgy){x^GIyetK*Gn>J>-*CSJDd!UPApz?$T$2N(n5!D^^6t`>mUaf>?kC$0`>k?(P z;$<-Z#lI6Y3s2S)bor@dRgrv?PuPms;4G(glY`^YjUDnbrYP7Od6W0a^iPs{ebqy4 zHwcwE^x?H3_A+#C0ZSx(lcnes+Pud*h!2BL{YM^uS6q71VAYrs)|nefL)$#si28AGhVZg!LUf&@)RcHhBfsne2w?m#Ue5oovEfXs?a zjmD9`^1KvQqp5Q_DmTODz|uvzv^u`|YNiJUa~z(`Zwr3(@f5DOK*2r*q&ivg0tA5a z6$I8hDoI&-Gq!?#SY)ctQA|RUGE6iD4rYht5j2~c_OoKsy9u5^Pn<7DiAMQ8ZTmqVq$ZdvZSyjX$DU)YU|k%BZ4w)v-nb#br$oAbNUw<@pPx z2$>80ZR!$PLE;*EJF6+8ymvvTR8#Ei)N{Ot+5r^?4m3!Iw7w7rTH${Wh`ql#tc-29 zNc?j45ZiF&cbXw_{>81_{wOQDg+`C5d=iy!+rCNe=f8+Y@>~>J-xxM7{HxO%hHY(W zZDm+8qOVjAmG@vyrRzy!HDEl(pH&l+<5DCq0n7I4ko zF=1Mm<);{PRYT!jwW*6HVdBxU0vd}M=*4vKN@=<(tFHv>YW-Q)#YwmmvPv;`4~lE*L0(gM!E{_PU7sG1r_$#&~IYAZGq33fzV$E1B@&)u!y8mjS4jMKf;Yb zFa>#eR;AYGSIKC4r$OzDd$+qr%-?J?NAsGaZ4&gWJT!X*A(&kj3{1}r59Tg<`9g=e zXl|t9o`wR@#9bo3-u6?OM`iU;|M%URorua3d~*Q|b&A5~FOrOdYRKh>ZTI>i5tF5JtJ{j6l?-XA<<&FAFC5+cZ{y`!#e!Q%)D9 zbA3IwarWxz7V;^cmjj*0H9pNFlB(}Z39^orovI_Fpul0Syx1~D(ZTs?c1*8-XX7Z@ z{{J0Tg7XTgTRv+&io@;x8stOjId+?g zb8Yu0s{My-cEAa-b^+Ae6X0pXdC;3@uYNZ!8udd{^k?AnH1hh>8Cl0dhL}7EexcaH zvQp%FR^EPTt;Wm!Dao^Ya!)6ZXuqKVH(>`2C>unZ_FwkVDH_FyVPP(jNwkL3KoxfF zQJf8*E21aMsr!`EH{gwh`xYIAI=h@m(rw}2?k;rYtQ3-}o5JVlRHsF;-cg=|pvC{V zt%AJ{Pdcy~+6ALaKokmEf(}RWy#DMaG+mOs{Ew(560z!SWm|;LSy+1#2IU8(;7ET= z$O%8u*=Sdha$emtg`N#2nFYMqO^cyKp@b7{hB<)2b;dP+BwA}j&`^a@t0i`aZ08G4 z?QY8DG3O+))4IS!X^pn=j5lkK#>?HlHdi^BF8cA^lMK2@L($9a_u%yOg9mmfTc?~<(1H#e0ek!%{oqFMZPeB&`;b8N9| z(-p}vWpDl92KhP1W|D3Dd^ABLX7z_TntzjUV@3Pw?b|v|XT?qe^gJu%3sp`8hL!8I z1q(xCvDq|FH-8p$f!F+vVa{jJCgAip_2pMEj{CxO{#me~-HfClP|Hmrv^uLJ&P7YE zi$qZ6Y+pPs4&b4IF5*GpVzF(}c8oN9QXn5Q(8v*G9ffwor_xx_%lC2ib2i&RHLw0z zYY|%}SdWTD@or{JNf7W(_hUq@U-tIISw&|8diafqgQahPOLkDLXoyj@@= z4B50OI;__gUVaed10pRwf_WJj*HbO&zmn4@7Y0L%&o^ZP%9Auo|tuwf>HTG!pP}8 zpk1bdWlaSTJ;yUz%uxd+Gr4T(!({=Wc)F(6 z1N5w7oFAmH!?N=Ohe@!Gf(VM$)=`d738&pS8YL%9S1f&S|BnBD$Ld5&aH-JtI$flz zp__&Pqsl-{-AVaWwCGnR3Qom(VmMWv;c||&-mM!K8x+UIQJT^U>17)8y&&*eN>8&p z%j(1TH@%-i`86`w)yb)j$Xgr)}$LMcBaEAS4` z4Y@HqVa}!gl8Z>6(^BGg_8(b1K|r9ns|z?pkTK})2vlosWHR?2R~-raW=02Ju=@X; zQEXMRc>!^&B5#2lJ7yJtIu80;JeY+P8W?^&G<-T zX@2t%PxhA_8D?PK#foRST&-?g2urj8zmv7wHU`(-&)dw+u0Mz5n}ynI*FGdHML!31 zeySzvm;sg-vUV(A@mHg)vFX$|KXStN!>v8wjyqa*AOc*&`nk9MGBel^eh7Ym4s}ty z-5k~2hIinp&zEu9-i3K zT{=$r{Nb&cw%iE3#K3t9i|uk>5t$m@Ir$qRz(CWqxX9HMY<>Vch<2EQs*cIB&g-)9 zT>S|kZ1w)HxtIR=Te({2-$a#~;;!a|(%rm?3fTBqx)&%)*?{L8)#vt`IsXXm!!_BH zZ_}BQq00GrkczfHiv$TZ&e2v*M|i5Z5&i=3iS@Zuf`wjeyN|*ZC!G^{5dtZ>Q&&2d zxUXk@8gvpuB&k{*>$foiUCM86HX7@>&_rMc&bB!WBZ*=)b;EbAs%LOLHAeJ^g4S9+ zjuU&S57{iYbuhp!n-lgDpb8DTo5gxpCq_WLt)}#trv51`gptM1+^QJ8Qg5YHO-30x zqOBoGLu;_EB_wmfkVqM&(7cGT9L>KAyy;C2uORq1GFc2wS-bOKOcEPe?xS-eE>7EX zvKVs<={Mh!!O^XVQ$l>yb!ZvS3{0o4CLCDOFwF7RNXN%9*ykGfY*UgM(lW-t)2-Tq zA0R=|uA+%q?{(-Nh!PvzDM6a&5BK%G61%Y2&DRpc52XLlygP}k!M$JDekxXOF3t|% zOER!d1Jz#Y+k!pam~7R<2C}9wqi=N&thet&(TnMvb6^&;O=zm7FsJQ_I{Kp$mEIGgkAU>J)Y*SR>5r7k3Da7Xx}K_0n(ZVK_@u) zRu@pwIScqxU^OQWgMQPfJxvyMIvjj)w$+;a=j7tZJ7=(h=XYKCwzG)~fjjwjjfj_J zI1OYN3Ib!WOQSjz%^#PfQ+OoW>RDUrruP3_Z>*&}zsA+IAMyhVc{5xkQE=XcY&_s4FK5VY0AKqVF(xWDO_0+CT#SHkVG|+FZ7^2u9QIxlI zB_`G2^^W_vFK1tGoLO5f!E9f`4Jp-!#DZd;Ce)dI&0A%lAqz15Ze*?(VG56Y$z*z{ z2cn?`1XiQB4`A=ZCnsxI#{FH6PO`kPe=e#ud~tKR#Y% zE(E{&?3rzcM<8pmB2(~seO6i-X=<;EjW_e8dia!!1>3pN(o#{d7`72LtD$W$Gecrwo63PKOu(;ybqh{VnL;D)9;|&_T#l z1p#&1_(r#az|if|@vCovB1C@*DeCP;T3JZAR$`GU6B{G6*&}mNubzzE{Fv5QAv_C_ znN>&#qbqm%mnB+)`GmPX!sA<}uOUUMEIPSh@x|&#K19;8(r8n6rjM9)`N}sAKUIi1 z_nwI*&Vr7)mES+$ibCL?CHtfOZl1pKDQ5a)?}iDK{LTq+UtPNfg_19?$ljR&)#DT3 z$y|XTV+gnhKVX@t52{2);B_{QUJki6n^P4X8VLVf?p~ZvuCvdJG!C07klM3l@_Ose zi@rC`GWe~yh&Mqz2s82f>B|*ZLFnho-G>wIJ6>YwFs0>8r+P3Qq$qc@EDS*LsEZ=| zxW`tdjF7;Kl&+liD?TICnI=$sSEC{lf}sLN!nK9jfqV@xWDf(hty5cg5h(&?JwTJ% zjN$=p&pDJu`v6LZxJc&UVsS9fwU+x5$6Qfmt~AEkyqnthbf8s)^P zbTepDh#voKDb!Wxk{3sI@X&Z(Zw(E<`H@&1OhY-hx^f!One7xI>N-zP)?ko!oY<`a@(z3WgCRz&INp(RW8p$5-*5w8_Gi2Kyb5DbJ zf-2VxP3*n!E&3QM2=zS5CkxWy#`w(9^m53gpgLLZt7O^ifbl*|5#B!0jO%}~mXt1b zu7G+^R~$;?sJs8sNR=G8m$aOsb0N0(VRmBbjay_ZXWY)tx!~)%Sx^^*pr{AkW zctn7p2o9U4IDkdqJT3KmvrG96=?9&yg7YpfYM%>~q&NL4|<=DtH@8D5+3zmI5Qm@HmC) zP_BEJ5V8rsQ}*u7dCZF9aW3O}S#v3myNloNz5^PO6tbzPP9I4vj6^v;(2gu z$&Il=^*sf#ffiu6oO5DF1S&p+CjP(y|R4cb0UP)_g7I)MFe{L#wwC zGc*kG9hB=xJtYb=XY(=?q!bhvwom`Qy1Y6PX44HH)G6c{%Xs?0PF&A&JH2}qH!hyqbz zZ4;m?rn?ChVl4e;_W(01aAtgisGVl?qWgOJ9N5cLc^&F{9;9xSDCjI=b<$iPT$tfQtHFS zygz6sCWfR3K971xE-UW-68&5SoDDN^(08a>>5EnCwN+Yn3x@3Pg#QVQ`@Lc+vTsX~ zKx6b|_5CVcFr+A%5bGX7EKmJH#ibWQFR;!MVrT_Oo*CnZS6bzz9pHj8ss9(n{d#M3 zNjE_PyW8v7b4WH8qGc8_okb#`1I3NbGU{#be-;&hfh61pGeXd{c^?iw-bd_M|G$0iXexa-*!?s zp z3(D4={<~(bD(pyIN%*6iJcAgRws6hrp*`jhL51}CB!9HvM7X*XvPxziLxd;`_dx-R zEZd3z{#JdV4t!X9=T{2WB@jiW>XLsPA__8|vr>zwwkEZE|WB zK}@89Q1f-TI?1U~RR)b<7s!p072Kj%;)^(QgHwIKfc5^)`@FOGef4DPKH_}${WwV6 zsY9XZZw)1S{h_qQrY(FS1&IiOP&0Ty+Ri1wKlrE+C9Jq3AU=LG<@jMf zby*jccScL`#SnpYe3Os^M2v6zo@WLSpGXS&bEx}j3{}-EQM^=9-h`?!QC_Y)juj~; z$m~}EfWBCSn5)nngpKDHBypStZ9yW5k#$-zD{L_8|8FM1bB5Gpfr26)I@i|r`Otdo zXTUH#$e9C5+al#JqcW&HW3v$056{?7GdOh`EHvl%5P?t0GCZxkO07N8NV3atoQ*k{ zH}uJWH5^(Q_i-&+b8XkVFEOBaD%`e%#DidobN;Bv)q!zIw{*k53YTT^_=d3XEL<$t zqJBg33u!r9GMX2IMo!`HPsICXoTuRb|Bt;qXNXZhvB&8j;JXS-!)Yu9_zr{&w>9SR zj3H8a3Zj2i71!9b+4$-YaZGcls$+?8es4ca4^UJ~C7b|4s=`y#(?J3rO;WF~?yzi_ zCy>VUow=ub@z*KC)dogiIDZ9mlgbgNyEZPYU{CC^LNOqZP;TJ>U!lE8vHDeSo>lb@fwUOGJN6s42NKc4-{1I{t$c%0yfw-c$F z#tIy1YQV(~-7zbQ@~P$e^Qsk6BQep*5jZ6)gCz^OPQTg?DVg@SK-#@1cQk}Zy6d}) zqpToC$2XO(XqB3^YoxmF)zGD})e0U&2uc*0x?OC0Nk<7yL-C`Ze`hX6edYz>+h3P% z!zU%D>)c=T9(q43O&WmsP}D>BNHGx%`q@tEVt^~xHr_se z3$LG(w4Ut5v_8di3u$wEz7$AI0%2FC_+<%t+6VF3*<_EHIa9#8GlI%?Sb({b;pXjA zuoUP2;8FIpAY`5i_SSm#vTQe)Ox%TXyS>ARECR5jiYK}59@V4O!g&R+xI_zpo*V5tdY@u zi9T;n$7a>OX-{zkTS%{Pc>bl(DDsKTDYHF))f60QMQ{#NfpZyM)WzTLBPued%^!l7 ze^*KB&0|Q5ePB^^WNdTgh=hyq{wzomD$uo`YSB6`fBq_56Bkbk6{mPKJ0$rGl8R;o z`74%>S(|GElciVI>O5q%YX(CES4z{0(>c7D6O;nz7)Cj|Pac5_19!ka^Rl3kxP~bt zaUuI|dRbSkJO44s7{XZ3-?nF{7a_(DCVTi+kMltvO}JNnc>#KFT5Xa`l&%%I09Lh$ zy+KvudFd7qx(l|an!Iox5EYAn3Y8g*LZt(3V$KOLLV8E7$q#9my!Geu#g#C1`;RFS zkE|aajOu?UEXc_QWuYgLp0ARVZ+wg@0m8n^k>SBn`$$WbbJha|GT@&iQ3c`JQzEo6 z;*f0K-(lzoNk6~L_U#(E`ahrDzj%wN@f1StgB%YW&08MsKA&elg&zZmDgc(e5=@7E z$m7PvPPQy^$rK7UXyY9>;IuX@S>OeN4Jj_eA$O_SJ71>kg78DcM8{1YkY&!VS}+zJ z(wn5#<>+bAW#?TP;NIaCy<8{|0u-|)%UMI4c~T&oW=#&5;}pN*#o(Z<5)Th!(dtlV zIFfGpY*(SxR;f+o2i^Rw(rbSch61I-U1K_zue1i87F01-p&dGZZS9(@L1YbC_+Eu} ztru6&tfqc@^Q(@>kUNRUh#|P!*#dwYYbwcQS>Xaq9hzeT(uo+B=N*Y|fm_FoDc9KDohn zZ_!`(6^9&A*!@oJVR3)=MD0f(4W-YW^L_$-BojtpF;pLJq5q$V6tDJ)S{Kd}ytw-m zW{{>sJacVz-3r=YrcSA{H6F^@z3&WB8P#q58Ny;uBf~r!JG^qANLy|jwg?rrY0 z(QdcA^UdY@)b2e)L2m;HUo(s<>ULSoUWuMIc~I@>FJ)b3(l9`l%6Q1Xl!fhi%X1}& zP5VQIbMnPW-4kYNJiBe{L?=msnU^Z}Ef~7}ilJ|9x6rQo5P)+4RMdy#q#ZXFy}U5D z1A!I+6`?w$Gh+Ao@fDR3=c}536Qbmr-%hW7^fh>9))GBb%J#HRDGHx^qGgxp5?6s* z8#8XcO+QgfV{x+n71gwUREcy2Elv4f;ntE}Y%Jq*9lSwt7Rp=dw_M;_=1UMa`Z#+d z)cn9Y&i83yW#c>A+oN7DEW)eS^9F~6T6{{v~hqAvUSvlo<2?>iK9?inIoo8$!HrY`sU7|=3|p2 zc`d>yg>IVEq!zwrQQq;LTKcImCqq4zvy@(Mb>66NQLWB1|4!|+Rg-!)5xgVy zeHXqFlBAeS$tTEJ<#4~fk8HR9cTV!8_~HaQRbirguSR?KyTSt|;fw5%KC3~OoT4(W zCl^dd>nlf3dI~jE@KKWR&u~b%dYkY}x~Ic2D}$2x((>0qc6N3!FBf`{56OjEzYsmS zmR6`}9-4fEf3t1bU-qZf0@kSu@9c&KshPl*JWy;m;%;kpn&%@L3G-oQf# z@A%@q>qviHNu$v82BKhMu28hRmm}e6fuP*g$|Vh8Zh4b0L{W|$oT5J_eIJ>f;LZxE z!Vf&!D<5DQGa-`k>5ENhlez1ULsdcpTW7(?d%Z$Xb4<&h)Ztg0Y_*qjGV@s@thQ3{m&G{42}+F$Rb{PnuMO#R}6V*`B?Fc&p+x)vJ$ zT#z@NjeBVmUbt*_6zVxQN)o@uF7K=Hcd9VtO26#Iw9a#ZkZ*j!N{y>(stkar|55Tv z?H`~0C%;4-2PV0b&!_l!_~rv0J?mbxAeD1zZQO2Lx&=2i;o2$hl_S3NZeJ}BQ6W(0;E*lp ziAkMaxTJZFo*;sEog}b)v_t+jO54;Zh#qRtBBh^7DPN@$u4%3enFX`a$zwWvj;JT| z9*g@=9}F!HF+ZLwaoRB?1ma*6wJ*AbH?`{1St-fC_+`5LmVO!^IzHn0kd&^@nFWvUonjzcuiZ80ekoe536c5IJew0`okU5nhflK^0m4r)k@dj{(g3gq7lMM{uP!w@3eecohSy{ zjFWOioOkMqX}!_%e2^P~W|-md`m-O)*VuWkGw#_7BphX9y!LOcyJxDT&q}lJkLn+* zr^w|6-42+U`fs*KvZg{k*LRksTbb10;aK!=o%_F_I7QXpYo!?vj}~Y~YWMt2h>x>` zv40h74?DzBN1ZoK2veQxm-g#Ky53jak;9iy1|75+TMW8BD{)jp=AhKJ+HBSix$GiI z!~WV@#QSXHowWI%ai-^c%CqC~-!AFodpRlX^ZoSpK*{4bRl({KwQuRFXx2ZBp*iL@ zQPn7)Z$Tz zja*CycWTdd3>UR{A$NLD_PLKv{8jp6v71|@e0yK+Lr+KDN9*<>vq59?o@=j|l&WRa zZ8v_B?k9fNV|Y`n zZp2OQI;$Q@B-Otc+V@d~yi}v~Yrdc(=W)6(zgrzuPJP=SJLcff%l)ZT{pgt4HzChB zhwnk9=a0Pi3F;3GjG2v@ot0>vNw&+&&(`nW0ioxf-ONL(tMr0U>2 z%YWS(@-TGMBF)=U*oVkY`9b|p`x_!A9`ICs`E}%wu^6&`He{i7IC-O1|CqOi@T?;# z$lrB1%T6YSiXOQn*LIW{x=S+K?jPL5-X$JmFjl7oCu;vu3)w7_%=$rME8*n)T;*C- zC0bT*Pzg~*>v^uGzghSe-?58lTdzdDV&A#+x7HP;z8Z?ft9*6HKfEuvY!kh210t$@ zT(B=2ab)bMlf**DWPawS0?%Z6_~s;T4Ih%9E|XvYgFb!rSP8|E)sqSY>ubDYW$KOQ zOD7-&qB@Egn0Hcb^mjyyg#o*$e(b`T`VS{g!q z*dtKl&<>FiMtqDYFD>po{nh11P|3ZCbBgpAL+VSE)jzS&zgoYx+nTKKnEzRuubl4b zw8?IhgBMvR^#40yD8%m|fm-PQcct=FfpGL+BU26Id_33q{A!E)CfpCtu6#+?6c;lc zZ91diN_HJH5rse;e@b~>`H!?Sq;~&m4gG&7yHD@EWBy9&@;`SgI;A@1k}eGAJs+=6 z)Zc1wyK%TuWI!~*&p+Qn$0O9f+SnCtjO-Kv9dZ`Iju+we5d9p3vaw?Yl zpiJlb!}Y54`IFh9}WYZSTYF{6(*XxYbB@ZdiT8$bUG22u!SP!0VI? zRuxHdk7&!CA6ZXK9PZKHZfm%Lw=1~k66EluN%20p75+l$paYSK-}#W=6cj4E2Fd*9(;mvPZ!7by>*+xu}-Ta|`8)yC2nOc2B@ z$+@=9+bLQ>p^}vLZ`jeFM-u|hAK3Q0%vZ=Q)>_X!%T?ZRJKf%pCCaE;t74-9Q>||E zKE7Ds(!P?A=!mADZO!wMB%a1x+0DzPnGy5^DfZiXaV4R;#+yl&ab5gz;9h5Sbv0EM zqV+)>Yd}S_X-ZcgM3uB3Cg3ee!u(@B@ftrQ17;LNhUz@WY-C2{C>7>^uj;>z#ArT= z^DV7*ODA2SU)obdKj1FSxNO_Ju-+`=7AMZ%DINB_S)ZL&?P|X|J`PYgIB@@dt5fF*cv#OSPA4W4&tM1(dyfl)*TmwQ@*yMt+Co zzuqm$?JX^YR)rlpPTw4_NinRZOFG}YRxkd%B$vbG9ThN{XVvC&;YSzV`M=(K2Q2Z- z@{KE1cwb(bJH-%^72vgWR?c@Sqy$)IZEvpsK0kItQh;{r)@gG={&&=}$F!-Bn*>D_ zIq&;I^*J+-=F`Hh3b&;4FAp+B$8DrcA#W!76~XI@WW(6NWJ<}u!LmiNq?`JYZtWT8 zD>8$K30(z`GOP+(R_dI?Tcdx870zw3qQPI8tUWSSoZo%=Kv}e_uDW#uk8pmx-D6sA zp!6ka&9;QF%pVE`eA6Oc{T2+A*B0k4$tYD~ zyT3pmL=GITY;enrmEJ0&W)jfa?0d^$E%lEmg~{6j96}$cIQTumM?nTScposMJUl!s zY9zFD5>Zj3`@qI{8*=9Wrt;^~)8WpK6&F)J(rB|yd$qN%b8<)IpL-V^RS-9r>~=l8 zH-Uu=`E4Vw8{VcUC{50l$T+p}>WPTe9UiZZOQfw7Nk;#QhNh0!n`aX8y5`RH*1@2b zK3oo^(emOIKOXGzPRBf+0Tl+#d38LS zpbg&L+54>T#k)kB`q%uo6Dh8q!kZ2Cs(z@ip8OD2*yPv|RRL~wbjW>s)Jo%AZ%c7x z_H2LrIAp)qR76*C+15iXB)`g@iD-CYD(Z-G*?t z-~FrGh9=Bf^a<`sk|vhN$^>}9r5%Md^@LJ)`*l^#VpBEV%Ow25qvfWDSxh4&Ub1i{ z&z8K{1qFb2_FhS7jgRxoGAXm(ITV_lmr`!m^GMG1W5CVu%P56iZblDXP!Gy$@q1 zA&v;9i!bu|T)1`H6Nek)zK#{Si)xw;@>sTZd(c-TjOpTsI8iRQoojqX)2Y1 zKCJ=6jOfk=BA+C(We06#y2NAY46>;NnRl{BS!s@SKD(*#^~T;A3x>zGCmXn`ikZ|B z2_|!iQ3Ab-aY%i;0{QSc_TQnQy$z1)d#zPeDvdGfkLJ1t5(dU!8M@MWX21D3>wVFg zZ1Aq*uhz>oMMB0I*wsnPLf5C(x%hh-aX_)mQWQIG;*%GvAQ48?YCO@?`w z6zg03P>7`LA#3O4$1GPylSxK$jI_EK@sZ_*+!;7R(;aj*sh{Mp9))qu(vO2cdu-H`k=dfdNv>zQ8LpN)!Nsp4A9e z7f;VjH>D4abMIYZBoFZpUuw=}Jo7VbUwB-kUYiQ?QZ^NjRf-P;k-XcqeO{0Nh^40h zJEmjjWF$BZkdkU>4x9(@wAVyD=RAz!BYPTt%A^ zk5y_Z9A`|6{&OH;pf6%@1@xqrYsuBfPr#)e+hKBvg~UK#>>!gT!GXt+wD3ep#O0wb zmmeJuDBfv3gp6p1X4-A)B5vnWVZ=#u4#qGCXj(IW?Y{De8^XxZ5h#zK@!?FVsqs+x z?3mOXc(x5p9R!-h?f!wpRN!&~u~)wzBFv{Zz}^*q;pRT)??Vyy@2$!oDoKmL`bE*v za5E8s5$3w*wGS+p`U=vxgTkUKS+o;ULI29HE^dKG8C7*%pSsxEgP`CIqmUB0TLHw4zHIP70|qa+sG z9m*FD3#e=d`P3*W5O24xG`(LzKTa3nS?J0T?7SQAulN|B3zZ26@LZ31Kjvs%HRXNT z>f>d5*Ia73j%FtDI#VR|5B3q(MB2{Bmu!{V?wrM1rlML}B*{NgL+f|(IHrEF>oNy7 z>+EnGJ!@b;r2kjDMj)h;n{)_bRI8$ug4{En)n3xk#q=BgK}1{CG=WnbV1C|QlZ_Gt zn2TzqP1zy!c*g3tv=&=xy-`*l^L4$fP42g6*{LBsQVTePu%?3@jZLDt`LV z6J4SzK!6)uw#hGFbNvdLbxdVLnQ3)s`x6efck^Wc3$BgSE-j8v5>^Y%#Rm{ zomMxX&6}+@%iNC4p@C=YevPhqwl>eeUhEtKDXoBU)K=M}njd=F%ncU7lo9Dl!E}wv zkQkI>%N(t%TQMD6HKgD~xjo;{T{4X|4c^!4`R7?XG_PDSLJ61ajAOWMTvL2<(BML3 zK?0gM((xmfqDlq|2QHAvs+Th`{_(_pxd`@_t~jLXLs$*%J;Di8AFAev%Svl5bOOh(d&JR#EVJ_<`Eu}a2- z=WFgl^FutUpD#V^8>y{@L16POzjUKC%)DG2C^K#hX|Q=!+OSC{FM$qFS;---4We^# z-}Te=sixud2TX*DCqf?9>$|b;r}c_APbSeP@^r^l3_s7eRKaHoOxbZ zcctie#r?@a#qy*>iO($eJ1J|s1RvF>X1DJtK@@K#%cB`^BeQiTunZ_;=)h9|;2*Pp zdYq zP*w8E^dC++A~6v1z|PhiMJN#p@U9v~V&x3> zJ^!IfrI)rCKgLa)h#qyFY@1Mpz{fn-OJKb3P}%Hu$Zs=#bY*go=7G#!lt!Q~A2fLWqa|S6y<>Q@b!M4_E?%)&)dOjjuxC9h5@lBh;NsPJ2m}OP0u7=V3JtlK$ufy3w#2wkS&^ASp@Q7Xpp}xf+6D z0imXo-r34=E$NnT1*!{dHC45+N!#~J(-$=GC#OSVVrtvB(S^la4qA#cZPG+h*P%r)lEjL%eK#7ilfnv23Fc3GlHzKqW7Qn=#YZZl?N%BIq+W z3S~V${+W-;La8FDZsD}enj*R)E_fMoM0rwa^GAoGHL1OE!Hs||tn2|4)q2;j41tLO15Nmz9cyH1TN)oNpNFO2 z=lk@X6tr3|7bRLGTWseE!0^(!d)R9SU^tbr8ZZ4IC0)C-6Vj-^%DADY?d<6*GjYK5 zxV1b%IOgyYw7Q{F&>U}Qy3`T(9YWFooXqok>3X=6z|@&=N9Vjp4B3+`4${bee)Meu z!da})a1~je?=(AF6cywAqlm_sdrDOO>&u|a8&hzxuB3ldnT7C!X3C3mcKv?M)JBX~ z>UW;W#e{H8)I+G+oYGG>xmjTHNRbKJ+ZZy_23ZJEWXLwUdlUvBif9mUY8DTMp$#WO z=>RC1k3sr19Xk{A}qOu>DWQ>I4Ty&K;ci5Vt9%QV4AFD zE0MX3X1D@5%j9aEE8W8FhE5iRa>xDMS{u2YV{3xywVRL`aw;#4Ut}d~!-q|Ea zylZb`4r1J;rC&JGBmw0Ec-L(Z?p3fo>{#E%2TzqN!p;a{7GiqhmEj7mxyr5$YaS|>XMV7 z5D8>o!>Ut&CdC5@twUzqR+GstEQ+aGh8U&o$`%HO#54_dfJk)PLo{p`oJEYXWgRoD z0{U!w`4a9whq&fQguL-xaqEUxR5Hs-)c7YgNWcID{2l2x$G@?55zn}&bVG0SJe77w zW>C;u&D*REkJsOQgSC1;1+S$il;G?YolNzaa%;l=rwMkT*ROA!#=Oey^c)-J>#if- zf8%Pw)P^5fQ=wI@rAE#5kbX3yERec(&*@C@Qo#0TvY6(1Tq~cn3@3Yg?U9&)7)xZO z`InQOjpLh=)|nUsZ&rRkv!LBoW`1>+8D9@zMBSJvr{VW8Dr(!7c260^Y;_+_y;KOM zo08yUoHI?!ODd>~Tw*K|tOIC@s8X_~DE`@7b)8~72m>p5^i;ovG<63;l4xP21;+g4%ME zJp*K!j~tP@3H<4<51RYPVFEbPPqhfSOR;Iy4BIW8CRyc()LKMp!YsR^^ij(b3WYCTA*qdM=;I2P&c!i5Nr-Xg!xaP0~|bi#fRPz0GB zWKFCRf(mNp8_W~IgN^K9&$U?!C?qBIUykESTL`c@N8kd|b5f`MnanX&2oWN{C`nDc z&L?Ut#~l~~vZMyl8qA%SEzf7YmXehjM_|RyEcqRJf^9|g`J^5PO>2Nk7uY$dQ}NFc z>wSq7_ChvYA!cGpRVrAAgfm$@p@gjdXr72h*0}V#A{RD3Y>J?&4ZT( z2_{;jqne+7fY|~IOh2&ovuYny{p8GTPO3q&Bw)`A)Yhy+^ZdtaAhDKS){>)0x?4A;Fv)CI{q zO>~8J+Z8$OZN{PS2n!b>f_ZD6r0qf@VK+eK8vU2g7mluM{3$|+EAzDmln9;zO3*tQ z_p~MdFM)MKyqL8;Spu~ERm6vMXFS5^u{dRD*Qm%mpf2gZ1?ZLsrxzAl97ZVsptlrf z&YcVuE193cQr6`=MsX}b;8m@C;jsfa{A3o@Ph}v7GmS>my79@z$NGBLa(PYJMtA6vV zDQo*p0j=c(D!{SfO|)j*WWbzM0Q6H7MyQ5&|6~y4-Kx0jJC1cISu$w8zBag(y%4g~ zUe*rKLDIgZrC7?ode-@LoqDbaX)6j=(iN-<^&6`B`eQG3rS1a^0K2i|nQ zLDTx%@69XI6|v|moI6G~+no!#Qijh;K;6`^*K=78bvEy5gpI`$G|pkEp+p*$!YTHW z^Ef%v`sNC|j&J3&&8*TQa#wG&Q4-5L3lUwwF3I&(mn>N=?P@-t&;GLCeTOfZ-m zdqfn8Q;-$PzUGqp>vI{7w2-_67>>rE%SkJ76h=3i@=H|}N9}<%;X{oAdhFc9TpXcN zoXp190Q0RYDm_ZWPxw&((1tsBcczGU5W@L6ZLJnpSyGesj>Tqj<;Ap59t?l zGX_O=Fxa3k>{ShSpd8P8faxm?1y4pD9I-MrSkivPpE#*pLeZ)<eW6rV1L~6`S~ms(wD;3oXJA)bB_MJ>ZQ|S0z-qj?b*Zwk2tu-tpD&If}8_o z+($^9Qdko$-AP2DqbmTy_`m$uwZ#@)7C*Q1uESBynD7_6JXLGKr;9)sjceT~q-BDR z$n<$FIil_2l;kvb8K$bH-R+#u+RYCOppuuXaMenxLOoF}a7Ln4aTu@+=LY+jejMDJmL>9N*Wxu3cMYf6dF*(?#G2+7Zba268yx!q^TycE zDw{Oc60oFT3YZp@E2(Uu;hC)OhX|$H-LVRO^WexRNflCN+#8k8j~<=4vc7zAOy_!h zcR~QW<}S&N`W(iKd$6gdN5&`mK$k$NI>WpPQzW*{MCI=L z(&(%IiW^^4Ph+iQJ-KNMikq%EIpkQ%-W}dephjMmM%5SU zsoUt5jn&T3CI*n78TH+U-&B$Y!923%PQ z0bq6!O8+o?t#Uj$ZArlZO8d8B!wI9$0Ozh8q$UZ1F0|F6R!$8Ds)Q2Qlg0tmqSjNc z<{MY*)HB?!)#i@J4{+LyPasnl&8hh26^(-9c0%LtMoG0OyEgiOjFtWg<-Kr;S7jWW z$gkm?gH>6BDkYwwpBT)VC(Xf@^pR!wxQM}r)rJ8Q)FYYTdUA!a?gmCtr@<%z{rh`3 z@_?MJ&13u!Ykkp6&4Q#=(kLpdZe<(iwP$oOmlu^2jJhIKN9qLTB#~Lx!Z<33N>PcE z=EP-j*Q}h-*@wZM0Cs}>4imydcO8>PEd2xP31pBNw~=dYZ4DT9yD~FVJbZQ`QAW{V z;m4kyq`i`##L!G#xzDR-PA!Ywy+m}S*^W*zB)lp2%WZP|5od0l zhV=YPz_4PjY2?1)R9q2K45S(RZ_D367LqwO)H1%;WGZ~ikTTcr@*w&aPS!o&D z_ecSV0Hzbwlh&G$C(9*(xo-G6iG)&-cHhM1k!@*XYOIWk)yUN+EMN(1B5e4o+$Oyd zCb)M#(D%*wcTSvgC%7iKiKT1U?S$z!Q?U7XJyaunBF^>+&BZ1+P9RbO!(ZHBOqY`n8Ec55B)TV@8^lB|dwRuZ6&BuvuN(iZ|vIr+^ z`smmC$+yB@o}_ESo*BW7m9bXE(GfmljmIq`h~k!Ml}Nldj`Z#dLame|fGpv#9^raP zZmyuBhycG@rGalTk7E$&I1cd%v0NWkWG}nL1`J+eUd5Z6n|hR=?F8ejE}L1y0f;>R zGTveiUCH2Kmu-S=xzdkslYaiDQQz?qcBUYj70Y@kl&w7L#^Or-u)UKWN_oWJUv&O9 zp#+&vEk+P{EH{qpxvPdurTz(ZjVNC2H&}zf{d0^iPRX*M6%6f5nz$*_$;KH3$B(yH zk$^#pP}0`P-ZkD$a~vTL$qFnK-rr)(n$N&f)0wfTy{_WbUm)~XsLn(TSFRJFqPM3Z zl#l6m>d-$No<9c|=XADrsaA zcmA~uNlH7Wk(uw2qbi(kU-7HwgGe^s70$6HUdGUp^i;w^*KjXE9_NT{kiXgriksUyJ9$?zx45WX8m{ zFqva^i}3OdbhHek*)^LL3w6~f#AhxV&;w9}!DAPTl8U;PYfR+55EYDYRoDk24U|i*bGzyjn-l2d zhedOC7aPOo(F9i_zsakNNC|YpKxsjGaT^?Aq~sO!?Rf@UmwFjC__~Ffb&}P1IVc;gXWcP&}zZ87}4U`lx`sEe75dZA4 z-!S74mv0lOZi1_rzN}ATC!CXmeSq{q2Q4rrWaVg?;vgrSlx6 zUl+y1Dy0|sDdDFHO!)*g3Tg=Gn)vY6pnYq&A0X(1ND`S`B%f6SE(Ks5R;cIFFHB7K zyW`SF_hC#nIy5zQcYj3UNZQ($W??TP14)Kk##75UwX>q^cqVP6eK%rN_)$18isph? zB?bpT>x-<{=}S|lv3D+U+Wq2PY!@Ly*R*kQzS_w3W9tN(RU$oy>@=smg9$sQa-&fZ zqLVK!IUxqbU()OzN>yP42mO`jnCDe6l{Bb~x9 zQyRCD$*$=@73Y_?X7W(?R>i-wsEF@#2g?w9OiDFicF4L6BGvezpoe1ad^roOb+y%XrywucM|2k?@e=eCDDH z*cpTIX!~58oW=G1^Lo;my<=$p@ArgPBu`BW<<}v){%$Fu%CixC2M&Lzr|n8UIZ>&> zYVV7-@17%AqlNNCs1h~Xh13OY=s7BJmrHr3vPy`&ly}HYqt^iH7ttL+URReivUw}G zAl^f{Opv{@PrDrcaCd^3{@N4a#Okm5@UlGGz6l40hb;6EhlNW@NPe*1Y{lx4c5p3pHQbQJ8;|L5e1Ib2wkytFrKo{amdLwiWe!mYbb298Q8Z zF#4m$Tdo~VWllfp=EBG>F(0>d*(Aq~M2gB1%c@HH=DZU5b?+C+Y1z~jTW>0Iitn7P z(w*g3>reg8E>$*)bWMD#3!p%ji*#4*0taV%@y#sr~ zVri1G0iAps2y4ZCC>H7#<5VD$g=e0t^hkA^K$vF>3yx<+^9ml%`Jxkvc$c8Py=ZA* zuWmyT!-gM*1So*pgMze;>swBU3faGhpyDRYFEdIO)%k@vZ!(u8A}=k$qU+83uh-QC zs9#gZ4FmMU-8(2KFF{O=%&RiP2d(Rx7OJa^8RTnlpvBn?Guyy7XBU`M1Z1D6a|O{= znPHOTfx8|>(+_C}*(6j71s3QCT~@9!mwu3<2N(zcBNdY)x{WPUpA3(wT9C%CPD*{&Y`PNPvNl*q3lOWR3uMTR zMwk!VJU0}dKb~EV?W`kB^Cy564BGB}B*Mh7eRW6Um{p!_V#t$GvZqPd_vzOajP0L% zEaSXjh}X=5ZrfS?1t66Y@8Dp8qLqW^rOg7+mb9674hx249(1n0@~f9!5JJR9n|Y$y z^=!L!?JE(0gK*-}TE3%KeMp3SH@9=y{u+d&g_r4;TL9Xd5~Q%L2u%gm% z#1!~C^r}Kr`e^pt@}uMr9_{Cz1uF36HcP-ujj+6|iEbUge-GL{@>5?Nw2+f8DIOmb z%Tw;6o->%9zy|iv8WTQRnT1D$+e>F1*-f7}rI+u@x!crwzPc1aXL)uJ$0+U^@{3Kp zPDbtr*#U(pw@K+eF4c6XSb!OUp#0+Mu@nmlKqJUme1|CBXdhw{zfPk8pgp1r$)8== z-&+5~)Gui=*;>b!e|f|y=iciA_GNkhVs){*cT{!L#N?wiAbo&Kcb6Ph*ovUEg-C>! zX|_rVXUWpQQ?kMZdsc&GiLWLt-Hs;61%N0W=}PH>8|>T8to?CREhM_814Ihfn`@Tx zoO)t6m$Q_DSWbxIAaCJe=Y!|pI)ui%B?EG{l)c@tZ7XQ!;r?w`& zqA-R_O{J&cV*V0GSg7?)&lHV{YCE@^}3A4Gnk-XDyBM6JRMO?2)kS2QflIcqN6=~{@}=C28#htwoE zvP*q&V6L!ay-3R}s6ow}qOax-`wbns&SvpAN6`C`MhT>jK+wsgC9|bnZe%DjUAI_L z5E#xyb?|gs9Oi|3cJEMP@1a1=LeMLIKbQ;Eom3w#7SJIOxcV*<_x0IplUHg4F~NuJ zI&%GeZs8P)<*?oex8f>-Mu9o57b}4aR5Og1*s5tACuG=V&+j_R5VLm$a+AT^a?6duiHDBsuWnB{uEsUg3=Tsd^v>`&?0F+}huV+<{z54s+Swp&vhWVCy0J2VC z9F5pl;avbU`v6UPx%)8)-{R7ynyH zF)ufIr5ng(Vvui2LTJa?7)ZJS;%1ro<}NTICP^r}T+SSLd~xIinC#ls%e^JB-4OAO z{QL)u>Wz>&6M)=z7eiD`C-|>*$LiyA`Y}7e1oa5Mt+8Yq@*l3$rd(=`Y6NXAlP}dX z$$#$~kS`HLQpHv9 zY^#tjKf#iz`2Xj3!yzgr)yS`Zm$87ak2xIRQe>u7kgR2CAr z%ois=sHR{|7WTa_&#TS0$$)C`%-{jv^}44%qr@fPgA-ToEAG{FS<_r=S6A1Wh$dII zaWO$MO`5ET_!Vi0Y_mEZB4=T+Ul4E{w2A*3OCaLnWn#o>^HUv2W4QHj`OQKx@pbuH;WT7s}ERot)JoAYX%JEr>TAAHnly|7feG$`ImyuuEl*DR@p5 zOVL0s(CIcmVaLPnsfDEBv6Qckshx0?tAH_8@lf8zy-e0`F3q?aHk7?U)Q z1=UX|soF<(Q-dgd<-5)$beTrdx^>--z>_5bR;J8a$p*7!SoKlHQtL;@B!aw_K(#Vi zOfP#$mMC=3LHC>4bZQe2qyB>zh`gDWosU!#*L?jGJY~-GbGr-4#vtn}NgZtU;l*N} zlBN7-qxVd{X7@D$_}Tcbg+AD1?B%}!X1l18MCt_Z`H5aWBspQd?R0T%_*fdP=@`Vh z$X;QKWCPE$Ck?$`JgwV*VlG<*Amp;8iMk4OE{jWfj}VlzFWBi^JnOzRVrs>6(=^oG0Q- z35Kr%10W%stF7RV+Oyeuhln1rpgoBjPRLsMoi+LoJ@F2&N0gMN#o(zhU34Q#md%r9 zT7c9uno3UD{qq}dK_f@CXXLuOjX@wfLr2&{B91GnLJlE?avMzo-PhkEh_t($j+V@t*(Ck;V%rpy z$cmMZen@{Si7uM7(KD#miYrI+CN1}#_T8N~xN zGrzu6nLrMt+`_M#*@7jnOAfg5MOS)Qf}wTG_2bNlP~>*y!Mf(pX7?}D7m#Zzo6yq0 zpHT;|w_U>Hhz1ReA#zxPzqFl=c6D69Y+kb>vS+)1yfGXr<`j|_`kE%{v@odc3ob@c z0UNM*tjA-h*Gn2skex6bLt@nobNd= zNKGkpp-|b^Hb8#o=>AKU0R`iiwMo)Wr~LZE)yCxJ+36x@F+D z;k1-pYwK}-M&pZL#zcx*?6@4e-lW*&x5lVWW)X~6sJq33@l0o4Or39En@&>J^OZ!O z%H$~HG#EV4txNr$(Tyt_$-)31`r_Zb851>P4kC`A7;GEnWhlQ~i~CV9KL>xd@zAuD zg4S_Rj;Pf$wtjPw)d~I7+HBZ4^Ji{tsSt-S>bO?%VqWhu%p zMdG4YU(AG3CDUJvllv#`8W7)GYgKz_25J`QU#8|nn)(HBLT}A9FdYYih4}nxbW6IyFs8vnAgR5r7T}KeqNM(${Ip0q*Z*Ny?z>ipwca4~)xIerI+cgCL zYVZk6r4sa^8gtD*E}AWj3sL*~=G*PMb`}*q*NdF=3aP|Cl`gW+_3pFTFdpC~7YAND z%i|Jl^_Fg0B)r1JXvDx&(bFcN3+>(B)V=Z1OaGoBbCs9N*PG3!99?z-`e|q+!zCK!$VS$BHs2TH) z;5OdOMIR=1`9x6d&&W)aYVllu^Azz*Eu>B4>hFB5%*Qmp43>1=CZ=7=W86Xkfh6!? zx~lK)yi54IB9z(|TuVwl~LQ^|Q)U)zP8x^aDov!$usjs6y*2xqy#@WL_s+`Idftn>Y*pI3vDr zy^&$uL2hfmT|VLruefYlt=2DiYAf*bxx`ShEm*OAI5w`l72&(kboqwn8RfP~sKTRw zNnqJr0RnMvi*YZE7+QUPjS|8^zQ1^#P7~SP&jR!C{(@e(mKW9Qc!WQBb(&gMrcmxP zeGm-~j_!TGHS_e0XlP05DrqMTurQ~*qmAp!y?Gd^)c5K9%#Hf`n&{%XWaTQ)B2^2w zPi@T2-4GRZU!((nr|72)T&)ga*^KM(v0DsR3^)sJ&AI$xgrmNn2W8sP{i*`~mE-{m z`OR{LVe$9bT$Mlws(rKV4v*QIX$I0Y&7b*M*#O;NXe_;)YkgwT^ie=jA`423T)hC>v}JO9J6-p_9#Z#R~U|5hnl zk~!JZ#H8YjOd;lNkBzl;!O43rN^CYb1u&e46Ha*M`dm`qz(|2uRNkV@V)rrANc_lW z%jrkIjOQeH)ue3i+hjO`KR+7SMuVHNE zqtoY7C(l{)yr3xAnRRVbX@DnbCIgPn`>0#P2)~8PbLs@+z7kOGOPw8bvQN%jO4sDdroz)VD2vp@-seEiP%MQHfLU*e%8 zO2x(rWjFcT&K05$N?29eyo;Uo^M5HtH+i0alaLgzL6hbRh7p8)27sHK|9)i0qF+|F z$TN%)8-0m!#`0+{!jqEPnJ6oH%-+>!SV#wX-`j8Qv9JCqngunX*#FIZhwyu3yiBX7 zzlJet9@BQ~m6DH2`wQc_(CE+Bj|&q8A+>%wnRK9>6Ln0*CsDvzl~34hetx(F!flvG-j&in`o^IgQOOG~RKEtnmG?8i*wRBSL5A!y zl1`^$_owvlr+DC#TCC=|u&odFO;YyS%S-C~hDDkp1=;nA#`;wPBQykPwDZRsZtD5< zN(Y_a|LP>sh*1u7fn9fU@6jhJ`L-~xLC%C`zfS?vQWylyvZ$F|X#Zf9ag`C^5ryqa z*o4lZEVWgC5xo?%|BXu{evj{p2Mc3f9B>`Fa0f6->KZ5A`^BfJNRP9yU5`qmU2-Y= zO#)LJ3ULnqSQZ2_4-~p*HOU;KQ;1vm)nFH&IkY14`>X&|uZp2(I#)(Q!I(SVrODy} zm|HDMvDhoeOdw0fQI@tYg$swMD-{@jsuv3~+}((of0IfJ$K8+HE$ge7j|GV;Kq93N zk7d^ZrEu`!@T@u^)lV+&7vaOoM$U*s%|jSZiL`V+of8Oo)2LAd>+;8JoksVppgE-dLc>lahh{8S|y4pF883&A%) zobd?uA-`P7%&m{?YvzYDm++g+^e_ITku0xODVB`=PkG)1pEJq70C&jRy&B_N>j4F6Tg{&fQIUPCM@*~p&)H}OdX33P%T9E)ZY4KQfaMzZ zzK680v7G>x)EE`3UmA8gLa}kNzgb6t!GL`iNO<zJmHRR=Q!lh zf+p4vbGIQ`3|DQQUmr^cu&s9m%d71z!Hu+@dS5b?3&nB{cZ2Kctd|cSwJWI;_UQY4 zAUeW+^&TH^s_BlSKlquvhV2Fe{J&mS%P=fo^-j)<==q*Lu1$_P>vYOdHm(ZTVvZ>) z#KgqLaKOZh$$P()HSj21=4aEQ7zRew6MKaefFhP^b>|^~lVFr31M~U!QfF8MVIvc_^nSCXyb+hSWC^_T7W3phbcy&72s4K=^C+}dOX7W@8fC{ zq>12BXPUsD%Q?_|&xnhG7P>nvNn5g5VpTx9nAke7Fy$^F!TGm$D;IpkU<%XRR1<1| z?5DjtE@F;d*R35PnkAor;@aMSxGX*d`fKnVq5C>?5GL)T$ zG5Z6n?z~%dvp?N*H=vQMsma?-wBkjMA)yZJD?$JeJCDzmrQ>CM=+NK^2#;0{6h~Yu ztR-Q+dVl}lQl#hm6~QwRQT9{7r==}-a}N_8*!}@B1F6d=c%^U=5dMD=U%F#XRY@Y7>#X?CyNwbNh3b-w zJ}bp2620?sz$s4Kno!vRzlt7cp&iJ9HhmQOpA|!)a`o)C23T7jV3-FWi9C-S`k@87DSyY%ky^i!yW(uctRKa-QzHH5(G?$be_P%6&0<*27% zU{hgj)(=uN1^{C7zg$2U3I#G+kvKN+zfEnl<+dRgtU{r9q};F-X>6C$=YVY)Eq;Yr z3P?vXv`u$812M$q|D~IlktkGSP1KPbAf7)+^#MLqj1>dI{4YXGD2c+xE*LC|;*x)2 zO9A{p2ZeL2cI#PHna3ky;HDl2`yo)Mg8hWtP&8RTRxv;|Rm7%-1lPsUx5Y}Yo&3+G zmnGhQiDQOxURT!U`SZsU9<{h zorPgVYzj?+k*G*~mA!mq93|Wf3q$b(Mac1~28_RKFwrb9a0@(aMgq|)#V~oLfsdb2RDX<`Q7K#eF4MZ z5GY}zFF$*e+bj;>5b#QZR4@7B(BZc<18Q_s(YJ3z|Mw!yNz|S*;>ykRr^5N;O%*3F zFj8#>J`y$x7^{u8Ca@F9hATm#x`T!ffP2$fkOv6we=)g7A)?TEr7GZ~ zgjI!o+n$6w3&)MC$cAIy!c`3oD%2Nb1n)n!&L zkt&ou!pX#f03(SCcnVzOe%D<#@F7R;oC~M8I)IWdWxk0sAOUE;hO(+>#k>S!=_sda zoX!_G!!nrvW5gtoTlW|eO5)1s3-;UV_?^U}ZUMF-YxXD`BYTu{4G#2q9N*G&Y>cW$ zcjkAG??WC`#RLjn3i!nR>BpTGZ1~`7A9)iD476?uwi0weWw z40#83WJm}lW~9J4UFJQE)Xx#4_|%w@Kx5fY1O!&*2Hx5~6RZCNtO806Z|7^5nQ%bb>R@*Yc(1E{DKMxDE&Q7AXWQHPAal>t-( z%CrWUVew0m z{bB`X=b%YTI+70_kr2{`yiaEO=a%Hj@u5(tG1LO6Mg9o=v=op`Q)8RQzxN&!V<()| z{Jss*Hh8L|&OIJBjRGX&>vK=O|F^WMD9Y2rj<2@`-_pOxpxpeYB!yIg-$^rAYRHQt zmC#)Zs2it$D#G*Tj0gh;MCFC2zR@97L9+1uLpW%qwZauuQnb&2!8#Q+f;ca8HjE9A z#AA+3V0!%T8iZ2l1ZP2@0gTW&V0wrntkH9%RgeS}NX=-9fpM;E#Dtq>2sM;4H-JKS zMYZqF^e$QE-d3i1ZgJbQf7=5BuXp$HIf-me11GZZ6AA@LW~FEqDk5~{zjKJEgYN_-pitWdftJKa zw?J_TD0*RMH3IV+Ff>&zDVlt6Wd}v1!v1-Cocna=U>;io=)&g^OVho3q3h0q1QF-O zICr*!gg}FEO8{Jwt~5m2mS|SALBqF^iReDiCKstn8q(uY_NQoE~5FsK`1wlH9H0jb_DPloU zoEyL2l(lBo{4r~?-glX$kUY7$&pG?-z3(}S;ibd8E2v?0MxG@P33sqlxG@)2%d;eM zq%a^@I(_haC~#<%)cPOgOwQ?{+`He)jRqurZlON9 zP|t`d7R*x0HUSIC31$uRvBmPw&*`e4FYc9WAy<;Rwb&4=5Xkj3#4hK}8J67|4au7u z6tlGv2|l&!N1_m&m49)BQ^?Zj%Eg?=_c*mlBntWR>S3EQrj$vo`vVN1^3!r1F(MP8 zpETH(M^o$gU6acwAm9#DcYULz4Gzc8C=vK{DO=*IRE`ugFE~t9FLk%ml$l76d*=b# zaL&=^fQSXb>a>q=r360Hoj+}YQ#WbA^E?;s9It4YzY6-4%NDKG1>F}h8OP!#L(gp2 z-5>~vIT8N-T3pNniM^<@CwBY%3Kx%?$@+JNnpUadwLTpTGl@~mmZ!wgc z*r=hK*JvQQ63(7X-69O7hQ0HwcnNz7S#d_KUfrGZQIr8^=hR$)$K z79MFA0Jl~_l46e)5>b%Wx{`%E=@d569&AW@$i$O#GcRq})?Z-To}5_uc&xdv<{ z3Uad<^pKhpH2QShSWXlBmt*k31N-3v5rYQb*{1seO65N11%N-BJFhgJna#HOw(dCM zaxOT^0P;4IkT7iy#SVvVU5qH*2`}}%5LqZIM9nA=xTdS};B)ox^N$&u<7vEbG~yoA zkAvc7Fc}YVSg{?l4~k8OX=HX;H-aNzeX*9(YR-OOyxFJAn)B=T9$0G3wKxkU`GKBr z!i1_*NgY29;0!0&AlWylUqa+kYOr65CqP!vw+HsUfug;j?=tQ&>W|0Nx2Q776Z=-V zFe+-hr(d#t5GO=0EMHeEob&Qq@m;cR|)B1S@WbGpfQ zpuW3O6L{g&f66=n1Lw*M_AVCONlH#<$h)T=e?gE5 zhL;5u`$RYbJmSbQ(kdx2x&>?I?l|iD*iqL9YRDwt+(M{|DBV4315=jyvwGQ0(&!7H ztOndU4}eUJE#J=7Y&xhKj|_IryaO(oW0;^OgU@uOzC!%J71mi^x-|Eq-6dxCsDNKt z8QAtNqAkz`uvib|wJ|kFyL&&EQUI(S^8~l!tYRW5oN)=tJ<2LEMslU@L1F41G~6}G zuRaT=v!wMHs|lp#Z&S!5d`T35jO(eM5_TD&r>J4)&m0s@iMY>5h{Jidd6lr4hPO2F zhOhdypXoEi*vH2_h|P8@e#sZ6r?$Xm)FO4pFHlo^91hO8IWqMI9&Un~^9LyDS6>F0 z!c<3XEUu5k(dVChPzvUgpk-ru=mIdC+$6iumfusBo0-ou83hv?g8S-uu>9)AYgC{);l~ ze+=jFMbTW2<*B2M$Z6BH;+vVg61hIJJfaICN{e}n)Gz334zY$(YM4^#6#*9p*7PZM z4vFhF%#3WyEf^5&HJ~Q#;%C1mE6^mLCMnfkF#7Mz`Q(kOT(6}~2f5jd$Jn|8z{p4~ zOs?y*?%00N{aOFMI3#990+U-b=R;++lU$12VLYlDVN6N#IKo^y8QhF+Z)xbtZlsD_ z(7S(5kCfiKaP74X9n`%%b^4u%${sj(ANX!h(PPIhz(Fuv0|GhcH1bOPQfyFYQ;L^x zl8bQC(DPCN_9XQJAF@G$oWP-xOac2T;H!DA_~;{nxE-OZOEWy(6CG^zP&EJUT|&H! zrd4)Y7qv;ZXfOB40x(%bZQt>H&5QQpY~wTIZ*&t#-mO#YoEW_i02tR=GzZSE9l+9&?d@Kp!!pc zr8ZnF3q}z{BxJiyQ{yd&5{maO^frtE7K9;*QEI0?`u+62{?<7RIn}JrBAY4e^mZk4WdJu2>w%U zd1}h1K;to~YBTA<4NyPg!`Vv$`PF(M;#rlab3sGCCgD{I*^}YSE1#;x4XH;Y0PPNa zz$V_d)14N#xqFh>&s?87c7_H5c{pLi%ZgknzD4RNHZ7W(W&;-`7~ zZ})o0>K4rz22d~I;gc(tY#WZ$%7a^FbunB*^W?caL}@Ll-{Ws&*6wXW)gw~pw|JhP z?eUY%DW}ZBbbd2HvhjT(e2z@1iUBVk*O3S=uuIUu_V!U)X}}qXVVsgB($-UK{B`u9 z7|Z4bom`lhJpd=CUtBMKIlzy$h%XJnSI|S3`>6FPBsWD|mz=JjSboNiC``9*z4i=4 zyX)WZ=OP3`k{Zm0d8%};oneV%?6u(Q0kdwa7rY44Aot@$dzqcSvJ{4aiDxjPwJwS^ zP6%JQ4Teu9xun#(Li5zWlR-TUV%K%3^J0at)BEM$B#Pe#WoUi{sgz~#-pJCm?d9%~ zSDrK&s)xXG-L@Tl!Sxj;OUs-n00 z^_P=i*dX?ZhT3y5#dB+-_IaCIO>w1Z(qgAj_3ntV+A@N?dmH&yLd=?9N{lbzyZscp zJ^-vKlwX}wbNQm3!xURr`X!JNx>byaDoL{5TM!r^M6;bnx<-WXH?GZD?;y@uJ22J9 z^U{Ai3X8O))~nfeVag=gR2QLE$u;o~m5?qKN$$K|gI`#Prx5r`SZYC20L2@GH~Vqh z6lem|FM+;LF!?y3h7MvV^}V=NG6MyzI*Gp|N%7mOfsMm8LPNdJ!T!V2i0^^~s~Bh4 zUY)9cp9r`U%@SRl#oaJ_Dd?H+U`s6Glj%YQ5s++&V7;$(uS~#>Sj014W!$ zDBq?~qgw!nCbxt*77~L@jCl~a)Wx67NFr@g27(Fk9nHeSK{p)?rj%x{^n_?ZAY6c& zfRT_Z({T3jbUp!r8q%A$Focgej$M5QFsWd72nBM+S9z7L2mOd6k(xetuaLGuJ%6>} z@%9y@ST7*qVz5ia&PZy8M?o!iaEl#=9hDycp~0u|^=og}$RP!5f016}2&_3k-XE8i zOC^#riJ-xST|w=(D&PmY!n~t7@7`gEc_SSvMk{z@C5RO{sv(@QMR_xQMWb3?T8mBx zOukFHB#)SRJNL*G)wZjUsxV+=o`$``S9}2)R>Si|k_!sb>jsKBF(BL>liR$lV{yH7 zqo;)wUM8Qab+Bj#ijEP8k6(NCjDL(KTX<|%B*@DRcc5R8Su*+q^_?qs55EX8vX!pm z%UZMNUjT&8i4bEVSHhxG$A;?P>FN>Z)PBeVEF|kaBW4SDq6D#|H?nfJOpI(IN_+Wg zI|1ur49en|+^agEYK8kiv4&6v-&BW(BzeFE3uRF&#xJ6@NtcX7zBhlTJ_Hw4AZc@wZS5T~o5nRCaNr%b z300f<)&-+Lb!CTu3rSUuz;{VxorQw=dvFsGTC{e&5sG}7@7?WBo)Y)`0ovY4ye)L% z*|pXCl|(QYQaGvOxgf%s3okYDzh2^vtQT!r(c5;q;FfDz6K}~8rDEwQC>n!e1=N3e z$0H;R!;N2ooAA>idi=nEFfbu3q@Ei1&Rzb@$kvy2S8&YnD;T*p722h87}gj3J?xDd z?yFMY@-LfLFN7iGj(`X*@zPYalHgW_jX}Ns2gZx2Tz?5nQO`NG*vy%KG@Sy-{t{eR z>PvWp6Vk7ig-ERY=!}e*O$w z*GTvqy_=^vA?CNZRCMSsY69NRD+0!|k1Wh0fTy;K0#-0g70Nn(9btO7DEyJh6(A2~ z%+G~xMP;#?+HH{hsAGU!2-3;{@3`MdganWN}>*(QB^D83i4U zi}Qr>tg(>*()CdGBv2u@XFGU3sYne#ADJe0OWqBcheC%B{&86KJ~$kjDQ6--i@Hrs z!#JhiNbXMc6W>K6zV$z)fWs!azkq zPoGFnPoFC5Un|>s8AZMGoZr&uhutdX2l@G$^sfcHp#>i(PKLiB%Pt5|v&sd5v>sOi zu?D$mgN@=aU}ZW55kHOB=PdhkAd=)frUs;7u_QfoPD`P>h{j?cJ%k;HZqJUF3IFi( zy|%?TJfutp)7^_IYIo?M)t>bt4dg8M{YOX+E54Q~_~Y{On00^+ACTcOmnd5W~$j8yLfGs#>(;;tst7Sk&=QN#W3xNN7En45$PKZ(sh@Ia~xp zs-MsjRKAB~O87hb(U$UZap4RwDLcPZqlPPr8|vKm{Rprab^wZn{ibYFkno0Rf~BM5 zkaEU-e*ye|E6}&)FigjJT`=oQ#dya~9BNarvL6;6j=t5palKr8Oa}mB4MUQ>pQSO{ z5bJtb#bi9?5^HmEYy~s7T8fH3ieoy;00Ie>8LadGk^8+r2J^P;&an+Rd_Sc>=4Nw7 zQSra`kL@fha!90R=R75-?H;o?t-urH5hn)3L*Ja|L{ z2{IWol`DGrFjZDUI3qR3u<9+p5R~GV@^cV5sxbF9MpBTvRrGTYtI6AmiHW(eZ(Ea> zOT^#zXp{*-DoEZi9ywXrkxYI3!H<8%jvO#z=InAR!gK+0^U~!e!KlhOlqN9aTn@bt~3(ZLZ zQcYswH*c{##vj2@Z5 z4#{S<{9#JG;eb=`8ZlApoSbH1Ax1t%jl(kqLL4}qWSN~l>VXP^{VG8M6B#E;M+PA ziAUjZ4pYYC|BzAR)2BMvkWYx%C-?UB#&lVF6+vtoGmGF*o*33}8cWl=`B02C{6o=t zIOq`NJCI5Q0YGR!%zphHG!m|EGG4C;{h}Xc#UufQs$2$c{AFxJOrB(*HXZoB-O%H3 zQ{Goq%=XcDSMCmVP#CcSrv4 zu^f}c;n=-ChaE)HN^{$;T|rNAxF~u169wbI4wxz9cZIwdY`D!-=y-Zj)mu?kU5_{> zL7qocf%TGthV$ZqMq6}=W@%-nVSZv7L?@Gr(bmF2Hy2DA7aJe2LD^3&iP1wXag$Ae zOmef~=B?P))CSDvBq|O_A!rQ`lW|Gzag}$@?hPF;0o}%rE^tGiVDDRGgk)F|#FCsF z@nD8nzvSq4GI<+Bwn(E?tspXnMJVnz(qW2S|Ak^#AQ&AAP*n`V~ix%)c<$)wVm6YQX!v{u2Mk2kn!&O%c){der zEV$`w9iBN2i4xZhD%;kC+EvoSkN}9U6fHe4N+tFMp@~XUp(-X^koq$lF6J6qbbnV7 zfIg}6+6ywj-N9rQx=1fl1S#h>QiI`N3T|oGI)n2Yp`IaUSA~bFveU)&o@VZ(G^;A{QtNg*#{W$o|P-sKbKU3AU0L-0JEY%SeISk`60m zn*Rjb*VS%U^p5<&NX?fW%Pp?gnSwNji|k&-qa^e#fSu)1@NPyL23N z9EYDiTRVN526D~-vF?bVFO!|5O24>ww>fYDpAa;8_h=yUwLnDNOJ~2&){@029+)F} z@o1f{VZEiv7m;~wHnYkrb4565g(7zAVv438@D!F=HZ3qh;Ia>F^Z4CnusGf&UFh|Z zkgmm_gAv&7rrvN`!&vDqJfAlIO$G@Q-DRV=xbU`a43p1Li7e}2#Tyyk?& z*vHZkp4!AoU`Xqmu2&kXyxMky+ZPd;zO$tsd)=OxT#?=!*mNkbS9AS^6C4m>F;y`+ z*~;w0vWf(8c@N$v=2zSR-v zFsx#~gQkkvJ}i3lBKb7sH}^mE0V0v)<*nkSi}h9u(Xyyh5>`sNRG}-k-c6v&%3Mh? zH0M}v$?bs(hP<6xJ4}oRYmjo1qu$4YOR2AZV2YMnk(V!Vy}Dmgn(^q&i6uSizFev# z0VD+;v|>%&k~v{CRLK{(Z+rwS-WlR11C}K@ghY}roD?C`DP_KHC4eFD$+4Xf^8wTY zjE!lkKr^L*8(%c{%G?|jXqQ-7S@SIAT?~Oh1MtoV#pOjp$m@87SLYD7y(tuvFi!dC5HQ4$n*@v$TdgGBPfWU%j-fAdl4nP){Dn>y8o?igSN7fT; z<}zU$Dx3m0MGE`YVH?ki1a@}TFEWGFc(bk&ZbOAQycTzr?5i@84T4Izw#-zF)m*>) zt>VU`*d%jw`L&oKkW+~_iai?Vi!O9nW6&@C)SNZ5yj(cc$Bu9_f0?vW&S+C z;oQ#FT{oLYMTcr34Hm*_)nCo-6Q0Y`IIjX_mgY) z=%S1*I|<&5l+r9>Qwjzs(>!r+F5W!$T>g0V7f2-la~a?x(UM;PM#n{nP(}O*oDQMl z&M}(YA`PL#2l$jUHALnkk}&7#J>+mG$Odqe2F1%boar-pig2?uT=OKDii~-nhj1Q& z&#xLA<`}|lsmqQ>VR56O`o><=y6k^mQ6dRABZhnZcP=fq$E0Zm-)9)8S07xqz!0rz z0%;*W*f5n`+0Pt#d<>k+sSQ+~}lYl1)EL1Xep$r?reBE1;Aa2pnWufXp zv{!PNS`Nfn@If)%%yV5-xJqhY|AmABE(nK{k{f;H1u6<*Q1+wVb^d#J!sYuFQ-pm% zq=S5@8Dtj%eIP9c)vHbKQ;MI^hbLt9NAj(E#;S=2TF0*-0H!WDnPpc31>Xxa*-$D7 z@s8_c9t3h0GuzdEpAncoD)9S;Xf!&<5Y~G33Dp|qTmdSzP-tX_a^#jWhUb=I_exIK z+}ZRZ>H^;&NMUke8YmcT(u65P0v9C1W^7L)@cZyQ{R1)`s1dpryNBH2!g5RrFC;I) z-IneH-Z>v!<%mB7w=!qos!?>e1~jZ8B)Or(zsHf}3*2Oq#Z@I^U>chjDO{4tRF;bDre zG+^O%q?v@uIq+7Ifoes$HNr_tKpE#YU`=w_hzLiDDJWRQchI>?o;6^~T-nod1&D-B zwV_y9Y7a{fs*9317aDA)$V z_@YryUAQnFMFbZGLaY6M(-yE7!Iq23&})Z;qS^F`BNHALR+s{}Hk}Pb<~F6t zH!zp?-wBdHbaBJ$k z4G+4;?|0b^eVnC*^qu;*JjHCwHHrggt&M^}2@F~2;+STU){n)m+_VX>;v63gA0S(WKP@)<;pe&mT9l*;# z!49uMgN1OEdT961=%lq9+M*ft$QS5x2YeoAi?|&6dSVU9d;c4H;l~^#>jo5BX-D7Z z1Pvth-IssEhtFkQBC+avHK9V!;eaVZHvuqH+o15inSzkOmgx!{@UnL;s>GyVCoOSO zV(ZcL15N6)O(hUc$7N>D){Br;xgp{kD~`ke${HkquSzrvIchVd-lY;UzxWQsNSN&O zl^g`kG(i2{+_VgRaRJ)#Lg&QEl7J>Yx|pL3KCuGGPuvPuNq_?aV#5{!z?mjry8`BOGm$M*R^LwRir>1&KDed0rTnyEcvC*3Mx47>=;jlChjg501 zMxFs@_EK{n?>F~6FTUX1G!k$nPpWyLgVKHMS!ng-3({-h zM@ot{uRK>J+XIs*0_MIhEO$Of(=~ZQ;lWWvn%wskgNAciP6=nkEEuUC=&cKto0DbZ+U0$$& zWB3IF0NNB8L5;iNhg{WW`ED58q#>;HttETcD$?+QX9tL0GD?-Aw@y&`f+)^-qO5Br zV-`;>4U2@8d848TQUHtl_TTZ#ZP){+<*3)~TRdG(d{5^+ATlUf}J_7r^FQ zk84Qe*1U+(21U@X9DpI>ELQ$^mLiQY%zQq5AzM#Q9jF$%y0PSK0C8QQEAolZ6TPE| zR^WZX;?&q?n?xP5dR$e-8g)=@W~s|5{#eVK(qOYLIYf+fl>;`FV1F~iK)jgnX8eAO z&OJb4$`~!DGcA3s{XIpkg(U|ABtLQ+K)G=^uzkY04DX+VM$YsKmTi4q6VKX{RR^}o zTmS`e4h~cHQ~dh*uBs+ez7A6&y`ds6TkUat`LEG)FBTE0Oi6NlzkvHfN6Ro(AWr*w zj|z{Cm1K_Wa43h7oF}EUXc=@?f$;$Tq<~*NhKjMiDFS6oOmgTbgcNCsdnt3ZHr~7^ zkluinE6yti@4`kjK4nk@LWUQ?_dnGF(|3M?^iTm;kXZAgmNUsYgh*7Px&xe+I2;TgK4fcgpz%h;ic{@4>{){9tD$w@hd zUEsD)fYaz7LeESdWmirlL&kVkc=^!Z{&b(e0%XZ8kdA|ClU{pMU;sOhz8K{3ZE*m=kv@Q%Omes2? z+r)>)_qg&!&T4S;x-!HNa^g-*L+u#;`UX9uKQmM3MRD1al@#;7j)In$tGg1C$fpQZ zcppR8Ahk?-AljGzpQ3%BNcF{gt~P*M0f3sX@@CHiS`s)m*;lVPKqmINM(e4(u*~QU z9dG={bQ+JlJ}ZJUhyv-QiK~<|qh%B_3FFz8S`wopx48?6h{24`1Xuv-$7Fn!nJvjh zDCrEQK1Bc=c!sQI{$G`RA*;-l6Ic^A49|faKX6M~cXJ|bXPVf@8%28YJK921Oj` z10uaR++_=F4j%XJi$NpgDJHl6K7+?m$%RR{32+Aq2!J~a$c6$y`YZiYCo7+2vy3%N za@lioMcC@J1M4#D7$;_V?yOCRbxPM;Z?Hr4bGtV$maQG=Fb&iB6F`r{MyJQh+qPyg~_i` zz(t?a1~hrP=Dbg(wAXX*rk>xFuqhBVSYrWVM4*Z@P@}K}U&ptb6A(+mZ_<=9c>Qv(8GqUw{MOEAx0;md$q5C8+ z&144Gq`0E@=|d+VF(QR+9Qp!gQ2=dt3LS5`@^s?DRftIl5`CxZKz7Y5(jxSSC{WeSCxyCi5yB##~K&7~L zFvsvr5z2VXy<^NY+^E|QVKTD1lMzOqYoH-u#_F3%a@Yhni)fvSwdu``@~OXYsD z1p!>9gVCe;4=Yh@&}H5zw1wj3@z8%ns)lV!?@EE&!P#V%0)A`UKz$e&8v2UI%)Pw) zYQ7<747kGFJxnBm)|xM$S2%DtmK>a&ox&3LyGFP$_*4I&eNcxAn^HTCesi)e+a2nG z4s2eG%lkT;JQ*{`w#abyQh;V>S(RI#-uZj19(G12bWe^cQCet^Nh%sU4xeo@84oBy zorCoTF7IDe6v*mb5n$xFVn1Fd{`s0Qi_!V8()Dnb;X8}w5?&FqI$iC5+?R}Ne}j|d zo0?PZpB&@6Ri#Co>-Y)gZw=c3c7rD+GL6JX7(&U16QGt)w4_yOY40eHfK3m zMC?Q1vttb4(@m1JJUdSoGp=EI;6h$61+UwDk{cjY1^TpWSeoQrV~F2Y+8d+>^(2z_ z(!~?19=67|hJOIH)KV)=LubHhRc-pXBiI0t91UdVn61mVS%5j1M2cns*3gHF@CQ~F ze7KVFgsD|N463b&qa&7aVOTXjMdWM`XNrQj-B{_}>f>j9Cp`)B5j+SmY$!hd#Q171 zD?_1;3L7xweHp1}+}foc;_(Fsn>^D|D&p?J3@63-aX5SHNw@>nsXTSFl*I$bBX>mu zyN4{>K9|C!_b!8D-PwfXl-WnKm3-Kbk;7+o1~Yd$&>+oaONM}WNOo}r7PM1ydF|su zVX0~cbFP0X1^Yj~1TJLTr!D_{C!ObSy#EXiW+PwdmT>NGc;{f`gx;7S*(Al*iT(w? zvwp;@@!Lm^HLi41-hjL^!Gi(erUo?tmL;nHp6W_DKdd&@#NVO+}kGKskPQe`9`h# zCd0v94^JU#vSDunMfQnHUi?MhA>YI|&)#WPlc=qsU0~aKQrmGlQP|kQaKdhJ=tPc) zT**as*@p5e>B~LNz{}BkB50AqZofYmXg}Korn4(5h{)_+P7ZzwwT+38G#$^obUHBSc_ooWL};&!Zpw9~ zNJ1#2^2C7mUX5X4|F@Hk0kQO>F_kVB)FVXH50_R%cy}IXHETTn^;9M(&?y$IyB!zX z`7eMtm-jE|K|g4Ti0@)8czCry^rHSUypG6qB2iB;CL$YdErGn2yj>@iq0SS<1Wj0^ z_;es1qI4N@U^>8LZ0v>EdA&y!XamqxBX@#rWFx`M9++Q#`9)gxcxWT(fCX>i zhy4pnJy^7?>{L7d0*It@^@@SvxDeN7GAI(0@}BXypVf(4szG?!stMu@*0W0wBrsZ<}OvcoV^ zrfe?Y1D9#Q-A$PsCr5CkDBi zg;#64!E)cN?8M;*2r0mk2-xSK=kDXH9E^Eg%;WH%w-ul?@2&xV0TLh@AN%H#hUO~+ zD}^EkI9zWiZg`xk+?|p9{3{yz2GI7a*6gI$KkffyTua&;o}dKrrQ*1>;d-?mt4mfl zNprC}y8T$yGp20fxoefF1F2pFP~7KFh0Kq@OZQ9q;=3i&5fsCSo&X(M2qZ&Slz|N+ zB@B!KV>(qMBOGLRoCPIz{ro}Rb(;d%a&bApQ{U$8x#bf+p%2?mx7qC19G*0-KapXu zB~F=e(XORSKz|R5^<_AXx zv1k^lYy9)99luR+>f@u5%e%aokddRY?VdP7Zv)RKb>4sG%-zcJotoQb2n) z{DWNDX%4_LRBN&csSKPHW(y>BYxk(z4j+U#@c!5>erL0 zI)B27$cz4nP|hO(67RNSkB)S-g-tfT_a)T$u!~>m5A8gRht2T-ct@+2LpLDJm4lP z^8WT68IM{6TNq|dQS@_90N|3xGlsF{)WV;GIQ+Cn7L;>7R?eSjvisKvAc^4=VH09NE! zd}kVKr+I1Wr9uOMa=vW)WF54uSS+sUcz8;rms`8Vp3XXAA<1B9w z=WJbV?)q}*(PyN|qoCXlQ&uv13DpXQau4|}XBoieMw!qi9sfDf{`>pj`A5E(lKhZ} z?m~y!eq{AhazA(sP#=J4dUo|)(_`nFT>i8kel%_}OtGNP>Ey?__3*YQd&UeUM0GuI zxM;a8(U~i!YFoI_Tfhwn`>XXgT8+5mK@@=lakOHz()#V!EeHexnsS&2<4P^%0ZC4wlzQx@|($gJ86M37wJ=3&8$>h84Yjo7Gl^Ed5Xe9XA+49?`%YTd# zdifG9Jg?|%(K?NCI&H4ozd}B`SY&Ii3d2mX?>KOw3FtOSfDMDxIPG{U(zg!4!H<|$ zAkUQ#;Uf!x!-L$fpAUOmmD*v7d?OclrW#myvjwK}?5Bbq4AP^F*dcLCW|0lV3H`8c zK93N7DbcWENv;Hz=V@kNyDISA=BIN)WRZsk7JS@_SaL}=#opp_Z1vf+U1P*rPSvJv zcDCV7zITqMm6*ZrGJr<ChiEZxcGe5{u6#h`>%S}iULqGYHW6S! z|1lSY>c%)(O|lnK53}Cg7HMT1A}}EG?D{uefD@7N)Q^mLFvdPd2Z&5xg?gVyB-dGR z0yuMJ14z#|vu{lvw06I3+kkW3`ocm)!Ph;5M94JjVfl0%nVKkHMizEUo~cf>zkZYU zV`Rb2$eay*rf|KDizTUuH5Y& zOwG9rU9%|AmIH}G0R*@$k+P40jqU51q%DLg^i-8XzbYbgU5e#JVxD3kRtGhAjjNky ziGyh&P~@YrY{jEfZ8G+=y6HwxXifwFsNQ3)^<~UvQe0lBROHT(hFOb`-l3S-D*--N zD|9hEj{iPgzAKOF3yGSkE1Ob>cLHT-!5;sB1;-!wjQVqcT??Y~n3bVc2FeKRT2x!) z)Dxi)*e`Z)z;e)>tAU6=z1+gy(amr^g-c}Cw9?Zhy?y_(m#hf3MME9j^YV0p@D`Whggt8nTQ`gFg7N?{M*Lwn8~-aVVy0dg z%4*~W^%NN7sTb%WCeqrLQXvdygRYn~v)DCrg6=ro%%#?(M`M_{Oa+R4cRZN-&b;mN!%5`j-FA`B$to z&%>a7btPSXHQw$=hPkNKWIw4w6DDPIl>^7V8Ah-f!({>Or3!g*oS2sifeS4iG)>f1 zc?qJ-b3%ZL1{?tK;F3V-W;=u02xviGR7p^ttlGQ6m&t`31{UF6&{~d|IKzK2N^qVP zYP)!vD*KR0B_9k~mDBh@Ew)AJ;Eo6AvcEozpP2fV%~Ce}#y3!kK}Z#rpmPW2g|FRs zV}xJE3Ie#y&rKXWqMNvHqG^5Eq}b@KWJ;tekJhHYRLNJTf`?)D%WkFS3iaOtQYQnx z;BxNB&9Sj`hLP+KB)&zH>YwTZ6mqRH!%r!%a`=i5Nt5&8u_o!z1ca4)EKdYddF6Rd z?k9s}WG@ptUJCS{w6y?VcM~i0YI_v7i0NKP!rr#4Kd2pmp^8fLBF_;F)F;fWK+tS7i8Dwyg?p%OWC{GJd74>2VFIN|r>Hp@^fdT;VX2<$Gu@ zi(+PEZZnIn@r#Gy>bRmQS0y$jNFr>N4jSL&?rMM*ZEe*_G?*--2P7sh>!HRt7-ngj zmXB|+yk8<&gw!BFDWhY^bt^Xe`?FQMWbbYl9VnU3|19J;)&Q7>`MkQwcd&{R@$kG- zponzTgKj)`j(>W1dC9H2)Lr<$aK(n88c~VAbmsRTPNxXY_01nhGaPS)Rd;qs&b>0$ z3!a26gm4x(k_(Jkj*Iy5w%r5q!^#?QJ|)-`zM?-cu`EtcOCqv#SQRtYkJXTV8zO_7 z9_G5JE;xXZ*}*6zOkpURPA$FelBf7?_17!%2n71*^ zRTnSkR_;Z*LV0fBzjYm?hYMMbJWc%+s2$&zCOm}ayMA%azJ|mk-u)-l(jreHv^V7s z$k`KF@UDzQVA=QLQMx`nqq4*X0 zbTzz7GnDiQjb&zsirSl%PVbb_%@0`@nuaM=sUV`Mq)dVgAD}tpomf3qky<`WxL2=BU3qw#z3wHY2`_cWqZ)v}V@I zxE&f7{0DW5rffvP2rAC~hoM8D+q`C#g}HVqLxkeZ+a}}1i5CUtriu9Uli6EJKJ7G| z;*TrMWB)Sz6mF88G=A`UaBt99yC<$_ZjoZzJQOk_xP&(vLndGnXhcj>c4)aJQZ8on zXI(8ZU8A2COAI!b>a#s+! zFojewKtgeVC>dw=^=H-9J!J6|t9(azYy0%%p||D&Z)@}A%H}g~gMthtf&7}cedTPVg5u+Qr)d5XQuu1=&X}z=(wMIkCRPb zaDsndx$^kxdver~&kNe}4%?;>ucmt=dsIgk^>O|GSl<7aYs*k|mYsikToc9k_s8zS zw4b5Fe$<^Y{`J%Cg!0&^D z^j+`0ZsDR|-xVH8^&E+Y@GboQy>iPG>3ld~2KrgfHtr&pHu82~f0G|R*s9h7YxDMN z?df-i$Mo>MmZ`!N#xq@eHKC z{@Ix)Qxa86ltq0VS-oMc=rB>`ki4gsH&0nieFn9Kaeu$jo!{Rd zNoL;r@W9Fg>OEie`Iu4kcy_qKeo6`m9FApQUiYK{1%p3B`!J1!W{}TK?<3 zuh)`AgO_(-5B>E1CDUdG+=EeX(r-+YGCc9yl-*Z9`nFdu^!e}TZU38_?<*EK=+67dw(K_D}-=NIOSJ>Ok?Qhg_6{-;b?;D;KFbxA|WXean1_&AW4j65jHA1K)Sb-HbT zIZ;M5^7rveC*=N!Q}nOW-PYx=*FJ{-oKBMwi`dO5OiYF;Is9HwD!d=w+3d0|cHwyC zg_Ftb5K-D>HgrSr`1$9N+EhV#$Ctl8o>IQ>dE}KDK_zl*%_I1cW7PMYTaFQ*Dqc8h z{wUfn{y1;85E%{L{$uoJ@yd^dsHNYp&-(s-F|F+w{zg?M{nX7Fi6dm_0&KP^&-Q3# zs_ZRk>`t{rWW(iQ$3l~crg}(mj8ZqUkC8%VtgZisfv;cOjqhsUV0$Ho6rZ9^OvZa5HF6pMI z=I2*`CCuivTL{;#@!_P1C!I`Q|M#BJFY0U0>*%AOJKM^|VI8;^TKh&WdD5rbe)4cf zsr^}b;TZX=G409iug|@b#cur)wJv)d`4$Sb4Eff;`BC%N#DN)N-SV04@WBpcr<22h zvhefkscQRxZ+B&OpB&dbFjoFL*LBNJ*5=N3m-76dclR#AzPyWn!9AK#VY;L3+2?qT zv+ZCreSIUf%J-^MtLyag+pJd;53Ht)XJ7yQr0_#XeP_wK%JJ5M)t$fl+h2by7|sms zHhB1qdoW^*LgLdk7Xv^$l&nUR1<~VQzL{=4kA6l|Ekivk`*EF}##uMF#vbFgn@|%E zuC3>O+fLc|fcNm5tv<~ix;+{^KQl?S-_CyBc{w`u>qpsZO33yo5qAT+^#TY*X@Wi1 zRQEk~T}J!)2mTiSsNSY@{ScmPxp5w9iG6PNS2U$2s&`0gdy02-;ox{<-k-4x^i87v z>Yd}#*@w5Yq|%NhiD*_qA8qHo<-Z8$t0RwZ4+}@jLb7kyp9Fnpw0HJa-aocg{}X@y z(cd}=$xDw&w&ouyqo({`-4dE;v;j@+Bm5?B|IAH9!NJcYe|~xAhf>oOKDVO|{TDkg zR;ss`<_4JV5oH=5Oy zHIDl2fQKzZfRFuS)avh+w|``CvI?^*J5N{an*1Jl+s=HxkZI-Uc*Xo`CCU5I`_JT- zBYe&^PNSK=<-~&vp|gj6l*?I4KE0pBPk^Q)M9g^Ac3PU6mbcI)R}5&|BSQ?obNv0a z%<1r7qeo8RODEfsQ{EY3?O`N68yhc}@R}QFZe*gMG%qgpdG6tC=~4Re_eaCK#gSbv z{o6WupM+|SLTn0tjyZnMelNs5ZkqPfe;YA;AfRmf zq22V*|*hKFE~DgW_e$H7(Q^Nwfsgzo;yD09HDBqzyzgG zaOT+edzOD9&k20!Ovpx2W9`~y0e*h<3;`H<=j*Ab!86v8`}m`31TGs_jc}_ z8)tp&@G7XEm-DyU?)impBE1islO07#RnL9q3kt{moN$x+ncAP1-cGe0d})Yc{117( zd*i$41sUJ#`j|i$i@c1sD~;ub7L&Ql&nNw<-R#Kcd}r;hM0Wgs{gPp7MYOm0*YRxE z53&1{Adn+&Io{p`o3zn;zA0>eCV8i{74(lVH*XAE(%YBM^aTc@MC%XkFWY8)re4%F zejT)SEc)5?LMy^IpOWLLt6SIlzcKgb@ldbv->{BT&N+qCMky3Z9ic*YQ^%HsBxD(q z>{|+hF*7ZvY+0fxYuU5!gJGsfl#q2W#@GjAW-w+3V`lD4=l8mQ_w#%1=Xu@FbN_SC z>%7iO%=i1fuIqbUpY{EDe_o_^eY@5O&wGgX=WnZ}Mg_Hb4i!+das2)~FRM0H^$%eJ ziCxvl&;QwHuFuZkeM(o>UH!Pi{Za`1R1wi5iG7iz#lNkxE^DC)qjN<~i1Ec|OxRbG zZe{k};JkqNE_B8q&;^vYPMv`nUu`o80h2w+%Hd}ZD85nWU7_e)_IP4&x?{mmdCBdb&mj(hhrTUwwdSxH_GWdZIQ?H@JxVS~YZhhAwMr`WYn$zbrqLp%_G)eZ?x>HI0cjMRzzc-g=>J7mhOP^u5n zRkm4B;A3#lL5$>Bch`=oK2)z?$me}HQ(5WV-Y-@*_U*yeOItqzZ0yEO$7X-O0r&3c z1&p#@wAdl$sLPFh@9s48LdL~Tc`Ddgb$6NH+`uyZ#vb8uJ1RaX+jqlKLpVutWzJ}T z_e^PH9o6>DuIkkZ<*bX6%>Ov@$KLc;rM)Ac6kokg(Wr-x!z z%O0+9aJX^17D}Q(9G{zfHXrG@8ko{zH7(+PKM|9GoS6~i2v z=4t!u&34YT8!U-rHu;`8+el}n_cl*fTXHNy5#Jt`Q`Ef1g4P=Z+m4Y6EO6|J2?W7qD*1;2Cq|mUB=9mMP9X-%O zPJ^0CiweyClkwY;G1#(>ck5IIgnfi`?u_rv!P-UXhsdSzGIH@@Q;!Jl+x`16 z<3+m0W)0*ora~e|>xrzVye%$SyP{;?%HI$A{qT0E+=zHDu+dPpHvA!e-9 z&{SqCfJaIzt}{K_2V^7DBq{H)w&>9YMq(ZZtdQx_8gfQd3>HFvJHpQnL>gX9(2v*B z^U?;YzAzJhK{=q#j9ym1m|1@m)TatbBCl^`0n;%?G1ko_={fDf-36)FHy;48zMSN9 z;M4?IHMx{s?>==#KYAR+f~=2JK)8vQUafSyn&TGQJLsGRYAPXkxf$Q^w6KL^q=p_q z{ZNWLjvBLKn>nFFf+-=?1g*L8>wUlMH%SWm#6&|P2u!jh2D{>viGA}*wnER<;{x+MEzXR(Gw_mTs2133 z2Dsi$uUJ1$%S?m13h$qXN+e_h6w~BN8wE_OAS#LsRWfXUtGAwY#?XhIkNhgkL!`5xN@v*Zf1& z>ZU=`mjg;omj)c~?|cc6 z*aK<~fu%nIE8XDPeI~_14PX9sXa8S3-rOd|@p|v}YAqrtVZV&!EOyYd5lgEi6gMA8 zcVMFM2%;LS&(SR$R!MGZdc;7m3UY(ijwf|*8G*v!L6?s|YF>j`aheE7C5QST6AVkT zni$jc1eS_hp{qeJd}eOZ}Xi9l?8@ z<*+W3W7-RWr@qG|9EDv*)QJn6up_c|eDuqQox)Z(X}aM~4`<}Cjm&z2@bOy4@5YiB zDTUIScQ&|a=a8dKa2vvJ0&$W%uKT0Xs&ifSnJc=xpFO)<)cX$Wu)mLK3k@gFrWYWX zGfP7yv=-NXHVIh+Bm;pZDWbO?r5~@6h9CDBQR>JH`c~~`3JK>EtE>B|u;ZblLwL0J zP+F8t`E6vInk{e$2{}2ZXGFDBx9xGzcKpO9 z7w;^ET0kNf(z=v+)?%{L@x zVDD~TlRWkNPyb85}UzjSAmM}CG*PWXH=(`aZ2B_=26sroHNf)(Y~lLFbg zz-L$=xhnU!fmynme9l$5v06J787+3J>0!ggkq;{SgBORnb7%P+8l~d7u=LmTrHNef z$m+qUB0mfZvu~6NR{vzq;jUEf3Q=9$DP-tlX2|KBy-F=>m;l^7)6EV!x$0q$_8%w> zdnlvDeDTmJoHwdj;~qZ(cTSR%kvQ*pKAQ-~A$ks|7p`oK#U8=XS{ z3F$J0V_0Y}zS$)x(RgaIHc66#ru0FF&nW~wUUXYD{sJ% zT{dN{>_1-~PB^&V!u>=JcyUU0%~8_Df8+g$(Vnzy9K(!I@3Fn2tK2bT16UnEUmCqo zw&JS&5b5gE!MYAHg6P=czF!_a+KIV6TJ{{R9-12N*~O|NB?@8xIW!kNV> ziVkk!(JVDy*X&ko6QyN(O3k>CVYsAj^Hp;XN==Qr(X(HH1Sf};0IA%H>2eN*ZI-y; zv>%lM!RjiYSmim*nH$0|K#2f^2ZYg^p*Hy`^te`~zO<1yx%sTE5a z^{Cuy)c?18c5)Jkm6sh_K?{l0=e|n+UcY|cx4bYb)b~^C?$L0jADU2ZmR=vsJyc`} zDlcFYm%i37`>Ow1ZDBnmDYWl}QSPXJ=X|TvZh?0$KIe4zfhV^MvWYGCr&|4qnCyoW zyR}7*Ts44nZ>&yW=ot`xwwR(lwW=elFf*(m=Zrcj6cJ`O8mxZ$bWYs3E4r(qDSSi< zE)~3SlC!KGKK%XdUh(Z?vF$xN?D74wEG_2jL66?7;6+36WzD;@b?W1{Fn(A{JOQvq z9J`PUo)U=oy2Vg#3T1wKO>gE7e|udbqPhNK##|#hDqds#9k)aB{>uAE?nWCMMhfFR zLGk0>yquCX%7RZBHWlQzSn14_Yx^rGIpJ%)YNWAzf9hCWu5p4Ux3nhObEw$b)-ill z`Jp?EI>G&3)2mBaw<2*TXMm&`q+v?p`}_NKpVJ<2&Ctyk=h_YVT=d+qLpJJ@A)0h) z5%&y8p+5rN2EilaC4pxVfI}>N;_x;(bWn8MmY}iJnVWD@1|KZlz@89=gjO_!LQ~73 zy_)nVkDrFfb!egJx)9;LYZnrd!w^BlK>eE=H?|^yu3Qi0f-WMBO_i}kMGOfnx=-c z8>fqXv_h7GmMtSiC&KVn=GvL~M*alceS4$rpasOiIH1zo7K&UHqT}TKzg2Hj6I9N& z(J4Iy`P5gX&d!c&l-`FE*;e;VsB2x3B;^kfI%q_c745SC9Y1GMC-O zI`)e|ky>0%q|0iKVX9*g?`6kvHqSWw#VBfflblr-LL57+kdhrdcGi_G&F`!21uuvI zr#CaF8T^vFvc&~0s&mgC_Tuh*E2)r%Z~u;-2nB@hqIzq$U0kNRll@DWKt$g%qgOVHxz14DUhB$G zcS4*R$ef~Dq-zCt>=NX&r@PpY^%WHhh-LuIQKo6}xli;$rYH!d%_xeF!$kL+JP5(c z7Z)GsTer{)Tj}d?4yVRx)_6dJK84vrVce1cEV0&WK-@VrD++{Kg8DYe$0Isj2J|>< zSRv`~#dh(DFp8o~I3o@}f#7rtNrm5AT32RGcnktz<2icCZ9{k1l0ypR)`6Sqsn5-U zv8qjdd|J`4eA6yP8oqQ?CY+w@H6wpqBVeN0M4c2g)t(xmL9Ouv4qj|8#DduAQ_;LV z>XEI-`fOFv%o~NM(PV_MN}UO3KJ=}b?+#}zYfin>U)s|3A6&TL3Ses9{do=vQoz_5uQoI_Nvfw1**Lz~pMBs= zW9CvEsM_AOIcIdkeJ@N+J|!V3!ZH#qq!Do^M42fXkYT(ys_u^nTun0MukN8x*$waS zwG?}SVzsmaut=>K>4?u{@78rMD>Q`&U!b)GnH|&ik0|WPv{GMNthHbfVK{#x93apJ z%pst}Tc+BVLj-SVDlZzuIxg&+nYA*9wuBFT&vsvK#5Q0(s$ z!H#w*AQEZKTs4*+y;043607w7J5E_9#vnKHct~iM7zj3@DBbGj8jG_ms`i@#&DFty z;ndP>U7z{qNe4`l(p3F&&RnDGAt1bUhEq$p8--|XrN;lq4ncGAW_i43LCeH^1wdxh z#Nf&;!fme_%@r70^U@ZlX?+cN(MS)lsDH%97}07#)QM*V7|fv|bJi8ut+iT<0xX}V zo$4537(&_&08T!K^pnK+=py|N^Xd5_E=x8CWS)ZQefs00Wez*ar(}r=PHLL?zP1xN zq34`9er2WBC6}fTtC=aB`%4C-i|*OQI~=YkFJ3z8k%{O&oC;wKCTr6#-djhyHU&}B zb~XT7L|FS$kgQljnO)6TFtI1|fJeDQK#Lg>+vfde?)xCKfQg*2yLR}}21A2bF|InC zRSHaogA}w6J}c^YN7w?7gXL~fZih9>nHU@rHTPelvNb}+H$GTFq=2cBxrg4&snlg1 zzi`p(8SqKaKF$}W>U{A2ACB2o-EEI*8Dl&ca5r;M4Y*=ZVpCdqj`Drr^mF`9v+sDp zJ^uK@UXN^K_-tRt_3TT2K)iuH3dIC?TOk-sTV;!|sSh5!j_i;ElKB4MQ3m;)oJ6b zP=N3WId<*ZbGn{fZA*B#m7+s^IBoN6jXQ6Bj_f=R$7d?HY;4OWKn^>7s;91K$8=1C zQZ$oeRf-DD=I5kjW!hAu7EsFAai1>5p73hvH^kafoq>+u4{0#5i?n*mSgW1Zxvc$1(a^iY&!DL% zW?hdjAjUcp^g>C^5dKw;Cr3oGw`HN#B|IUtTUE1IdMlp5stRZ)?8|K zO@DoVfA!kh8WlDIrrQ(7YCntH-ex$z?idGQi}1z%Q`7UUL5ubFbzUt&fp;^lOKboo z>H*8Hl(L-NePN8hklLnV@?X#|Mtmxqky#8v1+Irrrt3aWC}OHxBjRTq1HZPzcma=;QOI^ck#tE(@{gtHb4gP;}R zGfgf$^!+K&Id*9#^WF*_$A2NMvIWf7z~dgwv?2aB)3(Drvwxzo&+KhMBUeP5M{o2{ z_Z_Lc|LKXk6WlG=;%hmAO$cBXzqZiZseE#%4qYAf$lxsxSRAkfZWt;{g%C`H@Hb|L zQk`wAgNI^XGso+XRp92ge9G->LgFY*HOMVcA3p3yc+&*vamBYCY;5Yg5>Eo}ld*0A zIjg7$-^@Q-Vw>+ZrG#1Dnt;={nRw@RN{n;^JquKov!(A(eoBCBb%Ll;l^wqdfFzJuPw<1`CDFnXIt^YmzVRbrlUO(aT=(dZ9pxuE$O4k>8#xo#FL9`)=nfF* zvUGCdeQbfDeMh}NzRR2lXE2k^H9|gZZ)TCZvmvPN;vq@xxghr6yl?A1JJ#!tZ9%K` z9KDHWj@G!3*6UJ}KLA!ea2qVB><`sJ&M@&=Az$oT>c`jeq6;TNxxpW5v@hB<-Mg?K z0U`I+@}r7kk~wJ~__n`3+VjqGpNTgZG?YC7SxiZ%w{UuZMja37ASyc4%H^cX6Iq@5 zqT_*tJm3&XqxwyRu4_a^8PU~QAW^4xe|rQtUsAwKp=t*O9WRWDOMu*I1>PEi%O(+n zEkeq_{kdaGC^yZneML_+`+iMoZOws3(k=lt3UGlp>7_lxN;u#qR5U(V*ONt$+>JeG zo)+5kw$SteE&i0}XwKZMHm!+Vt}Tt)cwf++POs?&+aB^Vsz;05Oex1c*zm{rH_qAE zIz5CYpEd*Hsue?rSLzW$KDDQgKWGycqCvjYQ_G1>q>GF7V%-j^K4ZRxpPpLi&=!)- zHJLzyYml4w`eT^G!uh9ijpB|1@$B?%q%k)^7rEG))^E%_P)GVr(nQXpQv8t*w6}zjd~-uru{v@s2201a|aR0B+MLFmSc>pI*z)_x`3JtS>(ma-Y<)<7_6r$Cx!S$_KzHcvO|!u?#6MGD!{;x_%<( zsi^L9Ch#nlUq)JJ&wSVNeohV(C82udr4c7g`s-WZ&Y_Fe&kPMoNbVdJJ-)!xCd|riaJsd=42lP7M=)T2}EG{%)cUbpS~_ud!#TP_;WhCc%m1HYq4g1fzA4 zd?q>HG$jM|bxxbE1<1#pQA7C;-Me=VPzu2pvnjmik(SI5)F0zKujq@%A|(BEIusq6 z1B;QC?RKsN3gzUpC%#r)%m98I$M<<1yOqG{Zyaw{^QucKAQ7!^a^tnVNZc=2HY?piXKhj;0JVZ| zZZtlw!ODu1B6*QPVr-qaw^xt3QDf!wc5NGgiMStUdOv^bN`R~-A_?^S?R9q8{#~1% zqy70U@Zhf@3K9)QpvBguuXEzgL7y5u%Iq-i%q43JJ>JMeuS~$1)4$7hGj=nxr3?l`sgJBi&i&hW1z}=r;ppZ3(5&V zG)rUc%}hn(a!v(qV>@r)K+74yJtF?2)ra>BPXu=^zg3A^_#o~?gbyMORPPsl0!r>C zMeL9m4a5OKLe+;2G}oVf77!$^BdSH!Ou@VV-fbJQaG!fCU#a%hb^Vs)#)EPYiMguw|5gBb6>>~P4Q(rgw< z{=!P{3xgPZncq-T_nSZvv7^+>Cg}F5njwpdOk`Rdu3VYgVp5_P7>n80H%T?|u7Ci^ z$lgfC(g`$7$V~%ykk~Z20{flRCUs1$uK$5RQSfg^PS}BPu0MuR4)A&aOy}8G?Y7!f z*9@Gja>K3^5S|An*hUt(xCAoSd%+4 z9Y85s+StrD0Z@*`Xz)U%5)`9Zt}+uzk6L&}&ayd_Z{y@y2o2%7^^`v2_sHisqp}_n zx&abp9Go?v4tp>gYx)TqTlMJylWHE^^_ishSip21NvEH`^J7fS2NdLWm_v1X;naL_ z-QL*tB-4;eEVmwHk41e6PBrdKU~b=3=myj(|Hg-#tH_-Q-X^8ORU{MWGQMPp&ele9 z5EM5X<5PiI>j!}+K~Y_q198LGy1PN_7aB;7OU*#poRb3Ja2<9|mrgHcvq_;>i~je3 zs;K7l0@?}1tg$FO&^HNvK8~csp}M&>P>zT?F~k;FO(5AsxZ=w8>Z_hR>ye=Xl2Gze zu+QL}W^5hcSZIU}Tc~_lRx3qBinOuEeY(@E(@DMpMSDAwZplmdeM#6O@SoyE>SI^LPMY4-{RkM! zK~!9WonxvgY-ta)+@ZHo#UbnOg_>8FObynj;Pmx{M#BQn_hQcbL6zUnh-+uOOPjIC zF}(*2^g~{|?e(VBG$412>=^T2Lj1ukgaJdMI`FQu;vNseLok3Q*#pw?IA zH~WtHH_f%0Drs*Hs_dcayo$0Zvk%^SOI`*i2vIM6|NE-EqW!C)V8~42+cskw#w?p^ zkrBGYP&PlR_rMtop zN@@ok!FJGu%rz#TomKW*lLviB60dL)0rTEYkI}W8y1Vyb>Ddpy*lE)$xi9G0aPI4~ zqzDFapjQWl{vv@HtHw|5xnq8 z2%r_YN$2)k7WlcZv9#&Byw6r_;=_*A0f^Ud?V=qy!6?b2&GW{ z(4*nwBNt+F*2nI~$|h(m&)GW91n2ZETmYe#VmDKnrqwMU>F~)35JEeA_;B24z&apE zx@G|3Y2@bJYdcz)4OktGmo+2snXq|zo2u?wCnOWVC6C~7&Nz_Pb_!p8W-J-jKxwir z#c}-gLyl?X6;-YbM$3nlbEq&9Ut^{aC4O$lKL=DtvGd)%nv`|)1QK6OU8xx^SMVRM zecLeFY*(WO!VlBNJFmE}DVQ?6=Wf;Q`gB2oun)Y`y1i+-VWoF^Iv{b zOoArRh+&^pyTP_pbin%RFYf6Gtfyl1L+j?cQed?4vjavEq*S>NOX5Ib6k$OeCB~!| zab&)SLm3?C-FX3D=n%kZLJ4S| z!x_d#6%|$9Ur2;PvbIdf=9yRyFOpKxk`{HnZ5(X8NE%K)SUWAnvvNkGLkhK4NrZ6# z$_S8y>1Azw-7OBi2AV5S(v*f%=RR-34il~ey1QP&RTIqpui9-zwN{|hh3a>p%fD}} z68@_Hkz>eE^BImx2hO%vUrv~s11PPpv?M;I*csH``V_75}xrJ4?k_N7Z{!&#Dli6+Bk!S3gkxYtS~?mnT<<%2ckbKk96U%>^Jkwe`KpwfruKjcA?)tl zPzfDAd9tf|A2G@y+QaXRqN0+L7^OFxudMueW)(Q~V4=%>@JbYXh4X>g+%pOa4w5m# zw@DK%9zAJ_{ylFrYfw0(buED4@3b8ovikVrgKI}{dpm=-<)iPlNMuhqOU0HsHYQ0+ zXx{Bb@!s;+6e{A5>c-HILj0FwUMm1F%gfv6>OK3WT`k^htmG!h`&C+(q%@U-O)XET zhit6O&`}#zrOnOv+MYia47qiS`LZLc(wDIJp&WZf*waAVEY4H?;jS=r!Qpo8>yn%1wZHy_xTMD#ToDfs*uE zDq462V78!%kz+8G3Gyyd6%pOR>ff_;BcqI{81_~YGdTeFa5d=^9n0o{Z4^AXKXRzG z15kinHZWLCES8_yC2;Si#jdu=%srA1emoYEirC14Q%k(Y{LtgID>y#z_9XTn7CW?z zCH#;^W?TpllHq%mq$o;XeW|$rw|I+ zUYLoLZVrge(tS>COfB|;kuGd@qbDW;fAmx|v$jm+kIQ#GP(vTR6h4Uc$PWGzChpYC zS!uB;w+|*kVSo1hvpQ2h;oUU_r(#IFgIXaAKR#F)cLYXtR%Om4Xl}7Ev_+sK)JaJ4 zCl2kBS#lJug(5kb-r=F-GQtd$dwOhQEi|Le4c-qs2mvfaQgs)CU? zALH;)70<~>_T0&x+Vq{|>ilk-3a88+Me3B)y8FbjJze!~Ceu|GV7imw!i{As>)_L1 z>!|KAO*!{c?gQerCgjBOqH?JRwrDDk>_nZ7jPE$~(DQ`E*egI0^jw6%h{w>*Dl79J z(wfgIw_NmP6Mw30V7f$Dqr9vsk;TQL<8ZdHI%#Wmp!ZV1un`3SQkCuEqlR@8~o!f}*bcO@{JkqpbV*x3NmyBdO& zK&SD~S|EwicuDP;xz`G|?p3D*5^+D#B&xRog5qvtLxV=F;;u*A&vr=Nzy}lyJ(sYr zx;xj`k@>^5%TKc7Yly?8Pr%ZV4DFl(8orV|k;`MZ#zv46RFGN=KS0KNdVwPqdbP9> z$=!Mne{s9=zDtUITIIjn&K@TTlh6q5+!Tja`$MPKZjY(|ZD0XmjIX@p_!D4}z*|DB zOm%!Xro+!M4fky<+dwmZxG)(;OOZ1gcv5IrB9Xm}o-vm<*9jh7n;Z4F44z7VYel-= zjM+xOKJYkn+_POfouY*|D;I&n(giR`ai<77wi98b9=!fr_^~?5mgtcPfUZ@tZ{A4u z8i2D&tKbN)BT(NPey<^KFE`PoWLb8#o);=2zRuZ#{n2LUS3wV(=tQv6L&x3%dDWYd zjT!|2A&^n*?A6tBrvXd@i>bA* z^B&UrGYj;zxPkcXi1%Hf2+^bNKttb6=B{;BAUNw{>KrCj#iK{rK(6beXz2r}Mato> z`ikbQF^~e_tk?8Ns!vUlvrs1;gn^VDY7-qN07D1c2-ey>Q|sou`h#^#`7a~Ah zWfREGOzq%nW9}eMlmOwW@;(9~=eO~}Bcl#%P!Jj&j0IY&!oH;`AX_h@CXC?@)q-U) z7`>x@*1?nMCXMUepE0@32pHhA1RW`5t3#lHH3i+DyXU*pKnx%ys-8}wQv$F+du3_E zoU-)@y6Y>s+QJeVep?ej-~_kM+JHo(cz~bx%`a{!0raOZr*$_9(~ZE&YL+tVDYBF zj+qks$6k;}NH+fPPw_CQ1(C6DaTn%KM|ln{4Mljwx2da`C`G8O($?6-Q;UZr^y_|F zK)(vCi9eT9lR?1QS(-le4Vtd=-+A1vXSShWa)5-GHR1E?e#6|peh^$At8~$~v~$ewu3YH+G{X>+r6YSGZRm-6b>Ifr_k zV1ox)-U9`}>6gmdALWq9SzEnMgbASdzEaVb7z}3{hp!J(15~WjirNs zB{??vj_&wgj-bCF&(&OeZnm^GWUMedQ)6{7Pa6fwAH=%T3lXv$>H9x=aoZA3n?|KzpK0Z=0u;euaoY#IA1 zDk=Se$lB?7yaW0(a`~EcDoO6-ogzI@m%HyLx*0?#{w>ngloN}E25t(&MM$XSH*FK) z6mFg|NXFb>%*RtN_0^>^DcqS zUw&E@BAfS?Jn&G|TbG4QZP>VL!pUspR(qeZ^j}hvI-wtj3F{a4wOEnk9sEZhPPOuj zMUHH->B=epgIyxB34DHNliXjLL9~9asV9b0#_S_{uf$_w@xAG63m_SXU>t#WtDlbV zEQjNiM*K_zXj1~(fB=4+C2!zSi91Jh?>Ckjf3J_(@;f1*YP7?ln=>Z{qw}ga={BB=D>&)`+&LZU8u&*{KA+lz%B9dD7Fu&iB z|JP`|mn0YcOu~3?y`Q+aI7r79yIVK|5=CHdx;37jk%)pkeyP4TXrG8wud> z#tB-Xvr((+#GuHT27_7`{l&wnr1KF8RdBlQRF!e|*V?*d3oUZ%-iqQ#cNA;X_MnJH zIQo6Axz1`SimY5}SHqTuZ%k$yt>LZ+vYVO17|Q#(AD#d1sb;07l_ih{z0P0G+^eL+ zs0PDORwh-tw7s^S2--YMM(Hr)z(EV|&x3TS^AR>S3rNZRPxskm1u zDVagjqb>EzZ`<8L68SqYOxTNs(h-a@GT)(Lqt2Ejlf$9I2kSH06R5?-LaETrId%8% zZ(cWjKu?6DR+I9XKj!DUY@LGV%)tMq)_W<_TZS%0l~x-i-!encHX*;yWM6DU}Da zmEoQ`st^9%OlLa+MRnI7>5J-KPMOPuNuBBH2f-rT=6r`={b;o_ZlbvOIAZ-|Hi}p0 zW(OoYW@#ywQ0pZbZ4g2XF5KiR?F%qY%G0Jdd7+6R-FIsWkvfR+21}5f0L{1}B`*PZ z8RS$><{nWE%5rppSG^A@gg~tq6K6gPro?22uWh;}=r$wmk?pU=S#*+d${Y|Y95(5h z!L1tingqR!;pVU!$&dwGNL=KXeLtnMY<}^#e*`;N~c=g{f>Y>1GCMK9}}~WwC$g zTV#bVtulW+5ctV05l8IL?<#Me{-RLP1e)#KKIeXEjxg zj&)y88TTCkd?jPY6F}HIgga8jVJ!{GZm%CzaR*W>beYE)_Q{RLa%df?PT@1#40HJ$ z3DtA}lp#Yg!L;EPQ*P~13C({WI@TP<&v@`{(<4(iuru+K0jMFk`+i{f_Bp9IgQ!k6 zmvEY-?fPXILvMUmLG!TnuIcJJXjSD}6{;#RHJ6kmT$qjgB<8TLG4nmS)dKAYU(16l zVYVYol9p;lr6X3pmZ8THe8FTwU4X+=_q_m){0Yl>QIvg2@WdT({5_jNj`qQ)Ol`++Z|fbyHnFGc*LM4&Pc=&W@OF zG?r??_?%zxu}bd-Fv%B3YlP@JmF36wffkQoP^wONsP^Iu6$=^(m(bu*_(^b{Uo#m= zM?E;sUhT*lbxnAmR?z+C0wt2};e=|No5DT=T5J>O7q{FYuQc5Uvr+(b^{LuT;gavj z%J@TNYD;gH!TP#YZ$P74Z<|Q^pGYNjIM`mi?N~MlL~eChIpfxf_dci9GjbD#{E+=j zim89g91R?bM(YwWHBT0YZUIn%!pgnbIM5yBok?cHo}tU(<{bYH+~!oXj&U~paYDb888@};KY1`_4#xHk*3sGB!;ma*Np&A}f)fjVhREsUd=Rvw-JuP*svnNq~5k zlvNX>i~xIt7s2m8{&xNR{}!dq|FVJwqx_#T<8)%lS+XWMnXrPH_+0s{6TnJLc)ss_ zIn-i_}U%VQF*y$o_i5&1}7( z&Xcvic2t}!nDe7O`e3;P4oo1^?dz@q)6(25?MHK19_3KNz$c-BEO>v0=3t|WU*joa znEd{}s*e5yhz)FpJz+}Dwyylr5l2P_O)c`e@b#+TQVc1*X+`k?V_r(yMV zIL)NV+~D4kpdQUyf5{exxiFeF>Jh{m4Vr%&v^fW=jrLQURpO*AQmpq1i?K$Y4wsd5 z3;zOcaSxcNC&a1Md#Tl??cJCBqKf8SBmJAA<)VAEmAmx|Gyd-L6jo1PD$MrWTDK=i z>^xNo^yZp<{A_P@6nMQ2tsT-uA;UuJKpPBh3h>T3u*iU(zDe<>A^ zP!^0I+ne|72Fo0*mHTSX5@FK&`}Qg)Qu=?FPxKpjvRn0mobb`VAI|^q$nGNq3>b=^ z8s;-?!#Ca1v|j6RUg&Kn`nSA}@+2V+?rQ6h23}NB&X1Cce(+W#VLwD6APf2pd94Q^Q4jQ(gVyHKem_#w}^qp9IEj_ z`Toz*LEkTc<*IcN%!DiLw~Q52b>FAUxSm6r{Cp4`g@bJNk|ApQCDI2*G9&n#!hy=I zW%Sg}r9>aUvGc$%z=1hio@}18d#`8P9tqW``iV3rl)zPvn?q7UjuZNVvN)8Kb5acU+!psqC__wFQ$XYf}XCk%Yq;X#@=zULvGUt0qv-18Um#dC|v~)uwQa6~I7A5Qy@e=%Z z=$UF9Z&TjYX1M?67K;$0SbE#?rMein;R$EZqG`LbU2J{SG;Eb3vO(t}^!BN&Tp(J^MO0`V}4eX_$dm zXVZu8fV)1*0IdrXV|&1r%H=`5IR4EV_glAKtpY%<+i{5?=1|_5paC;4sjcbQ7b<_& zk36v{&ohioqVuy<%Gvr*;>Hn(*0LF_39l-uC*Z@)0teLdfu%jhkfzAKZHbilr`+CD&f zXO&=Jl^j(=&+9S|F==L)<-fJ**I`6M3q6xm$D}~6Ke4E&sDIy#oUx=Kj(06TUp43q zwN4xy4#7P?Tq66#9DcmJa_i>bsZb+>^`(;N;~xybep&A`1_B01qvX@Il=AhdNCN}; zjLWjd{r~)4%Vnt|w1g!LND=z#6ITsXkvi1cgK?6kf0hO`y)&J^`edLc~ z=qp9V&tlsYcX9c7TB$rLsQqRZ&+GdLM$8qdUvbk7n@%X8HB(5>$fNW z$h5Ez&Ymf*H7h=``;9q}fs_3arJqF+kJLvo zuziDL-DP%19splM+u+*>0|ykM)-g5rJHX>1T?-r+9enf5pf{ExjD4z&Zj3c zU*fvb)F@?=AQyvB`MR?xZxY<>3RDDoRu*y-fQzJ&Z|qR(xeB24S}K4o?vcZtYX_PQ zIV>uj1Nf)}D%yV&Y&N`g>lVmd-wa;|pmnr`1z7jXGDb%x6|3F5$BL~LAUF)3y0N(l zG8r-T;G)EHe+V{TWz&c=Gc)wDBhu2O7C7u+Ao$C;fK}2Y^cO+F3SiK3QXV~eq+nbA z<{Q`)lha3B9>KgwSVU8hNdO#$LZxVfUvKXPw=xCM6i}rVM)7xabOb+lw}ZWPKlr-I z6a4QYBJt7DCe4zdql=`Z-=&=?D7gN!%c&5=y?8ucQUn5#9X5Y-VCU(M60-YNw6sH( z&wvokG!;0qk&%(d;&z_ie`eo`(dOEm_QUz>yzNbkOf5GpfX#Shis8uEJc`texLs(z z`0a4}m!&Z^g6&Hg$)1d(k_Yy&A+z1r?QT4&UDo|Sw4BbB=r zi`+C+umM_)kn-Z_VTt0q?YEj6{?@l{sQl#@+7XlAe!BshQh@RUn9uiYJHQ%B=!nZ22-s1sc>&`FFcKI)^dzodGW}}AJ=8_H_FZCX} zRa4+Scw=??uiHQTErPZiM5G2QaNa~Mzma!uj<|g<(h44|{V;~cw}}#{(L0NG@Yycf z?6O_=oA-7#>y(ObY1RpP-U`1b=uGDcfKHEXd$?OADd+myV~g#qEOhhh{a!1iChwRj z8b`tNC;LxdG+a7(O9RDIt79RAUZMYhT<%#<~< zdM0?*vzRvBK0isV@ z1#GMQGn+z&Udnn87-gf%BQ~!Kh2N34D7IBrgcc{AVp7q?Hr+udz;38D_0SS{S9FHC z@IFl~cm`Cdn?SZFfA_o6-zG{a%vMEV%bl#_B*IyC(58D@2f>k)xX>!lHe7Cv326$o z3}}82BUyiWQ6Q*!*|&7hz8~`0XKat~~FNb2IZFr*;8i)ufl_)7w=1MZpQ>G+IrjVgB zmrx3&5QQ?&GRr(9MTkwxJY?ABS;o$__xt|7zq8I+>#VcR`R84|_THb*Gv3dAU-xz0 zPqL*B!CY~qNvb37gbaCh6caGke{|?5&b1ClrMTQ(m`sCDo0CZ7AEG;*o~GZL z5?bicq+=AOjBWheOyJ5;ZfV0RRASbV<~D&9X*My~8^Y=j5+=Hw^cpk7-$yV0M> z!08og26Kfe4+QmQzws4Cd%4nrJ|X7qLAr`8V}?wW5rj@#<2ForzZB zyKOraWk@TA`e|g}l?c(X@9fr{vib$SK}U1#Vw@&soB2o|he{iJZe3q0TbYqLWt1*< zA5WlMp2Jk<+}gAA9m+!PG%n;nXgYgieab^Gw>Cv3DqQ=h{H>Ymt!XKTrCZY85Mg~b zr5byBtET_3Xd9oKyS(0QTx`+E%GN1^WTzrVH@YAw(G`xQp-&5LyO zaq_|V(#-~bmg;rgkoLURjrH~dwJ^Rnq`4@j$9Gy%!vfga8Y1MQx*g~OcQdD)HPy^Z z9cmC@0iJ)4`v-6N?PJO&RpJO4hnKdL{+EFs=M?rm- zrFx)vA*6Ig-j|kJhlhak$#V(QC^6dBX5?W$roC)uP3ux*C)L+SwFCO9_!x}GRMjhv zpA|nd^@_ClZhqf?@UWY+(wlhL!B0>A{anhp+)KpnVn#{)=B=91kBFR+V=rXp8$_K4 z=_4epMZl(-KC38K=>@CSmSIUa78m0#qV8JErKKKlP%pERJfup09{W9)CwP=;B~wFA2{@$^Ij;^86LMWY9zOE% zJgT|bQe0-Vjgt6l5XwWWqE#B6mhhq=j_W#;pNx+r(+$)r9wM1joyk>RIW@uOR#V%L`Z;(*`RM zzx>)r!$z@~7e}f$vE$wRR9l;^i0X#LiDr>!BC3iUCz2&MzSjoF+vC?NGREh_mBON@ zZG$KZ)nN@7Ffk_Y&E|YA zdAB=Nha2MU&pVHvsQz@TTa5fenSq>S**7=qe~+cGdGJSU4T}BjPtPc8h8NO&+P<~9 zw^1_JRe0`&foxm4Uhx%%)8XVtGT@QC*5PMwYNlw%+xOZ`bkw!!@R|LeKD?hL;@Cf0 z$-`lC*>R-&BWf*qZdGG^t`~aRvTnbC&216N$(CsEAL*^RmgMfyN*kZINHaPurKp7)SeLsZ*TS+-||v{riR7g>^_r^I@^L`M-@ z>q8x+nHEkwD64$^MMv>32bk&JCwNToDSbMo`Q#6t z71f!~Fv91yqlH;^M|LtA=ndAA9QWScbQGN(ttlcNSEc>X=2vL*K#k*EJh{ySs*JWa zPZD}izQlnh_N1k@+*?i@nLpXQH!p?!Im175Bxzsqj1Q~(6=1bf;(=^BlbuW=Ei+-0 zgDDxB2?4II4vVQAs4YX4Ku^)>ngK#QI3#0NCN9UWORO83~Q09fx7^D0D4#JWXG3epL zR}Cj@LcDisWyhnTqpvuyXt!*8C6IFUzVt_XZbK1oZ}SBUIlm`rJ*Bm;rxvy)cwTtt(h#5b;Go)GZ(4yz+udEXq#{{gCt9)NN{TXBsu!tRP)ha3U+uvGu zx+8DuS4<77M%Y;{6pZOx*>Le)+dV1u`I&rZKqVSF9$4w=QsnyUK6fOWhXGQ@U2dZ% z9LEz=bXt>x1#h=}_V;^A;Lea$SABjtK#^%_`%VHSn{4xRU;BqX*3huACaE+%V!MDp z<9qwzoZAuZt4d1YBkclmD+@XT=FuUia@|XQWVD3~v=`|FDMx8Xh;V7iyDZ*Xb#j}2 z0I%e3k8p2-tr4p;Ii&lyF%Dm+U1J@HE$<#&yr}sxE7DaRY1iQoaGq?9mzR@pT^v)a zNlnhZOb-0x;kMROxyHqO4a30#hussF`m@@zU}r>?ehQA0(-^ zPM#8J_ukKYqc)aVXx!tch$9EF_kx^W_T|>(uQkVsO~a8dW2u>6afjadE!)n@n!c5i zj>Y<~*b=*?cE@rmX|x_=v|wr%#^R;UUAgv-BSJKx;GAJmyr<`l#gDGP@;XjE5wDe& zh_9&;H7B`v(_F5lHj{)~!$5JMO|GQF;S^*2#^U_Rz7R$pa)5fHnaAwZCc1D6aSNm|U~I z4Pa1sjiz;U3<*Nrst!3rjct|QLGvDe6Cy-05j=+KmmYPy>t|+i8C-#DqavZ!6ur{t z>msS6`6^sUr^<(lR_AMwT#tH5q5GPePCBQULD<X1Yy4`g4>{%fmr)J@jgGaYx-}r`<{8b13BP3I{;fkx%BQ~c``m(j; zrS$hcJWbV2fL@HyYlh9ckh0e zu{wXrQ*kYBdh8T@?jM+StQb+!icw*e6LI=}RLSWi3VdP7`M%KC|xi5Uuw6*FpzS*30=;-T3mZ_`c(x z55$9ugxr)uCjpkqbQ(78_ZMdu1S0Uxdj~HwKIeM>o>c(1V;(+2j{xtV|IDxatwb4r z-g}W%zGVB&PBOe7crP;=wD(m}?c`%YW9|X$P(RAADHF)WUg1E)F}&g!AyH52i|uxN z66bnfu05|FADGqcmrK^eDfquE_;=O8x&*BhMOg5|hcy#-GmwoKEVxB$Qq1G+q;o?n zMKeF0xch&dCyt}H7A(4o&2dQ4v1JHY!sh<(!^UxxQ$(*L{|!ze>+aN_SIFy=4oygL zFdzI=If1vk$!6a~&+!-Fy-%SL-Bgs^Od?*h`^B$sZ&)4zU6)uL8XwABr0})xCHQ9@ZdILo zzMdvHL#1=^Q7x;H{JVB^fvt^;ezcWCCsx)d|AI9hKf9ZhCGjugPve z1R4A&U(c~l$u?UU%Yes|NkxzNI$nv7JmVr*Ks6z!k_~kC>`B&dbD+z}u?hATYabzEVpB|+X_-`9e)Q~ zG2^qc*xFr!&!Zzm7d|sPuUZG6vLwGj0*H;^HL(^U8ypN7S>?qXKWbsqg&Q9-R5D5bN1}uji{t&m$Xv+x!%XnrX z#RNOH%uI_PiV4am4nN#sh*>?V%Gk?U?Pp$1HSzO3@RH}(MEpe`2kN9~!}4-#C(7<2 znB!R-047ue#iH3fpzB4P)|QR{wEemXP!#QqA1ac>7c&1w$;^l^ZSlsL8m+>d5FV`` zsqut@ScTA*#1}w!n&XnMb~zv}@a({QcXi|=I34_!geBKqjPJe9Hq*+8+*ncI*10o- zsHgW1;zTn}qpQq#0(ESN8W z9K4=*$6KD1brbB=(H}~{kXXfw=Ue1-GDd;Lq=ZJg>!oO)IQNqbkG#qpIdJn^WZGnx z)`xdP-_*Va#S_job(dJQ;_3w8Oba71@dUK6b0?sS7(-F(9v(?at>oAz`!HD{8D-jW z@>EpKlU<}W)yvAci}%<`MR?`*X24r+);r9`63+)XjK5~WjZ%$c2)OQNU%dD)aKb{h zRk8N@1tX6%dTxj5cYX|Qy%mLaRSw5oirfviQY7AxW06d!=bo=-m|Y?xONUyoRBZ~i zT`%<6)imA5rF!0^Ci%iv!h`YOp++qq=N9dU1&dd2PXVA>nUCKJmm_x1kzKYs4xGK! zdw!xT<)Sy%ShygzT0EclYief9CX=A%`Eqr(C>@Ybwp-&D1vvN&WSgS(GB5Joyw%8x zQa*8dxn3#SVGXaG>Ox_Uv@+7YCxod?BS{t8V<`@OhVdVokqWN@xp_39J{=`}J)5Y(0 z9k3FGh-xIWhSg=Jnqi>7|JgCjg)HBG@v6a25Pra01Lc=h)XAzpBu+uV>h)06Mtzi-}3Q9Rcm+Tc&} z{LTYt62=9`?n7Z_hmRa#bN|!Vk~}e?{E6WK=1S&{M ztd3{DTKrX{gL(M8i$x@M{1@U1nEv|kWP@L7!>3Bz4c`b7+l+FZ(f}7!Kjl)ps3MV+ z5UGCUiUsYRh0#_tUBq!zF-oF~{RF1=pl9W#w%K~2p5pmt1(`tB`j^KW#qVxR2mz?g zwamQ}M|1EwLJa!7(GRx;pN^uU``|WM7wWSqL1pZ>T!cuRMuNl8Gk`EN1K52*n1vGO zWG#i(loZ>()%%WoH@}Vi%p3p_5tKtm$sXY6y$*B~1<#pHohK*F2OtO8d46pE| zMGp*1k}caN{}j3#9yrUX5ieT0Wy}0DTr7<%SK2=xRkvx36gRC4VG(zphl+Ik`Efk+ z12;Bc0(i2#1eSO2?aqRk3QD$Ol(`coodVT;552r_CG;7){!dDzke_t= z(c)-R_%oi}*nJZ3=)zcAzD>{M!0(-Mcx@~RNE!1i6<7WN8(mC7Vk5WhDj&-yPpe;9 zZh!iGjog<98zMxN%&@*>S)WE4t6N!FJ=n5sWpSdT%)^7A`|ZQY6xZch@QHsG*u{{? z>nnh$WB!P<@w8x_t7lj~A+ZycX;O2$J;%KE)rt6QG@Zk14+Z1Cefu=toqv$u$X^9OBe|E4AwM8vSd7kroNtm_E zREZE~8+9_(87F5y?NhPpDqQ&fUJAp&8v)yN{`#ZM4R=wV1uZCrY02^SU_yIP8Ssn) ze69=VoD~Vs?yK^p+_nu=Hv4$50&L5jq%VUR2TAxNa+~%d3wDEn$6))89S0;_|KP6m zYa&md7D8r#YGhoRiRITxkPH=CuHnyk=CFRA6%@|=ET0@rzdY|Fs7OW#*>t1zxBx+w039@psLhN{S0BcWSOtlC*2y2xX93UhaY&!2#jk zD6Qn0?fi(Ek&zKWx3dcMC8VUJnE7w<-Uq;kb@{z-`%u!*$ZesOlFjwJzHAB7 zeNY_T!(ze879cn3G#a(bZDn3J>&Dmjpbf>v#KbG1R_SSjf)976uCl?aCp}H5pTK-@ z5>GR!31H#9w#m9r;u(wRBIX85IgYg2%N3CQ@Ez+aTEdF|HeKm9Mo4IUbwUGFx1dw{ zg5;;P0W8i`jug`^uXQ9`-8SV#ya@-L_R-Tj zwuLP9c$8hqyq@DtIXp1H=QREk)y427{(qXjJCj)YSAuAHP;Bb6;1BxqP4=lPh-6d$W^0>bAIckn?%X3MPlTb2bebDd zl6#WN4%JbcX^VX0Iw8#ZRRydz0Hr>X6+;W^cX!T;KjV ziD%g;bL8;hw~<(i_jbaEznu=T^7tZb1M0|mKYDadC}8*;dX5RWvpj1mmrsGcys*%f zX(EGm?&3gbebsXh#ZMV|1sg(~0s|A1^X%`h)6=#I6tjoe+1clY8)>Pi?cXFOc=vh#8brl;U zMW28}ud!j|N=r-gZO{N}YHAdUp|-BzsjRA6n&{9;y>bRQakITylLvq^0zh)IU!~MF zGU@?^zj^;SHzWx~ccW0BOPd^al) z!qvy5T=xZi+lM+vZjP0YcEF}SA(UTP0`PLMrY7qo;Xv;Wp?# zPGL}GdMm9Prdu>t*6#v0JHch)8{7G@C_+NKN#OLM#d!iC{7eBuSsMl*;-f{4~0)zP6GZJEPey=6tTa$sy5$B9W9v!K!kr{00O1t#r zNxT;af?yV0OJw%Z8N^iLsFtr}li}K0R>((`ymt$t)NsoT#YPDWz>Fdx~a z+?*2H?cU+A-S=1K5e_leXd>MSE?spY8<92Ly~qDidi}0Bj%CB(>j4yd7(Bo&Q(09# zjzwrdi6oa7mIh(Zq5a{cKEC_^&Rh9^$qMO|Ym5Yb?9V*r%|N%uZYd@tZe^jhq-bKY zInH_GK63W+#D!*sBsae#4an}1v0R=7atXaO&BUO>JTL5fMB`NF7+>%epN+9;!3 zi>|*Z==JT|Mu$zYN`M8IhkvuJjB-m~O4JwX=!#cr%}L_$eiXad8|9vS!7JywH|zaN zL5L2KzWjMcD|!Cf`xh20A|rE+Q$J_t30(Su#M@2LL9folP|$`T$ZCy|v7jxz>EE8C zR^_Aj;rvJY4nbDY`26nXA8E-N!#$--5qE*C^bcsPA_^%tQDqeSG&8^?d3EzF6$#t+1W?C^qB=?M-NFzILX2f z>)h#GAtkn&DzUB^D86`}@}unqnSjpBILkIlkwXSGZAN3A!fq=>7Hf93*5tegPhDo0 z@?8+^_l{nv>fQ0#U*pN%q;R8+3d-6r4bi*D{@z-i)yq>vpMT4Bgg6j`B#gR?lPbKl zQeSgvYkE;2CrBi4AG@5OoMfr9{0GEOIY8?r&cLtDE$Y9#;4;+nXG*ft%{7mC%5c~Hf3pwVe@NqL3Au>?(0ZTwC5_c&t;S$IH`RpbKwG@h1JEQ;yK1%7H!%e z?{MjBH;N~u=u9jn;{a8oqsYHI8(X@jF69$zF^mdc8?NeX1$8<`ClvN0-%;t^8vDfk z&xMfS`H}Jm;52B&-x|{f8HLruz|Jm6^POk+{p05O&2a(_BV8&YmLZ6blC_Wxukuv@ zzr}gYEoF7AItgwAcNs+|zr8h#B<5x-+!*4b!ud7t@Q8T_nF63j@=#SQ- zDK0LqYVlJFMR&)(>@oZp^eu*xhSSY?uzuC1QNSTF>qcf54b6|~zP!VBbzmuo>m{gI zE?BUL##2yZ9ykzUfA+^FahhmGGYV8~O2guH5a<~&cvnn_FTKUfA3R8EU+%EUb#w9K z3#|=p>n?Hb+XnD|II1--Eu25oZROdMCZslJ}p@D#mJ-p7T23V^HX1!t_#+y49i`0(?X_yVGPhf>7R>pL&)w*%N#Gu4h+n* zA-!zm{SDR<*X0YL1+ukob??t|8j64m&{wrk!v-ozQ0emJGiOFSbb=5<4d6~jj-wo6 zBLo!w!2+q@3b+FL()S;Ui* zG&DLTRaogy1g%M3xbRg)tI+0&nHH8)P@auW$jRE^x<%CoTL8KJ%3gF80p>>aG_va_ zk+3pyRtU^tk&{lkTY(O;9eXL{D-|PxD>rbDkSy?roo}4wM=lUdkW%H(Kq$q{*Zai6>(y4fO*Xpexpak=;dx2>TZ4F>f zxLOi4InDZXLT|9HZ>;{#|NHb)O7+pwb>wk%vpDR}z`yU^{5t6O zXT2QcxbM1dqewQ_NHYlQ?|g6BmSd*kM|n_maq`r0<)1uts;RHn)-}ZRJ^E#SjVFVUNbNeFL*PQBXn&ou(k17U^2*kGj@sWmb%kWD$A0APxcLNW?8kfL7_M+?D=LIqjA=(m zq&bdRl|A-Y89PmVAUa&2HSq-pfVX*KYucN%bc3>VxXOW{a7qC%RuqBgWA*FRX`|Ud z6p6dpsQonvhIg>meC>~DyrA1kl89roed7ro9)a7b;M}f!#xY7vASJKrQtsMx^tYce zL4FcMC4c4=YyIHU8IQLEpIjIdxc6cI$?sXFP z#a9yba#Z8wlhviEnG)UI)@K#`QZ!s$K@_2)m!Gkc9jvo#WNl4Na+*Mf6s67mnE)vR zfLxrgUfp#d`db)mROG#j%z~UUG-+4U0wscC6QZxWiP$kDica z3XmgC{)GKaJTFdRVxtQ2Cv99nhK!nd!&+G_>*B@vpC)pl zYT2QCjSv#(7<3Gw_P-yaM^|3A=c|Z3HO&e+Mdn^_Ji$~ljnT2*nH)qC+Gsf*9#GK}nz7<6>a9~9rF^oaIca`J3bKn)YT^G%!n z|L}?bpZzjW#fqy7?DPs-9Jbpy_CIPCE@fK%BdZ?BO10hgkNy3;FR~r9`v5a+xy<;T_qhuuHH$Sy_MlZOzNhtrVUmaRv2$#y^3N-6&eRGDq# zrw<+K#U)np8hAoJ@_uKpU%!q51JcIRvLv#`b7XTkj+{MvmYw}UW$z0Toz~#vbrHmY zw#@is0FcOq3l|Wfe;^+EAbA-i==0&KjL#7Fm& zzOYMM%;kfp=O#HaWhChmUQBcF+-_FZ+alWz9_Hmm;D(o5fGKhc(bU8&UR|*K{~le zKLNxphm?>vV@{BrwR-kHU!kLD3^`+Rnf$0^q|baGJ$iIRCcsu%t8qUgqpUH81}!XS zp<$CaxNl$8b}zG?yLR26#@drTiX@(GZ~0~fT;IO~m+3u^IEnmZ&1l|U8oZoS_1$?@ z)#t=RwueaBP+4*OqTJ8Sru-ELy5fwkU|^}q?4& z0Phvn693C;Sjd%J&*b|P;py`Y(MyP4{4-Ge>bkq4kZsq3-Xma zyekw`NkcY@wB~QjUh2I?id1qBWsE)Re`T9N4orKQv3N5n!+vN1QClCKTsM&WO`FsM zD>uCXFowzAfY0m0)YRY4%r>^s9DxjUbeG7TzhFFV0XTkcX{pRj z0Z7D^l3r>I7$<281UPqHJ;lMn!NW7TcnZmc_es3A@#W>`e~^}M3&Q2A6N`?G?(Xi! z#>OZ$BosoSOlC329>pxSPk6HfL`87UAG%Fhn*6d$#?wwm6+&M99vjogVsTy=4GY+M zoJ0ik)VwYql%Jv=SG%#kx^M4ZG()R@V8CN_A|F9=SG#(mDmzc<=1rS6U~<)X44WGw z{GK%BTIOY>=s=wW5}nVk+E>JYjFY;cUhb7V;xzWdzi(TkphZg}z^v|EJFJ}i+l>*T zp`=OK_)3NmZ2ux-Sn38IFf#HrH0v8h)-LBgw`_g6!Myt&9y}=D>|s-X`D-wDf1~7B z*?e=;W&exvq1v|+lR~%Dph!$@$f=I9@$$c5oO<3kQ`vhn-H_lz4R+7i4liPocGR@YZaO-(9FR-t$!mm^zHspF0LCtd6DEGpP|S@A%UXbR&km_`4vhr zq0o_1q?j-}dtX!_?g^`1eu4MsUwc#`KHi-MGP!eZo|T&H<76!joXFZr!Q35vy?{0H zc_xaDy?$imyCE}@tfiHoUg)Tir0#U@@N3Ativg5ztbTvAW8b+L1ckEuy({8CCqjhik^X3av8P9hkG(mo|zz*Jt7p179|AntXOlK})jK481ucqh=fp+&` ztK(wo+}ZL{=y{35cb6`GI%`_{9kAqic}8Pflp;($UDgsUovbE>yiov4uy_$sF(@|M zOu#-K9`}zlAOfdLLEl$2yT}Cdn+h;I--fg&|86Xm0XJ$@JxmIC@ho=z{N(G>V1_;V zXQDk$D`mcNXL|E~j=`_cp8Y6v1ee#WG5NM3+>sI&rJ9R+upl`Y>$bL~A0cmqZ0=k} z5_|eI=Pn*VJ*_dmTsk3vfk+ZP+{UHb>=6IT+t;#{Yke&{-B9D>9R%tg>!u?6)YDv+ zCi9~>PhhzF`uc|YF+w^r#=IvbuCjTge&9{hPteT$|4q$WkHZvOY_V!EX-cB2i!nIEA*){vx zAI5^P-|H#js2<8Q`u&0{m7^(+OQ`yvx1<}2l9A{XlSdW5K&4+@oSh@g6K`i5uqz>= z)=&X=5VQu1R>g#ho`wiQg_nqf>~ZB0-0k*@&t`d6aIjYC%ZnIn#fJ~+=wf@XnSMon z0Wql`16U~WQWyvZgpbKKWm*|mN1LQ#WVm_1J23e^@q-!{8c0HXlA8;`ytp;p5NvL6 z{>QW@e#MO3lB*3>F*c^6foHm(XBV;Eu||s4)c=W^L(bm6%q(#Kaa9+`Hp&#m16rwx zvx69A*888vcW!}$s>=A*tb7y5e>?V(IOv?pXDnWUWnASu<}U>P3)7i1;ldgk2~d_` zE2Zip{7=@_da@T8ykA+!1egy$e*E}Vdb#aHI8+vWRp7Uvpwrw4?qzJ|EV=<1Y!iBIIcN(ZLn;b&1~2a(f!s}4vfg^#x@g~t*JFnxs% zY6G9)xmQ2!nD|3UYcX&cy!xp!op?p~$A(3IJ7G0@sX2GiFfgYGTIV?D?mwjt_6^}& z*V;1X?UcU7sOIwO8|4AJ+ubcW=1hW$E{ln-_ZP-V+1ikkG_Ic&LitVo5Xg7rB{`Uv ztmlb1wOH!};Ea#N>3SuUC&^e!`m)8KeDG--5r1~eABnnrpdd&;r8{UP;!k;i@A8q$ zBO_{HP3ssrE;3uy(D)xrl6y(FH`;w2K##JU6IK{M9R;PIpL&{Jq3s25LZEFkf7YI- z5_acR#U2MWw~msdQvK zI5?^WGK%EimO{tw?)``;-=P40rU~F1f?8*1rin$nrgeLwxkkjf_kj{;f58^iCG=H+ zx5w(HsUr41Ug=}I(7(OgJu_p`bkSuAy`$$rZ!9_0F4l!w&fNq+`9=sF3n{SW%Qpp| zOqv@8VX}0kHq(;6nZr`sZN?k;2=d;#>RTyBRvZaonAiy3qhB{pDonKFu11stPFAaO zU}Bi7J8jW?sjHCq#s*plZkfPj>7DBfp+3|aOOqs466dy%I6zJrQCX&h{^}f4q(R^y zEw&Zq57k5Boa23i|JFNm!u&-3%9Mwaj2)XoNTPx8U~Sy`Dy-F5-ZnW23Ghr-7ZExJ zb0!dY-jbj1Sle0Z$dNnT0MIe|YnnF_v&4BwEd( ziPu3lgd#*v9J@R>)U;o{y2og47>H)cpJ+5R`rWY)AU$1VmU0E-?3ebkp3S-<33!Z*PykkeG_G^h({&z7O2t?lV? zpp&AYX`y6;_@=S!p-g{k>FQEj`2!LW&r|c-qrL~lOOZfP64m1E3(8KHexE9{BysX~ zJ$d}yb2F1b?OeyhSPJTSVgeKYmyViO7-)uPyq#ryW}r5#^!%N#<%T7Dbis??AyN6Q zDhzwfxi&Tr?0xjydncB1w02sKQ`+Lhz^ChIs|Z!q`WQhb+YFN$V$)uOW?xY)aW?<5 zo1pe~WJn1II3XeTZpOohSU%Asw}MZltU;_v{}(pZ)?87pP-VDrY%F7;=kKg;&F!_)JyMf~+QQU`l{PSWvV3M&LAeK?}ZFf6%e3(eMuglS7(kA-7iU>u(=92*nJ&G#JTm= zvAWYZVa}RD;xp5p)6@{rUS@RQ?CXH*vIfXq=*;x%Wnm*Cc0w#ZSyRIX@y6-WTOz*4 zofovA$((Cc8n!J4NysJ=cLBL{Jmm23@MlK8w0Uir{8BpJ%I7zC?0~Eq#un-pnHg0m zzn1UNM#W#(#ZeS?He9IDfsSGN(>{1+z*Sj1D$v1&>A#KUc@9a@(PZUuB>X@4jYb{< z5mhg4)f89~j==y*j4cs4_L9l(;ut=#K$x+Hd@ z1M)Nyjks5t&WHbTU>h`Kzd+KR`kobMrmPw!qD3qIJM74D+C;g1AQB>XI1Fd*ZZvOf z#b9mT-Y&oxn0`QMu30 z;0;^;c#R+IWJtber4m39E`RR~k5ef13KjjpkIZbLpK5Ac(*Pe?q-#iB3 zK$AhM=X#=d<&jfGS6|K!9)&|i5FY2Svyh0G>uF6r!N!($)tbb>{lko>2ONYJi$pH3 zdLuUfJnKf&izB3`UwTv<3;Xevc-Uj=Q^$^S9VjRpnXl zIvp!N(QYt>tTMwzW7)JGcMTjQySr;%QdPZ|dizpHtFns3XWxxEv!U~85xMiI@|tH2 zozGYsVhSn-!3e1!hn1Y}`!W&|xBTgfQm{g`K>s&-2K_t4SjC2ztt{2+*qk|2%vQO z^PxFTqvBz_edpJj8i$9!={OG=8IeGi^oNztxz3z{`q)aucS%kx;d?N5yuH{5KC@SR z33U;sb+D@sfB*TzX>&C~*yMlO^NBW9S(JTfVGP+Dt(17ZTryib+q%s5g18G8$0Y%a zM71M0zH5Vzn)hEq{IvI|RgphaoO;~+cPTl_@lIW67}SoyzPqnmF;miL{zx5dk>l1L zX_B#K$P9R@QRWT<5X=QfQSz@<$MVlJ;)}0GTaX`wfR&Qw+(yD7;#5$`m7kq@4moGd zoO)+GmZg!^b9RJx8ZEsrTPNZuZ&(BcFd;#qxw`~XI}GmHTZX- z^=7Faw}j?6c)+zGtzTZ``JX7^o8UQjrKu@E?_iK)N#*AWD^z)E1NHa{y zG;v%juy*Ol(<|#<{60Ha8`>!NvCQSJR#MRX1pGb<>dAAx*w_VX@V@46LHa{V8!q+e z+)AAAJYC8uDJh4li%5e9bHj2Y9Xs{|YR8~LCoLG&5Imj1Yk99*HWvnLEifs31785Edhn%fFl3o-^*Dfp& zO^V)}jzYxLDl}A|g5c=OJ3;G~l+X~K+_ZyG&BVzknQO=;Up1`uBX-!t(RGIf>^}HS z7g9f4dO9D?Cr=?Qn_p@U~FYi^)&=ctJ|0Fy%GxK#^ zBQF(+D63T5&7nU#q@?GfmmTjz)0&cD(Z;ilqJ7kNABX#w#b=eh1$%tpA*HpmaKDan z;i;#TD^Bd3>`xdWG14{SIqy#LAOA?Zd2)o0c-{!D<$!GBJ9um>0ljL)YX*zeOOcV`N?*9ve&u2)RPPSF)}!S24_CAz3P5 zd@2K~gpc;L&Tr?2kN#3#3qRrupWK;wydz;@?B6M39@|mo0Ox+IH@g@cT+(@+t30lc+=r<>{EsTAY1ZAH~Q#IXS$-VpGw!x{4i4Kitbi3an zN-uJ1FA}0ZUB&vgxUSrFn6${)ZmxH;EBHi|PQLtbP z19Ij}m<)fLr=d6#dn??PK4^jXjHco+?R}}%lVQF|eD7S1%%qW{Yx8r8! zbrE+1cxB%X0A4vF6U2%rTR(qv+IW4&cmJuC{9$Su?coN%8*WPxw>zK5&JA1hNv$ko zM;w5HAeM;^(DvUEM$>hyvRD)Rh;xiRt_Zc&6~sd<;vFM_Z_F(^P+!G zo;BDzqaAIy;o`0hi$rRhQGQ8Hbd2>8#;;gi?8iPQz1*}JF55faSowZ4hJ<=~z+x;S ziO!C#DG6??;8BA#WYKik+b3E3)L^5>(hdvfCGD=l(Prs|QR3*|H(qIQEXCj>CK$rAzZ(gNAzX0NNrxbdbaSZD=<+aFDG$9CrmG;O?)Yj!#h{&Q zfvpeuo0Tp#>v$}ibbn*#RtZs&9A3?gbf_DGxp8F2D?*gm<8fM!NI?2B*CoF;RA;I? zk3EZ=#p6Iu92Eqz<7-0z%#8L(5>d`R#X8^MBMuP^kuQzhiQ@m+O@Rwjb=TPCkia6R z!8(c$F*HScZ>(wP=i%6rWfnyD5q0>o39<}U>|#lt8JJ>Rt|;5S6A|Vpjw3h@<=dsW zATd?vBB`q$4BY?rDi#cm(O1OXlLMc24AxtcTnF;O`H8*?ba$MF4Ut+a^R*%MI1RN^ zU*GP0#}`|RYok9?({tbapJPoi?3HAhHG9%i*m}{sqxn7jByD#9-GQWNX(itezMlfi zC~5IDn!Ro0+<@&Koaa>3j%k(O%gP$wHW_&kf0-c?i2Wk#?E^_GEmVxX*E(d3*NVI- z=^0-q?<3X8JdWuONuj4WZTIJcFCB#(1lS#d1oqB~#ffxBC)noVgK$eYcU`{sZM#=2 z6pc=4=oox4k_Uy3k-e6SQjYV9a6*TtkDWP#jBVqIU04$34`LOr9@1@U!sr66CdsQd zF2g68wv}ry?XumKk!AjpLo4})h;Ib{twVBSKf)UN+V`H8=_<(V13KTkpGwLCHBelf z()3J4e1&v#<*T1^qpkwyPbgC z;yQAN?QMCs>~p|=ZBM~PO*GW!)^V<02sXUnS}bTaom2TeIXK`T!U;HT$z4Top!M>g z8Arm+r}8;Pt|h2DD0oGsmX7rOr5*@K;s!iI(4eDhlJ-oo3JEoA5M`1aVeg$d zQHZ-~zQ_0~M_eb`p9TGYsth3oDf?O;L6%KcM4Z%%#V>wk*G3>h2n5-ZOAnIu`|Yn{ zFQHC{k;JiMT@y(OIUbS>RTP`b2NBl@D!t*d)Q*33t}|AyHjF^f7VAO9Na!?uY0)+L z!$$#riuXujBKkBCk`AB(HWk4~*4j~v7tj3;AzQ>AliLy(AtCXeld;S8SRYzySN~J3ha@1X(Ld4wn8H z0T&vq!=v;TaoR9II$-=4w-JO762BhT0-fkyl><%NG_I?+-38|R4i^Wgg{{r^(v>uPP$u24ypc+rEU)LNf`(l{42L@fa-@?xT!TNMLh^4w$55t^Nr7S@9Vvt)0SV8~yWl zbd_Omw1nGyw~L7VX9v0^+++(W;Q9(~>cs#Rc76m<0y{n&g{bi);BM)s{AL0FDe5|R zY6P*$I^m_X+R-~%;T;QH-9{i8NGS-l!iHSWf89811$ zUmscT7_k}33OWi=S+O@86<&~2kr1Gw_)2fBvQj8ES*mmX+(Y`px&y$CAA=AH0b)@P z=Axo)n;m=$mM0J(Ci1V{%iOU-6C~Uai@rDluwb*hJ6L1y--E4D6A-3sVuKnySH-=3 zMr+v2K|3kfN9DyLw)O$A9~`Zj>wtgizlCA$L7pI2W}$xY0K`Duti;RI6J3@f{_6+D z5+TB&i;YvDXS^kXi-=^wVE(1#OM|buEZWj1xA;BNL>lCWl93ze9#6qc^k*WDsGZUu z+A($j$*L+JU`=If(c{0A5-`i=mw^1G=zy9AepsR7w}OKFpI5*Dl9aUWs%;OvVem1! zh7}pLa7iPj@Yl+7n~***34{ZEN;leb6G=CCzWvP6ueY~sK|z(papN;m$$)1N&{lfq zc>g}ypk&c)a7ypiA6tU?WZQHf{!i(R#qu4}-ktQTyrzPaqW_ZH?2}_Ef-Kd!(DK`q zHszReX`+SlA%TuyqyL263^;rHtzN!1)Nsx%4s>m}uq<2vZbFXtDpYCU z0;1TD;(=U?xOgQ;hzAutMDGK$PtxTJ=8BW@onILv2@S<4y(=C-ueLCXiJEA~IxyGI zOVkqpW{zYqb`?G6JNQAx_W^%_TC=DteL4Kwz5_{SQCe4C=iP37D-6aZP@Zo%!o~xT zV$euH`87l^%esEE*SA%+s zLIe<4Jvsgsn{3m$&ERDBRnac=`G$+@2Z}={PXxf&gLRJ3<{`EK{5v$=nND~tvsx-ybnAC@#o0{ozI+1NJ7ZU7lYaC*B6flFO7 za*IP+u_ei(r`v%}CEqz0m4D`mPy~AK9LucHEGuOaP`CZSLc_`eVI|aDdO5N(fm|BL zQ2zoV&x8OL8n|PmG~a#XX^NZx2lzAgEqlCxpw4Y&WOXXn_C3Vv5uYtTeY!QIh}9TfwQCIXlwurJ>Bne9gK$o zm3Kwb-qW{&j|UDlXl<+^DfF!=#^RTHX68yCB}vchpO0`>Ki`6I48DC=5k1Q(x#iiU zTWWEuHSKZSI*aneEn5{p7Cs^az~$f6nfUBuE*;RmoPX~zD}!^@OU1dX0R>16*!4oA97LZ!8eve!zK=rxZoAvwG$KjTMg)7>7F_ONaTEHA!BLf3y(dXpr-z3$My z#q2wN3g7O0Qh7}=mTyWUq3lzYPtv6wZgc+q{fcIVp+B!6l~h@YniQy&ly>F)VQEFP%B^Hl%_~(#1L>Lo zSllQHoW$NcEq}`zf+3}GCpre?;)@rp#`0)cTlT5oJ(T+o9)u9alyCyzJ$E#3!E?p1 zmV`jKjlC7X9>BZ(IE>r~@k$U!9k|9wsxHr(uQ(u6hi%8|ynR%}vxneTeyHEb z)(EVvpx41z8XHe&-FN^grRk&M)I?_z4GX_{5Y>Lom&cIAZCK`>&b34)1=aFY+3QIf zXn^~ew3tAVfnR+$Dk>^S%s0>9wta&6nRhe71yc5~^&I6&zFQ3pK?xG~%hV9O=O>)A zyExWnSw#Vo#Y6{rTN7TYaGm^iTV->b!^rpd{nbxxdP>E`#5&0mCA~F8%JUZ6wy{V$ zTC9(@Ry}2CTUmgp)Z}xb%iZM_U~%45zc%y}pWhkQ23KSY9pkN{J-gp=>o_covS3+^ z{ITn3%QQhu9E73?aq^=*Mp`LcX3B(&0}cN(+1B48M1|~!zM+=n?2KB9Hco_~lL`>& z|8?(ZBO68rp4d^(k9U%8$RW815_ndu{HyO(w4(F$<&I>^kUZzE+a@4grRn9)&dhvt ziwO+0)I|&L^5VKyIHoO%k5eMJ}_TE7wuYpBnXJ=>Ko^yufpU9J(oFfqRkx74{ zQpUDhA+So(Oj1YTOC`z&$3dXF`3T!+ZJ&2V>roC4k|YoJ?BKaoAZsE9JxS%2LgEfV z{TmkekBmtZAGQxDEUooXl9C)8v3L~e%j6--;@p*f8xMvEOekVYBsgTat6SRG)*Ma%LE1nVl-T_>cU;LjTkyKB-A2>E*upqQ!a71-Z5T4Gk?NF{FTLyGZ`V^Ai>QG%PQa;GvPDQnsWM|g4 z;sf-#4C1&jJ~KgZ5Rc{oQJ zTJql-AhqK$D472JQU(0Z3@Kxb*k6RN!Wc1bNF8NmAESFBB#3QV1-60@RY!-8R6OqO zy|!O`>n6`4j8Yxwpb&0--l1ju4~h2I8-?*P#rQxZhA5+cL@p#Z> zsAN}HR+#*@+SHIn`Ok3@O;!(MKQ~xMx?~vF$_SY^-+-i711TdGs;eAaTo#bg=1<=4 z$eY0W1s74V*k%0ZnYyfF%5QHSL;lE?S|Y?Wtf99A_jHpaeG+afkeA>1_FAnO8dXua zDFT+-gr;8l=xBs+C-1c-UIaNaDhgMhy^7Vjdf&7qQSI{o!PZ9;iJ(05ioRH#zu@3AYY7(Xm|J2^Q?6@1uX zh7vl+^PF#Txx_#!1&O&rn=dqexXGV=au~B_QJh&LJ)>g&EjJ~_78SfNPwkF(ovD=y>a62GgP(*#>M-<&jI!`f zv4}XV+WYtKuXofj+F<>;!+R$jx{pH#sTzNetrC^^P(Y6BEa{;l_Sv)F&ppNkc4BV+ z4zUwD@}@d$s*^JMbPrlAUFX%+Rg9jID7fxBP;Ca;L{I-19VH8@^N5hP9)h?D4e~Zz zbGde62Auv2JH*9{7jM8&QiaXv5=!T7J9qEcank_r=;GhdVBsboOI--ys#FqZZ-U_U zb1Vflj&RbI;08Se(#P8vp1SxeMvIcUW-V2V``M$AdM?clLz(z85GR+-Yf*Mq3^{y# zk7&eUsSGu185Z8{^*Qw#30aQnNw(c8m7KOi#C3bUJrj-tH9t{&7_RnXr0T`D@lK~x zby4z1bz62S4bB_laXsI`E0BvEOHfE@5=f$Ly*c0C8#6Q5IC)Ig>1&13qn*`hW9xe&jKJ%Dh&_7!pP~XluB%J!Y2^+A7r~Xm zmd+-T%>cGFgx)+ya08jQVeSRD-QB!dg!K_2$I-Z|U|g~-9haP%LKTF9g7W(5j=E%h zq$1D?A&$<^x)gk^Ee$DHr2MjNH%R;cl8KS~!O!Oy89!sB^T+Ce2bVtLw`|~(U(HZh zzHHS3wq4jD(WaB{h{*-8JpgaU^DVueaES2@e?O6=YtcHSHZ>fN4g`qmBnb`kGziqf zPU7DhQ~y*32)3bngn0$YVeunOK>S!yt-~n=^ljGx*jtg@7FwLUEn5=B6|psCxyd?7hT89ZvV6XbgXn=@kC0DYZH3JCxbme? zwLEiJNS-rYmjw@*#oy@H~uXUtNE znFZAjyU)MPcb&f2Fx9(2{8URuz0QTekmzXhx_>-Yyahad-@f~uZFffPQ^>#mQ_84? zH|A+yqM#P>ECpRRvN}B{ohSS`!===Dz24dNXV^wNjk!!dU6?4{TDa=eZcz8^Aj(3m zQut@;XDA3L{^9=It+7qM{wYDc@cHZvo<6-_MHmsHpnl z@3>YFBAia(RWxWz7xHwQqa`%P^AE>IgQDBP1_L7HhA=VA2xK2S_BR#e61LH6!j2l~ zxz3E#AB0*iLK-;>oRkXlkp8SL0p8b$5W1ivuecQ+%mUT-IPJ~Y#7&|jROgudtR&iSmUtHVxM@eX){Xa zytiSXm*#+cZ2(F9@9ix>lU(ozW4$z|;x^wHkMbr^%e`MfEkD``dEWk`FMdu!!puAO$+LTzr$Qn%?v6+?DjoA{6Ws$+s(f766l^b^)h z1COZyVX)S8y7o@i8zE-g3f4dw=Z5VocJCFqL2?|AFw<_SC2K~>!9^o0f8Ajw4sJ=X zCdFU(Rfd?#&;8R7Gf`lo-15a=S=Yd;ZE4A>IxX?xFR;K(Y|l*Mlu9H^C7IVIrkN#? zrG%@~42Yf6GwDkU*#ng9+btPyVv>RclXi1k2hNXL;3h(k>xP(t7Y-ICgZrWNcyHHT zTWk3ivv6unKZ}r#qE5c**4En{-^0ZQ13J6hDLa@mBnfGqXfjDk9J+;{UR@Th4415( zug|p)G4-4eqm7GEZn!V#gqZ2)7@1X>!kLA-M$wXT&*IlsTs74W$+U^&$6?RE-k25Y z+?`qVB9-#mX(Ee{DwLUzim6MXMJCn5jTnIzvaQbDyDk0IfN(S)?P&4mRF>tEcdKBT ztP~Wk)X5Pok00u217?i7*eiI9mv{7sZ;;n=b%-`e!o#JZPI1M^al|ygGd>Vh7$6=D z;j@Ftb3o7IkEM^Yh+8${b>QUTKC*3BVuBV%E96)zpH<}^DoS%RMw(K8_TulnR+~wQ zl|d7CSEw|qg|cRl!20x|j70astNgH;N~#V``hH{7mpAd#=>*G0anp}})BQnf_YwY0 zb-WV25i-|q6=K?7M^M&u$)S}k2xqfxt4^znNDbgpo?0wk1EbPzQ}Ox6{riy6Ircvx zR96Otq@*y4%x$a__wTv6oKbl55x=&WMLSQpjKq7Lg0;NhgWiPYB2P`d2z&wB7D6?P z;tdkYW*caR#7y}N#{_=uZXz+TcfIkwM$69QH^RhpN&MRW2eeO|2r&Zz1kv!^Y@@P! zT#;y^Vf3tif0cHUPMgj8OmlNGA^2!;hJETy&6$D9!3?$qVdfCC+DNNd6$`U!^H_CT zxC+y{JS$?H_xlKGS=$Di>gSuARo+}aZLJORF~s!z-`~f(3M}CujB3??4(eol)J)0Z zmNdD58hUH0dbHXH*2j+yPGzsdd&s-nO>AW@lwbRhl0YYq+^Hk#xUm{BOg@^V^+!H9 zkMA=ydgIROqZ>?3OVl#VXruRWKC!<$7nGL~0;fmc^v8OiyS>q71-fM^AX;8qg{sjA_Pbw`X z&A%Y2J$F^9Bu&p~oW55Tl1WgGu#L>Q_H&^2EtQmPm|cExb!u-vB?bDxwsb%kj~?QP zgov0A=hM(F=sSK^ud_9O;P0GYQ1`M_`k6p2l0RZ`v+0H1&2c^I$<>)bu)?sYh4UmTC7 zt&lBcyxstm$xH~Byk7nU0A;k-ZTDnq>%Z|`v%}#EIJl-4T5XHlLIwp!tM(JV8~F(j zQxxaw&n?V6?dqwOg0CHq`(@He)B5EJ=o}^P7&XX>s+!l-mv|q&TIwXFU@+VO(Aje> zmlouiZ3Cf$=UgyMAG5nh+AIj}0!8`YqY7NJgtzW&S14MC;VeKbtJb5iqYCLA{QBB$ z4)i(R1k77U{wXhiZ&D%7RebReOz**JB@`PePfc06`(yu}(<>C^Qox=XeB7`hB{`i+6qFQwzOE zGbl~s&w4Lq6#_O7>bdJGq@&lDvLo&LAlA#W((WKpLv_17) zuH6xVFY_&zFgPEyP~$G7!fICacE1pUxB~&vM}kn?LE?uPY3t6eRpf!n=l*&h>Z1JQ z!Pfm;5g!=GU6P*XrE5Ri{(19;locT>c*Iiki{dKX;>n6W!+JK{rY{?Z-$2bb)FUZT9B%&&yN>6 zsk+bet=FRp$6$5*dGzOn85)}@W;@gwb&=O&E;J15?PPsXOuS9Bf=lYzt^HNOHRqy< z$-B9MWU`7h)r5V}m8z{kQx+^j;%cUZ6{xh?6x${u@H7rF*wADYUOftfAsSQ!ST=un>x;G>)BOND5eBg+ zf22xC-8WtV>;pcD zDu2YYfBiN5+Jo-U|74>Vh)&^Bz|8kGjfpM*?kUAKvXN#E8A;_oqTOa0mDf)*IwMb< zs|ZZ6pzK>9E?ELuhssWE?2JLk46RnRH6hCu1ES`i&L{5dxoqZ-LS9H@w_>EltYtdZt)?BfR{ z(pnO8O?>m~NFXh)zxli2ao`^K73y%bu9+{bEElNz;GP{(yf2S<3!g_aD< z5Yy)OCfuit^owmWZ3wcN-;z?S+pOc`l;;=MAve07pLPB1zo;3wo|OnzG}FPdAtqEr zyE9+6ElpcH?S_Pv4!5j8B!6XE;#6-^gmh9(SQ4vnqH@r&N;ulWjdrwjD~v#eGz5Sp zTlJhqg*;Rkqh%~XM8^;_e-0BIz1Ya)XP==)Khud|SFEypw>zD$GshU-ve0)lCs^SB zuP5H}oT=rO?Qjf%_%J)oEIo^mZxX#_d*XO!5;I3)79mMFh?F%!wJ4l?fB=dgB(Dyy zrzur|Sadyeq_ZtqKi$ByxhlFFts;(I`LskKCThedcIGE3@n-TRsB&7(qi?%z){WRp z+8km!$iiwPEc>g>AgQS(TWE(!Sby)F=yDjgj+axLUp(^oX~xoQ>e1^kqRhhkI;xpb z6`CF8B{RH0YT#^6OY~YzT$)QdAOS)MHD^|flW)_{#vUp2?m4E6o<~1TVwe6ZLkE@S z1ZXDDE&H84*PPhK&1RVw&v3T!t&6nxj?K29j64<;x6*8Qlz?U+{s&;#C%eXDt#s(Z zJ440uO8E;Tl!Y$FPFP6Y?Tg9*SiDShH5vSKN);5H&#ohNM=rIbnt=38H-E1$yF8eh z^6(dMT{fN8Ryq6-F2HLfDV+9~RaeZ}tsuFL%NWAvPe-sPl6*NtPaiz!tT&KltsScp zy5ey+|Dw~iy?1yaO3!mtCY%9i3>T0469xzzcj!$WQx>xr=~I66XgD@>-yYIW8_-yj z=b#c3){O|IZQ8mWECYCTC@|a9Y77%!GOAISo$UTc{r`-08VDFpsjfkepr^m?ZYlQz z4eAt^_V5=_!>h~aTN4)qfKJISxAcyc>ySL`pIHz{mv{Smc(}xa2$^jCO9i$%#izgV8Gf_kW`o+5aA_=?jf3WZ^<> zMZj;H-vZ-As7l%5G?9Fd$(`qSgZ@cl3Co)dsH%?SQ(FJ_@q-6kp`U}wzg*bG8wJ%) z$U`4VvB+|L#P89)eO0koD3sT7*Z*ls{T8!RaiL>g&b3t4;_g+@!Zu^ zwwUxl=_+cd8U4WJAtjjxF02U;JNWa#E4Y?H&KU84z&$7_+2Ziap9Gi2%}n>uB*Nn2 z%2+UJ=o`=6bMN&B>tLf~e1}KDw?ugI0@!F^S*Nz7%VNNK;=n|rS}BKHM`QyEyu-6& zOUK!toWTsNf6rC+i2D?SJBP$|aZ9p0yv0QM0h%z`JSk<=VME|r{vU8rY_e^qjjxpO ze=e~N4sL~}=tt~;0qHr4Ux3wGeorCeK3@O7e))G!&T1}}9Jc@dxX<_RHwIij`2!(- z{y%Tv_$=cFVDY59_zSQ&CTBqEj9;ON9>afb+-51$LMg*IV&hYA@jFzY51d{YW6=|s ziLczzz#q}#tYfz@8@QV{qNjdCTdPO^+h5^;0rEmA75btMa6@|(?qh;~;${rS5#>9w z1(usO5LL!tj6d3$je;-T!^~=Djqh>(p#Aq5iHR8Jdu03!yefN~ZslTfK8MCP$`WPv zaLdbclsbBJWQ9JHpqT{U$15NPa`n)t!v{ab_)*kG{>v!RE;0ke4puDJ(V0PP>1I1Q z9sARf!87%+#_>siE)Kn4;5yTQiGaUAMdKfP_FMygY|1;murN{o+B;~5$r7F1{_|p9 zp!l@~MZD2enC=>Q&RvhM*wk?E2zEY*^1Nd*Y&vBab=B1B7e~tSEOJh%jo={|uk2D< zI%#Ef)eucS4!Po|?k%I%^pybWE!Csf=7m9J9dq_!G?4qx@4SN1Yj@7w-WGj8*C^OF zgRP$4Uw>ZTSfq^WAz^Vrvo+gJ)r%L6TsGOwfkXK4z3wI7ohrQL?9R%}E2CUlXmk-qH!@%*Ewp z7Oqp1Eg@$g?SMOC=joGJFyR|k@HyEp*~>p4mT7nDO~(GKpO?|w8(0nV(1%LAG@gBL zsl}&^_vroop9Xkv2y|Jt8~pzLyKXNtyeLfn#r;m8Xu=yTf8W2Tm`DAp-QOGS-oiT5 zEVQmXd*BUC(Co0|FlnB)vyx<1(VUT&({SNvGUMkPtKwN9??=)7wwwmSk?C?~XXC>(?#=Kg>E1^?g&-Z+a&OM!R&EGJwqd94Z|u#PN6z1j3WMkmwS@SZm~GyPn<{$zicAMkq4{=pDuPU>_Spt(z?V**I(a0%oK z`Z=#QZzU;&hYAVu@|OLk9#P+(2|O1l@q__X`>4#%&w@pc|A8GlPI9qCy+t?$_Qc=M zFW*XbOOf>9Um zO745+26hOo|5o%lL69=t3!OQ1ICZQI7Ii01ARj`h!MW{-D`d3W%fBE|K*KZ$ za{M`O*MDl^O0&SRIG)HT@n2tCPRg%Q!h|${?W}ugqBi7YxPrC@jXGK>*qJ7{(%77U z@JiSIUq6jPHz3CZY6I6D#cj;&lW^x0;N$CPFZJMq0yn~!&FAYysG{C$0Dytf0W}L? zD#MXJ#6+*~1Dx%xLRgP$S|`E-T;=$1YQ^P>2fETuV>iI>*>;*KFjkv|%)OYu|9*E} z2(4=9c2MJqS)e{}X@2f|)}KfIJPrTHIxdVkYKz$*NY#3OjnQ7JvPS_FQPqm)k>6pT zVG-r}#TpHzj3VOq`;etjUcG_VB1y0xKu!o3x9PqUku{GuZ$>SgW@3G^J_;>ins#~< zk`fU^NsLU-Tr1ow|3JUTJV&b-ZEO&(QDXe?aYQZN)fk7L4|TFzsJ02$_#4CY%HC@n zkho-0aj7Alg2Guj*OyyaMBoVruiKl9M|#%Baflhxmq_Og?>Qx5ouNr%v|H7^cnuC& zYm9b`ozC#?-m(AMVBJsqKd+Jg;|ihtpKs>}ihlkA=^2K#CT#Af9~xIz|Iu+yF9H?` z6}(ohB8arm4+YNmVkHkX;%BM0Sh#}fBR|ihqIGEK?Ru2F?8FLPvf=6xJ_|Loy2PHI zx9b93E<*=VVFZAfH4Yt?iZNV0qBeT$=5t z)}z8sB~3WCx9HYY{(fnL#$K`3kPzndq6Q1e3h{FCnrZQV-M?RJK8xo)W~0fDN`UpX17a7TX&JlG zoM*}LZp#lZ8Z}hMpJ4zw_@Un9`x zIDrkM#2ZCO30jChsbA2DBRd%%SzXe*1__x|ujHd21FHVq3F~WGA+cw?(a5F2we$f& zEA@6TTIDDOyi@1h#r3t@yR&kh1{kO@e)RKijM513TAISm$w_HuNWf0Qnrfrz5%w3< zbqHKgP(TGxq|Ezs+gWm(K*aD+H1 zJ@Tz=(n575|LaAVC0aIGiv_^+Y_tVl8?h>6wU7skF6aNgHLeHJsGAs?k}}e&6|Z)i z8bbUAPE1_uIP=uhQt^7@yQ}D_2%2dR|K;t|4#d@1BnU50K$BxX0F0*b{+;+BLConm z)y8=mV8hNlT%SPAYimnJVP=6VC>|86d%8hs30s7;Sk%i#;sjdpR$=oosX(`#d0L}j z^gCx;I6PEt7=b`c*>bEo#a3n)pR!f!NN*te?qC5YhC#_#9b!XNXe{%Ua85?Oyw9#3 zJARfv!0rCkD$EQwLwgG-+_&!3a}Ig5J>NkY6{grIb!~^eS^7q;WS+5+r8O|iBnj6b zXq!=W95Rx5wk1M7K{;r)QV?l4$~(~x6Ot1#gfhy>uCaq?-?=h_wplc)6{MLDIJDD- zQ0z}1T& z&IH$GJI0JzaCR#q5DqbM;|H5JV_a>((In}D`?wU07V;>Nmk`lo0DnIi9(G_oeRo@S z$u)m4lL_gC<%+vA3^4dr@UjI|_wj(U-p6YzgG1M#CUZv_Xe?b#FE35@VJUiX*7k=CMcv4HVgZOGOR*DmoAKh^73O1 ztHYU-Z5Ld6XYNCuBa~LqnG)LGpxk47DFFbAeH^4>IgHwVD3W*gp$o;e$9oRQ-GFUL zZ-vC2R}Am@%6DmI5{@5G;(OyIoH?N&o-}3{C`>AdTxA#0yVCptbHtBzxKX`-!dB*Y zQezSsg{vL~rDZ|MJv;Ni=WAhYoZAi)3N(W-1?H-1VaH)?jB^U&Y_xp*@B&I!_#-oJ zRqBKxyIK^us>i;Y>q_uZET}luu2Wa3+<*hi{m{k)hKjSp z=s}Xc=mK3`{gqmBJgR*>^BE>Em6>QWrp|l*!UwoK&yuQwp@ZMJ9-F8(JEmXeZS^!}Xa=;Dq|J^1KwQZ~FXAO=h^K&5qLrK%|<8Pdb?PBbXId{<{m7d&0O|MfqAQFhBCZeLoX<4&#><2>HnqZG z>do+JT#L(s&F4oMT7z{F4STClxZ6sE+NMK zHGPqeWo}(hRsfykL4#O3k6}WWaYj15SnTd*C4PSm3F;` z`7vpl%he6FKr*rxB%Jyg7GPrE7ub~ zoZTv{`{k$wMauNwr3G~TKn+jh)u-@iMF$#IVMJdmtE%x+3eTTo>4(53DaS<;HX)yX zJo+(fiIvomi>fJ<5ib36XFrUWaAx!dcID?SB&0z7j8ny*m+~z91WMO0_`|<7L|DTm zhw)-_gezEm-W`TKdC`T?{+3IcYeDXga7Zj5;ybSaC^01Qr#cZ-bHh-2V1#k`R-iZXlw*2ZKJ?`VRYI_=i|eQ0271F z87*-ePm>&0^MK=`@A46irlP`)+c~HdsVQ;r3>2s9UP(93=uTchiA37pt}`ECdpYGh zEUtEU?gL1X2U~SA{>|z+3pNEt1zj*`MjKg1|DjL2q(6wxPNW!m!9Mh;#+Ne`CM`Cu z#Qj2iXcIHr4mwUTYEAA4%^Q~8 zSu42o;tH)jT(-4nyHEbovkWMxsEaeNSd!>-}g zJ$R(ZC`G#eHcnezfTCH_X7*u1F6^4YkW~brqO|eq9m9?DfCMkCG;;|qB*|Ll*`@WV zF=!+Wt7Ubbd@dFqn1i5-9ezMj@;jKtBL_BFcPF=JBKcw!CZ(hdi{}^$s=d3k9}1f5 z3Gnvbm?%e=pKiSz@eT*B<59wr0nYLQ!!_WAII5%zSJEKE*u@zh zuzA^am>-2Yo%#5&jju;-7iLC23nvoC4>FEvT6``=D4S)CLwj(tZFm5Q6?@a!tBNMc+>wh{; z=sKpT*l1~F>$@Aj1plT3(y*Dqd#l+Z>=q@iteZ7NwjJBQ&EPGO2-_~yz(gKuUYh;Z zmL4t}n$;5q3)uK*i1URFZ5VZ8A_Hqm?8ye#(Nwfkr?-MCwWpvw!xFeS48Ca&i^M{>EF#b|9&cAWhVOdW&g~wNwUew+MQy_qf2`B`U}gnyj*>I;R%YB zNc6O)UNr5&Sgr*%rapMSUC5-u{O8|N9$W6U4Jfh)$Q)f??RK3!v+%@tZ5LEkb1HC;VeC6s@YXX(v^msAB;kP=U&;;IHUrl2LYVqme?etlK@ zZh4l{YxhVeo!!!W&K=JWo@?jX%G54NDG_zD&Z3HBcSre4j=Xt3WRG8y?GE#1sgo-D zHh-8n_*gJ)zFm50$b(*uU(dD*w@yT2Y&fR+Us^j@7+x`D`&imoR%o}A-sWouVWPI~>*MlD=+wq{elci&+ zN};NMN6*%X;XXaqm+5_iR33h1yGGTvn|=|_1_~=baQE`dv{0+$O5XLu2R02g4hXX= ztPZb?Xu0-may}eCvpQOAUiz@Z(|OrltFfW(>Id zzz=EWAi+UF#mYg&?zK^Ax7ackiu^)fT6KHWA^Nw#lXu@Jy$sr~Rl`>FtT&LdS3QGq zw-!7BqFh9ewO&Q^7SQ&X65)PaWHlHm=INH4^QM*~PMuOEG*~!MOCfd*u+RQZT-Mbz zkmjZTtCYK`-np>t5CX5kw?TG8uW#QMaajD3J6*N%g9R-tW}*j&spAbqaE(H0Wu#V zlN!Z@(=YFqIV>0*@m_<@?}=fnX)xT7N001*hUwS|W@l zJJR#Bt2h667c>27OYYE{tU;!GHVa=$?v(F$oo^c}CQ31HQBJEA&~~Ri zR2HwOED)Y=t$VIC&~9a4wu?a7#MC*u>w$55MucL~{O2DQKAIVygG=`jh=aVBOc}t`)L*7VIwxP46NcHKihh#oZjT+P=6t z7o=W|nJ(QKHM;$D_q1EdWOi}?8^5%PqE1QXn)z>m=F@#EEZ@3ohR1?`Q@!ttk~`JH zVKNejE;tP0^5=X^+swqgJn-q#onizX?0q|>&P<5i{zs%xTq7<+emj$c-${0X%U?g; z=a9*e^}R1m$R`L~9x9zv3aoRYn)Q4V>G-9PV&KkSd7dp(ZX1(B;7Rr`ft0V$wTJwp zf7zvf>+`=)IYYz5@=d3P8|~@+ZLWhN+7SlU^{=cmuY1iUY)q-E+e*Ht)+xR5JanP& z5u4p$q<-3Bt<+$DXnLcXe%h%*>euHSQpvP~Os{ln&Jx4e&7U6);ULPb+8blw+>3eElemmZk65IkPzB zy1txmYUWm$lvVYYp95__ovHpDE|0sP{Go07+=ge{z)jpeHGT)@YyEux#G=cQ&Tk!}h=V2u9az zVfl5N+mjz<>64|AM2@m%EsOH~GXc&H(`mEZR2%a0c(vhrJL7)K2lg;$cZKtD+IDot zeA3&+{_Nm6J1QZvV9a02yirxwRdwmbABWQF&(b?5HfH>Hy$3(mmc{NEg%}ns``*vh zb*FU@iGnH8cW+8k6N6gcs&j}Rp76?M-0~_0!+X2)I^Pu?i%7NIKA{FxF@femsxtPf zORxN;#!vC>WGci3jT_!37CJ4cZ;s7ut$V9%8Lp;#r^QdMI@2?s)V<L`zss=Y7xWGZAvPx1eD&%mb6 z$Yr`R@#9&cpiiS%$#Qhl!DDlay~=4-MxOY)3y>8qL;%+XT1OW5v)Y*+bz|y%hHiIdp_~esofjT{nW^) zMARxf;$-aJ5Vx@tvaT$4WxF^ej;=2CKM`u@F)MQT8TA{Je50P4^0dwqo?9(8ZAUEl zRims0C(iinsy^tOA^%yo{;|6N)9mE{(T_$$pF_9LREs#N>gO_44?E2@v@|I8I9=%~p{6t9@=BpWz$eyxl5N7} z=Z6++hxA&Bo*MB3_2EU-l8Xl z>Fm*lXy@R$>G|y9u0a0E*5atx(7K}KfhwVNstt#6&347FmhonO!qbnA^HnMfntAU^>kZ(L5qBRCs|=FOk|pRAO$Q@Fh;**2 zf358~r)Q_^)E3*DzMiv2zFT#K+kMWQY!>N=D1$>$0ULnVs?4~*dnLbX_cCr~yug0m z*`Fs8Px{l(!e7F!^tW``0^f-QHRa8(7Od8=>ZFus{0znAJ7NMQ-6uM0BHF}N?VGOF z{^ci5QS8Q_>HVq=9&glNanf3nH4Dvsxve>+b`t3K~J!-~-24(lRU97v7g zwar>m6};V3Q_l~P51tEGT+i(dl%+LU1F~9P9&1d@?i$;V*w8$;zpJOBLA*AOESZqh z5^?9O{cvpUZrPf_44T_ixqzieVaDE&cFzU#KzSMC$hyq3-L)#mB~%;roV0_)L-kw4 zRmEKGu)(EQi=-FBV>eu))jCeyHat#xqCmjXYjrfvGTbVKKpN<DpCEyP!L;|Z}f%M3`fUU)g<>te&bR`*>dkcW61AI?1?<)EDJW zG98Ycp{z%Aub9uLG*eC;+RpRx5t|xmeJO>RgB0m>i6op|lQ%jbs2En*v)mnxOr>P| zpy3+7GGf_tK75sF)#j0oGA-)U)rRIXwK$r&$(#yuTKdgX=u<{aMt(C@0lk6D8Y@@~u7unuMeU+ndh0J`iOU_7Jf9Rn-J6e@ zx!b4K5@({~)zwK}tJA~kgZH;jle_sUO#=JXiB)JTV>ga{9l-uVS4oC$o5o~V*1>A7a_Z!KA7GglCC(Nm5nk+sl{ZX_|3l& zw@~ELjm8pGg!!>+*Rs-=6>HpLPmVg?QJ-JzrMG0Cw7j-$cg7>EqG^FC*+O zIdw@Z(v=nHl>vd2B?|qZuj!|Ie6Ib$iCH?V+I;rclZwlBYdf{A^a-|^lV?Xm8`UY7 zgv|131FoxD++Hqms@0W(gFpS!TXdvnYM41&Qp?RkBSWLVjs~m-4%oK0sFGXmiVr}^ z_&i`IlUo2%`3B`M(Sv@+6$T|{Ic;7(KQt2*-G{7mmz3#b5BBU4lJBW7Krr_jgN>5BBPN_JhY^tlvSwN$cMZvRO;(=Sh5d*>+fQp@9H{AjJD za$1{ty5d~yag>>QOb5AAsf>7TZNa|hKI48gb&e>ntqi&@hU6|)8=f>e9b)KJ@?Fc& z#dy;x(Va{)G>PJm`hVIiOm*1jE{~6WJh!wEz09@}y&e=MmcG8$b~~ZXnWE=T>m#aj zg-^^`)O@{L(O;ohcU`NWq+&PnUfaIb)nI)B#!a^o3^%ILr$c8Z%Uq|*fyI#Yh^rmN zvBdRU-py`hyw%<0VoR#T=*S~AS8Yk7(}6pEtWs(P$bH-sFPAMAz8^|RBmwI{oaW1I zPV}HqlQk}$vJ;1uH7x7*K)aA zGf;!|HNz+`x2~3T_wIEs0ktug^)C1Hl5Lm~`Lqy;)|%F$5M3MP#=bC%=??q!VzW+1 z3=Pg1xv9U$b4=#VYHb%y=@93Oh#$N{I7pW z_dV;dQ>o~$PFZS-$NZ+R49d^JH8 z@3>7hxf5Cj(v;ES+f(JT0WY~-O)J>cBg1`p)kusen0Mt*r2R-3CWwY2cI}qA3thm3 zDj$4Iak^Qx`810}g?152?Wkup0i`RnROKzw548N z^4nZmmfDSx9&hq4$)SgiGT&;Yq|W(>Z`me3;iiQpUMt)64KLpQ+xhh3p5b_MY^Y6? zt9hL*ARTgvH+gN9oLyW9NDp|_0DX{5T+SA-L`o$6<6*$@0t6$Q5{tdkzu&BVb*6TJEHxO>IQ8C} z#U?~A4OrR0y)EIkUg^r=xMc1`(ZEy%S>5AJn-dj}ZcfW2*GdsC-CcD^C&S;cAIMYTk?{q{QXTQEA(K=f;CLXz% zX5D7y-XXzEz`Hn>cgrR>ws*5PpGn0)g`!tsxI8T*!bxZ0a<xC}xqj-<$*( zL?r(hBATM*PGyIy6KcxKpZ#stEp zK#jUFdSdJ?IzoYRq~S(gj%}AB?ano8v+9di9}AM57>j5g>XkWR#u()4KYaKWy1Ppc zPs#|Wj?{#j38ly_b5JKWJM06ofrjp7KE{MNt2fZO9iJRc?EQ61WKX4_nBxb=zmmn+ zSqFX(Kt~i=?Mn$N1Z|>i;J*Ur*SF5x245c8-+UV%I4qU!BN3-R$dE=m_|p z&2aWtLe~fELO1=)VuSya!2>s-WpW$8vdCRuOU%2?YLL^&>d~4{>YJx--~!7N6Ey)_ z??ow#eq@vB{hJwLC5#dWu5b%o~}j-B(} zv8{^~CWh@Yt1XTuz44J>%bs$$_B|ix-8PUcWaC&|&idefw&~B$1(RnlJiW!h(ryBJ z*(FtjxVc3)pUD2w&ms!PM}OzUc%NBzrz8%o_6#+HkRp?nMa+zCEH?lB(xxpZt#5n@ zpsg7jiF^rEN2=>Ks%GGnPdjOH!6rW`C^UA0osqRs=YMAaOnaD>vpO|J>ouNolWQfb zNh(g$X$T*ji)DMiI5Gep5e#?WVecOSvM`sCV%cJU!pT}|mB*P%z!~LSSvBj%BKJ%% z+5yI#o~0e$#$*C=K(B>-1^@-BICrikd56y*hfaEV#HrCp<-?sosP1iQGY#56Sfmz= z*u@Wn*s$K!E@h(^E+9Kr&+iw=6^1yieE$Y|wm5~L;8@y4sy)-`RxMfj6?cEs(VW6F5QfhP!-AjycMr~KVZo0j-MyD>9jbqiPivHd~n}Y()cV8*nC$a66Bcsv=WtF_2YUcV)AAU=U{sVnw>=9^;wDuVL0L6f3gn)f0j~MF5HfK42C~ zYfl=>Hms9Nw~{(zYu9`o`A^J+Oa??5+Pc%{YD`4Z8LM!C7$xV5>a;*F@j!)etK{M7 zUEgGZN#ks z6#ICwVs=-%Jdkb?>`?t<;_%vLFp&c)O- z+w!QvS~#roCl+!NlEa@LvadSQnIX-31(^R#UG>^>Hsfhiq<;0`128<6o^QP8C86de zLLqJQuyN{7W8eM;75j<8&g}E7rxaI=m*%P~6uTJrRET@REnrMWQzm?j|5QfQFFok6 zKoD~7mQP5DvL)v0N>yz}yzQ}&57KLcGW*ZhH1#EPZNg5NYKUiqy-UR@rRC$)4aw^ zuivIFvfaRkxO{Ef1y;Yh4E&=R}_&Msjo5E%WY?>cT`c z#Y+-V+uD3KYnGZil16$juYKQh#!Dp(wTjo0>3R4BfaEQv&V*4iK03N3(y{>%+7#?O%{yoA{Ufy^ZR7)olnB+jvH8$9=T(}mwq-1c zKl*1u9rBO*a{ycVo-3u*lVw&eOgIll0_rJVB_O0?&@Q`dCD|F=NvL7nc{a!XQ?mrc z#WO|onR4?tmKJOYKHGU~bgb~U(kTi_wWkBqC<=(%0U|Ko3(56YAKCG*`Q4GqL1|wO z=*#T!L=hlDbS{rOMU#(01nKp@C-2?_X-$6kZ9w+kLgyW+%M-y zw5)2k(f%e4Y#2+)m?lb!2o-GB6&8?P0YU=ZL-9-o1=q={Yg+Z9!!_RRmsb_iVBC62GofoM;3CUPx2!(E1a(i^aAOcVLH;3n0n#D)5-6bn3nQf z^Z&S9>?J+@b)|cqj70j4>rf2d3Frp(_mB7xEa-MuS;qHgQQLVQg@EXba!-!un%;bm zwU&w#ETw>SU9%D)`Z*RFgCoVM#mp0d(evTVlCSEvWN@Oay2p@s&L4Kj2mg7%s!zqR zDzpeU!c?E-y4!~joHTtvO3dWUq0d09lbBfm~TurW(m;C3ceB^u-o^sZhfxLN=}A`H)bQrzBQ;ka`_s zFR2<0SCH4WjYMFn@0@EBADTZ94)z3zl3mkP#(I?hvaA2?+#Fvlv%5Mp#PrX;~ zA0XQ|s)COfSQ%EwLIC^(pbR4h{@Hx82k?+k3f?k7eG9(t5k(TOK2c#h-3LNBQi9p@ zD@fiZjTt0jRQm#HN_D()k@SfoWxMg*+nbmSdWE@-FHVD))rhXz!eUDA zm$l+sdYF`52heG@ks4)U;4y9w;HBfP5`s|L)gfyziH8Ed=G?_;y)obeNjOD}%hJAG z0Awa>#zZ;&QqXAeTnFI?#!X6inwt5$*--co0B;thp*7 zi;kRQvmt5?zufD3zZY-*52e1=;yU5%x5V}ABKE?L*S-vjaW;dOES`rk z%<>D??)zlQc-)aVcQ|UT4M#I10&Z{X=$`G^D!-k#|85+a#$MP4@ZqVL-M8~QZwvDw z%ff!4u=K6N&59Z`8N%&u?KR~W;hwC+bnn_A7Y4G&8*d5=IlEKFOy2Fa;_8ueK7C4x zRqybgr&|i9FFCdG-9|Izcg>6v*93W&Z>UN0`*JlfmXEiijwl~}SRFC^K!7ZE z%g9W=!iw2pL>9=5>Q2SbuF9189XJ2{uiV@}HhD*o&drP0=N^m9aJMWrQ$6=2^1IESN@T?%?VII0Gh59UI%jy-pDz?(QRS5*9qnEfL9Cchn;uIi zsF!u;(g(l@b*;^Jc-U(i7Sb%4jLtOj$MY4Afas2LAHbv%RQTq|`C2eW_b~tCGX;C&{!#6&F(hCJDED)Tk(c|1jhX>aJo`` zfUS6_fmWX`H+ed%TLA=Akh+cg3mYkiTQ)fd5rr6+fzl$VHx5R+ngausrM>80(-da< zZK`o01$sFO_z>^gaCt{E}|*JUa|Sa4BU@xx_+ zBbO(l)xAz)EuLp1xF2A;VdO|dM%YeV*K;uj2mXcAxvb?@Ew;JQq{ftJs^&mB+su8I zJ{%L@&SPV5kBSSl4EI=SMADAaD?lNSF*=)CRZB!oE;vzzr8-{#9CI6WEZb&TQhnS4 zYi0h1O<}YDEs|+_2F_Ez<8@j^jQw_}BYjGa)Ca(;%N;vO#N7*P^k^ysQmh`G`)qK6yPN0ad+9e2D1B9bhpBPaS^nw<1-sgtfStJfd@kbo^p z`_^l{J7;`YZ1HSE`|~E8qqc18dm@z9M?yj?w{$C;5Mq99e0^;q zcWnv67s>B`h8lLwpDt|uf7-k9ucq!i`Yb~^)(&y>96SyP)}`tc0g1>W2v%)HRJ2$n z>?(@^0|-dKkVMNAjzdL-8rBfT4Le#;C_#2pI#mQjsca@dM<7uo0TB{nSmwU@D|*g+ z`GIptczNIV-uL^t_kQkuFX*uSylkRK7h2Kg65U};CHy1W>6GuuIXb>q%ctUB%X_ zy~_}f!UbDo+gc`6fBizisbRy%cyt-=f6Cmuye&Vf&z%t?YqCW?^vlxtm*>$pqoY_x z3p{K)`1D)?1I*m2l$pRQLOqY}cM^}b3lTUNKEEywA%c??xMFVw(@m(uUH-Pq0*4kJ z6DzM~#QlB_Oc6HlB=)NHd9SdhS-B3^ij7)c~lQJZH^`2Og;BV6L9Yt_`MzGM0= zKUXy(m6z+4%k5S=4ir{)Me_2EAAV>)g!mkz z32J0*&Pm=%>eUUu+Gm5SGKNMg;$Qs}y=ip+{C(^sP$Y-_!|JEv1}~shE-hg_MWb$6&LbtYn3{~_^e=A+ZY)fRL!%>Q z_%nUCa9>sMURjEZPJtB&XQ=atiW3us(RP8p@HEYijd`F1KvufSp4e^|wkOkRmHA2Z zl$!CcKQF3y5y@%)6ICB*xR~WKGY*l2;lN?5eP~SqB6wqB=&hWf{C}I#AF8;Tp&XcZ z(+x;gBtDumL`;t>1Uxx&>d?My$qf1s*;eyT>k2Q`jv;kJe=L880s89sEu_fwZ84a$ zmS=9M+3RtPFk90t^UDMd?)vj8Gj2Y&ZngU?yR_B@tAF2?W;jl^LN@QFrBc_}t}IX$ zrZD8s1*-1cMq_h((Us+1=2k-CY0d z?oo*B!ZVSG1x1zSfkk9{OWk}JxP=RK6flsbNbtcC#|Uv43p}iT^8uiPasycg1oU;D z8MHA{Z=ZY9h{k~^$c5XV=cYZu%kRGp2QT4(Wr|&}ehJ0V$_o zXs`Z~&g&!`jBao)h<>5r*n|DdH=dMBo!1NHFxmWVZ3k9a4s5C zeBbtu(pcJ_)fgwN^1DaTET#(}D%_<{P~HQQ!}dGLSc83$uxmh=7{W(-6`erTD|@4B zVawv~nyp6vO|F;jYkzibwLJ zaL}x--JJ z;^PgE4lyRA0v=k8S#AhCfm~X`SK!)S8?R*8#9d%B95{Y9jjqUUz03f_F5k53+81GE z^=7&AY(;Ve%&*){bXyFOy8f+rXYhjDauK!Onvz|*&Yq}{48OXDt!O6(Ez)PZ0B6V9FGlWdn>l@46T(H=F@TqwFD=r8a1QcNLb`C@5$Jmw& z*QMuP_4^SztmwDFO21p%yUstlR~t$_FlX6)b5sQSggol?Y;ZW|x4(cNa2U`174hD5 zj#I>+XUtQrnKdB_i&jIl08|ql^MFbD_E#?&k3q|=Q#UlHB>Wt`}!s;mwse3 z*RUeQAR_Y@!|7qT7h;r-4VU3P@II~S|B0PEF2j|x6gV;5HX9^p zw&yWb5wZnFex60vC4vNlMW@h0(l z+}l*p;Z^D}I^d3!0DLJdDodVqK}-=0;@iQMZigAMWA*x1Ujj{>=V#E!PaeHz!7cQ| z&7*B2Yh`v`jRJH#P~0$kfp{QHJJOXA?k$tAIo@4qnAgZQj{go{O)xX)Ivrx_UN>le z7a_}0e(O9EJe8=^|nm0xMUWQ}-kJ+dCGxjH>Umb%GX5XBfIj%3 zRADJ}T1N|R*ndOu!@8^vO>LXwP-e&>y~flCQst7m%uc>mbpU^g#YF?^jY zukfeE)|H1uzQ?pyIRW;w2356+UVHXy5wBbpJes8H6mFZ?ZJ83QRf7Gcn!sSm4&<`8rcBxD4 z;;!Jv&PF^BdB#YF=Sh68D7B)f^>Bu$;x%~nY*q+f-SZ7@FC{OM6GA2fSqP*ukg7-! zfuJElLxP3`4G9_&G$d$9(2$@ZK|_Lu1Puuq5;P=eNYId=Awffeh6D`>8veo1F!a!y zTNGvS%5&%TL&Vl3J}>dOiNF7Um2H@wS9hPI@N;&h@Xf+4>oa_R#vj&4@k>O1owFAm z`&f%7sXthP>JYLJUx9J=?;vyJ-^d9eGlZ-KQXELNB)C9;l28J|kqBud^qA-cqKt?# wBFcy;BchCmG9t=|C?lebh%)-4D5H;zdQ3N6m6;l_EGf@j`*)Up9`en90pFO8e*gdg diff --git a/sdks/python/setup.py b/sdks/python/setup.py index cadc4f34c86d..ca585ccf7167 100644 --- a/sdks/python/setup.py +++ b/sdks/python/setup.py @@ -349,9 +349,11 @@ def get_portability_package_data(): 'interactive_test': [ # headless chrome based integration tests 'needle>=0.5.0,<1', - 'chromedriver-binary>=100,<114', + 'chromedriver-binary>=117,<118', # use a fixed major version of PIL for different python versions 'pillow>=7.1.1,<10', + # urllib 2.x is a breaking change for the headless chrome tests + 'urllib3<2,>=1.21.1' ], 'aws': ['boto3>=1.9,<2'], 'azure': [ From b117ac8b08374c2c4ed2c0e57c691e00e6dcb92f Mon Sep 17 00:00:00 2001 From: Anand Inguva <34158215+AnandInguva@users.noreply.github.com> Date: Sat, 7 Oct 2023 10:02:30 -0400 Subject: [PATCH 066/435] Update executionResult (#28884) --- sdks/python/apache_beam/testing/load_tests/build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/testing/load_tests/build.gradle b/sdks/python/apache_beam/testing/load_tests/build.gradle index 144f7d12ba6c..538d4a01bfee 100644 --- a/sdks/python/apache_beam/testing/load_tests/build.gradle +++ b/sdks/python/apache_beam/testing/load_tests/build.gradle @@ -59,7 +59,7 @@ task run(type: Exec, dependsOn: installGcpTest) { ignoreExitValue true doLast { - if (execResult.exitValue != 0) { + if (executionResult.get().exitValue != 0) { throw new GradleException('error occurred') } } From d5b8fb8295827fed6cd13c2767e73cdfe4af1ef4 Mon Sep 17 00:00:00 2001 From: Anand Inguva <34158215+AnandInguva@users.noreply.github.com> Date: Sat, 7 Oct 2023 10:29:26 -0400 Subject: [PATCH 067/435] Add edge segment size to filter out change points that are observed on the data edge (#28780) * Add edge_segment_length * Add issue to remove workaround * Move is_edge_change_point into find_latest_change_point_index * Fix python formatting --- .../testing/analyzers/constants.py | 1 + .../testing/analyzers/perf_analysis_test.py | 10 ++++++ .../testing/analyzers/perf_analysis_utils.py | 31 ++++++++++++++++++- 3 files changed, 41 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/testing/analyzers/constants.py b/sdks/python/apache_beam/testing/analyzers/constants.py index 8f8bdf13300c..09ab5c595908 100644 --- a/sdks/python/apache_beam/testing/analyzers/constants.py +++ b/sdks/python/apache_beam/testing/analyzers/constants.py @@ -72,3 +72,4 @@ }] _ANOMALY_MARKER = ' <---- Anomaly' +_EDGE_SEGMENT_SIZE = 3 diff --git a/sdks/python/apache_beam/testing/analyzers/perf_analysis_test.py b/sdks/python/apache_beam/testing/analyzers/perf_analysis_test.py index 9c7921300d9d..15344ab13b3a 100644 --- a/sdks/python/apache_beam/testing/analyzers/perf_analysis_test.py +++ b/sdks/python/apache_beam/testing/analyzers/perf_analysis_test.py @@ -34,6 +34,7 @@ from apache_beam.testing.analyzers import github_issues_utils from apache_beam.testing.analyzers.perf_analysis_utils import BigQueryMetricsFetcher from apache_beam.testing.analyzers.perf_analysis_utils import is_change_point_in_valid_window + from apache_beam.testing.analyzers.perf_analysis_utils import is_edge_change_point from apache_beam.testing.analyzers.perf_analysis_utils import is_perf_alert from apache_beam.testing.analyzers.perf_analysis_utils import e_divisive from apache_beam.testing.analyzers.perf_analysis_utils import filter_change_points_by_median_threshold @@ -242,6 +243,15 @@ def read_csv(path): metric_values, change_points) self.assertEqual(len(valid_points), 0) + def test_change_point_on_edge_segment(self): + data = [1] * 50 + [100] + change_points = find_change_points(data) + self.assertEqual(change_points, [50]) + + self.assertEqual(is_edge_change_point(change_points[0], len(data)), True) + + self.assertEqual(find_latest_change_point_index(data), None) + if __name__ == '__main__': logging.getLogger().setLevel(logging.DEBUG) diff --git a/sdks/python/apache_beam/testing/analyzers/perf_analysis_utils.py b/sdks/python/apache_beam/testing/analyzers/perf_analysis_utils.py index f9604c490fc0..91c339a766d2 100644 --- a/sdks/python/apache_beam/testing/analyzers/perf_analysis_utils.py +++ b/sdks/python/apache_beam/testing/analyzers/perf_analysis_utils.py @@ -143,7 +143,22 @@ def find_latest_change_point_index(metric_values: List[Union[float, int]]): if not change_points_indices: return None change_points_indices.sort() - return change_points_indices[-1] + # Remove the change points that are at the edges of the data. + # https://github.com/apache/beam/issues/28757 + # Remove this workaround once we have a good solution to deal + # with the edge change points. + change_point_index = change_points_indices[-1] + if is_edge_change_point(change_point_index, + len(metric_values), + constants._EDGE_SEGMENT_SIZE): + logging.info( + 'The change point %s is located at the edge of the data with an edge ' + 'segment size of %s. This change point will be ignored for now, ' + 'awaiting additional data. Should the change point persist after ' + 'gathering more data, an alert will be raised.' % + (change_point_index, constants._EDGE_SEGMENT_SIZE)) + return None + return change_point_index def publish_issue_metadata_to_big_query(issue_metadata, table_name): @@ -231,6 +246,20 @@ def filter_change_points_by_median_threshold( return valid_change_points +def is_edge_change_point( + change_point_index, + data_size, + edge_segment_size=constants._EDGE_SEGMENT_SIZE): + """ + Removes the change points that are at the edges of the data. + Args: + change_point_index: Index of the change point. + data_size: Size of the data. + edge_segment_size: Size of the edge segment. + """ + return change_point_index > data_size - edge_segment_size + + class MetricsFetcher(metaclass=abc.ABCMeta): @abc.abstractmethod def fetch_metric_data( From a3bf007cef1f749b2759309c14ced1c5d9e38d76 Mon Sep 17 00:00:00 2001 From: Anand Inguva <34158215+AnandInguva@users.noreply.github.com> Date: Sat, 7 Oct 2023 11:35:14 -0400 Subject: [PATCH 068/435] Append UUID when test_name is None for perf tool (#28882) * Append UUID when test_name is null * Update sdks/python/apache_beam/testing/analyzers/perf_analysis.py --- sdks/python/apache_beam/testing/analyzers/perf_analysis.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/testing/analyzers/perf_analysis.py b/sdks/python/apache_beam/testing/analyzers/perf_analysis.py index 0074625d3b47..109e5bfcc286 100644 --- a/sdks/python/apache_beam/testing/analyzers/perf_analysis.py +++ b/sdks/python/apache_beam/testing/analyzers/perf_analysis.py @@ -23,6 +23,7 @@ import argparse import logging import os +import uuid from datetime import datetime from datetime import timezone from typing import Any @@ -146,7 +147,7 @@ def run_change_point_analysis( datetime.now().replace(tzinfo=timezone.utc)), # BQ doesn't allow '.' in table name test_id=test_id.replace('.', '_'), - test_name=test_name, + test_name=test_name or uuid.uuid4().hex, metric_name=metric_name, change_point=metric_values[change_point_index], issue_number=issue_number, From 92d5bf877969cd00a229a97d35575f7f5d1fc09b Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 9 Oct 2023 09:10:00 -0400 Subject: [PATCH 069/435] Bump github.com/aws/aws-sdk-go-v2 from 1.21.0 to 1.21.1 in /sdks (#28891) Bumps [github.com/aws/aws-sdk-go-v2](https://github.com/aws/aws-sdk-go-v2) from 1.21.0 to 1.21.1. - [Release notes](https://github.com/aws/aws-sdk-go-v2/releases) - [Commits](https://github.com/aws/aws-sdk-go-v2/compare/v1.21.0...v1.21.1) --- updated-dependencies: - dependency-name: github.com/aws/aws-sdk-go-v2 dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 4 ++-- sdks/go.sum | 6 ++++-- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index d1c98259f1e9..b8dd03de6b07 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -30,12 +30,12 @@ require ( cloud.google.com/go/pubsub v1.33.0 cloud.google.com/go/spanner v1.50.0 cloud.google.com/go/storage v1.33.0 - github.com/aws/aws-sdk-go-v2 v1.21.0 + github.com/aws/aws-sdk-go-v2 v1.21.1 github.com/aws/aws-sdk-go-v2/config v1.18.43 github.com/aws/aws-sdk-go-v2/credentials v1.13.41 github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.87 github.com/aws/aws-sdk-go-v2/service/s3 v1.40.0 - github.com/aws/smithy-go v1.14.2 + github.com/aws/smithy-go v1.15.0 github.com/docker/go-connections v0.4.0 github.com/dustin/go-humanize v1.0.1 github.com/go-sql-driver/mysql v1.7.1 diff --git a/sdks/go.sum b/sdks/go.sum index 1c39718964d0..f177dd8debb8 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -81,8 +81,9 @@ github.com/aws/aws-sdk-go v1.30.19/go.mod h1:5zCpMtNQVjRREroY7sYe8lOMRSxkhG6MZve github.com/aws/aws-sdk-go v1.34.0 h1:brux2dRrlwCF5JhTL7MUT3WUwo9zfDHZZp3+g3Mvlmo= github.com/aws/aws-sdk-go v1.34.0/go.mod h1:5zCpMtNQVjRREroY7sYe8lOMRSxkhG6MZveU8YkpAk0= github.com/aws/aws-sdk-go-v2 v1.7.1/go.mod h1:L5LuPC1ZgDr2xQS7AmIec/Jlc7O/Y1u2KxJyNVab250= -github.com/aws/aws-sdk-go-v2 v1.21.0 h1:gMT0IW+03wtYJhRqTVYn0wLzwdnK9sRMcxmtfGzRdJc= github.com/aws/aws-sdk-go-v2 v1.21.0/go.mod h1:/RfNgGmRxI+iFOB1OeJUyxiU+9s88k3pfHvDagGEp0M= +github.com/aws/aws-sdk-go-v2 v1.21.1 h1:wjHYshtPpYOZm+/mu3NhVgRRc0baM6LJZOmxPZ5Cwzs= +github.com/aws/aws-sdk-go-v2 v1.21.1/go.mod h1:ErQhvNuEMhJjweavOYhxVkn2RUx7kQXVATHrjKtxIpM= github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.4.13 h1:OPLEkmhXf6xFPiz0bLeDArZIDx1NNS4oJyG4nv3Gct0= github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.4.13/go.mod h1:gpAbvyDGQFozTEmlTFO8XcQKHzubdq0LzRyJpG6MiXM= github.com/aws/aws-sdk-go-v2/config v1.5.0/go.mod h1:RWlPOAW3E3tbtNAqTwvSW54Of/yP3oiZXMI0xfUdjyA= @@ -133,8 +134,9 @@ github.com/aws/aws-sdk-go-v2/service/sts v1.22.0/go.mod h1:VC7JDqsqiwXukYEDjoHh9 github.com/aws/aws-sdk-go-v2/service/sts v1.23.0 h1:pyvfUqkNLMipdKNAtu7OVbRxUrR2BMaKccIPpk/Hkak= github.com/aws/aws-sdk-go-v2/service/sts v1.23.0/go.mod h1:VC7JDqsqiwXukYEDjoHh9U0fOJtNWh04FPQz4ct4GGU= github.com/aws/smithy-go v1.6.0/go.mod h1:SObp3lf9smib00L/v3U2eAKG8FyQ7iLrJnQiAmR5n+E= -github.com/aws/smithy-go v1.14.2 h1:MJU9hqBGbvWZdApzpvoF2WAIJDbtjK2NDJSiJP7HblQ= github.com/aws/smithy-go v1.14.2/go.mod h1:Tg+OJXh4MB2R/uN61Ko2f6hTZwB/ZYGOtib8J3gBHzA= +github.com/aws/smithy-go v1.15.0 h1:PS/durmlzvAFpQHDs4wi4sNNP9ExsqZh6IlfdHXgKK8= +github.com/aws/smithy-go v1.15.0/go.mod h1:Tg+OJXh4MB2R/uN61Ko2f6hTZwB/ZYGOtib8J3gBHzA= github.com/cenkalti/backoff/v4 v4.2.1 h1:y4OZtCnogmCPw98Zjyt5a6+QwPLGkiQsYW5oUqylYbM= github.com/cenkalti/backoff/v4 v4.2.1/go.mod h1:Y3VNntkOUPxTVeUxJ/G5vcM//AlwfmyYozVcomhLiZE= github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= From 76fbb8e9686b97fd51af5b8207c167f901671012 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 9 Oct 2023 09:25:22 -0400 Subject: [PATCH 070/435] Bump org.nosphere.apache.rat from 0.8.0 to 0.8.1 (#28675) Bumps org.nosphere.apache.rat from 0.8.0 to 0.8.1. --- updated-dependencies: - dependency-name: org.nosphere.apache.rat dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- build.gradle.kts | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.gradle.kts b/build.gradle.kts index a4a9a09e504f..fbea1a59b282 100644 --- a/build.gradle.kts +++ b/build.gradle.kts @@ -19,7 +19,7 @@ plugins { base // Apply one top level rat plugin to perform any required license enforcement analysis - id("org.nosphere.apache.rat") version "0.8.0" + id("org.nosphere.apache.rat") version "0.8.1" // Enable gradle-based release management id("net.researchgate.release") version "2.8.1" id("org.apache.beam.module") From 604629798a363c2a62a2aa46a1be89fca4a6b784 Mon Sep 17 00:00:00 2001 From: Anand Inguva <34158215+AnandInguva@users.noreply.github.com> Date: Mon, 9 Oct 2023 12:21:55 -0400 Subject: [PATCH 071/435] Add dataclasses to perf alert tool and refactor code. (#28889) * Refactor code with Dataclasses Refactor * Add pipe to add extra line for test_description * Fix lint --- .../apache_beam/testing/analyzers/__init__.py | 5 + .../testing/analyzers/github_issues_utils.py | 43 +++--- .../testing/analyzers/perf_analysis.py | 144 ++++++++++++------ .../testing/analyzers/perf_analysis_test.py | 69 +++++---- .../testing/analyzers/perf_analysis_utils.py | 141 +++++++++++------ .../testing/analyzers/tests_config.yaml | 49 +++--- 6 files changed, 284 insertions(+), 167 deletions(-) diff --git a/sdks/python/apache_beam/testing/analyzers/__init__.py b/sdks/python/apache_beam/testing/analyzers/__init__.py index cce3acad34a4..136d9f5f5d8a 100644 --- a/sdks/python/apache_beam/testing/analyzers/__init__.py +++ b/sdks/python/apache_beam/testing/analyzers/__init__.py @@ -14,3 +14,8 @@ # See the License for the specific language governing permissions and # limitations under the License. # + +""" +Peformance alert tooling for Apache Beam. No backwards compatibility +guarantees. +""" diff --git a/sdks/python/apache_beam/testing/analyzers/github_issues_utils.py b/sdks/python/apache_beam/testing/analyzers/github_issues_utils.py index 82758be8f180..bbcd2a8b11b5 100644 --- a/sdks/python/apache_beam/testing/analyzers/github_issues_utils.py +++ b/sdks/python/apache_beam/testing/analyzers/github_issues_utils.py @@ -21,10 +21,11 @@ from typing import Optional from typing import Tuple -import pandas as pd import requests from apache_beam.testing.analyzers import constants +from apache_beam.testing.analyzers.perf_analysis_utils import MetricContainer +from apache_beam.testing.analyzers.perf_analysis_utils import TestConfigContainer try: _GITHUB_TOKEN: Optional[str] = os.environ['GITHUB_TOKEN'] @@ -140,25 +141,18 @@ def add_awaiting_triage_label(issue_number: int): def get_issue_description( - test_id: str, - test_name: Optional[str], - metric_name: str, - timestamps: List[pd.Timestamp], - metric_values: List, + test_config_container: TestConfigContainer, + metric_container: MetricContainer, change_point_index: int, max_results_to_display: int = 5, - test_description: Optional[str] = None, ) -> str: """ Args: - metric_name: Metric name used for the Change Point Analysis. - timestamps: Timestamps of the metrics when they were published to the - Database. Timestamps are expected in ascending order. - metric_values: metric values for the previous runs. - change_point_index: Index for the change point. The element in the - index of the metric_values would be the change point. - max_results_to_display: Max number of results to display from the change - point index, in both directions of the change point index. + test_config_container: TestConfigContainer containing test metadata. + metric_container: MetricContainer containing metric data. + change_point_index: Index of the change point in the metric data. + max_results_to_display: Max number of results to display from the change + point index, in both directions of the change point index. Returns: str: Description used to fill the GitHub issues description. @@ -168,25 +162,30 @@ def get_issue_description( description = [] - description.append(_ISSUE_DESCRIPTION_TEMPLATE.format(test_id, metric_name)) + description.append( + _ISSUE_DESCRIPTION_TEMPLATE.format( + test_config_container.test_id, test_config_container.metric_name)) - if test_name: - description.append(("`test_name:` " + f'{test_name}')) + if test_config_container.test_name: + description.append(("`test_name:` " + f'{test_config_container.test_name}')) - if test_description: - description.append(("`Test description:` " + f'{test_description}')) + if test_config_container.test_description: + description.append( + ("`Test description:` " + f'{test_config_container.test_description}')) description.append('```') runs_to_display = [] max_timestamp_index = min( - change_point_index + max_results_to_display, len(metric_values) - 1) + change_point_index + max_results_to_display, + len(metric_container.values) - 1) min_timestamp_index = max(0, change_point_index - max_results_to_display) # run in reverse to display the most recent runs first. for i in reversed(range(min_timestamp_index, max_timestamp_index + 1)): row_template = _METRIC_INFO_TEMPLATE.format( - timestamps[i].ctime(), format(metric_values[i], '.2f')) + metric_container.timestamps[i].ctime(), + format(metric_container.values[i], '.2f')) if i == change_point_index: row_template += constants._ANOMALY_MARKER runs_to_display.append(row_template) diff --git a/sdks/python/apache_beam/testing/analyzers/perf_analysis.py b/sdks/python/apache_beam/testing/analyzers/perf_analysis.py index 109e5bfcc286..27f8398a0fb3 100644 --- a/sdks/python/apache_beam/testing/analyzers/perf_analysis.py +++ b/sdks/python/apache_beam/testing/analyzers/perf_analysis.py @@ -34,66 +34,103 @@ from apache_beam.testing.analyzers import constants from apache_beam.testing.analyzers.perf_analysis_utils import BigQueryMetricsFetcher +from apache_beam.testing.analyzers.perf_analysis_utils import ChangePointConfig from apache_beam.testing.analyzers.perf_analysis_utils import GitHubIssueMetaData from apache_beam.testing.analyzers.perf_analysis_utils import MetricsFetcher +from apache_beam.testing.analyzers.perf_analysis_utils import TestConfigContainer from apache_beam.testing.analyzers.perf_analysis_utils import create_performance_alert from apache_beam.testing.analyzers.perf_analysis_utils import find_latest_change_point_index from apache_beam.testing.analyzers.perf_analysis_utils import get_existing_issues_data from apache_beam.testing.analyzers.perf_analysis_utils import is_change_point_in_valid_window -from apache_beam.testing.analyzers.perf_analysis_utils import is_perf_alert +from apache_beam.testing.analyzers.perf_analysis_utils import is_sibling_change_point from apache_beam.testing.analyzers.perf_analysis_utils import publish_issue_metadata_to_big_query from apache_beam.testing.analyzers.perf_analysis_utils import read_test_config -from apache_beam.testing.analyzers.perf_analysis_utils import validate_config + + +def get_test_config_container( + params: Dict[str, Any], + test_id: str, +) -> TestConfigContainer: + """ + Args: + params: Dict containing parameters to run change point analysis. + Returns: + TestConfigContainer object containing test config parameters. + """ + return TestConfigContainer( + project=params['project'], + metrics_dataset=params['metrics_dataset'], + metrics_table=params['metrics_table'], + metric_name=params['metric_name'], + test_id=test_id, + test_description=params['test_description'], + test_name=params.get('test_name', None), + labels=params.get('labels', None), + ) + + +def get_change_point_config(params: Dict[str, Any], ) -> ChangePointConfig: + """ + Args: + params: Dict containing parameters to run change point analysis. + Returns: + ChangePointConfig object containing change point analysis parameters. + """ + return ChangePointConfig( + min_runs_between_change_points=params.get( + 'min_runs_between_change_points', + constants._DEFAULT_MIN_RUNS_BETWEEN_CHANGE_POINTS), + num_runs_in_change_point_window=params.get( + 'num_runs_in_change_point_window', + constants._DEFAULT_NUM_RUMS_IN_CHANGE_POINT_WINDOW)) def run_change_point_analysis( - params, test_id, big_query_metrics_fetcher: MetricsFetcher): + test_config_container: TestConfigContainer, + big_query_metrics_fetcher: MetricsFetcher, + change_point_config: ChangePointConfig = ChangePointConfig(), +): """ Args: - params: Dict containing parameters to run change point analysis. - test_id: Test id for the current test. + test_config_container: TestConfigContainer containing test metadata for + fetching data and running change point analysis. big_query_metrics_fetcher: BigQuery metrics fetcher used to fetch data for change point analysis. + change_point_config: ChangePointConfig containing parameters to run + change point analysis. Returns: bool indicating if a change point is observed and alerted on GitHub. """ - logging.info("Running change point analysis for test ID %s" % test_id) - if not validate_config(params.keys()): - raise ValueError( - f"Please make sure all these keys {constants._PERF_TEST_KEYS} " - f"are specified for the {test_id}") - - metric_name = params['metric_name'] + logging.info( + "Running change point analysis for test ID %s" % + test_config_container.test_id) # test_name will be used to query a single test from # multiple tests in a single BQ table. Right now, the default # assumption is that all the test have an individual BQ table # but this might not be case for other tests(such as IO tests where # a single BQ tables stores all the data) - test_name = params.get('test_name', None) + test_name = test_config_container.test_name min_runs_between_change_points = ( - constants._DEFAULT_MIN_RUNS_BETWEEN_CHANGE_POINTS) - if 'min_runs_between_change_points' in params: - min_runs_between_change_points = params['min_runs_between_change_points'] + change_point_config.min_runs_between_change_points) num_runs_in_change_point_window = ( - constants._DEFAULT_NUM_RUMS_IN_CHANGE_POINT_WINDOW) - if 'num_runs_in_change_point_window' in params: - num_runs_in_change_point_window = params['num_runs_in_change_point_window'] - - metric_values, timestamps = big_query_metrics_fetcher.fetch_metric_data( - project=params['project'], - metrics_dataset=params['metrics_dataset'], - metrics_table=params['metrics_table'], - metric_name=params['metric_name'], - test_name=test_name - ) + change_point_config.num_runs_in_change_point_window) + + metric_container = big_query_metrics_fetcher.fetch_metric_data( + test_config=test_config_container) + metric_container.sort_by_timestamp() + + metric_values = metric_container.values + timestamps = metric_container.timestamps change_point_index = find_latest_change_point_index( metric_values=metric_values) if not change_point_index: - logging.info("Change point is not detected for the test ID %s" % test_id) + logging.info( + "Change point is not detected for the test ID %s" % + test_config_container.test_id) return False # since timestamps are ordered in ascending order and # num_runs_in_change_point_window refers to the latest runs, @@ -107,15 +144,17 @@ def run_change_point_analysis( 'on metric %s. Since the change point run %s ' 'lies outside the num_runs_in_change_point_window distance: %s, ' 'alert is not raised.' % ( - test_id, - metric_name, + test_config_container.test_id, + test_config_container.metric_name, latest_change_point_run + 1, num_runs_in_change_point_window)) return False - is_alert = True + is_valid_change_point = True last_reported_issue_number = None - issue_metadata_table_name = f'{params.get("metrics_table")}_{metric_name}' + issue_metadata_table_name = ( + f'{test_config_container.metrics_table}_{test_config_container.metric_name}' # pylint: disable=line-too-long + ) existing_issue_data = get_existing_issues_data( table_name=issue_metadata_table_name) @@ -127,37 +166,39 @@ def run_change_point_analysis( # convert numpy.int64 to int last_reported_issue_number = last_reported_issue_number.item() - is_alert = is_perf_alert( + is_valid_change_point = is_sibling_change_point( previous_change_point_timestamps=existing_issue_timestamps, change_point_index=change_point_index, timestamps=timestamps, - min_runs_between_change_points=min_runs_between_change_points) - if is_alert: + min_runs_between_change_points=min_runs_between_change_points, + test_id=test_config_container.test_id) + if is_valid_change_point: issue_number, issue_url = create_performance_alert( - metric_name, test_id, timestamps, - metric_values, change_point_index, - params.get('labels', None), - last_reported_issue_number, - test_description = params.get('test_description', None), - test_name = test_name + test_config_container=test_config_container, + metric_container=metric_container, + change_point_index=change_point_index, + existing_issue_number=last_reported_issue_number, ) issue_metadata = GitHubIssueMetaData( issue_timestamp=pd.Timestamp( datetime.now().replace(tzinfo=timezone.utc)), # BQ doesn't allow '.' in table name - test_id=test_id.replace('.', '_'), + test_id=test_config_container.test_id.replace('.', '_'), test_name=test_name or uuid.uuid4().hex, - metric_name=metric_name, + metric_name=test_config_container.metric_name, change_point=metric_values[change_point_index], issue_number=issue_number, issue_url=issue_url, - change_point_timestamp=timestamps[change_point_index]) + change_point_timestamp=timestamps[change_point_index], + ) publish_issue_metadata_to_big_query( - issue_metadata=issue_metadata, table_name=issue_metadata_table_name) - - return is_alert + issue_metadata=issue_metadata, + table_name=issue_metadata_table_name, + project=test_config_container.project, + ) + return is_valid_change_point def run( @@ -185,10 +226,13 @@ def run( tests_config: Dict[str, Dict[str, Any]] = read_test_config(config_file_path) for test_id, params in tests_config.items(): + test_config_container = get_test_config_container(params, test_id=test_id) + change_point_config = get_change_point_config(params) run_change_point_analysis( - params=params, - test_id=test_id, - big_query_metrics_fetcher=big_query_metrics_fetcher) + test_config_container=test_config_container, + big_query_metrics_fetcher=big_query_metrics_fetcher, + change_point_config=change_point_config, + ) if __name__ == '__main__': diff --git a/sdks/python/apache_beam/testing/analyzers/perf_analysis_test.py b/sdks/python/apache_beam/testing/analyzers/perf_analysis_test.py index 15344ab13b3a..5164c8d8fd36 100644 --- a/sdks/python/apache_beam/testing/analyzers/perf_analysis_test.py +++ b/sdks/python/apache_beam/testing/analyzers/perf_analysis_test.py @@ -16,7 +16,6 @@ # # pytype: skip-file -import datetime import logging import os import re @@ -33,15 +32,18 @@ from apache_beam.testing.analyzers import constants from apache_beam.testing.analyzers import github_issues_utils from apache_beam.testing.analyzers.perf_analysis_utils import BigQueryMetricsFetcher + from apache_beam.testing.analyzers.perf_analysis_utils import MetricContainer + from apache_beam.testing.analyzers.perf_analysis_utils import TestConfigContainer from apache_beam.testing.analyzers.perf_analysis_utils import is_change_point_in_valid_window from apache_beam.testing.analyzers.perf_analysis_utils import is_edge_change_point - from apache_beam.testing.analyzers.perf_analysis_utils import is_perf_alert + from apache_beam.testing.analyzers.perf_analysis_utils import is_sibling_change_point from apache_beam.testing.analyzers.perf_analysis_utils import e_divisive from apache_beam.testing.analyzers.perf_analysis_utils import filter_change_points_by_median_threshold from apache_beam.testing.analyzers.perf_analysis_utils import find_change_points from apache_beam.testing.analyzers.perf_analysis_utils import find_latest_change_point_index from apache_beam.testing.analyzers.perf_analysis_utils import validate_config from apache_beam.testing.load_tests import load_test_metrics_utils + except ImportError as e: raise unittest.SkipTest('Missing dependencies to run perf analysis tests.') @@ -50,23 +52,23 @@ def get_fake_data_with_no_change_point(*args, **kwargs): num_samples = 20 metric_values = [1] * num_samples - timestamps = list(range(num_samples)) - return metric_values, timestamps + timestamps = [pd.Timestamp(i) for i in range(num_samples)] + return MetricContainer(metric_values, timestamps) def get_fake_data_with_change_point(*args, **kwargs): # change point will be at index 13. num_samples = 20 metric_values = [0] * 12 + [3] + [4] * 7 - timestamps = [i for i in range(num_samples)] - return metric_values, timestamps + timestamps = [pd.Timestamp(i) for i in range(num_samples)] + return MetricContainer(metric_values, timestamps) def get_existing_issue_data(**kwargs): # change point found at index 13. So passing 13 in the # existing issue data in mock method. return pd.DataFrame([{ - constants._CHANGE_POINT_TIMESTAMP_LABEL: 13, + constants._CHANGE_POINT_TIMESTAMP_LABEL: pd.Timestamp(13), constants._ISSUE_NUMBER: np.array([0]) }]) @@ -77,7 +79,7 @@ def setUp(self) -> None: self.multiple_change_point_series = self.single_change_point_series + [ 2 ] * 20 - self.timestamps = list(range(5)) + self.timestamps = [pd.Timestamp(i) for i in range(5)] self.params = { 'test_description': 'fake_description', 'metrics_dataset': 'fake_dataset', @@ -123,30 +125,33 @@ def test_validate_config(self): def test_duplicate_change_point(self): change_point_index = 2 min_runs_between_change_points = 1 - is_alert = is_perf_alert( + is_alert = is_sibling_change_point( previous_change_point_timestamps=[self.timestamps[0]], timestamps=self.timestamps, change_point_index=change_point_index, - min_runs_between_change_points=min_runs_between_change_points) + min_runs_between_change_points=min_runs_between_change_points, + test_id=self.test_id) self.assertTrue(is_alert) def test_duplicate_change_points_are_not_valid_alerts(self): change_point_index = 2 min_runs_between_change_points = 1 - is_alert = is_perf_alert( + is_alert = is_sibling_change_point( previous_change_point_timestamps=[self.timestamps[3]], timestamps=self.timestamps, change_point_index=change_point_index, - min_runs_between_change_points=min_runs_between_change_points) + min_runs_between_change_points=min_runs_between_change_points, + test_id=self.test_id) self.assertFalse(is_alert) - is_alert = is_perf_alert( + is_alert = is_sibling_change_point( previous_change_point_timestamps=[ self.timestamps[0], self.timestamps[3] ], timestamps=self.timestamps, change_point_index=change_point_index, - min_runs_between_change_points=min_runs_between_change_points) + min_runs_between_change_points=min_runs_between_change_points, + test_id=self.test_id) self.assertFalse(is_alert) @mock.patch.object( @@ -154,9 +159,10 @@ def test_duplicate_change_points_are_not_valid_alerts(self): 'fetch_metric_data', get_fake_data_with_no_change_point) def test_no_alerts_when_no_change_points(self): + test_config_container = analysis.get_test_config_container( + params=self.params, test_id=self.test_id) is_alert = analysis.run_change_point_analysis( - params=self.params, - test_id=self.test_id, + test_config_container=test_config_container, big_query_metrics_fetcher=BigQueryMetricsFetcher()) self.assertFalse(is_alert) @@ -176,9 +182,10 @@ def test_no_alerts_when_no_change_points(self): '.create_performance_alert', return_value=(0, '')) def test_alert_on_data_with_change_point(self, *args): + test_config_container = analysis.get_test_config_container( + params=self.params, test_id=self.test_id) is_alert = analysis.run_change_point_analysis( - params=self.params, - test_id=self.test_id, + test_config_container=test_config_container, big_query_metrics_fetcher=BigQueryMetricsFetcher()) self.assertTrue(is_alert) @@ -197,24 +204,32 @@ def test_alert_on_data_with_change_point(self, *args): 'apache_beam.testing.analyzers.perf_analysis.create_performance_alert', return_value=(0, '')) def test_alert_on_data_with_reported_change_point(self, *args): + test_config_container = analysis.get_test_config_container( + params=self.params, test_id=self.test_id) is_alert = analysis.run_change_point_analysis( - params=self.params, - test_id=self.test_id, + test_config_container=test_config_container, big_query_metrics_fetcher=BigQueryMetricsFetcher()) self.assertFalse(is_alert) def test_change_point_has_anomaly_marker_in_gh_description(self): - metric_values, timestamps = get_fake_data_with_change_point() - timestamps = [datetime.datetime.fromtimestamp(ts) for ts in timestamps] + metric_container = get_fake_data_with_change_point() + metric_values = metric_container.values change_point_index = find_latest_change_point_index(metric_values) - description = github_issues_utils.get_issue_description( + test_config_container = TestConfigContainer( + project=self.params['project'], + metrics_dataset=self.params['metrics_dataset'], + metrics_table=self.params['metrics_table'], + metric_name=self.params['metric_name'], test_id=self.test_id, - test_name=self.params.get('test_name', None), test_description=self.params['test_description'], - metric_name=self.params['metric_name'], - metric_values=metric_values, - timestamps=timestamps, + test_name=self.params.get('test_name', None), + labels=self.params.get('labels', None), + ) + + description = github_issues_utils.get_issue_description( + test_config_container=test_config_container, + metric_container=metric_container, change_point_index=change_point_index, max_results_to_display=( constants._NUM_RESULTS_TO_DISPLAY_ON_ISSUE_DESCRIPTION)) diff --git a/sdks/python/apache_beam/testing/analyzers/perf_analysis_utils.py b/sdks/python/apache_beam/testing/analyzers/perf_analysis_utils.py index 91c339a766d2..2b89ac9fdba9 100644 --- a/sdks/python/apache_beam/testing/analyzers/perf_analysis_utils.py +++ b/sdks/python/apache_beam/testing/analyzers/perf_analysis_utils.py @@ -31,7 +31,6 @@ from google.cloud import bigquery from apache_beam.testing.analyzers import constants -from apache_beam.testing.analyzers import github_issues_utils from apache_beam.testing.load_tests import load_test_metrics_utils from apache_beam.testing.load_tests.load_test_metrics_utils import BigQueryMetricsPublisher from signal_processing_algorithms.energy_statistics.energy_statistics import e_divisive @@ -54,6 +53,54 @@ class GitHubIssueMetaData: change_point: float +@dataclass +class ChangePointConfig: + """ + This class holds the change point configuration parameters. + """ + min_runs_between_change_points: int = ( + constants._DEFAULT_MIN_RUNS_BETWEEN_CHANGE_POINTS) + num_runs_in_change_point_window: int = ( + constants._DEFAULT_NUM_RUMS_IN_CHANGE_POINT_WINDOW) + + +@dataclass +class TestConfigContainer: + metric_name: str # make this list instead. + project: str + metrics_dataset: str + metrics_table: str + test_id: str # unique id for each test config. + test_description: str + test_name: Optional[str] = None + labels: Optional[List[str]] = None + + +@dataclass +class MetricContainer: + """ + This class holds the metric values and timestamps for a given metric. + Args: + metric_values: List of metric values. + timestamps: List of pandas timestamps corresponding to the metric values. + """ + + values: List[Union[int, float]] + timestamps: List[pd.Timestamp] + + def sort_by_timestamp(self, in_place=True): + """ + Sorts the metric values and timestamps in ascending order wrt timestamps. + Args: + in_place: If True, sort the metric values and timestamps in place. + """ + timestamps, values = zip(*sorted(zip(self.timestamps, self.values))) + if not in_place: + return MetricContainer(values=values, timestamps=timestamps) + self.timestamps, self.values = zip(*sorted( + zip(self.timestamps, self.values))) + + def is_change_point_in_valid_window( num_runs_in_change_point_window: int, latest_change_point_run: int) -> bool: return num_runs_in_change_point_window > latest_change_point_run @@ -81,12 +128,16 @@ def get_existing_issues_data(table_name: str) -> Optional[pd.DataFrame]: return existing_issue_data -def is_perf_alert( +def is_sibling_change_point( previous_change_point_timestamps: List[pd.Timestamp], change_point_index: int, timestamps: List[pd.Timestamp], - min_runs_between_change_points: int) -> bool: + min_runs_between_change_points: int, + test_id: str, +) -> bool: """ + Sibling change points are the change points that are close to each other. + Search the previous_change_point_timestamps with current observed change point sibling window and determine if it is a duplicate change point or not. @@ -105,6 +156,18 @@ def is_perf_alert( for previous_change_point_timestamp in previous_change_point_timestamps: if (sibling_change_point_min_timestamp <= previous_change_point_timestamp <= sibling_change_point_max_timestamp): + logging.info( + 'Performance regression/improvement found for the test ID: %s. ' + 'Since the change point timestamp %s ' + 'lies within the sibling change point window: %s, ' + 'alert is not raised.' % ( + test_id, + previous_change_point_timestamp.strftime('%Y-%m-%d %H:%M:%S'), + ( + sibling_change_point_min_timestamp.strftime( + '%Y-%m-%d %H:%M:%S'), + sibling_change_point_max_timestamp.strftime( + '%Y-%m-%d %H:%M:%S')))) return False return True @@ -161,12 +224,16 @@ def find_latest_change_point_index(metric_values: List[Union[float, int]]): return change_point_index -def publish_issue_metadata_to_big_query(issue_metadata, table_name): +def publish_issue_metadata_to_big_query( + issue_metadata, + table_name, + project=constants._BQ_PROJECT_NAME, +): """ Published issue_metadata to BigQuery with table name. """ bq_metrics_publisher = BigQueryMetricsPublisher( - project_name=constants._BQ_PROJECT_NAME, + project_name=project, dataset=constants._BQ_DATASET, table=table_name, bq_schema=constants._SCHEMA) @@ -177,37 +244,32 @@ def publish_issue_metadata_to_big_query(issue_metadata, table_name): def create_performance_alert( - metric_name: str, - test_id: str, - timestamps: List[pd.Timestamp], - metric_values: List[Union[int, float]], + test_config_container: TestConfigContainer, + metric_container: MetricContainer, change_point_index: int, - labels: List[str], existing_issue_number: Optional[int], - test_description: Optional[str] = None, - test_name: Optional[str] = None, ) -> Tuple[int, str]: """ Creates performance alert on GitHub issues and returns GitHub issue number and issue URL. """ + # avoid circular imports + # pylint: disable=wrong-import-order, wrong-import-position + from apache_beam.testing.analyzers import github_issues_utils + description = github_issues_utils.get_issue_description( - test_id=test_id, - test_name=test_name, - test_description=test_description, - metric_name=metric_name, - timestamps=timestamps, - metric_values=metric_values, + test_config_container=test_config_container, + metric_container=metric_container, change_point_index=change_point_index, max_results_to_display=( constants._NUM_RESULTS_TO_DISPLAY_ON_ISSUE_DESCRIPTION)) issue_number, issue_url = github_issues_utils.report_change_point_on_issues( title=github_issues_utils._ISSUE_TITLE_TEMPLATE.format( - test_id, metric_name + test_config_container.test_id, test_config_container.metric_name ), description=description, - labels=labels, + labels=test_config_container.labels, existing_issue_number=existing_issue_number) logging.info( @@ -263,13 +325,7 @@ def is_edge_change_point( class MetricsFetcher(metaclass=abc.ABCMeta): @abc.abstractmethod def fetch_metric_data( - self, - *, - project, - metrics_dataset, - metrics_table, - metric_name, - test_name=None) -> Tuple[List[Union[int, float]], List[pd.Timestamp]]: + self, *, test_config: TestConfigContainer) -> MetricContainer: """ Define SQL query and fetch the timestamp values and metric values from BigQuery tables. @@ -279,22 +335,18 @@ def fetch_metric_data( class BigQueryMetricsFetcher(MetricsFetcher): def fetch_metric_data( - self, - *, - project, - metrics_dataset, - metrics_table, - metric_name, - test_name=None, - ) -> Tuple[List[Union[int, float]], List[pd.Timestamp]]: + self, *, test_config: TestConfigContainer) -> MetricContainer: """ Args: - params: Dict containing keys required to fetch data from a data source. + test_config: TestConfigContainer containing metadata required to fetch + metric data from BigQuery. Returns: - Tuple[List[Union[int, float]], List[pd.Timestamp]]: Tuple containing list - of metric_values and list of timestamps. Both are sorted in ascending - order wrt timestamps. + MetricContainer containing metric values and timestamps. """ + project = test_config.project + metrics_dataset = test_config.metrics_dataset + metrics_table = test_config.metrics_table + metric_name = test_config.metric_name query = f""" SELECT * FROM {project}.{metrics_dataset}.{metrics_table} @@ -305,8 +357,9 @@ def fetch_metric_data( client = bigquery.Client() query_job = client.query(query=query) metric_data = query_job.result().to_dataframe() - metric_data.sort_values( - by=[load_test_metrics_utils.SUBMIT_TIMESTAMP_LABEL], inplace=True) - return ( - metric_data[load_test_metrics_utils.VALUE_LABEL].tolist(), - metric_data[load_test_metrics_utils.SUBMIT_TIMESTAMP_LABEL].tolist()) + # metric_data.sort_values( + # by=[load_test_metrics_utils.SUBMIT_TIMESTAMP_LABEL], inplace=True) + return MetricContainer( + values=metric_data[load_test_metrics_utils.VALUE_LABEL].tolist(), + timestamps=metric_data[ + load_test_metrics_utils.SUBMIT_TIMESTAMP_LABEL].tolist()) diff --git a/sdks/python/apache_beam/testing/analyzers/tests_config.yaml b/sdks/python/apache_beam/testing/analyzers/tests_config.yaml index ec9cfe6f1ac0..2e72cd5cc301 100644 --- a/sdks/python/apache_beam/testing/analyzers/tests_config.yaml +++ b/sdks/python/apache_beam/testing/analyzers/tests_config.yaml @@ -19,7 +19,7 @@ # {test_id}-{metric_name} pytorch_image_classification_benchmarks-resnet152-mean_inference_batch_latency_micro_secs: - test_description: + test_description: | Pytorch image classification on 50k images of size 224 x 224 with resnet 152. Test link - https://github.com/apache/beam/blob/42d0a6e3564d8b9c5d912428a6de18fb22a13ac1/.test-infra/jenkins/job_InferenceBenchmarkTests_Python.groovy#L63 Test dashboard - http://metrics.beam.apache.org/d/ZpS8Uf44z/python-ml-runinference-benchmarks?orgId=1&viewPanel=2 @@ -30,7 +30,7 @@ pytorch_image_classification_benchmarks-resnet152-mean_inference_batch_latency_m metric_name: mean_inference_batch_latency_micro_secs pytorch_image_classification_benchmarks-resnet101-mean_load_model_latency_milli_secs: - test_description: + test_description: | Pytorch image classification on 50k images of size 224 x 224 with resnet 101. Test link - https://github.com/apache/beam/blob/42d0a6e3564d8b9c5d912428a6de18fb22a13ac1/.test-infra/jenkins/job_InferenceBenchmarkTests_Python.groovy#L34 Test dashboard - http://metrics.beam.apache.org/d/ZpS8Uf44z/python-ml-runinference-benchmarks?orgId=1&viewPanel=7 @@ -41,7 +41,7 @@ pytorch_image_classification_benchmarks-resnet101-mean_load_model_latency_milli_ metric_name: mean_load_model_latency_milli_secs pytorch_image_classification_benchmarks-resnet101-mean_inference_batch_latency_micro_secs: - test_description: + test_description: | Pytorch image classification on 50k images of size 224 x 224 with resnet 101. Test link - https://github.com/apache/beam/blob/42d0a6e3564d8b9c5d912428a6de18fb22a13ac1/.test-infra/jenkins/job_InferenceBenchmarkTests_Python.groovy#L34 Test dashboard - http://metrics.beam.apache.org/d/ZpS8Uf44z/python-ml-runinference-benchmarks?orgId=1&viewPanel=2 @@ -52,7 +52,7 @@ pytorch_image_classification_benchmarks-resnet101-mean_inference_batch_latency_m metric_name: mean_inference_batch_latency_micro_secs pytorch_image_classification_benchmarks-resnet152-GPU-mean_inference_batch_latency_micro_secs: - test_description: + test_description: | Pytorch image classification on 50k images of size 224 x 224 with resnet 152 with Tesla T4 GPU. Test link - https://github.com/apache/beam/blob/42d0a6e3564d8b9c5d912428a6de18fb22a13ac1/.test-infra/jenkins/job_InferenceBenchmarkTests_Python.groovy#L151 Test dashboard - http://metrics.beam.apache.org/d/ZpS8Uf44z/python-ml-runinference-benchmarks?orgId=1&viewPanel=7 @@ -63,7 +63,7 @@ pytorch_image_classification_benchmarks-resnet152-GPU-mean_inference_batch_laten metric_name: mean_inference_batch_latency_micro_secs pytorch_image_classification_benchmarks-resnet152-GPU-mean_load_model_latency_milli_secs: - test_description: + test_description: | Pytorch image classification on 50k images of size 224 x 224 with resnet 152 with Tesla T4 GPU. Test link - https://github.com/apache/beam/blob/42d0a6e3564d8b9c5d912428a6de18fb22a13ac1/.test-infra/jenkins/job_InferenceBenchmarkTests_Python.groovy#L151 Test dashboard - http://metrics.beam.apache.org/d/ZpS8Uf44z/python-ml-runinference-benchmarks?orgId=1&viewPanel=7 @@ -74,7 +74,7 @@ pytorch_image_classification_benchmarks-resnet152-GPU-mean_load_model_latency_mi metric_name: mean_load_model_latency_milli_secs pytorch_image_classification_benchmarks-resnet152-GPU-mean_inference_batch_latency_micro_secs: - test_description: + test_description: | Pytorch image classification on 50k images of size 224 x 224 with resnet 152 with Tesla T4 GPU. Test link - https://github.com/apache/beam/blob/42d0a6e3564d8b9c5d912428a6de18fb22a13ac1/.test-infra/jenkins/job_InferenceBenchmarkTests_Python.groovy#L151). Test dashboard - http://metrics.beam.apache.org/d/ZpS8Uf44z/python-ml-runinference-benchmarks?from=now-90d&to=now&viewPanel=2 @@ -85,7 +85,7 @@ pytorch_image_classification_benchmarks-resnet152-GPU-mean_inference_batch_laten metric_name: mean_inference_batch_latency_micro_secs test_cloudml_benchmark_cirteo_no_shuffle_10GB-runtime_sec: - test_description: + test_description: | TFT Criteo test on 10 GB data with no Reshuffle. Test link - [Test link](https://github.com/apache/beam/blob/42d0a6e3564d8b9c5d912428a6de18fb22a13ac1/sdks/python/apache_beam/testing/benchmarks/cloudml/cloudml_benchmark_test.py#L82) metrics_dataset: beam_cloudml @@ -94,7 +94,7 @@ test_cloudml_benchmark_cirteo_no_shuffle_10GB-runtime_sec: metric_name: runtime_sec test_cloudml_benchmark_criteo_10GB-runtime_sec: - test_description: + test_description: | TFT Criteo test on 10 GB data. Test link - https://github.com/apache/beam/blob/42d0a6e3564d8b9c5d912428a6de18fb22a13ac1/sdks/python/apache_beam/testing/benchmarks/cloudml/cloudml_benchmark_test.py#LL104C7-L104C41 metrics_dataset: beam_cloudml @@ -104,7 +104,7 @@ test_cloudml_benchmark_criteo_10GB-runtime_sec: # Python Combine load tests at http://metrics.beam.apache.org/d/WNzYt13Zk/combine-load-tests?orgId=1 combine_python_batch_2gb_10_byte_records: - test_description: + test_description: | Combine Python Load Test 2 GB 10 byte records Test link - https://github.com/apache/beam/blob/5e38decf9e723a385057131b01bbd33d8c60bda3/.test-infra/jenkins/job_LoadTests_Combine_Python.groovy#L76C24-L76C65 Test dashboard - http://metrics.beam.apache.org/d/WNzYt13Zk/combine-load-tests?orgId=1&from=now-90d&to=now&var-processingType=batch&var-sdk=python&viewPanel=2 @@ -115,7 +115,7 @@ combine_python_batch_2gb_10_byte_records: project: apache-beam-testing combine_python_batch_2gb_fanout_4: - test_description: + test_description: | Combine Python Load test - 2GB Fanout 4 Test link - https://github.com/apache/beam/blob/5e38decf9e723a385057131b01bbd33d8c60bda3/.test-infra/jenkins/job_LoadTests_Combine_Python.groovy#L52 Test Dashboard - http://metrics.beam.apache.org/d/WNzYt13Zk/combine-load-tests?orgId=1&from=now-90d&to=now&var-processingType=batch&var-sdk=python&viewPanel=4 @@ -126,7 +126,8 @@ combine_python_batch_2gb_fanout_4: project: apache-beam-testing combine_python_batch_2gb_fanout_8: - test_description: Combine Python Load test - 2GB Fanout 8 + test_description: | + Combine Python Load test - 2GB Fanout 8 test_target: apache_beam.testing.load_tests.combine_test metrics_dataset: load_test metrics_table: python_dataflow_batch_combine_5 @@ -135,7 +136,7 @@ combine_python_batch_2gb_fanout_8: # Python Batch GBK load tests at http://metrics.beam.apache.org/d/UYZ-oJ3Zk/gbk-load-tests?orgId=1&var-processingType=batch&var-sdk=python&from=now-90d&to=now gbk_python_batch_load_test_2gb_of_10B_records: - test_description: + test_description: | GroupByKey Python Load test - 2GB of 10B records python | GBK | Small records (10B) Test Dashboard - http://metrics.beam.apache.org/d/UYZ-oJ3Zk/gbk-load-tests?orgId=1&var-processingType=batch&var-sdk=python&from=now-90d&to=now&viewPanel=2 @@ -147,7 +148,7 @@ gbk_python_batch_load_test_2gb_of_10B_records: project: apache-beam-testing gbk_python_batch_load_test_2gb_of_100B_records: - test_description: + test_description: | GroupByKey Python Load test - 2GB of 100B records python | GBK | Medium records (100B) Test Dashboard - http://metrics.beam.apache.org/d/UYZ-oJ3Zk/gbk-load-tests?orgId=1&var-processingType=batch&var-sdk=python&from=now-90d&to=now&viewPanel=3 @@ -159,7 +160,7 @@ gbk_python_batch_load_test_2gb_of_100B_records: project: apache-beam-testing gbk_python_batch_load_test_2gb_of_100KB_records: - test_description: + test_description: | GroupByKey Python Load test - 2GB of 100kB records python | GBK | Large records (100kB) Test Dashboard - http://metrics.beam.apache.org/d/UYZ-oJ3Zk/gbk-load-tests?orgId=1&var-processingType=batch&var-sdk=python&from=now-6M&to=now&viewPanel=4&inspect=4 @@ -173,7 +174,7 @@ gbk_python_batch_load_test_2gb_of_100KB_records: gbk_python_batch_load_test_fanout_4_times_with_2GB_10byte_records_total: # this test looks little noisy. Remove this if it causes too many false # positives. - test_description: + test_description: | GroupByKey Python Load test - fanout 4 times with 2GB 10-byte records total python | GBK | Fanout 4 Test Dashboard - http://metrics.beam.apache.org/d/UYZ-oJ3Zk/gbk-load-tests?orgId=1&var-processingType=batch&var-sdk=python&from=now-90d&to=now&viewPanel=5 @@ -188,7 +189,7 @@ gbk_python_batch_load_test_fanout_4_times_with_2GB_10byte_records_total: gbk_python_batch_load_test_fanout_8_times_with_2GB_10byte_records_total: # this test looks little noisy. Remove this if it causes too many false # positives. - test_description: + test_description: | GroupByKey Python Load test - fanout 8 times with 2GB 10-byte records total python | GBK | Fanout 8 Test Dashboard - http://metrics.beam.apache.org/d/UYZ-oJ3Zk/gbk-load-tests?orgId=1&var-processingType=batch&var-sdk=python&from=now-90d&to=now&viewPanel=6 @@ -201,7 +202,7 @@ gbk_python_batch_load_test_fanout_8_times_with_2GB_10byte_records_total: # Python SideInput load tests at http://metrics.beam.apache.org/d/-E9aGlFGk/side-input-load-tests?orgId=1&from=now-90d&to=now sideinpts_python_batch_1gb_1kb_10workers_1000window_1key_percent_dict: - test_description: + test_description: | python | Side Input | 1 GB dictionary, 1% of keys, 1000 fixed windows Test Link - https://github.com/apache/beam/blob/5e38decf9e723a385057131b01bbd33d8c60bda3/.test-infra/jenkins/job_LoadTests_SideInput_Python.groovy#L120 Test Dashboard - http://metrics.beam.apache.org/d/-E9aGlFGk/side-input-load-tests?orgId=1&from=now-90d&to=now&viewPanel=8 @@ -212,7 +213,7 @@ sideinpts_python_batch_1gb_1kb_10workers_1000window_1key_percent_dict: sideinpts_python_batch_1gb_1kb_10workers_1000window_99key_percent_dict: - test_description: + test_description: | python | Side Input | 1 GB dictionary, 99% of keys, 1000 fixed windows Test Link - https://github.com/apache/beam/blob/5e38decf9e723a385057131b01bbd33d8c60bda3/.test-infra/jenkins/job_LoadTests_SideInput_Python.groovy#L133 Test Dashboard - http://metrics.beam.apache.org/d/-E9aGlFGk/side-input-load-tests?orgId=1&from=now-90d&to=now&viewPanel=9 @@ -222,7 +223,7 @@ sideinpts_python_batch_1gb_1kb_10workers_1000window_99key_percent_dict: project: apache-beam-testing sideinpts_python_batch_10gb_1kb_10workers_1000window_first_iterable: - test_description: + test_description: | python | Side Input | 10 GB iterable, 1% of elements, 1000 fixed windows Test Link - https://github.com/apache/beam/blob/5e38decf9e723a385057131b01bbd33d8c60bda3/.test-infra/jenkins/job_LoadTests_SideInput_Python.groovy#L146 Test Dashboard - http://metrics.beam.apache.org/d/-E9aGlFGk/side-input-load-tests?orgId=1&from=now-90d&to=now&viewPanel=10 @@ -233,7 +234,7 @@ sideinpts_python_batch_10gb_1kb_10workers_1000window_first_iterable: sideinpts_python_batch_10gb_1kb_10workers_1000window_first_iterable: - test_description: + test_description: | python | Side Input | 10 GB iterable, all elements, 1000 fixed windows Test Link - https://github.com/apache/beam/blob/5e38decf9e723a385057131b01bbd33d8c60bda3/.test-infra/jenkins/job_LoadTests_SideInput_Python.groovy#L159 Test Dashboard - http://metrics.beam.apache.org/d/-E9aGlFGk/side-input-load-tests?orgId=1&from=now-90d&to=now&viewPanel=11 @@ -245,7 +246,7 @@ sideinpts_python_batch_10gb_1kb_10workers_1000window_first_iterable: # Python CoGBK load tests at http://metrics.beam.apache.org/d/fK0U4JqWz/cogbk-load-tests?orgId=1&var-processingType=batch&var-sdk=python cogbk_python_batch_load_test_2GB_of_100B_records_with_a_single_key: - test_description: + test_description: | CoGroupByKey Python Load test - 2GB of 100B records with a single key python | coGBK | 100B records with a single key Test Link - https://github.com/apache/beam/blob/5e38decf9e723a385057131b01bbd33d8c60bda3/.test-infra/jenkins/job_LoadTests_coGBK_Python.groovy#L32C25-L32C76 @@ -257,7 +258,7 @@ cogbk_python_batch_load_test_2GB_of_100B_records_with_a_single_key: project: apache-beam-testing cogbk_python_batch_load_test_2GB_of_100B_records_with_a_multiple_key: - test_description: + test_description: | CoGroupByKey Python Load test - 2GB of 100B records with multiple keys python | coGBK | 100B records with multiple keys @@ -270,7 +271,7 @@ cogbk_python_batch_load_test_2GB_of_100B_records_with_a_multiple_key: project: apache-beam-testing cogbk_python_batch_load_test_reiterate_4times_10KB_values: - test_description: + test_description: | CoGroupByKey Python Load test - reiterate 4 times 10kB values python | coGBK | reiteration 10kB value Test Link - https://github.com/apache/beam/blob/5e38decf9e723a385057131b01bbd33d8c60bda3/.test-infra/jenkins/job_LoadTests_coGBK_Python.groovy#L96 @@ -281,7 +282,7 @@ cogbk_python_batch_load_test_reiterate_4times_10KB_values: project: apache-beam-testing cogbk_python_batch_load_test_reiterate_4times_2MB_values: - test_description: + test_description: | CoGroupByKey Python Load test - reiterate 4 times 2 MB values python | coGBK | reiteration 2MB value Test Link - https://github.com/apache/beam/blob/5e38decf9e723a385057131b01bbd33d8c60bda3/.test-infra/jenkins/job_LoadTests_coGBK_Python.groovy#L128 From d17063412ea4a3417b5a4e3f2d14fa1640c8caed Mon Sep 17 00:00:00 2001 From: RyuSA <12961775+RyuSA@users.noreply.github.com> Date: Tue, 10 Oct 2023 01:31:46 +0900 Subject: [PATCH 072/435] Update BigQueryIO Documentation (#28591) * Update BigQueryIO Documentation - Updated the description regarding failed rows for Storage Write API. - Made `PCollection` formatting consistent. * Update website/www/site/content/en/documentation/io/built-in/google-bigquery.md --- .../documentation/io/built-in/google-bigquery.md | 14 ++++++-------- 1 file changed, 6 insertions(+), 8 deletions(-) diff --git a/website/www/site/content/en/documentation/io/built-in/google-bigquery.md b/website/www/site/content/en/documentation/io/built-in/google-bigquery.md index 7a31b63a3c96..26ca0baec0cf 100644 --- a/website/www/site/content/en/documentation/io/built-in/google-bigquery.md +++ b/website/www/site/content/en/documentation/io/built-in/google-bigquery.md @@ -883,10 +883,9 @@ explicitly enable this using [`withAutoSharding`](https://beam.apache.org/releas ***Note:*** Auto sharding with `STORAGE_WRITE_API` is supported on Dataflow's legacy runner, but **not** on Runner V2 {{< /paragraph >}} -When using `STORAGE_WRITE_API`, the PCollection returned by -[`WriteResult.getFailedInserts`](https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/gcp/bigquery/WriteResult.html#getFailedInserts--) -will not contain the failed rows. If there are data validation errors, the -transform will throw a `RuntimeException`. +When using `STORAGE_WRITE_API`, the `PCollection` returned by +[`WriteResult.getFailedStorageApiInserts`](https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/gcp/bigquery/WriteResult.html#getFailedStorageApiInserts--) +will contain the rows that failed to be written to the Storage Write API sink. #### At-least-once semantics @@ -901,10 +900,9 @@ specify the number of streams, and you can’t specify the triggering frequency. Auto sharding is not applicable for `STORAGE_API_AT_LEAST_ONCE`. -When using `STORAGE_API_AT_LEAST_ONCE`, the PCollection returned by -[`WriteResult.getFailedInserts`](https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/gcp/bigquery/WriteResult.html#getFailedInserts--) -will not contain the failed rows. If there are data validation errors, the -transform will throw a `RuntimeException`. +When using `STORAGE_API_AT_LEAST_ONCE`, the `PCollection` returned by +[`WriteResult.getFailedStorageApiInserts`](https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/gcp/bigquery/WriteResult.html#getFailedStorageApiInserts--) +will contain the rows that failed to be written to the Storage Write API sink. #### Quotas From 66bae3fa5bad949250688f45d34e22575c0ac1c0 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 9 Oct 2023 13:18:14 -0400 Subject: [PATCH 073/435] Bump github.com/aws/aws-sdk-go-v2/feature/s3/manager in /sdks (#28890) Bumps [github.com/aws/aws-sdk-go-v2/feature/s3/manager](https://github.com/aws/aws-sdk-go-v2) from 1.11.87 to 1.11.89. - [Release notes](https://github.com/aws/aws-sdk-go-v2/releases) - [Commits](https://github.com/aws/aws-sdk-go-v2/compare/feature/s3/manager/v1.11.87...feature/s3/manager/v1.11.89) --- updated-dependencies: - dependency-name: github.com/aws/aws-sdk-go-v2/feature/s3/manager dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 34 ++++++++++++------------ sdks/go.sum | 74 ++++++++++++++++++++++++----------------------------- 2 files changed, 51 insertions(+), 57 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index b8dd03de6b07..d5b7d7e3e41e 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -31,10 +31,10 @@ require ( cloud.google.com/go/spanner v1.50.0 cloud.google.com/go/storage v1.33.0 github.com/aws/aws-sdk-go-v2 v1.21.1 - github.com/aws/aws-sdk-go-v2/config v1.18.43 - github.com/aws/aws-sdk-go-v2/credentials v1.13.41 - github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.87 - github.com/aws/aws-sdk-go-v2/service/s3 v1.40.0 + github.com/aws/aws-sdk-go-v2/config v1.18.44 + github.com/aws/aws-sdk-go-v2/credentials v1.13.42 + github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.89 + github.com/aws/aws-sdk-go-v2/service/s3 v1.40.1 github.com/aws/smithy-go v1.15.0 github.com/docker/go-connections v0.4.0 github.com/dustin/go-humanize v1.0.1 @@ -97,19 +97,19 @@ require ( github.com/apache/arrow/go/v12 v12.0.0 // indirect github.com/apache/thrift v0.16.0 // indirect github.com/aws/aws-sdk-go v1.34.0 // indirect - github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.4.13 // indirect - github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.13.11 // indirect - github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.41 // indirect - github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.35 // indirect - github.com/aws/aws-sdk-go-v2/internal/ini v1.3.43 // indirect - github.com/aws/aws-sdk-go-v2/internal/v4a v1.1.4 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.9.14 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.1.36 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.35 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.15.4 // indirect - github.com/aws/aws-sdk-go-v2/service/sso v1.15.0 // indirect - github.com/aws/aws-sdk-go-v2/service/ssooidc v1.17.1 // indirect - github.com/aws/aws-sdk-go-v2/service/sts v1.23.0 // indirect + github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.4.14 // indirect + github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.13.12 // indirect + github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.42 // indirect + github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.36 // indirect + github.com/aws/aws-sdk-go-v2/internal/ini v1.3.44 // indirect + github.com/aws/aws-sdk-go-v2/internal/v4a v1.1.5 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.9.15 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.1.37 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.36 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.15.5 // indirect + github.com/aws/aws-sdk-go-v2/service/sso v1.15.1 // indirect + github.com/aws/aws-sdk-go-v2/service/ssooidc v1.17.2 // indirect + github.com/aws/aws-sdk-go-v2/service/sts v1.23.1 // indirect github.com/cenkalti/backoff/v4 v4.2.1 // indirect github.com/census-instrumentation/opencensus-proto v0.4.1 // indirect github.com/cespare/xxhash/v2 v2.2.0 // indirect diff --git a/sdks/go.sum b/sdks/go.sum index f177dd8debb8..7891171f2095 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -81,60 +81,54 @@ github.com/aws/aws-sdk-go v1.30.19/go.mod h1:5zCpMtNQVjRREroY7sYe8lOMRSxkhG6MZve github.com/aws/aws-sdk-go v1.34.0 h1:brux2dRrlwCF5JhTL7MUT3WUwo9zfDHZZp3+g3Mvlmo= github.com/aws/aws-sdk-go v1.34.0/go.mod h1:5zCpMtNQVjRREroY7sYe8lOMRSxkhG6MZveU8YkpAk0= github.com/aws/aws-sdk-go-v2 v1.7.1/go.mod h1:L5LuPC1ZgDr2xQS7AmIec/Jlc7O/Y1u2KxJyNVab250= -github.com/aws/aws-sdk-go-v2 v1.21.0/go.mod h1:/RfNgGmRxI+iFOB1OeJUyxiU+9s88k3pfHvDagGEp0M= github.com/aws/aws-sdk-go-v2 v1.21.1 h1:wjHYshtPpYOZm+/mu3NhVgRRc0baM6LJZOmxPZ5Cwzs= github.com/aws/aws-sdk-go-v2 v1.21.1/go.mod h1:ErQhvNuEMhJjweavOYhxVkn2RUx7kQXVATHrjKtxIpM= -github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.4.13 h1:OPLEkmhXf6xFPiz0bLeDArZIDx1NNS4oJyG4nv3Gct0= -github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.4.13/go.mod h1:gpAbvyDGQFozTEmlTFO8XcQKHzubdq0LzRyJpG6MiXM= +github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.4.14 h1:Sc82v7tDQ/vdU1WtuSyzZ1I7y/68j//HJ6uozND1IDs= +github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.4.14/go.mod h1:9NCTOURS8OpxvoAVHq79LK81/zC78hfRWFn+aL0SPcY= github.com/aws/aws-sdk-go-v2/config v1.5.0/go.mod h1:RWlPOAW3E3tbtNAqTwvSW54Of/yP3oiZXMI0xfUdjyA= -github.com/aws/aws-sdk-go-v2/config v1.18.42/go.mod h1:4AZM3nMMxwlG+eZlxvBKqwVbkDLlnN2a4UGTL6HjaZI= -github.com/aws/aws-sdk-go-v2/config v1.18.43 h1:IgdUtTRvUDC6eiJBqU6vh7bHFNAEBjQ8S+qJ7zVhDOs= -github.com/aws/aws-sdk-go-v2/config v1.18.43/go.mod h1:NiFev8qlgg8MPzw3fO/EwzMZeZwlJEKGwfpjRPA9Nvw= +github.com/aws/aws-sdk-go-v2/config v1.18.44 h1:U10NQ3OxiY0dGGozmVIENIDnCT0W432PWxk2VO8wGnY= +github.com/aws/aws-sdk-go-v2/config v1.18.44/go.mod h1:pHxnQBldd0heEdJmolLBk78D1Bf69YnKLY3LOpFImlU= github.com/aws/aws-sdk-go-v2/credentials v1.3.1/go.mod h1:r0n73xwsIVagq8RsxmZbGSRQFj9As3je72C2WzUIToc= -github.com/aws/aws-sdk-go-v2/credentials v1.13.40/go.mod h1:VtEHVAAqDWASwdOqj/1huyT6uHbs5s8FUHfDQdky/Rs= -github.com/aws/aws-sdk-go-v2/credentials v1.13.41 h1:dgbKq1tamtboYAKSXWbqL0lKO9rmEzEhbZFh9JQW/Bg= -github.com/aws/aws-sdk-go-v2/credentials v1.13.41/go.mod h1:cc3Fn7DkKbJalPtQnudHGZZ8ml9+hwtbc1CJONsYYqk= +github.com/aws/aws-sdk-go-v2/credentials v1.13.42 h1:KMkjpZqcMOwtRHChVlHdNxTUUAC6NC/b58mRZDIdcRg= +github.com/aws/aws-sdk-go-v2/credentials v1.13.42/go.mod h1:7ltKclhvEB8305sBhrpls24HGxORl6qgnQqSJ314Uw8= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.3.0/go.mod h1:2LAuqPx1I6jNfaGDucWfA2zqQCYCOMCDHiCOciALyNw= -github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.13.11 h1:uDZJF1hu0EVT/4bogChk8DyjSF6fof6uL/0Y26Ma7Fg= -github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.13.11/go.mod h1:TEPP4tENqBGO99KwVpV9MlOX4NSrSLP8u3KRy2CDwA8= +github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.13.12 h1:3j5lrl9kVQrJ1BU4O0z7MQ8sa+UXdiLuo4j0V+odNI8= +github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.13.12/go.mod h1:JbFpcHDBdsex1zpIKuVRorZSQiZEyc3MykNCcjgz174= github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.3.2/go.mod h1:qaqQiHSrOUVOfKe6fhgQ6UzhxjwqVW8aHNegd6Ws4w4= -github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.87 h1:e20ZrsgDPUXqg8+rZVuPwNSp6yniUN2Yr2tzFZ+Yvl0= -github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.87/go.mod h1:0i0TAT6W+5i48QTlDU2KmY6U2hBZeY/LCP0wktya2oc= -github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.41 h1:22dGT7PneFMx4+b3pz7lMTRyN8ZKH7M2cW4GP9yUS2g= -github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.41/go.mod h1:CrObHAuPneJBlfEJ5T3szXOUkLEThaGfvnhTf33buas= -github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.35 h1:SijA0mgjV8E+8G45ltVHs0fvKpTj8xmZJ3VwhGKtUSI= -github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.35/go.mod h1:SJC1nEVVva1g3pHAIdCp7QsRIkMmLAgoDquQ9Rr8kYw= +github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.89 h1:XPqSyw8SBSLMRrF9Oip6tQpivXWJLMn8sdRoAsUCQQA= +github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.89/go.mod h1:OkYwM7gYm9HieL6emYtkg7Pb7Jd8FFM5Pl5uAZ1h2jo= +github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.42 h1:817VqVe6wvwE46xXy6YF5RywvjOX6U2zRQQ6IbQFK0s= +github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.42/go.mod h1:oDfgXoBBmj+kXnqxDDnIDnC56QBosglKp8ftRCTxR+0= +github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.36 h1:7ZApaXzWbo8slc+W5TynuUlB4z66g44h7uqa3/d/BsY= +github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.36/go.mod h1:rwr4WnmFi3RJO0M4dxbJtgi9BPLMpVBMX1nUte5ha9U= github.com/aws/aws-sdk-go-v2/internal/ini v1.1.1/go.mod h1:Zy8smImhTdOETZqfyn01iNOe0CNggVbPjCajyaz6Gvg= -github.com/aws/aws-sdk-go-v2/internal/ini v1.3.43 h1:g+qlObJH4Kn4n21g69DjspU0hKTjWtq7naZ9OLCv0ew= -github.com/aws/aws-sdk-go-v2/internal/ini v1.3.43/go.mod h1:rzfdUlfA+jdgLDmPKjd3Chq9V7LVLYo1Nz++Wb91aRo= -github.com/aws/aws-sdk-go-v2/internal/v4a v1.1.4 h1:6lJvvkQ9HmbHZ4h/IEwclwv2mrTW8Uq1SOB/kXy0mfw= -github.com/aws/aws-sdk-go-v2/internal/v4a v1.1.4/go.mod h1:1PrKYwxTM+zjpw9Y41KFtoJCQrJ34Z47Y4VgVbfndjo= +github.com/aws/aws-sdk-go-v2/internal/ini v1.3.44 h1:quOJOqlbSfeJTboXLjYXM1M9T52LBXqLoTPlmsKLpBo= +github.com/aws/aws-sdk-go-v2/internal/ini v1.3.44/go.mod h1:LNy+P1+1LiRcCsVYr/4zG5n8zWFL0xsvZkOybjbftm8= +github.com/aws/aws-sdk-go-v2/internal/v4a v1.1.5 h1:8JG9ny0BqBDzmtIzbpaN+eke152ZNsYKApFJ/q29Hxo= +github.com/aws/aws-sdk-go-v2/internal/v4a v1.1.5/go.mod h1:kEDHQApP/ukMO9natNftgUN3NaTsMxK6jb2jjpSMX7Y= github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.2.1/go.mod h1:v33JQ57i2nekYTA70Mb+O18KeH4KqhdqxTJZNK1zdRE= -github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.9.14 h1:m0QTSI6pZYJTk5WSKx3fm5cNW/DCicVzULBgU/6IyD0= -github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.9.14/go.mod h1:dDilntgHy9WnHXsh7dDtUPgHKEfTJIBUTHM8OWm0f/0= -github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.1.36 h1:eev2yZX7esGRjqRbnVk1UxMLw4CyVZDpZXRCcy75oQk= -github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.1.36/go.mod h1:lGnOkH9NJATw0XEPcAknFBj3zzNTEGRHtSw+CwC1YTg= +github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.9.15 h1:7R8uRYyXzdD71KWVCL78lJZltah6VVznXBazvKjfH58= +github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.9.15/go.mod h1:26SQUPcTNgV1Tapwdt4a1rOsYRsnBsJHLMPoxK2b0d8= +github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.1.37 h1:Mx1zJlYbiUQANWT40koevLvxawGFolmkaP4m+LuyG7M= +github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.1.37/go.mod h1:PjKIAMFthKPgG/B8bbRpo3F8jfr2q2L+w3u78jJ12a0= github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.2.1/go.mod h1:zceowr5Z1Nh2WVP8bf/3ikB41IZW59E4yIYbg+pC6mw= -github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.35 h1:CdzPW9kKitgIiLV1+MHobfR5Xg25iYnyzWZhyQuSlDI= -github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.35/go.mod h1:QGF2Rs33W5MaN9gYdEQOBBFPLwTZkEhRwI33f7KIG0o= +github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.36 h1:YXlm7LxwNlauqb2OrinWlcvtsflTzP8GaMvYfQBhoT4= +github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.36/go.mod h1:ou9ffqJ9hKOVZmjlC6kQ6oROAyG1M4yBKzR+9BKbDwk= github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.5.1/go.mod h1:6EQZIwNNvHpq/2/QSJnp4+ECvqIy55w95Ofs0ze+nGQ= -github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.15.4 h1:v0jkRigbSD6uOdwcaUQmgEwG1BkPfAPDqaeNt/29ghg= -github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.15.4/go.mod h1:LhTyt8J04LL+9cIt7pYJ5lbS/U98ZmXovLOR/4LUsk8= +github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.15.5 h1:sAAz28SeA7YZl8Yaphjs9tlLsflhdniQPjf3X2cqr4s= +github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.15.5/go.mod h1:HC7gNz3VH0p+RvLKK+HqNQv/gHy+1Os3ko/F41s3+aw= github.com/aws/aws-sdk-go-v2/service/s3 v1.11.1/go.mod h1:XLAGFrEjbvMCLvAtWLLP32yTv8GpBquCApZEycDLunI= -github.com/aws/aws-sdk-go-v2/service/s3 v1.40.0 h1:wl5dxN1NONhTDQD9uaEvNsDRX29cBmGED/nl0jkWlt4= -github.com/aws/aws-sdk-go-v2/service/s3 v1.40.0/go.mod h1:rDGMZA7f4pbmTtPOk5v5UM2lmX6UAbRnMDJeDvnH7AM= +github.com/aws/aws-sdk-go-v2/service/s3 v1.40.1 h1:FqIaVPbs2W8U3fszl2PCL1IDKeRdM7TssjWamL6b2mg= +github.com/aws/aws-sdk-go-v2/service/s3 v1.40.1/go.mod h1:X0e0NCAx4GjOrKro7s9QYy+YEIFhgCkt6gYKVKhZB5Y= github.com/aws/aws-sdk-go-v2/service/sso v1.3.1/go.mod h1:J3A3RGUvuCZjvSuZEcOpHDnzZP/sKbhDWV2T1EOzFIM= -github.com/aws/aws-sdk-go-v2/service/sso v1.14.1/go.mod h1:fIAwKQKBFu90pBxx07BFOMJLpRUGu8VOzLJakeY+0K4= -github.com/aws/aws-sdk-go-v2/service/sso v1.15.0 h1:vuGK1vHNP9zx0PfOrtPumbwR2af0ATQ1Z2H6p75AgRQ= -github.com/aws/aws-sdk-go-v2/service/sso v1.15.0/go.mod h1:fIAwKQKBFu90pBxx07BFOMJLpRUGu8VOzLJakeY+0K4= -github.com/aws/aws-sdk-go-v2/service/ssooidc v1.17.1 h1:8lKOidPkmSmfUtiTgtdXWgaKItCZ/g75/jEk6Ql6GsA= -github.com/aws/aws-sdk-go-v2/service/ssooidc v1.17.1/go.mod h1:yygr8ACQRY2PrEcy3xsUI357stq2AxnFM6DIsR9lij4= +github.com/aws/aws-sdk-go-v2/service/sso v1.15.1 h1:ZN3bxw9OYC5D6umLw6f57rNJfGfhg1DIAAcKpzyUTOE= +github.com/aws/aws-sdk-go-v2/service/sso v1.15.1/go.mod h1:PieckvBoT5HtyB9AsJRrYZFY2Z+EyfVM/9zG6gbV8DQ= +github.com/aws/aws-sdk-go-v2/service/ssooidc v1.17.2 h1:fSCCJuT5i6ht8TqGdZc5Q5K9pz/atrf7qH4iK5C9XzU= +github.com/aws/aws-sdk-go-v2/service/ssooidc v1.17.2/go.mod h1:5eNtr+vNc5vVd92q7SJ+U/HszsIdhZBEyi9dkMRKsp8= github.com/aws/aws-sdk-go-v2/service/sts v1.6.0/go.mod h1:q7o0j7d7HrJk/vr9uUt3BVRASvcU7gYZB9PUgPiByXg= -github.com/aws/aws-sdk-go-v2/service/sts v1.22.0/go.mod h1:VC7JDqsqiwXukYEDjoHh9U0fOJtNWh04FPQz4ct4GGU= -github.com/aws/aws-sdk-go-v2/service/sts v1.23.0 h1:pyvfUqkNLMipdKNAtu7OVbRxUrR2BMaKccIPpk/Hkak= -github.com/aws/aws-sdk-go-v2/service/sts v1.23.0/go.mod h1:VC7JDqsqiwXukYEDjoHh9U0fOJtNWh04FPQz4ct4GGU= +github.com/aws/aws-sdk-go-v2/service/sts v1.23.1 h1:ASNYk1ypWAxRhJjKS0jBnTUeDl7HROOpeSMu1xDA/I8= +github.com/aws/aws-sdk-go-v2/service/sts v1.23.1/go.mod h1:2cnsAhVT3mqusovc2stUSUrSBGTcX9nh8Tu6xh//2eI= github.com/aws/smithy-go v1.6.0/go.mod h1:SObp3lf9smib00L/v3U2eAKG8FyQ7iLrJnQiAmR5n+E= -github.com/aws/smithy-go v1.14.2/go.mod h1:Tg+OJXh4MB2R/uN61Ko2f6hTZwB/ZYGOtib8J3gBHzA= github.com/aws/smithy-go v1.15.0 h1:PS/durmlzvAFpQHDs4wi4sNNP9ExsqZh6IlfdHXgKK8= github.com/aws/smithy-go v1.15.0/go.mod h1:Tg+OJXh4MB2R/uN61Ko2f6hTZwB/ZYGOtib8J3gBHzA= github.com/cenkalti/backoff/v4 v4.2.1 h1:y4OZtCnogmCPw98Zjyt5a6+QwPLGkiQsYW5oUqylYbM= From ad5c2f55bfefed2e91339b1d3ff1aae67ec6b5f6 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 9 Oct 2023 13:19:02 -0400 Subject: [PATCH 074/435] Bump golang.org/x/sync from 0.3.0 to 0.4.0 in /sdks (#28856) Bumps [golang.org/x/sync](https://github.com/golang/sync) from 0.3.0 to 0.4.0. - [Commits](https://github.com/golang/sync/compare/v0.3.0...v0.4.0) --- updated-dependencies: - dependency-name: golang.org/x/sync dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 2 +- sdks/go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index d5b7d7e3e41e..1014531ec0f9 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -54,7 +54,7 @@ require ( go.mongodb.org/mongo-driver v1.12.1 golang.org/x/net v0.16.0 golang.org/x/oauth2 v0.12.0 - golang.org/x/sync v0.3.0 + golang.org/x/sync v0.4.0 golang.org/x/sys v0.13.0 golang.org/x/text v0.13.0 google.golang.org/api v0.144.0 diff --git a/sdks/go.sum b/sdks/go.sum index 7891171f2095..dc6318b72025 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -558,8 +558,8 @@ golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.3.0 h1:ftCYgMx6zT/asHUrPw8BLLscYtGznsLAnjq5RH9P66E= -golang.org/x/sync v0.3.0/go.mod h1:FU7BRWz2tNW+3quACPkgCx/L+uEAv1htQ0V83Z9Rj+Y= +golang.org/x/sync v0.4.0 h1:zxkM55ReGkDlKSM+Fu41A+zmbZuaPVbGMzvvdUPznYQ= +golang.org/x/sync v0.4.0/go.mod h1:FU7BRWz2tNW+3quACPkgCx/L+uEAv1htQ0V83Z9Rj+Y= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190312061237-fead79001313/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= From 5491b5cdabc37e84fb836cfc610e52b7702fdc91 Mon Sep 17 00:00:00 2001 From: Robert Burke Date: Mon, 9 Oct 2023 10:49:11 -0700 Subject: [PATCH 075/435] Update container version in BeamModulePlugin.groovy (#28879) Ensure boot loaders are built using the latest Go patch release. --- .../main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index 3a332d8d4d08..9f4ae8bcfb29 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -2154,7 +2154,7 @@ class BeamModulePlugin implements Plugin { def goRootDir = "${project.rootDir}/sdks/go" // This sets the whole project Go version. - project.ext.goVersion = "go1.21.1" + project.ext.goVersion = "go1.21.2" // Minor TODO: Figure out if we can pull out the GOCMD env variable after goPrepare script // completion, and avoid this GOBIN substitution. From be3909a62d74c32b2b317918b6480db5fe63cf46 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 9 Oct 2023 10:50:28 -0700 Subject: [PATCH 076/435] Bump cloud.google.com/go/bigquery from 1.55.0 to 1.56.0 in /sdks (#28892) Bumps [cloud.google.com/go/bigquery](https://github.com/googleapis/google-cloud-go) from 1.55.0 to 1.56.0. - [Release notes](https://github.com/googleapis/google-cloud-go/releases) - [Changelog](https://github.com/googleapis/google-cloud-go/blob/main/CHANGES.md) - [Commits](https://github.com/googleapis/google-cloud-go/compare/bigquery/v1.55.0...bigquery/v1.56.0) --- updated-dependencies: - dependency-name: cloud.google.com/go/bigquery dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 2 +- sdks/go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 1014531ec0f9..bf55e376467b 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -23,7 +23,7 @@ module github.com/apache/beam/sdks/v2 go 1.20 require ( - cloud.google.com/go/bigquery v1.55.0 + cloud.google.com/go/bigquery v1.56.0 cloud.google.com/go/bigtable v1.20.0 cloud.google.com/go/datastore v1.14.0 cloud.google.com/go/profiler v0.3.1 diff --git a/sdks/go.sum b/sdks/go.sum index dc6318b72025..3579eedeb512 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -13,8 +13,8 @@ cloud.google.com/go v0.110.7/go.mod h1:+EYjdK8e5RME/VY/qLCAtuyALQ9q67dvuum8i+H5x cloud.google.com/go/bigquery v1.0.1/go.mod h1:i/xbL2UlR5RvWAURpBYZTtm/cXjCha9lbfbpx4poX+o= cloud.google.com/go/bigquery v1.3.0/go.mod h1:PjpwJnslEMmckchkHFfq+HTD2DmtT67aNFKH1/VBDHE= cloud.google.com/go/bigquery v1.4.0/go.mod h1:S8dzgnTigyfTmLBfrtrhyYhwRxG72rYxvftPBK2Dvzc= -cloud.google.com/go/bigquery v1.55.0 h1:hs44Xxov3XLWQiCx2J8lK5U/ihLqnpm4RVVl5fdtLLI= -cloud.google.com/go/bigquery v1.55.0/go.mod h1:9Y5I3PN9kQWuid6183JFhOGOW3GcirA5LpsKCUn+2ec= +cloud.google.com/go/bigquery v1.56.0 h1:LHIc9E7Kw+ftFpQFKzZYBB88IAFz7qONawXXx0F3QBo= +cloud.google.com/go/bigquery v1.56.0/go.mod h1:KDcsploXTEY7XT3fDQzMUZlpQLHzE4itubHrnmhUrZA= cloud.google.com/go/bigtable v1.20.0 h1:NqZC/WcesSn4O8L0I2JmuNsUigSyBQifVLYgM9LMQeQ= cloud.google.com/go/bigtable v1.20.0/go.mod h1:upJDn8frsjzpRMfybiWkD1PG6WCCL7CRl26MgVeoXY4= cloud.google.com/go/compute v1.23.0 h1:tP41Zoavr8ptEqaW6j+LQOnyBBhO7OkOMAGrgLopTwY= From 9eac03027bcfed79de751d0d2f1dc05f87b19e5b Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 9 Oct 2023 10:52:05 -0700 Subject: [PATCH 077/435] Bump github.com/testcontainers/testcontainers-go in /sdks (#28761) Bumps [github.com/testcontainers/testcontainers-go](https://github.com/testcontainers/testcontainers-go) from 0.24.1 to 0.25.0. - [Release notes](https://github.com/testcontainers/testcontainers-go/releases) - [Commits](https://github.com/testcontainers/testcontainers-go/compare/v0.24.1...v0.25.0) --- updated-dependencies: - dependency-name: github.com/testcontainers/testcontainers-go dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 8 ++++---- sdks/go.sum | 20 ++++++++++---------- 2 files changed, 14 insertions(+), 14 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index bf55e376467b..a2b5ed097642 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -47,7 +47,7 @@ require ( github.com/linkedin/goavro/v2 v2.12.0 github.com/proullon/ramsql v0.1.3 github.com/spf13/cobra v1.7.0 - github.com/testcontainers/testcontainers-go v0.24.1 + github.com/testcontainers/testcontainers-go v0.25.0 github.com/tetratelabs/wazero v1.5.0 github.com/xitongsys/parquet-go v1.6.2 github.com/xitongsys/parquet-go-source v0.0.0-20220315005136-aec0fe3e777c @@ -77,10 +77,10 @@ require ( github.com/go-ole/go-ole v1.2.6 // indirect github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0 // indirect github.com/power-devops/perfstat v0.0.0-20210106213030-5aafc221ea8c // indirect - github.com/shirou/gopsutil/v3 v3.23.7 // indirect + github.com/shirou/gopsutil/v3 v3.23.8 // indirect github.com/shoenig/go-m1cpu v0.1.6 // indirect - github.com/tklauser/go-sysconf v0.3.11 // indirect - github.com/tklauser/numcpus v0.6.0 // indirect + github.com/tklauser/go-sysconf v0.3.12 // indirect + github.com/tklauser/numcpus v0.6.1 // indirect github.com/yusufpapurcu/wmi v1.2.3 // indirect ) diff --git a/sdks/go.sum b/sdks/go.sum index 3579eedeb512..516adec93875 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -396,8 +396,8 @@ github.com/ryszard/goskiplist v0.0.0-20150312221310-2dfbae5fcf46/go.mod h1:uAQ5P github.com/seccomp/libseccomp-golang v0.9.2-0.20220502022130-f33da4d89646/go.mod h1:JA8cRccbGaA1s33RQf7Y1+q9gHmZX1yB/z9WDN1C6fg= github.com/shabbyrobe/gocovmerge v0.0.0-20180507124511-f6ea450bfb63 h1:J6qvD6rbmOil46orKqJaRPG+zTpoGlBTUdyv8ki63L0= github.com/shabbyrobe/gocovmerge v0.0.0-20180507124511-f6ea450bfb63/go.mod h1:n+VKSARF5y/tS9XFSP7vWDfS+GUC5vs/YT7M5XDTUEM= -github.com/shirou/gopsutil/v3 v3.23.7 h1:C+fHO8hfIppoJ1WdsVm1RoI0RwXoNdfTK7yWXV0wVj4= -github.com/shirou/gopsutil/v3 v3.23.7/go.mod h1:c4gnmoRC0hQuaLqvxnx1//VXQ0Ms/X9UnJF8pddY5z4= +github.com/shirou/gopsutil/v3 v3.23.8 h1:xnATPiybo6GgdRoC4YoGnxXZFRc3dqQTGi73oLvvBrE= +github.com/shirou/gopsutil/v3 v3.23.8/go.mod h1:7hmCaBn+2ZwaZOr6jmPBZDfawwMGuo1id3C6aM8EDqQ= github.com/shoenig/go-m1cpu v0.1.6 h1:nxdKQNcEB6vzgA2E2bvzKIYRuNj7XNJ4S/aRSwKzFtM= github.com/shoenig/go-m1cpu v0.1.6/go.mod h1:1JJMcUBvfNwpq05QDQVAnx3gUHr9IYF7GNg9SUEw2VQ= github.com/shoenig/test v0.6.4 h1:kVTaSd7WLz5WZ2IaoM0RSzRsUD+m8wRR+5qvntpn4LU= @@ -428,14 +428,14 @@ github.com/stretchr/testify v1.8.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o github.com/stretchr/testify v1.8.4 h1:CcVxjf3Q8PM0mHUKJCdn+eZZtm5yQwehR5yeSVQQcUk= github.com/stretchr/testify v1.8.4/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo= github.com/syndtr/gocapability v0.0.0-20200815063812-42c35b437635/go.mod h1:hkRG7XYTFWNJGYcbNJQlaLq0fg1yr4J4t/NcTQtrfww= -github.com/testcontainers/testcontainers-go v0.24.1 h1:gJdZuQIVWnMJTo+CmQMEP7/CAagNk/0jbcUPn3OWvD8= -github.com/testcontainers/testcontainers-go v0.24.1/go.mod h1:MGBiAkCm86yXQoCiipmQCqZLVdk1uFqtMqaU1Or0MRk= +github.com/testcontainers/testcontainers-go v0.25.0 h1:erH6cQjsaJrH+rJDU9qIf89KFdhK0Bft0aEZHlYC3Vs= +github.com/testcontainers/testcontainers-go v0.25.0/go.mod h1:4sC9SiJyzD1XFi59q8umTQYWxnkweEc5OjVtTUlJzqQ= github.com/tetratelabs/wazero v1.5.0 h1:Yz3fZHivfDiZFUXnWMPUoiW7s8tC1sjdBtlJn08qYa0= github.com/tetratelabs/wazero v1.5.0/go.mod h1:0U0G41+ochRKoPKCJlh0jMg1CHkyfK8kDqiirMmKY8A= -github.com/tklauser/go-sysconf v0.3.11 h1:89WgdJhk5SNwJfu+GKyYveZ4IaJ7xAkecBo+KdJV0CM= -github.com/tklauser/go-sysconf v0.3.11/go.mod h1:GqXfhXY3kiPa0nAXPDIQIWzJbMCB7AmcWpGR8lSZfqI= -github.com/tklauser/numcpus v0.6.0 h1:kebhY2Qt+3U6RNK7UqpYNA+tJ23IBEGKkB7JQBfDYms= -github.com/tklauser/numcpus v0.6.0/go.mod h1:FEZLMke0lhOUG6w2JadTzp0a+Nl8PF/GFkQ5UVIcaL4= +github.com/tklauser/go-sysconf v0.3.12 h1:0QaGUFOdQaIVdPgfITYzaTegZvdCjmYO52cSFAEVmqU= +github.com/tklauser/go-sysconf v0.3.12/go.mod h1:Ho14jnntGE1fpdOqQEEaiKRpvIavV0hSfmBq8nJbHYI= +github.com/tklauser/numcpus v0.6.1 h1:ng9scYS7az0Bk4OZLvrNXNSAO2Pxr1XXRAPyjhIx+Fk= +github.com/tklauser/numcpus v0.6.1/go.mod h1:1XfjsgE2zo8GVw7POkMbHENHzVg3GzmoZ9fESEdAacY= github.com/urfave/cli v1.22.1/go.mod h1:Gos4lmkARVdJ6EkW0WaNv/tZAAMe9V7XWyB60NtXRu0= github.com/vishvananda/netlink v1.1.0/go.mod h1:cTgwzPIzzgDAYoQrMm0EdrjRUBkTqKYppBueQtXaqoE= github.com/vishvananda/netns v0.0.0-20191106174202-0a2b9b5464df/go.mod h1:JP3t17pCcGlemwknint6hfoeCVQrEMVwxRLRjXpq+BU= @@ -597,9 +597,9 @@ golang.org/x/sys v0.0.0-20220408201424-a24fb2fb8a0f/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220715151400-c0bba94af5f8/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.2.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.10.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.8.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.11.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.13.0 h1:Af8nKPmuFypiUBjVoU9V20FiaFXOcuZI21p0ycVYYGE= golang.org/x/sys v0.13.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= From 4ead788840c32669477efcdc73b1b6abf01f2cbb Mon Sep 17 00:00:00 2001 From: Danny McCormick Date: Mon, 9 Oct 2023 14:00:30 -0400 Subject: [PATCH 078/435] Reenable TestFhirIO.* on GHA (#28435) * Reenable TestFhirIO.* on GHA * Propogate last error * temporarily only look at failing test and better error * temporarily only look at failing test and better error * temporarily only look at failing test and better error * Restore integration.go --- sdks/go/test/integration/integration.go | 6 ------ .../test/integration/io/fhirio/fhirio_test.go | 19 ++++++++++++++----- 2 files changed, 14 insertions(+), 11 deletions(-) diff --git a/sdks/go/test/integration/integration.go b/sdks/go/test/integration/integration.go index f3cffd176110..d3eb1c3ee7c3 100644 --- a/sdks/go/test/integration/integration.go +++ b/sdks/go/test/integration/integration.go @@ -304,12 +304,6 @@ func CheckFilters(t *testing.T) { panic("ptest.Main() has not been called: please override TestMain to ensure that the integration test runs properly.") } - // TODO(https://github.com/apache/beam/issues/28227): Grant github-actions service account permission to healthcare.fhirStores.create. - var user = os.Getenv("USER") - if user == "github-actions" { - dataflowFilters = append(dataflowFilters, "TestFhirIO.*") - } - // Check for sickbaying first. n := t.Name() for _, f := range sickbay { diff --git a/sdks/go/test/integration/io/fhirio/fhirio_test.go b/sdks/go/test/integration/io/fhirio/fhirio_test.go index 03e3654d5c49..01f6db9324f3 100644 --- a/sdks/go/test/integration/io/fhirio/fhirio_test.go +++ b/sdks/go/test/integration/io/fhirio/fhirio_test.go @@ -96,9 +96,9 @@ func setupFhirStore(t *testing.T, shouldPopulateStore bool) (fhirStoreInfo, func var resourcePaths [][]byte if shouldPopulateStore { - resourcePaths = populateStore(createdFhirStorePath) - if len(resourcePaths) == 0 { - t.Fatal("No data got populated to test") + resourcePaths, err = populateStore(createdFhirStorePath) + if err != nil { + t.Fatal(err) } } @@ -127,11 +127,13 @@ func deleteStore(storePath string) (*healthcare.Empty, error) { // Populates fhir store with data. Note that failure to populate some data is not // detrimental to the tests, so it is fine to ignore. -func populateStore(storePath string) [][]byte { +func populateStore(storePath string) ([][]byte, error) { resourcePaths := make([][]byte, 0) + bufferedErrors := make([]string, 0) for _, bundle := range readPrettyBundles() { response, err := storeService.ExecuteBundle(storePath, strings.NewReader(bundle)).Do() if err != nil { + bufferedErrors = append(bufferedErrors, err.Error()) continue } @@ -145,23 +147,30 @@ func populateStore(storePath string) [][]byte { } err = json.NewDecoder(response.Body).Decode(&body) if err != nil { + bufferedErrors = append(bufferedErrors, err.Error()) continue } for _, entry := range body.Entry { bundleFailedToBeCreated := !strings.Contains(entry.Response.Status, "201") if bundleFailedToBeCreated { + bufferedErrors = append(bufferedErrors, fmt.Sprintf("Bundle creation failed with: %v", entry.Response)) continue } resourcePath, err := extractResourcePathFrom(entry.Response.Location) if err != nil { + bufferedErrors = append(bufferedErrors, err.Error()) continue } resourcePaths = append(resourcePaths, resourcePath) } } - return resourcePaths + if len(resourcePaths) == 0 { + return nil, fmt.Errorf("failed to populate fhir store with any data. Errors with requests: %s", bufferedErrors) + } + + return resourcePaths, nil } func readPrettyBundles() []string { From 21f822f6580f04c3bd581005d93846e555982367 Mon Sep 17 00:00:00 2001 From: Ritesh Ghorse Date: Mon, 9 Oct 2023 14:14:07 -0400 Subject: [PATCH 079/435] Catch EOFError along with StopIteration (#28896) * catch EOF instead of StopIteration * add stop iteration to close * catch both stop iteration and EOF --- sdks/python/apache_beam/io/gcp/bigquery.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/io/gcp/bigquery.py b/sdks/python/apache_beam/io/gcp/bigquery.py index 986919fd6b82..184138af7525 100644 --- a/sdks/python/apache_beam/io/gcp/bigquery.py +++ b/sdks/python/apache_beam/io/gcp/bigquery.py @@ -1310,7 +1310,7 @@ def __iter__(self): def __next__(self): try: return fastavro.schemaless_reader(self.bytes_reader, self.avro_schema) - except StopIteration: + except (StopIteration, EOFError): self.read_rows_response = next(self.read_rows_iterator, None) if self.read_rows_response is not None: self.bytes_reader = io.BytesIO( From f63e9ed28e4889c2645bfaf5a3450c10a205cf62 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Mon, 9 Oct 2023 11:19:21 -0700 Subject: [PATCH 080/435] Additional context for decoding errors. (#27024) --- .../apache_beam/runners/worker/bundle_processor.py | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/runners/worker/bundle_processor.py b/sdks/python/apache_beam/runners/worker/bundle_processor.py index 935ba83709c0..c7fcb9587455 100644 --- a/sdks/python/apache_beam/runners/worker/bundle_processor.py +++ b/sdks/python/apache_beam/runners/worker/bundle_processor.py @@ -227,8 +227,13 @@ def process_encoded(self, encoded_windowed_values): if self.index == self.stop - 1: return self.index += 1 - decoded_value = self.windowed_coder_impl.decode_from_stream( - input_stream, True) + try: + decoded_value = self.windowed_coder_impl.decode_from_stream( + input_stream, True) + except Exception as exn: + raise ValueError( + "Error decoding input stream with coder " + + self.windowed_coder) from exn self.output(decoded_value) def monitoring_infos(self, transform_id, tag_to_pcollection_id): From dafe928a15b5ef5a865f5889eb98848770b462a6 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Mon, 9 Oct 2023 11:20:34 -0700 Subject: [PATCH 081/435] Add Java variants of WriteTo{Csv,Json}. (#28380) --- .../schemaio-expansion-service/build.gradle | 6 + .../providers/CsvWriteTransformProvider.java | 145 +++++++++ .../sdk/io/csv/providers/package-info.java | 20 ++ sdks/java/io/json/build.gradle | 35 +++ .../org/apache/beam/sdk/io/json/JsonIO.java | 283 ++++++++++++++++++ .../apache/beam/sdk/io/json/package-info.java | 20 ++ .../providers/JsonWriteTransformProvider.java | 142 +++++++++ .../sdk/io/json/providers/package-info.java | 20 ++ .../beam/sdk/io/json/JsonIOWriteTest.java | 145 +++++++++ sdks/python/apache_beam/yaml/standard_io.yaml | 7 + settings.gradle.kts | 1 + 11 files changed, 824 insertions(+) create mode 100644 sdks/java/io/csv/src/main/java/org/apache/beam/sdk/io/csv/providers/CsvWriteTransformProvider.java create mode 100644 sdks/java/io/csv/src/main/java/org/apache/beam/sdk/io/csv/providers/package-info.java create mode 100644 sdks/java/io/json/build.gradle create mode 100644 sdks/java/io/json/src/main/java/org/apache/beam/sdk/io/json/JsonIO.java create mode 100644 sdks/java/io/json/src/main/java/org/apache/beam/sdk/io/json/package-info.java create mode 100644 sdks/java/io/json/src/main/java/org/apache/beam/sdk/io/json/providers/JsonWriteTransformProvider.java create mode 100644 sdks/java/io/json/src/main/java/org/apache/beam/sdk/io/json/providers/package-info.java create mode 100644 sdks/java/io/json/src/test/java/org/apache/beam/sdk/io/json/JsonIOWriteTest.java diff --git a/sdks/java/extensions/schemaio-expansion-service/build.gradle b/sdks/java/extensions/schemaio-expansion-service/build.gradle index d23330d73c22..246c0c155cbd 100644 --- a/sdks/java/extensions/schemaio-expansion-service/build.gradle +++ b/sdks/java/extensions/schemaio-expansion-service/build.gradle @@ -32,8 +32,14 @@ applyJavaNature( dependencies { implementation project(path: ":sdks:java:expansion-service") permitUnusedDeclared project(path: ":sdks:java:expansion-service") // BEAM-11761 + implementation project(":sdks:java:extensions:google-cloud-platform-core") + permitUnusedDeclared project(path: ":sdks:java:extensions:google-cloud-platform-core") // BEAM-11761 + implementation project(":sdks:java:io:csv") + permitUnusedDeclared project(path: ":sdks:java:io:csv") // BEAM-11761 implementation project(":sdks:java:io:jdbc") permitUnusedDeclared project(":sdks:java:io:jdbc") // BEAM-11761 + implementation project(":sdks:java:io:json") + permitUnusedDeclared project(path: ":sdks:java:io:json") // BEAM-11761 implementation library.java.postgres permitUnusedDeclared library.java.postgres // BEAM-11761 implementation project(path: ":model:pipeline", configuration: "shadow") diff --git a/sdks/java/io/csv/src/main/java/org/apache/beam/sdk/io/csv/providers/CsvWriteTransformProvider.java b/sdks/java/io/csv/src/main/java/org/apache/beam/sdk/io/csv/providers/CsvWriteTransformProvider.java new file mode 100644 index 000000000000..4e07a06197f5 --- /dev/null +++ b/sdks/java/io/csv/src/main/java/org/apache/beam/sdk/io/csv/providers/CsvWriteTransformProvider.java @@ -0,0 +1,145 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.csv.providers; + +import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; + +import com.google.auto.service.AutoService; +import com.google.auto.value.AutoValue; +import java.util.Collections; +import java.util.List; +import org.apache.beam.sdk.io.WriteFilesResult; +import org.apache.beam.sdk.io.csv.CsvIO; +import org.apache.beam.sdk.schemas.AutoValueSchema; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.Schema.Field; +import org.apache.beam.sdk.schemas.Schema.FieldType; +import org.apache.beam.sdk.schemas.annotations.DefaultSchema; +import org.apache.beam.sdk.schemas.annotations.SchemaFieldDescription; +import org.apache.beam.sdk.schemas.transforms.SchemaTransform; +import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider; +import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.values.PCollectionRowTuple; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.TypeDescriptors; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Strings; +import org.apache.commons.csv.CSVFormat; + +/** + * An implementation of {@link TypedSchemaTransformProvider} for {@link CsvIO#write}. + * + *

Internal only: This class is actively being worked on, and it will likely change. We + * provide no backwards compatibility guarantees, and it should not be implemented outside the Beam + * repository. + */ +@SuppressWarnings({ + "nullness" // TODO(https://github.com/apache/beam/issues/20497) +}) +@AutoService(SchemaTransformProvider.class) +public class CsvWriteTransformProvider + extends TypedSchemaTransformProvider { + private static final String INPUT_ROWS_TAG = "input"; + private static final String WRITE_RESULTS = "output"; + + @Override + protected Class configurationClass() { + return CsvWriteConfiguration.class; + } + + @Override + protected SchemaTransform from(CsvWriteConfiguration configuration) { + return new CsvWriteTransform(configuration); + } + + @Override + public String identifier() { + return String.format("beam:schematransform:org.apache.beam:csv_write:v1"); + } + + @Override + public List inputCollectionNames() { + return Collections.singletonList(INPUT_ROWS_TAG); + } + + @Override + public List outputCollectionNames() { + return Collections.singletonList(WRITE_RESULTS); + } + + /** Configuration for writing to BigQuery with Storage Write API. */ + @DefaultSchema(AutoValueSchema.class) + @AutoValue + public abstract static class CsvWriteConfiguration { + + public void validate() { + checkArgument( + !Strings.isNullOrEmpty(this.getPath()), "Path for a CSV Write must be specified."); + } + + public static Builder builder() { + return new AutoValue_CsvWriteTransformProvider_CsvWriteConfiguration.Builder(); + } + + @SchemaFieldDescription("The file path to write to.") + public abstract String getPath(); + + /** Builder for {@link CsvWriteConfiguration}. */ + @AutoValue.Builder + public abstract static class Builder { + + public abstract Builder setPath(String path); + + /** Builds a {@link CsvWriteConfiguration} instance. */ + public abstract CsvWriteConfiguration build(); + } + } + + /** A {@link SchemaTransform} for {@link CsvIO#write}. */ + protected static class CsvWriteTransform extends SchemaTransform { + + private final CsvWriteConfiguration configuration; + + CsvWriteTransform(CsvWriteConfiguration configuration) { + configuration.validate(); + this.configuration = configuration; + } + + @Override + public PCollectionRowTuple expand(PCollectionRowTuple input) { + WriteFilesResult result = + input + .get(INPUT_ROWS_TAG) + .apply(CsvIO.writeRows(configuration.getPath(), CSVFormat.DEFAULT).withSuffix("")); + Schema outputSchema = Schema.of(Field.of("filename", FieldType.STRING)); + return PCollectionRowTuple.of( + WRITE_RESULTS, + result + .getPerDestinationOutputFilenames() + .apply( + "Collect filenames", + MapElements.into(TypeDescriptors.rows()) + .via( + (destinationAndRow) -> + Row.withSchema(outputSchema) + .withFieldValue("filename", destinationAndRow.getValue()) + .build())) + .setRowSchema(outputSchema)); + } + } +} diff --git a/sdks/java/io/csv/src/main/java/org/apache/beam/sdk/io/csv/providers/package-info.java b/sdks/java/io/csv/src/main/java/org/apache/beam/sdk/io/csv/providers/package-info.java new file mode 100644 index 000000000000..646e69b7cb8c --- /dev/null +++ b/sdks/java/io/csv/src/main/java/org/apache/beam/sdk/io/csv/providers/package-info.java @@ -0,0 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** Transforms for reading and writing CSV files. */ +package org.apache.beam.sdk.io.csv.providers; diff --git a/sdks/java/io/json/build.gradle b/sdks/java/io/json/build.gradle new file mode 100644 index 000000000000..fe1f607a3696 --- /dev/null +++ b/sdks/java/io/json/build.gradle @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * License); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an AS IS BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +plugins { id 'org.apache.beam.module' } +applyJavaNature( + automaticModuleName: 'org.apache.beam.sdk.io.json' +) + +description = "Apache Beam :: SDKs :: Java :: IO :: JSON" +ext.summary = "IO to read and write JSON files." + +dependencies { + implementation project(path: ":sdks:java:core", configuration: "shadow") + implementation library.java.vendored_guava_32_1_2_jre + implementation library.java.everit_json_schema + testImplementation project(path: ":sdks:java:core", configuration: "shadowTest") + testImplementation library.java.junit + testRuntimeOnly project(path: ":runners:direct-java", configuration: "shadow") + testImplementation project(path: ":sdks:java:io:common", configuration: "testRuntimeMigration") +} \ No newline at end of file diff --git a/sdks/java/io/json/src/main/java/org/apache/beam/sdk/io/json/JsonIO.java b/sdks/java/io/json/src/main/java/org/apache/beam/sdk/io/json/JsonIO.java new file mode 100644 index 000000000000..3abb29a80427 --- /dev/null +++ b/sdks/java/io/json/src/main/java/org/apache/beam/sdk/io/json/JsonIO.java @@ -0,0 +1,283 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.json; + +import static org.apache.beam.sdk.values.TypeDescriptors.rows; +import static org.apache.beam.sdk.values.TypeDescriptors.strings; + +import com.google.auto.value.AutoValue; +import org.apache.beam.sdk.coders.RowCoder; +import org.apache.beam.sdk.io.Compression; +import org.apache.beam.sdk.io.FileBasedSink; +import org.apache.beam.sdk.io.FileIO; +import org.apache.beam.sdk.io.ShardNameTemplate; +import org.apache.beam.sdk.io.TextIO; +import org.apache.beam.sdk.io.WriteFiles; +import org.apache.beam.sdk.io.WriteFilesResult; +import org.apache.beam.sdk.io.fs.ResourceId; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.utils.JsonUtils; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.Row; + +/** + * {@link PTransform}s for reading and writing JSON files. + * + *

Reading JSON files

+ * + *

Reading from JSON files is not yet implemented in Java. Please see https://github.com/apache/beam/issues/24552. + * + *

Writing JSON files

+ * + *

To write a {@link PCollection} to one or more line-delimited JSON files, use {@link + * JsonIO.Write}, using{@link JsonIO#writeRows} or {@link JsonIO#write}. {@link JsonIO.Write} + * supports writing {@link Row} or custom Java types using an inferred {@link Schema}. Examples + * below show both scenarios. See the Beam Programming Guide on inferring + * schemas for more information on how to enable Beam to infer a {@link Schema} from a custom + * Java type. + * + *

Example usage:

+ * + *

Suppose we have the following Transaction class annotated with + * {@code @DefaultSchema(JavaBeanSchema.class)} so that Beam can infer its {@link Schema}: + * + *

{@code @DefaultSchema(JavaBeanSchema.class)
+ * public class Transaction {
+ *   public Transaction() { … }
+ *   public Long getTransactionId();
+ *   public void setTransactionId(Long transactionId) { … }
+ *   public String getBank() { … }
+ *   public void setBank(String bank) { … }
+ *   public double getPurchaseAmount() { … }
+ *   public void setPurchaseAmount(double purchaseAmount) { … }
+ * }
+ * }
+ * + *

From a {@code PCollection}, {@link JsonIO.Write} can write one or many JSON + * files. + * + *

{@code
+ * PCollection transactions = ...
+ * transactions.apply(JsonIO.write("path/to/folder/prefix"));
+ * }
+ * + *

The resulting JSON files will look like the following where the header is repeated for every + * file, whereas by default, {@link JsonIO.Write} will write all fields in sorted order of + * the field names. + * + *

{@code
+ * {"bank": "A", "purchaseAmount": 10.23, "transactionId": 12345}
+ * {"bank": "B", "purchaseAmount": 54.65, "transactionId": 54321}
+ * {"bank": "C", "purchaseAmount": 11,76, "transactionId": 98765}
+ * }
+ * + *

A {@link PCollection} of {@link Row}s works just like custom Java types illustrated above, + * except we use {@link JsonIO#writeRows} as shown below for the same {@code Transaction} class. We + * derive {@code Transaction}'s {@link Schema} using a {@link + * org.apache.beam.sdk.schemas.annotations.DefaultSchema.DefaultSchemaProvider}. Note that + * hard-coding the {@link Row}s below is for illustration purposes. Developers are instead + * encouraged to take advantage of {@link + * org.apache.beam.sdk.schemas.annotations.DefaultSchema.DefaultSchemaProvider#toRowFunction}. + * + *

{@code
+ * DefaultSchemaProvider defaultSchemaProvider = new DefaultSchemaProvider();
+ * Schema schema = defaultSchemaProvider.schemaFor(TypeDescriptor.of(Transaction.class));
+ * PCollection transactions = pipeline.apply(Create.of(
+ *  Row
+ *    .withSchema(schema)
+ *    .withFieldValue("bank", "A")
+ *    .withFieldValue("purchaseAmount", 10.23)
+ *    .withFieldValue("transactionId", "12345")
+ *    .build(),
+ *  Row
+ *    .withSchema(schema)
+ *    .withFieldValue("bank", "B")
+ *    .withFieldValue("purchaseAmount", 54.65)
+ *    .withFieldValue("transactionId", "54321")
+ *    .build(),
+ *  Row
+ *    .withSchema(schema)
+ *    .withFieldValue("bank", "C")
+ *    .withFieldValue("purchaseAmount", 11.76)
+ *    .withFieldValue("transactionId", "98765")
+ *    .build()
+ * );
+ *
+ * transactions.apply(
+ *  JsonIO
+ *    .writeRowsTo("gs://bucket/path/to/folder/prefix")
+ * );
+ * }
+ * + *

Writing the transactions {@link PCollection} of {@link Row}s would yield the following JSON + * file content. + * + *

{@code
+ * {"bank": "A", "purchaseAmount": 10.23, "transactionId": 12345}
+ * {"bank": "B", "purchaseAmount": 54.65, "transactionId": 54321}
+ * {"bank": "C", "purchaseAmount": 11,76, "transactionId": 98765}
+ * }
+ */ +public class JsonIO { + static final String DEFAULT_FILENAME_SUFFIX = ".json"; + + /** Instantiates a {@link Write} for writing user types in {@link JSONFormat} format. */ + public static Write write(String to) { + return new AutoValue_JsonIO_Write.Builder() + .setTextIOWrite(createDefaultTextIOWrite(to)) + .build(); + } + + /** Instantiates a {@link Write} for writing {@link Row}s in {@link JSONFormat} format. */ + public static Write writeRows(String to) { + return new AutoValue_JsonIO_Write.Builder() + .setTextIOWrite(createDefaultTextIOWrite(to)) + .build(); + } + + /** {@link PTransform} for writing JSON files. */ + @AutoValue + public abstract static class Write + extends PTransform, WriteFilesResult> { + + /** Specifies the {@link Compression} of all generated shard files. */ + public Write withCompression(Compression compression) { + return toBuilder().setTextIOWrite(getTextIOWrite().withCompression(compression)).build(); + } + + /** Whether to skip the spilling of data. See {@link WriteFiles#withNoSpilling}. */ + public Write withNoSpilling() { + return toBuilder().setTextIOWrite(getTextIOWrite().withNoSpilling()).build(); + } + + /** + * Specifies to use a given fixed number of shards per window. See {@link + * TextIO.Write#withNumShards}. + */ + public Write withNumShards(Integer numShards) { + return toBuilder().setTextIOWrite(getTextIOWrite().withNumShards(numShards)).build(); + } + + /** + * Forces a single file as output and empty shard name template. See {@link + * TextIO.Write#withoutSharding}. + */ + public Write withoutSharding() { + return toBuilder().setTextIOWrite(getTextIOWrite().withoutSharding()).build(); + } + + /** + * Uses the given {@link ShardNameTemplate} for naming output files. See {@link + * TextIO.Write#withShardNameTemplate}. + */ + public Write withShardTemplate(String shardTemplate) { + return toBuilder() + .setTextIOWrite(getTextIOWrite().withShardNameTemplate(shardTemplate)) + .build(); + } + + /** Configures the filename suffix for written files. See {@link TextIO.Write#withSuffix}. */ + public Write withSuffix(String suffix) { + return toBuilder().setTextIOWrite(getTextIOWrite().withSuffix(suffix)).build(); + } + + /** + * Set the base directory used to generate temporary files. See {@link + * TextIO.Write#withTempDirectory}. + */ + public Write withTempDirectory(ResourceId tempDirectory) { + return toBuilder().setTextIOWrite(getTextIOWrite().withTempDirectory(tempDirectory)).build(); + } + + /** + * Preserves windowing of input elements and writes them to files based on the element's window. + * See {@link TextIO.Write#withWindowedWrites}. + */ + public Write withWindowedWrites() { + return toBuilder().setTextIOWrite(getTextIOWrite().withWindowedWrites()).build(); + } + + /** + * Returns a transform for writing to text files like this one but that has the given {@link + * FileBasedSink.WritableByteChannelFactory} to be used by the {@link FileBasedSink} during + * output. See {@link TextIO.Write#withWritableByteChannelFactory}. + */ + public Write withWritableByteChannelFactory( + FileBasedSink.WritableByteChannelFactory writableByteChannelFactory) { + return toBuilder() + .setTextIOWrite( + getTextIOWrite().withWritableByteChannelFactory(writableByteChannelFactory)) + .build(); + } + + /** The underlying {@link FileIO.Write} that writes converted input to JSON formatted output. */ + abstract TextIO.Write getTextIOWrite(); + + abstract Builder toBuilder(); + + @AutoValue.Builder + abstract static class Builder { + + /** + * The underlying {@link FileIO.Write} that writes converted input to JSON formatted output. + */ + abstract Builder setTextIOWrite(TextIO.Write value); + + abstract Write autoBuild(); + + final Write build() { + return autoBuild(); + } + } + + @Override + public WriteFilesResult expand(PCollection input) { + if (!input.hasSchema()) { + throw new IllegalArgumentException( + String.format( + "%s requires an input Schema. Note that only Row or user classes are supported. Consider using TextIO or FileIO directly when writing primitive types", + Write.class.getName())); + } + + Schema schema = input.getSchema(); + + RowCoder rowCoder = RowCoder.of(schema); + + PCollection rows = + input + .apply("To Rows", MapElements.into(rows()).via(input.getToRowFunction())) + .setCoder(rowCoder); + + SerializableFunction toJsonFn = + JsonUtils.getRowToJsonStringsFunction(input.getSchema()); + + PCollection json = rows.apply("To JSON", MapElements.into(strings()).via(toJsonFn)); + + return json.apply("Write JSON", getTextIOWrite().withOutputFilenames()); + } + } + + private static TextIO.Write createDefaultTextIOWrite(String to) { + return TextIO.write().to(to).withSuffix(DEFAULT_FILENAME_SUFFIX); + } +} diff --git a/sdks/java/io/json/src/main/java/org/apache/beam/sdk/io/json/package-info.java b/sdks/java/io/json/src/main/java/org/apache/beam/sdk/io/json/package-info.java new file mode 100644 index 000000000000..1ee191835713 --- /dev/null +++ b/sdks/java/io/json/src/main/java/org/apache/beam/sdk/io/json/package-info.java @@ -0,0 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** Transforms for reading and writing JSON files. */ +package org.apache.beam.sdk.io.json; diff --git a/sdks/java/io/json/src/main/java/org/apache/beam/sdk/io/json/providers/JsonWriteTransformProvider.java b/sdks/java/io/json/src/main/java/org/apache/beam/sdk/io/json/providers/JsonWriteTransformProvider.java new file mode 100644 index 000000000000..9e030821e5ca --- /dev/null +++ b/sdks/java/io/json/src/main/java/org/apache/beam/sdk/io/json/providers/JsonWriteTransformProvider.java @@ -0,0 +1,142 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.json.providers; + +import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; + +import com.google.auto.service.AutoService; +import com.google.auto.value.AutoValue; +import java.util.Collections; +import java.util.List; +import org.apache.beam.sdk.io.WriteFilesResult; +import org.apache.beam.sdk.io.json.JsonIO; +import org.apache.beam.sdk.schemas.AutoValueSchema; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.Schema.Field; +import org.apache.beam.sdk.schemas.Schema.FieldType; +import org.apache.beam.sdk.schemas.annotations.DefaultSchema; +import org.apache.beam.sdk.schemas.annotations.SchemaFieldDescription; +import org.apache.beam.sdk.schemas.transforms.SchemaTransform; +import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider; +import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.values.PCollectionRowTuple; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.TypeDescriptors; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Strings; + +/** + * An implementation of {@link TypedSchemaTransformProvider} for {@link JsonIO#write}. + * + *

Internal only: This class is actively being worked on, and it will likely change. We + * provide no backwards compatibility guarantees, and it should not be implemented outside the Beam + * repository. + */ +@SuppressWarnings({ + "nullness" // TODO(https://github.com/apache/beam/issues/20497) +}) +@AutoService(SchemaTransformProvider.class) +public class JsonWriteTransformProvider + extends TypedSchemaTransformProvider { + private static final String INPUT_ROWS_TAG = "input"; + private static final String WRITE_RESULTS = "output"; + + @Override + protected Class configurationClass() { + return JsonWriteConfiguration.class; + } + + @Override + protected SchemaTransform from(JsonWriteConfiguration configuration) { + return new JsonWriteTransform(configuration); + } + + @Override + public String identifier() { + return String.format("beam:schematransform:org.apache.beam:json_write:v1"); + } + + @Override + public List inputCollectionNames() { + return Collections.singletonList(INPUT_ROWS_TAG); + } + + @Override + public List outputCollectionNames() { + return Collections.singletonList(WRITE_RESULTS); + } + + /** Configuration for writing to BigQuery with Storage Write API. */ + @DefaultSchema(AutoValueSchema.class) + @AutoValue + public abstract static class JsonWriteConfiguration { + + public void validate() { + checkArgument( + !Strings.isNullOrEmpty(this.getPath()), "Path for a JSON Write must be specified."); + } + + public static Builder builder() { + return new AutoValue_JsonWriteTransformProvider_JsonWriteConfiguration.Builder(); + } + + @SchemaFieldDescription("The file path to write to.") + public abstract String getPath(); + + /** Builder for {@link JsonWriteConfiguration}. */ + @AutoValue.Builder + public abstract static class Builder { + + public abstract Builder setPath(String path); + + /** Builds a {@link JsonWriteConfiguration} instance. */ + public abstract JsonWriteConfiguration build(); + } + } + + /** A {@link SchemaTransform} for {@link JsonIO#write}. */ + protected static class JsonWriteTransform extends SchemaTransform { + + private final JsonWriteConfiguration configuration; + + JsonWriteTransform(JsonWriteConfiguration configuration) { + configuration.validate(); + this.configuration = configuration; + } + + @Override + public PCollectionRowTuple expand(PCollectionRowTuple input) { + WriteFilesResult result = + input.get(INPUT_ROWS_TAG).apply(JsonIO.writeRows(configuration.getPath()).withSuffix("")); + Schema outputSchema = Schema.of(Field.of("filename", FieldType.STRING)); + return PCollectionRowTuple.of( + WRITE_RESULTS, + result + .getPerDestinationOutputFilenames() + .apply( + "Collect filenames", + MapElements.into(TypeDescriptors.rows()) + .via( + (destinationAndRow) -> + Row.withSchema(outputSchema) + .withFieldValue("filename", destinationAndRow.getValue()) + .build())) + .setRowSchema(outputSchema)); + } + } +} diff --git a/sdks/java/io/json/src/main/java/org/apache/beam/sdk/io/json/providers/package-info.java b/sdks/java/io/json/src/main/java/org/apache/beam/sdk/io/json/providers/package-info.java new file mode 100644 index 000000000000..312454f8733b --- /dev/null +++ b/sdks/java/io/json/src/main/java/org/apache/beam/sdk/io/json/providers/package-info.java @@ -0,0 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** Transforms for reading and writing JSON files. */ +package org.apache.beam.sdk.io.json.providers; diff --git a/sdks/java/io/json/src/test/java/org/apache/beam/sdk/io/json/JsonIOWriteTest.java b/sdks/java/io/json/src/test/java/org/apache/beam/sdk/io/json/JsonIOWriteTest.java new file mode 100644 index 000000000000..71fdcd6b3d94 --- /dev/null +++ b/sdks/java/io/json/src/test/java/org/apache/beam/sdk/io/json/JsonIOWriteTest.java @@ -0,0 +1,145 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.json; + +import static org.apache.beam.sdk.io.common.SchemaAwareJavaBeans.allPrimitiveDataTypes; +import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; + +import java.io.File; +import java.io.IOException; +import java.math.BigDecimal; +import org.apache.beam.sdk.io.TextIO; +import org.apache.beam.sdk.io.common.SchemaAwareJavaBeans.AllPrimitiveDataTypes; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.SerializableMatcher; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.values.PCollection; +import org.hamcrest.BaseMatcher; +import org.hamcrest.Description; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Tests for {@link JsonIO.Write}. */ +@RunWith(JUnit4.class) +public class JsonIOWriteTest { + @Rule public TestPipeline writePipeline = TestPipeline.create(); + + @Rule public TestPipeline readPipeline = TestPipeline.create(); + + @Rule + public TestPipeline errorPipeline = TestPipeline.create().enableAbandonedNodeEnforcement(false); + + @Rule public TemporaryFolder tempFolder = new TemporaryFolder(); + + @Test + public void writesUserDefinedTypes() { + File folder = + createFolder(AllPrimitiveDataTypes.class.getSimpleName(), "writesUserDefinedTypes"); + + PCollection input = + writePipeline.apply( + Create.of( + allPrimitiveDataTypes(false, BigDecimal.TEN, 1.0, 1.0f, 1, 1L, "a"), + allPrimitiveDataTypes( + false, BigDecimal.TEN.add(BigDecimal.TEN), 2.0, 2.0f, 2, 2L, "b"), + allPrimitiveDataTypes( + false, + BigDecimal.TEN.add(BigDecimal.TEN).add(BigDecimal.TEN), + 3.0, + 3.0f, + 3, + 3L, + "c"))); + + input.apply(JsonIO.write(toFilenamePrefix(folder)).withNumShards(1)); + + writePipeline.run().waitUntilFinish(); + + PAssert.that(readPipeline.apply(TextIO.read().from(toFilenamePrefix(folder) + "*"))) + .containsInAnyOrder( + containsAll( + "\"aDouble\":1.0", + "\"aFloat\":1.0", + "\"aLong\":1", + "\"aString\":\"a\"", + "\"anInteger\":1", + "\"aDecimal\":10", + "\"aBoolean\":false"), + containsAll( + "\"aDouble\":2.0", + "\"aFloat\":2.0", + "\"aLong\":2", + "\"aString\":\"b\"", + "\"anInteger\":2", + "\"aDecimal\":20", + "\"aBoolean\":false"), + containsAll( + "\"aDouble\":3.0", + "\"aFloat\":3.0", + "\"aLong\":3", + "\"aString\":\"c\"", + "\"anInteger\":3", + "\"aDecimal\":30", + "\"aBoolean\":false")); + + readPipeline.run(); + } + + private static SerializableMatcher containsAll(String... needles) { + class Matcher extends BaseMatcher implements SerializableMatcher { + @Override + public boolean matches(Object item) { + if (!(item instanceof String)) { + return false; + } + + String haystack = (String) item; + for (String needle : needles) { + if (!haystack.contains(needle)) { + return false; + } + } + return true; + } + + @Override + public void describeTo(Description description) { + description.appendText("Contains all of: "); + description.appendValueList("[", ",", "]", needles); + } + } + return new Matcher(); + } + + private static String toFilenamePrefix(File folder) { + checkArgument(folder.isDirectory()); + return folder.getAbsolutePath() + "/out"; + } + + private File createFolder(String... paths) { + try { + return tempFolder.newFolder(paths); + } catch (IOException e) { + throw new IllegalStateException(e); + } + } +} diff --git a/sdks/python/apache_beam/yaml/standard_io.yaml b/sdks/python/apache_beam/yaml/standard_io.yaml index 8a9e0c100496..c4748483b04b 100644 --- a/sdks/python/apache_beam/yaml/standard_io.yaml +++ b/sdks/python/apache_beam/yaml/standard_io.yaml @@ -105,3 +105,10 @@ 'WriteToParquet': 'apache_beam.io.WriteToParquet' 'ReadFromAvro': 'apache_beam.io.ReadFromAvro' 'WriteToAvro': 'apache_beam.io.WriteToAvro' + +- type: beamJar + transforms: + 'WriteToCsv': 'beam:schematransform:org.apache.beam:csv_write:v1' + 'WriteToJson': 'beam:schematransform:org.apache.beam:json_write:v1' + config: + gradle_target: 'sdks:java:extensions:schemaio-expansion-service:shadowJar' diff --git a/settings.gradle.kts b/settings.gradle.kts index f4901d7df92b..c370c5da27d1 100644 --- a/settings.gradle.kts +++ b/settings.gradle.kts @@ -244,6 +244,7 @@ include(":sdks:java:io:hbase") include(":sdks:java:io:hcatalog") include(":sdks:java:io:jdbc") include(":sdks:java:io:jms") +include(":sdks:java:io:json") include(":sdks:java:io:kafka") include(":sdks:java:io:kinesis") include(":sdks:java:io:kinesis:expansion-service") From 2bbb3485c78ede3c4acddd462158814157f2b46f Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Mon, 9 Oct 2023 11:21:07 -0700 Subject: [PATCH 082/435] Populate top-level display data in yaml main. (#28512) --- sdks/python/apache_beam/pipeline.py | 28 +++++++++++++++---- sdks/python/apache_beam/transforms/display.py | 23 +++++++++------ sdks/python/apache_beam/yaml/main.py | 17 ++++++----- 3 files changed, 47 insertions(+), 21 deletions(-) diff --git a/sdks/python/apache_beam/pipeline.py b/sdks/python/apache_beam/pipeline.py index 042b483d50f1..14177cd603d8 100644 --- a/sdks/python/apache_beam/pipeline.py +++ b/sdks/python/apache_beam/pipeline.py @@ -88,6 +88,7 @@ from apache_beam.transforms import ParDo from apache_beam.transforms import ptransform from apache_beam.transforms.display import DisplayData +from apache_beam.transforms.display import HasDisplayData from apache_beam.transforms.resources import merge_resource_hints from apache_beam.transforms.resources import resource_hints_from_options from apache_beam.transforms.sideinputs import get_sideinput_index @@ -108,7 +109,7 @@ __all__ = ['Pipeline', 'PTransformOverride'] -class Pipeline(object): +class Pipeline(HasDisplayData): """A pipeline object that manages a DAG of :class:`~apache_beam.pvalue.PValue` s and their :class:`~apache_beam.transforms.ptransform.PTransform` s. @@ -133,9 +134,12 @@ def runner_implemented_transforms(cls): common_urns.primitives.IMPULSE.urn, ]) - def __init__(self, runner=None, options=None, argv=None): - # type: (Optional[Union[str, PipelineRunner]], Optional[PipelineOptions], Optional[List[str]]) -> None - + def __init__( + self, + runner: Optional[Union[str, PipelineRunner]] = None, + options: Optional[PipelineOptions] = None, + argv: Optional[List[str]] = None, + display_data: Optional[Dict[str, Any]] = None): """Initialize a pipeline object. Args: @@ -151,6 +155,8 @@ def __init__(self, runner=None, options=None, argv=None): to be used for building a :class:`~apache_beam.options.pipeline_options.PipelineOptions` object. This will only be used if argument **options** is :data:`None`. + display_data (Dict[str: Any]): a dictionary of static data associated + with this pipeline that can be displayed when it runs. Raises: ValueError: if either the runner or options argument is not @@ -233,6 +239,11 @@ def __init__(self, runner=None, options=None, argv=None): # Records whether this pipeline contains any external transforms. self.contains_external_transforms = False + self._display_data = display_data or {} + + def display_data(self): + # type: () -> Dict[str, Any] + return self._display_data @property # type: ignore[misc] # decorated property not supported def options(self): @@ -914,7 +925,8 @@ def visit_transform(self, transform_node): proto = beam_runner_api_pb2.Pipeline( root_transform_ids=[root_transform_id], components=context.to_runner_api(), - requirements=context.requirements()) + requirements=context.requirements(), + display_data=DisplayData('', self._display_data).to_proto()) proto.components.transforms[root_transform_id].unique_name = ( root_transform_id) self.merge_compatible_environments(proto) @@ -970,7 +982,11 @@ def from_runner_api( # type: (...) -> Pipeline """For internal use only; no backwards-compatibility guarantees.""" - p = Pipeline(runner=runner, options=options) + p = Pipeline( + runner=runner, + options=options, + display_data={str(ix): d + for ix, d in enumerate(proto.display_data)}) from apache_beam.runners import pipeline_context context = pipeline_context.PipelineContext( proto.components, requirements=proto.requirements) diff --git a/sdks/python/apache_beam/transforms/display.py b/sdks/python/apache_beam/transforms/display.py index b52a8fd5b6dd..0d1dd552413e 100644 --- a/sdks/python/apache_beam/transforms/display.py +++ b/sdks/python/apache_beam/transforms/display.py @@ -45,6 +45,7 @@ from datetime import timedelta from typing import TYPE_CHECKING from typing import List +from typing import Union from apache_beam.portability import common_urns from apache_beam.portability.api import beam_runner_api_pb2 @@ -101,7 +102,8 @@ def __init__( ): # type: (...) -> None self.namespace = namespace - self.items = [] # type: List[DisplayDataItem] + self.items = [ + ] # type: List[Union[DisplayDataItem, beam_runner_api_pb2.DisplayData]] self._populate_items(display_data_dict) def _populate_items(self, display_data_dict): @@ -112,26 +114,31 @@ def _populate_items(self, display_data_dict): subcomponent_display_data = DisplayData( element._get_display_data_namespace(), element.display_data()) self.items += subcomponent_display_data.items - continue - if isinstance(element, DisplayDataItem): + elif isinstance(element, DisplayDataItem): if element.should_drop(): continue element.key = key element.namespace = self.namespace self.items.append(element) - continue - # If it's not a HasDisplayData element, - # nor a dictionary, then it's a simple value - self.items.append( - DisplayDataItem(element, namespace=self.namespace, key=key)) + elif isinstance(element, beam_runner_api_pb2.DisplayData): + self.items.append(element) + + else: + # If it's not a HasDisplayData element, + # nor a dictionary, then it's a simple value + self.items.append( + DisplayDataItem(element, namespace=self.namespace, key=key)) def to_proto(self): # type: (...) -> List[beam_runner_api_pb2.DisplayData] """Returns a List of Beam proto representation of Display data.""" def create_payload(dd): + if isinstance(dd, beam_runner_api_pb2.DisplayData): + return dd + display_data_dict = None try: display_data_dict = dd.get_dict() diff --git a/sdks/python/apache_beam/yaml/main.py b/sdks/python/apache_beam/yaml/main.py index eb0695f337b4..e2ec8df9cfc3 100644 --- a/sdks/python/apache_beam/yaml/main.py +++ b/sdks/python/apache_beam/yaml/main.py @@ -51,19 +51,22 @@ def _pipeline_spec_from_args(known_args): raise ValueError( "Exactly one of pipeline_spec or pipeline_spec_file must be set.") - return yaml.load(pipeline_yaml, Loader=yaml_transform.SafeLineLoader) + return pipeline_yaml def run(argv=None): yaml_transform._LOGGER.setLevel('INFO') known_args, pipeline_args = _configure_parser(argv) - pipeline_spec = _pipeline_spec_from_args(known_args) + pipeline_yaml = _pipeline_spec_from_args(known_args) + pipeline_spec = yaml.load(pipeline_yaml, Loader=yaml_transform.SafeLineLoader) - with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions( - pipeline_args, - pickle_library='cloudpickle', - **yaml_transform.SafeLineLoader.strip_metadata(pipeline_spec.get( - 'options', {})))) as p: + with beam.Pipeline( # linebreak for better yapf formatting + options=beam.options.pipeline_options.PipelineOptions( + pipeline_args, + pickle_library='cloudpickle', + **yaml_transform.SafeLineLoader.strip_metadata(pipeline_spec.get( + 'options', {}))), + display_data={'yaml': pipeline_yaml}) as p: print("Building pipeline...") yaml_transform.expand_pipeline(p, pipeline_spec) print("Running pipeline...") From 3844972d681a317c1281685edc5f6fc8783c8043 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Mon, 9 Oct 2023 16:15:10 -0700 Subject: [PATCH 083/435] Revert "Populate top-level display data in yaml main. (#28512)" This reverts commit 2bbb3485c78ede3c4acddd462158814157f2b46f. --- sdks/python/apache_beam/pipeline.py | 28 ++++--------------- sdks/python/apache_beam/transforms/display.py | 23 ++++++--------- sdks/python/apache_beam/yaml/main.py | 17 +++++------ 3 files changed, 21 insertions(+), 47 deletions(-) diff --git a/sdks/python/apache_beam/pipeline.py b/sdks/python/apache_beam/pipeline.py index 14177cd603d8..042b483d50f1 100644 --- a/sdks/python/apache_beam/pipeline.py +++ b/sdks/python/apache_beam/pipeline.py @@ -88,7 +88,6 @@ from apache_beam.transforms import ParDo from apache_beam.transforms import ptransform from apache_beam.transforms.display import DisplayData -from apache_beam.transforms.display import HasDisplayData from apache_beam.transforms.resources import merge_resource_hints from apache_beam.transforms.resources import resource_hints_from_options from apache_beam.transforms.sideinputs import get_sideinput_index @@ -109,7 +108,7 @@ __all__ = ['Pipeline', 'PTransformOverride'] -class Pipeline(HasDisplayData): +class Pipeline(object): """A pipeline object that manages a DAG of :class:`~apache_beam.pvalue.PValue` s and their :class:`~apache_beam.transforms.ptransform.PTransform` s. @@ -134,12 +133,9 @@ def runner_implemented_transforms(cls): common_urns.primitives.IMPULSE.urn, ]) - def __init__( - self, - runner: Optional[Union[str, PipelineRunner]] = None, - options: Optional[PipelineOptions] = None, - argv: Optional[List[str]] = None, - display_data: Optional[Dict[str, Any]] = None): + def __init__(self, runner=None, options=None, argv=None): + # type: (Optional[Union[str, PipelineRunner]], Optional[PipelineOptions], Optional[List[str]]) -> None + """Initialize a pipeline object. Args: @@ -155,8 +151,6 @@ def __init__( to be used for building a :class:`~apache_beam.options.pipeline_options.PipelineOptions` object. This will only be used if argument **options** is :data:`None`. - display_data (Dict[str: Any]): a dictionary of static data associated - with this pipeline that can be displayed when it runs. Raises: ValueError: if either the runner or options argument is not @@ -239,11 +233,6 @@ def __init__( # Records whether this pipeline contains any external transforms. self.contains_external_transforms = False - self._display_data = display_data or {} - - def display_data(self): - # type: () -> Dict[str, Any] - return self._display_data @property # type: ignore[misc] # decorated property not supported def options(self): @@ -925,8 +914,7 @@ def visit_transform(self, transform_node): proto = beam_runner_api_pb2.Pipeline( root_transform_ids=[root_transform_id], components=context.to_runner_api(), - requirements=context.requirements(), - display_data=DisplayData('', self._display_data).to_proto()) + requirements=context.requirements()) proto.components.transforms[root_transform_id].unique_name = ( root_transform_id) self.merge_compatible_environments(proto) @@ -982,11 +970,7 @@ def from_runner_api( # type: (...) -> Pipeline """For internal use only; no backwards-compatibility guarantees.""" - p = Pipeline( - runner=runner, - options=options, - display_data={str(ix): d - for ix, d in enumerate(proto.display_data)}) + p = Pipeline(runner=runner, options=options) from apache_beam.runners import pipeline_context context = pipeline_context.PipelineContext( proto.components, requirements=proto.requirements) diff --git a/sdks/python/apache_beam/transforms/display.py b/sdks/python/apache_beam/transforms/display.py index 0d1dd552413e..b52a8fd5b6dd 100644 --- a/sdks/python/apache_beam/transforms/display.py +++ b/sdks/python/apache_beam/transforms/display.py @@ -45,7 +45,6 @@ from datetime import timedelta from typing import TYPE_CHECKING from typing import List -from typing import Union from apache_beam.portability import common_urns from apache_beam.portability.api import beam_runner_api_pb2 @@ -102,8 +101,7 @@ def __init__( ): # type: (...) -> None self.namespace = namespace - self.items = [ - ] # type: List[Union[DisplayDataItem, beam_runner_api_pb2.DisplayData]] + self.items = [] # type: List[DisplayDataItem] self._populate_items(display_data_dict) def _populate_items(self, display_data_dict): @@ -114,31 +112,26 @@ def _populate_items(self, display_data_dict): subcomponent_display_data = DisplayData( element._get_display_data_namespace(), element.display_data()) self.items += subcomponent_display_data.items + continue - elif isinstance(element, DisplayDataItem): + if isinstance(element, DisplayDataItem): if element.should_drop(): continue element.key = key element.namespace = self.namespace self.items.append(element) + continue - elif isinstance(element, beam_runner_api_pb2.DisplayData): - self.items.append(element) - - else: - # If it's not a HasDisplayData element, - # nor a dictionary, then it's a simple value - self.items.append( - DisplayDataItem(element, namespace=self.namespace, key=key)) + # If it's not a HasDisplayData element, + # nor a dictionary, then it's a simple value + self.items.append( + DisplayDataItem(element, namespace=self.namespace, key=key)) def to_proto(self): # type: (...) -> List[beam_runner_api_pb2.DisplayData] """Returns a List of Beam proto representation of Display data.""" def create_payload(dd): - if isinstance(dd, beam_runner_api_pb2.DisplayData): - return dd - display_data_dict = None try: display_data_dict = dd.get_dict() diff --git a/sdks/python/apache_beam/yaml/main.py b/sdks/python/apache_beam/yaml/main.py index e2ec8df9cfc3..eb0695f337b4 100644 --- a/sdks/python/apache_beam/yaml/main.py +++ b/sdks/python/apache_beam/yaml/main.py @@ -51,22 +51,19 @@ def _pipeline_spec_from_args(known_args): raise ValueError( "Exactly one of pipeline_spec or pipeline_spec_file must be set.") - return pipeline_yaml + return yaml.load(pipeline_yaml, Loader=yaml_transform.SafeLineLoader) def run(argv=None): yaml_transform._LOGGER.setLevel('INFO') known_args, pipeline_args = _configure_parser(argv) - pipeline_yaml = _pipeline_spec_from_args(known_args) - pipeline_spec = yaml.load(pipeline_yaml, Loader=yaml_transform.SafeLineLoader) + pipeline_spec = _pipeline_spec_from_args(known_args) - with beam.Pipeline( # linebreak for better yapf formatting - options=beam.options.pipeline_options.PipelineOptions( - pipeline_args, - pickle_library='cloudpickle', - **yaml_transform.SafeLineLoader.strip_metadata(pipeline_spec.get( - 'options', {}))), - display_data={'yaml': pipeline_yaml}) as p: + with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions( + pipeline_args, + pickle_library='cloudpickle', + **yaml_transform.SafeLineLoader.strip_metadata(pipeline_spec.get( + 'options', {})))) as p: print("Building pipeline...") yaml_transform.expand_pipeline(p, pipeline_spec) print("Running pipeline...") From 3d574b485abab1256b5597db2a94af70b858fa1c Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Tue, 10 Oct 2023 10:02:33 -0400 Subject: [PATCH 084/435] Fix sourceJar task dependencies (#28900) --- .../beam/gradle/BeamModulePlugin.groovy | 26 ++++++++++++++++++- runners/flink/flink_runner.gradle | 23 +++++++++++++--- .../maven-archetypes/examples/build.gradle | 7 +++++ .../gcp-bom-examples/build.gradle | 6 +++++ 4 files changed, 57 insertions(+), 5 deletions(-) diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index 9f4ae8bcfb29..705bcb978e12 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -2395,7 +2395,20 @@ class BeamModulePlugin implements Plugin { // TODO: Decide whether this should be inlined into the one project that relies on it // or be left here. - project.ext.applyAvroNature = { project.apply plugin: "com.commercehub.gradle.plugin.avro" } + project.ext.applyAvroNature = { + project.apply plugin: "com.commercehub.gradle.plugin.avro" + + // add dependency BeamModulePlugin defined custom tasks + // they are defined only when certain flags are provided (e.g. -Prelease; -Ppublishing, etc) + def sourcesJar = project.tasks.findByName('sourcesJar') + if (sourcesJar != null) { + sourcesJar.dependsOn project.tasks.getByName('generateAvroJava') + } + def testSourcesJar = project.tasks.findByName('testSourcesJar') + if (testSourcesJar != null) { + testSourcesJar.dependsOn project.tasks.getByName('generateTestAvroJava') + } + } project.ext.applyAntlrNature = { project.apply plugin: 'antlr' @@ -2406,6 +2419,17 @@ class BeamModulePlugin implements Plugin { generatedSourceDirs += project.generateTestGrammarSource.outputDirectory } } + + // add dependency BeamModulePlugin defined custom tasks + // they are defined only when certain flags are provided (e.g. -Prelease; -Ppublishing, etc) + def sourcesJar = project.tasks.findByName('sourcesJar') + if (sourcesJar != null) { + sourcesJar.mustRunAfter project.tasks.getByName('generateGrammarSource') + } + def testSourcesJar = project.tasks.findByName('testSourcesJar') + if (testSourcesJar != null) { + testSourcesJar.dependsOn project.tasks.getByName('generateTestGrammarSource') + } } // Creates a task to run the quickstart for a runner. diff --git a/runners/flink/flink_runner.gradle b/runners/flink/flink_runner.gradle index 30fb922e9c7c..c087575f8023 100644 --- a/runners/flink/flink_runner.gradle +++ b/runners/flink/flink_runner.gradle @@ -46,36 +46,51 @@ evaluationDependsOn(":examples:java") /* * Copy & merge source overrides into build directory. */ -def sourceOverridesBase = "${project.buildDir}/source-overrides/src" +def sourceOverridesBase = project.layout.buildDirectory.dir('source-overrides/src') def copySourceOverrides = tasks.register('copySourceOverrides', Copy) { it.from main_source_overrides it.into "${sourceOverridesBase}/main/java" it.duplicatesStrategy DuplicatesStrategy.INCLUDE } -compileJava.dependsOn copySourceOverrides def copyResourcesOverrides = tasks.register('copyResourcesOverrides', Copy) { it.from main_resources_overrides it.into "${sourceOverridesBase}/main/resources" it.duplicatesStrategy DuplicatesStrategy.INCLUDE } -processResources.dependsOn copyResourcesOverrides def copyTestSourceOverrides = tasks.register('copyTestSourceOverrides', Copy) { it.from test_source_overrides it.into "${sourceOverridesBase}/test/java" it.duplicatesStrategy DuplicatesStrategy.INCLUDE } -compileTestJava.dependsOn copyTestSourceOverrides def copyTestResourcesOverrides = tasks.register('copyTestResourcesOverrides', Copy) { it.from test_resources_overrides it.into "${sourceOverridesBase}/test/resources" it.duplicatesStrategy DuplicatesStrategy.INCLUDE } + +// add dependency to gradle Java plugin defined tasks +compileJava.dependsOn copySourceOverrides +processResources.dependsOn copyResourcesOverrides +compileTestJava.dependsOn copyTestSourceOverrides processTestResources.dependsOn copyTestResourcesOverrides +// add dependency BeamModulePlugin defined custom tasks +// they are defined only when certain flags are provided (e.g. -Prelease; -Ppublishing, etc) +def sourcesJar = project.tasks.findByName('sourcesJar') +if (sourcesJar != null) { + sourcesJar.dependsOn copySourceOverrides + sourcesJar.dependsOn copyResourcesOverrides +} +def testSourcesJar = project.tasks.findByName('testSourcesJar') +if (testSourcesJar != null) { + testSourcesJar.dependsOn copyTestSourceOverrides + testSourcesJar.dependsOn copyTestResourcesOverrides +} + /* * We have to explicitly set all directories here to make sure each * version of Flink has the correct overrides set. diff --git a/sdks/java/maven-archetypes/examples/build.gradle b/sdks/java/maven-archetypes/examples/build.gradle index 56b4a7c84285..1edb55a10f95 100644 --- a/sdks/java/maven-archetypes/examples/build.gradle +++ b/sdks/java/maven-archetypes/examples/build.gradle @@ -72,6 +72,13 @@ task generateSources(type: Exec) { commandLine './generate-sources.sh' } +// add dependency BeamModulePlugin defined custom tasks +// they are defined only when certain flags are provided (e.g. -Prelease; -Ppublishing, etc) +def sourcesJar = project.tasks.findByName('sourcesJar') +if (sourcesJar != null) { + sourcesJar.dependsOn generateSources +} + sourceSets { main { output.dir('src', builtBy: 'generateSources') diff --git a/sdks/java/maven-archetypes/gcp-bom-examples/build.gradle b/sdks/java/maven-archetypes/gcp-bom-examples/build.gradle index 541c91bd6adb..f9fabcfe19b0 100644 --- a/sdks/java/maven-archetypes/gcp-bom-examples/build.gradle +++ b/sdks/java/maven-archetypes/gcp-bom-examples/build.gradle @@ -71,6 +71,12 @@ task generateSources(type: Exec) { environment "HERE", "." commandLine '../examples/generate-sources.sh' } +// add dependency BeamModulePlugin defined custom tasks +// they are defined only when certain flags are provided (e.g. -Prelease; -Ppublishing, etc) +def sourcesJar = project.tasks.findByName('sourcesJar') +if (sourcesJar != null) { + sourcesJar.dependsOn generateSources +} sourceSets { main { From 152f9c52b57e8cc3c7d791ab787217c333e71f4d Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 10 Oct 2023 10:17:26 -0400 Subject: [PATCH 085/435] Bump golang.org/x/oauth2 from 0.12.0 to 0.13.0 in /sdks (#28912) Bumps [golang.org/x/oauth2](https://github.com/golang/oauth2) from 0.12.0 to 0.13.0. - [Commits](https://github.com/golang/oauth2/compare/v0.12.0...v0.13.0) --- updated-dependencies: - dependency-name: golang.org/x/oauth2 dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 2 +- sdks/go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index a2b5ed097642..53eae7db4539 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -53,7 +53,7 @@ require ( github.com/xitongsys/parquet-go-source v0.0.0-20220315005136-aec0fe3e777c go.mongodb.org/mongo-driver v1.12.1 golang.org/x/net v0.16.0 - golang.org/x/oauth2 v0.12.0 + golang.org/x/oauth2 v0.13.0 golang.org/x/sync v0.4.0 golang.org/x/sys v0.13.0 golang.org/x/text v0.13.0 diff --git a/sdks/go.sum b/sdks/go.sum index 516adec93875..5dc085b859a0 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -548,8 +548,8 @@ golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4Iltr golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20191202225959-858c2ad4c8b6/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20200107190931-bf48bf16ab8d/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= -golang.org/x/oauth2 v0.12.0 h1:smVPGxink+n1ZI5pkQa8y6fZT0RW0MgCO5bFpepy4B4= -golang.org/x/oauth2 v0.12.0/go.mod h1:A74bZ3aGXgCY0qaIC9Ahg6Lglin4AMAco8cIv9baba4= +golang.org/x/oauth2 v0.13.0 h1:jDDenyj+WgFtmV3zYVoi8aE2BwtXFLWOA67ZfNWftiY= +golang.org/x/oauth2 v0.13.0/go.mod h1:/JMhi4ZRXAf4HG9LiNmxvk+45+96RUlVThiH8FzNBn0= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= From e01de66459416f8f40dc3fbe64243fef9694ffc3 Mon Sep 17 00:00:00 2001 From: Ritesh Ghorse Date: Tue, 10 Oct 2023 10:42:23 -0400 Subject: [PATCH 086/435] remove unused os import (#28914) --- sdks/go/test/integration/integration.go | 1 - 1 file changed, 1 deletion(-) diff --git a/sdks/go/test/integration/integration.go b/sdks/go/test/integration/integration.go index d3eb1c3ee7c3..5b7473fb561a 100644 --- a/sdks/go/test/integration/integration.go +++ b/sdks/go/test/integration/integration.go @@ -38,7 +38,6 @@ package integration import ( "fmt" "math/rand" - "os" "regexp" "strings" "testing" From e130352d7c7eee8503e82c5e921126d12b88a111 Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Tue, 10 Oct 2023 10:43:10 -0400 Subject: [PATCH 087/435] Remove issue_comment trigger for load test, performance test and other jobs (#28915) --- .github/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml | 2 -- .github/workflows/beam_LoadTests_Go_CoGBK_Flink_batch.yml | 2 -- .github/workflows/beam_LoadTests_Go_Combine_Dataflow_Batch.yml | 2 -- .github/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml | 2 -- .github/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml | 2 -- .github/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml | 2 -- .github/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml | 2 -- .github/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml | 2 -- .../workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml | 2 -- .github/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml | 2 -- .../workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml | 2 -- .../workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml | 2 -- .../beam_LoadTests_Java_Combine_Dataflow_Streaming.yml | 2 -- ...am_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml | 2 -- .github/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml | 2 -- .../workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml | 2 -- .../beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml | 2 -- .../beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml | 2 -- .../beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml | 2 -- .../beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml | 2 -- .../beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch.yml | 2 -- .github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml | 2 -- .../workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml | 2 -- ...beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml | 2 -- .../workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml | 2 -- .github/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml | 2 -- .../workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml | 2 -- .../beam_LoadTests_Python_FnApiRunner_Microbenchmark.yml | 2 -- .github/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml | 2 -- .../workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml | 2 -- .github/workflows/beam_PerformanceTests_AvroIOIT.yml | 2 -- .github/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml | 2 -- .../beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml | 2 -- .../beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml | 2 -- .../beam_PerformanceTests_BigQueryIO_Streaming_Java.yml | 2 -- .../workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml | 2 -- .../beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml | 2 -- .github/workflows/beam_PerformanceTests_Cdap.yml | 2 -- .github/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml | 2 -- .../beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml | 2 -- .github/workflows/beam_PerformanceTests_HadoopFormat.yml | 2 -- .github/workflows/beam_PerformanceTests_JDBC.yml | 2 -- .github/workflows/beam_PerformanceTests_Kafka_IO.yml | 2 -- .github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml | 2 -- .../workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml | 2 -- .github/workflows/beam_PerformanceTests_MongoDBIO_IT.yml | 2 -- .github/workflows/beam_PerformanceTests_ParquetIOIT.yml | 2 -- .github/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml | 2 -- .../beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml | 2 -- .../beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml | 2 -- .../beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml | 2 -- .../beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml | 2 -- .github/workflows/beam_PerformanceTests_SparkReceiver_IO.yml | 2 -- .github/workflows/beam_PerformanceTests_TFRecordIOIT.yml | 2 -- .github/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml | 2 -- .github/workflows/beam_PerformanceTests_TextIOIT.yml | 2 -- .github/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml | 2 -- .github/workflows/beam_PerformanceTests_TextIOIT_Python.yml | 2 -- .../beam_PerformanceTests_WordCountIT_PythonVersions.yml | 2 -- .github/workflows/beam_PerformanceTests_XmlIOIT.yml | 2 -- .github/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml | 2 -- .../workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml | 2 -- .github/workflows/beam_Publish_Docker_Snapshots.yml | 2 -- .../workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml | 2 -- 64 files changed, 128 deletions(-) diff --git a/.github/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml index d90b7a5a4cc3..0d4c9ab48d27 100644 --- a/.github/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml @@ -16,8 +16,6 @@ name: Load Tests CoGBK Dataflow Batch Go on: - issue_comment: - types: [created] schedule: - cron: '40 23 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Go_CoGBK_Flink_batch.yml b/.github/workflows/beam_LoadTests_Go_CoGBK_Flink_batch.yml index cf355a2ee98d..18238c796e47 100644 --- a/.github/workflows/beam_LoadTests_Go_CoGBK_Flink_batch.yml +++ b/.github/workflows/beam_LoadTests_Go_CoGBK_Flink_batch.yml @@ -16,8 +16,6 @@ name: LoadTests Go CoGBK Flink Batch on: - issue_comment: - types: [created] schedule: - cron: '10 14 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Go_Combine_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Go_Combine_Dataflow_Batch.yml index 423290d3fdc6..172f48a83e33 100644 --- a/.github/workflows/beam_LoadTests_Go_Combine_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_Combine_Dataflow_Batch.yml @@ -16,8 +16,6 @@ name: Load Tests Combine Dataflow Batch Go on: - issue_comment: - types: [created] schedule: - cron: '40 23 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml index 0226b003b58e..e5dbef1df53d 100644 --- a/.github/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml @@ -16,8 +16,6 @@ name: LoadTests Go Combine Flink Batch on: - issue_comment: - types: [created] schedule: - cron: '40 6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml index bfdb19c1f5d5..ae4528ced8e4 100644 --- a/.github/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml @@ -16,8 +16,6 @@ name: Load Tests GBK Dataflow Batch Go on: - issue_comment: - types: [created] schedule: - cron: '40 23 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml index 7c4d95738a09..a26fe19b79b7 100644 --- a/.github/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml @@ -16,8 +16,6 @@ name: LoadTests Go GBK Flink Batch on: - issue_comment: - types: [created] schedule: - cron: '20 1 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml index e4b4f0997d4d..18dbf26409ad 100644 --- a/.github/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml @@ -16,8 +16,6 @@ name: Load Tests ParDo Dataflow Batch Go on: - issue_comment: - types: [created] schedule: - cron: '15 18 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml index 34211f9270ff..094005e2f532 100644 --- a/.github/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml @@ -16,8 +16,6 @@ name: LoadTests Go ParDo Flink Batch on: - issue_comment: - types: [created] schedule: - cron: '40 2 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml index cad15e4eae0c..6265fe428c00 100644 --- a/.github/workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml @@ -16,8 +16,6 @@ name: Load Tests SideInput Dataflow Batch Go on: - issue_comment: - types: [created] schedule: - cron: '40 23 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml index 8600c5cd3717..20e8b4d54dd3 100644 --- a/.github/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml @@ -16,8 +16,6 @@ name: LoadTests Go SideInput Flink Batch on: - issue_comment: - types: [created] schedule: - cron: '40 23 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml index 0cb601522a81..3ad2cdf995e7 100644 --- a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml @@ -16,8 +16,6 @@ name: Load Tests CoGBK Dataflow Streaming Java on: - issue_comment: - types: [created] schedule: - cron: '50 10 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml index 758d196f273c..bc408402c792 100644 --- a/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml @@ -16,8 +16,6 @@ name: Load Tests Combine Dataflow Batch Java on: - issue_comment: - types: [created] schedule: - cron: '35 7 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Streaming.yml index b0df52354722..029b7024d6ed 100644 --- a/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Streaming.yml @@ -16,8 +16,6 @@ name: LoadTests Java Combine Dataflow Streaming on: - issue_comment: - types: [created] schedule: - cron: '25 14 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml b/.github/workflows/beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml index 28c17ffb535c..5f4b8594ecde 100644 --- a/.github/workflows/beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml +++ b/.github/workflows/beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml @@ -16,8 +16,6 @@ name: LoadTests Java Combine SparkStructuredStreaming Batch on: - issue_comment: - types: [created] schedule: - cron: '15 18 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml index b490ce69323e..9f7da2c00049 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml @@ -16,8 +16,6 @@ name: LoadTests Java GBK Dataflow Batch on: - issue_comment: - types: [created] schedule: - cron: '10 6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml index 19cc12f91eb1..fd718e23fd0a 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml @@ -16,8 +16,6 @@ name: LoadTests Java GBK Dataflow Streaming on: - issue_comment: - types: [created] schedule: - cron: '50 6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml index aeb8006dfd53..318f157e57a6 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml @@ -16,8 +16,6 @@ name: LoadTests Java GBK Dataflow V2 Batch Java11 on: - issue_comment: - types: [created] schedule: - cron: '10 7 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml index 23f9f34d9747..543cb86985d8 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml @@ -16,8 +16,6 @@ name: LoadTests Java GBK Dataflow V2 Batch Java17 on: - issue_comment: - types: [created] schedule: - cron: '50 7 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml index 9a0b236e43e9..fcd55f761c04 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml @@ -16,8 +16,6 @@ name: LoadTests Java GBK Dataflow V2 Streaming Java11 on: - issue_comment: - types: [created] schedule: - cron: '50 8 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml index 3580aaca51bb..5df7d3249192 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml @@ -16,8 +16,6 @@ name: LoadTests Java GBK Dataflow V2 Streaming Java17 on: - issue_comment: - types: [created] schedule: - cron: '50 9 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch.yml b/.github/workflows/beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch.yml index 039a9fc3b752..27505e9e88e9 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch.yml @@ -16,8 +16,6 @@ name: LoadTests Java GBK SparkStructuredStreaming Batch on: - issue_comment: - types: [created] schedule: - cron: '10 10 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml index 6bd52d2ebae0..406cf18c85d1 100644 --- a/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml @@ -16,8 +16,6 @@ name: LoadTests Java ParDo Dataflow Batch on: - issue_comment: - types: [created] schedule: - cron: '55 9 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml index dbee457b6ec5..4b6fcb4ad51d 100644 --- a/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml @@ -16,8 +16,6 @@ name: LoadTests Java ParDo Dataflow Streaming on: - issue_comment: - types: [created] schedule: - cron: '10 11 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml b/.github/workflows/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml index afbb79caec91..89a903cc146c 100644 --- a/.github/workflows/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml +++ b/.github/workflows/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml @@ -16,8 +16,6 @@ name: LoadTests Java ParDo SparkStructuredStreaming Batch on: - issue_comment: - types: [created] schedule: - cron: '25 8 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml index f7d7a056d595..d8b97f2d6032 100644 --- a/.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml @@ -16,8 +16,6 @@ name: Load Tests Combine Dataflow Batch Python on: - issue_comment: - types: [created] schedule: - cron: '40 5 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml index 165b23703562..d7d80e164283 100644 --- a/.github/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml @@ -16,8 +16,6 @@ name: LoadTests Python Combine Flink Batch on: - issue_comment: - types: [created] schedule: - cron: '10 6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml b/.github/workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml index 1dae586de21a..66ac3bd2fc08 100644 --- a/.github/workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml @@ -16,8 +16,6 @@ name: LoadTests Python Combine Flink Streaming on: - issue_comment: - types: [created] schedule: - cron: '10 7 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Python_FnApiRunner_Microbenchmark.yml b/.github/workflows/beam_LoadTests_Python_FnApiRunner_Microbenchmark.yml index b74a44c647c8..f3e2181991d7 100644 --- a/.github/workflows/beam_LoadTests_Python_FnApiRunner_Microbenchmark.yml +++ b/.github/workflows/beam_LoadTests_Python_FnApiRunner_Microbenchmark.yml @@ -16,8 +16,6 @@ name: Load Tests FnApiRunner Microbenchmark Python on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml index fa3459992d0e..edf2e97857d6 100644 --- a/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml @@ -16,8 +16,6 @@ name: LoadTests Python GBK Dataflow Batch on: - issue_comment: - types: [created] schedule: - cron: '10 2 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml index da0cf8eefbb8..d7e17c2676f9 100644 --- a/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml @@ -16,8 +16,6 @@ name: LoadTests Python GBK Dataflow Streaming on: - issue_comment: - types: [created] schedule: - cron: '10 4 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_AvroIOIT.yml b/.github/workflows/beam_PerformanceTests_AvroIOIT.yml index a6c56287da41..14d798bb3e72 100644 --- a/.github/workflows/beam_PerformanceTests_AvroIOIT.yml +++ b/.github/workflows/beam_PerformanceTests_AvroIOIT.yml @@ -16,8 +16,6 @@ name: Performance Tests AvroIOIT on: - issue_comment: - types: [created] schedule: - cron: '10 1/13 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml b/.github/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml index 7393e0e39b37..b84d56f34b9b 100644 --- a/.github/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml +++ b/.github/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml @@ -16,8 +16,6 @@ name: Performance Tests AvroIOIT HDFS on: - issue_comment: - types: [created] schedule: - cron: '10 1/13 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml b/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml index d29acbfc765f..7f6daf23d05f 100644 --- a/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml +++ b/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml @@ -16,8 +16,6 @@ name: Performance Tests BigQueryIO Batch Java Avro on: - issue_comment: - types: [created] schedule: - cron: '10 1,13 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml b/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml index 067d0e4b95b8..13c60e107834 100644 --- a/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml +++ b/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml @@ -16,8 +16,6 @@ name: Performance Tests BigQueryIO Batch Java Json on: - issue_comment: - types: [created] schedule: - cron: '30 8,20 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml b/.github/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml index bf10d4be522e..75a68c65a8b3 100644 --- a/.github/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml +++ b/.github/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml @@ -16,8 +16,6 @@ name: Performance Tests BigQueryIO Streaming Java on: - issue_comment: - types: [created] schedule: - cron: '20 15,22 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml b/.github/workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml index 58958de69c89..ac84517443e9 100644 --- a/.github/workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml +++ b/.github/workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml @@ -16,8 +16,6 @@ name: PerformanceTests BiqQueryIO Read Python on: - issue_comment: - types: [created] schedule: - cron: '0 2 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml b/.github/workflows/beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml index f0fcd20bd3b5..d2e5ddf3d07c 100644 --- a/.github/workflows/beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml +++ b/.github/workflows/beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml @@ -16,8 +16,6 @@ name: PerformanceTests BiqQueryIO Write Python Batch on: - issue_comment: - types: [created] schedule: - cron: '0 1 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_Cdap.yml b/.github/workflows/beam_PerformanceTests_Cdap.yml index f45419a1223f..269809121449 100644 --- a/.github/workflows/beam_PerformanceTests_Cdap.yml +++ b/.github/workflows/beam_PerformanceTests_Cdap.yml @@ -16,8 +16,6 @@ name: PerformanceTests Cdap on: - issue_comment: - types: [created] schedule: - cron: '13 4/12 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml b/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml index 0e82c0fdf7d1..ed5e19454d26 100644 --- a/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml +++ b/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml @@ -16,8 +16,6 @@ name: PerformanceTests Compressed TextIOIT on: - issue_comment: - types: [created] schedule: - cron: '10 1/12 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml b/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml index 78c9d3e8ab7a..f43f96634459 100644 --- a/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml +++ b/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml @@ -16,8 +16,6 @@ name: PerformanceTests Compressed TextIOIT HDFS on: - issue_comment: - types: [created] schedule: - cron: '50 1/12 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_HadoopFormat.yml b/.github/workflows/beam_PerformanceTests_HadoopFormat.yml index d73c6f6fb5f5..f5514f765a57 100644 --- a/.github/workflows/beam_PerformanceTests_HadoopFormat.yml +++ b/.github/workflows/beam_PerformanceTests_HadoopFormat.yml @@ -16,8 +16,6 @@ name: PerformanceTests HadoopFormat on: - issue_comment: - types: [created] schedule: - cron: '16 7/12 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_JDBC.yml b/.github/workflows/beam_PerformanceTests_JDBC.yml index c5995480e9d5..350718e94449 100644 --- a/.github/workflows/beam_PerformanceTests_JDBC.yml +++ b/.github/workflows/beam_PerformanceTests_JDBC.yml @@ -16,8 +16,6 @@ name: PerformanceTests JDBC on: - issue_comment: - types: [created] schedule: - cron: '30 1,13 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_Kafka_IO.yml b/.github/workflows/beam_PerformanceTests_Kafka_IO.yml index ea709238b8f3..6bb79bcc5b61 100644 --- a/.github/workflows/beam_PerformanceTests_Kafka_IO.yml +++ b/.github/workflows/beam_PerformanceTests_Kafka_IO.yml @@ -16,8 +16,6 @@ name: PerformanceTests Kafka IO on: - issue_comment: - types: [created] schedule: - cron: '30 2,14 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml b/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml index 5834bd8ab3e0..7f533c157234 100644 --- a/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml +++ b/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml @@ -16,8 +16,6 @@ name: PerformanceTests ManyFiles TextIOIT on: - issue_comment: - types: [created] schedule: - cron: '10 2/12 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml b/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml index 03163a41dcf9..96bb0573a911 100644 --- a/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml +++ b/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml @@ -16,8 +16,6 @@ name: PerformanceTests ManyFiles TextIOIT HDFS on: - issue_comment: - types: [created] schedule: - cron: '50 2/12 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_MongoDBIO_IT.yml b/.github/workflows/beam_PerformanceTests_MongoDBIO_IT.yml index 0ad21f99f8d4..83e0a73a9c9b 100644 --- a/.github/workflows/beam_PerformanceTests_MongoDBIO_IT.yml +++ b/.github/workflows/beam_PerformanceTests_MongoDBIO_IT.yml @@ -16,8 +16,6 @@ name: PerformanceTests MongoDBIO IT on: - issue_comment: - types: [created] schedule: - cron: '14 5/12 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_ParquetIOIT.yml b/.github/workflows/beam_PerformanceTests_ParquetIOIT.yml index ceb540b16b1f..03dbb650c4ab 100644 --- a/.github/workflows/beam_PerformanceTests_ParquetIOIT.yml +++ b/.github/workflows/beam_PerformanceTests_ParquetIOIT.yml @@ -16,8 +16,6 @@ name: PerformanceTests ParquetIOIT on: - issue_comment: - types: [created] schedule: - cron: '10 3/12 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml b/.github/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml index d0c40599eb62..49c2296340be 100644 --- a/.github/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml +++ b/.github/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml @@ -16,8 +16,6 @@ name: PerformanceTests ParquetIOIT HDFS on: - issue_comment: - types: [created] schedule: - cron: '50 3/12 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml b/.github/workflows/beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml index 3a16e482979f..97082ea8aeda 100644 --- a/.github/workflows/beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml +++ b/.github/workflows/beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml @@ -16,8 +16,6 @@ name: PerformanceTests PubsubIOIT Python Streaming on: - issue_comment: - types: [created] schedule: - cron: '11 2 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml b/.github/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml index 06014a56e682..2a93234e94e2 100644 --- a/.github/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml +++ b/.github/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml @@ -16,8 +16,6 @@ name: PerformanceTests SQLBigQueryIO Batch Java on: - issue_comment: - types: [created] schedule: - cron: '0 7,19 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml b/.github/workflows/beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml index 9e464ef58900..09e165e64e52 100644 --- a/.github/workflows/beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml +++ b/.github/workflows/beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml @@ -16,8 +16,6 @@ name: PerformanceTests SpannerIO Read 2GB Python on: - issue_comment: - types: [created] schedule: - cron: '30 4 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml b/.github/workflows/beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml index 8cfce643f749..b1f7761b1133 100644 --- a/.github/workflows/beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml +++ b/.github/workflows/beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml @@ -16,8 +16,6 @@ name: PerformanceTests SpannerIO Write 2GB Python Batch on: - issue_comment: - types: [created] schedule: - cron: '0 5 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_SparkReceiver_IO.yml b/.github/workflows/beam_PerformanceTests_SparkReceiver_IO.yml index 58c561f6ef64..8f6c72afb415 100644 --- a/.github/workflows/beam_PerformanceTests_SparkReceiver_IO.yml +++ b/.github/workflows/beam_PerformanceTests_SparkReceiver_IO.yml @@ -16,8 +16,6 @@ name: PerformanceTests SparkReceiver IO on: - issue_comment: - types: [created] schedule: - cron: '15 6/12 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_TFRecordIOIT.yml b/.github/workflows/beam_PerformanceTests_TFRecordIOIT.yml index aa96b7e2bf81..93e9132e1d1d 100644 --- a/.github/workflows/beam_PerformanceTests_TFRecordIOIT.yml +++ b/.github/workflows/beam_PerformanceTests_TFRecordIOIT.yml @@ -16,8 +16,6 @@ name: PerformanceTests TFRecordIOIT on: - issue_comment: - types: [created] schedule: - cron: '10 4/12 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml b/.github/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml index bbc4a79aa0f3..dab14af741b7 100644 --- a/.github/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml +++ b/.github/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml @@ -18,8 +18,6 @@ name: PerformanceTests TFRecordIOIT HDFS on: # TODO(https://github.com/apache/beam/issues/18796) TFRecord performance test is failing only when running on hdfs. # We need to fix this before enabling this job on jenkins. - # issue_comment: - # types: [created] # schedule: # - cron: '17 8/20 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_TextIOIT.yml b/.github/workflows/beam_PerformanceTests_TextIOIT.yml index 9daa2b29dd2a..c313731206be 100644 --- a/.github/workflows/beam_PerformanceTests_TextIOIT.yml +++ b/.github/workflows/beam_PerformanceTests_TextIOIT.yml @@ -16,8 +16,6 @@ name: PerformanceTests TextIOIT on: - issue_comment: - types: [created] schedule: - cron: '0 7,19 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml b/.github/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml index a98379b281a9..118605fe1026 100644 --- a/.github/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml +++ b/.github/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml @@ -16,8 +16,6 @@ name: PerformanceTests TextIOIT HDFS on: - issue_comment: - types: [created] schedule: - cron: '30 7,19 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_TextIOIT_Python.yml b/.github/workflows/beam_PerformanceTests_TextIOIT_Python.yml index cb2b7fb34a9f..deb589b18c51 100644 --- a/.github/workflows/beam_PerformanceTests_TextIOIT_Python.yml +++ b/.github/workflows/beam_PerformanceTests_TextIOIT_Python.yml @@ -16,8 +16,6 @@ name: PerformanceTests TextIOIT Python on: - issue_comment: - types: [created] schedule: - cron: '0 8,20 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml b/.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml index 53b157d691c5..82069f943329 100644 --- a/.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml +++ b/.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml @@ -16,8 +16,6 @@ name: PerformanceTests WordCountIT PythonVersions on: - issue_comment: - types: [created] schedule: - cron: '12 3 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_XmlIOIT.yml b/.github/workflows/beam_PerformanceTests_XmlIOIT.yml index cd0245f269d1..5b2a64f1a919 100644 --- a/.github/workflows/beam_PerformanceTests_XmlIOIT.yml +++ b/.github/workflows/beam_PerformanceTests_XmlIOIT.yml @@ -16,8 +16,6 @@ name: PerformanceTests XmlIOIT on: - issue_comment: - types: [created] schedule: - cron: '30 4/12 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml b/.github/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml index a89f0b5dcb0f..3358085a0120 100644 --- a/.github/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml +++ b/.github/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml @@ -16,8 +16,6 @@ name: PerformanceTests XmlIOIT HDFS on: - issue_comment: - types: [created] schedule: - cron: '50 4/12 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml b/.github/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml index a29454ced4bf..ee84de4ac618 100644 --- a/.github/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml +++ b/.github/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml @@ -16,8 +16,6 @@ name: PerformanceTests xlang KafkaIO Python on: - issue_comment: - types: [created] schedule: - cron: '10 5 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_Publish_Docker_Snapshots.yml b/.github/workflows/beam_Publish_Docker_Snapshots.yml index 41262a58b1e7..75ff5df3de47 100644 --- a/.github/workflows/beam_Publish_Docker_Snapshots.yml +++ b/.github/workflows/beam_Publish_Docker_Snapshots.yml @@ -16,8 +16,6 @@ name: Publish Docker Snapshots on: - issue_comment: - types: [created] schedule: - cron: '0 13 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml b/.github/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml index 9a4ff4144ac5..ce1d824df6e0 100644 --- a/.github/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml +++ b/.github/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml @@ -16,8 +16,6 @@ name: Python ValidatesContainer Dataflow ARM on: - issue_comment: - types: [created] push: tags: ['v*'] branches: ['master', 'release-*'] From a7abe61288cd39ad8fc4baf74e782ef7cefe0b69 Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Tue, 10 Oct 2023 11:48:55 -0400 Subject: [PATCH 088/435] Remove issue_comment trigger in most postcommit (#28916) --- .github/workflows/beam_PostCommit_BeamMetrics_Publish.yml | 2 -- .github/workflows/beam_PostCommit_Go.yml | 2 -- .github/workflows/beam_PostCommit_Go_VR_Flink.yml | 2 -- .github/workflows/beam_PostCommit_Go_VR_Samza.yml | 2 -- .github/workflows/beam_PostCommit_Go_VR_Spark.yml | 2 -- .github/workflows/beam_PostCommit_Java.yml | 2 -- .github/workflows/beam_PostCommit_Java_Avro_Versions.yml | 2 -- .github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml | 2 -- .github/workflows/beam_PostCommit_Java_DataflowV1.yml | 2 -- .github/workflows/beam_PostCommit_Java_DataflowV2.yml | 2 -- .../workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml | 2 -- .github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml | 2 -- .../beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml | 2 -- .github/workflows/beam_PostCommit_Java_Examples_Direct.yml | 2 -- .github/workflows/beam_PostCommit_Java_Examples_Flink.yml | 2 -- .github/workflows/beam_PostCommit_Java_Examples_Spark.yml | 2 -- .github/workflows/beam_PostCommit_Java_Hadoop_Versions.yml | 2 -- .github/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml | 2 -- .github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml | 2 -- .github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml | 2 -- .github/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml | 2 -- .github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml | 2 -- .github/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml | 2 -- .github/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml | 2 -- .github/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml | 2 -- .github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml | 2 -- .../workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml | 2 -- .github/workflows/beam_PostCommit_Java_Nexmark_Direct.yml | 2 -- .github/workflows/beam_PostCommit_Java_Nexmark_Flink.yml | 2 -- .github/workflows/beam_PostCommit_Java_Nexmark_Spark.yml | 2 -- .github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml | 2 -- .github/workflows/beam_PostCommit_Java_PVR_Samza.yml | 2 -- .github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml | 2 -- .github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml | 2 -- .github/workflows/beam_PostCommit_Java_Sickbay.yml | 2 -- .github/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml | 2 -- .github/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml | 2 -- .github/workflows/beam_PostCommit_Java_Tpcds_Flink.yml | 2 -- .github/workflows/beam_PostCommit_Java_Tpcds_Spark.yml | 2 -- .../workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml | 2 -- ...am_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml | 2 -- .../beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml | 2 -- .../beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml | 2 -- ...am_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml | 2 -- .../workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml | 2 -- ...beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml | 2 -- .../workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml | 2 -- .../beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml | 2 -- .../workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml | 2 -- .../workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml | 2 -- ...PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml | 2 -- .../beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml | 2 -- .../workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml | 2 -- .github/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml | 2 -- .github/workflows/beam_PostCommit_Javadoc.yml | 2 -- .github/workflows/beam_PostCommit_PortableJar_Flink.yml | 2 -- .github/workflows/beam_PostCommit_PortableJar_Spark.yml | 2 -- .github/workflows/beam_PostCommit_Python.yml | 2 -- .github/workflows/beam_PostCommit_Python_Examples_Dataflow.yml | 2 -- .github/workflows/beam_PostCommit_Python_Examples_Direct.yml | 2 -- .github/workflows/beam_PostCommit_Python_Examples_Flink.yml | 2 -- .github/workflows/beam_PostCommit_Python_Examples_Spark.yml | 2 -- .github/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml | 2 -- .github/workflows/beam_PostCommit_Python_Nexmark_Direct.yml | 2 -- .../beam_PostCommit_Python_ValidatesContainer_Dataflow.yml | 2 -- ...am_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml | 2 -- .../beam_PostCommit_Python_ValidatesRunner_Dataflow.yml | 2 -- .../workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml | 2 -- .../workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml | 2 -- .../workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml | 2 -- .github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml | 2 -- .github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml | 2 -- .github/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml | 2 -- .github/workflows/beam_PostCommit_SQL.yml | 2 -- .github/workflows/beam_PostCommit_Sickbay_Python.yml | 2 -- .github/workflows/beam_PostCommit_TransformService_Direct.yml | 2 -- .github/workflows/beam_PostCommit_Website_Test.yml | 2 -- .github/workflows/beam_PostCommit_XVR_Direct.yml | 2 -- .github/workflows/beam_PostCommit_XVR_Flink.yml | 2 -- .github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml | 2 -- .../workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml | 2 -- .../beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml | 2 -- .../workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml | 2 -- .github/workflows/beam_PostCommit_XVR_Samza.yml | 2 -- .github/workflows/beam_PostCommit_XVR_Spark3.yml | 2 -- 85 files changed, 170 deletions(-) diff --git a/.github/workflows/beam_PostCommit_BeamMetrics_Publish.yml b/.github/workflows/beam_PostCommit_BeamMetrics_Publish.yml index 0ea3207b505a..bccb2c8e78c6 100644 --- a/.github/workflows/beam_PostCommit_BeamMetrics_Publish.yml +++ b/.github/workflows/beam_PostCommit_BeamMetrics_Publish.yml @@ -20,8 +20,6 @@ on: tags: ['v*'] branches: ['master', 'release-*'] paths: ['.github/workflows/beam_PostCommit_BeamMetrics_Publish.yml'] - issue_comment: - types: [created] schedule: - cron: '24 2 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Go.yml b/.github/workflows/beam_PostCommit_Go.yml index dc1180314d67..5fff2d334a5a 100644 --- a/.github/workflows/beam_PostCommit_Go.yml +++ b/.github/workflows/beam_PostCommit_Go.yml @@ -16,8 +16,6 @@ name: PostCommit Go on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Go_VR_Flink.yml b/.github/workflows/beam_PostCommit_Go_VR_Flink.yml index 21dcf7f8e72a..3923e9d74f94 100644 --- a/.github/workflows/beam_PostCommit_Go_VR_Flink.yml +++ b/.github/workflows/beam_PostCommit_Go_VR_Flink.yml @@ -16,8 +16,6 @@ name: PostCommit Go VR Flink on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Go_VR_Samza.yml b/.github/workflows/beam_PostCommit_Go_VR_Samza.yml index 90e107ee0c17..4e0663be0dbf 100644 --- a/.github/workflows/beam_PostCommit_Go_VR_Samza.yml +++ b/.github/workflows/beam_PostCommit_Go_VR_Samza.yml @@ -16,8 +16,6 @@ name: PostCommit Go VR Samza on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Go_VR_Spark.yml b/.github/workflows/beam_PostCommit_Go_VR_Spark.yml index 07cd627059b2..85262335c9a5 100644 --- a/.github/workflows/beam_PostCommit_Go_VR_Spark.yml +++ b/.github/workflows/beam_PostCommit_Go_VR_Spark.yml @@ -16,8 +16,6 @@ name: PostCommit Go VR Spark on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java.yml b/.github/workflows/beam_PostCommit_Java.yml index 3eed85bc2026..0943e2dec263 100644 --- a/.github/workflows/beam_PostCommit_Java.yml +++ b/.github/workflows/beam_PostCommit_Java.yml @@ -18,8 +18,6 @@ name: PostCommit Java on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_Avro_Versions.yml b/.github/workflows/beam_PostCommit_Java_Avro_Versions.yml index 1bd828d08ee0..63ffda3864e6 100644 --- a/.github/workflows/beam_PostCommit_Java_Avro_Versions.yml +++ b/.github/workflows/beam_PostCommit_Java_Avro_Versions.yml @@ -18,8 +18,6 @@ name: PostCommit Java Avro Versions on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml b/.github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml index 952273e810d2..f74b429988d7 100644 --- a/.github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml +++ b/.github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml @@ -18,8 +18,6 @@ name: PostCommit Java BigQueryEarlyRollout on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_DataflowV1.yml b/.github/workflows/beam_PostCommit_Java_DataflowV1.yml index 55e4f99afc74..b6cf668ddfd3 100644 --- a/.github/workflows/beam_PostCommit_Java_DataflowV1.yml +++ b/.github/workflows/beam_PostCommit_Java_DataflowV1.yml @@ -18,8 +18,6 @@ name: PostCommit Java Dataflow V1 on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_DataflowV2.yml b/.github/workflows/beam_PostCommit_Java_DataflowV2.yml index 7ca2f57ce78d..f99c08c36f8d 100644 --- a/.github/workflows/beam_PostCommit_Java_DataflowV2.yml +++ b/.github/workflows/beam_PostCommit_Java_DataflowV2.yml @@ -18,8 +18,6 @@ name: PostCommit Java Dataflow V2 on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml index a998e0242254..3d2523654fb6 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml @@ -18,8 +18,6 @@ name: PostCommit Java Examples Dataflow Java on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml index 0c87a81c5bed..67780ae4a848 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml @@ -18,8 +18,6 @@ name: PostCommit Java Examples Dataflow V2 on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml index 2526f9b56531..74e1787945e3 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml @@ -18,8 +18,6 @@ name: PostCommit Java Examples Dataflow V2 Java on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Direct.yml b/.github/workflows/beam_PostCommit_Java_Examples_Direct.yml index 6fd1150aecd1..9508dfe22617 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Direct.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Direct.yml @@ -18,8 +18,6 @@ name: PostCommit Java Examples Direct on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Flink.yml b/.github/workflows/beam_PostCommit_Java_Examples_Flink.yml index b123134cd239..c359ae2f9e53 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Flink.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Flink.yml @@ -18,8 +18,6 @@ name: PostCommit Java Examples Flink on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Spark.yml b/.github/workflows/beam_PostCommit_Java_Examples_Spark.yml index fa28cdf402bf..78130d9ec247 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Spark.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Spark.yml @@ -18,8 +18,6 @@ name: PostCommit Java Examples Spark on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_Hadoop_Versions.yml b/.github/workflows/beam_PostCommit_Java_Hadoop_Versions.yml index f9c7175ccd21..189fbb083a39 100644 --- a/.github/workflows/beam_PostCommit_Java_Hadoop_Versions.yml +++ b/.github/workflows/beam_PostCommit_Java_Hadoop_Versions.yml @@ -16,8 +16,6 @@ name: PostCommit Java Hadoop Versions on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml b/.github/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml index f048fdc6f1f7..7f1d76a13936 100644 --- a/.github/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml +++ b/.github/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml @@ -18,8 +18,6 @@ name: Java InfluxDbIO Integration Test on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml index 3ee009747a88..d5b23c095d0b 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml @@ -16,8 +16,6 @@ name: PostCommit Java Jpms Dataflow Java11 on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml index 398be301eecf..0b9a366211ee 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml @@ -16,8 +16,6 @@ name: PostCommit Java Jpms Dataflow Java17 on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml index 44b89acb642e..97f03ed59211 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml @@ -16,8 +16,6 @@ name: PostCommit Java Jpms Direct Java11 on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml index dbeb84ab660b..fcdbd63f7e97 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml @@ -16,8 +16,6 @@ name: PostCommit Java Jpms Direct Java17 on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml index f4ba8ada32ba..832e211a4d61 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml @@ -16,8 +16,6 @@ name: PostCommit Java Jpms Flink Java11 on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml index 9758fda66eb7..14b5e5380921 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml @@ -16,8 +16,6 @@ name: PostCommit Java Jpms Spark Java11 on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml index 74b786b599e1..44bdb4e51ca8 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml @@ -18,8 +18,6 @@ name: PostCommit Java Nexmark Dataflow on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml index 3b7836990b69..50a72af7909a 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml @@ -18,8 +18,6 @@ name: PostCommit Java Nexmark Dataflow V2 on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml index a0957f643279..d78793172898 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml @@ -18,8 +18,6 @@ name: PostCommit Java Nexmark Dataflow V2 Java on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Direct.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Direct.yml index 4daa13da8b13..e7c6e4655671 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Direct.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Direct.yml @@ -18,8 +18,6 @@ name: PostCommit Java Nexmark Direct on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Flink.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Flink.yml index a03c447416f3..7dc59a1a2e72 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Flink.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Flink.yml @@ -18,8 +18,6 @@ name: PostCommit Java Nexmark Flink on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Spark.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Spark.yml index a43e7e6d311f..fd1f9745d80e 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Spark.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Spark.yml @@ -18,8 +18,6 @@ name: PostCommit Java Nexmark Spark on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml b/.github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml index 991e4f71b1c4..efab6427642a 100644 --- a/.github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml +++ b/.github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml @@ -18,8 +18,6 @@ name: PostCommit Java PVR Flink Streaming on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_PVR_Samza.yml b/.github/workflows/beam_PostCommit_Java_PVR_Samza.yml index 041c031f3f76..942028ea3569 100644 --- a/.github/workflows/beam_PostCommit_Java_PVR_Samza.yml +++ b/.github/workflows/beam_PostCommit_Java_PVR_Samza.yml @@ -18,8 +18,6 @@ name: PostCommit Java PVR Samza on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml b/.github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml index 2a375d71981a..6b47a64572ea 100644 --- a/.github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml +++ b/.github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml @@ -18,8 +18,6 @@ name: PostCommit Java PVR Spark3 Streaming on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml b/.github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml index 8d54c7707258..82b132522c12 100644 --- a/.github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml +++ b/.github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml @@ -18,8 +18,6 @@ name: PostCommit Java PVR Spark Batch on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_Sickbay.yml b/.github/workflows/beam_PostCommit_Java_Sickbay.yml index 7f914dc35880..5d2da81e2ab0 100644 --- a/.github/workflows/beam_PostCommit_Java_Sickbay.yml +++ b/.github/workflows/beam_PostCommit_Java_Sickbay.yml @@ -18,8 +18,6 @@ name: PostCommit Java Sickbay on: - issue_comment: - types: [created] schedule: - cron: '0 0 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml b/.github/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml index 40a1dc8faa64..ac8b5cacf40b 100644 --- a/.github/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml +++ b/.github/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml @@ -18,8 +18,6 @@ name: PostCommit Java SingleStoreIO IT on: - issue_comment: - types: [created] schedule: - cron: '0 */23 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml b/.github/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml index e19831c60732..3239281fb938 100644 --- a/.github/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml @@ -16,8 +16,6 @@ name: PostCommit Java Tpcds Dataflow on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_Tpcds_Flink.yml b/.github/workflows/beam_PostCommit_Java_Tpcds_Flink.yml index fd35fe4ec776..c6dcdcfaa3a4 100644 --- a/.github/workflows/beam_PostCommit_Java_Tpcds_Flink.yml +++ b/.github/workflows/beam_PostCommit_Java_Tpcds_Flink.yml @@ -16,8 +16,6 @@ name: PostCommit Java Tpcds Flink on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_Tpcds_Spark.yml b/.github/workflows/beam_PostCommit_Java_Tpcds_Spark.yml index 2284e7fa06bf..73fa4e5ba143 100644 --- a/.github/workflows/beam_PostCommit_Java_Tpcds_Spark.yml +++ b/.github/workflows/beam_PostCommit_Java_Tpcds_Spark.yml @@ -16,8 +16,6 @@ name: PostCommit Java Tpcds Spark on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml index 596ef873c964..0a0095e941ae 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml @@ -18,8 +18,6 @@ name: PostCommit Java ValidatesRunner Dataflow on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml index 19c2f3f8cb16..e7d9bf9ccb6b 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml @@ -18,8 +18,6 @@ name: PostCommit Java ValidatesRunner Dataflow JavaVersions on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml index 013b34bc807b..094133af2bb9 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml @@ -18,8 +18,6 @@ name: PostCommit Java ValidatesRunner Dataflow Streaming on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml index 3a9e0140f818..4331772d32d4 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml @@ -18,8 +18,6 @@ name: PostCommit Java ValidatesRunner Dataflow V2 on: - issue_comment: - types: [created] schedule: - cron: '0 */8 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml index edf9a04f22c3..e0f90156f370 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml @@ -18,8 +18,6 @@ name: PostCommit Java ValidatesRunner Dataflow V2 Streaming on: - issue_comment: - types: [created] schedule: - cron: '0 */8 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml index 79447f610809..1020b3869b97 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml @@ -18,8 +18,6 @@ name: PostCommit Java ValidatesRunner Direct on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml index 8e5dbac0c4cd..12bd26d070ed 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml @@ -18,8 +18,6 @@ name: PostCommit Java ValidatesRunner Direct JavaVersions on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml index 2ff883dafa75..5de57835a190 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml @@ -16,8 +16,6 @@ name: PostCommit Java ValidatesRunner Flink on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml index c0339100845e..aa5c24ac2b21 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml @@ -18,8 +18,6 @@ name: PostCommit Java ValidatesRunner Flink Java11 on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml index 97e35490c25d..62e003a1660d 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml @@ -16,8 +16,6 @@ name: PostCommit Java ValidatesRunner Samza on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml index 5800c338ed16..083e473a3a56 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml @@ -16,8 +16,6 @@ name: PostCommit Java ValidatesRunner Spark on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml index 08504316333a..74bb0b15b8de 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml @@ -16,8 +16,6 @@ name: PostCommit Java ValidatesRunner SparkStructuredStreaming on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml index 1cc0193b1e69..8082f2143b38 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml @@ -18,8 +18,6 @@ name: PostCommit Java ValidatesRunner Spark Java11 on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml index 21bacac11d59..530c360ea346 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml @@ -16,8 +16,6 @@ name: PostCommit Java ValidatesRunner Twister2 on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml index 75c07bc49783..61e0affa5e81 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml @@ -16,8 +16,6 @@ name: PostCommit Java ValidatesRunner ULR on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Javadoc.yml b/.github/workflows/beam_PostCommit_Javadoc.yml index 7185f588f463..8413f39b3b9d 100644 --- a/.github/workflows/beam_PostCommit_Javadoc.yml +++ b/.github/workflows/beam_PostCommit_Javadoc.yml @@ -18,8 +18,6 @@ name: PostCommit Javadoc on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_PortableJar_Flink.yml b/.github/workflows/beam_PostCommit_PortableJar_Flink.yml index 3f682ea57dde..18941e70a881 100644 --- a/.github/workflows/beam_PostCommit_PortableJar_Flink.yml +++ b/.github/workflows/beam_PostCommit_PortableJar_Flink.yml @@ -16,8 +16,6 @@ name: PostCommit PortableJar Flink on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_PortableJar_Spark.yml b/.github/workflows/beam_PostCommit_PortableJar_Spark.yml index f9884d651978..55cdffcb86a1 100644 --- a/.github/workflows/beam_PostCommit_PortableJar_Spark.yml +++ b/.github/workflows/beam_PostCommit_PortableJar_Spark.yml @@ -16,8 +16,6 @@ name: PostCommit PortableJar Spark on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Python.yml b/.github/workflows/beam_PostCommit_Python.yml index 3fb650606b40..97299644e5dc 100644 --- a/.github/workflows/beam_PostCommit_Python.yml +++ b/.github/workflows/beam_PostCommit_Python.yml @@ -18,8 +18,6 @@ name: PostCommit Python on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Python_Examples_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_Examples_Dataflow.yml index 40c508b38e6e..81687e498b8b 100644 --- a/.github/workflows/beam_PostCommit_Python_Examples_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_Examples_Dataflow.yml @@ -16,8 +16,6 @@ name: PostCommit Python Examples Dataflow on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Python_Examples_Direct.yml b/.github/workflows/beam_PostCommit_Python_Examples_Direct.yml index 85d766b0575c..903101e5a01c 100644 --- a/.github/workflows/beam_PostCommit_Python_Examples_Direct.yml +++ b/.github/workflows/beam_PostCommit_Python_Examples_Direct.yml @@ -16,8 +16,6 @@ name: PostCommit Python Examples Direct on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Python_Examples_Flink.yml b/.github/workflows/beam_PostCommit_Python_Examples_Flink.yml index 17112cf18e2a..aa1404a4aa64 100644 --- a/.github/workflows/beam_PostCommit_Python_Examples_Flink.yml +++ b/.github/workflows/beam_PostCommit_Python_Examples_Flink.yml @@ -16,8 +16,6 @@ name: PostCommit Python Examples Flink on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Python_Examples_Spark.yml b/.github/workflows/beam_PostCommit_Python_Examples_Spark.yml index ccc7a998df27..7716ca3e67b1 100644 --- a/.github/workflows/beam_PostCommit_Python_Examples_Spark.yml +++ b/.github/workflows/beam_PostCommit_Python_Examples_Spark.yml @@ -16,8 +16,6 @@ name: PostCommit Python Examples Spark on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml b/.github/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml index 0fb2302b8ed1..7c8b7d2526ab 100644 --- a/.github/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml +++ b/.github/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml @@ -16,8 +16,6 @@ name: PostCommit Python MongoDBIO IT on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Python_Nexmark_Direct.yml b/.github/workflows/beam_PostCommit_Python_Nexmark_Direct.yml index db95a48a3007..bf96c1cf74be 100644 --- a/.github/workflows/beam_PostCommit_Python_Nexmark_Direct.yml +++ b/.github/workflows/beam_PostCommit_Python_Nexmark_Direct.yml @@ -18,8 +18,6 @@ name: PostCommit Python Nexmark Direct on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml index fb7102a5b52e..713f7fa94e20 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml @@ -16,8 +16,6 @@ name: PostCommit Python ValidatesContainer Dataflow on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml b/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml index a95682e415d4..18b57aad8057 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml @@ -16,8 +16,6 @@ name: PostCommit Python ValidatesContainer Dataflow With RC on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml index 572f477773b6..6ff8edb9cce8 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml @@ -16,8 +16,6 @@ name: PostCommit Python ValidatesRunner Dataflow on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml index a1652f3a18de..de2ed09b9614 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml @@ -16,8 +16,6 @@ name: PostCommit Python ValidatesRunner Flink on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml index c41605391d40..b86fcfac218b 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml @@ -16,8 +16,6 @@ name: PostCommit Python ValidatesRunner Samza on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml index d42835727d36..65fa97a5b5ea 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml @@ -16,8 +16,6 @@ name: PostCommit Python ValidatesRunner Spark on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml index 31025df81ddc..6c2255f60f1d 100644 --- a/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml @@ -16,8 +16,6 @@ name: PostCommit Python Xlang Gcp Dataflow on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml b/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml index d17d15029c2c..36d100473f98 100644 --- a/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml +++ b/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml @@ -16,8 +16,6 @@ name: PostCommit Python Xlang Gcp Direct on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml index 386d91518795..eb9665d39d7f 100644 --- a/.github/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml @@ -16,8 +16,6 @@ name: PostCommit Python Xlang IO Dataflow on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_SQL.yml b/.github/workflows/beam_PostCommit_SQL.yml index a753cde6e56d..3cd72fcecc42 100644 --- a/.github/workflows/beam_PostCommit_SQL.yml +++ b/.github/workflows/beam_PostCommit_SQL.yml @@ -18,8 +18,6 @@ name: PostCommit SQL on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Sickbay_Python.yml b/.github/workflows/beam_PostCommit_Sickbay_Python.yml index 2153e9908898..7e15ac7e8873 100644 --- a/.github/workflows/beam_PostCommit_Sickbay_Python.yml +++ b/.github/workflows/beam_PostCommit_Sickbay_Python.yml @@ -18,8 +18,6 @@ name: PostCommit Sickbay Python on: - issue_comment: - types: [created] schedule: - cron: '0 0 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_TransformService_Direct.yml b/.github/workflows/beam_PostCommit_TransformService_Direct.yml index 9387e9f74831..491688e097a0 100644 --- a/.github/workflows/beam_PostCommit_TransformService_Direct.yml +++ b/.github/workflows/beam_PostCommit_TransformService_Direct.yml @@ -16,8 +16,6 @@ name: PostCommit TransformService Direct on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Website_Test.yml b/.github/workflows/beam_PostCommit_Website_Test.yml index cd06181c456d..bb12d5804b6f 100644 --- a/.github/workflows/beam_PostCommit_Website_Test.yml +++ b/.github/workflows/beam_PostCommit_Website_Test.yml @@ -16,8 +16,6 @@ name: PostCommit Website Test on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_XVR_Direct.yml b/.github/workflows/beam_PostCommit_XVR_Direct.yml index eaee2b669873..8db745b8b217 100644 --- a/.github/workflows/beam_PostCommit_XVR_Direct.yml +++ b/.github/workflows/beam_PostCommit_XVR_Direct.yml @@ -16,8 +16,6 @@ name: PostCommit XVR Direct on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_XVR_Flink.yml b/.github/workflows/beam_PostCommit_XVR_Flink.yml index 67effa70b8b5..640555a03ec1 100644 --- a/.github/workflows/beam_PostCommit_XVR_Flink.yml +++ b/.github/workflows/beam_PostCommit_XVR_Flink.yml @@ -16,8 +16,6 @@ name: PostCommit XVR Flink on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml b/.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml index 2f8a2eb8a3da..be10e5fb1bb3 100644 --- a/.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml @@ -16,8 +16,6 @@ name: PostCommit XVR GoUsingJava Dataflow on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml b/.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml index 386fa304606d..4148d0a2d468 100644 --- a/.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml @@ -16,8 +16,6 @@ name: PostCommit XVR JavaUsingPython Dataflow on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml b/.github/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml index 2432ddc30e2a..6c8e01124d0b 100644 --- a/.github/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml @@ -16,8 +16,6 @@ name: PostCommit XVR PythonUsingJavaSQL Dataflow on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml b/.github/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml index bebac78a6315..486fe84ff45c 100644 --- a/.github/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml @@ -16,8 +16,6 @@ name: PostCommit XVR PythonUsingJava Dataflow on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_XVR_Samza.yml b/.github/workflows/beam_PostCommit_XVR_Samza.yml index 8cf16cfe225d..8a6c9bda6539 100644 --- a/.github/workflows/beam_PostCommit_XVR_Samza.yml +++ b/.github/workflows/beam_PostCommit_XVR_Samza.yml @@ -16,8 +16,6 @@ name: PostCommit XVR Samza on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_XVR_Spark3.yml b/.github/workflows/beam_PostCommit_XVR_Spark3.yml index 193526ffe7f3..b61cfe3251f0 100644 --- a/.github/workflows/beam_PostCommit_XVR_Spark3.yml +++ b/.github/workflows/beam_PostCommit_XVR_Spark3.yml @@ -16,8 +16,6 @@ name: PostCommit XVR Spark3 on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: From 2bccee14cf9261f95db5a433a404454bb28b0a9c Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 10 Oct 2023 13:14:11 -0400 Subject: [PATCH 089/435] Bump google.golang.org/api from 0.144.0 to 0.146.0 in /sdks (#28911) Bumps [google.golang.org/api](https://github.com/googleapis/google-api-go-client) from 0.144.0 to 0.146.0. - [Release notes](https://github.com/googleapis/google-api-go-client/releases) - [Changelog](https://github.com/googleapis/google-api-go-client/blob/main/CHANGES.md) - [Commits](https://github.com/googleapis/google-api-go-client/compare/v0.144.0...v0.146.0) --- updated-dependencies: - dependency-name: google.golang.org/api dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 2 +- sdks/go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 53eae7db4539..0bc60b630bf0 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -57,7 +57,7 @@ require ( golang.org/x/sync v0.4.0 golang.org/x/sys v0.13.0 golang.org/x/text v0.13.0 - google.golang.org/api v0.144.0 + google.golang.org/api v0.146.0 google.golang.org/genproto v0.0.0-20230913181813-007df8e322eb google.golang.org/grpc v1.58.2 google.golang.org/protobuf v1.31.0 diff --git a/sdks/go.sum b/sdks/go.sum index 5dc085b859a0..7743b2bc3247 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -670,8 +670,8 @@ google.golang.org/api v0.14.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsb google.golang.org/api v0.15.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI= google.golang.org/api v0.17.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= google.golang.org/api v0.18.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= -google.golang.org/api v0.144.0 h1:01xgplvIwdMpnrlenPHMgRAAgAH9N5Zv21Qu6XwJxSU= -google.golang.org/api v0.144.0/go.mod h1:OARJqIfoYjXJj4C1AiBSXYZt03qsoz8FQYU6fBEfrHM= +google.golang.org/api v0.146.0 h1:9aBYT4vQXt9dhCuLNfwfd3zpwu8atg0yPkjBymwSrOM= +google.golang.org/api v0.146.0/go.mod h1:OARJqIfoYjXJj4C1AiBSXYZt03qsoz8FQYU6fBEfrHM= google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/appengine v1.5.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= From a859863d3926db18dc0f2112dd0b7e9a5105923f Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud <65791736+ahmedabu98@users.noreply.github.com> Date: Tue, 10 Oct 2023 15:54:02 -0400 Subject: [PATCH 090/435] update dataflow containers (#28904) --- sdks/python/apache_beam/runners/dataflow/internal/names.py | 2 +- .../www/site/content/en/documentation/runtime/environments.md | 3 +-- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/sdks/python/apache_beam/runners/dataflow/internal/names.py b/sdks/python/apache_beam/runners/dataflow/internal/names.py index 8347a597e968..579764aeb7c1 100644 --- a/sdks/python/apache_beam/runners/dataflow/internal/names.py +++ b/sdks/python/apache_beam/runners/dataflow/internal/names.py @@ -34,6 +34,6 @@ # Unreleased sdks use container image tag specified below. # Update this tag whenever there is a change that # requires changes to SDK harness container or SDK harness launcher. -BEAM_DEV_SDK_CONTAINER_TAG = 'beam-master-20230927' +BEAM_DEV_SDK_CONTAINER_TAG = 'beam-master-20231009' DATAFLOW_CONTAINER_IMAGE_REPOSITORY = 'gcr.io/cloud-dataflow/v1beta3' diff --git a/website/www/site/content/en/documentation/runtime/environments.md b/website/www/site/content/en/documentation/runtime/environments.md index 624b5aa6b881..452fb6141e63 100644 --- a/website/www/site/content/en/documentation/runtime/environments.md +++ b/website/www/site/content/en/documentation/runtime/environments.md @@ -115,14 +115,13 @@ This method requires building image artifacts from Beam source. For additional i ./gradlew :sdks:java:container:java11:docker ./gradlew :sdks:java:container:java17:docker ./gradlew :sdks:go:container:docker - ./gradlew :sdks:python:container:py36:docker ./gradlew :sdks:python:container:py38:docker ./gradlew :sdks:python:container:py39:docker ./gradlew :sdks:python:container:py310:docker ./gradlew :sdks:python:container:py311:docker # Shortcut for building all Python SDKs - ./gradlew :sdks:python:container buildAll + ./gradlew :sdks:python:container:buildAll ``` 4. Verify the images you built were created by running `docker images`. From d2420ad6ee5120e13fee9bd4061e9906f858e2dd Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Tue, 10 Oct 2023 16:04:10 -0400 Subject: [PATCH 091/435] Fix buildDirectory (#28925) --- runners/flink/flink_runner.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/runners/flink/flink_runner.gradle b/runners/flink/flink_runner.gradle index c087575f8023..b1a459337e51 100644 --- a/runners/flink/flink_runner.gradle +++ b/runners/flink/flink_runner.gradle @@ -46,7 +46,7 @@ evaluationDependsOn(":examples:java") /* * Copy & merge source overrides into build directory. */ -def sourceOverridesBase = project.layout.buildDirectory.dir('source-overrides/src') +def sourceOverridesBase = project.layout.buildDirectory.dir('source-overrides/src').get() def copySourceOverrides = tasks.register('copySourceOverrides', Copy) { it.from main_source_overrides From 65eaf45026e9eeb61a9e05412488e5858faec6de Mon Sep 17 00:00:00 2001 From: Ritesh Ghorse Date: Tue, 10 Oct 2023 16:15:11 -0400 Subject: [PATCH 092/435] fix lint (#28917) --- sdks/python/apache_beam/runners/worker/bundle_processor.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/runners/worker/bundle_processor.py b/sdks/python/apache_beam/runners/worker/bundle_processor.py index c7fcb9587455..7ff0ad258bc2 100644 --- a/sdks/python/apache_beam/runners/worker/bundle_processor.py +++ b/sdks/python/apache_beam/runners/worker/bundle_processor.py @@ -233,7 +233,7 @@ def process_encoded(self, encoded_windowed_values): except Exception as exn: raise ValueError( "Error decoding input stream with coder " + - self.windowed_coder) from exn + str(self.windowed_coder)) from exn self.output(decoded_value) def monitoring_infos(self, transform_id, tag_to_pcollection_id): From c62a81b1c57e771875aaf4415aca76cf9e4c2783 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Tue, 10 Oct 2023 15:51:13 -0700 Subject: [PATCH 093/435] Reapply "Populate top-level display data in yaml main. (#28512)" This reverts commit 3844972d681a317c1281685edc5f6fc8783c8043. --- sdks/python/apache_beam/pipeline.py | 28 +++++++++++++++---- sdks/python/apache_beam/transforms/display.py | 23 +++++++++------ sdks/python/apache_beam/yaml/main.py | 17 ++++++----- 3 files changed, 47 insertions(+), 21 deletions(-) diff --git a/sdks/python/apache_beam/pipeline.py b/sdks/python/apache_beam/pipeline.py index 042b483d50f1..14177cd603d8 100644 --- a/sdks/python/apache_beam/pipeline.py +++ b/sdks/python/apache_beam/pipeline.py @@ -88,6 +88,7 @@ from apache_beam.transforms import ParDo from apache_beam.transforms import ptransform from apache_beam.transforms.display import DisplayData +from apache_beam.transforms.display import HasDisplayData from apache_beam.transforms.resources import merge_resource_hints from apache_beam.transforms.resources import resource_hints_from_options from apache_beam.transforms.sideinputs import get_sideinput_index @@ -108,7 +109,7 @@ __all__ = ['Pipeline', 'PTransformOverride'] -class Pipeline(object): +class Pipeline(HasDisplayData): """A pipeline object that manages a DAG of :class:`~apache_beam.pvalue.PValue` s and their :class:`~apache_beam.transforms.ptransform.PTransform` s. @@ -133,9 +134,12 @@ def runner_implemented_transforms(cls): common_urns.primitives.IMPULSE.urn, ]) - def __init__(self, runner=None, options=None, argv=None): - # type: (Optional[Union[str, PipelineRunner]], Optional[PipelineOptions], Optional[List[str]]) -> None - + def __init__( + self, + runner: Optional[Union[str, PipelineRunner]] = None, + options: Optional[PipelineOptions] = None, + argv: Optional[List[str]] = None, + display_data: Optional[Dict[str, Any]] = None): """Initialize a pipeline object. Args: @@ -151,6 +155,8 @@ def __init__(self, runner=None, options=None, argv=None): to be used for building a :class:`~apache_beam.options.pipeline_options.PipelineOptions` object. This will only be used if argument **options** is :data:`None`. + display_data (Dict[str: Any]): a dictionary of static data associated + with this pipeline that can be displayed when it runs. Raises: ValueError: if either the runner or options argument is not @@ -233,6 +239,11 @@ def __init__(self, runner=None, options=None, argv=None): # Records whether this pipeline contains any external transforms. self.contains_external_transforms = False + self._display_data = display_data or {} + + def display_data(self): + # type: () -> Dict[str, Any] + return self._display_data @property # type: ignore[misc] # decorated property not supported def options(self): @@ -914,7 +925,8 @@ def visit_transform(self, transform_node): proto = beam_runner_api_pb2.Pipeline( root_transform_ids=[root_transform_id], components=context.to_runner_api(), - requirements=context.requirements()) + requirements=context.requirements(), + display_data=DisplayData('', self._display_data).to_proto()) proto.components.transforms[root_transform_id].unique_name = ( root_transform_id) self.merge_compatible_environments(proto) @@ -970,7 +982,11 @@ def from_runner_api( # type: (...) -> Pipeline """For internal use only; no backwards-compatibility guarantees.""" - p = Pipeline(runner=runner, options=options) + p = Pipeline( + runner=runner, + options=options, + display_data={str(ix): d + for ix, d in enumerate(proto.display_data)}) from apache_beam.runners import pipeline_context context = pipeline_context.PipelineContext( proto.components, requirements=proto.requirements) diff --git a/sdks/python/apache_beam/transforms/display.py b/sdks/python/apache_beam/transforms/display.py index b52a8fd5b6dd..0d1dd552413e 100644 --- a/sdks/python/apache_beam/transforms/display.py +++ b/sdks/python/apache_beam/transforms/display.py @@ -45,6 +45,7 @@ from datetime import timedelta from typing import TYPE_CHECKING from typing import List +from typing import Union from apache_beam.portability import common_urns from apache_beam.portability.api import beam_runner_api_pb2 @@ -101,7 +102,8 @@ def __init__( ): # type: (...) -> None self.namespace = namespace - self.items = [] # type: List[DisplayDataItem] + self.items = [ + ] # type: List[Union[DisplayDataItem, beam_runner_api_pb2.DisplayData]] self._populate_items(display_data_dict) def _populate_items(self, display_data_dict): @@ -112,26 +114,31 @@ def _populate_items(self, display_data_dict): subcomponent_display_data = DisplayData( element._get_display_data_namespace(), element.display_data()) self.items += subcomponent_display_data.items - continue - if isinstance(element, DisplayDataItem): + elif isinstance(element, DisplayDataItem): if element.should_drop(): continue element.key = key element.namespace = self.namespace self.items.append(element) - continue - # If it's not a HasDisplayData element, - # nor a dictionary, then it's a simple value - self.items.append( - DisplayDataItem(element, namespace=self.namespace, key=key)) + elif isinstance(element, beam_runner_api_pb2.DisplayData): + self.items.append(element) + + else: + # If it's not a HasDisplayData element, + # nor a dictionary, then it's a simple value + self.items.append( + DisplayDataItem(element, namespace=self.namespace, key=key)) def to_proto(self): # type: (...) -> List[beam_runner_api_pb2.DisplayData] """Returns a List of Beam proto representation of Display data.""" def create_payload(dd): + if isinstance(dd, beam_runner_api_pb2.DisplayData): + return dd + display_data_dict = None try: display_data_dict = dd.get_dict() diff --git a/sdks/python/apache_beam/yaml/main.py b/sdks/python/apache_beam/yaml/main.py index eb0695f337b4..e2ec8df9cfc3 100644 --- a/sdks/python/apache_beam/yaml/main.py +++ b/sdks/python/apache_beam/yaml/main.py @@ -51,19 +51,22 @@ def _pipeline_spec_from_args(known_args): raise ValueError( "Exactly one of pipeline_spec or pipeline_spec_file must be set.") - return yaml.load(pipeline_yaml, Loader=yaml_transform.SafeLineLoader) + return pipeline_yaml def run(argv=None): yaml_transform._LOGGER.setLevel('INFO') known_args, pipeline_args = _configure_parser(argv) - pipeline_spec = _pipeline_spec_from_args(known_args) + pipeline_yaml = _pipeline_spec_from_args(known_args) + pipeline_spec = yaml.load(pipeline_yaml, Loader=yaml_transform.SafeLineLoader) - with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions( - pipeline_args, - pickle_library='cloudpickle', - **yaml_transform.SafeLineLoader.strip_metadata(pipeline_spec.get( - 'options', {})))) as p: + with beam.Pipeline( # linebreak for better yapf formatting + options=beam.options.pipeline_options.PipelineOptions( + pipeline_args, + pickle_library='cloudpickle', + **yaml_transform.SafeLineLoader.strip_metadata(pipeline_spec.get( + 'options', {}))), + display_data={'yaml': pipeline_yaml}) as p: print("Building pipeline...") yaml_transform.expand_pipeline(p, pipeline_spec) print("Running pipeline...") From de48383bdca60f72d95c5889b281eed3be0fb5a6 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Tue, 10 Oct 2023 17:41:22 -0700 Subject: [PATCH 094/435] Fix typo in docstring. --- sdks/python/apache_beam/pipeline.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/pipeline.py b/sdks/python/apache_beam/pipeline.py index 14177cd603d8..f52616307e7b 100644 --- a/sdks/python/apache_beam/pipeline.py +++ b/sdks/python/apache_beam/pipeline.py @@ -155,7 +155,7 @@ def __init__( to be used for building a :class:`~apache_beam.options.pipeline_options.PipelineOptions` object. This will only be used if argument **options** is :data:`None`. - display_data (Dict[str: Any]): a dictionary of static data associated + display_data (Dict[str, Any]): a dictionary of static data associated with this pipeline that can be displayed when it runs. Raises: From 99c87a2c68b09020d9d5fa40da18d432b501c39a Mon Sep 17 00:00:00 2001 From: Chamikara Jayalath Date: Tue, 10 Oct 2023 21:09:58 -0700 Subject: [PATCH 095/435] Updates the Transform Service to accept Python extra packages through the Java API (#28783) * Updates the Transform Service to accept Python extra packages through the Java API * Addressing reviewer comments * Addressing reviewer comments --- build.gradle.kts | 2 + .../core/construction/TransformUpgrader.java | 2 +- .../python/PythonExternalTransform.java | 39 ++-- .../transform-service/docker-compose/.env | 8 + .../docker-compose/docker-compose.yml | 3 +- .../transform-service/launcher/build.gradle | 3 + .../launcher/TransformServiceLauncher.java | 135 +++++++++++-- .../TransformServiceLauncherTest.java | 185 ++++++++++++++++++ .../transformservice/ExpansionService.java | 114 ++++++++++- .../ExpansionServiceTest.java | 7 +- .../utils/transform_service_launcher.py | 14 ++ .../expansion-service-container/boot.go | 102 +++++++++- 12 files changed, 566 insertions(+), 48 deletions(-) create mode 100644 sdks/java/transform-service/launcher/src/test/java/org/apache/beam/sdk/transformservice/launcher/TransformServiceLauncherTest.java diff --git a/build.gradle.kts b/build.gradle.kts index fbea1a59b282..ea1b4e6784e3 100644 --- a/build.gradle.kts +++ b/build.gradle.kts @@ -310,6 +310,8 @@ tasks.register("javaPreCommit") { dependsOn(":sdks:java:testing:test-utils:build") dependsOn(":sdks:java:testing:tpcds:build") dependsOn(":sdks:java:testing:watermarks:build") + dependsOn(":sdks:java:transform-service:build") + dependsOn(":sdks:java:transform-service:launcher:build") dependsOn(":examples:java:preCommit") dependsOn(":examples:java:twitter:preCommit") diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformUpgrader.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformUpgrader.java index d657bb31b184..5e1609f27a39 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformUpgrader.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformUpgrader.java @@ -108,7 +108,7 @@ public RunnerApi.Pipeline upgradeTransformsViaTransformService( } else if (options.getTransformServiceBeamVersion() != null) { String projectName = UUID.randomUUID().toString(); int port = findAvailablePort(); - service = TransformServiceLauncher.forProject(projectName, port); + service = TransformServiceLauncher.forProject(projectName, port, null); service.setBeamVersion(options.getTransformServiceBeamVersion()); // Starting the transform service. diff --git a/sdks/java/extensions/python/src/main/java/org/apache/beam/sdk/extensions/python/PythonExternalTransform.java b/sdks/java/extensions/python/src/main/java/org/apache/beam/sdk/extensions/python/PythonExternalTransform.java index 4a5f4f12a07a..5ba3484964c1 100644 --- a/sdks/java/extensions/python/src/main/java/org/apache/beam/sdk/extensions/python/PythonExternalTransform.java +++ b/sdks/java/extensions/python/src/main/java/org/apache/beam/sdk/extensions/python/PythonExternalTransform.java @@ -495,6 +495,20 @@ public OutputT expand(InputT input) { boolean pythonAvailable = isPythonAvailable(); boolean dockerAvailable = isDockerAvailable(); + File requirementsFile = null; + if (!extraPackages.isEmpty()) { + requirementsFile = File.createTempFile("requirements", ".txt"); + requirementsFile.deleteOnExit(); + try (Writer fout = + new OutputStreamWriter( + new FileOutputStream(requirementsFile.getAbsolutePath()), Charsets.UTF_8)) { + for (String pkg : extraPackages) { + fout.write(pkg); + fout.write('\n'); + } + } + } + // We use the transform service if either of the following is true. // * It was explicitly requested. // * Python executable is not available in the system but Docker is available. @@ -514,19 +528,16 @@ public OutputT expand(InputT input) { projectName, port); - TransformServiceLauncher service = TransformServiceLauncher.forProject(projectName, port); + String pythonRequirementsFile = + requirementsFile != null ? requirementsFile.getAbsolutePath() : null; + TransformServiceLauncher service = + TransformServiceLauncher.forProject(projectName, port, pythonRequirementsFile); service.setBeamVersion(ReleaseInfo.getReleaseInfo().getSdkVersion()); - // TODO(https://github.com/apache/beam/issues/26833): add support for installing extra - // packages. - if (!extraPackages.isEmpty()) { - throw new RuntimeException( - "Transform Service does not support installing extra packages yet"); - } try { // Starting the transform service. service.start(); // Waiting the service to be ready. - service.waitTillUp(15000); + service.waitTillUp(-1); // Expanding the transform. output = apply(input, String.format("localhost:%s", port), payload); } finally { @@ -539,17 +550,7 @@ public OutputT expand(InputT input) { ImmutableList.Builder args = ImmutableList.builder(); args.add( "--port=" + port, "--fully_qualified_name_glob=*", "--pickle_library=cloudpickle"); - if (!extraPackages.isEmpty()) { - File requirementsFile = File.createTempFile("requirements", ".txt"); - requirementsFile.deleteOnExit(); - try (Writer fout = - new OutputStreamWriter( - new FileOutputStream(requirementsFile.getAbsolutePath()), Charsets.UTF_8)) { - for (String pkg : extraPackages) { - fout.write(pkg); - fout.write('\n'); - } - } + if (requirementsFile != null) { args.add("--requirements_file=" + requirementsFile.getAbsolutePath()); } PythonService service = diff --git a/sdks/java/transform-service/docker-compose/.env b/sdks/java/transform-service/docker-compose/.env index 5de5982cfa30..ed27b267fed3 100644 --- a/sdks/java/transform-service/docker-compose/.env +++ b/sdks/java/transform-service/docker-compose/.env @@ -12,6 +12,14 @@ BEAM_VERSION=$BEAM_VERSION CREDENTIALS_VOLUME=$CREDENTIALS_VOLUME +DEPENDENCIES_VOLUME=$DEPENDENCIES_VOLUME + +# A requirements file with either of the following +# * PyPi packages +# * Locally available packages relative to the directory provided to +# DEPENDENCIES_VOLUME. +PYTHON_REQUIREMENTS_FILE_NAME=$PYTHON_REQUIREMENTS_FILE_NAME + GOOGLE_APPLICATION_CREDENTIALS_FILE_NAME=application_default_credentials.json COMPOSE_PROJECT_NAME=apache.beam.transform.service TRANSFORM_SERVICE_PORT=$TRANSFORM_SERVICE_PORT diff --git a/sdks/java/transform-service/docker-compose/docker-compose.yml b/sdks/java/transform-service/docker-compose/docker-compose.yml index b685be10a329..39235533b9a8 100644 --- a/sdks/java/transform-service/docker-compose/docker-compose.yml +++ b/sdks/java/transform-service/docker-compose/docker-compose.yml @@ -32,8 +32,9 @@ services: expansion-service-2: image: "apache/beam_python_expansion_service:${BEAM_VERSION}" restart: on-failure - command: -id expansion-service-2 -port 5001 + command: -id expansion-service-2 -port 5001 -requirements_file ${PYTHON_REQUIREMENTS_FILE_NAME} -dependencies_dir '/dependencies_volume' volumes: - ${CREDENTIALS_VOLUME}:/credentials_volume + - ${DEPENDENCIES_VOLUME}:/dependencies_volume environment: - GOOGLE_APPLICATION_CREDENTIALS=/credentials_volume/${GOOGLE_APPLICATION_CREDENTIALS_FILE_NAME} diff --git a/sdks/java/transform-service/launcher/build.gradle b/sdks/java/transform-service/launcher/build.gradle index 83c5d60a1ef1..0952f37109eb 100644 --- a/sdks/java/transform-service/launcher/build.gradle +++ b/sdks/java/transform-service/launcher/build.gradle @@ -45,6 +45,9 @@ dependencies { shadow library.java.args4j shadow library.java.error_prone_annotations permitUnusedDeclared(library.java.error_prone_annotations) + testImplementation library.java.junit + testImplementation library.java.mockito_core + testImplementation project(path: ":sdks:java:core") } sourceSets { diff --git a/sdks/java/transform-service/launcher/src/main/java/org/apache/beam/sdk/transformservice/launcher/TransformServiceLauncher.java b/sdks/java/transform-service/launcher/src/main/java/org/apache/beam/sdk/transformservice/launcher/TransformServiceLauncher.java index f52fdfed710d..c0a9097a762f 100644 --- a/sdks/java/transform-service/launcher/src/main/java/org/apache/beam/sdk/transformservice/launcher/TransformServiceLauncher.java +++ b/sdks/java/transform-service/launcher/src/main/java/org/apache/beam/sdk/transformservice/launcher/TransformServiceLauncher.java @@ -17,9 +17,11 @@ */ package org.apache.beam.sdk.transformservice.launcher; +import java.io.BufferedWriter; import java.io.File; import java.io.FileOutputStream; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.nio.file.Path; import java.nio.file.Paths; import java.util.ArrayList; @@ -28,6 +30,7 @@ import java.util.Locale; import java.util.Map; import java.util.concurrent.TimeoutException; +import java.util.stream.Stream; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.io.ByteStreams; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.io.Files; @@ -62,9 +65,9 @@ public class TransformServiceLauncher { private static final int STATUS_LOGGER_WAIT_TIME = 3000; @SuppressWarnings("argument") - private TransformServiceLauncher(@Nullable String projectName, int port) throws IOException { - LOG.info("Initializing the Beam Transform Service {}.", projectName); - + private TransformServiceLauncher( + @Nullable String projectName, int port, @Nullable String pythonRequirementsFile) + throws IOException { String tmpDirLocation = System.getProperty("java.io.tmpdir"); // We use Docker Compose project name as the name of the temporary directory to isolate // different transform service instances that may be running in the same machine. @@ -83,14 +86,14 @@ private TransformServiceLauncher(@Nullable String projectName, int port) throws ByteStreams.copy(getClass().getResourceAsStream("/.env"), fout); } + // Setting up the credentials directory. File credentialsDir = Paths.get(tmpDir, "credentials_dir").toFile(); - LOG.info( - "Creating a temporary directory for storing credentials: " - + credentialsDir.getAbsolutePath()); - if (credentialsDir.exists()) { LOG.info("Reusing the existing credentials directory " + credentialsDir.getAbsolutePath()); } else { + LOG.info( + "Creating a temporary directory for storing credentials: " + + credentialsDir.getAbsolutePath()); if (!credentialsDir.mkdir()) { throw new IOException( "Could not create a temporary directory for storing credentials: " @@ -124,10 +127,84 @@ private TransformServiceLauncher(@Nullable String projectName, int port) throws } } + // Setting up the dependencies directory. + File dependenciesDir = Paths.get(tmpDir, "dependencies_dir").toFile(); + Path updatedRequirementsFilePath = Paths.get(dependenciesDir.toString(), "requirements.txt"); + if (dependenciesDir.exists()) { + LOG.info("Reusing the existing dependencies directory " + dependenciesDir.getAbsolutePath()); + } else { + LOG.info( + "Creating a temporary directory for storing dependencies: " + + dependenciesDir.getAbsolutePath()); + if (!dependenciesDir.mkdir()) { + throw new IOException( + "Could not create a temporary directory for storing dependencies: " + + dependenciesDir.getAbsolutePath()); + } + + // We create a requirements file with extra dependencies. + // If there are no extra dependencies, we just provide an empty requirements file. + File file = updatedRequirementsFilePath.toFile(); + if (!file.createNewFile()) { + throw new IOException( + "Could not create the new requirements file " + updatedRequirementsFilePath); + } + + // Updating dependencies. + if (pythonRequirementsFile != null) { + Path requirementsFilePath = Paths.get(pythonRequirementsFile); + List updatedLines = new ArrayList<>(); + + try (Stream lines = java.nio.file.Files.lines(requirementsFilePath)) { + lines.forEachOrdered( + line -> { + Path dependencyFilePath = Paths.get(line); + if (java.nio.file.Files.exists(dependencyFilePath)) { + Path fileName = dependencyFilePath.getFileName(); + if (fileName == null) { + throw new IllegalArgumentException( + "Could not determine the filename of the local artifact " + + dependencyFilePath); + } + try { + java.nio.file.Files.copy( + dependencyFilePath, + Paths.get(dependenciesDir.toString(), fileName.toString())); + } catch (IOException e) { + throw new RuntimeException(e); + } + updatedLines.add(fileName.toString()); + } else { + updatedLines.add(line); + } + }); + } + + try (BufferedWriter writer = + java.nio.file.Files.newBufferedWriter(file.toPath(), StandardCharsets.UTF_8)) { + for (String line : updatedLines) { + writer.write(line); + writer.newLine(); + } + writer.flush(); + } + } + } + // Setting environment variables used by the docker-compose.yml file. environmentVariables.put("CREDENTIALS_VOLUME", credentialsDir.getAbsolutePath()); + environmentVariables.put("DEPENDENCIES_VOLUME", dependenciesDir.getAbsolutePath()); environmentVariables.put("TRANSFORM_SERVICE_PORT", String.valueOf(port)); + Path updatedRequirementsFileName = updatedRequirementsFilePath.getFileName(); + if (updatedRequirementsFileName == null) { + throw new IllegalArgumentException( + "Could not determine the file name of the updated requirements file " + + updatedRequirementsFilePath); + } + environmentVariables.put( + "PYTHON_REQUIREMENTS_FILE_NAME", updatedRequirementsFileName.toString()); + // Building the Docker Compose command. dockerComposeStartCommandPrefix.add("docker-compose"); dockerComposeStartCommandPrefix.add("-p"); @@ -136,21 +213,37 @@ private TransformServiceLauncher(@Nullable String projectName, int port) throws dockerComposeStartCommandPrefix.add(dockerComposeFile.getAbsolutePath()); } + /** + * Specifies the Beam version to get containers for the transform service. + * + *

Could be a release Beam version with containers in Docker Hub or an unreleased Beam version + * for which containers are available locally. + * + * @param beamVersion a Beam version to get containers from. + */ public void setBeamVersion(String beamVersion) { environmentVariables.put("BEAM_VERSION", beamVersion); } - public void setPythonExtraPackages(String pythonExtraPackages) { - environmentVariables.put("$PYTHON_EXTRA_PACKAGES", pythonExtraPackages); - } - + /** + * Initializes a client for managing transform service instances. + * + * @param projectName project name for the transform service. + * @param port port exposed by the transform service. + * @param pythonRequirementsFile a requirements file with extra dependencies for the Python + * expansion services. + * @return an initialized client for managing the transform service. + * @throws IOException + */ public static synchronized TransformServiceLauncher forProject( - @Nullable String projectName, int port) throws IOException { + @Nullable String projectName, int port, @Nullable String pythonRequirementsFile) + throws IOException { if (projectName == null || projectName.isEmpty()) { projectName = DEFAULT_PROJECT_NAME; } if (!launchers.containsKey(projectName)) { - launchers.put(projectName, new TransformServiceLauncher(projectName, port)); + launchers.put( + projectName, new TransformServiceLauncher(projectName, port, pythonRequirementsFile)); } return launchers.get(projectName); } @@ -200,10 +293,10 @@ public synchronized void status() throws IOException { public synchronized void waitTillUp(int timeout) throws IOException, TimeoutException { timeout = timeout <= 0 ? DEFAULT_START_WAIT_TIME : timeout; - String statusFileName = getStatus(); long startTime = System.currentTimeMillis(); while (System.currentTimeMillis() - startTime < timeout) { + String statusFileName = getStatus(); try { // We are just waiting for a local process. No need for exponential backoff. this.wait(1000); @@ -226,6 +319,7 @@ public synchronized void waitTillUp(int timeout) throws IOException, TimeoutExce private synchronized String getStatus() throws IOException { File outputOverride = File.createTempFile("output_override", null); + outputOverride.deleteOnExit(); runDockerComposeCommand(ImmutableList.of("ps"), outputOverride); return outputOverride.getAbsolutePath(); @@ -238,6 +332,8 @@ private static class ArgConfig { static final String PORT_ARG_NAME = "port"; static final String BEAM_VERSION_ARG_NAME = "beam_version"; + static final String PYTHON_REQUIREMENTS_FILE_ARG_NAME = "python_requirements_file"; + @Option(name = "--" + PROJECT_NAME_ARG_NAME, usage = "Docker compose project name") private String projectName = ""; @@ -249,6 +345,11 @@ private static class ArgConfig { @Option(name = "--" + BEAM_VERSION_ARG_NAME, usage = "Beam version to use.") private String beamVersion = ""; + + @Option( + name = "--" + PYTHON_REQUIREMENTS_FILE_ARG_NAME, + usage = "Extra Python packages in the form of an requirements file.") + private String pythonRequirementsFile = ""; } public static void main(String[] args) throws IOException, TimeoutException { @@ -288,8 +389,12 @@ public static void main(String[] args) throws IOException, TimeoutException { : ("port " + Integer.toString(config.port) + "."))); System.out.println("==================================================="); + String pythonRequirementsFile = + !config.pythonRequirementsFile.isEmpty() ? config.pythonRequirementsFile : null; + TransformServiceLauncher service = - TransformServiceLauncher.forProject(config.projectName, config.port); + TransformServiceLauncher.forProject( + config.projectName, config.port, pythonRequirementsFile); if (!config.beamVersion.isEmpty()) { service.setBeamVersion(config.beamVersion); } diff --git a/sdks/java/transform-service/launcher/src/test/java/org/apache/beam/sdk/transformservice/launcher/TransformServiceLauncherTest.java b/sdks/java/transform-service/launcher/src/test/java/org/apache/beam/sdk/transformservice/launcher/TransformServiceLauncherTest.java new file mode 100644 index 000000000000..4ef84b02061b --- /dev/null +++ b/sdks/java/transform-service/launcher/src/test/java/org/apache/beam/sdk/transformservice/launcher/TransformServiceLauncherTest.java @@ -0,0 +1,185 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.transformservice.launcher; + +import static java.nio.charset.StandardCharsets.UTF_8; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.io.BufferedReader; +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.OutputStreamWriter; +import java.io.Writer; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.UUID; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class TransformServiceLauncherTest { + + @Test + public void testLauncherCreatesCredentialsDir() throws IOException { + String projectName = UUID.randomUUID().toString(); + Path expectedTempDir = Paths.get(System.getProperty("java.io.tmpdir"), projectName); + File file = expectedTempDir.toFile(); + file.deleteOnExit(); + TransformServiceLauncher.forProject(projectName, 12345, null); + Path expectedCredentialsDir = Paths.get(expectedTempDir.toString(), "credentials_dir"); + assertTrue(expectedCredentialsDir.toFile().exists()); + } + + @Test + public void testLauncherCreatesDependenciesDir() throws IOException { + String projectName = UUID.randomUUID().toString(); + Path expectedTempDir = Paths.get(System.getProperty("java.io.tmpdir"), projectName); + File file = expectedTempDir.toFile(); + file.deleteOnExit(); + TransformServiceLauncher.forProject(projectName, 12345, null); + Path expectedCredentialsDir = Paths.get(expectedTempDir.toString(), "dependencies_dir"); + assertTrue(expectedCredentialsDir.toFile().exists()); + } + + @Test + public void testLauncherInstallsDependencies() throws IOException { + String projectName = UUID.randomUUID().toString(); + Path expectedTempDir = Paths.get(System.getProperty("java.io.tmpdir"), projectName); + File file = expectedTempDir.toFile(); + file.deleteOnExit(); + + File requirementsFile = + Paths.get( + System.getProperty("java.io.tmpdir"), + ("requirements" + UUID.randomUUID().toString() + ".txt")) + .toFile(); + requirementsFile.deleteOnExit(); + + try (Writer fout = + new OutputStreamWriter( + new FileOutputStream(requirementsFile.getAbsolutePath()), Charsets.UTF_8)) { + fout.write("pypipackage1\n"); + fout.write("pypipackage2\n"); + } + + TransformServiceLauncher.forProject(projectName, 12345, requirementsFile.getAbsolutePath()); + + // Confirming that the Transform Service launcher created a temporary requirements file with the + // specified set of packages. + Path expectedUpdatedRequirementsFile = + Paths.get(expectedTempDir.toString(), "dependencies_dir", "requirements.txt"); + assertTrue(expectedUpdatedRequirementsFile.toFile().exists()); + + ArrayList expectedUpdatedRequirementsFileLines = new ArrayList<>(); + try (BufferedReader bufReader = + Files.newBufferedReader(expectedUpdatedRequirementsFile, UTF_8)) { + String line = bufReader.readLine(); + while (line != null) { + expectedUpdatedRequirementsFileLines.add(line); + line = bufReader.readLine(); + } + } + + assertEquals(2, expectedUpdatedRequirementsFileLines.size()); + assertTrue(expectedUpdatedRequirementsFileLines.contains("pypipackage1")); + assertTrue(expectedUpdatedRequirementsFileLines.contains("pypipackage2")); + } + + @Test + public void testLauncherInstallsLocalDependencies() throws IOException { + String projectName = UUID.randomUUID().toString(); + Path expectedTempDir = Paths.get(System.getProperty("java.io.tmpdir"), projectName); + File file = expectedTempDir.toFile(); + file.deleteOnExit(); + + String dependency1FileName = "dep_" + UUID.randomUUID().toString(); + File dependency1 = + Paths.get(System.getProperty("java.io.tmpdir"), dependency1FileName).toFile(); + dependency1.deleteOnExit(); + try (Writer fout = + new OutputStreamWriter( + new FileOutputStream(dependency1.getAbsolutePath()), Charsets.UTF_8)) { + fout.write("tempdata\n"); + } + + String dependency2FileName = "dep_" + UUID.randomUUID().toString(); + File dependency2 = + Paths.get(System.getProperty("java.io.tmpdir"), dependency2FileName).toFile(); + dependency2.deleteOnExit(); + try (Writer fout = + new OutputStreamWriter( + new FileOutputStream(dependency2.getAbsolutePath()), Charsets.UTF_8)) { + fout.write("tempdata\n"); + } + + File requirementsFile = + Paths.get( + System.getProperty("java.io.tmpdir"), + ("requirements" + UUID.randomUUID().toString() + ".txt")) + .toFile(); + requirementsFile.deleteOnExit(); + try (Writer fout = + new OutputStreamWriter( + new FileOutputStream(requirementsFile.getAbsolutePath()), Charsets.UTF_8)) { + fout.write(dependency1.getAbsolutePath() + "\n"); + fout.write(dependency2.getAbsolutePath() + "\n"); + fout.write("pypipackage" + "\n"); + } + + TransformServiceLauncher.forProject(projectName, 12345, requirementsFile.getAbsolutePath()); + + // Confirming that the Transform Service launcher created a temporary requirements file with the + // specified set of packages. + Path expectedUpdatedRequirementsFile = + Paths.get(expectedTempDir.toString(), "dependencies_dir", "requirements.txt"); + assertTrue(expectedUpdatedRequirementsFile.toFile().exists()); + + ArrayList expectedUpdatedRequirementsFileLines = new ArrayList<>(); + try (BufferedReader bufReader = + Files.newBufferedReader(expectedUpdatedRequirementsFile, UTF_8)) { + String line = bufReader.readLine(); + while (line != null) { + expectedUpdatedRequirementsFileLines.add(line); + line = bufReader.readLine(); + } + } + + // To make local packages available to the expansion service Docker containers, the temporary + // requirements file should contain names of the local packages relative to the dependencies + // volume and local packages should have been copied to the dependencies volume. + assertEquals(3, expectedUpdatedRequirementsFileLines.size()); + assertTrue(expectedUpdatedRequirementsFileLines.contains(dependency1FileName)); + assertTrue(expectedUpdatedRequirementsFileLines.contains(dependency2FileName)); + assertTrue(expectedUpdatedRequirementsFileLines.contains("pypipackage")); + + assertTrue( + Paths.get(expectedTempDir.toString(), "dependencies_dir", dependency1FileName) + .toFile() + .exists()); + assertTrue( + Paths.get(expectedTempDir.toString(), "dependencies_dir", dependency2FileName) + .toFile() + .exists()); + } +} diff --git a/sdks/java/transform-service/src/main/java/org/apache/beam/sdk/transformservice/ExpansionService.java b/sdks/java/transform-service/src/main/java/org/apache/beam/sdk/transformservice/ExpansionService.java index 17fe5472f9fc..0a2e65099e7d 100644 --- a/sdks/java/transform-service/src/main/java/org/apache/beam/sdk/transformservice/ExpansionService.java +++ b/sdks/java/transform-service/src/main/java/org/apache/beam/sdk/transformservice/ExpansionService.java @@ -17,15 +17,22 @@ */ package org.apache.beam.sdk.transformservice; +import java.io.IOException; +import java.net.Socket; import java.util.ArrayList; +import java.util.HashMap; import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeoutException; import org.apache.beam.model.expansion.v1.ExpansionApi; +import org.apache.beam.model.expansion.v1.ExpansionApi.ExpansionResponse; import org.apache.beam.model.expansion.v1.ExpansionServiceGrpc; import org.apache.beam.model.pipeline.v1.Endpoints; import org.apache.beam.runners.core.construction.DefaultExpansionServiceClientFactory; import org.apache.beam.runners.core.construction.ExpansionServiceClientFactory; import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ManagedChannelBuilder; import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Throwables; import org.checkerframework.checker.nullness.qual.Nullable; @@ -40,6 +47,12 @@ public class ExpansionService extends ExpansionServiceGrpc.ExpansionServiceImplB final List endpoints; + private boolean checkedAllServices = false; + + private static final long SERVICE_CHECK_TIMEOUT_MILLIS = 60000; + + private boolean disableServiceCheck = false; + ExpansionService( List endpoints, @Nullable ExpansionServiceClientFactory clientFactory) { @@ -48,10 +61,65 @@ public class ExpansionService extends ExpansionServiceGrpc.ExpansionServiceImplB clientFactory != null ? clientFactory : DEFAULT_EXPANSION_SERVICE_CLIENT_FACTORY; } + // Waits till all expansion services are ready. + private void waitForAllServicesToBeReady() throws TimeoutException { + if (disableServiceCheck) { + // Service check disabled. Just returning. + return; + } + + outer: + for (Endpoints.ApiServiceDescriptor endpoint : endpoints) { + long start = System.currentTimeMillis(); + long duration = 10; + while (System.currentTimeMillis() - start < SERVICE_CHECK_TIMEOUT_MILLIS) { + try { + String url = endpoint.getUrl(); + int portIndex = url.lastIndexOf(":"); + if (portIndex <= 0) { + throw new RuntimeException( + "Expected the endpoint to be of the form : but received " + url); + } + int port = Integer.parseInt(url.substring(portIndex + 1)); + String host = url.substring(0, portIndex); + new Socket(host, port).close(); + // Current service is up. Checking the next one. + continue outer; + } catch (IOException exn) { + try { + Thread.sleep(duration); + } catch (InterruptedException e) { + // Ignore + } + duration = (long) (duration * 1.2); + } + } + throw new TimeoutException( + "Timeout waiting for the service " + + endpoint.getUrl() + + " to startup after " + + (System.currentTimeMillis() - start) + + " milliseconds."); + } + } + + @VisibleForTesting + void disableServiceCheck() { + disableServiceCheck = true; + } + @Override public void expand( ExpansionApi.ExpansionRequest request, StreamObserver responseObserver) { + if (!checkedAllServices) { + try { + waitForAllServicesToBeReady(); + } catch (TimeoutException e) { + throw new RuntimeException(e); + } + checkedAllServices = true; + } try { responseObserver.onNext(processExpand(request)); responseObserver.onCompleted(); @@ -68,6 +136,14 @@ public void expand( public void discoverSchemaTransform( ExpansionApi.DiscoverSchemaTransformRequest request, StreamObserver responseObserver) { + if (!checkedAllServices) { + try { + waitForAllServicesToBeReady(); + } catch (TimeoutException e) { + throw new RuntimeException(e); + } + checkedAllServices = true; + } try { responseObserver.onNext(processDiscover(request)); responseObserver.onCompleted(); @@ -80,18 +156,41 @@ public void discoverSchemaTransform( } } - /*package*/ ExpansionApi.ExpansionResponse processExpand(ExpansionApi.ExpansionRequest request) { + private ExpansionApi.ExpansionResponse getAggregatedErrorResponse( + Map errorResponses) { + StringBuilder errorMessageBuilder = new StringBuilder(); + + errorMessageBuilder.append( + "Aggregated errors from " + errorResponses.size() + " expansion services." + "\n"); + for (Map.Entry entry : errorResponses.entrySet()) { + errorMessageBuilder.append( + "Error from expansion service " + + entry.getKey() + + ": " + + entry.getValue().getError() + + "\n"); + } + + return errorResponses + .values() + .iterator() + .next() + .toBuilder() + .setError(errorMessageBuilder.toString()) + .build(); + } + + ExpansionApi.ExpansionResponse processExpand(ExpansionApi.ExpansionRequest request) { // Trying out expansion services in order till one succeeds. // If all services fail, re-raises the last error. - // TODO: when all services fail, return an aggregated error with errors from all services. - ExpansionApi.ExpansionResponse lastErrorResponse = null; + Map errorResponses = new HashMap<>(); RuntimeException lastException = null; for (Endpoints.ApiServiceDescriptor endpoint : endpoints) { try { ExpansionApi.ExpansionResponse response = expansionServiceClientFactory.getExpansionServiceClient(endpoint).expand(request); if (!response.getError().isEmpty()) { - lastErrorResponse = response; + errorResponses.put(endpoint.getUrl(), response); continue; } return response; @@ -99,8 +198,11 @@ public void discoverSchemaTransform( lastException = e; } } - if (lastErrorResponse != null) { - return lastErrorResponse; + if (lastException != null) { + throw new RuntimeException("Expansion request to transform service failed.", lastException); + } + if (!errorResponses.isEmpty()) { + return getAggregatedErrorResponse(errorResponses); } else if (lastException != null) { throw new RuntimeException("Expansion request to transform service failed.", lastException); } else { diff --git a/sdks/java/transform-service/src/test/java/org/apache/beam/sdk/transformservice/ExpansionServiceTest.java b/sdks/java/transform-service/src/test/java/org/apache/beam/sdk/transformservice/ExpansionServiceTest.java index 298bce87f901..9905abd1d9ba 100644 --- a/sdks/java/transform-service/src/test/java/org/apache/beam/sdk/transformservice/ExpansionServiceTest.java +++ b/sdks/java/transform-service/src/test/java/org/apache/beam/sdk/transformservice/ExpansionServiceTest.java @@ -60,6 +60,8 @@ public void setUp() throws Exception { endpoints.add(endpoint2); clientFactory = Mockito.mock(ExpansionServiceClientFactory.class); expansionService = new ExpansionService(endpoints, clientFactory); + // We do not run actual services in unit tests. + expansionService.disableServiceCheck(); } @Test @@ -131,7 +133,10 @@ public void testExpandFail() { ArgumentCaptor expansionResponseCapture = ArgumentCaptor.forClass(ExpansionResponse.class); Mockito.verify(responseObserver).onNext(expansionResponseCapture.capture()); - assertEquals("expansion error 2", expansionResponseCapture.getValue().getError()); + + // Error response should contain errors from both expansion services. + assertTrue(expansionResponseCapture.getValue().getError().contains("expansion error 1")); + assertTrue(expansionResponseCapture.getValue().getError().contains("expansion error 2")); } @Test diff --git a/sdks/python/apache_beam/utils/transform_service_launcher.py b/sdks/python/apache_beam/utils/transform_service_launcher.py index 33feab9bf29c..ac492513aba5 100644 --- a/sdks/python/apache_beam/utils/transform_service_launcher.py +++ b/sdks/python/apache_beam/utils/transform_service_launcher.py @@ -86,6 +86,7 @@ def __init__(self, project_name, port, beam_version=None): compose_file = os.path.join(temp_dir, 'docker-compose.yml') + # Creating the credentials volume. credentials_dir = os.path.join(temp_dir, 'credentials_dir') if not os.path.exists(credentials_dir): os.mkdir(credentials_dir) @@ -111,11 +112,24 @@ def __init__(self, project_name, port, beam_version=None): 'credentials file at the expected location %s.' % application_default_path_file) + # Creating the dependencies volume. + dependencies_dir = os.path.join(temp_dir, 'dependencies_dir') + if not os.path.exists(dependencies_dir): + os.mkdir(dependencies_dir) + self._environmental_variables = {} self._environmental_variables['CREDENTIALS_VOLUME'] = credentials_dir + self._environmental_variables['DEPENDENCIES_VOLUME'] = dependencies_dir self._environmental_variables['TRANSFORM_SERVICE_PORT'] = str(port) self._environmental_variables['BEAM_VERSION'] = beam_version + # Setting an empty requirements file + requirements_file_name = os.path.join(dependencies_dir, 'requirements.txt') + with open(requirements_file_name, 'w') as _: + pass + self._environmental_variables['PYTHON_REQUIREMENTS_FILE_NAME'] = ( + 'requirements.txt') + self._docker_compose_start_command_prefix = [] self._docker_compose_start_command_prefix.append('docker-compose') self._docker_compose_start_command_prefix.append('-p') diff --git a/sdks/python/expansion-service-container/boot.go b/sdks/python/expansion-service-container/boot.go index 90a97c35425a..ba56b349c4ea 100644 --- a/sdks/python/expansion-service-container/boot.go +++ b/sdks/python/expansion-service-container/boot.go @@ -18,8 +18,10 @@ package main import ( + "bufio" "flag" "fmt" + "io/ioutil" "log" "os" "path/filepath" @@ -31,16 +33,15 @@ import ( ) var ( - id = flag.String("id", "", "Local identifier (required)") - port = flag.Int("port", 0, "Port for the expansion service (required)") + id = flag.String("id", "", "Local identifier (required)") + port = flag.Int("port", 0, "Port for the expansion service (required)") + requirements_file = flag.String("requirements_file", "", "A requirement file with extra packages to be made available to the transforms being expanded. Path should be relative to the 'dependencies_dir'") + dependencies_dir = flag.String("dependencies_dir", "", "A directory that stores locally available extra packages.") ) const ( expansionServiceEntrypoint = "apache_beam.runners.portability.expansion_service_main" venvDirectory = "beam_venv" // This should match the venv directory name used in the Dockerfile. - requirementsFile = "requirements.txt" - beamSDKArtifact = "apache-beam-sdk.tar.gz" - beamSDKOptions = "[gcp,dataframe]" ) func main() { @@ -58,6 +59,79 @@ func main() { } } +func getLines(fileNameToRead string) ([]string, error) { + fileToRead, err := os.Open(fileNameToRead) + if err != nil { + return nil, err + } + defer fileToRead.Close() + + sc := bufio.NewScanner(fileToRead) + lines := make([]string, 0) + + // Read through 'tokens' until an EOF is encountered. + for sc.Scan() { + lines = append(lines, sc.Text()) + } + + if err := sc.Err(); err != nil { + return nil, err + } + return lines, nil +} + +func installExtraPackages(requirementsFile string) error { + extraPackages, err := getLines(requirementsFile) + if err != nil { + return err + } + + for _, extraPackage := range extraPackages { + log.Printf("Installing extra package %v", extraPackage) + // We expect 'pip' command in virtual env to be already available at the top of the PATH. + args := []string{"install", extraPackage} + if err := execx.Execute("pip", args...); err != nil { + return fmt.Errorf("Could not install the package %s: %s", extraPackage, err) + } + } + return nil +} + +func getUpdatedRequirementsFile(oldRequirementsFileName string, dependenciesDir string) (string, error) { + oldExtraPackages, err := getLines(filepath.Join(dependenciesDir, oldRequirementsFileName)) + if err != nil { + return "", err + } + var updatedExtraPackages = make([]string, 0) + for _, extraPackage := range oldExtraPackages { + // TODO update + potentialLocalFilePath := filepath.Join(dependenciesDir, extraPackage) + _, err := os.Stat(potentialLocalFilePath) + if err == nil { + // Package exists locally so using that. + extraPackage = potentialLocalFilePath + log.Printf("Using locally available extra package %v", extraPackage) + } + updatedExtraPackages = append(updatedExtraPackages, extraPackage) + } + + updatedRequirementsFile, err := ioutil.TempFile("/opt/apache/beam", "requirements*.txt") + if err != nil { + return "", err + } + + updatedRequirementsFileName := updatedRequirementsFile.Name() + + datawriter := bufio.NewWriter(updatedRequirementsFile) + for _, extraPackage := range updatedExtraPackages { + _, _ = datawriter.WriteString(extraPackage + "\n") + } + datawriter.Flush() + updatedRequirementsFile.Close() + + return updatedRequirementsFileName, nil +} + func launchExpansionServiceProcess() error { pythonVersion, err := expansionx.GetPythonVersion() if err != nil { @@ -70,6 +144,24 @@ func launchExpansionServiceProcess() error { os.Setenv("PATH", strings.Join([]string{filepath.Join(dir, "bin"), os.Getenv("PATH")}, ":")) args := []string{"-m", expansionServiceEntrypoint, "-p", strconv.Itoa(*port), "--fully_qualified_name_glob", "*"} + + if *requirements_file != "" { + log.Printf("Received the requirements file %v", *requirements_file) + updatedRequirementsFileName, err := getUpdatedRequirementsFile(*requirements_file, *dependencies_dir) + if err != nil { + return err + } + defer os.Remove(updatedRequirementsFileName) + log.Printf("Updated requirements file is %v", updatedRequirementsFileName) + // Provide the requirements file to the expansion service so that packages get staged by runners. + args = append(args, "--requirements_file", updatedRequirementsFileName) + // Install packages locally so that they can be used by the expansion service during transform + // expansion if needed. + err = installExtraPackages(updatedRequirementsFileName) + if err != nil { + return err + } + } if err := execx.Execute(pythonVersion, args...); err != nil { return fmt.Errorf("could not start the expansion service: %s", err) } From 88139bdc9a04b039d58d2669ee85914223ef9bd6 Mon Sep 17 00:00:00 2001 From: Andrey Devyatkin Date: Wed, 11 Oct 2023 14:05:35 +0200 Subject: [PATCH 096/435] Retrieve workflow runs for GA Post-Commits Status dashboard (#28844) * Fixed the GitHub Actions postcommit dashboard * retrieve workflow runs from GitHub API * fixed throttling and out of memory issues and excluded 'skipped' runs --- .test-infra/metrics/docker-compose.yml | 5 +- .../metrics/sync/github/requirements.txt | 6 +- .../metrics/sync/github/sync_workflows.py | 400 +++++++++++------- 3 files changed, 250 insertions(+), 161 deletions(-) diff --git a/.test-infra/metrics/docker-compose.yml b/.test-infra/metrics/docker-compose.yml index 77f07bad13fa..3d847ff79676 100644 --- a/.test-infra/metrics/docker-compose.yml +++ b/.test-infra/metrics/docker-compose.yml @@ -85,7 +85,10 @@ services: - DB_DBNAME=beam_metrics - DB_DBUSERNAME=admin - DB_DBPWD= - - GH_ACCESS_TOKEN= + - GH_APP_ID= + - GH_APP_INSTALLATION_ID= + - GH_PEM_KEY= + - GH_NUMBER_OF_WORKFLOW_RUNS_TO_FETCH=30 syncjenkins: image: syncjenkins container_name: beamsyncjenkins diff --git a/.test-infra/metrics/sync/github/requirements.txt b/.test-infra/metrics/sync/github/requirements.txt index 14a64686e55a..5b231565459f 100644 --- a/.test-infra/metrics/sync/github/requirements.txt +++ b/.test-infra/metrics/sync/github/requirements.txt @@ -13,7 +13,7 @@ # See the License for the specific language governing permissions and # limitations under the License. -requests +aiohttp +backoff psycopg2-binary -ddt - +PyGithub \ No newline at end of file diff --git a/.test-infra/metrics/sync/github/sync_workflows.py b/.test-infra/metrics/sync/github/sync_workflows.py index 0b4ddfec3802..25065878c4bd 100644 --- a/.test-infra/metrics/sync/github/sync_workflows.py +++ b/.test-infra/metrics/sync/github/sync_workflows.py @@ -1,4 +1,3 @@ -# # Licensed to the Apache Software Foundation (ASF) under one or more # contributor license agreements. See the NOTICE file distributed with # this work for additional information regarding copyright ownership. @@ -6,180 +5,267 @@ # (the "License"); you may not use this file except in compliance with # the License. You may obtain a copy of the License at # -# http://www.apache.org/licenses/LICENSE-2.0 +# http://www.apache.org/licenses/LICENSE-2.0 # +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + ''' -This module queries GitHub to collect Beam-related workflows metrics and put them in -PostgreSQL. -This Script is running every 3 hours in a cloud function in apache-beam-testing project. -This cloud function is triggered by a pubsub topic. -You can find the cloud function in the next link +This module queries GitHub API to collect Beam-related workflows metrics and +put them in PostgreSQL. +This script is running every 3 hours as a cloud function +"github_actions_workflows_dashboard_sync" in apache-beam-testing project: https://console.cloud.google.com/functions/details/us-central1/github_actions_workflows_dashboard_sync?env=gen1&project=apache-beam-testing -Pub sub topic : https://console.cloud.google.com/cloudpubsub/topic/detail/github_actions_workflows_sync?project=apache-beam-testing -Cron Job : https://console.cloud.google.com/cloudscheduler/jobs/edit/us-central1/github_actions_workflows_dashboard_sync?project=apache-beam-testing -Writing the latest 10 runs of every workflow in master branch in a beammetrics database +This cloud function is triggered by a pubsub topic: +https://console.cloud.google.com/cloudpubsub/topic/detail/github_actions_workflows_sync?project=apache-beam-testing +Cron Job: +https://console.cloud.google.com/cloudscheduler/jobs/edit/us-central1/github_actions_workflows_dashboard_sync?project=apache-beam-testing ''' +import asyncio +import aiohttp +import backoff +import math import os import sys import time import re -import requests import psycopg2 - -from datetime import datetime -from github import GithubIntegration +from github import GithubIntegration DB_HOST = os.environ['DB_HOST'] DB_PORT = os.environ['DB_PORT'] DB_NAME = os.environ['DB_DBNAME'] DB_USER_NAME = os.environ['DB_DBUSERNAME'] DB_PASSWORD = os.environ['DB_DBPWD'] -GH_WORKFLOWS_TABLE_NAME = "github_workflows" -# Number of workflows that fetch github API -GH_NUMBER_OF_WORKFLOWS = 100 -GH_WORKFLOWS_NUMBER_EXECUTIONS = 100 -WORKFLOWS_OBJECT_LIST = [] +GH_APP_ID = os.environ['GH_APP_ID'] +GH_APP_INSTALLATION_ID = os.environ['GH_APP_INSTALLATION_ID'] +GH_PEM_KEY = os.environ['GH_PEM_KEY'] +GH_NUMBER_OF_WORKFLOW_RUNS_TO_FETCH =\ + os.environ['GH_NUMBER_OF_WORKFLOW_RUNS_TO_FETCH'] class Workflow: - def __init__(self,id,name,filename): - self.id = id - self.name = name - self.filename = filename - self.listOfRuns = [] - self.runUrl = [] - -# The table will save the latest ten run of every workflow -GH_WORKFLOWS_CREATE_TABLE_QUERY = f""" -CREATE TABLE IF NOT EXISTS {GH_WORKFLOWS_TABLE_NAME} ( - job_name text PRIMARY KEY, - job_yml_filename text""" -for i in range(0,GH_WORKFLOWS_NUMBER_EXECUTIONS): - i = i + 1 - GH_WORKFLOWS_CREATE_TABLE_QUERY += """,\n run{} text, - run{}Id text""".format(str(i),str(i)) -GH_WORKFLOWS_CREATE_TABLE_QUERY += ")\n" - -def githubWorkflowsGrafanaSync(data,context): - print('Started') - print('Updating table with recent workflow runs') - databaseOperations(initDbConnection(),fetchWorkflowData()) - print('Done') - return "Completed" - -def initDbConnection(): - '''Init connection with the Database''' - connection = None - maxRetries = 3 - i = 0 - while connection == None and i < maxRetries: - try: - connection = psycopg2.connect( - f"dbname='{DB_NAME}' user='{DB_USER_NAME}' host='{DB_HOST}'" - f" port='{DB_PORT}' password='{DB_PASSWORD}'") - except Exception as e: - print('Failed to connect to DB; retrying in 1 minute') - print(e) - time.sleep(60) - i = i + 1 - if i >= maxRetries: - print("Number of retries exceded ") - sys.exit(1) - return connection - -def getToken(): - git_integration = GithubIntegration( - os.environ["GH_APP_ID"], - os.environ["GH_PEM_KEY"]) - token=git_integration.get_access_token( - os.environ["GH_APP_INSTALLATION_ID"] - ).token - return token - -def retriesRequest(request): - requestSucceeded = False - retryFactor = 1 - while not requestSucceeded: - retryTime = 60 * retryFactor - if request.status_code != 200: - print('Failed to get the request with code {}'.format(request.status_code)) - time.sleep(retryTime) - retryFactor = retryFactor + retryFactor - if retryFactor * 60 >= 3600: - print("Error: The request take more than an hour") - sys.exit(1) - else: - requestSucceeded = True -def fetchWorkflowData(): - '''Return a json with all the workflows and the latests - ten executions''' - completed = False - page = 1 - workflows = [] + def __init__(self, id, name, filename): + self.id = id + self.name = name + self.filename = filename + self.runs = [] + +async def github_workflows_dashboard_sync(): + print('Started') + print('Updating table with recent workflow runs') + + if not GH_NUMBER_OF_WORKFLOW_RUNS_TO_FETCH or \ + not GH_NUMBER_OF_WORKFLOW_RUNS_TO_FETCH.isdigit(): + raise ValueError( + 'The number of workflow runs to fetch is not specified or not an integer' + ) + + database_operations(init_db_connection(), await fetch_workflow_data()) + + print('Done') + return "Completed" + +def init_db_connection(): + '''Init connection with the Database''' + connection = None + maxRetries = 3 + i = 0 + while connection is None and i < maxRetries: try: - while not completed: - url = "https://api.github.com/repos/apache/beam/actions/workflows" - queryOptions = { 'branch' : 'master', 'page': page, 'per_page' : GH_NUMBER_OF_WORKFLOWS } - response = requests.get(url = url, params = queryOptions) - retriesRequest(response) - jsonResponse = response.json() - if jsonResponse['total_count'] >= GH_NUMBER_OF_WORKFLOWS: - page = page + 1 - workflowsPage = jsonResponse['workflows'] - workflows.append(workflowsPage) - else: - completed = True - workflowsPage = jsonResponse['workflows'] - workflows.append(workflowsPage) - for pageItem in workflows: - for item in pageItem: - path = item['path'] - result = re.search('/(.*).yml', path) - path = (result.group(1)) + ".yml" - workflowObject = Workflow(item['id'],item['name'],path) - WORKFLOWS_OBJECT_LIST.append(workflowObject) - url = "https://api.github.com/repos/apache/beam/actions/workflows/" - queryOptions = { 'branch' : 'master', 'per_page' : GH_WORKFLOWS_NUMBER_EXECUTIONS, - 'page' :'1', 'exclude_pull_request':True } - for workflow in WORKFLOWS_OBJECT_LIST: - response = requests.get(url = "{}{}/runs".format(url,workflow.id), - params=queryOptions) - retriesRequest(response) - responseJson = response.json() - workflowsRuns = responseJson['workflow_runs'] - for item in workflowsRuns: - if item['status'] == 'completed': - workflow.runUrl.append(item['html_url']) - workflow.listOfRuns.append(item['conclusion']) - elif item['status'] != 'cancelled': - workflow.listOfRuns.append(item['status']) - workflow.runUrl.append(item['html_url']) - for i in range(0,GH_WORKFLOWS_NUMBER_EXECUTIONS): - if i >= len(workflow.listOfRuns): - workflow.listOfRuns.append('None') - workflow.runUrl.append('None') + connection = psycopg2.connect( + f"dbname='{DB_NAME}' user='{DB_USER_NAME}' host='{DB_HOST}'" + f" port='{DB_PORT}' password='{DB_PASSWORD}'") except Exception as e: - print('Failed to get GHA workflows') - print(e) - -def databaseOperations(connection,fetchWorkflows): - '''Create the table if not exist and update the table with the latest runs - of the workflows ''' - queryInsert = "INSERT INTO {} VALUES ".format(GH_WORKFLOWS_TABLE_NAME) - cursor = connection.cursor() - cursor.execute(GH_WORKFLOWS_CREATE_TABLE_QUERY) - cursor.execute("DELETE FROM {};".format(GH_WORKFLOWS_TABLE_NAME)) - query = "" - for workflow in WORKFLOWS_OBJECT_LIST: - rowInsert = "(\'{}\',\'{}\'".format(workflow.name,workflow.filename) - for run, runUrl in zip(workflow.listOfRuns,workflow.runUrl): - rowInsert += ",\'{}\',\'{}\'".format(run,runUrl) - query = query + rowInsert - query += ")," - query = query[:-1] + ";" - query = queryInsert + query - cursor.execute(query) - cursor.close() - connection.commit() - connection.close() + print('Failed to connect to DB; retrying in 1 minute') + print(e) + time.sleep(60) + i = i + 1 + if i >= maxRetries: + print("Number of retries exceded ") + sys.exit(1) + return connection + +def get_token(): + git_integration = GithubIntegration(GH_APP_ID, GH_PEM_KEY) + token = git_integration.get_access_token(GH_APP_INSTALLATION_ID).token + return f'Bearer {token}' + +@backoff.on_exception(backoff.constant, aiohttp.ClientResponseError, max_tries=5) +async def fetch(url, semaphore, params=None, headers=None, request_id=None): + async with semaphore: + async with aiohttp.ClientSession() as session: + async with session.get(url, params=params, headers=headers) as response: + if response.status == 200: + result = await response.json() + if request_id: + return request_id, result + return result + elif response.status == 403: + print(f'Retry for: {url}') + headers['Authorization'] = get_token() + raise aiohttp.ClientResponseError( + response.request_info, + response.history, + status=response.status, + message=response.reason, + headers=response.headers + ) + +async def fetch_workflow_data(): + def append_workflow_runs(workflow, runs): + for run in runs: + # Getting rid of all runs with a "skipped" status to display + # only actual runs + if run['conclusion'] != 'skipped': + status = '' + if run['status'] == 'completed': + status = run['conclusion'] + elif run['status'] != 'cancelled': + status = run['status'] + workflow.runs.append((int(run['id']), status, run['html_url'])) + + url = "https://api.github.com/repos/apache/beam/actions/workflows" + headers = {'Authorization': get_token()} + page = 1 + number_of_entries_per_page = 100 # The number of results per page (max 100) + params =\ + {'branch': 'master', 'page': page, 'per_page': number_of_entries_per_page} + concurrent_requests = 30 # Number of requests to send simultaneously + semaphore = asyncio.Semaphore(concurrent_requests) + + print("Start fetching recent workflow runs") + workflow_tasks = [] + response = await fetch(url, semaphore, params, headers) + pages_to_fetch =\ + math.ceil(response['total_count'] / number_of_entries_per_page) + while pages_to_fetch >= page: + params = { + 'branch': 'master', + 'page': page, + 'per_page': number_of_entries_per_page + } + workflow_tasks.append(fetch(url, semaphore, params, headers)) + page += 1 + + workflow_run_tasks = [] + for completed_task in asyncio.as_completed(workflow_tasks): + response = await completed_task + workflows = response.get('workflows', []) + for workflow in workflows: + runs_url = f"{url}/{workflow['id']}/runs" + page = 1 + pages_to_fetch = math.ceil( + int(GH_NUMBER_OF_WORKFLOW_RUNS_TO_FETCH) / number_of_entries_per_page + ) + while pages_to_fetch >= page: + params = { + 'branch': 'master', + 'page': page, + 'per_page': number_of_entries_per_page, + 'exclude_pull_requests': 'true' + } + workflow_run_tasks.append(fetch(runs_url, semaphore, params, headers)) + page += 1 + print("Successfully fetched workflow runs") + + print("Start fetching workflow runs details") + workflows = {} + workflow_ids_to_fetch_extra_runs = {} + for completed_task in asyncio.as_completed(workflow_run_tasks): + response = await completed_task + workflow_runs = response.get('workflow_runs') + if workflow_runs: + workflow_id = workflow_runs[0]['workflow_id'] + workflow = workflows.get(workflow_id) + if not workflow: + workflow_name = workflow_runs[0]['name'] + workflow_path = workflow_runs[0]['path'] + result = re.search(r'(workflows\/.*)$', workflow_path) + if result: + workflow_path = result.group(1) + workflow = Workflow(workflow_id, workflow_name, workflow_path) + + append_workflow_runs(workflow, workflow_runs) + workflows[workflow_id] = workflow + if len(workflow.runs) < int(GH_NUMBER_OF_WORKFLOW_RUNS_TO_FETCH): + workflow_ids_to_fetch_extra_runs[workflow_id] = workflow_id + else: + workflow_ids_to_fetch_extra_runs.pop(workflow_id, None) + print(f"Successfully fetched details for: {workflow.filename}") + + page = math.ceil( + int(GH_NUMBER_OF_WORKFLOW_RUNS_TO_FETCH) / number_of_entries_per_page + ) + 1 + # Fetch extra workflow runs if the specified number of runs is not reached + while workflow_ids_to_fetch_extra_runs: + extra_workflow_runs_tasks = [] + for workflow_id in list(workflow_ids_to_fetch_extra_runs.values()): + runs_url = f"{url}/{workflow_id}/runs" + params = { + 'branch': 'master', + 'page': page, + 'per_page': number_of_entries_per_page, + 'exclude_pull_requests': 'true' + } + extra_workflow_runs_tasks.append(fetch(runs_url, semaphore, params, headers, workflow_id)) + for completed_task in asyncio.as_completed(extra_workflow_runs_tasks): + workflow_id, response = await completed_task + workflow = workflows[workflow_id] + print(f"Fetching extra workflow runs for: {workflow.filename}") + workflow_runs = response.get('workflow_runs') + if workflow_runs: + append_workflow_runs(workflow, workflow_runs) + else: + number_of_runs_to_add =\ + int(GH_NUMBER_OF_WORKFLOW_RUNS_TO_FETCH) - len(workflow.runs) + workflow.runs.extend([(0, 'None', 'None')] * number_of_runs_to_add) + if len(workflow.runs) >= int(GH_NUMBER_OF_WORKFLOW_RUNS_TO_FETCH): + workflow_ids_to_fetch_extra_runs.pop(workflow_id, None) + print(f"Successfully fetched extra workflow runs for: {workflow.filename}") + page += 1 + print("Successfully fetched workflow runs details") + + for workflow in list(workflows.values()): + runs = sorted(workflow.runs, key=lambda r: r[0], reverse=True) + workflow.runs = runs[:int(GH_NUMBER_OF_WORKFLOW_RUNS_TO_FETCH)] + + return list(workflows.values()) + +def database_operations(connection, workflows): + # Create the table and update it with the latest workflow runs + if not workflows: + return + cursor = connection.cursor() + workflows_table_name = "github_workflows" + cursor.execute(f"DROP TABLE IF EXISTS {workflows_table_name};") + create_table_query = f""" + CREATE TABLE IF NOT EXISTS {workflows_table_name} ( + workflow_id integer NOT NULL PRIMARY KEY, + job_name text NOT NULL, + job_yml_filename text NOT NULL""" + for i in range(int(GH_NUMBER_OF_WORKFLOW_RUNS_TO_FETCH)): + create_table_query += f""", + run{i+1} text, + run{i+1}Id text""" + create_table_query += ")\n" + cursor.execute(create_table_query) + insert_query = f"INSERT INTO {workflows_table_name} VALUES " + for workflow in workflows: + row_insert =\ + f"(\'{workflow.id}\',\'{workflow.name}\',\'{workflow.filename}\'" + for _, status, url in workflow.runs: + row_insert += f",\'{status}\',\'{url}\'" + insert_query += f"{row_insert})," + insert_query = insert_query[:-1] + ";" + cursor.execute(insert_query) + cursor.close() + connection.commit() + connection.close() + +if __name__ == '__main__': + asyncio.run(github_workflows_dashboard_sync()) From e3c1591bbb8c2b91313267fcd39d77720c768b75 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 11 Oct 2023 09:27:52 -0400 Subject: [PATCH 097/435] Bump google.golang.org/grpc from 1.58.2 to 1.58.3 in /sdks (#28938) Bumps [google.golang.org/grpc](https://github.com/grpc/grpc-go) from 1.58.2 to 1.58.3. - [Release notes](https://github.com/grpc/grpc-go/releases) - [Commits](https://github.com/grpc/grpc-go/compare/v1.58.2...v1.58.3) --- updated-dependencies: - dependency-name: google.golang.org/grpc dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 2 +- sdks/go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 0bc60b630bf0..761798d57c9b 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -59,7 +59,7 @@ require ( golang.org/x/text v0.13.0 google.golang.org/api v0.146.0 google.golang.org/genproto v0.0.0-20230913181813-007df8e322eb - google.golang.org/grpc v1.58.2 + google.golang.org/grpc v1.58.3 google.golang.org/protobuf v1.31.0 gopkg.in/retry.v1 v1.0.3 gopkg.in/yaml.v2 v2.4.0 diff --git a/sdks/go.sum b/sdks/go.sum index 7743b2bc3247..ac862972b2ae 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -712,8 +712,8 @@ google.golang.org/grpc v1.26.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8 google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= google.golang.org/grpc v1.27.1/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= google.golang.org/grpc v1.33.2/go.mod h1:JMHMWHQWaTccqQQlmk3MJZS+GWXOdAesneDmEnv2fbc= -google.golang.org/grpc v1.58.2 h1:SXUpjxeVF3FKrTYQI4f4KvbGD5u2xccdYdurwowix5I= -google.golang.org/grpc v1.58.2/go.mod h1:tgX3ZQDlNJGU96V6yHh1T/JeoBQ2TXdr43YbYSsCJk0= +google.golang.org/grpc v1.58.3 h1:BjnpXut1btbtgN/6sp+brB2Kbm2LjNXnidYujAVbSoQ= +google.golang.org/grpc v1.58.3/go.mod h1:tgX3ZQDlNJGU96V6yHh1T/JeoBQ2TXdr43YbYSsCJk0= google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM= From 62efb96788455d8b8cddbe9e60335821bf735f65 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 11 Oct 2023 09:55:42 -0400 Subject: [PATCH 098/435] Bump golang.org/x/net from 0.16.0 to 0.17.0 in /sdks (#28939) Bumps [golang.org/x/net](https://github.com/golang/net) from 0.16.0 to 0.17.0. - [Commits](https://github.com/golang/net/compare/v0.16.0...v0.17.0) --- updated-dependencies: - dependency-name: golang.org/x/net dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 2 +- sdks/go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 761798d57c9b..e5baa554fe6d 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -52,7 +52,7 @@ require ( github.com/xitongsys/parquet-go v1.6.2 github.com/xitongsys/parquet-go-source v0.0.0-20220315005136-aec0fe3e777c go.mongodb.org/mongo-driver v1.12.1 - golang.org/x/net v0.16.0 + golang.org/x/net v0.17.0 golang.org/x/oauth2 v0.13.0 golang.org/x/sync v0.4.0 golang.org/x/sys v0.13.0 diff --git a/sdks/go.sum b/sdks/go.sum index ac862972b2ae..f1d0c53d9685 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -541,8 +541,8 @@ golang.org/x/net v0.0.0-20201224014010-6772e930b67b/go.mod h1:m0MpNAwzfU5UDzcl9v golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= golang.org/x/net v0.0.0-20211112202133-69e39bad7dc2/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= -golang.org/x/net v0.16.0 h1:7eBu7KsSvFDtSXUIDbh3aqlK4DPsZ1rByC8PFfBThos= -golang.org/x/net v0.16.0/go.mod h1:NxSsAGuq816PNPmqtQdLE42eU2Fs7NoRIZrHJAlaCOE= +golang.org/x/net v0.17.0 h1:pVaXccu2ozPjCXewfr1S7xza/zcXTity9cCdXQYSjIM= +golang.org/x/net v0.17.0/go.mod h1:NxSsAGuq816PNPmqtQdLE42eU2Fs7NoRIZrHJAlaCOE= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= From d437afab9615f2d3983da68cdb86a98209375ba6 Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Wed, 11 Oct 2023 10:10:19 -0400 Subject: [PATCH 099/435] Run self-assign and pr-updates only in create event (#28928) --- .github/workflows/pr-bot-pr-updates.yml | 1 + .github/workflows/self-assign.yml | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/.github/workflows/pr-bot-pr-updates.yml b/.github/workflows/pr-bot-pr-updates.yml index c882c18feeba..f3734e0adcbf 100644 --- a/.github/workflows/pr-bot-pr-updates.yml +++ b/.github/workflows/pr-bot-pr-updates.yml @@ -18,6 +18,7 @@ on: pull_request_target: types: ["synchronize"] # Synchronize is the action that runs after pushes to the user branch issue_comment: + types: [created] permissions: read-all jobs: process-pr-update: diff --git a/.github/workflows/self-assign.yml b/.github/workflows/self-assign.yml index c6b7cc69ce97..29ad240cd0dd 100644 --- a/.github/workflows/self-assign.yml +++ b/.github/workflows/self-assign.yml @@ -16,7 +16,7 @@ name: Assign or close an issue on: issue_comment: - + types: [created] jobs: assign: permissions: From a54afde4a05769f88826459168c9fa17c9bb00ae Mon Sep 17 00:00:00 2001 From: Danny McCormick Date: Wed, 11 Oct 2023 10:15:46 -0400 Subject: [PATCH 100/435] Fix missing backslash in arguments of job (#28942) This is causing repeated failures (e.g. https://github.com/apache/beam/actions/runs/6478627017/job/17590709062) --- ..._PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml b/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml index 18b57aad8057..44df23f0296e 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml @@ -87,10 +87,10 @@ jobs: with: gradle-command: :sdks:python:test-suites:dataflow:py${{steps.set_py_ver_clean.outputs.py_ver_clean}}:validatesContainer arguments: | - -PtestRCDependencies=true + -PtestRCDependencies=true \ -PpythonVersion=${{ matrix.python_version }} \ - name: Archive code coverage results uses: actions/upload-artifact@v3 with: name: python-code-coverage-report - path: "**/pytest*.xml" \ No newline at end of file + path: "**/pytest*.xml" From e56d1a1157bf4ea12f85a0a13f3f863d31ddff0e Mon Sep 17 00:00:00 2001 From: Jack McCluskey <34928439+jrmccluskey@users.noreply.github.com> Date: Wed, 11 Oct 2023 10:49:18 -0400 Subject: [PATCH 101/435] Enable missing-timeout pylint check (#28920) --- sdks/python/.pylintrc | 1 - .../portability/flink_uber_jar_job_server.py | 4 ++-- .../testing/analyzers/github_issues_utils.py | 20 +++++++++++++++---- .../benchmarks/nexmark/nexmark_launcher.py | 3 ++- .../load_tests/load_test_metrics_utils.py | 3 ++- 5 files changed, 22 insertions(+), 9 deletions(-) diff --git a/sdks/python/.pylintrc b/sdks/python/.pylintrc index e649e07e5ef6..a67f00ff2f82 100644 --- a/sdks/python/.pylintrc +++ b/sdks/python/.pylintrc @@ -116,7 +116,6 @@ disable = missing-docstring, modified-iterating-list, multiple-statements, - missing-timeout, #TODO(https://github.com/apache/beam/issues/28240) Enable and fix warnings no-self-use, no-else-break, no-else-continue, diff --git a/sdks/python/apache_beam/runners/portability/flink_uber_jar_job_server.py b/sdks/python/apache_beam/runners/portability/flink_uber_jar_job_server.py index 9a40a55c7601..3b302e334a5f 100644 --- a/sdks/python/apache_beam/runners/portability/flink_uber_jar_job_server.py +++ b/sdks/python/apache_beam/runners/portability/flink_uber_jar_job_server.py @@ -80,8 +80,8 @@ def executable_jar(self): return job_server.JavaJarJobServer.local_jar(url) def flink_version(self): - full_version = requests.get('%s/v1/config' % - self._master_url).json()['flink-version'] + full_version = requests.get( + '%s/v1/config' % self._master_url, timeout=60).json()['flink-version'] # Only return up to minor version. return '.'.join(full_version.split('.')[:2]) diff --git a/sdks/python/apache_beam/testing/analyzers/github_issues_utils.py b/sdks/python/apache_beam/testing/analyzers/github_issues_utils.py index bbcd2a8b11b5..cbbb9e5d3a2e 100644 --- a/sdks/python/apache_beam/testing/analyzers/github_issues_utils.py +++ b/sdks/python/apache_beam/testing/analyzers/github_issues_utils.py @@ -61,6 +61,8 @@ _AWAITING_TRIAGE_LABEL = 'awaiting triage' _PERF_ALERT_LABEL = 'perf-alert' +_REQUEST_TIMEOUT_SECS = 60 + def create_issue( title: str, @@ -89,7 +91,10 @@ def create_issue( if labels: data['labels'].extend(labels) # type: ignore response = requests.post( - url=url, data=json.dumps(data), headers=_HEADERS).json() + url=url, + data=json.dumps(data), + headers=_HEADERS, + timeout=_REQUEST_TIMEOUT_SECS).json() return response['number'], response['html_url'] @@ -118,7 +123,8 @@ def comment_on_issue(issue_number: int, 'issue_number': issue_number }, default=str), - headers=_HEADERS).json() + headers=_HEADERS, + timeout=_REQUEST_TIMEOUT_SECS).json() if open_issue_response['state'] == 'open': data = { 'owner': _GITHUB_REPO_OWNER, @@ -128,7 +134,10 @@ def comment_on_issue(issue_number: int, } response = requests.post( - open_issue_response['comments_url'], json.dumps(data), headers=_HEADERS) + open_issue_response['comments_url'], + json.dumps(data), + headers=_HEADERS, + timeout=_REQUEST_TIMEOUT_SECS) return True, response.json()['html_url'] return False, '' @@ -137,7 +146,10 @@ def add_awaiting_triage_label(issue_number: int): url = 'https://api.github.com/repos/{}/{}/issues/{}/labels'.format( _GITHUB_REPO_OWNER, _GITHUB_REPO_NAME, issue_number) requests.post( - url, json.dumps({'labels': [_AWAITING_TRIAGE_LABEL]}), headers=_HEADERS) + url, + json.dumps({'labels': [_AWAITING_TRIAGE_LABEL]}), + headers=_HEADERS, + timeout=_REQUEST_TIMEOUT_SECS) def get_issue_description( diff --git a/sdks/python/apache_beam/testing/benchmarks/nexmark/nexmark_launcher.py b/sdks/python/apache_beam/testing/benchmarks/nexmark/nexmark_launcher.py index 2296588ec496..bdf6f476212d 100644 --- a/sdks/python/apache_beam/testing/benchmarks/nexmark/nexmark_launcher.py +++ b/sdks/python/apache_beam/testing/benchmarks/nexmark/nexmark_launcher.py @@ -420,7 +420,8 @@ def publish_performance_influxdb(self, query_num, perf): auth = HTTPBasicAuth(user, password) try: - response = requests.post(url, params=query_str, data=payload, auth=auth) + response = requests.post( + url, params=query_str, data=payload, auth=auth, timeout=60) except requests.exceptions.RequestException as e: logging.warning('Failed to publish metrics to InfluxDB: ' + str(e)) else: diff --git a/sdks/python/apache_beam/testing/load_tests/load_test_metrics_utils.py b/sdks/python/apache_beam/testing/load_tests/load_test_metrics_utils.py index 01db2c114efb..1ff46a3f7d19 100644 --- a/sdks/python/apache_beam/testing/load_tests/load_test_metrics_utils.py +++ b/sdks/python/apache_beam/testing/load_tests/load_test_metrics_utils.py @@ -535,7 +535,8 @@ def publish(self, results): self.options.http_auth_enabled() else None try: - response = requests.post(url, params=query_str, data=payload, auth=auth) + response = requests.post( + url, params=query_str, data=payload, auth=auth, timeout=60) except requests.exceptions.RequestException as e: _LOGGER.warning('Failed to publish metrics to InfluxDB: ' + str(e)) else: From 9cdcb036cc904885254e8a58b75b32354f23ed6f Mon Sep 17 00:00:00 2001 From: martin trieu Date: Wed, 11 Oct 2023 08:28:49 -0700 Subject: [PATCH 102/435] =?UTF-8?q?move=20nested=20windmill=20state=20clas?= =?UTF-8?q?ses=20to=20top=20level=20files=20and=20organize=20di=E2=80=A6?= =?UTF-8?q?=20(#28506)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../worker/StreamingDataflowWorker.java | 2 + .../worker/StreamingModeExecutionContext.java | 483 ++- .../runners/dataflow/worker/Weighers.java | 2 +- .../worker/WindmillStateInternals.java | 2830 ----------------- .../worker/streaming/ActiveWorkState.java | 2 +- .../worker/streaming/ComputationState.java | 2 +- .../windmill/state/CachingStateTable.java | 275 ++ .../windmill/state/ConcatIterables.java | 55 + .../worker/windmill/state/IdTracker.java | 253 ++ .../worker/windmill/state/PagingIterable.java | 129 + .../worker/windmill/state/RangeCoder.java | 76 + .../worker/windmill/state/RangeSetCoder.java | 46 + .../windmill/state/SimpleWindmillState.java | 42 + .../worker/windmill/state/StateTag.java | 109 + .../state/TimestampedValueWithId.java | 42 + .../windmill/state/ToIterableFunction.java | 79 + .../windmill/state/ValuesAndContPosition.java | 46 + .../worker/windmill/state/WeightedList.java | 54 + .../worker/windmill/state/WindmillBag.java | 209 ++ .../state/WindmillCombiningState.java | 170 + .../worker/windmill/state/WindmillMap.java | 449 +++ .../windmill/state/WindmillMultimap.java | 732 +++++ .../windmill/state/WindmillOrderedList.java | 305 ++ .../worker/windmill/state/WindmillSet.java | 146 + .../worker/windmill/state/WindmillState.java | 68 + .../state}/WindmillStateCache.java | 338 +- .../state/WindmillStateInternals.java | 171 + .../state}/WindmillStateReader.java | 530 +-- .../windmill/state/WindmillStateUtil.java | 51 + .../worker/windmill/state/WindmillValue.java | 160 + .../windmill/state/WindmillWatermarkHold.java | 259 ++ .../worker/windmill/state/WrappedFuture.java | 64 + .../StreamingModeExecutionContextTest.java | 2 + .../worker/streaming/ActiveWorkStateTest.java | 2 +- .../state}/WindmillStateCacheTest.java | 135 +- .../state}/WindmillStateInternalsTest.java | 282 +- .../state}/WindmillStateReaderTest.java | 6 +- 37 files changed, 4725 insertions(+), 3881 deletions(-) delete mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillStateInternals.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/CachingStateTable.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/ConcatIterables.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/IdTracker.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/PagingIterable.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/RangeCoder.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/RangeSetCoder.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/SimpleWindmillState.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/StateTag.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/TimestampedValueWithId.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/ToIterableFunction.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/ValuesAndContPosition.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WeightedList.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillBag.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillCombiningState.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillMap.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillMultimap.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillOrderedList.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillSet.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillState.java rename runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/{ => windmill/state}/WindmillStateCache.java (92%) create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateInternals.java rename runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/{ => windmill/state}/WindmillStateReader.java (70%) create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateUtil.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillValue.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillWatermarkHold.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WrappedFuture.java rename runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/{ => windmill/state}/WindmillStateCacheTest.java (75%) rename runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/{ => windmill/state}/WindmillStateInternalsTest.java (97%) rename runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/{ => windmill/state}/WindmillStateReaderTest.java (99%) diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java index 5d4c0288c838..32b272c67280 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java @@ -106,6 +106,8 @@ import org.apache.beam.runners.dataflow.worker.windmill.WindmillStream.CommitWorkStream; import org.apache.beam.runners.dataflow.worker.windmill.WindmillStream.GetWorkStream; import org.apache.beam.runners.dataflow.worker.windmill.WindmillStreamPool; +import org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateCache; +import org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateReader; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.extensions.gcp.util.Transport; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java index 3f4cb0893755..c8fa6e6dfb78 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java @@ -49,6 +49,9 @@ import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GlobalDataId; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GlobalDataRequest; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.Timer; +import org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateCache; +import org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateInternals; +import org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateReader; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.io.UnboundedSource; import org.apache.beam.sdk.metrics.MetricsContainer; @@ -82,7 +85,12 @@ public class StreamingModeExecutionContext extends DataflowExecutionContext { private static final Logger LOG = LoggerFactory.getLogger(StreamingModeExecutionContext.class); - + private final String computationId; + private final Map, Map> sideInputCache; + // Per-key cache of active Reader objects in use by this process. + private final ImmutableMap stateNameMap; + private final WindmillStateCache.ForComputation stateCache; + private final ReaderCache readerCache; /** * The current user-facing key for this execution context. * @@ -94,20 +102,12 @@ public class StreamingModeExecutionContext extends DataflowExecutionContext, Map> sideInputCache; - - // Per-key cache of active Reader objects in use by this process. - private final ImmutableMap stateNameMap; - private final WindmillStateCache.ForComputation stateCache; - private Windmill.WorkItem work; private WindmillComputationKey computationKey; private StateFetcher stateFetcher; private Windmill.WorkItemCommitRequest.Builder outputBuilder; private UnboundedSource.UnboundedReader activeReader; private volatile long backlogBytes; - private final ReaderCache readerCache; public StreamingModeExecutionContext( CounterFactory counterFactory, @@ -133,86 +133,6 @@ public StreamingModeExecutionContext( this.backlogBytes = UnboundedSource.UnboundedReader.BACKLOG_UNKNOWN; } - /** - * Execution states in Streaming are shared between multiple map-task executors. Thus this class - * needs to be thread safe for multiple writers. A single stage could have have multiple executors - * running concurrently. - */ - public static class StreamingModeExecutionState - extends DataflowOperationContext.DataflowExecutionState { - - // AtomicLong is used because this value is written in two places: - // 1. The sampling thread calls takeSample to increment the time spent in this state - // 2. The reporting thread calls extractUpdate which reads the current sum *AND* sets it to 0. - private final AtomicLong totalMillisInState = new AtomicLong(); - - // The worker that created this state. Used to report lulls back to the worker. - @SuppressWarnings("unused") // Affects a public api - private final StreamingDataflowWorker worker; - - public StreamingModeExecutionState( - NameContext nameContext, - String stateName, - MetricsContainer metricsContainer, - ProfileScope profileScope, - StreamingDataflowWorker worker) { - // TODO: Take in the requesting step name and side input index for streaming. - super(nameContext, stateName, null, null, metricsContainer, profileScope); - this.worker = worker; - } - - /** - * Take sample is only called by the ExecutionStateSampler thread. It is the only place that - * increments totalMillisInState, however the reporting thread periodically calls extractUpdate - * which will read the sum and reset it to 0, so totalMillisInState does have multiple writers. - */ - @Override - public void takeSample(long millisSinceLastSample) { - totalMillisInState.addAndGet(millisSinceLastSample); - } - - /** - * Extract updates in the form of a {@link CounterUpdate}. - * - *

Non-final updates are extracted periodically and report the physical value as a delta. - * This requires setting the totalMillisInState back to 0. - * - *

Final updates should never be requested from a Streaming job since the work unit never - * completes. - */ - @Override - public @Nullable CounterUpdate extractUpdate(boolean isFinalUpdate) { - // Streaming reports deltas, so isFinalUpdate doesn't matter, and should never be true. - long sum = totalMillisInState.getAndSet(0); - return sum == 0 ? null : createUpdate(false, sum); - } - } - - /** - * Implementation of DataflowExecutionStateRegistry that creates Streaming versions of - * ExecutionState. - */ - public static class StreamingModeExecutionStateRegistry extends DataflowExecutionStateRegistry { - - private final StreamingDataflowWorker worker; - - public StreamingModeExecutionStateRegistry(StreamingDataflowWorker worker) { - this.worker = worker; - } - - @Override - protected DataflowOperationContext.DataflowExecutionState createState( - NameContext nameContext, - String stateName, - String requestingStepName, - Integer inputIndex, - MetricsContainer container, - ProfileScope profileScope) { - return new StreamingModeExecutionState( - nameContext, stateName, container, profileScope, worker); - } - } - @VisibleForTesting public long getBacklogBytes() { return backlogBytes; @@ -304,11 +224,8 @@ protected SideInputReader getSideInputReaderForViews( String stateFamily, StateFetcher.SideInputState state, Supplier scopedReadStateSupplier) { - Map tagCache = sideInputCache.get(view.getTagInternal()); - if (tagCache == null) { - tagCache = new HashMap<>(); - sideInputCache.put(view.getTagInternal(), tagCache); - } + Map tagCache = + sideInputCache.computeIfAbsent(view.getTagInternal(), k -> new HashMap<>()); if (tagCache.containsKey(sideInputWindow)) { @SuppressWarnings("unchecked") @@ -455,6 +372,10 @@ public Map flushState() { return callbacks; } + String getStateFamily(NameContext nameContext) { + return nameContext.userName() == null ? null : stateNameMap.get(nameContext.userName()); + } + interface StreamingModeStepContext { boolean issueSideInputFetch( @@ -478,8 +399,84 @@ void writePCollectionViewData( throws IOException; } - String getStateFamily(NameContext nameContext) { - return nameContext.userName() == null ? null : stateNameMap.get(nameContext.userName()); + /** + * Execution states in Streaming are shared between multiple map-task executors. Thus this class + * needs to be thread safe for multiple writers. A single stage could have have multiple executors + * running concurrently. + */ + public static class StreamingModeExecutionState + extends DataflowOperationContext.DataflowExecutionState { + + // AtomicLong is used because this value is written in two places: + // 1. The sampling thread calls takeSample to increment the time spent in this state + // 2. The reporting thread calls extractUpdate which reads the current sum *AND* sets it to 0. + private final AtomicLong totalMillisInState = new AtomicLong(); + + // The worker that created this state. Used to report lulls back to the worker. + @SuppressWarnings("unused") // Affects a public api + private final StreamingDataflowWorker worker; + + public StreamingModeExecutionState( + NameContext nameContext, + String stateName, + MetricsContainer metricsContainer, + ProfileScope profileScope, + StreamingDataflowWorker worker) { + // TODO: Take in the requesting step name and side input index for streaming. + super(nameContext, stateName, null, null, metricsContainer, profileScope); + this.worker = worker; + } + + /** + * Take sample is only called by the ExecutionStateSampler thread. It is the only place that + * increments totalMillisInState, however the reporting thread periodically calls extractUpdate + * which will read the sum and reset it to 0, so totalMillisInState does have multiple writers. + */ + @Override + public void takeSample(long millisSinceLastSample) { + totalMillisInState.addAndGet(millisSinceLastSample); + } + + /** + * Extract updates in the form of a {@link CounterUpdate}. + * + *

Non-final updates are extracted periodically and report the physical value as a delta. + * This requires setting the totalMillisInState back to 0. + * + *

Final updates should never be requested from a Streaming job since the work unit never + * completes. + */ + @Override + public @Nullable CounterUpdate extractUpdate(boolean isFinalUpdate) { + // Streaming reports deltas, so isFinalUpdate doesn't matter, and should never be true. + long sum = totalMillisInState.getAndSet(0); + return sum == 0 ? null : createUpdate(false, sum); + } + } + + /** + * Implementation of DataflowExecutionStateRegistry that creates Streaming versions of + * ExecutionState. + */ + public static class StreamingModeExecutionStateRegistry extends DataflowExecutionStateRegistry { + + private final StreamingDataflowWorker worker; + + public StreamingModeExecutionStateRegistry(StreamingDataflowWorker worker) { + this.worker = worker; + } + + @Override + protected DataflowOperationContext.DataflowExecutionState createState( + NameContext nameContext, + String stateName, + String requestingStepName, + Integer inputIndex, + MetricsContainer container, + ProfileScope profileScope) { + return new StreamingModeExecutionState( + nameContext, stateName, container, profileScope, worker); + } } private static class ScopedReadStateSupplier implements Supplier { @@ -501,15 +498,156 @@ public Closeable get() { } } + /** + * A specialized {@link StepContext} that uses provided {@link StateInternals} and {@link + * TimerInternals} for user state and timers. + */ + private static class UserStepContext extends DataflowStepContext + implements StreamingModeStepContext { + + private final StreamingModeExecutionContext.StepContext wrapped; + + public UserStepContext(StreamingModeExecutionContext.StepContext wrapped) { + super(wrapped.getNameContext()); + this.wrapped = wrapped; + } + + @Override + public boolean issueSideInputFetch( + PCollectionView view, BoundedWindow w, StateFetcher.SideInputState s) { + return wrapped.issueSideInputFetch(view, w, s); + } + + @Override + public void addBlockingSideInput(GlobalDataRequest blocked) { + wrapped.addBlockingSideInput(blocked); + } + + @Override + public void addBlockingSideInputs(Iterable blocked) { + wrapped.addBlockingSideInputs(blocked); + } + + @Override + public StateInternals stateInternals() { + return wrapped.stateInternals(); + } + + @Override + public Iterable getSideInputNotifications() { + return wrapped.getSideInputNotifications(); + } + + @Override + public void writePCollectionViewData( + TupleTag tag, + Iterable data, + Coder> dataCoder, + W window, + Coder windowCoder) + throws IOException { + throw new IllegalStateException("User DoFns cannot write PCollectionView data"); + } + + @Override + public TimerInternals timerInternals() { + return wrapped.userTimerInternals(); + } + + @Override + public TimerData getNextFiredTimer(Coder windowCoder) { + return wrapped.getNextFiredUserTimer(windowCoder); + } + + @Override + public void setStateCleanupTimer( + String timerId, + W window, + Coder windowCoder, + Instant cleanupTime, + Instant cleanupOutputTimestamp) { + throw new UnsupportedOperationException( + String.format( + "setStateCleanupTimer should not be called on %s, only on a system %s", + getClass().getSimpleName(), + StreamingModeExecutionContext.StepContext.class.getSimpleName())); + } + + @Override + public DataflowStepContext namespacedToUser() { + return this; + } + } + + /** A {@link SideInputReader} that fetches side inputs from the streaming worker's cache. */ + public static class StreamingModeSideInputReader implements SideInputReader { + + private final StreamingModeExecutionContext context; + private final Set> viewSet; + + private StreamingModeSideInputReader( + Iterable> views, StreamingModeExecutionContext context) { + this.context = context; + this.viewSet = ImmutableSet.copyOf(views); + } + + public static StreamingModeSideInputReader of( + Iterable> views, StreamingModeExecutionContext context) { + return new StreamingModeSideInputReader(views, context); + } + + @Override + public T get(PCollectionView view, BoundedWindow window) { + if (!contains(view)) { + throw new RuntimeException("get() called with unknown view"); + } + + // We are only fetching the cached value here, so we don't need stateFamily or + // readStateSupplier. + return context + .fetchSideInput( + view, + window, + null /* unused stateFamily */, + StateFetcher.SideInputState.CACHED_IN_WORKITEM, + null /* unused readStateSupplier */) + .orNull(); + } + + @Override + public boolean contains(PCollectionView view) { + return viewSet.contains(view); + } + + @Override + public boolean isEmpty() { + return viewSet.isEmpty(); + } + } + class StepContext extends DataflowExecutionContext.DataflowStepContext implements StreamingModeStepContext { + private final String stateFamily; + private final Supplier scopedReadStateSupplier; private WindmillStateInternals stateInternals; - private WindmillTimerInternals systemTimerInternals; private WindmillTimerInternals userTimerInternals; - private final String stateFamily; - private final Supplier scopedReadStateSupplier; + // Lazily initialized + private Iterator cachedFiredSystemTimers = null; + // Lazily initialized + private PeekingIterator cachedFiredUserTimers = null; + // An ordered list of any timers that were set or modified by user processing earlier in this + // bundle. + // We use a NavigableSet instead of a priority queue to prevent duplicate elements from ending + // up in the queue. + private NavigableSet modifiedUserEventTimersOrdered = null; + private NavigableSet modifiedUserProcessingTimersOrdered = null; + private NavigableSet modifiedUserSynchronizedProcessingTimersOrdered = null; + // A list of timer keys that were modified by user processing earlier in this bundle. This + // serves a tombstone, so + // that we know not to fire any bundle tiemrs that were moddified. + private Table modifiedUserTimerKeys = null; public StepContext(DataflowOperationContext operationContext) { super(operationContext.nameContext()); @@ -570,14 +708,11 @@ public void flushState() { userTimerInternals.persistTo(outputBuilder); } - // Lazily initialized - private Iterator cachedFiredSystemTimers = null; - @Override public TimerData getNextFiredTimer(Coder windowCoder) { if (cachedFiredSystemTimers == null) { cachedFiredSystemTimers = - FluentIterable.from(StreamingModeExecutionContext.this.getFiredTimers()) + FluentIterable.from(StreamingModeExecutionContext.this.getFiredTimers()) .filter( timer -> WindmillTimerInternals.isSystemTimer(timer) @@ -601,16 +736,6 @@ public TimerData getNextFiredTimer(Coder windowCode return nextTimer; } - // Lazily initialized - private PeekingIterator cachedFiredUserTimers = null; - // An ordered list of any timers that were set or modified by user processing earlier in this - // bundle. - // We use a NavigableSet instead of a priority queue to prevent duplicate elements from ending - // up in the queue. - private NavigableSet modifiedUserEventTimersOrdered = null; - private NavigableSet modifiedUserProcessingTimersOrdered = null; - private NavigableSet modifiedUserSynchronizedProcessingTimersOrdered = null; - private NavigableSet getModifiedUserTimersOrdered(TimeDomain timeDomain) { switch (timeDomain) { case EVENT_TIME: @@ -624,11 +749,6 @@ private NavigableSet getModifiedUserTimersOrdered(TimeDomain timeDoma } } - // A list of timer keys that were modified by user processing earlier in this bundle. This - // serves a tombstone, so - // that we know not to fire any bundle tiemrs that were moddified. - private Table modifiedUserTimerKeys = null; - private void onUserTimerModified(TimerData timerData) { if (!timerData.getDeleted()) { getModifiedUserTimersOrdered(timerData.getDomain()).add(timerData); @@ -804,131 +924,4 @@ public TimerInternals userTimerInternals() { return checkNotNull(userTimerInternals); } } - - /** - * A specialized {@link StepContext} that uses provided {@link StateInternals} and {@link - * TimerInternals} for user state and timers. - */ - private static class UserStepContext extends DataflowStepContext - implements StreamingModeStepContext { - - private final StreamingModeExecutionContext.StepContext wrapped; - - public UserStepContext(StreamingModeExecutionContext.StepContext wrapped) { - super(wrapped.getNameContext()); - this.wrapped = wrapped; - } - - @Override - public boolean issueSideInputFetch( - PCollectionView view, BoundedWindow w, StateFetcher.SideInputState s) { - return wrapped.issueSideInputFetch(view, w, s); - } - - @Override - public void addBlockingSideInput(GlobalDataRequest blocked) { - wrapped.addBlockingSideInput(blocked); - } - - @Override - public void addBlockingSideInputs(Iterable blocked) { - wrapped.addBlockingSideInputs(blocked); - } - - @Override - public StateInternals stateInternals() { - return wrapped.stateInternals(); - } - - @Override - public Iterable getSideInputNotifications() { - return wrapped.getSideInputNotifications(); - } - - @Override - public void writePCollectionViewData( - TupleTag tag, - Iterable data, - Coder> dataCoder, - W window, - Coder windowCoder) - throws IOException { - throw new IllegalStateException("User DoFns cannot write PCollectionView data"); - } - - @Override - public TimerInternals timerInternals() { - return wrapped.userTimerInternals(); - } - - @Override - public TimerData getNextFiredTimer(Coder windowCoder) { - return wrapped.getNextFiredUserTimer(windowCoder); - } - - @Override - public void setStateCleanupTimer( - String timerId, - W window, - Coder windowCoder, - Instant cleanupTime, - Instant cleanupOutputTimestamp) { - throw new UnsupportedOperationException( - String.format( - "setStateCleanupTimer should not be called on %s, only on a system %s", - getClass().getSimpleName(), - StreamingModeExecutionContext.StepContext.class.getSimpleName())); - } - - @Override - public DataflowStepContext namespacedToUser() { - return this; - } - } - - /** A {@link SideInputReader} that fetches side inputs from the streaming worker's cache. */ - public static class StreamingModeSideInputReader implements SideInputReader { - - private StreamingModeExecutionContext context; - private Set> viewSet; - - private StreamingModeSideInputReader( - Iterable> views, StreamingModeExecutionContext context) { - this.context = context; - this.viewSet = ImmutableSet.copyOf(views); - } - - public static StreamingModeSideInputReader of( - Iterable> views, StreamingModeExecutionContext context) { - return new StreamingModeSideInputReader(views, context); - } - - @Override - public T get(PCollectionView view, BoundedWindow window) { - if (!contains(view)) { - throw new RuntimeException("get() called with unknown view"); - } - - // We are only fetching the cached value here, so we don't need stateFamily or - // readStateSupplier. - return context - .fetchSideInput( - view, - window, - null /* unused stateFamily */, - StateFetcher.SideInputState.CACHED_IN_WORKITEM, - null /* unused readStateSupplier */) - .orNull(); - } - - @Override - public boolean contains(PCollectionView view) { - return viewSet.contains(view); - } - - @Override - public boolean isEmpty() { - return viewSet.isEmpty(); - } - } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/Weighers.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/Weighers.java index d2231b8b47bb..eb4e0f4885a7 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/Weighers.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/Weighers.java @@ -28,7 +28,7 @@ *

Package-private here so that the dependency on Guava does not leak into the public API * surface. */ -class Weighers { +public class Weighers { public static Weigher fixedWeightKeys(final int keyWeight) { return (key, value) -> (int) Math.min(keyWeight + value.getWeight(), Integer.MAX_VALUE); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillStateInternals.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillStateInternals.java deleted file mode 100644 index d4edc0afc0b1..000000000000 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillStateInternals.java +++ /dev/null @@ -1,2830 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.runners.dataflow.worker; - -import com.google.auto.value.AutoValue; -import java.io.Closeable; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.io.OutputStreamWriter; -import java.nio.charset.StandardCharsets; -import java.util.AbstractMap; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.Comparator; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.Objects; -import java.util.Random; -import java.util.Set; -import java.util.SortedSet; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.Future; -import java.util.function.BiConsumer; -import java.util.function.Function; -import java.util.stream.Collectors; -import javax.annotation.concurrent.NotThreadSafe; -import org.apache.beam.repackaged.core.org.apache.commons.lang3.tuple.Triple; -import org.apache.beam.runners.core.StateInternals; -import org.apache.beam.runners.core.StateNamespace; -import org.apache.beam.runners.core.StateTable; -import org.apache.beam.runners.core.StateTag; -import org.apache.beam.runners.core.StateTag.StateBinder; -import org.apache.beam.runners.core.StateTags; -import org.apache.beam.runners.dataflow.worker.windmill.Windmill; -import org.apache.beam.runners.dataflow.worker.windmill.Windmill.SortedListEntry; -import org.apache.beam.runners.dataflow.worker.windmill.Windmill.SortedListRange; -import org.apache.beam.runners.dataflow.worker.windmill.Windmill.TagSortedListDeleteRequest; -import org.apache.beam.runners.dataflow.worker.windmill.Windmill.TagSortedListInsertRequest; -import org.apache.beam.runners.dataflow.worker.windmill.Windmill.TagSortedListUpdateRequest; -import org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkItemCommitRequest; -import org.apache.beam.sdk.coders.BooleanCoder; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.Coder.Context; -import org.apache.beam.sdk.coders.CoderException; -import org.apache.beam.sdk.coders.CustomCoder; -import org.apache.beam.sdk.coders.InstantCoder; -import org.apache.beam.sdk.coders.MapCoder; -import org.apache.beam.sdk.coders.NullableCoder; -import org.apache.beam.sdk.coders.SetCoder; -import org.apache.beam.sdk.coders.StructuredCoder; -import org.apache.beam.sdk.coders.VarLongCoder; -import org.apache.beam.sdk.state.BagState; -import org.apache.beam.sdk.state.CombiningState; -import org.apache.beam.sdk.state.MapState; -import org.apache.beam.sdk.state.MultimapState; -import org.apache.beam.sdk.state.OrderedListState; -import org.apache.beam.sdk.state.ReadableState; -import org.apache.beam.sdk.state.ReadableStates; -import org.apache.beam.sdk.state.SetState; -import org.apache.beam.sdk.state.State; -import org.apache.beam.sdk.state.StateContext; -import org.apache.beam.sdk.state.StateContexts; -import org.apache.beam.sdk.state.ValueState; -import org.apache.beam.sdk.state.WatermarkHoldState; -import org.apache.beam.sdk.transforms.Combine.CombineFn; -import org.apache.beam.sdk.transforms.CombineWithContext.CombineFnWithContext; -import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; -import org.apache.beam.sdk.util.ByteStringOutputStream; -import org.apache.beam.sdk.util.CombineFnUtil; -import org.apache.beam.sdk.util.Weighted; -import org.apache.beam.sdk.values.TimestampedValue; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Optional; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Supplier; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.BoundType; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterators; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Range; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.RangeSet; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Sets; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.TreeRangeSet; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.Futures; -import org.checkerframework.checker.initialization.qual.Initialized; -import org.checkerframework.checker.nullness.qual.NonNull; -import org.checkerframework.checker.nullness.qual.Nullable; -import org.checkerframework.checker.nullness.qual.UnknownKeyFor; -import org.joda.time.Duration; -import org.joda.time.Instant; - -/** Implementation of {@link StateInternals} using Windmill to manage the underlying data. */ -@SuppressWarnings({ - "rawtypes", // TODO(https://github.com/apache/beam/issues/20447) - "nullness" // TODO(https://github.com/apache/beam/issues/20497) -}) -class WindmillStateInternals implements StateInternals { - - /** - * The key will be null when not in a keyed context, from the users perspective. There is still a - * "key" for the Windmill computation, but it cannot be meaningfully deserialized. - */ - private final @Nullable K key; - - @Override - public @Nullable K getKey() { - return key; - } - - private static class CachingStateTable extends StateTable { - private final String stateFamily; - private final WindmillStateReader reader; - private final WindmillStateCache.ForKeyAndFamily cache; - private final boolean isSystemTable; - boolean isNewKey; - private final Supplier scopedReadStateSupplier; - private final StateTable derivedStateTable; - - public CachingStateTable( - @Nullable K key, - String stateFamily, - WindmillStateReader reader, - WindmillStateCache.ForKeyAndFamily cache, - boolean isSystemTable, - boolean isNewKey, - Supplier scopedReadStateSupplier, - StateTable derivedStateTable) { - this.stateFamily = stateFamily; - this.reader = reader; - this.cache = cache; - this.isSystemTable = isSystemTable; - this.isNewKey = isNewKey; - this.scopedReadStateSupplier = scopedReadStateSupplier; - this.derivedStateTable = derivedStateTable != null ? derivedStateTable : this; - } - - @Override - protected StateBinder binderForNamespace( - final StateNamespace namespace, final StateContext c) { - // Look up state objects in the cache or create new ones if not found. The state will - // be added to the cache in persist(). - return new StateBinder() { - @Override - public BagState bindBag(StateTag> address, Coder elemCoder) { - if (isSystemTable) { - address = StateTags.makeSystemTagInternal(address); - } - WindmillBag result = (WindmillBag) cache.get(namespace, address); - if (result == null) { - result = new WindmillBag<>(namespace, address, stateFamily, elemCoder, isNewKey); - } - result.initializeForWorkItem(reader, scopedReadStateSupplier); - return result; - } - - @Override - public SetState bindSet(StateTag> spec, Coder elemCoder) { - WindmillSet result = - new WindmillSet(namespace, spec, stateFamily, elemCoder, cache, isNewKey); - result.initializeForWorkItem(reader, scopedReadStateSupplier); - return result; - } - - @Override - public MapState bindMap( - StateTag> spec, Coder keyCoder, Coder valueCoder) { - WindmillMap result = (WindmillMap) cache.get(namespace, spec); - if (result == null) { - result = - new WindmillMap( - namespace, spec, stateFamily, keyCoder, valueCoder, isNewKey); - } - result.initializeForWorkItem(reader, scopedReadStateSupplier); - return result; - } - - @Override - public MultimapState bindMultimap( - StateTag> spec, - Coder keyCoder, - Coder valueCoder) { - WindmillMultimap result = - (WindmillMultimap) cache.get(namespace, spec); - if (result == null) { - result = - new WindmillMultimap<>( - namespace, spec, stateFamily, keyCoder, valueCoder, isNewKey); - } - result.initializeForWorkItem(reader, scopedReadStateSupplier); - return result; - } - - @Override - public OrderedListState bindOrderedList( - StateTag> spec, Coder elemCoder) { - if (isSystemTable) { - spec = StateTags.makeSystemTagInternal(spec); - } - WindmillOrderedList result = (WindmillOrderedList) cache.get(namespace, spec); - if (result == null) { - result = - new WindmillOrderedList<>( - derivedStateTable, namespace, spec, stateFamily, elemCoder, isNewKey); - } - result.initializeForWorkItem(reader, scopedReadStateSupplier); - return result; - } - - @Override - public WatermarkHoldState bindWatermark( - StateTag address, TimestampCombiner timestampCombiner) { - if (isSystemTable) { - address = StateTags.makeSystemTagInternal(address); - } - WindmillWatermarkHold result = (WindmillWatermarkHold) cache.get(namespace, address); - if (result == null) { - result = - new WindmillWatermarkHold( - namespace, address, stateFamily, timestampCombiner, isNewKey); - } - result.initializeForWorkItem(reader, scopedReadStateSupplier); - return result; - } - - @Override - public CombiningState bindCombiningValue( - StateTag> address, - Coder accumCoder, - CombineFn combineFn) { - if (isSystemTable) { - address = StateTags.makeSystemTagInternal(address); - } - WindmillCombiningState result = - new WindmillCombiningState<>( - namespace, address, stateFamily, accumCoder, combineFn, cache, isNewKey); - result.initializeForWorkItem(reader, scopedReadStateSupplier); - return result; - } - - @Override - public - CombiningState bindCombiningValueWithContext( - StateTag> address, - Coder accumCoder, - CombineFnWithContext combineFn) { - if (isSystemTable) { - address = StateTags.makeSystemTagInternal(address); - } - return bindCombiningValue(address, accumCoder, CombineFnUtil.bindContext(combineFn, c)); - } - - @Override - public ValueState bindValue(StateTag> address, Coder coder) { - if (isSystemTable) { - address = StateTags.makeSystemTagInternal(address); - } - WindmillValue result = (WindmillValue) cache.get(namespace, address); - if (result == null) { - result = new WindmillValue<>(namespace, address, stateFamily, coder, isNewKey); - } - result.initializeForWorkItem(reader, scopedReadStateSupplier); - return result; - } - }; - } - } - - private WindmillStateCache.ForKeyAndFamily cache; - Supplier scopedReadStateSupplier; - private StateTable workItemState; - private StateTable workItemDerivedState; - - public WindmillStateInternals( - @Nullable K key, - String stateFamily, - WindmillStateReader reader, - boolean isNewKey, - WindmillStateCache.ForKeyAndFamily cache, - Supplier scopedReadStateSupplier) { - this.key = key; - this.cache = cache; - this.scopedReadStateSupplier = scopedReadStateSupplier; - this.workItemDerivedState = - new CachingStateTable<>( - key, stateFamily, reader, cache, true, isNewKey, scopedReadStateSupplier, null); - this.workItemState = - new CachingStateTable<>( - key, - stateFamily, - reader, - cache, - false, - isNewKey, - scopedReadStateSupplier, - workItemDerivedState); - } - - private void persist(List> commitsToMerge, StateTable stateTable) { - for (State location : stateTable.values()) { - if (!(location instanceof WindmillState)) { - throw new IllegalStateException( - String.format( - "%s wasn't created by %s -- unable to persist it", - location.getClass().getSimpleName(), getClass().getSimpleName())); - } - - try { - commitsToMerge.add(((WindmillState) location).persist(cache)); - } catch (IOException e) { - throw new RuntimeException("Unable to persist state", e); - } - } - - // All cached State objects now have known values. - // Clear any references to the underlying reader to prevent space leaks. - // The next work unit to use these cached State objects will reset the - // reader to a current reader in case those values are modified. - for (State location : stateTable.values()) { - ((WindmillState) location).cleanupAfterWorkItem(); - } - - // Clear out the map of already retrieved state instances. - stateTable.clear(); - } - - public void persist(final Windmill.WorkItemCommitRequest.Builder commitBuilder) { - List> commitsToMerge = new ArrayList<>(); - - // Call persist on each first, which may schedule some futures for reading. - persist(commitsToMerge, workItemState); - persist(commitsToMerge, workItemDerivedState); - - try (Closeable scope = scopedReadStateSupplier.get()) { - for (Future commitFuture : commitsToMerge) { - commitBuilder.mergeFrom(commitFuture.get()); - } - } catch (ExecutionException | InterruptedException | IOException exc) { - if (exc instanceof InterruptedException) { - Thread.currentThread().interrupt(); - } - throw new RuntimeException("Failed to retrieve Windmill state during persist()", exc); - } - - cache.persist(); - } - - /** Encodes the given namespace and address as {@code <namespace>+<address>}. */ - @VisibleForTesting - static ByteString encodeKey(StateNamespace namespace, StateTag address) { - try { - // Use ByteStringOutputStream rather than concatenation and String.format. We build these keys - // a lot, and this leads to better performance results. See associated benchmarks. - ByteStringOutputStream stream = new ByteStringOutputStream(); - OutputStreamWriter writer = new OutputStreamWriter(stream, StandardCharsets.UTF_8); - - // stringKey starts and ends with a slash. We separate it from the - // StateTag ID by a '+' (which is guaranteed not to be in the stringKey) because the - // ID comes from the user. - namespace.appendTo(writer); - writer.write('+'); - address.appendTo(writer); - writer.flush(); - return stream.toByteString(); - } catch (IOException e) { - throw new RuntimeException(e); - } - } - - /** - * Abstract base class for all Windmill state. - * - *

Note that these are not thread safe; each state object is associated with a key and thus - * only accessed by a single thread at once. - */ - @NotThreadSafe - private abstract static class WindmillState { - protected Supplier scopedReadStateSupplier; - protected WindmillStateReader reader; - - /** - * Return an asynchronously computed {@link WorkItemCommitRequest}. The request should be of a - * form that can be merged with others (only add to repeated fields). - */ - abstract Future persist(WindmillStateCache.ForKeyAndFamily cache) - throws IOException; - - /** - * Prepare this (possibly reused from cache) state for reading from {@code reader} if needed. - */ - void initializeForWorkItem( - WindmillStateReader reader, Supplier scopedReadStateSupplier) { - this.reader = reader; - this.scopedReadStateSupplier = scopedReadStateSupplier; - } - - /** - * This (now cached) state should never need to interact with the reader until the next work - * item. Clear it to prevent space leaks. The reader will be reset by {@link - * #initializeForWorkItem} upon the next work item. - */ - void cleanupAfterWorkItem() { - this.reader = null; - this.scopedReadStateSupplier = null; - } - - Closeable scopedReadState() { - return scopedReadStateSupplier.get(); - } - } - - /** - * Base class for implementations of {@link WindmillState} where the {@link #persist} call does - * not require any asynchronous reading. - */ - private abstract static class SimpleWindmillState extends WindmillState { - @Override - public final Future persist(WindmillStateCache.ForKeyAndFamily cache) - throws IOException { - return Futures.immediateFuture(persistDirectly(cache)); - } - - /** - * Returns a {@link WorkItemCommitRequest} that can be used to persist this state to Windmill. - */ - protected abstract WorkItemCommitRequest persistDirectly( - WindmillStateCache.ForKeyAndFamily cache) throws IOException; - } - - @Override - public T state(StateNamespace namespace, StateTag address) { - return workItemState.get(namespace, address, StateContexts.nullContext()); - } - - @Override - public T state( - StateNamespace namespace, StateTag address, StateContext c) { - return workItemState.get(namespace, address, c); - } - - private static class WindmillValue extends SimpleWindmillState implements ValueState { - private final StateNamespace namespace; - private final StateTag> address; - private final ByteString stateKey; - private final String stateFamily; - private final Coder coder; - - /** Whether we've modified the value since creation of this state. */ - private boolean modified = false; - /** Whether the in memory value is the true value. */ - private boolean valueIsKnown = false; - /** The size of the encoded value */ - private long cachedSize = -1; - - private T value; - - private WindmillValue( - StateNamespace namespace, - StateTag> address, - String stateFamily, - Coder coder, - boolean isNewKey) { - this.namespace = namespace; - this.address = address; - this.stateKey = encodeKey(namespace, address); - this.stateFamily = stateFamily; - this.coder = coder; - if (isNewKey) { - this.valueIsKnown = true; - this.value = null; - } - } - - @Override - public void clear() { - modified = true; - valueIsKnown = true; - value = null; - } - - @Override - @SuppressWarnings("FutureReturnValueIgnored") - public WindmillValue readLater() { - getFuture(); - return this; - } - - @Override - public T read() { - try (Closeable scope = scopedReadState()) { - if (!valueIsKnown) { - cachedSize = -1; - } - value = getFuture().get(); - valueIsKnown = true; - return value; - } catch (InterruptedException | ExecutionException | IOException e) { - if (e instanceof InterruptedException) { - Thread.currentThread().interrupt(); - } - throw new RuntimeException("Unable to read value from state", e); - } - } - - @Override - public void write(T value) { - modified = true; - valueIsKnown = true; - cachedSize = -1; - this.value = value; - } - - @Override - protected WorkItemCommitRequest persistDirectly(WindmillStateCache.ForKeyAndFamily cache) - throws IOException { - if (!valueIsKnown) { - // The value was never read, written or cleared. - // Thus nothing to update in Windmill. - // And no need to add to global cache. - return WorkItemCommitRequest.newBuilder().buildPartial(); - } - - ByteString encoded = null; - if (cachedSize == -1 || modified) { - ByteStringOutputStream stream = new ByteStringOutputStream(); - if (value != null) { - coder.encode(value, stream, Coder.Context.OUTER); - } - encoded = stream.toByteString(); - cachedSize = encoded.size(); - } - - // Place in cache to avoid a future read. - cache.put(namespace, address, this, cachedSize); - - if (!modified) { - // The value was read, but never written or cleared. - // But nothing to update in Windmill. - return WorkItemCommitRequest.newBuilder().buildPartial(); - } - - // The value was written or cleared. Commit that change to Windmill. - modified = false; - WorkItemCommitRequest.Builder commitBuilder = WorkItemCommitRequest.newBuilder(); - commitBuilder - .addValueUpdatesBuilder() - .setTag(stateKey) - .setStateFamily(stateFamily) - .getValueBuilder() - .setData(encoded) - .setTimestamp(Long.MAX_VALUE); - return commitBuilder.buildPartial(); - } - - private Future getFuture() { - // WindmillStateReader guarantees that we can ask for a future for a particular tag multiple - // times and it will efficiently be reused. - return valueIsKnown - ? Futures.immediateFuture(value) - : reader.valueFuture(stateKey, stateFamily, coder); - } - } - - // Coder for closed-open ranges. - private static class RangeCoder extends StructuredCoder> { - private Coder boundCoder; - - RangeCoder(Coder boundCoder) { - this.boundCoder = NullableCoder.of(boundCoder); - } - - @Override - public List> getCoderArguments() { - return Lists.newArrayList(boundCoder); - } - - @Override - public void verifyDeterministic() throws NonDeterministicException { - boundCoder.verifyDeterministic(); - ; - } - - @Override - public void encode(Range value, OutputStream outStream) throws CoderException, IOException { - Preconditions.checkState( - value.lowerBoundType().equals(BoundType.CLOSED), "unexpected range " + value); - Preconditions.checkState( - value.upperBoundType().equals(BoundType.OPEN), "unexpected range " + value); - boundCoder.encode(value.hasLowerBound() ? value.lowerEndpoint() : null, outStream); - boundCoder.encode(value.hasUpperBound() ? value.upperEndpoint() : null, outStream); - } - - @Override - public Range decode(InputStream inStream) throws CoderException, IOException { - @Nullable T lower = boundCoder.decode(inStream); - @Nullable T upper = boundCoder.decode(inStream); - if (lower == null) { - return upper != null ? Range.lessThan(upper) : Range.all(); - } else if (upper == null) { - return Range.atLeast(lower); - } else { - return Range.closedOpen(lower, upper); - } - } - } - - private static class RangeSetCoder extends CustomCoder> { - private SetCoder> rangesCoder; - - RangeSetCoder(Coder boundCoder) { - this.rangesCoder = SetCoder.of(new RangeCoder<>(boundCoder)); - } - - @Override - public void encode(RangeSet value, OutputStream outStream) throws IOException { - rangesCoder.encode(value.asRanges(), outStream); - } - - @Override - public RangeSet decode(InputStream inStream) throws CoderException, IOException { - return TreeRangeSet.create(rangesCoder.decode(inStream)); - } - } - - /** - * Tracker for the ids used in an ordered list. - * - *

Windmill accepts an int64 id for each timestamped-element in the list. Unique elements are - * identified by the pair of timestamp and id. This means that tow unique elements e1, e2 must - * have different (ts1, id1), (ts2, id2) pairs. To accomplish this we bucket time into five-minute - * buckets, and store a free list of ids available for each bucket. - * - *

When a timestamp range is deleted, we remove id tracking for elements in that range. In - * order to handle the case where a range is deleted piecemeal, we track sub-range deletions for - * each range. For example: - * - *

12:00 - 12:05 ids 12:05 - 12:10 ids - * - *

delete 12:00-12:06 - * - *

12:00 - 12:05 *removed* 12:05 - 12:10 ids subranges deleted 12:05-12:06 - * - *

delete 12:06 - 12:07 - * - *

12:05 - 12:10 ids subranges deleted 12:05-12:07 - * - *

delete 12:07 - 12:10 - * - *

12:05 - 12:10 *removed* - */ - static final class IdTracker { - static final String IDS_AVAILABLE_STR = "IdsAvailable"; - static final String DELETIONS_STR = "Deletions"; - - // Note that this previously was Long.MIN_VALUE but ids are unsigned when - // sending to windmill for Streaming Engine. For updated appliance - // pipelines with existing state, there may be negative ids. - static final long NEW_RANGE_MIN_ID = 0; - static final long NEW_RANGE_MAX_ID = Long.MAX_VALUE; - - // We track ids on five-minute boundaries. - private static final Duration RESOLUTION = Duration.standardMinutes(5); - static final MapCoder, RangeSet> IDS_AVAILABLE_CODER = - MapCoder.of(new RangeCoder<>(InstantCoder.of()), new RangeSetCoder<>(VarLongCoder.of())); - static final MapCoder, RangeSet> SUBRANGE_DELETIONS_CODER = - MapCoder.of(new RangeCoder<>(InstantCoder.of()), new RangeSetCoder<>(InstantCoder.of())); - private final StateTag, RangeSet>>> idsAvailableTag; - // A map from five-minute ranges to the set of ids available in that interval. - final ValueState, RangeSet>> idsAvailableValue; - private final StateTag, RangeSet>>> subRangeDeletionsTag; - // If a timestamp-range in the map has been partially cleared, the cleared intervals are stored - // here. - final ValueState, RangeSet>> subRangeDeletionsValue; - - IdTracker( - StateTable stateTable, - StateNamespace namespace, - StateTag spec, - String stateFamily, - boolean complete) { - this.idsAvailableTag = - StateTags.makeSystemTagInternal( - StateTags.value(spec.getId() + IDS_AVAILABLE_STR, IDS_AVAILABLE_CODER)); - this.idsAvailableValue = - stateTable.get(namespace, idsAvailableTag, StateContexts.nullContext()); - this.subRangeDeletionsTag = - StateTags.makeSystemTagInternal( - StateTags.value(spec.getId() + DELETIONS_STR, SUBRANGE_DELETIONS_CODER)); - this.subRangeDeletionsValue = - stateTable.get(namespace, subRangeDeletionsTag, StateContexts.nullContext()); - } - - static > - Map, RangeSet> newSortedRangeMap(Class valueClass) { - return Maps.newTreeMap( - Comparator., Instant>comparing(Range::lowerEndpoint) - .thenComparing(Range::upperEndpoint)); - } - - private Range getTrackedRange(Instant ts) { - Instant snapped = - new Instant(ts.getMillis() - ts.plus(RESOLUTION).getMillis() % RESOLUTION.getMillis()); - return Range.closedOpen(snapped, snapped.plus(RESOLUTION)); - } - - @SuppressWarnings("FutureReturnValueIgnored") - void readLater() { - idsAvailableValue.readLater(); - subRangeDeletionsValue.readLater(); - } - - Map, RangeSet> readIdsAvailable() { - Map, RangeSet> idsAvailable = idsAvailableValue.read(); - return idsAvailable != null ? idsAvailable : newSortedRangeMap(Long.class); - } - - Map, RangeSet> readSubRangeDeletions() { - Map, RangeSet> subRangeDeletions = subRangeDeletionsValue.read(); - return subRangeDeletions != null ? subRangeDeletions : newSortedRangeMap(Instant.class); - } - - void clear() throws ExecutionException, InterruptedException { - idsAvailableValue.clear(); - subRangeDeletionsValue.clear(); - } - - void add( - SortedSet> elements, BiConsumer, Long> output) - throws ExecutionException, InterruptedException { - Range currentIdRange = null; - long currentId = 0; - - Range currentTsRange = null; - RangeSet currentTsRangeDeletions = null; - - Map, RangeSet> idsAvailable = readIdsAvailable(); - Map, RangeSet> subRangeDeletions = readSubRangeDeletions(); - - RangeSet availableIdsForTsRange = null; - Iterator> idRangeIter = null; - RangeSet idsUsed = TreeRangeSet.create(); - for (TimestampedValueWithId pendingAdd : elements) { - // Since elements are in increasing ts order, often we'll be able to reuse the previous - // iteration's range. - if (currentTsRange == null - || !currentTsRange.contains(pendingAdd.getValue().getTimestamp())) { - if (availableIdsForTsRange != null) { - // We're moving onto a new ts range. Remove all used ids - availableIdsForTsRange.removeAll(idsUsed); - idsUsed = TreeRangeSet.create(); - } - - // Lookup the range for the current timestamp. - currentTsRange = getTrackedRange(pendingAdd.getValue().getTimestamp()); - // Lookup available ids for this timestamp range. If nothing there, we default to all ids - // available. - availableIdsForTsRange = - idsAvailable.computeIfAbsent( - currentTsRange, - r -> - TreeRangeSet.create( - ImmutableList.of(Range.closedOpen(NEW_RANGE_MIN_ID, NEW_RANGE_MAX_ID)))); - idRangeIter = availableIdsForTsRange.asRanges().iterator(); - currentIdRange = null; - currentTsRangeDeletions = subRangeDeletions.get(currentTsRange); - } - - if (currentIdRange == null || currentId >= currentIdRange.upperEndpoint()) { - // Move to the next range of free ids, and start assigning ranges from there. - currentIdRange = idRangeIter.next(); - currentId = currentIdRange.lowerEndpoint(); - } - - if (currentTsRangeDeletions != null) { - currentTsRangeDeletions.remove( - Range.closedOpen( - pendingAdd.getValue().getTimestamp(), - pendingAdd.getValue().getTimestamp().plus(Duration.millis(1)))); - } - idsUsed.add(Range.closedOpen(currentId, currentId + 1)); - output.accept(pendingAdd.getValue(), currentId++); - } - if (availableIdsForTsRange != null) { - availableIdsForTsRange.removeAll(idsUsed); - } - writeValues(idsAvailable, subRangeDeletions); - } - - // Remove a timestamp range. Returns ids freed up. - void remove(Range tsRange) throws ExecutionException, InterruptedException { - Map, RangeSet> idsAvailable = readIdsAvailable(); - Map, RangeSet> subRangeDeletions = readSubRangeDeletions(); - - for (Range current = getTrackedRange(tsRange.lowerEndpoint()); - current.lowerEndpoint().isBefore(tsRange.upperEndpoint()); - current = getTrackedRange(current.lowerEndpoint().plus(RESOLUTION))) { - // TODO(reuvenlax): shouldn't need to iterate over all ranges. - boolean rangeCleared; - if (!tsRange.encloses(current)) { - // This can happen if the beginning or the end of tsRange doesn't fall on a RESOLUTION - // boundary. Since we - // are deleting a portion of a tracked range, track what we are deleting. - RangeSet rangeDeletions = - subRangeDeletions.computeIfAbsent(current, r -> TreeRangeSet.create()); - rangeDeletions.add(tsRange.intersection(current)); - // If we ended up deleting the whole range, than we can simply remove it from the tracking - // map. - rangeCleared = rangeDeletions.encloses(current); - } else { - rangeCleared = true; - } - if (rangeCleared) { - // Remove the range from both maps. - idsAvailable.remove(current); - subRangeDeletions.remove(current); - } - } - writeValues(idsAvailable, subRangeDeletions); - } - - private void writeValues( - Map, RangeSet> idsAvailable, - Map, RangeSet> subRangeDeletions) { - if (idsAvailable.isEmpty()) { - idsAvailable.clear(); - } else { - idsAvailableValue.write(idsAvailable); - } - if (subRangeDeletions.isEmpty()) { - subRangeDeletionsValue.clear(); - } else { - subRangeDeletionsValue.write(subRangeDeletions); - } - } - } - - @AutoValue - abstract static class TimestampedValueWithId { - private static final Comparator> COMPARATOR = - Comparator., Instant>comparing(v -> v.getValue().getTimestamp()) - .thenComparingLong(TimestampedValueWithId::getId); - - abstract TimestampedValue getValue(); - - abstract long getId(); - - static TimestampedValueWithId of(TimestampedValue value, long id) { - return new AutoValue_WindmillStateInternals_TimestampedValueWithId<>(value, id); - } - - static TimestampedValueWithId bound(Instant ts) { - return of(TimestampedValue.of(null, ts), Long.MIN_VALUE); - } - } - - static class WindmillOrderedList extends SimpleWindmillState implements OrderedListState { - private final ByteString stateKey; - private final String stateFamily; - private final Coder elemCoder; - private boolean complete; - private boolean cleared = false; - // We need to sort based on timestamp, but we need objects with the same timestamp to be treated - // as unique. We can't use a MultiSet as we can't construct a comparator that uniquely - // identifies objects, - // so we construct a unique in-memory long ids for each element. - private SortedSet> pendingAdds = - Sets.newTreeSet(TimestampedValueWithId.COMPARATOR); - - private RangeSet pendingDeletes = TreeRangeSet.create(); - private IdTracker idTracker; - - // The default proto values for SortedListRange correspond to the minimum and maximum - // timestamps. - static final long MIN_TS_MICROS = SortedListRange.getDefaultInstance().getStart(); - static final long MAX_TS_MICROS = SortedListRange.getDefaultInstance().getLimit(); - - private WindmillOrderedList( - StateTable derivedStateTable, - StateNamespace namespace, - StateTag> spec, - String stateFamily, - Coder elemCoder, - boolean isNewKey) { - - this.stateKey = encodeKey(namespace, spec); - this.stateFamily = stateFamily; - this.elemCoder = elemCoder; - this.complete = isNewKey; - this.idTracker = new IdTracker(derivedStateTable, namespace, spec, stateFamily, complete); - } - - @Override - public Iterable> read() { - return readRange(null, null); - } - - private SortedSet> getPendingAddRange( - @Nullable Instant minTimestamp, @Nullable Instant limitTimestamp) { - SortedSet> pendingInRange = pendingAdds; - if (minTimestamp != null && limitTimestamp != null) { - pendingInRange = - pendingInRange.subSet( - TimestampedValueWithId.bound(minTimestamp), - TimestampedValueWithId.bound(limitTimestamp)); - } else if (minTimestamp == null && limitTimestamp != null) { - pendingInRange = pendingInRange.headSet(TimestampedValueWithId.bound(limitTimestamp)); - } else if (limitTimestamp == null && minTimestamp != null) { - pendingInRange = pendingInRange.tailSet(TimestampedValueWithId.bound(minTimestamp)); - } - return pendingInRange; - } - - @Override - public Iterable> readRange( - @Nullable Instant minTimestamp, @Nullable Instant limitTimestamp) { - idTracker.readLater(); - - final Future>> future = getFuture(minTimestamp, limitTimestamp); - try (Closeable scope = scopedReadState()) { - SortedSet> pendingInRange = - getPendingAddRange(minTimestamp, limitTimestamp); - - // Transform the return iterator so it has the same type as pendingAdds. We need to ensure - // that the ids don't overlap with any in pendingAdds, so begin with pendingAdds.size(). - Iterable> data = - new Iterable>() { - // Anything returned from windmill that has been deleted should be ignored. - private Iterable> iterable = - Iterables.filter(future.get(), tv -> !pendingDeletes.contains(tv.getTimestamp())); - - @Override - public Iterator> iterator() { - return new Iterator>() { - private Iterator> iter = iterable.iterator(); - private long currentId = pendingAdds.size(); - - @Override - public boolean hasNext() { - return iter.hasNext(); - } - - @Override - public TimestampedValueWithId next() { - return TimestampedValueWithId.of(iter.next(), currentId++); - } - }; - } - }; - - Iterable> includingAdds = - Iterables.mergeSorted( - ImmutableList.of(data, pendingInRange), TimestampedValueWithId.COMPARATOR); - Iterable> fullIterable = - Iterables.transform(includingAdds, TimestampedValueWithId::getValue); - - // TODO(reuvenlax): If we have a known bounded amount of data, cache known ranges. - return fullIterable; - } catch (InterruptedException | ExecutionException | IOException e) { - if (e instanceof InterruptedException) { - Thread.currentThread().interrupt(); - } - throw new RuntimeException("Unable to read state", e); - } - } - - @Override - public void clear() { - cleared = true; - complete = true; - pendingAdds.clear(); - pendingDeletes.clear(); - try { - idTracker.clear(); - } catch (ExecutionException | InterruptedException e) { - throw new RuntimeException(e); - } - } - - @Override - public void clearRange(Instant minTimestamp, Instant limitTimestamp) { - getPendingAddRange(minTimestamp, limitTimestamp).clear(); - pendingDeletes.add(Range.closedOpen(minTimestamp, limitTimestamp)); - } - - @Override - public void add(TimestampedValue value) { - // We use the current size of the container as the in-memory id. This works because - // pendingAdds is completely - // cleared when it is processed (otherwise we could end up with duplicate elements in the same - // container). These - // are not the ids that will be sent to windmill. - pendingAdds.add(TimestampedValueWithId.of(value, pendingAdds.size())); - // Leave pendingDeletes alone. Since we can have multiple values with the same timestamp, we - // may still need - // overlapping deletes to remove previous entries at this timestamp. - } - - @Override - public ReadableState isEmpty() { - return new ReadableState() { - @Override - public ReadableState readLater() { - WindmillOrderedList.this.readLater(); - return this; - } - - @Override - public Boolean read() { - return Iterables.isEmpty(WindmillOrderedList.this.read()); - } - }; - } - - @Override - public OrderedListState readLater() { - return readRangeLater(null, null); - } - - @Override - @SuppressWarnings("FutureReturnValueIgnored") - public OrderedListState readRangeLater( - @Nullable Instant minTimestamp, @Nullable Instant limitTimestamp) { - idTracker.readLater(); - getFuture(minTimestamp, limitTimestamp); - return this; - } - - @Override - public WorkItemCommitRequest persistDirectly(WindmillStateCache.ForKeyAndFamily cache) - throws IOException { - WorkItemCommitRequest.Builder commitBuilder = WorkItemCommitRequest.newBuilder(); - TagSortedListUpdateRequest.Builder updatesBuilder = - commitBuilder - .addSortedListUpdatesBuilder() - .setStateFamily(cache.getStateFamily()) - .setTag(stateKey); - try { - if (cleared) { - // Default range. - updatesBuilder.addDeletesBuilder().build(); - cleared = false; - } - - if (!pendingAdds.isEmpty()) { - // TODO(reuvenlax): Once we start caching data, we should remove this line. We have it - // here now - // because once we persist - // added data we forget about it from the cache, so the object is no longer complete. - complete = false; - - TagSortedListInsertRequest.Builder insertBuilder = updatesBuilder.addInsertsBuilder(); - idTracker.add( - pendingAdds, - (elem, id) -> { - try { - ByteStringOutputStream elementStream = new ByteStringOutputStream(); - elemCoder.encode(elem.getValue(), elementStream, Context.OUTER); - insertBuilder.addEntries( - SortedListEntry.newBuilder() - .setValue(elementStream.toByteString()) - .setSortKey( - WindmillTimeUtils.harnessToWindmillTimestamp(elem.getTimestamp())) - .setId(id)); - } catch (IOException e) { - throw new RuntimeException(e); - } - }); - pendingAdds.clear(); - insertBuilder.build(); - } - - if (!pendingDeletes.isEmpty()) { - for (Range range : pendingDeletes.asRanges()) { - TagSortedListDeleteRequest.Builder deletesBuilder = updatesBuilder.addDeletesBuilder(); - deletesBuilder.setRange( - SortedListRange.newBuilder() - .setStart(WindmillTimeUtils.harnessToWindmillTimestamp(range.lowerEndpoint())) - .setLimit(WindmillTimeUtils.harnessToWindmillTimestamp(range.upperEndpoint()))); - deletesBuilder.build(); - idTracker.remove(range); - } - pendingDeletes.clear(); - } - } catch (ExecutionException | InterruptedException e) { - throw new RuntimeException(e); - } - return commitBuilder.buildPartial(); - } - - private Future>> getFuture( - @Nullable Instant minTimestamp, @Nullable Instant limitTimestamp) { - long startSortKey = - minTimestamp != null - ? WindmillTimeUtils.harnessToWindmillTimestamp(minTimestamp) - : MIN_TS_MICROS; - long limitSortKey = - limitTimestamp != null - ? WindmillTimeUtils.harnessToWindmillTimestamp(limitTimestamp) - : MAX_TS_MICROS; - - if (complete) { - // Right now we don't cache any data, so complete means an empty list. - // TODO(reuvenlax): change this once we start caching data. - return Futures.immediateFuture(Collections.emptyList()); - } - return reader.orderedListFuture( - Range.closedOpen(startSortKey, limitSortKey), stateKey, stateFamily, elemCoder); - } - } - - static class WindmillSet extends SimpleWindmillState implements SetState { - WindmillMap windmillMap; - - WindmillSet( - StateNamespace namespace, - StateTag> address, - String stateFamily, - Coder keyCoder, - WindmillStateCache.ForKeyAndFamily cache, - boolean isNewKey) { - StateTag> internalMapAddress = - StateTags.convertToMapTagInternal(address); - WindmillMap cachedMap = - (WindmillMap) cache.get(namespace, internalMapAddress); - this.windmillMap = - (cachedMap != null) - ? cachedMap - : new WindmillMap<>( - namespace, - internalMapAddress, - stateFamily, - keyCoder, - BooleanCoder.of(), - isNewKey); - } - - @Override - protected WorkItemCommitRequest persistDirectly(WindmillStateCache.ForKeyAndFamily cache) - throws IOException { - return windmillMap.persistDirectly(cache); - } - - @Override - public @UnknownKeyFor @NonNull @Initialized ReadableState< - @UnknownKeyFor @NonNull @Initialized Boolean> - contains(K k) { - return windmillMap.getOrDefault(k, false); - } - - @Override - public @UnknownKeyFor @NonNull @Initialized ReadableState< - @UnknownKeyFor @NonNull @Initialized Boolean> - addIfAbsent(K k) { - return new ReadableState() { - ReadableState putState = windmillMap.putIfAbsent(k, true); - - @Override - public @Nullable Boolean read() { - Boolean result = putState.read(); - return (result != null) ? result : false; - } - - @Override - public @UnknownKeyFor @NonNull @Initialized ReadableState readLater() { - putState = putState.readLater(); - return this; - } - }; - } - - @Override - public void remove(K k) { - windmillMap.remove(k); - } - - @Override - public void add(K value) { - windmillMap.put(value, true); - } - - @Override - public @UnknownKeyFor @NonNull @Initialized ReadableState< - @UnknownKeyFor @NonNull @Initialized Boolean> - isEmpty() { - return windmillMap.isEmpty(); - } - - @Override - public @Nullable Iterable read() { - return windmillMap.keys().read(); - } - - @Override - public @UnknownKeyFor @NonNull @Initialized SetState readLater() { - windmillMap.keys().readLater(); - return this; - } - - @Override - public void clear() { - windmillMap.clear(); - } - - @Override - void initializeForWorkItem( - WindmillStateReader reader, Supplier scopedReadStateSupplier) { - windmillMap.initializeForWorkItem(reader, scopedReadStateSupplier); - } - - @Override - void cleanupAfterWorkItem() { - windmillMap.cleanupAfterWorkItem(); - } - } - - static class WindmillMap extends SimpleWindmillState implements MapState { - private final StateNamespace namespace; - private final StateTag> address; - private final ByteString stateKeyPrefix; - private final String stateFamily; - private final Coder keyCoder; - private final Coder valueCoder; - private boolean complete; - - // TODO(reuvenlax): Should we evict items from the cache? We would have to make sure - // that anything in the cache that is not committed is not evicted. negativeCache could be - // evicted whenever we want. - private Map cachedValues = Maps.newHashMap(); - private Set negativeCache = Sets.newHashSet(); - private boolean cleared = false; - - private Set localAdditions = Sets.newHashSet(); - private Set localRemovals = Sets.newHashSet(); - - WindmillMap( - StateNamespace namespace, - StateTag> address, - String stateFamily, - Coder keyCoder, - Coder valueCoder, - boolean isNewKey) { - this.namespace = namespace; - this.address = address; - this.stateKeyPrefix = encodeKey(namespace, address); - this.stateFamily = stateFamily; - this.keyCoder = keyCoder; - this.valueCoder = valueCoder; - this.complete = isNewKey; - } - - private K userKeyFromProtoKey(ByteString tag) throws IOException { - Preconditions.checkState(tag.startsWith(stateKeyPrefix)); - ByteString keyBytes = tag.substring(stateKeyPrefix.size()); - return keyCoder.decode(keyBytes.newInput(), Context.OUTER); - } - - private ByteString protoKeyFromUserKey(K key) throws IOException { - ByteStringOutputStream keyStream = new ByteStringOutputStream(); - stateKeyPrefix.writeTo(keyStream); - keyCoder.encode(key, keyStream, Context.OUTER); - return keyStream.toByteString(); - } - - @Override - protected WorkItemCommitRequest persistDirectly(WindmillStateCache.ForKeyAndFamily cache) - throws IOException { - if (!cleared && localAdditions.isEmpty() && localRemovals.isEmpty()) { - // No changes, so return directly. - return WorkItemCommitRequest.newBuilder().buildPartial(); - } - - WorkItemCommitRequest.Builder commitBuilder = WorkItemCommitRequest.newBuilder(); - - if (cleared) { - commitBuilder - .addTagValuePrefixDeletesBuilder() - .setStateFamily(stateFamily) - .setTagPrefix(stateKeyPrefix); - } - cleared = false; - - for (K key : localAdditions) { - ByteString keyBytes = protoKeyFromUserKey(key); - ByteStringOutputStream valueStream = new ByteStringOutputStream(); - valueCoder.encode(cachedValues.get(key), valueStream, Context.OUTER); - ByteString valueBytes = valueStream.toByteString(); - - commitBuilder - .addValueUpdatesBuilder() - .setTag(keyBytes) - .setStateFamily(stateFamily) - .getValueBuilder() - .setData(valueBytes) - .setTimestamp(Long.MAX_VALUE); - } - localAdditions.clear(); - - for (K key : localRemovals) { - ByteStringOutputStream keyStream = new ByteStringOutputStream(); - stateKeyPrefix.writeTo(keyStream); - keyCoder.encode(key, keyStream, Context.OUTER); - ByteString keyBytes = keyStream.toByteString(); - // Leaving data blank means that we delete the tag. - commitBuilder - .addValueUpdatesBuilder() - .setTag(keyBytes) - .setStateFamily(stateFamily) - .getValueBuilder() - .setTimestamp(Long.MAX_VALUE); - - V cachedValue = cachedValues.remove(key); - if (cachedValue != null) { - ByteStringOutputStream valueStream = new ByteStringOutputStream(); - valueCoder.encode(cachedValues.get(key), valueStream, Context.OUTER); - } - } - negativeCache.addAll(localRemovals); - localRemovals.clear(); - - // TODO(reuvenlax): We should store in the cache parameter, as that would enable caching the - // map - // between work items, reducing fetches to Windmill. To do so, we need keep track of the - // encoded size - // of the map, and to do so efficiently (i.e. without iterating over the entire map on every - // persist) - // we need to track the sizes of each map entry. - cache.put(namespace, address, this, 1); - return commitBuilder.buildPartial(); - } - - @Override - public @UnknownKeyFor @NonNull @Initialized ReadableState get(K key) { - return getOrDefault(key, null); - } - - @Override - public @UnknownKeyFor @NonNull @Initialized ReadableState getOrDefault( - K key, @Nullable V defaultValue) { - return new ReadableState() { - @Override - public @Nullable V read() { - Future persistedData = getFutureForKey(key); - try (Closeable scope = scopedReadState()) { - if (localRemovals.contains(key) || negativeCache.contains(key)) { - return null; - } - @Nullable V cachedValue = cachedValues.get(key); - if (cachedValue != null || complete) { - return cachedValue; - } - - V persistedValue = persistedData.get(); - if (persistedValue == null) { - negativeCache.add(key); - return defaultValue; - } - // TODO: Don't do this if it was already in cache. - cachedValues.put(key, persistedValue); - return persistedValue; - } catch (InterruptedException | ExecutionException | IOException e) { - if (e instanceof InterruptedException) { - Thread.currentThread().interrupt(); - } - throw new RuntimeException("Unable to read state", e); - } - } - - @Override - @SuppressWarnings("FutureReturnValueIgnored") - public @UnknownKeyFor @NonNull @Initialized ReadableState readLater() { - WindmillMap.this.getFutureForKey(key); - return this; - } - }; - } - - @Override - public @UnknownKeyFor @NonNull @Initialized ReadableState< - @UnknownKeyFor @NonNull @Initialized Iterable> - keys() { - ReadableState>> entries = entries(); - return new ReadableState>() { - @Override - public @Nullable Iterable read() { - return Iterables.transform(entries.read(), e -> e.getKey()); - } - - @Override - public @UnknownKeyFor @NonNull @Initialized ReadableState> readLater() { - entries.readLater(); - return this; - } - }; - } - - @Override - public @UnknownKeyFor @NonNull @Initialized ReadableState< - @UnknownKeyFor @NonNull @Initialized Iterable> - values() { - ReadableState>> entries = entries(); - return new ReadableState>() { - @Override - public @Nullable Iterable read() { - return Iterables.transform(entries.read(), e -> e.getValue()); - } - - @Override - public @UnknownKeyFor @NonNull @Initialized ReadableState> readLater() { - entries.readLater(); - return this; - } - }; - } - - @Override - public @UnknownKeyFor @NonNull @Initialized ReadableState< - @UnknownKeyFor @NonNull @Initialized Iterable< - @UnknownKeyFor @NonNull @Initialized Entry>> - entries() { - return new ReadableState>>() { - @Override - public Iterable> read() { - if (complete) { - return Iterables.unmodifiableIterable(cachedValues.entrySet()); - } - Future>> persistedData = getFuture(); - try (Closeable scope = scopedReadState()) { - Iterable> data = persistedData.get(); - Iterable> transformedData = - Iterables., Map.Entry>transform( - data, - entry -> { - try { - return new AbstractMap.SimpleEntry<>( - userKeyFromProtoKey(entry.getKey()), entry.getValue()); - } catch (IOException e) { - throw new RuntimeException(e); - } - }); - - if (data instanceof Weighted) { - // This is a known amount of data. Cache it all. - transformedData.forEach( - e -> { - // The cached data overrides what is read from state, so call putIfAbsent. - cachedValues.putIfAbsent(e.getKey(), e.getValue()); - }); - complete = true; - return Iterables.unmodifiableIterable(cachedValues.entrySet()); - } else { - // This means that the result might be too large to cache, so don't add it to the - // local cache. Instead merge the iterables, giving priority to any local additions - // (represented in cachedValued and localRemovals) that may not have been committed - // yet. - return Iterables.unmodifiableIterable( - Iterables.concat( - cachedValues.entrySet(), - Iterables.filter( - transformedData, - e -> - !cachedValues.containsKey(e.getKey()) - && !localRemovals.contains(e.getKey())))); - } - - } catch (InterruptedException | ExecutionException | IOException e) { - if (e instanceof InterruptedException) { - Thread.currentThread().interrupt(); - } - throw new RuntimeException("Unable to read state", e); - } - } - - @Override - @SuppressWarnings("FutureReturnValueIgnored") - public @UnknownKeyFor @NonNull @Initialized ReadableState>> - readLater() { - WindmillMap.this.getFuture(); - return this; - } - }; - } - - @Override - public ReadableState isEmpty() { - return new ReadableState() { - // TODO(reuvenlax): Can we find a more efficient way of implementing isEmpty than reading - // the entire map? - ReadableState> keys = WindmillMap.this.keys(); - - @Override - public @Nullable Boolean read() { - return Iterables.isEmpty(keys.read()); - } - - @Override - public @UnknownKeyFor @NonNull @Initialized ReadableState readLater() { - keys.readLater(); - return this; - } - }; - } - - @Override - public void put(K key, V value) { - V oldValue = cachedValues.put(key, value); - if (valueCoder.consistentWithEquals() && value.equals(oldValue)) { - return; - } - localAdditions.add(key); - localRemovals.remove(key); - negativeCache.remove(key); - } - - @Override - public @UnknownKeyFor @NonNull @Initialized ReadableState computeIfAbsent( - K key, Function mappingFunction) { - Future persistedData = getFutureForKey(key); - try (Closeable scope = scopedReadState()) { - if (localRemovals.contains(key) || negativeCache.contains(key)) { - return ReadableStates.immediate(null); - } - @Nullable V cachedValue = cachedValues.get(key); - if (cachedValue != null || complete) { - return ReadableStates.immediate(cachedValue); - } - - V persistedValue = persistedData.get(); - if (persistedValue == null) { - // This is a new value. Add it to the map and return null. - put(key, mappingFunction.apply(key)); - return ReadableStates.immediate(null); - } - // TODO: Don't do this if it was already in cache. - cachedValues.put(key, persistedValue); - return ReadableStates.immediate(persistedValue); - } catch (InterruptedException | ExecutionException | IOException e) { - if (e instanceof InterruptedException) { - Thread.currentThread().interrupt(); - } - throw new RuntimeException("Unable to read state", e); - } - } - - @Override - public void remove(K key) { - if (localRemovals.add(key)) { - cachedValues.remove(key); - localAdditions.remove(key); - } - } - - @Override - public void clear() { - cachedValues.clear(); - localAdditions.clear(); - localRemovals.clear(); - negativeCache.clear(); - cleared = true; - complete = true; - } - - private Future getFutureForKey(K key) { - try { - ByteStringOutputStream keyStream = new ByteStringOutputStream(); - stateKeyPrefix.writeTo(keyStream); - keyCoder.encode(key, keyStream, Context.OUTER); - return reader.valueFuture(keyStream.toByteString(), stateFamily, valueCoder); - } catch (IOException e) { - throw new RuntimeException(e); - } - } - - private Future>> getFuture() { - if (complete) { - // The caller will merge in local cached values. - return Futures.immediateFuture(Collections.emptyList()); - } else { - return reader.valuePrefixFuture(stateKeyPrefix, stateFamily, valueCoder); - } - } - } - - private static class WindmillMultimap extends SimpleWindmillState - implements MultimapState { - - private final StateNamespace namespace; - private final StateTag> address; - private final ByteString stateKey; - private final String stateFamily; - private final Coder keyCoder; - private final Coder valueCoder; - - private enum KeyExistence { - // this key is known to exist, it has at least 1 value in either localAdditions or windmill - KNOWN_EXIST, - // this key is known to be nonexistent, it has 0 value in both localAdditions and windmill - KNOWN_NONEXISTENT, - // we don't know if this key is in this multimap, it has exact 0 value in localAddition, but - // may have no or any number of values in windmill. This is just to provide a mapping between - // the original key and the structural key. - UNKNOWN_EXISTENCE - } - - private class KeyState { - final K originalKey; - KeyExistence existence; - // valuesCached can be true if only existence == KNOWN_EXIST and all values of this key are - // cached (both values and localAdditions). - boolean valuesCached; - // Represents the values in windmill. When new values are added during user processing, they - // are added to localAdditions but not values. Those new values will be added to values only - // after they are persisted into windmill and removed from localAdditions - ConcatIterables values; - int valuesSize; - - // When new values are added during user processing, they are added to localAdditions, so that - // we can later try to persist them in windmill. When a key is removed during user processing, - // we mark removedLocally to be true so that we can later try to delete it from windmill. If - // localAdditions is not empty and removedLocally is true, values in localAdditions will be - // added to windmill after old values in windmill are removed. - List localAdditions; - boolean removedLocally; - - KeyState(K originalKey) { - this.originalKey = originalKey; - existence = KeyExistence.UNKNOWN_EXISTENCE; - valuesCached = complete; - values = new ConcatIterables<>(); - valuesSize = 0; - localAdditions = Lists.newArrayList(); - removedLocally = false; - } - } - - // Set to true when user clears the entire multimap, so that we can later send delete request to - // the windmill backend. - private boolean cleared = false; - // We use the structural value of the keys as the key in keyStateMap, so that different java - // Objects with the same content will be treated as the same Multimap key. - private Map keyStateMap = Maps.newHashMap(); - // If true, all keys are cached in keyStateMap with existence == KNOWN_EXIST. - private boolean allKeysKnown = false; - - // True if all contents of this multimap are cached in this object. - private boolean complete = false; - // hasLocalAdditions and hasLocalRemovals track whether there are local changes that needs to be - // propagated to windmill. - private boolean hasLocalAdditions = false; - private boolean hasLocalRemovals = false; - - private WindmillMultimap( - StateNamespace namespace, - StateTag> address, - String stateFamily, - Coder keyCoder, - Coder valueCoder, - boolean isNewShardingKey) { - this.namespace = namespace; - this.address = address; - this.stateKey = encodeKey(namespace, address); - this.stateFamily = stateFamily; - this.keyCoder = keyCoder; - this.valueCoder = valueCoder; - this.complete = isNewShardingKey; - this.allKeysKnown = isNewShardingKey; - } - - @Override - public void put(K key, V value) { - final Object structuralKey = keyCoder.structuralValue(key); - hasLocalAdditions = true; - keyStateMap.compute( - structuralKey, - (k, v) -> { - if (v == null) v = new KeyState(key); - v.existence = KeyExistence.KNOWN_EXIST; - v.localAdditions.add(value); - return v; - }); - } - - // Initiates a backend state read to fetch all entries if necessary. - private Future>>> necessaryEntriesFromStorageFuture( - boolean omitValues) { - if (complete) { - // Since we're complete, even if there are entries in storage we don't need to read them. - return Futures.immediateFuture(Collections.emptyList()); - } else { - return reader.multimapFetchAllFuture(omitValues, stateKey, stateFamily, valueCoder); - } - } - - // Initiates a backend state read to fetch a single entry if necessary. - private Future> necessaryKeyEntriesFromStorageFuture(K key) { - try { - ByteStringOutputStream keyStream = new ByteStringOutputStream(); - keyCoder.encode(key, keyStream, Context.OUTER); - return reader.multimapFetchSingleEntryFuture( - keyStream.toByteString(), stateKey, stateFamily, valueCoder); - } catch (IOException e) { - throw new RuntimeException(e); - } - } - - @Override - public ReadableState> get(K key) { - return new ReadableState>() { - final Object structuralKey = keyCoder.structuralValue(key); - - @Override - public Iterable read() { - KeyState keyState = null; - if (allKeysKnown) { - keyState = keyStateMap.get(structuralKey); - if (keyState == null || keyState.existence == KeyExistence.UNKNOWN_EXISTENCE) { - if (keyState != null) keyStateMap.remove(structuralKey); - return Collections.emptyList(); - } - } else { - keyState = keyStateMap.computeIfAbsent(structuralKey, k -> new KeyState(key)); - } - if (keyState.existence == KeyExistence.KNOWN_NONEXISTENT) { - return Collections.emptyList(); - } - Iterable localNewValues = - Iterables.limit(keyState.localAdditions, keyState.localAdditions.size()); - if (keyState.removedLocally) { - // this key has been removed locally but the removal hasn't been sent to windmill, - // thus values in windmill(if any) are obsolete, and we only care about local values. - return Iterables.unmodifiableIterable(localNewValues); - } - if (keyState.valuesCached || complete) { - return Iterables.unmodifiableIterable( - Iterables.concat( - Iterables.limit(keyState.values, keyState.valuesSize), localNewValues)); - } - Future> persistedData = necessaryKeyEntriesFromStorageFuture(key); - try (Closeable scope = scopedReadState()) { - final Iterable persistedValues = persistedData.get(); - // Iterables.isEmpty() is O(1). - if (Iterables.isEmpty(persistedValues)) { - if (keyState.localAdditions.isEmpty()) { - // empty in both cache and windmill, mark key as KNOWN_NONEXISTENT. - keyState.existence = KeyExistence.KNOWN_NONEXISTENT; - return Collections.emptyList(); - } - return Iterables.unmodifiableIterable(localNewValues); - } - keyState.existence = KeyExistence.KNOWN_EXIST; - if (persistedValues instanceof Weighted) { - keyState.valuesCached = true; - ConcatIterables it = new ConcatIterables<>(); - it.extendWith(persistedValues); - keyState.values = it; - keyState.valuesSize = Iterables.size(persistedValues); - } - return Iterables.unmodifiableIterable( - Iterables.concat(persistedValues, localNewValues)); - } catch (InterruptedException | ExecutionException | IOException e) { - if (e instanceof InterruptedException) { - Thread.currentThread().interrupt(); - } - throw new RuntimeException("Unable to read Multimap state", e); - } - } - - @Override - @SuppressWarnings("FutureReturnValueIgnored") - public ReadableState> readLater() { - WindmillMultimap.this.necessaryKeyEntriesFromStorageFuture(key); - return this; - } - }; - } - - @Override - protected WorkItemCommitRequest persistDirectly(WindmillStateCache.ForKeyAndFamily cache) - throws IOException { - if (!cleared && !hasLocalAdditions && !hasLocalRemovals) { - cache.put(namespace, address, this, 1); - return WorkItemCommitRequest.newBuilder().buildPartial(); - } - WorkItemCommitRequest.Builder commitBuilder = WorkItemCommitRequest.newBuilder(); - Windmill.TagMultimapUpdateRequest.Builder builder = commitBuilder.addMultimapUpdatesBuilder(); - builder.setTag(stateKey).setStateFamily(stateFamily); - - if (cleared) { - builder.setDeleteAll(true); - } - if (hasLocalRemovals || hasLocalAdditions) { - ByteStringOutputStream keyStream = new ByteStringOutputStream(); - ByteStringOutputStream valueStream = new ByteStringOutputStream(); - Iterator> iterator = keyStateMap.entrySet().iterator(); - while (iterator.hasNext()) { - KeyState keyState = iterator.next().getValue(); - if (!keyState.removedLocally && keyState.localAdditions.isEmpty()) { - if (keyState.existence == KeyExistence.KNOWN_NONEXISTENT) iterator.remove(); - continue; - } - keyCoder.encode(keyState.originalKey, keyStream, Context.OUTER); - ByteString encodedKey = keyStream.toByteStringAndReset(); - Windmill.TagMultimapEntry.Builder entryBuilder = builder.addUpdatesBuilder(); - entryBuilder.setEntryName(encodedKey); - if (keyState.removedLocally) entryBuilder.setDeleteAll(true); - keyState.removedLocally = false; - if (!keyState.localAdditions.isEmpty()) { - for (V value : keyState.localAdditions) { - valueCoder.encode(value, valueStream, Context.OUTER); - ByteString encodedValue = valueStream.toByteStringAndReset(); - entryBuilder.addValues(encodedValue); - } - // Move newly added values from localAdditions to keyState.values as those new values - // now - // are also persisted in Windmill. If a key now has no more values and is not - // KNOWN_EXIST, - // remove it from cache. - if (keyState.valuesCached) { - keyState.values.extendWith(keyState.localAdditions); - keyState.valuesSize += keyState.localAdditions.size(); - } - // Create a new localAdditions so that the cached values are unaffected. - keyState.localAdditions = Lists.newArrayList(); - } - if (!keyState.valuesCached && keyState.existence != KeyExistence.KNOWN_EXIST) { - iterator.remove(); - } - } - } - - hasLocalAdditions = false; - hasLocalRemovals = false; - cleared = false; - - cache.put(namespace, address, this, 1); - return commitBuilder.buildPartial(); - } - - @Override - public void remove(K key) { - final Object structuralKey = keyCoder.structuralValue(key); - // does not insert key if allKeysKnown. - KeyState keyState = - keyStateMap.computeIfAbsent(structuralKey, k -> allKeysKnown ? null : new KeyState(key)); - if (keyState == null || keyState.existence == KeyExistence.KNOWN_NONEXISTENT) { - return; - } - if (keyState.valuesCached && keyState.valuesSize == 0) { - // no data in windmill, deleting from local cache is sufficient. - keyStateMap.remove(structuralKey); - } else { - // there may be data in windmill that need to be removed. - hasLocalRemovals = true; - keyState.removedLocally = true; - keyState.values = new ConcatIterables<>(); - keyState.valuesSize = 0; - keyState.existence = KeyExistence.KNOWN_NONEXISTENT; - } - if (!keyState.localAdditions.isEmpty()) { - keyState.localAdditions = Lists.newArrayList(); - } - keyState.valuesCached = true; - } - - @Override - public void clear() { - keyStateMap = Maps.newHashMap(); - cleared = true; - complete = true; - allKeysKnown = true; - hasLocalAdditions = false; - hasLocalRemovals = false; - } - - @Override - public ReadableState> keys() { - return new ReadableState>() { - - private Map cachedExistKeys() { - return keyStateMap.entrySet().stream() - .filter(entry -> entry.getValue().existence == KeyExistence.KNOWN_EXIST) - .collect(Collectors.toMap(Entry::getKey, e -> e.getValue().originalKey)); - } - - @Override - public Iterable read() { - if (allKeysKnown) { - return Iterables.unmodifiableIterable(cachedExistKeys().values()); - } - Future>>> persistedData = - necessaryEntriesFromStorageFuture(true); - try (Closeable scope = scopedReadState()) { - Iterable>> entries = persistedData.get(); - if (entries instanceof Weighted) { - // This is a known amount of data, cache them all. - entries.forEach( - entry -> { - try { - K originalKey = keyCoder.decode(entry.getKey().newInput(), Context.OUTER); - KeyState keyState = - keyStateMap.computeIfAbsent( - keyCoder.structuralValue(originalKey), - stk -> new KeyState(originalKey)); - if (keyState.existence == KeyExistence.UNKNOWN_EXISTENCE) { - keyState.existence = KeyExistence.KNOWN_EXIST; - } - } catch (IOException e) { - throw new RuntimeException(e); - } - }); - allKeysKnown = true; - keyStateMap - .values() - .removeIf( - keyState -> - keyState.existence != KeyExistence.KNOWN_EXIST - && !keyState.removedLocally); - return Iterables.unmodifiableIterable(cachedExistKeys().values()); - } else { - Map cachedExistKeys = Maps.newHashMap(); - Set cachedNonExistKeys = Sets.newHashSet(); - keyStateMap.forEach( - (structuralKey, keyState) -> { - switch (keyState.existence) { - case KNOWN_EXIST: - cachedExistKeys.put(structuralKey, keyState.originalKey); - break; - case KNOWN_NONEXISTENT: - cachedNonExistKeys.add(structuralKey); - break; - default: - break; - } - }); - // keysOnlyInWindmill is lazily loaded. - Iterable keysOnlyInWindmill = - Iterables.filter( - Iterables.transform( - entries, - entry -> { - try { - K originalKey = - keyCoder.decode(entry.getKey().newInput(), Context.OUTER); - Object structuralKey = keyCoder.structuralValue(originalKey); - if (cachedExistKeys.containsKey(structuralKey) - || cachedNonExistKeys.contains(structuralKey)) return null; - return originalKey; - } catch (IOException e) { - throw new RuntimeException(e); - } - }), - Objects::nonNull); - return Iterables.unmodifiableIterable( - Iterables.concat(cachedExistKeys.values(), keysOnlyInWindmill)); - } - } catch (InterruptedException | ExecutionException | IOException e) { - if (e instanceof InterruptedException) { - Thread.currentThread().interrupt(); - } - throw new RuntimeException("Unable to read state", e); - } - } - - @Override - @SuppressWarnings("FutureReturnValueIgnored") - public ReadableState> readLater() { - WindmillMultimap.this.necessaryEntriesFromStorageFuture(true); - return this; - } - }; - } - - @Override - public ReadableState>> entries() { - return new ReadableState>>() { - @Override - public Iterable> read() { - if (complete) { - return Iterables.unmodifiableIterable( - unnestCachedEntries(mergedCachedEntries(null).entrySet())); - } - Future>>> persistedData = - necessaryEntriesFromStorageFuture(false); - try (Closeable scope = scopedReadState()) { - Iterable>> entries = persistedData.get(); - if (Iterables.isEmpty(entries)) { - complete = true; - allKeysKnown = true; - return Iterables.unmodifiableIterable( - unnestCachedEntries(mergedCachedEntries(null).entrySet())); - } - if (!(entries instanceof Weighted)) { - return nonWeightedEntries(entries); - } - // This is a known amount of data, cache them all. - entries.forEach( - entry -> { - try { - final K originalKey = keyCoder.decode(entry.getKey().newInput(), Context.OUTER); - final Object structuralKey = keyCoder.structuralValue(originalKey); - KeyState keyState = - keyStateMap.computeIfAbsent(structuralKey, k -> new KeyState(originalKey)); - // Ignore any key from windmill that has been marked pending deletion or is - // fully cached. - if (keyState.existence == KeyExistence.KNOWN_NONEXISTENT - || (keyState.existence == KeyExistence.KNOWN_EXIST - && keyState.valuesCached)) return; - // Or else cache contents from windmill. - keyState.existence = KeyExistence.KNOWN_EXIST; - keyState.values.extendWith(entry.getValue()); - keyState.valuesSize += Iterables.size(entry.getValue()); - keyState.valuesCached = true; - } catch (IOException e) { - throw new RuntimeException(e); - } - }); - allKeysKnown = true; - complete = true; - return Iterables.unmodifiableIterable( - unnestCachedEntries(mergedCachedEntries(null).entrySet())); - } catch (InterruptedException | ExecutionException | IOException e) { - if (e instanceof InterruptedException) { - Thread.currentThread().interrupt(); - } - throw new RuntimeException("Unable to read state", e); - } - } - - @Override - @SuppressWarnings("FutureReturnValueIgnored") - public ReadableState>> readLater() { - WindmillMultimap.this.necessaryEntriesFromStorageFuture(false); - return this; - } - - // Collect all cached entries into a map and all KNOWN_NONEXISTENT keys to - // knownNonexistentKeys(if not null). Note that this method is not side-effect-free: it - // unloads any key that is not KNOWN_EXIST and not pending deletion from cache; also if - // complete it marks the valuesCached of any key that is KNOWN_EXIST to true, entries() - // depends on this behavior when the fetched result is weighted to iterate the whole - // keyStateMap one less time. For each cached key, returns its structural key and a tuple of - // . - private Map>> mergedCachedEntries( - Set knownNonexistentKeys) { - Map>> cachedEntries = Maps.newHashMap(); - keyStateMap - .entrySet() - .removeIf( - (entry -> { - Object structuralKey = entry.getKey(); - KeyState keyState = entry.getValue(); - if (complete && keyState.existence == KeyExistence.KNOWN_EXIST) { - keyState.valuesCached = true; - } - ConcatIterables it = null; - if (!keyState.localAdditions.isEmpty()) { - it = new ConcatIterables<>(); - it.extendWith( - Iterables.limit(keyState.localAdditions, keyState.localAdditions.size())); - } - if (keyState.valuesCached) { - if (it == null) it = new ConcatIterables<>(); - it.extendWith(Iterables.limit(keyState.values, keyState.valuesSize)); - } - if (it != null) { - cachedEntries.put( - structuralKey, - Triple.of(keyState.originalKey, keyState.valuesCached, it)); - } - if (knownNonexistentKeys != null - && keyState.existence == KeyExistence.KNOWN_NONEXISTENT) - knownNonexistentKeys.add(structuralKey); - return (keyState.existence == KeyExistence.KNOWN_NONEXISTENT - && !keyState.removedLocally) - || keyState.existence == KeyExistence.UNKNOWN_EXISTENCE; - })); - return cachedEntries; - } - - private Iterable> unnestCachedEntries( - Iterable>>> cachedEntries) { - return Iterables.concat( - Iterables.transform( - cachedEntries, - entry -> - Iterables.transform( - entry.getValue().getRight(), - v -> new AbstractMap.SimpleEntry<>(entry.getValue().getLeft(), v)))); - } - - private Iterable> nonWeightedEntries( - Iterable>> lazyWindmillEntries) { - class ResultIterable implements Iterable> { - private final Iterable>> lazyWindmillEntries; - private final Map>> cachedEntries; - private final Set knownNonexistentKeys; - - ResultIterable( - Map>> cachedEntries, - Iterable>> lazyWindmillEntries, - Set knownNonexistentKeys) { - this.cachedEntries = cachedEntries; - this.lazyWindmillEntries = lazyWindmillEntries; - this.knownNonexistentKeys = knownNonexistentKeys; - } - - @Override - public Iterator> iterator() { - // Each time when the Iterable returned by entries() is iterated, a new Iterator is - // created. Every iterator must keep its own copy of seenCachedKeys so that if a key - // is paginated into multiple iterables from windmill, the cached values of this key - // will only be returned once. - Set seenCachedKeys = Sets.newHashSet(); - // notFullyCachedEntries returns all entries from windmill that are not fully cached - // and combines them with localAdditions. If a key is fully cached, contents of this - // key from windmill are ignored. - Iterable>> notFullyCachedEntries = - Iterables.filter( - Iterables.transform( - lazyWindmillEntries, - entry -> { - try { - final K key = - keyCoder.decode(entry.getKey().newInput(), Context.OUTER); - final Object structuralKey = keyCoder.structuralValue(key); - // key is deleted in cache thus fully cached. - if (knownNonexistentKeys.contains(structuralKey)) return null; - Triple> triple = - cachedEntries.get(structuralKey); - // no record of key in cache, return content in windmill. - if (triple == null) { - return Triple.of(structuralKey, key, entry.getValue()); - } - // key is fully cached in cache. - if (triple.getMiddle()) return null; - - // key is not fully cached, combine the content in windmill with local - // additions with only the first observed page for the key to ensure - // it is not repeated. - if (!seenCachedKeys.add(structuralKey)) { - return Triple.of(structuralKey, key, entry.getValue()); - } else { - ConcatIterables it = new ConcatIterables<>(); - it.extendWith(triple.getRight()); - it.extendWith(entry.getValue()); - return Triple.of(structuralKey, key, it); - } - } catch (IOException e) { - throw new RuntimeException(e); - } - }), - Objects::nonNull); - Iterator> unnestWindmill = - Iterators.concat( - Iterables.transform( - notFullyCachedEntries, - entry -> - Iterables.transform( - entry.getRight(), - v -> new AbstractMap.SimpleEntry<>(entry.getMiddle(), v)) - .iterator()) - .iterator()); - Iterator> fullyCached = - unnestCachedEntries( - Iterables.filter( - cachedEntries.entrySet(), - entry -> !seenCachedKeys.contains(entry.getKey()))) - .iterator(); - return Iterators.concat(unnestWindmill, fullyCached); - } - } - - Set knownNonexistentKeys = Sets.newHashSet(); - Map>> cachedEntries = - mergedCachedEntries(knownNonexistentKeys); - return Iterables.unmodifiableIterable( - new ResultIterable(cachedEntries, lazyWindmillEntries, knownNonexistentKeys)); - } - }; - } - - @Override - public ReadableState containsKey(K key) { - return new ReadableState() { - ReadableState> values = null; - final Object structuralKey = keyCoder.structuralValue(key); - - @Override - public Boolean read() { - KeyState keyState = keyStateMap.getOrDefault(structuralKey, null); - if (keyState != null && keyState.existence != KeyExistence.UNKNOWN_EXISTENCE) { - return keyState.existence == KeyExistence.KNOWN_EXIST; - } - if (values == null) { - values = WindmillMultimap.this.get(key); - } - return !Iterables.isEmpty(values.read()); - } - - @Override - public ReadableState readLater() { - if (values == null) { - values = WindmillMultimap.this.get(key); - } - values.readLater(); - return this; - } - }; - } - - // Currently, isEmpty is implemented by reading all keys and could potentially be optimized. - // But note that if isEmpty is often followed by iterating over keys then maybe not too bad; if - // isEmpty is followed by iterating over both keys and values then it won't help much. - @Override - public ReadableState isEmpty() { - return new ReadableState() { - ReadableState> keys = null; - - @Override - public Boolean read() { - for (KeyState keyState : keyStateMap.values()) { - if (keyState.existence == KeyExistence.KNOWN_EXIST) return false; - } - if (keys == null) { - keys = WindmillMultimap.this.keys(); - } - return Iterables.isEmpty(keys.read()); - } - - @Override - public ReadableState readLater() { - if (keys == null) { - keys = WindmillMultimap.this.keys(); - } - keys.readLater(); - return this; - } - }; - } - } - - private static class WindmillBag extends SimpleWindmillState implements BagState { - - private final StateNamespace namespace; - private final StateTag> address; - private final ByteString stateKey; - private final String stateFamily; - private final Coder elemCoder; - - private boolean cleared = false; - /** - * If non-{@literal null}, this contains the complete contents of the bag, except for any local - * additions. If {@literal null} then we don't know if Windmill contains additional values which - * should be part of the bag. We'll need to read them if the work item actually wants the bag - * contents. - */ - private ConcatIterables cachedValues = null; - - private List localAdditions = new ArrayList<>(); - private long encodedSize = 0; - - private WindmillBag( - StateNamespace namespace, - StateTag> address, - String stateFamily, - Coder elemCoder, - boolean isNewKey) { - this.namespace = namespace; - this.address = address; - this.stateKey = encodeKey(namespace, address); - this.stateFamily = stateFamily; - this.elemCoder = elemCoder; - if (isNewKey) { - this.cachedValues = new ConcatIterables<>(); - } - } - - @Override - public void clear() { - cleared = true; - cachedValues = new ConcatIterables<>(); - localAdditions = new ArrayList<>(); - encodedSize = 0; - } - - /** - * Return iterable over all bag values in Windmill which should contribute to overall bag - * contents. - */ - private Iterable fetchData(Future> persistedData) { - try (Closeable scope = scopedReadState()) { - if (cachedValues != null) { - return cachedValues.snapshot(); - } - Iterable data = persistedData.get(); - if (data instanceof Weighted) { - // We have a known bounded amount of data; cache it. - cachedValues = new ConcatIterables<>(); - cachedValues.extendWith(data); - encodedSize = ((Weighted) data).getWeight(); - return cachedValues.snapshot(); - } else { - // This is an iterable that may not fit in memory at once; don't cache it. - return data; - } - } catch (InterruptedException | ExecutionException | IOException e) { - if (e instanceof InterruptedException) { - Thread.currentThread().interrupt(); - } - throw new RuntimeException("Unable to read state", e); - } - } - - public boolean valuesAreCached() { - return cachedValues != null; - } - - @Override - @SuppressWarnings("FutureReturnValueIgnored") - public WindmillBag readLater() { - getFuture(); - return this; - } - - @Override - public Iterable read() { - return Iterables.concat( - fetchData(getFuture()), Iterables.limit(localAdditions, localAdditions.size())); - } - - @Override - public ReadableState isEmpty() { - return new ReadableState() { - @Override - public ReadableState readLater() { - WindmillBag.this.readLater(); - return this; - } - - @Override - public Boolean read() { - return Iterables.isEmpty(fetchData(getFuture())) && localAdditions.isEmpty(); - } - }; - } - - @Override - public void add(T input) { - localAdditions.add(input); - } - - @Override - public WorkItemCommitRequest persistDirectly(WindmillStateCache.ForKeyAndFamily cache) - throws IOException { - WorkItemCommitRequest.Builder commitBuilder = WorkItemCommitRequest.newBuilder(); - - Windmill.TagBag.Builder bagUpdatesBuilder = null; - - if (cleared) { - bagUpdatesBuilder = commitBuilder.addBagUpdatesBuilder(); - bagUpdatesBuilder.setDeleteAll(true); - cleared = false; - } - - if (!localAdditions.isEmpty()) { - // Tell Windmill to capture the local additions. - if (bagUpdatesBuilder == null) { - bagUpdatesBuilder = commitBuilder.addBagUpdatesBuilder(); - } - for (T value : localAdditions) { - ByteStringOutputStream stream = new ByteStringOutputStream(); - // Encode the value - elemCoder.encode(value, stream, Coder.Context.OUTER); - ByteString encoded = stream.toByteString(); - if (cachedValues != null) { - // We'll capture this value in the cache below. - // Capture the value's size now since we have it. - encodedSize += encoded.size(); - } - bagUpdatesBuilder.addValues(encoded); - } - } - - if (bagUpdatesBuilder != null) { - bagUpdatesBuilder.setTag(stateKey).setStateFamily(stateFamily); - } - - if (cachedValues != null) { - if (!localAdditions.isEmpty()) { - // Capture the local additions in the cached value since we and - // Windmill are now in agreement. - cachedValues.extendWith(localAdditions); - } - // We now know the complete bag contents, and any read on it will yield a - // cached value, so cache it for future reads. - cache.put(namespace, address, this, encodedSize); - } - - // Don't reuse the localAdditions object; we don't want future changes to it to - // modify the value of cachedValues. - localAdditions = new ArrayList<>(); - - return commitBuilder.buildPartial(); - } - - private Future> getFuture() { - return cachedValues != null ? null : reader.bagFuture(stateKey, stateFamily, elemCoder); - } - } - - private static class ConcatIterables implements Iterable { - // List of component iterables. Should only be appended to in order to support snapshot(). - List> iterables; - - public ConcatIterables() { - this.iterables = new ArrayList<>(); - } - - public void extendWith(Iterable iterable) { - iterables.add(iterable); - } - - @Override - public Iterator iterator() { - return Iterators.concat(Iterables.transform(iterables, Iterable::iterator).iterator()); - } - - /** - * Returns a view of the current state of this iterable. Remembers the current length of - * iterables so that the returned value Will not change due to future extendWith() calls. - */ - public Iterable snapshot() { - final int limit = iterables.size(); - final List> iterablesList = iterables; - return () -> - Iterators.concat( - Iterators.transform( - Iterators.limit(iterablesList.iterator(), limit), Iterable::iterator)); - } - } - - private static class WindmillWatermarkHold extends WindmillState implements WatermarkHoldState { - // The encoded size of an Instant. - private static final int ENCODED_SIZE = 8; - - private final TimestampCombiner timestampCombiner; - private final StateNamespace namespace; - private final StateTag address; - private final ByteString stateKey; - private final String stateFamily; - - private boolean cleared = false; - /** - * If non-{@literal null}, the known current hold value, or absent if we know there are no - * output watermark holds. If {@literal null}, the current hold value could depend on holds in - * Windmill we do not yet know. - */ - private Optional cachedValue = null; - - private Instant localAdditions = null; - - private WindmillWatermarkHold( - StateNamespace namespace, - StateTag address, - String stateFamily, - TimestampCombiner timestampCombiner, - boolean isNewKey) { - this.namespace = namespace; - this.address = address; - this.stateKey = encodeKey(namespace, address); - this.stateFamily = stateFamily; - this.timestampCombiner = timestampCombiner; - if (isNewKey) { - cachedValue = Optional.absent(); - } - } - - @Override - public void clear() { - cleared = true; - cachedValue = Optional.absent(); - localAdditions = null; - } - - @Override - @SuppressWarnings("FutureReturnValueIgnored") - public WindmillWatermarkHold readLater() { - getFuture(); - return this; - } - - @Override - public Instant read() { - try (Closeable scope = scopedReadState()) { - Instant persistedHold = getFuture().get(); - if (persistedHold == null) { - cachedValue = Optional.absent(); - } else { - cachedValue = Optional.of(persistedHold); - } - } catch (InterruptedException | ExecutionException | IOException e) { - if (e instanceof InterruptedException) { - Thread.currentThread().interrupt(); - } - throw new RuntimeException("Unable to read state", e); - } - - if (localAdditions == null) { - return cachedValue.orNull(); - } else if (!cachedValue.isPresent()) { - return localAdditions; - } else { - return timestampCombiner.combine(localAdditions, cachedValue.get()); - } - } - - @Override - public ReadableState isEmpty() { - throw new UnsupportedOperationException(); - } - - @Override - public void add(Instant outputTime) { - localAdditions = - (localAdditions == null) - ? outputTime - : timestampCombiner.combine(outputTime, localAdditions); - } - - @Override - public TimestampCombiner getTimestampCombiner() { - return timestampCombiner; - } - - @Override - public Future persist(final WindmillStateCache.ForKeyAndFamily cache) { - - Future result; - - if (!cleared && localAdditions == null) { - // No changes, so no need to update Windmill and no need to cache any value. - return Futures.immediateFuture(WorkItemCommitRequest.newBuilder().buildPartial()); - } - - if (cleared && localAdditions == null) { - // Just clearing the persisted state; blind delete - WorkItemCommitRequest.Builder commitBuilder = WorkItemCommitRequest.newBuilder(); - commitBuilder - .addWatermarkHoldsBuilder() - .setTag(stateKey) - .setStateFamily(stateFamily) - .setReset(true); - - result = Futures.immediateFuture(commitBuilder.buildPartial()); - } else if (cleared && localAdditions != null) { - // Since we cleared before adding, we can do a blind overwrite of persisted state - WorkItemCommitRequest.Builder commitBuilder = WorkItemCommitRequest.newBuilder(); - commitBuilder - .addWatermarkHoldsBuilder() - .setTag(stateKey) - .setStateFamily(stateFamily) - .setReset(true) - .addTimestamps(WindmillTimeUtils.harnessToWindmillTimestamp(localAdditions)); - - cachedValue = Optional.of(localAdditions); - - result = Futures.immediateFuture(commitBuilder.buildPartial()); - } else if (!cleared && localAdditions != null) { - // Otherwise, we need to combine the local additions with the already persisted data - result = combineWithPersisted(); - } else { - throw new IllegalStateException("Unreachable condition"); - } - - return Futures.lazyTransform( - result, - result1 -> { - cleared = false; - localAdditions = null; - if (cachedValue != null) { - cache.put(namespace, address, WindmillWatermarkHold.this, ENCODED_SIZE); - } - return result1; - }); - } - - private Future getFuture() { - return cachedValue != null - ? Futures.immediateFuture(cachedValue.orNull()) - : reader.watermarkFuture(stateKey, stateFamily); - } - - /** - * Combines local additions with persisted data and mutates the {@code commitBuilder} to write - * the result. - */ - private Future combineWithPersisted() { - boolean windmillCanCombine = false; - - // If the combined output time depends only on the window, then we are just blindly adding - // the same value that may or may not already be present. This depends on the state only being - // used for one window. - windmillCanCombine |= timestampCombiner.dependsOnlyOnWindow(); - - // If the combined output time depends only on the earliest input timestamp, then because - // assignOutputTime is monotonic, the hold only depends on the earliest output timestamp - // (which is the value submitted as a watermark hold). The only way holds for later inputs - // can be redundant is if the are later (or equal) to the earliest. So taking the MIN - // implicitly, as Windmill does, has the desired behavior. - windmillCanCombine |= timestampCombiner.dependsOnlyOnEarliestTimestamp(); - - if (windmillCanCombine) { - // We do a blind write and let Windmill take the MIN - WorkItemCommitRequest.Builder commitBuilder = WorkItemCommitRequest.newBuilder(); - commitBuilder - .addWatermarkHoldsBuilder() - .setTag(stateKey) - .setStateFamily(stateFamily) - .addTimestamps(WindmillTimeUtils.harnessToWindmillTimestamp(localAdditions)); - - if (cachedValue != null) { - cachedValue = - Optional.of( - cachedValue.isPresent() - ? timestampCombiner.combine(cachedValue.get(), localAdditions) - : localAdditions); - } - - return Futures.immediateFuture(commitBuilder.buildPartial()); - } else { - // The non-fast path does a read-modify-write - return Futures.lazyTransform( - (cachedValue != null) - ? Futures.immediateFuture(cachedValue.orNull()) - : reader.watermarkFuture(stateKey, stateFamily), - priorHold -> { - cachedValue = - Optional.of( - (priorHold != null) - ? timestampCombiner.combine(priorHold, localAdditions) - : localAdditions); - WorkItemCommitRequest.Builder commitBuilder = WorkItemCommitRequest.newBuilder(); - commitBuilder - .addWatermarkHoldsBuilder() - .setTag(stateKey) - .setStateFamily(stateFamily) - .setReset(true) - .addTimestamps(WindmillTimeUtils.harnessToWindmillTimestamp(cachedValue.get())); - - return commitBuilder.buildPartial(); - }); - } - } - } - - private static class WindmillCombiningState extends WindmillState - implements CombiningState { - - private final WindmillBag bag; - private final CombineFn combineFn; - - /* We use a separate, in-memory AccumT rather than relying on the WindmillWatermarkBag's - * localAdditions, because we want to combine multiple InputT's to a single AccumT - * before adding it. - */ - private AccumT localAdditionsAccum; - private boolean hasLocalAdditions = false; - - private WindmillCombiningState( - StateNamespace namespace, - StateTag> address, - String stateFamily, - Coder accumCoder, - CombineFn combineFn, - WindmillStateCache.ForKeyAndFamily cache, - boolean isNewKey) { - StateTag> internalBagAddress = StateTags.convertToBagTagInternal(address); - WindmillBag cachedBag = - (WindmillBag) cache.get(namespace, internalBagAddress); - this.bag = - (cachedBag != null) - ? cachedBag - : new WindmillBag<>(namespace, internalBagAddress, stateFamily, accumCoder, isNewKey); - this.combineFn = combineFn; - this.localAdditionsAccum = combineFn.createAccumulator(); - } - - @Override - void initializeForWorkItem( - WindmillStateReader reader, Supplier scopedReadStateSupplier) { - super.initializeForWorkItem(reader, scopedReadStateSupplier); - this.bag.initializeForWorkItem(reader, scopedReadStateSupplier); - } - - @Override - void cleanupAfterWorkItem() { - super.cleanupAfterWorkItem(); - bag.cleanupAfterWorkItem(); - } - - @Override - public WindmillCombiningState readLater() { - bag.readLater(); - return this; - } - - @Override - public OutputT read() { - return combineFn.extractOutput(getAccum()); - } - - @Override - public void add(InputT input) { - hasLocalAdditions = true; - localAdditionsAccum = combineFn.addInput(localAdditionsAccum, input); - } - - @Override - public void clear() { - bag.clear(); - localAdditionsAccum = combineFn.createAccumulator(); - hasLocalAdditions = false; - } - - @Override - public Future persist(WindmillStateCache.ForKeyAndFamily cache) - throws IOException { - if (hasLocalAdditions) { - if (COMPACT_NOW.get().get() || bag.valuesAreCached()) { - // Implicitly clears the bag and combines local and persisted accumulators. - localAdditionsAccum = getAccum(); - } - bag.add(combineFn.compact(localAdditionsAccum)); - localAdditionsAccum = combineFn.createAccumulator(); - hasLocalAdditions = false; - } - - return bag.persist(cache); - } - - @Override - public AccumT getAccum() { - Iterable accums = - Iterables.concat(bag.read(), Collections.singleton(localAdditionsAccum)); - - // Compact things - AccumT merged = combineFn.mergeAccumulators(accums); - bag.clear(); - localAdditionsAccum = merged; - hasLocalAdditions = true; - return merged; - } - - @Override - public ReadableState isEmpty() { - final ReadableState bagIsEmpty = bag.isEmpty(); - return new ReadableState() { - @Override - public ReadableState readLater() { - bagIsEmpty.readLater(); - return this; - } - - @Override - public Boolean read() { - return !hasLocalAdditions && bagIsEmpty.read(); - } - }; - } - - @Override - public void addAccum(AccumT accum) { - hasLocalAdditions = true; - localAdditionsAccum = combineFn.mergeAccumulators(Arrays.asList(localAdditionsAccum, accum)); - } - - @Override - public AccumT mergeAccumulators(Iterable accumulators) { - return combineFn.mergeAccumulators(accumulators); - } - } - - @VisibleForTesting - static final ThreadLocal> COMPACT_NOW = - ThreadLocal.withInitial( - () -> - new Supplier() { - /* The rate at which, on average, this will return true. */ - static final double RATE = 0.002; - Random random = new Random(); - long counter = nextSample(); - - private long nextSample() { - // Use geometric distribution to find next true value. - // This lets us avoid invoking random.nextDouble() on every call. - return (long) Math.floor(Math.log(random.nextDouble()) / Math.log(1 - RATE)); - } - - @Override - public Boolean get() { - counter--; - if (counter < 0) { - counter = nextSample(); - return true; - } else { - return false; - } - } - }); -} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkState.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkState.java index 529bb0a41907..9858666c40a2 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkState.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkState.java @@ -32,9 +32,9 @@ import javax.annotation.Nullable; import javax.annotation.concurrent.GuardedBy; import javax.annotation.concurrent.ThreadSafe; -import org.apache.beam.runners.dataflow.worker.WindmillStateCache; import org.apache.beam.runners.dataflow.worker.windmill.Windmill; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.KeyedGetDataRequest; +import org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateCache; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ComputationState.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ComputationState.java index a902d2b13a77..9d7a9131f584 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ComputationState.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ComputationState.java @@ -23,9 +23,9 @@ import java.util.Map; import java.util.concurrent.ConcurrentLinkedQueue; import javax.annotation.Nullable; -import org.apache.beam.runners.dataflow.worker.WindmillStateCache; import org.apache.beam.runners.dataflow.worker.util.BoundedQueueExecutor; import org.apache.beam.runners.dataflow.worker.windmill.Windmill; +import org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateCache; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.joda.time.Instant; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/CachingStateTable.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/CachingStateTable.java new file mode 100644 index 000000000000..bcaf8bf21a2d --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/CachingStateTable.java @@ -0,0 +1,275 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.dataflow.worker.windmill.state; + +import java.io.Closeable; +import java.util.Optional; +import javax.annotation.Nullable; +import org.apache.beam.runners.core.StateNamespace; +import org.apache.beam.runners.core.StateTable; +import org.apache.beam.runners.core.StateTag; +import org.apache.beam.runners.core.StateTags; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.state.BagState; +import org.apache.beam.sdk.state.CombiningState; +import org.apache.beam.sdk.state.MapState; +import org.apache.beam.sdk.state.MultimapState; +import org.apache.beam.sdk.state.OrderedListState; +import org.apache.beam.sdk.state.SetState; +import org.apache.beam.sdk.state.State; +import org.apache.beam.sdk.state.StateContext; +import org.apache.beam.sdk.state.ValueState; +import org.apache.beam.sdk.state.WatermarkHoldState; +import org.apache.beam.sdk.transforms.Combine; +import org.apache.beam.sdk.transforms.CombineWithContext; +import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; +import org.apache.beam.sdk.util.CombineFnUtil; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Supplier; + +final class CachingStateTable extends StateTable { + private final String stateFamily; + private final WindmillStateReader reader; + private final WindmillStateCache.ForKeyAndFamily cache; + private final boolean isSystemTable; + private final Supplier scopedReadStateSupplier; + private final @Nullable StateTable derivedStateTable; + private final boolean isNewKey; + + private CachingStateTable(Builder builder) { + this.stateFamily = builder.stateFamily; + this.reader = builder.reader; + this.cache = builder.cache; + this.isSystemTable = builder.isSystemTable; + this.isNewKey = builder.isNewKey; + this.scopedReadStateSupplier = builder.scopedReadStateSupplier; + this.derivedStateTable = builder.derivedStateTable; + + if (this.isSystemTable) { + Preconditions.checkState(derivedStateTable == null); + } else { + Preconditions.checkNotNull(this.derivedStateTable); + } + } + + static CachingStateTable.Builder builder( + String stateFamily, + WindmillStateReader reader, + WindmillStateCache.ForKeyAndFamily cache, + boolean isNewKey, + Supplier scopedReadStateSupplier) { + return new CachingStateTable.Builder( + stateFamily, reader, cache, scopedReadStateSupplier, isNewKey); + } + + @Override + @SuppressWarnings("deprecation") + protected StateTag.StateBinder binderForNamespace(StateNamespace namespace, StateContext c) { + // Look up state objects in the cache or create new ones if not found. The state will + // be added to the cache in persist(). + return new StateTag.StateBinder() { + @Override + public BagState bindBag(StateTag> address, Coder elemCoder) { + StateTag> resolvedAddress = + isSystemTable ? StateTags.makeSystemTagInternal(address) : address; + + WindmillBag result = + cache + .get(namespace, resolvedAddress) + .map(bagState -> (WindmillBag) bagState) + .orElseGet( + () -> + new WindmillBag<>( + namespace, resolvedAddress, stateFamily, elemCoder, isNewKey)); + + result.initializeForWorkItem(reader, scopedReadStateSupplier); + return result; + } + + @Override + public SetState bindSet(StateTag> spec, Coder elemCoder) { + WindmillSet result = + new WindmillSet<>(namespace, spec, stateFamily, elemCoder, cache, isNewKey); + result.initializeForWorkItem(reader, scopedReadStateSupplier); + return result; + } + + @Override + public MapState bindMap( + StateTag> spec, Coder keyCoder, Coder valueCoder) { + WindmillMap result = + cache + .get(namespace, spec) + .map(mapState -> (WindmillMap) mapState) + .orElseGet( + () -> + new WindmillMap<>( + namespace, spec, stateFamily, keyCoder, valueCoder, isNewKey)); + + result.initializeForWorkItem(reader, scopedReadStateSupplier); + return result; + } + + @Override + public MultimapState bindMultimap( + StateTag> spec, + Coder keyCoder, + Coder valueCoder) { + WindmillMultimap result = + cache + .get(namespace, spec) + .map(multimapState -> (WindmillMultimap) multimapState) + .orElseGet( + () -> + new WindmillMultimap<>( + namespace, spec, stateFamily, keyCoder, valueCoder, isNewKey)); + result.initializeForWorkItem(reader, scopedReadStateSupplier); + return result; + } + + @Override + public OrderedListState bindOrderedList( + StateTag> spec, Coder elemCoder) { + StateTag> specOrInternalTag = addressOrInternalTag(spec); + + WindmillOrderedList result = + cache + .get(namespace, specOrInternalTag) + .map(orderedList -> (WindmillOrderedList) orderedList) + .orElseGet( + () -> + new WindmillOrderedList<>( + Optional.ofNullable(derivedStateTable).orElse(CachingStateTable.this), + namespace, + specOrInternalTag, + stateFamily, + elemCoder, + isNewKey)); + + result.initializeForWorkItem(reader, scopedReadStateSupplier); + return result; + } + + @Override + public WatermarkHoldState bindWatermark( + StateTag address, TimestampCombiner timestampCombiner) { + StateTag addressOrInternalTag = addressOrInternalTag(address); + + WindmillWatermarkHold result = + cache + .get(namespace, addressOrInternalTag) + .map(watermarkHold -> (WindmillWatermarkHold) watermarkHold) + .orElseGet( + () -> + new WindmillWatermarkHold( + namespace, address, stateFamily, timestampCombiner, isNewKey)); + + result.initializeForWorkItem(reader, scopedReadStateSupplier); + return result; + } + + @Override + public CombiningState bindCombiningValue( + StateTag> address, + Coder accumCoder, + Combine.CombineFn combineFn) { + StateTag> addressOrInternalTag = + addressOrInternalTag(address); + + WindmillCombiningState result = + new WindmillCombiningState<>( + namespace, + addressOrInternalTag, + stateFamily, + accumCoder, + combineFn, + cache, + isNewKey); + + result.initializeForWorkItem(reader, scopedReadStateSupplier); + return result; + } + + @Override + public + CombiningState bindCombiningValueWithContext( + StateTag> address, + Coder accumCoder, + CombineWithContext.CombineFnWithContext combineFn) { + return bindCombiningValue( + addressOrInternalTag(address), accumCoder, CombineFnUtil.bindContext(combineFn, c)); + } + + @Override + public ValueState bindValue(StateTag> address, Coder coder) { + StateTag> addressOrInternalTag = addressOrInternalTag(address); + + WindmillValue result = + cache + .get(namespace, addressOrInternalTag) + .map(value -> (WindmillValue) value) + .orElseGet( + () -> + new WindmillValue<>( + namespace, addressOrInternalTag, stateFamily, coder, isNewKey)); + + result.initializeForWorkItem(reader, scopedReadStateSupplier); + return result; + } + + private StateTag addressOrInternalTag(StateTag address) { + return isSystemTable ? StateTags.makeSystemTagInternal(address) : address; + } + }; + } + + static class Builder { + private final String stateFamily; + private final WindmillStateReader reader; + private final WindmillStateCache.ForKeyAndFamily cache; + private final Supplier scopedReadStateSupplier; + private final boolean isNewKey; + private boolean isSystemTable; + private @Nullable StateTable derivedStateTable; + + private Builder( + String stateFamily, + WindmillStateReader reader, + WindmillStateCache.ForKeyAndFamily cache, + Supplier scopedReadStateSupplier, + boolean isNewKey) { + this.stateFamily = stateFamily; + this.reader = reader; + this.cache = cache; + this.scopedReadStateSupplier = scopedReadStateSupplier; + this.isNewKey = isNewKey; + this.isSystemTable = true; + this.derivedStateTable = null; + } + + Builder withDerivedState(StateTable derivedStateTable) { + this.isSystemTable = false; + this.derivedStateTable = derivedStateTable; + return this; + } + + CachingStateTable build() { + return new CachingStateTable(this); + } + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/ConcatIterables.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/ConcatIterables.java new file mode 100644 index 000000000000..4bb806bd70fd --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/ConcatIterables.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.dataflow.worker.windmill.state; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterators; + +class ConcatIterables implements Iterable { + // List of component iterables. Should only be appended to in order to support snapshot(). + private final List> iterables; + + public ConcatIterables() { + this.iterables = new ArrayList<>(); + } + + public void extendWith(Iterable iterable) { + iterables.add(iterable); + } + + @Override + public Iterator iterator() { + return Iterators.concat(Iterables.transform(iterables, Iterable::iterator).iterator()); + } + + /** + * Returns a view of the current state of this iterable. Remembers the current length of iterables + * so that the returned value Will not change due to future extendWith() calls. + */ + public Iterable snapshot() { + final int limit = iterables.size(); + final List> iterablesList = iterables; + return () -> + Iterators.concat( + Iterators.transform( + Iterators.limit(iterablesList.iterator(), limit), Iterable::iterator)); + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/IdTracker.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/IdTracker.java new file mode 100644 index 000000000000..5090626ae8ee --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/IdTracker.java @@ -0,0 +1,253 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.dataflow.worker.windmill.state; + +import java.util.Comparator; +import java.util.Iterator; +import java.util.Map; +import java.util.SortedSet; +import java.util.concurrent.ExecutionException; +import java.util.function.BiConsumer; +import org.apache.beam.runners.core.StateNamespace; +import org.apache.beam.runners.core.StateTable; +import org.apache.beam.runners.core.StateTag; +import org.apache.beam.runners.core.StateTags; +import org.apache.beam.sdk.coders.InstantCoder; +import org.apache.beam.sdk.coders.MapCoder; +import org.apache.beam.sdk.coders.VarLongCoder; +import org.apache.beam.sdk.state.StateContexts; +import org.apache.beam.sdk.state.ValueState; +import org.apache.beam.sdk.values.TimestampedValue; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Range; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.RangeSet; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.TreeRangeSet; +import org.joda.time.Duration; +import org.joda.time.Instant; + +/** + * Tracker for the ids used in an ordered list. + * + *

Windmill accepts an int64 id for each timestamped-element in the list. Unique elements are + * identified by the pair of timestamp and id. This means that tow unique elements e1, e2 must have + * different (ts1, id1), (ts2, id2) pairs. To accomplish this we bucket time into five-minute + * buckets, and store a free list of ids available for each bucket. + * + *

When a timestamp range is deleted, we remove id tracking for elements in that range. In order + * to handle the case where a range is deleted piecemeal, we track sub-range deletions for each + * range. For example: + * + *

12:00 - 12:05 ids 12:05 - 12:10 ids + * + *

delete 12:00-12:06 + * + *

12:00 - 12:05 *removed* 12:05 - 12:10 ids subranges deleted 12:05-12:06 + * + *

delete 12:06 - 12:07 + * + *

12:05 - 12:10 ids subranges deleted 12:05-12:07 + * + *

delete 12:07 - 12:10 + * + *

12:05 - 12:10 *removed* + */ +@SuppressWarnings("nullness" // TODO(https://github.com/apache/beam/issues/20497) +) +final class IdTracker { + @VisibleForTesting static final String IDS_AVAILABLE_STR = "IdsAvailable"; + @VisibleForTesting static final String DELETIONS_STR = "Deletions"; + // Note that this previously was Long.MIN_VALUE but ids are unsigned when + // sending to windmill for Streaming Engine. For updated appliance + // pipelines with existing state, there may be negative ids. + @VisibleForTesting static final long NEW_RANGE_MIN_ID = 0; + + @VisibleForTesting + static final MapCoder, RangeSet> IDS_AVAILABLE_CODER = + MapCoder.of(new RangeCoder<>(InstantCoder.of()), new RangeSetCoder<>(VarLongCoder.of())); + + @VisibleForTesting + static final MapCoder, RangeSet> SUBRANGE_DELETIONS_CODER = + MapCoder.of(new RangeCoder<>(InstantCoder.of()), new RangeSetCoder<>(InstantCoder.of())); + + private static final long NEW_RANGE_MAX_ID = Long.MAX_VALUE; + // We track ids on five-minute boundaries. + private static final Duration RESOLUTION = Duration.standardMinutes(5); + // A map from five-minute ranges to the set of ids available in that interval. + private final ValueState, RangeSet>> idsAvailableValue; + // If a timestamp-range in the map has been partially cleared, the cleared intervals are stored + // here. + private final ValueState, RangeSet>> subRangeDeletionsValue; + + IdTracker(StateTable stateTable, StateNamespace namespace, StateTag spec) { + StateTag, RangeSet>>> idsAvailableTag = + StateTags.makeSystemTagInternal( + StateTags.value(spec.getId() + IDS_AVAILABLE_STR, IDS_AVAILABLE_CODER)); + StateTag, RangeSet>>> subRangeDeletionsTag = + StateTags.makeSystemTagInternal( + StateTags.value(spec.getId() + DELETIONS_STR, SUBRANGE_DELETIONS_CODER)); + + this.idsAvailableValue = + stateTable.get(namespace, idsAvailableTag, StateContexts.nullContext()); + this.subRangeDeletionsValue = + stateTable.get(namespace, subRangeDeletionsTag, StateContexts.nullContext()); + } + + static > + Map, RangeSet> newSortedRangeMap() { + return Maps.newTreeMap( + Comparator., Instant>comparing(Range::lowerEndpoint) + .thenComparing(Range::upperEndpoint)); + } + + private Range getTrackedRange(Instant ts) { + Instant snapped = + new Instant(ts.getMillis() - ts.plus(RESOLUTION).getMillis() % RESOLUTION.getMillis()); + return Range.closedOpen(snapped, snapped.plus(RESOLUTION)); + } + + @SuppressWarnings("FutureReturnValueIgnored") + void readLater() { + idsAvailableValue.readLater(); + subRangeDeletionsValue.readLater(); + } + + Map, RangeSet> readIdsAvailable() { + Map, RangeSet> idsAvailable = idsAvailableValue.read(); + return idsAvailable != null ? idsAvailable : newSortedRangeMap(); + } + + Map, RangeSet> readSubRangeDeletions() { + Map, RangeSet> subRangeDeletions = subRangeDeletionsValue.read(); + return subRangeDeletions != null ? subRangeDeletions : newSortedRangeMap(); + } + + void clear() throws ExecutionException, InterruptedException { + idsAvailableValue.clear(); + subRangeDeletionsValue.clear(); + } + + void add( + SortedSet> elements, BiConsumer, Long> output) + throws ExecutionException, InterruptedException { + Range currentIdRange = null; + long currentId = 0; + + Range currentTsRange = null; + RangeSet currentTsRangeDeletions = null; + + Map, RangeSet> idsAvailable = readIdsAvailable(); + Map, RangeSet> subRangeDeletions = readSubRangeDeletions(); + + RangeSet availableIdsForTsRange = null; + Iterator> idRangeIter = null; + RangeSet idsUsed = TreeRangeSet.create(); + for (TimestampedValueWithId pendingAdd : elements) { + // Since elements are in increasing ts order, often we'll be able to reuse the previous + // iteration's range. + if (currentTsRange == null + || !currentTsRange.contains(pendingAdd.getValue().getTimestamp())) { + if (availableIdsForTsRange != null) { + // We're moving onto a new ts range. Remove all used ids + availableIdsForTsRange.removeAll(idsUsed); + idsUsed = TreeRangeSet.create(); + } + + // Lookup the range for the current timestamp. + currentTsRange = getTrackedRange(pendingAdd.getValue().getTimestamp()); + // Lookup available ids for this timestamp range. If nothing there, we default to all ids + // available. + availableIdsForTsRange = + idsAvailable.computeIfAbsent( + currentTsRange, + r -> + TreeRangeSet.create( + ImmutableList.of(Range.closedOpen(NEW_RANGE_MIN_ID, NEW_RANGE_MAX_ID)))); + idRangeIter = availableIdsForTsRange.asRanges().iterator(); + currentIdRange = null; + currentTsRangeDeletions = subRangeDeletions.get(currentTsRange); + } + + if (currentIdRange == null || currentId >= currentIdRange.upperEndpoint()) { + // Move to the next range of free ids, and start assigning ranges from there. + currentIdRange = idRangeIter.next(); + currentId = currentIdRange.lowerEndpoint(); + } + + if (currentTsRangeDeletions != null) { + currentTsRangeDeletions.remove( + Range.closedOpen( + pendingAdd.getValue().getTimestamp(), + pendingAdd.getValue().getTimestamp().plus(Duration.millis(1)))); + } + idsUsed.add(Range.closedOpen(currentId, currentId + 1)); + output.accept(pendingAdd.getValue(), currentId++); + } + if (availableIdsForTsRange != null) { + availableIdsForTsRange.removeAll(idsUsed); + } + writeValues(idsAvailable, subRangeDeletions); + } + + // Remove a timestamp range. Returns ids freed up. + void remove(Range tsRange) throws ExecutionException, InterruptedException { + Map, RangeSet> idsAvailable = readIdsAvailable(); + Map, RangeSet> subRangeDeletions = readSubRangeDeletions(); + + for (Range current = getTrackedRange(tsRange.lowerEndpoint()); + current.lowerEndpoint().isBefore(tsRange.upperEndpoint()); + current = getTrackedRange(current.lowerEndpoint().plus(RESOLUTION))) { + // TODO(reuvenlax): shouldn't need to iterate over all ranges. + boolean rangeCleared; + if (!tsRange.encloses(current)) { + // This can happen if the beginning or the end of tsRange doesn't fall on a RESOLUTION + // boundary. Since we are deleting a portion of a tracked range, track what we are deleting. + RangeSet rangeDeletions = + subRangeDeletions.computeIfAbsent(current, r -> TreeRangeSet.create()); + rangeDeletions.add(tsRange.intersection(current)); + // If we ended up deleting the whole range, then we can simply remove it from the tracking + // map. + rangeCleared = rangeDeletions.encloses(current); + } else { + rangeCleared = true; + } + if (rangeCleared) { + // Remove the range from both maps. + idsAvailable.remove(current); + subRangeDeletions.remove(current); + } + } + writeValues(idsAvailable, subRangeDeletions); + } + + private void writeValues( + Map, RangeSet> idsAvailable, + Map, RangeSet> subRangeDeletions) { + if (idsAvailable.isEmpty()) { + idsAvailable.clear(); + } else { + idsAvailableValue.write(idsAvailable); + } + if (subRangeDeletions.isEmpty()) { + subRangeDeletionsValue.clear(); + } else { + subRangeDeletionsValue.write(subRangeDeletions); + } + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/PagingIterable.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/PagingIterable.java new file mode 100644 index 000000000000..73f076d92013 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/PagingIterable.java @@ -0,0 +1,129 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.dataflow.worker.windmill.state; + +import java.util.Iterator; +import java.util.List; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.AbstractIterator; + +/** + * An iterable over elements backed by paginated GetData requests to Windmill. The iterable may be + * iterated over an arbitrary number of times and multiple iterators may be active simultaneously. + * + *

There are two pattern we wish to support with low -memory and -latency: + * + *

    + *
  1. Re-iterate over the initial elements multiple times (eg Iterables.first). We'll cache the + * initial 'page' of values returned by Windmill from our first request for the lifetime of + * the iterable. + *
  2. Iterate through all elements of a very large collection. We'll send the GetData request for + * the next page when the current page is begun. We'll discard intermediate pages and only + * retain the first. Thus the maximum memory pressure is one page plus one page per call to + * iterator. + *
+ */ +@SuppressWarnings({ + "nullness" // TODO(https://github.com/apache/beam/issues/20497) +}) +class PagingIterable implements Iterable { + /** + * The reader we will use for scheduling continuation pages. + * + *

NOTE We've made this explicit to remind us to be careful not to cache the iterable. + */ + private final WindmillStateReader reader; + + /** Initial values returned for the first page. Never reclaimed. */ + private final List firstPage; + + /** State tag with continuation position set for second page. */ + private final StateTag secondPagePos; + + /** Coder for elements. */ + private final Coder coder; + + PagingIterable( + WindmillStateReader reader, + List firstPage, + StateTag secondPagePos, + Coder coder) { + this.reader = reader; + this.firstPage = firstPage; + this.secondPagePos = secondPagePos; + this.coder = coder; + } + + @Override + public Iterator iterator() { + return new PagingIterableIterator(); + } + + private class PagingIterableIterator extends AbstractIterator { + private Iterator currentPage = firstPage.iterator(); + private StateTag nextPagePos = secondPagePos; + private Future> pendingNextPage = + // NOTE: The results of continuation page reads are never cached. + reader.continuationFuture(nextPagePos, coder); + + @Override + protected ResultT computeNext() { + while (true) { + if (currentPage.hasNext()) { + return currentPage.next(); + } + if (pendingNextPage == null) { + return endOfData(); + } + + ValuesAndContPosition valuesAndContPosition; + try { + valuesAndContPosition = pendingNextPage.get(); + } catch (InterruptedException | ExecutionException e) { + if (e instanceof InterruptedException) { + Thread.currentThread().interrupt(); + } + throw new RuntimeException("Unable to read value from state", e); + } + currentPage = valuesAndContPosition.getValues().iterator(); + StateTag.Builder nextPageBuilder = + StateTag.of( + nextPagePos.getKind(), + nextPagePos.getTag(), + nextPagePos.getStateFamily(), + valuesAndContPosition.getContinuationPosition()) + .toBuilder(); + if (secondPagePos.getSortedListRange() != null) { + nextPageBuilder.setSortedListRange(secondPagePos.getSortedListRange()); + } + if (secondPagePos.getOmitValues() != null) { + nextPageBuilder.setOmitValues(secondPagePos.getOmitValues()); + } + if (secondPagePos.getMultimapKey() != null) { + nextPageBuilder.setMultimapKey(secondPagePos.getMultimapKey()); + } + nextPagePos = nextPageBuilder.build(); + pendingNextPage = + // NOTE: The results of continuation page reads are never cached. + reader.continuationFuture(nextPagePos, coder); + } + } + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/RangeCoder.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/RangeCoder.java new file mode 100644 index 000000000000..0e11531226f7 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/RangeCoder.java @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.dataflow.worker.windmill.state; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.List; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.NullableCoder; +import org.apache.beam.sdk.coders.StructuredCoder; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.BoundType; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Range; +import org.checkerframework.checker.nullness.qual.Nullable; + +@SuppressWarnings({ + "nullness" // TODO(https://github.com/apache/beam/issues/20497) +}) +/** Coder for closed-open ranges. */ +class RangeCoder> extends StructuredCoder> { + private final Coder boundCoder; + + RangeCoder(Coder boundCoder) { + this.boundCoder = NullableCoder.of(boundCoder); + } + + @Override + public List> getCoderArguments() { + return Lists.newArrayList(boundCoder); + } + + @Override + public void verifyDeterministic() throws NonDeterministicException { + boundCoder.verifyDeterministic(); + } + + @Override + public void encode(Range value, OutputStream outStream) throws IOException { + Preconditions.checkState( + value.lowerBoundType().equals(BoundType.CLOSED), "unexpected range " + value); + Preconditions.checkState( + value.upperBoundType().equals(BoundType.OPEN), "unexpected range " + value); + boundCoder.encode(value.hasLowerBound() ? value.lowerEndpoint() : null, outStream); + boundCoder.encode(value.hasUpperBound() ? value.upperEndpoint() : null, outStream); + } + + @Override + public Range decode(InputStream inStream) throws IOException { + @Nullable T lower = boundCoder.decode(inStream); + @Nullable T upper = boundCoder.decode(inStream); + if (lower == null) { + return upper != null ? Range.lessThan(upper) : Range.all(); + } else if (upper == null) { + return Range.atLeast(lower); + } else { + return Range.closedOpen(lower, upper); + } + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/RangeSetCoder.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/RangeSetCoder.java new file mode 100644 index 000000000000..291a83e81ee6 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/RangeSetCoder.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.dataflow.worker.windmill.state; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CustomCoder; +import org.apache.beam.sdk.coders.SetCoder; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Range; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.RangeSet; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.TreeRangeSet; + +class RangeSetCoder> extends CustomCoder> { + private final SetCoder> rangesCoder; + + RangeSetCoder(Coder boundCoder) { + this.rangesCoder = SetCoder.of(new RangeCoder<>(boundCoder)); + } + + @Override + public void encode(RangeSet value, OutputStream outStream) throws IOException { + rangesCoder.encode(value.asRanges(), outStream); + } + + @Override + public RangeSet decode(InputStream inStream) throws IOException { + return TreeRangeSet.create(rangesCoder.decode(inStream)); + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/SimpleWindmillState.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/SimpleWindmillState.java new file mode 100644 index 000000000000..bd7f8041c680 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/SimpleWindmillState.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.dataflow.worker.windmill.state; + +import java.io.IOException; +import java.util.concurrent.Future; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.Futures; + +/** + * Base class for implementations of {@link WindmillState} where the {@link #persist} call does not + * require any asynchronous reading. + */ +abstract class SimpleWindmillState extends WindmillState { + @Override + public final Future persist( + WindmillStateCache.ForKeyAndFamily cache) throws IOException { + return Futures.immediateFuture(persistDirectly(cache)); + } + + /** + * Returns a {@link Windmill.WorkItemCommitRequest} that can be used to persist this state to + * Windmill. + */ + protected abstract Windmill.WorkItemCommitRequest persistDirectly( + WindmillStateCache.ForKeyAndFamily cache) throws IOException; +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/StateTag.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/StateTag.java new file mode 100644 index 000000000000..13c2a9e66baa --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/StateTag.java @@ -0,0 +1,109 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.dataflow.worker.windmill.state; + +import com.google.auto.value.AutoValue; +import javax.annotation.Nullable; +import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Range; + +/** + * When combined with a key and computationId, represents the unique address for state managed by + * Windmill. + */ +@AutoValue +public abstract class StateTag { + static StateTag of( + Kind kind, ByteString tag, String stateFamily, @Nullable RequestPositionT requestPosition) { + return new AutoValue_StateTag.Builder() + .setKind(kind) + .setTag(tag) + .setStateFamily(stateFamily) + .setRequestPosition(requestPosition) + .build(); + } + + public static StateTag of( + Kind kind, ByteString tag, String stateFamily) { + return of(kind, tag, stateFamily, null); + } + + abstract Kind getKind(); + + abstract ByteString getTag(); + + abstract String getStateFamily(); + + /** + * For {@link Kind#BAG, Kind#ORDERED_LIST, Kind#VALUE_PREFIX, KIND#MULTIMAP_SINGLE_ENTRY, + * KIND#MULTIMAP_ALL} kinds: A previous 'continuation_position' returned by Windmill to signal the + * resulting state was incomplete. Sending that position will request the next page of values. + * Null for first request. + * + *

Null for other kinds. + */ + @Nullable + public abstract RequestPositionT getRequestPosition(); + + /** For {@link Kind#ORDERED_LIST} kinds: the range to fetch or delete. */ + @Nullable + abstract Range getSortedListRange(); + + /** For {@link Kind#MULTIMAP_SINGLE_ENTRY} kinds: the key in the multimap to fetch or delete. */ + @Nullable + abstract ByteString getMultimapKey(); + + /** + * For {@link Kind#MULTIMAP_ALL} kinds: will only return the keys of the multimap and not the + * values if true. + */ + @Nullable + abstract Boolean getOmitValues(); + + public abstract Builder toBuilder(); + + public enum Kind { + VALUE, + BAG, + WATERMARK, + ORDERED_LIST, + VALUE_PREFIX, + MULTIMAP_SINGLE_ENTRY, + MULTIMAP_ALL + } + + @AutoValue.Builder + abstract static class Builder { + abstract Builder setKind(Kind kind); + + abstract Builder setTag(ByteString tag); + + abstract Builder setStateFamily(String stateFamily); + + abstract Builder setRequestPosition( + @Nullable RequestPositionT requestPosition); + + abstract Builder setSortedListRange(@Nullable Range sortedListRange); + + abstract Builder setMultimapKey(@Nullable ByteString encodedMultimapKey); + + abstract Builder setOmitValues(Boolean omitValues); + + abstract StateTag build(); + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/TimestampedValueWithId.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/TimestampedValueWithId.java new file mode 100644 index 000000000000..e180efafb65b --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/TimestampedValueWithId.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.dataflow.worker.windmill.state; + +import com.google.auto.value.AutoValue; +import java.util.Comparator; +import org.apache.beam.sdk.values.TimestampedValue; +import org.joda.time.Instant; + +@AutoValue +abstract class TimestampedValueWithId { + public static final Comparator> COMPARATOR = + Comparator., Instant>comparing(v -> v.getValue().getTimestamp()) + .thenComparingLong(TimestampedValueWithId::getId); + + static TimestampedValueWithId of(TimestampedValue value, long id) { + return new AutoValue_TimestampedValueWithId<>(value, id); + } + + static TimestampedValueWithId bound(Instant ts) { + return of(TimestampedValue.of(null, ts), Long.MIN_VALUE); + } + + abstract TimestampedValue getValue(); + + abstract long getId(); +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/ToIterableFunction.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/ToIterableFunction.java new file mode 100644 index 000000000000..3db058c79a03 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/ToIterableFunction.java @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.dataflow.worker.windmill.state; + +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; +import javax.annotation.Nonnull; +import javax.annotation.Nullable; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Function; + +/** Function to extract an {@link Iterable} from the continuation-supporting page read future. */ +@SuppressWarnings({ + "nullness" // TODO(https://github.com/apache/beam/issues/20497) +}) +public class ToIterableFunction + implements Function, Iterable> { + private final StateTag stateTag; + private final Coder coder; + /** + * Reader to request continuation pages from, or {@literal null} if no continuation pages + * required. + */ + private @Nullable WindmillStateReader reader; + + public ToIterableFunction( + WindmillStateReader reader, StateTag stateTag, Coder coder) { + this.reader = reader; + this.stateTag = stateTag; + this.coder = coder; + } + + @SuppressFBWarnings( + value = "NP_METHOD_PARAMETER_TIGHTENS_ANNOTATION", + justification = "https://github.com/google/guava/issues/920") + @Override + public Iterable apply( + @Nonnull ValuesAndContPosition valuesAndContPosition) { + if (valuesAndContPosition.getContinuationPosition() == null) { + // Number of values is small enough Windmill sent us the entire bag in one response. + reader = null; + return valuesAndContPosition.getValues(); + } else { + // Return an iterable which knows how to come back for more. + StateTag.Builder continuationTBuilder = + StateTag.of( + stateTag.getKind(), + stateTag.getTag(), + stateTag.getStateFamily(), + valuesAndContPosition.getContinuationPosition()) + .toBuilder(); + if (stateTag.getSortedListRange() != null) { + continuationTBuilder.setSortedListRange(stateTag.getSortedListRange()).build(); + } + if (stateTag.getMultimapKey() != null) { + continuationTBuilder.setMultimapKey(stateTag.getMultimapKey()).build(); + } + if (stateTag.getOmitValues() != null) { + continuationTBuilder.setOmitValues(stateTag.getOmitValues()).build(); + } + return new PagingIterable<>( + reader, valuesAndContPosition.getValues(), continuationTBuilder.build(), coder); + } + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/ValuesAndContPosition.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/ValuesAndContPosition.java new file mode 100644 index 000000000000..a1002fee4380 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/ValuesAndContPosition.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.dataflow.worker.windmill.state; + +import java.util.List; +import javax.annotation.Nullable; + +/** + * An in-memory collection of deserialized values and an optional continuation position to pass to + * Windmill when fetching the next page of values. + */ +public class ValuesAndContPosition { + private final List values; + + /** Position to pass to next request for next page of values. Null if done. */ + private final @Nullable ContinuationT continuationPosition; + + public ValuesAndContPosition(List values, @Nullable ContinuationT continuationPosition) { + this.values = values; + this.continuationPosition = continuationPosition; + } + + public List getValues() { + return values; + } + + @Nullable + public ContinuationT getContinuationPosition() { + return continuationPosition; + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WeightedList.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WeightedList.java new file mode 100644 index 000000000000..1b39d07c6fe7 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WeightedList.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.dataflow.worker.windmill.state; + +import java.util.List; +import org.apache.beam.sdk.util.Weighted; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ForwardingList; + +@VisibleForTesting +public class WeightedList extends ForwardingList implements Weighted { + private final List delegate; + long weight; + + WeightedList(List delegate) { + this.delegate = delegate; + this.weight = 0; + } + + @Override + protected List delegate() { + return delegate; + } + + @Override + public boolean add(T elem) { + throw new UnsupportedOperationException("Must use AddWeighted()"); + } + + @Override + public long getWeight() { + return weight; + } + + public void addWeighted(T elem, long weight) { + delegate.add(elem); + this.weight += weight; + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillBag.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillBag.java new file mode 100644 index 000000000000..7cdb3776dfa1 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillBag.java @@ -0,0 +1,209 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.dataflow.worker.windmill.state; + +import java.io.Closeable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import org.apache.beam.runners.core.StateNamespace; +import org.apache.beam.runners.core.StateTag; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.state.BagState; +import org.apache.beam.sdk.state.ReadableState; +import org.apache.beam.sdk.util.ByteStringOutputStream; +import org.apache.beam.sdk.util.Weighted; +import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; + +@SuppressWarnings({ + "rawtypes", // TODO(https://github.com/apache/beam/issues/20447) + "nullness" // TODO(https://github.com/apache/beam/issues/20497) +}) +public class WindmillBag extends SimpleWindmillState implements BagState { + + private final StateNamespace namespace; + private final StateTag> address; + private final ByteString stateKey; + private final String stateFamily; + private final Coder elemCoder; + + private boolean cleared = false; + /** + * If non-{@literal null}, this contains the complete contents of the bag, except for any local + * additions. If {@literal null} then we don't know if Windmill contains additional values which + * should be part of the bag. We'll need to read them if the work item actually wants the bag + * contents. + */ + private ConcatIterables cachedValues = null; + + private List localAdditions = new ArrayList<>(); + private long encodedSize = 0; + + WindmillBag( + StateNamespace namespace, + StateTag> address, + String stateFamily, + Coder elemCoder, + boolean isNewKey) { + this.namespace = namespace; + this.address = address; + this.stateKey = WindmillStateUtil.encodeKey(namespace, address); + this.stateFamily = stateFamily; + this.elemCoder = elemCoder; + if (isNewKey) { + this.cachedValues = new ConcatIterables<>(); + } + } + + @Override + public void clear() { + cleared = true; + cachedValues = new ConcatIterables<>(); + localAdditions = new ArrayList<>(); + encodedSize = 0; + } + + /** + * Return iterable over all bag values in Windmill which should contribute to overall bag + * contents. + */ + private Iterable fetchData(Future> persistedData) { + try (Closeable scope = scopedReadState()) { + if (cachedValues != null) { + return cachedValues.snapshot(); + } + Iterable data = persistedData.get(); + if (data instanceof Weighted) { + // We have a known bounded amount of data; cache it. + cachedValues = new ConcatIterables<>(); + cachedValues.extendWith(data); + encodedSize = ((Weighted) data).getWeight(); + return cachedValues.snapshot(); + } else { + // This is an iterable that may not fit in memory at once; don't cache it. + return data; + } + } catch (InterruptedException | ExecutionException | IOException e) { + if (e instanceof InterruptedException) { + Thread.currentThread().interrupt(); + } + throw new RuntimeException("Unable to read state", e); + } + } + + public boolean valuesAreCached() { + return cachedValues != null; + } + + @Override + @SuppressWarnings("FutureReturnValueIgnored") + public WindmillBag readLater() { + getFuture(); + return this; + } + + @Override + public Iterable read() { + return Iterables.concat( + fetchData(getFuture()), Iterables.limit(localAdditions, localAdditions.size())); + } + + @Override + public ReadableState isEmpty() { + return new ReadableState() { + @Override + public ReadableState readLater() { + WindmillBag.this.readLater(); + return this; + } + + @Override + public Boolean read() { + return Iterables.isEmpty(fetchData(getFuture())) && localAdditions.isEmpty(); + } + }; + } + + @Override + public void add(T input) { + localAdditions.add(input); + } + + @Override + public Windmill.WorkItemCommitRequest persistDirectly(WindmillStateCache.ForKeyAndFamily cache) + throws IOException { + Windmill.WorkItemCommitRequest.Builder commitBuilder = + Windmill.WorkItemCommitRequest.newBuilder(); + + Windmill.TagBag.Builder bagUpdatesBuilder = null; + + if (cleared) { + bagUpdatesBuilder = commitBuilder.addBagUpdatesBuilder(); + bagUpdatesBuilder.setDeleteAll(true); + cleared = false; + } + + if (!localAdditions.isEmpty()) { + // Tell Windmill to capture the local additions. + if (bagUpdatesBuilder == null) { + bagUpdatesBuilder = commitBuilder.addBagUpdatesBuilder(); + } + for (T value : localAdditions) { + ByteStringOutputStream stream = new ByteStringOutputStream(); + // Encode the value + elemCoder.encode(value, stream, Coder.Context.OUTER); + ByteString encoded = stream.toByteString(); + if (cachedValues != null) { + // We'll capture this value in the cache below. + // Capture the value's size now since we have it. + encodedSize += encoded.size(); + } + bagUpdatesBuilder.addValues(encoded); + } + } + + if (bagUpdatesBuilder != null) { + bagUpdatesBuilder.setTag(stateKey).setStateFamily(stateFamily); + } + + if (cachedValues != null) { + if (!localAdditions.isEmpty()) { + // Capture the local additions in the cached value since we and + // Windmill are now in agreement. + cachedValues.extendWith(localAdditions); + } + // We now know the complete bag contents, and any read on it will yield a + // cached value, so cache it for future reads. + cache.put(namespace, address, this, encodedSize); + } + + // Don't reuse the localAdditions object; we don't want future changes to it to + // modify the value of cachedValues. + localAdditions = new ArrayList<>(); + + return commitBuilder.buildPartial(); + } + + private Future> getFuture() { + return cachedValues != null ? null : reader.bagFuture(stateKey, stateFamily, elemCoder); + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillCombiningState.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillCombiningState.java new file mode 100644 index 000000000000..98359913c703 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillCombiningState.java @@ -0,0 +1,170 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.dataflow.worker.windmill.state; + +import java.io.Closeable; +import java.io.IOException; +import java.util.Arrays; +import java.util.Collections; +import java.util.concurrent.Future; +import javax.annotation.concurrent.NotThreadSafe; +import org.apache.beam.runners.core.StateNamespace; +import org.apache.beam.runners.core.StateTag; +import org.apache.beam.runners.core.StateTags; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.state.BagState; +import org.apache.beam.sdk.state.CombiningState; +import org.apache.beam.sdk.state.ReadableState; +import org.apache.beam.sdk.transforms.Combine; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Supplier; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; + +@NotThreadSafe +class WindmillCombiningState extends WindmillState + implements CombiningState { + + private final WindmillBag bag; + private final Combine.CombineFn combineFn; + + /* We use a separate, in-memory AccumT rather than relying on the WindmillWatermarkBag's + * localAdditions, because we want to combine multiple InputT's to a single AccumT + * before adding it. + */ + private AccumT localAdditionsAccumulator; + private boolean hasLocalAdditions; + + WindmillCombiningState( + StateNamespace namespace, + StateTag> address, + String stateFamily, + Coder accumCoder, + Combine.CombineFn combineFn, + WindmillStateCache.ForKeyAndFamily cache, + boolean isNewKey) { + StateTag> internalBagAddress = StateTags.convertToBagTagInternal(address); + this.bag = + cache + .get(namespace, internalBagAddress) + .map(state -> (WindmillBag) state) + .orElseGet( + () -> + new WindmillBag<>( + namespace, internalBagAddress, stateFamily, accumCoder, isNewKey)); + + this.combineFn = combineFn; + this.localAdditionsAccumulator = combineFn.createAccumulator(); + this.hasLocalAdditions = false; + } + + @Override + void initializeForWorkItem( + WindmillStateReader reader, Supplier scopedReadStateSupplier) { + super.initializeForWorkItem(reader, scopedReadStateSupplier); + this.bag.initializeForWorkItem(reader, scopedReadStateSupplier); + } + + @Override + void cleanupAfterWorkItem() { + super.cleanupAfterWorkItem(); + bag.cleanupAfterWorkItem(); + } + + @Override + public WindmillCombiningState readLater() { + bag.readLater(); + return this; + } + + @Override + @SuppressWarnings("nullness") + public OutputT read() { + return combineFn.extractOutput(getAccum()); + } + + @Override + public void add(InputT input) { + hasLocalAdditions = true; + localAdditionsAccumulator = combineFn.addInput(localAdditionsAccumulator, input); + } + + @Override + public void clear() { + bag.clear(); + localAdditionsAccumulator = combineFn.createAccumulator(); + hasLocalAdditions = false; + } + + @Override + public Future persist(WindmillStateCache.ForKeyAndFamily cache) + throws IOException { + if (hasLocalAdditions) { + if (WindmillStateInternals.COMPACT_NOW.get().get() || bag.valuesAreCached()) { + // Implicitly clears the bag and combines local and persisted accumulators. + localAdditionsAccumulator = getAccum(); + } + bag.add(combineFn.compact(localAdditionsAccumulator)); + localAdditionsAccumulator = combineFn.createAccumulator(); + hasLocalAdditions = false; + } + + return bag.persist(cache); + } + + @Override + public AccumT getAccum() { + Iterable accumulators = + Iterables.concat(bag.read(), Collections.singleton(localAdditionsAccumulator)); + + // Compact things + AccumT merged = combineFn.mergeAccumulators(accumulators); + bag.clear(); + localAdditionsAccumulator = merged; + hasLocalAdditions = true; + return merged; + } + + @Override + public ReadableState isEmpty() { + final ReadableState bagIsEmpty = bag.isEmpty(); + return new ReadableState() { + @Override + public ReadableState readLater() { + bagIsEmpty.readLater(); + return this; + } + + @Override + public Boolean read() { + return !hasLocalAdditions && bagIsEmpty.read(); + } + }; + } + + @Override + public void addAccum(AccumT accumulator) { + hasLocalAdditions = true; + localAdditionsAccumulator = + combineFn.mergeAccumulators(Arrays.asList(localAdditionsAccumulator, accumulator)); + } + + @Override + public AccumT mergeAccumulators(Iterable accumulators) { + return combineFn.mergeAccumulators(accumulators); + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillMap.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillMap.java new file mode 100644 index 000000000000..43490a725ac4 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillMap.java @@ -0,0 +1,449 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.dataflow.worker.windmill.state; + +import static org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateUtil.encodeKey; + +import java.io.Closeable; +import java.io.IOException; +import java.util.AbstractMap; +import java.util.Collections; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.function.Function; +import javax.annotation.Nullable; +import org.apache.beam.runners.core.StateNamespace; +import org.apache.beam.runners.core.StateTag; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.state.MapState; +import org.apache.beam.sdk.state.ReadableState; +import org.apache.beam.sdk.state.ReadableStates; +import org.apache.beam.sdk.util.ByteStringOutputStream; +import org.apache.beam.sdk.util.Weighted; +import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Sets; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.Futures; +import org.checkerframework.checker.initialization.qual.Initialized; +import org.checkerframework.checker.nullness.qual.NonNull; +import org.checkerframework.checker.nullness.qual.UnknownKeyFor; + +@SuppressWarnings({ + "nullness" // TODO(https://github.com/apache/beam/issues/20497) +}) +public class WindmillMap extends SimpleWindmillState implements MapState { + private final StateNamespace namespace; + private final StateTag> address; + private final ByteString stateKeyPrefix; + private final String stateFamily; + private final Coder keyCoder; + private final Coder valueCoder; + // TODO(reuvenlax): Should we evict items from the cache? We would have to make sure + // that anything in the cache that is not committed is not evicted. negativeCache could be + // evicted whenever we want. + private final Map cachedValues = Maps.newHashMap(); + private final Set negativeCache = Sets.newHashSet(); + private final Set localAdditions = Sets.newHashSet(); + private final Set localRemovals = Sets.newHashSet(); + private boolean complete; + private boolean cleared = false; + + WindmillMap( + StateNamespace namespace, + StateTag> address, + String stateFamily, + Coder keyCoder, + Coder valueCoder, + boolean isNewKey) { + this.namespace = namespace; + this.address = address; + this.stateKeyPrefix = encodeKey(namespace, address); + this.stateFamily = stateFamily; + this.keyCoder = keyCoder; + this.valueCoder = valueCoder; + this.complete = isNewKey; + } + + private K userKeyFromProtoKey(ByteString tag) throws IOException { + Preconditions.checkState(tag.startsWith(stateKeyPrefix)); + ByteString keyBytes = tag.substring(stateKeyPrefix.size()); + return keyCoder.decode(keyBytes.newInput(), Coder.Context.OUTER); + } + + private ByteString protoKeyFromUserKey(K key) throws IOException { + ByteStringOutputStream keyStream = new ByteStringOutputStream(); + stateKeyPrefix.writeTo(keyStream); + keyCoder.encode(key, keyStream, Coder.Context.OUTER); + return keyStream.toByteString(); + } + + @Override + protected Windmill.WorkItemCommitRequest persistDirectly(WindmillStateCache.ForKeyAndFamily cache) + throws IOException { + if (!cleared && localAdditions.isEmpty() && localRemovals.isEmpty()) { + // No changes, so return directly. + return Windmill.WorkItemCommitRequest.newBuilder().buildPartial(); + } + + Windmill.WorkItemCommitRequest.Builder commitBuilder = + Windmill.WorkItemCommitRequest.newBuilder(); + + if (cleared) { + commitBuilder + .addTagValuePrefixDeletesBuilder() + .setStateFamily(stateFamily) + .setTagPrefix(stateKeyPrefix); + } + cleared = false; + + for (K key : localAdditions) { + ByteString keyBytes = protoKeyFromUserKey(key); + ByteStringOutputStream valueStream = new ByteStringOutputStream(); + valueCoder.encode(cachedValues.get(key), valueStream, Coder.Context.OUTER); + ByteString valueBytes = valueStream.toByteString(); + + commitBuilder + .addValueUpdatesBuilder() + .setTag(keyBytes) + .setStateFamily(stateFamily) + .getValueBuilder() + .setData(valueBytes) + .setTimestamp(Long.MAX_VALUE); + } + localAdditions.clear(); + + for (K key : localRemovals) { + ByteStringOutputStream keyStream = new ByteStringOutputStream(); + stateKeyPrefix.writeTo(keyStream); + keyCoder.encode(key, keyStream, Coder.Context.OUTER); + ByteString keyBytes = keyStream.toByteString(); + // Leaving data blank means that we delete the tag. + commitBuilder + .addValueUpdatesBuilder() + .setTag(keyBytes) + .setStateFamily(stateFamily) + .getValueBuilder() + .setTimestamp(Long.MAX_VALUE); + + V cachedValue = cachedValues.remove(key); + if (cachedValue != null) { + ByteStringOutputStream valueStream = new ByteStringOutputStream(); + valueCoder.encode(cachedValues.get(key), valueStream, Coder.Context.OUTER); + } + } + negativeCache.addAll(localRemovals); + localRemovals.clear(); + + // TODO(reuvenlax): We should store in the cache parameter, as that would enable caching the + // map + // between work items, reducing fetches to Windmill. To do so, we need keep track of the + // encoded size + // of the map, and to do so efficiently (i.e. without iterating over the entire map on every + // persist) + // we need to track the sizes of each map entry. + cache.put(namespace, address, this, 1); + return commitBuilder.buildPartial(); + } + + @Override + public @UnknownKeyFor @NonNull @Initialized ReadableState get(K key) { + return getOrDefault(key, null); + } + + @Override + public @UnknownKeyFor @NonNull @Initialized ReadableState getOrDefault( + K key, @Nullable V defaultValue) { + return new WindmillMapReadResultReadableState(key, defaultValue); + } + + @Override + public @UnknownKeyFor @NonNull @Initialized ReadableState< + @UnknownKeyFor @NonNull @Initialized Iterable> + keys() { + ReadableState>> entries = entries(); + return new WindmillMapKeysReadableState(entries); + } + + @Override + public @UnknownKeyFor @NonNull @Initialized ReadableState< + @UnknownKeyFor @NonNull @Initialized Iterable> + values() { + ReadableState>> entries = entries(); + return new WindmillMapValuesReadableState(entries); + } + + @Override + public @UnknownKeyFor @NonNull @Initialized ReadableState< + @UnknownKeyFor @NonNull @Initialized Iterable< + Map.@UnknownKeyFor @NonNull @Initialized Entry>> + entries() { + return new WindmillMapEntriesReadableState(); + } + + @Override + public ReadableState isEmpty() { + return new WindmillMapIsEmptyReadableState(); + } + + @Override + public void put(K key, V value) { + V oldValue = cachedValues.put(key, value); + if (valueCoder.consistentWithEquals() && value.equals(oldValue)) { + return; + } + localAdditions.add(key); + localRemovals.remove(key); + negativeCache.remove(key); + } + + @Override + public @UnknownKeyFor @NonNull @Initialized ReadableState computeIfAbsent( + K key, Function mappingFunction) { + Future persistedData = getFutureForKey(key); + try (Closeable scope = scopedReadState()) { + if (localRemovals.contains(key) || negativeCache.contains(key)) { + return ReadableStates.immediate(null); + } + @Nullable V cachedValue = cachedValues.get(key); + if (cachedValue != null || complete) { + return ReadableStates.immediate(cachedValue); + } + + V persistedValue = persistedData.get(); + if (persistedValue == null) { + // This is a new value. Add it to the map and return null. + put(key, mappingFunction.apply(key)); + return ReadableStates.immediate(null); + } + // TODO: Don't do this if it was already in cache. + cachedValues.put(key, persistedValue); + return ReadableStates.immediate(persistedValue); + } catch (InterruptedException | ExecutionException | IOException e) { + if (e instanceof InterruptedException) { + Thread.currentThread().interrupt(); + } + throw new RuntimeException("Unable to read state", e); + } + } + + @Override + public void remove(K key) { + if (localRemovals.add(key)) { + cachedValues.remove(key); + localAdditions.remove(key); + } + } + + @Override + public void clear() { + cachedValues.clear(); + localAdditions.clear(); + localRemovals.clear(); + negativeCache.clear(); + cleared = true; + complete = true; + } + + private Future getFutureForKey(K key) { + try { + ByteStringOutputStream keyStream = new ByteStringOutputStream(); + stateKeyPrefix.writeTo(keyStream); + keyCoder.encode(key, keyStream, Coder.Context.OUTER); + return reader.valueFuture(keyStream.toByteString(), stateFamily, valueCoder); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + private Future>> getFuture() { + if (complete) { + // The caller will merge in local cached values. + return Futures.immediateFuture(Collections.emptyList()); + } else { + return reader.valuePrefixFuture(stateKeyPrefix, stateFamily, valueCoder); + } + } + + private class WindmillMapKeysReadableState implements ReadableState> { + private final ReadableState>> entries; + + public WindmillMapKeysReadableState(ReadableState>> entries) { + this.entries = entries; + } + + @Override + public Iterable read() { + return Iterables.transform(entries.read(), Map.Entry::getKey); + } + + @Override + public @UnknownKeyFor @NonNull @Initialized ReadableState> readLater() { + entries.readLater(); + return this; + } + } + + private class WindmillMapValuesReadableState implements ReadableState> { + private final ReadableState>> entries; + + public WindmillMapValuesReadableState(ReadableState>> entries) { + this.entries = entries; + } + + @Override + public @Nullable Iterable read() { + return Iterables.transform(entries.read(), Map.Entry::getValue); + } + + @Override + public @UnknownKeyFor @NonNull @Initialized ReadableState> readLater() { + entries.readLater(); + return this; + } + } + + private class WindmillMapEntriesReadableState + implements ReadableState>> { + @Override + public Iterable> read() { + if (complete) { + return Iterables.unmodifiableIterable(cachedValues.entrySet()); + } + Future>> persistedData = getFuture(); + try (Closeable scope = scopedReadState()) { + Iterable> data = persistedData.get(); + Iterable> transformedData = + Iterables.transform( + data, + entry -> { + try { + return new AbstractMap.SimpleEntry<>( + userKeyFromProtoKey(entry.getKey()), entry.getValue()); + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + + if (data instanceof Weighted) { + // This is a known amount of data. Cache it all. + transformedData.forEach( + e -> { + // The cached data overrides what is read from state, so call putIfAbsent. + cachedValues.putIfAbsent(e.getKey(), e.getValue()); + }); + complete = true; + return Iterables.unmodifiableIterable(cachedValues.entrySet()); + } else { + // This means that the result might be too large to cache, so don't add it to the + // local cache. Instead merge the iterables, giving priority to any local additions + // (represented in cachedValued and localRemovals) that may not have been committed + // yet. + return Iterables.unmodifiableIterable( + Iterables.concat( + cachedValues.entrySet(), + Iterables.filter( + transformedData, + e -> + !cachedValues.containsKey(e.getKey()) + && !localRemovals.contains(e.getKey())))); + } + + } catch (InterruptedException | ExecutionException | IOException e) { + if (e instanceof InterruptedException) { + Thread.currentThread().interrupt(); + } + throw new RuntimeException("Unable to read state", e); + } + } + + @Override + @SuppressWarnings("FutureReturnValueIgnored") + public @UnknownKeyFor @NonNull @Initialized ReadableState>> + readLater() { + WindmillMap.this.getFuture(); + return this; + } + } + + private class WindmillMapIsEmptyReadableState implements ReadableState { + // TODO(reuvenlax): Can we find a more efficient way of implementing isEmpty than reading + // the entire map? + final ReadableState> keys = WindmillMap.this.keys(); + + @Override + public @Nullable Boolean read() { + return Iterables.isEmpty(keys.read()); + } + + @Override + public @UnknownKeyFor @NonNull @Initialized ReadableState readLater() { + keys.readLater(); + return this; + } + } + + private class WindmillMapReadResultReadableState implements ReadableState { + private final K key; + private final @Nullable V defaultValue; + + public WindmillMapReadResultReadableState(K key, @Nullable V defaultValue) { + this.key = key; + this.defaultValue = defaultValue; + } + + @Override + public @Nullable V read() { + Future persistedData = getFutureForKey(key); + try (Closeable scope = scopedReadState()) { + if (localRemovals.contains(key) || negativeCache.contains(key)) { + return null; + } + @Nullable V cachedValue = cachedValues.get(key); + if (cachedValue != null || complete) { + return cachedValue; + } + + V persistedValue = persistedData.get(); + if (persistedValue == null) { + negativeCache.add(key); + return defaultValue; + } + // TODO: Don't do this if it was already in cache. + cachedValues.put(key, persistedValue); + return persistedValue; + } catch (InterruptedException | ExecutionException | IOException e) { + if (e instanceof InterruptedException) { + Thread.currentThread().interrupt(); + } + throw new RuntimeException("Unable to read state", e); + } + } + + @Override + @SuppressWarnings("FutureReturnValueIgnored") + public @UnknownKeyFor @NonNull @Initialized ReadableState readLater() { + WindmillMap.this.getFutureForKey(key); + return this; + } + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillMultimap.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillMultimap.java new file mode 100644 index 000000000000..1c0b3df44c21 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillMultimap.java @@ -0,0 +1,732 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.dataflow.worker.windmill.state; + +import static org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateUtil.encodeKey; + +import java.io.Closeable; +import java.io.IOException; +import java.util.AbstractMap; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.stream.Collectors; +import org.apache.beam.repackaged.core.org.apache.commons.lang3.tuple.Triple; +import org.apache.beam.runners.core.StateNamespace; +import org.apache.beam.runners.core.StateTag; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.state.MultimapState; +import org.apache.beam.sdk.state.ReadableState; +import org.apache.beam.sdk.util.ByteStringOutputStream; +import org.apache.beam.sdk.util.Weighted; +import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterators; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Sets; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.Futures; + +@SuppressWarnings({ + "nullness" // TODO(https://github.com/apache/beam/issues/20497) +}) +public class WindmillMultimap extends SimpleWindmillState implements MultimapState { + + private final StateNamespace namespace; + private final StateTag> address; + private final ByteString stateKey; + private final String stateFamily; + private final Coder keyCoder; + private final Coder valueCoder; + // Set to true when user clears the entire multimap, so that we can later send delete request to + // the windmill backend. + private boolean cleared = false; + // We use the structural value of the keys as the key in keyStateMap, so that different java + // Objects with the same content will be treated as the same Multimap key. + private Map keyStateMap = Maps.newHashMap(); + // If true, all keys are cached in keyStateMap with existence == KNOWN_EXIST. + private boolean allKeysKnown; + // True if all contents of this multimap are cached in this object. + private boolean complete; + // hasLocalAdditions and hasLocalRemovals track whether there are local changes that needs to be + // propagated to windmill. + private boolean hasLocalAdditions = false; + private boolean hasLocalRemovals = false; + + WindmillMultimap( + StateNamespace namespace, + StateTag> address, + String stateFamily, + Coder keyCoder, + Coder valueCoder, + boolean isNewShardingKey) { + this.namespace = namespace; + this.address = address; + this.stateKey = encodeKey(namespace, address); + this.stateFamily = stateFamily; + this.keyCoder = keyCoder; + this.valueCoder = valueCoder; + this.complete = isNewShardingKey; + this.allKeysKnown = isNewShardingKey; + } + + private static Iterable> unnestCachedEntries( + Iterable>>> cachedEntries) { + return Iterables.concat( + Iterables.transform( + cachedEntries, + entry -> + Iterables.transform( + entry.getValue().getRight(), + v -> new AbstractMap.SimpleEntry<>(entry.getValue().getLeft(), v)))); + } + + @Override + public void put(K key, V value) { + final Object structuralKey = keyCoder.structuralValue(key); + hasLocalAdditions = true; + keyStateMap.compute( + structuralKey, + (k, v) -> { + if (v == null) v = new KeyState(key); + v.existence = KeyExistence.KNOWN_EXIST; + v.localAdditions.add(value); + return v; + }); + } + + // Initiates a backend state read to fetch all entries if necessary. + private Future>>> necessaryEntriesFromStorageFuture( + boolean omitValues) { + if (complete) { + // Since we're complete, even if there are entries in storage we don't need to read them. + return Futures.immediateFuture(Collections.emptyList()); + } else { + return reader.multimapFetchAllFuture(omitValues, stateKey, stateFamily, valueCoder); + } + } + + // Initiates a backend state read to fetch a single entry if necessary. + private Future> necessaryKeyEntriesFromStorageFuture(K key) { + try { + ByteStringOutputStream keyStream = new ByteStringOutputStream(); + keyCoder.encode(key, keyStream, Coder.Context.OUTER); + return reader.multimapFetchSingleEntryFuture( + keyStream.toByteString(), stateKey, stateFamily, valueCoder); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + @Override + public ReadableState> get(K key) { + return new ReadResultReadableState(key); + } + + @Override + protected Windmill.WorkItemCommitRequest persistDirectly(WindmillStateCache.ForKeyAndFamily cache) + throws IOException { + if (!cleared && !hasLocalAdditions && !hasLocalRemovals) { + cache.put(namespace, address, this, 1); + return Windmill.WorkItemCommitRequest.newBuilder().buildPartial(); + } + Windmill.WorkItemCommitRequest.Builder commitBuilder = + Windmill.WorkItemCommitRequest.newBuilder(); + Windmill.TagMultimapUpdateRequest.Builder builder = commitBuilder.addMultimapUpdatesBuilder(); + builder.setTag(stateKey).setStateFamily(stateFamily); + + if (cleared) { + builder.setDeleteAll(true); + } + if (hasLocalRemovals || hasLocalAdditions) { + ByteStringOutputStream keyStream = new ByteStringOutputStream(); + ByteStringOutputStream valueStream = new ByteStringOutputStream(); + Iterator> iterator = keyStateMap.entrySet().iterator(); + while (iterator.hasNext()) { + KeyState keyState = iterator.next().getValue(); + if (!keyState.removedLocally && keyState.localAdditions.isEmpty()) { + if (keyState.existence == KeyExistence.KNOWN_NONEXISTENT) iterator.remove(); + continue; + } + keyCoder.encode(keyState.originalKey, keyStream, Coder.Context.OUTER); + ByteString encodedKey = keyStream.toByteStringAndReset(); + Windmill.TagMultimapEntry.Builder entryBuilder = builder.addUpdatesBuilder(); + entryBuilder.setEntryName(encodedKey); + if (keyState.removedLocally) entryBuilder.setDeleteAll(true); + keyState.removedLocally = false; + if (!keyState.localAdditions.isEmpty()) { + for (V value : keyState.localAdditions) { + valueCoder.encode(value, valueStream, Coder.Context.OUTER); + ByteString encodedValue = valueStream.toByteStringAndReset(); + entryBuilder.addValues(encodedValue); + } + // Move newly added values from localAdditions to keyState.values as those new values + // now + // are also persisted in Windmill. If a key now has no more values and is not + // KNOWN_EXIST, + // remove it from cache. + if (keyState.valuesCached) { + keyState.values.extendWith(keyState.localAdditions); + keyState.valuesSize += keyState.localAdditions.size(); + } + // Create a new localAdditions so that the cached values are unaffected. + keyState.localAdditions = Lists.newArrayList(); + } + if (!keyState.valuesCached && keyState.existence != KeyExistence.KNOWN_EXIST) { + iterator.remove(); + } + } + } + + hasLocalAdditions = false; + hasLocalRemovals = false; + cleared = false; + + cache.put(namespace, address, this, 1); + return commitBuilder.buildPartial(); + } + + @Override + public void remove(K key) { + final Object structuralKey = keyCoder.structuralValue(key); + // does not insert key if allKeysKnown. + KeyState keyState = + keyStateMap.computeIfAbsent(structuralKey, k -> allKeysKnown ? null : new KeyState(key)); + if (keyState == null || keyState.existence == KeyExistence.KNOWN_NONEXISTENT) { + return; + } + if (keyState.valuesCached && keyState.valuesSize == 0) { + // no data in windmill, deleting from local cache is sufficient. + keyStateMap.remove(structuralKey); + } else { + // there may be data in windmill that need to be removed. + hasLocalRemovals = true; + keyState.removedLocally = true; + keyState.values = new ConcatIterables<>(); + keyState.valuesSize = 0; + keyState.existence = KeyExistence.KNOWN_NONEXISTENT; + } + if (!keyState.localAdditions.isEmpty()) { + keyState.localAdditions = Lists.newArrayList(); + } + keyState.valuesCached = true; + } + + @Override + public void clear() { + keyStateMap = Maps.newHashMap(); + cleared = true; + complete = true; + allKeysKnown = true; + hasLocalAdditions = false; + hasLocalRemovals = false; + } + + @Override + public ReadableState> keys() { + return new KeysReadableState(); + } + + @Override + public ReadableState>> entries() { + return new EntriesReadableState(); + } + + @Override + public ReadableState containsKey(K key) { + return new ContainsKeyReadableState(key); + } + + // Currently, isEmpty is implemented by reading all keys and could potentially be optimized. + // But note that if isEmpty is often followed by iterating over keys then maybe not too bad; if + // isEmpty is followed by iterating over both keys and values then it won't help much. + @Override + public ReadableState isEmpty() { + return new IsEmptyReadableState(); + } + + private enum KeyExistence { + // this key is known to exist, it has at least 1 value in either localAdditions or windmill + KNOWN_EXIST, + // this key is known to be nonexistent, it has 0 value in both localAdditions and windmill + KNOWN_NONEXISTENT, + // we don't know if this key is in this multimap, it has exact 0 value in localAddition, but + // may have no or any number of values in windmill. This is just to provide a mapping between + // the original key and the structural key. + UNKNOWN_EXISTENCE + } + + private class KeyState { + final K originalKey; + KeyExistence existence; + // valuesCached can be true if only existence == KNOWN_EXIST and all values of this key are + // cached (both values and localAdditions). + boolean valuesCached; + // Represents the values in windmill. When new values are added during user processing, they + // are added to localAdditions but not values. Those new values will be added to values only + // after they are persisted into windmill and removed from localAdditions + ConcatIterables values; + int valuesSize; + + // When new values are added during user processing, they are added to localAdditions, so that + // we can later try to persist them in windmill. When a key is removed during user processing, + // we mark removedLocally to be true so that we can later try to delete it from windmill. If + // localAdditions is not empty and removedLocally is true, values in localAdditions will be + // added to windmill after old values in windmill are removed. + List localAdditions; + boolean removedLocally; + + KeyState(K originalKey) { + this.originalKey = originalKey; + existence = KeyExistence.UNKNOWN_EXISTENCE; + valuesCached = complete; + values = new ConcatIterables<>(); + valuesSize = 0; + localAdditions = Lists.newArrayList(); + removedLocally = false; + } + } + + private class ReadResultReadableState implements ReadableState> { + final Object structuralKey; + private final K key; + + public ReadResultReadableState(K key) { + this.key = key; + structuralKey = keyCoder.structuralValue(key); + } + + @Override + public Iterable read() { + KeyState keyState = null; + if (allKeysKnown) { + keyState = keyStateMap.get(structuralKey); + if (keyState == null || keyState.existence == KeyExistence.UNKNOWN_EXISTENCE) { + if (keyState != null) keyStateMap.remove(structuralKey); + return Collections.emptyList(); + } + } else { + keyState = keyStateMap.computeIfAbsent(structuralKey, k -> new KeyState(key)); + } + if (keyState.existence == KeyExistence.KNOWN_NONEXISTENT) { + return Collections.emptyList(); + } + Iterable localNewValues = + Iterables.limit(keyState.localAdditions, keyState.localAdditions.size()); + if (keyState.removedLocally) { + // this key has been removed locally but the removal hasn't been sent to windmill, + // thus values in windmill(if any) are obsolete, and we only care about local values. + return Iterables.unmodifiableIterable(localNewValues); + } + if (keyState.valuesCached || complete) { + return Iterables.unmodifiableIterable( + Iterables.concat( + Iterables.limit(keyState.values, keyState.valuesSize), localNewValues)); + } + Future> persistedData = necessaryKeyEntriesFromStorageFuture(key); + try (Closeable scope = scopedReadState()) { + final Iterable persistedValues = persistedData.get(); + // Iterables.isEmpty() is O(1). + if (Iterables.isEmpty(persistedValues)) { + if (keyState.localAdditions.isEmpty()) { + // empty in both cache and windmill, mark key as KNOWN_NONEXISTENT. + keyState.existence = KeyExistence.KNOWN_NONEXISTENT; + return Collections.emptyList(); + } + return Iterables.unmodifiableIterable(localNewValues); + } + keyState.existence = KeyExistence.KNOWN_EXIST; + if (persistedValues instanceof Weighted) { + keyState.valuesCached = true; + ConcatIterables it = new ConcatIterables<>(); + it.extendWith(persistedValues); + keyState.values = it; + keyState.valuesSize = Iterables.size(persistedValues); + } + return Iterables.unmodifiableIterable(Iterables.concat(persistedValues, localNewValues)); + } catch (InterruptedException | ExecutionException | IOException e) { + if (e instanceof InterruptedException) { + Thread.currentThread().interrupt(); + } + throw new RuntimeException("Unable to read Multimap state", e); + } + } + + @Override + @SuppressWarnings("FutureReturnValueIgnored") + public ReadableState> readLater() { + WindmillMultimap.this.necessaryKeyEntriesFromStorageFuture(key); + return this; + } + } + + private class KeysReadableState implements ReadableState> { + + private Map cachedExistKeys() { + return keyStateMap.entrySet().stream() + .filter(entry -> entry.getValue().existence == KeyExistence.KNOWN_EXIST) + .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().originalKey)); + } + + @Override + public Iterable read() { + if (allKeysKnown) { + return Iterables.unmodifiableIterable(cachedExistKeys().values()); + } + Future>>> persistedData = + necessaryEntriesFromStorageFuture(true); + try (Closeable scope = scopedReadState()) { + Iterable>> entries = persistedData.get(); + if (entries instanceof Weighted) { + // This is a known amount of data, cache them all. + entries.forEach( + entry -> { + try { + K originalKey = keyCoder.decode(entry.getKey().newInput(), Coder.Context.OUTER); + KeyState keyState = + keyStateMap.computeIfAbsent( + keyCoder.structuralValue(originalKey), stk -> new KeyState(originalKey)); + if (keyState.existence == KeyExistence.UNKNOWN_EXISTENCE) { + keyState.existence = KeyExistence.KNOWN_EXIST; + } + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + allKeysKnown = true; + keyStateMap + .values() + .removeIf( + keyState -> + keyState.existence != KeyExistence.KNOWN_EXIST && !keyState.removedLocally); + return Iterables.unmodifiableIterable(cachedExistKeys().values()); + } else { + Map cachedExistKeys = Maps.newHashMap(); + Set cachedNonExistKeys = Sets.newHashSet(); + keyStateMap.forEach( + (structuralKey, keyState) -> { + switch (keyState.existence) { + case KNOWN_EXIST: + cachedExistKeys.put(structuralKey, keyState.originalKey); + break; + case KNOWN_NONEXISTENT: + cachedNonExistKeys.add(structuralKey); + break; + default: + break; + } + }); + // keysOnlyInWindmill is lazily loaded. + Iterable keysOnlyInWindmill = + Iterables.filter( + Iterables.transform( + entries, + entry -> { + try { + K originalKey = + keyCoder.decode(entry.getKey().newInput(), Coder.Context.OUTER); + Object structuralKey = keyCoder.structuralValue(originalKey); + if (cachedExistKeys.containsKey(structuralKey) + || cachedNonExistKeys.contains(structuralKey)) return null; + return originalKey; + } catch (IOException e) { + throw new RuntimeException(e); + } + }), + Objects::nonNull); + return Iterables.unmodifiableIterable( + Iterables.concat(cachedExistKeys.values(), keysOnlyInWindmill)); + } + } catch (InterruptedException | ExecutionException | IOException e) { + if (e instanceof InterruptedException) { + Thread.currentThread().interrupt(); + } + throw new RuntimeException("Unable to read state", e); + } + } + + @Override + @SuppressWarnings("FutureReturnValueIgnored") + public ReadableState> readLater() { + WindmillMultimap.this.necessaryEntriesFromStorageFuture(true); + return this; + } + } + + private class EntriesReadableState implements ReadableState>> { + @Override + public Iterable> read() { + if (complete) { + return Iterables.unmodifiableIterable( + unnestCachedEntries(mergedCachedEntries(null).entrySet())); + } + Future>>> persistedData = + necessaryEntriesFromStorageFuture(false); + try (Closeable scope = scopedReadState()) { + Iterable>> entries = persistedData.get(); + if (Iterables.isEmpty(entries)) { + complete = true; + allKeysKnown = true; + return Iterables.unmodifiableIterable( + unnestCachedEntries(mergedCachedEntries(null).entrySet())); + } + if (!(entries instanceof Weighted)) { + return nonWeightedEntries(entries); + } + // This is a known amount of data, cache them all. + entries.forEach( + entry -> { + try { + final K originalKey = + keyCoder.decode(entry.getKey().newInput(), Coder.Context.OUTER); + final Object structuralKey = keyCoder.structuralValue(originalKey); + KeyState keyState = + keyStateMap.computeIfAbsent(structuralKey, k -> new KeyState(originalKey)); + // Ignore any key from windmill that has been marked pending deletion or is + // fully cached. + if (keyState.existence == KeyExistence.KNOWN_NONEXISTENT + || (keyState.existence == KeyExistence.KNOWN_EXIST && keyState.valuesCached)) + return; + // Or else cache contents from windmill. + keyState.existence = KeyExistence.KNOWN_EXIST; + keyState.values.extendWith(entry.getValue()); + keyState.valuesSize += Iterables.size(entry.getValue()); + keyState.valuesCached = true; + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + allKeysKnown = true; + complete = true; + return Iterables.unmodifiableIterable( + unnestCachedEntries(mergedCachedEntries(null).entrySet())); + } catch (InterruptedException | ExecutionException | IOException e) { + if (e instanceof InterruptedException) { + Thread.currentThread().interrupt(); + } + throw new RuntimeException("Unable to read state", e); + } + } + + @Override + @SuppressWarnings("FutureReturnValueIgnored") + public ReadableState>> readLater() { + WindmillMultimap.this.necessaryEntriesFromStorageFuture(false); + return this; + } + + /** + * Collect all cached entries into a map and all KNOWN_NONEXISTENT keys to + * knownNonexistentKeys(if not null). Note that this method is not side-effect-free: it unloads + * any key that is not KNOWN_EXIST and not pending deletion from cache; also if complete it + * marks the valuesCached of any key that is KNOWN_EXIST to true, entries() depends on this + * behavior when the fetched result is weighted to iterate the whole keyStateMap one less time. + * For each cached key, returns its structural key and a tuple of . + */ + private Map>> mergedCachedEntries( + Set knownNonexistentKeys) { + Map>> cachedEntries = Maps.newHashMap(); + keyStateMap + .entrySet() + .removeIf( + (entry -> { + Object structuralKey = entry.getKey(); + KeyState keyState = entry.getValue(); + if (complete && keyState.existence == KeyExistence.KNOWN_EXIST) { + keyState.valuesCached = true; + } + ConcatIterables it = null; + if (!keyState.localAdditions.isEmpty()) { + it = new ConcatIterables<>(); + it.extendWith( + Iterables.limit(keyState.localAdditions, keyState.localAdditions.size())); + } + if (keyState.valuesCached) { + if (it == null) it = new ConcatIterables<>(); + it.extendWith(Iterables.limit(keyState.values, keyState.valuesSize)); + } + if (it != null) { + cachedEntries.put( + structuralKey, Triple.of(keyState.originalKey, keyState.valuesCached, it)); + } + if (knownNonexistentKeys != null + && keyState.existence == KeyExistence.KNOWN_NONEXISTENT) + knownNonexistentKeys.add(structuralKey); + return (keyState.existence == KeyExistence.KNOWN_NONEXISTENT + && !keyState.removedLocally) + || keyState.existence == KeyExistence.UNKNOWN_EXISTENCE; + })); + return cachedEntries; + } + + private Iterable> nonWeightedEntries( + Iterable>> lazyWindmillEntries) { + class ResultIterable implements Iterable> { + private final Iterable>> lazyWindmillEntries; + private final Map>> cachedEntries; + private final Set knownNonexistentKeys; + + ResultIterable( + Map>> cachedEntries, + Iterable>> lazyWindmillEntries, + Set knownNonexistentKeys) { + this.cachedEntries = cachedEntries; + this.lazyWindmillEntries = lazyWindmillEntries; + this.knownNonexistentKeys = knownNonexistentKeys; + } + + @Override + public Iterator> iterator() { + // Each time when the Iterable returned by entries() is iterated, a new Iterator is + // created. Every iterator must keep its own copy of seenCachedKeys so that if a key + // is paginated into multiple iterables from windmill, the cached values of this key + // will only be returned once. + Set seenCachedKeys = Sets.newHashSet(); + // notFullyCachedEntries returns all entries from windmill that are not fully cached + // and combines them with localAdditions. If a key is fully cached, contents of this + // key from windmill are ignored. + Iterable>> notFullyCachedEntries = + Iterables.filter( + Iterables.transform( + lazyWindmillEntries, + entry -> { + try { + final K key = + keyCoder.decode(entry.getKey().newInput(), Coder.Context.OUTER); + final Object structuralKey = keyCoder.structuralValue(key); + // key is deleted in cache thus fully cached. + if (knownNonexistentKeys.contains(structuralKey)) return null; + Triple> triple = + cachedEntries.get(structuralKey); + // no record of key in cache, return content in windmill. + if (triple == null) { + return Triple.of(structuralKey, key, entry.getValue()); + } + // key is fully cached in cache. + if (triple.getMiddle()) return null; + + // key is not fully cached, combine the content in windmill with local + // additions with only the first observed page for the key to ensure + // it is not repeated. + if (!seenCachedKeys.add(structuralKey)) { + return Triple.of(structuralKey, key, entry.getValue()); + } else { + ConcatIterables it = new ConcatIterables<>(); + it.extendWith(triple.getRight()); + it.extendWith(entry.getValue()); + return Triple.of(structuralKey, key, it); + } + } catch (IOException e) { + throw new RuntimeException(e); + } + }), + Objects::nonNull); + Iterator> unnestWindmill = + Iterators.concat( + Iterables.transform( + notFullyCachedEntries, + entry -> + Iterables.transform( + entry.getRight(), + v -> new AbstractMap.SimpleEntry<>(entry.getMiddle(), v)) + .iterator()) + .iterator()); + Iterator> fullyCached = + unnestCachedEntries( + Iterables.filter( + cachedEntries.entrySet(), + entry -> !seenCachedKeys.contains(entry.getKey()))) + .iterator(); + return Iterators.concat(unnestWindmill, fullyCached); + } + } + + Set knownNonexistentKeys = Sets.newHashSet(); + Map>> cachedEntries = + mergedCachedEntries(knownNonexistentKeys); + return Iterables.unmodifiableIterable( + new ResultIterable(cachedEntries, lazyWindmillEntries, knownNonexistentKeys)); + } + } + + private class ContainsKeyReadableState implements ReadableState { + final Object structuralKey; + private final K key; + ReadableState> values; + + public ContainsKeyReadableState(K key) { + this.key = key; + structuralKey = keyCoder.structuralValue(key); + values = null; + } + + @Override + public Boolean read() { + KeyState keyState = keyStateMap.getOrDefault(structuralKey, null); + if (keyState != null && keyState.existence != KeyExistence.UNKNOWN_EXISTENCE) { + return keyState.existence == KeyExistence.KNOWN_EXIST; + } + if (values == null) { + values = WindmillMultimap.this.get(key); + } + return !Iterables.isEmpty(values.read()); + } + + @Override + public ReadableState readLater() { + if (values == null) { + values = WindmillMultimap.this.get(key); + } + values.readLater(); + return this; + } + } + + private class IsEmptyReadableState implements ReadableState { + ReadableState> keys = null; + + @Override + public Boolean read() { + for (KeyState keyState : keyStateMap.values()) { + if (keyState.existence == KeyExistence.KNOWN_EXIST) { + return false; + } + } + if (keys == null) { + keys = WindmillMultimap.this.keys(); + } + return Iterables.isEmpty(keys.read()); + } + + @Override + public ReadableState readLater() { + if (keys == null) { + keys = WindmillMultimap.this.keys(); + } + keys.readLater(); + return this; + } + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillOrderedList.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillOrderedList.java new file mode 100644 index 000000000000..c92e2e93ddfe --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillOrderedList.java @@ -0,0 +1,305 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.dataflow.worker.windmill.state; + +import static org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateUtil.encodeKey; + +import java.io.Closeable; +import java.io.IOException; +import java.util.Collections; +import java.util.Iterator; +import java.util.SortedSet; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import org.apache.beam.runners.core.StateNamespace; +import org.apache.beam.runners.core.StateTable; +import org.apache.beam.runners.core.StateTag; +import org.apache.beam.runners.dataflow.worker.WindmillTimeUtils; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.state.OrderedListState; +import org.apache.beam.sdk.state.ReadableState; +import org.apache.beam.sdk.util.ByteStringOutputStream; +import org.apache.beam.sdk.values.TimestampedValue; +import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Range; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.RangeSet; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Sets; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.TreeRangeSet; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.Futures; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.joda.time.Instant; + +@SuppressWarnings({ + "nullness" // TODO(https://github.com/apache/beam/issues/20497) +}) +public class WindmillOrderedList extends SimpleWindmillState implements OrderedListState { + // The default proto values for SortedListRange correspond to the minimum and maximum + // timestamps. + static final long MIN_TS_MICROS = Windmill.SortedListRange.getDefaultInstance().getStart(); + static final long MAX_TS_MICROS = Windmill.SortedListRange.getDefaultInstance().getLimit(); + private final ByteString stateKey; + private final String stateFamily; + private final Coder elemCoder; + // We need to sort based on timestamp, but we need objects with the same timestamp to be treated + // as unique. We can't use a MultiSet as we can't construct a comparator that uniquely + // identifies objects, + // so we construct a unique in-memory long ids for each element. + private final SortedSet> pendingAdds = + Sets.newTreeSet(TimestampedValueWithId.COMPARATOR); + private final RangeSet pendingDeletes = TreeRangeSet.create(); + private final IdTracker idTracker; + private boolean complete; + private boolean cleared = false; + + WindmillOrderedList( + StateTable derivedStateTable, + StateNamespace namespace, + StateTag> spec, + String stateFamily, + Coder elemCoder, + boolean isNewKey) { + + this.stateKey = encodeKey(namespace, spec); + this.stateFamily = stateFamily; + this.elemCoder = elemCoder; + this.complete = isNewKey; + this.idTracker = new IdTracker(derivedStateTable, namespace, spec); + } + + @Override + public Iterable> read() { + return readRange(null, null); + } + + private SortedSet> getPendingAddRange( + @Nullable Instant minTimestamp, @Nullable Instant limitTimestamp) { + SortedSet> pendingInRange = pendingAdds; + if (minTimestamp != null && limitTimestamp != null) { + pendingInRange = + pendingInRange.subSet( + TimestampedValueWithId.bound(minTimestamp), + TimestampedValueWithId.bound(limitTimestamp)); + } else if (minTimestamp == null && limitTimestamp != null) { + pendingInRange = pendingInRange.headSet(TimestampedValueWithId.bound(limitTimestamp)); + } else if (limitTimestamp == null && minTimestamp != null) { + pendingInRange = pendingInRange.tailSet(TimestampedValueWithId.bound(minTimestamp)); + } + return pendingInRange; + } + + @Override + public Iterable> readRange( + @Nullable Instant minTimestamp, @Nullable Instant limitTimestamp) { + idTracker.readLater(); + + final Future>> future = getFuture(minTimestamp, limitTimestamp); + try (Closeable ignored = scopedReadState()) { + SortedSet> pendingInRange = + getPendingAddRange(minTimestamp, limitTimestamp); + + // Transform the return iterator, so it has the same type as pendingAdds. We need to ensure + // that the ids don't overlap with any in pendingAdds, so begin with pendingAdds.size(). + Iterable> data = + new Iterable>() { + // Anything returned from windmill that has been deleted should be ignored. + private final Iterable> iterable = + Iterables.filter(future.get(), tv -> !pendingDeletes.contains(tv.getTimestamp())); + + @Override + public Iterator> iterator() { + return new Iterator>() { + private final Iterator> iter = iterable.iterator(); + private long currentId = pendingAdds.size(); + + @Override + public boolean hasNext() { + return iter.hasNext(); + } + + @Override + public TimestampedValueWithId next() { + return TimestampedValueWithId.of(iter.next(), currentId++); + } + }; + } + }; + + Iterable> includingAdds = + Iterables.mergeSorted( + ImmutableList.of(data, pendingInRange), TimestampedValueWithId.COMPARATOR); + + // TODO(reuvenlax): If we have a known bounded amount of data, cache known ranges. + return Iterables.transform(includingAdds, TimestampedValueWithId::getValue); + } catch (InterruptedException | ExecutionException | IOException e) { + if (e instanceof InterruptedException) { + Thread.currentThread().interrupt(); + } + throw new RuntimeException("Unable to read state", e); + } + } + + @Override + public void clear() { + cleared = true; + complete = true; + pendingAdds.clear(); + pendingDeletes.clear(); + try { + idTracker.clear(); + } catch (ExecutionException | InterruptedException e) { + throw new RuntimeException(e); + } + } + + @Override + public void clearRange(Instant minTimestamp, Instant limitTimestamp) { + getPendingAddRange(minTimestamp, limitTimestamp).clear(); + pendingDeletes.add(Range.closedOpen(minTimestamp, limitTimestamp)); + } + + @Override + public void add(TimestampedValue value) { + // We use the current size of the container as the in-memory id. This works because + // pendingAdds is completely + // cleared when it is processed (otherwise we could end up with duplicate elements in the same + // container). These + // are not the ids that will be sent to windmill. + pendingAdds.add(TimestampedValueWithId.of(value, pendingAdds.size())); + // Leave pendingDeletes alone. Since we can have multiple values with the same timestamp, we + // may still need + // overlapping deletes to remove previous entries at this timestamp. + } + + @Override + public ReadableState isEmpty() { + return new ReadableState() { + @Override + public ReadableState readLater() { + WindmillOrderedList.this.readLater(); + return this; + } + + @Override + public Boolean read() { + return Iterables.isEmpty(WindmillOrderedList.this.read()); + } + }; + } + + @Override + public OrderedListState readLater() { + return readRangeLater(null, null); + } + + @Override + @SuppressWarnings("FutureReturnValueIgnored") + public OrderedListState readRangeLater( + @Nullable Instant minTimestamp, @Nullable Instant limitTimestamp) { + idTracker.readLater(); + getFuture(minTimestamp, limitTimestamp); + return this; + } + + @Override + public Windmill.WorkItemCommitRequest persistDirectly(WindmillStateCache.ForKeyAndFamily cache) + throws IOException { + Windmill.WorkItemCommitRequest.Builder commitBuilder = + Windmill.WorkItemCommitRequest.newBuilder(); + Windmill.TagSortedListUpdateRequest.Builder updatesBuilder = + commitBuilder + .addSortedListUpdatesBuilder() + .setStateFamily(cache.getStateFamily()) + .setTag(stateKey); + try { + if (cleared) { + // Default range. + updatesBuilder.addDeletesBuilder().build(); + cleared = false; + } + + if (!pendingAdds.isEmpty()) { + // TODO(reuvenlax): Once we start caching data, we should remove this line. We have it + // here now + // because once we persist + // added data we forget about it from the cache, so the object is no longer complete. + complete = false; + + Windmill.TagSortedListInsertRequest.Builder insertBuilder = + updatesBuilder.addInsertsBuilder(); + idTracker.add( + pendingAdds, + (elem, id) -> { + try { + ByteStringOutputStream elementStream = new ByteStringOutputStream(); + elemCoder.encode(elem.getValue(), elementStream, Coder.Context.OUTER); + insertBuilder.addEntries( + Windmill.SortedListEntry.newBuilder() + .setValue(elementStream.toByteString()) + .setSortKey( + WindmillTimeUtils.harnessToWindmillTimestamp(elem.getTimestamp())) + .setId(id)); + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + pendingAdds.clear(); + insertBuilder.build(); + } + + if (!pendingDeletes.isEmpty()) { + for (Range range : pendingDeletes.asRanges()) { + Windmill.TagSortedListDeleteRequest.Builder deletesBuilder = + updatesBuilder.addDeletesBuilder(); + deletesBuilder.setRange( + Windmill.SortedListRange.newBuilder() + .setStart(WindmillTimeUtils.harnessToWindmillTimestamp(range.lowerEndpoint())) + .setLimit(WindmillTimeUtils.harnessToWindmillTimestamp(range.upperEndpoint()))); + deletesBuilder.build(); + idTracker.remove(range); + } + pendingDeletes.clear(); + } + } catch (ExecutionException | InterruptedException e) { + throw new RuntimeException(e); + } + return commitBuilder.buildPartial(); + } + + private Future>> getFuture( + @Nullable Instant minTimestamp, @Nullable Instant limitTimestamp) { + long startSortKey = + minTimestamp != null + ? WindmillTimeUtils.harnessToWindmillTimestamp(minTimestamp) + : MIN_TS_MICROS; + long limitSortKey = + limitTimestamp != null + ? WindmillTimeUtils.harnessToWindmillTimestamp(limitTimestamp) + : MAX_TS_MICROS; + + if (complete) { + // Right now we don't cache any data, so complete means an empty list. + // TODO(reuvenlax): change this once we start caching data. + return Futures.immediateFuture(Collections.emptyList()); + } + return reader.orderedListFuture( + Range.closedOpen(startSortKey, limitSortKey), stateKey, stateFamily, elemCoder); + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillSet.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillSet.java new file mode 100644 index 000000000000..4afb879e722e --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillSet.java @@ -0,0 +1,146 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.dataflow.worker.windmill.state; + +import java.io.Closeable; +import java.io.IOException; +import java.util.Optional; +import org.apache.beam.runners.core.StateNamespace; +import org.apache.beam.runners.core.StateTag; +import org.apache.beam.runners.core.StateTags; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill; +import org.apache.beam.sdk.coders.BooleanCoder; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.state.MapState; +import org.apache.beam.sdk.state.ReadableState; +import org.apache.beam.sdk.state.SetState; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Supplier; +import org.checkerframework.checker.initialization.qual.Initialized; +import org.checkerframework.checker.nullness.qual.NonNull; +import org.checkerframework.checker.nullness.qual.UnknownKeyFor; + +public class WindmillSet extends SimpleWindmillState implements SetState { + private final WindmillMap windmillMap; + + WindmillSet( + StateNamespace namespace, + StateTag> address, + String stateFamily, + Coder keyCoder, + WindmillStateCache.ForKeyAndFamily cache, + boolean isNewKey) { + StateTag> internalMapAddress = StateTags.convertToMapTagInternal(address); + + this.windmillMap = + cache + .get(namespace, internalMapAddress) + .map(map -> (WindmillMap) map) + .orElseGet( + () -> + new WindmillMap<>( + namespace, + internalMapAddress, + stateFamily, + keyCoder, + BooleanCoder.of(), + isNewKey)); + } + + @Override + protected Windmill.WorkItemCommitRequest persistDirectly(WindmillStateCache.ForKeyAndFamily cache) + throws IOException { + return windmillMap.persistDirectly(cache); + } + + @Override + public @UnknownKeyFor @NonNull @Initialized ReadableState< + @UnknownKeyFor @NonNull @Initialized Boolean> + contains(K k) { + return windmillMap.getOrDefault(k, false); + } + + @Override + public @UnknownKeyFor @NonNull @Initialized ReadableState< + @UnknownKeyFor @NonNull @Initialized Boolean> + addIfAbsent(K k) { + return new WindmillSetAddIfAbsentReadableState(k); + } + + @Override + public void remove(K k) { + windmillMap.remove(k); + } + + @Override + public void add(K value) { + windmillMap.put(value, true); + } + + @Override + public @UnknownKeyFor @NonNull @Initialized ReadableState< + @UnknownKeyFor @NonNull @Initialized Boolean> + isEmpty() { + return windmillMap.isEmpty(); + } + + @Override + public Iterable read() { + return windmillMap.keys().read(); + } + + @Override + public @UnknownKeyFor @NonNull @Initialized SetState readLater() { + windmillMap.keys().readLater(); + return this; + } + + @Override + public void clear() { + windmillMap.clear(); + } + + @Override + void initializeForWorkItem( + WindmillStateReader reader, Supplier scopedReadStateSupplier) { + windmillMap.initializeForWorkItem(reader, scopedReadStateSupplier); + } + + @Override + void cleanupAfterWorkItem() { + windmillMap.cleanupAfterWorkItem(); + } + + private class WindmillSetAddIfAbsentReadableState implements ReadableState { + ReadableState putState; + + public WindmillSetAddIfAbsentReadableState(K k) { + putState = windmillMap.putIfAbsent(k, true); + } + + @Override + public Boolean read() { + return Optional.ofNullable(putState.read()).orElse(false); + } + + @Override + public @UnknownKeyFor @NonNull @Initialized ReadableState readLater() { + putState = putState.readLater(); + return this; + } + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillState.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillState.java new file mode 100644 index 000000000000..59fd3f8a1b37 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillState.java @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.dataflow.worker.windmill.state; + +import java.io.Closeable; +import java.io.IOException; +import java.util.concurrent.Future; +import javax.annotation.concurrent.NotThreadSafe; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Supplier; + +/** + * Abstract base class for all Windmill state. + * + *

Note that these are not thread safe; each state object is associated with a key and thus only + * accessed by a single thread at once. + */ +@SuppressWarnings({ + "nullness" // TODO(https://github.com/apache/beam/issues/20497) +}) +@NotThreadSafe +abstract class WindmillState { + protected Supplier scopedReadStateSupplier; + protected WindmillStateReader reader; + + /** + * Return an asynchronously computed {@link Windmill.WorkItemCommitRequest}. The request should be + * of a form that can be merged with others (only add to repeated fields). + */ + abstract Future persist(WindmillStateCache.ForKeyAndFamily cache) + throws IOException; + + /** Prepare this (possibly reused from cache) state for reading from {@code reader} if needed. */ + void initializeForWorkItem( + WindmillStateReader reader, Supplier scopedReadStateSupplier) { + this.reader = reader; + this.scopedReadStateSupplier = scopedReadStateSupplier; + } + + /** + * This (now cached) state should never need to interact with the reader until the next work item. + * Clear it to prevent space leaks. The reader will be reset by {@link #initializeForWorkItem} + * upon the next work item. + */ + void cleanupAfterWorkItem() { + this.reader = null; + this.scopedReadStateSupplier = null; + } + + Closeable scopedReadState() { + return scopedReadStateSupplier.get(); + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillStateCache.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateCache.java similarity index 92% rename from runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillStateCache.java rename to runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateCache.java index 700c7bbe01c2..6c1239d6ebd2 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillStateCache.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateCache.java @@ -15,12 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.dataflow.worker; +package org.apache.beam.runners.dataflow.worker.windmill.state; import java.io.IOException; import java.io.PrintWriter; import java.util.HashMap; import java.util.Objects; +import java.util.Optional; import java.util.concurrent.ConcurrentMap; import java.util.function.BiConsumer; import javax.servlet.http.HttpServletRequest; @@ -28,6 +29,9 @@ import org.apache.beam.runners.core.StateNamespace; import org.apache.beam.runners.core.StateTag; import org.apache.beam.runners.core.StateTags; +import org.apache.beam.runners.dataflow.worker.StreamingDataflowWorker; +import org.apache.beam.runners.dataflow.worker.Weighers; +import org.apache.beam.runners.dataflow.worker.WindmillComputationKey; import org.apache.beam.runners.dataflow.worker.status.BaseStatusServlet; import org.apache.beam.runners.dataflow.worker.status.StatusDataProvider; import org.apache.beam.sdk.state.State; @@ -84,14 +88,6 @@ public WindmillStateCache(long workerCacheMb) { .build(); } - private static class EntryStats { - long entries; - long idWeight; - long entryWeight; - long entryValues; - long maxEntryValues; - } - private EntryStats calculateEntryStats() { EntryStats stats = new EntryStats(); BiConsumer consumer = @@ -119,130 +115,52 @@ public CacheStats getCacheStats() { return stateCache.stats(); } - /** Per-computation view of the state cache. */ - public class ForComputation { - - private final String computation; - - private ForComputation(String computation) { - this.computation = computation; - } - - /** Invalidate all cache entries for this computation and {@code processingKey}. */ - public void invalidate(ByteString processingKey, long shardingKey) { - WindmillComputationKey key = - WindmillComputationKey.create(computation, processingKey, shardingKey); - // By removing the ForKey object, all state for the key is orphaned in the cache and will - // be removed by normal cache cleanup. - keyIndex.remove(key); - } - - /** - * Returns a per-computation, per-key view of the state cache. Access to the cached data for - * this key is not thread-safe. Callers should ensure that there is only a single ForKey object - * in use at a time and that access to it is synchronized or single-threaded. - */ - public ForKey forKey(WindmillComputationKey computationKey, long cacheToken, long workToken) { - ForKey forKey = keyIndex.get(computationKey); - if (forKey == null || !forKey.updateTokens(cacheToken, workToken)) { - forKey = new ForKey(computationKey, cacheToken, workToken); - // We prefer this implementation to using compute because that is implemented similarly for - // ConcurrentHashMap with the downside of it performing inserts for unchanged existing - // values as well. - keyIndex.put(computationKey, forKey); - } - return forKey; - } + /** Returns a per-computation view of the state cache. */ + public ForComputation forComputation(String computation) { + return new ForComputation(computation); } - /** Per-computation, per-key view of the state cache. */ - // Note that we utilize the default equality and hashCode for this class based upon the instance - // (instead of the fields) to optimize cache invalidation. - public class ForKey { - private final WindmillComputationKey computationKey; - // Cache token must be consistent for the key for the cache to be valid. - private final long cacheToken; - - // The work token for processing must be greater than the last work token. As work items are - // increasing for a key, a less-than or equal to work token indicates that the current token is - // for stale processing. - private long workToken; - - /** - * Returns a per-computation, per-key, per-family view of the state cache. Access to the cached - * data for this key is not thread-safe. Callers should ensure that there is only a single - * ForKeyAndFamily object in use at a time for a given computation, key, family tuple and that - * access to it is synchronized or single-threaded. - */ - public ForKeyAndFamily forFamily(String stateFamily) { - return new ForKeyAndFamily(this, stateFamily); - } - - private ForKey(WindmillComputationKey computationKey, long cacheToken, long workToken) { - this.computationKey = computationKey; - this.cacheToken = cacheToken; - this.workToken = workToken; - } - - private boolean updateTokens(long cacheToken, long workToken) { - if (this.cacheToken != cacheToken || workToken <= this.workToken) { - return false; - } - this.workToken = workToken; - return true; - } + /** Print summary statistics of the cache to the given {@link PrintWriter}. */ + @Override + public void appendSummaryHtml(PrintWriter response) { + response.println("Cache Stats:
"); + response.println( + "" + + "" + + "" + + ""); + CacheStats cacheStats = stateCache.stats(); + EntryStats entryStats = calculateEntryStats(); + response.println(""); + response.println(""); + response.println(""); + response.println(""); + response.println(""); + response.println(""); + response.println(""); + response.println(""); + response.println(""); + response.println("
Hit RatioEvictionsEntriesEntry ValuesMax Entry ValuesId WeightEntry WeightMax WeightKeys
" + cacheStats.hitRate() + "" + cacheStats.evictionCount() + "" + entryStats.entries + "(" + stateCache.size() + " inc. weak) " + entryStats.entryValues + "" + entryStats.maxEntryValues + "" + entryStats.idWeight / MEGABYTES + "MB" + entryStats.entryWeight / MEGABYTES + "MB" + getMaxWeight() / MEGABYTES + "MB" + keyIndex.size() + "

"); } - /** - * Per-computation, per-key, per-family view of the state cache. Modifications are cached locally - * and must be flushed to the cache by calling persist. This class is not thread-safe. - */ - public class ForKeyAndFamily { - final ForKey forKey; - final String stateFamily; - private final HashMap localCache; - - private ForKeyAndFamily(ForKey forKey, String stateFamily) { - this.forKey = forKey; - this.stateFamily = stateFamily; - localCache = new HashMap<>(); - } - - public String getStateFamily() { - return stateFamily; - } - - public @Nullable T get(StateNamespace namespace, StateTag address) { - StateId id = new StateId(forKey, stateFamily, namespace); - @SuppressWarnings("nullness") // Unsure how to annotate lambda return allowing null. - @Nullable - StateCacheEntry entry = localCache.computeIfAbsent(id, key -> stateCache.getIfPresent(key)); - return entry == null ? null : entry.get(namespace, address); - } - - public void put( - StateNamespace namespace, StateTag address, T value, long weight) { - StateId id = new StateId(forKey, stateFamily, namespace); - @Nullable StateCacheEntry entry = localCache.get(id); - if (entry == null) { - entry = stateCache.getIfPresent(id); - if (entry == null) { - entry = new StateCacheEntry(); - } - boolean hadValue = localCache.putIfAbsent(id, entry) != null; - Preconditions.checkState(!hadValue); + public BaseStatusServlet statusServlet() { + return new BaseStatusServlet("/cachez") { + @Override + protected void doGet(HttpServletRequest request, HttpServletResponse response) + throws IOException { + PrintWriter writer = response.getWriter(); + writer.println("

Cache Information

"); + appendSummaryHtml(writer); } - entry.put(namespace, address, value, weight); - } - - public void persist() { - localCache.forEach((id, entry) -> stateCache.put(id, entry)); - } + }; } - /** Returns a per-computation view of the state cache. */ - public ForComputation forComputation(String computation) { - return new ForComputation(computation); + private static class EntryStats { + long entries; + long idWeight; + long entryWeight; + long entryValues; + long maxEntryValues; } /** @@ -297,12 +215,10 @@ public StateCacheEntry() { this.weight = 0; } - public @Nullable T get(StateNamespace namespace, StateTag tag) { - @SuppressWarnings("unchecked") - @Nullable - WeightedValue weightedValue = - (WeightedValue) values.get(new NamespacedTag<>(namespace, tag)); - return weightedValue == null ? null : weightedValue.value; + @SuppressWarnings("unchecked") + public Optional get(StateNamespace namespace, StateTag tag) { + return Optional.ofNullable((WeightedValue) values.get(new NamespacedTag<>(namespace, tag))) + .flatMap(WeightedValue::value); } public void put( @@ -362,43 +278,137 @@ public int hashCode() { } private static class WeightedValue { - public long weight; - public @Nullable T value; + private long weight; + private @Nullable T value; + + private Optional value() { + return Optional.ofNullable(this.value); + } } } - /** Print summary statistics of the cache to the given {@link PrintWriter}. */ - @Override - public void appendSummaryHtml(PrintWriter response) { - response.println("Cache Stats:
"); - response.println( - "" - + "" - + "" - + ""); - CacheStats cacheStats = stateCache.stats(); - EntryStats entryStats = calculateEntryStats(); - response.println(""); - response.println(""); - response.println(""); - response.println(""); - response.println(""); - response.println(""); - response.println(""); - response.println(""); - response.println(""); - response.println("
Hit RatioEvictionsEntriesEntry ValuesMax Entry ValuesId WeightEntry WeightMax WeightKeys
" + cacheStats.hitRate() + "" + cacheStats.evictionCount() + "" + entryStats.entries + "(" + stateCache.size() + " inc. weak) " + entryStats.entryValues + "" + entryStats.maxEntryValues + "" + entryStats.idWeight / MEGABYTES + "MB" + entryStats.entryWeight / MEGABYTES + "MB" + getMaxWeight() / MEGABYTES + "MB" + keyIndex.size() + "

"); + /** Per-computation view of the state cache. */ + public class ForComputation { + + private final String computation; + + private ForComputation(String computation) { + this.computation = computation; + } + + /** Invalidate all cache entries for this computation and {@code processingKey}. */ + public void invalidate(ByteString processingKey, long shardingKey) { + WindmillComputationKey key = + WindmillComputationKey.create(computation, processingKey, shardingKey); + // By removing the ForKey object, all state for the key is orphaned in the cache and will + // be removed by normal cache cleanup. + keyIndex.remove(key); + } + + /** + * Returns a per-computation, per-key view of the state cache. Access to the cached data for + * this key is not thread-safe. Callers should ensure that there is only a single ForKey object + * in use at a time and that access to it is synchronized or single-threaded. + */ + public ForKey forKey(WindmillComputationKey computationKey, long cacheToken, long workToken) { + ForKey forKey = keyIndex.get(computationKey); + if (forKey == null || !forKey.updateTokens(cacheToken, workToken)) { + forKey = new ForKey(computationKey, cacheToken, workToken); + // We prefer this implementation to using compute because that is implemented similarly for + // ConcurrentHashMap with the downside of it performing inserts for unchanged existing + // values as well. + keyIndex.put(computationKey, forKey); + } + return forKey; + } } - public BaseStatusServlet statusServlet() { - return new BaseStatusServlet("/cachez") { - @Override - protected void doGet(HttpServletRequest request, HttpServletResponse response) - throws IOException { - PrintWriter writer = response.getWriter(); - writer.println("

Cache Information

"); - appendSummaryHtml(writer); + /** Per-computation, per-key view of the state cache. */ + // Note that we utilize the default equality and hashCode for this class based upon the instance + // (instead of the fields) to optimize cache invalidation. + public class ForKey { + private final WindmillComputationKey computationKey; + // Cache token must be consistent for the key for the cache to be valid. + private final long cacheToken; + + // The work token for processing must be greater than the last work token. As work items are + // increasing for a key, a less-than or equal to work token indicates that the current token is + // for stale processing. + private long workToken; + + private ForKey(WindmillComputationKey computationKey, long cacheToken, long workToken) { + this.computationKey = computationKey; + this.cacheToken = cacheToken; + this.workToken = workToken; + } + + /** + * Returns a per-computation, per-key, per-family view of the state cache. Access to the cached + * data for this key is not thread-safe. Callers should ensure that there is only a single + * ForKeyAndFamily object in use at a time for a given computation, key, family tuple and that + * access to it is synchronized or single-threaded. + */ + public ForKeyAndFamily forFamily(String stateFamily) { + return new ForKeyAndFamily(this, stateFamily); + } + + private boolean updateTokens(long cacheToken, long workToken) { + if (this.cacheToken != cacheToken || workToken <= this.workToken) { + return false; } - }; + this.workToken = workToken; + return true; + } + } + + /** + * Per-computation, per-key, per-family view of the state cache. Modifications are cached locally + * and must be flushed to the cache by calling persist. This class is not thread-safe. + */ + public class ForKeyAndFamily { + final ForKey forKey; + final String stateFamily; + private final HashMap localCache; + + private ForKeyAndFamily(ForKey forKey, String stateFamily) { + this.forKey = forKey; + this.stateFamily = stateFamily; + localCache = new HashMap<>(); + } + + public String getStateFamily() { + return stateFamily; + } + + public Optional get(StateNamespace namespace, StateTag address) { + @SuppressWarnings("nullness") + // the mapping function for localCache.computeIfAbsent (i.e stateCache.getIfPresent) is + // nullable. + Optional stateCacheEntry = + Optional.ofNullable( + localCache.computeIfAbsent( + new StateId(forKey, stateFamily, namespace), stateCache::getIfPresent)); + + return stateCacheEntry.flatMap(entry -> entry.get(namespace, address)); + } + + public void put( + StateNamespace namespace, StateTag address, T value, long weight) { + StateId id = new StateId(forKey, stateFamily, namespace); + @Nullable StateCacheEntry entry = localCache.get(id); + if (entry == null) { + entry = stateCache.getIfPresent(id); + if (entry == null) { + entry = new StateCacheEntry(); + } + boolean hadValue = localCache.putIfAbsent(id, entry) != null; + Preconditions.checkState(!hadValue); + } + entry.put(namespace, address, value, weight); + } + + public void persist() { + localCache.forEach(stateCache::put); + } } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateInternals.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateInternals.java new file mode 100644 index 000000000000..c900228e86b0 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateInternals.java @@ -0,0 +1,171 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.dataflow.worker.windmill.state; + +import java.io.Closeable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Random; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import org.apache.beam.runners.core.StateInternals; +import org.apache.beam.runners.core.StateNamespace; +import org.apache.beam.runners.core.StateTable; +import org.apache.beam.runners.core.StateTag; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkItemCommitRequest; +import org.apache.beam.sdk.state.State; +import org.apache.beam.sdk.state.StateContext; +import org.apache.beam.sdk.state.StateContexts; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Supplier; +import org.checkerframework.checker.nullness.qual.Nullable; + +/** Implementation of {@link StateInternals} using Windmill to manage the underlying data. */ +@SuppressWarnings("nullness" // TODO(https://github.com/apache/beam/issues/20497) +) +public class WindmillStateInternals implements StateInternals { + + @VisibleForTesting + static final ThreadLocal> COMPACT_NOW = + ThreadLocal.withInitial(ShouldCompactNowFn::new); + /** + * The key will be null when not in a keyed context, from the users perspective. There is still a + * "key" for the Windmill computation, but it cannot be meaningfully deserialized. + */ + private final @Nullable K key; + + private final WindmillStateCache.ForKeyAndFamily cache; + private final StateTable workItemState; + private final StateTable workItemDerivedState; + private final Supplier scopedReadStateSupplier; + + public WindmillStateInternals( + @Nullable K key, + String stateFamily, + WindmillStateReader reader, + boolean isNewKey, + WindmillStateCache.ForKeyAndFamily cache, + Supplier scopedReadStateSupplier) { + this.key = key; + this.cache = cache; + this.scopedReadStateSupplier = scopedReadStateSupplier; + this.workItemDerivedState = + CachingStateTable.builder(stateFamily, reader, cache, isNewKey, scopedReadStateSupplier) + .build(); + this.workItemState = + CachingStateTable.builder(stateFamily, reader, cache, isNewKey, scopedReadStateSupplier) + .withDerivedState(workItemDerivedState) + .build(); + } + + @Override + public @Nullable K getKey() { + return key; + } + + private void persist(List> commitsToMerge, StateTable stateTable) { + for (State location : stateTable.values()) { + if (!(location instanceof WindmillState)) { + throw new IllegalStateException( + String.format( + "%s wasn't created by %s -- unable to persist it", + location.getClass().getSimpleName(), getClass().getSimpleName())); + } + + try { + commitsToMerge.add(((WindmillState) location).persist(cache)); + } catch (IOException e) { + throw new RuntimeException("Unable to persist state", e); + } + } + + // All cached State objects now have known values. + // Clear any references to the underlying reader to prevent space leaks. + // The next work unit to use these cached State objects will reset the + // reader to a current reader in case those values are modified. + for (State location : stateTable.values()) { + ((WindmillState) location).cleanupAfterWorkItem(); + } + + // Clear out the map of already retrieved state instances. + stateTable.clear(); + } + + public void persist(final Windmill.WorkItemCommitRequest.Builder commitBuilder) { + List> commitsToMerge = new ArrayList<>(); + + // Call persist on each first, which may schedule some futures for reading. + persist(commitsToMerge, workItemState); + persist(commitsToMerge, workItemDerivedState); + + try (Closeable ignored = scopedReadStateSupplier.get()) { + for (Future commitFuture : commitsToMerge) { + commitBuilder.mergeFrom(commitFuture.get()); + } + } catch (ExecutionException | InterruptedException | IOException exc) { + if (exc instanceof InterruptedException) { + Thread.currentThread().interrupt(); + } + throw new RuntimeException("Failed to retrieve Windmill state during persist()", exc); + } + + cache.persist(); + } + + @Override + public T state(StateNamespace namespace, StateTag address) { + return workItemState.get(namespace, address, StateContexts.nullContext()); + } + + @Override + public T state( + StateNamespace namespace, StateTag address, StateContext c) { + return workItemState.get(namespace, address, c); + } + + private static class ShouldCompactNowFn implements Supplier { + /* The rate at which, on average, this will return true. */ + private static final double RATE = 0.002; + private final Random random; + private long counter; + + private ShouldCompactNowFn() { + this.random = new Random(); + this.counter = nextSample(random); + } + + private static long nextSample(Random random) { + // Use geometric distribution to find next true value. + // This lets us avoid invoking random.nextDouble() on every call. + return (long) Math.floor(Math.log(random.nextDouble()) / Math.log(1 - RATE)); + } + + @Override + public Boolean get() { + counter--; + if (counter < 0) { + counter = nextSample(random); + return true; + } else { + return false; + } + } + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillStateReader.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateReader.java similarity index 70% rename from runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillStateReader.java rename to runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateReader.java index 192a40299e0a..07d652992c1c 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillStateReader.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateReader.java @@ -15,11 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.dataflow.worker; +package org.apache.beam.runners.dataflow.worker.windmill.state; import com.google.api.client.util.Lists; -import com.google.auto.value.AutoValue; -import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import java.io.IOException; import java.io.InputStream; import java.util.AbstractMap; @@ -27,43 +25,41 @@ import java.util.Collections; import java.util.Comparator; import java.util.HashSet; -import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentLinkedQueue; -import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; import java.util.function.Supplier; import java.util.stream.Collectors; -import javax.annotation.Nonnull; import javax.annotation.Nullable; -import org.apache.beam.runners.dataflow.worker.WindmillStateReader.StateTag.Kind; +import org.apache.beam.runners.dataflow.worker.KeyTokenInvalidException; +import org.apache.beam.runners.dataflow.worker.MetricTrackingWindmillServerStub; +import org.apache.beam.runners.dataflow.worker.WindmillTimeUtils; import org.apache.beam.runners.dataflow.worker.windmill.Windmill; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.KeyedGetDataRequest; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.KeyedGetDataResponse; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.SortedListEntry; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.SortedListRange; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.TagBag; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.TagSortedListFetchRequest; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.TagValue; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.TagValuePrefixRequest; +import org.apache.beam.runners.dataflow.worker.windmill.state.StateTag.Kind; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.Coder.Context; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.Weighted; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Function; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.AbstractIterator; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ForwardingList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Range; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Sets; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ForwardingFuture; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.Futures; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.SettableFuture; import org.joda.time.Instant; @@ -80,7 +76,7 @@ "rawtypes", // TODO(https://github.com/apache/beam/issues/20447) "nullness" // TODO(https://github.com/apache/beam/issues/20497) }) -class WindmillStateReader { +public class WindmillStateReader { /** * Ideal maximum bytes in a TagBag response. However, Windmill will always return at least one * value if possible irrespective of this limit. @@ -116,190 +112,44 @@ class WindmillStateReader { public static final long MAX_KEY_BYTES = 16L << 20; // 16MB public static final long MAX_CONTINUATION_KEY_BYTES = 72L << 20; // 72MB - - /** - * When combined with a key and computationId, represents the unique address for state managed by - * Windmill. - */ - @AutoValue - abstract static class StateTag { - enum Kind { - VALUE, - BAG, - WATERMARK, - ORDERED_LIST, - VALUE_PREFIX, - MULTIMAP_SINGLE_ENTRY, - MULTIMAP_ALL - } - - abstract Kind getKind(); - - abstract ByteString getTag(); - - abstract String getStateFamily(); - - /** - * For {@link Kind#BAG, Kind#ORDERED_LIST, Kind#VALUE_PREFIX, KIND#MULTIMAP_SINGLE_ENTRY, - * KIND#MULTIMAP_ALL} kinds: A previous 'continuation_position' returned by Windmill to signal - * the resulting state was incomplete. Sending that position will request the next page of - * values. Null for first request. - * - *

Null for other kinds. - */ - @Nullable - abstract RequestPositionT getRequestPosition(); - - /** For {@link Kind#ORDERED_LIST} kinds: the range to fetch or delete. */ - @Nullable - abstract Range getSortedListRange(); - - /** For {@link Kind#MULTIMAP_SINGLE_ENTRY} kinds: the key in the multimap to fetch or delete. */ - @Nullable - abstract ByteString getMultimapKey(); - - /** - * For {@link Kind#MULTIMAP_ALL} kinds: will only return the keys of the multimap and not the - * values if true. - */ - @Nullable - abstract Boolean getOmitValues(); - - static StateTag of( - Kind kind, ByteString tag, String stateFamily, @Nullable RequestPositionT requestPosition) { - return new AutoValue_WindmillStateReader_StateTag.Builder() - .setKind(kind) - .setTag(tag) - .setStateFamily(stateFamily) - .setRequestPosition(requestPosition) - .build(); - } - - static StateTag of( - Kind kind, ByteString tag, String stateFamily) { - return of(kind, tag, stateFamily, null); - } - - abstract Builder toBuilder(); - - @AutoValue.Builder - abstract static class Builder { - abstract Builder setKind(Kind kind); - - abstract Builder setTag(ByteString tag); - - abstract Builder setStateFamily(String stateFamily); - - abstract Builder setRequestPosition( - @Nullable RequestPositionT requestPosition); - - abstract Builder setSortedListRange(@Nullable Range sortedListRange); - - abstract Builder setMultimapKey(@Nullable ByteString encodedMultimapKey); - - abstract Builder setOmitValues(Boolean omitValues); - - abstract StateTag build(); - } - } - - /** - * An in-memory collection of deserialized values and an optional continuation position to pass to - * Windmill when fetching the next page of values. - */ - private static class ValuesAndContPosition { - private final List values; - - /** Position to pass to next request for next page of values. Null if done. */ - private final @Nullable ContinuationT continuationPosition; - - public ValuesAndContPosition(List values, @Nullable ContinuationT continuationPosition) { - this.values = values; - this.continuationPosition = continuationPosition; - } - } - + @VisibleForTesting final ConcurrentLinkedQueue> pendingLookups; private final String computation; private final ByteString key; private final long shardingKey; private final long workToken; - // WindmillStateReader should only perform blocking i/o in a try-with-resources block that // declares an AutoCloseable vended by readWrapperSupplier. private final Supplier readWrapperSupplier; - - private final MetricTrackingWindmillServerStub server; - + private final MetricTrackingWindmillServerStub metricTrackingWindmillServerStub; + private final ConcurrentHashMap, CoderAndFuture> waiting; private long bytesRead = 0L; public WindmillStateReader( - MetricTrackingWindmillServerStub server, + MetricTrackingWindmillServerStub metricTrackingWindmillServerStub, String computation, ByteString key, long shardingKey, long workToken, Supplier readWrapperSupplier) { - this.server = server; + this.metricTrackingWindmillServerStub = metricTrackingWindmillServerStub; this.computation = computation; this.key = key; this.shardingKey = shardingKey; this.workToken = workToken; this.readWrapperSupplier = readWrapperSupplier; + this.waiting = new ConcurrentHashMap<>(); + this.pendingLookups = new ConcurrentLinkedQueue<>(); } public WindmillStateReader( - MetricTrackingWindmillServerStub server, + MetricTrackingWindmillServerStub metricTrackingWindmillServerStub, String computation, ByteString key, long shardingKey, long workToken) { - this(server, computation, key, shardingKey, workToken, () -> null); + this(metricTrackingWindmillServerStub, computation, key, shardingKey, workToken, () -> null); } - private static final class CoderAndFuture { - private Coder coder = null; - private final SettableFuture future; - - private CoderAndFuture(Coder coder, SettableFuture future) { - this.coder = coder; - this.future = future; - } - - private SettableFuture getFuture() { - return future; - } - - private SettableFuture getNonDoneFuture(StateTag stateTag) { - if (future.isDone()) { - throw new IllegalStateException("Future for " + stateTag + " is already done"); - } - return future; - } - - private Coder getAndClearCoder() { - if (coder == null) { - throw new IllegalStateException("Coder has already been cleared from cache"); - } - Coder result = (Coder) coder; - if (result == null) { - throw new IllegalStateException("Coder has already been cleared from cache"); - } - coder = null; - return result; - } - - private void checkNoCoder() { - if (coder != null) { - throw new IllegalStateException("Unexpected coder"); - } - } - } - - @VisibleForTesting - ConcurrentLinkedQueue> pendingLookups = new ConcurrentLinkedQueue<>(); - - private ConcurrentHashMap, CoderAndFuture> waiting = new ConcurrentHashMap<>(); - private Future stateFuture(StateTag stateTag, @Nullable Coder coder) { CoderAndFuture coderAndFuture = new CoderAndFuture<>(coder, SettableFuture.create()); CoderAndFuture existingCoderAndFutureWildcard = @@ -393,9 +243,9 @@ public Future>> valuePrefixFuture( * Internal request to fetch the next 'page' of values. Return null if no continuation position is * in {@code contStateTag}, which signals there are no more pages. */ - private @Nullable - Future> continuationFuture( - StateTag contStateTag, Coder coder) { + @Nullable + Future> continuationFuture( + StateTag contStateTag, Coder coder) { if (contStateTag.getRequestPosition() == null) { // We're done. return null; @@ -403,45 +253,6 @@ Future> continuationFuture( return stateFuture(contStateTag, coder); } - /** - * A future which will trigger a GetData request to Windmill for all outstanding futures on the - * first {@link #get}. - */ - private static class WrappedFuture extends ForwardingFuture.SimpleForwardingFuture { - /** - * The reader we'll use to service the eventual read. Null if read has been fulfilled. - * - *

NOTE: We must clear this after the read is fulfilled to prevent space leaks. - */ - private @Nullable WindmillStateReader reader; - - public WrappedFuture(WindmillStateReader reader, Future delegate) { - super(delegate); - this.reader = reader; - } - - @Override - public T get() throws InterruptedException, ExecutionException { - if (!delegate().isDone() && reader != null) { - // Only one thread per reader, so no race here. - reader.startBatchAndBlock(); - } - reader = null; - return super.get(); - } - - @Override - public T get(long timeout, TimeUnit unit) - throws InterruptedException, ExecutionException, TimeoutException { - if (!delegate().isDone() && reader != null) { - // Only one thread per reader, so no race here. - reader.startBatchAndBlock(); - } - reader = null; - return super.get(timeout, unit); - } - } - private Future wrappedFuture(final Future future) { if (future.isDone()) { // If the underlying lookup is already complete, we don't need to create the wrapper. @@ -452,59 +263,6 @@ private Future wrappedFuture(final Future future) { } } - /** Function to extract an {@link Iterable} from the continuation-supporting page read future. */ - private static class ToIterableFunction - implements Function, Iterable> { - /** - * Reader to request continuation pages from, or {@literal null} if no continuation pages - * required. - */ - private @Nullable WindmillStateReader reader; - - private final StateTag stateTag; - private final Coder coder; - - public ToIterableFunction( - WindmillStateReader reader, StateTag stateTag, Coder coder) { - this.reader = reader; - this.stateTag = stateTag; - this.coder = coder; - } - - @SuppressFBWarnings( - value = "NP_METHOD_PARAMETER_TIGHTENS_ANNOTATION", - justification = "https://github.com/google/guava/issues/920") - @Override - public Iterable apply( - @Nonnull ValuesAndContPosition valuesAndContPosition) { - if (valuesAndContPosition.continuationPosition == null) { - // Number of values is small enough Windmill sent us the entire bag in one response. - reader = null; - return valuesAndContPosition.values; - } else { - // Return an iterable which knows how to come back for more. - StateTag.Builder continuationTBuilder = - StateTag.of( - stateTag.getKind(), - stateTag.getTag(), - stateTag.getStateFamily(), - valuesAndContPosition.continuationPosition) - .toBuilder(); - if (stateTag.getSortedListRange() != null) { - continuationTBuilder.setSortedListRange(stateTag.getSortedListRange()).build(); - } - if (stateTag.getMultimapKey() != null) { - continuationTBuilder.setMultimapKey(stateTag.getMultimapKey()).build(); - } - if (stateTag.getOmitValues() != null) { - continuationTBuilder.setOmitValues(stateTag.getOmitValues()).build(); - } - return new PagingIterable( - reader, valuesAndContPosition.values, continuationTBuilder.build(), coder); - } - } - } - /** * Return future which transforms a {@code ValuesAndContPosition} result into the initial * Iterable result expected from the external caller. @@ -588,14 +346,7 @@ public void startBatchAndBlock() { return; } - Windmill.KeyedGetDataRequest request = createRequest(toFetch); - Windmill.KeyedGetDataResponse response; - try (AutoCloseable readWrapper = readWrapperSupplier.get()) { - response = server.getStateData(computation, request); - } - if (response == null) { - throw new RuntimeException("Windmill unexpectedly returned null for request " + request); - } + KeyedGetDataResponse response = tryGetDataFromWindmill(toFetch); // Removes tags from toFetch as they are processed. consumeResponse(response, toFetch); @@ -605,17 +356,31 @@ public void startBatchAndBlock() { for (StateTag stateTag : toFetch) { waiting.get(stateTag).future.setException(e); } + throw new RuntimeException(e); } } + private KeyedGetDataResponse tryGetDataFromWindmill(HashSet> stateTags) + throws Exception { + KeyedGetDataRequest keyedGetDataRequest = createRequest(stateTags); + try (AutoCloseable ignored = readWrapperSupplier.get()) { + return Optional.ofNullable( + metricTrackingWindmillServerStub.getStateData(computation, keyedGetDataRequest)) + .orElseThrow( + () -> + new RuntimeException( + "Windmill unexpectedly returned null for request " + keyedGetDataRequest)); + } + } + public long getBytesRead() { return bytesRead; } - private Windmill.KeyedGetDataRequest createRequest(Iterable> toFetch) { - Windmill.KeyedGetDataRequest.Builder keyedDataBuilder = - Windmill.KeyedGetDataRequest.newBuilder() + private KeyedGetDataRequest createRequest(Iterable> toFetch) { + KeyedGetDataRequest.Builder keyedDataBuilder = + KeyedGetDataRequest.newBuilder() .setKey(key) .setShardingKey(shardingKey) .setWorkToken(workToken); @@ -760,7 +525,7 @@ private Windmill.KeyedGetDataRequest createRequest(Iterable> toFetch return keyedDataBuilder.build(); } - private void consumeResponse(Windmill.KeyedGetDataResponse response, Set> toFetch) { + private void consumeResponse(KeyedGetDataResponse response, Set> toFetch) { bytesRead += response.getSerializedSize(); if (response.getFailed()) { throw new KeyTokenInvalidException(key.toStringUtf8()); @@ -885,41 +650,10 @@ private void consumeResponse(Windmill.KeyedGetDataResponse response, Set extends ForwardingList implements Weighted { - private List delegate; - long weight; - - WeightedList(List delegate) { - this.delegate = delegate; - this.weight = 0; - } - - @Override - protected List delegate() { - return delegate; - } - - @Override - public boolean add(T elem) { - throw new UnsupportedOperationException("Must use AddWeighted()"); - } - - @Override - public long getWeight() { - return weight; - } - - public void addWeighted(T elem, long weight) { - delegate.add(elem); - this.weight += weight; - } - } - /** The deserialized values in {@code bag} as a read-only array list. */ private List bagPageValues(TagBag bag, Coder elemCoder) { if (bag.getValuesCount() == 0) { - return new WeightedList(Collections.emptyList()); + return new WeightedList(Collections.emptyList()); } WeightedList valueList = new WeightedList<>(new ArrayList(bag.getValuesCount())); @@ -1013,23 +747,19 @@ private List>> multimapPageValues( private void consumeBag(TagBag bag, StateTag stateTag) { boolean shouldRemove; - if (stateTag.getRequestPosition() == null) { - // This is the response for the first page. - // Leave the future in the cache so subsequent requests for the first page - // can return immediately. - shouldRemove = false; - } else { - // This is a response for a subsequent page. - // Don't cache the future since we may need to make multiple requests with different - // continuation positions. - shouldRemove = true; - } + // This is the response for the first page. + // Leave the future in the cache so subsequent requests for the first page + // can return immediately. + // This is a response for a subsequent page. + // Don't cache the future since we may need to make multiple requests with different + // continuation positions. + shouldRemove = stateTag.getRequestPosition() != null; CoderAndFuture> coderAndFuture = getWaiting(stateTag, shouldRemove); SettableFuture> future = coderAndFuture.getNonDoneFuture(stateTag); try { - Coder coder = coderAndFuture.getAndClearCoder(); + Coder coder = coderAndFuture.getAndClearCoder(); List values = this.bagPageValues(bag, coder); future.set( new ValuesAndContPosition<>( @@ -1082,18 +812,14 @@ private void consumeTagValue(TagValue tagValue, StateTag stateTag) { private void consumeTagPrefixResponse( Windmill.TagValuePrefixResponse tagValuePrefixResponse, StateTag stateTag) { boolean shouldRemove; - if (stateTag.getRequestPosition() == null) { - // This is the response for the first page. - // Leave the future in the cache so subsequent - // requests for the first page - // can return immediately. - shouldRemove = false; - } else { - // This is a response for a subsequent page. - // Don't cache the future since we may need to make multiple requests with different - // continuation positions. - shouldRemove = true; - } + // This is the response for the first page. + // Leave the future in the cache so subsequent + // requests for the first page + // can return immediately. + // This is a response for a subsequent page. + // Don't cache the future since we may need to make multiple requests with different + // continuation positions. + shouldRemove = stateTag.getRequestPosition() != null; CoderAndFuture, ByteString>> coderAndFuture = getWaiting(stateTag, shouldRemove); @@ -1117,17 +843,13 @@ private void consumeTagPrefixResponse( private void consumeSortedList( Windmill.TagSortedListFetchResponse sortedListFetchResponse, StateTag stateTag) { boolean shouldRemove; - if (stateTag.getRequestPosition() == null) { - // This is the response for the first page.// Leave the future in the cache so subsequent - // requests for the first page - // can return immediately. - shouldRemove = false; - } else { - // This is a response for a subsequent page. - // Don't cache the future since we may need to make multiple requests with different - // continuation positions. - shouldRemove = true; - } + // This is the response for the first page.// Leave the future in the cache so subsequent + // requests for the first page + // can return immediately. + // This is a response for a subsequent page. + // Don't cache the future since we may need to make multiple requests with different + // continuation positions. + shouldRemove = stateTag.getRequestPosition() != null; CoderAndFuture, ByteString>> coderAndFuture = getWaiting(stateTag, shouldRemove); @@ -1187,102 +909,42 @@ private void consumeMultimapSingleEntry( } } - /** - * An iterable over elements backed by paginated GetData requests to Windmill. The iterable may be - * iterated over an arbitrary number of times and multiple iterators may be active simultaneously. - * - *

There are two pattern we wish to support with low -memory and -latency: - * - *

    - *
  1. Re-iterate over the initial elements multiple times (eg Iterables.first). We'll cache the - * initial 'page' of values returned by Windmill from our first request for the lifetime of - * the iterable. - *
  2. Iterate through all elements of a very large collection. We'll send the GetData request - * for the next page when the current page is begun. We'll discard intermediate pages and - * only retain the first. Thus the maximum memory pressure is one page plus one page per - * call to iterator. - *
- */ - private static class PagingIterable implements Iterable { - /** - * The reader we will use for scheduling continuation pages. - * - *

NOTE We've made this explicit to remind us to be careful not to cache the iterable. - */ - private final WindmillStateReader reader; - - /** Initial values returned for the first page. Never reclaimed. */ - private final List firstPage; - - /** State tag with continuation position set for second page. */ - private final StateTag secondPagePos; - - /** Coder for elements. */ - private final Coder coder; - - private PagingIterable( - WindmillStateReader reader, - List firstPage, - StateTag secondPagePos, - Coder coder) { - this.reader = reader; - this.firstPage = firstPage; - this.secondPagePos = secondPagePos; + private static final class CoderAndFuture { + private final SettableFuture future; + private Coder coder = null; + + private CoderAndFuture(Coder coder, SettableFuture future) { this.coder = coder; + this.future = future; } - @Override - public Iterator iterator() { - return new AbstractIterator() { - private Iterator currentPage = firstPage.iterator(); - private StateTag nextPagePos = secondPagePos; - private Future> pendingNextPage = - // NOTE: The results of continuation page reads are never cached. - reader.continuationFuture(nextPagePos, coder); - - @Override - protected ResultT computeNext() { - while (true) { - if (currentPage.hasNext()) { - return currentPage.next(); - } - if (pendingNextPage == null) { - return endOfData(); - } + private SettableFuture getFuture() { + return future; + } - ValuesAndContPosition valuesAndContPosition; - try { - valuesAndContPosition = pendingNextPage.get(); - } catch (InterruptedException | ExecutionException e) { - if (e instanceof InterruptedException) { - Thread.currentThread().interrupt(); - } - throw new RuntimeException("Unable to read value from state", e); - } - currentPage = valuesAndContPosition.values.iterator(); - StateTag.Builder nextPageBuilder = - StateTag.of( - nextPagePos.getKind(), - nextPagePos.getTag(), - nextPagePos.getStateFamily(), - valuesAndContPosition.continuationPosition) - .toBuilder(); - if (secondPagePos.getSortedListRange() != null) { - nextPageBuilder.setSortedListRange(secondPagePos.getSortedListRange()); - } - if (secondPagePos.getOmitValues() != null) { - nextPageBuilder.setOmitValues(secondPagePos.getOmitValues()); - } - if (secondPagePos.getMultimapKey() != null) { - nextPageBuilder.setMultimapKey(secondPagePos.getMultimapKey()); - } - nextPagePos = nextPageBuilder.build(); - pendingNextPage = - // NOTE: The results of continuation page reads are never cached. - reader.continuationFuture(nextPagePos, coder); - } - } - }; + private SettableFuture getNonDoneFuture(StateTag stateTag) { + if (future.isDone()) { + throw new IllegalStateException("Future for " + stateTag + " is already done"); + } + return future; + } + + private Coder getAndClearCoder() { + if (coder == null) { + throw new IllegalStateException("Coder has already been cleared from cache"); + } + Coder result = (Coder) coder; + if (result == null) { + throw new IllegalStateException("Coder has already been cleared from cache"); + } + coder = null; + return result; + } + + private void checkNoCoder() { + if (coder != null) { + throw new IllegalStateException("Unexpected coder"); + } } } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateUtil.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateUtil.java new file mode 100644 index 000000000000..3cac5c3c5724 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateUtil.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.dataflow.worker.windmill.state; + +import java.io.IOException; +import java.io.OutputStreamWriter; +import java.nio.charset.StandardCharsets; +import org.apache.beam.runners.core.StateNamespace; +import org.apache.beam.runners.core.StateTag; +import org.apache.beam.sdk.util.ByteStringOutputStream; +import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; + +class WindmillStateUtil { + /** Encodes the given namespace and address as {@code <namespace>+<address>}. */ + @VisibleForTesting + static ByteString encodeKey(StateNamespace namespace, StateTag address) { + try { + // Use ByteStringOutputStream rather than concatenation and String.format. We build these keys + // a lot, and this leads to better performance results. See associated benchmarks. + ByteStringOutputStream stream = new ByteStringOutputStream(); + OutputStreamWriter writer = new OutputStreamWriter(stream, StandardCharsets.UTF_8); + + // stringKey starts and ends with a slash. We separate it from the + // StateTag ID by a '+' (which is guaranteed not to be in the stringKey) because the + // ID comes from the user. + namespace.appendTo(writer); + writer.write('+'); + address.appendTo(writer); + writer.flush(); + return stream.toByteString(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillValue.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillValue.java new file mode 100644 index 000000000000..1ea6e56435d2 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillValue.java @@ -0,0 +1,160 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.dataflow.worker.windmill.state; + +import static org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateUtil.encodeKey; + +import java.io.Closeable; +import java.io.IOException; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import org.apache.beam.runners.core.StateNamespace; +import org.apache.beam.runners.core.StateTag; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.state.ValueState; +import org.apache.beam.sdk.util.ByteStringOutputStream; +import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.Futures; + +@SuppressWarnings({ + "nullness" // TODO(https://github.com/apache/beam/issues/20497) +}) +public class WindmillValue extends SimpleWindmillState implements ValueState { + private final StateNamespace namespace; + private final StateTag> address; + private final ByteString stateKey; + private final String stateFamily; + private final Coder coder; + + /** Whether we've modified the value since creation of this state. */ + private boolean modified = false; + /** Whether the in memory value is the true value. */ + private boolean valueIsKnown = false; + /** The size of the encoded value */ + private long cachedSize = -1; + + private T value; + + WindmillValue( + StateNamespace namespace, + StateTag> address, + String stateFamily, + Coder coder, + boolean isNewKey) { + this.namespace = namespace; + this.address = address; + this.stateKey = encodeKey(namespace, address); + this.stateFamily = stateFamily; + this.coder = coder; + if (isNewKey) { + this.valueIsKnown = true; + this.value = null; + } + } + + @Override + public void clear() { + modified = true; + valueIsKnown = true; + value = null; + } + + @Override + @SuppressWarnings("FutureReturnValueIgnored") + public WindmillValue readLater() { + getFuture(); + return this; + } + + @Override + public T read() { + try (Closeable scope = scopedReadState()) { + if (!valueIsKnown) { + cachedSize = -1; + } + value = getFuture().get(); + valueIsKnown = true; + return value; + } catch (InterruptedException | ExecutionException | IOException e) { + if (e instanceof InterruptedException) { + Thread.currentThread().interrupt(); + } + throw new RuntimeException("Unable to read value from state", e); + } + } + + @Override + public void write(T value) { + modified = true; + valueIsKnown = true; + cachedSize = -1; + this.value = value; + } + + @Override + protected Windmill.WorkItemCommitRequest persistDirectly(WindmillStateCache.ForKeyAndFamily cache) + throws IOException { + if (!valueIsKnown) { + // The value was never read, written or cleared. + // Thus nothing to update in Windmill. + // And no need to add to global cache. + return Windmill.WorkItemCommitRequest.newBuilder().buildPartial(); + } + + ByteString encoded = null; + if (cachedSize == -1 || modified) { + ByteStringOutputStream stream = new ByteStringOutputStream(); + if (value != null) { + coder.encode(value, stream, Coder.Context.OUTER); + } + encoded = stream.toByteString(); + cachedSize = encoded.size(); + } + + // Place in cache to avoid a future read. + cache.put(namespace, address, this, cachedSize); + + if (!modified) { + // The value was read, but never written or cleared. + // But nothing to update in Windmill. + return Windmill.WorkItemCommitRequest.newBuilder().buildPartial(); + } + + // The value was written or cleared. Commit that change to Windmill. + modified = false; + Windmill.WorkItemCommitRequest.Builder commitBuilder = + Windmill.WorkItemCommitRequest.newBuilder(); + commitBuilder + .addValueUpdatesBuilder() + .setTag(stateKey) + .setStateFamily(stateFamily) + .getValueBuilder() + .setData(encoded) + .setTimestamp(Long.MAX_VALUE); + return commitBuilder.buildPartial(); + } + + private Future getFuture() { + // WindmillStateReader guarantees that we can ask for a future for a particular tag multiple + // times and it will efficiently be reused. + return valueIsKnown + ? Futures.immediateFuture(value) + : reader.valueFuture(stateKey, stateFamily, coder); + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillWatermarkHold.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillWatermarkHold.java new file mode 100644 index 000000000000..a800c2eb6dad --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillWatermarkHold.java @@ -0,0 +1,259 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.dataflow.worker.windmill.state; + +import static org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateUtil.encodeKey; + +import java.io.Closeable; +import java.io.IOException; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import org.apache.beam.runners.core.StateNamespace; +import org.apache.beam.runners.core.StateTag; +import org.apache.beam.runners.dataflow.worker.WindmillTimeUtils; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill; +import org.apache.beam.sdk.state.ReadableState; +import org.apache.beam.sdk.state.WatermarkHoldState; +import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; +import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Optional; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.Futures; +import org.joda.time.Instant; + +@SuppressWarnings({ + "rawtypes", // TODO(https://github.com/apache/beam/issues/20447) + "nullness" // TODO(https://github.com/apache/beam/issues/20497) +}) +public class WindmillWatermarkHold extends WindmillState implements WatermarkHoldState { + // The encoded size of an Instant. + private static final int ENCODED_SIZE = 8; + + private final TimestampCombiner timestampCombiner; + private final StateNamespace namespace; + private final StateTag address; + private final ByteString stateKey; + private final String stateFamily; + + private boolean cleared = false; + /** + * If non-{@literal null}, the known current hold value, or absent if we know there are no output + * watermark holds. If {@literal null}, the current hold value could depend on holds in Windmill + * we do not yet know. + */ + private Optional cachedValue = null; + + private Instant localAdditions = null; + + WindmillWatermarkHold( + StateNamespace namespace, + StateTag address, + String stateFamily, + TimestampCombiner timestampCombiner, + boolean isNewKey) { + this.namespace = namespace; + this.address = address; + this.stateKey = encodeKey(namespace, address); + this.stateFamily = stateFamily; + this.timestampCombiner = timestampCombiner; + if (isNewKey) { + cachedValue = Optional.absent(); + } + } + + @Override + public void clear() { + cleared = true; + cachedValue = Optional.absent(); + localAdditions = null; + } + + @Override + @SuppressWarnings("FutureReturnValueIgnored") + public WindmillWatermarkHold readLater() { + getFuture(); + return this; + } + + @Override + public Instant read() { + try (Closeable scope = scopedReadState()) { + Instant persistedHold = getFuture().get(); + if (persistedHold == null) { + cachedValue = Optional.absent(); + } else { + cachedValue = Optional.of(persistedHold); + } + } catch (InterruptedException | ExecutionException | IOException e) { + if (e instanceof InterruptedException) { + Thread.currentThread().interrupt(); + } + throw new RuntimeException("Unable to read state", e); + } + + if (localAdditions == null) { + return cachedValue.orNull(); + } else if (!cachedValue.isPresent()) { + return localAdditions; + } else { + return timestampCombiner.combine(localAdditions, cachedValue.get()); + } + } + + @Override + public ReadableState isEmpty() { + throw new UnsupportedOperationException(); + } + + @Override + public void add(Instant outputTime) { + localAdditions = + (localAdditions == null) + ? outputTime + : timestampCombiner.combine(outputTime, localAdditions); + } + + @Override + public TimestampCombiner getTimestampCombiner() { + return timestampCombiner; + } + + @Override + public Future persist( + final WindmillStateCache.ForKeyAndFamily cache) { + + Future result; + + if (!cleared && localAdditions == null) { + // No changes, so no need to update Windmill and no need to cache any value. + return Futures.immediateFuture(Windmill.WorkItemCommitRequest.newBuilder().buildPartial()); + } + + if (cleared && localAdditions == null) { + // Just clearing the persisted state; blind delete + Windmill.WorkItemCommitRequest.Builder commitBuilder = + Windmill.WorkItemCommitRequest.newBuilder(); + commitBuilder + .addWatermarkHoldsBuilder() + .setTag(stateKey) + .setStateFamily(stateFamily) + .setReset(true); + + result = Futures.immediateFuture(commitBuilder.buildPartial()); + } else if (cleared && localAdditions != null) { + // Since we cleared before adding, we can do a blind overwrite of persisted state + Windmill.WorkItemCommitRequest.Builder commitBuilder = + Windmill.WorkItemCommitRequest.newBuilder(); + commitBuilder + .addWatermarkHoldsBuilder() + .setTag(stateKey) + .setStateFamily(stateFamily) + .setReset(true) + .addTimestamps(WindmillTimeUtils.harnessToWindmillTimestamp(localAdditions)); + + cachedValue = Optional.of(localAdditions); + + result = Futures.immediateFuture(commitBuilder.buildPartial()); + } else if (!cleared && localAdditions != null) { + // Otherwise, we need to combine the local additions with the already persisted data + result = combineWithPersisted(); + } else { + throw new IllegalStateException("Unreachable condition"); + } + + return Futures.lazyTransform( + result, + result1 -> { + cleared = false; + localAdditions = null; + if (cachedValue != null) { + cache.put(namespace, address, WindmillWatermarkHold.this, ENCODED_SIZE); + } + return result1; + }); + } + + private Future getFuture() { + return cachedValue != null + ? Futures.immediateFuture(cachedValue.orNull()) + : reader.watermarkFuture(stateKey, stateFamily); + } + + /** + * Combines local additions with persisted data and mutates the {@code commitBuilder} to write the + * result. + */ + private Future combineWithPersisted() { + boolean windmillCanCombine = false; + + // If the combined output time depends only on the window, then we are just blindly adding + // the same value that may or may not already be present. This depends on the state only being + // used for one window. + windmillCanCombine |= timestampCombiner.dependsOnlyOnWindow(); + + // If the combined output time depends only on the earliest input timestamp, then because + // assignOutputTime is monotonic, the hold only depends on the earliest output timestamp + // (which is the value submitted as a watermark hold). The only way holds for later inputs + // can be redundant is if the are later (or equal) to the earliest. So taking the MIN + // implicitly, as Windmill does, has the desired behavior. + windmillCanCombine |= timestampCombiner.dependsOnlyOnEarliestTimestamp(); + + if (windmillCanCombine) { + // We do a blind write and let Windmill take the MIN + Windmill.WorkItemCommitRequest.Builder commitBuilder = + Windmill.WorkItemCommitRequest.newBuilder(); + commitBuilder + .addWatermarkHoldsBuilder() + .setTag(stateKey) + .setStateFamily(stateFamily) + .addTimestamps(WindmillTimeUtils.harnessToWindmillTimestamp(localAdditions)); + + if (cachedValue != null) { + cachedValue = + Optional.of( + cachedValue.isPresent() + ? timestampCombiner.combine(cachedValue.get(), localAdditions) + : localAdditions); + } + + return Futures.immediateFuture(commitBuilder.buildPartial()); + } else { + // The non-fast path does a read-modify-write + return Futures.lazyTransform( + (cachedValue != null) + ? Futures.immediateFuture(cachedValue.orNull()) + : reader.watermarkFuture(stateKey, stateFamily), + priorHold -> { + cachedValue = + Optional.of( + (priorHold != null) + ? timestampCombiner.combine(priorHold, localAdditions) + : localAdditions); + Windmill.WorkItemCommitRequest.Builder commitBuilder = + Windmill.WorkItemCommitRequest.newBuilder(); + commitBuilder + .addWatermarkHoldsBuilder() + .setTag(stateKey) + .setStateFamily(stateFamily) + .setReset(true) + .addTimestamps(WindmillTimeUtils.harnessToWindmillTimestamp(cachedValue.get())); + + return commitBuilder.buildPartial(); + }); + } + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WrappedFuture.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WrappedFuture.java new file mode 100644 index 000000000000..035f6ec8e93d --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WrappedFuture.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.dataflow.worker.windmill.state; + +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import javax.annotation.Nullable; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ForwardingFuture; + +/** + * A future which will trigger a GetData request to Windmill for all outstanding futures on the + * first {@link #get}. + */ +public class WrappedFuture extends ForwardingFuture.SimpleForwardingFuture { + /** + * The reader we'll use to service the eventual read. Null if read has been fulfilled. + * + *

NOTE: We must clear this after the read is fulfilled to prevent space leaks. + */ + private @Nullable WindmillStateReader reader; + + public WrappedFuture(WindmillStateReader reader, Future delegate) { + super(delegate); + this.reader = reader; + } + + @Override + public T get() throws InterruptedException, ExecutionException { + if (!delegate().isDone() && reader != null) { + // Only one thread per reader, so no race here. + reader.startBatchAndBlock(); + } + reader = null; + return super.get(); + } + + @Override + public T get(long timeout, TimeUnit unit) + throws InterruptedException, ExecutionException, TimeoutException { + if (!delegate().isDone() && reader != null) { + // Only one thread per reader, so no race here. + reader.startBatchAndBlock(); + } + reader = null; + return super.get(timeout, unit); + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContextTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContextTest.java index 57e29bb21dee..6620dbdaab79 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContextTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContextTest.java @@ -56,6 +56,8 @@ import org.apache.beam.runners.dataflow.worker.profiler.ScopedProfiler.NoopProfileScope; import org.apache.beam.runners.dataflow.worker.profiler.ScopedProfiler.ProfileScope; import org.apache.beam.runners.dataflow.worker.windmill.Windmill; +import org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateCache; +import org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateReader; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.metrics.MetricsContainer; import org.apache.beam.sdk.options.PipelineOptionsFactory; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkStateTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkStateTest.java index 1f3dee4b76ba..12ae816de829 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkStateTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkStateTest.java @@ -33,10 +33,10 @@ import java.util.Map; import java.util.Optional; import javax.annotation.Nullable; -import org.apache.beam.runners.dataflow.worker.WindmillStateCache; import org.apache.beam.runners.dataflow.worker.streaming.ActiveWorkState.ActivateWorkResult; import org.apache.beam.runners.dataflow.worker.windmill.Windmill; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.KeyedGetDataRequest; +import org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateCache; import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.joda.time.Instant; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillStateCacheTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateCacheTest.java similarity index 75% rename from runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillStateCacheTest.java rename to runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateCacheTest.java index eca431af11a7..cc6633f1b704 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillStateCacheTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateCacheTest.java @@ -15,17 +15,18 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.dataflow.worker; +package org.apache.beam.runners.dataflow.worker.windmill.state; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; import java.io.IOException; import java.util.Objects; +import java.util.Optional; import org.apache.beam.runners.core.StateNamespace; import org.apache.beam.runners.core.StateNamespaces; import org.apache.beam.runners.core.StateTag; import org.apache.beam.runners.dataflow.options.DataflowWorkerHarnessOptions; +import org.apache.beam.runners.dataflow.worker.WindmillComputationKey; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.state.State; import org.apache.beam.sdk.state.StateSpec; @@ -38,7 +39,7 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -/** Tests for {@link WindmillStateCache}. */ +/** Tests for {@link org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateCache}. */ @RunWith(JUnit4.class) public class WindmillStateCacheTest { @@ -153,10 +154,11 @@ public void setUp() { public void testBasic() throws Exception { WindmillStateCache.ForKeyAndFamily keyCache = cache.forComputation(COMPUTATION).forKey(COMPUTATION_KEY, 0L, 1L).forFamily(STATE_FAMILY); - assertNull(keyCache.get(StateNamespaces.global(), new TestStateTag("tag1"))); - assertNull(keyCache.get(windowNamespace(0), new TestStateTag("tag2"))); - assertNull(keyCache.get(triggerNamespace(0, 0), new TestStateTag("tag3"))); - assertNull(keyCache.get(triggerNamespace(0, 0), new TestStateTag("tag2"))); + assertEquals( + Optional.empty(), keyCache.get(StateNamespaces.global(), new TestStateTag("tag1"))); + assertEquals(Optional.empty(), keyCache.get(windowNamespace(0), new TestStateTag("tag2"))); + assertEquals(Optional.empty(), keyCache.get(triggerNamespace(0, 0), new TestStateTag("tag3"))); + assertEquals(Optional.empty(), keyCache.get(triggerNamespace(0, 0), new TestStateTag("tag2"))); assertEquals(0, cache.getWeight()); keyCache.put(StateNamespaces.global(), new TestStateTag("tag1"), new TestState("g1"), 2); @@ -177,12 +179,17 @@ public void testBasic() throws Exception { keyCache = cache.forComputation(COMPUTATION).forKey(COMPUTATION_KEY, 0L, 2L).forFamily(STATE_FAMILY); assertEquals( - new TestState("g1"), keyCache.get(StateNamespaces.global(), new TestStateTag("tag1"))); - assertEquals(new TestState("w2"), keyCache.get(windowNamespace(0), new TestStateTag("tag2"))); + Optional.of(new TestState("g1")), + keyCache.get(StateNamespaces.global(), new TestStateTag("tag1"))); + assertEquals( + Optional.of(new TestState("w2")), + keyCache.get(windowNamespace(0), new TestStateTag("tag2"))); assertEquals( - new TestState("t3"), keyCache.get(triggerNamespace(0, 0), new TestStateTag("tag3"))); + Optional.of(new TestState("t3")), + keyCache.get(triggerNamespace(0, 0), new TestStateTag("tag3"))); assertEquals( - new TestState("t2"), keyCache.get(triggerNamespace(0, 0), new TestStateTag("tag2"))); + Optional.of(new TestState("t2")), + keyCache.get(triggerNamespace(0, 0), new TestStateTag("tag2"))); } /** Verifies that max weight is set */ @@ -196,7 +203,8 @@ public void testMaxWeight() throws Exception { public void testInvalidation() throws Exception { WindmillStateCache.ForKeyAndFamily keyCache = cache.forComputation(COMPUTATION).forKey(COMPUTATION_KEY, 0L, 1L).forFamily(STATE_FAMILY); - assertNull(keyCache.get(StateNamespaces.global(), new TestStateTag("tag1"))); + assertEquals( + Optional.empty(), keyCache.get(StateNamespaces.global(), new TestStateTag("tag1"))); keyCache.put(StateNamespaces.global(), new TestStateTag("tag1"), new TestState("g1"), 2); keyCache.persist(); @@ -204,11 +212,13 @@ public void testInvalidation() throws Exception { cache.forComputation(COMPUTATION).forKey(COMPUTATION_KEY, 0L, 2L).forFamily(STATE_FAMILY); assertEquals(127, cache.getWeight()); assertEquals( - new TestState("g1"), keyCache.get(StateNamespaces.global(), new TestStateTag("tag1"))); + Optional.of(new TestState("g1")), + keyCache.get(StateNamespaces.global(), new TestStateTag("tag1"))); keyCache = cache.forComputation(COMPUTATION).forKey(COMPUTATION_KEY, 1L, 3L).forFamily(STATE_FAMILY); - assertNull(keyCache.get(StateNamespaces.global(), new TestStateTag("tag1"))); + assertEquals( + Optional.empty(), keyCache.get(StateNamespaces.global(), new TestStateTag("tag1"))); assertEquals(127, cache.getWeight()); } @@ -225,8 +235,8 @@ public void testEviction() throws Exception { // Eviction is atomic across the whole window. keyCache = cache.forComputation(COMPUTATION).forKey(COMPUTATION_KEY, 0L, 2L).forFamily(STATE_FAMILY); - assertNull(keyCache.get(windowNamespace(0), new TestStateTag("tag2"))); - assertNull(keyCache.get(triggerNamespace(0, 0), new TestStateTag("tag3"))); + assertEquals(Optional.empty(), keyCache.get(windowNamespace(0), new TestStateTag("tag2"))); + assertEquals(Optional.empty(), keyCache.get(triggerNamespace(0, 0), new TestStateTag("tag3"))); } /** Verifies that the cache does not vend for stale work tokens. */ @@ -239,35 +249,35 @@ public void testStaleWorkItem() throws Exception { keyCache.put(windowNamespace(0), tag, new TestState("w2"), 2); // Same cache. - assertEquals(new TestState("w2"), keyCache.get(windowNamespace(0), tag)); + assertEquals(Optional.of(new TestState("w2")), keyCache.get(windowNamespace(0), tag)); assertEquals(0, cache.getWeight()); keyCache.persist(); assertEquals(127, cache.getWeight()); - assertEquals(new TestState("w2"), keyCache.get(windowNamespace(0), tag)); + assertEquals(Optional.of(new TestState("w2")), keyCache.get(windowNamespace(0), tag)); // Previous work token. keyCache = cache.forComputation(COMPUTATION).forKey(COMPUTATION_KEY, 0L, 1L).forFamily(STATE_FAMILY); - assertNull(keyCache.get(windowNamespace(0), tag)); + assertEquals(Optional.empty(), keyCache.get(windowNamespace(0), tag)); // Retry of work token that inserted. keyCache = cache.forComputation(COMPUTATION).forKey(COMPUTATION_KEY, 0L, 2L).forFamily(STATE_FAMILY); - assertNull(keyCache.get(windowNamespace(0), tag)); + assertEquals(Optional.empty(), keyCache.get(windowNamespace(0), tag)); keyCache = cache.forComputation(COMPUTATION).forKey(COMPUTATION_KEY, 0L, 10L).forFamily(STATE_FAMILY); - assertNull(keyCache.get(windowNamespace(0), tag)); + assertEquals(Optional.empty(), keyCache.get(windowNamespace(0), tag)); keyCache.put(windowNamespace(0), tag, new TestState("w3"), 2); // Ensure that second put updated work token. keyCache = cache.forComputation(COMPUTATION).forKey(COMPUTATION_KEY, 0L, 5L).forFamily(STATE_FAMILY); - assertNull(keyCache.get(windowNamespace(0), tag)); + assertEquals(Optional.empty(), keyCache.get(windowNamespace(0), tag)); keyCache = cache.forComputation(COMPUTATION).forKey(COMPUTATION_KEY, 0L, 15L).forFamily(STATE_FAMILY); - assertNull(keyCache.get(windowNamespace(0), tag)); + assertEquals(Optional.empty(), keyCache.get(windowNamespace(0), tag)); } /** Verifies that caches are kept independently per-key. */ @@ -293,7 +303,7 @@ public void testMultipleKeys() throws Exception { TestState state1 = new TestState("g1"); keyCache1.put(StateNamespaces.global(), tag, state1, 2); - assertEquals(state1, keyCache1.get(StateNamespaces.global(), tag)); + assertEquals(Optional.of(state1), keyCache1.get(StateNamespaces.global(), tag)); keyCache1.persist(); keyCache1 = @@ -301,22 +311,22 @@ public void testMultipleKeys() throws Exception { .forComputation("comp1") .forKey(computationKey("comp1", "key1", SHARDING_KEY), 0L, 1L) .forFamily(STATE_FAMILY); - assertEquals(state1, keyCache1.get(StateNamespaces.global(), tag)); - assertNull(keyCache2.get(StateNamespaces.global(), tag)); - assertNull(keyCache3.get(StateNamespaces.global(), tag)); + assertEquals(Optional.of(state1), keyCache1.get(StateNamespaces.global(), tag)); + assertEquals(Optional.empty(), keyCache2.get(StateNamespaces.global(), tag)); + assertEquals(Optional.empty(), keyCache3.get(StateNamespaces.global(), tag)); TestState state2 = new TestState("g2"); keyCache2.put(StateNamespaces.global(), tag, state2, 2); keyCache2.persist(); - assertEquals(state2, keyCache2.get(StateNamespaces.global(), tag)); + assertEquals(Optional.of(state2), keyCache2.get(StateNamespaces.global(), tag)); keyCache2 = cache .forComputation("comp1") .forKey(computationKey("comp1", "key2", SHARDING_KEY), 0L, 20L) .forFamily(STATE_FAMILY); - assertEquals(state2, keyCache2.get(StateNamespaces.global(), tag)); - assertEquals(state1, keyCache1.get(StateNamespaces.global(), tag)); - assertNull(keyCache3.get(StateNamespaces.global(), tag)); + assertEquals(Optional.of(state2), keyCache2.get(StateNamespaces.global(), tag)); + assertEquals(Optional.of(state1), keyCache1.get(StateNamespaces.global(), tag)); + assertEquals(Optional.empty(), keyCache3.get(StateNamespaces.global(), tag)); } /** Verifies that caches are kept independently per shard of key. */ @@ -343,28 +353,28 @@ public void testMultipleShardsOfKey() throws Exception { TestState state1 = new TestState("g1"); key1CacheShard1.put(StateNamespaces.global(), tag, state1, 2); key1CacheShard1.persist(); - assertEquals(state1, key1CacheShard1.get(StateNamespaces.global(), tag)); + assertEquals(Optional.of(state1), key1CacheShard1.get(StateNamespaces.global(), tag)); key1CacheShard1 = cache .forComputation(COMPUTATION) .forKey(computationKey(COMPUTATION, "key1", 1), 0L, 1L) .forFamily(STATE_FAMILY); - assertEquals(state1, key1CacheShard1.get(StateNamespaces.global(), tag)); - assertNull(key1CacheShard2.get(StateNamespaces.global(), tag)); - assertNull(key2CacheShard1.get(StateNamespaces.global(), tag)); + assertEquals(Optional.of(state1), key1CacheShard1.get(StateNamespaces.global(), tag)); + assertEquals(Optional.empty(), key1CacheShard2.get(StateNamespaces.global(), tag)); + assertEquals(Optional.empty(), key2CacheShard1.get(StateNamespaces.global(), tag)); TestState state2 = new TestState("g2"); key1CacheShard2.put(StateNamespaces.global(), tag, state2, 2); - assertEquals(state2, key1CacheShard2.get(StateNamespaces.global(), tag)); + assertEquals(Optional.of(state2), key1CacheShard2.get(StateNamespaces.global(), tag)); key1CacheShard2.persist(); key1CacheShard2 = cache .forComputation(COMPUTATION) .forKey(computationKey(COMPUTATION, "key1", 2), 0L, 20L) .forFamily(STATE_FAMILY); - assertEquals(state2, key1CacheShard2.get(StateNamespaces.global(), tag)); - assertEquals(state1, key1CacheShard1.get(StateNamespaces.global(), tag)); - assertNull(key2CacheShard1.get(StateNamespaces.global(), tag)); + assertEquals(Optional.of(state2), key1CacheShard2.get(StateNamespaces.global(), tag)); + assertEquals(Optional.of(state1), key1CacheShard1.get(StateNamespaces.global(), tag)); + assertEquals(Optional.empty(), key2CacheShard1.get(StateNamespaces.global(), tag)); } /** Verifies that caches are kept independently per-family. */ @@ -379,22 +389,22 @@ public void testMultipleFamilies() throws Exception { TestState state1 = new TestState("g1"); family1.put(StateNamespaces.global(), tag, state1, 2); - assertEquals(state1, family1.get(StateNamespaces.global(), tag)); + assertEquals(Optional.of(state1), family1.get(StateNamespaces.global(), tag)); family1.persist(); TestState state2 = new TestState("g2"); family2.put(StateNamespaces.global(), tag, state2, 2); family2.persist(); - assertEquals(state2, family2.get(StateNamespaces.global(), tag)); + assertEquals(Optional.of(state2), family2.get(StateNamespaces.global(), tag)); keyCache = cache.forComputation("comp1").forKey(computationKey("comp1", "key1", SHARDING_KEY), 0L, 1L); family1 = keyCache.forFamily("family1"); family2 = keyCache.forFamily("family2"); WindmillStateCache.ForKeyAndFamily family3 = keyCache.forFamily("family3"); - assertEquals(state1, family1.get(StateNamespaces.global(), tag)); - assertEquals(state2, family2.get(StateNamespaces.global(), tag)); - assertNull(family3.get(StateNamespaces.global(), tag)); + assertEquals(Optional.of(state1), family1.get(StateNamespaces.global(), tag)); + assertEquals(Optional.of(state2), family2.get(StateNamespaces.global(), tag)); + assertEquals(Optional.empty(), family3.get(StateNamespaces.global(), tag)); } /** Verifies explicit invalidation does indeed invalidate the correct entries. */ @@ -450,13 +460,17 @@ public void testExplicitInvalidation() throws Exception { .forKey(computationKey("comp1", "key1", 2), 0L, 1L) .forFamily(STATE_FAMILY); assertEquals( - new TestState("g1"), keyCache1.get(StateNamespaces.global(), new TestStateTag("tag1"))); + Optional.of(new TestState("g1")), + keyCache1.get(StateNamespaces.global(), new TestStateTag("tag1"))); assertEquals( - new TestState("g2"), keyCache2.get(StateNamespaces.global(), new TestStateTag("tag2"))); + Optional.of(new TestState("g2")), + keyCache2.get(StateNamespaces.global(), new TestStateTag("tag2"))); assertEquals( - new TestState("g3"), keyCache3.get(StateNamespaces.global(), new TestStateTag("tag3"))); + Optional.of(new TestState("g3")), + keyCache3.get(StateNamespaces.global(), new TestStateTag("tag3"))); assertEquals( - new TestState("g4"), keyCache4.get(StateNamespaces.global(), new TestStateTag("tag4"))); + Optional.of(new TestState("g4")), + keyCache4.get(StateNamespaces.global(), new TestStateTag("tag4"))); // Invalidation of key 1 shard 1 does not affect another shard of key 1 or other keys. cache.forComputation("comp1").invalidate(ByteString.copyFromUtf8("key1"), 1); @@ -466,23 +480,30 @@ public void testExplicitInvalidation() throws Exception { .forKey(computationKey("comp1", "key1", 1), 0L, 2L) .forFamily(STATE_FAMILY); - assertNull(keyCache1.get(StateNamespaces.global(), new TestStateTag("tag1"))); assertEquals( - new TestState("g2"), keyCache2.get(StateNamespaces.global(), new TestStateTag("tag2"))); + Optional.empty(), keyCache1.get(StateNamespaces.global(), new TestStateTag("tag1"))); + assertEquals( + Optional.of(new TestState("g2")), + keyCache2.get(StateNamespaces.global(), new TestStateTag("tag2"))); assertEquals( - new TestState("g3"), keyCache3.get(StateNamespaces.global(), new TestStateTag("tag3"))); + Optional.of(new TestState("g3")), + keyCache3.get(StateNamespaces.global(), new TestStateTag("tag3"))); assertEquals( - new TestState("g4"), keyCache4.get(StateNamespaces.global(), new TestStateTag("tag4"))); + Optional.of(new TestState("g4")), + keyCache4.get(StateNamespaces.global(), new TestStateTag("tag4"))); // Invalidation of an non-existing key affects nothing. cache.forComputation("comp1").invalidate(ByteString.copyFromUtf8("key1"), 3); assertEquals( - new TestState("g2"), keyCache2.get(StateNamespaces.global(), new TestStateTag("tag2"))); + Optional.of(new TestState("g2")), + keyCache2.get(StateNamespaces.global(), new TestStateTag("tag2"))); assertEquals( - new TestState("g3"), keyCache3.get(StateNamespaces.global(), new TestStateTag("tag3"))); + Optional.of(new TestState("g3")), + keyCache3.get(StateNamespaces.global(), new TestStateTag("tag3"))); assertEquals( - new TestState("g4"), keyCache4.get(StateNamespaces.global(), new TestStateTag("tag4"))); + Optional.of(new TestState("g4")), + keyCache4.get(StateNamespaces.global(), new TestStateTag("tag4"))); } private static class TestStateTagWithBadEquality extends TestStateTag { @@ -517,9 +538,9 @@ public void testBadCoderEquality() throws Exception { keyCache1 = cache.forComputation(COMPUTATION).forKey(COMPUTATION_KEY, 0L, 1L).forFamily(STATE_FAMILY); - assertEquals(new TestState("g1"), keyCache1.get(StateNamespaces.global(), tag)); + assertEquals(Optional.of(new TestState("g1")), keyCache1.get(StateNamespaces.global(), tag)); assertEquals( - new TestState("g1"), + Optional.of(new TestState("g1")), keyCache1.get(StateNamespaces.global(), new TestStateTagWithBadEquality("tag1"))); } } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillStateInternalsTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateInternalsTest.java similarity index 97% rename from runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillStateInternalsTest.java rename to runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateInternalsTest.java index 9f2d5eee8f87..8971c39ccaa1 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillStateInternalsTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateInternalsTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.dataflow.worker; +package org.apache.beam.runners.dataflow.worker.windmill.state; import static org.apache.beam.runners.dataflow.worker.DataflowMatchers.ByteStringMatcher.byteStringEq; import static org.apache.beam.sdk.testing.SystemNanoTimeSleeper.sleepMillis; @@ -56,8 +56,8 @@ import org.apache.beam.runners.core.StateTag; import org.apache.beam.runners.core.StateTags; import org.apache.beam.runners.dataflow.options.DataflowWorkerHarnessOptions; -import org.apache.beam.runners.dataflow.worker.WindmillStateInternals.IdTracker; -import org.apache.beam.runners.dataflow.worker.WindmillStateInternals.WindmillOrderedList; +import org.apache.beam.runners.dataflow.worker.WindmillComputationKey; +import org.apache.beam.runners.dataflow.worker.WindmillStateTestUtils; import org.apache.beam.runners.dataflow.worker.windmill.Windmill; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.TagBag; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.TagSortedListUpdateRequest; @@ -113,24 +113,21 @@ }) public class WindmillStateInternalsTest { + public static final Range FULL_ORDERED_LIST_RANGE = + Range.closedOpen(WindmillOrderedList.MIN_TS_MICROS, WindmillOrderedList.MAX_TS_MICROS); private static final StateNamespace NAMESPACE = new StateNamespaceForTest("ns"); private static final String STATE_FAMILY = "family"; - private static final StateTag> COMBINING_ADDR = StateTags.combiningValueFromInputInternal("combining", VarIntCoder.of(), Sum.ofIntegers()); private static final ByteString COMBINING_KEY = key(NAMESPACE, "combining"); private final Coder accumCoder = Sum.ofIntegers().getAccumulatorCoder(null, VarIntCoder.of()); - private long workToken = 0; - DataflowWorkerHarnessOptions options; - + private long workToken = 0; @Mock private WindmillStateReader mockReader; - private WindmillStateInternals underTest; private WindmillStateInternals underTestNewKey; private WindmillStateCache cache; - @Mock private Supplier readStateSupplier; private static ByteString key(StateNamespace namespace, String addrId) { @@ -141,6 +138,67 @@ private static ByteString systemKey(StateNamespace namespace, String addrId) { return ByteString.copyFromUtf8(namespace.stringKey() + "+s" + addrId); } + private static ByteString encodeWithCoder(T key, Coder coder) { + ByteStringOutputStream out = new ByteStringOutputStream(); + try { + coder.encode(key, out, Context.OUTER); + } catch (IOException e) { + throw new RuntimeException(e); + } + return out.toByteString(); + } + + // We use the structural value of the Multimap keys to differentiate between different keys. So we + // mix using the original key object and a duplicate but same key object so make sure the + // correctness. + private static byte[] dup(byte[] key) { + byte[] res = new byte[key.length]; + System.arraycopy(key, 0, res, 0, key.length); + return res; + } + + private static Map.Entry> multimapEntry( + byte[] key, Integer... values) { + return new AbstractMap.SimpleEntry<>( + encodeWithCoder(key, ByteArrayCoder.of()), Arrays.asList(values)); + } + + @SafeVarargs + private static List weightedList(T... entries) { + WeightedList list = new WeightedList<>(new ArrayList<>()); + for (T entry : entries) { + list.addWeighted(entry, 1); + } + return list; + } + + private static CombinableMatcher multimapEntryMatcher(byte[] key, Integer value) { + return Matchers.both(Matchers.hasProperty("key", Matchers.equalTo(key))) + .and(Matchers.hasProperty("value", Matchers.equalTo(value))); + } + + private static MultimapEntryUpdate decodeTagMultimapEntry(Windmill.TagMultimapEntry entryProto) { + try { + String key = StringUtf8Coder.of().decode(entryProto.getEntryName().newInput(), Context.OUTER); + List values = new ArrayList<>(); + for (ByteString value : entryProto.getValuesList()) { + values.add(VarIntCoder.of().decode(value.newInput(), Context.OUTER)); + } + return new MultimapEntryUpdate(key, values, entryProto.getDeleteAll()); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + private static void assertTagMultimapUpdates( + Windmill.TagMultimapUpdateRequest.Builder updates, MultimapEntryUpdate... expected) { + assertThat( + updates.getUpdatesList().stream() + .map(WindmillStateInternalsTest::decodeTagMultimapEntry) + .collect(Collectors.toList()), + Matchers.containsInAnyOrder(expected)); + } + @Before public void setUp() { MockitoAnnotations.initMocks(this); @@ -203,9 +261,8 @@ private void waitAndSet(final SettableFuture future, final T value, final .run(); } - private WindmillStateReader.WeightedList weightedList(String... elems) { - WindmillStateReader.WeightedList result = - new WindmillStateReader.WeightedList<>(new ArrayList(elems.length)); + private WeightedList weightedList(String... elems) { + WeightedList result = new WeightedList<>(new ArrayList(elems.length)); for (String elem : elems) { result.addWeighted(elem, elem.length()); } @@ -660,25 +717,6 @@ public void testMapComplexPersist() throws Exception { assertEquals(0, commitBuilder.getValueUpdatesCount()); } - private static ByteString encodeWithCoder(T key, Coder coder) { - ByteStringOutputStream out = new ByteStringOutputStream(); - try { - coder.encode(key, out, Context.OUTER); - } catch (IOException e) { - throw new RuntimeException(e); - } - return out.toByteString(); - } - - // We use the structural value of the Multimap keys to differentiate between different keys. So we - // mix using the original key object and a duplicate but same key object so make sure the - // correctness. - private static byte[] dup(byte[] key) { - byte[] res = new byte[key.length]; - System.arraycopy(key, 0, res, 0, key.length); - return res; - } - @Test public void testMultimapGet() { final String tag = "multimap"; @@ -796,7 +834,7 @@ public void testMultimapRemovePersistPut() { underTest.persist(commitBuilder); assertTagMultimapUpdates( Iterables.getOnlyElement(commitBuilder.getMultimapUpdatesBuilderList()), - new MultimapEntryUpdate(key, Arrays.asList(4), true)); + new MultimapEntryUpdate(key, Collections.singletonList(4), true)); multimapState.put(key, 5); assertThat(multimapState.get(key).read(), Matchers.containsInAnyOrder(4, 5)); @@ -887,22 +925,6 @@ public void testMultimapLocalClearOverrideStorage() { assertTrue(multimapState.isEmpty().read()); } - private static Map.Entry> multimapEntry( - byte[] key, Integer... values) { - return new AbstractMap.SimpleEntry<>( - encodeWithCoder(key, ByteArrayCoder.of()), Arrays.asList(values)); - } - - @SafeVarargs - private static List weightedList(T... entries) { - WindmillStateReader.WeightedList list = - new WindmillStateReader.WeightedList<>(new ArrayList<>()); - for (T entry : entries) { - list.addWeighted(entry, 1); - } - return list; - } - @Test public void testMultimapBasicEntriesAndKeys() { final String tag = "multimap"; @@ -950,11 +972,6 @@ true, key(NAMESPACE, tag), STATE_FAMILY, VarIntCoder.of())) assertThat(keys, Matchers.containsInAnyOrder(key1, key2)); } - private static CombinableMatcher multimapEntryMatcher(byte[] key, Integer value) { - return Matchers.both(Matchers.hasProperty("key", Matchers.equalTo(key))) - .and(Matchers.hasProperty("value", Matchers.equalTo(value))); - } - @Test public void testMultimapEntriesAndKeysMergeLocalAdd() { final String tag = "multimap"; @@ -1389,10 +1406,10 @@ false, key(NAMESPACE, tag), STATE_FAMILY, VarIntCoder.of())) entriesFuture, () -> new Iterator>>() { - int returnedEntries = 0; - byte[] entryKey = new byte[10_000]; // each key is 10KB final int targetEntries = 1_000_000; // return 1 million entries, which is 10 GBs - Random rand = new Random(); + final byte[] entryKey = new byte[10_000]; // each key is 10KB + final Random rand = new Random(); + int returnedEntries = 0; @Override public boolean hasNext() { @@ -1429,10 +1446,10 @@ true, key(NAMESPACE, tag), STATE_FAMILY, VarIntCoder.of())) keysFuture, () -> new Iterator>>() { - int returnedEntries = 0; - byte[] entryKey = new byte[10_000]; // each key is 10KB final int targetEntries = 1_000_000; // return 1 million entries, which is 10 GBs - Random rand = new Random(); + final byte[] entryKey = new byte[10_000]; // each key is 10KB + final Random rand = new Random(); + int returnedEntries = 0; @Override public boolean hasNext() { @@ -1477,10 +1494,10 @@ false, key(NAMESPACE, tag), STATE_FAMILY, ByteArrayCoder.of())) Iterable values = () -> new Iterator() { - int returnedValues = 0; - byte[] value = new byte[10_000]; // each value is 10KB final int targetValues = 1_000_000; // return 1 million values, which is 10 GBs - Random rand = new Random(); + final byte[] value = new byte[10_000]; // each value is 10KB + final Random rand = new Random(); + int returnedValues = 0; @Override public boolean hasNext() { @@ -1497,8 +1514,8 @@ public byte[] next() { waitAndSet( entriesFuture, - Arrays.asList( - new AbstractMap.SimpleEntry<>(encodeWithCoder(key, VarIntCoder.of()), values)), + Collections.singletonList( + new SimpleEntry<>(encodeWithCoder(key, VarIntCoder.of()), values)), 200); waitAndSet(getKeyFuture, values, 200); @@ -1509,55 +1526,6 @@ public byte[] next() { assertEquals(1_000_000, Iterables.size(valueResult)); } - private static class MultimapEntryUpdate { - String key; - Iterable values; - boolean deleteAll; - - public MultimapEntryUpdate(String key, Iterable values, boolean deleteAll) { - this.key = key; - this.values = values; - this.deleteAll = deleteAll; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (!(o instanceof MultimapEntryUpdate)) return false; - MultimapEntryUpdate that = (MultimapEntryUpdate) o; - return deleteAll == that.deleteAll - && Objects.equals(key, that.key) - && Objects.equals(values, that.values); - } - - @Override - public int hashCode() { - return Objects.hash(key, values, deleteAll); - } - } - - private static MultimapEntryUpdate decodeTagMultimapEntry(Windmill.TagMultimapEntry entryProto) { - try { - String key = StringUtf8Coder.of().decode(entryProto.getEntryName().newInput(), Context.OUTER); - List values = new ArrayList<>(); - for (ByteString value : entryProto.getValuesList()) { - values.add(VarIntCoder.of().decode(value.newInput(), Context.OUTER)); - } - return new MultimapEntryUpdate(key, values, entryProto.getDeleteAll()); - } catch (IOException e) { - throw new RuntimeException(e); - } - } - - private static void assertTagMultimapUpdates( - Windmill.TagMultimapUpdateRequest.Builder updates, MultimapEntryUpdate... expected) { - assertThat( - updates.getUpdatesList().stream() - .map(WindmillStateInternalsTest::decodeTagMultimapEntry) - .collect(Collectors.toList()), - Matchers.containsInAnyOrder(expected)); - } - @Test public void testMultimapPutAndPersist() { final String tag = "multimap"; @@ -1582,7 +1550,7 @@ public void testMultimapPutAndPersist() { assertTagMultimapUpdates( builder, new MultimapEntryUpdate(key1, Arrays.asList(1, 2), false), - new MultimapEntryUpdate(key2, Arrays.asList(2), false)); + new MultimapEntryUpdate(key2, Collections.singletonList(2), false)); } @Test @@ -1615,7 +1583,7 @@ public void testMultimapRemovePutAndPersist() { assertTagMultimapUpdates( builder, new MultimapEntryUpdate(key1, Arrays.asList(1, 2), true), - new MultimapEntryUpdate(key2, Arrays.asList(4), true)); + new MultimapEntryUpdate(key2, Collections.singletonList(4), true)); } @Test @@ -1709,7 +1677,8 @@ false, key(NAMESPACE, tag), STATE_FAMILY, VarIntCoder.of())) assertEquals(1, commitBuilder.getMultimapUpdatesCount()); Windmill.TagMultimapUpdateRequest.Builder builder = Iterables.getOnlyElement(commitBuilder.getMultimapUpdatesBuilderList()); - assertTagMultimapUpdates(builder, new MultimapEntryUpdate(key1, Arrays.asList(4), false)); + assertTagMultimapUpdates( + builder, new MultimapEntryUpdate(key1, Collections.singletonList(4), false)); } @Test @@ -1731,8 +1700,7 @@ true, key(NAMESPACE, tag), STATE_FAMILY, VarIntCoder.of())) ReadableState> keysResult = multimapState.keys().readLater(); waitAndSet( keysFuture, - new WindmillStateReader.WeightedList<>( - Arrays.asList(multimapEntry(key1), multimapEntry(key2))), + new WeightedList<>(Arrays.asList(multimapEntry(key1), multimapEntry(key2))), 30); multimapState.remove(key1); @@ -1753,7 +1721,7 @@ true, key(NAMESPACE, tag), STATE_FAMILY, VarIntCoder.of())) Windmill.TagMultimapEntry entryUpdate = Iterables.getOnlyElement(builder.getUpdatesList()); byte[] decodedKey = ByteArrayCoder.of().decode(entryUpdate.getEntryName().newInput(), Context.OUTER); - assertTrue(Arrays.equals(key1, decodedKey)); + assertArrayEquals(key1, decodedKey); assertTrue(entryUpdate.getDeleteAll()); } @@ -1870,9 +1838,6 @@ true, key(NAMESPACE, tag), STATE_FAMILY, VarIntCoder.of())) underTest.persist(commitBuilder); } - public static final Range FULL_ORDERED_LIST_RANGE = - Range.closedOpen(WindmillOrderedList.MIN_TS_MICROS, WindmillOrderedList.MAX_TS_MICROS); - @Test public void testOrderedListAddBeforeRead() throws Exception { StateTag> addr = @@ -1897,7 +1862,7 @@ public void testOrderedListAddBeforeRead() throws Exception { TimestampedValue.of("goodbye", Instant.ofEpochMilli(50)); orderedList.add(helloValue); - waitAndSet(future, Arrays.asList(worldValue), 200); + waitAndSet(future, Collections.singletonList(worldValue), 200); assertThat(orderedList.read(), Matchers.contains(worldValue, helloValue)); orderedList.add(goodbyeValue); @@ -1940,7 +1905,7 @@ public void testOrderedListIsEmptyFalse() throws Exception { STATE_FAMILY, StringUtf8Coder.of()); - waitAndSet(future, Arrays.asList(TimestampedValue.of("world", Instant.EPOCH)), 200); + waitAndSet(future, Collections.singletonList(TimestampedValue.of("world", Instant.EPOCH)), 200); assertThat(result.read(), Matchers.is(false)); } @@ -2266,10 +2231,6 @@ public void testNewOrderedListNoFetch() throws Exception { Mockito.verifyZeroInteractions(mockReader); } - // test ordered list cleared before read - // test fetch + add + read - // test ids - @Test public void testBagAddBeforeRead() throws Exception { StateTag> addr = StateTags.bag("bag", StringUtf8Coder.of()); @@ -2282,13 +2243,17 @@ public void testBagAddBeforeRead() throws Exception { bag.readLater(); bag.add("hello"); - waitAndSet(future, Arrays.asList("world"), 200); + waitAndSet(future, Collections.singletonList("world"), 200); assertThat(bag.read(), Matchers.containsInAnyOrder("hello", "world")); bag.add("goodbye"); assertThat(bag.read(), Matchers.containsInAnyOrder("hello", "world", "goodbye")); } + // test ordered list cleared before read + // test fetch + add + read + // test ids + @Test public void testBagClearBeforeRead() throws Exception { StateTag> addr = StateTags.bag("bag", StringUtf8Coder.of()); @@ -2313,7 +2278,7 @@ public void testBagIsEmptyFalse() throws Exception { ReadableState result = bag.isEmpty().readLater(); Mockito.verify(mockReader).bagFuture(key(NAMESPACE, "bag"), STATE_FAMILY, StringUtf8Coder.of()); - waitAndSet(future, Arrays.asList("world"), 200); + waitAndSet(future, Collections.singletonList("world"), 200); assertThat(result.read(), Matchers.is(false)); } @@ -2328,7 +2293,7 @@ public void testBagIsEmptyTrue() throws Exception { ReadableState result = bag.isEmpty().readLater(); Mockito.verify(mockReader).bagFuture(key(NAMESPACE, "bag"), STATE_FAMILY, StringUtf8Coder.of()); - waitAndSet(future, Arrays.asList(), 200); + waitAndSet(future, Collections.emptyList(), 200); assertThat(result.read(), Matchers.is(true)); } @@ -2436,7 +2401,7 @@ public void testCombiningAddBeforeRead() throws Exception { assertThat(value.read(), Matchers.equalTo(29)); // That get "compressed" the combiner. So, the underlying future should change: - future.set(Arrays.asList(new int[] {29})); + future.set(Collections.singletonList(new int[] {29})); value.add(2); assertThat(value.read(), Matchers.equalTo(31)); @@ -2480,7 +2445,7 @@ public void testCombiningIsEmpty() throws Exception { .bagFuture(byteString.capture(), eq(STATE_FAMILY), Mockito.>any()); assertThat(byteString.getValue(), byteStringEq(COMBINING_KEY)); - waitAndSet(future, Arrays.asList(new int[] {29}), 200); + waitAndSet(future, Collections.singletonList(new int[] {29}), 200); assertThat(result.read(), Matchers.is(false)); } @@ -2527,12 +2492,10 @@ public void testCombiningAddPersistWithCompact() throws Exception { Mockito.when( mockReader.bagFuture( - org.mockito.Matchers.any(), - org.mockito.Matchers.any(), + org.mockito.Matchers.any(), + org.mockito.Matchers.any(), org.mockito.Matchers.>any())) - .thenReturn( - Futures.>immediateFuture( - ImmutableList.of(new int[] {40}, new int[] {60}))); + .thenReturn(Futures.immediateFuture(ImmutableList.of(new int[] {40}, new int[] {60}))); GroupingState value = underTest.state(NAMESPACE, COMBINING_ADDR); @@ -2717,7 +2680,7 @@ public void testWatermarkPersistLatestEmpty() throws Exception { hold.add(new Instant(2000)); when(mockReader.watermarkFuture(key(NAMESPACE, "watermark"), STATE_FAMILY)) - .thenReturn(Futures.immediateFuture(null)); + .thenReturn(Futures.immediateFuture(null)); Windmill.WorkItemCommitRequest.Builder commitBuilder = Windmill.WorkItemCommitRequest.newBuilder(); @@ -2743,7 +2706,7 @@ public void testWatermarkPersistLatestWindmillWins() throws Exception { hold.add(new Instant(2000)); when(mockReader.watermarkFuture(key(NAMESPACE, "watermark"), STATE_FAMILY)) - .thenReturn(Futures.immediateFuture(new Instant(4000))); + .thenReturn(Futures.immediateFuture(new Instant(4000))); Windmill.WorkItemCommitRequest.Builder commitBuilder = Windmill.WorkItemCommitRequest.newBuilder(); @@ -2769,7 +2732,7 @@ public void testWatermarkPersistLatestLocalAdditionsWin() throws Exception { hold.add(new Instant(2000)); when(mockReader.watermarkFuture(key(NAMESPACE, "watermark"), STATE_FAMILY)) - .thenReturn(Futures.immediateFuture(new Instant(500))); + .thenReturn(Futures.immediateFuture(new Instant(500))); Windmill.WorkItemCommitRequest.Builder commitBuilder = Windmill.WorkItemCommitRequest.newBuilder(); @@ -2880,7 +2843,7 @@ public void testValueClearBeforeRead() throws Exception { value.clear(); - assertEquals(null, value.read()); + assertNull(value.read()); Mockito.verifyNoMoreInteractions(mockReader); } @@ -2956,7 +2919,7 @@ public void testNewValueNoFetch() throws Exception { StateTag> addr = StateTags.value("value", StringUtf8Coder.of()); ValueState value = underTestNewKey.state(NAMESPACE, addr); - assertEquals(null, value.read()); + assertNull(value.read()); // Shouldn't need to read from windmill for this. Mockito.verifyZeroInteractions(mockReader); @@ -2984,7 +2947,7 @@ public void testCachedValue() throws Exception { resetUnderTest(); value = underTest.state(NAMESPACE, addr); - assertEquals(null, value.read()); + assertNull(value.read()); underTest.persist(Windmill.WorkItemCommitRequest.newBuilder()); Mockito.verifyNoMoreInteractions(mockReader); @@ -3087,7 +3050,7 @@ public void testCachedWatermarkHold() throws Exception { resetUnderTest(); hold = underTest.state(NAMESPACE, addr); - assertEquals(null, hold.read()); + assertNull(hold.read()); underTest.persist(Windmill.WorkItemCommitRequest.newBuilder()); Mockito.verify(mockReader, times(2)).watermarkFuture(key(NAMESPACE, "watermark"), STATE_FAMILY); @@ -3109,7 +3072,7 @@ public void testCachedCombining() throws Exception { value.readLater(); value.add(1); - waitAndSet(future, Arrays.asList(new int[] {2}), 200); + waitAndSet(future, Collections.singletonList(new int[] {2}), 200); assertThat(value.read(), Matchers.equalTo(3)); underTest.persist(Windmill.WorkItemCommitRequest.newBuilder()); @@ -3149,4 +3112,31 @@ private void disableCompactOnWrite() { private void forceCompactOnWrite() { WindmillStateInternals.COMPACT_NOW.set(() -> true); } + + private static class MultimapEntryUpdate { + String key; + Iterable values; + boolean deleteAll; + + public MultimapEntryUpdate(String key, Iterable values, boolean deleteAll) { + this.key = key; + this.values = values; + this.deleteAll = deleteAll; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (!(o instanceof MultimapEntryUpdate)) return false; + MultimapEntryUpdate that = (MultimapEntryUpdate) o; + return deleteAll == that.deleteAll + && Objects.equals(key, that.key) + && Objects.equals(values, that.values); + } + + @Override + public int hashCode() { + return Objects.hash(key, values, deleteAll); + } + } } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillStateReaderTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateReaderTest.java similarity index 99% rename from runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillStateReaderTest.java rename to runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateReaderTest.java index 1981a63c0ed5..b8c4803a8f34 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillStateReaderTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateReaderTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.dataflow.worker; +package org.apache.beam.runners.dataflow.worker.windmill.state; import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; @@ -33,6 +33,10 @@ import java.util.List; import java.util.Map; import java.util.concurrent.Future; +import org.apache.beam.runners.dataflow.worker.KeyTokenInvalidException; +import org.apache.beam.runners.dataflow.worker.MetricTrackingWindmillServerStub; +import org.apache.beam.runners.dataflow.worker.WindmillStateTestUtils; +import org.apache.beam.runners.dataflow.worker.WindmillTimeUtils; import org.apache.beam.runners.dataflow.worker.windmill.Windmill; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.KeyedGetDataRequest; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.SortedListEntry; From 6a456f04b68b3e19c66cf55937e6cf2be4cada19 Mon Sep 17 00:00:00 2001 From: Robert Burke Date: Wed, 11 Oct 2023 08:42:58 -0700 Subject: [PATCH 103/435] Add user counters and msec metrics to prism UI. (#28929) Co-authored-by: lostluck <13907733+lostluck@users.noreply.github.com> --- .../beam/runners/prism/internal/web/web.go | 40 ++++++++++++++++++- 1 file changed, 39 insertions(+), 1 deletion(-) diff --git a/sdks/go/pkg/beam/runners/prism/internal/web/web.go b/sdks/go/pkg/beam/runners/prism/internal/web/web.go index 7bfbe19a910b..b7afad35aeee 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/web/web.go +++ b/sdks/go/pkg/beam/runners/prism/internal/web/web.go @@ -188,6 +188,10 @@ func (h *jobDetailsHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) { trs := pipeResp.GetPipeline().GetComponents().GetTransforms() col2T, topo := preprocessTransforms(trs) + counters := toTransformMap(results.AllMetrics().Counters()) + distributions := toTransformMap(results.AllMetrics().Distributions()) + msecs := toTransformMap(results.AllMetrics().Msecs()) + data.Transforms = make([]pTransform, 0, len(trs)) for _, id := range topo { pt := trs[id] @@ -224,6 +228,29 @@ func (h *jobDetailsHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) { strMets = append(strMets, outMets...) } + var msecMets []string + // TODO: Figure out where uniquename or id is being used in prism. It should be all global transform ID to faciliate lookups. + for _, msec := range msecs[pt.GetUniqueName()] { + msecMets = append(msecMets, fmt.Sprintf("\n- %+v", msec.Result())) + } + if len(msecMets) > 0 { + strMets = append(strMets, "Profiling metrics") + strMets = append(strMets, msecMets...) + } + + var userMetrics []string + for _, ctr := range counters[pt.GetUniqueName()] { + userMetrics = append(userMetrics, fmt.Sprintf("\n- %s.%s: %v", ctr.Namespace(), ctr.Name(), ctr.Result())) + } + for _, dist := range distributions[pt.GetUniqueName()] { + userMetrics = append(userMetrics, fmt.Sprintf("\n- %s.%s: %+v", dist.Namespace(), dist.Name(), dist.Result())) + } + + if len(userMetrics) > 0 { + strMets = append(strMets, "User metrics") + strMets = append(strMets, userMetrics...) + } + data.Transforms = append(data.Transforms, pTransform{ ID: id, Transform: pt, @@ -234,6 +261,14 @@ func (h *jobDetailsHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) { renderPage(jobPage, &data, w) } +func toTransformMap[E interface{ Transform() string }](mets []E) map[string][]E { + ret := map[string][]E{} + for _, met := range mets { + ret[met.Transform()] = append(ret[met.Transform()], met) + } + return ret +} + type pcolParent struct { L string T *pipepb.PTransform @@ -244,7 +279,10 @@ type pcolParent struct { func preprocessTransforms(trs map[string]*pipepb.PTransform) (map[string]pcolParent, []string) { ret := map[string]pcolParent{} var leaves []string - for id, t := range trs { + keys := maps.Keys(trs) + sort.Strings(keys) + for _, id := range keys { + t := trs[id] // Skip composites at this time. if len(t.GetSubtransforms()) > 0 { continue From 2bd23b166660a5a6fdf78f0615d3cd6a101122f2 Mon Sep 17 00:00:00 2001 From: Anand Inguva <34158215+AnandInguva@users.noreply.github.com> Date: Wed, 11 Oct 2023 16:20:31 +0000 Subject: [PATCH 104/435] Add counter for MLTransform and data processing transforms (#28927) --- sdks/python/apache_beam/ml/transforms/base.py | 39 ++++++++++++++++++- .../apache_beam/ml/transforms/base_test.py | 25 ++++++++++++ 2 files changed, 63 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/ml/transforms/base.py b/sdks/python/apache_beam/ml/transforms/base.py index a45928f5c8bf..49ce6e9ec1e0 100644 --- a/sdks/python/apache_beam/ml/transforms/base.py +++ b/sdks/python/apache_beam/ml/transforms/base.py @@ -25,6 +25,7 @@ from typing import TypeVar import apache_beam as beam +from apache_beam.metrics.metric import Metrics __all__ = ['MLTransform', 'ProcessHandler', 'BaseOperation'] @@ -88,6 +89,13 @@ def __call__(self, data: OperationInputT, transformed_data = {**transformed_data, **artifacts} return transformed_data + def get_counter(self): + """ + Returns the counter name for the operation. + """ + counter_name = self.__class__.__name__ + return Metrics.counter(MLTransform, f'BeamML_{counter_name}') + class ProcessHandler(Generic[ExampleT, MLTransformOutputT], abc.ABC): """ @@ -194,6 +202,9 @@ def __init__( transforms=transforms) # type: ignore[arg-type] self._process_handler = process_handler + self.transforms = transforms + self._counter = Metrics.counter( + MLTransform, f'BeamML_{self.__class__.__name__}') def expand( self, pcoll: beam.PCollection[ExampleT] @@ -209,8 +220,11 @@ def expand( Args: pcoll: A PCollection of ExampleT type. Returns: - A PCollection of MLTransformOutputT type. + A PCollection of MLTransformOutputT type """ + _ = ( + pcoll.pipeline + | "MLTransformMetricsUsage" >> MLTransformMetricsUsage(self)) return self._process_handler.process_data(pcoll) def with_transform(self, transform: BaseOperation): @@ -230,3 +244,26 @@ def _validate_transform(self, transform): raise TypeError( 'transform must be a subclass of BaseOperation. ' 'Got: %s instead.' % type(transform)) + + +class MLTransformMetricsUsage(beam.PTransform): + def __init__(self, ml_transform: MLTransform): + self._ml_transform = ml_transform + self._ml_transform._counter.inc() + + def expand(self, pipeline): + def _increment_counters(): + # increment for MLTransform. + self._ml_transform._counter.inc() + # increment if data processing transforms are passed. + transforms = ( + self._ml_transform.transforms or + self._ml_transform._process_handler.transforms) + if transforms: + for transform in transforms: + transform.get_counter().inc() + + _ = ( + pipeline + | beam.Create([None]) + | beam.Map(lambda _: _increment_counters())) diff --git a/sdks/python/apache_beam/ml/transforms/base_test.py b/sdks/python/apache_beam/ml/transforms/base_test.py index df7a6d26b47c..2e447964541b 100644 --- a/sdks/python/apache_beam/ml/transforms/base_test.py +++ b/sdks/python/apache_beam/ml/transforms/base_test.py @@ -27,6 +27,7 @@ from parameterized import parameterized import apache_beam as beam +from apache_beam.metrics.metric import MetricsFilter from apache_beam.testing.util import assert_that from apache_beam.testing.util import equal_to @@ -244,6 +245,30 @@ def test_ml_transforms_on_multiple_columns_multiple_transforms(self): equal_to(expected_output_y, equals_fn=np.array_equal), label='actual_output_y') + def test_mltransform_with_counter(self): + transforms = [ + tft.ComputeAndApplyVocabulary(columns=['y']), + tft.ScaleTo01(columns=['x']) + ] + data = [{'x': [1, 2, 3], 'y': ['a', 'b', 'c']}] + with beam.Pipeline() as p: + _ = ( + p | beam.Create(data) + | base.MLTransform( + transforms=transforms, + write_artifact_location=self.artifact_location)) + scale_to_01_counter = MetricsFilter().with_name('BeamML_ScaleTo01') + vocab_counter = MetricsFilter().with_name( + 'BeamML_ComputeAndApplyVocabulary') + mltransform_counter = MetricsFilter().with_name('BeamML_MLTransform') + result = p.result + self.assertEqual( + result.metrics().query(scale_to_01_counter)['counters'][0].result, 1) + self.assertEqual( + result.metrics().query(vocab_counter)['counters'][0].result, 1) + self.assertEqual( + result.metrics().query(mltransform_counter)['counters'][0].result, 1) + if __name__ == '__main__': unittest.main() From 7a1245d401132d2175dc2fc0efec9b9c704e8e84 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 11 Oct 2023 09:27:35 -0700 Subject: [PATCH 105/435] Bump github.com/google/go-cmp from 0.5.9 to 0.6.0 in /sdks (#28940) Bumps [github.com/google/go-cmp](https://github.com/google/go-cmp) from 0.5.9 to 0.6.0. - [Release notes](https://github.com/google/go-cmp/releases) - [Commits](https://github.com/google/go-cmp/compare/v0.5.9...v0.6.0) --- updated-dependencies: - dependency-name: github.com/google/go-cmp dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 2 +- sdks/go.sum | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index e5baa554fe6d..edb524545bd5 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -40,7 +40,7 @@ require ( github.com/dustin/go-humanize v1.0.1 github.com/go-sql-driver/mysql v1.7.1 github.com/golang/protobuf v1.5.3 // TODO(danoliveira): Fully replace this with google.golang.org/protobuf - github.com/google/go-cmp v0.5.9 + github.com/google/go-cmp v0.6.0 github.com/google/uuid v1.3.1 github.com/johannesboyne/gofakes3 v0.0.0-20221110173912-32fb85c5aed6 github.com/lib/pq v1.10.9 diff --git a/sdks/go.sum b/sdks/go.sum index f1d0c53d9685..3cdcc9ef8e99 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -259,8 +259,9 @@ github.com/google/go-cmp v0.5.4/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/ github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.6/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.8/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= -github.com/google/go-cmp v0.5.9 h1:O2Tfq5qg4qc4AmwVlvv0oLiVAGB7enBSJ2x2DqQFi38= github.com/google/go-cmp v0.5.9/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= +github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= +github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= github.com/google/martian v2.1.0+incompatible h1:/CP5g8u/VJHijgedC/Legn3BAbAaWPgecwXBIDzw5no= github.com/google/martian v2.1.0+incompatible/go.mod h1:9I4somxYTbIHy5NJKHRl3wXiIaQGbYVAs8BPL6v8lEs= github.com/google/martian/v3 v3.3.2 h1:IqNFLAmvJOgVlpdEBiQbDc2EwKW77amAycfTuWKdfvw= From 2bfcb9fda37be8d5b9550a1b118ed87d03fdf12c Mon Sep 17 00:00:00 2001 From: Joar Wandborg Date: Wed, 11 Oct 2023 19:00:04 +0200 Subject: [PATCH 106/435] Remove unnecessary a space character that got through Javadoc CI ;) (#28863) (Tongue in cheek) Spotless caught me when I did not end the first sentence of the Javadoc with a a dot, but it did not tell me I had an extra space before the `` tag at the end of the sentence. --- .../org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java index 971b91d89b74..f6224cc24b2f 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java @@ -768,7 +768,7 @@ public void testReadingWithSplits() throws Exception { /** * Regression test for [Bug]: BigtableSource - * "Desired bundle size 0 bytes must be greater than 0" #28793 . + * "Desired bundle size 0 bytes must be greater than 0" #28793. */ @Test public void testSplittingWithDesiredBundleSizeZero() throws Exception { From aedfa461fb354e0e97d75694d2a117d9ad505602 Mon Sep 17 00:00:00 2001 From: Minbo Bae <49642083+baeminbo@users.noreply.github.com> Date: Wed, 11 Oct 2023 10:57:24 -0700 Subject: [PATCH 107/435] [#20970] Fix gRPC leak by closing ResidualSource at BoundedToUnboundedSourceAdapter.Reader#init() in Dataflow worker (#28548) --- .../UnboundedReadFromBoundedSource.java | 14 +- .../UnboundedReadFromBoundedSourceTest.java | 180 ++++++++++++++++++ .../dataflow/worker/WorkerCustomSources.java | 7 +- 3 files changed, 198 insertions(+), 3 deletions(-) diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSource.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSource.java index 67697636a363..53fad782da96 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSource.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSource.java @@ -17,8 +17,8 @@ */ package org.apache.beam.runners.core.construction; -import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull; +import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; import java.io.IOException; import java.io.InputStream; @@ -288,6 +288,15 @@ private void init( residualElementsList == null ? new ResidualElements(Collections.emptyList()) : new ResidualElements(residualElementsList); + + if (this.residualSource != null) { + // close current residualSource to avoid leak of reader.close() in ResidualSource + try { + this.residualSource.close(); + } catch (IOException e) { + LOG.warn("Ignore error at closing ResidualSource", e); + } + } this.residualSource = residualSource == null ? null : new ResidualSource(residualSource, options); } @@ -465,7 +474,7 @@ public ResidualSource(BoundedSource residualSource, PipelineOptions options) } private boolean advance() throws IOException { - checkArgument(!closed, "advance() call on closed %s", getClass().getName()); + checkState(!closed, "advance() call on closed %s", getClass().getName()); if (readerDone) { return false; } @@ -505,6 +514,7 @@ BoundedSource getSource() { } Checkpoint getCheckpointMark() { + checkState(!closed, "getCheckpointMark() call on closed %s", getClass().getName()); if (reader == null) { // Reader hasn't started, checkpoint the residualSource. return new Checkpoint<>(null /* residualElements */, residualSource); diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSourceTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSourceTest.java index cd4b49262fcb..31f6842a42bc 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSourceTest.java +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSourceTest.java @@ -26,9 +26,15 @@ import java.io.FileOutputStream; import java.io.IOException; import java.io.OutputStream; +import java.lang.ref.Reference; +import java.lang.ref.ReferenceQueue; +import java.lang.ref.WeakReference; import java.nio.ByteBuffer; import java.nio.channels.ReadableByteChannel; +import java.util.Collections; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.NoSuchElementException; import java.util.Random; import org.apache.beam.runners.core.construction.UnboundedReadFromBoundedSource.BoundedToUnboundedSourceAdapter; @@ -69,10 +75,14 @@ import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** Unit tests for {@link UnboundedReadFromBoundedSource}. */ @RunWith(JUnit4.class) public class UnboundedReadFromBoundedSourceTest { + private static final Logger LOG = + LoggerFactory.getLogger(UnboundedReadFromBoundedSourceTest.class); @Rule public TemporaryFolder tmpFolder = new TemporaryFolder(); @@ -280,6 +290,38 @@ public void testReadFromCheckpointBeforeStart() throws Exception { unboundedSource.createReader(options, checkpoint).getCurrent(); } + @Test + public void testReadersClosedProperly() throws IOException { + ManagedReaderBoundedSource boundedSource = new ManagedReaderBoundedSource(0, 10); + BoundedToUnboundedSourceAdapter unboundedSource = + new BoundedToUnboundedSourceAdapter<>(boundedSource); + PipelineOptions options = PipelineOptionsFactory.create(); + + BoundedToUnboundedSourceAdapter.Reader reader = + unboundedSource.createReader(options, new Checkpoint(null, boundedSource)); + + for (int i = 0; i < 3; ++i) { + if (i == 0) { + assertTrue(reader.start()); + } else { + assertTrue(reader.advance()); + } + assertEquals(i, (int) reader.getCurrent()); + } + Checkpoint checkpoint = reader.getCheckpointMark(); + List> residualElements = checkpoint.getResidualElements(); + for (int i = 0; i < 7; ++i) { + TimestampedValue element = residualElements.get(i); + assertEquals(i + 3, (int) element.getValue()); + } + for (int i = 0; i < 100; ++i) { + // A WeakReference of an object that no other objects reference are not immediately added to + // ReferenceQueue. To test this, we should run System.gc() multiple times. + // If a reader is GCed without closing, `cleanQueue` throws a RuntimeException. + boundedSource.cleanQueue(); + } + } + /** Generate byte array of given size. */ private static byte[] generateInput(int size) { // Arbitrary but fixed seed @@ -298,6 +340,7 @@ private static void writeFile(File file, byte[] input) throws IOException { /** Unsplittable source for use in tests. */ private static class UnsplittableSource extends FileBasedSource { + public UnsplittableSource(String fileOrPatternSpec, long minBundleSize) { super(StaticValueProvider.of(fileOrPatternSpec), minBundleSize); } @@ -323,6 +366,7 @@ public Coder getOutputCoder() { } private static class UnsplittableReader extends FileBasedReader { + ByteBuffer buff = ByteBuffer.allocate(1); Byte current; long offset; @@ -370,4 +414,140 @@ protected long getCurrentOffset() { } } } + + /** + * An integer generating bounded source. This source class checks if readers are closed properly. + * For that, it manages weak references of readers, and checks at `createReader` and `cleanQueue` + * if readers were closed before GCed. The `cleanQueue` does not change the state in + * `ManagedReaderBoundedSource`, but throws an exception if it finds a reader GCed without + * closing. + */ + private static class ManagedReaderBoundedSource extends BoundedSource { + + private final int from; + private final int to; // exclusive + + private transient ReferenceQueue refQueue; + private transient Map, CloseStatus> cloesStatusMap; + + public ManagedReaderBoundedSource(int from, int to) { + if (from > to) { + throw new RuntimeException( + String.format("`from` <= `to`, but got from: %d, to: %d", from, to)); + } + this.from = from; + this.to = to; + } + + @Override + public List> split( + long desiredBundleSizeBytes, PipelineOptions options) { + return Collections.singletonList(this); + } + + @Override + public long getEstimatedSizeBytes(PipelineOptions options) { + return (to - from) * 4L; + } + + @Override + public BoundedReader createReader(PipelineOptions options) { + // Add weak reference to queue to monitor GCed readers. If `CloseStatus` associated with + // reader is not closed, it means a reader was GCed without closing properly. The CloseStatus + // check for GCed readers are done at cleanQueue(). + if (refQueue == null) { + refQueue = new ReferenceQueue<>(); + cloesStatusMap = new HashMap<>(); + } + cleanQueue(); + + CloseStatus status = new CloseStatus(); + ManagedReader reader = new ManagedReader(status); + WeakReference reference = new WeakReference<>(reader, refQueue); + cloesStatusMap.put(reference, status); + LOG.info("Add reference {} for reader {}", reference, reader); + return reader; + } + + public void cleanQueue() { + System.gc(); + + Reference reference; + while ((reference = refQueue.poll()) != null) { + CloseStatus closeStatus = cloesStatusMap.get(reference); + LOG.info("Poll reference: {}, closed: {}", reference, closeStatus.closed); + closeStatus.throwIfNotClosed(); + } + } + + class CloseStatus { + + private final RuntimeException allocationStacktrace; + + private boolean closed; + + public CloseStatus() { + allocationStacktrace = + new RuntimeException("Previous reader was not closed properly. Reader allocation was"); + closed = false; + } + + void close() { + cleanQueue(); + closed = true; + } + + void throwIfNotClosed() { + if (!closed) { + throw allocationStacktrace; + } + } + } + + class ManagedReader extends BoundedReader { + + private final CloseStatus status; + + int current; + + public ManagedReader(CloseStatus status) { + this.status = status; + } + + @Override + public boolean start() { + if (from < to) { + current = from; + return true; + } else { + return false; + } + } + + @Override + public boolean advance() { + if (current + 1 < to) { + ++current; + return true; + } else { + return false; + } + } + + @Override + public Integer getCurrent() { + return current; + } + + @Override + public void close() { + status.close(); + } + + @Override + public BoundedSource getCurrentSource() { + return ManagedReaderBoundedSource.this; + } + } + } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSources.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSources.java index 872dc1e89a79..a9050236efc8 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSources.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSources.java @@ -776,6 +776,9 @@ public double getRemainingParallelism() { private static class UnboundedReaderIterator extends NativeReader.NativeReaderIterator>> { + // Do not close reader. The reader is cached in StreamingModeExecutionContext.readerCache, and + // will be reused until the cache is evicted, expired or invalidated. + // See UnboundedReader#iterator(). private final UnboundedSource.UnboundedReader reader; private final StreamingModeExecutionContext context; private final boolean started; @@ -862,7 +865,9 @@ public WindowedValue> getCurrent() throws NoSuchElementExce } @Override - public void close() {} + public void close() { + // Don't close reader. + } @Override public NativeReader.Progress getProgress() { From bd46c76dd086f2546006460d280cbbe7ae19d3ee Mon Sep 17 00:00:00 2001 From: Bruno Volpato Date: Wed, 11 Oct 2023 15:23:46 -0400 Subject: [PATCH 108/435] Bump snappy-java version to 1.1.10.4 (#28655) --- .../main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index 705bcb978e12..19ed6aaa4276 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -860,7 +860,7 @@ class BeamModulePlugin implements Plugin { slf4j_jul_to_slf4j : "org.slf4j:jul-to-slf4j:$slf4j_version", slf4j_log4j12 : "org.slf4j:slf4j-log4j12:$slf4j_version", slf4j_jcl : "org.slf4j:slf4j-jcl:$slf4j_version", - snappy_java : "org.xerial.snappy:snappy-java:1.1.10.3", + snappy_java : "org.xerial.snappy:snappy-java:1.1.10.4", spark_core : "org.apache.spark:spark-core_2.11:$spark2_version", spark_streaming : "org.apache.spark:spark-streaming_2.11:$spark2_version", spark3_core : "org.apache.spark:spark-core_2.12:$spark3_version", From ebad3deb5a0a497b0a4a46ca38f896ad3bd7b09b Mon Sep 17 00:00:00 2001 From: Aleksandr Dudko <116064902+aleksandr-dudko@users.noreply.github.com> Date: Wed, 11 Oct 2023 23:26:28 +0400 Subject: [PATCH 109/435] Add GitHub Workflow Replacements for Jenkins job_LoadTests_Python_CoGBK subgroup (#28764) --- ..._LoadTests_Python_CoGBK_Dataflow_Batch.yml | 127 ++++++++++++++++ ...dTests_Python_CoGBK_Dataflow_Streaming.yml | 126 ++++++++++++++++ ...eam_LoadTests_Python_CoGBK_Flink_Batch.yml | 135 ++++++++++++++++++ ...oGBK_Dataflow_Batch_100b_Multiple_Keys.txt | 28 ++++ ...n_CoGBK_Dataflow_Batch_100b_Single_Key.txt | 28 ++++ .../python_CoGBK_Dataflow_Batch_10kB.txt | 28 ++++ .../python_CoGBK_Dataflow_Batch_2MB.txt | 28 ++++ ...ataflow_Flink_Batch_100b_Multiple_Keys.txt | 28 ++++ ...K_Dataflow_Flink_Batch_100b_Single_Key.txt | 28 ++++ ...python_CoGBK_Dataflow_Flink_Batch_10kB.txt | 28 ++++ ..._Dataflow_Streaming_100b_Multiple_Keys.txt | 30 ++++ ...GBK_Dataflow_Streaming_100b_Single_Key.txt | 30 ++++ .../python_CoGBK_Dataflow_Streaming_10kB.txt | 30 ++++ .../python_CoGBK_Dataflow_Streaming_2MB.txt | 30 ++++ 14 files changed, 704 insertions(+) create mode 100644 .github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml create mode 100644 .github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml create mode 100644 .github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml create mode 100644 .github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Batch_100b_Multiple_Keys.txt create mode 100644 .github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Batch_100b_Single_Key.txt create mode 100644 .github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Batch_10kB.txt create mode 100644 .github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Batch_2MB.txt create mode 100644 .github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Flink_Batch_100b_Multiple_Keys.txt create mode 100644 .github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Flink_Batch_100b_Single_Key.txt create mode 100644 .github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Flink_Batch_10kB.txt create mode 100644 .github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Streaming_100b_Multiple_Keys.txt create mode 100644 .github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Streaming_100b_Single_Key.txt create mode 100644 .github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Streaming_10kB.txt create mode 100644 .github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Streaming_2MB.txt diff --git a/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml new file mode 100644 index 000000000000..284ba2420654 --- /dev/null +++ b/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml @@ -0,0 +1,127 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: LoadTests Python CoGBK Dataflow Batch + +on: + issue_comment: + types: [created] + schedule: + - cron: '30 11 * * *' + workflow_dispatch: + +# Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_LoadTests_Python_CoGBK_Dataflow_Batch: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Load Tests Python CoGBK Dataflow Batch' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 720 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_LoadTests_Python_CoGBK_Dataflow_Batch"] + job_phrase: ["Run Load Tests Python CoGBK Dataflow Batch"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action + with: + python-version: 3.8 + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: python + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Batch_100b_Single_Key.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Batch_100b_Multiple_Keys.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Batch_10kB.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Batch_2MB.txt + - name: Set current datetime + id: datetime + run: | + echo "datetime=$(date '+%m%d%H%M%S' --utc)" >> $GITHUB_OUTPUT + # The env variables are created and populated in the test-arguments-action as "_test_arguments_" + - name: run CoGBK 2GB of 100B records with a single key + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + --info \ + -PloadTest.mainClass=apache_beam.testing.load_tests.co_group_by_key_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_CoGBK_Dataflow_Batch_test_arguments_1 }} --job_name=load-tests-python-dataflow-batch-cogbk-1-${{ steps.datetime.outputs.datetime }}' \ + - name: run CoGBK 2GB of 100B records with multiple keys + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.co_group_by_key_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_CoGBK_Dataflow_Batch_test_arguments_2 }} --job_name=load-tests-python-dataflow-batch-cogbk-2-${{ steps.datetime.outputs.datetime }}' \ + - name: run CoGBK reiterate 4 times 10kB values + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.co_group_by_key_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_CoGBK_Dataflow_Batch_test_arguments_3 }} --job_name=load-tests-python-dataflow-batch-cogbk-3-${{ steps.datetime.outputs.datetime }}' \ + - name: run CoGBK reiterate 4 times 2MB values + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.co_group_by_key_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_CoGBK_Dataflow_Batch_test_arguments_4 }} --job_name=load-tests-python-dataflow-batch-cogbk-4-${{ steps.datetime.outputs.datetime }}' \ No newline at end of file diff --git a/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml new file mode 100644 index 000000000000..313ba94f9df3 --- /dev/null +++ b/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml @@ -0,0 +1,126 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: LoadTests Python CoGBK Dataflow Streaming + +on: + issue_comment: + types: [created] + schedule: + - cron: '30 11 * * *' + workflow_dispatch: + +# Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_LoadTests_Python_CoGBK_Dataflow_Streaming: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Load Tests Python CoGBK Dataflow Streaming' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 720 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_LoadTests_Python_CoGBK_Dataflow_Streaming"] + job_phrase: ["Run Load Tests Python CoGBK Dataflow Streaming"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action + with: + python-version: 3.8 + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: python + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Streaming_100b_Single_Key.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Streaming_100b_Multiple_Keys.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Streaming_10kB.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Streaming_2MB.txt + - name: Set current datetime + id: datetime + run: | + echo "datetime=$(date '+%m%d%H%M%S' --utc)" >> $GITHUB_OUTPUT + # The env variables are created and populated in the test-arguments-action as "_test_arguments_" + - name: run CoGBK 2GB of 100B records with a single key + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.co_group_by_key_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_CoGBK_Dataflow_Streaming_test_arguments_1 }} --job_name=load-tests-python-dataflow-streaming-cogbk-1-${{ steps.datetime.outputs.datetime }}' \ + - name: run CoGBK 2GB of 100B records with multiple keys + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.co_group_by_key_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_CoGBK_Dataflow_Streaming_test_arguments_2 }} --job_name=load-tests-python-dataflow-streaming-cogbk-2-${{ steps.datetime.outputs.datetime }}' \ + - name: run CoGBK reiterate 4 times 10kB values + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.co_group_by_key_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_CoGBK_Dataflow_Streaming_test_arguments_3 }} --job_name=load-tests-python-dataflow-streaming-cogbk-3-${{ steps.datetime.outputs.datetime }}' \ + - name: run CoGBK reiterate 4 times 2MB values + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.co_group_by_key_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_CoGBK_Dataflow_Streaming_test_arguments_4 }} --job_name=load-tests-python-dataflow-streaming-cogbk-4-${{ steps.datetime.outputs.datetime }}' \ No newline at end of file diff --git a/.github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml new file mode 100644 index 000000000000..6d668c1b2566 --- /dev/null +++ b/.github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml @@ -0,0 +1,135 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: LoadTests Python CoGBK Dataflow Flink Batch + +on: + issue_comment: + types: [created] + schedule: + - cron: '40 12 * * *' + workflow_dispatch: + +# Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + GCLOUD_ZONE: us-central1-a + CLUSTER_NAME: beam-loadtests-python-cogbk-flink-batch-${{ github.run_id }} + GCS_BUCKET: gs://beam-flink-cluster + FLINK_DOWNLOAD_URL: https://archive.apache.org/dist/flink/flink-1.15.0/flink-1.15.0-bin-scala_2.12.tgz + HADOOP_DOWNLOAD_URL: https://repo.maven.apache.org/maven2/org/apache/flink/flink-shaded-hadoop-2-uber/2.8.3-10.0/flink-shaded-hadoop-2-uber-2.8.3-10.0.jar + FLINK_TASKMANAGER_SLOTS: 1 + DETACHED_MODE: true + HARNESS_IMAGES_TO_PULL: gcr.io/apache-beam-testing/beam-sdk/beam_go_sdk:latest + JOB_SERVER_IMAGE: gcr.io/apache-beam-testing/beam_portability/beam_flink1.15_job_server:latest + ARTIFACTS_DIR: gs://beam-flink-cluster/beam-loadtests-python-cogbk-flink-batch-${{ github.run_id }} + +jobs: + beam_LoadTests_Python_CoGBK_Flink_Batch: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Load Tests Python CoGBK Flink Batch' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 720 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_LoadTests_Python_CoGBK_Flink_Batch"] + job_phrase: ["Run Load Tests Python CoGBK Flink Batch"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Install Python + uses: actions/setup-python@v4 + with: + python-version: '3.8' + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: python + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Flink_Batch_100b_Single_Key.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Flink_Batch_100b_Multiple_Keys.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Flink_Batch_10kB.txt + - name: Start Flink with parallelism 5 + env: + FLINK_NUM_WORKERS: 5 + run: | + cd ${{ github.workspace }}/.test-infra/dataproc; ./flink_cluster.sh create + - name: Set current datetime + id: datetime + run: | + echo "datetime=$(date '+%m%d%H%M%S' --utc)" >> $GITHUB_OUTPUT + # The env variables are created and populated in the test-arguments-action as "_test_arguments_" + - name: run CoGBK 2GB of 100B records with a single key + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + --info \ + -PloadTest.mainClass=apache_beam.testing.load_tests.co_group_by_key_test \ + -Prunner=FlinkRunner \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_CoGBK_Flink_Batch_test_arguments_1 }} --job_name=load-tests-python-flink-batch-cogbk-1-${{ steps.datetime.outputs.datetime }}' \ + - name: run CoGBK 2GB of 100B records with multiple keys + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + --info \ + -PloadTest.mainClass=apache_beam.testing.load_tests.co_group_by_key_test \ + -Prunner=FlinkRunner \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_CoGBK_Flink_Batch_test_arguments_2 }} --job_name=load-tests-python-flink-batch-cogbk-2-${{ steps.datetime.outputs.datetime }}' \ + - name: run CoGBK reiterate 4 times 10kB values + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + --info \ + -PloadTest.mainClass=apache_beam.testing.load_tests.co_group_by_key_test \ + -Prunner=FlinkRunner \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_CoGBK_Flink_Batch_test_arguments_2 }} --job_name=load-tests-python-flink-batch-cogbk-3-${{ steps.datetime.outputs.datetime }}' \ + - name: Teardown Flink + if: always() + run: | + ${{ github.workspace }}/.test-infra/dataproc/flink_cluster.sh delete \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Batch_100b_Multiple_Keys.txt b/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Batch_100b_Multiple_Keys.txt new file mode 100644 index 000000000000..d5ba43180738 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Batch_100b_Multiple_Keys.txt @@ -0,0 +1,28 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_batch_cogbk_2 +--influx_measurement=python_batch_cogbk_2 +--input_options=''{\\"num_records\\":20000000,\\"key_size\\":10,\\"value_size\\":90,\\"num_hot_keys\\":5,\\"hot_key_fraction\\":1,\\"algorithm\\":\\"lcg\\"}'' +--co_input_options=''{\\"num_records\\":2000000,\\"key_size\\":10,\\"value_size\\":90,\\"num_hot_keys\\":1000,\\"hot_key_fraction\\":1,\\"algorithm\\":\\"lcg\\"}'' +--iterations=1 +--num_workers=5 +--autoscaling_algorithm=NONE +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Batch_100b_Single_Key.txt b/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Batch_100b_Single_Key.txt new file mode 100644 index 000000000000..47ebf22dc835 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Batch_100b_Single_Key.txt @@ -0,0 +1,28 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_batch_cogbk_1 +--influx_measurement=python_batch_cogbk_1 +--input_options=''{\\"num_records\\":20000000,\\"key_size\\":10,\\"value_size\\":90,\\"num_hot_keys\\":1,\\"hot_key_fraction\\":1,\\"algorithm\\":\\"lcg\\"}'' +--co_input_options=''{\\"num_records\\":2000000,\\"key_size\\":10,\\"value_size\\":90,\\"num_hot_keys\\":1000,\\"hot_key_fraction\\":1,\\"algorithm\\":\\"lcg\\"}'' +--iterations=1 +--num_workers=5 +--autoscaling_algorithm=NONE +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Batch_10kB.txt b/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Batch_10kB.txt new file mode 100644 index 000000000000..13161125b570 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Batch_10kB.txt @@ -0,0 +1,28 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_batch_cogbk_3 +--influx_measurement=python_batch_cogbk_3 +--input_options=''{\\"num_records\\":20000000,\\"key_size\\":10,\\"value_size\\":90,\\"num_hot_keys\\":200000,\\"hot_key_fraction\\":1,\\"algorithm\\":\\"lcg\\"}'' +--co_input_options=''{\\"num_records\\":2000000,\\"key_size\\":10,\\"value_size\\":90,\\"num_hot_keys\\":1000,\\"hot_key_fraction\\":1,\\"algorithm\\":\\"lcg\\"}'' +--iterations=4 +--num_workers=5 +--autoscaling_algorithm=NONE +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Batch_2MB.txt b/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Batch_2MB.txt new file mode 100644 index 000000000000..052c2464a1cc --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Batch_2MB.txt @@ -0,0 +1,28 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_batch_cogbk_4 +--influx_measurement=python_batch_cogbk_4 +--input_options=''{\\"num_records\\":20000000,\\"key_size\\":10,\\"value_size\\":90,\\"num_hot_keys\\":1000,\\"hot_key_fraction\\":1,\\"algorithm\\":\\"lcg\\"}'' +--co_input_options=''{\\"num_records\\":2000000,\\"key_size\\":10,\\"value_size\\":90,\\"num_hot_keys\\":1000,\\"hot_key_fraction\\":1,\\"algorithm\\":\\"lcg\\"}'' +--iterations=4 +--num_workers=5 +--autoscaling_algorithm=NONE +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Flink_Batch_100b_Multiple_Keys.txt b/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Flink_Batch_100b_Multiple_Keys.txt new file mode 100644 index 000000000000..4b8a2f72010b --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Flink_Batch_100b_Multiple_Keys.txt @@ -0,0 +1,28 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_flink_batch_cogbk_2 +--influx_measurement=python_batch_cogbk_2 +--input_options=''{\\"num_records\\":20000000,\\"key_size\\":10,\\"value_size\\":90,\\"num_hot_keys\\":5,\\"hot_key_fraction\\":1}'' +--co_input_options=''{\\"num_records\\":2000000,\\"key_size\\":10,\\"value_size\\":90,\\"num_hot_keys\\":5,\\"hot_key_fraction\\":1}'' +--iterations=1 +--parallelism=5 +--endpoint=localhost:8099 +--environment_type=DOCKER +--environment_config=gcr.io/apache-beam-testing/beam-sdk/beam_go_sdk:latest \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Flink_Batch_100b_Single_Key.txt b/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Flink_Batch_100b_Single_Key.txt new file mode 100644 index 000000000000..3aeb927f04ee --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Flink_Batch_100b_Single_Key.txt @@ -0,0 +1,28 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_flink_batch_cogbk_1 +--influx_measurement=python_batch_cogbk_1 +--input_options=''{\\"num_records\\":20000000,\\"key_size\\":10,\\"value_size\\":90,\\"num_hot_keys\\":1,\\"hot_key_fraction\\":1}'' +--co_input_options=''{\\"num_records\\":2000000,\\"key_size\\":10,\\"value_size\\":90,\\"num_hot_keys\\":1000,\\"hot_key_fraction\\":1}'' +--iterations=1 +--parallelism=5 +--endpoint=localhost:8099 +--environment_type=DOCKER +--environment_config=gcr.io/apache-beam-testing/beam-sdk/beam_go_sdk:latest \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Flink_Batch_10kB.txt b/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Flink_Batch_10kB.txt new file mode 100644 index 000000000000..e350e2d29944 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Flink_Batch_10kB.txt @@ -0,0 +1,28 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_flink_batch_cogbk_3 +--influx_measurement=python_batch_cogbk_3 +--input_options=''{\\"num_records\\":20000000,\\"key_size\\":10,\\"value_size\\":90,\\"num_hot_keys\\":200000,\\"hot_key_fraction\\":1}'' +--co_input_options=''{\\"num_records\\":2000000,\\"key_size\\":10,\\"value_size\\":90,\\"num_hot_keys\\":1000,\\"hot_key_fraction\\":1}'' +--iterations=4 +--parallelism=5 +--endpoint=localhost:8099 +--environment_type=DOCKER +--environment_config=gcr.io/apache-beam-testing/beam-sdk/beam_go_sdk:latest \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Streaming_100b_Multiple_Keys.txt b/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Streaming_100b_Multiple_Keys.txt new file mode 100644 index 000000000000..a687f0cf5de7 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Streaming_100b_Multiple_Keys.txt @@ -0,0 +1,30 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_streaming_cogbk_2 +--influx_measurement=python_streaming_cogbk_2 +--input_options=''{\\"num_records\\":20000000,\\"key_size\\":10,\\"value_size\\":90,\\"num_hot_keys\\":5,\\"hot_key_fraction\\":1,\\"algorithm\\":\\"lcg\\"}'' +--co_input_options=''{\\"num_records\\":2000000,\\"key_size\\":10,\\"value_size\\":90,\\"num_hot_keys\\":1000,\\"hot_key_fraction\\":1,\\"algorithm\\":\\"lcg\\"}'' +--iterations=1 +--num_workers=5 +--autoscaling_algorithm=NONE +--streaming +--worker_machine_type=n1-highmem-4 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Streaming_100b_Single_Key.txt b/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Streaming_100b_Single_Key.txt new file mode 100644 index 000000000000..9141182b90fc --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Streaming_100b_Single_Key.txt @@ -0,0 +1,30 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_streaming_cogbk_1 +--influx_measurement=python_streaming_cogbk_1 +--input_options=''{\\"num_records\\":20000000,\\"key_size\\":10,\\"value_size\\":90,\\"num_hot_keys\\":1,\\"hot_key_fraction\\":1,\\"algorithm\\":\\"lcg\\"}'' +--co_input_options=''{\\"num_records\\":2000000,\\"key_size\\":10,\\"value_size\\":90,\\"num_hot_keys\\":1000,\\"hot_key_fraction\\":1,\\"algorithm\\":\\"lcg\\"}'' +--iterations=1 +--num_workers=5 +--autoscaling_algorithm=NONE +--streaming +--worker_machine_type=n1-highmem-4 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Streaming_10kB.txt b/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Streaming_10kB.txt new file mode 100644 index 000000000000..7250f073f25e --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Streaming_10kB.txt @@ -0,0 +1,30 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_streaming_cogbk_3 +--influx_measurement=python_streaming_cogbk_3 +--input_options=''{\\"num_records\\":20000000,\\"key_size\\":10,\\"value_size\\":90,\\"num_hot_keys\\":200000,\\"hot_key_fraction\\":1,\\"algorithm\\":\\"lcg\\"}'' +--co_input_options=''{\\"num_records\\":2000000,\\"key_size\\":10,\\"value_size\\":90,\\"num_hot_keys\\":1000,\\"hot_key_fraction\\":1,\\"algorithm\\":\\"lcg\\"}'' +--iterations=4 +--num_workers=5 +--autoscaling_algorithm=NONE +--streaming +--worker_machine_type=n1-highmem-4 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Streaming_2MB.txt b/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Streaming_2MB.txt new file mode 100644 index 000000000000..59723107d53c --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Streaming_2MB.txt @@ -0,0 +1,30 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_streaming_cogbk_4 +--influx_measurement=python_streaming_cogbk_4 +--input_options=''{\\"num_records\\":20000000,\\"key_size\\":10,\\"value_size\\":90,\\"num_hot_keys\\":1000,\\"hot_key_fraction\\":1,\\"algorithm\\":\\"lcg\\"}'' +--co_input_options=''{\\"num_records\\":2000000,\\"key_size\\":10,\\"value_size\\":90,\\"num_hot_keys\\":1000,\\"hot_key_fraction\\":1,\\"algorithm\\":\\"lcg\\"}'' +--iterations=4 +--num_workers=5 +--autoscaling_algorithm=NONE +--streaming +--worker_machine_type=n1-highmem-4 +--runner=DataflowRunner \ No newline at end of file From dfdbcb9006b92dd7b896d09c050726ebfb44446c Mon Sep 17 00:00:00 2001 From: Aleksandr Dudko <116064902+aleksandr-dudko@users.noreply.github.com> Date: Wed, 11 Oct 2023 23:27:04 +0400 Subject: [PATCH 110/435] Add GitHub Workflow Replacements for Jenkins job_LoadTests_Java_CoGBK subgroup (#28742) --- ...am_LoadTests_Java_CoGBK_Dataflow_Batch.yml | 114 +++++++++++++++ ...a_CoGBK_Dataflow_V2_Batch_JavaVersions.yml | 132 ++++++++++++++++++ ...GBK_Dataflow_V2_Streaming_JavaVersions.yml | 132 ++++++++++++++++++ ...a_CoGBK_SparkStructuredStreaming_Batch.yml | 114 +++++++++++++++ ...oGBK_Dataflow_Batch_100b_Multiple_Keys.txt | 27 ++++ ...a_CoGBK_Dataflow_Batch_100b_Single_Key.txt | 27 ++++ .../java_CoGBK_Dataflow_Batch_10kB.txt | 27 ++++ .../java_CoGBK_Dataflow_Batch_2MB.txt | 27 ++++ ...w_V2_Streaming_Java_100b_Multiple_Keys.txt | 29 ++++ ...flow_V2_Streaming_Java_100b_Single_Key.txt | 29 ++++ ..._CoGBK_Dataflow_V2_Streaming_Java_10kB.txt | 29 ++++ ...a_CoGBK_Dataflow_V2_Streaming_Java_2MB.txt | 29 ++++ ...uredStreaming_Batch_100b_Multiple_Keys.txt | 25 ++++ ...ucturedStreaming_Batch_100b_Single_Key.txt | 25 ++++ ...BK_SparkStructuredStreaming_Batch_10kB.txt | 25 ++++ ...GBK_SparkStructuredStreaming_Batch_2MB.txt | 25 ++++ 16 files changed, 816 insertions(+) create mode 100644 .github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml create mode 100644 .github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml create mode 100644 .github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml create mode 100644 .github/workflows/beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml create mode 100644 .github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_Batch_100b_Multiple_Keys.txt create mode 100644 .github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_Batch_100b_Single_Key.txt create mode 100644 .github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_Batch_10kB.txt create mode 100644 .github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_Batch_2MB.txt create mode 100644 .github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_V2_Streaming_Java_100b_Multiple_Keys.txt create mode 100644 .github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_V2_Streaming_Java_100b_Single_Key.txt create mode 100644 .github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_V2_Streaming_Java_10kB.txt create mode 100644 .github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_V2_Streaming_Java_2MB.txt create mode 100644 .github/workflows/load-tests-job-configs/java_CoGBK_SparkStructuredStreaming_Batch_100b_Multiple_Keys.txt create mode 100644 .github/workflows/load-tests-job-configs/java_CoGBK_SparkStructuredStreaming_Batch_100b_Single_Key.txt create mode 100644 .github/workflows/load-tests-job-configs/java_CoGBK_SparkStructuredStreaming_Batch_10kB.txt create mode 100644 .github/workflows/load-tests-job-configs/java_CoGBK_SparkStructuredStreaming_Batch_2MB.txt diff --git a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml new file mode 100644 index 000000000000..25feaaf288a1 --- /dev/null +++ b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml @@ -0,0 +1,114 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: LoadTests Java CoGBK Dataflow Batch + +on: + issue_comment: + types: [created] + schedule: + - cron: '30 9 * * *' + workflow_dispatch: + +# Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_LoadTests_Java_CoGBK_Dataflow_Batch: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Load Tests Java CoGBK Dataflow Batch' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 720 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_LoadTests_Java_CoGBK_Dataflow_Batch"] + job_phrase: ["Run Load Tests Java CoGBK Dataflow Batch"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: java + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_Batch_100b_Single_Key.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_Batch_100b_Multiple_Keys.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_Batch_10kB.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_Batch_2MB.txt + # The env variables are created and populated in the test-arguments-action as "_test_arguments_" + - name: run CoGBK 2GB 100 byte records - single key + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_CoGBK_Dataflow_Batch_test_arguments_1 }} --appName=load_tests_Java_Dataflow_batch_CoGBK_1' \ + - name: run CoGBK 2GB 100 byte records - multiple keys + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_CoGBK_Dataflow_Batch_test_arguments_2 }} --appName=load_tests_Java_Dataflow_batch_CoGBK_2' \ + - name: run CoGBK 2GB reiteration 10kB value + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_CoGBK_Dataflow_Batch_test_arguments_3 }} --appName=load_tests_Java_Dataflow_batch_CoGBK_3' \ + - name: run CoGBK 2GB reiteration 2MB value + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_CoGBK_Dataflow_Batch_test_arguments_4 }} --appName=load_tests_Java_Dataflow_batch_CoGBK_4' \ No newline at end of file diff --git a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml new file mode 100644 index 000000000000..35575aec0bef --- /dev/null +++ b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml @@ -0,0 +1,132 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: LoadTests Java CoGBK Dataflow V2 Batch JavaVersions + +on: + issue_comment: + types: [created] + schedule: + - cron: '0 10 * * *' + workflow_dispatch: + +# Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + (contains(github.event.comment.body, 'Run Load Tests Java') && + contains(github.event.comment.body, 'CoGBK Dataflow V2 Batch')) + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 720 + name: ${{matrix.job_name}} (${{matrix.job_phrase_1}} ${{matrix.java_version}} ${{matrix.job_phrase_2}}) + strategy: + fail-fast: false + matrix: + job_name: ["beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions"] + job_phrase_1: ["Run Load Tests Java"] + job_phrase_2: ["CoGBK Dataflow V2 Batch"] + java_version: ['11','17'] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase_1 }} ${{matrix.java_version}} ${{ matrix.job_phrase_2 }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} ${{ matrix.job_phrase_1 }} ${{matrix.java_version}} ${{ matrix.job_phrase_2 }} + - name: Setup environment + uses: ./.github/actions/setup-environment-action + with: + java-version: ${{ matrix.java_version }} + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: java + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_Batch_100b_Single_Key.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_Batch_100b_Multiple_Keys.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_Batch_10kB.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_Batch_2MB.txt + arguments: | + --influxTags={\"runnerVersion\":\"v2\",\"jdk\":\"java${{ matrix.java_version }}\"} + # The env variables are created and populated in the test-arguments-action as "_test_arguments_" + - name: run CoGBK 2GB 100 byte records - single key + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions_test_arguments_1 }} --appName=load_tests_Java${{ matrix.java_version }}_Dataflow_V2_batch_CoGBK_1' \ + - name: run CoGBK 2GB 100 byte records - multiple keys + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions_test_arguments_2 }} --appName=load_tests_Java${{ matrix.java_version }}_Dataflow_V2_batch_CoGBK_2' \ + - name: run CoGBK 2GB reiteration 10kB value + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions_test_arguments_3 }} --appName=load_tests_Java${{ matrix.java_version }}_Dataflow_V2_batch_CoGBK_3' \ + - name: run CoGBK 2GB reiteration 2MB value + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions_test_arguments_4 }} --appName=load_tests_Java${{ matrix.java_version }}_Dataflow_V2_batch_CoGBK_4' \ No newline at end of file diff --git a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml new file mode 100644 index 000000000000..46dfaa986920 --- /dev/null +++ b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml @@ -0,0 +1,132 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: LoadTests Java CoGBK Dataflow V2 Streaming JavaVersions + +on: + issue_comment: + types: [created] + schedule: + - cron: '30 10 * * *' + workflow_dispatch: + +# Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + (contains(github.event.comment.body, 'Run Load Tests Java') && + contains(github.event.comment.body, 'CoGBK Dataflow V2 Streaming')) + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 240 + name: ${{matrix.job_name}} (${{matrix.job_phrase_1}} ${{matrix.java_version}} ${{matrix.job_phrase_2}}) + strategy: + fail-fast: false + matrix: + job_name: ["beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions"] + job_phrase_1: ["Run Load Tests Java"] + job_phrase_2: ["CoGBK Dataflow V2 Streaming"] + java_version: ['11','17'] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase_1 }} ${{matrix.java_version}} ${{ matrix.job_phrase_2 }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} ${{ matrix.job_phrase_1 }} ${{matrix.java_version}} ${{ matrix.job_phrase_2 }} + - name: Setup environment + uses: ./.github/actions/setup-environment-action + with: + java-version: ${{ matrix.java_version }} + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: java + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_V2_Streaming_Java_100b_Single_Key.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_V2_Streaming_Java_100b_Multiple_Keys.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_V2_Streaming_Java_10kB.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_V2_Streaming_Java_2MB.txt + arguments: | + --influxTags={\"runnerVersion\":\"v2\",\"jdk\":\"java${{ matrix.java_version }}\"} + # The env variables are created and populated in the test-arguments-action as "_test_arguments_" + - name: run CoGBK 2GB 100 byte records - single key + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions_test_arguments_1 }} --appName=load_tests_Java${{ matrix.java_version }}_Dataflow_V2_streaming_CoGBK_1' \ + - name: run CoGBK 2GB 100 byte records - multiple keys + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions_test_arguments_2 }} --appName=load_tests_Java${{ matrix.java_version }}_Dataflow_V2_streaming_CoGBK_2' \ + - name: run CoGBK 2GB reiteration 10kB value + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions_test_arguments_3 }} --appName=load_tests_Java${{ matrix.java_version }}_Dataflow_V2_streaming_CoGBK_3' \ + - name: run CoGBK 2GB reiteration 2MB value + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions_test_arguments_4 }} --appName=load_tests_Java${{ matrix.java_version }}_Dataflow_V2_streaming_CoGBK_4' \ No newline at end of file diff --git a/.github/workflows/beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml b/.github/workflows/beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml new file mode 100644 index 000000000000..3d18345589ed --- /dev/null +++ b/.github/workflows/beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml @@ -0,0 +1,114 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: LoadTests Java CoGBK SparkStructuredStreaming Batch + +on: + issue_comment: + types: [created] + schedule: + - cron: '0 11 * * *' + workflow_dispatch: + +# Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Load Tests Java CoGBK SparkStructuredStreaming Batch' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 720 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch"] + job_phrase: ["Run Load Tests Java CoGBK SparkStructuredStreaming Batch"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: java + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_CoGBK_SparkStructuredStreaming_Batch_100b_Single_Key.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_CoGBK_SparkStructuredStreaming_Batch_100b_Multiple_Keys.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_CoGBK_SparkStructuredStreaming_Batch_10kB.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_CoGBK_SparkStructuredStreaming_Batch_2MB.txt + # The env variables are created and populated in the test-arguments-action as "_test_arguments_" + - name: run CoGBK 2GB 100 byte records - single key + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ + -Prunner=:runners:spark:3 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch_test_arguments_1 }}' \ + - name: run CoGBK 2GB 100 byte records - multiple keys + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ + -Prunner=:runners:spark:3 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch_test_arguments_2 }}' \ + - name: run CoGBK 2GB reiteration 10kB value + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ + -Prunner=:runners:spark:3 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch_test_arguments_3 }}' \ + - name: run CoGBK 2GB reiteration 2MB value + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ + -Prunner=:runners:spark:3 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch_test_arguments_4 }}' \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_Batch_100b_Multiple_Keys.txt b/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_Batch_100b_Multiple_Keys.txt new file mode 100644 index 000000000000..6402c43d1567 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_Batch_100b_Multiple_Keys.txt @@ -0,0 +1,27 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_cogbk_2 +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":5} +--coSourceOptions={"numRecords":2000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":1000} +--iterations=1 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=false +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_Batch_100b_Single_Key.txt b/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_Batch_100b_Single_Key.txt new file mode 100644 index 000000000000..5d53c3b3d7ef --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_Batch_100b_Single_Key.txt @@ -0,0 +1,27 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_cogbk_1 +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":1} +--coSourceOptions={"numRecords":2000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":1000} +--iterations=1 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=false +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_Batch_10kB.txt b/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_Batch_10kB.txt new file mode 100644 index 000000000000..501fa6b3a57f --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_Batch_10kB.txt @@ -0,0 +1,27 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_cogbk_3 +--publishToInfluxDB=true +--sourceOptions={"numRecords":2000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":200000} +--coSourceOptions={"numRecords":2000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":1000} +--iterations=4 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=false +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_Batch_2MB.txt b/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_Batch_2MB.txt new file mode 100644 index 000000000000..0d5f57fc47b8 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_Batch_2MB.txt @@ -0,0 +1,27 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_cogbk_4 +--publishToInfluxDB=true +--sourceOptions={"numRecords":2000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":1000} +--coSourceOptions={"numRecords":2000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":1000} +--iterations=4 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=false +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_V2_Streaming_Java_100b_Multiple_Keys.txt b/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_V2_Streaming_Java_100b_Multiple_Keys.txt new file mode 100644 index 000000000000..24aff12bad79 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_V2_Streaming_Java_100b_Multiple_Keys.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_streaming_cogbk_2 +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":5} +--coSourceOptions={"numRecords":2000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":1000} +--iterations=1 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=true +--inputWindowDurationSec=1200 +--coInputWindowDurationSec=1200 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_V2_Streaming_Java_100b_Single_Key.txt b/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_V2_Streaming_Java_100b_Single_Key.txt new file mode 100644 index 000000000000..eead04aeb7e4 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_V2_Streaming_Java_100b_Single_Key.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_streaming_cogbk_1 +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":1} +--coSourceOptions={"numRecords":2000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":1000} +--iterations=1 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=true +--inputWindowDurationSec=1200 +--coInputWindowDurationSec=1200 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_V2_Streaming_Java_10kB.txt b/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_V2_Streaming_Java_10kB.txt new file mode 100644 index 000000000000..4b45c7df3f9e --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_V2_Streaming_Java_10kB.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_streaming_cogbk_3 +--publishToInfluxDB=true +--sourceOptions={"numRecords":2000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":200000} +--coSourceOptions={"numRecords":2000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":1000} +--iterations=4 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=true +--inputWindowDurationSec=1200 +--coInputWindowDurationSec=1200 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_V2_Streaming_Java_2MB.txt b/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_V2_Streaming_Java_2MB.txt new file mode 100644 index 000000000000..f522fd7ec36d --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_V2_Streaming_Java_2MB.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_streaming_cogbk_4 +--publishToInfluxDB=true +--sourceOptions={"numRecords":2000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":1000} +--coSourceOptions={"numRecords":2000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":1000} +--iterations=4 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=true +--inputWindowDurationSec=1200 +--coInputWindowDurationSec=1200 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_CoGBK_SparkStructuredStreaming_Batch_100b_Multiple_Keys.txt b/.github/workflows/load-tests-job-configs/java_CoGBK_SparkStructuredStreaming_Batch_100b_Multiple_Keys.txt new file mode 100644 index 000000000000..747f495b144d --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_CoGBK_SparkStructuredStreaming_Batch_100b_Multiple_Keys.txt @@ -0,0 +1,25 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--appName=load_tests_Java_SparkStructuredStreaming_batch_CoGBK_2 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_cogbk_2 +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":5} +--coSourceOptions={"numRecords":2000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":1000} +--iterations=1 +--streaming=false +--runner=SparkStructuredStreamingRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_CoGBK_SparkStructuredStreaming_Batch_100b_Single_Key.txt b/.github/workflows/load-tests-job-configs/java_CoGBK_SparkStructuredStreaming_Batch_100b_Single_Key.txt new file mode 100644 index 000000000000..b17549a702f3 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_CoGBK_SparkStructuredStreaming_Batch_100b_Single_Key.txt @@ -0,0 +1,25 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--appName=load_tests_Java_SparkStructuredStreaming_batch_CoGBK_1 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_cogbk_1 +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":1} +--coSourceOptions={"numRecords":2000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":1000} +--iterations=1 +--streaming=false +--runner=SparkStructuredStreamingRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_CoGBK_SparkStructuredStreaming_Batch_10kB.txt b/.github/workflows/load-tests-job-configs/java_CoGBK_SparkStructuredStreaming_Batch_10kB.txt new file mode 100644 index 000000000000..84f53ee120a2 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_CoGBK_SparkStructuredStreaming_Batch_10kB.txt @@ -0,0 +1,25 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--appName=load_tests_Java_SparkStructuredStreaming_batch_CoGBK_3 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_cogbk_3 +--publishToInfluxDB=true +--sourceOptions={"numRecords":2000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":200000} +--coSourceOptions={"numRecords":2000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":1000} +--iterations=4 +--streaming=false +--runner=SparkStructuredStreamingRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_CoGBK_SparkStructuredStreaming_Batch_2MB.txt b/.github/workflows/load-tests-job-configs/java_CoGBK_SparkStructuredStreaming_Batch_2MB.txt new file mode 100644 index 000000000000..8f8bdb899551 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_CoGBK_SparkStructuredStreaming_Batch_2MB.txt @@ -0,0 +1,25 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--appName=load_tests_Java_SparkStructuredStreaming_batch_CoGBK_4 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_cogbk_4 +--publishToInfluxDB=true +--sourceOptions={"numRecords":2000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":1000} +--coSourceOptions={"numRecords":2000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":1000} +--iterations=4 +--streaming=false +--runner=SparkStructuredStreamingRunner \ No newline at end of file From 09f30537015c2383402fcccfe688ddf428b1ea7e Mon Sep 17 00:00:00 2001 From: Aleksandr Dudko <116064902+aleksandr-dudko@users.noreply.github.com> Date: Wed, 11 Oct 2023 23:27:32 +0400 Subject: [PATCH 111/435] Add GitHub Workflow Replacements for Jenkins job_LoadTests_Python_SideInput_Dataflow_Batch (#28826) --- ...dTests_Python_SideInput_Dataflow_Batch.yml | 186 ++++++++++++++++++ ...w_Batch_10gb_1000window_first_iterable.txt | 30 +++ ...ataflow_Batch_10gb_1000window_iterable.txt | 29 +++ ...flow_Batch_10gb_1window_first_iterable.txt | 29 +++ ...t_Dataflow_Batch_10gb_1window_iterable.txt | 28 +++ ...Batch_1gb_1000window_1key_percent_dict.txt | 30 +++ ...atch_1gb_1000window_99key_percent_dict.txt | 30 +++ ...ow_Batch_1gb_1window_1key_percent_dict.txt | 29 +++ ...w_Batch_1gb_1window_99key_percent_dict.txt | 29 +++ ..._Dataflow_Batch_1gb_1window_first_list.txt | 29 +++ ...eInput_Dataflow_Batch_1gb_1window_list.txt | 28 +++ 11 files changed, 477 insertions(+) create mode 100644 .github/workflows/beam_LoadTests_Python_SideInput_Dataflow_Batch.yml create mode 100644 .github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_10gb_1000window_first_iterable.txt create mode 100644 .github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_10gb_1000window_iterable.txt create mode 100644 .github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_10gb_1window_first_iterable.txt create mode 100644 .github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_10gb_1window_iterable.txt create mode 100644 .github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1000window_1key_percent_dict.txt create mode 100644 .github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1000window_99key_percent_dict.txt create mode 100644 .github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1window_1key_percent_dict.txt create mode 100644 .github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1window_99key_percent_dict.txt create mode 100644 .github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1window_first_list.txt create mode 100644 .github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1window_list.txt diff --git a/.github/workflows/beam_LoadTests_Python_SideInput_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Python_SideInput_Dataflow_Batch.yml new file mode 100644 index 000000000000..eb58c85e92e8 --- /dev/null +++ b/.github/workflows/beam_LoadTests_Python_SideInput_Dataflow_Batch.yml @@ -0,0 +1,186 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: LoadTests Python SideInput Dataflow Batch + +on: + issue_comment: + types: [created] + schedule: + - cron: '30 13 * * *' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_LoadTests_Python_SideInput_Dataflow_Batch: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Load Tests Python SideInput Dataflow Batch' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 720 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_LoadTests_Python_SideInput_Dataflow_Batch"] + job_phrase: ["Run Load Tests Python SideInput Dataflow Batch"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action + with: + python-version: 3.8 + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: python + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1window_1key_percent_dict.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1window_99key_percent_dict.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_10gb_1window_first_iterable.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_10gb_1window_iterable.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1window_first_list.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1window_list.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1000window_1key_percent_dict.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1000window_99key_percent_dict.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_10gb_1000window_first_iterable.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_10gb_1000window_iterable.txt + - name: Set current datetime + id: datetime + run: | + echo "datetime=$(date '+%m%d%H%M%S' --utc)" >> $GITHUB_OUTPUT + # The env variables are created and populated in the test-arguments-action as "_test_arguments_" + - name: run SideInput Dataflow Batch Python Load Test 1 (1gb-1kb-10workers-1window-1key-percent-dict) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.sideinput_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_SideInput_Dataflow_Batch_test_arguments_1 }} --job_name=load-tests-python-dataflow-batch-sideinput-1-${{ steps.datetime.outputs.datetime }}' \ + - name: run SideInput Dataflow Batch Python Load Test 2 (1gb-1kb-10workers-1window-99key-percent-dict) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.sideinput_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_SideInput_Dataflow_Batch_test_arguments_2 }} --job_name=load-tests-python-dataflow-batch-sideinput-2-${{ steps.datetime.outputs.datetime }}' \ + - name: run SideInput Dataflow Batch Python Load Test 3 (10gb-1kb-10workers-1window-first-iterable) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.sideinput_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_SideInput_Dataflow_Batch_test_arguments_3 }} --job_name=load-tests-python-dataflow-batch-sideinput-3-${{ steps.datetime.outputs.datetime }}' \ + - name: run SideInput Dataflow Batch Python Load Test 4 (10gb-1kb-10workers-1window-iterable) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.sideinput_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_SideInput_Dataflow_Batch_test_arguments_4 }} --job_name=load-tests-python-dataflow-batch-sideinput-4-${{ steps.datetime.outputs.datetime }}' \ + - name: run SideInput Dataflow Batch Python Load Test 5 (1gb-1kb-10workers-1window-first-list) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.sideinput_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_SideInput_Dataflow_Batch_test_arguments_5 }} --job_name=load-tests-python-dataflow-batch-sideinput-5-${{ steps.datetime.outputs.datetime }}' \ + - name: run SideInput Dataflow Batch Python Load Test 6 (1gb-1kb-10workers-1window-list) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.sideinput_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_SideInput_Dataflow_Batch_test_arguments_6 }} --job_name=load-tests-python-dataflow-batch-sideinput-6-${{ steps.datetime.outputs.datetime }}' \ + - name: run SideInput Dataflow Batch Python Load Test 7 (1gb-1kb-10workers-1000window-1key-percent-dict) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.sideinput_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_SideInput_Dataflow_Batch_test_arguments_7 }} --job_name=load-tests-python-dataflow-batch-sideinput-7-${{ steps.datetime.outputs.datetime }}' \ + - name: run SideInput Dataflow Batch Python Load Test 8 (1gb-1kb-10workers-1000window-99key-percent-dict) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.sideinput_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_SideInput_Dataflow_Batch_test_arguments_8 }} --job_name=load-tests-python-dataflow-batch-sideinput-8-${{ steps.datetime.outputs.datetime }}' \ + - name: run SideInput Dataflow Batch Python Load Test 9 (10gb-1kb-10workers-1000window-first-iterable) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.sideinput_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_SideInput_Dataflow_Batch_test_arguments_9 }} --job_name=load-tests-python-dataflow-batch-sideinput-9-${{ steps.datetime.outputs.datetime }}' \ + - name: run SideInput Dataflow Batch Python Load Test 10 (10gb-1kb-10workers-1000window-iterable) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.sideinput_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_SideInput_Dataflow_Batch_test_arguments_10 }} --job_name=load-tests-python-dataflow-batch-sideinput-10-${{ steps.datetime.outputs.datetime }}' \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_10gb_1000window_first_iterable.txt b/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_10gb_1000window_first_iterable.txt new file mode 100644 index 000000000000..204c07bc16a2 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_10gb_1000window_first_iterable.txt @@ -0,0 +1,30 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_batch_sideinput_9 +--influx_measurement=python_batch_sideinput_9 +--num_workers=10 +--autoscaling_algorithm=NONE +--experiments=use_runner_v2 +--input_options=''{\\"num_records\\":10000000,\\"key_size\\":100,\\"value_size\\":900,\\"algorithm\\":\\"lcg\\"}'' +--side_input_type=iter +--access_percentage=1 +--window_count=1000 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_10gb_1000window_iterable.txt b/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_10gb_1000window_iterable.txt new file mode 100644 index 000000000000..1ae64bb4a369 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_10gb_1000window_iterable.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_batch_sideinput_10 +--influx_measurement=python_batch_sideinput_10 +--num_workers=10 +--autoscaling_algorithm=NONE +--experiments=use_runner_v2 +--input_options=''{\\"num_records\\":10000000,\\"key_size\\":100,\\"value_size\\":900,\\"algorithm\\":\\"lcg\\"}'' +--side_input_type=iter +--window_count=1000 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_10gb_1window_first_iterable.txt b/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_10gb_1window_first_iterable.txt new file mode 100644 index 000000000000..0759517d9c2e --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_10gb_1window_first_iterable.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_batch_sideinput_3 +--influx_measurement=python_batch_sideinput_3 +--num_workers=10 +--autoscaling_algorithm=NONE +--experiments=use_runner_v2 +--input_options=''{\\"num_records\\":10000000,\\"key_size\\":100,\\"value_size\\":900,\\"algorithm\\":\\"lcg\\"}'' +--side_input_type=iter +--access_percentage=1 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_10gb_1window_iterable.txt b/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_10gb_1window_iterable.txt new file mode 100644 index 000000000000..c555c0d32d4c --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_10gb_1window_iterable.txt @@ -0,0 +1,28 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_batch_sideinput_4 +--influx_measurement=python_batch_sideinput_4 +--num_workers=10 +--autoscaling_algorithm=NONE +--experiments=use_runner_v2 +--input_options=''{\\"num_records\\":10000000,\\"key_size\\":100,\\"value_size\\":900,\\"algorithm\\":\\"lcg\\"}'' +--side_input_type=iter +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1000window_1key_percent_dict.txt b/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1000window_1key_percent_dict.txt new file mode 100644 index 000000000000..4b3cee817f43 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1000window_1key_percent_dict.txt @@ -0,0 +1,30 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_batch_sideinput_7 +--influx_measurement=python_batch_sideinput_7 +--num_workers=10 +--autoscaling_algorithm=NONE +--experiments=use_runner_v2 +--input_options=''{\\"num_records\\":1000000,\\"key_size\\":100,\\"value_size\\":900,\\"algorithm\\":\\"lcg\\"}'' +--side_input_type=dict +--access_percentage=1 +--window_count=1000 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1000window_99key_percent_dict.txt b/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1000window_99key_percent_dict.txt new file mode 100644 index 000000000000..00ba6feef50c --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1000window_99key_percent_dict.txt @@ -0,0 +1,30 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_batch_sideinput_8 +--influx_measurement=python_batch_sideinput_8 +--num_workers=10 +--autoscaling_algorithm=NONE +--experiments=use_runner_v2 +--input_options=''{\\"num_records\\":1000000,\\"key_size\\":100,\\"value_size\\":900,\\"algorithm\\":\\"lcg\\"}'' +--side_input_type=dict +--access_percentage=99 +--window_count=1000 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1window_1key_percent_dict.txt b/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1window_1key_percent_dict.txt new file mode 100644 index 000000000000..07e4a5ecba62 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1window_1key_percent_dict.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_batch_sideinput_1 +--influx_measurement=python_batch_sideinput_1 +--num_workers=10 +--autoscaling_algorithm=NONE +--experiments=use_runner_v2 +--input_options=''{\\"num_records\\":1000000,\\"key_size\\":100,\\"value_size\\":900,\\"algorithm\\":\\"lcg\\"}'' +--side_input_type=dict +--access_percentage=1 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1window_99key_percent_dict.txt b/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1window_99key_percent_dict.txt new file mode 100644 index 000000000000..b565598c1a16 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1window_99key_percent_dict.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_batch_sideinput_2 +--influx_measurement=python_batch_sideinput_2 +--num_workers=10 +--autoscaling_algorithm=NONE +--experiments=use_runner_v2 +--input_options=''{\\"num_records\\":1000000,\\"key_size\\":100,\\"value_size\\":900,\\"algorithm\\":\\"lcg\\"}'' +--side_input_type=dict +--access_percentage=99 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1window_first_list.txt b/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1window_first_list.txt new file mode 100644 index 000000000000..de5b02198177 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1window_first_list.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_batch_sideinput_5 +--influx_measurement=python_batch_sideinput_5 +--num_workers=10 +--autoscaling_algorithm=NONE +--experiments=use_runner_v2 +--input_options=''{\\"num_records\\":1000000,\\"key_size\\":100,\\"value_size\\":900,\\"algorithm\\":\\"lcg\\"}'' +--side_input_type=list +--access_percentage=1 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1window_list.txt b/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1window_list.txt new file mode 100644 index 000000000000..078ce2cc74fe --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1window_list.txt @@ -0,0 +1,28 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_batch_sideinput_6 +--influx_measurement=python_batch_sideinput_6 +--num_workers=10 +--autoscaling_algorithm=NONE +--experiments=use_runner_v2 +--input_options=''{\\"num_records\\":1000000,\\"key_size\\":100,\\"value_size\\":900,\\"algorithm\\":\\"lcg\\"}'' +--side_input_type=list +--runner=DataflowRunner \ No newline at end of file From 6bab4b579923c52a04723ab0a9c99874bc9980c4 Mon Sep 17 00:00:00 2001 From: Vitaly Terentyev Date: Wed, 11 Oct 2023 23:27:48 +0400 Subject: [PATCH 112/435] Add Community Metrics Prober workflow (#28842) --- .../beam_Prober_CommunityMetrics.yml | 77 +++++++++++++++++++ 1 file changed, 77 insertions(+) create mode 100644 .github/workflows/beam_Prober_CommunityMetrics.yml diff --git a/.github/workflows/beam_Prober_CommunityMetrics.yml b/.github/workflows/beam_Prober_CommunityMetrics.yml new file mode 100644 index 000000000000..5095f8d51737 --- /dev/null +++ b/.github/workflows/beam_Prober_CommunityMetrics.yml @@ -0,0 +1,77 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: Community Metrics Prober + +on: + issue_comment: + types: [created] + schedule: + - cron: '0 */6 * * *' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_Prober_CommunityMetrics: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Community Metrics Prober' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 100 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_Prober_CommunityMetrics"] + job_phrase: ["Run Community Metrics Prober"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Health check probes for the Community Metrics infrastructure + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :communityMetricsProber + arguments: | + --rerun-tasks \ No newline at end of file From 104c10b3ee536a9a3ea52b4dbf62d86b669da5d9 Mon Sep 17 00:00:00 2001 From: Damon Date: Wed, 11 Oct 2023 12:39:38 -0700 Subject: [PATCH 113/435] [RRIO] Create Caller and SetupTeardown interfaces (#28905) * Create test Caller and SetupTeardown interfaces * Update Javadoc * Defer Call transform to future PR * Rename package to requestresponseio * Add username to TODO --- sdks/java/io/rrio/build.gradle | 7 +- .../beam/io/requestresponseio/Caller.java | 27 ++++ .../io/requestresponseio/SetupTeardown.java | 34 +++++ .../UserCodeExecutionException.java | 38 +++++ .../UserCodeQuotaException.java | 42 ++++++ .../UserCodeTimeoutException.java | 39 ++++++ .../io/requestresponseio/package-info.java | 20 +++ .../org/apache/beam/io/rrio/CallerTest.java | 126 +++++++++++++++++ .../beam/io/rrio/SetupTeardownTest.java | 132 ++++++++++++++++++ 9 files changed, 462 insertions(+), 3 deletions(-) create mode 100644 sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/Caller.java create mode 100644 sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/SetupTeardown.java create mode 100644 sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/UserCodeExecutionException.java create mode 100644 sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/UserCodeQuotaException.java create mode 100644 sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/UserCodeTimeoutException.java create mode 100644 sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/package-info.java create mode 100644 sdks/java/io/rrio/src/test/java/org/apache/beam/io/rrio/CallerTest.java create mode 100644 sdks/java/io/rrio/src/test/java/org/apache/beam/io/rrio/SetupTeardownTest.java diff --git a/sdks/java/io/rrio/build.gradle b/sdks/java/io/rrio/build.gradle index d65df370e0ca..32fbd9d22e38 100644 --- a/sdks/java/io/rrio/build.gradle +++ b/sdks/java/io/rrio/build.gradle @@ -25,9 +25,10 @@ description = "Apache Beam :: SDKS :: Java :: IO :: RequestResponseIO (RRIO)" ext.summary = "Support to read from and write to Web APIs" dependencies { - implementation project(path: ":sdks:java:core", configuration: "shadow") - implementation library.java.joda_time - implementation library.java.vendored_guava_32_1_2_jre + // TODO(damondouglas): revert to implementation after project is more fully developed + permitUnusedDeclared project(path: ":sdks:java:core", configuration: "shadow") + permitUnusedDeclared library.java.joda_time + permitUnusedDeclared library.java.vendored_guava_32_1_2_jre testImplementation project(path: ":sdks:java:core", configuration: "shadowTest") testImplementation library.java.junit diff --git a/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/Caller.java b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/Caller.java new file mode 100644 index 000000000000..32b514c43a15 --- /dev/null +++ b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/Caller.java @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.io.requestresponseio; + +import java.io.Serializable; + +/** {@link Caller} interfaces user custom code intended for API calls. */ +public interface Caller extends Serializable { + + /** Calls a Web API with the {@link RequestT} and returns a {@link ResponseT}. */ + ResponseT call(RequestT request) throws UserCodeExecutionException; +} diff --git a/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/SetupTeardown.java b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/SetupTeardown.java new file mode 100644 index 000000000000..2bdc8113d98e --- /dev/null +++ b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/SetupTeardown.java @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.io.requestresponseio; + +import java.io.Serializable; + +/** + * Provided by user and called within {@link org.apache.beam.sdk.transforms.DoFn.Setup} and @{link + * org.apache.beam.sdk.transforms.DoFn.Teardown} lifecycle methods of {@link Call}'s {@link + * org.apache.beam.sdk.transforms.DoFn}. + */ +public interface SetupTeardown extends Serializable { + + /** Called during the {@link org.apache.beam.sdk.transforms.DoFn}'s setup lifecycle method. */ + void setup() throws UserCodeExecutionException; + + /** Called during the {@link org.apache.beam.sdk.transforms.DoFn}'s teardown lifecycle method. */ + void teardown() throws UserCodeExecutionException; +} diff --git a/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/UserCodeExecutionException.java b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/UserCodeExecutionException.java new file mode 100644 index 000000000000..3a4c002f52e8 --- /dev/null +++ b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/UserCodeExecutionException.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.io.requestresponseio; + +/** Base {@link Exception} for signaling errors in user custom code. */ +public class UserCodeExecutionException extends Exception { + public UserCodeExecutionException(String message) { + super(message); + } + + public UserCodeExecutionException(String message, Throwable cause) { + super(message, cause); + } + + public UserCodeExecutionException(Throwable cause) { + super(cause); + } + + public UserCodeExecutionException( + String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) { + super(message, cause, enableSuppression, writableStackTrace); + } +} diff --git a/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/UserCodeQuotaException.java b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/UserCodeQuotaException.java new file mode 100644 index 000000000000..f16f078927f8 --- /dev/null +++ b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/UserCodeQuotaException.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.io.requestresponseio; + +/** + * Extends {@link UserCodeQuotaException} to allow the user custom code to specifically signal a + * Quota or API overuse related error. + */ +public class UserCodeQuotaException extends UserCodeExecutionException { + + public UserCodeQuotaException(String message) { + super(message); + } + + public UserCodeQuotaException(String message, Throwable cause) { + super(message, cause); + } + + public UserCodeQuotaException(Throwable cause) { + super(cause); + } + + public UserCodeQuotaException( + String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) { + super(message, cause, enableSuppression, writableStackTrace); + } +} diff --git a/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/UserCodeTimeoutException.java b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/UserCodeTimeoutException.java new file mode 100644 index 000000000000..22b067449858 --- /dev/null +++ b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/UserCodeTimeoutException.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.io.requestresponseio; + +/** An extension of {@link UserCodeQuotaException} to specifically signal a user code timeout. */ +public class UserCodeTimeoutException extends UserCodeExecutionException { + + public UserCodeTimeoutException(String message) { + super(message); + } + + public UserCodeTimeoutException(String message, Throwable cause) { + super(message, cause); + } + + public UserCodeTimeoutException(Throwable cause) { + super(cause); + } + + public UserCodeTimeoutException( + String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) { + super(message, cause, enableSuppression, writableStackTrace); + } +} diff --git a/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/package-info.java b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/package-info.java new file mode 100644 index 000000000000..cd9c11c13f86 --- /dev/null +++ b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/package-info.java @@ -0,0 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** Package provides Beam I/O transform support for safely reading from and writing to Web APIs. */ +package org.apache.beam.io.requestresponseio; diff --git a/sdks/java/io/rrio/src/test/java/org/apache/beam/io/rrio/CallerTest.java b/sdks/java/io/rrio/src/test/java/org/apache/beam/io/rrio/CallerTest.java new file mode 100644 index 000000000000..5258573f4283 --- /dev/null +++ b/sdks/java/io/rrio/src/test/java/org/apache/beam/io/rrio/CallerTest.java @@ -0,0 +1,126 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.io.rrio; + +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThrows; + +import org.apache.beam.io.requestresponseio.Caller; +import org.apache.beam.io.requestresponseio.UserCodeExecutionException; +import org.apache.beam.io.requestresponseio.UserCodeQuotaException; +import org.apache.beam.io.requestresponseio.UserCodeTimeoutException; +import org.apache.beam.sdk.Pipeline.PipelineExecutionException; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.coders.VarIntCoder; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.util.SerializableUtils; +import org.joda.time.Instant; +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Tests for {@link Caller}. */ +@RunWith(JUnit4.class) +public class CallerTest { + + @Rule public TestPipeline pipeline = TestPipeline.create(); + + @Test + public void canSerializeImplementingClasses() { + SerializableUtils.serializeToByteArray(new CallerImpl()); + } + + @Test + public void canSerializeWhenUsedInDoFn() { + pipeline + .apply(Create.of(Instant.now())) + .apply(ParDo.of(new CallerUsingDoFn<>(new CallerImpl()))) + .setCoder(StringUtf8Coder.of()); + + pipeline.run(); + } + + @Test + public void canSignalQuotaException() { + pipeline + .apply(Create.of(1)) + .apply(ParDo.of(new CallerUsingDoFn<>(new CallerThrowsQuotaException()))) + .setCoder(VarIntCoder.of()); + + PipelineExecutionException executionException = + assertThrows(PipelineExecutionException.class, pipeline::run); + assertEquals(UserCodeQuotaException.class, executionException.getCause().getClass()); + } + + @Test + public void canSignalTimeoutException() { + pipeline + .apply(Create.of(1)) + .apply(ParDo.of(new CallerUsingDoFn<>(new CallerThrowsTimeoutException()))) + .setCoder(VarIntCoder.of()); + + PipelineExecutionException executionException = + assertThrows(PipelineExecutionException.class, pipeline::run); + assertEquals(UserCodeTimeoutException.class, executionException.getCause().getClass()); + } + + private static class CallerUsingDoFn extends DoFn { + private final Caller caller; + + private CallerUsingDoFn(Caller caller) { + this.caller = caller; + } + + @ProcessElement + public void process(@Element RequestT request, OutputReceiver receiver) + throws UserCodeExecutionException { + RequestT safeRequest = checkStateNotNull(request); + ResponseT response = caller.call(safeRequest); + receiver.output(response); + } + } + + private static class CallerImpl implements Caller { + + @Override + public String call(Instant request) throws UserCodeExecutionException { + return request.toString(); + } + } + + private static class CallerThrowsQuotaException implements Caller { + + @Override + public Integer call(Integer request) throws UserCodeExecutionException { + throw new UserCodeQuotaException("quota"); + } + } + + private static class CallerThrowsTimeoutException implements Caller { + + @Override + public Integer call(Integer request) throws UserCodeExecutionException { + throw new UserCodeTimeoutException("timeout"); + } + } +} diff --git a/sdks/java/io/rrio/src/test/java/org/apache/beam/io/rrio/SetupTeardownTest.java b/sdks/java/io/rrio/src/test/java/org/apache/beam/io/rrio/SetupTeardownTest.java new file mode 100644 index 000000000000..a8c5c45ede5c --- /dev/null +++ b/sdks/java/io/rrio/src/test/java/org/apache/beam/io/rrio/SetupTeardownTest.java @@ -0,0 +1,132 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.io.rrio; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThrows; + +import org.apache.beam.io.requestresponseio.SetupTeardown; +import org.apache.beam.io.requestresponseio.UserCodeExecutionException; +import org.apache.beam.io.requestresponseio.UserCodeQuotaException; +import org.apache.beam.io.requestresponseio.UserCodeTimeoutException; +import org.apache.beam.sdk.coders.VarIntCoder; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.util.SerializableUtils; +import org.apache.beam.sdk.util.UserCodeException; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.UncheckedExecutionException; +import org.junit.Rule; +import org.junit.Test; + +public class SetupTeardownTest { + @Rule public TestPipeline pipeline = TestPipeline.create(); + + @Test + public void canSerializeImplementingClasses() { + SerializableUtils.serializeToByteArray(new SetupTeardownImpl()); + } + + @Test + public void canSerializeWhenUsedInDoFn() { + pipeline + .apply(Create.of(1)) + .apply(ParDo.of(new SetupTeardownUsingDoFn(new SetupTeardownImpl()))) + .setCoder(VarIntCoder.of()); + + pipeline.run(); + } + + @Test + public void canSignalQuotaException() { + pipeline + .apply(Create.of(1)) + .apply(ParDo.of(new SetupTeardownUsingDoFn(new ThrowsQuotaException()))) + .setCoder(VarIntCoder.of()); + + UncheckedExecutionException exception = + assertThrows(UncheckedExecutionException.class, pipeline::run); + UserCodeException userCodeException = (UserCodeException) exception.getCause(); + assertEquals(UserCodeQuotaException.class, userCodeException.getCause().getClass()); + } + + @Test + public void canSignalTimeoutException() { + pipeline + .apply(Create.of(1)) + .apply(ParDo.of(new SetupTeardownUsingDoFn(new ThrowsTimeoutException()))) + .setCoder(VarIntCoder.of()); + + UncheckedExecutionException exception = + assertThrows(UncheckedExecutionException.class, pipeline::run); + UserCodeException userCodeException = (UserCodeException) exception.getCause(); + assertEquals(UserCodeTimeoutException.class, userCodeException.getCause().getClass()); + } + + private static class SetupTeardownUsingDoFn extends DoFn { + private final SetupTeardown setupTeardown; + + private SetupTeardownUsingDoFn(SetupTeardown setupTeardown) { + this.setupTeardown = setupTeardown; + } + + @Setup + public void setup() throws UserCodeExecutionException { + setupTeardown.setup(); + } + + @Teardown + public void teardown() throws UserCodeExecutionException { + setupTeardown.teardown(); + } + + @ProcessElement + public void process() {} + } + + private static class SetupTeardownImpl implements SetupTeardown { + @Override + public void setup() throws UserCodeExecutionException {} + + @Override + public void teardown() throws UserCodeExecutionException {} + } + + private static class ThrowsQuotaException implements SetupTeardown { + + @Override + public void setup() throws UserCodeExecutionException { + throw new UserCodeQuotaException("quota"); + } + + @Override + public void teardown() throws UserCodeExecutionException {} + } + + private static class ThrowsTimeoutException implements SetupTeardown { + + @Override + public void setup() throws UserCodeExecutionException { + throw new UserCodeTimeoutException("timeout"); + } + + @Override + public void teardown() throws UserCodeExecutionException {} + } +} From e8e38145d96a7b0561a418c19519917b0567591e Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Wed, 11 Oct 2023 21:16:13 -0400 Subject: [PATCH 114/435] Setup Java 21 container (#28833) * Add Java21 container * fix spotless * Update website * Fix jamm * align configs in recent change --- ...a_CoGBK_Dataflow_V2_Batch_JavaVersions.yml | 8 ++-- ...GBK_Dataflow_V2_Streaming_JavaVersions.yml | 8 ++-- ...ests_Java_GBK_Dataflow_V2_Batch_Java11.yml | 14 +++---- ...ests_Java_GBK_Dataflow_V2_Batch_Java17.yml | 14 +++---- ..._Java_GBK_Dataflow_V2_Streaming_Java11.yml | 14 +++---- ..._Java_GBK_Dataflow_V2_Streaming_Java17.yml | 14 +++---- ..._PostCommit_Java_Examples_Dataflow_ARM.yml | 2 +- ...tCommit_Java_Examples_Dataflow_V2_Java.yml | 2 +- ...m_PostCommit_Java_Jpms_Dataflow_Java17.yml | 2 +- ...eam_PostCommit_Java_Jpms_Direct_Java17.yml | 2 +- ...stCommit_Java_Nexmark_Dataflow_V2_Java.yml | 2 +- ...eam_PostCommit_TransformService_Direct.yml | 2 +- ...reCommit_Java_Examples_Dataflow_Java11.yml | 2 +- ...reCommit_Java_Examples_Dataflow_Java17.yml | 2 +- .../workflows/beam_PreCommit_SQL_Java11.yml | 2 +- .../workflows/beam_PreCommit_SQL_Java17.yml | 2 +- .test-infra/jenkins/JavaTestProperties.groovy | 7 +++- .test-infra/jenkins/NexmarkBuilder.groovy | 4 +- ..._LoadTests_CoGBK_Dataflow_V2_Java11.groovy | 2 +- ..._LoadTests_CoGBK_Dataflow_V2_Java17.groovy | 2 +- ...ob_LoadTests_GBK_Dataflow_V2_Java11.groovy | 2 +- ...ob_LoadTests_GBK_Dataflow_V2_Java17.groovy | 2 +- ..._LoadTests_ParDo_Dataflow_V2_Java11.groovy | 2 +- ..._LoadTests_ParDo_Dataflow_V2_Java17.groovy | 2 +- ...it_Java_Examples_Dataflow_V2_Java11.groovy | 2 +- ...it_Java_Examples_Dataflow_V2_Java17.groovy | 2 +- ...ostCommit_Java_Jpms_Dataflow_Java17.groovy | 2 +- ..._PostCommit_Java_Jpms_Direct_Java17.groovy | 2 +- ..._PostCommit_TransformService_Direct.groovy | 2 +- ...ommit_Java_Examples_Dataflow_Java11.groovy | 2 +- .../jenkins/job_PreCommit_SQL_Java11.groovy | 2 +- .../jenkins/job_PreCommit_SQL_Java17.groovy | 2 +- ...ommit_Java_Examples_Dataflow_Java17.groovy | 2 +- build.gradle.kts | 14 +++---- .../beam/gradle/BeamModulePlugin.groovy | 37 ++++++++++++++++++- .../core/construction/Environments.java | 6 ++- .../core/construction/EnvironmentsTest.java | 6 ++- .../arm/build.gradle | 6 +-- .../google-cloud-dataflow-java/build.gradle | 6 +-- .../beam/runners/dataflow/DataflowRunner.java | 7 ++-- runners/spark/spark_runner.gradle | 5 ++- sdks/java/container/Dockerfile | 3 +- sdks/java/container/common.gradle | 14 ++++--- sdks/java/container/java21/build.gradle | 30 +++++++++++++++ sdks/java/container/java21/option-jamm.json | 12 ++++++ sdks/java/testing/jpms-tests/build.gradle | 21 +++++------ sdks/java/testing/test-utils/build.gradle | 29 +++++---------- .../jvmverification/JvmVerification.java | 16 +++++++- settings.gradle.kts | 1 + .../www/site/content/en/roadmap/java-sdk.md | 6 +-- 50 files changed, 220 insertions(+), 132 deletions(-) create mode 100644 sdks/java/container/java21/build.gradle create mode 100644 sdks/java/container/java21/option-jamm.json diff --git a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml index 35575aec0bef..ff172cc338b7 100644 --- a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml +++ b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml @@ -95,7 +95,7 @@ jobs: with: gradle-command: :sdks:java:testing:load-tests:run arguments: | - -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -PtestJavaVersion=${{ matrix.java_version }} \ -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ @@ -105,7 +105,7 @@ jobs: with: gradle-command: :sdks:java:testing:load-tests:run arguments: | - -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -PtestJavaVersion=${{ matrix.java_version }} \ -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ @@ -115,7 +115,7 @@ jobs: with: gradle-command: :sdks:java:testing:load-tests:run arguments: | - -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -PtestJavaVersion=${{ matrix.java_version }} \ -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ @@ -125,7 +125,7 @@ jobs: with: gradle-command: :sdks:java:testing:load-tests:run arguments: | - -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -PtestJavaVersion=${{ matrix.java_version }} \ -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ diff --git a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml index 46dfaa986920..efa61373f2ab 100644 --- a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml +++ b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml @@ -95,7 +95,7 @@ jobs: with: gradle-command: :sdks:java:testing:load-tests:run arguments: | - -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -PtestJavaVersion=${{ matrix.java_version }} \ -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ @@ -105,7 +105,7 @@ jobs: with: gradle-command: :sdks:java:testing:load-tests:run arguments: | - -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -PtestJavaVersion=${{ matrix.java_version }} \ -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ @@ -115,7 +115,7 @@ jobs: with: gradle-command: :sdks:java:testing:load-tests:run arguments: | - -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -PtestJavaVersion=${{ matrix.java_version }} \ -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ @@ -125,7 +125,7 @@ jobs: with: gradle-command: :sdks:java:testing:load-tests:run arguments: | - -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -PtestJavaVersion=${{ matrix.java_version }} \ -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml index 318f157e57a6..5c333013e2a2 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml @@ -93,7 +93,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava11 \ + -PtestJavaVersion=11 \ -Pjava11Home=$JAVA_HOME_11_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11_test_arguments_1 }}' \ - name: run Load test 2GB of 100B records @@ -104,7 +104,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava11 \ + -PtestJavaVersion=11 \ -Pjava11Home=$JAVA_HOME_11_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11_test_arguments_2 }}' \ - name: run Load test 2GB of 100kB records @@ -115,7 +115,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava11 \ + -PtestJavaVersion=11 \ -Pjava11Home=$JAVA_HOME_11_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11_test_arguments_3 }}' \ - name: run Load test fanout 4 times with 2GB 10-byte records total @@ -126,7 +126,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava11 \ + -PtestJavaVersion=11 \ -Pjava11Home=$JAVA_HOME_11_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11_test_arguments_4 }}' \ - name: run Load test fanout 8 times with 2GB 10-byte records total @@ -137,7 +137,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava11 \ + -PtestJavaVersion=11 \ -Pjava11Home=$JAVA_HOME_11_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11_test_arguments_5 }}' \ - name: run Load test reiterate 4 times 10kB values @@ -148,7 +148,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava11 \ + -PtestJavaVersion=11 \ -Pjava11Home=$JAVA_HOME_11_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11_test_arguments_6 }}' \ - name: run Load test reiterate 4 times 2MB values @@ -159,6 +159,6 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava11 \ + -PtestJavaVersion=11 \ -Pjava11Home=$JAVA_HOME_11_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11_test_arguments_7 }}' \ No newline at end of file diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml index 543cb86985d8..4fa5c364728b 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml @@ -95,7 +95,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava17 \ + -PtestJavaVersion=17 \ -Pjava17Home=$JAVA_HOME_17_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17_test_arguments_1 }}' \ - name: run Load test 2GB of 100B records @@ -106,7 +106,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava17 \ + -PtestJavaVersion=17 \ -Pjava17Home=$JAVA_HOME_17_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17_test_arguments_2 }}' \ - name: run Load test 2GB of 100kB records @@ -117,7 +117,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava17 \ + -PtestJavaVersion=17 \ -Pjava17Home=$JAVA_HOME_17_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17_test_arguments_3 }}' \ - name: run Load test fanout 4 times with 2GB 10-byte records total @@ -128,7 +128,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava17 \ + -PtestJavaVersion=17 \ -Pjava17Home=$JAVA_HOME_17_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17_test_arguments_4 }}' \ - name: run Load test fanout 8 times with 2GB 10-byte records total @@ -139,7 +139,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava17 \ + -PtestJavaVersion=17 \ -Pjava17Home=$JAVA_HOME_17_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17_test_arguments_5 }}' \ - name: run Load test reiterate 4 times 10kB values @@ -150,7 +150,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava17 \ + -PtestJavaVersion=17 \ -Pjava17Home=$JAVA_HOME_17_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17_test_arguments_6 }}' \ - name: run Load test reiterate 4 times 2MB values @@ -161,6 +161,6 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava17 \ + -PtestJavaVersion=17 \ -Pjava17Home=$JAVA_HOME_17_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17_test_arguments_7 }}' \ No newline at end of file diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml index fcd55f761c04..6432a315d8a1 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml @@ -93,7 +93,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava11 \ + -PtestJavaVersion=11 \ -Pjava11Home=$JAVA_HOME_11_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11_test_arguments_1 }}' \ - name: run Load test 2GB of 100B records @@ -104,7 +104,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava11 \ + -PtestJavaVersion=11 \ -Pjava11Home=$JAVA_HOME_11_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11_test_arguments_2 }}' \ - name: run Load test 2GB of 100kB records @@ -115,7 +115,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava11 \ + -PtestJavaVersion=11 \ -Pjava11Home=$JAVA_HOME_11_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11_test_arguments_3 }}' \ - name: run Load test fanout 4 times with 2GB 10-byte records total @@ -126,7 +126,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava11 \ + -PtestJavaVersion=11 \ -Pjava11Home=$JAVA_HOME_11_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11_test_arguments_4 }}' \ - name: run Load test fanout 8 times with 2GB 10-byte records total @@ -137,7 +137,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava11 \ + -PtestJavaVersion=11 \ -Pjava11Home=$JAVA_HOME_11_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11_test_arguments_5 }}' \ - name: run Load test reiterate 4 times 10kB values @@ -148,7 +148,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava11 \ + -PtestJavaVersion=11 \ -Pjava11Home=$JAVA_HOME_11_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11_test_arguments_6 }}' \ - name: run Load test reiterate 4 times 2MB values @@ -159,6 +159,6 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava11 \ + -PtestJavaVersion=11 \ -Pjava11Home=$JAVA_HOME_11_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11_test_arguments_7 }}' \ No newline at end of file diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml index 5df7d3249192..cd8c34279593 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml @@ -95,7 +95,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava17 \ + -PtestJavaVersion=17 \ -Pjava17Home=$JAVA_HOME_17_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17_test_arguments_1 }}' \ - name: run Load test 2GB of 100B records @@ -106,7 +106,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava17 \ + -PtestJavaVersion=17 \ -Pjava17Home=$JAVA_HOME_17_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17_test_arguments_2 }}' \ - name: run Load test 2GB of 100kB records @@ -117,7 +117,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava17 \ + -PtestJavaVersion=17 \ -Pjava17Home=$JAVA_HOME_17_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17_test_arguments_3 }}' \ - name: run Load test fanout 4 times with 2GB 10-byte records total @@ -128,7 +128,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava17 \ + -PtestJavaVersion=17 \ -Pjava17Home=$JAVA_HOME_17_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17_test_arguments_4 }}' \ - name: run Load test fanout 8 times with 2GB 10-byte records total @@ -139,7 +139,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava17 \ + -PtestJavaVersion=17 \ -Pjava17Home=$JAVA_HOME_17_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17_test_arguments_5 }}' \ - name: run Load test reiterate 4 times 10kB values @@ -150,7 +150,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava17 \ + -PtestJavaVersion=17 \ -Pjava17Home=$JAVA_HOME_17_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17_test_arguments_6 }}' \ - name: run Load test reiterate 4 times 2MB values @@ -161,6 +161,6 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava17 \ + -PtestJavaVersion=17 \ -Pjava17Home=$JAVA_HOME_17_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17_test_arguments_7 }}' \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml index 939d2646d352..8add1a2445c8 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml @@ -111,7 +111,7 @@ jobs: gradle-command: :runners:google-cloud-dataflow-java:arm:examplesJavaRunnerV2IntegrationTestARM max-workers: 12 arguments: | - -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -PtestJavaVersion${{ matrix.java_version }} \ -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ -Pcontainer-architecture-list=arm64,amd64 \ -Ppush-containers \ diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml index 74e1787945e3..ab862df1847a 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml @@ -86,7 +86,7 @@ jobs: -PdisableSpotlessCheck=true \ -PdisableCheckStyle=true \ -PskipCheckerFramework \ - -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -PtestJavaVersion${{ matrix.java_version }} \ -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml index 0b9a366211ee..3f6352b0a948 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml @@ -80,7 +80,7 @@ jobs: gradle-command: :sdks:java:testing:jpms-tests:dataflowRunnerIntegrationTest arguments: -PskipCheckerFramework - -PcompileAndRunTestsWithJava17 + -PtestJavaVersion=17 -Pjava17Home=$JAVA_HOME_17_X64 - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml index fcdbd63f7e97..360b9c757a02 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml @@ -80,7 +80,7 @@ jobs: gradle-command: :sdks:java:testing:jpms-tests:directRunnerIntegrationTest arguments: -PskipCheckerFramework - -PcompileAndRunTestsWithJava17 + -PtestJavaVersion=17 -Pjava17Home=$JAVA_HOME_17_X64 - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml index d78793172898..5782928a8b4d 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml @@ -107,7 +107,7 @@ jobs: with: gradle-command: :sdks:java:testing:nexmark:run arguments: | - -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -PtestJavaVersion${{ matrix.java_version }} \ -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ -Pnexmark.runner.version=V2 \ -Pnexmark.runner=:runners:google-cloud-dataflow-java \ diff --git a/.github/workflows/beam_PostCommit_TransformService_Direct.yml b/.github/workflows/beam_PostCommit_TransformService_Direct.yml index 491688e097a0..2ba83506d395 100644 --- a/.github/workflows/beam_PostCommit_TransformService_Direct.yml +++ b/.github/workflows/beam_PostCommit_TransformService_Direct.yml @@ -84,7 +84,7 @@ jobs: with: gradle-command: :sdks:python:test-suites:direct:xlang:transformServicePythonUsingJava arguments: | - -PcompileAndRunTestsWithJava11 \ + -PtestJavaVersion=11 \ -Pjava11Home=$JAVA_HOME_11_X64 \ -PuseWheelDistribution \ -PpythonVersion=${{ matrix.python_version }} \ diff --git a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml index 03f4d32861b1..186d75674c71 100644 --- a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml +++ b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml @@ -115,7 +115,7 @@ jobs: -PdisableSpotlessCheck=true \ -PdisableCheckStyle=true \ -PskipCheckerFramework \ - -PcompileAndRunTestsWithJava11 \ + -PtestJavaVersion=11 \ -Pjava11Home=$JAVA_HOME_11_X64 \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 diff --git a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml index cf8cefe388f3..cafc8cf5ab6e 100644 --- a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml +++ b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml @@ -115,7 +115,7 @@ jobs: arguments: | -PdisableSpotlessCheck=true \ -PdisableCheckStyle=true \ - -PcompileAndRunTestsWithJava17 \ + -PtestJavaVersion=17 \ -PskipCheckerFramework \ -Pjava17Home=$JAVA_HOME_17_X64 \ max-workers: 12 diff --git a/.github/workflows/beam_PreCommit_SQL_Java11.yml b/.github/workflows/beam_PreCommit_SQL_Java11.yml index f1c733418b8d..3412365bf13e 100644 --- a/.github/workflows/beam_PreCommit_SQL_Java11.yml +++ b/.github/workflows/beam_PreCommit_SQL_Java11.yml @@ -103,7 +103,7 @@ jobs: arguments: | -PdisableSpotlessCheck=true \ -PdisableCheckStyle=true \ - -PcompileAndRunTestsWithJava11 \ + -PtestJavaVersion=11 \ -PskipCheckerFramework \ -Pjava11Home=$JAVA_HOME_11_X64 \ - name: Archive JUnit Test Results diff --git a/.github/workflows/beam_PreCommit_SQL_Java17.yml b/.github/workflows/beam_PreCommit_SQL_Java17.yml index 7547bd396815..2215bab8ddeb 100644 --- a/.github/workflows/beam_PreCommit_SQL_Java17.yml +++ b/.github/workflows/beam_PreCommit_SQL_Java17.yml @@ -101,7 +101,7 @@ jobs: arguments: | -PdisableSpotlessCheck=true \ -PdisableCheckStyle=true \ - -PcompileAndRunTestsWithJava17 \ + -PtestJavaVersion=17 \ -PskipCheckerFramework \ -Pjava17Home=$JAVA_HOME_17_X64 \ - name: Archive JUnit Test Results diff --git a/.test-infra/jenkins/JavaTestProperties.groovy b/.test-infra/jenkins/JavaTestProperties.groovy index ce7446a6e71d..5403cee5cf9a 100644 --- a/.test-infra/jenkins/JavaTestProperties.groovy +++ b/.test-infra/jenkins/JavaTestProperties.groovy @@ -17,5 +17,10 @@ */ class JavaTestProperties { - final static List SUPPORTED_CONTAINER_TASKS = ['java8', 'java11', 'java17'] + final static List SUPPORTED_CONTAINER_TASKS = [ + 'java8', + 'java11', + 'java17', + 'java21' + ] } diff --git a/.test-infra/jenkins/NexmarkBuilder.groovy b/.test-infra/jenkins/NexmarkBuilder.groovy index 044b0cbb9561..69fa3dcc4277 100644 --- a/.test-infra/jenkins/NexmarkBuilder.groovy +++ b/.test-infra/jenkins/NexmarkBuilder.groovy @@ -145,7 +145,7 @@ class NexmarkBuilder { rootBuildScriptDir(commonJobProperties.checkoutDir) tasks(':sdks:java:testing:nexmark:run') commonJobProperties.setGradleSwitches(delegate) - switches("-PcompileAndRunTestsWithJava11") + switches("-PtestJavaVersion=11") switches("-Pjava11Home=${commonJobProperties.JAVA_11_HOME}") switches("-Pnexmark.runner=${runner.getDependencyBySDK(sdk)}") switches("-Pnexmark.args=\"${parseOptions(options)}\"") @@ -168,7 +168,7 @@ class NexmarkBuilder { rootBuildScriptDir(commonJobProperties.checkoutDir) tasks(':sdks:java:testing:nexmark:run') commonJobProperties.setGradleSwitches(delegate) - switches("-PcompileAndRunTestsWithJava17") + switches("-PtestJavaVersion=17") switches("-Pjava17Home=${commonJobProperties.JAVA_17_HOME}") switches("-Pnexmark.runner=${runner.getDependencyBySDK(sdk)}") switches("-Pnexmark.args=\"${parseOptions(options)}\"") diff --git a/.test-infra/jenkins/job_LoadTests_CoGBK_Dataflow_V2_Java11.groovy b/.test-infra/jenkins/job_LoadTests_CoGBK_Dataflow_V2_Java11.groovy index fc7f39d28a0d..55501db4429d 100644 --- a/.test-infra/jenkins/job_LoadTests_CoGBK_Dataflow_V2_Java11.groovy +++ b/.test-infra/jenkins/job_LoadTests_CoGBK_Dataflow_V2_Java11.groovy @@ -188,7 +188,7 @@ def loadTestConfigurations = { mode, isStreaming -> def final JOB_SPECIFIC_SWITCHES = [ '-Prunner.version="V2"', - '-PcompileAndRunTestsWithJava11', + '-PtestJavaVersion=11', "-Pjava11Home=${commonJobProperties.JAVA_11_HOME}" ] diff --git a/.test-infra/jenkins/job_LoadTests_CoGBK_Dataflow_V2_Java17.groovy b/.test-infra/jenkins/job_LoadTests_CoGBK_Dataflow_V2_Java17.groovy index ca8c6689ad0f..8fb09fd07448 100644 --- a/.test-infra/jenkins/job_LoadTests_CoGBK_Dataflow_V2_Java17.groovy +++ b/.test-infra/jenkins/job_LoadTests_CoGBK_Dataflow_V2_Java17.groovy @@ -188,7 +188,7 @@ def loadTestConfigurations = { mode, isStreaming -> def final JOB_SPECIFIC_SWITCHES = [ '-Prunner.version="V2"', - '-PcompileAndRunTestsWithJava17', + '-PtestJavaVersion=17', "-Pjava17Home=${commonJobProperties.JAVA_17_HOME}" ] diff --git a/.test-infra/jenkins/job_LoadTests_GBK_Dataflow_V2_Java11.groovy b/.test-infra/jenkins/job_LoadTests_GBK_Dataflow_V2_Java11.groovy index cc2d5d2e5554..2191e448fade 100644 --- a/.test-infra/jenkins/job_LoadTests_GBK_Dataflow_V2_Java11.groovy +++ b/.test-infra/jenkins/job_LoadTests_GBK_Dataflow_V2_Java11.groovy @@ -253,7 +253,7 @@ def loadTestConfigurations = { mode, isStreaming -> def final JOB_SPECIFIC_SWITCHES = [ '-Prunner.version="V2"', - '-PcompileAndRunTestsWithJava11', + '-PtestJavaVersion=11', "-Pjava11Home=${commonJobProperties.JAVA_11_HOME}" ] diff --git a/.test-infra/jenkins/job_LoadTests_GBK_Dataflow_V2_Java17.groovy b/.test-infra/jenkins/job_LoadTests_GBK_Dataflow_V2_Java17.groovy index 7405f9154b83..2520f68f0178 100644 --- a/.test-infra/jenkins/job_LoadTests_GBK_Dataflow_V2_Java17.groovy +++ b/.test-infra/jenkins/job_LoadTests_GBK_Dataflow_V2_Java17.groovy @@ -253,7 +253,7 @@ def loadTestConfigurations = { mode, isStreaming -> def final JOB_SPECIFIC_SWITCHES = [ '-Prunner.version="V2"', - '-PcompileAndRunTestsWithJava17', + '-PtestJavaVersion=17', "-Pjava17Home=${commonJobProperties.JAVA_17_HOME}" ] diff --git a/.test-infra/jenkins/job_LoadTests_ParDo_Dataflow_V2_Java11.groovy b/.test-infra/jenkins/job_LoadTests_ParDo_Dataflow_V2_Java11.groovy index a4535d52e6cf..b7154e840e6a 100644 --- a/.test-infra/jenkins/job_LoadTests_ParDo_Dataflow_V2_Java11.groovy +++ b/.test-infra/jenkins/job_LoadTests_ParDo_Dataflow_V2_Java11.groovy @@ -160,7 +160,7 @@ def commonLoadTestConfig = { jobType, isStreaming -> def final JOB_SPECIFIC_SWITCHES = [ '-Prunner.version="V2"', - '-PcompileAndRunTestsWithJava11', + '-PtestJavaVersion=11', "-Pjava11Home=${commonJobProperties.JAVA_11_HOME}" ] diff --git a/.test-infra/jenkins/job_LoadTests_ParDo_Dataflow_V2_Java17.groovy b/.test-infra/jenkins/job_LoadTests_ParDo_Dataflow_V2_Java17.groovy index f0f2179ebb3b..df6c66e02aed 100644 --- a/.test-infra/jenkins/job_LoadTests_ParDo_Dataflow_V2_Java17.groovy +++ b/.test-infra/jenkins/job_LoadTests_ParDo_Dataflow_V2_Java17.groovy @@ -160,7 +160,7 @@ def commonLoadTestConfig = { jobType, isStreaming -> def final JOB_SPECIFIC_SWITCHES = [ '-Prunner.version="V2"', - '-PcompileAndRunTestsWithJava17', + '-PtestJavaVersion=17', "-Pjava17Home=${commonJobProperties.JAVA_17_HOME}" ] diff --git a/.test-infra/jenkins/job_PostCommit_Java_Examples_Dataflow_V2_Java11.groovy b/.test-infra/jenkins/job_PostCommit_Java_Examples_Dataflow_V2_Java11.groovy index 6687ae0e6f8a..6229f7c48a72 100644 --- a/.test-infra/jenkins/job_PostCommit_Java_Examples_Dataflow_V2_Java11.groovy +++ b/.test-infra/jenkins/job_PostCommit_Java_Examples_Dataflow_V2_Java11.groovy @@ -43,7 +43,7 @@ PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_Examples_Dataflow_V2_ja commonJobProperties.setGradleSwitches(delegate, 3 * Runtime.runtime.availableProcessors()) switches '-PdisableSpotlessCheck=true' switches '-PdisableCheckStyle=true' - switches '-PcompileAndRunTestsWithJava11' + switches '-PtestJavaVersion=11' switches '-PskipCheckerFramework' switches "-Pjava11Home=${commonJobProperties.JAVA_11_HOME}" } diff --git a/.test-infra/jenkins/job_PostCommit_Java_Examples_Dataflow_V2_Java17.groovy b/.test-infra/jenkins/job_PostCommit_Java_Examples_Dataflow_V2_Java17.groovy index b275fe9276d9..7e52a7e09789 100644 --- a/.test-infra/jenkins/job_PostCommit_Java_Examples_Dataflow_V2_Java17.groovy +++ b/.test-infra/jenkins/job_PostCommit_Java_Examples_Dataflow_V2_Java17.groovy @@ -43,7 +43,7 @@ PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_Examples_Dataflow_V2_ja commonJobProperties.setGradleSwitches(delegate, 3 * Runtime.runtime.availableProcessors()) switches '-PdisableSpotlessCheck=true' switches '-PdisableCheckStyle=true' - switches '-PcompileAndRunTestsWithJava17' + switches '-PtestJavaVersion=17' switches '-PskipCheckerFramework' switches "-Pjava17Home=${commonJobProperties.JAVA_17_HOME}" } diff --git a/.test-infra/jenkins/job_PostCommit_Java_Jpms_Dataflow_Java17.groovy b/.test-infra/jenkins/job_PostCommit_Java_Jpms_Dataflow_Java17.groovy index 4e26c164319e..f518985ca7a8 100644 --- a/.test-infra/jenkins/job_PostCommit_Java_Jpms_Dataflow_Java17.groovy +++ b/.test-infra/jenkins/job_PostCommit_Java_Jpms_Dataflow_Java17.groovy @@ -42,7 +42,7 @@ PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_Jpms_Dataflow_Java17', tasks(':sdks:java:testing:jpms-tests:dataflowRunnerIntegrationTest') commonJobProperties.setGradleSwitches(delegate) switches("-PskipCheckerFramework") - switches("-PcompileAndRunTestsWithJava17") + switches("-PtestJavaVersion=17") switches("-Pjava17Home=${commonJobProperties.JAVA_17_HOME}") // Specify maven home on Jenkins, needed by Maven archetype integration tests. switches('-Pmaven_home=/home/jenkins/tools/maven/apache-maven-3.5.4') diff --git a/.test-infra/jenkins/job_PostCommit_Java_Jpms_Direct_Java17.groovy b/.test-infra/jenkins/job_PostCommit_Java_Jpms_Direct_Java17.groovy index f31373ecaada..04c31389ecbe 100644 --- a/.test-infra/jenkins/job_PostCommit_Java_Jpms_Direct_Java17.groovy +++ b/.test-infra/jenkins/job_PostCommit_Java_Jpms_Direct_Java17.groovy @@ -42,7 +42,7 @@ PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_Jpms_Direct_Java17', 'R tasks(':sdks:java:testing:jpms-tests:directRunnerIntegrationTest') commonJobProperties.setGradleSwitches(delegate) switches("-PskipCheckerFramework") - switches("-PcompileAndRunTestsWithJava17") + switches("-PtestJavaVersion=17") switches("-Pjava17Home=${commonJobProperties.JAVA_17_HOME}") // Specify maven home on Jenkins, needed by Maven archetype integration tests. switches('-Pmaven_home=/home/jenkins/tools/maven/apache-maven-3.5.4') diff --git a/.test-infra/jenkins/job_PostCommit_TransformService_Direct.groovy b/.test-infra/jenkins/job_PostCommit_TransformService_Direct.groovy index 0d7f58e71706..03d29069a52c 100644 --- a/.test-infra/jenkins/job_PostCommit_TransformService_Direct.groovy +++ b/.test-infra/jenkins/job_PostCommit_TransformService_Direct.groovy @@ -43,7 +43,7 @@ PostcommitJobBuilder.postCommitJob('beam_PostCommit_TransformService_Direct', rootBuildScriptDir(commonJobProperties.checkoutDir) tasks(':sdks:python:test-suites:direct:xlang:transformServicePythonUsingJava') commonJobProperties.setGradleSwitches(delegate) - switches '-PcompileAndRunTestsWithJava11' + switches '-PtestJavaVersion=11' switches "-Pjava11Home=${commonJobProperties.JAVA_11_HOME}" switches("-PuseWheelDistribution") switches("-PpythonVersion=${pythonVersion}") diff --git a/.test-infra/jenkins/job_PreCommit_Java_Examples_Dataflow_Java11.groovy b/.test-infra/jenkins/job_PreCommit_Java_Examples_Dataflow_Java11.groovy index cabf5dbdd457..9f733a42ccce 100644 --- a/.test-infra/jenkins/job_PreCommit_Java_Examples_Dataflow_Java11.groovy +++ b/.test-infra/jenkins/job_PreCommit_Java_Examples_Dataflow_Java11.groovy @@ -50,7 +50,7 @@ builder.build { switches '-PdisableSpotlessCheck=true' switches '-PdisableCheckStyle=true' switches '-PskipCheckerFramework' // Gradle itself is running under JDK8 so plugin configures wrong for JDK11 - switches '-PcompileAndRunTestsWithJava11' + switches '-PtestJavaVersion=11' switches "-Pjava11Home=${properties.JAVA_11_HOME}" properties.setGradleSwitches(delegate, 3 * Runtime.runtime.availableProcessors()) } diff --git a/.test-infra/jenkins/job_PreCommit_SQL_Java11.groovy b/.test-infra/jenkins/job_PreCommit_SQL_Java11.groovy index 9742ab756cf7..34154b37209e 100644 --- a/.test-infra/jenkins/job_PreCommit_SQL_Java11.groovy +++ b/.test-infra/jenkins/job_PreCommit_SQL_Java11.groovy @@ -26,7 +26,7 @@ PrecommitJobBuilder builder = new PrecommitJobBuilder( gradleSwitches: [ '-PdisableSpotlessCheck=true', '-PdisableCheckStyle=true', - '-PcompileAndRunTestsWithJava11', + '-PtestJavaVersion=11', '-PskipCheckerFramework', // Gradle itself is running under JDK8 so plugin configures wrong for JDK11 "-Pjava11Home=${properties.JAVA_11_HOME}" diff --git a/.test-infra/jenkins/job_PreCommit_SQL_Java17.groovy b/.test-infra/jenkins/job_PreCommit_SQL_Java17.groovy index 158fa683c1a8..472521cb1b08 100644 --- a/.test-infra/jenkins/job_PreCommit_SQL_Java17.groovy +++ b/.test-infra/jenkins/job_PreCommit_SQL_Java17.groovy @@ -26,7 +26,7 @@ PrecommitJobBuilder builder = new PrecommitJobBuilder( gradleSwitches: [ '-PdisableSpotlessCheck=true', '-PdisableCheckStyle=true', - '-PcompileAndRunTestsWithJava17', + '-PtestJavaVersion=17', '-PskipCheckerFramework', // Gradle itself is running under JDK8 so plugin configures wrong for JDK17 "-Pjava17Home=${properties.JAVA_17_HOME}" diff --git a/.test-infra/jenkins/job_Precommit_Java_Examples_Dataflow_Java17.groovy b/.test-infra/jenkins/job_Precommit_Java_Examples_Dataflow_Java17.groovy index 3654a4c75edb..387b707123e7 100644 --- a/.test-infra/jenkins/job_Precommit_Java_Examples_Dataflow_Java17.groovy +++ b/.test-infra/jenkins/job_Precommit_Java_Examples_Dataflow_Java17.groovy @@ -50,7 +50,7 @@ builder.build { switches '-PdisableSpotlessCheck=true' switches '-PdisableCheckStyle=true' switches '-PskipCheckerFramework' // Gradle itself is running under JDK8 so plugin configures wrong for JDK17 - switches '-PcompileAndRunTestsWithJava17' + switches '-PtestJavaVersion=17' switches "-Pjava17Home=${properties.JAVA_17_HOME}" properties.setGradleSwitches(delegate, 3 * Runtime.runtime.availableProcessors()) } diff --git a/build.gradle.kts b/build.gradle.kts index ea1b4e6784e3..5bcfbb3ed06e 100644 --- a/build.gradle.kts +++ b/build.gradle.kts @@ -712,14 +712,12 @@ if (project.hasProperty("javaLinkageArtifactIds")) { } } } -if (project.hasProperty("compileAndRunTestsWithJava11")) { - tasks.getByName("javaPreCommitPortabilityApi").dependsOn(":sdks:java:testing:test-utils:verifyJavaVersion") - tasks.getByName("javaExamplesDataflowPrecommit").dependsOn(":sdks:java:testing:test-utils:verifyJavaVersion") - tasks.getByName("sqlPreCommit").dependsOn(":sdks:java:testing:test-utils:verifyJavaVersion") -} else if (project.hasProperty("compileAndRunTestsWithJava17")) { - tasks.getByName("javaPreCommitPortabilityApi").dependsOn(":sdks:java:testing:test-utils:verifyJavaVersion17") - tasks.getByName("javaExamplesDataflowPrecommit").dependsOn(":sdks:java:testing:test-utils:verifyJavaVersion17") - tasks.getByName("sqlPreCommit").dependsOn(":sdks:java:testing:test-utils:verifyJavaVersion17") +if (project.hasProperty("testJavaVersion")) { + var testVer = project.property("testJavaVersion") + + tasks.getByName("javaPreCommitPortabilityApi").dependsOn(":sdks:java:testing:test-utils:verifyJavaVersion$testVer") + tasks.getByName("javaExamplesDataflowPrecommit").dependsOn(":sdks:java:testing:test-utils:verifyJavaVersion$testVer") + tasks.getByName("sqlPreCommit").dependsOn(":sdks:java:testing:test-utils:verifyJavaVersion$testVer") } else { allprojects { tasks.withType(Test::class).configureEach { diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index 19ed6aaa4276..c32717aae725 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -946,6 +946,29 @@ class BeamModulePlugin implements Plugin { ] } + project.ext.setJava21Options = { CompileOptions options -> + def java17Home = project.findProperty("java17Home") + options.fork = true + options.forkOptions.javaHome = java17Home as File + options.compilerArgs += ['-Xlint:-path'] + // Error prone requires some packages to be exported/opened for Java 17 + // Disabling checks since this property is only used for Jenkins tests + // https://github.com/tbroyer/gradle-errorprone-plugin#jdk-16-support + options.errorprone.errorproneArgs.add("-XepDisableAllChecks") + options.forkOptions.jvmArgs += [ + "-J--add-exports=jdk.compiler/com.sun.tools.javac.api=ALL-UNNAMED", + "-J--add-exports=jdk.compiler/com.sun.tools.javac.file=ALL-UNNAMED", + "-J--add-exports=jdk.compiler/com.sun.tools.javac.main=ALL-UNNAMED", + "-J--add-exports=jdk.compiler/com.sun.tools.javac.model=ALL-UNNAMED", + "-J--add-exports=jdk.compiler/com.sun.tools.javac.parser=ALL-UNNAMED", + "-J--add-exports=jdk.compiler/com.sun.tools.javac.processing=ALL-UNNAMED", + "-J--add-exports=jdk.compiler/com.sun.tools.javac.tree=ALL-UNNAMED", + "-J--add-exports=jdk.compiler/com.sun.tools.javac.util=ALL-UNNAMED", + "-J--add-opens=jdk.compiler/com.sun.tools.javac.code=ALL-UNNAMED", + "-J--add-opens=jdk.compiler/com.sun.tools.javac.comp=ALL-UNNAMED" + ] + } + project.ext.repositories = { maven { name "testPublicationLocal" @@ -1492,7 +1515,7 @@ class BeamModulePlugin implements Plugin { options.errorprone.errorproneArgs.add("-Xep:Slf4jLoggerShouldBeNonStatic:OFF") } - if (project.hasProperty("compileAndRunTestsWithJava11")) { + if (project.findProperty('testJavaVersion') == "11") { def java11Home = project.findProperty("java11Home") project.tasks.compileTestJava { options.fork = true @@ -1504,7 +1527,7 @@ class BeamModulePlugin implements Plugin { useJUnit() executable = "${java11Home}/bin/java" } - } else if (project.hasProperty("compileAndRunTestsWithJava17")) { + } else if (project.findProperty('testJavaVersion') == "17") { def java17Home = project.findProperty("java17Home") project.tasks.compileTestJava { setCompileAndRuntimeJavaVersion(options.compilerArgs, '17') @@ -1514,6 +1537,16 @@ class BeamModulePlugin implements Plugin { useJUnit() executable = "${java17Home}/bin/java" } + } else if (project.findProperty('testJavaVersion') == "21") { + def java21Home = project.findProperty("java21Home") + project.tasks.compileTestJava { + setCompileAndRuntimeJavaVersion(options.compilerArgs, '21') + project.ext.setJava17Options(options) + } + project.tasks.withType(Test).configureEach { + useJUnit() + executable = "${java21Home}/bin/java" + } } if (configuration.shadowClosure) { diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Environments.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Environments.java index 31a555989afd..f531b5be344d 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Environments.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Environments.java @@ -94,7 +94,8 @@ public class Environments { public enum JavaVersion { java8("java", "1.8", 8), java11("java11", "11", 11), - java17("java17", "17", 17); + java17("java17", "17", 17), + java21("java21", "21", 21); // Legacy name, as used in container image private final String legacyName; @@ -119,6 +120,7 @@ public String specification() { return this.specification; } + /** Return the LTS java version given the Java specification version. */ public static JavaVersion forSpecification(String specification) { for (JavaVersion ver : JavaVersion.values()) { if (ver.specification.equals(specification)) { @@ -137,7 +139,7 @@ public static JavaVersion forSpecification(String specification) { } } LOG.warn( - "unsupported Java version: {}, falling back to: {}", + "Unsupported Java version: {}, falling back to: {}", specification, fallback.specification); return fallback; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/EnvironmentsTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/EnvironmentsTest.java index ae429fb1fe6d..b71a654f1031 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/EnvironmentsTest.java +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/EnvironmentsTest.java @@ -291,6 +291,8 @@ public void testLtsJavaVersion() { assertEquals("java11", JavaVersion.java11.legacyName()); assertEquals(JavaVersion.java17, JavaVersion.forSpecification("17")); assertEquals("java17", JavaVersion.java17.legacyName()); + assertEquals(JavaVersion.java21, JavaVersion.forSpecification("21")); + assertEquals("java21", JavaVersion.java21.legacyName()); } @Test @@ -303,7 +305,9 @@ public void testNonLtsJavaVersion() { assertEquals(JavaVersion.java17, JavaVersion.forSpecification("15")); assertEquals(JavaVersion.java17, JavaVersion.forSpecification("16")); assertEquals(JavaVersion.java17, JavaVersion.forSpecification("18")); - assertEquals(JavaVersion.java17, JavaVersion.forSpecification("19")); + assertEquals(JavaVersion.java21, JavaVersion.forSpecification("19")); + assertEquals(JavaVersion.java21, JavaVersion.forSpecification("20")); + assertEquals(JavaVersion.java21, JavaVersion.forSpecification("21")); } @Test(expected = UnsupportedOperationException.class) diff --git a/runners/google-cloud-dataflow-java/arm/build.gradle b/runners/google-cloud-dataflow-java/arm/build.gradle index e79eeedcd828..71cbc7c58e86 100644 --- a/runners/google-cloud-dataflow-java/arm/build.gradle +++ b/runners/google-cloud-dataflow-java/arm/build.gradle @@ -76,10 +76,8 @@ dependencies { } def javaVer = "java8" -if(project.hasProperty('compileAndRunTestsWithJava17')) { - javaVer = "java17" -} else if(project.hasProperty('compileAndRunTestsWithJava11')) { - javaVer = "java11" +if (project.hasProperty('testJavaVersion')) { + javaVer = "java${project.getProperty('testJavaVersion')}" } def dataflowProject = project.findProperty('dataflowProject') ?: 'apache-beam-testing' def dataflowRegion = project.findProperty('dataflowRegion') ?: 'us-central1' diff --git a/runners/google-cloud-dataflow-java/build.gradle b/runners/google-cloud-dataflow-java/build.gradle index adc1f2e09bc4..e4f34687d31c 100644 --- a/runners/google-cloud-dataflow-java/build.gradle +++ b/runners/google-cloud-dataflow-java/build.gradle @@ -279,10 +279,8 @@ def createRunnerV2ValidatesRunnerTest = { Map args -> // task ordering such that the registry doesn't get cleaned up prior to task completion. def buildAndPushDockerJavaContainer = tasks.register("buildAndPushDockerJavaContainer") { def javaVer = "java8" - if(project.hasProperty('compileAndRunTestsWithJava17')) { - javaVer = "java17" - } else if(project.hasProperty('compileAndRunTestsWithJava11')) { - javaVer = "java11" + if(project.hasProperty('testJavaVersion')) { + javaVer = "java${project.getProperty('testJavaVersion')}" } dependsOn ":sdks:java:container:${javaVer}:docker" def defaultDockerImageName = containerImageName( diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java index 26548038a1df..891b4c0454c9 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java @@ -398,10 +398,9 @@ && isServiceEndpoint(dataflowOptions.getDataflowEndpoint())) { // Adding the Java version to the SDK name for user's and support convenience. String agentJavaVer = "(JRE 8 environment)"; - if (Environments.getJavaVersion() == Environments.JavaVersion.java17) { - agentJavaVer = "(JRE 17 environment)"; - } else if (Environments.getJavaVersion() == Environments.JavaVersion.java11) { - agentJavaVer = "(JRE 11 environment)"; + if (Environments.getJavaVersion() != Environments.JavaVersion.java8) { + agentJavaVer = + String.format("(JRE %s environment)", Environments.getJavaVersion().specification()); } DataflowRunnerInfo dataflowRunnerInfo = DataflowRunnerInfo.getDataflowRunnerInfo(); diff --git a/runners/spark/spark_runner.gradle b/runners/spark/spark_runner.gradle index d0dbe453ddfb..74013de6107d 100644 --- a/runners/spark/spark_runner.gradle +++ b/runners/spark/spark_runner.gradle @@ -63,8 +63,9 @@ def sparkTestProperties(overrides = [:]) { def sparkTestJvmArgs() { - // run tests with Java 17 using -PcompileAndRunTestsWithJava17 -Pjava17Home=??? - if (project.hasProperty("compileAndRunTestsWithJava17")) { + // run tests with Java 17 using -PtestJavaVersion=17 -Pjava17Home=??? + if (project.hasProperty('testJavaVersion') && + project.getProperty('testJavaVersion') in ['17', '21']) { return [ "--add-opens=java.base/sun.nio.ch=ALL-UNNAMED", // add-opens below required for Kryo FieldSerializer / SparkRunnerKryoRegistratorTest diff --git a/sdks/java/container/Dockerfile b/sdks/java/container/Dockerfile index e0fa8d4a0a6f..9c266ea132b8 100644 --- a/sdks/java/container/Dockerfile +++ b/sdks/java/container/Dockerfile @@ -15,8 +15,9 @@ # See the License for the specific language governing permissions and # limitations under the License. ############################################################################### +ARG base_image ARG java_version -FROM eclipse-temurin:${java_version} +FROM ${base_image}:${java_version} LABEL Author "Apache Beam " ARG TARGETOS ARG TARGETARCH diff --git a/sdks/java/container/common.gradle b/sdks/java/container/common.gradle index cc427494ed6e..9f6cbe64a294 100644 --- a/sdks/java/container/common.gradle +++ b/sdks/java/container/common.gradle @@ -29,6 +29,7 @@ applyDockerNature() if (!project.hasProperty('imageJavaVersion')) { throw new GradleException('imageJavaVersion project property must be set') } +def javaBaseImage = project.findProperty('javaBaseImage') ?: 'eclipse-temurin' def imageJavaVersion = project.findProperty('imageJavaVersion') description = "Apache Beam :: SDKs :: Java :: Container :: Java ${imageJavaVersion} Container" @@ -71,19 +72,19 @@ task copySdkHarnessLauncher(type: Copy) { } task copyJavaThirdPartyLicenses(type: Copy) { - from("${project(':sdks:java:container').buildDir}/target/third_party_licenses") + from project(':sdks:java:container').layout.buildDirectory.dir('target/third_party_licenses') into "build/target/third_party_licenses" dependsOn ':sdks:java:container:pullLicenses' } task copyGolangLicenses(type: Copy) { - from "${project(':release:go-licenses:java').buildDir}/output" + from project(':release:go-licenses:java').layout.buildDirectory.dir('output') into "build/target/go-licenses" dependsOn ':release:go-licenses:java:createLicenses' } task copyJdkOptions(type: Copy) { - if (imageJavaVersion == "17" || imageJavaVersion == "11") { + if (["11", "17", "21"].contains(imageJavaVersion)) { from "option-jamm.json" } from "java${imageJavaVersion}-security.properties" @@ -97,10 +98,10 @@ task skipPullLicenses(type: Exec) { } task validateJavaHome { - if (imageJavaVersion == "11" || imageJavaVersion == "17") { + if (["11", "17", "21"].contains(imageJavaVersion)) { doFirst { - if (!project.hasProperty('java17Home') && !project.hasProperty('java11Home')) { - throw new GradleException('java17Home or java11Home property required. Re-run with -Pjava17Home or -Pjava11Home') + if (!project.hasProperty("java${imageJavaVersion}Home")) { + throw new GradleException("java${imageJavaVersion}Home property required for imageJavaVersion=${imageJavaVersion}. Re-run with -Pjava${imageJavaVersion}Home") } } } @@ -124,6 +125,7 @@ docker { buildArgs([ 'pull_licenses': project.rootProject.hasProperty(["docker-pull-licenses"]) || project.rootProject.hasProperty(["isRelease"]), + 'base_image': javaBaseImage, 'java_version': imageJavaVersion, ]) buildx useBuildx diff --git a/sdks/java/container/java21/build.gradle b/sdks/java/container/java21/build.gradle new file mode 100644 index 000000000000..038064102dcb --- /dev/null +++ b/sdks/java/container/java21/build.gradle @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * License); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an AS IS BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +project.ext { + // TODO(https://github.com/apache/beam/issues/28120) switch to temurin once available + javaBaseImage = 'openjdk' + imageJavaVersion = '21' +} + +// Load the main build script which contains all build logic. +apply from: "../common.gradle" + +dependencies { + dockerDependency project(path: ":sdks:java:container:agent") +} \ No newline at end of file diff --git a/sdks/java/container/java21/option-jamm.json b/sdks/java/container/java21/option-jamm.json new file mode 100644 index 000000000000..5647ff66be5c --- /dev/null +++ b/sdks/java/container/java21/option-jamm.json @@ -0,0 +1,12 @@ +{ + "name": "jamm", + "enabled": true, + "options": { + "java_arguments": [ + "--add-modules=jamm", + "--module-path=/opt/apache/beam/jars/jamm.jar", + "--add-opens=java.base/java.lang=jamm", + "--add-opens=java.base/java.util=jamm" + ] + } +} \ No newline at end of file diff --git a/sdks/java/testing/jpms-tests/build.gradle b/sdks/java/testing/jpms-tests/build.gradle index 6321f874c903..2a25463931c1 100644 --- a/sdks/java/testing/jpms-tests/build.gradle +++ b/sdks/java/testing/jpms-tests/build.gradle @@ -23,10 +23,8 @@ plugins { } // overwrite javaVersion before applyJavaNature -if (project.hasProperty("compileAndRunTestsWithJava17")) { - javaVersion = '1.17' -} else { - javaVersion = '1.11' +if (project.hasProperty("testJavaVersion")) { + javaVersion = "1.${project.getProperty('testJavaVersion')}" as String } applyJavaNature( @@ -42,13 +40,14 @@ ext.summary = "E2E test for Java 9 modules" // direct compileJava to use specified java version. project.tasks.compileJava { - if (project.hasProperty("compileAndRunTestsWithJava11")) { + if (project.hasProperty('testJavaVersion')) { options.fork = true - options.forkOptions.javaHome = project.findProperty("java11Home") as File - } else if (project.hasProperty("compileAndRunTestsWithJava17")) { - options.fork = true - options.forkOptions.javaHome = project.findProperty("java17Home") as File - setJava17Options(options) + options.forkOptions.javaHome = project.findProperty("java${project.getProperty('testJavaVersion')}Home") as File + if (project.getProperty('testJavaVersion') == '17') { + setJava17Options(options) + } else if (project.getProperty('testJavaVersion') == '21') { + setJava21Options(options) + } } } @@ -120,7 +119,7 @@ plugins.withType(JavaPlugin).configureEach{ // JPMS requires JDK > 8 project.tasks.each { it.onlyIf { - project.hasProperty("compileAndRunTestsWithJava17") + project.hasProperty('testJavaVersion') || JavaVersion.VERSION_1_8.compareTo(JavaVersion.current()) < 0 } } diff --git a/sdks/java/testing/test-utils/build.gradle b/sdks/java/testing/test-utils/build.gradle index 50c815dd57f7..6e30693d8894 100644 --- a/sdks/java/testing/test-utils/build.gradle +++ b/sdks/java/testing/test-utils/build.gradle @@ -43,24 +43,15 @@ dependencies { testRuntimeOnly project(path: ":runners:direct-java", configuration: "shadowTest") } -task verifyJavaVersion(type: Test) { - filter { - includeTestsMatching 'org.apache.beam.sdk.testutils.jvmverification.JvmVerification.verifyCodeIsCompiledWithJava8' - includeTestsMatching 'org.apache.beam.sdk.testutils.jvmverification.JvmVerification.verifyTestCodeIsCompiledWithJava11' - includeTestsMatching 'org.apache.beam.sdk.testutils.jvmverification.JvmVerification.verifyRunningJVMVersionIs11' - } - doLast { - println 'Java verified' +['11', '17', '21'].each { + tasks.create(name: "verifyJavaVersion${it}", type: Test) { + filter { + includeTestsMatching "org.apache.beam.sdk.testutils.jvmverification.JvmVerification.verifyCodeIsCompiledWithJava8" + includeTestsMatching "org.apache.beam.sdk.testutils.jvmverification.JvmVerification.verifyTestCodeIsCompiledWithJava${it}" + includeTestsMatching "org.apache.beam.sdk.testutils.jvmverification.JvmVerification.verifyRunningJVMVersionIs${it}" + } + doLast { + println 'Java verified' + } } } - -task verifyJavaVersion17(type: Test) { - filter { - includeTestsMatching 'org.apache.beam.sdk.testutils.jvmverification.JvmVerification.verifyCodeIsCompiledWithJava8' - includeTestsMatching 'org.apache.beam.sdk.testutils.jvmverification.JvmVerification.verifyTestCodeIsCompiledWithJava17' - includeTestsMatching 'org.apache.beam.sdk.testutils.jvmverification.JvmVerification.verifyRunningJVMVersionIs17' - } - doLast { - println 'Java verified' - } -} \ No newline at end of file diff --git a/sdks/java/testing/test-utils/src/test/java/org/apache/beam/sdk/testutils/jvmverification/JvmVerification.java b/sdks/java/testing/test-utils/src/test/java/org/apache/beam/sdk/testutils/jvmverification/JvmVerification.java index ad29e8b6a1d6..a6b5d6dca6c1 100644 --- a/sdks/java/testing/test-utils/src/test/java/org/apache/beam/sdk/testutils/jvmverification/JvmVerification.java +++ b/sdks/java/testing/test-utils/src/test/java/org/apache/beam/sdk/testutils/jvmverification/JvmVerification.java @@ -20,6 +20,7 @@ import static org.apache.beam.sdk.testutils.jvmverification.JvmVerification.Java.v11; import static org.apache.beam.sdk.testutils.jvmverification.JvmVerification.Java.v17; import static org.apache.beam.sdk.testutils.jvmverification.JvmVerification.Java.v1_8; +import static org.apache.beam.sdk.testutils.jvmverification.JvmVerification.Java.v21; import static org.junit.Assert.assertEquals; import java.io.IOException; @@ -39,6 +40,7 @@ public class JvmVerification { versionMapping.put("0034", v1_8); versionMapping.put("0037", v11); versionMapping.put("003d", v17); + versionMapping.put("0041", v21); } // bytecode @@ -62,6 +64,11 @@ public void verifyTestCodeIsCompiledWithJava17() throws IOException { assertEquals(v17, getByteCodeVersion(JvmVerification.class)); } + @Test + public void verifyTestCodeIsCompiledWithJava21() throws IOException { + assertEquals(v21, getByteCodeVersion(JvmVerification.class)); + } + // jvm @Test public void verifyRunningJVMVersionIs11() { @@ -75,6 +82,12 @@ public void verifyRunningJVMVersionIs17() { assertEquals(v17.name, version); } + @Test + public void verifyRunningJVMVersionIs21() { + final String version = getJavaSpecification(); + assertEquals(v21.name, version); + } + private static Java getByteCodeVersion(final Class clazz) throws IOException { final InputStream stream = clazz.getClassLoader().getResourceAsStream(clazz.getName().replace(".", "/") + ".class"); @@ -91,7 +104,8 @@ private static String getJavaSpecification() { enum Java { v1_8("1.8"), v11("11"), - v17("17"); + v17("17"), + v21("21"); final String name; diff --git a/settings.gradle.kts b/settings.gradle.kts index c370c5da27d1..d1069ec2d352 100644 --- a/settings.gradle.kts +++ b/settings.gradle.kts @@ -175,6 +175,7 @@ include(":sdks:java:container:agent") include(":sdks:java:container:java8") include(":sdks:java:container:java11") include(":sdks:java:container:java17") +include(":sdks:java:container:java21") include(":sdks:java:core") include(":sdks:java:core:jmh") include(":sdks:java:expansion-service") diff --git a/website/www/site/content/en/roadmap/java-sdk.md b/website/www/site/content/en/roadmap/java-sdk.md index b65424b57a3d..a1c85e139193 100644 --- a/website/www/site/content/en/roadmap/java-sdk.md +++ b/website/www/site/content/en/roadmap/java-sdk.md @@ -17,9 +17,9 @@ limitations under the License. # Java SDK Roadmap -## Next Java LTS version support (Java 17) +## Next Java LTS version support (Java 21) Work to support the next LTS release of Java is in progress. For more details -about the scope and info on the various tasks please see the JIRA ticket. +about the scope and info on the various tasks please see the GitHub Issue. -- JIRA: [BEAM-12240](https://issues.apache.org/jira/browse/BEAM-12240) +- GitHub: [#28120](https://github.com/apache/beam/issues/28120) From 9c75db4760ca3440a0a2ed12f031d84bc011aa96 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Wed, 11 Oct 2023 18:19:29 -0700 Subject: [PATCH 115/435] [YAML] Avro format for PubSub. (#28899) --- sdks/python/apache_beam/yaml/yaml_io.py | 21 ++++++ sdks/python/apache_beam/yaml/yaml_io_test.py | 68 ++++++++++++++++++++ 2 files changed, 89 insertions(+) diff --git a/sdks/python/apache_beam/yaml/yaml_io.py b/sdks/python/apache_beam/yaml/yaml_io.py index 3321644ded57..b2bf150fa558 100644 --- a/sdks/python/apache_beam/yaml/yaml_io.py +++ b/sdks/python/apache_beam/yaml/yaml_io.py @@ -23,6 +23,7 @@ implementations of the same transforms, the configs must be kept in sync. """ +import io import os from typing import Any from typing import Callable @@ -32,12 +33,14 @@ from typing import Optional from typing import Tuple +import fastavro import yaml import apache_beam as beam import apache_beam.io as beam_io from apache_beam.io import ReadFromBigQuery from apache_beam.io import WriteToBigQuery +from apache_beam.io import avroio from apache_beam.io.gcp.bigquery import BigQueryDisposition from apache_beam.portability.api import schema_pb2 from apache_beam.typehints import schemas @@ -146,6 +149,13 @@ def _create_parser( elif format == 'json': beam_schema = json_utils.json_schema_to_beam_schema(schema) return beam_schema, json_utils.json_parser(beam_schema) + elif format == 'avro': + beam_schema = avroio.avro_schema_to_beam_schema(schema) + covert_to_row = avroio.avro_dict_to_beam_row(schema, beam_schema) + return ( + beam_schema, + lambda record: covert_to_row( + fastavro.schemaless_reader(io.BytesIO(record), schema))) else: raise ValueError(f'Unknown format: {format}') @@ -162,6 +172,17 @@ def _create_formatter( return lambda row: getattr(row, field_names[0]) elif format == 'json': return json_utils.json_formater(beam_schema) + elif format == 'avro': + avro_schema = schema or avroio.beam_schema_to_avro_schema(beam_schema) + from_row = avroio.beam_row_to_avro_dict(avro_schema, beam_schema) + + def formatter(row): + buffer = io.BytesIO() + fastavro.schemaless_writer(buffer, avro_schema, from_row(row)) + buffer.seek(0) + return buffer.read() + + return formatter else: raise ValueError(f'Unknown format: {format}') diff --git a/sdks/python/apache_beam/yaml/yaml_io_test.py b/sdks/python/apache_beam/yaml/yaml_io_test.py index 72675da278b0..7071860a7bf1 100644 --- a/sdks/python/apache_beam/yaml/yaml_io_test.py +++ b/sdks/python/apache_beam/yaml/yaml_io_test.py @@ -15,9 +15,12 @@ # limitations under the License. # +import io +import json import logging import unittest +import fastavro import mock import apache_beam as beam @@ -167,6 +170,48 @@ def test_read_with_id_attribute(self): result, equal_to([beam.Row(payload=b'msg1'), beam.Row(payload=b'msg2')])) + _avro_schema = { + 'type': 'record', + 'name': 'ec', + 'fields': [{ + 'name': 'label', 'type': 'string' + }, { + 'name': 'rank', 'type': 'int' + }] + } + + def _encode_avro(self, data): + buffer = io.BytesIO() + fastavro.schemaless_writer(buffer, self._avro_schema, data) + buffer.seek(0) + return buffer.read() + + def test_read_avro(self): + + with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions( + pickle_library='cloudpickle')) as p: + with mock.patch( + 'apache_beam.io.ReadFromPubSub', + FakeReadFromPubSub( + topic='my_topic', + messages=[PubsubMessage(self._encode_avro({'label': '37a', + 'rank': 1}), {}), + PubsubMessage(self._encode_avro({'label': '389a', + 'rank': 2}), {})])): + result = p | YamlTransform( + ''' + type: ReadFromPubSub + config: + topic: my_topic + format: avro + schema: %s + ''' % json.dumps(self._avro_schema)) + assert_that( + result, + equal_to( + [beam.Row(label='37a', rank=1), # linebreak + beam.Row(label='389a', rank=2)])) + def test_read_json(self): with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions( pickle_library='cloudpickle')) as p: @@ -346,6 +391,29 @@ def test_write_with_id_attribute(self): id_attribute: some_attr ''')) + def test_write_avro(self): + with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions( + pickle_library='cloudpickle')) as p: + with mock.patch( + 'apache_beam.io.WriteToPubSub', + FakeWriteToPubSub( + topic='my_topic', + messages=[PubsubMessage(self._encode_avro({'label': '37a', + 'rank': 1}), {}), + PubsubMessage(self._encode_avro({'label': '389a', + 'rank': 2}), {})])): + _ = ( + p | beam.Create( + [beam.Row(label='37a', rank=1), beam.Row(label='389a', rank=2)]) + | YamlTransform( + ''' + type: WriteToPubSub + input: input + config: + topic: my_topic + format: avro + ''')) + def test_write_json(self): with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions( pickle_library='cloudpickle')) as p: From fb99630d1e0924eec54239ee369b2042b15d5187 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 3 Oct 2023 13:28:55 -0400 Subject: [PATCH 116/435] [Website] Announce Beam 2.51.0 --- CHANGES.md | 18 +- website/www/site/config.toml | 2 +- .../www/site/content/en/blog/beam-2.51.0.md | 210 ++++++++++++++++++ .../site/content/en/get-started/downloads.md | 14 +- 4 files changed, 226 insertions(+), 18 deletions(-) create mode 100644 website/www/site/content/en/blog/beam-2.51.0.md diff --git a/CHANGES.md b/CHANGES.md index 0c2c2e3f79f4..43ade8c530b9 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -87,16 +87,7 @@ * ([#X](https://github.com/apache/beam/issues/X)). -# [2.51.0] - Unreleased - -## Highlights - -* New highly anticipated feature X added to Python SDK ([#X](https://github.com/apache/beam/issues/X)). -* New highly anticipated feature Y added to Java SDK ([#Y](https://github.com/apache/beam/issues/Y)). - -## I/Os - -* Support for X source added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). +# [2.51.0] - 2023-10-03 ## New Features / Improvements @@ -104,6 +95,7 @@ * In Python, the [VertexAIModelHandlerJSON](https://beam.apache.org/releases/pydoc/current/apache_beam.ml.inference.vertex_ai_inference.html#apache_beam.ml.inference.vertex_ai_inference.VertexAIModelHandlerJSON) now supports passing in inference_args. These will be passed through to the Vertex endpoint as parameters. * Added support to run `mypy` on user pipelines ([#27906](https://github.com/apache/beam/issues/27906)) + ## Breaking Changes * Removed fastjson library dependency for Beam SQL. Table property is changed to be based on jackson ObjectNode (Java) ([#24154](https://github.com/apache/beam/issues/24154)). @@ -111,9 +103,6 @@ * Removed the parameter `t reflect.Type` from `parquetio.Write`. The element type is derived from the input PCollection (Go) ([#28490](https://github.com/apache/beam/issues/28490)) * Refactor BeamSqlSeekableTable.setUp adding a parameter joinSubsetType. [#28283](https://github.com/apache/beam/issues/28283) -## Deprecations - -* X behavior is deprecated and will be removed in X versions ([#X](https://github.com/apache/beam/issues/X)). ## Bugfixes @@ -128,7 +117,8 @@ ## Known Issues -* ([#X](https://github.com/apache/beam/issues/X)). +* Python pipelines using BigQuery Storage Read API must pin `fastavro` + dependency to 1.8.3 or earlier: [#28811](https://github.com/apache/beam/issues/28811) # [2.50.0] - 2023-08-30 diff --git a/website/www/site/config.toml b/website/www/site/config.toml index 6a1907e60591..c7b0cd3412e6 100644 --- a/website/www/site/config.toml +++ b/website/www/site/config.toml @@ -104,7 +104,7 @@ github_project_repo = "https://github.com/apache/beam" [params] description = "Apache Beam is an open source, unified model and set of language-specific SDKs for defining and executing data processing workflows, and also data ingestion and integration flows, supporting Enterprise Integration Patterns (EIPs) and Domain Specific Languages (DSLs). Dataflow pipelines simplify the mechanics of large-scale batch and streaming data processing and can run on a number of runtimes like Apache Flink, Apache Spark, and Google Cloud Dataflow (a cloud service). Beam also brings DSL in different languages, allowing users to easily implement their data integration processes." -release_latest = "2.50.0" +release_latest = "2.51.0" # The repository and branch where the files live in Github or Colab. This is used # to serve and stage from your local branch, but publish to the master branch. # e.g. https://github.com/{{< param branch_repo >}}/path/to/notebook.ipynb diff --git a/website/www/site/content/en/blog/beam-2.51.0.md b/website/www/site/content/en/blog/beam-2.51.0.md new file mode 100644 index 000000000000..aaa4142bae62 --- /dev/null +++ b/website/www/site/content/en/blog/beam-2.51.0.md @@ -0,0 +1,210 @@ +--- +title: "Apache Beam 2.51.0" +date: 2023-10-11 09:00:00 -0400 +categories: + - blog + - release +authors: + - klk +--- + + +We are happy to present the new 2.51.0 release of Beam. +This release includes both improvements and new functionality. +See the [download page](/get-started/downloads/#2510-2023-10-03) for this release. + + + +For more information on changes in 2.51.0, check out the [detailed release notes](https://github.com/apache/beam/milestone/15). + +## New Features / Improvements + +* In Python, [RunInference](https://beam.apache.org/documentation/sdks/python-machine-learning/#why-use-the-runinference-api) now supports loading many models in the same transform using a [KeyedModelHandler](https://beam.apache.org/documentation/sdks/python-machine-learning/#use-a-keyed-modelhandler) ([#27628](https://github.com/apache/beam/issues/27628)). +* In Python, the [VertexAIModelHandlerJSON](https://beam.apache.org/releases/pydoc/current/apache_beam.ml.inference.vertex_ai_inference.html#apache_beam.ml.inference.vertex_ai_inference.VertexAIModelHandlerJSON) now supports passing in inference_args. These will be passed through to the Vertex endpoint as parameters. +* Added support to run `mypy` on user pipelines ([#27906](https://github.com/apache/beam/issues/27906)) + + +## Breaking Changes + +* Removed fastjson library dependency for Beam SQL. Table property is changed to be based on jackson ObjectNode (Java) ([#24154](https://github.com/apache/beam/issues/24154)). +* Removed TensorFlow from Beam Python container images [PR](https://github.com/apache/beam/pull/28424). If you have been negatively affected by this change, please comment on [#20605](https://github.com/apache/beam/issues/20605). +* Removed the parameter `t reflect.Type` from `parquetio.Write`. The element type is derived from the input PCollection (Go) ([#28490](https://github.com/apache/beam/issues/28490)) +* Refactor BeamSqlSeekableTable.setUp adding a parameter joinSubsetType. [#28283](https://github.com/apache/beam/issues/28283) + + +## Bugfixes + +* Fixed exception chaining issue in GCS connector (Python) ([#26769](https://github.com/apache/beam/issues/26769#issuecomment-1700422615)). +* Fixed streaming inserts exception handling, GoogleAPICallErrors are now retried according to retry strategy and routed to failed rows where appropriate rather than causing a pipeline error (Python) ([#21080](https://github.com/apache/beam/issues/21080)). +* Fixed a bug in Python SDK's cross-language Bigtable sink that mishandled records that don't have an explicit timestamp set: [#28632](https://github.com/apache/beam/issues/28632). + + +## Security Fixes +* Python containers updated, fixing [CVE-2021-30474](https://nvd.nist.gov/vuln/detail/CVE-2021-30474), [CVE-2021-30475](https://nvd.nist.gov/vuln/detail/CVE-2021-30475), [CVE-2021-30473](https://nvd.nist.gov/vuln/detail/CVE-2021-30473), [CVE-2020-36133](https://nvd.nist.gov/vuln/detail/CVE-2020-36133), [CVE-2020-36131](https://nvd.nist.gov/vuln/detail/CVE-2020-36131), [CVE-2020-36130](https://nvd.nist.gov/vuln/detail/CVE-2020-36130), and [CVE-2020-36135](https://nvd.nist.gov/vuln/detail/CVE-2020-36135) +* Used go 1.21.1 to build, fixing [CVE-2023-39320](https://security-tracker.debian.org/tracker/CVE-2023-39320) + + +## Known Issues + +* Python pipelines using BigQuery Storage Read API must pin `fastavro` dependency to 1.8.3 + or earlier: [#28811](https://github.com/apache/beam/issues/28811) + +## List of Contributors + +According to git shortlog, the following people contributed to the 2.50.0 release. Thank you to all contributors! + +Adam Whitmore + +Ahmed Abualsaud + +Ahmet Altay + +Aleksandr Dudko + +Alexey Romanenko + +Anand Inguva + +Andrey Devyatkin + +Arvind Ram + +Arwin Tio + +BjornPrime + +Bruno Volpato + +Bulat + +Celeste Zeng + +Chamikara Jayalath + +Clay Johnson + +Damon + +Danny McCormick + +David Cavazos + +Dip Patel + +Hai Joey Tran + +Hao Xu + +Haruka Abe + +Jack Dingilian + +Jack McCluskey + +Jeff Kinard + +Jeffrey Kinard + +Joey Tran + +Johanna Öjeling + +Julien Tournay + +Kenneth Knowles + +Kerry Donny-Clark + +Mattie Fu + +Melissa Pashniak + +Michel Davit + +Moritz Mack + +Pranav Bhandari + +Rebecca Szper + +Reeba Qureshi + +Reuven Lax + +Ritesh Ghorse + +Robert Bradshaw + +Robert Burke + +Ruwann + +Ryan Tam + +Sam Rohde + +Sereana Seim + +Svetak Sundhar + +Tim Grein + +Udi Meiri + +Valentyn Tymofieiev + +Vitaly Terentyev + +Vlado Djerek + +Xinyu Liu + +Yi Hu + +Zbynek Konecny + +Zechen Jiang + +bzablocki + +caneff + +dependabot[bot] + +gDuperran + +gabry.wu + +johnjcasey + +kberezin-nshl + +kennknowles + +liferoad + +lostluck + +magicgoody + +martin trieu + +mosche + +olalamichelle + +tvalentyn + +xqhu + +Łukasz Spyra diff --git a/website/www/site/content/en/get-started/downloads.md b/website/www/site/content/en/get-started/downloads.md index 9a753dafe32e..b564a5801cd8 100644 --- a/website/www/site/content/en/get-started/downloads.md +++ b/website/www/site/content/en/get-started/downloads.md @@ -96,10 +96,18 @@ versions denoted `0.x.y`. ## Releases +### 2.51.0 (2023-10-11) +Official [source code download](https://downloads.apache.org/beam/2.51.0/apache-beam-2.51.0-source-release.zip). +[SHA-512](https://downloads.apache.org/beam/2.51.0/apache-beam-2.51.0-source-release.zip.sha512). +[signature](https://downloads.apache.org/beam/2.51.0/apache-beam-2.51.0-source-release.zip.asc). + +[Release notes](https://github.com/apache/beam/releases/tag/v2.51.0) +[Blog post](/blog/beam-2.51.0). + ### 2.50.0 (2023-08-30) -Official [source code download](https://downloads.apache.org/beam/2.50.0/apache-beam-2.50.0-source-release.zip). -[SHA-512](https://downloads.apache.org/beam/2.50.0/apache-beam-2.50.0-source-release.zip.sha512). -[signature](https://downloads.apache.org/beam/2.50.0/apache-beam-2.50.0-source-release.zip.asc). +Official [source code download](https://archive.apache.org/beam/2.50.0/apache-beam-2.50.0-source-release.zip). +[SHA-512](https://archive.apache.org/beam/2.50.0/apache-beam-2.50.0-source-release.zip.sha512). +[signature](https://archive.apache.org/beam/2.50.0/apache-beam-2.50.0-source-release.zip.asc). [Release notes](https://github.com/apache/beam/releases/tag/v2.50.0) [Blog post](/blog/beam-2.50.0). From 6b3b3eb603fbd6feccf72a093bd520c5ca973cb5 Mon Sep 17 00:00:00 2001 From: Robert Burke Date: Wed, 11 Oct 2023 19:43:06 -0700 Subject: [PATCH 117/435] Update go_tests.yml to cache (#28954) Configure caching go deps properly in Go Tests action. --- .github/workflows/go_tests.yml | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/.github/workflows/go_tests.yml b/.github/workflows/go_tests.yml index 6884dd692522..66c57d219a83 100644 --- a/.github/workflows/go_tests.yml +++ b/.github/workflows/go_tests.yml @@ -46,6 +46,8 @@ jobs: - uses: actions/setup-go@v4 with: go-version: '1.21' + cache-dependency-path: | + sdks/go.sum - name: Delete old coverage run: "cd sdks && rm -rf .coverage.txt || :" - name: Run coverage @@ -70,4 +72,4 @@ jobs: with: work-dir: ./sdks go-package: ./... - go-version-input: 1.21 \ No newline at end of file + go-version-input: 1.21 From 4a7c4842aceb6786040d5cd699b7916c4509bb7b Mon Sep 17 00:00:00 2001 From: Kerry Donny-Clark Date: Wed, 11 Oct 2023 23:35:25 -0400 Subject: [PATCH 118/435] Add new resource hint to all sdks for number of cpus per worker machine (#28848) * Adds new resource hint for number of cpus per worker. * Fixes minor bugs.` * Go fmt and removes unwanted .python-version file. * Adds tests. * Fixes typo. * Fixes Java tests, adds URN to parsers. * Addresses FindBugs issue with int parsing. * Applies Java formatting corrections * Adds generated go protobufs --- .../model/pipeline/v1/beam_runner_api.proto | 4 + .../model/fnexecution_v1/beam_fn_api.pb.go | 67 ++++++++++----- .../fnexecution_v1/beam_fn_api_grpc.pb.go | 2 +- .../fnexecution_v1/beam_provision_api.pb.go | 2 +- .../beam_provision_api_grpc.pb.go | 2 +- .../jobmanagement_v1/beam_artifact_api.pb.go | 2 +- .../beam_artifact_api_grpc.pb.go | 2 +- .../jobmanagement_v1/beam_expansion_api.pb.go | 2 +- .../beam_expansion_api_grpc.pb.go | 2 +- .../model/jobmanagement_v1/beam_job_api.pb.go | 2 +- .../jobmanagement_v1/beam_job_api_grpc.pb.go | 2 +- .../model/pipeline_v1/beam_runner_api.pb.go | 81 ++++++++++--------- .../pipeline_v1/beam_runner_api_grpc.pb.go | 2 +- .../beam/model/pipeline_v1/endpoints.pb.go | 2 +- .../pipeline_v1/external_transforms.pb.go | 2 +- .../pkg/beam/model/pipeline_v1/metrics.pb.go | 2 +- .../pkg/beam/model/pipeline_v1/schema.pb.go | 2 +- .../pipeline_v1/standard_window_fns.pb.go | 2 +- sdks/go/pkg/beam/options/resource/hint.go | 37 +++++++++ .../go/pkg/beam/options/resource/hint_test.go | 45 ++++++++++- .../resourcehints/ResourceHints.java | 63 +++++++++++++++ .../resourcehints/ResourceHintsTest.java | 11 ++- .../apache_beam/transforms/resources.py | 16 ++++ .../apache_beam/transforms/resources_test.py | 6 ++ 24 files changed, 281 insertions(+), 79 deletions(-) diff --git a/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/beam_runner_api.proto b/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/beam_runner_api.proto index 2483103b5794..db958f183c45 100644 --- a/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/beam_runner_api.proto +++ b/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/beam_runner_api.proto @@ -1982,5 +1982,9 @@ message StandardResourceHints { // SDKs should convert the size to bytes, but can allow users to specify human-friendly units (e.g. GiB). // Payload: ASCII encoded string of the base 10 representation of an integer number of bytes. MIN_RAM_BYTES = 1 [(beam_urn) = "beam:resources:min_ram_bytes:v1"]; + // Describes desired number of CPUs available in transform's execution environment. + // SDKs should accept and validate a positive integer count. + // Payload: ASCII encoded string of the base 10 representation of an integer number of CPUs. + CPU_COUNT = 2 [(beam_urn) = "beam:resources:cpu_count:v1"]; } } diff --git a/sdks/go/pkg/beam/model/fnexecution_v1/beam_fn_api.pb.go b/sdks/go/pkg/beam/model/fnexecution_v1/beam_fn_api.pb.go index 1d547470ea1a..9d14cff3c7d6 100644 --- a/sdks/go/pkg/beam/model/fnexecution_v1/beam_fn_api.pb.go +++ b/sdks/go/pkg/beam/model/fnexecution_v1/beam_fn_api.pb.go @@ -27,7 +27,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: // protoc-gen-go v1.27.1 -// protoc v4.24.0--rc1 +// protoc v4.24.4 // source: org/apache/beam/model/fn_execution/v1/beam_fn_api.proto // TODO: Consider consolidating common components in another package @@ -1883,30 +1883,57 @@ func (x *ProcessBundleSplitRequest) GetDesiredSplits() map[string]*ProcessBundle // first_residual_element. // - The current bundle, if no further splits happen, will have done exactly // the work under primary_roots and all elements up to and including the -// channel splits last_primary_element. +// channel split's last_primary_element. // // This allows the SDK to relinquish ownership of and commit to not process some // of the elements that it may have been sent (the residual) while retaining // ownership and commitment to finish the other portion (the primary). // -// For example, lets say the SDK is processing elements A B C D E and a split -// request comes in. The SDK could return a response with a channel split -// representing a last_primary_element of 3 (D) and first_residual_element of 4 -// (E). The SDK is now responsible for processing A B C D and the runner must -// process E in the future. A future split request could have the SDK split the -// elements B into B1 and B2 and C into C1 and C2 representing their primary and -// residual roots. The SDK would return a response with a channel split -// representing a last_primary_element of 0 (A) and first_residual_element of 3 -// (D) with primary_roots (B1, C1) and residual_roots (B2, C2). The SDK is now -// responsible for processing A B1 C1 and the runner must process C2 D2 (and E -// from the prior split) in the future. Yet another future split request could -// have the SDK could split B1 further into B1a and B1b primary and residuals -// and return C2 as a residual (assuming C2 was left unprocessed). The SDK would -// return a response with a channel split representing a last_primary_element of -// 0 (A) and first_residual_element of 4 (E) with primary_roots (B1a) and -// residual_roots (B1b, C1). The SDK is now responsible for processing A B1a the -// runner must process B1b C1 (in addition to C2, D, E from prior splits) in the -// future. +// Example with three splits of a single bundle: +// Let's say the SDK is processing elements [A B C D E]. These elements make +// up the 0-indexed channel. +// +// ** First Split ** +// Channel Split = [ A B C D <> E ] +// Primary Roots = [] (No elements were split) +// Residual Roots = [] +// +// Say a split request comes in. The SDK could return a response with a channel +// split representing a last_primary_element of 3 (D) and +// first_residual_element of 4 (E). The SDK is now responsible for processing A +// B C D and the runner must process E in the future. +// +// (A B C D) | (E) +// +// ** Second Split ** +// Channel Split = [ A < B C > D E ] +// Primary Roots = [B1 C1] +// Residual Roots = [B2 C2] +// +// A future split request could have the SDK split the elements B into B1 and +// B2 and C into C1 and C2 representing their primary and residual roots. The +// +// (A B1 C1) | (B2 C2 D) +// +// SDK would return a response with a channel split representing a +// last_primary_element of 0 (A) and first_residual_element of 3 (D) with +// primary_roots (B1, C1) and residual_roots (B2, C2). The SDK is now +// responsible for processing A B1 C1 and the runner must process B2 C2 D (and +// E from the prior split) in the future. +// +// ** Third Split ** +// Channel Split = [ A < B C > D E ] +// Primary Roots = [B1a] +// Residual Roots [B1b C1] +// Yet another future split request could have the SDK could split B1 further +// into B1a and B1b primary and residuals and return C1 as a residual (assuming +// C1 was left unprocessed). The SDK would return a response with a channel +// split representing a last_primary_element of 0 (A) and +// first_residual_element of 3 (E) with primary_roots (B1a) and residual_roots +// (B1b, C1). The SDK is now responsible for processing A B1a the runner must +// process B1b C1 (in addition to C2, D, E from prior splits) in the future. +// +// (A B1a) | (B1b C1) // // For more rigorous definitions see https://s.apache.org/beam-breaking-fusion type ProcessBundleSplitResponse struct { diff --git a/sdks/go/pkg/beam/model/fnexecution_v1/beam_fn_api_grpc.pb.go b/sdks/go/pkg/beam/model/fnexecution_v1/beam_fn_api_grpc.pb.go index ac9e402750c4..cd53ea805705 100644 --- a/sdks/go/pkg/beam/model/fnexecution_v1/beam_fn_api_grpc.pb.go +++ b/sdks/go/pkg/beam/model/fnexecution_v1/beam_fn_api_grpc.pb.go @@ -17,7 +17,7 @@ // Code generated by protoc-gen-go-grpc. DO NOT EDIT. // versions: // - protoc-gen-go-grpc v1.1.0 -// - protoc v4.24.0--rc1 +// - protoc v4.24.4 // source: org/apache/beam/model/fn_execution/v1/beam_fn_api.proto package fnexecution_v1 diff --git a/sdks/go/pkg/beam/model/fnexecution_v1/beam_provision_api.pb.go b/sdks/go/pkg/beam/model/fnexecution_v1/beam_provision_api.pb.go index a24609b2fd05..26cf245f7206 100644 --- a/sdks/go/pkg/beam/model/fnexecution_v1/beam_provision_api.pb.go +++ b/sdks/go/pkg/beam/model/fnexecution_v1/beam_provision_api.pb.go @@ -22,7 +22,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: // protoc-gen-go v1.27.1 -// protoc v4.24.0--rc1 +// protoc v4.24.4 // source: org/apache/beam/model/fn_execution/v1/beam_provision_api.proto package fnexecution_v1 diff --git a/sdks/go/pkg/beam/model/fnexecution_v1/beam_provision_api_grpc.pb.go b/sdks/go/pkg/beam/model/fnexecution_v1/beam_provision_api_grpc.pb.go index f9c6f5681399..9064b348b4c0 100644 --- a/sdks/go/pkg/beam/model/fnexecution_v1/beam_provision_api_grpc.pb.go +++ b/sdks/go/pkg/beam/model/fnexecution_v1/beam_provision_api_grpc.pb.go @@ -17,7 +17,7 @@ // Code generated by protoc-gen-go-grpc. DO NOT EDIT. // versions: // - protoc-gen-go-grpc v1.1.0 -// - protoc v4.24.0--rc1 +// - protoc v4.24.4 // source: org/apache/beam/model/fn_execution/v1/beam_provision_api.proto package fnexecution_v1 diff --git a/sdks/go/pkg/beam/model/jobmanagement_v1/beam_artifact_api.pb.go b/sdks/go/pkg/beam/model/jobmanagement_v1/beam_artifact_api.pb.go index 6a7663d77e9c..85bb2e368970 100644 --- a/sdks/go/pkg/beam/model/jobmanagement_v1/beam_artifact_api.pb.go +++ b/sdks/go/pkg/beam/model/jobmanagement_v1/beam_artifact_api.pb.go @@ -22,7 +22,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: // protoc-gen-go v1.27.1 -// protoc v4.24.0--rc1 +// protoc v4.24.4 // source: org/apache/beam/model/job_management/v1/beam_artifact_api.proto package jobmanagement_v1 diff --git a/sdks/go/pkg/beam/model/jobmanagement_v1/beam_artifact_api_grpc.pb.go b/sdks/go/pkg/beam/model/jobmanagement_v1/beam_artifact_api_grpc.pb.go index 6b381b96f3d1..28e43e21fbbd 100644 --- a/sdks/go/pkg/beam/model/jobmanagement_v1/beam_artifact_api_grpc.pb.go +++ b/sdks/go/pkg/beam/model/jobmanagement_v1/beam_artifact_api_grpc.pb.go @@ -17,7 +17,7 @@ // Code generated by protoc-gen-go-grpc. DO NOT EDIT. // versions: // - protoc-gen-go-grpc v1.1.0 -// - protoc v4.24.0--rc1 +// - protoc v4.24.4 // source: org/apache/beam/model/job_management/v1/beam_artifact_api.proto package jobmanagement_v1 diff --git a/sdks/go/pkg/beam/model/jobmanagement_v1/beam_expansion_api.pb.go b/sdks/go/pkg/beam/model/jobmanagement_v1/beam_expansion_api.pb.go index 0f33c7ab9e3c..8f7ca43ec0f5 100644 --- a/sdks/go/pkg/beam/model/jobmanagement_v1/beam_expansion_api.pb.go +++ b/sdks/go/pkg/beam/model/jobmanagement_v1/beam_expansion_api.pb.go @@ -22,7 +22,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: // protoc-gen-go v1.27.1 -// protoc v4.24.0--rc1 +// protoc v4.24.4 // source: org/apache/beam/model/job_management/v1/beam_expansion_api.proto package jobmanagement_v1 diff --git a/sdks/go/pkg/beam/model/jobmanagement_v1/beam_expansion_api_grpc.pb.go b/sdks/go/pkg/beam/model/jobmanagement_v1/beam_expansion_api_grpc.pb.go index e2cc3c4f77ec..f1c3782f5fb8 100644 --- a/sdks/go/pkg/beam/model/jobmanagement_v1/beam_expansion_api_grpc.pb.go +++ b/sdks/go/pkg/beam/model/jobmanagement_v1/beam_expansion_api_grpc.pb.go @@ -17,7 +17,7 @@ // Code generated by protoc-gen-go-grpc. DO NOT EDIT. // versions: // - protoc-gen-go-grpc v1.1.0 -// - protoc v4.24.0--rc1 +// - protoc v4.24.4 // source: org/apache/beam/model/job_management/v1/beam_expansion_api.proto package jobmanagement_v1 diff --git a/sdks/go/pkg/beam/model/jobmanagement_v1/beam_job_api.pb.go b/sdks/go/pkg/beam/model/jobmanagement_v1/beam_job_api.pb.go index d93130d26d9f..62e0b313ec2d 100644 --- a/sdks/go/pkg/beam/model/jobmanagement_v1/beam_job_api.pb.go +++ b/sdks/go/pkg/beam/model/jobmanagement_v1/beam_job_api.pb.go @@ -22,7 +22,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: // protoc-gen-go v1.27.1 -// protoc v4.24.0--rc1 +// protoc v4.24.4 // source: org/apache/beam/model/job_management/v1/beam_job_api.proto package jobmanagement_v1 diff --git a/sdks/go/pkg/beam/model/jobmanagement_v1/beam_job_api_grpc.pb.go b/sdks/go/pkg/beam/model/jobmanagement_v1/beam_job_api_grpc.pb.go index 08da7e4643c3..38f2c85a1c1c 100644 --- a/sdks/go/pkg/beam/model/jobmanagement_v1/beam_job_api_grpc.pb.go +++ b/sdks/go/pkg/beam/model/jobmanagement_v1/beam_job_api_grpc.pb.go @@ -17,7 +17,7 @@ // Code generated by protoc-gen-go-grpc. DO NOT EDIT. // versions: // - protoc-gen-go-grpc v1.1.0 -// - protoc v4.24.0--rc1 +// - protoc v4.24.4 // source: org/apache/beam/model/job_management/v1/beam_job_api.proto package jobmanagement_v1 diff --git a/sdks/go/pkg/beam/model/pipeline_v1/beam_runner_api.pb.go b/sdks/go/pkg/beam/model/pipeline_v1/beam_runner_api.pb.go index b20a5dccbe05..49df2b5c2e59 100644 --- a/sdks/go/pkg/beam/model/pipeline_v1/beam_runner_api.pb.go +++ b/sdks/go/pkg/beam/model/pipeline_v1/beam_runner_api.pb.go @@ -22,7 +22,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: // protoc-gen-go v1.27.1 -// protoc v4.24.0--rc1 +// protoc v4.24.4 // source: org/apache/beam/model/pipeline/v1/beam_runner_api.proto package pipeline_v1 @@ -1857,6 +1857,10 @@ const ( // SDKs should convert the size to bytes, but can allow users to specify human-friendly units (e.g. GiB). // Payload: ASCII encoded string of the base 10 representation of an integer number of bytes. StandardResourceHints_MIN_RAM_BYTES StandardResourceHints_Enum = 1 + // Describes desired number of CPUs available in transform's execution environment. + // SDKs should accept and validate a positive integer count. + // Payload: ASCII encoded string of the base 10 representation of an integer number of CPUs. + StandardResourceHints_CPU_COUNT StandardResourceHints_Enum = 2 ) // Enum value maps for StandardResourceHints_Enum. @@ -1864,10 +1868,12 @@ var ( StandardResourceHints_Enum_name = map[int32]string{ 0: "ACCELERATOR", 1: "MIN_RAM_BYTES", + 2: "CPU_COUNT", } StandardResourceHints_Enum_value = map[string]int32{ "ACCELERATOR": 0, "MIN_RAM_BYTES": 1, + "CPU_COUNT": 2, } ) @@ -9223,42 +9229,45 @@ var file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDesc = []byt 0x65, 0x63, 0x75, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x74, 0x61, 0x67, 0x65, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x72, 0x49, 0x64, 0x48, 0x00, 0x52, 0x05, 0x74, 0x69, 0x6d, 0x65, 0x72, 0x42, 0x08, 0x0a, 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x22, - 0x8f, 0x01, 0x0a, 0x15, 0x53, 0x74, 0x61, 0x6e, 0x64, 0x61, 0x72, 0x64, 0x52, 0x65, 0x73, 0x6f, - 0x75, 0x72, 0x63, 0x65, 0x48, 0x69, 0x6e, 0x74, 0x73, 0x22, 0x76, 0x0a, 0x04, 0x45, 0x6e, 0x75, - 0x6d, 0x12, 0x34, 0x0a, 0x0b, 0x41, 0x43, 0x43, 0x45, 0x4c, 0x45, 0x52, 0x41, 0x54, 0x4f, 0x52, - 0x10, 0x00, 0x1a, 0x23, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1d, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x72, - 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x73, 0x3a, 0x61, 0x63, 0x63, 0x65, 0x6c, 0x65, 0x72, - 0x61, 0x74, 0x6f, 0x72, 0x3a, 0x76, 0x31, 0x12, 0x38, 0x0a, 0x0d, 0x4d, 0x49, 0x4e, 0x5f, 0x52, - 0x41, 0x4d, 0x5f, 0x42, 0x59, 0x54, 0x45, 0x53, 0x10, 0x01, 0x1a, 0x25, 0xa2, 0xb4, 0xfa, 0xc2, - 0x05, 0x1f, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x73, - 0x3a, 0x6d, 0x69, 0x6e, 0x5f, 0x72, 0x61, 0x6d, 0x5f, 0x62, 0x79, 0x74, 0x65, 0x73, 0x3a, 0x76, - 0x31, 0x32, 0x8f, 0x01, 0x0a, 0x11, 0x54, 0x65, 0x73, 0x74, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, - 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x7a, 0x0a, 0x06, 0x45, 0x76, 0x65, 0x6e, 0x74, - 0x73, 0x12, 0x30, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, - 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, - 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x1a, 0x3a, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, + 0xc2, 0x01, 0x0a, 0x15, 0x53, 0x74, 0x61, 0x6e, 0x64, 0x61, 0x72, 0x64, 0x52, 0x65, 0x73, 0x6f, + 0x75, 0x72, 0x63, 0x65, 0x48, 0x69, 0x6e, 0x74, 0x73, 0x22, 0xa8, 0x01, 0x0a, 0x04, 0x45, 0x6e, + 0x75, 0x6d, 0x12, 0x34, 0x0a, 0x0b, 0x41, 0x43, 0x43, 0x45, 0x4c, 0x45, 0x52, 0x41, 0x54, 0x4f, + 0x52, 0x10, 0x00, 0x1a, 0x23, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1d, 0x62, 0x65, 0x61, 0x6d, 0x3a, + 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x73, 0x3a, 0x61, 0x63, 0x63, 0x65, 0x6c, 0x65, + 0x72, 0x61, 0x74, 0x6f, 0x72, 0x3a, 0x76, 0x31, 0x12, 0x38, 0x0a, 0x0d, 0x4d, 0x49, 0x4e, 0x5f, + 0x52, 0x41, 0x4d, 0x5f, 0x42, 0x59, 0x54, 0x45, 0x53, 0x10, 0x01, 0x1a, 0x25, 0xa2, 0xb4, 0xfa, + 0xc2, 0x05, 0x1f, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, + 0x73, 0x3a, 0x6d, 0x69, 0x6e, 0x5f, 0x72, 0x61, 0x6d, 0x5f, 0x62, 0x79, 0x74, 0x65, 0x73, 0x3a, + 0x76, 0x31, 0x12, 0x30, 0x0a, 0x09, 0x43, 0x50, 0x55, 0x5f, 0x43, 0x4f, 0x55, 0x4e, 0x54, 0x10, + 0x02, 0x1a, 0x21, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1b, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x72, 0x65, + 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x73, 0x3a, 0x63, 0x70, 0x75, 0x5f, 0x63, 0x6f, 0x75, 0x6e, + 0x74, 0x3a, 0x76, 0x31, 0x32, 0x8f, 0x01, 0x0a, 0x11, 0x54, 0x65, 0x73, 0x74, 0x53, 0x74, 0x72, + 0x65, 0x61, 0x6d, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x7a, 0x0a, 0x06, 0x45, 0x76, + 0x65, 0x6e, 0x74, 0x73, 0x12, 0x30, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, + 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, + 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x3a, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, + 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, + 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x65, 0x73, 0x74, 0x53, + 0x74, 0x72, 0x65, 0x61, 0x6d, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x2e, 0x45, 0x76, 0x65, + 0x6e, 0x74, 0x22, 0x00, 0x30, 0x01, 0x3a, 0x3f, 0x0a, 0x08, 0x62, 0x65, 0x61, 0x6d, 0x5f, 0x75, + 0x72, 0x6e, 0x12, 0x21, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6e, 0x75, 0x6d, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x4f, 0x70, + 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0xc4, 0xa6, 0xaf, 0x58, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, + 0x62, 0x65, 0x61, 0x6d, 0x55, 0x72, 0x6e, 0x3a, 0x49, 0x0a, 0x0d, 0x62, 0x65, 0x61, 0x6d, 0x5f, + 0x63, 0x6f, 0x6e, 0x73, 0x74, 0x61, 0x6e, 0x74, 0x12, 0x21, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, + 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6e, 0x75, 0x6d, 0x56, + 0x61, 0x6c, 0x75, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0xc5, 0xa6, 0xaf, 0x58, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x62, 0x65, 0x61, 0x6d, 0x43, 0x6f, 0x6e, 0x73, 0x74, 0x61, + 0x6e, 0x74, 0x42, 0x78, 0x0a, 0x21, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, - 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x65, 0x73, 0x74, 0x53, 0x74, 0x72, 0x65, - 0x61, 0x6d, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x22, - 0x00, 0x30, 0x01, 0x3a, 0x3f, 0x0a, 0x08, 0x62, 0x65, 0x61, 0x6d, 0x5f, 0x75, 0x72, 0x6e, 0x12, - 0x21, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, - 0x66, 0x2e, 0x45, 0x6e, 0x75, 0x6d, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, - 0x6e, 0x73, 0x18, 0xc4, 0xa6, 0xaf, 0x58, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x62, 0x65, 0x61, - 0x6d, 0x55, 0x72, 0x6e, 0x3a, 0x49, 0x0a, 0x0d, 0x62, 0x65, 0x61, 0x6d, 0x5f, 0x63, 0x6f, 0x6e, - 0x73, 0x74, 0x61, 0x6e, 0x74, 0x12, 0x21, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6e, 0x75, 0x6d, 0x56, 0x61, 0x6c, 0x75, - 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0xc5, 0xa6, 0xaf, 0x58, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x0c, 0x62, 0x65, 0x61, 0x6d, 0x43, 0x6f, 0x6e, 0x73, 0x74, 0x61, 0x6e, 0x74, 0x42, - 0x78, 0x0a, 0x21, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, - 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, - 0x65, 0x2e, 0x76, 0x31, 0x42, 0x09, 0x52, 0x75, 0x6e, 0x6e, 0x65, 0x72, 0x41, 0x70, 0x69, 0x5a, - 0x48, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x61, 0x70, 0x61, 0x63, - 0x68, 0x65, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x2f, 0x73, 0x64, 0x6b, 0x73, 0x2f, 0x76, 0x32, 0x2f, - 0x67, 0x6f, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x2f, 0x6d, 0x6f, 0x64, 0x65, - 0x6c, 0x2f, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x5f, 0x76, 0x31, 0x3b, 0x70, 0x69, - 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x5f, 0x76, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x33, + 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x42, 0x09, 0x52, 0x75, 0x6e, 0x6e, 0x65, 0x72, 0x41, + 0x70, 0x69, 0x5a, 0x48, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x61, + 0x70, 0x61, 0x63, 0x68, 0x65, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x2f, 0x73, 0x64, 0x6b, 0x73, 0x2f, + 0x76, 0x32, 0x2f, 0x67, 0x6f, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x2f, 0x6d, + 0x6f, 0x64, 0x65, 0x6c, 0x2f, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x5f, 0x76, 0x31, + 0x3b, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x5f, 0x76, 0x31, 0x62, 0x06, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x33, } var ( diff --git a/sdks/go/pkg/beam/model/pipeline_v1/beam_runner_api_grpc.pb.go b/sdks/go/pkg/beam/model/pipeline_v1/beam_runner_api_grpc.pb.go index d5e65f7b768d..20a30cf4dd01 100644 --- a/sdks/go/pkg/beam/model/pipeline_v1/beam_runner_api_grpc.pb.go +++ b/sdks/go/pkg/beam/model/pipeline_v1/beam_runner_api_grpc.pb.go @@ -17,7 +17,7 @@ // Code generated by protoc-gen-go-grpc. DO NOT EDIT. // versions: // - protoc-gen-go-grpc v1.1.0 -// - protoc v4.24.0--rc1 +// - protoc v4.24.4 // source: org/apache/beam/model/pipeline/v1/beam_runner_api.proto package pipeline_v1 diff --git a/sdks/go/pkg/beam/model/pipeline_v1/endpoints.pb.go b/sdks/go/pkg/beam/model/pipeline_v1/endpoints.pb.go index 74348ddc3b33..2dfaffa2bff0 100644 --- a/sdks/go/pkg/beam/model/pipeline_v1/endpoints.pb.go +++ b/sdks/go/pkg/beam/model/pipeline_v1/endpoints.pb.go @@ -21,7 +21,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: // protoc-gen-go v1.27.1 -// protoc v4.24.0--rc1 +// protoc v4.24.4 // source: org/apache/beam/model/pipeline/v1/endpoints.proto package pipeline_v1 diff --git a/sdks/go/pkg/beam/model/pipeline_v1/external_transforms.pb.go b/sdks/go/pkg/beam/model/pipeline_v1/external_transforms.pb.go index 0bc21a56685e..edbe82264f5e 100644 --- a/sdks/go/pkg/beam/model/pipeline_v1/external_transforms.pb.go +++ b/sdks/go/pkg/beam/model/pipeline_v1/external_transforms.pb.go @@ -21,7 +21,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: // protoc-gen-go v1.27.1 -// protoc v4.24.0--rc1 +// protoc v4.24.4 // source: org/apache/beam/model/pipeline/v1/external_transforms.proto package pipeline_v1 diff --git a/sdks/go/pkg/beam/model/pipeline_v1/metrics.pb.go b/sdks/go/pkg/beam/model/pipeline_v1/metrics.pb.go index ec3e0d704a80..60edad2363be 100644 --- a/sdks/go/pkg/beam/model/pipeline_v1/metrics.pb.go +++ b/sdks/go/pkg/beam/model/pipeline_v1/metrics.pb.go @@ -21,7 +21,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: // protoc-gen-go v1.27.1 -// protoc v4.24.0--rc1 +// protoc v4.24.4 // source: org/apache/beam/model/pipeline/v1/metrics.proto package pipeline_v1 diff --git a/sdks/go/pkg/beam/model/pipeline_v1/schema.pb.go b/sdks/go/pkg/beam/model/pipeline_v1/schema.pb.go index 717fbbfb7a69..4bc6a57044cd 100644 --- a/sdks/go/pkg/beam/model/pipeline_v1/schema.pb.go +++ b/sdks/go/pkg/beam/model/pipeline_v1/schema.pb.go @@ -24,7 +24,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: // protoc-gen-go v1.27.1 -// protoc v4.24.0--rc1 +// protoc v4.24.4 // source: org/apache/beam/model/pipeline/v1/schema.proto package pipeline_v1 diff --git a/sdks/go/pkg/beam/model/pipeline_v1/standard_window_fns.pb.go b/sdks/go/pkg/beam/model/pipeline_v1/standard_window_fns.pb.go index dccd7d427503..e0522806df73 100644 --- a/sdks/go/pkg/beam/model/pipeline_v1/standard_window_fns.pb.go +++ b/sdks/go/pkg/beam/model/pipeline_v1/standard_window_fns.pb.go @@ -22,7 +22,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: // protoc-gen-go v1.27.1 -// protoc v4.24.0--rc1 +// protoc v4.24.4 // source: org/apache/beam/model/pipeline/v1/standard_window_fns.proto package pipeline_v1 diff --git a/sdks/go/pkg/beam/options/resource/hint.go b/sdks/go/pkg/beam/options/resource/hint.go index 1538fe65def2..d823f4feafa9 100644 --- a/sdks/go/pkg/beam/options/resource/hint.go +++ b/sdks/go/pkg/beam/options/resource/hint.go @@ -196,3 +196,40 @@ func (h acceleratorHint) MergeWithOuter(outer Hint) Hint { func (h acceleratorHint) String() string { return fmt.Sprintf("accelerator=%v", h.value) } + +// CPUCount hints that this scope should be put in a machine with at least this many CPUs or vCPUs. +// +// Hints are advisory only and runners may not respect them. +// +// See https://beam.apache.org/documentation/runtime/resource-hints/ for more information about +// resource hints. +func CPUCount(v uint64) Hint { + return CPUCountHint{value: uint64(v)} +} + +type CPUCountHint struct { + value uint64 +} + +func (CPUCountHint) URN() string { + return "beam:resources:cpu_count:v1" +} + +func (h CPUCountHint) Payload() []byte { + // Go strings are utf8, and if the string is ascii, + // byte conversion handles that directly. + return []byte(strconv.FormatUint(h.value, 10)) +} + +// MergeWithOuter by keeping the maximum of the two cpu counts. +func (h CPUCountHint) MergeWithOuter(outer Hint) Hint { + // Intentional runtime panic from type assertion to catch hint merge errors. + if outer.(CPUCountHint).value > h.value { + return outer + } + return h +} + +func (h CPUCountHint) String() string { + return fmt.Sprintf("cpu_count=%v", humanize.Bytes(uint64(h.value))) +} diff --git a/sdks/go/pkg/beam/options/resource/hint_test.go b/sdks/go/pkg/beam/options/resource/hint_test.go index cf24b47b6c91..7c2a1df79294 100644 --- a/sdks/go/pkg/beam/options/resource/hint_test.go +++ b/sdks/go/pkg/beam/options/resource/hint_test.go @@ -111,6 +111,38 @@ func TestParseMinRAMHint_panic(t *testing.T) { ParseMinRAM("a bad byte string") } +func TestCPUCountHint_MergeWith(t *testing.T) { + low := CPUCountHint{value: 2} + high := CPUCountHint{value: 128} + + if got, want := low.MergeWithOuter(high), high; got != want { + t.Errorf("%v.MergeWith(%v) = %v, want %v", low, high, got, want) + } + if got, want := high.MergeWithOuter(low), high; got != want { + t.Errorf("%v.MergeWith(%v) = %v, want %v", high, low, got, want) + } +} + +func TestCPUCountHint_Payload(t *testing.T) { + tests := []struct { + value uint64 + payload string + }{ + {0, "0"}, + {2, "2"}, + {11, "11"}, + {2003, "2003"}, + {1.2e7, "12000000"}, + } + + for _, test := range tests { + h := CPUCountHint{value: test.value} + if got, want := h.Payload(), []byte(test.payload); !bytes.Equal(got, want) { + t.Errorf("%v.Payload() = %v, want %v", h, got, want) + } + } +} + // We copy the URN from the proto for use as a constant rather than perform a direct look up // each time, or increase initialization time. However we do need to validate that they are // correct, and match the standard hint urns, so that's done here. @@ -130,7 +162,11 @@ func TestStandardHintUrns(t *testing.T) { }, { h: MinRAMBytes(2e9), urn: getStandardURN(pipepb.StandardResourceHints_MIN_RAM_BYTES), + }, { + h: CPUCount(4), + urn: getStandardURN(pipepb.StandardResourceHints_CPU_COUNT), }} + for _, test := range tests { if got, want := test.h.URN(), test.urn; got != want { t.Errorf("Checked urn for %T, got %q, want %q", test.h, got, want) @@ -154,12 +190,12 @@ func (h customHint) MergeWithOuter(outer Hint) Hint { } func TestHints_Equal(t *testing.T) { - hs := NewHints(MinRAMBytes(2e9), Accelerator("type:pants;count1;install-pajamas")) + hs := NewHints(MinRAMBytes(2e9), Accelerator("type:pants;count1;install-pajamas"), CPUCount(4)) if got, want := hs.Equal(hs), true; got != want { t.Errorf("Self equal test: hs.Equal(hs) = %v, want %v", got, want) } - eq := NewHints(MinRAMBytes(2e9), Accelerator("type:pants;count1;install-pajamas")) + eq := NewHints(MinRAMBytes(2e9), Accelerator("type:pants;count1;install-pajamas"), CPUCount(4)) if got, want := hs.Equal(eq), true; got != want { t.Errorf("identical equal test: hs.Equal(eq) = %v, want %v", got, want) } @@ -223,12 +259,13 @@ func TestHints_MergeWithOuter(t *testing.T) { func TestHints_Payloads(t *testing.T) { { - hs := NewHints(MinRAMBytes(2e9), Accelerator("type:jeans;count1;")) + hs := NewHints(MinRAMBytes(2e9), Accelerator("type:jeans;count1;"), CPUCount(4)) got := hs.Payloads() want := map[string][]byte{ "beam:resources:min_ram_bytes:v1": []byte("2000000000"), "beam:resources:accelerator:v1": []byte("type:jeans;count1;"), + "beam:resources:cpu_count:v1": []byte("4"), } if !reflect.DeepEqual(got, want) { t.Errorf("hs.Payloads() = %v, want %v", got, want) @@ -248,7 +285,7 @@ func TestHints_Payloads(t *testing.T) { func TestHints_NilHints(t *testing.T) { var hs1, hs2 Hints - hs := NewHints(MinRAMBytes(2e9), Accelerator("type:pants;count1;install-pajamas")) + hs := NewHints(MinRAMBytes(2e9), Accelerator("type:pants;count1;install-pajamas"), CPUCount(4)) if got, want := hs1.Equal(hs2), true; got != want { t.Errorf("nils equal test: (nil).Equal(nil) = %v, want %v", got, want) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/resourcehints/ResourceHints.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/resourcehints/ResourceHints.java index afd6a6ccb151..85cb2df9deab 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/resourcehints/ResourceHints.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/resourcehints/ResourceHints.java @@ -49,6 +49,8 @@ public class ResourceHints { private static final String MIN_RAM_URN = "beam:resources:min_ram_bytes:v1"; private static final String ACCELERATOR_URN = "beam:resources:accelerator:v1"; + private static final String CPU_COUNT_URN = "beam:resources:cpu_count:v1"; + // TODO: reference this from a common location in all packages that use this. private static String getUrn(ProtocolMessageEnum value) { return value.getValueDescriptor().getOptions().getExtension(RunnerApi.beamUrn); @@ -57,6 +59,7 @@ private static String getUrn(ProtocolMessageEnum value) { static { checkState(MIN_RAM_URN.equals(getUrn(StandardResourceHints.Enum.MIN_RAM_BYTES))); checkState(ACCELERATOR_URN.equals(getUrn(StandardResourceHints.Enum.ACCELERATOR))); + checkState(CPU_COUNT_URN.equals(getUrn(StandardResourceHints.Enum.CPU_COUNT))); } private static ImmutableMap hintNameToUrn = @@ -64,12 +67,15 @@ private static String getUrn(ProtocolMessageEnum value) { .put("minRam", MIN_RAM_URN) .put("min_ram", MIN_RAM_URN) // Courtesy alias. .put("accelerator", ACCELERATOR_URN) + .put("cpuCount", CPU_COUNT_URN) + .put("cpu_count", CPU_COUNT_URN) // Courtesy alias. .build(); private static ImmutableMap> parsers = ImmutableMap.>builder() .put(MIN_RAM_URN, s -> new BytesHint(BytesHint.parse(s))) .put(ACCELERATOR_URN, s -> new StringHint(s)) + .put(CPU_COUNT_URN, s -> new IntHint(IntHint.parse(s))) .build(); private static final ResourceHints EMPTY = new ResourceHints(ImmutableMap.of()); @@ -212,6 +218,46 @@ public int hashCode() { } } + /*package*/ static class IntHint extends ResourceHint { + private final int value; + + @Override + public boolean equals(@Nullable Object other) { + if (other == null) { + return false; + } else if (this == other) { + return true; + } else if (other instanceof IntHint) { + return ((IntHint) other).value == value; + } else { + return false; + } + } + + @Override + public int hashCode() { + return Integer.hashCode(value); + } + + public IntHint(int value) { + this.value = value; + } + + public static int parse(String s) { + return Integer.parseInt(s, 10); + } + + @Override + public ResourceHint mergeWithOuter(ResourceHint outer) { + return new IntHint(Math.max(value, ((IntHint) outer).value)); + } + + @Override + public byte[] toBytes() { + return String.valueOf(value).getBytes(Charsets.US_ASCII); + } + } + /** * Sets desired minimal available RAM size to have in transform's execution environment. * @@ -264,6 +310,23 @@ public ResourceHints withHint(String urn, ResourceHint hint) { return new ResourceHints(newHints.build()); } + /** + * Sets desired minimal CPU or vCPU count to have in transform's execution environment. + * + * @param cpuCount specifies a positive CPU count. + */ + public ResourceHints withCPUCount(int cpuCount) { + if (cpuCount <= 0) { + LOG.error( + "Encountered invalid non-positive cpu count hint value {}.\n" + + "The value is ignored. In the future, The method will require an object Long type " + + "and throw an IllegalArgumentException for invalid values.", + cpuCount); + return this; + } + return withHint(CPU_COUNT_URN, new IntHint(cpuCount)); + } + public Map hints() { return hints; } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/resourcehints/ResourceHintsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/resourcehints/ResourceHintsTest.java index 3cc522176374..c7643f718aa5 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/resourcehints/ResourceHintsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/resourcehints/ResourceHintsTest.java @@ -92,10 +92,13 @@ public void testFromOptions() { .withHint("beam:resources:bar", new ResourceHints.StringHint("foo"))); options = PipelineOptionsFactory.fromArgs( - "--resourceHints=min_ram=1KB", "--resourceHints=accelerator=foo") + "--resourceHints=min_ram=1KB", + "--resourceHints=accelerator=foo", + "--resourceHints=cpu_count=4") .as(ResourceHintsOptions.class); - assertEquals( - ResourceHints.fromOptions(options), - ResourceHints.create().withMinRam(1000).withAccelerator("foo")); + ResourceHints fromOptions = ResourceHints.fromOptions(options); + ResourceHints expect = + ResourceHints.create().withMinRam(1000).withAccelerator("foo").withCPUCount(4); + assertEquals(fromOptions, expect); } } diff --git a/sdks/python/apache_beam/transforms/resources.py b/sdks/python/apache_beam/transforms/resources.py index 7bb202ab5660..7c4160df8edd 100644 --- a/sdks/python/apache_beam/transforms/resources.py +++ b/sdks/python/apache_beam/transforms/resources.py @@ -42,6 +42,7 @@ 'ResourceHint', 'AcceleratorHint', 'MinRamHint', + 'CpuCountHint', 'merge_resource_hints', 'parse_resource_hints', 'resource_hints_from_options', @@ -177,6 +178,21 @@ def get_merged_value( ResourceHint.register_resource_hint('minRam', MinRamHint) +class CpuCountHint(ResourceHint): + """Describes number of CPUs available in transform's execution environment.""" + urn = resource_hints.CPU_COUNT.urn + + @classmethod + def get_merged_value( + cls, outer_value, inner_value): # type: (bytes, bytes) -> bytes + return ResourceHint._use_max(outer_value, inner_value) + + +ResourceHint.register_resource_hint('cpu_count', CpuCountHint) +# Alias for interoperability with SDKs preferring camelCase. +ResourceHint.register_resource_hint('cpuCount', CpuCountHint) + + def parse_resource_hints(hints): # type: (Dict[Any, Any]) -> Dict[str, bytes] parsed_hints = {} for hint, value in hints.items(): diff --git a/sdks/python/apache_beam/transforms/resources_test.py b/sdks/python/apache_beam/transforms/resources_test.py index 939391b7adcb..939bdcd62651 100644 --- a/sdks/python/apache_beam/transforms/resources_test.py +++ b/sdks/python/apache_beam/transforms/resources_test.py @@ -46,6 +46,11 @@ class ResourcesTest(unittest.TestCase): val='gpu', urn='beam:resources:accelerator:v1', bytestr=b'gpu'), + param( + name='cpu_count', + val='4', + urn='beam:resources:cpu_count:v1', + bytestr=b'4'), ]) def test_known_resource_hints(self, name, val, urn, bytestr): t = PTransform() @@ -56,6 +61,7 @@ def test_known_resource_hints(self, name, val, urn, bytestr): @parameterized.expand([ param(name='min_ram', val='3,500G'), param(name='accelerator', val=1), + param(name='cpu_count', val=1), param(name='unknown_hint', val=1) ]) def test_resource_hint_parsing_fails_early(self, name, val): From 835bd655948d9a8ebc47277dedb25fb4c619f239 Mon Sep 17 00:00:00 2001 From: Danny McCormick Date: Thu, 12 Oct 2023 09:25:37 -0400 Subject: [PATCH 119/435] Add docs for per key inference (#28243) * Update KeyMhMapping to KeyModelMapping * Add docs for per key inference * Add piece on memory thrashing * Whitespace * Update wording based on feedback * Add references to website in pydoc * Apply suggestions from code review Co-authored-by: Rebecca Szper <98840847+rszper@users.noreply.github.com> * Remove ordering implied by wording * Lint fixes --------- Co-authored-by: Rebecca Szper <98840847+rszper@users.noreply.github.com> --- sdks/python/apache_beam/ml/inference/base.py | 9 +++- .../sdks/python-machine-learning.md | 48 +++++++++++++++++++ 2 files changed, 56 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/ml/inference/base.py b/sdks/python/apache_beam/ml/inference/base.py index 90d43cfddb94..753e1468137c 100644 --- a/sdks/python/apache_beam/ml/inference/base.py +++ b/sdks/python/apache_beam/ml/inference/base.py @@ -482,6 +482,12 @@ def __init__( from the cohort. When model updates occur, the metrics will be reported in the form `--`. + Loading multiple models at the same time can increase the risk of an out of + memory (OOM) exception. To avoid this issue, use the parameter + `max_models_per_worker_hint` to limit the number of models that are loaded + at the same time. For more information about memory management, see + `Use a keyed `ModelHandler _`. # pylint: disable=line-too-long + Args: unkeyed: Either (a) an implementation of ModelHandler that does not @@ -491,7 +497,8 @@ def __init__( models can be held in memory at one time per worker process. For example, if your worker has 8 GB of memory provisioned and your workers take up 1 GB each, you should set this to 7 to allow all models to sit - in memory with some buffer. + in memory with some buffer. For more information about memory management, + see `Use a keyed `ModelHandler _`. # pylint: disable=line-too-long """ self._metrics_collectors: Dict[str, _MetricsCollector] = {} self._default_metrics_collector: _MetricsCollector = None diff --git a/website/www/site/content/en/documentation/sdks/python-machine-learning.md b/website/www/site/content/en/documentation/sdks/python-machine-learning.md index 5e0cf483ff3e..0076fa370b0f 100644 --- a/website/www/site/content/en/documentation/sdks/python-machine-learning.md +++ b/website/www/site/content/en/documentation/sdks/python-machine-learning.md @@ -215,6 +215,54 @@ with pipeline as p: If you are unsure if your data is keyed, you can also use `MaybeKeyedModelHandler`. +You can also use a `KeyedModelHandler` to load several different models based on their associated key: + +``` +from apache_beam.ml.inference.base import KeyedModelHandler +keyed_model_handler = KeyedModelHandler([ + KeyModelMapping(['key1'], PytorchModelHandlerTensor()), + KeyModelMapping(['key2', 'key3'], PytorchModelHandlerTensor()) +]) +with pipeline as p: + data = p | beam.Create([ + ('key1', torch.tensor([[1,2,3],[4,5,6],...])), + ('key2', torch.tensor([[1,2,3],[4,5,6],...])), + ('key3', torch.tensor([[1,2,3],[4,5,6],...])), + ]) + predictions = data | RunInference(keyed_model_handler) +``` + +The previous example loads a model by using `config1`. That model is then used for inference for all examples associated +with `key1`. It also loads a model by using `config2`. That model is used for all examples associated with `key2` and `key3`. + +Loading multiple models at the same times increases the risk of out of memory (OOM) errors. By default, `KeyedModelHandler` doesn't +limit the number of models loaded into memory at the same time. If the models don't all fit into memory, +your pipeline will likely fail with an out of memory error. To avoid this issue, provide a hint about the +maximum number of models that can be loaded at the same time. + +``` +mhs = [ + KeyModelMapping(['key1'], PytorchModelHandlerTensor()), + KeyModelMapping(['key2', 'key3'], PytorchModelHandlerTensor()), + KeyModelMapping(['key4'], PytorchModelHandlerTensor()), + KeyModelMapping(['key5', 'key6', 'key7'], PytorchModelHandlerTensor()), +] +keyed_model_handler = KeyedModelHandler(mhs, max_models_per_worker_hint=2) +``` + +The previous example loads at most two models per SDK worker process at any given time. It unloads models that aren't +currently being used. Runners that have multiple SDK worker processes on a given machine load at most +`max_models_per_worker_hint*` models onto the machine. Leave enough space for the models +and any additional memory needs from other transforms. Because there might be a delay between when a model is offloaded and when the +memory is released, it is recommended that you leave additional buffer. + +**Note**: Having many models but a small `max_models_per_worker_hint` can lead to _memory thrashing_, where +a large amount of execution time is wasted swapping models in and out of memory. To reduce the likelihood and impact +of memory thrashing, if you're using a distributed runner, insert a +[GroupByKey](https://beam.apache.org/documentation/transforms/python/aggregation/groupbykey/) transform before your +inference step. This step reduces thrashing by ensuring that elements with the same key and model are +collocated on the same worker. + For more information, see [`KeyedModelHander`](https://beam.apache.org/releases/pydoc/current/apache_beam.ml.inference.base.html#apache_beam.ml.inference.base.KeyedModelHandler). ### Use the `PredictionResult` object From fbd147f83548f7586eccb0a364d7804318fbac86 Mon Sep 17 00:00:00 2001 From: Danny McCormick Date: Thu, 12 Oct 2023 09:27:14 -0400 Subject: [PATCH 120/435] Use latest released beam in notebook (#28801) --- examples/notebooks/beam-ml/per_key_models.ipynb | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/examples/notebooks/beam-ml/per_key_models.ipynb b/examples/notebooks/beam-ml/per_key_models.ipynb index b529449555d0..a5dfa0d54e31 100644 --- a/examples/notebooks/beam-ml/per_key_models.ipynb +++ b/examples/notebooks/beam-ml/per_key_models.ipynb @@ -107,12 +107,7 @@ } ], "source": [ - "# Note that this notebook currently installs from Beam head since this feature hasn't been released yet.\n", - "# It will be released with version 2.51.0, at which point you can install with the following command:\n", - "# !pip install apache_beam[gcp]>=2.51.0 --quiet\n", - "!git clone https://github.com/apache/beam\n", - "!pip install -r beam/sdks/python/build-requirements.txt\n", - "!pip install -e ./beam/sdks/python[gcp]\n", + "!pip install apache_beam[gcp]>=2.51.0 --quiet\n", "!pip install torch --quiet\n", "!pip install transformers --quiet\n", "\n", From 7449b67a6d1019c82d852a69ae8cb093199335e6 Mon Sep 17 00:00:00 2001 From: Rebecca Szper <98840847+rszper@users.noreply.github.com> Date: Thu, 12 Oct 2023 06:54:39 -0700 Subject: [PATCH 121/435] Editing notebooks to prepare for DevSite import (#28949) --- .../beam-ml/automatic_model_refresh.ipynb | 8 ++++---- .../notebooks/beam-ml/mltransform_basic.ipynb | 6 ++---- .../notebooks/beam-ml/per_key_models.ipynb | 18 +++++++++--------- 3 files changed, 15 insertions(+), 17 deletions(-) diff --git a/examples/notebooks/beam-ml/automatic_model_refresh.ipynb b/examples/notebooks/beam-ml/automatic_model_refresh.ipynb index 9cbab0a14178..3bafa4f07887 100644 --- a/examples/notebooks/beam-ml/automatic_model_refresh.ipynb +++ b/examples/notebooks/beam-ml/automatic_model_refresh.ipynb @@ -248,7 +248,7 @@ " This example uses `TFModelHandlerTensor` as the model handler and the `resnet_101` model trained on [ImageNet](https://www.image-net.org/).\n", "\n", "\n", - "For DataflowRunner, the model needs to be stored remote location accessible by the Beam pipeline. So we will download `ResNet101` model and upload it to the GCS location.\n" + "For the Dataflow runner, you need to store the model in a remote location that the Apache Beam pipeline can access. For this example, download the `ResNet101` model, and upload it to the Google Cloud Storage bucket.\n" ], "metadata": { "id": "_AUNH_GJk_NE" @@ -392,7 +392,7 @@ "source": [ "2. To read and preprocess the images, use the `preprocess_image` function. This example uses `Cat-with-beanie.jpg` for all inferences.\n", "\n", - " **Note**: Image used for prediction is licensed in CC-BY. The creator is listed in the [LICENSE.txt](https://storage.googleapis.com/apache-beam-samples/image_captioning/LICENSE.txt) file." + " **Note**: The image used for prediction is licensed in CC-BY. The creator is listed in the [LICENSE.txt](https://storage.googleapis.com/apache-beam-samples/image_captioning/LICENSE.txt) file." ], "metadata": { "id": "8-sal2rFAxP2" @@ -424,7 +424,7 @@ "cell_type": "markdown", "source": [ "3. Pass the images to the RunInference `PTransform`. RunInference takes `model_handler` and `model_metadata_pcoll` as input parameters.\n", - " * `model_metadata_pcoll` is a side input `PCollection` to the RunInference `PTransform`. This side input is used to update the `model_uri` in the `model_handler` without needing to stop the Apache Beam pipeline\n", + " * `model_metadata_pcoll` is a side input `PCollection` to the RunInference `PTransform`. This side input updates the `model_uri` in the `model_handler` while the Apache Beam pipeline runs.\n", " * Use `WatchFilePattern` as side input to watch a `file_pattern` matching `.keras` files. In this case, the `file_pattern` is `'gs://BUCKET_NAME/dataflow/*keras'`.\n", "\n" ], @@ -483,7 +483,7 @@ "source": [ "### Watch for the model update\n", "\n", - "After the pipeline starts processing data and when you see output emitted from the RunInference `PTransform`, upload a `resnet152` model saved in `.keras` format to a Google Cloud Storage bucket location that matches the `file_pattern` you defined earlier.\n" + "After the pipeline starts processing data, when you see output emitted from the RunInference `PTransform`, upload a `resnet152` model saved in the `.keras` format to a Google Cloud Storage bucket location that matches the `file_pattern` you defined earlier.\n" ], "metadata": { "id": "wYp-mBHHjOjA" diff --git a/examples/notebooks/beam-ml/mltransform_basic.ipynb b/examples/notebooks/beam-ml/mltransform_basic.ipynb index 820bc3400b58..fd305bddb3ba 100644 --- a/examples/notebooks/beam-ml/mltransform_basic.ipynb +++ b/examples/notebooks/beam-ml/mltransform_basic.ipynb @@ -65,7 +65,7 @@ "id": "d3b81cf2-8603-42bd-995e-9e14631effd0" }, "source": [ - "This notebook demonstrates how to use `MLTransform` to preprocess your data for machine learning models. `MLTransform` is a `PTransform` that wraps multiple Apache Beam data processing transforms. As a result, `MLTransform` gives you the ability to preprocess different types of data in multiple ways with one transform.\n", + "This notebook demonstrates how to use `MLTransform` to preprocess your data for machine learning models. `MLTransform` is a `PTransform` that wraps multiple Apache Beam data processing transforms. With `MLTransform`, you can preprocess different types of data in multiple ways with one transform.\n", "\n", "This notebook uses data processing transforms defined in the [apache_beam/ml/transforms/tft](https://beam.apache.org/releases/pydoc/current/apache_beam.ml.transforms.tft.html) module." ] @@ -423,8 +423,6 @@ "source": [ "### Scale the data by using the z-score\n", "\n", - "Scale to the data using the z-score\n", - "\n", "Similar to `ScaleTo01`, use [ScaleToZScore](https://beam.apache.org/releases/pydoc/current/apache_beam.ml.transforms.tft.html#apache_beam.ml.transforms.tft.ScaleToZScore) to scale the values by using the [z-score]([z-score](https://www.tensorflow.org/tfx/transform/api_docs/python/tft/scale_to_z_score#:~:text=Scaling%20to%20z%2Dscore%20subtracts%20out%20the%20mean%20and%20divides%20by%20standard%20deviation.%20Note%20that%20the%20standard%20deviation%20computed%20here%20is%20based%20on%20the%20biased%20variance%20(0%20delta%20degrees%20of%20freedom)%2C%20as%20computed%20by%20analyzers.var.).\n" ], "metadata": { @@ -607,7 +605,7 @@ "\n", "The previous examples show how to preprocess data for model training. This example uses the same preprocessing steps on the inference data. By using the same steps on the inference data, you can maintain consistent results.\n", "\n", - "Preprocess the data going into the inference by using the same preprocessing steps used on the data prior to training. To do this with `MLTransform`, pass the artifact location from the previous transforms to the parameter `read_artifact_location`. `MLTransform` uses the values and artifacts produced in the previous steps. You don't need to provide the transforms, because they are saved with the artifacts in the artifact location.\n" + "Preprocess the data used by the inference by using the same preprocessing steps that you used on the data prior to training. When using `MLTransform`, pass the artifact location from the previous transforms to the parameter `read_artifact_location`. `MLTransform` uses the values and artifacts produced in the previous steps. You don't need to provide the transforms, because they are saved with the artifacts in the artifact location.\n" ], "metadata": { "id": "kcnQSwkA-eSA" diff --git a/examples/notebooks/beam-ml/per_key_models.ipynb b/examples/notebooks/beam-ml/per_key_models.ipynb index a5dfa0d54e31..53845c0b3e19 100644 --- a/examples/notebooks/beam-ml/per_key_models.ipynb +++ b/examples/notebooks/beam-ml/per_key_models.ipynb @@ -70,7 +70,7 @@ "\n", "In Apache Beam, the recommended way to run inference is to use the `RunInference` transform. By using a `KeyedModelHandler`, you can efficiently run inference with O(100s) of models without having to manage memory yourself.\n", "\n", - "This notebook demonstrates how to use a `KeyedModelHandler` to run inference in an Apache Beam pipeline with multiple different models on a per-key basis. This notebook uses pretrained pipelines from Hugging Face. Before continuing with this notebook, it is recommended that you walk through the [beginner RunInference notebook](https://colab.sandbox.google.com/github/apache/beam/blob/master/examples/notebooks/beam-ml/run_inference_pytorch_tensorflow_sklearn.ipynb)." + "This notebook demonstrates how to use a `KeyedModelHandler` to run inference in an Apache Beam pipeline with multiple different models on a per-key basis. This notebook uses pretrained pipelines from Hugging Face. Before continuing with this notebook, it is recommended that you walk through the [Use RunInference in Apache Beam](https://colab.sandbox.google.com/github/apache/beam/blob/master/examples/notebooks/beam-ml/run_inference_pytorch_tensorflow_sklearn.ipynb) notebook." ], "metadata": { "id": "ZAVOrrW2An1n" @@ -81,7 +81,7 @@ "source": [ "## Install dependencies\n", "\n", - "First, install both Apache Beam and the dependencies needed by Hugging Face." + "Install both Apache Beam and the dependencies needed by Hugging Face." ], "metadata": { "id": "_fNyheQoDgGt" @@ -144,7 +144,7 @@ "\n", "A model handler is the Apache Beam method used to define the configuration needed to load and invoke models. Because this example uses two models, we define two model handlers, one for each model. Because both models are incapsulated within Hugging Face pipelines, we use the model handler `HuggingFacePipelineModelHandler`.\n", "\n", - "In this notebook, we load the models using Hugging Face and run them against an example. The models produce different outputs." + "For this example, load the models using Hugging Face, and then run them against an example. The models produce different outputs." ], "metadata": { "id": "uEqljVgCD7hx" @@ -350,7 +350,7 @@ "source": [ "## Define the examples\n", "\n", - "Next, define examples to input into the pipeline. The examples include their correct classifications." + "Define examples to input into the pipeline. The examples include the correct classifications." ], "metadata": { "id": "yd92MC7YEsTf" @@ -387,7 +387,7 @@ "class FormatExamples(beam.DoFn):\n", " \"\"\"\n", " Map each example to a tuple of ('-', 'example').\n", - " We use these keys to map our elements to the correct models.\n", + " Use these keys to map our elements to the correct models.\n", " \"\"\"\n", " def process(self, element: Tuple[str, str]) -> Iterable[Tuple[str, str]]:\n", " yield (f'distilbert-{element[1]}', element[0])\n", @@ -402,7 +402,7 @@ { "cell_type": "markdown", "source": [ - "Use the formatted keys to define a `KeyedModelHandler` that maps keys to the `ModelHandler` used for those keys. The `KeyedModelHandler` method lets you define an optional `max_models_per_worker_hint`, which limits the number of models that can be held in a single worker process at one time. If you're worried about your worker running out of memory, use this option. For more information about managing memory, see [Use a keyed ModelHandler](https://beam.apache.org/documentation/sdks/python-machine-learning/index.html#use-a-keyed-modelhandler)." + "Use the formatted keys to define a `KeyedModelHandler` that maps keys to the `ModelHandler` used for those keys. The `KeyedModelHandler` method lets you define an optional `max_models_per_worker_hint`, which limits the number of models that can be held in a single worker process at one time. If your worker might run out of memory, use this option. For more information about managing memory, see [Use a keyed ModelHandler](https://beam.apache.org/documentation/sdks/python-machine-learning/index.html#use-a-keyed-modelhandler)." ], "metadata": { "id": "IP65_5nNGIb8" @@ -428,9 +428,9 @@ "source": [ "## Postprocess the results\n", "\n", - "The `RunInference` transform returns a Tuple containing:\n", + "The `RunInference` transform returns a tuple that contains the following objects:\n", "* the original key\n", - "* a `PredictionResult` object containing the original example and the inference.\n", + "* a `PredictionResult` object containing the original example and the inference\n", "Use those outputs to extract the relevant data. Then, to compare each model's prediction, group this data by the original example." ], "metadata": { @@ -505,7 +505,7 @@ "source": [ "## Run the pipeline\n", "\n", - "Put together all of the pieces to run a single Apache Beam pipeline." + "To run a single Apache Beam pipeline, combine the previous steps." ], "metadata": { "id": "-LrpmM2PGAkf" From 223dded769df48270df317868dc32144ec2fb353 Mon Sep 17 00:00:00 2001 From: Vitaly Terentyev Date: Thu, 12 Oct 2023 18:57:35 +0400 Subject: [PATCH 122/435] Add Inference Python Benchmarks Dataflow workflow (#28943) --- ...m_Inference_Python_Benchmarks_Dataflow.yml | 144 ++++++++++++++++++ ...rch_Imagenet_Classification_Resnet_152.txt | 34 +++++ ...Classification_Resnet_152_Tesla_T4_GPU.txt | 36 +++++ ...ch_Language_Modeling_Bert_Base_Uncased.txt | 34 +++++ ...h_Language_Modeling_Bert_Large_Uncased.txt | 34 +++++ ...torch_Vision_Classification_Resnet_101.txt | 34 +++++ 6 files changed, 316 insertions(+) create mode 100644 .github/workflows/beam_Inference_Python_Benchmarks_Dataflow.yml create mode 100644 .github/workflows/load-tests-job-configs/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Imagenet_Classification_Resnet_152.txt create mode 100644 .github/workflows/load-tests-job-configs/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Imagenet_Classification_Resnet_152_Tesla_T4_GPU.txt create mode 100644 .github/workflows/load-tests-job-configs/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Language_Modeling_Bert_Base_Uncased.txt create mode 100644 .github/workflows/load-tests-job-configs/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Language_Modeling_Bert_Large_Uncased.txt create mode 100644 .github/workflows/load-tests-job-configs/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Vision_Classification_Resnet_101.txt diff --git a/.github/workflows/beam_Inference_Python_Benchmarks_Dataflow.yml b/.github/workflows/beam_Inference_Python_Benchmarks_Dataflow.yml new file mode 100644 index 000000000000..117dc39a0fb7 --- /dev/null +++ b/.github/workflows/beam_Inference_Python_Benchmarks_Dataflow.yml @@ -0,0 +1,144 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: Inference Python Benchmarks Dataflow + +on: + issue_comment: + types: [created] + schedule: + - cron: '50 3 * * *' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_Inference_Python_Benchmarks_Dataflow: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Inference Benchmarks' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 900 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_Inference_Python_Benchmarks_Dataflow"] + job_phrase: ["Run Inference Benchmarks"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup Python environment + uses: ./.github/actions/setup-environment-action + with: + python-version: '3.8' + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: python + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Vision_Classification_Resnet_101.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Imagenet_Classification_Resnet_152.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Language_Modeling_Bert_Base_Uncased.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Language_Modeling_Bert_Large_Uncased.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Imagenet_Classification_Resnet_152_Tesla_T4_GPU.txt + # The env variables are created and populated in the test-arguments-action as "_test_arguments_" + - name: get current time + run: echo "NOW_UTC=$(date '+%m%d%H%M%S' --utc)" >> $GITHUB_ENV + - name: run Pytorch Vision Classification with Resnet 101 + uses: ./.github/actions/gradle-command-self-hosted-action + timeout-minutes: 180 + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.benchmarks.inference.pytorch_image_classification_benchmarks \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + -PloadTest.requirementsTxtFile=apache_beam/ml/inference/torch_tests_requirements.txt \ + '-PloadTest.args=${{ env.beam_Inference_Python_Benchmarks_Dataflow_test_arguments_1 }} --job_name=benchmark-tests-pytorch-imagenet-python-101-${{env.NOW_UTC}} --output=gs://temp-storage-for-end-to-end-tests/torch/result_resnet101-${{env.NOW_UTC}}.txt' \ + - name: run Pytorch Imagenet Classification with Resnet 152 + uses: ./.github/actions/gradle-command-self-hosted-action + timeout-minutes: 180 + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.benchmarks.inference.pytorch_image_classification_benchmarks \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + -PloadTest.requirementsTxtFile=apache_beam/ml/inference/torch_tests_requirements.txt \ + '-PloadTest.args=${{ env.beam_Inference_Python_Benchmarks_Dataflow_test_arguments_2 }} --job_name=benchmark-tests-pytorch-imagenet-python-152-${{env.NOW_UTC}} --output=gs://temp-storage-for-end-to-end-tests/torch/result_resnet152-${{env.NOW_UTC}}.txt' \ + - name: run Pytorch Language Modeling using Hugging face bert-base-uncased model + uses: ./.github/actions/gradle-command-self-hosted-action + timeout-minutes: 180 + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.benchmarks.inference.pytorch_language_modeling_benchmarks \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + -PloadTest.requirementsTxtFile=apache_beam/ml/inference/torch_tests_requirements.txt \ + '-PloadTest.args=${{ env.beam_Inference_Python_Benchmarks_Dataflow_test_arguments_3 }} --job_name=benchmark-tests-pytorch-language-modeling-bert-base-uncased-${{env.NOW_UTC}} --output=gs://temp-storage-for-end-to-end-tests/torch/result_bert_base_uncased-${{env.NOW_UTC}}.txt' \ + - name: run Pytorch Langauge Modeling using Hugging Face bert-large-uncased model + uses: ./.github/actions/gradle-command-self-hosted-action + timeout-minutes: 180 + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.benchmarks.inference.pytorch_language_modeling_benchmarks \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + -PloadTest.requirementsTxtFile=apache_beam/ml/inference/torch_tests_requirements.txt \ + '-PloadTest.args=${{ env.beam_Inference_Python_Benchmarks_Dataflow_test_arguments_4 }} --job_name=benchmark-tests-pytorch-language-modeling-bert-large-uncased-${{env.NOW_UTC}} --output=gs://temp-storage-for-end-to-end-tests/torch/result_bert_large_uncased-${{env.NOW_UTC}}.txt' \ + - name: run Pytorch Imagenet Classification with Resnet 152 with Tesla T4 GPU + uses: ./.github/actions/gradle-command-self-hosted-action + timeout-minutes: 180 + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.benchmarks.inference.pytorch_image_classification_benchmarks \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + -PloadTest.requirementsTxtFile=apache_beam/ml/inference/torch_tests_requirements.txt \ + '-PloadTest.args=${{ env.beam_Inference_Python_Benchmarks_Dataflow_test_arguments_5 }} --job_name=benchmark-tests-pytorch-imagenet-python-gpu-${{env.NOW_UTC}} --output=gs://temp-storage-for-end-to-end-tests/torch/result_resnet152_gpu-${{env.NOW_UTC}}.txt' \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Imagenet_Classification_Resnet_152.txt b/.github/workflows/load-tests-job-configs/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Imagenet_Classification_Resnet_152.txt new file mode 100644 index 000000000000..c65317b49573 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Imagenet_Classification_Resnet_152.txt @@ -0,0 +1,34 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--machine_type=n1-standard-2 +--num_workers=75 +--disk_size_gb=50 +--autoscaling_algorithm=NONE +--staging_location=gs://temp-storage-for-perf-tests/loadtests +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--requirements_file=apache_beam/ml/inference/torch_tests_requirements.txt +--publish_to_big_query=true +--metrics_dataset=beam_run_inference +--metrics_table=torch_inference_imagenet_results_resnet152 +--input_options={} +--influx_measurement=torch_inference_imagenet_resnet152 +--pretrained_model_name=resnet152 +--device=CPU +--input_file=gs://apache-beam-ml/testing/inputs/openimage_50k_benchmark.txt +--model_state_dict_path=gs://apache-beam-ml/models/torchvision.models.resnet152.pth +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Imagenet_Classification_Resnet_152_Tesla_T4_GPU.txt b/.github/workflows/load-tests-job-configs/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Imagenet_Classification_Resnet_152_Tesla_T4_GPU.txt new file mode 100644 index 000000000000..c1b7e273c6e8 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Imagenet_Classification_Resnet_152_Tesla_T4_GPU.txt @@ -0,0 +1,36 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--machine_type=n1-standard-2 +--num_workers=75 +--disk_size_gb=50 +--autoscaling_algorithm=NONE +--staging_location=gs://temp-storage-for-perf-tests/loadtests +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--requirements_file=apache_beam/ml/inference/torch_tests_requirements.txt +--publish_to_big_query=true +--metrics_dataset=beam_run_inference +--metrics_table=torch_inference_imagenet_results_resnet152_tesla_t4 +--input_options={} +--influx_measurement=torch_inference_imagenet_resnet152_tesla_t4 +--pretrained_model_name=resnet152 +--device=GPU +--experiments=worker_accelerator=type:nvidia-tesla-t4;count:1;install-nvidia-driver +--sdk_container_image=us.gcr.io/apache-beam-testing/python-postcommit-it/tensor_rt:latest +--input_file=gs://apache-beam-ml/testing/inputs/openimage_50k_benchmark.txt +--model_state_dict_path=gs://apache-beam-ml/models/torchvision.models.resnet152.pth +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Language_Modeling_Bert_Base_Uncased.txt b/.github/workflows/load-tests-job-configs/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Language_Modeling_Bert_Base_Uncased.txt new file mode 100644 index 000000000000..66aca5fdbcd7 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Language_Modeling_Bert_Base_Uncased.txt @@ -0,0 +1,34 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--machine_type=n1-standard-2 +--num_workers=250 +--disk_size_gb=50 +--autoscaling_algorithm=NONE +--staging_location=gs://temp-storage-for-perf-tests/loadtests +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--requirements_file=apache_beam/ml/inference/torch_tests_requirements.txt +--publish_to_big_query=true +--metrics_dataset=beam_run_inference +--metrics_table=torch_language_modeling_bert_base_uncased +--input_options={} +--influx_measurement=torch_language_modeling_bert_base_uncased +--device=CPU +--input_file=gs://apache-beam-ml/testing/inputs/sentences_50k.txt +--bert_tokenizer=bert-base-uncased +--model_state_dict_path=gs://apache-beam-ml/models/huggingface.BertForMaskedLM.bert-base-uncased.pth +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Language_Modeling_Bert_Large_Uncased.txt b/.github/workflows/load-tests-job-configs/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Language_Modeling_Bert_Large_Uncased.txt new file mode 100644 index 000000000000..d6406271685b --- /dev/null +++ b/.github/workflows/load-tests-job-configs/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Language_Modeling_Bert_Large_Uncased.txt @@ -0,0 +1,34 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--machine_type=n1-standard-2 +--num_workers=250 +--disk_size_gb=50 +--autoscaling_algorithm=NONE +--staging_location=gs://temp-storage-for-perf-tests/loadtests +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--requirements_file=apache_beam/ml/inference/torch_tests_requirements.txt +--publish_to_big_query=true +--metrics_dataset=beam_run_inference +--metrics_table=torch_language_modeling_bert_large_uncased +--input_options={} +--influx_measurement=torch_language_modeling_bert_large_uncased +--device=CPU +--input_file=gs://apache-beam-ml/testing/inputs/sentences_50k.txt +--bert_tokenizer=bert-large-uncased +--model_state_dict_path=gs://apache-beam-ml/models/huggingface.BertForMaskedLM.bert-large-uncased.pth +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Vision_Classification_Resnet_101.txt b/.github/workflows/load-tests-job-configs/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Vision_Classification_Resnet_101.txt new file mode 100644 index 000000000000..5a0d25043909 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Vision_Classification_Resnet_101.txt @@ -0,0 +1,34 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--machine_type=n1-standard-2 +--num_workers=75 +--disk_size_gb=50 +--autoscaling_algorithm=NONE +--staging_location=gs://temp-storage-for-perf-tests/loadtests +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--requirements_file=apache_beam/ml/inference/torch_tests_requirements.txt +--publish_to_big_query=true +--metrics_dataset=beam_run_inference +--metrics_table=torch_inference_imagenet_results_resnet101 +--input_options={} +--influx_measurement=torch_inference_imagenet_resnet101 +--pretrained_model_name=resnet101 +--device=CPU +--input_file=gs://apache-beam-ml/testing/inputs/openimage_50k_benchmark.txt +--model_state_dict_path=gs://apache-beam-ml/models/torchvision.models.resnet101.pth +--runner=DataflowRunner \ No newline at end of file From 63b99fb9cbe676df1aa37c37f1f14e26f1433946 Mon Sep 17 00:00:00 2001 From: Jack McCluskey <34928439+jrmccluskey@users.noreply.github.com> Date: Thu, 12 Oct 2023 14:43:38 -0400 Subject: [PATCH 123/435] Remove govulncheck for Go Test workflow (#28966) --- .github/workflows/go_tests.yml | 5 ----- 1 file changed, 5 deletions(-) diff --git a/.github/workflows/go_tests.yml b/.github/workflows/go_tests.yml index 66c57d219a83..02947eff5ca0 100644 --- a/.github/workflows/go_tests.yml +++ b/.github/workflows/go_tests.yml @@ -68,8 +68,3 @@ jobs: go install "honnef.co/go/tools/cmd/staticcheck@2023.1.3" cd sdks/go/pkg/beam $(go env GOPATH)/bin/staticcheck ./... - - uses: golang/govulncheck-action@v1.0.1 - with: - work-dir: ./sdks - go-package: ./... - go-version-input: 1.21 From 3e066e3524a7f8b04945e67750867d9021903cd9 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 12 Oct 2023 14:56:00 -0400 Subject: [PATCH 124/435] Bump com.gradle.common-custom-user-data-gradle-plugin (#28762) Bumps com.gradle.common-custom-user-data-gradle-plugin from 1.10 to 1.11.3. --- updated-dependencies: - dependency-name: com.gradle.common-custom-user-data-gradle-plugin dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- settings.gradle.kts | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/settings.gradle.kts b/settings.gradle.kts index d1069ec2d352..b71ce249c155 100644 --- a/settings.gradle.kts +++ b/settings.gradle.kts @@ -25,7 +25,7 @@ pluginManagement { plugins { id("com.gradle.enterprise") version "3.13.2" - id("com.gradle.common-custom-user-data-gradle-plugin") version "1.10" + id("com.gradle.common-custom-user-data-gradle-plugin") version "1.11.3" } From a94d29ffd21710eac3e88be9325346171368e219 Mon Sep 17 00:00:00 2001 From: Anand Inguva <34158215+AnandInguva@users.noreply.github.com> Date: Thu, 12 Oct 2023 19:14:39 +0000 Subject: [PATCH 125/435] Add pyproject.toml (#28385) * Add pyproject.toml Update numpy bounds * Use subprocess to run grpcio since it is not imported in pyproject.toml Update setup.py Remove ImportError from gen_protos.py Update subprocess run and raise RuntimeError if proto generation fails Print output of setup.py Fix linting issues * Remove build-requirements.txt and use build to build the sdist Modify buildPython task * Use wheels to run tox precommit tests Upgrade pip in Dockerfile Move _ELEMENTS to shared file. tests are not importable by each other Add missing element Remove shared_test_variables Remove installing wheel in a test suite Retry run_tox.sh with no installPkg flag Remove natural language test. codepath is covered in the postCommits. Add back tox exit code * Remove cython tests. default tests will run with Cython extensions FIx toxTask name Add no-extra test suite to precommit and remove GH duplicate ubuntu test Skip failing non-cython test Fix tox test name * Force type cast inputs to list * Update stager to use build. If it fails, use legacy setup to build sdist Fix mypy issue * Remove cython env and build-requirements for tox.ini --- .github/workflows/beam_PreCommit_Python.yml | 1 + .../beam_PreCommit_Python_Coverage.yml | 2 + .../beam_PreCommit_Python_Dataframes.yml | 1 + .../beam_PreCommit_Python_Examples.yml | 1 + .../beam_PreCommit_Python_Runners.yml | 1 + .../beam_PreCommit_Python_Transforms.yml | 1 + .github/workflows/build_release_candidate.yml | 2 +- .github/workflows/build_wheels.yml | 17 +-- .github/workflows/dask_runner_tests.yml | 8 +- .github/workflows/python_dependency_tests.yml | 5 +- .github/workflows/python_tests.yml | 16 +-- .github/workflows/run_perf_alert_tool.yml | 3 - .github/workflows/typescript_tests.yml | 7 +- .../jenkins/job_PreCommit_Python.groovy | 1 + .../job_PreCommit_Python_Coverage.groovy | 3 + .../job_PreCommit_Python_Dataframes.groovy | 3 +- .../job_PreCommit_Python_Examples.groovy | 3 +- .../job_PreCommit_Python_Runners.groovy | 3 +- .../job_PreCommit_Python_Transforms.groovy | 3 +- .../beam/gradle/BeamModulePlugin.groovy | 40 ++++--- .../main/scripts/build_release_candidate.sh | 2 +- .../apache_beam/coders/slow_coders_test.py | 3 + .../apache_beam/examples/inference/README.md | 1 - .../apache_beam/examples/kafkataxi/README.md | 4 +- .../io/azure/integration_test/Dockerfile | 2 +- .../apache_beam/io/gcp/bigquery_test.py | 39 ++++++- .../io/hdfs_integration_test/Dockerfile | 2 +- .../ml/gcp/naturallanguageml_test.py | 19 ---- sdks/python/apache_beam/runners/common.py | 4 +- .../apache_beam/runners/portability/stager.py | 28 +++-- sdks/python/build-requirements.txt | 28 ----- sdks/python/build.gradle | 7 +- sdks/python/container/Dockerfile | 2 +- .../base_image_requirements_manual.txt | 1 + sdks/python/gen_protos.py | 105 ++++++------------ sdks/python/pyproject.toml | 36 ++++++ sdks/python/scripts/run_pytest.sh | 4 +- sdks/python/scripts/run_tox.sh | 19 +++- sdks/python/setup.py | 59 ++++++---- sdks/python/test-suites/tox/common.gradle | 12 +- .../python/test-suites/tox/py310/build.gradle | 2 - .../python/test-suites/tox/py311/build.gradle | 2 - sdks/python/test-suites/tox/py38/build.gradle | 2 - sdks/python/test-suites/tox/py39/build.gradle | 3 - sdks/python/tox.ini | 32 +----- .../ml/multi-language-inference.md | 2 +- .../sdks/python-pipeline-dependencies.md | 13 ++- 47 files changed, 268 insertions(+), 286 deletions(-) delete mode 100644 sdks/python/build-requirements.txt create mode 100644 sdks/python/pyproject.toml diff --git a/.github/workflows/beam_PreCommit_Python.yml b/.github/workflows/beam_PreCommit_Python.yml index c891a79cefd0..35e7b937068d 100644 --- a/.github/workflows/beam_PreCommit_Python.yml +++ b/.github/workflows/beam_PreCommit_Python.yml @@ -97,6 +97,7 @@ jobs: arguments: | -Pposargs="--ignore=apache_beam/dataframe/ --ignore=apache_beam/examples/ --ignore=apache_beam/runners/ --ignore=apache_beam/transforms/" \ -PpythonVersion=${{ matrix.python_version }} \ + -PuseWheelDistribution - name: Archive code coverage results uses: actions/upload-artifact@v3 with: diff --git a/.github/workflows/beam_PreCommit_Python_Coverage.yml b/.github/workflows/beam_PreCommit_Python_Coverage.yml index 33be9644d34a..4b274d643e07 100644 --- a/.github/workflows/beam_PreCommit_Python_Coverage.yml +++ b/.github/workflows/beam_PreCommit_Python_Coverage.yml @@ -86,6 +86,8 @@ jobs: uses: ./.github/actions/gradle-command-self-hosted-action with: gradle-command: :sdks:python:test-suites:tox:py38:preCommitPyCoverage + arguments: | + -PuseWheelDistribution - name: Archive code coverage results uses: actions/upload-artifact@v3 with: diff --git a/.github/workflows/beam_PreCommit_Python_Dataframes.yml b/.github/workflows/beam_PreCommit_Python_Dataframes.yml index 2862d7d5936c..f03716d06795 100644 --- a/.github/workflows/beam_PreCommit_Python_Dataframes.yml +++ b/.github/workflows/beam_PreCommit_Python_Dataframes.yml @@ -97,6 +97,7 @@ jobs: arguments: | -Pposargs=apache_beam/dataframe/ \ -PpythonVersion=${{ matrix.python_version }} \ + -PuseWheelDistribution - name: Archive code coverage results uses: actions/upload-artifact@v3 with: diff --git a/.github/workflows/beam_PreCommit_Python_Examples.yml b/.github/workflows/beam_PreCommit_Python_Examples.yml index 7f980885180a..d629ee09b725 100644 --- a/.github/workflows/beam_PreCommit_Python_Examples.yml +++ b/.github/workflows/beam_PreCommit_Python_Examples.yml @@ -97,6 +97,7 @@ jobs: arguments: | -Pposargs=apache_beam/examples/ \ -PpythonVersion=${{ matrix.python_version }} \ + -PuseWheelDistribution - name: Archive code coverage results uses: actions/upload-artifact@v3 with: diff --git a/.github/workflows/beam_PreCommit_Python_Runners.yml b/.github/workflows/beam_PreCommit_Python_Runners.yml index b0c5ab4fa34a..f823112e23dd 100644 --- a/.github/workflows/beam_PreCommit_Python_Runners.yml +++ b/.github/workflows/beam_PreCommit_Python_Runners.yml @@ -97,6 +97,7 @@ jobs: arguments: | -Pposargs=apache_beam/runners/ \ -PpythonVersion=${{ matrix.python_version }} \ + -PuseWheelDistribution - name: Archive code coverage results uses: actions/upload-artifact@v3 with: diff --git a/.github/workflows/beam_PreCommit_Python_Transforms.yml b/.github/workflows/beam_PreCommit_Python_Transforms.yml index 88ac59c9de96..7374af7f38c9 100644 --- a/.github/workflows/beam_PreCommit_Python_Transforms.yml +++ b/.github/workflows/beam_PreCommit_Python_Transforms.yml @@ -97,6 +97,7 @@ jobs: arguments: | -Pposargs=apache_beam/transforms/ \ -PpythonVersion=${{ matrix.python_version }} \ + -PuseWheelDistribution - name: Archive code coverage results uses: actions/upload-artifact@v3 with: diff --git a/.github/workflows/build_release_candidate.yml b/.github/workflows/build_release_candidate.yml index 3956551431c6..12f1537dac18 100644 --- a/.github/workflows/build_release_candidate.yml +++ b/.github/workflows/build_release_candidate.yml @@ -314,7 +314,7 @@ jobs: pip install -U pip pip install tox # TODO(https://github.com/apache/beam/issues/20209): Don't hardcode py version in this file. - pip install -r build-requirements.txt && tox -e py38-docs + tox -e py38-docs rm -rf target/docs/_build/.doctrees - name: Build Typescript Docs working-directory: beam/sdks/typescript diff --git a/.github/workflows/build_wheels.yml b/.github/workflows/build_wheels.yml index 94248be0c008..f4ccf368bacb 100644 --- a/.github/workflows/build_wheels.yml +++ b/.github/workflows/build_wheels.yml @@ -92,11 +92,6 @@ jobs: uses: actions/setup-python@v4 with: python-version: 3.8 - - name: Get build dependencies - working-directory: ./sdks/python - run: python -m pip install -r build-requirements.txt - - name: Install wheels - run: python -m pip install wheel - name: Get tag id: get_tag run: | @@ -117,15 +112,15 @@ jobs: echo "RELEASE_VERSION=$RELEASE_VERSION" >> $GITHUB_OUTPUT - name: Build source working-directory: ./sdks/python - run: python setup.py sdist --formats=zip + run: pip install -U build && python -m build --sdist - name: Add checksums working-directory: ./sdks/python/dist run: | - file=$(ls | grep .zip | head -n 1) + file=$(ls | grep .tar.gz | head -n 1) sha512sum $file > ${file}.sha512 - name: Unzip source working-directory: ./sdks/python - run: unzip dist/$(ls dist | grep .zip | head -n 1) + run: tar -xzvf dist/$(ls dist | grep .tar.gz | head -n 1) - name: Rename source directory working-directory: ./sdks/python run: mv $(ls | grep apache-beam) apache-beam-source @@ -155,17 +150,17 @@ jobs: - name: Build RC source if: steps.is_rc.outputs.is_rc == 1 working-directory: ./sdks/python - run: python setup.py sdist --formats=zip + run: pip install -U build && pythom -m build --sdist - name: Add RC checksums if: steps.is_rc.outputs.is_rc == 1 working-directory: ./sdks/python/dist run: | - file=$(ls | grep .zip | head -n 1) + file=$(ls | grep .tar.gz | head -n 1) sha512sum $file > ${file}.sha512 - name: Unzip RC source if: steps.is_rc.outputs.is_rc == 1 working-directory: ./sdks/python - run: unzip dist/$(ls dist | grep .zip | head -n 1) + run: tar -xzvf dist/$(ls dist | grep .tar.gz | head -n 1) - name: Rename RC source directory if: steps.is_rc.outputs.is_rc == 1 working-directory: ./sdks/python diff --git a/.github/workflows/dask_runner_tests.yml b/.github/workflows/dask_runner_tests.yml index 423a304db825..35c320086992 100644 --- a/.github/workflows/dask_runner_tests.yml +++ b/.github/workflows/dask_runner_tests.yml @@ -44,12 +44,9 @@ jobs: uses: actions/setup-python@v4 with: python-version: 3.8 - - name: Get build dependencies - working-directory: ./sdks/python - run: pip install pip setuptools --upgrade && pip install -r build-requirements.txt - name: Build source working-directory: ./sdks/python - run: python setup.py sdist + run: pip install -U build && python -m build --sdist - name: Rename source file working-directory: ./sdks/python/dist run: mv $(ls | grep "apache-beam.*tar\.gz") apache-beam-source.tar.gz @@ -78,9 +75,6 @@ jobs: uses: actions/setup-python@v4 with: python-version: ${{ matrix.params.py_ver }} - - name: Get build dependencies - working-directory: ./sdks/python - run: pip install -r build-requirements.txt - name: Install tox run: pip install tox - name: Install SDK with dask diff --git a/.github/workflows/python_dependency_tests.yml b/.github/workflows/python_dependency_tests.yml index 6fd865bda754..166899df90cb 100644 --- a/.github/workflows/python_dependency_tests.yml +++ b/.github/workflows/python_dependency_tests.yml @@ -7,7 +7,7 @@ on: branches: ['master', 'release-*'] tags: 'v*' # paths where Beam Python's dependencies are configured. - paths: ['sdks/python/setup.py', 'sdks/python/build-requirements.txt', 'sdks/python/container/base_image_requirements_manual.txt'] + paths: ['sdks/python/setup.py', 'sdks/python/pyproject.toml', 'sdks/python/container/base_image_requirements_manual.txt'] # This allows a subsequently queued workflow run to interrupt previous runs concurrency: @@ -38,9 +38,6 @@ jobs: uses: actions/setup-python@v4 with: python-version: ${{ matrix.params.py_ver }} - - name: Install Build dependencies - working-directory: ./sdks/python - run: pip install -r build-requirements.txt - name: Install base_image_requirements.txt working-directory: ./sdks/python run: pip install --no-deps -r container/${{ matrix.params.py_env }}/base_image_requirements.txt diff --git a/.github/workflows/python_tests.yml b/.github/workflows/python_tests.yml index 57ec895c2431..0309329e84e1 100644 --- a/.github/workflows/python_tests.yml +++ b/.github/workflows/python_tests.yml @@ -78,12 +78,9 @@ jobs: uses: actions/setup-python@v4 with: python-version: 3.8 - - name: Get build dependencies - working-directory: ./sdks/python - run: pip install pip setuptools --upgrade && pip install -r build-requirements.txt - name: Build source working-directory: ./sdks/python - run: python setup.py sdist + run: pip install -U build && python -m build --sdist - name: Rename source file working-directory: ./sdks/python/dist run: mv $(ls | grep "apache-beam.*tar\.gz") apache-beam-source.tar.gz @@ -99,7 +96,7 @@ jobs: strategy: fail-fast: false matrix: - os: [ubuntu-latest, macos-latest, windows-latest] + os: [macos-latest, windows-latest] params: [ {"py_ver": "3.8", "tox_env": "py38"}, {"py_ver": "3.9", "tox_env": "py39"}, @@ -113,9 +110,6 @@ jobs: uses: actions/setup-python@v4 with: python-version: ${{ matrix.params.py_ver }} - - name: Get build dependencies - working-directory: ./sdks/python - run: pip install -r build-requirements.txt --use-pep517 - name: Install tox run: pip install tox - name: Run tests basic unix @@ -148,9 +142,6 @@ jobs: uses: actions/setup-python@v4 with: python-version: ${{ matrix.python }} - - name: Get build dependencies - working-directory: ./sdks/python - run: pip install -r build-requirements.txt - name: Install requirements working-directory: ./sdks/python run: pip install setuptools --upgrade && pip install -e . @@ -192,9 +183,6 @@ jobs: service_account_key: ${{ secrets.GCP_SA_KEY }} project_id: ${{ secrets.GCP_PROJECT_ID }} export_default_credentials: true - - name: Get build dependencies - working-directory: ./sdks/python - run: pip install -r build-requirements.txt - name: Install requirements working-directory: ./sdks/python run: pip install setuptools --upgrade && pip install -e ".[gcp]" diff --git a/.github/workflows/run_perf_alert_tool.yml b/.github/workflows/run_perf_alert_tool.yml index 1bd8d525c2fb..c61665c1bc7c 100644 --- a/.github/workflows/run_perf_alert_tool.yml +++ b/.github/workflows/run_perf_alert_tool.yml @@ -46,9 +46,6 @@ jobs: with: service_account_key: ${{ secrets.GCP_SA_KEY }} export_default_credentials: true - - name: Get Apache Beam Build dependencies - working-directory: ./sdks/python - run: pip install pip setuptools --upgrade && pip install -r build-requirements.txt - name: Install Apache Beam working-directory: ./sdks/python run: pip install -e .[gcp,test] diff --git a/.github/workflows/typescript_tests.yml b/.github/workflows/typescript_tests.yml index a4aa14c42efd..edbe8399e7d8 100644 --- a/.github/workflows/typescript_tests.yml +++ b/.github/workflows/typescript_tests.yml @@ -89,10 +89,8 @@ jobs: - name: Setup Beam Python working-directory: ./sdks/python run: | - pip install pip setuptools --upgrade - pip install -r build-requirements.txt pip install 'pandas>=1.0,<1.5' - python setup.py develop + pip install -e . - run: npm ci working-directory: ./sdks/typescript - run: npm run build @@ -146,10 +144,7 @@ jobs: - name: Setup Beam Python working-directory: ./sdks/python run: | - pip install pip setuptools --upgrade - pip install -r build-requirements.txt pip install 'pandas>=1.0,<1.5' - python setup.py develop pip install -e ".[gcp]" - name: Authenticate on GCP uses: google-github-actions/setup-gcloud@v0 diff --git a/.test-infra/jenkins/job_PreCommit_Python.groovy b/.test-infra/jenkins/job_PreCommit_Python.groovy index 0e439d788877..9c9740e3c97e 100644 --- a/.test-infra/jenkins/job_PreCommit_Python.groovy +++ b/.test-infra/jenkins/job_PreCommit_Python.groovy @@ -29,6 +29,7 @@ PrecommitJobBuilder builder = new PrecommitJobBuilder( '^release/.*$', ], gradleSwitches: [ + '-PuseWheelDistribution', '-Pposargs=\"--ignore=apache_beam/dataframe/ --ignore=apache_beam/examples/ --ignore=apache_beam/runners/ --ignore=apache_beam/transforms/\"' // All these tests are covered by different jobs. ], numBuildsToRetain: 40 diff --git a/.test-infra/jenkins/job_PreCommit_Python_Coverage.groovy b/.test-infra/jenkins/job_PreCommit_Python_Coverage.groovy index c0cb48cf6231..43a204fd7cfc 100644 --- a/.test-infra/jenkins/job_PreCommit_Python_Coverage.groovy +++ b/.test-infra/jenkins/job_PreCommit_Python_Coverage.groovy @@ -22,6 +22,9 @@ PrecommitJobBuilder builder = new PrecommitJobBuilder( scope: this, nameBase: 'Python_Coverage', gradleTask: ':sdks:python:test-suites:tox:py38:preCommitPyCoverage', + gradleSwitches: [ + '-PuseWheelDistribution' + ], timeoutMins: 180, triggerPathPatterns: [ '^model/.*$', diff --git a/.test-infra/jenkins/job_PreCommit_Python_Dataframes.groovy b/.test-infra/jenkins/job_PreCommit_Python_Dataframes.groovy index e2914e9bdb8e..dea034f613a5 100644 --- a/.test-infra/jenkins/job_PreCommit_Python_Dataframes.groovy +++ b/.test-infra/jenkins/job_PreCommit_Python_Dataframes.groovy @@ -23,7 +23,8 @@ PrecommitJobBuilder builder = new PrecommitJobBuilder( nameBase: 'Python_Dataframes', gradleTask: ':pythonPreCommit', gradleSwitches: [ - '-Pposargs=apache_beam/dataframe/' + '-Pposargs=apache_beam/dataframe/', + '-PuseWheelDistribution' ], timeoutMins: 180, triggerPathPatterns: [ diff --git a/.test-infra/jenkins/job_PreCommit_Python_Examples.groovy b/.test-infra/jenkins/job_PreCommit_Python_Examples.groovy index f4ef9f51d7fb..3dd7bf6f6f47 100644 --- a/.test-infra/jenkins/job_PreCommit_Python_Examples.groovy +++ b/.test-infra/jenkins/job_PreCommit_Python_Examples.groovy @@ -23,7 +23,8 @@ PrecommitJobBuilder builder = new PrecommitJobBuilder( nameBase: 'Python_Examples', gradleTask: ':pythonPreCommit', gradleSwitches: [ - '-Pposargs=apache_beam/examples/' + '-Pposargs=apache_beam/examples/', + '-PuseWheelDistribution' ], timeoutMins: 180, triggerPathPatterns: [ diff --git a/.test-infra/jenkins/job_PreCommit_Python_Runners.groovy b/.test-infra/jenkins/job_PreCommit_Python_Runners.groovy index e80dba6cf5cd..4ae1d283b7a9 100644 --- a/.test-infra/jenkins/job_PreCommit_Python_Runners.groovy +++ b/.test-infra/jenkins/job_PreCommit_Python_Runners.groovy @@ -23,7 +23,8 @@ PrecommitJobBuilder builder = new PrecommitJobBuilder( nameBase: 'Python_Runners', gradleTask: ':pythonPreCommit', gradleSwitches: [ - '-Pposargs=apache_beam/runners/' + '-Pposargs=apache_beam/runners/', + '-PuseWheelDistribution' ], timeoutMins: 180, triggerPathPatterns: [ diff --git a/.test-infra/jenkins/job_PreCommit_Python_Transforms.groovy b/.test-infra/jenkins/job_PreCommit_Python_Transforms.groovy index dd16d48b1731..ccd3f08b78ab 100644 --- a/.test-infra/jenkins/job_PreCommit_Python_Transforms.groovy +++ b/.test-infra/jenkins/job_PreCommit_Python_Transforms.groovy @@ -23,7 +23,8 @@ PrecommitJobBuilder builder = new PrecommitJobBuilder( nameBase: 'Python_Transforms', gradleTask: ':pythonPreCommit', gradleSwitches: [ - '-Pposargs=apache_beam/transforms/' + '-Pposargs=apache_beam/transforms/', + '-PuseWheelDistribution' ], timeoutMins: 180, triggerPathPatterns: [ diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index c32717aae725..6fa5ff0ee5f3 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -2998,7 +2998,7 @@ class BeamModulePlugin implements Plugin { executable 'sh' args '-c', ". ${project.ext.envdir}/bin/activate && " + "pip install --pre --retries 10 --upgrade pip && " + - "pip install --pre --retries 10 --upgrade tox -r ${project.rootDir}/sdks/python/build-requirements.txt" + "pip install --pre --retries 10 --upgrade tox" } } // Gradle will delete outputs whenever it thinks they are stale. Putting a @@ -3081,30 +3081,40 @@ class BeamModulePlugin implements Plugin { } return argList.join(' ') } - project.ext.toxTask = { name, tox_env, posargs='' -> project.tasks.register(name) { dependsOn setupVirtualenv dependsOn ':sdks:python:sdist' - - doLast { - // Python source directory is also tox execution workspace, We want - // to isolate them per tox suite to avoid conflict when running - // multiple tox suites in parallel. - project.copy { from project.pythonSdkDeps; into copiedSrcRoot } - - def copiedPyRoot = "${copiedSrcRoot}/sdks/python" - def distTarBall = "${pythonRootDir}/build/apache-beam.tar.gz" - project.exec { - executable 'sh' - args '-c', ". ${project.ext.envdir}/bin/activate && cd ${copiedPyRoot} && scripts/run_tox.sh $tox_env $distTarBall '$posargs'" + if (project.hasProperty('useWheelDistribution')) { + def pythonVersionNumber = project.ext.pythonVersion.replace('.', '') + dependsOn ":sdks:python:bdistPy${pythonVersionNumber}linux" + doLast { + project.copy { from project.pythonSdkDeps; into copiedSrcRoot } + def copiedPyRoot = "${copiedSrcRoot}/sdks/python" + def collection = project.fileTree(project.project(':sdks:python').buildDir){ + include "**/apache_beam-*cp${pythonVersionNumber}*manylinux*.whl" + } + String packageFilename = collection.singleFile.toString() + project.exec { + executable 'sh' + args '-c', ". ${project.ext.envdir}/bin/activate && cd ${copiedPyRoot} && scripts/run_tox.sh $tox_env ${packageFilename} '$posargs' " + } + } + } else { + // tox task will run in editable mode, which is configured in the tox.ini file. + doLast { + project.copy { from project.pythonSdkDeps; into copiedSrcRoot } + def copiedPyRoot = "${copiedSrcRoot}/sdks/python" + project.exec { + executable 'sh' + args '-c', ". ${project.ext.envdir}/bin/activate && cd ${copiedPyRoot} && scripts/run_tox.sh $tox_env '$posargs'" + } } } inputs.files project.pythonSdkDeps outputs.files project.fileTree(dir: "${pythonRootDir}/target/.tox/${tox_env}/log/") } } - // Run single or a set of integration tests with provided test options and pipeline options. project.ext.enablePythonPerformanceTest = { diff --git a/release/src/main/scripts/build_release_candidate.sh b/release/src/main/scripts/build_release_candidate.sh index 057a38833adc..d0e6310f50aa 100755 --- a/release/src/main/scripts/build_release_candidate.sh +++ b/release/src/main/scripts/build_release_candidate.sh @@ -346,7 +346,7 @@ if [[ $confirmation = "y" ]]; then cd ${BEAM_ROOT_DIR} RELEASE_COMMIT=$(git rev-list -n 1 "tags/${RC_TAG}") # TODO(https://github.com/apache/beam/issues/20209): Don't hardcode py version in this file. - cd sdks/python && pip install -r build-requirements.txt && tox -e py38-docs + cd sdks/python && tox -e py38-docs GENERATED_PYDOC=~/${LOCAL_WEBSITE_UPDATE_DIR}/${LOCAL_PYTHON_DOC}/${BEAM_ROOT_DIR}/sdks/python/target/docs/_build rm -rf ${GENERATED_PYDOC}/.doctrees diff --git a/sdks/python/apache_beam/coders/slow_coders_test.py b/sdks/python/apache_beam/coders/slow_coders_test.py index fe1c707a62e5..7915116a19a3 100644 --- a/sdks/python/apache_beam/coders/slow_coders_test.py +++ b/sdks/python/apache_beam/coders/slow_coders_test.py @@ -25,6 +25,9 @@ from apache_beam.coders.coders_test_common import * +@unittest.skip( + 'Remove non-cython tests.' + 'https://github.com/apache/beam/issues/28307') class SlowCoders(unittest.TestCase): def test_using_slow_impl(self): try: diff --git a/sdks/python/apache_beam/examples/inference/README.md b/sdks/python/apache_beam/examples/inference/README.md index 19262dead586..cd92d9c127ee 100644 --- a/sdks/python/apache_beam/examples/inference/README.md +++ b/sdks/python/apache_beam/examples/inference/README.md @@ -29,7 +29,6 @@ Some examples are also used in [our benchmarks](http://s.apache.org/beam-communi You must have the latest (possibly unreleased) `apache-beam` or greater installed from the Beam repo in order to run these pipelines, because some examples rely on the latest features that are actively in development. To install Beam, run the following from the `sdks/python` directory: ``` -pip install -r build-requirements.txt pip install -e .[gcp] ``` diff --git a/sdks/python/apache_beam/examples/kafkataxi/README.md b/sdks/python/apache_beam/examples/kafkataxi/README.md index c4e808cad8b4..72a8d8f85c03 100644 --- a/sdks/python/apache_beam/examples/kafkataxi/README.md +++ b/sdks/python/apache_beam/examples/kafkataxi/README.md @@ -157,9 +157,9 @@ Install Beam and dependencies and build a Beam distribution. ```sh cd beam/sdks/python -pip install -r build-requirements.txt pip install -e '.[gcp]' -python setup.py sdist +pip install -q build +python -m build --sdist ``` Run the Beam pipeline. You can either use the default Kafka topic name or specify diff --git a/sdks/python/apache_beam/io/azure/integration_test/Dockerfile b/sdks/python/apache_beam/io/azure/integration_test/Dockerfile index e9ac396b8e17..257fa72cb668 100644 --- a/sdks/python/apache_beam/io/azure/integration_test/Dockerfile +++ b/sdks/python/apache_beam/io/azure/integration_test/Dockerfile @@ -32,7 +32,7 @@ COPY sdks/python /app/sdks/python COPY model /app/model # This step should look like setupVirtualenv minus virtualenv creation. -RUN pip install --no-cache-dir tox -r sdks/python/build-requirements.txt +RUN pip install --no-cache-dir tox # Add Azurite's self-signed cert to the global CA cert store. COPY cert.pem /usr/local/share/ca-certificates/azurite.crt diff --git a/sdks/python/apache_beam/io/gcp/bigquery_test.py b/sdks/python/apache_beam/io/gcp/bigquery_test.py index 7e9c1e634748..95b6c2a5fa60 100644 --- a/sdks/python/apache_beam/io/gcp/bigquery_test.py +++ b/sdks/python/apache_beam/io/gcp/bigquery_test.py @@ -49,14 +49,12 @@ from apache_beam.io.gcp.bigquery import TableRowJsonCoder from apache_beam.io.gcp.bigquery import WriteToBigQuery from apache_beam.io.gcp.bigquery import _StreamToBigQuery -from apache_beam.io.gcp.bigquery_file_loads_test import _ELEMENTS from apache_beam.io.gcp.bigquery_read_internal import _JsonToDictCoder from apache_beam.io.gcp.bigquery_read_internal import bigquery_export_destination_uri from apache_beam.io.gcp.bigquery_tools import JSON_COMPLIANCE_ERROR from apache_beam.io.gcp.bigquery_tools import BigQueryWrapper from apache_beam.io.gcp.bigquery_tools import RetryStrategy from apache_beam.io.gcp.internal.clients import bigquery -from apache_beam.io.gcp.internal.clients.bigquery import bigquery_v2_client from apache_beam.io.gcp.pubsub import ReadFromPubSub from apache_beam.io.gcp.tests import utils from apache_beam.io.gcp.tests.bigquery_matcher import BigqueryFullResultMatcher @@ -82,6 +80,7 @@ # pylint: disable=wrong-import-order, wrong-import-position try: + from apache_beam.io.gcp.internal.clients.bigquery import bigquery_v2_client from apitools.base.py.exceptions import HttpError from google.cloud import bigquery as gcp_bigquery from google.api_core import exceptions @@ -93,6 +92,42 @@ _LOGGER = logging.getLogger(__name__) +_ELEMENTS = [ + { + 'name': 'beam', 'language': 'py' + }, + { + 'name': 'beam', 'language': 'java' + }, + { + 'name': 'beam', 'language': 'go' + }, + { + 'name': 'flink', 'language': 'java' + }, + { + 'name': 'flink', 'language': 'scala' + }, + { + 'name': 'spark', 'language': 'scala' + }, + { + 'name': 'spark', 'language': 'py' + }, + { + 'name': 'spark', 'language': 'scala' + }, + { + 'name': 'beam', 'foundation': 'apache' + }, + { + 'name': 'flink', 'foundation': 'apache' + }, + { + 'name': 'spark', 'foundation': 'apache' + }, +] + def _load_or_default(filename): try: diff --git a/sdks/python/apache_beam/io/hdfs_integration_test/Dockerfile b/sdks/python/apache_beam/io/hdfs_integration_test/Dockerfile index 487d5c3487ab..ab7940563394 100644 --- a/sdks/python/apache_beam/io/hdfs_integration_test/Dockerfile +++ b/sdks/python/apache_beam/io/hdfs_integration_test/Dockerfile @@ -30,7 +30,7 @@ COPY sdks/python /app/sdks/python COPY model /app/model # This step should look like setupVirtualenv minus virtualenv creation. -RUN pip install --no-cache-dir tox -r sdks/python/build-requirements.txt +RUN pip install --no-cache-dir tox # Run wordcount, and write results to HDFS. CMD cd sdks/python && tox -e hdfs_integration_test diff --git a/sdks/python/apache_beam/ml/gcp/naturallanguageml_test.py b/sdks/python/apache_beam/ml/gcp/naturallanguageml_test.py index bad7443d0d94..891726cb2688 100644 --- a/sdks/python/apache_beam/ml/gcp/naturallanguageml_test.py +++ b/sdks/python/apache_beam/ml/gcp/naturallanguageml_test.py @@ -20,11 +20,7 @@ import unittest -import mock - -import apache_beam as beam from apache_beam.metrics import MetricsFilter -from apache_beam.testing.test_pipeline import TestPipeline # Protect against environments where Google Cloud Natural Language client # is not available. @@ -60,21 +56,6 @@ def test_document_source(self): self.assertFalse('content' in dict_) self.assertTrue('gcs_content_uri' in dict_) - def test_annotate_test_called(self): - with mock.patch('apache_beam.ml.gcp.naturallanguageml._AnnotateTextFn' - '._get_api_client'): - p = TestPipeline() - features = [ - naturallanguageml.language_v1.AnnotateTextRequest.Features( - extract_syntax=True) - ] - _ = ( - p | beam.Create([naturallanguageml.Document('Hello, world!')]) - | naturallanguageml.AnnotateText(features)) - result = p.run() - result.wait_until_finish() - self.assertCounterEqual(result, 'api_calls', 1) - if __name__ == '__main__': unittest.main() diff --git a/sdks/python/apache_beam/runners/common.py b/sdks/python/apache_beam/runners/common.py index 99cd26cc4098..ed0dc2d9a0c1 100644 --- a/sdks/python/apache_beam/runners/common.py +++ b/sdks/python/apache_beam/runners/common.py @@ -765,6 +765,7 @@ def __init__(self, # Try to prepare all the arguments that can just be filled in # without any additional work. in the process function. # Also cache all the placeholders needed in the process function. + input_args = list(input_args) ( self.placeholders_for_process, self.args_for_process, @@ -1437,7 +1438,8 @@ def process(self, windowed_value): return [] def _maybe_sample_exception( - self, exn: BaseException, windowed_value: WindowedValue) -> None: + self, exn: BaseException, + windowed_value: Optional[WindowedValue]) -> None: if self.execution_context is None: return diff --git a/sdks/python/apache_beam/runners/portability/stager.py b/sdks/python/apache_beam/runners/portability/stager.py index ace573de0a62..4afe5eaaa370 100644 --- a/sdks/python/apache_beam/runners/portability/stager.py +++ b/sdks/python/apache_beam/runners/portability/stager.py @@ -49,6 +49,7 @@ import glob import hashlib +import importlib.util import logging import os import shutil @@ -771,13 +772,26 @@ def _build_setup_package(setup_file, # type: str try: os.chdir(os.path.dirname(setup_file)) if build_setup_args is None: - build_setup_args = [ - Stager._get_python_executable(), - os.path.basename(setup_file), - 'sdist', - '--dist-dir', - temp_dir - ] + # if build is installed in the user env, use it to + # build the sdist else fallback to legacy setup.py sdist call. + if importlib.util.find_spec('build'): + build_setup_args = [ + Stager._get_python_executable(), + '-m', + 'build', + '--sdist', + '--outdir', + temp_dir, + os.path.dirname(setup_file), + ] + else: + build_setup_args = [ + Stager._get_python_executable(), + os.path.basename(setup_file), + 'sdist', + '--dist-dir', + temp_dir + ] _LOGGER.info('Executing command: %s', build_setup_args) processes.check_output(build_setup_args) output_files = glob.glob(os.path.join(temp_dir, '*.tar.gz')) diff --git a/sdks/python/build-requirements.txt b/sdks/python/build-requirements.txt deleted file mode 100644 index 4fe47079d8d0..000000000000 --- a/sdks/python/build-requirements.txt +++ /dev/null @@ -1,28 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -# TODO(https://github.com/apache/beam/issues/20051): Consider PEP-517/PEP-518 instead of this file. - -setuptools -wheel>=0.36.0 -grpcio-tools==1.53.0 -mypy-protobuf==3.5.0 -# Avoid https://github.com/pypa/virtualenv/issues/2006 -distlib==0.3.7 - -# Numpy headers -numpy>=1.14.3,<1.26 diff --git a/sdks/python/build.gradle b/sdks/python/build.gradle index 762bed268d63..7795e77e3963 100644 --- a/sdks/python/build.gradle +++ b/sdks/python/build.gradle @@ -30,7 +30,8 @@ def buildPython = tasks.register("buildPython") { logger.info('Building Python Dependencies') exec { executable 'sh' - args '-c', ". ${envdir}/bin/activate && python setup.py build --build-base ${buildDir}" + // args '-c', ". ${envdir}/bin/activate && python setup.py build --build-base ${buildDir}" + args '-c', ". ${envdir}/bin/activate && pip install -e ." } } } @@ -46,7 +47,7 @@ def sdist = tasks.register("sdist") { // Build artifact exec { executable 'sh' - args '-c', ". ${envdir}/bin/activate && python setup.py -q sdist --formats zip,gztar --dist-dir ${buildDir}" + args '-c', ". ${envdir}/bin/activate && pip install -U build && python -m build --sdist --outdir=${buildDir}" } def collection = fileTree(buildDir){ include "**/*${project.sdk_version}*.tar.gz" exclude 'srcs/**'} @@ -96,7 +97,6 @@ platform_identifiers_map.each { platform, idsuffix -> exec { environment CIBW_BUILD: "cp${pyversion}-${idsuffix}" environment CIBW_ENVIRONMENT: "SETUPTOOLS_USE_DISTUTILS=stdlib" - environment CIBW_BEFORE_BUILD: "pip install cython==0.29.36 numpy --config-settings=setup-args='-Dallow-noblas=true' && pip install --upgrade setuptools" // note: sync cibuildwheel version with GitHub Action // .github/workflow/build_wheel.yml:build_wheels "Install cibuildwheel" step executable 'sh' @@ -110,6 +110,7 @@ platform_identifiers_map.each { platform, idsuffix -> } } + /*************************************************************************************************/ // Non-testing builds and analysis tasks diff --git a/sdks/python/container/Dockerfile b/sdks/python/container/Dockerfile index 73d83343e033..a49933ee6604 100644 --- a/sdks/python/container/Dockerfile +++ b/sdks/python/container/Dockerfile @@ -45,7 +45,7 @@ RUN \ && \ rm -rf /var/lib/apt/lists/* && \ - pip install --upgrade setuptools && \ + pip install --upgrade pip setuptools wheel && \ # Install required packages for Beam Python SDK and common dependencies used by users. # use --no-deps to ensure the list includes all transitive dependencies. diff --git a/sdks/python/container/base_image_requirements_manual.txt b/sdks/python/container/base_image_requirements_manual.txt index e952b2126604..f2f3ea44b44c 100644 --- a/sdks/python/container/base_image_requirements_manual.txt +++ b/sdks/python/container/base_image_requirements_manual.txt @@ -43,3 +43,4 @@ nose==1.3.7 # For Dataflow internal testing. TODO: remove this. python-snappy;python_version<"3.11" # Optimizes execution of some Beam codepaths. scipy scikit-learn +build>=1.0,<2 # tool to build sdist from setup.py in stager. \ No newline at end of file diff --git a/sdks/python/gen_protos.py b/sdks/python/gen_protos.py index 94d80c8d263b..2b488af0afb5 100644 --- a/sdks/python/gen_protos.py +++ b/sdks/python/gen_protos.py @@ -18,7 +18,7 @@ """ Generates Python proto modules and grpc stubs for Beam protos. """ - +import argparse import contextlib import glob import inspect @@ -27,9 +27,7 @@ import platform import re import shutil -import subprocess import sys -import time from collections import defaultdict from importlib import import_module @@ -60,7 +58,7 @@ NO_PROMISES_NOTICE = """ \"\"\" For internal use only; no backwards-compatibility guarantees. -Automatically generated when running setup.py sdist or build[_py]. +Automatically generated when running python -m build. \"\"\" """ @@ -321,43 +319,6 @@ def find_by_ext(root_dir, ext): if file.endswith(ext): yield clean_path(os.path.join(root, file)) - -def ensure_grpcio_exists(): - try: - from grpc_tools import protoc # pylint: disable=unused-import - except ImportError: - return _install_grpcio_tools() - - -def _install_grpcio_tools(): - """ - Though wheels are available for grpcio-tools, setup_requires uses - easy_install which doesn't understand them. This means that it is - compiled from scratch (which is expensive as it compiles the full - protoc compiler). Instead, we attempt to install a wheel in a temporary - directory and add it to the path as needed. - See https://github.com/pypa/setuptools/issues/377 - """ - install_path = os.path.join(PYTHON_SDK_ROOT, '.eggs', 'grpcio-wheels') - logging.warning('Installing grpcio-tools into %s', install_path) - start = time.time() - subprocess.check_call([ - sys.executable, - '-m', - 'pip', - 'install', - '--target', - install_path, - '--upgrade', - '-r', - os.path.join(PYTHON_SDK_ROOT, 'build-requirements.txt') - ]) - logging.warning( - 'Installing grpcio-tools took %0.2f seconds.', time.time() - start) - - return install_path - - def build_relative_import(root_path, import_path, start_file_path): tail_path = import_path.replace('.', os.path.sep) source_path = os.path.join(root_path, tail_path) @@ -511,33 +472,31 @@ def generate_proto_files(force=False): if not os.path.exists(PYTHON_OUTPUT_PATH): os.mkdir(PYTHON_OUTPUT_PATH) - grpcio_install_loc = ensure_grpcio_exists() protoc_gen_mypy = _find_protoc_gen_mypy() - with PythonPath(grpcio_install_loc): - from grpc_tools import protoc - builtin_protos = pkg_resources.resource_filename('grpc_tools', '_proto') - args = ( - [sys.executable] + # expecting to be called from command line - ['--proto_path=%s' % builtin_protos] + - ['--proto_path=%s' % d - for d in proto_dirs] + ['--python_out=%s' % PYTHON_OUTPUT_PATH] + - ['--plugin=protoc-gen-mypy=%s' % protoc_gen_mypy] + - # new version of mypy-protobuf converts None to zero default value - # and remove Optional from the param type annotation. This causes - # some mypy errors. So to mitigate and fall back to old behavior, - # use `relax_strict_optional_primitives` flag. more at - # https://github.com/nipunn1313/mypy-protobuf/tree/main#relax_strict_optional_primitives # pylint:disable=line-too-long - ['--mypy_out=relax_strict_optional_primitives:%s' % PYTHON_OUTPUT_PATH - ] + - # TODO(robertwb): Remove the prefix once it's the default. - ['--grpc_python_out=grpc_2_0:%s' % PYTHON_OUTPUT_PATH] + proto_files) - - LOG.info('Regenerating Python proto definitions (%s).' % regenerate_reason) - ret_code = protoc.main(args) - if ret_code: - raise RuntimeError( - 'Protoc returned non-zero status (see logs for details): ' - '%s' % ret_code) + from grpc_tools import protoc + builtin_protos = pkg_resources.resource_filename('grpc_tools', '_proto') + args = ( + [sys.executable] + # expecting to be called from command line + ['--proto_path=%s' % builtin_protos] + + ['--proto_path=%s' % d + for d in proto_dirs] + ['--python_out=%s' % PYTHON_OUTPUT_PATH] + + ['--plugin=protoc-gen-mypy=%s' % protoc_gen_mypy] + + # new version of mypy-protobuf converts None to zero default value + # and remove Optional from the param type annotation. This causes + # some mypy errors. So to mitigate and fall back to old behavior, + # use `relax_strict_optional_primitives` flag. more at + # https://github.com/nipunn1313/mypy-protobuf/tree/main#relax_strict_optional_primitives # pylint:disable=line-too-long + ['--mypy_out=relax_strict_optional_primitives:%s' % PYTHON_OUTPUT_PATH + ] + + # TODO(robertwb): Remove the prefix once it's the default. + ['--grpc_python_out=grpc_2_0:%s' % PYTHON_OUTPUT_PATH] + proto_files) + + LOG.info('Regenerating Python proto definitions (%s).' % regenerate_reason) + ret_code = protoc.main(args) + if ret_code: + raise RuntimeError( + 'Protoc returned non-zero status (see logs for details): ' + '%s' % ret_code) # copy resource files for path in MODEL_RESOURCES: @@ -548,7 +507,7 @@ def generate_proto_files(force=False): # force relative import paths for proto files compiled_import_re = re.compile('^from (.*) import (.*)$') for file_path in find_by_ext(PYTHON_OUTPUT_PATH, - ('_pb2.py', '_pb2_grpc.py', '_pb2.pyi')): + ('_pb2.py', '_pb2_grpc.py', '_pb2.pyi')): proto_packages.add(os.path.dirname(file_path)) lines = [] with open(file_path, encoding='utf-8') as f: @@ -566,12 +525,14 @@ def generate_proto_files(force=False): f.writelines(lines) generate_init_files_lite(PYTHON_OUTPUT_PATH) - with PythonPath(grpcio_install_loc): - for proto_package in proto_packages: - generate_urn_files(proto_package, PYTHON_OUTPUT_PATH) + for proto_package in proto_packages: + generate_urn_files(proto_package, PYTHON_OUTPUT_PATH) generate_init_files_full(PYTHON_OUTPUT_PATH) if __name__ == '__main__': - generate_proto_files(force=True) + parser = argparse.ArgumentParser() + parser.add_argument('--no-force', dest='force', action='store_false') + args = parser.parse_args() + generate_proto_files(force=args.force) diff --git a/sdks/python/pyproject.toml b/sdks/python/pyproject.toml new file mode 100644 index 000000000000..d185c45f6191 --- /dev/null +++ b/sdks/python/pyproject.toml @@ -0,0 +1,36 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# since we rely on setuptools and according to https://peps.python.org/pep-0518/#build-system-table +# this is the minimum requirements for the build system to execute. +[build-system] +requires = [ + "setuptools", + "wheel>=0.36.0", + "grpcio-tools==1.53.0", + "mypy-protobuf==3.5.0", + # Avoid https://github.com/pypa/virtualenv/issues/2006 + "distlib==0.3.7", + # Numpy headers + "numpy>=1.14.3,<1.25", # Update setup.py as well. + # having cython here will create wheels that are platform dependent. + "cython==0.29.36", +] + + +# legacy installation is needed to generate `apache_beam.portability.api` package. +build-backend = "setuptools.build_meta" \ No newline at end of file diff --git a/sdks/python/scripts/run_pytest.sh b/sdks/python/scripts/run_pytest.sh index 01f2318164c4..ad35b48972b6 100755 --- a/sdks/python/scripts/run_pytest.sh +++ b/sdks/python/scripts/run_pytest.sh @@ -42,10 +42,10 @@ echo "posargs: $posargs" # Run with pytest-xdist and without. pytest -o junit_suite_name=${envname} \ - --junitxml=pytest_${envname}.xml -m 'not no_xdist' -n 6 ${pytest_args} --pyargs ${posargs} + --junitxml=pytest_${envname}.xml -m 'not no_xdist' -n 6 --import-mode=importlib ${pytest_args} --pyargs ${posargs} status1=$? pytest -o junit_suite_name=${envname}_no_xdist \ - --junitxml=pytest_${envname}_no_xdist.xml -m 'no_xdist' ${pytest_args} --pyargs ${posargs} + --junitxml=pytest_${envname}_no_xdist.xml -m 'no_xdist' --import-mode=importlib ${pytest_args} --pyargs ${posargs} status2=$? # Exit with error if no tests were run in either suite (status code 5). diff --git a/sdks/python/scripts/run_tox.sh b/sdks/python/scripts/run_tox.sh index ebbacf5494ea..ac60f26b32ba 100755 --- a/sdks/python/scripts/run_tox.sh +++ b/sdks/python/scripts/run_tox.sh @@ -53,12 +53,21 @@ if [[ "$JENKINS_HOME" != "" ]]; then export PY_COLORS=1 fi -if [[ ! -z $2 ]]; then +# Determine if the second argument is SDK_LOCATION or posargs +if [[ -f "$1" ]]; then # Check if the argument corresponds to a file SDK_LOCATION="$1" - shift; - tox -c tox.ini run --recreate -e "$TOX_ENVIRONMENT" --installpkg "$SDK_LOCATION" -- "$@" -else - tox -c tox.ini run --recreate -e "$TOX_ENVIRONMENT" + shift +fi + +# If SDK_LOCATION is identified and there are still arguments left, those are posargs. +if [[ ! -z "$SDK_LOCATION" ]]; then + if [[ $# -gt 0 ]]; then # There are posargs + tox -c tox.ini run --recreate -e "$TOX_ENVIRONMENT" --installpkg "$SDK_LOCATION" -- "$@" + else + tox -c tox.ini run --recreate -e "$TOX_ENVIRONMENT" --installpkg "$SDK_LOCATION" + fi +else # No SDK_LOCATION; all arguments are posargs + tox -c tox.ini run --recreate -e "$TOX_ENVIRONMENT" -- "$@" fi exit_code=$? diff --git a/sdks/python/setup.py b/sdks/python/setup.py index ca585ccf7167..4a05544526fc 100644 --- a/sdks/python/setup.py +++ b/sdks/python/setup.py @@ -18,6 +18,7 @@ """Apache Beam SDK for Python setup file.""" import os +import subprocess import sys import warnings # Pylint and isort disagree here. @@ -62,7 +63,6 @@ def get_project_path(self): return os.path.join(project_path, to_filename(ei_cmd.egg_name)) def run(self): - import subprocess args = ['mypy', self.get_project_path()] result = subprocess.call(args) if result != 0: @@ -155,12 +155,18 @@ def cythonize(*args, **kwargs): # We must generate protos after setup_requires are installed. def generate_protos_first(): try: - # pylint: disable=wrong-import-position - import gen_protos - gen_protos.generate_proto_files() - - except ImportError: - warnings.warn("Could not import gen_protos, skipping proto generation.") + # Pyproject toml build happens in isolated environemnts. In those envs, + # gen_protos is unable to get imported. so we run a subprocess call. + cwd = os.path.abspath(os.path.dirname(__file__)) + out = subprocess.run([ + sys.executable, + os.path.join(cwd, 'gen_protos.py'), + '--no-force' + ], capture_output=True, check=True) + print(out.stdout) + except subprocess.CalledProcessError as err: + raise RuntimeError('Could not generate protos due to error: %s', + err.stderr) def get_portability_package_data(): @@ -188,6 +194,27 @@ def get_portability_package_data(): # structure must exist before the call to setuptools.find_packages() # executes below. generate_protos_first() + + # generate cythonize extensions only if we are building a wheel or + # building an extension or running in editable mode. + cythonize_cmds = ('bdist_wheel', 'build_ext', 'editable_wheel') + if any(cmd in sys.argv for cmd in cythonize_cmds): + extensions = cythonize([ + 'apache_beam/**/*.pyx', + 'apache_beam/coders/coder_impl.py', + 'apache_beam/metrics/cells.py', + 'apache_beam/metrics/execution.py', + 'apache_beam/runners/common.py', + 'apache_beam/runners/worker/logger.py', + 'apache_beam/runners/worker/opcounters.py', + 'apache_beam/runners/worker/operations.py', + 'apache_beam/transforms/cy_combiners.py', + 'apache_beam/transforms/stats.py', + 'apache_beam/utils/counters.py', + 'apache_beam/utils/windowed_value.py', + ]) + else: + extensions = [] # Keep all dependencies inlined in the setup call, otherwise Dependabot won't # be able to parse it. setuptools.setup( @@ -213,21 +240,7 @@ def get_portability_package_data(): *get_portability_package_data() ] }, - ext_modules=cythonize([ - 'apache_beam/**/*.pyx', - 'apache_beam/coders/coder_impl.py', - 'apache_beam/metrics/cells.py', - 'apache_beam/metrics/execution.py', - 'apache_beam/runners/common.py', - 'apache_beam/runners/worker/logger.py', - 'apache_beam/runners/worker/opcounters.py', - 'apache_beam/runners/worker/operations.py', - 'apache_beam/transforms/cy_combiners.py', - 'apache_beam/transforms/stats.py', - 'apache_beam/utils/counters.py', - 'apache_beam/utils/windowed_value.py', - ], - language_level=3), + ext_modules=extensions, install_requires=[ 'crcmod>=1.7,<2.0', 'orjson>=3.9.7,<4', @@ -250,7 +263,7 @@ def get_portability_package_data(): 'js2py>=0.74,<1', # numpy can have breaking changes in minor versions. # Use a strict upper bound. - 'numpy>=1.14.3,<1.25.0', # Update build-requirements.txt as well. + 'numpy>=1.14.3,<1.25.0', # Update pyproject.toml as well. 'objsize>=0.6.1,<0.7.0', 'packaging>=22.0', 'pymongo>=3.8.0,<5.0.0', diff --git a/sdks/python/test-suites/tox/common.gradle b/sdks/python/test-suites/tox/common.gradle index ee183dff4064..3fdd0c0c553b 100644 --- a/sdks/python/test-suites/tox/common.gradle +++ b/sdks/python/test-suites/tox/common.gradle @@ -29,18 +29,12 @@ test.dependsOn "testPy${pythonVersionSuffix}Cloud" // toxTask "testPy${pythonVersionSuffix}Dask", "py${pythonVersionSuffix}-dask", "${posargs}" // test.dependsOn "testPy${pythonVersionSuffix}Dask" -toxTask "testPy${pythonVersionSuffix}Cython", "py${pythonVersionSuffix}-cython", "${posargs}" -test.dependsOn "testPy${pythonVersionSuffix}Cython" toxTask "testPy38CloudCoverage", "py38-cloudcoverage", "${posargs}" test.dependsOn "testPy38CloudCoverage" project.tasks.register("preCommitPy${pythonVersionSuffix}") { - // Since codecoverage reports will always be generated for py38, - // all tests will be exercised. - if (pythonVersionSuffix.equals('38')) { - dependsOn = ["testPy38Cython"] - } else { - dependsOn = ["testPy${pythonVersionSuffix}Cloud", "testPy${pythonVersionSuffix}Cython"] - } + // Since codecoverage reports will always be generated for py38, + // all tests will be exercised. + dependsOn = ["testPy${pythonVersionSuffix}Cloud", "testPython${pythonVersionSuffix}"] } \ No newline at end of file diff --git a/sdks/python/test-suites/tox/py310/build.gradle b/sdks/python/test-suites/tox/py310/build.gradle index ea10fde831c6..f1e40a17951f 100644 --- a/sdks/python/test-suites/tox/py310/build.gradle +++ b/sdks/python/test-suites/tox/py310/build.gradle @@ -28,5 +28,3 @@ pythonVersion = '3.10' apply from: "../common.gradle" -// TODO(https://github.com/apache/beam/issues/20051): Remove this once tox uses isolated builds. -testPy310Cython.mustRunAfter testPython310, testPy310Cloud diff --git a/sdks/python/test-suites/tox/py311/build.gradle b/sdks/python/test-suites/tox/py311/build.gradle index 1bb3766500bb..fabf9fd4365a 100644 --- a/sdks/python/test-suites/tox/py311/build.gradle +++ b/sdks/python/test-suites/tox/py311/build.gradle @@ -28,5 +28,3 @@ pythonVersion = '3.11' apply from: "../common.gradle" -// TODO(https://github.com/apache/beam/issues/20051): Remove this once tox uses isolated builds. -testPy311Cython.mustRunAfter testPython311, testPy311Cloud diff --git a/sdks/python/test-suites/tox/py38/build.gradle b/sdks/python/test-suites/tox/py38/build.gradle index bc4aa99c79b4..b1ed5f88c7c9 100644 --- a/sdks/python/test-suites/tox/py38/build.gradle +++ b/sdks/python/test-suites/tox/py38/build.gradle @@ -43,8 +43,6 @@ lint.dependsOn mypyPy38 apply from: "../common.gradle" -// TODO(https://github.com/apache/beam/issues/20051): Remove this once tox uses isolated builds. -testPy38Cython.mustRunAfter testPython38, testPy38CloudCoverage // PyCoverage Precommit runs test suites that evaluate test coverage and compatibility of diff --git a/sdks/python/test-suites/tox/py39/build.gradle b/sdks/python/test-suites/tox/py39/build.gradle index 380cc1486daa..5bb73b60a5d2 100644 --- a/sdks/python/test-suites/tox/py39/build.gradle +++ b/sdks/python/test-suites/tox/py39/build.gradle @@ -27,6 +27,3 @@ applyPythonNature() pythonVersion = '3.9' apply from: "../common.gradle" - -// TODO(https://github.com/apache/beam/issues/20051): Remove this once tox uses isolated builds. -testPy39Cython.mustRunAfter testPython39, testPy39Cloud diff --git a/sdks/python/tox.ini b/sdks/python/tox.ini index 1caf25caf080..1e797d96074f 100644 --- a/sdks/python/tox.ini +++ b/sdks/python/tox.ini @@ -17,7 +17,7 @@ [tox] # new environments will be excluded by default unless explicitly added to envlist. -envlist = py38,py39,py310,py311,py38-{cloud,cython,docs,lint,mypy,cloudcoverage,dask},py39-{cloud,cython},py310-{cloud,cython,dask},py311-{cloud,cython,dask},whitespacelint +envlist = py38,py39,py310,py311,py38-{cloud,docs,lint,mypy,cloudcoverage,dask},py39-{cloud},py310-{cloud,dask},py311-{cloud,dask},whitespacelint toxworkdir = {toxinidir}/target/{env:ENV_NAME:.tox} [pycodestyle] @@ -44,9 +44,6 @@ allowlist_externals = curl ./codecov chmod -deps = - cython: cython==0.29.33 - -r build-requirements.txt setenv = RUN_SKIPPED_PY3_TESTS=0 # Use an isolated tmp dir for tests that get slowed down by scanning /tmp. @@ -67,6 +64,7 @@ commands_pre = bash {toxinidir}/scripts/run_tox_cleanup.sh commands_post = bash {toxinidir}/scripts/run_tox_cleanup.sh + commands = false {envname} is misconfigured [testenv:py{38,39,310,311}] @@ -81,28 +79,18 @@ commands = install_command = {envbindir}/python.exe {envbindir}/pip.exe install --retries 10 {opts} {packages} list_dependencies_command = {envbindir}/python.exe {envbindir}/pip.exe freeze -[testenv:py{38,39,310,311}-cython] -# cython tests are only expected to work in linux (2.x and 3.x) -# If we want to add other platforms in the future, it should be: -# `platform = linux2|darwin|...` -# See https://docs.python.org/2/library/sys.html#sys.platform for platform codes -platform = linux -commands = - # TODO(https://github.com/apache/beam/issues/20051): Remove this build_ext invocation once local source no longer - # shadows the installed apache_beam. - python setup.py build_ext --inplace - python apache_beam/examples/complete/autocomplete_test.py - bash {toxinidir}/scripts/run_pytest.sh {envname} "{posargs}" - [testenv:py{38,39,310,311}-cloud] +; extras = test,gcp,interactive,dataframe,aws,azure extras = test,gcp,interactive,dataframe,aws,azure commands = + python apache_beam/examples/complete/autocomplete_test.py bash {toxinidir}/scripts/run_pytest.sh {envname} "{posargs}" [testenv:py{38,39,310,311}-dask] extras = test,dask commands = bash {toxinidir}/scripts/run_pytest.sh {envname} "{posargs}" + [testenv:py38-cloudcoverage] deps = pytest-cov==3.0.0 @@ -124,7 +112,6 @@ commands = setenv = # keep the version of pylint in sync with the 'rev' in .pre-commit-config.yaml deps = - -r build-requirements.txt astroid<2.17.0,>=2.15.6 pycodestyle==2.8.0 pylint==2.17.5 @@ -143,7 +130,6 @@ commands = [testenv:py38-mypy] deps = - -r build-requirements.txt mypy==0.790 dask==2022.01.0 distributed==2022.01.0 @@ -173,7 +159,6 @@ commands = # Used by hdfs_integration_test.sh. Do not run this directly, as it depends on # nodes defined in hdfs_integration_test/docker-compose.yml. deps = - -r build-requirements.txt holdup==1.8.0 extras = gcp @@ -206,7 +191,6 @@ commands_pre = # Do not run this directly, as it depends on nodes defined in # azure/integration_test/docker-compose.yml. deps = - -r build-requirements.txt extras = azure passenv = REQUESTS_CA_BUNDLE @@ -335,7 +319,6 @@ commands = [testenv:py{38,39,310,311}-pytorch-{19,110,111,112,113}] deps = - -r build-requirements.txt 19: torch>=1.9.0,<1.10.0 110: torch>=1.10.0,<1.11.0 111: torch>=1.11.0,<1.12.0 @@ -353,7 +336,6 @@ commands = [testenv:py{38,39,310}-pytorch-200] deps = - -r build-requirements.txt 200: torch>=2.0.0,<2.1.0 extras = test,gcp # Don't set TMPDIR to avoid "AF_UNIX path too long" errors in certain tests. @@ -387,7 +369,6 @@ commands = [testenv:py{38,39,310}-tensorflow-212] deps = - -r build-requirements.txt 212: tensorflow>=2.12rc1,<2.13 extras = test,gcp commands = @@ -399,7 +380,6 @@ commands = [testenv:py{38,39,310}-xgboost-{160,170}] deps = - -r build-requirements.txt 160: xgboost>=1.6.0,<1.7.0 datatable==1.0.0 @@ -416,7 +396,6 @@ commands = [testenv:py{38,39,310,311}-transformers-{428,429,430}] deps = - -r build-requirements.txt 428: transformers>=4.28.0,<4.29.0 429: transformers>=4.29.0,<4.30.0 430: transformers>=4.30.0,<4.31.0 @@ -434,7 +413,6 @@ commands = [testenv:py{38,311}-vertex-ai] deps = - -r build-requirements.txt tensorflow==2.12.0 extras = test,gcp commands = diff --git a/website/www/site/content/en/documentation/ml/multi-language-inference.md b/website/www/site/content/en/documentation/ml/multi-language-inference.md index 0d7a972e0765..1480b37ab484 100644 --- a/website/www/site/content/en/documentation/ml/multi-language-inference.md +++ b/website/www/site/content/en/documentation/ml/multi-language-inference.md @@ -99,7 +99,7 @@ Finally, we postprocess the model predictions in the `Postprocess` DoFn. The `Po The custom Python code needs to be written in a local package and be compiled as a tarball. This package can then be used by the Java pipeline. The following example shows how to compile the Python package into a tarball: ```bash - python setup.py sdist + pip install --upgrade build && python -m build --sdist ``` In order to run this, a `setup.py` is required. The path to the tarball will be used as an argument in the pipeline options of the Java pipeline. diff --git a/website/www/site/content/en/documentation/sdks/python-pipeline-dependencies.md b/website/www/site/content/en/documentation/sdks/python-pipeline-dependencies.md index 378032ab6b58..c99c0b9c7cf8 100644 --- a/website/www/site/content/en/documentation/sdks/python-pipeline-dependencies.md +++ b/website/www/site/content/en/documentation/sdks/python-pipeline-dependencies.md @@ -66,16 +66,17 @@ If your pipeline uses packages that are not available publicly (e.g. packages th This command lists all packages that are installed on your machine, regardless of where they were installed from. -2. Run your pipeline with the following command-line option: + 1. Run your pipeline with the following command-line option: - --extra_package /path/to/package/package-name + --extra_package /path/to/package/package-name - where package-name is the package's tarball. If you have the `setup.py` for that - package then you can build the tarball with the following command: + where package-name is the package's tarball. You can build the package tarball using a command line tool called [build](https://setuptools.pypa.io/en/latest/userguide/quickstart.html#install-build). - python setup.py sdist + # Install build using pip + pip install --upgrade build + python -m build --sdist - See the [sdist documentation](https://docs.python.org/3/distutils/sourcedist.html) for more details on this command. + See the [build documentation](https://pypa-build.readthedocs.io/en/latest/index.html) for more details on this command. ## Multiple File Dependencies From 05861613f484a6159c21796006c52d8da5e10b2a Mon Sep 17 00:00:00 2001 From: Rebecca Szper <98840847+rszper@users.noreply.github.com> Date: Thu, 12 Oct 2023 13:20:39 -0700 Subject: [PATCH 126/435] Fix typo and remove extra button (#28965) * Fix typo and remove extra button * Remove button cell --- examples/notebooks/beam-ml/mltransform_basic.ipynb | 12 +----------- 1 file changed, 1 insertion(+), 11 deletions(-) diff --git a/examples/notebooks/beam-ml/mltransform_basic.ipynb b/examples/notebooks/beam-ml/mltransform_basic.ipynb index fd305bddb3ba..e44be91fe1cd 100644 --- a/examples/notebooks/beam-ml/mltransform_basic.ipynb +++ b/examples/notebooks/beam-ml/mltransform_basic.ipynb @@ -1,15 +1,5 @@ { "cells": [ - { - "cell_type": "markdown", - "metadata": { - "id": "view-in-github", - "colab_type": "text" - }, - "source": [ - "\"Open" - ] - }, { "cell_type": "code", "source": [ @@ -77,7 +67,7 @@ "id": "f0097dbd-2657-4cbe-a334-e0401816db01" }, "source": [ - "## Import the requried modules\n", + "## Import the required modules\n", "\n", "To use `MLTransfrom`, install `tensorflow_transform` and the Apache Beam SDK version 2.50.0 or later.\n" ] From a7c6728b004f667a860b31d38c36fc3a6a2cc7ea Mon Sep 17 00:00:00 2001 From: Vitaly Terentyev Date: Fri, 13 Oct 2023 16:19:07 +0400 Subject: [PATCH 127/435] Fix publish snapshots github actions (#28961) * Add authenticate on GCP step * Fix docker registry --- .github/workflows/beam_Publish_Beam_SDK_Snapshots.yml | 11 +++++++++++ .github/workflows/beam_Publish_Docker_Snapshots.yml | 11 +++++++++++ 2 files changed, 22 insertions(+) diff --git a/.github/workflows/beam_Publish_Beam_SDK_Snapshots.yml b/.github/workflows/beam_Publish_Beam_SDK_Snapshots.yml index 7aa5f6d4ceee..e2cfbd9f0a45 100644 --- a/.github/workflows/beam_Publish_Beam_SDK_Snapshots.yml +++ b/.github/workflows/beam_Publish_Beam_SDK_Snapshots.yml @@ -45,6 +45,7 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + docker_registry: gcr.io/apache-beam-testing/beam-sdk jobs: beam_Publish_Beam_SDK_Snapshots: @@ -68,6 +69,16 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.container_task }}) + - name: Authenticate on GCP + uses: google-github-actions/setup-gcloud@v0 + with: + service_account_email: ${{ secrets.GCP_SA_EMAIL }} + service_account_key: ${{ secrets.GCP_SA_KEY }} + project_id: ${{ secrets.GCP_PROJECT_ID }} + export_default_credentials: true + - name: GCloud Docker credential helper + run: | + gcloud auth configure-docker ${{ env.docker_registry }} - name: Setup Java environment if: ${{ startsWith(matrix.container_task, 'java') }} uses: ./.github/actions/setup-environment-action diff --git a/.github/workflows/beam_Publish_Docker_Snapshots.yml b/.github/workflows/beam_Publish_Docker_Snapshots.yml index 75ff5df3de47..3b6e996a87a5 100644 --- a/.github/workflows/beam_Publish_Docker_Snapshots.yml +++ b/.github/workflows/beam_Publish_Docker_Snapshots.yml @@ -45,6 +45,7 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + docker_registry: gcr.io/apache-beam-testing/beam_portability jobs: beam_Publish_Docker_Snapshots: @@ -67,6 +68,16 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Authenticate on GCP + uses: google-github-actions/setup-gcloud@v0 + with: + service_account_email: ${{ secrets.GCP_SA_EMAIL }} + service_account_key: ${{ secrets.GCP_SA_KEY }} + project_id: ${{ secrets.GCP_PROJECT_ID }} + export_default_credentials: true + - name: GCloud Docker credential helper + run: | + gcloud auth configure-docker ${{ env.docker_registry }} - name: run Publish Docker Snapshots script for Spark uses: ./.github/actions/gradle-command-self-hosted-action with: From a342d2817a91f5cdc91fde0ed66c37e3a2449cfd Mon Sep 17 00:00:00 2001 From: magicgoody <131876064+magicgoody@users.noreply.github.com> Date: Fri, 13 Oct 2023 19:03:52 +0600 Subject: [PATCH 128/435] Github Workflow Replacement for Jenkins Jobs, beam_LoadTests_Python_GBK_Flink_Batch; beam_Java_LoadTests_Combine_Smoke_PR (#28897) --- .../beam_Java_LoadTests_Combine_Smoke.yml | 105 ++++++++++++ .../beam_LoadTests_Python_GBK_Flink_Batch.yml | 155 ++++++++++++++++++ ..._PerformanceTests_xlang_KafkaIO_Python.yml | 4 +- ...am_PostCommit_XVR_GoUsingJava_Dataflow.yml | 1 + ...e_CombineLoadTest_load_test_Dataflow-1.txt | 26 +++ ...e_CombineLoadTest_load_test_Dataflow-2.txt | 26 +++ ...e_CombineLoadTest_load_test_Dataflow-3.txt | 26 +++ ...on_GBK_Flink_Batch_2GB_of_100B_records.txt | 28 ++++ ...hon_GBK_Flink_Batch_2GB_of_10B_records.txt | 28 ++++ ...4_times_with_2GB_10-byte_records_total.txt | 28 ++++ ...8_times_with_2GB_10-byte_records_total.txt | 28 ++++ ...nk_Batch_reiterate_4_times_10kB_values.txt | 28 ++++ 12 files changed, 481 insertions(+), 2 deletions(-) create mode 100644 .github/workflows/beam_Java_LoadTests_Combine_Smoke.yml create mode 100644 .github/workflows/beam_LoadTests_Python_GBK_Flink_Batch.yml create mode 100644 .github/workflows/load-tests-job-configs/java_LoadTests_Combine_Smoke_CombineLoadTest_load_test_Dataflow-1.txt create mode 100644 .github/workflows/load-tests-job-configs/java_LoadTests_Combine_Smoke_CombineLoadTest_load_test_Dataflow-2.txt create mode 100644 .github/workflows/load-tests-job-configs/java_LoadTests_Combine_Smoke_CombineLoadTest_load_test_Dataflow-3.txt create mode 100644 .github/workflows/load-tests-job-configs/python_GBK_Flink_Batch_2GB_of_100B_records.txt create mode 100644 .github/workflows/load-tests-job-configs/python_GBK_Flink_Batch_2GB_of_10B_records.txt create mode 100644 .github/workflows/load-tests-job-configs/python_GBK_Flink_Batch_fanout_4_times_with_2GB_10-byte_records_total.txt create mode 100644 .github/workflows/load-tests-job-configs/python_GBK_Flink_Batch_fanout_8_times_with_2GB_10-byte_records_total.txt create mode 100644 .github/workflows/load-tests-job-configs/python_GBK_Flink_Batch_reiterate_4_times_10kB_values.txt diff --git a/.github/workflows/beam_Java_LoadTests_Combine_Smoke.yml b/.github/workflows/beam_Java_LoadTests_Combine_Smoke.yml new file mode 100644 index 000000000000..fd63141fcdff --- /dev/null +++ b/.github/workflows/beam_Java_LoadTests_Combine_Smoke.yml @@ -0,0 +1,105 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: LoadTests Java Combine Smoke + +on: + # issue_comment: + # types: [created] + # schedule: + # - cron: '1 1 * * *' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_Java_LoadTests_Combine_Smoke: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Java Load Tests Combine Smoke' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 720 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_Java_LoadTests_Combine_Smoke"] + job_phrase: ["Run Java Load Tests Combine Smoke"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: java + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_LoadTests_Combine_Smoke_CombineLoadTest_load_test_Dataflow-1.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_LoadTests_Combine_Smoke_CombineLoadTest_load_test_Dataflow-2.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_LoadTests_Combine_Smoke_CombineLoadTest_load_test_Dataflow-3.txt + # The env variables are created and populated in the test-arguments-action as "_test_arguments_" + - name: run CombineLoadTest load test Dataflow-1 + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CombineLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_Java_LoadTests_Combine_Smoke_test_arguments_1 }}' \ + - name: run CombineLoadTest load test Dataflow-2 + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CombineLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_Java_LoadTests_Combine_Smoke_test_arguments_2 }}' \ + - name: run CombineLoadTest load test Dataflow-3 + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CombineLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_Java_LoadTests_Combine_Smoke_test_arguments_3 }}' \ No newline at end of file diff --git a/.github/workflows/beam_LoadTests_Python_GBK_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Python_GBK_Flink_Batch.yml new file mode 100644 index 000000000000..cba3bf9bd24e --- /dev/null +++ b/.github/workflows/beam_LoadTests_Python_GBK_Flink_Batch.yml @@ -0,0 +1,155 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: LoadTests Python GBK Flink Batch + +on: + # issue_comment: + # types: [created] + # schedule: + # - cron: '1 1 * * *' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + GCLOUD_ZONE: us-central1-a + CLUSTER_NAME: beam-loadtests-py-gbk-flk-batch-${{ github.run_id }} + GCS_BUCKET: gs://beam-flink-cluster + FLINK_DOWNLOAD_URL: https://archive.apache.org/dist/flink/flink-1.15.0/flink-1.15.0-bin-scala_2.12.tgz + HADOOP_DOWNLOAD_URL: https://repo.maven.apache.org/maven2/org/apache/flink/flink-shaded-hadoop-2-uber/2.8.3-10.0/flink-shaded-hadoop-2-uber-2.8.3-10.0.jar + FLINK_TASKMANAGER_SLOTS: 1 + DETACHED_MODE: true + HARNESS_IMAGES_TO_PULL: gcr.io/apache-beam-testing/beam-sdk/beam_go_sdk:latest + JOB_SERVER_IMAGE: gcr.io/apache-beam-testing/beam_portability/beam_flink1.15_job_server:latest + ARTIFACTS_DIR: gs://beam-flink-cluster/beam-loadtests-py-gbk-flk-batch-${{ github.run_id }} + +jobs: + beam_LoadTests_Python_GBK_Flink_Batch: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Load Tests Python GBK Flink Batch' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 720 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_LoadTests_Python_GBK_Flink_Batch"] + job_phrase: ["Run Load Tests Python GBK Flink Batch"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action + with: + python-version: 3.8 + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: python + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_GBK_Flink_Batch_2GB_of_10B_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_GBK_Flink_Batch_2GB_of_100B_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_GBK_Flink_Batch_fanout_4_times_with_2GB_10-byte_records_total.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_GBK_Flink_Batch_fanout_8_times_with_2GB_10-byte_records_total.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_GBK_Flink_Batch_reiterate_4_times_10kB_values.txt + - name: get current time + run: echo "NOW_UTC=$(date '+%m%d%H%M%S' --utc)" >> $GITHUB_ENV + - name: Start Flink with parallelism 5 + env: + FLINK_NUM_WORKERS: 5 + run: | + cd ${{ github.workspace }}/.test-infra/dataproc; ./flink_cluster.sh create + # The env variables are created and populated in the test-arguments-action as "_test_arguments_" + - name: run Flink Batch 2GB of 10B records test + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run --info + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.group_by_key_test \ + -Prunner=PortableRunner \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_GBK_Flink_Batch_test_arguments_1 }} --job_name=load-tests-python-flink-batch-gbk-1-${{env.NOW_UTC}}' \ + - name: run Flink Batch 2GB of 100B records test + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.group_by_key_test \ + -Prunner=PortableRunner \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_GBK_Flink_Batch_test_arguments_2 }} --job_name=load-tests-python-flink-batch-gbk-2-${{env.NOW_UTC}}' \ + - name: run reiterate 4 times 10kB values test + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.group_by_key_test \ + -Prunner=PortableRunner \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_GBK_Flink_Batch_test_arguments_5 }} --job_name=load-tests-python-flink-batch-gbk-6-${{env.NOW_UTC}}' \ + - name: Restart Flink with parallelism 16 + env: + FLINK_NUM_WORKERS: 16 + run: | + cd ${{ github.workspace }}/.test-infra/dataproc; ./flink_cluster.sh restart + - name: run fanout 4 times with 2GB 10-byte records total test + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.group_by_key_test \ + -Prunner=PortableRunner \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_GBK_Flink_Batch_test_arguments_3 }} --job_name=load-tests-python-flink-batch-gbk-4-${{env.NOW_UTC}}' \ + - name: run fanout 8 times with 2GB 10-byte records total test + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.group_by_key_test \ + -Prunner=PortableRunner \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_GBK_Flink_Batch_test_arguments_4 }} --job_name=load-tests-python-flink-batch-gbk-5-${{env.NOW_UTC}}' \ + - name: Teardown Flink + if: always() + run: | + ${{ github.workspace }}/.test-infra/dataproc/flink_cluster.sh delete + + # TODO(https://github.com/apache/beam/issues/20146) Re-enable auto builds after these tests pass. \ No newline at end of file diff --git a/.github/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml b/.github/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml index ee84de4ac618..b28a1824d88a 100644 --- a/.github/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml +++ b/.github/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml @@ -51,14 +51,14 @@ jobs: if: | github.event_name == 'workflow_dispatch' || github.event_name == 'schedule' || - github.event.comment.body == 'Run Java CompressedTextIO Performance Test' + github.event.comment.body == 'Run Python xlang KafkaIO Performance Test' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 240 name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) strategy: matrix: job_name: ["beam_PerformanceTests_xlang_KafkaIO_Python"] - job_phrase: ["Run Java CompressedTextIO Performance Test"] + job_phrase: ["Run Python xlang KafkaIO Performance Test"] steps: - uses: actions/checkout@v3 - name: Setup repository diff --git a/.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml b/.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml index be10e5fb1bb3..6231c683c89e 100644 --- a/.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml @@ -73,6 +73,7 @@ jobs: python-version: '3.8' - name: run XVR GoUsingJava Dataflow script env: + USER: github-actions CLOUDSDK_CONFIG: ${{ env.KUBELET_GCLOUD_CONFIG_PATH}} uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/load-tests-job-configs/java_LoadTests_Combine_Smoke_CombineLoadTest_load_test_Dataflow-1.txt b/.github/workflows/load-tests-job-configs/java_LoadTests_Combine_Smoke_CombineLoadTest_load_test_Dataflow-1.txt new file mode 100644 index 000000000000..7cd503ad4499 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_LoadTests_Combine_Smoke_CombineLoadTest_load_test_Dataflow-1.txt @@ -0,0 +1,26 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=smoke-dsl-java +--tempLocation=gs://temp-storage-for-perf-tests/smoketests +--sourceOptions={"numRecords":100000,"splitPointFrequencyRecords":1} +--stepOptions={"outputRecordsPerInputRecord":1,"preservesInputKeyDistribution":true} +--fanout=10 +--iterations=1 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_LoadTests_Combine_Smoke_CombineLoadTest_load_test_Dataflow-2.txt b/.github/workflows/load-tests-job-configs/java_LoadTests_Combine_Smoke_CombineLoadTest_load_test_Dataflow-2.txt new file mode 100644 index 000000000000..bccffadea8d7 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_LoadTests_Combine_Smoke_CombineLoadTest_load_test_Dataflow-2.txt @@ -0,0 +1,26 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=smoke-dsl-java +--tempLocation=gs://temp-storage-for-perf-tests/smoketests +--sourceOptions={"numRecords":100000,"keySizeBytes":1,"valueSizeBytes":1} +--fanout=1 +--iterations=1 +--numWorkers=3 +--autoscalingAlgorithm=NONE +--streaming=false +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_LoadTests_Combine_Smoke_CombineLoadTest_load_test_Dataflow-3.txt b/.github/workflows/load-tests-job-configs/java_LoadTests_Combine_Smoke_CombineLoadTest_load_test_Dataflow-3.txt new file mode 100644 index 000000000000..44bd342b462c --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_LoadTests_Combine_Smoke_CombineLoadTest_load_test_Dataflow-3.txt @@ -0,0 +1,26 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=smoke-dsl-java +--tempLocation=gs://temp-storage-for-perf-tests/smoketests +--sourceOptions={"numRecords":20000,"keySizeBytes":1,"valueSizeBytes":1} +--fanout=10 +--iterations=1 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=false +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_GBK_Flink_Batch_2GB_of_100B_records.txt b/.github/workflows/load-tests-job-configs/python_GBK_Flink_Batch_2GB_of_100B_records.txt new file mode 100644 index 000000000000..4cb5bfb0d988 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_GBK_Flink_Batch_2GB_of_100B_records.txt @@ -0,0 +1,28 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_flink_batch_GBK_2 +--influx_measurement=python_batch_gbk_2 +--input_options=''{\\"num_records\\":20000000,\\"key_size\\":10,\\"value_size\\":90}'' +--iterations=1 +--fanout=4 +--parallelism=5 +--job_endpoint=localhost:8099 +--environment_type=DOCKER +--environment_config=gcr.io/apache-beam-testing/beam-sdk/beam_python3.8_sdk:latest +--runner=PortableRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_GBK_Flink_Batch_2GB_of_10B_records.txt b/.github/workflows/load-tests-job-configs/python_GBK_Flink_Batch_2GB_of_10B_records.txt new file mode 100644 index 000000000000..2427e21cde45 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_GBK_Flink_Batch_2GB_of_10B_records.txt @@ -0,0 +1,28 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_flink_batch_GBK_1 +--influx_measurement=python_batch_gbk_1 +--input_options=''{\\"num_records\\":200000000,\\"key_size\\":1,\\"value_size\\":9}'' +--iterations=1 +--fanout=1 +--parallelism=5 +--job_endpoint=localhost:8099 +--environment_type=DOCKER +--environment_config=gcr.io/apache-beam-testing/beam-sdk/beam_python3.8_sdk:latest +--runner=PortableRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_GBK_Flink_Batch_fanout_4_times_with_2GB_10-byte_records_total.txt b/.github/workflows/load-tests-job-configs/python_GBK_Flink_Batch_fanout_4_times_with_2GB_10-byte_records_total.txt new file mode 100644 index 000000000000..bf9085141eab --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_GBK_Flink_Batch_fanout_4_times_with_2GB_10-byte_records_total.txt @@ -0,0 +1,28 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_flink_batch_GBK_4 +--influx_measurement=python_batch_gbk_4 +--input_options=''{\\"num_records\\":5000000,\\"key_size\\":10,\\"value_size\\":90}'' +--iterations=1 +--fanout=4 +--parallelism=16 +--job_endpoint=localhost:8099 +--environment_type=DOCKER +--environment_config=gcr.io/apache-beam-testing/beam-sdk/beam_python3.8_sdk:latest +--runner=PortableRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_GBK_Flink_Batch_fanout_8_times_with_2GB_10-byte_records_total.txt b/.github/workflows/load-tests-job-configs/python_GBK_Flink_Batch_fanout_8_times_with_2GB_10-byte_records_total.txt new file mode 100644 index 000000000000..a59f873eb775 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_GBK_Flink_Batch_fanout_8_times_with_2GB_10-byte_records_total.txt @@ -0,0 +1,28 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_flink_batch_GBK_5 +--influx_measurement=python_batch_gbk_5 +--input_options=''{\\"num_records\\":2500000,\\"key_size\\":10,\\"value_size\\":90,\\"algorithm\\":\\"lcg\\"}'' +--iterations=1 +--fanout=8 +--parallelism=16 +--job_endpoint=localhost:8099 +--environment_type=DOCKER +--environment_config=gcr.io/apache-beam-testing/beam-sdk/beam_python3.8_sdk:latest +--runner=PortableRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_GBK_Flink_Batch_reiterate_4_times_10kB_values.txt b/.github/workflows/load-tests-job-configs/python_GBK_Flink_Batch_reiterate_4_times_10kB_values.txt new file mode 100644 index 000000000000..0e5d00b96151 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_GBK_Flink_Batch_reiterate_4_times_10kB_values.txt @@ -0,0 +1,28 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_flink_batch_GBK_6 +--influx_measurement=python_batch_gbk_6 +--input_options=''{\\"num_records\\":20000000,\\"key_size\\":10,\\"value_size\\":90,\\"num_hot_keys\\":200,\\"hot_key_fraction\\":1,\\"algorithm\\":\\"lcg\\"}'' +--iterations=4 +--fanout=1 +--parallelism=5 +--job_endpoint=localhost:8099 +--environment_type=DOCKER +--environment_config=gcr.io/apache-beam-testing/beam-sdk/beam_python3.8_sdk:latest +--runner=PortableRunner \ No newline at end of file From 2dc766512c060faac71f9efcdc97710e00bb3615 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 13 Oct 2023 09:12:21 -0400 Subject: [PATCH 129/435] Bump github.com/aws/aws-sdk-go-v2 from 1.21.1 to 1.21.2 in /sdks (#28978) Bumps [github.com/aws/aws-sdk-go-v2](https://github.com/aws/aws-sdk-go-v2) from 1.21.1 to 1.21.2. - [Release notes](https://github.com/aws/aws-sdk-go-v2/releases) - [Commits](https://github.com/aws/aws-sdk-go-v2/compare/v1.21.1...v1.21.2) --- updated-dependencies: - dependency-name: github.com/aws/aws-sdk-go-v2 dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 2 +- sdks/go.sum | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index edb524545bd5..5b0d0c8af6ce 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -30,7 +30,7 @@ require ( cloud.google.com/go/pubsub v1.33.0 cloud.google.com/go/spanner v1.50.0 cloud.google.com/go/storage v1.33.0 - github.com/aws/aws-sdk-go-v2 v1.21.1 + github.com/aws/aws-sdk-go-v2 v1.21.2 github.com/aws/aws-sdk-go-v2/config v1.18.44 github.com/aws/aws-sdk-go-v2/credentials v1.13.42 github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.89 diff --git a/sdks/go.sum b/sdks/go.sum index 3cdcc9ef8e99..fe5e8d348bfe 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -81,8 +81,9 @@ github.com/aws/aws-sdk-go v1.30.19/go.mod h1:5zCpMtNQVjRREroY7sYe8lOMRSxkhG6MZve github.com/aws/aws-sdk-go v1.34.0 h1:brux2dRrlwCF5JhTL7MUT3WUwo9zfDHZZp3+g3Mvlmo= github.com/aws/aws-sdk-go v1.34.0/go.mod h1:5zCpMtNQVjRREroY7sYe8lOMRSxkhG6MZveU8YkpAk0= github.com/aws/aws-sdk-go-v2 v1.7.1/go.mod h1:L5LuPC1ZgDr2xQS7AmIec/Jlc7O/Y1u2KxJyNVab250= -github.com/aws/aws-sdk-go-v2 v1.21.1 h1:wjHYshtPpYOZm+/mu3NhVgRRc0baM6LJZOmxPZ5Cwzs= github.com/aws/aws-sdk-go-v2 v1.21.1/go.mod h1:ErQhvNuEMhJjweavOYhxVkn2RUx7kQXVATHrjKtxIpM= +github.com/aws/aws-sdk-go-v2 v1.21.2 h1:+LXZ0sgo8quN9UOKXXzAWRT3FWd4NxeXWOZom9pE7GA= +github.com/aws/aws-sdk-go-v2 v1.21.2/go.mod h1:ErQhvNuEMhJjweavOYhxVkn2RUx7kQXVATHrjKtxIpM= github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.4.14 h1:Sc82v7tDQ/vdU1WtuSyzZ1I7y/68j//HJ6uozND1IDs= github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.4.14/go.mod h1:9NCTOURS8OpxvoAVHq79LK81/zC78hfRWFn+aL0SPcY= github.com/aws/aws-sdk-go-v2/config v1.5.0/go.mod h1:RWlPOAW3E3tbtNAqTwvSW54Of/yP3oiZXMI0xfUdjyA= From 3a749a9a6ba7c39709e67874887904462dd0bd8a Mon Sep 17 00:00:00 2001 From: Aleksandr Dudko <116064902+aleksandr-dudko@users.noreply.github.com> Date: Fri, 13 Oct 2023 17:42:13 +0400 Subject: [PATCH 130/435] Add GitHub Workflow Replacements for Jenkins job_Python_LoadTests_Smoke and job_Java_LoadTests_GBK_Smoke (#28854) --- .../beam_LoadTests_Java_GBK_Smoke.yml | 116 ++++++++++++++++++ .../workflows/beam_LoadTests_Python_Smoke.yml | 103 ++++++++++++++++ .../java_Smoke_GroupByKey_Dataflow.txt | 23 ++++ .../java_Smoke_GroupByKey_Direct.txt | 20 +++ .../java_Smoke_GroupByKey_Flink.txt | 21 ++++ .../java_Smoke_GroupByKey_Spark.txt | 22 ++++ .../python_Smoke_GroupByKey_Dataflow.txt | 23 ++++ .../python_Smoke_GroupByKey_Direct.txt | 23 ++++ 8 files changed, 351 insertions(+) create mode 100644 .github/workflows/beam_LoadTests_Java_GBK_Smoke.yml create mode 100644 .github/workflows/beam_LoadTests_Python_Smoke.yml create mode 100644 .github/workflows/load-tests-job-configs/java_Smoke_GroupByKey_Dataflow.txt create mode 100644 .github/workflows/load-tests-job-configs/java_Smoke_GroupByKey_Direct.txt create mode 100644 .github/workflows/load-tests-job-configs/java_Smoke_GroupByKey_Flink.txt create mode 100644 .github/workflows/load-tests-job-configs/java_Smoke_GroupByKey_Spark.txt create mode 100644 .github/workflows/load-tests-job-configs/python_Smoke_GroupByKey_Dataflow.txt create mode 100644 .github/workflows/load-tests-job-configs/python_Smoke_GroupByKey_Direct.txt diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Smoke.yml b/.github/workflows/beam_LoadTests_Java_GBK_Smoke.yml new file mode 100644 index 000000000000..59cdad972d78 --- /dev/null +++ b/.github/workflows/beam_LoadTests_Java_GBK_Smoke.yml @@ -0,0 +1,116 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: LoadTests Java GBK Smoke + +on: + issue_comment: + types: [created] + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_LoadTests_Java_GBK_Smoke: + if: | + github.event_name == 'workflow_dispatch' || + github.event.comment.body == 'Run Java Load Tests GBK Smoke' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 720 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_LoadTests_Java_GBK_Smoke"] + job_phrase: ["Run Java Load Tests GBK Smoke"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: java + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_Smoke_GroupByKey_Direct.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_Smoke_GroupByKey_Dataflow.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_Smoke_GroupByKey_Flink.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_Smoke_GroupByKey_Spark.txt + - name: Set current datetime + id: datetime + run: | + echo "datetime=$(date '+%m%d%H%M%S' --utc)" >> $GITHUB_OUTPUT + # The env variables are created and populated in the test-arguments-action as "_test_arguments_" + - name: run GroupByKey load test Direct + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:direct-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Smoke_test_arguments_1 }}' \ + - name: run GroupByKey load test Dataflow + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Smoke_test_arguments_2 }}' \ + - name: run GroupByKey load test Flink + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + --info \ + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:flink:1.15 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Smoke_test_arguments_3 }}' \ + - name: run GroupByKey load test Spark + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:spark:3 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Smoke_test_arguments_4 }}' \ No newline at end of file diff --git a/.github/workflows/beam_LoadTests_Python_Smoke.yml b/.github/workflows/beam_LoadTests_Python_Smoke.yml new file mode 100644 index 000000000000..720561d09e06 --- /dev/null +++ b/.github/workflows/beam_LoadTests_Python_Smoke.yml @@ -0,0 +1,103 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: LoadTests Python Smoke + +on: + issue_comment: + types: [created] + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_LoadTests_Python_Smoke: + if: | + github.event_name == 'workflow_dispatch' || + github.event.comment.body == 'Run Python Load Tests Smoke' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 720 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_LoadTests_Python_Smoke"] + job_phrase: ["Run Python Load Tests Smoke"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action + with: + python-version: 3.8 + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: python + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_Smoke_GroupByKey_Direct.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_Smoke_GroupByKey_Dataflow.txt + - name: Set current datetime + id: datetime + run: | + echo "datetime=$(date '+%m%d%H%M%S' --utc)" >> $GITHUB_OUTPUT + # The env variables are created and populated in the test-arguments-action as "_test_arguments_" + - name: run GroupByKey Python load test Direct + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.group_by_key_test \ + -Prunner=DirectRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_Smoke_test_arguments_1 }} --job_name=load-tests-python-direct-batch-gbk-smoke-${{ steps.datetime.outputs.datetime }}' \ + - name: run GroupByKey Python load test Dataflow + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.group_by_key_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_Smoke_test_arguments_2 }} --job_name=load-tests-python-dataflow-batch-gbk-smoke-${{ steps.datetime.outputs.datetime }}' \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_Smoke_GroupByKey_Dataflow.txt b/.github/workflows/load-tests-job-configs/java_Smoke_GroupByKey_Dataflow.txt new file mode 100644 index 000000000000..411aeaeaf586 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_Smoke_GroupByKey_Dataflow.txt @@ -0,0 +1,23 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--sourceOptions={"numRecords":100000,"splitPointFrequencyRecords":1} +--stepOptions={"outputRecordsPerInputRecord":1,"preservesInputKeyDistribution":true} +--fanout=10 +--iterations=1 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_Smoke_GroupByKey_Direct.txt b/.github/workflows/load-tests-job-configs/java_Smoke_GroupByKey_Direct.txt new file mode 100644 index 000000000000..a062dbc4846a --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_Smoke_GroupByKey_Direct.txt @@ -0,0 +1,20 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--sourceOptions={"numRecords":100000,"splitPointFrequencyRecords":1} +--stepOptions={"outputRecordsPerInputRecord":1,"preservesInputKeyDistribution":true} +--fanout=10 +--iterations=1 \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_Smoke_GroupByKey_Flink.txt b/.github/workflows/load-tests-job-configs/java_Smoke_GroupByKey_Flink.txt new file mode 100644 index 000000000000..bfa07b5f2dcd --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_Smoke_GroupByKey_Flink.txt @@ -0,0 +1,21 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--sourceOptions={"numRecords":100000,"splitPointFrequencyRecords":1} +--stepOptions={"outputRecordsPerInputRecord":1,"preservesInputKeyDistribution":true} +--fanout=10 +--iterations=1 +--runner=FlinkRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_Smoke_GroupByKey_Spark.txt b/.github/workflows/load-tests-job-configs/java_Smoke_GroupByKey_Spark.txt new file mode 100644 index 000000000000..9eed902195c7 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_Smoke_GroupByKey_Spark.txt @@ -0,0 +1,22 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--sparkMaster=local[4] +--sourceOptions={"numRecords":100000,"splitPointFrequencyRecords":1} +--stepOptions={"outputRecordsPerInputRecord":1,"preservesInputKeyDistribution":true} +--fanout=10 +--iterations=1 +--runner=SparkRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_Smoke_GroupByKey_Dataflow.txt b/.github/workflows/load-tests-job-configs/python_Smoke_GroupByKey_Dataflow.txt new file mode 100644 index 000000000000..9a069df2bd1c --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_Smoke_GroupByKey_Dataflow.txt @@ -0,0 +1,23 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test_SMOKE +--metrics_table=python_dataflow_gbk +--input_options=''{\\"num_records\\":100000,\\"key_size\\":1,\\"value_size\\":1}'' +--max_num_workers=1 \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_Smoke_GroupByKey_Direct.txt b/.github/workflows/load-tests-job-configs/python_Smoke_GroupByKey_Direct.txt new file mode 100644 index 000000000000..7490675e4383 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_Smoke_GroupByKey_Direct.txt @@ -0,0 +1,23 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test_SMOKE +--metrics_table=python_direct_gbk +--input_options=''{\\"num_records\\":100000,\\"key_size\\":1,\\"value_size\\":1}'' +--max_num_workers=1 \ No newline at end of file From 71b8e93b6d385d1117e2659ee3f362ccec3422f6 Mon Sep 17 00:00:00 2001 From: Aleksandr Dudko <116064902+aleksandr-dudko@users.noreply.github.com> Date: Fri, 13 Oct 2023 17:42:35 +0400 Subject: [PATCH 131/435] Add GitHub Workflow Replacements for Jenkins job_Python_ParDo_DataFlow and job_Python_ParDo_Flink (#28894) --- ..._LoadTests_Python_ParDo_Dataflow_Batch.yml | 126 ++++++++++++++ ...dTests_Python_ParDo_Dataflow_Streaming.yml | 126 ++++++++++++++ ...eam_LoadTests_Python_ParDo_Flink_Batch.yml | 131 +++++++++++++++ ...LoadTests_Python_ParDo_Flink_Streaming.yml | 155 ++++++++++++++++++ ...thon_ParDo_Dataflow_Batch_100_Counters.txt | 29 ++++ ...ython_ParDo_Dataflow_Batch_10_Counters.txt | 29 ++++ ...hon_ParDo_Dataflow_Batch_10_Iterations.txt | 29 ++++ ...on_ParDo_Dataflow_Batch_200_Iterations.txt | 29 ++++ ..._ParDo_Dataflow_Streaming_100_Counters.txt | 31 ++++ ...n_ParDo_Dataflow_Streaming_10_Counters.txt | 31 ++++ ...ParDo_Dataflow_Streaming_10_Iterations.txt | 31 ++++ ...arDo_Dataflow_Streaming_200_Iterations.txt | 31 ++++ .../python_ParDo_Flink_Batch_10_Counters.txt | 29 ++++ ...python_ParDo_Flink_Batch_10_Iterations.txt | 29 ++++ ...ython_ParDo_Flink_Batch_200_Iterations.txt | 29 ++++ ...hon_ParDo_Flink_Streaming_100_Counters.txt | 31 ++++ ...thon_ParDo_Flink_Streaming_10_Counters.txt | 31 ++++ ...on_ParDo_Flink_Streaming_10_Iterations.txt | 32 ++++ ...n_ParDo_Flink_Streaming_200_Iterations.txt | 31 ++++ ...hon_ParDo_Flink_Streaming_5_Iterations.txt | 35 ++++ 20 files changed, 1025 insertions(+) create mode 100644 .github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Batch.yml create mode 100644 .github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml create mode 100644 .github/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml create mode 100644 .github/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml create mode 100644 .github/workflows/load-tests-job-configs/python_ParDo_Dataflow_Batch_100_Counters.txt create mode 100644 .github/workflows/load-tests-job-configs/python_ParDo_Dataflow_Batch_10_Counters.txt create mode 100644 .github/workflows/load-tests-job-configs/python_ParDo_Dataflow_Batch_10_Iterations.txt create mode 100644 .github/workflows/load-tests-job-configs/python_ParDo_Dataflow_Batch_200_Iterations.txt create mode 100644 .github/workflows/load-tests-job-configs/python_ParDo_Dataflow_Streaming_100_Counters.txt create mode 100644 .github/workflows/load-tests-job-configs/python_ParDo_Dataflow_Streaming_10_Counters.txt create mode 100644 .github/workflows/load-tests-job-configs/python_ParDo_Dataflow_Streaming_10_Iterations.txt create mode 100644 .github/workflows/load-tests-job-configs/python_ParDo_Dataflow_Streaming_200_Iterations.txt create mode 100644 .github/workflows/load-tests-job-configs/python_ParDo_Flink_Batch_10_Counters.txt create mode 100644 .github/workflows/load-tests-job-configs/python_ParDo_Flink_Batch_10_Iterations.txt create mode 100644 .github/workflows/load-tests-job-configs/python_ParDo_Flink_Batch_200_Iterations.txt create mode 100644 .github/workflows/load-tests-job-configs/python_ParDo_Flink_Streaming_100_Counters.txt create mode 100644 .github/workflows/load-tests-job-configs/python_ParDo_Flink_Streaming_10_Counters.txt create mode 100644 .github/workflows/load-tests-job-configs/python_ParDo_Flink_Streaming_10_Iterations.txt create mode 100644 .github/workflows/load-tests-job-configs/python_ParDo_Flink_Streaming_200_Iterations.txt create mode 100644 .github/workflows/load-tests-job-configs/python_ParDo_Flink_Streaming_5_Iterations.txt diff --git a/.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Batch.yml new file mode 100644 index 000000000000..9f605282d131 --- /dev/null +++ b/.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Batch.yml @@ -0,0 +1,126 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: LoadTests Python ParDo Dataflow Batch + +on: + issue_comment: + types: [created] + schedule: + - cron: '30 13 * * *' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_LoadTests_Python_ParDo_Dataflow_Batch: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Load Tests Python ParDo Dataflow Batch' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 200 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_LoadTests_Python_ParDo_Dataflow_Batch"] + job_phrase: ["Run Load Tests Python ParDo Dataflow Batch"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action + with: + python-version: 3.8 + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: python + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_ParDo_Dataflow_Batch_10_Iterations.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_ParDo_Dataflow_Batch_200_Iterations.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_ParDo_Dataflow_Batch_10_Counters.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_ParDo_Dataflow_Batch_100_Counters.txt + - name: Set current datetime + id: datetime + run: | + echo "datetime=$(date '+%m%d%H%M%S' --utc)" >> $GITHUB_OUTPUT + # The env variables are created and populated in the test-arguments-action as "_test_arguments_" + - name: run ParDo Dataflow Batch Python Load Test 1 (10 iterations) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.pardo_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_ParDo_Dataflow_Batch_test_arguments_1 }} --job_name=load-tests-python-dataflow-batch-pardo-1-${{ steps.datetime.outputs.datetime }}' \ + - name: run ParDo Dataflow Batch Python Load Test 2 (200 iterations) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.pardo_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_ParDo_Dataflow_Batch_test_arguments_2 }} --job_name=load-tests-python-dataflow-batch-pardo-2-${{ steps.datetime.outputs.datetime }}' \ + - name: run ParDo Dataflow Batch Python Load Test 3 (10 counters) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.pardo_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_ParDo_Dataflow_Batch_test_arguments_3 }} --job_name=load-tests-python-dataflow-batch-pardo-3-${{ steps.datetime.outputs.datetime }}' \ + - name: run ParDo Dataflow Batch Python Load Test 4 (100 counters) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.pardo_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_ParDo_Dataflow_Batch_test_arguments_4 }} --job_name=load-tests-python-dataflow-batch-pardo-4-${{ steps.datetime.outputs.datetime }}' \ No newline at end of file diff --git a/.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml new file mode 100644 index 000000000000..b681a2d95419 --- /dev/null +++ b/.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml @@ -0,0 +1,126 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: LoadTests Python ParDo Dataflow Streaming + +on: + issue_comment: + types: [created] + schedule: + - cron: '30 16 * * *' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_LoadTests_Python_ParDo_Dataflow_Streaming: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Python Load Tests ParDo Dataflow Streaming' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 200 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_LoadTests_Python_ParDo_Dataflow_Streaming"] + job_phrase: ["Run Python Load Tests ParDo Dataflow Streaming"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action + with: + python-version: 3.8 + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: python + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python-pardo/python_ParDo_Dataflow_Streaming_10_Iterations.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python-pardo/python_ParDo_Dataflow_Streaming_200_Iterations.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python-pardo/python_ParDo_Dataflow_Streaming_10_Counters.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python-pardo/python_ParDo_Dataflow_Streaming_100_Counters.txt + - name: Set current datetime + id: datetime + run: | + echo "datetime=$(date '+%m%d%H%M%S' --utc)" >> $GITHUB_OUTPUT + # The env variables are created and populated in the test-arguments-action as "_test_arguments_" + - name: run ParDo Dataflow Streaming Python Load Test 1 (10 iterations) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.pardo_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_ParDo_Dataflow_Streaming_test_arguments_1 }} --job_name=load-tests-python-dataflow-streaming-pardo-1-${{ steps.datetime.outputs.datetime }}' \ + - name: run ParDo Dataflow Streaming Python Load Test 2 (200 iterations) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.pardo_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_ParDo_Dataflow_Streaming_test_arguments_2 }} --job_name=load-tests-python-dataflow-streaming-pardo-2-${{ steps.datetime.outputs.datetime }}' \ + - name: run ParDo Dataflow Streaming Python Load Test 3 (10 counters) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.pardo_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_ParDo_Dataflow_Streaming_test_arguments_3 }} --job_name=load-tests-python-dataflow-streaming-pardo-3-${{ steps.datetime.outputs.datetime }}' \ + - name: run ParDo Dataflow Streaming Python Load Test 4 (100 counters) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.pardo_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_ParDo_Dataflow_Streaming_test_arguments_4 }} --job_name=load-tests-python-dataflow-streaming-pardo-4-${{ steps.datetime.outputs.datetime }}' \ No newline at end of file diff --git a/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml new file mode 100644 index 000000000000..66bbc72b14cd --- /dev/null +++ b/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml @@ -0,0 +1,131 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: LoadTests Python ParDo Flink Batch + +on: + issue_comment: + types: [created] + schedule: + - cron: '10 14 * * *' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + GCLOUD_ZONE: us-central1-a + CLUSTER_NAME: beam-loadtests-python-pardo-flink-batch-${{ github.run_id }} + GCS_BUCKET: gs://beam-flink-cluster + FLINK_DOWNLOAD_URL: https://archive.apache.org/dist/flink/flink-1.15.0/flink-1.15.0-bin-scala_2.12.tgz + HADOOP_DOWNLOAD_URL: https://repo.maven.apache.org/maven2/org/apache/flink/flink-shaded-hadoop-2-uber/2.8.3-10.0/flink-shaded-hadoop-2-uber-2.8.3-10.0.jar + FLINK_TASKMANAGER_SLOTS: 1 + DETACHED_MODE: true + HARNESS_IMAGES_TO_PULL: gcr.io/apache-beam-testing/beam-sdk/beam_go_sdk:latest + JOB_SERVER_IMAGE: gcr.io/apache-beam-testing/beam_portability/beam_flink1.15_job_server:latest + ARTIFACTS_DIR: gs://beam-flink-cluster/beam-loadtests-python-pardo-flink-batch-${{ github.run_id }} + +jobs: + beam_LoadTests_Python_ParDo_Flink_Batch: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Load Tests Python ParDo Flink Batch' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 720 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_LoadTests_Python_ParDo_Flink_Batch"] + job_phrase: ["Run Load Tests Python ParDo Flink Batch"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action + with: + python-version: 3.8 + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: python + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python-pardo/python_ParDo_Flink_Batch_10_Iterations.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python-pardo/python_ParDo_Flink_Batch_200_Iterations.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python-pardo/python_ParDo_Flink_Batch_10_Counters.txt + - name: Start Flink with parallelism 5 + env: + FLINK_NUM_WORKERS: 5 + run: | + cd ${{ github.workspace }}/.test-infra/dataproc; ./flink_cluster.sh create + - name: Set current datetime + id: datetime + run: | + echo "datetime=$(date '+%m%d%H%M%S' --utc)" >> $GITHUB_OUTPUT + # The env variables are created and populated in the test-arguments-action as "_test_arguments_" + - name: run ParDo Flink Batch Python Load Test 1 (10 iterations) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.pardo_test \ + -Prunner=PortableRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_ParDo_Flink_Batch_test_arguments_1 }} --job_name=load-tests-python-flink-batch-pardo-1-${{ steps.datetime.outputs.datetime }}' \ + - name: run ParDo Flink Batch Python Load Test 2 (200 iterations) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.pardo_test \ + -Prunner=PortableRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_ParDo_Flink_Batch_test_arguments_2 }} --job_name=load-tests-python-flink-batch-pardo-3-${{ steps.datetime.outputs.datetime }}' \ + - name: run ParDo Flink Batch Python Load Test 3 (10 counters) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.pardo_test \ + -Prunner=PortableRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_ParDo_Flink_Batch_test_arguments_3 }} --job_name=load-tests-python-flink-batch-pardo-4-${{ steps.datetime.outputs.datetime }}' \ No newline at end of file diff --git a/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml b/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml new file mode 100644 index 000000000000..82ab10d3eea5 --- /dev/null +++ b/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml @@ -0,0 +1,155 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: LoadTests Python ParDo Flink Streaming + +on: + issue_comment: + types: [created] + schedule: + - cron: '15 15 * * *' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + GCLOUD_ZONE: us-central1-a + CLUSTER_NAME: beam-loadtests-python-pardo-flink-stream-${{ github.run_id }} + GCS_BUCKET: gs://beam-flink-cluster + FLINK_DOWNLOAD_URL: https://archive.apache.org/dist/flink/flink-1.15.0/flink-1.15.0-bin-scala_2.12.tgz + HADOOP_DOWNLOAD_URL: https://repo.maven.apache.org/maven2/org/apache/flink/flink-shaded-hadoop-2-uber/2.8.3-10.0/flink-shaded-hadoop-2-uber-2.8.3-10.0.jar + FLINK_TASKMANAGER_SLOTS: 1 + DETACHED_MODE: true + HARNESS_IMAGES_TO_PULL: gcr.io/apache-beam-testing/beam-sdk/beam_go_sdk:latest + JOB_SERVER_IMAGE: gcr.io/apache-beam-testing/beam_portability/beam_flink1.15_job_server:latest + ARTIFACTS_DIR: gs://beam-flink-cluster/beam-loadtests-python-pardo-flink-stream-${{ github.run_id }} + +jobs: + beam_LoadTests_Python_ParDo_Flink_Streaming: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Load Tests Python ParDo Flink Streaming' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 720 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_LoadTests_Python_ParDo_Flink_Streaming"] + job_phrase: ["Run Load Tests Python ParDo Flink Streaming"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action + with: + python-version: 3.8 + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: python + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python-pardo/python_ParDo_Flink_Streaming_10_Iterations.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python-pardo/python_ParDo_Flink_Streaming_200_Iterations.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python-pardo/python_ParDo_Flink_Streaming_10_Counters.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python-pardo/python_ParDo_Flink_Streaming_100_Counters.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python-pardo/python_ParDo_Flink_Streaming_5_Iterations.txt + - name: Start Flink with parallelism 5 + env: + FLINK_NUM_WORKERS: 5 + run: | + cd ${{ github.workspace }}/.test-infra/dataproc; ./flink_cluster.sh create + - name: Set current datetime + id: datetime + run: | + echo "datetime=$(date '+%m%d%H%M%S' --utc)" >> $GITHUB_OUTPUT + # The env variables are created and populated in the test-arguments-action as "_test_arguments_" + - name: run ParDo Flink Streaming Python Load Test 1 (10 iterations) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.pardo_test \ + -Prunner=PortableRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_ParDo_Flink_Streaming_test_arguments_1 }} --job_name=load-tests-python-flink-streaming-pardo-1-${{ steps.datetime.outputs.datetime }}' \ + - name: run ParDo Flink Streaming Python Load Test 2 (200 iterations) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.pardo_test \ + -Prunner=PortableRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_ParDo_Flink_Streaming_test_arguments_2 }} --job_name=load-tests-python-flink-streaming-pardo-2-${{ steps.datetime.outputs.datetime }}' \ + - name: run ParDo Flink Streaming Python Load Test 3 (10 counters) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.pardo_test \ + -Prunner=PortableRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_ParDo_Flink_Streaming_test_arguments_3 }} --job_name=load-tests-python-flink-streaming-pardo-3-${{ steps.datetime.outputs.datetime }}' \ + - name: run ParDo Flink Streaming Python Load Test 4 (100 counters) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.pardo_test \ + -Prunner=PortableRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_ParDo_Flink_Streaming_test_arguments_4 }} --job_name=load-tests-python-flink-streaming-pardo-4-${{ steps.datetime.outputs.datetime }}' \ + - name: run ParDo Flink Streaming Python Load Test 5 (5 iterations) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.pardo_test \ + -Prunner=PortableRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_ParDo_Flink_Streaming_test_arguments_5 }} --job_name=load-tests-python-flink-streaming-pardo-6-${{ steps.datetime.outputs.datetime }}' \ + - name: Teardown Flink + if: always() + run: | + ${{ github.workspace }}/.test-infra/dataproc/flink_cluster.sh delete \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_ParDo_Dataflow_Batch_100_Counters.txt b/.github/workflows/load-tests-job-configs/python_ParDo_Dataflow_Batch_100_Counters.txt new file mode 100644 index 000000000000..a5bb7979b86b --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_ParDo_Dataflow_Batch_100_Counters.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_batch_pardo_4 +--influx_measurement=python_batch_pardo_4 +--input_options=''{\\"num_records\\":20000000,\\"key_size\\":10,\\"value_size\\":90,\\"algorithm\\":\\"lcg\\"}'' +--iterations=1 +--number_of_counter_operations=100 +--number_of_counters=1 +--num_workers=5 +--autoscaling_algorithm=NONE +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_ParDo_Dataflow_Batch_10_Counters.txt b/.github/workflows/load-tests-job-configs/python_ParDo_Dataflow_Batch_10_Counters.txt new file mode 100644 index 000000000000..7e35ef74dfa1 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_ParDo_Dataflow_Batch_10_Counters.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_batch_pardo_3 +--influx_measurement=python_batch_pardo_3 +--input_options=''{\\"num_records\\":20000000,\\"key_size\\":10,\\"value_size\\":90,\\"algorithm\\":\\"lcg\\"}'' +--iterations=1 +--number_of_counter_operations=10 +--number_of_counters=1 +--num_workers=5 +--autoscaling_algorithm=NONE +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_ParDo_Dataflow_Batch_10_Iterations.txt b/.github/workflows/load-tests-job-configs/python_ParDo_Dataflow_Batch_10_Iterations.txt new file mode 100644 index 000000000000..734360397c9b --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_ParDo_Dataflow_Batch_10_Iterations.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_batch_pardo_1 +--influx_measurement=python_batch_pardo_1 +--input_options=''{\\"num_records\\":20000000,\\"key_size\\":10,\\"value_size\\":90,\\"algorithm\\":\\"lcg\\"}'' +--iterations=10 +--number_of_counter_operations=0 +--number_of_counters=0 +--num_workers=5 +--autoscaling_algorithm=NONE +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_ParDo_Dataflow_Batch_200_Iterations.txt b/.github/workflows/load-tests-job-configs/python_ParDo_Dataflow_Batch_200_Iterations.txt new file mode 100644 index 000000000000..825fee427a31 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_ParDo_Dataflow_Batch_200_Iterations.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_batch_pardo_2 +--influx_measurement=python_batch_pardo_2 +--input_options=''{\\"num_records\\":20000000,\\"key_size\\":10,\\"value_size\\":90,\\"algorithm\\":\\"lcg\\"}'' +--iterations=200 +--number_of_counter_operations=0 +--number_of_counters=0 +--num_workers=5 +--autoscaling_algorithm=NONE +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_ParDo_Dataflow_Streaming_100_Counters.txt b/.github/workflows/load-tests-job-configs/python_ParDo_Dataflow_Streaming_100_Counters.txt new file mode 100644 index 000000000000..71fc818d0e07 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_ParDo_Dataflow_Streaming_100_Counters.txt @@ -0,0 +1,31 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_streaming_pardo_4 +--influx_measurement=python_streaming_pardo_4 +--input_options=''{\\"num_records\\":20000000,\\"key_size\\":10,\\"value_size\\":90,\\"algorithm\\":\\"lcg\\"}'' +--iterations=1 +--number_of_counter_operations=100 +--number_of_counters=1 +--num_****s=5 +--autoscaling_algorithm=NONE +--streaming +--experiments=use_runner_v2 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_ParDo_Dataflow_Streaming_10_Counters.txt b/.github/workflows/load-tests-job-configs/python_ParDo_Dataflow_Streaming_10_Counters.txt new file mode 100644 index 000000000000..fe48dedfa837 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_ParDo_Dataflow_Streaming_10_Counters.txt @@ -0,0 +1,31 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_streaming_pardo_3 +--influx_measurement=python_streaming_pardo_3 +--input_options=''{\\"num_records\\":20000000,\\"key_size\\":10,\\"value_size\\":90,\\"algorithm\\":\\"lcg\\"}'' +--iterations=1 +--number_of_counter_operations=10 +--number_of_counters=1 +--num_workers=5 +--autoscaling_algorithm=NONE +--streaming +--experiments=use_runner_v2 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_ParDo_Dataflow_Streaming_10_Iterations.txt b/.github/workflows/load-tests-job-configs/python_ParDo_Dataflow_Streaming_10_Iterations.txt new file mode 100644 index 000000000000..84d2cdb7a85b --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_ParDo_Dataflow_Streaming_10_Iterations.txt @@ -0,0 +1,31 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_streaming_pardo_1 +--influx_measurement=python_streaming_pardo_1 +--input_options=''{\\"num_records\\":20000000,\\"key_size\\":10,\\"value_size\\":90,\\"algorithm\\":\\"lcg\\"}'' +--iterations=10 +--number_of_counter_operations=0 +--number_of_counters=0 +--num_workers=5 +--autoscaling_algorithm=NONE +--streaming +--experiments=use_runner_v2 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_ParDo_Dataflow_Streaming_200_Iterations.txt b/.github/workflows/load-tests-job-configs/python_ParDo_Dataflow_Streaming_200_Iterations.txt new file mode 100644 index 000000000000..02636335cb37 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_ParDo_Dataflow_Streaming_200_Iterations.txt @@ -0,0 +1,31 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_streaming_pardo_2 +--influx_measurement=python_streaming_pardo_2 +--input_options=''{\\"num_records\\":20000000,\\"key_size\\":10,\\"value_size\\":90,\\"algorithm\\":\\"lcg\\"}'' +--iterations=200 +--number_of_counter_operations=0 +--number_of_counters=0 +--num_workers=5 +--autoscaling_algorithm=NONE +--streaming +--experiments=use_runner_v2 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_ParDo_Flink_Batch_10_Counters.txt b/.github/workflows/load-tests-job-configs/python_ParDo_Flink_Batch_10_Counters.txt new file mode 100644 index 000000000000..4d8bda8ac2f8 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_ParDo_Flink_Batch_10_Counters.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_flink_batch_pardo_4 +--influx_measurement=python_batch_pardo_4 +--input_options=''{\\"num_records\\":20000000,\\"key_size\\":10,\\"value_size\\":90,\\"algorithm\\":\\"lcg\\"}'' +--iterations=1 +--number_of_counter_operations=100 +--number_of_counters=1 +--parallelism=5 +--job_endpoint=localhost:8099 +--environment_type=DOCKER +--environment_config=gcr.io/apache-beam-testing/beam-sdk/beam_python3.8_sdk:latest +--runner=PortableRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_ParDo_Flink_Batch_10_Iterations.txt b/.github/workflows/load-tests-job-configs/python_ParDo_Flink_Batch_10_Iterations.txt new file mode 100644 index 000000000000..e84cee2f50cf --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_ParDo_Flink_Batch_10_Iterations.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_flink_batch_pardo_1 +--influx_measurement=python_batch_pardo_1 +--input_options=''{\\"num_records\\":20000000,\\"key_size\\":10,\\"value_size\\":90,\\"algorithm\\":\\"lcg\\"}'' +--iterations=10 +--number_of_counter_operations=0 +--number_of_counters=0 +--parallelism=5 +--job_endpoint=localhost:8099 +--environment_type=DOCKER +--environment_config=gcr.io/apache-beam-testing/beam-sdk/beam_python3.8_sdk:latest +--runner=PortableRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_ParDo_Flink_Batch_200_Iterations.txt b/.github/workflows/load-tests-job-configs/python_ParDo_Flink_Batch_200_Iterations.txt new file mode 100644 index 000000000000..4d8bda8ac2f8 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_ParDo_Flink_Batch_200_Iterations.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_flink_batch_pardo_4 +--influx_measurement=python_batch_pardo_4 +--input_options=''{\\"num_records\\":20000000,\\"key_size\\":10,\\"value_size\\":90,\\"algorithm\\":\\"lcg\\"}'' +--iterations=1 +--number_of_counter_operations=100 +--number_of_counters=1 +--parallelism=5 +--job_endpoint=localhost:8099 +--environment_type=DOCKER +--environment_config=gcr.io/apache-beam-testing/beam-sdk/beam_python3.8_sdk:latest +--runner=PortableRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_ParDo_Flink_Streaming_100_Counters.txt b/.github/workflows/load-tests-job-configs/python_ParDo_Flink_Streaming_100_Counters.txt new file mode 100644 index 000000000000..b17e2cecc2c8 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_ParDo_Flink_Streaming_100_Counters.txt @@ -0,0 +1,31 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_flink_streaming_pardo_4 +--influx_measurement=python_streaming_pardo_4 +--input_options=''{\\"num_records\\":20000000,\\"key_size\\":10,\\"value_size\\":90,\\"algorithm\\":\\"lcg\\"}'' +--iterations=1 +--number_of_counter_operations=100 +--number_of_counters=1 +--parallelism=5 +--streaming +--job_endpoint=localhost:8099 +--environment_type=DOCKER +--environment_config=gcr.io/apache-beam-testing/beam-sdk/beam_python3.8_sdk:latest +--use_stateful_load_generator +--runner=PortableRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_ParDo_Flink_Streaming_10_Counters.txt b/.github/workflows/load-tests-job-configs/python_ParDo_Flink_Streaming_10_Counters.txt new file mode 100644 index 000000000000..957bc6c086d8 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_ParDo_Flink_Streaming_10_Counters.txt @@ -0,0 +1,31 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_flink_streaming_pardo_3 +--influx_measurement=python_streaming_pardo_3 +--input_options=''{\\"num_records\\":20000000,\\"key_size\\":10,\\"value_size\\":90,\\"algorithm\\":\\"lcg\\"}'' +--iterations=1 +--number_of_counter_operations=10 +--number_of_counters=1 +--parallelism=5 +--streaming +--job_endpoint=localhost:8099 +--environment_type=DOCKER +--environment_config=gcr.io/apache-beam-testing/beam-sdk/beam_python3.8_sdk:latest +--use_stateful_load_generator +--runner=PortableRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_ParDo_Flink_Streaming_10_Iterations.txt b/.github/workflows/load-tests-job-configs/python_ParDo_Flink_Streaming_10_Iterations.txt new file mode 100644 index 000000000000..baa34ec455b5 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_ParDo_Flink_Streaming_10_Iterations.txt @@ -0,0 +1,32 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_flink_streaming_pardo_5 +--influx_measurement=python_streaming_pardo_1 +--input_options=''{\\"num_records\\":2000000,\\"key_size\\":10,\\"value_size\\":90,\\"algorithm\\":\\"lcg\\"}'' +--iterations=10 +--number_of_counter_operations=0 +--number_of_counters=0 +--parallelism=5 +--streaming +--stateful +--job_endpoint=localhost:8099 +--environment_type=DOCKER +--environment_config=gcr.io/apache-beam-testing/beam-sdk/beam_python3.8_sdk:latest +--use_stateful_load_generator +--runner=PortableRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_ParDo_Flink_Streaming_200_Iterations.txt b/.github/workflows/load-tests-job-configs/python_ParDo_Flink_Streaming_200_Iterations.txt new file mode 100644 index 000000000000..44483a6e51cc --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_ParDo_Flink_Streaming_200_Iterations.txt @@ -0,0 +1,31 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_flink_streaming_pardo_2 +--influx_measurement=python_streaming_pardo_2 +--input_options=''{\\"num_records\\":20000000,\\"key_size\\":10,\\"value_size\\":90,\\"algorithm\\":\\"lcg\\"}'' +--iterations=200 +--number_of_counter_operations=0 +--number_of_counters=0 +--parallelism=5 +--streaming +--job_endpoint=localhost:8099 +--environment_type=DOCKER +--environment_config=gcr.io/apache-beam-testing/beam-sdk/beam_python3.8_sdk:latest +--use_stateful_load_generator +--runner=PortableRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_ParDo_Flink_Streaming_5_Iterations.txt b/.github/workflows/load-tests-job-configs/python_ParDo_Flink_Streaming_5_Iterations.txt new file mode 100644 index 000000000000..571b33fb7a49 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_ParDo_Flink_Streaming_5_Iterations.txt @@ -0,0 +1,35 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_flink_streaming_pardo_6 +--influx_measurement=python_streaming_pardo_6 +--input_options=''{\\"num_records\\":2000000,\\"key_size\\":10,\\"value_size\\":90,\\"algorithm\\":\\"lcg\\"}'' +--iterations=5 +--number_of_counter_operations=10 +--number_of_counters=3 +--parallelism=5 +--streaming +--stateful +--checkpointing_interval=10000 +--report_checkpoint_duration=python_flink_streaming_pardo_6 +--shutdown_sources_after_idle_ms=300000 +--job_endpoint=localhost:8099 +--environment_type=DOCKER +--environment_config=gcr.io/apache-beam-testing/beam-sdk/beam_python3.8_sdk:latest +--use_stateful_load_generator +--runner=PortableRunner \ No newline at end of file From c0af78a9e980e0c0d094b8ceb4d7e590fee16b5f Mon Sep 17 00:00:00 2001 From: Danny McCormick Date: Fri, 13 Oct 2023 10:24:52 -0400 Subject: [PATCH 132/435] Revert "Setup Java 21 container (#28833)" (#28968) This reverts commit e8e38145d96a7b0561a418c19519917b0567591e. --- ...a_CoGBK_Dataflow_V2_Batch_JavaVersions.yml | 8 ++-- ...GBK_Dataflow_V2_Streaming_JavaVersions.yml | 8 ++-- ...ests_Java_GBK_Dataflow_V2_Batch_Java11.yml | 14 +++---- ...ests_Java_GBK_Dataflow_V2_Batch_Java17.yml | 14 +++---- ..._Java_GBK_Dataflow_V2_Streaming_Java11.yml | 14 +++---- ..._Java_GBK_Dataflow_V2_Streaming_Java17.yml | 14 +++---- ..._PostCommit_Java_Examples_Dataflow_ARM.yml | 2 +- ...tCommit_Java_Examples_Dataflow_V2_Java.yml | 2 +- ...m_PostCommit_Java_Jpms_Dataflow_Java17.yml | 2 +- ...eam_PostCommit_Java_Jpms_Direct_Java17.yml | 2 +- ...stCommit_Java_Nexmark_Dataflow_V2_Java.yml | 2 +- ...eam_PostCommit_TransformService_Direct.yml | 2 +- ...reCommit_Java_Examples_Dataflow_Java11.yml | 2 +- ...reCommit_Java_Examples_Dataflow_Java17.yml | 2 +- .../workflows/beam_PreCommit_SQL_Java11.yml | 2 +- .../workflows/beam_PreCommit_SQL_Java17.yml | 2 +- .test-infra/jenkins/JavaTestProperties.groovy | 7 +--- .test-infra/jenkins/NexmarkBuilder.groovy | 4 +- ..._LoadTests_CoGBK_Dataflow_V2_Java11.groovy | 2 +- ..._LoadTests_CoGBK_Dataflow_V2_Java17.groovy | 2 +- ...ob_LoadTests_GBK_Dataflow_V2_Java11.groovy | 2 +- ...ob_LoadTests_GBK_Dataflow_V2_Java17.groovy | 2 +- ..._LoadTests_ParDo_Dataflow_V2_Java11.groovy | 2 +- ..._LoadTests_ParDo_Dataflow_V2_Java17.groovy | 2 +- ...it_Java_Examples_Dataflow_V2_Java11.groovy | 2 +- ...it_Java_Examples_Dataflow_V2_Java17.groovy | 2 +- ...ostCommit_Java_Jpms_Dataflow_Java17.groovy | 2 +- ..._PostCommit_Java_Jpms_Direct_Java17.groovy | 2 +- ..._PostCommit_TransformService_Direct.groovy | 2 +- ...ommit_Java_Examples_Dataflow_Java11.groovy | 2 +- .../jenkins/job_PreCommit_SQL_Java11.groovy | 2 +- .../jenkins/job_PreCommit_SQL_Java17.groovy | 2 +- ...ommit_Java_Examples_Dataflow_Java17.groovy | 2 +- build.gradle.kts | 14 ++++--- .../beam/gradle/BeamModulePlugin.groovy | 37 +------------------ .../core/construction/Environments.java | 6 +-- .../core/construction/EnvironmentsTest.java | 6 +-- .../arm/build.gradle | 6 ++- .../google-cloud-dataflow-java/build.gradle | 6 ++- .../beam/runners/dataflow/DataflowRunner.java | 7 ++-- runners/spark/spark_runner.gradle | 5 +-- sdks/java/container/Dockerfile | 3 +- sdks/java/container/common.gradle | 14 +++---- sdks/java/container/java21/build.gradle | 30 --------------- sdks/java/container/java21/option-jamm.json | 12 ------ sdks/java/testing/jpms-tests/build.gradle | 21 ++++++----- sdks/java/testing/test-utils/build.gradle | 29 ++++++++++----- .../jvmverification/JvmVerification.java | 16 +------- settings.gradle.kts | 1 - .../www/site/content/en/roadmap/java-sdk.md | 6 +-- 50 files changed, 132 insertions(+), 220 deletions(-) delete mode 100644 sdks/java/container/java21/build.gradle delete mode 100644 sdks/java/container/java21/option-jamm.json diff --git a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml index ff172cc338b7..35575aec0bef 100644 --- a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml +++ b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml @@ -95,7 +95,7 @@ jobs: with: gradle-command: :sdks:java:testing:load-tests:run arguments: | - -PtestJavaVersion=${{ matrix.java_version }} \ + -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ @@ -105,7 +105,7 @@ jobs: with: gradle-command: :sdks:java:testing:load-tests:run arguments: | - -PtestJavaVersion=${{ matrix.java_version }} \ + -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ @@ -115,7 +115,7 @@ jobs: with: gradle-command: :sdks:java:testing:load-tests:run arguments: | - -PtestJavaVersion=${{ matrix.java_version }} \ + -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ @@ -125,7 +125,7 @@ jobs: with: gradle-command: :sdks:java:testing:load-tests:run arguments: | - -PtestJavaVersion=${{ matrix.java_version }} \ + -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ diff --git a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml index efa61373f2ab..46dfaa986920 100644 --- a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml +++ b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml @@ -95,7 +95,7 @@ jobs: with: gradle-command: :sdks:java:testing:load-tests:run arguments: | - -PtestJavaVersion=${{ matrix.java_version }} \ + -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ @@ -105,7 +105,7 @@ jobs: with: gradle-command: :sdks:java:testing:load-tests:run arguments: | - -PtestJavaVersion=${{ matrix.java_version }} \ + -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ @@ -115,7 +115,7 @@ jobs: with: gradle-command: :sdks:java:testing:load-tests:run arguments: | - -PtestJavaVersion=${{ matrix.java_version }} \ + -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ @@ -125,7 +125,7 @@ jobs: with: gradle-command: :sdks:java:testing:load-tests:run arguments: | - -PtestJavaVersion=${{ matrix.java_version }} \ + -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml index 5c333013e2a2..318f157e57a6 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml @@ -93,7 +93,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PtestJavaVersion=11 \ + -PcompileAndRunTestsWithJava11 \ -Pjava11Home=$JAVA_HOME_11_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11_test_arguments_1 }}' \ - name: run Load test 2GB of 100B records @@ -104,7 +104,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PtestJavaVersion=11 \ + -PcompileAndRunTestsWithJava11 \ -Pjava11Home=$JAVA_HOME_11_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11_test_arguments_2 }}' \ - name: run Load test 2GB of 100kB records @@ -115,7 +115,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PtestJavaVersion=11 \ + -PcompileAndRunTestsWithJava11 \ -Pjava11Home=$JAVA_HOME_11_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11_test_arguments_3 }}' \ - name: run Load test fanout 4 times with 2GB 10-byte records total @@ -126,7 +126,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PtestJavaVersion=11 \ + -PcompileAndRunTestsWithJava11 \ -Pjava11Home=$JAVA_HOME_11_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11_test_arguments_4 }}' \ - name: run Load test fanout 8 times with 2GB 10-byte records total @@ -137,7 +137,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PtestJavaVersion=11 \ + -PcompileAndRunTestsWithJava11 \ -Pjava11Home=$JAVA_HOME_11_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11_test_arguments_5 }}' \ - name: run Load test reiterate 4 times 10kB values @@ -148,7 +148,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PtestJavaVersion=11 \ + -PcompileAndRunTestsWithJava11 \ -Pjava11Home=$JAVA_HOME_11_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11_test_arguments_6 }}' \ - name: run Load test reiterate 4 times 2MB values @@ -159,6 +159,6 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PtestJavaVersion=11 \ + -PcompileAndRunTestsWithJava11 \ -Pjava11Home=$JAVA_HOME_11_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11_test_arguments_7 }}' \ No newline at end of file diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml index 4fa5c364728b..543cb86985d8 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml @@ -95,7 +95,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PtestJavaVersion=17 \ + -PcompileAndRunTestsWithJava17 \ -Pjava17Home=$JAVA_HOME_17_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17_test_arguments_1 }}' \ - name: run Load test 2GB of 100B records @@ -106,7 +106,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PtestJavaVersion=17 \ + -PcompileAndRunTestsWithJava17 \ -Pjava17Home=$JAVA_HOME_17_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17_test_arguments_2 }}' \ - name: run Load test 2GB of 100kB records @@ -117,7 +117,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PtestJavaVersion=17 \ + -PcompileAndRunTestsWithJava17 \ -Pjava17Home=$JAVA_HOME_17_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17_test_arguments_3 }}' \ - name: run Load test fanout 4 times with 2GB 10-byte records total @@ -128,7 +128,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PtestJavaVersion=17 \ + -PcompileAndRunTestsWithJava17 \ -Pjava17Home=$JAVA_HOME_17_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17_test_arguments_4 }}' \ - name: run Load test fanout 8 times with 2GB 10-byte records total @@ -139,7 +139,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PtestJavaVersion=17 \ + -PcompileAndRunTestsWithJava17 \ -Pjava17Home=$JAVA_HOME_17_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17_test_arguments_5 }}' \ - name: run Load test reiterate 4 times 10kB values @@ -150,7 +150,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PtestJavaVersion=17 \ + -PcompileAndRunTestsWithJava17 \ -Pjava17Home=$JAVA_HOME_17_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17_test_arguments_6 }}' \ - name: run Load test reiterate 4 times 2MB values @@ -161,6 +161,6 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PtestJavaVersion=17 \ + -PcompileAndRunTestsWithJava17 \ -Pjava17Home=$JAVA_HOME_17_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17_test_arguments_7 }}' \ No newline at end of file diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml index 6432a315d8a1..fcd55f761c04 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml @@ -93,7 +93,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PtestJavaVersion=11 \ + -PcompileAndRunTestsWithJava11 \ -Pjava11Home=$JAVA_HOME_11_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11_test_arguments_1 }}' \ - name: run Load test 2GB of 100B records @@ -104,7 +104,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PtestJavaVersion=11 \ + -PcompileAndRunTestsWithJava11 \ -Pjava11Home=$JAVA_HOME_11_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11_test_arguments_2 }}' \ - name: run Load test 2GB of 100kB records @@ -115,7 +115,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PtestJavaVersion=11 \ + -PcompileAndRunTestsWithJava11 \ -Pjava11Home=$JAVA_HOME_11_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11_test_arguments_3 }}' \ - name: run Load test fanout 4 times with 2GB 10-byte records total @@ -126,7 +126,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PtestJavaVersion=11 \ + -PcompileAndRunTestsWithJava11 \ -Pjava11Home=$JAVA_HOME_11_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11_test_arguments_4 }}' \ - name: run Load test fanout 8 times with 2GB 10-byte records total @@ -137,7 +137,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PtestJavaVersion=11 \ + -PcompileAndRunTestsWithJava11 \ -Pjava11Home=$JAVA_HOME_11_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11_test_arguments_5 }}' \ - name: run Load test reiterate 4 times 10kB values @@ -148,7 +148,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PtestJavaVersion=11 \ + -PcompileAndRunTestsWithJava11 \ -Pjava11Home=$JAVA_HOME_11_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11_test_arguments_6 }}' \ - name: run Load test reiterate 4 times 2MB values @@ -159,6 +159,6 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PtestJavaVersion=11 \ + -PcompileAndRunTestsWithJava11 \ -Pjava11Home=$JAVA_HOME_11_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11_test_arguments_7 }}' \ No newline at end of file diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml index cd8c34279593..5df7d3249192 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml @@ -95,7 +95,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PtestJavaVersion=17 \ + -PcompileAndRunTestsWithJava17 \ -Pjava17Home=$JAVA_HOME_17_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17_test_arguments_1 }}' \ - name: run Load test 2GB of 100B records @@ -106,7 +106,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PtestJavaVersion=17 \ + -PcompileAndRunTestsWithJava17 \ -Pjava17Home=$JAVA_HOME_17_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17_test_arguments_2 }}' \ - name: run Load test 2GB of 100kB records @@ -117,7 +117,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PtestJavaVersion=17 \ + -PcompileAndRunTestsWithJava17 \ -Pjava17Home=$JAVA_HOME_17_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17_test_arguments_3 }}' \ - name: run Load test fanout 4 times with 2GB 10-byte records total @@ -128,7 +128,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PtestJavaVersion=17 \ + -PcompileAndRunTestsWithJava17 \ -Pjava17Home=$JAVA_HOME_17_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17_test_arguments_4 }}' \ - name: run Load test fanout 8 times with 2GB 10-byte records total @@ -139,7 +139,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PtestJavaVersion=17 \ + -PcompileAndRunTestsWithJava17 \ -Pjava17Home=$JAVA_HOME_17_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17_test_arguments_5 }}' \ - name: run Load test reiterate 4 times 10kB values @@ -150,7 +150,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PtestJavaVersion=17 \ + -PcompileAndRunTestsWithJava17 \ -Pjava17Home=$JAVA_HOME_17_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17_test_arguments_6 }}' \ - name: run Load test reiterate 4 times 2MB values @@ -161,6 +161,6 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PtestJavaVersion=17 \ + -PcompileAndRunTestsWithJava17 \ -Pjava17Home=$JAVA_HOME_17_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17_test_arguments_7 }}' \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml index 8add1a2445c8..939d2646d352 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml @@ -111,7 +111,7 @@ jobs: gradle-command: :runners:google-cloud-dataflow-java:arm:examplesJavaRunnerV2IntegrationTestARM max-workers: 12 arguments: | - -PtestJavaVersion${{ matrix.java_version }} \ + -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ -Pcontainer-architecture-list=arm64,amd64 \ -Ppush-containers \ diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml index ab862df1847a..74e1787945e3 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml @@ -86,7 +86,7 @@ jobs: -PdisableSpotlessCheck=true \ -PdisableCheckStyle=true \ -PskipCheckerFramework \ - -PtestJavaVersion${{ matrix.java_version }} \ + -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml index 3f6352b0a948..0b9a366211ee 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml @@ -80,7 +80,7 @@ jobs: gradle-command: :sdks:java:testing:jpms-tests:dataflowRunnerIntegrationTest arguments: -PskipCheckerFramework - -PtestJavaVersion=17 + -PcompileAndRunTestsWithJava17 -Pjava17Home=$JAVA_HOME_17_X64 - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml index 360b9c757a02..fcdbd63f7e97 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml @@ -80,7 +80,7 @@ jobs: gradle-command: :sdks:java:testing:jpms-tests:directRunnerIntegrationTest arguments: -PskipCheckerFramework - -PtestJavaVersion=17 + -PcompileAndRunTestsWithJava17 -Pjava17Home=$JAVA_HOME_17_X64 - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml index 5782928a8b4d..d78793172898 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml @@ -107,7 +107,7 @@ jobs: with: gradle-command: :sdks:java:testing:nexmark:run arguments: | - -PtestJavaVersion${{ matrix.java_version }} \ + -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ -Pnexmark.runner.version=V2 \ -Pnexmark.runner=:runners:google-cloud-dataflow-java \ diff --git a/.github/workflows/beam_PostCommit_TransformService_Direct.yml b/.github/workflows/beam_PostCommit_TransformService_Direct.yml index 2ba83506d395..491688e097a0 100644 --- a/.github/workflows/beam_PostCommit_TransformService_Direct.yml +++ b/.github/workflows/beam_PostCommit_TransformService_Direct.yml @@ -84,7 +84,7 @@ jobs: with: gradle-command: :sdks:python:test-suites:direct:xlang:transformServicePythonUsingJava arguments: | - -PtestJavaVersion=11 \ + -PcompileAndRunTestsWithJava11 \ -Pjava11Home=$JAVA_HOME_11_X64 \ -PuseWheelDistribution \ -PpythonVersion=${{ matrix.python_version }} \ diff --git a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml index 186d75674c71..03f4d32861b1 100644 --- a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml +++ b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml @@ -115,7 +115,7 @@ jobs: -PdisableSpotlessCheck=true \ -PdisableCheckStyle=true \ -PskipCheckerFramework \ - -PtestJavaVersion=11 \ + -PcompileAndRunTestsWithJava11 \ -Pjava11Home=$JAVA_HOME_11_X64 \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 diff --git a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml index cafc8cf5ab6e..cf8cefe388f3 100644 --- a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml +++ b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml @@ -115,7 +115,7 @@ jobs: arguments: | -PdisableSpotlessCheck=true \ -PdisableCheckStyle=true \ - -PtestJavaVersion=17 \ + -PcompileAndRunTestsWithJava17 \ -PskipCheckerFramework \ -Pjava17Home=$JAVA_HOME_17_X64 \ max-workers: 12 diff --git a/.github/workflows/beam_PreCommit_SQL_Java11.yml b/.github/workflows/beam_PreCommit_SQL_Java11.yml index 3412365bf13e..f1c733418b8d 100644 --- a/.github/workflows/beam_PreCommit_SQL_Java11.yml +++ b/.github/workflows/beam_PreCommit_SQL_Java11.yml @@ -103,7 +103,7 @@ jobs: arguments: | -PdisableSpotlessCheck=true \ -PdisableCheckStyle=true \ - -PtestJavaVersion=11 \ + -PcompileAndRunTestsWithJava11 \ -PskipCheckerFramework \ -Pjava11Home=$JAVA_HOME_11_X64 \ - name: Archive JUnit Test Results diff --git a/.github/workflows/beam_PreCommit_SQL_Java17.yml b/.github/workflows/beam_PreCommit_SQL_Java17.yml index 2215bab8ddeb..7547bd396815 100644 --- a/.github/workflows/beam_PreCommit_SQL_Java17.yml +++ b/.github/workflows/beam_PreCommit_SQL_Java17.yml @@ -101,7 +101,7 @@ jobs: arguments: | -PdisableSpotlessCheck=true \ -PdisableCheckStyle=true \ - -PtestJavaVersion=17 \ + -PcompileAndRunTestsWithJava17 \ -PskipCheckerFramework \ -Pjava17Home=$JAVA_HOME_17_X64 \ - name: Archive JUnit Test Results diff --git a/.test-infra/jenkins/JavaTestProperties.groovy b/.test-infra/jenkins/JavaTestProperties.groovy index 5403cee5cf9a..ce7446a6e71d 100644 --- a/.test-infra/jenkins/JavaTestProperties.groovy +++ b/.test-infra/jenkins/JavaTestProperties.groovy @@ -17,10 +17,5 @@ */ class JavaTestProperties { - final static List SUPPORTED_CONTAINER_TASKS = [ - 'java8', - 'java11', - 'java17', - 'java21' - ] + final static List SUPPORTED_CONTAINER_TASKS = ['java8', 'java11', 'java17'] } diff --git a/.test-infra/jenkins/NexmarkBuilder.groovy b/.test-infra/jenkins/NexmarkBuilder.groovy index 69fa3dcc4277..044b0cbb9561 100644 --- a/.test-infra/jenkins/NexmarkBuilder.groovy +++ b/.test-infra/jenkins/NexmarkBuilder.groovy @@ -145,7 +145,7 @@ class NexmarkBuilder { rootBuildScriptDir(commonJobProperties.checkoutDir) tasks(':sdks:java:testing:nexmark:run') commonJobProperties.setGradleSwitches(delegate) - switches("-PtestJavaVersion=11") + switches("-PcompileAndRunTestsWithJava11") switches("-Pjava11Home=${commonJobProperties.JAVA_11_HOME}") switches("-Pnexmark.runner=${runner.getDependencyBySDK(sdk)}") switches("-Pnexmark.args=\"${parseOptions(options)}\"") @@ -168,7 +168,7 @@ class NexmarkBuilder { rootBuildScriptDir(commonJobProperties.checkoutDir) tasks(':sdks:java:testing:nexmark:run') commonJobProperties.setGradleSwitches(delegate) - switches("-PtestJavaVersion=17") + switches("-PcompileAndRunTestsWithJava17") switches("-Pjava17Home=${commonJobProperties.JAVA_17_HOME}") switches("-Pnexmark.runner=${runner.getDependencyBySDK(sdk)}") switches("-Pnexmark.args=\"${parseOptions(options)}\"") diff --git a/.test-infra/jenkins/job_LoadTests_CoGBK_Dataflow_V2_Java11.groovy b/.test-infra/jenkins/job_LoadTests_CoGBK_Dataflow_V2_Java11.groovy index 55501db4429d..fc7f39d28a0d 100644 --- a/.test-infra/jenkins/job_LoadTests_CoGBK_Dataflow_V2_Java11.groovy +++ b/.test-infra/jenkins/job_LoadTests_CoGBK_Dataflow_V2_Java11.groovy @@ -188,7 +188,7 @@ def loadTestConfigurations = { mode, isStreaming -> def final JOB_SPECIFIC_SWITCHES = [ '-Prunner.version="V2"', - '-PtestJavaVersion=11', + '-PcompileAndRunTestsWithJava11', "-Pjava11Home=${commonJobProperties.JAVA_11_HOME}" ] diff --git a/.test-infra/jenkins/job_LoadTests_CoGBK_Dataflow_V2_Java17.groovy b/.test-infra/jenkins/job_LoadTests_CoGBK_Dataflow_V2_Java17.groovy index 8fb09fd07448..ca8c6689ad0f 100644 --- a/.test-infra/jenkins/job_LoadTests_CoGBK_Dataflow_V2_Java17.groovy +++ b/.test-infra/jenkins/job_LoadTests_CoGBK_Dataflow_V2_Java17.groovy @@ -188,7 +188,7 @@ def loadTestConfigurations = { mode, isStreaming -> def final JOB_SPECIFIC_SWITCHES = [ '-Prunner.version="V2"', - '-PtestJavaVersion=17', + '-PcompileAndRunTestsWithJava17', "-Pjava17Home=${commonJobProperties.JAVA_17_HOME}" ] diff --git a/.test-infra/jenkins/job_LoadTests_GBK_Dataflow_V2_Java11.groovy b/.test-infra/jenkins/job_LoadTests_GBK_Dataflow_V2_Java11.groovy index 2191e448fade..cc2d5d2e5554 100644 --- a/.test-infra/jenkins/job_LoadTests_GBK_Dataflow_V2_Java11.groovy +++ b/.test-infra/jenkins/job_LoadTests_GBK_Dataflow_V2_Java11.groovy @@ -253,7 +253,7 @@ def loadTestConfigurations = { mode, isStreaming -> def final JOB_SPECIFIC_SWITCHES = [ '-Prunner.version="V2"', - '-PtestJavaVersion=11', + '-PcompileAndRunTestsWithJava11', "-Pjava11Home=${commonJobProperties.JAVA_11_HOME}" ] diff --git a/.test-infra/jenkins/job_LoadTests_GBK_Dataflow_V2_Java17.groovy b/.test-infra/jenkins/job_LoadTests_GBK_Dataflow_V2_Java17.groovy index 2520f68f0178..7405f9154b83 100644 --- a/.test-infra/jenkins/job_LoadTests_GBK_Dataflow_V2_Java17.groovy +++ b/.test-infra/jenkins/job_LoadTests_GBK_Dataflow_V2_Java17.groovy @@ -253,7 +253,7 @@ def loadTestConfigurations = { mode, isStreaming -> def final JOB_SPECIFIC_SWITCHES = [ '-Prunner.version="V2"', - '-PtestJavaVersion=17', + '-PcompileAndRunTestsWithJava17', "-Pjava17Home=${commonJobProperties.JAVA_17_HOME}" ] diff --git a/.test-infra/jenkins/job_LoadTests_ParDo_Dataflow_V2_Java11.groovy b/.test-infra/jenkins/job_LoadTests_ParDo_Dataflow_V2_Java11.groovy index b7154e840e6a..a4535d52e6cf 100644 --- a/.test-infra/jenkins/job_LoadTests_ParDo_Dataflow_V2_Java11.groovy +++ b/.test-infra/jenkins/job_LoadTests_ParDo_Dataflow_V2_Java11.groovy @@ -160,7 +160,7 @@ def commonLoadTestConfig = { jobType, isStreaming -> def final JOB_SPECIFIC_SWITCHES = [ '-Prunner.version="V2"', - '-PtestJavaVersion=11', + '-PcompileAndRunTestsWithJava11', "-Pjava11Home=${commonJobProperties.JAVA_11_HOME}" ] diff --git a/.test-infra/jenkins/job_LoadTests_ParDo_Dataflow_V2_Java17.groovy b/.test-infra/jenkins/job_LoadTests_ParDo_Dataflow_V2_Java17.groovy index df6c66e02aed..f0f2179ebb3b 100644 --- a/.test-infra/jenkins/job_LoadTests_ParDo_Dataflow_V2_Java17.groovy +++ b/.test-infra/jenkins/job_LoadTests_ParDo_Dataflow_V2_Java17.groovy @@ -160,7 +160,7 @@ def commonLoadTestConfig = { jobType, isStreaming -> def final JOB_SPECIFIC_SWITCHES = [ '-Prunner.version="V2"', - '-PtestJavaVersion=17', + '-PcompileAndRunTestsWithJava17', "-Pjava17Home=${commonJobProperties.JAVA_17_HOME}" ] diff --git a/.test-infra/jenkins/job_PostCommit_Java_Examples_Dataflow_V2_Java11.groovy b/.test-infra/jenkins/job_PostCommit_Java_Examples_Dataflow_V2_Java11.groovy index 6229f7c48a72..6687ae0e6f8a 100644 --- a/.test-infra/jenkins/job_PostCommit_Java_Examples_Dataflow_V2_Java11.groovy +++ b/.test-infra/jenkins/job_PostCommit_Java_Examples_Dataflow_V2_Java11.groovy @@ -43,7 +43,7 @@ PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_Examples_Dataflow_V2_ja commonJobProperties.setGradleSwitches(delegate, 3 * Runtime.runtime.availableProcessors()) switches '-PdisableSpotlessCheck=true' switches '-PdisableCheckStyle=true' - switches '-PtestJavaVersion=11' + switches '-PcompileAndRunTestsWithJava11' switches '-PskipCheckerFramework' switches "-Pjava11Home=${commonJobProperties.JAVA_11_HOME}" } diff --git a/.test-infra/jenkins/job_PostCommit_Java_Examples_Dataflow_V2_Java17.groovy b/.test-infra/jenkins/job_PostCommit_Java_Examples_Dataflow_V2_Java17.groovy index 7e52a7e09789..b275fe9276d9 100644 --- a/.test-infra/jenkins/job_PostCommit_Java_Examples_Dataflow_V2_Java17.groovy +++ b/.test-infra/jenkins/job_PostCommit_Java_Examples_Dataflow_V2_Java17.groovy @@ -43,7 +43,7 @@ PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_Examples_Dataflow_V2_ja commonJobProperties.setGradleSwitches(delegate, 3 * Runtime.runtime.availableProcessors()) switches '-PdisableSpotlessCheck=true' switches '-PdisableCheckStyle=true' - switches '-PtestJavaVersion=17' + switches '-PcompileAndRunTestsWithJava17' switches '-PskipCheckerFramework' switches "-Pjava17Home=${commonJobProperties.JAVA_17_HOME}" } diff --git a/.test-infra/jenkins/job_PostCommit_Java_Jpms_Dataflow_Java17.groovy b/.test-infra/jenkins/job_PostCommit_Java_Jpms_Dataflow_Java17.groovy index f518985ca7a8..4e26c164319e 100644 --- a/.test-infra/jenkins/job_PostCommit_Java_Jpms_Dataflow_Java17.groovy +++ b/.test-infra/jenkins/job_PostCommit_Java_Jpms_Dataflow_Java17.groovy @@ -42,7 +42,7 @@ PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_Jpms_Dataflow_Java17', tasks(':sdks:java:testing:jpms-tests:dataflowRunnerIntegrationTest') commonJobProperties.setGradleSwitches(delegate) switches("-PskipCheckerFramework") - switches("-PtestJavaVersion=17") + switches("-PcompileAndRunTestsWithJava17") switches("-Pjava17Home=${commonJobProperties.JAVA_17_HOME}") // Specify maven home on Jenkins, needed by Maven archetype integration tests. switches('-Pmaven_home=/home/jenkins/tools/maven/apache-maven-3.5.4') diff --git a/.test-infra/jenkins/job_PostCommit_Java_Jpms_Direct_Java17.groovy b/.test-infra/jenkins/job_PostCommit_Java_Jpms_Direct_Java17.groovy index 04c31389ecbe..f31373ecaada 100644 --- a/.test-infra/jenkins/job_PostCommit_Java_Jpms_Direct_Java17.groovy +++ b/.test-infra/jenkins/job_PostCommit_Java_Jpms_Direct_Java17.groovy @@ -42,7 +42,7 @@ PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_Jpms_Direct_Java17', 'R tasks(':sdks:java:testing:jpms-tests:directRunnerIntegrationTest') commonJobProperties.setGradleSwitches(delegate) switches("-PskipCheckerFramework") - switches("-PtestJavaVersion=17") + switches("-PcompileAndRunTestsWithJava17") switches("-Pjava17Home=${commonJobProperties.JAVA_17_HOME}") // Specify maven home on Jenkins, needed by Maven archetype integration tests. switches('-Pmaven_home=/home/jenkins/tools/maven/apache-maven-3.5.4') diff --git a/.test-infra/jenkins/job_PostCommit_TransformService_Direct.groovy b/.test-infra/jenkins/job_PostCommit_TransformService_Direct.groovy index 03d29069a52c..0d7f58e71706 100644 --- a/.test-infra/jenkins/job_PostCommit_TransformService_Direct.groovy +++ b/.test-infra/jenkins/job_PostCommit_TransformService_Direct.groovy @@ -43,7 +43,7 @@ PostcommitJobBuilder.postCommitJob('beam_PostCommit_TransformService_Direct', rootBuildScriptDir(commonJobProperties.checkoutDir) tasks(':sdks:python:test-suites:direct:xlang:transformServicePythonUsingJava') commonJobProperties.setGradleSwitches(delegate) - switches '-PtestJavaVersion=11' + switches '-PcompileAndRunTestsWithJava11' switches "-Pjava11Home=${commonJobProperties.JAVA_11_HOME}" switches("-PuseWheelDistribution") switches("-PpythonVersion=${pythonVersion}") diff --git a/.test-infra/jenkins/job_PreCommit_Java_Examples_Dataflow_Java11.groovy b/.test-infra/jenkins/job_PreCommit_Java_Examples_Dataflow_Java11.groovy index 9f733a42ccce..cabf5dbdd457 100644 --- a/.test-infra/jenkins/job_PreCommit_Java_Examples_Dataflow_Java11.groovy +++ b/.test-infra/jenkins/job_PreCommit_Java_Examples_Dataflow_Java11.groovy @@ -50,7 +50,7 @@ builder.build { switches '-PdisableSpotlessCheck=true' switches '-PdisableCheckStyle=true' switches '-PskipCheckerFramework' // Gradle itself is running under JDK8 so plugin configures wrong for JDK11 - switches '-PtestJavaVersion=11' + switches '-PcompileAndRunTestsWithJava11' switches "-Pjava11Home=${properties.JAVA_11_HOME}" properties.setGradleSwitches(delegate, 3 * Runtime.runtime.availableProcessors()) } diff --git a/.test-infra/jenkins/job_PreCommit_SQL_Java11.groovy b/.test-infra/jenkins/job_PreCommit_SQL_Java11.groovy index 34154b37209e..9742ab756cf7 100644 --- a/.test-infra/jenkins/job_PreCommit_SQL_Java11.groovy +++ b/.test-infra/jenkins/job_PreCommit_SQL_Java11.groovy @@ -26,7 +26,7 @@ PrecommitJobBuilder builder = new PrecommitJobBuilder( gradleSwitches: [ '-PdisableSpotlessCheck=true', '-PdisableCheckStyle=true', - '-PtestJavaVersion=11', + '-PcompileAndRunTestsWithJava11', '-PskipCheckerFramework', // Gradle itself is running under JDK8 so plugin configures wrong for JDK11 "-Pjava11Home=${properties.JAVA_11_HOME}" diff --git a/.test-infra/jenkins/job_PreCommit_SQL_Java17.groovy b/.test-infra/jenkins/job_PreCommit_SQL_Java17.groovy index 472521cb1b08..158fa683c1a8 100644 --- a/.test-infra/jenkins/job_PreCommit_SQL_Java17.groovy +++ b/.test-infra/jenkins/job_PreCommit_SQL_Java17.groovy @@ -26,7 +26,7 @@ PrecommitJobBuilder builder = new PrecommitJobBuilder( gradleSwitches: [ '-PdisableSpotlessCheck=true', '-PdisableCheckStyle=true', - '-PtestJavaVersion=17', + '-PcompileAndRunTestsWithJava17', '-PskipCheckerFramework', // Gradle itself is running under JDK8 so plugin configures wrong for JDK17 "-Pjava17Home=${properties.JAVA_17_HOME}" diff --git a/.test-infra/jenkins/job_Precommit_Java_Examples_Dataflow_Java17.groovy b/.test-infra/jenkins/job_Precommit_Java_Examples_Dataflow_Java17.groovy index 387b707123e7..3654a4c75edb 100644 --- a/.test-infra/jenkins/job_Precommit_Java_Examples_Dataflow_Java17.groovy +++ b/.test-infra/jenkins/job_Precommit_Java_Examples_Dataflow_Java17.groovy @@ -50,7 +50,7 @@ builder.build { switches '-PdisableSpotlessCheck=true' switches '-PdisableCheckStyle=true' switches '-PskipCheckerFramework' // Gradle itself is running under JDK8 so plugin configures wrong for JDK17 - switches '-PtestJavaVersion=17' + switches '-PcompileAndRunTestsWithJava17' switches "-Pjava17Home=${properties.JAVA_17_HOME}" properties.setGradleSwitches(delegate, 3 * Runtime.runtime.availableProcessors()) } diff --git a/build.gradle.kts b/build.gradle.kts index 5bcfbb3ed06e..ea1b4e6784e3 100644 --- a/build.gradle.kts +++ b/build.gradle.kts @@ -712,12 +712,14 @@ if (project.hasProperty("javaLinkageArtifactIds")) { } } } -if (project.hasProperty("testJavaVersion")) { - var testVer = project.property("testJavaVersion") - - tasks.getByName("javaPreCommitPortabilityApi").dependsOn(":sdks:java:testing:test-utils:verifyJavaVersion$testVer") - tasks.getByName("javaExamplesDataflowPrecommit").dependsOn(":sdks:java:testing:test-utils:verifyJavaVersion$testVer") - tasks.getByName("sqlPreCommit").dependsOn(":sdks:java:testing:test-utils:verifyJavaVersion$testVer") +if (project.hasProperty("compileAndRunTestsWithJava11")) { + tasks.getByName("javaPreCommitPortabilityApi").dependsOn(":sdks:java:testing:test-utils:verifyJavaVersion") + tasks.getByName("javaExamplesDataflowPrecommit").dependsOn(":sdks:java:testing:test-utils:verifyJavaVersion") + tasks.getByName("sqlPreCommit").dependsOn(":sdks:java:testing:test-utils:verifyJavaVersion") +} else if (project.hasProperty("compileAndRunTestsWithJava17")) { + tasks.getByName("javaPreCommitPortabilityApi").dependsOn(":sdks:java:testing:test-utils:verifyJavaVersion17") + tasks.getByName("javaExamplesDataflowPrecommit").dependsOn(":sdks:java:testing:test-utils:verifyJavaVersion17") + tasks.getByName("sqlPreCommit").dependsOn(":sdks:java:testing:test-utils:verifyJavaVersion17") } else { allprojects { tasks.withType(Test::class).configureEach { diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index 6fa5ff0ee5f3..efc08a6371dc 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -946,29 +946,6 @@ class BeamModulePlugin implements Plugin { ] } - project.ext.setJava21Options = { CompileOptions options -> - def java17Home = project.findProperty("java17Home") - options.fork = true - options.forkOptions.javaHome = java17Home as File - options.compilerArgs += ['-Xlint:-path'] - // Error prone requires some packages to be exported/opened for Java 17 - // Disabling checks since this property is only used for Jenkins tests - // https://github.com/tbroyer/gradle-errorprone-plugin#jdk-16-support - options.errorprone.errorproneArgs.add("-XepDisableAllChecks") - options.forkOptions.jvmArgs += [ - "-J--add-exports=jdk.compiler/com.sun.tools.javac.api=ALL-UNNAMED", - "-J--add-exports=jdk.compiler/com.sun.tools.javac.file=ALL-UNNAMED", - "-J--add-exports=jdk.compiler/com.sun.tools.javac.main=ALL-UNNAMED", - "-J--add-exports=jdk.compiler/com.sun.tools.javac.model=ALL-UNNAMED", - "-J--add-exports=jdk.compiler/com.sun.tools.javac.parser=ALL-UNNAMED", - "-J--add-exports=jdk.compiler/com.sun.tools.javac.processing=ALL-UNNAMED", - "-J--add-exports=jdk.compiler/com.sun.tools.javac.tree=ALL-UNNAMED", - "-J--add-exports=jdk.compiler/com.sun.tools.javac.util=ALL-UNNAMED", - "-J--add-opens=jdk.compiler/com.sun.tools.javac.code=ALL-UNNAMED", - "-J--add-opens=jdk.compiler/com.sun.tools.javac.comp=ALL-UNNAMED" - ] - } - project.ext.repositories = { maven { name "testPublicationLocal" @@ -1515,7 +1492,7 @@ class BeamModulePlugin implements Plugin { options.errorprone.errorproneArgs.add("-Xep:Slf4jLoggerShouldBeNonStatic:OFF") } - if (project.findProperty('testJavaVersion') == "11") { + if (project.hasProperty("compileAndRunTestsWithJava11")) { def java11Home = project.findProperty("java11Home") project.tasks.compileTestJava { options.fork = true @@ -1527,7 +1504,7 @@ class BeamModulePlugin implements Plugin { useJUnit() executable = "${java11Home}/bin/java" } - } else if (project.findProperty('testJavaVersion') == "17") { + } else if (project.hasProperty("compileAndRunTestsWithJava17")) { def java17Home = project.findProperty("java17Home") project.tasks.compileTestJava { setCompileAndRuntimeJavaVersion(options.compilerArgs, '17') @@ -1537,16 +1514,6 @@ class BeamModulePlugin implements Plugin { useJUnit() executable = "${java17Home}/bin/java" } - } else if (project.findProperty('testJavaVersion') == "21") { - def java21Home = project.findProperty("java21Home") - project.tasks.compileTestJava { - setCompileAndRuntimeJavaVersion(options.compilerArgs, '21') - project.ext.setJava17Options(options) - } - project.tasks.withType(Test).configureEach { - useJUnit() - executable = "${java21Home}/bin/java" - } } if (configuration.shadowClosure) { diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Environments.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Environments.java index f531b5be344d..31a555989afd 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Environments.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Environments.java @@ -94,8 +94,7 @@ public class Environments { public enum JavaVersion { java8("java", "1.8", 8), java11("java11", "11", 11), - java17("java17", "17", 17), - java21("java21", "21", 21); + java17("java17", "17", 17); // Legacy name, as used in container image private final String legacyName; @@ -120,7 +119,6 @@ public String specification() { return this.specification; } - /** Return the LTS java version given the Java specification version. */ public static JavaVersion forSpecification(String specification) { for (JavaVersion ver : JavaVersion.values()) { if (ver.specification.equals(specification)) { @@ -139,7 +137,7 @@ public static JavaVersion forSpecification(String specification) { } } LOG.warn( - "Unsupported Java version: {}, falling back to: {}", + "unsupported Java version: {}, falling back to: {}", specification, fallback.specification); return fallback; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/EnvironmentsTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/EnvironmentsTest.java index b71a654f1031..ae429fb1fe6d 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/EnvironmentsTest.java +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/EnvironmentsTest.java @@ -291,8 +291,6 @@ public void testLtsJavaVersion() { assertEquals("java11", JavaVersion.java11.legacyName()); assertEquals(JavaVersion.java17, JavaVersion.forSpecification("17")); assertEquals("java17", JavaVersion.java17.legacyName()); - assertEquals(JavaVersion.java21, JavaVersion.forSpecification("21")); - assertEquals("java21", JavaVersion.java21.legacyName()); } @Test @@ -305,9 +303,7 @@ public void testNonLtsJavaVersion() { assertEquals(JavaVersion.java17, JavaVersion.forSpecification("15")); assertEquals(JavaVersion.java17, JavaVersion.forSpecification("16")); assertEquals(JavaVersion.java17, JavaVersion.forSpecification("18")); - assertEquals(JavaVersion.java21, JavaVersion.forSpecification("19")); - assertEquals(JavaVersion.java21, JavaVersion.forSpecification("20")); - assertEquals(JavaVersion.java21, JavaVersion.forSpecification("21")); + assertEquals(JavaVersion.java17, JavaVersion.forSpecification("19")); } @Test(expected = UnsupportedOperationException.class) diff --git a/runners/google-cloud-dataflow-java/arm/build.gradle b/runners/google-cloud-dataflow-java/arm/build.gradle index 71cbc7c58e86..e79eeedcd828 100644 --- a/runners/google-cloud-dataflow-java/arm/build.gradle +++ b/runners/google-cloud-dataflow-java/arm/build.gradle @@ -76,8 +76,10 @@ dependencies { } def javaVer = "java8" -if (project.hasProperty('testJavaVersion')) { - javaVer = "java${project.getProperty('testJavaVersion')}" +if(project.hasProperty('compileAndRunTestsWithJava17')) { + javaVer = "java17" +} else if(project.hasProperty('compileAndRunTestsWithJava11')) { + javaVer = "java11" } def dataflowProject = project.findProperty('dataflowProject') ?: 'apache-beam-testing' def dataflowRegion = project.findProperty('dataflowRegion') ?: 'us-central1' diff --git a/runners/google-cloud-dataflow-java/build.gradle b/runners/google-cloud-dataflow-java/build.gradle index e4f34687d31c..adc1f2e09bc4 100644 --- a/runners/google-cloud-dataflow-java/build.gradle +++ b/runners/google-cloud-dataflow-java/build.gradle @@ -279,8 +279,10 @@ def createRunnerV2ValidatesRunnerTest = { Map args -> // task ordering such that the registry doesn't get cleaned up prior to task completion. def buildAndPushDockerJavaContainer = tasks.register("buildAndPushDockerJavaContainer") { def javaVer = "java8" - if(project.hasProperty('testJavaVersion')) { - javaVer = "java${project.getProperty('testJavaVersion')}" + if(project.hasProperty('compileAndRunTestsWithJava17')) { + javaVer = "java17" + } else if(project.hasProperty('compileAndRunTestsWithJava11')) { + javaVer = "java11" } dependsOn ":sdks:java:container:${javaVer}:docker" def defaultDockerImageName = containerImageName( diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java index 891b4c0454c9..26548038a1df 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java @@ -398,9 +398,10 @@ && isServiceEndpoint(dataflowOptions.getDataflowEndpoint())) { // Adding the Java version to the SDK name for user's and support convenience. String agentJavaVer = "(JRE 8 environment)"; - if (Environments.getJavaVersion() != Environments.JavaVersion.java8) { - agentJavaVer = - String.format("(JRE %s environment)", Environments.getJavaVersion().specification()); + if (Environments.getJavaVersion() == Environments.JavaVersion.java17) { + agentJavaVer = "(JRE 17 environment)"; + } else if (Environments.getJavaVersion() == Environments.JavaVersion.java11) { + agentJavaVer = "(JRE 11 environment)"; } DataflowRunnerInfo dataflowRunnerInfo = DataflowRunnerInfo.getDataflowRunnerInfo(); diff --git a/runners/spark/spark_runner.gradle b/runners/spark/spark_runner.gradle index 74013de6107d..d0dbe453ddfb 100644 --- a/runners/spark/spark_runner.gradle +++ b/runners/spark/spark_runner.gradle @@ -63,9 +63,8 @@ def sparkTestProperties(overrides = [:]) { def sparkTestJvmArgs() { - // run tests with Java 17 using -PtestJavaVersion=17 -Pjava17Home=??? - if (project.hasProperty('testJavaVersion') && - project.getProperty('testJavaVersion') in ['17', '21']) { + // run tests with Java 17 using -PcompileAndRunTestsWithJava17 -Pjava17Home=??? + if (project.hasProperty("compileAndRunTestsWithJava17")) { return [ "--add-opens=java.base/sun.nio.ch=ALL-UNNAMED", // add-opens below required for Kryo FieldSerializer / SparkRunnerKryoRegistratorTest diff --git a/sdks/java/container/Dockerfile b/sdks/java/container/Dockerfile index 9c266ea132b8..e0fa8d4a0a6f 100644 --- a/sdks/java/container/Dockerfile +++ b/sdks/java/container/Dockerfile @@ -15,9 +15,8 @@ # See the License for the specific language governing permissions and # limitations under the License. ############################################################################### -ARG base_image ARG java_version -FROM ${base_image}:${java_version} +FROM eclipse-temurin:${java_version} LABEL Author "Apache Beam " ARG TARGETOS ARG TARGETARCH diff --git a/sdks/java/container/common.gradle b/sdks/java/container/common.gradle index 9f6cbe64a294..cc427494ed6e 100644 --- a/sdks/java/container/common.gradle +++ b/sdks/java/container/common.gradle @@ -29,7 +29,6 @@ applyDockerNature() if (!project.hasProperty('imageJavaVersion')) { throw new GradleException('imageJavaVersion project property must be set') } -def javaBaseImage = project.findProperty('javaBaseImage') ?: 'eclipse-temurin' def imageJavaVersion = project.findProperty('imageJavaVersion') description = "Apache Beam :: SDKs :: Java :: Container :: Java ${imageJavaVersion} Container" @@ -72,19 +71,19 @@ task copySdkHarnessLauncher(type: Copy) { } task copyJavaThirdPartyLicenses(type: Copy) { - from project(':sdks:java:container').layout.buildDirectory.dir('target/third_party_licenses') + from("${project(':sdks:java:container').buildDir}/target/third_party_licenses") into "build/target/third_party_licenses" dependsOn ':sdks:java:container:pullLicenses' } task copyGolangLicenses(type: Copy) { - from project(':release:go-licenses:java').layout.buildDirectory.dir('output') + from "${project(':release:go-licenses:java').buildDir}/output" into "build/target/go-licenses" dependsOn ':release:go-licenses:java:createLicenses' } task copyJdkOptions(type: Copy) { - if (["11", "17", "21"].contains(imageJavaVersion)) { + if (imageJavaVersion == "17" || imageJavaVersion == "11") { from "option-jamm.json" } from "java${imageJavaVersion}-security.properties" @@ -98,10 +97,10 @@ task skipPullLicenses(type: Exec) { } task validateJavaHome { - if (["11", "17", "21"].contains(imageJavaVersion)) { + if (imageJavaVersion == "11" || imageJavaVersion == "17") { doFirst { - if (!project.hasProperty("java${imageJavaVersion}Home")) { - throw new GradleException("java${imageJavaVersion}Home property required for imageJavaVersion=${imageJavaVersion}. Re-run with -Pjava${imageJavaVersion}Home") + if (!project.hasProperty('java17Home') && !project.hasProperty('java11Home')) { + throw new GradleException('java17Home or java11Home property required. Re-run with -Pjava17Home or -Pjava11Home') } } } @@ -125,7 +124,6 @@ docker { buildArgs([ 'pull_licenses': project.rootProject.hasProperty(["docker-pull-licenses"]) || project.rootProject.hasProperty(["isRelease"]), - 'base_image': javaBaseImage, 'java_version': imageJavaVersion, ]) buildx useBuildx diff --git a/sdks/java/container/java21/build.gradle b/sdks/java/container/java21/build.gradle deleted file mode 100644 index 038064102dcb..000000000000 --- a/sdks/java/container/java21/build.gradle +++ /dev/null @@ -1,30 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * License); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an AS IS BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -project.ext { - // TODO(https://github.com/apache/beam/issues/28120) switch to temurin once available - javaBaseImage = 'openjdk' - imageJavaVersion = '21' -} - -// Load the main build script which contains all build logic. -apply from: "../common.gradle" - -dependencies { - dockerDependency project(path: ":sdks:java:container:agent") -} \ No newline at end of file diff --git a/sdks/java/container/java21/option-jamm.json b/sdks/java/container/java21/option-jamm.json deleted file mode 100644 index 5647ff66be5c..000000000000 --- a/sdks/java/container/java21/option-jamm.json +++ /dev/null @@ -1,12 +0,0 @@ -{ - "name": "jamm", - "enabled": true, - "options": { - "java_arguments": [ - "--add-modules=jamm", - "--module-path=/opt/apache/beam/jars/jamm.jar", - "--add-opens=java.base/java.lang=jamm", - "--add-opens=java.base/java.util=jamm" - ] - } -} \ No newline at end of file diff --git a/sdks/java/testing/jpms-tests/build.gradle b/sdks/java/testing/jpms-tests/build.gradle index 2a25463931c1..6321f874c903 100644 --- a/sdks/java/testing/jpms-tests/build.gradle +++ b/sdks/java/testing/jpms-tests/build.gradle @@ -23,8 +23,10 @@ plugins { } // overwrite javaVersion before applyJavaNature -if (project.hasProperty("testJavaVersion")) { - javaVersion = "1.${project.getProperty('testJavaVersion')}" as String +if (project.hasProperty("compileAndRunTestsWithJava17")) { + javaVersion = '1.17' +} else { + javaVersion = '1.11' } applyJavaNature( @@ -40,14 +42,13 @@ ext.summary = "E2E test for Java 9 modules" // direct compileJava to use specified java version. project.tasks.compileJava { - if (project.hasProperty('testJavaVersion')) { + if (project.hasProperty("compileAndRunTestsWithJava11")) { options.fork = true - options.forkOptions.javaHome = project.findProperty("java${project.getProperty('testJavaVersion')}Home") as File - if (project.getProperty('testJavaVersion') == '17') { - setJava17Options(options) - } else if (project.getProperty('testJavaVersion') == '21') { - setJava21Options(options) - } + options.forkOptions.javaHome = project.findProperty("java11Home") as File + } else if (project.hasProperty("compileAndRunTestsWithJava17")) { + options.fork = true + options.forkOptions.javaHome = project.findProperty("java17Home") as File + setJava17Options(options) } } @@ -119,7 +120,7 @@ plugins.withType(JavaPlugin).configureEach{ // JPMS requires JDK > 8 project.tasks.each { it.onlyIf { - project.hasProperty('testJavaVersion') + project.hasProperty("compileAndRunTestsWithJava17") || JavaVersion.VERSION_1_8.compareTo(JavaVersion.current()) < 0 } } diff --git a/sdks/java/testing/test-utils/build.gradle b/sdks/java/testing/test-utils/build.gradle index 6e30693d8894..50c815dd57f7 100644 --- a/sdks/java/testing/test-utils/build.gradle +++ b/sdks/java/testing/test-utils/build.gradle @@ -43,15 +43,24 @@ dependencies { testRuntimeOnly project(path: ":runners:direct-java", configuration: "shadowTest") } -['11', '17', '21'].each { - tasks.create(name: "verifyJavaVersion${it}", type: Test) { - filter { - includeTestsMatching "org.apache.beam.sdk.testutils.jvmverification.JvmVerification.verifyCodeIsCompiledWithJava8" - includeTestsMatching "org.apache.beam.sdk.testutils.jvmverification.JvmVerification.verifyTestCodeIsCompiledWithJava${it}" - includeTestsMatching "org.apache.beam.sdk.testutils.jvmverification.JvmVerification.verifyRunningJVMVersionIs${it}" - } - doLast { - println 'Java verified' - } +task verifyJavaVersion(type: Test) { + filter { + includeTestsMatching 'org.apache.beam.sdk.testutils.jvmverification.JvmVerification.verifyCodeIsCompiledWithJava8' + includeTestsMatching 'org.apache.beam.sdk.testutils.jvmverification.JvmVerification.verifyTestCodeIsCompiledWithJava11' + includeTestsMatching 'org.apache.beam.sdk.testutils.jvmverification.JvmVerification.verifyRunningJVMVersionIs11' + } + doLast { + println 'Java verified' } } + +task verifyJavaVersion17(type: Test) { + filter { + includeTestsMatching 'org.apache.beam.sdk.testutils.jvmverification.JvmVerification.verifyCodeIsCompiledWithJava8' + includeTestsMatching 'org.apache.beam.sdk.testutils.jvmverification.JvmVerification.verifyTestCodeIsCompiledWithJava17' + includeTestsMatching 'org.apache.beam.sdk.testutils.jvmverification.JvmVerification.verifyRunningJVMVersionIs17' + } + doLast { + println 'Java verified' + } +} \ No newline at end of file diff --git a/sdks/java/testing/test-utils/src/test/java/org/apache/beam/sdk/testutils/jvmverification/JvmVerification.java b/sdks/java/testing/test-utils/src/test/java/org/apache/beam/sdk/testutils/jvmverification/JvmVerification.java index a6b5d6dca6c1..ad29e8b6a1d6 100644 --- a/sdks/java/testing/test-utils/src/test/java/org/apache/beam/sdk/testutils/jvmverification/JvmVerification.java +++ b/sdks/java/testing/test-utils/src/test/java/org/apache/beam/sdk/testutils/jvmverification/JvmVerification.java @@ -20,7 +20,6 @@ import static org.apache.beam.sdk.testutils.jvmverification.JvmVerification.Java.v11; import static org.apache.beam.sdk.testutils.jvmverification.JvmVerification.Java.v17; import static org.apache.beam.sdk.testutils.jvmverification.JvmVerification.Java.v1_8; -import static org.apache.beam.sdk.testutils.jvmverification.JvmVerification.Java.v21; import static org.junit.Assert.assertEquals; import java.io.IOException; @@ -40,7 +39,6 @@ public class JvmVerification { versionMapping.put("0034", v1_8); versionMapping.put("0037", v11); versionMapping.put("003d", v17); - versionMapping.put("0041", v21); } // bytecode @@ -64,11 +62,6 @@ public void verifyTestCodeIsCompiledWithJava17() throws IOException { assertEquals(v17, getByteCodeVersion(JvmVerification.class)); } - @Test - public void verifyTestCodeIsCompiledWithJava21() throws IOException { - assertEquals(v21, getByteCodeVersion(JvmVerification.class)); - } - // jvm @Test public void verifyRunningJVMVersionIs11() { @@ -82,12 +75,6 @@ public void verifyRunningJVMVersionIs17() { assertEquals(v17.name, version); } - @Test - public void verifyRunningJVMVersionIs21() { - final String version = getJavaSpecification(); - assertEquals(v21.name, version); - } - private static Java getByteCodeVersion(final Class clazz) throws IOException { final InputStream stream = clazz.getClassLoader().getResourceAsStream(clazz.getName().replace(".", "/") + ".class"); @@ -104,8 +91,7 @@ private static String getJavaSpecification() { enum Java { v1_8("1.8"), v11("11"), - v17("17"), - v21("21"); + v17("17"); final String name; diff --git a/settings.gradle.kts b/settings.gradle.kts index b71ce249c155..1c851da92e60 100644 --- a/settings.gradle.kts +++ b/settings.gradle.kts @@ -175,7 +175,6 @@ include(":sdks:java:container:agent") include(":sdks:java:container:java8") include(":sdks:java:container:java11") include(":sdks:java:container:java17") -include(":sdks:java:container:java21") include(":sdks:java:core") include(":sdks:java:core:jmh") include(":sdks:java:expansion-service") diff --git a/website/www/site/content/en/roadmap/java-sdk.md b/website/www/site/content/en/roadmap/java-sdk.md index a1c85e139193..b65424b57a3d 100644 --- a/website/www/site/content/en/roadmap/java-sdk.md +++ b/website/www/site/content/en/roadmap/java-sdk.md @@ -17,9 +17,9 @@ limitations under the License. # Java SDK Roadmap -## Next Java LTS version support (Java 21) +## Next Java LTS version support (Java 17) Work to support the next LTS release of Java is in progress. For more details -about the scope and info on the various tasks please see the GitHub Issue. +about the scope and info on the various tasks please see the JIRA ticket. -- GitHub: [#28120](https://github.com/apache/beam/issues/28120) +- JIRA: [BEAM-12240](https://issues.apache.org/jira/browse/BEAM-12240) From dad13cfbc9a94a316be3e64fd4147e78ec1866d8 Mon Sep 17 00:00:00 2001 From: Anand Inguva <34158215+AnandInguva@users.noreply.github.com> Date: Fri, 13 Oct 2023 14:54:21 +0000 Subject: [PATCH 133/435] Fix `watch_file_pattern` condition in RunInference (#28948) * Fix bug for watch_file_pattern * Add fix to CHANGES.md * Remove _enable_side_input_loading flag * Add tests --- CHANGES.md | 1 + sdks/python/apache_beam/ml/inference/base.py | 5 ++--- .../apache_beam/ml/inference/base_test.py | 22 +++++++++++++++++++ 3 files changed, 25 insertions(+), 3 deletions(-) diff --git a/CHANGES.md b/CHANGES.md index 43ade8c530b9..5db1b11c595a 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -79,6 +79,7 @@ ## Bugfixes * Fixed "Desired bundle size 0 bytes must be greater than 0" in Java SDK's BigtableIO.BigtableSource when you have more cores than bytes to read (Java) [#28793](https://github.com/apache/beam/issues/28793). +* `watch_file_pattern` arg of the [RunInference](https://github.com/apache/beam/blob/104c10b3ee536a9a3ea52b4dbf62d86b669da5d9/sdks/python/apache_beam/ml/inference/base.py#L997) arg had no effect prior to 2.52.0. To use the behavior of arg `watch_file_pattern` prior to 2.52.0, follow the documentation at https://beam.apache.org/documentation/ml/side-input-updates/ and use `WatchFilePattern` PTransform as a SideInput. ([#28948](https://github.com/apache/beam/pulls/28948)) ## Security Fixes * Fixed (CVE-YYYY-NNNN)[https://www.cve.org/CVERecord?id=CVE-YYYY-NNNN] (Java/Python/Go) ([#X](https://github.com/apache/beam/issues/X)). diff --git a/sdks/python/apache_beam/ml/inference/base.py b/sdks/python/apache_beam/ml/inference/base.py index 753e1468137c..45c5078c13cf 100644 --- a/sdks/python/apache_beam/ml/inference/base.py +++ b/sdks/python/apache_beam/ml/inference/base.py @@ -1032,7 +1032,6 @@ def __init__( self._clock = clock self._metrics_namespace = metrics_namespace self._model_metadata_pcoll = model_metadata_pcoll - self._enable_side_input_loading = self._model_metadata_pcoll is not None self._with_exception_handling = False self._watch_model_pattern = watch_model_pattern self._kwargs = kwargs @@ -1133,12 +1132,12 @@ def expand( self._model_handler, self._clock, self._metrics_namespace, - self._enable_side_input_loading, + self._model_metadata_pcoll is not None, self._model_tag), self._inference_args, beam.pvalue.AsSingleton( self._model_metadata_pcoll, - ) if self._enable_side_input_loading else None).with_resource_hints( + ) if self._model_metadata_pcoll else None).with_resource_hints( **resource_hints) if self._with_exception_handling: diff --git a/sdks/python/apache_beam/ml/inference/base_test.py b/sdks/python/apache_beam/ml/inference/base_test.py index 1b1a7393872c..7075810ff0f0 100644 --- a/sdks/python/apache_beam/ml/inference/base_test.py +++ b/sdks/python/apache_beam/ml/inference/base_test.py @@ -1513,6 +1513,28 @@ def test_model_manager_evicts_correct_num_of_models_after_being_incremented( mh3.load_model, tag=tag3).acquire() self.assertEqual(8, model3.predict(10)) + def test_run_inference_watch_file_pattern_side_input_label(self): + pipeline = TestPipeline() + # label of the WatchPattern transform. + side_input_str = 'WatchFilePattern/ApplyGlobalWindow' + from apache_beam.ml.inference.utils import WatchFilePattern + file_pattern_side_input = ( + pipeline + | 'WatchFilePattern' >> WatchFilePattern(file_pattern='fake/path/*')) + pcoll = pipeline | 'start' >> beam.Create([1, 2, 3]) + result_pcoll = pcoll | base.RunInference( + FakeModelHandler(), model_metadata_pcoll=file_pattern_side_input) + assert side_input_str in str(result_pcoll.producer.side_inputs[0]) + + def test_run_inference_watch_file_pattern_keyword_arg_side_input_label(self): + # label of the WatchPattern transform. + side_input_str = 'WatchFilePattern/ApplyGlobalWindow' + pipeline = TestPipeline() + pcoll = pipeline | 'start' >> beam.Create([1, 2, 3]) + result_pcoll = pcoll | base.RunInference( + FakeModelHandler(), watch_model_pattern='fake/path/*') + assert side_input_str in str(result_pcoll.producer.side_inputs[0]) + if __name__ == '__main__': unittest.main() From 58e3bdd678b459c7a29b379edacddfebda42d0ad Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Fri, 13 Oct 2023 11:07:49 -0400 Subject: [PATCH 134/435] Shutdown more Jenkins PreCommits (#28840) --- ...ob_PreCommit_Java_Examples_Dataflow.groovy | 43 -------------- ...ommit_Java_Examples_Dataflow_Java11.groovy | 58 ------------------ .../jenkins/job_PreCommit_Java_IOs.groovy | 54 ++++++++--------- ...ortableValidatesRunner_Flink_Docker.groovy | 41 ------------- .../job_PreCommit_Java_Spark3_Versions.groovy | 37 ------------ .../job_PreCommit_Portable_Python.groovy | 59 ------------------- .test-infra/jenkins/job_PreCommit_SQL.groovy | 53 ----------------- .../jenkins/job_PreCommit_SQL_Java11.groovy | 52 ---------------- .../jenkins/job_PreCommit_SQL_Java17.groovy | 51 ---------------- ...ommit_Java_Examples_Dataflow_Java17.groovy | 58 ------------------ 10 files changed, 27 insertions(+), 479 deletions(-) delete mode 100644 .test-infra/jenkins/job_PreCommit_Java_Examples_Dataflow.groovy delete mode 100644 .test-infra/jenkins/job_PreCommit_Java_Examples_Dataflow_Java11.groovy delete mode 100644 .test-infra/jenkins/job_PreCommit_Java_PortableValidatesRunner_Flink_Docker.groovy delete mode 100644 .test-infra/jenkins/job_PreCommit_Java_Spark3_Versions.groovy delete mode 100644 .test-infra/jenkins/job_PreCommit_Portable_Python.groovy delete mode 100644 .test-infra/jenkins/job_PreCommit_SQL.groovy delete mode 100644 .test-infra/jenkins/job_PreCommit_SQL_Java11.groovy delete mode 100644 .test-infra/jenkins/job_PreCommit_SQL_Java17.groovy delete mode 100644 .test-infra/jenkins/job_Precommit_Java_Examples_Dataflow_Java17.groovy diff --git a/.test-infra/jenkins/job_PreCommit_Java_Examples_Dataflow.groovy b/.test-infra/jenkins/job_PreCommit_Java_Examples_Dataflow.groovy deleted file mode 100644 index 109456b3bc4f..000000000000 --- a/.test-infra/jenkins/job_PreCommit_Java_Examples_Dataflow.groovy +++ /dev/null @@ -1,43 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import PrecommitJobBuilder - -PrecommitJobBuilder builder = new PrecommitJobBuilder( - scope: this, - nameBase: 'Java_Examples_Dataflow', - gradleTask: ':javaExamplesDataflowPreCommit', - gradleSwitches: [ - '-PdisableSpotlessCheck=true', - '-PdisableCheckStyle=true' - ], // spotless checked in separate pre-commit - triggerPathPatterns: [ - '^model/.*$', - '^sdks/java/.*$', - '^runners/google-cloud-dataflow-java/.*$', - '^examples/java/.*$', - '^examples/kotlin/.*$', - '^release/.*$', - ], - timeoutMins: 60, - ) -builder.build { - publishers { - archiveJunit('**/build/test-results/**/*.xml') - } -} diff --git a/.test-infra/jenkins/job_PreCommit_Java_Examples_Dataflow_Java11.groovy b/.test-infra/jenkins/job_PreCommit_Java_Examples_Dataflow_Java11.groovy deleted file mode 100644 index cabf5dbdd457..000000000000 --- a/.test-infra/jenkins/job_PreCommit_Java_Examples_Dataflow_Java11.groovy +++ /dev/null @@ -1,58 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import PrecommitJobBuilder -import CommonJobProperties as properties - -PrecommitJobBuilder builder = new PrecommitJobBuilder( - scope: this, - nameBase: 'Java_Examples_Dataflow_Java11', - gradleTask: ':clean', - gradleSwitches: [ - '-PdisableSpotlessCheck=true', - '-PdisableCheckStyle=true', - '-PskipCheckerFramework' // Gradle itself is running under JDK8 so plugin configures wrong for JDK11 - ], // spotless checked in separate pre-commit - triggerPathPatterns: [ - '^model/.*$', - '^sdks/java/.*$', - '^runners/google-cloud-dataflow-java/.*$', - '^examples/java/.*$', - '^examples/kotlin/.*$', - '^release/.*$', - ], - timeoutMins: 60, - ) -builder.build { - publishers { - archiveJunit('**/build/test-results/**/*.xml') - } - - steps { - gradle { - rootBuildScriptDir(properties.checkoutDir) - tasks ':runners:google-cloud-dataflow-java:examples:preCommit' - switches '-PdisableSpotlessCheck=true' - switches '-PdisableCheckStyle=true' - switches '-PskipCheckerFramework' // Gradle itself is running under JDK8 so plugin configures wrong for JDK11 - switches '-PcompileAndRunTestsWithJava11' - switches "-Pjava11Home=${properties.JAVA_11_HOME}" - properties.setGradleSwitches(delegate, 3 * Runtime.runtime.availableProcessors()) - } - } -} diff --git a/.test-infra/jenkins/job_PreCommit_Java_IOs.groovy b/.test-infra/jenkins/job_PreCommit_Java_IOs.groovy index edeeed5f0970..09bf1982d127 100644 --- a/.test-infra/jenkins/job_PreCommit_Java_IOs.groovy +++ b/.test-infra/jenkins/job_PreCommit_Java_IOs.groovy @@ -73,34 +73,34 @@ def ioModulesMap = [ // These projects are also covered by 'Java_IOs_Direct', and won't trigger on default patterns. false: [ - 'amqp', - 'cassandra', - 'cdap', - 'clickhouse', - 'csv', - 'debezium', - 'elasticsearch', - 'file-schema-transform', + // 'amqp', + // 'cassandra', + // 'cdap', + // 'clickhouse', + // 'csv', + // 'debezium', + // 'elasticsearch', + // 'file-schema-transform', 'google-ads', - 'hbase', - 'hcatalog', - 'influxdb', - 'jdbc', - 'jms', - 'kafka', - 'kudu', - 'mongodb', - 'mqtt', - 'neo4j', - 'parquet', - 'rabbitmq', - 'redis', - 'singlestore', - 'snowflake', - 'solr', - 'splunk', - 'thrift', - 'tika' + // 'hbase', + // 'hcatalog', + // 'influxdb', + // 'jdbc', + // 'jms', + // 'kafka', + // 'kudu', + // 'mongodb', + // 'mqtt', + // 'neo4j', + // 'parquet', + // 'rabbitmq', + // 'redis', + // 'singlestore', + // 'snowflake', + // 'solr', + // 'splunk', + // 'thrift', + // 'tika' ] ] diff --git a/.test-infra/jenkins/job_PreCommit_Java_PortableValidatesRunner_Flink_Docker.groovy b/.test-infra/jenkins/job_PreCommit_Java_PortableValidatesRunner_Flink_Docker.groovy deleted file mode 100644 index bb14a792291c..000000000000 --- a/.test-infra/jenkins/job_PreCommit_Java_PortableValidatesRunner_Flink_Docker.groovy +++ /dev/null @@ -1,41 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonTestProperties -import PrecommitJobBuilder - -// This job runs a limited subset of ValidatesRunner tests against the Flink runner in the docker environment. -PrecommitJobBuilder builder = new PrecommitJobBuilder( - scope: this, - nameBase: 'Java_PVR_Flink_Docker', - gradleTask: ":runners:flink:${CommonTestProperties.getFlinkVersion()}:job-server:validatesPortableRunnerDocker", - timeoutMins: 240, - triggerPathPatterns: [ - '^sdks/java/core/src/test/java/org/apache/beam/sdk/.*$', - '^sdks/java/container/.*$', - '^sdks/java/harness/.*$', - '^runners/flink/.*$', - '^runners/java-fn-execution/.*$', - ], - ) -builder.build { - // Publish all test results to Jenkins. - publishers { - archiveJunit('**/build/test-results/**/*.xml') - } -} diff --git a/.test-infra/jenkins/job_PreCommit_Java_Spark3_Versions.groovy b/.test-infra/jenkins/job_PreCommit_Java_Spark3_Versions.groovy deleted file mode 100644 index f13c4c0a1e2b..000000000000 --- a/.test-infra/jenkins/job_PreCommit_Java_Spark3_Versions.groovy +++ /dev/null @@ -1,37 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import PrecommitJobBuilder - -PrecommitJobBuilder builder = new PrecommitJobBuilder( - scope: this, - nameBase: 'Java_Spark3_Versions', - gradleTask: ':runners:spark:3:sparkVersionsTest', - gradleSwitches: [ - '-PdisableSpotlessCheck=true' - ], // spotless checked in separate pre-commit - triggerPathPatterns: [ - '^runners/spark/.*$', - ], - timeoutMins: 120, - ) -builder.build { - publishers { - archiveJunit('**/build/test-results/**/*.xml') - } -} \ No newline at end of file diff --git a/.test-infra/jenkins/job_PreCommit_Portable_Python.groovy b/.test-infra/jenkins/job_PreCommit_Portable_Python.groovy deleted file mode 100644 index 2992cbbd0d06..000000000000 --- a/.test-infra/jenkins/job_PreCommit_Portable_Python.groovy +++ /dev/null @@ -1,59 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonJobProperties as commonJobProperties -import PrecommitJobBuilder -import static PythonTestProperties.LOWEST_SUPPORTED -import static PythonTestProperties.HIGHEST_SUPPORTED - -PrecommitJobBuilder builder = new PrecommitJobBuilder( - scope: this, - nameBase: 'Portable_Python', - gradleTask: ':clean', // Do nothing here. Add test configs below. - triggerPathPatterns: [ - '^model/.*$', - '^runners/core-construction-java/.*$', - '^runners/core-java/.*$', - '^runners/extensions-java/.*$', - '^runners/flink/.*$', - '^runners/java-fn-execution/.*$', - '^runners/reference/.*$', - '^sdks/python/.*$', - '^release/.*$', - ] - ) - -builder.build { - // Due to BEAM-7993, run multiple Python version of portable precommit - // tests in parallel could lead python3 container crash. We manually - // config gradle steps here to run tests in sequential. - def lowestSupported = LOWEST_SUPPORTED.replace('.', '') - def highestSupported = HIGHEST_SUPPORTED.replace('.', '') - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(":sdks:python:test-suites:portable:py${lowestSupported}:preCommitPy${lowestSupported}") - commonJobProperties.setGradleSwitches(delegate) - } - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(":sdks:python:test-suites:portable:py${highestSupported}:preCommitPy${highestSupported}") - commonJobProperties.setGradleSwitches(delegate) - } - } -} diff --git a/.test-infra/jenkins/job_PreCommit_SQL.groovy b/.test-infra/jenkins/job_PreCommit_SQL.groovy deleted file mode 100644 index 3cd81e330cac..000000000000 --- a/.test-infra/jenkins/job_PreCommit_SQL.groovy +++ /dev/null @@ -1,53 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import PrecommitJobBuilder - -PrecommitJobBuilder builder = new PrecommitJobBuilder( - scope: this, - nameBase: 'SQL', - gradleTask: ':sqlPreCommit', - gradleSwitches: [ - '-PdisableSpotlessCheck=true', - '-PdisableCheckStyle=true', - '-PenableJacocoReport' - ], // spotless checked in job_PreCommit_Spotless - triggerPathPatterns: [ - '^sdks/java/extensions/sql.*$', - ], - numBuildsToRetain: 40 - ) -builder.build { - publishers { - archiveJunit('**/build/test-results/**/*.xml') - recordIssues { - tools { - errorProne() - spotBugs { - pattern('**/build/reports/spotbugs/*.xml') - } - } - enabledForFailure(true) - } - jacocoCodeCoverage { - execPattern('**/build/jacoco/*.exec') - exclusionPattern('**/AutoValue_*') - inclusionPattern("**/org/apache/beam/sdk/extensions/sql/**") - } - } -} diff --git a/.test-infra/jenkins/job_PreCommit_SQL_Java11.groovy b/.test-infra/jenkins/job_PreCommit_SQL_Java11.groovy deleted file mode 100644 index 9742ab756cf7..000000000000 --- a/.test-infra/jenkins/job_PreCommit_SQL_Java11.groovy +++ /dev/null @@ -1,52 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import PrecommitJobBuilder -import CommonJobProperties as properties - -PrecommitJobBuilder builder = new PrecommitJobBuilder( - scope: this, - nameBase: 'SQL_Java11', - gradleTask: ':sqlPreCommit', - gradleSwitches: [ - '-PdisableSpotlessCheck=true', - '-PdisableCheckStyle=true', - '-PcompileAndRunTestsWithJava11', - '-PskipCheckerFramework', - // Gradle itself is running under JDK8 so plugin configures wrong for JDK11 - "-Pjava11Home=${properties.JAVA_11_HOME}" - ], // spotless checked in job_PreCommit_Spotless - triggerPathPatterns: [ - '^sdks/java/extensions/sql.*$', - ] - ) -builder.build { - publishers { - archiveJunit('**/build/test-results/**/*.xml') - recordIssues { - tools { - errorProne() - java() - spotBugs { - pattern('**/build/reports/spotbugs/*.xml') - } - } - enabledForFailure(true) - } - } -} diff --git a/.test-infra/jenkins/job_PreCommit_SQL_Java17.groovy b/.test-infra/jenkins/job_PreCommit_SQL_Java17.groovy deleted file mode 100644 index 158fa683c1a8..000000000000 --- a/.test-infra/jenkins/job_PreCommit_SQL_Java17.groovy +++ /dev/null @@ -1,51 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import PrecommitJobBuilder -import CommonJobProperties as properties - -PrecommitJobBuilder builder = new PrecommitJobBuilder( - scope: this, - nameBase: 'SQL_Java17', - gradleTask: ':sqlPreCommit', - gradleSwitches: [ - '-PdisableSpotlessCheck=true', - '-PdisableCheckStyle=true', - '-PcompileAndRunTestsWithJava17', - '-PskipCheckerFramework', - // Gradle itself is running under JDK8 so plugin configures wrong for JDK17 - "-Pjava17Home=${properties.JAVA_17_HOME}" - ], // spotless checked in job_PreCommit_Spotless - triggerPathPatterns: [ - '^sdks/java/extensions/sql.*$', - ] - ) -builder.build { - publishers { - archiveJunit('**/build/test-results/**/*.xml') - recordIssues { - tools { - java() - spotBugs { - pattern('**/build/reports/spotbugs/*.xml') - } - } - enabledForFailure(true) - } - } -} diff --git a/.test-infra/jenkins/job_Precommit_Java_Examples_Dataflow_Java17.groovy b/.test-infra/jenkins/job_Precommit_Java_Examples_Dataflow_Java17.groovy deleted file mode 100644 index 3654a4c75edb..000000000000 --- a/.test-infra/jenkins/job_Precommit_Java_Examples_Dataflow_Java17.groovy +++ /dev/null @@ -1,58 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import PrecommitJobBuilder -import CommonJobProperties as properties - -PrecommitJobBuilder builder = new PrecommitJobBuilder( - scope: this, - nameBase: 'Java_Examples_Dataflow_Java17', - gradleTask: ':clean', - gradleSwitches: [ - '-PdisableSpotlessCheck=true', - '-PdisableCheckStyle=true', - '-PskipCheckerFramework' // Gradle itself is running under JDK8 so plugin configures wrong for JDK17 - ], // spotless checked in separate pre-commit - triggerPathPatterns: [ - '^model/.*$', - '^sdks/java/.*$', - '^runners/google-cloud-dataflow-java/.*$', - '^examples/java/.*$', - '^examples/kotlin/.*$', - '^release/.*$', - ], - timeoutMins: 60, - ) -builder.build { - publishers { - archiveJunit('**/build/test-results/**/*.xml') - } - - steps { - gradle { - rootBuildScriptDir(properties.checkoutDir) - tasks ':runners:google-cloud-dataflow-java:examples:preCommit' - switches '-PdisableSpotlessCheck=true' - switches '-PdisableCheckStyle=true' - switches '-PskipCheckerFramework' // Gradle itself is running under JDK8 so plugin configures wrong for JDK17 - switches '-PcompileAndRunTestsWithJava17' - switches "-Pjava17Home=${properties.JAVA_17_HOME}" - properties.setGradleSwitches(delegate, 3 * Runtime.runtime.availableProcessors()) - } - } -} From 3a45ecf4b271997b5ce03e1181676356eaa351e1 Mon Sep 17 00:00:00 2001 From: liferoad Date: Fri, 13 Oct 2023 12:02:25 -0400 Subject: [PATCH 135/435] Updated the DoFn documentation with pickling (#28970) Co-authored-by: tvalentyn --- .../content/en/documentation/programming-guide.md | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/website/www/site/content/en/documentation/programming-guide.md b/website/www/site/content/en/documentation/programming-guide.md index 98dd045f4281..564b01a7146e 100644 --- a/website/www/site/content/en/documentation/programming-guide.md +++ b/website/www/site/content/en/documentation/programming-guide.md @@ -1212,10 +1212,13 @@ Here is a sequence diagram that shows the lifecycle of the DoFn during the execution of the ParDo transform. The comments give useful information to pipeline developers such as the constraints that apply to the objects or particular cases such as failover or - instance reuse. They also give instantiation use cases. Two key points - to note are that (1) teardown is done on a best effort basis and thus - isn't guaranteed and (2) the number of DoFn instances is runner - dependent. + instance reuse. They also give instantiation use cases. Three key points + to note are that: + 1. Teardown is done on a best effort basis and thus + isn't guaranteed. + 2. The number of DoFn instances created at runtime is runner-dependent. + 3. For the Python SDK, the pipeline contents such as DoFn user code, + is [serialized into a bytecode](https://beam.apache.org/documentation/sdks/python-pipeline-dependencies/#pickling-and-managing-the-main-session). Therefore, `DoFn`s should not reference objects that are not serializable, such as locks. To manage a single instance of an object across multiple `DoFn` instances in the same process, use utilities in the [shared.py](https://beam.apache.org/releases/pydoc/current/apache_beam.utils.shared.html) module. ![This is a sequence diagram that shows the lifecycle of the DoFn](/images/dofn-sequence-diagram.svg) From f4f1c3c22d94d824e39eb3b878cc3f4db9b9b801 Mon Sep 17 00:00:00 2001 From: Vitaly Terentyev Date: Fri, 13 Oct 2023 20:20:52 +0400 Subject: [PATCH 136/435] Add LoadTests Java ParDo Dataflow V2 workflows (#28744) * Add LoadTests Java ParDo Dataflow V2 workflows * Refactoring * Refactoring --- ...a_ParDo_Dataflow_V2_Batch_JavaVersions.yml | 136 ++++++++++++++++++ ...rDo_Dataflow_V2_Streaming_JavaVersions.yml | 136 ++++++++++++++++++ ...low_V2_Batch_JavaVersions_100_counters.txt | 28 ++++ ...flow_V2_Batch_JavaVersions_10_counters.txt | 28 ++++ ...ataflow_V2_Batch_JavaVersions_10_times.txt | 28 ++++ ...taflow_V2_Batch_JavaVersions_200_times.txt | 28 ++++ ...V2_Streaming_JavaVersions_100_counters.txt | 29 ++++ ..._V2_Streaming_JavaVersions_10_counters.txt | 29 ++++ ...low_V2_Streaming_JavaVersions_10_times.txt | 29 ++++ ...ow_V2_Streaming_JavaVersions_200_times.txt | 29 ++++ 10 files changed, 500 insertions(+) create mode 100644 .github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions.yml create mode 100644 .github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions.yml create mode 100644 .github/workflows/load-tests-job-configs/java_ParDo_Dataflow_V2_Batch_JavaVersions_100_counters.txt create mode 100644 .github/workflows/load-tests-job-configs/java_ParDo_Dataflow_V2_Batch_JavaVersions_10_counters.txt create mode 100644 .github/workflows/load-tests-job-configs/java_ParDo_Dataflow_V2_Batch_JavaVersions_10_times.txt create mode 100644 .github/workflows/load-tests-job-configs/java_ParDo_Dataflow_V2_Batch_JavaVersions_200_times.txt create mode 100644 .github/workflows/load-tests-job-configs/java_ParDo_Dataflow_V2_Streaming_JavaVersions_100_counters.txt create mode 100644 .github/workflows/load-tests-job-configs/java_ParDo_Dataflow_V2_Streaming_JavaVersions_10_counters.txt create mode 100644 .github/workflows/load-tests-job-configs/java_ParDo_Dataflow_V2_Streaming_JavaVersions_10_times.txt create mode 100644 .github/workflows/load-tests-job-configs/java_ParDo_Dataflow_V2_Streaming_JavaVersions_200_times.txt diff --git a/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions.yml b/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions.yml new file mode 100644 index 000000000000..eebc96a63959 --- /dev/null +++ b/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions.yml @@ -0,0 +1,136 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: LoadTests Java ParDo Dataflow V2 Batch JavaVersions + +on: + issue_comment: + types: [created] + schedule: + - cron: '40 16 * * *' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + (contains(github.event.comment.body, 'Run Load Tests Java') && + contains(github.event.comment.body, 'ParDo Dataflow V2 Batch')) + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 720 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase_1 }} ${{ matrix.java_version }} ${{ matrix.job_phrase_2 }}) + strategy: + fail-fast: false + matrix: + job_name: ["beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions"] + job_phrase_1: ["Run Load Tests Java"] + job_phrase_2: ["ParDo Dataflow V2 Batch"] + java_version: ['11','17'] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase_1 }} ${{ matrix.java_version }} ${{ matrix.job_phrase_2 }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase_1 }} ${{ matrix.java_version }} ${{ matrix.job_phrase_2 }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action + with: + java-version: ${{ matrix.java_version }} + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: java + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_V2_Batch_JavaVersions_10_times.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_V2_Batch_JavaVersions_200_times.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_V2_Batch_JavaVersions_10_counters.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_V2_Batch_JavaVersions_100_counters.txt + arguments: | + --influxTags={\"runnerVersion\":\"v2\",\"jdk\":\"java${{ matrix.java_version }}\"} + # The env variables are created and populated in the test-arguments-action as "_test_arguments_" + - name: run ParDo Dataflow V2 Batch Java Load Test 1 (10 times) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.ParDoLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + -Prunner.version=V2 \ + -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions_test_arguments_1 }} --appName=load_tests_Java${{ matrix.java_version }}_Dataflow_V2_batch_ParDo_1' \ + - name: run ParDo Dataflow V2 Batch Java Load Test 2 (200 times) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.ParDoLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + -Prunner.version=V2 \ + -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions_test_arguments_2 }} --appName=load_tests_Java${{ matrix.java_version }}_Dataflow_V2_batch_ParDo_2' \ + - name: run ParDo Dataflow V2 Batch Java Load Test 3 (10 counters) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.ParDoLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + -Prunner.version=V2 \ + -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions_test_arguments_3 }} --appName=load_tests_Java${{ matrix.java_version }}_Dataflow_V2_batch_ParDo_3' \ + - name: run ParDo Dataflow V2 Batch Java Load Test 4 (100 counters) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.ParDoLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + -Prunner.version=V2 \ + -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions_test_arguments_4 }} --appName=load_tests_Java${{ matrix.java_version }}_Dataflow_V2_batch_ParDo_4' \ No newline at end of file diff --git a/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions.yml b/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions.yml new file mode 100644 index 000000000000..dcec86f8b4e0 --- /dev/null +++ b/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions.yml @@ -0,0 +1,136 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: LoadTests Java ParDo Dataflow V2 Streaming JavaVersions + +on: + issue_comment: + types: [created] + schedule: + - cron: '30 21 * * *' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + (contains(github.event.comment.body, 'Run Load Tests Java') && + contains(github.event.comment.body, 'ParDo Dataflow V2 Streaming')) + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 720 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase_1 }} ${{ matrix.java_version }} ${{ matrix.job_phrase_2 }}) + strategy: + fail-fast: false + matrix: + job_name: ["beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions"] + job_phrase_1: ["Run Load Tests Java"] + job_phrase_2: ["ParDo Dataflow V2 Streaming"] + java_version: ['11','17'] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase_1 }} ${{ matrix.java_version }} ${{ matrix.job_phrase_2 }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase_1 }} ${{ matrix.java_version }} ${{ matrix.job_phrase_2 }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action + with: + java-version: ${{ matrix.java_version }} + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: java + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_V2_Streaming_JavaVersions_10_times.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_V2_Streaming_JavaVersions_200_times.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_V2_Streaming_JavaVersions_10_counters.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_V2_Streaming_JavaVersions_100_counters.txt + arguments: | + --influxTags={\"runnerVersion\":\"v2\",\"jdk\":\"java${{ matrix.java_version }}\"} + # The env variables are created and populated in the test-arguments-action as "_test_arguments_" + - name: run ParDo Dataflow V2 Streaming Java Load Test 1 (10 times) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.ParDoLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + -Prunner.version=V2 \ + -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions_test_arguments_1 }} --appName=load_tests_Java${{ matrix.java_version }}_Dataflow_V2_streaming_ParDo_1' \ + - name: run ParDo Dataflow V2 Streaming Java Load Test 2 (200 times) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.ParDoLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + -Prunner.version=V2 \ + -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions_test_arguments_2 }} --appName=load_tests_Java${{ matrix.java_version }}_Dataflow_V2_streaming_ParDo_2' \ + - name: run ParDo Dataflow V2 Streaming Java Load Test 3 (10 counters) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.ParDoLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + -Prunner.version=V2 \ + -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions_test_arguments_3 }} --appName=load_tests_Java${{ matrix.java_version }}_Dataflow_V2_streaming_ParDo_3' \ + - name: run ParDo Dataflow V2 Streaming Java Load Test 4 (100 counters) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.ParDoLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + -Prunner.version=V2 \ + -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions_test_arguments_4 }} --appName=load_tests_Java${{ matrix.java_version }}_Dataflow_V2_streaming_ParDo_4' \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_V2_Batch_JavaVersions_100_counters.txt b/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_V2_Batch_JavaVersions_100_counters.txt new file mode 100644 index 000000000000..7f05aedab763 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_V2_Batch_JavaVersions_100_counters.txt @@ -0,0 +1,28 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_pardo_4 +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90} +--iterations=1 +--numberOfCounters=1 +--numberOfCounterOperations=100 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=false +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_V2_Batch_JavaVersions_10_counters.txt b/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_V2_Batch_JavaVersions_10_counters.txt new file mode 100644 index 000000000000..e9c158698c57 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_V2_Batch_JavaVersions_10_counters.txt @@ -0,0 +1,28 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_pardo_3 +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90} +--iterations=1 +--numberOfCounters=1 +--numberOfCounterOperations=10 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=false +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_V2_Batch_JavaVersions_10_times.txt b/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_V2_Batch_JavaVersions_10_times.txt new file mode 100644 index 000000000000..cd6b006d8f3a --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_V2_Batch_JavaVersions_10_times.txt @@ -0,0 +1,28 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_pardo_1 +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90} +--iterations=10 +--numberOfCounters=1 +--numberOfCounterOperations=0 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=false +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_V2_Batch_JavaVersions_200_times.txt b/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_V2_Batch_JavaVersions_200_times.txt new file mode 100644 index 000000000000..d4b6bef42f78 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_V2_Batch_JavaVersions_200_times.txt @@ -0,0 +1,28 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_pardo_2 +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90} +--iterations=200 +--numberOfCounters=1 +--numberOfCounterOperations=0 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=false +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_V2_Streaming_JavaVersions_100_counters.txt b/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_V2_Streaming_JavaVersions_100_counters.txt new file mode 100644 index 000000000000..1dcb6f4a7cbb --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_V2_Streaming_JavaVersions_100_counters.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_streaming_pardo_4 +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90} +--iterations=1 +--numberOfCounters=1 +--numberOfCounterOperations=100 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=true +--inputWindowDurationSec=1200 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_V2_Streaming_JavaVersions_10_counters.txt b/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_V2_Streaming_JavaVersions_10_counters.txt new file mode 100644 index 000000000000..34fb4b4658d8 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_V2_Streaming_JavaVersions_10_counters.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_streaming_pardo_3 +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90} +--iterations=1 +--numberOfCounters=1 +--numberOfCounterOperations=10 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=true +--inputWindowDurationSec=1200 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_V2_Streaming_JavaVersions_10_times.txt b/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_V2_Streaming_JavaVersions_10_times.txt new file mode 100644 index 000000000000..9d4d918f2e6d --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_V2_Streaming_JavaVersions_10_times.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_streaming_pardo_1 +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90} +--iterations=10 +--numberOfCounters=1 +--numberOfCounterOperations=0 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=true +--inputWindowDurationSec=1200 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_V2_Streaming_JavaVersions_200_times.txt b/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_V2_Streaming_JavaVersions_200_times.txt new file mode 100644 index 000000000000..c7d0e3e4835a --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_V2_Streaming_JavaVersions_200_times.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_streaming_pardo_2 +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90} +--iterations=200 +--numberOfCounters=1 +--numberOfCounterOperations=0 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=true +--inputWindowDurationSec=1200 +--runner=DataflowRunner \ No newline at end of file From dd40403752f50cc8be4cdfbc5d204c9a5c884104 Mon Sep 17 00:00:00 2001 From: magicgoody <131876064+magicgoody@users.noreply.github.com> Date: Fri, 13 Oct 2023 22:22:14 +0600 Subject: [PATCH 137/435] beam_LoadTests_Python_GBK_reiterate_Dataflow (#28861) --- ...ests_Python_Combine_Dataflow_Streaming.yml | 114 ++++++++++++++++++ ...ts_Python_GBK_reiterate_Dataflow_Batch.yml | 104 ++++++++++++++++ ...ython_GBK_reiterate_Dataflow_Streaming.yml | 104 ++++++++++++++++ ...Dataflow_Streaming_2GB_10_byte_records.txt | 29 +++++ ...ombine_Dataflow_Streaming_2GB_Fanout_4.txt | 30 +++++ ...ombine_Dataflow_Streaming_2GB_Fanout_8.txt | 30 +++++ ...ow_Batch_reiterate_4_times_10kB_values.txt | 28 +++++ ...low_Batch_reiterate_4_times_2MB_values.txt | 28 +++++ ...treaming_reiterate_4_times_10kB_values.txt | 30 +++++ ...Streaming_reiterate_4_times_2MB_values.txt | 30 +++++ 10 files changed, 527 insertions(+) create mode 100644 .github/workflows/beam_LoadTests_Python_Combine_Dataflow_Streaming.yml create mode 100644 .github/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Batch.yml create mode 100644 .github/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Streaming.yml create mode 100644 .github/workflows/load-tests-job-configs/python_Combine_Dataflow_Streaming_2GB_10_byte_records.txt create mode 100644 .github/workflows/load-tests-job-configs/python_Combine_Dataflow_Streaming_2GB_Fanout_4.txt create mode 100644 .github/workflows/load-tests-job-configs/python_Combine_Dataflow_Streaming_2GB_Fanout_8.txt create mode 100644 .github/workflows/load-tests-job-configs/python_GBK_reiterate_Dataflow_Batch_reiterate_4_times_10kB_values.txt create mode 100644 .github/workflows/load-tests-job-configs/python_GBK_reiterate_Dataflow_Batch_reiterate_4_times_2MB_values.txt create mode 100644 .github/workflows/load-tests-job-configs/python_GBK_reiterate_Dataflow_Streaming_reiterate_4_times_10kB_values.txt create mode 100644 .github/workflows/load-tests-job-configs/python_GBK_reiterate_Dataflow_Streaming_reiterate_4_times_2MB_values.txt diff --git a/.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Streaming.yml new file mode 100644 index 000000000000..005bddfa49ac --- /dev/null +++ b/.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Streaming.yml @@ -0,0 +1,114 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: LoadTests Python Combine Dataflow Streaming + +on: + issue_comment: + types: [created] + schedule: + - cron: '10 9 * * *' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_LoadTests_Python_Combine_Dataflow_Streaming: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Load Tests Python Combine Dataflow Streaming' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 720 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_LoadTests_Python_Combine_Dataflow_Streaming"] + job_phrase: ["Run Load Tests Python Combine Dataflow Streaming"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action + with: + python-version: 3.8 + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: python + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_Combine_Dataflow_Streaming_2GB_10_byte_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_Combine_Dataflow_Streaming_2GB_Fanout_4.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_Combine_Dataflow_Streaming_2GB_Fanout_8.txt + - name: get current time + run: echo "NOW_UTC=$(date '+%m%d%H%M%S' --utc)" >> $GITHUB_ENV + # The env variables are created and populated in the test-arguments-action as "_test_arguments_" + - name: run 2GB 10 byte records test + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.combine_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_Combine_Dataflow_Streaming_test_arguments_1 }} --job_name=load-tests-python-dataflow-streaming-combine-1-${{env.NOW_UTC}}' \ + - name: run 2GB Fanout 4 test + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.combine_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_Combine_Dataflow_Streaming_test_arguments_2 }} --job_name=load-tests-python-dataflow-streaming-combine-4-${{env.NOW_UTC}}' \ + - name: run 2GB Fanout 8 test + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.combine_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_Combine_Dataflow_Streaming_test_arguments_3 }} --job_name=load-tests-python-dataflow-streaming-combine-5-${{env.NOW_UTC}}' \ No newline at end of file diff --git a/.github/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Batch.yml new file mode 100644 index 000000000000..febc54c0c23e --- /dev/null +++ b/.github/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Batch.yml @@ -0,0 +1,104 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: LoadTests Python GBK reiterate Dataflow Batch + +on: + issue_comment: + types: [created] + schedule: + - cron: '10 1 * * *' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_LoadTests_Python_GBK_reiterate_Dataflow_Batch: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Load Tests Python GBK reiterate Dataflow Batch' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 720 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_LoadTests_Python_GBK_reiterate_Dataflow_Batch"] + job_phrase: ["Run Load Tests Python GBK reiterate Dataflow Batch"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action + with: + python-version: 3.8 + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: python + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_GBK_reiterate_Dataflow_Batch_reiterate_4_times_10kB_values.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_GBK_reiterate_Dataflow_Batch_reiterate_4_times_2MB_values.txt + - name: get current time + run: echo "NOW_UTC=$(date '+%m%d%H%M%S' --utc)" >> $GITHUB_ENV + # The env variables are created and populated in the test-arguments-action as "_test_arguments_" + - name: run reiterate 4 times 10kB values test + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.group_by_key_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_GBK_reiterate_Dataflow_Batch_test_arguments_1 }} --job_name=load-tests-python-dataflow-batch-gbk-6-${{env.NOW_UTC}}' \ + - name: run reiterate 4 times 2MB values test + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.group_by_key_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_GBK_reiterate_Dataflow_Batch_test_arguments_2 }} --job_name=load-tests-python-dataflow-batch-gbk-7-${{env.NOW_UTC}}' \ No newline at end of file diff --git a/.github/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Streaming.yml new file mode 100644 index 000000000000..a904c74d24b4 --- /dev/null +++ b/.github/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Streaming.yml @@ -0,0 +1,104 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: LoadTests Python GBK reiterate Dataflow Streaming + +on: + issue_comment: + types: [created] + schedule: + - cron: '10 5 * * *' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_LoadTests_Python_GBK_reiterate_Dataflow_Streaming: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Load Tests Python GBK reiterate Dataflow Streaming' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 720 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_LoadTests_Python_GBK_reiterate_Dataflow_Streaming"] + job_phrase: ["Run Load Tests Python GBK reiterate Dataflow Streaming"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action + with: + python-version: 3.8 + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: python + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_GBK_reiterate_Dataflow_Streaming_reiterate_4_times_10kB_values.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_GBK_reiterate_Dataflow_Streaming_reiterate_4_times_2MB_values.txt + - name: get current time + run: echo "NOW_UTC=$(date '+%m%d%H%M%S' --utc)" >> $GITHUB_ENV + # The env variables are created and populated in the test-arguments-action as "_test_arguments_" + - name: run reiterate 4 times 10kB values test + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.group_by_key_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_GBK_reiterate_Dataflow_Streaming_test_arguments_1 }} --job_name=load-tests-python-dataflow-streaming-gbk-6-${{env.NOW_UTC}}' \ + - name: run reiterate 4 times 2MB values test + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.group_by_key_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_GBK_reiterate_Dataflow_Streaming_test_arguments_2 }} --job_name=load-tests-python-dataflow-streaming-gbk-7-${{env.NOW_UTC}}' \ diff --git a/.github/workflows/load-tests-job-configs/python_Combine_Dataflow_Streaming_2GB_10_byte_records.txt b/.github/workflows/load-tests-job-configs/python_Combine_Dataflow_Streaming_2GB_10_byte_records.txt new file mode 100644 index 000000000000..8535f85c737d --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_Combine_Dataflow_Streaming_2GB_10_byte_records.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/smoketests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_streaming_combine_1 +--influx_measurement=python_streaming_combine_1 +--input_options=''{\\"num_records\\":200000000,\\"key_size\\":1,\\"value_size\\":9,\\"algorithm\\":\\"lcg\\"}'' +--num_workers=5 +--autoscaling_algorithm=NONE +--top_count=20 +--streaming +--experiments=use_runner_v2 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_Combine_Dataflow_Streaming_2GB_Fanout_4.txt b/.github/workflows/load-tests-job-configs/python_Combine_Dataflow_Streaming_2GB_Fanout_4.txt new file mode 100644 index 000000000000..03b26b9d4e13 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_Combine_Dataflow_Streaming_2GB_Fanout_4.txt @@ -0,0 +1,30 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/smoketests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_streaming_combine_4 +--influx_measurement=python_streaming_combine_4 +--input_options=''{\\"num_records\\":5000000,\\"key_size\\":10,\\"value_size\\":90,\\"algorithm\\":\\"lcg\\"}'' +--num_workers=16 +--autoscaling_algorithm=NONE +--fanout=4 +--top_count=20 +--streaming +--experiments=use_runner_v2 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_Combine_Dataflow_Streaming_2GB_Fanout_8.txt b/.github/workflows/load-tests-job-configs/python_Combine_Dataflow_Streaming_2GB_Fanout_8.txt new file mode 100644 index 000000000000..46d68261342f --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_Combine_Dataflow_Streaming_2GB_Fanout_8.txt @@ -0,0 +1,30 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/smoketests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_streaming_combine_5 +--influx_measurement=python_streaming_combine_5 +--input_options=''{\\"num_records\\":2500000,\\"key_size\\":10,\\"value_size\\":90,\\"algorithm\\":\\"lcg\\"}'' +--num_workers=16 +--autoscaling_algorithm=NONE +--fanout=8 +--top_count=20 +--streaming +--experiments=use_runner_v2 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_GBK_reiterate_Dataflow_Batch_reiterate_4_times_10kB_values.txt b/.github/workflows/load-tests-job-configs/python_GBK_reiterate_Dataflow_Batch_reiterate_4_times_10kB_values.txt new file mode 100644 index 000000000000..fb5a3db9e6bb --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_GBK_reiterate_Dataflow_Batch_reiterate_4_times_10kB_values.txt @@ -0,0 +1,28 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_batch_gbk_6 +--influx_measurement=python_batch_gbk_6 +--input_options=''{\\"num_records\\":20000000,\\"key_size\\":10,\\"value_size\\":90,\\"num_hot_keys\\":200,\\"hot_key_fraction\\":1,\\"algorithm\\":\\"lcg\\"}'' +--iterations=4 +--fanout=1 +--num_workers=5 +--autoscaling_algorithm=NONE +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_GBK_reiterate_Dataflow_Batch_reiterate_4_times_2MB_values.txt b/.github/workflows/load-tests-job-configs/python_GBK_reiterate_Dataflow_Batch_reiterate_4_times_2MB_values.txt new file mode 100644 index 000000000000..b4b46682869a --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_GBK_reiterate_Dataflow_Batch_reiterate_4_times_2MB_values.txt @@ -0,0 +1,28 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_batch_gbk_7 +--influx_measurement=python_batch_gbk_7 +--input_options=''{\\"num_records\\":20000000,\\"key_size\\":10,\\"value_size\\":90,\\"num_hot_keys\\":10,\\"hot_key_fraction\\":1,\\"algorithm\\":\\"lcg\\"}'' +--iterations=4 +--fanout=1 +--num_workers=5 +--autoscaling_algorithm=NONE +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_GBK_reiterate_Dataflow_Streaming_reiterate_4_times_10kB_values.txt b/.github/workflows/load-tests-job-configs/python_GBK_reiterate_Dataflow_Streaming_reiterate_4_times_10kB_values.txt new file mode 100644 index 000000000000..6cb1e68aeafc --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_GBK_reiterate_Dataflow_Streaming_reiterate_4_times_10kB_values.txt @@ -0,0 +1,30 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_streaming_gbk_6 +--influx_measurement=python_streaming_gbk_6 +--input_options=''{\\"num_records\\":20000000,\\"key_size\\":10,\\"value_size\\":90,\\"num_hot_keys\\":200,\\"hot_key_fraction\\":1,\\"algorithm\\":\\"lcg\\"}'' +--iterations=4 +--fanout=1 +--num_workers=5 +--autoscaling_algorithm=NONE +--streaming +--experiments=use_runner_v2 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_GBK_reiterate_Dataflow_Streaming_reiterate_4_times_2MB_values.txt b/.github/workflows/load-tests-job-configs/python_GBK_reiterate_Dataflow_Streaming_reiterate_4_times_2MB_values.txt new file mode 100644 index 000000000000..712749090aec --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_GBK_reiterate_Dataflow_Streaming_reiterate_4_times_2MB_values.txt @@ -0,0 +1,30 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_streaming_gbk_7 +--influx_measurement=python_streaming_gbk_7 +--input_options=''{\\"num_records\\":20000000,\\"key_size\\":10,\\"value_size\\":90,\\"num_hot_keys\\":10,\\"hot_key_fraction\\":1,\\"algorithm\\":\\"lcg\\"}'' +--iterations=4 +--fanout=1 +--num_workers=5 +--autoscaling_algorithm=NONE +--streaming +--experiments=use_runner_v2 +--runner=DataflowRunner \ No newline at end of file From 155f8507d07f663f9e3c69fedc907e590ad6f7f4 Mon Sep 17 00:00:00 2001 From: Vitaly Terentyev Date: Fri, 13 Oct 2023 20:23:50 +0400 Subject: [PATCH 138/435] Add CloudML Benchmarks Dataflow workflow (#28941) * Add CloudML Benchmarks Dataflow workflow * Fix tag --- .../beam_CloudML_Benchmarks_Dataflow.yml | 93 +++++++++++++++++++ ..._CloudML_Benchmarks_Dataflow_arguments.txt | 23 +++++ 2 files changed, 116 insertions(+) create mode 100644 .github/workflows/beam_CloudML_Benchmarks_Dataflow.yml create mode 100644 .github/workflows/load-tests-job-configs/beam_CloudML_Benchmarks_Dataflow_arguments.txt diff --git a/.github/workflows/beam_CloudML_Benchmarks_Dataflow.yml b/.github/workflows/beam_CloudML_Benchmarks_Dataflow.yml new file mode 100644 index 000000000000..c34d9b5118e6 --- /dev/null +++ b/.github/workflows/beam_CloudML_Benchmarks_Dataflow.yml @@ -0,0 +1,93 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: CloudML Benchmarks Dataflow + +on: + issue_comment: + types: [created] + schedule: + - cron: '10 21 * * *' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_CloudML_Benchmarks_Dataflow: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run TFT Criteo Benchmarks' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 360 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_CloudML_Benchmarks_Dataflow"] + job_phrase: ["Run TFT Criteo Benchmarks"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup Python environment + uses: ./.github/actions/setup-environment-action + with: + python-version: | + 3.8 + 3.9 + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: python + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/beam_CloudML_Benchmarks_Dataflow_arguments.txt + # The env variables are created and populated in the test-arguments-action as "_test_arguments_" + - name: run TFT Criteo Benchmarks + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:test-suites:dataflow:tftTests + arguments: | + -PpythonVersion=3.9 \ + -Prunner=DataflowRunner \ + '-Popts=${{ env.beam_CloudML_Benchmarks_Dataflow_test_arguments_1 }}' \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/beam_CloudML_Benchmarks_Dataflow_arguments.txt b/.github/workflows/load-tests-job-configs/beam_CloudML_Benchmarks_Dataflow_arguments.txt new file mode 100644 index 000000000000..b1b45c4cc9e4 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/beam_CloudML_Benchmarks_Dataflow_arguments.txt @@ -0,0 +1,23 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--metrics_dataset=beam_cloudml +--publish_to_big_query=true +--region=us-central1 +--staging_location=gs://temp-storage-for-perf-tests/loadtests +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--runner=DataflowRunner +--requirements_file=apache_beam/testing/benchmarks/cloudml/requirements.txt \ No newline at end of file From f12f529882dc4fb30e136b582db4bfc049691c19 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Fri, 13 Oct 2023 09:36:40 -0700 Subject: [PATCH 139/435] cleanup --- .../io/gcp/pubsub/PubsubReadSchemaTransformProvider.java | 9 +++------ .../gcp/pubsub/PubsubWriteSchemaTransformProvider.java | 1 - .../pubsub/PubsubReadSchemaTransformProviderTest.java | 1 - 3 files changed, 3 insertions(+), 8 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformProvider.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformProvider.java index 9efc34d8b502..8cdd6aa9ab1b 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformProvider.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformProvider.java @@ -23,7 +23,6 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; -import java.util.Objects; import java.util.Set; import org.apache.beam.sdk.io.gcp.pubsub.PubsubTestClient.PubsubTestClientFactory; import org.apache.beam.sdk.metrics.Counter; @@ -104,13 +103,13 @@ public SchemaTransform from(PubsubReadSchemaTransformConfiguration configuration String format = configuration.getFormat() == null ? null : configuration.getFormat().toUpperCase(); - if (Objects.equals(format, "RAW")) { + if ("RAW".equals(format)) { payloadSchema = Schema.of(Schema.Field.of("payload", Schema.FieldType.BYTES)); payloadMapper = input -> Row.withSchema(payloadSchema).addValue(input).build(); - } else if (Objects.equals(format, "JSON")) { + } else if ("JSON".equals(format)) { payloadSchema = JsonUtils.beamSchemaFromJsonSchema(configuration.getSchema()); payloadMapper = JsonUtils.getJsonBytesToRowFunction(payloadSchema); - } else if (Objects.equals(format, "AVRO")) { + } else if ("AVRO".equals(format)) { payloadSchema = AvroUtils.toBeamSchema( new org.apache.avro.Schema.Parser().parse(configuration.getSchema())); @@ -212,8 +211,6 @@ public void process(@DoFn.Element PubsubMessage message, MultiOutputReceiver rec } if (attributes != null) { for (String attribute : attributes) { - System.out.println( - "attribute " + attribute + " " + message.getAttribute(attribute)); rowBuilder.addValue(message.getAttribute(attribute)); } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubWriteSchemaTransformProvider.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubWriteSchemaTransformProvider.java index e1c927930626..1d11eeaca6ca 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubWriteSchemaTransformProvider.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubWriteSchemaTransformProvider.java @@ -159,7 +159,6 @@ private static class PubsubWriteSchemaTransform extends SchemaTransform implemen "nullness" // TODO(https://github.com/apache/beam/issues/20497) }) public PCollectionRowTuple expand(PCollectionRowTuple input) { - @SuppressWarnings("nullness") String errorOutput = configuration.getErrorHandling() == null ? null diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformProviderTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformProviderTest.java index 826d5b462154..dd5a9abd5ac8 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformProviderTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformProviderTest.java @@ -364,7 +364,6 @@ private static PubsubClient.IncomingMessage incomingMessageOf( private static PubsubTestClient.PubsubTestClientFactory clientFactory( List messages) { - System.out.println("messages " + messages); return PubsubTestClient.createFactoryForPull( CLOCK, PubsubClient.subscriptionPathFromPath(SUBSCRIPTION), 60, messages); } From d636dd32aca169a8656996f70dfe57c1a2d72ff5 Mon Sep 17 00:00:00 2001 From: Rebecca Szper <98840847+rszper@users.noreply.github.com> Date: Fri, 13 Oct 2023 10:18:54 -0700 Subject: [PATCH 140/435] Add KeyedModelHandler content to the multi-model ML page (#28987) --- .../documentation/ml/multi-model-pipelines.md | 58 +++++++++++++++++++ .../sdks/python-machine-learning.md | 42 ++++++++------ 2 files changed, 82 insertions(+), 18 deletions(-) diff --git a/website/www/site/content/en/documentation/ml/multi-model-pipelines.md b/website/www/site/content/en/documentation/ml/multi-model-pipelines.md index 569a51b8db55..c42c8b8ae661 100644 --- a/website/www/site/content/en/documentation/ml/multi-model-pipelines.md +++ b/website/www/site/content/en/documentation/ml/multi-model-pipelines.md @@ -95,3 +95,61 @@ captions. The solution consists of two open-source models: 2. **A caption ranking model ([CLIP](https://github.com/openai/CLIP))** that uses the image and candidate captions to rank the captions in the order in which they best describe the image. +## Use multiple differently-trained models + +You can use a `KeyedModelHandler` to load several different models into the `RunInference` transform. +Use the associated key to determine which model to use with which data. +The following example loads a model by using `config1`. That model is used for inference for all examples associated +with `key1`. It loads a second model by using `config2`. That model is used for all examples associated with `key2` and `key3`. + +``` +from apache_beam.ml.inference.base import KeyedModelHandler +keyed_model_handler = KeyedModelHandler([ + KeyModelMapping(['key1'], PytorchModelHandlerTensor()), + KeyModelMapping(['key2', 'key3'], PytorchModelHandlerTensor()) +]) +with pipeline as p: + data = p | beam.Create([ + ('key1', torch.tensor([[1,2,3],[4,5,6],...])), + ('key2', torch.tensor([[1,2,3],[4,5,6],...])), + ('key3', torch.tensor([[1,2,3],[4,5,6],...])), + ]) + predictions = data | RunInference(keyed_model_handler) +``` + +For a more detailed example, see the notebook +[Run ML inference with multiple differently-trained models](https://colab.sandbox.google.com/github/apache/beam/blob/master/examples/notebooks/beam-ml/per_key_models.ipynb). + +Loading multiple models at the same times increases the risk of out of memory errors (OOMs). By default, `KeyedModelHandler` doesn't +limit the number of models loaded into memory at the same time. If the models don't all fit into memory, +your pipeline might fail with an out of memory error. To avoid this issue, use the `max_models_per_worker_hint` parameter +to set the maximum number of models that can be loaded into memory at the same time. + +The following example loads at most two models per SDK worker process at a time. It unloads models that aren't +currently in use. + +``` +mhs = [ + KeyModelMapping(['key1'], PytorchModelHandlerTensor()), + KeyModelMapping(['key2', 'key3'], PytorchModelHandlerTensor()), + KeyModelMapping(['key4'], PytorchModelHandlerTensor()), + KeyModelMapping(['key5', 'key6', 'key7'], PytorchModelHandlerTensor()), +] +keyed_model_handler = KeyedModelHandler(mhs, max_models_per_worker_hint=2) +``` + +Runners that have multiple SDK worker processes on a given machine load at most +`max_models_per_worker_hint*` models onto the machine. + +Leave enough space for the models and any additional memory needs from other transforms. +Because the memory might not be released immediately after a model is offloaded, +leaving an additional buffer is recommended. + +**Note**: Having many models but a small `max_models_per_worker_hint` can cause _memory thrashing_, where +a large amount of execution time is used to swap models in and out of memory. To reduce the likelihood and impact +of memory thrashing, if you're using a distributed runner, insert a +[`GroupByKey`](https://beam.apache.org/documentation/transforms/python/aggregation/groupbykey/) transform before your +inference step. The `GroupByKey` transform reduces thrashing by ensuring that elements with the same key and model are +collocated on the same worker. + +For more information, see [`KeyedModelHander`](https://beam.apache.org/releases/pydoc/current/apache_beam.ml.inference.base.html#apache_beam.ml.inference.base.KeyedModelHandler). diff --git a/website/www/site/content/en/documentation/sdks/python-machine-learning.md b/website/www/site/content/en/documentation/sdks/python-machine-learning.md index 0076fa370b0f..a700806f14c6 100644 --- a/website/www/site/content/en/documentation/sdks/python-machine-learning.md +++ b/website/www/site/content/en/documentation/sdks/python-machine-learning.md @@ -197,9 +197,9 @@ For more information on resource hints, see [Resource hints](/documentation/runt This section suggests patterns and best practices that you can use to make your inference pipelines simpler, more robust, and more efficient. -### Use a keyed ModelHandler +### Use a keyed ModelHandler object -If a key is attached to the examples, wrap the `KeyedModelHandler` around the `ModelHandler` object: +If a key is attached to the examples, wrap `KeyedModelHandler` around the `ModelHandler` object: ``` from apache_beam.ml.inference.base import KeyedModelHandler @@ -213,9 +213,11 @@ with pipeline as p: predictions = data | RunInference(keyed_model_handler) ``` -If you are unsure if your data is keyed, you can also use `MaybeKeyedModelHandler`. +If you are unsure if your data is keyed, you can use `MaybeKeyedModelHandler`. -You can also use a `KeyedModelHandler` to load several different models based on their associated key: +You can also use a `KeyedModelHandler` to load several different models based on their associated key. +The following example loads a model by using `config1`. That model is used for inference for all examples associated +with `key1`. It loads a second model by using `config2`. That model is used for all examples associated with `key2` and `key3`. ``` from apache_beam.ml.inference.base import KeyedModelHandler @@ -232,13 +234,16 @@ with pipeline as p: predictions = data | RunInference(keyed_model_handler) ``` -The previous example loads a model by using `config1`. That model is then used for inference for all examples associated -with `key1`. It also loads a model by using `config2`. That model is used for all examples associated with `key2` and `key3`. +For a more detailed example, see the notebook +[Run ML inference with multiple differently-trained models](https://colab.sandbox.google.com/github/apache/beam/blob/master/examples/notebooks/beam-ml/per_key_models.ipynb). -Loading multiple models at the same times increases the risk of out of memory (OOM) errors. By default, `KeyedModelHandler` doesn't +Loading multiple models at the same times increases the risk of out of memory errors (OOMs). By default, `KeyedModelHandler` doesn't limit the number of models loaded into memory at the same time. If the models don't all fit into memory, -your pipeline will likely fail with an out of memory error. To avoid this issue, provide a hint about the -maximum number of models that can be loaded at the same time. +your pipeline might fail with an out of memory error. To avoid this issue, use the `max_models_per_worker_hint` parameter +to set the maximum number of models that can be loaded into memory at the same time. + +The following example loads at most two models per SDK worker process at a time. It unloads models that aren't +currently in use. ``` mhs = [ @@ -250,17 +255,18 @@ mhs = [ keyed_model_handler = KeyedModelHandler(mhs, max_models_per_worker_hint=2) ``` -The previous example loads at most two models per SDK worker process at any given time. It unloads models that aren't -currently being used. Runners that have multiple SDK worker processes on a given machine load at most -`max_models_per_worker_hint*` models onto the machine. Leave enough space for the models -and any additional memory needs from other transforms. Because there might be a delay between when a model is offloaded and when the -memory is released, it is recommended that you leave additional buffer. +Runners that have multiple SDK worker processes on a given machine load at most +`max_models_per_worker_hint*` models onto the machine. + +Leave enough space for the models and any additional memory needs from other transforms. +Because the memory might not be released immediately after a model is offloaded, +leaving an additional buffer is recommended. -**Note**: Having many models but a small `max_models_per_worker_hint` can lead to _memory thrashing_, where -a large amount of execution time is wasted swapping models in and out of memory. To reduce the likelihood and impact +**Note**: Having many models but a small `max_models_per_worker_hint` can cause _memory thrashing_, where +a large amount of execution time is used to swap models in and out of memory. To reduce the likelihood and impact of memory thrashing, if you're using a distributed runner, insert a -[GroupByKey](https://beam.apache.org/documentation/transforms/python/aggregation/groupbykey/) transform before your -inference step. This step reduces thrashing by ensuring that elements with the same key and model are +[`GroupByKey`](https://beam.apache.org/documentation/transforms/python/aggregation/groupbykey/) transform before your +inference step. The `GroupByKey` transform reduces thrashing by ensuring that elements with the same key and model are collocated on the same worker. For more information, see [`KeyedModelHander`](https://beam.apache.org/releases/pydoc/current/apache_beam.ml.inference.base.html#apache_beam.ml.inference.base.KeyedModelHandler). From 250483c5f1fbfbb614b70bf02b43421be74c1887 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 13 Oct 2023 13:47:42 -0400 Subject: [PATCH 141/435] Bump org.checkerframework:checkerframework-gradle-plugin (#28979) Bumps org.checkerframework:checkerframework-gradle-plugin from 0.6.33 to 0.6.34. --- updated-dependencies: - dependency-name: org.checkerframework:checkerframework-gradle-plugin dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- buildSrc/build.gradle.kts | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/buildSrc/build.gradle.kts b/buildSrc/build.gradle.kts index 968829caeb8b..d99a1003c396 100644 --- a/buildSrc/build.gradle.kts +++ b/buildSrc/build.gradle.kts @@ -56,7 +56,7 @@ dependencies { runtimeOnly("com.avast.gradle:gradle-docker-compose-plugin:0.16.12") // Enable docker compose tasks runtimeOnly("ca.cutterslade.gradle:gradle-dependency-analyze:1.8.3") // Enable dep analysis runtimeOnly("gradle.plugin.net.ossindex:ossindex-gradle-plugin:0.4.11") // Enable dep vulnerability analysis - runtimeOnly("org.checkerframework:checkerframework-gradle-plugin:0.6.33") // Enable enhanced static checking plugin + runtimeOnly("org.checkerframework:checkerframework-gradle-plugin:0.6.34") // Enable enhanced static checking plugin } // Because buildSrc is built and tested automatically _before_ gradle From 47d0fd566f86aaad35d26709c52ee555381823a4 Mon Sep 17 00:00:00 2001 From: dpcollins-google <40498610+dpcollins-google@users.noreply.github.com> Date: Fri, 13 Oct 2023 14:20:10 -0400 Subject: [PATCH 142/435] Set MIN_PER_PARTITION_MEMORY to 10 MiB instead of 4 MiB to match the change in the backend limit. (#28964) --- .../beam/sdk/io/gcp/pubsublite/internal/SubscribeTransform.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/SubscribeTransform.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/SubscribeTransform.java index f3ffbb13c249..882294de1771 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/SubscribeTransform.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/SubscribeTransform.java @@ -56,7 +56,7 @@ public class SubscribeTransform extends PTransform Date: Fri, 13 Oct 2023 16:34:12 -0400 Subject: [PATCH 143/435] reorder args to avoid newline issues (#28994) * reorder args to avoid newline issues * Inline default args --- .../gradle-command-self-hosted-action/action.yml | 11 +++-------- 1 file changed, 3 insertions(+), 8 deletions(-) diff --git a/.github/actions/gradle-command-self-hosted-action/action.yml b/.github/actions/gradle-command-self-hosted-action/action.yml index cb793714aa91..906b35169d9d 100644 --- a/.github/actions/gradle-command-self-hosted-action/action.yml +++ b/.github/actions/gradle-command-self-hosted-action/action.yml @@ -24,12 +24,6 @@ inputs: required: false description: 'Gradle options' default: '' - default-arguments: - required: false - description: 'Default gradle switches' # Copied from CommonJobProperties.groovy' - default: | - --continue -Dorg.gradle.jvmargs=-Xms2g -Dorg.gradle.jvmargs=-Xmx6g \ - -Dorg.gradle.vfs.watch=false -Pdocker-pull-licenses max-workers: required: false description: 'Max number of workers' @@ -47,5 +41,6 @@ runs: if [ -f ~/.m2/settings.xml ]; then rm ~/.m2/settings.xml fi - ./gradlew ${{ inputs.gradle-command }} --max-workers=${{ inputs.max-workers }} ${{ inputs.arguments }} \ - ${{ inputs.default-arguments }} \ No newline at end of file + ./gradlew ${{ inputs.gradle-command }} --max-workers=${{ inputs.max-workers }} --continue \ + -Dorg.gradle.jvmargs=-Xms2g -Dorg.gradle.jvmargs=-Xmx6g -Dorg.gradle.vfs.watch=false -Pdocker-pull-licenses \ + ${{ inputs.arguments }} From 8de14655f783f2dd5754b87ee9f9dcd96e552136 Mon Sep 17 00:00:00 2001 From: Damon Date: Fri, 13 Oct 2023 20:51:03 -0700 Subject: [PATCH 144/435] [RRIO] Stub the RequestResponseIO transform (#28950) * Move test classes into correct package * Stub RequestResponseIO and related error element * Edit per PR comments --- sdks/java/io/rrio/build.gradle | 7 +- .../beam/io/requestresponseio/ApiIOError.java | 62 +++++++ .../requestresponseio/RequestResponseIO.java | 160 ++++++++++++++++++ .../CallerTest.java | 6 +- .../SetupTeardownTest.java | 6 +- 5 files changed, 227 insertions(+), 14 deletions(-) create mode 100644 sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/ApiIOError.java create mode 100644 sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/RequestResponseIO.java rename sdks/java/io/rrio/src/test/java/org/apache/beam/io/{rrio => requestresponseio}/CallerTest.java (93%) rename sdks/java/io/rrio/src/test/java/org/apache/beam/io/{rrio => requestresponseio}/SetupTeardownTest.java (93%) diff --git a/sdks/java/io/rrio/build.gradle b/sdks/java/io/rrio/build.gradle index 32fbd9d22e38..d65df370e0ca 100644 --- a/sdks/java/io/rrio/build.gradle +++ b/sdks/java/io/rrio/build.gradle @@ -25,10 +25,9 @@ description = "Apache Beam :: SDKS :: Java :: IO :: RequestResponseIO (RRIO)" ext.summary = "Support to read from and write to Web APIs" dependencies { - // TODO(damondouglas): revert to implementation after project is more fully developed - permitUnusedDeclared project(path: ":sdks:java:core", configuration: "shadow") - permitUnusedDeclared library.java.joda_time - permitUnusedDeclared library.java.vendored_guava_32_1_2_jre + implementation project(path: ":sdks:java:core", configuration: "shadow") + implementation library.java.joda_time + implementation library.java.vendored_guava_32_1_2_jre testImplementation project(path: ":sdks:java:core", configuration: "shadowTest") testImplementation library.java.junit diff --git a/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/ApiIOError.java b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/ApiIOError.java new file mode 100644 index 000000000000..b7c5524e8237 --- /dev/null +++ b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/ApiIOError.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.io.requestresponseio; + +import com.google.auto.value.AutoValue; +import org.apache.beam.sdk.schemas.AutoValueSchema; +import org.apache.beam.sdk.schemas.annotations.DefaultSchema; +import org.apache.beam.sdk.schemas.annotations.SchemaCaseFormat; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.CaseFormat; +import org.joda.time.Instant; + +/** {@link ApiIOError} is a data class for storing details about an error. */ +@SchemaCaseFormat(CaseFormat.LOWER_UNDERSCORE) +@DefaultSchema(AutoValueSchema.class) +@AutoValue +public abstract class ApiIOError { + + static Builder builder() { + return new AutoValue_ApiIOError.Builder(); + } + + /** The encoded UTF-8 string representation of the related processed element. */ + public abstract String getEncodedElementAsUtfString(); + + /** The observed timestamp of the error. */ + public abstract Instant getObservedTimestamp(); + + /** The {@link Exception} message. */ + public abstract String getMessage(); + + /** The {@link Exception} stack trace. */ + public abstract String getStackTrace(); + + @AutoValue.Builder + abstract static class Builder { + + public abstract Builder setEncodedElementAsUtfString(String value); + + public abstract Builder setObservedTimestamp(Instant value); + + public abstract Builder setMessage(String value); + + public abstract Builder setStackTrace(String value); + + abstract ApiIOError build(); + } +} diff --git a/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/RequestResponseIO.java b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/RequestResponseIO.java new file mode 100644 index 000000000000..2ff0d50f68d5 --- /dev/null +++ b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/RequestResponseIO.java @@ -0,0 +1,160 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.io.requestresponseio; + +import com.google.auto.value.AutoValue; +import java.util.Map; +import org.apache.beam.io.requestresponseio.RequestResponseIO.Result; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionTuple; +import org.apache.beam.sdk.values.PInput; +import org.apache.beam.sdk.values.POutput; +import org.apache.beam.sdk.values.PValue; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; + +/** + * {@link PTransform} for reading from and writing to Web APIs. + * + *

{@link RequestResponseIO} is recommended for interacting with external systems that offer RPCs + * that execute relatively quickly and do not offer advance features to make RPC execution + * efficient. + * + *

For systems that offer features for more efficient reading, for example, tracking progress of + * RPCs, support for splitting RPCs (deduct two or more RPCs which when combined return the same + * result), consider using the Apache Beam's `Splittable DoFn` interface instead. + * + *

Basic Usage

+ * + * {@link RequestResponseIO} minimally requires implementing the {@link Caller} interface: + * + *
{@code class MyCaller implements Caller {
+ *    public SomeResponse call(SomeRequest request) throws UserCodeExecutionException {
+ *      // calls the API submitting SomeRequest payload and returning SomeResponse
+ *    }
+ * }}
+ * + *

Then provide {@link RequestResponseIO}'s {@link #create} method your {@link Caller} + * implementation. + * + *

{@code  PCollection requests = ...
+ *  Result result = requests.apply(RequestResponseIO.create(new MyCaller()));
+ *  result.getResponses().apply( ... );
+ *  result.getFailures().apply( ... );
+ * }
+ */ +public class RequestResponseIO + extends PTransform, Result> { + + private static final TupleTag FAILURE_TAG = new TupleTag() {}; + + // TODO(damondouglas): remove when utilized. + @SuppressWarnings({"unused"}) + private final Configuration configuration; + + private RequestResponseIO(Configuration configuration) { + this.configuration = configuration; + } + + public static RequestResponseIO of( + Caller caller) { + return new RequestResponseIO<>( + Configuration.builder().setCaller(caller).build()); + } + + /** Configuration details for {@link RequestResponseIO}. */ + @AutoValue + abstract static class Configuration { + + static Builder builder() { + return new AutoValue_RequestResponseIO_Configuration.Builder<>(); + } + + /** + * The {@link Caller} that interfaces user custom code to process a {@link RequestT} into a + * {@link ResponseT}. + */ + abstract Caller getCaller(); + + abstract Builder toBuilder(); + + @AutoValue.Builder + abstract static class Builder { + + abstract Builder setCaller(Caller value); + + abstract Configuration build(); + } + } + + @Override + public Result expand(PCollection input) { + // TODO(damondouglas; https://github.com/apache/beam/issues?q=is%3Aissue+is%3Aopen+%5BRRIO%5D): + // expand pipeline as more dependencies develop. + return Result.of(new TupleTag() {}, PCollectionTuple.empty(input.getPipeline())); + } + + /** + * The {@link Result} of processing request {@link PCollection} into response {@link PCollection} + * using custom {@link Caller} code. + */ + public static class Result implements POutput { + + static Result of(TupleTag responseTag, PCollectionTuple pct) { + return new Result<>(responseTag, pct); + } + + private final Pipeline pipeline; + private final TupleTag responseTag; + private final PCollection responses; + private final PCollection failures; + + private Result(TupleTag responseTag, PCollectionTuple pct) { + this.pipeline = pct.getPipeline(); + this.responseTag = responseTag; + this.responses = pct.get(responseTag); + this.failures = pct.get(FAILURE_TAG); + } + + public PCollection getResponses() { + return responses; + } + + public PCollection getFailures() { + return failures; + } + + @Override + public Pipeline getPipeline() { + return this.pipeline; + } + + @Override + public Map, PValue> expand() { + return ImmutableMap.of( + responseTag, responses, + FAILURE_TAG, failures); + } + + @Override + public void finishSpecifyingOutput( + String transformName, PInput input, PTransform transform) {} + } +} diff --git a/sdks/java/io/rrio/src/test/java/org/apache/beam/io/rrio/CallerTest.java b/sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponseio/CallerTest.java similarity index 93% rename from sdks/java/io/rrio/src/test/java/org/apache/beam/io/rrio/CallerTest.java rename to sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponseio/CallerTest.java index 5258573f4283..0ba2d93c5411 100644 --- a/sdks/java/io/rrio/src/test/java/org/apache/beam/io/rrio/CallerTest.java +++ b/sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponseio/CallerTest.java @@ -15,16 +15,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.io.rrio; +package org.apache.beam.io.requestresponseio; import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThrows; -import org.apache.beam.io.requestresponseio.Caller; -import org.apache.beam.io.requestresponseio.UserCodeExecutionException; -import org.apache.beam.io.requestresponseio.UserCodeQuotaException; -import org.apache.beam.io.requestresponseio.UserCodeTimeoutException; import org.apache.beam.sdk.Pipeline.PipelineExecutionException; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarIntCoder; diff --git a/sdks/java/io/rrio/src/test/java/org/apache/beam/io/rrio/SetupTeardownTest.java b/sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponseio/SetupTeardownTest.java similarity index 93% rename from sdks/java/io/rrio/src/test/java/org/apache/beam/io/rrio/SetupTeardownTest.java rename to sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponseio/SetupTeardownTest.java index a8c5c45ede5c..9ef2f88a29c5 100644 --- a/sdks/java/io/rrio/src/test/java/org/apache/beam/io/rrio/SetupTeardownTest.java +++ b/sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponseio/SetupTeardownTest.java @@ -15,15 +15,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.io.rrio; +package org.apache.beam.io.requestresponseio; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThrows; -import org.apache.beam.io.requestresponseio.SetupTeardown; -import org.apache.beam.io.requestresponseio.UserCodeExecutionException; -import org.apache.beam.io.requestresponseio.UserCodeQuotaException; -import org.apache.beam.io.requestresponseio.UserCodeTimeoutException; import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Create; From 5e06f54371b38391ec1f62a211f6e827abd7ea21 Mon Sep 17 00:00:00 2001 From: liferoad Date: Sat, 14 Oct 2023 10:31:11 -0400 Subject: [PATCH 145/435] Update .htaccess to redirect https://beam.apache.org/contribute/release-guide/ Do not break https://beam.apache.org/contribute/release-guide/ --- website/www/site/static/.htaccess | 2 ++ 1 file changed, 2 insertions(+) diff --git a/website/www/site/static/.htaccess b/website/www/site/static/.htaccess index a2ef056a262c..ff5e56fc37dd 100644 --- a/website/www/site/static/.htaccess +++ b/website/www/site/static/.htaccess @@ -22,3 +22,5 @@ RewriteRule ^(.*)$ https://beam.apache.org/$1 [L,R=301] RedirectMatch permanent "/documentation/sdks/(javadoc|pydoc)(.*)" "https://beam.apache.org/releases/$1$2" RedirectMatch "/contribute/design-documents" "https://cwiki.apache.org/confluence/display/BEAM/Design+Documents" + +RedirectMatch "/contribute/release-guide" "https://github.com/apache/beam/blob/master/contributor-docs/release-guide.md" From 6b32a3fb7d97ec932bde8c0d8c8b812ed2e940da Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 16 Oct 2023 09:10:28 -0400 Subject: [PATCH 146/435] Bump github.com/aws/aws-sdk-go-v2/feature/s3/manager in /sdks (#29002) Bumps [github.com/aws/aws-sdk-go-v2/feature/s3/manager](https://github.com/aws/aws-sdk-go-v2) from 1.11.89 to 1.11.90. - [Release notes](https://github.com/aws/aws-sdk-go-v2/releases) - [Commits](https://github.com/aws/aws-sdk-go-v2/compare/feature/s3/manager/v1.11.89...feature/s3/manager/v1.11.90) --- updated-dependencies: - dependency-name: github.com/aws/aws-sdk-go-v2/feature/s3/manager dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 30 +++++++++++++------------- sdks/go.sum | 61 ++++++++++++++++++++++++++--------------------------- 2 files changed, 45 insertions(+), 46 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 5b0d0c8af6ce..2dffed89df2e 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -31,10 +31,10 @@ require ( cloud.google.com/go/spanner v1.50.0 cloud.google.com/go/storage v1.33.0 github.com/aws/aws-sdk-go-v2 v1.21.2 - github.com/aws/aws-sdk-go-v2/config v1.18.44 - github.com/aws/aws-sdk-go-v2/credentials v1.13.42 - github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.89 - github.com/aws/aws-sdk-go-v2/service/s3 v1.40.1 + github.com/aws/aws-sdk-go-v2/config v1.18.45 + github.com/aws/aws-sdk-go-v2/credentials v1.13.43 + github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.90 + github.com/aws/aws-sdk-go-v2/service/s3 v1.40.2 github.com/aws/smithy-go v1.15.0 github.com/docker/go-connections v0.4.0 github.com/dustin/go-humanize v1.0.1 @@ -98,18 +98,18 @@ require ( github.com/apache/thrift v0.16.0 // indirect github.com/aws/aws-sdk-go v1.34.0 // indirect github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.4.14 // indirect - github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.13.12 // indirect - github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.42 // indirect - github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.36 // indirect - github.com/aws/aws-sdk-go-v2/internal/ini v1.3.44 // indirect - github.com/aws/aws-sdk-go-v2/internal/v4a v1.1.5 // indirect + github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.13.13 // indirect + github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.43 // indirect + github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.37 // indirect + github.com/aws/aws-sdk-go-v2/internal/ini v1.3.45 // indirect + github.com/aws/aws-sdk-go-v2/internal/v4a v1.1.6 // indirect github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.9.15 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.1.37 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.36 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.15.5 // indirect - github.com/aws/aws-sdk-go-v2/service/sso v1.15.1 // indirect - github.com/aws/aws-sdk-go-v2/service/ssooidc v1.17.2 // indirect - github.com/aws/aws-sdk-go-v2/service/sts v1.23.1 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.1.38 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.37 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.15.6 // indirect + github.com/aws/aws-sdk-go-v2/service/sso v1.15.2 // indirect + github.com/aws/aws-sdk-go-v2/service/ssooidc v1.17.3 // indirect + github.com/aws/aws-sdk-go-v2/service/sts v1.23.2 // indirect github.com/cenkalti/backoff/v4 v4.2.1 // indirect github.com/census-instrumentation/opencensus-proto v0.4.1 // indirect github.com/cespare/xxhash/v2 v2.2.0 // indirect diff --git a/sdks/go.sum b/sdks/go.sum index fe5e8d348bfe..59115dcd9646 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -81,54 +81,53 @@ github.com/aws/aws-sdk-go v1.30.19/go.mod h1:5zCpMtNQVjRREroY7sYe8lOMRSxkhG6MZve github.com/aws/aws-sdk-go v1.34.0 h1:brux2dRrlwCF5JhTL7MUT3WUwo9zfDHZZp3+g3Mvlmo= github.com/aws/aws-sdk-go v1.34.0/go.mod h1:5zCpMtNQVjRREroY7sYe8lOMRSxkhG6MZveU8YkpAk0= github.com/aws/aws-sdk-go-v2 v1.7.1/go.mod h1:L5LuPC1ZgDr2xQS7AmIec/Jlc7O/Y1u2KxJyNVab250= -github.com/aws/aws-sdk-go-v2 v1.21.1/go.mod h1:ErQhvNuEMhJjweavOYhxVkn2RUx7kQXVATHrjKtxIpM= github.com/aws/aws-sdk-go-v2 v1.21.2 h1:+LXZ0sgo8quN9UOKXXzAWRT3FWd4NxeXWOZom9pE7GA= github.com/aws/aws-sdk-go-v2 v1.21.2/go.mod h1:ErQhvNuEMhJjweavOYhxVkn2RUx7kQXVATHrjKtxIpM= github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.4.14 h1:Sc82v7tDQ/vdU1WtuSyzZ1I7y/68j//HJ6uozND1IDs= github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.4.14/go.mod h1:9NCTOURS8OpxvoAVHq79LK81/zC78hfRWFn+aL0SPcY= github.com/aws/aws-sdk-go-v2/config v1.5.0/go.mod h1:RWlPOAW3E3tbtNAqTwvSW54Of/yP3oiZXMI0xfUdjyA= -github.com/aws/aws-sdk-go-v2/config v1.18.44 h1:U10NQ3OxiY0dGGozmVIENIDnCT0W432PWxk2VO8wGnY= -github.com/aws/aws-sdk-go-v2/config v1.18.44/go.mod h1:pHxnQBldd0heEdJmolLBk78D1Bf69YnKLY3LOpFImlU= +github.com/aws/aws-sdk-go-v2/config v1.18.45 h1:Aka9bI7n8ysuwPeFdm77nfbyHCAKQ3z9ghB3S/38zes= +github.com/aws/aws-sdk-go-v2/config v1.18.45/go.mod h1:ZwDUgFnQgsazQTnWfeLWk5GjeqTQTL8lMkoE1UXzxdE= github.com/aws/aws-sdk-go-v2/credentials v1.3.1/go.mod h1:r0n73xwsIVagq8RsxmZbGSRQFj9As3je72C2WzUIToc= -github.com/aws/aws-sdk-go-v2/credentials v1.13.42 h1:KMkjpZqcMOwtRHChVlHdNxTUUAC6NC/b58mRZDIdcRg= -github.com/aws/aws-sdk-go-v2/credentials v1.13.42/go.mod h1:7ltKclhvEB8305sBhrpls24HGxORl6qgnQqSJ314Uw8= +github.com/aws/aws-sdk-go-v2/credentials v1.13.43 h1:LU8vo40zBlo3R7bAvBVy/ku4nxGEyZe9N8MqAeFTzF8= +github.com/aws/aws-sdk-go-v2/credentials v1.13.43/go.mod h1:zWJBz1Yf1ZtX5NGax9ZdNjhhI4rgjfgsyk6vTY1yfVg= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.3.0/go.mod h1:2LAuqPx1I6jNfaGDucWfA2zqQCYCOMCDHiCOciALyNw= -github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.13.12 h1:3j5lrl9kVQrJ1BU4O0z7MQ8sa+UXdiLuo4j0V+odNI8= -github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.13.12/go.mod h1:JbFpcHDBdsex1zpIKuVRorZSQiZEyc3MykNCcjgz174= +github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.13.13 h1:PIktER+hwIG286DqXyvVENjgLTAwGgoeriLDD5C+YlQ= +github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.13.13/go.mod h1:f/Ib/qYjhV2/qdsf79H3QP/eRE4AkVyEf6sk7XfZ1tg= github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.3.2/go.mod h1:qaqQiHSrOUVOfKe6fhgQ6UzhxjwqVW8aHNegd6Ws4w4= -github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.89 h1:XPqSyw8SBSLMRrF9Oip6tQpivXWJLMn8sdRoAsUCQQA= -github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.89/go.mod h1:OkYwM7gYm9HieL6emYtkg7Pb7Jd8FFM5Pl5uAZ1h2jo= -github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.42 h1:817VqVe6wvwE46xXy6YF5RywvjOX6U2zRQQ6IbQFK0s= -github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.42/go.mod h1:oDfgXoBBmj+kXnqxDDnIDnC56QBosglKp8ftRCTxR+0= -github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.36 h1:7ZApaXzWbo8slc+W5TynuUlB4z66g44h7uqa3/d/BsY= -github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.36/go.mod h1:rwr4WnmFi3RJO0M4dxbJtgi9BPLMpVBMX1nUte5ha9U= +github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.90 h1:mtJRt80k1oGw7QQPluAx8AZ6u16MyCA2di/lMhagZ7I= +github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.90/go.mod h1:lYwZTkeMQWPvNU+u7oYArdNhQ8EKiSGU76jVv0w2GH4= +github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.43 h1:nFBQlGtkbPzp/NjZLuFxRqmT91rLJkgvsEQs68h962Y= +github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.43/go.mod h1:auo+PiyLl0n1l8A0e8RIeR8tOzYPfZZH/JNlrJ8igTQ= +github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.37 h1:JRVhO25+r3ar2mKGP7E0LDl8K9/G36gjlqca5iQbaqc= +github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.37/go.mod h1:Qe+2KtKml+FEsQF/DHmDV+xjtche/hwoF75EG4UlHW8= github.com/aws/aws-sdk-go-v2/internal/ini v1.1.1/go.mod h1:Zy8smImhTdOETZqfyn01iNOe0CNggVbPjCajyaz6Gvg= -github.com/aws/aws-sdk-go-v2/internal/ini v1.3.44 h1:quOJOqlbSfeJTboXLjYXM1M9T52LBXqLoTPlmsKLpBo= -github.com/aws/aws-sdk-go-v2/internal/ini v1.3.44/go.mod h1:LNy+P1+1LiRcCsVYr/4zG5n8zWFL0xsvZkOybjbftm8= -github.com/aws/aws-sdk-go-v2/internal/v4a v1.1.5 h1:8JG9ny0BqBDzmtIzbpaN+eke152ZNsYKApFJ/q29Hxo= -github.com/aws/aws-sdk-go-v2/internal/v4a v1.1.5/go.mod h1:kEDHQApP/ukMO9natNftgUN3NaTsMxK6jb2jjpSMX7Y= +github.com/aws/aws-sdk-go-v2/internal/ini v1.3.45 h1:hze8YsjSh8Wl1rYa1CJpRmXP21BvOBuc76YhW0HsuQ4= +github.com/aws/aws-sdk-go-v2/internal/ini v1.3.45/go.mod h1:lD5M20o09/LCuQ2mE62Mb/iSdSlCNuj6H5ci7tW7OsE= +github.com/aws/aws-sdk-go-v2/internal/v4a v1.1.6 h1:wmGLw2i8ZTlHLw7a9ULGfQbuccw8uIiNr6sol5bFzc8= +github.com/aws/aws-sdk-go-v2/internal/v4a v1.1.6/go.mod h1:Q0Hq2X/NuL7z8b1Dww8rmOFl+jzusKEcyvkKspwdpyc= github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.2.1/go.mod h1:v33JQ57i2nekYTA70Mb+O18KeH4KqhdqxTJZNK1zdRE= github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.9.15 h1:7R8uRYyXzdD71KWVCL78lJZltah6VVznXBazvKjfH58= github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.9.15/go.mod h1:26SQUPcTNgV1Tapwdt4a1rOsYRsnBsJHLMPoxK2b0d8= -github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.1.37 h1:Mx1zJlYbiUQANWT40koevLvxawGFolmkaP4m+LuyG7M= -github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.1.37/go.mod h1:PjKIAMFthKPgG/B8bbRpo3F8jfr2q2L+w3u78jJ12a0= +github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.1.38 h1:skaFGzv+3kA+v2BPKhuekeb1Hbb105+44r8ASC+q5SE= +github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.1.38/go.mod h1:epIZoRSSbRIwLPJU5F+OldHhwZPBdpDeQkRdCeY3+00= github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.2.1/go.mod h1:zceowr5Z1Nh2WVP8bf/3ikB41IZW59E4yIYbg+pC6mw= -github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.36 h1:YXlm7LxwNlauqb2OrinWlcvtsflTzP8GaMvYfQBhoT4= -github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.36/go.mod h1:ou9ffqJ9hKOVZmjlC6kQ6oROAyG1M4yBKzR+9BKbDwk= +github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.37 h1:WWZA/I2K4ptBS1kg0kV1JbBtG/umed0vwHRrmcr9z7k= +github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.37/go.mod h1:vBmDnwWXWxNPFRMmG2m/3MKOe+xEcMDo1tanpaWCcck= github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.5.1/go.mod h1:6EQZIwNNvHpq/2/QSJnp4+ECvqIy55w95Ofs0ze+nGQ= -github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.15.5 h1:sAAz28SeA7YZl8Yaphjs9tlLsflhdniQPjf3X2cqr4s= -github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.15.5/go.mod h1:HC7gNz3VH0p+RvLKK+HqNQv/gHy+1Os3ko/F41s3+aw= +github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.15.6 h1:9ulSU5ClouoPIYhDQdg9tpl83d5Yb91PXTKK+17q+ow= +github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.15.6/go.mod h1:lnc2taBsR9nTlz9meD+lhFZZ9EWY712QHrRflWpTcOA= github.com/aws/aws-sdk-go-v2/service/s3 v1.11.1/go.mod h1:XLAGFrEjbvMCLvAtWLLP32yTv8GpBquCApZEycDLunI= -github.com/aws/aws-sdk-go-v2/service/s3 v1.40.1 h1:FqIaVPbs2W8U3fszl2PCL1IDKeRdM7TssjWamL6b2mg= -github.com/aws/aws-sdk-go-v2/service/s3 v1.40.1/go.mod h1:X0e0NCAx4GjOrKro7s9QYy+YEIFhgCkt6gYKVKhZB5Y= +github.com/aws/aws-sdk-go-v2/service/s3 v1.40.2 h1:Ll5/YVCOzRB+gxPqs2uD0R7/MyATC0w85626glSKmp4= +github.com/aws/aws-sdk-go-v2/service/s3 v1.40.2/go.mod h1:Zjfqt7KhQK+PO1bbOsFNzKgaq7TcxzmEoDWN8lM0qzQ= github.com/aws/aws-sdk-go-v2/service/sso v1.3.1/go.mod h1:J3A3RGUvuCZjvSuZEcOpHDnzZP/sKbhDWV2T1EOzFIM= -github.com/aws/aws-sdk-go-v2/service/sso v1.15.1 h1:ZN3bxw9OYC5D6umLw6f57rNJfGfhg1DIAAcKpzyUTOE= -github.com/aws/aws-sdk-go-v2/service/sso v1.15.1/go.mod h1:PieckvBoT5HtyB9AsJRrYZFY2Z+EyfVM/9zG6gbV8DQ= -github.com/aws/aws-sdk-go-v2/service/ssooidc v1.17.2 h1:fSCCJuT5i6ht8TqGdZc5Q5K9pz/atrf7qH4iK5C9XzU= -github.com/aws/aws-sdk-go-v2/service/ssooidc v1.17.2/go.mod h1:5eNtr+vNc5vVd92q7SJ+U/HszsIdhZBEyi9dkMRKsp8= +github.com/aws/aws-sdk-go-v2/service/sso v1.15.2 h1:JuPGc7IkOP4AaqcZSIcyqLpFSqBWK32rM9+a1g6u73k= +github.com/aws/aws-sdk-go-v2/service/sso v1.15.2/go.mod h1:gsL4keucRCgW+xA85ALBpRFfdSLH4kHOVSnLMSuBECo= +github.com/aws/aws-sdk-go-v2/service/ssooidc v1.17.3 h1:HFiiRkf1SdaAmV3/BHOFZ9DjFynPHj8G/UIO1lQS+fk= +github.com/aws/aws-sdk-go-v2/service/ssooidc v1.17.3/go.mod h1:a7bHA82fyUXOm+ZSWKU6PIoBxrjSprdLoM8xPYvzYVg= github.com/aws/aws-sdk-go-v2/service/sts v1.6.0/go.mod h1:q7o0j7d7HrJk/vr9uUt3BVRASvcU7gYZB9PUgPiByXg= -github.com/aws/aws-sdk-go-v2/service/sts v1.23.1 h1:ASNYk1ypWAxRhJjKS0jBnTUeDl7HROOpeSMu1xDA/I8= -github.com/aws/aws-sdk-go-v2/service/sts v1.23.1/go.mod h1:2cnsAhVT3mqusovc2stUSUrSBGTcX9nh8Tu6xh//2eI= +github.com/aws/aws-sdk-go-v2/service/sts v1.23.2 h1:0BkLfgeDjfZnZ+MhB3ONb01u9pwFYTCZVhlsSSBvlbU= +github.com/aws/aws-sdk-go-v2/service/sts v1.23.2/go.mod h1:Eows6e1uQEsc4ZaHANmsPRzAKcVDrcmjjWiih2+HUUQ= github.com/aws/smithy-go v1.6.0/go.mod h1:SObp3lf9smib00L/v3U2eAKG8FyQ7iLrJnQiAmR5n+E= github.com/aws/smithy-go v1.15.0 h1:PS/durmlzvAFpQHDs4wi4sNNP9ExsqZh6IlfdHXgKK8= github.com/aws/smithy-go v1.15.0/go.mod h1:Tg+OJXh4MB2R/uN61Ko2f6hTZwB/ZYGOtib8J3gBHzA= From 6a57d0d49e6864ec16e6fb2482af6f77ef8a10e8 Mon Sep 17 00:00:00 2001 From: Bruno Volpato Date: Mon, 16 Oct 2023 11:32:59 -0400 Subject: [PATCH 147/435] [JdbcIO] Allow fetchSize to be set for partitioned reads (#28999) --- .../main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java b/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java index 6e7ad865cc35..f8dad23d1fbd 100644 --- a/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java +++ b/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java @@ -360,6 +360,7 @@ public static ReadWithPartitions read return new AutoValue_JdbcIO_ReadWithPartitions.Builder() .setPartitionColumnType(partitioningColumnType) .setNumPartitions(DEFAULT_NUM_PARTITIONS) + .setFetchSize(DEFAULT_FETCH_SIZE) .setUseBeamSchema(false) .build(); } @@ -1195,6 +1196,9 @@ public abstract static class ReadWithPartitions @Pure abstract @Nullable String getPartitionColumn(); + @Pure + abstract int getFetchSize(); + @Pure abstract boolean getUseBeamSchema(); @@ -1233,6 +1237,8 @@ abstract Builder setDataSourceProviderFn( abstract Builder setUseBeamSchema(boolean useBeamSchema); + abstract Builder setFetchSize(int fetchSize); + abstract Builder setTable(String tableName); abstract Builder setPartitionColumnType( @@ -1357,7 +1363,8 @@ && getLowerBound() instanceof Comparable) { .withRowMapper( checkStateNotNull( JdbcUtil.JdbcReadWithPartitionsHelper.getPartitionsHelper( - getPartitionColumnType())))) + getPartitionColumnType()))) + .withFetchSize(getFetchSize())) .apply( MapElements.via( new SimpleFunction< @@ -1421,6 +1428,7 @@ public KV> apply( String.format( "select * from %1$s where %2$s >= ? and %2$s < ?", table, partitionColumn)) .withRowMapper(rowMapper) + .withFetchSize(getFetchSize()) .withParameterSetter( checkStateNotNull( JdbcUtil.JdbcReadWithPartitionsHelper.getPartitionsHelper( From 6758aeb0eb4e5e1dff9777896a331562e7001091 Mon Sep 17 00:00:00 2001 From: Anand Inguva <34158215+AnandInguva@users.noreply.github.com> Date: Mon, 16 Oct 2023 16:37:23 +0000 Subject: [PATCH 148/435] Support metric_name as list for perf alert tool (#28902) * Support metric name as list Support metric name as list Update README Run change point analysis even for PR but don't publish or save data fix tests, lint * fix mypy --- .github/workflows/run_perf_alert_tool.yml | 9 ++- .../apache_beam/testing/analyzers/README.md | 2 - .../testing/analyzers/perf_analysis.py | 69 +++++++++++++------ .../testing/analyzers/perf_analysis_test.py | 12 +++- .../testing/analyzers/perf_analysis_utils.py | 2 +- 5 files changed, 66 insertions(+), 28 deletions(-) diff --git a/.github/workflows/run_perf_alert_tool.yml b/.github/workflows/run_perf_alert_tool.yml index c61665c1bc7c..bc59bd945fe2 100644 --- a/.github/workflows/run_perf_alert_tool.yml +++ b/.github/workflows/run_perf_alert_tool.yml @@ -56,10 +56,17 @@ jobs: - name: Run Change Point Analysis. working-directory: ./sdks/python/apache_beam/testing/analyzers shell: bash - run: python perf_analysis.py + run: python perf_analysis.py --config_file_path=./tests_config.yaml --save_alert_metadata if: github.event_name != 'pull_request' env: GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} + - name: Run Change Point Analysis. + working-directory: ./sdks/python/apache_beam/testing/analyzers + shell: bash + run: python perf_analysis.py --config_file_path=./tests_config.yaml + if: github.event_name == 'pull_request' + env: + GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} - name: Run change point analysis tests. working-directory: ./sdks/python/apache_beam/testing/analyzers shell: bash diff --git a/sdks/python/apache_beam/testing/analyzers/README.md b/sdks/python/apache_beam/testing/analyzers/README.md index 91b21076f88a..cc8629f9a57a 100644 --- a/sdks/python/apache_beam/testing/analyzers/README.md +++ b/sdks/python/apache_beam/testing/analyzers/README.md @@ -58,8 +58,6 @@ These are the optional parameters that can be added to the test config in additi - `test_target`: Identifies the test responsible for the regression. -- `test_description`: Provides a brief overview of the test's function. - - `test_name`: Denotes the name of the test as stored in the BigQuery table. **Note**: The tool, by default, pulls metrics from BigQuery tables. Ensure that the values for `metrics_dataset`, `metrics_table`, `project`, and `metric_name` align with those defined for performance/load tests. The provided example utilizes this [test configuration](https://github.com/apache/beam/blob/0a91d139dea4276dc46176c4cdcdfce210fc50c4/.test-infra/jenkins/job_InferenceBenchmarkTests_Python.groovy#L30) to populate the necessary values for data retrieval. diff --git a/sdks/python/apache_beam/testing/analyzers/perf_analysis.py b/sdks/python/apache_beam/testing/analyzers/perf_analysis.py index 27f8398a0fb3..5802fe041454 100644 --- a/sdks/python/apache_beam/testing/analyzers/perf_analysis.py +++ b/sdks/python/apache_beam/testing/analyzers/perf_analysis.py @@ -22,13 +22,11 @@ import argparse import logging -import os import uuid from datetime import datetime from datetime import timezone from typing import Any from typing import Dict -from typing import Optional import pandas as pd @@ -50,6 +48,7 @@ def get_test_config_container( params: Dict[str, Any], test_id: str, + metric_name: str, ) -> TestConfigContainer: """ Args: @@ -61,7 +60,7 @@ def get_test_config_container( project=params['project'], metrics_dataset=params['metrics_dataset'], metrics_table=params['metrics_table'], - metric_name=params['metric_name'], + metric_name=metric_name, test_id=test_id, test_description=params['test_description'], test_name=params.get('test_name', None), @@ -89,6 +88,7 @@ def run_change_point_analysis( test_config_container: TestConfigContainer, big_query_metrics_fetcher: MetricsFetcher, change_point_config: ChangePointConfig = ChangePointConfig(), + save_alert_metadata: bool = False, ): """ Args: @@ -98,12 +98,14 @@ def run_change_point_analysis( change point analysis. change_point_config: ChangePointConfig containing parameters to run change point analysis. + save_alert_metadata: bool indicating if issue metadata + should be published to BigQuery table. Returns: bool indicating if a change point is observed and alerted on GitHub. """ logging.info( - "Running change point analysis for test ID %s" % - test_config_container.test_id) + "Running change point analysis for test ID :%s on metric: % s" % + (test_config_container.test_id, test_config_container.metric_name)) # test_name will be used to query a single test from # multiple tests in a single BQ table. Right now, the default @@ -152,9 +154,17 @@ def run_change_point_analysis( is_valid_change_point = True last_reported_issue_number = None + + # create a unique table name for each test and metric combination. + # for beam load tests, metric_name and metric table are enough to + # create a unique table name. For templates/IO tests, add `test_name`. issue_metadata_table_name = ( f'{test_config_container.metrics_table}_{test_config_container.metric_name}' # pylint: disable=line-too-long ) + if test_config_container.test_name: + issue_metadata_table_name = ( + f'{issue_metadata_table_name}_{test_config_container.test_name}') + existing_issue_data = get_existing_issues_data( table_name=issue_metadata_table_name) @@ -172,7 +182,11 @@ def run_change_point_analysis( timestamps=timestamps, min_runs_between_change_points=min_runs_between_change_points, test_id=test_config_container.test_id) - if is_valid_change_point: + + # for testing purposes, we don't want to create an issue even if there is + # a valid change point. This is useful when we want to test the change point + # analysis logic without creating an issue. + if is_valid_change_point and save_alert_metadata: issue_number, issue_url = create_performance_alert( test_config_container=test_config_container, metric_container=metric_container, @@ -192,7 +206,6 @@ def run_change_point_analysis( issue_url=issue_url, change_point_timestamp=timestamps[change_point_index], ) - publish_issue_metadata_to_big_query( issue_metadata=issue_metadata, table_name=issue_metadata_table_name, @@ -203,8 +216,9 @@ def run_change_point_analysis( def run( *, + config_file_path: str, big_query_metrics_fetcher: MetricsFetcher = BigQueryMetricsFetcher(), - config_file_path: Optional[str] = None, + save_alert_metadata: bool = False, ) -> None: """ run is the entry point to run change point analysis on test metric @@ -219,20 +233,25 @@ def run( defined in the config file. """ - if config_file_path is None: - config_file_path = os.path.join( - os.path.dirname(os.path.abspath(__file__)), 'tests_config.yaml') - tests_config: Dict[str, Dict[str, Any]] = read_test_config(config_file_path) for test_id, params in tests_config.items(): - test_config_container = get_test_config_container(params, test_id=test_id) - change_point_config = get_change_point_config(params) - run_change_point_analysis( - test_config_container=test_config_container, - big_query_metrics_fetcher=big_query_metrics_fetcher, - change_point_config=change_point_config, - ) + # single test config can have multiple metrics so we need to + # iterate over all the metrics and run change point analysis + # for each metric. + metric_names = params['metric_name'] + if isinstance(metric_names, str): + metric_names = [metric_names] + + for metric_name in metric_names: + test_config_container = get_test_config_container( + params=params, test_id=test_id, metric_name=metric_name) + change_point_config = get_change_point_config(params) + run_change_point_analysis( + test_config_container=test_config_container, + big_query_metrics_fetcher=big_query_metrics_fetcher, + change_point_config=change_point_config, + save_alert_metadata=save_alert_metadata) if __name__ == '__main__': @@ -241,7 +260,7 @@ def run( parser = argparse.ArgumentParser() parser.add_argument( '--config_file_path', - default=None, + required=True, type=str, help='Path to the config file that contains data to run the Change Point ' 'Analysis.The default file will used will be ' @@ -250,9 +269,17 @@ def run( 'performance regression in the tests, ' 'please provide an .yml file in the same structure as the above ' 'mentioned file. ') + parser.add_argument( + '--save_alert_metadata', + action='store_true', + help='Save perf alert/ GH Issue metadata to BigQuery table.') known_args, unknown_args = parser.parse_known_args() if unknown_args: logging.warning('Discarding unknown arguments : %s ' % unknown_args) - run(config_file_path=known_args.config_file_path) + run( + config_file_path=known_args.config_file_path, + # Set this to true while running in production. + save_alert_metadata=known_args.save_alert_metadata # pylint: disable=line-too-long + ) diff --git a/sdks/python/apache_beam/testing/analyzers/perf_analysis_test.py b/sdks/python/apache_beam/testing/analyzers/perf_analysis_test.py index 5164c8d8fd36..4ef394d4ffab 100644 --- a/sdks/python/apache_beam/testing/analyzers/perf_analysis_test.py +++ b/sdks/python/apache_beam/testing/analyzers/perf_analysis_test.py @@ -160,7 +160,9 @@ def test_duplicate_change_points_are_not_valid_alerts(self): get_fake_data_with_no_change_point) def test_no_alerts_when_no_change_points(self): test_config_container = analysis.get_test_config_container( - params=self.params, test_id=self.test_id) + params=self.params, + test_id=self.test_id, + metric_name=self.params['metric_name']) is_alert = analysis.run_change_point_analysis( test_config_container=test_config_container, big_query_metrics_fetcher=BigQueryMetricsFetcher()) @@ -183,7 +185,9 @@ def test_no_alerts_when_no_change_points(self): return_value=(0, '')) def test_alert_on_data_with_change_point(self, *args): test_config_container = analysis.get_test_config_container( - params=self.params, test_id=self.test_id) + params=self.params, + test_id=self.test_id, + metric_name=self.params['metric_name']) is_alert = analysis.run_change_point_analysis( test_config_container=test_config_container, big_query_metrics_fetcher=BigQueryMetricsFetcher()) @@ -205,7 +209,9 @@ def test_alert_on_data_with_change_point(self, *args): return_value=(0, '')) def test_alert_on_data_with_reported_change_point(self, *args): test_config_container = analysis.get_test_config_container( - params=self.params, test_id=self.test_id) + params=self.params, + test_id=self.test_id, + metric_name=self.params['metric_name']) is_alert = analysis.run_change_point_analysis( test_config_container=test_config_container, big_query_metrics_fetcher=BigQueryMetricsFetcher()) diff --git a/sdks/python/apache_beam/testing/analyzers/perf_analysis_utils.py b/sdks/python/apache_beam/testing/analyzers/perf_analysis_utils.py index 2b89ac9fdba9..11b1cc18ca56 100644 --- a/sdks/python/apache_beam/testing/analyzers/perf_analysis_utils.py +++ b/sdks/python/apache_beam/testing/analyzers/perf_analysis_utils.py @@ -66,7 +66,7 @@ class ChangePointConfig: @dataclass class TestConfigContainer: - metric_name: str # make this list instead. + metric_name: str project: str metrics_dataset: str metrics_table: str From 6ca14e5ce4641119fceb943b4b00c5b0d877db9d Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 16 Oct 2023 13:09:02 -0400 Subject: [PATCH 149/435] Bump google.golang.org/api from 0.146.0 to 0.147.0 in /sdks (#28975) Bumps [google.golang.org/api](https://github.com/googleapis/google-api-go-client) from 0.146.0 to 0.147.0. - [Release notes](https://github.com/googleapis/google-api-go-client/releases) - [Changelog](https://github.com/googleapis/google-api-go-client/blob/main/CHANGES.md) - [Commits](https://github.com/googleapis/google-api-go-client/compare/v0.146.0...v0.147.0) --- updated-dependencies: - dependency-name: google.golang.org/api dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 10 +++++----- sdks/go.sum | 24 ++++++++++++------------ 2 files changed, 17 insertions(+), 17 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 2dffed89df2e..1fe7a2a82525 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -57,8 +57,8 @@ require ( golang.org/x/sync v0.4.0 golang.org/x/sys v0.13.0 golang.org/x/text v0.13.0 - google.golang.org/api v0.146.0 - google.golang.org/genproto v0.0.0-20230913181813-007df8e322eb + google.golang.org/api v0.147.0 + google.golang.org/genproto v0.0.0-20231002182017-d307bd883b97 google.golang.org/grpc v1.58.3 google.golang.org/protobuf v1.31.0 gopkg.in/retry.v1 v1.0.3 @@ -85,7 +85,7 @@ require ( ) require ( - cloud.google.com/go v0.110.7 // indirect + cloud.google.com/go v0.110.8 // indirect cloud.google.com/go/compute v1.23.0 // indirect cloud.google.com/go/compute/metadata v0.2.3 // indirect cloud.google.com/go/iam v1.1.2 // indirect @@ -170,6 +170,6 @@ require ( golang.org/x/tools v0.10.0 // indirect golang.org/x/xerrors v0.0.0-20220907171357-04be3eba64a2 // indirect google.golang.org/appengine v1.6.7 // indirect - google.golang.org/genproto/googleapis/api v0.0.0-20230913181813-007df8e322eb // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20230920204549-e6e6cdab5c13 // indirect + google.golang.org/genproto/googleapis/api v0.0.0-20231002182017-d307bd883b97 // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20231009173412-8bfb1ae86b6c // indirect ) diff --git a/sdks/go.sum b/sdks/go.sum index 59115dcd9646..d535c2dda116 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -8,8 +8,8 @@ cloud.google.com/go v0.46.3/go.mod h1:a6bKKbmY7er1mI7TEI4lsAkts/mkhTSZK8w33B4RAg cloud.google.com/go v0.50.0/go.mod h1:r9sluTvynVuxRIOHXQEHMFffphuXHOMZMycpNR5e6To= cloud.google.com/go v0.52.0/go.mod h1:pXajvRH/6o3+F9jDHZWQ5PbGhn+o8w9qiu/CffaVdO4= cloud.google.com/go v0.53.0/go.mod h1:fp/UouUEsRkN6ryDKNW/Upv/JBKnv6WDthjR6+vze6M= -cloud.google.com/go v0.110.7 h1:rJyC7nWRg2jWGZ4wSJ5nY65GTdYJkg0cd/uXb+ACI6o= -cloud.google.com/go v0.110.7/go.mod h1:+EYjdK8e5RME/VY/qLCAtuyALQ9q67dvuum8i+H5xsI= +cloud.google.com/go v0.110.8 h1:tyNdfIxjzaWctIiLYOTalaLKZ17SI44SKFW26QbOhME= +cloud.google.com/go v0.110.8/go.mod h1:Iz8AkXJf1qmxC3Oxoep8R1T36w8B92yU29PcBhHO5fk= cloud.google.com/go/bigquery v1.0.1/go.mod h1:i/xbL2UlR5RvWAURpBYZTtm/cXjCha9lbfbpx4poX+o= cloud.google.com/go/bigquery v1.3.0/go.mod h1:PjpwJnslEMmckchkHFfq+HTD2DmtT67aNFKH1/VBDHE= cloud.google.com/go/bigquery v1.4.0/go.mod h1:S8dzgnTigyfTmLBfrtrhyYhwRxG72rYxvftPBK2Dvzc= @@ -21,14 +21,14 @@ cloud.google.com/go/compute v1.23.0 h1:tP41Zoavr8ptEqaW6j+LQOnyBBhO7OkOMAGrgLopT cloud.google.com/go/compute v1.23.0/go.mod h1:4tCnrn48xsqlwSAiLf1HXMQk8CONslYbdiEZc9FEIbM= cloud.google.com/go/compute/metadata v0.2.3 h1:mg4jlk7mCAj6xXp9UJ4fjI9VUI5rubuGBW5aJ7UnBMY= cloud.google.com/go/compute/metadata v0.2.3/go.mod h1:VAV5nSsACxMJvgaAuX6Pk2AawlZn8kiOGuCv6gTkwuA= -cloud.google.com/go/datacatalog v1.16.0 h1:qVeQcw1Cz93/cGu2E7TYUPh8Lz5dn5Ws2siIuQ17Vng= +cloud.google.com/go/datacatalog v1.17.1 h1:qGWrlYvWtK+8jD1jhwq5BsGoSr7S4/LOroV7LwXi00g= cloud.google.com/go/datastore v1.0.0/go.mod h1:LXYbyblFSglQ5pkeyhO+Qmw7ukd3C+pD7TKLgZqpHYE= cloud.google.com/go/datastore v1.1.0/go.mod h1:umbIZjpQpHh4hmRpGhH4tLFup+FVzqBi1b3c64qFpCk= cloud.google.com/go/datastore v1.14.0 h1:Mq0ApTRdLW3/dyiw+DkjTk0+iGIUvkbzaC8sfPwWTH4= cloud.google.com/go/datastore v1.14.0/go.mod h1:GAeStMBIt9bPS7jMJA85kgkpsMkvseWWXiaHya9Jes8= cloud.google.com/go/iam v1.1.2 h1:gacbrBdWcoVmGLozRuStX45YKvJtzIjJdAolzUs1sm4= cloud.google.com/go/iam v1.1.2/go.mod h1:A5avdyVL2tCppe4unb0951eI9jreack+RJ0/d+KUZOU= -cloud.google.com/go/kms v1.15.0 h1:xYl5WEaSekKYN5gGRyhjvZKM22GVBBCzegGNVPy+aIs= +cloud.google.com/go/kms v1.15.2 h1:lh6qra6oC4AyWe5fUUUBe/S27k12OHAleOOOw6KakdE= cloud.google.com/go/longrunning v0.5.1 h1:Fr7TXftcqTudoyRJa113hyaqlGdiBQkp0Gq7tErFDWI= cloud.google.com/go/longrunning v0.5.1/go.mod h1:spvimkwdz6SPWKEt/XBij79E9fiTkHSQl/fRUUQJYJc= cloud.google.com/go/profiler v0.3.1 h1:b5got9Be9Ia0HVvyt7PavWxXEht15B9lWnigdvHtxOc= @@ -671,8 +671,8 @@ google.golang.org/api v0.14.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsb google.golang.org/api v0.15.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI= google.golang.org/api v0.17.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= google.golang.org/api v0.18.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= -google.golang.org/api v0.146.0 h1:9aBYT4vQXt9dhCuLNfwfd3zpwu8atg0yPkjBymwSrOM= -google.golang.org/api v0.146.0/go.mod h1:OARJqIfoYjXJj4C1AiBSXYZt03qsoz8FQYU6fBEfrHM= +google.golang.org/api v0.147.0 h1:Can3FaQo9LlVqxJCodNmeZW/ib3/qKAY3rFeXiHo5gc= +google.golang.org/api v0.147.0/go.mod h1:pQ/9j83DcmPd/5C9e2nFOdjjNkDZ1G+zkbK2uvdkJMs= google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/appengine v1.5.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= @@ -698,12 +698,12 @@ google.golang.org/genproto v0.0.0-20200204135345-fa8e72b47b90/go.mod h1:GmwEX6Z4 google.golang.org/genproto v0.0.0-20200212174721-66ed5ce911ce/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= google.golang.org/genproto v0.0.0-20200224152610-e50cd9704f63/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= google.golang.org/genproto v0.0.0-20200526211855-cb27e3aa2013/go.mod h1:NbSheEEYHJ7i3ixzK3sjbqSGDJWnxyFXZblF3eUsNvo= -google.golang.org/genproto v0.0.0-20230913181813-007df8e322eb h1:XFBgcDwm7irdHTbz4Zk2h7Mh+eis4nfJEFQFYzJzuIA= -google.golang.org/genproto v0.0.0-20230913181813-007df8e322eb/go.mod h1:yZTlhN0tQnXo3h00fuXNCxJdLdIdnVFVBaRJ5LWBbw4= -google.golang.org/genproto/googleapis/api v0.0.0-20230913181813-007df8e322eb h1:lK0oleSc7IQsUxO3U5TjL9DWlsxpEBemh+zpB7IqhWI= -google.golang.org/genproto/googleapis/api v0.0.0-20230913181813-007df8e322eb/go.mod h1:KjSP20unUpOx5kyQUFa7k4OJg0qeJ7DEZflGDu2p6Bk= -google.golang.org/genproto/googleapis/rpc v0.0.0-20230920204549-e6e6cdab5c13 h1:N3bU/SQDCDyD6R528GJ/PwW9KjYcJA3dgyH+MovAkIM= -google.golang.org/genproto/googleapis/rpc v0.0.0-20230920204549-e6e6cdab5c13/go.mod h1:KSqppvjFjtoCI+KGd4PELB0qLNxdJHRGqRI09mB6pQA= +google.golang.org/genproto v0.0.0-20231002182017-d307bd883b97 h1:SeZZZx0cP0fqUyA+oRzP9k7cSwJlvDFiROO72uwD6i0= +google.golang.org/genproto v0.0.0-20231002182017-d307bd883b97/go.mod h1:t1VqOqqvce95G3hIDCT5FeO3YUc6Q4Oe24L/+rNMxRk= +google.golang.org/genproto/googleapis/api v0.0.0-20231002182017-d307bd883b97 h1:W18sezcAYs+3tDZX4F80yctqa12jcP1PUS2gQu1zTPU= +google.golang.org/genproto/googleapis/api v0.0.0-20231002182017-d307bd883b97/go.mod h1:iargEX0SFPm3xcfMI0d1domjg0ZF4Aa0p2awqyxhvF0= +google.golang.org/genproto/googleapis/rpc v0.0.0-20231009173412-8bfb1ae86b6c h1:jHkCUWkseRf+W+edG5hMzr/Uh1xkDREY4caybAq4dpY= +google.golang.org/genproto/googleapis/rpc v0.0.0-20231009173412-8bfb1ae86b6c/go.mod h1:4cYg8o5yUbm77w8ZX00LhMVNl/YVBFJRYWDc0uYWMs0= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.20.1/go.mod h1:10oTOabMzJvdu6/UiuZezV6QK5dSlG84ov/aaiqXj38= google.golang.org/grpc v1.21.1/go.mod h1:oYelfM1adQP15Ek0mdvEgi9Df8B9CZIaU1084ijfRaM= From 420418cf23ef75dd1b2ae345f268cba72e40d234 Mon Sep 17 00:00:00 2001 From: Vitaly Terentyev Date: Mon, 16 Oct 2023 23:41:26 +0400 Subject: [PATCH 150/435] Fix concurrency group (#28989) * Fix concurrency group * Fix concurrency group --- .github/workflows/beam_CancelStaleDataflowJobs.yml | 2 +- .github/workflows/beam_CleanUpGCPResources.yml | 2 +- .github/workflows/beam_CleanUpPrebuiltSDKImages.yml | 2 +- .github/workflows/beam_Inference_Python_Benchmarks_Dataflow.yml | 2 +- .github/workflows/beam_Java_LoadTests_Combine_Smoke.yml | 2 +- .github/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml | 2 +- .github/workflows/beam_LoadTests_Go_CoGBK_Flink_batch.yml | 2 +- .github/workflows/beam_LoadTests_Go_Combine_Dataflow_Batch.yml | 2 +- .github/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml | 2 +- .github/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml | 2 +- .github/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml | 2 +- .github/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml | 2 +- .github/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml | 2 +- .../workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml | 2 +- .github/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml | 2 +- .github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml | 2 +- .../workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml | 2 +- ...beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml | 2 +- ..._LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml | 2 +- ...beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml | 2 +- .../workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml | 2 +- .../beam_LoadTests_Java_Combine_Dataflow_Streaming.yml | 2 +- ...am_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml | 2 +- .github/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml | 2 +- .../workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml | 2 +- .../beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml | 2 +- .../beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml | 2 +- .../beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml | 2 +- .../beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml | 2 +- .../beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch.yml | 2 +- .github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml | 2 +- .../workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml | 2 +- ...beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml | 2 +- .../workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml | 2 +- .../beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml | 2 +- .github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml | 2 +- .../workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml | 2 +- .github/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml | 2 +- .../workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml | 2 +- .../beam_LoadTests_Python_FnApiRunner_Microbenchmark.yml | 2 +- .github/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml | 2 +- .../workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml | 2 +- .github/workflows/beam_LoadTests_Python_GBK_Flink_Batch.yml | 2 +- .../workflows/beam_LoadTests_Python_ParDo_Dataflow_Batch.yml | 2 +- .../beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml | 2 +- .github/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml | 2 +- .../workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml | 2 +- .../beam_LoadTests_Python_SideInput_Dataflow_Batch.yml | 2 +- .github/workflows/beam_PerformanceTests_AvroIOIT.yml | 2 +- .github/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml | 2 +- .../beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml | 2 +- .../beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml | 2 +- .../beam_PerformanceTests_BigQueryIO_Streaming_Java.yml | 2 +- .../workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml | 2 +- .../beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml | 2 +- .github/workflows/beam_PerformanceTests_Cdap.yml | 2 +- .github/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml | 2 +- .../beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml | 2 +- .github/workflows/beam_PerformanceTests_HadoopFormat.yml | 2 +- .github/workflows/beam_PerformanceTests_JDBC.yml | 2 +- .github/workflows/beam_PerformanceTests_Kafka_IO.yml | 2 +- .github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml | 2 +- .../workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml | 2 +- .github/workflows/beam_PerformanceTests_MongoDBIO_IT.yml | 2 +- .github/workflows/beam_PerformanceTests_ParquetIOIT.yml | 2 +- .github/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml | 2 +- .../beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml | 2 +- .../beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml | 2 +- .../beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml | 2 +- .../beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml | 2 +- .github/workflows/beam_PerformanceTests_SparkReceiver_IO.yml | 2 +- .github/workflows/beam_PerformanceTests_TFRecordIOIT.yml | 2 +- .github/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml | 2 +- .github/workflows/beam_PerformanceTests_TextIOIT.yml | 2 +- .github/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml | 2 +- .github/workflows/beam_PerformanceTests_TextIOIT_Python.yml | 2 +- .../beam_PerformanceTests_WordCountIT_PythonVersions.yml | 2 +- .github/workflows/beam_PerformanceTests_XmlIOIT.yml | 2 +- .github/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml | 2 +- .../workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml | 2 +- .github/workflows/beam_PostCommit_BeamMetrics_Publish.yml | 2 +- .github/workflows/beam_PostCommit_Go.yml | 2 +- .github/workflows/beam_PostCommit_Go_Dataflow_ARM.yml | 2 +- .github/workflows/beam_PostCommit_Go_VR_Flink.yml | 2 +- .github/workflows/beam_PostCommit_Go_VR_Samza.yml | 2 +- .github/workflows/beam_PostCommit_Go_VR_Spark.yml | 2 +- .github/workflows/beam_PostCommit_Java.yml | 2 +- .github/workflows/beam_PostCommit_Java_Avro_Versions.yml | 2 +- .github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml | 2 +- .github/workflows/beam_PostCommit_Java_DataflowV1.yml | 2 +- .github/workflows/beam_PostCommit_Java_DataflowV2.yml | 2 +- .github/workflows/beam_PostCommit_Java_Examples_Dataflow.yml | 2 +- .../workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml | 2 +- .../workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml | 2 +- .github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml | 2 +- .../beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml | 2 +- .github/workflows/beam_PostCommit_Java_Examples_Direct.yml | 2 +- .github/workflows/beam_PostCommit_Java_Examples_Flink.yml | 2 +- .github/workflows/beam_PostCommit_Java_Examples_Spark.yml | 2 +- .github/workflows/beam_PostCommit_Java_Hadoop_Versions.yml | 2 +- .github/workflows/beam_PostCommit_Java_IO_Performance_Tests.yml | 2 +- .github/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml | 2 +- .github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml | 2 +- .github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml | 2 +- .github/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml | 2 +- .github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml | 2 +- .github/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml | 2 +- .github/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml | 2 +- .github/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml | 2 +- .github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml | 2 +- .../workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml | 2 +- .github/workflows/beam_PostCommit_Java_Nexmark_Direct.yml | 2 +- .github/workflows/beam_PostCommit_Java_Nexmark_Flink.yml | 2 +- .github/workflows/beam_PostCommit_Java_Nexmark_Spark.yml | 2 +- .github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml | 2 +- .github/workflows/beam_PostCommit_Java_PVR_Samza.yml | 2 +- .github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml | 2 +- .github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml | 2 +- .github/workflows/beam_PostCommit_Java_Sickbay.yml | 2 +- .github/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml | 2 +- .github/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml | 2 +- .github/workflows/beam_PostCommit_Java_Tpcds_Flink.yml | 2 +- .github/workflows/beam_PostCommit_Java_Tpcds_Spark.yml | 2 +- .../workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml | 2 +- ...am_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml | 2 +- .../beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml | 2 +- .../beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml | 2 +- ...am_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml | 2 +- .../workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml | 2 +- ...beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml | 2 +- .../workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml | 2 +- .../beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml | 2 +- .../workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml | 2 +- .../workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml | 2 +- ...PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml | 2 +- .../beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml | 2 +- .../workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml | 2 +- .github/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml | 2 +- .github/workflows/beam_PostCommit_Javadoc.yml | 2 +- .github/workflows/beam_PostCommit_PortableJar_Flink.yml | 2 +- .github/workflows/beam_PostCommit_PortableJar_Spark.yml | 2 +- .github/workflows/beam_PostCommit_Python.yml | 2 +- .github/workflows/beam_PostCommit_Python_Arm.yml | 2 +- .github/workflows/beam_PostCommit_Python_Examples_Dataflow.yml | 2 +- .github/workflows/beam_PostCommit_Python_Examples_Direct.yml | 2 +- .github/workflows/beam_PostCommit_Python_Examples_Flink.yml | 2 +- .github/workflows/beam_PostCommit_Python_Examples_Spark.yml | 2 +- .github/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml | 2 +- .github/workflows/beam_PostCommit_Python_Nexmark_Direct.yml | 2 +- .../beam_PostCommit_Python_ValidatesContainer_Dataflow.yml | 2 +- ...am_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml | 2 +- .../beam_PostCommit_Python_ValidatesRunner_Dataflow.yml | 2 +- .../workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml | 2 +- .../workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml | 2 +- .../workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml | 2 +- .github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml | 2 +- .github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml | 2 +- .github/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml | 2 +- .github/workflows/beam_PostCommit_SQL.yml | 2 +- .github/workflows/beam_PostCommit_Sickbay_Python.yml | 2 +- .github/workflows/beam_PostCommit_TransformService_Direct.yml | 2 +- .github/workflows/beam_PostCommit_Website_Publish.yml | 2 +- .github/workflows/beam_PostCommit_Website_Test.yml | 2 +- .github/workflows/beam_PostCommit_XVR_Direct.yml | 2 +- .github/workflows/beam_PostCommit_XVR_Flink.yml | 2 +- .github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml | 2 +- .../workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml | 2 +- .../beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml | 2 +- .../workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml | 2 +- .github/workflows/beam_PostCommit_XVR_Samza.yml | 2 +- .github/workflows/beam_PostCommit_XVR_Spark3.yml | 2 +- .github/workflows/beam_PreCommit_CommunityMetrics.yml | 2 +- .github/workflows/beam_PreCommit_Go.yml | 2 +- .github/workflows/beam_PreCommit_GoPortable.yml | 2 +- .github/workflows/beam_PreCommit_GoPrism.yml | 2 +- .github/workflows/beam_PreCommit_ItFramework.yml | 2 +- .github/workflows/beam_PreCommit_Java.yml | 2 +- .../beam_PreCommit_Java_Amazon-Web-Services2_IO_Direct.yml | 2 +- .../beam_PreCommit_Java_Amazon-Web-Services_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Amqp_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml | 2 +- .../workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Examples_Dataflow.yml | 2 +- .../workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml | 2 +- .../workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml | 2 +- .../beam_PreCommit_Java_File-schema-transform_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Flink_Versions.yml | 2 +- .github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_IOs_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_PVR_Flink_Batch.yml | 2 +- .github/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml | 2 +- .github/workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Spark3_Versions.yml | 2 +- .github/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Kotlin_Examples.yml | 2 +- .github/workflows/beam_PreCommit_Portable_Python.yml | 2 +- .github/workflows/beam_PreCommit_Python.yml | 2 +- .github/workflows/beam_PreCommit_PythonDocker.yml | 2 +- .github/workflows/beam_PreCommit_PythonDocs.yml | 2 +- .github/workflows/beam_PreCommit_PythonFormatter.yml | 2 +- .github/workflows/beam_PreCommit_PythonLint.yml | 2 +- .github/workflows/beam_PreCommit_Python_Coverage.yml | 2 +- .github/workflows/beam_PreCommit_Python_Dataframes.yml | 2 +- .github/workflows/beam_PreCommit_Python_Examples.yml | 2 +- .github/workflows/beam_PreCommit_Python_Integration.yml | 2 +- .github/workflows/beam_PreCommit_Python_PVR_Flink.yml | 2 +- .github/workflows/beam_PreCommit_Python_Runners.yml | 2 +- .github/workflows/beam_PreCommit_Python_Transforms.yml | 2 +- .github/workflows/beam_PreCommit_RAT.yml | 2 +- .github/workflows/beam_PreCommit_SQL.yml | 2 +- .github/workflows/beam_PreCommit_SQL_Java11.yml | 2 +- .github/workflows/beam_PreCommit_SQL_Java17.yml | 2 +- .github/workflows/beam_PreCommit_Spotless.yml | 2 +- .github/workflows/beam_PreCommit_Typescript.yml | 2 +- .github/workflows/beam_PreCommit_Website.yml | 2 +- .github/workflows/beam_PreCommit_Website_Stage_GCS.yml | 2 +- .github/workflows/beam_PreCommit_Whitespace.yml | 2 +- .github/workflows/beam_Prober_CommunityMetrics.yml | 2 +- .github/workflows/beam_Publish_Docker_Snapshots.yml | 2 +- .../workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml | 2 +- 245 files changed, 245 insertions(+), 245 deletions(-) diff --git a/.github/workflows/beam_CancelStaleDataflowJobs.yml b/.github/workflows/beam_CancelStaleDataflowJobs.yml index 226ee78f01aa..0733e1ceb943 100644 --- a/.github/workflows/beam_CancelStaleDataflowJobs.yml +++ b/.github/workflows/beam_CancelStaleDataflowJobs.yml @@ -26,7 +26,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_CleanUpGCPResources.yml b/.github/workflows/beam_CleanUpGCPResources.yml index 42aeccb1a3b0..31ca81ec2a73 100644 --- a/.github/workflows/beam_CleanUpGCPResources.yml +++ b/.github/workflows/beam_CleanUpGCPResources.yml @@ -26,7 +26,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_CleanUpPrebuiltSDKImages.yml b/.github/workflows/beam_CleanUpPrebuiltSDKImages.yml index 62197137ca0c..cc0facae57c6 100644 --- a/.github/workflows/beam_CleanUpPrebuiltSDKImages.yml +++ b/.github/workflows/beam_CleanUpPrebuiltSDKImages.yml @@ -26,7 +26,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_Inference_Python_Benchmarks_Dataflow.yml b/.github/workflows/beam_Inference_Python_Benchmarks_Dataflow.yml index 117dc39a0fb7..b6c10b6ef60b 100644 --- a/.github/workflows/beam_Inference_Python_Benchmarks_Dataflow.yml +++ b/.github/workflows/beam_Inference_Python_Benchmarks_Dataflow.yml @@ -40,7 +40,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_Java_LoadTests_Combine_Smoke.yml b/.github/workflows/beam_Java_LoadTests_Combine_Smoke.yml index fd63141fcdff..6660caab2e3f 100644 --- a/.github/workflows/beam_Java_LoadTests_Combine_Smoke.yml +++ b/.github/workflows/beam_Java_LoadTests_Combine_Smoke.yml @@ -40,7 +40,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml index 0d4c9ab48d27..f50408e09b43 100644 --- a/.github/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_LoadTests_Go_CoGBK_Flink_batch.yml b/.github/workflows/beam_LoadTests_Go_CoGBK_Flink_batch.yml index 18238c796e47..49894879a915 100644 --- a/.github/workflows/beam_LoadTests_Go_CoGBK_Flink_batch.yml +++ b/.github/workflows/beam_LoadTests_Go_CoGBK_Flink_batch.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_LoadTests_Go_Combine_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Go_Combine_Dataflow_Batch.yml index 172f48a83e33..365f48fa1be2 100644 --- a/.github/workflows/beam_LoadTests_Go_Combine_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_Combine_Dataflow_Batch.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml index e5dbef1df53d..7cab058f5eeb 100644 --- a/.github/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml index ae4528ced8e4..30f8ef076d70 100644 --- a/.github/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml index a26fe19b79b7..41eb61efc580 100644 --- a/.github/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml index 18dbf26409ad..c1aa013e6ede 100644 --- a/.github/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml index 094005e2f532..a99422287a5b 100644 --- a/.github/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml index 6265fe428c00..38373e98a4cc 100644 --- a/.github/workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml index 20e8b4d54dd3..079aac5ad7f3 100644 --- a/.github/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml index 25feaaf288a1..8c05e76a0fdd 100644 --- a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml @@ -40,7 +40,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml index 3ad2cdf995e7..91ba9639c9c8 100644 --- a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml index 35575aec0bef..8c990cbb231a 100644 --- a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml +++ b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml @@ -40,7 +40,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml index 46dfaa986920..a6e28740b3c4 100644 --- a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml +++ b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml @@ -40,7 +40,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml b/.github/workflows/beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml index 3d18345589ed..1c39a54e7ca1 100644 --- a/.github/workflows/beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml +++ b/.github/workflows/beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml @@ -40,7 +40,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml index bc408402c792..de317b4a87ca 100644 --- a/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Streaming.yml index 029b7024d6ed..13b07f16b259 100644 --- a/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Streaming.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml b/.github/workflows/beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml index 5f4b8594ecde..bae659d14d09 100644 --- a/.github/workflows/beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml +++ b/.github/workflows/beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml index 9f7da2c00049..e3aca91c1e76 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml index fd718e23fd0a..f28c7a147b94 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml index 318f157e57a6..a7f60be0a2ce 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml index 543cb86985d8..88de449d9729 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml index fcd55f761c04..39166855e39f 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml index 5df7d3249192..4f76b3f45eb9 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch.yml b/.github/workflows/beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch.yml index 27505e9e88e9..13fe4df9906d 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml index 406cf18c85d1..1434521f7c28 100644 --- a/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml index 4b6fcb4ad51d..aabc12c366b5 100644 --- a/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml b/.github/workflows/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml index 89a903cc146c..b5fc5792e0ff 100644 --- a/.github/workflows/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml +++ b/.github/workflows/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml index 284ba2420654..5f63bdb9c3b5 100644 --- a/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml @@ -40,7 +40,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml index 313ba94f9df3..4e25d81bfcbc 100644 --- a/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml @@ -40,7 +40,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml index 6d668c1b2566..17851d5e57e1 100644 --- a/.github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml @@ -40,7 +40,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml index d8b97f2d6032..15143708c365 100644 --- a/.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml index d7d80e164283..6f7d2a8caa0b 100644 --- a/.github/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml b/.github/workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml index 66ac3bd2fc08..636dd20c148b 100644 --- a/.github/workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_LoadTests_Python_FnApiRunner_Microbenchmark.yml b/.github/workflows/beam_LoadTests_Python_FnApiRunner_Microbenchmark.yml index f3e2181991d7..3077376a4da9 100644 --- a/.github/workflows/beam_LoadTests_Python_FnApiRunner_Microbenchmark.yml +++ b/.github/workflows/beam_LoadTests_Python_FnApiRunner_Microbenchmark.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml index edf2e97857d6..d9d9564d3bfe 100644 --- a/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml index d7e17c2676f9..cf252f0758e0 100644 --- a/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_LoadTests_Python_GBK_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Python_GBK_Flink_Batch.yml index cba3bf9bd24e..cba71c0bd526 100644 --- a/.github/workflows/beam_LoadTests_Python_GBK_Flink_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_GBK_Flink_Batch.yml @@ -40,7 +40,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Batch.yml index 9f605282d131..c16bafd34b2c 100644 --- a/.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Batch.yml @@ -40,7 +40,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml index b681a2d95419..9d4bcdba6092 100644 --- a/.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml @@ -40,7 +40,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml index 66bbc72b14cd..7a450fadbdb2 100644 --- a/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml @@ -40,7 +40,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml b/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml index 82ab10d3eea5..37824b28ce0c 100644 --- a/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml @@ -40,7 +40,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_LoadTests_Python_SideInput_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Python_SideInput_Dataflow_Batch.yml index eb58c85e92e8..91e4c6a10514 100644 --- a/.github/workflows/beam_LoadTests_Python_SideInput_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_SideInput_Dataflow_Batch.yml @@ -40,7 +40,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PerformanceTests_AvroIOIT.yml b/.github/workflows/beam_PerformanceTests_AvroIOIT.yml index 14d798bb3e72..cc3799cd59d3 100644 --- a/.github/workflows/beam_PerformanceTests_AvroIOIT.yml +++ b/.github/workflows/beam_PerformanceTests_AvroIOIT.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml b/.github/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml index b84d56f34b9b..3bf97098213e 100644 --- a/.github/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml +++ b/.github/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml b/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml index 7f6daf23d05f..a79c35704a8f 100644 --- a/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml +++ b/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml b/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml index 13c60e107834..d59b94253991 100644 --- a/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml +++ b/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml b/.github/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml index 75a68c65a8b3..775fe5abe938 100644 --- a/.github/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml +++ b/.github/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml b/.github/workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml index ac84517443e9..c51e9c7d7442 100644 --- a/.github/workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml +++ b/.github/workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml b/.github/workflows/beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml index d2e5ddf3d07c..c1d25b21e14e 100644 --- a/.github/workflows/beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml +++ b/.github/workflows/beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PerformanceTests_Cdap.yml b/.github/workflows/beam_PerformanceTests_Cdap.yml index 269809121449..d8cd8f8fdbe5 100644 --- a/.github/workflows/beam_PerformanceTests_Cdap.yml +++ b/.github/workflows/beam_PerformanceTests_Cdap.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml b/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml index ed5e19454d26..cb1b77c84db4 100644 --- a/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml +++ b/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml b/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml index f43f96634459..bea11acf259a 100644 --- a/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml +++ b/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PerformanceTests_HadoopFormat.yml b/.github/workflows/beam_PerformanceTests_HadoopFormat.yml index f5514f765a57..b65387a857ee 100644 --- a/.github/workflows/beam_PerformanceTests_HadoopFormat.yml +++ b/.github/workflows/beam_PerformanceTests_HadoopFormat.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PerformanceTests_JDBC.yml b/.github/workflows/beam_PerformanceTests_JDBC.yml index 350718e94449..897fc461fd19 100644 --- a/.github/workflows/beam_PerformanceTests_JDBC.yml +++ b/.github/workflows/beam_PerformanceTests_JDBC.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PerformanceTests_Kafka_IO.yml b/.github/workflows/beam_PerformanceTests_Kafka_IO.yml index 6bb79bcc5b61..149be8587237 100644 --- a/.github/workflows/beam_PerformanceTests_Kafka_IO.yml +++ b/.github/workflows/beam_PerformanceTests_Kafka_IO.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml b/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml index 7f533c157234..0736b9d55b0c 100644 --- a/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml +++ b/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml b/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml index 96bb0573a911..90a6dbdd76df 100644 --- a/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml +++ b/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PerformanceTests_MongoDBIO_IT.yml b/.github/workflows/beam_PerformanceTests_MongoDBIO_IT.yml index 83e0a73a9c9b..187706e6c071 100644 --- a/.github/workflows/beam_PerformanceTests_MongoDBIO_IT.yml +++ b/.github/workflows/beam_PerformanceTests_MongoDBIO_IT.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PerformanceTests_ParquetIOIT.yml b/.github/workflows/beam_PerformanceTests_ParquetIOIT.yml index 03dbb650c4ab..df0f9137acf2 100644 --- a/.github/workflows/beam_PerformanceTests_ParquetIOIT.yml +++ b/.github/workflows/beam_PerformanceTests_ParquetIOIT.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml b/.github/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml index 49c2296340be..2cd7ac021df9 100644 --- a/.github/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml +++ b/.github/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml b/.github/workflows/beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml index 97082ea8aeda..4f652caa74c1 100644 --- a/.github/workflows/beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml +++ b/.github/workflows/beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml b/.github/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml index 2a93234e94e2..370e038a6bca 100644 --- a/.github/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml +++ b/.github/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml b/.github/workflows/beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml index 09e165e64e52..4a951e720e37 100644 --- a/.github/workflows/beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml +++ b/.github/workflows/beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml b/.github/workflows/beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml index b1f7761b1133..17d4b1cb9072 100644 --- a/.github/workflows/beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml +++ b/.github/workflows/beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PerformanceTests_SparkReceiver_IO.yml b/.github/workflows/beam_PerformanceTests_SparkReceiver_IO.yml index 8f6c72afb415..5a8a26ca1f61 100644 --- a/.github/workflows/beam_PerformanceTests_SparkReceiver_IO.yml +++ b/.github/workflows/beam_PerformanceTests_SparkReceiver_IO.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PerformanceTests_TFRecordIOIT.yml b/.github/workflows/beam_PerformanceTests_TFRecordIOIT.yml index 93e9132e1d1d..8c1f89b999a5 100644 --- a/.github/workflows/beam_PerformanceTests_TFRecordIOIT.yml +++ b/.github/workflows/beam_PerformanceTests_TFRecordIOIT.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml b/.github/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml index dab14af741b7..0e0da80e62a8 100644 --- a/.github/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml +++ b/.github/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml @@ -40,7 +40,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PerformanceTests_TextIOIT.yml b/.github/workflows/beam_PerformanceTests_TextIOIT.yml index c313731206be..32bb4821208c 100644 --- a/.github/workflows/beam_PerformanceTests_TextIOIT.yml +++ b/.github/workflows/beam_PerformanceTests_TextIOIT.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml b/.github/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml index 118605fe1026..94cda3343d5e 100644 --- a/.github/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml +++ b/.github/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PerformanceTests_TextIOIT_Python.yml b/.github/workflows/beam_PerformanceTests_TextIOIT_Python.yml index deb589b18c51..f49a573c3d91 100644 --- a/.github/workflows/beam_PerformanceTests_TextIOIT_Python.yml +++ b/.github/workflows/beam_PerformanceTests_TextIOIT_Python.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml b/.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml index 82069f943329..4fffb9389147 100644 --- a/.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml +++ b/.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PerformanceTests_XmlIOIT.yml b/.github/workflows/beam_PerformanceTests_XmlIOIT.yml index 5b2a64f1a919..43451bc095eb 100644 --- a/.github/workflows/beam_PerformanceTests_XmlIOIT.yml +++ b/.github/workflows/beam_PerformanceTests_XmlIOIT.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml b/.github/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml index 3358085a0120..32ffbd63cc79 100644 --- a/.github/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml +++ b/.github/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml b/.github/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml index b28a1824d88a..4d830223a5fa 100644 --- a/.github/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml +++ b/.github/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_BeamMetrics_Publish.yml b/.github/workflows/beam_PostCommit_BeamMetrics_Publish.yml index bccb2c8e78c6..177704a179a1 100644 --- a/.github/workflows/beam_PostCommit_BeamMetrics_Publish.yml +++ b/.github/workflows/beam_PostCommit_BeamMetrics_Publish.yml @@ -42,7 +42,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true diff --git a/.github/workflows/beam_PostCommit_Go.yml b/.github/workflows/beam_PostCommit_Go.yml index 5fff2d334a5a..e373d4f623bc 100644 --- a/.github/workflows/beam_PostCommit_Go.yml +++ b/.github/workflows/beam_PostCommit_Go.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Go_Dataflow_ARM.yml b/.github/workflows/beam_PostCommit_Go_Dataflow_ARM.yml index 2473a44b010b..4a2f00a64668 100644 --- a/.github/workflows/beam_PostCommit_Go_Dataflow_ARM.yml +++ b/.github/workflows/beam_PostCommit_Go_Dataflow_ARM.yml @@ -44,7 +44,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Go_VR_Flink.yml b/.github/workflows/beam_PostCommit_Go_VR_Flink.yml index 3923e9d74f94..6f43cd461f25 100644 --- a/.github/workflows/beam_PostCommit_Go_VR_Flink.yml +++ b/.github/workflows/beam_PostCommit_Go_VR_Flink.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Go_VR_Samza.yml b/.github/workflows/beam_PostCommit_Go_VR_Samza.yml index 4e0663be0dbf..f317615914ce 100644 --- a/.github/workflows/beam_PostCommit_Go_VR_Samza.yml +++ b/.github/workflows/beam_PostCommit_Go_VR_Samza.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Go_VR_Spark.yml b/.github/workflows/beam_PostCommit_Go_VR_Spark.yml index 85262335c9a5..98abb1004b91 100644 --- a/.github/workflows/beam_PostCommit_Go_VR_Spark.yml +++ b/.github/workflows/beam_PostCommit_Go_VR_Spark.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Java.yml b/.github/workflows/beam_PostCommit_Java.yml index 0943e2dec263..601a438d5a4b 100644 --- a/.github/workflows/beam_PostCommit_Java.yml +++ b/.github/workflows/beam_PostCommit_Java.yml @@ -24,7 +24,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_Avro_Versions.yml b/.github/workflows/beam_PostCommit_Java_Avro_Versions.yml index 63ffda3864e6..69313a568f32 100644 --- a/.github/workflows/beam_PostCommit_Java_Avro_Versions.yml +++ b/.github/workflows/beam_PostCommit_Java_Avro_Versions.yml @@ -24,7 +24,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml b/.github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml index f74b429988d7..42330b5fc760 100644 --- a/.github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml +++ b/.github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml @@ -24,7 +24,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_DataflowV1.yml b/.github/workflows/beam_PostCommit_Java_DataflowV1.yml index b6cf668ddfd3..e9dcdcb874eb 100644 --- a/.github/workflows/beam_PostCommit_Java_DataflowV1.yml +++ b/.github/workflows/beam_PostCommit_Java_DataflowV1.yml @@ -24,7 +24,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_DataflowV2.yml b/.github/workflows/beam_PostCommit_Java_DataflowV2.yml index f99c08c36f8d..70423a06c56f 100644 --- a/.github/workflows/beam_PostCommit_Java_DataflowV2.yml +++ b/.github/workflows/beam_PostCommit_Java_DataflowV2.yml @@ -24,7 +24,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow.yml index 3b3b51b94f25..a302e631a40c 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow.yml @@ -26,7 +26,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml index 939d2646d352..2e2adeea7cb0 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml @@ -37,7 +37,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml index 3d2523654fb6..2f798f71a624 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml @@ -24,7 +24,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml index 67780ae4a848..aadb14b50ef2 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml @@ -24,7 +24,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml index 74e1787945e3..1277153e20b2 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml @@ -24,7 +24,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Direct.yml b/.github/workflows/beam_PostCommit_Java_Examples_Direct.yml index 9508dfe22617..f8c5835f5aca 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Direct.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Direct.yml @@ -24,7 +24,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Flink.yml b/.github/workflows/beam_PostCommit_Java_Examples_Flink.yml index c359ae2f9e53..908e7d4286f3 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Flink.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Flink.yml @@ -24,7 +24,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Spark.yml b/.github/workflows/beam_PostCommit_Java_Examples_Spark.yml index 78130d9ec247..0449186e1ed0 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Spark.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Spark.yml @@ -24,7 +24,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_Hadoop_Versions.yml b/.github/workflows/beam_PostCommit_Java_Hadoop_Versions.yml index 189fbb083a39..65ade4bf63d4 100644 --- a/.github/workflows/beam_PostCommit_Java_Hadoop_Versions.yml +++ b/.github/workflows/beam_PostCommit_Java_Hadoop_Versions.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Java_IO_Performance_Tests.yml b/.github/workflows/beam_PostCommit_Java_IO_Performance_Tests.yml index b38651352a67..649e48fb889d 100644 --- a/.github/workflows/beam_PostCommit_Java_IO_Performance_Tests.yml +++ b/.github/workflows/beam_PostCommit_Java_IO_Performance_Tests.yml @@ -44,7 +44,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.sender.login }}-${{ github.event.schedule }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml b/.github/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml index 7f1d76a13936..ba525e3dd424 100644 --- a/.github/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml +++ b/.github/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml @@ -24,7 +24,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml index d5b23c095d0b..4c52984bcaa3 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml index 0b9a366211ee..404292a83402 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml index 97f03ed59211..5046430d31b6 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml index fcdbd63f7e97..8c4a78aea663 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml index 832e211a4d61..2dcf6a4444f5 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml index 14b5e5380921..01680a55026a 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml index 44bdb4e51ca8..db15117d3cba 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml @@ -24,7 +24,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml index 50a72af7909a..4865e1f3cb6e 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml @@ -24,7 +24,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml index d78793172898..1c58bf2510a2 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml @@ -24,7 +24,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Direct.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Direct.yml index e7c6e4655671..3f230b0b78b8 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Direct.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Direct.yml @@ -24,7 +24,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Flink.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Flink.yml index 7dc59a1a2e72..a525a65e1ad2 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Flink.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Flink.yml @@ -24,7 +24,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Spark.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Spark.yml index fd1f9745d80e..6427a3ca33d9 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Spark.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Spark.yml @@ -24,7 +24,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml b/.github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml index efab6427642a..954aa045e6b7 100644 --- a/.github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml +++ b/.github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml @@ -24,7 +24,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_PVR_Samza.yml b/.github/workflows/beam_PostCommit_Java_PVR_Samza.yml index 942028ea3569..54016ea45cd3 100644 --- a/.github/workflows/beam_PostCommit_Java_PVR_Samza.yml +++ b/.github/workflows/beam_PostCommit_Java_PVR_Samza.yml @@ -24,7 +24,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml b/.github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml index 6b47a64572ea..550b794bc16c 100644 --- a/.github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml +++ b/.github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml @@ -24,7 +24,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml b/.github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml index 82b132522c12..c240ac5abf26 100644 --- a/.github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml +++ b/.github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml @@ -24,7 +24,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_Sickbay.yml b/.github/workflows/beam_PostCommit_Java_Sickbay.yml index 5d2da81e2ab0..037bd610ab0a 100644 --- a/.github/workflows/beam_PostCommit_Java_Sickbay.yml +++ b/.github/workflows/beam_PostCommit_Java_Sickbay.yml @@ -24,7 +24,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml b/.github/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml index ac8b5cacf40b..e79a525c3157 100644 --- a/.github/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml +++ b/.github/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml @@ -40,7 +40,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml b/.github/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml index 3239281fb938..9cb67bbdf3c2 100644 --- a/.github/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Java_Tpcds_Flink.yml b/.github/workflows/beam_PostCommit_Java_Tpcds_Flink.yml index c6dcdcfaa3a4..6864576f651e 100644 --- a/.github/workflows/beam_PostCommit_Java_Tpcds_Flink.yml +++ b/.github/workflows/beam_PostCommit_Java_Tpcds_Flink.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Java_Tpcds_Spark.yml b/.github/workflows/beam_PostCommit_Java_Tpcds_Spark.yml index 73fa4e5ba143..be4aef55f714 100644 --- a/.github/workflows/beam_PostCommit_Java_Tpcds_Spark.yml +++ b/.github/workflows/beam_PostCommit_Java_Tpcds_Spark.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml index 0a0095e941ae..1ddca5169180 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml @@ -24,7 +24,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml index e7d9bf9ccb6b..b50c02a0e31b 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml @@ -24,7 +24,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml index 094133af2bb9..0224bfd9dab5 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml @@ -24,7 +24,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml index 4331772d32d4..c15db0de5baf 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml @@ -24,7 +24,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml index e0f90156f370..768dcdf4fb57 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml @@ -24,7 +24,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml index 1020b3869b97..9cb613f2a1df 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml @@ -24,7 +24,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml index 12bd26d070ed..f2cbebf1407a 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml @@ -24,7 +24,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml index 5de57835a190..6df33f411b8d 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml index aa5c24ac2b21..af9b06f20d45 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml @@ -24,7 +24,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml index 62e003a1660d..cc68b3c7cc67 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml index 083e473a3a56..b1c0395dc87a 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml index 74bb0b15b8de..2047e5a448a2 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml index 8082f2143b38..44d0cd11d784 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml @@ -24,7 +24,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml index 530c360ea346..72ad389087f1 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml index 61e0affa5e81..3296edfd3e62 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Javadoc.yml b/.github/workflows/beam_PostCommit_Javadoc.yml index 8413f39b3b9d..4a38b9d22a3b 100644 --- a/.github/workflows/beam_PostCommit_Javadoc.yml +++ b/.github/workflows/beam_PostCommit_Javadoc.yml @@ -24,7 +24,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_PortableJar_Flink.yml b/.github/workflows/beam_PostCommit_PortableJar_Flink.yml index 18941e70a881..b7c67645699a 100644 --- a/.github/workflows/beam_PostCommit_PortableJar_Flink.yml +++ b/.github/workflows/beam_PostCommit_PortableJar_Flink.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_PortableJar_Spark.yml b/.github/workflows/beam_PostCommit_PortableJar_Spark.yml index 55cdffcb86a1..06360db4d82d 100644 --- a/.github/workflows/beam_PostCommit_PortableJar_Spark.yml +++ b/.github/workflows/beam_PostCommit_PortableJar_Spark.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Python.yml b/.github/workflows/beam_PostCommit_Python.yml index 97299644e5dc..22c15dec1bed 100644 --- a/.github/workflows/beam_PostCommit_Python.yml +++ b/.github/workflows/beam_PostCommit_Python.yml @@ -24,7 +24,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Python_Arm.yml b/.github/workflows/beam_PostCommit_Python_Arm.yml index 8be303a82d1d..bd7b7158db34 100644 --- a/.github/workflows/beam_PostCommit_Python_Arm.yml +++ b/.github/workflows/beam_PostCommit_Python_Arm.yml @@ -26,7 +26,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Python_Examples_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_Examples_Dataflow.yml index 81687e498b8b..329eec386ec0 100644 --- a/.github/workflows/beam_PostCommit_Python_Examples_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_Examples_Dataflow.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Python_Examples_Direct.yml b/.github/workflows/beam_PostCommit_Python_Examples_Direct.yml index 903101e5a01c..b1d39903a405 100644 --- a/.github/workflows/beam_PostCommit_Python_Examples_Direct.yml +++ b/.github/workflows/beam_PostCommit_Python_Examples_Direct.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Python_Examples_Flink.yml b/.github/workflows/beam_PostCommit_Python_Examples_Flink.yml index aa1404a4aa64..0c3017af34b3 100644 --- a/.github/workflows/beam_PostCommit_Python_Examples_Flink.yml +++ b/.github/workflows/beam_PostCommit_Python_Examples_Flink.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Python_Examples_Spark.yml b/.github/workflows/beam_PostCommit_Python_Examples_Spark.yml index 7716ca3e67b1..46315ae80e54 100644 --- a/.github/workflows/beam_PostCommit_Python_Examples_Spark.yml +++ b/.github/workflows/beam_PostCommit_Python_Examples_Spark.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml b/.github/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml index 7c8b7d2526ab..5a58f11e2dbc 100644 --- a/.github/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml +++ b/.github/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Python_Nexmark_Direct.yml b/.github/workflows/beam_PostCommit_Python_Nexmark_Direct.yml index bf96c1cf74be..b04a392c6770 100644 --- a/.github/workflows/beam_PostCommit_Python_Nexmark_Direct.yml +++ b/.github/workflows/beam_PostCommit_Python_Nexmark_Direct.yml @@ -24,7 +24,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml index 713f7fa94e20..5b491c4c3e50 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml b/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml index 44df23f0296e..a94b4f234e6e 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml index 6ff8edb9cce8..efb9833ba248 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml index de2ed09b9614..dfb1b15c8312 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml index b86fcfac218b..a8e4495e3cd8 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml index 65fa97a5b5ea..d5c88efe9074 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml index 6c2255f60f1d..3a5934fe7aa2 100644 --- a/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml b/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml index 36d100473f98..b1fcfc52352d 100644 --- a/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml +++ b/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml index eb9665d39d7f..c0b8b2c993c4 100644 --- a/.github/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_SQL.yml b/.github/workflows/beam_PostCommit_SQL.yml index 3cd72fcecc42..42b1a69fcc14 100644 --- a/.github/workflows/beam_PostCommit_SQL.yml +++ b/.github/workflows/beam_PostCommit_SQL.yml @@ -24,7 +24,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Sickbay_Python.yml b/.github/workflows/beam_PostCommit_Sickbay_Python.yml index 7e15ac7e8873..914f35bcd47c 100644 --- a/.github/workflows/beam_PostCommit_Sickbay_Python.yml +++ b/.github/workflows/beam_PostCommit_Sickbay_Python.yml @@ -24,7 +24,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_TransformService_Direct.yml b/.github/workflows/beam_PostCommit_TransformService_Direct.yml index 491688e097a0..44ebcb83fb35 100644 --- a/.github/workflows/beam_PostCommit_TransformService_Direct.yml +++ b/.github/workflows/beam_PostCommit_TransformService_Direct.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Website_Publish.yml b/.github/workflows/beam_PostCommit_Website_Publish.yml index a1e23b530a26..834c3d043afe 100644 --- a/.github/workflows/beam_PostCommit_Website_Publish.yml +++ b/.github/workflows/beam_PostCommit_Website_Publish.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_Website_Test.yml b/.github/workflows/beam_PostCommit_Website_Test.yml index bb12d5804b6f..6f2229ff76ff 100644 --- a/.github/workflows/beam_PostCommit_Website_Test.yml +++ b/.github/workflows/beam_PostCommit_Website_Test.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_XVR_Direct.yml b/.github/workflows/beam_PostCommit_XVR_Direct.yml index 8db745b8b217..e5771b4f9d44 100644 --- a/.github/workflows/beam_PostCommit_XVR_Direct.yml +++ b/.github/workflows/beam_PostCommit_XVR_Direct.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_XVR_Flink.yml b/.github/workflows/beam_PostCommit_XVR_Flink.yml index 640555a03ec1..5bbde0e86b94 100644 --- a/.github/workflows/beam_PostCommit_XVR_Flink.yml +++ b/.github/workflows/beam_PostCommit_XVR_Flink.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml b/.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml index 6231c683c89e..439453364e22 100644 --- a/.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml b/.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml index 4148d0a2d468..52021e547de7 100644 --- a/.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml b/.github/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml index 6c8e01124d0b..6bb2ecf62d83 100644 --- a/.github/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml b/.github/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml index 486fe84ff45c..924eb525dbf7 100644 --- a/.github/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_XVR_Samza.yml b/.github/workflows/beam_PostCommit_XVR_Samza.yml index 8a6c9bda6539..b00daae8f31a 100644 --- a/.github/workflows/beam_PostCommit_XVR_Samza.yml +++ b/.github/workflows/beam_PostCommit_XVR_Samza.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_XVR_Spark3.yml b/.github/workflows/beam_PostCommit_XVR_Spark3.yml index b61cfe3251f0..55275c85b563 100644 --- a/.github/workflows/beam_PostCommit_XVR_Spark3.yml +++ b/.github/workflows/beam_PostCommit_XVR_Spark3.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_CommunityMetrics.yml b/.github/workflows/beam_PreCommit_CommunityMetrics.yml index bb44ca0b5464..b3f100525bda 100644 --- a/.github/workflows/beam_PreCommit_CommunityMetrics.yml +++ b/.github/workflows/beam_PreCommit_CommunityMetrics.yml @@ -47,7 +47,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Go.yml b/.github/workflows/beam_PreCommit_Go.yml index b7b561d54c1f..c202c30d5a4b 100644 --- a/.github/workflows/beam_PreCommit_Go.yml +++ b/.github/workflows/beam_PreCommit_Go.yml @@ -47,7 +47,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_GoPortable.yml b/.github/workflows/beam_PreCommit_GoPortable.yml index 1c40a3c8d129..0ab59224aae6 100644 --- a/.github/workflows/beam_PreCommit_GoPortable.yml +++ b/.github/workflows/beam_PreCommit_GoPortable.yml @@ -31,7 +31,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_GoPrism.yml b/.github/workflows/beam_PreCommit_GoPrism.yml index 0e60579ec64d..1b0eecdcf8dc 100644 --- a/.github/workflows/beam_PreCommit_GoPrism.yml +++ b/.github/workflows/beam_PreCommit_GoPrism.yml @@ -31,7 +31,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_ItFramework.yml b/.github/workflows/beam_PreCommit_ItFramework.yml index 6161f1422566..b1000d9feea2 100644 --- a/.github/workflows/beam_PreCommit_ItFramework.yml +++ b/.github/workflows/beam_PreCommit_ItFramework.yml @@ -36,7 +36,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java.yml b/.github/workflows/beam_PreCommit_Java.yml index 9a58f42fef78..fdd51e703584 100644 --- a/.github/workflows/beam_PreCommit_Java.yml +++ b/.github/workflows/beam_PreCommit_Java.yml @@ -122,7 +122,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_Java_Amazon-Web-Services2_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Amazon-Web-Services2_IO_Direct.yml index 2d27ecc38be0..22877bc543f9 100644 --- a/.github/workflows/beam_PreCommit_Java_Amazon-Web-Services2_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Amazon-Web-Services2_IO_Direct.yml @@ -68,7 +68,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_Amazon-Web-Services_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Amazon-Web-Services_IO_Direct.yml index c812aac4d447..22c114a7c476 100644 --- a/.github/workflows/beam_PreCommit_Java_Amazon-Web-Services_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Amazon-Web-Services_IO_Direct.yml @@ -68,7 +68,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_Amqp_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Amqp_IO_Direct.yml index 8898cf36f188..82f11ccdd223 100644 --- a/.github/workflows/beam_PreCommit_Java_Amqp_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Amqp_IO_Direct.yml @@ -50,7 +50,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml index 84ffe4dab834..16d1f64b9dcd 100644 --- a/.github/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml @@ -68,7 +68,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml index 35f0e061f091..fd3f4fd68970 100644 --- a/.github/workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml @@ -50,7 +50,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml index b23490afe8f7..7161f766d7ff 100644 --- a/.github/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml @@ -54,7 +54,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml index 0dd75cb0a6d2..aa8d7aff5e0c 100644 --- a/.github/workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml @@ -50,7 +50,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml index 17e1876d20f3..c84472e16e98 100644 --- a/.github/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml @@ -50,7 +50,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml index 6f10c210b010..b855b8280f50 100644 --- a/.github/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml @@ -50,7 +50,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml index 6775708c2b8f..7cca73a505a4 100644 --- a/.github/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml @@ -52,7 +52,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow.yml b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow.yml index 29a637baff3a..500b386f41d6 100644 --- a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow.yml +++ b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow.yml @@ -46,7 +46,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml index 03f4d32861b1..65ed3b922439 100644 --- a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml +++ b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml @@ -46,7 +46,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml index cf8cefe388f3..64809809dbd8 100644 --- a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml +++ b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml @@ -60,7 +60,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_File-schema-transform_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_File-schema-transform_IO_Direct.yml index 1bfd58d6a0d6..549543ccef89 100644 --- a/.github/workflows/beam_PreCommit_Java_File-schema-transform_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_File-schema-transform_IO_Direct.yml @@ -50,7 +50,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_Flink_Versions.yml b/.github/workflows/beam_PreCommit_Java_Flink_Versions.yml index cefbf897de85..6590583925e1 100644 --- a/.github/workflows/beam_PreCommit_Java_Flink_Versions.yml +++ b/.github/workflows/beam_PreCommit_Java_Flink_Versions.yml @@ -53,7 +53,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml index 30e8d6d6c33c..b9068de73539 100644 --- a/.github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml @@ -68,7 +68,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml index 23d0afb933de..2f5e04f3f29b 100644 --- a/.github/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml @@ -52,7 +52,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml index 777725d7c3ec..b95d111e1879 100644 --- a/.github/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml @@ -52,7 +52,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml index 2b93fa4c524b..43f870711f4e 100644 --- a/.github/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml @@ -76,7 +76,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_IOs_Direct.yml b/.github/workflows/beam_PreCommit_Java_IOs_Direct.yml index 2825719a24ef..7696ad7f6059 100644 --- a/.github/workflows/beam_PreCommit_Java_IOs_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_IOs_Direct.yml @@ -51,7 +51,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml index 6d6bd34deef2..2174f836f2a9 100644 --- a/.github/workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml @@ -50,7 +50,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml index 2d36c05d2881..b364d2d1cc1a 100644 --- a/.github/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml @@ -50,7 +50,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml index fa8d2721d38e..05b081f4e737 100644 --- a/.github/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml @@ -50,7 +50,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml index 629190e328e5..667f00e234a0 100644 --- a/.github/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml @@ -58,7 +58,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml index 47411e0df379..6abd50dc6c4c 100644 --- a/.github/workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml @@ -68,7 +68,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml index 40fdb06d3556..0263ce70cf50 100644 --- a/.github/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml @@ -50,7 +50,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml index f210affbb21f..dfbc30e82161 100644 --- a/.github/workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml @@ -50,7 +50,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml index 6f30a3dd84ba..13c874737f0b 100644 --- a/.github/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml @@ -50,7 +50,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml index 8e8041d434c7..27345be9adc6 100644 --- a/.github/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml @@ -52,7 +52,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_PVR_Flink_Batch.yml b/.github/workflows/beam_PreCommit_Java_PVR_Flink_Batch.yml index b592d704661d..a029e071ca80 100644 --- a/.github/workflows/beam_PreCommit_Java_PVR_Flink_Batch.yml +++ b/.github/workflows/beam_PreCommit_Java_PVR_Flink_Batch.yml @@ -40,7 +40,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml b/.github/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml index 327f65436560..ac6b9e0471a4 100644 --- a/.github/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml +++ b/.github/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml @@ -60,7 +60,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml index c00d0e43b619..4847e9b2a01b 100644 --- a/.github/workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml @@ -50,7 +50,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml index b0b73d944275..810a787ac7b3 100644 --- a/.github/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml @@ -68,7 +68,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml index a259c09beb74..174df8a37a48 100644 --- a/.github/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml @@ -50,7 +50,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml index cd2741ead81a..a0e2dfeb9b58 100644 --- a/.github/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml @@ -50,7 +50,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml index 9c370e607368..eb2a8521c597 100644 --- a/.github/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml @@ -52,7 +52,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml index 3e27d5c5cfe8..14fc532c2f3f 100644 --- a/.github/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml @@ -54,7 +54,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml index c7d13e49f2a4..94db93a74b8c 100644 --- a/.github/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml @@ -50,7 +50,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_Spark3_Versions.yml b/.github/workflows/beam_PreCommit_Java_Spark3_Versions.yml index 9b2ada1da776..0a8096bf88b4 100644 --- a/.github/workflows/beam_PreCommit_Java_Spark3_Versions.yml +++ b/.github/workflows/beam_PreCommit_Java_Spark3_Versions.yml @@ -36,7 +36,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml index 7e453c379281..53a933cff922 100644 --- a/.github/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml @@ -50,7 +50,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml index 71528a884312..1ae098097cc9 100644 --- a/.github/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml @@ -50,7 +50,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml index 7d26f12ee635..98b6654c6d5e 100644 --- a/.github/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml @@ -50,7 +50,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Kotlin_Examples.yml b/.github/workflows/beam_PreCommit_Kotlin_Examples.yml index fb6c0f7696aa..2c16cd8e4749 100644 --- a/.github/workflows/beam_PreCommit_Kotlin_Examples.yml +++ b/.github/workflows/beam_PreCommit_Kotlin_Examples.yml @@ -46,7 +46,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_Portable_Python.yml b/.github/workflows/beam_PreCommit_Portable_Python.yml index 0134c497b836..bd35b1d0c1ef 100644 --- a/.github/workflows/beam_PreCommit_Portable_Python.yml +++ b/.github/workflows/beam_PreCommit_Portable_Python.yml @@ -66,7 +66,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Python.yml b/.github/workflows/beam_PreCommit_Python.yml index 35e7b937068d..1ee602e02c0c 100644 --- a/.github/workflows/beam_PreCommit_Python.yml +++ b/.github/workflows/beam_PreCommit_Python.yml @@ -46,7 +46,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_PythonDocker.yml b/.github/workflows/beam_PreCommit_PythonDocker.yml index aa119cf674c2..507600e6a32b 100644 --- a/.github/workflows/beam_PreCommit_PythonDocker.yml +++ b/.github/workflows/beam_PreCommit_PythonDocker.yml @@ -46,7 +46,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_PythonDocs.yml b/.github/workflows/beam_PreCommit_PythonDocs.yml index 844d41f2cb52..63e398aa5d4c 100644 --- a/.github/workflows/beam_PreCommit_PythonDocs.yml +++ b/.github/workflows/beam_PreCommit_PythonDocs.yml @@ -47,7 +47,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_PythonFormatter.yml b/.github/workflows/beam_PreCommit_PythonFormatter.yml index 23093e1db006..8795b71ade70 100644 --- a/.github/workflows/beam_PreCommit_PythonFormatter.yml +++ b/.github/workflows/beam_PreCommit_PythonFormatter.yml @@ -46,7 +46,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_PythonLint.yml b/.github/workflows/beam_PreCommit_PythonLint.yml index 8a4558c8988d..2e75b61c10e1 100644 --- a/.github/workflows/beam_PreCommit_PythonLint.yml +++ b/.github/workflows/beam_PreCommit_PythonLint.yml @@ -46,7 +46,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Python_Coverage.yml b/.github/workflows/beam_PreCommit_Python_Coverage.yml index 4b274d643e07..6bc3e3911921 100644 --- a/.github/workflows/beam_PreCommit_Python_Coverage.yml +++ b/.github/workflows/beam_PreCommit_Python_Coverage.yml @@ -46,7 +46,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Python_Dataframes.yml b/.github/workflows/beam_PreCommit_Python_Dataframes.yml index f03716d06795..5750d020607b 100644 --- a/.github/workflows/beam_PreCommit_Python_Dataframes.yml +++ b/.github/workflows/beam_PreCommit_Python_Dataframes.yml @@ -46,7 +46,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Python_Examples.yml b/.github/workflows/beam_PreCommit_Python_Examples.yml index d629ee09b725..39d65e7833ff 100644 --- a/.github/workflows/beam_PreCommit_Python_Examples.yml +++ b/.github/workflows/beam_PreCommit_Python_Examples.yml @@ -46,7 +46,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Python_Integration.yml b/.github/workflows/beam_PreCommit_Python_Integration.yml index 5b377f23774e..a4a46815a775 100644 --- a/.github/workflows/beam_PreCommit_Python_Integration.yml +++ b/.github/workflows/beam_PreCommit_Python_Integration.yml @@ -46,7 +46,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Python_PVR_Flink.yml b/.github/workflows/beam_PreCommit_Python_PVR_Flink.yml index c268b4ed78f7..6649958b4791 100644 --- a/.github/workflows/beam_PreCommit_Python_PVR_Flink.yml +++ b/.github/workflows/beam_PreCommit_Python_PVR_Flink.yml @@ -68,7 +68,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Python_Runners.yml b/.github/workflows/beam_PreCommit_Python_Runners.yml index f823112e23dd..a76b04be8fa2 100644 --- a/.github/workflows/beam_PreCommit_Python_Runners.yml +++ b/.github/workflows/beam_PreCommit_Python_Runners.yml @@ -46,7 +46,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Python_Transforms.yml b/.github/workflows/beam_PreCommit_Python_Transforms.yml index 7374af7f38c9..e82fec205d83 100644 --- a/.github/workflows/beam_PreCommit_Python_Transforms.yml +++ b/.github/workflows/beam_PreCommit_Python_Transforms.yml @@ -46,7 +46,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_RAT.yml b/.github/workflows/beam_PreCommit_RAT.yml index c129fc12114f..314a0cf55b53 100644 --- a/.github/workflows/beam_PreCommit_RAT.yml +++ b/.github/workflows/beam_PreCommit_RAT.yml @@ -45,7 +45,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_SQL.yml b/.github/workflows/beam_PreCommit_SQL.yml index ee9799346f1f..3e32c9653ce5 100644 --- a/.github/workflows/beam_PreCommit_SQL.yml +++ b/.github/workflows/beam_PreCommit_SQL.yml @@ -31,7 +31,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true # Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_SQL_Java11.yml b/.github/workflows/beam_PreCommit_SQL_Java11.yml index f1c733418b8d..1f22c71cb920 100644 --- a/.github/workflows/beam_PreCommit_SQL_Java11.yml +++ b/.github/workflows/beam_PreCommit_SQL_Java11.yml @@ -31,7 +31,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_SQL_Java17.yml b/.github/workflows/beam_PreCommit_SQL_Java17.yml index 7547bd396815..3e216f9cd4c8 100644 --- a/.github/workflows/beam_PreCommit_SQL_Java17.yml +++ b/.github/workflows/beam_PreCommit_SQL_Java17.yml @@ -47,7 +47,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Spotless.yml b/.github/workflows/beam_PreCommit_Spotless.yml index 394f627adbab..855c9b75d345 100644 --- a/.github/workflows/beam_PreCommit_Spotless.yml +++ b/.github/workflows/beam_PreCommit_Spotless.yml @@ -43,7 +43,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_Typescript.yml b/.github/workflows/beam_PreCommit_Typescript.yml index a2df0033f98b..70ea8bf9963a 100644 --- a/.github/workflows/beam_PreCommit_Typescript.yml +++ b/.github/workflows/beam_PreCommit_Typescript.yml @@ -33,7 +33,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Website.yml b/.github/workflows/beam_PreCommit_Website.yml index 4ae4e1a99a0c..b268ed6668ef 100644 --- a/.github/workflows/beam_PreCommit_Website.yml +++ b/.github/workflows/beam_PreCommit_Website.yml @@ -47,7 +47,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Website_Stage_GCS.yml b/.github/workflows/beam_PreCommit_Website_Stage_GCS.yml index 00cda54912ac..88e02f0525ae 100644 --- a/.github/workflows/beam_PreCommit_Website_Stage_GCS.yml +++ b/.github/workflows/beam_PreCommit_Website_Stage_GCS.yml @@ -31,7 +31,7 @@ on: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PreCommit_Whitespace.yml b/.github/workflows/beam_PreCommit_Whitespace.yml index 065c5cc7fd8f..5f430779eef9 100644 --- a/.github/workflows/beam_PreCommit_Whitespace.yml +++ b/.github/workflows/beam_PreCommit_Whitespace.yml @@ -46,7 +46,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_Prober_CommunityMetrics.yml b/.github/workflows/beam_Prober_CommunityMetrics.yml index 5095f8d51737..d22b7db8309f 100644 --- a/.github/workflows/beam_Prober_CommunityMetrics.yml +++ b/.github/workflows/beam_Prober_CommunityMetrics.yml @@ -40,7 +40,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_Publish_Docker_Snapshots.yml b/.github/workflows/beam_Publish_Docker_Snapshots.yml index 3b6e996a87a5..ce42829598b2 100644 --- a/.github/workflows/beam_Publish_Docker_Snapshots.yml +++ b/.github/workflows/beam_Publish_Docker_Snapshots.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml b/.github/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml index ce1d824df6e0..c6352a102fc0 100644 --- a/.github/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml +++ b/.github/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml @@ -42,7 +42,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: From 9676fccc6e2c4b126832366cee155e9201ccad67 Mon Sep 17 00:00:00 2001 From: Anand Inguva <34158215+AnandInguva@users.noreply.github.com> Date: Mon, 16 Oct 2023 19:43:11 +0000 Subject: [PATCH 151/435] Change the if condition to check if build is installed (#28992) * Change how we look for build module * use try except instead of module check --- sdks/python/apache_beam/runners/portability/stager.py | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/sdks/python/apache_beam/runners/portability/stager.py b/sdks/python/apache_beam/runners/portability/stager.py index 4afe5eaaa370..d59b3e32bc17 100644 --- a/sdks/python/apache_beam/runners/portability/stager.py +++ b/sdks/python/apache_beam/runners/portability/stager.py @@ -49,7 +49,6 @@ import glob import hashlib -import importlib.util import logging import os import shutil @@ -774,7 +773,7 @@ def _build_setup_package(setup_file, # type: str if build_setup_args is None: # if build is installed in the user env, use it to # build the sdist else fallback to legacy setup.py sdist call. - if importlib.util.find_spec('build'): + try: build_setup_args = [ Stager._get_python_executable(), '-m', @@ -784,7 +783,9 @@ def _build_setup_package(setup_file, # type: str temp_dir, os.path.dirname(setup_file), ] - else: + _LOGGER.info('Executing command: %s', build_setup_args) + processes.check_output(build_setup_args) + except RuntimeError: build_setup_args = [ Stager._get_python_executable(), os.path.basename(setup_file), @@ -792,8 +793,8 @@ def _build_setup_package(setup_file, # type: str '--dist-dir', temp_dir ] - _LOGGER.info('Executing command: %s', build_setup_args) - processes.check_output(build_setup_args) + _LOGGER.info('Executing command: %s', build_setup_args) + processes.check_output(build_setup_args) output_files = glob.glob(os.path.join(temp_dir, '*.tar.gz')) if not output_files: raise RuntimeError( From d38fdd8e7d075effca51980b4d7dfb47882dc47e Mon Sep 17 00:00:00 2001 From: Anand Inguva Date: Mon, 16 Oct 2023 16:04:33 -0400 Subject: [PATCH 152/435] Add no-cache-dir to tox install --- .../groovy/org/apache/beam/gradle/BeamModulePlugin.groovy | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index 3cea900278a8..85d4e3de65a1 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -2963,9 +2963,12 @@ class BeamModulePlugin implements Plugin { } project.exec { executable 'sh' + // TODO: https://github.com/apache/beam/issues/29022 + // pip 23.3 is failing due to Hash mismatch between expected SHA of the packaged and actual SHA. + // until it is resolved on pip's side, don't use pip's cache. args '-c', ". ${project.ext.envdir}/bin/activate && " + - "pip install --pre --retries 10 --upgrade pip && " + - "pip install --pre --retries 10 --upgrade tox" + "pip install --pre --retries 10 --upgrade pip --no-cache-dir && " + + "pip install --pre --retries 10 --upgrade tox --no-cache-dir" } } // Gradle will delete outputs whenever it thinks they are stale. Putting a From 75508e5a92c0aa6203fe60494524a56024b52a97 Mon Sep 17 00:00:00 2001 From: Anand Inguva <34158215+AnandInguva@users.noreply.github.com> Date: Mon, 16 Oct 2023 20:15:15 +0000 Subject: [PATCH 153/435] Revert "Add no-cache-dir to tox install" (#29023) This reverts commit d38fdd8e7d075effca51980b4d7dfb47882dc47e. --- .../groovy/org/apache/beam/gradle/BeamModulePlugin.groovy | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index 85d4e3de65a1..3cea900278a8 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -2963,12 +2963,9 @@ class BeamModulePlugin implements Plugin { } project.exec { executable 'sh' - // TODO: https://github.com/apache/beam/issues/29022 - // pip 23.3 is failing due to Hash mismatch between expected SHA of the packaged and actual SHA. - // until it is resolved on pip's side, don't use pip's cache. args '-c', ". ${project.ext.envdir}/bin/activate && " + - "pip install --pre --retries 10 --upgrade pip --no-cache-dir && " + - "pip install --pre --retries 10 --upgrade tox --no-cache-dir" + "pip install --pre --retries 10 --upgrade pip && " + + "pip install --pre --retries 10 --upgrade tox" } } // Gradle will delete outputs whenever it thinks they are stale. Putting a From 1b50ebbe157d205f015a908ef8a597a7cbbf4a6c Mon Sep 17 00:00:00 2001 From: Danny McCormick Date: Mon, 16 Oct 2023 16:17:34 -0400 Subject: [PATCH 154/435] Fix concurrency groups in Readme (#29021) --- .github/workflows/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/README.md b/.github/workflows/README.md index 8fba73ed8fc3..daf873a11099 100644 --- a/.github/workflows/README.md +++ b/.github/workflows/README.md @@ -119,7 +119,7 @@ Concurrency groups are a way of making sure that no more than one Actions run is ``` concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true ``` From 7df9b24c6c61d8e5b399f1c15590271ad348e50d Mon Sep 17 00:00:00 2001 From: Damon Date: Mon, 16 Oct 2023 13:39:54 -0700 Subject: [PATCH 155/435] [RRIO] [Cache] Stub CacheWrite (#29012) --- .../beam/io/requestresponseio/CacheWrite.java | 119 ++++++++++++++++++ 1 file changed, 119 insertions(+) create mode 100644 sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/CacheWrite.java diff --git a/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/CacheWrite.java b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/CacheWrite.java new file mode 100644 index 000000000000..0fb14af67c35 --- /dev/null +++ b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/CacheWrite.java @@ -0,0 +1,119 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.io.requestresponseio; + +import com.google.auto.value.AutoValue; +import java.util.Map; +import org.apache.beam.io.requestresponseio.CacheWrite.Result; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionTuple; +import org.apache.beam.sdk.values.PInput; +import org.apache.beam.sdk.values.POutput; +import org.apache.beam.sdk.values.PValue; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; + +/** + * {@link CacheWrite} writes associated {@link RequestT} and {@link ResponseT} pairs to a cache. + * Using {@link RequestT} and {@link ResponseT}'s {@link org.apache.beam.sdk.coders.Coder}, this + * transform writes encoded representations of this association. + */ +class CacheWrite + extends PTransform>, Result> { + + private static final TupleTag FAILURE_TAG = new TupleTag() {}; + + // TODO(damondouglas): remove suppress warnings after configuration is used. + @SuppressWarnings({"unused"}) + private final Configuration configuration; + + private CacheWrite(Configuration configuration) { + this.configuration = configuration; + } + + /** Configuration details for {@link CacheWrite}. */ + @AutoValue + abstract static class Configuration { + + static Builder builder() { + return new AutoValue_CacheWrite_Configuration.Builder<>(); + } + + abstract Builder toBuilder(); + + @AutoValue.Builder + abstract static class Builder { + + abstract Configuration build(); + } + } + + @Override + public Result expand(PCollection> input) { + return Result.of( + new TupleTag>() {}, PCollectionTuple.empty(input.getPipeline())); + } + + /** The {@link Result} of writing a request/response {@link KV} {@link PCollection}. */ + static class Result implements POutput { + + static Result of( + TupleTag> responseTag, PCollectionTuple pct) { + return new Result<>(responseTag, pct); + } + + private final Pipeline pipeline; + private final TupleTag> responseTag; + private final PCollection> responses; + private final PCollection failures; + + private Result(TupleTag> responseTag, PCollectionTuple pct) { + this.pipeline = pct.getPipeline(); + this.responseTag = responseTag; + this.responses = pct.get(responseTag); + this.failures = pct.get(FAILURE_TAG); + } + + public PCollection> getResponses() { + return responses; + } + + public PCollection getFailures() { + return failures; + } + + @Override + public Pipeline getPipeline() { + return this.pipeline; + } + + @Override + public Map, PValue> expand() { + return ImmutableMap.of( + responseTag, responses, + FAILURE_TAG, failures); + } + + @Override + public void finishSpecifyingOutput( + String transformName, PInput input, PTransform transform) {} + } +} From a6917d410d33ab794daba7799a3da35aa9e50664 Mon Sep 17 00:00:00 2001 From: Anand Inguva <34158215+AnandInguva@users.noreply.github.com> Date: Mon, 16 Oct 2023 21:55:47 +0000 Subject: [PATCH 156/435] Add no-cache-dir to tox install (#29024) * Add no-cache-dir to tox install * Temp change --- .../groovy/org/apache/beam/gradle/BeamModulePlugin.groovy | 7 +++++-- sdks/python/apache_beam/ml/transforms/base.py | 2 +- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index 3cea900278a8..85d4e3de65a1 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -2963,9 +2963,12 @@ class BeamModulePlugin implements Plugin { } project.exec { executable 'sh' + // TODO: https://github.com/apache/beam/issues/29022 + // pip 23.3 is failing due to Hash mismatch between expected SHA of the packaged and actual SHA. + // until it is resolved on pip's side, don't use pip's cache. args '-c', ". ${project.ext.envdir}/bin/activate && " + - "pip install --pre --retries 10 --upgrade pip && " + - "pip install --pre --retries 10 --upgrade tox" + "pip install --pre --retries 10 --upgrade pip --no-cache-dir && " + + "pip install --pre --retries 10 --upgrade tox --no-cache-dir" } } // Gradle will delete outputs whenever it thinks they are stale. Putting a diff --git a/sdks/python/apache_beam/ml/transforms/base.py b/sdks/python/apache_beam/ml/transforms/base.py index 49ce6e9ec1e0..a0bc4a906100 100644 --- a/sdks/python/apache_beam/ml/transforms/base.py +++ b/sdks/python/apache_beam/ml/transforms/base.py @@ -33,8 +33,8 @@ TransformedMetadataT = TypeVar('TransformedMetadataT') # Input/Output types to the MLTransform. -ExampleT = TypeVar('ExampleT') MLTransformOutputT = TypeVar('MLTransformOutputT') +ExampleT = TypeVar('ExampleT') # Input to the apply() method of BaseOperation. OperationInputT = TypeVar('OperationInputT') From d32a8e55183d98517a381ed472d755132de80536 Mon Sep 17 00:00:00 2001 From: Rebecca Szper <98840847+rszper@users.noreply.github.com> Date: Mon, 16 Oct 2023 19:19:50 -0700 Subject: [PATCH 157/435] Update the Beam ML readme (#29031) --- examples/notebooks/beam-ml/README.md | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/examples/notebooks/beam-ml/README.md b/examples/notebooks/beam-ml/README.md index a1fe7ab19f51..0ae937e9e284 100644 --- a/examples/notebooks/beam-ml/README.md +++ b/examples/notebooks/beam-ml/README.md @@ -57,6 +57,7 @@ This section contains the following example notebooks. * [Apache Beam RunInference with Hugging Face](https://github.com/apache/beam/blob/master/examples/notebooks/beam-ml/run_inference_huggingface.ipynb) * [Apache Beam RunInference with XGBoost](https://github.com/apache/beam/blob/master/examples/notebooks/beam-ml/run_inference_xgboost.ipynb) * [Use RunInference with TFX](https://github.com/apache/beam/blob/master/examples/notebooks/beam-ml/run_inference_tensorflow_with_tfx.ipynb) +* [Use RunInference with a remotely deployed Vertex AI endpoint](https://github.com/apache/beam/blob/master/examples/notebooks/beam-ml/run_inference_vertex_ai.ipynb) * [Use RunInference in Apache Beam](https://github.com/apache/beam/blob/master/examples/notebooks/beam-ml/run_inference_pytorch_tensorflow_sklearn.ipynb) * [Use RunInference with a LLM](https://github.com/apache/beam/blob/master/examples/notebooks/beam-ml/run_inference_generative_ai.ipynb) * [Use RunInference with Beam's windowing semantics](https://github.com/apache/beam/blob/master/examples/notebooks/beam-ml/run_inference_windowing.ipynb) @@ -67,8 +68,10 @@ This section contains the following example notebooks. * [Remote inference](https://github.com/apache/beam/blob/master/examples/notebooks/beam-ml/custom_remote_inference.ipynb) ### Machine Learning Use Cases -* [Image Processing with Apache Beam](https://github.com/apache/beam/blob/master/examples/notebooks/beam-ml/image_processing_tensorflow.ipynb) -* [Natural Language Processing with Apache Beam](https://github.com/apache/beam/blob/master/examples/notebooks/beam-ml/nlp_tensorflow_streaming.ipynb) + +* [Image processing with Apache Beam](https://github.com/apache/beam/blob/master/examples/notebooks/beam-ml/image_processing_tensorflow.ipynb) +* [Natural language processing with Apache Beam](https://github.com/apache/beam/blob/master/examples/notebooks/beam-ml/nlp_tensorflow_streaming.ipynb) +* [Speech emotion recognition with Apache Beam](https://github.com/apache/beam/blob/master/examples/notebooks/beam-ml/speech_emotion_tensorflow.ipynb) ### Automatic Model Refresh @@ -77,7 +80,7 @@ This section contains the following example notebooks. ### Multi-model pipelines * [Ensemble model using an image captioning and ranking](https://github.com/apache/beam/blob/master/examples/notebooks/beam-ml/run_inference_multi_model.ipynb) -* [Run ML Inference with Different Models Per Key](https://github.com/apache/beam/blob/master/examples/notebooks/beam-ml/per_key_models.ipynb) +* [Run ML inference with multiple differently-trained models](https://github.com/apache/beam/blob/master/examples/notebooks/beam-ml/per_key_models.ipynb) ### Model Evaluation @@ -85,4 +88,5 @@ This section contains the following example notebooks. ### Data processing +* [Preprocess data with MLTransform](https://github.com/apache/beam/blob/master/examples/notebooks/beam-ml/mltransform_basic.ipynb) * [Preprocessing with the Apache Beam DataFrames API](https://github.com/apache/beam/blob/master/examples/notebooks/beam-ml/dataframe_api_preprocessing.ipynb) From f8ca45ebb7b455f6fa31943347718ccffa6a5334 Mon Sep 17 00:00:00 2001 From: Bulat Date: Fri, 13 Oct 2023 18:45:13 +0300 Subject: [PATCH 158/435] [Website] add linkedin case-study --- .../site/content/en/case-studies/linkedin.md | 284 +++++++++++++++++- website/www/site/data/en/quotes.yaml | 5 + .../case-study/linkedin/bingfeng-xia.jpg | Bin 0 -> 99417 bytes .../images/case-study/linkedin/scheme-1.png | Bin 0 -> 72662 bytes .../images/case-study/linkedin/scheme-2.png | Bin 0 -> 91660 bytes .../images/case-study/linkedin/scheme-3.png | Bin 0 -> 207951 bytes .../images/case-study/linkedin/scheme-4.png | Bin 0 -> 98569 bytes .../images/case-study/linkedin/scheme-5.png | Bin 0 -> 116720 bytes .../images/case-study/linkedin/scheme-6.png | Bin 0 -> 26758 bytes .../images/case-study/linkedin/xinyu-liu.jpg | Bin 0 -> 100350 bytes 10 files changed, 285 insertions(+), 4 deletions(-) create mode 100644 website/www/site/static/images/case-study/linkedin/bingfeng-xia.jpg create mode 100644 website/www/site/static/images/case-study/linkedin/scheme-1.png create mode 100644 website/www/site/static/images/case-study/linkedin/scheme-2.png create mode 100644 website/www/site/static/images/case-study/linkedin/scheme-3.png create mode 100644 website/www/site/static/images/case-study/linkedin/scheme-4.png create mode 100644 website/www/site/static/images/case-study/linkedin/scheme-5.png create mode 100644 website/www/site/static/images/case-study/linkedin/scheme-6.png create mode 100644 website/www/site/static/images/case-study/linkedin/xinyu-liu.jpg diff --git a/website/www/site/content/en/case-studies/linkedin.md b/website/www/site/content/en/case-studies/linkedin.md index 5ed8b65a5578..b16c1f162157 100644 --- a/website/www/site/content/en/case-studies/linkedin.md +++ b/website/www/site/content/en/case-studies/linkedin.md @@ -1,8 +1,17 @@ --- -title: "Linkedin" -icon: /images/logos/powered-by/linkedin.png -hasNav: true -hasLink: "https://www.youtube.com/watch?v=rBfwjbrMJTE&list=PL4dEBWmGSIU9OkXQU2OAXmITPLhiMSPRp&index=33" +title: "Revolutionizing Real-Time Stream Processing: 4 Trillion Events Daily at LinkedIn " +name: "LinkedIn" +icon: "/images/logos/powered-by/linkedin.png" +category: "study" +cardTitle: "Revolutionizing Real-Time Stream Processing: 4 Trillion Events Daily at LinkedIn" +cardDescription: "Apache Beam serves as the backbone of LinkedIn's streaming infrastructure, handling the near real-time processing of an astounding 4 trillion events daily through 3,000+ pipelines and thus powering personalized experiences for LinkedIn’s vast network of over 950 million members worldwide. The adoption of Apache Beam brought about a series of impressive enhancements, including 2x cost optimization depending on the use case, an astounding acceleration from days to minutes in labeling abuse, and more than 6% improvement in detecting logged-in scrapping profiles." +authorName: "Bingfeng Xia" +coauthorName: "Xinyu Liu" +authorPosition: "Engineering Manager @LinkedIn" +coauthorPosition: "Senior Staff Engineer @LinkedIn" +authorImg: /images/case-study/linkedin/bingfeng-xia.jpg +coauthorImg: /images/case-study/linkedin/xinyu-liu.jpg +publishDate: 2023-08-10T00:12:00+00:00 --- +
+
+ +
+
+

+ “Apache Beam empowers LinkedIn to create timely recommendations and personalized experiences by leveraging the freshest data and processing it in real-time, ultimately benefiting LinkedIn's vast network of over 950 million members worldwide.” +

+
+
+ +
+
+
+ Bingfeng Xia +
+
+ Engineering Manager @LinkedIn +
+
+
+
+
+
+ +# Revolutionizing Real-Time Stream Processing: 4 Trillion Events Daily at LinkedIn + +## Background + +At LinkedIn, Apache Beam plays a pivotal role in stream processing infrastructures that process over 4 trillion events daily through more than 3,000 pipelines across multiple production data centers. This robust framework empowers near real-time data processing for critical services and platforms, ranging from machine learning and notifications to anti-abuse AI modeling. With over 950 million members, ensuring that our platform is running smoothly is critical to connecting members to opportunities worldwide. + +In this case study, LinkedIn's Bingfeng Xia, Engineering Manager, and Xinyu Liu, Senior Staff Engineer, shed light on how the Apache Beam programming model's unified, portable, and user-friendly data processing framework has enabled a multitude of sophisticated use cases and revolutionized Stream Processing at LinkedIn. This technology has [optimized cost-to-serve by 2x](https://engineering.linkedin.com/blog/2023/unified-streaming-and-batch-pipelines-at-linkedin--reducing-proc) by unifying stream and batch processing through Apache Samza and Apache Spark runners, enabled real-time ML feature generation, reduced time-to-production for new pipelines from months to days, allowed for processing time-series events at over 3 million queries per second, and more. For our members, this means that we’re able to serve more accurate job recommendations, improve feed recommendations, and identify fake profiles at a faster rate, etc. + + +## LinkedIn Open-Source Ecosystem and Journey to Beam + +LinkedIn has a rich history of actively contributing to the open-source community, demonstrating its commitment by creating, managing, and utilizing various open-source software projects. The LinkedIn engineering team has [open-sourced over 75 projects](https://engineering.linkedin.com/content/engineering/en-us/open-source) across multiple categories, with several gaining widespread adoption and becoming part of [the Apache Software Foundation](https://www.apache.org/). + +To enable the ingestion and real-time processing of enormous volumes of data, LinkedIn built a custom stream processing ecosystem largely with tools developed in-house (and subsequently open-sourced). In 2010, they introduced [Apache Kafka](https://kafka.apache.org/), a pivotal Big Data ingestion backbone for LinkedIn’s real-time infrastructure. To transition from batch-oriented processing and respond to Kafka events within minutes or seconds, they built an in-house distributed event streaming framework, [Apache Samza](https://samza.apache.org/). This framework, along with Apache Spark for batch processing, formed the basis of LinkedIn’s [lambda architecture](https://en.wikipedia.org/wiki/Lambda_architecture) for data processing jobs. Over time, LinkedIn's engineering team expanded the stream processing ecosystem with more proprietary tools like [Brooklin](https://github.com/linkedin/Brooklin/), facilitating data streaming across multiple stores and messaging systems, and [Venice](https://github.com/linkedin/venice), serving as a storage system for ingesting batch and stream processing job outputs, among others. + +Though the stream processing ecosystem with Apache Samza at its core enabled large-scale stateful data processing, LinkedIn’s ever-evolving demands required higher scalability and efficiency, as well as lower latency for the streaming pipelines. The lambda architecture approach led to operational complexity and inefficiencies, because it required maintaining two different codebases and two different engines for batch and streaming data. To address these challenges, data engineers sought a higher level of stream processing abstraction and out-of-the-box support for advanced aggregations and transformations. Additionally, they needed the ability to experiment with streaming pipelines in batch mode. There was also a growing need for multi-language support within the overall Java-prevalent teams due to emerging machine learning use cases requiring Python. + +The release of [Apache Beam](/about/) in 2016 proved to be a game-changer for LinkedIn. Apache Beam offers an open-source, advanced unified programming model for both batch and Stream Processing, making it possible to create a large-scale common data infrastructure across various applications. With support for Python, Go, and Java SDKs and a rich, versatile API layer, Apache Beam provided the ideal solution for building sophisticated multi-language pipelines and running them on any engine. + +
+

+ When we started looking at Apache Beam, we realized it was a very attractive data processing framework for LinkedIn’s demands: not only does it provide an advanced API, but it also allows for converging stream and batch processing and multi-language support. Everything we were looking for and out-of-the-box. +

+
+
+ +
+
+
+ Xinyu Liu +
+
+ Senior Staff Engineer @LinkedIn +
+
+
+
+ +Recognizing the advantages of Apache Beam's unified data processing API, advanced capabilities, and multi-language support, LinkedIn began onboarding its first use cases and developed the [Apache Samza runner for Beam](/documentation/runners/samza/) in 2018. By 2019, Apache Beam pipelines were powering several critical use cases, and the programming model and framework saw extensive adoption across LinkedIn teams. Xinyu Liu showcased the benefits of migrating to Apache Beam pipelines during [Beam Summit Europe 2019](https://www.youtube.com/watch?v=uQcpr34RUKY&t=1694s). + +
+ + scheme + +
+ +## Apache Beam Use Cases at LinkedIn + +### Unified Streaming And Batch Pipelines + +Some of the first use cases that LinkedIn migrated to Apache Beam pipelines involved both real-time computations and periodic backfilling. One example was LinkedIn's standardization process. Standardization consists of a series of pipelines that use complex AI models to map LinkedIn user inputs, such as job titles, skills, or education history, into predefined internal IDs. For example, a LinkedIn member who lists their current position as "Chief Data Scientist" has their job title standardized for relevant job recommendations. + + +LinkedIn's standardization process requires both real-time processing to reflect immediate user updates and periodic backfilling to refresh data when new AI models are introduced. Before adopting Apache Beam, running backfilling as a streaming job required over 5,000 GB-hours in memory and nearly 4,000 hours in total CPU time. This heavy load led to extended backfilling times and scaling issues, causing the backfilling pipeline to act as a "noisy neighbor" to colocated streaming pipelines and failing to meet latency and throughput requirements. Although LinkedIn engineers considered migrating the backfilling logic to a batch Spark pipeline, they abandoned the idea due to the unnecessary overhead of maintaining two different codebases. + +
+

+ We came to the question: is it possible to only maintain one codebase but with the ability to run it as either a batch job or streaming job? The unified Apache Beam model was the solution. +

+
+
+ +
+
+
+ Bingfeng Xia +
+
+ Engineering Manager @LinkedIn +
+
+
+
+ +The Apache Beam APIs enabled LinkedIn engineers to implement business logic once within a unified Apache Beam pipeline that efficiently handles both real-time standardization and backfilling. Apache Beam offers [PipelineOptions](https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/options/PipelineOptions.html), enabling the configuration and customization of various aspects, such as the pipeline runner and runner-specific configurations. The extensibility of Apache Beam transforms allowed LinkedIn to [create a custom composite transform](https://beam.apache.org/documentation/programming-guide/#composite-transforms) to abstract away I/O differences and switch target processing on the fly based on data source type (bounded or unbounded). In addition, Apache Beam’s abstraction of the underlying infrastructure and the ability to "write once, run anywhere" empowered LinkedIn to seamlessly switch between data processing engines. Depending on the target processing type, streaming, or batch, the unified Apache Beam standardization pipeline can be deployed through the Samza cluster as a streaming job or through the Spark cluster as a batch backfilling job. + +
+ + scheme + +
+ +Hundreds of streaming Apache Beam jobs now power real-time standardization, listening to events 24/7, enriching streams with additional data from remote tables, performing necessary processing, and writing results to output databases. The batch Apache Beam backfilling job runs weekly, effectively handling 950 million member profiles at a rate of over 40,000 profiles per second. Apache Beam infers data points into sophisticated AI and machine learning models and joins complex data such as job types and work experiences, thus standardizing user data for search indexing or to run recommendation models. + +The migration of backfilling logic to a unified Apache Beam pipeline and its execution in batch mode resulted in a significant 50% improvement in memory and CPU usage efficiency (from ~5000 GB-hours and ~4000 CPU hours to ~2000 GB-hours and ~1700 CPU hours) and an impressive 94% acceleration in processing time (from 7.5 hours to 25 minutes). More details about this use case can be found on [LinkedIn’s engineering blog](https://engineering.linkedin.com/blog/2023/unified-streaming-and-batch-pipelines-at-linkedin--reducing-proc). + +### Anti-Abuse & Near Real-Time AI Modeling + +LinkedIn is firmly committed to creating a trusted environment for its members, and this dedication extends to safeguarding against various types of abuse on the platform. To achieve this, the Anti-Abuse AI Team at LinkedIn plays a crucial role in creating, deploying, and maintaining AI and deep learning models that can detect and prevent different forms of abuse, such as fake account creation, member profile scraping, automated spam, and account takeovers. + +Apache Beam fortifies LinkedIn’s internal anti-abuse platform, Chronos, enabling abuse detection and prevention in near real-time. Chronos relies on two streaming Apache Beam pipelines: the Filter pipeline and the Model pipeline. The Filter pipeline reads user activity events from Kafka, extracts relevant fields, aggregates and filters the events, and then generates filtered Kafka messages for downstream AI processing. Subsequently, the Model pipeline consumes these filtered messages, aggregates member activity within specific time windows, triggers AI scoring models, and writes the resulting abuse scores to various internal applications, services, and stores for offline processing. + +
+ + scheme + +
+ +The flexibility of Apache Beam's pluggable architecture and the availability of various I/O options seamlessly integrated the anti-abuse pipelines with Kafka and key-value stores. LinkedIn has dramatically reduced the time it takes to label abusive actions, cutting it down from 1 day to just 5 minutes and processing time-series events at an impressive rate of over 3 million queries per second. Apache Beam empowered near real-time processing, significantly bolstering LinkedIn's anti-abuse defenses. The nearline defenses are able to catch scrapers within minutes after they start to scrape and this leads to more than 6% improvement in detecting logged-in scrapping profiles. + +
+

+ Apache Beam enabled revolutionary, phenomenal performance improvements - the anti-abuse processing accelerated from 1 day to 5 minutes. We have seen more than 6% improvement in detecting logged-in scrapping profiles. +

+
+
+ +
+
+
+ Xinyu Liu +
+
+ Senior Staff Engineer @LinkedIn +
+
+
+
+ +### Notifications Platform + +As a social media network, LinkedIn heavily relies on instant notifications to drive member engagement. To achieve this, Apache Beam and Apache Samza together power LinkedIn’s large-scale Notifications Platform that generates notification content, pinpoints the target audience, and ensures the timely and relevant distribution of content. + +The streaming Apache Beam pipelines have intricate business logic and handle enormous volumes of data in a near real-time fashion. The pipelines consume, aggregate, partition, and process events from over 950 million LinkedIn members and feed the data to downstream machine learning models. The ML models perform distributed targeting and scalable scoring on the order of millions of candidate notifications per second based on the recipient member’s historical actions and make personalized decisions for the recipient for each notification on the fly. As a result, LinkedIn members receive timely, relevant, and actionable activity-based notifications, such as connection invites, job recommendations, daily news digests, and other activities within their social network, through the right channels. + +The advanced Apache Beam API offers complex aggregation and filtering capabilities out-of-the-box, and its programming model allows for the creation of reusable components. These features enable LinkedIn to expedite development and streamline the scaling of the Notifications platform as they transition more notification use cases from Samza to Beam pipelines. + +
+

+ LinkedIn’s user engagement is greatly driven by how timely we can send relevant notifications. Apache Beam enabled a scalable, near real-time infrastructure behind this business-critical use case. +

+
+
+ +
+
+
+ Bingfeng Xia +
+
+ Engineering Manager @LinkedIn +
+
+
+
+ +### Real-Time ML Feature Generation + +LinkedIn's core functionalities, such as job recommendations and search feed, heavily rely on ML models that consume thousands of features related to various entities like companies, job postings, and members. However, before the adoption of Apache Beam, the original offline ML feature generation pipeline suffered from a delay of 24 to 48 hours between member actions and the impact of those actions on the recommendation system. This delay resulted in missed opportunities, because the system lacked sufficient data about infrequent members and failed to capture the short-term intent and preferences of frequent members. In response to the growing demand for a scalable, real-time ML feature generation platform, LinkedIn turned to Apache Beam to address the challenge. + +Using Managed Beam as the foundation, LinkedIn developed a hosted platform for ML feature generation. The ML platform provides AI engineers with real-time features and an efficient pipeline authoring experience, all while abstracting away deployment and operational complexities. AI engineers create feature definitions and deploy them using Managed Beam. When LinkedIn members take actions on the platform, the streaming Apache Beam pipeline generates fresher machine learning features by filtering, processing, and aggregating the events emitted to Kafka in real-time and writes them to the feature store. Additionally, LinkedIn introduced other Apache Beam pipelines responsible for retrieving the data from the feature store, processing it, and feeding it into the recommendation system. + +
+ + scheme + +
+ +The powerful Apache Beam Stream Processing platform played a pivotal role in eliminating the delay between member actions and data availability, achieving an impressive end-to-end pipeline latency of just a few seconds. This significant improvement allowed LinkedIn's ML models to take advantage of up-to-date information and deliver more personalized and timely recommendations to our members, leading to significant gains in business metrics. + +### Managed Stream Processing Platform + +As LinkedIn's data infrastructure grew to encompass over 3,000 Apache Beam pipelines, catering to a diverse range of business use cases, LinkedIn's AI and data engineering teams found themselves overwhelmed with managing these streaming applications 24/7. The AI engineers encountered several technical challenges while creating new pipelines, including the intricacy of integrating multiple streaming tools and infrastructures into their frameworks, and limited knowledge of the underlying infrastructure when it came to deployment, monitoring, and operations. These challenges led to a time-consuming pipeline development cycle, often lasting one to two months. Apache Beam enabled LinkedIn to create Managed Beam, a managed Stream Processing platform that is designed to streamline and automate internal processes. This platform makes it easier and faster for teams to develop and operate sophisticated streaming applications while reducing the burden of on-call support. + +
+ + scheme + +
+ +The Apache Beam SDK empowered LinkedIn engineers to create custom workflow components as reusable sub-DAGs (Directed Acyclic Graphs) and expose them as standard PTransforms. These PTransforms serve as ready-to-use building blocks for new pipelines, significantly speeding up the authoring and testing process for LinkedIn AI engineers. By abstracting the low-level details of underlying engines and runtime environments, Apache Beam allows engineers to focus solely on business logic, further accelerating time to development. + +When the pipelines are ready for deployment, Managed Beam's central control plane comes into play, providing essential features like a deployment UI, operational dashboard, administrative tools, and automated pipeline lifecycle management. + +Apache Beam's abstraction facilitated the isolation of user code from framework evolution during build, deployment, and runtime. To ensure the separation of runner processes from user-defined functions (UDFs), Managed Beam packages the pipeline business logic and the framework logic as two separate JAR files: framework-less artifacts and framework artifacts. During pipeline execution on a YARN cluster, these pipeline artifacts run in a Samza container as two distinct processes, communicating through gRPC. This setup enabled LinkedIn to take advantage of automated framework upgrades, scalable UDF execution, log separation for easier troubleshooting, and multi-language APIs, fostering flexibility and efficiency. + +
+ + scheme + +
+ +Apache Beam also underpinned Managed Beam's autosizing controller tool, which automates hardware resource tuning and provides auto-remediation for streaming pipelines. Streaming Apache Beam pipelines self-report diagnostic information, such as metrics and key deployment logs, in the form of Kafka topics. Additionally, LinkedIn's internal monitoring tools report runtime errors, such as heartbeat failures, out-of-memory events, and processing lags. The Apache Beam diagnostics processor pipeline aggregates, repartitions, and windows these diagnostic events before passing them to the autosizing controller and writing them to Apache Pinot, LinkedIn's OLAP store for Managed Beam's operational and analytics dashboards. Based on the pre-processed and time-windowed diagnostic data, the autosizing controller generates sizing actions or restarting actions, and then forwards them to the Managed Beam control plane. The Managed Beam control plane then scales LinkedIn's streaming applications and clusters. + +
+

+ Apache Beam helped streamline operations management and enabled fully-automated autoscaling, significantly reducing the time to onboard new applications. Previously, onboarding required a lot of manual 'trial and error' iterations and deep knowledge of the internal system and metrics. +

+
+
+ +
+
+
+ Bingfeng Xia +
+
+ Engineering Manager @LinkedIn +
+
+
+
+ +The extensibility, pluggability, portability, and abstraction of Apache Beam formed the backbone of LinkedIn's Managed Beam platform. The Managed Beam platform accelerated the time to author, test, and stabilize streaming pipelines from months to days, facilitated fast experimentation, and almost entirely eliminated operational costs for AI engineers. + +## Summary + +Apache Beam played a pivotal role in revolutionizing and scaling LinkedIn's data infrastructure. Beam's powerful streaming capabilities enable real-time processing for critical business use cases, at a scale of over 4 trillion events daily through more than 3,000 pipelines. + +The versatility of Apache Beam empowered LinkedIn’s engineering teams to optimize their data processing for various business use cases: +- Apache Beam's unified and portable framework allowed LinkedIn to consolidate streaming and batch processing into unified pipelines. These unified pipelines resulted in a 2x optimization in cost-to-serve, a 2x improvement in processing performance, and a 2x improvement in memory and CPU usage efficiency. +- LinkedIn's anti-abuse platform leveraged Apache Beam to process user activity events from Kafka in near-real-time, achieving a remarkable acceleration from days to minutes in labeling abusive actions. The nearline defenses are able to catch scrapers within minutes after they start to scrape and this leads to more than 6% improvement in detecting logged-in scrapping profiles. +- By adopting Apache Beam, LinkedIn was able to transition from an offline ML feature generation pipeline with a 24- to 48-hour delay to a real-time platform with an end-to-end pipeline latency at the millisecond or second level. +- Apache Beam’s abstraction and powerful programming model enabled LinkedIn to create a fully managed stream processing platform, thus facilitating easier authoring, testing, and deployment and accelerating time-to-production for new pipelines from months to days. + +Apache Beam boasts seamless plug-and-play capabilities, integrating smoothly with Apache Kafka, Apache Pinot, and other core technologies at LinkedIn, all while ensuring optimal performance at scale. As LinkedIn continues experimenting with new engines and tooling, the Apache Beam portability future-proofs our ecosystem against any changes in the underlying infrastructure. + +
+

+ By enabling a scalable, near real-time infrastructure behind business-critical use cases, Apache Beam empowers LinkedIn to leverage the freshest data and process it in real-time to create timely recommendations and personalized experiences, ultimately benefiting LinkedIn's vast network of over 950 million members worldwide. +

+
+
+ +
+
+
+ Xinyu Liu +
+
+ Senior Staff Engineer @LinkedIn +
+
+
+
+ +

+ +{{< case_study_feedback "LinkedIn" >}} + +
+
diff --git a/website/www/site/data/en/quotes.yaml b/website/www/site/data/en/quotes.yaml index 5cd09eca9076..3a5225b3f29a 100644 --- a/website/www/site/data/en/quotes.yaml +++ b/website/www/site/data/en/quotes.yaml @@ -11,6 +11,11 @@ # limitations under the License. #Cards with quotes will be displayed by the order listed, e.g., first card will display the first quote +- text: Apache Beam fuels LinkedIn's streaming infrastructure, processing 4 trillion events daily through 3K+ pipelines in near-real time. Beam enabled unified pipelines, yielding 2x cost savings and remarkable improvements for many use cases. + icon: icons/quote-icon.svg + logoUrl: images/logos/powered-by/linkedin.png + linkUrl: case-studies/linkedin/index.html + linkText: Learn more - text: With Apache Beam, OCTO accelerated the migration of one of France’s largest grocery retailers to streaming processing for transactional data, achieving 5x reduced infrastructure costs and 4x improved performance. icon: icons/quote-icon.svg logoUrl: images/logos/powered-by/octo.png diff --git a/website/www/site/static/images/case-study/linkedin/bingfeng-xia.jpg b/website/www/site/static/images/case-study/linkedin/bingfeng-xia.jpg new file mode 100644 index 0000000000000000000000000000000000000000..ca07935b689b6f1c3e93fbbb063fba8713ff58c5 GIT binary patch literal 99417 zcmeFacU%+A_dmJ_f(1}idRMX1tDuNVmtH~(MF^po284iuAt;ZcBA_B5AVrYgqy+&1 zD=58&5{imaLQ@GKfsnhw@;uMy`?=-!yRZA#Wnp*DoO9;PoY|S#oz1>?82yZoa)P>- zygdNG!~{470KhhYlLY`+L5hW=13a+&qI*Dk%LdJqrLeI7s?BwCy#mTvgS77g6-o1MTcsMtT8Fmq!CTMgS* zc8+b^ckE>OPazAih4r^^H^9Qi!n%cR%T^BdZChD6l|Ug6E1TpVg{@~TT=(wxy{gE* z@808*a|aIcTE28sQue#nsm&pE-|FkfZ!oM1UwUcSA)Od&cimSR{;K?4kWcXMWq-Zy z#*HoL9e(iSdU@=dp3l_GinqOA7HvENZp38~8%X0zYWlXGf#LDlm5qH9%RIoAEnsQc zHp;}ozEx$T2$BkW*uWC_?%%sr@#?*O%o4n`?EI>9(Cwq&wfovuyfCctHz{^z0XPn+ zyy^lAk-_h-3sLolfBI8||ML)jaEGk^KOF2RabwT3!Hq1P4X&9$7n*?e#mx}f8{cL2_Ig&Xw09ByI>Vh?cbyXQjO3NEv*;rqaqVP+Jy9wv zBDr#iM3<=&i(hkIy?eTF@wp=dD5$l5&L^8&kTn+qE@qIxxSA~35n^1&|FB4 z$hnvVXPA@6@uR#rZnmJ&bTkCD>!dvnjjM#wMk?g&UCe{(c=??*;!ZYL9S}o@vW69|Fz6wdFkunp&Qs+xoZlu6dbr8VtU%e@+F&)PaZLCsjsS{ zqPS!ffe@G__C=RTu3oI{sRPqKDw%rh8+S8XPn~SnzIBzbdUVB$8OaJ{E*r#}abiQj@;P zmkdDAi0`vSAJ)S1lk5wuoV98Y`84ql^{3~pJWGysN)Er@*V*a&{itloxCR`4y&A^4 zsy|><*{6GF_2hBFNTS*Q4RRJsKUOWBD#tR zoj-G`zkM8kY=z`WyQDmp>VsJCem6k5{Z=ghV@hFM6^Woy-!%%)w@v$^*d!Q_dR*+G zy|dP6n;bkUVzAMcW`mFBBRiGUryE4&ed3A=5AR$Esy8c2wGx|9){?Ho$BBeN>v~!+ zSo#i4nLCT(E2*tCgiBiIz58?NtK=xfr=u}?drL1$zHK`Rjg7CQURD{_MUEour)Y5@ z(JRfh#tb08{4-~x`|43%d=?p5pjzXkJ<{6S301{M<7_aiX5Dn&*v_@Soa<(`%~P`{ z{u&57LmIW^s*9oP!adYjW4`XxF2D8n_gBkKx5JvZ z25XvGYjsBy2m1~?*-7^^0JAkQT?X)^xW}zCz|?9cKvjwCr1m*Tu=kYCXFHO|dNCx4 z>_Sam8Hpuo@jWMW_BqvOT1^YYfA~0@jEi%DTx$w_5~MOlQfugQ*XV=MvwNt1Nmk2? zyNrbDC-xjVnJVuxI?yA%7|>ZAZ{?)AiWzB|F6kxcm)6iRT_Lxr zKd1w6seUwPnL?#4B`68Gue@f4h^4u7(C;Ov_R)f5v)6V_@5uZya5!%Ok%B$;t}rZ@ z*c9QNP4Z|;dhlUl04jsPpI-_JR#Ye{A$&Y8Wphpgp?G&}3|*{<)&io}Upl`%S|o>rs$JsiF|UWD00tm}tO{7df8*=4^@;I$@P z)?-!A<_q)OX@6hXzpM_{T!C38tVk~RS=+U7wcg^M(RKEE+F%_;@uDgX9;F(u{-o_b zsifdWW949}M!4}i%0u+j+6& z+c1rJvUqaq+15)XpWfe$*Xp@JLlqA-YObb?c51EdQGzW+8H(}UTgO^xOz0PLJ+wdZ z;dU`EX@z_`6W$ZF-dK*pX}gCz;}M3iM=z;)$naEK z(EtND3$I&0ld7&HW!Dza^?~ymf%l9`dr!dp0B_+8vawvdZJg!hLGIO%=11L|VL@ z#|!Ij8oIy$9Lm;zP_H;Hk2yJ2#BfQqB9leGEmaW-N0r~~yEX+Iy&4-e!#RGsmgs~h zX{J8G4vm^N1qQX?+ZU(vnsOMx`Sx)zS0{+*PB!Xuqz#GdZO`crmtCkN=(M=@)X7=n zx3GmJvd3wcjHTNQz|eBMWQbgZ$H|YoJdl1iTM$={-jyL(RM;>Rtmxu7h1VLFeYdFf zDk#JIRzLu$FHUH}k*O_A-AgJ8W(5ZRx6ak@wL!Sm`QRz!3yB7r- z-CfrssHbaZ)_oVDD9@>tl}mG;X3ahUm+LzhH4llKk_#U%Mwgf0mPIU|ZttY=yY*oi zKuUUE$Gwtl>+2?N;XU4UN($F);L4geS`fv7nbkwt_6*?I;BbFi8TCQ8(`W|W6;eNY zXL?B~r~in`2-(l?7m6|XK~iIxk@rSPq$(nOPv4|tY7S;8GV%DD=a&TB~*7*9k>hS|zH z`M~lJLi3n~l+;|$5*7PrCb<=AhvIG^gpKH#B$Zf_;L6Bo#!BjB$5r-s*5T(mr}=Ay zR*Nz9DAb6^Y$Wdql`8Hnznzt!y%AiAG3w3L#j$SF`T3cwlNpHcW5A^Q*k7FR!gH)}lsXGVAfS$W^t;S-R}o z>pebLTK0MD5g5B><@)W_5kc3|!qrroHpBN6{y+rQR zWx$hpADX;hlv_2DZq03rN3)e)?Czop(FT|B@pQE*h=Yh8>N6KoD(u<)rBz~TX640b z%gF?~c+3KSi!3G0AsvLY9_U4B z4UD85nQ1_G`)-kh3= z!M71SbuR5{Bh!;7lht<2p`A5w7G@Az^{u!sjP z*(@8{Lc0YGw^ z3s;XLq$9Fd|}F3NZju(ozj#u@q4{jw$R5i66FuG`Ku=FT|0W>cibdz086Fu58 zb=flyCxVn9DGQ(2a5p zaN<=*c3cv-tSN*|QAL*s7R35d+kQl?Xcn0_+7TB z;`Gt4DyzE{Fxfdg&bqR%xnu^W21VhJX1ni$VuJ%TzJY0a=6akNeXwRtSW4Tybc%p3 z+k5T-zP;0{ z&^6?t7WLi%n?tH{>GPJ20}1*$^VIsK$Bt7!Uh}SGeL`JmIEh|lb5JPCR+-7?T~w0I zQkntpO`%<@DyezobuODG*wJC%=i$oMhNtQCmx!;t0)Lnk1r7?M=R51EX0&Zw+rJqQ z7+v#P1^}DRWI#>y0x^$zC;wPRzro4K*l?9XUo26LT7)eu?!rt3_ELMp`$-k;?Am7) zPt^}4UL%oSbOBr)Q72O`ddMTQJUv^3-d6T4p&uvRTWU*!73B@)BNBHyjCcDCeygq| zj&_heCuLXFsF(>Y{lah)Vi|YAlJWqe2zy<3)lL_h9FN^OO_#EFdAz8`0Me_A^$C;T z^Jxn=JKB4YdEp(RP9`!3?VOh}YfZ?;O7whMbFGV2&q(1%I*L|9F<YPEMVo z@6ukGKYRI`iWMUut9)nue156N-pI09ubV@kSm?7_X&rR7|h|!B4JR|bO~c|DR&Ky(3ym1FAMIwU&^g2_-4`R(HU@R-`t>13(xW) z^vm=>Ywjm6&Ri=z^(ZR0^IOn$Q&dIyeZ##k6-ovxhgw_L#*6k&%k>VWB;8RI8AG9b z;v3TTR@BsgM|2bPpWQfhBI?X3>Cu8(t=VDzV{x|#S)X1IF4rxE==ipo-s6QHaqrWVL`h^Eu}<~v^~#!FG^7pR-WXU<8l~0$kn38NaU64^ zsYZ=ypEh#I`q&%Ob0){xv5T@IU7#{s7A##6>r6?Ss$Zbg^bSlRsdKi?DQdX7;?Ebv zvTt0JtsZysFuBb4JbcgAxxmsU#L_4wQ2{Gk!H91`n}ust=aN%rk%! zMQ)!z4LF}ZbX?p9a*pJ*bYI9$goF?4%xziHXu0Y_Mi#I4hE!k{ZmY`**KoHN_FDCR z7oC2tFzXxAiGR5CtnN-hu~|1w&{`|g9en2FN9T-Qg-~1)Vj^^F^{RIsZ{q7Ts@2~@ zOK|dQ9I~$~F8lOJu|_cV?sUHA!bm%`=jrR ze!9Y>(3UlW(LE~&3$IEyyeYnACj*#&7pvjTosigkO9a2ioD%HN<-!$^$TLJp{*Zag zewB;28d@WIC-7$CYf*YulYH#`Q(F!x?~3tnOZK!5)Sn`SS=o2Zm*$5*)1t0i=B>e0 zjAm4<$5s$t<}QX@Up*;0-VljKp@PLesn!#^P^o9zyw(;gJD$qCppCTMgBL#cuZu@P zhDbBAh?zxt?_5aZd{Z!_xEMw5JR)A7WaU|RZ9>X#FKhad-N97?qR^8w_Ih8>w!E&L zrwUD2FO`s}jrQ?Tfi$(CSb|x0H%#+<-aAJdi^w`Q@fXFqQ*>?a;+(jiWpVpn_}G26 zx5UxF$59O6Yb2`P3r2UDWB^mrBhMaj9!SFGA!%fz+^9K@9vp#$Hzs7SmA|{o*$NS_ zxPRxF?%d+2`Ed_^qjxnqr=M%g28%-y_!Nn?e-Tbv&0?W2Ev12#X)7(cY}~Gn-Urji zdV-#{RLtI5MiCPWa=}a7#R-kWI&W(zm6m9@Oi6$8`MZtx<_a@^RJbQLOe(mIwt4w!)U=+idkn#bi`l! zPia;B{_DBj1heQKKfHQ(Z{vFyZf$sJtR(fl!|>t4fkbgTEDG!9WX~-!Eh9EwaM(XA zvd1a@I%zA!?gd^0SwZ5zfgD`6r&LhMn5lwAyH%~1k~r=MYbn{?3Pr&QNpatuJWhqH zW|dJ_QCIAzf<{|{Mq3Y_(eU+mgE5$yD>#*oyg)1toh2lk2VoI9P>yfbRv^mV0YE>u&s9#cBn zN%-q*Gp#D}UhLGwC|0ab!?6J~46O?jb}9;Jv|Udr54}mj@s}DVu!q)>7M~>u^eW+b z$A_K6diQC(UP~Ui!vJ~?boSB2f(skpG^654+-re~O*7scuu5AzZg!9XWR>}7sEU)F z;=4yfcoMKeB*pu?#`|uM(GC%8+Jkzh1Y*?+aRq5FWaXP>LTopT6cltkGdLP+?Y}YK(G#eZ$ z?paV;+*OhKxFn9&LPJt(RUR+uKl$|8)lTu^p&{vIC^ULY^|00az*=!fRfPBx*f{y& z&Puxf4`qQs_OO4u;G{IzD`JU~f>fgXbbP zTd^(^TJYRqY3y@lN5pXg75Z>~;yBf1$`wOjNK z5Y`}CCj5D%mZ(gZf~Ec=g?%Qobm0q1zSw;HWv<*ldm6!WlAs-;5?DquxwrqqrF|?@ zhut72&ZUOQR8*)^!o;Xb(Uh#EE4sPDy8_1UO~q_hOo zdo1^@3$GokNw5)99Vs&zsa3b>2ttSmOot^+vyTP~&Q{Ji6U_!S!6#8&uZsH?OJo*N z62pa(jGmc!IK%?g&<;n*`8TqTKb4*CAE4IoKVEe34T?ds^p2RwU9IUUjI64E5?q_w z5)_e1B+B3DpJ1;!>%OgLcF=SbvFL2Sn3wO{=4_*8-#j=I1N9MwIg-NF$x98kMNJg) z$`4Kk(0NElpyYZ%|sUE!IEWqa=$ zveW5|p)WF`Dg#0Ra9hLn{Ku8KDq%BAop+JJ+dKG)SwT^rRb>25B0fD;n_Ab5H{#>w zjf1!nGx%&M!}edXNJIj7pI&2xozA9Qr_0uoc9b|d*&jgUmt+kf=;u=c(4%zSw;%n+ z*1#w2l$bN8^9!wD=kfgMkZUwr8_%H@)5i!ufe)??kE#RsA~oYX(V8vbvnbp3?l=Ko z+70=ZWKp~rk1%u@A6#ceNdZOVib_jV=(J2GAD|>r4f@ zuJSFYO75Gt6=6kh@G&pb5&^N6~PO2tBth8Q2Xj)wbGmjQfEme{Hib1w5~#C^j37xU?%nqdbsaD_GW;EaGn@0*v5#N?2w zo#spHE!z7>&3fpE7o77b)zJE#{@n;>*E9j>bUKGi({kvXXTKwIXX8ud!xpS zcMLTQJaC+iLW>sVc8I8zD>k&EX~D!0&jms|LCNX%_>msnIh>40_ow*KF4^p!mhAex zp_wEuqKJ_|C+yb2xy%~jXW(u4B2RTif&l3=$M~~5&{W%!R^dv=>7kjH06PX?T0L4? zTv{ePQ=gKLH{uGk^%eVm?CO)}YCVm8%ON*tb%`bT1VK?Z@}sYFiHhjpf=2F=8Eoow zT^|gIFaJ`~+tc3N2~i8uY6}v7awcZZ_5u%$wUsxn{0kISuCynf7S|VHk!nn)xIk=4 zU9Cw+2j|a6Kc;DQ(1`XB)x6gp{AWE4n^%>JD-ZM$^WVeo;Cg#)jzejV2s~dO?0XTh zdX6ra9XUBbgS+JRbsu}LE_!M}PpML)#<_1EyNtfP=)9`Ao;sPF5K&rB#pUsR`bfsd z+f0k}ok+|pOo@e5G!C{`jMrlwhRIqF*B$)C8NIXllph$Wlp3SNzM!;<7hGY~q4Hq+ zie@>5#eo=v!uM1!_w~?*hAkupp!y9e9au9^v22&~DK#KCd$RMv{HtEiJyjIJjF)pf z^a?9BV_YpY%aCR1R2F-PtoV=D(|zL}mDzo?i}rUvh^PgXBl8ghFg4Iw*`$-x9EO2L z6!xOph|4Duwp0gN_wfrApit4i>o8Otsj+eO9ax@m0<&0y%t@HZ%BcYelP!`=Y@KaG68!J-6%YK;R3l* zyoateIb4;JrIpP9wmtPZyUe|szn1S;S(^$GYUuw4{g%&Np#Ji&B%yiiVOSg=#{|Kl zHssg{FWSyV2W&F6?4WnH1qU zScTn3B#x5gnxq~ExK)ZZ?~;7`=DQ+HH*E+=joJVK-uvv?ho(ffAMz3pUz7hh^ z;FZuV;*{(Y!4>b`hGn#SXrn?^yTs=*#aPx4nFpiYbL5(3 z$l!nzq#`zyq#AJbMNHM`U~pYhy{4+QT;5ce!@WS77Hw~A_)Uf3%Li7LRXRrpJ6=sN zfa8P3b28|`2;Cd1m_3J6Go-Qf!oZ3m^LLUr40>dRr|0dPiVtw?5~(gW^T49nx0gD% z*xIhv-LadW9?WRmtKOkPT6_5YogQ22;q+;__wgm`5q3k|>0c=i9i~<_+n?H3EvYsQ zQwHfN?IhGvDWVn7o|L269gowWa?S^jIQE_FdnbdQtspJaz>cr=q1L5K)|_0| z4tWzHVv^KoL{h)B`^9> zzH;H5oDyV%-FZst9+Igtm)3Db%k_jGa^QpX&MeV$C0x9d|V#!HvfxWrVgHzi}HWC;tg zg(rRHL{(cHGzU0Q%3EG%L^!t2q3Mz2nNgaV3CuS9o(Xkcg5Pm)q+^>*c&QpXGRpsO ze;?$=L4+BDRDa_eTF!!4C0)k8Dlqm~aJkIE>PgSepc2e_OJCD1vYtPhc@(&(uZDO2@hkt*Y{l`1YvVLjNi+ip zv%6bamqVh^GSx+tOGz>Gppga!&}I7myU}~Q^-C+Yn6ngRcmA<{-ODN$Dam6Eq>xp< z(E%Svlk0Dzt{Bjq>V19a`hu-BvvzYT{oT|X^@PRPfW^8DJ-5B-(|Q?Lm$?i!mQ;7PMNU8Rb>kx&ex}FciIfkUB%-D>b(ylE1h;?3yQx3V>X@pp+AAlDu=X2#3#iIzO z1#S0-ZZd#8L~ns z80L|t%F!K-aNPC1*ErGm+nJ00nD3p>0|oCn?eA=8c~?D=I*>qqw4JL!*x2au+by*z z^IsW2|C#r1-oI}tR=)7%4$^#QDlEUqne5O+x3$BI!{hJ?573MCNe-fNVlyqLPC2B( zzw9zlZ(NQVUytL8J@P1T4)(5CJ04={Di56+h>o@kPJ%cWMh<{8y{pt^T7XUJezseM z`k6Pkp9g%s&^R+jU+&_zdyB9F(eDn+!3S48vVet^I`FybC`A&S1W;y*X0s-QQc(#q zi^-?6UfalJM(mx-98e94tzY9ZAV5kKHKd$$SH_99@%ougC@Injg~j3No<=01q_Ai< z^;_y>Y-+(UGy|uGPf`=Pdmn z4&dyW+5zpE&C7;eurf1xE1BM|vlEFuvz$~ai>V(RUae~b-xhfr@2a+Y)dev$D_YlA z6U6{bPH|5@omuZpr%p zaePp# zQb@Z!dVKdWH@%30$sX0BR7i#uOz1<1%ZgOfbAaDy(M+mJAdWh5S0){u9!xTuqF~mb z;8P6-;8ddjK#b4v;C9O7i>H^)XTY{%mWet`Uu`Od3WUo!N-XQL?uzJFz6pqhRN74X zuW`zm%?d-yuU!(4bwbVhE)j)a%x1%H-Dp16S^3O-tg6?H7_!qanvGx8;G6lqqM64b zj~@uyv))(kd+C#mo*j_MgSYHq0Q>bKwfdo=*nAP!lBa%RQib3&00j#5LeDJ>e`<`d9nE1w5C!%dq74Elho1F5mHlh(Agz+-5XJ1GBj$k!Md*FFxhid<7 z8|YtJmeIUztg2Z0Vp)Z_E0{&%1grF-kY>>sp?1$pmAx8~?mt@c&*mBBn|i2wq0JT| zg&n43dI)M$wUIT{Ac7*TUU#0F>N)waxqczElk19$)xq*nXv&ZKc7k_(34z@-!q*6% zB1D*)=*v%rr{xoCGX|BVPj?>_&`P4Gw(5ii_MX#0WzT!^q3FdZCLO^hJgG!Bk}Pre5eX4b&p=9_9#r%EaS0ik{%*nupjJzSBo zPF_CTRQl8f%Bt&H@X&kc0tCT2L+Bo;33`l*as4WCsp+9P45{&|f0L z@u-&?mw6vBQN9gvaP>%GeG){On3wPQo!3)sn5Qv3{>YD_W%YTWG%G%;jSbHghd%^f z)`<+@Lx*8uDej=zxizhop zF4_I%5Xd7EuySyemHZ$ElQBjeLRWMnD9P$0&D(e7B&aw@kio0S7o|0E+IgZwO2JNm ziV~`yt{al(>$Sn}=VSYmC~woqQI`7g2`x zUOlb;f%|ml6+$SiE==on{ra;4I-K@CtaETCi1-C3&~-Kccxw7uHu$p`3QBXK*2N;) z{1oMy-jy9b^cuI6OtPCZWB@G$h5Cna(tMXPOR-phTVSqX8dcXf8r&M!ZE0*V0)v(zFTrRNf7HR&93sRG2nY zsPVL7{w@UDC7le`S=J_%@21-z2SU_=*VBrl;$K2^CB5WOMV&<*%hyG0QNe@yn&H=M zN(v6{N}JWcH^+0%a}U6wJ^E6~NmV||xV-)4_9EGWz3&VYYwPs2zaKry7+}n*zWDpq zawS=L;KVsY6E@~+=>V`xgv9}16_(N$0bgHdzQMjBliHB&Zf&RGa~kWmZ}kZ%D&G3q zbVH`GA^QRzcY{0grEdV(>Ej2!scwez_rdvtQRdt7fH~IR$_;}-ZfF>y{o!yl4#gBO zX=bR6^kc?%ZKSpGzQoix=k3o-!xaCt3HLK~MZv9g?W{r8&&+-%F!#e?JgngUI3F{& zi!e~g377+Z00#W?03-k_01o&AIKYQ#3fSTEM;nZ7m^p{^NB=4jryI^2>FG(Z8Jlopkg26S4Hfn*51q z`=Pb}j-dVjju^Wkv413{p8f%UBDyG~&YuWakAIs!2Sa%Ns)(Pp1X${y(*{ce;uiqG z(p^Hr9fNZ_gZaRWZfyE~=>Nexhy0&<=ltBQA!vWS1C~hUx&5=9L%K`+H@+nn>A%4@ zzl1zv`A2<6037Cz@zZhjcV&98QAl%7bL{U{!K9hWV8|>sU<~*8Y34t&wFco4|1Dc9 z81g6I+z)ofo@rrY+YR%>_}F^E!IKNNP;a#7@9N`b3M|1gox}M1V^By8+VfWu&fgMd z5&psF`pviW_VoIvVCQcEShhd<%)Y`p`?Cx`VG&@v`w5d^+2}`yHe}qtG6e&z&|kfe zYlFw^pJE%Dg8z=QE^NfNZydn`%SI~&!6eLW$FEKTvNuYwxWP35^9L9Qm_3W}UpLlCrue^#82@i%8+EabY5R9S z+;Rr&m*D=Z-gKDMhKxCuo%xqc43vN{8RlWbju4Y#H31JlJ!3`z7V6I(9825)ZvYK= zf@2AMBa{IyfgD$+(uV545C7o(vGFgBe<=Ln|2>!hXdIF`<}t@#b~g+T?T-DAz6|qM z0A(A+XLf);tqf%TM=J%+{b|k2qJVw>Ph;JPvtyCoFgVr*Y0T_yEdR=9>xK+u?*Vs? zjfkPnKl0-6^uyqM{>5U)_<4JJ{~1Yjt(ex$Hh7>7E?1mCMjwub`?>nV-9ZLmec9*t zg<3dtv;7&LX`%f7jHmpUxZnW`mw{xWheMaT|l|_5iq_|9?@p z`OW^f`c5}bZ446Q_m9fu`Dy>0{vQFD1Te**nRUJ0AL9dFtg-Naj$%8I;BfapxSVbq z!{h(p?*vb=*MF*S^sFPyjKCfQHfOD`8}!c}w{L@HHsPP+A5#O6$!EsFJ?-bt@&RCx z21yWDCIAA(Hi2ZDK(b9B*(Q(-{6%CFNVW+i+XRwr0?9UkWSc;;O(5AOkZco3wh1KL z1d?q6$u@yxn?SNnAlW96Y!gVf2_)MDl5GOXHi2ZDK(b9B*(Q)|6G*lRB-;d%Z34+Q zfn=LNvP~e_CXj3sNVW+i+XRwr0?9UkWSc;;O(5AOkZco3_P-1y`}rQXB=~wZ0N8>% zxTXpS-vZ78?jY#O4FpO^fDkM%koE_6>;?sbM1Xz&HyRRvA|Ma$N=%bK^Z1F$-~iab zEWm%pN6v9Ce}5khIXN^|)|I(fiYyF+lDp*UBc~uMF9)2`y5!>uL%{td+~6P@L396n zP5pieZ+Ff8wkjs_CO+D5FK>e&Ke$zpsWmJJ0aJJ1uXS4D)Fq8eC?6Eu-&Nug3W>&Q zT+-aXVO#^GnQXcJ5*sT12+jQ)3z$gQnV3sxWBlL}>ay}OFnM_e2{mp?eBU?28})N+ky@p3-j~dK%Gf2ErN(DoWJINu(UtjK>3)M z{4xArYYc^A`nI8s_16sm6aQb;FO~@`kh6qiF*rXMTsHuY_CN4DxjXDnS|1QvvEhI_ zOb(8Oqd+w*czzWAWaaaFftx=`{#Cd?;*FE~I|J|({de`>XP>DEI;^3Cf#H~W8tQ27 z*H%|jRMOEqcTQR5td8!96G~^bbx)jC(>|f2rmJ>VP49OaLp0Xk6%B(k(|}fFy}jKv zl-$A66t(1r=d9j>`w>7RT^3RBK`uC`?8E)3VW({oCz-A3>*1%>B{Qsy1{+_eI z(crB`AUGpoeA;pZyv1`dw=~l;G}hgiU~Lg__3=S6VY4Xkdk^489VG1R9V9r$!Ko5+ z`UP&TFs#p6b8}-R{X1mRe{V~qpSs;LOj|SbjHnaW^)gvcW|}= zO4h(#9_Hf*egt9zqH7-L@57|8fc$-aV4*mO;Fld(T0mL? zyrTl);4CvBE$#LXzUMz^6cWeGm$@-L<%FXlR$$1)5sLsiAQ~SFeuDrBV)8F2d47`r>6d3nzJB>X@NO$H3U)qFhCZN2I-&EQAK7wgK5ESsrOg? zN&n3Y8wCK(dhja;HGlKY*8@O_8u$^0s=s+hP5=PUP5^jU<>Tt-x?zqLJZ=HM!~lHH z1|jm`w<`8_fN7vu4<+Uo7(j11&w<}i8DTIM4uPof7yzK(WiVFL84P*`_|1YL0C<50 zm(-N}wWQ`o3RYk@>lW6Z)8-8u;0G9(t8B8fv9WDsW8b=!eLFk2@7%`DzHKMxjvYI9 z?BLw}b4kr#!OzHl9$7dzIJh{u1a|Kh_}^S*^A)(tCJU8?%E|&ROZjt^%{M?R=;X%w zn!m~lx(lwL39hcm0&E8tF$EXi{38Gf7Ut@j;JTWu;KG`$tZZAhY+){53gUuUdB9aS zcT4W!kWkpWe;=>>S-x$G{O1mwIdo82N=aJ<%n(elg%$LkZ7T~0>sHVZzymJDxmDq; z#s0mnzKZ+UuiiU%fOq%f5=%Fwmwwj{N_5`W=9BvPE!_&H%u)I(O#Z8mJO3VL)?m^< zo1{7D%$W?Y7cxeG8ki^h^)i6upsa`L1GjQ>Bksm+=^TH6c`wqzP94_28my z0?uJO`~5KG8_!Z~xNXl=*qWK-=6kb$$b5fPsfV&sKA|hz%onIE7{}u6Nb=utG_P9m zTajK2`tsJo*~GfrLovRN*|ynTZLZX?JIAt9$R*j>hwsW$rO5kRj1*%JjYK>=X^`!O z=!e|4U;1v8$eHNxP%cMn^W3)%zi5BaEIXaupY-N5jQ?y^)*VElNx`fV@3EZkX6}!7 znPi%qepC^4hDbd;=lb6I+=KnErbPKK8E4;lwdLhpc?re6f9tixPVe5wREt>Icoj#N zxXkUX#&pqiX=**&j+By}N)@Upk@v2|r*ua2yE!ePB7&|8OxA`6m=0S&`8wDUL3xW8E5S!;}@*l zRWV+CFt0df|AUOjjviF(_0}qftqzAf${OMXeH?C{-1nf_>Ih|~=^>@?0u`EGajE-I zWkzn=Gua1M6TF_4-KxoN8%wrys|dcwR~YBv^YTOCgBsT`dFh4TGXcT64o!*; zg$owvqxQb}6x0yoQdMGcI{VREqvO_mFSpj~wq;%Wf(v?d#IX!}BG4!MR>#wZIPVKW zqNWS7Hz$lTuIOxsgpMbvyz6|h5R`OJVovY&vB(0-#+niHlSxk@2 zzv6A1e4cm%IsT(hnR8OF6VGz4!o=xvso*WW2n$W=17S9?J7#Y337zo>bEx2^a|SvG z6kOx7SH5j|B~jRzC`Ees_Wr>m#o}1r)+cA*N^>>DQ7P0X2fpYF7@K86;2vpC+xQ=U z_Yg_rIP5B3Rm%0qwMXIl&TCokAPVQ`hb;0+?DLI;Aa6t6?@8r-ZatOV+HvAjfM=OS z5+HG>)h9qb&Ef6BVF!}YJm>j|gZ(CNZFD@!Z>4v5Ll2zHDf>&ogq0`ry$1crRAHHQ zg3xxh&#Vt~+$|bE81vja3vmO=?Sv}~&O-$|+#v_cq;`lB1zMf@yer3Z?@0IS8Qd_@ z^1Hlz`~e?&<|BngFKEEuIc7-SN7-g~c<+v9MFu>lZN#43zT$kNKlFl9V(1+5@xFxX z{B3jJKiKi4b7xv|em0aGdb7`>A@R(WJ?9V!V~Skrk~)|DtR4;x4HML`Z#`6W7Yey* z=1MOmem{P*3d)%re>io_%#D(mlf2uIFHuqRnz{G+ibwkPYUXmLheiA*son*4xR(xJ zXu0G187hY9$UAq&uM9X+)_K`SJN9=p%IBS9Q%;N8;oA}VL(FxHA&bY@o*!pLN^-OJ z#>iPuJWoA-FLGZeqHmwA24E#1T5|PDLxDiY+4lpSz#8Jr+#UVkasy>NOT+$04GH0(ns&_xA%aW$DJ3Wa%G!6fk zW1^uleb0+a9qO+xh>4CSPTL`{3*=Q9*5VU0Y&-8NouOJk?1nZ(bk*6Ly&?MTGqX8T zX(dIBD%rg@m8ea78j@All;JheA?>VGFg#hj%e*N$;iYQAoPO1D{?lSNdyg9yTerG- zCCMI{{-DBhsBZ5JRCC`wBQ}+(k#mUF+F<;%r8aNc#oa7VoJUVodjL6BWshNh2R`dU?;I?h(pW}1rC z?z&JV$LFNPtD<@6#O=_J4+J{cSntN_BqmCJ=s2Q@cp0R2o&YbnFO)kA=H(_DG~|~)V!v_e+^ag}bmQJkdxMgKLT4ZJ9=WTolSf3@ z*zQty?^hHaAN+FT-cq~PYB@Hpki(87TsRn;##oj3N z*QGqpGddRE-l^M4)A0B)zRae75nY;QWNxR!Y2^cHVIpc%DFo({Eq@5FTXOCC}w%tNHnWdFq{L%Ah@<$$Ln~aEF^!2VK zG`WZG%4fNbN(k4X5Fr*4DwfCa0Ypvqsz9|2&zxVa?@C^Ollq~TFUF*2-=B};Ih*!y z?~5D3!tGF5hez#3b^bqIv+{*@Dbk=ysO_6=A4g=iyN9F-sI zxKIA8F4DF7Vpdu9x%316i^^v2yBE$a9D9&lnfX+w!(>~G^MsRaBs74<`$6F+i|y!e zRsK}u`*1c%onyXRVxMdwp+4W}g2>~KXvda+C% zL0$s4d?>B_xTrusHT9xqI$J-dfBQqzW24-h_S-Jzd@)@Q7BwI~DOuWaUVbg{a9mu$ zQuX)6VTH*`ff~%GOylZyd&<7t3S)!Uka5d7zm&Qo@m$__PrlF7(Wz8BmZo4KNjb1h zUh;|HxO&x(;V7>{Nzz1mb)M4kiv2?KbtotdErHk@cy*t}_vh=i<-$%M0aKycdIvVlXzw=mf$D7*L^AyYg;jF*KJQW zg)MziGrl6XtMPoM(%1E*=bsvz%i+G4CsS0~>WXD?jXT;-Tp;dJGCz^<*W9V(lUlJofjh_NZIjgJ z$N3sKj%#W_-%;nP5DOwtmJ(_+?q=?F*04lv=?}HEq@Wuqw@zzk_Q^x*I1A&%^e(a&- ze5pbGi1bdP@s`??IeXWSYTISwPoR+t_l!#&qRO}8hXluveDnwM>|3qYLpgn^i1gH7 zbP-yAUZ`+0A3U0hO)q%1(@&PI&f=bkreXy1=AQ+>_sF((Ya4$lsnArtRNp9UKVs-j zE&8KokW%h8#CXft_T|jewO3>>+D$fY^_{YqUW@$TJ$)0wC|MMM>w5NE7f^2^$hDp<~;NBAVR89HW!DJy9dEFwMvnVsp25@+57oLVN zlZ`07+jUZu96jUso`q2wNnIjKWbk@3>89xu6F$kKUFHT z8+A{}vX}o>>DK7>l(TRl-Ssd#0>|K4+W4!HJ~lG*UY3MS32s^jJgs(3k2$rT4g(Ji zwI8QjAbVPFc32)A<=C2Sr?@y{-3wjgL|6Qja zpd%&xzJu@;sCaB>QfIunSPb!PQ1q{Ci;&o^qnzvU#=O5zpsZx>`1;qg0NPKfE?>c= zl}ryln=zr65qnrw*s=2yylZ66D+-#;~jGk9wwPoW zWSaDGJt?QHWn|UBs>V(PZBL~Bd9oWUG3&WUH!{lUVm^L;Cqj``(H3k}iPj%;t52{H zm=;U_&~s00XM096;2e|_Jd=qE7r*)*Y+JZ#UA@cj;ZwcjKRjRVv4JteQrv&nzDg)> z8_`XxIIk*N8EV766uvD02XN-PhiJRF?w)-su3Y!v@O zP!do%$-v27$ffful`JP}<#0RXx0C0m_k$C!#uvRDg@0$4hYx$YWYt#&6WBGAwSsca z{;)6obrD+HN_S8;+E~g)9*ci#>-ag*8uzCM+N#m?Af&|55j+mD{c2<@*K9?m&U3MT z{6cKV_A~tW^Hej>mr9R4gxr|T`eK!MYDdhy@{D2ZU(oMC!~MVHwH@WhbFSD&Yt#8~ z>QDORu!mo5DT;DhmZfcxS}bExil^R??E4czsx+coijtK#yT6SU#?Iwmws>H}u)lMx zc2}jO-N-a(1Fv`>+GJI(!enO+mYVg`V005?LIMvjy;(LXER#yhgZYz$8GYgUm9@62v==#HV-H zYld{uW55(}aeVJQeV5TPbTEICK3#7{FnBjs`Z1<)y(k8l?b+fv6##lUmtjU7L$=_`8sf$V&j){S;&SwEaA%FGNIK}&pkFU$=L`VniRj=FNk z#|9k*s&qM$S`kU4>z!N?Z6o_E#3=p46hCu=O++6PGKt*AY#!#r4BMpc|2(qEr$qXW zlthdsQD%z*hcnKvgy{NZXYb8x4am!i{*;P8#m-NJRC4X_IrW3Vbwk|M{6;NN^Clsh zTvBo-o%fJ7iltx+)@ot4uqVqC{W&F{OS~+HfBI{opVXnLysC1S@FA2=?E50l`lF@= zU1oqG&dm5WuU9o931p0WH9={brP!S{>8I6EjqO*F58-O>fG(`vXsO0g;0(j zHQDm#tM2VEjz z{yfXpYJ#;1fI36}@w>!zt|E>z+CBe6ud~tk{11klu@6e?3d^!T9-3?0knW%AI~bSh zP4$}UP{P>spC&}9T3-jW@KN(0A*%8@O7WNdUl|?Y1l?Q>8|My0Qrco4Lm7+@p9DYqyup#EqOk@R&Ls=yD`;Z(giKG=6`6{gthyt zH8$q8_oEE+$mOsBo?Vh!A5$mIu}~fTz`*Rx*>+mOqJTH~fTp-i`N4yKcygEKpixhs zFCmNf$#%C!gfs(0Tba)X*fQE$_HH8&a)Lh#xe8N^(Ob zl9Lz-IIXs3G|z5tYG+m-Zr;JRD$iEl%MR*Qu(Oii@(j*#w)$A<)Duf}cx`IWOWI79 zDUXU$q)uiUq%ZDNk7&n>*;=N{G5hq#MLGngwEpxFT z@9Wts=fdqx!}`j%ko3o;ok1{%?DBHrh5y! zx9nS@`m1*TE3o%qHDY4xSGYtUUL%?L1?v{4t$l#oRZQEO)xT#+AQOivRjl3JE*T0g(iR>G$e{MJ+x3W&89bMaEH5gVgD`JY05AmPdwk{12x;Gc7%Ai%$wKG<~r#70&l4ThnGa&;i zdw6JiaDKj=E3Lk@H`^S=KG%9iMhAil{&jghuhy@E0T2`A#}4iPHOQM7VZmNoL2VM( zyr*@Zobm4^PXse<-AbNK|K6H=yw43Y27*$=S!>1wqAd#v%bE9GjV7e8n+qn-lgRhY zgcG!kzRLL|NAA;0AM>cR-Ga>|3*VlPEBCaE)Oz{^i@|@9NVy}so%xBKx663EJ|IhF zZoalsR+<89iZu;rY6zPG{^6zUbff#*wI0@$iyY`$)dh*Bt%Az-DTT%d4qr^jzT*em zb0-^r5(3$pwI3?WXs?MTk+6TL<<*+-dB(P{xPETz zB>BfXRV2TY=60>e29=38tpI2u!C-5CWIX(3xR))`HRWTWgIywN;^yLXPhPH?pxmkC?Sy?YS4H#pt`RlC4~`I#mX^{I$rF@~*hBHfmF~ z&#K8)jxdp7XpS8;0ZA`Piptd)BIj&ae)|jMZ>xNEN5UW7JQFbOLvO;{FazDQ<-t_6 zGpn!st(QyQm#ybjx_qtr995>N*=6;2Bw^ft-)DQ&z}(Nz`Xx9a3}2STZ`hO=Hg;1N z{b)vGY4dGpQV=LARNbp~y&<=IhIukDQH%I6bm4=2TaR*j_QO8Z#7IS%RFZ(#g58UA zzdbgfrQv8fVnRq;RGFasJs)bYgGewSr>%|l){YoeO8%aF_|idut%2Jtf~EZW zA~<{U`4&5iocq;)(S4COgFK?I|8A#ExED3%9Q-j_-Sr%Qr>jmQ#NtD1dJZ{^oXDqn z@c>+%{JMP8g&eGytTYunzt znUD8vTetDm%lDX-ex>Aq{Xqq{3n%BGva2}!c3N7Sp* z6Wab(tqh8$#oCUH-ak_higY{yhzNfDLb3IX*NOb11ib2xl5nhr|0FKKR`VZTRK4F3 zSJJmP9yX}d!4Ha`Yp%ym+dwFCw#>Kp8dF4fH{w?ncI0)6`Dl@5_lgmEO+4ZJj|mi4 z3`L0mZ*{-FEs?ny6Ki4|y@lV`?;hmRYfC(F=Ef&oq_aU^JQ%buVKNl|pxk@3xWx3F z$IM7)Q(e*ME1ZJ82ZkyflA!>oTZs)f&kUtro^+Q_k`gm=Z{p#qV_OPQe92(b`x&#E z#EJ^Dybp=-&=;&?I?|t|T9_ae&(%5TqeiJgF`P|2T1ZCj(HKU?9TUc6i6zIK_Wgv4 zzb<)QB!D40W=J1-=Q_;k=#D>Prz&)keOSjwPN@$6jC$HZ1G9equv%5`+he_kXOh2O zoLo`#8UN9Bo1AREXW5{-$6fgH+eLSFupnUjg?>G9G~EI71enz@tNpv z-hO`iIvpPk3#7`+|P5;q{AXg-Rv3fz1!Sb zlnfw#(Pd@WJW;7DQTk#=E(F_loAZ15qe9{dHNKR9o*ocw9+}B<9~K_-lT%HERoRlS zq$|-$?tW*Pz&(1Z$i#=NoLz;?8XgYziYHH>h29dP9zJ1YdR@p>a{r5+47|!?e{^xl zd~=jX$~ihutJyQbmE5WzcS9hCFe+?=hF82xNk^0hV{Q-5A7KAbp^7guO$zKyB#(+X zr8RPT{o0;43{RF1FF#n2ZE`$fwSn`sFqz(%Kn$vlF)YteMqRO(5Bk$<`KG1rfse+N zL6CoASt|W(04wE+S`aB=r$d4fq3r!0;lb1nEnwqA{fRs&6SmG0u@zBJ$qu#U>rnEV zhpMVQZ04~8Xglws#+Bc?PRJWIWQuO2jJTqPJuCTm5Nm;DPXFRA%BoS3XMoX~%rwNd z3zr%x!|*pgy;ABTMKd};1*_S9PB;m}_8C?1oM|6^c}o+8S0Im@WhTKVz|ETDu@g>o zt|6i+q~TQCS0pF|3HSnlz@XFgw%K%6&UCPpItpIwAk&r;Z3@zSB6ARxy^0$F$q4|3 zA-Xq21p(@i@NlDGX*x1gp~_K|n*JE#upUVnLTa*lD1GJR2C&-A(u!Kc0m?H{( zLcP|ODgn+SBw(i=qN5$-D`F^ara>c!D45ZqYpR&y93>Qrpr}HNglN9Biv*04S2&?o zPS(-W+lrNQxX zH~z+`CYJWl$!oaz9YE04R1vUn2Mk6gF)Pc@3Puo^!lDOVLjyZ${fhjmh{O6fQvEQ@ z6!tyA_YmjiFBK(tD%avr<-UX%Tq~IgrGj87gEcrpIWSq1qZvqq1G_}Aq=d&>1SSe! zmPm!*9wLCUkRz&`hlhx#Hmn@32|;tfGFg=ol?c7D-=x zLP5zLKoAT9^iylkC^cwMx4k}I6Qz*D7j(S32_j}>@w>IZgVr{Nx8ug zyx=^zA1DPEdJ|bOaQFrWlc(cBsTn}AhCs9t!j$~5aB4l{tPOQ$0!)}{=vf%XPT_0~ zaj2(?qOf4obdxLSB1tvR!EUobv2-eQ2DslQ3GY@y_S#aqVJ)`^k3Zk==^%P^3=Xa; z%n^yaJcLTJbWpw?nu1_j7U+(QWPdY5Nizf6Cfq`CB~=}Tah!O(0;D7&g6jnf%0yDH zu*nK&LKwg(gfEdVuX#8qiYN67wV}?3K8M5g{7Q%jkk~mY4_D&w7=@EfJzxUgd#?(r z6--rf?8bhVhNhO52XQ#Ho|X#w*Hp*QyH87(xcPszPbV3LSt+%J#qwl6IM=;Iy4<=(Ky2*3?~V zS(1b>#1Vwpi^UHR9XNr=`+TUP7fk`+f=iU-1uW3%NqA)UV@{~BP*_S=6;QSIZ03rW&=^4+N(W^Yd$tfIf>w#M>y4#;sJmf?bO3H zGYlfS9tx!pKrpkaX_3^6a!A#2&`$W>3!g`QXUveoYt?aODXV|@?OG`Z%x4@0aXN}f z&q>Zf3r#=xfCVQB*r{6K5{(}B3&Q=F36*#WRdr%i*rv32k<TEkE};-Q{Gbsal|qnRqau{iX{Kz?8_Xf_6iJS`QD7>I&Y?Ls0NLE3X~ z3LTHj5u(sk9c3h#4-^?KD_X)sXo#E-czl`aO~Rc=XAmI50d+v(`{ePqCI(cgr|A*F zkg-sQsHKknG(8DkIxvk+^^^k2pC|0QH3QoQW${~DfTfnMRP2VP7*@tB2(B3xL{<}> zH(wPe6esji@;!Yjqk+9;~^xvo*>Nr1s6YW-c^;>#&)s0Ib-qVpY?xuG?>^F z3q75+n3%0-$rpPkbkqQAbN0O)FpsF&k|pm6;9c!oycC5SYJUw9F`tsUnU42`Ku!l? zv4`IRJK3i0=S{9eYw=2#yP!W`=;V~lok7K>pzl?b@QI$+2lO4lQWot=6)>2}zva5@ zhXit61TngVN}NSaJLluuixgk5dRn4pf8P2|EKLpHI%KLaQ+ZI3e&<2QXb0mrK?k5b zt)>w({ka?>Jdy+`S&)O}70+(ck|k;MiA3gXPPKGC@0~<3fGupT-B9m6O}!5(9Q;>$zxjuj;BR4QC?N0LGCECz~KteE`<^|>7!2G^4DsZ6uX^GL`K>`3?a zdZ4!!U0jMNG2B%?hA(u2+kDvk0-RDckfoIyky}E`k2jy`kVZtkXM^UX1oli70SPd~ z2z1m-i|NvMEjmh|!YKqYA3@t^#}`#zlILnj;DN#LbxfqrVvygB(Kve1(pazd;WLcZ zc>FL6hsblivx4O4{w%1*N-4od&0|`0)}#{Zz^trZTx=1KyBJX#uk8%AmO!BZc|09$ z!m<}o#3%9SGdkQ$YuZF`A_zo&11E4l!#l@+QHB$uU(J+eS(f z#z#^5sVoq4d_X?&*dqW2AE-nW6e*OIcNUQta0CI}oAGFPBR8q1;tux!vgPY7lj6_| zG0)#=mK^B#Q~Fa$bQKg8L5G zw-`&fY9h5ot2v-ZFBDy7PJj>M4PE)(@p#(JFOiQJA5Me7CxWXeVK6-5bX4Frniv%f zIy{a;mS=vY%tud^jv5YMVe>;-O>jZ}aPF0kQ8?~ks(8W+-^em}9KmssN3mB%`=26>uT zQI4#V)c3$6u51M@Rf3*7&E2;CPAU_3BLL*XF*BCbT|k+sT}TytHOcnV{kg4*}%ydtfFbZlLQvC zH2N;pDqjIB^Bw}hO{JlKkla0AWQ!{5Qa>sWhDn9uc#lWC^gsbhcNdA%c&Z2#)PoCn zWK8;0RgvG}u`_wq*W&fLf-*i;;G{nu2cqgv0^n)5+|U&XTiH%PhIgd5?U$82G`i5i zEuS~P(}M^OR41ZA@Fa(arLormY`hy;f}fL?pmX!n%O0Cet)V61bQX6Ysqf()wynhM z6sE;c?m&-8YBS6W4r)b^^i04pWJi*wHEAQSqP?#s4MROM2KU+M^CphJAjSCYY;*S` z_b^re@M<88cG?oB*}=dTN;X8zP`>8SS^&r7#>7{C`hL%Av4CzTG~w9{fKU zOBJmG26OgEG+crD<3F+li~JyP3e7=;J^bcq7weEqLzT}$2kxLY>eHvF(1JcM3jm+0 z_>t~e18i$}159&|zD4gsw1^*bQ0wg}aS_75HeKSDHcdxkQ1FSC>YY~;n-*Y8%u$|@ z*}$;kl-V!SvEs%uoK5Nog>LRarv+o8P^f#Ow38)epbSMKRC{baRcV)lfLUzyONAJP zDsM8K%M}m_hMlGi?%|NJ3=Z2Hv6UE=W#Bl9lMU=>d|ptXNg|A|MJ%?ZUTUbPA@+KA zKM|XFVFe4usgU>ctqcxqJWvM08`^9bL}e63cBx3D2Vj?t7S^U#fYNpcud`}IeGtb$ z82{lRlL$wwIXgNjidm2}fR)zZeplyC%- zlUD3A`7y`ymUhDb=$yURaX=lKp`VAt{V_Zd{VV63(47P>xIs7+`;>P!V^Xyk(>5^% zw;1~vQ?WuwkOyD6rQ}KK+U$DCh|8q7D@rKlop_`|Lh|=L9i2Qpt^}wMpuIDZvJ+za<- z&BIwfD7X!dlVfzU>_AyUo@!Y%UbG4uPOip_0rjvs#N{UvoJqH0a5$FL=#L1v@S~Iz z2UJWE55@`YgBK8FfWxJXvQ0*;*P+t3g_7=CMn+VSy0y0I0X{~S|95C*RgmiE}-ZPx?rITO= zQe#2H>;rw1#6(n>Fa-`OSA4i0_+%wi;Z%iyH*#{r9~54?1IAfnRa8f~1fcDH36Zv< z0wxbfA|!0GcxTjWm^PX$C@+wDg?&f9U_M$ANIMEfrvkWe0~!Wb!QA2=YS|nEIDg0w zMQ(cqA#5z;DQ1UBf*abGF8-Nq^r5h}?L0tDgz z)7Ql)$Ig{J9Tg8+TmgXB!=EIF5`|_}AbAmIS51X3y=5y#A#k_AI>w18;m#phVaq&( z=p8Hm;RwgS&(7#b<#4F=Mi|wjJ&Gai_ z1yF%4i#8+Y`_?%`TR}N&hk0Tgr{X+h2x--;*wOX$sD!-yPDKnicOx#ns2DlNObfTj>*96L$u$LqQ&S- z-29lK6aY5`Fc??7fN1gJCXn;ON!WMzx>z-74Uyjo>>#X$*d~t2EvN)fJS9G*0)$d+ z^+y&6;K7IXkQH=QW#5T|Ll_TG_;}HJI4d>=*TaPbIGg5^6qrMjc$7BHkC{Td5S4`t zH7mkH%Hji0d;9?Oyx&mxP}*G(QgD(Q7j*m%IlN{4aBMsQ7!6Xd;-ZPs)L;h6-iwhX zzlvlZ!bMaE0)x^~G#G`d@MS6;soqQ3kwaqgyR?)SfuIhR#}*vNin!8IA@-?vq}N7{ z;~viarZ{8o z0cab81`2MN#wPI<5tRM>iT1SnFCoj|b%?I2j3ZD)xK0fas0}|^o%Qj9v7zqT!C09J&KAXl4=bB8(Vrw5P|#E*hLwbaNLSo&N0jVw{x<=| zISC5I6!bY8^djfka9RnBrdz~SB!u8q`29dY`rPs+rJ|lDSz`e*8XIqlq$Y`1!q};R zy#gHGctV-SAV=Feqy|62Fo&wBWByGgJrIK%9FULm{k|o@sdP_?sK3<+#O zixOWrWIV8701G5wmxLuEF_;`=0Ynx^M=QHHu0SLr6*|u&$xxLfI_1OjH1aK5jA_Rk zlszAAKlH2%rjLq>Y1(f*H4T68QN9?>SVw#5(SkZYuphGR5#WZ0HBn;XV5H-j%g3HTZls9&;&&VsD)#yh=9KWuqeKK z$(SJ(Hp)B>m<||?qBSk>eW_=^#H^$EqYbE(vdj(k_ORQV=rQmR+{*4nWOJ~X{!JHD zq#%AkdJO5usq(FM=`@%+=@2H+E=oWVU40(|<_HHf@Dac8qM%dAD^mfrCV7Ym{z#n1 zbBNY@!@}Up;-Akej)*QMdWL@NttpGIdT^3}hrtwdV#NqrC(s7}2~OTHm6PcyDg^0| z$C}E)=m?$Hkmzb3x|8HcJr!^sCR{i?6Au!~Hrh%cKyoXM7JQ7x!lI*I25`{HiYh|s z;PAy5#ePZJX-pZe-~$7=YvHQ0seL#kyQ>gUds7Eg7~oVyGyq%-gd2EZBgJP`-TM%8NxDR#38rn|cI*n1Uz(t^g9@5DTYccw>jUUo-@% z+seW#l4U*a(SeBaPL@pTSk$%iHdG6$LL9b+>qNt3_Lw4*;;#UU_T4dH_Cpud10w=+C06fKv#qp$C!3+UJ z1PEtZbuJnlBrvXGifWtBW(Fp`Rl|`1so#Iu!j;d zcs$(Pz!@+K7EW%yC|VraO6?$=oK0<3=r*x~Nz#18tB^@|Z*(o|*xONVZctf9q&}gZvq5B$mj0AI$Jj5Yt&xOHsKrydBrW+HKJ?94d{#sq#c!^Fx;b;F3AHFSBr%O zH}RSfzP3~Rh^uW#a75+zhWk!EEl&N1XD>UZNc2eS()X*YIy*!myNBB1^gp!&j*cfL zhl1ky{{F^Q&ij!L)T>3r+#RtS7VMDS;gxt|l&y~Bl5qjwuQjYh2{qB_BpI7d9q`j^ zxFx%8%n+}b52_SfFCB_tr$R0#1K9ha|L|hlvyXxm#vmL-w4HoK6gXMI1yU+u0QIAP z_+GdE8UuWMzR*lhP-#p6I{8Kt+@xC*p8alxHcf&zRoyW)+GC30x~U{ESB-A%+0=Vj zXc#VHgHp{_8U9Poc>$p@2B7|8Mw*cHK+HKcw3$@F;;E8Cr6~Y(d9-s@LUxsZ=-`2I zNW{@Vz!Ao76C-pQ;J$~O91s>OYo-Z<dQtl+Ql;ia#?Vay%X+WSN}jcSF3m%M1L@+B*hnfoh#nk z(iY13R7FDds-5SkBb8Ha7=fJzb7B^Q1FrtFuJm{|(V9-8ibx&S{)Z#_WM)j+9YWX1 z-^6GCQgw=<8SsKvsmWD)HorSj<(3$QPFvc44Sla=7j1?X^Wm<>lPD?LRLctZ8?BKV zxlkh1{Fp)Sdq6nt9n72{?Si$2S&~pSkFy63RqV(s&;Vf2xCBUAFe(AYlpbmG-kqse zp#?V=TSt?lwUoRz3fRl6xeRvK(2>N%h=9_4eBtMbaOhfTB^+yztoL5B{|tIT1lPgP z5tQ8fP8q=q4bkUF56+v zqt#PIoKdegpa~^OQa3r1`uI&cM@ z3)qGI0Jx+2HWa;K(TER+;)Uv|#&14G7=h7YSXQ+^mObxOLdwh_%xQ2%Je-Lqj_SZ2 z!<6mt{=-{Mm|(47nlKI1{H?=eQMc9*sMo$gl*&_!{eNUR^HqSXUCX$CCD zHg*@1o_94WKeA+ZBG z0ms@&vKOjVdmvTf@pl>l#GDbG5Kus6TPoEi+E=Jl3jH?*%c`Jy@&7Y~ptYe;`nI$CZ{P?k4qX_j=bn|@Im>8?Z9sI5Sz-yNVImxIq&2j`B<*LrpmuUDJ-%X_~ff}@gt z?|z>1VRZ4jGyg)M>hU4KMY?d?eOP5`!Qoa?d>$|$)7%6e_)=pK3mE4 z^E#L5oqd_VJYUUPL(njjyu@R zp95d6nqO%+*0W@Ch>aX-R!rVrbh>u)7tGc>`wBU~D}76D({mY@HXsoEg=JZ(N3z=( zskU~Knr@`oJt$7QDpUQtjCa4?++XD3pIV-Tw;{TV!iPV%i~5$I&{u@JnD6b&9B5VN zb|AL<;(H)b^TO2@H}~i)i6-X`tP5X#tE^6^-hQCj)@;&N|9gh zrF8Ol^!8Pm1HM$|jJ^%pPNkO*XsB0f&>blY}dAZjvF|6MS@@I5J6l(62jSw=HmEx^Q*3?6irv#WWkMwU(m4dHZ5=V(<1)#FX6hHON?}QxVezuZVAAO=d@LLKrscDM( z?q?z$JmR)onAF0uW$u4U`(-3^?qYB-$m#mi8<}oTSI_9vw%P2mXF0v|K~3c@J+VLc zAAWST9Cq2-?lP;eTA;eu2RzLrw99#Be88BltBk<;@ zV*qj7rN78s!>Zgp+m;!3ztXSBXMajk2l{TD(k7mGXMvVJwB@pM9x-jVn`@f*OPZAJ z%QM-Cn?C9{U06{4Fze)D_)98kztyKUyLZvtYP0^%;xIc)RsZV|gWr{0MiBg>Seok} z-r0(=y!bynj8AUafWEKpVL(y$0?Wt0OhUGV0xf-htN@C;#w8;w!6&Djrm=4G&Q)Ih z?lmTBhl0X}?flff-h#@c-VYvW+Az`Pe)+k&TGR6NZT0TCQO%$jt#?+)UBKNxJeJ)k zRx9iHM;ezFj$fO8CB zq?UWJZP(E<`MP&g#I37o>oj1a=}sf%i@8GIjqbiIC_%nQwm6fbFS_yPOXn|kfiC=y zDwa)FJq-8T$FJCH>?IZ|-nUK%91CByRIPf_yxBAo$O$y+W^AmGHKZ>|S+UmeDol_sUg_$k|tUEQNd#zq4Chqb%w7w^L7GwFi#pFBl=0q>&a`7tUB{XPeq6~mkY8TBnQ*kDh} zz)z=^P4(l?+hl6B6Tb{OwgREc^Gn;-4L-K=H5*U$$G1f!W_Ci0&Lx>Y@%P@|9g2UXG-{}AS*#Kg(UP34wE6y4%y)ez{>;GB_#l(NHdByC zdwAbSvz0Y**7faht^VmS(D!lvHA z{}HQ|m$9H&fyVlK)8V8W_7zt~JsQ*1k|#m6;>uigpTy5M8YR|k41${7TZc^B$F0Ht z@Y+3*B3cg;I5d8}!wWi}A94-adUT;y<#IOTKS;k(;bW*=i5gnuCo@~1i~q6R5bJb& z2BU}Gv@Q<)ZFGIqT3g*~d%4s~CZ6r>D!WX{8P|JArP)4v@Tk1OxBt=MFW))uh!#y( z@4EH+HV~Ol_4j=yEmbz5=dopRpw&j}hU^5E)*i>Q!}aIU>C%m}8$p`uOk4-I(g(6C3(x`y6uF_;;DYEq511%-_ zoIOYB!G$OFSZc)>+E-BmU5lGOaY0GS8!RDqrwyu4my(I+2Rz)l{l7HLo?dgmO33sb zG=1ytU*Tl)wBz}=A^QA;-gWOdfgJjw)+Eb5Cy?&o!g_5h+76hew__ zWDm-H89qPs{UfRwPiD?>G4R`UMp~nq5jE`4e1yU~bZxMWHlKuO0Dy@58r+=c#vn zQH;CaNh-4*+>ZRi+aHhg*| zx9G@?w@HUqEwFn%wa!>&Fh=l@tHDKZ++fMHOPajS$2Ftoo1{P`q%K`Ld-#Ryg6W;< z2S=T6#txUwL3QNcoB~y<<{O5P47<96A9|JF*S)=tgB%FIed2zN{iDo~J$)#)DAs)4$mzIDae3B=(TcZ?DfgbQU1(@x1FPY^n^QK3YiX?F04B(eQfV?3W!%<6BxDfkQ(~4)DZT0uEL~x+4SIMDUe6D zLr5niVE5{V%(=Q*AQm=zBrHKrgv_yeJF@FLW^+{2N+CXeEbX|SeZ}C@WpC^2{^ecS zWx{voirQQT*R*dC$Hb90Rt-(FE0j*E=$3^&JLI1FqPs{og?wpI0T-W(yZCXz>evq1+S-r8TGF>G(FlkmyoX znA(#PZT>P@Sd}jSMkp`v%U!(^3KF)A>t$U)PvB=d|07nb?N!2lM_1z1riNS|6@BR<9)0 zxRA#-Bf%HzC8^3`eR5|e3k}Tg+tcS~&AxWcrJIXKLnhse44O9=>+gFdc+SeTTwFRCw zp4c+A9EfL_7Jp6cw+!;NtT0(#xsahJvw3$S^AB%f_#a-3_A}nAskdjk%SWFK7u-|W zXqh$QH6qw;`a&i8-u$j8vnfBjjyt&Cz0^z~MmBfZRMt3mZak^E{ffM}Y{bVjwo*z8 zq-fzcR@CNPaV7SfT1Y6I2l{YiUp=p^>z%8PLmu-G&ODFoui|zJH-2Ol**u*wm$SWk zw820n&PV)%iXFf~77H>|3H#~lY2-F7`>CkvEk{B#VRN!ve0!63CB^N0@P`q}ignM0 z$8*1XFG2! zzFZaS`TeAEo-^RLf3;)3_O70G(+rB=@av{q%+)&`&s!Z;c|I&VB&J%_vi*_Cvf_hL z=rctQp+`-7M3dh>;b`!xc`ceWw&Du&$!C_-jB^gY+{ELb4E5;5ER<>BY! zM}qYNZRxiQn(b5B6Zp0U>GeU!N;=A>4o4@g-AL1|wl?~wG<~A)V}e|BSsl4K1tON* zk`$ZXK3TO<(kSjuul=^cn)t{$(6Dx`8@74;_o7u%;hvAq@67{TC=!cY4q9%R0y444-VMOoLxILG;c~Xl~h>g zy(z33h|Do$4b5G;kGFxy3E_z zWo>fdP0{k`)5rNUAdhe5uk(uEZa&n3P_6n$eyu$dQqT z)wP+_`TbFrc5ij6Zk}^0Jo;U@MB1yy8sQ+3<HMK_(QxUf+M(sX*H<+Lkjw!ys}o^SE0F4- zqucd@tIMVFW(f7`KKh!M5erMxUHKTE#6=gHWt-KDMs+tSFO8f*7)4Wkm&(@54-xJCYG{ z{xW;?*W|mQ=?(qD%hN5h6hCdNJO3cr7FCwx1XlsullGkM@}p0IcdmjWe!n3_i#k@r z0YM)s-5JV^L`3B6sBxDM}c(SV8%id8XUL zj6OsX`#zP4m?lNAeJ;H!*{E9jRn4KLjh@5t`XK__v+hrj5#R_5mGo1+bt zv%YC00y=)mnprXN3!PqGQ4TqZpF?sd7ES17v#<2YDdT5fkrl`eJ4RrPcT7!Rk)b}g z%>CxQarvFJ8JzOWY9sFc7-kXjv^^+2JXnAZeCNW1%$cfac#>i5TlgxZhyz82zJCk}Lf6gY9lZ8ELgZH&G(!EAQ>EV;c~C24blxyN`C%L{F` z-sbYt?O1HK=NqoJou3aMlXPpgWN-8PW!7fVx%`KSW|jBzZuj^JvYS>W);xdZv7eW@ z)b-1kW_)RFeFnMWOZ{Zs{{S<#%DWG{Nt(U4-L73fige5St#d@yaTgl5$nCo~d26yk zrB366-Lg0Kx0tIq@c1mQyKTAZ8QdmI1(?JS{aLR60OT|iHhy^fZzL_6a?g^E3L6H+ zi@3!<52)+dAH-qt7Sp!g);h9!d@;1O?P*OTw!KS513F0BhrkPEw`7dW;+$g2>(=*{ z7dUI%d*}n@X|(QSuzXJ)9U1=st)uGj-TQ2Oqr1suJgcf^VZ&uLY_zydD|EDi=g3&= znW5VJ1@K>O51U(?Gl#d_Zn1eAXUB%Q*EWonD`C#Zsl%hqk+{$>w)vb^M{9gIy#Dc& zmRy(Jv+~SiyB`#;b;|&S(Z%<-vc+wU!(=S;*lp(W!xMA5?w67`0kPX;HHfgM6=`Pa zjg~J-edcoUyDky<3v8xRFF$zt(*FQi#5|o!);CL$W+lTLkeRZ|`2&af9eWjVd3ke; zACmG#yk(zm))-wocaviuWefHk2eMhLK9TSlZ@faWPKayIHrD`#z&J zg!27SW$j-agSe}5=BUk1k#Y9h{`2GQdL5S=n!(t~sNC%sZ65aXBed>#JO*m(6a8qi z%30pwym^d%OLx?!%Z-~*+42}h?)N&C{{WJ0Fj%c`Qo}`Sg2^}RcRiaQk#500x2N1| z{{XWW1n?UvUmdz;!Uu^u(O$>eVzy6t#v zE^Wg}E+XRoSs|Y6{bk{4xGAz%ILsShiMG)zsJ`oI5t8d@3k))rndX1RY|A`TZA*xwHAuiIba=3$EFG2L$yOHML)nR}Fm z(6e{#!l=6PDPr^WFDf zK_hMNhQe0Mi;K1oMLMhm=CifoUX7qZ#V z21ymH0{x63@Ug@~!C4^$HZP#JpkXWGHDe7MA5PAOslb!l)b}*dssbL$vu3pUBHJtt zmRih*Tas0?D=fxy?a;2;_`4uir6-MHP`Xm z%?oVGTV$_%ceCU$x4V|7j?{0g@NC5`2JZd%{o8t*YT14)f5|ILTI#Es2<@VCFMERS z72Q0`uj_lKkTe)Np9+vP=MN+1%C&5@~$#Sb{jL;CHA#x`7*cK?VeW>-F8^qufKSnC8d!Z~>LE3Jd#Z zi0&eLUoc_&fC!zWzJA$6fPrwp@3@YlW-pnk*KxE~2@`7E$8$%-#KlpzQ8DU6R-2vB6oo=G(@y#Seup^qs#KVTsRVY$e0x zY;7}}GqC>vl!I5*yR%trG>*5su*ceJ^2?dZLv>@LYM;z-Ux1+Qjp5FmGvoz_A_{2w zyr}N)EPrIJtYR%}@13~Jl%cjaU4EzIQ02(l7F7hGy{jT&~JIEP(X)L^}lnuuDMgsd?h6$s$$--p0xo>l1Z)>i* zT>PvJ8GXiER^It5Av4F2?`+kU@eS;jcW~XwZ7GSihvklxGPnM}Ek+rwOnaVc4X|sC z#&aL|-E$#$Al9MSaQcm<{Eo3@v#npk)D}L0$-RZp!a!1%Quw#+qq)4 z72;eRIXpbtc5e%d#W;P|_V=5dfSI-!EvI6xG1&cv-^j8`b|4&Je@B7S1#sB!^e8E> zwgK6F1C``Vd7oagycs_k+g#0ep!^OvOueMIoSsFNxIKg9d#;@*j^V+#S3RpKGnI*s}|n0PePn<8+O8W7?evc*VfwA_8%LB zyz>75jI5n^=KMB7VYu#6$z`1KPsjX+%TlY{hm+pimTNN%{{YJ{V&@rsY-Y5O_FK7R zo7~K$^@XLg_l#&(d7mOo=HK~Dx;wq{wwUdr`diIU%5X=Rz4NZWVPXB9+2R^Mnyq%Z zyp5E{UFw#y-efN=Y>pc>kHp;PGVOD&GWm_uJ7tu}ZPqesv)t{v+)wgOb__liOH7Z5 z8t=H9$&w7F?XP&=&o_>a3u~)Uifl&vPj&}0vu3R2$!31#%Xgkv8D)%=h{e89rMstXcTL`6(&alTGSh|=W4db14;mvaaV}!?f zIqv>p1HPE;d#}Ds;9#Ctro@}woMVGm=xsx}l{j^x0Debl)zFycJCSS>$b@W=^kOnT}FEfT?amz&n znQWF$O$Se#WW1X_nb$70E%zYS<8qd-UUAl&hu?fx;_N=Hhw{~&qO@EVQXY&YJemz?o1?8-G zi4~o{%aPXD`%G7g@pkcfhhR6yA(q4W2P)`%NxEChlh>}V^!adk`$=$GG!|Ocp^$!B z#?Kf{u8piWKbyGm-mL|$@2K10viTfaXU1f3xj3;p-Y#gdaClngNi^RcoqrW3nSm+Z zCB#iW4_v>-4>nmrF6S6+JdXi(t$>IO-KH-bAi-Kc9j;sCZv;B#BHrN7ny_}7*SM_P zY-Pb~wz9+Fi(4q8yw6~=Hdu>XUd?Bv;4%^3?J;XO=yG|xcw))v+J+Bl@rKc;O?!KD zc+3^I;+gokZDo%B-gx4W?LnFv# z8514kQEzelw-a=4wmU|?<6(wpTEi{c@~%hl*e?iSt}e1e3-a1|pBs5$k?~%`bsg>Y zCUF(?S60_H*Li4^S-r*Ww{Cu0jAfi1kU=bC$#A;=0LmqxJ8!r9t#@Cyp6@fN+)pvz znQw>jNY=b*b3Vs`!AUIoH{9rVIxc=YOdh+8$LYA+hYu7tFe9I7w9H~`_4%!_6J#T> zg5!G1Ut(ywZO^tF2n){{SJ4+HErUU8q9K->2ilg7|!O zmS$^R+~aB6?1e4%y5W4oD}V2}vySE*{;24#E^xhy989d67p@qa6Bxnf+sR*_@eaH( zmMC1HpdJ=zg}~8k1#j}tfd`{ z@Wv`TjQ*Ky=}(#rhr^Dt?GasMEhT@E@9l9@Z#u}dxZ15}W#&E!G|jIkX{ZgRElthN z?b@z;UfcX{mBU+JWM;_QSmdP)*s=~e4-q86yS2N8`>gd(2kP;h6|A;+%MJS>Ynbq4+fA%&yvb~Cmm{UwHIT}CZb5um zNkeg-V%9bbY=+f)n%;BQ_zA7-@_SAyKN)6rIWKY8j7-*ZXQ2F!&ESup9e|g}I|j+N ze=nEw9nl5<04zTt60+U7x=qytRt%4eP`> z*&mJ3L@l?|b~ZiwK6T1J6hSh>__JBEn#u;nZ(6BqUm_Pzt?D}b7)&Sj17(i^qV6B{ z{{UEog`*VdWXO3A{9EARZ~(^F+GS<8xP!?Jl08rG8Cz*KE2C$z+E`^a+}>{khMM;t z`@0;Ty^O|f8%DX3!_A4?Fj;#%58$-#;lk2-J}btpgBbj!zWcA}`0O*VUJ{jI@3o8? z);AoT&8G2j9NQm^^~^iGbNqY9_oiPK>BYW8icBYvb{&^7l=1$e19d{sS+qP!lX0Y2 z+P4{jjIu8q$qoFP#kBZK9CgE_WYUt?H@OCOnH}148Ae4D09DU7|mz`mf<5Pa8uVOX4);eqauERCXpNqjg{{Uf( zk1KJ1ir+2ujE>8++~3O#_b;3Cm;V5b`v$`AvVA1R$l>On8OwjH-D#U`yDseIfx3hS z%fs1si=3vr{ejr4evBXfap- zar)_oPUl(zA=i|wkw!F*SW8^mIg}O`N& zmhu-EJN)&&!JWnCtz*{hv)X;EQ28GfVC;tPwB=_0BeDJwOB7Pab+14!cAYlIV!WTm znFd>A{2gPKxz1t!%fs^k+ph^~7#oir%c1I1CwSLMa2_S`^WEinq}4oyW#%iR4kLNg zu59uF-vh_}cHL~SWQ%>V)W&#sBbs*B*cPz3m*csA1oPMGdcL?W@`^8)aWI3wX0ysNoRy~PuIn^I^OqU z*RA8Y+w30eecu}tZ`*a*gZHdOqiwvctbuFfeNyaN=CIc-$V~%<-K3W>O>PFo0B_qx z8#|Z?Oi$eK7Uo^|RbbllxUI7@{e`)n=N&MGlKA%58GW|ptxj6T%RscTvA}JEC6~uR z*>&a{$!@%lbtIj!GDgR2D{+;^W3rc+JVdq@Q5;;CaqCwXcNiQMli>aZnf@VZl*QXx zzW&Nr!z^x&;zlyqUs$@@C_Xs6r4WCJO9-FC%=r9f#{y4qQTKQGkLvP9uYB9AE<47X zef~E6uWT=%jtI}$VVd@{bs^aA>(fB;&AwgpZaW|kyR?y^mJ@uI`TQNZmirtpPWPYK zvDw@eo>M7tZKCC7yv6w9T%G1y#b(*#!|jI{d|i#38;0DGSNQHLS2Tgi&UxumDV9i+N0nC;5z z{mFm)Z)*|Dvc(uPTKC0(lK%bvG-bR%IzG|m;PK-a8t3G1r@Xb;q1yJy=C<;mkzi1H zkIBry?SV8-#wYKf@cobTgyHl8H3xixOlb8 z9~ZwjoMoVVa#@B4urZAtAp(++*f;IT`lhX??bzM733dwG7+szI(#cpsSnPe?PV;8e zuXW3!3s#AwL4$`$p|{Up{od5eb8~-rXC1tnO99&pzFW<=84Bqx9am%kxuAua#vWrX2#LqRIUI8G3g^3Q^g!)4HG%i zFFS4j0F%x5Lu!`|x8AfgXGhDje`zj*vs5ce83VL=A}OthR*gd#+iVuO3oS7hS229p zJ6P`Rwj8_{`7II>?Xi~8O?l)t7;uLm*QXeqZLa(5J0Je5*j&@G*ox$`#K)%PfM$n| zZ+*m}q2Jh0`G??ThNowj?;+)pS zyg%e`j4fYnvhP_740W%M_EKiGxM+Lc=;g*<$vka$YVPiBK^C{uLLkW7NFODMa}GE& z?rqAC>OsS_xju#04Ws0HL4C(*-uBrw&02Zoo_J+2$)snSoYSw5{^9+cls|ZND}~9; zjLPksALgzY(p%!2o1ES*NZlT{7mIA=!{aQm*!)J>n0R%b40FF?jntNASFRV;7PSt8ui6*gRJ6tY;we-Wv^l zf|P;{emtWDdPaS&82;G^%`}V!^Qw(R(8fO*gbw(#*x{7Sm)KzN+Xiy;JC$Rf8};pq z44svv?2zGP?UK#UY>|RvoH``Ti;0?PdzqViWP;`7!934xtK_3=+E36K+^)JSx3mk1 zq;c~_lu&r6iUw796F(zlyC3_<_FeVz3HKDIxS?wCSHLw)gIxxn5I+%EWHHv)HyNx3KQ_~2cV};&x3-LSyZt|$pR+B7bbpxMUR&c0s*KdJA*U{}(f{87GZ*o})R9oD+>{{US+jKB@iwkjB;kfxVsw* zUE6ZL+H$lYRE6V z+AZBd3G9pjq?EN@O<3~THe0g+XnsFAJf9l4bZ7wh_a^Lq=-5yD6Z<7Fcas$=s^*tC z7I>mBk5k!=x&>*vmGblDo|ZciK3C9(s+^HWt!J z;tsgTvKH&t@&`olOqPTH0KxI*3VhyY%Nyi*>+%I^c+;k28)uMlklxu~a|S!DZz{>+ zaPL+V2_TE$or2mIme^axC)-5B3nC4f%q{OBv&_au{G)-->@Bv38ywe?Sy8Lb4AN%u;b2XfwX){%` z+PVJ#$9Al%G))|IP9*S3WRGLA#9ZQbjDKVGZIbI9ki~4~?mK+5Vrz4e_RD>9G-D>_ zLT|(5Zek`0^6i@5Th<7!vg~5YY}VG#vNGSl8*JE^$9!$_c~k07(YR(|hwzrj)25Ik(qmg*pRQzf z3#~(4v2C}BH5=P}$BVTbUNYZ6@zkcuApPD_GrYKzDBSgiOwWXE|WY6Dh zIlu^13V{aI-)UGem!^G3HDqLuBVln9@cS&bFK&6l7c#BMMCZ$&V}RHq54PFujl`Q=lD^wy$lA8v8;j>*Fb*-}u~Eiaz{WMrExMF4 z9n$E!9D{o<-CA4r_W1p=P2$UE_cqV%%=&iKoyFa2{$It-i_c{AtnN2gxW(MJdWb`k z1eM&P?eE6TG09rp8?1b>OIB>xI*vD2w8~i7ZW|`d-GAkq_=ja-@Gde-e;fG#H~24@ zaI)n!+^2*s?yh5L4xvY?bycmyqPEBBS~f!qy2on<-+Jke&o%b_^XD%1#^JI=i;QGX z(ma0WO7%oIKG=JH^Nhy{MiT4e!(B=o_w^Unj)=)>0CqXCxzI2swS}hLk7QW4IM`qh zb~6Tht*qgO7PXWv=_A7<%wCA$*d%1q*0y>^(SIICYnhGy zYc0zkbDqJ=nZe<4xCpa;Yuiq9Yj6XC3hn59un-qpVJzGtf#r>rNIO)~{H>A)M(02a zzRn<fmu&#x|XcYUa~0ivs&h(#=-YT@7y8_!Fjnd zJ~8t6BT36l23yaX^A)4y zF?nwkZ_r!*KQr>M{A<9MI|aVu9H!`j7# z%&}td*gb>F8r8fK`3ovT_PwHJPj|In$9s&9++%**$KE(>L5(1XKi-vHsE?{z1LM5Y zF}nn|l0^HaP&wU&lF+ociK2U*a}%gwAWa#B>-&woaouc|H);~cCOC%&8c#C-$V~Ff z+1OTlHwGrsX&)?%VJEhQt@?HEA>{CpU*%=VYz4s0BSCC!?8!C8Yk3|ehD$blcC(8~ zE?IPhl7jdxpBpXluTg9EGTcv(@*XV5w|NdoXWMZX%l(U({5Y02zr?v4`22(9V9V<* zhh)5$HF=hCW!9&Tj>A9XEjtr=d*$uAc`}$c$bW8E}Np7*=so=*0UA8T-Cc74Y` zF2}R{eXIo3=w5ZYJ)2V=Zfnx5#9!ayN9MVF0_e>IOOYY-_%7+PW~#*gXLI;o93OVKhf|(N`#&+Uqu$lT3|GAf}cE#^%=f?U3Rn zXT>y+F1C@(WU|N|X8JqjWPy+FGRW&o$vXj>W=I)#+S@W--?*Fj-Ya3gvDan$tE{fq zxat_$GB=x6d9Jg@;qj47ZD4)K`@fWYu*}9D^4qq|7#T5Zyi>1?-ru_lV(j(|?apreG%5u0qcxJ1mi6r~XsdvuthzbxYh85E;b$kt` z`0JFNw-Loj0dZ=C2N@)JjiH!sdr}RWrJzn&TeuWs)f(yNn2J$8onmBsUShDIvIImhLVK9Op>V z_ji`YSQ_dwM^@pEmN;c_f5`8M*#1{7tl=ALnUeC|xr;en0(g9Z*A6pD1h(akd(6%P z&f(pyidHe&P6ADbZ~nlahh)3HY_K$%x+u-8p^aebqxu7haHR=tL{0!o!iR5fg*=ZA z7bACxjzN>xA$uJANaT*ztaXfG^IJ4Mk(C@ytWB-XKHwVK zKyD{}t<(}U5Lxt>&^pl4CrCS{V4#0%@iMkGci%H1E^Fn5%@I!|GD+DmdrxP`W1P8x z@OM31k}kSZ2KPgTw9VPy=i_+TB%R-7^SPJ zk&+0kP0OZ+SS7w;wZ_21Nr+_6KO;{@_L2l=wT(1B&S7eEL&t-B#cu0)ty&GnDUTQU zZo?h3ntlOCK)_!cT5?y`IQ)M5bk=}?H9p9NRjgd%ZL>QwHT!*3eNBuP87NJi zMYn%@9p1$1mt&14bZl!bBXAMO=fyJsR+kbOT=o_{{{SK+InFNuhCz4pv$G=WOn$3- zon`yxbKNwONfDM{=^+8aIS0j7_f^h;)$Sg`PKEDkbI3S+jrGvGz~bc+L2+nqNbh4l zi9urxXaN8^gpo#q=AWh1s`0;{^Eg9~%3`wicM-A8n$D!a;G~VsS=^>43yt2l+>YCQ zN)W$fS5#X;jEU3VZFVCD_SU`M8F8ziY@M%l9_NNDr~O#Rks~Nud2RAG-QOINv8C`x z=%g)sTQCXZ=m))^Qs=n$U(06}n|XA}=gH1nE9WJ(sbtP#F78o2hI#(7`n(nHWHb%` z06lguj_WroUwL7kyJLRH$#CxY#E%j?umoIaV5C{(9AO z?2IyaX+^wm+TNdSxo%m#uOPuiMcR)*cqJOsT7joAv)#{su*V^*%~ew#?}}7AB#z~; zYP6TILo15}vNc~v7<=VtU=9+uub=}`@{&!#DAK&-pKKtbQD&EfPj^B~JwfehFXzt&S ztv_kImx})Y+8>d~{{S@pJ+Xdw^MaH<+!S~|=i5!~u13Al`#agR?qxft?WfSEr^jhU zZT|q(`jy|qNqZ?JYTEX@{7Q$rx=zoZ==lEt|Jncu0RjRBKLFNg(6L2yQC^$??WyRB zEm=wIAwBZB^A{1w(ZPa>E8s1#a=#t;{{a8m00;pC0tP<-_P{(C-veL&+5iXv0RaX- z04_kV1QSZ{E1`QBN_D?s?x^`rNC?Ce`fvaRpYu^tr;9`8-!mg_D2+pl`fn<2NP z(FF*?;E**s(26nSL*-u8>OHqF!Qo9NEP_5mjw2@+egey_jYFFvi^9N6JAw_Eee{`y zK~lFGX{{vObY>~wWNp#(xjKfmW63c5$574pR~gO$wb_hP0pE6-)CL@xV`eI-@iAK^ z^;2%z)*l@q$H{>aSgu53UfTJOdgplHP8&_y--eyFEpiy1?Q?TjpbL*t ze{rDQ%$KZm-Lzj*Q?miPdW#;P6$})b2!V*mCCNKS`9Z8+CQ0I&cpQVVgLUa^Ub6Av zXM;DED3MrN&;fUFdQp=fl9mKt!W`IJy=J#1G`XB!1Y1>}L#(80_ODvb`g@hjGd3ia z!)7|PKZow=Q$;%w8>=W6)ag@jxLnRAJ~~ck!-hk?a!vHGHv?LJHVGo-zOH1n*-W6v zj<@Iya^F>DPF5&YAcAJG5=h6+1^)n+vpG&rlUx;tCGuctdNPnax2#-#I|^C&%!yRK zRAT#-YAipO`KfWZjLdu)TM#U=G1+6b&$N%lYU8+WKM8YlF=LHAIXW|q3$~tX^{kx5 zkt5@>;@_DB1V+(2wzfgm!llE;hFr{Lh#-5~%FANR-34IcL60%afGlq43jm;WxT$lp zV4njL0>W&m0bRv@3lm&EOy~kiB$U0tlHGR`M>~|kc|^Hw99iz7MiGOxM((?5aw9TB z!Q?_`$aIkGc^upT28&sXj?pI#i!#VLJs%=Nd-^5%?OMr`^wBRU&~3x?1@{O^;rHfsUa(mjKC~kAH;t z-Kt6W;~C)jbYRi7$&4cFVfbq}*|PFJr)xCRsS?L~X5viF>Pk)dqJLAoY+ImK*jRX+(q5kmx}c*Y-Nd zq1DBTo`$k`%yxWicC_(@B~+GeZR1%SUm-G-fp^B<$Si}DSs3~pNePg%vi`2M)g_O*NK_cW|u&LphH<4h!b!KCN7agGu6iy9;4&wEb zixx6v$AKE8ewiM!Z`JM(Z7Y2E&lz{HO{AL|<+EaCjQacJ#yS$CPW2BjE-dk7PB-Gt zngUL{TBCy|In$7Zcgu_x&C^4#YUKHdeJ|B{R@ne79itv3o9!dQv+>S1&GH#cCS0ep z5IVQtTy7Kz>7=dm*={}%=TJWSz~(Z2KTV^&$tbxdKKp;FxYmkam&1ev5kvvKjlRmr z@+yd^O5qXFS80kQ8Q-h>f=QP;9?- zHyMyh2AREg+wr0vfYR}NLl|J4<7l_)tF4XiP-5mrG8SY8eSi-&v?A5TrhI{8Wg%i; zjyP|n*n2BGj|TCGWlJ zQO(q~{goFdkCh~xRA50I41{@`s~c7>E=*7^G*Wtxnsk!gKwH{|4qU=tP8JOz>@C~f zNZgN|@kuyu9gnHBTEq&(&c&M|3^?abw`DX zn;sr4G5XIF0K}4Y7OXs;O@tUYQL+-2ADBnP1vk~Wiu}hJjgT4$TINlcz1Q7Xk}5VQ z6)ryZk;j;uUv2%Ax%jV-CN@AwCK8X$^py1#hr=#>uyGJ4sV*|u4dYc{eTK3nGD{yV z1~aZ6bFd+-y$$~W4RM)S87lbsx=OGmiJbQFH4bH`kk1)eaoCWpewV5E9J2g=K343E zd%fa9y8VWeKaL`9R$%y1H&O^TceQ;pog1I_2_Z*40|0va4Pa$*JTlFdG?vH8LITFX z*fNf$LWh+W(qxAg>jl#hXt1NO@ z2a%1oss3+eao<%zh4T2DOnFnSgD_*|X8sf*+J0o9DEHKk_o%$Kj&Y>G{3eqcQ)dH4 zG;jEIKHB4Xh@yKy>9_$;^Ije0F0?i@4GjBZ&57# zw77Y5AaMCMFtG5s8diDXqQ}a_z}pba2IyPBEkh?RJym=B><`fxi+M#N;Y?c}CU-dZ z@2$ul>iUx)6aaQ)FW8iE)Rcj7i*YYY?qP z@-dku2j`gj`0lwwA_j!^$x4hwx;CxndQ6Am`B!9%5HqtUklb2Q%Jm@%@t`PzR?|x z&z7GTfx}dIP%9F)h`NndpNh+d3@q}+V1`SD+B>c5Ck#CKIrv+>On3&}#YamY!&yJ6 z;7+`LJ3J;;7D$4fM1=nUg-HC2^5kO@<`$1dP&W+-=wJkyw9HVrFHR3269;O0-LDE=OwJzHgk}^S>{&-uJh{{ZjdK*;OSz|*xDW@}Em zF>evtr^A{wagw`0BK9Yttk0+U9Cb)A8ImU-6M}rT#$sW(h4Ps($>>9P-ngmqWk9?x zI>3QCyIRGKaeQmz2gk{maLd2L>GuBsRZ2ku{eJ3fOvD()2c_!9JnWY1dgaC`KPcE_ z#!pjsr%};Q@2pks7?V*C7dNaRIw;(03yM;Q3P=`Nm)hS6_ts|{!?L$9RJtSRrZ9Sf zblpt9E*8~Q@ORN zv9RC~az-^Wl~4j z7QaP9U;y`49zP?>sLbKm%aM-$UAxAo-CWjo8<|ph$&HlgR1FWcY931=q?&NWV~RDs z*o#{A$H%nC8sxn1A6=mMnv*922gr~&=SGLh!rce9r_JRI*fC+CmK=3`rEk-9t1mwV zgsPJWRykVbo2fd~xw#oO%2?Za4}nx4gjQz}z^BYc0bIPZtZ|0xw%;3%0a5Xsw;wdz zGEDpVJccl#395Mr7GB}2E`B~FTxU=sl>8EHHooe~z!JpS?CTzEXmu9xq>lkbGx5>-?$)wBw9txVhHQB=;*q7siJ4YwblcZowxG`C zaL!*56pb{|!gn@1H-*o7es3olIOCSW5XwFzIyJjBV9Ue2jv_{UuuqgnUnvVKgg25a zd9FIKuhx@i&he13fdLD1;75AI=5jHii!UYGY}~-Mkx>2(I)hXFDaB*M#$!yZCC(bL z-Pg%)h}54=B*~iC(Lbdz3K(CZaqp}tjPs1ioUTMMuaSW#akt?W$#GKTAJa3KJ{DLc zq;fyR2=~`5z;Jnyuhe*?`jfSTO$qF++Tx-d-yHDq*uGy$*jsSu;L>46@MOVao|M^YQ$L%laQVVd(AKecgO@tX%ew zdK^c{+iMlmr8m@P2+aA028Ih4V1dD7uxqOMsmhK1kX@w#2haS=+SQMd48;|uL`cqo zw;tk`n#YJo$8y5#Q{fXY#0VWPL(qyzWWzg=qS1kG3w5jxN0=sFI&=`o82)5R=Hbt< zABv7vJjL2QvKNoc8Cy|%c+@fy5z2tqI)hX5B}0xz1sG~YgGz_Z{A-TN%E{X>Wyp-6 z@~nz%RlfR)b$c059etv(GlTDFGUG_a?Qq>isn1cShuuUCK>(Go>0FL{fi_Eq3?p^i zDR6aKcJ9r(vBB$RKWG&ZBTdX}1F&#GBfWk^WE-IF>vrwRWai}u2_Nak8SHAFtMeF zi{o8dV+bN?JCt%dUWvxyB8Mw~r;7qG4xro3bJexh^?~&MG>r^r57x^qZH*uSwt{b{ z8de9<+&uHo%(i32m5&wUASt^Y$4#r(N@+RV&za=8cV>*pZ(=nn8}V(fOON6)fBPnA z(5Ujgy1KVR!DeEylXVgjN-u_HYaY=*X*orZ!-!M#7Rpm7aQ6$;wnC7D4t$FlZVqe5Suhc zGpE{X{Guc$nO&Zx#GClaqrx zVatkELo`|d6y2+dvH1Dp zh9+%2NRMyxP?lZI!Qm*4fw*5?&G!!-C>1((VsFx+o@<4Tue5g`ZF4e&3nVcJQMw(s z1ZXG@q;;%g*U2G2^wc10a@O`&5~Jb--M-oh>NIx3h%rNJje7elKcC8iOu1r{(!<3< zLY=*L0oQS?=j38BI^r_J0Tx7Sd33Be-APxC8O5|4wI-h`R4Ip(2#7cJcByjtDBfse zxx)x9sKdD)H7uFY$r=^f;n0J){{Sz(gdBWL4m?g;6qpUt55idcDKfKR-!h-fk4sp0 z6tF^U8bIZ3(M4qCaamDI1mk?TvXNrMTJ|-QjgYqv8PHsRaFTCx@2R9oN=)cu@*8f}!MVV5Hv_C6bH0$v_iqs9zzv8J58^elOs!?lnyjUt69&PMfJYg=KiSUFNpbg z#tsjUp5{Ut%JLQ0ZSPk3RQ*TvpHpqT{{R*K`%`3nPmH*Y7DSDglOX~({KN1bwiT0s zi5f-6!GwTjhP-8Da7fcl`clipImLAeAdNIfYiqcBY6vr;7|wDPu0962xgwij4-g=$ z9)$L;PlU{HnK=Gzan+T;y0V{rKNrb|gknr%%tPfNLAB~GXN1RWoDm${7e+UwD&pyatN**K8eW&*?1U2NCRbTgRmb z$&y80s-WriSLCzbQ@Og+dX4Wr&$5;^{IUhZsUXyTyKdA1{l)0xOBr~`y145}jH65R zJ!xT#dEy|4KZf1D+L3toHW3+S?2H#e2=-D(9E8gq)UXyB_11(?`j3&Kgp6*r{7puz zs))b|9QjVu*w!>0cPkNv8-Jr9xFnnEX*06D1(kriL`mMg^)-{pX5!u#^S7kLv0>A% zYR}ArD8$Tv+r;)8SNQEM67RrPW?qn?UrAN zX+{dgF0MnV8-ef!uLkCVRw|{1l334WC&IY@0Ij&rG&Ua}H_SY-*o9bH#^bGUpZ-w# zemS!7;KfNJi2%CT?rp>3A+OL^(_B>8qx7R9<>q5XcM=mq92;3xzYTI9No7j$@%S>X zW<9iJVx(*b`2}Tv_~VkuTzt6UF>v`Yd@T##XkvdIe4Zj4Y^)_o9aw^l>!#pGjby$- zF=k_`!#q*~Mf!q=+gYO-N6ldW0DWL#<*~l&pJiflIJ``gJ{B`ZVGg!!--wg$tezEk z;{#0YvPch_M<>d@+gA_uUmm`coKTDA@<=GmSZ&yQnxBy6;7?C05R$gvwWJP-rR zv^Y1i{{TmYI;k#MR>bzUnp0%uqVjkzv}43NqhDH9bo3MS!PgEc)qQ*sco$cI62E#{)QlNAJ6;_vRSMi`7yWZUNo3xgfT>-aX>m}6xa z;m(C|nJsou2Uqr*n~dQ=jwNr)$(2YO4{=h5p88gn`314MoGt2TS<&eTngsh`N zef2aG$Rb4!4bODxML@Y$AfI-X%UyLizytoOVQz-x(x3j6pTe>IhU4$79iZGO*T6B> zqvWFFrpUxt(HkX3-~b&69&WT{iwb#m0%VLv%1`}3RQV9$40MqBl#ehs?@PScsI?za zTW5z#DUAXK*LM9$rcne)*S(Zcl0`mlgS8&ip-)b=p|t^hXm4W74{r*2CWILO01IDQ zl*F2Y<`?Zv6cG$OQ4gl69k(}$E!LmXaI!lSyz6p(w5~yY)-587+SQ2AWM5-%nOmz5 z3I`=5ftUMAnwG4r`SKJO-SXJL-b)V=Shz|ENtbgW2Sea$IF>u+qk!$Ik?yRwkB({C zq7A&Zm+m^&2P?;LL1xEr@vv<#(343ABZu?ZSu9;5QE;Bne%j-><>TFb2-T8C7TIHB zq}F!_$`yx>Mto2qK7US6F=9@g>lcsX`DRQkX!heIj*LnT4g6~Zl;Lt>#t<1&X(e4{ zNlxYi`nOQ!G5IFQi;;7Wp6qmsW~v3hnz-&mH8$gVVoYXViR4a|1lUa=&&>S5ReBX` z1U~vsLVrRja>5Tl>9S90?|S4iGBVU=Wk$@93nW)M*}QK{fZSy9=4{wzhiQqBABbDE zaC2hdW4gZP3pR-Eop&CatJUS;#+&K11+xw9>C8Z~73kZ3E7ScqjVzdaZe(g^DnBRs zN>&fjoK9O}Vq?W5ypCGv^I1;TCYKyMJ;mTNYfNbgutjM-ZiUFVf!awUZt^Y-9*XLEiNeWa9cqVDh(m8xSn3p|xVobH8_PpKU8V zMU-CBC;X}(r=$wJw7qeNu>-`KjX6FsbGWWWj59{6tjf$huXF9E*rqrnk)AEbU4;2T zx3aRAK5}FWv_uDnx2`XUgCk}pbgGRcqRKVb_g4!xGy*InjAJyZ@UxMlH@>pkNW`&? z-hSbGnjD~%Wbhu5o(OtK()d%!2JB+2T1c*RziPQRJ~hfY?-&E`t-d7Qi5&zjPy_JV zzi89g0t+v!;%FTw2U2alo%Sg?b@SfP@~2(uiHT* zaPi1BQ>zLo2@b=oWY|zfVl8{6?kR)BA8%WBpujX>bg!7h$+YS`g@4AAA3qK(T)46Xq~>EO{9rY(t);5$&vAFPOkalQ3>fphE^<89Uymmm`aa z;S;$gVhOR(ZT4QX(~!hToJi*lMU;9?@K}!ZFBhMXc;(GgD=Qt$tThD>zOV*ME=htC z6E`3Ow&SS1alB41Ck#u1Jwqa}{L+63rbZFFI|B=#yLyM=G@NqbNMEdGbwAE zoizAqV9PN|_@AE~_VPLbd-&JR`q;Mhy$$z24fflm`qa5@M2c5o_Ftt!?6w6#*WFGR zMjdMz(n7GSl_T0J#l9^Rj@1Tt1D5%mu4w$sXpf!-9|K8kM0!}Rwlugr z-dj22&`poC@iP1?Plc6t87X+c>9?Tpx^+N18TXeQ>RJ(31C_trT*Ez|9;A&At4x7%5`3AklW+ar-#lntxj zX`vS*p6!o{#2DF&1L0zN(@Z29kI9byZ+#Y!k}g0A6hLuv&(Ioun{0oK&d>J{&FHZ*rRcI)Ok*R2sT z-{ZGkDN<5hdIEb;1lfNH8d`wlBl(OHHj}30{-I9}JcdkAN2Xii+x(`N5F)}oFeZlA zFJaht*D|>=$%Q1cm%&yCa2`7N(&b5z(Ze6YK=ZKnTU*o-=Vvr{k~orcIVFOPLDcoB zv5zX*w~qUuAhIAGE9x#gA3KqghA=X5a%Ce~W&A^P0BwGi$7SZr^LgPFnBMF`r_Gxc z^QV#@O#>lxk6xMz&W2uXxV|?X+SvB=q}%Mhn{9Bsp2E{dG8u9KH+VtSuUtI%IXL4B zk@I0Y-)YmNmo%AKIr#DA(qLA2ZexDa>c#6x!!Dyn&`DKS4oY@eOYnIl(NBKzq09X0H znEhXQ^fqq2dT*Zap_NPeLsF`JqN9y7RH~>0dGjB#nLT4>TeOI*Km${)aq;k&b5196 z&}1;$pJ+9aFD*PJWto0iBh(~hdAECLwk)#Cht5M0zQjNW;fdtCFKqf0R**V6M^mq)>huX-(xoHpUmVH>rD{Fb7j=GiHtLAl(dI zlz5unJ_Dz^tSnsgt?%(w{{UsbV6xnItH!oHiJ=Zm%B}cGt%2@)TCclvekzbdAWW!5 z*&9oP;X)ZyjonG1u>M%oP+3~`=xLNb2Ig_+2wY<@@6Pl zH%lU=6<&OT+qpj~{`@qd^fL5p};$8jEzy z>5V9OaopWO2G&mlS-fW}9z=YOJU7Rf#?30Pa;$Z)r%nXV$fUUuGRGq`7IUcH-9e{r zK(8Hra@=kuZ|iQ5n66{!J_Een?2t zao9|lTzm>F7tL;!5J-WBhXvM zi(OwPQ?>}YyA!9`YR=?2JgG9Xb6FqaJ5VN}0PHVZhbD^Y!Q$ft%9*-5ve+R0V71Ed zxChpo4lHg@5oBu?nOK!ZW7azZrE!vQnbAO)5%15%V0He9!)obbX~3fs^qXSqn0c{<*^oJD#aUBARTs}8suHNiBWIWAV3u%=^VLD74kZ8YrORnS*k`zpt+ z4La9NYQ%%?s|^9`Kr8r;QjV z8KI3aK@TmCwPBG==G5-KRmB^0%oM38te~n_o+>TO|3p`hlhZfd7JS|*yE-q$F z3b^2Mz**xbq3igmF*vSr8L=_|c*aH2SlO26Ux!+kljpfSV6dw;Fo+hpQ>D*J+jdiOJ=Mj*G&^u`5`2S^eJ)pRN%am^zP*zYGw`fr^|$jiGXe=?s5H*8 zfM4Qqrk)n>T;^=(trEckGaXvXyUYGvYdUT@L}@B{m4BMuPqa|xA0@e8m<42XTf9MU zv>SdJi#`LkOm?`pd-VtGtZ6u`R}2rUn=>iV_S(eK!O3LEVvaq6H0a0$tZV-O7G6YR zz6*gAF|bl`mA3|4_%ZVe;(n!U_sPsNJ_M}F0A(MGPYYKeV|z}XZ)VE3g>py77$4tJ zs*sSX2i$v++P+lEQ9xC4hf;%0wUEo@iUul?_|#ZPeJ64Y$?QLb(&1s9h(_7u9ev+z zMC>k+wZQgwro3LWsQjfJg?f`6p?aY3G+>VYmBUdWZ5{@#)Y0-?RCS>oUlG>h!iq1( zqRsEC5o)nEHPqLDpbnz?)PQ%Z1E+CH$}G3mo+s))3IWE8)QSZ)2V2#@m!R>aXy#>% z3mDg>>I|8&Mr3$6{;a46n7W^cAL*>D9GFHnA3GdRnK2ukhR|21Me(jjlH#I|D*_i> zA5IZ($wjiT2HtBLlL4TT37$b4q~1~ko~POFs2(@RoiI>2$3eMUXtkW_V`_#>LmNjd zo3i-qi?>d`o#-;n4A?MAVvYq(uM@9DE8$Um)Iv{ER1OLL(q6%oe=@amW>s18F>xaK zEK>VgOR3u;e<{6j_?)gRv+{g`eC|w_2E~sT)pZVDdSWZUxuW~$XlX=W8i#m?5;U+o$@gu-zHX4 zXpVO1qsGRcAAw;HAR0D_hr%|nJw4RVmQR)l)tk2ad8t_>y4uy5f#$TaBh6*qpy^q$ zW=J82Er^CkAZ}tT6bg*T1Z@w>HsRV0oKanPELOK9V**^N81LbdvFUI?B z*FM@2am=UGUFmUq-%C~|c5^JN$)Q}|SX@LJ{{R8)tskWfYIxjCftihyxU(=}@PY8! zv)cL@5CNz1`zZVCIgdjCU;ZkCjf3KGx>g^B1!POQ(ZUE>4|23KBw(+XE8pBS6dp6Z zhJun7U_DW}ymX=e0L*WNd}>MZz+9$ZBA;yqUomYg#M%U3jMGM0 z&ftAC{8pgE6rBdEZEI0)8nHUi+pTo1de)i>2en$atq^qFE=>X2)LDKZ)Kn=ff*9QS zo5Hgu#`ELlLSSg*(a)*)mh}85CyX1DWmIlKl|rI`EW3|Q1t&4W+Zq_7^l~Rv4uV^N z3vKUeha)W}&dP;CLLrE*^+Rv~rfkkJ;f~iLZ_2Ws_wQM7M;PR@i7f<*zF5iD#P_VM z+}N43a$e;CJ3WHkE0vp)!5H()3d@6uHk*R?TEJ|aa~~ldOly|fujH#*-L$AD;zG+F zHX_F~Zl^(gHE(ByVdO-)Fy+R<9M90VzL{n$;+t*^zT-;lf?F?bJ>0F=E zaWt72DzW5aLnf-k^(=nMwkCOk6$MpeRRj+Uk$S#HRI*>GXJdcgUY_g3+G|T95O$E( z>Mjk(@2&@w8Ii*-Rx8BeMg20JZSS>ZFVy@`&*t$Gg@-p8JgkYRG67syJI10M6a_hX z0SIA*b+g7kDonqnAoTDtV>sDaDxnfb8ZOb_{MD1kVkfOU&K-7qAQ5|?b#dGW9XSy3 ziW@R$jknKx^&S_l8K)ArD?TK0<7KX;+Q5}QHH({(#=jksuoz?Ym(4iqxvF@ghja2! znTU{uYlHYy_O)_N$mK`K#JJNORsb-^B?)Nme-`TXY<^RH_x8QEe+mJl?eL2IG^4|Z zADTxUWnTc_hMN}^L@OBbN8{`#yBb(#!TCIB_cE#cOnRDQ8*L*%J9{WDQx8;Xj65rJ zjk~U+OV=TkfPl7Lq;(Y}f-ob2l)E0PFKsKD!q3B!ZWETp4F*lOAO%hDNyg;BlQ4o* z#|}O?RQY=NR1nObu0C2BU>6#9Wj}pMhYrQXRXQJtQ!}r`58qK?Wl8Hnj%@&Mz&({L zpkH*182H+s*mR(etV}y?k(+$%JZe;1f|p=zrN?tpWIe+j6!Zt%K$COrt$KJ< zvzrgXD{ou(qcbrsO}5hlw?KZ{c~U!f8-@P>jWBpa?>mkB1t4MN0C{cUPMdUXI#RO4 z42$str5HjaX=}QX*1a5vS}7Lc9SyB+`qa45#*!GoDq6~2dleM%q)v|?#2dLCg|DO;bigU zL0M*RNx$jVi5Sr3haN_PQEz^=gM$9Nc{w){u|mj(*8<|QKU9iLem@@?Sxj;vVI{g4 zyf5siQ_S#?%zS9E0Iy zZRl)z(wMVWDd1mJ;-i4%E~etL@_6T)BPaWZ5eZ@C@wG^qSn=YcYy}PX5_?<5lOi0Z zU7wVg?7wB2xb9~?rBR6+MJNCSD#G@o^$rY^ESU0?NTxnkPs9;w-K6Voq*cz0JUlBx zK=7v4J49I&C1!jeYKYiddBUvk71;Zhvme&4}CV(u)S;6l}}Ez zil7y}tZ4d?joNhaprSv_eMf~xji6(42AWi=tahU4J@uo0>QXy)qDZkm73Bi-71z$c z1AA-bvf%=M!H_f~+|{%pk19P9W!B($-k)d# z>v|xK2jcNHU}<&t(;)YX*HWhS21V>*b$@P#vvZxyf=qR}Zq?4k$K@P#mP8V=jaVl@ zeU${<%rIl}=ViKQsC|zt0$A=p~dBzSxOEnB?&6u zs}~;1X=0QA02%t7Ky0o#fe^6q7q;j3RNQ>B27zVtR?Ff!)$Td+lcql;^kwUe7-VGqZ!AT0QS=L^oJ0K&yR|}r?XEFvawgwP`H3> zBjFYn1lC6z$fC*Xks!z#6;sl`Px;pQ4^!0iw++7yzv-#6W8`;*m6A9S#fVddc;p!(FtVwT4Mc?KqNd2<;fssnq;E{4+-=#_hLxGYavZ6Q;V5ip z+vIT|xiZ`ETgqYD#}Ee68Zg_(}j}qZl+Ufl^W60puKCNgHjb=%Z+y%_zIR+JH%`ay9!+g zr+onH?4aaXx(eKSuX>Td_^q$A_FlEtuU#na8@vaFY;o*UOaQ!Xs<76CzMw>LxY>Kq zj$}4rc;z1p4`HNd8=;VX`VSerO|IVxEP~s7YR9OnuTi0+f5a)5*-~S%0!GINZ+k0i zUr_BADUiwsxK=5}#aX#2$y|x@dICNn9qLT%iC2%B34UHS#RA37w) zBb07eQ~bpFs{`k7qr!ZQW3pJ(6#k2QDx2m#Lu+s=kd7>jmRT|;NYXi( zSZ!iDgHu7q86`-zzC`mNnc^VY;hOsU%|(;Qyj+2TNS;QPG+AVch+e|Ot$*ZSN+?XT z3eTCu-Aip@a>@L{u%XAAM9nM)JPI}uFM(R-kJNLA&RKmqql^z$Ev`E4?^qn3Fn5;} zDnqzfGN`qUst6~ZUN?|}B>9M=UCSfBstNa2HeOar;bZ266EXPNRB2cz zBZ}msF=XPoRzJ+7=;QiZwM0i}llalF>8&XpYZVhsqjPQXpqAYc0G98qZXF1#4y0;) z1$@aDb~ZwE7be1_$(S@b6DUTC4z(N-Ht9AO1KDbqI%T|3#$Hl7g;A}l9!C}zf;@rwVqvyGItI7(Qt?@m%`QhL zGR29Kr*)igg2l`aPxI^*PV`3W{&@9<2&ZU%$wt-Y$kD@Ouvm7Ag0MXoCWVrd|0 z7FUQ1g16W1t`mq0ahEum$?#G?=SdOi8mJ#&wIYs{x7$ijZuG~sTeV-e?AQJRDV2r6 zt85$2m+7azmnoF4ZYZ)(ZDfnOH@mebFN6zm?17vqyQ`kNg)EExsR#v!jWvdV>00P$ zp`gAMz~qDJvgCYc;kAusa^F#NGGRr`WJ^2~#+Qo6U4+>A={Sxn9R@Ioql1u9=A1C| zHQ0Pr$z<@UkCBGC?HfUyfRacW+@Fuap-xH9ACoLY8c#$pRVX}~isXKu<*}C|ox+Xg z;W;tpBoF&dC$_$(myyPo7D=H(5=ASx0>QT5t*Tt?&M!A9{YROtpjK%K>3vm&e818B zRQxu69A&blWf8%V6EDN+blj)LxHQATOxZDlu6S9H2^1#MzNK*dH#fw~hrxWA!O02v zh=PAA68``RDtK`6n-lVrE4`8CFB_jXU0>I!*8c#=qxnBacgEZIb=~*&sRL|e2J!i}-|ef<3EkgjzrKiMW%r7Jw3}f}M%*uJ z>rID~=f%iEge}g8#;`HCd{dm@tZ5MPfQ#)v%TQ+WSuKc|Q%5Wqxa3?)Iy*V9sJR5f zlbGf+rfg_8Th>)5Hnq;xEAF5<95IrT60^k1wrJf?!lt-cno;^Jn9{nn@3o5GV_Wf8 zD~F!j`1f%x6p=9tZs-SUem;DHq(;aha!%qt8&vt2@rd(ejHz%hYjmtUyw_=RGjmo4 zF_NU0&~EoubQxmsNd$`G2AWfJd7C?Q}q~=<;!jLS+ApbikWPf+2sh1LT(a98nT+tURcYW$-?4%JjhhO zGbQeyg@s4XX6;O#KO$i!{TfE-ulG|$ zj-*V(S-TPLH5+9@C;53jRAxU6o;1ql(2!SOhf!^7mE#O;6mZWY3N;t)z?+(^|&#>1!&n3+}DM zt$XS}Q&kcMZ?M-XK1|Za!2@A3>KTBs1o%*gCLQs(*<~JZZLeB~*27%2Mz72;v%L2krX7xkCY*nxhRnqd6z9I4h<_}@=$C)FQK%t+=U{P>t%m942G zved%`+ao2E`p}VRg|#GK#<;JkIP_D0%(;GJ0-mSXry&q)(K={#=9mYuh>LanM-k_V2ht+&3)<_ci z2kg}(X{27In&S5!%Cs`tB^M;wfd{Q!LG;=o>oD$pl@p_Ei_8J&Y%4jXbz}1ku>;*y z)Nlxa-luG;G#062Uw}88y&z*~hMu2e@25l-+lrrk&1Fohq+MIvS0?Rid#hR?ojRI^ zc0=YKR<}xLw!aPOU60|U$QRHN_t%!SuZD$%4%Ypej_4+k)u`6JhOb>J)~1N4 zZR_ycO36p+U&=fM74W6kTUE%p-B~CZeK}Ao#`;_z<*!g9tN{5W3TzpFtEF)GJf=i> z9CmcYa2Zs3u`w3crDj2wHRgb{MFym+YkJ5{hF?cmKt5|7dRNr(XT}?hkuLdY*VyH4 zwD@hT>`dIo$IDiB;$$qtdxj(__8%IH{{SJt9A-ZcD17YcgCLt7KMFDUx>VU1Y|-H3 z<+EVm+V@D!_g}4zaQQJxU}^C}peD)`6!N8)0w}KEHXG{R(&^%_ODgs--VF= zSOV%GlcvMd%UKzCAmrglmq|xWb+Pu={$BY108VlFVsvy-Xr7!=-P%%#3#t?4><{*0Ov?IuZAp&Bh1K zF}?gKsp`Gz{U{K+k_AJM4UvEZcxgpOr>!rz13^?>YWAUi^=cK~_1?6r_OE3DUdr(I zRqs~<_0#O44{z8hn%6&v-BUSI5&X)1m!=)4dj%%FSbrMA%g)95*h*}fPgK+5wM)az z3(bR-DU$)mlv%ag>@}0b#b8^qAHcOY5OLRUTWkT3Y z7a~X7D_cMgg7x)*%26^rvKF_($7t_dME?LGF>WeUlQm%*3n)MQR;=Vd?q<)&Kc>bM zu-j{GYjh&AccR}En3rhO3ty#h*`!&cW`an`u~YbLDk)`|!Uo!|tU(|hY%kil9y5{5 zUzTDdURqlT5&@_UP~va^avXReGvaaNlltKCYt(#RHaNoKpc}FAmdesP^)dnAe+@&& zLus?Jq}$~e5v}!NN8uI!0FQj|^>=z-Kex&A7QV~$psp`*ps4}U)b4=pTWwL)*2Y4t zu&W#UO=(|kauG=7Qd#1HRvMcVZ7*5eJ}G8{6K2m;3%2wONw>a;0Ye>aZuAtQu|E~G z{o;c_?-tXdq>KTzN|+NEy+gW0bG#qEvT^h9MR~;alCOX@y zllWM6KKj>CNEMzfs^i8#d8~#a?88ItHP*z@UkbBpV_2*Q#r_p-Emk6dYkSXXvg_LY zt3ma1RO4hyIXicJo zI-6>K+F_^!>3X&48{0;45@ce^pfeG2g}iDBA<1k$qs{m{4ZB{T*`J2EoOzIQjXD-~ z3v*>|`-O7Z@I}7~=O-RQEWj4N;q0jgNK4(aPerwTR_IHj<;fbkJxKrq?$)z%^NV9a z3~3{>o1}sI0DYINNaf1ei;CN}KbbbSP)C8JkBc!c0m}m`#uono?g-Y`)mOr~33)75 zn;J<`v7k$fU2Hb$I-24}RuqYw$x5ro@lxCc9n0Ut+gzs_jkk@!WyOC^<}knlN_Mq+ z_!&y47@4C+Qlwj_TF6bmsdBu`hR~vJmJn`P!T3>Yl*sygNt^Xl@-{AhOzHW!D89>e z@UPMSN?+8yFPi;E{{Wlw*S`M%=BVU7>CN;k1*$l%4(n)X!s%#*{{ZU!gWp_JzDZbrmPeJPs(sCVUkx3lY+zYrna3lvv?X9xd?c^Xg0>i^~!^zEHK!0uxr$S`@ZTV_NF7hN<|~${xyhzTFb3o^h+Cf9@NacK3EZ8I+I2` zZRs+dMxiQ1IUyvyl|iVt&*~^PVczH2UX<+bAwK{bf9#<(%yG9xZ5o0DeZuN#3HMPT z+O#0kZ*Zhgct5s2DG2O5Db`Q+_|N5xugCsl;YChc5!43PN-LAOje{qpbGz)M^y6dd z3$rlO-AWrp+o7$%-nFCRY2;z>zTka|D|zqx5e|f&zimr{E)0qu0}+v;AsxcmNN*nW za3%HNBP_oIEM^h~bS1-z2Tnn##mXjfQq+lOOOmn)Y7` z`Bh@7?RJD{G_HBV5x+eG5H4GEinYXXQDH-wCm|S)F*rL^fCoX>tzq%sQ^yQxmlH)K zAcBOy!lwTK3s)lq5ee~db6G?TO~NZ_wD|iexhc56SB-o-<3{cSM=FiV_V5)y)L&4j zoc){H55MWtt{#uxS z__OK3jSB&fmCs75jva-7P~K)>PL+*5ZzAo?W}(Q)Mf~mL>$Y&-HTsX~aew5$+u67M zzT13%cj2YiQ9|R&SQAlg_(1F7SyJ5Dp->0noe$Yu)QuPtNh3o9wz*3V-W8GR2AUY$@5x@9PQG9(5)^X9+WWh7 ztO(d$!F##uU_5Be2jxixoo7?3kZvq{R~5~26VH>!te%!rFZRoTN1FN`v~r`z^wt>w zNd<&~*P{Zu*CHgDrF0R(9gWz4FRA-!m%p~0R)(Ty9$mlzSkeKbCF`eJJLKH9)C%Mz*fnddyb{GG>YQP zojhm|dYg)|>qOeb^rd#Y9^e}Et66EeVHlc_sx)4-1=8M~N~PZ1}{UsJa#s+xM+wO$JBGa{P1OkI7}< z7~N}zQ>9@xHz6M*%86EN;Un#FJZvAQ;X=O+N6SSNapUr^B&yL7Zg*pqgqz~~h~@~wJbG5H(4 zFP*oN{{YY9wwZ6`u(!UpuXB6Uj^Et^%UKGYVu=;~=UUhHHt&VGNP}D7tw+f`{@}M96?z-Nmbsb8T|u*Y2!{Swlqs01^>!X<q1Qj~5v*xp0{9m4u+S(!PQ#}g@v`q<>$hPJxX zTEj|v>Ot1ExCDT0+WqM{UK^6d%*TwZjS~_Xd_x|Ujf<7Y!3@ZQ)!xBCu+$28rsktE zMQ~LvNh7wD(S1L}l~KwnvcLd)`%vEJ|5Ft8k4Q&M;G-Nzbjf*6Auu?^}w^3gy^E&#QeQkDH-EZ*IN(Vwn zJ&ho9sW%||X`7(%u6tXBnH7DWS|74^wZHXc+4dUxcy1*6ZD#LsqrZht-5K&EC&Q;| zl-s=rF@oi9df}{hF^*2Xt2u{Wl@=1Jm&y_q4XE>VCiRPnhYCR=#-d`KSogW_T++6x zB?JEehML{#P({HPBeexo^N#fyz{!&xC3O?Vr^vvb>U6A6sW^5L&x-(joDPAMfJ-gA z)*le4QWkyLeVI{yAU}mg$8gxOM+^u9D%d$AOR=D1jL6sMTgpD#GiFNMr!0TBgr?C% z4@>ydXTl=mFxOEeYxgd+^0q=}M!Ga|s8MftwJtt-Hy_B~Xj|SkJuVxyUy;U(AsAS* zfI$bq)V!YoGsLli?TU2(cqpL0u$na%^XY48LPF_&q~6}@aQQni$*=&M-1N1o4p-_a zCd^&m=~}B{X#x(Sr(xHko-|I#K}+Z8Bf5@uDJ)m9#FW zKmqMrxb(Dq!mNoMwDn2a-}Mtol`0ILXDt=-GNfNJ1FppIwP)OUkq+B`?5-kPT%_Y) z8qSj$T1ki!IJL;VWILg>Fqe_obR*kY*|`9W338n>CXI?o7#+hcWqgOK*g`@Pqv^t13xt-`jFt6Ay$YO(Kg*yV03s!ay)Ny+6#>)QyCKOqP^cs%}!-I}X z#S(oan7{Tt^88K*=0@XgBmEy8;UjZDDcTY-V0tIvCmG zT|+zc{{W)D{{Wu+)<6D1==%QvIdQw^Z@X>!+xjX}Tnv51R397A9e|_z&3!=QQgc{e z+$})Wt0$Mh$Vss>r8$juG02wG?LIY$i9TE8#mB^bHIr6CbRd(dsGi>(uGh7%tvZ2n zP3WG0EpX9v{Fgt%voGA*^}}xl#y;9#Q+mF(P&?N+NB;mk055a!iuao2V|Cp-rQq=- z3V2lBL>_hUBdtdr^fIE4d2K5P9Iezs74Pj^SJc@d0*qpI3s?f?xXF*^!pLQ{)q4^}PwVIwC-Zb(nqH)aceXAw7mx#O z+ScL5t04HvzU%ud22443;O0t&V{WM1h1cCtdRB1pD&2YyEoFS1O!G+FEC)gFsd!#j z5k@9RZTeleAH{lNoT`W~L@gHFCMY3_$N!vzat^WF1c=mi3 zQ@Av3-`zw@kaFXF4bGrf(br3c$Ou>2d;b81DyYx^KXqrVuD+9B?yfCgp~)Kf3!0Wl zlo@1%hJOD54RidY#F6Pqr%eU=R!ZW3YbxewN0^nl>Q8!?nB)|v8OJXD0kp226nl@w zQvFvX_`j^>DaDFm8N6%JW3cY4MbE~hzMdrO_tq~Ykp;?fQbrBo(!f^>cM`0M1EjYm z>~C;u(g?eecL2RU+t(rW5t3|dteKuD-x^V}RK2#pxKteX70nEuIypeh;*n113bplJ zPg*}vO)xBAyTr#*K((#fmzKiGBxgA}xV9{aq~Bi(XyRKGOM<$6)s=wHAL_W%wo@{- zq*sl%DQ@drSJlu;GC1eSj}tI1tsIb>YCg|3bKm|*%tVJabsT5Y_^zNxqyh}9e!IMD z^&dz709W}hpY``O{&RKr-v0o)rpjqcYG&bv4YHS7S??Bd9uE zPeLdQd*~k;%ze9=VUow}sZtw$emct`@MBzQ>%V>e^GP78bhbr-ipE_Sq zfN$E`cvX&-4q}V^xMJ(#DlSV75Ha%cy-9~rxyH8lUZBTPG{}XIx!e(0o}jt((2Dx2 z2qlcr$1-Wy0e|B~Wor@J#8ax{kOraqQ$*43MLqNsC1eRJcv}AeeH#@lf0zUJno_Z= zh0}72k9M`{7!oqW%0L53g995$IRe3)byQtVPm}>%azfmV2^3>>7b8p3B%qiP%SL)y z`hX~KdD1kvyp#hYj%GWEE2FLbl_$%}L;NL2jTD`me_}?I20Df}zO(?16?}UEn?Gd0 z(!N}vCxZ2>myj{q)}nw$LKyI0#v!^@-l;~R!0em4kkucAv47^G8J}n)Rk`zm5k9~eqba`=TdhM z8y@w@@_eo%jIhQiFdS|Lj)1B6*AWg-5#{5mlLf`XjZU7*byzp1Ol4CeqNpRrpvsps z^*^Vgot-dufYHduQow7p)IU)1a5T_M34Hu-PzYKwHQjNvvi={%Lz3e%y-aNBOottS z4;hiKunyo>FCWDxq?yR4E@HZkDmw})LT5Du$^N&Ce-o-$kSsWG^z{XDra zl6|GhhE*gGO-aIJ;0V(*A!D4OExDsV5&K1CMT`K)hC-2mZ>6YSoF2ZJzqr)c`4@by zC-R+7bt3z|>co{s)yS3|qu8zmW2ovP7Cs(b4ShY#Sr-eM$8I)xPn3xT+%Z1ihKe7y z{uIh|xV1@S^*@PyHRyYR*3|?e<&rgy7)Nr% zf@=;rSQ(fGVQp?lO1Diktfq+p$wl6m7bLE_8pg!jQK{%Qqs8CSu2R^yc0dk)okF7_QwEtgLk(Ab0Cxn5!yX*kXUALL<}sM+hW zyunZ6+D4RpodN4vpy&h+Z~l{9mn$;aaS^sL=v~;{RPwf|E`qsT*4y(j0rp4*3AsS< zEjc(S{UkZ=(lzG}%h2vCX8$a`3;kc=wY=m^pXJz|1mw zw!f;Y_-sA(>*M0d23oi|IP()6(?$Rws+|gvP;eNSGVlXql5jr)j;L`m2xU z@Z(IbI%wlFG%xCG$ZrK57NB{FZGd{p-K;6#jY-(oiq$4CUAx8SRu?;~p89HRmXD8v zCmLm+aV&_=2}K6}UoUEtJmp23k?6#nUmF^hlnW9a3`R zTDrQLKMDJ2NbBXji4+aAZkDAa*k7%4c(F{;B1;*F+}zy(AbqtYIoRRyB$IE<5DO4I zO;3y)(-65|AtR@{gDzB&i~bj3`+=?>9z6VqhFqxC&`R2ZbsCzJAN%;LZ_96t3wYE0 zMaIMPI71yGFh~8xBd*moDmt`T10&4~f{dUyf$LnBUmFOS@v$O|zJP$pO@KXbSIAxM zlakS}e~L0mB7Y=3!iR{biS4n0zqW_fe@t#bjp|6CUfv;xb#KK& z47pc^=2b_n4D1pnduqtXO}_eBpk>I6{{Y;GhEjcmX;v||okgqrXWc}}A}-#bu(0>h z`QBCqDlXB64f~enpNIMv9iqaOLS_9&I58w>elJB$i@m$lTu&ScKuhFI+O*Q2joABZ z{418p2HP>7!`o`)WR=aNloGxms6WP~Xn=0xUin=lH2GUOZkq5ppm&5|?&$ z1(+E%xN26lf{lOnQ%ktqjYWMMhTj3gigFK2h}M_t49vC6A_ zJ!=)QAzXM&!Dq}Lh%vUSQRF4F`1zatwoG3dht;UZgkY2uJ*2lSfeAy_~}p2`Pdw0Ed88a#PK%W@>@C}m<} z!kL(I2_u;etmsh#SBy z>t1&~kc*GZGy`*)#5YhRGz{4}l9Tb4o3y;WcsSTqfc~h~mmJRoG38{%8IvdAQP%hESbUTmt`2@aJj(HmS#(9{*HdcF`N&<~ ze1j@XZbGB#i>+euILKAQF?J%tIQqPY{UU}Z3o$%L8@30Pn&T#5*O0Hf;jTj$ij|?o z#*Qhm;XY*E#>TVLk;a+{F(5N%3=&D1?gx|&RBK7}r>=O8Hx5QzOjpuur+&Vz?X9%) zP~lk(kB%F88$mq&0-n#?3`+PO8$>rrnZB6dFeWLWO!hLO%00PSl z(g`9K{)VQ`%#s|=H6SiB;afA?PMs^9l?gG)F7>{K#16H=te9~-oN!sqw=p{0n)!9c zj7sv9Xrn%D!^YLSc%Ag4|BqWUnhT2f}3V!?0f_~f9;7Qs~+AUG>Ic%#MBN%|N zExzFHzlCLRv#3aMGN0u9Om24qe%d>T_E86~V?cDPgK(^ATKJYoKL-+xok#uA%z-}Ju<~aN(nl}1ypiHcrmdI<7Avf{0H0q)}i+Q46j8EdCNYH6+_pS#6 zlYRN;5hW}-lp^Xr#nQO^NR+g>IkBus)SU>RR&~hdS;I8U#Zns^^y%fSVEJCG8PJ0! zaUMYN_;#k_ak62_%;l3_Pamqo{DHhpLeWeCk0FVI{{RiZ7pbI{VFU)sTc+e1g$A0hqo{D#j< zqYiE@c7(kz2gXbHF$E!~yDA9ZJQj}eFFECffWRTk}7d06qZn_qN_qT~YI zYm88yRDlKibf>yp1vj_4vt2X|l`fy^q3r8Sy@FFA8hMZIt|yUGxBbdRDd-A0wW+hQ z!wCW_4Z4B|z3)NXfXkr=t?KvCRmbc$@H9z#HTzapJdd|D}NkS9u&Bg6pGwl-T%4;U<%IM2+IdTUKY&i8c8QP$uH9M}aaHSFC1(I7aw(teE{k1X7;_w_vHr#kjgb^1_bn0t4 zd0ceH$-_m7AdbZq2)|KkiTV+b5dQ$EipcOTZSAeB#9K`T#aq-X?-sA`-k6C;LvTo} zFPD>ZBxFp3Thl)on0DIBi9eYSzrvk4Y!oswHtF%($Ta>_PaybYmPZ;}s}n>Hriz}v zn7ui*?k!Nln!l|RPYtO z!MV_iIatzcoNHs>*-VjQsgblTuih)we8xuq0Oeat(wToV$8TEvs9l(kgpDX9vOG{N z?2!(){%>^&3rM7&GZIY#^SsJO#O-nSR>V>4YB9=fyHH1sBhH(!*x6+8p)@iH7Qp+j zM2f#^1IB^-Xe=?af(dsW6-RST!{D!_#$dAvOMl#yY5xFCW1_}WE=CN;vj#Qv)C2zG zMlJ3@Xd1(K;c4U(DRUS0_X@m)Vb~0rj!Y@p`mH_15GkB53l%L^Fc(0~#u{ZT*+Q-~)dMqS!7}0U@ zlXPkiYXM(1@~77I{cleH0G+nq6~Dvw801~(dV~HHnF{0t=)M-LeEBk`#mvIqkj(>Z zkZ#jTMT_cgOfESxN9P3Z2L63D6_cOHb8}CfJ=#f)tWRKcqs6SAp@v8yNxXkBEo7s% zi!_nnz14`U46M8eXO?zp8RSJ~Epjh%EAzO#W@~lylz#zM{UaFP`Um`|EAY?|dig-C z+%KkCuyG*^yB;!3d%dc3ua@O7<8ibG$b&~eVbY{nxd9GAiLDHS&2l;u(EXGHE~K-A z>?V}=wN88$Y3<>yV5?}-&9yE5TkBZ779LPeKRxAnv3A@KO>ek+>UaYd^vf#_)sYR! zUB^?}f)+NTBONWJt0H+^Gq1wU;5rIqXB~dZ-H#EKm#wRN#6roXfVrU-G;Sn*ts?1v zeIvUlKaRqgivWY+(zI(5H9aUKV|q8lE6S$FsMf-Tl#xs2u(w(D$#Pi;0dQZ7ym z)*~M>F*HKn3f~P+H#PL>#>jQC%j2YFZPu|qnX)Y@6Y{GT?ZPK3W;f_S(vkRHHe7tB zQawz90NHMUUZ%_Wm3)b}NYJlgMnx;RKMStlX)le4po@kvElCC}v11qbpBfO2Vc&4C zTlE?JZ<_S|ZO47KU#&ql+ZhGT=n_5E<{BL;QN&~nJjFZJ*jyWHV^A>5#0sx`rz(8W zgQynoY7CaQTb};f$;wl2PbL??x@D|~8W|>*Kz;5_eJFT52=gMCKhJuNw5o14DtvEJ zO_9aN1IK-^-*F?hqKSW}nPyh80ms9Bwxy3F1>z1xrMHwH17A`90Q9m1VEdFEKKkQ6 zZ;OIyWyTUoAyK)F`jg{QzCR9;x~n*A17f0-oh%5q+e_}L>@Q_o)Sun zTyT$D3BexCn%5&3=#!Kl;xAI8s1coa4(5uHYg0&;(OHo^NF6F;C~SEdyvj5y<4gul z=!@+Iv}!3mwK_LaL95GWCqM;_jS&rnv>U7|Zc!;?@eY(2NE{LFpuxZ%H@TuGm@eOh z_|OZKm+-y6jRO{3Oiw_p5AdeBQZ3~stb<~yxg+sb6k7tyQ15>bDNkZ8R|2%Ff(bfp zG;wEnM6r?_sumZbWXO~)yvNcvtX@Bc$rE}C?~w9&0*j3`wzc%n2#`wh;;YQr0SQuu zFt)hvD8?n-Z^T8b4h~2w;k0va-(m?^$zHl>YYz*B$>e#LNf-zwhRGWA@<=b=PPH#5 z^?q{kxY6Aq2>v#VE zEk}(uQbj5)_A#-g5ac6{zE&M*dBLd|kQZ&sTw2uNWFT@F5D#;vf|UrlvAFGO%g6Xc z(!`8=i_AJxqOdz=yM&VZ`{{Yl1Xk3k=%(v58 z`kn9BIFGO**AW*VjXq{i$*$Ppjh^Hlm*^^djF6mHsSEmWvE27%{8s5okE_MLC-$;A zQh#*=+TY_-S*%$=`-SMf?L9qfi%x{f%n$l23qy2Vi=Pqc*VTBTMeuT?V7p7To}wZ~CWs|`W8rsOe{Bihoa(&C{IN41$iUc+@y!$p+QE+_<*I+DZ2khV;$ zCS^$6vA3w*w5P=N{D^sq=Zak)U!0 zAFF-LJawXj;ibux26WLIy^RGL4+=C1phwt!wAPJI_$p+rp@~ z^&a#VL;$(DVcw7lEadJ|3B9~Du2B~xV#CE^wseve-*Tjow!VRg*1z&k(&PEB)4xjj zeSIr*^=l{4RMQm zDqh3ew$+Fl-k^=jYuv4kbJ+4Z5ONH#b@K1n3f_vhMK)pWtB##TM8IopeUwz`ezc?w z6j^0SqgP%EQY^IVpz6hCbro zh*m~+I)Nn7gCYx9dqqO-w{&1`HMiSN9@|RT-=(R&X_0YTNe^D9*-EfXtnSJoh9P!+ zdkSx;zNAy~q@&?+X7(Wm#Nwk!kVgzaEK#VkfIXDg8lLs0rmlMa%ip1?IYbKiEh zH*^00x}xQH@IgLYh)=|_4I57i3I>O*VUwu!6kGj5zMM{$40itjpjWb@0u9-dYs{3)YDWs!?W$D)HqIG=oLpkY`5n}E!I<-KH0BM~kO1nFokrSnQ(WT%E)1 zd)7*kr>U>qS%iEc4bQ%p-AV`Il-C@4ojsjJVJz^gB0zz3)amT3wm8Bi80}z;!uohr zJ&6bS)OPD?mrn|jm2OBDt~e2Q0G9hjU>B3oRgLzrAThY;E1u7Mw~q%3XrvGL-e91G z6lIe~VPmK_zfP2^e%jHAq#kVr>ut3It!g?_vS`bwC#IF?Wk#(s>1P3!-)$aGje;m< zT`Z#8t0r>KC#bgA=HP4E^ns;xvAZe@QYPu8gn|kYK>U}d5cs`Mhu>-wv;gFTyVY%N z?goQB^wUZ<`zYSMwO}gOponf**bbEk3ymf5c+qLLo2yIS{{ZEva7?>!`3BhAjgT+z_i6~~YQl&4rCop2D}lkaies~r`@d!gzyRciHw4&vcKpqM%UpaIa-_T)mzxRH zmLft<^(=bW)rE+$I`+7(7xUgD#TVJQmjLSi{(BlYb0(KX|XF6!h4H`?l5_4~Ntm zQx9RKLEu#-Id!*pLIz*tYn&2EhC+L$E_r&Z6qulA80offu5`k#wAx#*nwkM zNhihYARm2ki~N|Re)>J?4nj5E^5*QH%%?^lXsCW)7+~?Z!)`7W?pJoKN%opy)otbm zvgPCoE&|00i8NUZUba-5SJUlOa?GH|j;| zCgb;K;^mX+6&_`4-py*o{cch@3zDNxI5#uoIp?ztWGu4TyhaU|F5 zucj5X4Gen!0H9V3Yu-!Ixb_;u&X#E7hl`zZru9bU^w&*2wd?V@f_S1mQrZ#XD+B6$ zDC3Ve8zo}MnQR(FeKvqS_pIVR5|#tp-DnZvTy2lVAt&EsTqqi|TEp!XkGxsEB)Izp za(d_@)}j9ZCV8KpiD_E##dB!WvWi(FJ6#UtVbl>p-0jSa1fD+;{KA4us6ALwkJLK{!%~~vP$#`< zsH-R!8XMN`pm(AJD`d6@Wiy=1RhR#@53 zUZhp%llFbJmBKqP2t7!Zj+*#a8xT8-N~~D;^{5B(V}blXX1{e?_t6KhVdF>={W;%*D=6N_ZGro0KXqE#(I{=ZL*Mw*zVb@-qcy+>z53T3H2z_W=1AL7e|0o&f-JTc zidg)=!k>i5flNwn8Jo;M_tHfIGCXSDp<{jC7OtM(-&vcG*<{#cP<+S6u)qgU_rLjX zT>JK1H@$r>0on{vmG>BhU}_29)9kAqZhBYM-%D|79Ajh*e03}pnZp9gFWS^{%xx`! z^_LKkQEh$1Df3s;SPzu{06!G2`M7UzK(VvBn&nB>z-|8ks=mtN2XiAI+-vFU3o#9%1Fb zN>lK==)m}jD&1E~AwgE`)Z3WY*4JGC(u9_S0ySrB$t*L1Pd$g&dzxJU(N4yW)d+9}u zL+fpgn$CO zhxMSzZ(kkL%#5wvRtk^^A<2IK00FP5HPB!=2isprfS{jBE01TQ4RK&7{K*=?f0Wdt zFDPVZMs}PSY|@rJfd2rUXL8t+WMMvR?U5ayYbkH$+ok>Vo|n|wh$DNZEX}syb7ty~#8v1TQH!(UC?`g;3WYtC2@ zt8@O5T(DBwV>#Ek0Mt9kq`MM%4XahOT!~^r}B!f-f&^u5)Z>|3TZxJ&#z`2nYZvnVorB)x9q&yRMvDU!Rz{+j2fp}Qx2ox0R z@wieAZVPR|Uesf^oehQi(*P{#@dHD8*idD&BWP;jZGMyru=d`%jXU11+FpwtN&Bg< zU;)$vtraJ6Z-GZrO?uMY0B!--ts<{LG_2$XW(W}>0F&bwKjUBATi(E2eYItAa O(;N(0GQxa*l9nS|4FwS| z9Xxggm@*Nd>sa}c@o`bxBn@_r~ zxtxWUsk;z9!YNq6k(U!_zN~lqjeMqG846`D#0}M#!s4WaW5QMS+qnM#%CYkKDOVdX z23fIVQp8y4YV%=aWJ$kk#sC9v^IX)}0}XL8jGNJ!&#AO@y7awe#TF($dS&Z8c?Q-U zt=CHVKaf8?d)r|7eb@MH;kL2xd7^%H7rI6c=17RJRyB*c3a|~K?d3JkC-dQzKWP>8 zv{7|fqZa=F_=Qi)M7wzx{{U}*NWR${P2XicUQ0CjbFq16b9XM?Nuom$xLJVE8d3M_ zOPR;z%a4;9*M{tY$kq-Dvi3=E0>GLw0`!XkYoyKne>wyEQPq~qlj z&ggQ)h?1$f^6TdE_|(|=ze>xB_XToVK_e!;nv#AO9!wm3wF>zcLc2Q>4eOZ5pS0Ki z0Cmc%b{i0iS2?_z$PVk37B=<+Ssb1|UPH%}5sEtPVI{13cBLmTAWC~%N`p3UsOQJ| zl>{I_-eI8w-9}-{3LwBdNU(@(+f9BgmhWe6XSfBQsmqy%rBif)wHy}6fT`QJf z7=p0(u&<_Jh}n#5b9-0_`qnQ8jXJ&$9~fzZkZ(Qpxu7er>>l2<;B~&9>(REi6~o1C z7rlKpt=XepKg=nVvC_o-)yV}m3WHx71oR<)g)h3YeyQanA0fr=lNvW3SD17nvUz;d z`PnbIQ?e)FQSPmJ{@Ve2_M&ZT4!+vsWhbsG5Z*QxreZH4I*)x_dUvmsp6VQcCI_V% z23M{@vA0`LiCiY_FMVm%q-NeNYxnV`c_iF`cRJIe%jyXM0WafAjF*VT-ufCK$-UQ5 z2SG(vYsgp3N&XckHX@zG{{V3Jq-h8;me<`$bpYDshk64`c++h?y43ii&dw}t?WRJ0 z6GV2ab~=5vDy!J=nWkxi^*1;E2B2Jd2g=3eSl`XLZ+%Io8B=9(8B&Xh@Z>SaBOk)x zclOt#mCN%X9du>&Btn18po(bziOc14@v)nJblE$`)wYU(t*5%5G-ORA;ukVYDIwZP z4Vg(gn-g3XJ1yBU6v^T*TzoFuky*C*4=rhHS85j*1azltS^oefND990wG-qbNJ{B- z*1bp))A8k=C~ff)JSzpW8M3CxEt??OxF{y}Bl=F2^%h^J1T2xpak9esgEHBI-%YFR ztgH?$39{#Sz_{~oeZa97&|BGGq-*Mao9XSi-M_89^a>w~Q{o7$cn&T@Pmd8<^IU#K z!2WLSS>bxg>8U5Z+*ciCc;=H3q%x={Ynbd0vV*f-*pne`ShuvAy;`qAoli>|W!+=Y zR}v%DRYm>vz$w%Qh=Km0Q*IjH{40dPsGRUtY4zI zS1ZQxN*5|c&`T&C4~?r&_&QwnrVIFnwZ+8Z!C?#cQ)!{1)~oa@(2p8@Qc81 zC|sR1z33KZDx;VeFdKpHJ(NwYzgoZFM0+di6gMm^zkz&*{FetN6G0-^PP?QLl|fmQ@&JNbcSu%xD9ztuzAO(&I|R#&TB-Y!gVo5K(TG zAmF(`(j-SFRLB;xsJXR;NH2CmL2lX_zwHtQ53!V$J-}^i3pQ>88c8OCH)3N3=VuRG}>R;M1|ZNQ})pemkp&B16uysCWqvU`>WtluZ0bYpdR2*J3UkP(*96i z8r!p9V6P~!9ji*+KuIQtqKc76l{K|OI+8lot7xyPaF;;XZ1(NCM44Awj%XONM#UxCJA=}OP9%+wErCZ~x9z1u zywtxh%A$T;qT5}3T9szfH~4Z#_RV=({=rctjiha+{syPVjY7tf62$ensU$)gNR+T& z5G#w2kdE!Sya2Uv0|ZOR;S8}ZY;dmqs)qNn*C&&WJjWCes*_xm@m+dWMz?2?NAIsg zzvkgH} z$GBNG65oc`tH#@n_JF$D-ZdsCth{>5B9_X?7DYwWp4E-UaPRtuGV&>W*Bg*VzY{3< zcF^Oom>dozL_q=anMRC2_H?UTQU3q|G#C!Gb`)0DUkU_idbpy!1!`_9r&{=KO?fmv z>MBjh>P;=}W2G4Z7VSW&6i0vAr4%OD6eDEy81!lbZ&6W;SSsmPJ?Ng@>pKCMuj)8& z6hp*;SvYbn=EarbaC>bgh%2Qmu@W|&DC8;Gml!2kbSJs2ag7p5jhp`f+>!ya_Zs{( zBm-nFE|1sjwMFW=SxB@+{L61~rH?h>F);I6~P+4&0i#kE+>}%_BQKPJoO&avnSbN0* z^Hau?s|C9@!W>xFCy3!yPD2=`kb%5ksHJ~Jnlq6Y+`58KOIxjcw#NCNkhaTfyKP*J zLHp|-NhI{~8c9YxA$4K6-HEO^skQk&Jqfm?8J42ob&?zGVb<0422U3@*<`*pXmhU8 z-51+Rlf>{dC}`G2BJ1KmW|e;r-|Q6XMa{0DYtvfd^QFR9j$S+!kx25LgjmyMWNhNF zLm2~PUvWa$J5*G8vbIKiu`;)+%vD!sW$~%oi^Uj`soUyF9_DXo?8c;qXk?oZt>Srq zF2ld&j^4^|sj|Ya)I7qNILuHs`f&dMHM)cNspXlyNu?#^bD<@*B>lC2+iE`Qia<1e zzWPyZYTX3@O?1+%D%~pgy#;Fl*wTP{_n>Wa+|scZw`yRb$FQI;YU`zYXx!0{uMY4iBph^YD_C_eDPxL_6@=Cv?I+S~L9XTz*RB{-yNtUPz25bc(Dr4x?L&Y3a zNK}6|npYdMjm80v24Jk(2D8>ysW%J1Gm0_Frw(nptr&k{4Fd@)T`!_t&P)62Mx~x<|S& zr@{Pa$DNTU&7+vhxVqom4Jy2o=3um`Dnf+%VU>rC>-Dcqx9q>6f5W$>ICJG;(y^g3 zERTfG1eO}8{0eRCt}q{na{O+e{{S-5L`{K9`*bxl*z%T6Ly(SH6Q#Q8TwL6D!Hb5t z(p`2P>uZ})7SweWe#F|{=t0KgNtG$`?Tt%VpW$3LAmJf2oyD#P7>*?-z^M;_zUic%PO4%M`M zgw>B)>3SBRXzm@W$~3F=VOv?iJ*qSO%cZ-~F55-4y?I{hdse2k=~~evtuZ=KWB~3q zUuiY^ObiSBSxw>nG(EqtvkR%x-8J#9OFi5DyD~y-J~?zgUtsTCx2L7#QU|wgv{pf5 zTYBkRM`KI7&Kpm%mTX3BX9B>l!hPy=r<2viY1nNqEJILyZC;w?am^YX5*mV>b*@a zHY;WQM#-Vb%OzrCZlg=z?4ohFcpyd|Q?~XV%_~fJff|Mpi-AiF(um@He1mPR?yt|3 z&h{k83H}GYNt-M?{-gSxJs6{NV;(yXh=Jy{k6QHZH0!O&HDUh%4f_t1kz`xt^KBE~ zpnt$?(`{Irr`uX}?^^Znq3xh(`>Sob(R1#g>C_vFSaR;tE`av{R@e(SKyT8t76P>C zS2cPp7+7|#PneB8^jw|!a*IJ1^0)IEl4)4WBa+V{)Uh?MUkcakuN!m~hlvI3ur4J4h#q-iv^XU$jl>X8f)gH8QhCxv}$AhH;n+M z>vd)psi5Gv+%;l)HtaEwmWJStXNP` zijjr#j@0WI7u&Yyr7D)Uwvv3~yNBVy%jBjfO{y-dS-gu(X2eHmuoBh_3ZE!Z*jDR; zd(ky|_L{X?y_LB>HPugn8j8lsQ@<(|5akcWxpdw3)}q(e)Ys7Vt1;A2*5FsVu0>zD zpa{KhPy=)ouqM6D3yXfuD&*hUMYm~d+qc_CFCgzf_8~n>RGE1u@KQm|bC+D!}TO2b{cQlL|>aYU)H?|L$$b&6PhPe9uc0T)?Df7&dHH?c&1>4!ysy*TdHIQ^L~cB4BWv8r>{H&j>A7Wt zj?HUEiHcM)7PAIrwMO^~BgGCjs-1TptyrT2AVz?O%E=@E6zkN};_z5YnV$gLFDD=b zU%3^Llg)DS7y^${1U3zBfFRPL`DnLgskxd>N&7lisVf&2yq&(vYXIR*)7ePz4XlG~ zp89Rfe4|4~-RV?<(qCjKy;++zGr~a|bAjst>Q)9I1RQ~|Vl}Cn$*+qOS!}eeH zTz$3I+g4$vHYQtb#n1Aw>>m16g_`2%g{^J-bHD9 z(&?w{+iTvAUQUP&qe0c!_O(7vK{vmaJLs$0!iVgzrN7%xJBvxS(Nh8bX8UL!v;ZAC z+JpC6-T(xfnq(&0+*2NdrCCVUm=ST`!o0d2Yhze#@bRU;PL!_&oF9t*73w1!zPm5w z*V$Ghr(09T59PSAy*voJ&@StH1$>XJeI57NbGv63Iy2qlw{E zB{6qp+YspvZdbUzo%Z;A_;i!&TPc9}3*U=KX8SaYVE2BgTURy7X4v zMzl+HzJj--Z7bK>JS%nkZrj?kJxQAr4mDI{U2f~pDE|N&IJ`b9VnHU7`#dPz z-jBAg?V)Php9Vcg-$ZQM>fH#Vyc8$DYA50& z;Ub3R)`X0j8n_yGQ3ZbL2hsP?3N3!rHx0(6S!-{2BY0YK(^fpG(?NIo`h*zSGPe#-_6q09{oCs8wH80?)=@~{5GxGZe)l(mt8IW zRTeD4Wx$tnh8h4AYA;#7nrw-tb%lM3)~3lt065#Ydnw(8-OZQYO)5Cpsy49|qi#nR z9_lyk(uoGdw%Q55a2=@GOEFv8xV|K&_>7`#wHjJ9)SLV3iNPQ*83YC3Aly8c0EV}P zsUlJEu^?agP`R=2pxb=I;-XVfMzjFxdQ%4KE7|)#+O=*Ke*Hl9()~ZoxEw}fTOL;F z42*6E`AMKvgP|8kT|vI1TC>Nw`t;VnEFF6hO0Cnq05t&Coh!-?wW#StZMB8!Y~0yK zJ{~yVB#+Ra0YoDhxO{VLNCtqB_>Yb2TYaDK`U-r_&{Ky~=|*;!{2=@2-=XP18a@?q zsP92+c6ifmHw0<#p<&ZY(;Mz)@$F8)jXKub`zprUt$wt!ri?T`>JlcG)Y9uf5uxcu z-EVtSD>az(9jGW5V0w7g{k`q>)@DRQ)nhWj9y0lEd$sC3X>PE%EevI@RQvt4C*_1& z4a&U(TUuOL?t7M`ENm0Ex}Eph=8*0sBTIYg=^(h*Q%Ng<>682@>9qLP;B5T?@S&N< zZ8wGCu>Wthp zk6>-!Xq%0_#cl&d=}N#@pNsYr`)q63USQjM9|Kt5NPR;fie&O}yn#A4pW1SN6%{uu zdRtz;-wHSQygSm!7WJ-zt!lAbnk{dKr@p4mb6Esw)OZI6v| zH-XXKwD%AAFITAYd1%^P7=wC^?m;@4FsuX~Z(!qhxPFsDA+ z^vLVpimyS|x@&vT4XCM8yK6;;hV&0pL|;o#RMkM#SD4U$!j*AAuT!Pz0c&?8Rkt4+ z}EMTCwR@2I_lJWIGsiU`QkGub0k5&cr%wl|XCz7;8u5 zGrnGN{{Z+io2fnkn)GuaXxQ{`!hOV5m(537Uu{@-{kQ#wgASmJHOL_LBHijfFY0~V zhY&oG<$vCbvXXxiiiqB0>&uW{JxC(npXH!UZiCKEjHc+t$1&*!Am1OIg_KN-eDe zdmGW**9XS5uK`zaV0+RLW4Zm$2{d-roh5FI)t$xZLJ*sP8eYLe~d))P}Z*P4j>+be-q3LehanjcP^$*SZ zey{%kO4qk-)vfLAp>Nrz?AEol`%&v}Z5#Mf^!8u;YCrsU)cGro{{T0;(6_Ss^yyh% zhsl1su=+OLZPu)>*Z2OZxca`Q{{Xl8YuR4k{{W$xv;P3P^uH?p?$o>Q>fgF%L>?eALrUdk8i+J(B(?(167p}O09U!^CfZ&TUQ KyL!7m+W*kCA0qRI z@(&@9;;;*cMyDYJbkkm-A%(rzn1-&RxFsng2^0 zL%q|EdV0^3?lK;fgxSW^XE#s^d1S)%iMdTqN5riRiPf_9&pb?=@6goC2s0adGRa>V zcvFA$;nCTRE5sI$6i?9+zj54KY%<@>FCQ8AT(2MT*wglG)aSL{JJ|Ie-WKuP){Dn@ z7s1RYHucW|DshMGJu!llpArE?>L))TERo-^Jir|p^XguyCq}3M{+8BMb`;%)R)x^L5lZi}!A1d_D30cqM z9{3mw*?zaQTxrf(4j#zNxECypU?T`N7DeuSNv59&fA^Os7VO{I7!r1xD$Z&UsrxFg z7d*DlIzGxfPC9AILf=na?180+RrKaL!-#{HY47i>Czl@o4*Jpf^E^>m(6{s4teCBz zT&3r(;IH|n1`rZp`*OXkNg$n(U3kK%q2AYtWj zWpY?C>V!Izp+1zh0iVSG`P2D-uOR*_W6zmm8(S;Z!sujDJCXJ2UU86A@{fnpPnVX)?wH9)xqhYwepBP zQOtDb+AZ*K;zQg#`rdSR*``}wbt}xPL2l=Xg>&lFaEI_KitgEXY{WLo!7GoPlG`}7 zo$YE|$098^`=xT=7%vzXt5M`-g2W^Q9JkQRfV5zVeOxjf%V7HYeVPRmZyX^T8KP>n z0^QFRKV|VJ;T1b)v;hXboH;+@>T)4ba9b>)XKnKM5Pfx|oCt_y3O#=1(|q>08JaEO z_|<$zbxzZBq^71#>!QnClIH=kUyC<0D_-48$R*Fh-WheOGP7}6G-@C#DT%Xjr4q}o zvbT6b-cqgmZhdQ&3hH?(D(XKFiu%FICDtgI$KSuMKT#zp9Hr~0syE$29$ap=k?qK zt>1c#=l?^tigvBT9I3nXip%Y7N!@lbL!aJ?3f#B3u=9IdW?;{j`8$U|K^4DV+3yMV zYg>sfit4rktg&-pq;A4&>WGMiVL$mZCEoC!T)4SUwgY!stgiwQdy>cD7-Dn zT&$^q=>dt-7iIqbq}V>BdEdujHmYnowlVLT$_J;TScI+q=&#*dzqTiM#d8nDTC7{# z>oQTASKDFJ?-83VQP1{ATjZ$uMZ&;PovHAu{!GT`C6*VI>Y>SABDwmKAHOJTGG6a| zd!E`>OhEMDrOUV5)TsKccLM#38k4G(Lp_ESrCST)wpEGJJ#jpu_0Ov?Gut}-^BTwM zR|6N5j+8x?5J=T`LXReTv`KgM#Y$%eu)>w;flgix_}1!{EI*}%!sIS?#iHUZw+7pX z9wv4#@*Q5cS@Eu)ank6~)Zg`BZftB*xkW}3_*-7IIHYLL2gz9GIIyLtL66j{oYX*} zuS~(#tsBE_w>Qq(ty72ue3bWd3l#U(?H?f54dZ-sv2{ggk@5xPq zj3TSI&CD0`Mmv~=-xKU{y}46v6gjgOxbBd4U*(oEC6n9o<=oGblu;f=k>h(JLD4?> zD|^xQM+3P?l_MvWYbaOw-Dq7Rz1+9YnM14pI8leeVg;&Xg#c~A7b$Vpi6+AMSBhr) zOxn9e)$4TG97@W5RE5=>9nR||Q{9C!H|C~28RWq_soqM+PIBx!wHQ^BI5Dq*aUEXB zruqCq>8WlSrmMWZ=)Sox|KnoPV{@d~9AixJO6Yv#M=3-cv$*}Kbpt=UN2=uk{9?uR z#NX%84lQbpk>!%-i@UR3RnVdzM)HrY`>>Np5BFrDJ_k9`M;a9_i5l#?L!Bp4rQShG zPdQmjuE0#!j?K>xX(SzNkoG@DCgcgl&W45Zw-)4EZeDcfLFkPQ}R2cJ$BN#lv4@prSXEqVaSm%>BnI?Z{OALR)EwoaIJ?G|{74j!Sqe0FWBF8_ z7St-??w%hA-ZjAY-LD4Md9=xQiYJyQ27HrSZ)jf6t{|U|sz&<1N>fuT*6wI}>)-DL zJDl__8kusC{`MjzN?QwV5zdx?D?AsqtP0bb$y=8 z=(l@`d*;3J`E$P3s(eN za;eE8J}5OwJ-$eB@k$rXiYw!jdxPteFPTDlSl>%pCsbO8q6*;~a;w9~(H-k)*_XGC zDb&mloDL!e&Uz`Rin?2@r$>5!80peEP5Fb>=i)jHf&y|7KJ0JH64 z`BfcJEHZi2;>n7>Jk@65ekGc?Ofo1{I`So!HW6(?tS{CZ(U_BSiF}B2>zjm6jxDCd zGKL^T;Yw2^v9Bce{5G3?t_S<~D=q5$w{DJ~ri2qx>fE4&6G(kEtRWXGtj~pe(v~e%FU#ebkz_=Wz}<-~2IrYr)X}VjhQ*qWrth1O7Cj zqYdlHIwgr7h5MMPZZPul$O*VOT1wXCe6*KYOZjq1o{#4@TZPzrhZwes(?ZH>rtYqo z$VVyJW3FOS<%^uTYkVr_m|2VKKHI0G{%l}|lQU93j2w3bDw;>L-`?#C%MD#_M;_fs z97Ai5u%jn|zoiM})DZNuSBVb;KJS?D=_Ik}jWUAL9KcJOkXkTUPJpCF08 z{f!n^zPFPKYsV65PptC1=L&phj)*nqI6kEHw_}B>LkUjB(%{xM&>kh`@QlogHh$1b*zO=xqz4DZ@&Dt`9Oug z_9d1Yto6#g6TuTvAwl-hJtMQecf$QG>4QN#<~LWn-+Pr+yp_D;Yn%Yk6T1T~kD*kM7TSTU1ESS(I~md(JJnfXMlPH*3KYH8=~) zR1fIwpwz&qW0zAcHaLVpxKl`w8g0bNwCRof+{*9LH3Lk342WC(naKX5Oh+6bOY1h} zVUTh4$%hn4Xrw~iq(bS#5aPTEAX+8GQ|^DzmixQV2(^&*yjOGZu#w>N4Al=djORah z78YK5l@CO5tirmbB58F|grsHu-DMfoyu0f8w6ek}N_2h!Upv+fsVUnXZ zE71BbVf&i#^46S*_)pes$zQWAA@O-{ZaPfKvlY@WG38ZIOh>)yPPJi{v`C|I4p_BD zdR3Jq9yVx~51BH4AzKem@;KIQj-ebmR;Egb6}ESEozn3{sfF%$k;*eXM}P5{xv!Ay zk*20#8yTBgcB-SG&ct2YeLGpbXMNB1IDuZ}nsq{a~CBc9faGy9j?wv3JrPm)e(yLS`k&P}J%P=qZ~y3I^bh9Mn?u%@u)4c<@0zSj5{BE4bqG)c--0Mh>*%@oUV5^o2#@qMJkNiuKmWr^Z4ila`tYsn^=2J7T()FRlNc%CZbvZCKj z;F6{hyXLP>F60VFp@jJ>7#;?XudqsyX$Pz)jNRa-SX=>ql38k;dZ6ua%^~!<&*{&9 zj#_y}E-d*SzS?Lw9)1|s2{ZRo+d=~$nX$HLR6ToRRrIAbR+feFaWA9&VP`x*2Yh>6 z(A3Br)!7da8NWx5%HEC@uS>3`UZ74AHoPC9ZkwgmclB7%)^x{Wd(YMlvz* zv|p_g{HG-t|MUm`?wsHx%^vK?O=G`WwbJt>>=dBx#BgCb_FqC0@G(||MN3C~e^b)q zvE6|z1%<~$x$D>#=IPw(Gze*=5PzZtD6l8MQQ)*ulpL=86hAB2Gymh@#lOak|3k1# zdQbBv#Xnf&Fzz?-9EUgb6qOs5U_A+N(BzFPW5EL&mpk$a5%_Zui;$cw1$k^B8!U9Z zFbEBA4dboH8cc9p9BEd$lgX=)7^U4E$jQATsX6~x#q2L4^M3JdxLC|Mh+=6XGoK}V z7W-|@@C5%LgO<;ThvC8pR;;H3P&$j-eQ2M7Q_kH1WIBa$v_fr{| z3&_fXI<3%ZS!7K93xK7DzQuELFGL9tbMpJYB%1$2Q+Yz-{}J$5OcwNzc(pmptKTxi8wKt$^6#|K{#6gxd&km%6b|-6_cV8GeQ7i zi3i@4bB0Jh`5qDpCR@UNg~RH-j(u$>^H!>K{a4u!XjSc0+^sa{PMptzzBIF*nL7!C zEcu@)v{Nat?a>~{C4NceGMdbZ#Kfe3}*EvMT18a!q92Jw_(2fQSSr%d&tn=-Gt zLwK=al~H6u@A>r?`ez|2DtPqwqva5eRq0xJ-?1YeZ^Cto_ZV;3zOplsqrUHi;jpvw zuylH3=lo&-YHg{>G-6;epQvH*gqsFOMk1b!<@l*JohGBIptx%3p!hJ6G%KRXQ`BOGncc-p?S1W~I`+5s905i<*emMY}V=_*{h&s}QOzytNY9e)O3 ze><7c%D(HVNRs4p0_XaAyI0}n^T3uUWv&4~md7b}lBovIKsN8utsz?XY!m-2zDOtS>nPn>=OO|rwWmO!cpS1*t2u6 zT9lP86PDQ?&Yb1Gy^t|hYAzsp=c4(<{$g^nSyzmaZcwZDPCA49K$arXZ6cIjxPQ1n zr`Bc5(P zS7txe(6RmpI{N1Hh2YgU4sD_Ioejvz2X@2xTFc)raPbwso1H=TCjam!&5Abgsb*{I zj^$<6d)ailHRM-Skcv}+;;gjegv3fnyg1ay)pDdNrU-?cvIjH|1;;;dFy2E}#5!zx zZlylfn~MGt@6<4+4)_ z2I)`>@%C2*zR%8hj;~dv(P&)ME%VS_cM2BXWGFHoO@+6T>@@VRIKV6Ixoma46ybt} zG}mc2s>Z!7=Bs9AM&GRD1BM^@Az@nW++KW8G^`v9mSP zb#vCl>sLoCx{b4E&pb!k7j)0`t zq2%P%CR&b0^KuX984m{Q-Ky8YbOM|tQt1w-)EH=2s-)Y@6C427pJW}1+o!nK_06td z6&PJ7(Wq>+ckMh5I~2EbrdvVY3_(o!3)$T$@Ts?>mGM56SR{=+ZjBTU?fHeg+P#|D z7)iQwoRAuthw^NC*-G*ya<=1+4)@mVdo=Pm;y7Ky>h*wvFbDmEmnkK^(2eTFOx~Uc z>B~GfTlJj#q(Cm>ZmY|3mBmB82#3CLX4x~l=7)bamwz?AeS6id<8D@&vu+u20LFf# z3W^zl$|LJh`gdn%>HF8wJ5F*(v2+b23vPm9h&F3>K-Q0j<`)UqTP-QXWOLMf?0B=3LQ3m zXpwBD!9jxAh^p-#Vc3J&8gS>vWcO^3;9`PLTg$}#L5IzAVKZ5zGR6}!Vq2eXh6HWX zHZBT~+vbE`6MXI_4(H))fTnxvDk>`O9anw2CZM$XUV&Z%?YXaEuOpZnS~TXciR35Y z=3FSrei0+QnqOLy9PJt$w4b3in^xx3WxjqDwViw{g09qQts?dMbt)q?9+-zcVMnoz z$H%`7@~$U44jcpYD769cvb8%=SOBwNf0XHPe>TnZo5XyNzYyn9l>Rm9#>dU5!dM68 zn}vTWe@+LZd(x+hJP^^1WpxEbBOJ40JE^3*!W$!iuRww=tVfq;XG9MKd}am4?{Oa0 zx1k?rg#{(Ag80&nW;U2^*U^uuO-g!$}z=*1z5|JLW{Sv*x&y*TE!21g-C&$onqS$0OP5hnP*pao?)owHh6hLQU*F5}WH?&HH^q`YNIg!If&^wa{ zL&9v8=G}3z4bGigSPvptlof{necWSspyYxchHuEOCchFYm$lVovhlE#!v+Secko#f z;9uwuFE*&ZQ(t+frck#Kg6|d|RXhpe(uyc2$h|+&E1UkcBksQ27dB1H4ynM`tOFw^;z<01@lATIv zghd`wXz{`EAX#eU<(rXd<8VRzc`ZV=^sd55&;DCsB2iceIk|Sf)fwTwjST?5DUsYM zW6Z}6h)>SYG-Cg)cpcL^qMo_7;V8hoU;J?HtmSywUF?_)v#-8WKXY<#KiTU@%q zjXv60d(V)0>g=VRBeLaO_GT>My{xC+o!o3jobUV^IDK(Gc0{UTZrixYNYMj?;~O&z zQB41KggJXg?cEnO6DJx zMo-yjK1+NuYNG~?SUVUqhb>`De$tQa+KX+y(mrnHtq-#=5D^&v<+Uu=St=gIM{X1- zbKRZY+a6u^d1Gy*-N*%DG zeeR*;*IR_lm6NU-YTS&}Mjz$5Y@aH6AAe$(>|5O2g?$l5HdK=L{-kHAYn5Ppvehkf zebwoB1Azf!PFmYNo_SrAvE36twJ2CTHRU<8{NBXmxY4U~i=Df~=tpZ!r0fA4Mx1vA zHe?ufv*$5Y_T4oZ(H8H~dbIiSCmElZ28ZJsA#^Fng-Xe6X$!2RU=L!w5ptF{^klwE zTxDc1_c1oB8jRWpb3 z^+#e2$=xuv!apyBzBV$8q9V%t-B)ziR#eZPOxHfA~Av7@p)H0eIlF3V5G9k{rwvHO>CG;}rhzCNepe`=nEUEci^tI0V;9Qs^D zB98I=^}5wQEtGO-KW4_V&u7)s2%p*-&Vv9_uD2G^PN%{(ik=DR zx~a`KXpMAt5wf<71gSqAF<|}&hGL=z$_CCWYh~zXgyJH)9yaLQmKB51@EGqwaI)H{ zS=sj8(6AfNcwUmDm0<3RJ@!>0Gkc91#3K<*k3?E9)g=lR%#t@UX#(5x()k<_``fwS zS*x93iP3*&(aOd2U1Vw;u;EN)CDtf}SlF@S@oO;{46~C3&%GW5ktrLY{aN^kE+ISj zv&OZ6&EumIx{}@}6VR4j2e^ukcpWU{%XYk9=b|wZVb79r>{m6L8~cS@T$#<#qtPeuS&~g%QFg?+n|(M`xuJ-St6lfuiw)5m`{VQ^k69Lz1seMmSPl)sL=g7x zK6~sTi>B0Ba-%1F&>gvGSLBh_`~mGi=CF;q#k%{ z7|Ilz_5gW!21N@@b&k&ssvEXTy^Cmk^-^BV^fPR?VdEGsya5mc*#RroD>=Pqk6gxfi9hN(wtvsh zP$)jFpi1v#qd$UNyP%^`6U0dxaoyI%{(06_qETtF~@xb}%8&%=3Sz5yHV|rcJ2l-oiYf z=T_8Rh@6@VKf;&_ym%Oa)vrmVV7&SCN}S(Wp}XtozhFP4fY*$Wt?$eda38+ltc)Jh z_q#2=vVAixLH(izI-*N^5C6{Hj(r!A`u9i;c6lIAT(-dR#ris08IhB<3Vs@yi9Ae+ zo*Q+}`p8+}=4O*zK6y?eJ~-!_?1#X+BL>vbdAO=L?3H;sY@QN#q@3W6*nj@N#2o2e zz<2BSMit0GfBRQ{A%WRGTQ6D8=w7~HP{%Rp0Z*(i8Vd)muom7GhH&RjxS;ASF%+;D zBezr?{{qPcWj+|*<>4ZPR<)@0aMwPX3MDt??^Oo2O_E1H9sCW z{rNq;v(?F^frpcpVMiGBzy+>4B^Miz*>2~=(5?su@bFYTuxtO!gh>j&jMfyX_>FyCD%fVaI>Md>YOldJQp*efl4)i$HK{Vy!dc+}-Zl*?OfX*e+h7{2J`KLZ0kW(G7YlZ% z)vKp#urhn`JuU`wj>&%U8w=QJwY;$?<@solOlHgHSj=*rL+5}37NAbwa(c%??w-42Vk5op#~vz~-8WL34(;-2796<6yX^IjH0ke0>bavO3nFD|*LL3@Au zz7k#PWhaxa@V-43);35;IV&!JlgJCfRya#U$GY_}=i1qOZnvF-q`+GUZTwUN3j~Zm zy#n%b&NrMf`*{M}tptkHSOBGFD3qh(O7rYsudLkO+>?61^oX^bv5P!4s&@&=fekdv z@P#V9H?4lKrYh@@`WEL`91H?Qg-v!OF0QhO-s3C4GXG40K00}vpBeIH$NqS| zQNF10$YaQx(1-hL-Dg=_AXJeWZ?F;7`WL3l5It;C){t+!!Q0c6fM1)SJ6T*DeO|(V zY%6G3!v7L|hh19&ZL~rL)8)i3%AP4^*hsjS6&ILG_{Rvl*&X~kD;}el4AvK6N-=gY zot(Ao)*1W`Gm_hPE~3%-AF)Ru4d0vtK{Lh1=KyZ`_$+Ban5$@nry)YABnW{^38+$<>&E)dK*@ z{b1jFDM`eyPu2kCY?*qUp%JlI)2=erjut6sisyPj|nTh0c`(t#_$ zAJ3gybF0Yp3S>SOfY(Fo7E2RKM4 zW!`mTB32)Y*e-Fxwke3zzg3kyb`nhY74J$ z-dXBfBfpRiaehyn-#p<36q@W;K%?&#F5x9X*ZjzWlS-Ok>EZ(5f^W}%N(85n+)+r- ztQK?Mnj?}h=N1Rvyz?3S@ZWC3#Km=pL*DfP-O2`oFDu7&h! zb-}>Di4WglaU0z6-&B+!kMCxEqMEn<=wNkh7fTJcQv&X-lRj~xJ#Gke`SMa<`q-`} z&3fkPzS*|0tXndnFVN7n(c&G1Cz8oGO+`VFsW!o5C1<&8KtXQuOZ?)ZI`7n}Q_Bnc z*C9(@co91T#4}eHMlD_2B#f*J_wev2k$UDUjqpZIu8})Qce#Z#D{jQS>#|Es*6H>l zq~ntl&}U=f0f+77ZN|35;O{a+*(K_8Ee8qPwrZN0GQjfi_Bg~3P36TGFb=ZD= z(f*mwcD#(w`(Vd;vxS=ZKgxcK@lt;VFw}`H_{LTL`*t6t$b*bwlpbV>9k*aV3Eaf` z(j1k*2U{cOtorqa&%7sr=-W?T{7S`5VG4=s+Ng4M5?3(ce^?v8z`AfLI>lY`uPX-2 zgp&!7pnH}iQoZ2!ESSznNLw`Tw$3@Z}eDUQ)s;X~LA!m}D?|X$0OrO@KUVsT^Rzg?_w( zgdJY|QE3i?FV}{NMuO$qpC(0}kylhBLIC+@c%uL2f>QTgE~G&co50PeHiLRLTX>CV z?a>Y|EF+c|_AH1>9~1-`a^vl4H`esg)X%6a7*x&gJq+2$m^Ma7O}?@v1PqG3Cz&!k z?GmPLSET7P;7j?`YDy=Hv!2h}bUtwK?$%dX&OHH+XTaxX!Dk_Ul$pbH-g+FpKIKmi zF?poauGTUu+@B7P^oIMy`Z6hP}e&R@x6Is&SHS=90yj^ z)f<>S+!SQm^l_Rmbd|CaL=?a)vQCMf9T>cuC2(!~u$4N`&Q1HqT+;o#fmVcfa0hD; zm0k?}E#Uh_;mmzmQ=9nFCe|0vI!AHm3-H0Yd0d;*W&$Y%cF_s3ygjntesYWRlBcD0 z{8()W*F!D~#(?ZB!k*Z)GiqgpDUpge;A^*pnla$jjT0#V*=t#>6aJ5t?B8m)4mk$7AuN0@plT4 z_(_#X3};CVx+*Aw{8k$)3v=H|O&pKTDw?7{Rn`y}H}1P$4Ok&GFhIW?0%_$6gkYez z&Oy7}YEpWd$71GKG2{oClTMew{zR$6?1g1EHjP2=jfL)_Jz5=>Vq%Iuo zA~W;{tc+u*sqpp!Og_k?68%ocu1H%m_+8UXnyJlpn0;;SieUAqx~}oDv2wBcHMuoK z(fv{4POsHZ_Nw(}>9axax@^cQdIYa_jZ$`stGGc4w5N0P3bVT&^N923zEQh4*NsdJ zw}un6)Rphik>2-${joz@i!^S&+xdlYLyh@tdYi~14{Ne|ypkJR1AN=zE zN=yse#@sy7jRGyRieC15Lnfp}o@M=ya$JvRMS1kmSb05kcLE{Lq&3b$8cI@%Z(g4X zU*Bv;Rb7#fy;nVDJ9y#6l@`VYHwMq#sl}aZOF6Z#1ZVTZFJyXNCDJyA2CZK;E-v2N zHC`L{WXP@I5(E1ttf|$(rk3$pwxskK>tSJ;-S4bCnHA-dYxeA)3Cn(DMR68_G{kEw z`X#{_etR)mA}!6VnvTT#1Z^>$#@7=yi0zwo=~BNRNiL-$ybRC@W^eB9HhUfHmXq$D z%Ny+3EvHlmLB#$h1ce=%~9q85A}IW+8L!ckX8TPjX|7t<%42B zbTxh|+wX3PVz_we&Ad=@N{KV%@M8>JQ9BipVwusa5-X@N8MpSvp!McuMw-t-R=1M1 zJuEA3JaFlV=)e?z)3Sm6W_HWZwbI%IQXm^v&!C#)U0>Itv;hzEN_V^Is4%qJ_Q=Mj zw9jWKDgE$)$!6u=wV^Y5Rw#k%VcX$cqpe3^FP+g3s%wzPLzNu8L0WPNX&m>bKjs}j zc8R&QkiY2JHgDWxu*o2<5=)wH^;ArzZTFQ=dX%tey_2Wzmb=43FmEZGhndRY*1}v< zPk_5lb=pkB!nSLfIBO4_d0{RbWP_JNhw^d}6x$!=*R1PCvV%`Udh7AFSDS^3c((uN zkcQqcGkd=Z@k@5SnlHIqGVHhcDOAt(ooCjbnKKsVaRrKzEqOHTbc5z?+cMwSJd)3I z3S}Q2>{}k5MILJ>yHak(!FI&BsBX{h&UnoQC5o@=*|1;NS+{QX8G=`ezH``5vtM@@ z7FH~uP6bHtNFaRkDW)UFpFy5f47k5#h<-)R_D3I#tdSG17%#ta6!SdV4}bV6CVl}{ zlyob|NxRFyKx=R`oj(qb2+dIClAI>XOi}Y$-A^6eZWStu~*keidEGLUbX>454_nw#OP@vd) zpYf$+Wed0pyx;r=4|Mp9V%$0XR?kPp^v~0IT83pDF8LoModXe+3JwVxe{p<-85;qW zi%hMdnlYk)@|= zj@reTx!0l6PS&?ta=Ap`Ui>}u{$-pkv#KXM5TGGw*xjTbqMlRNcQy(BdVMJ*6?T(x zdf!20ReHhXF?$!Loj&i({X492*Or!_hEe8*G&3*Hl23;M*TGnmgVZW7@M77Y2QDjf z?*V0x()*w4o~h-^Q}+?W>?c~Yp<2>j_y~VzcE~Re3R7k>gCQypL}fZXl2vqH6cn6S zf@`O*6vqYLdt1OFfdV%5vvDZvnYoH9n z;t!v#qRLh$9%nKj0FO={~+sT^x{x|tF5SzgnTb32IJ*%wo zZw9uVHzAOpCqbku2q1~NK?S|;jLII?z^YlyN>YCI9C%AQk$8rUN-`+(nPvBkkv@s( z4VBCtJ@yb;HZrjQbqPA?pGOB9CdZ0uqjRg)47qSd7IB$A#~I*@4Iu?{^=@~I)i-hv zsp*h?BU~Ld^f9}wJ0$GiaxEHOwb^2$&!(=kKbUoJMm=+|OK!?z4w0MtJmgv8^9?U` zVW{0?#Vx(b=Bja2=CK!Sg%5el>CCk(`paN7;Y%i#xckm9a*{3l`OZ;t&cr{l+6BA`Pq)=NiCz7uX-ZT5-bLF#Cy!a>CqN zhV2D)LN2HmTu;lFsabHm#zkiM$Xs7OKKY1B{@7@nO|8;A!Ia>NMHul%V9%8^IF!6X z45f?jwSSLul}N2{(dr7{UL^4rvyXOX$U$V;-cN0Boy`qHnu~i1b6{SqvUY>;!~Jh$ z27J2tgSqx_ii=Wyf(<2BQbXRJdkuimte>uM0PTD=SJAtqCbDILqv%Uht}XSapx;k7 z!}r~ZGsGQ2#TwIUI^?4(b8*ysuDV$DgJI*C49$9`3YSTe zPT*CIL!SAB8aD~BR?ekeZ5n?>7wToBYH>I4*TwYc{aA&|9KMb`0#fgCI!rVS^xd9y zS++RvpoZhkx7#Z0tP4w?3hQDRa>J&vevmoTxF-ztai&f`p>%fh2tk+3MMMHTD+N5f zME5L??=FYt*Bd;0Hiy$eV;SOPSKVhkS4&pOCVo=07S$R_4Ilk7VW{N>J_nPvEk!an zSLkZdmW&M>C)tyL#a5re_kw95JujQzEN>nEi7nehP&9>UT7;Z#$q(sEqj(tes@wd? z8}c;7g-4TizmIB zcnjP1(oCg@?5$wRs-|~2iFprVl9!2iG^AL#lH&unm7uBL)$+uqg#GKaX{nuN2Vgd@ zpa^j`xB}5BlO(o=kqC|jOpL*Hg$z#{bZ;Rt5|vez=ubcEJ=b)0qo=Sff&O8Hhy|@d z`h9&LubUOLCbJmRz5^GRygne8C;Q%|m26YKDF0&mPDzS5E+0(x+T_(Cn&?MuQg+&3 zRdjGVPIIbUC`JUJDC9;kqh zC#f04GT2UlcL#_UX-y7pO7$|0SD&LLggA5IOT46&`~+txei#2zw<%Y-av^5wU_szO zulZu}PYwHN&I`sOFWuyHCYvkLwB_{GGJvD&r(oRZ#4NUc{F1+LbR)}kA~%vPckh1a z-3kf=x{x5JD@KtXi!2-H;ae!Pu=LKvv={145I8}@&By8Pw>=ac_7+$Pv)whcW1>GfEM z>Y4QP~eA9UiS*H?$*wy#Ha3&8GIADXQPo z9xpbLarFi@>XuD!PUld_Cp_S(2bHwvQX&;ZcLxV^YD|6X?gLQrQliMzDX=f`pRga6 zYc9FiYT*!&8Y3tJ<-ZA^yEzhbbxx4c;bj~$inDG2>TKz}tC9)Gi3Q+)+I9@`H5LYj z{J_5i$V4~7vGNzB>^m^f-s25#W5HzrUP21+sJSgSzyoVWgw@!Enok>M`>c=n6tI22 zJjpL!8Ml9(rqC`HN#NV_7qByIdQ%4W3#=VXR6D5%e0b--Iop1~>kB7oc4OC|e~U+Z&s~Ux1lxf|A%mqE^BO6*mknlA!n2mNCX>MDRLaI^NcV5gbASE&dz)>OV68CwHhr)iKLxRZwX6I7e6$L4DGO2A^N4A2pTk!xs*`-;kA?Zp=Oq+xt$HJM zhENTz8rqwl=XzzqhOBenH1UOmwpdXB+C<(fQ1GDkRGBRM`Id=(tg3tOx&&%Ei>x}n zblS1sFPlM;rwj>Uz;KHda2c=GuWkr;W`eSDFe4PbT4(6g+aqal9bCc=RTpQaKxXC& zFFrFPu}EX3$)urf(Sl@>5XMf9(+N1aH7_?xJ{S01ULO{v~KbfF+v921N!T{&P_!;pN`o%d!w5dT2?`^h7Op5Tqsa}jrv znxrqkHE#P!*W7q9U^VSNzXPSc!%Y4syXT`kGs${%+n;p;2u$qjo{VOeNIz)mAxyJ- zrd0Su@U`kqPYzTp5B4nvtQx>ANbsk`|y4L0N0QYG6ud zY_S6nZ}Rt*bG%(VP79`LYUAsFP<7%!Q-6CE`_Dv5=)Y{PPJf2meP{df_phch;C|Uu zH#Umz?(Q-{GL!bOS=nnR;oEiS;bs}AjZ%;*U>%!<5`vgB`1#WT7qt8Z|@GILk113i_#?=#l2^O zsJRL`Q79BM7$p7zZdZmybyHIlmIZ4HN4Pxs->dk??(p0=FR_o~O=i;F*VsW}k0lmd z<_;2b4`ih+X26AmYP&6pT3T4JaOinRYdbW#pMxxM;+e-( z?zFg3<}?yCvT6jA9(<8)EVZR6jIKX3GIkd0`$Elyg_z9r3oO*u{m z=T#9xB6N17q*wC%h|_Hj_R^P!@2~cEuyGcNaMD%uNMw+PYiOP`O(qAJRblsmLF$+} zA*59X-zt&x`Ml=D_T3`$Rp{ZkGg`xrN}B*8%l4+CFRXg{VCa64%x~8LFaWXc{){Yh zwYd(P#lp#R5DShI!F_v{G}AoUn_ow4cTh#ez`)?mr$K);+(8%n97go*a!T zf(G^$7SMy8`vMt7Q!Y9}MjHu4!3yY$y+VBk)yG`Oe6cXGjvhU(%n&*T1v&<=ssj)9 zB25SV?g;Tdx4WKdk!jwMKGcw2?NXdC>liO!uK*K2_S$U$`95`=)Tj~ z4TbCK=g9Q;5A>`{$q%Zl*xWG(B$jp3Z1c%QrS?;EXUw&WLq1iI%w~9>u}IE7Pn{Z( zok_0tE0D3dU$(C?=9z(Eop*%Z0X?MG@vkWZ+v92u`{lTpXxeLpkvZ-S2f1qUoilBd zKSA=c#<|bGD7(QGd2qI$DOy)k#niMjTrtCCHPi;YC5{(65uImDho>r2w_c?PNo?KAKMxI3X%6SV`Qg*e|VkU*Niq%vVw~Rna_M}tqJL?pikza zCP*Si>1yv^0;NypzC9m~kunU7$O|^;&Cs|h8R*<6Q1dI+IDL?3bwJ$G(Dlpk|6%Vd zWzA$j+-gECg_x$1;2a$m&+X~*dd5G2A;bB8Z>FwaP4YYr))c%zo zG}PVY`kC8x_&%zfQT+oE%CTfDofJA3Ij#Yiz) z2HJpYEL*zB&8K{u>a1MamK^s`@y0AsspZ=MUlG~8NO$^Pzlw1BFXPBvtVBW|I;(a_@U^SYtyUIYl=RV!*FRRc_ z{W)@$n<9&!t${vN@3k+E_a@Itz$?3FXJ_}SIJsr^K;%@4zxykf7W1IJ& zIV;o(|J;0roFrla9n_D@(Vh9onnJ^JNEUi8zpx>3U~B2k5c8|-GoZ%~LRk7dSh?`K zDK>g8$VVrMXYNbKqT|+%JJ)#!#M5dXy<@wrQX#%BJ}pqzSW`fVXi0f8S$}o)r*WjG zH)|0tIiB=pHWFcp0ofxdzwVELAnt29iCY%8hEhaHmDe{hdmA*!B;4G&hh5_t?O(bDZa4T zUlq`6tQpsQhivEHf-Ah%NRx=#4P^_RXmoI#Fa4Fh$9I1^IoH%F2$A2auez4E?dE9;Xvg%5u{f%r7lgEbU zQ#Bp~YlcEh9`!1HH(%AGXkc7!e8WmuW{TV< z(0;J<;3n?hI%Yw7{qtz`Q$)dDsayVxBQvYR{qT7~-(x_-?!0m1+EHg>| zvyFlsA(NHsuoSP;%p+47t=~}Vyiu4#IndRE7)#W#L3`1bSiTIH2`&C!@zUd^+l$ve zDgLZ}rY6eyu2cdrWdl-}cJ)@ttwiCh1|^;+40#Po{L`3AFfRCMq*Tp@zj(GfLie;L zUVtr`;sUXvC@UG>-QxeO< zNVw}~UY%hnw|19=87-WqV=?!4UFgg+mWryZhxRYakGwTBaCeW(=Cj(!K9>hL`fn`f z+c}D^SkT=1M7BEba(!b$AVO#Pb+uz&CpYU}aP7!VkgTPVe42e#tR(W?ayK>Bdc3KW zrbt9hU@EQY0;~*~4`o4_Q}JQ$De>*s;jCqZQFXs+31d{JvTm_EEnO2FY~M#YhRp@?^! z;IGHfM{t&eUO`E8Q);}<&9Q=*$-haCo0ZJ>CaK&l4e@g-Zp^@YP}@XuejrdH=4LJ& zW~Mup*gjV5c|1;9NvcXXOiZ92;xU!DKa@{?FW$8Pv+I61bRXnhn79qI!Ab_3n&1OC zKm_z(QkmUc9@c2l6b-G#Nk_WWfKR3{IYn|uoT^vzQvZD0Ffwg6^ zfSeU}hYBwhFjc-rJvG{aD4<*)XbW}q$z}Xg9;)eRm+pn|f^K35th4VEm+Wh)h8LZ? z+7_m1v-QP^o)i0ph->K1#$P?J7&pg#5!-#A&?AALQ`p6w-JJ8QU%E^ug6F18<2n&0 z63=8oHW90&8MQVljMp?rUZJaf0da(@;5XQZ4`I zoRx_A1^jN1K-S3PTQMob?-!d0p@?_ABJ|WubBf?pUWtT#wr)?yQZ);NhnwxJu6_#=ZJOda1Pmj{!QeF(qQd-O-HXl_aSWwrBgMpG17(~yV zmcYtnq+L~5&e04IKdw^heUnyZCQxWNL`;u;_#|gR!GCLgbrk{E)e)C7VwO5QrY$r~ zys*bhHmoF$=zRHQZ};t}zWTNs)6kQIJ=ULi51FRw6EroiT($1r$Axtf_qU^Ti$G|Z63`Qt2}%Vp}NB&cYINwDoJnK^RpinSwy!Q zHgP4o@0Lo~oekNbN`yL+#L@*DG{S1th@Ub6N+GU0P(5T?3Cg}5Uxwpry7X7CcF~(Y zGa|HzF1bkjSpDesr)DQm#B@Q<=kr71U`BgKt!`{65m9j{*e8EIX_ft{-%gu47U2yI zpbhn>ESWx8alp4g8Fu4D9|8UKe_4k8ud24)8{k_h^Ly)4@kwqAvDR*YqYu%R&w1x$ zoz#wJ|4hdukR1bRT+%PnPFRHRzm%^RuL}0PxZ<4yvS3hbo(>io3{3dM$=xUjjv7oh zBcToiWv5>!OySNy6@EeeSgeG8xR}vx0s?~EPEf8X{(X3Nke^FZ9nFL$J>JI-`|X4$ zi$;TUy|3Tft4xj)cR0*gF0j*&FiJxm9kC-G*mXyFxF^8zBHo;iE+Gso6Z)%ptC-tf ziy~MiU3jPN^oKKQjAdKt3}W$x*e?=tR9;KXzi`M+uTi8VoYa-7y|JJ&?~BA7P3@Z* zM^@IE!gb2Go-WhbNtB~%lQB+8HHT5y4=0yrK?2n+DXX4G;OJhfc3@3q8#UNYE(O+k zn8T}x7*jqBXv*io+?Q_cC=S%@pz??p<k!gf)TaHXTg7-)RjtEKB{AGVJy}B{OQ0ZAzgLK_d4_QkP7zxdJ-=H&w zsG~>T`?ZnGK4I&m&}uVUL~EQpt0o5x=1*tEDH6exotSXQDK3igzr*CQ(%(Rg2S<6BUq`2OCH(Dp5ROy8eO%Bm?JPTs+qeiu)BE^nO$ zcTeBPqKYzT*^ac-4ih#}{d36hH|i^+q1QUPBiCf9StK%D3;HXz`vOkX^)N0Ks1%f> z@^5++^H7V~#pe1$vMn~&>m)5dH42VHzhw6KzkJ$}LQc#(pp8 zC+;~e50pO76bN|#sE#sHd#m>!iHK=AzzI!XT;Adl=q2`Gf0W5|X znl|3{Y<>a`jlV}K%_IV6-pg#Azf(%K^QO8uAxL@&Eq%^P#5Q7r@6UMnEghHzq?Z>9T=y`^ho8sXTItM=Rk1rN1vnc&t|5_58pP6c!cMIJ+6b$W z6YqZfS_pH<(#w}Xlcmo1ozyszFd|>)kYR#yy=v(Jl?ERB=2vO0d8TMX9WVNOkxU`q zLd-=OQ^-QI#~s>hBs#@)J)?{5pvmaH85arusOb`FL{0x%L^K{vla9v!D!Yfo^thBJ zldf2pGk=IFVQIdBa6htH<{3R+1S?`IaCmzP^oFD(-?E5kiUYW@x;hy2xG zjUZ+9mo4>H5I0r!Y>-JVIMBHr%4?&drj=Ul~9QxbOMy`5P-K*Wxx+jf`6^|mzoF*GhxbDB_uhbi@wUzTJ1 z!5y0-S097BWQafwT2F)*j1O2$$g*W5B#S{)ZunRnOjxj1??6H6jMK_xiT#J|BRv~$ zbIJi$u+za_1dvPN^{Tmnb1Vt(1)q<;VHp)Q{510h-;`$onQ+X zTqqk4%W>1;Iip-(RCuJb$CF$A8r8e$Xy+zRspT)n8633hSh;;<{CszBkA@mohl9Wm z)+XPGZ*dk)=Dqa<53KM;G=b;YQ?;+nzm-6#>}^v`qek$9uYxA?a?I0^N3a5iU&ZZa zHbhOX%h9lcOha5{+mpg;@H;f=Z|84;*H|O5g5`K8OX4K*I?2xOX0JQ+zZHL<5=KoC z2c&Pf7Liak!bJ+kp9c;T5r5Jq%`A>N4~Xin;za!i@(J&1+zhUoS=orcJATe+fm@#H^Jo_xu3L~VsK{DFe20N=cda;QbJJI+zWez zR1y;tvwZP_3xe?@_4WD^`K&aYWw#*=6xPO=+{~2AsBm_$*vrrMqam(Bi_>Z}%t_== zQ<9pvd*gh_{PNvvG|Ygc1d;I{yuD}|I%?o?2Zy5dngi$xQKH`QGZDPa3|t-;e;_cx ziPL5xye|){IrSVYyI1_$i?~oh;1EXIq`)&ZVZJ1TVurzZ2LB_6v$@2a<5+6m`;R2( z14Zmq<@HFV0u&Db8Kx14ULJFbEj6ep5_3UY`cjCDp(KB6tG$0|0 zdFGBe)L(upF&PIxNVl;>PN(i16N!*341HtaR7gHXhh`wFETmUnQxkT#HnIMNVw@zF zDHMz!9Y*!{GukmT-#VVkztZrSd6T%D89(ZL`!oyYJtNMH|H_~0s3M}0_%|e2Mi(EetZZv_`c5&KCw-+C7nbAgutyTt6uX%DT5Xe=K+_SRn1{PY(EUh8`g-iO& zO+I+T^VAe)PJpAqiO+90QkdL}J7pHZ%9-dVJc)YV@X*_g$LDM*|o&+?N_+v9PI+?V?{Ja<0ePvurtR=O`@ zz=|{Iv<2Bs0;fhH5e&o~IbA8}Jbo3A%~j2X$hR{fJmVBI9YOoIJHR+H?e1%hon)VW zg+&ZcSvZ8Qaao3(nVoO;!TN*eR0;~0Ni{#{pv5sgXmbk|=xG@ZH0)#)4U1xfkyTAJXB(xM{H$SEgl(7G72fR1xAIR6-1aGWe{5TlI9k`Tt` z8pat*F7S!t8Lp|8H_lWOKHuKms|7SO7T`Zp3?SoT+qx`rFo16Zh!bI|(Fc0>r3t=X zC69I^ciDLy1qRN>V+ZF{VxtoHUTLzaX%CK3NgDvgUow`ZVTsEEFg#NyzTl#Ue z?c;O?b^SA$wBSq{0b2`@t{d0e|2d_G+qQb;MTE`n@6mBOoqhy?&LeEB?THz#!uRR= z2VXMrz-yBb>AlRfelggs6i0`dqe6B97e>zgL@Q&XZbsnYPW{Gg$jRN(7(`^opWv9} zdtVrA3f$-iaS}bZn(-AFqbT+K>Q_F(MncIE>xEgs)Iy%V!o21yaZFZ~5jDcpp=Hc{KSa5#bFe(%i$#DifF zLq2j>zvS5?ojOR<19GK(tuA)GIFoJnM&%8U}9ej<|mS;h0a0tGMheMER2H;?z2g#7#q>X&%vzx!^+V zVN(LmQVKlfyl`Qve@R5WLqyhhO!<0s0qoCrzMv~^djTI7@h40b{{4cS;?k_4LR5uv zMGXg@?lxGk71CLajJY1ID4jd^*-4-Hcd)f5Vx~L9HKESypBNZ*M>ncrzkb6MFa3nF z1hpYjL_=bEIP-D`Y}*?b(PMZpR;oX78?u&|?~De(@1Ldk1nxUonVjDLcNq4621XkP zV3P3*N3!}9BR2RGiGl8WjOn&^$&EV(b%K8vN-2L39H0@+?K}hlO;zb1!2v+qAYMMN zI=#D4`Su@Uv*7}?mw#%U?40&&rgsZnGR%|QZt5C<8SrnwxDA)xbX}SCSPA~0F?O5+ z>8IEKpBDuE{J&@(**N@Y@^J#ZhQAJ<`G0f^?)_sP{e7inW1we82AfZZN`n~Tlj;Q; zP0Z^%O5&0HwgA8NoWC5M^@d1EVXhpYjDsMtR8K~}K(F9)ojOEI(ssoQoUk}`f%&WR zt7Thc1KJWoK9fo63M3L=G$-(qiny#RLLGLm1^|yo1|G67FX4maq|DF@VFv*GkFmCk zJyfNi9@(}AY~Q5N8Y`J!j#~rBJ5TJ%11bAli$t%3wZe;YcW37#(1VKItKZ!hd%v!a z9UWf8v(p9EkB6%=UWR&g27m;>R|FiKR-e?iB&My7OogNKsqjebufqXqA6%12SeM^u z_rUp>+gpW`W1(F?JKcsoWDX#n1>%i6A4opRb6|{K6@#};(&Hz(erd7I0sK}_H+%P< zDGcm70q#!91r!Q307@`2UVudV#Ql)DZ8udTcH!b)=s|;-PpI?3ug6x;JK{zIe?O$O zsNP<8I*JA<$^oz+*xSjmvZdbyk!YxQy2@aNUnl@F3LXm6MJDfszssi|no?(vSA2Y{V&x)GBy8Pttu51&>@ zw{#o7exGTTz5PR=kKeFnfm~;=P_6LNrFJr(s47+d%x!-WQ(7AqwmgcnoF-FHXPIDn z#BfSOyjz9-$+rjWhK;KgPR)}-=Kx1Wcp$X4cA-3m-(kL^vn2UC%2`F5IrozkVgt3R zu{wwI{?yQt)xTl*)10p!FrI z)>;I;?TJq!Hcuyt3rtKSnNnjfFcG*y6Sv*qT(}S{sQ!#aTH+;OZ+$_PJRQ#MC2!~Mw^3N2&RnT)>N+)X=W1pAMCy{| zC(AE3!u5#^4|k%%mZ1V(ujGZwynR5n_55ac3+#b-tFrwR%3(dYnYHz)o!MaK_>ayt zLyX=a(Xs3v;tx7XTQCdQFsdxCMp|Du>=3PfVUw19#m=C>Q)>5a9;zBvME`*sM3cjkL^XXlGcYZ$+$n zB0Hm#r^<8`OZP)e8zuMua0E>3Q-wM7+NGx;WQxOJ)bQYrhUS&!=Dp@*iRC3Haiz>M z_ubq~psb+^)x0h0{<%tAtt98sJF^?X1Y9!b8(!C$$kV4?IlbDf1{Ct4ab)qk7Y`IT z)fkd4_Oo=+N5V}?F>6|xy+M)+_Vt8{yd~gGuzucYf~cqp4i;hPsUY|HW#%<$9oPAC z^rA>%ey`#3TRR|W@|r6wx8$uX)Ys(EH{Ya|@R*CUme{?%A$q$6({Ppso_rmvIJZwK zZs~gb=KQL^hPm#B&G6R zogbDf5fP3~ns(A%(6eHOp-Yki(FAmEU3^r79AQ7?F3mZ2eiG z$<=}`S8j`v1UEE(x2vnH!V0!icP^ea<3EkC;gCT%33YcLAXub4Dpqh~^^;ET>{~BJGlaS5`3;!FWP4ALrnnN{`CGRfRb#d49Erj(XE?E%zS)lvOTeV!=+*Mm7@*N z;XBrCYk1nYIenllx|g5UL%7J_)b$rD6c{~pw6;GUV~R4qVIL;W;lAi%ZI)3@L}e>r zE9&0UlZBd|1^p2s0(zA-GjuVedewl(kQpfM+4@^Vh-l>j9}xm> z4J_#SzT=VuM8*4C-o`3l{40msjAhDd60HjOjEujO^KHq7g~mypSQTThq$OYV2{gGJ zCJMW&BORM9MW#wX7fZAK@t2!IyxQ?q-S$g8Ax)jbksCJ946(iKt6b8292{8uU!D2FsODl;oa; zx|)o`?n>l^#%L41xK!@fwlrQ0!3LZf%3Z@syRm@{hg*(O>dB+~)r&njYqhe{@x=-x zsIN@vSY3Ld!gQQ#moKN*oWejF~wkWe=E< z8k0Uc#7RRW=ZPH_*$_SV6XF;(u|?4BKBC-;{Lyp(7_c4T7elZkx2 z*OA}zBqg)ocrqY+@g9k>_$mnaf_pZyxfPX8%geW>v|TqU=q#n57tA`v`TC-5mIITx zTx_4e@u?i%n|Zr}>X+W6lQ<9{QE%9w8{4n28eJ#ty%plQ7oO{dCERLpdz28`;&i_? zfJa&v;mJ!ZTDc_@ul{UOT`=pC^4s;(aG^|Qk0aRewEfMNr%uXj;s;XtD|sdd%$%f2 zXAS6YO$OI^9+s$7Mn=6hnh;0l&iQ}NGvm6j%FpOOa09HMgX9G1zXk5v>n z2SmLg0*&u-0M>E2?M5mG3_k9m7_1*KeBD)ad{EA392pb9lBp5U;Q6{+mMmG~KD$az zal;c6Yn}?aBUYLMvKx|Oe-O=S^twyd-*iLrAoYmvAaR=-+e$R7t#$9;|6orb-&`FT z*yko_I+)oF(Ka$!lC2-dZW9(tJ!EFf`w7T2?wX5DlKMs&4p*5TUx{GXK%Cokv1xt= zxT{86X(U%>=Ie^$G|zVBtp%%lT@DXpEB`gAt~*mS9C@&E)H(sLV~W&9@2iMnnAAx( z3dKWHI)?K!M|KJ-t*?kFi|b^xewWP;-`@@#1WY{JDon1*KDSF@KtNyjiFu7T{}N!* zf)c=N@0}@iUe(uRO_^KMse)lY+Mw6a?Q#U`44vbM#Mak-$**Uyy4$dCTI(VCBpDQ) zCGDG(?4P7AWx#1t-R*-~S2*oAYa5;rDMSJARA&x^s3JGGxqf+YvDU$Md8CL;?$8s7 z`EDb9!!oTiAK|&F+zd*_wAd}qS{Bh^6_z0c1F76M>8cyinsTw$mqd%ossOK%uZSw| zHf1kKh3!O=S8}@8N>(&(&8d@_P~H_+LAg#LR{7}4@2Ugz zM5E*ICo*L42s@j>4H6YwQL14ragVw2%!vT+E@;Lrlvjn5gG4VkS!Zi{mt+W^dBqf6 zA8M#eZo=VL{y^>dEk$EhHKYLDEpJlea2ft#_3b>(gqtj=BJy)HPIBO`|C*pWTITjr zO;`TfkebR%8g1ZUJnuQbBqUQj(8)=KxifcE>w@;f=OQ~<_`E;(6V0~3H6-*V}9eqX*44195$Yen14{u^x|EDw3Aq!tLMWL3y=y zFtY>V(s%xqqZd#bvaPd2zIs&BxEa7;5j(>9Mr^G1D8DGz7f=ttS74PgD=2iKwowZBVg``UxgmP?~h~ zaLeg^CRhqOp^HoA{ESo?(d2mFTW~v+xvF15uP&rk<7sZ~yu zH%QzLo>^E}Xn8Xe(5!HNXkP>Z?@{_I5w1_s*;{0x$7v`)lpLtADbGNLM%=;b(F8(u zlbDRdk>*W9T(C{eOd1LGXMYpc>-W!ZT8y{^VN4a6$RmaC?+h|7)nS5R59y<{;5}iN zTz2RJH|!`P&Z$hLQ#ZeR`W26@zcJXRWlts!bvJ}rT$MYU^!nP8-9n_?*4#JLa-{gn z9k+>F4HK?E(Tk`Jrb$pUf+Wg2)Y1RaOd=Aq#8R5ey4m20wMbn*Sl@zqyqgLfthoT4S2U0Ww9fW8{HIen{pOj)Sxh1ePn1yP* z>i`=e-Fi;>&TlJfF z&w$QOB9*a!HA`0xfuch{2h?U5`5_gX!X?~% zb91^b&WNYKD7b^6wU_*Q9BP@1DQNRqh7#@PIA{NcL%@GFYoEx+UPU6%`=o<>Vh2AS z3>(Q`^?0r1u&Li-FOUm=Vz_&Qh$6h;Pp|3gp_Nmf~sPDZCLj|ZzF`*KIr<32703y`v z&^;!vB(wG>$G6vr0sGvq-Tb+lJ-39POrr?duPcYOTJ`SgU%cA(W3alsNBdnz>-HRd z4`0Kf)KJz7tCh!w%QMPdq>;W|y%!r7S2rf!Lur1+RjT7g_^6F4{l)5+Yjk|3BeFF_ z5z9GUHb}Ly-d8Y%vlzwLi})Jev2sRlZ2z!}JQiZj2S5|51fmfC!ck3yH=wffpZLk$ zRJrNd8I@HkqVXk+jf=gMY~H@zbh7x0OXx+=XufQmqW;gC3!C>OehZxhQp+EMNyI%r z?(FqUSpq&^_bZ0$xiR@wKaSN1NaGH{5|E*!@2JXGMwa%>O*Fs+?h&bJarvL9Ka%mS zW#oO2NTF6sj&YG}#s|WJ2fU1qKal1)@iG;nhfX@zKi#K2?X-oQG-H~G zRr9sTOxzP!i*yqoSE|Zziz)L<_k}B&wl{!oNIqLvFZQ_CbP(?XX#aplK8)uy>Xdzk zj2c0YsAOKC@v!@x@GZ8SpTlxnO#q1Q;K1cGy)ew>cygk@Br!j_fZfWF6M)nbM`RvR}aF7$;?&2Z;yRrC3Z1B@J zbp~v3rSvZ_*2&TTXJ;KZ!W*RiSc&J+_4{)=9Avxsn0uWUvR~Ms>BUC9fk=T~Y=e*O)+E1>~DF7gn3f1*TdjRqP zS`l>u1%XVB9EPhG0Jai-&Rb{v>B;AWrH#&b-`W>psDJj|s(|K(4X>d4vt-~N_}nrS znmMo~3h0HL^njaM@~=tNf4lgBpZ03~eAMa*f(IKKpnm|_?z4M3qMsGQMoSPDKI z_;`=?Qd8XNNM3AY$O|KomGJBb>Z-P~_CDcW(QO4tnOorrq|YCDdPRBurCFRo>YGwf zRf_z<+Vs;RG?fpcf*x=8W|hS8b!OnC>~>66C`9jp?mYlk#_vtsuiT0zk<3wWen0es zRniK4!((TMwc&hOQpFkX*beNRw{u@bUjdkMtcm^?KhRt~V{vDV+vzPpMDhEcx6`@k z;d;kD{0pGSri5is_CAk88rnS`RT9Sm-tGC10K3r{JL>IIY_D6#X2T-1CO;#sm7RDAWN~z4J&5OAK+||< z(y`aH6OBe^aBLg_g2YF_k_5C2$Wk|lvr0+*sBX16+$xFl`|ta&N;X|OoRs|ix)GOugQ_LZ?;|IXnF>TBzLGLvQ&?VC54tH z@CxqIp}?R31`we~`~i-MT5YRq_^jf+*GK=Qi<0WwEDo27o%>*X$!8JR5&^2W>DjUb z1f?sGKdpxzgrtXMsIgffk!h(}cgH$~7XaSKo3ySvc5!Q8*{+(n3ajzQ@BlVM?+d86 zKGH(sf`hsTB2xDoG97u!=YC3NWj0;l-TkN=VXLAdNeWhU`An*4$vx|KG+e}*e%PSv`YcS71Qo&PII;%OKVKHX>O&+-bH+Wz~U03!6; zV*8Vh%fDy7SOPZ3-wcmaEU3)iM;W~9|Gg@+kMS%+1Ls=9FS=zz&7{OTczs$YKoz(? zqXEX0t%32p(s|AOjRKDSL>GIo>NcJ4y9|utcXt~1>Ju&=|9atm82M$nZ)TKCvOhjV zgnxJEYP@31VEnY`{(;ofaaEuo8;qR+yUr*ngE9*hNIg3EnZ2m)gP)&YNnk9jyAHLIH^wv`1X!sb zp>P@hqr=H28OPhfic;g-V^fP(H+n8gRr@5+t?RO=-M{(}>$uTN*5oQR{5}y|BS9y; zqr=}9utyOSV{50x0Euw{(fN-U7tn=*1R=T-b!e4{0D#{OGimx>i7I6v&dbWWLwvkN0bFlL?*`Rjao{me;My}Wy zot*fI!4gMOa*GUBS~O~P99aIEf!p&h@VR}`*|I0KJ=-8}yuy-;b5;0Q2k^W!*?se{ zR>;nr2@543jn#@hCon~a=P?+Jnuh(TewQq9ADcZX&tD-x*?{SdI;yAE^*hkf*H)!N zPh2vud{q?WEnQp>j!3{tFY>$e%QCWF=;h_LTYVR&hLQkjbVGq9(dkJ=9ngWPX=JX$ z#%KCcA1niwmnvKOGBo+;c>(awai4uAgRgv}yOM9fqrKCOzVG6i;K?0Jc<|QnF|1&3 zi~X%?C73pAeKu>$PSk`iN9p=5z9osSt*27Mf!#Do1Hg6GefFMhZ)b~kH=yqIzOrau zL~V3Nr-(MI4f>){Zt>Qwqm;W`Vpne9z|7*l0X{=w4E@)CoD5o4fZ0MC1huo=XNL{? zoTR1TXOeiF5_<)ZXrAfKDVV-Qa6+zwzHs3e@XK7OSo4ViEWS_;E z=Qf=S$OS4ErXvXgEZ(+#kSqZDMQzseCT}2*Q#{#V0{bLx+edx5iYF>V1MHRXS}m=l z!#=CAvZOciTng3}Q~bk~8Tz{|_)51+qn4*2wxsX7b0 zku@-duy+~9C`}APn|J^N<|kOry{tTp(Mxyx{lMj!MvtnRbyJ4w+ljyz;GFknAOMHu z;^TGoEzR4lx~WFhqI(ZA1_#|5oiU$YuIH33*Rrp}Be^XTjRIq#9N>h0FD&ClsRk(b$fPPTY9k^;T~N}yU+eDL6?bdiWE zK}#r_cY0tAB(zI07xpETuSiWQd%@9IdwrcrB=W@+;RjK*Y-@)@Sq`g1{Vn@_FE1_T z+&n3xj7A4NQU$k~Ee1(;y`!AoXKRW)K#TuvJcS3o9JC`oQ*~|vYzEQ29oFDzQTd;0 z^x}gdAUe01a8^rhm#%{Zt++aXt^;dQo7h>C9W1aKQBO9kcQ@66_l!Lr4W&I>c-&Gx zya*7_#KMN>^Y2!NM93})asTBFV9BeZzYReD$8-HljxEk31+^?kLce0z+R4>L+8so@ zAm=J41^jykll8SIQKkNoKqGnXhh)B$YcT^e#T&MTZuBlc3^8r`2i#(pB-y23q<-vO8WN3&hE1oshc^*!V;7Re#Hw zw#V`G1s*mo&c+ZG;)c-{RB0-d=Z}Bwnd0k}RnUZ{@{QulLlpQ&3#7 z^avh8gaEZ%_*&lKo}of=z5U~go(rQz&eIcbplTx$mCM4CRlbxE!?ka1E9EO-fSBG3bN=efy=y{~g1V*v7yS$S_g@EU8%axmAZB*Gle=9V z-}u6DBD3|oVfJsb&oMCpcq!y*-PPNLB31^09g3m0aNc;bJHQF06Ta%OHYk|5ZzcB* zcTiKWuT$S)^Xwq%CcHrk#7NVHl@;}ut!Sl?LLHf@Ydzlr7S-_Z(ArBk?lS^CyNZ_>E)#&8 z3v$h^t@0x!wjICg{SwH$#uhN=pQ2EQ`I^}sxfn=31*{*upS%meRnflYG9%59ADfmH z3oSe)=kdWdjyZ81x4D-*X5Mn-6K^F+{1RKmOweh^3Fcn;Y0|`(Tl>=%)4VvKC2bn> z61$c+9B_yGYxT;$#xOB%d$HJYC5dS=_nxlkNK7FkK{_oRd!H9&=v`**G6n&3fAH|4 zn@9N!=4?rd{B%P;YD!&OSBJvJEML|7ZCZDL^HNEu9)IS0qdLyL-9Dt49{F8sb=^I! zj6}~Ln6tUPppVhccV|E~vR5nr3HjRQECv%dXSai%M?S8Tph~`#nIZDX_02hmR|ioq zD?|kaV~luPm%V2he>9(A$@KY*DYID(gRcT*rFbnf8nDVJnFlnb{AoHWmon)k zAeV00(9nAU?{+q61SBBUQY;e{IVY{7(T!$)`~L0Ejc(ayhD_c5^iau1I9pOV%IOhFy}l(! z%lkxxA~l2-yVT~lrac`sg{In9a22j3gu%-l65MF%uo6`(By9p;sckGbk8&i03b7bx z&Ar{xeY>yCp}Qvad4rh0Xu_N{S;Yv9h?i&FTkx1;=R+? zAmy`o7`YY}bfSb-2X+j*DGV?bikb)+(=qfd=}BY}9L(})SKKz)nQC^8t{sha)1 zZIiEM5hv&evETdF7|R?MN7oCJ+FPr&6ErKH%OzF6obcanGZWzHn*!92nEQu6o^cSc zJWX-r<^H6@AndBk$Su09sFf*TnFWF3)i|@u78%Ltnq$WJqvI=cz2CQz5 zgvx2Qe(I3&<(#wMWaMRgs|UgDhAidi60$D^|PAk+S0& z_!7YPM4I%;A?TLz5$91H4=6t10ij|86@y(z3B}Fj%Cq@6nE^ zWlF^K@Ra1ZI*joOBRo~R;oJP2d*P3RHO6Ebk`He0d(N%fKc07#O)ZhbjESfW-yorD zTlBvbd|wU2hC$p&zMrG=^=yiIz~E*}qQzm|k3;DO(MwxjGuw0L<7LTGgPBYtITZl0 zO(ainw;eG4&J~MpwY7egXIs^C-+VJByg6QTzv?GIGJJgm_Ch5=>dzEc~jkr&z0J1hCLVa?=G4HS*ofhRyIcXqsp$Jp4I z8ci?&I&O*4*hoy5rgBLZG@o~RYjKpO^N6RvLb^4xxJ-S?mGZ{`GEBx6qNTZF}GccNy~?Y>ZQc@-&pMWYc|nh*#7(U7KHXwOwf1lGi(yWxU;j$R#v& zEr35;n>jpt_HGoLM!y+l7F#EKz)$+@_?xUnQcoy8hmydvongDh8zJxybi)cVc^bot z=9=R_WUvxKb3x-iw_UWa*i-2&`yZfn1=sG>AqW>+6)h~Cr;pD0vZN0?` zk2<>;p@p&okHLQCxF2Ntegd<$N851pF{v*tci)xEgaI2?l`kWGzMGiFe794(M14n( zT>auKPwduNLQ=n~{6}0&MY9oS=SGgd6)g7@Aqp%#s0TNBw`bSK-OQDAUd$4y-+c>eCf-Z+1xjvCqw47``KXH<0_p1QU2t0BweJSO1l`~y>30S?I8TKda!^kC% z=E*R-kKM0Wm?kZ_#)B`qy_o;%B)Lya^y8T-{vENYokGZ&%==b3NdDB%-sHpRH}jF^R@d6b zrOy_N*SI-qygYMF3dWoplz%uI*1^=@BesjqDp_F;=kvHa93NM1>9`PtRtywMPxv|V z)_AtG5Iq~`v`r-*4Xcq>*ig;?L`Fp{8%Hksa}YP!ncMzm08upe85OwkHt;(#EgDwJ zb6Rd4Wa?$g1Mj5qTaxgCG zw3;SR2*Y;|p~bX&>X4hSb$#l~qF7hf`>$WvHj}6)>p)=wiZ*@qu?4prEY*>!h_rO1 zCzn776N6S^v?f>_?1vdvCxtQ-Vb2pc<+Dab5%&A*yrjFS;dI!t^yQkP)%EPJUvBQI z&iz7440+O$XFFeRr(Q#Fu&K1C?TOyAiz?c_RY z2B^t-*#`;n{5Cys@x!0}pE`aRVY4&JmTe(uiyAI2{JUZ=ByT9tv9cGj4qRs{(5|v+ zwSdXp6d^Ph?U(gT6Z?6^I^dmNy2u(sUOX;qCdJCn9a2&a?cLhEfys_*R`fFd=WlU; z)xpNnv#PoI8+Df>ToZFRyj6yk=6O^CSDQUVD(viZa!il*#?L=-`^|~V+z+2mjm_R0 z=n6qRuL};-Lnjw1ZfH9@I|KEYrHak!uYC?$dmX`;ize50fZS`$fjB71u}A3K%H^zL z|M#uHsl~3wwrg1|G4ss$*0VJ%4`sqo+@)`4s1l+h141?!J4^s~o=FVQq&7CmhSjMd z0|{^2N=(BAJ^^Ye^>B@|*tvu|nU!>pUTZAb0&rECjr4%kPu%49U52KvweN6VV_M(r zVBBj*Z{sv!F+rv5zPY3t=Txp|YLS|OE(&oT$0v``{(Flt3+esE7uW21i*W}>A0RY6 zMvrIDX$0qzWDVH~P(|hi*w_Gj#K&6C#!AvB{GAK6&qScs8qLSP4pK{t#1)ZuwtBUc zz2KUf2f%Jdw1HgT!{WOS^{#a-?%3`K<|$JOB~xa1t>iT&c&z`-swCpRJaC5UYQa#| z@Gy+SSX^|898|kG7sDWg2O0i6J6;rnNjJKJswL@d3Iq><00#;Igg6gvJ%EYuq zZ=1(tG}V2!V%trv^!w^wM#a-oS#r3nfrL6gFRM0N_=Ff5=u8wwnd+qIj^@)k<2V3Y zp=VV09PP6VUfb%-PtI@TaM*$Mb_FwGuXznxX{i`4N7 z8)mj{AM#2RE0VAaqU9Dx$HrS3SPchXRA2zglN6^kvPZ-w3#sAM?pyn)bFW|n&BmGHppDo9>4WJ$b0W_IJfS9coIkC2uF$*f^_>wT~HkKcR$m6(0+d*A!s zd#$zC`mE0?xEQ|9dqHD*YpNv2ue3qspla@unE z`*DOa$VMi_VcxzD7##hxL^7M~Dtv}={ViKpsW^eOlE%};n4kUXu;Mb;vtYjA{%2q6 zBRNIySdDF}2yi^lXK*Hdutf+5{=C40Fn=U@bx{r2JP0i)%9q;4;Ddenk%nPFa+UpJ zMoLObEd9ggMse>hKi-HW5GsvOR(SyZLrm;+-VTfc0%uho@z}-ySjP@1E_Outx1o-pBD+0eu$LCBQBjQTtBdu%Rtho`gXIvO*Zy_F<0D?XOy5(MV2=JXF zRbZyqiBJ{|l^KUMul*`Fw2rKY*YjoV7kU*EE+a_9G@?20T(DkuO%s+svc_WdQP){G z8CNjlHaQ3J;(Gz*9GHMC>#J58D%3Fq{$r~brJr^X@Bo~i=9oz?ukt7tN*|P_zZUoB zi%qAzF|W4UZ+Tk#7;ZIS$3rd{-u|r7;0-5wp$1~Vsy*k!o99Ci>7LYzuGNDxEt~a0t2hz827Y zn|aS0;%ARrPZgJ@#J#sP&P-5suS2c`v6N*Z`Me6QW9RHRQzerO582{c(OA=QO-t{^ z?u2lEd0`}N=_??E6f34-||&LS>!%Jci5G`s&dLL6svLKxV8 z0H{!B;HU~ht#JZC3Q#p53HTad?r{AF#)gpGeog67C;=Ks9%MaLEA+3Z{Ao@SpwGAc zF77C_HcRrqryLg0xnN*^##Q_mjk1A(!A;zCzZ5?|zkfyY|4xE!Xn@u2&I>mu8BKm}+;LjIf{F7hf^c21_Bho)&~e zO|?ta8iTen=DMGfmriZyj+TSit<%r8-O^Pzx1%^nUZ&MKkh6AZC@xZ(W(88=UdPu9 z?%W5F*_XX`r-9zT8oQV6iLWp?$L}&i3{W7@jY(lx1k{(HZyUvjoISCQ@*qMym1r~= zx9BuY>TkZVxF48(3#HsVfF@s?5r>}dT;54g-08+7(5D}KNyJ8QL=FFzWu%h4SJU!>F z*E0!Al=}q;VqT@y*cp!l4fYz`~7rXoPx{DOK-6`jv6B_M&fR5Icw7=~y3(lHV; z=wb?PqQMrBtS_gpYb;y_JXhD|A5oYhoDvs-+mv@iX3NCcO7*v7{^QBG=?H*DD82Jn zB>z*>w+WE2syx=(`rZX1KNL5$ongkq@?a8}RX!izB$CLh)%0xvuu8Wn5BI>hz2zYR z&F}|<7yr2+2{9@~6`GlgSSa@u%-|9;r+7E0Eka1YuRBETGSbG?{_vP+y#x`j`WsR} zm}wg1;@rl-H#W2BWA%Zqf*bi{<@J$}05|v709mmjQ-X6FY{&*20AFP}K*!XW!88Dz z$#Gs&$`rw*ezAC@=XEf@x<9vpvB0<|tC6`q6G5NBj(F^=qw~RXYkVfuN7QD>dK&o8 z52YwQ&n<+3`xFUbO9#(L8eGAORuDWOR`7{=U0w0lY_ZUvZc|gg|75)Qe>2URu>8fP z{^F?nf5JcVmvH+cZp!&@tR$y@B|PK5&`Nc6b!QnW|DsbmjoyE|keT45C2*OJDD8}* zuBEY2d_HY#0{WTFtu*m^0JcPUK-%1Sslc@dl!T-w;?*CT4Sve9=bc9@F zLg0tMY6IYJ_aVY2gjzmn01cM_xctVHz!Nee&xl!G_le;wF$5@SjkC>ZM1xG+M`TpU(| zm=>7Y03Fy5sEL7l1E5b&!LArffE64g^66d1V)I>mOyk7j>s?RUhM@lqsN_(SZ65Mcn1E_6`PNAw)c^onKrOO$6@sO*!hZj5u*-}2 z4p$Q82lM~@kH7dJKl=95=#9p3^np^c__;anI1I;mR<69d>V+~veMgIB0)NN1yo^); zmWvK{(sg;$vqem=C;eQ20G1}ey7xthY3V_FP;}&GQ-7iJ`g8LN$(M-B1=*$N9ZD1{ z&XA?uWMD~f2LO_d_-}x$gKz~AOu-<7(n489)4%Y(Z3Om;SXzMXVJAAZJ;GH8;&qdvMP}9TWlt2Jr7exq=u+vfj?*eW0OJiKW+7O5t zH)dObDdDs981Ok{4Faa7g&b>!W6bglVT3>!lhuAx6|kAo zfrrYvgIl?Q@c|KU0@<;Ws_?@p=OXY1-yVQP-=fD0xx}r*^F0b43SP><4d?_P!63oh zuxr4?(3&RzfC)I`9*YYG=icQ|?Dr(WU=~jat6z-p9y9-f=8S)>5Q?;dK%PCkcR`1S zUE{N=+S1aJOlFeTVb*$}qmf$>fj+4ZAgmHjNsUD2Jf~{_mjNFjEVwdFKs67kczxUf z$@b16wE02+Pv&;C(Qc8k{=v4_l&G6PN;{34+B2UL1R*(5$znp?gzC?;CfXoWw@cgc ztRdV*skUOj9+W`-ME4MtG;sI0XIA8+H|{l^C-X*CB)evqJW(aalMf{I`QkL)M0{AaZT_6TIO63^C;bSW{meCQ`o8_wP7xliTuxOm4 z`X&CyW0k9%Lf0+|R)qt8wmzE&abq2;p|6~sOt!qX=$sZR8Wx>LL}CmbB7k77s?t(YsFyNP44R zB^ymp($`Fydp-fxVUPI_@t455J|uWOUt9EWz!*jfWjn-E5W%p?8xTkfheBUiMn8*; zU!B>Q_<*n(DtOa(hL5Ae0G^dDdjSD=RMA$0D)bd&%g6K4)`e!q13YSp+FAmml&njS zSjWi)kTO~(z|vbOb06!{6ELW8U`GrlrrVEu?oEJS{HY#iqxk?X^jNj$4nc9mfM{Trha*mGozS zYLYU=HhkIFiQ?&MT|PYo4zjqsi66VUs7`4M zp`SZr|6$Dw!2x=8z-2SRes(Eb zFLaUw9Bk?jL_@U9Pv!fl_>y1OJ^kbHRj<7OWZj-N1xyS~PZrFpFKy&Z#T)oVcz@93 zAsN&h_#m`%=cMYmI3&NJ z`?No!!09bowlf1F%RHHa2MpM&Za%L-bn5K)Xz;t-+3y;C@4P-}&bG*d?N`(R#=(E~ z#kY%!m<%KLxxtDzu-tHGGzvYHfsi_Zoqvsk9E>CPEUD!tug<@kJAh#q$)B16J{tC2 zO7FoPVZLJBtNVjJKwmU|{RO%^W5$CN&7ulGvMU_+g?(wgrP~}wh#9%;gAUHBCiz64 z?gk3<`{;YaV%?U}kNCla(OxG!+2Zu$2)Kub92hvj%DCOs*gtmI92>~HbTsiy47`t7 zYU$dhalW?eW&ARUnV^>Mw15|MJbS@s)^FT*(N1^r=9AGTg5XEVNH8la|AsRE|KC0U zvjBMxSKyIn1YBj&FmEphC#SY_nQ%iOI*VHV5Fx8lD28HpLH5R|bq%(ZKEsI%$Wz0d zymGq2$)^;el%Q0YzHa%r_X$3&u1*(F77khix&|12NeNdfvj5#Lt*p$^Z%;77DOZ7HasrWJnFbk9*=A~{robg@XZN+ zZ!b~atwe#O>Xwy2nXcQ_Hv5{;+QfMVyQje=9)*riG01ZP{nk-axl)GSW_UMib*pbO zBG_f5?jbQst7g-8cE(cOtG2~1{;8Qd*3&t+Ob+H_QBl7g_@vBfZ_2&EDQ$vNH1nyI zI%gRhg$|i>O{XKc`GGP{uE7QIo?$s}(%>W?{F#KwqwqC$z+N~E+u3p_4wJu0`>sUy z;JX~K3a|rvW%PI_LOFoRa&6#1t+a`X40%>#jyGm$*Tn9KML5ydG|X5=9g~lbIxW?b zgiS@QRO!wahgu3!8qGW`qZh$;l9cAiH1v^~)HJDBM0vfQjK-6f>COuq6-_1`Z)f%t z>zaM9gF(~NC%=}-CE1Xb=^h9pouG{(SFr<&uP}Kw@Wl6#bs$~y{kh_etex2R0Y)N05ym*StpPe-^NS8&B}zkI^IyM(5HU3(PI zKhe%R+F;Z&Y0X`w!wC!-?1db$;v(#!2n<`ABT>^7gqYsC;LfzZj=M8#{j{PYdl^67 z<92}mx^6c&-gbOrV7g(Qe%zY7rLMuvOv=$ux8V2~kIpa0($ah6^Em8mYaSRhEP~fD zhtXKd)-(-mts4~+r%hYK^-o%;HDus@>0w+l!dWVZ#!oR37^E>~*rq1bV`{KmuvB*` z`dyB0>|4{jPj2SGn<~48b zjOE7`xx|-QM~!5USQmPgSCtctqIQoOKB_k9@*H+4p)l-#@Gyp4?JB$`r9w_83R5mO z_+iMtrm53E!;DLyT(|m@Q%zHb$>b>Bss>xGI^Gk_WAqHO;~YPity?XSHfha*m#--r zw`M8W>x6z9bdp5r3goqOm&u`D-5pm|Yz6CWpf9(~vSKB_2AFRwsIqa;m1Rw7?xxUi z{wNJ-ZwzDe%zqH!gh-OG2Gc^ecf5?8G$@$oqncb8wJuyzZhum<9n&KX*3HwQr!}RD zWlnzN9u}Fp92jI@+)CA?0r>96xb=n?*nx+mwx#qaa~yr3(co~f{i8DaULG8|mEe#7 zQOEqH^N{84)k*!a#SQ%4@qP1`8$7(YM^kGkiF1WQ^m%(mkhTjjrj~E6b^SyQQ1$sIgn+tYq7UUaSKEl(8eClhNfYSmfG&Z2{gQ~AB)Opq!xA04AS=p9W_RalggSaB3TvTh- zqTX}bKOd%5Y>PCSKK$8;FE(RlwOYPu=IxhelCZP%aQ4^J^Z(YdLeuM&?mo{d!K|?Q z@(Lv$14vM8m5rz{F7|>%7lJXW9;QM>9?XavNjvYe{P%D!-J#YvosVl&OmEaPWTp(O`te&yuvg>Sr`COlba!>yy z%L+^E{0yhJUu2|OjCX&$J||@VnxxTmeCYJPmX^QoGwYWB9kBIY+eq9?yb1`A62HlQ zig)Y$sQH{l7e_=ffp#Qm3;M)xm;WItckN%}%5ckc1dIgFj2!4gGdG?XCjM{>PQ~UsPe0g-XUD zD%iP5?RDBsr$|oS$jJ-g%{)&vZSIzbk1$!frTgwy@L0WH@gOONE|x*9e0!wm?kWtn zs-gs>&21?Sez?HSk{c^OeT4ch^oZ0eNzW$|@9q6wl%D-ow)dn{vwuqLp7zOveMA5F z+MljAAxfT`cRb7E>Z6XEx2jjXuXjaKwa{BxS=s%#QV3;;%flYek@6jmPLe z%pY(XP!T3!sI+Mqm+=h>MWs{R?+Bo;RE{xCr?H$-SXPLxlMy-q_#Q^{d=?VTbw^jdUFG;Kmeb3B@b?%M9fKA zF{miSb4hcX*`HPM_!o2WcS)qm?+XEJaQ|h0ca3>p*I; z7mm|-#B(kicg2~B{KF;*%@656uKD}=brsYKUl5+|eBNg;Y#B+7iL6iBMVx15_TFd8 zDVG^nLZ@z4c81hp_%+qaDSVDCf8Nag_b)s1z#M)ap{;i1xtUGezC3;77BcbA@F{(<8uI6P!&BL}Pgwl)&sWz~_$#c=R@V>~ZWlmg-gOcm4I0 zP45#wR2~ka^nfO;gG^O)jh?w+=IzSx<{S1o9>4VK`|cEPS*(;KD3GHfUZGQt;$KrI z1!~(=l~ioX3P(EV%~KgrPi!?-&xvdfujq#F^@-FA7KY`&MAZ(||2}Nsvys>EctDFh zc;SA9lvnG(s@I%=$Zw4c=B3kTdnU_<|59%H=eB15mA6!v)~XtQx!5IT+lMK6F5mI9 zUIDux$6Uks_$|(`>8)c*2&G~+nbckA%`j`^XNkkux70|-f|}ztv6W0#=|ogx9tKx>Dcq{p?CK>nj09(=x$h@ zd~f)&Y2(d9IG#e+k7^*J@t+p{lFU1KuYD23m)82M@iOq%c(-Me*$*ug*G+lx@1BQ& zXu_VW$2VWxZbeM)RTvp|Hhdwdr&!a61DIO|~H7xMO%A2h5UVK5on**D;v#Ex$3 zQL0ih!*EU=ZL_;F@cE;4iKsTS%tTre>Rx2S<<5zUdY)b#dQur8;sRG}(HKvrPsaou z87-L_`8@{brALDss7;;+Jhe6`SxE%xb@me=K_%sww$ z1&o9CjI4N!X%=SuP?0U-SeRc`N@%h6(xE(0M0x-R`eZd@e~1}YJf&acRb6P=CsZ;A zESI0~vU|jO1La_s*8P~>lAL!-;$GjkBu#htq+GZtx-NPzhc5idQ?#x#JV}<%CL!lf zfYcCtc$Q)YCLX+kJ@%yzSvDqAdh z)T@rJ*G#+1I_Y-hv_;4v3>8I;w^^IcMgf78^F zh?U*UTDOi=|G<)Vze&vacezr)_K&1=sY>~l@8Om} z`AW0Y)bw2ZOyqe!Xl@v5=M9H%UUdlL0kn!IGqIWS{Hfjt&o(%sSNBCw4wB} z736Cf-&19SCcV2)ETud?g}qT~leEKC`*VOcpvrh~EZtVR+A0evm!rfH@s@3VvAi(# zi9A2_*PCT1)6Wvi8DK>Zl-2)AT{qrS64@cL>k#Y`2BCC$X_Ivy%XW0zY%NyE{9uQk3pRn2*b(M1ZvW`f?q;w>mF(s)Ai_Rse!eGjE*cbR@P<3(3{8zt1B+8H=$(q|!X{gO#}nZtH`Sb* za86OKMViDYOlOkeQz0tpHM(+L+hQvS6?CbQVC}*i1!@H z#O3YVeS3^+Rquo42;3rC5T|ED?QMC|uM^Kr1%U-O zwuckxrBsU%l0Em?!5WwiUSC(ZG;F=8AF?KO5}-MRix&3g;4ST!t2`o? zo?B+AY}{2aF0s_)d1m+B7xzc@d1>RClr@;YfUaa66;Zteq)(7SU`Px&T{@|AkR}Tk&;ZZ>=@F9T z5V|9Of{bjNj}S-vA#B?)s*)|-(Wb&OHtGb8FXm~Aoq33!i8#-KER;oUZ)PibrB6GC zJOoO-_|y*yy5_F`)CKjW6;U3J2#evZptqe=xssvd?mm5XU;?h!{hNhPdI*BpGvf#v z6`6E2O0et7bKVxSwy_A5;}qjMyBTBv%NY1yL1gR*;c}H)7sdAGr=|gG`_IK)-OvR37sQs>s-HjI-T&nuO ze|&ni3Hnl*-}YBtsCSeUkhuKUUkhC=F8=jTNY<_Y&yL+FC`5qdz~f8o{1sgb(u&>> ziTOYj{>De?)njQMY-;_kluXFnCs1HNG^0W`qJZ`@jP<-p_g&xFw^q4Ea? zfhG;&IY^xCGbjAL@imC!(|QZ51DCmxN|c%JX=A`D@o0G&oY~u0U^%Db1bMbWQW-?& zA+$9|xu++zKhI(Wg!cSraSOsf#s7cnNIwU8zL?*zQC_(gRk0&>JS3&`VHX;n8kt{t z_^xuk_rM>o2u`EI`3t!Bua=4tcKt~8_Gae9V87i--UUWa4Z`d}8Z`(U5kqyvd$1eo zI-W-BbAyVxRt9&0j)TqRyzqJynbBJkJ+Ft5*_B9Cx@ZNzze8UOO=4w9Sj=Y5n^p%#=Y_<9 z9l4604m0NZygf|BVb@z6072Y<3eKi{)D-&PyhINZMJf@ zU62e~SPX4{5RXoe-7W8UN*bMRW+f^7;lYqS1Dj`Vtg?3uulpi9cj%!Uso=frfle?S z#y4dlNQ;&iehG^iIw1;^y%{=v0}-P17^li!=bLeF^zKZaDG*wH;HDh4O!u9-^T#AC zj`I#VD$%`ZZ*Em7=|5l$lvtC(dd3l(}|lwOS`VX#PwiEiscG%It$Q`C<)c!i3cszfBdPjva4 zS3!rV_Vb5RybTU7*5}OI-3|f8OVg*jJ2gW}viQs+Eo7k->KMK(GMqTxbSN#8ZMD3G zT;$Ad-I4EeJa9!?s7x_+5j{uB3Zkc7wBxrvt?#Ls$iUe=m(#WmEh{h<`FU2#)F{>l zYd`4DU?YeY$8iys0X;N5teL^=mza$H;LVLT zd8_aX`QT)5sbuL5EAn}>|8(8`d^WQBOyB*^rYSydk6z-aop&^>$aC|9ef#DDSj94N z1#@+=tlI5;#MpOlb9L*=X4YYL&2qy!_nK+>)cAKI~K)8CP2t zT&hNnn=#i}HQLMeJub7TUV9jTKGF8~(qhb&?h;e^IX!%6dN{?irMT`qz37F|rt<4- z+NAuR6~B=7@TTWevb*HS}v-9DOnmnJ4uxiq-l}! zCPII3NK6;kAIbVwrG|0dGgC_g`H;N-(DSb9gY1lkGdJ1_@Tt|*BR)H!+(f%_;v#l6 zy{wUY0m{RvdIvwRk;)X{JNKf8GTykM${)XdOeBN8wNvEVd@rz=lvTZ;;!%P+bu4RK zeYVOKlU{3)GmsjYB*2jguw@%cpnPn*7+P=2s7-$FJ)l=HzDIsHGh0q};HDMkwb^%J zf2pZDC)*kop;V9Rkb&btF(uKZ%G`iw*;c6THw#f{1&;7$4kCqdWmM_shzu&3aV?Az zGVBxLtteuUTGgxJ5g%L#A$Q=aTC-r{dH93<^7HEZ^M0F%bAOf=_>Cyuf)~i5S#>j2 zZW)hlQrS&@s@N0i8`z4?cdU+tiiakyZ?ZCFn?J{>8J}=asHZ=FR$V`j-VWl@rKQOu z_fB8IjSShKRKAT;nz?1Z0cV?xX~#Yd5eWrhG+fC|+o?O7A`0~z68+p&1GyPLP$rAc zS82MRFzAcJs_s-ryjPV)mzO52SXXm!vThHToS5kvrx|k39eM2~Ou?0>IYsrCbb=rL z@R`~v$i0{VBoocypD$&bN;f_4)9AS=BzYbee#_q$3+=t-uQ_1vjrnL3s1RC?C>L4Z zPb-1?pmthI3pzV_=BJg~onz5MqAKgibX3eS-J4wmraj4vzFR-!cTUw?2*;C(| zs9^Qrv}=%T5tgA@wbNVU9`druGvtfQ=O+~HjGjL+QiCP-T(G{oJ8^v9^hsIM4a8$d zmb+@R&pqEBQy!{E(~|PQQh7NLeX=<+h1L>&-?^8v z(#hI!L*pU3b4xho7o!GguFl4s;dkT8r!P*$>X0DFwl37vthRLB-C0#C$a(!de$&u& zFg2a~@d3oa`&Q_tk8GKbM45qP#}z7~g+NAc)(c_hZ^zN8$t+Co+19>ZgL+ys3FD`M z6o^21Mx>v^zsXE+&>6;N2GHCa&e*yBGd4nrJp4R0kwQ3}LRoxql=YVX=9awjPEaSL z-IuBgb7u3J5Go-AKXQ!k=b{@^^uX$LLNP?}&_s+chLPmQ` zOyEadh7Rd__8Tc;-sUMGsv?=9`nC2+?Ol<1SuV9}7`3?DxgMHsJ!<6u>I5zjxg2Fj zp2N7>c+Xu>3RJ${tMRMx&RE{SKylNUa$_nc*%l=k%8FuVA+yb7rP;bvVN5l6z^=>l zYg^2I+CgLJ3Ecz?2WM`-@_v_mC_;&NNX+;RPtcpiLu{9yAQ7=*XMqeU1G~=AkQ@3{ ztt{Gx8$N!RUs8JeB6q{;fhk+AHe)4AMS8A2$K+S;$8Kv~Nhh+|eIA>4n74lqQyJ91 ze7CCX_C)C-#_TOh=4HXA&v2eUE<05joI+RhYL z+bDJ}k5Qg8ajW z1%3C`><|k-pZNy-z&URf+i5c4fEktr{vzWumuw z<$*~>f*}j5{yM$AC|rYC;kv^@6xVXeBthpp!4ybT+@_(<%QEe)VXK~Ix}*2%^wYIT zi*MAFPnrttW<5nKY&h2s1kJGt-x;X^>Sze5y1*ejy@h@m=Ihn+j zh>oOc4}F@&4||J#eat*0Q%1{@=ICr9`nJUs;kD_$KkuprRc^-87e8JKwVm0^?`US7 zAGqm^iVn^1aD=kh!m)ldWbC3asYum{m?~YKXl@H$3g%LEl_sGX1}S6oQv9X*XNYov8>b29 zAB)Vw*m!^Q&OUoSKYmD1ZvOj2HKuL_Yl{ED6Ei|Ec5{&sNmOyz1wT1ewcY4C`vxjC z3@dAp4zNx=n_2KsRWIpr4vL&~--Qw}&o@a>xN@>FyR_=9PPWEvL92mWY(_hW#d>Dp z6W5={ECc*diX4C{H50z&iy)86K*%73FYrcWe?X1d?<$UoNh;5;u*GSil9oJJGc!Ph zO}RY55kAU(!$1}#hRgrC68A=Q&|+@&NrNIHVIV>r#KEd zlm2Fmk^@ai(m$B&7~5?8G+zeA_`;f9KiuteMvbPezFlmFJEKebT26vW{X%DEGD<}# zVW=BhzE+o%-}2VDwLGxc2h+LJdU;$Wu*t95Kv=#|Duy`=SaIoy?$)_;y z(3P;2me>}S0DdZu_=n@%6H3$-^r zsOYBM>c(csy|lB@5z@N?oZz!5szKKkv#$=#TjbPvEZ<>%^|%CQbFAltYo%HQZqVb( zLej>EX#9}T3L?OIYFK~jF~1(y-@BIpQi(b4G6ir^pSPW;TX-2Nn$FA$X`)d)xku(Y zN9FpAR|VoN9bzVg@1jTVWs(7lTM6rCOd8Hs%E^n@EOPoIB3NM&>EccAbi&Jvm*CW5 z^~d~GzueKUD#&>vd`~t|4_-NJ!j9aBKTD9lu>9%BK#r7!-OHv(JV0K;{i!xk74LD_ zuxF-z5UU6CaO`oz^I0)-)MTwz$N$<*!{yq8Q*ePW^0~RTm2km|O)W=o%pQemgL+lM zZt+scfKY}#s=Ppgs%xNBr_9MB$z0$7Vz4#1kr~iX?cUL}An@MH<-bF5(g;k(msg(9 zfIM22t?ZEQt_v1&L>*>YAu47!%an-P&2D-&;VDeUWrr_YSI*W>mCp|@-Ze81xkDjz zGcNY7Kk2*T6D<)SZ(1dy^af2x!yIyfcI8Zevzw_DEJ$-Hn@<-DNK=8H&+um}uUMloyGd$EmmaICcn1kMfWIH2r*c5~@Ri1y|$b(|=#&RHat zsD$28y(-(M&Y3!GX4)3u;Gs^bCx{e_e)1w)A4s!B_CJ+{qry~nG9PI~s^~?f4=J%# z3<$Uux{%%cDd7K^v3H5GN?=1idq}x$PWJUHK4G5O({n`0lejEcu z3~Y<fo+qF$5w^_V>LKIg0UZ#N26>d#4|BOMi$?RA#-FEw zYP_HWnqUCvg*mqsPAw4y8MB*i)-MZ%Rc%;o)HUa4|7`w#q>4mbf!5OjUXe7o-`Rk&IKiX=kIwE83I%elc#->60 z?|=ogRJkXcnE_4zC_5gY0n|_msv}bZ&|X;1VO4OctG|USmMid9fH^h^VTaDOcR;T&X36yr#KYd62iX=1)rv@ac2JqED-fNI5sDf)ahQoM z-LX%Q8<$4dbb|dFIvS7s+4GXIJpkD49zK)>8QO;n zV?=5c%CfK(-y)UC(D{leXQpZ=>`g@ zZoW}mm#Y7wbtx+6m8xa1u*27t%)KDW(@=@eVc9II-rF&%^O>M^&HwoRa)jiFxeogf z;vp@%Al=eR4P=UyEa~gj;dOC)nb@`0P59R55_G>Q5jD<9pe~2@7C0!pwiXl>X=uuq zIdFVb$bGXRr71zB$#v>>M`MI$(1pEa<%jd`ewl*Mqg7>p5OKE~h5aQR(rGcGm=wvnAEA!ORzc+|10et_9a9{Z zAoiuo(q4Mnz5VjF41gW~!=az*<+=TndVVw0k@4Qk~>CrZC@A_%X#&lYg$mcLZtWjq- zfooTD#BG10e=`70@-6BgT~Py&9i;IxgX`CGcD=;YA;w*5Z3FXc#Y?^eL*nx9(@2;z%9R;&{9gvk4ZGrPSxOY_)>;khs~~5ZGqrd@-5pzlXhy zb5GM51f)^L#fz&6C7(Nim+!J%RNe=Pz`z*)yjM}MVJBdYUBWrcLnOD4SE)GiV!wQ1|4B^jr)a)1N}f zF@bpe7H)y}{eypYeij)ij9J;AvoSnyZqV3w+j5(hn@_0_-YWhIjM2sw|HJ=0agdzl zEW~B|iZ#mLXWD9t26MS}>;9kMOFstUm<*Xzp0M3_F0BIx-RZ`$?>swAzcAnp>l80I z9NRP8;ecWHh`P6%+{Z;=)ZxKGPy_pge^^0mry(;RI~1;?os>z3^JDM(1u6BJ@&%=a zkNG6)jDR8RSPJ~6nniQ^szDZ_Y>|Y(aHC6P zGnA0R0kWRb36qm`HWxt_Sf-;WjbkjA&0NuCNmqsni>+) z{%#Rx%OKwOBE*Ns3iHQl^vZII?(c{v#Qfi?wEPDwHx>l97!N!wi*-1`1Hffa3n+ z?X$tK?hs;%N3=?!Tb2$EP7+>Ltf-spAt;0ncUYmW?lLfMcVqjJ>`xz5^-sH zg-2N}6ra!xZnk{<2X<#6jjTEyG1!eQ8c5KsK2d6JIyYf&x=FSMs;c*iY!hlyb8vre z!GJNty z44!JT{vrP!TbjuWDLygU{$CFEKZNNG;Xj0+gU&sQbvfa-APUH&LCiOJ#Tc=BfTTg7|T3g&) zaZbS}WLK0)m$J0+f)fDxbB(j@PFXpvmbUB`Nw|-@A(TfP9KM7r!NL`Yw((55^j6`_ zdtNr?xCob{kIZv>-ddMQ)=Hdz&;b&jv*l8F3#1|i!7_d`jQCMg@JVM7=`YfCsRH4D zkL^y9KD=`JMdftN_ei z3j!OMoR7`C(wWBS=-w2q1dDu`NDB#yYIxNb{Y~e^^?P`YC~Ih?GPRy&8Hm8lEjHZ1 zQDJU)?&p-C*rxZ_s5ago?7bd$84P{jIzD~$E;@OD^f}vEscxShkk!q7d@+|>_T5#D zk3aRC*^aJF3=UtL>=S%;94gblKl;Pe!|BZrw|CTfOC1V9iF^8V3fGv4=lVdNY7m#l z4*kaHTwqCIXDLU;2j}|vESDW$oTQwboE@@xbFG~*B!BvZr+VG{p&K=4z!B<6k{p#+ zc`Ws(npN6sQ@>(Oql>H_9qwPV>qT>YT-4LJnvkH&Fc zx)}8MW=eJ)+kBi6`Plr}!8t?d6R99CNpNJV5{$tlajQ&Ye}C38Hfhpn5-R4=EMoK? zYw`+zzu}Fl1O9kTV=I9zbxyE?DzPLHHQu2BRxKAk$#m3AZTBI5v;u?^d9us+%i}TB z6%!lp_YW($0!{Qa!aPrYsn^{;UZ`uAf|o2Z(!TtA(7bf#AVyC(>F6px{H|?9RpdAm zInG)wTprw;S@E%Qr`}YrG}$dj0>CI61f}v*ZoO+wvI(gC;duquz&xvvbjjRishDab z>BQd7Mq~avUXPl&e#MH%DsY!QccAQhy~Y8uKmxsfe7S$PEdQmE(X*8=Uy;jfqLZQe zMGv&59G=-|+>;rPE1MKB3r>DAzUI=}rB96o$m(7}&X3GVUiZE1w(^Q51+?48gT;#b zBA%Sd%@*K6f|#fBys0Yx@Q&kB1xG-|#5Q~MRdNfPew}hJ&e;29QkI^!4t2Jw+m9Z? z=(I*CF7rJ)G5z6Q{hq6^Ics@o>0?)Zy}+9H&(PJ4#F{!5MUTBA9xL+{E(fR9Sd07+ zRek*H?MFoqFGZx2vXZ;kvfRJ1i;A}WRv&6%)0aWiiQ02;pk`EBD#3e5N#!1o%=g>v z^rgNu*Dg)m^c}UIj{0K8k^QB8dj9rqGBbx2zu!OAlicP6J9w1xLZ*HA;3M`>{I&7tv=hBn zp_6Z_cAL@7J}%ZzHr%T+W)BaV4eU7m6;Nb!f>>P*of|jTwFcoDHMi)Pe;npw3YV*X zLdl*9^6``GtgqpYZ=fglS1gwk^UCAtGV4!td90R)*pG@1QAuAqOM1KBvh;tu+`~1= ztED8ubBXa8qEVSN>Cw)reSg`0_NZGIM=?YeEK}N?21jH0gKep_l{I&xgT?wA@RLPY zu|YR1-owyR)GDyqtK!4V=eMkEu#@?pwb*=i>HdP|BBg@?6Qkx75_P;28c)$Wu;*3j zGUaDFVX+szo#NVmL_DOQYh zMUbgbN&W!s{XqxyJe*zsdQR_&WN!Vo)RhF!5UXM5{GgBC2W`>Y{S5)pkIe8yU%|rb zr7+q#@nQxa#jPyCTaAyY7TET>#q5zuyGzad?zX>rx?`{8F*m@pf*+OAj+}l2AG%Yk zBm6uYJTN7KQmdnkGO{d}Jlis;o{G5|O?d&oMczNEOs%!FBx^JsM3f^UJi9Vo&yK?k zjnb}rw@-{Z9gb*f822c7QRmce?-&k8K*Q5^h}MH5t84-$BC(p|`T1)uYX>y5A2X9a zE$X?5U3xa*(XPAkYQ~PWyja3rJAQl3$Y@+5M5<+X{Mq1Ru8@iJ5e>%<<9_`2mj3O{ zsMa+ujLr5Wd2!xP+6cj4FgP4S9|Tl{K+O02$CJ^3rr4lmh=Vo(tGKeYTpZjyCkaV@6LTy7{9G` z@>&mzivHn#dwP;TUDcgh>(m99-cIY_auf5<*jC0%@Uebi(O%4|@r9zY8lL@IdKRHL!(ASp1QRZ-q_sZiscgQQu1tFq#k!CXjEOgSHg zQX+Y8g;=lgx1+}p-2n%Ake3>%cG~`Qei548Iu`7Dhmuwa7bflP`qi*g?Cj_FTxT*M zm5M4=Ta=bl5OV{OhF|)|Mb$*8(zs~x+z6MOk?fw*`O*3V1M5n~7^Q#)hY-`5-E@XA?A=A z806l|Nw40FtpQLnUi56{n9jAA=9Vie&cftD%)2+9MYd!vg(s6Z>0=MCZ1^^qX(3Sh zia;j+mP6Y2@mIoK!1bpEQ`q;l7w>L!)f0agtDvv?Mb>{iwGrRBVsF&zOj1{9T76vh zGtAs2q8GaDM{fSg9h^Z7hf%tpr~vwFys+272fh5Ou-<#2Ef&w+I^21e!*jz3?-}t{ zRZFXDt`=_h3m;|L&v1wy;p0vRQukUUtrew)dMYF3!e#6vxSnALF67!SY^jd2L-t1; zD->*ovu+xRUd!rtoqou_d7a6P<9=qur&z#)l8DE@- zuhc)7{s_LV`mYy+`8N~b^q}^?J__rBr08=uWEE~pH=%82BR3ng=g57%oT~~i}E+~@Js?ZM%>V6w+_y1RKgpSfQbPjI#ZFzPw07JO}b-X0qZf^b;TL2 zvNh2BE3%?V!EQ-LTJW-aJN{Pc{pwqCNUe=J6ELqs8E~prdXQ>Gm_mFNyqzfpW3Nwd z)>}SzZ$)xtiJNx6ma1`l{atKq7C_1l19|_kekp1_?()!#yACz9zwh{x!C`{p z3qc(AjAvYUuHy0TtdmnZc+OB)zDw5zFpb1uJ(V25gjB6!)K(lv&=g!Lt=d0bx@Pk6 zvtS^xsPUi^19S8h?Reo%3lk+ap0`HxR|*67DfiXVqa78*_sQ`!{lx}3KZN98VgVI{ zK+4oAwP3bEFF&GpBj4qg>B4D5!LG_m{4ixRA=gA^KnCT;^LzW>a#3!5mvFS^GHi@| zEjR4!9Q%UAQK@m-&^@(ixrSy9Nz&p;VLt<9;Mp@N2GOu6&d1n7EH+io;##4rRlA>h z?AX4Ma989x;Q5XCr7M|Eg}bz?bL9}bgNVsre4?@MF3lyia?fPTIdN&Rqg-@*JE6q~ z*8-iZwv=^Z*$6*-E$(mSlEZ?oL~u~XnxTQqCVXoL!TxMil=@s2Mk&U6@Rc&FPE0Lo zyDrv;85?Rd8;bV@$t?;orzCo9%2d{g=bWl2rdTTrVxwzcp^Z`0?y5I2SLLmSn*{*F zcIdQ382Q27p8Yo2ae@-)@F}wy2`!mKD1gel{lZ3jmd`I2(*yD6<&c8Eh`_9L>pJhIn=&fnw5BS9RaI&sIQEtCSr zOm<5eed2wib&=P9*RCt)M94^&54m^7hBESEc@P5xyY&cnjpy8cyYk}6_fUJuTHa8y ztS`%0^HTr^BUV~??^fO`92eP6h%jklfMgl^N{H4sh}0AoPr`qDRJJ7$(e6(*S>Kmf zn<)JA;ES3MhT3W%6p3kHzaf`(L9FdOKgPy`y^Hv}&^4^xo(m~ac$#J)>`#xtc5w)m zhE01~!XibY=@e#7Y<7R#9*ZRQR7dgo_8KoKbj8_N>RU_f217;41mz#5n+$ajrf-M< zbs10A_@Y$M=20QH*_4quRn3U>ocQ!PdIy=FJz33$v766lF=pf%F>DYmX>yaOXOAh$ zoa|aDdM$dHFisp`IBl04MX*Xz{fC1|-A|N?5xtiO3_!{@l|b}hW3N2IrMGL_n`3|R z3%gOqp+%&pU$bJ$1 z!nx*Q^UC8G`wH26DU7T4M3N%{!K6ssD#hFD6J9HRuOU`H%JFVp zFP$ByFfEOGTaME&vfurLE^54xjW4Fo23!85wMjCsYVCjY7iLh@Oiuc{Vd;^_zbAag zJvjPdsXWisZ6OzPDSU9}qUS^+=d2*0!yvj_l~` zp^gou$P2TZ$|bFD62_|F2Mc>Vwc~f9c9NqvRMe3XZ4>W${)t9Qw9r_c2)MjwE?l7N z&2!TU#wPH~1;T0`POIf(+1@^AB6jm&@a$EA~FW4HfxhPH(>7U^WkHDKzbi**-!n zmJEN-0f|H+0%2+zxs3L6Rxf)A*4FNHZsFGXcS2u_V5}~{z`e}MMm2biVwnN97U)3aotK!Im`QJs^e9i_jbAvU745XY@&Ccc?Zn0ynNTu zW2{L^M~^z!W+A;Y_C>w_xrELfl+=ToH9|JhH$Fym<~I);w}0>s-g|e04;T!1AS=07 zN3tAqSv~vIkMy{cA5i<;|7$@O)G(|pM;ZThy1b99Q5F7Y66a+nJa9UVyxTProBPi* zf?kOJ6@?botBiLYBbo@pl;WHg#^$wgQ5GaQY3$&cFh5s)?jL8Pbvkiw3oO_(z-xw& zpFeg@!N<1w;3V6q$=kPmv&B)Mc%6lx+Zx<~VEh9Ndta^k41Kw-jkuyc+CFS~K_mf)x6wLn@oL8X)PM4#C_l19t3?u_$9kq;}!opL&LE zKgjF+RU(zB_bCCNz9E)FBG$eC`?GH!GfGk#+M=x>wH@Peb@iL~tsI~rlB^E$PBxIq z(cvf&CG>xtHaAFBMUFiTw{9r(ubdt|jyP@_N2V4F70< z)W|w4&;MH7f{-(gbM$}|xi0St?2e1R9wk*-K*>C@-yUxmHU8CTAe=Cvcv{FnTVa#A z@8LE5_#bX!Rj;zF0zs(hHeVu;oI`l^YVWGQB_WN5&R7MR;&VJ1BiWt3je&S)V95`a7SE zU0rLryG_ek7{3La{b1u262hvC)t_3m#D>P>)7OPXC~Sg9)Z(AEp@?hJIaT{TNW`v< zdO|KW{hl+`WVAE3WO)5*G_rZAgomwQbEg{L`fs+@WuZ8qCnWZbsfjvW@ihaz_*dWe zyvN_&t-Q{U4vRD_Ldci%G}M}dt_L{~+;;S*ue5i<$e#&TW*#{4#6GP(YGYp5w_uC2 z@>Y6bbLMMlkZ@#GemnN0bkjj?PC=<>02t`BBDyJ_zYM>FzaMPws`+lvc{g71sSVCE z>rUB&Q}*7_Z}Ex9(Y2M>7b6C4K316(v>CoLf}TCNUplDsEcxE*N)bNv{li2fCaY>= zXMHV8OS&qkqbH+fxV9T+Yx7Zo1onaAto&|L8hLpp2=)dc&bD}>=eS4s%Il1M616>E z{k23+{nU#b^U8{fW#YDZ6^Ds(r)twHXGBCOTQT=eK0FW<%cWR3lioaHhZv$z2`Fix zSF4n}mOm4fQ?h2JyW8Gf2m8%}JxpQ$UZzWZ(p?D${GowEV+3dexuyz;I3pgGE=Rpp zCE5fVs)fglRVdHR>LFmWBbtE&8G9q7WPGN2{N2*-LDU^uzYQMuC>`!9k5G`S7&zSX!63 z5r85YQ-gJkntOokOq#-pvW~-FlmOuA36B|`+YKbIbr6oGy8%`Rf8&z47C&*#YnM-{M;h-Ickbti>VO@T`2#$(ZhD7evtX zs2GG_S59~P2IWgWt7`QOC~;{Nb45XBFGfKtm?6M-G;w9FM|#i)gyceVxqM zCdS?z+EILp z@9!=_o@cq^)U7WnI?z_^#fomNuCFI_;lg!8BFOz!=VT%{AJ^C3jKpcW^}Jb+8VFS1 zi6Xg7h}*&oOKVC|S^06Qs^qCFd#|j9->gI_G|}Fzr{UkOyl2F~x1Zt+nR3|b6~F3j zd!SA_80a9s+ps1FtY@=poe#=;wbjU`HQ;F-)!G|_7 zYi&~Ye_<0B0S}(XFGk2xkX}N<@@u%j{j%XJd5vt#6gy}536Tk(dE zDTs9GknWO#ppf7W9IXL*65_$To_^Zy>MI}|wjULOyqu;4cS$3#fZawE^FWAEkur(D zk~+ARstiN}(;9ZUh&>;B?EQyZ4M7GX=AkCFoW(N*fVYxsheUsJZgjE)DRZD}b(5o^ z(6q9<_C#0N+8Ds?7~{rkjZ4;orC%Cp&twvZfko`sk>W6$Wv;*CNPf5bd7y5aL1BhX zZKaKx%&$BCNdG%aU?B97BPDAejiPMrcl7InUBF@2Nu^pa-`KckW(hGiu!P99 z;0ZgjI=Yzx{&sn@goj46gt9AIpY329Ls(R9#6|6%0n)Ek5AL?VC^Bi9y)10Mp})W< zvRejFHW{D4h)O#;HrLR>8x8ghEs}Rw(6qlz2xci{$Q9R2eZchF-`Jp2O=QjtGKTJ5 zgR)ra^B>)F`(#L~kI@AHTiW-#Xb!0nxp6W(pXCffo2YJF(OixSEBP>3L-^SYUmynn_Zve5R`9c3(k$m}vQCGF*^4i@$XR`xC^{*h znn@j@DNCpVS$eG$dLhPJAv`x6zkXLOi!qMe5H+`A{xI*t$5T2}?L)lO1T4(sr(Q30>zwrei<385F#n>_^`1W3*|4i8cBQf0o-jSrQ7JSn zbwsAnFJ>6F9B^WYr z#%WtZGq2%lY4sL`LIkH(z^{HKSi%=knAad^6GLKr-iI)uF4_U487{ypXKZcD?Y%WY_cH3Z;Y@b=BH&{G)SOCme#Y?z9B@0zZx$ zs!s~?^d6s)Nx4w>u=g}G)F0gZAbELz{>gIFfzV-)uW@5&Uv=N5WhRd7kx;E;$kl&B4WI?t`D zG9P_ft6mAUXrv5~F0geAAE~HMcy>t&7n8Kp4=&n~(c4!0fdhd+P^;ZlZV8%ZUjtZ! z6Lt=_`=thKrb+H)eAD9e@2-UK1)RZfL1Tw1tb(P zrtU5uLOqeQ$UyIsB}S(QB=Xk6Ne`@t^!vf<1FqZrt^-d`*lTwmCB?p{PH;)Q^&7&d z$jU3ypI;Y@s`}+h(j`Fg%XT94r^#Z*3yu4f&JqdPiUaTVts-pl1+%1#OOUzD6LPtj zH@pyR6o`&h?+kr;U8#mJNdVlT-+cR{c+faY++*>ob8m~bmR4ue{LCr!oSF#hp9%;2 zNR7%BtZ|tC$3R{%fNR0HEPpLzZL?h@l60l|3VCQQuy`(JNNAkMdmD8;I7~ z%KkJcfOx+LD4iO14FM;t_vy8;Tz;wER&Q@f8+c*gjQ0xZn-t?|zvDSb$2wQmXXctn zbW|tvOc!ub6`MMpdu2MT{t9?7ZM1;8;tt>)AV$1L7}Z{<RW&8F#yf*&)Y!q&LN(qSC$@0X;r~+ z-P(5#+rZBw26O=1ppUS_S^1LP3d&mlF-Sw}LkWDO)cgV8zKsR&e|k((vR>Jr7p3L-{SmBk66yOdV9$>kSDw221>$fKcQX0W)$7V{?VDBKow`bzR!oH zmXB3Qy#QYMnA1pET&J|Vajluk?ZqLuK*2&IQO>$7HVHTL90s#|IT>DR@KY*KCV52C zD}dMSX>;$0LqHgKAX8ef2Bo@_leFBln*`sbNSJWx9pn1|@iAg`@?-Kw`>n2B9V4nB z$rklGnp)_|z~-;n)C+QCEJ7jR4L~(9QM6+_)ihw5KVK4jt|qCpGI{C1SsbQLDf(u& zkwiNolVC+X@C%v=Fj=s(j~}?5|>QSk_xetZ(k3``M<&)9p}T{k7^s7x&Iv~XQGJqr5R zA6yX^$<(lcXKr`KDoz@5C{U9_WMT4?n)S6XZWZFu`J65n5O$2$yjBl$d0o#lZ?AJV zoZ{@p`h8M8ejmSaPoNawuT8~`ylq1{G^{LP66rW;gF8j{o@J9rWuIwd*RylvCJ3s> zEuBKkB#rm5t-28)5r6KiusQUO)HhK7%Ja_WF$_h`6<1LFB8FE<-d?!;!kHku1q6>? z-pcyeDwcN|6i(3_DXo6_Lvwh&cqV8ePt$dBWt>=it9J{-tzT}M zUGhHor4sFC_L|99=jLC)0XLYW*+?0@bS^1sAmd9`FTJQf}rOMGTLU>a@!)eb7>ONM}?=8e}=qw+yfs zr1sK?5F;700TrDZloG zWc0-VK#n;m#PJE54#a@{NE_j6KudHtyai$8P3qejd$?AP=q=u8&PxF#4ZYWeZN-wG zbWWoZM=TAVTQLEr{NqA~zw3*1MI7S~gr~zxVQ_NTVR03zvw7dB@!>%E>r(dH=X|=icjU!+5$9 z|F<1wtL|b8h(i5vtp~tjAr@`aRE}G|)BqbKPe0x*5kC=l>7m_@4f*6-w;x@ZOeSO@y8)UOF*9mReb ztk)yfq6KK{E^? zuA;T$xAHQ?&9#W6L|`}h`13VLvU*a|nJ5u}!^(l3ySHxk;uxAJ0R`Z+CVAZ`dbW4- z@~ToKAQfR^Y|U6~A(OUs$7-uh)^?je!=^<>UByvcBrAK~kGG1Z18|Y}nNn&4wZMH= zgd9x4$?=xwZLMDAf~{bDsxYs_7}bdadPk$-c9eSJ6i@+HggqAsMwtiAu%0aw=}XH| zbH(dgDHHWl!J%Mpy=HjugW=U>+Z{tcmsAHA5Ky0&L`4o5NU*4BfV)jb6>I@-Mnf?H z=2wJe(Ji<5c~Dd64q5@`_$ukDeRG56>5$iTPVmwJZJj%Ot2>zn#RL5j`D=eZ%LhVS z0?kC+hp+8dsjZ}Ih8u7L{{%jc*4lmx!&Xc;qW`l{wG42T|2$0=1f1LBX5 zpdjyB=|C@W!@s$z_ARpg+@53pOtjDgOIDK?4?TadnQ>-_;Gv#2ezU%sN@OSVBUWyL zdP|alO|iN}J~Jt)toqiggSY9~6EY&nVUD$tUvtPBKD}ktJ)8(oY#@(0Nnf};@Q@v` z4afk0={j)Bg9=vN;``9~!;2me0vvu5_?^U*py6vjpgeRJ)qa+0AUlcua*oQxQ#VRnj?N zpy++QQIZ4AcgnN_#{N0c-bih+h5wLNO8Nsy-h48ja>C2Rl6Qk5^qN*T3R(+(M=uE=|hJejHe(6k)29_~~UCB!{83OAZJKQpT8);HZ{k=!v zg@7t57L_PM3HJ7e1bYSbk#ux1f%ZW>)@8DSj=CRn zn*|TVxCe0>E)#Jpagn2+xmRf-cAg#yaUj+$vnaADALWj)tG*p)r^bH zpv4e7hwf^YS0I?hDpxcZ#%-zWwF9NF2DXOn^}EI!nnJXjRO?hUag`(UzlhL9ATCfrKSX zqckmutw-TX@MhwOBjc3yZ(V>Lt5xJyEh!%4h;ntHr>xDt*CCw?!I1mMnMo1bYjc+F z=&Qah2TVZY4-}=_W2Y&`p~@esO5# z#B{C2_t^K{Y3Rd$KcWoFS}HH-%520ZJqwqmNVm!228`M?K}m$RWNge<&*=oyL)i$- zMC!B-mqA!wUSD>c86K8K>gvuzNsPXXOP&_B^{^yhN}4SkgLtFzWoxl_K9lMc>joe& ztHa3f8%iG+TD65Bp}g;KPOU2by8#klXAC^84;rgZ4S#t0d)i(e%Dgz&r@xg_deE{V`C&X zq*d7*BQyPP-!eYyUL9)!gTd`*g!Il^SJ+8YShHHT!KuwCjDHjNvrUH_Kknj`OmZ|( z{U7=zu4*ZM9{d%kkvpv|i#i#X32{L2FBbT4^OWAeM9aHKi&-}Q#Sieu*hUREAhb=1 zI+{=oi2fx7OMRQQ(+(Eqx_g{o@zy3N2TFNq4wj~dYi~RONy_4 z|NldauCBBHJ@R2p-r;+}2{>kH$Y|9AI(gSGe~oe`1NeevCyP3)z{SWz z(&S-4-I4G(1|N%Gid{;Z6TNmd8gjV)9`6E6{omt`CHKp}Q)T`?x%2;C*#E6%P|Oo@ Y8@CKFxO6<`@XR-jZW~ryzyIWa0Nq;BbN~PV literal 0 HcmV?d00001 diff --git a/website/www/site/static/images/case-study/linkedin/scheme-2.png b/website/www/site/static/images/case-study/linkedin/scheme-2.png new file mode 100644 index 0000000000000000000000000000000000000000..2ab4e4281019590303b4c8078cac874e6041b65f GIT binary patch literal 91660 zcmb@ubzGD0_dh-rQBV*;LL{Z673q?YW{7knQX5Ej3{Xlyx?!X=jL{tm3Ib9iMk5`g zYXSn_o0q=czCORde|t=D_r`r+_jR4?oaZ^``CN#qiW~txB|ZoQB2bWjt_A|(JpzHQ zrr*Q`j+C@o76QNU?B#WxKp--v%YRoC)EIYwgIAo?4CW>oV#{5?_%F)Ve}i_3o%a<2q-Z>Gg|d znU_t~p-NucXY2*di1~?&?fjZdt*`nLRj&{~-f59>7QE8Qri&`;P6+2qzBu+~v2rk$?+f66nbAb5|K}3}keiRL z{P!aeUYxl9z6SA9URC@*2Pg@y0CVx5M|hAQ@qb=|{A8si@cw&1;v;<~&VLVN4*vf> z3?h=&+1Uc@jRDc0zkXqE{O@Xham2m6$o`eE)$_S6^0kwYAJsOyGm&O97kL`97^2Nh z0$5WXQ7=EQnb!j+<72Eb_OM)3u4o9R_-&u~KXc``m_1Y%t=tmaXv|Z{{g}ww?_v7e zs1qZMJC0_G`asr`Bc@TbNJ^)6;V&BZv(&j+-}c%piD5Xyes1+WKSS4YdkRfq?rrYU68vvaesR_2J!Gtp9z`+8MI`kkCnv`wCbEcm z9=sA8H3go{7G*|C3vh#4?nXPT@OBH=L^tiA%9qXt6*h-pyp(kzQ;(JoPws5k)UuEA zaQ+kQvkBiJBMjf7_^u91Ja4}5#oSf7IVnY>tUUL?1k}IQ;CD|`M~4Ewp+!ncYH<~Q zs1telOzHIGj5ZAUS`RU&Fq~RROH5NTH#h%wY>(Y<@U8oLMiE^yq@e^mCC4;?nDvrM8ryM&BKGrwVA9z<$hUL;^}G;=og}$nfHOQ`<>WN~GuJq!i5VH()FQ4WPR-YD5PnNE z_F+xsx8cEWZcHjnSg+sw^Am)(RDLN$l*o|JkjN9eN`2Qpd>tg#Au?%#`LPb-tkqCpZqxKPc}BflxGL3 zM`NfKn1wu3ZUYx|z|M75VS-OHjqd z#U-h)F!F*x{5IE9qN4}jbl@S^?;V-hD<~9FiMq>A)Y{n<>4QK6W3HPgMub#?kdBE6 zY7u!oy%gOzSkaY99sQ=e|8+&oYTDKUh7nXk550~yp4MjAO9>3Uy8-g+$TNNEu*?6H zjM{Cg{Ig#SSKlkO5_%@Rrhg3_1d<>rXK~q-i0LI{qZW1^GZ3XMiKffjc(fsF)_rttv<~{*FkjY&To!SFv#5O?6wa=lpnxM zMsI;QM%-&{P=O3%hUfgX0?OW_&B@$?f=7$Oh#VY{gefX2hO4WCJ>dCcJW$jJxMtiN zg8Rt-d_KHp!lC>7_c)a_VPFWmaj~)L*4_@0l3mOOVFDmf%v&+8z*}rGA{Kv_kvHYw z;2^3;I+&0Y1frDo@$va)j-`l$%<0XKp-F(%W@HyBFw2OPaSZSNGf#dNxxN?Yjt8qR zh3tR$9c+xNPgGm8fIz&sh|C;%CL3C5P@y>mmfN%q)pFgFbuy^ zE7n4MFKm89z$!zck^S|--*fQtxJ}=5yZ(=n#e@V!Hg#|{j_6nYogBdC81{>JUTQ<| zKz?R(q&eX=y$}A2ARuOQZE}MuJ6Ayi_nM@t`f6+kgRJoWgGwYAoK)6@$4Q!S0KZ#E zH)`d(_8$xb(KrOJqGkv{QOX;yZ0hJZIcNTkjt)L4=fNs{ zh=q1GT@nsR{<=%Gdl(t%|FDw-5ZE5cMNmBnQ@MQ&6m^qQ6;)o4o&Ask4(3JL6WaiT z?CQQ>Qk|&t*CxKitKdvJzor2BJ(us;{bv0-(Ml1Rv&Gv3>mFVqr2lu)j&f`bV;eQp z0ZR#U5B|@D&9gBuFu039eX0(uet_A7NVtvT)W$2~5nyax-)TBu;|=HicF>0d7tS=o z&fns`JOg~RhNdPSsLBkXSIwh@8AarRQ~9sbX|(#B<&>2Tc)NY;=>axAG5vT4!%9x6 z$a^#6ym!PPI^DX?lAs_6%;$C8<+S?=Q;B=)ba!|EWLE**WZ>lN{UswVd5IT?JI`CI z&*vh&3QJ0Om7vi2|Bt1`#+siCD*0juGGo%+(kGO13Vvh8o5k7LQh-yW1NrSZ^CIJ`?%x>+Em_17l^Wb)kw@lHiE$X#CAzG8BIcw{;JpR3=s zRSwO0m`VO;07M!uuKERu{1a9|AX`_#E?^RF#J6b{1M%}Nd?8LO=&Rl)H{f$GPxMNS z8bldU#-i82sb3>6Df!mS%uL?MDE(k{P#W;cR$crw_3a8rF8o-V%bSQ$+ETSCkz{)K z&vF7M@T+6v;NV3Nva_*$|MD0F3W+LNcnRZl0*s5S?WFc*firo$@)LI37mJ}8!(T++tiAo)vsgzdISjrb2BrXeoG4rzzc;$ zy;*qK#Nv4c1YtWG8;L68{NE;!FgV1Z`;~DpU+jUq0RdYKxDyZX`g#EOTPBbM&z==# zl<0qsLW1gWb+DZSxI?vyaJjjn7Xv4N$M~?0KiWZDnhO~6{~o+NUfGEffMsj0QCM&f z{#nf5Z-W_UE^izd3x(|UOQZhtke3op#*_%(?OEgk#yo#}q=UE&O#bh?Al@P&C-QF< z=Io3tEJ_{W57_Ic8}W$q3*1w`!-bR-6}!!Len|3o4l`ZlKdv`Tv&7&muD?XINP=J}x1-M`(2M z^=!-WyDwgLbFb^&y4yeZSdV%$Ae(-DpmSiJam_dsFA-RAsi^V7mf|HJ$o80Ozr=TY zFhC?QNb?u{|rQRJy;SE!kv-@M9uoq(lYTJh*P zvhwUfOkzZ>dY&?Yk%M70*1Cwg;A z)c#X*Iq;1)B)ApVB=qXPDPV<0T9hCZmJ?Mhn5GwQ>U>4DAE<;x-^h=MZRM_0l&17x z@A7hb88~XasOYSK{PpQ{*P&l?ytY%>ZXxjM?48xX7ivcOKC8yxhB_Ep(rcLA534mN zT$v3hR!S;ygA8ay-7|Bt=hxRMqt}h#O`WMK&qbtR{{4!*Se!xt&^qGA!fP3hN$cN9 z3{DwO7V_Z|Os)#rJ=3;5G_TFiAS7S`r|=qW)xnyC?1CMf;rM4Cl$-ILa&yfOBXm6* zw^lwR+a;m8yC*Lz8??Bzq!39XhO?VX zVl`JZ4k5+SoKFIcoc2~R&rTu42OdW&E3&7J!7#n|E9?h5V{DqAeUvlgkms;5sj?q@ zy>BPYp5e^Wgyvvca9@Nk0#0hG_SzuIFT9c|6>XVX`5Ip1TiLGA-EzT*5f(#cL{nNp z@i-s57$fT(;orTSEac_8#DWB54~z3q@T2SC9zt|OXE8)t@h{l?Z=5Aj4Sk02`Ei>C z`UpDJYr43c+MyF0{?;$1#xi*q#gY8f@@+CC)%le39gw%oQ0~G`k9q}^oU&pkDEA8Y z#Yar)&2lDRVA_|tfgtC_Qww)kxPNvRDpCI0KMq%<`&aQZPhkLQ)x8OBEL zYS_9c5xd0)w7~Y2d&*m2drNK6?e9M4o~$w&s<>)>!6L&{Kt(4sN*R?|n=AA6)6s04 ztciK3sI#z63??sjBK+k`hEbalQ$$WxN!J7g|G_b_LsQN3!pAgUCzx+=DiwKKW^|q5OW7lZe;Kci2BrpFs6MJR2U+PJfpE2rDyHGBoRS3WhIrwxl*yt(Kb@E(h;b1kOI&98Ft-+$Ug!_XG)u(dl3 zU9Vajbw3Ul+Aej@U-;nYW3;|7k5FxZ#v)jK8TuiD?~L&;X64%5UfpK(0h;JZy>N9c{^66Y4DcFJjb`m6a`1 zld-VKqw{f&=^0Nk2SN!!%GjX7N5JV&5)YTH?SD^h1e0;;y=h}#C{cJMke1`9(zLb! zZrW^#!fbSUuHU!GsxYLfWL|b|+cL4{K?y}ew`WPUBVs-5`bzjbwCCNq(v)vwDNYQY zTJ*(WOJJToo>Xq~3U+tKmr*>s6;mm20+pQODP+$(JuDiNwhhKkr6_1(F}&_OpM*2) zdME5kPJK<EftwI#+zvk};-5Vq3+*$FMIg{fDD10V3kxu@l)gCXsg-6Ws%#L;ovy z39h4&8Ob+CL*|;AL?wTXY2JN9a8~SO1Dz*isb}99pkL{^x4QHZ_eOP2q1qIM+ylI52EUvGF^8bogV`jl9D6NN*3$H- zBhMFi8_lRA+rWEjU_8ZZHFHV#|TNQ9vw%InnDYnv-7XI^sS&m#L_rQinHIM6iO9p(N$N5 z)`{-CKoov?b>ZS8%KrFZGuDo6x^6NDLi$c2f*(py7+~7U-G?ryYiJG?%PQ>Oek1wO z**J}qr|RL`N|)3YmOd?U0NAK+TT?|Co;`v`GX#xHj&`Q|jgUQADAK=cwS|K{)d!D~B;q<7Np@|vw z{(;wf;C;SNfnnnO8RarAIMwrC4Gk4LPlC!?Z%um-QB*!V@%*({bNX&OBZ76PWLvC{ z%v-Vb*7$a-c*ohDDX8Ab_<=~g*w*S8&sJXflgNeWvew8k#8?}QcK4*z#h(GbC@voN z^jGQW*RjMVxT&`Ee?SE&=RGQBFREv{F@vsrpetuQqDwGM@Z=+(!t$H#*EhsBengc% zby|MlQgd&xa}aOf$AUAtb;Yja@myP&)Yb}N->MU*!A}SIuW#q)5u-W!Kk=l}IEFe2 z>-$nfZz+e42FM;bEZkQ`g-F0e-OO)Iueggxia47Fn^wiVwL$PXta_`YmAE>#bSQ2Y zhf$wEA!{jaNiC``q}QyLZ~12;;cOOlgrWmBZ3e0LdFbY7Sk6#2fIGD;MbGbeSD#; zV@H((H6c8elX|i}50j3eQ*f@7eqBx4oI8xrLO-Lu+*{nLY)ptS=))QD&o@JxXe(7$ z{U-z~V+-#@AXdH|6THJ8O+8&*wLWS|57P|~_5WgJ{Ateii%e5Hr#Yy|9 z7^w+`L3S@&T$c}t&`_7FV3xOV5;?u<%tpDdv}sY@t{NIiBzb}H;A<3Hor5;tUBIY% z|EEX%f~0oSTrPM5A-N+s@END-GNtP$l+zne=G^DLnb0SOh1Q~vDU2MQ8gcu{OTuj( zMP_7;1i$aYhC{4a+%VLZs@x*qTi+5!V8X&M;t3D4wu8~u!tOiwiETxuFZ$Z5&76%g zRujMw@2!xT23xtFB~si``$#8cE>^IC(l;TeS-f;CD(7zh*NYACays%592;4Lv7+6K>c+(wK+ ztT!A)r?AnLeZmZj6{kARTVc2@4?V=D=&c!pt9ZBNUzV7X{w?Xqzl^Su+@DX!pwFUv%7m1^; zzuFeRl+o*BfuP~Jkr{&baY5Wlh8btzUL0#G(%=I}#tfT2A|;32h^WbwY{$O6C3Z6p z>RL?Fs^%I?`bpBL10OQ}sRTw8IX7vKguWc)?{`U2Gc?U$v)90)*mUNOz6xoP^DS&! z7xvNA0^7D^bLMaDF6trt^X=T&xbgt*so2zd7v+w6mi16II?=ldbVWkzleKO;P1=~m z>~&X5eoN9IhzJu&GFcX>v`+U~agUgrTfUZh<=54g!(Usdkj6dOdvWPW3I7Lej&1 z`x7nP@twCtG|tDdkgi zr%RXTpf6I`a3_2$k1OK$R+8;-wu`f>0B^3uhN=;GS^_I)sbiVpmth9dXPlp-_wIG8 z_c!S{%|1p1r%07(ej4x|pJQ}O74UkOXs>295c*is-TE{{i za&4;Z-Y_qNicStO0`xh8>u;hWe|I9(M%Lx$^WNmjPXT0VD9UEboUqt4mb42!D;->qcGe%G>mtq-0~Si zPshA~)GLNs(uHuMk`pjj? zKALBxn?IN1?rmvx6YEyg0&8afwf6IHmEn#&%&{VNTO`73W9ZcwBc@TI8N)Q!LKinZ z+A9?C@_*%ZKZ_!}3~zOruJrFiM4^q%nv5qpkql}idb%y%xqJ!<+aAUU)Z$E#SsqDx z-{_OMqvCqK^YPGEf(k5xVPQ4bS>7J*=~@!JiQ(rD2e*41wO*U%M!+?)Iqx8RX+7zZC30W1Rr_cwm~Wk-@nLSy zi6k6&OnTSL6{JPJvJQyz(ED)tpYWUU3Yi%g*3jeP4BPARCGuSJq~DH)#BsV=J#SRES+bo?Jzwm;W12|a zE`S8vOEI@kAw-Ph?8LokBzl|$b{PZfI9qtC2>!*qQ$u5!+t`ITW`*3s;C$Q}Yjviz zT+2m&01(VIyy(qNbRiZRX;(Oz8MB$d%(k&Ua6%gnyRM;`U>F`kko+3T2C#$6pJHM^V1UWBpsG_@ZfI zl0_xt_?(T6EhjTmg2;Hmmh!<~;;Qfy&-Uk@FnlGfR z?l0j>uDfGsf)8g8zf9)gJWCw;7TumJWWViOS61aizKc%_?a^a7OP?n!Uv=L&r-{K& z;M_k~$(`+l%Z3xe2>5g`x2wh9P^$xN=#!LD@)@h5Q4gc3)9eevCMiWv?>b|4MsCY* z{G$muRqoe*2S?DqCHdXltYB>;A6XyIuf3nU@?VoMdya9qLOR4=3YsoI;IW+=dv!gh zlLq!lIbj?-^eQsQ<>Zh+gG#UAjrq65(caO9E{#J;>@kowFu@R;)%qbN4w}*_z^+6n z%5Cv`H1sWvS{I%gZrmJ!6qm(#GGm>y$4ftCkp!N28bQDf>&H`5wmhue2?7L5x0Ojt zbnR*^cYeIB-pWUY;8s)a`Ai^vVdCDtt{-bQJ%IY%)7|k}J0ry8e0c^zE@?5D?j(d( zThE+nAi*J1NY>fWwKv7?yGui}Wm@tt&2BXXz{*^`1d#l8@|-92KqWHwbGO1t{WeuF zRzGq2%H8G}>J+eM!^KHj8{xp1={v3J%4d;bp*6|);U~g%&%zFyxLhG$X`~N@n~}IL zNvKb3-k-FnC!5aY4VX$kJ3VYnj)3wLu&!A{BA0|MzNha;c$A2gYzuB}->Zg!IXO9($?X31s(P6y zl%wuWpA@D_YH!c8+oL`BCWy{u^-KTczC{4ehc`wfo_W*gp8fzF+6Hao74O?=uQ-l$ zd=FNYmVnAaoZ0Ie?|ENUb$kS;+U`@+T%m^%z+){GSvujq<4BB^-*c7}Og8CNIKnFw z%$_0e{n!V#pDLO;+}z+2)yxf4?gmed%{TpV>(xSPR4S#RiqeV=0Uu9~e_V4*IZR2R ze&l>1lKQmvsP$>HQSl=tX)j@3t~P~U|A;W_QL>;aDNUE~_VD{-Z>;WU`EuK99phm= zN=&!(?fP!l_ff}bZ9P%yTv+NMHqLrXd4!F%pDna%i5)3*F1W%?9h;^Z*cQH|XVQHp zCe5zj>sr+?f|a^v8gg^ps;oJuiX$6VE`1=+NvRgfU-1003kf{@$Sfvn(j13+-MUGJ zjHAenyhq}CeOKpzGWl*ybo5=(jO1ii2$gWx@#oV@1YXMxX|@%?HO6EW=Uy+Tpp~2g zYBfr+lbA9L24@3E(CE6B-`?0dxIOy3qE{HY%y)O$KtQDNoQ+?24LemC>Dh9wcR#b` zh&_Dn{B-*!v{;QiO>l(mj?t!DOmdLVR^-FS`KQM{A?oY5t@)T&S}W}@v~Jv?xmR~* zE~oCbsFW$HKfAVMx4OkarxsRUYMyg|Wi`n&sJOb-dL$|_cXk9kL&csLyKi`)R%>@_{ zFP{HXr2kRog|)ph&;+G`p1+f+F7<_WsK;c?suR(s3A+$y=1yEzM9IifT!qh04r76w z^(JM8iUc(9Eea*^D9`c`h8yO%OXfS(`{l6yT>#T}X?UF$Wz7jMy8 z&H2H$R$KEbH>CkB9JSGJGVN&){qu$u-n2XJn`hz*CH7m z!~-Kk`pj}QgSP73iVD4RNBCqTC@hS;+(eiVTyrH2S)Z%S3UHQ1wRmslk2Iv5=s;Ox zTttDgB~NUF=ehwKJ9}^Th|0)iM^`NvEeL;KPWKXSS`@nM^<1=b;rYOk>yf*ul>EqB zj+K?>2+hH*ShQQEd<5&tetIsaOe`H0I%?q>*zU3uTqLP%vo>0!$?mjnq`|8h_a}jX z@XH$VNxtTTyz)Kn^^d*YWjR?xU0n+F55wqObIq61(TVW_u=pFG-dGh7iK3OKaMw^& z2u<3C;`3WkL8#5qgh%BA__(`8km%dU={Qz(f0LXBHeTh5gdSv*&{)E^bPi?GJ1=0U zfN15$mPtFj#ta}Vv9IXIcDULXej$CO7^m^eEujdg_n=f`rip_oiF;p2cg|^`qn_}k zFF5c(S65&qv%N76aA@hagOZ81Qbi}%sdp9Uzc4q207<9x1>?HO!8te|nV8ZluI-OY z7`2j~#@w}>ewwE+cS7*FXS%^9p}x}i-WRTC$V3jKCf>z{jg2D|g8ef$2&5KY6YEs- zxER^1AMr6TP0LiC^lcH*TcU(5Q^VLa%2)XHaB%!XMlIOq9LPVtEajsl^T-!lLy$k6 zN(j)$nPlziyRRbUtxVyI_(1JdkH^GHqT;oDm*gC1b{BS0cO(lrX^ow#q#c>bCqd0V zkQNZrPu?v-WvbPk4c7?jz3d)8+FW)p#gS*UTncHhy#e%Q1*H|X96kU&#DDfayTM^b zNbNMOXhtGhX)DE`k<`Fd%L^J(!|Wje6`RYEbwa1rrU1Q3UfK!NS3(-G!|!1{URzh| z?H3;pJ(Pt)ERW!BQc@zZ%;AZHAGxA629b6xi>}+uG9#LCH0eh<60`tc4*4_CpT%nszIG52QEBGMQjr+uEx{vYz-eV}(ePn`03=%SCxgfJI7JG+hz zvsP~I(p#H?1&0H1fcS+%=x-ouvd3qFs=kKBZ!Ltz7nhvfZlckvW|{I*(=y`P&FM$j zY*v&kNMva1r+Mfa*Gb~v62PtHM!F#jW_(vpP7E4st2)!X8aL6(wA{~-`72N6XSRYw zEg2^@e)i(32Nsvw>}7XO1843%KA8c!<3#l<=`aoE8Ga@TH59(|Q;U{44`UlQekh95 zalfuS#>&X!_`76ZwRp5`mkF1SUCsbY?oxpnRlQ{aTwrpm)HKR*G9I!VrgQ{>$7C-ZFGHMq-I=e<6n-V4O|iH$l%ETLe4i76O~899c-FPj zdov3FWBIJnH@LBzvE(I5{;HI!DNo(rmgw$> z7e34(4Fu{Cy0i5axpuQN%q349zm&!Ko;lOb>euOnhSjE?af}-?3sF~m_aPL=+#YI( zctso&ZWtm(PQX^5_)#trx79Mk{dFlw-lwmoP{^XBmxM0fX2!rws@_UbgzDP*ZP)^tf>A(--X4%}2$wSYTC>DUV!Nti8~S`n@$@J~qU*&)B`yAb`jU2fCOA*!P9qqp-AaD##!+H0;?#p$`UKvv&E4zG z#$|K+*$E^WFFEiQ#rWhRsalb>F;J_0uA)L_5_@wEGfZs6ZUatK+*|{>g;60x?}60r z%)S^P9$QEdbN#9&!+Mf(NsIv8i27vBb^+l4?#AHJ?pm5YJARQ)vG%< zgZ)o#q}xrZ)LV8}MTN?lb{f#Tfzyc`cI4Q{>)I7OJeti0G6&X`JTroDec*G9q|mYF zSf`U%?&r?NXW_8Vk+6BbJPE^Dw_n_1TB23(r`LxCB5 zJc4H41y(2HT*+LPbZ;Y|-^N~Nv*l{$`jsUWH`WKcdhK(ky=`SsQPo+pco44DMkflh zVKZ>ZM`iA$Jv9i@G?YxTud0aB7VIW8S=&5Xh-h7!_;J*!RQF2RXj|fXuKip3X=50; zYCbxa`sW~ac>stz*_P&5{BEcrmNPQ5`aFBg(0nydT~$-jc-;726V*Qy{apyQnZS zJ{7z%;C-3t;^X5pGBC8CaP>?7bGt z#9rKtmoEr~$hM1+*fl|%M&d{d^jRtQCA>Nj+;z-A^}h&VWokE5%C(bv zVz-m%%$Jt-8mccSOC3mRq*n8+jJNK^!kYHX*Oj*rEPGfu1ZuxXFa z_4ZFDCw&228xT-A^_Er)i8yL-+2mMB6BrYX6gRG`dE1B07N;~5-fp6I!S&#kH@Ayy zk=vphKWY2ME-E8sobl_Ikx6Zru6bASd`wHQ+q6fq5VEsv{%env)Em!@zO=J!ki21F zV4&mTx4V_AUmgQ(U5ST`828};JR^V)`A$2KaV1ISci+?xx1iWFGRO_C8PN1jz_!)H z-f^sx!zn<1>g^+^H1?*~MchJH75fIlv<&t@l2xJMz zNw@YKr0jdC52Rf1TvI1SrBs661mrnnDREN|7I5}gHN^hYgE?`*45RmZhKa(w=1I>Ur4`axNWJ8;d{hMFs*G#$5ss6 z+{6p0wPKUMD-Yl&38h!sDL2XGvts~aPx3e^h!1{G$y)6X>!A)ZV0r#ardmp3*L9|h zUad}?Xk$R42S@wI*J)uT_N9P@(xg~rUO!$f{mek<3kxZ+(QM?g9Zl2IMZWhp2#y1a zU*9W5y{Izbg~5aKRRp$8mDG*t6pKxSirrZW9*Z3eI1e?bu#49lZ|f9$Uy-K)DCVCP zNdb~Qz$$#mIO%?`G)*7&`Eu*YNViq}yXorOx`*}XNeGl6g7TB8Q2h!>JNq02BG;~Z z;!Zi1SFQ!$EqXY7Z>wQntMcsRB=94Bu2-INEP!T5 zS2g|S>Dfit-Ouwwp3V zFJ>cmYYip~HqSx4;s3N1fsreA>=v|uev>WR2pK&BV>Yzx2I+qD^CYUT-y*~9 zx0aA9KIi2ps2v+|!6l%4bye6?*Pqg(*{dr65A9jgxb7+idBAHE8^4hi0br>iE3iMC zB6hm?VSEcNn28%{iNI8)#0P~vwBgG?opbFSV}F-bp;K(g$8tg`*bejbw$6PFLms+p zjaFB>{K6)do0iJ!!~^jLlv)0g8bF9vQ<`xHP1Gv*=muV@KNcxZCo%V?J(qKuGPR}_ zWhJHPfB!Nx?l$N(%UuJrYnAXMuJju}LVq}he#xTbo~%L7HuMqRTXT{ld5<~-K%tJI z3~O0>I_^7mWRM|u4?{^drb!%wsMF9z?Idd7;qT+ZkRM^z z_BreLaja9+0r}KP_y{^ei&PCq*%+5Q##>JEARuPjvA8w|K1QUeaW$V(`K;4{${|50 zV@^&aG&rj2YhpTjUDaxdyx3~mxcxzPr3Dbtjb#&+G9ACq+eU0N4Ef_*+J=+aLe%0G|vHDH{23vl`IeWQKy| zq}eDF2K~C;xsi250 z=5fi?6B_M|i*h;In|Sxvmb~(n3Glu_#$?HB$&K7O8SQg;Asi782?*f*^wDOT_sg6nOaWT3UfM-s6&MT0cp-`-$JRKMo-}pc zlVI}GdU!j_d{pTQg60T>x1!S*l;;DYn@DJpo18x*5FX*>HL-2~nMg)fJ3w?t)U zXa86ldhyKgk~-e?2D>$lLP6u{2$~^u^3tr=`YDJ|5u8Rw?z?>IOtMhYbXr2nS8Wuz zxuo=y*&{#_NkG5{{m~w#wKiS@vn)dmX1+aZ$Mu@3(u*?`Q}CB@EfzqYLL^uc&|i)x ziXg_uEBr;jn$k_K$q;1qXH7y|BcjD4olYsOC)J{HG4PRZ2R~Wa?;tvIJ4>|#xd=x2 zasrJXx~vZ?T#h((nlQhps2k7)tf=YgLdj0OcK<2^K5dVmco|-b0 zrzIX-u>nc!$GusIrdD+fjyL08uC1+oR*gxY$YK;jX5&2?oF}-v zE6SHhqw3;(@8hM!WjVE~H+?l`(+#p(bC29FY}2mGlLD@lu)ZXgv(HNL$mJFpM-izC z%2Z6VT|vT73u$Dd_sL3PZNLASUBc`w{9Ct{zC|;ry?klH9CYQ9YEKo(`(d$^R#FTP z;2%2%N5U60IZU-1F0Tus4A<%(2vwj|bgbZ32WKuh;yxKAw|%VCA}8_~k?!0dSNOSy za*RS;lyA~7%e>pMdK)N(2yo6;DZ2w9{`*oz1yBExEs!H|NjT+@Gbn(E?m_!RvraRy?E?~Ng_FE|19=Laa>RsAgs4$RGJ*mL2A3l6|l^go~ zyE5RL?*g^G3Kl28%RVi5&UZ=ecC2-AiZB_BFcg1w_aBx!gh#*TO(Ea?LgyGBU1kQi z?HNvp-aa`;%-68Q?ok^N^V1^<9aeUBB_Jty(?)%n7U0DBY2y45;Phe6xj#@%ml z!8{xvPhQsP$4vJG#HHLK$O5@21a4n1N5R+u27GKAQ2b^1Kmd||4{)TMJ!^o?;UwgF z04Y0J{kt)MLTT$@mDNZ$2Ty=Q2Z>PR$sBg?WGHs^a_SPK2yiKz^!y zP+g?G8laNWz3jRZcM(1U2;0ez)tasaWsKT7c!3GD*~MY zz{(ipB5C5cTQ4rfYILXA?LwK27hu83zyJst-UrlxhF)PVDuwg5i{ zFf`m~_~!IftxV|e!FIg8c4HB6T1HL8(=$uGYplFq+ISq!_ z@-zC(Xh3wY17h_eGz!WfHuK&qw$s5=?s z83B^?$UK66({Z2)5Dh8JQL#+p zqcNsw{pBja{8@3b$lI2>cuHxnO$rzf{IZ|QEEjB-@B7*U)^CyV|9uB-fcyYFl4DO| zM=1|P<>KQkg=-2DMRaf+(ECK6vy)_ny3kO3$^8U;QnIv86Slw=LRxVSpa`o`4FUys zcPmz|TITq)U})dwDyp(xYJr|3(K^zqh312*T`ARp0KZtE!uuE>Q>7YO)XrEwuoHrp zc@GV11xU_0K&RxssC$&lMCy)0=vl)zXFPx|-RoiKYCB6W;Fl+l5Z8ui-8As1K8K@`fB7yc03Tc-PF#Zktgsn3d^%A2E z1Z|^~;iPfd)Rq76W$2(#!8IVX$H&CP=+)j;mMStu0UF?dA*vP@$OGXk32*oxOkpO5NZHS7PG4at4L!r9#j1_#Uqd3 zE|t}0cm8YjM8neJ;x_=-*q}}%iWKAjUYDRvAw=?!5d;xTdu}&bgP`LP=EE59Mww+0 zt18;ae4R7{SnlQ~ENeOI7j_)UOBVX z_meI&K%PiNu>|;TgPy^Ua@RpbB59e!{C4j@i18_f)hVc!R4BXDGDxB9) zGAXIKN(tTj7qj~nPMw4#dhQ#8H_R7r`qYM#b`r!SH@=Gz$+>48R{KXcPxl#!E3cR> z0EIJg6Bi(4>=VNp><5y|YfjeMCC0OBk2Z)Wvrr$-GJrDY4jON(k^`%tZDhm+81UdH zf?2O@)8M$N&6$m*ow-ljrqsfVH)AG&-r`aLepXH65mbo5p)^9{9AY;80 zlpLoC(AV3UNdOu-6O;M76Nf4LsT`h_pZ732Uo$=e=_B17Pz{#ivew;67xTg&pjs_2 z8TniVGeH_PTU}4^>&Sj~4-iGFUdCx-*kQa7Cl{A%73x3%a0N?arjhYL+)z|eiSYil z%Mz15(_Mvb+G)Gx-@JGQv?~)R^N;qQX?!H^QVdzRQkZYdn@?E-fzp$t27$GFs};3$ zdxoJyOF$BOZ#ew&-s8vLP6XjV|7c3(Gp8HfdSaNAq)Hz=d5tp?>xYB#O_a9*f)Zd{ zYWv4UoB_VH)8;P!2Jju=F(A|O;>>i0dbH1OX2%Jb?I=n9dD zRN+la0L)pG^L5(CL_9kQzA3x~NS&BA&v%<|faV!=Td_c8H|OHc&_!&0;$_eS6nLqp zX+SrK?ifM}_sSOq0*P}pxn;{E&T&AhZ{Aj?|NYn(c66t!tBcrup|0?=L+mepJ3{BC zZ+*6Q=YB8A&%_3e4g_RR?>zVj#r_r#GR8T*mR(w2CRVxXN3j+;b^W^aVz0nT$~x7_ zwYNTb>vwkZ;i%4&51U+oRR8fay4A>;rKP0`rxg|52Iu7Z-t%&wogbeo9G4W=hVmni z>1jbk#++GguK^y{_%lQ2*GcQDzX44_OT*nk<-LavfF$RY3bJud54zUiyvA`&<-x0> zx$kKr#+RbRoel?e{{`klA@(b>ztIBm@~5}gs?T5RvOqwf!Arqu)7jDF-d5vw%KrHH zIFI9^ywK@JK6KAi%_x}FGt!byF+(cs@U&E;e@Pp%@3z6nSdEg4) z3!mi2b6QYzDFOtVmut_kggXTBT2g+h7^xz^4$6cwO}d+1w0_rJ8WpNnT1XvMEz28# z6AF11eDb6YB77nl`>s0h-HxEGl*3dGvll#+>5pum&ZNd}md5e_G4|%+P_}Q}_(X_O zB1DN~387?{rH~;@_MHkLS+ZswrNxqEtl3qT>>11070JHu22=KZ>|+@-?=|XqdcMc+ z_a4VPfAq)mFlO%iy07azKj&vL%=H3Z4S)h9xi5dfR|urXhg+$c?Hdu7<*oWZ^O4d^ zY`w*(B`V@86nx0+6l+J=oaZB;2de9H+YA+pR0|Smb+!vTqjrVytgYLH!5+)Qm(Bh5 zGH`foT^!1>PEFE%`6_ts!B#s$SLHrK>0LB3bfqE&$TRzZW5CJ0q01pVCh(u-l#S1# zn8z$4i=BVVztYcn$N`56by|p=tMGkcmswTAvZob$4}T`u6~G@hyiu;)d0}3xJ{?Zb z_em}6mt`p^Ex=T%P1bGC&EgELQ+Q-A3ODPRKk3U1-NK;J$74n9_thV_zrS3!+bGC! zr1TDar5A+Y)4lEow6~l`^0gr*9p~Uub~0LdR3@Ehk=a{>(Cuk)2NH1ytpFh3PR%-! z@oNR+#it!G?qkoID8Y3*$tV!nB}*8SoQu2QSglOlc-L(Q(wPK9FoMMjwP=1n`#E=K z4S2}iU{~4O>27bDKQypQjIeXNo|Mtc%|~(g{@`Qho^3tTWX8|JtsdQ!`uxsdp5YZx zhQ*u|D3^QmQMML?65Mcm(Oh84PEVmq#=Y=74~i6E=RW*8nmIVJWc%d_${n6>+=%op z+}+-|A;E5%x2-O%n_jTvQi<>R^nzDw^5ilS@ANS)-+n#clpB9)$#Zj4LA?cO=3mmi zja{WT+>hVg!7vgi2bV^V2r*7$|3(Li>t{1;kPgxc`ySFN32g3%UN3G58 zR_=z&nxzjZvsa7N{F=**#WxTYTJTQmDvJ*7iKLz8IkpmX+gFn4b_BjKu++2vZ> zI=a0f#@QPkQ>C{2&I^r`sXmE;Wy}NHk89aC;>1WUa~^nVNNVD={rH8Z+ef>$cSzQy zvDFH#Ff7Z9ukW>$vv+@tGe1}7KQuF>A2WB!`A~4#ItbP)Ncuql0(~?ppaa@LS{}ou zE8fn?R@{x(SbL~n$b|9mWB`|x`?Bzh3%`%QnD512kw>V4|0^ebjm7@#Cf+mqYg~OQ8sS7z&-^F>1IUHi@V)lNeef|s;br?QsOQpScn!#bo2|9nF zo2a%|7aSv&eGe4;ZjJ5VU%k4~DXUOWd1qtu!kYe%x2JHKI7*(-%u(Nhgpi zBYcP@Zw#vBQP1GYr1$cfNu=L!q?F%+Z(VX#USs9qgRBd3tE&u0Vm7M(OVe4Tjs#%>8P0|GL}*^nYbRjOE~z}%`LtvzP?mjvEV3HIZ=RV zw0`SS8k4jdl_)bOA+K2J${{z`%Qcb2KiofnUUh6C%n+pR5dssTzx-Q_UC%F|zc)sc2s8eyYU`;|uK? z@y8|2$w*}#hjP7o{K}ff-Cz^<(w$QSW^R+lhi+`H=Q_X1vtOvCBc$C_@bflT3n^J@ z3dSw}_&Q}^5k=grg>E*&<`kB^>=OHWNl;E>MK4hOb$SVl_n9lHDt}ggoy_=n$8tmZ z#eL&DDl^G-1XE%;3a&By%KlXS_E6x~lATfE9)3$}%~OO%7hoiO zmW#^93_9X6OBxs0p2b96!Mi-UBjaa})MNk_O4p@`U#9uf{89~JfVBfrw$gy|mvf)O-kk^(O5+`n7 zY;OG`-|iXkl-hJpj$SCZIhRV1?MkHi0Se`G38TXS-U-wnQ%GyJC=@N;Una>A_o<(I&PR`y6 z7wd|}oLp6U&aA00DRuBy>^e~~a2W#5WpAzF4AC-)eeM*!E$Xi25SIOdSj%8pJ`uNf z#*TZm5utou0qTq5*Kv2B*p=XJt=DX(RLpx@lCcVjdW zuQy3u6n^mWpsZC=*Jy9H(_GDg;UE?Ya0;^3b(N;;)*=};xx9b$8*Gg}b>2t`>Dl)d zB;T%3BWyy7Me`N+cW%g@&ND_%jLjvvbi(!WjRy9fdmV%emdoko89d~@8qdlYCToV{ z#?qn}NtTL`(9Say;HfOE!&g#56}JK+w?8Gc`)Q}!mvSodeKU8fTuTc=(zMQj>){V@v7V*clr^m<%nuIN#=-%E(c7uPc3;fTy z+z*gquwQ>{<93<4!1dPMbW>x1)_(@0p@c*)>?%u-h5ORzK#6}}&OM5}Cb4|6a9IJT zDQs1Sc9y#H{z6vUScMB<4%YS#RKjYUt3n7pZx!*^O?~+4s7$g3Qe^gYd{PDIK3p?J2y;Ae27r_5SB0)C zwE$piuh+eg=u!$f!hr#rpSe!~p=M`tuM#|5;v5Pm-PYAWNO(7+mUfp}FB@efF7@^G z<)+F6v$E}D>GtZnf(f6(70n2_yL$Ny60YK3I?xX1(3sS7hE-;`Gl!zCHA8qLS5ow# z3a+7xp<*M8o!f|xDkCFvB?$wY3-qbE8yQyWSWEEJvFy6!(j)t`f8V%@McgL;CjWW$ z4UZLe=YbgToKotTj@HUgZ^le|uY}5-6o){a0qH7JN$h`mHQ?E z#Fn!p_SFVe^^Y4-`ujrt!q}T#x3qz6l_da2g|=R;zMu)w)6DrC@h*%dL;kIWbn z5rqrPxk}IIC=#Bf8$DWLYmswq4|$HPG8IgfR<$2Wm-Q~HXvR1XE<&#e(lD)CtUKb= zr;|Mzu@e;RXnQIh`Q9=|`A*u3Ei?(XFz+1tfVDqQcHPRq&E;{b~K9 zAA~chaKHXDhXg~N5;wb)5;gra-66Nkl>uXyxt2H77gX027zdW7&3cRV1?g@Tb%}B1 zyB4pbs1dj0s4rM8WZi3b6Qz!xcb^ws;yD(T+4L+xhsJI|_75v1{XLk&wg|%Z?im(8 z4h0*F`kNh$U~fio^PoNZ1<@efmrv7kb1h=%bFIV*X)YC3Yimmrh8M?=ShsbAPZGW9 z4msG3gGuS*on1PJXr5RWz&0Y@&s%IS@;IaFLI|^o3Vyz@w1Gr?n0K1PFiD)H$Iejr0)93N}p%n2(K93cU$k@A3~L9&*FGI-AV#Jtq=(j5jkUY zmDILT0sS9ho>iKN-??kh2F?(*d4&KjBe%!$Zu1SNQ?61!cO7)AbcJ^&=l1~~)z449 zdIt~IJ{2pGvEpfSxq;?iKvyf;sU6@ zW7-6&t}RaiW3KiwedF?Ej|tBge6lzLpMAz3+^+O3(zn5Q$?~PbUwsjk1XOYh9?a~CehWHAwTUsd!ruxMy*6BH06o7 z0^iK=uc#ksmuacLoD%o-&#vqvl;D+n1Wvh4mEUHsFpeA@TJZd1FWKE5%5?Ltipo9C z8r|HV=AZWjLFa*~5v=u?pacB^XVn3v^Lis=n>l`iac?oV@D{1czI@G_IQ)7z?EPlL z8|srK0aC68K923DOFXN_j0)|IM|5>8>(!YbN$yQYaS8|<%qYUfoba^|r3;g~`L|>+ z0_XNN_>_6u2%HmRe?*LG# z4A3zq^9{gkvr(124errx9z<6(ft30g1M&UN-y*fB1QnJ*(dOqY`?N}btXV3ZOde%S`kvacNQ|IEXC68DV zN0ewc!eMCj>kJttcuRI*pZ?Rn^}92SA(*<%K^;1wGi*V+MqIUTGs%!Gu)Rk1x#SCb z`^gt?XR0NV3`8#9k1Y%Jo<9o;G8Voy{Q*d%G4P6NClM|KTm@}PXRwnZfTwU@rQQPe z!l7AIZ0zF#DWBblr>L@lYurgrFUQ;~pxb>E=P08pTP%`y?gp14FOjd@^pe7e^Z?f{U(&1GXqdW?`1`? z{{s$pm#T-A*bQ~+KqY3PWg-(%rjoAn4{DJ*dU^@>%k=0Bj-cVw*KC%?z@S7WRmPX( zwmrl~;T0U5`K_m~NXRSyA%>;B-k_Gz#_5OC8Y0c6N4UQe(0$*rBXiD^UlLSg78aO< ze1@+}02^Lcsb9w>?7u>60sIJ!*_Pj{$2MZSEiyyj6VL+ZD)7vk@(tCVo2pRdHt$rg z#ca5vhT-exRzXD>^Rwj_SHs3NX!cr~yO$iQ^%Mdm1~-k0gD1T=?a!JnEA2P(PYUw6 z9FL~O$rC&h?+Dr%w461_vFfVrZc7el+#9ih4-uHlaUIA>Zyb-yV6}s1O{46LK0C7< z9XOQyW7SB99P19tY;|+zM4Y7m3rNRv^sxB@&cN&=KfhX6ObVZ14K6aftbcU>^wxS4 zYa3X^mIVBv(8K!t-U3tOWBxxTQ~H5Ttgel&7BxZXf{ODWQi~i_A1fIkobwXCNV5Xk038n)|~UL&cKEGkFf6&SYE2>S! z^_4hY5VHKd%`(#}chczal7%k;4ih+Q{%usQ%_x6t7~F8r8byG5d7YjYWHk3Fss?cP z<48q59@{09U>#1&k8pr)TS20^fc^A2h|RTCU?Ch;4SW+{j*s%kuK!a$@~9Ff3E(n{ z4UiU8tx8IO21DUec@yx*(7?TJR+c?6rfD2ukA7$>qnRGF?c$G4_=>7BpAKl8EasY? zw#nE(6JHHFa!D{C@`I=jeS%=`1@#rmj< z!Td(qgVKn$lpm&o_@$=7Tk38xuW6cC99P=BY4iHib+eK#m6+Z19ltU&a3?c=yK;q%S0xh6OvPRRXVuy$rkjtO z1^w1fwzRi{%Y15o&k7jvR{NEm)dGgDSG=oxwGXEt(r#vqE4%-w3eLILQ*t+hS;q7s z7A;0+2ZlkJjmk!;+9a}?{t7P!VE@Q z*(Ofk$Y|-3zl_rgjFg@nKEDu1mBh!o6Q-x=B;(*`ovyZn?Ye82UOh z87`&tkkXX0k^J1lqypHMO`uj5e9u&&Ny=b@ePS+DV#8XVI+;FB+0TeCsBG$ooqvJcAJd#1cbvHc4zHY!4OvJad~D zvVttjP8z}hB;;%}r%tS4IOVyG{v9La=3x}=GloR;+gPGjudg_{xVUg~NqGc0?%!l@ zlc5`_$=X?284V^Nu^}ctnky)?T^BZ0Gb1kNwX_P}^V;418gWyNva5IDJ#pUpk4nr= zqU$thc}BVL%RB6Xc+K?cJ3f7W+#7|0?G_15eHi~ZBlJmbG}lKo#?Gfv zk&Rz(SzDzr)fRVDK#A~`YCmpk|H{iLr8Pix+8Z^}zyOnDugxnB*;y&PQ%&LAfD8qp zRhJ(I@Dd=JTG!6KubaT5D<-|4J^p!G%9~U^h(K2w>GmH!UJtaS4atk7S@&(T2%D?^YF;u8nYjy!ILvfmX zblj^)ue_=jTdT&$U_UNU+k|Jsvq&kf?q*;Q-B%F|OFIE0lPE0?=b@U+9G!XhOAPTh z1@E(Lq!4J(gj)~Inv7~u*J=-7e16oD+qZbO${OC1cG2-rn^xG8Wv-w|3xnb|sNjnK z&KL#{`j!1>n@xe8i@)X6#ZO}&IQ}W&No=9E%fm&Pk0Rf^6R2LhLpI$>H|}^*q91Tymt!qj0b1185F~03WhXl=I3J_53!#Qf!Uo zHwIoJU*BDk#Lw>@|E|(-7jTA!n)>MVO&-=_ZFHgNKG?ZB_274`+#49W_Pv3@!`Y5R zZZxoAaXKt8t!kCVj$LWG=<=>T{-zSX=y{Qe$d2`7;uar#MM@RYEIVZdH+F;@>`j>*ko~5Ff!B@ukr|K zhw1y6``F%@t(DmP4(k|TPYyp*$Ir~R-}ROhOzGBfj7h9EXRtj@lc*9-r&KWlm!Cb0 z6xLq*Lkr>4ZT7?cK*uF_IF^BoWW{?Y*zpJ7BmqMqpZc?W>U-hAY^IC;(QE=)C3!w0 zf7&&5Y)hhxiI>v95)0?gIL!zT?3MJR`@sc!>RC7Ns!>Q3h zux=*yx|+gF#nEjGML&0opU=WB(WA{D+qRnGJk(1HkY$D_xB0^`DN(1y{UH@?78QB@ z+l)z!2Gu$`fvUCKG$lX9$zP@U1uC)?NremDnuZ{fp zRGD*s7}?Y-M2t+BeY(xCWbn@(^7wm!U;4nY(}q#XO)MTsM9#6+ZD!nvXUM#T0UzY`imIcijU`$P6G_Z1ihuw0UD{xcBzZ zb;!LSC?PQ^YaO^-LZ2*?T;`bjmhk-&d)F-o66kHy%D^P=vgVg{6ascOhXu;)vKQzp zH!=uZk0_#dmMKeYT*#vrg*J89eRm4KirU0vftiQxxY0*ZoC!S6{LHIQqJG?Rc}Jwh zq%|r&5zAgia0v&yZ~W%RLLeRQ$U^}=Lw~@&HrHeo~gE!s6q_5-zh87~Yd^c~JR%5FK{{m8#o%h6I;g?XHUD_q|MZ&FbBY zx38eT1!y^vel|zbGppy!Vmu#Rht9A%XEx8b*%jt~6Qs7XwX1C9Rda(X)b8o|6xN;n z%#jhLFJ>yZ5x@#n9Yvn0^{Wl0`tXZB;bb9|vX>!W-C0#|^xyQj&};C6{|_n#KHO>}c{p(OZ z6@W+$CJnVhnQ!fb3T>w0jI8i6&Ym=Flqh>uE#pYJa#@+ZeHgk5Q;RwtL!5VkG?-I$ z@KI)dSq&ddwY9+h#Cv``DTP8og;6@T1x^uDk_;%6fDG_L^y$8%Qe4u-pwSx-NpxkJ z%3PPvpP(J64DTvBnp7CwUR;<{*oiwnC|MfbTq>Ijlcv7CGRA{&tScqNu#ZT*aO#DB z`WEY{H6D3BP!SQX)aevys@o#57cw24oNWIi5_sxs=bs5Y-bAehcTuG((NxcM^A-L* zEd&({WHRS}goJ?uSTxL09Sr%;zo+S8Yx77{7%gF7>PD-E`5d~vsOSPcgtntp8JJSE z{FzW(1_ZN{IyWWo&(~AO?)l{AHv!?Yv!D2VaQs{)7UAa@8SgRLhw7!CR0{lmm{XJ- zj?VIn4A~jXe;<3Wmi5i&>;yM|Yh7Tf=j^G-M(N~ zSk22n4%TwO?I`$_w=_{ zZ!m7q_rVC{O%&=)%EI37W)akJE@pIub^;%s;3YTRUmF=+A~FDz-8=#Wt;{+eAlsBog-isbGy!vmsrN&}joI1@Vh-=-Z-j&(RlozA{jnb;Kxt<**JWIFvKZ=| zO|SQQ8WSJSWW&!0{M0O`$=JJO`S3%Jg9(F)it78fe~MDywnZWLV1*ShtBlbn|-P&fv%!^It=1Olw^9`0ov$-h^1$Qzd ztn$GQd$=sG^HLX>D9}rAnon0h9F#GCToZddx*MmU_3Y)E)<#=P3$PZs^&=@2|8@ak z(RA822+W@D&(=tECBc^A6vqJ^O!>3iR_4)vQ*r#$OI+pzm_UjZu?8pBiEs<4VAqAG zqy83iFSFatwS+4D71&?w4jt zQ#HAT?ho=rYNv~KSBiATH$<^wNu-ttI;G|VQeST%NtUy2XMdF4p)WR}jhagEJI~f9 z{v1E-E2t)$$xPRh)yfqAi$U-0HC??yljfEkGwUVMvrXCN58=nllueEqERUB^6 za2LtIDC%jqdaDAHa*$qKJ@!hKm=(b0d^CS=m3AK)`lA#f?oSY?ioU5Eh+~?QMRFk= zj`Sg|3iz{giT6VQ8;Gkp5}>PB?!FVI;{av-hR?w|m~m{*MFZz>z2ZfnpNn)DLjF zSMzqnkl)M1V;eSbgzRj8`DTva9D{<($Bh58Bk|ZmKYL?WiL-a*V!oAAEY~&y=oA6Z zaYM-tVTX07SZkVVQ)D#Nabcx=6)7Ac?7Qz;aj;r7(;&LAW0a`G*QV!IF@8kiD^{iplL=j1hRX0w; zc$TfyW_UVZ%(}V!nMKD8k;C*O{DPyAp$Io~8WXIKAK}`f)zZ=#tHkOX5ur>9hKD99%(mxNU^E1?CD3L0yFd%;lRNZ2q;>?fV7q}(g5Tsp zZ$i4y9P5jfr6}&&($|sa@E+=4N)tEockfK^CpPM6ZCu4qvdVUP-RLT;cr+|AXcxs` zV~*)a|G~3X)Q}Py4T^p`fYq%BuS_spDRLx)&dA`Qx3HCVEujJHV9BOH; zf6rMT@~(7}^@x{Gyqa<*_P-5PR9`0my68gBr?-&x=?gxYyM>oH5e_xY#TtQxx~w2f zao5cEhaxdPy?r-$3+ju8_8;k(X`4Ph_keucM~(?`}pnRl4k}&%g3>n z|6|A@w60tcXJr-Uul2IwBp5%}WCfmkK9~!zyxCWzSHOTUn;JX@DxJn_}36 zV*?igR%`)OH>TN8mUgd;Lbdc$44>ZnMP(xqTY#_=xs1?s+_qZ3&DGm{6yqL9Fcf?} zF<2U+BYjYwK5k*JRP<@wP|tfE?VHt1FH667=xVEui*jU?(Bi3;a9nKH@7SI@eReN@ zTQXhApn~eN5-9n06dYsx@ZtrrD*AFrBGRU;)va({64rEdzOrKRMrrod2(eqp%hcualHXBxvIAsVgL z&eJ7#+B&Jm9CMqrAAQ+r z&Uv=2nMV2E=5TC|oJgjKU7jz}EcaMe@RkJUmvKoqw_T?W-*J?GdQ=|T8+aG?`bVnw zEM~i8m*7JewTzfQ%Q#O4RlMvMvtA`BzGhs;KD}?ojTKs{c9qE1@8Y6PvH4;-5SMG& z@Fo4i6Z@i`!OXod+K@j61IIi&4iOIQ}5F0 z{NWb^KM)Qqcr0W7`Rk|? zS4CirKoJJ~AX7fj4b=L2)5-KG8iy|09^~z04#Z6`Kw(>&{`w?i>r1ke?d$sitT;I?^(3e`rOjP1_(mAdh}%KM3qO@9Y!1|Ti!n*7 z#NBJ30bwc4z{yu_*HZ2Fhv?|o!6z{@{S}p3>Tv?e(vD~#0+blIA0!2xj7>zT5w(;e z!n19_$CqPhRFs1oe6vIwdF9JXT1Q^fL2_iAY@V7x+%y?LrDD@k_H=|Duz?*}Ui8EkU|s|U^R zo5K!ED+v4BOC85UE@k~jd%s^LDt=JUr%$t2Sy|D~;&tvnF&xaJe5E#+w|5u# z#kD4sqM^#l@#h%rZw@KlYv2H2Xm5?TeoJXA(6)AX@{S7&f?@TkP9Ty7)&ozJSM?l& z|JcHTNBva~^EcoO5 zCKt7ls{;d1oe{++B1_o7$9lR`4!+&Oc8{LcpRp!!8`cU|e?uI$5oxyob4{L}oo%&F z_2nXt#X;!o$XEevB_B4EAH{7X4b1L#Dw_Zmlvw6Q0^%fE$CSkXho#2kXjJto$l%qi4K!B!PLy5z#dKg_~f zeZVa~%JYKwXAbsH!Hcy(4lG#joSYfX^Y?!e#y|ut5cEj_5^qa03)DHxu_$d&nD~xt z3P7R$2+wyOBA2H4)-RTX-|(+n`0rT2Zk;K(DN~E*1d|G)3o}wIcwT+6`|?|0v&WXb zusgFY^{jkp=}#~?xEL<+F)^G#NCmhI>i)6n+%N4VZ=3>dSi+4UDJeM9tyNy-&Kv5EA!Wg|MEe}@ zN^y-e1p&a8AnX_nn9_{y-o3ka3k{-HgL;Rj;2?b0N?$|Ls{~Q$tQ8agePy)tRD->5>+F1tpmdu5PyP>4S(M;)%lS<-Xcahs z(TPahHlZJQ&BhGo-vqV?vBAJN6}~+?n_Vf46p$+<8kCD(MgIi}LG>KH zyaty_x6IAU1CNnVc1{j3X4DGRzo3n^|HrE9=;J}2S!#?u5zkIfe?>GRn|YNRJAG>Y zk5uvZPe2sDnJU&xD=$4IWd2GA>DaAD0%d=lxq(Sx+P@GDSkP4NY%LNJb{fNy4Jx*_ zB7H^R>cc@p2g(M~-_NxaAT0p)G?k9=DBe3~u z>Q3}Rk_t${>P%U7&or}FyYgp3LAtG$Y+s37*~$Wt!QJ3r3Ywd>C*(5 z8TJ6AYz0VIXUWvIWI`APMVDxVZAZl2X#_i;6#%!kzwZd%^rNN5;3&Ui690J8iwvhM z5QZ9=Eaf3uZgo1L5Y(f}AYhCVHNYs`gW8huGB?cv`}ND%5DSWR8i%Bizsa5EicrN9#fcnKZs134|7R%?0NiqLO{nmk4_vLd8lBZz>&4p8Vj0_N@`R!6H{&GvV-EpE@*-#emR z(5X#*iAQ>%jad$C>jI1AQ(-5wamR6_yg-^xqH>~%N%-=Em&A?DXmuxpItC^^}t!2J*x+9A&z;rHorD_~Z@-vYAGp-UHm_mQ@I(|=Giu!NOX zDM4rvP}9F4dar^LPY&@+Pt94M>K*ytqQU?1gXR%g>XcNe1F{8RC=P*Mo3kt^9#7nq zl*+TDcn>x@uFge&_w70ZntxC8?*qSQ&T{_=X@)QlmDx*?$$hqD3a@`nd{FIlHHeoV z`nYp2qbFE;^xt^4xApaRB%G%MbwB_T$gvG|MD4JE6l73pbq+e14cR`(zO2gw&f=## z@9FKJL7M-hY5((3veAQ@Mjp7ea55<(fJB`d4>E449>G1BmU0A0UCKiY^3hDrPq&eg6)RvNJ^QT6ZRt^gH*jAh z?hQ*P#opu*x*Ig!nBak-H2F}Gt#SE3cG0Kk7hF!=vhW`h@w}6NfC6S}ur-`Zm-@}D zeaYIjNxN@D&h@)Xx3H`>ElKthNm*uCaDMdw>hPGwdtXyw;_Cae()`O?o10JQ?h3JJqRgUM6Ar_ylv^ zQi%q|ls86Pz15srj0%{%U#r%-yu1cXDzm?ETskmd^nJIWA!EKI(%rWIzVg*cCJ9#K zRB!xCY{$~fG<_vWf)OH2sgu&&I#J96hzNkO5>rBG)9&BsTN-_$l|Q)IH+hoc;<)96 zdQ@bY)t&oVZ+Cv_T>B-@eww>6s;WNHAx_Uti#Y{>yurt}^W}9dIdi>p?_Bqsv&`E! zVdT_{g1{^O#DPI!-FjJjf=J=uwrQ2?S+<@d%PJ3|cTu+8$Oiq(QBaAeJEDDV6D>i~ zhmG!|fcpuhR>=6QnsH?BZ)-Ae?IP}jbLINUh^Hl9JzL!7Cn_A4huf^``G?}2!g~E9 z%B1_2Xm6*YGj-paC9LZ>p?Ml~$OO z)`U+CoxZ%Vf>n)YembyIf3mQ}rR&X^tDxSc79W@VWwZc{DQ!KJ$T&Fyz+h)=1vN%j z1;S>iL9TCAS{X4~xapGT{rjk!UDa2M=dRK8>y@u+-(GyCPx9`!O~}zr?`F4AWV4nM zrsf`Zo6En9t=Nh@`&juL^@));XIPm}ar02q#XsoXO*lY(`v^R2&v+)?COx2#S5{vg z`dMLp=^0tPWE)s(Q6~i}f0zuqJ9xngo17g$44@1RA|LXknA>$l+v7otFH?*C*EAUt z_y&S%t{!2IScm9nTI$a!5Q zxe3GHvAX~>vKGH;)Hx&&I{xl3g-O-^9YO%j08ZjQd zj4oQgPP0790VuOw9ds~ify%1@t5x${T3Y}9p9Abo5+nDBG91-1AwP{cdy{tW!8&BUbXK=|*{nfOlS!~lG2}o5 zm;vi_%{_s>r-Cv7c8~1E}HyQA)EtX-l$yEO44c4liE z(pYtBy=w^LVhQ8C3R{!}wOTeXT#fkE@TU0;Y}g!uKv*8`hYA66{>!K+kUH?(^C#1x z{G}x~JE<}UY9Okc^II;WAkj_F8+rh?u>6%%I^n(DQHLgbdo?+QsE75smC5_JDo;lQ z11KrS(K#*+RP^!UO`OuHv9S!dLK*&60o{*OLAhcZTJ}g$?KKDW+PO-jWLqbtT(E&% zq-L#fBmg_r%f~Y8ewYXV=Hw|#HVIb!2AMAD&A@k+Ppaq2D)=~Tq0NS`TQVE9;_r|S&arF9h^)!co z`@{+jJ@4WIhpd`VFGBT%v~&Q}tnkIqP9^{0mi;lU+u%bN(pBfMRU$uAF!Ftne z6ngVF{|xLCcV`DK0Zy3CuS>{}e>?+^-t6cfnphTSa48Af-<1je^BSSpLv~Gbz@wHR z8{fhaVfoTu?z%#cWWR#_YhD|V5WONGWm7ROGBtks0b6kO4yKHZ43^Kw=IV{aX)Zft zydFda=a(COK+$JUmQj^;7l>%yJxdKI@WyjQ$llzPN~P;`nziJLk5?7qZdn>>T4+y_ zm;+qME9eMJViyHO5~{N<7{%+q_s`XlV;C45RB>}F-Sv^7JWc}4NiRbnTcToNN=9xU ziwGBe2Vv!>>Z^_y-MQ)uBOuB)dn;aEs)n@?q02QYA8| z(!6Cz1;qFDO`r7ee9li(icDlj@E`a&((WRQrsI%q--fqm7}I|wB!nukQ^u8*iBi61 z?y&Ea@qYK!cV~!&##Lug5ZED^bI zFBh7w27!$K3deW%jg4K)&(H529DIx?yCjF%V77n@kdcuAAAX=Y zA->AgLme=_29trRrIw;gduWt=J3OEj&2E2X)G{1{}5Eng+;_Wc@` zTMGVob$J?Y^&}d|>c=Qa=8OF(U^qd17P}dNA~^AdH$-lBt4k~WhfGgm-V@r(oarhs z{nM5&F{gVUvKDXY1+)y>S9%Hb=Fv-^;65gY>5=tFoM23o?IRjyejVQuM23cj68E?* zGMQ9h-#Y#&9}Q1Xh>1$N&B?3vf5>|8Xt=wm4OE&WN+i4pf<*KlJ-Q$mHOlBE(c2(; zmk^>edKVep5N&i4g6NEHl<1x4oqObczwf(u-SbCQ)-24goU_aGJbQ0oS=z?e*aqY6 zM7W!IDToLeB5<)X6VmX4D^mZ-6tjZC=!XgRA2BGWFBpCS)xnv{{UohPm+vXDo`MZ- zd6XX5Tkza@`+ICrt494jWI#3ViWGwtS@KwD%}bRZo3~ZA9aaq{J(sSwQ5y)JZwS`d@XPr+AfEgs)GLqjfjhB8dAgYDX8Ko zgh6k54|$cIP@`?q5bxHCT=t6p2zHj#88*Z6ZEj4`3tX5{XI zuT2#Ag04h0wqLNMm(PoDMokzVMLcz$=(Qct-!pCX4$Us(dJ2Bs1uD>yJVX~3Pz=JO znz;~`TD)X|TlBTc1-1?SVmnh3d;+&B4iYCV8nF1y$E(1qs z^^ljhB;{yS^xLG3Ey}0mYnew+t(Iqvd6ftW#=!EoIgm>1j^)g>p$3*26rb2(=d_KX zoEuA!ZSJ;Hr(Sv%vbQvK?tsdQU{Wh>B3`Y~#gEt<<-K4H_ffH|b>qfp0iC#GLi$ zUU-%V#?~8@xEF?ETO-d77gWCL5SX#1h6$HA~jZe&vC z}eJQyjXmfvAk|wI;WZT&u3nB~qAQR{3d07GKTEq55 znZcj@?$OvpWo@#&hts=z-A|`-#C3 z7+ouC>qXb5v$SiPz8<7w?A2h(rre|yQO(G0P6;}v*6FgYSKn{v|E}}5>8yA|@gI8% zPU%{4-BRlv?+U9(Y`Y?;84*Lowu+@(25#e_%%0D0f9i1L=u?HVLrWK^u7GV7-_c=- zyk-83ja&|oQMt)#!59kMSB$jl^jp|ho`;C#T#RM*Q~VW$PuI*z zUH*1@>1>91*;Uxa`>jbC`TZYUMY1!|v0uwm*zibaZul zvTJK~n3*l439weYL5NqI+~4EWOnNJ{fjol7=%YF?9JNJ8wq+no`l0h*p8_nbi=j$2 z`mSsHpk-IdRkwAj+l{%1;4_=z1%1f(oRPnyo!9ZkERNeflq|J zKNGrb=Ol*Dp>1WLI33aJu?b9#_oM?X$=!I8PiH{RunPxkbrYn4bEpBRLlVU4f$QV# zKD$CU^$RObSVkHu1fnC7KlzBgT=UIZL5R9?c~wNe+*=UL{0N_q_Rbphz(dGO(rFrc zSJph_ln^}LdP*s1KxZ|-gvXv;aD^Sa|7ySOzl^dv5z6J^f-$Dl@nb>?AS6hTc!=S+ zG6{cv@(MXA77A@1tk_?O;CbUC&@Gisf51Kt6^?1Ta{2{fp%`{1tW6fM^wQ&mXc%y^ zZ|zjx?=30YJ+E#*_=4S`p5>@==<@!{=|a0}U`*_cZ*8)g{*(l;4O=q90RAupks%iu z5a@?iSxC`Ta>yENKIn)!q}3ccZ5m4(eK|3>K`ccQ}iH73z=Y;_gc*FI6}tm&##X{wgbX0m`MJ}xOYakH<(>P;>)$30p0 z6R$?C?2!M9H?-pz@6m%u@MEt|eRiQCc5FA7T96@8pP}+qXpuU*n#-J&JK;{etW&@a zuHUXaZOOSLO-p--xyp+VbpKxKLM*Yms?JNxo@D3JOgNMhj1aFBKHodb&K;9{lca6~ z;2mTY+lC5d{?Io6t=Ho)jb*+;yF_L&{i{5(PIOefj}KH{*t_ThTNgs<)Ui>->AztJ=lRKVP20cnCRVEb|DF-?+^h!~? zFHF;5hCU;!AriGSVvm=tG!)RrJ*h7xa}-Vz-3@PP;*zjOUn<-(bM3yUWz|P@dKC?S z^QoPELpFEM+)}p*IJtPeg1kIVVB_C+gwMap3f3PB8 zydjTh6#b{J#1;*|KaT2(h1!(q)Y*D?m3KFILvu?kbJ@CCPkpf83CI=FYO`(SC}%K- zZlyBtmSFc7B?Y2*_U~X}O?Kfhywt5{VoAK7vf`Jb{Y=;PEWY>>GuKninC=znR^O&u z#29N)BSS-)9=jA6Lx>b&nih<7izM$0sz)0*=2!K~|I}^L6mHUg%BDHV-WV$S%)RlZ zL+Hibh@V@o#A5@Ua)zb^M8f^-7Aj-d9r?~k0 zU}vD~OMb-bnoTpt#mx)ad_@kh>L_;fZXdN5>&5QPs*JdPjN{R?o^KFd4%`w~e`z&N_K97ul*LCCsk4`J@E03R=?BMxu37uxqo{$O(pNe?9Q5P`@eT4 z4g>ck=wc`-TWev>iGLBBkkHYaR@Y+`IYn93uBlT{fcKM`banINCwUVz;=UO%qNb`( zspd3Vx`l`Q{e(TU0s>5ntDb;&3&vBOl`1yHbuok|R1&a8Vnzi?@t<3|L5}hMoY9wQw-1#6cC}k(8paDLT#s2xUHT=D#8b%9ifLd7QcvIGBt{w7gl)e zwz>}RHc+XepALM?wl-aN91Xn~L$piK{I4ZB(hy!eX?R8Ll-?;jQ7g%SVz92Q3W z2__kUB?z|;&lz@29S%|l(TpowvN?9*#!~M^?Z$EE>IE`02AzqHwWgP^C7I#vtFL?o z@eqY(Gt}i)!T4Js(0;R}ZVKb=?1od-b5WfO%l3$=3&A|ZsO~MZGiC3H;Nlc%u`Z#d zUO0%?!59C2(EBHvx70f0H(lm}5BY?Q%<0iBpF;(Os|S_F?d%dVEb$y9)d>-Dy*R#y zR(`AG_BZGl8qC~9qhA>ZV>c7N%*0 z_phJzn?(C{wum)}kuXCJaUIfRwValP^NaADykK*;lR&m(qji-xB=k{i0>&ES2S-Q2 zq2@Z}X8J+QN+U0O8ZOoyH2jLwxWB9W)#A!Qg!z>R9(MqYg zcv)Tb2_^EXAl@P#E#aULx+}LY4g>XY+a6<4_Qw2~N6L-L9xcg1KBiDop+Cg@H#M07k;f&u9L&bUMlL0lpv!f;lza$^12Ds$|#+vTM9W44*N z{4H&waBEY$vz)kWD}$-$DST<^=lwMxka`Rtok)kL3K$~3ufMSmFJX+R=_e5 zxA7jFhMDF$>;&=2@1Mt@M|K-4o*Y1?b?N3r)M=-W?glfYo-FQHe4;(wRU@R_X`<_k zxxi6$8TLF(iVH|x>b?9p`C>iPVV%#`M`^@vl9izN7)91uDql(d=6490L*ERSaRFRLc)ps7(((HXn=v2%}(6huxHC{!{aJFW%83mT{cfBs?E_OCROCDM|_eW=`>Es zkUy=*uI6DJ*3QKF_gYujL5R(mUuzb5jYci?rV&?#W59H_kpxT*wt7jgmoRD{X-a{yotKE#; z?g*Z5g=-2yc4x?#roS@$9CV?{ZaG<-iHeaq?E1aCC*{FODe}`W)&AmC29;>rouk~q zHE$)E-!HDLIULD@0Ewr$=~7s}!`XqEfaPlEL0Vu8YucTHf3M!phLMdeI*Hr-SD+da zbB(3x6}h+`e2E$wx~*)tebl8Nx+Ld*VsV`cyj5JXl9hYO;1Z)z2*_>HjrrJ!w}MKh zI-8Mxs%Ym4FS+;rqda|Y2cIMES4}G+jEa`cu94$J*)I0nUNS+Z zo7-plGnKn!mU$ZH>mo(vrwPP;jr)?Xpl-7M;I?fpT_4JJwlWD%}5+ zY|>G1uw=`ZXgkGsiE24j1{W zAu;uB(yV;N8%B>JXR0YZtOlN<9~w{HGahKyQ5{b=if9u#bbB3=yCMlUgmWN$-6XQM z8)@7d{}D}XkqCY*%I+?b8Zq)4OvgK-^)|T0KhdZFCfc(Z3Nx%rWXsJLnk*@*#RP^P zQY;&3QcjM}KJMC1H>Q#qzCE+?klEvX3r_wMekeLFPjzr|+%Q(i%=<@u$>?%9cN39o zZ?E`&?mNnMqD+w#pMH3#9(L}-Y*IW*V!ya(%4Ip{#>93YUkun2InKvC44-a&@B_RE zE{rl#&^qZBh$^)tXw5VEh;DIbB1V~YoR|aDS>4m&hu@=@!wxy&qVa0$6-Ck5ky-AW zLQ842n_*kq6n2v@A~+a&G>{C>;Tg-s%A}f;M1r&f_K0>dvO+=CtO6+|oj0rVgLVnT zq07JaEj1y078Etau6@BG(>3eF?l$sXLY0Ct;!JOx5id-0CDj(c%DA{SHomwBfdKpB z>}HE^u04x593_92!9VRU@^DxKBl!2d0v@WOGG%zWCmd9nbOTMV-g}Ij)Tis@^c`rI zt5Y{Vs%xwx_%_?;i3*zY&5wN3yhJ;VTXJU3wD?a~PtOkzmK<|Qb|9W~h;y~K2t2pm zcn|=G@07pB1b;im_6$o{U8rV=kfTMLrAmLU{BTOtpplw;gw7l|HUF;f z4S-7l3=fM}tYs^0F1W^BGEF%a8q43Zr1(!xCP+JaTjjgm*!sTxRJN2|HIb_Khij^K zS_?5X$lZc8%{VoUrz>D;YevUlp>y)GcAFc12DCdmBJb-m~XgfEO0$ zG)#BjI-79rKhOYtKVhk6%G09TSS#{HQL427&p&k!t@>7>DY<4kfI^VvSnj%;Hj^Ln@2JHk+=M%XBwee+z^8z=-urN>?{!W2#OQv_hc2 zZWRi8P5%#}m}x#nTeHW%h+h*Mh`{26Hz%_)mUpcbY2n?4d?3UTTd%c zwx&9bE1Eghtua0%?Ou~|AxATDP>eszjJ1}Bwra{3;H5ryz8jf%gT_}bS|Qo~i$aoD z(s(Vq8Y!&>Ztjnwn)0T{q)Rvb8rST6&r&vKnr7-to@EpG9=H6fAo#nX{L%rr1Gavy zabhKcLUnEVM)p5aDPeT9W{FHsqVu$EF}Sp2U3IvCcdti;ct=e z4TU`^JfNFj^XV5D;=BkTY^s2&TH&;7sD|KCu%vsH>{lF6RQ+0WZsObK(5{j2u@W9p ziNy9AiqJ3lHL}U!JXRv~BV%i#;zWD=baK@Z9q(kV=_QYzRaTv~Xy39*;&RUd?+IZw zskK8<^+iQ(lJDA`YX9f}i3yH+q@6p|tT*vB7?I)q+S=NBa{jt=>Jrh!^qp4s!sM9Zo^7QoKCGge3;#j>Mq2_D-7>k7S>po3? z$-@GfjN>uW-Ld!x309Hxz=(>5DK>t^IAmXwNnmCw&+o;G%jg+Z>HFdtFTiBEa%2iI zr0k*UM_rMs3+ZtfYy8Px)3ZzX-;@5^pt7=3RbQVCu)a4xVEK;T###u#v_HQt?m5Mw zEEi7SGS&9ryB{N&z7(}U?XJBrCVD@^Cf)*(fPIV-wj6)M%rFuWkJ4|{@~pDAR-Z_w z*=bF%seq$92)th5gq(|{$NSi~R-uazLF42~mPWQ3r>-zd%2yEmflk?pWMt#4Q!9cZ zM?cQ#Ecv%?>6vwcuY8W4dlKSRi=q%wKoZt>&dOd3N! zwEBzEY?0t$_mIp3a%<^d@d6l0U+yl>%zUlG85oOQbsM(Rtxw|lG3+FpH5(NW8W>zv zt+MO!Em`&0;zPwbEHB^)k1*KjT)f%#O?1qo4(f$f6*;E4J^AUo^dapSA?gmw@}{R5 z87|q{A1u}n6XtZ{#w#@yYngN7Vpw&?Eq3aY%8WYvEjM#DqN!%iK{e2eVJ~6JkAcWW zx-KYe{sz*?{%6S9QE8*_xYsFK#%ko4(>=?Ebrihl23VvA>WOO zi?Yc&6AkpTOhR}pD6Z*T=t$^_`xh)0k!JYF_P2_Qy&4pV>dcK5GH4Su93Fxirs@y8 z?4a4PDSBG*VA~JP-D)sV)Rn9Hu0ONM2@1jX#DTaW+H zJnG;Oq*M)gy@xoF9L%=ZcZV;pHe^A`kF|Mxb80L;%d`kO-^JV#x`fjt@m#**_8T0( z{6Wg+aw3>-!|i%V+LO)w*EAZq){5wx8rcNx6%z2$;}L3XkCPr9|Mfm7`|@c=_wjps ziAv5AGC=(l?tEH z!Ig%eR~bIX`t{GV><_EoKeW8xhR2zmRh;aZU5g5^Ok(<$Lp2H*nVA6^`7hWHza9ey zv~+$HTC+OKZgDVKOYu9g>IT({yA(4qD{>hDq1aWLYMq%DZz^6Y>s)%~RWz#5pi_Og zx{DC8q>sql)1|}?*vN$U73N#k8`k8!dwTUtoV2PAP7(xf$5zm&D%u+c+2(>XfrSNs zju;Z_sowFZv1~K%Im5>fFJm~)(G~#%-x7>5!eu-6_u@hA7Jx2ofSe0%#A%^~^Pj+G zl8;I|rY2ez(c*!7%_G4Eh7+a4nWWukp*uXE=$^1Y?0X6zNJ8tHk+@%VqL!C^UkHoI zSf>VbJks4ncNCe(y-_=-w+_F`$^oc}as=tKGP?AxQXAv@=Uc!lMM_kP=m8{&2mB+4 z#i{a0bmK{WOnqB1m)Lq6K6$_ATv?ggSWG2zOA6wzcA6TPTPq(~vopZZ z!~EvZadU$=9dnswGkI*Xv9mF52q7OS)0xjMW4pddpffR7oYO(RkDW%3;A8!EIefvY zMr(`^$y*Q^pqc$9W1^Ct=>DvoH~a~E)KZVRtef2zCmDoOtQ3P66o>r+fhTAzFpJ<3 zu0F?oof*B1ynuwwuk{s)>7C3!Xupqrl>CJnpLnXltx zniQ7=ZwO2;+0$cCs2=l*j9*4&BmgFLn}Hpn5r6WZRMN<3wE%s4_$m4J`Qzw z(a`fY7!@3V8~&q7r5j~8&<1?r2uQK0SkdTk>U&{*1)HgnpY1x?xK=g@AU+@Bt>^BB zImJd&J!2v1i;-+4ZKR z?LT27e@C8{i4WeUcn%;*Y2@RPTs|W>bY6`>TQd3LXr1dK&@zoGH;10FY8rJ7s>#*@ znL|973RyU4dq~l)Uu~biF;*jw=Di5aFMD?~$6i%;!ct}KbPni|9mm!8WX5`hz1J`V z;BW>Iqk?1?cvM~h40mpGuATCjA1e$W{k3qM@6&c=r1#Y+vc%(*yU}zBPupp$F45%Ycv3WEau2ePH=XP+hPOE=Pc1?WMoj-71<~FFeNLQ#ad!LJ0t*mkG zO|zt)0YJg1*>ob>yeLCLv4=`dQ`xs|R4IwIYg>16(Y$`>c<()6bl}C)$6=bRh9*9m zLgfcZo4aXu8~jw3}Hwb8>A84k?G5}NZK=Ix_>ZaF(IonDLGqU4wo zoaU&t_DRSny{s>DUU!j8&}}>0I^T%5fhSz6k1xBQe~ux2w3E(*eVLP2vBefkBu34S zW(6_ zv`_gz{&+fzxbe#0VuiZxPH(v9d9P5=Je63*R)WJUt?eReb#%QOM~eK<$z_PNnUS2O z#+TxovJl;(i|(6cT_DVt>@ZM>R`7Gl|v5hesQt2Jvo)V>?fjI~LR1o7K; zcvVc7y+q%%GKz4p{4{Ayu$;1!16n1c+_xdU^!-jZRT&oQvM?rG+LmdOnla-PVZn_G z^en?b(UB;xi81l;!rxjsaef^+HCzcMuW)NkExO1jAUE1}>Mp}bCu?}V&a-%L>wAki zz2YD26TkJGyndy7#5}$4__%f^FvE7^eTqV}9HgYDj9t|K^fVCW-3HG3>U*J!bkp}< zQ!+Dk>Bh_LhDmSirB}@-hnK5XcV^7a$;29*>x8GJ8KooBSVeT6*rW-sdNg9g%uY zvpoWFTQ+{b8^kX^IFccUm=%zq!@2+yoj$ZI@7+JW%c5su`R$HBPndBD3F1eVUGLfJ zNl_hJ%eL^*b}F&mUomJh{2P|IRSqBa-j#@XjwJLLWBT6iEt%6&_M)tN9fz4^P6aiU z8iirho|%xsXX7)vS&n0kBBFl0?z@X1w~kWOSCl#=)z?s4SzZ5qlptH>v~#8<;)}G0 z=c8d8DQqGlVrz%n9^QH8eH?Lp0_qvsSom9G>T>h`N57B&+8y@aL1zS^zJ7YUu0 zQzv20GsSIPTWhOg>$Rt0lFt&ZCS$TozZHt@Z4x!8D*>D^yu8fNwVnAUOca{-82`f!>CO6A{!zSS#zmy(98I3_+ zJ^>wFhp%=%n8-{V%=0_uSHOMJ)~)SQ4PD4jDyb0N?L4V}Hs%DBdO6>eb-vco=-A57 zojWQILITJ1E(oG@xF$#5yPHsbme7EOmAS25zY?LO8@F zxmu8REi(gq%p^x4Y0}i1#TJ*_`&7N-Cs59G@2xAuIc(`yR?59DSr0iQajp$lVzd@Y zP!RLSs0r|k1Sv$8%tY?8AEtSp6@^8V0G2c!Z7yuu*-lGzO16HP2IF>km4J+f51=H# zCOW0;q*XBU!46$=u9@0hoH`4u!{KA)yJYRy=N>sU)UV`N-!k`-B=Bwf@4w)3ooc3V zNPXK*r|V9`%zc~#Pqa;69i_ZiT%IAYy_K+PUerEHVwN@_c9)wHcb{8px;2BSpwOYA zeBSkdQe%p88~j=w_jK;Ac_pU(Oxo4uf()@+uAAl1P#A1*Es?DH<3=bG!bMrQn>9|g zIE{?c3k}53zBfv8OYwa+7mjNG7Dkusx`xylRpFKyskLvEiXb4hw9U|qM2U_2%#0xx zWu<6*Es0-}#M^|%r_)K7UrO)FK<{Ch^l)A(NGx3|f=YZZtb-0f7oni+tgNFZ-S&t_ zONyVMcaj}3`lapU%Gx9OpiuOoT}Q9v zOu2}<>CEWK6F7ya*MDlq-#lsQfW)GVw-;aqvPg7)zX{CsaNPLz7SZuoR+Y>{>-(J5;b# zS;Bf}p|6b6xz(qCyuClU-QDBZ0a4;2%AU|eiSyWO(irHA168~ZGCdQDV%05%v!$LP zdIeY>Q3|-pqTh;X;!`^3bPH6et}AGG$E+CdpX3jZnQV8{A=?dSPu;imA3jMOY16&MAPuPSft{N_Nn01j3UF=$C8hsXh98nYpqT zOQpknZ`>AyKQ;FAj3IpF5Fa0($qu8T%jen3#2xEY(1UTrxc(e5e0lQz8l&s4G2&#r zmb4EInfzBNLJrx_U^pP?cmh%-fO?m6-Dk-Epe-TSejejbhgM-Gl+)t{BGMdn z&PkTpCYIQdWWITZl#wD0wUEa1iqur6G(1tE{>xWDm`T$tcdM^>$~V;W$-PuiMlvTx zxiAnSAvW_3;t@i8Go-=5!GoAi9s+qc9R4bzQ#q{xD);dl9F?#~u{|Z&d$`GwJPhY4 zGO5{x+yrL_OI#CF&Sep`{9B9i{0vcC4u_+igGXA@T+8vPY7)pA6c>tu`nj}Pwj>Z&rupO_AI;6?Y=KuptP|j z5OtO`z{pv8ulekNt+7DB4w4}cT;M>K!I9v>lxoxaZtPaCy)qxy$>!zZ>HOdy)bmG4 zdCV5~D(`#s9xa%$TT^=UF5Mc==Qx38vnV9zWsf>`Hk)*7aPandJ3Z=RciM=&sQ8p> zzyj_Ftfxf5&2l5g*1M$MEj_rt4B~=5`x(lQudV{Hy!CU8hri!vT&?!WQpr+m`>D$# zWTzYY1fM*e!#2SavFIGRpi%YNsa#Tb`iExr;wE_qHUG1DEWiP*K27EMV>7+0!0KmQ zvzoI$N2Q$R5q@sJ7-<|y?Xi*oLe}*AjG%dYs6(U}*DQ@(y9~Xa4OFnqX1a{6SOc?q z(Dzd$oo_QapgrZqeC^nQd#^xo!-2l?r)#~l-(}nKUMsW4Kg2tcFTl~6MeQ)YIiNkG zUH^9f99vS%38BbKtt{I+mX(h3!B92wjYr$kpU|r5 z9J5UoveUDC(VtCnklM{x<%(G!nco$gIPh+5#{B0xlz6L+C#z9o-qzb`in|T>C|dFd5*_V zH$x~f9X{#m$t2HwZM9=XP}#mS{8RYGwmu8jYAIXCD{!DdFRT9_Ir1bDuA%&DX6H}e z7U@$;N4pF<$2%*SqMc4F8AvEU!~Cff1+J9R11>7JGHRu=fpr6cZWh}FQpmb)n{@-4 z84VvN_-EjPmypj#rIUj;Ga$VZq*{or?~nF;@|(0sdQzCiD~v~3$}GDaOWOx?4VYr+ z^%$`xKiI2Y<$ic#o=wYuKM6+5@Dqx<9%Y^E4(o130o962eUC1xb0JdynUC>g|G|&n z2HfXg-(~<4y}n4N)8e5r_Cluv=G3670S8?aLb z9YU&isuxHf5bk>)e4)>G_cq*cSUfS#9@>s58xeBV=z3uM2GBnyBAJNUT*VaSyRiVq zB*fYh7r0N(nS5}-(E(Cy<=SxU_aoCy3XSFV+C_rVfzEuQ6QwdJY`+T1iMSY!?R{R} zD3icXF0Nh%k`mCBU$Ck;zGKKVMW_rh7^9S_?^C2ERy2zXbk#-g&2fyt8r>|`_h)e(11_n~i zGBc0J1k2i+HI(gq<>HZ({i}TBS_9IObc4D2bh3S?;kqV;4rxfNs42i`zV8al2dsqY zO{|8^-#`2VVz3!)k$mj-#R`e`(Fzy){{WCIPW-npHv%ynL#QFFY9?TDm{5!Nr>NSl zRQp56G#YtP|GM+seMXiXhQ^UQt%pjNZQS45{HE`zk-9Kq;{z>=O^T9izHgP=I!d@>FBON%oySexNbg0)g1a(uZxzWtawqa}ot6aR zCKF?=di~cf#Fx{VR>uv>d98e3&Vi!`W<%v18iF;|SL)U!CPmAYH+ums5PugSugvY? z#7;^jn4YRBBI~sHUlD3?p|BcDtC2#I8{(SNUVA+c*6%SK5Rliq??g65+YDk|Z^4wK zO6KU{U4>gp^Bgq-*_mo*4zCm22cEw9bI`EV_nbTt{ZL&Kmh6y5W>mJFu%XARlD%wU zTSy<)^b5}-yQP5KlrrI3Ds{b9*#8##_$!pL-j3bRTD1IS5%syFX z!>903W1Fwf=w1?whX}%+NBE!8mMmCBH}Cx%L|aicZAwXb`72XCzY`sU5BIUFPFFJz zgk5T+Vtbg?7GC`wmmz^EXo?^Di4_^6PR7X|fOfUrlH3^?*h$JWt|A zh-~L5QZsa9^Amg-wM&mjs`+5xd~w^9+jmT8<>fSfdBbNjNfWxHw!`qKK?y*X5_$r1 zl5!pm^bt)ekPGQ0!NP`g*v#}&xILlYwDVTYG6Yni0ci!8XqRS=9gp?XbA_QPOhYBh zm2;6P~|<-k^c#yAM1=HiA_3r`HadIwOTXjWUzQ{Hka)Wg+$6yd?rSR{4_?2m^^;si1S%UU$E3rlHJRzP|2Y8J0Us|doWNUKK}!1zUPJo; zwC7#*A?F(}JtHoErRHpJs7FjQ1>pLOh{8W6RhE0@*T#M6BABRPAwt`bC$egGpl4TY z-fg6mkkTAT@;ZR%#IC_b#&q=E!F2(}X zyIn6on=DYgm09ww%B8%GBeI>ylRYg;_38nm^h6lT5NoNGln$?)hugPE@)9y1<)51A zUPOg(w2>Y9m7II4xomMEqjm&8Puo^P0^I|9ixrdJX5V03JFNXunaHm`pvvMw`r ztt`?KW|-yk5FzpwLmp*z0Y0aHR%m4072`9V=1l^g8!q(!%XUe$XMTFeuxl&0$SLHa|7Z8G=SJS(z+|++mce z;Hj&42Och&zZ+uMt^X-{O+@2O2g4iX4^llSJAb;q!XsRa?sxafi zh_B11iO9L=7}1iq$?Apwyu1bJC{|;Je~L+sE80IkjsmNF#ZM`Cv22m{{m*inhl zQ~mdWG5h7inL7DED(EEwhDP{Caq5{=<;PgVD9vXZtWAMxuN67UtQlM!v?r9+Uc%Mr z+H0k6O%+n&W#TOs=iShRsBU@byj+Kmbt?_^I)!sKYiu6STfw@Dmt%`rV(qJa4?#5< z-~{x%8?SdQ22@H~+S>B|9inp^l+1$)`{0|hht9A>>VPv|Uf%mr3V$E^FZCSc6zhdh zW~@p@L%^G3ge*cEQTZlosdcY{gIuArDml}jZEIh=WOo4<&tprh<4)7 z-42$ML?`GSCNy~*U$E@uQ{{8B#vgMB83TpL)e6CAXvP~rNst9NpqkPBJwW*4GN#!= zeTWqVB|tOtK%J@Z!&$Y;M(XWQEZhd|R9t9V*NxhCJ^pK6%te<3pU6^|?aW$njrO*x zZ^1+iwk%0ch9&exS@UyfwM^B3Jv@D;n<6)9o5`eTFhoixzdg%#Koy@+19OX&u63#) z9Wm^&@%G8t!O(%n)(G#w5uEX{G0M68(+4L*kv5=zPC7c+L;um%{P0X9LL?P2YCKS% zRN9ByQctvMB6AT0BZFaEWr`_#jw6m*Wmev->N&*msWXp|Ax^{l7~`|GW$UySpfCG{ z)}VNYS#xb8Q}M9ra&#zZa%ruapKehO-{ZS{%v(~auQ_8YqIm&ZuVFZa-TE$`D?zP) zdQ+s2jNbrQ9pFk@L|4DWH2;^^nTMuI5d+PL{d02xrWj_(EzZoLBw0^gfq_nWa)VB5 zyL$<)YgbAlpFMUb{1wQJI*%&v#_#ThpXuOtxG9MR$W}_}8YIN?htPR#Xzgm+gg-7J z2`@zaL+1U#4HqYy-!e~fVN@<-R3#AA^d8O3CFboUzp(=zNAi!SLikw2lma%`enWvW zsGpw%c=EC~hl)b@TuzgPT$>1SA*%qVP|MZuNPR|*tl)MXA?+qHBGCsQC^aqJ(=J_s z)>w_QlxBvZ+~wmM-l;8CJJPD1UT&1E*Qb+vp{-T-P8+onncr}lHEeRf30;~ZBxN5w zLIQ?l#_3QL*{86|t9$Lemk+$QxO(k+KT;<~@(K@Yi_6^-1(W+?dtx<6Wk){MYw^{8ihLa9nu_$vNE`|vhOWGr^2&KtPNg<*PsX}H9 z4{|N6`9d2eqUhfI`8kK@W$`Ju*W!08mH3Oux1Qd6USY<9hHAR;)9{4OeCSUyvZpOb94fJt68RkT`k^OSAmxat?Mdjl|95%4;tKDwb6L^Lp?Pf>&-y|gOwcd~6hf3Bc}#l-U`XS>@K21EuZYv>9hH%&ZQ zs_JQ@zsbr=O@tBj>k7*XQzshPEXPeF;*3&fqt|}x?poO!7I`N?wb8#rp;ToXlLwGV zQL~ilDauGgJ6vLC5~F+q-j0!FLoNgJ46?&9-#_+xSe=4USDifPobvBV1+Vavjy8%M z0;=E|^8?J{MX^hxp(g$i`zJlOQj-P``Gp&?(A z&}eA>J5O}z;nA@*idm-iqCIy87KqyS%-A?J*`4Z@&cpxv0KSID;i$*&G#ze$BJEDXvSeZf2Rg zN|etf`emA)19KGJjuOVqdX5q!`_(T_bxToh5Z&h0cc`9}gHI1;u57%k{tc_Kv%u&; zM-ZAOoOZfN>0Ya%j0i?OxPDbURDt+oN9i0@LAn~DmfnSm&D+jJu|unukD~u{e0Gab zR}JylrMpryB%S%arMc6Q1Hwy*5wY5rBxCPU=Qyg(tuA`i9wj2mE)_nHM`;U;Bh z2Xq}jds5$i0lXp;n_QE+1v9dpH6QA^o3Ib-K%?h2n$n?=Q;;+`UVid;db)FMKmX4lD@55Fe+6{dDIYi1G126=DEli*AAX#e5Z{c~$(^P3_4jP2i@ z*}n(IYcdPDY;kQXXNYP*C#$r;=abCZLRG~dgWEV2O2y0%p=O`ry;m);l$%;s@;B+-nF*wZJN66SoO=T(m=^h zZoKLRbZiGv4~}qr25gdsU!jh9(@J;|yLJF z^qn%1VV-(Ny|Wdf7^fA_(z|6F+vdx~nscw91T$Cq?QXt*qcV-qsyV(1NMK<(5uR4l@x{jlRQ5vVV|vpG$c-Gw`a zbt2xw=`M2YFVrj8nAz-tvS*JN78?xtWLNU(Eq$`-)@Ko8>y~uc@p>xR*ADw3P z?-i8U72Z0N<*9$Gp{#m{^Kw+FkrjsFCAff~-xBm$Jj~?)#hk=$r52fH*ShbmucKBi zZEOzfIReGG&=a`@2btE)Rkf0ksnNdnZ86HQvRC}U7Jul;yu}XZuS04?`zpI_ote3> zn9lew_85CrZ@88EsdtLdI3$kYODx`5+xet8hcCtrheSSQ3Ai{UHu)mI1 zgLUl)A}zkRP<0P*hH=}x^DQ>(Ypizm(PiO4X3pDCSNY77oUDG;MQ~{G_yXA9mO5wH zJ#|~V#J_m-dmlWG!lZp`aJB?Ij4)ikYrP8PD}~yoEo-v*V?4-3E0^W@C%~t?s&(%b zaTtY)VL&lJTV=NVf;E3SOfC0I;iscw#j!c%(QLGtVjrfaY0KQMGKXz)W4pb>r=h#O z?EnN-{!V7A1qWqDa&wL)F1It!FAuj(T$?a|&%-_)`4e=es&W8m9qB!7(_&Wn4*2lL zd3g~4F0f7ob!LskN)T5)3VXI`8irlOBJy*^P)Mm1En(*@!Od(*x~uYX#BqqS?2pWS zoe!od*uo=YuQ=K%$mexn+7-=xARui+UQ24dT$WtlVXX6(N_+Z}h%G8K;-OOm%mUg z0SOTpX`~wz=~ARqrKL-xF+f47p`>d7DUt3$LApUYrD5nAa^^dOK99cdde?WmmVXWR z%pK>PefGZg@7k%eTfi+Q?G1y?ycyDD+ff#ARpB!z8Lvq4pkb$a>M>d&OZ73^S)*9p z*}}U}dOo{x$D%D{qTsucQb7i7@ruTAIfZSvt2t;#ILm-O4o!MX;IZpW{b&#!M|Hi= znU#NXDv}NQd~{H?8v|tIf~Xq-;g@ zY7;D13x+(J!QiZaNOh$++1Cy6>#kOGzP65O$EP^=G?YKmRO`{G%MqO%05<5+q7P4D zD#zJzU^qTjG~+qTDG4h42>3yj+ecG;IyX56% z!6>T5i2)>|&l(n0H8aTz_Q9MZ(C1|MQ7+z$eZ;m#KnU(fE??C2#bvB!uiFoHX%f1f z0j#}+4>RSJ5*?q!8LIOXx>U^h?ndw&%@VK=>xC% zTCn*7(g5wq07pJQp5ES&4oW{qyMlsvKChX3R6V`UgVw1}^6uLxbI3b}rwl|zaSxiKT|H<%*KQYa z*2S}}C-tY2M(t5$`;?1%V$Mc$MN~i}s6rJ9%265)(e$M)1&#*>gb;gJ$W6OXCBI$Q;SSPoA6*lLFOZK2L2K?`ltl{Ja z&gOP(YUh+Z4K34bckxBuC~~!3+kg&q54NI{_|9c5Mzr18^~}Iwnc(V2J*m4Eo8aq) z+R$t`Xrwy|1>PHkeh4c-paJR%CuuwpLEm}cnQc2^uBi-|>NY@A|E0IaSW{Rv=^%8^ zvAbYu^HZKtG3T{wiuYejUdF8YWb^H7kBZoRt7ih@4jS?5wL*}5z1vy2)}{01_K1WU z)uM%_Mjy(;)^{zkwP~vNbbELHra_pjrSy02tsb&Pd;E%4$Bx$d1#AHV)wbGq-dGq}NZtzd|a zHWd0CWOI>)RGZ|HX@!{f)Vne;!38nik8{+hCoE!^r!2RHrNx>qOS+%(eKhU;BFh|Q zdSty9yN=+>UwM>dPR}y77`A3&c9g^y_G4mN&#gO|RpD%iqTwL6P_v()^%Oe|P;(oW z9m^YOP@uZkC#61t##?Y!Wa9#JT$Mgf3V@ANyGvXnp^gfrv_%Yt(5&5r1MCm8Ey~=N zuByP@sBCADK7upwo?eTDHIGhdSs4+Q>DIJ$d#v;pxO}VqE3nD#l2__=7jG}iY$kzO z&)Ct`D=N)$TxI6UBff7zsy?bMJLHNI{QkaGkRhwuTx+rms998Mn5Mpj}8?0pOkgsj>kTZ?88GKE7qxuW!v6Push zITL&L?XlDm|F9CCNUe88HWNo;IH0p2jCatK^Cr`-+``lyBN8H!zB)YDfgGwO;wn*} zy>>rX0F1zQj%v$$X~VM&rk?Z;+HFk`rYasOR?w`od}n>pO8s}$0U|hDVx`o#$X8Ol zk3J@q$RhvXV{hGPp7ru3TMFDLpV0gth{x(y|6*och41FedB<)f*Dl2re?h;=l8+wf zf$*yWiaXF6Xpaqz4lC9*8lk&lw&^B+w8Htb;k@w3<^+bLm-*ddQ)cmtFP}ATH}WhH z;3_V8CcI?q&aM}$nvA#`o`j}+!1H3;Tm3QnuloxT$%+caVsVOI^oqqgb{fI3(U@bmvdD$Zf| z%?`6K;1N{-;LXaKD#^mZ8rfF-miYck2ww}@WDj>&vkQ`6n@{~8IGu5KdUW3D8-5ma z;ocsJIWd84L0R3pA0=s(YuCMEqK9d$@>GhWUw%FuDRg!-u&$`ERmG3$OU++cW#tkw z?m=k++B*@`zr*< zuRhxLJZ+O<9bRkEv^KJ#S9QYhH3ga`H-UN+jVT z;#WkTQs-3Wh6nlw0A&m|p{9AH^7HrY++_pwo8SpIEjc}h#l8v$m|d=BH2S1(V-D3V z#th){jPWZ-s|JuiKZ08G3{BED9|Fj}-BY^UyyL*psLWUY7ONnneP8+p^T$j=W4Mgj zlL1@I^igGPPDJX$fH8S_Wlb-zYYQC3X%NU$Yg z>{=79GH|N7qyN=aHkb486g_N>oxxvp;_+DZo!i+Dsaz{4UHa_4ec8BDyli6Sc!;uj z($;1OeFS`oLrUX40nQD)2vWw8pYD;S0t$w{aPw_s;iNgBWT#-aQQ)`;;pg`h@iqd} zAG$Y4_FBFT9GTxHqcp#&HPWYA#2*V3(CYthtZd)YzqR3Hba2>NIA&AYhu+cdQZ zOt;cRO~2J=6rS>h0icX7a+MrlB|$rj^}V5xGCLsRQWTI~_5iHD09!1?^6Vh^bjL)t zeRuiM{I}BfdDIx~dt#_AUkM-cH;;Vy?i?9%ew2~pCp%U<01WBm4-69XZO3=1Gh2(B zge_e&?z_|`0htK`+sW;Ns-9{E>rTfpHX%ygoPPf%Qx155N3i=Qx~%`X903H?93kK} zEpw7^SiQ*3WWJsY=lj*)8IQ`E;vUt>eD3j~(XBJ0ptI&-PO!`l*!wp%z#y0FY~Kr|oPwe=IU#I3tME30tt{ zF>Sd*DfZ&aSMSl$(YdAGbgXfK6z7uAya5j()@S0^9qD1RZOBy0XBQyzbHt$ zH?AY$XN1tEEqSQRt?+k{iH8~*fJ5fmdpwc)6OKjNEjWIzm?65S$mu-uozVTKKSf9P zQJ)gm8V_8<*w}vZ+4RMhYc$m^y=8V2*++8!67C>=k9Ljm1O)}-$wrO}24e3MaQ6T< zqBG0w+hM>(04K~7kX8lp*p5*fF^;oM3T{h$|9+<+e(cx|;p2%@?bF)qj0~Ac&kb=a8ymh2G9bbX2i#6{>~K%d zV>D&<%`QYexMc&Q`Jw|*2)}v9^__iSM#_gbFp-K$w7iXNczb-&shv_)kx3|X3g6Z? z+w%Ndn=hf-`}=lXi|T)DUw%VQgH&JOlECqHvPk3o~ER#>IJp?dNy>piY)}5*IN-f-5H`jW_6A6 z%|fPdPZkx3l%V_4bRb&Bb9B2v>m>pznY;xa`r{NV-zNrf5Bl=s8$7=s%jYW4h|d)j zI^!MS(!A3IAJRvskoGyYGR3s)uZ?vI*i+RXPoI|@&Yx=5y~)IOUAnq_yrsmckFB7F z0d$BJ2I)wtIfj@me=r1e%g7Bfh*Zdy#hoL{dzYl9;6H&_4DxlYL^E6!f&=Mu-kwt^ z-lOg_QJE7NThJ+g6B)+J_5l9!AKSgmCrl)tehdvMWqyK3>MsXGxKslpzZTwZF)|8O z291CuQ*idrtKIvDL^65@(nJua96vg-p1P&W11CjzOP)kTMA!%FAX={g2@|$O3Aijh zxW~^@q^+tbP^gsq5S|W=pP%*#M?0mco^?sGFSLr%ytgp{S3x(&VUD zSYp}VY9;!yfN0deTexN1?kEmAP3HWD#?E-6Sn{{zhn*Pbl#NZQ+3?>;(O50Dvf&I}LIBMw{g#>+EOQz^dgnP*l1BV1M;h!hGuP(pPR{0N{HU(<#-_LWo?&Dkx zD4s%=*gk5U_glZU8Y^PK%`!14B&EiXFn8%qkN`vknXtO|iIA~`>H@uSH6?{4E9E%}(cft(_Dl)=0P zy7EE2xOE(COHc5)cgw)^M5gM*Vj;i$@0@F26skLTJ!)A;rUt++&W=`-d@;=TbjCYnWfNH$> z&5;V3KgJbc{0rwWrb>?Oz8&homzuID*j>!S{Yd!tcSvy+vZ^x@v0{x%rmv0UCBgq& zcd@4z@5m8*QXn60Nu9zK=hL9!jMF1I(wwE-Q1XYt4Sp!C>=`Ey7W@E4gWYfZd$e55 z0SvnF79yq*Hi%v*@7@1?oRrr8g@-Rg@X((95_7;uon1B}Vf^!3i&(E;RJ69v<>Tj% z{UwdnZ%*&RRd^Ex;U)sfin1kY$97i{XC z|Nl0mn?^xbK?nyKPbnAaE)e@A#e8y7Puqi3Wo2h8Kuq6y ze<&)yo$()w81zF1tsYx@3Mu%H!CjFagqUstoF38D0oW>8ZD7)Q2>sCNEDSV14`xvR z2`>SLS*m2sE2p6<8mj$XNKX}KiBcriI=M$hOB2up${h&+>v!P4FbE`U=@~X^CZ8DY zZb}rxfVk>zBQ}VJe`BO6|Ldjln{&Ch+!Mcssy&>c5xldnfwUh|EctmlcO1|_I$8Wk zPjJeI%)9+&mmDU_8Us5RkCQ@*i>u?A|7bvz# z5-8w*zuva*2zd;jW>dOfBf;D8kMK$6eWS6PKZl+x&h5o8dZgWap4Wob&e*iZFhJCY z5)}+V!i9x2@ZAuSzqvqpEd1{)f-r2X22ONS^J~?RQ@q=#I_%qSsa>clo4_#5&@uOB zdJxvt7|xmAKK!N%9B5kJ@P3S3W(2>oGG#Wp^e-4ZeeN%rjGmW7@;O|b>`$MrKq@K3 zH$7Gni+O(T@xF~NR%3^e_xINYa+yoTQSF~E&uz)0ON-Arhyp=}NUF6zNP=s@u>v}W zTh=vw`k`Ap$+enR%u$hI*f?~~5B+NCFGMN?l7hKF@Vakwl=XMVx7)iKvJVre4@XkR zG$v28r=6!(D;yDO+p@k=+ZeQLYn{up@ka=j;_sg5b2#H&YaslAX) zs89YP4#a`XLcTQze*bLN*rim)R>{C%U8Zjsr#BtAgyHFi}W=0*18#jRyJ{7pW0?>6gQP!#6q%{ ze%b~tH{D{em_CsNhURL4n%`Mqtm)0r4=$pGoF+N1(ddvXF^u%Txlu|BXs;Uqn;Ss= zdv-?Ao7Pq%6&f9^@|WDDWrI*Nxr~^Gp?8ZXq3>Q)$I%8Uw%2t>w2T#$Z1^_aW=Ysd znLB(-+!F(C)=D0c|DG2!hx1v&O3OYL+w*;Po4|j%zi{9l3i|I$?0lqV>J<(pTI;sk zVZMiZGnB4L@bvk@4W){O%L}J9uNYWPjM*A_^9=kDO<-xNow=X(Mcs|r8w^0J37Yh? zx9sf1et8EmoQl1 zr1#EP(FtOsdGNIEt?$j!(&LQEN?o6Wg50Ob4dYWk_JjSz?Rq+6q0*+0?5wUDg3qj@ zHj{m!w!U5lU_#+!dH;sVX?l0xU0N=1G4C>w-@4KdMCUW4EH_xP%)78yZv8*k41vA7 z&p=A#KR5mIwbNY7Ln~`*rK&aS0tEB%q=(WzV$0md(4iaY1PqN8g~pkIzofv>sTs<6 z{;TMD(f)!zgBn9Ez{bK9^||hQ?fRtPR3x>9*Ify( zR;D#JkUFkjxOR0arD7)#;rn3J2jgsIB>X7OcbkK-?nrJK*35=r!~EPmmfSUec8nt* z&zmrp2XHi?V9?BX8Y|R$ z@5PdkPVR{ouj#MqK&hEXy%ZSeu5%S6+=dZLx)i>sorG^pc=noZCAbCN-$yrkjc>;f z@K}yBwzi`P{W^qm)pSsVXUqPi$B(Ll*(kJfg2JBUpOqXkY^hZ0>$xenU4TKoEm|xU zH-8|%Tz6xLRbCQZPS-AtL2%k7Ta?*F%)p+;dd)Y_C05}Te)(2tveD*t%}WXWbAc;9 zH3gb{5KMXhsJaf|W{vtxz=&%5EV%)rLn9Rr+t>FW^n^Fa&3)kQ3iGW2MN@b8$`3T6 zg9gTihPPCLRc5w4zAwJNe9>Jtnaf{ritTk>wXRq4(G<+h+`o%C&~3&7eu%W5OVz$t zv@eI5v=VBKTnw-8=HfYe#&w=m?Q7_|jR_7i<`04!Lv#fcp+js;OpWG^@SD;B z*U5z3ou!X+7{MIj9fAfJ|lGNeeZmhVPp??L&Demo4*Ur zTjSF6X8#D5FE_m8F$JKf=V(-tB?kDxj4jR;u@!`+>i#;uvW(_*!ZzAtHcod?9h+fQ zn`x|-p1p2nDR;wtkf}vZtWIrtRba80v z*=FDTaRnI2{s8G&T1?W$gteAa&V(^^F<7WMuL7<#wr?EtqJ(^WxfN;{gXvv{Y4hEO zz`Nr*1|;c7^R@ZvmPIpFYGvu2rl+5blp@V=dg^rZC-by222Wl7Sdf}({&L5Oe;EY! zCRlTvU2?fgbioss%l$ip6ik|vOe7>06P)K>o*(+I$ya1;X&J_$@545h>}j;JKNL#_ zm~CsT#cXpC$w%-)uX`f>Ae_)o_K{8Unh`X&7E-!<_oac?mJAP)+Bl}TON8wwVsCG6 zyBc%Ce!J!Cyz$T?Tz0q>gnmtb^A^-*&`~|7QCH5dmS{;g?OUB zzI?r1yD@p|?IqeEeT+Ysk5LGXI90jb_jln7!ugn1wfSJd{Wv5Tl+2nGS_*g3?tOY* z0r!nOA2Pf}S;fX2kg?FRx@X)vx5NfYlq0X@_m2n9aji{&fMhZ6D5h(8{$HF6r07$2 zHp}xVqe}cGVu_<(wi((D?^%^fWS7kLdZAmy`cAvCP2{n*WY5inxf16^rJ{q+)fuU& zckE7yeZB5Ra~=Df*2*oVc0x)El=>i~4fl8|%;gwaK z1WUzc`&5(|h8yQ72<+T+pD?ikJoXXqUsHX8(;<0Gdq#}Uee`aYWb@#_AOK9XN4L*Z zZ@_rFgl^d#Pl;_&j~ui#4bN_GJ01S`p{8gyk3ZA8^YKXhNwq}ml0c)*r~Bv zJ4wH!g~Z@BYwIDjJQX!SUs)vF`EG-c3J>-Ao@e7S*K*Z;MKFFI;KI@my_}tk#SXrP zny!Ve3({c@yL|=NZfA?l`jT=qUMJ#*U%-;U>vKZp@*p3r8YTbR zPWt8ji&&Jw-DLtPC%f7;y7@`Z=q61*5AlPl6R(EudAh}u^g|kLkjnBoMXD6qN^HK| zG3}jji~YvX?%q^INEb^_Ofk+9oW%;Y8eTf;Hnczta`_&5SgKKmoXEx>K=f+WjYmqY`R)$0tEEY+jlHl(;D4kn@exFmb@sgS1>A&Tj;J! zHxXj_DYFHdMgcgrs9s@TMwku&g9x;)x6;ZxNvNLxubKq{NKH*M)pN-24xo8HrGOH& zI(wl!-P%7rOrIH)-09euPo*n(FOHt)q7BC;l~GDcr$q_Y!jCKZT9~^x>qQ#c$WQd9R$*nb0&g3DoSk#XcNbr z80uDRa=K$`Q6@-VR5FH1+xrA-B7M0H42wHu=q5_RxMN1=-bRY1b?f-DvO%qps=e6Z zbKQ*;#_V*6xdsK4WNwWCFo&71MNRV3m6w&o9j|-O1`i9EnVC5rmz0*a_CCs=j|_-m z2!H2}2?c}gbB$Bchh%OeKXeY)2f6(~V`QBg*zq&!6#$ej(Zv^xfN)gvH8Q(%E~U>7gJ6b-wf#~Ce>7S(t=z1vN!n_L-DU$ z@;&g}oW6@QZ8u>nr}$qX0YV?RgSQCCB~zs3>%)a6PG9eY$HgqJ?FwObBr#;wl25;1 zVBzG9j)1BS?{AzKXIc+rL`N?So8%9=9Q6FGIPIpSrhCu_(F7Q~)vd$Xt{4z#Q5QA? zLRIAXb>9b&UG~$BN4d9j$*T7WjgUSvE7?2Eo!n8Uo%*UlOB`ut3;*vU$-acgb?V({m%9?8>INv`+! zaY0H!^Zkz`Z6^!z@;=yVYimO#by1IaKRq7m>7lPX@U4q2VG-Inl_V(zuPN$|G-ls^++Be^rTXG>6{8oDd^+Criau9*Y z=V0I4BWX&$vqn~DW|rPTWG&^pcoAyo*hp?ZTqatgX#n!I8z2B9^jj}(VzZ!UiO{nD z&*>55q)xXfsLimOr(==I!&9(11Qw>MvLQEEd{1VHw^uikgKb4Ub}cIpX5NFT>hx`e z$q>bF0Fl(?0vogXMZIA|$} zM!glW$s!Lz?5$yL9UdOijAms=KvmmJ>LyJ&RjG6ij`9jx=e3oC@~H)EKAp_wWAuGC zxiD13+!}T2ww%u`k_~UZy_EQQnD5!{?VbIwwHvZg!7H0_`$US;Z%N2NKHAf3UXw0@ zM%0K;zdajsm>e@7ZfqCH<$aj>LGL+4{)TFiq=j)@oa)E-3<+;u!odS7I~tK=MLApp%dl@n@_L{%sk$r?^iFI%jdBJ+mb*VaG;In0!rOLoe^<@yX7IyE33 ze>`tR!DDQA_#a}Z)yI1c3VQPwmo|!MRQqND64V&<*Q%JX+Xnkb6g>z*iL9!Nq@1PZ3rOT_1 zkB;e-eCng5YZGo$!E2HjOow0{YBXeRt$a#=P55Bjo5#Z?mQpcpKQ?>ez6}ppQ53WD zs-)1fThPuGh2~H?icrDs7i(9y!$XzBE*o#(G?_S_Is5_bT;r-sg+)o#<3c2WN}P-Ap|HGe)ubZWwrE!|dGU6ljRZ?$W23oHR!R!J-B`L@ zNZgdYb=_5qBoX_<|LVNP*Hhp)PhNqiU{0# zgC~ZrMAZGD^URB)u6k5^7wWNl1L9(AA1v3(%B=fdOVbnMO#+qau)P3%lG3+tJz}#( zkYXW>CohsIjHk5R;sw^{Uxy!#JlkzuK2vVX9(o+^FTwy8iAwwl|Jm@CRJGnO;|U`P zlNIP8R;1GEYkLhD@1z>4>Yr5kW7umNRm%~jDur{}3I4JHkus*7=t!%-N*jNwP1)cy zi6yROF8vNLA~2Qe15^2@f%g=_;x2&JzgZ8d@b~`kXg^4?wO%PP^{>hS1P(0aohD@& z9u_x_FZG%M_esWtwaE}#aaPE91ErKxh}$wj8f#9-k>Hux zq15m5!kdlm)+-G8H#{NqzX$HeIp7@9F*&MXMv7qiFBAlywfZtzr5jNSF0Lv#AYuiH zB*bn?_-OjYXw__1mH5_O{r}8;peiX3wT;k(_3Y^>EQ7%x1-xl-KpSw;J%!jZ&i{Gh zqE`xC(SrM5VW)_INNfND3eYlLTO5Y}t%d)$>v35UZRlzg<1Rux4{qxaS@yhGZHZRW zXFLx+hdRawg+J%AJEct`i>DgDUn&;`O#R3&C)Ncn^*xF_2zS`uB9ALvpsh(#aOrF~ zgK9$$HCC)Ji0Gmy$`?SyG6fe`=qa-}-}1}wnkIM&(V`@Q4@hIjAL zlp{ZWFaj6ab5RiL(~bJHv+l%uU=;1Ml2XYQT(XWq?8CbEt_yV5AR)MQDO@ULELbSJ|3TKkXT0l6UJ}Nu z2(9aQfV=PEA*MKZ)C4R8Yzid{G$Sw3h?r{W;X=OJu4?X8>pw~UB|7uF6c-;qtN0TS zqMnW+GJ@7)B@x$K{&!0lc47i$u+X(~tv$G@i*Y`j4}`s29<{h`SdTayzn4uTIecWl zZy?PTRb0OgtLpMp>9>8EczAmM+U?ffp%reB_sHd#hy;-K$#ssa6S6TN^oj?hv8ZJ5 ziZdM_3?@3jur21`pbybfEAojYg2*SM>s2wp3=8u0>(k`#u9I%ycsypsmR=_SP3%MHE@_Q?sn#N-duL2P^K&en4u>KQJUYY_SGL^pL&`TDsYFxM%B zi`xtZXvudXut`Mr{ZR@*Qi#7z#}~P`^AsOZ-e?(D4f7jCd zCLIsuB&qc;-4J5@Dzz9548Xq+Kw!KjaAcnCr=&M7ODH`G$P&i{P41sdy6k62KKc3EAp+(?%3&Ryvc|^Thw;y&&MIVj^{6_-yw7m^<-i?Hu2QKns0x# zO&GxN{J9-rni9!e&+9<7$`2u>pXPrrct$omL0?)L)nyo(9E?iNpM{J}11DLDZ`R36 zOkhMgfPOrv?lySg&JI6zFd(q_+p>OO(vDjZ9$+aTaRe6w0PNZ0D#0>|D?(~mAc=Ck zM=<=>kDlAa*Vu!_lem9AjTJ{K(UgsWtlUV=VeIhi;SS>FY*S*qOSNIOaY+8o@p~*s zR4^38{kNSbE^A9Lf{6z}c+JPrC{A5+)WLe=&eLO^5{~g()X9+?brV#?gJvv zE}V?el$Cc>u6-Up1&|+J`#li}K%OkL76zn3GK6o6AlC6kAUi&90*oeLKdl|B^;g;Y zcQ#;LQCboJ9aVIejH)eGNdctjoR8%4>tV_Es>2=Fj zRMpMWWeNRW0J@7JU*TBdpfHxUtjVFkMAQdbII-%9-^axZdbO7GId5kMJ3l%mcA$@9 zOupS9xnMQMYZya(baXAhez$lp*JLqsvCh3+3uKY)&0TM052G?S6;Cym=O zG&v7&-dU)u%Q9RA%QsgMZrSKU@PD!!2!s$2!^`@lkt678Sw{EuAdTtmiWvdI2x*YwiO+`@()&^i+iKm86UaY~N-WunD*(P!L&4VX?=>aRO(ef0kCEAh(v0mztzx!i752#vi^5bY`ICukzsxfc|mi2p;-KtCiKBitade(AtcRL~G;A{RQhqC)sSxop7B$ic8n zb#;vETYyU zOVOD7$w{L*e?~K%KJ?{R_rg6Y14ThoevjRjifEha=e0IkRx8g^gQ<^Lu_Xt+xnZa$ z;q7mBpcTjb>;1Y0D%#~dsu1zVX~$U8YiNVYkcAuX0QD9H`x%#NcxlTu2;Wy>q19sBvz&sIqKbpi}s%HYn1O#Jr=TOQ?H)vx@75^`Hdz&&D7tS&uh*LUAbuVf%x`> z%dBgWS^HoG1a?|MwXX~Io=y#m7EEG-7XFMRgd@4!x_oo+=E8o5@Ukz(bzH{?586S` z!-X3AkLA?19P6;!!CRqX3Q~mL(PK^NU91mHs}=E&=q!4^16yN5tV8#gx7OmcYUNFA ze-@IWoV2uipMgs3a1JVIHF!sebFX#!3Hl2 z2X5LtU}E)2zFc9yO0ct{D{mB@3MIM8Br1p$QU#uaLBz@A+bY= zGkvK!ga(M8QP?6fq=H%!C1quHGyfA@Gn6lq7d*y@fY*^)!bAwf(74f9y)MT%=&m=_ zydTGnPJ`b~=^73VZ>wYyk(-ckKvyOr5T{fwi(Ypeww%?Tt@amT;ryhR-gY(~*>9~D zH(vSHBue$J&o(2q*pz^6n`^lB=aHE2uyadw;z4rs#9EmBv_MCI)aG1Wz zxB58kl7pnrI*#A!^kUMFVK%j;v?q+%P!>qrB1!!tToM(bDZj)=zhC|(e4tA-l4ghy z0ITk9iK&TZrtRckF&j{O_iPe?NLr7h6=s>I=yviy#G2wAy51mUFIN&UTK4#>NCxq1 z*NI}z8x*>Be2(jk1pmZdVddg2;W5xd^JifIUrD+MR*x@O@zN8zTpp7Cg*YE(X@Yb&hiJ&R=Q;)K1a7WNu(qwOug^9xFfg-o_(9Sd@0Xj-t^g3V zV%KSJO=gqpzhmZ}4O?HdOr=|E+a)C6b*K8!Jn*obxCMgl!3tLZ%H^9xiGcI{bCDpU zv~^Q86X_|i{g(RGy$FpFaa5mi|jz8R(LGJauMuWxM^`9Vd7f!Ed4}&zxQ1&3>Hg_GyrgU@jlf`QO-$6 z(w>>gA#B%;$~+#xejSnPMqxpL|G za8^xCM-A9P-MLn`^`fNm^AF>vlS8gcb-SW~5*4OxOcFOmd5Zps{BX?L>n2;r=Hsid zDmsfgHtYDv<7_yOkAvWSr=YQ=HuUexv&(Y+k~!_lN-o$AutR#1|F;vf@K}v-@4oHh z!dLR5&(Si^p6!Dlb{E?_L@D1UK$5JXNdPa}_KM}RdKH( zB%uYyv7N3X=H3>3OPN#TJJr07uO3ggmOO=#{eECYj37!Ml=za@aD<&OfANhqoibIB zIxH40BeyBv-6IL+=~*R20Qq)?O zL7rznZc_?BgZ~o~c^xH3g;K+GNezk8Nc=!wenLf!t*bn|L>YRSPQ+BqF0$4W6<3g6 zkl%hp5*j!5%#4X7MLCv7ms#}&m&u1_2Rf1>TaS}+b9$_wHT1M=$!)%7rz_aoBG2{g zUPF&gzaV%QPz%#Hcb;!hXVvin*!%bW{xF!21+z-co3Js+^o^L+S?wy#$ls$bj0qU@ z#B!s782~l#&`=y3u|ihBNI7Z^*(bu=9Pg#>e|_L)#Fn@t{(|0(JW}ywt?ttp*|@c` zL5)>{bK%zEhe!ZHSCfmZQL6)+PuO{@F`JZA}^6 zaQ17vlaA{sYrzo1Ad4gTl$3kK68B-U+^yU@gmKN+FDnbsehS?BY6&Cn{`fAC`SYh- zQtr6>*NbO}KC#i>zL3t_=GT#OtL+B8^XBe`f@ZGO7eSH)%gwn?!NJ(Awl>yBwF+`r!x(6whrx|@-M-aSKQ`1-j1iQZ!dO<_n%7~FQMw4kD0P;+9BSai#9eg zYoc>^iDj?aNOd?^ipyP$RaDiIk;xD{Y2n#<(FK|nwQ62y2K9Lxl~@)Mb`9GetQBw` z|9CXNc5F~r@T%{TT0vN;^^kv7XDpXJ3?))}XTKYu@&lv`CQ9M|d}}gWd(8IxTrvo}bSlrc!RV+{*Ph?I$eOIFW@+ z1eaO-s_b^!H)uwvRqUpMSyY~Gsor31At%ENr!?Po)A$cs!dvc+1gY#K9LtqO<^xpDxj5y}=%vjPLN-IMUIXF1$cS^x%>ghN36$k` zhUJ`m2_&fRaqFY8w$4*ukLi8YyO9A$O{z%FD*Hl0sIb+Q#=gzIeBzC@U7bDJv2Ewm zVB&_Cx9Uz^34A(EvT9?>HDCUiYYo?YiP)J?s(+dPmQI&dDXD_%*>z|%u@B#rGh)4A zWYOU1E1j-qiay_H3-&sltcPN`hY^RZ+N#?gG>N-=eJ1J}d_2#-TIzT1_c+!J^D<@} z=TBUXjv|K|oCS(fE@cmgC{@!u)Q@@}_u*X@WA(jAQlT)ZSJZKedb!O z%uY@7xOdIir!n~Iah_&&$vusl-RsX<+H@6IC=2Tqq7goYA#c4^(j+F+8l6zT_PKuR zrqnLB_(TIyivL_lI;MN^;|;G3w?czF6*r;!Z5AqOv0m@N&(pLXo2l=5T3`DjBEBjJ z?hr9|ikOvkTq3zk$RsnH_XU2JiKN_0XL(dg3r>aZ^$Cs;=@naivS_R900li;-d{OGTSS zRCb5`r{!|M>KJJAe{JqH~=jw1&h``?W$rl=0+D+6SYm^k&xFeb=ocA3)f}E2(XZ=IQ#(Q?0*PYp`e6(>DJwD75LYE6G zj=SRFqL$Y?+>@}2@{&ls{IYfXpsiCQxnO`abM`f5CpPvQEdJ0g?x-j}m#424jnQpZ zPFw!Y^J(uFh^l1SY$qHS>E_VqY47xdPjB<2&nEQx_09AMrwC@nc#QlkJm`Ts zIXh}G~ikaj0BD;2XL$I6f8I)xlF6Wq}5R%D<~yvVnhC%hosRkx~0CM1kpB_?EHdZGk& zSSFudmgV|9kBYpZ$FMBlU`rPA(ejn;=-$|6u8LPg1}rV^hE8uINJ&`Fve8n>*F{`D zt&Ugnik&WJCAWp$ls*T!qJYQREsjbf%+{!RG*77&XnwS7h0t`}HvF07q5W6^Bzl69 zgMxD2S@qM5mTjxHSC;2?f73dz*t9!ay%F;XA3uEHxFjtMyiUzuYrCn|dxy-ytTbO4 zWi7^bMVN_t-;vIwo$KPY%M)%E6>THIwAjl{nq8mi3>j-OuJTplKuK;|3MIhF9r(UR ztd}6m@yo{O_f5b@zx_+;g&p_4>8!tbP2D z@d0lM2Tj}}!*3Zymd)*bLvsPt6mfulyyJOV>M0DlP_(l0Ot;Rbc5ZG?-pVTb1Fbk{ zmtV5ITSx2ex+g2o!f)9#*Mz_3i-(GFfMvXl7+Xh7XsGLL&Ww|ct%@ajZ`mEY2bev7xSxyB4M-&ZN-GIUnYiRr!=tNnSl zv?NCNp9&qqTl`?v$O=hu2YmzbcQd+~1vcBuIi1fpGQV*Y*zNDrBwr-kUg7k430+$+ zV4Il3ky7gqc&tPn0QxhC@7B;VjedMT?f$g2|7VJVF9;ac;*_h``xF(yi}WsGCy`fq z$Z&K2#Zt|~#YA7DQ8(nZ&V=Gc`NM7U?HMKOPVV{P)>qR`=byQbUTXbC!9RKvJ^1`> zWXY-kfHPPd8&X{LiFIqObXp%57T1fCW5cytk_867)n3JXko2FQsyFl=?>BVPJa}el z$%p(-^(rqdEvLVxs0g-s(6%V5t$MUiS=_l5h^m}Fu1PYQsEmrf=E-@y+0c$HvPeB! zUjOXWBN$!qqHDZM7}?=N!}pLZ^rIy~{=4EduZ*u_+vIF2pHxBzepo%qknBfwtUfZ2 z-#SnGLL)2=MlyERvn7@6cs}ykvuEdk1l7$I-YQ4aYZZHRM6j&-9{pw5Wi7MhxT1Q) zfZke5+$w-6l#6Gg%_@qC3(L5;pvIwBqC@KX_@U{iEW}NZa!0BeU6-k+z=4;cTTwQ! z&(>;yabj>!kaKt3eEG@oy?ceMI{&Y&uYhZ^3;P~|C@7&mh=PQ4iqajTG^j|32pDvC zGY}L+rArzqB_}x;rJ~ZKgvnqu3C|Jv=oe@=qNucuH;qM#0^A+8-xyF$c5cg9DO2G-Ksr8w~>gBJ*=l8!;NcY z(W9uYC-A7=O3THi%pP6ID{EqCXhrVjOaa;DFB!aVP%U&NM=84l_J|du9wOM*m({gC z^&yo1`cThX^7MLU)euo`t*Curq>1)i!3TqPxW0YF^F5^3)=TFMU*DJeJI{v^dI{=V z7+e#J)do)q!wfZv+n=`Lw*G0Sp`@Bh5~uH(EpE7E8?TQhYHmGjno$ zqj{RApniYT5##B3h2Gt=`?6p0uFMkiJsW*qvedB+-XX{1*h*ISFUL{QcypV_Ht0$l zkS*?YH}qCj=4A-&{royKPL!-418f{#x>Phjk4=p=udnY?kYk# z5_2-++TKVz{migcXKY6H6J9Gr&)xcwFKh|cGi(7mro%3dnnmjKtGxj|s8IvNYZYaO z&23UeGX^=|%F945YAV$+@93+seT+|GU}W42pm$OfUT+BbIZ^L9-+92vJSOQATD*8#-sU9BlBECx z)nH|qCuQ?qG3Kj@^TfXMClxsUXEjnk+`Umgk?M*OqT$SyCrntV_2;&0naiu+VTM~) ztI|^IBbmPKO}??rc2*l4d9bKPF6eU(3=5Upci(_@spRXZhXhIkSR(?d_fZ$Vzs6U) z_?mIb7qcOCoW2cKXLJL30omo8H}Ic;a5hg%Hn^!_up!3ZV)ih!g;x^gqU%3zd*%S+pLCk91GGz_4PeP+E7gn{QP;E768Q4tZrU+Wbi){ zFJdG++nxQC$_%=FJrQP))ksAU%Q)xQhVIM0u7%!KV@U?U zEIZeY(KG*jlNOs&G@Fd;8Hps`pz{`?&u7Qc|zB2%b+k3hI1DynFQb=?$cCZF6<7#_~J1joaG0&HA z--edhb|VeW^*Kix(Ehfig?PqSdi@vll=F7{B3KPS+cz!{_`N}j5J(ZzHUT`c*p1ZsWiJgbTR(l-miwd;o z-T|6S2eIL1Sle_C*LEZGoQ&R!7nyqd{?Vn|Bcu>aNVnW-8D?Ux#NeJQwKNO_@W_&v zu={}1g(CRFI=Mic%Nbu8dN^Ng%7Y;GdcBO99)6G6E-;^@G-8Lkd z)x;yj#~1r0J^@Nm;)=XlzCH**R+Nt5Dz0*EE8Om${Mg zhbsJz7;j8O@cNSw`0C^ZBIC`8Cw;lv{#0P_Cx6*h}Q(oDQt1pw0%$27Y+neQ*bxg z(a)izv&p{kXQ<;|oW$4z?yb@OL-zbw<(*Zn`o*%oXQ(m6kACOA{f<)6<;%$ApHE<# z7>|IXSaOhJkA^xBsps$j|7||NrvZ3JY&@9n)v zsxSL1^%J*p8T|mJk=w4Uq+TP84BTk9MVl@K-8X%DKALzpZS z$-&dsYU;|;zKvi%KSB7Vx(z1VcxY8IDh5ES`Rj~6bc%NyceCU&dl@>ldG4qz5cxAs zSOGn{wvX>lH4`>wxcjiyHLq4^&J0QdJh@}6C|if zDTF-l8%=Alr(Culd_Dj91txUSWPEB+D0zZt*8BGtsB&^|eo`gnLM&H5jqP&|_ncR0 z&GoZ6d5v+jY(kt-&sJf!pm!scs={U~Q2bqPLTsgN-(rvAK<>|(MRXT{7lB1r z3XL~~?##Txjjng(!a+SUK;L}Ke-VFM4LH0tt2FzRixi1}p9g}j>FTftv1YZUs!g|Aq$zl%2=F+t3Gba>vjhrzuRoz8h!Od?s^O9O5V zI<$Tab!Nb3@=zcZUllQiux^E>Eg-~SS2J@mo8a@qXw^$!UymR>JJ)4+m~ z%x8&uXMbT~qUPnr4SbFzj~#4i)ZybV(u!^m)r=3-u!MSm&Y9u~k9CDAKrC$Ex<(rW zr%_5aVx^mImcdeiz9Jw*sCI$V^#NR05Oto1Qp<` z(KY6K6jfO!k#N&YEsZMu=b)nm=EKmZSgH(bLtBjyB?jrcjKg%kSO7+_(;;Le#}S z9s|2`IUQd0uJrzhVAN)9)2@475h2cCz#wk_ZpR`m((W;D0<mZ~G@{m3x+oK|4iM9dsu!KiJ(!FXS}>=xP!* zj~9TvAp9f1?q|l>OK_>(@s|f!lPVvNB};7Yk}=_FotI3-GjwQt;Z69%+8pO^d+`_qEo-p0_ju;HeG#dccRK$&QJ$FLw;6)=+d~P=HrBJR#eY6U|$>A_6M%N zZ-vR^tvVJk1XoYxl+KOUM?pyt1vOzSe zLZMH9-F~yDR251ER54jh$-8Nf0}We0ke~~F?%@X zZ2-xV$eqHHI-IVqd~ylkplK@?Vhk_w4yl-e2AXg1yEFBUe-gtdUOS(seIzJ!P}N^A z*WpNSV+Z;Xs}q+ci?DKr{oZW~2<@$2gf||caPc_3({lzaU2w@#h?jKNaNU92eRT?m zuLdw7THD$vqOh~Glps@o71#?_Ii+z95^X^hPPA?9H=g8IFX{JHuim!7Y{ynM;L8^=1GJ+C!-;T<@hvDPgjPFn z+dr#2q+1#+O7) zeVbqMSG-jwuEvV(I=3Z@-Px4&Z%9V1{T5+6d8&0ZKfMp(nAeK)B}6Rc2cHl6Mcir- zfkLjljC-e8c#HwLgojgcTsSbR4oiPyG~9|>PL2YIc&CGY9&^#P(TTABhyY zZ}2Gz){^SI$I$KdEKsg;TdRNjG+Fnk!}5z35gOa&&8vRsf>1DNSE$0)f?p6Bpnkthi`e}>Uj zTBB<7YbGt&Hu_eb-N*q zz^=}3L1X4my>m~9tE;OyaFJX)Z?w1BM-c(p22f| zRKA-GE=5`??HHB|hbQqf6VJ!z_aK}c+-4T{+-i)Lq3zt~@Nr^#V#L1R-q_gq z!+YREkKjgz9w!2F-FVh_fUc9vDm~!6Jp~2{5*D3%ZL;%sMAJkG<^0+k98@VI{`~{N zzpgb+s(@(iec|#?cQ1~Vf7a7mx?ljL;Oyz(>!fexl0b0B?rXdqL-EbF2+m_+^`t0`XceZ4s#$og@ znk^IK8(HZ9?a;D%dxbsZ+D?x7KeV@$td2fs+`O)>mo_$1rdeGY1EMlFH8wUPEPYOB ze5rn-{BnPIh(6R5=9}z0qL;iCr^zWKbf?c`rY^yC%0nW>x$jq5-r9I3F8-=AC0QBZ zFE3txbV=Gve4{8j)<6rfmEor?X``m}9HazmL&jfUyz3eo|IaaQMX?VIKfSVk<=vvs zw8{^OUF`vf8ea4@wc6zI;hj$Wo^ldzl)~_5^ zGg)Hu0*!3*#p2h8>!9jzN8YdG@u6qiXbkVHQSx%4zZ(x$T|JV_ieV;oo)kS8s0SXF zmsfS0T58(Oa<>FN8k>4?L;w`0#_-;c6r&>Fy?vm425R#|HNug=1|2w4WHC_!?*LT~ zTxPO|W}WqfvjMya!r@*=mJ;bq+xo9Jf52a>eJ1t+rn){rI;=pIS zPZQSu11`~?G9Z-Fq>aC+445f%3(@GM@;R@q_J;RQ{H!1Gj|)8VHrd19PSR3<7QSWh z$f|F6e{*!~kzmJRdFPd>N#!^9chdkKVJb?V3qIohrq~!R)Z+;PTWzn-?Z4vI2Ue4g zo6KWHzYPv6l%sD^ECtW^LX%^@$)0eiO{tl*n7F8hfoJt%69F2VzH}fK0+reu-COQh zImbTz;zE9^4_)eLO!$Qsz{QlX^R=R8o7QJ(0A(?JYIqtj2_yk$t$lNerQIykUt{)Q z`#c~QcJ}sT9CV6az3`)BP|z5}bvR|4$_q6Pm&TCsWm7+@x?7PkX_vlN`|4%t zM@H}1=FVgWCJqkVQJbCU=h&;c&$+NNl`U`9))||2;ZR$Cc-?gm4=afsgd3_4)(BBT zvMTcP^H*xbDM=x~6$6G-M9uD2ylYr|yu<~NBR z`b=j9Qmf#%B<{8PZ;+I4_iA#JHbHpg+*ihS#6OFQ9bT@#li@vS!)w;~_3ZtqR9ik5 z?bGQp1=&EM8ddm45EIk>X1TzR6QDQQKIm~v6HsIE?a~00|HrXp2*kzC#wKB)$e<(c z&Y^yxgkkDyK(>PI%JtsY>K`(8J?@i)mDc)(GNV$R z5NNUy`@1$>g<_VUNZ`p#SnZxUA6;f?v-NtH(4o^@d2sV)^}qP{kV z>0Cmqh>*b}Y0J)qjyi(a=fr4SJi=HoH5b6&P_mtX_cjQPQxB3Wg9P%g; zGGvhT1du7K=i}ND4EdS=&TQKp=O-qk*fSgcBj(U*~NO~&|xIC z8Z{WYiPf&SV`or89Tpq!Dd*YF%YE-g0lid+hEU^r;fPC1Ksl%T;jf@nDt>L*r#CZS zj;voPONd>sFE|@vEZ!;ZHtQa&Zu4y+#k`KAYHXF0?U|4vK>C6(g0hK8I>Z#$xBy@4 zR7h-jgtJ1Kvd3|=Sp4vUBJK-UlyfL3bn9s`05E~MrDZE9CY%0Dx^=Ky-q<**C)B4B zl9Iweq)(X_E-kQ0vd5I)R*AYS7(z}ZYzCe4q?UXfSP_ZrS2@*>dDN0r00Z|Cr_coyr8!-ly~dhF~>V-4TU#XPSpb24TCqn@tWs z-d^TZW4R;X^Dq(ifLEwtyKwRBf!spw^Sqds>=u*X!gU|$S^s#WsgWx)H`dOe0>Vfd zg?#*Z1|jM&*qhTFyi}ZBX)dxPagbh(N4}$TQXnJGO;t6NWvs5+T2lAbaaJZVG_+?& zm$ORjy&JcC(Q^*Ds5H@d_r>^}>2CnmgQb2_yj%O1(AAbKix-h;YtAjxfG6B8QB<)%QAx1RSUj*j1; zJrq861M_*O_ft4|;<~4~>i4v-H9iStzKU1>h` zZ?n(96^B2)|Eg!u=hnhMZ7J1#xx_j>Dai>cI!phk4}VKcT>J$xb!qmB>!}z4IjbL$ z{0U`YESACc&uD}=1vUGhL_IN~0Q z$U(nK*3K4YcBpd)n&>D@5mXK~;-T}Smv+nTK7S?=b{)%woK^w8ki#kNjL!K&pB`t+oLk%HJ5 z8NDNc$S}+qNyS^s=y{<;<@rx@M39>vZ{JE=C|tQYRGX`^(#abWnmCk+peT$&T6OI7?bp_-?ylJ!jb>7Q=Y-B|(7Dob)>?pd=mXC|c zwH4dBj+Nh%s$GD40kd=qHp{@kFg^c)z{Q{GC%YGeu3BjY!GQMuT#L))9Q8&?zSOXe>g;3l|@_e~{cm@9ET$A=hO&T0Ar+ zI>Ci0)z|xZsQL594=y9;X6^EwgDy*7{H(3i`e)_a)s7?Z-pF?K!O=U_9ivQQ$HTp4 z(X5W7c1)YX?ezryS_w~c)&1sAm1Vy)ht?$JS%MGC!ruqR*?DRG9uIk=MGGuzc{<0w zcsZ*|Kd}$#G7DX`*&tYo06i$&M|=$*N!PG{G@z9>6sP(=kZ|s!u!%cJRXShoq+`AT~u=&D%k9F;g-cof|dASDE zZ%b~iozTdpurq(h98eXe51PRp(?`6(UI#YLg8Nv8oZobU;h={8fS+qy9(`MkD)VI* zLbZdK>rywM{}hYKYa?_M(|Tt-FB4ne_H5I}t4cnVg`%3F^Zos9w`vRALj2R^g$-w2{Y*JV%?~ zWv3d?_rf|p zODQD14JJLBME3ieAkLBS>$f^cD!4V`#ZCWmYTwXK7uRoGd6_d4NZ3kPssH_3u>A2C zp_$)yvmHPg)0geXX=cW&GDA6J16IDA>S)SX7rQL8>D<=YD*7bbq9Y?75%s(C06^5s z_mgK_+Y82o`_bRgfZc#SB4X)%-oYlLEh1QEcP>ZUB8Ns{|L)=L^!>H3XNo!xh=Kh5 z%~HQCePfx2b+UVl3SHYP=bC3e zBpuZGAL1=yPF)qz6sFYVFbkMF_@oa&xzI7QtXiEzEvw^_}m-5?l3Aof2Qz zia11_)_UV+UU$Iae}H4eAf9qiVl-QJQ3(3rx7T3mPHk<-OnXu3+tX)lUd?^#YZeXQ z5Zb@vr#2r#BOkq0I2xz?n&t+k(dbl^v|1^`w{fU+Bc|w$1fvc}UB~<2ui>uftjY2i z9eG!)<#h2Mb3w38OgwkGG(Y|{;!igem&-uC?(d`RazA~#{uJu7qX2BFt(PC@71loy zgRqwVt9w_^>HqPCm-mfGlWK%9ds^R;RIjarRzzk(>$ToVDiN15tIp?TY-Nl@NMSJ~2(tEj zu!R3lK}1l^QsC#qwKogUYkNy9LwUmrhlg+Nf2~ySls^STpc6YEGF=cxE!{7|r<$kG z+Qyo06~zzkF(39l0~+3SI+r~Fr7$-hVgb@U6Z5k`m|@zz#;Cdh-PP*{9{{vX{`cHN z9jRu`B7>4dLsTOlNYxgUmL>wCCzdtf+NHSsjjBH3C?L)d1U_WL^4+9L0mgKDt}AB#+(2t>)?U^M{s-E6-c&7L|ohH}39NcW1z z0m=KBbh>yDpwpsv16)-TKF@x<4S%3YWP=xb`Jrid;DW{D7Ple%5cAH#GENM1K;(b{ z_zyrBbP?bn{Yw95Sk>>76v3&}pBL+-4*y%6b>8#A{FAi?wN=j5@RvC_z+Obz4o?lnhojc=~bI3id zkW}70z0ZYy6kDPZfKQG&*ku*??^I08^WKjV8)g)CH%oE= zHd?up@=asoRMDRjYR{fM^B#RQ(L6K7#1i%aG}#c?Tc#iSo+k1+7f^9s1@P)5iyxh^ z7uXV`1{J`+QA^xNie9}x9$5{9n=^O;AR#~UuHc{UONwxx`R_^7+)0PGpaC6D)37`8 zoXhoioo@%-3WQ(^3L`Vg*)zHog5|+|fpUO&b#s_e#-T1hU&XP;QAx4~WP`ZP6y_H! z{TF|gP0WCx7++w`aunbrXDEY$A`IE}%r{`5GG8&@r~n=zl!}F`N~~&PMpRTx(#!c9 z0<_*Z*AJ{b5jcpYB`(4V(21RDPwZpyCnGUnVSRyMK@MAj^G7p@Ry}yy|IPWz=dOwj;U#EL#dxazq zBpy}ud~{g?YGCd6EiFTRMf@97B8Pkh9ILOxNuz8ss zNSKe}l#!{szhz^)Rf<{Les<^G7wx8=?Zc7fHh`!6iYNLXQiL$V9q2(3;dG!&(_zVm zr85I)7WQ)xSM*EI`|{LM4jX0c0k2b!lO$)hAwx_Yk#;|9X%D6YZcm1A_l*}4G+~mJ% zHXZ9jpZZ!IN_plDTOlXiMRbrZ)<>sGY;Uuy)uSo2xRG$M+!p;_`71ud+dsV62p?Op zojZ_bQ2K}$M8j^Fj1y~m6xq_K5Xi@|r~{AydM+bdgeo*IHTP6FPFWg&L}P!h4p3Fmwl&$+Gxb=In` z`eHVKEt#+@RrG&qf`JC0p^tf?_{`Z$te}Zl^ugAc8F6u0&mmPzWpMV>#Oi%kYx^06tqs23Xx$&C zrNp{j2qX+ZTS!JKeWS_aJp-$a*VfjwB&|LNk(}%JIry`ku=jIQeI4`=CnK#nII$PNCw?xEsC95~uzci{9&(s<6Zn@Z z75ndQ)qfqloohNx*nVSd7D~hl?zDQ0I)*@A*^GaF?%MO^)NPa5<|ui}D7#k$ zkJXgCeOJc6SgIvL=il!gje4GA$PtWx&c;-qFoHi{8_Idn>PTBdR#tn%_D%r0Jn9$# z>>$SKL0N;tHxI8E+M2`o3(?3m~ucsjuEu;-8~?UoQCUt-qZ^6 zw8nvrA~Lt3=_Dc|()C!mDxB=EWCGy?M?Di0L77-UdNpZf{Ex z$8cT`blW0}!J`KT?C4DDiJO&pA@u(r2>XHWwEbLOKO;o3XRfNvC*b$*s=sps(>AtI z{**`6OvD|7y#G_N_4A;i1KCT@nOV*uJGwtg`RJqPPP)&KL)18{jdiVfc!zNRDQiaP z>*(ls%SuQXg5@S+&1;KvY8bsKt^@U+4VK{%yuMVsP4C`cbqek>Ai^oAzM!u^diao! z`0XItVE?SNE>&W9MN<+iF9&WRw!w1eW#ZE|Vr!(M3jyd@asZl4(Toycg@-;a==*%d zyaWMQr6+YB3mR(2IGR5<_ngW0hEQ_>zA{Zg8bdoSb>C*SPrSAV_Oa&=k3Vost zdoB4-Ir*Q_fPWg(p68k)eE!p`1$K0IHB31bl@7 zF(zOvfuCIR%}wC4Pm&~7{;1J`a=8Z2A(y|FCRoD)KynMZG$rQebM;>bh0;VsHIKpd z@ZTW0&RT{N@C$Gq zwBD^LWPZELzA-9`|1tj5fwr~)( zbC>Ws-6oZ;m6hm31YkiR=gKSQsjeJBGQo)&Z@GMP?e}jguzD+$SEXP^;igT0jA%Ac z-vVl(?8t#A1@BXoog1VAAcuMck2atWAbynMi|3@L))urObHU)8;{$oYYvZ|rcB*@R zsZHNWAWD{X-WBk0a{E6WDcR;&-T^jHU<@KbXF;Nc;*7Uz1&B~0V|H*U|C5Ku5P%>Z z0HR~~MZ)R=j2pc>K%fP2H?hXLr2rfcE6+#*Ax%FnmV)ok5;9lDm=JpGZ9&3Z(H;a7 z*+wYb0eB|yU$jI!I8wl97zNq&d&Q9R7$8c5zmmt+-&B_t5)mm&QTwCqw);XomvDmo z{F)HR==vE%aPSFW@?L-}@$vD2bj=W1devXJ13)^06Y#_2wtCBkucgxkrIW`iqW2dF zw(}HTRizK;(}{%`2MJk+=M`jRhWA<8gfP$5kd$wCcL0Wr2%NYl8&T_DN(LbdHcalzj91N6clcuk2adKzN5U&-+h09^3+i+{+vMf2C6$ zb8TH)8NT=yqc1RMsg8C@-QV(AqU03=_EbM@F;z1tZYNrgJc5E^C^jZ{S)O`C2MIX& za~}LdiVi?K`W}to=ts|#eRC-_Iz9DYdGGoEllLI@t8%uQkM&=X4Yo>9&r2Xt5rzYCq6*AmW`Dw6M{mrcr`0$C$))r%bc9Z={wtQjFXcMLXVJz$$B=s9d8(L;=*hWOPcXf5?>gjO+CTy^JcM1v% zg`2^Swg4D_S#h4H`l)L93}Ac9<<*5${I#Tj@OlTtf(D48gG#&m;tpu@AK>9oFuV>} zD1n2v(`}EO+-221Mgu9ti5oAjcmp>twfT2Oaw`0wv>9L|gECguI)qfU>R?I%T}4NY zQ5Emj!tQxKbc*|LkIbHD8X6jG`3_=WXCmTI8+o9kWAHcrx3}PqU}9@^Dt{n4U~aUM zL@xtCz2$hhxw)VE65)IJ-vg+MD-rnF=MOFY=YkLt7l9?9!+pEO(pg?fT z%vS+w%g&MCVpx18p`I*4eAZV#-C2C?ya75D>8XM;|HujDL_J9`I>FHp>*Ao*H4R*#u9Kh$?jpjBoGKx)3wjpOi?MTm6GDXq< zS9E734TYL`k;24x3uk99g3(BbNi5%hCnv{zVaIy#4cP|%yNY1PgxQ3YmSq1^;v&Ic z)Ij02g~Jdn7Em(Pv}t`(0bA8RRAro<5ch{@@OZ#IXMj==UqkzB8ib{#fodHF##FXM zQoxdDBassk5s_V9QSoi9XgkSTjYv^L!pnSoDsjL%PXyZM71C=#7m^l$3zyE+}Jd{(}ra zUieY<1;tVm<$+0^(SLSkEx^&Qc!Qe2_@pFFKzIvW76~dYB9J)yt0|uuOSZ?-Y#Oq? zJ$7+%Gk=6hM*ap>3wTImRA12xS~@xgBkwu>RnoaYS*8Ji5+FTHBQfAl;9sm|bW4BF zVNs+%mxWSUvQ)Kcb}_+F+YsTs5JX6*rNaIvB-xV$Q#Y`T?4`GHN}R^G!;cYg#H9>y z1P80!WlCkJ+(OfvnJ8CANCt_FxppdiZsLp5b$7=x=)ney;e3rtMFfaj3# z`edWFNeMLNI5BFIl4@)Yp7v}?-(WuK>WzdwSJ!eG{X4E>o}9VyWvC)Ipv{l3-5`VZ zo_6)!4h+Qwu5V{~=gZLtcK~EA8QIm9(ATF+>`e1>lSB)iDS z1&6)07?b*;#}3(lGbf-L+F)`X_hU#6c{H^a-!CRG1(#C^0>(;h$bRE=1Ek7RD=!yM z8B?lOD1K_F$y^5O5>`I4hrVCWob!PN)o|d=B9svjQ7s5VHS$McF8D6EOa>H>j%1pD z$7e)VnabkhFsAhje4~U_Skhee(*^uX9ANb0>o?^FtN0V&HYi1r9lL16y{4GrVj%BI zQ*9gThF%zKVDc-XYS}axvXOD*7alTh;(C_OvIMO&!9J5(cEd%XYEX?*EFE@r$Q|4a zW>|OkO2ek$;8Y!)$<73OfWuO*RnJDl@0@x1326)HssAOyRnw>@a9@mg+k~>r8Hdt`-NbP%w$z(v1acseW&Gb+K6x-{6fkzs9m{xeSzR`@+HZz09p9OukzHEINkC zi^kNX-N!-ZxLV_AA%Q-ksY$g3`(shoDhKX%c}x7M*a3nOK1FroTSXV{VRx<5YH?=g z6nD9f*R|n<030O@g;_W>hYu$>+%ksq4=)#>YKj&-XC4&cykTpH%L>bZR^#meiV>PP zf`R*-j2nTgF9VF$pRWV|%o!u^0%`J(gY-kmwjtJ~>0-&LjAhfY+Il}Jm@P9j>T=YB z1z)+0^yN{wIWzlcK-7kRdP|oR)676ZqT7g5;Q66Dumfj9p4W5_!m(oER%Vj{4(U57)@KGGd9lqJlfiSkPz-o;NHA6e|QKVu8dj$!5d|b&A|pOum5Rr zeZ#OWQS;qq^a?vQSulETxza6TPaN&}tShSe05`jJ1@nZYX{K(&A$5@F~F>CO3~_sW~O#*ipl+8n-5q$&arl|FHwxah(!cgK{&ZJ|7NS z)2`o}t%3@-e0CbcbnW(<(!hLrH^&;%@cR|`Ngw+rR+UQZ#buuh-OqNx7N|YnT@D>t zE_kq<@L<>@BNIs}(aX#}=;=dHEk&`65j>mH_m(rTrbwT|+T{i6|2fL~6vuS@h!QE* z-zuc0?y7u7OT@*)72{E%?Fq-zQ4;R4b3hodkM04%YCZhAs@TnE%YSQam3hmXlpWvPLtGI_*xG_ob_S*%HRX@|V7q=#Yjny2o}g93F<3iIcVexsd$b#zn?Ttw98EsGvtB3vyL(SxYWm>P&jKW`%t1^*&Fl%-L; z@l-6^@1*Ke2lj^{btnXGsfChDGHD|f3&MAcnw8_qh9Q^3nU~GKsv8{L>nF^xyGoTm0$~A>bcrsog{lMVtIV(f zw`(kdY15j zWHOWdn2AF&9oC#aU|A~jJO5uJ8;oxKoz`EY^k}R>%s2n-qeFZ}G3pP|_R+7wkF1t| o`>-9o*7NL7aN+MAb2kVqr7z@><#y@6h_9*N*11)B^KtP10seVq$N&HU literal 0 HcmV?d00001 diff --git a/website/www/site/static/images/case-study/linkedin/scheme-3.png b/website/www/site/static/images/case-study/linkedin/scheme-3.png new file mode 100644 index 0000000000000000000000000000000000000000..a7d1dd01b88ede66bdbe5f313bcc681b634833fb GIT binary patch literal 207951 zcmd42cT`i|*Di`6s8m5Z2-2kY-cbakBMQwo#Q$;C z>7M>Q0)qM^vU59P0wO`U_Jey*0pc zeAH&VN&b?N^Cd}XboP~vDMMUkS-^RDD8q& z{%4DX+Rxkn*}_z~Px&7$5#&by|C-JdZEA&p{lcc^W^GT;x|3jc_Zlf_=|T8lnq>a& z8#0k(FIZn`-fj(OB{lR6!=vwi(95738X9^}zifK534`1i&E=JrmM+o*1!Fw`3UMN< z+w+an2G~S zKi~#xzARVDB{k$6#%?HYZD^R(iCJ*fp4JB{m^hB*@mG-d`?QBmI!hgx0RWAGWUdO! zRLjfD8+$V|GaKHYat&;?${G_#-O8rzEy|`X1){0gWs=74j0sCRj))~SKUrEq^?DnS zqL++?5G8J7!t)0G^89niqZ38{U2=F{$#jV#XcIF573>2TbA+uh?E+F)0lG#AZk+7B zvSDZ&PbMJMAd?iq?H~Ed;1J;PGY>AZTHx=a053mF%A5?XZc37T&+g0q$Cp*vb zFwlVXaB0m+f%cOd)~O<#aukDpA#-BhQb2Z+w;{4PZrOX-#=lw+-hg9$ymg}6rOHl@ zg`i8%ZTP;Cr(xeHu8{4L{yfdVcJGJ#XYOI|bA)BeEUmgzXgNN9KR^6xqtPxL`}q2* zAVUwf)2@qwlp|Uw!}Xc!UZn95HnyR^19yhymFb$gU8l&Mi#$rmidC^)h2!@vsq~{K zAqct6uClzD`z34*eHL_j_f_t0g6zseMIi2*lki)#I4>^xTi9BoXmC4nyQm1-nRm2@ zxs|etTnWW4b4)jbKu*U&?fOU3kL6$eh(P&_^}oq6z}Dd&RQa&Wu>+=Vbu9YK4$98> zU=W)JpQe%5nti6{lkmIKxT22y)L1ZAgX~rS@qw^5W{8Z!ac$%$K@1?Z3pAylIoigv z2}o7RkA=!}t|Cf&5E&r_X-9J8TX`jD9a(Fc%i1b@!&1g$RlYLgZ3MR}f*#=$HYR*8 zKh4JmIi`It-?BkD^W-G&!yX`Ychzl78*(~R7mLY*J1Ky$$prx2Wc*Fj>u6@7On`Bh z>Ym&sZeSg4AfA`j045n;&!gRt&O*!o9ub&U4R}xlOgjwqxMxRtxElLe1>$%n?;}_1 z)i^l*=G=-AgXnlZ(Aa?Mq=PqGWzGAls9wSkZ-ky=wP}5uM#bXfmEmDnZ-vb+A!2H2h@jeFH9ErC`-P zGJ77b7<@wks*g42Ef*?LgzpujPr%;d)@nY!56zWd_B+TCv5o|^OcM)NjDF?$57#1T;@(2 z&!PiX61WJM!o6i^GM48J)CMUxF0b4xNSosSu#>$TFoKKbq`>y(CbZ+Q4?bO zHn7IF1!e6FeKtRYl^@j4V$w5Pr06NWi5IZoa=jTJ;>YL1#*ktE{PaCkk=qK{y>Jgw z-7NX++z^D!3B6CE#Pe_OKTHz`#;+?v`J_(5MB$2}d>O1fmh(2zXTYafXptt=TA@QO zB07CIc+jtazUSDm3%%>+5M2FXNZPpz{Y-Q|%<02U*nmHMsz>m~S+IlbJ)6ckx?!i+ z_Xjzq`7?47??A2zo2G}`C)<58$+2krb#b;gJXdc&&6Y(+M>jPylRig%dXAhf%bort z;SGO?+}_s9*}YZp>s>sPWFH`BXh;Wrta7@^#l*~ zkM*5uIzkn8G|`U>PM-|zJRhC5IjEdQq8t=}evpb~?y?*Ms0OImgW=3aSXj77uMG8( zt`;+71@u^Ga^WwV4h>%Wb_0s^fqcrrqpSUfmt%p5wSPf-Ub}|{vBzYs*tqKD`G;vd zKxUt*WlBSlO;zJQ_-O=dSgI|qlAT&0KPkfEC9TSIq0MbwE7#aVlBIyVXdmlUN5y4i zS;4LX5O0QlCRJym>ZqrLgOG#>~8od?I942{cctA1fX z$s^MW&R>`!Nf)6b7*OWuRZOI~fj-t@Xqq4CnCV(lCm=9c*Fg zhdqtwBusZb>}gO2lj8Y**i1yB+j6@!|JxkATV+bz$}AeVEzkakj&whHf3otKQW`}L zds5-=PySD9#Caq3LCM16;;0+x*E@(mBuOocRD6|Xwp$TF@@RpM>(A{sOZXpuA`gu3 z{`?~u{pVqK^dhtNC|rL6L*JUI8*XNS|MXN#{;d+Rg0;z~r>EcSX`_gv^Qit@vJ?0g zB1$7UAQr1xykU5z`Imxqe^mcy`t)g<7IFN)Jx`4I@Uvz3A0*94FUq&v{Lfhbe#cV$ zAJ8--9u7Qjwn~J9NLhF}1WL^mo2R!KfP0h^4-M!bpzv@VWWez{#I0MtX_C$lY@q?bOsXs z?cWfx9$YkX3}e4+Jvu(t#*-{PJiU7P^5xaPQzcp8H0_bU?bzJjZt+P!_wIV^zkASo z*Y?DQ+wta~j{koD(B)Gf{X6>KHlxb_i~PDgHG)+sz-VD%VdQLdbX3B3>&d?|Shn;8 z^PFLJ(zdo%1lbTO0l9SV;T9a#v&Q&*37%n8tL4&g_5k|MfecyA^K}L5(df z1^?Wo#?J>@iu+fnrQFq>f}guPm?Ryu$Uc!ReE-h?%Takc;*wPoECso!$$)DgCQz9p1y>|1uq&?x7As)In`c&2NhZv2OyP%j>Hfr~|MLhXf4zF&p#Mwr_u1Ls=Un`c5$TXH zmh-=D_8w{VzDHj?)5aU*8yRWo8~?j*vERu5Q=JJyZdCXGkAvROmHz{j`Ty!LXhM^m zm`E9jJBF54R_f&B=Kcn(|1Ra{}Srl@4lu1Wn_-XogKPA{r*q2yJKr>>l+S_Nq@~w=U&^o&3_>ukKMZU zYq}Nu0Wn+E_g>A}I?vrd!}BTQcXc#NyodbG;J^4vidPa&@waz)>$yC2;CmU&SWb)< zPrU=sQf|Y`)+CXZNnGc7ai7kMqimY0T)vdXGVSeZBu%!QJ||uJUxC~S`1jfSRXWLy zb82-D62rvc2k&2quKnUqc^Q*XaF^)uM$Gw&aYeQ2<3E!jxcXl{DQpsEO+XTad3f0c z`7rmj*f_LMixWt*^`5qVOMxltA~2@ug13S(wEgd$|I4GEf7}-#sK4d;u;i?cTV{fX zh*!bkM~_ELb3t;f|8wKxd+7)7tQNj~3RrFjlsjnbxBI4uaO^cXaX`^>a61PIU)np66vpxu7{U$kFk)_*b<$ zoRHe|O01@?hGFvoXgfM(QyIls(Ry&Bg4N_N>=8)(uX$}AcrZQ?cu=Xu;y(?J4f*`isRl<3MWlM}HT0qV{)jMqPx0opGAQhYPNC z*;uF19G>Ij<2OfAc~w$}prI(6l&GW<$R3w`Scx&*ohvW6%tjz7F?B#K5|Rz>xHM<6cwWQP72Ye(%`zHRmi*0~1 zAAkQ`EXZS7px|r?aW%V23iEK?97n^#o<67XNS>)Cb?!BwhK_|?GFAdj70TvCajTHU9_dn20d zz)Z)`d+Wh?8#gB_!$UhU@&PvulfyYHhw&rVbA{PfpvRJIcT;Q(O@$cYXzX;!Avx@T zpN=L?vK0G;x*`ly$Fnv$(fskFtpVG4`jcq7f!l$W@KVpXJY{P*l?O*M(am<}qq$@w zmmV}r!YWlTi8T<|KSJgjVPEtL8NATu$xKOXOWx)*9@e%#+mU92P0=b4(>k^AQ!HX3 zAlR>4@)q^((!)cY8bAJg#+qvR?L|PF1OFp?d1^$RoL215@n|v8{^OH;~^||c~AQLKTdeQ z?q&Ai^xFzsKcT5OZGDok%nR;6Ee-OCJP7`VlTKojxGViB;lWjc!+XJW(}ZPrnrysc z0y*5~TW~WR!tyo!&DhF}49Bng5=8OfvuLHR&8>c&XhL}Rp8$*0%3 zxd<7zl7E7V*&GNWqi<_Wi$6J%WTo_y+W;b|ZY4t!c}(m&Zx_8v&djm0=jfrwbbG*U z_qiYWL>>$0kAH_X)ynZoTvc0Z@Z{?3?5xgwjPt(CP-}uLQER}kDaDWY&!gl^ntAP_ znnnzh`l^gyN?!_;-m{)*Dv1sXCMjuenF4FLjZ*{szIhQGI^a&7MkaW!FT??rw{=Y> z7%0l;sM=zvU?<;_`}sSEN0{wdg6LwF>$mU5erx!SoD6D7qyI@ja7X>hxK2EWA*bQ*)~3zN;pK(t|6P-a8$NhyMlP2iqKe+uPS-JjNuK796dO1Q4zy^@DQ z&wwFGcvv98`!17*YB@PMBL(e+kIM|w51+gBJp{_Xc=upT%A>58s9kMSjBaL-iYdm@ z(;xEf)GD>}waS$U`Lkm%vjX+2r#$^1)H`UGn5DBUjv%Y9R{1~$hWSytcS1@67UKBc8|va(78E2M1HVG z$S0F^nKwHT2dmM2<6$w;fb_m3X7T9E@^aa*M5Yj>xE&glj7LAL#x0F43G}|EvLcyu z@fB_T&?7>E()*TuwRrPwyvcoFW(}-1^D2Ay+U7`%wKM98-v#*fPe;j5%Azeh^P(ts zTN3A=ySG(>ZVEwd4i>0zrUV3EjPUz`Jl(2M`n@Hhs9EJSuOtUN7EL}R8|cRDRItqL z&UcIpT`le(7!hte2tqG4&6{@0`R{5ptmS`S<@o@hpJm#iAs`siHVb@OsBL#bw=~aW zXH_l8Rv1HFcw8X1%OX4JL0ZM_NNM0SmROBe<8p16p3B_Ta6Dqg{Iaid7|+LT$Q`_W?C`1Kct%vans9L|pg z13HRAHi>I6PcjdDrk6$EHJ8*EDOVLzdpUYt(SaaNxn!qaSN?R%%k5x7uJ z#rTuXMt!*dHJ?1n6br8JtguYq_bjd<^WnR_oPdm2y{?B5 z#%0P=qrk^j{R+Kz87Vz_Mz(^rF4M!uW^b|ejBaF-6s^bp7VHN_Zx!z@Zg>datz4PQ z@BTl5mKj&>DDMo8RW$o{&8z2j)d`Wy=Gv|Yslb#d3DjQUncw|tLjhHVXIBBuQ{ksq zD_9-gIoy-dlx`?od5VI=4f}MqozqVuHq^n(L=xTC*GnYYG~8Bs>4yg%nYXZx2F#N@ zxVZQf5K&sAZsM!2L~yoTr-6Y89eO6k2}!@H=vy^Zt^hU1G-I z<}Yba$n3jd%&;2&0&B;saVXg0Jr`FScj{ocyS*}~4>WjuX${9RirnOrvSY}dL#Xn@ z*Z5J*Td2NwGD4nNA14iBC^2&MeG1=aW=cawwF8Ab0F;td8)I#+KJN>jc;?D_4wfIc z*|yaN8};WJh`MBm+V&r>gY+-0kX1z%t@E_crGx zZs?Z}`z2S(_&G1vmop-Jo4!8Og%MmIb|2oR2rF7He)@jNCjLSa=7!>)5`dADY~xwIOP6N=K@NZO3i@ElLL1iwm`F19tc-wzSxE zCyV0Y=n5W=@ShJ28BF>c(w}Wm7Dvo4=Xa1~+CcNZHH`wcOblHZ*wF)olLpO)kokVM3|18_$8UkJ?#{u#t|!Z5t67obzq_pgd4Vyb84+ z<)+L?^#6_hx_G$OWvhzAcu)n##HioPh^c1PYABm% ztHp21Z9@3k5gPnJWz1=QH}ejJ%1d$ggJc2vjG2tyYAtZ2V?APQ-Mn!lhGWmu`m(I+ zvOGY$xo!xjL_sh>9_fEC?pU|?Ac=c+kssG=U-I$Ol$#PV_%MxHzHMC!xoFB;4_g^^ zk^X?%VHS%nexJZ7T(8=0shOy(ez@lE+2vW;S&2dSSfS zE)sI{WT8spg?i^@6t$x@0~O37GfwE71@^v)IJvnjF61ytV@z6QA_RN-ON=4h$?DhY z^LH;(h8~Yk7YRzmPtTTv7}=S}b282q6nr8@#seAR&U*^_qss#>>KUROw4;IWBiVZ4A_l`K-PnN5cI$UMRg5wdJ zJJhf@Mxqx}#M+C}j6OnfCIMxOh$z*L<>TWm6lsS;hDZnflAGTRIU(kpO?{{>HuxrS z0P}TJ3-)CS5P2~ewKyRJ*ywvJLt%F$z0=>NvbraBYvJW`V5hU#3%c(}8Go{mK7pOF zyFTXC1U+$drbvCFm}~4-DF-dn$KTfh^hxGNUZHzihpNZmeq)a$gUp@6>Sxm*QNZ1h z!o?MLo9M9MM?>8mKK(hNxx&ui2fNZM&}_FxE9@{)UyA{>+c&NM?D1ROGfYyDnCImYkMN{ zlJFlY9vw0fdKTzQOt&I<%;0H28B{%*Rk>GMDvzse!qvh_-B>XcOfe2|wP{2)JIKf< z*znc{9(B~3zsE^8wZak?0eihuhV;+q?-OVF(h-4oA&Ucg%e7EpQ_Av1BNm%2@oZFb zRb(GjRouE}-Vv$?nYh;`Y_sgE5#@`b(zjGUub8f2pM*@;lDdj=}(#^Kpb5X?Xx zeIHUM?lSdYyju~(Iw4U0-Q64I?%&}w$RAX*fwz(C#%wwI8D#@%kSBh(=}rc&ZzN)K zxdcn5!wWUr+1-;@o^Cyk@}h6laC56Z!X1E092Zz)9vrqRik>B@_sA)_UkO@<9dO*C z1dAM271gkfOz1{Cw$5dZn^4`2_VOzO3LchT(ovmZAkJ3-i-tJ;=+Iw-AQ)EkvZ}}~ z4>)#-4!>0}-T{fF;`+Hhca0OOBq0^wn0pw^$za0*sk6%?Ny)q{gNu!woRVGYo?onl zC+w*a@Mww7b;kZ!+C+E#>eo$uV*qKo$!9VNoHG3wsUsTYJO7w&Ssoc#YmhdIX>gV+ z0vDbzk@PO8+WCO@+}10b_J<_=cPD0R$5r|GYpxZgm%4i zH9^2e21!w7E3-ZJx{r#Ap@~rfk$Rx!ReOCT17rI|SQ02?JM+2sCMbIJxcz6}qB7Ay zRMd^yBvjYb{^VZQ6u{{289wezEBhN&~9vVR@bbPh^-Jx*_&byYQB{nIfs-nbFpGq0e!vnZFzjREv=!U))>L4uOQvxj9OWde8nuSN^N51k{feBF z0Ear8w)zb~_O2jG4x<}=_+6;aGqE;1TLog29r%Q#KWFF&T#$q_*7M1=TulgY7z9`c zrF}=r;mOJydb2mT@To0)ZBB#3Fg;pjjS~kpo}A&Sw+B1=!SBwpq`<}t_?1!PsdLfk zT?l2dE^SWRzz3S)TL9xn+5KZ=T1mIse9TV~K(Bo27M#oZ#rUPliRVHp=J=~!mj&dm z3lQxE6M#1!%vQL!#I58PA3(GHA_w|6$_kPwafRcugpW`53$f^UcR>c5Om9=i;g!Jx zh2t~!n$!=Yyx+eYvg|3?x392qRl73~lxkRRwo|xfCz5b5zj+b#(=k%3qB|x7iO}Vk ziu`Y&zHz>?pxww=x zrwl}{Y}2GsTMJpe(`^v^rm7ALMSYA~$nFZSHm!@~GkI+<5KUs~@+qwjFTNw?N@d0p zfh8ZZr75szr`hX}2?Pm&nwNf-JEnw}?$&mhU$(s--Tih~m;*;?{?5bg#kimk_2 z>w1Idtuj9eW<1o2gUy}^l%S5R=j=1x#fu!Tj5|y^PkGWCRZI1lg^dqYC#GA){ye8m zYX|4kuU{R_M!M7kaX9FLLQ`3t9Pp8H@A8tZOQ|7QP;l#;#&hwQyiMu1po>Rz#^NtuNkzSWe{@l)X>fr?(x9|g4`(FAFsw0F7Y-FC zIM_!oe~8@mX8*>APot@gMkTvj41eW9kBB)t z@~KaHRb#A9Mt|8eQ?49}rpDOMl*P;s9t?`?@wy~$N-IU`uVyvEJU_NKJ{a@RJ`3TJ zr>l_fF~hFL_^?yU3iev%L(59(OOxbC)szO)WPSkVbX788%Mz!Ol=b(CZeb z@mxna=Wu0;H%Hz2(USYaH7sE}smdzxU&I^5MqTG7ewLs@4i^2p0`PV zp6halCWjWI{D34SiBZeFiL%hm76 zD)*Hs63Q-LRE#YnF(&_L>QB6;#^sH({MJ%JdG^>m8|+KZGtBc8yxaRB-&6)D*V}2c zsq2SUpVYq4smle&^7u#*xW_v?((yMi3wKQ~-7&$GCNy^1^ylA00v6(35A-x07!NHt zoUZf=m_*sFUNfcgWU&fjX(F|e7}Q@j3)BwDha4-l94XsTh%#Mu4?&G&=b6vu$u~yn zW&W1kWmhOP88)AtGU6p=@ZEkwm`huzk;}6Iv8a$&MM<%pi4LvkSJNFn*$BZh5C1fI zhV87k^$tyv1E_&ku7d$ziV!TJdmVIgTk^&%kd~G{mRkW}?DTKh{AQI*9!afk{%2v+ zxP`TWc3;~zx@Ge-9SHA3-+ao`B?5v0$mhe_FQj|bk&nS(@S7q|BJqlCq^#vP411l# zo7Q@X{juv@cEipxH1cfevA9JZiQ_aHmwaJj_xrjbjIs8b1Tzj-jmjuUys!Cx3pY5c z-ZMW2;$nVXIh*w4W(Xxx=^o#Dw5DaU{%qKRf;seRz}>2uh9o;}f_JEjJs_#;ZkJPO zlV^bJI>6Wq)nF|;Cv|h`qsW;(bG49|NGoK&w^X|*&ZgDDSjdS=vpLJL7wmsGQYKR2 zL&?+DN>t(E;>xW|n{R9qWXHQq#ztwnU*i@rG_YpVrQl4a7n>3aE&=WEJ`_OUs8Q+i zJSkdfV>@Q}<;j+ta0NarG{to-~_1aVN z8#kJXOQT{?5Nv$)#VqMy6tvU#tb>BzF6#YiYTo9}4h*)P(h_FnN#TB;N7WhoYB|vp z6;gN>)>hnGQdiY3oHF2{M-6m_<6l253rv!9miVK|0|Zx{}XxlNN96k zs4ddZOVM9cdhCkANP>0pZBgwq`jJ1PR+iL>giXmY7@R$!;zMZ65Xr zIle*;>ehA4GDDwhE7O(^zBEv3Xw>;g;vq> zHD=Ho1H_qDBft9(=T*~+m89&>PmbKf8KJ#a1y8u{EUitVK3ewGNO?|S8`waCyKRpf zex^GFf)UMT5>V`WdaTH2_sdT%1#E|Tv&85G`wbQCeiV4mFT`>WM*=GYEFa~;d^?HT zY~P%#yl9<|+wDx`>TtuPp9P*W;RE@F@A06$?zKdiMIylCU1CE*iO+2=gAa;sO3=!C z={<)maQ83Tq^3=aracNjgc=8$zHTXJH4&AV`t^D0B!A%Gv9&Y?+#wsbk9+=hpw9L{ zx~lTl+nxa6!$%LN{Egu_*$0%Mm`;!uy!@a@JfkzA|9tl;;T-_C+oZmGsL(eoe=NB1xo~+2r4*-DlNl2>slgmBvIX_vT@o zhoJDe7rlT0MYN=FzyT>nhdGOJ>p%u@PZfJ>QTjjihySmJAwJ=a|uP2 z?ESFee~2d--unS^ft5ku-RP>q&U907qGugAN2idSE1*w3;wivz-ou@F&T!X|Idd#e z3j3<)VdEu#!*NWBr3rTLs#70qtuu(y>db7b zi0sJh*WnZ@O36byWcWcluBO-DhQbQlPR^dgue=dWE~$v+gu=ixJlA|s38J@LA4`ce zSt3&APe;upN_(t2q`5(W;VU&Ht&wVRgv%>+7J6A@sXXdQo}fH|Rs0^C5x+-{H3u!V z`jYJbc#G6>^HMat=H)>^K%Vs5a|Ly};)|VHT4c}{-zy#y%iv6C6lF}GE89QdEPHud zP#{6bBdGZZ_>_m&R*^r&|5So4v70v3Fo^9`lF1hlRH5P@Mf+BO$qg9PqmW`4^2qz+ zmBmw*2bgo+5We?Fd_10+L5E#Zm6FLzx3nEBPG5c zdAbOK$iVC60ir&kwY!!SH}h%~8*h7bJma3fFjzx z_Q;AbykNR@L3hM|%jVZfgJv7u7XgqBO`gxi7+U@2KXhCWv@~pCOTj&O!a8sgg|BRk z&9yO92zK}0pd2F<2%dbuDE85&Gj`}4?o9DEfqpp%JRLH&LYA|5+;)hsQi=KPAclmm zegCMhS{!O7|6?Msi#dSgBZcj9kjV7!u-_vx`9QVJ-lY_)(I-YJ#VtP!xr}^k<489VSNdz_9BHNbbia(asw=E##mv_!s>Z0}yPP=@(cLfR54z3IS zE>)_nuYWYAEUQ9(--M29&f@uUvnHc|j#t?fTHd6X(^%gBL!MHZ;c2+bRluZ>ahp9P z4Ymp~6o6o;j2|f*lSWbAm<(153;(wHt*2iL$Wc5JES{1x&CM%%?yD5f;?ybLbsj&^ z#!wa#WGghB(NFNByT-_%M;g-8JLW8}iX^n?p>;Cb?Dc&A^r2M`VZqmw+PWy3F|u{N zI67QcMkAjwv3TBVOH}(4^@x`!1BBSEc}m$1AMMjSmVw7BGYTW>>5dW~242m1~ zCnm=@hh?--gZGz3BfpJjN)Oc#n7-A8^v{{W0x@^aov$3XUnjV(g*V9dt{@Z&#f50( zz(?eXbRL26`8>?hSNAOQy6z$$OzRQN2kpd9P>2V#FFL;SQ&-nKMqxLhnEu@>+_q`R ztrx%Tkhyd8AaFZCT65H==_aIZeU<LqqeopV#?hq;Ug*al3&a{ic*<16b>_ei0E*-A-RcU5* zeFoHM$P>=KRjvG|JfXJ)O0f)JPS4|)-rP(lqaw(_lWP(|UlUR>#OP84SIVvaHT6f| z^GR1U^~FS#s61|+{D7vWVOQT%2^ZNFv{dTf>bo+K6vS0t&TctJEOD zWVpEuz{-C0$m_Wmpw&>T-;G+J=;6`j2+ae0lOg)&z%mTG|#{Kn|58{y+CvS}Nz88X`* z`Q2$mw>$972w|*AS^wtMH0JZYw@6_G6@Mog>U6Q@ibw5Il&Jx9IK$Gb0Plg2WQuP} zKC%0k>)q|H+R%g$%K#++JlkSx+ zVd&l?{%dXY&(qT|1aIm?Faa{NTzNyK1LLj_(?u^1Xa?W0`QtOhG!{wGHb2W(yOrO) z#nG^muH#E=MByQW11Axw>oVU;rI)G!a5DAA6@u`(#Fq=&&LH1FLI|A+0K)LwG%_1y-Q?2 z4c;9Ke}8!nXGka>HATqpG-}$I3m-1pzTZAkO>M6Ix?awN+B?yr8gv0(yGrn8ZRU*( zL+@;wHHY)KJ+;2bSwG(FN{0)!HWcb8Sa^hJKhTpneHjVkw~ACDm%yKtZps(n6P$%F zkkX@T8iT~X3QN@{`}W~BW(foU>K!2u`NF z$EIjK0)p+$1Ci^@UA_s)W~&jDJJ{ zZ*CbgWafNCwwp(B-IsC1$~o+oCTuXuZu`nmjZq?M_*oDFyQMD{FEmmUAeq?*!jqZ} z&c&aOxI%HeSKfhpAe(!h5W9*hc4c_I6=o(+o&g_N^rKxNz84LYGpA$Mj1Y$C=(Yu48XwlD;AQTuEkao3tWF#UhbK-qH;%I)!;Gx1TE`nKgY9_3>rNz^7{o zh60+57-VjmWvj)CEMWt0ZA`RtL95P-p(QxPb>UZZxbR`0F6W-xt-uY_6xUhCrxl1+ z$i^P!dnH#Tr#(pQcSeRY;c&)qD}>@y>#xA@0_tsC!T>}3dL^^C&*@kX=jCYxHMfe( zdFYc6Gji}1wZzW*CCy1H51oJhpfh!#X>AFlt&N=2EwucRp_|;XT%7LO(#`5Ii)vN* zKDM7%(Rw?2fl#8n2sAJHHCqKd|112E;h}Gze1}z(Ha=?4pNysmrNpx|dLKqzC%dJ} zYiv3}28>J(f=QrF!I)0uX@Gjq)&9NRlhJ(e8IkA|nIahPFLrV%DwTA4Ng9wos8F#(a2-KeO zbq-&XESHq=U4I{tsK0Co`LgNj8=%T{D6{hDnx7PcpZY@>_Ua4qv)2*$X(Y zYU)_{3eSvmdWCx1>g^A&Uti%ulkz7kcJ#LDrVp z?t$EF9}`?@&ClNlJNqxZNGZkV;{D?^Y*z1h@0+Bk{?dizPK?LqdDUhi9s4c&&ShfG z7cXzhAGQqiC%p2sKve|hc_{52LSm!6L{(|SN84;=GV0{Jz^V#--al63?|1+YK(=w` z`3In4h;+u8?bl-MiE@eU}LlrN0_Gd{mcspNy+ z0*Y*Y(NChLcyS)!y)3ET0wnRUd$(fK-%Kgl_`6=tUsz;OKThNAR`zEztS$#;0D&Qn z$R9t_48Rv#8*~2S&*wgBkgcak8@68P#~pf)wo4JYtS0C2Abqn|QI2&fSM#_y0<58t zldpugg$k2f>AC~6zyGia0lP1XnEQ47dd|KvH#IdSbZ;8BKC(0d&B?d+G#hfynkTNw zY-JPmSn^i8?I9E2*;d*M&eT|bdepr!TJxxEQ9M@|qx)(JW0F-oh_6thiu8XyIiYV^ z&-~i5mk(C9@-m*)V?WHK`I|r`j3bX}LSwBchv^H&ic|z37o9a6kJwT@XVL^8@RnDsNKbg`w}ddpyo2r-8DmBqi>QNq88 zB&!9-j2>isK8wGzr{&L55PH!UkWjCA*TpRj*2mBcUOu>kmb`of2iUSR?}lEmaPlS~ zaBv9I@S)+s;(+);#uwhiUQ1#q<8bHZ<*jTiT?*n75YjFSx=sMsd8-6^-tapk4uIHp zM*3R>;0v+zSRC6Z`W3i(r3&zMKluRKLYLOkuP*J-DMTCHslMlpug=TT7 zwOUTl9Me!ILr)Qcd>kZ zaQBu=m(`1Isg%J+Sh(#WmecmEje01wyFVlW=@I@2u6kjcGQPs%iBz~a4?tPm?z(S_ zURmaR%JSNB2f2&>wH8iievvcKeWsyuK@5AjN0bLB*xm?w_IE8X%E;|VMg9rA6 z6UNbLwyX*T=5era*LJ-Sm)1KM=fwVTo@A!q%9wOL1=KhWw%V5szYeTvs0Xu7lWpIz zY+X;ojQHX6TtmX3`ICiizD%I#|@X=Vy7}t*mH0jqjy9dI|J4fW%mX?jK`kN*L zncCN<@#p?cLu?XBhTSX0+T-KjA;}L4pjQLq6S5R+HJ8oLB+8BWM{sL&u_xAkbrE4n zx?-OG*ySoK%Z%v3Q^GI&znvnV@b8jB1&{fL-p&;C?@y`PSTqgHx29zGdf|ILNqoR< zSDOp1o%z+p4{kaFH@#V&aDUmR-_r&$JMmptr6EEL=?C6Q%FKdpRBZYUH$9aDvFc+R z>3KfE(npDbM15%ex+RegPF2t&Q7Os65|9eW-G6{9ZZsWwW`mu{!G~Oo5gAUm54|WS z+&BF4%{@9ZKVFd%qX>osV92-7o$JT9)!JG2XBQ=Y<SfBT1zNVIQe4a6j<~t@?J@d+~0{>JFCUJWk4CV^W=aAkx}AfA}XadW-)yA>Z4h8 zqjpM3McKynS<^%hyTqqtKdkTw!tZX~;i$j$$_X83sg#uTT+Ai)OspiVdsKrh(#vMA zv}jyEROLDV5@P>iO!;g`yRcrwR$DTIC zuRSQBPbxbN>d;3yV$*A%tQsRQUw`_c2Q8kuc5Eq#V)?qJ5ka_7!7?M8$P9}onJ-yf zyMAVjBK^jW8(EJVjeA!2@fL2;A=B30#)lGgyc+B{VDyRk zjX6i&3e=t2^G3&a){XD_?X!hcDy`keUKINot6&Ki%igy$(d*Pn`!lXK6`Kk#b!{l9 zecoG^p52EUvSS4ZEpRm>wz`4Gj*Il(H<{8W4J^Am8S?Gj$^FvmoY@_>H>m?k7HSQ*JEpTvj%$p1RBVXQw!9wS4wgL!Clv?(?xu z?W~RH)tq+=UK{xY)c|JJrtK1;nrX}j6IL|T<}v$ojxX3e7GK_rDt6xV+A*CwT8etb z{3FPLTL)C+*kTiy#?~R20IM)#;cW?UFi(6V1R}{&Py#K6HF!JLVZ%Bnd`nN3?^Q-! z+U+vbk*|(pIfhV-GqBZsFltmGiQLJwx(D4Z?AeZPx>!~)JVTaTAuyW5zc?k~R(bvM z`P-u0lpK7mwRf`!to|V$lZf8;+W%t)Qiu-9BJ1Hy%QI z1(>fVN_T%O?%PXws2A1+vrWq{vx{$$^AXgXjC%$~YJ}jPeJ*Dx#E_Zp*cp&nc9On4 zy!V!HCwnxIqh`4Bb#B4WXOD^V>8oB1)bv_+A{4`~q;{Ep0xz5U0Hw0_hsT2g=h=R| zqFdNnVlp!88FlOsHlBo?_KsRw_0a9uxuoS*H|2UzwtnLaFD z)Ob7HoTQv}^#3sT)=^PzegE))fr@~NfRZ8#D2^y8AfX5mpAqi z?TdR+WaW87PNjvR$`-i&u_5KYdt%y^hoF0c5$d}TTIa_W5>K;GZ0QR8PVL{@P;E6U zeYDG+qi<<08l}o*AgM^J#~p0j635Zzxp(F|ap&HTEUJt$YxZCqiD;e%cUq@OH&nx> z`v&AuA|gTtH@=Qu7tt8838mhB-4V;H@wEIa2g+s_RI=Oa$Fxmx5;-4iGgJr(8Q4vi z{iT1WUA(*R!+GuB7c{T>tOlWtC%;Pt)IW}|;>oq zr3UNbV%9cQwTk)*RxVYn*0(UTdm&{zW$Ra|?huaoxPmd>bzV^<1QkW!%6pl88p>3XzAS+0lXdjrIHI%?aD#3}Bg+;4f^ zbb?nra9Op7=QtI*KC=xfv6={SiiR)WmyE~|U@aE7694eZM%g9^Bf zB^w9IW!JvZ&f4T#-ulBgF~pVaL?(UC%JOinSGx@+XNdKL$ubuOh7eY*Hf5CX^6rOj zq(sBYbH#dhL46Ni-dvw)xxmNZe#sguX#Z1I)9J3I$(hn-5t{rIwgCxco6XXsi_tV~ z*2o0qR8`J}7^zc%r;_IthqLTmQ%E>C^(f9e!Y%l%XEc8!xQO5{C5jljDTe3r3%Gn~ zUxeLP*x@hx?QZVRXvLNKOyc##mVt7OwRnP*g~xz zTk;Nag%CD%r}vc3pe8o-B{kTe9f=j4s1~te&69^H;au769Ze+5L-_jDrPyzKmlI06 zeEX!-xQtg~;zIN7%44{)TXRXrjS`xIgoG)u(X6uXliU(oXIb3<7X}~ zdY+{{^%ge~?kiwk^P_*aHs^x!9AYBw&`3imAzY@=#%NyKxo&`+U$_Po3X|NRP}<6-*YRuC?{_DdGp7(PSd`GxPE|^8%S2e_wdi ziCHrJ_?#=taUKz{bNe(GP43l>w&X3DJOzf1g65m8cdjXl^5&Udk1-8XCjQYUG+#py zv*gXNKUwa_nW%0p(niDagMOn7H`2;6#od}cN+^~Fs{3*sebng5FSAXyt%rZbg$zF9 zY!D%y*~>0m(kby;+li@bVHXVRMQDx6&K0D5#Yr-cdY4krrBcfb`1gjpic|VedVIk+ z@<(A^VKC47SnR0?`%^qUraC%F(TJt8l?bK#@64FFq-Y*+Tw`%Vu?f~ZxO;+uG}-w; z#Ny%SEZV@K0o&SNF-(VJ{2AHwj4-+O^>;6bV~123Qiz$?uUBYyzemEuD(mi8m-<-@ z8{Dmrd|G2tIg)j2h7*(y79v(b&fmZ*(o@7k(yT3exCdirD0^`z;*^ItYxv?S#uN8O zXTPXKxFmLbArDplx_Wz0T~)>8b&99hKmmWRNo0M~0OQKsp zz^)X(U)j=$a-H3?WmTi&X*O|k2{2kVfcu(T2#dD7Z>Y}eHj~uzN^IQL?&^(W54Yxp zT~o(5mFa=fPS_sGS)BM@ELU!X)==fmDeiPlZ+OK%+f@S+r7q?2=oY?15U4h=m8Mmq zd_8qpQ&a}DHwkC8wi-XHTtdjEwGnWsb;hoS>7$$FX~tdA%{TXS;bp)5ryu$)_TGEI z@2wvIEb1x8oVdmOsr3opB_?@YftQ839@WCp{!SbAZK)nhKHXo8cq$FMX4-VFXHWeg zV$+edjZWYBw(v7&#hH0fTF+_7vq)>tmEqE&4MT=O20B7SLwu%x+|)g*0IkfgDxXgo zN(hrV%ZB~<{Y`%OHJ1IF!Mh>CHS%u)C9G$O*3#^luVma?QogZ7b4asi^xk0q;+(+4 zYfWyBk9S5EgAS5TM5gN{C-Q%}{8MaNi$&*fkAUej%`KR_gXK_an=rTQLEm0g!Z@wb z{l}cbRE1rJX)02M{bzF4tGQi3B#raV3eC=gnIx+80S*cFC1ii$bFmB0zI!(hhsuLfG1otkjG*MyZ1+ zN4d4bvBEQ7%jU!7nN=Cy>8Z_!>Dpf@%R9 zO1#S7dO&@90n>9Ymp#YCwvF}Oha7b%D5_wW8amr*XlH=rPvZm?y;(s;*f)hI3|IfZPycJf0+kX zZ@#grTuhF3PSbkBVl}!>p(xJE3mP&~o#n+3XRtrhIom0ZyO1OPfJnG&`B7Xfcx5jo zQ`yVTGV<_NmG|izb{xO>oBk5gah+uTIV-FMPm&P^A$5|$=_B8|>~r-7tjqAF9d^WQ zB#zr8w5O-%8TtcBvrlgN$&|!`f%8LLqrQf}3{Khvo*|&cPo(xOfXir69UU3bv~~fR z!4%U!2nWH@M|E17PIEUQ%oS{ZU*FHEFs-{s3lPbNlD^f|0Q_jRti$CGj>;PfsHb;L z!el&p^QhC^{f&}&S(po55yicuZ_2d-bNWjE@je{?<9_}m07w{KE*m}u6a1+1-{TzI zT#^NdDo5XPmzOgy1CH#`PXI#1QTc}cXLo?S5+R(GnfV8Eg#s_Nj%EV@E&}aOss1~_ zMFz3yza!oJFJe$+-na&i3AZ@N0xIA*O#1F*WMlw`3mWmiLO?4yrUC%ntR1vQaHdpO zYaac@{b*|g>;ax$aJ@H~k*KV|dK*&zFDsgX3}e2*;y;csQoiZGHB|G3o#w zJp#8;H2PhtksQ_S|qYsPZ6Ek7YA^#oD zQJ(6TT;ZR29sL`8;(rZ05+JUCsp@Vn4R;R?vK8vLeTj^uK3?8O4M31xhCS+9+}5D( zJTH&xXgcw)&}+HZ)~WzhCV|6TN?0-gU#S$fTX+>A>{bmDmydQj7JYQCyRi7FGPn;mbobv?6A-{IpWWw}*45gh9U-U4$Y7B=;m(2QHNxy=Q6yMzvHvSEw}7eEg`HVD8snsL<7b z-XE(()K>Sj;a+8K{pH?RW|;e`JR}8NHiTd>F;6K{*ab=sQ5dT#Xwas(^cmR7%uBqy zyu>h=-r^JG%7&oR&Km~g%LF3#X;i3$!!UgNr5Dp<9^O5Aj6Twfw01G~6)-bRUXi1^SXf?pMB1Zy~ z;{hR#c?^crO-b?o4F*v0_xT3_Y}LQ@0_>ys70_n={*kvIRUz3SIAS(jt_UDD9P~t!)t>DbP|kCv^j!^R3qY>WB^GgxHDzT0{*2XlnBgO>FlCLFpoH%C zbO*`wbNs%i+~4(CsB;0%B19%Nc2sYLaZ_ja`T=mC0!8yOK%hYY@ik%-!rpNd`@=lJ zRY0XayOX+aOTKXn5PAE}&VMM;#cPJkl+exzK^sr_7@|)~GZleO1NFCx z9Kokx!I#V$R0X@wbe^7=ae~M~Wc*st3D|ie!1p3B2*RwkdyxE4g%hA7+|>X6dUM>R z3IA9*k>=l^hs+Z&;b$gD)PG(7KQagG_XmjR@9eWo3bO&7CHLlb=7g~A*7bk9lnQ$3 zr~kxF1K3ZovGk6U{=BWbs8s2oe-roR)fM*G(EkJ7Ls));CV5rIs7uFmzqveK4{&!G zfu=qN*@_Q84h=y1IFA@4Y@reS4n*Y_Fo=KY=Z`A?U-j|d;8GA`m*R7gRGAovNjb4c zV)PUvB{`^3PR72s#86zFl@x9Z^dN*?VFuhI5n>4{lbJXm-6MBqp-dS#zrp;#iWF&w zHvZ9d&Xcg;kfylgJJ&0FkDs5v_1w3e9VY_V>t8Vo7w7ojoOC$>B-&%@^xhsKs#F!N ziRk+(R+oC50g(|%fVsNm{XCRuA0$R-d4n&y%FpHYkJQ}?CLb>o|K37no zA|{1+(#dYJZqW7$yx{>be-!cLsOtx52_ER+LhG6%nn-J}sgvlkoIRRun^r85-kb3YU2YT@J6$tj4m99z< z?1ZJuc{;qrJ_TvEdqaSQVD;|-QHspXVyS>k$6)ZNUm^sI5C`yrHPznB_~O34zTW)T z38L0re}DgD9#lMhP5=vj;Kfb|JMRJIY+UT|);ymzGmyL2xbYf+37k;!MLc-U@FzNf z8$gz9Zp>6VuHp9tNGv;`lpwKu)M_Nr9w)4YUnK0vhy*5;(pEY+*gv0#AmCIOllh(2 z!N%bS-gE=lddEK=|ML%<_|iY@LyBV*GG1PmC0-t$e+u?Uep}-sRSiAD{mo^i5yhJX zU7ipXIF9ZdGs0yE%+AJzMJoLDNZ!9Jt7E1~V=X;~%De@2(*V(iAHiSlfpT20$YU$@ z=$Ei~+W$2G@OTNrZWM}%h?Mpo7A*7Mv?RO6=x_x9tpXYG=YK#86Dj|HcSr#4o7=Jm zRKFhW7v6;ec3D(Z)bT#D>Vwk^41YujB$)OAEuLFH+dm~e2a7)s_`KlFzChsO`lAja znA}UCLLqF?4!m@@b`w$ujMwnQDGnWT;@Gi=oste^6kG1?JlJ$_zge?#rmkHnbag6%YcOEv7O4jISC3jPtfl>~VQ!EkuFNk{y0u6E+# z3}uvBsGAy(g%4H7uqBGSU}4$>C6hF`wC&H{5+rL8+}^(Qee$1>jjXo#pG5crY`0yP zyr?uoL&)-EZx%7{}?W=pvp#Av@!b{UwQe+H`o{z+GW zF|4;F#;IE>CQxzHDO-|RnwGt+JK=Rve5dTU*UX|z#azTsjQDIk zoqlG7%U2+#2EJp7??%8SbG2r-_of=HbLgj#hGq>Kl3ov=dRu60wu++*u`%Cuc7{^= z-83mR93wE!{eo35cr4J`BmIvbxWrN}<+x3sa_ruUQas;(&$9AP^{Tvz9jfR#%EDF8 z+oWsAnno+otYY9&joMlqMQiXKbuErEmB>iVaU{dP93-A9=8V|m{=r2P&m#sOm zo~Aq=b6{3aoOi=j3%Ojt=+=K-sHCn)aCbCxij5)6R`&2!tm-aF21C%0_3(&`1xqUG zx?q(B=v$b@1~|`nUlQztvn{EA;d0w8K*w3cH4PDa*RY6yeII`bJcL{=SkhBPjst&Y8DYC5UwnkvqXeTx}rfsVv=PYS8os6B?QDzLFt2Y^5d1m!8A($7 zu3V8UPx5v!iqSA^U^>CmDvMszVx+&0A0D|j@6sL_aNvS&>yOMsIcB#k5OfGR=TH)= zu65dmTX#uKVl7xSSq`>Ro(ry_ePMvoe!c)MC&+61>^=>k$7>L9M8`rAP?B70VpI8- zb92$i&uy9xX*DJtHcA$@QR?Tgi7^__tI$R(u?>7Db0}Njwo*0glEiu)QE6Tt!+SqJ z1=m;4ud7zt|8!8Pnoue;{;~R<33vUWhWF{P#Pa9ljVv3j|DsgbGjYhc6A4A0X}_I~ zitExDmKRRtS+QR{)sef=jgECwGo>l-+oo0;AI5Rje`lr|ZZOJ!t)Y>FPKxU$6a{;D zE#z;P5yoy3>(FQVDia;@DRV42>E5f#Scv(4OCDK1K`gSN9zx2Uu?}a zG|$dASAr%E<4`s)>j%9Wcr@YTfW=p}O%MOoX8VbVaGooIHd`m~ywZ^$uzu}+0zL>) zfBVQ?A--EG(k`@XTKM?wP^l&GIOL>|%B8(pHQY%?^B8d!65a~OmiFWemO(kn0=xGL zGf{nO(Z!^i2W>@PK2@E8VkX-PfU>;;%45r`0gNJegoLlYihZ6`oEUoU8L?7zQQQ67 z9_7?wQH!8y z?CWc|TG>*CY1pvPlht>U`Ycn$1M$>=Nfg$hu;jgF*;kxK6I+Y{b8o3_Z82?qUj_<# z>BmYbhHtSUk*amMyWhkMn*>T2+1B}XcHm z$QKt0{Dqr0S6TZ`Mqa~S<(hyU*H4PkD3~ALnBJEb77R%+V(OzYuMl3TzJ0Pdv~%MBh2obTiI&O4}Yk3*h<{Eo_9kpl7Df6jXi|T zZ-2ZPj_mr%<5grKoH8g&LxreH1$SOVEL7rR25kny9T(^G zNF5&zF6wEWZn4qP8l<^_DYg-q1_vRJXIYU6xT$dAEDqU$aI8<#zowg3|GoVlAw64Z z*YeK^0GcD_5iQJ3D8^>!^jknUc*f8DshXOxd)lK|FKvD;lA2hj1)58zXWo|WL@Ez& zaj)G^5wlc!o6?c*9d8~vtr54rdypf`vV>W0?`t)wUdwJJF&f6x0qjDy=qGK(#iU2c zD5KvRs$;)(NjD<&)B6<3K+zJ=h(n=IXEIR22V=_Kmpjm5CDw0~y|c{K&Glc~4LsrTkvY>E9M{qgdOP;ZYu%LZ`0B{4rsLv*zM!m;2wrV+=4i)2RCf z5XFTz0$c;Dmt}m>$hYmWWpg`*Os^1Fi=BazyN7wAsr`gV(dSh|F$OZ2A<9TWRpeW< zsd`M%w2riaTib`bzGeb8)R;tn&qXo&@@(^}9{Az(WbWXIicOlQ*}%2!Pf@wg2PI=y zE2?bb%FWnYy!_eYRK32PZW_2&ma^Srr&9`oET1z#lOKr91f3uveFm`0x&F5GgNaEp zk?!sEkD*3LL$+U6F^7F;;$m&h7EQdKQMGdsPu5Wz_o3hcMqjA+&)wQF&R8&n% zm*%K`caD>_jepa@RoN+Zqmz7Z-Fa_8M^XFFs2(n(cC^htu|X;q7>+l5Zf zwmcWJADA5aC=hld*4pqZQ7P`*2&#{!EJ%$>1yj~nnVq0$E7T=}eIwZRJ9|jhw!d=3peWemS)X5`EuHo zt;L69ahj9J2c5BM2eXK*pmS?nVLg>gE4_z&Mk==S{D-z3rd-ASdivT4iby-dui+iH zl~8_IM;$Fbnk^v%Hl?09h7uCStieNK$?DKoJd645=t=dvgDvgV!wZ6IbJjUt-_C;& zugKgvd<))^f$(m`lcX1G)d$hM7O{YnE*Hd^I=pSui+A??bm?r0D33Zfd5HCtRyAC( zH>f|$%k=%-W3b=X7QpRDP$Xk_>hrN@JgMbs%nEsGB1VYS2>)EcG8Lp;?zg8Mx~_&^ z`V{6jg*7odr3IUzJU?2y%%AU6+>N3{}zzbxjwSX?KZH}z`gCQ}?@ZSe_{2Hdpp+>hw`0{)VS5YF-(k9|H| zrL2l$42^~|{IfOviD{FZQZXSt%uN~J!39ry+_*-Lt#x}n^-txT!*7kriDl{~Vf|LW zN}_OvhE=EQ!L0ytbkid5<&9=J5+-xF}3-^16pVzQ!&T1~`V*}d`oNPahzb)bw5#CMDHMf~? ztyHe{p9rIrbfVDY_ck`OK0l73oqN;$K4C3?CZK(8-<%zhFoe@E6Fo3z&{v`%80>#q z*vYSU8gwE8n7vzyjn&^Mnl{2G@Y~V93!jR@S5X)5ARp!A zwsh%e+(K-byuCayS8PMMf+l8*Y0);*3b8_<6^E{3L6I! zRD>BHcN7^hYz9(%;K`NLdeM{IFQMal_d-Jh4sw9NocQU%*E(C*^6*e78r=r@P5b5~ zEKxGZEVXy3Rh>7_jqp_8QKlJe{aHG_MH`76R(h6R^$ke30VL`H!XK5OYf-v*@I=;L zbt|>ujT6cxrNKt^y+K;QsQrvQClIs+>{{EUj#>c(bg}(eFxh@Y8+iva#n3LaU4$0fV~`Limk2)y)osq6a6yO)hGeU;HuR}*4}h`RZ=rf zwQRFJMRfHs!cwPrkPW4|6HGdK#7v=pnMAxJQ}A(^48t>Jbq0`_kb4FKH{I6&nD8|% z{(v$vlEH4Sn{&XcwR?cLj(W@k0C46r+R%mSw{dkGG1Y~IE%sfpRMzRQCsP|t*%&nK zjbHWq=bgV*S;+3q9YZ;1L=fApt)w+nz;u7O+<=EUwd+k~ld_v4M~bIQRLKGtwrSHq z3b)|Ka$gocy=I&&U^Efovs+$u{R_IYsa;!t!?v<`vlw{s=Ll`7y+9x;e*HhP^brpR z&{I1Ok|;-cW#uSVW&|)rtu{i!p6`c`d4JguCC*=H(Gd#$cCX6%Zse_1$lSHqT4DQ( zTfX;t`C{Hu=nPMzZ&UOjm+_rK9JrybY8vgGZ8o@RH(`Ne)U{0?HXW+6GE3Z&Cf?+@ z6>U}L&B9KJ zOh<-4c^=XT>y;HXWjsa>FMWkV&3~cP=T`3&Q^Xr}X)t^0?l>b7M#>pXrPg`COo|rs zDZbr;$5xnjElzX22-74S9C^60mtU;yw7KA>maDqT-M^SL*j6pLSijzfKDQCQ`!r0i zR}|AtI%RxVVLdx$V>PT9bs8~HI2~fFH8;A5*^SnD05ZuiUc5_zpCCBwdpON;n`Q+% z3Ea25(a0W!VZYQh-L+6UpV2Tk>ztKe!QYV}!^7RHEo)1!WciC(C$f0oBFa-fi1|Bl z29i~ZIB&kmFSJi`=KTOuY!y{8aQ}j&a;mXP+jU>^ZWafSZe)sSf;g7h-4elzd~ zu9}e&3|buoGghqvE#|W~XSQijS%sVt0{*zGQ{L{v9@>Zg#8t?u-NJj8$2;O)YiJqb z@<9$Bs)iffOc~+H&ZO)S+-_}r@+gFoU&T{iRm7l6a~c^w?=`q2dwGstsTUMz3OY*qKoqmXx)(vZ#3B62AOip9G6Yyv&KY{Vl%aASLZga&LEbn-TybjfL z)9e16B7wMBhp8weRdD#6@Pz6{nPK@dSJ-&vO;>Mi`a{J!cHZtLcIiS08PUS}uU>bUY| zR6F`79@5qE3|A6sAUTCz8Z%?!T5S0JakS}!GDTc5VOMf_C~9`Q%lrlu5~@=vYlad1 zm)2~__Hp~kGewJTEJnYy7gv^NqN*``9j1LWdaoaF){|>o*LeCwyyt@-p8`m^GJ*Sp zs;h&-%iz7N>{3-V1`E?&ZK2hE8hM#jS>s*WuDL z5dlW*_6WbG?3r}RNv!>ljX&&^ETSBhSnR045S8d$+ivlnQ5F1Y z^w+@0iqP|p?NHK(ghuQ6su(-X1<$M5#(Azm1Bgp!6m3D{!e+5?asIDD*rlvSPC<^4-MQh_K_s$Mgz zTa~_cb~iWl(;5(zGZ&DmcS?Hh?{~Aqg9R;OsY&esIXooBw$9L?Ojbs2P3moYRGR{( z3VFaU-(?#*YW0uv%+goltRASam&@QbdNm1R>L;NsO9BaRyMoS}%!3)24z6G$J^!zvMr>H!TI6BJmQarwWR4aTl%V2rH;AEBa3H**EcDw{{5Zw>& z>UR3k5bI_U?$GzS3F%#tJg%>WHYXi5M(eW$qqOxGt7^~)FI0kh+sqR6PC4#zaZ(JL zqRpSXPujbi1XZ5KOFH*0kI{qC7@P-wG)IxgTtDG*JtX8hlQ^jEL*wu3u8ps%%K?J& zs&;KeoR4(S{7?{1Pf6YF2tJ+V+Wt_pYVekv@l(`1cWsKjIdPqPW8IDjrgt661<_-n%eaZi z=!1bd5b@Bl04y=s18}hPI*wAXPY-v%vsrhwz_A6FJo5|qn6^4H8m+vSjq)QYhlk$| z?)$`U&0~!9Kp30jJGc##l@GkPEt*e(87g}yp`wFWP2@#4qn<48!}|QGw)nS0 zml$>?=--0MAg1)2-xlUA+#|*oFB(feH29Wc?Ip#Fcc@fLQ`amhXDB!fx?;+KOW57b zB?cZT0c8;{4?#!5diyEq4np(Bb&kf;k&N7nphEvFsJg}1+qqk%9OXjb5`gXK);4$U zC#7tB@%5HMIX$zJolqg%Id0k(Z(0FymE;qzd%J>EEY;- z?+!NPu61dyE(lKL;IkYRz(f}KUA9BJuLivQvmVwl>l?_e_8cOiyu=D%e>Ph08m%Od zikg5NX8)H|iw84z3d2PHOrpT*bP|ZGt_I-iSbWGfcOt|}(p*MbXy`*uKTU`vB^O9C zl1Tg@gRm+rCE+wtvFyLJjT&6yCC2qYlJ?Ml_g0Z$qfL# z#SkUCGaGJfARp7Sf#rXa1wvyL#96cLs-z{_=Dj1l9&CQ7kqzW8xsovMJyYP}{COB4 z%KJXht&JkL9$3k_AI4SHkQHPJVV{h~#naqj#Ub1O(u0Y{<^qhjC6$QHj%LRG*Rwc7MCnZexHQ@mg3FEs%?Lucb)cgZ|3rhfRwZ<>)sxHg=CxArzFAl~oILpX113?sLBG!}E$W zsbw~a#oD7Zhry@z>WD@*v6o=AKeADcRRGa27&KhONN;Xzq-?LvKv7nhp<){~_O@B@ z+reV9xi;R>*3`|#!LYt;l_018F{rSdiSa|oX5Mk~r&QRFv_UJ?!23U|upF`NF2n-f|kKJPU1(R@faPCcIRzcH9vF^rtR+RT!Kl>BRSGN0w&5n=E%*^UiR5f#u&7aVq?MS9 z(MsdO*%7*{2&Ay3SR{sJ4>7?8opTQA-oJpueToZ#EPjNegg`TM6-AvRNUb90fV=mLC!-eNqUvo~&_5HLpk=~D$QEv-mlH&;koxGVK$<5!+n zy<7$&28`=7!AbLY`wEaxoB125A0rU{@5{c?xik4Pj33G@c=j$A;<8#tVj)h3of&WYm&45M4lPRsc<&zP8q0vxd!Gn!I@#2wH2 zUFJ(r(*17N31CPM8JbPiVpElY3VREl;r@QS+&W(6*sv#+X87ME1_Fz=ezf2wojBjy zf2kEOMXz(#hv*4z&1&hR>Hz5q%mUI~>kt->vF%eqBi&v@B2^>{Fp#p*w~D1D^S=aO z@p@t*#ZQ-ZM_CSA6HjGmNS914p*<-Jx|DG9Y;J5O-)PpJV$kUhVZO`5Mg8h8_=*=1 zLooODPM=9lLOkstgY^CELnr~G8S42`!k=X+$3#sJfIZB9Gt~XK)~q~BECk)w#^bV4E?L+i_ST}Fydy<8tCCkfg7iho(xkGwlhk0pG<)?IO^3OV z`57zOMmtl1a=PsT(T+sU?h^U~VQQ~!r99Oa^7)~u#DXC+`$Hoq=gHAz3&GW=O-R%IxiR6FGrpfSrKm(*%N*#< zw7kh^TW|MKglP>GPt5qXdW>Z79r;7m_O!^+Y3? zXS;iSVJi0>s5+83G&FOTJkP$fuV=F8eObt+m0U*v{>E1_NRlIr@-VgV+lO<9~GjBo#r$_z5DDk$~F{VqscNFX}giGz~Qd4e8ir zyO@i4Y+67;-}#jrC;|d zu%n+3OefdoWU>3BCYdJ*VaUvh(te=cP>={2K(pmle)011Dwx|c!Z)}GQz4|Ot;{JPk zjt`l3j(QcpiXV`Y3F4^G2jAC(w}uXRT`(qx4C6q~Oi?sd@h8k{SekY`V1V{KCs$=s zBc&K9uvxAK5smy|Qz@#fR*J#a_CB=aNRW(DzL~O?rO{4b|3Uh|0MG659Jkk^kd^?$ zkT*pU2%Om`%6S!BIW=w%)hQnp_S@}?s=9nDGeS*=nFuMEa?()3A13kBB@~^>F7o&l zcDVS(w>zk#CS0(i{=>#DRLKTL=g)r=;Cv>LO%744% z$UIf+dnHrx9<}<>zAP}X1ZZdXP{`D23581{HkaB`@iqc@>d{G4R;ra*=@tJ4Y97&z zgfN?r>4srM*u;t9YTkoeRXkh5g)@-&56i1s4tKn(g|LLA#mhM1^_ItMcsYuQw$lCKU$;i>p*IFDmu|J&b@n(-ZEPJG7shXLDscww*A0YaOx% zF}6xPiwwV@E$f_*uyZi(5a!L~sZ7@f^;IV6eC-MQQSF>zoyciz1| zJecp0-7dolZ@lHNxNAyDG1k6#*k(Lty1!48iIwir=dqv3=}$e}DidxeM2vrXddMNT zJ>)yb1ZpMgj4!(So^#3{Ix`1o%`G@Gp<2T{do5FV?+s4%gPi)y6ujnGdm?v_ekEeh zq4Mi?$b9wh>(R=WF%DIC$F@3?+ya$&jYc7203-4dh~>tge34<^(kGnjnDZhEJ;b+P z6R}i*X|x#}4tROObCF5zr3}lUBdKG6z8*~z38h3C1Zop{*6{<7hJMXl; zC;0SJ|MFiYe?&Z6gnj9C0uh0&u&T4F=;dYy6LkM(K*<|H#0sJy(?QVouWIr#l8c{x z&BC9(Y#3B_{2ix(eESG6&_^Q+tbxTYJK!Vb?y?g#5wYmETJMYyn+05K+gKlTyB5w0 zY?S+_Vqn4{t!LVMKo`)??k=i8uMyP!sro>+6^qZA>>N#7^GHcfv{$-2RRH=!t>(!; zz6SWZO(cj~rF3<5EnFmc2+T(JzQkVOZnL~RawoFiXqIk!$=-mDWR?pADY9jrZh_o( zpS*EH#s2Q5fiM-jCTP5g9|_G7K$+PK%L7lqrk+J#gPROC_GCWUGA}3^!7*eMqS;j}k4@f*D3nFm3D9Nz9(rn7Pph#10eYKCSw{tkAL^CS4LSKUr_5cl; zoW*8?-BA|R5z=HNBq`h`efTLIc|>rBFft)8mtEXhkTiMg*V3;U>;*JBq5W-8Sf7&! z%-q#S%@46$IH){$F39Up1-ZB5A;JQAPshXk>i^9TtlSQ~ zTwK9NDizEk#-_Uz++O@@pwK`TZ{Xss>6~n!2{bZp2-qz&^W3|mQO=d4UdC~BUKn7) znt^M1c}JZ&qeg&BpbD0TpYM^ca2xU_8F@f90R|g*l;)3^^7zbndmsO0HptQiTK(9N z0`DJ&4(fQ+o)4z1#4Il6$i#vD#qTTr0sp?w z0MyfkhQifCq38Z>sQ<@4U&1&2#K&;y;N6u!vw(G;1zP0KczqiGLn@{}X;N5r4)4tX zR=EqgRqk`t2*jR&vi;Ek#arXoBGsQTz7m(#CVTXVKOf!deI|kROMx7}7(4Ml->^-& zL_$KMCyYQKfR`&WxV2#@K@{+^BFaslFn;aN*#O65etup;GW5<5yjOP2G)em4e4c@8 zCx4`dv1`RDWxs;xT2=F9xo_w2xV?PjXwm?k2+{oiaA2x!(;fjEDDYJ{i|Wq<%rgn5 znrCx4hlqzZo4I0{!vr?zzytb8#3-schgZ?6iXdvR?gPHS-%_&>)76Bkbi4&$kj_Bh zX9qUs8gLQbez{7|7tZr=MIC3ok!g*ZGGXVR=G-0GN<(T-uA$e9`m-e^!zyROJ515# zoWM`mM9mZ5ZYf=#=qY^rh%j|7_~)VG{Jc!Y?)pk!!I}gcDkfVK9SVHlnRKjmhWD^LS-Y5{vMC}Ug1W|xk<$)6p|bwF_1*KD(Je=83^EG#(; zGu?q)y{tI5`*gVD$+QVBN_G#A6cJhfs}sM&OmBgQOk#U`+6eMOef$i5&s8qlD~rbAL5Jt zLkLXG1LLl34cIcwTdj4GGrp8(e}RU*6!NC3lS|+$FhiEMD_H)UK)iK-_&f~kfW&3z zN99YAw%l=0RAnyuFE?eEYMUR3+j$zV)*>*uW!Djc_9&7#xy7RmgvH-jBUQcuGbB1J z;C)o_q3>%R_At$6`zDp%c!dw7jk&DGUe|X#As&dSIAGkk^N!Gppqfg`z4yK=i|sJ^pP4p&%G!V7*E%|%KQaQM*7 zmCzkMd!$q#o(Go+hlJiy`LYj1G>*d-;%y)U5(q$QgQUsMpVtM%>&~~C6&O5nRdTLJ zL4kkK?<9;<5lGx>XGhiiH)6&HAe<&?QN69tQ)Mw)jRfKZA0Nrdgbd@N$TlK|`mwRs zfO*5MYv`rt#sZD)*E*zN_h=d{^CoO6{_u$V4>5|6Et)wp*JTf++u=daot04o;hnC4 zZjk3Kv?YYyD*|rE964AGGA-SPDxsnto6J@7vbm$UPI;1rDkOgM<+meGRqSqK5Cw3P?Xh8r#?)ryc1( zv#~iVGn5l&lAnOJ<{W9U-osLXH7U0qu!zXtS~wZ3_1bg_@+)u3()L-{r>8((197`D z(Qp>#o&ieGdXe{t^*pi6=Y~C}TEal=F%|Mk9KG4(&;A{lY;t?m9)}H3gGIkak-UL{ zt8nKeGO`3xmQYPhq&Ys8eq@M??ijLO56dZNifyD+07b}H>Bl-ZA;l&TWuhpy0 zbc%g8FMkhsM8*}`8~nJ*H?kblNe2@#`CD1CcA3^gaYR^760pwkLoV|nt1?4!6zeN0 z==-WImGu;7VFTZ4q%;y4+<{0Lv&S46>;3L%KVCap5KYwdN_O zv0mQZC_tEvb-xk9oOi=3sxkB65bUTL$LF=7@Vl_GRc`fMkX53LUbac`n z#0k`q#M<;t*g#K4+NV(r!E4g1n}o122kvvDXMxp(42Af*Haf~)LV_5fK)pS4QiLWF z=qIpzLn6mo^Zr~7wD8h{%^_<~X!E2%Bnw1tjc-+-BBs0n3nw35Q*i1}3_h^ELC`5a zy_s%DR2vP2a3>D#;^TCkv=@=&pg+>E`cK&Q?QCbldSTrDohaAC-O;$|j<`4wY^J5o zedrniGdyuIU30kxJedthRSF8_KhZbgllHbok1N85lC& zajjA9gZ=K)EQQY;kN^p5miq!R+DC<+2LdJ_vBqVy7~0s_*6P=(M#OF}WU5c1tC;C`OB zjQ2a^9pj91)*l5@RD)p+D^gGw6Td_#b|F z*E=?{wB)`9Jq)y4;K~5d0cuj!q2F{JPZS*$C%7x~)q@bTMn^+KGf1uVj7-6vbivff zu|GQss!)gaMDg8{6UxH7m;EUyoOg?0s0xZ>DwctqIOf7WK%Wu z1iIcrAoku-oC|`3oSYnbj(W|&Z>jYZ8f4iNvp?l16_5C5=R-ltc?-4Lad#ElNv(Cd z<{^-GXbwMICY)pj08^-Df!?4iXSde{A%E|X{tIv+dz%kLJSMt-F9QEJKfFDIVh6+q z2lV<~8sI#fFf|AV$_{z2iWFh%L4fLQ012H-0 zpJv1+3POj3I0KJvK83EhVlGtv%6hTsR8xz1G)wdE*0Th@6120cm)AAw;;RzHAAnva zG~?gu$akjRpI(D;{ErY+9#H@2z}|_0YYQ;z3~;^Xufcxwj(-j7K?V_X^Zs50MR@== z#rGsAZ?0q}zpkvT{OORH604y={tdDg1k3w}O;X5j$GXyQ&Y|_q+cP?42&ikpJAgg` zN4`)~b*URBm7Jc*1cCd&awrL+kZefDU_HdOVYf^$eDd)%^k{LeP&Uf>J1~Zj1bb>T zwnrIA_t!r@RHR_PPq9dOLqqgDY-Z?BTNHJZ77t>clor<_8H~(cg2mt6QUc%HyUR~d z|DpeEgZi2}lMOU7DCWR<&Bc%8|gZEK+ia-&57FhT$)OAjLT`{}`dP{21J#zNiM8c2F{GBA<; z(7Ju`Q2<2;7wzcr*xfh#GiU2rKo?TMp3K*xT&w zNYIn+HY(Jb^8dkl0RaBO++4PXu)$}54uPddB@d{aSb|+@0`%g$y$yfN)cXjbJHuse znx>|vbI_KA{@8UqeA0{%Od)AkL73N2HQaX|kJmsT5WC?6qW)&i72E<>?JcB9pV^Gj zH>19+Mlm&fc;*|l(7`m`{=d+Hf*ODtgF*WV`gxEqp)ViWUm-qSpyw^nd0k&$zsLx6 zG_jzwfj-VI+XT!gex}O73+F(Io)N>ZzZ=u2WN7|`X^qDP&S%gi3eoEa;kA)G0@_pR z%l^Sq!6N(9#*6Ll(q8`m#6yjZfD(%*`5^UWp!^AfE_zO^Q&x<66{sz&K3_|sxD&OS zn`7bEy9k}2{UJ6pzxzX9tW+gN4o{jv5C-se>BzuPcGf}N1TBTe<${`;8l6}WM8UQY zoc^cmfj&kElM6(_)|v3@UELqT^odG{LC*unf7g=k{$;N}{lEWk=h!G%`!>`D&Yl-6 zO7VY(0nR~9Fn^4-ySoTH6nvtOyTu+jX>ah*607E@V!O^%=Kz3rI83+3?5-eP4-|jX zn&k`RQGornNeWHv;VMvE%`LBmvK#f}Tf-+!8|Nk=wA0JWt6^_n?B$_=e;89(h$<(^ zZ7xzWNDHwuUF8bWA}*si+?SqJAl@CoSfI8r+zpp{nVqt$Kfr|+EAbxAovXms>LXyV zAD?!ye0&4Fuo{G1Wa#g9#ISegpk>w?!{37+sVbTS*8sS#cu{D~HHBSAuYT}P4O;2$ zopasZJ}1sH##3m$4A%}0Hn05s6YYRLzzKSrw;!N1dVm1bUq8Wx|AW=AuLGX6CfI-; zFveFOh*U2(B|W##+`{F3axAwTv3XOf9oaRa(=0fOsl@R*Ofq$TEQt%ej2Q1@|M~io z9S8?enotgR65mr+bX`$7j*3aNS-q>EW{x7KW)#rCLMFA18utS7yCS8SvP*YIlcc+y z`6-Rd0b4E2Ii0F1qjyN_wFw7(aVpN7w4CwoXV9>3)ulxytFk2h+0c;90 z?jCnusrz_CF`y^r{FLbMAkOv-Wzt;I{Se_|R-7eFTwm9X!XD~rmJdP@tMiS6U;n)J z$hWzOA~uMf>!@`fU+lPPh!u_fpe_vSa{g0nEA}9kONr>uMmmTHNB>Br=`sN?i59yu zSJLf(-`#GRrV5ypC$%e`Dt>}F-_)5=t=Pl4dWg?P>Oy_%zg981dPxyu{X=)<-?~;4 z#T>LQD6KWI*JcH-)kX?0d|#Vxt6ui>)k$NElVkcqCet7s`p59gM4X$6t}rHU!!Vf= z?j5Jk!I^r^k2T*p4Y3Ay7+y=$i8g(&fxq%vf+WrQ12+-2;?s_4HF`>5j=J-qdU%>?`ER8^J@ccc=d(@{kOfr9NZy%f1Fv zyN|{UC|xE;e!iz;1qOe!?~k~LCNl)d(Ie0i7QZ~*UWCs0SXk##@Wk~@Hc84-e81J^@|IU_dt>R;Urk%&@12u~_0fcbTO`MR z@3GkuX6(gBzKnqasl%VGzvKY!c{Ta)YL=SWl89U zd1IHMj#0!N7xEK#J=!v8KR8@{T;*ezmd!ANcDt(+Ci5zMUkzp>((17l6E0Y1AQwXs z9u4XgU1g<-te|T_m-)2k462_HIT{@_IouqI51;s@;iHz@mQ6o0jTK${RL)s8QX@}4 zB!2Q?ZQc#b5h63rHJTT87$y?)H$N1gB87|ZUXTNgVL0U}l(EhL0YO`wux13Cutxid zHVi;1*D#KtEJ~%=f+qMO$lKu*AE+)q470e3@t;7l>*X&H%u|I-N$Cx%H3B9v`X2D* zMo!!O+Y2l=ZT#E1=i#b+InxCdGdW&gGlOf-oE`SQsK&PNtgFvgB1qCHMfw%pvqo+= zn<@|nim;3md5DcuKSqb&+QcU!p1JVpeWEAstIzJ$ zPhtO6!aLlzRX233pAyu*N>?(SQA8n*R}3^MV3risy-_A-Gqb>*&uwq&ykyUII%HCd zV5}tdiq~ov?a+xH-wD(l5+5j^AMs6N5RKQ3h3_HdMq@kNjk@&Ag=yOlX4?C2Ju!BZ%4m0_ZD$p}ZuF%<-r5&s>CZ{pI3^x#pBdmh zi(hv%`qW1VULIi1ub!F}O|_@+9vPVq|EtB>n6f-uqe>^tF#Tkp)N-~UON-^3b3zPR z`02`-C)44>r~B_948B}0RfY)akI<>>qWE{}%XgxiVX$j`Kx;A@Sme`nnW6QZ+jQ0Q z8@_*eTY}92l`dTd#TNMt1=D)@(xv-NbWCy`fKA9<^-tw4!_j(G5hCcix^9oWd}H%` zLp|E^RQTjTYzEL!a0~$5nQgu~FMX7_@!7Nm1v0!jbZCBEfp3{5cX>Lain3ghH;WMetDi@SsA+}U8Pu(II zU$Mp6EYLh?X0LH#u8i>BRpNl6d^tSaL+n@y%W%8v==S5^PmEo&YS?ggp2Yfsta82I zF;xW3{G3yFUML!iF0HjUbLM4x#pNPVlEdWaX(j$4x)JmtTt*_&zI6QC*7TH%N3`mP zC0>}KSWKT|X;oE~bhSnOBU)J9Z{>2K{T@WyVIQBuu90QEZa_(21IwL-B61O+#XzDJ zQV7mLg^u6t!kL~%y$tXS;5Xsk4z(+t|o=flh8%oii234fln_1E3J^Jjk_Md&DQm zp72G*5FmW06%C5X-Lf#ci~>wSL3L!4lr^#l(i5MPZ(^>SSW>Ot@AhL1YPs`6CZ&-n zQFK)1m-j~~x`+#6kz%sa9GtSbOY*%~**j!`izV0ywf zIzWjK2oxW*;?f$=3${QR8HZL&xNf_Oqgd*lUmE4%LUChTggluwSGv0!;CDf|W>B$id zSY>HpQVCfQ6Liu^XVaAr!dwB0yBLlh4j0OI$~Iyi&sF89$;=os(NS*8Em0o+DXSP; zJ1U@GKyq(iH89Up04^YfgINp0C8_06 z`i$kZSB(UC*CaXoXA8P3+8h11-@pmZ<(!Q!;rAwOb%8GA%}W3x)ZKQ?7?%MUE;x+- z3N3%%V662)qKLSvydltFs+I4;rFC7952wxoli?tf5p$aO++)$x@3~AxMI?bDrg7{( zYt-<@NMh#?Yf400>&*yaGiqZ~_}5Cp;q}bReZ%&(R@Z}T#_RL0Vsx7LSL70(9AnQH zi}l4+`$`aH_X9A8iY*Y$kbD{N?G}4_rhb4~{iM>v5Zm}x_>oV856~Ap^$>=x#yc|y7O-dcj(1Fe z3yzSmtFVVdoj8mtQ4N5@b9?Mk{i)dVu0D#2M?#=G#H`E#L?9zIH5GyYfSL)s`#&7T zKjLY7>%A`fARl;3`B8Bo5gr9CdYl9#$JvD_Lpp$(u&xD=R3R1js2de}L2%5{{%Ll= z4jY*a@1~$Wk$f++k4no;iDVG2R0S zph;`e*@FiU?lNn|Ij4prR4;o13On2vz-X-pId9gaL$BI3&lTEtd(sT-hoIf=S<6EW z`GczFrK&fu|FrA=zuI;cV7QOFe}|q=J-QA9Eta>huYM*QxUSHzsJ*m47;Ew#`Zv`W zAQV5IT*TBWEQHAbaHm6ovYmKr7eaxuUfA(-cN9p2orQDjwG^@g&^#T6ojDDOgGKtM zm|*d0P#a9Y2Lg8)g4iu}8oq-UYN9tM&vlUQ1*3&XC)|zZ4Jm0oQXF#^c#~ zC6RxhNEtvVE%+cZ8K6{0WoInGgEbC@F>q?sZlRHd7!HtDXzeCZET8p{E%SlJ}IsDH=jZ{vO3turHp9&s6RG^RuLZFW%_FY>?^73 z$}ldYos@w2?8C6c``|+w+Px6#km3fPVc`y}i)S>5gaeNUG+FzOI6ruQ-pn3N1v&UP zh2(Gw!}@|bMH+Oh08Y-=*0_ENS~no%F8Zp45H$5R7otEQE&1LNqH)h?so-BFH~95#Yz_#>>DhbEp1j`|UxBq{XQg>XCja4KKn^nX z>Q}<=9lX>pxm7DxgelxsUED3BXsg2Rn~+q{u*tGxqq4{~vwD*+4A; zO&R=)FZ>7a?>b10eK_HrXh!OH4DBPne^GG%lgw;(PXPr>z6UXvz<^v0Xlb)+YpHm{ zK3JDD#OP2Dsrr9keK5ey0xElE z@0KNjyXpSlb=IdKMjXgFK#iSP>Po&%tNzWuYcV7t%H&E5X` z_3I!MqQPNg3O=~T?Mrh6LT6BuqP6@nSlze(iY32$1KopcFkV0N1$7(#_iF@j%KZR& z00d12e-28}aAmajxvlZ0a8!Sfro0|P{_2X}P%WkX8mL6|l!eXl1*TO!O_3ONNfQbj zs75A>z*x~@_Z7g}3xnHMfM7KIhgA$oF!HYV=T(;JrOOpGDHjnm@=}rJV0 z2h#!Miho0nE@z(Jn+T%Z`u2JbdWLJGWbHM>Cu29Vnvo9OD*)&V^QDp9CflS75t|R( zKzoM9HrjM5IR{e|&4sDJL*eFad=mY^f5gYWp>v?gUgStzImeJ*j?6KAcCVF zYKw1aVO{&fS;pS4jFeWs)DddXZnPL+(eDM&<8R%gDaxNpvU>Z>>Fku<7n(XTuXx1D zoqX340CYz#Rq`%p%04wlio{4sNJ(_>s6MIQB2z%OeOAm(v@1U1r3u2RKQD2FcHh^f zS;-oN+HAq)4xDv!ab=J`U)0by+kuf$Bjw3Qb9Zhj8 z{l;hCl=JY?LsbxTK>5)#Lq1fOIX&NUNvPr8LZ7Kw3h#bv^HhtP{9&_v4Yr%U7jAeo zP6v<2cJ`t;>&u;xjkChc{OsgV`KQldq$z(A841n%dat`OFX90 zz4DvBY}8#X2~C$MX!fAk(iHI>@7K5@zqZWo$N^^32D(NF=;Ay17|O(#D@O2^ZY$aD z9ZlxSHkQY)2TOM*yAjbR%s5) zR52eIPiFuXJhef^y*o_>e1UL91Hue5^ns`uFpOaE*ZTbdCxp9TnIrlO*eMf6VA!YK z*paW%WjLsK$TT$j;K8U;qmfD?I?+ZsWT8Sb%Ecj3SZ3;B{-wLNeOumzV+KWy%N094 zwZ3Xu`9YKqC(XZ{F?X#bai&Ea@qPRlUU(0)Lw4)?klIu-jDk`GvAZ_~ir|EcD!| zm!N3?7yFOO+f;tz6sUKZ05YspH7M)Hb|DYNWvs?#oM5&*T#|3i@Z_$Zb=j9sRtstZOxr?TE#X#GVPOm(WeDZhQ#?Bq9|c6a`Ekh+e|yex z$K|U_j(8#Fb+i%Y5;{>mv2kUky>hU489;smR44a&9aSI}Vp{=mK=~`B{;Lc{Vmngx4(9(d|oW+KUg$J!dLU>cQ)v956Vhvas!*Fv6~EXvFdRM zvYhU9RgZi^-eTwX@3+>B0?p-jzIe|Sw)JOxdb6m1+6=ikyamU1W=zJL!Exd-gt->< zvR%eRyz5}0w^5p0*@HZ*w67WJ!n(D>3j}`=#6l;hMt;`DNIYJgV`a>vd^O70ct_e^ z+?cPHkQy*w;T%Am*JV0b=Mc3A*%Jtwpu+xfkh5+6F+EJ=m2PgwU59N7DWe|LT;Cs- z{VcuFSfz<r$+4O-tXjJ%-pL8-oS6Esio?$6Lw!+Yi22OI9A8ZP5dJA$i>XJWjrhaj z{|cZd-g)9Nx*}08rd6AqwZ=vYDAebng&7%p{Bp8*N2V;5L1Ps_zvLVMifvG6H!OX> z#p;z1m81|%{>W0ob<`VJ4`iyVHV7+otM)NPFBDYsjQBSTX1|%f5NYY2WhVNfG^_o_ zwZ;V@zL(-hMwKn_^<m8v_m}2kn<dQk?-w!{uSF>8)f6P%&7otvU&F583uk?u!-gCo z57EqZO31$2SuIJEo!QpF{?gQ^HJW5>zIY=DccL`l`lG5I6mRuTbZ$<wh|%|%4zo(B zcAX%z8R=`f!J1~(TaT{r|8~iivGquyA5Agvz)fK-de$W}+k}l5&L*G#N3df$Lmj{d z!jRu?6kP2&bNjgCrLekQh%>mNAf_e4zu#t0av-o+0)j#AUwL%4>dZ$XEc#d$gmO!7 zM)no2D=gVp5@Xq;Hl0UHOA}&xEo)Z*dEoYkJea1SFj@#R&b%ah3MXQ5c<$-poQ_5~ zJ%5^;4x{i5KjJWZTYK&ai9$fY#&{CdB+g-R5oMS+J5!@FIc?89{V0#kIgw8#pIbJ5 z9RM~?(>s7hbCa6cXoNE^RTT?)(ZNKtL31w-B!B{epx}z^)#r$ajYN{YM5<9Xs0XIB z@1Ck2S38vm#zS;#;ih^I`IY=GJvx*tTw}3Kmi1fjT{RTXq<iy?l)~3w=ph?kpB})t z1KpU~<^@r4+Q9U+w0fxiN|!0EJW}RCvb^4PAx*u458GSGd~e?#`fD;#w(8>%ui&LL zdGqQ%tKkXhXq#qm>#DWY;`9fR&Au~uIL9qE{ZW3s!H7;ZZ+s^IBe`Jmk4Zy&0ZKth z#6VfrFA22W1I~&-7d@5-rI<L!F*h^m?3@^7cdr~LS$yt~d^SGpm2bXsd|n#ZQKjo3 zS0-K^)t$eUUI)4~rp9*$++drh9;R`RGh0a)@M8;qi1e;S7=E=G30ScE+2?}Pqj7_R zDpbD~POI93fCV7ajz@Nkqi`Ul`MXHzK&2%e4yrr*%nCEEBF+!TvfB<k6<@oL&SB8I z>GXx6JE-iGbk+D4202KPa_y9>buc0+h<rw^41QQ>E0fIWY?KCqmzKA~|AjW$aQQdd zBwoQeDElgTbp%cRzDD^jp&a6l){f2ww{JNsISFWNcOBYLo+C$;>aGmI<w0aNw8yWs z316S_1Y(kB%;@F$Mp<m&yV{is^0{-B^!~;_vz6krn3V0og30g95JsMTJVEs8ICL;a zcQoV3rN#&E&lp_hS7X7ctvbow$m(Zl@YM1aD&;i-w-Bk{>MPwu**veqpgEmf?+#a- z!1`ri7yW94J<?<nl-eeiL-3y3dUTOp`k^NnI8!BYaQf-z@Ed2>>DE<X5<p}!!HjZp z$-D>)VP!@S^ISFbE}}D+TUnA5kLiBTXJnQyI21dxuW*}aTk&8d%qg{b%aZ)1gYI>^ z3c`S}bEqb@WV~G&Vs<#kOMg|iE*GQ;%X6+yuPU8)z4~Rx*bRsdoW3LvVEm@it5mQO zY}jQ@e09-y5Odj;8<9zpop?X07F$fr%zxf~$*Sj7t9%9OiSx4)Kb6v}k|l~JHA-J< z7Aal&RxT0FGLdoee#J)Ms64R-Icm5XWD?z-A+B`U;i_;gDqmVIDcqfLD8i-XE4<NW zCf}9)IiX$&^F-tNdj5<xeO`O+w(uikk7{OQZNM@)+@%(E^JXRQV^q$Js!-KM`p;OT z_*MbNhcsfyjzbjDjW>}mcWl<Yra_%;Q$Qbp8T}nv9?mj);~&1su<^6LmX1c%_*0CN zWA5P2PR(!L^^t^~1#+|=f7kScKkbaI(I>%D$wy)S7DDvRF59l^=Rtv#7RFjzxU*wr zRL`nV>%SUnwU&ALhPT!wxA)|z&06;byFxy1?I#?~fPV`3=!n5+ly?}Ih3?z6CJABU zy`jSRC|qHUq5pu+ibU9Qkj|Dg$K+r6wJkU38GE;;mmUgOdlS(Mg^FK)79|-E^6wX7 z2SHdjG29!7U8U%qmG5?C_n9V33gnzE%35M$fvx%%v3`gl2aMmk$IZvBOvlZ}@@9q4 zd@|-Bp)<pZ!mg_sZpFHQ7>6f8Qw%)`F+^M4F1}SYJyC&D;N+5zw!yYN?_WhAF*|eN zR*cxCmdqX7riU3&Ch&P-nS$9`4dd@0eQzbtG<<g9`_0>tQR<84@wbx(%bU^)OnQR# zuCs2IV&2M|B7#fz<qxKkprlUP^s|A#f8z3taARldG05qHWm3UHL&O>|4mJ<pydKq8 zEW_XLV8Z_;FT2BoEY8}m5-4wUfvvG3G1mg;RVz&7kp4SiDa^QuCdQagfjuSe!$MBA z=9BCi&rSbU<A7#wQrAt)_l}!qJL%$CbGdg43#06FQPa2Rf2ojG8#366(zI}H(t>u; z*dWHXPC}dZ2<HTxx!;MlWOB8~P&3Hn1CC(5-rcgsUx1sqex0<~2u7HVPO%L$hiwNk zuM;L?aHzZoCcNf^hmK3KZv5xaXRP37N|GXTkfNG8jW5<SrPoSUYqYB6tx^+A7Jj?{ zXq9_I;AVkwp{gMlodi4M&=$}K*!F0h7ToW6VgG%$ylsm$PcFtYNjYbR)k>XkmpEHF z8eLA!ny0b~%jfcVvF$6Gt0audWN!a(URPkiDM$2!wk-$Z8xx$JK9Gfkuwj7zcaVOM z!28jUq*<eJT(_jqK}zS{D%+?%zOAbk?K{5U-<(%;mamkqp!|pjdX7t~UqpK8nV%Mn z<%~91OAEr~HW%Xs+Hhq^P0c>tdvILID*@Qa{IlG&5?V0s{j8Ef_`A(|p9190Ik0R7 zRxU&t7cQA0Vm?^fbBht|@AlI9O%s$i?6?^7!ejFqMGVxi*h>xak)@<*?uV&RjhLz& z@zXAvc65GS?vVh7oO`U9R+mTmxc{kaKY^98pI!x#_J+xaT=A&Y*r%(^#w@A_Dx1<j zF3-QvdhVFx5=}nKfbVsM=kY&Ms9*>@Ur>vUPOif8k$jXBqrl>#8))?K-@Y{~lq~kZ z3vNj%a!K`dmKoXJidg4j)}fGF3t}=c-wQt2@nQYmFI<dW=8V=kf7GKIRpS;+lx1n$ zOw1h{@v0GKdQP~dbv7+CRp{5_`6#CkB%Pg<2=^f%;8>3*+zm=Cbdf;`j;ww9FkZ7! zpzdWJK~#ws$xU7RMAIxF)_1zTRLYIb{2Ecd`FbuTm5soLB*3}p%h5@TqXwv2mVRxN z?R-B|Gs;i1<)&xjXD9S4lD)|X%+}0TCXUQ4&y)SLOTr8~W@+VrTZD2(EY#rcF68Pi zgp+*Nofxc#duKV$SXLE!>y37n;etuuFU_@Nrn;SqN3BLB^-N*{xj;=}Ok4+00=kE; zepKsv-<OWeC;<hI<?0B8G~@Vhfd3qVc@^j74FAY&Dr=YcVLQV3lK<(H##AM*DVTx+ zS?=$FGJ-uzx`}9@*hsh>lTg)(<Yy-aO8OCnl_<Krsq{|@8(%vmkD8y2rjt<qe9;_u zHE*=!1?VrDhhBXzZRl){fC=7U*C*q>(RHe36_=yB{X*<ecXUP%UcJEt9DQ=o$d2oo zwHc;!KM&`KiJ!b5**WS*dbJsA{V<ZwOiAQwj-i~F^*JUNUoBcCOL(kpdbTnD@b8*M zV%Fza{og}N?U8dC2L(fUb6nX-ETm-4=DBCM_OCWygd5#Ws|VufimlUQ1c&*CC6;}r zhkLS)VU>o>P&+(FzI<EoKuxSt)sE?i#d&Y-gJ!mH{A$-oH*E4Z@|6i~U_rom<@c91 z%@S@^N#<#c`(KP{BkYB0JQ;h#5s^4@4|2qqywfbvSo(zSP?JyN7bkK5>AI?Zg_k(& zJesXH^hyNwlV4i+DuPh$t9LUYsb>9epJR40Jsd%K_hF*=56dIrRxDVq8aEHq-ZuP` zCkwI-dc^es6wV_wH8bbiso~VH@d=~Kxz)4+y~-C0=R0h4EF!#FZ+b18aa2Yi!d<=- zBBS;3^o=Cr0>AdDm~BhBP0?^%el=%%?o<5x2j)6k5&o)CI>muY>2B7m$XaZ@0Ifc9 zCwc0AS(cE$rcJFSCZZp)>}$YLnOnQw9Kh{<YE^}$qdZLGB2wxU>1|bc;Z8YH;)q() zaycQ_S2x<t*$y~3r1={lwOCM6^2^U8`WRU|_~T}DmnN#?eG5svJ=|bPQ4hr{9Kt$W zMy7P9l-f`xo$sGGl&-uqcxIs-D)iv4Lg__%HKlT{8z&gw+A}|jkE9ctdWhWeZuX)j z2Vs4Y=iL6<`t2#6{x*zbN5#8}d%(l<LNe=xmvr|`*2DUB9&>PBKY6dkSVE=Fs`Oza z9b$UnIj-R8?G6qM9{uEq-J1gsS56o#+KFCd8?Kpeh_=O6%m31IUWnI!7BP0G_)OG^ z4mD#=m8{wNuxLFFem!01i+ykOBf`fF5al*KU}hAXDt3>lI3epQMy*t?&$B;8Qa?J_ zdHs!{f$fT;Zjs!&hfANiu~-SRtFp{bz|x#sN@}#@cZF5KtGMW5+zW)zQ~Zt4gRAH7 zFZL)s%+h<A_Bbpm^h%LW*0GMeL1~Gz1~cj3H#%&vi>v*br+gv_6H2R+ziPN$VkS@Z zyXlV_`AGM7bDljs(kRY>y~@^qp99wZ)mOZ}ihJSNLfI+ia}4N_82t86LGks-ya%JN z;Fezyu0hpU!RUylTJ19H3r!Pb$=bN-e#0GvaFA-m0{>I|ic6GDb4_zlZ*bVeXqrCt zia&UNWXcV(F^LX`6YrOn6Q<S^Cayp0oW|Mm9#LbRZk{6Mof#{2^eZwrAdFWh-d|l_ z@Vf0i+8kX>j67j33%c}{V!qk#f!(2_cNWgh0)XAD^HR3peFW>qi+tl;iWq-v8S|;A zb0_i|J1I)%DLv!W?89a{an&D(_IsenbA39obSxUBc*_G>C~t>nUCJB-5&rB4KJXaL zEQe3(9^F=F9&5?vc{0#3e(-+jKrO(T@sp}E!MM!F;z(7e5xUu_d@=6g72jY|oM9s> z;VGUAbsF~|!V><Wb<8^aek^Llsyh~SW}vY|&<MXdPcukNc7r>ws+Gy{y+=^o(((=( zt;|2A?0n_y%(05`M>I)C@p{EMG5>DOjcFumVdTXbOjESk%zjKam;N9FG55D+=il+O z$&%X3@+|w1DwvAxGXa$a;!fRO0D@)knBLbjvG&c$zs~on@Y!sW(|E@db&>Z*{0tWB zUUXosSp?auCA$~&gPl#k!lz4u{jM#o_f)br`=e`7(iYX8@**4t5oBa-=9J4*D-8Px ziL&aQ_9J7Ja6{`$yR*lLD5gR+R<b*gsf8vjSib_P^jQ`^qLqHNpu$oYt+E=0X-@r7 zkRR_mb&G#tqrmIYuF)8TU@C0XGyDYhrA52r9T3YbE+rMeLkgx42Cb-S2-rMgtw|V7 zYy5u5aQg@UD;wSJ#CvE*?F&CxU3Sj2&|JR2P<IxIcp}ndt@~W4#?NN0=#(P=Q*HSN zo6lVq;>sjngf#1*-#&<ZbjL%D{g8DNzt1tvzaQckF$RHCMMX|xX9Q1j*PR}Zp-Y~^ zN)6r7$T(HplB+lE*ty`A*;0y_``K7Dv_@B@)OE=yMkPd;?LubOXBJY4+w0Fiyp&6F zuU<P6pq}aN@}X0JPp2ZVIuw!5){N}7@;jr;*!C8anQ^3YdS5|`!ie}d$s=D)(u-|E zx|8N}qlh_&R8`=a_0(;WD7sP~$x++M^3}!vNl$r!<k`b#G<hDmZu&|Yb#M@T&*wB9 zW7|Lam76JTT{(iMHVoZaRrDY@tCTKnievd9cYdgEjnCp{a#p*ukb>~Cj8#UzV<0AW zr|I?YHbb*XQNnodb?Pu>?jR??T<3sM@}SpIrJ&y*I0119iGHBj8>x*`k6%NwdElYK zJf3NJpvojve8tC4yIn!csWF?GMuv#{9d0=YtX6I#t0i6sZ4Gs5pAKJ$vQX!+&5k?7 z8@?|*jrEZ@C@zZl+?p4N{g@v?8a}APbp?@o@pd!&;#DrBfC^iT-ns@90j!T+s0OZ0 z#yCw`=_8XooW9x<6ZQ8+#J*uPim;ee!kvo%HEVmPm9r{YG@SJvn>mUvcgo<`nu_4L z%)gykPB&jfy0f%J=D-<wlH1q0%IsqwHq9kE4|6H8HkWp<<1!V<{@$nU(o9`t+KQr! ztz4%3jEp;*H0jfa!;ZMTTBu$dD!7&~M|*C;G-&gZA<Ory@&tX%4}PFtj87<|jXp7K zJ)DZs58nrg?)$xHXYn=IhrjJ&h#G5h(N5ztN%AQr`H@zP<2X7VRhH70nzjlQix;ik zBwSY4ov?TV-SXT>gAGf_J{->B@}_1EE^HJ1d~sM(KA22svP>F${;1zW^I2=+AU4BZ zQl<v=+}NU-fJu_4%-yav*DmRF_1zV<K-M(CUS|*d2PAYj$sWQtpDX27dUp7m6aBZ- z6&@XVqf5j8bRXB4NYauzhkbkp%HMq=?kjip(Z;35$)2p4-tWt_xwcl;QVO<j!u-OA zbybKZCds*^ngD%?b1|Q}tXb2e=~BT6gC~5`CQdCk@fTwKoXg1ReKB-fN+DBtB7=2$ zlS*^hDc9$%fBRo5l--ulx|FjV8#U>2&gz`u20oi^EVArm<Y&X;)E5#tVuw%6G+!KQ z2wL+rZ+&!=@gP(c84e&>rXM1+j<5ChblM2|crvcF7<)_b|0*h;5-~qL0e?$aw^q56 z_nocZ8~BLnghVVjRial{x?E_-GmaeWiy(}O1(#ePnFS+P&S88nTAYdAIIf(#u(kTO z%oQ;^?)DLkFW9a>^ZQmhYO=YlbfLIBRl?bii+*jomWhiN?maDYcHabQ#J?JK((wmV zdV|b@(VV>3lnFY_UG>O#!(4J_K;x13Y1PYXw-wme<5cRa1T0LhOK8dyC7+P-oi*B8 z`%1gXVp0803M0CMwQjvS=8;ZZQMYxL7aShguUmKjLF=j2MsY3Oh<e`*4=&D)x^CdF z%9}X1t<0o5>@@X*hxW*Oo7uO2l`7}6Tif@Qs+MH>DV6i`p_IEWA#bo1hP&${o%~#s z+%s!V_NhxEq+K>OjWdrHfGpVjm0wu5U(00OdSZ3?*=Lg)9HUxzIDE548D^U^&1@4h zRI};6m4d03puZxw*}E#pj*75i!tzG7xq!yWxYW6=!-iIm`z0HecfY!1+3Zfc$|9k9 zee`;MXUv>}`Fe}Mc~+b)L#F?rQ%|)}QO@%iPZk%eOi#f+H+WztzH&0Q)qlrJ@3c@s zSb1qt9d2F1t{;y0^nJ04*sd!+-dAo`=olNbFk90g`ocyTKI5l=*Y;>fhjjqT&vMw( z5Ois4i-|83jZ8i;=E1j@d2`kusn+<oIa*5#_AU_8{F4Ufz7Mz0?3*ZWRvIWcbyYww zD`|0kR_U+m`1PSyBBQQMG}c$7WKUwq^iUKCrS{}RK)!Y_e}xU`MDIQI>^z;N#~v>6 ztmyD5LFtqB-}$bqckz9`j2tS>`gp{%am*qVZ+p_nK!?r}CEw;4T*R^vZY8SQHewl5 z^{L3-<9d%vV)^S|8<V79(o|^k<p!?ino>UccS>~^8?JGZ3?|n~7<s~p(>R;eckMp` zllZ#>)bTZQPvVZmRNDHud=o!AoEFw&G^i?MoOB?5LMAn>%|xfa5)`eeT~5ITIt1;Q zNfEa3jELw!O|t>7;!E>Nn3U(qh&4mQCnzt!Q4be5?f6PxtH-&ieW<LZQy9W?=CAqw z{4-gpr!d2^Qm5KlZYKM;%_TfL;sF}_(|NTjtM9A>GqO$<M%~E~a}Yg8N-j%?tn>K* zvtI0hN~-4Kq(}+WCc-g;W!|+Gfgsj!tg09gCL)Ap<L7;t9QyqFuVh~&lU0Rl3b)1} zd<@K$M!}CARSOO&(KYoOe*$Sg;NPwR9quJQ(I<31&hg4sefD+<{H26kw)>A|zg>HC z#QKe4z?z#5jkv$NQzev;y(>@smSWW7bVxrVs2IlU;9&SGlcD#3X5y7c65T%H4N{D1 z3F<^7$f{VLH9{(!!+xV;+vt0ABOr+E3UR^fWx)w9@KEFMr*|Dae;sMMsauvQ24}U! zUcCXtsT0e-%iBo|W$Lr8-OTl$w&hv+4Zf6MpT9mdu;TbyE7GH~If&RuK`z>j=9C^? zuCAKAW!&FSzdoa~qW8nUF5mB{Icd%>L<-zbsv>p8(cc5n<b4K>n>a(po4b53=y~4o zw7W*4c3qXFHsSK9J}<pOQ_a+R&2|klD>3=iff-BqqzrRU&#{h1?=vj>C`kPsRSCDy zHR<bKY`X>q1;GA?97b8<EL7z^stA3nT@YE7vc#YP-<=FCR8^yaDH;}#zdGfNKgzOy ztE=aD{>EL(vI3%JMvO5$tAo;Ib?os4Eta`i*&=#fbVRGQzq?4z8(h*~-i29fafhUB zrQK+S8@>zN6lokAzsaB=F>W_upx`$Z-~MsSIH8HZCBLmfSnOEuH|(k>lP>nmla%oB zSo^OP)3+}3Ji6V?jl{Ez$4-2yT86^4{F@Yl)Y_N!bop{$6m~}`JVc8&aV6Z+<LpYf zUC&`{TNlyVGJh3JWW`C{&Kn>2UN@3@Z!#db+n==2_j?s<rxZw;`+v{tCcl5x@+c!R z=Z%I>&FZAdwX1=}HuGw6Y!8P~6ZSzVK~uNjvz|PM*_%s)r?@BYw<e6%uAVRrEX%A! zPGhrA1#E+MHV%)P>@ysBQ%lm1wnNj%$T{opaiZx!@Vcv8l%aYjGz;%P+TALw^m&Y7 zXKB@Ny2|q9g$8yl7JLq-TBv$^rLTA=^^jAnR$j6fDGo1hJ8It^rympHsPN{=Ia{Od zD-6eNheyxfE4eg0W0hMl;?<LxyO3-@8czVvUyHvJgyqQIVorZG82Z|h0{DqUXXzW< zjrS%t)?Qd+FCcHkq^0R;AN4Ls%bOp0<=!^Qp89?5x)U*AaQOg%=k+1jBqLU)pyhdo z|MCFl_oik)Dj0<g68a|J?qxhV@J)1UF|y1~LDR?omJ?=yGT?5$O42zS4MCXzRkn_i ziB{z(k0g9kD<Evh{|cNki@^xbm&xHzF9(Q@ok@ERUvnwL$c@)DtqWpOi*nPyHRoU= z1CYR@*FFM$V^$zdvc{aa@Z8&kDAWHb``fxpM!!Anz)DWyn-Nk(jLD?fro39_%6#LK zmwvR59nPuH?}<F5E^sL^;>fO52n6d`IVtr}p(;lr1omFLdesbl*{ljFM90b!&UKG1 zJ(A9R+r6#S3xAz#XmK%P)gQP&hYJX35dfpDpUM53`NA7Zk!cz%N*J`+pWbI(V;xS6 z@cy<cdux@Tu)Qo0>e+Fq>9G_bp7dEywe(TGU<yj_{8*R4;QOP0up=9<S{ZHzYF)@J zmt!3Gp4;6h+c@vs=#r#XY#E90pP$1P-g&GkPYxvd5%TgR6>PsCJhgBIM?KD3k+?UQ zny>B`@gMtqnW1c|KEW2}mYgZW-%PG=-F&fnZkUCqvg4~#bIl9S;P>CeWjNH%WCscJ z<lOPmm?p7cKN%cA4g~xrE>?*ctd3mojukRZE=48tpbmxFUt&)$26q@-PmAv$Fr@Fu zvqVePZj!N2)P-%Z3NxEk1Ud9)u3yfLD?a2!LXWeF{&fYC872)8HW9RJ$DVqN(+Kg` z^tMo*^XkKSgb%ST_vgF<XRYm|3>y}o4bZ?xUSE(N1(W3{GebQN#R#j{5uMYR$nuWP z_M9%k0)YkWAmYk5Z=Iw0$|3%3!9$7dah#-qf^9z)n+gN05?Y7*`<Z@S6kET&D6IkW z@HrNm1)5+AMIl%!x4{S%jKvoD5u)L0d3Kd9w$VYhx@+gc$<N3MvpbC~xBrMPLzsAO zT*1osq&<;d(F<%ma)WW8Vy#j$V_|3M4O3}LgraFE&x)g3W9wZ}w@|SHrxQlc+;7TS zrqWpeMPEM8EbFSv(C@z`&fQOtz4s>K{P(h*?eY0i)d4OB6YDnf>iJs>4G#~O$=nEF zklT)2_U?D_t?5)|-1tr-yD2f9hQyS%D1~?s=jn5r^G^AMyF@uU<)5hpil1oDlg;8C zoz3z-jiSxGZZwJJZgJ*jK_yitCU-sZn@dq96&-8p_Z*xGtvIpr4RHShwE%==x==b@ zAmf&O@Mr|)`FklztptIFcmAI**8~GH+iG=4Ggv;ezA7tqBe-Vi=POw6j3fM~3F4Uq zMF?Hi+fLB_QcJ|FJ6~7rAh&pNczOhJ^~Pd^SDb{u5!qt6BkE_$J8CIqntFr?|B_~t zvtfWKP+c}-clpE}DA_G{<=c8@%@OOM#WeW`)l;kABpQ7$%{(XE{Q&NWJL=(Ck~2|r zFov+=bZrfJiunbcVy>|2*Dl}ZY&=q}KpA0uSUXqRx#Krou979@CJDc4#(vgmbUd6E z=3)U&r*!QAu?=KJqnHZ^-dIkl=Kj+$+1k?=9%p4R*B57TUDFzK<zifVzN94YHhS=Z z^umifYFf{;<e3@oN1-Nus`4DRRrW?^c`>&g)|#`QyGFyhDQ#J06>K%kJB&riS>`!O zr=_XY5X44oqcoR|BKb@`@-?snasK$EOGX(gxdQ{4SzBvcv=1|~tYgb!qZGR-#8!&o z>JG?9vAXgR77Hd}iV>L3HsLDkhDo_M1o!145By|}XJN`$mGFL?S3&I-X~GNNM#MyB zM`?PT`=Z`cVOHo<Uh8QpUGxCgD9&2z`IKIl_oe`Hz-+xLn79#op=-kT0T0Gz=nY^o zTy8EMe<!x9+s1);4*Nrl7M1h9$o7U?T14l<c{8nX0c5y7We~Hq)Lg{p*=}EbL~Bmy zM)UdzTwk3u6rd#t1$&{S)cK+le>|LkviBy4NuP(JymMj`wazWh{_YO9^&jrDSK#+j z9ppNjSoSp1G2qP=J)sE401=iwVR932;(-R9LRzNoSE9(a+}#{rWH|=w9)0s_A^HL3 z^L%`-<TjS99DsOZZl=>b?VYnn{`6m+>PZrsTZ0h9xF%-Q+~M73)!P4Jxj>r|9j@~T zjQ<pSFZ2}yn>jBqmzn_$76894FW`Jdt4K2bZ1ETH0D*+c&{P)ivi#W@yt4|qite=6 zZb|}~?Mh>?FUzr`M=yrxt){O3&QQm|uaH+27(S^dncb0-V|15JuIAu+vEJJg{Kl2^ zb4wtZ?O}%f_~|qm^6#FX+QC3G^Q{V0u$Sy(gSW83w-t7lqlz~_2|8F$DF$>>w(#@f zXVSZHG3!uy88G*WGXvsO%w?A;G7kiMg!UBseD#<e93}(E+H8B4!#*<vZveL*jgOFj zj3NnGc>;eiF37T#vcXB=1|C$lJ)$QzyQ71EQ<6wUR%#*8kUJ*_05zGz@<jG<j~r>S zNW2r=rlR!<_W3knu!z^&+xydMhm0`kut+~hP4$(aFVxtoph`?Y?xnxx(V*<zn3Y$4 z|7+821-2MA0%<-)yHq`ccn!J>0p(=dA<hLb=M723z=`O~(-|rpgDF};_cPGDa9v3X z4>>bKZqYW7{pr?nbg&76gK}9ZMt02l#D!08h}H4p>FNBr>A12_DM(yvPPj7rko~GZ zAlHAQIY}-LWoFb&!hA<O?rwGV2VJfHJooAMkmjMD1tbAkp`vdd_?I*`c}EEjjDg~U z_wVIm_p|eqkfcLXtw%8EF#IDSk-8RJVq<RMq2xTSivc4gX=?Xk!C>9{qZlR70YMWI z0M_B>%T&7tojb7bX-2XBRYCXT91_fgUoaZla}<Q8d<hChuHC^?&>M9LfoCm1ZYW@! zVvsvyA1vNzyG*+{F5ee5UZs*=WdPZagE*rdAzA<iettGn4S_D5W@m2;^6{Rtps#zL znJ2Gk_l%-V3*$|M=JYbG3g4L^hJka<o@3DU1smWoG7%&FxBVD2O%UQ^{fi#VsDG#N zAU>?v-t?mJya9>cYX4;;<W))6bWU^F5OUJ%`(xV#yZ3%@cPiN^jN>%)p}}z}DF^2) zM}6GY|873yo$j?Gz3O2=T3Od}7aFLAju0M)ULPOoxGAqB*uz6jS28zB4tC+e)X&(r zQ{cmaQG~ZvyJMgs55#w!Z~XkEa06eslgxlu67}vEVEcixv>Nh~0<%Ty0Whg<Q0w#S z<=p~CY1t{qAX}&tq$;W@)PJ?(<7Zp`z)8LgnY%`FZjT)TyP_D}T?C91$>556PJ)3a zU>O<j>I>Xkvo)|Zz?$Q&RZc6_mvDOTb`W4wX-FM|@wU!orlkU8H4bL4nVA~k=z}~I z!jum}0}wj$0-vIHXCoV^PizOEg%BbUX3x%8X<Y7p|IF#rw_(L;d-se9@p*Y@5t_&# zF9WN-PRKdXUKGgs-dCHa()G?WgjAdqq~nhF*o{C2pDi9bf~m0tFl@kwciTP=-sd`( zJ+CvE1r4MQvZ0-WgM+=Hy$fc;?rgvR-FzWrZMnd=`Yy1soY4W3KKX`4JVao520W|0 z#-FfJ=OCz-s)@h|JH{Ho`J_Sgr^kXYPlA5!mag6=u;~N$LwQG>68!cd@e1sRIFQ2B zQNE5+4&F8#-JGC&qQ9+p-rF%}O)(_@QL7qlQSEu&ha0)ClRs<>htI@k=_Za89nMQU zB)T)q%OCDve8Vd(FT6$mnQh~uucVQxiClf}_4k7oXwj^Q!6CYXe=p_k=f2HSGJUi` zWncr>Dd#D^er-f<L~P`!7&*YjTfI}w7t@az=|^a;t+hipH@Vl>4U$uG1Xl=~jTLvu z|6%RDqng^j{n1$A=mA9zqSB6VY;=y&n~H!91nC_?ii9E{y#x_OYV;_Iv?xe#QWAQC zVn9Jaqy!Q=LLfi_1PCqRtsOkyd+!+c{oZ?Hy!{7bBs(O#tUcFUbAHO}ZO7v<at=`2 zz@23Xy?f#+B{0S&vjB1#;lGiNE(&yjQtYqoVjDzF|Fym4f)Js<w(tAFX#IC{Wt+@^ zDXTT#C^wef^}5M1R6Qau=JzFr_Y7MATR%hX4Ft^v_`D?O|Lh9@KKCZjU7CHfX$x^Y zsWfeVo0pI*>dtOTAG0LUq&@^FZT@&Jru(BSOBVzxTn2lu%F_^=h6f`OBT?Tvq3~fZ zJ_~((A1zTt(>9Dnt)sE;U@~{`Jk|{Ro5d#ZHv$16({qSMG5<Hm1n&f2YKaUu@6rKu zlfdB*Efd2^mslMc>TE&RM&rKXO&cNL)WTqxp@1suU(l!6-$^+2+WH?b&dt;`m@~J| zu-|X3^~2599}>SEou=e1if>te{Q-La<{4@TJCr&Jas3x>sqq1!>_57kAp9Nt|8$|D z?}b%DTJvB#HrbPJ#O4&`#CZ<VQl6{){2apzIW0+M*Z~6DJz%x8v%A)|j^+pWpX`|& zn3g*rUjN@+?%0<9y4KU^O<QlVzdbKMsXod6A|s0`YJi_)V8GVeB846>5(a7Af@aru zIHRF;Gu7)VyNF}lyL0u}Cl4LyMVaL1DtD*8y<zQ>gkE>+mku$+X|%b43SL;{?4v^( zS8>dFuQ=S9T>bRxxXmHpD-};zC!<i#-;$qeEWbOHy)$VO+233kSatX7KNR}IhY#6h z>h4S6>o%5iHPfLRGZ+<5hJ8<Dj36|bBplN!7*rc$$soMnG~SYm!JTVK6p~;yU#S*= z5e=~;EBUv=#Qj=>L_TY(Ql0@pw{iIIe?Z5WbCA&hvv+SnirhM*pLP}Y-Ge7BSsre> zJK}2(^n9_mhBLbZfN)hr#iW?X`qkRe$b-P@>D6(N`J5vIni!;|dJmj7n|a2T5e1{- ziR(Hbt2Yj;(zaPNy!z6iuxY?2JKY4>#u4XSAZ{P<*2U}1b3#5zu$xE0#B#c=%(3O4 z(+KbiJbV%7@<O##AzB&?_Tps05R)2!11g2bMIZ1Z;()-=zt6O+G!3^61c<->6v^Eq z3-OiN+cExLDlFL@y3*jE{_EX49I(@=hm~M2oOCG9fT`tspq*Y37~ZALpKVd$%$5aA zrhuo400apkdAnFLQ2)I5^`BEaL#%-PgyHMVY=ce-Df_$+tY>Y^?(BL5Mjsln&3XVv z-gp^ECfRrG|2&oi6xl{L|9QN3jvf4c%l|+!P|<e^QdR>aN$<anB=GvC(@^b5Z`tV3 zKc{XKbaL$9&*nSd<Y@38nKeRf7<=#eTX=CNB$O{POMrk}uKt_nDt~#Zbf7eGfncFh z|7%DMGJyDr1k1@?W{?pZ61uLzb|1OvxAXlrhuOO9_A0SEL*ZiI|6dO^bXG8;?5_YS z6TTQXzJ5kyG3EjwqJl%~ZXc@}xdswLoy9F8yhLq+Av4$i=E;-9?8gVmxxIy^r(x6Y zmqy;9Uzc8^PqklrU<WRc%7bYGNIO48+1Hr@;M$N-74$Q*CVsQwW-a2w4NI*OkccII z!bNivn4Lz<oc+Wd;89%~*i#Wi*TT~<>h#bQanaA-ZHfx-HJPozJastiH%d_GtkDcb zruc>xM(x$KkayTq_@mjTnyjq=SFl9%_J_hMbCZ$#;>@j_i54Wk;$P61{&iDL>f;_d z=^}+ePtX8(La>(C`pyWj!n-o}0X1n>CFMMSE-)p{F`Wzc3m{_V*F~cak3EwOpbkF3 z6K`~H_z}!By#AEK0I3CRUJpM;ZK+{7-D?Du6Sq`b$Hb)m`1JVT39Le-ld}18Ba@M% zmnw4o(IYK<$asTejC``zy>wdy&(C-?Hn&_;*_D^v=0~u8BW+iATFR~5E`7n@X+*&z z{6nCyZ>{GLS9eLPY1Zu5rB*e&kohD(3G3C!YK_l>QTnzbCmz49P(qlNT8^o7C}$C< zAxR8H&tHM?*E9kw>&`#Y(vmWJSjX&3{o$b(sn7GIc9yDlujQ(Fk-O_%)kSk0^-1Mo zhU#q@(h5c39tktA9eDQB?Uk2JuD-bz$p!UPQMF!R7xl0vTI4u)`=tB}!tK4^`l=3J zAeBu_QZSakQP84hV(}lWiq{6hLPV}3>Q?fXU1m!(C)O}Nb5?x2Dg!^KUHRgr@|fz7 zsOa8r%=J16&uZ?RSiJ9)ur<UGnF;=~Xx&{r+mwtrq!TVeqTz$s6Z+^Np={J<xp76t zeizJD+helq<44DPG^@=Yg>_FP^t54a_ovN=zFS98OISZPGIyJqHv~IbhODPl2KVBd zaGf9PJUaQ4I!sHk)oWp;vG+~CwUBU@1G1rfN%d15=`Ws$`(js@ZWxsgERJ`$O-t`a z&FW6gq_ozn*;wcXg_YQ%%h$>AdoMZaNzWNvJxdIkLJK>_`3a+hG)Z-g*Z1H~2`$RL zBU8~o3R-Ecy&uQr9fdYcwZBsl>YIT)Q*C78gJ-0?zPgMnKm}>+piI5B8ZyE*sxkOl zRp}w1HFV9aNGL2J(O0^o4Jak>(}i2Et|fl$W6x-+AoKkn!3vsqz$AL#D(B&@VtYWZ zmfU@L<?>A)%+u8jQpSL7)Kp(d!*V<UPiwpMJ}=QsVz^@N6RUMLarwct^iRxt$9yjA zaxAAU6K|6mOXh)zm*zV0a~l`dYs=R;f2><evz9A1lDd|~^s=4hxol&1pi>)mu9;WL z!`Jl#=9EjgSMHTj8WEmjwvaDaTbMrajJ&>THFB=NU1Om*A)numzF|<gRP5XFnV9JN zr?^J59I<vKKcm#X^{4m<!?O0ly%eejnDVj#ul^53G02bsiUa?NvZC&QRBQIvUNV%( z!`|8$nn*<Q@zftg-^m@ktZ?OGj_tIxmFC9-cHP=h$3rd|Umjc>Z`00h;-HT7-^SNi z&r4R_D&{i52ge&2?4>y9DI8?hZ%_n!$rJgU{G|T!$&eWg$+M_red|q4uZLJd$0J+i z)d>^hJ(7<DQ`^0}sLL_K1rGwGpEla<{SZw8$iaGnrU@(5b+81NxEzPRG+5>y^t0^2 z+JY*+=54us7>Mg|Ak=|pv$=B~?PPh<pSggFK=M=0<9Ng}(xAs)4~^k8dAt_3_uy&K zNJ~=2x?z{_g-?n!Zef9v`Umq#+S*u8jzglZXf)@wIT=iYYuA?O2R^AwEkytIhLe^K zI*)Na_M*|Tw#V)vPTYyFQy4U3#=GXfDZI&Lb|MtSh|Q6_a;t9^?u^I$0Xa?4$oEK7 zjFs`F!uni`q^e^#D&xPQ(lyV7XD}N4w~AwHo;t2AV?@x_Lo#hil@6(0%|65RH7@n1 zn*~Wvm19d2rcSTQRCYTn^sSvwl4n*W)qHWpNe!7Z!|FC)Bry6l?{7EY37oVm2T6y? zms#sgR>Be*Ra-1cVkAHfcTi|casV|PB0hVe?|NvTc}ul_0-pGkpj+5ou4Kx~bqCLi zTHA6yIqd%v<}@#VO5Cd%U2LkMSW@kvx0n)>CNOVnL=gh;c<nDzpB17J;lT+#nBQay z)u&F<wyzXil(thah|jlpcA?MRnq$clK3V&+#d<T~&#C#^)=fyVu(qYBUME_o`9j9* z=-Bf<yxdg3a}jxIC40E$8R7E$`ZKOc)7d25_;ZP7g@{fctI5F6N~ePdb0k2<1-6Lt z#Qh*wdCNPc1Eb6b6uf(4;5k|2wt|<6PNjTJ>Tt92ZYH;gX>597nZ+21{b=z|M{gmG zSK3yoLgKs6%ghDVa6arkc>T)ZiC12OHtEp?RYr@$oHqwUI)${qJGZ@RiDjxWhaG#} z9zLe)S6ykplW_QCTCnJQUMY*VJ0AiwkT;@|Pb6&iW;zVg??Kbi@L<i%tk}aUVi>Zq z!*{zH?b)C0FYIHUqsFg%v8R>z^c7yYy4rg}h-<{i(?pPO8uXrRkNqZL<Gr4Juc)jg zoSWmN4pJS%_c3Me(?Lh3x~9(@#%y)IpeJ@w7y|0nt7#mmJ=pd{A9-dKcZ?q7g6VFb zl?ms+gd)-l%_TbPjf%2dBwDJ6Rbo2VM8x<wwTfO>9TLeG&MjB6<^63KAO5m%NY6OS zw<1(8KOp?4uUKT%K~0SPAh|iP<eZ~WMSbzoi5OYa2H#Vwi?daOVLIj)Xb5W&OXVsO z->lm1%1Qa#SZTZ^tL_ZeCZKLHgSaltS~sZmqG@UWxw@VI%_;~nYr{cTA~tZpA_Tkp z+Mc^<=H<Q~*z8i|@qI&)<E&WTF~`=2q_v<51HbBdFv)Sqb=Y{tt9TY#aeiY{AC|?T zg=2?u7ghS@o<o!N_$vhqF98EM+`NjlK_P2@I2-+nwOm+rwD^>&5H`-<Xz$%63<0sS zA^L!KGk`FZ8cVq=hb2C{9(4|yXxt8qnTf?J7nq6@{Z*cMcd7?~f}UfqI6&&JAl<TS zT<^?IRz_>d3+N12z2&@f#o$cjbHjvtZC?~933YQbt!oZyak1Zf(v#%E>eqrwO!-h{ zD=}x{U{|xedrYh1n6Y6U^#0gmUfi$o*q2RyXa_sdqT727{TmFATQR*S7cU(6+#?VY z#OytY{@m$^9bXCX=r@k~UQ)B#Dz`^=LVZ>FL4ZnQfB4rG&UtI*r;RT@e7>n9>eR~d zlZ~80eNU#w)8gJ5&cWjuUf3|i$`(=%-+wu3kE69*0IytEUn6=vn1_5n9$tNWkQ}#i ztVR8@cTKHDR5>MvfL}LM^(syim}tnk>xBz@AFLPeL0%vibQaw#ep=SDsg(tHGoBRm znwIXU`%ph+X^nl=qU^oR6ex5=V!F$Qk{{)bn|U=%j#_2OMf*6_I1@gOJtKnYfm`Qs z-BMYiT+4dOcHd9mcY?n+IQ%?;W)w5={r>UB6-DU#KZz%hHe#1>LNfJ(xB2QHYuC+> zQC=BJRRnr1`mND!U=xcPwtl=SMMF<AtUDb_kI64vZg?n~m1+n*jk$kS!Hc?9czP1x zU=QDn`mJ1>AbEDT3{TqGf_*~``J4t1u1E3vj28Nn7S$rd?Y4f@jvam9=`JHqFlk|a zVd(6WYb}-&yT~*4Va2ZD-3ilLhreJ)@8pW!Rq~=%?}_8tXeF)J`RvPk+%#cpQ}D_} z=I1|?5DZqin#a*@eMVC}F~p;~co6%y8E@R%n*av;%&Ovm{>l}6L9|Fv(FW7t#cGm; z`x&zc0EG2;d%9{W-rhi2(z5h#J5@*I{9xVUDrKvs8YNA<s8D)Qe%Mh2Ug7V7p1q&% z=M^N)2EhvMCx$nli22aHU`~z)jwGC$KLG~TC~Lt|b*bh8zzoe?`59p*NFry1ZN)Pa z7g-FMlG2iQ!7b{;b^=P@Xh>fP4!YZ;0N$FMyES?~*0ZfNFO#NOEtN1mq0(ehDD_Hm z*=m57qPbVuu-JEhKc|s6KbtpIOF^5t+23vE<>KVXt=^NfJ?4auft5Z&@w1mgb>{uy z42eO+c7ryFB#Gg1tn;F4_4<EL_)I+4N36OnAIRTQYrRjVN~wg8@GP+2#&{_tYppKL z%Cxii+W;S2MyMhZlp~>qyb>@(YgGO-@+6N^f93ke@lDRtFZb-4><d50A1;~DGWNXn z{>jUGRxY1Ds4IT_@z_yhol)+Uz;^=}v19){lB)sm)(>XfgwG)W-rCHYa{R_OGvx&{ znX^a6;*Oa;_2yRKtiZE+pHRzZb%t?_;;^i6;dxatAoTmn^iXti_|}hYNpTglz{b-B z=gJt#xJ8|}@89^iN=1gC{S%CDS()(461R$@V^`8zgdIHt=DKWGQMSRgC(7G;xwc78 z1U@MR)fx>)0p8z+oX{upB+d53uNabs&+qoTW1$x)*XB#gXR3z=Dt%F{x-?lo_NuXN zR>O_J=oTZq)RBsta0v>dolPe=tq<lgSfRFr4vS+n`D|0eAP+$T(zN8zVukZqv%1*4 zQ5#apOZ4+At#UbxkL!m}DX+?*F@uGLpw*lCT(pB?e0#Mj6uKv8jn9-mYS+kG_3H8C z+Fj8`QZx1Did*3${8P>PTq8`1IS`&jG$1^cYYstewzR}Cnv*pwEsPd31UcE>4bx7L z9?t!~Fy^bexa~kxgKD6=@ds(Ma7v}m^eLj80uV6f{P{T<!^hWr+P3%1Za(GF-lC2E z0uzbY(n*-5^5A_uvDw40ZtSs;<~giRMD9TKM1Zmy-&kdEX@asxPo}I{mfKNNi?y<K zu~H$s)F$hhbC=JI<QJ6=y7_cZMW>n?N>rsLxS33b34~c|qEZsVx-!=%PJ~3+fm_v` zRbcv<^KgUg-x<z)3zooaNada3v;3q!bb_ss=lbzoc`>H>0yip`zs(!L@s?i+QRckL z&w7i>t+>%iR)eqPRTWzwR{P-b{-x)lXZJG7XPYg!&ZFpYI=cb8mTa8Wmw!@Rc-Vh7 zAuIe9*}G`=wy|D<V<owmnT2^R4?l0y_jT63qSJN7t0!=hezpVUJo~OIgg?yv5>d7E z@7G@J;vMWCC_=sX^OOGMTbrXMA4Q+Znnq-~@&H_bf5l|1p`k_L^R+qcxTEhILZW|I zg&^LjysMw*G`nzM^yjFBa>T0XxeMi#^Db5B5t7$-?HbV?8#`jBWDtBS<YD6XvsYId znF4VIW~J2NtP^Sy;!jQbzQ_}72mJdbEpJT4x-&fMvp6UC9@g>)C~%Eka8pc0I>u-D zCuTNGg1Z!VJ#n&3_oBpQPsjFq#i5>`x0=&_M%?0QE{qqme4I$G|1y<${(FCcvt!(S zi*FuT^!ehda0_O<eiT)7Z_R9PGfw8RPMnSxd1354VP*J7V5$js8!as0{#m^%>qG)o zK6@ZYd04p}rTpA2)tf3Q*}jsYz8S|XBD_+Gb9;ZXgDjQZ*M-jX^E+Ghvcscof~HqU zZVN1Vgg;iQOqWk6kmUBtMdJ+TV|^tG#F|6XJSVZ`)l<b_26#dfDA`(TzFJ4?-n9_u z6?O8L<#@0SsJlW1H4bjUvl);r1+a>d01IUGE(&0zBh7Hg?=AH;zA2`ABsjFVkrm@6 z+=7)$!$D$64^N+X`PHXi>?fvdLhiYJZ{Y7-Q_67L;v%(@rU}kr`6Dfqhh9TgYDkJF zk7sPG36ENwt|#r-*mKi@uj&hp&6LH?-h5F5pOKvCSOdqb^#p>>W*euRw)`hSrLSXE zCWSI3s;ltQ<0X5KlFoh#Ac;mO4(k6MscqOULCnMO{`t-Kh{qiqE1@o~z=fxOVB_-( z?S3zD%evJc<MsVU1Ft>6G-WyG%)<%_Y;zPVoxO=~5e$zFlHfR3G1<2N2huLIb|b8w zinH|}I*u~pK1`O&kV>!3xp_r{K-M3v{e$ckG^F!Pdg|{;xM9!-+iHo<5O-Wskcgl- z2aSl+NFb{yDfO^M68%{^38zYwdCXa);#($~ijlE~LiE_ym2!&@GG>g>U=mWNAz)}@ zHtu>Ty;ksty4eb-!e~WvLyNxmZ_c~quz+x2;Ntks!q#M`qRSw{OHEjh@b-&pSX-a3 zP*IN(^WCN{$w#_=;emmSxT@a!9~}k)51%6aPueelf=Z&$XtJu`Bd(znA(aCOx_ezR z4;UH$(`GV#*^q}yln^LM;7sc=Mc2(^1iDK~=Y-E!E7T6nwPjf*OnE5smLhedCYSof z%wLj5n!~m8iqielc{*ZZ?V%}OG-7!fL(X+yke)i7c<Wo6jer7QQq8!zPPOD^C+|?v zw7`())l%zs#e(nCo;uIRR(PfhyztYrxRCfN+tGs2D-avu7D|03HFBd6hby01rQr}0 z$taK#x>)ialE3#jO#10F3dSYeIBJSmY`{-TX!Ykg(9v?C@1}KXUvZtSUtLcKqXDvP z0?i8xwectKcdC5<mMmX;X*|i&MS@Pl^!vPvd?p2=7Z{%sU$^kdwwjb=ellqgH*-qQ z`)O>-&8>4WC;hNijr<sCqG(mo_MmR0yd!#0D%$I40uPMxIhkVik3jfV_l;YG^eorK z;Hgg2qYtY%3$8TIroc~BmD)6HjE{-m7L!u<{orKSrGY_DSa{8;+6LC-P4OfB2|7h% z#}M4PdQN86wgbqP=tAE~$2Z#dryqKG%=HsxIAq1#FAN(P2B~Sc^;86U>i<mnaZWpA z{AkIc`KR(iC%+BuSLwUt@uz&JuW=Gw^>f4A1b(H!c6L-}$?tyWFjLRY49hKz`@meS zb2M=fZ0kFdQD6#luev-m<)v;M6`<v*1vDj4v;m6$3RLR@H9j*`OFLV`FPYsIfqZ_9 zKBJ0p*tHYI@>PjhXot#*Z~pbw6=SoaYR|Q9&Kv&;ckcRxG|}QyuI?=`DKs@ky>7bd z(nNWFw4iJEgAdoxxl0;0)kVz4uYuarr->yx{n7>D3AyFH4U9oU!PI_Sd#Z@rT5bK& zCF=|D^Q4Y`1e7K;3*UKN{*fE4oN-`0ZF2*ZEs-6mMuKg5!u-BRkRmONTWUYL46I7* zkBKfEBfMNZ%dyt6TBH0Nem(c6?5sH<+8<x@3VF+O(Hi4+w{DT{<5XoTDRQq&#ag)n z&6t|C=l(2zTZ6mM6pgUSEB>q`>@5^SC!_4PN3<V*F@5A)_NuS64jbr|=Cx=IgwrGZ z=SL8KGAF|h<>0~>x|Tasar)gus<f>fAZ||%+vNTW^rVo`Ff-Dr{PWvKb~<{`{4{`N zPDoTG2J7urE})GpnK!}kub#(VYN8#t%PTP58|DMyGSS@`a*?9SQ9r05<c_+xYbwJ> zV<tz(Ug-)KZl)y9T07;ZQ@qkISYM;IioWmlyeMIdLbe~2mpTErdrjy(55=wKC$4xU z9v;t_GTr`cwScP732LjOR`-OEuZSZz`eQ?k@Mgui-8XpLVn@3^J(oAG<o3M%@I#U7 zuC4J7Oh*DJ0GV@@gf;w4n7H@Gl6wIi;;G+rrQ}@tZSM1mwHonel@GbS(-RfGJ59pZ zb57qG2-dOb#)OtK8I^%T)(S@!;JV6R2Hf2f#A({6`iZkw3l)Zabtk=ZUzHoTrzux# zgaU)a-(arpuG0;bUfT~VHxm$2!i)9%+MM&RtZ3Irz#|i&&nspmW~o=TT83|QTg-XQ z!RLI{P}EHzN|MAbbVmm4*^}!f4$V4X#>v(G{*$$-c(hY2elgds0i73W#+`O$?WFiE z8)o(jaH|T)1vO*gmE)ijt-C0HSwnz|rNuLJo{E{3#QI0KU$&k%4Q&1R(P7T62JLC! zY$i8#{?T8Php#s<nm54`wf4OD`Xoi)@VT4X$F%-!zgf?-8M@WEg4YFwI}N-AeI7T4 zhUTRB<W*QaBNO4#dz6q~VFmr<%--2a-h;!vFZM(%$|nXCeGP7@S7R8yD{#+j3IF@E z23oZ*IW;+}bfo;<8nM8fRE114lZ$SjTF0J-i*l?rtCY4FW&28k<->!^p1pz;8<8gn zma(^o8zb0)HOE=mOSQuG0UcS~os05S1yT}r-AjW#SjCd8?UnM`0!>XT!pG{2mo27V zs)onQ1>iAd(DZRDjADf6`ci#zU<9!$KCrvd_{hmRE3Ege+HoW{4O>f191l_(UzaBg z8Ap``e^u2Dw~0Dj;RtR-%ae`<1n26g)=04+Diy5ZlD0%$%}Xf!n{|i-(bA3xVtO2> z&~DgGKEMT{wK7YN=WC7cziVe_x2T5L2aC{;lWh(FE96>uQx50=JWLM!`Msy^QH!`@ zaJ1Ef=1$?f-!5M9E9dOdetYt=p61YsjwQAT$Eo(2zBLSVJqldA&ER97)_ci3l}|_Y zl>`08%8~qc5?r-TBsDt}S-6W~J5N5}_8;)meq0JlvD=5X8ocIZF8_sJEnVNri@jDh z5;W+AtXB*_iaL%;8sRtjr*$Icx?arzqxxLy^kOxPb0!~rt_|aY;ZNF+3ZGi1gz8Ra ztGUQw^DU*$P02MQ2v0U+YvS#1a5ui3Tz#(mth<<*F!2Xpo^Xj^r_!)oiGf#o^sx{R z*azMgmU~Rup=iRB0x}*H1Ti(%rpmbS>X(J14K@e#8f?-IZd#ToV8VYM(_Je_3Q8=9 zTn}v*ed$MQCdI*TuMd7B#O6)CCbY>DtWAp0`uPER=e+6$<g<-3ZI)12b!(doC5NEC zKrNxi%#JkVe^tIjy-r8NQOhYji_6Uvd%78|COjF2xC124Hhq*ZYP3nX<=&Z$66;cW zvLkkBF*wJkxX$Azt#P21yLlv?Kj3#WFUd}_{Up7xrF@A2wJl;Q3PBH~9qOjRU@o!u z2Ob?lD0++xE1C13%yHG+n07><u=FoemG&PI`bH+)$DZsv7_O;Kf4~&~&Sc(*V}83+ z88}~4J)))6qWWXMB5fp(nXc)F^mPreb&ys)wYx~UdOh9COJ72U)pVhksXe)k9IBz` z;_0bg8!uWXki0P2prbOj)@mxnjvwp(1nQ2B1BZ8IPIj*Hzl-9me(ZzK<eV$ALWEgv z_s+&CWS~X+4?m2N5;J+(TC6?MU})nRmj2!3=|yd|adn;xF$PzaZyupRO%|YZaVfLa zNO^(hVJ0W?XPNFZ2d{h&`@JEevL-Tr@=hdv?|h|;Rji*+@g|2!#~B!K=l+zNBoGpd z-*C#7*oJxgb+q2eJx2Ga8djRL`lGqNMJ^^NVZun}uOut$sTGXB2f1*_sqB?Qaf`X_ zki6h<LGia<cj`pHZlH{rssOD_Q@}wIS}>dlC6V4HISZ!dy*+v|PcS~7E|X}Lk=o$J zEH93*2+EkXxpi4XrcFw>t)#dpNojuoD6IJC^@r)yjfbYy%8}88m%${@2YNG6{kde; z;-bT5XS`ta(lLk7GVT`NurD5O#V(#e9Y-#aKg-JzAL=@F6X6c73N$pKaw&vA!CDcH zWDHbuH=DP$oZlT5<Sti|kVL;5*nV(WZ$BvEMsV)ezR?w&)IQbG@OC3ba(5ttIhr+7 zDx93sJJwWol$6_%?t6*n@A&Ho;KHYG79Q@ALUlj{V7IRGT&$zCw#6+7O&Do-<-|E! z9@=y9mJ;m|0G&3zTxt?w-S1)8t$pIZEfZ(($=>h{#hmA6T&(WO?tqn_6rJ@3o+J1C zQlXL<XIKv=D?zq7<N4@!KD|f_!yyUr+jw$RUJfSj8k`hNdg|T6YVJ>+r<26q75AG; zlzN?}h15AZJG8?6sDsLQOAjZiYS3KWDa()@r0M9Ye>&z}2!mBlv{d<YZF`HjeSBq8 z3aMtgw^Fq7!t({A_246PL2{rf-6VtNpJ;UECX{Tw@C(aR*CZ-?q7hn_>#Fw^Rx4Ij zG<%!cfenb?@^n{^L}si_&13hddCzJVrF9~%chn|oDSY9lFMd|^=}NjTRz7==1z?0( zZ_IMwtkk?r7d;DrAi+hI)!{b}^OT%xR4mq}8Up&%J-go>u~Kg;cD@x5{*+n2J~hQh z%=~0`rq6#KlY0YSS6YM<n;%gW{9&?4om)U=!;*g&Du$TqFYbumiOTGS$K9zjtvY_G z{;^b)tmq$q50_QdCM;sH$H$Og<%ZG3uZ|;JJb|@d-S<tfxb;;g-hB3PvJZSRFP5Jc zV{khNmucg=6+ZBow{T9OYeoGHT^-7$ZuYIVgoH;e#;?22k!!-KyJ5>IoIgN?N7R@w zg1g%@n>pzgg&ChE_l07L?Qa`1Re)z<vXrG&n2lK1?FSuc%Do@#&*Os0iuoks@lNyR zl+=e0v;7c2q2)#v)-GuAOx>_}RT5JpwWU_FZ+_qauEF>2crijt!Z*q8eAh}}YrXIr z`x_Jb8T^YDMm3Z2u(Ta$Q}Tt75uzDaR|4PPvDck*QK%B<qs5vN=QGu(YkDs8gZ`05 z_(G3w78XZOEKTRBmDpV3G9xh8U))kp4PSa;gQpjk)YiKip5Vg?hfdesV+d?p?tLWc zO8>OY`k2SScV=x^M{N(E+ZQY<n53Eed}|mrw>6WeP*avJ|2iC#_t}Jd@^_&S8Fi)G zlb+-@O`#QKkaT5G7{21P?ZVAlRb2_O5w!TdE25+RWFbXGL8H0ZEF?>dxZSp=S0PJz z&O|lCaP#(Yl1+d|LAXe7S1XF&&gh;eHi#x<b5KxQU`!2#_T`EFFts}NVX8H$X)_6_ zUG%7agy%>6%59A&-bERqOEnbMPe(s;x}5}ZL&3;P+QGB5*2iPg+oFX$AtT<BHE6o> zkV7FuuGyTwOru;Lu94cI>Q%jaOjUlYgF__vLVL~V^5%rKz_DW*gcIqSNR3qn-m*A= z^;zw6$T(xt_srB!U%2!&MU%ufbKevsi8{1x8DQ&`Cx}~%SMsd!ai|HSu95@(ptYY! zuLx>?S@m42zCF=KW9nTfRehshGu{$@OO18kPkL8G2iqIj{>M*Q8_zCco|8+Z;ZS8z z_20v2g002gIiS=kHPUZT$+QOD=UUKZ2fz=I;e9w$75{tFoz%AOz&R3QyYz|M2FI(M zq@+&lWM7Ee?n_xs@;ua7!plc$7NMqU4)N$>N#8&7t@BGIH9-62&O6QzT+4^d9;qrU z(JgsO3b=$4gxflG=XJ9@G}aaZ3LjPy7d;U!)#dhCeQ9zS5JE#V11X5?GpNwjctZ0m zww#){%q%|ScPf<s-AA=o!AV?>h$R~NTg}@w?5a6G%19Z@b3Lvm1~zP`xjcP*^98aO z%jAe-=DQTt2-zM~{ShEMFxa)eHb~Y`6*p$|J8s|Cnf@DNlTpa|4|6hyL-%o4d04iu zlwb(Wd?uh6rMa^Esgv9p&Ae~)TmIO8GWs{}H!au*qHz5mYN$l%PIzcDt93Cyfckm1 zbypLoLmx|H>cg<P<L2U1k1O)+!AYELa4B)l<}KZ;m{t1GP<INZVSj{gF90V2vCOYa zPI~}0V{G*UVk60_8$d5^fyQhhOXE`@$X&&uv{_q8CNfCb-psg4RfaE#%A&^TLU%&3 z=ZP5zaB=y(iCjX30-q_{YcsLNE0+U!Bx6VQYxX%sAwbNVE(qmm%;(yL4{_5n<H@T7 zDPLxabrs&<?a`eT%X@t!RIr05<uSD!$gtg19B|+Y^JC8IJ&Gyo<k5j+G@Eu6ZCq;I z6sJjbdVGTyy<oAr=l#Uq@|`T#k3H<BeD&xZOjventoGaT3samXnC{4$=VcwJp*lpk z^FmSiDx6?t=<W0VlVfo*jnk{oGi&v{=<kZAz4baie^v~ecLv&|%i0uIn_kJTJFiJ& zw5aRA-pWC4Ag2ca`tb)aH()~*0rKd_);irt@iSROEl{>SSxzi$=><v>yF2Y5Yl<{- z9ZbjPfE7;NlE@t@e64n6*7IuBP_Sc)BJDw{#Y4~|drQh}HMhvMl%JBl9Tf=83z-MZ zqeiAW-WY}HMZIq=Hit{tY9|$KDf0}9S5+qv0J{H^*5W$Pv(`HqW3av)x5Az^R)Qi6 z?do#E!^rm_0as6Z=Qe$|0K@FvlS|6S2#n`*?a}^ou9H?@Q>$4f(&eK_^P1}<_Z4|s zrb{)alV!|KZhh^r!ISSg?n~-;G8ijQqergeP{YbgsfKic(wI@}AjG(foR{W6Q0=5w zeGX>a!Shs__H1{m*VYTSi;`1sIn7L`6rvd^CLZzfA|#K$s!ZFlHH#gh=E=w*fcUet z(ef6o@Iil3^};C&g?I&`+CJ7NqkPOiY){;nO}tY?Khu27!_;x8ImmmGOfV&9DSSVH zG9fir*<O{y@D$(k_4_h-Lg7=?9i^zXuK<(r(GV^Rx&=;!blL6Yl*ILO9GeaT4AZIe z%_@WAs{@8Dq8;nbd>c;)-$u+dB}57<c#8H#*!Gld6NAS;$Bb(pi-zo4cc5;~t8~=1 znvh0lb)fKAHMIBv(0_dX3rLEeHEGAn-e3akqJ&}$$C1Vw`>{{^v?HV~o3Q}ua1T^X zl2Y4RVihV4dp!pp@0lUOH8jQVbKF;e)=_ICi2zb^4FaA*%N?TT@4F)MGciPys2r6U zRb;5d3Ila~D$d|m5NWZNc>NB~QR4d2t+)t%&wW<hQ{X;F>Z18-$9+Xd|KO`EM{WJg z)drr3Dl0^bqkKJ-t$LHm`zG$&rPL86H-u=Kapeu6-y|p9{3bux6_0h+1oUiae-WSh zt4c1RGbU(Gr*6Ri$^xZ>gfmq?3LN2btvtI=;0kRHF@{52h+#u|z0b^RQmjo&xR24- z$N`}qJ+z5x?d)d3F9rO=5^eK(0@Svo*9iy0@!n?<;hql#cU3HTBS&2oWMu_y<60j$ z@zlay(&beU4xtI*{G_u;#ks-tZbRQAmztN>`Ee3R&F!MCjO6Ix-mO^iBreO(eHr|R z2JH@1nf7T#ZGf8E&o6R`jL-Do;^ek*hVo%f*!u@RzdRquCHZdv*qUWKc9jU%-j@`( zHRqxUe99`P0CF-5i`?$w-!7{#asyf<yTf7R6o87$A$c~OW42e(WFi9>S3sFK6;x+z zr@51jAp{8kUE;TO)Wf&dj&G|TfdzM!Nt?#T-oM#kZ*Z2wN3_yN<~#3Lru=K$+f1y& zXXDD(X;!$uO}m7BsnZAShK^jS_IaA$#k%5JqL#zyC12R%*P8^N?Eu<C;&Bw87gD<| zS1ez?(q1U@M~olqlzF@#PH_BY800B{zHI9|L!r>a^e+2N@Fn%Ojx_^Ev1r4On9LJ{ zdm~d<E~p|dSBxZ)#a@z>qvth)%bcy!50&*|xI8Nph$ynbJuz?3--dQIx-KGb)_LO! z9r<X0ic{FhjJ5`>1?}FKcH6<rxU`P#M_!ppG!Qv2S61$Wy_E+iE#Z058NGdVFF4yY z1*-x~mS9SuNr}?YhsDHa`l}t!yBSGth4dutesSVg9f5B{UR(7ua8)Ucg{fzqg~Zg| zO}c6<SsPtz$2{HcQPJR~h#UaQ0)15q?ls%5IbwB-eLlDsiAV4Zx*h!|^?ZPSfnu}Q zaELBC)}d>hk<g8!=^QQ^ST~YrroTYViLZui%N)umW3@eO<_w{Z1O#&n?9)C|fZ_7V zQbyLyi<nyqb(_z>5O^%J>n(5?19u;Sq{rU^Wr8kSjKZIaT?;l@-)-D))X9R0Dx&XO z`gxVR7cM+dlQMs%;y@i7@IwfHMEMwq6K$Iv)kHt=7i+g1n0;X6A>W|P|Lxuykr>fk z#z;JMqA;E0GfP~Ig_(O;^_q_GC(U;h{akIa`K<(u<8r0wj8IBV>%_o$aiY9$(G+^< zScrN%=KR89cGjne_Q)3r1=hP^>5vHieeN$~wfc`~DUYeAEwzDw#*7C*0{0@;cQ9oQ zp7!?Fh^(y{npmTP?Drwh-7Qa=2|$wd$N;Ezr}Om3-8V*T|8Niy4x1C{aGR`iFKYZ_ z$gI@O=FAm&yrqPhKL+nEA^I$^t(XySIl^-_ZUyHvd8Ru}*;nPr?X;)wq%mdtVLHFS z@tHOQA0IV2WoFp`lco6?<28Lyl6+eMp?Q}k+qzlI&A8MZ1C(GJ#P;+v;1;5(XNMi5 z^ys0JN;6~4jkyHc(r)`<apBWlhVSYG#2vBg&6>wexFjM2T$<8S4p_Gr>^~re&h__1 z^=}w-19k2}>B<R@KYqqXka(>`sI3YD`CLQzbzUiR`bKH-AN}9H8wB+I1apdJL99&? zQ^(iX^PEsEU)tV?nF1?DoyyVry6Mcg0;#r4_A#fr?i@26!u4;_bhb%6F#Wg#7Ce1l z$5QhG`H^cuirNBS1qEZD8DwGpNl@F~N~F*^smygDyBjZ@-kz@SFLE1YF*$U?M0(d< zi~K#2LLsF~6&0E;Wja~e@S~Ti1n1&M)J=oJLAIS+ak9@0FzQLnB`!PnbtX?}u{t^R z@pyCLYFZ+CWV-gYhD_{Ydzb!E7_ke0-O0J!u>welLd46voh1|LgJ#pfbRb8?+LkaG z*tCb?xk5?tKp>HH;^Z4#HQXrWDQBCL@*JUnQzfbJL4snvX-SU4^8!0~U#Nc%V=}_j zA|B*c-J2Fs!u#S=3erisBI>Fpzr#UVJYq(;Dp&=TJ2G1xRv(=~>211QPFfM1y@+oj zy7_~`$adOd3P8i;zlpoCtNK{?i(4A6E#=9PDF={Tt24fEQ{)D=tBl0jip%Q2c}9ot zis<vZ0LJV^mr4Qbj?dq8M&7UqcPU7qEiH8Z%#r*gXE-?v6<1a5SoeY#-syG+YA=)w z9#x18U5UIAl<I_@*M%2;!G}<ZAM)=7%Kjd@wXC7LmY2R@XAwniz4R#=N38kkYw6!> z^7>2)c*>M?5-d4eXqLa>@EcFjU2}%qk@z434SukIsh0AIqP4=;WN0DN@gBVxli`L< za(=fP!6WSuNS<nDLflB#bU!R`$dA<}KXRS1-dsPZp-}q|7`3H6TqD{Mo|CPbg7O_i zM}Eg*ro}f2*nQ7wbBKAR0x!Z;I5Ka7(GSurUlxUxpe<W8$dVTy!HZSJ7jcz)Ai{NP zXL2-!&3ZAMe+@mDK)vS>+3;Kp+dBePISucm_uKB>5`|sJEl&_U8auyoNK$W;A$06` zOB_CMB5aXJMpR~HAMo$u_TQxE;&77!jQ7xb??lc9Sf%Hf4h4}3Kd6e~w&_`U5vwoK zOW(7e-yads+$0z}#=6~`ym&l8`*5LJ70;14G(Y@QpYu2P-fGc)zFEqfToL$2P+87+ z*Uiw?0Bl3h-H>fr3b{tqs;Ft{M55f4z+yqS!s34Z*>ONr@{^+mG8JhJS@Q~QXudIr z%|(m#^N4bKu2h&#(MO+b05#3)JBEu>Zig#G`rUF5s}9a+At(`iR+^O<;7nI3)#Cv{ z9^30eiu{*`Bol4oWh^=Q`Tm4o4&&Kf!X>dA_Cgmpen<G4XSD`6^02)jr;9l2LdnnD z{j@ID9O!>jy4@FLnXPjX$Zj|_nUt}@|FZV0-RI=7*;gtbXjd;i%sU|<(eLjx$v|%k zJS?&vbI(h_yp=nIrt^Qy6kHOIrfO78Rm@BYn5?*&G@y_B3bdRRp$m@2qN<Ll!A+v5 zt#F(|{Bd}-wbb9uM&V&aaRP1yZTR3Xj)@oV)H!IlX`b)Y9%c~Td{?$W6i}TI{7Sd9 ztyrv3r0o4kX^;_~^lz0@M65J68z$`%6Ev?c>2N9V=YWyfBnw~bx?dKI5!7QPM^{~W zq}_}kOYFYD^DJn5G1>^R{m%tMp30c{q^27Q7_*~EGe9{4?h;?Iwb2<yt<bBok&!f8 z?I;a@L43Z)Az?Q{4g+b~tKZ+A{_}{VM5jkqO`8b?I>UTk(VJjtGFKp)7vwCD(ZtcG zDD=ERbCQojUtpFz59t_^v%U8T?Xs<-+@XV0NSkXi9SzRpkc@7n5jTXb0Jd#)ducR} zHqqd%p^);Zo(nKWlo)A@OWM0(qiF$vDpxDz)CPqjSVYClbXQjXYi*~3fW)mF%XM+0 zmcoHAewU-3f$z9zMUo)Kn-25R625ogC0O^Z-(m8wNm>RxC9y}#cz<h2YT>I@4`|2< ztv~HE*qO7mkyy5h2)#z~a83<#E+P>XD(BClXvH1(>SpU#1)<{188wcEOdv?It8w5o z2r#mbFYFjSg3_>6DF+RvqzKq5vd5j@b1Qp~Us1U~dT}vy=12Apb19;u^<yn@AssTM zU_k9fG2O&JwnG5-;|=7dwByhR&MIezd$K?UrGOj2{t8cD3;ufQHR!0R$Z*X4+s55O zYJm$AL7f4lCXj=lV+A<>ubtifPWWB0T8SOsM;Bd)J`uhZxDqrN4>imq@{7Ri^uI9W zA__oj-4q2FKmU_LmSmMK#qKx#%aeK~V`n=U5iju1zlLKuKuevb0*ahHCv_7A&+YFA zzDD4eVD`25M-IX$s=>>yfW$MkZMMh$?*~wrhPI0|P!Ack2E^3HyF0Os{Wj=G@B4ou z>#;xhFNci%g7_bPD|qPvxLQ8}v23RC;U0*Jbnid+1MJLUGs!^Ra#?13N#<U2{PBCh z&aEA#31DI11dUCwBYqHL2#D18{3cWw2*C=nfhvDC;ECp-m~$@w#R%e4Jc$(1%mDdn z92q(a*`y>|Km#BR<r{&?Kklx5!>_lYbyM|kZ?uy%p^vfK%7u;)mh{*5?-^h0{r2X3 zVW&%HMmn_TKcIG~84!kjF!KYte=s&#43J=CJawAIG9cV|qwy=6x8FuMAxOj+j6<KF zR~P-*=8s?>n+5h3%Wn23y8}-1Z?Fyw8=ee(*R&A0)@cxf57biO^1p1Jz>bFh?(X<M zuP(cc1^qJ^OA7x|wZMyu0{>F6z>BwB=K+~e#ZO!am>s681^o?KRPO*~pyMR~asouT zhnRX0Lr%JOI<x2BQBx4RX#?3<#*!f#FyH{)NdI0I8{k#Hs1r<YSA?{!3-&<xv50)= zftk(UwB@N)1w&jOI5D=TA+S@|VK=1#lkC`y!HL)qo5est#JrvRplE!k6v3RO(AgJ5 zi9dAVBgae?j_G0eMzp|qi%mjHs~bOUVN6*69<A|GV{7ct8JH6POPb%xr?67aA*aRd z8@n!^U)*kL*k)}4%G-@CsofFIfIQh|_a`i;xOyyLuD?t)l%uid@UEHa@D?!^>g_;< z%Rz_;m)7Lmk#-J@_V=-f&j(>IF14G$a<Vz$zjJYc1<=hqJfGrVD}Zu(1a^P*N`z?K zVKvwV-feSAJ-q|s#)Z=u;hma;`|NN2;%|Y4!^%3aY=roFoUWYrU^hj0a>4;s5PAdf z)p|!p>_QU+{<-l=@*3>>%_u4eIxhgZ?i++wYBb+re5#lqnEvuyYz90nU?AK~8DXtf zX#+FJ98`@j*-c<q<NF!>rfnJSyz@6eAdFqpHDbw0!d#4uC6Cnac`KBT9qEOrgf}sW z%_@Y1CGT!o<`urj+si1{i=#T=4w3UXhqYmcV72rAjb50O2%uCEUyt@uK}gRjT~pNZ zE(S4Sbnp+8=*;;!8grO-3O4;@O}D1&xYjgxgzrOJ9@tF*HW$!7ST$(r?qaD0xS~pm zVQWny(!z^}a>uu$H<8<G$jo%!e|eW3X%?0H<8g8-4zT+rV9eSRflZ|S23-UWjw|Ph zU?X^Pl1nO6_Sq8;H^Y)yQAJPy0N=|0e;m_SLU0fg>G*l)_rRm&4?F0e><eZ=NCt6x z1|pv=hC*g}Y%@==j~&|Cv2hpBK(gFeDMi?*(SR%YYipltyLYCu-y&u}#>)j$N`=-l zH)I-KLR?@q6d=28WNB_>u^)RQHqNQ2*B!flY%iP2mn;n}C9uC8hX~5hq5mgfKrh(a z+ka2Sq{}mF*cW;)ITqY{5Ps@Fgrm>_E^vn!$)+0vs`;Y4=Vk7H`NC`f44W4$2tCH& zab`cppdbjQ!v`RU?rmH)<JJWI$sgc>cCdRC5(@nUn?b!ZFiw9|jTrtnI5*y%Whb%F zXTbJv*kix=Us>7gJ^wEPJ;YRnBKgP2eRwnIpLTejqoelhfbsRkneJFVi68Yr^c^;( z33!J5hB^51_W(y3^fIs*tqmLo;YYgW=RbR4F5Q5z%?s<c+u9a@?Nq&9+XkN#(hRQ; z)Btn$vMyp-cV`;dnYRlELg{jx8qg2R+CbBMDDU5&al7|}*E_Q<m=4(d9_SMhw!bci zv36ZT@GlBDn=!tRwz08M&`It@p`Z`it3d2!Fvs?i+0QU|i;V)=iSkcbRbZ~Y29aPP zG9>!}y{{mo9(-Y!EHkO;5IFGV@vzc<!l*ENuykKiWD}n`U@w4C>su&mfqn!gJD>tt z#{atoKimO|v0qod^VNgzd<^FL^%raQ2biObz(CpY?6jvC05doCHY!QNArLCy@r2S1 z9`JnGy8&wP&Q7H^(fhBVNDz_#A9qTyhfNcOYR}M@#~(lcAL~gr;T3-WzHGz#R4Sex zCtZ8CJylT#I+(_P+1bo3vgR$|?kf%Q{u|OYfSK7tWH#e+=1F!p!EL$*mjAfieFs^- zl=q->b&qHfbT?pl9ep97V_&|_CPU<^W+3PvF~E9NA*^Hx%vtOdjbMjpAg%)bx&MIE zf4|8K(j%e2?~pywz~WYM22gX1fX;PW>3f%qa7n)(U}@gHO13KsN6_^UY9*sX-^&;} zVfuUbLI#<^Mj#DqRXhnt?M&D`V9gEa`MY2ZV*vegZuf&?OOR}KgY`~DAgklK9dg4a zv;%;vY9~x+9e%uTw015d>@*78pYN{(U%}T`=m^|KJq~d%MU@eG;li@*A~SLyZp8Op zo$Pbvv^-Ac{4lc=Pv@lD_)SjUD+pQ2C;zb1TpJo+Z;jiQ4I`6MfhNfWL_~&Y&6J(s z06p=)f`eOO0$bA}!m?{=C8kA0+Ois0u&UauucKosBrK=(6_vmI2Kz1rN^t<Bs5w7S zksF8w?82RBW$_D@Idco40@&1l{Tn=X!MaD%l=<J|g6$o<2|%w>vS4s1v|HrN6K%dm zXyuSonxEbjqx3^3yCFX_9n%1gu?qwjAR?B+0z!3f12r6<JYg%B-1J~ig|}k=4&5C9 z`)-x#;a<6_CXgvV3u;KQ9vKOpDBnw)q_6Xiq<biVceNGr(eI3U#~kchh@JX^6{Z>! z>0-PGdHsDn{6f;faxjaZ7V_I5G{j`nA54V5R&|9nTKKgoQu8%Lj+v)W>$AMkRiKOr z4_@v<oW(jDo}?B?$lNls@mr6Q_^6#-DWbM+9n8{IEqGWt78{EA;d4wH=?%()@dwcO zO~-h-KK-EOvR3Ok4RwCSk*yYBgS<LZDzfDXI>yVaA`ALJr7!tbXWs9OpGKY8U=$aH zZ%@kDrT1BH3>a?mEqTW>;}~iHBi0+1(D!lAo*M<EM1@b9`R<_nyQjA^;jm?>r?HCr zsh8FCNmLGyL#Z+JJ+aBNnW8C%SpldOyRDM?;;<xgaZy=tOR>Me%9X<5Ix5ka$x<B| zwhm2TCQiI)dToebcruGi-WhKZGmxyAQeo@KS{+eNKf)6|KRbbwg&ZO(22nT&1cNDP z>rE8H7+UR;-fe>j8hIqOk0$EQSL(66u3bG_hPwqV%*Ey%#6G`navtl_6<b@0c8Nb1 z2_N7M(0J)j9W>dh@56?1=pa}oxmuBOhk{th28ONNt*XJ`&b((<<_l#br5zSojsmoV z`4(ELP=zzfC~9N)IQL2zeONV4)~+EewsFiHJ}0B7t|Wfnye?sFkkhUY9uw;Bx=hwR zO#6BUCE#cMDUiUP=fJ&{PXpn#Lyl)bJUTV@6-HwFL!SR@uINhP>k~Y*V*XkQ0g3t~ zEe9t#>@gRQ=>pz&b<YYHlUk1PZMHg!M|X+lDH^0yES@-_@=BOFteM7B8|UUz=G8?( z`(wwGdpoq^X)+4#CEz%0=Dj;nMxG><uGMDHC|n_a(s@>*R+Z3maOg1$c`Ni+1di~1 z9UA~BDW`~julCp~4&L{x1(nJCCt0l~Q$8j&{*TO|L#2w!GTK9EXD_=g(({g)c?wwG zswuEuWA#N#peo*PgvFoVZEKhgfI7u^%n8r~TFG0m{-zu1FKiWs^&t(R{+PcTjS!|M zX$A8%TGcGQxHp1*H1jbU8?sSRvhA=jNH#UYdHr!T&R`JK+N_vG^$qWy`(xy`d6eQO z{F_$H_)EXOs~h_;ltYnAC|yakh%Zzd@xGa90}r>UKNI=NP)=r5DjY?se`VD+#c-PZ z?x=CvV(;;ZwVxiJX^w(H9-rySRnB90!P>aar=|@Dr?$f<lUX@y?mOemegou_ux6>t zCIssM-=-17`jc(js5(oN%K1>r0ENLbx<OaPDp2GXEBZaT)tI9kooK%LIGCC8DBrGp z!Knj_@)#bw?eP*lF3mHS-6uQGMpKuRCgff%VS9K{Hr(Ob&m=ELD!co;zwf<QsU5bI z-_1u9g}rXFTSVI)b_bPgD__vxF&mWFoN87DHE8}cLv?A40BDRTQO~FbBoI9XUvP50 z8Zpv1ebRROTQ|}ahYZS7T-DJ$G#N3mpn#Qkv!w_zI7C7#Pb|OzW<VoVxEkTOvdIbu zVioMozLpo9<LUPnCe`oWP2>xK^-O7&c$fTvAZ@<#<^BbFndPUx*zaZ*?pd~lUCNG? z(dkCPx1z|IQZui;s#VgtcqHQh^{9|(|NexH#!<A?JymqOGispdla<F|YPwy5?Rj*u zoq*7bm^?1)=447jP<u8?s$gZ3KqVpGaz-h*@e887UG$XL>L4ITVyT>{R}EdepCad| z!`5Pgxe_#Kto1xvoWYQQ3${eCL)Cw-p(oKtfb_w+-K9G)bgZ?lVBSot@>GIb7|#hS zFl7}Mo3<SJSnLcWQuh(54UCS;&DvBz5LP~G);<d;<5xlkO!3}vK`b(Y6$GFkk6fb8 z!;u46J7K|wVhO7e(>BKf3oF#7^~8iQTvUW~Y*rPwp_iii;XM_erCy3DhkX|ap1~4+ z<#KUkewzn|DT-dU5G6ArzaSI`hth2z6qC;j(k!rS3^yH&_#G^CXa2b%eGPz=j+%_I zC=V035IFmA_*lhU|M%&OeRjhPdrD=VzL`Qm$C`MdwrUsX?U3{2nq9~=F}0(`7A*%x z@q&1Ar9`ukM<|@sgCUM%1TuT)4^`m;b*jB3b=@qt{hJK*`b#aen~=TncR-x~!y{;R z=7Ez2SjzN(Da#hYTzSCOnQ4=*liW#NKzU)MRigC49J>LuKINzc6-Qt>I8Gt?!ao0! z)blP5@A5gT@hsgnk4<_ODZ81`bwJ^=A4<T*q@k<_PA9o#H|WO}gaEeY10B*xaw_Lp z9oJ%ur#`NE4F!)U3<W6GvUW)-u(y6tk8C8WC=BFh-5+;EH&h1QCjTS4VNv4}Bq?RB zq*-i>{(DygULKZ{3-QlgL7_Q9KgM8?QKj)x#Y5ltY>3NRP@RX{Y|H@<o|3rL5)x{} zFsR;`>ocCLJL*1(X=%2xD1FXZ)BA+^=}<<f0?0U=T;wD8J^b51tHrZgWhit46<@HQ z0Nmmn+kN|r4-ZG4FX$I?SJFj0e!tF6KXZ_rLqq5>?08Va=+IRAA-=JCfPVF91x}jv za#TpMVM4ts*Qss)k)B{6+HsF`p480;G(xI|{Ea9kj_ER167<U1-XSB(3D`|xP+b{K z<4NkmXlib$U53*H*Y#Q?DB9T-%uW5O%L$F<XD~ctMVz2$E<Ot3bpFspGW)Si{PLtg ziGKR>YeqWvDV^{0>ts90-HuInaM3?N!CE<;tLqb55Ed^Ufe3FQ+*aM#d>wy4!k?1- zviAvjbzU{O&)O1q<c`NAHjfAtyl#LM{~L&a0o{T-v|;AVV(~47EoP?n!5`&L%zM2q z()yshp;!DA$iM=Pi96MtO;KRsog>j_n*z0WBZ{;;)##yy*%ClBAS7wCiIF>#Sx@!E z&F56!AB%ianqWTwS_`q{LxTt}qH$E=lUT;!@a6B4eWS0WkbIr04J?dFqG8dBdJ^;a zi4)?$KKp+Y2@z&-EXn20_6->*-y*-ymz;V)C>a8IU64}=aMnx|k}>Oi4S?kO2nJL4 zhq1ASqNUBmNe*OFa|ktUY1Y1Az+^q7%7e5zzE0_tnBCtvlei;e0EC=NiEfVUQq)@; zo+vf|kjd)O-G1lu=<WZ-+Ihw`m9^0x84H7oqBNx`j*TiJO{xW~G$DizN+*OCdKUqe zYNJRm(gF!m5=dx*0@6`PBB6s2sR2T-cL$uA_r2|RKiu<4kU-8kIeS0**=wzT$-aV} z;)P>fV5bh{UxT2aou5|<{L)2F#;UB&0tiyRuKLt%%c^3grLi=F{ms?BTbx}%Ydg4H zc}@Wt`lb=L#5C#UqzgvG2#U535QRlI9rK^-E01m}TA8GFB}k@<L_r29x$2&v9`|jS zq8IJ9U_+|50_g8}3w4<o9xD8AYS+h3Hn~jr(vf9DM>_*8p*qc7wF}puHSdl_+F#_G zBK5zbEBhxcWj|=Kun<5+Fs@#5T+ORr<v=mWzh`Py*!=2bBU<>rGFCvZv^peBe}C}; z)`6<2QqDx{Ux+-aau=z)IbB_KEUTt}U1z<-jg;!_Ybd*n3R8y3KT#L>Da~7_Td3+( zRuB6Qaz^x%Kiiy56fx@F-ic0)Zn1ctZCBWo4>gsy3aK`hL~u+Q&CO1SVctg9MuF%C zkecIsr2Wm_^>+Xx;{;e6s|Y#)@yDMa4ue%7d`Oo?Q6*6LrqvSTqxlM(J91Z4C5yp? z6w{%wwG{(3Q55XsMCnsHcDA(6VpgXsr0nh1aq|f_si)6Y<SyuP45M6f;p%UPbobZ0 zyg<M2<B(L`0Cx+uEsr%0rV5S@0=Y*%f)lc$SO$s^cO+}OQ20l~9HII8<rW(2D4b{| z^+o*U{J3oRR<1|ixz<(Cgo_Z5GbC(JLTf}*NZ8krb4K<QP`2eqP1Qc<Qv*q_wA`Wr zmnntJ0gum1$;td+VL1tw#(=A#7BF`xKvO<d{MP3qps3R^^b$6Zn)QR3db^RF#_&wa zM_GVai%#Msm>QVs3^B8syd0{Xgm50)Q9J}PflPG}VOQ>LRzp=}gvK*@c#!PF^v;*g zY!P36D<Lb71jwDB^qRx@Tcn`#T3fne+}@EWdg&i{_D4$MKHg&gwY?}*rc3gU5FvW` zx^LQ32TZe3F#!0}>Ic$#@&#*zpgrNF2se;lQ=780G(D<fBb*BBQkgc?ap0bk1lD8{ z8{}-GPqoe;0hBBR(m;RzokoLEXJbp3Dot7Sv+YZ{ojuK%m+imkA#r}+3-n9Ae+qn5 z!|>^gFJd%#B7e%UF!ZCqD|SK?(a8HA=1C7&h9YO~!VHl_EL+vo?1pN{s6Ze)N*}=> zbUUk@DZSojWkyNQ*I(lE<KpMY^kS}22ox<cz^^1!W^y#ukk%8Lig6r#>e}Weol~^A zxU*gOj&iO;&0Y+fsG>O&`+CS&r)Oj85WgCI-)WLMy0rDjgM5^`IZGjKHrvh(N8dX- z+FK}fB2^aurq8LRH<yjmG<D}&l(^7Ty*O?Hl(xFlEgPpAtAI=@yd(OG$#xQ1`LRxk z&q9-R{s!+bu2KUbD4n?N`<D^t)EPmZ@mD6nqG01M<_Y(3Wa`#bZc#%*-cbl6Lsj1I z1zYFC{lSw54gg8fj6#`0NflEY%GhV(RAKmF+_Sq{;8Cx%?g`n7A@NR3bQ*t!-)lVa zIKzZ39Rqf;SNXpeqFQfwDTq`!r5TLD3p5}3s_Xo5IGI^nMXsxocrF-U&6X05L;!V> zZQ`wG7!;i512p^rB$KZGSnS)gQO*^O_5zd#MwIsR--<nD(klE~nbTR<!;X&};!h$i zU_#no!fv+u1pAz0;0D8kGxI>QLVx+7n?ECi%=6o?rsSgS`Pd7ZAp-&t?55|gaEwgN z_ABu(ZA(g?EfZiW^K(5e%BZ&`8M81_B5*DCmdE@CQyu{U`pgo*2H$eF(&az}(`Q2H zzDA)2D9aMa(NL1Bvyy&?tX0vQxtPU)9&V9j?{m?-!nb225r+!vH0p7x#}Avc-@tO2 zm&`pDM>m(lg!<}#jFh;<i-8^W2YK$P%_0FU8(=XiL`c2!sc?WC27mX=@9g+OyIgfF zy_O9m9)+e{=iU4v{;20-i-^Uo;8#GJa}{p&JQ;R2)#2s+7e!Ld>>~{T0_y4Kkh=b< zOe$*Kub|PU>$JeT?$-RmYArv-lwAmD(a$-PH3>w1uj+_=u7~Jib5W{YJLVJFD`p1_ zRRwVi>-p>yXoMd;I@Gn}01i3M9%pK>{|Bw(bn+BE-^(V`&y(kDUpZgBxKiIRgP8S# z26SH_LOpmu;Ebfv(sVt(@TFOIoAeGy6u5Y2OTG@m8ZQXd=!I(6qM6M6d=TDqwbT?K zJsb*K&S13*N4n##8_leyc&&dMsrq&TW0L=4HHquaMWO6^Z1J$2hqb?Z#KzhqO_A}l z<dt?_TMusP%lTL(q5QH!)C-Uy#R>zPwqr+00Ywf>9|wMz7ii@bnh<Jb#zC9FiX9&x z4(d$n`D(H`EBjnC${O*p<Qq9me9(22AG$mdzT0rJF%6Co*w4OG6!7sv+M8aU25RPa z((NOMFu?A~A&|Q?J5(fK;WmSTay$)HIb5+z4mB)Ogy<xM!Nm-&)iBY?p+#6@f5&*k z_rJ-ZKH<CM(8qt}vc|j?2!5?n|G{VsR{nhm>^1(_Q6;>6(*C4Etg>7JNCyA%D=qHN zU^W<2XcG*r*8^du##_^yvh%b`DrVFX(yGGqIj3wZcSgC3E?92WvZqn2ckWuwv?+{A zf}|W(mh+Mcz|`2Op4g)|+Mz>S9Ml~bh~C1wnlG)49<})=|EE!C^d$%OY<tKi?RlBJ zuCK$|mVPO~mxd%02FJ{KP_`>NMB9Rx)q<iZ=b6*t7}mmQ-LIC;1s8<$dMDQ&!CLNG z)}_v#X3?2N`odj})E!}qZh{eR?PH_w05uff55k)2onNd)1lr=Rk5t6KZySyopA&K7 zp6NI&=l{qz(k=j1nw0!?NkCD_G_Db^_{EEDN-~H%zpjgYOOYgROI1Wk^%m;&j+qxb z4R1=AaEnB=z{74S@{Mg(ps%*K2+_#C6^+PRTA%K9q-2BV9x`-3B{%F;4L@x5PR&a4 z<a)$PX$jG`1MJNG#SnimiccXSJ|*L{vCf6VtPyz6dj2S+ldeXkKUh5+DN~p+bpZh; z-i5HG5}3-}>Ev@Yd7=FYaH^43=`?EEBZZ_MA*(1+2IHj~S&faJ5HQn6>yYkxq1aGv zcMJ$pryX-BfhC#r-D>cVK6Fz88(4Szc0$B_&nCCHs1zPQn`5=JZwJKPiPha?om*`O zYC?aG;V`;+3^WQE<mXU6U7ln2ts@09RbGtH(}^SuJVKeoe_pBEHr*VQ@aH#==x~&3 zILa-RmBX#0X()4oF`x6(i@<u3!SV`Q2-1g&%VeUCIbVCkXWIHXfcJeblRn|F)$Nah z2E|g$&m_55LMx~bG#q`ZnHW{31`FxzP`?HcNVF7@b|2-!l?U>O%60exGTk^u!uk3C zjN5ASn05T|My5`5XG~(JX2dIE2YV300rg*x4#PFk2JNR=f1F~Ojo|Z%7zR33)1wTk z470A69L&c&pjo;RD?b{&MLQl`N_~bC00v3k^kOF8dg_y?)5)gJ_NgeIQSYh07&Ujy zKQKN9So<5NeOG>*+Shj+HU>gBuB+WPiJS2!>QNqKdQI+p*(sZ#5Uc$QrgRF%WTY^Z z2Lt}B@tbF|DrT!4yQwrcp;Yd*N_wo@(kN$H^8EX?b?(NQp4X(%__C0rQI?rB`P(54 zOyvYSclZ$u1v-<E2k}vBO2@hU!01*2jEm=o*`=E0_-lrl>GDcNGZnJ~Iw%9EV?iW3 z{p`-n%fTH_i>h(SzF$<eO<upvmpLH4ZXnm>BnJl^*~n6>>LiWc7<*+|DUE(+eJ*L> zfevUkw0mhy2xIhn6mCLMrJnF5w>CDoZH}IqP@T=_ds2}i0(1z}lCKPmes5QFGvDpw z2fkzBSL8kZ)diB#X;+_uPc>#%Z%vewMghOs5-kH`?a;D9_nJOleIg+6gRS;W<qXKN zsb(^$-&VPFF6aaZIASmemJ)as{|7(LMXOb7QyBWP8GeR}qeKBYV}Zez9mq!1nz_$O z$ZPMuyWh3iza{kbipV8KZq`b!)AG7HCFHBck=~=C;emG}E-IcAydh$^JU>EVz0$&7 zg;kul`0Qj@HR)btTGZjiXUIC*HeB4RS{E4kr5a$ovApcf4i%W~@&)iT4i`u-gXx3w z$;;G6-})&k`Qzm(zty|=ak5b3kxKB{pmZ9uToJW5kF;Q&!>xxbO;lc!FT>3^n542= z08S_zgO{NQ<g#y^9^FcG95hwfF@GqIJ^r|u3)<SR^H%84K-on46D4S2wWm&ev9D1x zDNkl#Ww53ww}HdGa@D0MG`!h1TAuiE!PMx^XkQqP&RjMu2sfz5d@OB0XAkj-Y=cKb zAr11n=Nq_uSU%Sq;Ae>3`7xsAykCkUnC*+&TwaL!$KCQM(hv{zh5G`w{3X-D;)R4J zfC-iJih5F+S$Ky60TEP!k@o}U2hv}M#U689GnwB8t2NP^P8hrBdn^m8ci{*+CA4L# zdQ)2X`K(>xR#86_PlHb;v$fH!sV4SA=BdSyn3+a5jE%I<N|6M*N}ONz6(`LwctS85 z-chNIa`=(UY|^SBFYwo?$HPXrYlk{UJ?00jNE!zJl0xsun;>tO;@Pgei1q3L`4`_k z=?O`6QYc7BIs#=3b5aRDBXH^4rTyZ3tVUm-7LwtOG^TaG`3F;>hOZP${B$l^wwxQx zkCF@GEsLwO4G&{h73x|q9MC-KZ6pprp(5PphQtRxJqfN}aDUQAxYATN>`ajjpSgCJ zaW5e0FHZ9g_EEdf@G)!@?xxXxXa>P4*oh4H<4VtXe#gK+l&LS`JXU)@Rn?%K7;2bN zZ>d*pZ>H0kZDLvN-oV$3^BA2WtVKI383$V?K=V&ii=r|56<D%cc{jdHp0lWsm50Z{ z8ow1GxiEPbxb!J5puJ^t<<OtzsgnmXvV(k&U?Tu87Vtr%9#pFn5?gq}A`oa9tC&%U z*z{@2ld2VIp253@WAUS8=P{%;svuK|7tldZ?<BsqdwG`lm?dz0u0MCE|EtuKt=MN7 zpVRR^T@4jfF1P*{MdL8wG;%PZ;>MhN=49ROus#ts4WMC!3rl>f<2t@AEH|fD$d9j1 z|5*7EO-5WoZ{fZh&tQ}kFCv*#+3T53HAo_*S~*C`zR8#QpLrTPNE?b>(T}D4wfvZL z_p3bBdKzH_9U<#*OYxiJe1=I5Y^>fj9AuiW&m=H=Uvmew>h2Nv=ZjD3`}euP+E$OP zj?LY`HG_;&{s*JZo*%9ny?E7>JEXEy!l`3;MwKZ;-;m_4lKZg-+rpj(qU7|0nT&sa zGN~-nx`G_AR@d2!P5Sf}P(isUX3wq|It=R-UhQ`~M;+WCR%<Cm6Y;E_mv3IlhjOIH z@YgIVg@<{~{N{qTJgL;O{P^A_{(PMX#(m7rEMK+kPS1bFCTX#XkRjvB56})=aHY)a zx)({-_T~cV6$M(ILQtV_5SbKo?KwJ@5X@O3UzXfAI6d=P3K_|_6y+ryQ>e{4XXn~J zCb{_VEwKOvAh0aXV}Z6_^7AD;bMWzy*Ud4*0|)o@F%gbFuaCmG$tOZe?H4P&m1>p~ zgt$$FNPbJMLsZnG7{aUk?Xk%~OsugC4;=?um3{l8$d_tq{PIo@2sfdO5z+c4&oGT= z2FiSY(k814Q+QS*+%c@e(J&94+z3f18cF_D^_f(Z6OLi0M$WlZUl3OWN!+{P#Ps{| z$@Tk#e|0n_fL`JXj38kD4Fd+%{>&f|tr=k;hOtnAl#1$L$Xd)LDv33!E)3^?GPK_2 z6%7$jfSfnL*s088c$9^nYoqmi+<Vu@mP1+RxbhPo5ijMkA;>akB#eq+AMYK&hwvvV zEs*Ciry@CCvJOSe-QTYcVj8h;+TJ~r!r4Kj*311BCAl{;RA`No|AGGuf2FPSBW&w> z-(SLA$Xs{ApOHZ(s8Te1F#(q)80Y9=W(up{l=MD=)nqSvyWnP5kY>o@G~)#e=pSq} zu$6S>e3DFIu$OI(`@<S$Siby|W?ld<G*R$P7JA<664IqdGQ1Pi(0031B=z$*7xxT` zvQ8wo@PW!h&nNj;aEL_H&vt_qIWB&H{psiuS?=^9W-g`5ny<qfZdKxIG~%dQ0h4Sm zQ{gv!HNZ<b9lhfJ%KHv|>#~3_o#<Hoo`p|dPu{fDM$4e+2~Z=_Th3_U4zE}Hx|<YY z3XmP3W^WeQ)IpoYspy?6`^-JHwPYX^mBsTT51s?%NCjD(L8Ts4O>Y=gO#9-yGTzz* z9;>{VXD7!P>J78Z+K?Tk&@p(%fd2-mo@7OXY;M;*{q93P_URh)guK&EgTSk`Nwjp> zaSQno4C(LirCs=EexOQA6BA11W08qZW3lRt70o%qxw~Dfl))HdXQ|axBu{65Ha>D- zKZSTBWK>U4B~&q;{IAfZ<Nq1D6mnRux7}s<t{hSI!gsQ_{D5GHL)kmB00|&F0_?J5 z(#%m9n3xDsS+%~+$WH<~X@5Tcy<yOu7pR^eG`B1s&2T$JQT91y4|<gCpf0v|T0R=% zv;mjqfWyp11uA)Kf9BQ^r;XZOgnlu#H+aeGIypYho3yBk?HXI@;AUr2*owa50VdA* zT#)xPyh7TB3zxk|aaKmbZt4()O{9U{XSgGOkWp@Xn;x2_)=^kQw$||YEr<fT7*M(h z@5&25mC^&<7u3Ga?d9p`vZE$rMK(N_Hdc>qvFh099xdQG_#*JLXPvZcXJ@ky!fB+x zHog1LF~I&5(LvZ*2-G6yxlXqti1J^w`Jfk(g<##mw@R$4r{xB-pDkhXyFDk&hijW5 zw}`>>68io)mgMjc`m<W?7&}0J(hOP3Zd>LVy%c^vI0?x(?JyzJM-<0u1yivq436b7 zHo<KfOBSVC@Qcdv6gv5Hi`#><Hl67YxZg;Vz_c{`*e9sYq2A@z<&1g<j>oBe$P&Zh z6ct_;n%m8!J6<)3rzXu*s`W@??j>6LM2XOR0?p-@RFISTfj?mNxRq%o2a)zEP9e(b z@asTZ`4OHlT@t48a+yQ_!q=7WXyb}-*lqbI1!YhZQ?X!<DcdPx4#+OrPCS102TX|K zwEqR7f@Xj_VOM$aW(YUTV8Y>)tS;tkU;3l+k1?bA`9?Pur^@VY*2(47*O&*`_l-99 z$yd$X$=aF~YtPGN=>4^Ribl^}jM>^4$eqkDy6{9jBW$oz^tpQn``!%$Nin-emuc{9 z0!FidQg>M6&{D>AtqQ7GJ``T~8ss!-9v8XQc%-;38Gj?xAyNUUP=BsYL2FnfNEF9x zd`Ym**vIlShP5`Cd|waj1DKPSV*1Swl3c*&WUje|0Do83|FBY#k2SW%JQ`TD`&u?Q z_Qe+|<`m?1FMl~V5pHS#4Stt0u)O*1=s<nnm-&}oIN;bUg^{@Hiy}VR?OzyKy`46( zd>J1w%DUkL@+S?wjW2G|2k$_=(!{MZ$<WOLf}6gMS^-bK4R>#|K>@E%lJyEb_(f(= zPbV~d={T%{sjs@cM30Jz5eP?2y^P7MLlv2X0ph3IqOvW^Xi0_wVp9>VT{LcY%Q`a1 zLmazWU|3z0uJ;x=VkA%3;v4w`klrHnwVFm`>Gzve%j@rZTd@LViERM**#HC-JXeLG zxv*uAQWH(FfeLi7Vul^%_Vz)EggqsXrsQvh2bdR<OMQEkF=Wls8$24HY-aDKrGt#8 z90*uOlBLRSv5$i+7^Z&ANrQMp<|ma3?REj9?7d+)uKY0vTyuYhb7{G~$HOI|rlU&R z6InS^?OMSq?VCN#0)Z#?y=Ce$La6N_K4-?3i)C#}a&7o~$@8EF2T7Yo$Zo;_`Gq9U zXvc7A-gH<CiEUrDOe3wRH3LulW035rOWL-`GszL8T{$(J4LL_?IxGi*ZBZVx-6c-L z0iuFGvyQbVcK$fW6lW;Zc>U`x-;?5@Fy;hvsOKS^9Ib4ttzAvx8EF;C>1Aaf8BqWQ zf_vn6nnh)H0S_JTIa3-R<<}0kI`^^v62qyDMN_eg*$DTNZ=uqrIUqu-?D6O9NjHT7 z?#ES-R9v|=vX{`cXN#Gbn#%vp;3(deRR*kIpnX!Fl76?3!Aw}|<>lKS&8K9@7*)BJ zg1+iP-Qc6ZOP9=S(I#xLT4a}Y;ZPZvVhLfBG?&g%dp(8kHzf@MS)>5db0Ft1Ea+lC z+l@q<iqnsin6e{s)}dgkG(Wjf1|sW^f@GtcDdARdtKL%^(WdLE^*_$_J(Bv7F4d@A z>y&GEYyDTHEclpe>a^aI_vaZ@n*Un*XMY>-x$<{~+C>e`D20w#9<vt0Uzw-P^AR=P zgn-M{<GPsS<CK1db8zE9QR77JCR9A)6H5NM&){j6<BhU%DNFD2$LiN3=seFSk%}wP zhT(1oV)k4!Fd9Y)xr)j3HHLHeNBA|7s(p=_5fo$1{o_w6omuK>^t>qbOo1U#1l5GE zsM-{QRZIGw3(648oCd`f_r%LPe8(mBBvrdW1t4g4T^ozb0lT9QfyyK>$^ZnP?eaWz zboO|jd%;n`0(-$xW_NW|u5YExhal}#EQ@(~A|@@K_vWf|5Vl>?GW%Cl=LC*KO6=EW z;d1~9+o0`-^Y^r9C+BSw?|qK>i59LCn>lDVvb6T&HUX#lILD>@JlSdX^q=*&uh<H> zxD_&$y`#+1<CgdPCGb+?H@FbFM`zU*f269hUI|myMqlEfy%I{noX8d?KLHOk3SnM+ zn^4l38)mU~9b?(D^4&tu(4gy^ItJog=>k}uOBt`NZgi;<?j`b4<vT2IJ<Bo7#%_4A zmbbCrp<S<f@<Cy-_`>5+t_^nRk>{6ttNba)jgxxKjY$j6g3tMO#(R%_h+OHDMZ*>l zJFA{r3Li<I-T!7o(02g<az^KfL&FhXdwR1cp3i%}8ahqe1(GnhKHATf5o7wotTxqu ziI}lqEc^BNaAYpC$5I?I2Wr}%48Q=M-PZ4fF2c(<swD=vQENbHlDpg`Jl*M_lEcFz z+&~9==5x{BLr4x4GYykcQ&rLf^p_`VH94-`CFC$gMqGTywowDn0-$vi`R>-s&!eXV zqi&&Z#zc4MXtTIE*y%1g{MEOx3g#6*QG+TIMG(TTAj`lNEi+JbVTzZw&^Nf7e5v%! zT<O%4R5CBDRju_K$+1V%SAjJ4ia1;rKmb%vH^Fg`f^Bc(iL%kfAp~58q}pG)6FriS zl|>unFDzxaPdm50NR9ggJ)cl+H3&A4@f)WphI}1$c9-UhCUIIYu<x2<SP*;DdacUY zpWtv2>%7EC$|?G?Mw~qz$zX^^_Dqva(LYZYHf?m?A0t4r=tNJjh1o}a^Xj*Haj4+M zah1{)x}9I(lQSr9!tSuBfv{3x0G@(MhNUWKp4=bKtiqZb-l-Za=8vC>EcMk7VC8yA zS*s$M_&``=b1|TW2<&_cDqlbFJj6NmG=_h=ts@wA-&~vs@@xkwXW9ZPrtt#7AZ>s6 zO#49DGUwCRPc}%t)nq|&st4an?>TB@YQ1%X_1KC^NWp@qz{l4B>FLu<=xH{GT`(wW zsJ8d9^3k%Lo3KtPY+p%fcBduPuU^O`8(;h)->?dZo`_TX;(26*TZUtKk|v1T4bEh} z_bbnZ8YQAUB80jPDd_`!9wR?eMT)GRKf8B}K+7<J6kEp7grXkcm`!j?zk2I&A04}K z2&{;~HKiGENkY}ZJ)S2|UD%X{d<W^drcZL%9iBnoEiZch(tPyF=4w6ss8C%b)OUd1 zKfJ~_CDcfn>%9{OR;Zl2e1{$aCp3zh)EplTDJ;o1HGZ!f5a8L?Aj{e4&?`Y@yMg~f zk|BzVIYJeM4s||JFZMK6lzx&dCWh>6EA%Cbg-<<t;ikia0yIw&)SZ*sg5V!*rk#oz zdy=?>%u{_ONrY^Q5^ZGn>v4=mg+0YQt=XY$M6*HkT1Zr1`4-Z4dBYcX1Gt3Gx};Dz z;?nZD<(z?~a0H!=xOZ5q1wV>a>NO3%?jls}av=TR;H9bGC{Gv3Y?rYt5D+Rslx7mS zU4E8t9-us@$x98nX@-@KjZtyo6xP_LrwPJAXXqQ~Nb~5ADlA59sWskXbd)E4H0ssB zf5t9-y2CQ}zl&W$=ols<(<_9>6{7C7!gx|)XxwDPS`%EE=51Jkr4eR}^&S)v&4mkf zi9N&DW3UNremonX8O!Vsvb?QtMT`wZdg%NSI)=y)j#AUP5Prpmg`<9rvr(j@ZJ^9w zHgAvOY2xYi7Xzk*oN_Jy=(ll19yZINTD^Xom_n_mMDT1NCS<HYer0!TA*a_&h+cq} zY%!00l-st3n1XSY=(t_9e(UFk=JiXauuPO|Fm=-S?ad@54CD;0Pf4QKWWndjLuWKd zR`Ip%Ztw+q4rsn0pM&O_XyLaXHDHely|pvd(|BiQG`EOgv_TuhGHqhWiW~UmN<i=| zOH|+_9kixv>Gv~qefq7@1~v%#vMhemA9w+x*0&FvuHKx4$XXmHyc3A$fPj6$*~yoI z8o?JAd=T7Ub+M;n;xkTY!7mBa@!&?j|DCMFOHWpEEzPYDR2fi>kXx2+7_d)8_<Z^A zTqQ}Q>wm;5{R8lHPgd5}x!QutI;XkVQMi?=RWrEo)m&w)gbAze&Q$8_kFEldptSyW zStEZq&qn!d60Gs#>jgaAa*JTXX^0agrEr_KX?SsqXX_Fld}CP&pX$0J1QR3bZMK~f zh9n=E_ZnNqcWjN0y4^|}1MpA8d%7$$(3Vc`{4M0}0=n%hytf1CEE^#@y|X|ckV&0l z^G68P%OJ4TSj!U4je?#P>{t1<`gDWJOQ^=`>a<j>KLCvg^D2(I-F%eY&pf#l#8wmB z3(PUy5tC;0Tp^p-UaaZ{00~|fsY}O6+Us<|vW;-U$cb9Le=R>YhAdrU%ZZV<eL6w+ zQ80y$nNj&9%BZFFa(eg<--CQ;;lr{)Z|xxUa7^s86re1#Unh%1`xb&&P4kNtW8&Jk z?mYAOHbMn{l)WP1tqGp$1{xZR>r;g}fXbBRN$*fc3Y$v3iiPl<z3KW*U5e~x#oFKl z^gt>b?MxJvw9AL4!zSG+S+2cs>8u)`k=OhCPdx5$q7T^&HeQjcm#ztp)UQXH0bB*5 zYtV1dJIP_s2_yLE5J;R#oEWMFJ7T~Uk;N{g`xkIzj3@;ILH*Lw()s_{p0V@mj#7eX z)go|ZDI@4Qe0m`8thShb@n67B=I(>%-=2G(S-n%oS$$>PXM{Eykl6ljl*&UaQ+L1H z3VpFLn_XxK;5n|nDCum#3%Vean1O$yquYmHYRWDQZw^1^3mc7xzkTn$tIghdAPj`) zK=!`pDKImjy9CxQSa`1VdjLVV`NmpDfXSv~{qg^E>_^w9Ps)t(_N<S>!rZhZrwP!( zyFSaa8X&Lkzd}F6oEks8bn~PFFoBYxho3hX)OV>ly^j2+Fsjf(yo`t;F*QB~ysZ|c zr+eciLwq1WB}J_(9{PEmy*mGB{{dsOKt$Ulh=M+plrQiX-oJn8&+B9KP*otseF8j5 z02;O|Y>Z!wNLA4kWdy8sF;~uvED>-fzpekfrie^VQTi)(Z2|c;JFbT8s1^nWgOfXn z;^Y=$g3T&g>z622GGH70=1)I%FBHlO3ZQ)~JTjW{g)3s$=`S}Er0YhCq7QQj-{tq6 zyOE&a<IV`uBycEuxTZrT+&VF2_Bx@CgU;xh+EwV4Shd}!D-L&CKNBB-;Gkad4~EWa zkc&X~qh8PF22THf3>0Z3l`9ScaO)88U)ld)1_N16JMMb_%_jfD=olJFaL2~dnVQ`$ zz>4xGL-x~MUs`qzFy_!N&lNg#6LS+dy@&d;2)o!hV6uz5W+x{{N2&!P62UCh)v!j{ zC^>kgA4jCuD%v0lU2gY32P3*~;ObL}8i3bEH!IO;a{I3M)noV=F5D6WmQdhU{V#J2 zs40Ot=49M6S944<9b5UU;BQywKJjwke&pz&BlqY|J-S7vW@n=&2gIz1fnVD8Ug9F% zilzQPyJfcN3|iye9$8?!EMRzj-t)pzb^|chuZ<-}V><Q?*}pu~;6+J}V~9_o+uSsO z=j$F_^Z#%E=6b%?9>{bS=wRsZ&3GvQi;H;`8X9_np(|Pn<Q9N#=<7e+H&&=(1}P$@ z)%3_(voPR4F*Bf>`E>S6WZaQs=xPK7GNc0V?&vVT`hZQ4B~tO5x9PXrme4H$(oq8C z+hI;%?+ca~+4pI2?sc^9Z9v<+00ABj`buVLjs5_EKl$5ByOAc~rmVy9kNJ1|GZ(|# zS-P`ueq};wdsOKJgSLjlSw6zUeP&mefGY=3`mUKb9NEdG4|IXHXJ<V^z=NNqK0yaO zwnJDyg%=mGev)&W)L|UzbK72{aE?`vDlOiD^)L<6t9@{?IMqFb;_g>ur@uS&;7bF# zPx<cy{{QUOJOsQ$rK{w>O_=y!SsNzwm~+SKHLLl}jBwHW^k;$odw)aMK(Au&V*wy_ zzsZ?&$>|i(klh2&L9vuZwI<+Y13|Um?e2MYV3LU-cpA8AceVI`BQ8%3_^tHs4hVn8 zQ~&$^|MsEg(#dao4`IgMvT_Qn%N2LWVETL7GfM%V6y2-AwHx33A7&|;|J9leUIVPa zl~hR?89m?RfAqv?`t$r>ZmrJ$?_9i2pk=1pqyV1p0sXxL&)9drLOQU}baRBWo~GeD za`IpD<!@`a!(g>v{kbDYj<6l0zh=+iCRAAYX4fjnfBN)&;8C;P`%oh3SBPjQ1?CBW zCw;j4H+X;h(keG*EB2;Wx@j8_MbW}!2l`KZaviPar2pgi-{V>QI^D5q#`G!pAE)g6 z%C526qh`aHF>wC+&L*vk?%H||2$ze)#{y*bDOc={z3h8dwtt+=yH>V;@6P`l+;TUU z#P9gdbnYMONiweiNYIq6Luvk{#)tpFInGy;KqVL+@TdGP`4C0-p3Q@a5HGT=_3j}f z!_2RSUAij$%z))kjkP=+44X+J{A*wDd_TZx^oqf(7<gM59?Cvv;l00WtB0umV{Zc& z`pCBh`hZk2Gq|%g81D0vf4%cLu-Jyuz+|i44uWU8fW0t0kxVzdfgcyX34FA%=8aM3 z)waP`z*Wnjt+^KJzJHGX_b=SisQ_?L|2Sa8z5Ppcn`7T6i@%>xGoyikovr;0Z=00V z51AGJW6%@b16{rcUgQ7#F1eNQO3&{1TJQ2+`>#*&L`K}KPck66ZjEr(TuKQTIhlI* ze|dNTzgCFvv6)%gE{1c@lD)fy1QhEdUVZN>f4+I}uUnD;o&vgB-E+=t?<q_JyC7qU zuZBSy!+(FArh(&r?>pSTX;_^xd&{XSk{}3MVs|)jrQciNe*>Pr!r#f*;Ob=WwuN@d zs36H~4}(iLq`?5slioX;peXHl&2PR<-L%+y6K7_9c2AMqx(p23fTjvO<j~CrLoUCL z)35g5&|C-L4yae#%XC*CsU`t0=p(V0+^^|BbHx}~z35R~^xwF9D2x4lN5Gf=H-|g@ z-u^!R+r9c3%)Rs!2ND?6!AKV$#Zt4S?!82i4x^Av5M!rmKt6Fh5JUVczGvwAdrtmm z68XQ60Pzk2t{(JUX3A2KN20WnPB0B!wcLF9WM%k#L;0aY(OIB*loBgKs~MB+qVwkf z+b;!O?yH8M!6u$i7NciK(EZ4gHe9A+yRKJoCVLj+Yo2ku@}8RXj-1x=Y=0`>;JU@< zK{12UGxTy#*Kd5}n&UuUcbIYEta>ky&;}0+$a4S=Vj|c-v^A)i_1;&^p+IY;sbp1u z9D7a8qWW%ZUo(D3>Ypj=f5#ze)Hw&XID0ERR(A@$hPHjWvIQb!bSf#!96?q^2g~fe z3OolUmda2r5l>Lmyy=e4PR;oe8B=t{c(dyoS<+VP?bc*)na~E-mIf;I_%4u2o|}|| z_^!i&z=K*EU(-U$1S;rz+WNt^H0Q+okIiH_=>ZJDlgmSQ$Aa_xO-F`HCeo7$K#+RR zP^TuVt8M*E*+6$`hHh~BEZ#?(EPjGRi2HPpx;&8LpM_UaqPkn$tJYP?Sh&w_LiYoL zHcRGK&2+z`(>u!)>dxVVRtFz`RM(O`_^06kOPBqn!4W}8f^*Xhu%qRd3{*NHW(W9> z1Pz_M5fuFw6YchPlRTG`ib8%m%g3uhm_tq^JxS)#AZ<a<6NmjFf&244&%E@sGK(_} zK+jIK6Fg?O<NA_aFH)fp!Q|QM9b*z<rWH8i%8FY*>pb4ZYtLQMo>PB{gUh{1mo2mE zGkEmL{vO|rlFU0wEKy5F57~#=#tpZp4Y=J8cOkgj?Im|gtqa#0>ks2w9TgF&Vh0SC zGR3vN7WUopaJgCA&og+>K#W<Mqogi1KdtAZgWL2dc^0*kr=5ArqxS^Nw6@u(R)-<E zoO)pI2~ZBB8@L!4ivK($uM;!TN)j3WGH)?<{XiNT>v21DoDHA2^7RBa8*9itofx~x zcTG(s<2i?&$A%O3lif|0rWoW!w)XFyMw?hs^VRzz2q7V%hdxhQt=p0u4klzaCOqp* zX4HM~X!M$rxah`-sjsLGBSM|gaUIxIGy_P7LNO8X?P^m-8$^V4yRZ53U&L}n32rF| zN$nGbYmnu|6eId6)@{!Xa7(pP6k%H-Xti|dJ2aGC!nz_peqE6q$JJOiNM`l1yusUY z47RM2ro}F1>|_7$GyYQyJJkPz^;}S~@8{>oq`|o=UtnfZ35kCLXDv(EI>ge`*;k<7 zRZ*!?xN+kzNaNuysZnady~;wk^7Gj*nDcFx%LPwm)tPy!K4yPYOo`$8BS0atrbH@t zA=}QqC&+^wU1yMS#+;{nn}l);(TwvPmOb#vRng9N>gKayBG#d|{L;InUz^*Ut>1Z` zQyz{Y=d~qogm5o;&X85PtHZlO!Uq$om;FY3IF%j#n5m94n7}XNM;js~e%~NjA)p|y zy>J066H{?BU~{&k)V2$V<{30|EynWN1wEMM){GA8YcX9Gbft7!Nli>L{7J|?kwFYK zJuAu?GPdYXQgovU9knflDInGIFC9r}dwDV^sIfnfDW@}~y*^}R(+{s(_Wl>*_>w-? zC4{Lk^J%n(@AGoIrwQC4b0Z%F-};FRIyqc^lsK@J7v>j|u<~`p!7==)ZCd}z_sF}( zIsO+O#+&WE+dvoHf(_#$NF}$FeSFoQ^!a2l_@*lJjmt-zCE1v?HuSztA@34SHl+~{ zVHI+N43O&|Ud50_kOHP|uMI^n+>-Z!3+eGz;PK4|v1|)XZuS9&c@HDr3vT8yru7oB zTxv$7*E$<r64nOxck8v}bf9BG`nFEteg#t<Iz4LU*`s{2M0-&q8;P<DQLy~9^VlZc zF*S<I=4zAKpo5b4fU(c&CM~O|lV`Sf=SD3(oE?oZ|6q=M)ANEg#&{qTK@>OI`1a5| zbL#C)a;se=|J?#x2n0c?{w~j3Inb}H<3aJKWiKtSaOVj+W^@Gk`1daNyVq<lN0FEK zWWV!Nj*si1Y&9+t^N3-~F|er5u!pc)RH%?e>Nv722*WHnS#gJ!J{k0$yo|@>!W=Y0 z_$2%kuSuH*^e2=zw0FiKz1B$+4GJI`k6LguUzm*X82ZTTXx+O^I%)1#PQ%|4sqJ?+ z;1*@$UFsi2Otqwnk(lLs@tvo;F_Q|j8#iVGKchS$gl-fZGSjX1({M~@Fx9~@(iBTo zR6YonFo0_hlKALHTFoU;V*S{CQk0DZ*#q&OmzfG*3^pU0yPLe8CuWr2w;pb`A=mhq zQU|M+?}I%AXRVfIzjS6h>-N&#Ghbh4^7kIsbA6H3FC4I3Uy8IL)?2)|{geZm4%=An zw~a|+3v?zhqHZLMXt7^%DZDJhu2L(Au~_<S<~=>Gl3wtT;wLbVUp6&XXpYD&xK?xT zz(5@dt^BPvpr`A+(Md@b(Q6aYA&33ON~G9aTx#n>Lm{Wi=5NU(mE$hQBmQE)ouGig zH@VH?+bu4MZ1uSbABaejqjs;$B_&kcdI?K}Eqv?8K|3FeE`&k}a9<zF+OPE)*<+Jr zbrMRcD*Fs%m1aq5e)_<jw<<1vG}kS|Z}S$CG2N!hsIrrz^{obtU|1`MY78(c{)Bdp z%nFLw_ITD{v9V05@E291tPtl%b54x*>TAva5D!r~mQzJKaGsa5vTF%ulCey!E@86} z`fwCI{)qH;yYC*NZo}|K>9_PNN!WXbgH?T3RyUV0F7;hjZQ+s2h^ZR1TY5*ZT!5d_ zI%%?+zumXru7=<D=v0b?V<nEnqp(oKtF#q?g~ubF%X?H~n^F?n8zI^REHlrt>gw<N z$)LHtTYIMcEEm-D|MmIUAe%w|W%!`UFZHt@>mP9J-_Yi&4~b1NDy^&B8jt$%*@L<B zk}D;u%HkWN;n8MYQ}3d0q28|k!wd<GhBQ9?6T$C|VuT`Jryw97zzXtUJrQMS*{g>a zv>a_DeZUBz_>3(VvW~Yp#ulo<rJD~Luszhe@+hVWn-j(AHlTPQ(p8Zlbq#afS>59Y z#n??~#=Ij@&B`*AqfU!3<7$kzl8##IT*EuPN1rzg5jGC5a~GAzxTq|8;b_jIsU5hD zDKWvHnH1wr+cq##-b~#2m6Y@}ZMMDYaKGC@E}H4N_LpC{+T@CFugS?_(KBenRA%&L zS1y*e35gu#Y&3+jQ9+VAkQ$Kuwq1=)wad$P+Pkaueb=+JgzAf%)DR1sMT;w`B`2ru z#SAU3X@*o<e4NsJVs!D)pyay3c?{a<yhh}`?$hqWc>4{d6y~z=jeZ$PNRuYWFSY;7 zKD+6b?Y99DU-IhWvhkaFP?_D)9s0$8_F=1Wwi9w=+XXVQ*yA#Fv!mil0lLLh4m_EK zdE^xS*)Cao|MSkgz?GBPsK|nZ%__axRPV-FAenNRz1DG_;-@x!KI^i=)0ua<hu}Fq z<G9<b)qFHCM|Kd*9m&l~lgKEpbB#Ov@};2g<)5QE9U8j#9|b<Vapu5q9eb?z%^Sx# zSDE<@quCdIR$tfWNxgdho$*4iM<riU+0DxxUc;q!hQoR7!umS=`>m~V7|lexGn6$v z{s?FG!;Su`eCE{t>!+8-l3)^iut?7Q{8diIF1Pi4=aPN;tB#Sqjy><)jLkwb$1>iv zlPq{;n4ZoBG1As+n$H)s$`~!5dmCgvY?cnIkbUe6H*mU)ylQ%uf*vh!^c3X^NM2Fo zzR+yamS2G=#GEoQ48ad!W|taol8{4YSI@ZTww%_A7x6-~+osED*>oQG^Dt_fBX06r z*`a7L&q0+7ZvN1d6w>+4_p82_*p`)|m0}Avu{9E0Ii3CQx_gc0)M*?LFRcxdKPW#2 zEO$=l*K85Y-TSt-C{GNB+WFb{54@pVlXX$E8x(crGU3PGG#a5=q!l;mZYQqmF5-#S zGaho%SqF@_z{B{`P|kKR(t~)=Jh%Cc7RtZ>G%5P<_lb%DKL@w%_lnXE!hP8Fq|3^a z&{`kic>~3C=<c8jiqub6f6)JVUU``1z<DFc`2#$gfuVx~PNkedmlwIvGwFh-)`p|n z{VodOU*=n|bA~XN@EWF{;}mp8Jy1#K;QY~reIKNywK07;GLC;WocSOQ`6z={nQ*o> zd+PA>*`&rty{T*X^D|x;BILL#E$zNN^VnwZ^POJZR-tRIIdj?Mv31vKq)SXBzttlr zv4>~*-(R(KA=9``d+bOhavsTubP9U8cxQXK`pn!Yl3UOov)O|wLU&B+yg=sErOP!! z{^(w3D)Vw5E}2iC%NT8tYM&ChZLoB&hF}-gAYx|>t9S^jIBxLlz2J4c^GwrriR`O& zmakaFKALFKX-nF8=-0_N{xM&ZvGpG7EAb6;RI7`IGNw$Z#7j+Q`jfs`!&8mJ*Bu@m zP+h?_VNi8%wy~lSIZoogFCc^B?v(V)tww)4<EU~uKSZv}5&E)KS4HS1BtO$6UX74< zRZHf>RYLRk154ibpR#h!HS5OMHNU&JT*jX<|5>%f+R`E1;K-dQe~SR2%{McZh>)(^ z<_VYOe+H>ROLWq>{d^#kRsAptB*z<OCqd;8_d^|`xTPIh@X0V~A!PH|gH)1fqCLgL z*qDGqK)LSTID<I|3G9%4Bf}Z;)YW)x{~H&r^VKpD?V9YusYJ~C0qqG0u9E9BW|^Zc zT+RbElkcUYH{G-yE-odyUmm8r9NU^a-0663Q2TxQqdb1Q*2OihnFAV!(`S&5{Fc?! z@$!7UouycrBt3eFe!M%jyRc0oUEb{G(~`WOE<h5!LRubnPhFe4#f>b8d3p4;PV`kU z4<{CO_?nLl@f+^@bTUuS_OWRE=ponYZM>wX{8)7zvT~`rK0I!QYCmdeT2n&o&7Pan z$o@ziS36pS_1DRVLSMuEJ_~Xil)y!!CGY5tx?II1y3ulJOwr$(*&o0kXhB>SJB&rd z*j-vAvmg5lsYSlbKasPAoJ^?;?q4&Ki8&X>eWlsmfw|YtUczsndV4+5j6ZXuyr1$y z_=d3YGa_<)bn!h=bP&<MHQVg5Q$G}xg^qE(Alm1CK^z)VFyqVkKICinh(2M;Wvk>o z6h=?^+C6-W4pM>I&FbLM(-xsexvjuA10QzVAvRa#f@;`5Xj!J)6FzNzj6}-2UGyEj zDY#}YCjd#F*vOyXZ`eKF?F=`s$h?@IJZn=k*&ysA)1Xv%SX{T?1;wwzKEQTWJ59A# zVyj%Lti(#aBR?UDY>Z!nGTj-LJYX+%%>k3%@NhJL1qm;>>48-JB{+ne>wTB^_FTu- zx3wm0-1X3-gELSDu^v&~$0g^(r{LU#n~cE@54#6w(jC<{&$O6TX1q$8LbsZ~ftAG! zWyE#FV3}XW!f^UJ^DTI9wcJv5^`A<~%VbvQ9bb-XHN9muDICOv>u$w<RKjwUS+Xm% zXeITS-*yxaynU^@{F&?8_aC{*E1=*AqI!qm*mEJ0%?}5U8JIGY&;Q|(Oi;m~7(KLS z6Z>Wa6MqF91YJq~fX#173O+<dK4}M@mG!+#s!0rUs}aHu89xBDqt_s_E}_W0kzPHV z3A^bb3iL;E)afIWdChw1y&KGVC){AHjGT6-5+uIrHq0Cicbl~<$>$GgWqYS}UKSdn zZm&{)hhkyGEd&*vc|-uow>fR|IqKs_&^Y%J?Dc_f(a+3R8{1W>r06=A*}je>TMYY3 zALg=7EKEu9z}$d8;<*=?ousDA9e0L{j}F+{UsE~_=&9Ur&-;kHPe?|u1pbhdsZ*I# z2=<kq?emhhg_ozZG_x-i(Ca2!Smy!JiU+-DFXvm|mQxFT^Q*z4tYMJ@x^FJ^1Y)R~ zw8iY!vn~62Q&{?<qBQ>pc!g&Z<A!X(>}Ye?B?Yk1BBgFGUPLTSLN@ej0B()``2t6v z3xvLaIY`}X+?{og=lnSeQ?s<pI(nB!cHOy*Q8gw{IUX6iHghwx90`vRhs4vKJ!9@? zKfzP57-Y{`p+8et;VK-df{v~5kExgxfL1me-4k7-QTb=XdCp-xXV#69kN&z8m>Edg z6n@GL;VQY2s^g>2iO<K*n9~wFauG`2KS$FN^wMFsV0sj4o$Z+Ya_bkj%GXCUaX42t zncmI&w^A|h-Pl*^5GX3^N&=Eos=&~4#9}4Yj*2{!k-z;T5ve`6ui?m~))0<sQ!sE% zr;G*=bzwu!I%%v-+%_v!eQI}cE@NSPTM`V-`bD!IN2OGpw$|O~y?`$>{Bgn`YX}Ai z4QVn4nVf}K)Ekd|$^5?jxb|80uf=(Mem=+35Va^@wf9#qdrm)p?fto1=P(3A)?vDI zawnmVy*BLLjZ-17*|6H@YiD)x3~C%=lDUls8zK!Qsyk2zq&a%XtsE}1JIlj6SPFVB znrinD9@X+x5mDpij>5U!4oPkSi~7fnx0QCXYX<wiS*OcEN)dyHhQk~|jbdkuEas5L zNV+$g6kh_3Tyk%w(%`I|vPj?qwXqk5_F1=4oG8||8`bc~kbd>v?Z(t?g2RJfEFnXC zjdE~`m+$ZH!dwci51)Ky7K2p292oZ!5D(|N!%9~!>9N|^_<1y(=~>n}kpEMv|1cyZ z@9O)*EXeCUA&khkdF0f?A#8Y5(B<zb=dZ;@^2_&Cn|u89xkh|17_<~?aULft^#Nj| z{DA6#$V4L0LAu==ee1GLL$Bbcgi2^OHSZ&XZeCw;TD4v)5xd||v`Ti2tZLV<tF20D z_j7#Yq`j|XeiSw>1LLuu8j?5kCu7qa6ArK2|GYJPpQNfwVPzq$J;pR25sB4EwUzA8 z(ydU>R@T>8rnW5l$tW%HrjaU6dWvQbR$+Uv{Il&I?E${ah3>tq=bFn<^!^9^NOP>` zsN@!Za(m8mb#9nG?Yu~=gC$By{%B-=Dvz@XIU2P3nESwM9d?I-a}Qd<SfMuB0Jq)H zi?chB_o{=;Dec;Wbsi<AA(N12Ee|^zi(Kja8T4$qFV%qc)QRg39XJ>P6C;$$5_JlL z!h#;sh-l0v8~G%}c1r!yJyMOQbCq8~XDa{whbbBr$%mjZEo~aO`CG}6J;(utgd=Bh zf;L#fRBOrsYDPIjnm?=>b0KrQkAK7IQb7)4kkeHyBqhH=)qF8=4Aw<{?j9FKUB+FQ z=RJ#z%0}7^Q~#(5ag~WjZrSkn)Y)w>8)%1AqB~NM`<MSZk~8J}^aiw6ycJ^}d-P?8 zYWKXWXm3gI`={GxprNr~H~}hwb}$fpcboX`VYO}EQoJpX@r?0$td=S3x*BimxKsO1 zFgYh+;5x-c>no2D-nEB*Jv|6S;=$<A%0bC5Uw&3^%u1vNY~xN{wRnM>LRajzYmP8K zo%V&l)i1$*3sqp-|1J8knqL6soXu7)Me2Ew84}#Y<eLJ_l78Ns@7`ajZaA2`_yfN2 zD-!{i{P6KxW`0m3vw}j|!bIs1A$gQ~_n^lzKQ^Yp6&)3oUSrH&La%kPb);3fw<!G% zuzqa+fP^Ys-yyvu*>$0NlUM4yvEJ$%-PL{W^y$W35y>@<{ASBtb3HM@p(mF;AFYDA z0I_u%eEPZ8GoKvGEs<=B!@{~{CO1h1{br4ih6Olf7S--SlwF5EJ~1vF&pRtV+k1kR zw?_23*L|Hc*++(ftFdo|BiX08-_d6d$Hl63!@+limgO~{KV-ZZ<jpx_oqm3%<ZU=M zZ;iThZgdLySNU2@=-1=ueAT>9X+e4GsE!gr0<I&pU0Pd4w6cnrQf(f^v-M+hxJnvd ziKYrehFd*$)-*A9P$zju1IW~vrsWEodrxZ6BUuIlN?zjnH*^Ne=>5^Hfr5g9yK<Zo zmx?&fRZH*P+5{Qrj6CH1ZO4?o9*8D+;ZGL~Hbt_HKfq2kay>*JyZEPsNBcwV`|2s{ zbt#z|B*MV_v%G@x`gi!7lr;a^&sLAO<iQ9TIFLS+T+zW!*=?VE4m~Q-m90ka*C`iW zR=ze|b()(I0ZG^t)MuU+HYxrjtY3PjI5LgVjd}Lcm+YBMVg^~4Fcy0|7=8OJ7iX{* z)7a{~w(Zl4aF4}bjmt$huOb<;G~+UrUkh5}5YW!?2$JBVv-}~(89}$MJBD~;sh$d6 z)16)wu;t6bKN&%mOW($4N^B%!gm#WPOGE21xrrnw+7oYo%Ok467?I&j*lbNzDr;FS zcdvJ-YD-`bz{uT!-GdoZTm*{RU0a<b%k9kzaZt|=$lTOy-^#({mi17`jZbqALznZK z7|15v2i<D#`fXoxbg*yt-BP<zTOmq$!<EWfru*Yn7V$RP4b{;Q8fiF98Q77BIh-~z zvZS0fyrSM)AsD-DZ7^K+EpP5)?~aOLB=r$_dnx(*+o5En0#fja=g#u=@r#3*YqG0i zjz2J-XC#T7w+3<RDf&F?+xynvKJ^?7{{#emTW<w_R#iR@NBKhU*8-k&l#1eD-==L- z$bO&u08TtboiVSYZ|PL!aII7v?0-lHaa<Cj$Ig-mk0~s^I`x%J7iT9F==Q@bK~PoY zz--1Pqm1J(dv(vo`bOv7*Fdtcql2@Qn*p7|%01RB7U^@@D0AyYkRn0fpjVam76mm$ zz*$#}hv#+1M#Xe&lsJ=2e#vH9Z(@-5%?<L`BK(p-*VOKHfo>800%Wv;>zl)J>wCu_ z;&<ktEin0Lkf-dHkj-#r?6L||cWUaauV^3TJ|dqQGsm;!x`SgY-kI}os2)rN@B3FO z+6ecg)8h@wN=`^a=)>~&b;7_@Tw4~FYX4CUuAUqgX4v|4tKva6s9B5fNN4;(5477` zs`}>6m&oky{vD-lMNcC%g5jOq_UKs#Wz!d^5xOR_6Y71UxCmuZ!FsyM{o#RmRSIjE z*2nA$N(xmRzp-uKTvcq0o18U>M9hZY*oQtE*#r{0eBf^tNV%PHFjN83ldL){zk^?% zQL;DX53pXz?_av@L1}DS!c9(WBQ9I)`0j50kC}Jp_D)AL+Ox-$#b;+fTV9Kb;%7!) z$YJ(+r2k&wuk2U)Dk>`ND{I|W#E{uHa6QU_eVABc$$_#&0?ysl^~cmKde-R9g`RQc z$CbYM17mWBoQ-N+t&e!SOwT&>zQr|rJoND~1p}_{@7=z|_Y$UM`_JNy_T7LyoAT|` zXH^$@g*k?i6UpMhgEME)yw4%(DZ}J<@4=L;1j)x(#0GLhL$}bop_>?mNiR397$~Z- zY*^fAbCj0;FjIzED)5pn8XS*SQaj8)QPmQwm?Cj^NsBf=R8(h+!HiBXH=WV--*+Rq ztj>Gg`K;nh=6ICzYifY)7E+WlhQ=%<mejg+cSnRD*_z;i78+PFW9%nR+9J}5XjOPi z=v-3bA6qC&KhJH*zV0O$3rm30H5reU0UW}5ocCkI>1RfWycNkAh4}bJjV|;Z^Bm2` zSaTqrH0rgw-qG}SR_ZWkdkM+-t2nJ6Tan5N?rgyCu-3&>oY@EwX9<i91UB9f+0^VA zt$4lh;Iz(~!&(UpKGl_{J=Ka3irsA&Dqn-VrMZ=QeAx0^=#IN#gPK{e<hFa%c?r&6 zmUF0(t=!wKg{l_MsK#7-IbvyZ(z-abkgWhQIr%0%r|B(~B?Ss8x9^>jF~n{wij<GF z*M_9Kc;TGG71Z@)xJcSK!4=!qe%Yz{ENqL_t&zS>*x`=MaWm3vz^uzwhF?$WThn;L zORul4zVHewxcN^{9IiXs;MY1L{Wy7neJaT9t-A=SX(`iu1Ld(r!oa&hjG3EpqO@c@ zIia0l4Bb!tLCI{_9@g0U#gmDa?)7%-6)gymG0!cQ+r1_=N2Tbuo?-DdDBs&w=bM&4 zYA9mrrj?w2B*}K2s6<~G;PzQg@QM~L8kRpmGy2r7TgGi+pH)#d*!Ot%mG3d7AD({| z6cj41TuC}w60%i1iykUs)EqwkRxk!Pmz4V?dEg{_s|)34%{Vf0>wE8Wm^8Oat(cN) zPP66GVLqvjLK7=HyUhIlkSe?z6vVDl(q`BPCT%N-A0Luf&&wXF!0+#}t2~}?_{|#i z8&_~|vf#BcD>d_<&ZRRM=kxLZgR=LGYijGZhfxpe2`C-~Q3SCZl_E`g$F39sAs~b* z3PK1i^b!z_3JO^0y-0wBrgTCPkls5aKtOs&dI;seHt2cIeeV76zUu@2Oxf9cmAU2| zW6Zf;&Xv>D2b(hvva%Gm(?oZ+^Dn*lKrCcm!{(eJY8JHTZm;xuXlZ`tXlLp#7{&&K z_3?W!#&;n*9u<?94MGI<1Ks5=f_1W!02u%UPNMDbD@nK2RLJo^E$x_^g2!L0jNOWn zc4}fgeeHXnL%05_I9;GOfckxPmL4ZaVm5(Z!|`U-8lR4ywN|<_>D{T;UejR6BGb{0 zBvFTl%{{xj-Zicpuv$IrQRI{Flv{d8b~4q3U+ZH&s=$49g{*Sqx~Q0ZU=Sl^*4MJs zW#R>sbaRrUUgj3i6TEmIn5#{noqRGDlPHcNqN{n!<e0y?IZV&L=t@@)YOs%1QJJUr z<h2~vHfx`L8!W@hKHMmaXvb%;Of3v8<6SZLgi_6|7mJt~H6pr`B^O>iaiwg$>KuE1 zdhJjKtxSS^++j`H(Xd26qx93s==ODU#y)4HJpOPf;{``8a(fRA<<aTdnei&A+d)P> z6lMM+pPw-a{WBoNrFB!nb!+qpLWA$QbO3d3ph>Kl#mA$VT9upKje1&gv)WS0&`{Gz zHE6j{)WU=N@I=#nn(eHWaP7)z5K<kunYxn;-)ZZa3-t71prd=ACs4ZCrk8J1CZ{Da z7wtADWK#7`$X3d8yNC)7dt={}bzfij-q@gffG*K!c|u)GAu0F^P;gyKYR<mAop|Rp z>Lvm22sla6fTkS>-NB=DuH+nKcY~_uM*BWn$}~tc%ib}5^A5PJjDRHt?#bd9>4Q}6 z!=4dhmeq-)U#TpWn;w@ad0&(qM6kGEd1BLc+PNTnF_!S2iZZ?8HlDD?9f%W4j2DD= zP;=eU{LZP}6{tt7kkuT(Rv5->#mQk;%1s1KHbl@93#}tA?At9J=~D&^4~)}o-ge}V zDKNW(^2JRkaUA_2(Ss{1OWs0<4>46#FRcx?yPsXW@lq)UQ=}wqgr#(ZcgEagebR30 z0Z!mmq#=gwj10SZTrkrVvUO=%z56tKJn8sdzaaTlSTD9(k^F>V;C?O@^lD<_qc!4V zqDD_!+hflsKwX@x3;LGMj)>WEB~kd#v1+gLvlpJA-=vG}P>y<N<_Z2~%%&NDPKs&h zBxui?Y(<Nq!oihB&+BlfW~tzA+)9$$iB}@FZO|_jz6>tcT)8?W3u~dq@nSp6XQghJ zU6#!-Az1V$T9*n3<wWgB^YodnK0mENw4A?z;1u<=(7OA_nr`uGfqGoh0@Z0kesOX2 zud^W}(ifezRyK?7GnXimYNG{u=-35a+?t3dsAMg9hrVgKXsnvPS%e8DUAVSEt8tK_ z+-<CG_aD+GEP<IcrdF!|p0CJ&*`HDS$J>vomqYqux&pe4I~-``)9@e`VZTmrD82W& z_$ozo7$1Asq%hFGD5`=09>jAR<ZE}uFZ>DBryaSw1Fsyd4{48n%j*s-$cgPdVD}Za z50X3toK8GkX9D(-O9213<3!t+)Oq!9+<m^n%Es}^&o7ovh02<eoy^N{%9{F3Ou11T zoArZVNUvT6Xlpx#2oXO82<Jq=dz!Fo%$YD`?M1Hf_LA~_a`Nnt)NPf{(%zUIwNF)! zyQUw)IYKD7h-N9V3C&Dgi42Qy-Q-MB)03n9q#|SE<1Y+v(uOIH1zC0dPY_X==s9La zP5AXj>uc{%=}TJ8^%>H49=g5nE{gCaHNstc#BpdvN>b@Pe;=kGaa6CW`Fd{iVeka9 zm4y@*#6nX>$MJL$=n1ceJO1qE<oZ_!>|~Enp2GSkAHm^pD7>W(LBT(l6JAivrLEs| zsN6;ND1*bx{Z<%4+l?Yl7?KMA5q~QuB1&j@dvG0S%0wAhH$8OMr&<n0RjaA-us5`= zrt`#z%x=BVDYnpAD7;~B5i%vS@NEl6v}dO$2lKhfWJtcjUzo7pJeJ-OA}Ppvv^{0Y za3TMqQ)MX^hVY1<Y}Xi-?N$x%ETdTIni)JI3LT~te2c%~&c>O;bF?{xVQiMP^?+26 zKg^qa>gXJnO!CLq>BaHu6+ei32%gRIF;sY*E|#X3&!4`apWC9Z$G4kA27DYqzJ*KY zWhsW2BVcTu9V6PcS2>4*Q>cqCp1^KWQ3&Z9vkD2x{WD5&<0tt;%|OC6TXvRy$>XRr z49>#VB4uh`81EXn-v}(!a?9QgH`*FeWxm%dbiWG<6>`FJ=ltPUqllra?d+QH{$Nq% zZptO{cgMEyz5){yJV}3}k5D<iSsmP4z?c`?iAIk%ToVwt`jgZnMGDC6FG?xvFnzRy zZFtW45XeQ}97ilz0nlNLl!GiY0Yk%lIemcxh@a`aoR5GNyFq!kK?%HOrRTo$5pZD4 zG3CZ#q)%0s*!5h^w=BqG!mK%eJA}(`cQ7lKNn!Du3lZtsLwxje%q2^N80yy?NBu|g z%~~k*kigc_LuK}jrTy6@O1HDQiiHlb<b7_wt{I^}{*XMnjN+9ZKi?;%O-pG=h_b?V zeM=lMfs@IbbyDpdY`kfljecWVK{K0OC9u(yQJe0Q6}2wi?$fe(#R=^u1PJ1b3d|yk zgBM(nXn(86VDdm|^A>q#pgy-OBbvA4!kXoG`sAWP%n61}VaS!O<f1+BkdKu<ccduo zkbmZqt%SjW6r$i{Ara6ACAD$KG?k_cykD4NB6Q$#8tZ%~<nOHkx0C;Ts?*xp>^Nz1 zo#+<<>_i{M)!M^jb+r)oxZqqYfic-5zmzjnsIx=BIsY@h)2~og_Sl#<6C+ZkAs{0c zmlb6x@yKzigVnTZxo_4UIe&_keTi%>Ko&HVdtR(;`N+0Indm=&k~*b%E-?9I#kM1C zu7EC@L760p+U}SW=(Fh7s%3z;qO(ifsks9{Hk6nFSz?9(P)w>};3XJPg=Twk_q))^ zX0Z~G!~l=y0YUPy+xrNGg~S(PF~1knKxbJw%9O>w?qpxq?9E~m*-{G~5)x+Wqg!Rl zu<-N;el%r+jBv{hw>uVn0REOi`+z<3lO^iBdU9sUWO374iJM>s2pwBEq})5?v@Vt7 zz6%9EJ)Fg&h<2WMVN%9cn{*>;zcWEw{*zWp3(5@f$LlJ^&Y!?+A{C5fDI+qv3<tp8 zsQ!+o|KbSfaz%e{J6Hpj`_XvaIojcVFifDu=B;0sq|bmBXWH%L1i|*Zae|+`RZ9xl zhc53pbX$q_bk_=~sB%S)#yI5$rzKuyx7T?3ZGpWA_;@6AxOl04dQ}%wthr^vYJ#M@ zv5LvU+Pl+hNvI?X^pICfDau(-bWWAefTh5%OZl$bIBm-@AJ*0AX=xrJ`ecoY+K{Am zPL-E6Ex2|H<yq1^LcUDhXORbLN5St}yXQ6htb<eeyV1L6YJuBTV}xCz22FH?I{=`{ zGAElvdV*9f?%GiKIy?Q`gXkR_Y#5|`K2vX22Cm;YGq*{$7Do)_c*56}#{WuxqNO9* z2iCJGBK4KwQl56Qb?cAu<K1F}ddUfym4w{H#5OxElfkj;%%*5`ELxM+=P2+Ew`7>R z?88rUG2i0J+U#{Gqc|F<di<p~U8`Yu@VSUKT=jYOf};f!0mT{mAvM&Dx8_pQb;vpI zNnPb(Dap)jZR!k8GgE0FSUWy0)0qCk-~Z^+b{tdtej6qk8&4Y#saLCn;VBu-+6Ud8 zZFWuU+k?+YIU9VY30(vpjzVOEQ!04w@?E7vV5agAQSYCj3l+1r$lS+;4P~bfT|Tio z6|6s&7jeeYxegoinLj`)T5JODqIB%tBDqmcWL$e}VJM-N$DBj{Gj<~$#B0L9PVgTd z<W|r!lyA;eWQmH|@7~Mxo<$qc84Vd8w|)$~EcvJQd&_Zu?KXVBB%9&v-JrS1CzrmZ z^BHt@bvK4#G7MJU9jTDiW!QGL89=!T93Dv7(0{GHPqkWoo!@t?3Qo{?lLhv^M~Bjj zt(Y6t*L5qVMcYM9&+r1{C%8-b1?DTG%!JQFwgNP-#~ROO(_|5-XS{m%n{4Cwv2%a3 z2ikQu*s0$ZskRm;oYUX(cDZbr-WEpLu<vbGW!9{GP*$W*qzEP6c>Yxx9XpLTT*!1; z@<Zu>ogj~+Cy%W4{%!mgmu2$s`pK^5EOG&lQ<#6o+8@oAy8L*~Sr~?wT2{MZI)%&7 z_>nwx^&Of!ZUu+zaDIuXFB~>SWXy#YRKWnvEk0rD4VkuDY0qu<raGrzQ&PdY5MmZ? z#pNoYD80T<U_zaJLSiNgJp8b41uzG^|7v|95f#z4x{54w+YC<fFV}{drThr<c6L=U zCX8##n@~8@U*eEUgu2OMOlYd+I=ZF5D?j(R9(mM=u_InwL!l|j1kV#y8l}5ACRH*p zuX~);PG(0p(P~kjK201e-d_-`d&bzl8RVmZWnSzDiI^2m*Ph%nz@SrI&y-0ovtV&! z^0~l{JS~pOKsBrzU{gCxJ8a9ObF>|1&HK?|MPRCmxD63ich(Obh_#2%hGp#e%Xh@B z`tHry-%BL|Q>fs}(1oqodvssqIiQEj*{1B!JZFb(hpWPjE1>f(ydDnkM}H)UKC9%L z3~K13!T)SlugrGu>>5k20gj1Rn>s`|^>bUAn}3nN1`-~G9cLavAme(0mp&3nD8OtL zh6x()-`yZDumSdFd%apd9HixusyAd&BoXjgHH*#MSc?P1hTONaBn!=;pIt>gI!9Ci zYf|&c*Vi`|Zr(m1AKFO>zI~QpK3|q?;k3dq2A#zb81T4%-j~nWX|Ate@Qc;0T`?%v zS^C+-WH7eN{$R7J?|;cmI<y}=hbTBZIt#fbPaf2}Tg?PP<Cd}M>2q^~vR70w#^eSm z*z|1QPsN3`z7DyOe%xBa-Nu+6k9@kA)ik_w2*3VZpXppfDxW#HvYP826B&h1yCn;` zXQ0)c-qoEesy?mc8%tavG!hVw^kov*na$WJ)#p;h6l*1-<>WA~IH5vBlq1`dAT<!D zR2?6n?ebP<V|f^0vlY;l-tQ^eb}tFGWQ3`5DW#)NsOi=0Q4Tp%-j1A{OE;Y@E=uF4 zSaCShNSDFqDc9Y4Gsm@MP0ohemQp7YhH(JH5p#E{MIsR5RDKPFbXNm)B{h3B7hF^U zZ0|!AuOpD%>GT16dxqC8bHFBS`c4wWbRIHReG<@^)M=Jk%_T?Yc;@g~2>-DJzN@xf z<<Efxhq0J(R=boUTN?j3abTr<Yl@IDGYqk@e_gzDYVW>d`;L648K6)oQ*el)DL8ui zAaMU6WDp>P$W2dHItBhh$|d01UgrM-9aHnG31&jmG;lu<P<WLH?#fVpzXc{#(J!Fr zpO<3k{*wl}_aYZv*?*3=+kFZ4?*rwaSG@NBOJ=|OkyyH$%I~cD@;~%VzJKHdMP;BZ zr#!c(D0RE8&gh^4uP6*UMcA@O5OfUG0twK(n|)yX-%K=kFSaiVU=r>to!YBE74qj- zs^zy^^@?3P<+|p9Uj@zDThcC54Csj#Aq{nP<C9%G#ivE5#10e{87nR1r_Wgd48I|} z+i7;&3=JrpGKxT&AqouK&pqc%y5Gy5`P=XMcV64ywyJS+ww|T)qgFTSf_Sw7SHqRQ zy2{xApetO$@FAMh?Mgmeu4&};NevqdYSy%OX@vatcXnV&>0Te@RH8|<7lHdxLNAW6 zmQkP%<x1aMR7nrr)qhVByDD%NFWtdlY5*U@*>05*-fPD`HM`lFqhyf={a@>|G3c&W z^X>_Ew#tDY*CQin!o$+B__+=ZfN_gLrOqjlF|3-=wb^+k$|Vdx)4HUaG0Nnzhto-v zWTv%t*P1nO*_R-Uaz09$O=k^s;??i}%7&t=i}Kbe8^#;j(`0)^W@Vol^e?)7qOzA{ z8GptV&xdsqqJ@|m@+s%UJq=);1DHl;JXjhjsYRC#Kj0C(Z`9B;<%ey+R+U^qgMFz* z8}j}*q(s&QFd(1R3K~!_FaCQKu%qUHvrh&_;LfYLO}dZ-x`9_!e7JP0Jcv?>&3^V^ zz}+MQ#j6Xv5U{?kLN>1rO-AK~)rWk#z9TUYg|rbMW~Xj0HP8ChfEBU}U4n;`KLyO- zT*3HRH{pkPJRS_$o9^JWNtF2X^my1)Re9qVPVpzR=fya(jrnybnc*$^XEi;n#*t@; zG8EubXffH3HHDdj+=t##j|6<7cFF_zTri(UA9##d`+g^i2QDqGQO<v9*O>>aG8AND z{$82#r{Am<9|p$_sj>6%nYHU$@cgm5i8pH-NTBOgn%0hZsuUv#(1C|)z>RO1o|xdu z?@wt;0h|8eot2lXLDTE~vdxw>b%`%li~W*$t!_&$hPdJTs@2^7Kby=VvwYA;|Mt&m zk20yYJz{0p7BxN5C^W&ebPz!K-2cdRcRQSp?%MSF`QjkU+=X{MDSW`g8nIjKU>$3T zGr>`ZRoittn?f<AE|`_C@2b^q%c7EIIk4Ln$SM-dDlAvyc8PZ0--Ol9E`MQyHeXhN z(=4W1WY$AK;t)KB;ZWZ-4m9Ap&fItx?}1rvwWU1@np&328T4?e2sV<3F>z44omANE zQH6JOil@T9i~1i)@8wYK9>q`dc{=K6rpcc*R$uzOlwNSLuu~`BH73`b<kk`?L%kjH z^DX|(L=}$(XZRs`I{dUwzD=~gu=*y_9r%+(G5e!L$Kn%_Z~YU$JrkPTWb(A>NPpdB z-GobX(!bduDs44e3*2e4=y+9yVctfcte{D_PWU?G6M{&XIk^CB+S{zEIXyAwD-}#M zO%qQea%~r(2KtOLuE<!g_{@in3#aOU9RlqCzj_diH>+mf$Fn2qZ|dx`R8sZ>KzmE> z@ChvUU?L~Bn)#Ytd7Nm>e$$OsB#9qS0`=OM%5p<U{N%xa{c=NF)HDQ|Ln9-lIKA9- zC{shJey97iGl58+LsESm%*(zg59Q3M#nyAE1~~YgVaxkWAI+R;7oX0p9QR$HxKmKS zL^6{j-_=Lkl%%{j3azS38cx+ApE`Z89l4<%RGFMxuq~OJXDIyos8jJ-db+pV>@y;N zjVE*Z!8=4JT%0(<<^9MENO8y08pZR_ew#-z5G(EPJLZzAyc9Ps$`8zwsI9t@TG~h9 zMl!&Z7sK=XSC6&_uD`>Mqcl4Oz`B_zDf_kkf*w(5O}Q3CA*q*z0yEXf>&}83RMnhl zqLiMc4N0<|nmYcG0iM@RV{WT+Fn}H4KE{GE1h(<Igv$J%oEqbaMxma1XTG_)D0K~U zPcXAM<5Tz^Ob`-BBYVF(Xos|DoIfv2pR}0?Liy8yt{3JV<A#u0aB);jN+BY&N^7WQ zd0`Occyx2+BcmTjeW3tCi0{I!<T|g)&r6HVhT|v%<7?f-sXn_jazMCz<Ebb1%({Lg z)!bvBccLZm{gW~*iYsf0Ijo}A1sPQ5n1(ZeIOJgY&)(=|@O63eiX}a7^+A_teO7R| z!=D=5Gh^CP?YS09DRtJck^DM@sHjOhK5J4^UhuQp005J(Bb4FH?Ij#o%s^ijpGkcq zV6(gh16JW6MkvvS|76!Ecq+qdmJk#8wx}~qtd6j}WCy(ntte~Ymam3NX+6D-<v!^5 zdcqmcg#%U5NhL~zv5>d&(cy2!DLp4&;zSJIp3Hz%UD1nDD@Peu3@0569^>xu{O+Fe zHNx|#F7Ng!q#J9#8z5t{(yxnaTNQVyl+^p*&lQZ9Kd7#IO>=fBr7}(Kiq!nm`m?Ii zo@v*2PFW-tF69t~=LiW2S)!>`H{g_yNGit}M>Mk*Lq_F-8JA47ZYkJT!bL+vm^@yw zY&6(MHch$Ky0p7d+)a>GThyghK={mvPG^0Z%mTAx5!9>rXT3~Zf@wL}+uQ$nvA{7E z*4|^&+E#s$H@@JTpEVR0?RVt=d}5#}!xXn>IndFoD14gVt^Ad3$Hz8QLUp}ec17nj zNo-1Q%Vf#}^GsRfER0t)X}_wv@s(=$cH0V%X3rUs2ixmoh{Ol>TNpYelR&KxnLkdt zZNZ2>Rp;1?kmj<iGU=J~s+{uJk?df5b`DkA!VC1!pn-E*%~WUeA8}Rwn!H1n&r@x@ z1iM3T!F729o5n!^b@2d(t@~3_iE^%$7?qw1OuxC7{F*Y!Jyq4+RBnTaEu$pefRDt> z-5mPa;ZhPN(cC+LDZIJwo8=O&db=nmJF-#InvEDGE#{bV93YQ{0a{uA^ofGNL$w+$ zQr)psjeIlU7GL1SViAfwPgssVysfTq#r<el6W^@!m8`r>`=I7UNmCClk+Jnz8`aTg zVJQm+)_@1CcDs=A)30q{;KQm@2WAuLW5G%q$~j%UwL0MyQEHgR8N<n}!Nohp&P*36 z<e|Her#oYsaO{!c%e=4f&Lu+)XF=`C0<iH7jI#0YTHU89zZw^ik?*a_N;>M%B594! z4xN<Pb{G%OqgYv-1F@w-!{an(Cy-?{^5C`FJ*K@U*{W73oKHl`0w)#<I>4Uy?lef_ z0Z#5u4WkG0usa35PmR+bh8o&7Wq6sc35z=+b*TZL^l#o&w+u~kj5J5x*E%8{cJZ)w z>EKY2f6Aa5dMW9{{8EN2yCW<@+AMgI4=)&^I!#gL;cZ6@lGOU1w693cJL@-<F(`V_ zQ1+n>*0W~@UUNojPSi2|NQ}CR7%xF&X}+sx7OTHAYm@b-_Q3{7y*Z`XqCB0&XFE4* zpz!6&jI*IHCXE<VJZBRle_EvGNu+p;?zZVi?xI6<-xPjDS29j>dOtk-vU^TM0!Z$( zbne==(uuvtj(r3`@eKK`N!2mn|E+$(s78o20$=2Z?LHa$H%B?7x-J=+Aey`SS^gL; zk4v_Me{<*Hm@NBjc|MXPU6n}h$?RL#@jwJY&$knr9%RK|uwS$I^w}wM2f--94Kw~t z9QDW{>L&h`x^c}RQ)N-WYV}sLspgzp;r+v+e;o}+kg26`@=bI)O1|zG=IZ4f@y0Pg zJ~hV4csuFRGOT)ogc(UEdhwkOFS|@D`B-qL%Ha%4QF;DX<KR31wmMh{xtpTSNf80@ zXFfm<XyV{lW5k$%C-v=?S>VH&ofX5Kr{GoO!eCkM#}>dFX)I;MY&C#@1h&TqT-yZR z4Rgp1?G1Hg`zUW$BAm``t^dVo-i{UVExvP~EwJ?(WzzIjC;4=q2Fq>bA8F-&Gv2)P z^=W`p??5_(KvAA4Dgo6CMN91@*Cxo1aLzcg;WGU>Ap$<g!}In8*R&6=_z(!`Ffpr< z<+1n?TxML=l^DE@Ek2dqO7T9<<|=TU2X+ND-Rb%aXNgdn?{Z@4_DY6N1S^LakQZ<< z!R}4d5b_&4r$8p(6>w<u1iEhxp#H`8hf>7XEJd?+tc(}*rp2RJ8t=fl_1blQD#6pg z!YA#@`R>7w@U=T)$m&Ft_=`u|LJs-`JTWVGVdb6O&B7`whh{0jnL|1ATdv@r;>2#Q zls87OHN;mEyx&Iy^)HX}13P!a$;C+3&ehG_$}mmMK&^!bEQ6aZQR&klRBgZUYKX_S zIIeh0_lKVH49|@r4Xo{W6BRx;t7thuM|bM*E_AP^xGYeG+5CC;03DYhY=~PSjRJ{> zcBdgDGwgecYiHt{)d>8Y!-UN^g;SJMoMC!=w9hP`EtuqJijN=9%YxT%#~2x=oA8`U zq%(9{gQwf~H~L5(_e(5Sw{q97%zZu9BqAyrM&J<__hd_@Zw&JewWuXbd5Jo`6K0YT z?1Tc@8jI)KQf&o+CUB7)gd_8q=C@37#8K&?=4hT%8)+WH@y&0>+H&-@L`&WGe*zxT z+7U6K(^B@&79PR5fyclL>-($lfY<g-<0($egaHy9OZP*tWzMbN5iwr+E-gJBP)p|w zGOYTaMla$+fB~%Yl+_vK=VottNKl7o)Q<uVt$lkSXF958Ek)mq*d3pJh0m&@y04hV zA>0B?<CxtN?{p&s1Vh*h4Q)i>U)r2{s&msxmjUcsk2#V2jwQLR?7#<|UEN44oo1DZ z{BnC9IEIG;SeV;YftH|4r||)NU&0Ni+1bicC+6rY^cNDot9w|3!==lH?V^ARQn41` zLocdr2O#)E9Ka~TkG~(Lp}{%aAjsG#H{4zf$re)sQO(fKR_Trzxp_Zb3u~W;0Cw&m zNLdHP7@7^nKi~f}18nD;W=M~4G7j&ITZp>hJVY10U#wm`Rmly|SYc%**o>*EHiTu# z=3;QoR{l4w4o6oHZLY&fOECE)v5<ZixiO*-k`i>@J!z@-<ejV9e3Zzg?Lu?dXx`~L ze=I{A;k<=PcD7@Gk~b1&8+@Vk>6a#1b`ccGEmONmf4Y}R!gPZXD_^S4W2cE>e61yF ztscYF7*YNejDmDOg%=7MbQ`C}hC#sx;P83)fYLR~h8EH^$~M_k_{G5^*T6c%0j(7Q zkFNt}&l!ly4n2`N?#cTyi4*J6;jOtt5!WkF>MRB^Q`l<X5+S(PCcw>lsX8WHBgWJC zvZ~_27rMI5^s~xHiz@oGXBFfWHga6WT~zG7eI+(<zMQ-nbERhGYG~tXd7v?lHzE33 zZTV1ioj~qenL|=>lAZGpPtSf+5Suuf(@!7D7P%vkYAyn_Hic)ed|_&I$iERcX1#(p zYY0v&&GSS?v+@}Po%Gd=RYKlF$2pubu=Y@a&*?cxdRU$0!!-FP<ma>_fSI}_82qCv zOLu#!Z&5a6K?Z5S${)$v5QFKyBA2(j<iqoTpPQBD)4xeu!nw@dcK~Hl4thWCgQf)7 zBMmijh&vk@JJ-N0m;@GemADNG85Hb-KCyjVUODLMDowR3#W#Yyh1JupvEX_OhJ*N< zd5u<pXe$>MOny+xQvRogQ7sU$=4!-_w<M;sWHA>`-_Idhf?l`-=hAqV0^H-2G!Enj z1AXFyXysR5uIcBg#^pU36gu7TFjUcG!aFd0IN8N9+f>9F<NRPLLoLZLicf>q<`i)> z7-T%{6Y((BtISKbUA}2i`T0wS-g?Z?6XZnG>euiCjWH2xT=-{6Eoljo91b*;`A`9b za(U9agUq)L0qT$|1iZUq-dxC)Sr*kjwqIe#>6ZqaPNCr8xx42@;12aCtsdSM!DfJ9 z9Z}_rhS^W9DAR0dJw+*Tv9o%e1rj%$u#tp;gyGesmME^|#6i6h<_TAqo336{)Xw_F zewkiZ=ZfGrQ&`oWoRCSqK38>#?2h~vi+sh%XT>EsW)VkHu)Kesd|TDZba#W9u0z+; zsA@}RZHR|N<@X@x1f`}E^u~|Sc|A93Q3>(`=7@8Bn>OF*cSf8Mh+EiXoHJ44yO<N# z5$$w6p^4M<pF85);4d_Ytv9@NACq%1XtCtNpm{mV#Pr<yGDjem^JP3y$?cvFJBe2@ zv&Q0F$@PAShPa!xQ_o;(K8758)U{E5_%Ws#^3(Afot=6F1CUZ!e>QL4+PXA&6;pa4 zl51qbpA|QZJnH&&Y{OShpt!(*L6Q@DmqEWe+%V@=uwJ)D(^A(Nh>HdW69w(bTL;Cd zbVf#lmv=FYz@u}u`@${}x=~*Ovgn6yp^Q3T^{B7N!*56I^n5+1=43>fd&hF@>FNuH z7iz%-y~B*meZqgxEz|$2-UV>t<Se#Dob|MwoVwOt@g}Bx3`@Hg3hRf1h*s1-#9j6F zf<(6)n6548f8%`TOoI%L&GqA6n5P+eSZGga6Ez%)>U@{aMU(y_@E9<|3xEhGJ$p+4 zvGEdoS;4jyoHO2H-O{z6?%+Yv9pE+S0SsIa9Cw}<Q2p+-$t%Xw0G6zI>1L;33hRL4 ze-c_4N3pg*R`*uJ8i2w!XCAGYVg<qjZ(j#s7=^E>;!5RAHnL-HM3*}P^(|X=XwXw7 z&MVHZ_q}#|1(43ZE&l#-Hc<;r%Uz~%a^yjb94g(g)avqs$Y&6S3VKzgZ{&d2^XC!I zfE-s0Y(Kz%^Sl-!aD|XK5kuV5<31^ul{_a6|5~q^!k=1#iPl+NK4FeB>$K7uA}V(- zy2D1(0Ab?-=G(zj&+^R<)79W0YLnIsM<5Whjd#t>&Cma;$>fDG2I_D-@$*<g2={MJ zUhM0W%xZ|MsJ476O+P6mC8c~De*}eUe53(X5@AczI_A%%?-~~#pgH#99XUOgQ_Rg} z&B#rK9-5kckJp5p=9D$Wsa6o?mA(gs_XACjclF$zzxTXSwYl9I4PMb|&JSh{{apI& zLv%Z)AXX^RBy-mU?R^gV>+%+CVfGU(gFhFBDgfsJwH8iD{Q}r!#<SYm+8`K)T5KfC zG^RT?;?K(m0k(`wOswq7x2@<yH_Mt|VWd;Aev`#24G^%OJEFiYpw11fK_aEoBG2AK zVzHaA7(t32)TeX`+=J2Q>CT1x$5akq=A>g}?7IcwyIyT&|9mD^;QK#S)^=|Jy#3q@ zqGbNv=+FJD^cD1m!oJ^SbbfzKjczw9@b`bgzs&#pGN-^q+->JZtNhN4ZBmy^Yu=wX z?MPM!(ze~Ew%6N4Q>lXXB-IFTe0_L$IDqaOPW?uviGcH%?nu!XTO6%l{fT8@ZTMAh z0et`0x3#j;($dB@R;D2d!|Lx5+w1YU%b)cM>?-$ujczn`k1L<7N=SelTBs>_euJ>! z-Ddo6^nUl1T~`qTSDj_R5x>7~YWWb9u(a#2bar<B0&KX<fcQleuy8)t5r_y$kBw|p zo^7-2TIeqSu3+uEH<;9&@xx$=2p*evi<0*ofCe0R0#%Ua4?tmmu8n{h<pr%eJzrZW zISdSqhhKy{pqy<^?9L^{L;v|q$uR^tWcGiwgAm*BKidB>F#kWtcOtm#1yx%GkQky* zYK|QMTk&9w!mr=h|J(TNELwnp-Bh|tr=Ybl{@rkALmT?!`9F@*)$EDIkho9Mpj&|N znw*l7vMamWyNE);(!85Ws9>2U93Hkk^M8zX*hEYKWHdb{ljOHj_R*b-*v0fW`bi`U z$Phu9iPpD3>;bi{=bn`5X3y=c$@zaWtIkElY|WUNw!~fs)J@3(sD1Q(I|~ei<?d?t zbt?QhF#*jSE+!Q>itxPHR$MH~aqgK&!kvE8mPu@mpO5ExH~w{E{Ew4MUD0XKn2!d~ z48Q`!wdmizjWxHn22*-DcaLP;P_%?FO|@teIU@|5uX*NF`{vK2F|4THs%`%Wv9YOJ zY%HfgQHc^ZHl|9H@dP-rqdXVio-xK4bCzafa>As|aC?TOKk1|=Om1_1H~`%Hz%?)r z@gUCJ(6mf51zHX^nJ)Q3a65H=`|bPVyz~PD!UN6(M$a{YK;~3F6ib8rih}{bodoVC zwMG6Sb?qj&%p%Y_Mp^)3stDL1ZA+GUsA`BV0)Zm>ffWcQdR_YxA>`7wjvW8A+XgbC z*P3{`W}}AS2+#y$a)~ww#4Vx?a~fi^)rNrVnb`u^<3do#Pz6l>0-$Y$;0iEKhA;)j zvobweCA8+9O;s=<`t;s{7@Hovv!@k%aHY~u0{mLc4X4DGv+yCVt(ft;I)t)ncHWK= zu;Ys`2FEM!LFUIz-H2W&oO&%U5iT{y1QhCgBksaJ$eF(>9xEfEva*yLRmw}9Cu|!6 zRJ#WZU!;QJxB^vh;n}o}VWMl%x1c7)lx~g+h-IhKMQ@Luq@(LkRLk4;tb2d{=8U`v zbIAd^*kgM)1@JXS8Q}D04_!-1E~?yg)Bw8Mvr_I2>{9ku*WH%J4r=IpIGg?O9;!Rt zNqJh_4E60kl(KRpoe#0${6rxIQms4Grw<lGWEZD6pm+vU*;O;7ffQ@zD<}PzTuW-P zhkG6JH2Gr>kb?sf3yG`_>-?*88pN;z8XCe~uUJ*1mFJXZP0{r8_Gwavejhlu>$CgI zu}cl_CfAZ=L(FIHz#TY*J=5*CkjIx!VGPD0GWZZ}1NFG%8b`Zz4#A`;O1QqGyLtLr zX}>Hb`Aat_g8^Xwjm@`4Zmm56%IpaAY<rKz1E2z$&3Bu=W?|XC`OIxcSD1s>T`zB^ zuy91mk*4wVmebs!)!7Rmz6WhWTaOOJSRf|jh3rf$nfc8_iTYZt;_3e4Ln=t6Ykwpe zX-EX-AlIbLX;XdUPGc>JCSnWOE*dXVj^vb~RN+CN)KR6G^PYgZ$o2{}snGTrrvNZl z0&|zfv`Wo9YF*m94rH)NkQ%69P#?W?MTgut<sOpph*bwA(EfA0NrlPv=^3KZU$bZy z8?^n_(D1F<jPeKpY$Zq)zE5=A2_>zz3?-P3`}=*E9_maZDr>a6xx{3sq7#?uHKWw7 zD@pJ8{B%06Ea|dXU*lqBE*Y`v<ZPMM7$6;-F5aGF;U-?fR*}3l>2l*gmQ=yzv?X}z z!j$y6^z2f2Kc8>;oiu}Qlg|&e@%mz47+6SJAO^bnqjH(xvGrX%9oX=%^kd7>Am!pR zKc<psh^wmLWR-Vj!O<ZkUc$i=+6|!z$<&Y6vzxxO=bO_2Q8l3z%B%4vql!UhDsrnO zYD@_|T6(v8tDNNbfwp$j3C4ZWu4D-B6+c^!vOpykyouF6e+@fVKAy5Bo8t^`(yBy- ze2V+NKr1%}`o!NYF~Ao*#yLOIW4zR(z$a%~n-oHCXo%(vd#gp=cxF+2`sC5*#?aQ& zMGS|k&)zG_EiraD_Ia^F6@MeCB@)yHj$boJ6yFV6$@aVP80Ibs`G+$%XW>pI!HjP} z8W8fBuiZYeD|i%sb$E$)V3J+$jJlthCixL+{&lAP*=?H0z+BNvk+f=zr@m_`Pe#F) zc0FpD5ZtStWkQ3d;Zy{W^1SY}%mgBaquH852>x0hjkzF!8*>F^M%-pyW5&8UG(dpE z+PGqxA^T+@?^!LnEFW8aQ^dMn74ui5VI#+|w@GQlqcc$MgL6jLS82yB?^1E}_;#IE z`BJs=_3A@sL)kXdwj>4qPQS><JgHAe`|iB{EK?v&I4vew(oR#q8Q#-gV!u^u%kkuQ zuK`Y+6`J7ZRC0NYVba(a2qxuH`NIR;_M@+}<5y2x2@_6c`+J*Uj@8SqwQaz2Nl_l2 z?QS|RoN5<(dCg}vS>mPb*DKE58qH(vEVzaJaZPp}Bo8(XMjau$bc+J%_L+v`TmDjJ zo_g6H>&`1HOcA*6>)A=9vZ-p%fb<HcSF)uWVcB)oYzOGi8}@%>OrT!RR<^&2Esr)_ zQ68DzmP8s3txawMEts+5PAKe0ftvo}M%&GBDGI3YNzd{V6azI15o)Da&-R9E={L9L z@H9-2HaDi#Q6Db#)#c_oT8_vd5%NZ-O0=Q~_+=0lFyMD?H>w8$S<0oR72{v74(N|q zcPQeZTfQezlNcMMT6Tr?<?+%+%Y6DYZ)Nq(_dz^wpCTv2EOj>C@;AS(P%9vLk5yzi zi#u6uNW3IB=4M9<_iEnK?9ebu1*|JdzQmssQ&Yg?d6hqBw8Zobmz$L9*>aW*=FGR( zOkB<A3r-3Yos`>AhwG20<eFM8<OA`?Wkr?Xu2iOmd)6lIljM12r<dKf&X_10XEoz8 z%Vn3}>MUsN0_AytDA!9Q_k#W?--ir8HV4Fabg7_&gVUj*LK|YQh7PIW)eCs-YE&!B z+GbZ8pn%@WgbaK>7H(L~$37GB7tCvN3!gr=^ui7ml5pTL2M0&l@Iui$d?DZR0B51) z{Of)mP*aT`81g!aouP%;F##U-@qo8YweTp0Ll+XpTIDGVE4MlB4&#>|I!Hym^|W-` z<djl&%69*RNejdGV978u7=u*Ues^%Ui6j5^iyJ~A003&cz9bM*rvu)56BMErgQTe^ zE-MS}siy;Dungm)mO-a+JSP0kNiZ`EQKyuHfcK!zKB{18(Q|9F8#D~L6-DH%`8s^U zLYxL^ywub?#Kih=9UxcTV(wnXGVWZ2H!>XtBtP5|lWuvFv3VY@3mIxdR99e`i*l!* z&1Wv9D>`h4CgUSLi(Lnp47xh}>z&WC&D*RCJ>VF?B=B%;;E$tQ1~6h0;vmA=fg&73 z3^lR9X2{zkmGUbI6FiPW0S_pe282LKsxe~S8HRcCg@1v{>mc+nX^IRR>*OoV3xE$P zcUJ4g)>&6i<%I=31~5PckTBx5qEa3jj?+pr_eqwPBLIsxwso_5#@WZ)5`Xd$+yU{U z0Bvq8hdr6F-w4X^F?iH|iv4B)9ImnyYZO6Y6Vu?OJVtk?F7&r0#RwvDFL(Q%5r!Y? zpz{1Z(WJev7|Sr`(Ay0(5#dMt5N%!l1B7pJJTQ#gq;z|Seamv7p}Am4oi74-P7{<p zZ2sY%><1d9U)9y<p}H6+WxAlsn1VK3&LepdTglY?BLm!r$M!$8Ah4xZojm$+%MpFk z0N%S%*qF-D$}-h5#MJmzt&<1{AVH8!V^)2?vkfRu2=$I^^+5jN)0b40U4k(<CZ^HS z$|UXUZBCNY9?jL`xxx1g0zzPkMngalQ%2=+XFbX{<=!oxI!6i@6OzMBJjejvql0c| zk>0)<cetkJAJw%i$5se^@MGn`!7=>tJL%plb61&VWVA$X6chJ+3^{WL7dE>q%Fh$6 zOSjUMBr=#&K1L?Qb^;xD?YaTnslLlf>$=cdAxDMgBlRKSoM8Bh6r5OpD=uTKLxT4W z?pr7!Hpwf#O2arW`qkX$+{+#%{5c-lHi*YN?wzh$n@dsB<JvwkIv>W*Sn)v_Mql)_ z1H-@P6xt->e_{zNQiW&bU+N%)o1o6!YZE*Ft!?@C?MGgOgYn<knn;@n9e~xD9Wdor z?#y13JtM-4?TW(J1^ZA;waT(A%kqP}&^RDf?r<~=T?It(bh|`><5$vad>)G;nK;-z zrrkw$Bvf9z$Q<35GtH5;lvD2Q1PSGvv^%$9e>MS@k>M<on2cii$a@P^7c5OBl10Rw zw2r>RYlXVY6_-bMfxr<I`1s{x^NDY`A@uideAyF<27A6gDS!FrYE^>Ot*(v^K<cPP z>pkNqt7<-~1$A8pP#Z5GlJ44D3*l*?Gm@C?5hyLX^wp1GKue}-XeeXeD^-JX77y|p zEj43X*1#%m+g>G%ViDzzuQ~UNjN!O(3+%|NjzBANE2f(jcL`q0KTG`-Wx{5qk9cE~ zD5-w9iT<?kpV~T>O_@06zgKws%uDBSq+zMr>3NyxF4Z^|I#9OLWmmpayk-D^hqX{h z`F8))Wk}cs+Ac%^g$GptTJi)-*~JZ*hsb&Lr@6^I)~-$T1Sn(DSq|TrssTv&gHw7$ zb!@9{sU-4%GXMH#51x_FbcX%g{kI8kl1w(r1pTn2;zE$i+TZv7WF(sWL|AgU(A&Ub zi-%>vAwM_^zyB02Hwor2YGG11R(VGl?2~RKC3ST}IQ*0tnMVw-Z*kth;t?tsc!Mto zm}S5o`^-j9|HatDr9`c-V6S773TcA9etS~hj9m9>(mf~bj9XIr9KYMuyW>M$K8TA2 zF9}yIB5o^b^(sV{yVnQHTCJssf~;P+DddHtI>jC>B^A7ZnernxZnf%CfX=c+6rfq$ zMl~LCdp)wu`cI6YkakZ$xi-T$By9_~|AB?%)awFf3n}}_M<Hu?zhrOKUR~nq%Nt%6 zPSZ`YI`CO(S`T()6NZM_j>2>ccx?Gw#`7XNow-HE(h+eLmhc7pRIv2#GhFKu2sJxk zJm>DP-TQ1tD+7Zg58aNcXyQ0SY|!qoW%;I&+$K*+RF(hWSW>{#0ZEOn>TfuYINwe? z(Ks}u`9|o(4*TcC(bTOE0pZp3#)?~7ZaSs-m?Pw<sY2D_j7QVQ_Z|iORhaZigNovN z9l5I#6TGS?H!Hpp4WVQU4{^Shq`NppH``V!ujWaIqp&S*y|Fy;s7NzJt4*c6wy0`# zyXd&m^(;^%rd>B5?`ojpujp)EE$7|<%IrLIEyvD_um+tbRa%q)e&|bm4l?3K(;6g< znbTZoYz&~xiMba+2sjT9&(}E(lVI~c0gs4*5-s@5(HSy_i}FssE9XBq3ib*h&;$Xb zyv=AG%Zu&n@JeQTjm~TI#LKFM(k9;sq2q)2e(q-{t=!lS0l4}715FmS^P=>i97zT} zNa#$4MRTqbGCX@^N-e@XQniw$w4)-teOybWt?!a~Y{sP*JqP^Ko@U5v^^Pfx;}l)+ zFO!5kFe}0z#;18kCGld;BDN(%YuVY+Spl6l^4<%^Bn;!zz7YJi#i{A3Y0eBD^1f!f z%Fy4pJfV~w4#3GvN&L-<cDLnU*W#XaSUJ{r>4TaA&jdtMo7M+_?<>B`k~-aWd%cw= z7=`k%(7bOI94552{f?%l&|p1{_xR3IRPU7PSiJ3jq``;h`%`I(_6d{rjzCL8<jv5O z)gpgy$NoRbRFSy?UHY|=t=~TnLNtM0czSsseTyCov94|=O<npSOBXw<#1f&Wv*TT3 zVSo*Np)Y&&)q||Zfs*OOSUEO66QOth`hTwR<^Tj_c*&{RDw(}c1#`+JZ_;Rl*3o5O z-$gm($(xA_&UWN9IHf|S%|7#h6;V6q058=$em!@MA|L$IorP=pCRWPuKNsY>^up>g zu!I}Wd9~9|#YQ%GTI%SoEAt_yr=-X;Q#l~fJ7-9G9I&`A&V`qo1A_5*5l(Gj<i1k` z$fme0gG#D%#Ux6?>&YP3smT{z8EX^Ec|r3MF<iQ8RECnbYANbvy*Ccb&486q26#j( zssf-*;76hM%c7M@V>z}r$K@l=-+cA=_NaLpgsB2A_p-MWmv{)jhZ!(2Azv0FRO5Cd zlu{4;lROaAUgiYplnq~#VkSPdM17j4PO(q3{@vs>dE($<NW(gTf{?-S7*8vHy?i6i z>EF;d^uzynL7bTNxB|!7a1R-9l*7{C3nsupxSlAJ4&;TWBdxYO+#&a)_2<nWNfnx! znmt-yVI<jwZyo&4r9ln^{&M$5qYkYHC6FU(qN3lgZ`zb$wqicm#42Y<&ve(Xu(3qT zcl>P|Bob;NLt7JcWX0_K>$9#k&|QEwiU%m@tI*K=hyc4ie#vVy*&!9lN#?NV-2l`s zPA`y51z?&D<(GO`p~vVy@xbmi7EdA;9-zb`f%f`0n7yk4-OQc)mEi|I_REbW^a4Z) zy|HZWcJbkF0q(9k^?&0<yVtG@n{(2K@^gXW6GF!U>V+8uT9AXl$E5DS|6{ybSRWJq zja1h(Q<uiud3ZHYV60<{7y<MByLlOO{tGHR;0#{p68JX_6C1*9GtLEp9A1kz{w1)2 zF5vZlXsrMJUFb9al4_enV^1QHfY%7M7Eb12O!u)9pFoa4_+A>Ci229ULXROj2WoZw z3Pd3E>GZn?%|U>nW&C~v4keOm{hZ5cmdg@9zp_+&CFb`m-tBDwZ^lglgn0KV`g>WY zM8{d(g>`_2|B59hnEt6QeeUWmp1k+@y}t*p0|7mN-46!*FX^zP-IEcMy-&rmLqr95 zgntIoZl>qUKR*D?GT<_VEsZymr#W_kzuoWcRkA7!GkR<{Ty=!zR`b4R5rjP>v%dyY z-Q^Hq@%M<8{NkN$nz^U^a?8zKaP?xBG`7$AkKOAicz*u}qW`x*a`(-V-HejG``r8c z|CPS*!k8==ZG*?y;-IYja{(^;n_%F!z5`USVGqQfC|%YEn3rz`h{m?lcg+M~4eW+~ zTtff&&P`iyrJ@UqFeA^h%&q4I6Ygq<rD#kss=*$G0H&`ppTa^e%_($U!BatELBq;) zcO%d#G=K`&;8%*xK|TSPNwG0BK$lqnXA-g>A?Yi91?W{<2Zx+uQ*d9grD_vajie*1 zwweo8!&PZwk!zp~QQhUi&r6*yhQ%afwk}z-s(=j9txBkh0;r)K)+=ll8pInCzHJF= zP0Z@ah2%Hs_)M?N-hWr|^Td_LjuZ#esR-zY)wZ|z>F4Kw^LOGkhzDXkR>O8p(<Yc| zp@%r#TfqE{z^rD)^m?A`FLG%IhC3!^bL7e@@$4_6uOQ;R-B3CGF6)rl6lpGMJ!4+C z?&`3@C5&<R3$vp<5q^9tvG%q1E!rAquCWBY|L3n?+Zv)p`>*O0#TwGr_)dO%UXgnt z+>-LMu$M}@tf9z!*s?FbQjY)c7UvCRrOc=<%8$wwpB%WcVsEQS<5<myv7!M&^ZklK z`0AIj=Ek8{RcZR(PNU^9$lfAz4+F9&Y491*BXtluRe{kpU}r1+i)VVwYds-B>KajR zlqdnj_-Jv?+dC`3e=kM>QUDhpU!I+pcZN=3XD=-RP#n8~l2JqF_u@6srQ<6D<62i6 zb)%H`MTiQBg(uu49t<RDRJS&NtX#P+XdWwt`4QM#y7PXwR#bER`uW7V<BYf*H{@Lw z@*-Sz!|}}bj@3>d<hzUZc_^9{S_lqLKQ+WpAG){|Ik9f#LF>jHL&h!YTq%x|U6%cz z+VXIpl6(gNb$DQf->+2OTB+{b2rSTdH)spopL0%Aa2S*t5CLKc6Og^DN!&BgTz?I< ze8lTTQ41?0C8AUH@=(=Q=$zt<*F->7)}p(?Jqw?>-Xu;LXe22=D4FwiqA`&LYdTUs z)#q+#v+KQ|{*mW!`T*P_455=!{s)K8mvAc}h49=h_^vtDbyqb-Di(Q3Z^lMD-NG7^ zd4DJy6o&h-u9xkAJi6jim-Ar5yNO$SVeZ>=CdQ3mIg@yv1W*+7@39fJ`)L<r1EC^k zkaEdza|&>m9=C@W;iXFi09Mv@1HJ~mOKWrhV<>7jat}RP7c>YyMR!c>Y28STpvu?A z5v{D9Z|Bd1M~EU_L)Io8SPM1aGXIo_$G$&kHSu6^c2U?(Ylr;`cKB=*r=h|e?K|5n zAMDo23zU;Z>V0N-O)gF6rLl8}&c&&>96E%gf|JDX>4!xB=L&qKqH`yzy2GhHqGV^> zC7~;vCyH*C4pZg}SGo`?dIuvyJ32c42&mp8yp>?jl4oaBF^eaFd`uOlP6+zWm)8dv z&k3b%j|>t?x1FLAE>b%{Y0nt|DGZQNs|hI=1yu;aSppB-`g7j1<Xn3P>zB+9Qz<09 zz5eC-Im$T`etciP2h9qka^@V=pDaly4OncQmF<fB>A}C8SQm2fjwai(9GsPf9&gsH zsD<b#da^OsTVj$2N`t&eWB)Vo#15t#5Twd_U}pr(oWzp2=aP7B)X}&Xd~~t%DVO3f zKuJ~FGsV0^joC2=SwhDDC=uUp%VSF_SRfiCV>j`eDGcKG!F_wpsja8GE_0o!%V&d0 z7I?ijF#PKLeDY8Fm{rboprCqab2q4gd0Cx7o&FyFZQmWuvOrn4RgzYwl%s<~5b8Ca zN4>y*)|xlSM@R!J52v#ygBp51({`QTI=0n~S?^xIq^-{PPu+OQ|E?R)TL;yRkNWSr z@x$>jAE{{1V!5;|S+{V@8jbPe4l1qzC?U5vVXkZxZQ2Jd%f88B6%*+*X65!6u@x*q z?kuE6$-3Pc2s+y@)f=*5iS720JE0y_uP1G$ZOq>-)syRy=aGRppi2yoX!oIndt#q# zIV5=v*99FJTi3WUYkO_B+vCcF1N~{VEmAd2-C@SPJ2OIjP5%S3@m068T(vFb{*~j^ zi>0iKTbAqSA0^Fq0@9}zOOkWgRy+ddZS_GG3Rfqlf6WQ!n3bOP1$i+NIrIGCt}1qb z`)?bds=Up=$F%fU?(TW#JcICh4nIGdukl>wt{Ey<)WAKMQ#ZO#b6ref(I-8$*x6_; z@mwSEnr^QD&`dQT=C;mA+i=Ym2%C{m6Sw);C>Z43D#7MOGegFz^ERzLD3&KWpaA3% zBG=0i?ZAimO>l<ngATdZ;5D_yI4@mhIENd^lh&;imvvl<H-h7qwBFyNWu%{ddPh?$ zoFYRML(TN(@P0!t7_L46*+N|Oyt5LFQ=~%C7-SnY@?K<ju3EBEw>>DZJ`|^nY{O=5 zbN?LnPqYmcb#qO9eo_22+qnA+r-H89O6%#iat+svE^Fr_g+dx{C-NxnY4z+Dj4OrZ zMun@M_uRMp9zENxri5+lw7X15!b{Md?eSfp0zG$j;pMyZImyk%@|5a;m}^iWqGTab z?nRI7{_xI6+%>svgp#mn30<Wf-k&-@yWjc`WdtM-Pb{(Y$4hUyT_2XN2~DS*;2z4s z(|sP}o62XSSW>X*KVrwKqR6Q_!ZX{T>~80ye&tO$+fB`dD1uNcJCB-v>b!oerfv1^ znK)oofOhPRwoU`n>-;IJwojQKIepa?p8otuU$)JB)b9G{E39dP)};j(SC759!4K;% zFcHwn{+3(Qv6ANUrB#c)or*p14A$M?_T>JtF<W@N$eru-DO<0>m_dczNn~knMU&y` z?w(cR1BY9eQk9wCQ5zR?QfBY;@;t)Bl8rv&DJWHri&I4&csy>YRgSGI?LlNa$0}8P zb>;{U)0|5dH{W+nWsv7Wiv3h&3XYs`Oc!ms8{lD?ao;42ky%A7LOd-GJ+fbM2yd+Z zyu!voT0K)e4iu5yDg0YEVNCbRkm!V*wByk;T4CI4D!JWpD@j7_r_wxZUw~a2S2LVa z7rEaQ_R5^Y+)}{6vOeGRtWql4&qZ@3`(lcyu(XhBfp`A)xIuR^%nd(rIovu8yX9k& z+!j%~ZLiO7pJc03@Wb4#I9JhVDm+I%M*a?{cKOG1k*N8q@8hM{;9E^e!_W9EJgypK zpX*5~`P3(+4~jf&(q7XF^&(0%va6P(zM{u@%YOEmJ?u`;nZds-oOUD^il$K)N(1^Z zo=Y=tPs?|MG37k0IVwFXtYP%9yM!9FQWnZZN7$MHW!}%Ckv3foli5sFUjHMh^4MGP zBD^Y8yf0c?qWZLY@0RZ~{=YBd&yBt|#g7Mky?%~WOa0h`=b!SqG=JRSA2ON#;*DXF z|0?kn9Nf;4acDfvs{w5wac>_dy<zZGJkx&JERB6K-L4P!1<xHk$+B)+?LyAz5S4Nn zumY(bv(k=aH4SD~*EWZZ=WM@<x@&iS3F1|5El|{7go@^Ok5qB%2+iR<nsN`Tr1}jf zs2*HO8Z_2i5uy)fzOYOPN!j1O`CU#wIVF(!>w`YFnXgBcuQ3uF7S@;qz8LfPS$C}W z)V=nZJzBDMrztbCsu`Ofk5A%I$x+uzp22H#2>af6QdaP~%kbJa-KX7|{y7pqCGCub zSl8j_nhd+8oIDOzpz$X84x_K<ZkfoEqmgbQOB!nGX*m@pxcR@GtXMTzmNwV)?qRKN zU%$7L)0#3ElMBb*!S-(+(_5$2P_$w+qCV}ffgLcjI799=&CM#0UrkWNhL<XnvDLRs zQalQBJJS2Vj9o_1kRX}Ra&K|%{+-xIe~;N03EaW$G_Q9Q{&wp!3D@OWob={<-*WaL zMR+Y#xc&au&L8(r{3_jkiuIv-OO1MR4gHhsS@db1^4qgnNg@WAK>?ntiE8hfl<cF! zZm1QO*i6*AT+P`E%RnewFT99^{fV8_BcAvOilaV8oL6D#*ECjsj`NhxZEn(uqtVEQ zvcW&A(-H^boBa(>^I33A6qH8D_RGnAfIYcgo9)5O`ooUzK1cWQJkqNR<M!sQQ|%!4 zCW-nc{1C?#ou8Q%T1MzT3|TmRvb_6VhgdBg(muZ-r!QzBf5Y=G{pJ4k$-)Pl+*`zk z9XVgAT)6{QG23e&hF@z}r1|xaOIi@Q5;T^Q=MTYG+9l1cM5|#(tEjS!fo!{O_pko) zD%vuozBNC80jkj6-PWpg+YR~JhpaB}N4z3DjO*49Z+`ydZVecJie2Cnf{ggny1!{; zrdVp+r`{~n&;k3AiMya;d%4e#(y>D!?riPKU|;s()gJwtzkWUL3acz~{K^B1D|lqB zxY5bm<AIa^kG=PfYHIt!MX_L^h(|yaDSA9AMT#PzAVuubOF-$0)DT)I(h)>N1yls- zRayc;N<t@61f?TQ5+EQYKnPVr4ZO7h&pE%}yYG$r$9v<BamQVMC~RP7ueDd1^ZVxf z<~f75>|^tYMK5wZ)@Wb`d96fgefZsoWfxm?iKNsQ^vmdAqFwbeA$#G%sjk%@3T|$4 z8SIHhi~-1YZZ{Gyileu~C2jJFUPZ}t%Z6RZvh9SU8$K*9sd=iiRFwJ{2IPs#dX;o^ zb)zPUw-?6?_Pl5Rcf2LTe0rOPj>DZ@hlu*(x5aN0p9k=*6psh$>hue-Qw^xsfX217 zN#aUfZIuFUw;vD#E_w5kUw37d(sHux{hCM7=+FPsVt3H653;>fk19_WkMP%J@~La@ z)_STLD9L$`;Cu4H!*<wS9Z$mKZp;EnzVE{vHlfg9`cTn6Njsw3<3maDYJuGt-0h&8 zTY6mbt=!U@Un(N9Y{Eo;MX0}+aTYR7KWu<Q;vG&sU~FeAwcd=m9@xpjGGIwY<{Qas zBD3ZmkY?jp`zMe|E~8@#iO8-id~&orL=kG{h2b?>Y_f}U`7HS>qh!s@Vt{j?5C0WR zvu&qKiIDe7VXW#jUd~Z1C%AQ(VtvhwF7a^mWrnrnE1$nTW4}vKEG}T)q<6rX1MvGr zgxBh@4@UxTa{SfMjcaFnZ}{Rs)JRPreZIGwF}4!Vt!{n+kIjPj_VV^0<e}YIgGpQT zZWdMe8awKR2_FTjB~R`R$@V<Iub|K0nq#Q%U@oPpct?R%43oc02)po$_|Yb@z_NAx zdyK?AmxHi%r-%0#KjB6-S47n8RQfLqb*7QH6TNmnpz8U4!b{c%CX{tAR)MIH=hKI# z6yo&HeZ!`dyk;7dlkvsK<pPh3zz{EM&nw$esx@n?{wQZ%`)9qky5KsTsC5pbN4)hs zg1x?dE<DF?>1?HeURs6ENlp!xun7K-l$NEpBnwx_e?G^Kjcw*AM<V=|sGBE?EB}ay zumI;ZPRQC#e`WWV&hE%{OZT{~7*`{Z;l7>oUK`BL&Y$$h4%i{!_MZ-VG=Op-w8jf) zj_X&Ax5&x{c%WK3!<p^`7EY974^)LU$99ts<-;QF2S3KO^uzA!Fnoofw3F^!`x92p zygzzA?eiOA5pAI^Ruz=`(zYIwhp8FlKe^Uo<Dd2!s}4JiAj4ywDix+B#o^s7vbR^K z$_`VquM$P|LcyM>%S-_U{e)&S-=zEWtqW?9`RoTeO>11`n`emj)1TmXljWM}ft{sV zZB(kXi83=<9%xA$o#U1cT&YejkPAj!^tQK_ACt@fF|D^to>6ub_|{1<*X!5EC@thQ zfhHJFQxNk+$>##*<CA@ccl3I4qzJFr6ZfNt$vRtcV^w9X@@q|oWv(Jm*3l+(kg>4U zMqBVRi@za+UzSi5X0ZdPtqp$o=)yxUONtUq!%5^E^HBA56zyZ8ZP&2j$&VM)nUFpX z8mB%eRiBA4elhV#8clcU8P}1_F%)s53%PZrHs;P_(WrEXE}(CV;1p$yPFIVM({m}c z<ya=HH+ZaFmMeDohNG-6ofk1zzG-QN0=q7S+VF}=T*|KR5xEwd0JFNlFM1V?oUzVq zWV68-5v_W>%^p=W#jGffL|OG+T3YQxsph~znL5>61O{OqYuoNQTOrM~;YC@y0iXIA zh%@!R!t{rbGna9VPn#OqXuP;ss({jy({R68ZJ6G;w$UQreWirbzucf(Aja9JE&QPQ zTo+a25QsB`e+mvRTRG;lgA2Zw*fjn?$c2Iy#}j-F-?0#Y7Um;iMNXUU?1`#jP_0d( zQvD%!&#gRJ>aA3ruu8cDZSi*kb>}6`?~(C+OV=eUu2o4~i9d$K%b7>{s=9^dUKqWX zoB@Q!1*6cXZ^x>}6(;B$RDCFm{X-<pHNST`CmASFKk|rW4zgzDlUvPWwH-a75&MoQ zVnTwdzd?K7V&0LyZ|y$WdG2S?1Cl+FEB8M4-af-NHN(ZrFfWQ)rf!vjL44lB2qQg| zUxtwD{!R0+m7-dbW|bz_<H+H9<~8~6met-07HFCA{j2L{<Hby>mGtgIYY*Cr50};` z0?hcI%vDf6;;0yk&HtVaDU-qYtc~R|n|jV2&oHHvRWzcTCCj@kAsAUIT$P7I$+7Ah zj5;q|JDT$5HS;GR^$suS{V%C^qL!I1!XPv9Hu|gW@a0pSe7_9RS5IB(nJz3dJaZ{9 z?A|pOm;Bv?K<0L{pF+DEVz40(u19^z|ABVUyK_>vN`mVmv6AkheISPlvLR)K`-~7X zU+rT%$?d+rWH{LPv1<5M@qw8AfsTvxlnd>pT7K$b_2dJeOJ`C6IK=rY-suwOt*#|4 zOi~i_7^t|A;)-Xb=yqrwy%W404F0u#du1qIffWBiD#G!rl>k<NLV6E#1uCx#X{(Fs zKNKddB>xd;%yA<cl=%bb;@ys7M+1U9<PrD^@6D%uVKu_R76tTRpuR)egMc%E&%+EI z+7i~Zh5xJ(t*2hGr_Qf8M5ynte@)OOHhVsH{BX~Qss4pfH|G(Iqca^7@lnawN8p0H zJk^!YFte^%J#YL;X{fEjN}G!nPpZ%Xlt&c$^;pn{OGiaNkD}g3yfWO?Bs-&m3Acp} z^>KH4CK&#*ty?t6M!jaTDVY=d>o9sIgserzEOhkUjxR&(;wg3Y4tu<s%_qB>N6%^f zwcNQ}kp~JS5~IUz>W*~Ew|vx>TC2Lo<)M=#;>F{gK$a`7#r+8Vnb5POl-#vmO}edl zZ7g#p^>h{`;axq?a!)U@OXg|uf1=N8PyR>XGdODq^-yw{8-#?_-W@uD)l$*zw~+?* z3$Y3q?r5RwCE_m=h&)XGS={e|`Zp=l{AWhJdhetK6L~z3NNc}MCLDe75CFK0VVsw> zlj`k9pOBYjZJ*q#Iv*IfE$se5p{%=-Z11}m#p)V@mpDExv)}@6^Pg{D!K;<)#=7=T zmy4z2x{w6Ri*4b3YMd483!1ZEV5uEa1r^<IHcHG3eENkvMNqHP?ggwJ;yF~e=KQl5 zxB)(2Fpts>Yv_o#_pJJES~+PRxHDXALER@SufHe8fH8Hw&8Rp(2K8)$?$Nqgh!pQ_ zAp+Pj4|_&*A6Dilqd0a$sv>^?!+V=NQ0xfG?;VabILBmOg7S(nfC(dWRQtJ|6Av=x z7alD%$*KC|xKOdn2b$Y_fv;qL@``4}6>p!y=bP58#>maiZ#Wwx&!6r~y?Vu3RQ;Ph zm%sj1V>tweeYswwHl`qRu2?rmFeJJ%+?gduSnqgiNSM5AfrTrsA|dhEns@LFGlDFi zO1MRhUF_{%JExL)p+3xwwL3%&7S&l|HSlX7^V27NMzmb+X8++0d9AN)_*<K6cMzqE zF_jVlGhb`nbVM=z*)8f+3q6duV(<Kn8=&x7YR}*X<JBhZH^%JW$?_f{+HLbAe)14C z59!{i#9eNOme-dQ(}rhDHnx6sFun!hn^|(>KhH2Xy}QW_@b(BJoL-XJE>69bhFp#` z1gsZ+7u8@81uw}$=$yLu@Q6ZF3N<ucNJ;Qxig*aWqE?jhwX<&1Ot@l)SN)rvr8==+ z311g<@7S+}_Jr3TiZ3d&NL0Ek`ib1~twO=eu2<mmWtYhj(<V`m;T)uK$<K*=8{QRr z{^^3DvP??*kE^TWJf5l#QAg4lBI-<t!2j%<ek6@LNJ~HSifb=tSK&Tg%(B#lrFggs zMG={;h<>bpPM@^84q*lRs;Xh#eflqCow<T$$pc2I!!7-D3h3KmUVWqbsg=SUDxIz< zFLL*t<zulX)_2y7zmqLMQADE_;gZcXnncCs=lyT1WFHhwUs)K@7Dk*D)$Rg|wdF3E z@CLPdyz-Vee_*a=F8st&uPed=5rKa(lPRz|@Tsayr>xhIYW-9Ts=>0&+Cqa2HN{-_ zP5M&DhR=9%P?KVsVm-`Y0#mcELnlvA$bsu-vNR*C59b4KH>}s`5{6x6-hY=v^h!Ka z#|r0IwN9QoEUl1VlTtT%^}68nF&TTK+!i^aq=^cnlXJG$Kw?N4O{uj+X9cbpQFke$ zq^^#XZdmu(L;_WO)FosirU|RIP9K%EOG}YOPUo(9RSK6%$WI?z@ww%~iUM-aUKsz; zE)4^HYO8%NI>|`q4(=T3Hc$xocb3bqi@<Rb|C9yG7m`LjGREoFGo9U>uf!LGAuG5P zXSo0Ef&x;QJ8=5(dc506g{mNK*S>wwAOEW<ilMSqa4B*gQ^VOa#Da`(8@iMqf=gY; z8PWp&{h@$Ld9~pP%HDW_B9%DmRJ4Rh6<jYD7dw1TE#_RJ{zDw~2LeLbRAad_pX`4h zbxoRm$5wzoEy)0aYg-cJ;7|w}ly(KQcAV@MK>A$<o!)kxZM5&!nu6PO-0w0MQ}zss zD_WU$p!(wjl7x#*y!8{uh~o;CbqoR`>tYbJ4mImvV$QiH_=Fz3IIwa3F8mY0w<+QR z1q1==JnAx(s{%zL9@^lO_bSfH$&my+>0;l@w6VMuWYZjbhHYTkD=y;H0>ppKxpl|P z4bO;cXS5j~eR2K5mKtu^4h1$f3{i#Kc@og)18pr3H(06*(|kKQY2(=pE`3$oJkmqn z?c^9Z1L~i(<FD>xc#j9=$)j(RDVCwTG<ddSt41JNbaPluA<=6-c!x4w4cO60)5C(; z3x-~>eP9}qObvqKJ=&U}jGeLpxH_^`6?HpqZ)Jy26|mgOT~`zZIa-BO^BSNK+Q?Db zk@wR!92%BtKwbPpf%`qZhcF*zA(=Dxf9a|Q!LbbeTtGz?ka=%^6WmTv5?q%4D<mZc zlKN#B5&NyGx(tM`{>7#Kzno?}WoHFmGuH(MAg5;xtF9<PGj9$Grv#HjMFpA>zHG;j z7$j^3<pWD{I|W}c*@<x+s)Jrg=Jyyswe?r#fBmLZ$9YS4WuSy@T+*4&_c?=Z35ZTZ ziHS)XB|@Y?kWCJe<9WJJhZSd*XKUD+K{DKwS>Ozx*X*^OQAqmrn5lC#0Aw$N7^1E9 zqTy;dt_CIJfG337`?HK>jwE4@WGH~-`>lVQelIiIb|hxM-~Ny5=Se%)=l=te^SJhV z@aC`m{(Z$55J2(+3J|Il(_RkUdXTvtrX4D`1$l2!Qkcc}k9RG8e0umlaNy>C<e4=C z{|++3eva;T1!-h7UrV(p@uypNd%LAj#yKOuGQ!@`-E9&66rxA}Yof6zfEz28a}=(1 z<@%UI1GF{RH+3Kx!JW|14X{L^&j78B|DT|W`&&su+t;`M-RBSJJt)?9`&&g{2{4I5 zc&=Q0;<Yo4zz?>w2LB=B*NSnt?wz~Pt3a1jfr^{=)x*=C5dZYM>R8;)bN|E1&ku(w za8n`aUubB){Iichp?@4~+g0bmm&<@kI`_9Q0${#t{{JB3cRH~BABX7+=yBf^F>{GU zv!7Gztq*HS>=<2yoElHhpyP6-4Unj~pVsHny$l?akWR?bw!S3v_ZrjNsdN2d3L93H ztCPZOPk=!A?>R7T&rxas*fMV(&T?Y0Bbt>gw>ln|fAI2zm&<@&u4~kDm)2-d@JZEo zt}da%V)N;B3tH203t(@jD?pXMWk0l@!60>idU$&t8FpB&y$kKE(W_@coRk2>T>`}x zum^h^8W_NRw|nE~G)kJ`1{(hr^;}ydJJ_zaNcQa5w;<~g#7;B#A;(nE?q=%Rbv<bG zZRq2h-mjgF{o}NdsyxSE1pMYF)*3;Fzaju&LCc=7`1X^LE~Kcdvi}edj<fwSFiD`4 zvS_UoMbqIr5t`$vr*$2c`u^0yjiX{1_3Rub?S@c3xD?PfCNCO$s{yJYKTOis?vV5) zJL-{gVXh6s#xYgO0s+&$?>3QY2I7dq_{`fp*Ma#(N7Ap3yxgiT4bpAX($WHdA88FS zzIf#@Wzi>OJNX6>WMmKZmefUa1tjfI`M?@D93G&<1x*V;ASd#dRga&R=j7$p2l>VF z2hOtU)T_r`1OWjT0pa>%@BxRyN1X@$RfVqh)XYZ)m8jb!yMQP{rHo^U<FjcTf`kxF zM>TMs?n`6Pu1sX-CFR&}5;z`-=*Uw(N*4xIo|`|YFJw9q<bh?aDjP_fTOuRw9)D)| zRLSx02&SFIcGBekD}MaJLjt>*JltY4YcO@Nm4Cpq=6OTk>bM|mzjMyBSG>~=FlsZK zlwcf<W*~bc@fFUlIayx4r2DQ(+m%tK0p82APp^@_TdL@o|7wbktc~Ll%5vF`b^+VT z1rVSBi1JKG_Xk=CPEg6v$5Cf{Eu7H3zJqUAo10@KD;n2+l`{8d2_>M01Fd7hq5Dzk z&K_y!bEuP%VZ8;b<jaCQOXKUyyPZhqx~=RrPv#$9!3~D|l-EvEO0JHT-rtLg)H9j- zsxQ}!{n2Ikf#AZj*JdThF;Tp-yGcE}w@yMN_{vJuAn$5CI8)&urfy4+XD+8`07R-l zZ94!yKve}8BPXtcWCdv7QcUK+%%S^u<yIXD6ii@tX2e4vT9({w`@PGU<t6Lhm3_yN znxl18qFXGe_ANPnPsG2Arb}5G&l`I^gG?5KRq|X&`JS|tRhlWFA?K-^Pe>U%)8W=i z<1K5B!Je=ky^|CNWg3K^Rn70oPV}6q=-GN05ZAVq-HQgwJD~vPq|I%|*f6y5GLc|l zK3_kscm32kuGuz<@=-Byx#Yo)jXnH7Ug|SVXTFQ}m$x~EVz}$_miR&_D&Mlzn8>7C zkHKK_sT!~|LOpqOMB2p*7O2?_CO<m^qSPfvBMf&GK|yHhr$FY25bBIDt6g503q4vW z!@9}2?{s}kb09Nfth?3`akHdWsuzfu8HVyJ9G@amZ?~4g*%EyXbxvv4FXZ-QxZo_+ z!&%)Y4O1|8XYxcW#GLm9;t;-aj+meTd<`{RqP-8HZi^-zu`JV<#se?+YuWcygL7+I zi8-?}?UsL{oTB06DWRrO{ryDT3~t5+t_asdB4y?b7INQY<EJ?s3`_E>czh>6%BiK( zOK0|0U_h9G9W_f56&gRo?Yb~lDeRQfJImclzC0~EapULxjU}bZL{?w5zNOyrc9_us z7gjg5GHhNB1D8NEr4e!g#?@)p4GR7to*_fzG=%!Rb3qVg6ePpCVFylH3$WYnoVMd^ z+u8K-hl?R&f=38eGWqjN$23eu>rG8yM7m0RU+U*~v_;^TloqCSZY-vHZ%$+NnwICj zWLXusB2s(Z1~mvs5JN(2d!>H~fJvdRA0~{pRvvqOeJ~}DDD*WSIMgRxH4l8t6-aZ4 zM<2CfowPMay$SbJ-GvoE$<V4}>Iz@lLlw@eP%7k{lW+eiaPIDkAp9tU>-muf@l)Rj zx`i4{x&+5#omdyRe9y;)D{n97B9b*eU?!d|6;)r4?*7R9kPc91UgcqNIPKT{J-6)+ zNw%!)n{mnse?{s}3(WXsoI{%Bb~Z~t+HQ|u(f@Fz^Vp<IV3jj<^Ud;lNnYD}7+!*2 zkiO7{1nwFt<7N&s*Q%D4i)tv7jb@^!Z)F(2f;kY)7S}W@MBH9oFOx*)X}v+D>3K`^ zI2J~3=>07CnajmnW5Tal-%b`Ty$Q<s`q#O=xnP7pr&SGy?Yh%!(CH{y*;~nM^yOM} zh%AT-vG_qZw~<EYl}0an|6I-*l+`m!xuTO)dY@FL$n$q#;AbxQmRa>80DXQe=(?ec z^u*o!-`KtgB;`N(=5OlDw7=x!_wZM`QJ@~rAD#zpoZ<?c?#ugdUf<N<@#HE{H?|TE zomj8lbtw=Iu;jlEEEEfAa;GNwcW%^3{rZ#AopX^&<-L}c2hKIyHKPoflq0%lnd;Up zRtTN0skWTwVT?NBT3s`^?Na41SjUPLKY-89wM-mZTL&?1`y%!)mG{0s`ba1p<mn*V z?wS>>MC_C8;U<Hs-MF5Qc`Ex}o{a@aJMoIst~7_PPnn+|7MI2g{K(YkOI`3aI+CM_ z%n1jtCXrV*h!qW;twc)OYDbjy>+Z_pDo_Vddt^?>Ty2*GV@+Y`B^;L+&fHKU>`Zq_ z-Bpwa;caJ3bbM5X;WfieWG-AMP!8tI-dxmOy-|BXXQJ7}w^Wp?w4*rNaIUbz*x-z~ zOEV#{l|L1DasG-iEpBE%7fhm2KI9I^!lPLEqhk7pr?g3T?cy;Hh*lf)8gC<vDoIe^ zY*nqdWFcbk$+Sckyhq;^rqu6JhUky!_F5+0w!`DyX<uF^!u1^6*!H64<7v@?;#qLg zdc&%z3Wr~p4~_qhPgsfMW%!sgVgHVW&Tx+#irr>-|Nj5*+5hYK?47fTVfFh~2=#l0 z|HEhhKl9l;gQaM=B{!1P^S?Qtz17_fU)r>2al`z;<|ZRxmyTGfTi6IVDAs+v`a@HT zaiy8TQ?D=H{{-{Kep_EA+o}K|^@IaQuC8z$_W1KaDI1er9)0GBrC$!G3_aNOm4rcZ zzi9?j%;gb=!_H?HPK3WGV=Y_nA?s60kvXzU9!(YRvatiXEiu`DQG6=9^GlZdNFWCG z7}Hj&%I7!QRsuqoKQaU%M50bdrJC1V3-dChDG8(OB7(bfXi4POrKqqL-$y?O?LmQa z70xQdJ73t)UcWSb4_kE`T_-xt&5yQD_1KadH+*agid2S%l9CVgj&lR4+iD#qeZl49 zw+)mTZ*g|(2uyr!e;jyI?+*>C^*K(5IJ_^#p^n6w-tTmw?^vfhu}lofKs6L9?Cxjh z#l>9Cnx$kfoJAEEiNdJuftB+I=i-$an>Zdd;$}XGo<W~ny(XPw(gWrD1h5e)S2{xD zGvQea1z48~K&g8tIgVm{<=0T&SLeoLas4Ry(@u=k&gb3jwCb7DT%9cVZnp27tL`I! z$<wSKMg1D>u#+m;NT`)}`D-9dES)iAk@U<3&Yuh=N^LdM8-GH12Y#n|O3hQFBT=Gf zm|h9od7pzw<rgU7_(S|#5cAq-K{m6LcRTNI8vViRN4tXp+B>K%53YCjtgLamEf#BO zuIUA4a8x1@FBaRD@8j)Usv>%C>^dc<pGYv>qhmv!D7xDvGgIL<;MX0ec{U{JrHQCY z7DG4EUB~BDxNOGsRY0TX$|$Vt&G#t97n-<)F!#HKP<l4#XHJyLdW<}=iTkVmCB|Hb z!G7Z1O;@~>9JYVoc%KiP7+n|PZ@Hh{Y}ACEZRtwdY%W4M{<5*iiQ~FzwzrAuxZOTy z=6Xv{QI~FO@A@)ZG9Tdl#fuM=CzD1DSnx+@_S>cp#>4k`Eq}ES7)GOWB!z)!HSF#y z>rv*D^8Nk&>AAVBojpCjQ$zY99#sM@n|?hLZ*%$e+3RawAf3<xXx~6);)|3vqh!w4 z>1tHA$t;-;Q=yZb-_sYyj_g%%rS`?he$_Rod>ZaVaMOFJx$7j*27me|W3bLaF`mZ_ zA9i22>(_5p7AuPG$Y@CqT+_LN=xQ*(f<73!7vqicw-URGmmpJy`Svg$Z7S%wGvm$m z$moqFag<ORJEe#H)I#>S#mL9`y|czhfpM~L!ARBs@uMY!vWEV!)SLIQEdfJM>nsM` zxt^Fp;!$01t1tSpYR&^3W6VJ<ht5WR`&E)g$y>fHO!eR#)V`mak<n8=SN9F@=WzJg za0_|iKlpPS65!A40e>#cJ_GpkhabO}_kaLyTY#7ub1oRi3{$423R`1iW2rmmw}2j% zN~N80Y=MN=f&Qo477B+{y#*>3Y`Hv+(egQXmt-M<lF0BthoMt2&+n%V7B6!h$O_4- zj^pRi`I`2|;cgDsqf*1iSo|YZ6<9|@ms3x!lXTYS!>xA)Q7Az}ETW#zAW->QdT)W{ z7%X1&1Y@PETtSz-I-8wQ9#v-Zw6b#V+Z2UMU2x$~tS+NB8IKjs{6ORe!{yLzkK_%X z=&s5C(D=aL*;npf#=AshA2zR&TR7bs+!Bka7$~yQrs$vG_v(em!VP-Q8vLrHO)4|u z@r&*$7hG4qcbR%^Q|Wv)9qhSog`~g)t{w|lN+j!Xbnk?h?+nJbuobI5{lx8RK|PNe z<5K56-JbACSAqRqQZu7UZqrwQeWRERVW3+<%{E7vFq>3+Um$$L0sl4LZUGYN$0#P7 zr7LyPl@9@?`JTD|Zrp-Vs;WxgNUGP|7p|@8M^wnYw-3>hXkWyc#Dq1ywyE$g@yWvD zx*kE5VS>QX4!cVYAA?DUb~|TiCr5ZS$GF^{?PKd=VPtF_G5i^~Mis1CH;b7J^k@iC zQs10kZyD6DaQyi(IJ_=gC|S@pqv7%Ctaw`~ANcqJKQ=abtok8$z&W*YvpohY`#LSs zl*~QU%OddsY2LjhVBCA;jVjlzyL*VVFL1s6yQH}a+4bbqI7u%Lb_p$8q9t&4pN8*o z&gy)b#`-E)`6MpO3tdK0i!b<0>O4%m?V(=WT5k*6PghLDvm*48znRdT61~O7%MX`c zzmhrTF2;Bk48xW-(}M5j%%&s+_Is;=T2bQnSM7s0V!soK@XZsc4giz-@)8NHStw*f z!$w%|{J)ucJil=)jXO}$e{_lO>#@JIVW<sri<rv-&wly`pa`RSFB)%lZ5aBcwV4~m zHYc1<F0ox792)T6|EEbRl5RK|r<yOK)0hf-j9VBN@?R0rroV}UyXl|TJE%I8`quRO z$Hx)X${$i%gV6^BVx{a`xvS4dL>LE_DecZ<N+@~LJ5V7Oqhw`ksbxSk8N!rhNaM}5 zW^QjT(($h_e`Y$a-h950LrD_H$eTE;UBYk-b}U$5+5KsHVf<^m8KZZ4+?_zp#gg~J z0^|PffNjj|x2Hsg6ut+1L8b;18m(0UNl0ig0YzB944V-k1ylr3S+n}kEp$Ur(D0{T zv<KJ<3|3m$LIi%LoNV~XpK}pN8(bi|UHyOy6)}+dk<0Yd*2jkEXAYBVfJGP-*|~I^ z-c24l&Bar=wlPci`l(T{tkrYhx>tYcZGXfq5%lQMO{-DAf0Om=zsdR%1v#2bu|`Ol zd|C>1;&Q&+eDZLc`)kz6%*ltCi!uDh^XO+KLgu4?>M<ZEJ4Y^m)8%XlxmGcj%o%3M zgHJ;>WF|$*ZU*jsv{;fSkDZyOb8?0ji1=Nv$F#mxndy>NFE_(a%E^hiY#7Tdzg!9O z<a?3*HOnPV9cxPq2oU|1sc7&-)<Z$8(l>%0)F?1bUNsabj4{beylww6)KizQ=d(gf zGN$`rZ4P0qVv<4$AP?7vfeAEneb)h{!wi1ODW8f^=-mu&!nB&*LM~hYSk}EZHffA} zF_>D4QPt8mX;ImwpjtZhhZ+BGp56y1%&}KnQh;%iqg<=8c1bNB+ttj`(vB>e`kwa6 zmbvy*F#8FAGE<yj+i;OnoD5a>gsG;u@~LxFbSh$et)7o$YqG#b^8LK8EL~Ogd0C5Z z=O2YhhNUAx!6eBH-Xh{mW0v`%M2^Jq@w=iKFO4@g1eSk-0_@SA5KmueUf|NW&-45i zPahP<$JVk`QtR4(rDG$a^ksySqGr_%lluBiYlx>OJ9dmAXzNxi;{>5&AVRh}qD$wD zN^TE)W575dnpT>lUvd?Pt4&MNtxsV{uOqcs;s|-eRwQU2O%#4MS{JPJ08}!><e(x- zy3_4gN|&DPe!$JFpIc{Y^qk4=%lFlm^*OinW2N$_iy2E{yFUi8&$O0Vwid!D9!C`1 zX1oK1yD%{4*QqKOy^hN3!mc_GKmPVTuqw|+Wg>1uV#0*2@=<*R`Zc*FY(SQIdcxkb z;xET(tb%?^#ck|sf;6Qfr$;?;;IUdf_p$8QD3o}Xce_gmJhi&ohBKR1zF~R2ll<2A z^#=JZ!I|Q=+QcG%nmtjoq}a89pn#1vD(Yr1M}rqXT|9hfVXmH^NG7ecxB=R-By%*8 z6vGSN|14>fPNLn)%Jrp#q(nd4dy-I*NVV`#R@QF^S4&G}pabvyOMk%*6GZ+gaXt5@ z-0qLGBw$iVpmonHulsfob~OrXWca;Qy8J<jv$8N-$TR02%$*r2A`N=*LrCRk(1XIh zV4a%+fjS)Nb9?SEAqBsV(2S~%a@plk68qoTu=nzmkeDW@{Y#IWh{<%<`YG)L^^CMW z`I4TXC6N=1$1GbDj>i==4y<&`=Nhctd4LUFryB$zmPIF@INL>v8d-%2s=%r_lD@C> zT)Z86_CuNB4autHLYvY2q?DMt&!yPJvl#EqV;z^<Bz2Qu7em-X>?=06%&jf*{{A+3 zzc#=NgjxINNa%9|Pb@#@-SXBIh{_z*S!oG*J1gZX*AYW2`MA`Ki8t6krRPLPQ$1I| zwv%q{n|a(Axj2>_)YGeIn2rh~xKYDtSV`ETgxBlUt4#c<KDp+7o<wXk7PgT-%60WS z`8n%A8lid2MxFV!4md0&K_-yw@q-MCl~8rI=dTDY8~r4D{BcQ}_dug999oy9Z%v_{ zUf0-{zF0UjQ#K>7-OMpQj*FP~K!~ey#-{9c)8z2b*7zbqDLUGWRfSb(eJOBwb7SZo zGD|$(<e{hv=b8@e9_0_LUEF(1fiH;bYdOPRm6oz&jaiDJxQJFtfn&Y#xO|w-#xaew z3q~D=<T{gd!<MHC7CMe1lgH>6qTzANl*Xm<5RNZ~NZ&s4!UN5w6-VONB{`ItMw+au za2Zo%e=@F6KaqJc*xyz2RBx3RA3(>GO0x*@N`U`l`4DSY`ps)0mIO*TvPf=`^yV+X z38t9y4pyScxsZ>Q(G_3ubfc$wCs44^?vt$Jy#l5sdKZIfab}k-0^#0wn290teCjq` zCm_qz|E(~qXsn{?j_n>Wsr}|%w=$+bJN(4oWbhl>$rSdxI5IsV51fk!CT_E)v2;{- zUcs~t#IP&IZP1GFfKhwSzA3Vsfn`fn3S57m>}!l|^6|Et*+q1vt*_7q-TM3G#yTe? zpw}EjrtylqV6hhD7G3Bt+HQx#FF*!M%{}v3F_2+gT3XulUQk=*`t<-o#vFV>K?fux z+2=|`(1d|#Q;tCyq3@1Yf7y~BLYAW6`n)-z{}JR@q^rU~qWg9`$+r#{2G!%l*FYM2 zQ=0OJDP-WV9iHwg;XePEJ>wOZSPZ1ZU$q2n>CMf}`xrC^x^s;XeE0L);$*zLb&Jcq zR#5bn+F(`-LZ^iUf{<fDJ5OlbEnw*8Ry0AONGvWcLJtj^fLmG=B;~x+rDqZisJS2V zn=NjYW<UYS68!W{>;_vxE91ydCQ{&aw-i5qhNO!itf%U{QA)kio}^1C!~nXYqF?@o zG~8jUuo(E8h6h<=Ww&CZ%7OkjPU{ul41Ti`43?q~_lXnx*Hu8}$gzFGLeA%GtU(^T z-}>@`vG4jYziJvX{LS~?UZ4dGH9%sZRv>XANasF5`x-C<PWlR2X4CQ?25i-=wKia7 zWrg~%GDohcT>i<dty<l+Lj(Ic9VL5#5~C2##-aXSF#*usK|{eniglh@Jro7N@plx! z6Bw~v&&bGV)@yxv?)L~;ro})b1S0d_54t0f1yE;(_MO<NRSH5d(0O_7I;CaPrHv|p z+m#<{iZ<K&jGbOwID3|f*wxcRl=Y*(KM8)qG`!nsh#`JWUfXY*A-C2cmiH7p4ma_- z?NnB0@Vlq6Jpd^Ofr1eOi!mr>dn|d2w0|_PBLLuM-x&hLHyHf>5Y$c*<^Nq=Fw4co z1)AF@q?N(8Qe(CC8_HEz7{{4zLIp^Fw|ha59r^iyp4B!{|NHv;qtM9tU3_EzU)TAz zufG-EE@b-q_n+_EzWz>d=epzM^fYcK^dafU`$v`V*TLT)FB3|^Pn4FDiUF1XAU?ac zm2nDE1vW8BjF$J7+3MaA6wW=DMsr<zGd5<iw(}VcFfgZp=n#E~&G&%T(^VPE79OZu z-!<AoKS7;n%fz{hNgou9kB|SY0ie;nb)=ym;5kh6^gisYSD|Z?t)5xcJ8Mlt@Mj$K zZUJoYucj#Zw-oD14Lv=stNaQ+<vV}BHM_x+tW?;aIB{Ypen8{kS@i^-tGUnJbD>$# z=hP@dyRiL;&~4ur?Cj-_zISz*<GcTH!L`I^7#S2VEs?-Ge$MU-cD&~`$JkGAm~GEJ zzk9XiTsseD0)$0(o?+)o=Dy!u-MR2@xr3cc|5KIe{=Ut}iGRO1D1Mx_f&tUl#CtJf zOFwMj#}BKWtpR+1pOYS9!FbCb2D=b#=jq;~L2n^_1UBF0fC=C}hXERzwDUX>Nx&-U z`|IO4R54}i7F+9FXkaC68El<9fmIp@QSJk1fL4Idc$-U&&}&Y{kvIyQnAdQz2iff| zj4zFhQPWdXH~AmIZR{@MuNK#XdnGy5UV&_wf2R27Hp)ba1;%it>2p;0Ou1!3Utzzz zee{aaLiK{|y#91UL~HZ6@EO+SmFNOr9Hr{({11W%SiTBGvbuf0)lrdh=)NpiP#a)J z#a6%<EUv1Bx#9W~Q2$dHj4l9;ym{I;?R}@!^LoIf*2&oJzFot6!E0=Nb#=GEM1`7; zzfJ5?DFsv31Ty$d6G`Am0B`wy;yEeI1y$AFO<(0STr|D#t713N7+W^lI9){b9Q~2U zJ{HUhXSsD$=qivsQnIR~(3eQkW_7QFUADj*aJoQejkmFhiT(EHfSm)p6%eIV5dIbn zikG0X^bta;5D!dS+sp=Te-WTv3uzNnYP0b9`zj~V8B2|YQUyyQrS&|U)Zk86{Q2U3 z+f!??cMZvyH!iFeu>t)grIu(+k7=!=x0ej<9&7$?1`WYWBuOyA*&!!PX>e!6BVM?R z0c#}-8s_JXAJb`P7Z&1kj+=N+Eys@i$deA3aiiKuVkIZ82?uu4W;U@C%On1*m%lwb zNXoCmN}_#vj%!Sg7n>&ZS13H8b{I$#$cWr>3uAR#b-WvR8J3?tIBE$MYh1LI2f609 zo{vg^M{&f9)YoSvbxS9A;kuiLs&)s*3FE8Cvn8S+tAw+>BhSXyC3Jn(Bz=_6%ONjX zdTF{a(OkxjoqJz`#HRklvh*2@*)o}sNZ-F9a=)jQy+^o1imqEm7)uFcZkiu`wcn8R zXi0jq5596e$XLRXw2AywKHjGg^#g~&s*`nl{_3;nPH?j~ycKWY=3Q9sn;nhg9Kaa( zDl0qjlr3gmOd&3%1}(9F2Kn0!wcP)TR@0FGTcov3I5#?!_V|EWRl0)!eDkSEMh@CH zJ~7rzZ=mpL2Ee~mk(sP^>L^s4O7W`5+G-+#7=QGQlkKw5+`<Poc`jRg5m|X{;ghj_ z1*FnynZB*k+7G6-1p=gNc-vAVzuI_4<M39R9&N(JlCmT1@_M_ZZ`bn=sCBN|lvC0^ z{lHPnIzszN>5+1->(AUDl0v`E%}yoI({rd@<x4#o@HG0iOsglIolFW8A8m?e`!g5m zxfWPBQ$v;_E;<sw&soUmip;d+wmj@y{e^@gu5=RHcTQ0-Hx*$v)+HiTPng@Qluw#L zTn;e`3Kc2$FOR21rs75sjcy&?$xP0@`hley+PKeg#vR?XnBZ3?b0A`GVJPu$omQ+@ zFMTXH1fH*}xo{@HicVG|t2X97pogba8%8U6sTvOg-;ZjYENdiYB)lmpGks1PWNYF1 zOSSLhy`i(}F>;7-*=fUtn{V!=GQAaa<{D71m>!Ba;e->aTzzoy{V^Y-%PLI3LfG@~ z!e~q>xG2=2qi1?=6)GG~Z`096njJ=IA5hJauBomna9>yra2p!@v0gia!`ZR;>-a3+ zg^hE>XqUC9{E`tt_hjZLL~7nqc`8O8k)2_rXn1yUC8+PK^}!+3|CVa)hNN1p`GHhx zd}}5yPk(^zp|!$#&xBVQy}PyFXLWt}I|b(694~7mtZ+G<$)RMow~g;4uE?S}jOr*n zH6Ch0T*%GPZ@{>ZoU4*rlx19So<beSpippGi^YqtFpHu<s@1sYQP=8Vd0=bMc&VK0 zSZmdWWIy4U?V83T+o^$tSA$sYMmu8T2YUE}!tpiSkp$ls4(B)pU)R}+Zt~UW%b|rD zAKkKXM~3Ivt4c4mT=Gf5yiJ|-zyQ^~v|;sx9!>DNb@;c8{O#RHk@1`E8I^|_4i4~* zzOsP9cqKibJHj^cLsA7c*C+y)$M)y-wI&RdR_T@%<l1{m!e>Q~%X^;9iAO0p*N5Uc zpJlGoQV1V1H4M`e*9Lsb6rNbtxDGfr$I904E4X2jO6ye+)0Z!=7;T)9Mg%W(gxl;# z2W@im2Jy3aJ{nH7jvzT7;&|re>UQz<*#7p5?Klo5&qp<>Z}Ce@4=7A{wY3cKSZ3{2 zqR!#gz{z*@ANam^ZU8goWabOE@izFLmf*_G(JDFXu0<YRboFaYbFKnk*U<;nIkvH} z88?hAMbPwS=WK!=YywoVWC2>OMyUcPq_gi%QRvV}NdJ9sfYE@R=NZvzYseT3XI27g zh*Ew2y4TVLe7@d`_>zzPrGh(8@AgalN2|5SN^{lyKv68he&$kd=eAaBdiq&}T;+fm zzNzHUrlDa;%vuy8Y3l7b<uIl#1E`F<BiDq#Vm*hjvm5GG$DYK;#HR0Um6b0sO}S)l z-7Mp4>~UgPIMK)M<sW?YLlnxhngAwdi_|4%$nrX=^LuxC14;Hn1p#)*fcN|cZ!xs= zuO!td2j$w<N^^bv$`j8h)l`dC3u}oJhjh;n-EQgiEf^FTmjJ@1lI2Wb+xjG8$dpvB z!Jv|>=OUTN|GsSXfwN+EQIYf$zDiMcZo7jOw8mYHYY&SE1n%%gu-jfV-UQmQJ&<y1 zA{R9rEyH!_#x0}UzP>f}Tv*<CJ?aNkhqcK%f=Ko4g}-dyxVxHXfqKMcCmbhue*7`h zED)G|o5QgH#dkvCg~%bL`ORmSq@*qiZd-SLFCBoFxQ&K%iO+Nm4A}8sG^=BgFmK=h zM|4AD0Gnt_uaJZatPa$S;cQuVE%1@mg{{pK^PUSQmC!zpepV0-Cb%iQF>h7R?4MfF zuY?D@Ou^+1Nu^<$vpbyO9%<5yy&Q%}*tiZtwdyV@8P`)jx_@c2xw?m9NpvM+4eix) zE2dF8<V1E@#6p;!!cxEo)#i}nOV`P=JU*|rbBhKtnO;^tuaz91C^<X@OpMI)8(tu8 zsKC4S6kuxjH)i*D^!FnYDU`S?NMJ10P$8*WsE8Tx>IydR1Fx3e%)oF91LCq{p%8Hh z-5EPUT7C|E>A#kN+IuT1X2~i1%@#_}fU2nuT#B-E+pLz?3zUHIBcBQZ#abZLd5<%O zo5`}p=MBv}{hl{Hce8-?>k8*Uq?m!Nu>vh!&LrhzEQ(R;Qs2Af2bTI#>d9X+8Ge8R zI|@3GZHK?95)m*-qy3D;_3ls6`YVw!vKE0ex(f5`1)l0o*~Q58Nk4)~;NJcXp`@k3 z{4aaD7ir&?nxG@7F4TBi?bTxx2^|2(&*Q(Fiee?DfoY{tvc7_8X=!=ORgMe~gKTL) zlG=eDno#+>nMk`B8Ul8IUtRCMeg<D3#;*<}(Z#2LA*n2V0nClbq`|i&I~p}S!**8f znmE@RErTuxNu0x*DRP<q0;)l(4`E^S0;2&o9rNyIr~$f{CHg_0O@*8~Dv~xeP2UsQ zQPNjD)c)0F(yj4>*r93*CYjZTw>8C88_B4?NM+SbMrwP_nrbq~`EsYl0#jekO6H)W zSUf0?eaKcd_Ob#HfyMiJQ0fX%?j^h5te~>84&uUvPfqHUpieymm6UG2`VH-$4zA|5 z2{`n-{#kCsz;`xiH}!k_*<RXnexwXC32hl#H@)v)e!Mi1k;ry{9C8Eo7g@k+v9|8E zM5S(5e?B%ArqM(nNcg|W!iHxpWS@-LZHDnr*!@x_k*SPT%K2Qg>O>Evp{teIPI+o3 zSHJZo9{=j7kHXyj!zK8k?tHN+&$E%kFTcOPtjmSbk*svUD&4Lh4ib-z<pwU0qC6BC zAQZYAD)OG>B%qQr+pj9^SOlP!>&Y05Xy$yzB&eR0<grY(FgkYk)Wvgs7P>L9ClH^T z0<#Fl?p^zuZz)mKVCIdr_7dp92RKth&)XT+a8F5WU!tn|%!jJEiK{>)R)0$*wh@TL zVgOw0icY&I21H`<n>}NJWM<$T+OAET`ogHWZw3uLJuPPq8*hrB-x!{{2rF1KZgc~d zK0RVjDvI;0Z-Vhd^<m&x=|n<~nbd{otL{#-<#}(*z#!KQL(7R3DC`I|`fe`y5^3Pk zS%P+KcNK5%ijvVtgQo5EC!ipD=>zo-PF*LeMf`OH{sJ%c+dT_Sghf2C-_?n<Rv(-! zP>%Ik0@GVXaOIjPqLuBC3^@BVpDK}-^zD?_267|0><${RDD1UrV%5Gk1kXn1ZGO&G z{pCaNEzA(=O{1@?Y+hk4!id}`{vIc=m+b#K8mOfvG>Pv>$5sSsQ61w)=9cW$hcnuk z{1_NaM;6Fs)8z|l>>qvz4fOl0QC-PUVu~9Fz{V5-0wDC^KcGfa0f2gZ&@X2P(SS+i zbX($dT<*4#>}oW#E-&y32Fcr9;vy=>!=+K;`t|3yZ2i75CKjp%#&Hb7uQ(+==A0{3 zRXCQ8Om@x12$5Rdu<4BCzKgw0EOg^*VpyLD?wQMmjSpbEShR`Sfu&B^3;n-3T_x<N zdCKsW1j1Z;^3c&8DcJ<Oek<K<eg0=+oL6OC*Kd3>q8s*T^(f~&_H}nqw=!83=E2-A z8Nfxbc5K)@aGQ>gj3iFs;_AEY4%P*q*cOwG=0r>VfW&0s-RnOv+GbM|IVYOT!t4)5 z!ic6u3P!IoO~+!o6a*ho41hkg7lahR(Y?=~@7u%h*+CO4i~VyigHHMhX}ce-tM)z% zu>c`Yu7iB>%%e6pCzVuvVCA+{X-)^NM$N!XLq>_fm{22k<Y#ynkXg#G%`w`4{gfiM z%w;Vh8$~Peo>p@53A_?m?8KO37;%|v9EdwldI5#m&~1g;!kmy3e5%YBMNM!&_Eq<7 z*lZTGmk^8R+EZ#?5rVjARB?5?fw!&|DL6Dkf>OlB2F7RPob_5PTB$#vaYmhIfqU($ zS7l0FFlJ~D(wX(;|5!oy_x?&ecOpPMtrAs{ZPL~HoZ528FvD06<#-TGZS5^I;h0_F z$-RCJJv}{^VCI8`Jn=)kx9^wD$(2G<Cp?sIz`jN~S}5O|H(H4L<>p$h#)D3IfnOO8 z(WC)qiv+~Y8qQ6_uB!<3`ZzJhW+}u#LU%Q&v>M~2Q>{#-%}F<>s}h?{tvqGaOOIlK z<m{wJ#q4_Mu6m3)Q!nW{A~&YrlSU*@uP#&VYKV|kmJaM2ii}Ik#rP%R#Pp(E7-v~( z5ZevA4id%~9O4=;G^KT<fE&y-`-RISiqew{q~S)n<va>^s+%1<4q*s~Z~m=9YcVj_ z|9V%h)^q^cHN(`jm}<?V!>V>oMiZt8gdU~<I&if77FiKCyZ)IpKe`lqP75~J$piyI zDca(ixo$u`^#S9wm;HL;^*rp<Vk-<sN%Ph=zYeSJxZ(}0%eQMgDzty*;fHz*f$@_s zXW#MOzuGGlhsM=mWQ-e|i21mmBEJ>|fJ7<4T>X-d!lB|o&)I>=mFZQ;v1OYiT`e4W za2GSOCf@P>HKDomo*h%NTz~0>VR%pWN?yqlKR-+540Qr)!kJ!>qaLE)GGIT}Tp>Fw z!*O_lSZc%T$T%}Er?+OZj@=fcC6X-J$>}*Fy_Mb1#kHyKyo}VBK#bOT0EuPdYoY+? zCI6qW8SvgYzUta4!}8EM%>&$6Y1qSN$EOnbmqr-H`^CGBR6`Mw{>%qsUeFH$o2otU z3+E9K5`Rqa-^HgCpW4_>vc1C=nTQ7wj9+=h6Jn+zZ^DzYyYpQ)riMAy+8xiEqyAJH zPv}nxTw?KjLtX44H5KETI98LDr~o!lO#Azh{4N31S0eRR$i<yps~6qC#V6uRXQQiK z&Z$>g>g%rpRa&=mB=^?mGjU+?u%d{77B;`Z`AavOU<%&4QJG!4(`t!{wOZGFVd&7O z@t~!se(E>f0vSxGx@c6o!3mJ1Ncav=9hMBOrY)l75o8`BfoNg}SPlTIj()#~fu#f- zUwgod+Ie|8zws_m3`<-)T?L$QkE`Q&Q2i0YiLqb<`gVV%x#!|SnMHF9KQUJxmQVB@ z>^aL1>;)+}-^$$<sob>pGaEMiAH)n)#t&WhzjB>HQED4rUYahO+2?CGp<sg?YXbLW zIdgV}AxTCY1Z^L)Ij5d*2?X%a+$EblD>hasRSN*Rj^SS)+z+OdN>uoscb&Sh2#S2g ze8~iresOMG5;R;yw!bS-|KvMdydd~FL$?5R((Mc8bV>OoY1IoNB1O@G3Wnm@Pi0M> z_;Oy=SyVV4y!NW0J`U6@izU&g-Dv6h!H`9wTd={eyZ#nR+yf@ay&)wxEiejpN=LxA zy(0Qg!n`?@b1V~o6xY3f<wOLc!W#YSS1^~!Ww(I+`tSBe`ftB*_rFjy=vR{BB5bzc ztdVcw0pyU7kR8+@b8k_flk#jiH=r_0DWci{<O*ABk(~il64Vs02zPn)WS%&9_N-p6 zG1lY+H#abfujioAjbM*U+z{W*V48Vh3t}AuAp@s(A)kXIP`s}8MeyzIJLvp5R<ZpR z$f$x#?f~mMSN??q|Gx4+Q*zA|lvw)R{7%h6mhHFNn%?3$O>F~Yc(5`sXe^edP4<#n zW8I3P6Q8hT2<yp9gLNZOQBiUFk+1m0DTP^kSKB)XL`ZM(Lsh~|%}fM9#~z(kS&js^ zKD{5p+qarEl-Y+5Jzn$c6hl(JQOSICiIe1UujsZ}5s8P@A{OEv9+j7P6AcekLa1ke zGcRxa*pE2?#PwuvIA(*aoLb4*o?N(0-?btzdqI*~p*@Js3dq}q9Glv|4TLgv1_Eef zsUhX&(qZ5rnjXG)0f?u@6e79Aj4ye6dt1zpeES_V2A)a44KSPk<y$GHX+~rL&<fEK zM@JG`xd5!XuZoFvP|%@h^NUIBaJ~8ua;VZp0J&Zse-lB|8xkP_Ut+{C7lqHcUvi3S zscvg%HrAXo@Di@fci-$~*Tn@XPdR1~Y(yvd+BK<YcR@xZkIkCuEFEL1uZl#DZIayQ z<CmHW%9i$%^wute=@cl~1#v=UAXg)%RbF``u?{S4iGP9gWx&LnoSI@ufGfQN#7NS2 zfnS9HD=GAgOP$oBf`R+6e<?~yfTO3~@syK+x*Y*{o|4j%F;xeGidy1UPFe!#WucnJ zo4jt42NT2mBM|}^!hPjNw}1u<ELK{R+G53HtXy`q!h|;Vg#N)gH<#uJCM%ORpn2NQ zp0%6TytE6P%uw4xinYw~nPl+w%R~f;_n9lu_(ce)vFVMf?J9LRQ*O)AjZBseMCtsY zS_QBVE@I*$&Bq$TRD+4hI6Y)#oxOJO(1J7NmaJ%Z_C_!O^dt(I!5TjH2-<FPtwpv! z3j8bXpS3$4ef6vYsh9g%Qs*OGs^uu~lIAh@#)61ac27ESXjRP2QOFtd9+>;a$S%#e z%5k`<FmcH2<~yZGO+L?Zd_?Z<Xd82GT`<1|pV$3JIk*HR@ai1>XTd{M<rbH>he6{m z#*mu(82vfM#9gIp^5sxb6rvQ1kI&}+{<1(Moz{^fW}w)XOCXDBAHdSGv$6zN`-)wP zZTtiNsIR;u|MZ?wSyf>`$Z(WPg@>o-er!TjLcwL)sg^|+3!>xe_$r=yjf8;9xv?xX zK-!2OdB3%@7=fwUW@84@PXFFoz(jQ+$GgR_DXh22CtffEK5~-QqE9s64bL^|l0Ugw zRa*s=X`Nup{Iz*{;G5g}qGU|*Up1E!dl7uHuGPAeuTZsPdOzV-#b@SL*HP1gpEA0O zeH_$x-DGSp%<-zes!3^^2b0>ADH-JrE$@0?&p$=>Ub3I=cb`!f9Pi_kjl2rntbQ{z zkIkm*%t~Ozv<HTM)oIzULaKFoAn|Bhla7meBer5b#t5&3Oj%rhs+K5pUP?M64CXcs zGjETVFv1^X-O~y~BF!H0?h#(2D$K$SMB4E_sNKth^IxvrtQX%r#t(q+id&^Vs6QvV zQ>hi}-}$s{2nm?h=mP$|<aXWw+Gdr`iE(i%KJ>bCODErbXSqv<=<v+N5s7{M_Z`Wf z`SFsHbBu(Px=;?M{%&B9cqcy3jrEDNZURJrD8zPxoM!K}t1VlEEujih)CRU0`wpC> z_yurtxgem#vk6RzKNY+>2Yjl2{qnI6&}8B+Ns3{P9<6Apr5Lr{<Q(U4n%i?V#X;B5 z!^5MwI*?h#fQZKRPeB~bfi3`TbIu<OVo7~gBQ-?gEqlkpCO9SOJJ5f;Cly(;`5Di; zToP$C|2d{mD4<v1#;>Vq6PD!?Rvw|I_fG_lMF?u>+@r0hw}hV%3(M($cl3$Y$wtOM zIsISB<jRK}&pB-vtwFNX$1ODu6wdhKyxk5TV42C~D;bvr#&^O|@r5Pz;p-F<7VV<= zqeG$2ec@el87I0sgIB#-H7zF&NC>JB>uWDe4IryR<4yQ@jxkJ4P!)REF+Rc+PbErQ ztBL%w;p16YeTAT-Xia1396C|B@Wg(q<&*U*)1?y&pBWE!_{<m;3F}Q?;Fa7{|1fUX zHDD(;0$TXqMUo_;HZwVSD+3x@xI%{(K$SzCeJ9@8Ot5uffg*Ly1s@HSs(i1Shd+aU zarkSSHnCNmZ`DGejxp6XB}&VJkVLCdJz}iF=5$=LaE|sG&!K~XXm@Kl858?NFEJ;% zXeGBHmywjkXV=|OeWNC_nn{@zr}6iE(tfrWUH;YSpuA!Bl`&%-<<t0~-#+0-vBl>L zmh*rhur|9ktknpYThZC)yfzp#UC2+U#Fy87eOkNt?X30-agv!}q{)Mpyj@>4&^lDo z22cH9%Wd4}kH)F3Zv;SYTuX`VZ5A_WA?FOULnhLJbk~{QL>D6_vRsLK{TmZ$D4qZm zI)OU9)%@P7J^f3MwS_HMFfJS&W5B^k>GR+yoU4&q?Pm{msk;lC=1}U`D7CS|I3*$l zIQMC#2?cI+?v{+Di*3k=6o=a~mt8n;(#Cs3hT{ovTu&lmnHWH9gUp6x*m~_W8gXK+ z0ew8+UUYY=8GWUt+tj2Q*6TVmppcCar@u)h1lg>Qd)&afHs7eH1Wg<JOq2{h;f(W? zas-RIW06HqRC(WLW7bMG{4Llq`Xo}uWC#^X*zd~%&>_fITdQD3^WL+CujKnS90PtJ zRn#PsB>{p>yOEEsBQ52>F0Xkh*}!!-YwZkj++pTgU3^C4YE=>;2Dj^zC=nqoL6JQa z+tIJJF0k=VHY%^I&!f*Cg(3=ZD3@%Xk@~8Y&b)EC(V_s+$bk<&29La-uUsR|x){VK zOJa%kMG0QOWn$<G-3lPx9#B+Wc}c<ky|o-pt1B%z=~>ndem%lpI|c@hm)j{WmY(?L zaKBh9iobBK7nfr^cx$MTu-o7YjrNP67%6)wFp>jv1l#6vjfd+`7QIhQGhzIM?-!;C zpt{)^6^icoSA=>i=iY@qw}+4@lHh^ST<POVJQ27GabLM_l<zURPc>Q-?fOk2qa7c{ z#CIsjgwIRVX?-M>9^NuovfFs*<tw(c7%cOE^^|SjvK3(tWt6FIR+F(<LaR-Q{OU6+ zdi87#5Tn3eMbZLJ7J5uA8ypXSi#n6{A3(zTU!8uIj;Q!F6aAHvw1n`yZRv36((}bi zJG+E;NbilsHcvX$O${(_BUi#ugHOgIemPiQX3207IF?13?$%}r@m3h*8VF@6wi~d# z`F{2SSzF`}20xo^2X1FNnts$OCeOJT>|0+C4hgehQKkRcB@bg`bo0`rUb#7xl?lpy ze<gQk*gW{MuTEdzj$)F*Re6O*Eb=F#RwI;)dDH!6hPvY(?eQOI#m-Dn?&j*^tcG^8 zpXr%f<s1R^)Y*G!MgJFT?-kd?+IA0viUm|uKvY0<t5iXeCLK`}kq!arAW8|H0HFp2 zK|v{sAT3mB0fLm!LXoagr9-GvLJKYQ`o9O<``yp`ya(UGcOQtq$z(D!ce~2E)+#RM z<2mCf^yEoPLi|_toCLAOYAw}!sS6@SOBQZD+{~HpsY8>gR%DECNpYlb`A61a4Nv7t zJTl5`4R5ml;mi_szk=r2`RZkZx6EFJLMfi}TGaVG&&v==IZ^4I%SP3SD9e%WDs};0 z0}im$jgDZMzH;ebQ%+?Om{(2Wc(Ic4>AkJI0mu+Kd0c{1Z|sfvWxXy#&Y!+>+pU2~ zFXOrxU}tATy5WNAcX=3bUK+Kz;o;8-dcH)kD5TJgaEd2&r(Mz$Le<{1p!N-5_I<69 z-T-TETg1yFHZR%0bQN44fk60Q4H_r1>S%agWK0UvS20XbC(leY8+NR~Z++#8Vxk#| zdvOli6)_dFwe^8JKb-SOxW6S}lWu-;wR1#(EkD?{!t>|lxY&9@C3x<v*?K9*YhA8Q zW;-|>zEo<1W~1wl0+^q%e-qDeMaeuvO*R*B50w0}>MI@av+r<LyIKMQFLE?Dajs$C zs*H2(CY4AcEwMXSj_0N)>K9RmdGqbw@i|qT_AzW|Q{CEeU{jy_*C4ZT(%@i!XIspP zQ=-B<o#P!#nw5AvhOB4YrogZ$ztyYL7IYnl8*MUbe4_UVnOPr%zD>)!m6~twvV>Nz zM4$-^f^AFGS;a=$1n!EQ(O<vT*SOC$M}NM2lw6H+BefKegzl3-^0TdP$Zx7Fv;{TA z4AOQqx*|CmuNtkwrmF}yU##4%0JgG^Pnk`xcQDkK{{Tn6K&9)Rz(eDt_a=jhagk}M z7ndivu3hAfE??u8%hGN)ENg~E(4Y!8>kA}^U0F=+ht#R7)*0pir?}ICUIM<-&*8jV zr3BSNbNLX!1zRx`cq}jL@m~-RWMy=7xY}BvWq`Y?6nNEZ`yrs{JfgevUD<n{7FP!U z+`gm}EQJ-En0VujER#Lb0k^blluKIZZQC=$c@C9O#589#M|yk0lXOUMo4@pqQKT6_ z(6p@IXy;kL2@2H>`0rH<Ric0uEA7-a7S84__^Z@o<2NPp)h5yq6IkQZzz9~imme?y zW70<;knV%;7^f&i0{V9jB+0;C7Kd_tCG0y7keyAiq6PFJ*Jz=6Io-PKI;V-oV-_=p z<>D6|+L_;CCN}P|!-gDS<89S$KD3k;?iM6d^(^)zOd=S1&lo#XI;N&bibE(&o2~<k z594Vn>;8qbNGV(AxpDYIgX5WJc8gSvZfYmk@4f7{h~;T7`nBB`mJt_OpwX&MMvQbN zQ77yGYADKA$^_9B`6U1<OX^)|Jg;t}ft&wP<K9KvZ?V08Pqyq)BPIVAKg=nI^6squ zA#M<;$<P1^uBLOb67F3dtJ+J!zCgz~QUo!vZ*Irg5soRkNsQnM?6U23Oi@Y@6z07- z<dF6DvMRK`z78Bm=^K;g^R~X1Y*N#Vf~rihm(ViG+?qCW?oD(|tcqKMwu?{I5$b)4 zZ>R-*tC1rC?ijoCS~T)a830>s<L>)<vuf+t;$ySAVf{ys_V$!p?eX26ubb0P-DNPY zDr_&zxMG~*nJ$pc>eb+vFCgiZguxUR&ogXlO!De-YX<%pO&1a7xrL?r+B}lzJ5&fe zlho%gG}L#Y^idz3p(Wfhw?Li6Un$n(7EezJ&S~`JPt^T{sjlU{d1jkxX7<20Cg~=- z%k3>cmjDIBZH)uaa<ihO`~kwt#euw<|D))(FsFzYkgP`6h4K5ReARaG)(kC*(qNhD zM-TYA*lkErk)Bi`J>qyA5*a|e{0n5DmY0`*2maP4{wA>vp6Au{8tOMVV5v0fG`-)c zjX5!<sb4>GDXrMr8m;p_72;z(Z`RNd$zNevt$})xWks3w6$GK%FfUm|m!r@bH0^<( zc5~Jf*eaG{Cz-iD7;&s4<Pn-|A_V8De6gpoI3k876clKY5V-d|6KpUoo@_Rl#4RqP z&p#4nhm>cl{fD;>A46FS`PqD7nW~zf896@SUzN1GrAk5xvq@e{JY?^a<(G;HuMc96 zn?cBRo^CMSy$E6(5NHsBT*N<Et}kb6yCdOP7{@l;A!PjHu;n?r=q+m_>+7m63v+ub z%Bv%DZq0A-)ootP@`|xxK33g4)4$pgJWn<0oXR8lZTV|kd+dm)ABQszP;SyW6Rua8 zn?xQokvY;vxqLSSQj?~4rxNm(S5u`3<z~|H3L|*ev({>YW+}y|sT_XCOSb?M-g}<G znfpsrtU>nIXgVhpJ)C;)CiCT!Z)U5A3RjcN=*V6ptEt8)J2@g3SLtNe=N1uW3v!@p zm2mEP47(sL(}fq*ADQ`gMji%k+)oe385bnmhcArDoyC@<0W;k?m{YKfb*JTQ;D};$ zL2BUrzyZb8l$9#X$ckUx63A|Tp}Gmv=5KducTZ-9aFd^+K~4k&QBH=lIXWU2Xj<lP zUz4@{5PDWM6s4n;BIe2MA2bg#f^^-v?Mh3SHbXwQ$m1YPc;WoNJ=ha-y6Zdfi4h#} zi5NJqUCNbTKahhBPhZI!^GOKIClqrtY9hjV>El(U8>u4lV@E3_KG=Iw=W@Cv6^Eu4 z+AuGBeDHobw|<_7%Y%1fsw>d67PC^hJH4HDC~CC<mQ9w${e*FARB*+Hw>K5=9*-bx zZ!Wv}UK!Yq(hbHe6l1)1R~)*le-&&8m|&ui<Lxc89-VUIS}GcyvVo~~VHJN8D()Ul zXDL-vyPbbOjf)7?y*-T7(F_K$&tn=j?rt+j$vvaGZL3KDUq?suRI-zFYId3v97?qp z`}mq$ShCmx&+<q#*ex#hVilHI+3Kmu<<*h|b3zH5=)A+~juarRIbKm8JFr88kBPR0 zeOK<jS3f;jGL>St);r;Kp7Z7tS#b(!9M?hEWpi^gzOxR9?0}eubdUSqM)Yv;ZtFY% zfT5+uVk1Hjbf=<Tw;s{YG0rNI#a;d(i##2>?2U|?6?eYr%05qER}Rx(P};(qDz2W0 zJaKA@?p4(BK=@;|x6hg-z!f<m1T7jlXz@=WBM6Mj$5U%C35!Q{$!n5TBCGCFf3PSA zPutywaqn~?{l%b3&d9Og9qkF5gxlDlWEl##S1ckIO6B?TQC@RdDp<j}&}ihC_+@l- znr1OIC0Me<<HyrYS{8eEyEhOoBCTL)mV~+dXkCWQwrdJm_SIzTl0jbj^ea&`?*lu+ zakRX;F%j2DwD1Wa4KUhX>|CFO$ew8n$2f;)s;t{;cb`3NJjKjalPvEW)RnZLAF*n? zk-KG`(f;8tX1`7++PFu75-`S_PPOG}EllVqUghe>4@rmuOdzIX>Wy4z_b4vE7|~QY zExO6T$?sV(rE?Cr+#37v%PzQcbBxHuDzN#*kOb>m9MD6d=&V+}DgGhiWe<*j$EID6 z&NlIfU!_9G7~X@AaCb%#UL@woGGMIBL9&!YN}KXWAd>yH#O%YyRpqumdequC`shE3 zfszN~P-)5Z^z<y>Ou~MM>KKS`KcZ!ov&@)AkYSkk6cO$+1LJ&vY&>yDiA-E;As2ns zDPWS*kN5$5A?D`dm0P6`XO$PsRy*nIo68c6%J%Fo{!tM0+0czQ_eHjZp+7nP+cRG3 zj~2};ga-QXb|Lz|0ssNH_3O_w>tEX6YlsRreCBE1TEV<c4S=I&T7mC&eW}ze(a504 z^DC%vycSvaa_|~;|CHc0PDi47;{y`DIh#e<-~LyJfWTH(OR&PI*1UU}vZmBWi6g0N zmD7*f?qlOnqIR)bSAXioS(n)Kv4Qu60lRII&2_N3a=pxue)Zu#vam$#<PCy;A#HY_ zk!ato9%2b%y=G5DU|5h^YAQ&w;)Snyq70II0`oETL^?=uoievPYhZx9SQ6m0tN)>Z z!3TNFA;o$ct5Vnx@=b&j-`w{WD~H!cRvA?t*AiDdZ67C|IM*F?<t9cV+xnHmY*Rj- zNpJ(i)!JWn!`12rND)>;Iil<O30c=1W8((OD!ew{k{#FfHdA-4Q`z!akXl(@XaTnE z^OxWii)8d+f=^F?zrV@q=T27y67^8gabs&HTgya|F2qJ+Lr?hRDHzK>vi1GRaZ&cv z4O%A?JFhiiVZ|J4@oJK=XB09b1io@{qC~7HK=X%d45v1CW^s4Wr(yIh4{=c`-`6D% z5WHGgluh@z&Fsza3SNsyS6ze7nSSOE*!o7X^5~V#)9h5kdcLACER=&+7xporF&Hg$ za}qF1brsfU*YZ>)ZUWCK#YS9do+E)gsr9r$fOe*U@$%u~!E4ui_jY+rXX_N>SbSV> zZPbU(lxZ|~TYrt<KLS$n2dLbWb+7$wsX!=u@Kt9m{a@-9yJ<g7u#V+2C?>j4dDY50 ziX7I2a<Q4EsD8JNVTAvQwC3;5r$?MABIm9w>l%RA2w^UGXyS?%UYn{MtY+^hrMk<_ z;9loY11U{}Ylay=a2)nZ3)4!vUWCYIbv9LExp01S+Rh3)NBe3jd#|R9iS$VEL~BiW zMFm#BvE`h}4L#SKiIhtRuwO2pvdY|V8zEL?5v)#Zqy=@~n-7?X(G%)L9N2H_IWuid z5!cW|=ewMa|9HN;-8+#}U-*TGTW64JNn__G!CopdLfBE5<xx|$sQvp_BWjIX_)kaO zFVl#`I+WXh-7M+POYoEWp-xBL%qft}WsWjeqB}l<UtSjV`u*eAk&Zcut@iNl2k1k_ zhv(k2Jm(p<3lkE)I-}fZYAKvS5y4vWq#&|rNGW0*!I;ev@50tGr^%!006TKi3~)0d z16cz30)wrrkt>(7xG@8Za#nF*2>|Wo(!@|rG`AiMp}y&n7M(2$T<~Et%+ZMKiHVV{ zBgcT7F$V}UK@1D+ZI61QQ}LJf9f8jVJJzFqf)kbFYsV_O^`o#KqszJOj`yX==YI0? z>Y4t0ERJ7Mapx)*)oMo<jOQW4gt~NA^0*L)M?@A_@90^pvs92W$*Y7Nx)-P(r5XNX zA6xWS3exajaPyU$bad7bQYFH!oo|EW&QM1VZfD6b<Ws8Xp7A!3o2{T>=epps{QcE; z>Q~i2aA;5Gmfif-YN?^lA)T%vvpuYl>hPSer<k`<$`JfMakrJbf=~H+oeCR3A;T-C zH#4=#Citw_o5;QWIYU8VuTt+PSfD@q&Gk`}y|o+s(L&k8$s{g<1xl62JSnZc?s0bK z?9z*OkO392&#8Elo2nK9efG6%+eIIOp^`<V9NC3pR<}>*Ex+oloH-3VrSMyZ%&+sd zSHJ##GdH_RpGL-1zTg_jK9Oz3BYQooX?A<p{}xZ`l?XyTznSD)z0a)D8p-$tlz7Bl z-nC0C#FxVZk+f~)Yxj{hQhEek-&T!Y#hFGYg(sK4Gi$gT53QDFERI*B8Kv|X^X1;g z7kpK^lUZ~v>rx_Wb-C-*y<ruGV|i5bj)7ILWcT}-BA)&{+r*6l=Cp|!0b1h30&~;& z1`DsiS~8O}S~lJ3t<6s-;hho@u`FGL$8OV)D4(pc(@=HC{e5cr3wPj*Gv>LeqoK`| z)>5#n#<+z=!gUU5P?X6`l~1pR2z?S2--eF1kr)Zj9|>@=|MfbY_1wv0wG&O~(1>;U zg2>M91<BWky+?M!Q*^g7D5vkj{CU)COBAv=KZb37q;tZm83~E9?G-&d$5O$nsqvI| zx^$!R5n33=-`F9lNeqN&BcbN!J+Z2hFxNY<<x{NI#J_n@EN-HB+0AQCnfzX&tPCKX z1-Hs3E(Ytn^1HeFi{y3&Z{Kx@1r7p*@74Wr{C?<=U_i8hgTTG9W902t?H&+0Ywrsz zKsrXrrW1GdlM;#4<>_tFeS9^rEXB^zn2Cux%1U7%DqSf?Fc9vr5!&8N`$`DQn7UVM zH`z>!wqR1N6!&5Qkoc4sUmqW5HNN`0p&4<W;e0NIAPNLl5EatDGJF)$=@vn1h`6X* z9j9vJ*d~gwDH97IuF(iqz+R?sVoMx4hr5Q3>9Qc1IAA@9+WG_l_A<8%rza$JO$SYh z)#rEZ?u`XzfF}yWLP*K1$70CE-?DfFP_Z{s1J2@@VyLWY)~Twic_)UNnL{#8QGLgH z^i^bfn=#SRDdX3+|2m#$@%u6I!*l{^8rxb4q{*i}BqYi#7<M0ob+s9={sI(aufm`p z0I`;~*|*x$ctogVL^D@tfe3pGb^vQOM$%}MU?lgEIv_ctGQ#x=wSFi1gH9y?xV=~S z@6;hE@o7^me#dqayOUw98hteaT2`7Mw~ugh1h6ACnZ@1Md$rY;{KOR4JHYYzM%2n{ z*AFXE&r28yFP;C|*r6sUe|Ie91h=B-jf{b0tq1LEZsvOd=$^KLG?Y`h6^S_pl!ge$ z56sdrbwxl~(8}9cCbd6T?g5<A<o0<wU};?bs>Cu1iUE8I&QUXAxMB`@Y+{7)L2P74 zVQXGD{ba?S7w~G}T%1V;hLI>xn;=GoT!)4XX#Z5Qfehb4g^TplgoFZM!iIHh8|g-H zH74^goy}}(p~tF+C{hXm=&Tb)2F+C`DEz|4BHG{3pVyE?Vr=V^c&qa;O*AeeJas|8 z$RU>F>*pzt>2=XQF`tlFn)92EyaqjM?>Uhpttr^-KGU(p^U$#0fU2qfxVF&UNQ(e_ zoN>p9lH%LjWV;A&7?vl^ki?S-W@r9<bna{7cEKskOQO~*0KDR9FHKk&Mgy6S-BNs0 zsq3$>G5UNO+go}ob1E1e^s0J-c=sjE76vZ$l@cq4$>mLetmq-S4S+}ofi$frG|*=b zBA)ypUl4~(uKb`-KdP=`UsW|?=?PxdpB66)C6y_zMICbFbxBbPt3s+K@Jltak?&Uu z`>rK=n_^C<E<|T|bkD$=Gm1HPRwNQuaX@-KQwG_K*#em{9j^$L`U$Fe`V#dgPakXA za+lW5fDMndX$`{@krDF4%~}yO>?LHcG!l#eFAOQD0lzFK>^$2u+&pwp#sl5TJ^;Zo z-~Ta46E2h{BvgW9HCOdcDOqb(^O#lhm@v4Jb(^@pLg;x)lIFnt_6tk+mTrjBy0hDO zekl7;;K?ChLgNH=13;#;yIej^LVFO)hFxv_;thp%H{q6#Y{XzvM|!tj_^F_~;ZKQw zsn=GIH=Iu`n*%%kgMu5HE-k)y!Ny9h3cwW*0t6r=R*U8VitoUVI4B|mWmQUIPAk(e z1i-35)u7<RGsk>;G%7QBI#IBD#W!d(`N$K4*44Vyn2pX>`XY4(u}2l|R;2Uz7#Vjs zPsXq=f=u5I2qjMAcG#@lY2B{!!W1nnHf(fi%NV%3k5!89C|#ARXv#zp-|YSsoIWmu zm|gcDcCQ$)3e-v`^$t`3`LvzbtjURtES9+SV?(+Mi1tsE7369-7^POn6}K*iXVR}_ zC2~#ICS1)Kl92fE6V=zkE^IBVELFyrr8cm=k=NcF8dR%~RF#!>z0D^naJt_~t@ay) z*y}DKkM_+E78y6*qMf^FHwbzmHyu>=DatfQ3%5WW1zK1<6a=H1*Z3Ix4UBOPUv?pJ zH~<Hs*TY7bsd7=}2uqy>9z8*&kz%kA398bgXx>aL=hc^TS?KKcgzZuz-O3F0dAP>T z3*G8C+HEUrdH*^4(7S%ku@dm+Kx&Ka_{ZMcXvXszVMPjIzx!txdGv0_=NhFqDbFGw zZTSwn>$l|j?{qBQ<-Pkk{jcG=z~H0z${?DW+by^$*zg@pRl@qcM_}rEK;?b=4T$_| z0vstt@|OiPSBxx-OMO@=j0n<4BJCXlH%Lh8a2APDsnJ}UjX<WZFR-5ojkCP!av5M+ zzOpvewc#%=l6MR8w#WKr6)>Kjc#(7^ifeO22x}PH^;3`uQ0!3MR+r1t(5SubAtAZ5 zOs32R8-39G?k3;a2Skmq+3hpWBAT=6C?TBx-_1vGy6%)<{~?O_ZKz;W;}{hdtGCmy zDX9L<YJz2mhr>K2JP9k2)rS)j6r9jg%I<d1DGwQ^#sPS$^%*7lGnO_s0wfaJB3qRT z(DCd0p6-;?+WKsjwH0}aMKWhmbxX8xC@AYS5w@hxLGlHYF(GBgZfm0O!zu3Ns5gc~ zjx+Rq4VodFn{N9<{K7?mW1@I}r19b*3O`8TT)NiiBkj17aS0eag)Oj3<<8zE*DMZ! z)P)!sB(Dmw$|X{~6n?*TPEh;e!b{4Zl%bF;CdF-s+w8|EE}YnR5{~T`V7x!G=QQCn z?24VXLgB>Hi__~+D;YhC>#K#i*!v`HpKc+QQ(*V|yDiB=fm5R+qM#DWnvqGCq}}<k zOEnpv_S4zgIy+J!b|6oR=S!jaFNr@`0_ck&tk6$!9ku3BnWIm0pIzj!kOCT%Hthhq z7MJl;|4}{Fmgpm%XNF*v-r+)JXs>O#B8U-vebL@dP~}~IfGf!Q-CbPwr~FA}eGQE3 zGLS?kWP?&a2xV3s!weAoM$Km^LY{E$*TPy5c@0X<kv0H-1K|qmuCsDgf{KI2If;tz zGdPUn+HzOmy-6iEw+aD3*P=`_^dj;0;PfMu)nz?Y1?A5Z;BCE3&;+XusF_UxMcm6P zE0GuZO(QtfQ-H&J$5;vQ&z!q$cAVn-L$W@kj_!GRzp5G!m2>KqIR23A_UeGL{~$Of zudtM$rdPaFEV9vzp8)8QS@PhZ!4?^yR@(aoY6WgwCQl5YT?QlUSug>}*`DZ>vLybT z?au|)q}L}3uXovp>OTTd8EZiatrN5y><WpX5LO4*<PKnH6?q9_7XeN4MkOuE2(Zw< z77?NNCiiV0rnX;cDb}JWN$vF<tey1A471_4@I#UT$#54p&?iO<pg_GrQN}NBe}sX^ zMH=MMAH*}EPTOzanf}33QQ`=8b-}E*2Os>ib)#CA_o2c2hxgGrQT%2>=@}V$!D92J z?FVNArrU#XLBxf$JyK_eRud#E4FGLTfUF-xz3zka8z5L3D+=5h$A0CnY2yz{iid|L zPEt}-UoFs<2^(v#0bLI@mB$flf82~S=XD{>=)W6=@}GPDbND|hePu67M0vphvq3f} z0s+10ci8LHf!7Akj^}s{md)zGGj{>M!h+$If38daVqe1L<;e0%w$}idNzDIXJmsJb z=P*F}moEm|MyMt|1O~oeb$9uy;MUMzfC`zj)<Y|ErQ();$@3=x!U6(NJ_59OHI4RC zEi<!J3tL-Z2rLFMjDf;f2(?IGl98UyCm{hQ(aQv&1a%i&p7#$H5>T@fPRONaXA2*I zA)#wY7iT~_!2|d?HG!*-!3+&~r?btuz5s{|pqu%*zBrohe2n7z6R1?&QgDWiHF^Jq zL*E5^2C&Sq`uOwWNV<=}&P`nKxeS89z7Vz>QfVH`g7%bxs6Wd!_%$arif4eXs$gjV z6wT~`=lMOM6O+-$1--ey)8{Y00b+L_F|TyJBY?+BXLoAxbH#S;?%D&&(tU6;ML@Dj z;jR*xRquCf-&cWxNczNsy$fRG(Pa8VVF2zg?Fg`ap^93F@SQ+J_S?S!Xrv2&RA;M6 z5atn@gJqm@Eu(j)AA0-0`srs_>hg>22*8F~wNHSDxv@rTO!4w($^sbd_x+(#-2G+@ zwn{KXZ6IOs18eKx_%}Ny>p!Rb`6E5}Pk;Y8qC)Wxg6q!_`j>w{{NLZ*7j{p(ko1Tg zCrFXz=+X6Ou7N>*eb_LIf`Y*Wr0xJmMelm%fk_O$`AzQ3Ooa}l=Z3O2H@^+;9tR;> zD|T11!I{P7;4et!`P~<6a@Pk>86$>l{?GLZ`t^9d;tgI48=D~OeeCEPR+Kp~zDZ<A z%0g_es2YsmPy6H5vKt!-u>2PQT709gyWCU?L@_VGA#dZ?FXQ{jw!y*uR2g*57yAPN z7v&Th>@d}K1C#@(FabiAbTCxGyA{|Urr+#9!symO>sZl=$Iy46J2}9wf_ZiAmCl)S zjo@5<)V2$xo&Bd7)j*^IX?Lm3CMhY>P7ZBP2%mr*rnvojpQf8=%YAU(Cp@&Moi7E| z@4+6YK{0}g^nGiyid9_pMG9$GHJU}BVU1{4gvz1(LZ!N$$W6l0@&Kq_7!43sVkgpx z2bcLDWbDzui}w|e3jyYZgZ1E{cg6PjtpepOXlx$<>A!s!bXzvx687DVRk76e-5)ZI zkp2YJj=%eFj)b>a|1d8eoJ7(2e~*nnpYH#W{{I4?{qM_MFnQJi%Dq4m0V;O?b5mTh z{~8_dS;V>o1O+31gtCKy1GT^VWZ?f87yBoSV<B%%RMhIb3az3X6g!97{+rVR<q-p8 zNzg90fAHT6NcZ#W2gg*J57LkSvK+t%FX8`)WJnu<_4IGD0O&h^!<->7;==Ysw5`y= zm85S#8AyU{#ekADT*hH@c?vQtlI?+SPR0-IG?xNRvQ-gqNIvQBEoNUx1EkkhT}@1q zE(+WHqa=ZrnqNWBd(=a;Gtkw2lD`c=LLrEqYYVifg~Q+}UoM>`+)VIxhCuAlDd3@R z81IvcBoE6V+YYKhR1E(A!bAUecm6}Kaq#OD(v)=(7l0AkoBcn;G->pm0`pGb4It+F zMfja(pSE{&JO?xzRtMjC=>P+%PQX3@t%?xKL`y12^g{{%zdwh#DOgxoAks_8gAHT< zll@&fXMci^{4-g=&XNrl391%Bz{j-T_BlKLLtX|t9DixwU4{O8=7N~B)&Ip4qh)Bw zDqgWM5&g%^0Av={gELSjhAt#3Bu3J6UpxR@;b-JTQNM?lx_YR81B7A!XQq1oZ<>wz zeHIY;k(F*k<*nWFtvP=1bq-b&1qFeTyP$*>>eDv7%VMGka!$`qs}2|)(2vW5pgH&h zHDmC^U~l`@IyE_Y8;m}v!dK`((){3r3!?ihAU{J6A%hON92|rGeVK3{GKT-vULJxB z(tn!9t0$kcBid72`iM_H-(+6c5>|>403jj3H-D7hUGp6!rZ&CZtZ}?SOHp4xP}{CW zobEH3DAu?UEILb0TVn<p_kZs(FER10P~aVWcBVCm#og+z?NpoZ3%XKJ>s5iryDQFw zubB4bo2qj{ym%rG%MrzEv96{AC7nl;5zCYIi1qLG{Pw?JKlMSf4~kpF65PK@vb!9M zZ~Ld?pgVt_jDg)^v1(Y~0((V-MSE0ZCf+ovq(mo_7S<!`FjfSFJ;d2?INZb_6`~0f z*1!dXJ>;ns($(T-2i6i@M>f!=*Hy3BZ3Tgv44~_QAQF7W1$H&DdNvt3fzPQU%N8Fl z=`_X;eAH@HF!8eGibz>g0u-7lQ5>I;A}hNsH--?@6!&j*4eJBo!j8_r$U9UO$p#R# zp4g1$hv*Fz6%`?fJ+vT9&Q0+>3cB1fgPZBbHtV1>@%VZxH#lD81NFNNH#Nd9lg^*x zFh&__F{jN;5r?q|Kt=lFXiH+6Or*NlAs-8g!5&pRF2hhZ6yYczZu?6OU)|Xf#MSH- z|HY}e!Xl0<lR>bnr5-ZRfG+xreC1A|35HVGap7A@TuJa)rZ?jn+H@R9P-r3Owb>r` zEN-7RYUB-TRCBY^R%vl@ujIj?IG~03d+O1Dy9PzVUVgPRf6UMX82@aP;^Z~4?lwb| z?E~#h7q6&I+AIcZQZZoZYt(cbfao!s*St6KQ4oRJ9bom}<47-x?eAe5;tHOh9OQhf zp{r}+c8=&#DS=Q-R%x9R&Mtpm{l>=QWH{z<iDp&u@xhJda+Ua-T3_YskC||rVt?MQ zLqDctQn+y!FoF56IlJGxr$OP9B!UHHKVW`Cg~b2b{uvVfwi6GQ`7=V6U5(#wtL7$8 zUPg>f@-ao}el?nV**&!e#L_V?m>|ZK7b%L?@}+Zd7|`MF99*)W_x>raOkK~Tf6h>P z?p@6V7HX_ooJ?r1H<Ef{YA0T^=`-(mw+&7$y~zfb|A92sJnwnxiX<?6Fa6sbyKo)G zLC)w$-(6Jdo7bn&R(+8Hu!Q%cd%w(Gr4E3-^YD3JoaT_EQzN;$KJk4>>Z@bIsiAbV zLxE-5&mc7tWjh=(<**tu4p=br<HCv!jv1|k;L)F%EWN=57B4&EJg}Leg+k@H?>w7g zjM3q10|_CaG#97VGBkPleSjFpholNzB^&Id`i;D)_9!bFRoGax#&4lSjE3{VFn0k% zW`Tr-nveAxjPuvx0@5vRoIfIk+6zN584mi5u72g=@AU`PoPeMBeCB-~E`6u3&yy<$ zR*QqA)O`if{MKuM!pc!kK4YVz=?FN6@pBE&GXzpRMVzJ;zS<OC-sxv?ci9_Fr1tNN z^(vJN%=0-$u==jPb<Rn_Ezp{82KLF_X8xVOr^ICsRvQ2T#10JB<Zg0BhX3m6cIlW= zy55XTA<y(R!W%L2&v{kr#drkg$S3AE8s15*($hEA8`Yuo9F>M#u5ApO59Izr%{^c0 zA3VHT<~UzQJ{ogPINXV)`33;kti5ImwZV^ACXyGWpY_^VnhT%Cm%)*5g0Bzh1~I)6 zEu7oI?tEV@KN=GEkMSl9Sj1L0nuLe`-mmsqc_3jSFfp9(<ff<ZTNb{UtPH`4C5HA< z*rwWKjc=G~Q*5NxcTT#j8NfJql*f`K%aAr}%P`ifP9eI>O5T_}L7G_jWnv*E=2c$r z_hRtwV$eqCtu@)BA@oJAvx}+T<TFK9v$@o{)&sefN1Zhko3fn`Bsc=^+`Ii9C%{0} zk;sv@JeESTU>I8N*Jtz%y+c}?s+q^3aG-K3OHj(w1JPHEtj8|2S?-~3TOP+i3LNBN z7Tc@!Uv@;$J+~xiS=%k<-KnDQ_)V&fWO^xz_ZK)`KHo1UY-wZ<xKkU`-?a36v*?%^ zl=FQ^wx-)wd$z@EbWzeE>j~led71VlQCbVwo|=hrH*Z|4UXF>B4Gt$*8PRfHzP-2? zf6JA`P)fFlt2XInmD=>w=u{MD!wry$QWx!%oFog-Lnh5PP=%IemJ0}L_96RqK;=YQ z@9MD+lUYGfk^TO9dQv0tpd0X`L1K-*G>o(4y1M^hVTuxW&Gm#9t6GP!+U+Z$8FOsC zRq`U;B;PrDluj{R`>P~Xrk)Wgfx)sV+hCB<eO;*&OE2yKh?$;60ykb!GYu@eSqiB; zG{;)$2X(fSVE4Y)!#dhT+FGoNCcfL}U16x|%YET9H8ph#<CKo+<;wQ%`r?0z&5-)+ z&T>vu`Z!w;P&l0jP~&N&yE@@*pKTllVd0g3nyNHK5f5m3U-4?WE(|vSPJ;0BRrY-a zp24TYr}}5zQaTMR(L_4ia+Bc4M78=vxE0>n2G{)^7Dl60^p5(DZwA5NI}j69;P``< z<ka$j=nNPDb!C+rxbI$+%w|{XJc)}d>FS+yiG4!#J)d@_OY+&8A;Fc?zFA*-W@$V_ zX#0H1na?yNi&^-|n@?|d^Lp(joyPA{zG@BesNeM}b?W&P<?UQD*q0I{roJ%eh)Imc zW*d@};?9BvaWUy7nrqL$F1RkdE@04X<fl_m^P8Xnm`O~8OBR+KRMT>+cWpVRW^4OR zWx0j8IWzn{T>HeEqXTy}E~wMuMm0IwVCXN@msLs+IuQ75F_HU)Ir}>c*9oZVfkJxM zU<u>h6f42MUVP}DlD_pK!!JEK1^U}+=5>+z&%IA+b8aUMzpIj_{#YwJ!pPW9ZL8bD zu#De(#I4SABRHALd3Da6ugBImw#$7)z}K2SUK{t!^~_NOtd%}3_iA-XU*h3+AY?{c zkhF99kQrU*7OA;+T*K*W8kga38F^79DaRB~+hX#u)si?-r>;z0OL?WMd`QW?@eOn( zUorX9N$r8JQh8bT=}bmDYWOQx)E91Z<$t*ncE}#(BnI2m{rEJMCTcth&werA-DpY% z-?ZRj$8d7x63JFau=-3OyOd)pnN?1&TEE2#S3g;WfV*Eh{f<-4H26lth4raX<=!j0 zpJeJ#9jj#fjkd5%;=S$;+FOHoBkX{oe)e3d+qJ_<*s_(I^LcU_#mb}gvB9k_&R0dt z*c}Hy-_)irwh3hLHLjj##9aWf1}USsjGas-vpF40Jg-aY&<l4%e9laG`)-V+lTa3Q zY+oPm<M;fSVajTg{C|8|;M$+DdQx9@%p0RKLamPCuJfm>unw@FS_;cw{fJsm^&0pQ zVDr%`!L_bNHlfOjp!v0qDyYM`UG@7<-|+m|&65lxiCrR3e{h-aysAiZ#t3ca30&O$ zg>ptd(lTsc^t~_Nd^ml)#fzS$KOmZQOaoSYn`BvjRb&X-&PGyH{fS_M%Db93RN>lV zkoT-B731(b_?RkH$=E!iFmhl9#}0c@!O<thxFG7WTxe0zV0LoStX-YqWKz-j^o{pB z>gikv-(2I~Cc+)bDih1`fU(sQ>(27k?<>V?6|6Q0o^;u6;pviTy=i`h8OfbDlPza_ zO_F3VqMxyCrbR5PM&@PI+Pk}1)y@@8JbD+Duim4m{ts2rE4EUevz!pu@fDY}zR2<t zj9$SZ%?J2~O3@MD&l;XL>8N(~E^zBoWyx^VjACBB$WRfq>|$BXU_G)_?X@;>Hb0=9 ziG8rZP#>42usu|)@-(8^B4r^m<9Ik{KBKTy$3|;FJ!!>X-TJqS<4w9_+Oi6iyxlfk ztjDnDbR}UY^+Oap=W<EGG>aB7o3FAx>ZRF43-7%fyV-#6swS2ewE3#g=(3okJStK% z%?-4>V|8Iga*fcq*?DS!_<h-;NxLOAqsLXUda<MH1-4(r{x{F6B~p26kXDpcH9?)? zKKDU7V}Du#<L+nbv=fR-?9s>tDujctd%l&v{s_r*=k|vT4hGp7P4`wZ(Y^5zQHsxg zp)hN>dFmvrOmCr)Hr^A6RM?qy>GPO!0EYF*ohXt=?~t<fYB;BOjgb9Yg5LC{47X@Z zCFh0Y;Zzp)CyWmZLYh7wN-78>p2aOGXg+mhpzMq-h;WYGD}_g9T}b6iaw?X$8j*g= zHs_YcVkOjG8sJ1dlP+uFu(4ey&4yY~s1LE6Wz-$F?psb-7AkR7=}{rx!2sn1p}aYl z61=;W<+ws~vZPqR38z6lMz1SMTkgQnS?w!1lUfykX)H2&d$^2Z^X#8s!T$4)WjQs} z<%a~zXkp4fWih0(HmZdrN5qHkt4(ACt;iyS{e6b|I&gtJ5BXkIS58hj6Em;Q%xv!j z;NA4n)5<VamefPVLN#vY9+`%PC27+()KyczHam_Yem5bl+>I-A&|R*;CTZiE8_yk% zEf#V?0xu_zH>ZWSC|vm|CghB6ii%OW$(7K{*>D2Foi^gbgYNrVUsR$n3ME&P0M(Sb zlR_C=+-N#e9mYYFmx^>cr7j^UB(H9`?S|A%4Aq^N>;K^%jVUJJWikR3(S58Ph_H$P z2AK7&P&JL%J{-bw?uzck!k0HwJ&_FO8RQ+prs*{a`L5AfR$V18d^r886#kd^!z-oE zItNWUg>8e;0jhPq8$vuxsYNm=y`>A=%`qdt5}2j>J_FOzduCozYg;HU5*w`ZgX?%? z!<Vbr2@Y()(8#rlQ<C$J23lAh)44J;n_rzYlr;y{8!8k7RK#8L;1ZLiz#7)-lEW%= zwmy;KM}F570fqw5s;xJHrwBA3?_al#abeH?_T;HGS8GyuxZ86Yieij4jt>?=_VMv; z&gpQw_oYErbBjB?AQ|x?V;pNZZlm@g?3Z6?hDIuv3&D9dC*cN+O``}CGVQ1ek@ezZ zd*N~T;em8@Gb?T#*tM<T!qY^(H<tdGR&+V$nYQGpbhiC`v|iZp9n5>w?R?FVc5yHQ zcYC_sZN4<!!s@QQ6<M*nlV%$Gc=x7mk$u}2K>H;3e&dPOqglPF`?T4oVl6W>b+?X_ zOTHb4%Kz|qg5y51D~PIdI;P@pth~*=#+RNnWU-X2XI=%6W5NWfiIzB9;jjl;R{(Q; zvbC`4l;bI<OyCHM3?t#P@%F{Yp?4s@8Uv47D}j`M2j)>`gKjfCTg>2Xy=?%R3w}ld z$DF>tb<P6f9H(L*HNCnUHSfS9`p!c5(u=kl+hdfXNE4fK)25PZE|MK&KkC>Y#Wf;Y zPx<KfB2ttr?~Kde!+E))%<Qrp@55sRVo@tgl1v4<C62~}ovr<M9lI++>_sG;B$USP zp5I$*PZ4r&VD~{3gBRm-aqIrQddyCH4rj+?xx9whP<lEdspahlQC94<Gf*oN+FN-u z)Vug_^7Ms$^FVVE3sDb$`P=cV4<Bv06DefTcEp`H4BYv?JG&cp$m7$&JHw~DYFNLl zv9S#=aV`dbN%H@3S(q}}D7=sA{q-o0-g1*dPwbCO5B7@!AI8Gl*L8~>T$_cCS1Z!h z*_M77dLBQ>j)nPQ<M4i<T=C+vera`WMOR`0&qhzViSo1F=MBCq{wLD?t6O&yqBi2@ zMb}D>N+*M?bvA<799dp2xwYONxBL9krJvb-P<r8EN*H%IVMDxWT%vIPWu%~`X`P7p zPS##~ecT=0!c1C&B-ez;>C`qYBncy;bmD~jcJ%Ge(*mdag1mkDu)C)Xa<A{Tb~<Kv znkO`LbqzePSfxK3agM4q_6a(;%pY;N(+x)KD>MCQouG61Dr=yR0`V7)b%hYSnZ1%x z&2S%a1xsxpsh12CNw}Zu!Ie6we$qef6gP`#mOV55q;$~B(7cpI3HDNK-9h&`?s4EN z!1CWzG*5fID?LYda-D^GcNcO>?X3A>436_BU}yWwM@7gPhI9PY$?W&thxax|=TE+L z*^h?@fb66y2+2b}=J1Tt_Su=A3y&n0-^?kq60FLR-Lwvx>`#<RZgA&YXjjyrl;L9U zyz+ng*<%LMHGIphkY~0+X>r$XM|6z_qGi}>w#E1a&(c4dR2tPFk{yByPS>iv<Rom> zc&#I6HtSQ)SSBo8WWS%lzAQ4r_E(<jH?>f>P2`gsB|_gDrS_t}&GRm?9(ws9^M85x zj7`R~{a`N7%-6rs%*2t;?hcWowh(0ZR64|A>bmuUFWx3malP9e1(XeNrB<qtuJNLK z*{v1E49p$kp#y6*{m(nZYk+dZVd9O>)9u#$aBUvO&L$%_F2^*HkY^JTj@!Ix$rU5j zh-^`_lG8Tj4f=6vzg8WYdoORa+81q%+-}M#c1^Z0Sdr{5fa!6!CFI}fTwNZdB2$;x zYuw{w^wVG(SYVEAj<i(7EEY@BXChv~er=n&xT3kePCih5+5!vXcAO<X8EMlqxz@gJ z*LgdbS;CY+@gxP~y~(T-=OzV@6TEfFAbX4Xrx_|&W%sTI!J6Ve6iUv4m6G|cLg5*! zDu(ps`pu2tzNRU4GatM@-rbFV&@=t9u<(g^e3zqH2LAr-<LfurlmtF}(WD;N+2KCT z``u}yJ^Z`aRav?-z=KO+w{K?b=kL~_R*c`C==^Mzb`vWW9h~B=Ut*cx0qb;txs$7% zqr}_Mk<@JG5vOjY&b@6CQu6az<?LqZdKA3dW#odE$6Ziep~MmIMnw!WKOQ~)mTBMS z|28~n>*=J|lkbX0&!~sU(K4Rg{EJU3Et>p5ioiha&6)p28ttuB!lw!=(HKxb+;E!< z*DrDW^^)SFLQIe0$V206RCe1N)#>dvikh@f8gFqOA49%r6<Xc_q)-Lf8*hLi?4#eF z!d9(Ml$!oZws+oof{M^=Qm!YhZaKiUcNAtypX(a1eJ0@7YV&s%xx4m7DIzzVu{}`{ zW}P`4pS~1#SNn@FBFkHD6wEY7u4Z?1&e1piWN4mVUzbwk{Sob$-P8K1Ik@g6mb5{R z`{FxSHNvfhQ_0?NcLzM}GX-3HpU-OP{#I}#Er}FxxUm*}iypkS=eEG$-niSN>LeBD zpXT~xXWNTCyc_4-KsaOZ+np!-MQ6C6x|nWY``#_Sy)-91Uc1??t)#V}LvK|_6-Lyu zgg>Vt(pGNxypp(gfu9ib6<fG_N@l)(OtE?Xh>09|Y%cG?40@+weTc4F<TPbX22>&j zY6h8bA#XRUMS-+!Fq%TmTnU-(YnVEOZ$-0tC$hEVzuz5J_6xxPSzDX7n~Yvn*pKFF zW>X)5s;%Kk;~UvB>`#yKbLwEIQMGLk1_POrW*-=?sh4SIFqHiCD(SjLx{G<mcFbwe zX|1g9y_K>;2s{x`Zm0}Gsjwp_0e7D*xQCZT7QFDy3`L<M6I{U?45{>ofM*^FTs7@y z2Jz0{mZIo1%;gd;@({I>+j23yg<RD`DME6g@Qrk)?@DOVXrVW0<4RvJDi(wH5aGN1 zud5!GP_~iqRS$XXo_9;(Ynx{t;95ITmV*xS<&2?()1*Q%)3^z7J);&<sJo}jP_C-o z6Jbs~VxlF<Jgz&-^NW>E<Inja(U{C~R(o+#;SzhzdioTMZxp_DLwWFhl%CC(7H$jX z(#`o}q#`l5>lZ~&8UhzC-?UWw>8PeKXNG5<nDgGmYFDaI1e5>X3+lm*84gE46gsZ+ zz;e5}d13yr&v=!UjiJN9T{H<%lXVS#vi9Dn%Dw96mi@c-X`Uc!^64Q+Za*B5yUphU zq^A@??AQ~_vc0=j0lhGlOW75>ZhdRulemjhR>hkd#qNp<FE*=nLS={WGpr<pqQeGk zUf@Hb*o7pNu*SCM+mbvb<XCZlIb_WEMSBPQ`02$np>=DXe_W1^)+?^0T@^O$hxnZ^ zCAQaqY7+7zCy3pmz3fptSUWisnJI*}aY*aI8zxPbrLGyJ*bf(o^^ex*TPg-vv_091 zAw}U^7)XU#F_96Omu7MIjF(;0T4X)hvUQ4Yx@7Nmv+gR7)l}5=B|p)cmAat7W#1`x zMF}Kjp`=ecNX)Wd1M)pKe^NeYDjZ@)xfyl>hpf2=R{D&hFiNsibBP@5NgH<yGlz5p zx`muZu`<#lFM}pHnv)O7_S!t{WDcG{=3oAGs(xd)Y^ES#r)v>Dl*d9I*N9fM=8o$& z8x1OJ^}L+E%@^N%xg@2yIj};Z7=75Qf84#a{MitH@9Mpo?@p%~N)Y;50WA1rwZ;np z?)b0;gCh@fp#TNoI1dt=yUpsgo#VL+<=j9gP}HiZR1fTQv0%C;=1`sU3=0?DKmV~Y z)F-wsV390u%5@|ATffTotBfB!Or`SM##z?NQ5{Bm5n5qsdW6-_VSP0s?MM7spWA1h z?`Qc+eZBZwd=Z}ZhPH(rFj7Wam;)62Y6o4&Uy2>4FvTJ{`nsyTISK{0V^+sI()lY( z2Hchr$k~yd;CJ3{{wHWUi}nQ+x+ZOwR_}VB>>xuV3)gCE$}iKF(-tZ>Kj>B>$@C9| z+81Ovb*~M&6UX(Zeg%eBr>|+c#7s(@g%vmA5YEWjiA%>?l6BIG_~LUiER%k`(D(8+ z-?2}X51i@3MXA%q<q&@48>~gf58?6UdG;pCS*kdaNBvj4&0OAJ-4UmP7d;9_B}0`6 zc#;@3%67fG#xr&!3lWo+b7{r()4a2Ov^I>x>D7-hceav&OiRnzzT-sEwQq6|9x}@1 zx)roZB?v@BPg>y{{_?)xTaaF1PmU~9y`9?Q*@9%MEtiaz$6Mq?@TtreT|9A=f9`sa z$t_dXhYsCgBbR8$#ZSL-fT(sr#&Dk+Jp^>sH>1vJr8@)T<G7XOCuJ>B5(t%01)a~B z=e~qv<~wfY13);8{{!jD@2{?8XT<tsPJSF(YRKgNEjS`RKtHldJXm74<K@S-35TWT zyHwf*HoZpGyjXG6UT)dy+AGzaECmJ!hYEFRo}d08%neX%%9i>W#JLiVr(qTq4A<K> z*%hOV2;$qzi%k~9K^FWW@$8+g%2I;K*!@a*_sb7YSq((B@L6~%v9Hb)tE4@tckJ6T z7)ektY4iQME4XM!XmfpPH{TC5kLNB;ixZhj8w8bjz4|_$k8QIQL{*!#qWNK3+no(9 zv_*cRZ#K)Sj6_%W&@Ow$h=ds_Yxt(S;#NXjC!4{{oJ;T}M&9M**69>xl62zW+U|;` z?nRuQzvv@)+z7%%lPG_$z{#LtMvw68>E_<Kc?PdibEe((w|z<}@~Dv(D)f+Q-O243 zxf8)KEe3u#UbLcDk_PmsK2xDB%{jU>9m5poJW(w!yw_$&?=XM2rmlkCeJae8m13Hc z=l3^wXzEk}T?Ht{WN#0mvUYDlRMzJN_eR16Dh*An&iPE`kt>W3AdN4A1OMEaw=%pS zO=X>}lhZiJk~T^JY?8Y63R{e)<2m*6)*oef-ww_=myDp-2oZJY_cH)mJ*Nu9%RqVo zKcVS*HRAEvu&;MiZ%?EKTPyJhS87(O!KA6K!FFtH;Y_MqBz2e|DY)xb$qSQbaW*(9 zuc=|~d*hWpw+fSeBjhazs}e$!Ul4gSk(%yVPO-|R8$l9m=!WO<<|)=ljOExN`4Nn< z>Z@uyCo$EpstHBOPf&ON>f;={#6H3|IViSzsleuT-kf}aUh8_hNV`ZMur2dfizQ)^ z>faP9ZwFpO>*A8W3z%M(dyhbq)21zKztp{-`_#lxvof?M`ViKmE-U<NrR(Y4k3D`| zWgDZ)^?(XX^cED$4er)Fp}>Mz57O9UCiVATI?|S3A$D$luTNt|EUq9aCv7pup*GhD zy{X_Z>Ii7kC!_{N#fN*ABNO>jT0SuvxUluax+_oj2!`v2sUZ7pw1rcgB(__w6%NKp z?rPMn;WF+{9y<ivP?~9^kLbw33-yb#Q-6a?0QDuQ`u+>d{ydMMo`l-Mevstb+og|M zD;BG>k(DhB&GysqciC?zWS%8btD($I+diHUIzCp#MP0>dcaG6lUhn&jQfmpV*7;r5 ztoK&!MZjCFL~&oq8>BhLIh-`yfMICxkq+i2%<Cv~eN(eCQhmi*h&J2^D`8_>BY<?0 z*?jq7p$__5-&n^ORQ$uOK?fr3tdmw}j+3S0k79ZKrCTlYCxd>VQ=iusF|C>v$eKU1 ziivVzD?(9Ueh6b#H+p<krot_ftCi<*X1j0H!;dn6P}Jve-K)ob#)7AJr*>gN3dv+9 zCzREAb&W%I*``|a*O=xw8hcrove*ZBQl^S_p%x2&fXwsC@;7yccfR!*)qgF(-C2}$ zFy>=ihh1rA?e0UC9jorvAlQxmq#f{YUMX`nbU!*9=N$mgVv#fh{j%jO0^@7(YD(6a z)TDpC>?O>{u-%@EX-)eY&FW3#b$O#<+7sicMxkvytYB@jMQB{rdfMTpkq)S{+Z{#_ ziHLNk;(`wBKn_;1V72?8F95zO6)Fp-xIjsNMU}$_%3HfrJ>PbYMA00?Zb#l|&`%uK z!pLLAw0|jn%sSr^QN*DUVNIZ}EeZ+nei$Sp%X=m1MYmbnaYkHoW!uKmVJ}r<PTyzw zhKW0k^xkbCXsdh>wC%9*^9C_nISz#_9R)ckx!Xr=ftgI9HlY^S*=?#`jbbP%_gddQ ze38}W<|&3U<n~W<)lq4wt$KH2A2C(^RVGJdc<^r-J?xG2QJvpPxe4EW59{P_=Eu-X zO}RbLKW5FYz4*0T(+JV;=6)w=s;XYD<IWEXOUI%{Tz}Yt>k6QfQ79p&j*9NQS#G(r zN9KSN+-QrOHsc-zu^OBz6mp&?ri~{mDCj@(luX0Ktv>G7?0#BVc~=Tx?f}L^11h-z z{1Hv~`EpCFK^a^g0upX7273uWT1EqG;3PlVeLMMNah&9;hR$2L&gSJ#oMS%h7pl0W zhV+{8kZLkUf&MT1dyI@7OgE`MYlz6_!eXNZXHIW*(#CHjgt{{}^535dq_ZVtqmHu& z?1@PgjkSw3(Q7z_6*<6o-@Rj{2TR@P>SoPj(gUZP8%G-x&ExJ!<We=sUmenHQBM`3 zbt!A18mC@t$@n-AlX^Z;*K;$Q%aqNg7(~<Z41%Z9yXtq#@uMQ2UFNwi#JsC3+L~2K z{$3fx4XdA7#X9lL;6;9yB@=5=mqqSH_xAD}?)!mJO8_hnzG+!me4*tM#wzmw&j!ZK zX;zPgSEmMxjrwcfF)H@wD>2UE9j87*iZI}XeOovn?(~b1xvyGzmFljQWhHYZGY`{g zx3SS&MjchB4266f+%XA|u=H@;rK3CMMWCiI#T=iI;X*wfPYT}jk<OdD)EEBl^DQsa z3R5w|)iw3#vKQ2}$4)tYf7tyrteZYXtyR9V<!+x2*KN$#6q(0hx#7N967UkPESCOq z6cUBw-?h0-3V%0J7vD@6^o%@gPE<?4&BacB?W}q1YiNcRR;?{_N{0vYE{7?<o!8?s zN;#uj;923wsU7%@w!~tm#E&|i^MXf@K})3b>;wHL=(wz`8S6=<`NM(S$Q^R87HQO9 zcqoUYxpMIXw|S~ieoz4;inTc2bRCBZ!M3(stH(tKit=hD-->h2`?0cieKq~sBFcl0 z$2@b74GQBOkg$C61;se|^LxvwQoY%s?8-v+DwEAwATL4-UUc=oAQOe39#FnC)xnf} z;<%Z3MYW(H5EVbB_TbxfR^HjqExEis%d$vuutRpGU|*Q=1)N|${ZP43%<Po5A#q-- zzjIVxmSo5ftG3>ZyV`Ys)MTfmlS<2!j_UQYxhLG`lu9}}b%+f%Q@S?jdRPbXFjByq z3srnO`Owpran%hTrMZ|Aa+=EL^6Bj{y6@9dyKSgWL0-opL9F4{+GnfDxycWs>GmI) zxl6)TDNB=|obTP~16wasJYp-$+U$}Rbu5d*`!FaaO#D<;QNZK$+XZL!{HaI|i?viQ z^?F#1*^FQQ5UU~;eKqkX5O^lLj)8;?2q1etPjt?U^hRP&EkGgPplobGSEe4WQS$~* zO4vS(>EvgS#wGWf#U=0D1Zr3slS&8NE`Y!J*{MM%B$AK~Fl5jw_x%BgUvppBk6)`h zrBrH2bA3@HHr+Umn(ZqVM22J)cpPRQ&!5LuB_5K-s-X|bQlCpdXK?rXr(0Bh{Ux~A z&jvR9smf?YnzeL?J^=p#0c_?IdZ5hWmw^k)thu{h;^_IC5oXyR2Df-?O0KjNG~k?s zd$Zxj=$4iD)Z$!U(%;}w)E&h{ymLX{SG02X8gQ!eOdcml=x;qqw@;2IWE-F)c``R! z18~kUlck)EjHl5o3O7Ti5Kp>IpcQk4>s#tq17+oG*U`Yxdl|D`U5)kV7NTlI#@z^7 z#x}$&GsZg|<;5Y`AYJQxEw({gjlya<O@JN3#QfehMgj<7N<zZd7Jz1Z3D*GdA|M~P zN=aE<={JAJcP^i2*9(v!Nu#zmcz5<VcDxGj)|@Z3w6!gy1#sGZ>8>lh?Wrq;yc$$D zx*c-Q@QQzdE4w|<xJ*2B^-s{8x|sePZ~BUi!smCHt6G`<>2RH5ow-fe9Gl|JipPdp zFvSs!wb*J8mmc*UC|8?3?byq)fSZZTT6J}^n+dS)-eA?%`58pNlaOwp`^q|v`xe)? zBF}mocSXG!ZQt|ck*d7DwMxH|?f8j_CIYEwDE7iFnz=|;S0MK6NLnd}MeB6;tq(UC z5cCK?g?ba$a9W03>pNL3bb&bypRq?r8&uz_B;Xe=#KyXmCGGGomjWem-9oGHPn(p8 zTl;#0SWky~{Ai@agdJB=s%CCUtVSVF<lV0~GQd0YxHL;m0to(Cz3&VVz<WV6P}^B` z9s&Wbll39Zx+VxuN%!V}z4K3)^Gu<aLD<Q@7^xwf|Hs>Vhc&fz{i3mL-C$RWR23DZ zib$`bfQo{M^rj#n(xgb}V8KEaX-Y=~q)R6R5-b#Hp@$wJgwR5S009Ez%oT9I@A>XI z?>+as&vVv4y22`R%{lt`jX8u=SRQ?ShWJW3=A7KJA{S(5%2{VQ5C%AIe(-YOoV#G@ zyF#OqxqZpBwD`2tUmp@R6W!2?*6#yu>ek%WM(9jVStR35nC3!&I+sq*J7+Hik?wwQ zI8zqYTJL<oZ!_IPqT5vFrq%BsZ{QC1gm8fad3QhGB->#Z;9p9`<l*H5#DT2lIOWzg z??IF&Y5L@^iDw>TIStukCgN$vd*}{N>U$4(m}c)x&grXJChu(}AH!+v(J14VuGdZy zoTXeN|4m(epFf?-J4vZXBMDww1ySA-@}49%Rrpr7=X0kr7)`oBZ$73U=h@jZ17`K! zidrxl(T#0;TT17^&by#-@Bi&)#b?w1?>8$}GqPg9Xc7CRwp!t(DnxJ9fr353&-%w? zLjmR(@_PGgRWYWI!kZm}ZflDtO8gq!>}L7^Tx28XaN|T)`2j=yo^(?^#FH6)+XJ!3 zIh0(#UPh7+6^89cCKl5`8!QYq-{(x9O?R;|WL+KnJf8J9^s}YesZkD%2D0&W!A0V$ zqfxXjtHS5ZJ(V-Mf&<Cne1m+vatD(dJ~KVb;h7LjUmZP*zVSZyj=reto(ox8x5A?w z^hSr+yX|?5=RND^O>%NVnI0i17S~Sdr*O{b(&k1ibVg+ccgncWu5tZbH20(YkYCOn z>FT;eBxgJ?d>>FreA*bCXxy-Rx7kKnW>PhRsWhk}ok|&_8T7Lb>I`9)5Vs#nqAisp zMy_#F^Arl61|(iF%Rg583qZG}Je2@i9)hucWBJoq5e-5tpfNFk+i2eGgAWpKHUeWK z_3m+$j!@l$o_wAX&?KGqxTo>%iKZZ(#S*i$lkambpTC{;E5`U73O=2e@09sCJQ-;n zCu;V<<mAg+iG7C^41V2w%7c!<ciUAAz(SrGG0ECla64YXDKD?s(BdJRRH{$X@zKSh zC!6Wvse1?@J$iG!K$cWF>~EnZlAB_e^ixIKRaS3tfvWM9%^*yxd1$8W<swxiAs_UY zw$iZ{*5gp{UG#kXuOP5q-tnEmOy}0*ls2`aLR3}cA^^0cgs5Sqg4O>jpzO|8ke4ZV z-@C?~LS5!)DaASh3};c<S<yXZa;$qUNN##4mGp#jYtT?YG}25@2}^4oZ*tXJb$0}% z^kq^{tlDe%k2iK2wtGwdHFNRqW1-ky_(TL6Y~Hk_6C+y_aQviVqG!N}qEBYNF*a%i z;X@WO&GyLuXvJmyZnWfgQO;NzC%<N_=>Bj$s~g0dDc7NZvwVT6wF2CKam47ccEI4w z**9*58y&85iaFNZC?m3CjsM!>IndR*_TKQ1yFu4UAq5HMoGh&Rx%bM5OUj%dl_J)y z7L_BqGfmK4wl=>{d{_x>sw+$A*~}iEtt$IVzyzvw@mq6;klnQE>}+3?MzHkzrP-pH z_fCC$_CyuxQki<`QbWe7=$_wO9U9?SGx&!?N-MQWXj1tgD5rwm7G%o28Bq4~WvwYe z9Bmvm@Vo$+3wg#x!t;j+WxHYxQnhxl1U(iF67;n*(pxr;$ys2XFm`%$qO3^o{f|Q> z4-v<*gKsls3*5x-eS3W3&fA|P<TDY`AxjV(KHGDySk@KI(m9AE*NvB7p^?J-vkr$d z&m#O>3lEV^q?j(W0i7me8&^Zg_cO0pjx2t}y-KS%IQZMFdwz0A*sQ9ZNB4csi}NZO zltSxpu3MsT#Op`TuGsZ7SzlI@414;-Uip?OZBsncCDlGpCHDabKL^g64)%~h9(Wd% zn};Ym44)J|I7+RNS8pI8@n{3>>iM=ntFX1a+ZS}2vlo&$`9>W9988krcAcDHCSP%| zuW)?LV34{9Ae&+PM}EeN%(Q|ma>>)mn|i!u<N<z?y?xHK?F##n*KsBF4fEzoYPr3H zeeSxNuew^tf9*}25ztNG^#BYvrgLqp3#f`6=%qBEF$6`Q!R!kN{VUj<ZNud5K6nFO zBlSKr_?FI7qySUR`OjmqOC})`1DQGI*<(gB*FHY}iMac|t?&X{X0Tt5V8+sGV`;4> ziRiI^kISCa7;GXg^;cSnxu<9df?ZQP-xt^wxZ%ZgMZrGA5;>KuPeM1}w@7NOFA;o3 z)PpSx5Xy6u)jWZT^-9Na8c&b-=guTS3WDi>*^?O5sWsni73EX~g;+2?M%7>7#j60` zo~r*dXXtaVSHCg0(EIfo<YtEwX2}>_1Uh0IKf2ZE^}<5fR<%aztbjXzSLR4O^&(%< z0Y(AEUFJPg$3Hg*K|4*Tcx_iVsGP{jzg7NG@qg8T80^#*wcSNE)pr|uk-C3_l4HF* z1MnMt<t0Q+#a{Hx+?;rpR*Lu`#p+%KkSxvy@JyiB?`+|C)dO#i$}_%&V7=W4_C8dm z$LL`Is%0SSn1h4ERg|(eQ<&u!fb*jgOx0(|v$L0=qSA_CkAw{-n1C}=4{6xv<cz%y z#z{P1+wnqIrX;8?0*+lHRDc3S;z8X_%i>hm-%vGE^a%!B|NAV|hZ4wECKQI))~os) z9{+;|^4!Ovc8m_ZpvDBgy1?Q#y=Y`IItOrt%eKe(rQX}=CbX8?RmAs~JQVx034qFw zrHET6=xT-T?dgGDJgxyRT{Be?Es!?_d%Nl^fl6TO_cAWejH<3i1pTEl1^SY<>I)XU z^b%{G-vC~DDa$)sQ1<~!>YzL`Y@I$k9Pl^^?kBcV$bJ6J`jt`x6XE9vV*Yw_H0#(| z>D$#3;=;=1CD)zijvK%8AfN5mT+>Lq<!bBA`Lfns?o9L-_+PJ7SDrsexG>Dx_3oh2 zyAh|Iag`F-K(}KOVXf+OeeSbA4w`*6-aK{m{CqNoPq_i(<$FeR*-bdf&AjkL7|(TL zgIhA*C9z@zN8Sq?&|7zMr_K%c&-e8iTA#o*-Hn`|pD*>JEzk0W?|>P;9~~W)K(VPI z1X_{`Z8DID`5d8N8>>*1B(nRhIhG8h0LR201?bZr+RoED8>_}ZT>y8sU<@nSl}06N z!(hTUXHOo*TQ|2=_(~g0i6+}WetcmE?8}AWSRb`1QG<~r#(ECi%3UaE3Wlrgl!XP> zy(uczizQD84eLn)qc)ZEXASg*!1i}zfQUm3K!|VZAS{s5F*s<VY+m+A52{5C^ytm0 z&Mh7@3kV3<Fg&>z_Sgx$GRu80>~@V)bB(&XdNJn6wSxx^oQ0nC&^Esbsh^m4o#lOe z{0)pE)zrgxl@ON&JmB51T7<00(>W8|n*MhxBuO&lVS%huy+igkHs_WW7a1j1le_hZ zW--uCw4Zx@)v6Y#34UU9R3{@S0T`|9IQUC!5uypsoCb!#INCXcj=^Bf`p1bMRVoZ` zK#UF@`RSkr;Y*81Ib*PlpW2Os!CxKkr4i)8yBSAduP(rKJv?SBgggJIaL<&TaoK}{ zg`__(t_|vn5_BK?@ut+WF1Q<ujRT(=R(-DmP96b;FG-@v*^7*?naLcocXho;8YmNz z=i%hk+}PM)6c-KXfpuWvx4*NezM-Kc;)`8Z=A}!QqM;$A4zsuwc64;irbqb^<#9iQ zU#5YMc^hkmbnmVC?}6rDr^IW;PziE$13mOxs+P&P6ge{<)t7zux7OL{U+WC6k&7H+ zu-ZP6^Ni>^jC8)<@sj-3?cM)cPtTqlJ;wOtE{o4hzZ|lE@Zb!M>(uz<<imf~-oEZ< z{N1OZ7_t;pOLcd3MTw%<7q+Hgd+-&grThcTx8-|UT5c}OnnQ(3uu`P{_|`f?XW&TX zyestxcv9O()l-P(wh!TF|7Rzv+R&)L6j)D#a;GUmoP}0bSC<_ipPJ<X`@aJT_JpRG z?5r$)GvE2Tsdt*9*J171`iY@*lk98j7_Haz*U<ecS(DqVG*TlkBpn(Sma)0Hxf`ZQ zh%qp~4%keSWmAqhkXhh{XRdz!o!VXZu9pdtbMic;m6VeGUffEM%1BE~W1!7Q1%s}I zWM88E)Q>mE`dyrzo!6Eu=u6;Pr#5{+X<`iQVSo;5W@bhc_T{*73>;*K`q6-oWDcDM zVihd?6O*}^S8q@0@7Ps)=k+&0ZYsprbc_rzjCTfRv~Mlc{HNWn;9{6N6M_w9YAA+Y zcl|C_51ZVtOkc&692tYPpBEI`US=`IG6yoNO&v^i<CgajPEzo?xT7{`d(B%OID0^L z6pWMy1|At1xi;V4etTyi^LY=X52*}$6qa=U27Df_2s6F5#mJwvl)z_rkOpG7od-(K zFHx!d(EShL!|#1sTUs8Z_;`CC1y^fC@JPNqocbM_c{@P<f5SvS{W}Hx^5v3s{j|v} z4wLI@iV^A(+Z+=+2Obj>wq1W6NC6Dpz8ZRbZ>p}q*4IEUsy+SRIa&O~1P0fl^iVtF z5Q7mka}rz&fByVg>O-A_7`DE?{tggmTSOj1G`ruGc`JrsCZyH-i77N9!j|`x*5ii{ z1$NZl38A;ttS>b8i-nG{Sr^tU8}GcgYK(BT)_+*JG8WQxHqP`N?4Imo`Ou6!ZdFUp zW$da)^`oAg#bjzVKbZK7;^L98N#9+&c7Yj*)+d;O4<7zQ0{70>EHnvC*(%eF<iL`9 z<hk$tvR;SlnTIcf%IISB_jH8=FwK0hR0=PId4atQQ;~n9H=@_?5ZZVBbH6xDIA|q; z(A|A^XR4pt-(pOj^N7MiF)--Jts-9Q7!r_y&1%QzqUT_88qwKS)H|?W-zG^UNAE2% z_^jt;U|olNA22w}4pxl)`orT#Rfwyrn1JN4)IER%9TC{Z#%mj%&}1;)Z?88}Hum6F zPVM;kgMk_!xJNaRW!*Sj3hb9Etsmnb8+K-QDkaab==8>z*rw(33Gi5_mKqxyn_>(o z{D<h`&6_v7RfwGI?AKtAPnDg9SZgm#Q^W%)4@y3Frn=>u*T|W9Ph~)XthP^{d)v8z z)Sui9kxF2O&F)hw#DwHzEr>c`urJ-v-`Le#cDkc{IUjHRXm|q%aQFSz`vcpu9xx09 zSgmU+c*&n5<bQwC9wm55LPCO;su-gr3|%jr?bK9rj-S74ZWvjDhq#|1Ex=z$Iq%`1 zZ&3zNZg+#4-hBf)ewOd*Nw5{dwqPgW(CD<+y;BHmm$q_}R-%lKvOmUWBno<Lsu>HV z0C4EtSh$|lBdMK@rslQK3I}=^v{Y4{qBdP=Twv5(mZn2g!$Qu?%1YqbqJgMC%e%E> zd*Cv5aBy&uvH!w=&Xyvmi!%PBz_tEg7O~9Ufc2`=ALg!nd+yvhAP1c9-8%^hjxYok z66**cRl(^Q{I1D^7nNY5rQ6JXFZ*u-tm^q~$a{19xdQ$hdMHrqcK>H1=pz}&<X@0F z-9EnT``@1c;@CP^N88g<J{3CpT!KW{@JbqI=j7z$ov_JsfI-i|CSk{p9s4`5P5QzG zD5XDn^CD{5WwKp!s(m!28*N@<^Mr$2r+WGvdj6|PiSk|q|0cOGqpVDM@0Y`1=yI1c z<VUuoLqymj^ukH5;b@;*T3VUINVsi-m6a9PghFfw>_xgpM3$pPz$L}yl?qNK1H{sd zMT*mUJNg#ZjB36oFL=G9fqWeZ6wq+B=M~xB+cCv!nYSy8j4Iq*stNEm;Na$#!+U5% zaBK6TR+rbX75?KDT~DwmX%YQd4}(pK-(VxMa#Kp%ZrGQho40Odm?1r^8}ug3gml#) z{f^ooIcL!qEH*W_WOh<25CmCZ$I?MZ0VS}RRU{0idOTmrx0y8cbT7TwZb}s0+Dk*d zQ}U%a+WC@E?x(Ky&nD6F=$}eFp1%w4$iJbqVm7+q&W_#*&^9G=qVEH$SE4d2d?r2+ zIh@$ks%@5gN8-jvABDX{$NXKK5P&kWA}bOR_|fHn74r>A|3#~mYcQbPE*3z`&kE6G z2_BF=Lm4v0cZ^@_64>3N%Ha64(i$!0y;U6YNvkuxNZa)7!f@Jmu?!CuBqphK@*2M2 zi4Q${RZ@hFwA|)G3PBQsA}Q>LeCce;QJZ+~icAyb1^KDmLw;#?Z^SkMCkmHAe9&4i zkbIZtDnpMWqkJM^sm(E(aGft}-TdT651EjpHr3FSFz%s+K|neR!>jW?od&#{s+tXL zqMxeEO<T*{?sYVpaD>3|KGdw4_@TKEw%|nDk#4?rH)^oPP~ck~%~OedOUc+%Nj#Yy zma3~)j%QdZ?VF(U5X6DQ=b4@mI)^tb*1Oz$V$jB7!$C6ze$M5)j9vCt_mLb1%QbX` zxRvg<hQsGM#N4mX_NjVQb8<UXw!Jb}pe#vo3UNjZ=k*y3{ax7qun+O0OxFD%cFh-F zojdI(>;BmfuD%=AW&yFWM|DV4lfim{ahV`P`0@ia1DR-~{t|w99uAHhFco-+*=+Df zcdWywhp)*2b0sO)=$jLRGdS8nxpO9`b9LH%+cHNBx{ddAHW`cF?B;iNmMfO3@0lp5 znl|Xl=UpYu%6Pa9n$z~x-RZT8Q11Ui8GM4RkrvwhQXtXKe`ePbC6t>sh8jd|KJ-vp z#s9#3yvvU1nN9rodZo__R}kRWKtd;|Yhc!;0p*VvizVe7A*(#h=9<FsK8F-m;`&|Y zP16;mu`}FxNevz8B|!+Abej;om6f}Ld4bO(8nx?InLW;Pu!|aT$3kb`?`NS|A{ip% zBn#7YfgMWoREuV(V_1oR>M{FJvwam#8#g?vH+WG70qg9FroLhG>+m6ed%+N*_phhu zCi59pT->@6p&?8eT}PExACvMy10}NM!VH7n8eL#Rm|}cUQPCF3{t6Z<xQ55$`(32R z&A)&DK6T64Th|*Hg2G?+1d6g*XO3yz*YS<db&m@fGl;HVxMMV1%}U6tbawJxZ@2}w zvtM3Ej8W7(gqa>&>l_uD`@%|$47q5y>1n@Qpk-1XAU5oW>6P%%4WTJ}Ob+%tHp{ON zl*N*<Mfiq06t{o^e^M~ffAY5`vmABIZ)||XjWa--RmWOJ&S#UdnjNtg)RD#2{Hz<^ zKAf7~csz~6+!+z5?`xs1Zcnx%A=<bkeAm3Mii@KC=6xf2lGFR`74$t1O)RX^c&-hW z`%yy4Y`8cdWgj4mPAK7tt5&@7N;Hmim@(?F8QcYWiUllH0y1tLx*dQy1j-jRmya|+ zGO_*dzxM7$ic+VirgW-y!8knbFq@QDyIrO>!#_0Fwb?gcKVCbxzM@D8zYumB3uK;{ z$li3musw(?Vp8GGt<Ba&U4t=Jaagp4D1s1FI`peb0v?`=Tb?YExB<UEE|NbeQeavY zGQ6~K+E#QVZnh@D_9nSw0MTFt*tx%NMD_j=*raHFBT{hiPX+TNml)3@VNVhFtG0+l zma_$dYDMInTv2T>EhXZg>&OkSyNA}kKA|XSNmVWO;tI(1Y{=3&JRzuBprB5}r2<7^ zQ7fjQ7eCBa-Y=cacC!Pm8)i<(w-U%|E^NrJPYP2+;_}Us{G2`5YT~Am-7F!|_Pu$@ z)QuK+UPwdL{1xd>a7Qm<cG7b1r;;~CC9C$K3zp_1A=ph`vW%i}x#6=AqEPJIm|r!% z9M_tx%q2LBVrv*IS2X6WGm)ZE`Wu?)H-EUb8i3t$WZ3?w!P*nyF&-~Wlq<i8C)s1A zVoMIjF!>UN>f@z2y*T%j1jUYje7lsbJ>pVN9W&+i{4F+v^O-GjHZZOZ*=bnP5Q3G! zQjRyf-}fbNx({ckC?aPNYovvw(T~4kBi6zqE3ip|BO6p(Epng%Pmf$24ZV_8)?hLu z3U{liN1%8i%`n~IvYAqqJ74o=#;q$%`V+=ENyHe`m`AyRXu~`O62yPQQuPEm|MW_9 z4^M9Zczq|?^WkD5RrB?n94OHUdo#i3Guq@p_+(ud-h#8qJ<H<@=lSUA<n!z6?W0|L z8Y4SO*vKC%%fT+3h}UMcU_QqHYVSl)l|3_GX!=Q6+(8eQ1HeuV)Gyf*xG>n`mqDQ{ zq)SCYCbNlgY+rLm^2w%QC~FzOCUM*ipZiT+d>v2;Lul0W!rJE^OPAk|(}q|pWD`WN z4q9yAM&mdNT})}d-6Y?GH7il29^ozs=@=$po8|vw{w3aT9q2h=LEgyPFgj0iILPgv zJ@3++r$stbAI}QE7n0P|D>`rgdU%${$9Gu<M-oxrK_nV+je1*;R#r+3b$pxr_4SEu z0552B7bE&&etzSCMF0?*%vHD|d=iq657y2DMJ%8rP@?+&#tbLQf4(k=w5dOG*2#T` z)vXV>7^^V(?m8#uR~B++2Zh7KMaTmntbmf>adt0V;(K7?*g8Eo7<mL{XbwH8^4DqK zje&BFF5n<^hVfQiUSWcL35Bc&V7z33+G7y0jRmp0qp<d*ChJ1dAi5&;NHRe@%|PT{ z-jwS*^5Hm>>EXF=YtonIZnNQSLiRMk9aqa<ra1R6w5?V4_vtv*1$QY6&4pKR<%y8; zJn3;tQN4#yn|V}|#<>s}EN#nU*tPfICq<v-2mF<cP)7G=?ipBM@9UV@R|iuL23dCh zn0!5^C%rmJ4)}-~lX}(%*<d0euw$^)4<Kp;SuemGj^tL}SnQgLw=cL3G+A)nJDuPr zjc6sxtg;O+g{9)-{^b@4x*=VzzFf58Bi46pof@7pa;l*EUa9F(<n7b)A!einH%=T9 zeV-p0c?TsY=emZ=b3sblYm*O37q_r}4G+DvNs|kHMq0G}3ud?pJa9ez>OmyjnN{iK z{Jcw@JS_F{3pYr&=@&OK8>k(r1;%DTLIHBFhRKj+H9Z2SOOqbhT*F%Ut)McoQr|kY zaefvXGd}dig3R=DmKo$%&ZwHrB}k9D50{Z|g|LiTuVQLT%G~-~Tjh@dx1-w;VF2PZ zd>Y|rV5vwo8*1(I03-SuFvD(91Lh@e!8t?rUtl$5m28gajc&3px>m978F@Cjb?s|C zEgGI9&qf$tD-dR>?Tub2vpHKNP$k){Le%BK47N;~Xt9yQxfS)rMy&Vz?DS&6oL@kX zEtxesx#mBfd`z)Fn(@{0tJ@$JeV+d>m>>uv*<0sao8L`2c@%c?>choR6DQ2hD#XsL z%72-S)nD$E#C&vEZlllR9`FMFZIGSxV|9YabKh3tNTxNCd4g9=>JT~EYXb3dQ0*}G zCJ*}ruTHE2n+w{qoh@yk(WK4P8p%-4VZ0lC{6}G1&gif(s|71uH-t;9=?P)~c~toR zFYoDu5XWY9P5DH{-zrv8Y#B(5^TrLi1lOnJqK0JC$EyWK`X@4wmOn2HEvD6XRPi~V z5KNkp=Hww}lW_9szA|x^>Qbn{Vsn=w2L)~OyFw{lvsNi>ziimPZGM`<ml}&-Y4Nz? z@J><vYrzWfj1zTA8(6-IW3~j>GhkHN4h<muXtQv#U{w@8)3)O8L5uJ^JxtHQl=`mE znhFUIepTK1!!-~=O<9&a(9?7uJ0cq#Q8!Ha*63BOTqNjCQ=a`C^qjv~57T<Xm)VHG z3{zPG79z|39f5zP&}PjJoUlk9t$Wr4YJ5L|;BAlQJ;X{sWHe@`8&x>iCHfv-?WP56 zWIDLmJTP3fVu4?_KgVQZV78RI)KT70{N_MK29RTIZRh=^us>dU9$<Jgp+X*kA!gJj zgkki-W`v&Cx)W+rrcIHD53O!1PhFRD7+h{$Lo`UXVHAtIJU2fIHHa4S@(PVVpO;H` zV@Br5Mcax=`qEY35Cy||!qtAb<S)!nK!UId6@U+uzI0RbjT&O!_S9So+djO@80N;T z96Pox+!I!-%CHg|kTPF~a`bYK!c+v}=1zYOzB7C*fL^v(f%81+M@Da0OzEanr{8~K z`$8S=zn*+U{Ypz#PXz7;^9em2^C>xV!8SthSA*9*#jVxbmiAYD)BwWB^Mky}g}-5s z)7(wwBrjgPqs+t2eG3LN%EUgXb{LnA)QxdV>>s30B_g!TzxRhpC(aw{<TUXLxu)A7 z2{AFvb~-OY%qBnnSnAfr$<J)0JH*9Q-^f6+Kk=T?@Xwm^Q(E8!{~(AT%!a7pg8|Y> zw`Ma9gwL^JHi{ZjZd%Xa9#44U+8yGrJ}loS9sHc_PBn&rWyi{!Te)0*y#7Ieexey( z5n{Fv&xiIKZy2md*{InAYdmluX%ns!!-${=6;z%E!7<xWE3RFz;|2mR6&{D^1=oE! ztDhvrtXB?P*|N^jD?PKGSYcC#GAri(`MK`t!J#xZjBib=y8I6-X`yA7?+u7=)iw(- zkD@ks12ShM?#inHxb0PdKB*g;na(ZLN^<h_l&PWCkZ+N_Fg`#Z1Xh7PNY;M)WwG9o z5(`vgWl^S2U$MBx$47~f2)koaGCVmZKDRMz){h|y&atTV%3x*d!tLDN)YTF?%{LjQ z^IEqVyu}Nb;8Z=DM|G6iEE6`j@X~P=y;!yIZB!sDEIQb&h&E`NZWSUwPWTI(v=OhL zZImX;R_NlpzGyibw0<IlD$nYHDT_J<Q;~>t0RbIjm*R6fRQA4q|Nfe=n0QW_A?V%h ztRUlT$XZtr5I`X<74ltL8`R)&=ra=0_f=Zzrp{OdOh;8nAGmMXttP52GJe@`b|SX$ zWvJyn(Kh>yUsDZl^s>D?j}ezv^AB}9v)wm5URCb5(J|}0;A*h$fJzlJ$zJQ5B8KV} zwyGAfSG<jD>+Swl#ZEXf=2KKmaUh!e*3dU&2fIe*{gwvnd~P2v9Hp3`-*Gn>hE<!g z5j}M6dR#(jGBsl>aE@<vW3+b`NEuu{)h`2eNA_7m-_(U4OD!q6D;#_`hrFkKEeZCU zzD9wD?2eon-}Z?&k0E$Ge-(15ze^<f{Uq!95AEey&wWXi-ib%3-W1Jy`t&LAHdJii z&6h<qdRUe|N=^z5-*3IzX{q)y+E&&p+)Z?Bb6&=)Kwl}PF);YU{Rh60_v<#2hnFIj zh?s`d<Yt7}1j@QZej-VU`qg*@UJy}c%3^_6Eif2T5;~dO#9GKj>j_7m8@^e;SA~D_ zIt(Uqn=AkD=?n6MOg;O)eY_MKp%ixe?9OV-J}IS5d?vC~(r43j-V3CtaJP+{)qd#? z-4|)&W3&FUKHocQmQ?OQH?N{4=iAO^#)c`$T#`SX)-Tw$;x%6#&aE69pF(q6vNuqm zUU_9<+>T%)AH#}@jnp}gl^qWWtt%?tc3Mg*L?U|Gd2!g^-pzYT$7n}$oeFalu^+5n zYFk9?J3|>{!%z)Lvhg+bRzcKzqA6(d(^AJlY0QfMyx*Ii&h!je<qC@H-`thQ_YI(( z>|nySk%T18jsjtmyH%U(ZgBz+tYybyLaIzW&?4AsiKLL6HpRqdL`6a!s@_fH{!W;w z03*sQ5d6%fBUXf&xv+_EXY#=2Mrq>J9kXo_I0^E;E&*4+wK-a+>jh(uDVNE*`uTh> zgs_LVP<XOH+|YumvD0HSiq4^a#|R$zbayt^i{^M!;HVsVKcUuUS}oS^BaL}Kl3wN( z(I6N%S6ozMpO?A4t*A{u@LwLjVLK$rTFol=vBb13_^3h5$3A$EUR)M182j0~$Ea3U z9~H<{m3?q<pUOv@Sv)!MZ;ne$?#1+L9*`NvEgjma_(Zg|t;Sc;Dhsh*BQR$vT{*W@ zLMSfFEg`qgeWc#;i>tDXyvRwD!_YpE4cQ`|;9ogiBM&UJ>B^JxuiGB8vZY|z33b)g zY=s-jBQD;h4n@1!Fb2ZIAneiLh&JJsclrE<X3ph2xcu<@`=O8hDHtXFP21DS_NZd8 zdn$Ca`$2*yRz`o|h||U|P+OO0HfY!;cUSOtQX!&V`RZ4Pw6h3-2Y%C0@WlBV87yFi z<~1sh&+H(G7=48G&d!f5wJa&rT{L6IA(hSG)3zAU1KA(xYg;K$Nl5p%8Yex;3Wn1+ zK6J_d3KM<&q7l-X19%aqI3xnhJMIu<mn`9B=-R_QD<eOa5L5M>oy3YZRuNvG^`EjK zaKy?gu*i}aE<PA<s7?t?MUzWWpQQDq`MQg*T}PPoNc-<6UZyj5PM8LG@2$QfmdryR zxSc6q_I&tS+Fk0+Q6jKoMODH>lWRQL>dNY3=i1a}2MoCEXvhk%gSeRq`+crGKUY1z zgxIJ4$wRg@gu}smgdgNR%z9(iZ%M4K-lxCh<VSq8DDvHoyn>pQQE#0H##>Q_$2XVs z;;<U|tRc@|F%!xwTl*alHf~ji52Cw0!|(&dbSI+QWL?UqGPls?n(Vu&(3hlc)LAw| z?rWC?+WNjOh)LZO1x}=jW*!K*jLy{1OO=H(l3GEGO;V{Xz6aQChFxO^9g0c@2G05) zC`uT%iooy?ku`M!zt$B(_)H6hx!B+@<&D|76)NXu$@qb?+|Hm2N+|T`493~--es!1 zN3|RL!ETt0g|4oy9gr1v-rjQC@$TKQ_PNvPLKF8uS}I~IK7Ei&=I!!qe-yBRx<JZc zpwh)^C#(v~*^5Sv&ahdS_RTpH%*NC;@)Z;<48kRW_kpb8_MdW>{URe*UTu#XsiC!b zR(EvtM!a<_lgvdrYpBD14T0P!NVb98D5<ZAFG<Gbw7mb?R92TH)^t0HzLlDT60vzK z?7T;=f=+KLaG|)Ga%cL;Gj-3{h8N~#fkm0k*HKqYowDIb<{4aj<rTBg1QAv)Tp0n& z;g7OzmY>1fSZN?wgtX`mh<qk<lK~^~`k*{s-wh*j#oJ-F9Wd9rA$wonYJ3RXYu|}* zE?KalFj$hLt+yK}eBF0yN8lW*e2?0BL0+A;mJ<4VzZQd`;ER0{n02d-8TP>eEj;f( z4vdKf;*3Yy)m1T%vSy4UaNS=f&8n+&VBFlt7scM;J?wjvN4T9Hd?2aFT<l!YBnvQc zfF08B-kM_wmFWDNK*GD?Uu`;m$EKis?EPNY8697Q;;j!?MnHg>8akMNJ_dV`TgmMO zsk~4*zRhOi9zNWbkTl(<=2d|b(%kb!GGmzh@r*vJS_Pt@!?X$Sm2kqZdFtCxQF_C4 z9Ww8vkzApri%8SEx_2Y09#Qr|S6(GpL<t;VWex-GQtOQoO<154E8i1F8dJBFksk%q zpqHzE78VF127?EX*=m=Co1W-Q2nH7_gCO;RpprgPf$0!)dIonJLtmo8JBNuC{%4bJ zA!@08R@$PKUzUJ(<fSev3X+f;9}|?iWsKqDB3ClpHf+A*70NbzY@-hm?{LM)0lR-> za`N_f%DDWISd86rk%dLTx+f;M4mAcW|3nSZ!2?B$#B*2ezHFQR(qf+Vm?pwCSYQwc zqUOrnfV;b2`X{5M*Z;X6vf!Y=DrbdDzjYk=ir)y#NKDMnh0}NSQM5k(VzZ{+Jfo;^ zZh3vNr-U(E;Js-lgxd@k6Os;CXq=eYK>O=*cr`yBplQ>lBlNi325%lOhoY=A>5dWH z^2v;_e6wUiiCb`r<e-aKz?MaS=l(2=4LQdG4`SAl6J~8t1vb5c*71Y*A-wp4y?ePE zrto{YBQ7t+#}vOllzxc~W8qHdBP`*qf}Y*4ZnLq(A{(a3{h9}G=-Xq6;Ph8k4L>UN zSC>|YYT(v5GziWVtD5;I57I|{`cr}`1I$l`HoQ?5Dw?@xid+y4fWatJ<wHW>8p1fG zGeL2J@jWPCsxlZ-4FO63D&uZE<THmvg&cwfeiTR<rq^LEMUz#JXT`@fk1Yd7haV%5 zPblVYbE-b&2q_KV$y`YgNy?Y8+FMmzIjq54R{(<QqVB6!ZF<$sYW)SKmCPW4GBRLA z*XpAAjw`Qg4S3@WCKRZZ&d2LNK9EZ3yG5RFNO2IDClGnC1*X*|^p{U=@!Ed6@T^?G zqa19rzd()0mFpE#Wz`#4&8<*0a~B8aEi2ulLf(C1&O=9T2nC46&v^$dxJsM~U_zP@ zzqX%WqtSGG7=f${aRSf$mPTn>q@CDm7%TF99R|JD&eqx3xQ@S>-rbug!xG^_wk0?D zD;c;H3NSgjCF#z-mx|N&TL_m3c3h?4Txj`Iy6|CfqMvis{L7M&ivAUBv8SI4PivcE zlQp3Tp`*olo`v|aS_(VaH+Ikq0U|}{OuZ)NCm{3&H2+c|+k&m|2pP!q6Mx_+zpC?P zPDt><Qg_|FdGpM`mD!_g>3Wo;7Ilr6#YOocEXa5Q-a6g$NrBaEJTMtoogNV=Z@8x{ z6`VwwoOMQmX(0Z`JA+AyxNJ6yXz(ilcHtgeg>YM;=iKw>Gi|FyvAJXRrj2FcgSLBq zFcpY3V`^(vKr$s&;-B0dh>fW<9Bu+QWfpQ^#;Isva!s0ER0^CsiQkoajQvjx5KQNP z%%Vc`4b%DnP(VmHxh-7UaUhxzmIA+)=KcqbF}a(M@B7rInBT-A*7*d;^14U-d%#FT zz^hdfdj@*=LjwR&0LTZ@FjD%Kwzln-@P{Q?lHz-&a)bgTcYqjJl?aqceLUV6$*=}U zd3s)00LL2pTYVoR+nA)XEmHxJL&ha+z#_C8lCptX9oz7W-3Ix_G0>k<k3sI~rAv3W z-BUnNZ6Kty^eNcXqu<(&JW|tN83sTVj5jf|Vf%RZ?%i90|0iJ#y*)Te@^6@2?T;T< zccxMf?B5T4i(FLbp;Y<CmF+Ay^fKrObowVy3C{kR?%Lunt{E0)JKW_0u)EsNud{8X zxC5=zpd@X3|GL>Bm8{#_+g05&0_px+F!%BhF_g>>75;zyUOE0wvon}Q2qb%_K8A=k zZ5Hee)Vg=b0D5z()1ya%iu83#|2)8J^)lc-zf|9J4#!K{cXp&}9D^k_&@Hw=Enw#4 zeODf@jUEbk@3_Ms2+xeKU%zfmH-wJ?iMUT7E%=~8$GdVO-X21Wh(EjFe}omywJX&v z&$z6~2@Vr}1JMM)MwHlU$0sMZX<d8<GL3-rb^|~g;nV{e*YRrr5_6akx&nyR6rck< zkh|{vgcOfM0}?Ff;eRcsCLa@rkU>F<2g+xu$KV)@#Ls(*2l-*AN>(<6Gyq2_nms%B z-o9GbBYc13@c>8Ewn;m_r~1Q(54&p@STd$o=)uX!$+P6&0yu!CfFKt}^97#I$~Ped zRjTe58AL-+F3_&KV>hh%G9!6!n72jP01j4M3M>-g7hES!0B;Ns%HMzxgMYh#q`DB- z@*fb*AFv#(eaBYPU2L;qA9OjuxOFx$W{`0r{Qmz89|ggvxUC2`=tjFH-NAA)ZoVh` z+e8Q^r>?E7y)(6B6}e@_2NG(X^8s+8rlw{Jf<3|93m60f%a;UrpX7ke)!CJNXFvOY zL}$TP@H7GV^vdg_hf25w)HO7^e*L-!&_fKU6~9*EHPDyfCd@rKM6^nP23{muy3M*9 zXua3BT`3c-j74IA;hNN2)mPa0Wn^VPaPO)ODhIf_Qk&NJ-72=lg9l-n84S?NejPbz z7XHB?&hO63$~p#MFYK2;zu(dBrJrwXT_F#P4c&sf3FJcS+pHEXF}5qP?eSDboS@#W z+CANag9*@7r2b>4T(;f|i#0xe$#Qm9jm<0?_E1|~)fb?a?3Si`{)WZgsUVnNxpHM` zYjMGyq`dd=1BVV}<>bj2MG1a!#-U1WG0DQOt7nUwJL|(Z!I1VsxGbrI)1WoQEr1`{ z2BY=Ubr%;GAEaBS`vQndKBrqD0Kd#G{X+q8D8sc4-hzgxBK_H+n*lq4&S37lK#2O! zVYvIhaq=RWK}L_807}SvbCswIFlrn;JX&Bs@a?r2H=`|+n!|Hjpn*Ur2nGxRLP4O> z$_xt$88S38m6f6e;5%lKKBOUpgiq@!CFhZ9*{we_AW5J5_wnL?Fkq^T2g_X^HwPp! z89yX+=+t*e+JW!*XCYriAV98yF|_}O1ctshknjnBAj?z%Hfikdf91agCd~7hredQ2 z)2+?_{tX}@sgjW2vAId6D>2ZM0ABkr40sG{E+8(aozn<;B0!|DDzR!az=y!FrwYb) z%IQ_tgRso341GTc+b+)yCqPh!h!yGtV>Mwte_&3H+u)aCC0=M;!m<E^2h*eVR-k%* zFM++;Q-B`gjX4+eOauD!AKayY@dxOXaRlyx5T<`l)c(j{@I&B#IDvvV;K+h9I{-Iq z-2jr)Td)*BB*-m^<OCs20frGMzbuf^W`MB3Jq%P~8Q6$yR`S(KwHIZAG0mhLC~=c+ zlLGvwO8}IXGz2W!qtH9tp8+miIDGNP%oaX$J6Znb82@xb(Hi(%@Lf;;oxz}Apkp`{ zz=8oiby-1Y4{d>@2Hj11B3`|E1pv7~%9a3esF;14kB`p|ldK4k08qys;57l%V;Ph- zl974OeP~Mq{u^=_Na3GK)Wy5K#9gq|Hz3jU0KFc@Z7|0673>WQJfG5VFqps`zZnzU zqm?y)bJqsHMF4s7avx7n1jz3FLFGb#E1cgRkeAxV!>M(u!UCm^2O&+5|CZ4xTBS5M z+==%>EbAb{3tP_sTyabpdwQ01#t14})F=rIb<%Y~F>v`bF8w=%LTpbm|8Qe!McaOK z2!Ktcj-XY%tZkObzq5Y=(`Ju5xPJHsYHz#$xQ{zK#Hv-8Hz^s56swYknz&5&`HYg> zT>Ukocy00Wv^xnOV=MswwCrWG%qD!Kcd14cOijDmxDSkU9i{PVUhELKUo$T?_%_JL z4bU4mRByzAELrKB5N}&$QS+qEW(|AvL)nWos&^=>@e(uOxj@mAxLpq|78K=LAl)%M zM!=Wkem)&NW5FN2>!EF2UeX3qVN;9lCGyFKK!k2M$dB7K$pvl*g!IcWpZ={N!@oI& za$pag=i#e4x>f?=+ARg4d)fwp^nk#A3~;~~hej$)F2py2`O_9j@$>VSL@_5wIo{ZU zJI96@9-s}}Bn6C#79V9zjr!5lXYlCq&&B4F04z;DnH55@fUWNP?GMfa>V$McY@+WW zV1iOBVUU(7#}7kzIiM1YBV<6QAZzQj?&%m5_AE@UrOvM;#WplORdUO)I}BQ+z+(5T z@~9BskvlB63_=hl_&5cybHdDpC^_2d$B<(%?o8=J6Pd?KpFwmSyJP$MOor%%)<dWy zMQj>aN$+D6OH!=H4}-N#x(?P7G44~E4e;|oP2P%Nuw-cw;Fgy!Urs|D35E_VMhJVI zd>0sgk&srHt@JF0er|6ALmViz)8$}iZ@lqx9t?E*TJXG%zkItAMlL}pLq&$T2Pngq z8UVRqFphgL1g|<5#KgeOxu1q)4k&&qy$G?-*1??l?{LO@>VS1g#POfe=1Q@hEzM;S zQ_<!LMk{^+-Y0iFewv?GcSy13{L*rU`_E}W8Y*#7<Q=7Kt2~!7z*AIwysQN?@XGl? zZiQ!I16S8~5@J5|qrYdJ)197SYX|Ri7XlMTizHvxg-{En5yzA@r-%(U5k6wKrp7dF z4Gi)ZF~)(`q@JmP2BEUsbJs|emRT{w(|I#KW{`6c0tjXx%J)$Gf*1fP4qW;FK-Pxl znVFfLC_h>;9u!S%!OnkG{{+vA^No<gGEDU0KTuQ!DXDXBfzXf><4rMBV@WfmSM}2o zyHXX+ipyiwO^un4n&)hMBTUY^%sz%S8v-pa51(r>cYBRVkah;H#$$89e@RzR8k|7D zmR-~vhqNBR8ukNFI3z3Y+B{n3{nmfOFpW!v7J_azNaDES;%n()Ei7!cWv0Hy?uWsM zw*ciB0m}KXWex&4)B&M0+cfiD`Z;DUE4#3rP~Opg-O}2Dd=^YsSwDK_{s{=wS?l$d zyu<z6D$iX5pfdtv-zUd!3{k6gtGq(2bXT^0NZ|*oFBc7%#qB3`(iNaWw=rq8US3N* z{TMd0MA+no8m|k|N%v%D0Gok*kCvc`1B*^n+oos8(*8H8%s_y{0&i5WHO&%3gy&$B zGD6yQN5x(D#X_WBP&pP{>NH%H_dtdLxBOKNaX&Qq+x}7M4%}j<bSuDGTmQjY12gB3 z2axoU#mgY$dfGPzhA!#ambY~riePOYwa>O|6rl6}1owstL@L|0l5b8Ctj@Pqo6_EE zO_hBp!4b3e9))Q>uoh98oVt}=A(o-34^2}QBvWb{si|ifxQQ?SQFga~1psA;%sLkr z-T!4DO+)fn1EmoM%mLs6Kngbupu_oPwg~OlO@9B_PD}*g8p9Xv^USd;&|(eW1HKc` z%Se0ILi#dp(<F8&=#L&g0Wne(xg!Y3IZzDX1;sK)zrX4N_7Mc~JYcye<~Bg?|G>@g zZ=%7I0O6U%K>iPdu%NqbBgq|VTei470Gn^DG)D3PF*I$iWKQFip{wtnlu=fW?-Wwj zZl4+lTc^7#LN~o5fA!jOe%#{Yfr<VZf9!mc0%W92-UBMG?JVxOn?CSyfYAl0@Pwu0 ztUWMc4?bCdXfy3Oq_AD^^?z)xX25@P3BD1cP=x}&-h+Q2iHxl6+YiY%VD6P4*fomI zUCx>Y$an~32X=F94hP;a5O8K%fD%`MmaN?3kB<i`{WsR!LC2F)fSd=U`T^L`&=aP1 zC*^Uvi3I^&3avzvHcEr-p&GOjqM$R_SzvSgo)!Ybv>iWe2Kn(S*mr<^lva5Xqhc7M z86qi|;f4P{b~CX51}dB_JgUL>Z;2lu(9E{tzS2->1bHM0Dvh`fN~3^e)!0^Uc9xc> z<-@n-3Qz{y?$cnAl?J0mfb0JFKQZj>DiFe!8KnUgUStN3k6JN&UhvRBt&>}A0h|7d ztB~gjfZtoLO&9P<%K)3M&A&Xr$uR3mOF&4kHv-u8-gnf1!s=XE@$^uTlZ%Jugb~VE zTCPAVax3}D05tSgl|}n9#O@A7c0G)w_CD_aDSVNX3K)BRiGjW4-(ClSAxI~+zkx^% ze16A&(`$uph7RHF5+50BP}IXM^Z2Cv2*@K$PaQc0y|o<`?1cp`C*+jseHNBX__yfN z-TuF!OKR@~56!F+Q!wEmY*FFiJGV4jg2{|C_dYML1cDR;)n0DUWq}@ys;aSJAlN&B zrCW^O$5)7$hzG<66luYGA8tkOQoI|eyVwt5&7ofzYCb0WwATvzKk@%#jWy7p0*ir4 zhyKGj+7EzbT3TK{2c1n`f~?~G`@18g9r_@&Ixv6pO)3ifO|3_Iq69C3T7iFs5x`#= zVWZXr=`09f#UiBtXI(}%mn*Q80w7lJ#(<X{k(=P}g^Dhq2OhgR$AYY_Yl9EJ^Kl%i z6o*zsCF{Q|P%z$rVLj+Gh6UXRyZ056jN8Y(BmdJ0DC&W=FKy0#MQiOnw4u3HCM2fG zTRAsuyT0CrBp32xo^V1p2IdTvERA0Bn)UP;LVG-^npsZJjX}{{<`x_A+C69#i4Pq! zQn?v-6eb)E*_iH#SEyyrwQsuuqPUe8`gEs)zSG}rrm|jSsiV$lo!es6)QsdIl!S>e z*T=&KPv+->2CUfJFpUbJ*uii=bj3qAT!4-Cn&*9us-Vi5@PsZ}d9~m2CT30+Kfn2l zFWc&B;4)chng0NeHk+jsPa2X=`I+9(g|1m`dr4*~<ZS+atPHj+VZ!UQGNL+S0h^GL zkcfiZNN;cN&b6hP5)7C>AJ2z__^HtW4JRm^*iks*(@T?VW~Yy72IRe#dS&wg-#_Gw zFPn-T*b8GbhtS%0N<fj^;yMLm+u#hG6I}}n3qWe3p}qv7s#>-%+nx+V#dun68Ci>X zn&~!g4MiEFG@%z~{dLGy2c@L!X+t#%F{)!JLySGK@_M+eq_~hz2HC6_NP>bLaBhj! zEZ*_({r&4}>6TAd`YOF_Jx5$<HO^s_)L;BggNHu%QKH_dqHpF&>GP7h$X3AqXj&s1 zi_}x=EXY)^L;e$sJEG1s4oF9F$hE61JAgF3_zhHCRD^9V$dV_QRXO*#9r7cQ+t{;A zmcc@vjYab_WpoTh4W?+p^uT!j<<bufs<NJ!YBs!gYT!KEUsVQM)hB>QpqjNW?n;k` zk!@A>!-MMntD&^SLJ#92S+CiZY9T+XvPYrVdtM<bE0Z1lbha=vR=)8N>YM{vtkx2$ zOAYj34LQyqk#udtUSD!!l`q1tBDv_X6YIO5PLu6szh<NNxtB1wu6E(e{oirLM>~T& zP(qx1sZ$mua!Z9~F>M+OZKkL66_&@d54)gZP~ySm{eC%e5vxDk)La~jsESxxkOG%H z_BYuVhjde<j=dhII(zw$6#U=u3@pP-lzkp$K_m-n@9FI=j!(~WjpI*QyS!HWgzV&v z<Y(&IwLlZDUmd?gQB+UVl#6D~&q;I<j}?p($sMa{6j_w3r|fXg1w#bB1C%)h+(#nN zFtb_#%fE2lXcVE|o=w5YV_|aM9piV%wn2OS`_$3yh<dv01~Ovf`o>J(eR6@klheYG z&m+tR)($hmi``WtU667h+w!<WN#JKsR9!<@!z#rw0?SM6ctvhiC$c&k(%0NiZk$23 z`U?4lSlQrx&qZvy8NW8<=Aw$s#j+7So5SeMiC=AAtIUoTD(fTZx(9@=;!(WSb7oUD zWz_WkvK1Xt)i7Y{mosc*Ykt25;H|5PtCTt~Rx;=P3HidIXeAwFrY$pZ!kZjFEVi?X zM&9@?feI+V=6W7PSY+2=?U~8erp-116R<WlU}A&Px22>FDTee5Ty{4SvHeMjYP=Uu z5Q$VHgmCid2ItwHnrJRzjVHZ$^~h_4FCsv?`Ph|l-J;ry#a|nvS*J~1>oblrtMjg{ z;g<^Ozj|@;@QV!%WK_V&VB3=U5Bn6c8Ig$!s{cZI|I#B5ZoFa*sTZe!cR}4onhDpp z;gCZ~K@<@ssXviiCs51=nQ_}5+`~g+(e*j%_wCcddE_Ja$XWM)#*~CUs<EB9vNZg& ziFhpH1eLyEOaC3ZU^?%MU;Tc77bPK-#!@qXpimiK1UBAIRx<SQo3D1uUla-d5cbyj zl#mn!me+3|PN|~IiD938MqiKcUvjfOzA=V6;wm{Peqqjs+RJMj^bO#;-L6!!WR!Uh z*^DWSrLQM??K2P-V6CHC`htA(>WFfpL^Q2r@1~s3Y_aEoR&xzmLzBj(VCtmU933yK zk@~}rLJU4EwX@-8MKxl%XI4_&iS;uSzl1!Ry$89jfmoy+@B)Aed#LTqm1uV61JSIN zHI0>D`N!vOa@-Bw)~o&$ANVFwmXfEyObZd)m`@C<Xv|hxo2PGt`U)&$D{z{AZv24i zIFl|s5lTsF4l4H`O~1ra?aQe53ib=#h^!kNk?&jK7_&tiAj4aWQ3`yaou<);5;hdr znDh9Y9b#7rWcY(8b(HnNI~|4n_?^ORTQ*K;c(@He!n!%uz2Dlu2rx)XKBx(B+2hRV z<+dCda&MNZw3$+(@>o>oQwRjC5#`(<WCrDXwoo0tG&W|&+B^_rE1gu4R8O{3_m2~U zpWdlXHq<fRPZ>ifdlTvY^&5QbzG?UMekYXBST%HiV63d|M0e&RwMmx@Q5N&E20swT z3%+*@*slVG)OGkr9YTtko4mdlmtanFof7?4-e)vd9<e_0>|1k?d7-Cvf?1(y-iv~i z@7eY}7R(|zDepDR?L3~pKC4Pkm>*G5AN`V*Rr~}AtdKpdA^IU94)o<Ba<WT;rBeA@ z2Y%OVdfCs@oeO>zGG%?GM&d%zl(b?5t7oj5l%eo&Ym7uar8Q2%{2GDF0X1{GMbIkp zH>c*RQFVc3Z^dZ0Xh7Zkm&m=WyjdR_?<d5}1)pZy4XW)y`1n?vrHK-d<vu?iVhlw% zKB~m-wIg&C*QN67Yq4s=i||8c6*yD$E%E3O{HqGZUUxMc|H1X5g^rwgnJ5k7!4Av4 zHS9)5p5841LUQD^JW(>&K7v}{PW3yKvAW*>+`@U^4y&On(e2X_)Ut*dyl07CW{_Fm z!B`JYVVpFNw7S%fCE=HkjzqKUD$_uFJ6{nO(_c`!i*&L;QI#bV?G@A&{6mXhv$W_4 zw`kgnB??jq)@1@*>g+z0Fszh9y)@H(haq}%(2-)74SH*9z+L5v1;IW_jk#LgjQ-FE zF{gxxiVJ|M-$AjPhtiSctL9N`IhIYJWc|F2Zsd_9(^{+85OeXSOPtIm&4UZVLMd<O zzh!6d_40oo;`^oYt<;<k*^KuV*$3-Te@r>VO=<9BlWUuyLfrZ)B~LIs!P-pk&3>U) z-J8hO%o&c0jg1#6RO^D_ea-#%#dn3QOL(L;tBr!vs`ZE8El1c_<I!wzXFt!jDb`b% znK6zD?Bw1uVz|Sl05s+A<6c@PFl?u_MObNb8{HZu!>n%gEjyE1-su5T@It`mQbWSr zcx*^&Zq64z4|buKvRP<!&J^{E3*o_TRUpBr9R55+*3mXa6WeBkBFiTQV`c@WBk`CH zV876p)9h!6I{Sx~L+O<~7i*lGd$OFA;cv+I(pIFgUXUiA(x5Nmy&GAc-ZwuZR5&1F zQ6g=FC8lTh+Y5|j8dfpuu!lz2jFqYXGTxMUtB5s2$8A12)gF2wpIx1`0Xb2azs|lY zfcyO$%i}E2nnYfTpHT?Lig;XVN%>xB1XN*a0=I(RyWrU~yz<5^H=3+$Fz14u>TaHE zvZ_;y@QPhhO}o(RaNpX?wv47sOW{~Z0$W<F_bcC|ckT5LXyc|dMd&KUt@}FV5`w+j z3`KeL&!bUpKUw#)g}{SUA3W4&5=JSX5<B;jzk;4RTJtb=wce7*uO;%UbtI}}{DICV z+Pzb{L$~GqN87bI+9<R#w_vwJt3LW8*Ts{s^dFpLW#If^v9I>sIsG<%#_Bvvh~p^G zp?2QLv3;SH8@^U2Pro*KI3-FA&z0(lND;`ZUmyIo(s{^~KIbk)_&xI+D<y_xF^FPK zadxz0v40WA>=n^lU}mPAJk0WYzVx<EY@*PLV8o+gU5-%`9whSq@0N!1v9M;U#pIIU zUxCV=%PqCVGB#56Ol%g~yZ!B%<%Kz<AwY#Y@?~DeTg;EUFHEDWn)<c-j}4|ct*n)s zB#6a|h<y~9kRbS8C{9GM{%*^vzTj3MOx_4G6~!4j$u=@aBKYG&pNW{0J-O?~%w0yG zc<pu+HM!(RnBrQbXPeOwKM=lOegZ=_+uB)8Bkt$YE<@2-C^~3*L5#~uIEGtcG0F>0 z%*BxbqHOa{V-%85Pd_VC|8+oi{HZzWs3dRX+gC<K$8f41*J+pUv)G4bwyNJyNQh4k zD5!jH4yxWL`-+-PUN{_yt!wH)w4L(jv65eC2-Z0gfFjN31+r&+JHTwJ!*LePzUv5H zeyofuJ1Cbz`c19B(;-@5E!!>hS2BSj@G^*EUii(5%Xhxzw57|^Z~1uBboy$11y*D1 z&m8q#7r~VYr-&Xwv_p0f&0P3d*c}6xCRGVsz_PoYSaA}96^?QoZ#`ha#-<-j`AWq| zY`QBc3yQ3SDojOQvBjLgOU7zc>7QavkF|j7xSVnnWu2LjHK=AIxRlh*nDHDlX=%_Z zxaMY)r@?I9q)0hVo;XT&>lioM428LKLe9nHh5q*)Ug}qb2~u=|Ud-zIS0x`5%r|Z{ zS$<)W|Gc42iYEjW!i67Sn)<4Gn_cJ`P!}7vHQHW;$Dz0tI4><~O$)B$8&YiO+Y<hL zrFCuPrKvAHssi_Ju9~(;sYBqrsBtCu6^X+;rcL)$dkzC@&;ldB`SF8ib>A&fF=E|8 zzdF}n4@fkezzbi(M|1H>dF+gyc`Fxknx*iHXWAl)eT$+>9Ec^lA<C4<^tzyr%pcYd z2Qy2B=pKt#v>3nUHS*-s?Os9Rb9+qDZUfQ3MbFqi%Ah(U<Tk!?$-~;tGDUbPah&Ll z(%wi--ZjBz=~gFRA{Kql=12ImOeL7`fvt%U-=vt<O9<O;<mOrg8SgJY?QUvfSmV_k zhRxJJUwAa^=&&>|N?Ng$L@w5;r~^Bcd=L$OKsds!QxHOx^+1KZGuJU63b}*ZdxP)v zT>skbWm7es8$N@RVz#`89DVXo1FNBKg+ZAW8?SITs6*7p8(yk;H^W&F!1b8-{*6U@ z^d#cdS<`)Mf~{<>%yv_sUqbKWVU)ZB=$VlrYM(@X>qG-{()yR7L`PvKc>&hbSA5$* zH7~0=W<vg`{{LX?JHwh<zIQq5(PQO^QUnDBl-^XDG#emD5$PpCK|s3nUMz@o0Rg28 z(wm{z1eFj00R`zLN((&_N=QPIJ3DxO<$v#e?wt<`VeiS_Gi%nY@~(H*0+|eML)C-z z_d5AlX;U;6WozSVTbb&6DM&DzJbpjLGMAh^qHC0<n;md8sy;?98!~|0uCSZF1t#Pd z<;uSCX{_^b%<5t09c`nlO$`QrMdPB~C!BdmWn^)0upCbw#_9#ckT#>vEJo%v4R7KT zea6S%AGR8|RjwQNbiU%dEwJ!YKQ1w4htsmRcq3xWRo>bMZPl=5Q88rYvv3%3eYoB< zG%W4S8x6=>11VN}DtU9n4(Qqz)nnD(E1yQh9Uxz95nbaWn&WAl{_N5F>Ls(0`0G-u zUSV2~oK<Hl2v*(dJV>8waV@X<uciv=%=)@h`Zw|Hm);XmOJ|M}h43sK=;pBHT4SUw z(*z@HQ#1mlbfCbNqM~B@E;8q9Ml83ZGUt{vA8P!US9G#TCu@z~gRUE;fd#dM9`|*j z{a$W#4mZTtzDbKpy$>~Uf>jei*c%0yMrGRfQy^38H$w!Ie)8A#eTCRbWK}RK97*$~ zD(kgV2rVJ6*+i}nDDF!SQqj%lS)t;{)@mjk3j1XaOUahfc|Z+fi_trDSjMbW)taIQ z43Af-YJ&guMZIu%^O&4*d;kPSDXOch6LGJw%L<Zd+334Hvi9NWfqqcb3u202C#`<s z4pamg*9unEn~z&vP_Z!TdJ<`BrgimU?TN8WvpM6_iDnkQ<~>gc7WRnXyjKTX9&OXL z@d~{bE<&#`-D=Nkyf=&c_0JEMmd`pH%Z?W)cFN}4IGNanT@Rhj--QijDEQ=4DMUM- zChrwZ5F{X8ajzy7hS$Y3%bTA{qX1XT?vIJP5xhY3D)C}gT6l%>(eJ-D`9g>W#hS0? zX*kD8IK44YTOdvb<$`d8OFfLrzGQs*9M{{)hVAVM;?Pr+;HnaNpf|5Yq1J|nz8NoJ ztI*YLm>+;mZXaks^ZvN5*wEru=(O+C+LhFh=BA&uQrG|dVWGOp`q-jw1`e>P3?K>9 z*HfJ%U%((dBjmJ0Hs6LfbIQdmlS26+>$Z;_4TKMT>(Pl~IDh7DH}Ni*auOf=sX^vU zrIs;RgXhpPW#nd@9lOf;9O2nOFTz7g+#ma|=<a&<iHV7u){nu_VV+r*Ld&)(cu6C+ zYkiPu!sS9-)5T&vFRKRPovd|xjgft_?B|B8yzfPgu|GvXai5^NhRVNMgAD{fgbDVx zQ1+s-CJIbj>v35+SfpISw0b~Dx=TGNE)||t?}A=375&M2<2`|L0_Wa47~XTGvu1l8 z9lKv1yR>=p{&@SyWC^H9*bJUVENaGIB{EXH8QJM6qPBz^FNzKwwW+J1PMjiX&ltv? zbbo!c;DYo_r)t&ktk}h(oX<tH2@LJs{h{Vc-}I@EeV-xiOv^uIzO&p>;GJLf22bj_ zkSW_cJ4b%)c+Q#RrPx5-Xh_C<nR97aYkXdTPv^6a!2vQ=a{Fe5>vm%&x`U|;2D0ma zfMRpj$7XIb_R$Gc5PTYy)SRk}mz8~X{rFbfcEk>nn8oSH`3Fo<jeDPxUQ=&oi%QR) zx}yc{IyZ$xujf0xx-MqnAXAwmjOde9b9zP)w?FKL7f?y1L3S}pS!z*Q6OP9cww9au zJfm_fnlpW4Le}zB%CW>u3U_B`I4NifN0OVYPez|vbWt`miC)@*e7ogfpGoTMM%V%R z1rcT>@LPM05|yjV+*H>Nlmk}Wdbwl#%GgKi84J`Qi;RqESrAEj0?J^r!nl-o)Q_Uk z{dJ}pWy!%~=D`hYPh^lj2);?W*<U*B{w-EG<kahtFhHVoW-n^3-mMB4J4_8GC%OFY zntYiF|J$Z+jP}WO;9^;St{$IdC;Kg1(rtS^9Pmg2+ox~-#8Y~AK&V4-5l!W7io`^h zJ~}5q;9P?tZEY1S$y?LpfSLH^&D751T7|6iO!Vx>;R$oT5{gE$rMS@p_gRbb()6Qq zR_d)ajyyeZk2SJ1*QEgRc!*F_Eh*sm5G{n<itiN<R7wgtqqrksx|M3#Wu;@1_j!}x z)^Lt(4Ebvc^#I2)M5hJN^>44kLh}II3=>nj;j0pYyV;A@IcE5d#-c=Y&Wo#yy(pTm z(&vcx0uq|4w_ZhsitoucsYwmDl6cTGmeIg+@ky;ojF|<idCgzBR$7Y<kxUEp(w0oC zH==Oes%$YVT1?Y^HvVuy#fLXT=Q@2!(k5<;d?~C%ov=KckVi|d(#wa8k4aBkM3yAv z@)Nmgh~%Fi17Jo+jBx?0<6^R~G$mK=a<pSarB&%w_?J!c7zQ==tLiV!-nlb1L>Y<6 zFphz@z6Tz8??zj86UkrYc_kv_JXlS2PmV_FADK(d{xxE@e*@{$LXF0&bF=&Dtw}|i zTD?dYK4c&F<r4ayl)QByD*Y0}!iv(GmI;A(753!$`Y2bzm@3A<zb};kV%XT#y$Qb@ zpu#)mY`7*)!1!iGH62pc^A%Gm%;+6~AHG_`s1*FZ3^>H7zMxzUxb+!)YV1yeF`Ll? zk*df4R{g{IM&eqQUbYBZGjB9$?i11WkFn?S?O|%ygr8IyC9f8<*i_P&e)e|zl^;Jj zSAlab=!|r0$q8k{ra5}KjZ2j4xt({iXrhxpvsY1iyjG@y5mn=0OYVM-$E>&YJndd( z=M$qX$s#=mVRYVXU~#1NUzg?*;3~Pt9sn5=(>v^Uv!fo>&O2Fq+FMuX^ajK2gU*$G z17t@4zLmj9c|!OUP)ZG?Int$RQx?}<&0FWUQ9{gT?nzuiTYpJ%wINp6sLwh9f4q}$ z#0?>DJ>!wpOig}{S0$B?)Q=i9<!OxI_YR>bPCg};7{TK*{i2pmVF7q@i+mN5AZPVZ z6R92r1`Dc|Gvs&OuXg)y097IV8(NfHmc@O`GgeR>>{OP0qtYdnhOcL(a%RWEWRKDA z8fm31)sj|n)7kl)dbftVCT6~k9A}I2qvAYJ^}40=O8&xsN52%~F;>V5L#E7nM6<T7 zr5}F~JQbT(g$w$L&bmDA(mNUAV)B+tftyAYpvB4atXsfkyS}1EU-xLxun`_DA-*JE zO-x8?A38{=jKssjz)yr0*J!s3V6D0J47$`lWvZ7Oe__G8&Dwad&9hNV4l89%Ilgdm zK=Anj&WxqM4tF0k_d4cAohJ3-s55&K@gH&w%|LSd$47vKO8%&1&QlzBIyR|#eMQZr z*?AkUyW<x$E_%j6=xGBTe@C;T`}_5V5W~YbD5o3h4e+q7mAu%eswKNHxr`M`ZzlHu zXRh3w{^^e6C1y#h58tH3^*ncHhw($uWpEE4HaCWmVt#*C>Y@zsvo*F`ZCvJrO27=F zV9}D^?uFO+%2J_I;A!Q)Rgd>#Bzqq65tK$PAiBBGKYg*%uDT%O;62nNP>a_J*rLh7 zWC&dg&U~&tNcN_2YT;JPj?7Qznk?*AJXY&Y1HjO{9q$-&cx!<gcn|5ruR0*v$;3!* zu+1c=m3Mb~L1w?5CF;?<Wo4Y;&m<^d?LzzNwNPuGv|4y4I^7pQG8VRd{k8&9%4;}x zp=*x=<Bmb3hD@zbRO45l4z-8?+k4H)r%7-U{dwf*Qiu?RR1v-7POd2t2g#bP2~*NB zw&j8YY+mctR;rqg`|mEUFmUj#EKmxXk?%2iC0kC<^Bg)OD^UN#XwWJ;bJC2%r(=0O zf>q?iofd$_^8C^I7^3}?aT$LNy^QcVtaZe0!3UK8+*%n+Il~OFkx1nj1Mug2k7-Hn z%w5JH1PReUUh(~P&--+itm|Ytbo1PKto5ni$#eh#vIhxukR=7=l%QR?2JZQn1@jv| zmb5bRIy~;$ZgREY<!|Dv=jZ>0sQVXUzzb4+)BoTOAfPvd{{!@PqkwLWVdX+!l6}Gn zWXKqq#ntD#MYZ9~JUSpsI^qaM7KqDl_*=eq+G0$r6<Cv?jgGJyTf{>#>@z+$RtLt$ zN*+NMU&TCR-~z;Vv!I8!7B`l0NlEP|zOuMWK%KFWxM40fP{PsV(4~qm!K(|Psu|pu z29o%*jPVc6laiLFkfs1VCTP_l!w=J85T4Zkf-9%?IbITidFqM1HqU_m94f<$-9?1{ z*c0G&CGSLm(Wfm2atyFq-b5aF!vK3edfTQs{MW1w(&+Fkx$HRu%<c`=%oT<JX4hEF zw29~oX=5;5RoEuavVr&SwLk*+9h7hcpyz6k=Z2oeQ=Y`_EKVr@6wX`#>OzbdWgt+( zm4Oxo5WnB~EvCjcU1rf`ZxN$+)wXBz;0=GQ<O1w2E;B&<jH0jPzjNSM!-3F!*hMak zHF(&irl)6MPiOrJ@<dEmZEeJY=NHPt+)}>8W||-_UQ>XoQvMAb1z?VU&^|D3<^7NW z%H;Vb#OCz*jN1%IiKiloTQv2MF!+48{lh7F2e5?QQQ<y)nt?cB0QLdjnl}iEFSI`1 zJF<%+1w+eVNrSZiF&^b~h^=l2IF2A?<K*N74$!`y#7Lh(H9{bJD-N=|GS&OQ&bA9? zWUbKu30)0p1^_iE$hF@c04d-<bAieN0fn0kSzyY3$PfhRBPJ#`EilQUgJz}+7rK~4 z)frgrc?((=bM`twX!k@wiKnuG{xqifNB@_<1!(L6Y>VS5ptJcq5QIYjKsa(BIF}== z`AQxH>p!!Y6J<+>VBiq84nk}&_q<{BT$8h<+atXfKqm$OV8t`@=>g_^Fwx+7A^edX z5JCvH1D4Nk^aDWT>o5SQy)uA4CB4s_Qui~@QJ+luw(shJg{3;q<Z_5)4&qiid-2GT zBkw>48+68i;DE25X8|=JI)Jyf_czQK!Vqg~X+6LIcLG}coMq&&kpYChI(X=_!9T5} zMPWHWh?+gw2}7W`0leNZpv9wR@|y10>|*inLbs(xCh4=oE_3y?kRA49A1yPHQm`r< z(jj^>Lx{O_uvi|35R?kFY#_-uc{yCEs0^2fQ(kK>)R<kBUDw=m@07V|6tlm-<NfjT zRUzS+V_>Y9JOv<-4nVuzhLZ2^FB3w+RN2_vyO)iu!a{&|rqFzVM!A}{yZBxH&!PmF zf42ZQE1>GP94$lj`(Owv6d}GtA2h4+JFlfd8;sFMNIiTN0$@Q1){vW<%h-UfTsygH z7Vl$t#lh&eXz}l2KqCJ?6Zj160(u7Rf28(z{}Q?%=)E40H-L0vH7|ZqgB2)yMi2tx zR0S{fZ@4jpB!lWd=``vHgfHm>F9$vKZxqEDd}h$r$TB|$gMsL^80eFKK*gtl4uICl zRfS!xznYf<FXqt&#ZAyR&+q%aHcBfQsaEJSFaf_&@7XS3%po8rP>c`>A^@$q-6a*q z9OaVveK!CVmBgj30ODG(N*PL>8(`a^8+I2uAbSCTk1GK9k^1J19s`D*ZQTrb_3*{x zC4l5FCN_2#rLk`pIU4rQ_K~6X8>q&xRiF-;=4B8H1PCDpuv&m(+hOEmjPDKz1{46L z=?OTF?Xm!eGbimlKw?7e5qD<5YJ(?8*DuR;x*1!zsX`U8#eI+W9_V8Z+5SWx0_3s| z9iJ})WnlY{X84f~u-vkNXZ~yu>@>`?00H}7dztPn?1Df*r|IDg=4imwTyDA`-ge6z zfLUYz8M#(J4uCXm?@G>aCApOM!4x5=%$|SWeiYJdK_Dyt2;itos{>^%0c|uuQinOJ zGN32`1VNpy03}3amMCQkK%+rO7_bT<Vj(E#f#H&Ix&`V24uV@c4!g~Mw4iNXKkT6i zkS_!F5G!e{Pp>k?$va_<xfKLTeh=&}E>aGd0pL;GdVFvY^>V1gE`W;IcUxj0P5>gD z1kG~~v}~ii${F;`p<DO4MWK&@@SuBSylb@pIdyi6!N2JE_V3@nh^YveSOPcWT#y-f zHi8n=_c{X=dImqWZ(WZIdS<8#Mga~dU}Of{*KH^x&+tqh1<#F=R-FLr6jO0L4)(UQ znZb4(KTPul_#?`Jd%AH#wLlXSKq*Yg&!wdblrn&UUx9?SXUHP=K^nx3W<nFzK1iKr z2Hz5@1Fg~@fnuJ!uetU9AIR=kcfG<Ws|gF}t472BHD!NZG-i-)gZH`u*pWlgP#_R` zM1VT72Wje~$ByM2y})3B2?LneG*T>s{Q#pXX!XAcPbMZ?0{Q`7u@O|EF#3(g*6v{v zMf`j3b+-^Ps^tbSD(bIrmC}$F0I$3J8vz!Ycl=a#<r$`{l#Aet??s>^2Xh2JtLl{t z&-TGyJYiHjfbYSnfsN{31So#VT5Bop^y$|0Ij1(dhpE-V>TNqfF7D!$pFnu!nA8k_ z+z&>K0b_Bn!ZIspKhuw8FiDkbm9tNPtTe=?y)N-&@0A9yf1r+EL$yHwJO@!-o^c(1 ze-4zE14ti8@fo9=MPvj=0Cc+ex{bo55@oq+M5A58$7gl?4Y;vha7guB`Lcx%GOfbX zr(9JT)X5F(vMhmpj3IUp?a<97zJW-v3qygz&FSG(P`-*TD&8e77yAvU28c(X+Sh%M z|C0~g#I~n~ap_go|CF-dHYucso&Y_>P!_EXT~Y3{ys#=O*c~7Y4>Q1P0e0bkL6#YI z9i!|J6#f<M0lIbJ*UwK7XEkNkMOW7*13Z4YwDDV)a||gw_n%0=ubjBMNPW;Nqdm+3 zpvF)eu|uAE@5}}Bn&_w&%F%=+QRbWU&gl1c{4pPUj6|*N9o3UgJ;ew;usmgV*T#eW z^Ib=~b1#Lp+!I@m&K>^43CYBsSX^f2q=me<?yA!EPSOb+UA^b<hj;GQ9uk^{M?N@4 zeN31>Wt0dTF}ahcd682?hVN?z?z3GWxiEknr$gMx!d4WQ^-l-uW_dLBj=8Gf7I2V_ z3EI4P1W*Qhvj=7rG{6sxHX#5B(ACSgk%|8hC4A*#!80-Kzm@wRm$h61t~Q6D-;;@{ zDJwG=Oxqalm-h~A+fWPT9(DKDhISk#-*0Qvn5vM1{CtMWIMV3S)+Aiw>ofP|jpxSv zH{i18PaT3TykY%)3Lbv!jOOjzw?_zIm87PnF`OUC!bVnnqoCnThd|aB8NvaT?Q8xU z-_#j`eB`NKRKV=D&xDIm7e{hj&`rX(n*l$n-C)XkUm*0M<a}ET^gqD-L2vQ9-8*?} zx5#HuTU)zK0uSsPXxD2!yJXbtC~C+rBbnu34eTSv-v0bUA-V<fn;;ya7#@_b{`=al z5g;TbB_#{P2<wF}Z!STFGxkoUmFbL~diXLp9=T@<evMI%!@^YbvF-N|fiL6ulbx}C z7>|@~GlypUF#j0P*h0o#!N9=4G7W5AU@Pp%mmJhlSgG#+?|&Q%&ZOxwyaX`hCKeVJ z%sJuu1v)m5z5cgL`*oVU#}Fj`U8e4ht5PpA#~Qs#yhrT|GXDFYGkfQ4==Qs_mn{Yk z`tA*WOarGzTn0v9mAxbtl%8sWx<L!(Ld>+~X0fr2v#k3Xn!Ny&j6VFIePwv_^|^R^ z%d||fRt&$NSm-b42j_7;-B6+9=wC*iut=qE=H@-1P6lcck<Th6frlz*xv!R)Td_Vk z{rc$=5{U$@$ghXvl*tdZoveUav+$b(G-BP0Z&Es0c;m~Z-gls$3gWwGtlm#nul{#Y z`YtRm5q8P02eL#$;SoIh*;IFgX>;L^abU`qatz}Cl0mN^Fx*`i8QiV+I8IK(-MGB{ z{;q8Pa=Y{K3RIm4>8emTsScpTGl{Y)M@lQq*UEud@j6~Krquug8pvR=)%X&Sg@cig zR{|qPKw~_<2%sI-cJ=+l)i*OUL&ok%zONI`*rnIMH>`S(Vo6^L$q-|IItKO8av2cU zyVU=DUV3YEu*@7S<X>93i&t&^5p~F=0otq=qpsH8lK~fhW#-M~)A?pnHw8;FY>qnm zrQr1h-~ncsd-v}|2e3Z%{N#W|1}y;G`G~uLHzUS!ZLrLDD7gw8Z=jQ<46;Z{sYznv zpzMy5>r#{$^axLN84}ARSRxA4!9PvyaTboTs+$FcPjdrHv8=otDnHEo{hE@WlarGx z#{h6yX2oHrE3ApZLC*A&0h<SUpq~K3v?eB>X^{TmFgQLY=Q)4pir1n6+xiQh(|3<T z+6Xj90P;-+lyz@XQL3}hdI`OMyu-(B`lH6~n!Sa~hY6*E!dAd@0xi&ccR(3rCOcLT zb9Fv?cY%dU1H(s}h{NF^J<*z-mXT4`U>W=P>nZi)-~*;YM00bxHsXrIz$Bo2DE5TZ z5jc;JK{b!T?A{Z_q05g208CNmiAHId8=BSryC>gd&FpfjLA<Zq$A2v6BT;&h0cWmw zJ`8G%|Lb#d$Z_BRfdPESfsf=d@9FN&6E-kfE3bddaH)W$0R2^RUQKK@3Gay99o^7t zM$tY>>t8M<ou7og623cm6}pIHf9`2w<bo{p^F0%TQ~2Wlo{QZ#y*Kgw`*$({=n2M@ zUz3rSZ$LJKYC?k~1Q6=aXH<d%gBT{rztmVJC@FLzL8}PrTJm!_IE|sat*JI0mMHuP zjDjbqzOM$(4pb6-S;ojzhhbber7Etn@=6Nq8rlZQP+;qK%WduT_=8F2!>72@w?fus z4-Dc%eo*l`!1^&Q?j#U28>f4khVYg?6VeOLt}RkW6PQD0rkA>mk^UE2)-xc_vv3p? zsY8_&RkJ{rplN`*em>IIcNY>ETNz-gXasC6YgFq3Tdehh?_v>pW1u3)kxCnz^tR56 z@By`}g{I(O?fv3lu&KSYZB9s<Jkh(`Q+G1okKNJZ@Q(LoUwc`PW@b4*>7@pGlavJR zP{T$5I5##GHQ_b1vd}mL@2_4-3t56Kv=JgUDRtOjNB?Oqms{0%YzwV!D8({hVqhUL zYa4h~q1U_?zZY%JD*gc(l=ogogd5y|I=z$Wx7CnJVOH6x(>{Tx&HdcFP*lL?Kx5NK z^u3vKd_<}mFkJX;DR7(wJ3;6lTLTqJ-!Dh3{tRaZS}N4P7SsnI6`=zOps~pCy(JB) z@4aFPtC{5Ru%mRkF>0#}sF!LNKR<ae&x$cmL@Jsu9+p&Z^#k?_v`HOl(u)oUQfs(i zP=#=l!s8iIdsS9@bvQm*H~n+6%K96X+e1MOG|wt#758kHH1D?W#fb{F1sxq{iu4h& z6DgM4)umlsxLW%4TAL~Zx6>YOr;~L{4cxW<U^0`^bgKvp3tMKhwX>@llN<vogrQ(K zSaaE!7mUC;C~ZwQ!jI*F<^-{q*#kucsA<i;wbb?uZHc~F3J9V58jXmrrUY%4NEm`M zExe5D6jz)|FR_r1snopIX?|Vm1AMi-C3t|BKb?Y|xcku<8M`g(GvQdFQ5UkYccjsb zmEwyHVt!W<xLGk+X9Fs57Gx)VgSy5lO|7ug(%Wg_^x6dHxWzC8a_9B2PB&o5jDh`} zaQ49qD)-jzB6@BPsZVX`v;#b!F7+xVf>4N1LjB}cE!Pk}WW|v=gZNreNWuaV;5LG$ z!v*0Y>LeZ|(WE+dBJ7Y~Z+V#n3u=;Jj{m~)Q-<)SH}HPY8C5rWH)^wd1ASq|!?GFw zVPmeYS%gwoFNQchM!>`AW7>5#RWq2gwN-#U(%Q^@%c-^;S{Bln>iTjkuNzFE!ZGgS zBE>rlTjPD<MG!{;a&=WgHLR(wWizj>sj$#!ix@(0BfGC5xJWH3aw?|c`GsOCL=3)J zAgD&C((=WFO5G6e>1UM%nPytuH!CO3_)NkQtG5=7du=tIsUpN?ohO|(W3QdY5D?Lr zaYQY(JJ5Bd#;G<g{uO%%Fz^^{;sV$LI37xQIlg#TYIj@17;SHB9XV{5#&4rqX(DxQ zb)X_s1H0{NsYRzEP&!e%bXX%bw{B2TnWeom0`7)Nf3#KZGnr1K5PQp}>69(wN_)Mu zjzRsx{p|gPYGcEhbaGy=ZCCs=VqY!Ybdi0{>}1?ECrSvFU{^KGQD-8qbC_#@Zl-ZM z`d!PIioba`LX`XItryLB<N=_m*=-#h9D@8n{m!mcmX@AwVOh=?E|}EWI-m&w`4wQW z@!2H1YssgI=v<-zp7nFEV3N&(rOa!G+H^_wi2L+e51#>kZ`D08DN;E+$JlT2`|D$} zqgpzdB^zvGYmbR73g$y<R!V+(O^R+SgzJ@~vm;wUR@C5@wol!6b@oI%_nzRtAsQ0A zGVJm_Uo`fEERu$<ncmt;wLXQpr~Wn)o-6zeJggP4av^^aB3W@+Uteb!nEOX+hTvmK zzQ#5YKF42z!!mkYb`Oo3VC!(l?>tRm8AjeMd_;V7e{n<>x%PY+JN2r}25oPDgWs3+ z!~~p7>E^oNue8amQry6!;k~J)<D$z&g5M8-13l$ccb;aXbz}!_&m(g~>Zep&b^T_o zQt?5y#I>AmeO<B37WmmF2}j=}<re0~;tA(u&6t_43V{7d8dxDK9^+Pw&11hQSRvL$ zya+uH4-c?}Q+OYXvD6bHUDsU4L$-hJU82*0YDX;iN9uohZp@m{&`G1ZTA(s#*dyR; z%^<lfwXoD_+&iev=6Jyp*H~3Eu*10!9_ceX&7r>|!*Vbl&EIm%O!NXvt#C275rUg3 zRPDd)cA48!k41!sR4dJcRVuuPv>G@kSC6vHMY7^QUXr*lnREgxjtVqhY}2RTF-HIm zX{OP<bU;AoyAG~NZ|Xj2+cQ;eSi_wUoLmN3Jjc#ld1Yh_!ZMHoR(tY~W|a^g4I*E% z{G5QxVSz~<FG+InS#gljc5+dViYw`yiJeroRMIP}Fid1UljWTWWUrGuXycnF(Em;( zU^?C&vJ460JPB%wwI!N=U1#H0@tD&hqa}jlnaC5#V&+g7&Y#p}nK>ceIG^(!pA#2D z+ky_086>tJ&BnGY+embg$8L)^@Pi#k6-x%CDHU<fn>DRxUm#^%q{>^M18mNon(@7* z5)XFYBh9ydbQXlf6x;KlrUChhUX6Hu#!4H35b-?eduP%J=lN#fiX*;9xYx80QS=G1 z^M7yMn1ac_Sp}Qg^EfeqFJ6t*S%<I;BhGzH_b$&~oBJg}CrH@exk<ouc19jfXl;!M z7D*7E@&=WDqURof6?Xgcl5G6f(?QNoBCXksQ)ec4-F!h*&mPEh(3|DcS%>w!gj|@O zUcfs}EBLpuTUjUw9`t?6$-E#7ysyx=zXt0jU!R{0Q#@WWU~l`W1TNjQg-yx#h!JwM zq%Mv$#wqQ5k8@bX#YNq0dHq}_ga$O%tqj2m91qZ7O;}lu{vza}A?mIN{g;ZUOyKe- zW}ZT4%Ho^v5%?f{L@K38Tj?-U>&vJg^@kH%-@jN?b9?I!@>W4ffAfAg#c?72>P|0x zr`LY0)){=na<C_U`K|jU$s+_F=QdQn^O-YLOB5GV_Mdtq;t2zhlbbztb)$;Zb}q** z0ebhR5Obm%mO{ZevMiQ2mDlD>CWXx)3g_A4DsXS?GH-JUmE&Bwm0xE1c~4`Eg)*0b z{_klVgVV-JEDh5{eP&DS9O2=&%1dNfofI)d8ZGZxgHFkbO!MAp>7@n>pxl|9Uo7%G zy(#_oJ&xH3c51>5n}*NY(_0!ot&$PSVl3CKIERAIE*#@9TC;8_Tpq{o7vt`$fNX*r z)AYch+Y#RJOoKXm250lyDsIh@zt|z~tiQfRApcTdEhLRmu*7g_Vwd68)Y|O31`|AF zJXFUMQ7G>g6p?~2Neg7xpOP^*R<J^0canYbz{r28rT$ch7i${3(y=C*CP#P#odWnL zsFOT9dfl}c8LS&lNm_pkx#`TMo5$4=OPnBH?Z#SVxLHJ6nSuI3@o`}9dvpB>Vkk>p zRaI3*S?J66(M9)|o@>C*z2m91{#*Rj9#7$R67{9Y>1i|Yn(sSHt`MuPMTQ&Xajlh$ z*@^5JD1n2Pw(tP_wuh<EjNgHZub{7NQs|pqSxOS7;<|0*85qxP*8oT7i$h;nQAnf( zo~}Z%;x(4i5M?3ckS+LMo(Xnv3nCBwg7zR6`g~c@GPcawT3$oEcAe1WW0SNIx%9AF z;^(x_qzAmx)M<Tn0>^SF*8z@1`ma=s(DPGqWWYc9a+TbM0p8oz_giZmb2(2YdTL~5 zgCZB*>G(R-T^}VxtkwC(9l6zE1hv`v>q8~<P)T#9e+5QuY%kzRRufvSdH??9#Pxk} z^>f>)2Vj7ML0^SZ5JDDU7kK=ZUB4$598A>B(h(9re?;;N0ENkTakjlCdm9AL?~N`O zGp~JT+&slPf}%RynL1}U51Cf3;{Sw8+#jd{M+2CU@W{3*bOBx5OE{rXncmRU7eJrq zThg$Nk0v^0XnME?jA?r%u3j&lrf*Y;1N>o01rKB1wMZl<EOshyO_8HId}4~nCVa3@ zapxzg6l}e6&}@l>l5&1&5S<d-EaU5<_%hn-=0Lqw4Qe>fz4FIQ>>1NaOTMh;rL9!{ zb~TJt!mNy$uDI~Q_Uan;MxexXqRl{B5FC4l*Spos^^D1b|M9KHy=vDCu8x)l%acvP zoD0f<V`duxx0Q_8DSqK=e;jEle9C6sHSbCp5ab`91;L+Ur{WRCZ4>x@CJz3J;7zUM z*3H`e`dMlpP5H-z2q)@X>tj(nWOi^=56okd9<>iaCy<&dy>6awu;nF*7q73lNIJV1 zprURzG%Y&yf_1REJ_>D?n%#l-$|lr$Pf=Jg4|9tGB|;En3n%lI%{I4aI&qV}g-*`j zN1G8)PCkEm(A=1Rwr2O0fLSK2_mzhn6v-J{oG$NS+n;|m8n>V0&u_ghgWK+xvbeH` z+6>A1Nfozk^t*%C)3But^fVMH?I6>=mtqoGw{)qWto*k|tVgKED2EHTiyhSs>=b0r zFKWsRI5h~v6f(`sbq!udd4&xjw-%pwxT)$nOKhFX_iJ$}>z;YA{tR_Es7Gz=?<u)9 zgp-{DXW3GWYdvy$6L<m+z`U3Wq%K&}W-a-jG4V~O1TUoYb-}kPzdM3Xim(0={iLuJ z@b=bzcg%nDr&<R@r=%_0H3xzqC6z@05BFS~-xGE#$Kr*slcAwi)H{^-3Hryd<r@XP zrn54h9x#y9kjOvPA@L041Nq5M-znUCbu+8(RD@5Op*{1xSM!dK^$1axJG|d`e$N~z zvfP~gC%*uES~g-MIQ*|4<dsq<<{19@RYRb|-k;)~tN~Gy&S}UH=gO;#`zP22O1~JP z6%NlS_{P&|Oq@Wd9s+RectTMpxPl+y0YsE^qE&FXw8+-d_h(pjWn}`B8o<hB9Leg; zPmKwk9v`>Z4T$jWdathxf4i@e4|~$(tKi=rfv}x0Cv?pXUB!2tyOF}s_)RK41Z?R; z+djd;Ad>+6sY`1w9V|dj6*M&i?g!!$kOgr>1bq5Cpa}gXRObKmi-V|ygk7B;2yxXQ zo57nxfg#CdH7V}C7~dh2;ycPX*EKZsC85`3|J$!h%gWLP2li>3*E(tJB3YT3KDm6K zIyVvN1Kz}E{_ET7>grcwao4Xx!J5!-k`8VaihrH5LaCqYsRQ;1<Eu2)gt+_4fRU-+ zJ@)}~XWRr_%Ea*i1R0?CE%b||XmK>u;eUAcSD|eQ<e*&K-QEAg$YzL3ThsrTTXgHV zU|S0mPVfZLGTXY}!8@j>r65iwW;gWiP`Yx^l{510{N|J3PzO#P9$m)Q&J*f~78e~F zry@n#T82-8n4>OJt`#sT7N(*kbXp+$1FF*n^Zuyge9Ai@y?O9}pJD8Mot>I{I9Sow zZI5I@!45fw76e^#T$lp;4@g9rf|q4Ptd8UqUMR?cU<i_$Z8yX%Y<7=gW7^;S_C|Og z2s6|wDk+sUo`WvfXuDDTcK5v{D+y;k{2&YXp2=^|9{kK`)z@|}`LAE@On)!ZUjXOu z+MfhWQ2!1r|Ko>)aAh@!JOSI&5e_{J6NhK+Ng@>DaJdiSq)kVwfMDl=!zUheK@mgf zivPz%M?54UpT8g;rO;yI;tCoX8W?jetOiVj-*c_uGi$^6st2=Rej<cJpkMx(YGZx| zRHs#vWjB#=-T1fcf4B?<lKzQ4+<WOI0y&_JU{hr%<Iz9ma1ciUFoe^42p-^E+Y5^J z?NfoIGP?g9xmFG(TZ^6Kn=Tms03)&h$XR-lD*#^s6yAg`y7x6A;2U!+k4h$!8XCmc z`_b%{>HaIwQK5hl0Eoa3=txjttKDwd`gmC4T~bogK1tz7mH7wT(Rq<KTiqDZ1klm* zh<bLY!#hThiws)p*(<>G-og*Rek4R)YU}LqBu<1h2GgH&1&g(~tQf7oJ|WP*mpPbE zi8!v?Cq1;|(74bp`|jFhQOX~8!q8~G1=;~*yy25TOgCK$<~@>kkV7j@W$n{)I^4vh za~0#Kq@V!1G#G&y55QZ#!^KiTDH|X>7VF;Wob()Xu6rluNLL|0wa8uj(PFtBRa%_w zc5;FR-yhEH-)7hU_utG7k75~PC|SwI##U+m8ss{Fm2{B&t#efMdpL6QGnranZ;1R# zUQ<}r!OH)daFXW^7_mX6mp71&`?DJ;nJO^s(2ga0%wvCEq4Np)?(yj?8p5BZ8gRj> zlN4l}ZN%lDtytw$q$jqZKX&C)Roqwn)K0+aur63M9h)exzM-2J$HBv`H_|2A9i?2N z+{VLI+%bN=JQ_L+%gLToGn)NtKf>u13VqXYbV+mWnYFW25sDKzH?ABTGWCPGKLzQl zYxV2JZoCT&x@0AY{tl*qK-*WvbY-vJmW9@+gZB3+Th0sXSxp}EEbMG-DGsYDwC%*b zes4F|1Tsq-cqm@_dwewv(9y-Z@xI1v)Jd_rot``VsW&TiGd)u#LRJgxXWJs%6y_fn zYO!~?$8Q)ZGJC9SNXIh`2`+%tf@|{W6hm1!SJmJN>!HHvl=DAo-cP7LmE+#|s=jJ6 z7MJ&&t`e)h@f75py%wG?g=p8*aEqXr+1X<G_b2}L`@pmFIeK~3tX}+dK4G=9<J4N+ z{gNCRg-fuiEZOl#utfnBz1+NiDvEuMcMb7P^@lGklD)th#dMt?byK=9CP8<K-a?EE ze!Y3Vu9@Pq|H?UQ+c^uvGQ|af1P}h#SBcIB8R98KX?#6N3;+gQ?HiFfX|BN@Mn&Sq z^RqbO3CAUo*y$;H$nw@$rE+$<R$ctOEFse8{@<7;H;cvdMk#2=Uqwy$?ya@ju}MCm z26d~*=_xyX{<w>;fvP&U8=((<yXm{YYrFb3ocmN`Ta)gjSAdb;d0}3dN4KH9cRot| z$r;C=dmQ9a1lPNkDEIv@p9Lc*p75<Rh}s~;CeclPb&XYn-;z94P!%+Su;!iE=9zpQ zy6W)CGeuX-ygb;4zSq){GNwtMj+-#@3DP@SI;?`;UVmDL#5H56M2w>d4M|}R@uAAw zjdJt{(W-m~qYqeXNiP$qF+AR1gOt;<lXs(S+3iL81$~f13^E0r-sW{KFuDy9M6j*4 zZ$5#o&ZS`!xJ{bmIN8|Z9_afYHMfZl*1n*qS2*7ndH;6gKJoF#D5{HgAB;coX3lLX zd0FoD0%(yn+V0S==`?m7?Fw!^TPS<pDOw}vI*j)8`K@1#^bZvkF+wQ4e({9qNyJut zWm00zbe5Wir%0aLT`jS(qWfIxK}0;wJxIPiF-ql10@CA0(}KApMv2F=9e1PoSo_PC zpH=p9(N>D@fM*hQH!L|NrHTvEHAfOb{9jB|^qp%6Wo#i{om&HBFztuCdwW3)wPU$9 zgidSw<+l@&QXJmA`h01yU_MHyG;{b~@kiS!UQ30D2NUq#ut-hO6&H0yS_50juo7{4 zklW|n=t!oKr@K}o;!LajtizqGoi<EQ$qT<vo2X>u4mzOkfNRrvFv?lC$~bG4l{`n; zy+%V8C>;A|h3@kq1X%*344?7+6cipz5*b(h2;cdd{iZn*hHCuk(a^u9Uoy3=ish-e zW<V4Xrv1ogEnBuT3>pjD=wQFdy)w5=yKeEB@@@=|(7Wic_7cjfmiV%BanwmFD5On5 zFMaQ83^44mLxFz&jq93+pPt?G1+0`CR&uNbZzhn`an;6prHv^pbUwPR2Ipp5SU<qf z#BNfgybLXV`2G0c+ANnb(BRdEMwwt57W5=mM*Yo>FNwe(?EvW-eFWRAJvTF8wrF@b zLPq!W&izI@&GkW&S|ocjXHC>5&>+lJi;}87SP$1x<VxP*-nufFVz)@sD`M_8-E1Px zH^a4Y;V74p66yN{F(0V&5rV?x+T8NCouL!=wi<cRQmGjkf)n=N0TU9r`Cv)pOo~zM z-iO-3$T^wf{a)*1*o4#~AEdfv;3m?tn}x`^nT4kU#V3L5=Tw``yEWWVCna3k)1y^4 zo`SC3!Np-)g6YZT!FEbTlC(_X+p4O7p+S$^4*VtC)m!O$q>6EN)*;hmViP*@30DyJ zj0{;c!W;d}XWnOX60=6a-fG5JN1s@^zEj9o{A1Rzm9UOb+~3*r=E^}Uspo>iThd%H z3!UKTw<SmGalEusTAb;1gl@zNd2ZC3l`=YvF>238R1haA?9$WO94a<Jy?;hKr$r1s zv~}jm>9jWy4AMD?c_lmk%h5C|nP~2oPH?8C(Q&j?;bD^X)TVQtrD&&$U5>7cUHa(4 zn$vdySx0mH`uh<Y8#ZgmV!Z@sh$?YUpO}B$wlz=!pIoj=NQpL6d<Tp=%@cU{_)~%m z4HSA`veT_1u=6;F0Xizxukr&ROsesgTKwrG&vM?yQF6jbi)h&Vl`Ssj0mA*y7xljq z&!?0r$rQW1lCU+b$uw$1mtxEdr`X4SflaU6T!>fvrD}q@fitNuC=0G4<91x4WTr2> z@#ds>?OWfUD}Zv`0(BNv{r*nCeb3n$%}-Q88Y;&GKSA8FpKy-cBK?Wi4JK?gf-~jn zyRrnt+hwkZ`(4-CX_-9fjyQvU_QroqPdW1{MSP5vYULb4ZUnZ0SB@V2IFaufuS@)2 zcU(DJtmr3$NH?(LEkAvInyX`$bo?(^pi^m;-W1G_bK=dLvBSN@2e${pm20*}5_AjT zu4!0`x{9J}7J*haX{Vms`_yWTh{Wf5)rpNx2KB~^BF?iMQwYk)PRY>~^6wSvF8J0= zT5&Bug=^!uq=K0`ZEthr`z;acMt?Ron%k$^wcM9GbQPavff41)iw6jZ1vTr}YC*cS z^j_HTa4l;Uig0&oqO+4^T_1?z{ZIl&u?Bg$VN*Aq{ZPYBSyszCMNRbTReBdDJbu`? zJbgQBs6A(s<x<+2-tC>p6>)(g_ObS*Q$g--id=DV-rY#{KtlPLqp(VH0f@3X+k+|h zs2h<B$hE%fYWddCu)PT+tz`uMWidYaaQ9O8@=_0M1NHXNnu>T&jsJ(aPdp9QpW4EA z0K`V9-3+E%*QQ!A{G6a#K|nVZUeRm*>lAzLQjL&q3LE>}m6ki>vBR0*TJhSvW+hyL zb}CBTa=S{UBETYYQTrTwg|Wp=(a1$EJyLF<*7=DKVL}3BXn8KfxzP-k3OGGJs}8OF zE{RqO?}H6{E##Sg^g-~v*cTO3E5vU2cyvXASLiK#>UIlKDMi5kBvC5uBjS6As)}*x zxLIJ2yVv`NTy<7sRyq4Os?g|^{4>!HO4SVGqjD;ZNP+e?jU&PFaSF1FOH)#M9Tl1G zeXxylrmT%0{jiH14f^qlug8$ru&2h3Q@u!+tZRE}lX>)4Z_8@-99|nSkXR^h&V2J` zw716niKznSQhP2MFYOlsUrt;{u?Nlb`3zDO(}gxC6lz^{Ga~uh2sjth+@jQJB_o=Y zp+R0u?<H2l1$L0oXJiQXTWU#H0=Gd7!eN5dWp8@}z9yb<YRc+vCev|tcTVjVW43|Q zIl`*6ZQoe>&RW1=mO#k`L@iQnfOFKWw~o6g(7(@L!%DWoc}Xl}yJ?C2rxm%tca`wn zL9b=D#StT#RsLwsc&ofF;;@a|V=4Cjt3fmm+}6rm9UbG@uU*j#&c>B;`%vn8O=56p zGI-p4z3#QnVm=r4wX^;XojGVGb~7q&&DCwO&Yy7jF^LANT}l}epP^OHc}3jpCM=m} z_N;XggBs^}9Zq-2<%hTrtsJbElr7(ka8qWbY<W<7v+Srg8+P3Z&7D;B1{HB451Lk* zskk6ct|B%M``nzmn#s>GmZWllOg`aNTx!eu=8%S;Uyg4{XTOn}3I}UU<vt`krEU26 z@Q{^ZP5`}B#$Jm2*$T^pxkCT)x#SXSoW5IHm-9L<n)V)ko_}7}&4sNu+?z#hy4knz z>sA8~{ql;*(%vb&)S1nXAMbW=i{yU{o>(eZa8(+4^)8HmJWzP)Yxlm~Vf?(vlwxc! zU%LWnW29j@P$DMRDq3HwF{()4mC_n8hzUo}R4e+q<|0h&#^7V@&Kr$39bvlZ=CQp| z%`CVEhhp5T#D&P|8K>0m2T;h&XTIfk$CRWi^j}Qh)%hGtjU7_5)HP6xCXOq(1mD(f zva*D8=dz3R4^VoJTlZNkj|m*M?w!J2U8AUyD0ia^qQ+i@sLRX3+;BmDwA0<~BWpAX z7i<Hud&n`i<G_$IHj++fp4@!AiP#=Ox>r0<l=%+%g{*k1y&ikq83s%g08zmLR%+}4 z4<ZaOwL`e!gZl@3js>~~ku0PWQpDL<PpowIM@qA3utEl+<7m4(2oPQQ`XK0Ae$wXm zG@XU(Nd=KHQK4>Y#U><P@gG4XWWe2)l~@!8jW!Xljd&1cB^{&Kz~6?DFb+CoDjidU z!qvHN#@JHy@VxhtvA$*23Q<eVAuBnsDoqhLMFKTyrHt6vrus35M}^o&^__H|O{n+f zz-Uh$w5Y#j;&{2nl3#S^gMw?fu*G27sFd-liZXpf_xlEBmX@3T>W*%*MO7^<{(S<` zd5MlsEpe!;(!s*ZFMHic>#v-&EWXJ~d1WXjwh<9;VWJZ)a^cde6S(Vv+$T?*jm#=R zAJTtop(hm5W3}hNMpe??^W`hRgdc}<ekx73=%B@{pCvzaujv_KPb75KxvL_k=u7#- zJsfFjRwf%2-{Mq5?&u?`bX1$Itl-t8=`q)mQvqoIsEGUVPDXj9#coBKp8Ry_6G8gX zA?wy@ao>2|%7(G(bmP+knCVLIJSkVw@HqXRbJpqU?{b;w%{Hpl9#cW?UV=xsT>Bd5 zY$R5`QZp)y_=x23ZW4EOvb<1H%=dKGE!&bQrQuGwfVX$NF1C5rsN#JCR_rvFhmH>Q zR$x)|4=Ano@w))IsH4#_QDZwhfxAI5C|U##1$R=hMn!3aOFoQb&n*{+i3(r1e$y+= zC|@n+u-<0Er=B#;_b)w-bp3#r=^q(l`Oxjqxc5ns`!np+QK_nHtnq|D8<I|l6q!fr z|Ah%#^N%<apRM04_r|zn>{iRP-Xx9?+*0}SPKx1mdr4NVJ29eCt5rTSrGNRCU+J9y z$7oSk8xMM*Y18AEy>J3(HM*W6G0J%{;=yyZj!&jm>gP0a;qKm2dhZpxeJ*XXsCFJ~ z-F&4&%&`KY_Qik$#}pfp7Nvi)=7c!=bXv~at?Zo54(usE^3|Z~r$xd9Dd@hRld&QO z_<i5U<07**$4!digA1Nj-zso*%t70+HOjPP(Ql246JAvg-`40{N^=U>qLnPy-~ejd zPxk@j)b_>LuBnf=t+(Z#m+_xXNo?ji|95i9!0gm!3H3vVtl45>7k>Rf<Lmmt4mpKl zRkneEEH?VqcwEpi1OgiMGzS?ONf`!~3JO;o$t0j#hbHbQWEf}8BpW|}A*{@Px8toM z+Qj0rgMb?^`@tuEciTIo^fL{mVRug_TAh#f&i59J+Pqn*ZgC_kDi{+>eP=syPWY3T zb#Vc~;1O4gi>yTmtlPjc>J0y-!8Df2E&1TG{Dga2=fM%CmMX@7{#q>2^7#{Xr*!-T z&eE?F_fpr*r1T|NI3Lvp)*_c2_{pKRtYW!YodwsU-z`pgRv`k<^?ylDWHBq+I>{Z< zEc3WQ?oVsIHD7Shicv?2oHJ$AEqdr&th&>UIuXxnbxD+0M|qn1{A!x7)okaEA_d5i z$>e_af}hUDRbE*oncn6Owz<mgZ{O>#arVdz_YjCLtoDo}vR<`Eh#5)3EJS&Fv&w_& z<U+D~>7@Y+P1iS2B)3Q@SAGH8i051z9mc5by*bSlafiNvfXIA<(0wdb0mg%op&$)t zfn~C=0NLQueJlS0Ns&L?2-#d2r7$bG!)vNm#_B38oBikChW8(EZ8>k4*Z;1HaMBu} zhCr;n7(|Fh4|_@d>C`mEd(VK=KJWUyx>t!f@c^6Y>1JznlCP2F;|t<{O?oc{R8A*! z#p%B|oY_+U=wPm(Zflf>I63cie#L9SWC3i@iioA04;_OOn5an*tc*7*wwefBPa8@M z<tw^EMkpSfint!a8NnSd_bt}_?vE|!F>^=b*TOQjMZ@9Jcgv?(`!(Cwy5syd5G1MY zl46+><v*!8iZCbn#}0hCCM$?>V&LLxfVo48?kDr4`P+Pm(nw^!NzrOj#klXd(GIuX zY0Koqx0a;Nm=jeVD&l8)jmR^VV=e)@GKP+BrQ@Z#vHZ0v4NJKTH+UL6s6puhRMn=y zE5qHKnB)`esP|j)A?807%$*#i>=6?)UWFj=!tnMUh)GBk0I;@g7YNE;YFT|Ls5?6( zir~$1F$&9UpW;xHt{aV1Pq8(e$l<+Zpi9=t5IQNVPvZ5j%MXr<dc8Io(UwXG5JUYq z)p3pGPBu87@JD`E&v|&8AICx821Oy)OBCWQCqbid5hGJE9p~*vE^)>CfJ`^yyV$HS z+-)4?iB(#7d;p0Y><ywczr)-*9r5hNn6Q>pxJ$%(wFzknqe9b|STx&wRg#oc&^vR! z?81Q*){+s`kLxx~!SXAkUiRktdyIw1sGXL(=xz7BPdwpH5<8#yCrd2hONrLwrV_`M z$TQpL`UF(`0=g+zCM)#L)K1FDD68bU!}(W|3Wit0hI^VDonJ%-3h0zRIOwP6Mxh-5 z@dA}9%~E7njrM<L4Ti1b@*o^;7X(`M2gKp8IGo(>7bSM`v0t2ULb%^~5%oB!6ql4y zR*<<e<{+V!`EZ)|KH`b_=MpW}IAa6ck}7a8RIJRfYMldRjq;o3Zi`+1Np;Fz#H>_* zHvh9dA5MrAR6clbVf1EUalPbTW(jSboWC@VY^k~#pL46&JS(~8jtvO0!BYt}iJD5Y z`=*UV%Hg#l+#W6gIrPYPQ93gk$KAS4#6;=i)DqgtYkJ#!#&_oSQN`P(nOPHxA_s)H zgyIx(bAH9Hq$M61ygmD@t~X@0C%GM!5&P7CZ92?qYBu5om+_p4QId#GxS6tenUT4; zZV{(OcUg1XqSA#lxc3TMe~5Yx%q@`H`n&r;{SL|cG?!s|F3j&VSA^@3g~vcXf350A zOE=+Z{{aVU3Vo(LPCA2}vHtHsJ0tvV0<sfFko6`82Bk0cyy<(fCa6c==lZ+Z3dTlh zR_IqcCncuP{>9CY%rg@<TOX^NPhu%NJ8Iq3-jJ;mY2NomnzCEG^)8`aHhinDi&7~r zyLy7Pps`GEwyZvsrz0alH=AN>m%l_)>gi3TO|S86l|Jc>aSXc7|Ks^QGjGJ%T0UK; z`3I>ZzdG+)o*qy)*N7I}`nh(BZLF{3<cz9crFz5l#X8#J@gSbPKl9Vzs4ZHMFO}9p zq^Xj3a5BnS1Wk*EG^{cwe)G*;6)yeKWBxUW;esj8k{Ds`xggtB3*PnHOM;Qb`DOe^ zlQ9Ep@=^hhlTJ)uI`2ZNULqlbJXRERb2Z<soq9u@T~iVw6UM2;5Nmz-^e}CQ0)uN` zr1zX8TKatQa>PxT%aoCSW}jd#YFA|-i<q9b$sT6?dur-V@LJ(ynOvEr@`O_p?}u!Y z<2n6@?;<j%RGWqKPm6XI0IGh~QX`S?NhSUU@t($7Moz|Vhw1`3JNzU2wVxNq*_Z1{ zCl)dpHveU!7<-j66&NRk=RI|g4;|_FcGwOEo6pU~o6LXnGhKbkcbpjUG|a*0%$Dx# zgBB-FB%5%m+H6h^I9O7xSThhg`G~qhrrIXuW?Qy#ly6T*G(U0XbDKOV;)=GT6>52% zvPMjC6j(fV`kU&S$MaC}>yN5A+5r^YgARsQvs8p9<rary9*d7whM5z)M%IeMW$u{X zRx5%>=FWsx^&27n2gXu)E&{yvl3UJ{DE)=@m8KA4%7|EU*R=ESk>B|ZPP?A;4wu|} z0KDRra!M=y6{7yvT^7}<Lw2)mPj5YY$S<7zN<z5(&k?KhFMh@PI9V*b8`uPrHsF!- zG|F-|l1nPcsp0xPi(v-5Lc><Xh#+Y6uoE4IJD|yauB)@dUux5^pp&)ON~LqmAd^=l zLBCNiPa<J~_5;RK81RAA94VOpCM=d!Wb0-KX%HJs^uxi@UENmM>D;&=spI(;eq!V% z|AT)a%8O5EM0R)`I2KrUfz)7{g}qf964u*zE7}*&6SUDB6sfGVCC!d`H!QwYqxVFw zWgl8jIey<M=fxmyRvydDDR%M*Pss9MDn;*Gxnj>}N@{bYjd<~dR}=5AE$6%<Em@&c z!U++%ohIV#H+aa8sI<#u{Iy$OqB`G6cepxM1d-H*5j^N;%>+9wQvEZlng)xFl5fVi z3d`yygrrU!od@6KKSdv)tZ8kmZML`FzyG$j*jxER%+MSW>*qM}Y|A&R;$%BwO!oev zS%N{2f$PcXa}zH>spP;iz=v$%!D#@ycIdy*q;qp~3|@CzQ05I)-;5C6<Cg>l1>%!V zEOky@H+38R5R;!4b6CYqN<{o4lyqriCGaB3O!E?ThN@Z&I?mO4&9-$u*I!7U)n}R; z$ewDL=3QtwB!LPPcUx0eube(!7wBIzGO|$PZQr78aW&dO&28BVR%USMi%96nf)_D| zV@d0i?p^q5i}M$_eb&2O;izA4j6X!qG&-ea`c}^I!W1zFTmy8A%@pzB%I<dEj{f~I zu3s`;ZES2-c;_q!6_H6r?V;qHUct;Ek{^3r3d>(TC^?>_N0Bm%Duf~<Yo#?m7OJ6q z(bV<v?WvRB5tFGibB6<CRglzL(LdzymVAHY4-1HkikBIWN6$nmlHQMt*NP>4TBj+| zxL2&qttkNCp)iznDS>BNZq|8<j<HkS?(sny>;LpH$e(;5*A^hpvJ(|iDCHeFrWT~l z-<wQHbE#iWs?pey;vTQua?LHe(IiCfvzo=t%UYi{Ej|4@?nKo>b#C$47OhN+ZH#St zk%J4z`=(4zM?B7NT3SNDJUQ+*O<vzs$<ibsYqkPby43m@Usl)EzAUkYa#M^7a?kY4 z;OX^6*)9c6Su}BYsAV;SbmlrsOO(bB9<wo&32uUTx+D1J32I1nl-sf_581mrfa+DT zq%%7oa}q^w^LcrzF%IEv3XfMnnimBRW$HzRdomJ_Ft@#S0Asuxac=|;tC*NrnFJ=7 zIpCKUILK`QyOS1@C>9rENLH|sxwe*bw<O|-+hd`G{ns=18CSq8enm#<;)EsARkR)W zlPA0#*JsKV-@KjJoasNwmN+-{(ahqsOF_b@0KQS_wy($q)fx81NWvlYD#x)C9lY_S zdd;CG<&TWD9ux^wFtLD$cBaq$pF^XXiliS=n8NM!*|x4_zoA%S*7zo(+&KFnfIv}< zJy*RJX)>+v8yC=;H?tZklj{j^PQy<T>*er|j=8j4IFCD>iY*jf6DNO|pYGEC37;qD zx{`M6@PSgtxAUy^TtkqX#&M+rlLL;%6(P(cb|5m5{8F#{eUf37|7!f^zLlI|OAXAC zRK>^T7JpXiS)S1FiT^x$c(eVf6ndj+h};Ahs6nr*^4u+*PbMC)L1X}c7@m5KUo_=} z#8OP+4gN0!V_E)~yl?(f3b$yZH6Du7_!`W*lYPr3WoDkZPW0KBMK4%$ZPL%R|F7<@ zGpwnsTQh@gL<T_-21c=ipaB$!Qb&{`3K0<jsfzTbAib)CLlFTL3y2^{73n=Rr74K? z-V}(T1q2BZLg22QL><4m-+k`A&;51tLpjOG$v$VVz1G@$zwf&iqGo!`1_jjK%!btT zbPu~erj}`@-&QyMiu6t~|NP!m$evCC-H;7E1muto!)@lACikymzhsYd`d_+v|9Mj0 z;eJ~K+aO_H!tR2?icLJOFrG!Yw+vl8oiq^`Tug8YR<O=64r0?aE}MQb_p;aC>R~2N z?G!8?NuMmP@2zra_w+Jay^l*Tat}+*a=(`otxkn-{NA}Lrw*C>-BKA_E8QB(=BjZ? zj~-3!-dI~*pW#gyxY--#pknZlq{RI@;Qu#<$2c7loyU<!Xf4Xber1}<YwCzgYEb^r zR_DbEjig5hf`&P1SL3S!ynN*;k7Ro79|$y-#2FfK3omzAs!@h4_%_6iDwNGfh_;<w zD^lMWVzj595GU<aH0?1}tDqjWqejvAJim_p+_A8w!|ZBY_RBrk@xfV|du@Y*RpD*f ztcApuxx6CTqm$29C_L6&d!zD1<b7w{KK7P0i@)pfp5HW~uqu$)pr92+CJ2<|E!H)^ zC?;_D77&D8h>6#hKi||!(i#-1J_ZmQ#I#GTz?LsGL{iHaqRN4B$=sef`8Y!Xu>zX) ziHZIm6xdOYjnKgz#u7f@a)STRnWnb0i>`r%_c*WhRW7~ln|e34W6m2Imo;2)G54q( zd7vDm)pOG(y)5-bsyF%g>T=lT&dnZneA>HFT2B6%t#vLcKu7L*b62qSbE85G?*=_W zlc|E<^j&E=N<Gn@+q=cQE^(K^glRA7c|GZTbbdywi)|T#x7zh~P&>!`d5m5sX8*pK zK>1=Ga;UC%uIi<L{di&06PMZuTKZx0(lFY-+cCXkyMjsQy_YoxGYB*8UM`b8&SQyX zK3VGSt1AxM3zU%LCL$_>hnz|b7G$|Nsc!%&c=0oU6$loO(UYhsh5~K~p-VP<@1-YM z`n+}Z#8o*>a*8lX<BD!|S#5P@YLa2dvpWqMjuU^``DtbkJTNeck+3D&8zrRo54TLX zhzTvu$ZhLAOK{5+)>d!m7ft2Maamj*GmGg9nOX1;8h2jE#p{`su@4xB$3D<$!U%pm znhtv-06mIESoAGW;s)VI019UVFe0j?OCRFUqSkrrOZ6gRdh@QpRPfck(wrQhb;rCp zhsTuxj1}aHU=<#-Zv=A)&(n+Pth5R--n_z2*fqTm<>>&G(+zt<qOEoX17|2J{Z#*Q z)wS!j8U~i;v4Y8hKDxajOhkF`_TA{$Hf*{g3jg^H6v3Xuo5Uz~ioz%ITr_K_d253a zfE>C9q4Lcma5)?nE9nJA3k_qiz#A6(04SC8WoUGb!j0j_DSb`~Cpf$(!Y7xV8fg-8 zbA7v<1@NY|`EyeWOVM+;VgEL^6S0ZV7g&^=5bZB~mr6FF)RSV){VNqwLN}E<UnE>Q zZy=lPvyyH2Ah8}pCA6idcCf}@`fuWXEUSlcOJb;q+)1=yyJt}8ppg-ID)M=TT24;u zSk20(n*9su#=!4P@R1Oj3G+e3R(SF1A>qdQ`uc9iRh2*x(^9zOw)Pemu!RuyH5R)l zIb#nPOWIT%Lej8^#Q_2+tPiOK-1_<7z`U#h1h^MZryl^b3lT_10S07MfB%4$mzOO4 z{Y>73!p>K(6kXT=Kab?)*>k$e>9iP#UUU*$R|sQE4fur02N+u{meAWe+pe-_4OW(w z&qPl3<ca7%0ZA9su5Yj)zzlBl+E2%!Rcz-6^8h4OKjFfbpUml;-U^}L65Lb~y=Bh; zcZeDjAjZVO{h7`rgx-MqCvj6huxRN#ioe?~tO`$Pg;X(4$_8xL51cU&PC)uwNQ*1d zk6~`YS40<IqQ0(3?ke;4EC%?xRz|d}`$41)`HRVB1IIUe5EMJ5)95>DZ?}hd$ybAm zm4W8PA0OKK2L-5qqreZWd_e~(q6Yvklg+p23|c=<XGGwe?8@4_>Bc0~ghr2igUt8d zQU>W%`A3LSXnSNinu$@WJOy+1#7M+NWKll1?b5Gw+!-8HYJ8i_qt94k*?*Wbptr;} z6mw4Vo#d{=+S{hJw4?mBB9ZSOU5xN?`?To{3KP`7m#nPz0aZ{)pUzKtcDaqQ0)Oi! zgiEFE2YIM+lT<uZt3=djTlc?P#_muOq2Bv}8`k=#Bd7t&mB3*F7}*4d<2kI|O2EcE z{q5>xdQ<?u^rf#CUqc`DEU{_X%+;JvKxlOSCzx+Y`Ipz|!HMlh<PQafn7x96tw?ng z#0-A+GwRJYI|=uhtOmA(IJ?{kw~OSW@@28u2X>6OFO9lff!3;!V7xTM1AROR%xYAF z-be;$p7TW<V(T$QBX<}8RE3p=##V)u&k6wS+i2Ru+k5bDJrAxZ+-E)~W{;odr;O$H z@}7ICu837}UQh1?OFXO%Z5<<s`nr$jsp0|zB2=|ll#kBkG!Mchq8Pkdzp1WbXPcjS zS6{i}n3EXU|CYFBG_N6fKVMu-e;WO7)ys(mO#9zu2NXIyIR`?T?!k3pTu|es)jZJ0 z<xq@3tj<MS_hzN22g0!#tCe3OENxmj$>PRFjxpCWZ_oqt3#VhdUrkQhx=g5qZ^(wE z*S1d2?*cc^UMzln%bo`YcYt+bxBj?^rP9vV0Ced&rP**~ap~a@zlko{^Uh3c)v?8B zCdK!4-Ho>wz)r|H595@bH^K?WcB%#(Cjt|7H(b)2WPT1`4-m)$6V#t>YR=`+j7auj z;IB?Gp>L7E8*iK5={Cp_xQ|6Us45sxC}pave$fovcW3(44Eu(=CD$F79>xn)%^x2W zUHFw(O7p0DlaB;;9r#GIR6S%yu{DnlS!DZSJ&P5_gIRobCsmydK)to8VtX*#-SE?W z4kzrxAI_dEpF3$~W>)R5YVux5@yhOg^2zH?n0yOVEU-`tXdr2WT#t_!W5M|-zgsne zuFR_pc*meICB(ov&<N~UC@U)QHm<G)rh@shsu3d>>mmzFKfPkpQ&ipko-nQD=d*=p z_wZ*4`*791Ms32*9lPZP<6=UV&l$6A2pZKN-ttD^N^lk4hr9Q;;ZLen%iWDW_xs~A z<UHqXe;#!eH@YP+Rg*un)jLdhFNd}4GcC^PpfP7tTVt~loBHk``K*h>vkSf|JH9*I zPU97_afZ%(;QN`Vfw@FzTy_oU*5+(q(AF`OyxjT{-A1kr6MZ|sssHtcw@Ta3FrI#; z2>`IG$L5qTSeLHb-`LoQkjs9W;!G4xRF(I?AC=Wt`HLV~p+_A$ViqDn5dR)V%Q8_Y zICbbCl;1@K_LgZ>pdW#g%a$GBE)X7Oj7VK0BMsmL)ZHh3{Fu#-9Gl-!CSUg(;u(F5 zB|i2?H~Iry`I}D#Cv^48rw@n<ECFIO^ON25h<@`C;O5RUQUc`;ru;*aMs(fJ@6fgX z<~{wwxqtd-u<9`yB;WX}@V6v@uJ1B%M7&pbbpR;#p2Pbt-Gt}#CaX&V`WZrFE)XVY zJ?{4Q7*sIzLOwKuZ0y36BJ#U{Wj@p3%{4XE2X1|ZK5x8M1dLO&`uL5MAX)=!2In;Y zJgs!LPNWrDn%{U4mD8#adI*{dLpv5`Gnqi<E<naI4U~N$RQ)sILa1l7w6nGM5f0T~ zbHJD`AsRN|&Vdp=DJ9HMq<B`S8NZQ{+(`s5mF^(QqPu0e;Q$Lu6ZKz<QPJsh@EOy? z-umugb9%q`CF~Q?N!XdN*Z=D_*@vrl{vUKCOAE@skn%Dl20eK;I+4-Q3}CSdco{sP z>yyA=nXNcpVD_o*=ebr~65#BU2m}J^gaau1+fe)^>jT0{19mUlO=Z8+2d3*x3siGe z3a96zVxx!;7fB`Xu8T$tHF#y3Lll#wd>#ifHar#KA%4xVbW6y8u6(FD;}3S=WS?Lf zk-i0eD%CDjU5XS0)Es8&i2!8|Utgu6u5O0adW!;V<wba;k`(BHXctmXgz|}P(?O=< zOv6k)t@rHmJyynsb_QrLNM^XJn_VmN;ig96uv}C*>norIHJ^Z#(^`{o7F2l{N$5w{ zXje%Ad$~2sv^Kjc2MNZ7M4~YT3LF%Rp?;yY47a7>-eQ=1P^>c}NGo)NDyN151akwG zcMCAe3pCq7Qb8$YXZ$vXMI7cfHDrR&VTRAhJiWvTs`lTAstnVUVZ`&B%}8SerUejm zbG*G9Y>en*%_L0emPn|pp*tZWMVKCS(HLLLvF+`B3J8)gqi(Tj?d=5MVcY8U1shZy zGadf`i3jo{nU8jAa8|wW03J4dBM0Ja;6wL==1XThU?##OR`?l71IKDhe1LBy@Fb9Q z+??`P*gUXBu0KSf+hZhL1;OM|M~p_O7J7mNOXuCG>tBvLFjW~WD?zKGks4J&SEV>v zEwJX}*^%i3Oic&7F3>JVC21Q5SLMS9@iP)$5xWe4gJ8WTnQsyxxqE!{wna=KJsxnj z?Gg)^ZTzhjY3b?a6cC+iLF#rn7L7MaNU;LcvC{rUx&#hGRPhi|CiBCglarESe57qq zb>aOpw+GHJIw6)7F(`m}JJw^9VV5#wC_aXHEbT1JmWItKEKL4Su6P<ButQ=~RzUJ) zo-4ncaE(Iret1>v-G^h-!ij%L|L1qCRD9jKea|x{KrKaM>Sfga>BJRs`Ld3b|M<IT znN975h|CF<>^fM-Igk4Stgd~I*XE!4DLu_avIBs4tX)wR_3W;15iyzxI_pp8Qfwyj zKZ%O6k!}=2&Dn4rTqb#+qRNgbGReVLCaPuk@mPwwgNGLCGQXo>o{0Ieg~)Gj_T&Qk z2JfnHH$*f6qR6<qGdc@YE6}`^Yx--f{>GwU^diWQj7d5E7e0*~*;%r;x3Lh}D}&jr zCq9q|2-vd`^d|ofY8nVC3py;9_+>!r1^p<Z0nmM>*;nU@-mMTywq+zE=m^Ae7&$oo z;A=wuKRW;+G<dyNLQEQ@Aqu!-RQVk#_AuJ2&6p+q%}3`I7CwX|J4b)A;4obTgAtb# zBO9=vV20Pd20b<Uq+Dmx0y9rvo%~l<{{xK~dg3LI*7!$;>N|?x3`E8zhST5$R4v_e z<;P?Vf}#>#XU^}tS_=|JL8@@-1)Ti&uW~mxko<$(e3vPsw<?OE1L;o&{bREEtW-Nw zmJJSX{mdMX=(DCZ|BHj0S@Z<uQ`}dXBGA`N1SLXLN|}+mj3X%|B-2Fr!Ipsdb>6_> z6lf)NADGvF%mEGXJGw<Y9SK4KID+Ff#9QIsKTyNyEU_Tk+iD*adFY}{$b&IGW%$34 zJ;c6TRO|2iCV&FcHi$>h#5TnvY!g^xw@-}^4u?bO7;v5-3Dni;1J%$o+6>hbNOf|d zuhk30Wn@HzP~@sOjmmj$p{(b8hGZ@1!$VmgE{v`~e~GRCI@O>bDqmyxSo(qGjsL|# zC0(SEC3zUJW!}HMVdd;BNrE|l!@GBMm9$Y;q>W_<OA3Hy5TzDa8=fd(8crB_rx<n} z5_Y+|^kv4@9V`MCKFm<k!ZNB8j&}iynC${G*;z<8wAzag!m2@7uu0#mSFb!S)1%X( z;kJjtu?AInv)lWHf&@CYD5ar3Vp;44@;D6zbsg>EJ8n$$XY9DPiT%dOetf}4pJh{h z`tPWahjf6&lEVM}wC$-pwJ6_+uWQRj=WMtG=F-tO2!Q~lBI7q<0fIT+0$@y7PpO#L zw+$P>5>P$JQba)x05l|eoQ@SxniTd?<zG=gMXyiWpAAAhZL2|FiyjV_Lma*Pg}doF zhc@u3RAGWx@3WQvgvIdiFnxH(CfoeI^KQW-_&zQ+i2Uuduj8U(wLS#fud4!0>YO*m z|2o{J+jEICOds~7nd(C<#RrBokuQNC3bfw8fB&U|Oc0&O^<D9)<$+uFQ&Y+|tcsm| zhDUb;$~IE<puE(wci);3xZ{?kr4k06%COntTG0hZP;cJ<H0$*YN`+&Et7UTxMxsiW z$r-pPGp(dZIhV<<#aTq3*cB>!2XZAY^i<F8yY7gf1`V4x51nogTz}o{Hd^Ro(cbs* z5kU%H(VMuJ`IMX3$Ui%Qh-;wrB$yB?di>GBJRhAKD@>3@1u^&U&3jbSvNtH&7u7lj zF_fHTEMwrMo)Z=0(Rz9&Bw$P_%>l&Gx;boWnUOlzP*y6HGaUtLQ;#sFJIL=P4VIT% z9W;v|0nsz6=3u;fu3bStB$3uWvA_4alJYvG>?lC7RG^IrD(IC9WT=#2W;`SuVKFDo zFpKY!2#GG%uuWFC6_dEd-o?mJ72ox4=w1zw$-7_hlZIwA_Z$MuAV(m{wgWRi0U8W8 zR5@?vyd}~FdC>Xuw$D0=ib>OB8ojjlYZYGKR!?ttdTQ_?x7BF}S5Ukte~r`AcFNUK zryV^KQEIYW1H)t|jBQ^!{k;#JzO+uPm%|#OBF*KB!NtAl?+pqwU3eTZo09@3<_rSF zy>p!~w>pYGp~&Y+!X6ZY-x^+Skmo6;J((gDZ43>aq53~zn~7Ax+efp}K5gwPI=$lb zRIf!Rf~Z(f&g$xjX`}wyH&)c7hL>&J-$Q~SHAXxtYHK}Yn8E6JWA>%2(9zIT>GE^s z+wZgfwWjsPrsOooPXnQ>(Zb~{9BMIb4P|Dhsn5<W-jBtl4yJeV;8MSE<^9I0;ob6) za^l7O0c}Ge64}k-beRF4AX)zPkx8EQYX~yB+B|!2(yrIU&YkF=()T!`Axz<XF@J(w zhIei$$X)cJ@jm2NzVZ$`K@1L}IRsFQOUfDs)`|}Z2f55Fd($S22Q$@WaiuOZ1*3oQ z2Rfx{;=hzw?tZs$u{^pUnDRlHb~aa}aXlCD<kA6o6*wX5@cp+dTZnI!iM39RayqRF zc|DkA-y%a>)5rS!rE5W2?@WPVZue-(t>JT%or=E?S{5{0xXhK4d#Z^_b29vSB4t?9 zBRxthS<Mz&C8E;qI8hAdNhfBp!<?v9N-@vHFE38;=NE-JJst2Y)+Z`1%kQ@)8;p*~ z8BQAn#Qg-7<0qdxeE<=`v=M{XgRRkzp-*Z(J>s37e$m@n?2eJ}z`0Hs1Ptr87d4H0 z77r9{oajk5gezkk{gtQ3d}MiMXY4KWd>!VBn(Dz=oE<R|o;Z$KI4_9Oq&%pXD$g_I zUDQ0$Ip$fciFZp68E6f8=YMXIa#W5d&Ud<|uv`=0RnB<V{x%g$H3+b~MKQy^^FM4~ zD$6q@nK1+n6X}=c8TWRbBD=mfC%!WGTD$FIqrV4H5euf`flTFhQe^emcr@FK-1^UN z7vCgmNJa%?l^=J#ShajC-p}Ari2OtB9am-TJQ3H^X6}OLrrGnumsm^tiS;t&j`+iY zQF%6P3wD>{MFSdoB3S>*VkbC9*tJ>NKhsR=e)G9n+^Qp8(?Wk)Uz^0YTDaa<#74qo zpGyT<g0G~uI?tRm&OXp<_aNq2Lysiis6^|DL9X#=&;D`!)Tz6XdHi$o``*1UC-E7+ z?NV*`KC|S#D`YXiL(4X=CB0QXMQ^ahDyGfyeODUJTH^BLxtO+5r_#a2XGXVO7ZY&S z2@d-Tf}NdnaH-e<s}4c!?t-gMkJ~2XQ#Ab&TW2&1;Kb0o(lm^=SxlQ6{^;mkDbteH z(N|T<QpG0y_IM-8iD;j-_%QjDsKJ-xtLqkD;Kr{$KI?-k=mC()pG{VScNB~dxtm~y zaCfE_^^2r0JP)->FpLPAuJrIIl?!=SWZv7IY?yO=g1*+-Uyki9SEe>wdKYOJmz#SV z?;_8~cS{`ygioV%^O)59q9pG<rw{Z+$m8)e%ltB*QGFSj!RURX@+29%&g2-I`6p9z z2E2bRk1DqHmYc^MC8a%&nwZKp{(W=9d`2tfYJ2UA*3R5kmxdZ?ZEG`r8PcFk|B8pS zcgEFXcACqSwm0T#&oox%N%U=Muc0xvN=gd;e!@nct8~2|cl0;5x%#4}QyJx~Iw?^x zY_w!iuIbdKiKEsT+DDRw#>r*c{R^%lmvx=P67*R^HQ4L?66z=xBCOFG1y4I!pZR<l zlAZKElVn^LIY#DLI$HF))k>gc;+km&^_eg$XKC5xBj{tEBY@Lzo0kq77<*6;k61g@ zHFE4P`IHFn=afaV{^i_pw%7gRclqp0lYUAza<C|vTa?j_r%mMI`0h=cG;C1wdr{}i zLG>TFu%+Bcba_Y`YLs|&osv!LvyV4^MGOrzF69{%&AM%_EwalhXUS&ai?e%<v2Lwt z9cky-XAr6D$~2TsTw<H{@D&L*&ZvFSm##<n)6xkD=K1}Y{G`WKe0GuAmchpL4}7Bs z)wt^p-zRj+E_b9Ep2Zy~2<|ND=BOAJC-+EFZG-AQkjk*fN}ZlY(EWY>Q$4(JqrW+6 zNSr*-S}M;IS&CgtGq61O;>c}R!RaydnfBFV3Xj<u7AdK4Fg*tU)?@H#R(Op@|5<8g z6j8BsxirG*DRgEMNqXx<n@wk(r*wy*7Y+t%?&6cF5rfe=xvkR)5D<oZa=<rHGw(Wm zP^^(|PS-b$9^LhP(O~pr(P=Q!7R>bETC#zqS)VZ!pw58{5)~sLk{srVOqm5gCUaSa zsHmAVcx~9BpN~@OnbJC40meX}To55Df)QZ(DB!;=+tMFt)T+`fL73F4qDr)LxU-=; z1vjK*WhQ*b5kGo1GJ4%i@cd9lO1$CO(TPhvPc;%nteyE{HIlXp7X)vTlx}cIIz8cn z|5EL97Gs%YXjB^4m)<?6+KhRUA=}=oyhrD=%S<rs;+`9B#!}QzsaW>YNu=T<3Elpu z{Lei}Pj0vR%cC13d<CH6M)L<&zJ#to@Lmis7q}nQEU4XFFnX7da^OajU^VPsvXlud zKJ%H>UsHqze@ZrP{G&v0WuKt7+`yQ5%x0o3_i*$%Dsm4uOL605*GvzR*V+l1TM~Q? z;+-RY-J06Zk~TWpF(Wd%zPDV@Tha5s(+Ss3ec!S4Yd9#u6ahea&L5&bEO!1sI9OU4 XXi4c%S?f26?(c+>#_@DTlk5Kh7il{; literal 0 HcmV?d00001 diff --git a/website/www/site/static/images/case-study/linkedin/scheme-4.png b/website/www/site/static/images/case-study/linkedin/scheme-4.png new file mode 100644 index 0000000000000000000000000000000000000000..3873b3a20b1f1e7a6393e85a6b5f0c942cccadf9 GIT binary patch literal 98569 zcmd?Rby!tx_b$3nz?TxGLn$d~Ns;bO>6UUK-5rXelyrAX_ku+uvH-~i(%lWx4QC?a z`+o2DJLmkf_x@+k>(a$sG3WC<V>~17agXV*C@+D5N`wjmfiR>bMU_FIzp6nX#LT~t zfp^~IliLC>C^nLs_8<@*)%8CFDP_u^z?%s6$`Zn$qMk>qzzO1OAvqxss5Aua{1p-i z!6#WtR7k}IVPg{0L*+x#_QkZUsn2`LB4wO-;|u#ZN#(P9D$1Tk&Dt$RhE>sh!d0p* z0rj=2clI5(Un>4w-NJovIyOlmRC9*o=09dR8{JofHbMo#N-4yk+qV`b%rBEd6tj>M zz7t2empCoeV`Iw77Dvg}5x1~6mE4^ew|6DW+mdMQC7|9%1`O`c;UfY?`f~#OVZ(BV z=|9fd|8F}F`91xI@$vDQX=y@t-uiGSM@3QbrzoJJqR#aUkB!kWH=f&p`HP3?hxObG zaq#e9NyX*mFZEm(aqYbVdm`FB8jo9<7bc+gY0D0tq$lkX#w;~ShbN|8vE(Vo-)O$g z`(H(kT6>>qyY6*!iOfTxP&?oc8B<f!i4%w92pR>y#{&sgH64lGU-mrD=SgGC2C`d| zw!7lkc02ViSuObaIA45Y8(U7VtsU1%@oYFh_khm&5hL^A*La`mRj!oQ9SmvTOHv5Q zAyGdog)T;GD?8mk@eBx_&KcR;Im4N>@lFAz8G89;=FYF_`XHQpd9N&`t*xxESkvAY zw{9O>9%ysB`_j4wwwEw{PQ8I#(tkpc61Wc|wthd^uIe_RVlCd5!6o@*dBAmp%Q*uM z{68OMWwc-)wr^TF)gHo|T@&K#&zUmyi6rO(R_f{Ln}o*q5#IBmhg6}!{^5yE=0J`D zW^6VT>e^@jaMNSe3DZf|Q5XI>m}J?-;S{`VSaO2Xr~n6sOWdeXig9#=0t|IE#0Tw0 zK*%N2ycOVsPtBC9-I7&L2n{DANeYyO$}k|_r*0KS@$I&sAt5z&&4<s1)|p~tlwVc! zij_@g_#!<5uA1RHo#TIYh6n=r_><3DWBgtPTe++6dRJdBCY`6FI+<!M`Vzves<eAj z1W1R7Ob$0jm+E#_)?fD~V<Z%R`2y}cBvse3{ZO~p!}DsdJD$V5FB&^W-wO(D2*hXh zOCE4_m~tsAFYk8Uh{AOsE^fSZv!6DCs}a-$djm3USsL&Ra+WO|20pF8Xba|=B5-a9 z3JUtNoL*f$GV#&H;bgn<>e9zPt&Jf$F){JWvd!uKs&7xu*=qs(VO{4W9nXV7+vR}^ zu(dVU5j?%kp+t@0#k5UN>q19FcS_i@@c1ga6H{cvYd3fI?zR{85S7wsLhB?;OUrY9 zv6-W%mmcT*t$ME)wj&@OL>oHbMbg3{`?ViW?US2(xPdiJY{O;$>t?NCRP|{&jI0}6 zqzBoHj8zj9AdoE1^(GjpWDyXUMt@~uYI<!h`|mOa{QgC0yG@{UCK9m1^Zv;EoAC(= z-R6|nU%7{X|LZG0whX&|$u(<v`G|lJ7UIo|2!B3$s{lx3U%2VUCK)^`pYAKy!p6nJ z1FnwU;W(^I<2+&e(b&Y~L-LcKfIfuk=?MAW+|gj-jin`r_tMhR^>>pXTYGNxI`{~{ z9npW^@#cG==R~)Iym?~=jHvVbqhBxoeNF;g1vmt?);2aG_0Q+<vHn6dh6NESYiQ7d zKnBKB9(!{3?Ok1w3LbOVOTZLBL40(8%eX!=bEhx1ob9$IxXmmrqkyd)FMt4Q`ng)U zU)6X~=-}WGJFI~b4(MMK;kB3f^^-KV*FBQ<_Vy$8S{=YjP=@&FH~=3sjXe3iT|@mU z9N+1xsYS~saLNH6{9IgGx?i0vu4fO7l!f!pg~&tNusOXt8LI#}5<tIx{aUiqZhmdw zVaO@pf87hz--p39K{h!#**`k^^xB*m{}Lquci~JrIlkq+l>@iM)AizOnFaCvm-B$9 z|JIP3i+vXVExDUh(EsxDrH40H&is&_O>G7SvykHB;eD?h)}sf3$SSYp!0o!5=~Az^ zbK;?_s7MBUo4`}W=0<*!*j|nL*XDdBm=jPPHzFIjxwV9Z1T7fQ7$3OA@3E7~tpsxf zPJ!=&K)G!Pz{LNS%+1Z4;C~hvrFfXwEfMez!0ivJF99(=zk93S>`FR17E>An#Fl{T z0JLY#_5u*54f202qEhpXy5C&=fBmBUTPXit#0P*QyYX(oZq39S)&~aiI`!R+ofVA# zd3FKlduB#POC21DfRsblnSteEd+=vf-Hf`^xyJr6!~eIC!R+h$fgj0h6BA(PO7ayo z0{7_={Ilvf`1lE2L;K<hz026w;?eO2#Fp6n#|Ygl&y3=&IB(8<G9LaO$p5Z$S724H z53{t~Tqz<VqHpU50lmu+@7~=3#+TcAqanABG?e<>s}b=I!s@jntv~?z{Fpxl7IFst z*2eU|pVBD*t$;T(e0%HBZf^4^`1d)m@^1BMZJL^zfLi_cs<*bbE(LSc!<Y6>uEj=j zBQ_hP-@feTVxZ?Zxc@)K?RH={KlpFscl&uL^}7pTO8`eKm=_q)_3>Png|@!27<YJE zA15Y%+ZysN?P~a_W~5ClT^9R9cwwI@OzNd*-Al1-9qMpLDB@M7piT__Cm!dToidqU zS7;g``)$)eZD`2uR4%8(b6$)0(w4FO6DVoI=fd@@KO?@LAhUDyuP%aWE=Z?Tm)qTJ zm!SK%dS;gXZ^^`I!1DN%6er%Q%_E+7J#y3U37^>)M!{i<41%USbUs|X10ra=*{{CZ zzXk#d#rvn^<S54;)2<{?>RuhNt}^|UWRXdEl+YAdn<K*7b1q!xm>}vj@$@aeljQap zBXF%0>fK*@AoaH%?8D;4tSjW$x4NX}F5B5xcTA**-7V){xC>2KBYm^)ae2nV>D{)t zb(&G_eVDnQcwFjmG3pk_zMkE{exF77Kq5p&uKR7@@%QhG;W1n<K|u}vUcU?o>g}{8 zF_~hj`K)YT6X=`a%0?R`Yr6^92{A$RLk&8SI>5FvFn8FxCE<(GAR;4GLht9~TYR0F z>z~`cxerj-y3(MrU@<c0WSL$y_qn*92;W?l`cJ(7YqMV3@zT@FP;<sq>o9f1F5n$4 zlB{-bMono>Vja)CLsSSfu5kHgZyH{GC*$ZKMy959Z_X5@ns2Tis-v?7FQl`thRAbT zy0Nk+M<g)z@?S(Q5DCnifq9=|3Y<%h*!S|6C$WeTjqBE{=j9)LOA_z3R)RA;eHuq^ z;v9+UGx9%JR0glc>AH?<rT46O63WwJa-X%8v3Ye8i(xA=5PwW;iZl7Dw<Fxvs;pYO zehxL8(_wCF6=%D#cpD8mu}^3;F3!BrKQzSD&6vZ?2d7Xz6Hba`)&fGWwYB=$J@U=U z%1R73`GbSgjJ7V8AU|F9ACU@QP(B6x+vp)7AuahFv;Umi4UpyZOD&dgE4GZ-!W_#y zvhX8x+NonxWo42LSFwe@o8vE2t-0FAo~vIYB9!6axOWeu`Ufc=Q|w5^<No@i!q8<e z4L)!ysR6~@jj71crop|@Ndhq~K`Ntg8EQGq?~!v%JCT*rM{{_qIzLnFc23K&jw9}) z)VNWO^5V<OY@;biTRRKF!qry5mX;xBZfJwrFuuZ`xvR-u;TH4aa}9`v9;sTX*z~<Y zh3#K%)Q0X>lVVvWNW!E>DP9(>Mh9tYXE`^UK^0FSI<bywZmc}PPv3<zOf6l7HjQ3I zmWD<|V7onHu1Lkz6Hl+KWHj3hcF=qgWXvP};z}~e_-qgi@&Df4{fz$1Vycd5;{+z| zExZ@9??Scb8PV=ax(dB^#xv*@v^O@S>DxS3$4{b`MWL`xW@(!tq*Y)?IMm0G7rh(} z-;s|tKDQ01cK5qFh^t+{i_);`iQ?r}SUo|=8ZRxqBxz}xbW+`LS*=p^i9cyDS(4s$ zVPQ$qXuhMmb1+n}`%+}7jF{-6f5_=6yQ0cMgbA*U5kxd^)SCt8i>9Jef(pUF+Ag4Z z->*24%{7U7G<6#KMwS+B#}~l|gt}8oMx-0B{CSP^FY^rjq36wqV#9^wCMk`^QxjhX zn6)9I-?wNn+~t?^i>98Oy}AFvvT;k_WO26y^(h>jYCNA!)ABjX@sYiq@-&^~k!sbj zmb1CG`^0LY{g<{W))7~(7~*BmjB!P)#BGheZO)#s13gIgz`UMnrOf@|1{|k}A-)>A zjq}wyH?M-d{O#aY*twh4wPIwF{t?_y9YTsJS)=ytuqi@P$=(lYP?sV4{&1?~Y>S-= zlYmYtktM4aboFn6-JAF{LyAk)df)s(GZBR^@kjG@;Z6VV-=$5a$p&@U8A=M@sB1iL zmTw_4AF>LKUK&uki&(=%PmtCe#S`Q><vtyNz!>guQk#Di+c_}wBFuqMRdpH#O;fb! zbxSsl$@Hb~?#6(E7)A$phJ381a8V*-7PVHRO{A@x?9;OXZwJ+`)Zm^*+kqj3UhOue z-VX3zXivKj#gfEBhQ}(2Gf0Td-47DQNVdTU*^)zNY(r;D-~1!e4we>sS*mtqV9-FF zWUG(KFFku#*v&~s2~c4g`ySTg4KhRu$0ltr6Rv%Npj(>Q?*P%jmQX*|RjiI%D9&zD zEY4${RP=j7=3Y5tKX;Y(9k!xSJ%zbfxZ6<d{)3*UQl{$}q7@a1C}@(8Hj954zHU5e zEHISn0=@G*Z8zK9do5j5_&B^DBvcUkNlgE$i?1^sS$1PURQt2q<|hySF(+WkTJG6E z#sg$$ja%@+yTS~m8ncl<Q8_;-D`SyR^ry;Ic=r92sjvyuHGcjCw*oalS98?zZZ<qL zfQ0$XYvtL1XQlcV_<a3g7{){UTIBcI4tjk0VOVa{KS)D}tV*@ZoiZleuL7#u6k2qB z3$H^npFPFfxRFH;FUCFo(;4b~!+A=qfbJHi|2T(k-GE)z(O7Q3y={O){fWfRqp2s! z$%jHQD|Ug1)BNPk=fd+lEYeNr1NRCoP3iIJMbid3Di%YsDk~?^HNVWIe|;aVsd?h_ zW7@e3ol~XAg{*)g$Patt6-r@=@^05eY-j%gJ%%K($$Kmw518`ibe7)3b|8L2-&??b zC^Ei)gMZc)EwmW86+SW1U=xUp&pu;-MCQ;)zok#4rYhRPe`pY_xs$P?yohBvnqg@f zS;SKo5QJ<D89h`z63XdV$8=vQYux?$B+W@7tc}9XD~r)<KX%P)<-CS;Jm>@&?J?Dw z==^$Go2BJu|AYsOx92eQqm@X-q3oD&hgV$%MjxxTDiI&-MR3`>Cqs7o)y4W%-wS?* zx1Ujuoj}SVEn3ke_IxSCocx3@)LzKR$moY=8;)Y_9SKfrYr$Km-sM9Mv#Qsxt<((f zZ4kuLWAdq!L-kwK)485!UV2jk;Lt_ZdFU!Eb)G1^1fTc5UvTZDdH~P5(Ok!&<cxq` z^vqc!ZVcDkzVEBkk|jr0wor-U*s`gOVVk#u*^+mu4rhcm(PD%tadBn31iK6zrT}N~ z7!SD=ys`kk*ike{-}%-d6qAE%3gInf!jTW-eFhuu+&l;rd4=T8(vp65F!@O_z~pUd z7k1})8pwakS*CLw#v3Ed%K=f7C5rr{^BFf4ZrOJMR}>9nL{u>`m%$;ldgk2Xa$cB| z96BygIr4~We7hp?z8p4T34A^^?cv!u5gB|wy*bo)ghjG4w!JSvtmV1d18Fm~wDwg8 zog<~YPv}^mrz1qv)e$PSP`VX3Z#SsSiqTh~ix0mR&^rk<J**@Nz<$wxG*qr&9Ybt= zS>DGE&m+1v`!<|fIM=U$3G)XMxRad_Zw7&r4u!6^l4thRny}t{QC8H}#d=Mh2)s|` z-<54mE-QEXTPWh_#j;lv$yrazDW*;+lDT2~NEfrw&nHG<`>={4nJB?K_-<KM+Q^HS zGnOca`|aQ|y37&nEe!-jT)f>_uC{`;uutH$aB%Sfl8j*$sqz>eEsoV{&MdC`S-eJb zg3I_bcc)|XU>dy;?m}c{id9Y>)BIyBZtr?Vf@j6X@*35P{dTYYJ{WH7jfB)|LFW(% z&ByFINQ_HIrrF-?q^~e0;zf32@_oj8<gfm8u@_J0O<?9W;d(uBV1qS9H``Q;(`iEp zFY}T+z4gx{yayQnbBfiK2EdH~#%OG4OL&#=LqLaVNnKkjs<oge{Vk&B)WA{QiPjYv zVgHe{-V6RNPN6LUj!S*hV|$^ntsasc)0*JHZ<<>t&RJDe;}Ze!Y(|fY#m{Q(Uq-#j zeq^`jRixv}vj5e`D@j15Y;K<Lqv2<`=A>u~@z^ePS<+{DY{v6vu@i|K?{fwn*^_@J zY&vOa^$?8DlRD_z+2#0=NaR1r;4k^Wu(278jOhFhi-Q2GgI~e7H2Ba2pK65S4JG@} zFj8A42cC*G`Fz_chqDT;AQAz0$H>!+;twS2=bl1IZxQg$E?dz)YiuW+o*FH=1|Bw) z=3!!pdA-{%e}tieu~UDa_TNb%0Fb;VS(k(OOWbAfHbf`Fg$v!KpK1{DBh-=lRLXRO z3)k-t?fjtE5gSgvMKE0ft>G~#04o~Uo(PZX*0-U*2CtK0ywmt@q_XfytmTRMuoXIz z@tm1l30;>BSTViEt@wO<t^V#EEZ?Vb&PHEYr9|dC5Ewr>NidYRt39MHI|%pF#zLtc z`M~Io^OaG$*S>;|j!1U2WL1idEOXyeS<OisJO&xsOEXJ26Jhc7cjDi_P*y!idOyXL z%O3d?$KYQG0+}3Qu#omekkIg^0bp0<N}NNbnVM6Bs4y77HbQ148G~xCMCj`%rM1@! zTq${HQa9g!->R0|-Q~CN!!Jk`r8DcBn>tf_uZq#p-4RxXR3!!95N0zd1)O)+3CD(# z<s;1e{!VogFVSdK2984$KX%Szlkr2DJ#lkKwny?9YAX$vu7>p!dX3L<WZ_Nnb<x7U z@BkqvRT#8{^rcC?HT#Tx^<le@n&kB^eSq-?%$PlJ6Y>59gkvkjNN$q`r%b~7=uP{? z#&MyAoMLW9eCpv>QR*R*KRxG{gGM6u)$MkSO+L|6cw_I&N0nv*_9`LWpoIB-#%!>t zc600?kHsvWB}%#LUQG$vdnRun?>Gd!gf<;LDa&2D=;pllEBpdyT}~%~?$L4M1)XEn zX6rty6`lh%5^5j#@JCT4#!n)nFi5ZhaBa-2NY0$N-FVq;`bS3aZA#UTORMbrS%OHa z()#RQVOHVbo^%<RdQU%kJfFuTNb&!bRTvxgmvo5Dzs}itY0XI#;$Cw~>U@M5K?z3I zD-)ZD@$dA6?i`mjG$)Od`VksF$UZxQ)v*Pg6n*0M*X&sI2yw~w`>5l_+=(Azj-J;u zaz0o_WvYs8`O=tB+MUAko^g)QM`IOE!n~GB$5&C>LxtKL3!F02`EU(f8TM4Wip%N> zqIsg&r*wyLok0+t$=CD@nV(7cNbriJ-uoFBtUaq?c9X}#Aj*<1NR{)N`Y00OsC{sZ zZlYcIUC6UD{^_*ao4wdNYoD;+sl=_{>K7FfddqF_p!vR0A4oW4r%?|jeo)WBhVW{= z_>e-=a~2<?pgM=p*UupPh>x>{Ijqnx$OK96S%X~dl*SL-XXE!*2PPl$y^7Lt%jv~( zhK+b(<ZK#jSh}-YutajQ2M1sF8DY={sY=d%L-8ItR@7q9WY;lN#f8&xj-|h*4G8o| zqkBeF4PRWuoOOwe><U|omw6k$mUASz7_fLZo49a|X)L@Qb>mB2olI1~#%7^Pl<h>t z9hr13ttn{5#QDm00;UCHIVk7%yo}Cro?I&^5c5&IgI%!~U_H?Yk4<ivk=Y)Hi8K;w zpg8d{H=|8Q8AwDS_Ff6VIx%HR4vK?P^f_RpF$T+3lrFZ#lEbfzPOn(9$7yt#&yx@@ z_cBWp3CLqErI}7&jw2uyKYge!?iZAmp8mB&u(K|ln-%9!jf2P!Y=wfUzfm+rjF&2f zTbOJ(>@sIOQjM%IR4_^4On7!giq6*)tf5CyK^2|^kqzJ_$*ZjFMh_9y!cig=ze@@K z{Ce>42Z_RD1KoV0<;bz|$ll?1j??7QMr#M{e8a@oe24JQzFDO^#u`ZES)F->o>JHI zBzt&^d-=?p6=fayR#B~-`c{``f8xn<4YM-(@LJ4mio<XEpm3VNO`EU(;HzrCr$;zc zXi_i6g;n=%V10WQU}-u97?SCW@oa-HrsK@$N%D8w%g$oZP1LTfAMK}9awH5#2HM*0 zN7fb|d=(Qa^mOsKoJHelZ-dUy<COKidDAva)E~l|{2)_L>2}35V-$Azd0Wu$jhfA5 zq#Nd{9I11{Pp-yu@QB+o*ef0tHrS0k*CRsi51>5^{e(+Y3Q-No=_&4Atr$C`)l)Qn z-mu&E#4fI|vhN=3#ipt@!<g06va%30i<5BM#KP6e)9!|gd)<2}2?<8XCIv@6J^7pY zOq=VE*hUODmtXU1lLvO<`*r_9rTi&=<48X24{)5-^NlMmwcN^cWE8e==MHPcX}J0~ z!;6OPD1Q!gQo8VqoX#;(@V!mdENYEd{7hoEiVIb&)5tbm4(MDAFNG8s`g5ay^pni9 zj1p)6k)|sCHKvihAW4o+)K5gkI4W{?w{{Wix$%Z#Y)DcYRkPSUf&8%{T(IlZG5_lr zdEvJ8z)Q>VX^qZY&d)E6!N#3Sj(6!AMg9031n!d-1rmu8(&@qsI^lcxg%_cIP7dys zte-1`z{~op{NU79RHW52Oa1QVl9P=nuZnj)>AN3Zwf#zK4F9Az{Sqv+Tc4f3U3`4m zKQKktoE%ocK%~>_9+7F^8`balbNA9l=G*$D&}u`UxryG_efb($opPDhq}eEkn1lpN zwWandRe{pT$bHq#J{!><7OhM4!OuCIGOj0I5qj4lmZNogK)@+)Dxu<5I^daGpu^<f zuLx5o(IU<hHH3!-7BP=Kdsao`oefRyh8>Fb_XUxZgU`~^F!`cuNNa*r;3n(P0ZWg~ zC{hWGDvgR4-Hn%E=O3F<mzG0j4k;VB)(!%W4L}^Q95k#}BzgV|qKqBUX%6FHx-PN< zhceU10AqdQRZ93ybCnqDJ5O(j2|8ayIn{Fx?TLtB<{BNo{$ag#g3KK6m0*H*o&w#) z-wK3`1%+E<59YjesNdo<W22z!c~Z5^qvkJQv6JXj7b=(G@zPE`)nrWuM}3ZtW?Pzf z-CO#G?vfdgtUz=P6AbaEVkq)fGXIEsi<_mY40zt|2~zhD*p_J-jBy*dYKE%Mm4abT zD-|`2Z)n@7>;wW#&$@;L<V_&>FJ>|8?_o`v=G0kV4?mOnHbu9o(?EzxrKfF157{?f z{Nx@WF_E17@ScuakL%)GGkxaKK?f|;zY6br>;jg8sebc9lPW3hu%~J`_eIN`os#J2 zQ!VZBMYfbe77;3<lH*l{!`7rjSi%(pd(of_ASAu(+kuQ)ZtWqF{eeT^@;ND`OEW#6 zW}N4sj$hbQ6g>y2;VWk-qFxUkG}fi0wSNGYU2TG%b<YjO%PPWHzBa%nu!TMOxtX8# z4+C5YElXTol^u)gA~8@by8kB%fdBYGiGhJJci`5#ye9?tlo>2_1;L5OeEkD#QWGyH z=6XUEJ5!tY8@8hYd-a+4g5y}R-qIK0AEmcZt+!mYYBZ|PFmQ8^rnO-N(`41Sg{Y`7 zJZ4FP%xcy;8G(7ZXe0nY0;%cSxBslWWTG@qGrtvVdbBT2%r$~+cW@Z6ES!T_JJHY9 zI<DSL_2h8BnJLEMV&4wshXO7Ndi|R~%kF-*Ba$5t(<o!zIOZ+%1}FRU^E%9gLjU-) z{A!Dl{me&2o>Bn;rXT2P*tt?TQ_Ra%ZAQyXk`@T}5MpD0#|T&bH|i*f{rK_Yv^KX0 zj8d~cs*G|OH@69jmpX<^aumDt_1p5{f%h&&-KJt0p1|ETlM$^i-pQU8FS0J~(xX&8 z*RZmJ@B*O>^r!(hEmY<LZnX43PO3vES7*m3vMYfeM9p*)_BJ&_Zn7g5ek9*;+fs>l zlN5Xrx^wXirsL@|t|1Qfu}|I(vx?P-D^^m`_VcZ)?Oe9f0RRWaT%0jOFGn~X3jv0f z|6-27C=JIM=J(mL7iMPMPZ+yBU&TyxmQm5rNJuYvezlMiE%^nZC;I+JsiE;v@AvOm zl(0eia9eWPdOnJ`<q=drx1s&xiSLVBbzm{(*5^9!dXuAz=9Ez`N9}gJXj4LE;%W6O z`W}G{Sbo=Q7=P&uND}Rz1qzYA&4=9XQ4WnKuU5GQq{693=y#_Q`d_Lu;y36~KfXGQ z+=nD%gPM+;Zv_9RstdZ0x3L;MHGQSfaXd3lLD<|NFwW|intzqgxZYb`;SxZ9zFsub znu}$TMBS_x(nIzZqPKdg8&W;}V@gcw@=ezw-Vufg>q+3146nKl)vvLUao?i!K0CQ> zPwKbzshS1!p7tlej}m3RC9FpmB-G6{c$T$SpiAW$6z^Z2PG=c)E@z!}CEQ?^;kWAs zf)U0b^5n$N;;qJGd7{p1mv*R|ep()q>Oa%3gH;(bNl_1z0<aU0enj$TryJQmxvevT z<|V4s-KG<mnlX!5=&QeF4eFFt#&;Uq#>j=u><2A)X*;c{u8x!|y{qTDxv#k2ZK<eV zzmMPrTh`G9z380?KlXez`#5$+5$X&52r-X{aIJ2t<FSy1uL&41-_&1I_L;pc9sBI# zI`})dQn_r|p?KNENpAma*s>!G+%blld{69{>wm+<#%KB=Cd>}$1yzZ&uY>+`pxOr< z2FJHWPcl9;rO3}6Sm$KwneJd!ZA17QrkB~S#{D$Gi&F*+Y$D=agS+&rrx(Rv=6;R4 z>95;bq2MWWCTxb(j&dY<QoWTbt|DG0TbtE7mSGD?6Dx6+aI^~i7Oj78tx=j~g#PBX zF28F{J{5icwmq`Xh337NY5a9!DNNTWJ4W$%EVbEB0?=147mQI7S%(hBeI$o}=9M5$ zpGNH?)_h~*ewkQQS*gouHAY0FUy`Z<z_++GU@LA$Rn^*s75Mzpd}89Gr%bz2CX08K zS*?pef^;@FHQQTbZuWKiNQUeI%e{Iy=d4wUW33o}I}VE5X+KTIw}?f&-i?laj+Q5r zjVoBgaa#?Ck|H~YD63`{zAo830yBVrp$4|4A8c>$^$Uub^taKy6XjhK{DJ&M0&cS3 zwE>bt=J)rFzpBWHf16x5{GnR`*C=(O&yK2EF@2V*Hj}M?r%?g(la^y-#+LKkK^@kj zp3tV^ajN58cNl~++0GW~mbE-d9fsL0tNDD{#xlubY~OgoY<JgG`ntYCPcK&+Kyx%z zMZqv(sLpEr86=n|d;F`yA#y9KCd4_cm|B7AxuKcwl=H-_#nr5*DSK-Qj|(117e5Qc zHIAc<D6<L=iet;HpS{98f>s5aP?4yiG)yxHK{cxcmD36fCkyPbLK=#26`a(!Ia-*2 zYLD)2aEejdn=__SwaQKp9lZucz=QEGRa|?6R*1Ej7{BY`>SlOC!js@&FZ=64G{7Gr z{<9W;C!wE&=rgO?Gg-yHZE`dqBpCB1Gcd7ru|k>K!)G}GST%6}p2oX}0`5=n)^ODi zb4022X|{@dpyyjXdON0v%uf*NACvA^auU^VN!ywkEF|};wTA#@7;5sEp`pX~Iec#W z_^;;a>0=UR@v>gwgCwc`4NL&l>cHu3F6*)r5A1{NhIM5hu@?HR4o~0L0oD6Cep*3o zrnq=9+V5Tzj(eIlTz-tnE2A7TZRaqFCbTL=J4B@F8)cl5RWKxvF|hu+F5}0TFf4PD z{$Xi;Xo&OLXb01rE2~!0tE`HNPy?t1J3TZAvJ_o~oy=Lrp(HO40X2bV|5aK3mE+0k z5Gls}(2t`kFRoXFEi^25mB@InP{G6LP}riE>tC$VH<N@leIVJ9m&y#6DXXHl)!G!j zSLl?U7SrqD6vxh)%&tEWEe4nnLQ609=Mf1JkMoMK6tve}2{DMal>9ATmc?WZR6!~+ zwa6txwD~QgL2ds^Hbbck))+5Zz7%q{5Ky;yaE~NuP{qtd=FZ^|*^Q2n+9+v_nmk9q z$7uU8G(^}Ula|LK9is|pp^6$4CYsd>yOt0vo~t79blz^;lU{g!PkO2SJ8maXs1%y7 zG+H_wwVbHvM(_XP7*Z5@otq6i@(m8U8|!zcDIUZC8~7x%bRr;t3I^$p(g1QY1GQQu zZO&r0BeAN=yfvOQOFcz3I-cY)3x0@o6+aLSGXn8m$JV5~$iR9He&nR7#rm5)$FtJs z7LRz$i8*{Kw&1F^RhZYBNd2c^CW$Q+Hf8m2SZsBjjnY$9TY2)323%0Sj7c$7Hxi){ zrNhc5oPgKCaS#B<So#(g!}Y{MM;jMI%y{NWxZ3;NQSULAi|WzKc2wv-pEt|eKI5T} ztk?A6V~%K2%Tsaf=MwapBb!B2^u%12k6A+eB<~Lo#kG_LWSNFi#%I7dwO>8H$em5# z2(kDpLEM)X@9bHhql)FUiA75?!urrfUuuuNKmjLYnpwZby47b$=ClskbU*f?-L(hu zvAG2Wt}FVlurX>7gLa(41jW6@@QV>A0jLAQ_#H>xk3V^0_$#f*I6mQk$%^``7Bg&| z$e9oNf#$mmMtm;~6$T$iZlUqb#qpNtD)o&l1m$0<=*$KA1gfTe!y<Gc4Du7CXkk1# zVaq!3buz2`z{y@w=Cmo!R`QIfg5o=5v-WFpXZNZ(jqx-SMk@G=1bxM~jM?bCPm538 zn_A^X=nqrnDKScT_kU`gKuafp(#reIHs{O*!W&C$@W+XoF^#ATXDz>z+!hOKaz%-c zQl9YiMb_3KL8+-X9T96`o*V27eCERIDIzX@uDX9N^ltNtyl7)wbnd;1+g+j6XuTa- z+m8yIW_iX7^7gVw_?Gm>F#_KXMRAXqqvGRzO(ipds(_}`6;?<0CR#YSX`XpY6A#q} z{6IWsV2D?&98KgEzPou!csRE*)bPY-GnXa077Q0IKlLx_O)vBUBFwmuK+~fSaus6% zd=u%NBXzF2U$6qKKKfY}tT6!oAa;GwNe(C%;pwW0wJJZQukD^_)M)EOtpb}<>slq0 zPJh(@f>vpw{P8Tr6Nnm(Qb<{}$;=GEEij<I7{_G(+%n@@ISuOmWWcrsRmFv2;6;3l z#_CfdBIxOr{*`3k<{H{pFbFokiB{*r3o|N&8P+X15qFw}8Vs-ZADjV&qAi^b4rI6B zN~9J#=suo@EGxA_Sn1GM9;3roZ8Wn!!!JUp82KA6Z^|`KT2Uby4w`_0{8)*S=b6@n z?y=$l*V<m;g(ox9(Xo5K0a2cxXx_~p{uwY?17yY7@HV2N(vQ{-#Pdixk$##E{Obg* zXm`yj36eNfW~qvc$2x&6*g^*ZgG{sd`pUuCgy2zAV8HG<yJ9yA+jL6{1oe>!Pa+tt zM9;ML$SAB|*+a9cL@4_vp6E7W68Ju|E0ti)u0F{6(k0DQ1b{g;pRVjOj9sWZ!DH(7 zifil4NSZ*+FbjIp<umkLJ6VIK<zc>Dh?b(0wnh_`-gvTcAL_HPw4^c~>3gy{<WU0J zHVLj48+&4C!x|g2(an*Bou8d_q^0@ED=V2lz&2;`IyXjV_bMx4R<C|Bg|Hv~_E#V> z<zmUB+DTw>1MPN#efASKR~2e?`aRQ_o3Pjx`UmMJn^TQtyK$W1g>$<>Tk5tFAtVtQ zwtB;Fp71#FhDCBdC&J>^@}5A4vvZz`So$z@KfPZKBx!`4daTy+U1rPO3B;(?U$9|^ zFw|=FoEkay7%>&7zj*%mNF}85x=09`!TJ-r{1-_^2nz|ojJBhTxbB!{;bBxG-fcb} z(BFs(>e!)&BJNp5<{}LTb}nMk^cl#smQj^$TvFW~L&Nw0;mHF`(^ak8woe{QMq$>X z?1)R9VgN+7?1|)^+dSZw)pW}r>k_iAu=TZEU~j?#-<KTLhS~BBGV*y7Xyj6Do!3+F z;ky|@=uvC<<}}Y$cT5+*gXx*|3Dn|5eIFB@w<`#XVHLJtyMwP<m}WMndL57Klc3O- zCf#GfcR=$9H}#ZTCQ&H$ix)41Lk_5k{c?w&d%doq`k6BJ?u&L!uTw%VhB}Kp75?yq zKup&=VcXZn?cjk_DZufDqlV2!16-R|xUaO7C&Fo!<>g)6Q(V@qEwc>KeA52qYgu}b zvWBa<Mh~3yQ>lKcRqZE}3Z+wx5v8lxq3F2=%WCeIxa2GaJkR7)Vr6Gfe)Paf6KV(z z%9^9lA93-#u+7zZ=sA`l5%Mt@U##nwplqoB2<n<8cSF2+pkKm6`?ktGYdzQs3P!KD zY<{TvhS0%Nt0Rk^Ho1cdMZP0ENI`(|<*uo}`J)foDl(mMB|G7+_Q<k(LgoqRV&i}1 z=YiJ&2vgY1g4gWxFe#rYL@B_c?Z+F7;n}g8lI5jIUUd)|$xU=KgLkbm0C@3XZq9HI z7C!~vJNfeBp`5k3BGY+|wpL0l?eZm5gnwlCX;RzW{xhLBEsW6y3W_yUFDC*Usb5|d zD+Io_%g(HA);bkc6NMXeh+qm(_?CWDC=mQE$&jys#d+D+V%`#+I~y&R0QsPrqN!G@ zVrO5D1e*K3z_%c$B=#?YjZJ+L)9Q;sdR;j~^m@-IE!cMxQmo$oK~~4h*CxuQZpbth zMAGu|pDQX4R9^vZwF&*hwSD5&V`Z+J>wx6x0+nGcpnxAQfC(@v;<=K`%9O9`LLLR= z@$TE&<Y?cm*htsQ=su1de(`_{=jL*Q{Y+GhGNyI1A<W-9I!H=%iAt{XbWoGQf0hkE znK7x7N8B+w7EUt5+Mbr5-tn<6O_0{BW=~vwGS=0mOqN>Jg^G!gt0{`!r5PKN5Md(n zf(Ot5h2E?hfyDGThotBUjhV(p+$n>sv_yr~W#bTZ-OH`D`iphuM|MqOW`8YDovFVc z%I~!BS0)s10q9fZlOr*Yer1uF_ihWRO(!}`tDA|GNi@`*q?oP?7erMRjX04-*)t;_ zwJ(ipT=>tpSP#;y+kdQENuJpRj+MFUYM_x1x_l*dtGk4&nzmCZTgiVeVAFF4B5xq; z`TQ;86(L5ZXpw{&Rcy1?6S0ihgk`U4pfD37p<85?(s201D87!i_!sJ$*!hQ^Ib+;R zV&-XD?r?7K@%g*g;nkj`*uCBt1+vhS`*98@4)eKBO&`4Gh0`%HGQRG@6Cl#?FK|g8 zRJsA)m|R`iGQkjQ4OOLr16o}_CVIj_)s4=_9f!Z>3nRI;sn7LFswBpeAJ6;9sC}+R zMQz<Yt9A)o?pPS*>c~v&Y~?cxuhzkl{sr5RMg1WvCIYx_kooJt*OxiLVV5jYE4`3+ ztTW8j?|g$~HX2XtTv=N=Ov)rpi>mFIV}dXj{m;I8?C7YHC3k3sr@f6v$4tA~8{!Xd z`IMn}Znby&CkM@Vr7elcs(3_nU}Q`2L06qN>k&)yJ45e&F{6DPg#Jy54y5?C$$Z%V z3rquovmd`>X?E0G;OU05d`%SpA33&W!qx?v04(<3jM`s!0sT!UCX;iipX!@0C}04V z^xw?pO#i>Pya$lA-||hX-y#1J!0(HJ9tEKOAMOtdxw$nNBRdC&l!%D$Kgr!>mA^OT ztx)>kPjANvl92&u-O>%o`#(9hg7JOC=6Lo06OZ}kHh<HJC9(C2x%Qs}^hcn`bj#iT zW23KoH0nyTxyC&B0g&_$%eN?1=ys)ky?#vpO{4z(?jLgR|E}}y-!QNwnw*?me_tP& zUa>kbL>wF(KpiC+e`m2szgsZaXOGUP9{eRKA@R{Q0|PMpzgfh8d(s;g>;J%GzL~(^ zbm9K@!|ND)Jph3C5AODpjMU0XIl$C~1HMkmC__J9TOJKGf{S}XDPWx)85JcBc-}j* zqRj@1OnP<T=$g%~t>~#KJwWCBJKEb>K1xMUcL41yNCd?Jfq|{tM+qr}p8P4YVA%3& z=m=B4C!F98Mg8O*kYsw0)G0Xt-aTb?&673$4MpPqs5N`40x&#1EW5?c%+0YulA?N3 zvYMLE#8fou$f6=9Gi&P@9Q|knQ0#*VtE!WjN<>{B+k*U;3+HDPLc+5k7CL|!*)@Se z>HrL>X++wr$poPFnK&PiAo$E`04^QKS@8|()>l?SdwS%7?{s95$8>ecBCzy0oC_@o z*8=FdkDm@JGr(vE88B2+iRI-vJ+wk#p%{m3v-;XKxnwuveFY%XOg}%#-V_055G<@m z4S0hkdcU#du{X}IpJh_cD3@QQx*8FI#HD@h#@5gYLax_q$^|gOx%RVPk;-w&d~9dt zT<2kH_W(Mw5&oEH4b!y&Vi{BJPhUAyV2qBA9=HSQ2nF(80naqpIZ6<X_i+npW>u9s zpx<wxI!w^>-Olb5Q2z*N@OHdgEfQ!3#8lOu;-Mh`zV5tY(F)gV7R#@CV1*DWTUfYR z=6!{L0K+bv60Sx9x+l`Shtr1jKxVSPz5!bP>+tZRiV*l6ftVC#^`jy_3Q(v8Kx#%~ zN$#Tq^a+mp7b2n{5t;)ksOeRW*-IdF_JK$3`^d<2@&Mx;>RZ7#;7xM_Vxx@w+Jx_z zf?DgV4?#XH4%lRXWo%m6l?><LOvV98a%!c#s+*+-3}8l}%~M$5O=HO4Jq^afD`|Hq zK+k&+-#q4eRoo~@^u;qx9;m~i1sA(JR4y!;qJa!Nq}^`KbS|(bx}&vU033a*5`>#R zgylCD`!d~^zG*vAQJpB&pr+@yJr!hfzZojOI!biBpOFC6-fO@n=)?_BHKSN^G`vXZ z(Z^YLKt9<2>{Nh3m!b!J8?px#**DUo6u;v$0<up0_6I%xPvg@nI?!*Y1)v#Z#9Vw& z0pdN%AB!hyYg>AJ;Z@G}?oZ#uGZMyp4GJLA=x(zt-0XMxYOQ9STP%PVZwe^x-g1qE zIv`N_e{^2p;KTx&M8J1FexQ{|(rc=#A&fUHog!s4TvCMb@H^uRJvFBUpjeU6XID9> zWF-%Qo`*>e*%<;&O%X$oLzy;U*W=wH#DLU6p(Yoqx*Oy%d3oEC9c-m{nE=_x4Xh~0 zTyMZSaGE9+H5ZqA6R#FycJ@;0iHp#v<&p!@8~DM{Flh_~!nwzRF}1>Jc5Ir-{zF-} zet4v3{?#~<_2XNxW<Dwk^Belv%3-l??Xxs91W=2}tptI)`$)BrjOR6AqsD;=GAk=R z1r^2aKEdQWm(<Ys)(@;C#B!SI?2t6_b;>Azs&YVsRd-G&Mt}@}48Jho2B1gHawlIx zx6v}!B=i_Rg{^w)56~%%^%1p7^Cn17t!t6TVp?U|PP&$USKoK|r<tzkx>;*x5ZQ~W z>#5%yf{m?<vxBmq8(3Ms*==9Yf4%b<%FV5;%&Hy`fGW~kDXPNZzhq;n`vc&Fc3<9A zuJnvSmng82@w#m4>*fQ>eK}R&V;Bh-zWE6ne1U5gN%E0*j1)nz|DRg)Al=e^Lo2?e zg&cseyS~%l6`t-6q?`*Td%D%~9<@xC&F6N9M?@GkWRnwE9<ask?tWORyPUp|)pK8L zQ`Us?ZX8Swx_3#liew=WyH|@<sm5Fl;2(Ab3XI}INOZl7$t1KfM)gC~N~w2Gj4M>` zfY3$%vCM*!^75iQ=@6M_EDU?_I(Z=SOMtU1ZD(&U+{;KyAJwP@p=57xKhpD;5TOQA zZPL8U^iD0nY(g#0+>Y9EQ?Y{m<iEb7c{rY@s^-9F0y`XAF1=@0@h%W|NQndDStW|> zat+dh1b6>))`hNyaL}iEiU4HM^f)eS`4!uF{|Kz!IRab-3i>qm#GjptBjd<o+Q}xG zt>M*sqb)BgD}fq($B$usR5-YrW@_gy2t7sOR#pJL0s+XqF^P-0+Cng6fg(Zoq036) z2Nn1R&@u;9idk_0av&!0@EZoWhN{-s*eL{>_Ka8ZrR|mSFC@F*EFD*S<5gS>011$c z=Qo2u;emf@Yv4C;!u|Y^ciigd_D(WWQw6ULuI)vu=#LQs67-JBN`^9FvS<!-E=h`t zwLl0WQ9!#iZFhH$3GJ8#c|!A0Qi{O9$acf(ChnVp&AtaggIkjHKv5arv*!i%3*7>` zXCR-jjNioP+jRRs+O??DVX&Xk!sUS;tjZ!7RigHqRz0EFR|bDY8Eud!g8|CbUwlJ~ zfvQp2(QAO~d2?GH3#I<2`w#U0!k>;2R<9Zg*)0XPhLc{DXem>bRVv9_LC{-GVEby0 zQSIN^?hXw)ndw56hOkFr?QdQh3#g$1c0o}3p8_<<%NxTTN|z8JK5Y6}j}?fl=%9t+ zo%wDMt6iDh9UNtBir*c+bm>_N;n1QYOoHiEW`Fs@vbKBqW)y4G^bx7~_)>QQc0BX6 zhSlg}039NW0xR8gX@yeP=~=JX0<DoZQCXmWCSYDH#vAB2pa3ZRYF?}ITM+;QhtmNH z@oJum%HEh(dTEk?Q?LBW;K%CW@OuGLXh?LxSxkWE_{OIH5{B(K2qba~H<HQy-KEA= z(WnQMQCnIL@aL1%nBh)5Kt)A(h_{S<qY`K|xTo?OD2yF0SA<Kc4i<gs90D4Bl?+F# zS34B5a08hJNF=2)g{95SU5F_#@)pJc<m(`6ZH@0P1WvLBUzJ%B!^|H!(xv~p0b!(9 zqeTXV@&dtSc)RmO;T=%!4A9kqNWk2a#6ttb6%KOOZJ?RCM+u0a89{ZNzkK#n$?s#7 zpo_VlOqMPBJ1Xf9s0n!6_OhCp*W%E^hXjDUH)3Mq5gWLF<hTZ^hw&b$Pal??<U_Vm zD^E)Y%=UIo+h@mOzD5@UT_XZyqUi{+fPTKWY`I2A2E@jl&G~>V4!J@o076m=?i^6; zQt&QQ0D_Ul-~9XQoyD@Sv>*wT(o7GcHP|J0CtpK4MKFsoRQ2F}oj{&@H+xX<liDYL zbx*P3(Xk;;G7H1ppuY+=C}MKQQ!S%62HwPqEmx_>5w5vyIJj~Hw&Cvc3Yt4UaG@ru z)@)Ogj=|xD4mmC_m28DZ;i85Y3T3S2#%n)h_YU@!c^2!BdOA)%%bS5cCS}Vx*^hq} zynpR?0t^6^HaWCNA)KPscZkvaV8Q~_oyu7QAxF7<=i!8k{wjcxrc>u-N$pn3wBB?P zKHL-l`@Q@Mp|{R1(5|90eJ-wzT<Go)H9FVc?_y{mlNm)H-8m<QX43tHIIH1uq@-$K zSG_a(y=rP2-$PK)eJz|UN!!w1^vcL7H7wL}N@)@1;)X+d?)~$J+>UvgpnZVX1H-<~ z=vhtyMb3N%{OQ96^w#<WtBJv*1?XsKw56B2{Rt!eEtK}>6b|E5UjgJekqrH@v9T;L zq!%4dX-K;_nxncGwE2UqTM?{HX&I6URzL)0d_-lE%5IKsU38YVeTP2*W`DreSn=)A z{+%<0t8-?%Lk2EkAZ)!kW@P=<1OQXb<SK%mfbQ5ZTh!7&mRA`0`6H~d@FAdQ4{W5c zEX7nBj^#C7mN)O}@tjR@yB-Yccjmri?C<+obDgC!`kyX!Mq<jPP!9+=e89o)4NtH- zjXfDj7mBmgZe;qH?5%XT$%7OBtD^GjzD+B6W0#N1QG@!|7`1YdK`bx+s-|q-UmaK| zb&P`b(tk}BP*!Jlbd>0(V<-h1ES~3@mg9y&sN7ED0Bn2B1G?UGVv2;{sp~w&#5H;R z%+<pPcku77M<1nhMt=Zg0w{Hk<+u?hN+(|*s`?^;&@m8V9`>^Y_-R)ZhGrEP_Bk(Z zj2@Xh2ja24?R;*^z|L({BwcpFjP=ux)?4g3f0hXp`ho}yfK4_{dw~5gSwtPfMrZ<4 z%&^`Go5d@HrCYwQNK#}43yS+Pjg3kAGS!f<EtEAVlT0g$wIhK}9i5#!iLL9%mwS>T z%!}*QuAd44tV5H_kAD}quT2<$2gq7zRZ=NyYUWzF8YY22;xY8#P|96h?|fOrFiZcQ zhNFh_76Gqu^o&eXPa+G=rvm~%s`xPmcz0L0!0PW&TCZwTP#$zMd+nf^SvBCCo*lOd zY#lQbu?GP$pZIWv*#M%X^EC_q+GltGmFCOy-YXsI_rI-)Y*1?3`})f}Pk#V$Q+#mq zbz%H@l+`+59x@9HyDcQxt*0J!<S+t?jzU+B-twcM0AV(|jmtu&nkr1q3%V4~%kAAT zkVq3I&F~_rs09@&+x?xrYBo%wDi)}$#<t&iD>!n@?0q7$N8>FA49RB)Rs+qZ5V&_2 znEuQu8f6Ny*ls^1W%@W2;N<}c6$wJjD_VVz*R&Jqv*Ds&8Hoa*dqpR9!zt4wZ_JG0 zt|4_Q*V;0#W(;4woF#Sk8v_{q40;|hyce>;pe9kLNyp(1p6xp9!$;&diSV@=8l02O zu!@ewgJ+YfKEjSRdDy|#V)T&nkYauU)K+wzCEQ);Zmsd9CDraREU%gzzwofEcqw$k zTOZ2p6;sMs+;D1c2Hr;#S5OL50La%E!NVH@v<C-mh)pAXPIt$m+fgeu{(>5$b32_6 z+s!ngrS05{@m4bJsZ;iw+h28FJabQuI6QQE%6O64#f41yF*zlPr;#S7x6b_0vNI*k z>%3!AU7CD#0a~VcN4~A-+U*9BrJj>7083kHoI>M<n+-}g-*2R<J$e*5v5~7>>X5e3 zn@Do#wi%|r@KdJvJ#4s8&r~*nzwzZk8&T%cOyIL4OhLQfed%|YME1G7FIxNNpEw@a zt(@S`%NM^T$e#-k%Wq)k+(%L_YM`FB;Qqn2iM#tPySU>?h}RRVcClgC0XfpBIf@kS z2A7?r{*t#oonWB<FKkx~O=SMMpy7tV5?$~BjT~Rrm`I-6=BIL#m<W~#Mm2B6oD!Qy znuDgo_7wGp55uV9;vtLk#1KEe;UT9f^m@rYD$C{ZmY&H8C@F5-6A%bpQAg+R_Rl84 zLfCRaalwhZ8Mo)qqhf9wQLh?rcMTu0yDX=5KpS3fj$BjL=t!|U>1A!-@obIyLTmiy z9WuvT95;r%PntpPi>Mh@RP#s+Bay{Pj`w9z-RWBK*X%fbEkg)Ve`@EC0@)K%eO)GV z(hKp4#Wg#K<%RZMsa3@N45ijZZ5~db_7n5@tr}Bq-WbP<$2y5Fm5E7K+71ABbjDW_ zBZ9*HL($STD~^Q-i{jXdA|ESQhWmd*t<vbJ+mjzd(pl3xg?iYp_w*e{(+ssvd`zls zz{g`a`P$zYnkBOpX52p$RzwrOv5~t3i4(%Vw~oeNpYU_vetCi{q)?v`<g+l)Tj5dE zDw>KAg?(9tmCSFw<luUkTbvNW0Wt*Y0p@3}Y<<eCIT|qEj5aS~&Z4uiv8gaMT(FM= z5VinD2YBcNdb6%`4<_Hx8l!I|H60@(N+nglP3MykEB_ZEOfoG`6)bd&{9dt3+5>VX ze(=6@U~$ER=(Ca*+yvn416;oF6fQRQhK<XtG9b_gP?kWR>33zy9p&A_;m%qhGTgtQ zWXi?{M&Ng>aPVTNQQPTPJT<f=hm0ttjph0a5BDFdpMPBFIBlKA^i`|XYPzV$Q6~7Q zZ;WR|IjR+(m$`aeTyjx923MQVHp=9@MlZBed;u>BIp9erPp-wE0!$~_$=FzLf>4y` zr~uTwmgI54$-$0I3u8*|aXfo6uL-}GQ`_+4hv84@>qn0nG%Y>0Hd)#num1X#tHu6x zy#PQI!LaXAG9>Cb;LW~&d?LaO!}zvh{?Y3^-J|H7$zG?0edi8SUeOkqb|(E*yyw~d zIH7oW<TaBvBv-(VVTm2E{2)mVVWy^^d=RV3ANBiXCnhK7{R;P?tx7e0O|T&vW>r`C z#YE`^3Y$Z?1<i%EU$DS&6TZm$&PhMBAR=g9$=SK8ZU;tu{iqP=;h4Dz@PLs!WS)ye zDU|?Cs{1W|cdss*v~~0>k#xIh#meP{`2m;knR>%dfL9jim3;jqq1O4GcHn6q2A4oA zOu&3>9FaO_sQ+cKZ}ey?=4Fl+a!T7!&q~y=7stxJqvJuEPoSDDVUU^y+_F@Qc$F8> zXe#Av=99PnhEHHU2Vl_ggqcddy?M5O>_;Pl`;^`I=iuO7ZNf!k8i9=&2cmsRr$7># zHFKzaE=9+hs=mBtY)<{A$B0YKN56`~elAEsAWyf;<s_kWJ~e1{o~!XBn4OC&A><&t zuT9|6mY?~~eWOa!04gq;x_AB%hCGj-d#uW8Q8+nLLzf5y`oQy-zBZtBe*Yf+_3L$v zLck1!1;Ar>)K+#3qEs2w$i%6zK9#Dwz?-Bg^qxUX0K1(KB9Si_26QmW0V!yv@sv(S zM~DAdHjpkwPAU;`UMMau_eKt3j%7K9a!UGkbgyb30W|Cb4z7@b5B-u=+?r>J1&g57 zfKap_6WpYa9f;})Q`#L$!HR}UUHQmK)I2=yqPW80r9mnKhk~1%o4zC=8hd#v+tXYr z6NGqHt~Mw7@q=%$IKn+2Sq^zo+iM@32Q3^?X@<L!`0dtmOh8_*?l&M%rwfuj;Otoz zYjTxd`s`(%8vRhs+RSo2bm(x;6fG@`NkYueMO>JEV1elBK2Z;=1S_-84}bJ3T8u3K znGd2t)WK4*K&xiwY)J0oUG8=E6J{dPAT01yjAr~14$dccw=q54r#EJQ?78SdTUS=? zkp^@tuo(~)P61DJQ9b)Aquc;aqfGzO{#w?(>e&P?YAX<{#3%u6$MM?8Kq|%uuwy0! zENidH7kfJZ>qJ?pRCM=joU)Q&Z)@DUPv^6k{&<~h5>EHwhoKzHn4zLc0^q?-V4v&$ zNq{L4-p5+txH(nN^#(+1m*S}1xmC?L>pA3l-OjwmuMTvQRwNel)``C{*T*uTpjdck z{M_az61U9^(%ueW>pAW|bSro3omvi>l}IyyU3&6}a21C<zN&o0t<3Dme8IcnSFN_) zXXC^4B#d~n#zXfrlXZ1x$-o-+^u(#<y!hEkUzu9*rQMe~7+2L6PUBT@-+mabjRTL< z*16tDsUnZFXD_e(P{SL=)P0D8A0rQu?(oqO?`skofO*12@3x+-f$NH8l3dOwfEsLG zve>++|Hy%V(%KVqr0S*FK?*a&MbJe3&(B}hU%2jK*pt+LB=G`_M(II57XQbZDIoH} zNa88X+#NB>Z~0s-Z~s%V@FaPs@lyQjcs$*y*MKD?lBi9)iaffwaFk$7;Owlk=hU;} zN**0zJ+vKGbGF6PwVHl$e|@`U60H-t;7D?^H|zgl?=8ckjJmer0Tfg~6r=_TQIHPl zMoCHOF6jp8#sH)d1f)w^8io!*K)SoTL7E}IJ*am)&;1_97tf#fJKpP$;4oM0Ywxw! zI@dbSmA}1$l2g8VKI}y_h+p@MaNon)9~L4E!*5uv-mcnMOmJ#Mv2fSE(LWK3$@2mU z7gK+8bWZ>$QAOiwkkp#sp3PWA*|_R(`m5F|C0uz$XX#N^sA%@ITZ3xLr@|vqXfL%6 zr%wA$ofhhZws~dnWqGH=FsV6H(ulI;u;=92uNj|Z`aP}38_Fseqsj?7^cq#K+Jjsi zCUP7vNXMLhHNP^1=G`bc*@nwk@_MonOra)}e^<@$m<G<2$pi{nS*nl5XP7p^EXjgJ zEG`LGo92sCU55{PvHOibm~tLy?Zgw_oivcbi9J><d6L?vT(q<<KwOh2_iSi{6#b@V zd%IO2)J9F$T-Ly9gPicQFUz-H-vnPaR<g7vu+f6zu(yRHWdT9T5x1qSgLX!Dg-7jZ z7(u+IS){qQZ}g>unz3GEn6LPapaWsTj+nr=C^vo$W|N<W(=}XstB-S0>6+9>UhBtQ z%_kZ%3o!tgw`=Y7Rtd=psIdwgAFo0x&^XIP!cT9N2~E)g#c)WJ&t5PPP}gQ#PDzar z&2p2B&5JgnLKS-(e0_yCIc6S5JIvb8bis)yIGQmygf+wt40)NqMset{<a-=JI#_bc zeQrKseQ@7=BF{mipgc*ydOA5&GowEa?(sI-N#9rgprzi|l6<XZ7p7Nn+TE3YxMr(S zZ{;xQV$ZX=^(~cN0p%g|&BsYuOp%8?M;lok4@2@~J$dNpN(=#^&MvdnZYAIAHoCm< zgfidLzITtalUq5Tnc&dQViThFlLaJLIh7vA79UAB$-9z}3n@L7t;6Nsu4~*d$k<gC zKocdQ)$$_|!W{dieVA+sw~pOeF`2M1o>_LTZyG*YuPOy_5zQ?m@nE1!7h~yvQgbVa zR4kP312UzH(6{8&{o#ka&t~DG3oJfb8}MI8B2X%Ldu)BEI$Q*WoSU`IvmK-G$Ha6L z%T7$kN&9#`R$NORsq!iX-V~mBY&cgl`SXPtWFjXFlgh6fW3YX+<ptG8(Pc!AKkHCX zLjAgV`e`^Iq72FvA_5bgUzmk}^WK7k%y2vQGDUJoZn~rlJ0LS<_wZH2lHHP1n|Z+{ zjgetgwwl^9YgAQLTKKIzZ2h*_SXGTQ1K_Q2<i?vcyg<u)_ZsWnY6Y%$-x~IP`qjQs zXL<Ethq3oG(0NWXIu`9;i<P8X>?o8)CO!n1A5|v^f|Y&x+8Gdt&wQU@bK1mi9<i*j zmJIpXw~3-9|HC)3`kRHpGHD$F80A^Qp#qI-0(Xg^P}I+~ttXAr=opb#ha6pT{hv7U z&^<hG#blryjgL_<ps)LepB%-lc|_7D20<@-N49lfZyLIsay<eLYha3m5)~pE`=d=r zt|&7rM3*{twHHS`%+|9S0Qp7JA$XPjA}J6f?(?)`U}fl$1XEi-en^G=HwQlGUFbK8 zU{n?Ia*A;{EA7PQsdoFj$I3nrB}-uvS4I}O9&z?F2YlF-d8d~NJjcFr<}9W5ZG6kz zktCV0qkXyD(}!2h){QNamS_0%bgz|9nLT$7*fv~PIQLGJ;Vn@P*pgaP9ePUcsqnhp z(>DtQt-1WHUw&>4Sq@na+v#2Lm0nID>2BT+KxM*)%E0@ZyzueJ`qn9-5`BgKLq;aY z$5;}VL2L3wtd5Zjs0%X>KmhabmwVe}=uewX9%o^7nJcnwR_L2-RaYlfA848F#&)4i zps!*3@{9$lhS`Sp?Ob^j-GgTI@?T>B=Lr%6J0z>adhY}pG5100<l#zr+`idaa_g{4 zyMVi0bbN3YByA4Qqsxrh{;2ji{kw7uMdIBFm{b>Z$~(}P9YMIfqxU(Q@v|iiCIHG$ zKah+J5?nd=BoR8fz%Pbm=+t+YXt4^|CeBqi&W06a#pUGmUVO(%(AGFtRN0A)y+EUA z{bGzz0uc;6tT^)FE(s3Hmz<y(qy4#hY3cx~Dlkkxenk|cGWjcOs~xYdzY3(Jy=8zE z_oTQ7QRWdZ6RjXuq=+a~w<9X$3-{1a54IeZU?(7{pOLLFeAi2cbG0g#TKf9EQkmXb zf6s>(s9a+nSG6N#e#)OX?UsobS}u({<Ia>nlJ8O0PHzgh>`7D8VPM6qvK|S(8w*+e zUXO7T`x_oTh$0exIf}uS`@DbPM@>I|oT9{F-)j?K)zL|)61xCx>0PIYdMxJ2Pd2;f zq3=e=eKk;B?h_J0qGGwA?*l1q6sdu+`>*C|zD6hNtYS#934<~A=a6&PT16+KnDZR+ znkz;2@)Nd>kl3iz?maJQ4<38h=E%F5k|8{fvsP4!OfFIWSY0E%Am6i>*Xq*_e)|4~ z%kAKu^TZ<JCiIF`w}HBE;_2xWdMTKg13%M4KQ6m*jKfRojZ3>i`82T!X$EYfE-uUk z@LlT;oM@H3Tg_Muh@WnFoYk}Q65{a|S!dRHIS3pa4|#SU9=lzhhxb+tsY{Mn&uDw- zjP}G3+DudoX>PxIVFr<BB!0o5d~fDj+-t#(E({sUuV#O|_2yvI+HfLY^7}o@>a;B< zI*}3vzPb@SD?q;vCyu+aE}{P<g7Rd1zSZ?xM<_;wYR)Nbj)`DKYJF`Os>Gh3s(;IC zCAB^qiKX{R96}sWZcmf08NZu~N2Jcobk1o7Thn$_toig&Fy5nACYl<g7b49gFU~GD zLkf<S0+CuCJ=ONQ4FVN!|Bv31cT7(}mGh&@tGYLK=j*xqO8_1-a!%j<vEk8kVVbym z)AOaOx6w?U&>QyeG|c+a6WB^eXsoQgdRP^!73tdVnJDZQ7$kE;RZAuO`Bqa7c3Y$; zIvf0GVzmf3oKrh`AyOuU^yCQjml19Fap77{kmWSYLuHiTw1dK}nE-QzmA&O|g^9Ol zG%-BBTg4Q;(8Rfg#KwPeg^_T=e1>%DsLY(h_h^i}rRYeFwqJM~%Qw`f<41~)3`7Xm z<^9w)QQSxjeIR-6`-7X$m17>=2%NrEj;2YT8*MV=_mhUvRe|>z6Sk~@ISJ;HFuFK~ zU!}(chtJl=>0$2D==@SjSO_euXn=9x0yfNWbT6!5w<((RToOJW8@zmySStXPYY=IA z_({pZo>%);W9o_&ZLM&wecHs=*5RCQsgG{+IVqc@7Zx&Z+RQe{HZBuMEIq({VKM4V zJ(<&bmYZAm&_P{mn`KBgc0jm)-(&N{`UQQ2jHu~;L{{Csf%+frTA^yH4b7tre3Zjy z>P`HgRt79Q49!;#)@1$4PyN4X<G`ykJ6PV+UvzNFtB@XZnu}BIajGLd3gp{NJ$pd^ zF}1$BZ#b>>C&S)x*^UQptpmPy(aJ$sAnuQmx5@03;aAEeXQA<|kkq*d18SO@mh-WG z(x5T{Qu+otyV>SKWj>8zfd_b2a<ru9@teSP!+2i`O~N$`eHgh{Z5^%3+PSx>po3LL z0kx`VPW>Jg*&$%;W>jUt4pVr~73KR8PfFdT#Y#<hh;RxfP`zbT@**gQZ|@EgevpPt z^(D3yh_G-XHNx(>(I%%TN&sv+&`7sARY`q0I_l@l`7vWm6Tnyb$tF{Eem<q4HD8T1 z3{Nk3A>l?<n6jWu^<?Jg1SB(6Us2$>k%{K!3&)9#81fb1N`J$9)wekVu+nGtZjXgn z`Wnd)WwXI6OU+js>uJpD5L49_E0Fw6X|1~WboP+GA{FeQmR&dn*fMrO7vFz@d<#*% zzS_H8*{eNNf;+Wp#097R(={POr<Bj!XG3p6N39ogTZbvM(<x$q4)a!7ZirP#@nGH* z-p)l;G#aIeh!3e??j=b(4A+9?xr!~G3*iC+hKW)%>?7F=*4=T=t^^_yt6n_iwp64? zZfvA{PSC`FR{YGV;0rd|AQtsaBrMDkVG^e^u70}$Tus2TEvkpHw(KN&5Vz{b+ewy& zy)t-Z_++P=P{qej?`ZR41c|;C!WlPB&`%`%=|D7m`V~}?m^gYo>0W!QaO#loYs@@- z_XIx8W3yFT+c`F)S<2XE^c6o6j#Jr}ym6#%K$+yF?=Kt4B=VUdy#>oH;<<(UtwVP% zN5{&UpWQ<VdMZ=(=WP+<YGM!ejS=2SHOVrd(hDaYd>CFXQ}*-XIJ)BiOY6P^)Kc<5 z_Ud5jg9eUThHMg^>-n^szQUnM6Pfat=A2irtz{K8G&okDk>^&MsETTkU{VTr`4;gL zVq?Bt-CHmXz7?w(#Z$vBpq$*Ip5#$GZq~-o=fS`bOA#FWG_5Wph2%--<@+$N2O*?D z&$i9}c*<WzmYLS);aTw5uy_;Jd!IYxxuU`_Mrck!-V}j)rY}k<^yRl-($B^wI)vLp zRJ}@xuC}(?>R+a`DhOdHe-uZ-a696~A6BL!nUcv9wObUgbxD*-a&c;aOBn&iPx)wc z!3sm4BYPDw6r(SS`;=rbO08|2-GfvuuQJsi77XPt%Bfqe9HL=35)7#2lUpp-DN%Cl zJI=42ib%Ln1=?;MHlws%K_8TNiWns)REV?P#@w$=55F{WK7Rt~{&aUM0KzHM5hMA1 zG=7sVd#%6(I~(5V7QI}g<nI49<x^kQhj60Z${L5Tp4qCdPbqQBEYU}gId~4Qx*?R@ zbu%D5$00_FzEOLqY)2I@uecdeB9v8}s_n6*vYd=777$?U6P0cmr0sDyB6Js~y7@K+ zq^b<)1~&zCVRJ<^;TDsnY>F|$C?ssRKQO$pD{g9uR-*f~ORk)3#Jj^X%L;}$P5S)} zw<CUI*f614Rs;V9*r(gX5fEZwLl7!D>!!_d=MNO_?BqH8JondS5q%9W4LV{`&={18 z2_+<xoa>5QYCFHUq>tsalJHiz&1Jr@^hxcrE4CXsUYt!Cm&<M3z2NH+$=ha8CCDrw z9_s$0WFG!ph%z*fR2`R$Mbm(1<&;IaITF6#gu-xC7~vaUx+lZkizlr9w$-&DCND~+ zUN=$9LssBqR)=z{7L&{&p5RU2QS7izh=MIPjz4U}pF+StQpPuJcQM_J4#3czdQ5p+ zY5SFLo|Ixs%gB_CDZkvBc6&9PS2cJ?B6nr<m`!;Js>^HBco@Bdfb2m0xcM!QyPZvC zrEQk&W$FM~hfNb*pNZ;nf3Z$<QSzgxw1S5bpnCu81;Y}$5U!S)j`*hd(xOp^jeGAn z<(e7!HVYFT$bpDz=tw39>&@N3yaAAP<H-s}=JyYDG)kD)Yb;-0hVwilOR1|ooRC*P zZL7dl7JZM(MzW5@v+Bs8UZbg8a9T`g)N2#Sm(2V*fU5|^NxE4>VW0%Q`)$5#6KxSP zQI+Kp`*Vq&OjvK<I`^qzy;97RnAC*o?~R<$_>UM~tEIy*v!#f~%sELxsCk{QZ))1} zkZgEq({`!0auoCR*(@z$?AR$kGtC0|xYNr*znsr?^`x#RBP~1M@p1CWZs=}u(l4gw zDPDNiyF>&l$9)l(Sk9<RZ8MSZGkdB&2Y|e!=wMmpxP|I^uU-qZvg89JmdRqbp6{<H zpqPZg<#snlgAf0V=_yF;^@2c29dFvD8h(;NBtR)Epzv78_HYRP2Ee{-6CF}Xo<34n zuXM~)5ZP4_x5v$W2GP`;qs__`I%O+%Sabg<6FU-q832J<W>cf4>RAumrKMlf+W{S+ z55b^H=lOal38{S@mb#ycrzh(eZCU9!tepP!MC$-Qip+O`OJ=+SBN$(N@|HM>`)uIt zvYP=I38hK0GP5G(bF<&^JMX<k<#yaUO(Z-akW{aHfkLm5bPnUn|DMcdj)v-Wuv}l; z-)ggZ>7}>Emtu?gN?19p@1C>)TX0=rf5xkm7d<IfB_3{_WZeueV}9Bl14t7g8;2@| zuQn#!gbN)n+KG^1i9(9@X|Wyfjx5f(y>>**g<Z|}sU3Eqt(x-5UM~SH*YLA%_U3c^ ziya>smi}W-??F5HpFYYt-usJ-20l6QCMOGhL%8p|BN(>aqdfFdSd&~-a*iK9Yc;?7 zkQuF7Q~%1z0-l62j*qm^rmF1)_XOo_2!_UO*A2xILeUs*wgHSRliU<xnj(|JrOPwC zm;tf0j)4*T6HT~Qi?pBXn99EUi|6=SFsnfBnZB68b|xVuuY)gy)4g{XL@v&9@>~w! z!hpkvL6oGWNOgAf6feZK_`D{jWtR#gm?N>G6zGD}GJ0jOhQ9YejeqBLu21)cUi_}M zSzcO7Knm*1(s^G8l=H<lj?Ns0$Jww{r2Bc;pCS^7b68U&1tlaUTT1x?rhk=(L{86+ zT)>c@o;q*G6Q%mf_&!!QY1Do$x-|*s8{1Vpsbclk!+QKpr<{AeSYG7#3b<Dmu$<y| zf>qMvbyUUa1INew`0RE1XE@8Y4&gtC6<b3tT6pW#r+#4PsfX6S4!#UATrmiBxtW^M zZ}Zt#)xLM%I6*cY2?!r9M0=6|<>y<GC9Nl{QP&2$XeahnF3APgFnvV$OWLx*hnwi7 z58(OiUhJo5>d)3Z8PuJt=`^Z(F)AVAb*eMg)E@l9<{-J)%kB^p6JxYdH94sn!)_U? z={ROMziZP$Oh9k-`~@J9Ed7)`Oxf-4cX{7!H<D1^u`I0<NNDA6RA*aaIx$?(G2iTj z$-Q-*KJf%;@HW+4PgmJdXUi?ga4O8!(I%#)K)l$wF#GZ>cfHXQISrqvsE?3W%@>lv zaim<xx{J5Ub-%^)@5_u<_==N{X-o+#K<_8_n$b3;>tSG)Q{~-wjdjPQEYzS}7#ff; z1J};#4_2<|&3PzrHC7+n1dvfQMbek-T8;)3c$g%kre7MZSexkiXB>)Mm!i8`p+<eW zA4T3vd|K(&X3Hx4#og+oL3gB|^&B75R_zh_)UwHnyXTLcK5@^d;q-fa;kWpboEYiQ zgU@ulgqS?l3^$oCO~Y<CoQ&)_9+-Qz#Y^V7s0a6$4!O1TkTOjHP%vi5v~<3+MD`+& z_xOP0$JHK*-HB<JqZ)j61!PyKza~9Jf^_%w^<AgBfu=dZjCsOmppdVY=ymC`zY=F% zN1c$6fJ{~faxiaU)Wxg#Tc9UEAkbh_+^#R+-Q>y}1#aKDmRjZd&LU%dY5&7v{!>i2 zpVy?RvQ^Q58N==ihAOMG=nF(qL5sSFhqGG(C}%UE+T02%#qKQWu?U5}xz@=lmbVQD z98xq3b3{c<zw@+}j&MiDPmfEhcn^jc4iyMShz)v+{L$dA9BlQJMj2~gfztWDijI@U z@Q>{e-8>CoY()cModD+NA`823nw{6~r(EM~agkGKN!XF>JvMF}l&5{MR5&aLur|j} z6W<p4#%U<t`;u)=WZ)Xp8Re$7mitOuTQBS%P3(M7N|XazIDXD-1V+{QcV81c(GLM3 zSM&YM!MG;<D_clOCiOT+hArw&&rH}73J-|`FZqXO3Pb`)gQ^pXqWpJ#5K)y^Et>9m zTi`8BjYuhSB|0q=Fn|^laFlGWU#|yEop!jW^0tV>@2UrTIhDrqyRF@hh7cIKC*A<H zK`GHQmq+D$jp&RZ15b<=1|5`yB4cK)>M9w)<LWO{dFxWl8tkD`eck~cb(u-opAg9- zDYtdS60Vxl@d88;y3jHp5p$VR2NN282Ff{DZHSkgeCY+oi^bxUANM_fPXHL*U3Vyi z<UrjrKvFPnasl#rJEBMkP+8mjaV~r944oz^D=WG7X86lSE#5EQ7>2P!S6-B77qH1W zFD2&WXrY9>T;$*2{dC3)Dh#M&x9ldtMj>2rQ{Qsgui%-`ZRJQj%m*euck;4QIHoqV zPvOb9nxz5FAQN@hTrahQ%cfA-y(FI{+L~TW2y?ujqR)|D`Bg&|b;0wuKIj4zK6M)g zrCss1=_j&BO8LEJ&(h1Mu<=%3xkl*v+V0o)I$5e7#u)T1zyK7MGV6yb#(JP%78%QP zjQU9{I%8yOP~W6X0ooN7lCOdy>2@k4eM-`MsF8Ay&(Fa@0Bg$+HT&QTP2P5Rr)60q z(fd9L-cKv^#nOEsCxKCmm&f;MDYu2IvS40bu;VnLaGK-vOm{1jBm$8kR_vXTP1-%P zsC9Di2*hK=O7Mg43z%9AJnfp%xkxC7Dd;fL)eOfk|NL}Xcu%J0LiCiDfr0&MUHPgJ zeIc$V+IdMC(tQs;wl9fwmGDjTl8d_j?5NjnJX}m5!t5qa`j%g5>dWcQHKSi?d*6HF zhBV#;8|Jys?icU2nb~pY%@-Rrm3ImTDNTt8(VyL|GhvVv8Uo=Y<C^gC8q2YlG#+ET zz-(C!mX0Y&M5@}ztFD{k&n|dh5|Km18KG8yd=F#kzY6I=K(kM<kfI-rm+!bxYtI}v zN1|$hY=tA2{c3aXMsssrp^c=LR;~L+Qhi)+aZjR$j&lB~24V`zXK@EI<QC02u>1g! zsMygu{nh8$J+UT)T=FO9)w!}3O#d#uAql+Ck+DKw#YnXbWdw#xYvqCZrM#k|394d1 zYU$U033^avAHZF|-2{{<IabPljIV(DwPK?!R3xT*b9kI@r)zXB8}OSNsiDHX-&6C9 z4>}H1BS}9i?|_~fSh<(UDerzdQb7%RU%js#>eeOEC?BtRWIa~pVBFpa*>xYU$hN-u z%)TB3eR7j)4y0uPFN@zZK%t<<!ROVippd}0dOHS+$c=aQJXrCbdJX5NE{u^%LO|qd zJ-H}PSzH2zLsrFl1ASe1Q>;|q4C`kQ1z+^1oh#KXf4Wn*G|Surr;hO})g2<tgm_-K z5!I`33w6Vnp|#DIbw9h?qPSW-Z&(g!6p()G&lYT=7lL1ioc8sQx!)xeIH|cE#JGgM z($SJP1bm@ygTu8@Je$7Rb{%>(stLynvV(YEO*(X$I@~*jbZ3n(SboTtZCrhjHcy(C zx0@ALzoBV=MH@9uG30i7c_I_~=|WXFjXtoSz6OOe=8*7x4;c_}K1HJ&_suFRV;jsD zCvA;A8dA}TMD(G#Jl&vRVF7yO59tGZ#otq+S-wS7pS2CCtoXHDE!TTdADj%SctzS& zG|QmS9tj4bYSV$Ln<9F_v^;-S0v2r(RbioL(D{!_*}}O$Ktwcje-3ZRdH$>Wa!E($ z9r{l|4k^geLw=YSo98nC3-+;KJ*V7rcx2pLPHh4Qa}S_0MdcAiCRrj$UKyt^V{}e| zfNGi^YHbQBZLgkEHo<zAEhhGAF+CLZ<dWb6Nol)J?N+BQYt1fj2`R?NH8`>pco=U6 z0vWL_f(_fjg{vVY9lRt)w&wxj%HHj-ty$6wwnpoy*Uy?=3szyGm+5<|RBY2%pC;}z zPQLEj|8=l!$vm@PCFP;BR+m9D`Px#fbjH?U^2DRxZa27VqjP3Pb;ow29_{oJIgAAC zlMA7~9OS@YUY4UuWh$w&F5Z_ma29DI?oqP-6kgfPJRD<~zb`g46J}rZ2990AEty)% z7s0oPqPuud;_JEm@oSRw0m~hT?tzI?6v)T4A-Z88dJYBcR7xy`@N6?(@g&5>5ji9- zFB_GlqF;5PhBx7|ietm`>y=unzkQP$JO2^;iuMua7v8!e3BXrxQ#oy-GM-^coG7#$ zox{;;h2?qCDr6p|!!PIJc)HkyL-TS!uMHI3%3Js(2Kx!BEC~xrp#iWnkZosg4JF~* zUd7|%;^NwC3+Gxsf|bTIzn1`AXN1&!J(4EI$Av6I$q=gKH$GCT%SvX>SR|vZoMg_S zTy0o?8KUfjQC@8M_zGzh0LLi5l1|IlTG>qfC@7<q!zvO5?KDT*-Qn<kctQeS@(bxw zm`vaJbP(zpd@$-|9ka>CXDUp3wbnPlDeb?k#SR_||H(-@>=|<4L`OJrZ^J`RS%({V zTlV9w9k=z;{Ke}(Ao~L0E1{a6=<ni~c@h9NgB{&1p^*Eu6Z<?<VSo97q-XrBpUjK} z2EB22?80u;mSWY#c=*d>k>f+)=`UZF*AQP;)_(VrKx83;&B>NQjnTW?^LO%n(tLWJ zKxW<5yOuy}A0{FCKGp98kS4JP!!N7`dp}y(QXm1SYTZuuKiMvZG{F}7L?Dan$>SpD zU9A*Hu}m4ZxNJeU3d1KzP%`5CHqYsv$|ka+NphbzC3>S%7T1~4dC+T@LVcC1J{p2l z`C^;~W0`yZ`a^%kGl;jztqR?&c<0A2%d>%^Y>jroiF=*Doe-!<XIdY&7}bL*$OA0# z$&yn~DA$(C@VSTT&t~AfeZRr{9GO9;@)G+E67Ui%gATCl%B1C<<Q*WoQw<c~BA`@% zfEl4wwsa1!Tc?KdR<;c1tO6JLnY^DxURWt+IIdM+0R7dk&>6}}WXJ{U;WrPuf~s{I zL<6c5G`?9I;N??-IvWY9v9UQN2BU5QuOV^J17vZKNc=a=5iXJq8n1XOa+#Tbe~`N+ z&)V!Wa*Ju2bLmvF8tg7uP;K~NCO-v`mNMe(m&vU$IS~<Jg7}7bcib{j!o{pKVZ*Ji zi+avLn4PUq@&XBZR$TTb5Jrm(Iq1Xsh-kglImqG_gZVX@&&|sKMjjxlv9&pQSPKe| z-1?MR0SzlQ;);9x65rbu(BVXzU|LsCm(4*o3L|$~zQ`a2vJf1RU;hCVoO@_GO*zP` ztD8Bl6%K3!_-a;FSKn;tb-wZVwXY_-MMUdx!RIy7_1wZ|Y_tlXXsqgCc5bv#4=Hmd zVZi+Bq|ejrvH)29z$dFg4hp47COwWb7aP8Va*MTro%YmbENGGkKedk!P;VO_?+%=~ z@+GuEgoL0L0nmixgZr-_y_xQF?qjeGv}JuuB}K1q-@h$}3+PwDHO<}p2rW$cJ&yW$ zon}bNBky}J7V<|3dm0tBz&nv4-6=zwR@GbVEWN2i8g$S%@Ub3G%JON~TxZ_NVl?cf zJJ2Wo?jtiaIC9F$8FUQwl6cb&XWk>MVSDT5_2IrPBVDaKgiY|vAIF2==s+Hoy1BVz zab2J*0+jh?NnQ0etfTy?@g3A5M*Z^Ty#6c(vd~;<w{qjlPeheh;X2)}$x_r%ZXHm9 zQ@9BceJ5c}AjSPb!xO+;ewS}FukO6+(3ZWmvyZ>lpgLOUP_DSc#ky)=$)zS=dA?t4 zK|Q2F8Ba?o8rGOSqVuj2G=WN0ny3OLQ%jFY;U6L57)a4i;`lG_ARLMC2l((Ogn(mQ zY(2BTEUTOPc=p}y2XFLVmD=>}^ysb}r_+ks+9`K_9v+Q59wPxg1W^o?e!A%1UmwlB zbM9Y4{bZGSE%Re4aD3ZtKL9KtqtZ8l%4VjZ&XPW}Cl*&@&-h5mHijcpHQ-Lcr#+wY zI8f;*i1TE2esO~l)R&5bT2!F*s`(jJlXVqY7pSkuFCvmR`8+Ef(i|S07HX^y)YH?u zEBf}=bL%Nbkw@>@e0N4@U0k;$DvIZ_YdPUK$wf(AX$gCekY_TEB}2l`48(nNy{<4Y zAa~RB+M=~x+brY^0LcU~D`70i_ehbV<&uXq0q965U>uVdgg}7r)S7$|xk7PV&%@(7 zn?5tjS{lqYQooNR_s?s#MxT8YaZ3R?zsHmFOn_X`5Ob@7Q?xX!BZ!b?F(b|j8{%DJ z@8}pCsrl{5Nus`_SC`!Tg!kR&0U8kJJOXJ1aLA<TJQ+t}_@Du(E>%VR)uTcTBv|F% zkOu!!`9RP-$Jf`l*Y4C>p7><pAsIja)b*FVZ%ex+m^&z4Y*cRKSBEfydU!$ge&9`3 zIs^OD)x&aHi3*vOI-OZ{ns#}*8A>mc$omo(U-#Z`eGo{)hYXH){JZ|LfcPj_H2@Rl z28_Cb!opsQ81s|>bSBR0iofeifsC45+ViNNOrQh~>E2&~sox)o#wgns4cuV>J+Y9< zxc@SKuRjYaHc`@?c3!|lx|i^WJQRlar%&LundPUZqPhpjZ|VpJYBsBW3r5MCz~)N= zLC$~l1N$vyC<xGnP9jTCym;L<N}gIkK)@oNT}060&%;3eYonm-uegn%s;w!}ZKwA5 zMuXtrzq`IMg2w|H(&$7xo0|_HD1VtY;y9<#9#Q|o_YO0Q)^b!EJwYRNCxf8=m5T|Z z1l-y#B6HWYu9ba247~2l<gFNJAqVuCC?TkO2p%ao`d=V4gu%%5AxcNX%#O_TXzty* z<^A?AUFiC+8VK4tJ28lMR#)%+^G>cmf&7zO{Xg~l|4eHpz)xLV=>k+z81Lo8JKL5m zN_{{|HJKj|-2F|ipZGABaf=y2Aw9L;N7TXt0;W`d{al<Dt0aVM45%9;2R$Xh(&<Hk zFvV6JA)9tD-~y`ZuNfooph3l7(KG?YhzjsuL2WSj%kdJxiw#Ny+~(U%B%=aiMuvuM zALbJ1AP~Qp&(t{0|F{AQ1Hg|E@)i)mVv`M)3u{20l}hAQ(`1MSuVJtQFr*7=8=OH| z^q)RsGdFVVxlD6zB_$y>a0T_KEPKU0{dn8$TEp(k*AbKMuC6E`GzMfj3UDFfz91Mv z4&s1XwRLw#0m>y<y4jI2T(*~01PeNdcP+)qP%eJN`U!Rnctwzy=r-YJfIthL>2Acq z9R{=KYS}=(6Bj~2jamMx_U1w)Ok&m-@Nl!!y-Xz!@pR*VL6Z_PU$GHDS~QKeKjch< zaf7Ugzibf6^RmZP!iNO?1jU8LZ*E=@e?Q6Km|C5}0K_<{m}fpr)J)R`7i}MMp+LVN z^|{a`4+eN(=I{eN5l2^jZonN^RN4v$1X^T>|4*QUPOmZi62Y;Ry5YLewdE*jY@B&; zZ~*2l0bwp~va%!!(_Pag?;wm%-B`jW=NOQ*rW1EVSin9u34%*JYI_s&*&V=A{|r>Q z0XtL_N+f#+GK#1b9V_ub@EtxgR!P3Gqvi?rUz=|Z1l`a&Q-1TNU$fG!11;2rT|j$b zT}0MJUgfsNNFWE=nNwPTl}=tR_G1BeS7zJ%<)Vbzb7#OZI4So!D=*xY^*S9uCY#K< z`A?n-xD&9by}#cFR_>q2e_u5JGn4+W`S`D91`_{Qx#!P4{6n(f|5ZP-U;oi~fV%U4 zGa3+x*k66hKUe)9|9$@-H=>{r!OF_|abyEGsplzBnE|s>KJa&Yr@tLl$fN<I`t}Xv z{=PoF|JVrd>+RYiqrQLI?|-)a`*Z)CP$c~8AH7)-qJ^O)#l@id@xSXU{0Gwu9OBtO zEV{2gpvxR+GYn{4|9*piuJZr5zvA+JU5x*rqw(i-;GaI=2xJY|bX@asY{~B3;-g$d zEZe_FDsYSB46({be~ZZhY9uyBfTjihTAc|75|G40f;^FHd7}UG-mX!AR3Z)dT7xQZ zx#VzYCtpAMeI`No((|p6A?i1m9-njzz~Fz)>>&8MQ(u{oM;=p&!@k;hh~IG4GhL51 zrZfVdKgO5JUHjTZb?14%gnS8#X`wd;ehlGzG2LaUU$X>^LHs0555KsYy<*60XVJq@ z;O9s6>y#S$kn4rPh25HlN^#5Y2RHV^kLtHLy#=Jx)~Hh>gpsj(!X(93A@vs^d#vVm zylw8fvzg#oo1<)~GwiX|&9QbkorDF+n_Imh%_LtzfxrKw2Js}i9v&WQ)ialumz$y! zUk9(e6vi*!*`N!uh(vz7g}3d&EG^oi1Mwavn|;v~%sv^8gAm;FKAmq-XRs=j=9YUt zTV9O=@q2mBH+e;BS{!q*zz7ixyl0b=(gKX|?MlQTD`}6jn9`y_N<YtRORO!;P(mOk zAgu;9%ChNL%S__W8To)P8OiFPapjV&v9WQiVyYPglBd@b@>$!BUK%gL_53Ep<PkCh z>kpf`sO|0T=LN0iFA$<FOf`v#iJL9zlP0v05~lqh*XawkB_X9BN3^G|oc5nEo3?_R z$77-ReuK!of+D(JrP|`sy}}SYgYL7BS46VySEN{*(vZ?G9(@@muCW-DTPNs{-q(sY zdM|=GCc_mFy-Fbji;*hW5PG8-L7;eGgl(hO_|U9^;_3CPS9Dogx*-_Y$9)&#O^E}( zCZ+iIBk)bg;tfqsgNbtMq+T0ItBSCW5Mmx6CUSi+aowu0`N?iQ&AVC#7|e^lcPa-b zz|dSDHU`)>b7918n?*`JPiZE@bMfEAJTsK|J8db^sFCiDZH|>FczJn22?)gPmOBX_ zILF+&{|0nv{7F;(<5ZvL&#4*~$z~fL1<tk+8pnTvS}u3SjDckfi5QBa*O0fiwm$OO zmCt5P!Bs@^J`DmpVczp=H6L&GzLAk=rB}fcIkq5hG~q?OSo7afswOzL`t<lfyT`kQ z1$YjnsDBItIIVi@HAh@9Zgq7vi+6i-Qwk!uKnOfX_1^-+;DfcbHmh&V!7%yk&wo$g zOAVYfh&QZvYAQh*BDha*{e^$Z@|3dvw<}yf9li8FnET#Ql(T<_M^F;y9}P5<(}B3f zpQE=w;|b!&@y0*TzP`FDo-R)f=!;Jw$wj}P{#r=p_tT@IqRN8-cKqM(`QyT`AK4@; zPQ~-FrQ6ZS*aD%}K!3kEUZx|=PQ|M#-Cc^uY3Joi;#dTp)_EeB5xwGn;sX79b0L;J z{7H`6YdwiQUxs6ryZB5;4N2TmIr!8Q8~vZwlx?9fsAwM_M=q$99L?F!PlR;5L#)Q& z-~M+mCCp+h9JFXw01KKXQX;#$n&-Q4z@5jWSBopY?gH-!nWcvVi0`j{%%N~8Orz@V z7>f3yN?^GfWX(N45g(a24$N5zz1R>%!VgjXV|@^>wwLns^t8XBL6E&Un4M$P<U=0X z_$tP+w&G@m%M!;xl1>#9J%enpeFyQ(PZ%y9J+A=$RUGZWoXyI*!<sK{F1U2#bIB+V zX7s8b{3L67p^5!3m7XV|<Hsk=iW?$K4Ce14uuQG5uixXeD*itA;BAN@t+L@GKccJU zc2<2?V>7@QzhM;%WcvKpeKk_$B`wLQj*hE(W)inA_GZ{N<4fNaOdgHVtv<+$O*QKh zr;kle_?P()c{L(}-Yt_<>1cc8JhIKVwkBOKWj@TOs8qS|7Be(6AJ-{kX@n!AxP5rb zPhd7&C9F-$ajnsQXo*yYrRoSxHF5h)M0t(^nRKpYqkuQHiM~<<KMCkHL1LFy5$jq| z<Il?D-Lp=#J8_WS>CE~3gSEh39?#@sJF#SeT6|U2T~E}<qr0mSwQ1VcihN_0xn7F< z+mp3VgH<f_Hy$Z_uT32U=FmLrqcmMB3#6Tky0=1!%#W2<$>z2*3D>ulWsS0vHzyA% zsUYpLp*rLJ)QX;_cRhZ;L&8#jV3h}f`6v^kij=s0H@f4}Lhon=lY`h|!VbJ0@s;VL zv8sKs=IBErlAes$OmjbQrel4cr$Qw@n7nZj`MkNCp+lnC6H&XikaVuO1m{{ZEMQ>N zcz|oTt#)Q}Iu}7SkpDvW$qTAzt<$Hbea$;B^^44%99F+pB$drOr&e{CG*_xB@po5` z@F|XssJ4RnWiomZ<SnMy@4r8$!LQQ15VTMDC_W`BbRXK`<6c+6U;gb}bf`*{Tc`Q( z!>yBeRXfamm`qWAIvNd<VWd_2nRZS7{rH><jKLhId~$;!1#Kt3vNIIFM#O#r{q1>< z3p+VEu|KE%7}jyR!>4Vk^1w?pJ>qp18J5fg4JdB&x!Pg$-r2DSe22!;BXRm5P@+0% z93D8NC=>5uqhUVG|2b-^G>2wW?!ntYijKN3iI?R%Mz|)Txw$}6=YvJQgr8N`Mesu4 zMUZaSd~BiA4Gra~GIkHs!>UK2EZv5V^sp;#^x(l$HTdN6$m8yrOQybZv@VW*I8A<L z;^@fv#jVUAMVjTCM^znzXBdk0I#-%oG!`h(w(Dp3)4j4~X_6&ZW0&{zM+;blkx3({ z_fjRqifUy)zG}s!-Inru-Bsm>c9U$~a-+MTpaU5v4R?bIrXR52PxI`4BVmEWN5d~s z!W=WZU8QzTTW1A^dSUZ#Y~2rOl78$xE(;E@Q)Zl$f3RXc^wx}f0k@TC#iD|#9ro=- z$N2LC>pU(M+Zs8Fr!<Y7@+n~JrNsSh_jNU8V%+41_Fp?h-c#bKx_g6(JnUD&3|#&+ znyuW(i(m1wgy5n7tSHGUr_*Kj8g01;`&8ZefYL^wpma^5tcs3p;ft5rc~6ouEAqzg zJh{I~kcTjbNnT&8!X<97F{sr%LAMN|pi`io85!+4DIpjgt#p%Bzxt*B{>U(EVbjgv zFgbwUeA|mESoDVIYrdk83Wl<cY*H9u+=+!SCj~!RQT?%p3p{!2+sv=D=N(IyAur;S z&Xg~-U?HO`s<Nd9?OGnqe#QD9U%HWlNPF*(wnva;k;to2bsQUi!IrD4trAo1wXfeh z5`PWTalYAaw^r5;6l_fmoQbH#SFqHj)TTv+2FxGck{+B;u62~SKbkdiH#0Fe|HQ`C z_BgRLCu98O^j)fuM=;X2!F|fO{tnt6a6g(J{Ys*O`e*1}i@<a&B`@J=xqHTP91}Z) zvLKT<O_#ysx4vrYepM#HYis5TIpjTB{Jzd}ALbt=Qiki+LLS11%!T&;sdZP#T%rS( z5<f?+L(V+!>EibsL3{>Iw^ue4-S9NoO#x#k&x;9%iD$sI5<~s1<m7G3m0>>qaeol~ z^&)wvydseaVeT9ZckW}3<;6Oj6?ElxQ&ac6h;%@qE1HpeyL^$P`YgwhU+v@9dsb~j z`ED!ud=Am1+niUcQ)r<q@2DRuX;)?~*DM0dUcU}M^&^$_o6p(X9nQ^>CyLF!z<4mh z+(lcf-@8|)WPSIL1z5vUhQD_LD!7@RE8bA6pWCJTa1Zx6jGZc|5Nw2ilm#sCPgteH z*}l?qe~Uyued|h`A%)igOda&J^~@$$k9CVInsYXLb``M+%{3CvIY~WU>%&#O)Q&lw zw#Hf3CD=szD05bAZi=M_D|t#C(u~)r;vla>MOHxzb`#m(J4!=^a9CQFwnM}%y>C_K zmZ$ye4(}WBap;ks-sg4EZCb#bQYNwY|J51mfMxhLIyG+R#E9GYGTU#k-<Y##inF`M z=hb8Tp%1Uk3LHufKFj1hG>%^}*;2&V{8qK0+Z=y#9{jWu=nRyi{ar64Zm;!+_Fe3Z zRro?1+FAwRl{xFAELGeCT*is9-3c=hMp8*pQFPUO(jVB&NhLb0)Rk~8{4KRkwo5wB z9OPYe5@mChy?CU*a{7MB2Vx9kws^Ev-zu+Cd~ax4RW1;%El19LbESVr-1m+;WdE%n zX;04LL+VueK`*s)l`)6lur7+EliY8$13~$wYugT3DrGWI*`bHP%lG&#sa5JvuL1iy zJv}|x*Y`;xU4DIgTgHZyuW(LGsb?i}TJErheG(h#LkEM_mz%PU!ezDox-Xe0vEPPX zPv)dWa>xKxpZY3dsn;6!WzrN}Y|ePbP^y<Y(D-UBN6JV@7HDZY!~-{?h4sGK1fvMS zn$1VMv-<Xxqi|Wew!~IrqqbtkyRnQk)xt`|CO7`gy#+y$sje<r+`MQUk;Ow9r+jS{ z+*fPB1mTJP6w!#8KfC|qqmAU_$&#a#oOb&A>|8}M)n7%;T8>f=<rrS@>S$iB`C1+u zd(>ySY7=RjWhmtymp=;74_WBaz*`#fp0`uNfHw&Li|-8l%^9y>sGgGs@0%#b8z&22 zD-D?k3dj>{j~#99EA?!YZMfKuB(lcmuO%q;IBjhg@aQm$+s(hdC!CicM8^Z=_*PL; z6O7+)oUaw{L${ut=lo{A(;q9*V#4!R{9T85Mv_Kbydl_g?v-O1X+I&w=}Nb)rm=-A z$X5w;zxTWNkYu)pK!^hzqW6$O5Zm(j-HRoMt5=89sXi;A1iY$OwXL~{V$}(>=EDl+ zKx1QY$bo4QJCX+$(mPDm$WXQ~EItn=h4_LkVuyFizjSn+Z77<N*OuKn)vuqgO-1-w zP=vqp@WHr$4!3P^%ltBTYl%Rw-LJA+hlA);Y)xSLtB|y=02}h9=C^FF38G6)q;-gs zqa&qB=`H*RD5!^>v<aW1wB5#%M~=~W>r`Dxwj9XCna!6q6n5$_p5-F5K)nvOO9rOU zj`UWx`P9M|Q}?PCY`<;f(DKJ4_7Mr6i_Vo8=2{lMQv>Vt^e)w{$+FD#lg9`5Tkk8} zunvjgr>|9=4t?$R_1@bsQAKEFmOBIaX!w?fOLesSd_g}Neqz#1EpjH-#05>@|5^m# z3CP{2e8*eXibJ1tH@c!Fl*GAaalbqhy+?r7EiWtAT-~u-mf8!?n5~ixH{$nb|6K)t zJf$fE5WwEPn@OmZV7e>DAfuwQoAPKnDGE;qI5<P=)+*W>B`|I}9Hr-u+hYp1EeA_Z zdhnEy86Lf$l*@i{(8))QULkxlMrmnePvS|l%bRGeDw|&O$D|9Tbd&d$WXz8FUXYbN z54-m^iIL&T-t0s3lKx4*)&|W)7cvK1-iOzTO8T7Lw?6!EA)<MRvvT~Si2w?3Xx_=b z!#Skwr+$C1a`%+>rq-8N*Y0@owK5k7<-nTV@z`-2AidfpH4aS~BWJxAsTQSO&3Vx= zRgOt$)u!dOWvKQuUHOClYhs<si;~B%I{${BTO9&g9EF94V=E==6fXDDr+JGmof%;q ziM>J}c%pQ?3}GR2<9C~xwH)AnC2%P2E&qk{ncPdOp9?iR9UUDj?zYgt(HYkvqxqUv zHx-@B6njn)-SlZ=6DpVLtnnjh8>;J2ZZ2!29h-IR;&?{q@EqKx4kC!i+ThT@-lC@o zJ~+A{-x{Avl8J{8)0Hxj+}9q3d3Z%exuiRN9&ymbqoWQ`=94|XD1Ih;gN;&sQ%n(O z%z;d}FAEgnVZkdoS2`2r{G9U7&bO3sZAsMa%<+CwYo|5Y@(NWhg;*=*3<~2W+O;Po zjE-nkA1N-6sq<rnM)rtle}5W8$=iIxxug&cBK6={R?h&Tk^~M+T%-X1EVE~^x>7e- zpv2t-)`FRl$9k#xeZm3vWJ#<lR|S`n=az%p=_@VkUHK^xrk*qPk0&A6;j-x;9NDUm zITL>ZTL2(~jlrL!N%E4tgLE#hHkhh8c)}vPr{RX>Dqtm3ZUN<)*`I%9*61@weNDl7 zSrec?RXkE>uGs_eftp*Y<_C?sHxtG{?bBxQA7?}?K`WihwF4jAT$&*bu)AZFJ`nk3 z2%pj9T6Po#VTjwGt}9=j;DNZzV|M;nF|pj0U@`TkaVChn)UphLAKBln<<<||I%{HJ z$pCc|pJxKeq?E@D>tMp*YEOO#@g_JLFpHP9wYAx4X^$f%(gy)X0BLzWJsBZi?SHR3 zHL$0LlJnv37ydT_+SrgpK|#r`uNSBWy11-~*Y@r2imND!3Is6czrhlS@rV3D6Os}7 zmx?3^ivvmC{S69S!wkvN0D(8)7@il^$X1m2dxECVf5NWq?QZY|m)~FbBS#C)`UesD z*DD}?{=bJ;{DriECt<2^jT~9(bwtr;mzQ%m*xP>v5DG}r<3M_Uhsnv;%DiA*{%`dB zg62G~63EHTC)I@42pVE3B$r9Ai9<>Wyf{!Gy#~KMs8U!6#}kNp=SC(1KBWBZ=crF? zWTx8>%H8(H+pQYex&r5b#EM5qA;y=z$gm@7aAG13q$TCS`vSR7Q+dwy)z2BEasB!D z0MOH2XEuEnQ+kb9%3U`e2XFZBF#YY@w}Ue?NmC|qirg6)2QB$G8uZv5TE6B^Kg+(x z6aT^-e^JncdX}{uRDf^Zg32`3`#rrt1XSd|1FB>X;U({pT88vm#*Bk@+S}fgoa<Q^ zdGo2UKz;k+ABB9&)U>p;y{_Hv?rv675%Zug+S3Y&M;Qm}wA5Q}Tu4lBLl|cGzx2zI zxK9^e?M$XoT0eLGH;09fk1xBRfU&wwnI^LTYc!e`B>63vh^+4gDT1W_OkR5@C$Nfp zd4UO!eV&Xh`cJ&f?-Teli#_%(#7ssU$@IS$3*zVA{(qf>zh4I8|NobmoaHr4nM`jn zR!nhtbd(Vi5~4^GDGy>}WC*n5I@bKVxBv084v&wG|0|m4&3>O__|1QV9uZgm3+a{m zgHyLJfI++QU)T7*pu)HRVwYd8F)<T#FcAO(${KoY6#kjB`Co&E|9^h|-|xBC6X5LW z(T<s=JL!9@Vg0#h*btmmhXO{~0?BNBGYRW2QIPSj+=ANJ%Y}4&g{!*D19|vN5K6(_ zClJbRBAw$F)+Gl8c@;4g?qy6Cd6qwC)z$_L@e*ggD%P1I3BIq{d*dQ;cGW#q&61wY z0#vDCMQ2M>YbtJuT`w3QtS#0$aLc2e<`bS{pIXMR<x(D!P45~0EdJ2TD~(I~@$DOe zIM<1ZmL!GHmyo>8%FC63tBb8GX$Y&VVw<aDPsv!4b&RWp+2=HbJT;{BeHIznB(lxP zkC~>6t4|R1A#^t<P@wowRIZKUo$nvOQgFk~R2JT{9(dckX|udq-W6VcJ9(()q(5*h zSP62s;1*PLB31PFhL(ING6alzQz9{S?U6dN358tkP4D4*TLFL(H^kcad#;#R%GE9n zc>uF?pM!RO#|Zp8q=BkU42+tuO8b@ed1y3c-JOj;JI83Tq2&73mNf8mvlVI5L4Jdk z8sooaB`gh&1@+bSAWWVYXQu;k19APr^8e9hv1+?eAoadJl1`Mk9pQNH!~X170FXe< zTTb_2O!xo2=zA&um@a5RO+3VeIU>tp@++}dO7Qtnq<_8torV-s;SJEw3NeI;<Nu)g z|0k;frosQGtWI$0;h0XQQtn84LiO?2K)m<_20B@$7=VqBH_0ZQsHxX)qnVx_bRXz2 z)KU?9HEQHcU#+*h4ea;Sh4Cd$55?qHET@_Bl6ai>AN)K{{Kk8=y0k-DN}tU9$3ADB zoIDkXhVh+kZ0^REVP*OaPsYmd&gI>PBrEY9Z=$ai2vpYWoyXL9$?8;Cz8vDc?+t(H za;G)nSClQ9=Ibe}{0q+|_)Yw)?~iV3vRj`Xmm14@9XXGecC)%|PvE>t`mKJ<#K7aU z^>Rl?(_=xmuW^4sNyVMBuWrvqa(k^#=MKm<DS1}R`5(yA*0FE$-aq|F*5-oVEw#e% zB?-EgiS~}RGHH<4n$6AeF!OBNm29f|lxJ)qmD6}9-YcM;>6PE#i)?W4CYF^@SJ@qw z-_%rp_2HA}PUxk^65K{f4FTVfc%*W;@uuan0L1Gt66Aph)nn^pme1!GS&rB#!C6`T zSq&|w`?4<w+OMqIn#gz5ZH&y6dJZgKCczhj-@l`;<g~W!etkNM>lJ%bOa1%E_KEWF zGPP#mFRXzwFFTiaMt?GBElY1P1)vAx-BWJ;X|hKdc0HRfOGC!V`fU(#8ho5T(Q#`$ zf`RwSVn_qxErgd6+~fYnr3&7WBsq}bl;Mx}Jtj`sgTcdT>v*6PKtb<(yu#GwO69Bc z+PdC1(%<jCT~^?A7-F4nH{)RxJAiB%D0+wW`JW90vH)TbBHB;^=Uw>Pb{Up&=!n31 zg|}qjSXs5#;J#?`Qn4gHC8*vyZQPRV=jq{1-{XA;?Q5Z;@m&1=@npYlXyzqp?YXSN zW8t4mX)3HnQ#ecJKKu8l%XB@JQz)<sg&~yV^h+ySX(~1K6pi$Am0LX653R(205NbN zna6eJ^u0{T)yLg+=iF#^^0@BV69y(MIc!a`o!%4W)6bTFc7G$_g732>$@*o>UK*$> zH%^ib#yfkHHB$I#0|gTMlqOu$W|uT0okLsc@RTL)lCLaoyq&_OEOPf_uPnnjJH3{t zl9D&Y3vS!#)Pw;Y&9kC|q7EX`Q9|kJ>_WHWl#6PMRGNk#u17(?tbaIrg?fpReEQQ* zsKw&Kx0N<@B~pe)b|;IM)G{vh7SlKNT863G?Gy>sj9mAo<jk5n*Hi}?%j$Rp$OqC) z@nGJRn|T?5c;2S_JkkDei8Nn5|HSnJ^>r#CyG9m%$y>TSoMXAXFZ?l+fWbek2jD-M zBKY-BF;}_)jxrf+<YhjBKcTI_uTtc0J;O+I_Ql$3jV*AXrt+y$-U7(86>sbwWauc@ z))|a#DfI-dut&L6)m%=hSe;owB_4`xDlq?gF!Bo#2<Eo0*bg&p1@m9F^<a*Be%pzP zoXO&@;}k%^ANG1K2_TO2|0Q~ZYtN=Bk^7N9Zilk+bdrLrTh;t{=-pUjyP5h-FZJEB z_I*i)i?oCMo5sprIP_s}z+&2aq!iGWo5iazZAJ3gdF5Q8A9uw4AWWSOR|vl~$!0Tb zU!->_T#_-zmHuccq>-yacaAgzoq?nZcK}n;Dm#9MQ^mIXjU4r|SAi#4p3d<_jz_>g z>HgBj#Y;FZZ^bs+*g}_&O}}j7`B+x>@q6?7Bzn>fYOBXD01V`06GQqr8tys7+`=z< z7&m{};+onht@xeVYNxK6B<n`@e2LRE{495oHI)so>*;mg`adR3=k^32&Ayp~8v|Tu zpg2r~WRf6TTJDC_b)M+!{dpJWbLP*t+%#RLW&oFypqq-@cKLf@GtAgaOU<$6z*J3@ z0vUhePD-S6ErC@whb+S|?!ds+Q3sugJZ)6%n>K;Lsq&N~4gs&jcN~W>ueN!LwNrEa z%x0gX7x#U~xPM&Ur8qKX9hr^dsrzK54etVV7yP0Ss*GD1nhs=&tdsl}Q<oc?EVy&A zu4ntvOLoVR69T!1xPCsn__MtaXq|+qQj|ZC-tFF3pK=h+N+AeGjPjtC>~=+r^~;6B zGM;T|9W8iX)keM1uJHy*lZnV-G;i{5X9m4m?~_KlG+TH0mw>l%Oo|N@G|dlko6nos z(1e8knlT63_#dY#bk&b}F0<-zMkro9E}K7`cgW+LDcUB=uTEc&eNevJ8BmbnrsDN9 zS5d+3yeIl<E0z&|BZ2RDYt5H}11krDv*CYFR`I+4k?NKGzzFJ=Ii20kNPWaj?bjNm z0@ohUKWYJrsjW&YzqicEe;dBP*ZcfD#t)BtvhZNM*&#Z#p)uT0=NoV6Iz=E}-Ba`k zVUg&*(y7Y144zzM?4qq(<3Fle$H`46j&zKbgv{R4?k|&klTcus#2$M2I;k92{pJ+> z(809jp!<1B$heK9BBy^22S`)7c<B&5u=2NyLbD67s#}O#LjenM!cQ~@->NOQ_rb)S z=F82=DIJaj1^Xs}eCV@Pg&5=J9Q&D^rQ5MxCz83VJ3F%V@Gk5AK*qW+N83|zj@y%J zF11Ra*!j*oIf_Pdk(ERNfwJN=Wr0^5ATRelU^l=<ie&oZI*r^-it&4xD*hDsb6?Q$ zFP~SH*MvSA#@WbHiCOd`0{d?W4P^<sWf(b~yjEFyb<MWv(#Vm)T=e`=3vEcQbK+|) z+s$T6;giJ-Ls~z~M1$uEJABL*)hW11Ix#*5zSRu2`KjwtH}p)bfBX2Bsm}Z4h?Kj$ zh3o?*jW;Wu0&M9hDSmU~4{&qsg~gw!&n`yFGgRB_;XP?<qCKSXKCt?~XnX6hsJ{1Y zbVO7Xl@B1HASg(;bfc6KBGRoO?a(2O3Mj22-5@oTbc2!t(gHG+#4t3H0}RZZwNZcH zIOjUoIp=-P{!53M9c!=k)O|nqLR>B}bZvV*1n}M5Es5sa9@s>p7f0QaRhMb@-|V)1 zu$R`K`CfT3#D>qKnrX_Qai@#(VN~c1#ek>H$tHL=eeb2VXt=_ST<WPeKl{uB**VZD z9yVF#50hSF8;{(zur$MtxFbg5GMW2Rm8{Ymq|Cve)O@98>x=apDxG@m>TzAiSeG=x zM|Pm@rKdHp(E3FG{BXz|Tadx*wRX2HpAcJjql4!m>8T<)Em|g>eQHUu^#na;s9jDD zdIP#uk6Q0=Qv~1>gj*b;9FQeuipWMEFVFQFv_AXS+aG=?-ucwYh8(4J{plD$n&RcA zNcn(O``4QofkIl|H#Y1QGciBVL}_v|b2X=jRWh`CpUdYb&58!URW#;fs&qykMZGRu zbXtFT5bdx0=9sOLu8X3|(KtPNNG%_M`dD!NJC?Y+!)f65GIE7yx`}>Ut&@P2^kc9M zTa9~)htuIA#W@4A>pI{Pf)p*xZO+u|*hBYH8;*m$AmWsHu9&U0Ha8(lEemb!yF7eq z&v-`sIu*IQko0Y~j>*3JzrL^wkxb!FpNeJl{}a<Nw{@|RGIz><PR~CIMM=En)lAG- zM)lEODSG434lqQLTshs3LO-{DL=CSwhdg=bExqM-pLD<4a!pnX?iKWiMuKg9V<^$r z!=cfZJO4se0MRT!E8A<wy`j?Q0fU1^T{uLBSg*H*SB4V}6!`6zaJD|WvJx_C)29^x z&k$28LG&J8l13zzNw|<<HLi@BbEd~+<d$k^USvg$*ZUmg3^ApO?87H=_nquK8fWD0 zExbZGzf08etzR2M=Qj(G27nT1gbQ|WO}S4B`wUs@YD?*0Ej-X0+KEop_J;q!*uJ__ zxL9Y~WHO1f1RkZSde1BxO}f8-5@~rfiuJzj7DuPP4>Yuqz3tJ@G``;E7uTb~vq;w6 z@!^~<#3%WjCDgiQa$!SSXJ`T4yj5v4+T_$Tt&`Fr)v7s8w^+T^bZk1MUvV=X=}gUi zP$sR^XYZFz_Wdf`bFyK0Q3Ean;dA%{#iQx7z_9;le0>p_6`Lc?L1@TuPZldK2G!U4 zDMN&bGmgY*FO&Xctz2T$b~q0ZlUH9N&t0I9b*g~-F9&j3P#A}Py0><_&TTDEztPw= zMlW#rmWON1j9mXh>CL;m5ND;rKJ>!Puna3vP5Z>Qk-J8+A3P&}D%%+(Q>v=BndizV z)sJe3<!yPd6f4+m3K*+LXCVZ?A06I__dp%9`3mj54)Eoh_cWU#84)ec`5vI>wq|#{ ze1;a3Pz8p-ZWu$aNTlMR4w@=>HIx38JB3!q+dq-?AGP<O_}>#Zz%iA2?dILv?ga-} z1cF6gJsK4vk7t~892#Y(PzjmR%Kb`qCl=YSq}kiq6OY|P^ee*0(`EA%H2Tz1b1%QU z=z3X=Bfw1ILW=j!7EtE&R=vTjFFbtq<*aTf`gygGVmZlg_!udtrlz*MveJ!Ytaki7 z^XQ@%09>peR?j3INCI#ZJC2fjm|3Z+{>`Ld>k-eVPp$#~sOadpQa`oyxL$EHws4%J z$V}l0E}heJ+AGMr_=j<>?P2+r==17zZ*k&|(!x*2rn%;$gh!^5y?uwL;gK23-6Ly# zI)%b_?}R^+pqc@p5{amNITL%(lXeZCEb624;_ia$KOVETn<U(9=A;p}17xmsTDL~C zNJ*idI6lVf&foaPQZ=$wNb570<gy~>)I=(mD=ZHypUwmlQO;8+&3cqKS<4Stu<6iT zjJGOT&;3AlDw+s?MV76`s|lC96XkcTcM?>WH#fP?87iIudm?X6x=t5gcyT1HhO6h@ zpw}$qXV0S50b@M=e13+7J=*;8Zk#ZcK6#rIzTCUP?{NO-LCBLR<T7t2UxTfS-v&i~ z-zw<`DTcpwcu?nf_Od09TK73XHp`({abu(4pv|!yod?&rGOVz*>kX(aJ1;_~H7FkD zT^jhFpo8#*4=-0}!NY9w!@l=QYq$Nn7a$PhjoTBT1Y`u3$(qCQu{omD0zeYdqKMPA zboA>xXwX<GVp*s$M+qYq;FZYb+eSmL8*3iYs33IW&mv!J#wvt9R@St;suZ9<%MXMG zzj5{x6fVBR9N(D~;#W7Cx8-Qk%A11xbfcB{kdloo<xsYoYR}s$T7;xpjpZRua<b}B zT<cU!t_m52@)o7^N_Y4CMu%s}on&XKZ^im1oOLb-S|Hk}(u@$rg{{0dO5AERbUsGS zU9OeDtbXazAZ69tJcvHe{GL)CN<2R2ey?N+=^!0peXv4bWT)(cL4Fy208q8+N5i*9 z96MQXi27AoJ@XHdep}9Co=&0Cx2xd~%ggk<mr^xTroB6a%hFUeLu;IOYh5~Ln-KNF zhRW#{!N*)csK_Ur4e?3tF$%9CwP3hv54dX+PC%Q-S!0+kSHh%ROCKk!lkMR={Z3yM zzDof^tWDMl!+(5Aw*w3C_*=5L@G%;Hlse>rPL8$cNno0|YY>vwYGSxuf*28zp7>Q9 zY~z>$nV<sKfx6z}%UKiH-E(JqOfDA5$uQY(Ydj;}={;Z>y@dG2)*$IKQ_18itDvsV zjAT@GSz(A{Rk#a1|6|RNXf1_&<HU^I+|r{%j-hB}T>9-k5MR-juYzp*`B#Uo-BIJ1 zN;oikxt*tl<Vh(Hy5Om*<ZLh5g<~*2w<=~L#_f+>YM*YtSCX;f#L~>90v_b{PpJ=m z=1GaBR?G|5K}hX~bwO;qjgVcBT_byb>Z^P;>YXCp{o~_Tc)D=jZtI+6=Kf|<#%8Vo zO3Uq{<7oK|SM~gO?QQ!c$vRAS%S{3{W&3*e<1u=l(!q8X_ViZGiH`cMZnYE<>Sw`S zU^pB;(f6Mro|as7V|Yj&ZDKlR<)K+%x;-r382up8&uJ1N(GqW*>rIaJ8j6*Ueavw6 zDrB7f)$<DJN*8@L^2DbrO^(GT>$hm4(YruW*r@~eSs_3yBU-g_{kiXh0xZDP&WPBo z*{_t9l=6%N_Lr@yK&JO;^Qn!L554hzHrkU<6-N8HTohJ54Bj8BEnBlYG^hL?$LI5G zCESL~quRUxOA`%u(j!{f3%;}#*2X5dmf|UVZ2HxM@rP!`vt&07SkwU6kf+R{8?Y{i zSAep~($0`tpFY`_cNS{tsJWddI3Ks_mrSJ1!EDogXy4ttHXnYpmy_bcuW4!fKFVz` z_0qwo{nJ}VnWPtwW>Gtm@Y&hY%$p};NW1->sFv0b<i^BZLp~<>2ZuR{joKa?U$B0> z3;x}!HTG)OlQ<z|(k>5QEU}sKxr=}HE2371du0%%aq5e>tfcY){@a?E!>b!H4a-}v zHlP?<Plm}5z{VZJcNckLk7O%x7KBm^R^+FmKRIJkT>6l&2WOS3B0LX>N;&JSwakso z3fYIeei{Zk_paW5t<g6TRl4`+pu(tWMiJ#aD9!qdyu<LK+@)d9^N&kAIjX(~88Ccz z+8eXXM>dl#4O~*w)H)Y`TxU=3G?o662{W}TV^e-i@i65pouUY1GsAB4)z@Ur2gvJ~ z?A?{d)CwZRs1z&4aK>ByQ`S^H73Wb)!Il!C1;Zj4T9jn*;&6(UNbJ!Veb(+Y?hdlv z+Ak{^U4<75A6|X5ThXwwLPAu{n;!81S|l*WT)3Kmp=<On#SN*tt!g3KGMrbW4qapp zT-DL1$I6TRPD)X8)w9?_-DvGXV;%P?D~j^6Y`EI$N|BLl)z;NczCbs{aT_~<RnB5e z5QV#T;C;XvNx7Zkrcon}|N6y2jnG%h7lmf0uw%!%3?UG<IADpRV9a&t&0NmtUR89m zdx`_GViJ*oHyqWlL?2sN;CT$0&Eaj)z=v{#%BJzNV%i^K@iQ%IAA_f=^r;LD8%#K) zno8NN$kq*xWoq4b-3Omb=t~6^7aeI+?$wwM;J-$2kVIX<tOTvt<y|0bu>EpWyAsOI zOT{fw-G|>lc!l=J-IH^a81<elL(NfpUo2Uh^<dKu3#Qa)L-3OlA|!=x+9zu!6+f2@ z%@7>q7wjNm<a-3c{xC-Nv#c2*iscm7dPTRr1}6hY#7J}B*l?xi|MNq)a<+Hpb5O@% zLHzK_%5p-C*72JdkusY~EpIw5Cx{(r1PCQ(*II(@#kt4(WD9A@ZYH2cP2!6Tw@3ny zjkTsWIyhD$9(GTn%))MT50Cp9iH;n1c>S`FN$Sw;s~;2}w8B*VU|@^!>&sBfBH@;h z9$z>eIHK4RuQQ~-xFFsbUttik*0v>#>xs^^t8*EqpzlSJpi+Z#+qXB>e7Sc&?CH1% z`>>z&ob@yUq&~9GPb=-G8#yNir%ik`B%8VT?$p&i>EzOS@rDTTo%o%h(~|rvc2BvA zclAz6`^(p-l~zW5rB!)~H)_^+G%I$-oQcwB<}B$FeINa3`hmv8V$kKMUxX38{|3S& zr9}pk47taQ1a#7zn)j|o?w-@f%+qnwo2lq|eLRg`KS=1m_Y4C{W`18?Qw*liUOLhb zMYAU!t_3$flv;osqGZ|ZbQ!9bgGbuS&i$Zz&jR}h>|oJ=Fb{=CnHqo!<kqu;NmPfU zowR3Ls>Oy8aE40>ZK;Jx`A44Z^}xOMu>Wk8YOS4}P!Ob{=K768Jk}*;U<+bZ7-B{@ zpHcsU3XMj$3CRPV%csFDmXSJ(wU(Bm6~Oe7a$cKN`ou*1;@gHO^(WR16BXIu1E_+i zl^#_R;`0V&uXhlxDUt0NtugyNWIRNszrJSUD*@9@f*T~yo=aS+Y9b}lb4gnU>K@*# zl85u{N>;ae$9g!L^y2*#yuGUG{aTVcL`>S8V>T4+9H)8OuMmcORtwPI`l-6A(U7J( za(B71z49%A)j4VWL0_PONXD=>K9{AL%g=GG<7@+6p&P?9^@}>Kb7&HY2-sIDyE^Vd zT{lH!z3mZ&^pw{oy>_4q!zZVs^BsWRGs7=z9DPPO1pEHASV)KC^m0j+ihk`8f^${f zVBl%4GgU`}%fWC<<+U!3>?@c}BjVnII+bT;?l%d^J47@E6GYmPAwr-8daO0h#!Zj? zHE_DPqmr?l>{P~)#N~=YT@%fccuDugO09<<|8ROBFzQemkH&x8@Ws@PnOVhn*&xRZ zpX0_DxyiNELf~N`@W9z!D9=*hcu7r~%~}hnLHWqDmsp4cCN_I+j$a7ZCR7E4n3cC{ z@eEdEGs&JWB4}cb;}tme+yUjzLhI38O_PJ!aJ<<=ro?~JKR`6uTF)yp$wXPJBy_`L z3&jdW1R7UvCi$_Yh7xUkA1@Z6+MH@Y`OxGcNy^3McwVzh2`38M7kR=*3iO(}Xg4zs zzk=egD6xX~v3S++rK*7AQ!z|qM-`?AQ0=*g%{6DvvXSX}2+<Qchs@l{o(FWjtd(PI z>*$J6lXurU99hEbO{Cn~QnST@a0DgYRh#Q2$+?_mTkB1I5ni`Qt@3<vUb02*<{smt zK;iD~O1Q_Y=Ww02;{%3|8um?d5|kWY<tk!DA}emjBX*;U;olf1nssS+_4Qsd*Wq#4 z{FElZk^*}!!#@84xEro#8sOGD9ezh0){pkX6N42K4A%D*vIy=I-pxo>4*-gQNKnRE z*)z%MNLeh@Bsz~N)K6LHnVc?SA>b)d3E%((_o6MH?^S?Bi&(odz8FB1t)eOT?uXlI z2O!g6c=RtPi)>eO{!r&wsC&9;VossUP~Ze?$~BkznHPYWp0`DoVe<`QVF{{HUJ?7$ zLp<%{Hn|pVG3|x*eV1HjmFlt{pA%s#DZ4fr%s-uQ9i3hjCG>0Ioqg@~^PoklJAlnM z=q_l~(*N}MPQq>Z%)sESvgsoU$ER;X==$uZ>BZuWJiRDCSj`JP5a0fAljEbDl^Lkj zT7C(ivuMn+TSPsS&rd-U&%6~`Alg!)X=%y?4j~!BOkyM>v=uovKtkuHF3Uz|(zrH| zvVmLyrA9?m+^Pu4H+uB#O8%ZE3YI{PDQ~BQdkT;{^cPaE=ey0z`!?;mykLyPM5)!x z0o1ziIvLZ&gn#JKSa_toNmLDS-<1U8pxAIvUs5MJhp2fTq-T8*g-V?c8SI@_98s-O z^J{^P)kj0J-0on+9^kD3w-j3gVuw517quiUINM7%G6hMoX9~OayVHMwoyO8e<bJ!t zi|M2{-*4J6zT^S*Wh*T+h1-?{cO@{CNG8aSsRtS~r_)aB7HJUbB{&+z)BfSxCoGh( z2x-&xT{HVS+j`o3DO5}Ef%mv$3-^$q53;&Sn3Um7)N4nJ4;b^0kMEaExM|)!mF5G& za`%U}8CJ?b68<|OFKMG&M}yv(xCzDL=YB64<Ntt_Zz_puSm7jK18P?i5<M*xz^6Xa zXcr(PB4dO-vSNzvif$Wd`<mqrTNb_@v438_<vq}h`Dn4i?)^%Lz4T&5leqhCY$T>v ziJI9Eu-C~Q_OJR~Ee5n*O|9eLJ~emDyw%znf!+{vjv)v?I{d~Gn*+IW)UGmjL+%WP z<xd9<{6$#A!%g}{K=keM%b1hy&w6CtikI7%sNt8~`pOuMnvE4Hv+V0_G`gL%uh2CY zDV@TV8fvakHld*=8SS?EL*0X`(tSBEC;MHiFJ)|^sI!IGTo}Is=Vq<rTsy+P5A2q} znC!DR#^~$C91~nx+UD#zqGxH|9qC##q$Vv1wa~S2SNUM~cIN2u)iyOE!hDx<#zsx2 z8-2$1Lb`hr1UZu3kk*XGp(2g@1q-u=INSyv_oln;AZ%M-LRfvON`17yZ>axB;!;CW zYR+LhcAIc@bZ96F^dgm}rtgovaC4b?={%U6-;^vF!K+w4Ut{%U3=sLRM|;rulvW{> zlr@cmN2GvQNNU1}biH)76MA$s>s(i)*>(QQ0r%b3*VWvMy>bUBS>>>)kXcQD19okJ zkD9Lp)I|ZZzt{PD?!X#YuEp74Fs<a84llGHADP!z+D+>3N=Qi~s$j2YYb;If0RR)1 zHX12D!?k1<i?@EsIJtuJGe4ie=}T%XFnEao*1m3n-!&Oh=4sIMplb>Mbz$EqUyQq- z`wtYI&(6SV`?GZ*dDUZA-W$))a@}XAzioO}!LFc*H6}E1-u`(DO^_jXdw|LDvW`~q zF|c2XZm(Ppca~(isf6#@D%Y=1HHeRoBu?#Q-;53QIqX#PeYC5el#>}D#@?`7D2v|` z>AjznC_-N&l2fueKDAdVkVJp<^W3YLP-nrx@LiUa=NZfTtxk@155$%}J}GXC^xesP z)|er(UVY#AXhg3ty-1$$To5)0SqUFMYZOX!I?RQO!Qod~>T$txg-EJnQBWPFBc~%{ z(_T`0&9!y?RteJCTqABNVa7>a%(iA~DvPvQ&yfegp7=tXS!vVutR*v@+BGkC<_KC+ z?ReK_$E5)l;p|wGDf;K$3$aZ|dK*z*iu|}FFAKH9*)6;}#h$Li`qJ2KzCfco?2rZx zmc6@U&U+GK)#nr+=^o;7qb)i`lEm{Mj3A!+K0cj;_0_9r;=tiYKSuig%pl~lU!JE9 zg$(G;V7x8WY3@zGFZ&GV!#A6!fL(o+Ds@n{)36(<)v(vgul7JW)+Bk#rGPJFWIHC0 z+(7Fd;1r77rRYvw2&%g(9^-M@Rhs#(rf9C^N2N35@Q-t$n7O2pUg3mo!wqUPNyh7Q z4>uGS(?}D1xR?+5#lGXH4m@lwORGg;(C7U-S=jUBw4!oBCKpLmE{9FwH9IVKQ7UI- zSL0JtsmNeg(sT(%u!17r2QGSza`6zDYWbP27<_}qP~e7?IB_`b1Jd3D(0rpEfr0ub z?s8XP>lSA*-zAM<NHuTL^hGeYoC!oU%arZaz=K3Xc)H8P_|$vJRE?C!rGQHV2wu~r zk@mYCs=ZNaQXZ{N+3H(6R89Nx%Nx`9OJAykMH6n<4oESwQ(lP;RWe|lGz{On3aXI^ zeeT9dfk)$Sb=_$VN{$u1vC`~X7g-Q2_4~7-TGIi+@MMSP^DW$<!`YXD<JNrfC6-?I zRojvadYWz2ONqh<9`t<`Zz?&|_0sqB!){5@SBx^m&@AzjYsuRb%hwAH3WnKk0v*YZ z=oGJ_EqzY@M-;EC+vjR19j8`|Lf`2(&7;$QgfJP0ZcDtcr>#iMvGGj_T0mP{?fRxD zICEtnRLWaHEbkb;Q*#!Og%;-kMr|?l{1Sz~wbnokD>E$O*(N?_7Qb@$DD|J84<V*Q z>5>ug0!v(2gl8e=ov2Aa{IOQwG^P>51X~WJ@f;|+W%prWHX8+1U0{z(-^AQ&E8%UH z$vp2(p!jC`*?m}fny@>|<4^ld%189fRBORg*eMM2HMC{Zfd@g_Zqn95)R3GQsQhvj zf}RS6Ap!ze#4XZ&&X5pUJ_V2*m0tr);1e1}THEkcKXC`WpErSj=lsX02iJBIJ_9to zKO|($<`rc=j29wXq_t7lFQN*A7kZNuK_7Ln?=F?0!yRp;ILj~4##^3&^jF@*^T9sv ztKNsnD1#@k_vkh#?clfgb`Qp@r<yD#xMDzGhy}dNuuwoMQ3egnxt{BI?muV0w6Des zqrF1W^SBiGn;F$}*ZAMNbi?G~b&h~;@Obt&`T3+TwA7vu^Di3VZOTniwh*Ncu+Oc3 zTbA2sm@Be2PWiX<{&&kV67aBSg$AN9c{wRvjmP|;1}NaZJ^?;V=t&f|)c8Ok4P4F# zLi8ITo}hlP@Ek-wfXdkSm&)khvWS+(dz|1mHO`jEa%+t}7C_YEi?N%i=Hp<bH+x2S zf9BpGa2k`kfO-s`<rFO3@^8cB=O?AWJ(atzj37pL9MAd(Q{0S=i;&#=L8$l=s*w`F z5<32MsayIiSK5SMOn9I6m*qEW54$sGy%q;Fm%+f(gG5P0C0iwr`=i^G2bu}?v%e=< zylUCwf4<O!I_=*wUq19e8A33CN`NZxy$>{U9;m`Tr-xqPCUpvS^9MAvrTHkelQM&h z7vO>Tx7;G&LR7+J%;Y|2RlYyDSbyW6_yt_}FCP=SiT`Aa{{K%|hRCJ~%o}60&}6je zOGyST<;Z#uXXqvTTX(mW&ZcN@ljzjgSUhuhW_MGwr>hA;S|F{YhaHa!a+3;quVVm} zI;5ke3mAQy?%Q%2fLoWq3eKxBVRagIjfa4nZ%@JcK*H+68X_l$rLH9ATqScj8#!K? z4|aLr)?uB;U_?{D0+4mPbl!htE9mUeWC37f3%pC|f%SX0n?&4Fe_{1V`{pt${?RI` z0Wbr%4)d%_;{f}({9WF4jiI7g(1Wh8IJ#0dcye(N)Q;>OTC-hu)g(xhuLmdt-tMdw zY18)M^nFEoR3(c{{^D~ZMX`ll{z18IVM&;P`iIh<rLl{!o+n3vj4~4ialX&MFI6O& zPnEwXs;Y+{o(FgKD<;qWe)tVQIZ$;uT*}e$6jH=noY34I0v6*E25DFt>-k`20<J;J zTr$cDQ+$#QOX2oPFj`}XRw;|93IMJ8RtfR^mKMLc#3C<UaJ@JiZZ8uR80}e70nLUp zR!=(I>0*s_aF|)Zw!8&P*yyqM)@VUg5&q8Pk&2p^pHHsry*50dDiSJQuV~UT)EpwP za&$DSm~*|Z_tpGo^b10nITXZ`)@J<O!QXG3A*i6jv%#k3rQx|7{b8>m2^J@#7;8_j zvUD$R?h>GuhF{Q!!Dr(}>7!J68jnnoyIV!&8^b<JRaj#sm`YMaruZODcD<CVn~HLi z22LEa$b96benOA--}?Ya)3CG%iA93X@9vy@mZtiDLt*}Y1tLpZk^jG>F#9uynuM%^ zs`@Edj~TE4;5F|FF`a^`mI0m1XBTo(JZ%yrz1&Wi(%tXC_Qii_;s3*Mh6u~5AEpiS zeJ;WTA~s_g?*nYg&5X;g9P~!~$G2Vw|CxJ8Bd9O=&jqb6{gv(dvslxD!EVj!9tD>C zFoD;A6AcbzDnKIo<t$Tu+zyx2{1POk*2;&-VExYlbsQX4egLuWmjh;_$FB6>o+Ia^ zgaCcxZz~1#&;M+6_$^FeMf`u!)8XHK`@}N;#oSCxOdx5C<KyG8aH0S1#Apq#Fg#hA zzpXIjl{!H{oB+yVDKP&9K5!=BTD-vJ{IkcZ9nBz{u=gB@Un~`r^@83Fv#Wlh@1Z`A z#uKCPSBL`YTulq!O0@k__4QPmXjl{NeNRu%QW{&w%tB3ervJH%zakO;cX}08832x} z+=4Fphl37osa%>X1YDt4Y^9(avikPZQd#g*Y4@S75%5BA(1Qd6&p4rb+MgL}5Cl(w z9m@El^z&bX`LE9}Xa4D?EEsmm(;T$By8PGe`Och&^#EU!HegzCuo>x>BuNL{t%T5m zj3BrZa3JNtK<{<;o=CvZ{C>&T@~G|DLPlpLe*#%P@SXo6<A>k=Epjppe1JbQvd-P! zb0OZcv;#QK`zOu>8eV{9_-pe6shJ!jQrcV4tlin4i_A$DcejX~>)Y-X=K)U8{0MdL zzn$u5IU0cX0(ljzwL0O+TqhQHTfKMEZzKBtZwt;iP(J|{=s9qvG+t&Q9WV!0NO8tm zhwzUELHK@Uiv8*-8E5&#?l{=bhnFbqM2~ld&NUcs2u2&mt^T3WfxA3=je7#Xwq(U^ zUKN^7Pi~*seyjC~8n7f)$IPaWo)pmetOmPGx(bI`$7d-Q-={MEbYnojsLoFBz*9=k zV@lA>vv$KEpRAwyWQF{6xOSp8lk=W<6=HX(UJ?EQIrLRW97|Z+2)r-C8f;4Eo}>Lp z>;S_NNq@LLxztLob}vfRFsg1tPv>~H^7s^>TE?gP2SZ^3P$!Gs3^H)K%kA(MTA5EB z&3bwSnKR*L7tyqNBhuCBxRsTrSpW9l)4s<ocP1vuP5x6Vq@V(~8OR8<9ymvNS-h`a zLn%+|8HGcq#3$TL*IYNH->u1-<s!hB^%}{}O08jr04Ug+eMU2vXtbdWu!3pu9<j{; zSv%`8u=`@U^XU3Bqq%plwH3;|^DC4?%t-hxTrP>5(9VSwEiv{A*TUC}5Z_A|PgCX8 zje3i(AuBk%?Caqn?PZa34STPhq!MZ%sS21w^)rxv-v#0oGk3WGs)QJIYJ~`=aRGKU z=H{L|B`jq!@^(}8k7}Q7X0>Ps27mgJ;MJHsQat_EGj+ZD<<%M@3q#joKbkB3)icDx z=G~IVWsQi@SVMQ&;cb*Kw<-%A5g%ubogPRPz4o5Bhb=AbjoZ<d^QgsE(XaFl=UZ)W z;t%ugtVEhMqGIrRqI<f4!eeYwVJM}xNv-ycK;AMYa&yb^HCG6_&T?vEH3{<}=V+Ix zDUA6i8QOUmQP=K$X&W^%QC5V==)KoHEKYA^WVDn4z`ogoH}FEevLYL4MX}hHHT1b| z?DH>y#DvDsb6e?wx+UEXG9A23*+4_nL;`NwC5XPv{Emp&@St<0)Zt#?b1nd=DZG)n zPpItV@$;uLDv#QREd<t*iQf%&IkqgiA4+AS?;+XDyNe{JI89PP=e63|b(`ff>C+#2 zQ<1Y`y^@%3q*bI}QWEWr^mg^tRB3WNw*%m)V?F1-GQHgS(5hp2g0Q{awR{n(8jXcx zyFGLu6?g6`+6sB4t=`WAWir}u`9QF~#`_}1$5v}tFl?oEl~_2udO~)(p>DY$y;re- zg@AR1e&TK87Rq2HB@#$m-kda&%GN)q2}3dJYDg?pual}EwbA_K*qYU)#L~W*-9M}5 zl!^rucB)D4YC^^x?4Y3HjE?XDLGE&BVgMhM93(roUmZ(@@#>Qbi~&j%>Od~gz-jj` zw?=mLZr{5N(Bt>{<J)uRLS|yFf-#?PM(b#`d_}lkydXd;$T~sR-J2jN;!Sol#PdZ= z3)f4w-Iv#LzKK&i<Xv)Z<x_BA>{aJ}N#%FY*+i$)optWQS}=F5gP(}m)QdFbuj5xP zF*V<EykslOUXm(bvTj<9>fNd^*c$CV%cbLWoROe9VAZR*y4@kYnRjPYqM4)3K^dpV z!A(<p0edxry{{61;Br34MYYkFf1df2iUNcHl$pc<Q^F5Nmgg<U=76k7{*B9AdpOjb zNMka`2McQhmaMzUD=|7k8n^EP0(Vs_pwG1;u~JZ0%B$)<GXurz4j;+q2w-uy7z!R< zyeu~y)_oybo8jv}+X3P_g%3ih{63O-2Y+f!HQ9St(5yIhA@_$8%U!PguI~8^q_T1J zTsuvy_DD_RKfiQ+=XQauvD8jU{qZw~Up@U+E1$e=645<5ia3v)ReYrv;q8zo5%Qtj z50vqT%UdNIB$3a=|M@56hN6V=E^2rQp=Ok#u<~M;20_4#PHMm(wRwcONgsHrH)I3E z{>OSLjUAxW7ENoV9@<CygqgkBrY)w3t(zg{2r}Z12?jbYv}%OuA!tSd=Bnw#D%U>6 z0K<?`Z@kzBLL$Q1ccW^(K!0z2YtNLX4c_h<qGdNLMll_UGEJQ=bIJh5PY)H7K~&Sq z^Wc=f>^5*HI+DfLoliZ!1+o^sGoKVfZBqEYJiU2hbf=5L?T=e|6lUqGpxl$!*ray5 zs9^2e!^W+?D($;4gQ_+Ab5Z#KFCZe5*jtQ>zQBO_faW<XLg_a<SitnH-s2UAWou~( zkH<a6W?GLQH%V+KoYi(4Q@ncPRY}-d8x_TqC9KpnX};>>s+9AmB7MR?JqQrn>yxhZ zSLUlGee%vUFq(v0Kdz8xqQSywa(CqH2O9%y01;iAW}Trr-0GHz&KmkAd*1x4cdG{n zLXy5>E!u5Of_kBbH=lJ#t73K7HDYrq6^`(n(%9A$=d0r$WufMn<55J2IxqaLPe>Fm z%<^A`&BaFM$Zv9a15fcgzyPH)dM#gdP-2a|C2|E27k_p=)EGeAea`rjqT)%=MBq8( zLVzhprgx3!Ow3V(bD!jXcmSO1M+U3+W+lxEATWAM9`Af{kPnbq?#yop$Q8OBGgn}v z=JKU3e@Yyk#Lp3FmJCP<@}XH7kQ)hC;QJw9`{wce!a_kT@AreR`F#O1y^4e>3PY7P z+NwX7dc`+vI^A^2>7m<amAy_5<8_>+!N?C7Hw(`}@?gMM^~(*^iRRRE^=Mwq|F*b{ zxFD(7=gvDnU_kD`hOBYc@wh(SyWn?d>0`YzvZl4>uCK1CdKF=+0*|vaP;E=hFcnWH zG6~DRpc){L0Jf2UYpij&{1yVPQNAW)@Ns}vdCe`fR}}M2;9A9<XmEGxo8EF9`H3W& zk-;eZke77ab&aW6M2fD^LMKKX7ltbB5(uRnflm?wbtBY0O0D1&&*i*w_l>F5aTKV! zeohZAae=}FkWB0SK>f~{PsGstg#-;B-ner5AJ8gNBoXmX^ZS*Utj#x{zi?F@_0>`+ zAjd!nGe2e8aA*YmaJce4d!=j_#Vh^_c!*7Y+&A$oU#8@D->#V}=WeD%8U^<0t>QyL zxS(}TpC@pCqV>uO_VGcPLGkTxTYxMU(m^|u5LY^)fHDulPMlT|f+8@0NEj2qm>&=t zni7C^T0psyA#W4*+p>=sjj0bTNpMX=wppzrMfaB3JHD510oCQ`r3HihSC_XJdbl=P zfc+&Y@DnNIDW9=s2``e;7&o^1Xv6B!7L$%Bms8Fi$i0LR(AUw~^)<rB+dk^6f3!Eo zc2;}#$84F4`+<ZvD%HO5!c;5YG`3J{sh6kt0j5At*s6xeQob!Y3+38kbrK%J(vCr9 zCqoW3b%Q?LNyBZ`8wq(F&Ftr15=d?@WRKfn$4fX<&fEs3-eX%y?{gaUCy#>g{Mzql z?=!gYB~Z$gDU31aR-*c>T1ETvtH0Y>uCR`}&jt_s%!2w&a=Y0<_$Y}@zNT8k_PYuy zb~Q)3QYFqn8zD#IqPq2<Rc|VJe>5<09UpY;J$43Or4|EY%586Jp$}UBpm+W|Jv)iu z1T1@#cJ`{8B)ufeL!QJfEX8B$mX>ma%_F_K{dyOduenmZeRz>c?#&E`Nn=I5w~pAO zB&V(oZAtvtkWXe&1poWlno#G)9sbeFq8pHn)Qkd`WEs**MdCfndDts?^_zTUl`_Aw z_K0U4D)?$jT@81(ZRRAEi+7z(2OEcNU)2cE4&i#U1?zwKB17T^`OVnbgM&dt9ibt$ zAm{}KM|l%ydhFLc2@Vt1PSfXJW|??2WMg%!?ZPefhAiW1R2wPshP;efdB~I3KinF+ z7|F*jvmuaIFg(rVbA@h?@_(eT63}@(&Yar$B2P@*eZAm_>`R>0Zd9m`L50okoAC$~ zV72L@Gc$9`$<a;Er7}#r<dP$GF_tz+;_z$+)wKg_7X!8k&|=J72Ljye>*8*=V2|<D z)Swx2O&)}kkICM=l`Zj6A*2kta8C|qTK)M>y$OvQ6vTy7i{usdJE9SFK}qf-1<NC1 zZ8E*R$#tb`lcFL19lA#c=Ja_Iz53{*-O^h0O=AQ6&mgdnf&Z>|`0{Dn<oQe;&?gyw zKcS^_T!p}7m}|mynWTl=Mya?EmxYZ$aCMf8s$(Y>HM-3%o&H>E*3<8ZJhi|}$~1Ae zw}Y#ypaAjxQ8|2k4}s>d&DI&&O~Lr`8dOd>wOtS>d*E$A@YqENj{G3>?8jR=kNI~y zlN-Lm5IyeH=ciYm^ih+T8HQPd)pH&s^fN2AKwz;oT$sCu;9QNlNi|!(rqe*+MlXBm z(>sB?)7>x6sB=M+Q)eOku>u`y@TV3gzgm&}Y|?IthsGA;tn3a+&b?Q)`pkT=21DOe z)TEE?p#Zj^Ts0(w-<?Vy+geXjF#Wcn$#rE<X*a@jJVmx*pFGy0<Na$1Q?CZS{0#mf zK!R)$9S~w|zF9(#-};$AIxaymP9BIHk*>(ddbk-E0uR~K@v9Ljtg(deNf0N0D7LJp zLpzKXO)kAg9hi;%z`Snsau+fgnaZiB_sGq1t078kb!>O4>)P-E>$_*6ZdoH%CpnTR zcGbjgew*}5QX)yfdmJ#My0|bdRulZe7`CL*w3_Xu&i2BZeU?jiA)X(r(pu^sJ4!m< zH62>1qV*-2<mf&a*h+h8YR*;At3V>P|5$AIyT;H-cBeu~x*wLuO2FMN3q}=k>b&>m z(w-cID-XS2vCFNeeDcyWT9YBoVQe;1v-!-X2my=7F9zb96HcEHU13E<J1^vkU&9m0 z;r^!dAvXhwOy^T@W7$gL$@~13Mg^>7;}ex@Klh}{b`o6FZdN<yGI(#cjpHg2%lDo+ z_eky{Vpe+*F`NkNV@+sfL^M@~s5bb#(Uv>lLk&A^{ro!EXBf4nu^G2=-A@Yi1u#nN zY^r%9mc2&|xbJfEsrgQJI7w~weZ!%L(=)4k+iZ<Qr?y<bofa&KNDln;$lL%^KvQm5 z-g#nfJ}-i5#6{1ZOOi0=MAM)A!Vt0xa{qpi`#&ZNzHmc*EVB}>%1v;N^wArSt+%po z9=GEU(FXZQT2!y3yKn8rl=#qaYOO^0t(TOtaBRt4>RWQvrnzhKth2A^bwzTc2hv04 zj$*e;Ok9Uiu2sqQ(unaeTDv?uHJ4r@uwPOIJZxR`VF{C5KU0KcrSt@@O#(Dcn5;M2 zPm0buwz(EGtY=EngO3a=_XoSsoKhMsYnF6}!9Iq#s?m*Uj5B)Fh<StpyA{ndodBb5 z;O`a5l(6VU{COngXLLyiU1In<V5cH8jX9M2pJJx{{D!wN4I4h7K2nrhy1CPLSV}4Q zWQr}Tz+APQ<61B7XFJZEyZ+}SW3P!Cq4Zu=+gNgKZMG$aXwZd+SE{J6)Z0SJ#&?*e z`>f*mQ|90wa;wRPwc2a_<|K=dYnR+7ZK`5TS_Zgvum@p%7KV6jF+6OIxv_>W+8|pc z{`~{-wQ^EfZa({RCu(ZyuPH$CBm1|c#%m)xbZ_z!;DhFIQVCzl)c-b7;|xCBm}*$s z?4a+9>1(^4skaUQ10_(K05fqsR7iy}_?VMh7!*NPEi98VXP$xLzqoYGRep<)Wqnd! zhUgtodg_XyUvbJw_4Rafb`x>z@dNC39WE-g#m8mIo<>R#NOV`~7;@j$+Z4UNq~E#; zA1ehtjlC!UNpD0}8z%LM6t_7$gB(F{^;)~s-rnA!2ciX_)7=Lih~lZfA*<rHK0n;{ zvl<W}hCT0jQa_94!9DoaX36Et3{#lJv4yFGGQg}#_sun^m#F0Wtw^u1Y9CM+ZUX5$ zgj<qG$m3QyxPAR)bfqHWONP1gM`4)igqns;U#Ta6Z2rTqOf-9deaht`RFu2a_Rj8W z$EC4Zr%jO&3Ssis*sDjNTBz@OT4eh55A=+ko7DC<el1}q*$IL0#QQDR?I&&&8gbh_ z?pSG=ycKR9B0ML%>RvxGG{ise<Y*>nH&4|j8g3RwQ`?ePWjOK1cf)I1eAQ{U=t({0 zs6(|a>FB4aiZ_Y#j*K3xvPPlRms=+9`2s5DIj$;zow$7C>$vE}wY8#k1$!MUHc{~g z-X<aMXg4cC;?6Z|I8sNsiM%JwjXgX(k{F&rclK#CpxU>$uV8r}eP$vMX0HVEQn+ah zRNdHp#n~EN1qXS6bg-=DQDCsf_|%j7?J)UG<^{Q~SN0VN*8^yl_Ggbpaw@jN7bHf- z+jaZyb%=b&DuQCYO@M_-XfQ79as^&~x5$|TClT)M^&264cI{vrTGCJxhN{L<yUv<$ z5-2?K;+pqch4(^-&&)17Dc8JupRm&=Hgr^>w&vOzKlm3V9b?7YHy$<|kL2V7-yw$O zb_3!<Obu9)c)g=*;&Oyobv7#W)kV`PneMM*?hO-eZA+!tlEQVC0a}^lz18U&OOrQw zPm?0AfilzjG}-73$eM~hxni#7)!sLp+cXUQ(cw92`q4Sck2zF$aLzFa{jR*bPa{cO zT5T7aR>{68P|a7dK|3a`bLZ`8W)466DVc33g!8uu9x;lP*t&=E2xwEVX*l!G5spA1 z`$UEYib#IfffI6dJ*#vdP{FKd{{$*fW1Uq7tKzkW+)RNbJ<H_iKLuDA7@m`wZE&J3 zF(oG``fWp6z@RXHoCnPR@8f@2sY754=rET8qeJ)SeyUDDjc^*_YzXj+1WP{=$PoW7 zV*a@_RC6az6oP}=@rj->bW8uPEDTj~ejirWC*|=DI<IG_O}-A=sQ-R8QGg~6^7lzZ zNv7~yKKO$qg5R#Gfe0zVYWaKK(svEDzdFd!g`h&{@8f^z*&(1)pr_FXI|%fT2S2cB z44!96dV72xCJ^(t@ISqLN`_qWU_NT82A$GPXFpjhm<+Zi0b|7p#;Wi}bIgA!F#&mO z$&->cD4IX%h`9cz)Lobc<Y+Eky!a86l{pu|oBRCZci#t0K$p<KiPQzW)W`p`F$kDn ze+CUe&#27;y)p=GXxnt}O%jEap<!0i07~o^fDj3&MGgrRwE@XVAo%ZXL;o537S9jj zU&B8jh5a8{E%599L8#b!Idl}LoHXnVDnr_SwpTZo^NX2*e94M2ruLcV@rp_6)UP0* zF$4~OH%KQ#V6vO1DambPDml;)W)-Fb(5l4QjS28X-^a$-poaoB17fG9<;^m$Of`6s z!+OZIN*Q#i{T_fi)=4e3JQ2j?GEs7$<g3s8{1yCmCPmWgM8`>{4ieK!n9Yhc?y@T& zaCm+UvWp%h<?mmSpA2b+XJ>mjO<4y(g_b|skl&mj=oiZ{8h{u!K=ep4tMl>g$P?Va z;Uh$-VD_AUr^tRMvU+T9-MvX%^(-VpqGJN&&4ZvP$JQN$+k2Y<k@FP*Cxb(g?BpsT zS}MjM@Db3n3Ov>C2{f+<pYrJeog;9FyoX-er5sOUnU%3B`x5~*m=)E@{00RV>)TG` z6luFE*}rC<UO!o;Kkq@>NUA{Zhy7#~{rA)dxZwk_nE>)Z+v0{8F>!IIg8%QJ($nAY zf%BjIu;G8}#NS&RABqGX7gC%AsLtT;`2tC2fy2rV;07|}kYLgO=syAA7lN0-<~M0; zCz)#7zoSmz^OF?|{TfV}6a8^jaBg*v2>>(y+mk~OI<pwS9?DK|Bmf`(f&am;_Jkqu z5276W#reUx*3|y@$%mzN9%8XWCzJC;`U#xg%Kl_A0b|yc?Eq2h#>y=J>Yqax6j&_& z%w6_H!%}Bq*C6V_$t?lfbU6wP4KN|Gf0kX;@5WB(?<d-630Cc4SED;XLH~~x0DmZN zI<c)F0un#o$&v1%O9gDZfBW-qSCFTXV<9~DFW~M!Ag>dD5;ll*CCO$<*}tD4<>Pr+ zri@nx6KQVfoheY!wr&gYLr9o>`f0RuwDsFm7DDI+OyNlj#y3u?>6tvr!(^HtBvFlT zBd*lBuXPeg)DUg_F-iUGko6XMn!iyi?BlbGr*lgox1fuCATm>3S$MJ3dT0TG?z8jb zK3SWv9w-s{d~>`aUs@8o`s+pH>M;wZ>L56+dy6JEyCzH2Izv7yx=G(jcJDbZek17< zJI6LUHD<7@d3&q&<P`o35IUE(l;X2@;DTb2j>35itLkwi1sl6F%B7*Y-KRbXd2bAS zA0CdU#B~OqSU{N7Wyook78+D#vuR|11S^SI<k6%v=DOCG=*C9N!R*GKrHE*gd`-`> z1~0Y4qtZm?f@Y<4ol6(00|5E(DS5x|ACPhC1Y{Jln}`Fu4<t!O&^)tDcGZ5_nWmW# zrAka$&yf#wR%3s)MoHhVbKAIYG?a6x&O5*qY_^pGg75I^!gu;i;Uh&xUZv8CWj*gk z%0NZ`e6c2f3_Nmg;>$ZQqf)vExWD|FIQJo^#!6BZur^-B68DVZs}t^}yB46fkC{lO z)-LqY3D|#g#v?bfT>$_?A*A<f-93wg8*ilr^u2F$Xdh|p?RT>HJ@W9;@VHKz+bO>r zZChNEu~ol6wS`hHI}YvdkuIvvERBpcKE&-zcXzi6#c<Z7-^RC5nY`sTetrkRvgf{Q zgvo-YPGzHEfH!V+Q28uZmD}Vi>ow##jz0Xtp>5E`)>hv(9IB{#Hd7D#(eT;E%xij8 znnp|u;#1#w7Fr!C3_1%aPuhKyA*HnjVlNPh2nGOVGU!F-*>D7}PL+A}OOReZ=s5i% zI)f<U>65oxJHn$T2IoUorG9x9TD{V|826pMvC3<7z1(xx2YOe93Fv_M4y{V{BtP7d z`-l-5tdi8Cqwki>+V6uMy3nzOUtjW+;G^M54u-?E^i#jAC<F4)IM?aIZ{j|O<x15| z#&vTM#g{zlO8|!JWA8B;`H`o%R_`lRxB}ZLIJtlgE0~exOYsQwP>yP`r*B7B@udq@ za;PYz%>8ft>T7TA1}NFIa6Pe0UcN1xM`af*qqgc*!k`@Bs6FBG>4I{AX2_S+OEkVB z^Fkq@!!bz1<2X3=@`b8aA=}gPC6OH6Xlm$EFl6va`4)<KAR5#O(te&fkx1tXN1+_* zy{Z4LDV`t|h96t*peC}=J&n&_vHQxNF6SE4$5Y2P={rs}xKm_3mU`2;`!sXOkzdo^ zW95t)L`*^TQYt%kXS(<}w7^Ljbih{Puijtv0z$xRZ^3)S=rs6syl}X$slirBtY7`B zxxDsB|4o|GvpP4wxqa+~Q|a4%($o$$z~>Jx*RA&;YUOrg1O1$yFinu5IbI5T<mX^V zaZRhow&jMd$vj86FPwrmgD1tGy%&h(dq1Z?L4WVsi*m9Q_-8tr+Td8eH#6nULYr-w z44Z93cV1>H(w}PceNz3^<H?}nGx{50Ttr-}$M+oUP2Eo;#ox8~w7rv1{7i`4DeJ5d zRDW>1&{gm)A>YO*-$tu&_aIFD{^UZ1&G1p5Xwx=kx*(q+`^xx<D_Zo09IOIFszWw- zxr;f?--Bu*<SN>AZcT!Vj+)fLFa`CgwueR3cEM>t75Tk1)r;spA6)jTBYyAt@Qr&7 zH<U8bG{G|8B;BWla+f>r$+2cpPqDo=XkOMD)ttiI9g$amzFNYPZ$xTTyOUw_nMsZ! zcI%K$+bZ}*_n>mb8ADUOjT)?Nw7Kp>_Zn@(hWc7xM%wbi+eNd#c;m-@&wUK!*Mydr zmK;sP*=~}gU1YjTMY_+*epZg*t@_2Pd4^A9MjLWjABb$^T;C4_=mwF!NGJ&~xa;Eh zA@qRzwSm(>;X^uyVEnefvSZEGXxFLO*+VurwGa6~+k&fMJ|h7(Q0%*_{W7n4?}U|7 zdm_LDr<z<Uz^TBivviYh$qmPvre(fctYqO*nxh<6Sw3T<!2VLG1{g=7Bn<8@mF&1* zN#;TYQThDM4>^?*Q6pDs*+Nj?Uh$226&uC`k*#b4W7R{z5588Gz4Gy&nH$h-hAmQp zHvkhb(zNX4NSEWyq%PWTUt3Z5F8H9NF2#-778PSiSuV=n%x%(O?sKisG&<VJHNh0d z`ch;UF|s=Degp3LYoLH5x0>YSbWwqjqg2I&7f*k1ts-ZLQ(rAzd~uGhkj8=Stek>b z{g)5H0)A5y2^~H@#6z=fXl;%Pxhh4Ew@mD{^UHw-rpG?(#KOY?!KRa~UQ|3%EcxvG zG5wcL&h0_~F*NjTZajA6EPv;kT!xBwSsFyXRj%<$E1sy&zP&2i5-vq-=khY1w0um^ zLHZ?omET&n0A)$zmTQH1_t28Ea6#|7{kV_|Mh@lBy{yGC6p|l1syX=ERoO8CZzt33 zV?LR%Q=(RYkTUk`s4DC4E{N|y6w6g9$RCfpXLA(m#?;DBW2v&&j>Su{O^3sY)(Zvk zg5AQ8IJ6D5*0NbQrmRQTs}M$)hgVWq@5;70bSKbg=cpCwg4_}uk4r_Qz}ASNK*s7N zO%0LVGvtm{j~By5vh>&qvq*T+d3ab`(c~PBuK;($PR8=_F_z?_e&aj#;@6AWtaddD zljOtSx(B})$p9xKeRAV6Rlqd+3yHaa-Oo4MsLBi`UgtgCMc5)HQ32<KB;W6?Nk809 zF>Q+>3jR<x*(om=wYZR*VydG^dbBv*$XL;mA&=J1maIAyL41<vzcEinq0`lEta&k4 z&RyQ%^-_w}uYHqTkNSqv%Ok!Al>*mfd+1$g{;uCzz>5(&1iO&NK$<I)k@#3nz4wn1 zZ@O-A6d4<@`LYmju8jJL`=AyChcR-WQ6(Lz-|k+&Nx)8zZCx(kc^|4_GTofn*AQQM zG#}47zx4|v{5?SFuBJ%mYZ6=gdcoaBzbBMM!A}|;v9;eXlz5xe90cvu^i6B}aMzlK zpgK=;@Eyltbb}V}veuFz&>>x(`xfSt0nh!4j4^WQM-w{sdZ~r4bcIRntgoD9PY}(o z6wk^{EAyMiCG?%Ejb1x}6I`SrGF13hl<kwg6v^}135(ge)b~1@Ie`$w%wBPE{!#{u z+*ihze2$?{7+A>~YkXxVt*>yj$=zcPDi$uZ2u-^$`VZNuDDNp@$^Cm3462%+tUpeu zsyiO-eyH=mP*Ts=BQZhNvPXC6eE8D@Qkkxi3MFn8g36POrvDfU$*dBI)C-I&N*8J_ z(X7+&q<F12auJ7o$S<2`DA>#r%wN%_>U>=+{47A|5bK_;9l|a^;^$|k>aHxfEo__i zUbWtmwaqRBF-Mo-mBBOlYwXU=SZ`0D>6BtW8FM%_E%KABvDEID*!0NkL$JjqIKo;T zH#BcqmC@sVy%vnbulC_MPpO>I*v0e>Z-psp%8XTD3vGsD+CqD+slC}kL%7txFv&$n z)#qop1$#{4Laz^dEUp;61>zOsgC1|J+XT#z;k5$SlrC1Sq5CZH9p2nx_=(CQexxnP zY^~WpfkFz)CrN_1YqKIhK)Z^O4=vWH_Wl`#yI!P0!d$0d@FB2^3JBoKu*6i{DxZIO zdQn+am3(c!9y%Bh{P<oS`sQ6ON1FUNbXKfk(&O7D(6)#$|8)qmf?r|zv6YX?>C)PU z@eVxuT}8n{X-uMM$ywN%CH5q+pIqDjTU&r#|1A{@4TOKhzvN+M6d`WT?;|bq--;f8 z)GndFELuSXK_Q6S!PB^JD(1cKAcjBM{$0oK*Z=!cq0~a?5bcLUms|FwtjdjphX$10 zk+OfkahG2|zbqeDhtR^TB>%1`$R7VbT%f(6tTKVZYc1xK|HywlG6W+Y)3c0%-W+f@ zo`0{J4TyViN^ZkAsNlPbE4LPEf%RYh`&U{}^b>M6y#zz{{c$vj+$YR9=z5D*|8?a* zcOmwBpR1GqUH`XTs3L@*x8^$k_1!AodgxiusWc-n^j6tFidKJ=Fs_Z2v2f{rx&f+8 zJQa7t{6L{;Y;Of%Jtfz{?hV-jUUI<iJ3cN2+>q;1-Xc(6)7XoL0KkRLQAuU|{AL5# zi+=2wC_M=jWlpL@Jv&I-;w-3F;Slhgu0krn;9aR`otU65wIUQWNMLx=#<td!xL>l8 z-m{Kzs%DaVdlpz-_7fM*PDjW^|IZm9YPv_|Z(&gFFq4K(pl|}=;Cc49;A9%fzw^A$ ztRQ1%S&7MgR(~^AVJ0SSq~2p26f-_+LURTFeP~Fdf&Mt>G^`NJEg6m^0G6v>s<}vE z7i<mPlIuxSqa4F|A8Pn2+N=BdlK9VCDpxAI%iYWcMzbX`1_Z9bQMSsl+@J!6{edIs zSMq>m(LU^?C!u;s3U=cIJ75CC<R2rBd#-xf|G`|<Rak;$K{U`Q2ftT%08-~PRJLkj zj4yT^Y&9pQcqDWh*28be6nlFHNG|&iICj#(`kjDc@_GN>EvQfeD6E2-OtJ4ISilzF zu>LskwEa#S9(`Vejvd!yVT%_xJV)zF!Oo{?2iF1(gBYQ2@j|mO4aG2V8>H@pGL~^5 zDJ6jETbAFXI=RXWv`pZnMo&c2FYs;xhHbhhF|7Q{J=x!n$q4NY39p3KfDlF7{o(3U zynImC#)Gj>Gn!L40$#@;;u@e+3oOz^IU%gP2h#5X+FkLJ8$gD}PD3?GR}!~BH<T{> zdJyW<C6rk~8MVm^7Uq2g?=ToB2?1MKlYkC9852b?CRUk8^ZJk+n1+^1_~tpkU+2a; zN2;aa3E#M6%H-pCea+NFW&(Cz`|ZEh09_(r!o^v6FvaFtjSUU`e&Vye%zQUZ6>-02 zTE={gv|uVmj<bSx;1;8d2=xQ>fwgWu(4bDaz^(G8tXC}GvrWm~-u|RW2DWoRBuNL( z<o#?8aA>>0tJMSM5wsTS@Qs)TFkhhw15V^3hE{v>{A8M-Q~i9QOv24?rWqu_|6vIF z6^3%u)Sy}*0SwvkXU$G;{!Wv@X~dv6_~-bKwfX<Vq@CD`2wKti#$Ia`0)Vw80CKZc zzu#*RoHPw9iZ};@1E9EaXCcwv0%PKLyj%XX^F4S=F@GJW(Uns&G$@wQ=96mQzYNj8 zip2jXyJD3Il63729>Z#0FdjM+xD=-@P&om|qST3LrQiR2Zg73FZs;AOV$x$^vi`&J zflR(0+(GQ0I}iqU@Mi@2|HDfEf3arHi~Z?4v!Of|7@&EzD&TXt^Y71kVvPZ!C3aon zId2_3%Q$z<{Qvb}jN3Q;x2^wuk0%c1B$WUSLtcN=i^Le@Pt3-V)N$W*eez{G(Kuh{ zdEupB8ze0L8JNbE25V$pc9x!G*&ud`U;N9{VPN}>J(sdpeL*gq&;>BE<ekK6P2`!^ zBsq+V5g(`VILvwf1&`6Xsp!e6<8}8OOg&oCn!W5eu=BnbSz1Lo%L}`l2BLm_@&RLs z_2hk5`}E_c$y8t0$2ycwr>KKd=DY@jht#gPJQ;+d%1UQJ(0(JJ1=Vd;v6u}tF1sL@ z!*_rVro!E`w}Q`)o5UHGvz$UIdyy&Ffd~rFMN*5Tppw6S>C^ie4dqm)koiVyZ86#o z9@{?+mhiYLy!fdw1x*hmxa1x_cnEzf?io~d$u50$j>LlH>M5(#Y9+4At8I7<S8;*i zswUoeu)$Dy>fTc<Xer=n*laH<cb~?sZm#P6A6&h4T+?6lKaPrmQX(MIA|gtPbc2+F zl+vR+1f)hY3|c@Xr9-+T2OFVM0s;=C*^n4LkQ}|=-TV9b{2t%mkAK+%cJJ$T&pqcn z<A{uWb}V~LUXTYC6pC{ZYUKaBv#uQU4ie^hw4Y~LJlnd96!q(VdNqyJre}DN`&(!9 z+u)kSfFD<R?uNO4k-Pe^rk)3~ye|PQWC00h>)sBLtKODAt}P!BdfeGl^@QW(kYP2c zFC<qOry3SLx^^(XU3wXFJ}^BQ?HslplWXSVd&Q@HWlNk{oa9*r>;w+$*ATnp!-9Jn zHkd0n8B8O$=Z|nX@?D+Gac84<zp97A4k9Ir+7JIQ4lnV*zZgb=p;qTSlej!E=;?PH z*ez*gzUf$jt4%8VHv#ap852qv+@kx9-Y~o9Lw)w)68Z^vir!8-;=qgn<m4$z-%}>* z9PHuw<cJFMAlE4*{D_fs$^4Fz#E1py3Tvf{kht968>0|v!uhL<PAOnVPWBY|+65Hx zG37k@ciBj?IH7{zcFBc+P4>Bo>s>MlTBsCDs769nk2w@h6h;V#%de+lgl+stL-Mi% z8x^tw;-?Jc8xAgw<Ce3@xf#wv7kyF;8yGX#UroY{`xu?t-m2fN&dYdUPWd~4ngeQm z@wXg!MWhoB77ts`yQ?aAH8*$qQ;K{gRKu{w*px(_>>T$hSv)==m?sFgz=5>UU*ISW zH0ScoH!f-oLfTq53xVAq9EmxYum58g2`kp45S{giZ@NQ@_*~9IB0U6x#SKOH?`+Z) z*{Q?@yxM~>ucgneRI@BfsU@3_G9%eTW##sMFPMz0cWsBx+Zx#=Tg8YgG;_;scc%0O zZ&U>Um5h8^9G4&3*Qgjwo?!Dr2(`D!p#!JcMte92hYY?QlERw<8E}1$+4Fth$y#jo z*=nQDxH(JtE?ZtUdBaT5R?*&B<hiC;6kSK>BXihU8L3hbZSHP;e(S_uSbzK3;8UmE zBcG6NyBd%^?#*})6@uK(2Z*=}19q1ROTQ@zg$;@%Q=e;%;)I^vn@S|f>giqFmMOfx zK|Ump6fh@*i{8^4UG&Z&cxinFZk34;i#Dz^*k;JXkgcuFDs74E1jS!}@%VAs_^*9m zM`q#G4pWTg^(C7p76FCro3cxH_7dCKzo(Gb&IF8iD=|3ChxlPyh2^#vQ}$Z7Ld&_) z;wW?C>!-sv6|1!c_!v$bMASy3mKgD5z<`nZY+~5Wy8nZv&Vg+`{{9;_!g{2FupP;l zkL`zhj4lxw=1g0vi~`z@*^@>gf{!^RF#fJ<H-+2bn8whbn~8V6c#or503%Y2NYrEL zif+hkE4z>@M@-sR^etP?MZ^-g{%kYm96!G46Ai36Kif;93Ux^3yimy~*vU3bd-eoK z2v8isJnKt5jGSq*<aFlbs;a3%DTt%y&8hOb<=tufe!-`}Axm1AW7_hpc*D>v`(IX& zMBD><QWBq>BESDZ7{C3y^S&0u5dJUE>6KC&Bru;00-9HTsIDHb8&Rzd-){-6FV8mf z;nAlK>lDTK+xp!ky-J%2KG8MVv~aVUL3K3Mzm=}DWgL<nlI#n0w9W!u|7mH5Y><T# zXKYNB6c-l<!a%0_zn8l7BxCzmUiA5X#`+aw;Dq!#iSTbuhw|F@cjVBd9)lskO)ta1 z4kq17A(;UV9_ZzQ<7avj0${hGM8t-L2v4pOp9ex+5>Tf9H}CgBkiYB{_`yHhjQ_s3 z(|`6BlW649-1fUrx5hOe03Jn-b>I5`epRn2FCZWAL6*phxQ<nT5ckciRsL$qRgzoT zAc?3)feg16h+^c4#ez?)?8?rD4&1>yHIhBBWoe(hIWxHL)a7#G`0SwpaA*p`JEEDj z7`8Ix>X=`Brp#j_vGNCb_<xt&%56sTCQ0;pL&gX(VvzZo>r=OateD~d$W_2!&s~XG z{>U+taejD-qg38#BYrug&e32?ol$igQ#y}pJt@cQ2!@A|g!K|z>cpR1Wz|>QWlb#> z#Vpw(gDd5yGP)>k#Qm76!2IKWJN$6}+305Uj|auzt@@^)3y2@YpRlX1!08+@;Am}h z_vdD;X0Q~BkQ$WTJrmn%#bd}uoU(#<TGD$?4mLFl3^vtDF*}{!L$XK#G35X$h4UJ| z8zc{p{<j9KH+cuyYzeXrJi_8u6@m}xoN;k*%4-p_CnJ@s-Z||xOU*WETxAXPe2ua- z0h;2NKpQim_8EnhBQHs8^nk+C0tR!^fpp&r<d=cuqx*uI&rXEo_roiG{BoKxF-h5x zj>ZP9%=MHf*q#d^Ug-fwtAaVb5}7fqzlJT|n=$KSiuuv^|H{{Iiw2a=iwO%il|JGJ zvOi-m$o3+Z?CaL?UJak7+uH=sj&|0|w&H&O!t*UOpVn#)1A^HwruAgFUx;vr;<L)F zr68D}-@rdw?FZ1;^jjyE3a8O7PK!JJsYM{_X>~}g|H8bSxQ^yNi+sr%Tc_K<UO?Hn zXfqKktLMCgHEMKxotqCZO=-v#6004$SFfr6E!q-ZZG4vj&+1+?xW0=P;$;N2a^&SM zkl?IAwyRhiut<h4g0wfv%kAm9{pNndi?z~>x~?IWdE?H^HcUS|K|?6u!`(NWv!x^E zdKO*SUm_<i*$E$hR>IMGA*>w=d;h*Q-MK6BDXf2g9ClCcq_pyFkx?dreBo3<VLx2- zd7a~cMiAe6zGU{4bCm{|LUrc?$%~F8s-~8&`^kF)5OM0GY|tqElDL$UCTM|>%&I%w z6NP(mkaQP-`V$#WgzSjLDo97|iC>t9c(WnSTqTNyLahx9eeHMY6ncNKzQstm8X^u5 z<s-g1ZPFAOf4sp3V{*vd{et?}q9DS_6LC(F!vrpP+8dKd4;XqgF5ji^iAr*^uLSzT zW-I=?5(C8MR2ZkWJNpv{uhs<bTIHibEwP~<>1f5cR}>`A1YRKC^h1!La(nB^)A6|W zXG1TUo?Nl4J1%g>0;6M+ZeE4vlv%$wR3xUv3O-ExCNR8N`P3$^yKUz&CWboSXb>4% z8Ti3hz%+yqn*9*|90V>rYI)gyfl1KphEfJOiaxUS-w#z2Z1TsLZ%=*l^jUjAxGepA za9~7m?p(+ab9F!*6|oG%5C>~}!~N5B`#1XOteHuy)=;Xb`!n!4vekE=M@9_z<Uf&E zwNP`&176QTE$C_FCW>(ZA;71act_a|ac|aUTh6SbNh*@GDN?g|gv01gz1O~adBW_> z{(8la{=@`=a^Ox4tU?|(8=*d2%cqc}ohAkB{fR${o4@mvLC}m_pz|Hbg4sy>Z8JF! z43vZaDp)NKvm$)o`}G16$#(v&?QIA2wP@xo`EC*BfwWDh<vf!S*4277q;}tkQ^fYK z9{<!l$!rIHow|0BZ@+<P+Pqc4Hsb4DFfE06kNp-TqlqURDv;;`{TSERjTo9jbqP7W zkeU0G*OgwYa|GXU3*F#w!O$svzSRT1V_L-Sk1?a+5EhQ7GL3{<<~NeoWAA;ohr5VR z$K+oT$|t9qk_A5V12a;)nG})ku3sF@D|Tr&AkdzfqcVQ$qn1M1pmM9H+{vG@y`QMn zo6Ea(u7+ckujtnQeHOShmm3_SQkZ$d*UvJ<Sln=|$v4s*J6OUbzQki$&0{duVQ&<A zAHvfSP`eW#dvvG}?}Gl5F&?;X|7BM=+Zxg!TeG><WN+&$E3Cl6ahyR~KSn11@AV#V zc(B6{NiAoBTIRp&`?sJ+`x&SxO_jgdlR6ACpI>Wdo3_-1SGCr;A9B-Vw=|rLQ*}^c zzbfgG-l^49#x`yqtk+$C>e-Yb4+}!Csy%Ty6JP<`0Q=y6GB$;nOM^qt`fI}aeyvjV zq$aF?JHKS|!KUN*r0%O)>(R_4q2H&+j}Z8~9(q6^#D_mv%1XPrmu2{QY@EzMB-$Ch z*E~JqhOf|~n1z{|!q&T$47_d5H>dx-eHl7F)51F{@<&>rDno{E*OYpZxZs?vC(8vG z>4C^N#XnCv{oNr!M&es>Da)un#~bTzC0Hpx)T-~Y^X944Rj1a~kzz+4fVUs~S42*! zG6Geh^hH4~UFv(8y4PAL-(7Zj%dAJmb&<PEgP+zR>iSbsl`Ax`ppI2~zEJ)ZIk$<q z`H-$<jP!Av8fWUvF2vI9R!T1#KKGIT*Y4u=y!XaSC6pri*tBJM>!<9a-Z4((nh9Yv z&Kb<9_r1i{oR*NLT{EQ_wVjg6J?5w{9lf8Tz(JTr98|qTzjw4?8R*fy@MY!+f~-s; zJFq|TvN`X%z#HATo`&L0SpoiFVq*kce`p}x(gS^5w0TJI5>snw1}kWq{m02BspS0o z=smeu@NP<+*pRANLOwYKz4Cp%UM>W8Ur5kd$f#iao!hk=eKe!`6H$VJG1hN)B=6K; z=X~!Aht=H6eNO$rtYLn~XP_*>;d>;xoQhaNFv5ea(&3T*hlOwFZzl!T?>G(gx#&8C z*f#}BZwJ1UvSBtd4+&`G-!>Z?(9P@9@CgzHw)GsA`gA7E#|4x1`9Vb<vz{<t^X?#w zhV$=w8!6}>2Suxj(NnC|(}1ihvsPJ%M6)`_p`Li<2ZM>pgT7qPq~(Sti*9KS?2tHu z>iFRJB$BE*sfTX2rsaAPub|F*?-FcphxAck>)~+M<U(Qu`Tt-a<=_rK-59Gp*q8+D zvoYFNQfuGNmBlW+k(?u(HX7Br`}bW4w|RB@2Yc7J0@A@=tI7Uvv+FR3Kepn7<Z1x4 zi%k{mvxp5-sbSW5-)`p2{f2>fc+x%9qVgos_P`l0Lt8K+p+^SZZiq{FXu7Q~o(&>_ zd{Bqj=$f!_wyUAQ&||@7e_W#n+W2$zi}ImL6F6^W*?R?=V*0lnaVocq9kQ!We=EY} zK-f^OOqkxZ$dEt!IAp}pv#d`)mu-W8ihZRij?)Rfi#zO6Ed2a$k%r)@zg~z5ZZWx5 z+PN?%KfU74TLfX%F$s{Zk?q$s2(t;Wk?X|Y`9m?ixHD98%JhE{FIzY$E5IR-59`;X zjk+IkXRY_rHPxY)lZLIZ&H0hKZ;$0P1x=E~738lM`CG5|kB0Rpa8%>gc;4LdR@Z&X zMdPrTQr*<Dd16>)YC+Xx*FnXyc=$V4Fs<)(R;-YT?EPD`9i7LM%^tTWZYX8XB_Um| zVb>&6&9}cRt#joVvj3a3T$5f2|FIK6PWVHYW6;LVbtsaQtj@PUDaL{F9nsyTU2nxW zLfAbf>mZF|vKAJOt>K5-*bAv%E5-beAN|)BeHgGYwePSt+a`jWh2?q?D%ZSKcbA9c zjtNgE-IjjQJ-6KYK}zy|me>zmuWkntc<|7f@>VpKBdROd>2*t})L2t}2VM-FUy6)J z&-!hIO3{067sd`hX=y$&ZuqdWS;9(LI*6YasU=ahXs~U%-Q)i&Q;I9jNWYvZnxDS5 z{M=Or&QZ=zx4snK_o)4=+rJac@o3HS*B!pJqBpya$otPX-a7N=&+7<Y6dOvqDAWvx zvO%)J5-Rjo$iQJ;2|hVsdjbjFc?SG_J~z4t$>T6*4hJ*F0v0~*YdG_?n*i5DD!7ha z1-Umc=SLhZGm(UuUKCVUeRZ3rT4n#E4WkV0731Ax1K&s)UGf`t<|J-x@`^iSo3K(0 ztnSYl>M9py^alU0*%<|9#JuKvk({ARCt&%*R1w!k=#`bXO{xfMU>I=UTw&r8mF4H+ zY#Xsm@_A9)0JZ1bM1lPZh;*R;dcHw$NRV*PU}s$s4_xJDEY=C#0fvhgN#6eg3^Kr2 z`|nE`N%9%NM3Eev9}IF1XUnZWl%#DhRbjO#E@z5v%@#v>4P}^~jw`)DUDYa)zx43A zUYQT@Byq;H5V@2z)0VjzZUANs@ji>-d@F!D)3w}s)3-G<Q*lw#M@e7D$C7^30@#38 z^Wtr1?Dg30xeLbw@T+pFftEyfmPnqXX7@P<EqkqRh4JL?f)65+ozDa=B|o}D??vcV zA5PCO8V&Jaf7k54nqMC{Al4DzuDAaY<&#J^YO#Z-8D|@S6!HouqGtlUpNBT_yy5c; z##fK1+#l@mr|*#1K0+T<Hbf6Lfc|tn@Os@?>;)HP;u(!95!*RnyXyG{_l80cuyb9{ zPL~yiE(+$2z}u%16(?UnvY*0d5TUw^DthMFVK80=DkREIn+eEK;8$7@IJ-<bt#z7? zd64okK-BfUFCbD57%HJFzd3=SPp($|BC|0R8~I-(YUt17;W&xx^X354lcWolLMVcC z%G-DlUzpsIQwjE{RcCL`upJQ2u4olxH;0D-^CQm8K!Ydw6klxMD&EUOPZQB^+D4w2 z1w%Q<H2DcX{Hq3P`=l-l>9_HC^Uao{ctahwO@=Do8+Cmiq<?Vs|G`j{^w{06BK@|s zNkC_PF6DRcv?cCoZuY+4%aRC4{-gew^6Ijg6i0jp_YXeL*lo2EfZ<9Qxhqu7z=%im z_8E2xGP1-W_kjcvcff%A2e_tX7|`<f3eL|<E>9VkZG&}iy~TKM=8u%)xq`2wnc}+N zJpY%E5nYRR5mrrbKh{IzRk5G*?TySn-9uh(ZCKw!k>xz0o69eZb!Arw5cPR+u=FrF zsBhr`tjH%~XV5FETp2(*cR?F7@XpH{oQV&;m#(fNXPZSTEqt|JbO_j^1oWP9A@21R z1MOI_JP9Hke6j0{bqtsGtp_uEVR^z}9_@ux&P?Rt`;C-X4FMf_Lzd@FE?w#_yuKCS z=u}+OluK|Ayi<Pvc|FJqM=3c*N*Jx8tYe2#Bat3pIX}dNtMvlzF`}n4iM7OJvI5+g zEbcL<o`JnOLFfgaif;}G+iB-oIqe}ox06zrKaz!tF7C8Q#Z*-}#RtmtTYrCZ8T7m( z3goW7S9zy(CwN#;xDwPT_drWBL{&y*Tm=XZ#D+?5kdjmu#^zfC)=CRS{;Sz;jxVu7 z=K8oh_byV72Rj5DAF$ya6S%&4jV!x!kaM$EZhBb}^6HPaVLuUlQeO#*_L(5_6pFC) z-YxT4pF+lK*>=zjFZG<~F@K2qtAQENCDqTTC<yH6I!oJps*V)m9rSA|dW1$n^Zr!2 zmsDi1RGpQt1@@1xekzRR%R%pLh)mX;rA|u6_(G@QV?_>pFo~F5+8noFqrMzHuJZAU zXA;53)mP$*v-*`o;-D!*GFvU8#Fo9i=VkTKh7VihKB-hJTm4n})O8L6&s^P&vk)de zzF(~u&YTJ7XU^O6-XqCI{F)C<e0L6v0E0NUxxU&dk{m7zXr-$Fkr5jbGN}dB1jX^m zwh;B>gNT!n1IvwHyb@|nZTK^ak+o62x~gX!L8raIXlmDl<UA2569#e=BpQAIKZ;^N z+yXSm-en}*&)PoNi2EMdg;Z-5W2?zzjtWEf=)X^V$n+^o&Gvf}^tCBjD@w~~mEH!< zg%2ce(??P<3YlkKc8=-j@xhe6h+g<y+~{y2Rjotp+A|f!J1MqhQMYy8$$}<Zb<ZK; zqqMnHm-IBqewEOwV#>i1(9`);v1C|ZjvMJGo6yK>NR`_TXxw=5KKx}x1oWOwK#AC& z!5zTTNzcuu5Nx`Q115uPQeK^S(tEn^WT3cE5k5Fa{vyoMUfuCN1m5}qcRjo*zj^wS zplf@ut8ajfqfp8NsIm2<wLiG(gS#Y&N-*rZCzvYgsibFjaePypH1MRi{6sJb@JbP3 z6u%9!r*@v15<9&^T{R=*8DHujxgGV2td_$os@rZ_rv<h7LR=beZ7s{gUW^pc^~fX) z;1f-PeD}+eqXF>#=pyZt0EhlYm#=?6l4U>PXq#6S8`Z5cIYaSN(pe`4DutIUd5(^z z7GwG3pZhTjpAxIQO;Dhq=NJ#N)2mFki{C!x?Vw^burp-Of|uTV%t`Cj3|pYN;w!)~ zFJf|d{-SPwr0?NR5aNpyLD9e-@-KSL@4nw?On+UUG;KO_Rk|i=`eqZpo->t4errWs zv#D@@gI~lDAf9bp_4@j^j%FC$T@H^=19+%NnTnjc;wOZs!L?T)<J4f&S^<IuHtbYN z0C_Dhu~XXyHC8Dk$@KlpXf!bbT9utl@Tf7~v_S05?-dHt+|s${Icz)fHo?$6!44lg zA2UlNB4jD1UBkIZM}qkB8#pqcZ>(O7g+_n-E-~(LYxE%;=4Or!T9c7R%{RGOa3c5x zzs1_ZfgA3!om(lip<rwlb%><zxmIw|N6$=j;u;_rjmcK|okB(DjU90deTSrG_re!2 zrBAf>`r6ylix3iiXVSQt^Zp8o^sl-2dj6NfV{=0$KHX7>o4nMF=J&DYOdo5lp8Cw~ zj%~gMBWNUhkzwg20ZoUWx@@yUdH9D^Bo}70vZY^V+23kyfsmUDK|jbJL>q7gdXP#J z$!}OJlP%AE1pX>**-_ts>#$1HrGK+#1f2CO<2v_J4eLP^+i#o8sgdpN!oGF$JMWCM zjRlKRRuGW~BiZjz8D+R$n`!quQyjF9qi<m6=C(YX)|3un35!GO`{&Uw)yH1E)a(k@ zD*WfqUN+Ruq8v>UaHKK0Bi}DV*uzI}jy}LWCBCR-kGt^|GXi?SJ@d$vSg7VZE3a+8 z8|I8C(-)<!#*oi8cKJ8dip?hIzlXOx@3E8Y7H94{*ngEDGP(Bc)w{xTFI>%<K&oJ? zB)Y|2>!xx1(R*h`h3eooQ<PwW_<V8oiH(>n8)uHkOq=`fleWNqw&eEL1CwWgZUbgV zTX!*xTg7MTXbb)Cg_oDuab88}js!nMq6wHW@TPo1)SF$}I-rPGrvIDW4H>HStwSi( z>YUHAbff8)EN8>xg7O<v51>v&E*no}%L}Gm78!-vR{UT}Wuo@0AvD~6ojb)31i5z* zkOS~sxEHDIw4nVCa111`-sl@OyR*V>6x(w(W=+!6XH@7#Do7b_nrF2gua!Fk3dLIs zzDNj3I(PiEYSZ2Lxdz@vV2gf&Y+Mr=*Pr%166LzJkKUU32)I;}eXb(46!##U@misH zU_{i<Hqtz=U}1<4;d0YdyUvrsog&;V$58%R2Y7FjaM?LBt*Ck4KGJl(qS36bP@uOa zJT1kmzZv!A{CI!XeUhOj>lF#oOvAe?rhItq2Q|cJeD_<CdYvzwjrp^sSDJ(IZ1t%J zcfo9V)}Ph+FUR)|wq|pt@<5Dj7?}Eb8jv+QI`E{rCF>=-|36f^{g49kH0IwSrt@62 zzI`DCwJ8_@$(;mozM<sWjoBKnR26FghW}?^^$=wPVVz#g+iQcZj88k4T?y~M<3l@_ zJX3}CC?p2+z~Dzgn@&wl28{E`3b7_sfiWxVkTkTZTQ0>>9+GwnPEcenb&7O})s`o& z)C#QB5DWv#CXLQQlZ&aO*)Ggtj9Py@f@W5=cuhuWCU<{d0~fL3r8~+@1KK^g)(P3h zOheI`=jUu<WOKGh?LHljt{A#??#BXpMLVSeItitA>rP`&O+-@aBsp-B+hW-P?n^t5 ziBMEG-;X}eS29z4o*$t6Zt)+Za6Ue`rq%E2R}2=Mk|m92-+wy7xg^9DhfH5PS{9x0 zO4tigjxLY#7U9c*WF|d`7ur-0l5r<W+L;b<fPT2`<N&G7T902EY0FQ(Gci0nJNtDy z<A3{?N<2{urVDPGaegP#=}_b1%VuiQTMN#v;?K=yJr(48oD>-Nmh26h)IB|I_?Em| z(xr5e5LQu-15Zud&n3W+xfPLRZ}sSSF{ZqhP!O9x6G4S94r()iIZ9I7DeV8Ln;cE+ z`ugO{8X2_3<>c)EU%nmw#P84h;t9ufz~!XY!1joS9(M4IH0CH+dOzoej@h=ULwxNB zuczp)E8{XdcY?jTe)hlYrgq?8ULe?e`&y7yi8eX>JbI7sepsUoC)lVDcy%AY|EgCo zw_x{zl5q#$6&N>EEm<`;X5=Y5$0srSa7E;F>6mKrtm(1s09APcHQwoX&Q`~vdRc9l z=T)mOgA6pT>Nf8XjgJUOV@-Cd=xtdLE2rKg05~z5ToQT`(z<|5uWhe2pM}=98o6iq zAOdl@P6_4+pQomt`#Cg|*-_RU{(n)laG70=C~V>*lzdFNn|IFXVoe!$M?<q$N5jVg z^Avdd&i3+bB$%#+`$@bg2VTWZS#6(1m$#}<W2P*Pb%e9?sQ;vP2-Hw|*5<a;zj`1T zbYDoh088G{khQzIwC=vYmOS||0OH5LE;x4f%FWVL=Oo5(0$7X@snBB#sJV{F^)^R< zAFUEA5TbqMZ`_X6t|H@DPdq-(?j)Ut#jU`G1X|)|Uk`cAdcqf9JaIG4gz22c($XkI z){N!BBJTT4I*WXU??>aT?(<7P1OOq6Q&>{C*n9*%XMtC<=;OFJ{5+Y+5Xff!a{cj` zGTs3DR2@rvS9&bnc*W7nj@7tE^o@RY#>?*LgU18AmaW+MwX+tqsTnK}<HsdG-4d5s zO=zi<PG3llliwopjvHSx9b)LG-z3PjH5WY(@WjOQh%48`S&XfCSl{g_uuTiLv|1gG zt;3Haovdz?hnkogm=u-jO0^^d%18t_ooT=FYuIFxj<gjwN|MWAf2O1PV%U;<pj3Ng z5dP%Pe&PCCXI*p`YdTV5@#=qzP*;;D9`r(EcozmcZgL6lzMEo;co`7+z*p)<+>*p> zK<x7FZ9Vt9Mk+lB)<b?w8FB<OOMn?DBpAbLswCV3?S7v`KrFkq|Io=)AFZPX{t)XO zsn^ew@K<L_%v=h584@DX$<io6v0?ZV`MOt!dNB`GQbOz2{F*nEXOvBR>Q;cCI$%|c zP_X*E$mM3-Z8;zG7zd74m&gt|dU<!ddSCBcICIKaG++6X2rtM-8@D`%u8%RUHF};< zyxzP{HkQzO0E;VWeIf7s$SFItrr<dH7VMX;*(Bt2!y#qLTDjr-&X&$QUBiSLH`(8f z90B{^yo}`~W{w}^1oP5lpm}o)3KHeJeBrJ}_UHlDP_$UOgY2NTjlTOf`ZspD^LLv~ z3z6nSWQjSvPEn4eE`-m9%ABpYpSz;nqhg5m-tN&wTHY>kn~ZMJ?Je7l5z|X*Jw>(R z#)Tnz)mlmI?A5BbIe!#%-G;VWA8aW+w3Bga5`P|b#NXxt&bGTaeq8FD-tGIu7$d{< zr5aXH7t2H914m-5xy<ycD<>&yDu`@gkp$GH0Ea83tuwsSi{UM2b0DsE=Q#Fn@83C# zlF-zA9K(g3z}+Nx-8!3ufKK~ifCtB1&C+}ejxPir<F>aWD|QU|xh#=Vh;lSPJ@4{c z87^cT*Z*5OsAK=xmV^6%ZS69GNeW^OZ0>OUk^XHcSD)0TwR|`n?nb6)QykC}|N3oC zjj}9})<gH8>gWB<U!8+Lle@w+vA075dhRjrVCgsu1d>g3HoheJ1+HoFK}qK|upil< zWfRmd@dWI6E*jW@K@4(B<ypJe`*m%4c%Yds)U_NE{!+G>d=<xlVx9=gmcvX=&0pfu zX}a|5u9%j@wN24@ea!4u6{kj6caXSSpwI8u2|I`wY%92rQ_fcs&O_`m=b=A6AJ;wm z11g?Ry(MUzV|;IUC)PkECY_CWfOoIb;<Nep$<pbyovV>fw!bnYe#Z}#`>M#?r8Xp> zTmAk}LfRgQ{BiZ`KAFEAP#k`1em5cU5w_8LJ1q29P;)D@QEqo2ZzxP{7*N&KuptQy zW9Aq)!}2k`%&$jF#~$OdsYY=RV20ZPG=8;Kch{>hqU)7N<Msem%_;ShnCOSV62|BJ zXjvw{=gUWq<p=IG1p@Tb4NmIVA$1u}?Fq(&2de&g!LntTno9Djiq`G12u5YCFTii< zH$+nV)~-d>GrH;?1XESzBtJ?`lO;he9{S|y(1NEClK<&h&5pNR&3TV^MXkx&nZUcl zp!=aYyLc)_r<g79JM98|rxuIX28a&K6F?AavlGiFLS#QRvgHzpymNE58}kHi&6K-o zPrAIa^32@jG~Co0J_b9AzqxvJ0qgHk@ceT?^wyI@0yeHk!+42D(`$E`ZJIO6s&lzf z-ZDhTEId<(5Bfd^{h*71f=e!nefLG%Jm>Ue(?JAnvKgC(eD&^fRuZVc%9rlyZv0eV z{WafYYhfd0Tl|Qlu}Oq(_m<3L7nZemP&-(=T)|tbv8}dDI>DogNeBIP;C`1^*P-os zL8c?WWu`u_w&P#kzt5%4ek_aY+U&cI@;#jQj?U@`jtr7Dy56CC&*h|CuGj9*=7C5< zzBw&?0@KqQ4m7d%%9q4vE-dCPe*fU)tup<vaW@O25X|~hC;}4WmZw|cegj{SW%@EP zt^v(t&TbxnjV$MkUfbhyO9u}4CC5#AjPD0-xK$Z9rY)D$riSqFx}jR!<X#zU*BtS| zzhBYro-sYcP>vcnn^YhL6ao+knh%km>vErERcXfN{|brnZ7KYBBzCQE(MWySCcr4f z!M;Hg<wY)2E^lJ8nKzYqbhSBe99`oq?u|yVP8ro}0;g7KY|;zf>7M~602WT1zj`*& z-F4ZuYeC18-*E(0)dPk=*4jEJ;~eYNcif!Ya6%qDb+-E9Wn4$%gbdJof;zMJuok%H zn#ufl=vg6Rnc_szEm|<8E{}84q3;mDHafK2ypX2mHsJ7pgYNwS4COy1;M&$Cy*}1D zBN)5&SpL+QZ)9<#|G=DgHcPEXOPy{f%2V}kaXZXW3nG`J=F9er1uYBiXRK`njeZ;7 z(d-XJBtD+=(rAs@WH9R)GR8h(d0xg~2FA21Y`F%KBd1(Fh4N?eH62reN4*VSLC27k z&J8m&HJpv*GqLRvguc(W(J*NRZ(A;yu>g^JDDo%y;&!mv_HRgWnMUhoQjAP?NA%m7 zlY_15%@2z^ou^Nze~z;_l0J$Tn4a0l4XLS~keu_ecWyFP2ua>I@sP3&_WW^gwkC!x zr}|5tDqi_W>HPK4ttNy_+4kZ(AM}gOvX`ukHa2VVdjG$P1%;|mk3Q^CN8^tRuM0I( zDS3G^a;&cDxv6~%H%u!bwgeIlWTuDyq{16wLVh|LO>Zjbz9>RkB!3D%eC-~0v;NDM zV<*+14ci>2nakMJv5G2_F|uoJ2q#F$YQ)f3i^A0XE~twXtK1mW(Q@|NNx$0+`IUxU z65o1;ez&^$<y-ycryKolO-+P2AepGxHJDroG*1@4(NR@evE}8P*0Mf)N~)9TbiX?B zblW!JtZ8%C9$R}LjGkYX{7UU!eif}A^ho;|Wkb%x83bSUKbBqu%RuPWh2juZK2;V+ zfbf`<aQ|pQ+{uTiOx)ou`9sGu5zDjKpS)pnVv4z`KE1YGz!0^N_MCP4L1Wi^(+f%? zYSmPDV&1LJ!QvM)?%k48A^8d^d{Y!yal0zxydn#T$dX@4qk~8?Uo?GOG5QnSl?>iL z=-hYy{qI7ucO4X3GxhA9f!gup@`=LRoE({@EuABeyi=2&y?IZc1pOI_TY_nH=JdoW zb6~b-K%)AoPRL^7Xb;MO-_|eqVr~^+2lELZb;Y|T8Aq17j8%N@Xe+j8;x@tr@NsN# zRAM+0lrdRv)1A*;d`4o}KD{o#zJ^a`mJ!QA97Lqk-HTh~{G>FLT6JTh9ncBpB25Pi zQ(4RTX~}30a2{sEkCMhUt4=3n8?ck2>@_X>(7XLIKKsY11N);C8VqBT=Nn%#1!L1> z>v=xEug{S&su$Yfb0K@v#8=m-vkg$lt>{Ghtn2Q*sw}yv%8Bfs9k-e#n{|hPD$YQE zwfe<eQ8I22>hwpPg4SmcJGWcwUj_HDMq5YFEd(E)u(%@jAH+a?-?ed(iexvMlg7pD zLbf{f*bzfm1Nz8E!@i6#9WE2U4y}Zx;kJCHd^tL_pjsMtS$!tp`kN#J+ChpvLCE;4 zVpDk44Y$Gww^DS&^t=s2^u(B$bd^sRL#n|RA)&GX_fD4Id`R*7oH0usoz9|T&%V63 z7J5AY4E4UzfiEcm8DEcnek$82Rs$JPvq=X&mF}pc<v+3*g<rx$hCyY2B3$=OQiZx> z6-RXM%)B^1iPF{E_MRC33&yF~Qc(R?wD3Y&%(fCI+kBe&_C*)d%ch;F>%Ug&SjW~n zq2y+8ixc|GHuBM@Q{e$La+dXv-|G3Lf?3+1{l`5@`1@2Q*@TIjm`ADG1Y_Q4QE6Sw zleSDx21Wg#Qe<W_TXu~Zt@M5Wo$~0b@txG>ZO`_3r;_|u9O7^LTt+_zG|r>O6z#7n zr3YTk2f8m`U^%A(UuWAOMXDagdx{mHCtc#IZeWaf9-6gWYwnim*^C!2$u`o7ZVY5y ztd@xL7qlR3w)({*<XDm~#O8G9qUBYakLzn5zRVJL6htUF)o<FQ+nV@cw<szLLoPp* z$A&2vxS=<Hh<g&Q^DJ=9&@Qk_M&KV8q8r*BrYE9#b_#yHDhSi&S4TaWPREt37|L!+ zrbGLG{RlpDx%hUvPI@EFQs1F>;8)qypZ1?3@ijLSJlxrApE%`CHB+3V12=Ii@;GSv zZVxn_sj{uDO=F=>0one_+V#Xf^5cCTEDPISg1dJ<Y|4<MW#A+}E|Ib_(~#ZA*+)@+ zy|%<Fx5a?_d7b-5@fn3Bw)~H2CR2JjCFOVK*H&8IF9UVl;2Y&Gw(~=GY%O0{l3(%V zdtTtay!x=kNk~OPh00Rs9ZR3mdoLp?uF)!}BhpRCT!|Go6TumGFX~`N(I{u!Ze2wC zB?HY&PoPEn`LxO9Jk$g%_dsNI-L>V6-I<~5_lliit8=^$T5E5#=f%eZ2E|Gzb+IyS zBl*GSRqrJ}*}m9+^HuNc4&vhMR7A|}0r5XI#Wjs$J%O*MUd7<bY!p?H_NOuHjn#fD zwyLPhX}aFaK>rS^YKQa(+FyxU0Avn<hOX4iran_YAqz1SGFLP__(yTJiIya~`^;E} zH>pe8SYPMCCVyvresJio@oL`PT4z!;S)AH-Sa5(8*WqwXn5KSw@5#!5>eu_0gOZ95 zKlf$Ah?>RK^)_Q1{Agr6R2VYrX|J-kY{pM!T0PC<6cLR%chR{x@F6wZ!wCNP()8wm znI{X+B~u3eL5A{9Efr+l$N=l|)z9R->QUS@)C@XbRf0UUSgKsI?~7cn=XrO)@J%?t zD7o@^mGS8)WSnNLh`Wemm)5N`&>)cqX;_cypUVsIRyUGDu;Ws;S=_M7>A>N|igZJC z<k=empHE&N^39HstjAGnEGQ1=P^@ho^WJ1~R(K-i(S@D)!2rH$*#XT8*alsB*-%4% z!r!L+##`>a)rNwCj?EkA#>vRL=_PCv!RXMnDVExvZIf{d)h@qsy=MRaUD^VJ63Qt< z^|t7nTgnFcw-TU+TP(3&$-pWss3+<T-BHjz1UO4rmwVeG%{wtPG%{wS;~9_6WLIi| z%8jADBDTb`HQ7Pz{)vTOsCLQOvq<TuhMQ2|3UiB;aWt`n7Bmk;eeDo)MGFT`8Icqi zZMbkxB`F)A`WfK9v=(3tI{1JmA^@G9-ss#?3`w3&2l_na^!wQBC}QKG)zY2M8UEg4 z+aqr#o1gCrL48_PZUBha9zKSqzwW)KZT2V0W&>h8qL@%l8dF`}Z0!rWFxXOt`MZy| zN&DpdV3jx-3S1fnP9bwB-0Be=V%}c_p<bah7;7D6mz<)~<99VfeahMRS$?6?*wdiD zdMZLY7hp?9!uOeZ?#|!#^VoW0t?5Ej3_FRhuN*!I0`ZN{EsBlJJQP_C7r#q87mebF z@zl#e9<1-wy!t9XWlnq7z33MH)rIocUw_t=$KyF}nTzEf!Sq?gnRtUOsL`qITwDh7 zc?@%VI)+oFKH@mOFw_GmHyf@EWyt>E%4=IQxKX@YgQm<>2^OmI4$3p1W>D{u&o_`i zA57^&+z!)i56&@s+Z6_~lM`2bLSLTK6!Ep8-w?f&sG4*Y*cE`%1U|num6f~mdcG_6 zJXL7{n6s>08pXctjDxDgjxdueQOfr|%$B!l59o=g$bV{<JnYod+R`#`|3u1(8A;J^ zGRag1&pb*0!=QNN@_FUC%N@HA5{}k+yT8|L3{2CeKwa+;eF@X`7ULNPJ#Tsa(D@w= zbGUfzTevz>CZ@4>6YT*tfbD34hq&wC0l+4?S>~gsk%n-YelWw!*TOx)J@9KYnv;QM zY&mb=A$cI-EUF}Iw@@r+;IUy`lI9won)Y#--y_cTMwjj8QEU_)h45e?Pui7+pdSAl zWwHatW20^xH{aKcv|YT_JfVG4XRYU86I8YdP7@Ojj6xACtFM%qs-%5{n^<0F2C_TZ zjDK2ia}`&{*az3dvG<I`;QC6Z+#f`09>f)(ElwfiC{^cHEC1%oP;7F%n|A#I9ZA6r znJ#d@VRDyki;o({Ei_58c%h26j>~|!`k)e&Yw6=QJkCvjm5$0@H$Ae#rFyV<rA<uo zTA)p;uDyWC&ob|2lRu?L9x%m8{%v(0;hH?{8MJN<4YO^VZ^*Zq95RgJE9{rCuIbR< z6nBx_PvN21qtA|I`x{;M;dx7Kv4QymMK}tyL-VJS2j`Ec_rOsN;z8VufxgfWEcJ^8 z_WRqITSXu<nxeu7uQ$faDzfAHZIEJ9o^gGDdt6aVE_M~TK@5J}ViF<<h(^>+oligI zI}wuKE-x6!KeXJ0iZbS$IU#Ig_I>*+*XsxbFI?p$zLeE2+B0Q3e;dpbHSXg))S}YX zNKWM{k}yNUgLfS3V@7@<FS~Xl3ij)(N!0F@t=vo)`ZJxqoS*ev7HUBK<z~kP&c+1G zR^PL)J?DL#;c6o^T_0Z7E5%r+@RL868f*@UI>0z#zki%>iEF5oVy@ZTaSk~~2vJK{ zF*3#_Yjr_2o7s*gY%_XTy&ez*WOb!N&CUfK7&k%v9eh&|li?32(eGdLDTlf1xxX8y zlZip;xK8Hs-Q*=Z-Yr+K?b$A+y&{}qQTyc)O-9|x(V(F$V@PUulZ|6VwTBW`t|y{* zmQIGLrcU-x{HCrAS#i^8%>qk5E&P_>o(#*bo_x(nSyr;9P_v}l7RSVPobUFkYrFC1 zLHo%)+pKtdsofz-ZPw1<p5C&Z9hKFy=;!=81Ym`Iys?(O(_3p&=kO&U8L!G@Z1nSy z0Z@k6evJH7phai&Ret}ENb=h?nI2zVg(($4P;OMI@6G2`FOGap>>3uL_a5()gx2Ho zTcv)pr$a^>V-8Rqv$Mu*jZC<~mE)QsmfXIFIc=K}{D(Jk1^{4NJ~!``A&m(-$_cIA z!Gw#Ul^A*?ECLum1iZStv}&7UoE6d-08a+{3tf29@f*9DwF7m+R1Nk(Zho)G2WK<s z@^A0$Bh~Pf!ws85BCa!mjdD(zZjIAEf3NmT<h*+)B+8*uW=i?wnw3B8U*rAHuC*~Q zIb{s-M;Wy^6ZVZLw;5tX`T#Z)qZDH^`eEZu&8@~AGtaa+v%po=QQL6NZoQXpZ%$KV zd0B*KkD$^Xmn7=glC@wU;(vXD8T&Q;zN^uJ)#wZvwDrW67WbYAdPN57)Qd6umP#>9 z3t;=ij<!cnjlIF#En9?emE^r_zYCC4NR%(bdR>c$?$#}79Kcks;x-8O(y#tnK?(2Z z9KKu*Ddx^S0`U+n2>qE4$yXzLUGULTHkY-}`m@tN^(ecNMBW^;?PW2U${O$MMpTg# zlQCI~e7rL3o|o0y7L0>+s$_+@X#0uC6C0YgG7BHf)5;Hh){e?Qe_Ll?!fzyxmVG6% zB3EEMc&eh<*wNtjZ@nh8$D3i@pV&rly{O%BxoTc!KRLtWx{tL65!2w^GA*7^{dG_< zI(Mb)6J=7MP$bn3&XtnI?K`=h&w{vw#{z?nZ093|M%Py|1!jTmCb+@*=)CI};?4_{ zmWmtsE3#!v#)}54e-BV)ib}ixVxU+*8b-a{>RL@fHuTB;`&PguJt_P!XN%Hoz}!h) z!o{e^k6-&@(scgO)%y9~viabB{MN5GPma7cgv@=-JeVS+k3>w5o;K`=_S9`Uae#|x z4ooT+kAA3F^XO^4*4pKeg*0v9zc=s-MiUXKk^NI8$<T3Kw^FB$rJg#kF5Js^pLbn^ zv9`{6{;ZFRh~^b$&WxeW&YY>Qp<RC)9hwKyhWecl0nBl)ZS7nmUe0zRViw%GC}??t zkcFpg{GsC8Y_|qfV=pn%v`DRZ<Y9!z#3U-*dzXK$Ty1)GgY1CqS%UnbN}%0HLhXqv z&#&~4L>79bS%WN*-h46Lu_SAWXr2kIEboSC-;hbr<F*A5;V=VXyVkmTu+q;;w)0iF zd|hp=a*He<g*;$eQoNsO6C`q~lz2?$Ju{<9{XX4cl(9H6_zG#1m3jGjMAYdsS>KU5 zp`?a2j(2GBbq<4(kjJD!e@*7>*<cn`>aJ#T)-a}TStLD7vw%dW0xwi)A~z>fh2`^z zO!Q;iM+a@O>L1^{$3;X@%COUng%3Ctwy)!I0~t0R*RXx=pU%>gZ#Of9iQ>R@7U6{q zH2_IfAAbA+mF3G9@cfs-_1F{M+1NmH(})R)CXjSF;$}a_*hgP`SA&T)@QJzd;{`>e zEGM0j8f-V-V;q+)!aKcqA$6lw9l2BnXBDoLe0ozU-Jx+VGTq@;KcXN6#OMgd>Ahfx zkvm=Zyk~8`tdE`t3sVow5Vn+F4_pUmf>=vXp5MM!dcC|)<F)JtC$_Q?J;OT(-7OU{ zmb(aEW><)}?2fp8{dz&)z>{v4^=07hATu?v^v4Nqw;nRu*Y}9v|9eMG#;bnbgR{+J zeQQP+?~yl|BeLK(Bg0S{ZE-YiM2BmzJ#k$fYw#-qyyw6PyanLroBS@Mn`sB^__nx9 z(Fh`KwDPF0j8s|bKfK*w4-3IKXG?fkAc#_MdLnbT6%6qs@>y0-cZYQT8&39}sG&s* z2)$Lnc%{%<{H(qopR1MBA;b|C|MgkY)X?mU4Oh5{jr{ytMDTLG!3dK(`;KpoKT3+L zE%3M;KBM=7b{RnKl7XZBHA7B!Qi?xLG!Is*#b08BnMwJ7x}0$>Z{zhf(FUF;n@3}A zP}XUyGS-=`2x4c5(ygqKhpg{C4Xe-tzlITNy?NPL`DM3wZG)E()e|pJP*bz`JE<Y- zCazE@H>{_R7O}?~kC`K9nNh*<jxBLhhg?jdcXJ@yBC)==cwU>L3qf}oZHMY50{Ztf z>LJ;o$E#}#hcYHaL1+Ujj1}n><nRTT-*P54+-M|M;7sv;-`G^-ePN{(ZP9Zi_d0Lc zPOAXBEBn^=@A=usJg`A@(Iifl!3;5zfeba5n7Ls9dEA`dAHov)VbCNvP}&d0<ebko z0ekq}PH(I4gY(E~=c5S)!mdK`TQagx04(Bl2zxqoK5ttn&;9Mmwj-O%B%vy%Y=bNB z{&qeY>v9rZ7jY??I2!5@@Z|1HjaVrb--1h};xe^A^8I#hb)O(sQ16F~rwj+2{NjtK z#6RL13>#{CWH2b|+4zuf?X-IXd7#98Wb0NSx-c>3)L1tlpCZmoCxt_$Y2Z@Gv!8`8 zlv5Uojbk@w51(i?VP{l}i$`6no>H7>_p_^$EQMEA+a*7r58*Qs+KPIm>F=E?Dv+qT zA&TGI`vNS+9h^JEDcDY}z5;c7RQb;f_v$JkmL-@Rf_eecv~%$8z#oDCcq{ik8LahR zIFB==3Ae7iAp_D(%L^pW1TPAjZyr@v>u=?V5;}dR<s`;h<l#FrGk}vr3`TObtLm%i z1{_+IHw|p_r8R#Ieb{1gyqrqDk@?VDt-(}_uSaCq{>^=FW(ktpqe=gQH);k}6O9M~ z=tM8QN^XQjUjx!k0f3=TbMH4s*T}JvXRtoWIr*VADnAb~;FT$A0H`+~T)nJtfUL^y z4>!f{EO4Gl?$KOG6*Q{AJo1hTQ+p>-Wn#|a9=#^$H{(^YwM0Bc0?3`WR6U-6xmK1( zq-tvzRUSSWXgUuap>y6{An1+=3=`EBc<e$PUUl@dF6FXpJ1*I0WZ9!yUfyqe)!9Dl zp|(X_5$i@Xpuigxc?aCoe-H$eC(m>Ym&zq)g%gO8`Q~x&GF-O$QpdTn{RFH+TWHMo zUAR1;i=&+{Q1VnU_f@kdoRB1%?Coa6CVCOAYxtXL{kLoW^^H{yIil&X^Q2RtMiWpH z0g`U?1|R4I!kQ1pHqI6A`-DIh9t);g1bPDM?}y`VlZcl8O0ZhFP*8%EhYT`SGUaC| z0aC6LBiIYZ*Ris>+e_7<S(l$I0Tt(y7LT3-%Q4dD%$LpX81MW_sRZrCQ9Z#Trnw}W ziF=F-mazT^A3zJV%liOW3wC)QYi~75o<7}mN#$JH9%I=myV?jOKORKVU!T7RkZUOQ z6UM4wQa5!!OUVHt)Gxpalp!1Nkgq1=<*~f8Qfqy7W<pXV%OOk<e)){Tsy;9%h&V?# zDa&kqeO;5d2}dV!)8mC6RkI&_ORwMg^_;SvUiFY$dww-L$^rOqAWTS2*w4?qD*`p! z<fw=|XlQ>I)DP5F8hm=_Lezoi<3|B_WuluBv7bio)Z5hvh$r$H;Z9%|tn#lm`81bE ze~OgIM`)00<+J4H@z&4_v>)$O+|h{I?vz|JxHoxvs9?CI-jxKDae5(U)NuX{Ik7BV zsSjScUa#zW4#vxdgi3y`xBt2N{k_UFu8a4Z$qzbf8H=~aP%)y)mvypdk6i-KcP*>0 z?k5qY94|BX-$?d!)4k4dGf}-2hKg#-*7<)8RKBfLHXRQEQetS+anlHrA}Z0Vb;~j> z+>{+~3^zzkGoMfphx>#i(>Zt!^Oqr3Qlboj*aTtz&H!vf#7hV>yd=!9G2|qEV~ny! z%c3=~&`kual&$Myt03HKbSJ&+9_`Lgz9JR493auXRW|gA*5D5K#x09jRrf_+;ngkx z?q_|1z)5ONjBjM;@wh6jM)dDqu^f}^0d@UX(IgtA&iVZXwhOXymHa+=KF_VQw}vkA zv;%<()+B2AU4i){Rr}BT$<8WH+)E+|8Qwu#;Hmcfovn?9t@Ne!m$trA^K!KA(c6~3 z`&efad%13JRKu`{4qLg+?xL*%aBuDIj25$!JwEOpzx-VVtlwzRGNr1;#^g=3%?ZV` zD>VXf5Pj?3vV6A?`SKr+FjkuSu;_=Hma}j8GnpBzz*=)hB!7%!2hD@kOaT#<KUXiw zZ<-EH?xdn?Yd!1yHX`A=`g<;#uPWh*$>lDybp?b}G0qfw7_>tAS4LqBKj2-O?MlA6 z>o;>U4y1pZ>?>c62=YN~{EzuEI1~(@+D|fYXyllC59B%vwjZaC<_;FnoR(OB<7Vke z=6IIVI+yOEc<+rwzmZvXTkHAxkJHSwG4y#?1+$j>0K&l8K)+ea^bFXn0jC&}T=TVX zb*{x)fNb-h-hk_F3_DphHIL45bq=4LUuLtUq6Q!Gx+s{aNNNmjp6=YLH(q*icip$! zhN?`YN+|Me7}5e6bS);J-~coS<tJ*4TA!bP=NVpOF)VV34tO9q(l^?_-X@fyV1P6Y z6r-}Y&pXx>eEbOD(EPT*@6RU97?X8!rL4C_ta~#B{4<htfwHf5^cg@3YxU{ItIv1y zul06Ep5LP=kv;M!H9hzI9du{0wvLdx+88u8_RcKdOXHoyPTyS=PFJ`SgbcUQ6HN{C z9hLBumW-T#KQAe0=qNOZ0pQ6oVvXJTuEl+yxFk8ufd*FmFyyGk5s&A1R{5xop89_r zO-M|2zS)0@w(3=ES}a&Sk5jls??dDOucEXR`YeP@L&2~l!BF;qztL;!sLQC`@DERY zX!Ys8r%kstG!_LuJIyhQiw=m65!^?pGe_IJ;wCbzX{i`@<uI@QkFy(4;xWbRJky}v zH1^(ItDnnjf7)MlYBk_WYxMvLSlMOxf{T5->PT*0L*3{PeQ%_dlG6;|nb}BVt6+Cn zy*nCU$(sty+0OqN(hxU{Bu!TT)#t^X-2Ql6wLO|HocwDNU#7}K?y}0oQPb==_lTQW zdLweET0VKn4WIn^J`qTx!>H9xjNn`wmBJH>Kpib2)mN?HR$8=s^5)^DB~O^U-~Z3* z2>hE|WGN`wl91b_%Y^Oo7m*=DeFpvI_#AoWm(hZ8UW{?KGl9yRDBE)jUhx=q^e{Hr zHT5wOHRWM=EJ^z-z!k6f(6Na{u^7z!aSYt)<r#sURTIRJH2<U216pYXa4Fzdig<<@ ztIx{o!NE{&+ae%wqv<{Sk8cA{uIjWKqel5RG%dohGhM|A(vn`IJPOZ83iU|!y?Bzl zeXR(u5evr9u1FX5+4j6vbG8ej5Eu+bw5{5g|G%80NX-z&J~Df_eY`zq5|Q*>f1VPz z8tzv8r-i2s*}2=b`KB-LV?Nf%(xR0BKE)W({AKf`puE5UsM$u&Sk}pOBFY3`P&Ui9 z?~lGTvkw@5+JnJsbU3%_2Dmll(tK0t*QrY00+va(|6|9cv%FwlF@x2SjfcE@teqB6 zaYIwDrO|7DCt~5fd(@^3^X&)JL4)%n>urSig1OHp;-|OWs-DUGxUYx9err<}W3YNG zb+^uLHGl7C5l`)j9vWLTG;0yy?m1=Itl)>q><bPR4Z=la7};OQmgP^>;ll*|__dmB zO=EBV0NhtaFu-5G_utzB&ehKKSF1KT=5>!K{FC_7Rjz-rz$THyXAUY^A~#%|K#VeZ za`(p_qWxZSF|Boqtlf)7kWV<k^Jy%P@_?%($N7KaPIX}O=&bI91;v5EC6O-t&8}ee zuS@fVv2=<&z(2CoLP{Q3sr;`HllXYe>3y4BK^5iZzv5B*@8?shH+-}&qurwct+{&S z2j)d2+cJ1djRS+9c5JKtvjZ|%7X&SDyCYDXCgUr>ggub{3?rsN88eGA*VFfX&?Zr2 zK;289g<RR&kozH^$O8c>HxM*K(}5q$f91#j_0;2oitmV__z1@?HuL{ad+!+))$;v$ zw^>vKMY4i{oO4c+Ga`ay5F|7?G&!mWlB49<M2Qk4)1V~DG)T^pB}vXObwKa^&GTRL z%*>h>&sxu#^Wt9VK7CG|I#s)N?b`MEioWEx#QpXNG=3S|`Gn{za^f^JpL1&s^tY`q z8+&ITg!w;{mOacHt0mbkN!@5Q_NGM%R_a5JcdEL0WiR3B>egjMCcg_vJ?Bxl40c_- z*&!1$>PhtaiX7Y6>Qw+6-X^5HR^jvz$K%KH4wAu-A|?xQw`{$<pnF#Fg*%~CIeEjO z_`!IQIIeE&w)>xVlkBpH{6Ms8=}4CIEli`r+3Bxcik74)58=qIWn7w@-#0HX)H<CJ zSE@M&27tb)d|5N2{L<m=3<(Z?=7oK*`8x}csm_n?JJ6R2BOcrf6~Ko`wAgDk0o!uU z56_^7lPrzLhx?Qnc4azAVAm>3!j=SO29$#i4Pk%pzi{s&gOcV8xbH}#3_c}nSRbHo zhE4$`3u5~KoENy7ki!{w4B|5Ob|&x^9|93wuqoOYxa!co7TlWwj@lf62UEaCbg=#W zFRBjW<DVa%LH%2vGF|KZ7|BCNr9uvX1s0=z#sXWQa4&}O1Wy<yhwsT<T;R*wGq*S( znBZN3P92{vdoQI2(0hG{KwRRm=LItWJOO%FjC5^>lHK9Wa9{9p=GRAVA@0^n!4fiY zYdhpfr?(Op;S-j-l%9sM+pB6A6z>33R&WL2+`t;%)HgO|AW51wak5hecw*~k;C0-8 zuN5N=eE&^|1Cyvy0&!sW-#MH<J=p%rzX-97F0hGoRXGEK5kDXpF@Y2;1Q4{y{4~eE z=A^*m_*>T*1gP)AaMrZeoL}yY@F~mw!$2{}{zCbt6XF%AsrPA{Ipd2plRCH_q{eLG z2o=Y39XDJ?ltIXWj(Bd?3*Cf|e_!Z^GOKy~cgVWo5Y0p;2hdEg%DrOvS$j!QyI44N zjtN|DLBHsM#tt>lwK^EZcmN@*ruj_-`8l!&13?3|(eBnAJ(-IPuT}%Xi*R~j`rB*c zhd$Y5b!0e!v41gUJcHK|?BG|v>HHW7p1*vck{`m>dX@89{EGuVJ8AW`yzwyc$7{dn zvY%cES70+clRtz|Q&khk41h%M^l7d<7BhJSeug+A1H}ixqyh+-bXD`);rKCXd1uFR zLjKUbit%{vyJhdx#LX($7w4cgskNA$RkqZ5udQnzCjzclz`|cfWcZI*x^KoI_I^*9 zNo;-(eF9KWzVDpCo(MSDmN#T+vHbd?EFOT%<Yj^)vw|Y`$9>q*YrDiy0TiKOA#Dz9 z7ht+{pax+CemjrrW_n5iA0(tG=Q6|zqwC=L-=~Dtf;EV4j=xDW|2eh+pzHbZUnqma zkSuNAVP?FIG}x*BmsdD(hKPf<wU>asu85iMcPIzJW;zHT1A6>9^ASQ00`vq#asbDe z9N6*w=Qsl@{`vQRx#9We|6>}YoD588<US9A3JTl|9YZ<S=a7ptW(4^47}RNp7XbT8 z{u&lZ`)YGbsSHB}*&nFH*$L3*b8%1)zjsC~LC4?zk8=Mh))~VrIS@em0p3OXFSrpT zzk$zOTyl1Ss>@&Q3-Fqb|KUB(btNT1ATYN7$M6C%%ZkAscyPCxKX<EI{8wZ-=zj$i z_n$1c-n+7%ki-HcF7XNQ{UH?rB?bO-2?zjr5$jC)_WQD?an+u+`)b8}(V-6(Y5w@` z6HM7FV2@#^`!4d0^B>?5KvhDtN5E;3pF5%s2w?Ct&iNT^tH_66a6#5S6cyC8`1Rej zr4y2iK-T%}Wg=(*HE#x7Jx*M(lUuN-?gchwzl}8#7}RroW-<=9$Bon{lfoLecBkVv zpZswzW$i(!Kq$7`=E<>j=YHr_iedm>`gIIqu4I4Tf~WjXP>4X!oAm6%r1SBlaNQ!H zTwPp*V`_~wKijNlTr)mCZZ<+(cE&npQUsj8!nnv}F`{FxB!_XQcZmhNnkRc_dM4pw zM;ZHCmppgW%dKA71K=erS`rjtFKbd*=GOMK=GryRME`+SvO7?-1r=a15AF@1&vKmS z_dfd+rdwovaxYe-?tKQ)Qm}1jFx&aZOln*nH#)#W_6F=WbI(f<oqxOl9#YsAJmkHO zgR_S`|G4twW@fkRMTUS8K-7ynn2sm@CE*NFZqL646Elw+03q<Ld<Ez}b=yvZ3SvJ9 z&F7Rm|K8IVyql7ylFQSOE?}&vZn1m3?S8o*bu+xz)$Yxz17yZFlTP^{iYL_bwzs#z zi{<{AEI$eX-8uzNA*LXBMtiH1d+PON@zxrjb)P~O-~O#V*yJ&UWnbJFjDV!wJ$af) zdoL{57oZMIzPtg=Wqf%z54Sc2#?&YJVGB_1L7zfNkJg^iG{M-)`?u(gSVo$6hD;R4 z9$LV{#n{aJAt8t`av1>d5deyEh>qzE4#?6Ze+2KWAL1|LAmcNFOc)efuD}SY1{0fb zpNm=d-Gd&vdki#p-M0&2*I}`Y`5`n=8lz^*8F?q%6Bt6-WGY31M^S&Yu6R2V`2kf^ zAAh6=WV<Y%4mLr7&U2WdJAfH7y@Dzs5-^d0;)AcJ%qVmV&lZt?RWJTu?U0r(kl})# zy)A-4hNiRT9L)oD`qb^+U86E4(Ayy*<2k8NNzRL3KQ7uwpbNkYOLV`!Ju9`WI|PsN z83CILwogEZ>J3*Ah=Bl`0E+6X+Xk_CmC)@wXH?(}NO@tw3XB!+5^aUCJU?X+?6rR~ zhNi_ZB_Htgh;qmyNC3b__Ac&}YjrP_Zo77Zw3d-`r%0(D*k?q^_2LE$fRmat0ODf+ zPudB#RKN1^-xYVb4Xi;YNf*ex`eQfjzE45iXuAucSooCp@aa@TKsLJfeFy-vXe4;3 z#^K7~P1s@usSBG{#(!9ZII(c9*?bzk2aT0}c5D&&I<AEoH#!01s23U)UcD39?q9ru zuNi_VI6MWESgV(3mY*TjzXI9Um#mt(apjolV@ZvG6<4%~>*N1S#Z|%QkWOzO8rptM zt44WjP<pT|TNMiE*We9wfF`8AL)gTJ0><tn@z?L&AC6oe3fp~$E<`OxcP5%&>bS8J zTYyGf?>+}rQQZs@2tpxzxrXjj-j}fX$#qYG$NLOuc+Uri^0hzMWJ~Ze9`T^{CS&z8 z+lyRDJ{z_Mkg6JYDDlrUcuhW)-_Of%r*eV}N1L)lf{f8drS@s7F@eUGUtFO&+L?s` z^`AI3<3Etd*K}EtC5gFuZwvSnxKjlR_=5{o*{^_g`T-pex4|yt_&n|=|0qu&yibF( zFwy7kTEVW`Ely34L0wgw$#cZd-cRkJz)UK#7*U%U>bm1J$y)mEjK2V5+J+!n_nxng zzsg-B4E!n8ZfJsd?p!W?rGsO1qj3q;uVlY%jX}UVW?4X>w%}^{^mE#h!(!td2SmB2 zNY*a5!_SFc)y(;|PlXnzE<~$`V{#q}XVvoq8g0UVKq^>vhavvqKIrsTE97CohSmD0 z73IrR=wImf%~mgmnq6AQ)I`e|6$=fi&;}BtetPNv0zqn1?x;{f9N^sbQEb-JCW?%C zLQzhaYTuc(&FIcp{*OaCZ`*^#%z+0M<*kC~S&98(0x8ai>)h=Lyk&D01cv3SE<agG zz`2{MYsCZ>V>1&iKYh2#Ntc|s#x{2EEp<*Pxq<h*lu=6wl~>n<cQwK9viVLu>B;B0 z^`zsWBy#9;V6XACz6yZ-$bHS$>rWB9x0&ukIM=^?GhjsiPe`QyyF)VA)%NH3Kk{$> zlFt7BReE59VNALIV~=G8lDL7@f3Ru;79wg4kln1dQ6cBk#1bEfj;(tgq?DdHC1<sq z2NB0%m~1!7X1vaY(31Hf2Prvd>Ki@xqjQJWm><|t7!@JBoSbsze7cwisSCiaq+<U2 zJ3HuilUZQz=Vm9brDzvM#n~N%Od+uzAtIhp8z3-q+gTtH$L7pW@Z5pKbaY#Z&HugZ zS^4WCIQp$*<j=)Wpm5z4m?yPq^;Bg1-zSL#wi~AOy!bzg_y8(mM*LWn11I}v7%|OE z5tPw>RszTk0;dB_THb5?oa0t9`HJ7u$HBA!9G;<&nwM@XSZt8*m^M5YtnbYL-Z6C4 z4RW>KhSfzBgV6_qn?tjZ5-@dq`X<WoLkP4jK&65l8s*Q<ZW2E$Ddk&A2!d?#1sXCq z&%V)hMf@uS_rC;X|No{JfH^f(UfJjl1q&$!zQXha$1yomfZPN6Y%rKr0923Q>*Oe% z@kA2mz4c$yjM}x*;HhCCXn)px7H)K%AWSRQn>2_~?i($ev}I0<e_4Fsth$-A1_~1z za6Aswsz>4VJ$t--_v15xGd>|S1qeE`txPS|5EFN&Nuvi)^MLdyVB=?=s}pK1R+<*b z4`^3CuBx?MJK5Tx9Vz}r4}hXy4ieW!tEdk>#^lb#;F4X=l?R1hf^d-v&+N?p&`LY2 zOY!Ol)b3c-wfAa|XCNqKkYhEv4-F9o9obF!0vpPGegTC+u$G4dVCCS5ev!|s-j_be zkJfTHHm_V*!{fY9agk{~Q4f1=CVW0q!FAa$SGpb4(KSq*K0OlWh4tl=*(j~B8pey1 z%wtALW05;6#MjG2_bBG5FLF)erbN1|xqW6y8Znu(%YvJ^+sGVrI*{ZFpiU0D()<~6 zYZ>%TEUVZ1@6<rOZ}^-U=_y6|jl^MmyR0un%Ihg`G6boz?!E~H+A(c*wAD~$A?C-3 zdv{6Q`5<^|Crx97PM44WIx0L!rVPMJyB)RNom>DNfnIH47Gw#?=;Ka>L&)BuMUL2} z1PAl`Yz6r4y4D910uK7y>oIXI#NI0I>Nh$^-8Tm3p@PIp;fu+B_;;cKUWwshL$hg4 z%+Z$7I}tjM^+m=Xmw61kVn0IgK%UKZ)Gu4hi0s%On+<dHcJ=gPSo}w-l}oFEa5|#* z8uHu*Rttvv5(RmSl~mt28---rCF-5{)I%v6oVhCQ{Q_M|y&l4nd)~zP3F)u8M^F1y zmX&creT_{6F`~59wDe^vq-u3IT3Mh4;7d<;rcdLcSKFH^xF=sZuc4y=kQ@Mpg)NKN zW{n2|a(-rmD+MP?2WJf6?SX=9r(@n-i&mPJ0F}#TK&w1@fZ)`_W#e$M_H^%%z1Md= z>Jm4)@7emcGp=cC7iX#r7%z2SLqK$la;8V}{M$darm#G?ju(?o;P#}TT**U*9W-_l zC2Ojm$-do3Z$D#rOTA=K`3ShxIR~S&QS@@Bi<LP!!L^$S*FgIIYn#IXi+mLiUu(DZ zn>j_+_?d^lv2kDg^6i$A`<=#CplCpzVXJ1~?q7fJ5;q8rV=a04+-OUrz2?ZEQ3~0w zm8m1@Tj#+wPbuokIXKzHhmC$zcd$Rrf4hmVyjFW8?LpK^N%Vq``lYbb;>Q&@o8Q>g zK4P6!(1%ED%Jo!@Hs|Qy(X{X#E*E0UdAWG2LzA0s3uv4H9i3SaFo9#6y9|TNFr6nr zAGxtBG4bUY*Jc1Jw%M|}kT<ZYt>6ebXZ=F|6J<e95umX0)o$6Chs+Ht6i7OZe_^g7 zZM%!B;rA+rlH)YZxmFbua=v-qoEwsG0P!5SDq@IVN!Kd!umXbCia*XqiAD*fD&~7_ z94uA!7%$U$nR&<UP-nIm(CyY8`s12x?0>+P9UoJC;n;GCVO<j?|I;lny8&FbP82P6 z2P*g3b?vDUO))j_YH?zVPY{qb@Yk_rFw#brxk0H5a5poY1%>IIlw&)x%fK(Xry!w8 z@OGa&m;S(1EX+w>Xr;u*l_A;xTJuS{Rg1VImY+4S>5yivD<>y{w(C}GKT5whR?CQI z)*@iT!q0E%mkl%k!FWRcJX)uggLy6xS`j90tdl6bD|7kT&zd_Y#_y|Tl1F2V0`kM! z35`&KHSP$t4X3=9QWWD#x#|GG>1Nlwo0MP0Gp42{xL>X1IEDOWmHDE{Wr*ec^1<f{ z7tt>uI|Frj)5NNs;T%&EAfS${T0I!~K@-Pm`N_!@E#}{2tcjQ+9pG~ZB-SMq5Jode zVr0}`g<egg6Ro`hVv1@Wnu85P0`H(Z=i~RuAgdD2K;FZQ6VEq??FRA*UL=rbSKLeT zm!@zS`%(zp<1cy!5#km#D?ujdA2kDt2_y-n-bd89m{vXVOKzop52RJxX2)GPF=Lzc zvM9$r(C@eFX&McJ{g@`G19B9ppawt@d;S;;(s#DXi=R@F@2&^y#Pp9LM8*azP7f-# z=GzKn4tFpwOFw?NaSSlxQ9AZe&|d|Q&1rvD=k-)5KxY9nM`EL{U^X5$_OpgEiLb!s zl(NryKEKwzFL|nU4D;9$fTa)gX5E|y$u3gs6ttUqIoOnq+DmYpM0XNfE|dJgd@h@* z%_-ijeJbm`qjaf$MvEKMOHaL54jSzOqW-?gWwqiWY>5|^Q13czo)}9I*xe`2`S|kj z@_K>`>HV$o>mj1=qf@dA8rXJMM>wg>!m3c%>eMvBtaFbl)o4(hZz-8w3tQWbGnMX5 z$0^55=)oj*MZql)Ea$X;ChB;b)+5A_flO~@Tz?v5$oO8DlRH2>R)a{$aM2|{?-Iku zd|^qLZ<=zt%dgYc#A=L^Pt<W*Sz7ehQB`&*&+2}j&$78e`>smVZ|d|gxHhSy>)Gaz zF^>Ucb4*E=5&<gBQ~r;}uhygNATy8cr02zA<WVEF_O{YG#94lKwNAW1o~%kn1=Ui% zG267SXNwG@r2<g5RxnCLgNz1moLBK}j2h~!nyGNtwJHZAWk4nKzDc|3)QYWjw{H`4 z+3a0A1>i*pWrC#ws8yxL-r4wo-P)3RD=xskyiop0nS&)Fy(I5|tCAhKkf228eB)@n z)yhN5Vj8Ab$<A#^$PY=7h2jbXZ=m1=n0Bg=HXl3Qh9zr&9l4QgE<5dj4qf}bQm@r* zdHQ8Q4$&=`RXf1wnUH`CBM=AUh8z;z%NcSx<~*d_HXv|`@V{QdxAlAaU1IG5vJq5U zH~$?No6P#p=*gc1KvNic8FkmI*WA`}oQ}>ynw}PSlr>rzs?6Y(^Q#3QPT|zrd=m<R zfbhpN0L6xh>w<lJ?7_P$BT+EVPY~$+@mZgpmS3Hl-)LiKISJ;FZ8vJ)rLH8u=$S$5 zp!h-n2xkSH39|$U?fd1NEimbt_$NzJ`l$Qq2^3WA`~nMVodA~riGP|V(!TV%0&9|o z!oExw!D`GWr~BkS!C(VCs3Dl}zFze8GTFE}bz7J*9%Ld2-hd$*$R$B@1gQSKfJZ#Q z$3Q9pgEdJ&{aifDRX5vp@&@=WI`;Mho(OHAgAqGF2c(n1Q$gOa=N)wr)9RR};@4yl zyampa<*z;2E--~69Z;yp_{<g&q&Sb=17RCzZmDZB0`@X@OoNCbq%sP1oB&~<=Fp+9 z4Ea#HCLkU$&;z11rir}k4LFe4FXZ4d-BA20h6|a^haeJd{A@c?NuHbhJPh4&Ho-`J zQQ+4G{C22AGJUBA+!*#g4Is)ADld4v1#UG_^5bgl=`ru3$LgUR&z_}3kEOc{*#m)% z!^_XM#vN*D<>Geo*_ON#w@@)_Qn2op4H|~Vd|#wr8wluVra)#-NmO$g3^fG<e3{NN z^8i?(<Ge6+8s>fgNZ7McWtjsLgCAkWiL71Aa+&dbf?D=NJzML+TA{;J<qkB@Z>|XL zjcnz3aEAb?VR59<@cLqdU_8UZMWycIj|m+BH_itJe^xD*6~Ek94X3p=hi`(4P@DWG zSH2~Y$u8f;X0r}8W|F${<1vOAW~3LxSFMkeeM+aWvcW!@r!`}#1XiaQd-IyK7tOb6 zs;-#1n9AMTHLfO4zFL6YKr={g91+2;NavNmZ~D+HVU3Xwt4r39CH%GFh8Id_zP)Ah z)XA_%WMn413*RCgw*eOy*XCyfhD+SZM->Z;{D#IxG6^XC0|q;g`kZvLJ(gXj`?R%8 z`TH?eZdg;3_yw4#j^FMsw~RlY1pmTa0N9M$rr*f)F1P}7*5Bg%Dc60aF6r7at)FRu zj&XsJLX~jhM71`Ilh^Tg^72IOgptPc=V5=Qfu6<T-WniIx5yeRIXgRx9{+hUM@5ia zaAME&0<2-Yhul9s%eYWf=Eq6N)<TjxMj31&JoU7XhLTslWpL*5g>Ku47<L_JnefuW z+fgqbWbLR8aNyUi^z`(!V|^(J8|>-sen{&2`;Dgkkiq#BwUgoNH~NXAIZlkS^14bp z;DW=11l6nD-?6cAv~2uzAtOTGwc5}A)N!Oo1Ury^XXCqW&BF338nzIFYL8e6{`xkC za8Y2XcoSY%SI1=iQxr%vael?~@0{r01}2FqA3{&UJc~s?Tlt@u>3;LVczdFJ&|^A> zp9AweNq<_uyxc}E$hc8E@7wwTsUUdB$c-L<1jEJ<xr!C$&#>oxYsdWxHfl@8P|_0q zt}SjvhOGbBFBH?M#E<E9$*Jm#QqmjOnXnhCS03=%Bla&ub&e+rxbb*Eb_X~+Fo(V0 zuZF4|3ib&-dqIfsp;CtIovqp&qYh3%oy)MT+O)LLji%D)o9tnaNvzO2k&HoXH({bZ z6{&Y$|KNUdS!nNV;GSBk*~hWfu$UN<&lC(T)69o3FUCO~YfNAkYR_l-x}GMMmQle1 z2j7kO3g27#yDX4)@P9T-hrvem-|y}!+{7Oz3%$3F+RB*2xd;pG->|C+;E{lB3GI~7 z3X`N_QcvB-54dXZT&eS>lWdsLYuRB{JtxMxq|y>&jkc;a)DmmH<Z028;K{ULJJvU2 zTgYa?bSJw*8-|B}o;K6b(9_Em6%}1x{ru$!7_@Sk+11@KQY&t>`*YaCp5dUfo1TNc zwIH+7t%I(sv$pjJE7aRte5ic=%0A~>PGKQ>*=22PVwl}__`Z_1_LSSo=QzEHApX`w z`-*$G^tXxpDz;0>cQ#6v+YA=N8U+qt=4n4sR!)kZMIzVYtb)e%r<-(Sq~I~fe9i^w z<A?V;<7kEG;Ev>+oCCK@xE8VMOU*|pK7IN$=c6knC3W8NMkU_SlX@H#?h}3ItlgK3 zRoBYMUzGucZ#&60@h$mqO%o*`Uw1`lCc$_O8&;UfYMQ4gLH%i3Hd<7Ks#C3$$PE4R z<E}?muKMwdZnUJc{GyCbbi?Xq+`zS46<l3?hx@%&{&wxl**R(XJD+p6#(Qs+a;>F* zObf&eyrXdK(o{Do0`8e!Sy|bp<J8yR9~-qgK<XVx!XK*w8C%fZV@eB21Y8G?zWS9G zUJVNgc?jgYXM<HVv$B8bgL3+BI!J}yU5D&A7jIwRgGjq24#YkwSxgL0$4kt_Js*wd zN?qaJ1+i&g?+Bqcqv$mZgc;mYFj-Pgn$g4C#IMJzO_%KnA4Vc>mwCTL$d_GJKU&1< zESXbX#A*r)5AVQgMW|h3tb_&^80>LQO^sF%si4J)hm)ykj_Y>cOB*1%=BfW7o<cs} zm`H8<=UyLJ7CL@^iU+*MSD$kEaJ1D*ep9q`)wa@U$kA2GN#L`yv9){?<7`D<;dJ&; zTw^}`CC^!_`oy2@dg{pYw*fRad~hvVI#I%Yk**V(Z8y49chN@F6DnV`9&Zt2W<yqk z#bT6esMf=yn>2C}t8eS9m%&EGP9KV$*7C39z+hW~(4dtpLE?sZ-;Q;c0Ih--Y?SuE ztk8Hb4cIP1;@8xS1q+*d4{PRtIa{&A9Jz--0R&iT+Q&0e!(S)Dj-@X#9Nb{9Cq8@Z zty<u%NSl~s@1JfELB7)Jnew$)-e)U|ZY%#^`Jv=4!?OZQU+jF+8Mm}Mt^CPpS;vgV zxJfV;EDuBq!7^yvjn@fFd5quAL#A7Tm|qqKb)o;f5(jH>lLAL&AT#s2hJyI<0s(UD z<C;w08<Sq7ULa<XD6L~pRH1}Wu5sSF7Y0tdjFHDuwIycd!b`0Im~7V2#90xO=h?m^ z88JT_C(lA!ceQs-fQK};XK4b1VKZZ51{PUQ-`ubqfBB??T1a)v$7pz2c$WF>m*78N zAzuP^Ck2xps)CX)s$qO2E$z}#N7b5AQiMi3&^qGz!GAUZ5pBrsQ`q;?z@x900&htt zER65GJ3%LHU4fjNZlmGx=i1t@3QWrloYwjO{aW;!F1Xi1YGoy(YyakE!7m$G5s}nC zmk0l=0b99YFwxP~v9-fo)B3i&n~`VM`|ONv;s7deDYwWNW<@@dU^Wi>(vn=QvZvYE z+2pLOS|+BoA$haB%WnT%PDlX_ML!SdsNrdJ^arS}(!5z*iiPLE_*0A`Uqvl1(?b61 z`LRF^wVH>I&mPMU&tWKWW<7ePpt@SIC{yeF#X%<uDPZ!<lHWlj&qc$8*j~#Lki$j` z@>g#T=_u0&2FitnvB<Nq_K;JzJo$57Y|O=|c5ImE;R}zgAkpXX;A7Ee(ydN&=M5J+ zOZ3K*fAkQczb8%3gLKfpq2qu4xAog-Q#dcV<udQ1s@w4(-&-kcnUHLeT4W~nQ#~2C zZ~NJN`}}G!Pn-);1UZ(=A?%E0vAS(B=9y_HLRBvY5+ZnS{3N$ydvtDm&dxrjWq$Ou zNhkWTQZ2g_fP#@vH)l!;yS^o(Xo@@T=*Y9#GOl4OD``E+R!D0-9ImZZhdm^^UfvLm zYat~^LU%enQZHvKvx2_uv^_RmRF<KYNNxJ@CovA0`ySA}_<MM&wzW!grbLeB6s<7H zJgj77X8y7n-;UT+7GZQ5AE7iTs3rHg-FE}j6e#-e%U+9b^r?LBX2zGM$)^w>#E3Ok zsR$k^phju>l9D14NWZk@g2m3iSBJs$#NN?w-kcwN^>MtryRCiHY;zdAE(O6&8E%_2 zZCG*vC~>!u1aJGmBB98Nm4EM6I~R`CCw}LiDkxU`_Ym4zq(Xk%w|%`DpHsCAcI#;G zEbl+N{UAI1_)su3G7^t9ENipck&vH4tto(nTulwX?Z>C^jv}#lET#iTsS!0N84(c? zL-QAeEU~RH!h`M%J574QUnx?j`g}LQ@s^YqUw9}jGeZ3-pO>0+AmMCb`25~h{>=Je zvnjJ;;{~a%rM1e6)VR1+ayMm|h*aSMFHFVnEUF^q;$#et_Fhm&v-Yx;tKOK?qzCUz zjYRuP5Lkt-5w9b^1vQUar>YJk4LAKNew&3@TA%&pULlz=^zHq+DxG)C!>vElo<^DG zFFTXc!oqo+Km2`i(C*pb1Y-5{JSyLNxK$SLZDzhRFJfX=E$ox08vtB+-R(+aPw}<w zhVZT!eONM3#PNDIG5E?y!%&XIgtOH3XkW@{Wo&vQdly&GHAFLnZ_unaL|V(u2ctq2 z;gRid+7R)Gf-dz9K7SWs0D4>N8qdh94B3a17jL})OZ(52l%7~yTiXB`qFAtf<IiC+ zO=^MWz<cl{ZQASc(@QKw2sBz8{O2$_q|NWKljAIpFaylltecwAH~A8Kjm$<xAFF{* z+jy)!j`+Hz3A#{3!2*MilHvI1Q5cW&XHAYr^3SVvaJ;ctM!tZI^fmU67BW&K%Qzp< zwtkkf*f1>v(AzCQ{#a5AnDMP_`{Gc>{?k=HBmxKVx^t5U9Xd3%dB&Gi(DWM>x+3YS zMr3O=I$^h$spDN^Fl+vGt~O=J5Zvv%_(*m7y$wsH`wP%d$%a_J2T=DEgi|)OkdK?= zCkZy(F4^lX6qDA_V*8<|r9-T(9-PGTBv9M6O3q?$$La!;6&T(lO3xQ>pc*Nc(i0Yr zvTCcR9q{H4lZ?}0sAdw2jnnAho?<Ipuy)3xoe7J^woIJ*`m<Sf@CxkP(>n6YIcnrX ztXz{PisLKWfs_VcbMH|6?n%%hP59i5jXN=@wr!U?6|qb7j*J`r8KW$BsE1}`Hz7oE z-0p$z<+|gG?=^maQRyEeS|@j7zN@gFo`?a1auk^o&3%P6eX^jSRDwutYqZS%g|t}X zss*vyRyn2nQ9M7MRMM-Hql-f&6)(WV^0KDzXCigP{ZJ3jbX4KA?4`DauVdO0!EScD z*2TFibSk<OL$-4SHBFcv?6Mr@maOy)El;ksM6jX6jN)Pyuv#OdbP5MDyMQq4BKKXX zF8<A2Hg@*7*-&3!o5KV_Y*C5NR&5c~O0P%Dt^XO2Ch>S;jX`f-*$xXU3Yo$)H`_<M zm5roC=MUT`=e&pay*;OBc1n4v60^d11adFw%JaH&@g0*4x)QMrJe0raPD*V!zMeu7 zle5qAeHuOHrKWe2^E4)+TTqb~H}Y@ISkaMj6Y;6hiqqmSH|2GcjVI*(k??YY-{LLH z;yl)3X^hyKL&wK@p0$1PIJn){q4bIhwNi8XGM#IJ{8XWUxTUqT?XWsZ{vl}f8$D30 zfAhEX!^&F{97fF;A!7jS<Iu`F(Ff}w*5|_qU`0(}vu9o0DChxsa4u>hzV|^*CTll~ z$I1P#bz6h_XCCx(F3Wtro$ohV67Ao^(4zd3*NHfWTs~`=Eb-2y%xlOBk#oaCmUFn_ z!VE-ag%_#6s;2sG4PJ4oudmGHKrL{0MSN+k8EA~VBW_{Sr6rU8sbWZ5q00Cz-Kh`m zSmI_gV{OE>HOA6`FBLO8yx%PGs8Od+_cz)EhnEJlJn}So*D6Ei`L`R0hRW9JwI}ee zdjTP$KL>$ii7(_rX%A3*e3ol1Ed35L)i-g>tYc&1%*8PS3Hhb(;{2SI*rks&Xbg66 z(W>bjW<Px$xgE3BM@rAoWvR_bO+7!fq&#j^zO-uasLMs)#kZ!~W%qESuIS_4{G{V8 z#g^}qSciMS51Hv7iTprpa8MHr=7DnIH?iq2Yksqu*k0d=JDo7RNXZkHb+E~Ls6Ha! zQs}KDkm?;DpX$q2t`|sg1L`i%UD6san<i2p4)cj*MMkUcAEU5lC!bFAht6eC^vZ@s zM-Q%45ZM%}K0$m-pTe8p=b58?Q76_#J)yo@oR_(K<>*Xu_x#Z>$$gEF*(oY5^CFUq zh>J7lEHf^j1>N;xV(LJTRqK-+5~f!Km_4!yJQXt7O;QD^^KdS4xda~SxeNFYa;7J^ zkRw~l8oOfOZRzc($4HmDrU@5q-KkdhFs*f!uVuSk6xL{SxQ;CsKy~9CTswwcm;^=t zG>kvt)N9m1tD4_LvKb%VO^$8K78=C-%YseuA(09C5dM9Z)3VH0Bf@g5-yjYI2WoIe z6uF4(_q);UcfYTREMIf>`;V9RA;aCYsOG*2o8tU>`10&P+xRjEcIsj4$wb^vC95os ztXQLki&{O~@X1}7{r!pyW%ziVxL)%<4)m-m>`VQYO!TbE%A_~3<=D!%Jd527B8);t zf3Hk!SdU5J2%#VP=kuyTE%BLtz(urfN6kQU+Y`nT=Yl?l7OCcyr5~J#=wpoI?q5G| zebnS!;~CoM!7HvdH?5fI=nPlQ9<7j}kW0LWIOV9_!AoX5697J|de1jhw6tR+?1-5K zW^P0l!xndtwfGvUjB#ci7x$x$$g4MM{diH&N(#l28i~x)<{OF{$pueoTq?HG2hyyx zjXoO;m$Z>b3I~#!7@OVW?CrY!0hdVg!1(F-^-a4&6wY%kho14(`*K^H`~-3{3SCwD z^7H&`qML}JfsI|_*jdo!cxo*yBDJo`OXeCs@nff07%kK%cX1A?S)0tfCKoP4r<xkl z>Oss`hn)5mSk`~%k4i$JD37ZjuU9Ov$3Kb1);QB=K6{!vp^q_6g*BsN3lAx2u~WZc zG;k4Xe>2y28d#^}$)Hj>YR1V^uW8cmUm*#W&`Y=3U8{6c-UbBuSm#8t?G65gib7=9 zVASFEteg1HBT1cWDd@=_iBGB5;^rdpCoEV~{lI|8)Ik$h`ei89*M`NJBlf$Fb1Qc5 zDDpy-8~ci3To8i^H^UrBT&cr&4ktY`|LlE*v<~Wnp3lCxm(k2(g~rzC!`0!6woxS! zx;N^Y@zcIG8S~N8^(pg<hojq_^1RW=HD;qG#Qu2RM0lZovv;+(x9Q38EIoy%zSff! zBwNMKfoYX_{Ceg*3cTV?J}~Xl77?7U9@>Aqa3-gG-lMJ-(Mi;$Xa5TS+zGd^_#(mp z93~6+%Ff<pi7fITg?L6*AWErtm$tX4tE9yELt09T6o~DFL_~CcZan%{v9TFgi&y5n zjK&=|dSDa@CeY+x#*Mn#Wus;rcO?zQE6K?1yy@BQ7uz3Sg(j6d&SGNzXVbwHQhv^7 zE&&{+5s~(#r3+k?Ir~lfba+cKFSN#9v37oUO5AjJ9s~3By?9bd`yJfpB{PQtYf7~a zAFC#?mR#L3#BN%K9aZNnZxRy`rMr#<CC^im2Sy~F%%FRlIa;(F7+Kq#AGqNy@d#EQ zURX1#5y1utA&+(YXHtrGb*o-@xc7KH3k$y*H#;6gaU<=^O2mHlfF_rz7e)=$gLhj$ zN18j_l)|sIMU*VK%OnI+g;VGZ@7uMK;g}WfG_y=1-M5bz4SJ@75!_8g32AJacjfEE z3btj|+LEryAx1+{7N~8*{ls0|0Fuq+-Y$E%IodmpT_KnGRh-qrdd`V~a$P^(as0OO zCOmM|eIDyFTBQ6RNBR~50|R4cgEaVuk6mZVY3`Tidv2z`UbDW<;#I0?9-Jr1{%FND zBGq(iuuv!-`tz27>YK}$Z|F1~!`1rN#Vl@@8xu6%n120Qd)aMVlUMR*+N+?X;~7QP zt5WayPD)zD-x0?IS-<GCy);yF=$5pZ@Ces~=;-r~DU+y%Jct)dTukgGbga-k*{Kxk zA$Uy}j%pUmtd)>IP`lU3+ZBIWUq!lgGXHDZbAGAfjR2*ae2s^@c0LI5L`0yrvb)^P zx7I!8gt~eLvE$%ihxqk!(eH1or#8Od%rktU`0e*oLk%5qoc-RV?8E%v-3r4_a*_0O zIoWpsBsd+OZ6LVr_6;v8(c*Co+oEh;V@cxZ{v{W$XC(_@qY0?%z?teyj;tS!NiW}z z*xaQW)zj)FR6*;<#>HN`Dzk+m(p!}0dP-aq<WVj*u{l^cAM4h?L(gcrZXx(GKfxj4 z;l|td8W9#)rC8_fALc12L4=f4P?*OO)iQ2hU|pe-c#!H<SzP-Ihj$XPYt>~&I^Bkm zqHDX_!G$kgEJs=y%LI@=Vy-R}cbg?`{VdD@#*D!W_@=UopOfFm(P`PZbTj$Q7D`05 z^%c)K44oX!ZQDN+H>WQMxpXaC;hOlIG#UJ6E3Jn6s;!_$<(daWrDA{nNVh>!X*<^? z^5IHk2R&w>+|I62*XfV~2?jwJ2uv}kEf>qZW9r=8CS)%!asl?p{4IzaHF-XbJFX$1 ze4qGe^6n!do*0)|CLA_6+`rxUv5eiA)TVOtqboPsWoyYgN6Si4B1rh5Wmf>Dw2Vx# z#P&@4_fVrGr+zw(H{po^Q;V223Mb31IKP=zXlm(qF`SO7QftJBm1sn~q)n>3mT#3A zN6(Ctj+xeCnaiLj{a96P79*szhE@I6Ji3jvx&|HwxNG-cr>s#(R>3=6Qf)T=Zn!U) z|9f9{o2ZuOxVkcackC$3$M596^2<#f;Xl<6^U6t27Xw&&g}TP~cX3<<1hR6M<*4_T zaylw_1Qt?o#WotkiDy3OTng~kKqRs2Eqq7ozdA+MHm=k8ViaPOb2a1i#iBV9Iw+K{ z39OnCN)f^P_1QCw;pm^qhcvVkScBC+U%%IIN3CBOay)R#l&jJNCfc+XAJhG{q5Kw{ zBPJnf35NqccX$4<&&8BU93GNZW%$u*@t7^T2z{8#6ZU=D4K?IcLE);0)(!Ni6O$m~ zuBYaC3{T&6%9}?onW#$C%e)b0sT^}cdB?gt?y_v6j=BcAmKE0y`}^Z`{p59IYB`ts zk8=oTBiMv4Dqhlh{Yy+&Ro^S3G+R{qXi+>m5eP=v^9VoYiTsXy6@meW?;MoojgSq4 zMos3@6l3M<LF>1jdtgRXx6dM)+;C$-RYedrO!*Ai)^_@;qKmn9czI+8*)=&)j{F)? z(wh``^%=so7fU$c-bQT@$r>p^53^K12QAmT@4VJOz5dp`=WomhL@|OeiPPTSQowbu z5pZ|S<}0a|T&<qQi{WM+sm2>sp;E}(msmsHJ1hs08}7S?GO`|49N}EeE5_wwA`EyB zl<wUmioCwlYVT!TS!^~$qxQu)5My~krv3>r=f1aX5ep+&;IJ0Jb6Y!kMe?9``j)4w zgZ+dcmRl)UYaufouw-f@VnZ7k{T5$KU*VSb#~WR+4ZKF8xrjr7p8wHR_j5vE`d0VC ze%WYggS+LM&aU?8!CAt1L|W)ozg)D!fq?LpP;5eA*wavjBHC`d<u#~+46Pj!vnWZ) zom(vW!6w1>l5Qdck&!hX3)Kn-`yyoOV01S)WRH@yzs%GgjhtKP#P3})t(zY0rDqxb ztT(o~J8rO~Ru2zDhYC8Q#JUQH;qg5T(wh~D?iBq5WV56!75n{bSIm?N36bzdrw48x zU&ZOtxrN0#=Ot!d8yNga;RgZBV9;bm?~FEc-^ikrgC-&Hi?n#=@R*o5dWVxS`Z;B$ zd6Oi20{)#mzt+zMNg>^vhB2t$qzWZqi6H5C7R2fdOl3ny;GdB*=`X9y3N}p9>!g@7 zQaqdECQAZQLh<^}mBjS-^0${b_b$uOfex^)COD7S43Op@6VO+~*L78#XLl80`4lv7 zcU8NDpW<1spL-C|mg4Zwuvns_<6ra0H;)@gp7=d}-O-kCI{C{O`2#J|{ykaFvZSRo z;?XP4&zDo@J&ZP<;UudSrz@{gec+t-`vC$3Yc)LY*n3%#yDNww%ARg-4=d4;7YMUp zzrN_l9|d{4Na$Y&Qrlb=Enz0VXCx_Rw>t{@5g`Tl9z1YNds@(*3b-zdn8(X>`pt>N zC3l*Kq<cq4$v*Jq2wQ{HdaWI~7?$^bVt-pI)5DUI(&4A~Fh6dQ$!_SuV~(btCD5rp zqtX`-zTp@UMDBdBsN=;S`<=y5wy4H+%XmveCu5%-`9j5gvKaNHS{U?r{eosa+0fFw z%e}o_y`NGF4<|@DIN~v!(lX*ne@Byb#U%UUo_z2YO2PMMf7^mwq2?)~x`#8luFc7N zcq3_r$Q{ETbyX*~P=45P+APxA`BU5)A=2Md=CbkcG}L-Tz{agG)BO7sEs0+<YSIrj z!;7q4;zre&gb!$@5R&3+#Ru>4Txw>wR3Fz^%8PH&OziBld*wPI4@K3%{(EMOa^vL2 z8|4eeq*@L^lsD!HW8+*$3_CHq$Mx2T=c~F1P0$ns{@$K<McBSvp1w6|98D2#k9WH3 z`5gn#rm+9d+9Xt8Wry{oCM#CbhANf!7oWEI4R4MMY__(_?k0K^y3s$-FEQ%!skyDk zb&0j3gZ{30Cj}O!S+1(PLX<wD3fHEaSqy!<oDZyvZTWy$@$0E?aqUH@iK8P;;Jtm` zHT~Q$6#u->=d+y7-Zjnm3u&vWQ}>M|CFs>1Q*|dF10T2(Eqc=^kZB6<%R|d@z763V zLgf;-35*!_iruO3UzOn-ab+XzML~~v=4vvpYDEMw9&|>$a;tQwA{w>wW++K980p`z z=%PL~uNZZ2#v&We<shWJzLBEOhu$LOz6*a+e=CnH=sq=<3-)>PE%eBjhVDr4w!WWM zN)vXLct>~mgaFa~xjn|z<1u<hFi}$EL1&%J!~STb&=d2kJP%q@8JAf1!N~eC$<p<! zS?{wJ<Q1d04|&2Q%CJNe)5ed-cXGUo*Ie8=9)joTH-3y6otAm4u*|gEVD)2td)8yj zI@t=V4`h^>k%>L94iLbIGcbpvqlU)2Uy0+BZ&@EHa?wl-QS+xiBEXDmKsGjVa~Sst zCP{IftZWZRTe5LpDHt#@*)f;K=<oR{n4d*i^0bLZ&`4Zf^=V4zWfNr3pX^}pWCU5` zBbzDh3ASI<5)blPir}`Nwi@=<Ke5>TB4aHz0G>IEA!6P*Wo}M|dn@S+Mf`KcLXDvN zpVnFoW~JAPRkL>_I46G!CQ@k@5-9xu*>Wy1)f*e)tfdn0DqoL!UvCYI8gt*RDD~QP zu>-oXK(-A*5Uc;(wPVnxA`KQOMG{Yy?@6$v>|O_R&7eyrD|DvJIti)1bBVVTx!?<p zM9L*_(99j&s<*tnHQ0nwccT;U82M<tu>8hSql~hPtBJQ)jMa2GwtE_^wb>B9W0Tu- zZ`O@-0)+CO$!#ve{Kg&3I6=sOLX+*Q(edQh*yQF_XQQZz3iysq>@H8Iu8id+0bnCc zb>{19NWG5*u-&}QvRR?`UMIBWBl3kJAS{}|@*)k72~|7=9%d<Bxp^YD0Cr8{YVQE% z`zsqNV(GvyB~j@XE6D3Y#%icP8NP+{efdZeOtP_-`pvI9FM~yNeV&Z0+L`&Tl6v}# z3i2mohbVrq5>Ew!0v+FxVLe6e=AS?1Y-45_SIH_2zs}AU*iq~H9#2KiSVKV&4T8B( z-_aM;xQqr673I25wcp~x%)e~<yign${yjZ2)i2R$#gtcw^|ZiS#sk>*h^Qn=r{c}G zB(<~Y^gHX!50uyp!W8z>s*JXxA|nm+U%l72uPIiFT|0UjWsi4EmzzP5E=Su5^Ve?P zzg3HM*9yL4`euFJEDf!%v+kZT&{-uvdHX->rMIVt#&RXAL_^cYHuL?hh<lE~T2>k< zTMbZX-?^|vR$97MSkoh;rW%C8d?_$<X66$x%9p{G)(CqX8!EbZ)8N&X_Gm|yAnKk_ zxi(TwlSp=^o+NtuF|?7Xv2oEQiD4$C=LhP86IlEM+<sEc&&@~omKZvi2nh6EUSjct z`}=t-A;_MA;r4Z_7njry;;K3#F8+pleh`g1Qx0pq9+3_a57~X5;jB_ry_ZJi&2G`S zQ^&`cVF;sxfcq{(M8K~&$zzGy`OG~#H)2s5wO!%A=J5=LC@EYholiUw6u_iEc|>%I z?N{1r%+)9*!l-bvMw!k#EIEB^V8IRq2(cV<wFRH)QVcGTiF$jhX2{0$7*u&{f-P9A z|E#N65kF9RjmB<`W^Q>JoFG`6EK@Fai5ZHesX2EEw(VM|nP|89kw!t$Sh$6C4+nmY zIFqxb+f|vNB}-|xS>9odAN?C;mHen`SD%{PuYPNtC}|`)`*w9mfpIs*J#$mS>l_w5 zy~(M8tik-1qn8m!iyWtGUufw0aSVIb<`q0PjO#YT`4T<n>-mkXFkTHvP`bXQAW~K2 z5s)NSLFi<yi-3T4%(M3K*qzVfxHZm|^y2lboh~~*TD)CsVaEC`UL_?K!Mj+I4JNXo z<Mk%5T4F+brwav$LeM+7v>~>?d*3~G^(Z9djfmzdD(-8%Lx%49Ij3Q{Q{g?XQ?S|E zW+V(!&Np-SZ*4o8m!g-vkpt+uj$#EMgSw3}O;mj{Qfg5wC^MFU$=$D-v9{0OHfTLR zEXPCCs<zEY+rbQfgz~sfD)-OwpYB7nrynZUTEt=8x|pxpMk{KXEgE?Ncit(EBdRq; z#7--SACc8xy;EJ*6;R}4-1wZ3wbFJu#4b=sjsdJd?P&^je|Z8g)iF*f+IVLXJfW`r zsLg+^aA3Y8&Qx6lITdJ5{wNr2|1mLO7Q=@lPPJrARD|@}ZG`zYIwQQYvVOdau4s1+ z<H4QuiA%H*$cZjG*6anhwPHJtLfJU`3MbM7RDOQ^TJLY9Iln+2-?;6>ptc^WtKX%6 zQr3N3nPO`*L2VQ6Xbl!SA62hEomWM^FBadsEQe65r^Mo)&}+i_Qkw9~%oamfLsO;J zie)AX$yt5KHmR4;n>2-Y8p=0ySvmeG4sH&!*ss&bE%Ti*e!)rwmD5dW?uF~#T*Ml5 zL@H{>d%rvfx^?}os>jdUxqG9M^p%u~yA~)th$7JXP+-9=A1CWhCP1z-T!@Na>~(7* zU))HO3I{W3f`1nunLre#nn8`0=Mn=EHWgL5dTZ-=$*z$>`k|Kw4Gqn#QP<JqTvL!T zC>uGxsbDkzploo3&N(mpIB73Hqz5h0xf7XLU+t~~W>D~FWIr6^cyNC0ee)vuF~pT2 zDd>}(9Zb$@i&Puv{=RH6EVH$#3Q~>~h^2Fljm1O+Kn&T^M{~<XiHlL_rc96I+F^Ni z+x6_KRc(qcOLfay9o%JpeBgBj`5ZDKsl2*=@k-oC&%Cu*q;QG8<q59{{dQnxWla!C zm2Us_VzWS+o|j;{`iXzo3G4U(auGJzU*2ZTV?18mb${g-zsUV6fzz!C;ldrM?{{Fb zJZL-nDJh@*RjBPHwmM@giQT+y5}e#e;=$-+U{^t2<iWbc4ANitnq0}Fxr)G3A~w|- zR}v2%0cs=fwcVx3uS8F0>^SqYP$+F^AUQimh2K#Wa9;%Lpf_KGWkQ$Z&f2O*?eStG zoxv$QO3YlYOY}sJg$u^W{UBIi5Jiu!EZhDyf^V3{5m7G(+t4Ind{xQQCf+GDWb1xH zyNnm1+Ups}k&~-Y_Q<%(&)c5uI#`ZVBnkt8IWZ{AK9*Q(vQ6}jo`GW_B1*LTP3*$S zrF*6xwJB=q!}2XS{y3L7O881DhP*Vw_}|H1H*9@xJ!dK$!Vv`p7%>?tbqY+;n;*4h z>gf-JPkVGbNo$W%)dU=tE5g8#i}BG*hS;hkJ-y<S+Fc2X;T4XWl!0Gn5sR*g7;v2S z&8-=JO`iQ8l3)v)xB!gQhCE;wiy7n?rkeu8KPhEGNtI{E`KsNws1R(%V1W%!erd@| z6j%l7lrb^*ZCDCEib40cEi@`h2c~dpn!yso%<-6NwVKCIfY>BJ`b6TlL^G!@yYTHz z>i3NH(0t&3(3=`a!4sC5ovq%VH}Q~~db+(b+?Fk*=l2J1(Sx|F=4_5L>j4o>O%y}Q z`|iO+_FKnGhee$OU7m%ap)ZfoymzI*GMMAWtO~1&q{NLe5cd{1w0A=YgSDfZj@l+0 zx;TgmGvRoJL;ubDnC%PJ%|yJ*R|&KjQPfXih}k(${|&5)w2X{agD5SEzt8y48F)AG z-m)ZW+nx_V1YO2f#$VI1vLih$O;($ZJ{%P-SX)QpfLM5zM@B}r=cRFQH~HyAO+`%u z$yda7?taE2^61_1<2;OTsvPnw*Co;}F&y(=GJ@^J(fYK6YpBaDNw=*lv6yZ!`I2Tl z{b=u0UnK|sJ^Oa%>vN5XN2K@7INuOx^+(ysg3)rOsDx_>wOg(>P}i=x_9ZU^|AEG9 zaeVfKu>Mk|3nO~4^0**cA|6E6j~_oIZET9t(%)MQ{WG3@V9)s?My$*>gZvuKY%_-y zI5Dh9?WU}<lkvdHI_POnQzRJ4=%ymw)4t--jv`ZCehITh8MFG~T7Amlr4#Z1L8P~( z1;(CY5;s?Px-GQjd88%lY5EH%xN9`E9G(kg!8vu<7mz9VZq(-Rb<20?r5_|KAvNQ+ z_U=C{ba3kGkovxV@{K^{P8tqpBOgVP9v6(PSc_%bzJ4}wN+8J0{1mLYu0#?n4G)hZ zr~M{x6KPV@sM7&YY@xDidVTc}^}t)v^wRxn4It~|+Y7KrPNZTIjF4PfR|2chA23q5 znz63*_Sj1c>>s6>jktXC(qwB#GJ5pyA>#wM)R;`YV83T~?))=(e7HlP_=9tJe52bh zUEmS^HslZ1oJ3-L+Y(h#shrVfmouC*JTx<+3$5a-b*g!-Q1wlF051y6eq=TLd!EUF zQ6Vg1rZ5<d-9Eg1SrZ#28BFzf@~%if!DGy(J<f8f2S!_QWni1;S7;mHpW_5HDJ@~) z>UKuDkculW9fmUBMDVYzgdam0d8y+sK@%H_iVk^*;kzF8(XKGywC!FDk6l;*VZ!Jv z5eO7elam`{j=Y+@N~DVpk;zU?AdeodWPIKCD>JRSTG%rYkZ?Q~%aN32CDKJkA@?St zAFkKC<<yW2kJ16d`uc0`R9N`e50C3ep=_h`V=~xNX2v+K4J?X2R*PA@i*cBfot`#h zw~4#T#w+-8qo$xez`rqkhWQOXW(gYyn{?kBg|w0_c4*P|g7Z7iHdxug06o-XKefCw z0P8kV5BOC<bR|d-2u(Aj%KPZt_1<;Me_&O5zBTUbu19yQe5==|mC|)PWx)d$kx?}4 zE@+y=KV@Y_@Vf|*4PHEoc{0gT*T-r9JvK^1PRU=;98s}$;E3aXp&R$VFFXPsCu7*( zI9`Asrzo!9L^hIwk6Bz&Qt~}T@J9>CDUi*Y?M_h#31dvx%j2(rP7p}XQzZEgEYG%7 zkjWd$5z@4f0Pqsizqv!6P$Jx<d??_SW9Z3?(A_i)Jlk*ox5~W4wf@bn0v~FgZNU8d z?f?Dze_9{(KQBb3Sid%eiISX_{;fApA&^9i1M{p(6P=ov`*7Oi*zlR_f~O>LXD$8T za~^sIwEJIyO)@|&6@CY+ydju?28In!Y<Tr=ZUUpraM@*W4p=2UJIA%2RrV?t2^kEV zT4dV$JNJ`+_fC7}-<a>f4Q>WWXAa$H>jO*MG%>zOB-rBH9B>czAp-4jhw<;dVQ<L5 zLQ*pcl+@-x1xynfd)?EYD2D09$z!+|Oh^Y_{(cAV@7di6L;Dp)R#S2pVGBA$7QW&x z{(jPRVu5n<$#xfnk^*>)yaG<Ho(BCr-?Llg-UQB<evmO_?2k82CKDyPHZk=yvzp8g zWx~TCar%RXa#-O8%&VTrF2nrU-=D$w6+u>Bk;T~6PkDzRc#e*mS^usdOb=xt`etNZ z)1)Ipz|900%$rAMMO}Pgp1s}WUs+sL<H;JZK3WFcHw39Ze~+5<r@*}7K7Z+;3MR66 zFrk^Wj6_O8!o@KDl^fLYFCR)tk@%KIU-@UE4gJt_oPh4{1<wBVKTC`J=OfRY|NHkp g<{zrBoMTRfkLzx>tS`Z~aTrWWOkOl!SoiIJ1BY-V=l}o! literal 0 HcmV?d00001 diff --git a/website/www/site/static/images/case-study/linkedin/scheme-5.png b/website/www/site/static/images/case-study/linkedin/scheme-5.png new file mode 100644 index 0000000000000000000000000000000000000000..e28537a18a8b15be3238459ae6474b23aced4ef3 GIT binary patch literal 116720 zcmcG$XH=70*EJkOM1zV)q$na5iiWDFlu&JeARrKW6%>>vy#;~@A|fq_fOMsX-aC;_ z=m^q6Kp^xQdI{m%c+R=cbH;mr@2}_07z|?uSJ>Cy>so8BIp^N~DoV1apckMJ2;`K! z+yhkz1lk0F95p(A1biYG7pMaMp|V$%y+<{O<3)l0oOms#Z4ZIaF&zF-LB7PWK&YC+ z<sZnXyTmLI=wtLWjoEiotnS_Gxr^F;d{&S0LF&qtfwM2~-3YZbsVcnuApFebMyJcD zkja!QR^RTNc*)8kahc;L`f^g1w0D2Rc(1C#HB+DG#;qK+cW-hH7{I0*jwyua@f{3? zSD`B_gqF>lswLf*V&Tr4ox@SF%Ukx6l;(;8=Z@j1vgsIi&#htX&!b?Rhp+3aC3D$- zzJ)-#Qj-qv^6=&LOqJ%(cM$2D>4$fE_)3UWr~309L`4zv*E`5{F2vt&&xOJMe)~b5 z@$a_{|IfbGi^_h~V{35~d+@8~?HIdbKNz%8^+#Js0+f#n(kq97um~CxbDbZtJnYqH z2Di0ytC7kJRu~CUxt4y!==(z8fB$$Ci@oC_;<92ya0E9TDPVFDl6EiP1mq4H207N| zu|;5hblUjb*y&dg$oI2GAE_Y-^P5x&kN*GWjK;cpvN-c&(53VWv2Y)^OQn>qZj+|@ z?Bj<sq<o&0RqFMAVzf<J*3t*a_wQY9x6TAzf>-}yS7)bqav`MOJQwhIqH))tEPZLt z!HERVZiZF5cbd&-pFX_4-3sp{ZvWw>sXgH*l^50P^#M}&bdE()+!%Xya*|X7p|V|9 zGXEF8*Ky&#D$U06c4@uF#>V;VZr-=yuz@NwL%p%doa(ft3slmE$JsuLt_$y0MjU|% zCZMaet9$ip9{v%Zcj9@vffLinFu{G(YQ*kmW9XAH?AWNn#wnWXf1F;0JL|Ir`DEo= z?FCWyv)gGHpROHY%@qCO%P1){NSpo>!{1mSvLF5)E`-<>GBM$1u8b~W!J8V(GxlCa zo<a*W-b;32YB4I(N_0jqbe3Bjg?JbfTeQD*1&rE0Y?3-b*jm%b3_5@n7D%MQAm7mu z>+R6iFGUg77Y7SXJK`_Foy1+=1l+!<k`r5ztyJEXrJz8she}(*P_5Sv3~U!$!B=H# zXS^q#<ZI~r_rH*Imz+j-Oxu6{=#({AJA#2rxwFMRqbiZos?`)h%HutJ>r(S_Exs&I zwPUA+IP{y|a3NIQa+vqEw$4vX+{4u!<3WTZ-F}Dhr|~+wBh=5<slv?zk!S46aygo& zc?^;VOZVRHb&6O1UN4ZE6jnAjOB16zsfEI(iN5cjZI9q&I74O5eQ`#kL$bGoXUt36 z`T0C^=krf(l^o2fOG6t`1{(Q~IJ4$fH4O{B^uNuYzKF^AX@E60G2CEyB)iP-e^g@4 zmapX)-y>lgoHHA3th&r8sP5p%Maglq%<Wu?GUIKt6zRUqD$AcV^ZeCBsO))&IWE3z zsZ?B5yf_qQbb4Hk1BB|Cc^IPWpvHUqRtq{zrzwbyUf;X#d~~w&BBgs!v)8wuMXZzc zo3LA?0P^U&nkB+)_z7>C$#DzO7BhppRy%SQEhh6_F`wo69m<e(a^{GB8y44;*sR=R zW)F81Q0`90GM~|7gzqV*JWjei|MO#D?8gJ~r76B%3+*BOCQfNgLjSl>mThGMt+0$7 z6a5WW;`d}I3v7$vJW4tL`&mxNne5by5D0d-UD5$E58=Za1q_?<{KQHBBpW6-ZS6sq z$&?M&M;epa_q4wkOw(Dn54G-2$GGT^i~76tai3=5UHH&0NGy!>NM5;kkH@gcy|WsG zNYM1#f?MJd1m+LSR|MD4?^E&4kK)6&r*WB^G0)XzZI?5IwZCtlw^)|pjF1*E%yV1& zoKYJgjP$KDyMv6K7(K20-S@02O%+qH@M*n+HtqgP!$+@_sY&CmB|~S-Xz4Ev!>^sZ zGM7y!ZOBu3DciIjjop6(#t(g~oPNa1`p@xeY)5vnE9BD@#aml*SLbKh@RSXV+mlG0 z6EVqqRCPywJlhO6mAqAGxHI)@AaBAJYBhK-$3*J&U7MZvQ4#mRV>HpJO3v68uk+4S zXA|zh7cak;`YNGyq5@U!inF>5*5AR>QruN>;7V2|are}<B6YOqHAI&M&gEM~Ml46u zd$jE5BxtQMXW{P7TLs23`e|~>&SIPWwYA|P<R<l6p|Gm6K4e!1IVA>P)V=j8u!572 z0)1sOIj#1_)roIvR4DyNPNe<lw++9wWjJD34*B5$29j{qB<9LfPDEFmKEsc<MSLbd zD=tcia2q7=J=LVo<-}9l=(uZ>*S60McRk`ax@mrMPu5Pm7PHo06dlCI3XeKUSFZ1G zCMmlxa&jOeKJ@)EyzE!Hx9=C>DEy9}Ye!@~kCmaM`w{mVH5yVfHZdxw(}Z_3`3}x| zT;K656H*E0IoA4EQxiJ6^`>-fJI1-YwnLQPN|i<;9NTS4uHrj?-I`E*?c%IXGNEor zf=m3R$T{l~JH#zSh(sx3{0U<Rrw_Gkr7y^!bST7RG!yI$MX)0`N89~n6ZVwAl>?n( zD}1OGH~AVoq(OiKl@YG8wooz%72VPCJ;=JHb;|L54(oB<<AHklLihi8Xei!D4`FOD z%P!z=0*|wLsexXlgRyuu?7WJvwVOFp*ye3=MP+!fJMZGqWDqNk0$YE54|{c~Tz&a# z;qF~|v&*9?+=L`zXPD46PBt&}X?eXJvK3d|w&;?(Prq(*bZ8Zvq^o97m$}7=B__RF z9Qo2W?lPun>wOnFFisF5(&-=hB-nWVETbAt`(P`CgS&6y(!&ad2a~~NJD+oi%87+; zr*Y}g$dW7T#yY8y=!aP-PbY4e<g%VYRCjm98n@~_<!j;(^`ekl*JdR;);JI|$!&}3 zcEbSys7ubNwBx+uC!u^V4&<0%*V&UVwGt{ML<Rn!2wR*kmXx^OAxt}Ye@IJY$nRL; z?xzRF6D*f5())#JWI8^&s%@M`C^&aIN;gn*%;%n7e3Ef{Yy7bJopH`5x|MYkiMt*8 zn$y!>EjD5yHd?wuT!eRXk5y^3!<k;yESDCyn+|1r5IK>8<HE_WOA6iF23?c+-BqGA zxcQ{G!{d+7Yiwk{gikU=khD`)&n~yk={F^@;}_ElRy4Ed$uY_&Pjjbx$F=yyjZ<sj zZr&9A<h&8Nts%k5IigNpTbY($J0G#~fHP1H$1Yh!8g^Z4!wd$9`PjJRr@J_FGAi9= zWWT6O`-Q{jP6hS)pc4Y>7n!uBrkZJ3=(4#yI(k9;qPR=$QaNP3sFdcTz_sf!!-3yM zNmx2H8l0lNW)_m{*Yx6SJy`Ruz7o2}&Y9BNc=R8ZGhZ{?2qmS@gwsWZ-rd+)bE^8b zy?ES_P5^0YYCp#Gf+-e0$h-JB+~MmNp@{*sl0rqd&F~(SzIm(qv4KfbXWcCYW=Nsg zM<EVOe0gBbtm1_(murhkXTqhpmY9k=^gpd%yc=X0bg-$_%~!a_x%4hvlS`7X57V|p z*n{#gY@NI>o0eg%rLZH9St;R+m?;W#h>shptB4=0+m*`A1!;6VyZajH%5CkOWR+X; zp)nEf@evv(-AgK1%NIjArg89(reXegqPd|6Gple=R?p>~&FdNVlnZZztR9D7u@_Ga z;ic(bB0p*>U|8hRSc8*i2Hg>QL)P#`Cl{P}YkmQ@Fn`9Sp@3ztGs^q*@jf(B_Bg{` zR;i8rP22B8A-(Nfr(k)Fudg&$Z5(!DdM8;QP1*Z;{J0ZaH}@E<bp|ZnHNg2riGI^c z*oXZAtsQ4>!*YU<LZrHkaszu$pXTk0xtQVi*^TNoyMs~H$d?f#7?<j50yA84{36eV z?FQ<C?di#M?<~Aip#__6GJ!FMb3bytpA(U5zJxuX-1j@}TT|!H!2LV52k3qmOlZHb zURw-@yRtQMfD4Ink2#ZbO&CgMjTH?>bgwULehSu^xLfu5^-rs$%d=d*Z)BT38IOg) zJ>gDEyNa%U9F>hh8<zbxnLl3MAfK$S$^rNZhT1fx19fiA7%b$jAmWUeC!70e(n9Ok z6zE9dX8vkRVY$|#T@+_*%!{H+M)FBTu^bKfwvBpTbqJk8z##W{*s)eIKJ_ki8-~ko zKFQ+-yFllfIX2V$cI<;C=(LwEiZSO=PhNXVF_tCW+ex%C>6}4W$im7h55wmo?H3g- zxG03xF3cB=_vO}1{51Uh8xIK{$@y?R)w<+`*|HkVq(|IB*xhL(nbn7Jbt!XC7#$YG z_$}-m#J=cCB%92>D&YkQw~B##dps9`gNgYD&)Q;)G+1BPl&iF0*Z6ie-FH6xoEQG+ zlX9~xK(#`XjolU+J-3}R9v_hkMZ<g}3dEhan{A98)|a-a5_NlC{p{c`x#d1F_S2UN zb$#r;-u7K8=}wM}jrzBFrXBF5<ifs5o|jBvil%+_eg|CC?`pxLk>`-1-*F(0IxlmJ zg$~6SSA}>(w4qbJ-KO&u)i9TZclNm}@oA-YV%HB<GPJ&}?&<mK4^LL7r`K@>-18fE z@Xm5W8#&`{Oq*IzL;V4LTwO;8Q(Lc~dQoLvk9B2#_B+yRP0#{~VIpB7VtWrzb{(e1 zYBW0?TCNc$Pt|A;urLRrnA208d&&Z$^5KqS=mN1tr<ZWs{r=3V+p~Y@PfSIAKU?Hk z*SP&!^CtSvykhOb^JsjLKo-O4v4mKgKcJe%isQ$}=!I>{Fph<WclP?IaoVA!wkK)V zYR&J>E2BSK0JNiH#)1B0D}V6q5{)t=u~W%MU)|+J%4@^$+cS$D5Ds_V*=!$fLb-=* z+9t@SW~s8DWw&y(uS%Y0yn7ZKZLId~o3rMqy8m{&$BXkxGwJx}8g4PL=FW%`Ny{B& z9o)0ClaI^<#8!K!bBFknb(lC1A$ke$lI)3mYo*xs>H5ou>m1(_tbDuaO@cK8wZ24( z+jYqU-;=bYh2t&@3(rIf&{R7wZs4daEA8yJm0n=0mZZOUw3Y<;%Et8dSBMH-{wT~M zmGT&K*;3rIF{>bWV~6_5u(FQhCX2+S11GhmMkYuXJ=!95JkI%t5$Bt?+w^qN#eIS| z)-hpB;-m7-!la!%2cGN*E8~d;UZ=5DsjM|@7Ts|)ltcf$scHC0DRi41GN-BbsWy48 z_WL#0O^$bwF<LlhIW)7uz>fc(5!~5>YsYFy<5NLL<n&vzZ9F$tQmy(3NjJt2jxiDJ zEv17zLnyTpm!&^BXfj<xgv2VfUsCbhnkPSc{JCv$v-h=X)RTw(GL2@An3$x;{3*jx zx=wKIymW%a4Rs{;9b?8rs23;iOAL&HuqzE>jUaP4i-d6T8QGndLcwQEnVjYq3eMxY zB}Mz6vXlihEs(@5<|l0qG0B|NUH261>!(DQJ<bd^XG^oN;rr&Z<Dvn?1(|9t-*j)6 zYdiMmaQHuz<END+eGc?dPFLEziL(C0tDjx==ooLjhhHLzLFN6DT<*-fCP8ogT(;`Y zp6=lllPa(^4dO9lb}Ibb-e#mxcI95Uf9nG~89Pdyke|BDuw$FDBo|&;!{~{aCn;|E zuB8}964Z^Gf~oiWMLa%YoAqlX6<R!%!hp`P{y~|vf3=f(-EcKc+rgkBY4SaL-%Z48 z5~n6dp4W?)s<(EmJROFXK1)GVa!P3|iwqa7y+bxsb*$W)!bk7n`>h>9Z<BWXryYnw zaJPZBQ`hL+?<DT^W*?d4xydy#LXMw~$g|q|0qqot;9U!GkXscHm%81uRfBGwMT^yc z>v6K6=Hh`iVs*mdJfUPOyPO)b+sP+@voOyg7b?ylNhe-yMLItRfYH}%F6`wtvcxP@ zm1d`f$jwsA|5!{E_Xy4Z*bWUF#Xc_XT7ot>t<GB%N0X1|Bo|5F%V=B7zvtx{qPJy8 z%0W8`N=CA`KDq92v2#i{-i4AZc+}9Yurn3f$9&2~w^pW_z`2b}va9xp@z5~}0`;K8 z7cc)RpD!X$&cH^JO2yNS+f$hN^+G)F+ouCdB|e)H)(9SJDeI!JeGN(4-UTa60{PoW z%&a=c^y~7f05vXbsSA1WG$X%uxzXu#@1%)O*!wb?`c1VDdC|^SW}6zThJ~}LGRY4S z{u(D))4ea1S#0WD#dRlfaQ3U3U=mi4w)UGccVFH`*O*o8!8m0bS2HG|vL3#S{K|}l z(HqK6N~%#ow7aFnOHs7a@y!CFXPh$&nD~}LOUj23OWc}QmpF*<vk1341)q=3+{%k+ zLpsBk0-c|(mXMlL36Y{ptDM|7#GK?XZG)g4Ns3@xE^$ib9_Ql_$bEb{VFCw87Kk~w z=-QbJ?U{uYB^I7ysNd@#iq0rq<#uAaflw)pHN`T<-41p(vhx=4*JskW)xq!ZbEj=o z)vbL<VJ#xRu-?5WV^5KKJ-R`Ou@L@8sNTaf^e@9cXAo}emR7w_g{+2Fl*JO2Q{Jqe zGAH9jK?zCUf(aCpKQly^=!GAw5idSe#h22YWX-95;R#RNxENLC@H_)ywY9xnr?OA1 zYuU_TXvj$ud^S*^E_3$1<<=a!lZ@2p9?@}Q-pK8NnlQC5frNhK=GwB@=Pi?oMdmnH zyrtEuD(EUyJwFCWrLZka4^ODeR1{|sKiTSqp=-2ul6`Rd3Xfmm7q}2z*304SOB05S z29Yc!U7)hB`y;{Ha2+2sb?y7raRqVa9qQXR*?Sh3E*YMcu+*6UslHrzP9q_Sy9a@} zyn}aH8MK!E`LX;IE+gZ|6@O=aWA;qq>C-qbECIK_eNpxn5_9(W%=RWA+@#?<_$%R` zoD^oZz~8l1O^QhOHl3<o$yC6RHN-c5EVV?v*op`sE+e}0zOl%v(+PZ~VL9iR3`P4j z$Zlw>)h_ydB_+ax(!;s8;Vm6<999|>-+FAUJtl!efms2@fF1b&uOxmAH61UWc(^zj z6jv4BojLhMZ1d(2-zkHy<h!(NHr`!e9vhTR^5fJ>0{A#PMnVb;Q!hwDc07I{FbNY? z^g2_cj=TphO?UzF>Bo$p!eBF@+duNiw0RT4#<{9AllCW~Oh1bGfl%eC1PEodfdjpu z<%*z6Fk4Oce$i{(JXBBIq;0oWrAh3`x55XlY&$Uz@Ht^Fs~dq%edv_rnnaXHuU4eT z$4+>YnNOJLUOcLakyZ9C<7cNkIbaGaE`m;W)#89j!)E_o=)z4LV5yEkwA6p9(IlR- za%OHGNrIva@-|YSn3#Hj*{}6x#CxfE`po>2?fTD8jg=}hKK^mO0~<md4&_nxdczdF zhY=v5ee$M4VCEfh$KU%E%HLO+G-qee$SASM>pSmH&WcemkR#*Ez)X7}Jg#v%Pg<AM zwbE|;@=W{14ljNsnV3xQm@U^$7HDcW`!SAF?x&eHL_@=Kq@xsnObdBSC@_AimpQwr zu{&HgKerNSuKC#MY1&HPB7DdrX}b90OD<89z9swo0YE)T%((<Qmr#8(zKrOi!!=() z{tU~pwS$SH&ah{y%j~@2QX8RcfXJd)RiDquzjO<-ZeWbL9H&7ayDUPCIHisOtiWh- zg*Xp?Km;sJ>rK+zP#<da)A5-Z0?9_fo6M+C!CVB^g?R4`>>+?aZZ~GfeBRySDeyFK zE7rs$;K7ryVj({(!iLtxI@_lqZZ&;wqpF_{dfX9Cb<)R_hVsF_9LM26&X@!6vz6nM z;-aSLkaINbc(9@N*m-79V{gZD?!jD5kw;8rRB7b#%n^M*XgJ}5pB2B_5UQ^Y+70O7 zJ}Tm@6*FV1+Q|;?dVxq(rdhf-8X<8OC!9xMLaRPE^mSvZF2mt<BoHcgduVjTA=pJP zKctjAOK=6SB?M_K_L)p&@mh_BvuT~`F{4k=(;pv;`qobPMRLUOSM;)m3C(5GNMm^v z8S8uI#{33<@l}tiG^y7VThTb{=@x9}x9es@331`cf!p3m&{bi)ZuK)qmu!_cLBcQb zx7{l*Y<{1a3GK@CV@Bkcj$Txy!7;K1#KPnZT*A?naF7S?+-f|3j8h`U6gP;r2asPv zj?qCc<FYo^+0wGIs0R(#GQ-RB3(sv{Rl1zgut1Bu{W`o(=PXJ2))I(RQfTpKb(t}? z*ZD$c0f<Q<g9mld7IWE?2u72SDaP;MiyaLlDCW`fUz1jCbB(N94>(veRnetwD#_6N zAub#1Th7{`Qad=Yr;~%G!O+a7luo{*e>LV`GxsJu@hS)(Uxk`a5k$G0rRm<uwzNJ| zZ4Cizv|n`zrpeq(Wz%u{dt1wd3SsiM-@R_Fx{UqilQa2hUu{K7#1SDU-_-sRE%eH= zPv4O;{HQL2eyMdQY%_qTD~~(O=g1l-%DIHR$XXY7Ljbq4b!n99%6evbR-!tUwEjoi zcMs(0xuvcUmdcgWy{*+~Y^O~@Ie<Hbh>b`Ki^ekci=vj&em2G7Im4ynm<p6RvZQg( zo1#n|tB6J-CGgAj_dAH5!ib<vTQ%J(eFY6$ddK7R9UXEG$SZ7}nhu%DtLHf3QiIcU zp0qwmwYVV#Te#NMbkpIM<&m|)ZOudvn?=9y*p8TUk4H*joaspR6e!wDfy`^%S;)h< zrKPa?R6aXCJrZ`kY){^RUX^C7V$}A>UDUG>rgZOz5ge%+`nlQqH+B-}H82T;TMzt` zYUF~t_dmmqOT0z9-#sK1%~Fq{eScgKKl&HfS){;tjz65e!~-;vwH5`>0=~fH+%!O& zL_C}@wbH(OdegO`_mM%RE%IUL-%ntW*#R6V5@`W9J3?kcqGLP^z7lpke$Q7yc}lW# zMB4RP1@4_-q;f@u(5$p5--~!Trc2jP)ICRw5|W;$i{N63<>Q>kCIX7Xe9vIg2C_NZ zH7Hl>E`jtHYxtv(^OU*#ZDUHhw`BotRe?>46<3uSHfl|Xi>vb2{h2-GF=4OWs+pO6 zOFm)&9|v+<QRgA?P6+tWsn$MbV)ztSrGSYUS5T^Pth`-9o9>OwLaVyuhyI?Ui(HR` z5(|R53+S~FN2vD0U~+M3`m8bQ9B97{UCd>Cl|`Syg^`}uS}zHEf(!A)T{$NFh#U}P zW-Jy#<`+imJqW*D#F_35BWyi<!pI?!QTtTSRhbKakL8hp^RKQcKF#M=ZY*WtF$}>L z)hT=3L(fv7UkaMS>_uMkBOofwheV~P4%ciVA8O;FY{0H4qGPdMh)0!%bc=wPsuG(U zV0)6C^)cWfzj0h)<o-2(#c+Xj^u@)s`FPD3qEgUz+qF-M^-h%$Pj)`zi!9`$(I{yU z3GIV$Y|=}Gk#F0^0iR&z@?RmGVdL#e>E7hNmObvpPnklW<d&BHST2Oc=h4VB5+lB5 zu6~^=W>KB7Vsa8OS=4x7t|21($-qJ1ZTanIuoU?v`78BCO>DP_Nl+&J9sdn2yYC-@ zMcDeq2dmusYSJEK5=<{5CXLj8eC04)2)st4JL7VDHbY$onYmmnTk9?42}w)S$0S6} zJ91phL#xY#%kP5;tAFA&79SiK7_nuVk8~@!mTd|dv}MDB$dXI<79Gat6AA?6z*o1! zAkP{_?teA8e7WqrJ@Ukhia=uY9(8|bxrQeI(wqUX2=!8si5k?_esXe?==OYkOS*Yu zc&W<2HY~93RP>g>tvnA!sREmJNk0X0w9wmqQ@CwF2-d<_Qy}#j=CXA!RN$2I(0ejn zy30*zR}W04&8JEk919Bc(d~n4cP8x*er6Nmf6C02Zmo(n%G}~K(??^b;E<hQw<b_* znjeO<ns90^eohsJ7Qtc&I$sGUk0s^UI{$DSzTyh<+Hts02BK-tAe0-CTZ+!n+ZmiD z?wyem5e<kv`35Z~Qz7trLv4&y2kN(+XEZ9COgOswM8Pl_I?kts$#xplIVANnEBGNE zS)c=67ekZeH`HZ*c+fu~17e+XB5=w(1$s~ZZ2cH2O!Co>Fn>drlIc%tnrDhx3m7St zAn=JLW7<%LzA-D<MQ1)a%;jp4d;)~t%LhETY58r^c4C~PXeV)*Pl_)DR56#M?%WV9 zFmwk?hD~tNC7mZmv6?s=;4aTH6wljQePP>byCm+1SEoA9_rg#<D7*E;i;K01ccL_S zD?^|CSn`pQfA@1t`O%}h5hpj-HfHC{r9e;TnJp}p9utpw&${b0f7UTmnX>TU>}+$t zsdY)rby0;gaaVg2Uvq03vRpO{tJQFv((B1oJ4GnQkVdX6$8hZ3T^i%av6viGUKL{Z z0?crjhrz(hbi0SvLj5}@e#%K&;*L-deURp=AKEu#Rl<;h*HP#!A|?q*n~8U4FXVN? z7)f#;d-<4=x0pC%=o#gXqvB!Tr^B8ye5bM+=sXs|hYjr-@R{>vf5#*y%G4&7-F^cR zlJi;w=F1J`Jn6M2;cs0Mf8FK?^IgQ5)8EE^7HPc0eSlwH|1mHykO-JLcFBvBr@sXY zc}&82n)p_~aUA<plY1ibA8sOvciMzWzJc4~fT8hU>#8un(_~zV#&YqxI+8Bu{$zfI zkmTpAxXm6yFA<Yza^`OMq7V6d3Rt%n-8=K8@5b1(s?=E=4MWsrZvE`u^(hpKu|M8H zee%SgT1SY1c4Tyxa))_mOiF2H2Ezk7G{Aw?ANc3(e9nqd+X{bUR|U=I&wHI5^R(}n z2Zru7DD<IW4f^9wRpUew=u)10=tok6#XKRqhbXJl7pd2;Hs-huO(dc!78cIo2s=+W z7nPhcPs{ec_Tm%Io{D~JBD6Nl*))y*=FtGw_B;N9wO{IW@oS-|ucrPreWHQ6p(<}& znuqfwea*OaCYF#-R#Ta$;$h7rp_bgj@zw9HcotyVoVeK{xAiSbOv0MDJ%S(naGqm% zaJ{VN0fn0B!n%`3Zmsf5N?ejQJ*@J_LRqcUJz9Z-Te|Gw<ev7i4?*o#<H2KltOn|7 z2#R)XMO5Wi^yz`_;fQiat?RrbKUAI?&6@bl_v$s{AF%3((+ToKn50n=T=4mGi&UC7 z(PeG!71J;qyVYoz__Cf|UG(a;rwxlukAwbb_pPSBC$C+w`9p(M#XuJ_6S%5g<4AkD zNJ)BHEf^&H7rv76dQ9R76_=r_H(I~_p;OJRCV9~+d?iw)vx}-ZSb<Ie&f_vxCh7%n z{OYw%C(ih01F&|UJ~8378&{}|T-4<mZ`*rhA?>9)!D;~+Tw_tPX&<2XVr}uV1lB>y zQg0qi;)hvr*)vDu<z}|KZ#XYs49j4rzh9Qp?qh9?NvKTb8SYq;h>yj0S3d(;@;fX0 zW&vutGmZGxTH8M-F-J8%uz=adToc24x*PQM&oVBwbU?R)+L~lwOCzP9ZjU0w&gOBz z&u+bXEG9L`E#%4P+ldf~jYoaUK$XF=sX|1YoY=e{rv~Rs#0MV@clcgD<U&5NP{Ven zT<oTZpGQTZJR|Q{j*n`SBUPNjH?F_5somQ75>+t5X;>V@>W}qAVDu*|?VF5bD&%Au z7MV3%Zi3p>66lbjB|_unsS*df{uKPMFZ(_cT(aSH!6ax8#cSUc`treVbSF;Hj`i27 zYU#E?_>8H`7%s9{INf`y4f1kgfo&lB9cQQHw7N{eI{99B&pISm9ihlb)cgYE29Ox= zcZv{7XZX~?oV<{b<oUr3E>vVro{%T5e1Iu!YKr$`J?okW23%ZT|B#5Td)xrozj1LK zqvxKBSsz)HuTZP5SrzRSB<9%NxR?Lg07Q3z!*|xZte&jGy1uE;hV(Pe=#Z`Fdq@lQ zZR!5rh|kkeu;vmAdzaK_i8gA<h*NAIYc-0Hl<RRL*>*+uDLs+<7;%jOw<=N(qm$0# z@-w(f)$oo-;lq#oRi<I?o+)qDb6e!(X)`k_p9+9_(?a@|d0Io@{o(8qy9}Rq_q+Tm z;dKtzH4b12iX6<l!OB*|DpQ1S&L6yAzdgMMA_*@nnCO3sHWSTCyAF`c#W)Q2)MrM* z11QAJ5>&|cyVTKL1|@E5wx}0}YY^pBmU|>iMyc|Pb~U$<{r&3D>_&QD_XNVD{C$<x zuS|CkU3N}SDixqg^*)kX6&LuyT#5?uw;M{I@q8xF4SA;xCPfHkHuLpi>f_ye?P=PE zmFhC{#=iT3rPsQpz<l>`r$YtoPddhFOO;iy>J`J84F@J(Nx09G5d0EHSS~L<N=VEs zH`3pEAYEY2^!i0t)lC#YacR7wLnRrCptQ}`aQ56Lr3}S>xS>ex12wbK+t>y6l^err z0F>$2H5yTzrOiJtO|P2SXeK-9(n$?hPGXX%&uq%yC8@9K;gYe80<?jKF6d=m@hePz zFUI4PD->Sg8)%>vw2NY_q_$tki_vAGausK!cOHy*r9f+2XRX=#hV7E-6fs6Cg-cIM ztfs^&Ymq9NtS1gmnRHvrJ|RHy^qpvg`!A*I%Om@WJk^dbQj0!+v<~k|${H0!U*yap z(S1jkT=qcwVx${>y>`)Z^1}+Yv|Ce$7R?$Tk6;feF(kjPZ1TWZ`zEUQ(OMAR<e1%Q zb3WL$nudH&$fy$PI~ymap71joGs2UhIqVyaiq#bs?bwuVvQ~Ea?yX1-S&ue7s<RKl zTQdogk&=A+PKAaanPAqw(}F`*iy4{LV?W!?nqx+XVubI|;ymM~3!ICJ`l9?sN-&8U zq^QrLHc3BGX{*m?YX(!vTP2N4ljBi%e;MKI7CA;~THBZ`@m8kk=y93=h?iuZt*IIl zhroF!u7Jd{;0wU#!;Ah9(Lvp&R=hd`<nnMIAv8Q(UHgNl+~LL1-|A#tuih~Dlw(>W z0ZCx8Rb5#H$Rt>?!<<DNT>SlNC=B9ux%&C59nN~3%*$VWhKAy*G+i=C>eN+}CSay{ z9_-*J4w(@q5&BzzOv!hW?<pO)FaT)!Gj9$(D3LHZrISH{u?yOqdok)%4uW=*DK0eL zwRrr;z(AY0s;eJn{T|*C{kEh75i}Z?v%6m5<dL#sOq(s`F!JFi+8eTtzIhs=p4gy= zPJ((}N^^PkdTYsp<X^K&uE`CBd7U1gzfh3D>bz9$wiV}*mG9b$AYI}5l(j1nFxwn@ z&E0I+zEc7?yu7Fw1bBN}=&bS8*Q|R4JewFv)Gf>T#~^gIEVK|z*E4>EDvh*FrWi^e zS$J4QRvF%oQ8{XsVZlV1>6R?ZdDASmtTJXe7Xz+9<+old<%2pjf*orE90zVYF-Z@J zW4sE8I<LP*3@8z=5f-*8c$r*j5|A<q7~Y;pKayq&209sb2DCsU<sN?b73u%<lO5~K zGiefjR3iK;g1Z#-26NdrAa(y4oN>&hcVpUvz<5QtbL2BG_p#~w7)ZIGnA;XmXbyRQ zk(t8B+s@nm6AA+7r67VR<Kwo3H}`?Rqj_juMJFH*cl=0~Nuvlv#g~b}3+So!M%#%D zZ#lsbsq9yN)%5+k^1C00K-wby9R5+S3xFG>eqm|3u<~&wxO^V?`2-FB`Uc1=j{l2& zuZy8a`L43Wu=qxE<}H%?OVdgJG;v8au1i)iVuO+5`df!F!t$5#ZT;TF7}lB+5%2|1 z*%5j^Z^4?i8k*XQP>zs#--3|(IY8NMM7gf5r}A4Y>;%rTO7)f6AFSUWP1~aZF1wR7 zfBFpFjCT`rgi%|F_9p`RAwRZ%t)}ZoJn@+9yrBHCf4>F%z0bG){!O=M4y&7V+*_4) zZg_=gU4wiTlJ_kAeh*H-({5LOOBVdH);YkmfEO6t`oSy7)v3n&8C=2DyEuV^l`z!q zczQ6_{LMIxK=dv#xo7LZks$B~@DQFjH2QTiD2v~`dGo0=AGn^Wapj|}4oI$b)56MR z)3y4H6n5lbSd+o)U5ieMs;x3Lb`Kf*HRJ@18g{E3x3`?Whd7d}^)9vHs2A|vKu$(p zlMPykwe6HpVgz?fZIjk00+6cf_++4;!GOz5Oe1w?lo-V$6-$vwJ?%M4M0r?GPz!vq z&ox@5?}&HXrj#3AhtS!i=_rDyRF&qt*Wg|u;9vUx^fg|<+*HD=*Szg~DZafQNPA#B z@n$(rjIto)EV<XVF}g-fjT71!n-&&!tn#XvJs?E~3Z>3@W<hv!6JG)A;;7tVfY>Af z9tXrtze=F3SSs`qd8zSjr?vQYW+V0JO^<_;c+pj7Wj43XC}qmTEPZvZ8zzB7iHI-T zn<_WtfzVn0Gq#(a;Cnu!6E93Sgu+}_;1}qT?h|jYJubg&lt&M&%&JLNsWv-7YMQ9g zUUqS8UfJ3ljafEjD{ywU+>jqa_wt{w5_OnSu~>NxBKxnvy?>`MN*Da)?<GYo>m(cZ z)e#%G$C`5-g?;pZ*wk^upIOoKSG<zsO@pmg8lx+x5Wu*(|LD*P%D}+jN6%ip(WhG{ z1vGq+7LliZvY1^;_g*%aT#uQm6R?n6Rcmh=>0{aZneI30Jk?AaNAB$`relh!p3a%` zEQ9bS{b#<w9d^+rr+cG@ku$W>+y-69ZN_$4&-|SMW#J|PtGJ6VB`Jwh^(d>jU7L2b zgHFX02ip?`lWok=f!tgN$mw1iev4}Cb;RqnpN?3)$mf=p<SAY2)|P<*T$z=b!+P_Z zd3rVpC-H-fmAdU4k1670XY0<>kngWSY=Tz;b4LpFl860TXuy|Xy)GoNcIulH!Nt;Q zVqu6HJ1oAl935<9tz>DXghX;ZTJfJsk8StYZ}(Teqr5-*67y@_EIKf(y|JDczbwAS zDiF7u^?XOYCjBkDjiPe@{)e&i6<Z6sn?L$ZC$*mPCVT@vyL`R{;7%l#T+_7v5P$Vx z8ehEF?8!=zh=`zqd{_B<R2Nd0ttB@*rnIgvTL&GyS|egDVvsfyq=Nw$r2U%zsFfs% zw%va*T$M(7<cFoAX}WhojrUd+#}h{6Qr(&&Bd8|q5>A!+C$RcygnW+hh4R{f1-}}y z8cjc&$T=J1oeY7;Oybv-8GR(!51cx1#~_e*A%FUktGYxyl6D4?+ne}b2JJtyx2yv- z4i%aN?KnpFViY>oOBszf1MPJJA1}Z$<)Fgchd&wV$Qxd89Lugi)-^9l?9F9cF1JY` zLM*RfLqXrsUNZcd>{?O50f9*017tVw5F3cN)B4Dc3?sM6Q*}93lH|@W!Ij0NW=!L^ z1G9;|c&niOd-mkf+w2F6Qw-RCQN-)1&M$$ofOyG&R#RUiH7A3)%mdU2PR&620^k+R z%oIoIOPSKu;+ZvH-i&(&_%;rvT$pC(s=ACz;B=N2+kxwtpHtozH3ae&Sek#2!G#O) zdYl_X-A{4SYU6zYOK}IKiJIx#m&laHx1&8SGkX{4B~oq2cLMzk2VICo?M<F3qSc8z z+bDA-a?N0f44@(>*QdYqY$<z^7yxa2nki<trFz9@E!|2p-Mg=0%bSKzKN+g?9AC_~ z(YFJPKl|~Zs~tXS3~UF0-CP=1$Zobt6i1x-7&O90$=#S98{MI;UyeOoit~Rb=v{w! z3cFs9Bw~f0mEO~NqQ*@45>jKLX4zbPWigPJq6Crx!Q~e|4n|MeFQE6_4sVw=9KGn? z)Jr&^kVakG(+yEiCAGR-4tTgITcd3S#kix3#f$Lqe9%My7?i9wfwzdeu*@Kj`8DpW z7rIyysA&zYI5dK;)_A*VP6S6#&46B|45+a|Zn^BBSUmu_^Vj+4|B=;G*vraDMa)Vb z_O26&@sSFcUzR(%iHhv|#cHTkiHQg)6j{^YwcS3QQWf_pCPnfXXPB8%dqFXR%e=&T zG?aHrs*_v)0$5+;fve&#fTF~K?ApV;7{75~ti-72{41nWXy4N1KY3f!j@Uf%r^EHo zt^V%SN1^@Z_I6!tsx97Gd33*Wc0x+5M7<`=(6tY@CE*PI_?d{FgUu+Tn%Pko98P!Z zPD<*uHvR)cAv1R3gY~p+Q)6TBCHN%@+)u|>DrRkzi!p8Wfj!<cTwO*X-(03iSFSBp zBdT^E%Lje$a3^|XNGDt;JQX^1P<)s9Dk9gu|EDU=de(mu8}*Cj9sr9B0h<DWOj&?J zm8CATR=oVmbED#dGvzJL{Q$R4Esz!mi>%&SRw7TMJ`qpoOcE;$LkQ3@5}T&DXt%Ll z`3Eq-=n`boxcCxqyO&tu{}>e>t7Oa&4zipRx#F?LR|34`2)jDpI~D?b2ly4d#$fqk zy8M1#9!cfRN)l-m0y2;S?0g)LXB)73mB%*c#KGQA=iWch#us_R5&@UDAzH)%fR$tF z{_FdOU%n<YMbO=mHD}lX;3c!=R>Axm3!Rn|t%e>?H%k(IN<<8kRn~IN#@T-6Y51ml zuS_)}=C(uM^L$2S2eA*W=;TX{7R?KFFO_GFd%kzJyzXnGU0lK=c@}m7!^8mD4OFKR zG}?UhPvPKJV!VAY2WNFg_V*G-;f@QmDeN;bFcJH@QTPt)L6V^Aih8cm{-99}!$JR? z|HP>IUDmanMW6&74ynR#aoExE7PW-Mh=(qz(%qt^j9^%l&1EC39j#|83GiUcV3`lO zfW4HWQt|lR7xhucL)FJ^&$m;xdbF13l`s{%@|aDfo!VFi9izg?q*Eu_>TY~nb407~ z$8-GUAxY4d(P1Zc|LvPu{fzV5iGc+-m<?Ck__&MPt0QD_p2r3FJa=}QACgC&)9XGT zq-LsBDT;)2*&G5%6EawOUQSSI<<)2m_c|J5M@1!e^~1K1%j^52{rm{Q{dO!oZ=XFF zTVWM<na2cMHs^0MJZRpwHA2e=+@>UT56{nA=}!VJmqeaN$3IlkkTKW%jNScO8?~{0 z6WER-lzrl>)ID?uEA3o92bFL)^MW?F@-YC$(*-+xfHlYsSpB`M<&PQ9StsS5>nH*r z21S49KYJaiYa!2gbY9r<qZG^H$gl`lvkWTz^Z%^UX*RLm|CliU0yI!Hn!t+cf}6XU zv)4xNxQ>q6AT69Qzfh>XsQm*Y@`q{X4u(v_%>h{f)OgK7l}ae!nNAFWU^E@az8>^4 zeT4+BHNe#sfo%>d=;#X^_H$#XiP7wGBSIsOu&*77bCzbw*M4Ae>Rp}F=jo|}iStV< z_}Zp(N_NaTA@Iu_*p|)jfG);)>Qd&wy-CK^qI!l}q5Ut9Z!9=vdAp>DFMM4wC>;5O zC|jKF*OvosSJ1xXuY6s#u`oto^OV*HJaU+JX;4G=VEmixzH%?8cbNa6OMicVyxE^^ z4A|y!2nRt*w9NfA-}z-XdijeOML$curdPkylYC#Q_olKSrB%RXL5jS-&9@haL{<)J zd9kvaqE8HgEgsLGPiJQU-x;i|4{6ao?0T!DoU$;qJwc+r`>2&gj~v8<ma=I#J`BBE zH{2~ZAFi>b#z#o7NjArMOlNlz*>jN%b1Sg?Cd)6eQO%k;u|Vf_$(O30lfc5h-*Mk; zK$SXf^14)8`>JhcbQne~;s9?n(bQ!7CVVks-lnS1RF+ZW2Kz>{<}4^O??(r7NgS~u zqmDpMnQ@ZJ)OOdFzk_C;-=yXJ^q-ss!nKQT5@3l*aKn<TnK&EI>2#?bS?&YEnwTt9 zhmMgv<QlwOZnru>0Lt3+8;4EX;FK68@LF`sv_@D(FZb9XuN}-tAS6Upl!~MHTvTzU z8r7oilu|^M=~sz#U7XIAKd9FrBmfv1Y7cGN<NH)gi0+jdq~5tf23hQh9b~as<5u)Y zOqs>~ipBa09&BjRV_zOcj#db9%QZd;Iv#9joW@zL;Ee~Rh?nZ`)&wN#i3|YJYa-%? zDQFUEzxG4AcWTQnvlIz88tHj8mJm|_CK0&O@gn)N=GMC?%LP+%E}_hIpUhC?u4r(} zJ{~I}L&k$gpI8zy{48N5eWGj(Ig|K|&}YLUui<vQf=By<5r|-0&#MM?354(^+)-q{ zp3E&@_Pam$<GWRXOn$ZO+N1mmG^{8CxC`Yj^?~P&!xnw8Qjc}qqlS2aEgN8#0sff@ zft9We_6Q$z^_~)luN-3xGjwTSK&|GJ4pzoV&dv<H+u_VH<XB-viIsMmbSpq8-KBB= zwR@k$lnsBKhWt>%8AlK()0S=pyK{g<4nmHTVSvCN9-njQL>N3V$*>m1flx`Vui*Em zJ7>@{dN!vnUav1+ag2E+{_5R$^%_u6?(f&_X<a&=1~gb@_jS-piNbBs55&fuyB6s| z?{Vxe2$INw5cI%rDG|2E(^(BR)iIYpu^QS=m1}-PttU@lhbF($9FS7EmDRtH`6Cp& zE&;&JFFb082T<_t6q=Uwd=LjC{q~NyTq+@)bQ@Rz!eWD3t$fsdfFvsg6d5_Tw6%b` z?eY{?w15~t&BvMOX9~qmOs)cx3D{``2#sC9#o)futtm5d&?{&p&cv(BtnT<Tzj$wB z^z<y03S#61+OOXc+>q{#-F5FV6gQir6NuWaRR~kBA)b=D<ysf4L>}a6yelwo-R&J* zS#VS0PUnzQOoElD(}Ih5&PC;VuGSOkH+Edd!E9z(vU9P_sMj<^=;b-?1%ZHF<>ota zlq2{eo5b1Wl83<T0JXsvf8$UT>;yxp=2*5oNCMhu|A+TI^d#=$akFE9g5v(o3rGtP z)o6NzQKRe3aZ-*e4V>xTI9Akp+lG{zT~li#g%E&AfqL%HFrZxZ3kD;Vt$0$PU&=V= z3(6L)mlH!2uK<ZL2k>*^hr!2eXx9Lyk5jF8KL&O@=c=VRKPUy$bw?ND3f&H^3s;k& z_uAL41S6O;>D)z-C(&CLxK!x)emvpm7=WOI^rMBq0S)?b!Rvo+DxC2ZL3EWaR;1dH z-^|X(OL?r&u&L3QwVuimFItxE;Md8m*2(=vZ4)JF?<QH46$g{!>MoewA)Y=yhJt@~ zS%n)ss=h;e(UVoVksi2E4NL5B3+_0!Gw7d}ncTCyzcmGMhkECl;0bHbIzak8cE$sM z|Ea-Wt+?%39buXG_wUM2QbAbseuGL@R=Dl>140p4N-dT;p+;92Vr9}yxO%%Pkh(Lr zni9jWN+Vo->k~J2rKRa$k6u%!@YiZYT=lfYTKXbraB#eK$cdexak^@=sbJx0_5yWa zw(gnBbAgJx1|ND)iQ=nxJ)mUlsGjl&hsj+U-96qQWxGesG`s@#H2zt-?Wbd%mtt*+ z&7mE^pf{Sa0YzLi8n(T*Y@>I8x}ClE2Dp%PCVQ*dMR#Zg*be#yc<Gs9hPUFQ1ItE_ zCW2*%qB-1;*w-0mDy<ndj}xQy4c5khn}=>&zTB_x9_&Q({<e<0Bzo8CmtRiz2B|NA z{j<;0T|UGa)CZl78>5nKP5*$2f~}XpCz1p7QwU`3j=D+Ate>@&!Pfo2PARkObZ_?P z-SILtRNk|>zWB(>#drx@f<@EF)@r^{4a!5=#Lcba_qu5`1e*nP>gI`^rd2o0zU;YK ziz9%@PW}RHD@<3ew;i<v5uHQ<guJ@H?YZoPT+rTuc9P>nIJuq-r2jsf0WFeAJWi5m zjBVHuD08G~YyTf({Jq=hFSyI7N+S?oPLfSI9Sh%>=1HkuXHYAtww~i*Rw9skdJGMP zJhp~rv4F~wu!*bQdeXC%{c_k@d!nBG3Aq!sHvo?x1qf-g!X`fMON_l$MiTUJ1E?tl z9+m#&#UwC*ubX%4*=>)_%w#Iq2;1z)$zQx8F6n=ZH&0zgwPfX4*#2j6qn|tTEvYMT z;5#X)SRvUyA|56Y8<0)JM(zRCd*I&cAuymknS~)jl=B>bONqS_wwEQkcmJ&97{OB~ zq5MC^k-BjkG-X<kL94-#4E4;l{S`ASQ0cPU$R1ZcTRDr}_Mg@Dd^IN^@MU+7HMP3f z`qjFX+QIyynlsjFZmC3Rf1~rs0m>$d5*;5)iFWMSB4?zk(u^W4Cb~0NC*q^G3wV?( z_hXtyb}Gl!Yo>&imE)r9t-?+Wf#N!DxAQI(QNZnmZ%^W)8_E$!5kP8vBpZAYWZF+) zzrzra4#56SrM?~Il(afS0<zUAYXz9B;N!E4oKZ0IxGHl`jzMbUcpou8w|D#TmE_p6 zaO!g4_z1SCdNP(%%lc_CibLisrrV8uZ43<xbYFo_pijKtk6jjyr~XKZ*3mxVK9oJ2 zBkRXaL<kagh(ax;&hdA4wGr~ZnobBR-~ajSh%^>3<j2~^>il%y`bArpr~gBRy-=m; zw77_TI`NMEC6d&XN`KJ1p2d8JvU;k=P#eClDJA6C)Tu<+=qm{I&cRlRBf3N=tbwFX ziEx+;UNrGzRfbW6x9<~1^&9=?p0u&Uohk)UAp)dI3SfGW4wr$1Ny}RWR+O4?3qQrj zjyrHI0ai7(IKr7L>~z;%WdZyA__61P@8{K$JFdzRlh*OwHxVk&rHGUF8Ld&@pRxP{ ze7U%|mSX<+ErkCS%Vugcv<6)w+vvtb%{vq_cM7)94m87iqY+8a7aF{BZ)r@}-SF$g zF#GJo#y>!y>>1d1;S`Nw;g!p#83kyZv!MUbByZ4ZGz$_3{)QfPf=N=spa&o?ojl%m z7P^04@Qvu@u51_+I)m*7^&waJZhLjVw5OVH_`-Jal+;U-#~HB{Xv^09{8bsQ_!qB1 z!m9iG(L1brGrmzp&^Y(aP1=4btZkh_3jO}Tdi=rmdW3<n$Na+{L)VGke$SL=7^B?* z0pUhTI1l2mv+Bz_(<n*FVeL@^V+Y1O3|${kg7QJdg9-JuN6AmtTGjxb|8NJzYo_8A zWw00aL*Lsl?Kz@N1lWF>Oz@=JOEm55yn^V8iBb_jW`k&aC1C`hmE@^2BQ+~GSy)+t zuwh%*co<a2=nQpS3Us1jy(FtA=u`6N5`ghWzhTHjPG_9)H2<jJ4<}}7h+yR3bty`{ zhW?J^K}@5gRF}xg@a%kNR5fL%ht^Qg@%118PSGI2q*l9(^cs?TaM7q&bH)A29*c%b zgd(T8=JHu=M>2FtU&!OYoz=wMEEziD)KO;>@5tkmmmxJ^uGqi-^>FisAuu`tI^^D= zoPdMUMuAP_Rsg)~1`9;l7!2j_GdVRpTlTZ`^E-HlnvS0b8%edFL)isOQgy%gha&{Y zCLQ>NQub$zb+mwhzy3F5^1%FRP&0JF^)ef{-PL_v*UD|<(lBUevb`LvII{9BOt{)^ zHDLM)8-<jv>uCoD{sd+9bG_eCYGWgyV|cYu_Mj)Y2>@6PUKg0wN^O{-g!&fGyTmgB zb-8n7M@LOsh?no*=%NkCov<8nU2B)g^aDzr3A+pF{(QXi!O-*wcFOP4wq~aUsZTHW zS8S)d<%E>f{y?20_8AM5=Nq2$+z22Zy=4CLO%jOLb-{7vyMFBZ$^vdH%pE&@ijkN3 z&EC=!X!5#jFHVc^HnB&3LFO-Mo>UkH*t5BLidXfpSutq}>es&kNdCCLi9~(OuXma; zgIjgV3&_E5*qvIND$XZo9gZ_qt}!w+QBsy7=oI?E%!75$w>9l?<<T$;^;Rji9)0Tk z^M8sC4eB*fRRU5w-Oq!ow|qz4Ec^szF7TQIo@DWW7-~`cL|LhC-_96UzRjjtZ|tp6 zwvFbQ%LYdk1ORI-i#WGZOz8n-Z$2An7~gAI;%+LrVvJVp1zeGP0ZMwL$^ssS%Jf%} zMjE+Bgh58SOO#&%G3Vh=w=8@)EkGO_Cm8`N=@KSy{gm>5Sa9bN*Xfb-C8M71f1YlT z16cZ`bZ_cu#YNOel$0w4M=y0y;~Xbc!cmjCnldf9jh7N84cvFcs%W(=U69BJYiJv1 z1hg+l%@2u!>foe>b=H5Krf}GOeg@>wCav<a+tPN5y;7xnQ0v)qpW<D+@3XUtZp^%G zfAnpM8LBV}R^j(i2sCxi|DpW<Q4bU))+5X)_%-&5Ra*AS1;6Sdr#H1ng(h{eySq)+ zyYXSEac6Eov@riP!YmUYl#Onj$K31)+ZrZi#e0yveNXhotz{{6c&zM4NNh6q$GIXi zyCbvrHmGvn|8r$zogaP4F?wNwgW5k6Bfit#P3t-PbF_RRTsd5!$L@gBss&z)LM`ZG zSLST=DQfiO#xH)JTdRJ~)|d8b@1$05w5RtltR98b8U1rdQIV0Q_K&t@|N7#w3loaU zv6Q|lhOJRGl_2Nqsms`{wz^TVwShY2I5*@RPN=6x?UXPa!opQ|Se)WPar*1^zjCm~ z$+$hUUVWf|c=;OteS8G~r{BHG);i!@9i{u5`1wBtlJgS~X-2?ZriK2C{`&XFA3-<k zm0bU){(3|@@PAN9|5Ii>`PUTxcDx>PpZ`9o1IRP7FBFxO`u@+OI{v!%|CIwszI#Q* z#8ghlxi3{LSC{huF*FzQ?}314V{ha<kczVZ#H`Q%o+2Q00^N-@{l9(uXV9Mg*HIxK zNr1D@{lxOGIRsX<;S1u(omRvD`t`1>IPPUIGWVr2GdsKdKVitfl-$22<A3E?m1Av0 zB2i*%`0yMR-`^tm|8Yplu-kl=yV*raWa)ny%Ri#wzlQiX;{5l#g3a)d@2b!KHJhTy z?bkrw+gmL8OUeA#2`wwb&R}3b`6m)h35U9=7Ub`Mn!EbDKRuj`|CM8BSOmku!a)2f zD*o3qVZi;nEdO`){A6TR=zkH_PyUsPZvK7b%(vta7=RR`U<U`_X@Eh2kRMU~=;^20 z{1=PKV;1?~bc%9p{#ub~BYohbiQaUziHV6|@zv%NUUBa0-LiF8X4YqW9NAA(84d77 z0coEXc6i8*`7eO*{xn$>b@szrAmM-}Vx6(9(Zufl?E3|TttCtazrFKRTa+KWXX(<7 zhh7DjT#0`S9)YBF{{W%+{aFvlFoDP52+%UgzKTyke4^q5+FsfotGM%4XIcf5{Z>7L za<7En@sKZ(k!gD~$02nVT}>j8F4n`u&<X*20j=;)_bM~Lo^%+joV@0U3ij)+uYW*L z^Mf>8LXk=3!J)r{c0E?+3FE>@LC)rR?T6Y&CIM?dN4ZNM(UYp&FY)rY3K*l1Bl``o z&Hxk&y!`^xa;20HrJ684Kk*41QezHE8o_Yp%4+kq;9dz6@cUza?AEt0II{EpdC9f^ zkGVGwhx%>*#s^ullc6F+BzwueE0OFBvPYI|qY&AbL<yChtjRLg$Zjk}l6?=^!`Roc z56?Apf9iAJpZj^f$MYP&<M_S*;ov>*_j#Syxu37|8oW5pLUq41Zqq?f$#?O|3<?Y+ z_Vsg|@6E0Zk2xXXwVyIHcdYQk)^k1iyVs^cR)xK<ac9SE#3>Mx7#IK>X}WWK>)uzH z3}SGOM()Y27Sp>6^M|SwAx(@Pxyu!DzU_dGumASkUMZ3sjyt)%m_$Ze7d0AOZP@l2 zHTGG)QY^3(j&Hv2iGN4vm0H`}7iC~BOnf+|<)q8Nc)H&{Ib%qj#-dIWKZyLokoeT< z#Ala*_u?)c)W4qgtYth59B>r#>kuP(4FYahAvSyc-#t$rk)kl4#C9qRu{^~FA&sfO z08A3!gOfvxUZg2PHx);UpF&`LUe%|3C$5zPD~4h_IYLQBBNFEh)DxX!Vo%1nT)B{6 zxzL?hX>}S0f!KFD4f$;0h?RspCoaM-jROyK7O~UvvTrvL9?_I32rP&Aoe7+Q+Dm4D zdUd5Dl;g!Uw`_kL*k265@(9JMiT`h;VE>#Fj=diYqp<$w0r>L?M_*{m0!g?-r2c!; zg0%q=*wMWI=ZF9QcDqRR5T-|m7(6lWtlu0qBya^BMy3M}1Go?1uxYkGfSi0Jpl}^= z(H(J_Q~lrXFgP@57Bmv)^W_Krkd(wT0iwWuF=9=GQ95z7{*%56x*BkbQH@Veko2KK za`~3iKD>G@gRg_=sC*_~8q=G3<n|M}LP3#*Oj0t5s;uq#oVI*d&KEH+Wej{RsM9Ne z=cqmiJY95Fwqmt#Y#$SD?AB#qQ&8GX@qT$_v+(z#u(xkgf!QPV(ao7V(&#zqd}{Wu zBtARw12rGEVp4hyjurB^G=lqOm|k<9U0g>v@f{?pFsb_E4tP5p?F)PD(D}?s7VKKV zJ;4ejFkeY{t5G~~kNv981^JjCvk}s#rCS-ZN$+Lvi%W9<<^YxiubT){-xgcr{V%o! ziBS|g$gv|-0sulz!}-87ChCiehz7hz)9`QnH*$I(O~}J5*1jZIj(r^`8*#XFaQiJ! z`PNWJP+R|DcjRcyKXZwosNuCgJ;lgo&e05VV^Du*!_C$o|AyWli{4g0Cw2)NuK))b z3NjCHF|)$>-Pmf@>PwRM4+7~hi*j}wWcRf07S`oaKFb5PX}P_jp<zkj6TraSaXndS z{?t!!s#)3g1BEb#?40wEPV)e9u<gd(SkI77OpseW|AS;oQMRdy(GcvF_gc)@MrpE- zYZH`c(p1*%g>IsU>=K{O++D!Ek;DK(ic<Es4wZ8mziq`d^Em}#e-L61Xp%ut$iEpN zJRNVVo}hP7{GnUK#xBc&tSeqVk4*x&xhTE$ipXU>%>%dLdyc=K@74~gGtQaEQswUR zRG9_89<x0*wr>NG;rW{ukZL|@vmJ|-Kjn=ct1cV)jNS4<cY-Gqx@`6Sg@cQ)!D&Pf zmiX?@sT|W?e?b~+OfU<haL<i_(Ty6N-QLWC{2kqa3*{I0b5>X6^X`QQCcZ*`)aUG+ z`fM(xN$K4^B?bY`^H1F))dU|qDi<6(@~A!PdK`|W;Vhqq<5X&b)Zv=i(>Eg{<8v<L zLj)v6R;Q1$Fl=1NE5HX;75f|=;s6q^37)Om9)&X=cIRPV90?$MImP!I_n$mhap6l6 zUq0Iwrr0;z1=Jy@xE7Hlf0n3rD`~9VSf2-qn*N}X4uSlg9Oa7Dw6gmjeBXF5-Or)< z;fBg6!yHsinP4XQke6F%4V~JOONZsY@38Rb0xW<PID2lCbBWOj)a@3pg8rQDz5~-6 z&2868aUdKI$QV!L#wPB(-EMb`1E!+vP-(rU)?s=m+<P&?^~XcqP^f_!X!P9p-Z7i! z=2evAvn08)kb^nMH`zlE0MNbWVIw`S_W<AFUen>~nG$U<i+WWe8}~Js6(?W+!wsH| zUB@yoj09S}5Horv@0R*22_orMookBM2m?FNhbm0Qbuon?Y6^o-f)=br_-jto-Rvn3 z2|jZ$x+y;ZNF<fo3QFdFvJL9`O8~9U<9X15MC;BYHkkMF#tuz)>6QYRhKyY=B(FQc z@36App7SD=1JqnPrsgXt3QLVLaLO*gro_H^j{^kAYFKiHLi+G|zj=Q3I%bJcV10Dk zwYt6@7=-xuXMp(Plc1^7iT6r-*CZKll_--fgNeeQOWq^jglD~rwt#2}>_<>|TYD_b zYU1}zv*Kf!G?9kyGO(Eo3UmFLX)$HZU=WG)20{jGCAdu06``cJp7bOn$IsMmIVJNw z(Tn;i`#2x~m<H{l>_y|#ByML;W&79gxOj#KY4B(TJbX4@=Dm-%J-V;$K7Plf5{(sR zP|z<SE}B!$_J0Ao8=t(}c{gjP!n*9F%f2Z{Z#BEydax)#tHD01_vutD*OA?Z8PqPR zM5Pl`-;shbgL5yz#V-Sl!Mpn;q>*E9##_|5#`L3tFzp!>d^;+XwEVEbd4Ct|K;TZb zk)+6Bj=Y!JKH``S$YeSR&z=57Bi^^!Nbpi1DrcH0im%M(YvN-?hW0LMtL*ve*9sNO z7>PS*`%=cc$kKaEU%Em8rwk6Sg7I6tcB8{RH;dqP8}0lXbWo|Q;;>i4y}2<xYXY50 zm$W@c-_XI{aB>WKS82<ZiC>Z^&uQwIxFRN{QREc;_%Bu#Vp(~jo=rzvz=Ab@E-E2b zaxhpmpEXvu$n$r_bbIJi?C&snnWY*xf~L$w8E6L@Z25v-Lt)$Fi2zU`Wswk($Uu*S zsu7*k&g31Tr$K>5Kyf}D@q)R|g{=#NCV8Mqy|m&@p`=z*-V55LfSrR?2lW*8d_Il~ zh&^>sr0f98N%mMM`1oz#dp{sAXk<SWZYgbt`Kgs$vu9F0G1;H&xkzDUK_iF%>2ncT z4cM4M%;SiLiTahjMw9TB^OQmcZaD6?#dq_Tk=qFqeT^N|N3ezZL7`z-O37<CIY|kR z+x7x$4Ue1Ux7Jh$-%Jx1VFs1x!9%6vU?SC7>yor9qx}ICOv|3b{%^GFpU0`m9q|4X z{ywDJtQEAKf9dL~!OFgAb=zuLvp9Vwv3_#j&rp)1cDVHD?!?-zn%r*FA&@P>e7&X% z--o=n9`N}DeCjZOFHC$kQ8iZmuDixcBAvD-DX3)pY4UD0YpUXLkX}~@bm`@~bpEc7 z_~?-7T^cF30X}5rh<Q=DjGJR~MiD)mp~uoJ-8`&S0E<ns8uGfN@8`=&Aq6^}MT3jy zRzL{~)a;Hi1IYsz7Qz6%8FXNJ(P#(B7h9-vHA*6k`RARj*|o(L<d}$f^Lc5zj$|n( zfk2ae^fuIivo`-QQ_9OlA~8AgK-)_w=74X^?cpPPFCBe>?c3-g;!8RBAHjHs;jN2& zVI&`C!o`<Ie<Htn*MK88r_L?pzO6by2h~QZy3o~pv<r{o3j(7wib9NPt(SYYcHjI^ znQCZl^{Q%WN`86?M%j@avlw)#TVv<uZduh7dF!?7aR-UqVl}dZmpMJJOuF-mdxXEV zDmB?rifMbQU*r>09Tj5rnmho2&#@PP`<e19MadII%L^-E;{`gYb=a)&S?@mgR=!d- zM(&*IbXi>sDKPav3r-No>%%XC;)7g_3Vtb!=4lfc**ML%-we>`Q@Jvw)2kboLL3KB zuZPC=MkDbt{@pq&v-m5RFTzt#`Umn1Jx2WKha}y|!K7-n-H!djy*0xJS)Xg&J~ys$ zKe*OKuh!aeu+`SQ>G<fOkC1}L^7=qb1JzL50B=j)4@$MOL1fSN8ep7*4=$%E%T%4) zQ?4euIYFzW^l0p<TeO&6DQv=SzZN~r#8Ge?w66|$o9OzRq~Ve-MMoRku8Pq<L~4nc z`|L3O__0T~Z7!a^f2kO~q*NqS_uMd@S;Blh{dP3l?&F^(Zc<gRB$<;xfBhuP*8HqI zvNj>){?#zQ-4o&BFDGY|aNAj(;PcsS0q#ol;ba3SaG1T-O;<bC{Pmu1Oj|(JUrkfw zSInrHYzzs1sr3R8q4CCMJmG-%@<zdvC3@PQw{v24!RReL6GP}VoCzk74xhb%IU1~@ zUlI8xUcY_xb;N0VbSlxSV)nkwOi~qhOcYtH(^+%zhq&K~l>Pm7Ebw+(U*7++wMft` zOdwZivG}uHP*JGrTs3`j0@<@Ka&$j$55d=htD>Pd37Vrn_A7TsKR8!na{b|06X$-{ zJH&i<Pu}ujpB4u5b_}NiW`x$xHm;oCUYOmj?qnDmqL_caKv&Re$>&i}wmeEFhEymi zo_HSGqLZs}1U>4}z0B#7YbNx%cRaykbPH2KLdM1KynOIFKfG-*JHFdPzVXAbN#oXZ z6rJ*AcX0}2CVOXvrocdz#W2OD%(~^ko@J`9dx0=h!I^#27O!Tw3GZM|w20G8ZoFFI zR~f;&{k||={To(UI)=3lGjEUunK`cVcE(P&t%LLpJso!4XA+QtJN@!`iDgutIp#WY zsJn5`pTOT`*Xlb9T%(2IpL<8}NjP*UX<pZ+7-dG%2Pww$+2G50e0mBU*LAhmqYn|s zL$R!DLuJF2y}I-{5W%&{o9<&Vv)<NC>2%jJJ9K~(!!>rVXe&--^hE-OE4JiX&0K&5 zp~qIu`%!UZ!4{$J-L}E~D#Cqvj-8;~bfvd=eUi^M?T-)41-T=!Qf5y;KtP3IJ~hSJ ziy;VH;ZITk^EGSIeFkK%LDHU>wvvvTtjXz_0q6Sqw-=x^Rq8FVYOs0~3V%gjFaq@+ zb;YZXfK$J8Wz*Z@vD|nL=c<c`I;+MYYw3^PKE<Wz`ENgeSlqrO7#pXGM|gzV?l0iD z&RCtqeqCkxPGG**mm{9lm#@{x$c<BK-!3KWevlztkNfn%O(5o=vzx#u_*EPxab28F zYJ9=6PFWx*|Fx)p0GZRn`Y801iAH;OGpF2B8{uZ`Ltgd*9wMC<I8MCRBUNC2aWujv zOk%F117BMgzmXQ3LpG{*x>VNDFvrF3xgRbO+lx7=5s@31@{TY#*DVHr622c$FkZLK z<)pR|8}M@HkamWE^w2_}*yAe^Z9%NKPim-+MwAF)hUS&I1ndkF_Ue13n#r{pOQo5` z>B!2hAhT+=F%zREuX(0bG@?AzH)Wsmt0iAT!M@q<i_Fd807<gmk&>H7I@|Y5c{O<C z58yo1QR4bsvOdFTRM)$1`zv9JHl4*sb@g(r_&Z;4ztU*IDC4;V&n(q{z-uiVzAo;p z6KLGgstP9@hO=k7#QE<=2Sp1#)zPQXd<63wTttyMWYoC~JAc~^t0a7p_w;LP0V3s9 z5TgT8d(Xig{O#)z73QbPekLIWMxe0^w(Zj2@q%_^FcCrY{(R3I90d}YBzFQ8;2dK= zTFK6eCCedY-CB_r?LyjeZ0&w8eUFz6f%HZ$FE4-i@Igt!?j~9THh!!6s6NOz>}c6X zps5hwI9DHIgrSXaf4KWXP{^s=r<t|vmL_VaU|K(9R~w3QAdqNMd#j^GFy^3|;jPoN z!y@9e(KvK&y^anls?PPLn)!Cwdl8mwS*_F|T$4)OS}7}-tYi6J>*A?GeKY<|c);#{ zYU;*?#1;F&${RdG1KKSJf7Dix4)ngI)iRy3Rf&CfbKKVg1Ywj8tk7bZTv8Tw_RJ_f zy8N`zH%(ZI(0CFOo?W7&6`v@xb1P;QqqJm~>O$WI!w*CdrsWW~mUK!=#s)H{29nQ6 zv~@9j-24FA!Q~aGP5|E&O>NR($su097l=O}Ma)AY^XBKH3TH7R)M<G&H8m5Hv9Dhh zg%T3qzKwL?Rws5{c?f}IvFFD^D23;RBqEwg$;lD)PA(qCsCHyo*+4~mCe=-tAN_~z zCioN4!%UuAyqzCB8{;!#j1MOk;0U*hUBMQ2Tt1z=!~C)a457DvehE-KzZY9uu3jFW zP^dtos_i*rLhX;sqgEAkbGWz&Vp?^WIuA>BeG01))BY`c6LsY`;$r&=aU)Bf&as0Q zg2OxpuCby+!L-pxgba_Q9{eF3^P=;nOKKifJiSohmKf!T>Dbow;o_;Ne$#+5hv9oY z1O7mplJX9tYconKY;00%BjHI+VTq&;!{aD=D1x(7l7qWEF(G+_hsuJ`#(f_ep3li2 zQ^dyq_y&6g4F3r+34{jj=*aI(1cj%6$<DqqM}TWFzAOn0P+tS|48)8#riF}xA}E^9 z+0}L9Y^RGhw47=T?%me1R{P^XKPu+?X7`WJY<kb`!qly4@QRE5#;=E}v|%uKH5hO% zqJL6%4Z^pExHYzEf}_qemwP07GB}-pWNo^u?Pa<5hmQ!{jVg3k>c%%f^!0_)y&X2~ zz&4g(?Ry9`kF&FwY^JvCnT4lH<@Db0YNP5oWPKX~cg&PfxOhM3;P&#|y(=>u+_A-o zx&wA(0j#FBU~efpPrNsS<N{9pI{-z$*y2AZx=##)#tt{F^8_M@jlX{sbZ#VpQGC~h zK7AbbY8>G&5r_|Y(KRqYr>~|yCjeEqSRS2%K-5(Xuj14}2klDB%7on%3LG}xCp_!M z>J+!hhnap#f)yWY0Sx92;{*4(X&?Z5hlE#FLX$loX=AjIZrhR+!>zJBUp>q<BpGf7 z#jW#2)|Sn6m=3-WoVmX{heK6u5+{G!hAM=%fSbnMK^w~Ln1>h-;j3BYmBzz6q>hL^ zbDQiI|3<mxq*krSQ4Mi-lWLNvQ>i-YT>P$X!n_|?5-M(A6P<e7r0uPU*^g5KA&a(p zn*_qeJE<EKGmEV5%&R38ospK3L88RI71IM!#kYBeypwDuO-G&?zXKmX#U(n!FrKo} z=}|9@tkBH~H76I+(1+E(HMhD{g<w{@hXWze($cEu6INIAO@Fk_2-@K`lP^v~9Ga&e zjf{-c7<jl8cUTjJr6KA<cZM!(d#yKXBwRlNEE)6}&&*`OtfGbA*^=}sJM^(^GVo04 z=Z<xz9|h-CZ3n6fh`HA%jN2W_Z2p+1E4aSTqgOh41#R~E3_Y4lB_Y9HUmjMEJAfcw z?}=fdBW!~LX7q>-T+@Nx%sdF#TJ$ndgIi{^TEe;zxZ&}67MD+1?%ybjwOSHL*t}nJ z;JP{7h6qe`nhiMP)|(+9J{vx|-UWr!cyqT(zwpMSzCXN2Wp@j<8fz|$GRmTc;m=5* zOUb;t?nhgr+Ol*dzL$0*h<lSY`Pf9h+{_>VWYEw;f@wULUn#vOAS8T0jIaIb-L?n> zqJ2(R8RFPZXW}_QKd3&kuBA85f*^i-TJlmog^5pb=k+m<!}ln6T&;#q0vjqP-khgd zRkpW<JO=k!5lqX5J1KD7;!MBNtpKO?PHwGLV%%2w&21lt9!_8_MD$?e93V7`wijpA zAEJzx<%=}#<9o&~KAX#VR?O9Sc8axVQUB(ab|HdUpen*QxVaT=q&~i|G9x?^AUYFF zkYFIb*`Z-Q*~zzTO9fW{26rr3ua<GKHnmfttpIK7UjDT6zE1^$xOES*QRY79+ABFe zOxD_`TZ{fC(prkITMhKn$-i@C=fOKtw#}s|oy1}mpeu6tcZl779D5w!D!G--%#4im z;lN3C*tjxEmNK$4i6GwCeUSV7cOV6}$xqgneVlB+?Z-$I*`QrrDr9cn^NX)Y8b4$k z<6O~?Wu*fP&R1Z`GT1R~*-9K-e0pe)ysGH;0@%;cFk`*z`|`!ibqhJB^@|T_#l*%o z`gpTMueUFM_C@&PYeTi~pf~)*;=F*R<QaY(d%73c)`Z(C0<$)juz>7s(H!Dy#rqw) zJv%k`3TX4fRLm#iej+Q%=cZiQ4NQRWlzc-Z)<pG*&WYA~9?c}hEfdo!Urs{jK6}=l zz*Wx7y_@IrUZ{N6z{G#-AWHwKO;kbs7V_pPE{L{xpFyu(o4ph=_%2f6se{3oSbV}A zB#<q#8kkPc0$LN^t95HpR|tk*LuPiUlowgSGBehaTcJTFP!*|OwU##!>W!adT-9ty z<+Nv(L+n|?PPMO$8s^HgxqJ)Q<x{%^QtC%PWkF{aT&h*x7_sgB{C9}D_nkNBvzR<< zuiYW;g|i}XmTRo$_L)4-YUrhKQTX49^XcU?&fce9y<dEW^&MTSJMQ}um;N0hGPU-X zwvOpJnHKx>p$Q4YAU1c+7DYW@Sk}Mx$m1-&h4ZGI!t6x<$i6m%^g}Tw6uaLq4o5}! z-<WOR|59kepJ)u@i;tVa_sh-XyHRbi5JqSG!zK<xX7n-sdHka$uByJ=YU8~=uFBpz zhA+ly4<65~$WJ4PogeAi)0S*XdW>n_6H4i{X;0eFF$w6SC35gd%p$ih93@gK82LGj zTceGv2t#0O0<!Gs#ggPXoU{YwQfD47e$NgTK5YI>Had3Q%=MbUf{jw2YKc7)(29eN zwpxS?oD%)miHgLb|H;Q&xxr=ieN~=Ns3jlC%;u<2xbfOcd6-{83<<m)!B#^@cL{lH z+<XW3_V!xOe9BDr<{V);X0kftywF(n!pg!zgag$J8JORldyjQjk#xrKPcRDt1+@;s zmVSc|{NnSxM;LBQEPF)jnDIpC=}mEQug;-F7Cr8mYu#xuC>nfj@r9E+Hs6S!gv9@R zf@LLF3c?fzhTo@&sUD9Lz|`B7@%fJ_=pK>I2)bDIW+0f)>?C>dTcQ=E#R+rGTS}Qw z(c1Zly2ZN}o%f$^O)z}~4j-j*L0{8UNThu5#?h_BpwW=g(yj~8``o}zWo#@#83kuM z=U^>p%eQk!%(?mRI|<^fV1D}64aPKvRDGp#Jwc2#mP;Y<Z*pGxt#@IRA0%P9d7)I4 z*MX$Dd0GV|B{9y?9@jm5AFZSH_(eoGMp(WB%l={Ku%+P1l_NFxu+iF|R4a|!W3_$T zb>Th-(U@H44#N4RQ7+f9jn&J^3}|D|irtcLdGeUas2O!4eV?OQ%NpsOXw$s9wKJJs zZ1~0?#7;+IMmp!b-s1t6mx4&qzo{7zH`i)mZC{^A!t{Mj$=3RtL~NPP8N#O6@I&fa z-5$zTF6q7RPBkCP;=38-N8iq*2EpHZbBW=~iFsWse_owW!hMHW0&z9<^!_+!kdPja zYF<1%B(SvJd-xf#eY*Y_2*d3LD_Pyi>%BhPrVc$DmG-*rTym@a0IA;(($4ifjRtQ) zz${^Yzk)*6avCN<FGp!UXZ7leS$iil$@jb>9I0T7bbO{TRBiX@SVoOS`=oCWkdWMr z$B}sje<goQ@AzwKn4jdE)H;kFW@=^|@?IX{&`ZT~5BSBoHIh-Eqt8_<Pj=?4jH}Td z6jIy>R~$huqZ@_uYb%JmbMvPJbEIpgYLp6E02l1s%*@TTieI4V<7s(}A8hcVX$3!E z3|i%5@t7zr6`6WU<Sq$Xi*=(P&Zcf}F^ObNe*X|5)U)tBL@A#xK3w0u!LwJhdS>ur z9<}zcvlcwV<&KeslT=sEVwXZ%dxi(R>`B340^%s%g*wV|Os)fnr2LWxTMtRm5y{X| z7uuxXyOHPl%7HA)<=ZCr{Ptv6x=A6L?j9abGLF?mW)i^ZT&4<`aMODp%+b~p!|<0I zN4B~9`XDQ{M-x7~gBMq3+Y(H8AM2>_As3O`R?_14qiM9Q7x7q1FM25t5YKU6G*e=F zK1#RjC-O>6Ej|z*i9~9)LDap>&CNG`j=yh4CUj>qgb!8HE2<3@Ik(OC<R%@hHCrUE z=Tg|dr$8K|-h{y<m!h_4didPr{o=vR>*xSygBPrcY&VtXSg(rQ$77aX)fN>J%4lE3 z!^7M3B=4me`%c*1hJ$Du@nC>J5YM^TA(x(uCHENl8Gsy7aM@%%n31?XhoF2vDJUfI zxqHanM48*NZO=BZ8Zlrx2P-bOci)w3h?Jtv=9KYNOy<MsrF9v)TANb$oiI!Gcg)Cq z+fxnCjs2Vtpqs&Uthkq(TittSF99r}VFuY!TXRdrt|rEh!seLt*qC#3r8Ej6w}Z>O z6<A>S!4z{FY`#Ng-r{~;YU&9;kWDy{l=yPc-}=;fLa2Dn<0h{$pIyN*4gxH{TiOcp z+z+b2<9~*)l#eB7VjMyAS>UxLkIvbaR@@y;Gd`FSUp`}UL5>s4ta6Vd<`XWxM&LFC zZSGRbkb#J9CR1FjSl<4RIPY{s+=vklYuXnE82;_xs+GHyM|EmyQ8$s;^l>sr<|~#n zZ?!q6jZAwfuo<#*H04#7^aXRiunXc>Mrg-ru}e9BO++M%iHSkdUAt1$m-Khy^-99h z6(4L(cbhB?g80oD>;i%cYIkpUXe5~73J!vGX{gD#0fC{L1%a{SCN9|AQxJph**g%y z22W3Z2?_2I2AOA7$AQ=x5Qr%<ZnO}P`cpb-Loe}17wGkDcDL+P9BkGeEWYBsrY>E5 zBs<?E?cKt~Zd#4^iFD4q(psx0-9Y0ykVnY(H&Q0JS<rz36X7bc*6ByxJh%-a?%{Y4 znW%@LSN(KN^4$0(CM#_y<*5l{nBTi}fI~37v*&p_Md$uY9GY{7$GYKrzPG369TPd2 z4!<o-h_3;Yp*#h{hq3%cd5M{+sf@IUh{sm0Gx+u`aLJUQ(dfA#JW%KXtdPaNKo%w& zr3j!4b<x<}$rE|9eu?6wCX{8t%F4PoAp?gOI6J9R!thVYf~=Aa2qvVE&8y~=I#>7H zM{mcvW`HgP1*G$lZI<U4yH5YZLs84i{S@TvmoL9hI0-QjJTWbR3=rA>7czDNWh=2q zOaPJzItbs&z;I+KK?L?3Y&*p6HdY^qlYUiSk|;#{5V-!#3^}06QatBenVG48VFuaX zAFCSA#u<P3JFB?exPfz79q{X^ISk+ZU__gf?SFp%va!)0`%%7MLC(P!25QrGj$rH3 zCrMXMA&80mfxl7uAKd1ci(rmY5*-?d`}@im|Bzj62_{EeEcZopV5^OECT_wM*!??E zf<L(N-#(L~tP<fM-}MJItG$p;hGT92hSheKHP5=^ZToj&1NR+YtH1vLZ?Xcodi?M4 zUEecVoXkZ22!GDk?bNuwJXmGauh%-+?7Cmf{x{+D&jh15;PRctQ%ynmG7}+)5jVx| z5J5`QJ7OR<_-bUw|Azbzw?B(Bi~0}Uk`O%k^dG!?)qKCQs>;LN9ZaC54{I{~{ceeK zR)5uLUSV189|6M$DKkySC+eGKY2Ir}K!-)<XX>jyri0pFDUFq<r=gez5yVOBGS|P> zTQ~K)p7G`dSeRoG0_4EA)^2jkUSjWuTfdRY2$@>$voJH06Vv_UvlGj+Vky-kv;Z1e z4{9NAUI<j5d7H~=Z7a=@6wDUfa8qIS9`^k}`0@=aSQ5ET`BoSq-C@Tz;h$viat?9P zV2dlSr@92)9m><e^7&*wDS+^m?(UHh9w6;M${$WmHimT1$w7i2&2_WQMF9KxNel0v zmLMbA-N%{~#uj|)snZ20$LG(VvmeY80v;?4mDqNly8#K7Q@{saE|b8wY3|tJJ;ORY zDA>Aj8<<~rwO6Lwt!U=AVIa}E&D2z4{pgXV*aITS#G{J0o1)Y*_7i_d5<@<x`s1b1 z%Skb8L<;XL_EqOt%O6ia5}oV42-)#M_vv<9DL@2+{*WHWDum1&N}}3&O=;~EkE{rd z3klQWUY5e%_dFNf0%N4}+#6?dDu_d++uCG_?Lc}*2*if!5AhoJg!yH@+gPn6Qwdpq zSMAG8{7g)y_g;6uH-AFnE2;YrTl#Wz`Aljgq@;KgHBYIB^foIb`h$IVH1&mSuIHlf z`Nls){ZjhzOekcW_U{(zSk@u`mwyX8$H=JqUK1EhEZ@PSQk;Q=OZ))EBh!F?@fui3 zNK=+L;YxSfGE4d}F3_#+wO+`oBk(M~WT5_dEZ4x4Ko~yv0gJ_aZpV<&?e9VdzD;iY zY(91@t+{SitnTv%@Q{;_zMV)se!*`c0@4bJ3<H}Ugx&LB_V3shK*nPyChob+rYy5y z-8C>1GIMS=5Vvv=Vbl>|-6!XX+5UfG3iz<^|1CrR_-xiIdSM5!nLsJcJUu1wE`<$! z=v($e&%@?!<Vm<mjRS*@r4xb+mH@64>U^WB$7_@Fb07KC$p9erbQaHv2|Ql*MDKBU z2yRpt1G~+*4646arH;n>zl~~{609t<!i|4DSR|vWYPjlSmI2W}?5o{RaPKBB^Hfej z%^RshrKV$33aPtlh&_kmjbDe0Z;(0QF1_K^tC=~v1xlA6<J=y|H;RxVzSD4S6c}}g zBdIR-TORL^J6#x`&;wv&!2Z?;Slg9DqKMUqP<r8XiUv)`XRgPR5ux<H`pz9!*D_~= z|5!p7Dhr~HwLf0m(f-F~8{%sYtf{-Zdv9AnrHoP;kSfUgDd24<KnV{^7L`3Pbs~u* z4ZdQO>EVr<C$0qC1->uz#J$K=c<xTebnhC<jPC4|Z+_4t{!h{1pqsnTOG($=WOn5r z_QNn;phCt_ktyW;uWi6So#@0n`96Bx{uB4%#lI-7`?k93$Qd=iysRUwcuX(kjqiyu z`6e$GLDCTTn*=VPZ=?BdmEIpK69jU*?nKY0SrVJ^C*}6^xbcs@lOJ@;68laLzU5-< z&&UvGcDi7aIr9%RC;mt0dxD={Ts->-R*+grw(t6W!PdQn%FS5(|KZl1vkVN%?=|O5 zwSSTLeoBhuFEtKk|IfEGOI0Tj%7d`yL?`#T>puVM1@Zs%f^j#HM=BCX4d5{Tid}AA zpHu$DVv*0{Afn=0Y?JYHlmJ%b5?=BGgrW)jh9&s-^NuY7cJ0hk&L&paVD3cI%R6xY zvP@~WtseQ6zmlT8Q-3VGrh&+F!Qpm7!#zC_s+oQNG7kTzk6^j_Eo7e?`}6;X``ADw zr5d~i{8Z=djx%KD{Vk1p_u$x2#CLsC3XFe1WGWQ@lV<Su|8EH}v#^PG<^pfw=lfxW za~_*$d>?=Doai5DoisN!trI^>h^-DE8~KOH_<xz!n-Tve@5CMgluX&$-^9=UkGUO8 z5vK6(Hsg=yg4iUBv+&a>X#-&sE8y3))e7=eN3b_oSM<k*9~&Snb0^;c8}}dYzcd(1 zzsq%Ix23kjr1Dj*h%)6VjisIS6?l?=zBikf5I04B>_5wT(V(4s{N721z;OX+Q;^5o zhRW!c@bGW5G^hQUwGAs@`3~X!^T)9}d90|H27i`&Qom*<`b1a3_g*bb2LC^0br6&= z+rMO3La}c5r@}41k9Iyc1T$)VWKPl(JHUG850CEd{<tVhdDB|i_>Gi2i(f`Y1_1#< z_uS_9@3E!=$JU96@fq9Hv2O&7TP(I-`-bo1T=rtGk?Z~Nn}XVr*vhKt`^7Qu6V9a7 z`i$>9@8K*OJbmmDAP?)kXnpTj?nQa_zhWk0VaYdoEv3Qo02E2kg!{nFW?P=fs~Ev- zz_C%TQ~I$r6W0D2M6bOM4;tn;g_)&b`0w7)RY$vNU{t8`7Tz%$ZH~*^E=bMY!fM6` z$}*r@9N~B9eri2CB}++lRVu#r;f+1R=Ql;4tf%h|8Y2p2j&0*t91*o&CT+4*SXkmi zm$G(@*IECm%c=@=WDS{!I=h#9*lwJ&ljX+}rt3$_QTra)SMa_V_M(ME;#|Pyv^Nc( zn+Fv*>as`dHrjzQ|FMiCd^X#dPRwbP-csA(qJ>Fx<#ImTw9!+8W9k9KqZ9x;a^CpA z?jpfPwlP#$DnngQ^lGU)USpF%Uz-<w*q1a-kF{JT0dg=y>QaRJ&Bo15uNq6gfq}8x z!yG#J@{_9WmLZ?<6#Y)nz-xaiYA|QPb|Z=cS#;t*3aN9D74J3*W8#c!Q*T>kw+wI8 z9=<`If#I*EA1RZmIrQ=g**k0*V`I!hd+=fzhK6}yI=)S{)W)dMQlFA-Jr`rPxx{_x zncF=-XZEsE6dWl93X4!9bx=k_l;$>M%{<iQ{l!SL65)d3PE?|dxERGka?5bG_rkJQ zGz`B*SHhx3>YDq-gv3I_7x9nM(?{`@yx)0Od9jA*vqtQNxFsmsw~mKQ=iz3{f<c`M zL)A(spe8m(Z^R0xsJ!Vm`I<5i%cmCQ&$%pHn`vo*oNz4!(v^)E9J%m;Mi2AI+j$st z(aoaFxF3n;;;aOO4aw@>`qvH@3tNRwQ0(m!rinvQD@WA;PGjus%(#WoLFyjqgkR|V z-kU4u>Zk5HKrPv}N2aoetVGo9vmAa!;S2hC4-`Rd{X+zF?xmHuBqx1ti+d{AF`x*? zh0Bgs^-fkl&@Zi}vb*P9Sq51_8}&T#8hg|?+!t?2^75wRLBj(l@hqmz$DJD1_>z^^ zK6CG6)gm|b$oj|M`uX9km@+UhL^@!NwGv(OKDP|*w#ajqB=$>0{)ME34;!8RO^^BR z|ExW6J2b=#)QJY!qdp*H2FR*cC;P|Yeev@afvUbK^9&ECPig_CY*HS@%ep>Y&gZa* z7B49>Z;TN#&BxZ;eDs~ilyJn48m@?QmJ|MWVV(Wn>Q-ZI)^H!;@7{gsozIcGgdWTQ z7cf|(g()4)l`oep^34}mjnY9?9|Eq_Y{%o1)!!;DT@Lw9F2;SNIm!a+_8A$pD^XwC z+PYTcuY2)h;Up(v{aQr_?)``15gd~b<oub5S{x~Lm30-N$9rde<QGu0*#Nq9_Y<_g zfQ}MxmptCwX7hytFhI1O2k;OZhB~2HpC)u3|K7rdixke5`j>YCT<~C7oWa4s(CkEi zA3Lh>i64Z5PGL<uo-=FT-rhG{XC{b~4m2y8hSP#b#~fwji{JB}-iDo1Za{dC&ti+S z3#}3EP#q7w0BzOs#CixwE8%fuAeR=Lc^c>&_h(tds~1=_^_UZz^HMUC2h%r}M=3zB zk9jy9^iyMPb)!pDyPNdcPJ&IH$ztJ!#d5}LDZt2fzICrjN8qAqp)EQR?>E*)xj6Us zr3Y5f7K8M{!a1jJ-QvqSimzQ~TWl+W^ex&wW|1Bl`2<onfnpmQa{KhMNs#sZDDg<P zc|Mo~dU4KTW47t;Q@L#I%c6D4M;<(_YB5A8d+H&=8yqA2#bl8*ttS6{5hI$IPq=f+ z4)`M#t2lg(jx)-&hZR|}l%Q9aEb7$n?JoGxxd~#C%jxFa=QNGrj4d9NSDDFw!DtQr zbmI1&H;;Oz=O`p+7njxcz@y1`igwtvp>D*}bArXrDfW+}k$GMjT#N!Z@GYD|uFkWG z#kyT%(v(AA=S;kkC<?PVt=%5I<E8qlrh6K-N4jBj_gw&vQQ3s84=Uw!YO4x3#poG( zF`4`M0`aHg5L%A(%*-b7vsG19ze0k>pHd{0O_HjA?o6WA^zowJJ2Sd>+`0P0pq9n_ z=lBgmS&_$!Ki`E*W=_so2|X!rcCN`h%-*2kt1d55f8yu95~llj3|I`wHPA5Nxb9Vm z7;la9hIes}{@8eoV!`$m6J&T4RSGQgJxo@0)H`HV#*r4NA5Soohj@fHtUQ4Fp$&}S z8d-zU5o|KkVMX62PCOxX1hV2o_TAa9=nzZ)AG%`dfe8QcQ&#(ga&$J{52bg!yozV) zJYxpGn)1jw+qNP6P1}tJuEwrhqi51cqZ%S*8VPFfA4W>OJk_s{r0$ZVdDAityl!*E z<-bB5vq#DJ!BdQvH5z)J0?!sb!JHq<i?BR451P)SBm10300_j#Awl#BKmlF4?^&Y` zhK2)3VzGdjRHx#a>=o##Q<=Rzgjy^02>+KFYvWLvovhEgwYESREwutt_j%Oqd-&!x z@miu2U_|pss+M(o$^jUPGKb?Nk>R>JnBao;EjZ(ysI%=AYMhQR?fn{=K+?w-&qftL zcpIF6rji<ZeV@PZfxR<K7FE^yg>F741-QP{HvU~D;%dE8v(V*xpM*Q)@k<4O*1a;l z3XFvBBWkrXz}M&zm|5D`*x1?GIXV`Wm6c^?W>!`0(>23VPc|F1((lUX&L<!)%*1f2 zUb%0~pf*jNaU>>bn{B6$x{Z;9_9>YY<sO&O8s(6XiDbh;4smaR=aM$hK!R=M*|f2Q zM_{2d*TeJORRfbAfsSG1R_60OB?gly%Op@`@b?F?AaxLOkl}EA?Xr-Nkl+HKSH;>( z#uIe=>Dtn8B9g#4N11l#MXhY_WO06_<L&D7qNlN`wcM)eEU&QWmrl@cy|Ms@-!j3+ z77W!%CLNRUAr|bv?0LiI-rnB2IvKIV{9D(gg*3CZ*@QBHQ)Ss)CFI=Po(pv@_n~}h z>Qx8ummP0KbL227CKOvv;}(O+=SW>oQ4f<S8EcrI&Apq$&T0h4jlS`EZN<eKl1@7F zP(Ei_834V4Z=M?YWZtBTh)y8$-g}%(W|{4`K9f`iy1x9gCiS9GA~l?C67nEj1XXrZ zHgb<*#zSVY)_FfS9q4rj2l7NKWw6%JQoBYiBPKNPkU~Llj$RA@3|stOoo|+xd%3D! zMgv4jWa9FN6s$AM?L=UMqSLfN*tknz?fS8smeHS?iD|dZiIrJcUe51iS7A&P5suh4 zC|&9leKfXN1HetB?q9wx<gAj}q+~FhUk}GMZEw?Mqtu7Jy1Kqy^9jFLq^@;4p<v<g zG%>O(my_RwB0oF4Am!p3=R;3e{F4{297u>kqg9Dv%Q7)1jC_NBYb~4e>v=qeSb4m7 zQ9sd}a%g}KfN#nVFMhb}-@ZkGdfvuJ6d!t(s+St(vFS!zh3kWjv{GmrCa?jIvV@8v z<*9q;UPWY2xZ_f=GJo$u;$xecn6$XXZI_Fthu^l_4qs}7rLZJQS@ihIT{)5i2)m(- zCgxR`T%;k)G(RN%$)<j63HhiDb+gX%shsXst*6cg*mi?X!?tCy6xKI@Shhy5EcHq( z_<EUD+di<cF-_NjH=??o8Lyp6OFts85uq_okWC>Y+O*4y%b>KjYwYNWH6C%E;(&tW zOu_jCeSZt$5MsM~fOzJ$hLvJS%E2#-fOERSkT)$U1?Wvd0sQ{`O9^eNs;WxDr4rAA zf^xJpU#Xb^6ltn(2&C1I!1FyGfC$b<$xRc@FaT`O!dyv0>jZ^=A6}rbE~9(-J=qkG z068Q+SQ>xTSYTEd@wHnj0S?9PY~O`%m}&oE6ec~2MGs^jR~~sX3B-+f@9%F~`ukv> zu{*gZiUZ`FI@M4THG)|GCEr4V$s|R7%3U0F3<7uLji`bc5!9&p16L=BTp@NjGNF{U zz#6Qd`DG0d4gh2qrGy`H8nkEj`g(gKo$*=<;@R+bFX=#06BN}itImNiZM*LX;4l{l z_e+!P`4@ADH#L{DJWNqP+FFr&@YLyOLs*I}o#zc6nxd&Fow4)TDw&oj`qM1uI9(%Q z<iDyOYskC0kl1&4v7ppkso>>Ac?x4TVsz)0gVl6+vd9v>T768Ru+whpgB7rp)`7M! z9C>*H6_3YUw{uKjDf^c>@(|3@9tr!FX5^kJG5I<^>2)W1!R`$#!1U!`7C^hJw^!T5 z#Kc<C@7=C6J&gCt*UKLebpr-6sr(z8?_p3<vP7TF4{-&DW58RZJFwVBleMw}PW~#* z)#&agC}ijP;8qC)@#dG)?ZuRRq{Tsan5~~sTJ24l9Lirty2zt=kV1CV;4pr$-gmj9 z&Qqfi#IKgOSoph#Yo20W=QC~?EeKy16cQ8`DnXrA%HmWX8X6O{${EVA%!$8$cT7DI zAPmvbOYp2X6sX&OM{9Jv1xTnqZG#SvsulP|h-ni+@T<gs)qp_#In?9rPs)>BdzN5W zZKkTCg5KQh)s~bsPD^J_E=S-NO93-u=<xuCZ<$GCP>QKzIG^$qfjbTSbX*{=`(b^h z-|jXn<<kOH{hRpIiW>Pow(>M=fOS%4(|bLV(AI@RY|+0V95E;WGk2#vRm@WVQa(gC ziQgOgb}4Nz@!ZY}s1j(M1(D&TymOcckOm++?OH^g?itGS^L*4s2Ay`K+mw|V^WlBg zxkLV6-Klm+NlD!Z-~E+;gOyPH8Gl!}HG-ZdC<o2Vj%srXV&NV6->9#EO#?uaR?ih_ zNB*;w(hz|H6FHclsDvzkfr&oM&*7P{gkL^HF;i_fgS!GT4$_oB9YsCOVlV3lpFL+N ztzaTQzJ;CSw=Wh)a}~JsXcQ+BTAgb|i2#_$qlVfqdP%%^@H9Fn_$ktzO;4{Agsa=b zZ*qt;hMh!2W}jUaenv|Mmc|SReOsy-T5@P*{I8g?Yz=AOS6(r=aGfR1_>|O)g5%$c zv6o)B#e>xQ(=~>lKcE77nkk3b#*q%X7Pq5lexA{VrFiW4xG!aNo-VZ;PSv8Qq&UKg zd)cD2W9N1>K=)m{#pcI&D^)!P;;xfwtDl9C2}e+NJ+q#98^E>v@sAv&5seg0NEGp& zbO!AnsMve^ceO_wtVWXaM)S4uNDasK4d4nVjpRWljF^dIW!`b8E6m30%*mF+FkHQR zSC?hNP?`}qmhp+>Vy`02FUKPCwSp@HK8}a(z`|~C!J<uX%Szpa;djLyj$aoWF$Vh( zXE;`AAC#nrJVcpyK)?N1NTNRcIq|G>X3k=6J(%Z9tBSk7_&j6hG?k@*+<-MUy1Ql4 z5(QvZLYH-<K}_*U*vV+Q5CCw^>M8_0?SYpe_HYVCp(GZK<jxBV2dSEp3s5Z{LHg>~ z0OsNqDL7h6f`R0B)>h%txVW$&Kz}fonXFWnHF^II@p?0YlRrpW#3@7CIUb9!3}B8_ z2d+-(R5dwX?|p;PWquP<kXD@YLeuue2G7VB1P{)A0*P#ITHpM_7diyzYF_TPaUV6> z7-<N<&?cHyqAmdNqA3;rfnpktx`N?Vj~*_NniL;bvkY1L?X5TSSuP|7-MGwu+&+Uj zFiKQ@Evu?BIIi~mtJHLp>X*_6FjSiDtuG*^yLk{IYj6BLiva$ty+&OPlBA$3Y4vjv z?oCw-yp!;DwSC5Hn*?UaN+=KzT&&pzj1p=_J~ucHO`*!90plNe+20xMvGys%K%mE* zEmn}SUQtg0{~Ur?{i-$;)X?&&RLYt}KxEaXXzCjP$}Pau)uY-;)$zf^hO7Q0;YAQT zDGGu5^WSyxQKNPnh$i}(oclVSGwp!0kCdx{ZAn4+JMX2dy3w5%tSh!!Q5LKN@TB%g z5J;R(pItHB@pZSbMpZMi*XhE>>3!=AD#Hh3J5PT!oiq`wTR36{{v)A|+oTij<J4g= za~)&~f_eriI-F0~{zfuD_zM+q0IONDJervXz<^|GcD)6B`qhuFs~-arQaC`ZQ<3?X zn5LLvYg(B<v^k+0=YxwBGkYe1*E$7*DvCN~)B^DEJ_UpJ(FG9qSX0g`4m8=gLsNTl zcF!7Jc}n3!DOLs|IDMfyyXxJoV0o0wrn{@+rPp|lNhckH+(O|2Z2wVy^XWT4O1s@- zxym@;K7BVpRrr<6uWVb!VyC7AKqfdw(rnJ2Jxj@V<trGzkXEBRD~S`u?n2s5_{;Te z&BR|5b5C{It^sz1w-RnK@`<p|2bMeB+1u~^${ky0PBHwqCx4N8<)9j6SnCZ?|Jy&C zBf#J(sJUBSRj^Y0$$|Hf-R*4j%M$|!YPR43kVv}iyAW!{UqM!uuQI#2cSBy_2MROL zsR&7UT-{wc6*Uqrwe^<wirYkEUUw)$^mmU0tQZ?l%RIr>DE)p#JATA~t$Q2x$Dj_I zFwh$i#-_MVH|IPSbl-~h@Qq}}v;J<D{39V8Wrn;~-ZgEO#X#;wLGkW>n6sB%G>rr( zaYfPu|Gn6cUB_|d{`93^?mTjMd_3jQ(PI;(F*no80$6a(s(lmWC~yGk+y*ff*xK&4 zY@@%g<indb%L~ZsfUz>58Pa)N-8u=I%dfQSDm(22Ox)8gE9`lCrb5Q<cQ5F;Os!yI zlI+S)PD*;x@cGqE`CVdBe_IfX1>;Enm;wDZ+CD5ze|DR&oLoz#_=3fW{Lx_fBe{N9 zV`h2n+=#?8=~w^I8)B9c!J6e?{kn^P<)SGfCRYgw2%Lx8lO(xD-kxXsS7S&1ZO3Zo zKh@ZeJyBMl^^cE(Az^rg4fHq4smNb!KsSImRoWlJda^bCY8f=_l2U~J>f`|Y+=^+s zz*VpvnJ4{%+gSJTy5a1cxYxnENV-9%<UrNp@$luXw6(^zNiy~yM1+mIFS3U-MB=;e zK>m{@9VP%o4JI%+j}ci3`Qh#v1E^3CiFey)Uj$YO{_iDvtlebiD4O*yUYvl1%*W*D zl{&j@dAbQ1sr3}v*f+poAMI>;KjwDT^tfL$$&xSdnb`%4ytSuFIOg_KJt#?2g;s21 z%IJ!u%0OzES!tbBUO`R%=zA`LYIPD~p=Su(G?XKDFG1|F7qRs7XS%bKP2_*Jzh>cG zlTo8c6@VQDD4K9!Oc&@&pIsJG62>B#97%sEU2Lwk*R%^}@x9lIn*SMR29cD@D3R5$ zl`;i#m)Pc=%1T1wDE60IiCY%SH{PN923XIhOztRTkGF=qzJGa(vCz;Joxdj|DSOjA z9R%{FNM*r*9KWK;2z!R#{iVCTphv6s{4=&zWCf@)cJ}IJ?R)v=F7OgL56<`}q~&2! zp734nBL$SzX*>iL#rGPB^x3%p8=Yo9?Zds^vF38&qSLmA<<Ttxwe~<VVxCj9?ak)p z5c@Zh)gb&kZ$+!aI{OBx44n&m{143)&<22@e#s397=Av%w-;)q4?(@|S3lEA0|2A2 zDXtR$Xvc}aYX7X+#t~-$^Q(v#(&$o2T_6C?oY2R13q~mhL>l5t^qVH$)@5Q(K^&{_ z3w^ai==pH}mA)}Fywb>6onylu6c*X$2Nd>HnO2nI`cBu7Q+UGgQ(f>H!9N1$^@*V^ z(fK+~Uz7G1x2VU~zL9w5aKm6JB;o)r$)~R4Srm%*mjsH0h++Ga55Fg5b7pzNWPuoh zIw<IG=)11i+DgZ^scPq+1GLGoH2y0~jV%vG<;z^&r|aWmS`uIKc-KD7=i^p#SdqW% z*ZH0XOfPkLRwetGSor31W>t-R9Ty=1grOubZ3>zR#>6A+_I_*3oVu`AlG}rdrW^BM z*h!C9*i6U5hwiFEO76~mdcwc7&XLvybkcPufx}V6jz<Op0ZZtr^hh>d1KGijw?UWl zVI^6pvpgUXqOHgPe0EAJFweJ1i>q)G3z@FYL*&FcI|T`OhU441pLWVp&VY>fvbsQT z!2-g6nxde;50Y2V9)Cjv43jjZeioy70g`Ha51+L2GxCL7az{1;Ifo@W26}nfgSP(h z%4J7x1B5mt2}qIN0lx6q1owA$c|O?9&2P!vdpf8Omj#NSE$?QpeuR#)rQ9HkgeKcf zopgJ!o$&TFfX>2J3}&8y*}7w#Kj${D;9Y0`u_6R>)_3UkklkH-N~AyV?c8vfpV#LX ziD$vW994fW?B!y`SZn$LJTs>)mC(O=z542b>_BRJbr+8B-7_iXhcJAbhnC?)mWNeY z2wVZRv4`Uq_Vcu}5GiwwZB+D#&Z@m|Cc%Z%t?ITzK8x0lC_;hO3X2Xe%l<PvmeUJQ zWy*BgWHCxdRLDxQHKVH!{;QH1>ceD(#Jh$n)vPw*(NO6R58KO_q#ss-$OQD0Iz40D zuR70&p{0xj4@}LqoUD^-0N!}M%+een@3pMm3#Mm?<Im)5<&>yjb&4o<dmmp+SM_+M zN;0CQo=(zc;Yq>vHiesjVOi>JKGilJGkJg8)EMeYY}5Kr)xGYVZ$xoVooR5bChqGt z^#Rzq$%KG&p!6OqaQ!lW#KM!}#}j-BH^#)RVD&{ox^V&@!`Hz-=MF6H=w_q4SueOv zseq9(Fp6g2Rx6r=@2A1j3Ubd-&VK2Q>K*Uck1}`HnuW-JqawKH0fZ&D;U1hf6@l$7 zT26ZE@T^Wo1=hS9EAp~RHSq6`d%AbC;XbE78xfd=Q(q{uYmD!V&Kf8lY^iy^5PFSd z%yJHNy9A}?Yfz7CvAslpK6mH4zBY~q`}Li*a^TFMni6$`zTC^P#W*F(Ms(oVq;vyd zAbg)Z|LjFedFIE|r69L>{?vPxm~fv+u}4{~ELusz2ukPfy~PjjW*AsznDk+$-ovAw zRdI$LNk^^R%qZ!{y$N4xz;HrTeT+l0fSkbI?i-L;wGCUo)B;FY4b1xClme0`<M4s% z;OcjM^b0z7*%=gcMh0r{;Z0pB+15YmKENfddFT4<?MR*<G{ao%SkV@FP15k}@1hN6 zVnDND+l#m$A#ph0q#7LJq8lHt@42GyJf@6m+$P{WW79r!U)ZF=_Bnhe3XC?xPG^0& zE>Y37n3Vb<yYcPlTera%nY|73wR=dwoNszC^(@be&S&C~tXVoGB~04IWIC6C6&@Iv zz3>tM(0?H%u$9gRoRj2P*#T8uvnPARFr*<ny5sFx^_phDXpfLkc9qPLzuR>v1X%9j zt;73+-F*TS-{xZy;H2aYrlmazU%0J<!Kh_|#-i><3mD|$iYl~oT!mlNH2qG*=R6&> zd&>mor&}+h&B;JBd%}Xha4x^K8?d0&^YQUm)J<<Osv3tI5w9JRqmqT_G|Ir)(_60K zi*AI4_r*qa3$h;ch8SNy8oe(jS<Ad?bnp#~JT~tLnWxTMVJer!C+v=Vy}Jxq``jdj zZVGwaRe|@@0#?<75qs2<6r&x2xLSSK5c%+arExvCD+-8J?R)jl&NvE%#Hp5z6If(Q z-ND*1qyQW4@K@aPo|0x!Sx(S&J)4qdG2Bv?^x+hNEPwG76(G%fjJoYqbs4wQ+-TWq zTimcNl!h~wg9Z$wT6+nm&l0o{5Z>W!?Zhzj(g&Wmv<(@-z1cN^4ZNm)ln!0|2fuf0 zRcmY@jjfo38_?K5qDljQIZ)tw=XaHcg?b4*`0Ul_vxZuUBk;kR9*=27-Huj_vfowa zk8%swX1~RQAuOG^oF1D41aRHnBf63)v$PD?Q-%IvWnK5kZ&1E=={ND;MsrcNZZ90S zeeCS{1s&Mi(Gp%GP?2MoLqAdw5QElDaK4o5%UW4)c}}58d}Zeo1H;9SC6@9;8n2I| zo`((wORxR{e-vk7K^mYiUxBr$ITu<JgV;;kocBHVz7O=_f!;E7&;3abG0U5^84nlp zR$d%1xX<vMn>*GA%voEIAn#x<I*eG4lr(HdXSB6#993Pq&2ZB5WBT3mJ9D9CnHV^$ zuk@dpdix|h%O4bnxpYWLZ>U>84mz-W?5Tly1b-E1K~WGd&;%5`)8`;-*KF$toEu2n zl^E&j%B)Z=(8z>CX-7t13b;UX`uA)|(m)Lw#!Pi@P(Q8p|Fl&COxwvn&K}qUIa4e2 zz7esV)^5WWJ0fuO7Bmb20>RGC#`vS#H0WHty>dS(QN2JNRu7zSnZqt^fp#rA(D;f| z>de_w^DcdlB!dIg1OZ3Bv2KdZGt$`@iY0E5e|e`6{tMO?f;D7>z>r~uaxPVC1YNv; zv6D(nU5?)~fH`(>tzXxef>%`@6@$K!L#%y$>@dD&z{u}#-p3i#xhx!GpP@K_gDg{V zE}*cCHD`%@kEk=$0;uQn*a$x2pTn?XFHoB)>zBuqa>vE6d1C^1@P&h<i8_k*L8NH@ z_{Dh0yf*aa^%QmU-mR-7Bn~4}k{`MlNI#WDA%MAoJExh3y)-3X<F^vL9r|_pkyp=O zc+9`q9(ORPmEtjyEYQm03xWyDP=8&f@euIa3^VS$%g7>VsDgme6Gb}vvPc(-;^_e{ z+N2i0@*b#$yLY*`P`uns3?pjkr?w!+CJoJ(heiHL6hSRS&TB?N7kY|Ng$OjQ;-aE_ z9+|gn32T2QhEaUgf$lsrSLezqRW2;W_DMwVg8hh`SrWJM?(6jgNER@?iY}wC$ojg3 z1#C4K{thw10fnF;1cJzAx7iCOAD1xAty(F#V-><+PQSR;kx6+2)PvMhXWq3?ZJH7T zXUibkOJ&rg&cUe-OECv-f>@Qo2xjWA1k9{`KY+3${%dFq_C&V-?08*_3WaJ)kfQa~ z|Hax{M@8LkVZ)>dD4>+mCI}LWgw%kd2uMf>ND9(1bPWswA`Q|=i=@EN-AYPHgLHT2 z0K<Iu=%bJ4oadZ(t?zyRbGcaXyZ62G+SlH<!uNXU+Hp}y=eejn;5XI0<mW!&2n)Q9 zS!-GJ@;N&~LT+$b-134lf;CzzKJDA5v;9wE6~N+Gthy_IXV1bw?K>F+1$N#n)Cum} z_+$Y1nhjJ)#ZZe2;6@zq_pdQ!IJ~FFjZP&sC^CbUugH5Nr9Q0I3_E$|dKnD3b$-I3 ztl)hRB`6MZgII)B@90jKqO$zlf6WF;a)IIAI{C>wa;+7H>&}zAfe0|}Z@xxgxUOoy zLELUlg~C75XB(h+VQU9suddHz%nK7!H;1(LY2U!3JysP;FjEU(_)Ji(Knk_#XLpfD zRhb^tnAs;6DPzzFT{aR>PRScg8-Grb0Cw;mj24HHbQDM^QO+7VsC=tAV4lwfbIeLb zYwsmT)SrX@WuB>ask;C8^-<1X0w}WDNwA3Y4?3t~;u)iF;MX{0HnOW%d@oG{Y3mZ4 z36t``HL0Iw?7!-Aqsm|MpuA|J>Ee0`Ose$%*MoC>0zh;z0kef!%;i;42WcI99kLhx zI(`|D3~OaEBQJ5nT`V$#o!z*Hl`bqW5InQ$pI}PEj0^vIUAEt0$S<d|__opaVetz= zAPBpyw8aDUo(<eM0Fh|ckd!ZP(jH1+GG<4FwpVmZnb18oL2-AjJT~Y8ONJ8PDw`Tg zVOyYeo>TVG(w6J17xGy*6r^Sa7Dq1;e9orbOoNUVsX-qWFSWmQBF~ukgm#AR25yTg ziz(ykHv%KO#K;&WJCCznbFwXsp_|4%0wits2>#U$1MBJ>)Yu~Yw4=`s^kmvk(H-@~ zE`(<B#M)eAgziWLs7k!yz7sBYV&jc~C6>#vaP5FJ^Dri7pz}bz-`eI;U)Lw>`c~Tw zV&^Ak8&~c#m7N!!1B(*oE_|TGl%t~Czng&3yBKx#h8PVGXc&>q*Wo$e1df3<CXj)o zePgssO?h&~u@00AjL%gkiNW-;D^~{BF_uvRijBTf3l;K04P+?Lvv2Q5t*F<l2M-3k za0i*)M?)jth~IX|b?45N_%8g}Ok6Px@Ah|{TKHZg*r$7>6<Vm`#)+C3>Z`(}uD-&{ zi2}hEHPQ<$c3pS|SG7^uTF~Lsw;R<pZ;pWfKT-jx<lX^kc;55}WqcGW@i;PzW`?KL zG+pf|>DX94EHe>IEq~%tY1VkNUMJf1y!z3BkNh~M!=ulZps>dGuvqGV6;l9TfB1(J z@&PRyY4kU|T9TcWJuN91Fv&tHF)xNt9q_M-Y0ze?dd_1J(^}9a<PS4j+u}eM^fef4 zk88X$se+2w5pQfPuX2qrUgDdI2f02nGHbk_7pD3~T^-&f=VFKJZ-BOYpd#MEUUn4> zL-G75i#^#?5Kk6tu${o9Lj;Cq`x{cwFjICf0Jiub44CDnA^zQ6=L`>>#oX>#uyw1# zP_?Q6xj#u<^QIyfwYIhqnK$kA7QVRX0w&leiF19f#8#V8@!{zKnu$$G{as?4>aQX= z9%`dxG#{!l6%AI9ABcxo;qO6f<cTibhLl~^PLSU<Z-K{GZPv!aO$t`ayIg8%)&z63 zRBA!;OF@L9qC_<Zzu;Anxvb%QYGC!n(3xiM;o?2<zjPT4q%<}9Q>Q9cyiSph;*ezY zxiS4`<x&Sz$&*T&Y7PA8etN>Qm-To?@Th|5<>0#O@Tz!2oO;QI5_#MjBTKlvcIk&q z`3GaxVD|=yjWEzvohUu&f7(ZNR<nN6EZBDM@VTPQ`wp#66)-J*J=^EXrG5O%guJ{m zfAQz|0V;i`URc)p7p+Mqzgl;@mG>qf{@;x6$G?6b<md^78P`CDHGMQbI)O%pAQC{A zdb!p>UlfmL7|bB|t1D%cQpdE%Ho;4r6Rvr+(ae7_D>=H7tPJXdcFF*YT2%NAMcNd2 zh099}9L3m1hh2Z$lZ(JML&nwBb-aFPWQ6S}+Ss{wd>eqt%|8G;44;pm1t6Xx!%fW? zh)b9HT}azpN^otD)lcAIDE66g1q>IO_tuBd1I5@!^Cy}2@179&9qs(3IcG5A2)i8e z=Gv}%5xXILoIhO%V0A0Stotzu3RgXNhX7=xf`Mb!KUw2me|JHqT^a*yMofA!Q!<(r z*W(xPKTUzrq91bz^Yi{cGttlE0aBGVQ=VGgq9DZ7PXIbxuRZ~2wq#9Iss2d%|DvP+ z=Pw@qIev387o{zg?&`ix%gCrwc@V*-Vli5F#Wf=W`=>Ji%<_m55PlJ~8QN6^`I+T* z2O^a0BNN~730N@$MPmw1OSa#C(vaU%+vTf7o%s3^+99J(@o<u;G%IW+<30M$m+UT) ziLL=A_~!L_1_D0cU&J<m@NTzKVa<E~eVjG}PCg5ZfqvY2%ti_I<I0|n051&=J}z?; z6R8UJ)Jprc?>`jU56iOm@Q}Y4A4?n*e2$@3!Jg{B_(+&i1o?^>Yv(V+eF=SR2bLmK zSlQTo38SSRE48j?9LqVa1l*G42%utMYlZEKJTKsT_S_f8+H3>Uqd(@M0FmTxgdIRQ z*g7HrC;Kn}+Mj>3kyRUBMZ#=uhTXSYR)RvI=lpMAH;8jYg~`(6h?eEn97!F~5i(r? zAftTcuYN(nRmL+vKdx9y`P;}ymP7B<lt*H!*}lP^y>JI0=IDPXt~1AYA6Xh4(Qwm2 zgk<6^?CaPd?3+aW)i;%YKKljh_Xo))2pO+udNDY5l=H;~SA>JjNv+RuE&IXvhiwxX zR<cHw<5p83Z4a|FRwuyvMb~~2oxb|HTmqa!aNT{*%3?p*!yibik`D|fK62csNR-1W z_6;J!;-C5&!G@XK?Q8?mPBQrG3UT!xiJbd+e<olp1LJ-+>tP+6F0{l+5Q_!8v(Qui z04q9HKn*ip`t$3<|Ksbynu^$Y7hPo|==?KkESNSNnQDt5!<y#>lVM;Y_8l2;92t*( z^^(*=NO4~yCx7@b3tzYeWd+|(1~J(k9v;vXU!0!08PD?9jBzWVc<X!tv*K@l@c}%6 zQC<Hfc_qqrA4g}((O9QG{3lCmYkMRWwgQ+TzQ0FYlfhBjTmRQ@P*}Vi3B`OjQ_EPE z|L<|r>*o*DMDKxdQ!s#FVxptN6fX;&r;{vs_*d5nQ=4#in&ZAcocVE(lLv{f9-V|q zXo^OZ!|eVZ_rwDx;{(<?$!0Hd<`E@|{;A(Hqr6x?kRMnBllZ37b36t+Y~}g&{zt#Z zJwttgizCB)#T@8l<nssq^1Nce?wZQYvB=zh_$4v^HUcf7Wt>l(@v<<2$xOxcf0^Wb za(lh-wLF-QZ}2-hNA4fy`>St{0X~piSka_EJ|ZdUK6q0HT!j_t$j)x}JP8S)USRf( z{hn=p_X^x3X3)Mt=5~|9?|$#UUW&M{<7DGU4y?w|-$So&F#0;zLQVD8_y3`E8LZI{ z7jZek(OKO8mOdje6IO|`Jkh_z&BC_*5{rxR$6@P7T}nTK!^+eAJ*rBG(J;E!&p5w& zrr9mt*mW+Lp*)K3n8{8m3P5P1wA;zt*DIC->L%Ly$KWr<9;_RPh<_V*GMH7I|2Z`P z))f3bev5^r&!niz^s8e>31$XgH+%#?{4YN<cSZjH_%r&7YnYGypgDSoLUMNPxt*?P zGaSjSjqsw8rGrAq9H{FKZK<ox93P{0_0fd6>fq5AdI<>s?nrO#_nnU_gF;&k4hb)I zX85#@#}*RjxfhJtb8_IOS&W1eA}nl}(L7%X%;4iw85wYqB&4KSjQ^q&Sn}#%vw_tk zWn#>+Z5kQjZWg<%xo)R}$EEjB!(r#oYbUnx)oca6ZC1&J?Y~c4b;((4&Y`UVod|T^ zTF*DN7t*F#!{3)BWXu>b#jPaxYfM|1LCNSaW;tiZZ@o)JKi5J0WzAS7{$No6atW_% zXh;?CmVl=PYvCLm4mjzaWnzT!a;hxkKE!!{5G22~2}X`5v=fJKIhl_ZKL%$zB~J-X zz?VnlGSQIJ4k)6p*2K|e&kY^R0lkTvemXx~EXR1?_T50w_vNCJqh}9C(sla?c_XAG z6X!Kk5h=?%b-*aV^hw{E)43I9EG!<d$`LE*4#ZIUPyNP9Dk*ae=DvUbJ{K2QUYHeY zaw}blRXIfFTEfk9f$hlA#RUj@1$RQN&|p9%i<SXhNUeRC#W!)Nzr0W@jB1<WW|rtW zbxzsnqjo#aLT^MdqDmb|tG;ie3=>W}(>73*t=(PKYMG3u$y;(0<CUh%jBky=y_tb~ zD=u+mZTdjO&76_724?X%qOwL4K|Mi>8CDEHX1=IkZ~X06u&|07b6|9P&+3<SadH5? z`6?}FhhL&Xe?;%DK#T?^DzwV2=Z+Q<gf~@SN5O8^d!HvBRaq@Fo<Xn3jSRvU1Ro|c z9B7pvHojjNK%n*&O$AQ%MrJp$a#D*UOwIQ`&X>qFetRb7wY$G>{e=XJOdO+1(Tr)S ze-4NP71AhgZoQtGP^Tz(jp<};Nt*|?1l+gl(TE&swN9NHZgWIh3t33yw-G^C+8~E> zNnM@IP!5T%GZAaJ=|QLagS98!K~&J^!6MsFMuHs-PuFXEU%XKF9e55gu{u@?5UB6b z6r0O1y@LV`W^s9CFync2=Q*ZJ|6|d}Z_ScE*|4%M*AN#L`}a+QW<FpRNIRBIS>au< zrgD8<=Es!;*g%`bCv?s3f}6llY(i_OLFdFFV{QDA@<D$odN~V%F7#y{d@8hmIQN#h z_IuIN*YPn$33v#4>|M;^P>8k*6jiYyRBX^eeqJueX}huuvDbqD&#&YR48qA!^cKJ{ z;(chwL@+U;x9V%Vp~$k-r}eL3Fs)zU6Pwuw|J);Xko3G}1)k$I4A3e24OcJ>q_-3j zzdb)zUmX9$OvYQUef@aZbiEM+MP<PJ+4{tKxcaKAcLEs{HE8B|vK8boULmmG2sV^| z`KHtrGOBG+ZF^8XgKn!WJA|;GeY~$8yad(LW<aj?ew2WtTD_y*9-W>Rj?dSQ7Ff?o zaiWht(zB~@oJ0b{u`ULNQ#sO%$?7AsKGiWtwNWk~i&d)*vjK8z;WO}v&L3Meu*fj0 z=@Q{b#)}8nG?8`kYk=qz@QLX3<z(;qm4uy(sX#V&_X-o2IknJJtGPBEoZ3f_(=LDX z4urHLbUXmPr!DJhTlTG;^L)H(@f^fadsPb=2(2oFLZ5JM6-5qIknvQz%0CCl^0&MA zO;>0xUCL;f1{16p=lrL?71iR1BDSxJlT9xi&bI9}BMhgQol#J>ocj!Q+x1lcQjx2* zHv}V?DBnNu9M3)78fqT)t`%8PA4KnJyOcY&&e^Rn$HrKrPdS-O&5+ATYU>F!{FL!+ zyVLlk%VkIF?xs+5KjV>HmfQSe2y}d(di*NQ^IQ=iOz@GNg~k2BFM!L<l2V~`>^l$3 zoA6Lfmu<}3IRlUYps@cUGp5^7ZOBlnOeGUbOZf`TNACdruQ~bC<m34h0(v(TaQZ7N z+USa-jZ^e-hx%B#VW*ty_LSpTZOx(F>2L?d5(H@&RbxLmw;>GKlU#24Hhi-2)?$Ae z@g<6d{&aP_uqzPVCfI~76|B|Rt%ES2Rjc{;V|(4Y4-F?q%1#&Dsv)qNL~z$g;3Vt! zGn;PZM=F8oG;S`g$;u|cvXlp-&>8nY^!Q^<2Po$8rHL@Z%>F%VtRAX>BnKArx}nuJ z{3k{h_ZCk#(7HDJ8t5Ex<3U)|x|(g}$iCZNf5o2S7~|}hBR%w&jjq51Vb}dbX7tQz zE_%y>FDz~FbVsfk-5;-VuyNd+fZ#;0p~BR)6rfcg0_+b@z@03#)D(arrFyI>rWWdU zWEd#B1FDNBwJBJag+=)Y_@h78o?yL>1_q=ZIo&G<v160IqNeaJDwp*T{9#o8I$xFe z68ZVyVAuW9@fGx8X>En=;FwnOZqFciZ+Sq@Wg@?mWG_*8zjeapa034R2#FYk=Pl<q zKSyXFhxCH0kw<J%-jK=y*kGy~3X1;b!0I-&!IA5FvvPHXueM0W)Y$y}3w4yv@#wj5 z<p_YQzEpk<dRhHy_pp+9|J`bs(!gp{6$PfT<mt0f6ti<#L}wtvCG)M!RT@hKkYl-V zf4)w%!F0E>v)SQFX5s?(0(4H^jc=DPGZAWhQm7m(yq^Pw@6h(rFleJ*E@;aIoja+m zT|De89Xx498*cRFR&b)W&~T(Lk9PIpbfeo*SN~IM=m{n=Vhp@~zx*;w*0KsE3t;wa zEG^xk7UtrTFHXh9GC2>jnLqZ}V4a{>;|U$f2g^@K#`kY7nXj**N=vKFd#}qd!A7vg zdLpH%D^eu=ub5#MP@I*F!H;{_f76uw>A)GTjx)DX)QpI(-;$)LvwA9ehL%f*mdpQD z{_s#i;JHXX$sNdc^p~x6qkEx-^RXf06FVYKbjFd;$6wwfJDnB;vA!imr<Z4E2M8qh zC6!V<X+U_Wa^=}>NrOjf+FM)8x87#1c^@=DNY8vHPBax*5pqf@vT02opYwFD+Ydw@ zjI(oc794sxg}i;?<O!|Wo@VEp7%FmrEyU)&eTyzZjy#;bGdM4OMxr$G>I?A^w9y)M zr(Evq%y(r(DGYk<$-3B~&H%}w!d3xm8T7&0)xq6Kgp=@%GLTfQyH`|Wc8wJfd(ZNl zZA?dtKYJEZc~c}KE9>LWPt`ki7!%~?V`esdCEK~;f6D51DQDbeH>x&-fIk?$MCIgs z2u^eQ#UcG<%h}4uc6YIx9Kir_-c@GgaPx%jd~`$Ye3uNfk8-=Hon@i@O7mG8soPa( z&RV4U5VH?e_dfTFW3QcFHtOkjf@wFE5IB-NlpZVVli;I9aWufDfvBlH=Q)X2`vi8k zGsT3Z0Il@#2MbP2(|PstmHR=e56Uw#THJAYk2ZUs7~gv5qlq7};(t?1SOk3X_TJ!H z)@BpLEc-lUC*|a5uR9UVTHvI4@<k_U_2WVA`+LtAuF^x`qcfx(FVIt!5+)_QtK&y? zt_ST-yB|tCD~Cfh_l^_LZHi{^y}>)uA?e=GbG9`V8BUWowxaSlPb>}ypx1!VyM78? z!{_08bPQPqQA&cA#GTilKCvF_frGr;sso<W@^VqvBFf!4oD2&q;KA6BH-Uz|$gSxb zN3mZ$+(wDU)wth<S73iO(p!totzsf|6FJ??-ha52yxuK55`20xX#MD5AAe_dFxRvc z8qI6xS2Y``n4XQ^)V$?rElB&kme2U}8$)7-5v7y*-a<xtZUftsi1m<b3nEBI!>;3j z{O|NRGdFi@dR%8ZT&9tm^<yx;oUC#A(bX)!_(QVpt1mvgQSd_~XCK5_Hb2F4GE-Jo z);;$@-sd_72)=yo$8Ct61|;zIeS8Dvz$upZF>ideaKycv@rejyV`a_wyhy64sd;<5 zpy2Mb_hoRFpf51S)>=7+K^N23wrwN*1A-0ScvrFB%eXIU&@)DR^M|4Jb&A0dR?=|U zn$wEyU=g4Fb4k{ZXijcy<dz+Gj;`cZ@;V&l9E{ne3&o$5S?Te)oqVYB3Zf>8?G%BW zUfe%$ySU=S9h<YaOuRFECb!1<BtRW{zetZe`BBm1<i=WxuZ2^;4zw!5GTuS2+KJJg z5%(;>p}yOa*xmaU+)1=#eO!63;Vk!D7i$vOmay|n$vlLbpHZ+{j*Sv@dRV;9YgDmo zlj*g#wr&czT>Y{`3i-vder7wXrx}rWo2Q{!+!IzBOwau|f%ddRMN8>ixOMRvdxr7y z8%mWQhgEpE9V&!Mj{5Y18=ND9Ew)O|9ldmEGfbG!EbwzJfVLgcragtbPVFeK_*+A3 z=a35t;zd3KPl^ts9ysx8oJG<r5->MVtqPLj6u^Mw=W%|RwAomNGW;$Afk+Fw!IL3D z=>9%{9%5eS6m{o3J+FrV6;`|VFDu`s^OjRw8-IXL0?x9jql|1IBnM|fyyv&th}^4w zm=K0oy4+1SblS7C%?xFSt1%o_9S2?RsA0XPK3+C2jJ&bBHPb{tQp<Mwa;#?H&dlty zp?$-|P!D%G5!E~9%d^*bDD!o6o=w>}jKbM|+(5B5KZJSiLCZ$80DrwQ(_VGD;3nqP zI!hi)FvmP9p-I~JEaluD%ZFGNjXcZ6Ji6khpxDSL?K~5^J9e)vDTnZmbC9`R2tRTE zE&c7I$RMsV0r`hN%}0&kWe_=u_p^+;r;BIKIz;3|NZf<Q@fN>47&!E<7u}g;Y|P5l zwlzMexQYxr-blC=98t4#G;ywTY23<jLz7wS^XaUo!z}3tLtMY$%aeGAf{t}ow(|8D zQ!1*{IZyrE3FjRdH`q&tKTaY`H^R0^rXnWKqB6PV2zE^odV<rudx<e|EMDND7H<;G zw}&;ak{d7PT__sIDI@AhDzGL%<DW--hU?c5RkB`WZ;E;^at#vL?@sg6hOqXZDJwU6 zGjb6W5`H^RW^p=njnuHSY>f&jyrimQ*vm)Il<tS_MmK)q%a^MNKFqD2rzy{XDKAMh z@tyWm8O{aPZhiHy#?A28j%=1vXV$vLGhjckrhZf!HDoY+cTgK9^CiI?xLX?E0r=EO z90eDpcNx(ohG1`r0yhZU2z+`PCO%*d;bCs#$ucVwBlFuVrt=MVPVYT;&#IHn7?B1) zgf%1~DM{GG(Ei5awWVG+Y};~Pv%FW3WbI|5_@}nHsrHYxhcfNj9U{hpWw7afl|ucS z8p5Zd%H>;uXuU1z?I!J*<Lnd3g`V0rTGF|I(1j1V8rlG<-!gj-8M~K*+SC&FHwqLt zt+j;95=HyjXy5}FO(=AG+xoi>`0KAOVsz%UjC}CL3c_w>@PhHzuU}2ll(YQj-d_;c zt3Ubt==z1v%J0ssroYLmV^{4yvp!L)?eg-ot)HJ?l@`>=$qC>Gf_ULhn4lMs6n%=t zCr_S4$L6Zd05$~7>|1uJ8W&mXZCK88x{+rE21mRlT}$A<BJ%v}DU2&3zlTrT>Dl;U zGQQ26T*bIcvBS*ArYC`}%~H^+`nncApe6S{Bk$kRVePb{#5<Pt5Tym4^Yv341Ryl$ z5iUP_z;~sBsE?bTqWyhaZME&DrJK2^p(5NZsh~h0@Y3$!-F4O{g0Gcc3hGMjnoHeT zgh~;hswp^B@l@Q^^ODcsW@RntR7mn2T-igqMALyX2J71g1VXmTPQg3u0?-@y$+)!> z|IU$V?wIPXR#850i6Lx*b_jxUvQ@!Xzu7a1p4l7B*KZ1@E!BINzZFO=vh${DYc7`~ z6td)qbaI+$z9|>tY!E-8uswCSH8XU#bi&%^-~!}t^~ZnusQq!K-V!(A0~U&863{Od zef#bL$Y`FLKqG>9q}aV%8J}KhDHnLbh&Jr8iaXLa+kb$uiCs;6v6u3YZlnAOMx8`$ zc%yjDrDNDuXzKEfoo!-Z@=ZnHHg&rA?;XeHHl7Y>uw`b6l89z!CzanS38-?oDFbUL zm}5iYY)1>L2e@73{KDH7Z2Rrq6ta-$FpZVXA+K@BTj)%@yITpDAg#;G{pQ`v$mO|y z><qaWUG@_Cn_CN$>XWqHcnBQUY$IHDcBQ1?!Ik30B+=VcrU1J!H#fJiNUJf!#u|Pi zElq6dAs`@-5&Mu@5!mHMG|<AnsaJTmYUqFwi;im4^Dc<C+jdv*XoO%jeDL{_Pc6s@ zWDL(4)fic(nB`OreJNb;YHhvDV?A3t?!q&`-Sl3mnWS&jwfUS=wyk15^XW^jZ}x+l z+p5Ou<EU+2)A7^h=!D_vjcRUc5QjaxzjMYVhT)`Lz4@Up|B1f7zEHGT!uZ+CY*?=& z+uKyfHdq(WNp}#mCQ4jskukY{4vJ0KSIWf~(x*JR7rBKGcbF4zERBc|q#(a>oyLxY z==62fiN`HP8=gT0(kCmGM3L;;t<Ce43IS#QU&dqhy;_+cObhBtG^=b4o=IDnH(kH+ z>{1j7!#X>=;zovLeX{r+wzT(TxH`?Dd|9H^e!847u{;W;SU+WXurUF3E62G|a!^qr zz(&Guy0p4#W@3_D^8!F<vNkq0H~0F6fYQz#NR6^|xN{6M3L3Nv=7N2SxTof+y=k%t z;1}{pQ0oWVJ_Z=q;9q?(V7i^L5Z?=hLW%^Dk)G~$j5XZwQPq;KJYhGB8U3iug7NK( zgBaML<ueh~b2qfrHb+}Rt_|ywc2zL`ivZISfdDa1Bzk$-WZBY4w227IlH_t&e#I!* z@$mHYI+2)#@*&F}WHQ_HsB`XSXa&)wC~c?hMu#nSH?LG3pkaE=a^#TKL}exmiQBNG zYmY7oAUyQjR9FX;0|p9}8(ud)z*+t>JF)%2)k;3mob_2<Sh|tm^?MH2J7u_%(h;81 zMLyn~XP6=xzYP;>;3GUMs-Jq7l<-oOcP(Ur6!F$A$yxS{jEuUtvuR9;mLM`x7)|wx zRv9NhD7gPE70GQ;J=aaJeqg>;Q?rSTg~r+EIg=mfttG&mvvMbF3x(2QEP09vz47_e zdn&rZjLR7>yGqm!#@IQv58KJNrAI>K{9<Yy)LhyYG%gKSCAjvCO}eE`c5Ul&0y4#) z!kT7LsdvXEaO9eso9mk*dmTWDH7R<RNzbk@i;3>4+c_^LhpZg0qeLr#yvWi|1cN;F zeFwMsUN$Imxi;Od2^du8a7pND-fk>K6VJEz+g@RL+cm5UcRLyR`juXH38(Ri%T*Qk z&%Io=mRBZX_^~(zXd670pL#Hn_!Y8mYO>rWzCHi->vB)eUAr87D&CQ^$M*1-j1#Xv zd)HSJuUa<HZ$&qLQNd$MyjMaTU!8X;TKt;NiEs9sL2M0tTlvzcn|13Kjt@2li>N6m zDfMmepD602M;&2alrj%ubkJAk{Y)oO^~m1AVWjGy{S8O+gvQBKQDzvkOq?aEcre>> zEyAda$!LLAI;Qi|kZsfJNHK`t`Zz16+TkvplYAI^mEn#+ym<-Z_rgOjq+WBtOJcc8 zkQ!^0#8#K5J4no9YV5>Z7>>}i41z`8u8wAC#8T-fUG61_$!tt1+TG|vPCcef=#nfa zV?H1f_^M1s<mJc(T4|{y%@*(Saj@w8rMaI6nQG$GH@7B=U11^b8<s@{-`77R3mIHV z#Uyu7Rg2F)S)w+Vttgz}Qx6(hM~Rpscd0e;Q&Z>sl*RpD^V^648S`graTy{GBWih> zECPee3L$TzkK1|b=r2y+CiQ!PuiI{>sH=(h2y0pi?4QY2+IUn%k#W$7O9gpVsTfxz zZb90ZNkW4QtQ$mqZbGjIl$4YvP%&j?N;@Nq)Q=xMkBER!g#`o!Gu6L_Uun0Z<G~BP z^6kc9zD`n2nr<6&mVIQz8(0(P`{WTmUeYBe<)ikUY5XDB8F|v@(2AudT1?JK!($S^ zuqIBsnmuwbXqZR`fzzp5&$fgRCpaZ%!L?l^YCa9*&%m@+8HVgPs<%JG_;O?KS$PPZ zRzUs#aFW(nWY;Ne``CHtNc@|emo3fkftReCD!ZdMS2m#h%{uzzGKI%ou?~V%)qtZ3 z8x#KJ%fdG@HlABtHCMrA<X4e<N=C7KB9{=_2I*s5raHL#T;$0t>tybXwt!<<@}_9s zhB$6a=b1;3@YWxYIk1x~vz_VobY*@k@6yPd>~!0Ob2-PCb~bB~fNA~FE(hKet0iUM zS}O9*m#O6Bt6up@DJc!^oMrrp^YaGFY!u*GSHQ#ZyV88;v5E@yL~ip;zRlqKb51e2 z@~E1bgRzm_jzj4o2{%;bVoBik!dzp5=}noc<xkHNt7<(s>4?v~IOz|Bl^|KAqB!77 z1F_I>LpMXAa0kv~)*4kk^M%jUF<sR@=6T4<3IA+1UB<_>rfB5y1E(pN;d}}GJuX6F z5X)UDKaKGm_@<+UR=WGyB5&0Q$QQqN?QA%ecPV~Zys$nBi(*&1NVNJj+3l7=s~3N{ z%C~3v`h{6oe@nJk`Sk;QvRBF9gP)<pWY0?oe@w^Q8~ZAgvc@_&GZ3YH#|ngYBIBB) zTu5KFOQY1Oc><TYds>!gwm{M*LCbZTVCsX<)n~P!m(lXKuOvY-a&7oy5IBtkeawM! z9(CqqO`7=04p){k2QUE}sA1MnDKxQp=EBCcM~@zHQKjH%9b;jYdF<OqMZEg>)vCa$ zrT#&SjZ4JUxx}KA&GaqmwA%Zdhvub^cwOjrRGK~9i#KY8=Yj|Qp*p4Xqrx^#YctJ( zv^*wCorI5r=#cW`t`hdpuqa<pl(ppACf-R}t2s`&CUQ%Jp=xG7A$fi4@}f5E5fMOG zVy%n##w-QpCU~>AHoHib%o%uBNw*hPNxOyfoRN2HzBX(r7A$M~-T$eXN@i*>e+f*~ z!AG>I-e0$+q%*w^pY=$3dhwWGj+JgNAmL(PDeDh6>STg&zHH@(P4D|03-!UmO7#U@ z{(vpzB57%x(RY%D_r!;je7TiLTrANmtYoC}+Cp4qL<QK(`#lf(9uW_rb!m)eib_fp zv=cx<LN}oLtTZYUTSe@#Si($JzbY;Kq)WyyftqnrXys&nr`Bc{4e#DMP|>Y_?Pi+k zwsNNC)kytV>9~ttP~FfSJu3F-5t^Y}oMeJkrFJKH0F=_-3{Dnw)}})@>;wTU<<M$o z+1^5|$cMMz=wY#7<L=J-<%8|q=Q*l3;l*aH7f<_zxz{Ocm!ai21c{7U096=eTg(_L z*ry7{1QsHVsC?_uC070ya_ByMvCAJLzx1I~6!lgHa<;jx?Cf|jc#mtAHTh!=eBDg9 zQ+TZyaBQ+0AQyIQIkDO6!+pH38=v($<+*zqXa4gy9^wPopwxO>QC$>?KyZj$PCs|7 zeTn#-hi)hqmP#EB+w%Tho($`_Y<lZFs%<)L>3rdlhqcG9RTqyZ8SQR^DbLR-ksB!f z=hc$=sxH~Pa(sbw_pLE-7V}d_=lzd0M9|xp7B!PBHnx^}=iGbJY&cBZKFFU|WsURA z3_x1-vhah29Ic%|0dt-p&64uOY2f2YmcDePk9Lk$ew^Fhf?9C&607AVGGB_Lp|#q> zl9C_$RW|sx$%=r88Cex<<TYwM{piuh073cY<*9{H3Xsq-k?{KWUfv9G+<&RBUmSGE z+GHq7=A9Z=L3C5DRT(N1cU3H3bOK58sZqwamk}=~4*Mm1>S{dq`~g7#o(>{QR7lS^ zfMz;#W`N-GU-H2nr@aC4+DG&uf=M;J02O5EThssgZkAt7a8`$XmJp6(m&;~SGg&!B zz@XzMA>|+IeUfqP=|IsYPIIEqSY5mR#o@q0Z31o-e5h=8P%=|GF>bq$zvbS;-6hED z>ikrT>5}R`@H$DhJ#VtIr2Q|jC(CetUziuVwqW7ueX_07Z*c0AVZX4EOULQ9SNPJ6 zOy#9NYPDhZY?tuXQ-efC=~+yfeOAStJPD2iV$DYo-i-m;fhT6saaDt0&-JU!bvw$Z zi6Oc7ZG}Pdf&3y(K)Qf`Znv9pdA^s7i(NZkp*nJrRcF)poGR9G&W&vrmXiCmmKU@Y zp7k&pm1K#=xrp@=erS>fg?t9&gOd~`Oi6>?k}~DI9n6^A&ktWDF<9U#wWzB(4%X{u z!2dYBD}G2a@E~E5*zaE7z}o0&MQu7O%(Hgy2*QBshgL#P^VY^;FKUiaEln4%(fT4C zY){wbKGs%LEydyjb(tw0y*Q$B8O?fWs0QBAKhxZ*m+GIGIH<hw5LnQf5nuNHZP;GF ze9q%mcV)InW4K+O3HPyIPV|1ORX6gEcAe5Bt0hjKT+&bA?XJuuE7+#3_C`j2B7O8t zO_nI#5~G0Gn?u%G{yH;D3N^!wSaY>F1+md*ARo9CD`0J<y$pTCj9n_ja%SGsppu9p z%SV~+Vn|7r&UT;1X@pppMBc_N<?yRIhbV0N5s*?D1CZsrxQdF(vu77JOtu__rYxfc zSkfQ<r$`Q1zN@ZGR7{mKGbXyA_Ht`fhlaV|3|HIlb!(rbA(N$JIT`2@cFL{?oAMrh z9N}v^>F)`7k-3$kyLYTxS!&0jc8TVGg>hYD_1tKe0QxYev`O|UgPY?*8i-n7aZ~xm zx#qr+=znkowndLHgFybbFw(L>JzZg0|G6zUx}{bjSpC=(H<?sa_-wTqQmWChgg>ow zzs<WJby_!M)qC-L;!ZKewc%+l`{0?WS*B`7`DG%!`Wo<#<+BWl2^j~<#fIC71dWoi z;C0yz(kE{&CFnldk8l*u@N~~&G6$M*eAfF508$Q09ILgKed0*=iV1&W)MH`;B==Q` zojfK!u3xLc?-IFRU`S!}`OqqB;?!Hn7iQMA-^mvxK0w(>AP$QRp#MG=W77{1>Z74R zzS54k%QXizylL;b)pN@c$R5Uf@ptkZ!gphF)oW1IB?OG&%ll0#=gxS*N+F?c6D=WI z@d31g`!f-3WbDanhDG~qBVD9ilx0M5SbO9wSXh_5K-ocFM#OaX0o6<LzJptQS4iqs zGITmM_A30u7BQkKX$p_hW+Lrh=wc;{qV}G8f9CvUjwqr@+!OG&$_0<lO2oL%H%Z8d zN%Jxq&VN-WuO=~T^UJv*VA*kt@TmhO-pmZDkn%V}js`+Hjh(i}Dt~g%jB{1u&fPVS zt3tzeR}vIDCRom@h(VGpIL-AZYFu?U<Vxt_pX9Y;e=2nk{j0Tip2hE&51A=O3sI4= z(Ak}`Tql_EYQteh#?Xhc4{Q><Bt9mlIy=Y-<7yML!2&*AyW`nnnte0ax^MHGyTbki z^@4Ek*rY^}PYJsFdv3PK|5g@p1(o`5FR%4+pS7H)=IGJZ)KHybl{W=dA10&L+#Dvl zCy`Cr<dLF2<5;af{n?CjSwGv760i92IysWI$DR@ouu)TbISx&672gb#2P7*0<zs1c z2mP(_&wTjhUsDBJzF#3Qy&|`X-yfb6f6}}%H`N2UuvI(r)1A%}Tc}E$5k7`&j^M8k z(}6B&3Fj}h@7#MVS$jVZ7?66i)>uAxd^2H9p-XU7p(^e;vY9c)u3!1%tb5NeDdd}d z(iu?66($#J55wTEGwYNm@iG!c|8ngy0GkveD>xgQR3leTD5FxJbmfaTNgIG5wVUPY zGP6fNFnAlNx3qD}S9Nzbm(N9wQGOZMYnG9Ig+t~&)z3M~sa2q~b+@{zSX`N?Z;NIA zIT>jOk&0My^e{8N&HNFHnOS!+I7`&^Ejhj(mr%z<IIfD=ci^(^zmQy>BRcosA6!O% zXF=!)v|NpWgWk1Y*~wwzK0uENRf22ndm2t)>z{6E(-NOahLzY=msYuUuIlw#2-xqX z=#eAt+wUZPmZPxEQKiWmW+lvh?m#zQY0?-wsL3rkKB@fiaR=P!`QbMP;43-;2jKck zNbEG6e+VQkSIs<*L?QwYQSJagq~=uns`TQ;O_D$xaTWIyLUjeL8%u(_E;GK7uVP=+ z=-yC&jZ(4MFyW}R)=kES!6xs}-p56adTsS4U)D;Vpx9+x-VenJXX85i59Y7Z-SmBJ z#c#b_?@Vs}>F!Lh8S2&ETOPKJukj)v5oJGJ7RkCdAoXHyLU8$DZD?~#F<AJn!LAXA zpY73iI6GaB$}%^!K%YrE^|tUo>vM~~8XoL6V%Z%fMlWpwV!~@)0nUSwV18VWkkE5j zs(5xVNzpuLxySmt#j+0_(eJNR^ki*r{Yx{I{qS**S*r>e;W{Ai;hS*$vV{`<7+YT# z8A_TPe=oD@&~wPwV~vLBOw9gs#6i{0tRAUc4F8n{yg^Nyj_q2`<xdA!UWl7M1Kx+W zE11~II+sZUU-tD8A*g#WEw3~}4SX`Z^4tC_w{BT2Ri*$ploSX+Ubx$9l|x2B3mS7S zrZ&)cF(zxH1WRz_y=UYza%I4_WFkA*dm5$tvVS?$MA01g_+XKD2Tx8>F0YELx=La# z%BF%wpsrTq+=(um_A<lYznjyNtPR=bRm7q4^}aMSp6nE-Gr$hzKrpdyzl6lMK+>;Q zT0<vl5G+^wV8CDS>V<1AG-IkAG6W8@C<+hsM=`C&UQmIY_0WiX^jGDmC`xOsB4%u? zrUl3D1(xJ=9RO`lJg`MU>X}l%u=Dcbo0Q)!?T?fb4-(?xKB#rNV(wP4Wn>P$auE1v zLsJ+o<d*3?T9yy=cPSVE^(ebR**LDLPg#%WcD-D!+d6v3;V7tmpEdZzNGUZtxQ0&{ z44<A-ePDb#^IuK;U0cD~pOu&^k<r6-R?txKAz;b^K0fpb)+DgUd$x9VzNMF*dL$A4 ztpK<Pj{py$dbw;|<vDH}Xe}BcI1XDW9;80qOEXr?tyumP_bPyfXZ!7~utew85IG^J zw~50dr8`Ds*TNtD6d9&KTw_o8&zs(K6zX((EJwI}O!4jQs4~+TE~A!Bq!+Q|XDrb- z71h^0qk3QU_4hyXo)Q1&t5ya5q>Xty(uE0FM~biFP3_veP0o?Q9H)iAT4CYiZb8G* z{pJ+}y`X)2OUV6?j{{>Y_YP^(WjbmX5K#+|%Fdnr)2{`f4{645<IxWh+Wr(FsMP;G zp8{4SE+Z3o@>D}pQ|6Z)fsv9={fz}raCDB@HS+RwI;$F`Bacwnp_j1aMaQ&4F`XNj zmWv3Z9lz|Gqwk=zfXP-f#Uf^YMKkdJ&Y3w(`STwMt-B`C8C`dUOnYI)_6e05hBd(t z)fy{h$*}5YZcb+C5>=E|$*tB$X(}z0*0k9j=e-{MXqLGG)x_9SsMTa0_V|v%vu9%T zLFB(<js91>TfYh&JVp<umZVX_R{Gwvn2KYWS^h73x(Rw~DMKOE`JZ3*kVpCX-RZdh z0Y}D_4>2qp!_B`xOPVZ)zJ7G#vq)R|NCiSUvfH&-bC>@d6FHul`;iDp$}z^#de-Yf z)q*~FBBXDM|H85<`Ogol@cdb3{rv0y5RY`iboXGoU8nNSyF%oYu-%iPxQ$0zWvmKH zm-e7nAdg!baY{s86G7huBg|K?%)9)pe(T2)iRDZo-uD0c<r4e0$|ohqInd?=LG7a* z(XciA;5bItZ3)ed4qy83lJJywZ1{hi<Cc|@+&kRg68t+_Upeb>4s@bETDN9bH!4Eh zf0hGn_drkVW@FH5L`+kINA0S>?#H0DC|h<$viEL?rG1Yl^`w=h*$r`L<-h)Uj&S}@ z%q{M}ox|Vx4n{-ppEf=#fA#NHtrPZ2#l7!@EpsVd!P2MC(Q0rCn_jSMbu+GeZ(FTr zW+6tkG|9Bz<(8^6yyac&pLg^B>rA&a@$02^>vwbm4V!6dF5KVk57I>E>UBg_Y9XDA zCh5on@Pl0|^*V?%RIyT-bclfq|EW)sz8{@E=>8%6k&2PMlCM*ui;y<>cMrjP>qA!3 z7H$69+$NeD3e!W~l=WTNiG`7B^zhaZT0D`z64K<cxO#<tWZ)em&nykLI>=zKe->m? zbp7PBqJQW_fqhH;ixzU1vmQ*IIBC-v8h1^hTmMop<w*i8A^&Z!-uc8}o;aEu!Bn=* z@}PoVfk>3Y!3ng1LD~@h20`5<zH$)C$UjR`PPDJ8(T#iTDG=HBU;pr*ap}iVe-!sY zUvCudQ_1~H5Qz;B&0-p`l**p6$-kBuIN+KFLmLW|z!76M&I{7ofgK_e&JBk`XQOPx zZ`TW*eRl=?eU{r0@?6cz$p2SgkdVb>Kg%tAWSf238}cx5J^RG0Ifm1=YP)|i&PMml zD!wMCj34h|OEA5)yN1`tgl<KUB>iRUx2!fan;3pe930#s@lP`cfoX?FqA{|#)bZcE z%xMKt*)oJu_#{(yB7I~aethe|=BgZ=d4Cb<tAQWF3?H4FFqn70w;C~Cb_cPx5t^vb z^0b`jS1kMKQxvAhD+7do8_!*yN!AvYzwR4AZJmR3{}~f?!g}l|U1UuTllvUg`gYAT zoih5Yy(nU?(^o2*z#b4$HwAI7TEHu98Uu#D5Z)AXoc$bBP}(8%u0pfNJkif|k9;<r z;;tg%^UN0i!Vlb7@n@+LdQoBz?>^aYhnIFnIVkKegy10=-sNyi@{{zBUgL%h<w<Uz zV@aQFe&5N%Oz<j7cZkA@($XPHVI)jJ{R4#-OK~l0aqYmz+Jln=yUdyMw+ht#itXIC zOV?op*Y8pl<!UkE<EGJ{)xdwxs`5-x@fw7@5R#!Mc^(AWpOs!p!!uzqlK(?ATKq>7 zVb;K}4T*tY;Xx~ylnYVOP!h}d70db;kKdw?y+QA$VFG(@qR!-{{q8fT^cr#MzDmmG zM9StggS7is<!{xpeqQe@AV~EE%?0~08$^o!Z6mjXeASN=MYy0`R-EH~AS^Eta`(S_ z4aK+Df2X4u*ZVb|zXYJq|Kt~^5gzlmdb(FqzA4?Z_3sn7X1|IU%0Jd?X7r;+7uvc6 z_3Z~xB*c1TF-?O8iEjY@S{vi*EG#LG|68Y(IjBrl0oDcSgl5&HTT&X)5Cbgc9N`eo z-mkc<FrAUWMDPz~4@NZr_d;SW6oA6%e?nXCG6HvSKj)^TeNgC?OHF&3Kd;^V0ukWV z)bv0xRnI}v+(yBM(l#PP6W>Hn>`*)Si!#;?>OB1adgJR%rrvbfUPATWOlR-1=&=5y zL%Lo@1ZTzfe)6}+uf1S;Xp%i8o&T1ziKg}<aRR+*#S;^xFU8l87wMAC+bJarDf^8y z2#>OSov`o9-<nl7Tb!n*qX39(On{>zmPYhXV(>VZuG2ZX#J&A~+mKSV$R9)4(B#Um zW&chLa<H}k9&J4W9>_8PegG?5R_}-hU=Cyz2Hw^x=o=jpJp*ztOg+JS>v=6R6&*Xz zQUwu>Y%q_D$bcm{LAdId1ziadTEXr246YmlhjXf%qpH13nfhfE6^{iX+34Nmi6G>r zPB%`r`{^ALKJrpEFo<w&FKoQ!gQeVD(ZrY7m-7CU;yIFfG`3_`(#@!ilGhi=;v10i zc4%;LEqh_Io9bq?SDAu4lJc;B>-_R<s&frdtLXK(oByO=@ACny1~tZHl22Djmn~Z^ z3Btd>-r>-r0ifNZO4X`YnN`F$cUNEW{x5OhIMiLVybm_%x|wn)s+D^h1{p7;3w_bI zPRpz=D0m@4YAnbsiYdGSR_ZR}H?S6@MC#XvdP<Zw#d=tjB)9izT~RYu_Wu6l{hZ!T z5&NCtm?n?T+7rY|9SJw7_!R5e0VztTBy{|C$0_38%DG$Ul^#86Y2xN5u~9cK$3{_} zdjcDjX4^?M<OpNnSNwWhvAc-u@tBahu`?ssqg~p~+)p9RD0H<0RlYkp+=J2%gnrO@ zSL7I4WSps_9cI}`${00XA;Sts9JrBgypy7q<O~jFU|^bam2uv@z0|rPjgI43J9<(+ zUBA}TyptMbo#IU(b%4&@tyAkEp9s^c75lXRPFwTl*?-b91E$+BM#1%061W?MEuGjK z;TrhJLIYCz$d9+|rGOIMLtVC{d`0@d1Z=>h1$`fFQZDS>^R#Z5d3oj{x5Q1ioD!sH zqS(W?JM4hCW7u>d`mwR``)MTrKPQgA^w<S(7N$mwrN|kf9|4Ni=q7Ox&nSGiED_<6 zu`%(PxVF!^R}Ba}(r)!;alTGnzC(g9&tZoUS+_FT5CtFEQ?%Izb1ypQfE2m0ko)e0 zc8}N2(R(*oJ!Mo;E_++exM@MwVrt4JRqiy#a;$E?490X29#*O6=8r9e#5Gn4*{_g8 zv((gSyHTSnPwbZRiD4?`s9vir_E98bk9z5W^R*#2H7-)pZ(5=BpJVw#v~^IFfBKG; zj0_)#nxf)jP=@hm<aPvn-XMM2)uCc*$Wn!`luHbtFQIdS2eF}9I!6I&0*1j3{w$qW zH1LOU?O;nJ|C)IHg|&2uZW~atoSBrFz@I4VM?$3t%W8S+gt625STnLmU-no<ivl4o zI_B*j;QRiP;PpzP6q$5Mby@$l!^9SM?C!Vj#$`mw-XddayIy(+E@8)n7cUckCxX+C zSGLaLP9Z!>H$oDYFDJO|J}`Dd%xYdY*JNN!L0gvGGql{ei|?889S@uCQ+NXXchj?u zM|X|;%`$b%mC6;qmx()@f3j^h-pXA^LXFpO0_}!*6A!hzEKhIsa6^1^E9VloJg`T` zgMv}D;n(I)VJ9O5U7_~Wrr8{oL{ujNHB=*w0dB#9=R}VDi{^3;nOC&J$IHZGoScJU zlM~mw^tL>RHXwbb8jcpxMb1G=r*D<=Z^K*I45SWFavSwEFbI-ofb&O!sin2P=RC~V zm!ERx?*O8ppzv8x`@sh?JN_Wgqwno>C#Q9DDo>TJZ{E&QxxNpGs$p+RP3)`;%7|uV zd^gnVz|y)utIi_2w5}}>Ktug_p8rfpttmnAzojF@Kx)UoFNVU4ao2#S-i7iU*x6ep zV@qq|10*j~p^>hUIV^paRaeiHl0!AxC=%~lUUryygK;n8re5M*9+d=acem*xdG1S~ zod$Xkd6LOlqTlUb*d8xb9>9w$*Rxe05+OVm6S_)-QT+q75pa*NCJ$OzApV+@XHW#f zhbQJmMv|l@<><lO;qj4MA|2=bB-OP1=SR(J)&A9o-iH}gy6)FtsZO;%Zh||+TdgF0 zVgoL>g`LG*+$+)2_iIJ-o;&X|3u~iFcejKxC5yMx5{U2|;375ciVz>p+{5SgltQ*a zwi7BT1hqHJVG?@pzea2k(!VnN;vL**PILg5WOaQsqlyzdLw>b|${Tm2g|l#Xj`dJI z1;fO^kSsQS<#?ev>ZRMAQK(~0d;cQrWm9gOXpxy&>5HCk&v8a`M;$1JM@F_z>{qdh z?eKc??^-Otd+jMV*O~2Uh6!J7`3sBdW=etvEx})%M|qZp4QGUOZd_ab<0Cpym$PK5 z>ixh47Fkr>XR91n-H!<G^-exKy)8D=`W=uNZ>~SEEmB`#C95XBjcxdzNv=)}YnHIA zW7CFXer(k4<@yG(?JwI&v7~G$Nu<dV;;EP+4*AY0u^d)Olza^7J#BJhjI}bj3UN2O zeRjg><;pSfRXbQaglwHv$9d;wW_Ofjt4i*g(->NDYa7=ucib(bRI0v7g&a|1QyhL! ze5kwQ)^FJ7hLWycK{!3&l<oRhtw?DGmqtdcvc99o^)-;xVLg@Jm?d$DV;w_1GCr;( zj2f*-c6jjSfZ=*=drwrB<pwvHbm!j70k}oz7fdTQ^kQOH^Tyr&{(gWPkSw=xKO`e4 zQKQj*@dBasR)1R(b`KNsVTK+#sT11QYMMfN4D6;QD|(YIBXCFrVO5)7Qa!zQx&kYS z0`Sbf`Wd5xL;(VeLEGr;lMmt5TDkF+ov#F5EZKm1FeznONOnvk%@Y7;@l@*eo$C@n z|BV5$vLS16tG0tZv=(p=LAY2-H5NJ)H)cAP=e|?=K0HVkf<5TY+^B1Q2JR=WD3S*l z??0731~4A`GxL{wuU6K6bDi|>KEgwI9Cg_`LEKL2q1X4wTLg~Mi616J-Oi`DM~Ltk zo`@h(UScn=y{dJNNeiJP&Zu7ZVXa3KeHrbt)o}e7M8xznb(uXu%FGZK<fnvkN{$Gc z{pB3ktK54tHd{AT+Lhpqn)pXHRDBzf+oz=ttq$wG>US@_+pcBMHXG;ssl##m0Tyus z1J9S$0ZaC}H_!9FwHr6=?Ccy(6$yTb*z`%-R>cX(K>)Z<p6(b28UTeW_1d&L(qAtQ zQZmaZWw-D)Hk91r_&DTK6@B76NW0;5i=BNYR+&1my^V9Sy8tp2+7_AHCv=Zo6W<Tt zF`_dCfrC`0v|{xG9i(U@N#ym}F4lLDTN-&!;)bqvZ(K!K%sSvLiGf@VR7kj-@i!Wh z6?U+X?g90GA|XNgtTe)SdR9v;z42z}dh@mh;O}_+FoH}rX`hB<oR~1C&7zP63AzM< zEsDFB4hp*TFTaZx^({}o%DsJz3w)L@<<ijIrb6RxK4{fv2<c-wBkICc@#%VM4%G}N zaqBbjS{#}0b(x7o$z@cq<(lr`dYJ!8kfc^w`FToa!WTOguN|LWvpysSO^t?wjVbDy z9_Kc-x3}^6;>NZYGxTUrDKiJ@tu}(ovUKKfTdCpZQX5M6?(EEv+o#gdd~9vw(EbBN zm_fpLVXhi8WOTPeX}4(B^#*dBk)N$*eIi!{4#LLz@h2s9M6K6)I-E{>8_}EUx)Qd! zQl5Cw`AO&4PCrJ&t9iz%kGsO^`^ZQp6wt}v#RUbCU~I+eNb<3R1D~CEr^nHDJ6_fJ z-EqauN;c-SBE$q^bOOwJ8PBUPmx3iNqditRNo%F2{!1f+w>YgzANyqWegCN2nk6o& zWls~nc`23YgzyV-nj8hV*r|k<BXIH;d^nS4SjiSbl}te+xN#quHmmSHS<}<sHFbC< z4XGYCL=k{Lk4d({1u8y&-gL0CgIpc8d)rHyCbt7ZjO}da1N_iJU|RF6WQGTA36(;7 z<VKIX`QihpduCeW4MJS3s<gKvqP@1nOyg&zL#Fe?xu0phx5d+ZoUNS1D}4PyyB3d} zbk8LDjhtyZyjydjt50tVQ2KTz(9}3+^#?C>KE7+ELj^brppm5$2AISJ$vo+{__F>3 z2W3Rd=xUn|H~PwJ$Z&#!7uK~7eLL0*r_0f*m+EY-RO(H}2+zK9-PYastgaGwBG1s5 zh>D-hD#Q(;hFgi#d>EM1>{Gn(GIX2GDb@{HJ4^fWSYNvHatdF!XSSCfERG1RH}$^s z1wt3JE4Oj20DBzJQZD#6#>B(`4jEYgEdj{Fc6yS4OWg6fliE{W`@{R!L-8TYZ@d)K z=KX~6m&frM&orKr<e$8@uF`N66RU-Ka5f3P1~USRZ|c358^v7jzi13y?z~xON*QOX zb173fiNAXF5+yd93$3eR!IPnDY4$LmNSqRXU|}pZ5uq(USPGbC?)gu264`;P6=CzX z-(M_py8_%x#w%EXFJx&3V^HNp4SwRo++7zimr<{%vi>&F<*Hs!9G)0+_X=p%P3fvI ze|~(CCI8#!*RPc4`Xm54CHm9=@JbZE-e}Y;40{))EnlYdVSZUMI4JSqey^n*?lISr zP(Yn{#>h|n^Rg9XgMCwf^+GtpT-%6^OJJ^EvV&&)v{O5IfRfcn3NZAuo>a^f9e%#C z1A`cEN|vWLK%C4ALdJ?AZn`y|eGg@(Ssm96TluzoS`I_akSWp}6*M4|CiMQo5l@<3 zSQvI>0l+8cS56;r-UE@}UUj*1c8@ZK(C6CMTM<+gYi9w|EF_n$COe5mC~0v+$Da}~ z>#1~Nv>mNYPw>5T<TlQP@k@A~_};FGPF~Oj0~ub#R0SOc`m+{iAGr}EZYLELJ=IWo z>B-ftJb6HH7}Sun1Tw1VIYOGllW<?za9zdimtUFP>8}ZP`5yH(6mJ`J%nRzm7HY2< z6x_oH;Xr@2%j&_0cfC!JyMdNIBEz8ER><hs6R~dIClXjge3hjLZl2kCB}p%`%=|6D z!_d&_K?FRZErVcN(T=j^{`{-@5%#V^rul^1(#x?Bj&O38{qp~hvbPMYvfJ8*5d?IB z^dck$0cnwL5RsC0(IqI|-K9t=Eh&P8!lJty1f@&5ySwX~3-9~h``OR_-uHO_dB}BK z>zZ?pIr1FmSn1?e_VS?&U4iy0{N&O>-Dqef(}*EX=Y4G50@D|aU09aMvq=%=oDWK$ zBS+$U-`{WdHXB9dZ%^R$Ago}%#AI}MTumFN>Ubg=T$rshrR6c64E=^k6zQL^0`F{x z?btbt`fJ5)`qIVPY!P;KO<uZ63KC+ylg48FZG+;2pBVkWJXUkZ(|N23hlKY#05*n7 zpN4t=Yuzi5f#n8ZkgAUkw}Yi|{PH||%aPYYIPr1LpD4o}*+L_jbg6J8>sp&o0N6BG z7&%F*CI8@U-v|1Ua(N%O9o-VKz*;4vN(h1H3j@mV&}>m%C3DHs1c5cJY*AOd69#1$ z+>>SIr>&tb^yI(f6OciWfn2l?eO|pPNReFgS#S|-vVoZ}PI5A;`uhQGRORWrG(`)m zfxvLs<Je0JA9UOCyz<f0aAAX~`F^O=(0<_v0LjpLgT_Ep(|$Wt$Xma-Us?F#t++uX z8Tb1vzTW9q&-Hs(nNCf)?CPMaN96DJ{B8Opi+jv`u;r$W5)k#ZMXy}=c>Ptv<o+3+ zoiY~^5$QYGO5{YOH&}R<`kSc7x^`1*l^Em7L3f<#xaB@e9OMD5nwaNDgb&rkv;}%B z@)%ZiglN0?#VZ%ll3zYiaJvxb-%OS-V%0`PL*1Yy!%~?p1|1bYcjv8=GzK6o(m0QM zsh@f#86mG$LSpjOv!>I&^2xIu-OifU#5orx5E5Xhgt(9+ce!w)x8p-Cq{d;7U1eAW zSc;yB_*R8IpN6FbRY1-kU|PvwhFw$ZT5F`*Q*4Y&2T0?bVIy|6Wfl8qTT<18HmNPc z@S}3atFsA8zZ7Nya}Ra=!9K>;)WO_W@8?C;0QMJFMq385>G3EEP6Q}Jf^Sr`&;bx) zhtUlpQuX5g07md9Bm&d4@)G7gTyTY+4b<>|<eFpb?V)Wg#ml9pyL!$~(y~t{>$%Zb zSN!ndB%Kt>-_De@HtM*1m|IUEK{63Tov0JE9df<i)H{!&Wi3CcZZltL$u?nBS4c&x z))*mFo9E{(zkT^y&31du`bE_5ZXS3M7$aMKZV1u`tvq(;-{;4ZugzC;SCx0|Bafk3 z2kB$`^{8rM5ahCfzP`Tx{->VHpoqgOCnuMblr*@RTCD#abrHh(wN8`cR$)V$t(vcT zvSP7R+Q|0I;nDAhy#N7putDq$HL1gV5SnCuR3Aupm%X3_q!iP=Hk$*p8@9uSOMrJo zEMS!|=SXM^ozFwQ5)YZ=L8zKi0-5<SBt(FlBww(dgmoAUns$7==0w$e=fDWEC@Mt} zUn49*F8q*LRR7MGLY>xC*`f|Q+2te~i92ht_=-6&faErsM3&Gremx(wB-|@fb)bzZ zZ4SkJtgoQf<7q=}yJf$3PWSyj?WO4TU*+n==w6tjqh`rPN5q^xHDwYJgK&LV_Ke}t zbcjF*E8o=`YOeE_sG`D|rTx(QtH7Ts57HMHdDTfMO^h+>-IFh*kG~w&X2&v1ariV4 zu14(7X;mD{_KR2nCzejLh;^H~0-}sVm710|KT>Ge)zt-hPmn~po9<KozFk^g-e$nQ zw$vNMo2xD6?CcCoW=MKoUg%@YwGM9lUNX>MubEJWLaL3L(0pz1{rysBXpo!t3kPVy z9dJoM(NDf`7+l*TE%N~7_3fO#Z6ldxSjy<zG=|H&!oIk75@pn=rj{5%uW2)AS3}Yc zL(u{EUqHCyQC4#y8PKtqVx7aR^#;^z7;O)~1qc0@>9T_ov)@}8$2xeW#886@$$r*f z8*-Xtu!VA&CffsRP^hDp7dF=G7py8dSinHCm<7EZ!TyAt#`K(4!$?_w-+AeASiz!5 z8m;lO;O>teoKz9nDb**DTVZUgCo`$PU6=i>q%EJmQ`|}k-E1|Ykdn#st4)^TPzYnY z=;NqOCWEi(T<V>gq7~R!^qgGAX55oKh<%q$#f4Zz3R`2Hwd0A0@R$o&*k{G>3H-~y zZudXxCw`$y_$2Ny4|%uNyUe*XUUWYbI?0of@g_Cd=d#PdP^&wUYuroimKpSb67JLw zdd{Zi5jlfr2q9ay{g&-R8z#sY))uup6a?Tv9JkpA`XUUdrXBN;DH7w*f~BjjrdoIK zXbYCAAOyuM1<+U)JwA!d_i+wzWcK+OCY(T#d->`B>3#HfvQ>JOab)jwrRq9}Fo}fW zECyHYmWtYL`FM;T*)I{E(qL5YTCU$SJe#7*c2Em94q@qAS``;vQIu2HLtn%$lfM@q zzx@Qlo>30prjOD@pu_NH14R8#RMB0pWi>S7M`R`5FsF1^KDkwR-r&+EIhBK47h{x* zY<tTNz<*KvQn}MUxZt&*0_YVn<y(c<sQSynl!E|<0X{w6;geUlPlG)2xnKk)1y-+b zdV~W8fQyI1+Yl_JuaUYPpi-XO=b*iXa1(Xt@X1|v%0Z=6p7UHA=p#CVaNO@wz1j^_ z1_^Xa-_P=pFDtPR!dQjW#0r3m@6Q=R`>0otx-8X%c_$`jS6uunvg`v3#&Z;qf?iF} z*56;I-6l$t&<o2uHFD_E0QmJuO7Fz1{WMuky$Y&m^AZy=le3H@0a-bLa1d&43lZz> zg$_KNeji22dVhL<HmkqUl409(d2OmTwm!yTHb#R8Bz642cO#`xlVQ0FQcPVdzkK<O z7yC`YwtL3QAINK;nZ$9X-#u1e(cSRx%dw$4I^X$1&Zs8Vf6CUXOInES-wUJiV%3CT z<{1>wOV#h+i3_=8N!QBE%EH~IBf0LT2JhC#4t0bEE<%V2h3LLk>HeTHiU5x5UV@Ew z&6M!K9+X~%egL%YJ#elA?TU_>7Tf%H^O2MMoWLf9rma^%S{NPnF^K5KidOUdRx3~K z>X;Y0`IrT1$T1AQ<nijH7K%C$rzrYnD3(TDB9f$o>x?5il!ak+1X_osD{CD6_RtpC z#paGXQTdgt=(^nd52$U^?z#}PPCk&JI60?`Vs?F9esJPECo8;fn5Fb6reJe(iY4cT zC*C{wvD@(Efl(ywac-=g0;n&cRt76RTG+hhkC<Aa-c{#$-f|Ti8Hv}VPmm<`Fo|oB zPwhRGljzR<w-D7_RzkfpoPF=Z!0M50nxmPxIYyjm;n@{{jMY+fwt485{T6H=q5I<s z8Q_vSJ#noc0SP!xVh$<n#Kbl)11&`7aqGF1goJ-aHA1{PTLGvWY^GG%;H<b~?FTV> zg=pL;7266O=I(rT|3%@TvMjwAg<?8NNs)De2-yKzJBo)-cIoY)fv6}47&r;*j5D_1 zHWioaBls;!3mZ;8e54YRy8rVE1(D0Ih6rD2S`5V+QeIhm>!1p)F<u=N4zu2wwA6q4 zj>syED}nPavNK4lL9^C8YiUK?%AC(oO5JDJ#OITz50m?Umq%^n9ek?5@V(BrejDv> z<}v9#@vU6d^W#BF_!RFf<73eL)!|eK>WE<P!cSkqV0^|%?<dqnlkDHzxf=7~&0$pt zblKDL_8G_8^Ir>@BYiNpY_NKD3l6^m-bqUxX=}*&6KNe(1Gb<hz6Vy?eFEYuj5xUz zbEEp+A*HlutBi^pf#h{fC5>KB1|L7h5E%b)WvFnoi}%|%tO*#w9?q~Lngm{}i9I4H z8e^q==2-ipQA+toEH1qmPTR?;52vhkTj)ARvLE#^!TQKz_P3sapG>VSjH?rV6Thj9 zrwt~NV|Ic`l2_b@JGDlu<|bUvtoIYa#u}?L<3`{m)JI?n;Oq+SptK!${C?$)VG2Qj zU_ewq6kmv49|@4d3jz_2Z>guYC4aGoSe%+VocE*&c#(`AX6J3jmJ8c!tlz7Oq7qEN z&zd&x_O(&FH9&)s4raB5Sh~voI@k1wADxcCf-%%jR<l|$=ua<s%Z(k$H+g#Q)c}V( z!~H7x<)PR%{YS6W!jL(pDUd81zQ0+Dtd(x;g{QZ4^o5G(qZ%ucn(sGl|ET0Xa1gI4 zKsNv_Ep0F>aZ?tK$wyQm!b>Ry$e&kc+Lb)oi`=Fp(~hWLfbOM-08(FU#B>z-C0lf3 zp17AoVi9z}8Ysvp0|l0U&(HeV2UQ(Kr46B19qzIn4&ATZ@{se(Utszq6G}KkF+O=_ ztXw~7g{54PFM94yG8oaLni^^~n?Sf1csIH;Dw1$j=@E3?{mnqi<wNC^hh~eLSQV`N zV#5oMNZI{g#$GO)xD~ZxqaO?!%^RNbM<E5fM-MdATp-7g?0%No#S%IXMx`SZk>W4O zw+RjzEuXV5W3!d$Qh89dfFJZd0GJ@)^cyc|q~|oDVj#3RC!6Z)vyIomStiu0+ILe# zMkWLkP=U7nnVCH^FH4<_8Ju%1_IkdzJx1?&*(JB~<+({$=1N^(pJ!TXU4)XJ{GbT+ z+O_!L^||`wWnpi9)ArQ3M;DLd2h^|C-u;dRKRzQW2>yoYtJidsj#VXgF$5zHcD`pq z{Q+UJ9Q8PQOWoHiDyE_@N2fCcgemomek`XuZ$({@o_ka~du|_%BFpA&0z>vKqjnKu z^o12FWZz$egDG`&ufK?-c3Rj|1K=%}uM{fHD<GVp)92~~7MM%Azp}Q5r?+)Mba<Zh z?nWg&|7yy-GTz0WT2eA`!U(i4<G0`!<1m~Jtu*DyG|*l-lYg%u+#ytc@{r~|CzVEP zEARltzOK#t)D74NZ(9cbO5{qUw7v#v8bw6t7c|aKC^uC@J!w9Z1<^jQx&b&-kO~M8 zJ%@bUB(@aqUT_ftx3h|Zy{*snfiBtSCGqY#2kX?40OrC)l5qax!55^hwb>#2Ox#}% zDk8t>@1p#K0nZmJd)@k3L~r~$qoNBkCzsO%=jDd_^6OvA_J~%*CpJ*^cS)9ML5Lj9 zSp`Hj8JFcu^<W+&)x4fm=lZS5YVXQ7JqgAuoGdc(EDEAgcHyG^HL0$lh{<`H>Nbg^ z4<EP4)106!SW|fBmp%*^TeXe5#w{OF0Yap5dYivs;O4mYhSL?U7$<Et6<|IJZP{3z zB%n6nH9<?o7JkfSI-l*9IJOsa&|B_CWTfTtT7Tl!&W1w9MgFlD@BDPe8)q5Gqep7m z`_T?<4D_lO<tNWaycXxsUkr%QTwaFR()exMtyr&5*SAa_nq};b&IahkJrxE300Mk% zNTYtKW+fh^d#4^RcNHCfD=ur3w9nDX4%Lz}M;V*eG*MH5nyk1IijVPC0>=IuWk;xq zQ#G*2G<kl{&@=5F8tb4~2Lf!sb16u}aj!m+eY|Gh5h^dt5v#|iG&vttmJKHB&eJ78 zTZE}D?~3`_TA1L;we*t(Og0h3_yJ0lfn$J&LmKfxq_MXd6~`Ajdei|<b<K?q1Lfij zI_M%pd%sjxwu69AzK6<rLatkd(Xp=CDP7(UdN;>tu9L?Krwmt)iElh~sl{8~9w7EV z;Th3S3dkBBdRITI#c@4Fz3})oT3AD@{4%5<_Z25m1;j!b1P)q-T*hs|Y#(yT@pP`| zSKcQ}ba;_ZI!HKl1g?}RMQj>U!wgjbDjq`uxPt=^Ee#u$>~&B<QARg7#lZE#+f#Sd z8b5u)TZE4VS!Ii%Kh;)wlTKXge1&9n*8V)F8x-RDLul~i^BYR$#Z9klOn(@k=Qu7C zr4!kwPf{MU?r&Z0D#n*x6!%owd*fag?C|_-c7NmHAx(4jmUg%`)??<yUI(*hB}BIX zBYzkn?f8e8o%K>7gy9Z1=Ok-tI=TmC(y;1z3D-K(`>~B(&@y$hcZvPsk|zSY;SRH= ztH4HB`uJ>s(IjRl!1Lm@HrE#&wAh_A9_?eIaH*CDEE(v8;zRri=Fv;`Pry$gT{Z$u z*zF9J(sXiVQr3Jq++o)*yW&Fq<0{caLS;{dJ@NNwEXu6?i-oR)E^0RIg76-n26CjJ z98xoaTAZM6dS=w$uN1OP_z@-uKhMv)HJI&58?5&rvI*8}&a5csZ*y>!>`XrKX0~r= zGtwz9lDn3{Qkxp)o_;pTVBJ_QHdgg?tbkXzQB1DzxhJw5c}{d$J!yWR;??sBzupy^ z>rXn&<$aSVjsO&dR1^V+!@aXWpE*Em9-$k08{c#6MgY4COV~@0ne51td%jo5H7J8e z8#WP`8)zOfgKuHO!85#}L!`$<x>Hn2(ju2<=ncLjy$>}(e-L+88_Xs~@KS6e%|4A% zkL+^6U1?h(jntcD(!sUF;Mwc(l|gM($5S(4&I)H%HVPxE1|`qF<{&tOkT8qAue~JL zFe%}$^hUL-8kV~6uYCqoXdeu8WLF>~B#NC$o%(=ikr-if+L0#a7{g0Gpcbi3O}r*V zd=KtdMKRwc<kV~=R5Wehq0O%n2({y;jcWWu_(Envy<p9}Z^-!Bum?xgQjSz#Ikf}| z4W%BjhP&zO$z$pakx&aV!qJQD_A(|Kf#=VibfBX`&j(;Qxm`i1$zi2+`8LqUTzUg- z>NXCuohFyUdmKo7nuza~^0(pDR#N)PO=#l|k|LjamJOw!YH1}7HI9sGh$q=2z=_Oe zXh{l+9SRMZfRYF69%2EnwHFxmLH+g9r%zdw#GQwPDOD6mMoJ(Vzd>999P7iJdqC(; zFz<*5e54A(Ne|BW_|*}yA_pDR<>$ipaZXoiq;^ju<}j*4*fv<}9_59xt+AfUA6Gq? z%n%8*s9`QLHF>?>Lk(+AtM<a5+^=4#V@kdHY%&vL*mErgT3YU7OeGPSA4DbJYVSj6 z5z#e(x;7<bu|X!hQDq2N8`GF5{lMhp=Kd+?<!#W?G+B^|f*?wHLbkgkOCF*GsDn8N zq>rAxQY9CDIgHXkAS2u1!hrnkr-~g2-J=i|;p!B0(*QOOMRW|zS30sL3<%-o)In_} zawgL2zbwqLws*{}O!r)sV8O<=s5r{0!ky-QvPNoV=qm_q>eFH~9WQzgL2K_pS};L` z##B0fcc&m3t@FF;4h6tJgaGc2Byo^d(9+U+M8Iz|BMJ^7v^6&;2eX(0JdeapHRYXU zDD4Z5Nm)5^Ma(awYC}2=G=z?X@q1+Eq5CiZ{7^K}!`?yA{H4!egG<QuRu$o41K+3j zaH_TKiof7AuQ*KG^Z6uYo$m2@l+0^p6W6n`Jn`ywY)ZQ-#E*otib+;M0H+KyKVTTN zK~*jB1;hi#E`o#iYLrH5f2Qsuo<ic9(~0AciGGF6$2|-}0wE*>5l|a28Ev7-PNz_O z^m`=ToDS;U(OH$Ckd>#Y^MagQ>K-@+QXhaW&e&!=S&8f}XEVF(@cFiCJQOA%?tu5m zVS({ep?;G_E@dr+c@C|JNBioZ^Sn*IN*^vpc@K%IqcHWJ!$uAdWCl>fA|Y)^))1+* zMba^cl&~~QfX@((F_a)RK|vf1$PG&nm9hzm9JKR&<oTi@az@H;gq#$NDi<cV!LKOi zwGobi5CNZ;pv#TFmIb(?AQgy?MFzBpsag;x)|xa(rrihMTu#6UluN-1f-4M(3U{s+ zEZt!RjJVIrD%><h2bUH}BNS?xat|Tan3WJ=ei3B{v6CTR7FT0Rl6<uo(H`~3MZ@a7 zyq3MY!KvzxMG?QfzzCA-!|N;z46nQu+rUDpNOw;?5_DKLj#}!9T+hq|SWxlK0DsX| zV=#B`)m+1+@8qX&b|5<Kk3lH$Xv!Z5!@b1I+fTsRBemNB@v#K7C5h>PK}FMbYqxUR z?2D_>CG2?z-U=zkeR@4&MmLtR85$9rc5*vt4NXs~j0RYbC^aB@0%RS7<kT-qzgd<M zNUfyZLaIQQB^4i#rfD-ZH#ZDe-9H7{8yOiHJjc;u>*m>lj7CIQST+l)F}qi?<g$<g zf+rElKAyUuh(^J^wzf9rCgo04wwU`DNI4`yEE@qUHwDg<cd_}YM%x$u;?**aHOZr4 zX)3taTr0~qTkzoK0}-=!E;}fz=W{^6GvfrlY^a(0oxR$)HA8r>BnpxZHo{!>Uz5?3 zVbL>YGzgc>q9N|mAIE_#k&d}}=<{&T{4R*`;tx6f*W5sgA;^*CYX;(Io~el~uj^q8 zxjk>@ng(6J@kV}tFO0*>GiNSs1@|R`k8(t|%_IjukO@pGGAyR|W`Y!9J<Dmq0WHc@ zU&GLlW+-m_mKL%Aa(56MyS+>K_U%WsBH{~fTEG`Z2H!n>P)ZU(Te;!e*dl$Yr~{_t z;Jq?6G$BWbr26ESP>JRDdv#Nf8{hF^adOUmI5Umqs@;7lxp|~+$a1>-lr<`Q23AR? zz7B*<^m%F-LBTaG^CP)8f3N?jB-rScTWE8)k6UKL`e6dEJos^>kucym=`1iR_PFU+ zC5t`<Hvy@)b?vB^8i8XeJQF&^XDw=ZVRPs&xpgJJFY@K?_BfuUXb|!vmdaqy^P|g< zvwKS5JsSQ7b0|mk@6w*Ui$T8Y$%%w?tdEG42p4&QyB(~Bw#kCH1S|9iVv3M#(My+^ zLpM2|t~8+YkyZ@FqfAf{>jY8f-pm>MvWdBaw&Ty2B1Za`Cm3t=p4C@NU+-b-P9Dr% zN>01;)fAVUE8bhf7rl%eU)l2I2M`J*4TR|#ffhNgt2XJI{Br}k%lZhIf7uQ$I_Wcm zr)JC$RAmq~-=1Za*U<oK@ogI<y}%V>0Q>U|=sX?B@Au_-{B^6f^NCa2eeSi4+e#YR zZ3JT@l5IS7M0t<QrVHyP)%qOuC68w(3(@qL)tR_ZSHiIDs=@1#Qv6eS(gD2%U*pIh z$ty|(L@(K@L@JVfYyhtW6kS$Y!QI3}NqUeffl4^n=(0J2?h_=@Z?hh1VmdlW_ef~r z>%e5<T9^}ep@M4(T}f`A4|wtt@p{FtF7;8Dawqa1gvdpuzjjv^`2*bO_n!9H#iLgn zFQ~0m8v%fll;7gnn?<ZO_5N3*d!GO@$2cIImgp16b49JLttk^@0JkUeEkZ*+fdo`T z)ZNy1U7n09Q#~+5B@qLZe>LKw7Zcmqh!T%*Nh3qI-4uve%h>YEJJ&qlusD@{q`PT5 zNAQjzA>DuZ^5sq*PFS1a*^4L!Kur0Q1ywobu<xa+YQClpbI6-*r!6-?e1Xzk{X+Gq zMq|qcn8Nn@c3=|^VqdF;l$X4yvm2rOjc#=C^i2R&H3aZp89Ba`_W_B@CuWAXx)HvS zJ~fsz^pVlCUsAHZ(QinY+F$+%P5vQpyQK%Ly*xKfA8_QkN+Q^q@=Hra9=aD)x^iM5 z8hiZ!p&{6|)XaM``==)JSy@>R0Eg#jpCrxr^<tx%5~!8-mEH2HfRNqUZtOdi04XpE zM9tzC3BaJbeOhEv#X?{u%lrXXRQlrW{%XZVI}8owJRf+H!;|3q;F`C^?<*s8npMls zJY@Ey*9F&Ho<x~>l3230UF6w8<;juP^ogsPl?Z+YCP<KrUT7=*Xwa_65O$B0#M`ib zAa@JVN;iYZ^dLn)S`tpdqQ43?1sPfm4HC;au_Sv9c~eu8@&$AF_cPDZp;x?JFhB!X z*f@M3x3I@V&;K2R#`G(4y-3Ot{v)*WRwl`dpn-~!6K%Aty!FGFpBS&jE0LSN_5TF= zrLdZEpK3&mnXa^4rtN-4O!Gmd$Zau=%(TN@y3BcANAtK--rq1Bkir{L*EMa`Rn!S1 z{8FTG?8QgfU&bi`MeoKp-HX;HEwX!xR|x^tdXDTg(Sv&?tPKLsCj?MZ_8*u&JjX_~ zGd2RG2B3BE6`BYkr!KLl<b|w#?$89T(I8E!S(NX+AIP8<4~loD_1Qkxgz}(fruP~! zG#n!Mxdw-W&NP1k%*Iqo9V@w@wr3%WR-JNf`tOWv&#!x|@DIsUQB}1Ltik0_+q|3% z4rVUObcIs@x`@-nYSz+cK;&-cFTn;9W|V_-2vH}UPUQ^#F1V0~JX(^}_~sP965=Nm z^t$o0>*ckr1ZnURGV@Q+#TLGa5re#*Vn%-s5Y&hzh>HIY8Z`e~AYh1zA^d&Y(ZNBH zCqVw$hEw5+99Ha8&r!`l1AP}zNf%T)K1X48B&x5y2o8!to99#$oAr}8s)ppOW67fE z@`@G9O5`Xb+Wq|a>_@q1ykVff)_JE;rA-8r?j&dw*}}E8_1h!H24tLjw>Pv}{oyC7 z<Sf@*z?I}!T;L@Hv?hJg(a}5OWYvpG0a2ADTm$sypiv5f06Bz`gm8dPrMghp^&jU- zh_e&3ara$??rlmdo%&S8xtr0+kbLf5sep9))${33vWhC-N&Iz%grJaMwTBj<l>=u5 zC@)Z$;*UA_o3ISH@1mapQI8A8GlIhh;)qHLmfx9KDbDU>=!rYF*P=T%WZXC63qcv~ zB^+%->oC+WNC#D#$HeA5=&4-EW44mG$ZQ`=mOiw5pW~wri9domnGlEP@u$}k6rJ07 zgEmotBC4r#;h-|eC_Ky)3cAKOs24B-t!D#?)}piS7iTb9psy9jI&cSk0L;8EKBm<2 zfwCde+jaR^fja?zfjmk}6#>Y@aG#um_VX$u4*C8KU6(r6<~}|2qSij9V5hz->9()s zuL*5l+=M*j&^Y?Fo>e13BJPS9$Db8$^Ow%hg4|Wfj@l*guXJblY1QRKb@snnU6&YV z@O41J0Vvh<+}w|6UqGBFqqvYfzkljd-Z55cp4DtiI5?aI3)FcSG$o0B#DbSetj`MQ zJ2xs!U)W(|$ifk78r^x+8(Pa}K~naR@{F9{;<lFhi|?%uj9p*GEndt$E;ev={CqYp z;yQCB3gk1G`rcgAhP4-!T~D+}+>0px2bN_Y0M%<MnNQ8H1%^rwJwR1OfD(0&ELfP~ zsDAHvi%OUC+F{IkTmDqaTnWd|V&eAoJ$(naiIbiO@~;|jkF9EQp7z@j7_earSnmZG z&?j^s#B3lPo{zb7#!m(Tc8hQCFZtxlKC+L<wb}lKNK4ury!R#n!Fc4GhwN`COj_gx zdYSIGoZV*5?@N|tWvdrW>it>Z=(^MzMR0lh<9Ehdtrh8_7Jo-PPPqe+f&2R5^Aa+e z{i`pezfiFWDj#Uce$aSfJ^I9mcHZD&@JEGYc3J!O<bYiHk8hiPszv9kZ0L0Fwr-); z$+5c*R?4456unLcF!;`$%?{Uc7n#M7mb*~MCLnPAL*8H)pWWcwv%=8$WhToS#2zVK zyYq2xI6`@QpYl?sb9}9n%6*!1^<D3gvWKbYi#XS@=_sk=EHkPzYy!O?JoF%_c(*|v z3u`{bov>;WyHEILknwy~W@abaM-Q=U>CaUAf|{|-hSm(&U4-%UKzKS*`ZZj=T6sfM zNp{2h5ev(JQ2ehgOkG>COHv)ckJTL}>}s~<Q<rv5)@PD>o%g{H-~qc(Td)@?AK}i8 zs_M~SJQR54b#EOMz*U;B$mkzMGl85M7~Ouf!J}ou_ab=-w8X4to~96l04U(<4DglF zf2b-cHH!a=<O>5eOVgRGkDm*6(qSCsLnLKHa-D*b0}8z@@2EA(%-efE55gLV6d&3$ zEH(-9d={kTQ5vypYTeT~R7+5y{zl+a<B|G8uXBK_SvCC!MD5*GPx`)jaP`3T15SmI zxnH=m({8nkb4r1~o*Lc)9p<1TRupHUXcrG%IF0l>g6Hf8edOTeq+A}|0>XDNmX?e` z%+p`fKMpTjGE}1Fwt8o$gqm-vtY3Xj{#LHOY5fTv%5;K({*+6V@M6nJdCe{YwRI%< zs0yV;^UPwlaQ=JHm%FDc4o7*Q9&-HV5r3I{PGXw+hhr*Y2k4`<i;nS^?_x8TO~6f> z3%VcPiPJFsUa1}{a>Y$b#;p`Whw*CtxN=3W-(x;ui0BC#0`n?mM|M?3-@Lp(S~lH@ z?hiZxJAc5-#S{D2cG69)7q_s3UPKum6Ni|~mJ3t80YOV(o|85iuVPURThznBHu7XY zqRdRjVDdl`3^P=swR>8Sk5_pc%_P1?Mdrs_&wX2R)AzaU=+II2b1l!k%HfHbEeEeV z{u!bhzZJCw$#w-FjycM3Uv~QZBdVhu8Ex|s$?r#_*d3V}1|}w6c$Nc4Ob7tE47NOs zTbccusuSa`MDHC!&7gB}<!%e=&vMMDLfhfhn!w<)u&@|r3FnJa(H$|rTB=2|<3cV> zsBMBZC?9|V0TWa9DB7qI{+r1;O;GpP&$>6NDfpA<T=*nF<z3`^<`1CTXjFIM`i<C= z-F7x$TCd&(S;pjNQ1r#3ECsMRJN}gHLKzSw%;)jyZ^<9|b|QaTHAivUSmaq)kDo+& z$b@V0s?|FEuAc5?e?4z!zW3rIz>2$~1vLzq6Th5;G`@!2J1L*oi;TE|-AB#?zJk}| z=ONs-IT6Bb0H7qp_*?xC(O(w~hrmt?{R?XdATy%%!Is}YkmN+qtZmN%WUzNNb-jG~ z#qcG2TW`zJ`wl&uhEciYx5{A7EeF}XV&7C=cyFO6r17dW?jks=_Vb4jVIMEL`MEs{ zU(iIq2qSPO(eH#&dAF9)TytIw6Y#WpE1m|-|8{j%(U$hU!)|w4^D~(Zupt@Si8|7C z%QN7QC#*=dZ;+e~{z1Y1v4<Pe_0t1y*9FTS0oU6Gb#v1y$zXt><w491X(k!^;i4&Y z?oQrS<Xi}q0o(f~n4%_}OkZ_j>dak<Ifr<ku<e9PM2)D5KyM&c1XtigWZmnY^3MmJ zxLrPPo)IBS3hvLv)w-Y!lE0cOUwaEJXUNN5;d`R4=_8ThS~hh4q=>6aT=Ci5F?vg^ zNQ;_H<P{EC-&LxYO3ONninZWALVLV@*oM2T`QzaoBfF9>e|8uSD8stm@FTRfX`UaT z(xgz3kU(|h<p%8yGDtnREn?<ZU#oaE;CaX-o@x$mcN6FgT=p7@a}eyE@5gI3xt(b$ zZ3?Wdc;A?@^4z+8ZlnwX2qG2t7}C85J68|q#lBML_F$JQ76zZ>E`1v`j%@uV-`wg0 z<qk}bw@6)o)_DnaCvcY~XnoM4^2<wRmudbJ4F~ogT$2w~yRE`(yo|Xy)4${1BS9WN z`t5Juv%L}s$dR8FtKDCf?%OWHJ#LkpYi$@DU!D}N*?x;1({Z={O5hg2chzWa`92Gr zq&C1j1DFKJtspNNTRl+8q`Me2rKo1e^DCVHR_OA&aQdkdP}EuMTZ}jgeq*o{Z$*-u z_MDccc3#tOVqa$dYCZ(sC<w&Dz9U>8vP2+~rKG!sdYvCmq;khdE&TzL)VA7p>mXpy z2<(?>@$&PCHr7!TzXz;o^JWSNvVFPjSYp5kclCK5X=_WwkP*-Sir$ZufIvW|2S!Da zWi(k`M|&(!cf$Xj)0Cth6Ody3i*`sdj3T2B%(@V@OGr%28kd!6`4XFAPPv==_OI<o z__TvxJCG7&3!vJAtyidJ!A}9YAWIE**)tlOUa9S!EkHLU!~njK)~Hy$6k=gDhY(P) zcj=Kzjrv*6jK=(hdcGJq*n+0cVQyZx$qAygO=NbBUi8*zf|whr`**I^Yz7DAKhyyr zz-TZI(SO(c>JRXI(5k{+bXPwg%tG_ermDRy7X|E@YaNp9Cr-Opm$R-(4{X>;6a<iV zhn0w;?i>O1kP}GaejO{Y3eFbwPOa6S&U)0c{%koq*ci}fs?7j;F@UN2&U)t;ab@vb zJn7eewXe67bB~XWWj5QA4Gw>U1+tN?)R!ik6sX##>%H?ff6e-n6Y$=RihAzCE=q6N z?#WKQNtsFmrwu}E2<6+S0AC`l@Xe?pXpMTMnI5B8&}m*R6xs8J+Xn%5T(|iZ`04SX z9HxPFXjL_O9&!W7n`x+%j@kj1#J^hPA0g`Oc*}gLSZ%_<)NVfiOdd8YVu;7V)>=^t z1yw8Y;m^-#FNXrtCO51#=2j}Iy6X-2pVCwoROb1;jM_~%+y|Ba5qf4Z`4V##fJ(Ex z7lb9Z9Nh$);YTq0{g%36_M<t%{;yr3n*JjgDambg3HxCaF#Q^O3HJM4Wo+GgROTS? zvMs!-hK%T*)zL?Acs}=<_47BdjzxSEjP;`c(gFI+M?F3W0Y}&#k;X{lS_|FPuQMpy zp?P={42c)CJY3|d>FUap%YPI7GXJlp+YOCC^!Y<KfE2?9UY*t~nu#aF8aiNnlF|%r zaB*x%e>=J+V_~|`1d7qh_oz&}uE8P9so8z|9IMyBZMB67C^X_Q5S{{{adar@7jYC* z{E@lOdnmGM@h?t{e|Q&isaH3A$ZU~xJ*1l|vK!iYtwfAP0eGme2VdGg48gogfQKVs zjzC7dAZ0ibez^7tpbZ~xmds&XZ2byKIDgUsqxzlqc#dc`oyRp@-B)xnk&aq7a}ADm z;YZY?umsvFs!(<%egLM?HJc}>dx)l@_7tBiUdC8KkQ#Toj}91OEL%^Q_6%g+fSWxq zl+=%WHe2}N`DbOth_0aJMtxR`Nz6OXk#T@5X~x2{MJK3aWm~@lKNE3w1DlOcX2MEU z_x`KB3}8zk2aEpGy@Ael=NA&W(6h^JzL5VyjF62Zz~J!ybE~!vJ6Yl&I9eyq$1G~T zSC4DNKtANc`)KR7ApA`C;`;h3dHL%4_*l@}&E36vWb81e%34%MQCx8z&q?XlN7CDh z1b1;!qn~MsNn7Zs`!P|8S@zM51SjCppgH=9edL#ZXxa5XVT#*Fj3j0P<2s-0@@%m) zlB2oPqnmHvODH*jzVq{<z;dUCNb>2Iz;bu2Vt7MMhv9nulR_}$7fb4-iF4aOo{KsN z26Z5H7HyigJL$R`s}A!`NTQI<O)=4XI4+m5bL-$^Vg=&^K}k37)q7wZ2HbA}zrK_H zj-L&&6r-AsYyvhPsEkBf$)8Cj`+oaW2k9XH@?Ah=ps-|P{UYxa=e8$1^vjGw1)S>W z?QG}IBiCDaEEraR?be9j*L-$C71}b6R2&r)i1Ba=?q@=d=RjW#Z^%I^5YlKiIEp8_ zWAQt`O+Ec3)Fl**Igj>yaEj07w<8ATncbJB{EpPXuzH#CP;K(A-GkL_BS`%4a%J;A z^%A^m+IyVU`8<d5@pnCf)xGg(Ffuyc+9{6F$~+`IirZT&Mel>{{7<}UIMt8*SD#-( z-?91D@i#CB^IA2*P|pSqD~*QGKk&YgSmYm<QOJR(ocAga73$p^Cjw;+mf9Gask?jk zz)$li!ou<9Voz_2;b5UiFT%}#g1o^b@nN}xeZS-xCZ}g-r$Y%wa3Jlx5vtE-5zh}t z)siopEQRMUIq1iiOBFtZw3)>6-POjRFNaf2p^a4O-l;zg2ZqLjl|6wxt+6lW`>7bU zYPP^CXA^g7Zq(ELTw<1Vc!Ry57A$3B?1A@|1kRVx-tYaO$1Orw{+5qQ|GBJmUKRer z)Bfo$0_EZ6B{_(e<mH<6cy(DHVn5glEXy^=#`p)lyR<m&zJxkZCRk&!X)97TI#s`( zILR_2&@w-K)=8?cL`GwjWvB*S>9b6N#jt3-y6T-SYcQOMEu8EKR1MYFlDiB3cy5Ke zZTrsKzO!-g*N_fZ9Fx#29Qjb<2~mP3uWyx`5E~c6{^X7pH(_nerRVT-gb=-iroTJ+ zBn+nfHj64(u4zVsxzQZ>r)5~PoW%pYo_(+DGTBJGI${D_Z*{Zvo>6K#DJq>Fa?E29 zzLK<p6Q;qbo|AT(&+ZhVai!Lo6g@15rwCky*xH*eWA&V$4I^8Uqx$P9!0U_Q$hA9N ztp-}(%N9-Rpyv#j?5x>C-+%j~nmHM<zuT1I_(7ckwz5VWmh#SRL81OmQ(B>UzyaFl zl<w|skU33AxPKVnpfyM1`577Md-`!~q_!h{yMKCpVirtI&3X!=^z|z7lhBVNgBGm^ zo~mFg$BfgslB7@l*HXt4>f&vI;7|$H=F;Wmoi0N^eIOMXauR58VoR|Ywc|?jm^Wrx z8me<`gI_l$w`V(d-7)|}?bEWkGmkPWwd9^OAKBw;ZE1JOUuFle6Gia(oEiXWEG7ZF zesJ1xwPW4gsKEb2oBX%&@Ub>t9^;+s4O>qoma2dk#VRt_5mS0UB-ic-!2P0e>PK#_ zE5*W6wO?LU3=p4V0|&SK_*2a!K5o5|=RM%Yve6z;&c++<+Zm1o3XY>{E{-`ay{6Gk z;j7#UQiAn*n$%wg5ZJfbvrHuU!TnrF^_d9Et*|oh)aN<$Ti6$7N^9)js~6;?e{+66 zQzKicvwJ5s_s*eo*V?Ud%a2!jL(;r@H<!2#T;kef?_GVD-KfWYJBk+5XRRF11Z8#j z)R$Y<7@t<L?EY^#j?T``E`Q_KtlS#GQ7R9IRMl5t6!89NIiz(kfT2#ls-13&Ypt2{ z?-u#CNf?BcL<6P+;#eMB=`F0*TfN`33UzIJ7I2y!JAz>kkPrp_$@(zgx-Q%_o(|#I z8p{X@d6?L7qr21WabhR$iF<52Eozl_H}Ns$rI?#H31yZ!862Yg^dd5u>Q2)um+-`G zNY#zhdULoph5Q!LIR08Xw)C?;{#}`l_-aVk4jftQ(}fz5gC3D?Tb-!NHvg$fVK$@_ zvqYn0a&soQlbK(^4dxq#Hj;Nv&DwK?jv>(b1-nxzm1|s;T3=p9&o+RqCr}SX$C<)V zqU9<bmrcTuSs4xyY&(ac#?&TLo@^weJ9$uhb3ez)yI1O)+4Z`o8|D03%`HTTIgej5 zEabozg7i6ZN>65QONmYh$m?nftIv?X)rmw=6%`wU&ctuR*`T+Rq1lis>~B%c^j25x zY3GmO5Qp<-mF_Z)l?Tn+$O{UrnDrPqyFYxt=x@ON0@K!yR)(9mm)^w+$BBGd5k7g` zx1;r7HhD182If239vJp{kUDQ7aQbs|4&zE_T`CO4V7g$SIMnvKj}ly+-3b1(8(lek zh0vAFZznuzm*8beuo-J3Ya85rPtJWiX>1ZN$8X6<#Jjtenl6(*o5*r<qo=5v8)%Z* zzT5&M#a#WTr8bOk$efu^hTUe>PRBx3RhQ9FV6#zx!rII{R^kw?eaRw;?VHt*ff1&s zIz-uQZ!JUFx8hs=SP(rh4DU6jT2fk@#2GItlDzab8?Ec{qbLfXD@&A#%GZ8r>->fX z^4y)Icf6Hc6{Pj@Ja;vlM1tu(Snr8=fVdOoMn|Jr*?roh@V@8eMlYRLR9Fmcic$_9 zdzOj3Iy?F}s_~T5jiL2W8lP~R$gy9oeD?G7*#4u{%QqgGj62Xj6K$J!sCe}9E8&(e zp(zSyMYMV5F%_)6%Fd;v;@WBz40viN2Mqz<yP5Qh7b@%dVg?<IK_7cW`_DUWg9&sE zhCnk3x{b^Jvx3Ss`EU$rZ=PEuXIZz{Dg0*co~g0bd0&W?4~M={9_B^p+do5s|6O>I zZ1})e`5|V!vV&HTwO?X75}6|D<9L*-^8`q|w6*oo3vdB3)`ypqw_8>ysDv)@y(d?n z(jBlc)Yg9p8|1=*p)x$UnVxGy{4do4niAyQyyL0BPE*OHhp8aVL?2u!M`mx`t_(kL z#>YvcJYJdHf*kCtaC}Yk+v*gK;gf`k$^4lmOokTu3pN}Hrlk6GfV06zOe#$e`bV1W zhLWg=BCo8FBlqZujzr9Me8l(~^@|;}@RT$vS?&25R{&LJy#2qemnuu!kp(f#w&L@_ z)e|xBV$lSt|8yJZs2GSXSP{44_k+psmaMSH`I*nEpr+Kd^#^E(n}0L6c=mDl%NynS ze_7$p#dAaXx6Qe6laxW7gq6Ow{sj)0%ok4A-^PGsBje`H@Dt|jr!93>efLHhYkd*= zS^*#Lwiqww!%x|Hv{|5B6dN_AM;8cY+)Cm9@3XlvOjDx&&wF^|DOjh#sc8R9O8(=% zNP}~0SOoX=E@Cog|Fge^tTkMFH42>HU#ml}``1^3!O2!so@kJp1>7827^o=Z@Jn4j zn2b63XSazDM-P)?MX9JJ$#@WFi1_`E6Dghf8?9ja)Q{>;z^s^69g`HymlU{?e;)WF z@xZ~%u{UtM`#)Y72YS6dm*4C!b@g}I(yVUN-3aY}mCTpW3f}{gM>h}L9Dv{t*z$_) zKZb_=<HwKjuFI`_|HrrFnUR9Co3Z=nB5FH^QV{)pA_OX0TG`m_@ik-y;r}d85jMm0 zXH@#1wXGmD-oFWg;5Rt@S6X@PKNsbFyoM5p1~Rl3p7&47-H6CP2Iwy_6m0KuM$KZh zL0RQcym{c}vbnqfT-BDAxm7MZ6JON+KDIX?d%HHy>MJT*#m7nuZt{(^>_$}m`1-Gd zN9ba;a5yqgxG)Op%>(~&_;2FEL`*C#KUsy!BGk-{&;7^o{c~)->8bFBd+9x*H-Ea( zDL$P4Z0;=~A)zIq6vammC#hrqK6%;yDR~nAdFDu_)PFl&$oKExce(Jz|9e0FIFNrS zG^(u3|J;dxshua2ou7jvePtW|IoJQV&;Ls<{&g|qtpJ)~;U(Gs`d|{Wp-<3ajAhNf zfkKHIA?ctcInu)bWlF^wp?CSn(*LKGm=2ANnN3}S%6z0%MD~AfWfNv-YsT@Rh*<%Y zlB<qoih{ndbazvZO--Y3fZ{8jfr{g&E5$XGV4k=g*#F<MNz&6DY#TeNp%SgM3`t5_ zT5UC{cwA|)MRR3-nuDxVyQI>7SV^g~6i?eexa6w<(h01|!DP()_h_qxl}Z1);eYg3 z=M3*>xS!k*3?q11AyJUV`f1%+T5648hTNv<bkWJwIT|Lv0?$bBsK0$V#h4u&5=ej+ z8T_B3<a2yL@o+M@0t9SN+AHp=<hrwJjHV56&_LI-dD0zpHMO)PQ}T4X<Ru=DM1P5| z(vx1rt7TPZ4p0D=gl<jI!2JlMY9g(o$N%25HxtkDY%>|9?t($%Xp+DO)FPdx;JdKX za?+D<I?Bw=THVoKEDC5|zw3ZW7PsG06mZz!2Satm-^)GyR`tUV2gBNKQ%4LAnGik0 zB)Cy(Hwu<w0G!Ze-pjvLSIq-w7hTmOVkWvpB0BDf5*QSdI67^6J35+GdbK7R?CXz@ z$!67y=`rdq3U}Xh#H_ax-?7-T|J5BZ{FX1sFJ2<x(05)8CT0iB*+6<(+3%X|fzj2W zakEu~UH#A7NJ0hzG7C!&0Ri{HVUkO+<$Iiamni5<4{Hk=EYkV(#=YJ7>pbrJ>BAb^ zaq)pe;d<kZD#K>b`i)a=Ozn-4+aaEWm7a<J@ugiG&erZNB}J>CEj|4Gsa8r4wNc-G z%_yCYLRUfmeeSq7N9$HPKI(FFn~AtfCO7?Tvw-+tA3QthJk4c&#>vGcUEWR#{K!8) zA^eY1@%Swbuvp;q9jU#6d>$I?fBPt2&&Z{&C@2!dl9QM7I6rRh$MSmoQHgcdBR$J5 z=L~@}#$Iwppcu!27c>n8!N~vA95d{Hxv;-bAV!3v{r$7`P0rMoz|BWL$9}}16WQNJ zdHZdb#;Bo%EZxPYj5kaNkATIutn&gc%ZuEpDP1i#)V2Elk9(N^MVoidNTt9Vq(0|! z>kU_4_@daSejFdDqn-MK+3AH#IO*<vf&?Eml5X@RP=jV3wVm8?s{`={p8EZ}VG<8n zZd9+%5cj_uNuRcgH8LSqV9>F9<Lz{G!$-t%`Zd55KBHTWz&`;pQC{jf%F+l9p^|2A zaQ!q2TSGA`n^AFl>}gn>fMCy&s*cX(tjA6TFofE`wh%!4p90;p^!3@LTJ57r)}H?~ zCqPjU-$;CSVz_ndMpNbQ5TQ-Vm06xz?Kg=DUL8kn3g<Dc$5T9mnb^d97kl(ELKDVL zHK{e)VNcwbv;!y^pTipc=1zapCvdwySEmu_P3EbgOK}}|et3p7Ux=}!4>X(^ZkAfz zl@AT;uu8VB7H%CU7KzKrB5pR0gLy84y7{vnor{jBuQvw!e;Kvs$Bs*t<GNr=$_GwY zb2j?UtOKycOw89G<Nw@Mg=X41ei1&3##TXA;fPvXn}6of?WbXkBdltd4QXl}=Mejl zLH;pnaj5`KRmn8i2J}=`YI38rv{=)Bmq1Ne<L@8kj`X+0izVZ&TTl4BTHwg`%4LfA z<4?BA9;<9)RV(x26vIUo6uZW_V-mbd3OncC?}YUZ6t-%An0>{XNh~WKYxS=DDCi9~ zkM)?zZ_amW5;&$1Y`gvc&=Kv}FjRnbovw91?oJXYx1MIb<U*P-Zwn<i;}|VA4m+T8 zuK>ff&CJbFe0~00z$-{a(pIwQ<<~0#eZ|nIhsW?euK6DdL-q4&^Js77qUea8=P(wa zsVXT~rJw8jX<U9ajs{ZUG4m9Wdx~yeC;b(}QcjoI)Cn6N8q#9ZOkSJ`XBs>>l*7+X z@?^#5k}OX13#3KGtt^0x%Bgyq)s`Io3T6*3_3K<T(vs{GetPmcmn(JIs+7BM+5Gh# z;PmfhVP{~G^-#)?%OQM?#3lB>ZR@}J8gY%dxp4i3osL{0bS(a!nC{^$E3Ov`ZG@Lq z-HYR1&%IZ|_UJRO;oeDukhn|g5BF^=_pv`s!2NQ6te??fa>=k-gs?w{`R1mJvS`^W zGZId)ix`+26LpskQ9r!=iS=&azubR8Jy?PB_LSv#nRLEqgKU}g^w!%kWVT4%eXH*S z%b(1L9Z4_mt+dq0X=w#?pCv|o64G4$3UmYit{_AGV`2+#{3HIDm|R8iZYI{d-m-GP zf4}#OZ^h*pHLAxjd)_xTYRrR~{cAYR|4uj2w{bY6b9KB4zH<k-PL#H&o~1809Z}f9 z|F*_8Xtuuqck%UF=u3qZp#PMV!;#=Fw_KXOCwnpeD=q+=R0!7>+toR1{nSBG_R(Nz zqqtadHM&yXef&(Y7lp0r*z2<DzLC@H#7>Z8qLQ{qmx2|{*DT$u7F@SWH;L1=n%&di z0kCU$d>-?;TRUp&{t%PuL5a79`&VJR7S!s@1MmWk%&#CRr~KHa*VU{&7kzoLW*Lq8 z=O4GL!T@#V?o_S8p%bhzPp3LvO=t$@aPrx!e|=)e;eOq<T75g4R$0@Tbx8o#O5Ht! zXC-iCF1dZ<b)(F32qaB@^l1G^I2(dh_Q1|e@cC1rIpYsZquX$h!;tpX+$-~^tRowA zZs7i4$^M>*@L4uvfgGc=%M`HUhl6CoG&Cr{utlYmmzS58av2CAkeP0>Rld5V_RI3C z`SeH!a+^8e0{<PKaPT&PM99M`4RrnI5~pVSnl!-D9jX|;qwl~bOPHBHu!Ct^hA5GR z?*sx%*cs2qGxmsmiS)|l4oD)}LH1vv&$1n1yJMj~Ad~`mfC(QaRirO7lT51EKwW3+ ztsY{zj4vm9v(}Ou6RtwH%=kTpEm~qPZvlhBzBjRy--C|%OGC4)CU-OB*W0QpHx7Kt zT(~HG+)(t}M!Lg%7Sr5s8p>7bwHb7A*F*aRP58?|`{!hRV$re!j0vjsc_J2^Rn2Hn z+spJRAsFgXf^xRFLXF)fzkU3;ky#yAzRTBAyCJowm*kdCeITFW8XSzsP&5w`HJpmm zKKTfOP~2*#&9&9l(JDuCLqkI_c+-X5mlo%aqM9?W)Akh0<>76gx;~yn@jp@Q<G0PK zu-+Qew5x5Pv9Tlvai`LJ;^E0Uyw2u=uDii|c04bkac>9}c(=44)WAiLk=Qwv9sRz9 zRAP3){et&~=B5efLPV88o&`<25P9eJwr-(YDO{ArcJ&k1N#j(ldq(*ZCUtF@j+}8s zS90z;(K0l4<b4l$#P@B<ku&Va(;t;!ii+y57vU^r3+3@%R_5EiB2FT2^h5c?j6I<r z$};l0qAQzQ4HB)BCsbhic?{!rk7FbGtTvW~<314@sieW6D*+n4EMnTqh1eS)FoeT{ zV>T4tic*EAhQ%lyZBO?Mv>{bYgpv!SJI8z}v<9t%DK7&|SC>xwT@t|hnXyJ*>SQDv z;IQxKiqp66|5(Nr9O1hjF1_PCVvgH6c;X|r+{OBNy<oua(41*P5vE@^b)6j`{ERs) zPzRP`EM#}G2+dX-e=0)uYA~>ahLwLeAF}&~Dyy?YVYTeF73`K;4$?%=fLe}jF5}EC zK2hb&0cOV{+FCJV$ZF}PuCJ(Pi(p|l`ncVi5f2)!_f|fB4^R>omG<@tv+))jTvl~k z>2Ru=l?P+$vv^yX)3gY|NQ%}Y7S7Y4P9bW2m}Va+s$XnLU_Y8*{!3a9RklO5<*@IG z2IG-0F!Zb0RJ;U-w||y8Z-Yj$M@TJmlgXZekHg~nEq5^x{%wE;EJYTye&mf4+eY*0 zE!O8Q(|8P?I05DH^+!b+1;!wfICfG|cUPO0mdfNXIeq6BzTwjV87o+$Zvnlq$7p&% z9udebzhiqB_=@`j@XirwmVm(zxs1FGP$s0|{R3z=eKuqz6G<v{vf5eJdLaI(8?0Kb zL@z4w)R1FHr_w3OOQ?qQ?G^zLwx8DfjvxhVs0xiUXEb;|{o-^THN%`Y^{_XkI`lD- zEYL5dMu$7UADKNbBoe4niSkkKmPGXxP07f}C@<%MQqngA!hG2?SqM_ZZd{^EJh};1 z!cUM%>YNEYp+aHWqgx2T&QF$k49;z79pTyNjR!0b4poQ9Y-1~A2RK{klgeV7UisDc zNP#r!VczMjTw>nGZ61%sK$5hxRUX8WDAI3qKgyKTrUAuf(+X*5t?_DI`w90;5^$Eu zcWw|FM5C>h1IG*7)HM>hM&$@`SRk#};K@esq}Xe=KbrK)VN~q<J%wv1MH&m19F?nG z&FZ0T4_hmGON?LgjJYgO6{epUQW=xQ68Hdm-T>+a(N1b9bh=tv<GsDTV`F1OLlU|V zhX!Hc2gd|R8^YHY_N%pN$8nO&I@M0nJZ(fE=|Vy_3q#$ncdPOZ1}>TfsV=BpO)Im{ zWPg7!QpMD(q*OQ787-L@iUqXP?pG9C8jxvmivxCsTZ_v^Kh#Pcq-7&$=vE+ixnw@v zv1WmxvO*q$XY!iBp%U#i-W;mS#jWsXW^W^>zRmx34|iaiqpOAM{MvXZ@;pvNtLHOt zl-esIFLkpR9}93Oq8#&yg=oCTv(=rIh2`p%YBtxa;ns_+EWNsAW7P9SNWf_${HhGO zfv5~vXoR?4&b<P;gslAhVT}MVg8hJ}A>d|VWpKYw==K-<@s{5@b9?U|u9be8RY~Nq zw<`HuO9EZYb_9KM@^?Cyx>y~~aBkDXP}kB%f7mI*QVf+=FK7L6LAv~)NvU{ib!2Yg zEr=C<?9dHQAz}qfgt?;-t}rNdDT0f-ZK8P5)XKF<CKHyIm4WjX?q46x=zMLn(L1^o zC}P8N`VHRTA?hPvB`_Na_Y+<hp%%XNU=9WAg`3YSnmuZ-cP3z+Z@7qx(E~wv)N3gA z43y7Yetr>h7e3$MfP38vR1}~r#TWzr-qq!KhI2rG#LoTig@sDS<N(LjFX-%zr+{x{ z6K8rZaXe*Rd!jTy&cAN$j^&rfsP19?uIBtO%9u9`$~#kWmGb=(Wis+@*4a?R`%$~r zcOJ!O4}g1Q^%bIJHmyxiRyfGqw46=K?Y=Z2yqs-tIToBh{jubr=`Fag@6jty^Qd5` zTd#Erz0Ov@_b=xnJ&CZ+Q|AJahzqlOI^{*yRk)R`NBVHF>|$tpu@%z-R*x2}?gyc4 zM_o7ndH_}2B<VXfB!}}%cuH<<bEmg}_Z>f3a9tMlg`1ypoMZRybHzHHNOBJkreFMj ztbKPN)PMZ{At{QaD7&d)m5{AeltN@BD`jLQdmc(k8KsQOLOA2>y@g~edy~DnGmg9a zJx}A)XMDfE-yc8!G~D~%uh;9jAJ6fsc*gO*>7vK$)+(pLXI>`*`gO?>0h-&;Ryh`Z zs7zW9Y0d`gRjw`76nc&LS*|_|t&UD<Mu$n|<=PsgwXLO;DrBu@S=KhPX9GrHcG6|p z3X!grmocL+(M6zg=1$wc=N<>QjS_LHBWz0OO5GeEf8>1Hpx=7ItFh^v^moGMw-|D< z5@~|UHQNm>sd65?d~Cg5=3;YkEr#QpQ992tF#SI;XFzljF8%<Gve&WaZ^dELUB30Z ztg^maIM}b6&$8h<rs%C_At}6h6C@U!x2ST2k?{>=GpB3IKvDNyDJ35={c>k)?A=)w zhJKmHn$Prjdmy|rH>YQON6@*<ecb~?*o$zlKkVQOXi0s3VWy&qKRt=+;0*Dr3>y05 z8^THyI+HrUx9tN52HWie!B;%8Pzu~NC*=q31YPXaXo61ujQ)PZ4Dq|G6gkZdE$_QO zR?JkF)qhQ@o@Q!uvBj2o$P_Z9^|JwrA;E_C=H1uXy=J$fUN`h&=~?T>DK)ejhfq%f z9e;3GYEUzWrWbErT7!1O?Hfu}q)@I^D9p(nMQ*>vDbY#YR{-<o+)o1c(`Q~KHMWc# zE{^Q`JcE$h)5&TT;#hTTXy?3f{z^`cpJxR+vr#hPl+y06&mE^3LV3_hNgbMYncS6| zu_&?!y!4f|uP0SzKv8_j5sZ5wMXN@~-5%44x>COq;(zO5-lMj|$}-*Up7{kMPFa`6 zelp8Fv=xB1m|ih4iO_qGn6_Y=ER-jO_s6mZGkb9TwEPBMdOU92s8shN2Oa-}X<KLO z6m3s_0)}wl>!hW4NjbFu#{jX}o2|3I_5Hh)D1SXWDLp&>=5d!Jyy28p3c)%=nItMe z{|tU)<Ve{R1VY*42QN%I>rBy2=6sTMk|=!Tw&}qmW9HNO!cAMd*fHs%ujI*6C9hC- z`$&g5J1bQWWw39v?Jsd504iY6vsmvo5$EN=D=&=uC`ifWo(a*5=#$w`%;Wav+^nA> zGyU>l9~Mo@#ugaIOq?apzKvB@=0o`>>VTu&d&lNMNS>tS)4vX){u2e=1v&DqGk8fl z1H)_x)#G&4Wv*~DeP)T`kbDIQ+tBk(I1jB`PDsTK#_=2GsXYmqgq%>6W+zpDR!j1B zrIB~Ox%;UTh__Xmn=YhD2*3Z}V(Vb;f!lGqGV<<2PGzzuvnD&}gCpN{14HD*$$}aD zh3q2S=eIIPT``f7k!2TtpNzuHrIOms(E>LslP({Eey(2fOJ~COif5A#{MGhrFl4pu z={#)j5E;s5uL|W8-Se5FR{|Mpj5k{|MstQ5*lBO=VtLgup1i@)ElUpJl038-y_ga7 z6UBr*4^TSsh9I=&130bR38(H{GgK-{qxg0eE_;8v{I+_JHV&&chND6%*dS)kZoP&u zq1y~Mk>|s+nOooKMHz(pz7&cz4|&40<JH{HFcCVsoPole&To!`lDDnr2v$$kW-|DE zRabRq=cE1y)H&gcAl=hbLR92fNr+Q!ConTc9^v3W!6hy@heCOx4bwZ@9g;@P&DmP6 z+?;N?v*g6va@&~kT?plgp#cWdxuLk7%@C_Vi&zS|hkK*MxpU_LCms2n9=QxMRkjK; zCB1Uu254H#6m)+Y%EcF4e5c8r*vGgK2v8j{N4U~5Jj)HImr+QAH=Lc@Y0=^fm`Czb zllWuML7fE6xCN)>n15yhp#TDf8U&cF&DVs4u+t*SxPHGuU@Kt(4XBeJqi&^aBPW|2 zJNf}at+=?$f+cXXXa?KQ+=55!BOp*0&8r1{dc@$7-^X-t3)zGgRhvv(_2#b0Vlx#P zna8KQ`F^JDATy=9|66<eR5b>JkrSpu3^M%dSp?bGJTo+Zim$Rh!Qq_L_g(obR|g@` z#>%X<q5%Tob`ee|Ll%-|ky}{+*y|p+;TzCpo4p@$a&mHW)eCM~DUI_3?C>#7`gclg z?&|k2s!zTw!e4}-d6MhjuRqA)w)Q@BpNX+?fW=K0xOID!hUPa(MkAq;>6LV(*c%E& zFL<_y{yKJWF6*p|t=RPjD9gmQ=y{KI_0CAq#$4e(MpnjODcIwytY^$2Vo_32iP36p zYdh9+0O1~`L`txB#Xu3ybXHz=wkioi)oI#-JxNAj7-bp`y^w9~?e7v_HG~N}Pv6{8 z;YIm7#gi&}w>vKuREEPQDHPTNi6F?p!T8_zFLE~I{_AW)tIRvjzgPOa85;9}Fx|-U z^vV%U4bI=$)SM$9Yr#5+6QJ8#6wYKpW99R)@rj)Qf3Bs%H2j$t<)qu2vLX7@+`qDh zbdQGURjg%tz76%5ZC77KxPOOCo_kFrU*AK&$ODtuU0ZoNGQ?-@xuWWweB2I%!LNJ7 zznOSk*Z26HB^L7!VHD14m`RXk`xN(%46$(fr=t<IIGG=4@aMOr7FTz{<<y~3&EH<= zB=YPz@yAML>6hJUAXX?SXw{a324wypTRE|hnv&cDm<iom-5T5Ri_JMR^ER2sE9s;l zM5IRZD4EaA%-9%(-I(P4<13fxRdP^(H+0PCfA&&kLLfMzAyL^CbM7xPA@bx;>_ef^ zbbQ=tz}!MYLUy*ciJ8x~JZ~Y-EJnuP9DFSYCC&D%VS8TM6ggs$;@{jSJT)_O4oNj> zn|SK?B96Xwe0Fs?8G7&=@`VKj<MN@0bJqn`GFo-82mtx3L4xHxM|w(1OsvgSwUrRE z$B%Fvg6Bbz;?PdTu{P;~*&l{qzsY9y4(R_V;DvG8zLf`n(G$hM-V`3vPBaMR4^Ys3 zj7ILaoQxeh9<i(%yfBt*`rrYec3-=olXMLB?wtg0A;xUiqTeMX!kUzla^$r0OZFBY zQWa8)6~o8d#D*{I5nrB;n6~!VeEf@n5P8}s_C0`%VM@vk%VW}K<EBEt{#GYv>CTvh znJ6kM0ux7~abZRrU|PLV<f~uUB7@&bT=9BqnyU%!rIY$@M#3CG+wFy?$UPcPnj1kb zW=uo$oA=a5l(Z*CDB|Q_^hHFr^z`&VCvI!2vaYVKme%xp*`o{Fws~@%l?b>68<eoV zMRhhameb2y2!Wu5oMQ1K+3%XQSa8jsl!}==daqE8M)&JE=bCj<<1Owe9+Q)X*A>Kg z0>lipX4q@|oxSZVmsJu^$w^I`i>k<>jkXe`a3gKNb*OPLId$^9v?|$!J^ee6s=iC2 zzJ2?)nYR%38uJ(p45@%*U`NsL>jPalAQ+|)g7nV*HbE*Axq2=o)jhj}rf-t+5&dx} zPJC?c<n=>;3A1!-Y<BD*LX~pc1qZHxTP8`TU_O9{wEJsvgftG27af2cJK1+fQ<7uH z9#fDqMl9-FV@D(<4(0DnIHjRme8K<2mE2-^)eE?*mS3%=MdS+hTLu0l>MuuHOj%du z?XQ-Tp{RFwE+{wzn)-*BB%GR<oi#TxNteJ%gYmf0!b*nVC%w*a(wzruAvwr7d}Qz4 z?=v^b^#ZZY9uA>x6p#Z$EjC8wyB~lptWD6Ch^s$2{nti9s|q{L^KvOEKLlUWle35c z>N<`XUQ43>(8d5A9uIfYA>>$4)OT)3jg_s0An@)K5$<%`gr@Bq9zFSzJzW8%n=TDt zgt7eKa03qeEF@AFcDFn6Lqgssr?^;~=N4}Tzk+uSRU3i@)L8oNZn1Btr|7KbARm}Q zW`o=XRdiX|wh5tz_x;Z}o2J>0>j~+2;d)l)VmB{Nr!0)ig6p2z!<tS;_!0LBCkdGR zaPug(3K|fhA2KslB3jzMeTzNPM3om}kGAT){_!_;AcG@YzfX04!f8x(b@hT1IY4hu zF1|n7NP}?i1ZOh4m)?<nd0OqFE6jigH@O{Q_<)4(9cdUa^D~ZHZ*ZfnpK)`T(w6cn ze;_@#Aa4C8XN3<Z0(e05SR4cKflbmeLq5u!<_N}5si%}~wBFyHqr}nBj=;AUp;3pq z4TmnT-DzFl+z|}2KN8sbW#G;0Uxb$a2(t;TW?l6;BZ{1TM{eIeMhX5c)SQVynnm*N z@i_D1;$jkd2yH3m=6ywu)}QW@Pk#@MhsOF=h=J6<UZxQB8Zf{2iN=s|GPyLA*lyz1 z6G&fwj;Ov9dnV_TOLS^KYZcUh0?u*I1>EsXwCF9Ro9e6g)u{nwQo5PnC=`@ZA?^Et z>DH?HD3e<Gg-=cQwQ?BoP7XvW$gka~scGn-OTYR{t~ga3R*PumcyqkIzvliWThHI5 z<6?Y5XlwM9X>xvGKDJ482zO&RUO|ENlqjocUGcR}&#?t^Pjr<3L*rA$dWNqeZ(q>A zsnx@jh<U8Ypux&>+On=B>2uVbyb>oF@Y?4(Iq#}^tPg|d>E!uf_`#*NCA8MrOu6C@ z7wwbwq@c>w_P;-F>bebE(DnS{s}`OMpE7n?(|f7Yh1k-KSs&ZH;S}bl2+GcuSsQK) zBh{t+wTBRUxwsy3xurQlKboDv`&gq<aJTsUr7shNgn&!m7!Zh0AXE<wW$~iRDK99J zi%JOTC5;o)<Htk#2x0ydJLW0$bQUWXBq<6iQPkJ2F?kE0Ekc#Co*q77IMyE?aqPCL zhvt6$Ff%P@Z_S6%G&}0OFN8GNr)_T+ILmaaD`S64&})B?`^BAJ7qM9E?%lg#q<zS5 zt8muKG3r(UkNeSiosm3W7tjHKb!UC+ki>p|{Z@WnYN*p7l~rn`UzaK6_Db16ztfE( z{j8w&f>SQpF3L~+s9($RT(^B%W-E#)+YJ;D(>@n1Fd*t`fme^O4h~95(`j+Ufb3#y z>U)t6H3U+~texpiktxY}JE%`>OYsSfEAqZPe+_gk?r~4#j9yC{V(bIIRIuC4jt2?q z$sgYdwDuj?SI_O!>7WZh(;p@c7GDXAM~FjESr_!2{=~`igT*LM>9+1JfsZ%x)@CkO z1&bF>ptz9ERVf{#$*vg-sg5HS;Sa~oNQn)v8ThYbdo^e0+nx)*ZEw9Dr1<*C{cm%@ zbJTlPR8`{`wOw6Z0Yf)6H3dsGr3RE*1fwx)Q$rEB1Y-xL&*eAbr(;XrQVJIe^NTHX zBs%qEA}Wkxr}r1orJQQkbY#gy>EUs^4d(`&E?&snh5|h&VQgr$%kYAG<U4kG<cYRj z+A`6>&v~EkKo>8Z`I}AQR0+vd0=_9*hF~FI{nbg>S{%x=)Y8#tjdC12-7&v8!w}r> zY>u_gwHK63bFOoB?k{y#RH>ay@t^I-$)&jrYpXOe3w{rocN-V8XKHdB^*1%<NqK9| zQ^ytR5j$MZxE1o`X_wdE6{t!r#B^vgP3$TbbSI!^sqluX7iJD(h1oyqsqMd1mk|b< zeK*dsM~3P;WKJ%fX6C=lEv7{CB}dxkn8$i5cC!B6F2R~-aGS<IcJXk2zx3YhtgO$# z`3u)oFSuftrSOaXPs80XPgOfyU$IJraP%z)Z`kMZ_4F+^GZA<9XF5;E5SRP=sQSY* z`1q|xdwUuD@hz(*8xh^MO{kXB>JkNsILF)areAxzG2TtVbgz8`w6<c<K|8}5Lf;i? zU8%mZP5M%t7M<nBh0?i;oFF5O*&G1f@?>Pl%FZwyiU2>ND7ld(`Lh1*n1Sq%iBgtX zH4<xY|K?HzoXlIL=6t>~+tbn9tfZ^U=W_N)<Z+ulz%gvjw}@W3FFP5EjGCxQC*6p` zmdRQgIzpzSj|1n5@vk?0coQ8T;Fu;`$68%^R@ofzIEsc!I<t`c`?J1iao0-C#|nAl zIiZ!?*}7h(M)sQkVW^3G97p#X<sc9PF~DcEc&tlcV_sNkYzG}5$%+%#bnFfFCYN{a zpcfO6OGc5@!hy>aecsxtdJ?-Tj&~!bD&NrwHrIW<Em?59@?EfwQwufI<~66V#(h1x zmccr0h-TsJOeowX|Gcl)nQ~AD95Sjio1V)-jU1&LmB-#Is?GGJm)Ot8qlSou^MQEc zau;!K*31%S`D9cadhE|rWH<3R*Vm0$D^F{7Fr(D4-Qz|T6|+23-#Itjx>Sk5u+lHG z5dTgV^=%NmYVKSDShQZ>hS$ZUi#p<U?tZcf{SOi$(((Nb&%p~)QZ85<oGIbuZkFIM z1~v=f$&%<T{}8pCUh;;-h3q{TuGR>N3H!DWemNKyMbfs!?C0M6*S0E3hLMCWH|!Jn zC`}RcXv)Tl1-iRq8P`AC=A2WUW?vJv>UO)(a{Qx2Pvk(MK-&CQb#)1Ge4c^Z*`W_n zB`e=+3!|bkiEV`|9mJ1!32&sXZ**m)$4it<^sm^f5RRC^HscN-3HpQe$O=w-I$nN- zyozR+3S6SGo$6B))jPgaiF{_MMi(+)t);5>b{~}R%ec>K?&^E4wjVT_-^~O1E7^L^ z>QehJ@^ENtzZWxWMkmlE^qAREzS>E01Jj*XEhbfWBV2n(>3_mr>5qf`rx6?g3yb<O zxs_&Ywj6X0o1aSIo9R0jW53Mf{BEv$Id@==u$JCDppNRqw;Y*8z6)`kQuN!X%Dv4I zl~44d+@N?drpU%nG~#W<y4;{bwL3aHO9-EG1G(n7&Y5^VlM=DUu1d)sojGZ!NEkyD zNIfRG$Wm3(0?TuTyrrqBI*MIEO8&X1vY8+~pd`Kk*TY(+56O5%a_eO9mdA?fg+86Q ztHh9X++w9BdZ2AN+xEwX@~{|;LwXmf$RWo(4AZ^cUBhms^H6c-wtx#DY{u-x)fcu6 zn^ZX5z2AioV?S>@Fl9Nbh4pnECAMsQqc1KcqWj0UV~fs@Rr)NNGDVi^eTmlM6<O*T zm{NC|U0CrRpF`r`7pAiJ@vnyE(h|p_RTMsCXQ{<zPtpzWazI%Yec&%;L57Q85rwcK zM?WK@CWPWr{fizH)g$lm{q@|;!fq&$4|8T^kIt)7`n>Z}7!R|oN<GEDaWv$OuCbkX z^gE**H9;NMZOfTSbO+MI$d(_kO0AvPh7VeQ`P|r5WGxaS#1>UuePw!EF~BJWJXSr0 z_UNtcLK;k3)M|czngfoR$_^yZD4iHJJ2gODm)%IdDLbcW{bg>PcXvwRNlXokOcT?S z$Wll3(Hrx81-KOF3}OjJwor{Qof1Q6dNqJ=?5*RS!d1F1*Z6WD#3OaWRxPF02=q}o zf-Sdihc&i;lyWI?AvvYj8_mcNB>>t{Z6n7THukL0xV;E{p8I-{o=X+rNuE;nnAqrz zG{~M$k!8|&bb@*;9HOW{t#cp+KNZu~)-}z;4YefiWKg4eC8jIGdk7ZCNiJrLVl#j@ zu|2Q1!e8`5#53-F)K}+^klkaPD<)?|>y8ZWA?>kco$#?kH~AKFm*m)l`LdB)&IG2} zKz8Q#`qw6P+BKK^bB(I%E6F-}nZvb;>Jr`9o;oxe&ex5qSmTyPxOMqkBRq0`0B<Sl zvcbZatcktcon832y|o(yj61YEIzsYb9~sgh79^mNp=`wfx4=#)q`N`%ma;0_w@Zkr zo;zc7r0rY-FWF%QApmiY4akkF-*O90fmROaKADtuXd*)$%PBdS?7t^HnEXXbDOI){ zU4-`Yb&HpiB(uQA!4V$geeuwwEiJ+>3Q5Sv^wUdMFP1y91?)4UDLD*wff3nt$6290 zboz9p=n9wA=6HoJah2yXQTsgFv>nT*$g<h4E$g<j=C3-VA?j3PK!;7fy<RI}K(#^~ zL%J-7jjC`dip?emTZ#-f6#C_v*XN@1H#6QFMRhIeu%*OhPnwwHD#z<;pq}=Vjdc&8 zebgxGv$yy=i%Og-q3q{Wg*;hrbki7wz<^!GZ&xDbWZPb1-A+sL{uXOQaFBP}61Fk2 zL7E=Ov5{EPjb8frQjH_q!?jLEm&@WeCb%PA?k#rJxUR~|vO88Zxh*w)*~sB3*eEsX zC9Eb|6C_Pme5`9458`9yycPNvKiBrHYHygVqs&Y5;|D9E+!kqO<d@x~Mq^9)B-|!H zanP*|(dR~;y@yNe0>ANN3J78p9|zPv(SL0)r+|4`TUi8f+Bo$hqAh##ZWi!(n?f=$ z%5|j+EN@&XIpm_c*Aujx=lTjkZSWO~Z3}o0x<X^q#+PFKoW5>)Y2gSq_!OV`@*DNB z$Ud}gsr*J%=UIcfnmYL$G&jD++}=jpv5srRvSc-j2F<(_h<p>V>|a%x%cq^=+GTJ= zLEH|rC|i2xVS6&Y)JE-UiEZzyTbhXd5|@Ed<Vs-F{LGp=@Y=0hA^a=)ALsx5r}qTV znJN^IJ`jI4j)dV&4$P}MWHcS;N&N7VUF)%Bzl!+ygzo017IAUj)F_fzZ#%v^-GO#d zZfut~I;Sl@g0}BlZ^|evlpnX(K4V&!z=B*zA@*w6lykk{>(5y$H}_ATKz>@~J3q<A zM8Hg3jvv=?`*t$SdlU(kxP6ZwE%CC*o0^zk(+VSej{dtGARLkby8?)?tXLf(c{K94 z__;{!^#z<y{1jocH=&ot&E`|C-{zdK|Jn*NNLOl%k0!NLUuor-=3{22-VAKeq!u5q zF{|U(AT0}`Y^^qNvn@h9xJh*SGqHegKD=>*%Mv*;<Q;z<HZ;IS(YWqlRzA(7u=zEg z;gq+Zmi^r5V@1jJm})05mVy1<7}=-1fB82Ez%yWNdw#9$++C!7&6`(7QC3sY94_Np zRR&aMd9I&tO5rT3Ue=Zm_k3xJIy3YA!n`DIFszD(c-v`%Ydtr|pVz_i6J|x^GBIzY zkr-*U5x{}pOy8`#jTDHVEzdAOE`4p0l3d5Xu+VUBT<cL}-<xOLT5&-{3b$epjJ^}Z z>Z+wV?uhjdcZI3D6iEa=v!Cm&T{aq$hm5q6?!1Kaik#(x&zykYk<dT{f*jayD;0eu zS+m5f>dnp4wU*VW($NgR4K5uQ+|9TVw9ClH*7&CUjX1HO($yX+DO_cDw+NkZV<~bP z&E)dPWGHCpYwjI^X%1^2qEjW$z{INWZtJnE0b<#x5#9Q#rHjFElVX@_YrtD0f5!z( z?<?7ehvSZ2@tc^>bVBANgFOib``+Br1!XJ4X(8v;xjvFXb+zYYgR(XSbFx;*>>^;r zfa4*6Dc1rV-Yq0);6H!RB@B<3vWsQ7bZ5`==8aZq23ma1|K8Lgu6AauWlUCQ%HiY3 z+1lXlD3`>sp!j}#n-AT*Q<~{;ig^q*%KHXx{$Q?@O-so8ny95lbdW;B!FG_iB!6~? z>8qN-Y=U~<79aaG<xfm5GU%Mj`yQU0DQRxn-O?@xT)LOUr4yr%A^2!lwVUle=32uh zmkXu)#iQj!QeFbEYy@H(#0>6q&*OK}RR{WWPXcoZ%J8nW9dmq3X)TpEPN?WTgFLYM zO`hSjltz);bb)=Y_^suZ7xN{^d3nCZwXZ$)w+JZ{?{llQb8WAeYsW_tSEt9=iFl46 zztiGaCa#3%*BeXps@gi#8}jn!+O?*~8(oNvlPRnH{FC7-jQx&1m$`O_yKI`=FV&QW zRV&}F3wk{#O*&`x_eyd&J2B3Y<tC}%e!?I$1O1aFdPE0OQbhBKivoP_&Ld|q*_3qL z1yk)E7Dxw$6}BcnSCKs9OdmRSb7M_xzph#_arouCsW4r~B!|lC>3O$DpF6!951he% z7nl>7u5iQSTsO1^=7^6uBy1WmJTy$iQDM4GY;GKqr#KUvdwEv=1qF=-vz>xkV=cw> z(;H5MnBHls2nqW!Uqy+Gdx{+c5c1u~%QibnP73+jkFCsf9dSjo1!R%xD^m8jkutde zzC%{Z-5HH=kvk`whp$}C#q&69rq^)iKdMdctvIGm$Ql0Fw(3{9ym)o>ZRE;ulGnUt z$;RVc`J5t^f>C@N(sf)|%S~$CSKR{oTv#F-36p%s*_P{I{aE7MYN^~CLJhJ%M4-g3 zvGu7Khed&<W6itlJ{H}c#m2Ex_0n$zcTHAy*5JoWk;{H$GN+ru(##eXs_9j9#piuI z8Ma5cw$_4K1y4?>zda-F86_#?3+^m&EyNVtNw<P6gF-L9;zJd=Bx_QPB5xamMA#(U zA%_@D19oc8t6rpQVI8tg;@~T>9jl7h(Uh3(Ga2vS>=Z0@9;;@u5G9VSR|boVWY~_R zaFhy-k91fP&p)i;vbA^HywXzBbluUi{~49v=E^G&RWIV^SZ>pdl*8Nti%_FBeh%%% zQDkbnkUBcwgv?v)mz8ozC>m|yA}o~^3-B_F;-3igvkR@~sASBEU_Je7fJyU5qz-CD z-?g?N)D93^lA5GQTKrDReR+CP?*Pe36^@P}m+mTd&c)>H7}!d(ps1NySW*};q;3j` z5)qtM5XEud8Cml&_FF(L25)|U7A5tnbkghKtQS-7MyDX3l+$b!T5NnJtg!?++twpo z-<_|o=I^6!LBK4U>J(Z`M2tr{EP8SI7o0I_?=z5(io|$J-C3?NN_87sbxexNcN&d4 zE~O(p{(Mkd@f?ps^7>pHN3S)quHyBybz}bQn?q`bw=P#M)XwjYQnn_%#l5wEdP8z9 zUTPyN<vBnq<69dd8GF)(I`ggLF}`-vk==V@b1$zkf&~y)Q!NuAjXRZks6(YeazW0? zuFYEFH$KhuL>BAUMOjRpU|(O`JfkS#)Q}V$DLVU9fJwElIa8E4+m$bWoHjJfzP&#G zNU)@kmFp-*_;h^fO7Bbcj)r7Ssp(O5ixfBWxhP?#6y$g#lThD^*=({Zi>2EpRx7C3 z*+{ctMmNg3;wmNAu6C0aW2=pmGHOyq&;GV<a8l}6qug+bq=omB9Fm_a2S$TsBO0d> zG(Dc^XJb43H)`n>uO}nNp7J#Hr`_+n!O>7Om^<Y^;wFA#Ja4BFN1*fMWQWAVswl{z zFVts^WeXFqio)5+r;VMg#++5X!jdf9Vy4D+BO|lj@X5BwNbF3MrK3}2I2T7j&RWT7 z>iPVcg6CaK_{~DuO{eDkmZg{orY47(`d5l{`Iqp0;!C*D4#khzS?!LdUCN&-e)@@L zy)oK$WjsGme6qd0ogQjg0Z)K-n4^8HgEWx1aXBvw<8BwIe(vV;u^qyjCJk{$n@b)$ zrwI>b>ZTGcOV1`55vmkbuWw8@l^`)|<zJ&hHpikY9ysSWPxZNuCuK@G&#ovA<c&nx zmg2rQtg~Th2s!I>W4ZH=^ATQQx`mmzNfr*?;^CE~jA{%ZSl^GhIrP+O^2B_sT5H1! zs2CQP45y0m;pcoZuOt$6T+|rBJkx5`1sy&}$`pJI>j@HxozI}KI8@9-l61K!Y#R&Y zJGzMp<%aj}`|GHdTq!X5Y(v@*pk}t7B~~L>{vj!MJRe%sqR|H*4f%=)tM#u9si?4^ z3<>!4u$)Lo4COMP<5HEUV)8zdS@S3fkz%*GV=s>H$|v|Fdp?&j(so?Iz#sd*_e;)J zR3ryJ4PRSkC|ORK4|n)P7tevjRX+AE6*PMk#g?CxH;THtdV?p@dA(CNA}EI7Jk;dZ zMHt!uCs@>kN6Y7XD^wnEoDs>irI;+7ao3Wwe6hfTqHaBOz>F5n#c!bOb_may7o>AT zQb(4wp^NX8Mz%^T{wDZ$JQrGTR-(_FRv2_hrp<g&{}OLjvhjLA+i|QR_d73&w|C|8 z2TV|ezd~xL3&H#=U*7n8Q(aBLIJdSp4T_4c8+2cp+xszggl5lq{5U<A>%6!9t^);* zokzLWZwjax`Q}f3OZHM%;FWTIiJP~Uso~=-egOi)1%mzR4l{`?$BlBJK{J@`A<d*Y zeaU6j3tuRI^v~SiMzMXgQ|IJK?ZbV_m!j1;wmd%RQ(NY}_Sg>VCaKl(I*yHz8)$8$ zLW}iRt(qti{2QWZ=Gd4bsH*iDTP;I6-6g&Lf&p!!Ta_=;QNLQY&8u%J#rA71qpyo` z88X7E-X6Iy5qd<j#NkSjX6wS5gV}}A=DN9e+Td`c1cI@U<K0a{)&8-@&|{RaI%u@A zFr7~8zpY!W-}?O`)(KKBT~`h>XZvl5H`M$5FpBC0C2993_w$L{v?ba5={U~#Z{Ku# z@b#&tSX6lH!sl!>iIeKQdcK{G3zif+=G-59$jiV@1Q$-|0t;BUGI60HCe{8mm8+?S zTTMt9j@8Vzy~+bM4#wHG^>)s_+PE$2y!&t5o$|W%m?&4oTM1EM3676uT}#i<VV>p3 zwO{OE5>(5gZ=2p3TRUQYwSvdrzW0-)LX>slox(;()$$=14b9db7@dqbOTXp%8K+|G zRR%PzW=*FpaK_yt%a-DnL$G+_`C@G1biekaD%|EwkD<=_`I1F>!ox~y%U(5gDfC=^ zEnhhHYqFN2sF-+Hm&n*hpZCFdr@?%C;!;sVzOMap|Jh+2ge27pZ%Q0nVh0q$FT$Z$ zwogs`Xy0Sp*M!#5(swpB9qE~}-9JUwqZNiqGGuB@Il(s%#g}zF$Fl!p<2=xEA}LQ* zk#b5`z+&@OChvPa!<(Oa?I2OZtn9^Hww3C0YxHDpajB>vb)VfkqVSdZikIMr2b<84 zXzfX=fHux&*1n-ehbBC0e^mAE6}2hCwB4z<cgS)7+vyJt2`MR1fOhIz9+Y3d%!iEn z6TWSNOjsY0B3u!wVagV&(LTPb`?8rfRr;W>N<-2D@_`WFaWVjM<4y<6E30Ee2oEpp z4{2EmlXn9rPVplxz71F)vSh9=QD(kq{exGVu%)A`LeI1{VPz_LZ2Y*Tlg$hb168lf z6rJ1NLcY}wzM_?<BZ{S~-<DH|IsTC)o0wO0;dVo^-hSElzhU_z?5i@5q&k^>@x}6? zW483V?H#H6PfI9sCr3}=anP0o?I4=yB&qr-kEt)z64&~lxaeA(O#Baqrw)~}?x=P1 zhSDK{aq}x`D-#Rcg-}gCMPFX;HJvs^@dfg!=Wm?uNQG068BmT1F>w}Puw<#byVATT zeYX%!8+-DwP?OVXmrL2KhlRL@Ut+X8%EpX&&W+zyd-B9P_0+RPeXdcaD|aM$kN)GV z;uZ7l`bDh!i{^aek8cuQB|V6b5F`2vXKRbyThWdQ?&vRYHMqI@Zj&cBg1Gi%1YNWl zTlbXJbqFHyORg4fGf^s9>EB=W8fEZh6WqotHpernCG4s{s~Y*aS$4irf0TO1#c9r~ zOH<vq^t&g6V*dB~WbZjGk^)$6hele0hoijtN_{>q1uFRKb5DJ#!)jR@HVO_r>eTjN z3RXmsU#$vFVUZiSPt|+TN$ZK_P1S&l%RcgVxvFAJKt04|RSNXx?3{uz6%^BuG@Rfc zKqpb(mAz^8JY_M5djtCLay}u~P8fz>%sRN6<&_aj612jKxHH@LK1m!aGJEk^Loz0& z;XaJNadLoaWJpi1UriC3pl?Xb561|>B^K`2KW)Y6!JK!}F(o2KHyrV(fl{%!sTFI( zDC^Ax!46>svBHIvd}QClI;yzwRM&|amIqgwl1hgn)Q!xw>}BdE2=V8yzPW8RkjO;1 z|7~~s0P^6O*%X@Pf%?$3Mz26A@wH+3aqYz^$_|Zr1+k%*1z2P(u#~=ON}B3^?h}#d z0n`aT;Q^G67TkZT<f2ZlN~v>4G7(C?|EW8veZv-PjVP?}hNR4@CRKC$%4puPxEI*z z_+9m9;!V`{V~81#Jx0s*RJ65S9Z+sMix;PPdbTwPb)u-jT%!Up@^!S63T0+byCfbz zc+7qouYY_pW^|)rEK$jvKWQ6avQBUG(liyD!(TaR^GRqO@~q6CP}uS9oa*(nQ~cio z+Fv-?84f*Le?l|evVW)e@<pG2oT6DGx?vJ|dpTQCP<vxT7|ks;_Kf)M{Jptlb;&lR ztjToiC<|6c(>hgu^MdsBxsO2|ABm$aR&Q4$ojjKBOp4_PcI?XkR2u*yxGqs;O;XdH zx7naWAa^cpH>*v*tgq#ChjX{o0g=0&*!Jd`n`0*)q`lXbVAXFaUfp;kf7P6XqyOEq zD}Sz!5#y-aTz6JmG43f`_F<Z`Z|J9_olVlq3T>j@Hq~H;0~Bv;TSF^VgwniZ7AhKw zc~TA<dT%onppEE0B!Vp{K!@pX&?8@tg$kS<I2&dZb`!F`seAeBnCcZMSsUXvUQqiF zKdY-*v2akQ?mr9}^^|GOe}_SoMv?fHCsbjnE_sWn8aLshwY4Mu_~^l^DO1A2u;Z%D z%MI1aZ^62ShJEr3k=kN*>*(#i=mVmSj;UF$7|gNwanXZIzBFhaq%l3XXpLG)1}e6y zRa0#ev-Ab58l^dhhqa)|Jf&ax2+l$Ns{U3%NqAR*$Df|)$$Ny5{&kJ}b!wOD9@=xf zJ1fj(s<!`$f1K>j`M`nA?BcG2?x8t!NSDW&IDTlq+pKyPK@1b58Y{Z{^1hv0T9%oY zn}q$PNb|YledX>VK9%{OR8{qOC9HKKNyQsbM&BZO!ID(hM+25{Z%wg!S%sh<$y2Z6 za7x3}*GU!)mp26SxXdq6Z6~hf*{Tl^hmZM}NNu(UB?XlV5f;_2^f9nn_nekN-+4GJ zg4TJM4ZI->F8AikSKzc>c3B;HL6a&v+4*{BjnqRzKpR6-KBhO&x%=G@hRNBN{+~Xc z@`>~k>U!SQhHr%*8K&^Q@pMqk(af_`F(FLqD|CrKGmQ<)Tk3+m^7@7_wgbj3)W`3B z-)LyMub@&IB9*Un;|pVfi@uK4)gtTXNhr?Ea)t+(h4Nm=i22ctPrzmgWsYom`dSP7 zTkcH8o8NfsFS+tK-I^#=a<8lTd2W=|;xURw$I0Q#<@CAR`H;q6pNVh)6d9OKX)Dq} zuvqBKdlc8M5mzc+x;(hg%;cNc_Hb3=pjBbV?#_+j$>RBeO#O9R#W&bKHVL4<*~yCE znDpOcjg$M%;0L&tsqpkv&qBr(kg6V$V<#!BlhyC8W;C-E&v5pm$TL-*bl&98Jkd|0 z-ouAvyng3(E!!C70J0upWrLzMj`L^=ZW6*C!JCrm;xrT;o)X;XYJ>@w_Y=+5Ep^MK zD_Dx??XvVQjyI?eHff}Ls!LpIE{gYl>3doo*MB)MDT6P{`EL7&ksqpf@LfaYlCbtU zt06vXLt27K^ct~ib!K&?<l$yNlj3AbSk}ns^V|U!hApm!z+<aK>ED6JA%R0|XU=fE z8iyu4T}r4b!gMW4w<b~V4UsQWrBrh!m2w%TFv>@#N)9iO>|u?!wG9vkIaoxSmd3N@ zW0QKTs(1S1H|=waBk5io+M>?2p-fSZ-}$~!jTcLO{CwR-Xm=Q{nzEz%?de%o>#t$s ziY_BXri5_Us<zStX*0R@Xh0qpthmhbMh7mQCfpxD5vO0AAeK6xcI(c<)X??1a={*~ zvV(Xob8aPogmZbU|8C(7gZgHCq=ZQMWasnzb+p@}+@lBBgbwq$yn(j&3fbpE(y*B+ z6=bpAi))_cAB|febez8fHDzfU*_2}jd<q9cMvDBY!Pop5Z>2()$lrubj_aE0jIntU z=4ECBS7DRr<czZdi%FfU#qR0%eOC+4oj02b1D2cm{vJ87q~O>^q0TP&a50a#cJ?Mt zRs77XZj_S^38VdEAB|>x(l}nRLB`ge!!9pSp8L)Q4uIXU>O4k(^19B+O(?H3i`Zrp zUyS11lh84*@P%&|vc(1CZb*n82seZsja}TqaqQhK_5B*(t_e#Sev)LlBS)&=0T^E- zfmBG$1C)|u{e|R5oXGw#{W!NeQfSRG#<JO@acUWThkv5CAwO`q!T)JZjR6gNiT=}f zbw#1VeGV8Cbq@Qf7zV<01SN5vOP)BoJW6bR*i_1B6y=0H#`N{}Z5B%i&s*94Gj@6q zx^oPCRC;873CiWhtCm2I&>`9;WLT}6fOXUuOfY<I9Wg^k40Yr0WF<6jlLss9+OG3N zUJ*u&y>B~e7}j^)DGaXll0H66$`ikCd_6VcfVTL~r<i+8R=05BU2tRJVUGcw>9y$u z77OvwWU9Fm>k*p$Xu@J>eoa)dh%hnoD{qK|@x<pC1y;ff1`F}^WD_ph$RMO3&|zm* z5YMrc|8Ut;a#m;~jB_(44XMQsn6-9UEk8$$3DDMO*?`mDRSwHd9=K)#t;so|O2V=? zEumO&N+m>fpHnjQh`5Z@(t`icAas<}U1?z&DOkAR6#$bW*AKk-kXECS;L+9#j;Vm| zt!b|k5?1Lhx#fmmjhAa!OCEVV&{(iu{+0LIc>!kj$NrmL_wkBF24&5re(dMt^|~=T zN1|#=zBwgXeYdsTU1!Pe*q0ZmFzbN!gPhJM61H0+tqcWy69PS3CA?fRSO0VbAKBVs z4fQ>!u>;97XAE84c_cV?3H3an2NCy$iF>U5K_QB;F?MyqBT$07es`^?{`HQwC8&>m zz}-`m?)zF=wY)rtPe|*7VeEFL_eMt3m7JsjnLMVv%Rbk>?fdrSth25X8m8J=wH1ze z3!awh{Y63&=tp;TT(&ZXFglJt@MtbjlZoq?dco+R%tV;1j;?*nr-NL(EKD5?^cQ<! zw`+|HCreo#)mkbP4+hCsQ|h7t3|ip_0OhqGY!M3a1$rVQ*(b;Do%kE%P~Ki6?ZCRJ ziG(InEuM=nUwh8Me2Pn^R~-Zj=v-SbIIzyBmqi+H&8d7b&dL3<_!dKm{EX;MYfWta z?cg9DI$g=RUrs-%6T~}KAf{z_^veL=sT-@}haa8K<XfNM$Jyq7u*LX#yf{uPIT=gQ z7a?|=g-?9y$x>#G_@l9BOi?G}0pNX3%4@VkUZWUF>T&fsnSoWOW&V`-b33<($wM6+ z*L!HUzKPByB{}0DLUiyZgV?TuN&(36E`|q*e#a(upPis@;L`d8M$whJZ1;$Xd-qWF zf|j%FnonA!#{2$LS~=oHWTcs%smD8@whYn_0nbSgGL*(tge*!Cjg!FUSk-FVUoT8` zsCZiyto{DO=V!xmR^xd^>m%qp4(n~g{>3w`y)@%@MBP@NoMx<#h!eA%H?eezy;q*5 zqDYKg=lPZY`pC_3RMpfpNix%XYozJZ*Clndd0s#8UCurzyaX<Xwz?}`xWcBPo^H{0 zb<4D;blbyPn=ii5RMk^ufktz+I&($FXBTqs;OWyNIW=v1Sb5tjL%ndqE|)`+1S678 zYR2Z8fQ;Nb46r^a>k~j4E6AA$Rq~k7*=k>wxUj(Yf8q?%_n=RdRDH$_Plb}^fRS>A zN4F-hqM#cglq(ksLtA;iTxzxNBFhcvXB^VHwV(4W<etIpX5+9`?(gpYvRn&K7x|&c ze>hq3J9m0=vO>Gq=Cgx4wqa^h9>m*NpMCYG|Mv`<m0pru)cE)B%SFZ1jiy@7;rykC z;rzEi?cePJb^akRuO9ckJ%R@#@h9i*IjBl`@G{Bmv5$gBH$9d`%PMmIn2E(Jb+;QP z*5!w#O8yB^ksXF9pD`pK!)hH1VLgI%_@`a|{mu>l5V$>IA{DB!|3Dn2@0IH<1L6QR z__ol{lDJi~itm6cwh#|J|M~Jcsk?Ue_FuI<IR9KT!eQn>jH#QjWj}WxBRqGs5x?r? zff;13QUBR@Qu$ci>QjCF95vM8Yu&#6Z5&E+_nJN3I=gMcGHawUVd8c#p)Eq!$jAuz zNmT3Y53?()N~SmTk1v{$%qk>ryFJMmCj-qngwGGyt_uDTC*>Z_t=>jf16T$plw|)w z_=A~+F!LQnnyU5TO%fU{+%y|UP4nX@GRO`yKqHI00qG*HZKReKQdjQQh}x4sCq6xg zc?P~=V(M}}b284x0jIbFNSuBdmn(U12e9SCmq0CdQr_t=27N2JxgfKA2O4eF3ND}Q z=;}JDqX9JQGl1e8jWg)HO9eb~8RsmWKC?p3ZSVkzYGm-Yg+squoqPm^gx<v$LV;^5 z&oz<73UWAg_OAd9t3jpJYYUp!+z~x&XQ|=v&R4_3(xe2m%kK_|yU=L_-3{4Tjlm7s zL!TKL!B<*6fTDKcHfLcwyZe85%8B$;V&WM^r#7U_`v99lW%;x;a8C+rpsUzmvTHcN z1Gu^D4C$gx@}GKTYZRSF7S!<PRFKXX=FCmn#Rc8cCq+m8BK0Uwy|hGtr`8u9=m}n% zdg^_0cNoiAh95+qoSjuQ>Yuv)ZB^naoFR!9+`UX$b)jQ)Yp5UH(geN0KI$%ZuYjEq zjL-IMZ6o9pidyY0`XCfFu?wGe&SI9hf-C*iKXO3Ahq;f@qs&mkg}+lAop7%#n$_At z@M|S5A(X4FhxzPQn#S&BA{w3BcnRjt*l59K0?q^Reik}xsS5V*lSd2<3`UmqVV)Yy zsEG{=IyK2hqWjz#7|H{R%TU7OAk^WXKGgwt48eTcc7{tY0Kdm9z%R@hJqmF5+}w2K zC!apOPRdF8vl--V0D}$Eg?ErK=1~5zLmYi(X6DzGbKB&h;@YV$Seq#}|Hq+22+`c$ zuAfA=gO91XiYy2)1UdmdrN*KUuQTq~wH4ikn!>O3Y>~ni6}Rx$APg5Qf5W{)>RO-L z(_J7tq?Z=C;bD6SnsHlqFZ1%hMAGHqr(Iqbfw_jfC<}_B)6>(lv$G2eojX#sTYYc% z+5BLbkUZ#le&D`Y>jiCq?b9TuW^NmFAz)4boP^P`wGH#_V49L;3CT>jQ>Z{!nq`*n z=IV-l3a-W2zUKIS=66Y43BtWtzNMFFAuHzc_jOo*nwgsN=`^`8CU^4(%UJ1<%BaKQ zD01v=-@5w%YK+4|XU>&mc4L^PQAzdq1`Z3O#4yu_w&Z&Qd}e9R54NB@XM1wxEnmAM z6Lv#^sGlO|p0q|u7G~zy9&!-PPoct$q@wlVKb47#-KUwkx%Q|NUvEKqwDGG7k#SW@ z2;V0>)Ze1hi%r#bLceCbE>x@Th$?Zbb20V=o2ACZNtW1I;!eSya%wC8cYNoXcXu?@ zHs;yd*+J*t%F4<$VbU#_&jl>GujNwisTTA@{_LQ7D!e}b>>#z*H?A*POP<gYe~bJ2 zB+D%OND)d8;QAPQkFf<ZXyWm*7ub`6Cbw=kRP|uC0tU`h+1AFNDL`>P_}6x~22a3t zXSy<Fe|pGgx88?%eXoXVDHNpgnWW5Y^59u8c5sCb7#ygU_mD0d@Ru9hxx-=*rk`|B zB|JMXU0y%D;LQw~gXe3p6q(}`J%O}fxow5FivqTF&HO@<a=I%SmH2JdG$fHg8Z~73 z)}3RfGhN2%G08E1`s=mdGm&4oXuwS3c4cRhw?boyfW%fy#xoUE5;ZW5&2wphX@PM; zWExKMw!qXOodc4Iz|`3_Y8qt%^uupI+}O2nPt_x1E1AD{fkJdEDY=iNSeqjwmTI?l zWiUV9S}1?-iA@lx`G8a5xdd5xd!c(3w-`Uwj_qL$-pnDdr1JTzS=*kGI*D*R{2amV zoyxgy4^=wCESIiq(_qZgoepp%?CX9%P&xtmTbgWxxLUT7vb0Wm$AQJkn%0eL_nw^F zYH;9GXoN~a+U<uwuGmh(CXAPD@q3<tRJ8L?4OorDi=s)R)@F(#nw1OQ9}0BLLrFt- zLTk_QLDeXv!qv0D4M$0SWYmh+ca^7LS<ZO+Q3p6p@{yWAE`*00O?RkUHdFY6|GhE? zm>E4+T#YEm9dy^U82TD37pELDz!PWsiv3yJx`n5Z+l$X|w6JjQ<uc3p<Tv>0GvXC< zO^!kJNj)RZ3s>BCKIA9&4cO@vYo!-kF2`U-w{nDy%E-*1%-mV4CcGkBlUrSV^_G91 zZ3n7^H@mrc&L1l5R;N%K9jf=^sZT*oTB+-X9foV)wMQ^5l!}i164H(gp<%B~S!Uc< z%r+osV`XJ!QhXixocGmBUAfpnx<t=oaVN@p=EFPtid&t!28i>MyOyk3&BE7DNPJ#d zQE-d&?CXnam5QU-A1+?pa$mg7Ro874>)+(|PFHg`r$mXtew&WU8ei-!7gV$zk$@j? z+n9yxH(EGU52Q{}D7#)K)yhq@#7y|ZU6peC5ivr=3<F|Y>$o%5@5k%+Quo&Hb$Hep zMImnvI5<0VMT|vDel1^IRV-c{b6)oqZgdHeB6LCIS;OCXe)*=``qHLXEv#?j4SNFn z=J!P3sl=wpjqhz8{nzLo5t3-|JP{Sx<>v;weUGXllKs8uIOZgTA{1iaa@n$$tNW!J zgc^+{*6>Y%!QR63+IQ~ULDi)Y@ynasUIGTRx_UpB#Pjx4LO5~iB0IP79p+b_UkFVd zOO*$*badbdM717H_$o#cqC4VL^7<uhl35bTqxl1CV+q%8<LM>9q4HBeAs*ydU70JN zM{ceV3051#=kq5E^74|0;!a7b?kpfqbNCZVCmyWr3OPf?E2O;OeXIO+(2f19;*wWM zTNkugt`3`bU7od1QM}oQZ;p2$R2dW&s<+NAKW6wA%%mf_ve>1lR&u^NuBnO5^fC%@ z;$?9Y0x>e^%=XQN5fM!zFc#n?OHBwkHBmP=r#RrU(MLL9ZDme9HJ^Rq;`s7)G;g@X z>Qzy=<y?hj{nbuM$qkc{#iX7N|5E&S&Ak%|<NoZ^GNdn_r9!w{Efmu}kfcF0dmXe+ zv*^vEXFs_ngVK|;vqxSVKW|)I4hNKx$>s8GdEpA0sK{Y@@jstztvyM8gc2#Ye^)KM z|J!p9<mPfpJ4<Y5`HUl@gVP}n>2{A>WJ)2VhG>G5)99DC6Y1dA+N{vXGzifmZ1k&H zlR-Bw$dz*_Q_8$nrfeGxvn(~;oF)*m)peauv(@X_uf_*l@YDEa6wnfQ3h~8yBz*$z zbX~X=OU;??OM%c1IpaMkMO<$htqZodv%A|9+7fYxkV9O~4%S^)r)DeyrsGmH2Qf$} zU_}zY(eSZ>>q%Gxrkg1DQyQ!Zu!658`Kq@-4J{;JvVY`Q0ph~W9a2nswx=ifgSUu6 zOVs5n@+kP6J3<+149hb2_SME!0SOix3%PV$XH$TLYTCd}x@`mSy81{)52F*f%+JPG zN!4<Dq)CrDsC^L2=im2VU*+}w8G-xkZPHgG#O3KYBzt>$!mb||w&>M(nCX}3`S|f; z`Cwk{5|@0ZP9q7p^Ih-gKfcRoa%r^gB=18=4Lb@?zdn^P7ip0RjrZ>s!S+_YnEPI# zbP36LkS7uTiHMAY)>82^{CpY@jZvKb`)(Z%tUV<QIir2t1%Wg&^mxsCS+Uga71!gx za&ROOp0NV}$3sg?TjHi{nx<!kil#rAz%3a?{-*8+5$~l>DDdvOj4^uE^7<5l+In&L zvXmMH;y_;4^)j<Gy$h)5x<37LW+Mn`Cd2EDe=a7P-<$m;6T<z_`Ev*P=ba<u$fm3R z^E(YT$PjSn+%j3p!1N_*4VfG(l29CfaoPTn{)rD#FJ1-#pB?CdsEJZDU~dRshfBH4 zDdPu&X#Rayn+uh6*y_r+r`11@bjZ-yeM;sHuglaLTL1Z_ku1U%A<Se5>ehD$G!{w^ zQzo<YoF@rRAf+cuKL3Hg==nCRVK+A?PzS+%?2mJVe^1EN=y}KigQX?^c~Wp4AeZ-c znoHEcS4W=xokqA!PK~YaV!dI*#q{*>Z)R#vYX9+F5AOl1AMlxIIvjIM6<#7ez+6uL z3g|m%0$lw2RNyycsk)^l;9KVBpCqkLJnqj$PzyvTAYGDiN1RA^5G($VB(1{n=Lgmv z{og(n7=;x6-+oUbBH{qU&*Q85lZ8*zkBxl<&4keVANa)C+4(P`Ar`WJaZ@4Osca*_ zG>l)W<NHa96N-X@{rT2hPWxa(O;62P5fc?JYspIqIMat>OxvSzoPUV|e1aN=*cDH> z@qQ{q65W3CF#I<VN6(*KqQG(4&_<r=a9@e581C`skS&3RUn=F0tg^lBq1P1q)0DR) z7yftOJXpyVU+rgSz(PqOUxt&k3hBpyAO54KusL@CNtD0^)~x;d9{#;QjlE426Yk+J zV?U-r^Xn^Le!*|7A~$Apb=}sF70l8!^`~g%qS`iK5FRYghmerY`%_ly{6*yVAQ1fI zKlvE3jU<hk+1XgE0ld}~1NToNB7=UM$4-O?=`ojx=OpPa`}$K>$T$Oj2~t@|XEm8S zZ+!@2j~r<Ke-Q|=0}S#$LrY6^zeB<gCGn51i4^m{>`!9IuCyPLNXF^&>p&3AU{rtp z7;};Mck3r>OB-+fwfol*ArP~%AR=2o?smV<1tBd)veREDF|Bd(V690EtT@^dBMW8A z8`utA%f13zL~|i@si_~EfKga2qXC|wb?zE5p><a>VL7MoS8sEdR;HBJhlruXZDwP> z)StbsMZ#fWaI7J$rBI)YU^CsGLXR<+h|1q}z^`>Y!wQ(L3>A9j$03EF1zHS5J`~!| zUzxY3G+t7n3%biou|1(T-)tL+#VG`i|Db>hZl%&j)VIHnx_NGO0O_Cl>P`8b?Yq?8 zTWd~|K1lcLgPHmHnsY0aK<^m)EOPFp!Scrw*J_(#)gGs<cuzaAP$8TQeyy5-pGS7E z(1BdE7-)6_PkdO-Am=8zi*Re(WhVLU4e*XT&#g6ENbg|z`Hn9j?ZECV5<YS3R$NL? zpve1jhy2sJJKOiY|CFEHOk$4EgA9ld;NfAucjzL0`%9@G=AkrxeF|0%glGbFE3N3Z zJ{mlMPeOQa31RWQy!!jn^!4}Q_X}n~Jj_w2@l(?3E!fvXK03l=1wN@TsRqG+3MeS? zyg!q=S=9!f`OtC0J@v!%G5FO>;0|fU539-KHv6^t4yI1nGDIg$IKeOkM9g7TM04PT zL=r-qw35?KecL8Z73ctSl_ijtdeo`uzh5P-Ki8Wt;JBsV>i+fC(MMnUfXO98&**<9 zDari&d>ElZ^7{0Sk;LQHfP9JD|HHGbTsGF$0#4jz*I+ST>!J)2L8F)EH7QAJJbR~Q z`Z(zr9*K-vfDMeQ58=Nj`JUg2#Cu=xc6*DB_LH_*`|vV4jI_PF(Km&_psr^PDgxG@ zR7}88dUf^oa%1GKwK%J7o#QTbOu<gl`Q1^-Cy|#+;P|f4TVyXIN!}KcxG{iL4+O_* zBAuJ-Uk6in(jFKIcdz=%p}<q_TVB=j0(e#2JpcNr{An8!&ly%fDS!RrqZ3y#k*Ztt zs@z{}u|+2`gkG`S{xZ@s|C0=9I<eGx7?5D6f9+HrfzaSnrX(H3f3rj~QzH2Ri9~Yu z$bX&YUIe23#g8rjH_uFi8+}C)1vd6yr?w5@js=E_0ll+t48|e{)_xzZdeO!KlBDpj z{ROMcflGp^eYsQS_|Iny0U;$k$jHw>_<y7GkIntR)A{6T&%aOjj}?qDP;af;(Aqk) z*&yQb|Hfm#-u+WksH1g!+3OztOT6;KALW8KrD%Blz}X+C9W74B@q)do_1_=Ox=8w{ z5qy;U{~uv^-hyvi;If`!aGO@;Ctv?Tp<iO(Rtrr*@v}|9*9rP*(*NTJz4q0LD=URw zKYplEFa0kS@(*^HD!nLlnDQ++cNnQ%wEqJyPZ`h7>-L;qZh;uib!DBY#b6z6S%Obx zZh{~S;<pbqXE}8vOO<$na2XJCObqR^YB=Z@SvO)98(r{m{7myrOhGNf_VaxSNljlM zZfg_M;;FB_$K}Kwx1|h!iH3UxD=Vv+xw&?cLwa6R`jvT2wv>_iD5p8%<^XP~A$Q<1 z-FTCbG;>oOx`<sOoN;5L<9Lr)TgH^5l{p0XJfVlql`RR%QY61`>OP^Y6f|KQSi1XR z!TQ=vyosD@gFm~%iwE12<4dQM@oW;|mq~F7_O%Pdpfb;f-<~MY;T@0eBx(M@gA7lv z|B#v!*RvOukb<ChUjASjJ1nfDc=JnX>-}=S8Nd>jv=oj9<5-Pe+ZEEqdf((JI~VDZ z)nskd<ch11uUWeCt;R^o^&M4mdw`3e#scFm9@T|vVo`9l^(J1WrSf1<KXr`U1*;Do z4Q-;fxW~5|`rXDqd1_gNuk-gU`3etAn?K1XQxL8hkcy~g$krH3WK<W7b$Q<QnAH%) z?f-Ju(Q}#~CY=6*dHF}(#6Zz^tH#KuM~}L!^v26{j=C;ks7!VR&T{$f4WEBAZ!W$e zr7Lf~;j!s(u66^*HZUqfqVKwl1aUZ>x2a!ob-E2%l0B%jZQbE%mn!u)Mb&*LPXIOZ zZ#$~EEYf{ime#tJs4LzVySKyW!6_-Va+NFXktriw3fT%slgZtO0$T#w)2FmkOPVgx z#moI~x<5PV3QDisa`$rNwHdle$&Jkyz5*N#w46vDS3e_&ejkX#iJnb}Ic;e<e~-#` zEe$vD?f<m*<>64a@86@6qSAsQp}iza5oM21vSg2Jsbnb$VJssmZMGCqwh+c1Qntxc zM3%A5cvMDZANw+vVa9vjjOuynyZyfJd%VZ-9`F2BW9GiE`#jI<yw2<M`JC6#Pp@T` zYWg%3eKFU%D}5y`iB$)8$?T#9DN>1j=c$BP?fSkh+s+hUX0MTQW$sM^CZeqg%?)GO zM&tr&=QNv_T<_W4S`O9kR?rErE{w$OJ*ww$nNn-Orkn7k;#uF%i=yXIOGw>5f7oBX zOf=VkO6l4A7f)dsHkF9nImYuQw6fNz6<v9~7UQyYhrZ~^l=KI#j~g<lgFbg@Ma_(> z*OBiSJG5Z1Z{k)2+7l?+39+ykcG*hkFpphz(dXQvhPAlv{2*LH{;j%q(=*#YsIV`8 z_$Ij|zsaGq-HNm4bCCA$*EmQ84sr%(*aXpdTf>|bU&Lmlw0hm%2ZB;px4UEJKGqLD ziHWyN)Y5Amv00<8Eo(DbDB{w7hAi5-f5N%5-puX8C1itdQxoCZ;WwD#8(+LGz;)}; z4%tw>Ql0oCqXpt?_wB#6jrrI;GfBgJiL5$iSjvtS6p_}#nEoys);~h*yjcTWPIB^2 z-wE`5?>rfpFZ1ziHEv~YEi>T}zy<~*O<L2f7Ad>QQbkFgId?ahiu{zv87s+rZZl0J z4NDJ2C!6d_*Z5YDG&;I3GwJ1ckzOf+F92?o=ta#enIOV+y3|TDl|AZGMZQYT<S&eo zowV^b`wEF7ve=3?McGNSgForW8qz}>oo!_PfP@2BL;_vgtKHqJha5)nJGl#UXzR;- z*M?N}pr_G2O`*7t@*absRO*;H&HZ8zjb^vmTJFP)PC8Sd=hqI&TJ!pTr<xN<f@&iB zDXl@XN}o7{54)qS=DO|fZQtepoR!S{fd4U?v^jgqG<i^GwUBuqeFUH1;EUZzFAcTR zq)E+fDaSXH_oePMFxM5C_0$`qN~Vj%IabElWf>T*JjQxrWAWt-gA<$yL9_O!HFM5I z(z_nj#cK$=(Ck0wqwf->iFv{?OUP|)-IX}c66k^ts7!Rh%1)&=bme77#y4*1Yl!R` zDf*MhqdRPG8f8C}D;y#0k~ebZwI})XMC^%H>%?)DCQdnwiv!N-Th-V`E_R6;X~NDZ z*X`-+ZLUi;?K>X+)ddr-x(lf#-=IT&R5^qU`e1m`G^yyFLO~7W|HRgo9O*t+SImbQ z>U3wPp;^oTzbE93Uh~wMF50cxzyR~WmlIf%7;)3l!fuU_@<g;t4o;^AGh$xcF<a6* z89Pm<&6cE?yNr&jw3qZY@T^hWz0v4Oglod!jwq%$IVHus8;u_Mj$LodCZl#2-=MZ? zu_Zys-P2{Bc3We*>t5T<a<S^o;Z2Sa&9)!duV&#e=`LU8Az`(IXB8J8bTIuOvQ);f zB8~HrVu<g$+ep~z2L<8jlPrUCQ{5&9^QRp&q`=+cVl{G2A-luLDX9kt=(Z*YO`VpL zx(boUyJcMp9Q50ayITt0)yNsP`-t0UR_-&Q_XkW|Gn?ByslhHiQ)W@xRA>CBb!Q^I zZpt)6uy-sPeduY(Wp{vT1M*`vTWMY|V_zz%cI>u4QJGVg{b+X6ssME>*62yjZzHbh zAuMe3wf<iUy8*$x)ns#(mqT&%)T9S7!g2Ncq6U_U(_LR;f}=C2NzA4<R8re*beIF8 zMn#I8)9NcCE)nKhr7)u`zSD2IT4gDz*z<)JZfs{a5SB*EPFD8o=C5%f!D34i!JDfE z4U_F^qED;3Z%hztBDR|dsd0)E!^lInhr64_jitb@){g~^hBYPRIHuK;iO*~;2-1{V z^Y{K0B6d#%9|v$`9f6bc_MNa-&U}*kD2cVJlT5bR7PHOzNr}IvaL!FhyRG;rI=12C zV>k{`zwS1zrfU;VI@Plr-)Aky=6sw}C^<FYjy-11xI~;>aKvND^Oo*N$$s=oXPZRB zowDWvl(dKDjZ=aT6369ig(|6Emi5vuMe#S@`_P=ooZ{Qk86%PmIQ{74cIyyd7~GR+ z+)&Q(nvm1SRs+e^RXM4y9K>w2l2IiO2oA{tV<vHgyLX(0Z4EkRXj0z$o1Pb6+k2J` z^ed#neie}fYI9DvTuW>5i{j$Yw$yr;iv#+GeksDeX$_R&hQtyLl1zqKS9rHbLSM{h z={e<Yfq_mf*EE*Xvi2_TsfWgHeeAi`tXhy^Zc!30k16+D+1(mj^tHZNUG|eWw){$m zu253Gl>9SF5#_6eC01bQeUjw25mUM9=5l9Rf1!5kfnv7MN==GtN$pdaybuy?99@{e zI?9qM+}mB16BXxqQ_Jp;(ir(}TCB_YO|SHu-oTN6$y_t&kDouEDVy1Z<P2<szL#31 zkId1BT`5?egw5iSCGK7sbhZ7Y@{R){LWIWx*txDr+g!Z0E;AuV&-E~dezZM~td7-m zXG+@@%Q0%F_qxcSo3q6GI%T_|u90AzsSY!ycy^9RD8J6#<+ZOO)md<8OP{S(3vsOX zTgt)xf0Aoz93oDj`zD_`A2+vXI1n>w%;SnTeCeLzE7M>oQi(Q8jZUT(@>upQNz2e` zG%vYZ%lGi?NrO{M;kZTj(Wv$HR!jx}q|a|i3O{^W&T{(R&VglGa6WT1sbVH`+H4eR zLkC0g=xqZJ9r|l)T``7}A53ysPOBEx*O=C&)^1*zlPT`7R5Fr(dd(y=t)F^wbxzRJ z$*a+x<v7R^-gADtH+n?#Rb*<Z9;4fdU9Kj1XXnSv_Ig^a%CmJRh7-fX)J=mudlNek zd91(TKBPwyh@chQc<T+f?Autx#ujOk(|-C&n(x5CZY~dc({!rZy0H8<_ixMobQeyD zwbICZw*2zt%PN%S=0@oG6C}OwadFDid6c20BP!jSAV_`XH*?CaQ8MPtsCfXtjPMH6 zUE$@YZAS0Jo-zhE<P`<h)UFE<Y~B;+o+W@OC=a5WSO*KI_}Z=%|45!*xo#A5LfvCR zhC{uR+!dt~C->#uH^UH?Rvs-={u{9q9bc~*Ur3xycPP3uwu_1wU982Zl&2y2Li6?9 ziLgg*CPYD70RR}ZN_4j|w&KW!)YfpCG(d~VOq|o0iPFWLgPIDyT%;5{bS<;fi|FY( z@y&d+bIZsqb$b$P1D!@Z$c6V<Ic-%d-Ep-qRod-<e%*-K^Z}cqf<)sbP<ivHwmWzc zrVDXW`l3T&)2I1(Tq7nm^Z2F@+pN}dcHbC%9BVPr(K0re7wev5k<OQ<-=tuz(9;M2 z=RpBp#b`mPiW3^NuNq}>83idvBy0`o2Pa#SD(uwEBDr`6y%m+eT!i7tyrk<DOO%I) z(}>~M>h|krxyhtgox2uk7m&9<jQ`DZG3XM0|FFa%S2NRQXXRTacR0H#B}P*IIcLhc z=!>R1N1M97wO~f-{YE9umHIur%qaoT;N#tGA`EO}em9<%y1TFp^U{0sZpGxeeA1+> zh<>o7mx)%(lL=h+mV0F>zV7>{;}cm&X`=!B;mxD`b@_ZDuP99wdF~#w8<SX#pD0a> zc5O-hQm0o6luh|`PG40@#M_k#%Q1ZiY>Lmvv%UDD@pMu%F~-!uFKv+Ysyf>GYb?N^ zPFR8ByDD$(OHUuYWq<7al|x2Z&uS%8Hy|^6yMFMlcT|k7ev0}DyU?k+%JZb_EL;h! zYr66~g2DtJyW0D-bPj(GP2w`v-fV|Iq+i!*#GF%E#*8hy3RQmk`~4SXdo=U(tgRC{ z>Z{BviX^v54n1DhTQXMr>S5;k>3rX^-6Cq%J;~;+{fRCrd=c{q(_glSg^?PZ2@`Lj z{tBVzk>=z?^y_$dD)P$LP@&%YxFw?b!D(o`NtBlJ!V1V=xXQPrBB)!&Bjft5qU*V^ ze$P%uPTS$lN8`LAXWzd9;b!>k?G*vc(nC=n@X5tr8X{q}5gvaksMai5s~l~Pa|>DX zfh-tjq1cP;hHMj9b(k~canrr*Yq}Cx4a8sPk~+U6r})nPDWATjllZ6N9K?r6bn~ZK zs*)D*pu^JudqwY~cZGFztRh^Gb9QV%8~aR46WsQvrz*>_?2k?vlXM&bFa#*5T?<f* z&mU)S-d;{lPJm;)QqdcaCS+WI{BeBkXd^$bvl?f$9rXlfHObCQYnyzTi^Kz1MJqmS zP49M)8^76+MAH<(j6|%s8jxd$FBIpq{tPo%qPFcJEGa3Zl-=QD{w$qFHewt?dyJ6{ zQ-EHnA~7;uZpbD3gsQ+MppWwEC^3S`2}=iLpm-_B=(2j^m7J)Df25_*n>1(oJfS-b zHdKnA6mRJqJv-sHN-ILJ0(VuY1ZYO(U(qhxs;yDtN6<nDxc8A@kNxr`)ASP_=axot zh(-&J=`i<R9LCoiHg~YY$&**DNXJp+KPqDFdIRx0^sfwC^MCUrYQ6zl@Dr&-d`A2A z?8Qo(6xesaJG_pZb(Gph9x4lpkgVEhZj7mEb4wcCXl5!u^a3|GnIgXYv0(C$l*m@c z*>}2mCDeQ}dXJGdfR;{qU`!ifwMT+^SJ`Ek9<{@JkXA$F<VJrH2!Fq(A)BhC!^}pu zr92pO|9WxsEnmWGIaw>prKI+>CNceBk+(;5B1n@hjoi5z>jTDkgxoSbUq;#w3At^p z63+Ei{tG^Md;xg7xqurHGK|5ECB2@i6GeK8r0mBvEFIR7c+AsgYpejjk{!DRDR73a z%iMd8g@DoJh^2b8SNogt2#(c8eO|#OblGqJCUSpDWOtOMB}w|xN5fY9I?-x9MQW0* zn~N@%>X3KaSj$B2itIz>Qs1M1R;Efkfovvd7T=ZooOLw*GoDVJ{ZiTRQ1ZddNr=PG z_6G|`9yC7Mn&OM=BR|n<B9yJB++ub*Z|9p;#nD(Nrhg^)7#B7?7{eJM+ppaxI?9xE z+d8$ZErqE3*KMEHo&{+0DlQ6uzyYj;e(=y#tqBg?JMAYlXx-t@<kkf<M7URm?RfOz z$6f_RNe_nlhUE2w*UVn&$Wcw&eTjF!`ynIk;@AFi>h!MKzP^Nv<8<S$VqO1<{0pl^ z^@9Ufie<JKgn471St#C_pNJZq)zTf_cGxc#5}{Q^dtFHHLhv7Ra;}(lUMT!BMkT_& zyFn_Z>m-ZIfy^qO#$^w2?f!g4Nkm!wU`=1`;$(5hTvDpbqYA5iIp!W$r}wjOApnm4 z58&N2TqGbs$AZou2Dq}&7Q_SRJo3k)X5bN7v<_*G8aN)2pxA45M!-!=W{6yB^b?4H zW42^^%jTN<;8Z+~Fch;|#W`d$j7KErLClmu>xB^qg|SBd>U#pYd}bctQdc=__~Ds; z3j2{UyMiLNY0Gkx&4Q3ZgNfWJzKJWiICi63&!iLe6wQvA1bt}Sqs+D|+@0yBpLG=Y zxxqB3{cH`%iO;Jd)!@jH+Wx))u<rqDtx&;=z>%@;{xJUh*hnT5T(J0s#1@eqgf~1n zI+J>e#`g)^lcsZBK1SuA=&(Cw#r%mxnjU+%789^v-m9ViK$=Q}*uhg7zXx-gGtDz! z@?z6(e><Xu7$!1YP)(ZFO)=kG*gEz;Wip2!Gp9#soBaR?K$ojK1VQ;G1W7_v5F#RD z%iX>F$dN_whutu=4FO|dCL0;TK?9LPI7BT$;bE6$KT0x8Hen-pr5>wylOC<|wO8b^ z?kg>}n>LAv)S7-2@E$*xVzz|BmbUyfZdQdQwf$?#SpEy1OU28d>-V)l+;10dQkTL! zDUrcgz9`PINLFgJz68ln9Z7!aK;GkQHRHcjt8?;J>$N7uoav(6+9;kg#+`(&#y%w` zrqG*J(6I%I5*5y2*Qq+icW0Sqv^YDO4d2>T`w(z^Xp1?O;E~LYGnwRuS0y7iJKA$% zy5`V5X-yDjZBhYo63-+Z9{o8tZA>T0N{(^%HJPhkxBdJA{j_|e4>>ZPnM-}$KZYJ^ z)H!pFg9W7?+i*$>2gpvb>{A2P%Cu9pbHqtu!dwBMnyw^SF^3yDrZIlE^%Udt>(PfF zMX8-7_D8I_VBQO|5Ag$B>8+8oDAew5c^i5oY5eNwzL+P6w>sy^3GrEaw}XCxc&r(H zzb{FPQ*SbIz_h01uFeZ)*>KfTUteTPB_X1pHC3K|GkKlR#fMOPst0_MAJbFZZYb%I zGcXXP;&cSZD_=j_IKiBkJz=lQoHH;+y!m>HWint^`Ta(Ztd6u?J*+^CD%a=prGnX0 z$Mi!_ePHFKj&9_SN_MWN{GBMcMp>oHOj*rSOh&Vy`vMF@1e2GGZWg-Ouars8&&Jbk zh6e(unnl@!g87ZL6slb&G&pjOjr_KSc(nbI!M$Q{Tvu_y?B})jFgqeeCi-`*z)0Fe zV@8bHh0`4jB>o6ZIuv;2-R&0ZG?BeE!!TYJ|FVk>slwm<<7kEzEf$IuqEdQDY~V%p zuZUV-P2rnf5vq>90T|Q=FlY4|0TIP?Cuf%@L`^4H>8U$;jSbnOg2+i1HMl{kuec`; z{U`CX&BW$EbW(g(zF3)=V@3o|;amg6O=PVOl@rlb-CY>&&7r@RHQNUU>CUiEkMW`( zb2%)YlEwPzlQac|ovW2LS18a3cVZpiMf;E?_f$cmXm$!eeVyyVlks${p&@P^A-={# z1RIVk&{Q(!e%U>1iz`s%=_vZdJHTyNKuydot!(7i({i2oyc-T{b+_H_(4XkHbJc0J z5c_3p0Pb@fSj7c6TGa-**BzXBE-058%q{)N)1dvrY-fJ2_0;%`;YMER-Ql<f=1>bI zxjU<Dr<)q~9dTx^GM~9p(w(N~T>N)NfMYN1t~1Gjdn_hmLgvr|XE#50`O~#{UdNtO zJ8ubKHM>fEtHMQ8SNkyo!$?$%+B`M3RwUi>Hf#ji+FTF6TxJKr_unud{k!92`8H!u zx9*36QsZY(X_`koCyQzgybW${(>%MyNDvN_6N@+r`cghQ&rMRRy6%`sp5E9i&V=AA z>5@jCFL|Vb1wqZ>%~+S`39(}*RNXFLGDP4yqV9Co+}Bg;DZbt2GKV4#%S`}C2jHH; z>90Wy1~Bd3>{^kW-GMD_xNEcTFiI{6;vtx`)$iPwUy+kUH2~~RTzV94?oOMUNov@e z*yc7?Jg)U#OKQp~VOHTYArWoYP>E~v?pCHIk%M4z)%y+XV33k_I{?WjFxK#NF-?E| zM^OitBWlx8<>tS$awk$Wqv-+^%?lzhl<HHgw=6)`=ipOgMuL0kY-46QB32DJ-U}5_ zkAWkRRmnYx#doO_HwrZlswsqo+fxna3ro(6iRLLNoh+Ey^6Z+!@<RmLBQlQA3B#|A zo;e37$#~Dyr@u?9u=y!;uS<?znTuA6(lboGYurpA53TPLq}|I1|CQE6q!0QSiIhWI z8KCuFxntp^&%baUuiG?#CB1&)$6~Y*HZqn$G5nRm{&!&We>s9`_+1fUvm&EQG93At zqQ4Nbzdq#ubA;@}?ypw$@kYkK@~{Ry2oGC*f2AMjoff!TG9mTJlP3hjP;$_}$#t{p zI>_i@<^QimM1oZm{DLN<%=_0|zO?J$nIbpG0}*-{<%QDlUw!AlA;TA`*1sq${(_4C zkGcP=prz@F0<B2u-Fex^qEw{r;3APcOYO_7pD~0VCE=V`LxAjM+v6&bVOcYXY<E66 zF3_!j>|O*tC-~;;U>lAfWFo>&+}bzRcfa^MH61I8XvPdfQw=j!z;p~P*SN5O&#}vG zC0k5|xW%2gMLl=cp8p}sSftDkNlNx*?wS__eDq^spO@C?d<&B9jQ}aoE4bM+ga?=W zi82>W#7Mt}la<E<4ScRO_KFv%=q!iJ8{MoupXdOp5BZ-}A1cEG=Z-|F3|(fvNTp4- z#iwrHDHhJORB!OiAzvgXT?ZbvdN;>H;)G@%8ev}Er0NQ~Y!Ec5g5GGV5q`{vU&d{% z>BhZ5f72&fRxEDwKfiXe9yOw<jvGZ288R}H{L9CNZZQ*2>3d?=JA)Pokqd<@u^Qmd z<7P+I$9Yz##n)G#xB3XW7=jo`P#8-Yzz6VB1<e;S^%h>dAqxVGh5ubqUNGOn|K6<o zkH4tkK1eE-2BDlPqQ$jm$j?NL7QF)NE;)18ypDnGC(YWA$4$uG1?d1_R)9K;87jE% zx|mIEny@m9F;v5r6^qp7)HP5M-M@p_{SVN1EGjIzKoJ7b1IRO?Sw_k1M|3tfe<H}M zosUGF+$Ak7jfBbX?)bmSD*x1+54&q2Tf#@&4lxvwi^`0}@JBM^Z^wU<x}abIfxA!A zdC>V3i2m^Q&5;4%78bRdKcexz!)~zrqB8J@jtN<_KrQg`<K4f=34gc-w8S7fhKkG( z5*g^6zu2@as0&5U{TorUw*8+8`vBQQ`Mh9++cFUFmBQSJH@S<#6u9(-!70`7Qe9+R z^AW+`<XWc4=Z!YbE=ZS9tQunk#gUpx_327J*Kl)m59fgSzsWHEMtcqFQ0K<A79oi4 z)9?gxP1fxTlL4F%CC$RZQXemM7nxqJtC+Z6yN`9ns|G#0w-c2X^<7}YP|d`O%G0N; zXoPbP4kCbDImRxDT>-L1sA5oAW<L~Zi?et!tWVcw;^;iB`<6+a(N9B)!L;!2S6-Cv zAf*Y&^30!k++5V<E%Tx1e3vjShWGQ&4!i3EjxI>=e!k9PiGwsi*p5PWQbpDs)8BlO zAr?X;nD5^+mQ>-bh|+C6im3i~qz=}l!W;AT4`R9#4EYtTIE!BaA3pf5BP&o}fymqD zU9f}U0@*u*>lunO6l%|%@7k~b?y?H*plnB4AYf@Z=DKo0rkHhY&4<qrpV1%;%vyBA z>{NRYWCUba@vlsRNKh6BYf=2GoDYHPoWs{Hs2lqdzZas5N&h5Lh%P>tk;sn)YEd`1 zEA%*x(6OLdz53fvE*MS0@_A!#P>}vY_3)3q2m-ssX9w?aSRILau2u>w&-fLeN3f&m z`7v4$nu~$Obg&i&B9z%#7`)@pUv7u0Y_I;6-iE}Uuc&=n=Y^X;Ir%JN$G!Sl`S}}V zc@dc<P!X*=7v-EN?fdsOiagi5E~_qTFfR*Sv?Gjb59ny&H*JQ$zU^7Z^y`%m0IDr} zj1_DH!`2U3Y!?yQzZgwEdAf@=<kE*R^e-^b`nOC5NWV>WkUZT8qxjl$`s%n**oMUc zy+MChOEML}FT{IO{d%2wWVc!9BdVB=wo~(k<l@u)6!mq`kP<~l5pN)h%&YnQzw?ne z)CqGIACs1~LvrGH{&~QEellbyj)J5)T*TaaYS{9NC`j|Z`(GT0=w98^pyJ}S$U{bO za#!flpd^m@;bJw-;Q24z8ypxowt6WFZLm1${jO5T1erdk_VNI*TPjq~92bs64aypP z#v8>WQ8q7`v)H{3W%nZ#<Eb*h^*wKuf`J6`t;@Rl-5CVTO?ze59iNfBdaL*ATmJwj zen{^<icI^UGMJyy+p~*{5uFzW>E|)&EjBWd4S(OFrs$%9M>@k%vG>5q9|M-!fwCOv zP}4s~`-jY**bjL^ixg!iT1x5E{=40D{9L*SU=?L2hRQyYa}M1_^!%uY-~9Q#D_q;p zyuP{<t*oHMh##R!$u{pvP|Fk#H)J=h$P@wLSYDyWPPB9Btq4K4b_~g5`VmGyoGgp( zV%MBQ@4LSZ>9#=~a|0u{?_nm)sj##51kDSO3vZN3xufIP?Yem8P^~A2Y`NOolbl5$ zeIv7A+G{4Gr~Mjn023n(9Vmr$Fu`D=lx$7{vfx9HBNGh*51B7nt8zkop)ZJtuJi2) zxfLQ9%L2eT(x8Fz*><KF?c(I5n}8xhi@<zpWCm$Qqs);jKw<TIvJ@7|ZvkMFp2;6W z6Tg;wDuK196x$a!f6F(+TX=cF@#UAFf)(XFr$o^g71|c$%7z(c#l~mn_7<p1Im0y) zYscFgc|D~(o{&E{`vK;FbVhNmYX3l|H?Vu;dgNjGtViN&s#tgj%+hrp#}_7@<P<7I z^(E#Oe9CD!y9e4uRxbu$U6S8R8Aa&f+P954D;l}o8E8nNQ+2$O?jRDu!W&AHE#2nM zhf7O0&yAV&ePx}%luUp}2;nUm?k*bCX5!!|LzAStUk{^a^|<&v8u(?|Q|3;f((5(r z#ua2acEyh?r^oq&%u=}3;=t>N=H{|i(W!3JkD60e?YwTt@FTjHg3~-oE}6s-pu3vX z_qO&!w;<VD;Wki}<V44wFQL>jgQZn05n<L|DWsd_17im->gAi4#qpF|yFAiPK%p)? zVkoSSvXiGY?V>F}PD@SdVOnCjFs-c?ruEmlPRHv-t)yVr_hZWusHVknO=Qg!IdVKR zN29MiV&kZK{L(0S^>)itb%~drZt44{o%#F$jFByVRuta$s)+j0B*6J7og+!pp=1A; zOh}zu1ju=SMD7Z(oQN~t{{)!8`v_U(QZEsB!Li5W<(CP<Jwyf9k|c2Y6gS6Sl#jup z1F~3Mp<H+BuAQR~&Cwx=ZWJ+lx_U!`I3BtS_=MIThctHb^(S_iPH|PzCHEX6s6B9P zAyfAbKvTJgyB*qcJe4RMp#xFw_=X<+=7Bl!A|f44c1mg&LwX29R}Wz%e3t}-hjuUJ zI>Uo6k<t%#ZA-Pz3{&vd2#<a90vHyzrIgTGHkRBqqtL1ejNkEf0YvGp{LD7ucmknO z-{E|?V@5hCHd^LG6}40{;-o=t&Wb%C)z*GI%cilATzPYgOfVNsr~|HA)hf#Z!vrga zC}cZ_L`^4(Dp>(9z3pi0icC5Mk_T`FDRk+XGi+-P1_!>^ZtcRW(4P!NfyjqW$2edB z&2)F$hIy)jjpeaMlkswM+Z^yoee7I-%QHEEmeh0(Ix9{1u>=51XSq{Hl0K`@?-NpM z+frtial_;~cP%(T^uxIqjnrpY*WrLUo5QNhSEnXlheUhI1czh@A@yif1){+}p5$EM z+^h5b3A&@a-^t@a8it?f0%E!XuUa=Xf|g5YYQo2K;?1jL=oZ)92m##z9UZ*^M6Ric zYgOF*qhk0fmZGu2JLScrgB~S2x`ACc6ff*(96da)(HkvT5#x9*ySvoa)-kb9_hASB zoM-BY&tA0<k;E}`(~Dm3doOZIT!CJLYwDYjM53gz3laK5S8T>vmUY^~2sj1ZkQiuY z@P?%St%L~cI?F&sif@yeoH15FB4KR#wFgH!VXBGF2#MfE(MNI>l#cixjW44k(C4u8 zYIk!PJd==IxT05wUrYZVMIN1dZ?uFiNf6^j8{Y+xdUw!~;okn$#`r5@4k3K!)Fm*c z)7h+=QKYp(MxelmJt%-!ZB-2@M9LsPnU{FNKtN~Bk>9+fCxDUd{K6)EJk&Wmw9upD z<pB28ze)%dNVw)g7h-$B<j#h3xsgMgfh?6rE(Z-`5v}%4Q-b*+;0s+^Qmf)7Pxxyg zEwnl67C>O;uoJKzSOM8~CnwFcU>vTu=7gcJlz}2EM*Nmu(l#tbFv|A2QhM^L+U0Cm zI|IOVk(bOq#)7uSnB<6=iRwB<+uVRBj%AFk(_wax_Zmr*&<7K}ZX#<+s^jWo2JQ_F znWtV38+>7bV%eg@oL*&3DjwY|QX=H36{g0S;KVH>+5LH0dbZ!}3Kn-R%hGHv?A)ar z2gzUaRJbo1neOC3`s<*E31pZ+sTuv5UY1`S6pU-oQ|#kj%LUBd!RLz4>0a$wa{*nh z6mlRAL+*dAH7dq`SU(FgciANQv0%kUmsAj}RhYX)qtukrm+S72LHPOjP3iuxlEW^6 zbn>Oeh{yX<L0x9d3gkE#STj`^%WPcLr%!C&8qqN?1sp8oJGs{)ZNe_JqAIYoF#f?O zV3FE8t7ndTk==2yj)Lm!K}JmYQn#1Z3ToC|h{@T)6zpbj6n)-E!Hcq`io!u#PVNeC zpb)M;ah38XA8+&!PX(ZPr-)R_vK#|bc4pY9!;5vHeMk6f9n4%C^3+TEV37e$*Un06 zXfg)c(O((bVFes!@uWokNstSKg(hE)h@=KQpFs$bJ}3VD`oTe>@&qIp@6l{DSJwge zC2!T9wK~8bJy_W&oFJKvzI((>o0d;4^pJ~17ZBB0bI^P4_|C8Ks6<8yd!+}Kq_Da5 z#d&XOozpmv;eZW0JJwxSdHc%o?)BNpk!eK=JMA#N^Z|&Aj>Af<Vz(w%VzBPj43a#U z#*&r8RZZ4*N$CNpS+dI7;b~C!fU9OTX&-KMMMKE?HL~^Pk$p*mtP4ixP-GZ=Nr(JS z445t;Y#Ql%&-QD%g{Egh7N&ksNr<g4!n1P@gXUL~N`-|1Sj4Tk3n_YvatY$M%6r+n zwg|>jrTHg{conRw2{qB4iSl`*)a;&7C#~v|WOAU5rEsvyOc18TrRUC$E?KMgnhzV* z&^B9u;!>(9c{<qZFVSpb$%s8T>H+ct^cNSQDP#cTQx|YhVN34O4ZKaJ@GD=3PnkyS zUd}hiI_LLtCc4&P46?F*ICW@tFreoRh?Faa^krMiBZ9E%=!T;FVtdfPTzwNKUy~Wc zrQMoXaN4`mS5Fb0NXTluN3(O|@6vi0RAh_|F!L_)#A0OKZQ~0RDQ)N-raZ#|Jt}=W zWo@?0YU8SUi<YN5IPyJK8#+~>H}_HM)iL7`EK4R`4%DjT2O@tCc54`Ec@b4gm!S4A znwJ>8d)KaABfo4scYslWl);e#U%z!Z+a<(;w6PIA18cFD$4mo=sgaidm|DD<anRc~ zP+7!;Tbk|mAm#AOI|}BJOn^RzRe0T<iXrJ4W%up40tc)5{ejF7+g!MYL>*>-x%U34 z)Sd1<x_S=IUiNp$<pFLd+P^6<`;%YGfWx!87Psw;=D?x>4x~58b*1`&i2mG(j|eYQ zuGO#^l=5-)9^T!p8z|Q&8;8DZPVxN+`A_Uqt~Vtw9~h7wTk_{Dru9loL*uT4|7~;j z55q$g2d*ok8D5N)(AUl{N6t`~dxxXW02Q1Yc+SJax>Rg4HoNB9x71Jk1=V&{&siZ7 zC?C56<mFAveARg_<<(tien#e$2>hmyO5fLEpCxc^6Kd_V#eFAg39`QJb)DH-pl%dR zqwW4Uu%&Q{C1_@Fo+_+d%4BNqFoYbS;MGPPE?t*Ue$Bhu?=%#c1g+M@n{C>~)y{2Y zB8M?yL=ywU=ZUU8@v<AH4;oZqNHq}9ae>Ob@{U;aO6X*G5W}<;FAhgggtTR&E2rBh zhwqkeO;9`J%M49GtlY~{<C(u5RRYp+64UI;&b3O}+uo1Z$X|CVYBQ3N#sVjUiJ|R7 z`A=!r&mSXH${1al9_>uFL@D6^=99{EIv3wUX>5drI^mh)jaek#=Z+%G*{#pmwl>@Z zQiE8kvL{*eCa|+{S)Je%57d8CzbxzntY3OA-q3;p+nwug;k3ZT0&<Ch0L&>FkSXBM z6kEsDNj<`rr97p+vj^VrKOX|Vb+k#vrINi$zGM9Du*NQFo8#V|6OH9Ky8H9{8!$0@ z<0%%)h4ujx`R3zZMzh^ApFNDFDeMx66%l>E2Is2bO{g61+n{n#VaK|;E4M?*ix90_ zGPlXA-Wo5@S#K&b4|Hu>J1+!wUHKk4I{}IY<dlt8J$nC`$TinYWp->Y6kQ!5zM)C- zxB2Qf={b18p%^vc$j+<&EibT;NV_HKjn8~nWr4by)~?VfMA@ap4%z9M|3N{*HZO=? zq$8m!bG8rIa{amVAQJ(t;cDp2M81n!uDp=>7?=_A)>6g-z<`I~9UmsfK8b;Xz+2yM z`W^d0StuX{xs2!3m7#28<2^s67#PiWq>M4;D$k8ltQMO1euJ$n#5kiha}loifwnQO z@%x?nLI?ChZT$-nw$KON1-sH@NxqCTuh;*xK}K(emguuKKk&4uMcM*&`@b;JUm+Bf zsKEElIyBP$H7pj4hxlaGZAK;X9hIB++#Ue7X(QPSjM{hP<OhVf&^QjP)JWfZ1Zw#$ zTEa-rCAIUc)vvhIg3-Q19)E=l8Ew;p6H5Qsk6-=y$pQZR3^V@O4IXyG;e<O66<=;G z;B4LV12+A;(akY5i2}nFj6P??HjEfXK-odgCA@=313983S&Z9Cz;-W_inv>aml<`y zPq@^VE&S0E32Hn-8A;zl-ILOuw>@e#xTClL*dYLrV_RXjmJxzn3bhKU1=|oHx^Hc4 z?|g4~;KCn(SW)mne$o<-9VqZ$F`V>7(m~E`SfD@{!4FWP+DP;8JhT_^MXeP+H#d;q zHvgD9%l=O7pp5Jm*_?%JB?=qIS!#CWx%J5DHs8U2#vjeCk#jMA9I>$As`3dq9#3@F zUyl}e{n}>!NQc!7Ze(GOwS4-N%9shiUS0-pa`-8<JA6=?y~(j$pdFSdaLgY}f;?Jm z)PHqEz?V6Pp*1chr+xR_rO!>SvT1n3w2O&-v(%d%JKhQG3NNaEyoqB^%JxSejM-M( z8ysl}Ta8jUsgDvLbCaP<_wy4Rz+*<|fv~%EZzbxc9<#zhr`mzxk&#n0<XQBAfj;-@ zF-ltvzLHa%?2YhHg76Rr&YVQ;Nf1OuX7j_xT+jY|0jpJrk=!=_{9UKlXQJCJ=YG8a z^))}%96mlQY0rzQxFW_Bdlyme&cAM_urmJT^Z0-Erq2bwp!eO~-P4{Vd2AbaIqL(U z+^aTjK(VH<`dsLG`sr(Xdps36tvGe@{=rz(#S0$4d44N-_}I|9pD$3p2?>+oLqC7F z3tcSt{%z;glwXWRb(i78;c$dZxX#pf^Bh}b7g6u@7Rq-(*}z4N*MIb;!okTIn&+SY z;8}lX|Hu|IJsjND;t{mqa|`c6ap-WOYW{?SW!^|<JE8>MmM!x+id_;LAy)1r^V@i7 z?xVcs7Y<jaAcq<nv`C?Xzn)#9uvMR_4v$l!?1MMyw;0#fJH@&9SYmKi;KFOI?*Ht~ zx@wxYd<;IR(dIW{6&c}l*<VcM*D+O3Jp0w%<=!w9(w(0za9?be2zs91xwg64@Yn19 zpMF+69{+UA`G^+=WB9AP`r9p~?~DB=z+Z0s%><thS<RnA_QL%UUUsKiaz2<L?ZwYt z|1@xD&5Y_VkNaP`?zdmBs~)(v*{b^RWYBNM;dR*F-|N_KBGq2xH;-Mu2|ps*o%YKF zjC|dCdFJY$hBq+UFl|XbKRKK+r2jrFfBS6z|4r0?FwJN9kM{~#<gk1jMC^9_n5J^Z IQ48Py1@q?rsQ>@~ literal 0 HcmV?d00001 diff --git a/website/www/site/static/images/case-study/linkedin/scheme-6.png b/website/www/site/static/images/case-study/linkedin/scheme-6.png new file mode 100644 index 0000000000000000000000000000000000000000..1dadc4c9126e5e7de0513625c4d8f507cb8a739f GIT binary patch literal 26758 zcmeFYc{r4B+&-*QDHZBl6hrbw*`=}$vhT{0>`L|+*|(WVn=FmWR@N+o86oR13}p== zJ7bxUWiW)9!Hk)CZ#~cR{`vm#9?$a}zdwGC!*P#$?)!FM*Y#P>&*wVN*gIzWr%zrw z$;QTZ+R#A9f{pEH0UO()&m6~qGgmr3;eo>me*@b<Hnt0*2fv35E&lxjoIDh0p|8bO z)+ay&{$TgGZF-xHts;?g|G`l<Hcx&-o!eGnhe!<0OvhlT>^^HTmCk;2psnqS!29$t z$zmP4nuA^L`LK~umv45W0=fOlc@Fy;P_`3I5i|}1HE_{X8`aq}7N39ToJ7%!Y!Qhz zmh#Cf!^Y<_?kB%DVB37(1(l&UH~CbG`D{!Cc0F?UxK3xl>a(D>s~EpX@!wU=+l(5k z>nDI$V`CG$?{oOz`1BaJ!I6UxKe-Qk9UR#%X<a^jaMaZ1C^&TR!ILBU#=(*8wD8-5 z31Isq%szN<WV`V~^U}fb>8t;lhvb28Y9Zeb&Va5_)_VP>uTd!fJ{2*VtX>;Uwdnkj zT-a~+GeKg4uoStoZaO_ZjVdY<r%))OPb+KJHoEIfGj2(n=BR@Yv1yj38ttzJTp>+n z4ff@@vEnF+_gvyUdz*SPlx^fpEhjJ#XMBaLe)=-&F<T*GbaeEF&#2TlVAR%Px8BZB zwv(xA25goVF9sD7jpRm1_!>#l3{Le>f-Z2zqtAqmja#fNhP+N{@pSQy*H{W+T=`~o z;dZs$C8d8hk)wTCs^{3)Uc{a#*l}lltEZ=(aOs{LN*QbdpD4^*<^|n8`!yF|<zYTU zI}R+2*johp_EYzY?RI9D{ew6jy(9C6LJ@3iExX4D?}DbslGxJqOz_yT@+%58I>ABC zIVWPHH_tr?+MhVgrWwf3(!|>57=AiI@tpiNJZG~S9s6^7Ity4uzj+~p9je&V!l8Mg zaT94U7&du=56*mih|T5jVXyPnn&$@5udnGfoN>9bTXmG}Y3#>g?t{f;^Zwtq15a~H zYvlV&c}UT}_9s_~x@cfSvmhuC45a(j((NqZE)Lh2kY9b#LhTGABJ<Ojk8}KGtZ#%7 zNWn5)T^xB))-WZ1u_(2E+j2ECpG)emfou919UDwB61C1|r47@vf{*mS5xXRqvs))h zsp&OLapgr;{~ELCd8Swp^rOUbHDThg_rd5tp-iV!K`2>}V9GFlGG(w<e9NAYq+q$Q zJa4&EVs7qR4pyk?Q<wPrsyFhlT{O@K1-NY{R73E0Q50epiqxJxT&WrnaHC19z6KI% zq%r*#UujPWW!?&I4hg?tl9OVW7@_;u89l`;YqhY!i`Bz4ysO1V-bXKjd4oj{XQahy zVuK|;ej<Lll<4nCn#s1yX;h`C*Qk@|8tTb~INE7^*ul%PnYymju6=(tGfEvD#;1bj zN(zg*pN$Uh;K5!S2r#NL*xY*(pN@y6<7P%uJ4zzS%T!5ZEy)}?dc53>K0}=y-oz4& zq(MVx51#hbO7Uf+L0ZkDy&z-njMSq)Wqr+|fpiN(EEq*kvsGd|zQ7IfH789LB2(8p zvJk1mNq8Yf{P6T(s>;ltNfffC!g|u#iB%dj-&+#n=>Bcrb$hZhDYbB6Df`{QKVUnf zo5VhNTqh=Bv^I4=)xEl~y8?L2piq2+hrW?US2hbn1SXJ(3XatGMwvV|n8IVkgZIl> z=~(+o%g)uAnV)yCB|aYbtu@Y88Oz2dYP9}fX(&`4N-NYbGrrKQ@^@Jh+Go8A)nb1i z3f4CoQngI4Emo|_o=J(AVGdO<v7*#T;PoX`Xh+kHcXoCXC`v`M8CaJ2uc^Czl|KzC zM1$*}s`&eySA#+42V1?m=Jsm6q*0{trewr)8tuY{DL4}LjtVBE+Od4yy`g>~-mC@k zu!WcLJPJzS^x9CoSJV_(W>iW75&iE6PaiMA2Rz=ur`+;mKEW>1RF7#blT<?rM|M6N zOO`wQ=fvC_vN7N)x*5M()NAMktdzxMIm8_50nVUiQj@Be+ITHCeoRlCO}XKuQVY(; zI!E0yy%4DCfpim>HO-+9=U9PR<4OE_U_JH;p2vy0pB0@(#`7l<H>U&-=KW62N#cN~ zA*xiQ0e6G*lQUM)+H5K*g37o;qI=nUhew!sNBZ6Krne^y<5j6c6ZoMN2?9OFN|~xr zkUF|q`c?fOdDLvn+AaJS>VWvIg2{DS!@kFZt~{Hf?LIKLwC-1?@L0U&?tnN;x%}(Q zm%UC@$v>AnX}pg9)VJp(%b>qzt3#$m_869x+>EVwxP}mfg5xS^?&^^2Kr8lXCq_w5 zLpQL|4APHhyH#e9#&+Qgx?by<wtur1pT=RLyl2<e9OC1I?*qt6#!k<0gGVHuu9bQp zl9htb`Iv-na9_lsxbCf*Hzm=q!0eH`&Kblwua7JH!@o7UuF9M{^anY%zl6Sw<u27I zYu@V&R}wfs#^awhodt&bTzD9I`fj_xsxy>PG&JffThLSarT!inBU7sy+P9{r1}DTY zZ@`;olpIw<7G`%(D-yDN>EF<94}~6{?R=AwZMFk<xBlzX9f_%+n1`$vuR&QUH|Ag9 zylCYtHRM8n!E`g`CP}?sJh)#~g;LWmK=B7nvTl+h9~c{@Z2BAxC|;Mb=8I7<k2IcM zxWXnAGoag(C>oF850g1Ow`}ORnoA*E`bSUS6Xcpa9kIkalrK_!Di#{H6jyTW<v`7R z8*KR1fE$mK{=&_J1($<a86-}ShX*4}QVoQ%jt<Fo!E39Vq-tsfGE@*BGiZpMtcaaH z13HWxKg?W+w20h5@aQKPo|lx}_dz#q?G9g0J-e#SJ<_Q=J~>jOP<|cpsZQX!J|Av> zYUp@r<YX2+?ReeGco`+}@<%d<7mAgdBH-Mh`Sy_1tm+__z59Tf05i5;X$;fgs^sJm z)aQJ=Rhv3Ou92B+YED<h2e$o=9|$PmXz58(DC?qLqe9yEmKHmQQ^29Cj?O>q!~V%T z{r!%w*1=$&J{Ny0SRl>$+{1(T=ZDmz^&haNAeQvArToezM6chbQe-LmYoi6Fu7<LJ z3LfRRpl4Et8O=@aBsAzt!-Mj1Z};|kQG+}Dwo23Ea}*AkC2j6)xxQ!Ap6G@kd7moP z$jpcwJFVsf>m!D9J6wp~81=~MKSKCI<6K1_CUShr2kX7~Uh^;`5xm!b<XhPORFmu2 z7e<zdj#GGhT(qVdx>UAdcRIWQp`{kuiPn(u-5#&@UN}B<(R_bfvnHgPbTA6G4)efc zgYP(P+(}$n9v-tBnarp46j8=(pqd#yC%$rlBVAIOAz8b|QqH{JJYTS8R=ql&q%ndk zlC8m1BS(4q!hFU8QG=6;#IhgwU@ngsr;Ed_%IK^f(k+?h-#e>i@b0d1Q!n^sT&|U= zV==ED@0CUw8*lxiJAb0d#OHXZR6HqdXQ=@ByjB?#7$Rr|SzF=C!=y!c{7slKekYF) z@(7ww!h5t9m4-P!ZLchGWX|tVnM>PaRDy-c)p0DVUcR>1laVqkq?cx8u`)58q2gZa zo^jEAYFf6isVV!hY+;d}%4#*niV}H-|CUA;?Cs0Wft4$GC!O=bR;Tsdp3GK=GwZ#x z8cCf=&pQ*1R$BZSS*`(XF-^?U$Eq5A+RM_D*+Rq4DjjxmtnnK>4=r>Krq1P|$*d+e zzZnx~%>Ls^{L}h;R;pN~MxZ_1nHX^;d1fdL?|#bh+Vo(RIOvQ=Hg!>$E}BMHf`>CO zTXbDyknjB{O6P`wYlRy;XDL9Ge5ZC}D&^q8PesgK`OhmlKvp(LQ!Xjlxy~!A<;i7a zfme)6`OEWDw%*b5=|6W}J_8Yq+;`r_`VD`n!oz^JWqnZ6wZ9Ip-*nAE7$=Kta(<*C zBG6~?>-$VTRNrTyRzyrRGoyp`&m-Zm1&@jTaBbPjW(zFzo;^H%Wy_ePF!ZQ8XrScy z;dyE8>Z#Wuyw0>A=pea=pYdJCAlUW6sJvD!astegfjRs29U6eTV9Ky1v7Oi1q*DkU zm97YJ0{;C|_Z#ivHxvBBAk@H6YLzPe<MhTfh6U?qPF7o8qZwfYQ;xRD_O1(!z<shB zG#b^v|H3XgR|Qvn$z50$=IxS1hEfZu6C=4ngv@XhZsGei2YBzCO;PpwyM36+;kw{S z&b@zM<k!%$kD~?Q>z45A4;Z<J&ZKR?S#+<SROvki2a>Scp;%~xTd_Wf)#<(ea24q1 zR!TMmBh?z$NyaIS=_-EG$Hi`&RMA>jI6l}XcKPwx2~EN^xJrB?Ej(O<;gYHi&FQC8 zN-a@@Y?18si4lA2Yty0R5=);*`*f4Mxcb^GJz+8J#@WS@BZ-iP?Tx`oDJ#L!HyJsC zbwUTI`$HVIp`k<&zNAX%f}cbJx!d!W<*m16WNM*N<N|l0>>7y+E!n(gj);^*UH_Sg z6_D$E6e~;OLfJcK#|%sF&AVE-5q5@}^kMHKn7)=Pp1q@9phm?7BjV$BE8E=;smyGF zXfn0T(d%#4s`qdl+3x9AcKroo|32W9&+&QQBYd#6GRzmqSYC!!)*EBW9@MXl50~Xz zHUfS@Lu*OR^`VZ3mELFhx11o>FJ+fRm|X>`E=`5>_t8P3oNwv18W|D_uxvYzUW0QP z8`MYqQNo;`bNc;QgUyV!Q49`DqIXML^d}8%X*Lw)i`Mi%Hl&>0%Co=*3^xulD{Tsc z`0PR);5{o+2F$sy^FZSJc)YFV;j?c5^m3w1p;u*0o2%x~(F+jVcDz^bk;LC4Lp+)g zLWQL7)c(BMTf`q)AoAo&m;*Tmkt(!7|Cw{<^UiV-e+Tri?$g%U53Y!=$)JCPgKl5# zwc6ZVg$UL)dk&l+f_WPiOplqX?csu!vU1G({UX)D^tC`#RbYr+(A){4>zL!m#eR4= zV5*lO-S-SI>m(yzM}jdl5Mhc7Of)OCy*%|c6H_EgqK9~_FaZc%sqp>}(|cuR-N@<8 zw0cPm?CNydU^aZm-i!LVZ+P%J6}w&$Cs*&p<`^<?bZNLW3L!yk?$uf}E@xInpWwo{ zYiu-m_`3zN|44MhSW?aH>h?2-hE^We&yL2EM)^5^8CbmN)AB0JJ3b0^`^D@M2WwnA zOpIK|X)yMB;noxflVkzm;QdSu4Zw_Iz74MnJE96k9s7|+5Sx{(;2`e|Fu2CVr6y>@ zya)+p%_C8vLkY>WV=>)0u<ccxI49EM)EOjQrSdY3koDefQmkJzcJw0WA}Hf!XL+yW z$FwXp@}rl=7jP=&QB9PxW@8_ru!8mL@)NUeY5Pn~KG0%FU*A}K_wtqGGDr2=2KVV! zkz<Q&Nm+Mj-?z%PWiC=F%7N}hd)rRgW*UrATMjx_>Q|KR!+<dA#L^J&dVTY{&)ti- zu+7cZGWfPMJVnr!wfuEn)vx)%PfDp{;d*W5j=ohw4vbd02Hv8vv#~v@@I+p3ONex( zRCx@}-XoF>q5U%-@Kh!y8J`y#*bTuAEfsXHDWifTs(sc?8Khg4_T>0J2=pe#h}FOU z&v{HTI?`aL9J>PtrC^2BC6MdyN-`$#_Gk&DWJ+rFQZH|zZIRPk755(|`q*VKNLr=V zZ<XtV>e7$ymW46*+3n8{#^P6q53>Nc)1&RDlqVo<@+c@feDUM$+@ROBf=Qyuu+4T} z%c{uLa#r7MX?R!rMz0_It=RM8!%bxEvBe#cr#MKjqt}&`erxvf(5RJ;ASK=Qkoq^@ zj{^4TxD~y#D6M)60+Zm1joAi_x2q$}^VcTSXI<bQgE1L}_jPANr(pTAG8S-t5g>$( z4VS$w8^D%Ja+ZV^&Ddt4g7qT3K{5Fo&<YWfV`$I{eU^;19ta-0CV}*z?r;;sC&~*E zNi*;8Mds#~x7qt!tu49Sq*mDkVy%qKO+#lWgsq$;;pCadJ$=+i@MhaiiD2{6KB2dD zS<ZqFd^gXVKFH;qp<YEBMH^-vV*504!}>DFE+W!`-j`F<osE$CkUiln4SqE<n1Yk` zBCs})C^~<j1wQf*I45$)EjP&6*+^DH*m`EO(`(V#Io0k=>!Vj}0)kpZ5o6yGh{;>t za|_@<Uv1Ddas8`MJgR4Ku02MaETxn9+nu^ryMz{0e0aDO!rWk~`)l;EY99;Ta&ta4 z6+X7Ky<^<uY<m^j58g3PHv+m6N&@M(4*3#EW~B`w1v8Fx3uh}SA)Wd6O*z2@U>}6( zG*>9%T7pbaKUkQ;9MvEEvDbVZ6`_CRCm)ZBeg?vh$4K9vORU_pfm@i;S=MkJGIF<; zxtfTG$mEcT{}P~<Dyj1){OUvd`SadFYfa`y@4f-0j4x$xQyWe7?`DCT{QJ{@QuRhv zq1kPF8qI>9J&md?ze(v^UM^1+bRTT(Zc44*@R2p0PV5u3E?<67n;GF8>h7(vO(?RJ z%@9vG=J`+pnHV*YQ|CFx<J~%@(Y)8!2ze6LA35{UOqEg^?jit2<c|dwkA*5fn>q*! z<zHnIFZN_du1{`%sq)?fsSfojvaDfn##poMO|t=Wl^<O#1LiZ;*GJ-@v2h#sF&z0K z25%T|;${Y@>N4net^}d*-x)_bl?LxWLCiUvUjezY>YTefz}z(W8dmaHN4&2eo;>_@ zPbN9Bd4N$mAC(!hGI@+<y79ft(m~Q!7071y-v1mHGQp5$(jp@5Ez<|yZ$;IKLDnN- zc6j<%V~Y*Tn9-$r$hxG(!f%4%(9Hg%SBEhI=p<cER&UY$!WZpXFReYXD5QRqyIb%3 zr3YT*vobc9`%x#vMP4-Rr8cAOkpiygXE)K-lE<I*f+i{#sKPHfG~-Dz5hTgHQXOS0 z*&vdTf?s}CwLF5wkSyc-=OnKyxRRL>ezmc#FfGug7nU8hwYrLE_8LkoNtKMR$O(J+ z$46TlUbnW4KWHO<@-o!sOfmAFoXn5_U!#~o45e+?r|_v@TosrUS*H*Tk90SppUIWL zzPIRq0x<%4I3>I9cb6AZz3w8QSFJ}itKw4uMETeNX#B;ZjITl;UA2+xzbNLJ@a0Ae z0spC9ph>Uz(#VjTZcofs1W%>j0^~(Q%vrxt&iqwhNOjUJ#6L_uO+h-8fNw|_O9o91 zMFfp30)^7~69qnPYR`QmB~ayx$aUVrqIH(CBE|P&%uuphKdE2sqXe84g%6#(h6=#% zq2`yvwY*$@+=9$VCHS4)Y=5p9v4t)*67<t*%>(JNH}Ph&`Wi)bO%O_OC%ERtk#<Ew zfX9f6jnfmu+NqCwDw{Oxu_?JbPyvm2!>zpY`U8l$`7y$E?)Und%r5KWc0tM7zs7e0 z_&_5=1H7vEXaCUoeQ(57Je)`}hfRN7)sBVYw&FCp?StAF<NYON^-o+fIgsKH1DYje zoBt(^*DEl>L(V8dRPN+XFW&;Wh3iJ5ydw(&fli}kmL@hkd#!qZ?!)kQzAY+rsY%Ls z-FQCl{hRG74~q@g3DV?F(fW|AiNXvu5*18!_;$np=Zk^f{QP_od1c*kzS5iBD-^Jd z4j(9Y1B8J;{1mKLZ+Gi3A)qGs#lS}Wuh8RyORMg)D_h$#g2he<V0?+e?7fthK8_bL zCvWGgU_UoH3m;zKsO|xO*l66|o=`spLXQeMzdpeP1`z5-cYMtd+|b1CUjJ0Ph2;_U z_ucqongLHfIzZ@;bV}2mya4!u`(fgx6mXTL8(rQuWGcH5mnT}YPphupaE`ECP0UNv z7k!g4vp!58X*O%XJT@94P~HtQ-=~Q_eO?hn9Bed-7%Ov(Mfnco>BhGD3@sL+@t!(- z`@c1ykUOhym?!8?>TCB0;=_yGvctCMM|MOwfKvP=hrO&4EA+TTMm}DNX4nj5c`iK@ zrp=kVB2i@+XP!kp<9B)8idwt(A;q=l?vflKnPFvWW@|hC^8#Ko!84*FGUuEHPVKw| z_62Sni4MxefGLcv56M^$P*|BL5>a1@V)T}J5uW&>Hgd3*5fzGWa*S3c$&~8VAy&G( z<9?1My@#Q4OD<xd<u-o|N+jahn3*@<41m0`eMy>vw9$`pU!+<(I5PCFTV0;0(fA~N z%4Q@Ii1cVX-(j=MN|z?HOR}S$6dSU)5{qy(Xp#>5(h@@00AWPjQp$ku<`*s(30SZ2 z!hdnCz`g-EzE|&Z7P2p8y2`?>(|GHS$_O!YU@(1<oQ6TJJ3|cVx5#80<DECoH9<Yp zoPudMMxvMY6+L`wC}9=2mk;_o?$eWjSn&d5nZ4)Qew=stC3w$qjw-NHMuh!*kyOve zcyd0vvpj0yXUcTVxtj>L$lSopj*ki5(J(Si244T?V*><K5dbu=(scum4}w9cjWk>C zX$5fPPnG@3yV78BG_t4&$|?g>)azwybZW>-HECw{O&KAK9o50iZKehaSt@<#r@)H= z{a3|n(wMf_U&5G;X+`1NPV$faqpmIn^<CVm!Mi9*)QMY=<ju%bgwB9S1NR7d;ezbl zUcsb;7H#h(1N|r4*&k7m&MB!oWzOfp$;GMJAtv@K3=JSoya8YHT^Gc76T5esGO1FL z9_f3M@iayDugPKw>H#`zY6CcU{!YLIq70-h!_Ua+@s|9cUxQ1Jo1R6d<j;lH1?!iT zh<(pSTeogH#oobsnxEZqz22&4QpUl?mgE?S%ubOCpBZ~6A=`Ar8WwJK;on5kAN$G& zc#oRGi{hr{(jI*VDSO9xy}6YA`m=`_T$rM4aO7Dw%M7`6JD$E&$#<?>UvqUf7Aj|H z8eAA$CE!dqepl8U+P(k$H@^1YyW#3`i;bueV->MFsfO*oesQ4x{qZpI>cg3vV9vay z!eIsJH!)WqW<N2BOs9@Mo*Hv=MQ9hjN%hq*pfi14Yn$iC@-IE@(&g*D(|yR-N|fuo z5qvhAULx38@cg(WI=JW1?h!7v8^zD+#}l_+02wI}IjE?SJ^12PJa3L%F~z{vmqT5y zL+&=?(b=Wqecjt@J0h#iAAz@+xT${`R|K|(&3u$lT_oEWCKdrRKauX<<QL%$_PK}t z;iFMqg|AYiz;;S(1;I|es6!u2UJRTnszu+Tai@uQass$(4{bC*WCmUR6t|!oj}WTU zaCja0ZLe9?UxTLQ45gg5&JC*GtXoOvM~_}a1$PxbCUhSPso+Y3?c&pYo2yoy;7)FC zRdmkt<x`_f+-t_R7wws7zhy4KG9G#Fye)ggnpBqNEbu5$6BS&N!b;)8Of<04l*ZtQ z6`0E8s77c}HRQ2%Uy3xpB_uS|JI60T`zcC?uh$?CO?}2W^KpH``4H&#dP85szg$@& zpP_SI8#YHPdFEihyEQJ|=*uvJLG7H;9XJWj^!T`Cj*lAk-fqk&_QJb0_UuzUG^e_` zt?K2rb6aj>a7KnT4(S@1=CD6=Ba@?6#2_(t|6yjB1OMwzCCW2WF}Ezb*cGQl4U<{t z?Q|4qeRsJp8~`tElms7peybxuDf$t6eh@s7%Ls5cY7j}t{`xOgkDQvlmkxB~NeZ1a z_O$vUi;X_#!qx|oV`bG(h7yPJ&HcVZo(MhF7Ef@mRu5<wTF4EuWpCy2PHkQ0aMuVQ z3J#_#=On%Sw7n#>+$9k0C$IOSbwOepKueY06C9@TZav>1?5&|w%-9?6^PmRuW;nTs z;wLLdYj{$5J3*^u-H1Jdvxi{yTXfEyD8m$BgU-}<<(@&PN6hHh;3p}pXic@pq>oQe zpQk-j0gaqQ_!v(UCf`XAcD-t4^UG8Hyokm?VP|M03k`=8gO8}u+Fk`sI>OUfpGPK? zC9{I@_JeM`ESQ}VYn=V-w~1O`z;m9T%_YxCu_34eHPRASrV`?a94ayC4ssC+geUYj z!S)WnM>;gZ)q`c$A*(#DQ<%lR#=O1gVCT{XoM$-?3Ecf{<xW|{dUp_>1{s~+DZbEj zU9~p1BQ|a>(QS3iUQvr?to~=_E~yo;dBr}*Oak{3YcYc*p<xDOH$CZ;Zl(V0o&UEK zGiTTWZFwAOaIjWm*mnZ1k?J|gQWC6QheP2NVaBX}qZN1=G8q1I>eB7)YAOG918HA+ z&g$!&Cz4w8O-lLn4Rd4Zs+r6P`XUmf$bwt0#%pwST!Pk3(ISORt7f()CsfYaw+Wsf zNORiGSv@vDaR^<AtDbuncy{Chj`S>=&Q*>BOLI>t7Uds}AKrNbi4wu>&c$2zn^NVl zA#mlWt&_99{YxX=JRR9&pK^`jJGITc+uJ~ANIyPAAm0|YnKxe|W~X+4p(3tUB%h}f zyBA~r#>-Kx*%hHIc8Kj~V_^svap1KaR#a=@qV374cUkYGg8VYFU-t2+n{RMsPhP3^ zTkZe6{k7g=yDH>xi&S;?cE&YR4dI&g)xOS+iRm<;2l)nTXhg^cG3L!bapV(ray3l( z6<J|FeSj^Jh4L{hb8%$PJePdPJ_jm%=#+R%TEN$m>*>#VqW|Q+Xt^7lX_j8MApY<( zM^0y;2GCXOa*aDaD1H3!5=Vx8(GCIcQINWOQ8*|>qddURi&XxhPxAdvIfk_!jb0i! z+9s`F_M>&V(g-3Nys_S;MWhbA988Hlo!{ykI&FVmPoD!fw;wp9YdbEmqumGty4xmP z;H}c=g)7OhgI=4PxU^|OweIE93BPvd6U`e!NJaq0CB8Ukb_WFH0nMRwY|^Dyqs1ZF zSugu!4N>#RYo_t`A<Z}iccCAWxt&0VaXwhUW+%l6Vl*8Yah%s`>}CZnUqlB-&puv~ zO`j@a{_6<qYB=7q{N>R=$xso5HV>?CRY@w2)Yoi$TvDQJ)!I?fo1rB@H<)i*%|D%v z5t?h$IbKz|$>(#UlPjSdfk2RbnyEfSX-Igy9KZp1T6n8QvcTT%%+#gY-epTL^n02l zIKL#h%p~KZn1zYU{g4UK=A4**?fU&Wg1NITYgcppBDhEh8lY{R0yNUHh=V>%sb&pD zF-tp2XZW9b+RX7m!c>(^Fp2ASe)CpkQX0bhecSlV%H!#qtAc{};RS{ldgwe?Jh^~= ziPFJd5fFif=!wMXy(c^6Rg$H>wVOX_5aj6$xZ0P}w;MjS4!*j;K23RF>+-|KmXR^S zT}s$UaN3!bYmS*|HC$>I=P!5Grsf1j*umB>dR?uU&Z`8YC;ET%=_Wiet)0%}-F@F) zZnjhIbn}auyqDYF_?%i3rC_Y^{D!iA4N#K-^a%NyBgCd`DS?@cW9T=o&st8b_hgFp zU86A7kcgvyyS*wK*T6dWX0Fpky{?oLG&eFeIT0L*vX^qj$y0q3;e=&=>dxCU10DGS zk3vY});E<OzGxVj$*=7UP(xh4mHOsOSn(a&y435Yo1AwWywTzFPd*D_B9f#=*E^|e zn=7(vZ!v<`>??P7vC-kBYEjK(^W(mf2;iN!49@B3$-x-XC98h2gnJ?O4eRfMEH6A7 zQ&r^A<DFFUf%7r`cC!^j`=TXwirve>RPLTm`enbKeqcMVaVrl^IT)i$qbVsB6t2`L zi_9)drZR3Ir!@?{mJ%)Sn^x$x-6!a^C>84vjSMhL3%$nBO+D)o?PPwzWhc1kE_I|B zGhj8bja%!?o=sap6=VzJLv`rOqpnUL?Zk>JBU=l<o&TDD;)=+=REgrnhZDzv4&DRt z{Yh*y8j}qa(fcwROYe0%7wC2uqC>5`xr&_lKo@>Q+QzL~^XuvNypOo;|9JI|y^qdo zIE9iU_>W%NWWfuLOxfN(=NH7**tn$+A1Xr{=i;A@6M!5Z61RO{y@O-%<-ozXOscP` zqaJ`$m}eJeq~z9nT9($c7)ciwC!$uFC(cY-ii+KSyb46pp}aj0R>5l$iGiuDomCv5 zZ%h4QI-}hBP+oNsOHJ!i$$e*u)QhDf@xeb#e6wUfez_-NkFkzH^V~t`RcF1v#ihGz zJBiNA_B>L#)qVcR?EtI!nfoV0M8e(}%_|r`2!+3n_=GfIFS={!SdDvqnf@Huql_l$ zAN^Cw@Ks*9c6eeutw$z2RNd*Z2;S}m9L`kUyRKW35U>no^oACVAe|-$)c-|@>l_Vy z{bxVdqJ0mY=$QU;y7OKZX09&pI##N~4%T-u{EkDU1JAC0#+wwI&j#AD+ZV_3KfJvz zk>#3s<*^gA|HzRZJ<+J@&8rwLV)+fYQaSU0`13S&^hGq;pGKREDk%Pp_<}_?f79<_ zvL;!d2JwSJ4*D+BGXp|*JZ=XYM^Y7>e0mQrHl0RQZp3&In)}RfvQ0|l3k~$jQfmu3 zxi&h2(zI7@=sQ$c6gxO#4L(75#zRv|1Esu7*S!oLd<5gUuFv`%K!I4<>dRfS#=YbS z>WL=%B5#3e9t2f1=QI&nh8|KZf2NWaiQ_?xDuRzSRKg7V^`zc8q<U6qn~WJI<$`ck z`B1$I4IVb$=pY^kLfTQR+ubY&i1S=s_xbsOH0+dw=;+=Q3H}}y+CFk5nlbHsKW(R5 z5FJs^k-y>0=!*7DmCV7Y(TAexoX+8EpBQm$s(uFs^OM7}!L3s#Hv1BI`Wk6EOF|V> z)J+AtbNx;R(0+(9&>>5UU{WTk=_RaG*j^fc)g*0rdrH>A?ZcVN;Yf1R5zcCX9h%f? zcB)u@U*7;G_&#>I?sn-_E%y3_C*G2!KuK(V@ygVjrBo^Pvg$sKqFfzKRfnVg<W50v zt<EqK#th7z3IV@J1ue|S-BA~_{*1xlv&U<qLrYU^9Bw?~gIituj<_f7qtC~A@WNv6 zPZBrZx^;p3cM6~^tRe}DR;4ukRP|hM%LX$_mF-^bp8O=sSrA$Islk_;FihJp>r3@9 zP8zs#js^ORv}JQ(ufFT$Xr%<%z}tdowFQk@;s3TCZRfd_SbI_};##@mj{l5|$g{13 z{kk;QGH_YI2PaAN_nstuy|-$avl5wvU?8xX68CjkD?nlMVE3AMbIA7-e>pxbS2zkf zB{(x9;i)fw+Opv`oEx?^v4f5mVS6|z%9kdW8&nHhIA3mD#k=3|_+zZ3)5cn+BPDEp z@m=I?UYM=-5RY|}YmNU*1#v*7e{9_to;d&R2yr$D@vyRqqcb0=S05LyHe&Yb{<^$l zNeJ905+;;Tte2Q7gh`1A(3cbZ7W7(9)rr^SVB_;A$6?}g?&!dsRH>)TC!s4atT=@k z<u6M9vbxjm5NCT8tb+_f)_`l8=Z~OV7ssE^4BWRH%D#e*))$co?ip!mwcbPgPE+kD z8pO(B0@s$L#B`;^)TP>V5_`;GzUrH^X+nGVMhae@Bi%tDvwLL2Lseusv&);qDWB$F zf6>s|H7Afek0hxbY$txz;W0@3m-Af%hElK&k53f<qn2J~@7GtLPg!1@-ojfB{dg%} zFj^byJH`X_=3)YQWb}RFNVb}4p)ZW)gofG2v?-2)Z_3h7@Wi7@c%$i_O1a=YWsib) z2{m@A5fW*w%SweGs=s{rBBw?r!Odb;il1|Q^sx^TFcOUr&fy3c5n37<z|2rWi4{^W zWYD$q^*hjImZ@{Y29<~Kcc0k*D*NDbD+}k;p>?_;oNbyhpFND(=03X|VP9-{%~9F? zLHZ_w)3P%`D(bP9;W<)cpqY6nVcP|!5usGxv=rSK+<GSu6X1J(a|`&!wI=r;KYb2I z_S~8Wdt8-YqGsFAjhW?5<)~2S19NCe*mLrNX#9MHCu3e)MpbE7C{i>6#{gJ0$Rpo^ z2F{4i909=8{evX_mntIG-F8z#y~Qd*0tiBsLXh)h5{$VALahpgAaSfl$W-fcG)Wim zN28go{ObUFM|BnI)ghUf@pERt%_BR+Xk@;j`Hu@;>BsMI%yckhq#)|<ZMD%0T%m-j zGBl<0nz(dn1a8XWch73`&0^M87&_fkZqg5^TMDRGvhq)Tfv*l@PE94U0aTdPoszPw zw#^?yGw(a>*@q;{@86|Njh;~JKk!fPbd|U3lTL<_O=C&5gl0wDZd${j?AV%FL(ZBR zwE44ug*{>bO}=Mg?gkXRODv-*!aj9a-B-^$nIqZ8!Jqndp2HyDSG)Sl)p+KhP~&3% zF*J(vuAA!no3>x<CaaWa=LvS{F0u|V!(g`%pyfhWCl>{46N~X>(*$iEdM!>Ys%dty zL4QpM=fdnM5hMVZ3fqG#DWvVLBogG8>+*7f!ymS5pI~Zd<|w4X>(=Fm{UW>O<^gUa zkU$FxeI^Dl4Wgt-tSl)~%nj2tJQmDdbrylCmo+6d2R=z>)Dnz_+;236ZY<dTp#$Wk zEx|!yEn-PID+neTp*+U<Ot$Cn@1xuLe)&A-F+z>B{;lqLKH2h<usp29ZXRdf#T)l{ zQp;zAo^>iUzk7-H$fBl3Q_tJi;@;Yr>mw=8Y?{=hOHh@9rS~B{Rmbu78n;_YwnD^s zOgeY-QI`-5?B5OJ4Q30rnj$XyyYqZB6I8;?X%saSuz5kkkhp%w&&MsfP6RkFagX5h zPcUQ7x;g?|sQUu#!LLben;m;`-n)O%=dH5-u-an>oUo@6g|N$63%y|3+5()t-B6Q| zjm3^R%>t<i3MnlQw@lw$=PI;q4tVIDa|;~lK1bIZOP^F$g|1UI41og6<N33>T>PW9 z7jT{{iFyMH%32d|ZbIK_%SXF_#~j1&G6REA*HsVC$&R7Oj#)>)x}Gi&abV>*vqm*J z-!W&pk4L!aU)?X0`sCzjDeQFNLBI>#Lg4+GekJNd7Qg|SwjIaidpF_(ogXXGY!k(H zg^V8W6f0L9M@ZLNLhK*tX373#^a8N1Hx(o;je>+m;u+70L|(4iqMTA+Q`IIXLwYt! zfGHQTE{|e>HcYEzv0}aI>oz+WqGuEz_=_r!k;gX=tGP4Q!u+$I>}a4mgbqIls#}jc zAFlhSCH<2^NBq(T>j!#vtY3>bUj9;>9l05m5I7`k#iP$b$4qRFP3j43GfFf*+PqKv zjmTgsNo!!3|0X6v8&uc@iFFSvzmKoKw~!@OyzVB|)$-7|YD&lTqypu}&L0?F2O4RC zN`Umo<$#mUG&IdrLrFzLW$;~x^ndWG7ON#`1yoFSihJYi*idzT<(&w8_1c<w)5eiJ zt+3Wbp`tPdE?K0&Np52ySe#yh@gqyhIKVa=wom@%PN}!Y;NGg_2e%uRgGR#K?vnVe zC);7|I<Gl9>+>+<o&Jj`i1IS@)WSAzr(Lz=MWemCjtD=!5}u_mL=XebZrL6$+w_4= ztVO<4APK1Doh&Qe^xb~3&g5ULvgm0!w{;ooy2(F8KIfcM*c)!>O<FdPPLg{qqtpIJ zr`qpz>V=pq&!)9Y`2@Ri*Ikvyx5r-3s|ZXUWCg(L9$lUu9DI+@Q{0|}RbfxoTTt?0 zo{p&PHo4gl<4*GM*^xelrGivtePpfcB62i;2}w=sHzy6b7uF)YCS<DJFsKO-(0&Fm zeRbht!7E!1@U|($90b|b5U;hsiXul-Yt{T`SaLPhZr=hgsAJmlQ(MRQ|MI<EDnF)2 z2Nl-fL`-}Bq9HbuGc<#PX8|;FfWn&Iv6`tM2|b{P{0W<7H$ebpD;t|YJ^NrnMtxS* z5KzsD9h`X@QN4C>{2ybK^8sw=8t|bTAUIXNy}Fjxcj-SF&-<+SFF29^`T|3LfAqh; zI6_1aPkbT*b*qa6aruuZryR4;G$R#Qz5@cac}A9nvGHNgOTNAgIQ82<*P%4>T6rKm z=g54KG@X9^fKL6X{tf){^!wAwOCQ*O-f$O8aY4t*Mf7DVpKxjXkx9hFvax-5t*H|9 zh*hDSuW-mDTww`~aPN+DF6F&^X#T<IXIpgsA-3<PQ#Zz{obtT6Kj7!$72$pI?Ygi3 zQG8mFF7NtE7Q@EI_tT=QAhsIfVblQsXaRv7{%|(0+^<?}<xiqddglx-a%g?Jp{lu$ z0X>soHbCptdBw(-Y#Mdh@wH0?VLdqlYq_#&$fln>N=?SAR&qzsm|gbb>%1l|*W+CP z`iv@9VPwQU^o*5{udgWx1VZ6()4z+^*oH^y{*&}ySl9+MKG~kK1Axiv?=t=$Bpvwg z6){u;jc~r_9@*hyzT;nngM)$!bmK1||C8h7q9g$bCHRthGZc%~*Vn6k#>&(_0AZCc z7MQ4~^05HJ^gv4R)!=_gHSym&^#5)0|No4V<p0je|ByuIzhz?c0FBO1azsSLprtuD z#VKmDJ!pTk6WE$tM8<N@vzGNWj7TIBF}OU^kqMwpPeCS-xHM_Zo#vd{#}n@fi9dIN zHneNRNQqNQ$B;t5T2j+?*2XEd?3+IK>>9(K04Xy)CKbfSRsi@Xx|2sh|EZdjck43u zH$@<0IUM+9!8tRf2R{Sn&l`<#=4)3UV7v7A1OE;BA>aAiSH&CTeHCoQ2bxSTRv!VJ z`L`%QVz|r;&hJ!67L-X4++Yp^XW!ihv{PT7Zk|LQw3Rf$2XPTC;C)p(0f+~$t^l{V zysdaO*y0IfT_qAwpsZXy{2TwY-su;xFL@9m>?A${3@mLv2~WY(4}hR?r_c@9sc*f6 zhISepT%jp_pv4kV8NXx42k-@lMGhWmrN?i?38ajtxd+b5SMyHG>)>8FfrD$0yA}V( zwFw2lEPuT%@oee80#y*v;_r2NsYXEG9r*dmK2-U)r2GFT=)zkbc}$o|8J*d2iCC); zAMALOF_c=`FZNWV>%D0FV9U|%p^qNfsj1DYnZO_^RuU=Ip&K*9fIJCUdJAV~=fh9C z(50o)&Hts`x~XL>%B;=Q<bt|Eu`4V57B)6Ehd-RW@aP+PA<a<=2(q8Lo1pYT1W0v| zou$CfCZ^-#<CA{VXY|GKp>vlE2>$cZ#PyM#f1je3x44`$WqO_g3o>5gV}r*<h8}(@ z;`kr>t#fWI5e9^iNk>4xWz;~SHf+hx93*@m9bj5NNX!|JW~qi2#$FeWCgbU&3%5^y zwc>Sd)b+Un7>Ol09s#dA^3|%+qd&Xx{D{tnx2YhH{l##vl331YG5OH-Oma>6=U(_> zz|vsM?VOA{?x)vKNTeu^jFjz#BZ{7szeTKgKDhDIq!vPWx8AT3%?=2;Vh>KSSy~Uv zcM^i<eW#8PWi`kKhdpm4s#6~w20(_%aQXEkUmpzs)7LRscYM$uqn-xPZC|~~09>RO z*bEmx^wsJX4}9!R<YnBG`9k)ekE<mPvw5oHgW2RpOxu8`3jT-m??3DmSpWZ?{C(if zU;nLyM%v)9l!;+wjDoa-9Q>!xWJAa)wK6E+h99O5T+ZZ@JwS?tZm|3X4Pwj*Pb^io zNkyj{n>LSUY>#nr!GlfhE+-!Y7}>7?Km7D9uRD|k<tmI3w9C2mcq$W%#aeiHc-Ubd z-L-4l&#c~eTq<y{kyjPc<pYhDIog~*7@Phrzy&mqOyg)XL94!@3&KIRYA|vIAmM6c z*7wiMB$KW#_GMX8>Z%sy+1UIaoda&V@!i+<XVfYw^i7?E6$fj5qtt-F9A%mGx=&ma zFS1s2Ogp|k5I3L4*>4Vb%<uu3$@6LBb?iTS{iHP$Nf{LlNWVNL(CmU1q=zm7=rf?n zvh+XgDBE^ruf<wXz%Nry0B$OFpZ&+ct9<TKrS;umQ?IV~x<~qZvnD@kIE8c@5_`Bx z&3Q}fT!vi}^?ect5(0}S<0v9giLSoYn@Z;M!DRr<{P_1V<Gwe+`LYdbb7(oN6vR?q ziw{R_3f_AJ9~jN!o^OxPw!Q!%G>JIUa($=}SErc^B4Rd;Bx=K8dJWf^HRtLdb;QDy zBj0>pd>L~HZ2kH>UbvR0^NwsLj+VZ<?Y+Sb{o=}BVnfJaiZz^H<-(LNFi!D01~ESX zEc5NaNnf~Q%|`xoh!kcEFS*5Np6#%&ls(auoyfB!jsW_OeEh%yd}WUW=KT1&tbNGV zhsnD-N)f~ub-VLAytmwQeO`RiHhq2HB%SXH8L^=8;br+iIX1#M=R4g7PTa65654kO zJaGMEreFD@!vuyH6W42!xYbj^&2Fp@JZ)*um+{bP?81cFxS+R2>(+v+stwN-0*;n_ zyPtOwF{Zxr(cGE@51)K;O7dQ2&dd&z%O)$oUd)!A?Wy))I$(-&{I~qjCvWse&{1BW zXF_LVN?P#C+Mq+B%25bk-Pe~!nzUoxm|y;>mT|Zba_Yp&cN~|+iE(+ks@C+hVRxo? zG%uNvf5?nFS3Y_ID}jPrNShaKhbbrYHM4(zMgnrt9WT%~vTv)2+(kNQe)t$w+qxF8 z%>0AM{XtWg9@pa2!RM)`bV=A-SR%fevCA^Zq5mKeeViRFt-S<<S62F`6xPCc>SUZ@ zel#kEsZYb(j26UH$$LL1ag(H*s3WqAJvzEUw$&AUh80W(93BdbUm^AN9RU}(R(5Xw zvsA1W{wOm|Nxww>i6f0%<ky{8H4b|*s(o~4GwQ_xPa&L!$QWFN?-BNV+g~mLP}gPc zz(3Cwmz1ddNibw|JTyhUSCW9qviyTf;==?OZ#Y|DmI^<cv(CA%EceZ&db8L!kq4YS ze@|*igE$xM`xu21EIU7+GS<HXwr3HXgDwwG*FCX>@ylXnU>+k>pMFb$rEuiD0IHkL zl$~}QX{Gs_d<Tj|%LB;#DZ1mVr(OmkVD)uw#2x35iQ~uC_4yb-ZAOc7hkd{mp9(j~ zwjO7)P5dV+la8%JK*Dd!6p0U%7Lt?Df{^`hTti{u<N^4lI*s9Q2_brF`blafM>X%M zC#L{^9sdBRT0ccUeAx2*w6f16Oe(xLHaN|GE4}^Z>S-aqF}p7=PTnu@pJ*;Sfmh|t z!VkAk`q5W|>}jjFdr~ADFk>cl_wSV*1Cnw_4DhZlheuvD-aYN0T2C<WJ~Ek${=C!& zvR8epmr;zK$6vUzaN;tMm%9#N!3i-l-;sb91==<5`MRxq_~o6>57b;3^->;G)-otR z2^7|sjDgGKqVv3!wfghyc5-7zi%3G#PwtHT$hvt>*l+*aB3794D!uEFxX)Wh`!0v_ z9LoKmkb5?WoR3EbkMlDDvpUyP0F-&>q{KQU+a5h1iz3q=q%}uUaC-qS0hZl^)wE^@ zcxB7TM1)(9w-{HZ)Ip@Agmw9Jh`*I-VdWlJrT`2;El*Eeu*NQYA|J$|G}v<b?UWYz z+1b;IGw-MU`kZBTenyPi7M)RZiX>ofGu|osO=W60->+T!-RdL^>a>ka;5lpl8<yJ2 zQqspvhuBOn)SBFx1k}VXVRj*>T&bUJiFG!z1`ycL>QF_eU=~8v?{@mlz<cRHhm1+< z%W4=hhtnS7>wQA;Up8YVG}z%EZap5)1f<BNSTDB-0$oFm`-R?pehdd%#2>MQc|4H{ zq=T(75tBE+YCV;o?8V5MC4y<cztZIIHP-si?yG%ws-k|2_U#2CbcuS!b&wy=dsYV| zTk%T{e`V)DWx<-uPE{?`kbM`35*pmQowZlUGKcB*kY>pS)}xsw*V&fv@P+Wqo$<ug zGTasWoo4`mxq;BcB9O+s3&LctA{l;;oYmMd_kO4G=04_343C8dleAHIoVEFmOdhg7 z%cl8S;#WVEN=4sM8NR2&E7SdKFn~3GpM@}=wz9T$EW07|`vDo9l7xl_;TWgV978s5 zZ!8Up#&1(ab^V2A13XMi5coeDeNS-P0uO2c(*w+<2l1XGPa|a7jp!;VYwXcHpYWh{ zIk!!vD5q9xI%wW{?|!FXq{(464xT0$sl;cn+o#-wGlt_~{!3l<G0Ae=!=FC#d(Msb z{EY85tTUXks#$x~p}|M~;TM$jmmPd&nf~hNrp`HRN}3Ir`K?3f9vHE`k8F0%f;7y8 zBYraT&My_wtK5JdK!-Q#(V&tYxuiK8abqCebuWCwVCj)?P$@5r_Ap|9a|2nFGn0Pn z@$LgWfWBt(^5M9TL^s=CQENSjzLvFfLm7nqSNpJ1V7_Ln-Fu%OSv_PqbP$qG{BqGT zjxU}gt>E8b@?L)ppS>#iD&MrTL+G)YJ;twR?9GW&OMN7h82jqws}f1unkt-GJ@iZG z_+)GYmXrAWdTV*4HRq<@+;F$j*DpaloH)w4Xc`xIRABF*%wT7;oR;rQfPC{E0>t&- z1kMi(h+x;=Q4$L&t3o}j+(0?NG#{wYK}Ac}gKn~of2zov{C+(!8rOvnXdp&Xj(9yq z8C^cDslW%&R8jUJI$`^<{AK8q`65r7_Pbfh%sE*W^%GEOK=WOv`9!oxT8~AZZH$W@ zplwYBZ@i|;vssSU@#+HuX7dU;jv4AOhHv=I?c6x|D<2T@u6eC*V+O}g94MH)DP(d1 z0F054sCj;`$y(nNY#-9oufIjy0<m`VWiE2$JMR+v0YEj{IJO-YFRy_Oux@(84J|Sa z0RrnC5}m+;2^#40Cje?OHSqQi1>pXObeXq^6Q_E>`SEfpV$uR{iv83XWVb!`SNWwu zeRO?~$>Nj9e3i)2AM5~Xk-D#>Aw30OUAyt*vRni}NwH3B2FffB<RO=n-dMxa1Qv6F zm>HV+j*xf@B!mkYc`dDSr480OQB+g~rzb?CSF}FUzmL(yH3hs{TJX1o?*h^f@Ke;c zgOKIQn=&v^mOsi5sK>}?8=Qoxp4d(HPZtw*3SN;=u{s$ER}#+njH)R;<O2x@RPS6h zkGc<D93W2j^qUpsV5Ok@uO?reM!L4N-t-YN;dRdqQ&OEw>l9+md=N8#6kuED0<)`_ zdv@ahjxPX#HKua%QT3y)yr8?aIWKE=N{tiMX*wg<^}8zofL7C{5h4<W=w7az+kA8y zc`?f)2dWMQh>SxAl}49%vwBu!-z1>d3*WZfgIzFFC6yA%YxhD51UM-v<<7X(p5I&l zpPHyhb8wo9sz;Hoc=p?}#;y0&N|WZV#;l^r@O`cgP#x<#O<#ZQBIr;^eV($8HKpS; z<IvWVa;Fan=)?d$4Xgs4>nIu2FK7Jzk{k4RyB7k`0@IH^pKaWpd3Vb}_O1FLbiE2w zbkz1O|3R@u?jtl@^_{A_W_l}dDP)mu6LwIY<=^ZX7VM$Vux822uIB#&P{>nvSt>a@ zqKvkD+H1S8Xp7GEnQoM>!CD~xPIAf-%P0e>ZZjl*<gnsyM=!h!kh0XZ8S9fGN6K)% z4LzgA`^}mCsp}JBOnL8-Htw+vAh0ddOdl@v&;O^UN`Dlt;0WzKZQb8azoE5a<z<-g z+3PT|I7mk87k5gx$hDXO9vxP@Gt>7VVBC`qq<QsJ-q~AI`+&&)d0^vaV~=vl{tq+l zAx!f<#&}WP%{(L8Q0P>$({}oirC*A$?M5KmdA9F`BOlK)Y~||RR;#=uuEms7f_w?@ zgtoPvZF2-8$z9k^p_H~7SE>7d;tVeuB<hhWg8D#{&Rf9BU(h0UAvGE!Vc*A=etgVY z?2imrVURKI4+d7Z*f7~wzDoag%}z!bSkB%r>u#y;&463&)YKJbO-L{HtB{5<XTnWm zDzB+(*ucph=p~u@U9L0O_l?Dx)BCMOTVKlWCGgG*t})L)AV+t;X@<>H>hm_|u+vdJ zS;5ZntF?egEB4(@)5}n*s+mxo(8z#D&5DvV!1cTZYNezjDz2C?+fLgo%?7g*<YyoF z`a7oXWc7{4%c<(L2Z<ae_Vp=;tG*hD^Tt27Wg7g`spxPy>jD{5ueADOXe$PQUN%@# zZ3FtkME1+kySE$Hl9P5c*1KDlg$hMRtW*FlJ;DSSzDP?No~`^|b^U!71+us!Tfd77 zH@GipXRihbmCmTCKa;d5YN^nv*)$ltd3g3fFZ28iOepM6$@31_ujujw6+k<<*VoL9 z!Y*aYo^A$4)&<eVExod6g57n4DwPq!pk-%b!kis|ie#I<IeBQOv|jRNk`i;PCZ6!A z&Tk=QFY;R1#r&Zg(c}yQ`BgL>n1y9$=2syD*6y#?)??$F_~)$o*~YY>*MB1GG*#OE zeXV;LG#g5;Lj20;A{`X^tk0*B^ENm8RJMoJ+7g?3>=S<(Z7HR?6=f!0kjaF%2?3>v zzDzFp9kH*0X)hQ8$hJkT`^y1vLEZQTS-d?3__z#LCBn**cwbLMi@-5ESEoP3uL!Zf zvsCXHb2eqHHC9dq9kY5g65tWzy@_G{x@Fh#O}K?s-UoB<{26n6t&cd6F(qq$$1=QO zRlnxA8bg2K>J2uQ@A|}PMi2*ekCZAp0Rek($Q%L+&$5V2gh1&HA~MY|Mm``2urf^i zS0c~VIqsfN1)}RTS9$R%1gz_2UPQ#+Rv?-tqkk1dQtgJOh*aYMbmt}U+SN8PhZ|^D zwVGcA*AL&*)=P0QSw25fzMx+2T(@K#u6e&q?ak!?v<Xu&1%dP@q#u#yi1VSfB@u** z+23$qM?chPprt;LM7q}JAq^XyA5P?5Qtl8yu{L@!`y-#^u%0`2A1eahD$ulD*1tsO zEf@OnZ$J`>*1q&F?LO}-nuG3Dkn>Z4^|*a^u7c3vDB(Qa&hhF(u`iFhVvp?Tqa9?D z#jgvPK6Nbngm@Z$qsbbd<~jNQ6n9=xO(o#BSI_|mB_I$`LLUX`s5Bv<bVVsj=uHu% z7`l`oWB}<+MTAix(xms^6@&~$Ae4mOLJu9103qR?%zxd-`*!cT=QV3(9dg1s-?#T~ z@5pg!!M3Ic2JV8Rs@%7f@Ioirx@&KV^0<6n_m;ek+B@X>T^^Yz#3ugDcDLpxI?Fn| z$5)a*qq5&?-%Id-9KL-wJg9o`cyTx8Z`qE@BpZLn5Q#X26#OlXCPeT%-db8fujo2& zmQAPU%|-J?OK|<-msx<?g>j*bk7>X?`kG&QHsI)a^>S$F!1g_3!H|f8d(_Phs{=o- zWj58!r2ERG9T79zkL|O1Q3$He)3!g%Z1;3w<$V3Ct~Hdv3iL=VerPrbe@x3*bNl(r zCGpSor-IPQkOme1ZGPrF5qti^jP8=B%+N_J@A9|iHM8Kq#KoiI+B%=R@vo@;Q(wp9 zF8s@}&%hux_=wS^#|^x^C9PAvUF9h1NI0w?2#~be#=dT3mNm2X4QX2IIAH3IdB}@) zyV_k#hQ2qo4eI)V2SyrLYa6hD7q=aWnOTBr2aQAdl69n!xCt1VrdYi51X$pcb#8z5 zQM~S;{MmEDk$ZI~JbNeL5EZf}VJ9%w#)*DP;kWdeYx13&eDlox_7k9)1&D(5$S`yi zxJ!k7pNu)%ZY1lkf6!%p=dv6-Wc1jc8!V$H<Z=9ge6@{m;4%{qx)RR|t6$CZSt+pO z9{tKG)6t(+;31j3z}H)OSniKJ>41_!7o3@@Wr&G42DC4vMtMti#_Zw8KcoEjekeGo zFDjU%Y^#4_SY8{mc8`*KI+fUfb!6|Bw)8!%p!frn{#RD)PwRH2$i-GLg*)`vyll=q z14!YG*dVEMe+-;es~57`<)HnIjcJ+WmOFE(0(;sKQ4v&Q5JR4SwOvLN7V3^8I~)h> z<A0aAy{@hcQ!ShNFS^%$nx{GY!V#&yE-R`cifVCL1~{DmvU_ABwKyMAyZ^q74F6H@ z54ps<rkiJDW^=N;4YP8e(Nb^!C$M+G#o_lkNgc@RwWxN||GxnbkCBW*E_4}}9vrsA zuQ^ZceOGK3^5~IEPy5%?ekQ*@7c*||it(B*kDp#~xZSk=oiH4j=Le2x8S83ZSv|iz zW6@w~^&E13zgD80$8ToM0rK1#&U39?AkEi9R}Bjc>p??H@9@`Nbo&VlO4ZS2#v+`^ z52>ia+FlAIS-(4g{RjAgcVHu-Y)f4RnZ_3cXs0uK!jN_`mTCe#U&&wKbV#?FJpSFc zf|X>Bbqd|%Lp9w(omNUFN|hlmtn8e&a=k%cUk_&6GG44j>;RkLmpCKD{#T&zDs=v} z)f>+`AlpYEfu*&gO<VX0I^#T2f4WNCa%c>oeIU;DhyPmW_?^AuZV}I&^{kS%D0lc4 zI_}Uof9h{~@m($D>LAjISgOh2hXUeoGeF);1*AIF9%8~`sKo}RaAtWNTkE8_x884> zE))56+lx;`QnCzh1XmGW<K>JuFtSPrZwyiK+wLdhi`?9KMjz!igkYX2S^o3*ozmIz z;8^09jisZo>~Y+P0RfaF)3UZ-H26DxVr-{)e04un*RhM6)hnZSVIz@>wv-^{zm+Bs z@%-!60kRy=x=F(4_&2IKQ3S&>OXW4{HHw|BXq$6E{@X3LuJPrb$1I(#sXUg?E(~Mx z8+S*Z-xQUc0=87(rhOPmBjM-xs`dl@zyaLU9L>k(O4p4eY|`X|eBo+0lMCXdnPQCD zoA;QgLP5$yo4p-4%i=0^!GX6F7Z?GYyeef;rF3>JbZJ#BDCi_(uGRTpdj;_nGpA4| zIFHTt4&YhT%bDs|<E@&`c%rT@Y%iw<+Z(+X$iCVL(cxZo55gse9`I96)(wMSjQc3^ zgQWu;LlPoY6ul&YIMI9))_IPJJpjbzs#tx*HFkwpfo_LfI5*V)ZaBF!k&lVCrHGsz z@5C@i(B2~DSB!)qEfOqEuA+vQi3}CaQ1+kB>|$I$v3k%CAeh|4IZyb-#ekj0Ra#T1 zHJpbJE$eaCD8(?`((Y52u%NM}>Gc3Cm^!$cta-t6xffH=E7j4HT;*8-<ZRUm7l#dL zBS7p~_afotZZb+$agi8DU`yE%A)@(<F?G4o)M)Y9{~hAAOB@wPy08hqYz2}zQPkmA zcQ=B;geL~dqc00;b^b&;#21&f3C+}#$}SyC%7%OqelJZ9=ZC8#FbTX>%AiiYS|aGE zBmmC_ixU}Sh#4a5d&j+wwSZFRMs^OaL_hBio4nv_@VWJ#za||xvk}CMyxlt*znF_t zC2c;rw^9MjOE_IJ>b=}G`<h@eY{Iz%AJnmlZA5xfbxV71Q>J$VFJAR<=tMG&@9dlc zsquA9sa+&T)-k-EO1vMyFXYoGrKj2^AJhsFLx)(mwcF@KzX!<A8fg?6Sp6<tJELl^ zf@((7n7tAf!ML+8o4k1DWR+?(cPX%MA#sh2eO<(|E4lb19f}%{^r*YjS49u74&%8^ zZo5&ITo7u%d@%}<@{vixL)WWE;IfH|&Kh@0Mg$^RR7cE=^~Yi@OtBQg{4;h}L8Fbi zyv=h5X((__=z<DoWkN$22@p`~81|IHetO5ns*06g%FS7<hUn<ZzU^U}wR$6|Ex+lR zMaqh`m(idI^H6&XH<)shJ|VPFCQUaz?j7G3gqlnxdf-elrSI*{OW;H`yr_#*2&_dE zb=Y*er@n++p@l9-{y7PP!D0K^U7D!WMPOG>D(Fq>-A;p49(jw+iu_L^mi?SAOS-wJ zt&CY%YWrb~DMZ^Xc}>K&g*!V!$K)^WUR*75v4;`Wr4)tXZa!-14$4wBZ5{N1R+eK1 zZm8*dsgv$>9K1+xP~+IMXxQJ^-;X^$@+p`GwNCGAHz#7;Vjo^X3wJC6>melI?X(xm z2-?*4GYyYC>hwKX@7te>*s;rWZ#*`v!htVK)&m*@j00lexa`LPf*ul!=I@5}09s4H zx_mgc_)n7^t5o-fy_KRjIVn9GgAnF#zz{Ne_hZ3diCzG+OH@qU*AK-+)4C<p5!z!u z{z=*B1JCSo@0u(zGjDWSZEMb8@ud#8_6rRsIXKY09rnxIrq)UgrRab-A3i&sUzNYq zy6_GBQ~D`;`fihf@0d$jasmY;%SW&|6PEy|k9BQrO%KA)E~zauRi#<#89aEg{VNl6 z*;AaMee%+#t7stQ;ORX0yt#6`bY`6Cg$O480I?@E1p~DHnfjGd8R3dpv=?NVI%zV( zD*TC&pac@=(P)@-xJ5Q9!x;dW$}PCRZpOo{eN(2qEmY>kxCkFtOOH=Mhph4|r)I1) z4UsT3a51;w!b#9F91#5#1_i_s%z~sgY}z%sP{sS)vJ22v5FbL9hi@k)pqz{20T}*F z0|<M$5}04(eM}*`m^&?yA{{;Jx1nqfW075z?t5>idz0-$vN*@1l%kf26jAq*0n6Cg z8O>XXGG*G8b5z`FcM)MP;+~W{E6jw7R`VYybYHMk3(+aRC7uiN3+w0zKRfblI}KbV z*5jka*G7_TOo>cx7Vj`jckt`xH#O2r26{E{%+GZ11llyVb{cV&yH%~m#T-NuUZp2} zi1;V!I$P9@a*K2-y7vuZQjWU!;fu1Z14v4X6{r&B^;p$>aFW8`FM3Zh{Uf|xsjdKF z`_z@_HN}=P$cyy+=HY9?jYe=xAde%TM)(PJtc@pmvwWzUBtJ+WK`h24E(WvVqFnV; zrGFRa`fp7FZ9`vXvbE!`a!4?g@0U~1_}}WMm~WpMjC+C#7W$2nC$8Ysgu|-6Wi5D5 z!23on$ha_O4Mq5gF<9XWG;lU6R<PB!>&PX)O^W3ig>i54zCy>pMZ3EnPsTFY`?*#f zavw5+y7U}ATixoP5U~A>OQ>V-s=f9^-m7&>d-iELw^#JmjMx6J3t<9{`yYWi$?4c! zTRnA4b|+j&S>|c}&K1{Iub*r?IaM^lCfaR7)xd93kF0YhW&AbL$qfq2pOrO<RDOdk zZq8-<7D<q|{!T)Gz!h>}yz34E|I3H{F(9fxK3x5xIl>uwST@7Nux`P`t`{X!7A#9W zS}$GiU`aJ2W=jLBEpm?=sj^|p7vYcub(*IcA`we4m@b?3y9jn^x@@^GJ`oU7@J*Rv zw(26wYThynlUZB$OMl1Ch$8%0w@}B_hayiArG!VbF;!VkS3A5Ny|!{gS-=AVz*LZV zkprR!awK5knr)^Owdfd5ylL^NnzL3zNN$YXL!w(DAj8Ed-*pw0zf2d_sQ$)`TK_)t zcu`$d5TCe#iMee}iJpA8y8rM@XMC<=t8w+FwkH?b#3bK+_B7`U?nG1;(gVbKIj-vY zC$Keq9`tFI5m{^$mW)5=mplJ9JB+wzVs%T!H0Rh_n+N)3aLkC~LHDsF@~7PN%S_Cl zIS@S$eodnP<2rNs6&DXa?}?FWT<^l05|z(`4#!YGG5iqE6Lw(lMO*#Rv9#uL+VQ|d zE(+aFojC<B%zw-iWsT>8L-~=XK}>vF+`Z??0~xxu5e|oxj49>(qHIO+7kDsS8T<05 zFPzIm(ynfU1D7ZA0N%bD!#Rc{-n>*2TsD2Or1vaHQn!}^$he)lJkZ85Qjq&6Z%5a6 zb&i^_5o}SZR>oyla^b9&Z$eEer_(^;rC7J(=Q$o;TeNPUhz2KP3u{(G3I!s-lVe-K z535+dbn>D(Cu4U<Pu><)%qPE-zaU<^>dV^J##HQ=foLz39c_zu3?g<v_XcL=lZdCp zz2YRR_t^Z(nEAj?CDP4APs)$+D7A;Pdae?tDk>^WwU#^vt-YgZMlu6Ej;S{3X60aE z&-fpAZz{yw_H;F6$ZyH$5K{9(esSDDfK#oIzK+QA-60+mRa@T%Mzmu7TyVlglK~Ai zIY}1b9WA&oV#Z%E9RJ?szx=mFnv(VW-HA(8k=*B>D+`vFmPyqUn2n89p2FZYp<+zK zwL;D{sY>-uVGZVOEe4OH$v-%mb92VD8gFzj(5&T~ZU+x2!FP-%zYhff*9u7i#BQHi zFP)IH-yR6flm1t@@8Rqm*V~SNmmBJ*rl*lVsF570r3%@K#F4YQ8=z!dIa0E9w<C_} z&TE8zolHam?@hJ$+mp6@ML=0sdgsN`Wib0x)Fp-kFyDlfcur1)R^yiTg%QP>n%kf} zP592TKW$b!oK(EsrYOiZn+X?5#CuAZfzGOY-Cf(+qJ8dtvBf4=c4=?PqaM@8_?z6% z50}^%#*L)TUw1(#GM=gNNk68IzcKrsY>2SA-jlAJTz-_V5PjOeVWq>xIZ%Gd3#|#C zZ*lf|BMALcCsyk^<@d|lhWzPfcJY;x{3<1;1U2kJh=l#f_YE12?yEewvOCixSBY%& zbt8X1EiR=fJD-}(07T!9Uy<x$x`YIb$vs`R;04MYYz$p5*&T;lByBIhTD?NTl~B>f z7f0m$Hua-3v3QF>=%*{{1+(1%HKMTj@^5`xpZSJ9j^!&iCM|wRSq8oZL{5;&V_hSg zLt5z--JA$?cY;XLjO2@|_9|}}1>{<^hGz-asbTwvxpqgJ{-<xS>pYX_sPf(&o!iK= zgIk3&H?5%V(9Xj}Mk8G?&rUoq)OfM(v=9zpgBd}jsbX81dO~!_4^1LV;&SY6QypB_ z<PtZ>la)Rvp@8Y@GYn2evE*l4$9%m&XAa|HS6tM#BJ`8Cs43Z<zwAZZDQ$gjcKk#Q z$N%hjFw)Yds1VeQ9lel^I}ou`A5uK288Nu#l~FzCrPH+DT^xBihY;g&=-zs4I2^c| z0bQ&D44uPJN|HKlW|SEOHJK?{0t}Z`xXMPObJs;!N)vrBxoc@$8-!1&H-VbP!<s>< zHl)yB=WCP2KM$Hc4lpk>5!gO?;x`M}mGN3UPoU)KAmkW6O{<bE%c@c12nbQHM_<V_ z=nWh9PVqIltZ4b}x0#KPCJeQbwp#|_!AAZqcWjHX%pv(Lr;H`<r-c7oaJ6mzbgmQu zJuxyQ@?#QwCks7z8wBi_FaH<y-xwV45d~4L{?LS}=|<axoT+^bJ~rB^PD5txEs=pb zkl+sV?hUOiZV@UHeAz~*HE`tF8#Ir8t-1s7oB+z5_#&7Slc1zDq11%?#~oyua)h*a z+8N-YDA<H_p}wmhb@m-h$>Tz9Q&Z0nuWYZ`fX9YABZe!t#-sjw{JXwmzSGB5%YO9? zUieYD|2M<H8dEDEjTB+2`<me@9_ZVsVvBZFKMUh)d=w-1IQFrTVt31bHey^C3KlbC z7pKQE>y>tv@827+fU|rULn#|!_*az%Y+4Tl2A{0>nC%3!q3sM?K%XTo06)^1zr*>3 zM#rS=?T$5XdlQ+KrukFG)&LRTW}t=&dqrMEFYbK_Hhm&FhuaT+pLgN;g&NHW1ToQ3 za<N5AC7~d;Z8y8GSlmj){#5Ttf$A_(EY3mP@R2$1Q=K#$nubs!fnEgVkwdx;<{FKk z1$Hf`F+3s^X-)R7;Kfywz5wdTJWI?%*w#L!e)g5ztefK8z8P#V;n3>b-fYR0{y6f( zi5uMQ^&4KsLblPtvnB`+Zigd=T(2+LKycvZUy|aPXLuyg7P!z1AM$4ZOZS_#tqcMv z5<A2`=mWf<&!?oL`u4Bbl3>~3&8EJQN*RJjr*V8GFdm23SDy00@HpO}Z0n5R=D@SL za{e?ynMt*`z%#q*?+OX{4q(|Cq*>kBn}^rW$QPJ-9mXb#Z?F+`?WyKqKj5m{Rx6dy zN@+QI1X2OLPdUT8_aWnhjeY-&uS_55m{Xgz2^H}1UeK&nL)Nk?4K$SlxmKZVg3)!5 z<pbmB16v8plJsNJGbV<~>Z+kYm)RzKPpA*Xp{p@&h#apwSiI5P;;2u2)3Vvgz50_> z`le1E^Wy6=Ui#)w$VRXyU;w8Aw#Ugk%sb|2dMisH5BT};_uuN*58ck#ZO2R>bT4@l z+G)Q-jij)RyV(-^y|cob@i>LE-9nSX+?oKQ6imqZ)vGt^>r7barzjVCsgG+7vvc3` z*V8hS=AV*oWX82Mi<Y;#^>Ylj5^=~^<2aH2W<1HzBx7f?XcS{o*L}O|o|i5Dgp<(K zMz0fq{j>)h!Vx2g&mo&YnW!AGE3j>8QJu&6=TpkHxJS$0g1k7@lL{Q-87Z&hWSCpf z%qCb<O$T#zDY^J0j@7|xFHdu?F#+WT=gt_ngYuz9ibyIGYNLPj)Gk6fgh)_w$Sna; zB#63O_y9l3QkSElsx}zGVrKoJJ~l=!7B5{d|NI4DLGWQqj?p!TO|<*>G+L?$)kL6x z_Rd%?W7AY19&g)D!yK-+pH3MkP&-)E^;7NkP*vY3Af_UeLrGdm#k9aZ%g|57IcG0E z;geFCNoB`t#P#on){1P8qL{PX!q+PNwwiniTj8C(jq;_!!aUGo18w&UGXQCjBR{zH zujrD5nVGIw)2R(nqBO@f;Pzt$rOd^aU&o0RwLlV>z~__R402laxw|S|wqx-s(^7r_ zOAT)6Ba}7lhIL-mL$^WtDT3l{`HQIl3kMO5|88_aGmyBP9B!9X;a0G_;UH1oZ}o)p zfqDF$(eg$0itXZK^N5yS@)Bc~%R4{J34ZbQKb4WK4)P}VDL<;ilk)0B4+3H<<idI; zciZc3*I83lm&`G=ti!=UXYHq~RSvwhXltG#ggmyebozVSQNI@K1ezy~N1!SZcK104 zdS)Gr^Q^6tPFu-D#`A9it9pUT3-2LHz2DW&A>@qu2c4_XUXdmKJ|%i<5>PQ~MEfX# zr7k_e<&n!9ROR-*e0S81whyqm^|zV~OT2^k&kp2iJI!-G<nRsckf&pjIj!%!?}gC_ z=P@taTf2#-eC#h3EluvUACOHi^X}^`Z;#=uQDp-i1Vr{zMR8MycxJPx>-dazf6TXy z4WoA-L`v->?~GYfF}#t?>CM9c=<$@S|F=eZ^W47!!SAqE8|hPQtJ~oMP2aZ_q<?41 zkh2r#n{20&bd=J_p+EtEy({2I16>*CagJHuCd|0lZ<0kRQN_9O_UqNf9z7bp`b9;k zzS#G*fKUY#gUk~%sXYhlo?;A>lYU@FFG+ewpBGCG9-_PKTw@*Wg*TN+D}i&|9(@^E zOS$*dV__T%)bsV18*ZYF7Hb<qP>mF9k;2u>fHZ%Kp~>(>hNxfhH;9vY6lvLhYH_Q~ z(K;W|hYreWi0mZ3m9|dZ@;9-^@k(@TEOf_^`-}+Pukw{v_9@9RBTox|$rQ$;wA)Vz zF1NqRyE!$jbC`i@u+jIA@QTx)>;E{Kr|lj^pqvJ6kc70(#qsV^#k2goL8SiEJ>>C$ z%0V4Oz$(xJ{73*=k&qPP3!`Hz_tLYaG)&@T%w=Ve(_^(F@O}njR@gw6R-Q*-yocJ! zOIUGqj4iS%4lD{lSu;R%&s@Ygp?6N{Y|rgErYUC{A5^nAtz7_dh$l^txhuA5{d0Mh zqK6cCO-nrk%KUl0(w7x_`D+Qny($V*OX(ZezUYJbBBbTX30KJev+5}2w9+5|40}(n z;1m!6SsAK^{3~0^)~9xCILABgj&YjzjB~|i-@Z#)&9w`negX~cyN<&LVSHrgG9i&- z>TAd*rSLQa@_Tj3K2>0=4nG)MxlR6AHdT_Sypnb|6F>)8+5bc?M$#K=nKS*jCsS-& z+b!>%s3E<!!q<&H1sUt2)-9@?U=f(8s{uYs*TmmDxv~R*W&8{fsM)t#&?|@s>m{K8 z20~m(G@E+i9Fs>2<N9~k31hLMZ2#@ea)ZQgM`L+dzNEKQjBi12YhjL*9=ndN_961P z9C?F|REsIoj_@NL5iBDTwtKOMb@i=JzOtT6%LcdzR^wVomamCKDzSdZ`C*hRY14u+ z)IV?&SnEgtf_e@zgu#3W*}{cLwJu9}Tq8e{JZHb}44ljo&;c|c7BO6XKpRAUG}JNA z-1V&aEN%0toY#R<>Vd}Wos&rzo!|yyOD?tfH@Qpi0Qa{d-`V}@5#?7Xx!u$Ce5pQ< zy%YQ#&7ICK9aB-;JBoZxgI`jd9J3CF^F+VnLmVF<4!+u<07HM?V_)DD9d*z)8WqH# zNq^hFI}2}#%0nI<1}1K-<mrR+_9~v^S0tlp=u^?(@SkDFF3<fWeg$P07fZ<1{>QU5 zrj2@@s!}!&)+gYDKyNirwMwsCWtiXP-7&RfSihGILr?4|L0gZ1d;Dt^0EsUM`|lop zgrDQk(oCjbmAl*ksrmc9;EcVoRO`$2n2AA-krK&tHPX1l9~v9XD7s{kN!!D===#1J z!lYWdu{s;jL9rEpY6mb4f5DkUXs+&6*gzm50f3_b)Go^zu#tdc=o}q`2af+cc=7+0 z#KHgLH<8;8Jp^!fj{DUXO)r7?!MDbmx}rS>d)Keg?S$^&UpPr+x$}3w0>&M{1AoyE z3JPNQyYRIzt&Ep^A2D&u<I%a_EB$v2^#RB%73f+4Fr|qamyzKB`pd`70}nGYGA@0y z&??S8ss;g<fSy7d1=4`$?8RMy_M@{hUJron9?Vb?rSozO&R_s1qo2jG77z5*007@V z!}R}iX9PgiukipLc}>lWu8WuKi{fN0-vGGuL9Cr)W&yH-@yEh#H*CqJ!bDaMg&VtX zaJE@M7;RX(w=JOgw3C+3c+S~xFNxFuVCJ$8-SL;GRke8^A96Tv1rvsWfiI=*b+E$u z#}|E6R7QXz$&r6N6h8<7d4$cGW<~n1rw~b`o9C2<IZF#}z7Xy#+Is9ukEWf_i9_|H zpDQ(vah}(eswAzI)LU2N<A$rA85s1<z47~^f?FyDk`LTJef|CYLkI+d{yABT+Pb>$ zzy~!tJ3Cv^gWv_z;cwuZH+{1t3dleJ$0s^04IKafe$3|o{FU1~V;V0+ToubP>7$?G NLu~`CG7bB%{{<yyz1RQ% literal 0 HcmV?d00001 diff --git a/website/www/site/static/images/case-study/linkedin/xinyu-liu.jpg b/website/www/site/static/images/case-study/linkedin/xinyu-liu.jpg new file mode 100644 index 0000000000000000000000000000000000000000..89813af2b09df122d9b0ce4e031c2014c42457cb GIT binary patch literal 100350 zcmbTdcQjnl_cuIaMi-ro7<ER}h!(v^CweD|-h!CXi4rwrFlvy|q7y_XM2I$`mmoT$ z3kjnKd3?Xm`~IG_-oKvper{QN-E;4`_p{Df>+H|o`~I8#w*sKnQqxoe0D%C2#_b09 zw+c{H4RCb=0G>Pn@Bsh-5&$sP6#%}qZf^nHyFCB^f_#Gi_X<H&KIs40|7rZU50KP% zdFk`g+vTMfyQrWLKvG%r3HU!f+~U7B*MCjwDHhbvBtRW_C!93%*ST`&zfJ%R2|*M= z6bQ%(AfN$)Xn_BE0BpB!CItQ$|9$NC1S9}~35keFNXf`=Z)l_j5CB0S0x*b>@IS8s zhTOITz%+!k9KuRObOyG>oCtc6*U1GWFy-1V2E*|`a8WyC1SuKgT?o@XuKV0PynJGh z#3dx9q#vuOs;O&eY8e@un3|beSlT-{Iyt+zx_SHf`uPV021Q0i$Hcygi%&^SOV4<h znU(#au&B7Cw5+_MuD+qMskx=~^SADv-oAe9!1syCsUOobvp?rn*Kq3_n_JsEyN5^k z<CD{~^NY*>@B#ur|C`qB`F}I}|L~%@<wXDngTTc9;RPh{{||8*Fd>I95v`H|u`Pm* zQ{*)Xy>fCvZ5Js_)bJ039dewE5iYjMb@(4@|7G@nM=aw17qkB%_J4TI10Dki{tFPn z?I?jjw<86<1tBrve?Uw^{9lm#KOp-r$p3>o|AT+God9q9xP6%DcE3YPO!_}<|JTO9 z#oKAg`)>|F0RrAmCJ+rk0dS3MXlcX?KG-r<^LxN*28c8%w9`crL@2R%g=wX_SHC&v zI^EjsSUL{mnNN|el%l*_qbq&$Xf+4qkdDK6NI^#B6!;oF?E+EtpB_t)ScNZjFV$z5 z&te^jh0rZ0_Zu7y;HnUQzUuAzA0%7y$G|5P8F)?VjduzGzLiwcnr`^l_);6GVbw1i zJ%imMmEJQ7mi6=ohf^|anWy<gR*#B@R0{9xeic`jDj_K5j_!o8S5p<so13tzD86}q zr>$DC?70hag24pFGz`!Vh-y$$L>kkE0Ln#DR9J;OS6lk-Xw~uaDEOoJeb+K|cXW7! zH+@**^2vI(Hu_c*73E^NkM<0oek&(<Dz2rCYLfW}pxP8QNQ+L>WOjFyfzY8Y@dv#4 z0`0pEfkZ<I?Wg<mpRiYMtE~nP`LxA6=%5Vx^slFAb}asM{MEE)|A<K%%-b==JuSH3 zjO=##zBRd2X`M+?!|r6n&QA3h<y*CG$aMr6SpP0>rs6QFAp8%o=Tt86PB<;mkd^5! z&x$=nvQOr?@^s1h9W7RS0Kz67OlFa9+<?MyN&vi3pp>v24TkE|@6lt_epjV`i)iL| zvQs2zMX<(KvmCESyoB6dL_hF=e>w&`??fh>tfH;H`IR2@1GWw#uP*XQBkIHb);%O~ zUz|lr*W;%*bUYJ|KH(nu5!3Q37H6oeV=n(1SAo87Z*=&u<M$OAKG-aOwbSm|5ItVY zUUDNos>K}1c`vh6XpZ&YBw?MA3zS7SkEl(pKB*xkZ51h9O$9W6UjHG4m1P;Oahf0V zN%dN0+0zIpfs&`LLn?}B#E`lezG{fg%C$UZjIb4!RcNX&Is<lYG<o-8me)|5cYv*g zUlkJ=T*!??QHT89EaPi;ue!Gd$RI8t{$i+xOx1}H4_G~6B4*j0PlZeIN1|}kl^xTj zF8KCx;!j<id2|{Mnq#!JwoICT=PTZLu0#nLO0q6Bw52e0Uv>K9%0Iv<QarE@0%|fV zudiLBE_7dw`9%<Ep5&|0wrfjdF*!(bdqF5d_~wbyZQn7YJlf<m;S4to&16|eU52gD zEjd+HSD|Ka(hYCS)~C~aVA`X$LDD&UnV>MAe}Ly&7KJHF{C_I>DBB|zj5g!VKi*%8 zbsu_4!nBgCdl!*uC{%S6E{)slp9p92WZ|OG>S|K)OU#l_v-&#d0oi>_^q_w@mry{< z$q?1M{$_&|cV|8k$5arR?uZWkgK~g+D^j(fn)biTtF5d6^E?B~u4aFbEmRLM#Psc< zgQo;4_L`Ki%>EQxQY?X+Re`Usu<&kr7=^mKJ~*RRL$(LW6qFa%-DsxzQp5zc;oa>2 zXRCT5j4TbSWMSr3A2JbyFwwDMHDa8?Ct3i%*PgJZ0iBlAnm-GA>FE_~IE3H-v8E13 zyJa(hV-}(b)r_}lnq1c_hra^E71^P<ct9o`-P>f4?zBJig(tzH(8=9dfp=vsMHAMN zlw!t0a$3SfLn{SkJZc$`avJ?=t4JM<U|05c$C~!*O#N+&Y(J8@oD^Xg^1zdt3!Oq9 z!AjHP_Q%a_A`-6f{{YkZq_7m(GBe20tk0-r8G`Ak`BgPRG!(DRIBUqOOUk}Zg_Nu` z`zCaH#2M{uCx=Kn%m)j+D*HGtTB|W2XH}zj^Tp)FJL5SbhD@SU&X1b>y<Tu@0!jIs z6S0JQoV@yv#r0S;^GOzEOKAJ7>=V7dP;CvX4Iyjpk3p%0q%J39Go9l6UwxE(&;<_R z{Z11fC!7X#US0}^MId1z{e>P3&i0OeWqg4nQlnLm;hGp5fF49@6Y~yLS^siTvM*>1 zT&E6~5Pt6&n;}<kvX2vL%N%fMfv3x1f=pJ06IjF#sE#5r$S{y`Nn$`@45xDYt8(ZV zD974brT^OKRaB6|%dvaf&LU3)bo$-$M&8WnXw8894m(+rs#F}v1s-6oO8toW<n*cl zPnh4>`L8Dl8!_D+_PGt!UeN5L(a+Pn&MM-+e;>#39Pv9`g7S8Bc`Q$~{sAfhua}vm z^2rVB%P1tM6<6498qLy#n(=HDD2cLuI4E)yXQnS?cLB0p$<0mI^`j|M{>m!_fc^=( zmdYox2~XDe8io$J%Z7WRn6ht*gsW{J&cdL0J&K~z?9OsR1<>em**A?oDVkIthxKFZ zz9OAM#`x#xRlOzH*J_2!6BF6(9E(ZzKM+LTx00OFR8A8>E3>-O>8M-X1Fij8HM7WE z;XZ^S1*<#C!Wx9KLH+O{9--x7<4m{ReIh5B>Z$ge9d%j`N6mfmVi<ssRn6Ye18To! z5d4ukJ}C;Mc>_w(Rk){aI73#UI3g*Zc2V$CP(OyppJ8Wz<}jZO!N$4L_fbdhEUT4o zDNpO?!gv|nfiiZ=)Z$5vnE8STTeYa6Y^o-SqVVfI3>iRst)HS0@wXI|8#uZeZ}p1N z5fD+BvXCO*2(hbdzgo{t=yx+;9E(nc;*U`O0G5DS_*?#ZfZyvB`P>tebranp)|P&Q zlS*U+0j7E=p<7b2ViGRNm6`m*OSBgt#{x&$N_mnF^NVmW_w#vC${1_L0PW<0Skpz- zhVmpY(3e}B;SdxVgHcPho{q6Q6&SCV&_rv3=zpvczrIl$bGD;+2TZkk?dnf;kT+7y z(|Nrx;qmey?|Znyzp-lvVY$Y>b1Ps!nXZ?sGcCI&Q&W3x?$^~PT-reHp;(zjFZJdx zv=W#1$RpS`dsJABgJ)uVXfvyZYXeE;U)-5pkD=d)0YTr9jTBRd7j4N^<W8&km|o}# zpOmj*VRDKG0as-N4x+uMex0d&M2K0V4FmXjcQr0DvM+=W$&te0k(Y&Kzq~h+Li<i8 zxv8*LZL8qfKR^QD*^@ZdZ^d;%$=H8@dv3i)sYl7%YTh>t6e1#nr@;|D`CR5coP~Ft zM+5GO5O{bjTNcH+=Nf(!|M?FffZds&r$5W$I)XY78<O@3xgK3;^_iP{gtfPBVq%D! z%nKF74`lGVSTp-5-ddo7KX<Xfx|l0FSCfMX?l<W7T;<2Qk0U8Q{m-?&k{76fz{$hb zv@I7|TEvNSE(9sCt_4!;R-?8z`MxFtvJifF?ee8@xbm|ggd*Otp-o;>8=2W9#nNV) zx-jujF~R{4wB|f!$NZ?ob}N@bY)8~&?v-2yEzDzW!J0qR%gdZnEDD;EoZoS_eBYAn zgw-%`(Rt*U6lPvU5s;kFZyQ>Git-@~Hh$~g*StrATP$yX1hD_`#iY5|gM%Uv{aOIW z=IjYpz=B`uuDaetZlBjv_=QCvQNkMOtBlpczaRC$tYOk)nGbqf{Xj!dub}uM)W-cR z)m-hEq|NS!kB!8fXwCDMgw}IUD4I5T1`7tPozgRhJ{U}K*R+&6=!wD;3bJS0h#o8x z7chY1528OjPg{{FXR|b!Yx@~s_WZL<GyU>n$@;pDck1%d2dq@)^#tktdL%<#8`WRQ z;tUH|kIlh18;pJ4bYWB>9W3XFu8&A7KC8gYdjsQu3^t<k=a;~EWSQUWNjdcipa0l? z@AfgNQy<7Sh(GywKHS@m+?gqT<qst&<YpQ@u52UYXTqk4beJ2>uq3U<gN08U%InvU z246DmwIz?Pnv^&Mz)|8)7bsW*%9TYS{S=B_HZ7y2-V-SY^xSsU6Y5Jlk&1V+>F(@C zchcs2lw#~Vf~Q1eBYe;mW$cNKBA(e+4mnyHRgKn|2!hnfKjn6fBq{LknzmoL(WGL4 zw|=nKRVG=BEg&iAS;>WikfR|n^WS*0I7XFk)ygL)Woy=&oPDbw5cqPo88N`qZs}*t zAJeS?u@LOlu4|l4b?K&TUX=)b7(p$0J%MgH_=3=j`k1kibf=5uGE$K`!qLWln`-!^ zQa9J3P@xUmg_F85u`r+iHhLGF8&I$%s!q{L){k=j0+1r-6XV|EMa!qXe>a>2&u1}4 zZ5^(?wGKp0VDPh3lYvio6Jk9&Q{w>^nhDBvuOq+cd26)b13?r~7|IiR?jci3e})yd zms;Xa)grtkJr&4q)g?-!oi8$P1oPfe8Sx2?$o1ATFG$x|s7fSudztBR3QVDT<#X!> zK&x3E6m5T>r<6%_TjhXqFvpN|9z0QJWiiM)YCqg|Q*&zSE*RhrS92j~#|<^B#`)nR z_an12@LK`;teV&>SGHGW!qF_DC7IaaSY0MHpq!{8B}V1fpNu(U0O`%iBc=h>^hikm z%PoV82-AgKa>;e)^JqGy2Q3RQimL5}f49<moR=bjU!iR0xPtPVWKX{5LcnOTUXx_8 z2)_djo=ZgR<-@_Wt(1Y^`--&vv?Ek*F(Ks$Xowa*&tOL%*0HXSX#BKXrRPOOM_m-O zRl1`{>FhwWm1d_m@$qTh6M9ui3Pkb%QCXZxEjgCNAo=x_ah<fZ^nGKmfyAD$%WYtZ z17E>w0m*Mi-pAz}0aICvSs&7D1cNBA><Z?Xo*B!K0b^PG!=8!iQ<rDiEx?c<3Lm)< z7m%_^c%MT!hBEw?Lfk9tdw>0QjG;DU=`GU*llRQdT$PoG<0p||hcVFE$EIKO2d3+J zdTynp9<8z;PWvT`lFszth(9P~cg?V=feG$<eD(8ReXyaJ=}KlDvO7ZB8DB2`&ROO) zII?*%-YS!*b3TTYX|&N;aYxizQRG%Fjy*)mZ37R;R3@9Yc*jH3^Uo@Yj`)i;?Z8Cc zgtiTTgu?PH<JDE1(R80spo8Ewv;gG_CQ9$Ko_nMs?XV}R;wOttSc1r-fIQDrn5&HW zI$8YUoQy-D?tW1-G^e%yq>@gf+^@E^AJeMbUDF5<%kZenV0yNK*Avl<V5zhA3Mt=& zSPK&Uu<!PB_bXdBF9N2~G$*}}5Wd<b;AkE;z(8%bzNlyt>D+~!t|V1y4SexVP9AZ2 z@*NcU%Q#n9k>=)?a#bW=oGK`9PzUhQC{gmafkq-k%z=Sh)~{H`>UXQ5?)dUyKFL{x zjs%oCuxs~=^ki0Xyb*=ok^P1#6ZZ1ZRDPHir2+C=(7Rv$ozr5l7WTL3n04tGDEnHQ z&{)9lR`o$Ah|+-xO3&myy(3RbUix68e`LjYg?=bmtX=01w~kPgw$TSae+8Cl*>$tv zc!Oe@1x5bW8ClT{<MZuFdNW74hzt+|B!qZnsw722*8c%c%D>J@ge|b;JlB|3y~AN* zKVNfh)cz){LxB(QZ~WT6#jAK^ycIDIJ>EH=pikHzQxp0L|C@YO(UYuN&XL|>L2v}z zMe>W>_tDHBFhgZ@9n0-?OOsO9Grn_$b;!^d0oyQ<eY#YVm&VBr+8!K!JS?K;lN^nB zj>I()c}G~}FhGY(-G?Sc4!Z@LdygCkCrL5qM6wwM-lQAHCMpRr@%M=ukteL$BR-vg zQTR)JLM+#H_l28v>EO1a+-}7ovQ9JIf@p#YPLDe&{f2*t**m)H2xE=Ba`jvfbYIUE zZteC0tUFb?>%>)43g7?XNIq+GQle%s+hD#xq{P~Ao;egAWFxL#TY+ffYT*-_PgHNT zRLQ<8{Asp7r#r7oCBB!OA6c4gD%<zMi`+KpF>0-X=qSe+<#6PUnI4P(bnl5sn)&OE z*hdy|iQ7^-m9IKl$^`Yd0<3+yhAoZt=N)7saI6>f>y7tPRo_%II10ACHtO8Z9{v2z z);WzWy-s6Ssk~5C&G}jDT8`KumTu6-+-t$91p8{JY(lC%(a#zRXgMgwB%hmXY817) zIvEz)LjBi`jk+^L^A~!ME&C?<85(D{iiSeLYj4>%?Y=E}LzXGqS{9@T8vKgf=qUuH zn9dlKjx<aSv(_ctK5^O{y_!$=3XxpPeBbgn3a$<1M!1I(RH^_H3hZMHB6Baw1+j^) z1ta5}fTFYQuwu1g=sa;I=suCUC0OmdahYEGS0NVf_!Qo#2?@xZk&sxTsx}WkZ;U@I zK4QmSQ$qcy!jCn|`*IA7RBA|N-Gn1X7Q0)1Yb1~S18}8=qe^3#qz**0-N%PAPVsja z4Dje+=n4G=+yTA7&PlB}_a$&ok(60SCDi7kYZfsTcGB{}^-NNk=myt8C}(hIUTt?> zVU(G-kE-;m<y#w9t)vWNLS%QY3jveEdnTPSQvhYupTJWFscgU4Ku2~~Z|tS>blm>B zYIn*+l!C-mybqB=Ju+V;Dbx4P?|6Vq8Yv<60&6HdMR6HCRmDEnNr#VC1s~=%KDG77 zGlIIF>-aM~7;<5h>vvYxYoP~<8xPRsT{w1K{?L9i$LkRM@Z!Y}J=x1CD!&!63K2T; zQMG}Sa>p!O7M9K1h4pWY_qjg7`;<NUk1aN2G`Z_EM)QYc<;;FpMVBWf`=~&hL_LM; zAaDNnuF6R7X!&CR^@MsOm7E4h)>sWp@oU#FdJ*N*4-NH2F7q<6Q~wRMPt4pmadtI_ z@8N51wMaM<Ehey{b@5wh$nLS=J1R~FBe<-GTz{i;1<3(d3g``v{|jP9J~y*;QFUYR z^xWzWD^x;JU59NXoavsJyLD2Ap%w+M)N(A>*UDJGAFzV*PMs@%vZ+aM>xg<%luAL^ zH*@pa6G&Gx9sbfN(upanA(>wS_GKTLy55s)60-T#NpZ|?^h7q&o`Et^5uT_fw?mS! z_?}7l20{<Ki>B#m57$`yrEnckX_`omO+86y{ReQMMi__^lyXN%hjON;=Ww+uWxTrk z+>x~b2z_0U$e$v?J=_vjpWDbmQ<r6D-#|TUi=whLPLX+Qg8IXRBrYMrEfux}Qj!D1 z0ml$}K<Z~JP5L7qp+2==zEZ!Fs^ouDed_!9bHHG#eDgyqLf{2F)6sEsun)j)zdT)4 z^niMlctroPkJSDJ8Ni*cTSkThKoe2b9eI3Q{8E4iPsP4r?#HH8_Dzz@sznh=G=tqj za2BPQx}*b`)8j9l!a27_=P3OhR#47|`$j5c6@BgZDOYFms`z+a@A1mNqRxVPDaqfY z^+#Aa9A(9TUX^z65Ew7`!!xg!0k|?T%{Sv*rUK9vdlcI=+Og!pm4I+Oq{t;1t&z~2 zx-A`M32=yYF@G;(@9VTBQ{P|u#YGz|*rmHd4h00TQ%%%6J#$Q7H1p<H&pqE_5PQEx zU_Pk!liL+acQVSB4iLAk6I+g6%xPIi`4t1|0^fgHv0;tK;xJ2aK#T0#6rEJ+x7%c% zt_BlG{~aSxCEAf!3^l;BSymBkr>NLQgIA`;ho*lENy$3{f((a0!)SJ?1V~QvNf^zS z=nem(5>TvtdGHsIAqx-8=Pf1%`UG%o1>_G<fCP8PP{_O~b9DK_JZS5}C!a`$j&wgU zcV+5ydniRBV@O0qPwP$TfjFAfC^2OBg~`!@1B&*G<cJy%fwpslXyeB+HIiovp92-j z7@*&s(gx_G#n~3WK5f}g3d;6V@v%K>!_Xy7m#g&J1;lL!ynOyGT!6V8rC<0EfcCro zU4v8YTBhz6HeT_()n*YtKcxQy1e8W5k3JKQw^*S8M+dmCpF#4$NoP9Vsq?pPuoqhG zeO)&URMSWRlSOI~^Qi&kbMOYj@4Q~&oOSSuR0%m8UcY)WIt^tewXitqGG`Dv*vLG( zk*@!ycAood)wB4A3#y0|rVF_;f|OAt5N{$P*%xMMaZmAmQNqPGA~`#El}o6NT}{5o zTVF;>W7=s>I4SKcp9ouT$4i6U?ZnL!SLrxm%IQ>!YtNEfE`7?TZMwku>9RhfTN4?^ zDt!7xhO+>~=$<0un<sC>&B#;u+A{;7Fi(;d9P6^>drwI<>s*qZB)5{4WKP~WT`86~ zH$FhGb>OJ&l>O!SX!C|~&yvkhuB~mK;&JgZThZ5aK-j;DYje$e5@;d5$DT&}M#M1( zhwJI#Tz;t-r?)Y|fw0fYa5ZKU?vmRDm{~0`Pnv?&J1MkrHHWYS@6lD?3dDtdR@^c9 zHI&>rF8nv45wYzS0FSPDCca?g(q(_<+ALh=^qO?GziclXz(Md_SnoR)F|6vN6Q8{9 zX-b@UKFrw<*FKHR)g7ZMJ)aca)NU0DxKcG#X?h-9M1A4FkP0<o_ak9s+&k-NxX2c7 zA`N#0zgn7oWCAt_ABws}ojg)mXy2*BE*9EvUc{D{6_V-Vb})A3J@L~Mb0O2qEfha~ z<WPTLknAMh(_P^o#&9OK8+I09Inoj-k$PIui>%v_0HdSkMy4aDL2p*XXq79ekf1eD z%puM4D_wbOs_FLv^Et_3QMa0@rzlE5^b?v=dNUXNlD`}~%4R)1O`ze4f;c}uA^r!b zQcSs@HsGmUd8a3MG9bD~3rQk1Dr;8|N758;-nme;&U!WOKv)~}eWmUfaboZ!E#RQ1 zg8|>Km9Rzk1k!?n(S--o?HJxgkI+|?P!cqQ4^`2OTnW)-aY$U5!XoHm(q^7d1CrM^ z*$QJYD{^}JOMHtR`r54Ez4(%M87@U}%k_4}Cy(Hw@r2TILA=Lan{24~P*DR)^@IQZ zH9oE4YN&GMS*Bsmm+vnt(}t!HZOEy2gjouN3raj{&IAC>3H+Te4IO*(fi2=vuuCKO zb1A1rHbG(b{Yg}4qnNj3z}d<c8Dd8jO>Os3(7fwn)OO+{OEf{`U^OJz$WbL?o1~9s zKE*Xv{RX|ys+lzA_W?^32Wv{FLzsG}KNz!bwD{Vro-Jh*%byy9mHe&JG_h=@itPd; zzg~wn61JmGd7T5=9Cf1HZNe3D7!?IRHj{Crp{>@j$0b&Q=pg}5@tp8F31!7!E+Knw z-d7wk({Gt1PizFDISlVGDw@%1z0;=0KqE+g=egA*BryBX7CgO|a33%9H(ILG<Fz6P zlXyaiO@X!W*r^-My=4d9dldP7W!1`4#&Z*`wL6vtTZ;V+#a-U3&0GslD0J-$yww&( z;@`BL-2X&+!E)u(EP$3KM2-tZg-t)$R}$}tEU#y10BvJNpZyR*e^&n8wci)gdi;np zJ!OTb(~nZ&HzrJ4Al&CSR~t`Urzs3syO%3HrT-0vu47x{jT}20-O+v18|ePcgaaq{ zIie)R_W`+vHZVo=^0dF^3!|dSmV2GXaCgo}Kp9)md}GnVZ$sk~Ts7ZtxF+3Tx_<^S z^i~9%Pdw!I?ptn?GLi3}e#;KE8fs30Qqu$UnibL?v=Z&nmdm{UTY7*BOE5Fqyh<Uc zVy^A4_qnRr6qer@m=i+uA^9u`c<{^@@byXp<J{MEt}_M&f+G60*#0tx=Ob^1)@*A} zQLQXme-v__w11)vS^0n_u_$SN+Wlg`4>1EaMt97FiZFSjjywBLN>Vh8q5Cg&e9K<* z-=|a`zI8}AE&MF`Xkt-ltJ&-pzf6Rv4-<t<yzei!aZV26kzX<<0XG`$sd$!7nr$H$ z0@ck$PHtpmr$P^}^PN+DTl-Db?^}sEU-=yV=Bh^AC$r~BPde8>a{RO|nkb2bAKpgp zaE%R3b~Zv!O-5vr2P(o<`2@NTu<JE)yhp@8H|CSj?j#|?XBG7xO)Z**)?HqjCfo;U zD>s4p<XXx@zegBIZKR~Y-UgZiNWLeLu{iGLRYR;@{Hjw_&VJv*VPffpp>ETeY2qJ; zi?5QPrpd^DoFaiB<-(Snd?Wc{LMGato9T@0roxAr$vUzD1<-Tql^J~5&`aOq@=bho z7cnyz&L+kMd+~;H18C93_D%g|u}}|OKY6|IYCf(zMAWq92qhvsXJBlf^sZV#t7rw# zNlbiD)(^=iE|)TY&Skvf{mV`9?;mX>{p*W(TOdcm`;;Km(H-Rt|03IfuC3B90CT>3 zs`yZA?>tM#)l|DW@%;sg9~m|yP3y;G{y16c;INZjDM!8tP{Md_ad2*A4I&$Wya+Ym zk6Dc9INvi+$e{GTl9ca{N7w+i8PpU(1!Y{<OTr`gR7I*hck~}!#m+}utr|GOI$gQI zVob@@PpoX{BQV8bVTT08$~6zc%1`7#Z5I!fWyN}&ffY0K<W@$Z5N&DfOm}I}A`t(9 zQOK1W5hctV4wL%Z>7P&jf_lDD(heCCEsULe?A`o@qSwm7f{D$N*=rOGkXj3eH(opr z^cXNoHn`I#aKdHvi0o7Rim)Mww^so9=B$uFQc^DQ2|xsH<gw94YnBf+O}TqyZ^~st zfA(=r+_4KM$(Ha!?L0EjW}PD340_W5RNeYJCMxdm_aoJDNf8^no>0cK{%nt)PfV;r z-xEo?_%W$Flk{j!GU<z1RH82M;>$LHrcGQ5ETvd!M6#@JBP1Dva)$dnl8tp)5#{%4 z_Z^jQsQvX21~BuZ(#_mg&7^Jt<z9RL*2}y<TcpD6zXUit`JfDTs76xB`jyn(C329o zc77QuW5RO=9H4M;&`@u3y-^MuHM*E~?49|!e)Nq9LagxoTfxn^0)8ko?ApywG3?*u zwK<VU8^lpMJJF;`Ilt{da!#~QKCS6Y=wl3HTu%Mi9|d_g^@Nu&z@4hLB$K{?ntRyk zUxH)kr<xR<9;4sGs-&k^E!iH-Z-Jme*KX|ixwMD)#H>(PMy_`u!_@?l8LTmH+1qGP zAIyI`2$ZAEmZv`!@+QA!%W4_5kxvIbZb^u>^(?J+yHowq-aZBzxu;|H=n+W@|MP}& zNgdD5Uv8~D8NXJx!i`U`H|d;9^lt~e)K*7Zj*FrKg+vNPVbB$PJm3xAg-1}Vhz&GM ztI5%^2~QuPODVimk86Jtb2m|S(1sCu&wSkm495|dAMg;&WTDw0aHQN5^K@t*szqiB zLo>;;`?fw&154a!nJ`pat#qM~ZY=?flQ<|8Bw0-UMN&FHl)w4R&%t?SDL;43r9Wkt zQ54U+7iq-K>h2M)#H=fglzUM`80fF%2qRMia@TTJ%{8qbNisQcB<WBc-Y-{b?Tr3p z#-*$79(m1M=vG-~S5Y1MterKab|PGWJj)ZLy!}CAfKYQMF$ytVEx-5P7Z1SOpZx7H zb?n&@z38kEhK%a0r3qB)X!RyRJik@}^uQY)o6Vn1Q2891a!PX2zQu`~0U2C8Owz2u ztc$!QiX(QmaWN!t2ZZY6@MhV_k5`&|G0&tCGh)E%Nj0Y<Y*&Ac$xXWD3OJF8j#F{@ zAq(+InJ1%>Z^yVws(?0kMu4EgTb4l9wE9t^La4}Nn3~jnE{3`rMIG#)Pb%oI-n4>d z^vus&D==96p<!6M`8`*C$K<Dz)yoI-L50o;NdMQM+86KyR!6M9m#&OHpfxTNE8*Q2 zh?}m~f~mUx5E&6honBS6{{!&YqKn^ca8Gh(aHSU}Fr~HNnD%byfx6(Xqq^_L>o%7^ z-e(T#w0846qYNB2Q7-pcdz{BZ-2?TWuF^1GtxZZ}WSMH#4IU%pAIyTeIFo6Waw(3e z>o&qh>`~SqHpQ8`R@P5%OA00}xw98FQdytr#7ozlk3+{t$zuB^JFj>MNW1$~7q}15 zLkhQflar~nK&nv=qm~HZwyOC4+yw7kV2V;i2=U~fN?{S}d4bm^G2O3sQ2JoxAcj;q z+64Z3?B>xURPvWkf#wX>$)h@9(gn5LO&FEE9tY^@I>O@5fH%=Qxsw)F9g^NORD|Nm z?Z@a7t~Q7U!q&4QlN60F{{gf<kKaXzg~5mxbK6;e44?b31h&s>l386S^&P~RZh^KH za(t+@z~GUMXX1P~rcSm5Mw=%{OA)Rpqy_~s)2O9=+6&(dDM=^EJ2#`mNqt$tzIWQ4 zUOpl#CnV(t|6QRaUU`x6^^hv|xL|WPJN=_4NaE+!cncBxMvx~oh_V5RrHrTjGnm<7 zx3H=Qzog`QETc7mr-_-Hy`h^SK02!r%UkPs>&SfZ&hJx?nU{!TR&2=>7;_0SpVRAo zpzusoy~y?5UC(M72e?|6T&L8xzpSn;9pS+`8-*dmshgC-SlrTkhS7K87G4mP|HsNZ zp4u%QINvu<nCSz?U&F4h`H{PG8jMU#y435@O!j&9@pA8NOj`O+BXE+wTXMu&uTmge z9GZN@x{Kv?*6OmU8Zfkg4e<Vix_n1Wb{xCAhDc{nqWAd(;uE8TH$sq7#lgX27h+@~ z@u3V(he_R6r(r2>I=EK2=R1Fm@#?I{+3fOK0O^@!%iiYHG{yw2MDVEqpASP<j^skN zpVVrdo=Fm>B0oW?+|)zRHSl^it&DI@MW-YijyD3gRX~Z3_8$C6>9t2GLD0l7z9y=7 zGAj;g*yF3+obI{<lc%$DBUF$(xse^bK=+Rcu6NS_a0>6S#4l|>%!bA@nZ&VCbrkD> zAiPmXVVZG&n&gO#eU-3=wMM&R%a#}bj=mtUppN5Q2?+}0^D%f+6mO%UaU>*(_0+i9 zeY%?EMYOOHBbW3DV*gpFf5S%OHe+(f$0(Jik2#cN^kPpPW6~A6a?HYegoS0WoBhTF zc^fvcGop`K)Odgs8u@DKGGoABrV#_u7qW50x~}i|A<gk?u!vIEXJ3$E#Y&K+DtTNB zRT^*^pHH4z^ZPVh2nIhKO9~qG;~V>q6UKfCvV9|K(wzG=|23Xk^gu*(nCwus)v-%a zlZl1g=jyXQD4x_5Hc1-UfIm8`I82x>$o!o?TRRHrC&<arpjVOieVG{qx)DQ>GN1m! zg$FtBDH6YjDr$Na@%-)d&I?MpD?GsAxFmd(J1f@I`}0kDhH}TE`a}-zPlzg3mCG#Y zSy(Z^ERBXD@^5$2NU@0zP5=0~cWzJc<&?Wc+QTQiVamLtjbtf3k*LFoa(3pnLG|x{ z=uwmaFlsG1y>s33XF|uy<lEnid95glvvM;eVaRHmT@D&t^{uX)q8pTLS6|?=tgt9` zwpohFc(dQPZeu(#0>aiEHirl9PFVn%H&O?20Z_!4G0*eC`EIa+KV;zYM|zU`w~eI1 zGsI13I_Sca<W4OngSc9FbWascBl^*>dV|Y;Q_pQ8#p`>5XS;?7Q83lF{6+`gJOQR6 zJ`0T-C=`FvYmz2r<@orhw@)s~O(4AWzQ0GZ+FDV}%1V5c2HYjv)4!Z8nRO~ftNO++ zMUiQ1xluKOQQVrk!uayY&HZ;iojSFHYk*<ve4ivsecq0aNW|TmA)xGG>TiAYHW@F6 zK3xMoWg*zU3IamqrRBRPim$BS1XM{`&XOanKKB2{9S+~++ohSRUSR0ldxC;Egb5^s zBa&z2cA>Nc8KClEej)zBm+cWj`=Fz;j_+~`S}n-_4Nr_g<F(6%P1wJwYwP5!?*mMb zH}+#|6^y~bQE8wUV%fvdRa4rBJOd1!o|GZAI!p^3p;raOV5;JP&8V-g5gRJpztio2 z47xly3}BGwRff(<SLI#TCM$~(WR~f1SGir1PuY+n38li^=utkIr-SmYxt$0JcappR z$%hU93}5SZMNh!x4Fg=8UGtvHfKke<6iz`a-pSWUeDgNa>LC3Opv-&J1e(xmP?IQ` z`0{yb<5W#evY&gRFy_g4Yv)Okc4m=bHCcrC^jKnc_+SR`gYSU8{MQ7HiEclYHzd4W z6-{;R%mdwmeu<L3+RUE^&_Y#%G`C5&^*4+$i>0~kA$(R8o{rxAFz^Y3nj_z<o^_`Z zlZDknNv;6w%A-f%aIS>n7!%?sw9r<rObFk&kwEPyTbft4G7e;;>Fqs%p?m#jhYLYd zRXjEou?T%8D@X-!XH!MX8DX|XzvD=fXzZBj^N2e6gUljX#ud^fdkZXGBY#_66s(pa zs{?T4#S3~VVT;2beEIdpp9DMlEL~s#n*3mbSG|uE6lr2<Av+}8g~(Nh91jDYUQ9gZ zjKET=>A{VDH<NI*#32u{R8!u)>MWh^)dA#5eI@wV^yZ0h?w0hm*5<a@7$$+>tEJq` zqR&S;{{Z7RX53qyop96^YvVK0=T2NQw^_V<p|_d5U&&)@;oPylLu6GE%n>A`?D2uo z#tZ7qD6}}hJE2fcfvnWwOlUx$bCX@OljSyG(%kZwzc#BKyZH1+Z3HBdIJ9Ss@aMY4 ztBfS6Wcp8iUaHy%9#?SxwN3?HVp;dsuX=8k9-Nw<c8lTm(AT4n$mE9nN|-(wKFrdU zf{6j}U?;m>O~g^l8SJ*kMEVo?Cw>V*MN?SG-C}|JQq}U~%&+Ry@kysKfNMU?Z~xMd z$@>?~Tg2tP0Dh~wC-8A7ZL3Y#n;)vd0h~h?Y)hkOm0J91=`j0*9{-)&HM_!0xR+<h z!h$)4SPPJHGX0!4*70&WG)`L8jf+A(i@%ITeNcDGIXKw7`qfwkD!B#GN$Hn*?ks;T zx7Y7+!{-vj9pUR?WYiubS~MWDF+$EStZ(d(m16BNGVn<GIT?UoV~kyW>eR0`<-n+J zkPBfng{ZqerUl|k_^W1E?-1)n^lcvT2S2s-6fjh`!8YHzWy%X|IR5=4qvz?%4bHv3 zhcj=h2-}9C(vLs)$0$V%Fc;$BG}}rCL1ypw@){J9PUP;7s!ns`^Xs$8qtmlYGP`rW z069O~$9`_5)cLE>nb&ahx^Vb9z2d6t;X(fWX{8<mtemkAfXXO-=gR)CwyUtK|7Ayx zFH3QQh{}hBh#7f@cnxbbhyr?Jav|(p0}OR)^o<?Pss>m??;`c!LE;J{{A{L&!`BzW zPlvcD?b45^9^M^D9s#n4GPq?YMjAT*=-{|6z49%89#XgDcQhu_Q;?*rH)L%8g2Er} zZO!nz?;-sQyy6@t!UA?-z$yqy#nckRSAy0$kT(m-7brq0+1CT$t?VzZYZ6UqTIQlC zTnFz_!D61K<@wsp=AT73Grbq?iC?+uQriK@mIbnFrD6ZtSHbV}p8q=cu+pkU6VZLh z^3+}WdsGMxDwpb)|4N@kvtYX}f?iLl{26%dn)>3MKkdub7Vj1xme^wLc$HSsgEqFg zX2Y=pqt&e3jp@gqgBvKXP65Ek<QeWO&Z#ihnt17+wWAQVkc2@--z2xhLF{G+G}6#G zZkpDw9uzrOHNWeA*F9fz6nxba`pSf-k~}M~Xo>c^U~T%rTGOGz8(%a3i#t^)tH)HQ zd6L{qU(UXTQD5CHZ|iqtEesBHN4#wk5@vgjftsz0C*$5$LF;Be;IU@PY9Z>wz5db? zeb4E4B*=X|*ERHn*-lhKq+`xTbq<aA5DUfkLswju9)9YGQa;p%z?hl*0b$F;3dO!` zEh);7IF7i-G1iCZU|eIU>ABu%*AIdH{kvvkRmB=g7THpMtw!f&Gq(|;e*o9K=v*G- z#=A+yk7j9-1;RNy6y8c*gzV*X*ScJ)>1Y#YeihPZhG8Oq8};%anncsEk}x#Fz(S~H z@@kfBkeS^)m(hYzl?7sRe8tx>(;V~o%PU-r(t&i+;QbE<bxc9qnzT`>ON#LDz<&Vw zAN21?A6;s5X8yEO>89{IkU1`jIRuvk#0f5RsOwUaTu#-0eqN|?z8uB3curJDpgHqB zT$4Vqe`xk{%8*^;ZauR>nkdoDiTcZ60m2XHbqRo7ZfgCrk5PYiy)tpA&{z5dN<w&b zJt4ngu5KW#xBdWqp|G`7l^?NY>s6W9A}5;oRm<m7@=&CgdxZLyB7OH`u?Mh39E62W z`AB9zMGq}FCVYHW(aU5<BL8%2V=Fbj?Td~|I@Dgi0SKW0s$QUP^YbcPxVWn!fHZ!D z_c5=IYHsjxc|Eju-~83QS3s8-0E<mPZ}IfQxWzi-IAxI(Im^W~${mJ<;$*p0*1+>A zjRH)<y3I<HaNljGV##%JTvCAafs?fyzVB@?SkND}fKmE^YG@Z>EjTci@bfyATfNwy zXd3ayI5NaoJfO5k@93&cX+c%zp3;&1@1xcmUz2bv<RD(y3$n%X!cxEX!trKXC)DRb zXE9wrTv=K7yo(tzwKZKFV^?jPNw_%atHiWWftZx+FP3||!f4du;BZ!D`H8BzAM;1c zq-i5YhAhgP*NoZB&x~a^pb&5GoP?Hi|Kl2qkO}=`Y)7R-%?p^(82+@NH;hSo-dEtr zDcK}WVg34gb?2mf#pqF#XQ-i<rd<78<_Dz)p}|+P@`HUY1S=^r#k90a+e#4jYrghk zMkf<yO{n*|(JH$qc0lg$=7-NFbbTYD=V$rEX<Rd|?E7Ti+cJ|Isx%E6AYs-o^#(Gu zH@04q{kLVVd?Pg?XktJ_hzr4eKFGlhwQ%C`_<EY@VoYdg=Dy+Yhev37P(-J5(z7Wv zXQdXiBzpfb%C3r?V5Qbo1CBe$ol5!Sk_kcw+)oDx*t}fb8M$?qRrxNQLFwBe2_~~t z>O3_w5qIOE&h`f*(<3AWEQJ;<YO5^{!6xscP=ecC>y$ig1m&)27!eHqEGV=gFf}OY z?`9f9=fKvEcG`(T`rTU(>c_Q_cHI5As9#>f5mJj9uUc2TamZXMFTUr3)GYfKsuzg7 zXKn?}PPQ9D2}v+H{_pEfQ6;*RQrkjmk_Du~5}C}2-id(YBL6@y<}HT{E0;Zk))bbO z(KJd6Ju##ZgM{T%?6w8^b+}bzwwJwM%<@6~Rz$GxT2U&iK`LbNJLb6J;ltnHx|}LD zVC=bRvZ<nmi53hjTTW5*hXhyiMP)qSi_VS0A@OVU6ZV8}8Qy3Kd&yxS{o@u49z*#x z@3y$J!3s!XM3DKT%h2C`dC$y~>^g^)Eo&;mM9=etJX3b|=Uaf96w_uY<6*=5^bW(o zey9;a!yj$Xb=|zcJ20bi=uW?u0fMu?ArXR4()!emC7Lt6f8V|Q_|d(JB3plFe_d&P z@C@1a0k}P~&cpE(&bFifA;UQ;A;KU$tZm<jFr6mlaf=7GpMkb#B-i#%xD3WykrsZ& z2W$Fby^mF6nKGu+S%s#iv#;0}Tu!dFpz8X){7FWDtjE@>v}ziIyg0UJE(DGlJRXd^ zS#iE2N?7_jD~(zhFP_s=G*?xXnNQ5G6UhAV@#vl@_yy1*p{`|sMe!`3EO<_WHGlt# zZ!7z~b_V3xhl)awP&(OXO49w?O6^M9)sW$PA$}}BSwaW_lAqrljrTw4-Z6DHqbC2- zYP!FzYMlPTBXy(GK8*t@B*~87GqkWCfN465WzJD9dxfO~wBT9x7Ap!54+Z(m$O!8G zR1H$L$SG8r61`A7d~$ys2IEfr(BP-WV4s|B@-7my_2@xbvPCiDP<1y=I*2_eEWuTU zyq8}J7*XD}6SKvTF5u4#v#kG^Eb9PDOy_FNg@m_U_yfp;_IEA~+gOsRWs1`c+Uv}P za2o(npWEnMaSHo{o#~x=)>-bOwYNDX63J>OeHx3m+$!p;8m9M-Xxjowyt;R;ar?b@ zZ5;%55H6M89uOBjLFRKkog~W_?eX^MhxN!xG$OpmxduuZbXqYzzdR5bm?5ut^nMGF ziz=P0dG=#1k3RJN3k9l%UpuvPyH+L1o*oD9$v5&nH32JIP?Iikl?KsJ$*bpo{sAIh zNg!OSWWTyhRpr=d$RFDrL30BE4ng;>`VW(OB!2i#W%HLLO0L9NLSx_5sx3gMf!LRw zHhu$}p?p`?&!L`Qf3TW-{c9CXvbv&e_a3Yfz(*ua?TuNt&&xfLDazI<Ol4+Y$;FkA zwsfBM`YDsbD4l#~W=>H<r)SJh%?9`u!r$f;cu$s(a?ASkA|!%PHS#;$l!b2ltZy$& zMHe-+<=5!F=)HR#p9Hq&<mY+!x~K5XNH~4<e}`o0@=}CS6FN1X4heo6B2q96*~?2& za~c>4N#xaUmXyR*yNZbh*32fvYi(5t4Rco0d@8!u+p-{Or?ryYOka1Wc*HJEBlY*Y z25W3~mf#yey?3r;b*PjhY*W_gZHGVT+!$*reg2y%jX%*F6QA5j^1)rhZ!4*<m9VXq z{489)(P5CkACD!K+p!i%tb3r$=%obdldNj|jN<8I_J`t=rrUADh~`Zg6e;=0SC9=* ziTQg^P;~6+Dwhe&gfNozn$Wd7sP;RPNvCeK`s2Mm{?Ys?F4c=>VcDSRsi1Hk38uS= z{m`l6s#x#cr08X4-(}+EzJBwz<|Z@dm{Z$kb`LT=-RsY0yQAGN-`;ahJnl)EF|Ggn zxgOX_0n*Ue0)f#$_LHvOQ1UK+=y>=?V_Z4(#&^oa@}_+Cg0pShldCy^IY^ck@{_6p zcS2lcTz<d$uJf@n+y4C-<F{LMFi-NFHg$*sPZyi6TU}trnVHJfI43NI<@J72|Mb@f zHw>Y(p<I7<5|?{Ecxa6$a=9!7w(O)-bzPCvjOzie^#1rK9e0&&OMj5enGmJuER5;8 zAYlTW?IrTZ6y5XFDDwN@GH1)_*i>x=gH(}^*fwT))xmo>&J1>rACcAMkcp{U1@H~E z$l|-Y*zH*x-l=jkjz>21!-Uxk;*$(AXkFVgIFwgwo8`Ye(Qx)vqOVNzjIDX}WHsG9 z!ySfko?OZyUl3_vPQx8+wd9j>aS!Nxh)L)$WzpDtml1`2RTp%KnPl?&q=txwv0rBA z+&rDv1bT_cY?iaLNANmC<CcV*Cm~(XskUZWI*son7l??UP~!w<0i=iwtMJTBb}hj{ z5;2^nFPP@sq%5DGvwWdA{kflUxNb<rtBYT-RKPxWlEGX)ys<uNf$Yc&sir_Cu!Yhp zV&wQ-i65QL<0J5Y`o|qjpvdJFWdEl`6N<TzQ#8VRPZv2lEL0}6Uk8>DFBGrKC<1(M zwqat(d}c9qI8WC+o@TOH>~Gz^&5<z3E)uoAg2;ZM5dp#ZZPce?985AAsXP%_5XSpx z{Cl__{E0h-`ke-v`9~lr3l(99&y?PiV%Q?5EQ~j+_zG*de!;qJJAJ?aM{V~58NjM= zyiMIY6Fr6!=$^>utjEFde8brtSEH{cdOqdORIJ24vgB<v(yL1~B8`gA2#;}1XIPQy zB#|Qmo?K24rgPKK9UaZy90WUmk+ylpgLAwHPF?<@2>(_oe6hY8B7R(groCEI^<ANh z;H{N@)ZOq!UFo;hfgP)#1iRy{bG)Z1IOUXVoG|?QBGL6HP%HtHwv!lAA({9)%{v+M zqd8tdfDbA)^1GWs@m1{ly3g_EZBBKipc~k5vc*)2DSC5PASa~U{$ka=+Pqv+76&$B zf6(v}PA$!v*6Hx})RooOD00*qcRvTlu?TnnNJONxt9Mk+)%o=N;V+pVPA17TJ)zG6 z`G|<=f&4rvvw~`foh5WVOC8h`Vsu<m@xjMaPg1V@IUINvxHpf_Cv<c_9=<<wv#-Z^ zsyG-S-+2G09PySS+rQ~+{RAy-UV~`g5mAg?rRYV%#wYpk;3-poB-j~^Bz<0Gk)h&p zHQsf)|Jzz)%w15qu9nYug<2>y_e5}QFxtzUQo;nr_hztsv|7(d3(~x74aCZ+$+^A` zT2tVhMI9`(O8yFQ`}*CMz*s?OCHKVQXV*_rM{`6X-=O9ZUo9LME=R}@w0ti3h`Hme zOrhJfLs>6Pmp)87LZiES_H*#$Yd(^lY#R+q`G0`nL^YSJA-{cg?%Zz$@B@-9<`_b< zJux-*d2=tNeDY^*vYKtOHSqX1Z82O#_I7}0xMi_fLq|nsw9IBg-(9>JTDl6L&5o4d zv8kK_4QswA3k-}KUxav({5~le6hO|uP2;}P%`T^K`x}sONwcbk+Aj?u-Ct}Qia<@< zB~C?sRz2_RTo`ko(Y&O51GWgg-@xXJq|Yr~DulpOy9Jf<2{mnUzG)tbV4SCmtpbqy z(CCO$mg}TOQ+G%vP0TLDm*}H1J>S*3t)}g+3vl0q`lSe?$;*Q+vM@6DP<Ij0eKHfx zIpV>cbBwXpfOwvC8Ii+=Ug#IMjp82KzT$4$y~g@0IF%MaUdhE2V4^;vfBic&girRO zs70p+daR%`WiTKn%E5g*`p-U8>G6XJh`{B$Y6@BapI&=43*$QbgRPV&LYe}?u)bFP zBI-p&?!(F_K1V#R-#jIuzC`s?Bov+u@w!8758A-~muy;3817aj&<VBOCdrW?f4S_2 zs7kmkw4B|hrhp;yu+Mh!WurBc9AQu{GNe0x^t$DASjmGfgTs-K^l6ce5A;NSswvb? z4~i%VThk9Ug%!-tl(;0ibieA-%1bL_@$RpYo%#L`(4gMWB+qQw9O<JS)!d$<aNL$z zOeNs5aDixAPQy@@@LL*VGUckDpLhRNKUe6wd&Ua+ld1g=Kn)wpwR@GUvi(KdyJa2M z#D7$1dOnJiGd;aL*fVMoS&~hF>#BSzM3qBe9WUY!Dcj%8Ma^`0WAP649sZ*0DHo&Q zvl_ls{dXd?8>VVpwViOYmv#DM8);cDd$KE*c4@6;g}<>F`1KGiey6lzr?i@6OF*9a z^VxpORcA2W+!W=I#Yy%d<r}jQ0y?Pg8k}Xr3PC+pLL@|Ma!-tcbH$xeKg5wntc3)b z8yds~&z2ev+<(Mn)dfnG#sYA;X}vZT%+PFa%|7)Ftu@yBs$d6vuZ*9UW@lN9j^h+S z9j8IGJ_w~V{Hb}*K$A@soO@ym5%=oGZ?-Pz#FCHbiZHWHqzl5s*;2XX+&pDqzUw9z z<~LczbTr}l5<3<<<-B}w5JSMgOQEGa-VtF%i@WFu3C6h)=^riYau=*8v2D(>kr&fZ zM!%V*=n0c6>4<F;Rozs8k|V;_d$EyeQeda5>ndg5#mHtc!sJQ7F6B2Q5yI?2&9nQ! zi^sF4k4DXx%e%M{2cmP$kJUMaO+fOc#bz?I0*Sd3R)Mc=lqKxGgjxQ6#f_8G`}QtQ zKFeS3y+HiM>nx=4R{8)qb(w{~*Cd-3KC*!58hKG(-|?!zPMrxWdmCkJsVvBATn2V1 zY(J+KYkQzz*^>B>+NyehgTppBTpH3!%F#7|HF~c8aXM9NtWp@=v_VMrC*(3Du%<JN zJRtqpsxo+A`Z4rSh)aFZhQv+;X(kiQ`Zf95{~UBUqj_Fc(!ZLZio3UuzVi})btc>- zuiwJFBp(S)habj&Yvgasmq)eQv(n%@H-aYZBc$v_zacn}S;EN7!W%}Vc?M*dF{s3R z=V}>T!CFD1NH4tndmScnuBDhLC)~YG?psXCBZ?kR;#vkzMcCUR880sqwvo!hi+2xU z)#Q{pVc+sNJ(C{`8e?qk6*Um32Bg^0w77?mR0^)*0?;eB>kES#^Zx?#Kn%Zpy2N=Z z+0H+PtwAC4+1LXdXQp{IqSyw@7n8Y@Cm?4X%`w^u6iEV~F-92rRPo%S0*J~^bMKF( zM;IaGPyiX{-!#A^ZPBgA@h2pD)Ux>-cE(T60X%1#azx7%u@2ns_8j!8d-1fX0hvSX z_WIBSt!}q3hT%xsK3;?Kq$t+ejy7$W+l=+kdXZK`^Ykn*In6RhWh=X=Jp0fDYR1^e zoOxw?uU`JOVFH^(a7GV6I@A2cHsRY1&fFgPsHBZ!WHFEej1l$bfF9Dry;ODYS`tST z(By5(^OAbjCzUCQu7JuDjP(2|;TkxDN6NoE{{Sjh0>qg(3$$^JW}*tafM7WE?M-P^ zZ96$$X`76RAeC2P8-o+K2P2PqU`s0Cn^q(v;FiY&kwwHZ3pOg-m^o|?M#UqH`kIkr z^CdB0DLZ!z^aGrG=A~p%UIX%smRxic0Z>j-NflL;5*czcf@vIvk})Pn&7AGPCkCG? z`P+A~45t~+0Ti>!(W)prfl_+($KgN{#K8$xIl<!{!Kn76zF|^OtAI1kMi0`I7D(09 z?*Qbf$MdJ$N|8jQ1tjx~<N8npO#2iD&H>Ir>rS_f88f|^03c_+dFfG-><|ICNy;(c z8hWvJTWM6r<BjX->p%<gBW2W$*~npoiW)0HBCIUmKW5}p9R~JNOJ}<rk&2DBM-n*e zpP2dqKo5!Jg-k^U%1GnAOZJIBxl&r%$uuHWkw9)ndext>P5%HrQh-1I*Zn?$cB>|M zB;u_}AV?TE18C>ovkBPlGqkDXpYW>zBbpH8hGW+TxXlN~wl@~wD-*k~YNQa`;vX=u z=mDmy#}PZ&C>=Xe%`6Xs!I&Q7kMX5(^0r(@$qOzrIvnQ}9Fs6&81O2rmhnXpl01-4 z3!Z6Yo;f!8I42loG{tKZgK+$#fS%N*W<i73)~ksm17avvJ+n{SD+Z014o4W~fGjyT z2tT}}6+VP_rp&<ypDbrQ0#yG1N~CT~c!FP%fa-c0xQ<#93FjYCKn^mgZ;_jC9Y#4d zAeFKJZ2lRlJJsBlIQFZaQsW`9#Q;p{4ZAkdc1y`zclD&Zg(Qt7fPK<9suzB24;z!c zTa4$Y;Yzc})4VT`RAop!5$iw~<19AHS(Vgu3!m#rB)9@h9y)>4`_*-jIQiMXyTu@O zOo8`(=mL1}5=Q_j9=WSOBDr8b_q|hagsEPHW9VwL%`7J+cPKuz0jMKinrNG52R-OW z=SBeV(sCFMDgbw*l+TvKnw|@Zk`P~okCo3S9V%NXATyVeFu(wuXWp-*K4i=ctCNA8 zagM&VM`xV}Rpf5<{A$c98YXwfK;Zj|S7Wjh%G@V*A24;sI$%_`(#9;MEz2yN1OC@P z(wT2AK33#0!RC|;c;%WoK`KZ-lw2%Ja3hou7$6{Z=qa#Tm@mA6H<*%vxkn`D9S5)U zsbY;9c>r)@J3z?pNEPHi7(Og%p9wrYHlaJK>o=(JSX+n0sK;`bVS}_T20e35F}{e> z*D*dje09@459zO_SO~8?Ln^<Ppn{gFK>?qxPdzxremVI60LFTTvwHWJ_a1fJjMGTX zoU^uh8PCcyjC<5xJMr$Lu4>cY>a$C8a9oL5&O@=u=DdRb`r_@2hujbMkKOhjg1rh5 zanYDnlUgE;_5H&$%<jz04;{ZM=BBxKi(?Wtf_itWmvTIthIkwjJ!_to#tMc!S48!C zoc9|m6Y~swqde6aBxD<u<TvR{D=AUCoO{$G=O=mkR#VWp!t6;TB|%Zs(t(noupplG zemgDE@Ok#DYh(MYu{?GoHHWoJbERRX^)md<m>mToIW6;at5QT0w|Zxkx%sGdmqOMV zt;yBYE<Hv%(ru3cn2$_VW<?#pT4)_J>r(2WI#$rM%D4nzbLmn2l^b#MFi&IBtIOtw z$ZQXKi+rbrlb@v?)f4Re$6Q&2P}{~&98yH)Fr+aAdehozHi0S5Ju16wX35SyDf3@p zs4k|odVyxga602PX)liO9DKgLE1yttpW*3|>ryKWVVIFc*gALE>Js@k?PHL~6)fyg zxJ}A?))Xxc+;u&>VzpqpF(=M*^`h*TO_m~^qd8T_&{s!mbumzIK=&X0dcp!Jqux57 z=T~hfHsfoK4<5Bho=F`x7DNn~%{tcg;%_eC5IO_Wver=?izIE8Cl$D@4x)v45)KEy zy=xbstk&k#Q#gr|GN*9reJj53kBKj|Yx_%$K4WEfZo%#6h0Jm*0x&lYN7Nr~gx8W< zwDu&$DBT!1>z-?-i^#Z%C4i0SLEDc{eADJ+wtm=r5cu%l5iFBV@tdmZ@F0ll8*sCg z0~^ACp!<L`-zL74iE$^G_yN=vBe?ai!oLCd+gtFKp>5(#6`f?V3e(%je{(!rlrkO; z81aLH$j5s9qwp7rEIeW0jU&YxET(-n`6OV+nIV*N@iF5pq;$Z=bJdI84leBKB?{A` zs}Rk|Aauo9i|s1p$OEz-G2B%pXLf{!81l<=*OSot8nnesYN+8Ij(%gFMPQbimnCLT zH_VZ`?%f`DV~(`PNdoz(+jnu(y+)4GBQ!uUjA6>0RL0|VqB0OO_k;8U>+4iVvLd{S z4-A;x_`o8iAVloT&hAD%Dgctkvd1P5nTAaCBc%-_wyc5CUl?8wI5YumA9@1aJhlhw zc&d_zk|l`k9#+)^3>;w8r+3;PVB5O~2cQ)kqxX_H=tt#14=U(^)2ibG*Cv@L49&f; z+<}3Sf&Mh%D0tQL@{FnV<Qh=BB2(2s3!bMwv;kB$Egmr6I5`;=aisH}6#;RKu?P67 zy8@nLjjgw*ujPtGF-`XdHjLzVGy!YoVr^LA26zM4y)3Gt1+m?qB=E#zc7HyV8kYnV z<%eEz#Wv<SP%A0kPhW3(0G?Q8goxMzybO-K`&3r|%Nj%(UNvw?BONpER*Rgkm~c)r z$8MELb_toH8B)Cux&Hv`Py_D7u71^T^2xLZZmK}`rw4XlobGIJRCAA78KP3F^Bv!K z1I8*6%aOM`LY_gv>L>!jgmudwx;f260DYvW8vz46?jO>f9PvqPyGJFt^~cteAbs(` z=|BxsU92zxC#ut$MV8#Fs8WXj<C9J(xm|>e_oZl`e$)lHpa^9=s7_Wq1z$|{s}Bg7 zM1hQCemvADtM-NmuHZjIQAV#T{F4wEoytFl6ah4(ZY57R=m#|#w!~6bB|+=Z=A~>T z%B)#|J9D33{{U58o5=qFRC2A3Gv0tILg8Zp`ulNG`H5B1c*5dUA-zAyr~RVe(9j3} z(ET=PV!gMLX<N&jD9G%26wx!<D=2X5$2iI9Ro-OUM2nmrn5Q8|AG^rll1CjoR|VO< zku0)BaTWnaJ#ma0rH#iQFj3fZ%{uCE$e=IIyI2rCb5NVFECPBSqrQD<T+%B;K_qot zFJZyQH5Agq%B?9V*~d;Z>s8uF-9bh^y=qk2@v+<s0h~8XQ@Ju)fdQB0efjRRmh!Y; ze|9ras>rG4q5?fs;19!^YRQ<%-ImWnXuBrJ(Y!#iB%utb++g-K6NrcJAS^o+>yN^! zM<kM=*&_@b4tgKzDIW<sG{mR_4nh0Ei?FgM*u_gQDtl+WJ#HQ~1F7`uQS2l$!6x|6 zKu&hhCYl)X>I(hRaf&X&$et^*Bq?yG1adLk+NQXUNZL6QmB&uxA6kX31=Xl_U?qrx zy}n;V>qsx<W>hPIw{w5`=(`Id!jGIX@CfJF(inodc6{fc^{6v08C;Ao%HZ^+WeO3P z<PNl5g^=fxXrwT0BsL13l{cM~3ys8ZNB;m`pt*`Wh*i>J9V)DiEHSIa^DjvTc_Z7> zsS?q(1=5(-6<?5W-1nttl~}BJYz%;XD<LLi04g1>GN5+)n$UkSicJ_Blwp+7V7W9h z;fO(kS0#HMM?+V9!m1FA0yD=v)&O^T5#wwW1TgxMS`tr%4E*5sH7yj2Q!Jt;F=sgI zikdsuo)a9=VB_x|_^Ro0rwfu4j-=GzY6}@IK1IfT!Qzn1Hms4io!N5S^vZ+B<zJK^ zwLgyq=D3<_$LC&5{{U~lno);R+mhZG`t3wO#Gm3M91L^%L*onS^nVQMx}0)|?PG>k zF8l;o)VnY~yHw-yufpFK_@eK{z9iJVOLFYXc)&%Tc0{@I*F0q7o`V^#iV=IFo6z&y z*^md4LQ6ZdA77ibakqCLelResGsg$>u8UBexoy4ro;a>b>fxng69i)w-j`N$N#Aoi zX=G#pcnUvC$(wlV-@RM8m&=X1hWzV1dD2D+89jwLJ1%RG*F=X1nnnb3+ZC&AVJi@J z5rbPA7K<gpMwnoAuS&GhLN77=qosJbyum@)Gv2{vlAWE7AZewo{!@m>Lsef-xEpdp z;B@t`OgczB`D`i;LqtDkXAzcVT!GU*)#JR!npbT4FYKon?0HSioTOxt*QP5mteieU z=N0Mlcy*;@k|5YC+<txgRz2Q<ECr-LFkW$jJJ&>U%G;cEGRe21IA*nw0yg#Unw2c3 zVY_B9JMo(BB++CGw64*?S0wZLR^s@6Qv^&DN?mcBWq+9FpVwpD$>q~q#Pb7VbHFi@ zI^<Hltjw4xw;c1&_*b#P;f68%%10PHV+ZrBJN*Tsc_LWbBcVMFTyd%OGoA)Z?4DCK ztf_(q;C%)$Rf>>K<=QK{z0;lr{pxd$Pin%r)9s?##O~fRz{Woe*JUh9eNGACXU(cP z@#Wxve+o^!WDHirTSlvuLPbJjjq%E_Ju-j7wv~9MHA>wIF(kPRMg?j_f#gu#Jt?By zbCDvb9<`wgDJljsPBT;WgiELW(sYTU&e4U>6*0M9j5(`H!2GPicJYJX-k)`18W8Am z5AhL!^vKUNEKar;uy0!stf6qqcs;9LDVZ0|i1MR2#dFIHkzpI~HGVl>A^|*pRn-c& z*yXK7rgeYX2_!2f++=*jj`gFbUcy*yBX3}Pn&*K1$Z_*2?LkL{l3Q>Tbvfr1vuxsb z>U#!}tbLZ@Lvs^wM^H)VG2fgK$mCbu-?6vFhVcIYi*J4!>z6)eo8km{t)iRmLv;`Z zyuO=cWZX_V*XLH1fA*DBZ~~qVM^^2T>6-5C<-4^=FSO~JE6X{exPnlAWO&>$Dn>94 zIUMntP26zY{r#3Jh|bkOu_7Rh<AM%P2l$8>$>SX=aU>Hurp>Lv&N2oo%6<!cMvvlG z!_7CuTr{zu7f3#DEt?^*IOys@`ik{SnWKebl;wQRPomZmR%Wx@lx8NGRFIOeIP~Dt znkMp97zc!L)2TH1+Bc01Y#(t0Bk55KfUtQo^bRxh!RDlbSqWhpxF-v-KRo{cN^a8f zlIll6_==IDww68WGFRqY9DX$(*rXxjOnkUt-OT_>>RwMY?-?KtG7saLWX4c-I$?VC zrbUR043H1-=cnUVV?e{@CC6}009HsuQ)fSUe-70sZL@hBNy?L+!lhS1m2aES`g>DC z%<fEN00-kh5V??B1_l0H?or><wM`L$>|>G*$96_f_lMpefu|}LjNC-|C$@8pR572j z#pFc4A<J@lIs3=)pa!cnZ@GS8Mh~$jq@ATnWKHV1j{}bR2Ak!ik%EnaM&beP4LP1E z+vi+JG2hehpbKo<wz=exGCeVhskP$6D9Rs{{-%~Ifw;#a1CE*Wt1}Ofd0{|2oPYZ0 z0-BQ<Duflza7RzZk|^77XCENWPwPn;W-GOR_C_<(o~+7#Spid!%13{#04uV@RwPgt zXkdMFO}SOtyG~iUcBGUPlu0=Y9?-e#w;kwc%&5y4S1LJPryjki0%Rb<K2jM^K9yoA zpm;=qhC37OS@E;Rry%4Iqa5&h{{SkMC}S?m5|vgt13!1y^`HfqxVT%3SsM&S<zbcj zj-1m^l2%FM`6TF6cc~?cL@GgM$UGCk{4r3dUBQZwLvlSwJ?H^T7WtImueN!oTr=E8 ztn2fF21Q32wCsGbw`n;i(yc_75=IPo8P8t$?LZaxkK!EBy+6B5c{Z_ET(IEco93_g z59dH1|Iz&;noYCh;{$>{y=lH{tL-EOY~b|jDUM_@g4-lgag{v&JX8}XDTiV0j&bc? zW!cwcPb9O#RgTtDpkN+&p<X9eEO1Y?RoJU1$m4Sk0W}OWq_VxaEU(k<<bOIY##@n9 zBMgpIdsH#cAU`vCi~Z7Z{VCz02x9q|gN%`Z{c4ukR$ZngC^^XMMcAxsLc%T5!ghuj zD90dFQoLJtG;M}BC3(eVdHzze%7bBS0zU9RN|ao(Mq-GuR_fU3K9pUBknwV<k+p+# zQh51l)Ki&r8<4{p!K~QlwN=gD@<GCqI3lb|3ce+`1Ppbe>@15R+7pEsPWY=)-Fc(P zkbtCQe~o8@B7x^H!TuccQOe840p1VFD7y<IQfSc#l0H-n8fdtKa+h!ZwL=Zbb=v&4 zFhHm#Dug<xn7s(bd7|tri>~&rq}xk}&Q4Bq^z^C4_Q;iIh4I~oJXU0D6o^bw7##lq z5ERs#XEN_4PB#zgJt~nc8X=<d13|!V?{^-a)p|E=tO|}f#bl8xy31}BL?<Ol!OyKB zmwcqUJWM-hBC6!rzxH)iW{uc0at7}I07{%Z(Lf~xu|xA?u4<o{laL4@!0XbUvAQXE z+inja{{T8J!pPEAibAq6E9NgkIM2DJc|!+lhQa5W%@<IvV7WL~=Op#|9@T66IyFZS zGdMqY2d{p#t}{kQiDFXgl^t=;J?X7+1E?sijy_(w867_w&$W1(*gBU3kio}HbrpT1 zL>Xo%6@0~&P6}-!k?)FYa9h~<gZ9VqoZcDuNS+x{<=yMIURr>`S?!f~NC54Vg(pAG zzaVc(5uYrTSNW9(X2(B(z!mi0?J@BdPlz8BwFyj~W}D&WdkY~U$^s7QT#h$8Hvsku zdgqUm?ykJty9QJQlnc_k=_Hnhl@eMbm(=ExO}JnbaC7Titj{j@MBBA_<JPq`8}lrJ z2;;yVN$!1XKI0J~+y-2Oo|xjgV<pt%ifx##OXNxSj}_2px<}fVaZ*ZhK|QlrR)T4+ zf}jtBoO@Tb=vHdB!0j7|=m+($4=}|vnoRd{yce~SRxI0Cs^KHR+5jHp^IbF%xeB0> zfzKW4&8@qtE>S=|@yIo{mYoP-GuP{16^O*QD<6M@!YEl-OI<zXP7hkrk4IbsASWDS z>sq>9q&rJCcaDSBx)@;vb}+{|#c@irj;Qw41t(^XYyJ^iW<aP33($1HH58r^id87Z zojQ;)UXyQZWMQ`-HZz|}yDgM>WGK9P9<_tUzO2?cd>h#FVc|<b8f_22#~@=Km8g6_ zWU#~xDm=ah1~N~2>(v%A0F6`uS0YqmH;jz*tlm0T(9z}aS}2^Pz8$pyBv#<$43+Dj zrE|K6f&SMqnXTkh`s8N4w$8~Wkh-Q0LFD!oul=45b_2BVc;oriM>LcS<&;8t*!eS7 z@I02#GJTxwbHg5Z=e2ON_yMlm7!!anc-nrw>+Rc}A{ke0%zko@yX<{&Sxuovfb0sM zfbsfbyD8^Sxy4%{i+UsD2>usp1hkvNL_M;9E-JjA4f5OK^JT{wjc{}C$>P4rywG5h zFl5|VbA=pMK+={V1aX7UKyYiT`*EY0@apV*s}I9liQQv+LW)le+#J?}{4ZNV&QfG> zFr*#B>y9hwskHXW;|cg@`ihQQW>30HAJEbLy2yD97LF&$mwpr0tO=Ut+)WNX_*|$S zr1qylYbskZtWL2o$SH@~f7)IMuc{i=t@9)!IT<L!C{MWW^r}~W8}RM&`R#A^Yo;9B z!gk6300Qhs;40~7Ez0ej^|*an@K!!Y)hyUZk{60~$KQ*$vveT)V!2zU+aU^g_u{^l z@h8LUvnZIvlgEq+;~O6z?I_4UUMs;oL!vd5@<!(yPZ`K0eJknm?0hOaBjqzJuB9gV zh^#NN7B?WTOjgsxjJ{-N02=d!St4M-WCNZJbrx6h?OYI7JgMzo>TT|PtyufVYhdA7 zSqA3ELEGNF3r?S%%u7b2Yxcllf;;vmyw1w#1HA|vxZ=BQI$;gaag!8^L-WQ+#WE7H z`YZbmc+D)lNvQlgzGmI2#Md@`l20(9XjF6a4W+Z`o-6cX<vg1WygLUu!Q3|T0M0u8 z75IJd)5Zz#@8TAR;|Rz|t?m*#fWLppnUM=2&kVm$x=nuHn}}{^H&FtqaH!ir!s8iI z277~?_pGP6P27o5@wO(yrU(a|lSG1W!~h?9<2mn<Qb??mMUEBn3v~m%OA~LhaV`O3 zK>7-S79fs4AeJONL=HI3LGwz)0>JecKcx`wlGrmIynO)vbpHS>qHkZD9q0jKbZ?Sf z!O-p@k5v@tTt_0xg!xirANGI7pmh?<8;~~+xZ^c@{#l~3a6#bkM?RDR122~&uHE4L z!<=zKGayN04VEM_=e0WRZQ4(h??%6x(=Gg!kjno6boU%$fCNb+U8-0v?m_pf1y<fS zafiYD@;X&}i-=<m$94`e_?nl>^0x*%hUL$vAoTU12(4p}ZuTKhCm&wavc{4=Ljc3I zEUAftrZB_s6p{vOwkj1t#~JjXmc*I{l!q)6XzABK%AATAgOYm=+|-O3Ol|<}IpF@Z z%UL}6*+)A^PwPOg#Qy+Sf4{jMI|{KRG3_QiWOdK4txVwrxd=(Y=ci6;lRo5=0OSC0 zJ?JSBi5-CqLn!;hZ&6Z*b^icKP7d4=j<k?EEG)8u2t3tt!HHl`myBTg&@p4ok_C|A zLC)ZDn$cmlHbfZ=G4G1bA~9-433$ffPk-~8wD$pRl1;$$91L~*XaWa{NjZ_o#|e(W zcBUp+#z{HoLG<;j6TrcPsqLTAs>I6BmTnj+{$he#E!ha)&5gP-!2`IZjom<ixc$?7 zgQ3Uus;dl6fz`?%Z(4bbFpbrQ;B&hf>D#peyDUl}56R^9`qX%b`@)&Ii6%w_ll|ZR zy%}Fo{{Vc@7ia&{{SXsNXg81bMBs+^sOCOWw$iK#3^~VN&Y+rUV~oO3x*n{<IHn7f zl~@1|91^(3E6Y0v%7RjWp;tVEo&`GMC`=03c%)wH*~$E>yUfQ73jziK#}!;(1elA3 zFS~#TIG_kjbEy)?Baz%5dG)JC;!ZI5ayn+5(vqzX+y~?ma0OK`o17S<Lh174bu^nu z3v!9RSDR|__ld@RsA&@$0=FZQ8~*^US*GcV80L(A?D}`9q_|%#e6~)R#_WGeT<BM^ zG?ERNRw_psC%5HILl9+BShEaeTC3&T7*Xq=yg#K*^0bO0D5D(b82xDHLClG+Ba9HN zu1+{RsZH8#RV}oRITaX*nL$!O_8I5%p}y0^ZWWXL-hZ7$r>=r>FUHG)0Sm@{Q<6{V zL{k6>jaYNeTdih6<tb?+Lcw@p?N698y1YCib4!_u?AuGSMg~FjJk&yB`=V|$&T5L> z?I0=7zADq0ojD<!wNj|Z%s<RF(UTb7eOI6BO`1s<$1^b*>`g)DG-aR)1CE<G8U1S1 z&es;$lmb*^E7Tf2qYh$i>SR*cQPUv*0QIWbjy<e*mIsUue;UQhm=YEx2T*aJ>MHDV zT*RupNWBgQ5B04hCeZgM^HK3-a)DT%-Mb9-{3`T*bQWP<%Chm5{Oca+jERsNbYw8d z=m%=QXLk$cIh?d|5LL0*4r$$mkzr2ed18J3VEy7_t~-xP=6rLl4I{!gH;S*eG7J9z zd1C;U!0*7uD|SSQBw1~Db-~-u9^=xn?=55T3(cqBwk>?O^Kr=uw1RQRW9wQ&a#zHD zCVW7KdrfA0Ti8}B>$uYTNOP7&yaHqm)8;6_zZ&vwY0bU1P#iV_^V5T#E7ZOt_>KHs z<NbTYVFXDn$btd~A`4ifw~QZLEPmZ<#`Q_e(=;IEBXP8T2D_s4G^%sf(X$=$<dT0X zrK--pV#b-m^&_yYxyXtT-*<2%`wHZzVpxVb-RZ_fH!CL0@X+R|Zih+XYXNh3UEAf_ zNcZGdwrN(O7j83ypTfNR!Is7}5|tR+v=Ti?{{Ra14L56GkiBume1>IS?Vq9Km{`Sg zBwB(OIoLV$J%wM646g&9T>k({b|WRY;0~U(JV^<_Rl)YJl&dStL+R+jC#kn!r{xXW zyBla!jpSsC<*eZ#4@&FpNC*XS)~O@ar%vwYL1hXnA;3BHtCto|b1DT2PiltJKnisH zYirAG*g^81O?mE4=b;GM9ORa9J{$r_;}xfCX&UVTGoI(QTaw%><ZwMI^`sfVIX^aP zqc^dt)#__OY&NSQ3)q4?es!%Kuq%bh9l$jurMYCs81%(#q+3@C0{YT(zUI8B$7@J- zlH+I|oobufsRv>y7~qjw*D@k20&6AbVi#*=eKFFA*%j>K%-L+_a6=q%ip_m33i5y- zTI!M$ae@iJ=B<5|Ps;s2ty__d;*&TFog3v+i<4Qu*@YhkGHcXY$S@e-4mqsroiPSZ zazM`%(Tt&Xd8^t8n52t<PhaU-*IIk4ZiH@D>OHHwxVK4TD8nQg%{KBXU~!M-RiTXO zN$hf~3$lpM%eS_1$EA4>#E%J!yH~$RmnqjA{J1>Vy<OUpyVMH6zk(Z!XSJMVWCc$Z z>g9OXMxwJkT*oa{VJ>8Tcih0}?x-*n_U<Y;{{VJDCjgF^uJ^|JDAIMw;D~KTB(6p) zhlm+KJr8l|U#DTK`&g+TkmeXqWj2jFGb*~cKQE<vZizC?qA3qk$JV^w*;Fu%hD~)E zT+&A{KzSUGYVSz$b?SOu;5QaubP@in<lyc8EF5rl=LZ%0EBiTkzJC~eD$)~Dkja0h z#r7XNIWt2ZJhD%A3AdjA0EK=&SR^;vbWq?HkxLMNyu5?{di1Z-zu3C?UVJgsG`lh+ zz0s}|>?Z}Hlx|QDs)9j0_N=NT2iedgNS8Lq&9@nhu;(MM8LQDOk8K+&k+Dj**0SNb zWpgAyG%_eFykV5_xPEnRd2V5inkEVyf>c%r3lkY*Na8L0Kz?;K%yK}+6g!njIq96! zT_kv(Sw2|f+H=>Ml|0F#-}jk#D~`G8ObYI)G@l~{*fai3IRi(q?j$l1l6M}WiAKPw zC&(NX4Y&BuKz|I<$-Ql(1cD<wcpY=zf?F&I#LlaZRR|~7kxFfd8!?3+tu<CgT*h!& zdg7(sDn<jGkUu&Fc2kx{SlTHRI-Wj)nuUe4gv%rZD(CyGdFR;E{K2^AoQ!=lO|;8f zBqwVU>==55Is7OKvF1?}hjD1sa1Jw?dw%qp;RU{2a(eWk_D-1zw`pvSxX)^8ZXY5> z2qWd^7(Te5S0yrec7|xw?;H_LR9RX(n`hZ8F4yhvkJ6(z4zV)4VPsSMJm#f^p^+ny zsa9Zk?V13JNbZ@HgqSUnkJ7E1gn~AS$l+B>4yP4b-dP0u7!0mPJw-NhzFbd|k~5s) zaxgn*G!EmVjX&2!O9=-<*wj)+GrB|1w@S4o+%lY@I5^sI!2ol`K+dkY-P9i4C=0S? z%!mr8!5c{awLuE1G)Iu58UFxlBB$EXrw4Iv20I#AEu(9E#`tgI1b3h=!e%KV2?^%_ zepM_Gq>_0~I6zzL$e@**W1b02V;SUPqhy_C4GMXvtH|_f1#)(`F%s|-v$(}7s+%I2 z-L&ol-kEQ9?2fYl2|WVxDh7SSFp>svK@<Tgc+@Ck&M7>|eSb=UlrhN$o8_;epb!7k z`|++^p+ft;zxwq|{_L*KcXCBYfvv+dY@r+b)c|LgAmAM2FdwCOyv+2uQ7h!1mPSud zc^LgD!bYdd+Rfj0r?o)xJY!_TIOlgttj@~W`AK2&oaT;Z9H`J~ti%OmQa+tewNstZ zM(bBGlpm!~X{RK-qm}mgNU5N@jFWK7fJhhzpZ@?=Chjgsml8ylHOD0V?3`!xr55n2 zw3`!-+2)~n<CV8Yz@JY_Y*P78``i^YhscNd$Z!=89*zG1)~S5h)P<LD2l1-UB*lq_ zFwynnf-1zGWbp=#a>K5DC=OIP;f23>X>!VYu6t9w%ecUHe5&$&PqjhzcPsoTNau=` ze82)Q1$rD%E=ZHdusELHO{KXic^^u6Xg4y*S$Fp8Gx}gv6B$rUB!Z_n1GQd`>SKTg zSdMoNC?Zt#&@$1VDj*$CJB3*LJjlLo&HkVs)lr4S=<YV7FJ<5nN|G641T!lj`G#mJ zG}W~%x7vKo>YGu%Vh>S)P{R9UJ)=2k19}cPs}bBk5Xks*^C>=+1;mm0(E{9$myR+| zI6svG_Hi#hZOD=(M}_|GNyrr|$O3tm`vYSq1m_i$7H!Rja5K}W{3}{BB$mw{(dpBg zsLt9F;n=-xA(kN<xDMd$J#c=N6k2@oMk0@C$vNkt%~XssZ5v2%II9!L(SlHrt&D<k z&(^k6k5f5C-*RMFnqsPaxfg;DL-_Gn-YnLj@ZW*FNw3b{NVC)K2z-6l+Hv{xt!0+p z;Gmoxye{F!4t;y$t$2^e>$7q2kHuC~xe>*uT}gGfjJ$Xdoc{o~o}bROgH7lpz2otg zdy#LZ`Q`UC?YI(tW-Li#j@*%4jnF1ZT!Y34_pLovK+cgmGU6!99)ofMxgT6-6_95Q z8kS;rbsm_m=<HomhN~jXiEO{Zyw)|nvb^zw{^<HvyyY<@#Bxsq+N0B#b>w7@3F5M^ zD?yinyQq>qYeKeI?NTgl@6x>*!W9<ka0Wp+`d1~UMg#znk}IgSlQE=q@VMDsv-KRZ zpCvWi+E{Q9ft;Rd^l`Lq4h9LuGTAT}+nUj1#xd9QuL@dR_Gv9mJ0(}=p&Wf{X>%Sc zD$+8Fp!D^vmrS4oKU`M@SFMjuleWg~u3Ki>>}_}O8<CE+!)fy20PYS2cUIDEg?A2d zT+>MOA+F`gB1RoKr^19XfVs)_#X-Ap-VO&!(6;%p)aQ<rk~C>2p|xkZ0XWCwS|yo~ zsRyU6QG@{CbAYv`yD$Mb_r(coV7o9)tj0s0R<vyN@q@W{XYj2qG3!<%!iv|G51Ew> zu!%6Kfu5PBENb1bYWZ+@1FbwNuLNM#D5E($vWY@5gN#)hfZK>9*4u2xefm`kji9k3 zj8k!qg=us-8Lh(0o}kub=(%n&{cCb?TRTlxXycJaSCQ7UO69BdImu#1U7&UCSvMA& zNdA@AT|$73gB*3N$>PoqYU24?k&RV!(ei)B&ksrEe`>;vNMG0MUotUhc223Z5tkY1 zU!cDbG<MZ=8;C$8svW;C8T#hF2)?$O+E=xkkdJ~oV-@;W4`O|!6mEV)#W-0)Zb;d& zNUkFnk({U9HMJ5gz1o=0Q1|r)u(W9-Sr*v+$v$6TYov(+#_&o0vNQ6ZRj;T=#M@gl z(ljl<V@-!G69eh|K!Ny>1%844!_$42R`Er;mc&r3#{)S_jjg*M??r*>n*8k0C0{ON z?&-LD6v4;oUwi(>w=BLbzO=WLhPbqqW$nHwc9kFS79@VPGeIS0ev&M9*AdSi3e4C% z^&JoOrH&Eyq8~dEM}JHTVVIkR!z(rs*z`EtjlYFX?}aj9iBq3r_}4Kc?=B*1eC1{Q zc&1!T=uuVtyZ~5^ha6R6=oq?WZXAQpzv0raLl~M=g~0ONGD+m~Ko;BvfT>_Z4!P^^ z?Ny^5WD65XxM1xbgjIO1Ofdx8l|n~O-73wnk~JzoIX@}KuRgRBBB7FanLusvslmqt zAJ(iio^jm~!2_Z1OAZoOD}uQBzO=}59mJcH0R7*5_n?s{1nn!%2h2xeJ5s}Tk;fd7 z#sSL?M?Wzi;Z^_<mS!q}y?tt|fsAb%j1=TB92Gt20@SwbZi~h?tKgn;GxVrsjkmD} z<_mxU>&UAh;RC73E1$}$+?J55L-&B^iU2J@4$C3HQO<pSwPfs$(9Mv=K`Nm4$R3qc zG^NSO0mcs*!S$&m466ub-k2(IaD94WfFCk!%Ptt;bKjb-%*hiff^pL)IH?e?l&E+u zk(#W{CP4=qm}lrH0_0{B6i>Hq(Vf1Xh|j)r(uWYEv@MPZ!Ol9<=aEk_r_0}-n5bh| zX97q0vJQQZY5<*LK|aXX91ag^Lo7~8rzKC!F`S$ltqR9)Iu&EdBPWs1JwFPJx7lZV zsJ?9Ha&|oR%>X*xT*#=8wXn=H?NUfCZQP>(LjdIG9cdiL_O!WDapjSWcWhMm1?NIk zu1W>x+r0oL&n=X@78`-U`Wl8+1iK96w_cSun!y2O+EimBy)tATLdYL+XBi%x8URFv zLX7S|jX%myM*QkjVYGDV>rT(G{OAM!)cd3DSlY9|TxS{2Ps0?G+LESQa@hXhH1j>P z2HMAS4ZQTtRQo#<_Jo;`FAM?eUIp3e5lkQbOP%Y%D%A9}w}>*dV1d)y^{4qwB7x=y z<2mn2v8=4%$`d2<qU;Bhi{)4sXzmHkMAoo?mlDQ8xFD5op4i1iVZDvSd(@_K8@UxI zifCK+vSudZ81?*Uy8*2MQRZp*59q@?m-=&xs<$Pat>P>)5snV;=qm7qgXNg0$IFjO zmF0v+Re}@Kb1yVqfed@1Pn3oqp%jSj&6vQ54&8I@+N;O*g!y7&A|85U6=G()RR&ZB zKUC@s7hpfkK4!^=(s}DvG8csV-HvnmRaCixQVDJ4M{$GsQa#2F^+S%>9q79PQq3!m zIF;MfuYYQmAsR7sEJ5r|M6J3%F@fElLG-2bGBX<>{{Up3XuBQD%mc5R8xYI<H5<tr z?QHpS4snhN_3cqC<PFNDvDX8ES2sg@9zfc_cNATU;Lu3>WQ~__<mROY2}z1*$4|UH zeQL6#US8SG+;?8Tl{qqi?D?ZT&jgAtz@5I;9wV0wn8L8*ij+IxA~Da`6#==nY(igZ zo`t(pOe#t8XAV2@+w-O7JC@|Q2%cj<Dt6<(YTC~o!k|zHVc)ZQ)Ht=8XjVu`Y;DNt zigZy&zGyMD1HjL3uWGfhiC}f|T0~4bADSX~WzGu&kLh1Ie%3m(X#W5MHHEi8vuih2 zus%m75g`8n3|G-=f<j`D9nmQYLBT3|{c&F<e%QBd2f};yoyOGFG|LE8jvXX(<~PgE zKwo;^jGgRDioMUs-BL!8(T4&|{#noI(yEx(%PctT1ugWS=}d>Bfw$YPe>`TSgl<q+ ze4@Qr>}6HzV_hi61E|5N^o3`;iE)9RwNmMu&VIwNt$h+Ud4UH64D<CB$%~RvXKq<Z zI?0~Rrl#9vUZnT*t(ztN=Pm7wRV^uZzz6`A$9|R6*jVmu<YNZD2NPG`XX*G<^i0^c z-a@X<ymkKo>sHKmvSeg_l`Vz3oMiGVsD{)wIvm%DUZYOP_NY^fDB8kT2WCxl2GF!z z;Yl6p^}d>pxO0Jy)zHJF+Q|r12P3l_*DRZQv(TjlbBwaLV=}Ho0sYgCYrD6BU3W3r z#yRGwu8y|Ti6X-0wjhS#8wlf`xfP7wr(W9=$8Q=Z45!#u<&~LWxbA%_lG?~wSp3}c zk4ow+)TvSi6b`=hlhCW*a@yM<#BR7?Gn$mxD{=)}VvUDmgN$bs)S0(n9E^3TPf}YN zx^QJ1M(%oat7%}X!CY1AR=@{6c&$kSU~&y~#a#;Q=HA*U@^CZTHC^CuFL1_l`c~BH zxyixDYOQmHj1?ev#wx{goM_x0Pkxx8p=Jjd&O6o5^^P!h^Vb=x=GGuMY?I!iZ$jYp zvBb}7CL5>VKl=4lS~!T1GJciYwU8iziihmVJ6Akp>{RqP8;f$M3~`S5>sgY;^KA7U zab5N0nmP=QN9$bO-JF5c?LUQG+g4^?XA!8x#==wpv1M+jjt8bI;7=Rr$!V?JxcM-^ zwt9TouhDy$b8hi~cLib7)K|ja6KwWsGm=PWkx-0d)K}Q>wjsu+Dn1j7@afaKN0vd0 zX<?9%$o&ZHYpb2GR_GiPgdq3NIsGe|vyHCeYZ88Kxl(_G=cYUKtt)G`Wovtnm6wmi z3|HuBv+|mew6#40MsQ=?2>Dr09oHjw>6-h;_Ar(hz9HO+$Xsd13ykwLp1%7@{A=^y zMVaIBRha((5IOfB;a_e3$(o<pJ~3&wQZ^#gBDjgYONK)q-~$o*)0W0k-2Go`TXhmK z0g3(M5#P2y4_cBK<l7Ws!4iT<$K5`id8lD_Sk^aUS~A#X<K@W%k6KHh@CdF`B?|M7 zypdeQkBGEWi+KPfqYV4ik&_~UkV0)ZJqrH-jW%UPxFrK}q1CZgBgwnU{GpJan~r#( zmdKLd?GTkN7jfv^^!%y8cCzkLtOo=hIPKr9H6>_~;%G~iC*}?M;~&<e-7nf&F%knL zWz_e=;F<}!D4ms<Ds=n643d7di9O5UG=FpvlU6?5=*sfOHcAw$FI;p#twi!k`{hu0 zAgMl-0VHr6q-B{mA4-XTc#~|(Dh;Q1Jt|}7q+7RbW&R;dNF->!*%XNC!Gp(~lf?i( zlvt#U2LW@}sHtT6ipw7)^~V?<l~OIiBHQw^u6Z2%-LdUY0ikn`^=xwg0AEo+2_g;^ zOJ&$+9RRD-N){r8ayuUVD!1B2oOwUIf=)pLl{GZZ(nJJfkU(#t6ai6VA~kWyP;feA znsZ2t3VfMy)MwV8<)dvsH_OQYW}FcpBn_k9fFhGB6_<?iJK)ppo+#va9y}R&EuJ&< z#aEFgS5<CTa3h>#RE`?f{wGl0TQ5$$&;(>~z+r$rbBufRs|^lgWRv8OVEXY$jT7DZ zbCR*Pcs{*FM&|-FyFYlGzip$X000RN`MP8PRN|b9cQA->2UE^!Rkeh;ju`<B_j&zl zx|0Uzr4741g#bEFHX^ca=yFF;DOwTb7lt#sL%XhbAIpk-P`m#Cqq{1H<Ze9uDt;hj zif7snOnu+ZfGEQ)piD@v92_vmLsNOyK9BiRC1A1;Hn7hGHh-lvf3W=M1OL_gf=I2~ zE*l_@+;jA%8;O+!3>2^MoxJ@GHzCRBm^jH8$of>W$jGX29Y=9rK6->G<9^k`I2`Au zKD7{*6=z8bgX`G-H6)S&i0%|@ak%r2KC~;tDDvcqOkjMcsl@;^NQi`70$7jTmFY@W zCfII#z_{e&iji{+f;kj<c*^tIqaa8(LnDPb+`T{_U;ecKI!(DATbCUN(xQ?`NA8`l z$J3f=f-=7;c0C8pfImulDT+3Q(GalV-1qe9KmttOND*><GyeeAr^MHuYOA9no_gdO zVUL_ds4>pKcWxu!ika4Xz#1*k`V+=+*NOmqvPS1`lu+R3Xy=N8D~Zu|$Gaf(Beg_i zxwV+AjlXvQj@5nUVt}In*ywxE0y9SHC6hbz_j65Dj%F=@tOwpApBfVTK;GTkpr;kM zZL2B-KzA1H$7%qU+F=e*HsE+_o&L@+(KzJtGgS-{1|~hFdhyhXY@S}y7BH{%pbOGk zM$D3_`SX#_6#{v4G@etHln=VxDL#UsRdm?X24y@Qy+u5qv_H(Hgl8+p(?AmsD1hw= zXHFX<f%NtI)bURSIEg?#bHNnVy1CBVn1X+I4l+l$#X4hXK$bVzFz7p<!hkH#=D_jD zfCX*=Q`)XB>3-=L$INlbA6lv8%J3FOX9z<a@ssUS+p8?BG4d48<Hb9HIgWA?SY^Y< zh2VYV$R$tL7_Wf8Z(Ukdu<*`_7zy^<sc!BOIy|riRB}F?SKmQiAeRZ2O~W|t^A5H7 zx%+cm!Evr=&?zl(Vu`o|&`Z4M`EgxXb$6)V*FQNmIb7a3D$F<-9f%mK7WXOSm1DP# z{c4V<CNAVLh5=gy?#4}2V}Kjhy*H-EDtDHo_bs`>QQo@W4qCh1D`P5G82m+X{$@Z( z##+5k!xmDi#KCtl?b5uQ%BL(@?_|_lt#z^LT67B>18QvtjE-xwvW$ccj0|GAy*>ia zBv{Tl4V-7!@~*A`BS4uXj%(oSH000GD61Nf00>^0J!^i}La&4%o}AVcKn<<C9M<)e zPGubMdXvs8%BvRMhpR#PomHuj_Xm#E)ma&W07%EZXIN?OR1yaxf;i1|@LQO6gYx9# zJ!_5go|QQyu0-)%PTwM@1N`=>p|-nHH$)CSE2g)JSgPPK>z<XPCA6Rtye>Nmi76dX zoLbbyv1#0t@`7X6(z-a8_hLPu^PUYyZ{=+O51XMTv{n^RFgf<)im0tMD^7(uE!_6v zqg#0*#(5sKu)yQx9MdK(6hnB<d(*NeLt`OOZmS78h9BcuDJXDMoKi;4WjXtxaa5$! zu_mR<?mLIlvhHOLxCf9rRo8IQ@Hwgp0g_G>*5P!wBDR5@aNf15uoO@T&%SF$I|WE` z7n+VpjAI1*)^d!Pomi)#OJH1=&O3A7qjdutMnUGMkwd04Rb<|{#~=(>JfhRE{i<YB zgk;7!&1KIH;gi&HTFw&#o_d}t2~;4@P-sn{x_cZ|<&g^Ia!LIw;?Iq`Q(E86a1?;d zj4{qg=DzloIU^lKeADr#K{Le^`*<Ma`{38TkXQD2i?io=iwx?}x;`gBAf0E8NDLPz zrU=hJtwnSX+8SP)h;i>)PZKM>yB*$MRQ`2B=#ra2%P7G2uhr?vNhXiR@pSoQ@436A z&zp^>xdnOrKdpU3@K?mmp?p#BwzC1)f3+6UQP{^g+T8ZWGoNbtJ4yygVN=uYx7Q&5 z07~_J3sIWJ`%Jxr;M!lL$G6G><37FfTO_tT`5(}Rr(jm$<Y`@%jza^1liwcT3Zyr< zg?z?t3NSr^r@^Slspy(My<&G;-&kAB$WP3tU=8);b@!<^BW75K-XMYcV!2-AjAB_5 zIiDgEw|A&M)g;h_3v#(3NMJ`ibpEu^n<ZyZGCGFjb5S>tlDJ7sS;5%Da-K&crYRF6 zLh|B2I^CohS+bd4KRSZ^`6+K8Po6SUBdJCQy(Pp_M;dKkGA3=@9zzm2`eQY6?&=q} z4GIzv;feKLF;tNt#YA!B;R21L(03n=H34>;o}^>gR96x#fk;wN0n(z#h0KID2>F-$ zr?=LfkQz<?tc-8ZQSDXWP&~mF5nCK><Z?L9d(@CzD+b1JoG``+C!nP%Euxv(L02G; zn?3XCMUWBNymCa#C?SN7GI=8hieznvK7*&dEMInK$N{9>3F8^<LmQMCL=A<=V0ilU zqR0h`I2tB}2r9?sEIayQk)WE|VKhZSakaVXKggv=l`|d$b1>`FF#d*>1m^*qOOQ!l z?wTxsm8C+gS8~2N+BhTYO#47GB9$d^$nQw156D5>Ph8+qI=n|EdFT#*`qWtgJntNz zy8x*_bozTy=DAIhU~C*WTCcdQvF`=9AIpjcBUvLnv*h6R6j=b*K<<)m4cOGPJ*&cE znM)}+QR!9VT*sVXgXvD5IMOn?CGoqaG5&K!kPlTzqerz;&W8hz)iigJw5-#x40?mc z+G7bJh8KviE!XQ#mOaQ?L{pK^9Fs+m47mBG^8$i495Lv6boZ#1H8#Y6?8?Ao@}wVX zoH&I*&zbV@t-usS+hGg4X(!i@<x<cMcZwsh$ASRpxTg8g{{V3R0N*ugS$8%|sp6Tu ze|>){K!5+%`+`fjZg(zl=sM#A>q_QyfI}lA2?qnzR1YdDG-X3%;IQruFO~L%RgD7e z=u2lGrF{A75&fMD1e!JAV+WCstvRf`tUa>fx(>Jko|elSZn}|!bQ^gTG?!^3Bq5|- zzM_C2mfG>7V8vQ9%K|g`b5TGew=CA`La@P7d(%v}YEPJikF6`oEQJ|DCOYptWBh0W z%Z!2iJ^uh2h3?`5Y{1F~ZJGYH6myW~JBWEF<uUV`y}Cpgi8ps6ARJHy0P#58WDHN+ zC;C*G)VBn7^CBIA<!WO!tB|2YZ=U?q=9)YKZUJxN&T*Omakh=nd2GS6-`1Jrm0=y} z+J~zs$*Oy9H}3gE^&c@6Y2IlfCgDy%``MrgWNS%+TZU$hcoJi8KTMzJQvUYZLXq<W z40X>nT$lt3pl(t#k?ZuOhT$MhxI0P5JkSNTX;rcuIR5D!1zL29bi&4p+;VF#Xk+HV zh}%DT*nmf;N|}w>A*EI4fK{?H`cMSzEVh%fx@{8UYcFCuel>Dim|85$gq24g^-f5n z%K3zx4Cg#lamQ>)GkoW+8<9X0v9_H8$r`bKH}%2w9M!R8mg=N(RAez7{{TA8Huk)| zx}LpM^!zI0M=5}3bN2%{pbK$LaS|+?ou}r<L!WNdDc(T4iPl1+=8PN?4N$j}k|N!e zATqJVT`}1>E`u_TNM7HK0ACJ?8lhG|*etK_{{Yv;en$S;SB-P=+UmfzB8yD3W<UrE z<2X3?8L!tULo{Jz7<BGWIO;!KSL9#qnR3zma=Hb=+&zVwNO9_ghw~%wuIw63>Nm0Q z_3Nzqgzg41_~SIj1ahaKs}}(MrO7$@jz^)a`N<&TBc9di>8ocn73E}X>CqXbM>yc} zE8nymW`+o*7za5Ym3W_p?M=+Fpi(k<C(^!&)1nc=k%in>0=`c*#ND)yxydlxwr5Fd zgUdzfw>ZJ|>sqQuIO+vw#(cC4KpEn(?)6V7l7KP6;5R*fmGbeX=6yV>D4mhi;I)lj zO#TCR^%Zgt6vYIfk{L2Fz!|R>*8EDgtF+Ffk&S?4oP*cWxQMkW=W!#**%=@LPZiNm zEOOl9tC@03L+%X=#&ODrdhibe&{w5t`i+jD%J9eu$IPsHSHSvbizl@Jt6(Qk8<Sn0 zoADv+<qdHuj6Y4w!S={M(w$72+Z{A>cWYex!a39UHh?`3TIubIyh;cscVBAx4?_6Y zqZUXm-D4QRU^izR{{Tw&PY--c@W!gH_V+m?DGJ{xADwxX^7+SPdYF9DFGkNqdrhpx zNKsY^AfWUZ%|MsdS8%McHcJl2G_2}_<{L_kf=>t9yw^gae(~&6#X?KnC0P%eM_$#X zC|86AUX{;5Au{=y<J+3nH;U(J;A6chQ)ax^Wl3#@!c<`Os6-{y9+={++)PRn9tH=k zWu?PJSIll0(u^+6B-C|gkWm_Z=Z~j)w``ydz;)uVWxWPayMX7nJbLz`PY_r`5E4kJ z!2bYyAJV(1;iV_3!-&PfEg9M6*#u`QdetdoSr;iHvjB3Vu4~Tqe;euc;B1&=Av=~t zJw1sP<ND9V`zt%S4W-Bhc9smMb_hKBAI`m4<;#}#JUY3Wi?z?7Ze0UE@4^*0!5sep zI?9VwiaC6vHufg~5sdz|^R!<O^xqJI_SdNm-VPQ_#$5XYOJm}Pz1<bVmAi?y?cKL< z_CNh<!g+e$#uW3tc~7Q9%E#s-=HuvT8JM9BjyiK*QK5LkJEQh_Bmzi|2x0RN@gDWu z>Dsd0hQU|rRH^C*2E4lXR~a7m7abarHV6!(Je99D_=#jMv^mumGDd}$jMt*GWNuTB zYRc3tmU!e+GrBey_0RduEIhd;nmVyjQ>5b0%w1*=vFh^&7+E>scdl;i5dQ!v_Xqy~ ztzNP5CrAFxxNEV=j_u%Kj@cMJd*}48E4u)l;Yj0*_vaP*##>*SoNj&{oKbMEE9^H+ zk>Qm|&Ua*tb+2Bxw%K!WU@%LVfc4CPV;+Yj3i2ov`JkNO5y<!Ay=H60I-Hk?3G$LT zM{KkyxD5O4uU=N&9tOQn+<$|&_WF;)?+ITdq`T>GD~9BB=b!dSBj2yJdNGaRc<wC+ znG@%7j-ZP1Z`s*Vz6ba+Xvk(kc@&-a5}m8~bNSb}D!SQ~Snx96fWhbR>s)f&2K}MM z)RL6}KvT8HBoKNJZ(4}DU0k7H9U3;;z+eW*<Livn_R5zr!F4v@It1DuuH@~J@BJy! zMq?Yu+BX&$`?>YUZ_bf2C3Xy}A9A}#Y!E^Il@eQ(wIVEh#hHh>sdq(lZXFNKxg-JK zrfMi;WVaDKWhO(G&pdk7t%)5B>mHu6H#>5!qpdv=<BR}7LB77rG2R;vG2hcZn5QhU zne(#?B9Pf!4gozrqN;tLY#By8seVsG=}ZacV2AC|0z{Y&J^qzp=e9Pk7jOdvhUX{x zRXdAuC4#ya3>8@8rcFA0wV>QE$7uS{0><eCq(}?JAj#vUF6Q!_L`x_jGGhfn{&jdP zG8eR)Ac)69jP$B@(Tjs5Q|I?*X>f2z?V13#@Iw+AT(;~DyN*B=RYGoO$KU3X=0{j| zIo%t!5ce4VbcN7t3Uh_W{{UWqC6d-qqFAs9Ip_ZX*HSy&gaT5m2+sqdsrQrnwZjlV z0;?U&?g;=e!k*sr0MO-12RZWr0D+PYdQ?|LiG0`raJb^E#cm!qkqWsSqa){#54C4Q zB+|g41~!tfyRPoP(tsqJ?Gw0mQl&`g#(1dYU{n@jPCq8kO0bJLkOIITztg5T_4lgM zg)>SbE!1P><3C)`1&P+;a|1&CFbbNovn$*SZ2s^LLFvUn%4UULFi^+IIpaRGiMny- zA{M~I<L@8vpb2Go*kQm2BzoYAdbEPwWI#DM$Q=bzjBms&HigOL@;}C>h|bntTxX#? z=bD)S7jw^;fA5d}y&IqYdLR4Xe~o6DU60D-V}tb+o8|ubsRRGl`-B#YELBW+#yXt( z)knRz#xg)B7y|~SmPJ>@9#JQ@NI%Hbaj9&z%7|Qa=YlKe&rpfq%u(g6sUA6CMolfW zzELrl*+UPNoOS$jQA&}s7Ab-|h9D7ERhC5%&x08q`+hV4@`gZ4V?7Qjk_gg0lA&fi z0HZwm)Gnsf;@fLS)3_jgdCf>Bl0mW#3wO`;pa?*bfQol0K->$Byn14y@+~vIRt$R_ zpYf@ZJBcLn8GO<|ymE8(^r^hI3A1AE?z?#u0Ti~<KGukUI)W-eb>&CpZwmd(0YOOi z;gQvceb^t$tD9SS8w?9^o=pH*dx1Dy@>F_oY02fe{pS6rjCQK&Zwei%`#z_Nj!3V` z+Y_!20|zt#q_*HTAaxnwA>^7aE#w?^amiI5)~LUk9>DlAFCQ-){c3;sn1$RJLm&5# z%t@dNBI0!bMz|Qm4Y@y^HK0ydKG5SFo`#qN#B7i<<YdX5Dg8S8RPnqZc}w#j?-7we z4T?vFJfh*k^U2L#7brl=srJQYy~mp7GY&S4ZX9Nu3vJ+F0qH;wdztNIZK?!M?b4=; zP&39#x!jzAj8sV|op4nAyyrjtYIL(F47pOhbI^~{fHX>pb2MZ3j50@VDp<&P$&7hz z{7vswB4(LY<0Bc);Cg-)qqQ~}80Q^FOqu|-6gkAq?Jh_MAO5=hoc+2qnJj)P>7F7& z9WHcM`$+>OTHa6tpZE-z_Q$<`vD-C}Ih9BVKyt@wAt$Ld@VD(n7lAxM4x&ehY<|-w zg@0)jEH@uGnB`Rr4_}b;UAS2!_bb`XQM^yeIrhmU(f|P@6OY7yOx7mTEcgcia((M+ zVfJG1z8HlBIvlG8RXET4Cy#$>%&=YeQ7QibmXZbjztX!gGv$TRmKK|pRC>3Bw3m`q zc=LtF74DW6Y^%55t#SSqurS5BMh^oRuS$Z}MU0-g=jmSwn9lQUAExEBToSpBsl1U} zzQf4&&3UcW#G#~}HyJ;K0nqfXN!8Ggl>s>zC#7-nSP=Wb2P2~%I@c8niYn~hyd34B z=MAN5vd@nv%r18`k&KhuzA2M<ds?+6);1aRIpfm3y4O!KAY%aevC}+$HQZ?W40GXH z&`7?!xgUWQ(z7mCq0F+*eNT~|$5TRc8|3xikJpNft?<X%17L6l@%+E7euQaX4>YTu zWq*{6?U4ahKSTaCyEnnF4^F$?t@3~6r<Rxl^EGnLA=F>k^1b5E&iy(ot80apG3AEN z2*VzKAL(AHplhPqGacJZTPgt&<#_rLUrk+n9J7J36}&fLdii@-ABo3rf5N%R{vO3+ zBPv^n(EQ39smEVx&E~Y<yJF#%)P%JycxT4fvBe#_&nJ<O;{N~-{=IuOmx+YRi1L<9 zbGxyx0hdWOiLql?{R*B<ZRxk0Csmb-Z^z5m7!}E0tvPCsa|st68T7rC<Tp`-Q?T_1 z)Kx3{k0L^aiw{t7TyKT-z38hxI*>mK>+eXoQs@Q;wRsgZXSqyS^+b-_G8`yQyStJr zi@EVsaFY>me(=fm^r>~|Ww_I1iV=svAm`BLyoT&vd7&Lf?30e5el$|v>{zSv*&RNi z;;ZL!wv16bvG46(N#b9KTD67Dma(eLj51V=<MFP%X_uG3EyEOC{{Rgs@P3oK%#m7F z9FyPr*RewpPCK77iNaKyz0VV$T-G&o^WoFrQP7z)lb=qwHH|bngp5ktNKAoHgv(%$ zuNe2meJ`i@OGn&Y6`Mc8cmll-P4In^Aaf9uu-kw*QTPMt*1Z>)!DzNP7F$uR&y(#u zGkI|o4}BKdBO#EL!!P1Gn(8eyomSrAO~f)OZ}o0L!w;`x_*dALo)@-;*fq7PBLT^7 zq<V5Gx1JwGUpOufLa@j^dE<|7O2%1)dkPs-QoBB2w$pFXTK>l2Ug|T3QV1k>9`)-! z7`xt!DT$Dr20_49@Ac2Ec9(iG#8OD3jg$<Wf!FY=c95jEOXi?KoP3;qmFHrr#(JaH z!%_BfvD8B@=t_Wj_2Q#q7-5D82OpJB6bo&*DP{!lO*TcyV!go?<h5pxZPoW$N5=mE z9CX!!+gB>Qr<Zmy_yb=qaW$+<xSS{?{c?Xw`)A{RpD*@3<jA|c>rXH71Rv>N4%(me zaHIG})Ag^?ayrqKG>^&nqKj2mW?r+58+h4U1Th4k&-z!SSfQ5K&632Z$Dueq{+asp zuRO5A%<GJ(`G7vXKdpMlhHVYaj-eWm9^Ta5LP_Jzft8hU{?|<V_phete1&M*R!IGN z{g-9){s-ud5z5=%$rB9c3~&g~9R?4%&3zdZw-7K%RYJDz`O4&jUon2jaa`JbG`P^` z8I3Pq<{o)?n+^yaxhymEucc*FX_Zx(PRyox`IzSwg^U|$b-G-y-W!pLz){f$LHJgi zOu;;qCwmS68ROEbSzMHd?BzdrhvseGzlA*@dr6&<iDGvQp43?rauFwz#hZG9Kt9H= z$bxv;XDnX<mPO+}oO@P#+`M-NSz;`NVTN)u>(Zb0@`xTZ-h+;~BR<$Pz>;;6I2P!) z&V^Lu^-@N7s|O2n<-aOJkUJ1@@7kfBQtlz#l1CsJQaW-E^rl=Yq=*%MRmsRcxS$3U zNLEAp%rFarco^?eNhD1aPCjtNal4@%`c$gfjH3d?_C2Xk#~LJ3MgUIZy#P%kd5R`R z8<cbJ?MWUID}_9?xFa2^vi|BxBY(O^N8?l3N<5vsZBv8mKnk%W=@f6kjYvP^P)igb z0~gE>f2CWM-EIO&7z)a8bBz9#5P(JmXg*x>bH_9Q$YE&<Z6s%qM{1Vd);LkJ`~$#a zoO=Fs6EBw{q;mp22Nfijh>^Ta8FuZ*^`HnNj>+O;!J$y#DU9Q<wKCym06H$_+(rik z1@-#Ui7uw{(m0V6k(JLUABPm_i<c6q4AJfqFZZ_*)9H!;v1K%JTs_3Awly0WN$d3d zs<Dt2at=oF6?2TK&l&ZlXi)O%I$+@Aujf%rn`2gk1jfO(`hX7<0jCUGEYb!AMhW%r znuchE6SUDk?wvthryV-xqc^Q8f-;PIH$m<1OoB2uTcVR0JHBI_dg6d2x$@+-42%mm zanu^F#nn8(*<c0$=~J>okupN3J<n=|NRBXxh9~nL^sEJlZ6Ka8G7N_2Jvpg=v!ed* z>sjh9<N0BbB6b|{OS=C6{(BF|(PRhz(ED(xjZPhl4upZ$l36^|RFxH&@EaWFs{oRa z#orxzcBL@DptNP(yLQEV`RWoOjgHwAl}-)^dYJ?fW#Y*#$IZ}HL*~VTsK62Pcm9;c zdE4e+yWAe5-heHeiGV5?1w0%Ir67s2%9{w_<Eg1yaTxjCP`3+_jMJvVGCK?t`2PSZ z0IV7WWoBsvtIDt#1K9f1#9c`q;UF$K&T)}R=R_EZ73AH=1EnL|NQ2CWDyOp^XaN#C zw;#Wh0r}L;5lj_z^y^S>Vz}TQoYZ*{ef~#tiU6|q%;2_HKIb(K?<sc5t(^Vj9Vwy1 z44?q^B=sJ+r-{r;Sj+}G@t!CEXrop0B9!y|Ey>MDmp73R7D6-A9P{Z_<8--`F}T}~ zNnO9zt=qVVG2}NtFMenOZ$4RJXxQ!q{K(yN?0@>y#kYl#N-IXgs&4D=&{Xmm-1+f> z41Bf+dM0am!pSG!#z5)C03?WFU_fjz!9Je!e9;aY%+DNhD20edD!I=9W};PRaLWE) zI3RQ%=hlEPFgEdvfTf~5F$0w~EEh1y24W12-?0@&NaVJY3g$@P?`%7i{b}*ADpn9v zp8YZOpbd#pK2vBmDL5)QsNpXoG$p)~&IvdK_8*DGSAn)VV4d0N(yf(>ONR^l)scAo z#!ulx+T-Ya@%u*nN^c5j_FfdWong7XxBFe}s1D_~1gf4h%Qzz(a5LW(`OmEDdTzUC z3R)IvBHxB|!!aWtd53YD^>2^9CY!{65p}Dmn7kfvwN=hAkS@>^`f=0On)&{C9E5cY zO7sMfGt)TcxTjy1XWn5lxW^2nmgw;9R@!ShCXPiaNjS;rgI9bxXw&LepyLDPJ^EIM zjx8j%)F8@&T?;l3UiELlc58R7qm1Gt3`hIeuTFLEK>5gFxiJ<;)0!L%u|^Satf2Bc z*HqAy+q@iqvrMytWxN36I|ebw0=h`x5=VwVH}bEC#MN$XAE42L<V@l&EkuRE3>UUV zX04Dy7%4dRuR*!fW|2>p2>^X7i@gtSSzU<7W5sh_+QW41v2M<9vy$b?sOlVmG40oy z(}Tw40f$AjGDxSFm4e0(*A?bEzNu+#5ED6U@>O$_*N*kYz4o`Qqg&Zqc{cq48?r}E zxE{51sVO@$WjRG?dz<`E(e&N#;>~P;TMZSc$OqFP)YgBrH^McMY`m-6g>DJMo<5@$ z^WTm99=dOXMeVG9Y|wyZBd<M42R%J&%`L8-E#vYdM`j3AILbGuexEKgo@>>mmC}~1 z<d$Jlnfepq&x^kYwe5+a>bF)`j8`cY!aEXk$CoEu40QbceLmMwwY-<?mop{pyp6Wr z>~QFDkdmkHGYpZ`an`&i!57~Tel_^_z#1->;Ts(yO$N>4)s`p_M>0neruCfyvxjUk z1SwOFPC4}ti+==sLEzitseC7S362(?TNaSYn`7jX&C3o)9Y^w<R#ylwaJ)8h=PpMQ zeQ!yN%Pg^#V0t$P>N@0m*P7ocx0sXFP6y*&f$+1%k^E8cebmVxld0)ah_7w3FgJb2 z9On$Y_XDPDhVfiYXQ@1FG4qkv)BJ0~sRs=YproTG&10(Y&C8MoHxbS=j^e$mPl|cr zXv+1)cu$04Yn#A`*uWpmSJb*bohXDBD!_nnI`LdnoSQu=YfGGM?wKrcjN=60<aHIs zY8DYh2=bSnK_j91*SOwVCc+&*F<xWheKJIe-AUWDpQZt)7VK+wc}3;Wi0xGw>4D$z ztSilO-ox+iX%Fx(^**14P}HxROtYH)QhcQ&9AuJn)9GGY7MJ1=7+%h?T{XNLcVS1( z>@$&F7-~+e?9VG5K3$RRx?hPUhEW{%(CMoG05M`w&d)=d^$iN~uL%|&CW>9y+qL5$ z?&p9GdspS&v!-0x_=57sP?}aSxe8s_?pzQ?GtV8Xu<&+=@cMflNBe$zy+2d?D$Ar? zTFoR;B!eZ9%9hE%ErJJ1`X!TlnltAr=2L^@N9p*w@tuy+jcuqzdt+-8>7JEh&%{>R zgwGB1M#|-m3@*6o+#KMF`5)nxpT#?0hS2!KM%I@4G~Qa${{VS1!mYz2vEh#x<ECrS zd_D1}Lf5uwekinkCi{3$#ZQ)7rwxvOF<xFH2U3)|TRki+<yBrs(6if-lwt_ZR~(AW zI&5-E9E=hD-YNQ3%re7-m`tF2rATa#!0}s=uJM({dG#dJt#i=!2}?qeg_I1FfIIU{ zk}NMh#c8m}IjE&kh3#28-saxPt&bb{ho&x};p?Q4a#A)U@-_Lai{-r04g$!){XwtR z4-wjnjXvrSF}gAj<6o9~<*~8WwJUaAs<$jYfE;J_ueRZtt684|#q@1yA=Y7*;TUvc zGhUnFwc9g5TLx5-eyUXS_#7X?xP1;Jo@+!Scbh4Tft-3D#=UdFdMawS63e(0i%+_2 z_YJdwUt3oZ$hAIS4S{u9(EUOEm$m5I!unN#DlU@7G(r#h*w4wIZ=Y^@@ZQ}0QEw_Z z4KW#9u1LliP7n3Rt$tzr0r5=QA<-_fL^gsIU|arJN)$=XI|e7}EA;k63NDsiyNh8C zamVnV`{y5A));(JsXH9G)>|wy=CK@h%^dE^e8nf6ah%l4b2YlRn-Ri;)aT_VvGw+< z%PT~}2~e`J7;rlARs^^KW-4M?0R*qH7{-0;wVoRl<ol9B%wujyBlPP_cM`?~icD)N z?JNf)zfZ!3SYt9KFkOPTBmfUNAC))D^3=y|AaS&fuk#<Sc%Vg~k<A=x7%_f_I5`y= zR)_a89$auo&5mj$ieMmTjZQKcXZ59tQ*V-_BNACxsq5=N4oL&d<z!qE6o%sm(0ZDd z`BFP(A&US3;ADfJuS#l0uuAPJ#-Z2}8<GeeMI(t)37NQ7A0g^bKAw~TYIONvl(OKc z<GJI%r9%giY|LaJ?rh?rk=5<dqWPGCoR0L7&YnSeBg?NC^)vx)^5tg4imM{0O!lg= z$~@LpZLPGnHs>G=@y$tuir{^t3bI5<J%eOmRC6&)m_Bv=*d!1>_C`Og08CXpl(|p? zx`To<fM^ni+R_|jfsXXsa-U~*59ScLC>IzY<FDgVFiE0KqXdqC<ca`fk~+xXe4V!r zPAWU5RWZ0Nf<QcVsiX4(o|$J+gU$y`)7C=l;zd%>IYtMO>~TO0QeB*`R1s0T$dQFX zxyt}M@#|MJ9Ku8aQ{^NP$2F2-kjAZ&r`OhiG_Sr`qHM33##9XcHABm1nO&Tdj*4;m zRl@@WStTKkLWUjl{VKGG>E=f019Z+i^#ebJ09b(?OIykrcb&kKQ(W4{kOHAYp5)Xj z9!cN~K2$(XI^@*1Wz4YzKQ0L%&;)U!`Dje;2^@gn5B&z6v47{hpZD;8@ic)9V`=BV zQT`Oahw4AVfIt7y`+VfHDlrO9Nl}dd0QIULCND5~Qpg)Q!QdLY=$I_8gV+I#3aKJa ziW@tDJ$)<Z&roFWNJ&P!F}F}No<B-?l&oP3d6*a{9cl=rl0sHU(ftlNr1_(Az+<*M zPy#R6#(vQlkP(0wasGYjkz3o_vGSFpXXj}7qXX(sTDOU1!2vno2kB8<H1HJ?Dgo-e z56hYWKpnHWlLN0fG~|0i3;BVu#sTN~)H|a`%Bsz{bH`8Cm?E~`Op&6o{_h5W87WNV zazN}3KVE6lv%(d*A;vJ=fl`demPpA_>OcDQ(*?cB-r`?e1H}L?UUpV>P@rQVWAvsp zNSSdMAb%Em(n!dPV;NQLo_VJ3+7(=lnDy^K4tt0uK@$eZ<X~}{kpAOo^6`;?M>NAS zxe>-e1F#>~tobOwF%9%5B7g+85`5d<H9WA+0jQX~Ok<G<KfrOEim@b!iM2rdX^%X& z2&G77Pftn!LM4l9%Ee;`KX<?9R!WJ#c>@@O40Fh8qPJ5rm}hP?o<?eC4vC1{%^+o0 zrazqkT#4s}JeDJ_a4HFIqGAYE%A667Gf_nHx_OK+<Eiwk;^~N02&56t2RZei2O*R) z%Lv{*JBrh^x=62Mksv4+JD0D@22VdxSizndcc~oq{{ZV&%o52gkjSwkqmZx82+16N zBZ{Q1wH-BjsXdR!j}l2Pp?=qMDf1fY>B9~;2bGA=&~aD`Zu3mBI`Ti2cD^^$r_lUC zd8ImSlG9PVho&-~XUp8<j(-}%wJjy#4aVGz{uSWjzBBdgE502sBg1|wN9Ahvix>d; zStHv9r}$TEA=EBz0p>M820dyoicVj|5CA$`kJh?RgRnf3-pG0eLHSpv-}^n!kmXMn z?tL2yvcVUx8UFx1tERMF_+iH!*EJM`w;Tb|x+@tQI6Q(Y<8ckC=zWzprH!-<*b(-s zclsrj;_tak$a`ao(S<NaVf|}z82sb_4a6Gcby|9zaH8d@<C^D$wAeP>d0R3z<xUPy zK8Mn_w7mw}OK_3zU8{`b5Pv~k?xUoo#6~IC<vq`8<fZWbucbfQZJf2aAcfECL&c=@ zMv}txu#e$Xy@E?Aty<>VRVgX~%PI8quQikQbkW*YlFL%Nw2wJ9<{<b|Mn(${&b`A* z@eTF50`Rc_;GA+tx4m}JYsPY?$#haWwgYGK>03uJn%K|#J`#44GCmaOT35m=X;;G* z>v4HHKbd!Q0hP<+8@TABfB_wAuh#Yd03LXOf+ntX$e3*(WsC+@BMpK{&f$^tH3_`? zLdF}CqZ!-QuDnu97{cHVMmVk3Wl~YIH>Z_QhN?RJ&JW?g!`QqF<NM8bM=$nE?KAAt z#^5R!Nfd`wA9R)L$;lnMS0V8Q1;)1R3Xhm$k5%VCjeB(0vRYbf+<e2vPh1-D3->eK z`OnG2AIy<lHR`!LoobDqhM$A1-aDAt@`R`->DT&K)Or=t#F3XAG6E0QzIpJ{Zk9Pi z05<I)k3ov|tuE<&lvX>!uF>s^<C|w~I$aLZ{(mt@QUa0J70hauk=|NJn8_$bO)T-c z6$2UWD7MBhLFjXyGm(mzPF&VBohviuPY~!=^V!<1ykJ8V(=7aFY0t_??_B1Ir`~B( zH1G4fk|YFe9^Sa(zLLDr%E>HosA%Q`aL@4%;;%f^G?uoV#KiAxe8F+R_OC{jR|lcV zUklEUoV;i7zCC_pt7(@JL3bwL+s*(i2<!g<)~Y@j_zkJecM$On>?vravmK<HjHtj2 z7USQydirTBt>yU)3Sw`(8=ThVoOYHW5-VIPjA0iin(tYyNGn|Ozp*0a%_2)Z3Turr z*F=d{%SpDkSR=QMuu8|kC)Awvt5bL@Kx;E*ZDJ!B&cILNPc^R&qZ_Wl69NYrUO@D% z7(|k;cCa6nbJoT>ubNEjjuk^haN2KzthCF9mrjyvRALE=Haj@_lU|Wy6p=--oVp%_ z_02;Kzs<B{XV;p&Zjvc6lk$(}TvD~}j+n)6$zCN`w+gE|Za|}q_pK)b$|KX7t0vN` zk4lqPm5z5+j5<oV$Y4K8{MPu5r;BeG>oKB@zHfd${{Z+`>7C~&<J+&&zdn8@+8_8$ zbweWjuJN$^L9eIbx}UvipAW@Uqd2RhoA`aA$$h44m(ioiCYSbU;|bRUkjFiM88!6B zgM2S0nWkz{&jUtQC_=M!K58Aqp!HMu*AMV2({k&U?4U<AxQ&4B6oLBgudg*bXp$F< zsKkpKdUA@R`U>^)3}g0=Et%vQZS5wH74QXwwhL>dP7d>AVkkP6j5ZJa2oJ;FzfyFG z<JG)3q3bc1f3(=ay_jPzh^XhkIb7H0C8Qr`@q{3bL~7lBt^Bep^gH2X>nFqA2uqB; zs}YgLJl6pJKz?=6l(i|{92|K@Fr^D4r-9r`yo2QNw>@*;+ZApgZY`Nvm2gf^>rh-F zwwdk~6t3X89)xxMDMiGQM#>d-zaxtJNgtO>dlHM5XycYg19ADu9C5&_7Yi9>VhnAL zeLbqQO>Hbq35m>+{LH&}Dgpet#ZHMN@QK^0k2pQOIQFR#Nu`$FK{I9JZg4t*)1_#w zrM#$;scg1z!{s0zao(#)UO1w;#(cB1kUE3yQ(hp0_qOtKa7UrY^zA?taXf5P10p%c z?>=T8;MTM-0Cqc)0LFcNt2~R(HW?&4l|d{Q(yXP#Z?&LDC4mQu0CnOl5s?#?{vq6b zKU#W3S{0Hqrz@9UrkMmlsz!HU2HXhGIUe-MEYjV#;lXpX9OD`M=mBPwJR|^D1C8s* zrMpG@9B#R0P!2)qO`11|h~<}XE8mk*DG3wXd71ME$}!%6E!td1xuA6_&IdlD*QHl= zM3CSf4l+2v^{0m!0zt9Bz&&sP%_X#nEUb>%DuIu<Gyv<8L?o#QFga86U=V3`#~fl~ zxJfokZaEE}{{V$4aXdj}2%*l^Jx|c|rcJ!E#1uBcjD_v_&;?C`OrdNJqw7iz+cr(# zy@AHkPC?FleJXNgKPPPe0CS(`NU=$lkx*s&?j12e7JU1Evk2r;(V)RqJg#{>_vWe@ z<dQWE8Zw~mX6Jul^dM6%W07{0QMkAyX9Iv~6`D6l?o)4_)1C*={{Tt=b6zE+%RI>V z5rsGz86E2E>X!iulE4kXk9vRxwn<~jbJj3M;LYfL1~biDhIua|6WbtGQ?^LQ05%N( zH_BxBK*$~QM#lc<>r<u7L3YZ7u;A0a)o=aA{OAM!())$pDOIF*3Zt$EAk}#!nLrlf z&Q=_D{{R}jENvMNYIiE|$jg<_uUe<&ushL&WSro3HS_1FJP(vg(U`#DzHa{jl_By- zFB|S1`_)<Hkz_eml#zzVPp&B=R%8B-FbL&O2A#A8xcM%lRQXqTJ$?R_5Rz3J?h`w{ z=sgIhEHa@ZARkO-tjLJg+wP}Q>-Sd}KhBX-R%I?#P8VrB@!FvZvVX5aM`p)*wyg5) zL<C@DoN<bU%NabeA=~^z8UC~qHxfb1kQDS#**Nv4wZ!)4W2*&YkVj9>mkYR!qmG&D z{OUm-L5w*?Je~ZE`%na%LE6sfRP_Yl(qh>%A|!L3Mn~sVHmQ?pfgv8){Ay>4=q~iZ zLE3;UNEy>EKvC0g2Q>;z%0yf8Pdu81+0MYE4Dvx0bgWk(MptBmp&jS~BOoO0+Hu2k z&`_nK#vNBZDG0TXK3ooyCtcrljVet9pkfWg{qVzq^q>dI;~{u`-8S>hG_oNM0R?*C z^rx#>#QS4Y6Q8<!XVWxF&M;L2<l(r;KaBuNkhI8N3HF1@1ZO|3OXM;Ib$l@D2XR#u zqj$<;40ToH{d-lfvf0Xo-s(e+M;#B-JkSNRC8x{3i9VS%G&c}ObAqm$&N4c4#(x9P z;Z;1TmJGUyJF72JJ?g^TO}PTF*|hFPUV3`x@TYq#jXOy+etdq?nm6`;#LZfG2H0=y z?DYr^bJcvgzwzK0{c-MmCe6T<NuH$f>w%2^wf5)jS>g4MNzgn+09JIln@gLHL~<{f zr2A!hSHu?IB7-0wDjm500A!GJ>6-CzaMtMkKPSX{OgZD95P|;ygn}m9xCr*wTkxJg zF4pn-6E5Gbd906!P=lzU7zAKsbK5`GwtgD!h6&<+3rZXGHP<OWtJv`6PD>Nad&FUO zsa`m(+Xl;w8s<`O-2lJB0Y5`l^y|q?`&?}VaD90d^0le^*JssImF$m8wsde89D|DM zG^k@?u<%LiTrGh{+H+m5nJNRZ&OIy5tk*S>?_uQ>taS%V5*!r{q~izDn)+NXm>whY zkC&W(O4PQHI;ahUj1K(Px%mKR`^qzsQ5_N3?9NPS_D+FTB|A7d?0tGrZDCd)yBrK| z&Umh(dvzd6jgsS@JN+v^X4VSe5uR`=uEO0#v$ceU5-1(Vy-%pz+uJV4{`N8j3`isa zz#L<>bN8W{o>s})ry~^AM|%&MoaVEs$>x=F!eNJ@{A<pxh9pM^sQ#q?09xpETZtrz zQCIhPB(rdGE0DO9Goq_!Z7REk&qwfcs?NT47?aSBF<#H7M!Q%b0B+;)uMzN@25`<W zJ%1|p%{oL>-Ma#@npQffNxjV%k3V?h2e*2?97+PP=RHBJ=a>+7o;_+<E`wntJa+ol z5crPC!s_O;!5jU>8O3tC=93-E0PEYR^{$;JWK;Cwvu^GLg+_8~slKNSqpqi$og3_= zu~uaUx?3>5SSo=?896xqmAKJ{TrdPRM%GYZ0x9!%A@gjBU`X9m;X(H1uapV_UW2Vs z3?l&%dUvf$epGH5$;DdL88vMPj6y~P3;;8lREREpNv)#_fHr~n)oB->yNXQ+&uGZx z8qA!y7(FxAw&ln|M{35oV90a9sFs$nN=YLoeo{h5MK$o>#mL&<#di|0%1o()j)Zaj zYwjD#kVbRzf`lJ_{{YgyRrr;o$#(bGZazd*+Wwn}uc68%8dK&zOFpOVY@%8CJ#QYF zWq$9((l}qLFwgS>zL3?WS*&fRT!P3r?sI`&6X887ODlL{XKcz9@PFAK{d)I1?V=kt zZ@ml#Pi_WrT{!8fRJuLfrkAs%k~uepkBY7*jP3~%AU=hcU>{tyev@eWZ}vZj^t}}k zNwcwnJcGC9R2%r`_=&HCz7pt7;y)N$7eOR5Tz#fhZiXvz8?SB0k8Zxz`V99`{iMue z-c)4cpe@cb_}9?nw5+a=&-niU8yb}%Nf1K>Qo@bqZW~B-_otgxm^4gw`9V%Qf_hbW z-_DiHahRD<u>+ipfzyv#bdrac?F%3>It5|(a#x|Rpm*wiYdy(rNrq`l7CVrR4mjyd zQzpU{m;k$QM_ha6ty~52k-q82By-QDEHbLVe9yW&nnacN5eZ4SLNV9Br8yPlQEeg* znQ#a^fl|&PHjQk+ox{vRJ7fO<*FvM*A~P|M<<3v=<B!KQ0G7(@0NgfXaL(+U0BXgn ztJ(QVRi+~;gX>j{Q<!|kDJq~R9rM%r(pZQi#DXxapbW2F-oDfUA-0wx;nU~HB!f*@ z{?Q8=bA;p_*dUL<)R#9BLp){GoDr6Af2aQdUZjM*+w5q(w0=46KS}_qcWfMjz-M-O z#Ypk3!~o=PA9UxT_32E26qZ;N0;!d#$&byB7gmeR2kxG}*~I`(BfN+YHBgeIXF13{ z`cqM*LK-NQSST4i4nGQPvuYZAix_2=+^_?1<R9x&UCkt{6`4d#K5QIuKouhmE?{)> z$vNYkpJP@@pZ#k5rV+P;)Kk%<LFE@#OdXCouhe(^>L!tlak_KnWn0|;0Fgiu$k#K* z&PnqWf)sJMeS3aY9Fj#Vrc8(*3c#Pb*!^i_l0vTJkO2#jFjxFVATqbiO1V+;vG4eL z&;*MTM!=WcoRS#-09sv=xC+HxM<_b&sRZ#ov_<eF1cQOdJpPp88_I;K!IiQ`eKSB2 zM+vu;Zm#44EZ~scGtlE7g)OVe6!XK72|I?<-1O(ZMr!PlCy-=N9AgZ)@B04$DuM0S zYGIrd8C<UfpM3V92;R~6XL%&@e!rbF?2rBC{Hn0My|;IXkhU1}>M7samw)rgzbXL# z)BB!Ch)nKf&m?EPLnkOcP0RJcsibFjVq6j19YOk4c_X>u_k-z?kLzDPdW0&ah**J8 z4tVST0P3V>k`_Du@#t}iw(?40+T~HZ73)zbSqNDPVh1!_2pRY#fiKsC+Nwz2SQ0#_ zx!_=PpGwh|7=a&WBp%@UeREY=<A)>3R&mZq$P^8`i5$0Zfb#i3H%yH2^{GooINGR+ z=ekre`3|iv=0<J`p8o((N{UNpBO}R-9m6E&2NVc?>rmcYkd`UcnYN7kaaLq9Oo45c z5kcx|td};-5(!}ptKS_en@4U*`y_IHe122`b>05})xH#C1%dwn>r^pf-#MF;$On)` zH6nBSw_m}y{c}?bOV#Ev9z6*J;(!)8gy-gH*@r+WZXzl;m&%i_<!+y)O!m>k7?tp^ z++(k`L_Wv&o1BdE+JGS6bCqdFBcqX>$JUkY-N&4=Zeo7uIHWh%3Z+&jbm$1^2h*CW zCPTuPS#zJ8r2tskB#fi%ld~Z|F#Kv^3~-=G(4Zr;XCLE93R}n#!htjJNHlqG6SP+> zaCsOM0Tj^-H(6#}p&YmW09vMoE1QWGlWb`J0Cy*bAJUx+&2EM(g)){uA#uhj%{9EL z0<b_juS_3mq<~27PM+@sf=uT=pdVxDTI+QU&^ZdsxWW6U@~Y5XE!2KYgdA`WdRMZJ z)@}Pste@TMO{HrA!+dnoCh-@-9caJ|w{|e4{7Nvxb-Mz`wlai_*XANJ!+$IhfU(>S z!CRbf=ZgJ%w~ohCvY$hd%13&SZ6<j;WdsqAOjqQ0i8L$y4)R+&t|!r_mrt7`48|!L zRfz4B+HucbwZV&~lDYO=HHmiicQ_v$+6itE;z9#=1319P^r?Iu9>r~>o^nnx?dw~2 z5y^Yyqn*ey398yXqiL@)0DQJ?zV*jarPYz`Vkt#C+f%-_omb}F!NL(+R<Z_^FHw#U z^sWLqT(KG2r1k4tb{EWy5ty7X86W<-@^MjZ9awocG>=}=oNe;Y1oS@j*IJ1`cI<ig z<BIc5H&8&!CQbpa_6zlRmve9hcoh|;W9Vp|gcnZ)!Q;kpf;V>jtDzc<(*+1XUi{?O z85_p!6v##Z{LiIyP)>;%k-*$fYMY*wG>)bfEWat|6pb5?nYc9-oS?<MPb7X+O*{Ec z$D9n&c4;V~WeFH!%ti+!d)FzcPGLu6V&#Fr&NEt9lS#BXI6KBl=eMPJw~Ty7)=M?I zxbr~(VD!j5VyY_pZf4O-R@BR8-N`Ju$OLg&@Fx5f2LOTdao6*%7R+2;KiZrU<mHZg zX1y0mi|sSU`0Idu1#2o!+LZ*G)b&jk*;d{)Kmo@%Ip)1HPfgLN9fl9RaXKxiR2=b~ z1B&mo%qoM_8s>46(CniWnQlm#aJ@&SDzuBp#~jwwsu?n;99AUz*Y6zEs!cnc*V4w7 ztmMW+^R~0u+B=fPNm4s=ipq}WRWdGdo|Q&zNX5j=LFh5-S~HHkI-{?S;yF10j&aRT zwTt9>Ruorp;XYxzj()YX70z2}IT)cPbFVV%M0SQXW*?P90h}6$$O?{gR#W9c=xbQ) zV^ebK-9;`J7{x;*#j?GD;-QgCDckbZd1TsvdJax#kpvUWSLGb%ilsPKj03@_E~j*0 zR1K%AXQgM{+{+UH&BG2jsA}yP)=jMpom%A<Hi!;D<KH}2kNDmg=hO8YcIhN=DEh3Z zbNK=*(=}Oyw=m~`dj1u|_@2$BNP?k%v^Y=)sOS1uw@L{`Sn;dE@=8X&mjrKj3`jUo z0OPX({Hw0<o0~fyE-ZP{qC|_FXV>)2W#~!eMym1t_>2smhmYlo%$DTq8t|S{Oco4x zG2{|}lgH2z_}4^W=OlX+ry6vepJVE;+22C)*HzR4D}A2PEMjBF?_cI)?Zyp#BCt&o zOB6B!%8JaVvna>o*0@iG+Dqsj2-EFZ1b27JFXvox4E(C+*#h(5&{uVQs#WKSh?5Jq z1Md#?_c?tpXH9oM2;w|fx5hc&VWIA2h3}hbcm~NA7~u2kQ><3Ektqi(>y^RA57L;| zOKi<4QheRH1aKQ2J?SNmGaQd@yJY|{!NyH@Hr}l8nDto?l>;eKP7Zk)rJW%$m|x{? zaz2#><<!5qQb_Xv+<M@OCm@BpVp*dg3=Y`u%^?hiWt|ZVDaXnIJpikOV7}$THWXsB zE<Vc(sERUoa(KpE`}Xfz?1imFu#<%!GM<_G&;@2jwUNvV@|*=Z<0OiZ%N?8f!|ur2 zWY0iy25BL=j9Pg_7G?Rqgb_`cEY{4(0V+=T&r%z|whkx)#-vz<w+oURap(9z<DY!f zBVloFIoQfRTsI(!a>~mz69g<76<<8`_NXREKGY!jmD?=1UJv7%092Od)<=-1%OM%{ z{OZG8xiPR+DaxGwdC%cUWZfE}$;StdhM|pT0kOy#`AYQ^0b%D7$|JeL{GhG|Mi1pw z{MdwrP8g0xNA#&=b~3EE3It(D$3VQ)b43K_%z?I@fRmhc?LZgqqI*<HA&fdp)F)6+ z*Bn(dwc}l=>UWQqgV^^6`c$iXcM7GtjHv6o2l<MIebBeeNK^9?pbUOPngBy^@X5A# zw}bMK4<UY-rbd>o50>UP8QNImy#OLHmn`n%3Vmu<MV$k(1w9FF^Z?VjWXs_0<0Fyv z?NVGtYcy!$;T8J;+>g?d7*Pq2pvF%*;-Qkx)*uGav;x6^=RW+<0*|y>OXa~aRh3C6 zs3Z*I@u?!Zp6|)Fc*f#{?+wo5(~MLxzUz4&Iiyk);EvSfwlYNw%P7fE7>*k&{xksl zYm3Whyos$^M?yhj*w5%{$J^%r0G}iO0Db=e_B7k6r+G3a*Z>0<&rf=<zT^IWKS}`q z)ccxbShF$!N%?cqnCwp8gnufYYiqOjoo55k?Z@GRS;pFUA9J1$_kL_w&z_*SDR6m( zH<8Z+`ctAVV=W|DIQ}n|p2lU7S_u!+82+^s*EcN3&5zrGKo#YVJRE{gQLyu$@TEnH z7}iq$W9J(WP=8ub_qh*i8UxU?^vC$qrGWX1Ir8I>j&^h>>;C}Opa`1jr*I^IjCWuE z09993-zjMU2&bod)EbOTr5Zp=j(O)lmo-vi(jX0lQ_fUmk?-k114CsBu?5L~gBhl@ z@W&dV1dRG~Q4n$S5^=yj)gVcfZ#ZSg)X)PaHqM2!gZ1a|6;3dbuK696kKGJ$$I`BB z48hhMNz@YDe=14kL-M41pZJS&&#|Bgrnq-xG7`#o;8RXsVG<y4d$8bBw?`&aut&#E zf|3ZFsfbEr2fYAK_DK|#jf7-!PI$!x-oyLIyt&nW@L{x(?bd?`Nd8-`?7f)a6ZNR1 z{?{rPMdn6&>>;o{dr$+VvyNp1H<I0Oef=pd??8;fDGthTx#~?rD%`Ro!dEelM6HAU zX#u;L3&|W&IOH&4$n~HJq!P-Rx!&py1~|d=qS82Sq}cf0zjb@!qeyPjBQq1vZ18>R zc>)-g>fL{My-JR1*1%jg{z)qt1m}<jF;~a$F}_bJ&}3(v_N)l*R4{o90^NA;@9A3& z6jG>W$YL|W{{Z#V17%_gW10<yQbu<0Pd>e?^YiwA@SK`&#I05o1!31Ahg3#mxHI5K zE)ECr3|HO8-uei9%XvX=Hiqw$jAz@g;a@s_(-wEK_<O_tAZsRHvR+GNH0Pvo&I?4O zj!0<~^Nu%rbA~1y&^@e%w<aT2d`&i#EubJAbv$#)tN=nrP@Ts=O6{hQ+zp3l+&7FI z<xW69jd8Pu4KElgpT@j4t)ceVqB4Hy&H?^)W-pc3rB(j`mT1|>;hM5r%mGl0`r{l| zJw?#0JACI^r``ErCIRO;&OgSxTYL3%P#V07*6hV5*J|sm^#s7$e+*ZVT8eJ=J?eED zXR7J<yKr5}^gS_M&8+J(F2>wz$gK4^LMa}+itaS4$bw}~N#`9n`q!U0+1T|dR!?KS zw4Fd`^R<EJJXY(MHmtjcBey2F_~sG~=LCVk9jf$KGRih8o(BXPldpYDU1Ij)+ShW9 zFmT&&Mm@7%G<d&K6W<$|0a9232<z{f`fFXbo$W$8oPp0c$TjmPjPzTyf?JrByGGl> z=lDlj?7>Aj>S0$+jSDNdk^nj`&^Y(4+YMLj<P>bKdJ6HIe-OqYB)&E;%%`ZwZ+hk% z;(dfS$!j!oz?<-8Dt7+>D)lFaQhFS*#5F56ecPb;meu^I*~(;eBRtor={GUB%E$;F zyk{o9H}I#4Ev`a568WdF45~-hKGp2nUx-~EBj!0-4;TfPkA4MmRl{3j)1!%nuFt0n zTK$!NW!&EN%H698I_K%qxOn_g9@4EY(!G0r73ZEI_{(>ru#V44yu7y!hm{8l*Zq^4 z=Jryxjn*kPdkYx%6=UyNcZ$>7xr}9$=Q!*ulF>YA7l`!u?`^kcUzf{qhF16DyW1G0 zXvX#Gyo$zDnq|tWXDJS%W1W1$Pa?bfEk@at9P^CV4XbI!-cK<`0dLE#O{UIqliP}M zy3psHw@0YQG;X1k@I`A!aGp@l7_JviyULW$Lz>(BWMvAwZR{&cT;*=&?3Ym=ke&$Z zn#j6zIUDdX*0ZL#jXIJ+92%ncOSFT?H8My=#|g6x`55U~^2iSd_<Cp3qid6%4gjk2 zO}tSUtDu*|b28<TrjU`yVUKPq#j3+4#4mC;Bu8)<=x|1VTBqf^NbWnF<LO!!POUE2 z`=E1P)K!u^X|8Izk=)A2W4Kf*q_8L0@IO;p-w%8>FNig*Bg62f*S@+}xFl!y2-~G6 zr$kae8lFKMh^!ZMf86%}0PA)9E8ac?X%c<6Sl4`w#?swvpg82n-|Ke%1b%hjfR8mr zax;qRh7r#8K8%dVbLWd-6hxvphoDcDN6;uAoj!2I5EH{LcpRY}e!2WA80D2OMZEC{ z<|<em;aC7T`~a&YDUEI;`|*HE{{XamSKLKeBz_5AeDxf>$}+y_mPBtcSQaaeRhRRr zBAXY1Tz>$5J7={yQR5d6Zom=CE<2xkvu_=>;(69`;mW9y806)9_Np=qxZ^OzEN2CC zxRIKt8FiEF#^DK>cPPdI_RU<`lrgX>WF%ooH~=5hH6^5}C6d@(TkkRM+aLgtdvib) z*sRkJIE07{I+MsEr-{<sNeGjCc-*}9{{SYl)JWl%%W*StHb!|Lcj?7WQdR;sQo{jQ zi~<2ZpZ$6ORfKG57mdT$j8ewiTYno{gN~x3wvO)E8aY5PpO_w`4oCF$t1&Cf0FosG zs9;Ckpaw83#FzOo{5%?&mRKDw;*=Fpfw*TMj%pO1GUv=*gJ~GXN3XRO@3U}={Zkw= zw{h%f0!w}Jq@rn48Qd6gnrk)Hl*(LgM;Htac=f1AA7qe5GK<wmC#_$3<NG{I9vf%? z<Pc~AyKR<FE>}<vTjm|QdXHL;;y7WA%nC55GIsUadvi^a6CpFY?*MUw_eEI~I$Ldv zsr$Q^aQwKS2<{mPif9}+m;)6J_i#F69cd-Jdz4t`jiTHzppL)bDT3ZLWSSxdP9$Bt zmh~L*Pq}s*RIzLf0!DH8&;)R<={&e)Nm?!6FCc<Zzm+v!MT!<Fy;%`YW4LFMJ5;I~ z;wdC!jy_$Z2Y1$)_BMFl@tp74H;&j*)DKz!oLn!JhJ>cZ$jHur8mul1NZVCi806zQ z2mC6`7c#|d6I(9-05B3%az;IAu}^U}?dSx5?)Uo81aip`wA~pWIRkG_zVxt4>miBN z0*qh|wG{T!+MF=|09nWfrYZLk{m`ljY!H12pb2h##gho2jCbOo-8c6a^QOrxp|L8d zY<(&}H2(l`e_8<l)%&(cRpTs%MJKO(kJ7XL&Rm6xMpK@hO=?`o(UMd*rUBx!e9<W) zINCXfIUnQ2eEI4PG7$R|jmg(1niuU-l5&#w+uVE9S2n~2Y0xh3;?Gk}8hzr&=0kAB zeFr1+pb6b&P*o9Bo(MHeMKEx(C{*sfbM!oOQX)vX5{6<%GuES1XKo8jB;DK0>G*rl z1oEwue9Icf$~fKfpZ>j4M2SgwRl#F|$Cdm#ROtfU{`NAB_3Mg5=Be_KZBf@P$MT>G z)5_{LMi@v>%68-fQ%3@$DD!tQ8If~?pXp9F6GEG#WjlH2{Aq#)xnDX@Eg88aJ_+{e zKm@QXndHgnpy2%~po(ai3GopA9R1)u{{Y6UOK#BsB9U@?smUJn-zMT@BI@EYoe5#{ zW9vW>51orBRwIF){{TuG2tb=C=c1M3q_|sn#CF%Rd1`r%Xt}5*ze_09MCRoijJona z8UU9YP^-rrndb~~{!Kv?j|@!F7C;VmE=m6Y>(ubX@`auPn|k4eO(bb7sPja~LFLfk z55p7zO(crxgu?-k3?JoeeF>`z2;KY0;VO6skbj;il`TgT#wBygAL1CLmF=!kU~<C) zp!NEH_2>by%_MF{BWSt-obmLg3FEe#%t7Uh^~Oa+uL6GVBnnjEmZ?O_LoAS)M@$2O z_32i&14XSZE~95c9h>HnbJ4q1i&&uYKF^Si*puF}ETXn)yv>qC*u#Kw8`IMjrnVL_ z$>vDgx10c97(II9+L#)4cQ)$?wYq#rScN@7KAzbXOT;=Jm*SlU@58#fyIWqwQbt3E zQluE#Jmhay0FPdrXCllSV~{nxYanL9NB$ph-~DR2bdp@gcrnQd!jZrr)3-9SGxn~G z)kyAsTzp6PcjAAATAJDG$j_wC#Wek5Fn-H&gK-HV<gD?JmDiv>3GL=H%9Htt69QFK zo(MT^#2?evzhoqPZ9`6<O4nhA^2+Q8?rrV>k{J<t291=d_XKizIUrZUpR{*@J{Nde z8)>`=qDQ7(!*LqkJ+rugq`{8QEVC#efI(2B0~oJ65r#?gBz;C3J9T3iJ<pcYa=@c* zNEC`B3%H7TIQd7)4<P!El`Xzlm0}13HRM%D`iwhwPUS#>b1BF^ooiOo;n9~E0Ps(> zM<fi<lNK}SP)D&Q5x#OiTH=&Z?LoG#&p@)4-atII<aOe@s~eyj<~u_Acds#y;q6Fa z_etnF*8P^CS%x#ut$Fy0D)&cK3YK;~Yfityun);SPp74Iw%3xxs>6}gn)0nWUEmGf z>(VV;2$4$y2*EYL)t!%GHC2uK>$aO{kb)U{9^$<BS+#~wGkPf-NWk>2%I@~oC{WDn zxGCUsfm}ALs#<wjNk-{C7q{nKSX-%G8CR;UbHKbs;E3duy|75h1e2bp>0W0yhy3f6 zw43G|wvaK`lhBI#GgdH&1;KR;z4L?5wRzUJZ4(u}w_wIH7@T7@>{Fb!XN6L$b2$q? z0_y<9eWV6g2Mk6zK8LM(zMH3bvrdK>H5lZD3ChV66SN+=9Y2*r;D@`tnA*p=RGr!9 zs2x8V_Lz;uy^Y1a=1hSGz<E56>03>%r%WX~Xmk2~>f6l}FK}jz{{XBEGROQY)9lTy z-GT^h;`xEXk&}$~>0UpiESEQmZR1vzGFeYv4SIf!0TRO{wnUS$J39<@tQ=!~5v*#> z9YvOtVPJ+!D`kwc2Vgn|{HxF+gn2Q|)UxF9-nk1K$Y*%~;3y%7e2Uqz)dPVL`9=>7 z?OfGq>U2}ZPiDtzFhz424l#pVwbhunlq4enV3S)KwfK#KgPp_#><Oyrb!~pGvM>^D z!RHtw`PUs-K4YWnI@y(Dl2{?@oSxNfrFlY_&N%98G?oX-JgE6{4ON-vNXUedgN}Kq zPTQC5BH6OnGJVj&vDezGwbE|DJk=R@c}UB_BcF3sg2<7yj5Cg$)>S93x?HV&4UCuI z&{cW^(x}5WV;F3c&(f~O!)ANut#3nJ%;xm4^+UMr?lDEISh&X=@q%&rQ<1RWUOzhO z{0ZRO&m8!LbbD*FW)(!TOsE{H=N$<iy>njH7M!X(9u_kZ96NPqu^pX-=Bs;apy-jA zZ!XTnVafzQdC3{VsNj+ZBn)$r_Q!@iKd1OBNVC%P=kwawZkkd504=6*`^Nfb102>b zhP)l1coNn`(qf(Nnn0H~lHsJ3`nSqB<DejYYqj2{=q0E*05d6EC_dj>`n(2P%RXcC zUL@i#t$yko43|)NCyB0OQxM3B$mMtTKD9K`#6NWtV6u$hcL%Lfx-WGUw)pboQXAK& zr}$FH&vO#AWckPj5cdn$n)e!OMtos@2wYu1mQ?x6j(vEl&1D^~pgfQc;6rr-ai&Lo zb7~ODmWEzY!fiZt2d+P@Y)^L<BDul&Pa%gqgUui_Ue4XzGROgp^N;@kUo`|*EoXMm zJZ`bK-6{!R#;sejw5*{O9SPjUyPiiFt1`wWjo7;?AC-cf=jrJ{5lJy=KHVP?vjE5L z?mfMD#YsK4nfugqaB$#fI3w{HsAE}#A7_=!kdK*FlngIVT8#{LuCc~24p}(PPp9QT z2qb?l2+FqEIAfduMt{epI@bPJ;n;EJBjJuQxOS!_a$F#a0Cy{8k(=*i@<$!2VHf&L zkvI&aA|?lKE04~ACA^(lSlx$~BpY9jp#D{E7*xpEB2__!z{WFK)5#pDW0(D63;|7n zWL7dp#3s|`5zgFD1o3~P$u+&w0L1QA^&d)(Y37z<B9m~(2zqcnl+Y4uXr`Iiwn<h~ zo(He^Q)QAzC|OC3PY#1591+h>C<4W~a!6&_@`Av7_M$lvcSy{3A;>t+De-TK#O_Ne z06sy%zCgJn8OMBN)iHFiMq<bzvI!%xIiL$SOJ-RoiOMvCspx+SOJWRf1dZ~8xUn5R zob%SJUP`iBlDi#aDgjfRkLgp*<jp9!hCGA|ka-x#rx~CK;6{u(D-4sB_g`Fgr`x(l zK%K}g*ccffk2K4<;umWEQH%w_D7+AV9x23>w?)Pf=PWtg55HqT7wwYMO?I8KiMuB! zIRtjB^Dqd_1lz+BRBiSAYQuev<41fsU%Ct9A5o5Kx0HrfHtMQ<+mEkG041L4?j??K zha3Z*)h)Srq}o@^C@=}{Nj!0s4<xD>aq^tw{F<9ixL6`ooPPB)p7{r*06J+JN8S-? zFQ5C1PWvp!mHC0^la91~mgCTW-&6tr*89eISrxptC*1cHL+4f6!*_mbM&e!ALytj| zj%vAExtGe3BUJ|@zJHZ``RWyeZzF1}zAzP!2CFe9LQ`@TA2Ki92imSnYaC1S;5SjT z{Ax2XK)cT6?m)qy4B1hx7Y`$#2Oy51)wPZmRSzkZz{VYTKdmbZ4CUHL^V1#b7gEFh z8b<al&*{Yg60#}V7apNk{{XK|lGAc|lJD9_Q`0pBQ-DktO3NTQ7#xx7Qk$6|L;J$& z$AH8C0Ixt5wn+%W?0Crehj1!VvCOKr$h&dQ>au1MB8dVh{{Uwhtow;0+`d}3PQ!`- znoDa)Q!YM!pDR;kb%|5TRXH5-lTgGr?w(Q*C-Ap=uP&o!DcaKri0!n{1c`pT9@X(F z`@vXaY3Y&q)u{F>4bh_p83P?Elvh@5@_>xl#`E&#iS7g^Y`at*j1EWViU5$!EK<Bp z7*00;IT)f_n0%6wNg|ivZCv2~bsv~aSh*nN{HzZ<Y9#wS;Hu3S1A~|OvGm0NPZZ9J zx@HV9g(Dk>*P3hUcA(-i7EE=<dG)9oT{VxF4YS-T3^8MYpHOOLkVh8ldw(qf^bL>3 zsURSdQzi~qaqFJuofXBCxJM4Gyc7I6_NN<&=R%hyMt<%_IsB<E5neP`c*s$}Q_u3P z5>^wETbF=>H`x#}78%Ys{&ll$_E8QP7WG!>54XK@BHrpnU$E_EUU1FMYSa@*nU3Nk z6P`UmKVH<DA=F1+CWV<!)(B1lsOT~4Rj$6#Y`ct}XdMP|g+8PD)>LdRW7`yjIvz%Q zR64$*`c{`Dw@9rUAsHu<dkoSso4dKr_`AjN=(dk<8U?+UG8ZHexd-$8E8;I5$oh_v zdo}!<<Y2BwPb`i2{{Z!?(|kv&&38NB-8fT{2RR^iuM6=!_cvDZ#TQpn0X&A!@~Nns zjgFi%_tR&XyMb)7E(lBvb>QZwGej^@=YgCIb*%ev>jOgj1F72F((V9(w2=OTrYpe3 zQ)>1<M$2ZWWNkqrE(tUMvP!wf9qJg+G<&w6lz0CC_0$Oan*b1V{&}x1{m*_5%FO7b zw_J=d`qQDD!5~sc9Z5aEDx+<8G6Ij>WALb0{QD7exbvS%&UR(OUiH!4+v<^|T#^as zYkJdLWsh%~-!R7&<TBj6Ocd?_9!Wd_>^s%ttn804sgnZ$^yk;Tar+KwT<LY>v^}!t z#CGw*l3-w_3Nq!top74IsA7c4BP%Y_$|GE`9)xDR#!YhR94gEk2N}oB&u+actanpQ zBfjR~<*<Lx71>7*%d#__Sxc$4sB6AqSb1+bc=>r8`ev<K>9==sAjZx3z+u7sGg%hR zWYPvMg@Mi(^PgJmM!TokDw~H-mm675dHPoKQc}?hloOVxvG^ZHNQi61jwb{K+B)El zYqZt0nC>KI84ehbN1?9;kK$&Qn|29FQH|<5*IA|fQqb=virVqoES_Y1pTh(EY8#u? zquHm1i;jn&cz;JrT~tc(f#)5$+n&EqrF7mS@N*OPtu9F;DmNr(N@LgN=lNHj>R%AF zy&72B+B0z`J7fW#ohq07E-Q`5XRu<UoU5PYL8W=FgLp;Q*NycZ7?UiF&eJTCWcL7% z>s<ue!dcB7%jAj3IP5cARNoRjHL3-QSw*~zeqi8(pUSD+>h{-)yEO=8P(D-)0R!=& zMo)7Zm}_i$72Whxxq;lO#=E@J_j&K_kH)xZZtpGiJ;nb3b_fc<@Zk5ZQ&90ldTVTw z;xdFD8bBB~W58jIS1TWmEbpg=IBubr2Mi){fOFqC$^5EaC)D$+^F^Nin#IMt?<!=B zV0@#Tp8o(!=%Sf!BHlW;IIbT?)NJ*L3BXrSGNHlTI(F&JbowToRc45MxNX4o!1S(M zMKqDqLQTC(<+>ws0nTWZ$Qx6iV^T~!wh@pI=}?WNWP$RY$E|sY^sFQRfs7BrtSjvN z>KL%D&5pgOW5}-DKEi84_m~nyaxxAEIXzB01JgOLqJL*yK1S3$MXsVhl95~3teZ&! zVTl|LoPm$dzI2U}M%dXLV}p$3f$i&EuYx~lJwM^ri+SM~wA*W$WxkSf&m(Vn_NpJ3 z92^clYWmEUn{&TI^X@OJYFww^O43~xNz|cW2pGmxfA#THp;Srda_m?fAE*BSTD(U< z_}O);Llw@oq^*p21A0l93P|f74<C(tJ)GiJH!#YNZ+94Elk1EF*PPedEjB+Umf|W+ zZq0WPGA?pK<kZ@Y(#aj9p>`GO$~M$1A+l*Al1;6g0tQu&aqWtpSr+C9p&)t6rAm{- zgYVjcOE-6M3kD6Dqmvvc>GVCSD@fIqqEvT8QGnc>9%?&y=Z4Gd(XQf|-TCJO)|%H6 z2^o+)OhDkCgirtu>13G~wiprAj+}ie_)zx6Ksa^fdVXGpqw<*SmEBbc`Hp!em}F@e zb)(}v4a1JO>p%$<bJ&&G1IEK^WwOc-IQnr{EfpSe@(sy@w~D5*OK&)7<WL)MV0s*S zR5sU-1X-97v0URFIT+{ZKo=Kf&{-51XMmlD*Pp_mmPryV$gEsBJg*y@-j$|F-4+em z0ZC#0&?l$0O4CVgAl$A~IAY&->+L`jd4hY0hQN|}WU$8XPJ2>W+&m&cAr7ucVn-j9 zFv!-It1O>1axcnLk<Y27q{h^^R^F~j&tab5S^#kSEDX{F^3aYz`@H*hsDhQYb{P?| z$WhN8xu-Jun^z%(z~E+@rQnWR0=wHdUz-^?&N><Zb6gd)!AuZ7UvW~GSuEj{DjYc6 zHuV@i>Z>fWr<Ul^yg@-%JmG;qmmO+#o+#~1INDF+iU6kY+8gNOi}zuamfAS{v;Au2 z&y)5wmOq^D#~ctxVk<pgWyCY&DOSN^GsX{UwQ&SuH@LUVitW9XPBKT~#Q;q&n;H2N z8-QcnX0ybo2{E~0)uStb53O2VMn#2+D<ZZsxyc`nDJ3P4T-*NgO}N1ze5a_#^`HjK zw)ap)INBAP0fON04rsE|;kZ)EBdJV}mz>kBZW%(v(6kBxz$0?|^T+b2rjB^jT*A(! zn>$qSMF3c2XwjpQK|}JL-AL<E%y5Y<xBA39mf=YKYV>-QwCs?{6pH(Cmrs<D{?>Z_ z6&p)0+6)nu3O4Uw_FM3v2_f^Rj7st$C(*zaFOvTN-(UOeQxib9C<uX1T%E_Bew6R{ zN8tYeo(ul~eIM|k5C7Nu%%mX;ZB=gDx{ubNRhIQiJM0VGE;IVnvpjops+<gGzG|G4 zsN0re#A6G<ub(|Z9aaJ&iHnifaHv@rWV)6O*l|;&hz=w;Z$nYbD2$eAFg*ax09MRs z8_W#YA1aahR82EPN6OKu86}s2_~x%H@JNTuiH~(Z*R@pG?aF1_KQBKo(Dk4OxQ=o5 ziDw1C!h{*f_o}i?fp+yIV+^_ewP9@w$|abX2;r4F6YJWR;@m*rXhvyN9(&+@C<8uS zG8tnI(JLwB<%mCksiV1$)po*+h%tn7k)Kd;Oq7d^PYj#Je(yp*3T&GR4JKnGa9@$1 zr>y`i(Ar2=^(SEExl%q{`gQ$kDU|LK7}7v`oDK)&R+iclo@bge2hYX<9^Ump+gk_5 z`-Y9V#~A)}0HPS25QSY&P<X{E#+JxX1$8}DzHiIkqG@14Qruwy#t8Yd>rIkmj~GzL zs6WzxBHCw0Y;6GHMk-}ZGwkGCK!N%eQOT;e7g4?>C8CgUS064tYMI>1U5O}o%8dS$ zEx^%{5>1h!NF&A<VaGo7ix#15NKhU>?&p(Gt?DhY5w_x^pIm(@vEL>Y*J<OBe)VVs zfv;{KbDt_P#@PrzmlbJXiX>+)G1#7&s#3ISJkZ+%cPF3DtNDtgM<@)ZkVQM0)LLA= zuNp9#FyXp<gCLr>_RFbIq^L?Bta{_=kLy|R-z~JCAtYn`@l(Yu#l)-$3J=Z+<0I4c zs=lW@b$XhuA&%S#6LMo7X3P<vUU{!6@%F2CENiATpe$Hs!20{wp-rk?Y4-DZ&ZPnq zxBmdIUq0(MR~JR)ng0OR?kB!^<MgVORgRx$6NS`m_btWDrzOS_N#hmd+UBI{TFhVt zU%j`d>t3Cz-6gXkmfTnn0qM}!iR*Tkw~@;6s|MV8&IUiNW#m<qQfTqz)>6yFo=w5q z71Wr^4g;PPf549(*{<U14UDciOcDlo&;J0gBD_!H6jxECfey)I0!Y|$L#HDhGtV{X z9t`m<uZgs1WsT38h!Rz9s_HrV<mCHT1}>G=(e;^DDZ*(sbx=$jC`TV!t#u=z#9x>d z3NcpPF^rt^MbFRo4+K}wRi|ywu~ih6*{!GCw9cr*k};auhCTC$WzHDj<Bl*-^sX;Y zmF?z7Vn_$py%x&m(sCppcysy@TvL-eX(aY74H8THR`VAMpll>(0M&VX4|DcLn#$la zgg-8E`d6lDR~C|K>GNQ0+I`5Yu$wfGIT+p2x#Ng%dYaUyccJmVxo5BGO)PM#iI<RF zavPto6~$fa-ZQeC<UtzCj&_nWkF9-~;;#zX>QVio5*&a@0N~e$U+4nXKbky*Jr5`O zSFxAlBRB66?AhcN+V-t-`M6^t2Mw_ZTIww|IH8C%(a5ZRQg)ZldHgz5j3v|Ok!_Q7 zUwM^>uc5BP!}^|p_bF{GrU%aAa0vQWuH891*zLm5Z&M=MTF{apx09HQgN2m}u6>Sr z*4CSQVIyu^j3#(sKp(ApuA8NJZ&Hrh8>?$+{Ln(blo;pS*IoYr3X{TDhzZl;A(a0B zvNwG*%^a}oeH9!t<<XuabK?I14#gU&x=E)P2g^Izi0#1QxAgr@wM38mI?`9qP|MTb zrFw^kJ{kC0#&#)YnT}pUMluJ~lV0y-r}$Gzwpk#wi5T))LNcfF=8tCvb;g`xts~|g zL%_Q8SS8k^kwTJ<$QeOz&x6vrX}lR^Hk*gs4iA=cy#5EJdxpE>i7)PFu+xgpBW=6F z@B#D({A!+;qdmfTmnhrE@jeKyJW_-e(T!e9*&~9{ejR9Xhij;Th@n1E$p?|@bDHtb z6Z|TRMDYZdx(r4eS$=;nJsTsj&*xunCYKBlAYq-usN)rts%ZLc)Ej2NZ2k5e_u{;o zSfw`6;N~<|=^j(yTY2Gy5x@i}LPCu8InUu<>87uijWV%2h7YzY&Ggve({&$|gC22# z{OizdZjf5PAPwAOp0$N$-sd$3%c~{3$dOo)^Hm7jDZHMipcR{Sscw`mbGDlW=aiTS zJ$n1s2P!+IS#&nR8*`dQaq{|gC(u@U&RlWaQ(d<Yyg>CIops^Tc8Ww#5|mx~74`jU z*Fe#4HQUAlJ7^@3BXwp9b^-6Z703KC(qXX+u50oMHBCtc#F0p$yrU@u5~nz9SGM>g z!&+a6;k&)pqmFA?7471Nal;H}cn6_ean`<m)YA9CUOBq(#;VTBzv;Gfk<9QAHm-Y> z9D7&a<hX7{c_Z>pC(PC&ql8)YHjQ@;#8Ipz@gVtw00BAranil3!@eifyf3C)GNVsx zw=9mkwquZ2@fGslhV{mlSZ(8wF*pE@fDl}MJ%1|q?G|^qi6NR@-fH}SoT(kgKAcy) zmb#xEIHY?flkpG3y3$0xd9`_!ScV`5X8XV#b<Q}h*(11Q+|G|5RFfDXahzll4Ssyz z>PN#~D!S1@D+9NiZsh*}3z4}%=b^_rucZ7T;@v{h+9X?pECom~vC5D~QJ#cRC1z)2 zd(5&;2uqSrmxUP|f*bnrPnDDHi4={)fXkDTYCA1L4O2r{t(cW)_{!r94K+ma%O9E} zEeH#{p{XUJ*qh~&G-&Xx$YGoeA8}Q#REs#}LRf?IDvSnP{*^58#w2+WaKnaNeeC*w zg+&D7NWRM=2MYZdZ97Ol{<xq;%b7=!7;F)p%FT{5{5?*7wGGUf21#c5_JnSLFh1R> zw=rq)TSOT=wcV0=I6bM7tTzl}j3Y5zg(n9q$nD%_fG^y}*3w5jE>uVnF^rA}T8#|% zzq@I}A!b)nG1Inl-kc=+Ws=VjDr4Gk$J4GVsNJ}|v_(veP6^{CfGk_ZiJHigF`b>) z>6%+XB1#PL;Hwe0x7QTOQ|%j?E0l`jA%#wgPg;Uk`%DF%V6qj%VDKNH!S9*?*t?1) zW%Acy#^B!m-{Dkd+lO+*w1aMO)S9myw9-O}aVtL6`EY-XbqDEFTt($+##ZK7*fs&r z%Rm+(I4rG&i4}Jq`BDD>*Q$i#KQRihk+KK-HE$p)$_@mpoCDUY!L(UC*63SfVYeQi zPACGi!^6m>)a@kW)9F@Wmt(3)=4@cBppk~iKb0|BDVc7YCt{p{cqgVY`qQMiNCad? z3XlnIyK(LK`p^ReD<qrODhcItoKvNZ9wun8938pGPr{aYEn}F-3dH9Jj(Mm3t4||4 z$adhc9B0=YPy`mUdGUESB;`&x^!2GCwz!XMOgE-}Qcr5DE5{pcVtM6U57hfqzhs?= z5Kq5&&tCKZ%ZqEDnYTIEcPLZP^!~Ly(Pe=hOp(bbMJ`5u??0|84=U)lm2mq8!g}+; z6yy%7u*w;c!NaM}IqosV0BXmqBCgiBDo0R%{Z#(|?U?@n=j-&Uj;y#<RB#3ZaZSVj z0M8`-C<Fh{`|5qDJiUaHdyLgd1<ZJKP!2FQu1#E@%#F7Bz~p0YKdn;oq+FDF17oL9 zN7lZ4^$2B(@>Sg<;PuV`Ablwr(1i|n1sEqIpTeId%Y_lL{N48sqNteQ4Y>-V9@GH0 zHy04DRQaj=DU9*&QAPHL#IkIQzQyB@T5@bg*x{EwTeU|$lt_1Zt1ON8M(ljQtpG(U zO(PL5Q$KaH)KzGt)L>>+Slf;jn*@)_rw@4(sNRwh(J`D4tul8@kk54=mpYy>ae+V* z813g#adc#bm#VKLKS7G3_sw&FtW`$<yndgJGAW)g3}!`RoR&HLX=S)+N(-3^qIDw_ z0DYS1e(+o@ss>8qZy!p{Wu8#Yc?b)i-LN>TGu<(7E*v~$aLzD3w2LH+H%J7b1~Q{4 zKVGx}0gd*IF5o+Z{VBo;B_qn=yB6SPn;@D(1Wg!2*BmgY)=6ziP6_@K)F1Gm3lcrt zE#%#KdG0{Z6<zKkjO{|qr-FJ9%8*>nLrZc{C>=5VYPne(_po7+huzQXKn^QzPUU4> zbJH32s**|z?o?CTDc2uLu_SW@T%=@YBoY4r*QA}JhiWyZ*I|&+p5FCwcVI?`?VL#y zc}v*w&#gHF#(b|e88gCxmQhhpCfLI%3`bSn&OLEdh2&*VmfKq$#wwZ<N$zbFfFOAk zwsW1Q2CK^$0VE49);mGYan$tuD=sUEW{}3vI|k@Mz^kn!cCnJ`zyXXg;)|T}gHPQ# zUl{6gFOjAKLtH1#+X_AV*OlJ{jlRzlGi^OjYU#Dj_gaCJo&Y8`&q3*2o&3y&N+`pQ zUp!~>rsUahl&@xSn#4vFX(o-N$-?dA=LhnxM^}#N^uQc#0CdI(9`)Sn7X^2_9sXDV zDE=aQ*9ob-lSw9UKwRTJj}(^X{cL=P@hOJM2ydHdDBbca%Df4x-e_8gxwZ_@I1Mtf z>ykQGx%^o0)81+CHGn>Ks*>3LFmw5Swc)-Uw~u6RF68p?8FQSG#dJ=LWlv+S6{Q&_ zeK`gEI)$p<TX2um0nnOUu{Z|2qfOUj(Qn1OZu>33;C5EX!Thscl@;1P8A%|Iy^j8s z^H|IzuG#kZ9AcaoF(LCr!v~J_uVZ{+bCb8**0JSa%n2FKtv}2t!G_QR{{VM6Kb?40 zXufM6-8#xB^j$Asc%t%tYyx@4MQuak&nsr}^0D;qUPT_J(UMH04@^}#FZ{_8HgZAk zNx-aqgSCz0io4YMF3Vg?h21kp6mAY%sUPE-p{i*qb8KXcpxxI$r{`W{q1+<JAv;}n z3@&-ELblc9pUk(o+m*%+-lHCrp&w@S-06*2vw^?St-g1106ZQz{&mbr;U76xYf=~t z<&%O3uU>1>^$RAQcf?m?^%aS-du*&_6;Xg7lg>xdyQtzP)tl3sPeUt3z1H+J+ix?h zV<ar2IX=g=dR2#tyiF=6+Ahc&PSBhXPAdx1>fU&gDUn&1p;3z8id2JQj0VpDu0R#o z@e85pQ_LI}D>dzB)>!$G-$;edTg)V{;08Ia+TPbul>*6kB+B?WA1d_voYw<qtKG6l z3n)<7`Bx;5>0ND|<f{rx3zbvAC!T95S9dxijOWz!9Wz0OB#L{kTq(+`Z5Yp7=aXGl z%VQ4Pbzn2;kLzB4V|95e29j57kViNGeJi?#LmH%v<n|-*uRj%4vZp?##9>!wnVbPw zddyq$(VXYo9lsiI)UEu_-aZ3o8_yJ7A309afpL+8oYy5S9nt4vYN;zD%CF?PGnnEy zEUm}VuUcwC8%2Qc;3;g5GJW$!{gy67nA}txah#L+RzgD{E`H!3WD}l6Yf;^vTa=cU zI#{KMxnuO?`qiY{CmaD-D>Fn3F4A$HzLj>~=(2|?$?PjQ(&lzXl;ze;+i%^k$a%-- zQt6OSt7>p*P(PMtB)flgR2Dyp#y=Y4^}i9@={D0`Px5&rDco=gZkYA0{{R8sz0q5^ z<c!?2Zc&_&gjdjIm@j=@A3vOCPMtX_Z&T@4y3_QLV-yzb_C;gz{$4@B-Jkdn73AL> zycu!g-wa2r=<%5KT?n=1qaF;2xEW(U#WF`+V!bxn<+ZIUIIj@La{{pnjeywb2OMU# zd_Sn#>hSq0Tg*_=JT7@wUKw+aN8l^(p%kK#`6Wro)NQll&xhLT%8=Yb+&UA_MP@%Q z<BajzzM=5$p>nMRKyL`6X;GYGsr+l?O;baE1AI)@JU)xI*8XvGWW6!Jmr>V%dRNtd z53lW-7*uX7(Sy8mjGPhpVy~$ZZJeLRdwA`2Zw%`Iov&~qkdC0sk}^F8E73G5kzt8B zM?hPXk(}}Qj@9M=02nXK`p%Q5hE|Nuu2Fe8jQac6r+9YQy{gPmm2tFW_vhQTX~}9y z^ml@_h^J$v-w!9uRcIL-9G=}f*Pu&k^4t}KHqpam^kLuLyaL<Jj@}<3{{VRw&>x{v zE8es{UOh)oMUjBHMG?5q@pjMD3eBQqYezlQK%*dF2;^X&uWF+#Z7jQxg5^q*!>1tU z^r>1ljrL489=JUZ(v_iIi7YRG=W8#gwIWIFC6EIg%Nas;<vwiGbGiAR72LoO&5vF( zJqW1hwwF$j-9aL(X@FGpVtadiYUDRb9FDAWzYO0m7(ImmMuiqA#A9-*02H46=|dK| zScvlSrHBU{R8m6}SGiDBFv{fM8k*q3&@IcqD!?}0NFI~{CixhsWL$EEw&aY|7`Kg0 zwZp2F=0A4qPZY5`LnN7Y5y%5ZK>q;s(Qv|hW{ynnZUmhO9epSPUIqI(F|>^*1gAjG zdYtwL=})whCz<Sz-;W!V@_8P;@lnS3`K48C00&TTDqHq~JDBH>kl<xURX(}J01>n+ zKiSkWvghxN0)INNprSZYl_%y=#%iOaJDfX*!3<CF)rN&6DI?^lTn@({N&u%OBRIB* zjjR}R!8kO>U5^aYqa;!t(r33(k?ZuQNf?ZTbtxY?VtRv5b8@h(F35I*0d8=o`I-Qm z?C?P}Mj##s%ON9>JLJ;I04@g3B;*5(W7@M`D~RGrke7}xy|udM+Z<9!avcz?CvX`J zgOEMw0#<+_Vv6cP+w=Y0dQ;^tWr}FtI81yHIv;wp(q<UiBBZ(<%E!O>)pne<v)lPX zHzR}IfGkFnT+CkWKrn(sc4hUd(=?W@Rw-2%0ggvd-;GBjiBcgrVo1hUBOSTxkyla{ zov||`f~u-<%I)up0IXt;L%vW!KiqBz^`q?8-|hbZ-&Cy<2Voc>1CG5<ttXfD{{X(I z1OL(c_CP@8Sx>nJpm~f)Gb6A)vC^$cX(}re!B1S|)jvH77}izXJw<%^>JhcV$_q%j zz`)P*s%Adwqn?ZoKcT13DVShjDh^2X`c-c=BwR-_{J+9_&;m;vk>$#&;kYDlaa9`9 z-X#6Y?QziKB!0CkUMxYpz<Ogi;;FNFpcDmPs3w2{*-)Q2oBR0dOnV9L90|&}$5PnG z;Z_=3=VcQGJ4aF8lHy{^^I%|h>p%>6^vGD53lHG{bInqC<!8(htg)U7w_JMSs<-+? zfI}RBj^uo#{&itY(2{NeNjfeMT+jrI8~H%oL<B^hJu5G4aWkxC;ZyfOZ<PKRt5PMw z9#pcHJv{~uMIEX}3(Wh<kCjNE2n)>P_po`0Kf2gs`c!XltU~z?iPHz4)~y*`okV~v zeTwnTL31Um95kCxW1MDy88j>+1|SOOC0m{<t3S*8&YgmA!`l^FTWfIJ#FH^S0O^{0 zqwEs2i4zaHk`I{vv;jPWVk1UcIZq?Z8ROR(tlig1DL!jV?0MVAA5OxbGEE^8M<Rv( zAjc;^UO4u}R&xWkz^q%jPxI2MJArZKh4cHf8-dF({VKa85~lfrxE(R;R40mPzFEi! ze($b5>PWuHZka}6-NrM;XwzYIi?huXaQTrChB4DM&fnb4<iE}i?uR{ys@G00CM^j* zQXii@eif5_eGr_)tTy8zdw@8l%FL%$&tsI<CuqbCyDFz+_RjA01?{p&ZY_x{nB4r} zj^mzcEmXwn!H!co=Z<P7pM9E3aUce0ffyUNXCQiX#VZpDUEIo+3u7#kBN;rn^0_=N zN9$f`JYp9n<ft;_le@1xaoW8W<?#fNs-Kxh0ATU&TwUIu1fof9;vshMPi$33bEWQi z4~bDvo}qUx;g!$cQV)NwE9Pd?4UVrYB}&HZT}C({n)>fnifO|bBUVgp86U%6KlrA> z)y=|?UL%4>Vf3z!XKNL9)~BB9bAIo4@S~{DLch+soiA0A#FZeJ*;SQ`ZW#Jvxb1WI zmJ@p8pZ34Pr_ruV`eaOpa8=lI&pw}}b7En5T=z1%&h}@r!+eiyvdDA0p{YF0>A6qg zTm*NrTf&PW;B*zTx6*A=58mxxIaZ#h(kj<lm+qOca7g-9NF{$XF_VGD3D5MW$vT++ zZl}_!L>qbB6a&!ru5MOTYoyNKPPhXMx#094&bp{|DPB}`W*p(Rj-tHg%5e;;+2ihz zJJ(Gu>~OAQZb;56nzcP{dK7E*Jx0~o#s=`p26@JENUM_A!8=H-H+;+6*z;axqv|qE zIw@`m=m$TZYu0Q|_=}z}ag*M$S5ZAmRj;|74~JqTv?cO73cDVOaTxO-Cmr+9eJi1{ z8$c|0V~<MOJ6T;-mjQeC<W@ZPIvk>NmKsB#4=SCm3CI5cUb@XcNr@pvC5GJh>zcC+ zkgy6EJ~4nez{PLb-CAv3>Onn5+|{WjZAzp~tv^nSWM~+YdFQoHsp8Ht-yrnuRo7mE z)a{N2W*>KqX0+1k+fuXhr%<gQC41!Y{c9?Tv!<pYa`K~!^F{1y6qvvS@;S~$ZwWrl z0!Ik|6~Q>k{HreS$w}gtCkPJZ<Bo(?X)h!`V-=6(1O^>zIL1lnc}}gKq;nGQ=8|wh z>ewHaE1COQPd9lVdjlMtR7<EkM#1C+DI*n!bAN25kTBf9@;z&=G`Y7isaht}-YbK+ z@s2xHJB#@eEwM;J$IFje=4ZY05f_#WFgtTr?2<>CC6Hw|4D;#EKOAH6uG%ys7LF>_ zz1Gp=emK|eFT7GTd1P}2pKlAx9&?wR2O0Vd*Q$6g#BPS&E!2&@T~5|G=sWRWF6$c4 z+kPQ<qH=u3iu>%#>9W*$c&}3M2DYnz8Q~v4fCj$12}&s@kB7%ra=K^fXMt{n-dxvP zoMBE0=jIvo;EHwATj*XXF*)1zrt{z+<98r7Prhq`@J6pT(=1Q8CdNE=91=e|>->D- zG^;Ibke7x~0qT6d{{T#!SG5;(e63hoTAnHULR$+z68K}qFakErVwmSRNe(*t=N0Py z3NLA*`H@DuVC3ZF;1YBA=cReK#(V2cPry;3hCuLLLo9M*?;Pf|^uG|nr09t%nNnA` z%!eC*J-DhekxJ_N4-)wLDdk}#HoM40;0Nu;=U(%qTwGe&Ko^jwJwLtE@~;f=9+W4# zI=#c;@1-PX9Xah^Kw8Sy@w5do76u6&*T3UM?#VivSsi36yM_U?@AR(o!<v|~)Z@97 z<~xzHOz<<tK8CnUJ(9d{A=<6RMQZKx;%L#nRAAe)*c{b3qGWvo<+G9{d|W^`L<z__ z<PXa<8(A)4c`aELor4UXH;QaiAG2BA#*W5F<0Y}w;)~XJB#ff6M(zk6nFlqABUsNb z?ll<pIUjg@=t%F|nt#smh4aizhYHG>=eBw2R_>#al$In{<3Awhr>1F^4g0P0Mn{{R z`yBKlfGJNU+}7ddR1-jWkPL|jueZ~hy0V!{hm37}9m~@=?@-Bb>_lo64qSO>p;hln z*4t7Uqm#>Ep^E<i3D0a$1j?nNRae0cjP<CVNoC&)NroLW(9*lW!KNqXILKZAA4);x zB(bWo0mk5QngE1C@`Xs|+WFh^_ceM3g~M80@AFfSm;7ogpE4O@Xolo=%XX(cFlt$Z z`H@H)D#MO|_wPUx%V}=Jh0aL%hU~Fl=}<tCq`pLQpDd~2j#*EoKq3e-%)(U24cz0k zPxh;MR{;o=3>MqofG=7okxE^UF>*#2WMZhu(Z=r)ARgFWeY@4D<aBMofw*_=_)^U| z^G3uQvG>TxKZO8NOG6PC2-0r$Y_2^qioO-O*fJZ8u^msgD>gZ;BWWaASxLw#g5Ni< zYyc|EYz?SqV=tHqNdViPeL4Pg0N7lR(K;)6O}8pJ=byrw%?i0;oG<X6PuDdQ1clJ7 zY^FvViR5Po+*PO=;n?p3Vb97)#Q<23f7z8~kyT0gTcPLksN7vZ>KS(qs^Ei~sKFm{ zIA-Ud=mjW3k&vo3_9r<5^PmY@NZpB*32gNoQf{{Y0G@66)i+r){;NE7^`_!a`Qm?C z0RPha_E;2gy!-sv&*@bqSYkq}8nY4dk;%<nciE96ZelnDW~(>a*#SsO5z+85_}9;# zp_?+!!5t-uM@CV@XX#Z<-Lx<J;ko*9n5*|NZp4=l9BuvR`jhpl$c_#c;wdxF&OvIV zfs~p-a>b#HsOgq(m-5G233b9epbq;w{yx<)GuxK?xIKx$sG3nA+IE87eaE1wWzH#V z4K0*J2+rniGCI{OhKe@acVO|yucaV`Eu<wBp~gn;dG)B~noMp{xSvthf(DDGE9F5P zeD@~|1wkS_ZXV%w`VxBo06JlNxlY+_)uTV%Trl^jq+c)OgL#nffw@0UXaUAY1BTq= zxb&#ypL%aOETfQWKeU)*KW%V~4hrMu2kG9T^VF^+kAiZ!9ldA*qQN}CoHo`xVDc%e zYLLG9qA$0xIs9tufJ#F^K4I5k+sEfs=D9v#kLKK?I3ssIod8D@LQ68UDLtEjKA!a) zip9ZHSzoHOhk$+RyNERtvSnbwx~gNJ&XRAl+Aopi?sLMN^Yz65K{H>2we8Y%2PNTe zlpju&1eaGTyJWX$V{S5}XY;AlGJJ@x+56j$+Kpi_`G$5@Vls-ya30_NdbmZmpcT?d zg5{)HBOHwI<J-R#mvwO@@3kDSVT#d}Q{~+#A^L^rrm-(%k`)3h0K|-+p{|t)8Ow5& z{;ZZTHNFP-7|*Hot}g1}Tul|zV<Af5ag&i+S5e5}B9v|^o_%vzuW7Rd&Hy}B@@c6Y z&h{j@vx?=`OU4SA&NGhHS#4}1wzbA&5ynsudJst*el;7*=1n3wC1YMa>9%@=B4cE^ z88|@8)Su-|(uuW|?oysNkg+%k&=PAR^{rkL0V6oUBb7ec^{&fQvIrvb$Zs=;TyQ_e zx$E0YXrTZF8Rr9_BigfYN$5MCO{j~DO~uy{4uA~tUPG<u&KBXW0_?aOlmo~f_4FR9 zx{jl<tOCR3j6UJi4^LX+b!!22Y9+aho_3Z4>)NHMbREx*^=o&8+}W}aag;yo`ql@8 zB-tF+j&X-1dy!t<<825<8MwFo<028j=V`B;?B|ltS}NqA-N^dVtIh1}dUy(oEcJ=E zvZEp7iq(jT6;+t;1DeZ(t;M`jFHmx7ZSpJSPBGfN8S`G!K8-Zh(V-!AV7b8crDxiA z=QtfItIF~yE_>6|w{CZFS^GQe7V2qQ&cuSC0&8`HISvkZ#c`AGwvaY)#cJ8#%_0q) z_V@g&nync&b<wJ#tj?C&NiE9AtW<{VE7J76QgdztvmMv~W52C<%#&OWU6>Kb82-7g zgGbjIClJX#-NOeynH*OwN)2dwG_gm}5ndzOOyR%d+yL3&ocFCOTl+&8JIO19oZ~sK zG}84pnpql8o+&W9K;s`=*Ji*ZD(SQ+Il&+Odf}A^bGkLtS7%jgG-7z<-cLM?_BGc7 zBysPL1Y;vDjt}QvX1bI=D-13a=Lgcdn@vhNEr-v$A#stw&u_03y^M{b)7Ix)?IT=6 zE^)I2fOC=g3b}KrnI!TpfKYJ9wn(fasm5E&EPyB(9UB!&ePSOs&9}-T3=@EPs#K;q zR#79?VV7`rnRq96uwo8BO0R!-mXNw2z+f0>zZm|tov!NMWD6`(U=|yTDD*uBde@)p zULv`0xFT}E1;EDKdvjXTqVIVeRj~=}xv@T~mM(4EnI+^E>4RCa31pB;OL5!UvFze+ zF{ZoD;z0qAe`?#ddG6n2XiDb<fxxci$d5hLoTaNQTgN1bBN<_yr?qvf6c#XU3x!yj z%M<)cHxK1So|e||gOEu$KBBUGS#oVOd%_#;+qOQZkLy(#CnS-q<NLi&gLM1XZxiZM zNHA{h=aiHE=_D*qzhU@St$10d7rt0=86>bhI#-L{ui143oHo&&%-@ZAPMLqX<GF!{ z&NhO5>*}aoBz#^fE1936KMk%BEvz%#?sYp#XYb?={@ML2+Pq<J9nO_~r7FI`0BB@9 zt2zzG1Ghpt*Ug^`E#}s(Emjbs>0B`Ealp-ea)ln=OKa66+o@6*^d$=Vd)Kc``=^14 zi|5ez{{Y8YulApT7Ad%qS_V&_g1fNYx~&%%Z7eWp7I8V2<f<tok$?w1IvU`0+o1)$ z#+M*62yW$%VCQPHf<G$v4*}RM#-VL-bMwv+M+$+{806KQa!&f5tKnT1+e1hU@tGZv z007Pyk@c@i^4?g!<KPkwMgRbh)}3T7V{j7R%t(kXHjIEe;MVZAM!JGqW^yND#xgo* zwln(GB)1{5)KJ*9oO>JQ0|0yf07{lJsHKVdc9n70uf1no*k0KAmvEu-$IexTaqc;( zrk55m+~;FUfI%H|_|Ya!&uQ?ku)ZYIf3tV8&1)XW&UnI)PClJ$rGVVs*oYkfo(wCF zq-Que?_N*fS?0CX)-@mrROgRs^(MNxNtzp=s>M{R4mY0J^x~q(31OQ4-GaP4sq3Dj z@}x0D(=_ljkrT@&-W_wC@yFp)O)OGD=fbSTeplP;RVIP_#JGl5hyVj+cXjpTRI~*0 zeV{Z~1gnH*8<ca0_4-x3d6FofbA;MG20dz$Y7*Nc$8d$wvGbyB7$gpvKhBGVyo4ma zXYOWv4^RalEi6$B$zx5vUNN7qwM6Jq?UQ&cgp6aHRgXCp1fVX*9YD#*sHR(Km`f35 zaz+_PL+L;hPat^LXkr)dV13$A<jm2}Y|hA>H$DE8`&pz%a*Rt6kDHO7raDv(^1~>0 zhmv_QoG0QjKnW%}@|>NgB(nbiOw-_aBy6Ve*zeGqMuJ&ZOO(lAKI)N-eQMRLq(`#> zM{-7S#Q;z<Uu|@eWBE=&&uSrCwq=$%)DKd49@wWe6Go!tf`KkqYaZUYrrW8KIql=Z zI`X?ur+%E!1%_0U%(z_d+EIYVB>w>Vs!Iq&cH$)_R2e&8q65?q@vQh~^J6mw2<$eu zu^-PhYR%uvb@KxEJWv9dJmDm9D9JnqZU@q$wGR!?+1&z`?VaDPHs<O$e7O!nAG|=_ z_<Mb6_YV|vLwJF_rNd=L;q=Z&KG>iIS5G*^%vFwf?NUo_P~o$;sl#@py1H32z6_}! zD{aSL#-8yNSmc9gAY_BsPz37~r9nqsxZ^E@kII+HpZn?m0DX_}ths-(*~>|A6^~Fe z*YFh{G>`fE{{Yax!hk>j)BORG?jSHSjt>mWnxIY0<g?4W9SWgb`yNjfJd7GcBxnc& z1ZSVtqHtwjHY<mS54@)Wyata@&$zm1N0o5`tFgc*?vegASlV02yQlKk9G!=peQNog zCkJaqakOJ|a7AazvP7pgts)=!=DGg8D7X|%c8{2?gA^R@cF#YRMRJL5tG-k#*a7or z^Q#P`$NP-H$Eab@eJZmgthl(IIS_Qh;OFZ|ZcXkmTgvP9g>xwb?n&1bPCq{8<88tv zst)_QnwsY7NJ=HS$vkfR#D6X-1d*jyYw07El>Dd$57woz6Wv8>sy5sgW*h`P{{X_F z^Ct?R+Q%aS%QF=?^#>hls=*F-g?O9;%JarWSR&%<Y%QCZ`;|B%ngYDORFA$mc1w&0 zknd6as;u!(Dw81IdT&GWpVNxF<*lf7f(B4H{o&jyi{Hp&+DHq~stf~63dn4&5H|Ai z=LK`psqSfHd?^{^GMo|Z?^+iU+I*X%2_`=A7#Q@=b5z=0jwHd62S5feeP{u`TZCtX zO&P`u9zj2*Dwo-0mlBy*&tJORyVR5FpuRS{{{XNCC^6f-e`#xD8UFxh{3@cg0*%b> z4%>;hFzQtIsNkL#w{XN9d+u+oL#W%qBrPN;K~cTA=L3)HSujr+2p@!D`~xh0yw_DU zlF-glX^#`Yr1@xY4<|k9<^8$3w}}eu;~?j)Q=UsSeU`}RSbq@6H7ndpGNcj)-oJP0 zTJtj*H*=r6kU<+GE+kQb-!;qJMKjq+E98^<S7WJLu#ibN9AJ0Ju6p@p`w>>oT>aG? zif!m9{6{Py5X>fKY+!XfX0-2PjT*~UQTw1m%YoSU_Qg?kMU{NgReF%stEZ~nLnV}s zrrt$l#(SQJ@HIzSaNBNYyLQSUDuX>#^fg&D-6)28^l2hrnToautlci+S8wdg*CFjO z8#X$q=y9K=ZC}|&=&>=tb_{dVp#Xk4&1F5wY~!zvpC#)hwp{@RA=3-fpT@aOQ%sf# z174{yH$_9(_peHh!%nidl?xUr^Jl0(PfEkMj7^J;ttuehk3rI<HZ*vav!EeMh?*h$ z)x>AM(_anv`$LmL@e4evR5JsTa50+wDb<n<P>5o;m3+QTfq4G_^>be}{AuuwwyC7e zFpM;|sw51&u`Al9Zs^*a`C2?fNW2ejr7}S{$EH8Vx;9gQcIPA3yz|3%j8#;Ppf`TL z;}z*BrH1u6KhnIMOjk9}slvA?qM18KDf<5a^{T)>C_Oy~y;*p-jDQCjHBMm4Ra>C$ zJ${wR8ML%IW}12fe)WMre0DyRxKKk3haQ{?gj|NoHhy0C#Xoe8xr{D7N&f)rRVl8d zd6$IJHZt9Kq?D2sh6547>s3`;{K0-<$>)qz=0Kt$AR`=c&*x3XGU#auyPZanuH4y1 zBUy68uHnb!n(gd-V=Ll04<z9HnHc=*$Ze<I{#fe7@&-WmtymUl)T!uC89Y}s>BT*c zxK)+xdqme3cVIZ1%RMj;sPwHXU1jYdNMo5@N`cR9z3awc^USIM#~potTHQpEL~8C7 zufJ-ZNcJjNiAC7-IebRmV>FBjJOWNlRBsVAvNJNX7a>Pbyqe|VNF*$Ptc~n?S5X$5 zH1Ng;5rfD)=lm-<@~(wKrAhTD>K9iMOC!Z2I}(hDKI)#8(Oc=JW4i>MfU*AoYw24W zev2Fc7?pm0x#ZVYnp?z*u>b+<-<sxiEOf$zrMb@9=;q-Wlx}X@_{J-yw6TiRMui$P z+=Gx!O9iZvx9?d~9S3t-k)uZ{#t!CwLB{}Aa-OE^4PoLkRf*ts^{*xPnRb?XB;dC* zCf>aV`4#M1j`(8;{I|CqtLBf4cSb+6TO(x1ypKxX1s5u)%Mp0N-1(zd3HB-GBPW8n z_2#y`IXS(G!4Lr>5DJg2bDkr)R07n08iS8<UEjg2FZ)YS6Wk#&K{}{CtL^F1(KF$3 zRa29(`Wx`8O#aV%DK~9+&_3>fbgyvugR6gJ=$AUQlB&q+3jlG9@IN~9j|SVdwxMTf z85<aYcX7FKpTm$V&we%d!b3dTWK8lxGaQ?G5;8yo>MO3D(y`=KmE?GyhXu!swb?BY zZjw8ulHh`Q0OWdpRre2vbjdWR;&73$P^DvkmGFA=gVPn@{{RNOJeoz6SJ!dwySbCh zmvZt1fbIN+eT?wi$z=?80Lg>^J~<fa$A8YZiftTqcQ!QZG`cBy3++y&cOH1;8rIc3 zw1(E|#x*lrg0cX|Bp<`q6^&<j>2D$x@~#vHI2gzr1MSwW6}h_q07yqMl)>W}JRa3) zSi(lAnnu%RH!Y3H^7Gd{tCf_?c2d{^%NPV4@IMN%Eb?4hEytL|1}ee0%V75GD?>=N zceRaVTu(D$F~}pYU&57%It?dOgGbb@uP#vGM%-uDze@HyEn7#`XWtB{R|6{jPCJiE z`Kw#Gl-{ke`2-()`}$Rj8?hXb;Tt@F;jx_e_B2@m^t`7pw%LM4=a4<Bdv@6~#?81g z;1W2^c#fIk?N3gbAW7O-fIE^*Hr{(4E70_>8R=JPo6y$tbp-Of5zlH)7Xzc;E#0Ne z899&8@qlwyZ5<v9L2|p-Es>spj+GpqRm9TX*_?(^ksm0-{Ay3KLe{DQW(0CR=~JJ^ zrN+P@fi2Z-<j<9s#yLEmDzBF`0!4;hu^vc0aaR$pcNmuq8iBWrupeC1S1<-)CgMX4 zg?{h9^r&b9$#NpOGB+!aQTUGaab0Cnn}eTDgYxFHmA=Z1ACZti^d6No9#TJ>8xNUt z&p6~#0<#NX{%JyX<T@!J4|=-@jbieV%BegP)0&<%S?7@o2aJr7j1p@-AQ9mCY@lR) z2h-AkEwq5526S%9bC4;<Qyhhv7m>pu=xU(1a}eGzx%qRzZ2dY_g+5SNp^JGvLv;FQ z{{XE(5zbO6RJwq!GuIr`j1fF!+c@Xc<0sR-BNe_45rhX%x^a-L`SVE>GsZSDMHGwN z9#6g~0il*KV|6NX&JJ<y(~(4J90*<n-lXKIIS1d`nIHyDk{Mn{`9KSmB=_R3m{Q#o zx9-_^Ds#{1C;|wCEh)}W$a8|+4`EkvCITXc4Zt|*_|;M&W4U&-bpoP=<VD*dWCJ*E zpil(zTF(=0X8`ra1vAN){(29{Rm)gox{J<{_c0`AK9yta7XH7j0Du3~`{A3*Qg)=| zkCoSrr`&h^>b1`K6&M6r2PBoqKT(>zKGI+<JI0tC1?YeJ^+|+;tK5JSJYm23)!@kW z3Re(`7cFlRH+G4(XYfDcRAjnw7m%_P+uL~lm8W+w#vxpL;2+f1B*tD8BC&5!03V1I zAX8+TL%vz2MCdl0f`2M}=1XNrl$pbP-#7mNSaqky@!Fi-IYa2&r{V8Sx0>oFdy5$c z8#yv38209X9}M=<2V0nd;~?xj8qc_r&U9x&3x5iMu;=nMd36hmfCa<yxyuE`XH6-U z&dxE_!0IYw0~60!L(Bp?@}Ras_U%KJ1#T?@0)F}ojJM}e%dv4JaTjI&H#i^Gqx(Wl zDU6f@A2C7weQB+L2bns&i4h$9!cUj-Jm#y-Z5PcF!W850cA|N=Bcw&#DBu(QYcAe- zuG&d=G-!jBbH+!Zt4Ikh5z&$#EI1s3PI>xN0t?wLcFLh6Bku>t>5uD8mg(hXSk#g_ zp4g(|0?}_Lb7XLPQOno~rh(%C$8iup-2>+Q>V)la9Bt;Y3yw%`KP*$61DMnTNf^&c zsdZ;@#B(<cIT`I*Q--9;*&MHk^@g^M<G0Dgc*r=;eQP5^)M1tw)G#v>^Do{b`CwL- zr)?UEmSLUPRrcn*%EsgSE5s%VK18`@RFm%DuWtFxbz0gS?oFP_A%o?eWR2JlyU>C^ zI)dWza>~)jr;f{<eQI4QeLZd9oT`^%Mp=51>&<J8ItksD7{ZB8S%+_br3BpPT`uMZ z;!+BM!R?xYXbDLf4&vAV<E3riS=!mf9h6r)+1qoEKmq>%>s3p*t;8ZWK3SDAhdsd` z(u;?g<vPMi9oN|=LRpRjcH|m^O`0odmKhL^PT};f+gY`R;PVI1C;>sva%-5nic%JH zg28ZAMhcH}-mM)lZL6}rzouLjhs3&IGcX735TU^_59^$Ett&g|=g~#<%p5Q$ZgO+j zcH~svAD-4b!4=Gmu_Cd`{Kc|9?%w$x^~>tfHh}4E6kcYPsJW5`BL*iqJ6z|C;Ge># zOHyX_I;$46`!2_~c98w>PfYSS#YKB%n{rV~ApQ_JBCcrfZw<6I_cE+4dC7Ch0CXA6 zbP?IL!Fz2eGsPcSTyR_1_xe=X3z?j3%Ln#(NJ7F}1cTJ$wRt|TU~lHRVG1hnyo{+G z52qwoxd?AAF1F7Ku#6n-UrsA2^y^VE5;#@mim=Ed`WjP8_BDfDPt31~9t<7~@eI>h zAx%E)`H_Nu)sxg`zIyh>Thrtc#T;R<M<nB@!8P`O#BTvx_{&F=PLU8Dt>j6!8zdih zJn}k^$33gz-6!o9ce7Ytkg{64!t3Z``A<E;&j-D5VyVX0KA$hHxl~TAAreM6b!;&F zYOT$iA@dzhF@b|y*0$>`N=X>NU%+>#$!Q>n#e=k;%zbOd<~^rL%Ttn)0-J=3umRlg zNBGjg3ggT{mB)I&cW-$aAy;xXFiGiI!Yieaa$Se;gMp9AmC>B2O69nwNl9FQI$$1u zD!(0=f0<8CGmMJP`z6ZAC3toN!60_7j>tx<wPaF1yPj$Dw6rdr4RtCV9o_N<-QV39 z6@J^p7dJ{$65|}<v-Iou*I8m?^J5)yI~-S}X^9-sutC#~YnoM^uFlv{S3F&O1%AYl zG{YFq*T)soY2FW--9WnYoe%ep?27exytra%r5IK`<F+czzNrd<E1_t!fN(li3li>c zEIRCBXu1xYVhH;_-NX2~Cy`wY_H)3yt<e<-=~?sLUEIiS7cs;>;*}X;`O`0T+jnR| zlkCT43bP!ZM{H3x<<zQhv$c)=J>qf_X%l$|03JcDp<#6m%yN0ap#gwX&QD|QSQ=4^ z?Son&b_#O%>)X@byUS}>p;VH6=0c%xdSF*1)tTv0h58J%vyB{HUNcWCk`)N~RC@dL zt4!WVbx3zH=Op9u;MMrlA}X(LMI+lY5hD|v)bn&Yr%fo_!o8Esjy&}%k~-s`TKN9} z;?43;;&zBW65Ijno-6A;PIk14XO{sXR&4uM!CpDlKGA)1aL@C|=b!d#=rT+@olD61 zeAc^AX!AW%_+3dO01n3^liLJ;JXhEs0{k=fZ6i&!mOMS~W_<Nl1YmpceXHb63rcSg z_>WG~A{iGmD--I8^Aq}G9`*W5;qh<byE_}3ZK2}M!T$C>Tzdi5zPa8yAD&f{=4Y?# znx>cG_+V*HZUSQwo>XHT@x^#whx|FH_~XSAUEC~<X9T%WdPq-0pKr#xT~<5K6Kl({ z+3a6yO5kHX0RykCeN*57^gs*0e-3C`b&|zy>VP2FNc+c*r`CwMIaubF*BzvIe$PyX z7~_pg6;+Xs_mpw#+PfLY*?flpIO=eEW7{0nr_rT?)u)83Wr4_S(_c{*(15Bg8OAz& z!~T2Mp%rCgigI@#l4x}~Gi+OT{Io2%1d7#)@@T+?Eth+#!voxL=~Cz>%Tqrhm~If_ zSh2=`N^glJT_)2p{{XHh00V*v{+{$$&gUZ0UEE&{JwbTMCI0~4^v~m6Y?3a8rjMHv zTM)qgc<cDqSBCCB(Qf-wY^>igCnT}-;;{T(deUmJXb`z%kwlWP=kE6XXtDy0xAv{< zlidsShDqf+M2Wl*Gv7bbt+u0irNFV?+cV0*v6A2Sg8=o%7^+bpvz16orq(2JoOZ`O zYUGk3kTS6)ha>!&EP%AWD$!aH%Iu_(kjUG!?ma&m(N7jw+^JV<c_SD*Lltlb)K-1N zVO0XV<Y#IHAJEo$T_QpfYklPM0|C!t(-gq;{To}jw?lKM+-!}A+Pl6{?T&NxuR^i$ zv1W~Rg==KVQ#LZKj{JJp&S{`&s>WEF=GZX*0G4mGb?M1J<W}T2y2hDqklcA!meCWo zVhYF=x!Om}4l(UfIQ0YSOF5#rwvO)7WRp20Bjn?@c&kz<b=@dz=XbdLE5J0(cGRqQ zT6O1@Am!0p9Qy6YO75&YZY}{$Sx(Xa0536&6&~Zi(y--VcaN7`q()yw91~Vyiq_)O z?GX)!uqU|fRdoBCd&t&%c~)Sf8-c+Y^&d);WL67>a;nY3yJdOw<E<vF1h&##Zjn`S zARz9}20too$C%QVmve9Kr}V`>dwGQZbS_D3cKJ?6O0Y!Y7@#BuQ;ooPG=QaPB%Vw8 zY|42Dan1*@sN`HnfTZr~!>`w<#aPrJhB<`MxL3f=4nh1WfX(G*J-CyC8=uS4fEg9> zg+xpiCdLB{l1Hhl3afVqn=*p^0|U)f+i^XvLY%J~M`QJ=(se?p%*((e5u6b~4U-SL zE&Ey_k@Fn=c*QWtf-uDc5->?D#GhVsRvIYGt#pgOaK;mZRc+KP(UBqCxdjRJ=hA>B z%lSThO02Q+!x%WLGffQk>?0%syx{byVV>Mv7xM$);git)JBo&Q;y~FevCdl@6Q4mq z5Jjd=P&5|?G(3gQf5N5y%)S2rpR@XZjZ!vu5R!v(FI5bA+Imss+<$!jbOHa>``xbZ ziy2U$RpFE;QT}_@Y}Yc~{H+?dKY2zlD)#G$f!<5R2d6mn{{ZXOe430Ce(EH6^Tu)3 zycr&$1=K-;ubr?CHjYXE01C@%q=6dV&o0OB7Uu)o-mct5a!zfPQB-p7Ybr=&QMjNF zobFfP{urqN?S18297N2(VDBA%l}<@5XJI2Ou*icrT;S8C2oOElkn`B!3Xna-Ig&O3 zUl`m*ar)G@1yOP@6_7J1`=mD%$Fz#bF3BZQdVt%7M{>~KFk7-o9SGolo$0D0DzPdl zUsC@7{c4-tfZr|&L7q#Y9^TFTUfqACK;PMBFo|~>$ao<~U)Gx}E*#>>CI?;F1bXzQ z%PK~INd!B3WQ-ctz(pd3UoYjE2Rt6M*`q~>65CEVj~rv`p0zTIR9)8C^B8fCaB7v- zqYRjjZU`81d!OlALM@rfO)&n^w6!RrGFgTW4+He2nrPX#pgl8Oywk#ru8)kKLF_%h z8oLGBJc0=KWRIIa!(B>|4stgve2~M;0J8!YKi(u(+;&fJ7;IS^ZU#FarE}I%zuEzY z+2-58W;<6ut#8F1px0Ziu7$f`?u~JfeRG~i;aWnTrgEt*j#F9Dg1W(Un_c-_1KPYx z#hxF!y0+PFtkPSSbt*aCp1*~CC4SepW(B>t-eU!F<A(fuS2wRM+}6g%6BWKAZL%tE zB>HC~{EE_a*)ex#hWJOsV)a(bQH`RK+=DNCs{FYFjB-VKlzPOI?}}wX41A*@&&&1W zpKAF_#1>cH6xCwaEM%T*i)k6AR%|TDbCn6tvFqBsqeAgrmc6Fg+}I)V-<cE)7~J(% z_s_Kvrmd-~<wvGPA(hhYp#s`ZmpR-(_2@q;($pfdk^~nqfI#^Wa!02=;<EIyHG-@% zuyk*jr(L9;E2ptdI(g=}xNu%Sn`!UH1_y4ne27+0W61A>Npa?BXzo+*D|F3f-N=kt zr;h`Sj+r&;_c|S;z|&iyGcy(#ZMg^Vu4}{|9kzxz<ws|gRC$ZkbIyMbl`bfx+R+?B ztSc^U;t{DMEC<Xu^{$rk*Gkr`rO{>q&BQ|_9*&(Dd-Ghq^1EFN0U(Ve1Ci8XrPAe# zOm>k$c4jzcKX)G3=QWJ?CQ^4Y<F(Z^yPItrR9Mt80y*)G@Pm(iq;cHWrfD}b#KtRi z0TX<D@tkxy@0w-KfIMZUOQ&4OEu;}|x1G2s#Bu4z<6It@uiWXEf7w?+t>&I1Nh5ws zU>}`-ya?w#x_7B2?DTyHOp<n%=HlbaCwU+Y?H`!W(~3wuH*0xlQ_c}i%)~T{%K&=} z@%h%~hkXvBa-|`PLA|>(dVMonQ_G?g6Wqyzf)Fvu1bqO`Xyk8Erq3q0@Wte^{f63P zxbt$Xjq-q_t`9ud$G@~s!nW|VgT=Z`5yZ(N!+i?2<FpK;a603!9M|YY_N5J-lyd0s zA(lc{&2hlbU!nJ}GVvCky3*WR-&%{GwYDZ-y}LNZ(s;-Kf%UBGVcep2;jvPLleziV zq1}lsqmu!Zj~sE2t#wZnjyHk9{D7_Rh+hk?{vF?o{W)Z93h9}YImxw<oa~dXG65sD z1ys}5&b1AY>$nWBAmkH}eLo8M%v}cuq51}2h>ciU`j|Hsk0v%CAaw?^ZtX46zj+=F zdK+oaARlELj&Ma~-&#C*N#kU0oD7f7xaQh=9*k-K05Qh6WQPL`8;&~EB0F*vv_(kn zILZ3f(_6VxLHoY-Xhj%>QPU;+8jovj7<)Z#h-_Zo5JR{b9AxzTYti%@o$9%`MnxP9 z;<#vZ>qLx(QV*tbYf{rqOc5NWAf#~P8U1StuS=t9Yh6!Qg428@UPYQF>M`s1)N5^~ zTBB@D%s+&;1MsRERg`ONJjoocKZUWH?DXp`IzODmSb>bO>z~57xsK&anOwx#CZ=PR zEq5e*ZcuiQ{=IKmcw)$uy0~98)bu#~tEjPc3*{<=Ry`Mh2hyz=!ZU5gJA2g1uBTlL zG;OH_HjqRJf;EfKsK?=2%7R$Om^W=4A9}8_5g79b^~P&P!ZWdzc`eV-R&Jx{b-FQ% z+?UA!6Sq5f{OY{p_h%Tu8S9Rfw$|8OxyJA5T(-4;1)aNW1RbX&^u=8jDa{;p<dj|6 z;@=Z>n5}fl5oFqB8%R@<I{yHLd?(^t*1fuDlwnlhpKkT^C&dpD!zpBplEWhczIgEV zfqUaG6GNn0b(zD;#GSE5oUwj5829AhcdyYhno5j0nfb<Rif*Hg&u91*;ERcFzu_#r zK#`#<XskKssrnzKeLgh3UtHF0?DY7Vgk*13T!g?so_`9)va*9ueK$+eljOIJcE&zZ zRP<sw#^cYo<6l(xH%GeDts%YnBe<0AkPc2Uj=Wb~B<CxiF;T6}Zv^NNX;7H&Ao3kl zWIArhZoS2Nc8jIiGR)A=xf3LuU>~6ES(aDRNqQ!p79b#Ac?TW(R*YVIA({nP*s=M3 z>L0HM73fo@lQ?Hl-pJR}qg!y-D#}#kfX6-i{{V$)PbxJ2eD=j63&TJ$zo%?hJS{JU zxqESiP+CPi6Yja{J?p8l)TM^nA3e9Kq4{{o_r-4`o?D9(Y0VSd!z)JPmUbIS1GYc< z_2swv(AV{eMaoO$!}5ZIJAgIN>w2@?{^|u}lXfyN0bi#z7LjN6Ns{Ga%%uF!{5a}< zze+}JnwD^`k!3cp(XvMxZa~}|bkFNt<S_=8E5=>YN=vhzPtC|4)vxg@UnUC;Dj3zb zrs*=<GoQPV0mgdc+PN59ym7*dxL`BmJQ2Xh=SM4h4leAsJHtHl#j)RL-N7Rqf#`GA zou}9WT6r5{L^Fe(#Cuh{N$;dCuqFbo(VSzb?^Uj*)NUkmyu>5GVlhRU+@_5Sn~Q8K zMCBZhmwCl97~+`XNU}iranyb_jSNt459Ktn6ZhMn#<gDAF0Fjoqzc2J8RH*H1VEPe zkxI=VW>9`qIU=^<y%YZG{^9MT$Y83u<P3XPIUDK<0!K895}w18`qjzB&~+=nV*m>r z6ZuoP8&PSRZQf<JyOc~0RHS&?M|K(HdsNGPJ%`JA6EMy^#QU%ASG8tKJEV&oJHTZh zH{Hce2BC2aAGQN<y#XgJ{JK=x0odrCCl=Gj_VR&~nNS>V1KYRPn)huc>f2qg5?c{A z_Q_H{c!>5SWP9;nH@rFx{A!Zfl$Fm84$=>?^{P=`>i!hEk5smjdy6&u!*=(G4)_?} zJ+Lb|LhKK!F5$nnK#eeX<iNlfBe$ujqrCFd%9Zx*B}PwNd)Gan_|7j7Y4I5Li3w60 zjgotg;EzLHF3~it1<EMokjJ?_fd2sN)Hv7)CYsyq`~=)ju21Jst1iPF5(RDmX5@YY zj(-|!Le|Z+Mp;*>QO-V;i*aV^6*m&U!y;`t_3u#78dk8ph`>rw@%0BC>Y4J=<Oj%S zLZ>5)<Wr@cl&M0ak6K%&Y4IS!gMsV*6o7@2Ye(FG$WH-r$363!mURKuqud>r=2jRP z_B26lz4kWrY;sFv5&3aWp5Q|ww*LS)NAlo*`j6*87oO59fDw^ma6bCrZu;|7t`UNy zQbYzni#fp^`%`Cz2S#a^ZUA859Q``d?Ij)&J3KA`+It!Rk(%B^nVc(uf*YULBAd3$ z{(7(d^Zx+XsHU?<E4u`R10ePNDlf3b>Hh$afAy#X|JM7KaX89MTh2VL2nt0&^9#tl zmSr0_U<V-mXj^$D9%Sg7damcj2iB`y&I%ad$IIDpYv=BIg$tIswf)pcT|1tm6&kP# zC6Xx4d#hoG^Q#h1GDKl{08@ZQGEQqS*w`TZOcKesbyLXCr&^Z4mTShi8vu<W9ZZLh z<yp5E1{1q#5(flwyM3x@Cbw1JIA9M97oI8yHo=tJ$G$Q%xSZgBjZMIPvtQ}}EJ~<^ z{o=Ui(xJIqNN09Nc9A$`KXe~zdoP;!hDHhoa1H>f*AoD{K32f%+?uGy><Z@k>jj!q zp1_WS)2(OSGtTL_le^Q3YlA1w@&z3+jMUc9{kBmZt|ejAV2lr3*JLSQa%VJ=v1ez1 ztl3eGpyxdL)p_(cb!p*72a}c@9^G?W_Li3R2&VAvmv<4SSdov?x+@6&(QhOs8CQ5F z7Iqsk?_JdBB(z62wp~vUpG&!tb+%|Wm2O<0kyPjA&#|leb(2|_v-1*9J@92JSUYwA z@s2CE)-;G32b${So^a9IobWw*WL8zsoZH*P(X<gJ3}rwV{{U$F;<V*!L`16g3|_$! zTg?HAAaHTkn+3Y1;GST^0}546;ywARUu=)=M#5Pb4rIwW9qOvyOxJOnn~^7$c!@?i z^s0*PX6JoQfvlx+j5oYwbSZ)f_sOhV+cX>gLfGePxBmcIx<$KtmrdcV!Q+<46_<5q z4AV*l)>Tg>NXZ|aQw1h?my7%(r|OobeT?#`#!zEu>;6rA%`e(~8?HUojFSD9RtDw~ z_f9+geeqv)>UwRI(1z32=jG~O0T}+Z;l4HSmE7|Est7jmhVsB*bL-Nfm40nmKZpE6 zWU`L>1`|heMoSFr_BH5!W|*@r%CHlV0Rw@P+dcmPD)|!AS)bwdr0IBPxYMO(6T^X( z86Pt8dU|?Sz3F<zobW+)V{;wU6M*oZ%nxFD!Cc_d8f|EgTegk{m6+TB#AjlR=k*w> znvJA$L|We@pvx1-e|pZ+H5(|TQp7W(a@%?_91g#&Z{LsHMtsR8nqF2QK2A?lO2x)a zyEDnPzYpBSB$8TzH<^YmKZo1BVY0R6$lL{zEXT(g$E|x!&7Hs6MT}cvmQAJ90!DF~ z^37jExiDTmi*J!+WAgBFf2CzOUFc~j>TKB?=-DQ6?y;(=MK}ZwIX{kTKgF6ec(YUA zpu!;3El~noFu`?SzJ&J#bM+O?+h5M2J1HEt&=y2&gShw4wRU<%)T=Mqpt><^V1>WD z8`J4YK`q6~aGFN0t9q9=CP-e~g}8u^nl=Xt2;-;(jyhMgqg&oQdcy=njI3vz<9OiX zALrJvz99Hk(^FXWtpZ4fw-jv76OXi_ZBUQSH;&EtVz_S#_^L^-6G>>Mf>mMmm?c~~ z6*(Y|;GT!lt~xsoKNXK@x6vB^08gGcqu3Yc1pfdN5^?mX?R4wAdv|R?BM!fIFgk(j z%~`e9?xl<UA7EpJ>5sZ<dz+cBE;j+@763lZiZRbMtJ=nrin+sTsif<+hey^dg~S(W zCA-^@GbnC;W<7DYAmbvwC-HBAZ+;u<aa&BXK9_pLTFegS0hnPXJqrQ%PaFfsdi@F2 z?6mzSPKJ93*7kD2n94bf9;6)g?ewlA#X5eS;(J;2opRPWt)vKisD4QS9PUDMhaSF} zuO}UZbNEk30grU5rE}zcGAM3g^2pj_!7Qg3&Uh!+im@h^F`vAx)Hgq^bXuo_H9riN zl{0&#z7a2!5>Y`U34qShqmHLFEPh1rtHH8G;09LU{{UaozF!fCsRyb09v=}3wop9I z^TM%BzsX{SyJNrSSW{>^ji3zhyDIP)E=TjPXOb*%pdV^zcZdkacDGP@73VhZsofN; zc`dj`j0qGgXB><g-L$iUQp^zZ+?@XaO72arp6mqBD}&B<=bz54L8a+)so4;W2I?Dc z_|!^IV&ajVr&~gi0NcHRJZ8FBLc8RQXVcoR+vz$?Fo#&#ut(l(kL6t~HhOxI<=Y`R z9nbhyE*Cu-Z77(q#WvtA$OQ584&R+_!=qhX49dYkAA227=UOpZ+m{7nRL3>991%vl zgoma`2C|hcM(1atn{A~*5{BL##|I;fRnrQo$^~93!|WkhScNAX)^4GH4T=)qHV0hQ z!w%(SjH=g3d&MnJQ~NYqiw~6jdm8!Q$37|{U6O1C2P3vCd&b^7g4tw^eKU;yeESOd z1I1n^zSnQRvNWjVxt`^MtZ{tILw($1^ZE+<JQiA_)S5m|9hp0|26GzC)~n*}Z8V)K zBRtnFv11^Oy#3-wOjp+40r-0lg*4c#Y@=DPuRv?b@GxC}grDyEjPh|&d@J~5-WIe` zA_?`&wI67iM*jdPwlJXkcj?W2f8cKd*=fQn`@m<BOJ*143CQOjwe;!|l2M8ASjxAX zY|rp7gS1h!U#zcxAS@Y^3`R%FpWXwvOjo&RdPTL$$e-$zIgMIX%A>9^xa4iftcTO( zmj2sN(<CaexF$Ekv`0U{JN+xWm|a?0eXewkp_!j~Q;^M&a_T>egT`ynqe?N3$CWzQ zdYJcfrJcvv<&4JZgcC+SMhBdaZY!X)w!V93be1+wH=!BF^37$#ePyg8!#v^z1LerZ z0QC0utCn%LqN7QgV;tv*rirkQ2SVPTnXZ`bb4pt1ZCgtvqX=(AcQ)OAW%-wm-qp(M z6I;TKcWiewxeFNksw-OZ?X>t!y~uXCgzRJ^zhV#Z0z20&ahWFaRWQ<XgU|S4w2_>f zF!bB2ST7-)RYrMaZdW@(uj`-Ey+=@!M@=#BZQ@}RcyimmLEERVHO?o8W|l~<1aP`_ z=aQ$V<6PH_?$_*T;ms~kfm}V#4gn)40-A$r<XphImGukh@2^=|<(D8Q%vDejgB`dg zt}3)}v5U+k@<6PuJ4rUoZtdG?mnK6Y>C=H-JTO}s0JwG;86<t&eQRjO$Z49ESGIBK zvPRHKWEcl3PJb?yS7KZ&o1<VG8*rsh(34sUY%RZe4C)~(midX~b{Ohw5n~#fGeYhW zUxs!o&#nzb(z$oJZpPN_(pzQn!mj|gPz`6@S>M_!!5pNj^T&GImCRPiU_0@R;ZE*H z8SPj)g1MbuTcC-6RTq*^zdTk=0e0I;Yruiji5UIj+j0IiJ<XfNwi%sSx-Qesed)HB z#DO9cAdC^yy-~W=XC73m04~|cQOF-!Kv<dOwNEN}6&MZy8U1rs?XBavWR7XdG>etO z_v5~69(#wKW13KkSp32Hz@jg)M*FTOc9D(~XFw0*#Q=3s+RF@u1iQ>#<Q#q+)wr7B zrOJ@2>lwpm+pa$&RV>3!<_2xew`e^`Q_gWsm>Wk}_A-&wgW7;L(8(J-)(|bcj6vB6 z+N{I)vD0_1E5GpPjAFc4>@||n-avM%IorGN{<*Iuw~^j#lFB~tENgD=YborMO&U#b zpLS46x8iVVIHm{DOLWo@vVi@=BWI7ot&6j{Rhuij11bsVd)Lo;KZ|ug4S^l#Ni72? z-kE0Swt3>c-qQZhRk2&mIIAH6w<+9I{Wv-HtY;N)73Pvb=Bt1Xav3^f@ul+8Rb*4o z7$c^AaY*dO0?+fH88{nT>yBxW6^2MuWGgAlW3GE+wPw&3?yn|9Ws$ZZ+5;%geTnQn z>e{qz16#RL<N&9DNId7>tE&W%M+;;V^Ml=e`qT#7PI;P0&Sg{ghH^c=lz_K&F}7uj zGR0kXunW5fudlU77NC-eHY1&b1TugR)0#J{t414=K4Fe=>s5|p5zURM$RrWS_sBfZ z1d?0=-?{)UIu;+}QTgxw&;9g2#-+(#5eu(S+ZUQQ$^QU6PyO(;0sq(gCgMjay4`%j zc-todf5xauWU?pvbjxd=cg9cURxVoMiSxXxk`E#?fIY=kl%$dD2<SVkhjRWL4r}LI zJwdYR@Wj4sdX!9Y^6xyHel-pm)FU`KKQUha0IecLXKu?hmhB@BfDDR;;_c>Q#@!(Y zw$d9FHv;UjSi+)H5dk?nn{Ypntih=0_+e`zeB81wLHv(FNU+<*43|+#>$OQL8?WGN ziGLH>PkOLiK_qBS(<w|3&b6ltf!DR#0(^njw_Mh3#H$`=c3uWbcdD{^acy*cs77{^ zgY13j);8+b27BcUsKBE2-{=pub<?C}I+k`dwEZ&UQ44c0`z!#A?&`zT4)xbtSpAg& zm6j-9g+v*V{{Ruw@-*u(B=cVGHDYn)4uUc4JJ$8J;#;9qjaMVAgL8fZ99N+VX(OJb zuE=c~TRTUIESezAm7ZJ|9^Ry4sY7iX_iG$dx@7PhBxLpc>q_PsJhMAW;h1$f>GZ4x zy_Ok#w7M=I<payX2iWt^;ac3qqh9w&Xsxjnh#Pk(d!MCIy0n%4aR%_f0>9kHerr-y zy|#uMgCUb=Os_dU{r><e(2?bwMAsmyzuph>jC)abO^ydvirOSd(cC#5fZ+ah!nBt* zGl@ZDB%Bg5b6&-$TljoN-c7rS2tPP(hw-m8)@{l%+`}1WKX8`Hf<1VpahCMmPf^q* zLnLv^a7J8mKEBn^+gLoVu}^CbPD?VL4?e!W)y(O0ubHs0-6G|}uS1c?UTeFwcpgTE zEwOY_mfFDMt|$X5S<r24cFkt;`HlYoEY3n7*Y&PPQiINnMLbMHazNm5*kh@$PlHaK zU;QfN+{F3FBoYU)tegEh&KS~ZR#g#(+lJ16N>&}uj=XK*D|x=rG<LpNVCUupw?38g z=Y~96eema4md0>p(WIAiMpq^S{q!81b;$S6YW)k=t?Vx%*&`KPAD%_e;nKb|_?h8- zJX~B{$NWbgVsJpg<a!_JMUhDM&kJ~x+g7%1QaR$1%II%`X!%|{A4BYG)AXH1obS8= z;{_K!fxQpmU!Hym*ROmBcOziSrCkzkZP$e#^wCu33_ZH$zPr<WQ)hXkLv>>?md@ri zXk<f@cK-l<e}RWgWYn}$c0J--+XYz~P+SnfHjkSFj<_|E4}=~_P2Td5NDC;=NhE`V z)3tLtCZqO?0M6gM-gX|j2h>+-rg*E$wQEJSdE7TRJOk^~rBc{ZoUggUc&EX7mCwy( zY^q&xyY8F;p1*}~P)|LA!32?%b<fP>x}WD?NnJJ8s}yXkIT$J!jlYgRrExwa@I-HJ zx94`|BXa}thV6{=QzfX>osNG;z0++p@jQxJd5dRn-W!PN>C&^jd*N%HNiDot;dp}T zT6|z(Pwy@i^1?&>O@YT9tDn2Hys+}d=2;bqUBfGd{{S4<bESB$-p0g50al*t3m`el zw_XkoI#k&WNvCs~@aKsw^|prgWoRL^jgioB%P$*)jycat^zAN4%<C<@%8H5$k5c^s z73W?k@J!?{jI<<UJg8ds(k1{-2U3Ik<I^0B)^>s7dt3OL^GcacaTCKJA;YLVka}%A zay!#2+Q{|Q(jQgT+Qp{Y$(akG#9;u>KU0DA#du%FkB%P?J`rlsXmMLyc#~e7A`LFl zkU5$``?#Ftj^^x$3uEYO+pP78G*1X?nzgHWSGHCXu$%zdo<;cx$Ri4R`ukVHUJ=yq zd>?E+C1`Oa*TjDo>J#ajKC5vO$aHl88_U~xoDlaOSlbyaJh1JIo`IwjmZy2C{80U( zE%bHPb-&r->cd32UCLvd%eZ)31OpR(^XCLIgAgsj9)~Bxy6=O&B3j&P9}WK3Z+T#( z>V6*7P$l)ZAQ>YA@5Zh2xEN(5WLHr)?8m8iu5ayy@s6;%^rOyuy+-a?w@MF~UE9nO zD9Je5-zt!+_i}twTF`uX;ZG53Gx&j)-{K#DVO?)Z)#4?Nt)eL+SR{}r^Hn1ym9S1f z)&^jvxN(>jRu@~JL6>9ahkKrt45DxuKp>K!kK!xCW7$s}*1JNCsy2hzpF>``dEm_} z$DS6j_=n+LmO6%^YYpVKsz|nxWkA?bv~7InBo2AtSC_g=1;^PgVR)p7fgmHlsPz@` zc&rslQnNnO1({K+9rY+apV$NS9C1@a2bJ<7h8^-NWtgc3IQOX-Cp`XG{<R>EGspmE zwt3BPTe<4OS9%rVjznB9$_VYvYozX^Hzy!wv&tC31+sezm`tmMCuzqtn(S}wEuqp{ z+(zFy8NlySOLrQne1+$V=1t=&5`cg^j`hoGI^(QvPXT%BTTsJFqtNHCicnW<?R6b; zC5(e7IO77mKgPa0hSmZeTn(o=Ju8s$pTwJOG6KG6X$L_W=b`Fr<sT6ErvCt2WdWU) z+o8!Jv%tnV2R&=)Fj+g&Qa)oFomsft=sZ*7Nws;6wU*~;SVoGu-0a62SJaW3`ZM6~ z!wa1SUjE__Ggi6^T2?swEJu*4PICVIahk~ZH}INk7q{27=wf|h<ZpsDUCVD7$0d6{ zIv-5e=x>9(Bcn%Q96%#Vz?l)gW+0GvcF8@x2Nmu*Ek0GT^A)jayPUs+G#E6cM7NLk z(jDx~xgouA+#jua{hWgG1&P%noJqGJZR&cF+N?pMTfux1)y#4Q-3fkFD8M=Cj!37? zr(8#IGszZOPvTBdxatl#=M~$Bhau$GtfcOA_ZnW6t7)xntL5HA!Ez5fhe8h>Mr%IG z&hJlwZSI1)Bf7eu;>jGJe|pZf)Gpw7rM&r!J5`!B=*!bSymzVYC;NKcI!MYMOaewT zyPiInu9p|k;BMvHh~Sh-9le*7(U8I4m0{=(27Z;2$R@f;;D0Vf7-U`-j(vScN?VJ~ zPS44T{{WZ~fl?omFnWws8fyt}WN8$u9Jn2^)34)AD+n&u`u)zKb0KyFDC3ikgRcU# zO}vuCe$i$_HUx4HTpGV_`g+~RYjTVb<%@p_9@SICdXnk(ixq^99G$=^-L-zTrLIhj z`>k5%L(=}wZ7aO|{D5a^J?ntDfi7d2rjz%KI4pVG+~@0ERFYlC3A?zBrSkyXF_X0S z&2z@e)!s=QELr3XmSg$~-kY){SoDOpwp)Wgn;y}*OXP;Ft7&2<%8+f!{DHXyarCMd zwiZ%Ih^prUB{P6e*Z%;mS4fiX%OeQsBP3@W?Or<6&Q9QH8&L|gI7AFV+82O5diAbz zRJW2?UP~K^M%Dmhkb0h+aa(HA5G~|%aH9-&VC4H%JBvvpc-A=tagV*<Iqz9YPUWqI zjpKOXb(o0)fXWUA?|!wRXK5U9w5A>Ma(6h$KT6An(%f8Orr#TV?SlmVJn>sUXfZg3 z+BKRH@{BPoKBE;E7~izrMjqXfVu<n>CsX;>W}|Z@oKZ^Af*j+1c-#3?yy@qc?6XF| zsU$W?$Kj8ETB$Tnab|bAZux=l^gXBnQuj}s=gH3Fo#bE+KU#_uvAC7swNkE5cjqab z`u3}G*=gojtz<GB45Eg>2hj1&S+|1j)sbx&#InB9PzJ|-ayjYkKpLmawm<0X9H1MW zIUxFRQ_FE4r7DS5NXZx(?ni2sZf;=`#WlpfVf(vPP#>-ZQe}!}#mtiL+yHg=_Mi<U zON(U-A>A7fk$}mj`Jq>E#D!1Y_4?5+ofLB-r1LyYji|@&oOAirD~K%PaAPmW%*9)g zjy*Y`4W`sqaG)Z^CO+W%r|Xegnq|e_iF_{fm)VjUNV+Q@LD%_LCd(9&{LJpBkaL5_ zYSlo-5YFX#V-3gy@b&LW#V|elPu64B?cVCqcHPE6mL}s3*N;*7R;+UQs~oSA8=M91 zI`{2fKj7=vveqN>CN3sChCjvI^v!z9z%9?6y}YrUVDK_?j8+vAObbrw1cqpGSd48Q zaaHCBO37~K7C(7kf@xajLQtn8u0X;3>eTREs)oY8;kX7FAb-980M@L)Ay$m6&dS6N z!>{X1RD1x3Xf2lBGCvMzXJn0jRSoERR6lLHiImA27;}Swc=w<S4Q4^av$Gz>TWH7A zns3<C{(C?D{Qm&!RIMyG2h57*AiaYj1b(!eSN?vX{{X&C0Du3``x0Wwun5)m@HUVz ze+=fLn^8oJiP0AwbJrf;^`G`znFpB|M2!ap3EhuOcc>)a6D#A*^$s!qHS>2pK>J12 z;r{?b^B{}+-lN*D2~;k|4(y|oFH=z5-3cwv$m--^Z8%jGQaEm)ggno^2<x54^Qwx} z4E=86C~cY4V4g6w<o9!ni=?%>l*b!57-rg;JvSV1E4bA!(L~|;AsEeYn#3E~oX5CI zoNgZ}9c!+V=x0(*Y<AGccI*9<sd;hSq<OTF5AJ2;ZRgh>^=`{hdx!fNk7TLx$nC%k zpL-ue`PZ1ktqCk-(e*O$&NiEQ3Cf|t+Enq39@(zLK-JS-uxm+({?M}+kf|TMKi&X( z@$2=kMkynnadLM(V@R2qL6Tf$v5aSd@0#=pwD5@(kY&SxyKW2Oyo18mk;@9g{{ZU{ zkX4Qjb6&NiT1hm8$dQ&lp-(@BYK-k|8BS=HHpI8`2*J;lBIUlBtcSgd*6Yl)EIr5v zKc#kayjPMmaNFHJ&dh-1kM_9Yv9I)&DwmTew8wGFaoEvk(3ux+wM*u*9!x<cWe0fx zh92IAt}5JL+eFS8%t{Lu!5Q|hN*mdsE3rhXa(-}6Pj1z<qh8A+3^$fecsbjgk5Nz5 zkdZXY+m?z!oD7g#U{W~egOObRuc1e%S*$kEff*%tL~=X&;=Awc#JHK}kq4MmjH-OC z_~xiwS%YZMM{IExSdLU@)_^=7D6Vu%hKAy3)FHz2^J6u+Yk1Fn<|JNXu^&5(j-9<~ zrnwYXS4{-yi3nVra@hyhiqX)-DnWu@G%pIhfck%00O{hmnb4&0qN?PEjGVXlQ(j9O zl}M&rfwW+GOS!Y|2^>`LN+P>gig=YzE`((6tz)XIi56%H4Z^y|fuCIFsw+@AO-Aa) z;Wm*86;X4ElVNV&{<ZVRjQlyL&kWLB#*Ae9s;!p(RrIdC1<ll|{%W#DpYSYu*9oeP zH71CNeAyZ@8!3>!^{axl&7sR}PlokP0?f}V#R+9!lma*$Ru++~YMu|cR#a!Pk@r2g zI0M~A4o~G@OKRR8vb|T9DZI6E+p!-C!||^n*K`!MZ#T@`f90RbNIkgpHP1M{hL?%y zIxe#or5&!Pr%tbH!psT5K*ymSh_6ww)~2ypX0j1Uf>drC_T+l~YvWjkuc2K30BGq1 zz{<e9Z^UwUIbP?A^?wWa@h^~_<hU$kg}7xTjP3UPD>%1pMavhd^p=;aU(0d!mXX*u z48sEi^aJs({(F&m6}(!LZpgrFg2M!45HL76ub-?uLk+wFWDg>aNY4Su;E~%E-DrB{ zvpu=H!@REBGQY)+PhS3%?q@Y^jm>*ahf|%Pwgcp10RuP(-yGMEYI;toqE7cBHuA${ z9>=yi8u}Ye)L>gyw2o3SyZAX7_p7b_n{#Bbsw|QKca~iL09px0i|Ae{h8B({jW<F% z%EgA`-);v_tyb~ZgPz`4d}rYp(Wm(&)L{e<wC>JWDCo=3V00$BtyjWUa!)3t35DDi zRZMNj;1AZhs~tyL(l7kW^uUlTXiD+45$}%F7MyyV?wR6=HH`yT(W4~8G&4jK?!Mwx zg0^}vBZ~QsJzmeln!bgk>R=nc2kU>@amMFz-sqaZ@{HE}475*$k$EUpv)G@dJZqrG zXdA{FLo635Hu<2>?yg86@}n5(&#iou@lF^#U!u)Ch<@J;vOTMz@}{<x*cS-h)=~kK za2S9_-f|t7c|`qL(jvlHC921=SW-oN?(T4T9XP=C$KhT<@v}m*gTh*uiginWKIcZ$ z^*AjAN<@+-rw`e_%#1QZ%QK@eAmD&N7{T;^hPs3rmx8n#>x>D`pBQul3>|V@f4Ug_ zzQosIsN2V^THI;T<t6(&HpWjeSdW+xPB~Hm=~kA8bt>JSG4Suh^7w1QUK#Ms#^E)d znP+EkMnMxr5N8~dj27GPjQdsp0225=L)WJN08UnpVVq2d=3I2(V|UYvzocB5CW8)+ z3l?UTu;GUXJaTe6*2!CIYp?ADMU|x`LC)ZZu1CKeYnr|q6&0E$jxJQHb3vavYkn8i z{5Zl{;}cnD3v(AIZ~IHw`)Bd4ZKk)uWnM>9)N|9C`mX71^^Gm@7`Rpq<|`a4<M>;S zLG>N+Uod=O@P~;!3*qMR?}oI=wLK;+?R}<({{T#~g)mxp*9XjjMoXR<M<5#TF*yvJ zuM_Sv{w9Ocsfjh@NJEwiGuIW5d#WX}8Ov@vxZ|j=oo_xn*<8W8($fKe`L~8#Fv|0f z;v9}Tb;07i<6rortKUbDFr<B!GtNIDSkcKRdr0fVaUB-@)9B9-{6Mm@NmZoW;C<C> zeweSE{8RB(^5{GknTj$BvuB)Fm};8**OIdqY~*8c<&ULM(fl{!4;bH#E5p`OU);*! za*O-8^VG9+P(69CcMp|QrSCL+_A@$ldX(=qE1gJ`+S26x-b)gFI#<_U0e@%BXHmGf z@i&Prq(XmrXKOERm}wUW9CWW^_&xh7Xx<Zs4-j~(I<~Om%}59DE#ceA<0Eqp-7DNY zW8-Uo0^AJ((%jx_a{xpr`xf3Kj5LfobI@a_>DMZqBeCPUu72^~cvrwS+B+?c!}(7! z`KKT`1pfdDCmdse(-qxG;Tf4DHtY6lt5*3YNf4J%4$**l74hbu@%zKtY_D(Qj~-tv z%PCWBquC3y3J$x;AObqzlbn8Dqj&LpMzqi)@wbD#b!+hpPqEo1t)|>*?WJqlb(F2O zZhp*JjPWB3NXRgtgOi+9(lVZ>&=%S~%q+6pw2tzF6sYI!XPl2*kzEajq2^kqoovI; zwntV@FvHlM^^4)Z5m@+}Pt$xsXD~LGuq^SpIa#B8jHw=k74{(0Bugo8TkJuiZ!oGM z;kgGVJY;?q(dKbf-o}h}23sqeY2;*0nUA1m-N_yLde=2-(?HQcT+T2?)BGp5*Xvr+ z$rhb%;Z>!TquUl+CnK$M_APp1p8IO69#p>r0F2hqdKkSnHSLy0)go<GH(NTiXCpgF z<Z<{_okHFSe6Zlf7~CahJAOmZ{{Z!?nbkZ}u*i@mWRn1&C=3JjCcNjwx`v^wKbL<r z<Q#mVv&MM&Mn0Ws*jHzLuK1bmEG_Lcf0-g?LvVQ;SFrUav$VTNuEdP(BP)@gFz1o! z=~3vKHi@R`sBR^A*;_3c`LJ7@?F9N)du}wHk-QMd9f!()gpa3VOjjt+cP*vA*ybcF zfFoA_031}lOc%CJ3xdc;Dn>v9=~q^GZmu?5Dz@CHBxf8}Mc{)^weu0uJ>=(}PqkWl z0^Y_&t1Yd>*C+y$fEl^K{{ZXOovatPFdrx+o>YX8K3%){$voz{ON~n1%7t=?pPZ45 z{{WL+8eK_aXCdD5330e0<m0j8vy`3998srtVzww{oWSzSf)xnQ<Ly{i7Z)uPMy4@> zGDjyBa@^fq!p7#&RuRHUDx8ve_U)RU)?67PhS|2B>Nwr?J*qAR1e!@A1#uFAw4MQD z=}QV~HsGu*v4;791aACK^sTF1EXOdvWrQ?*vPyDD&t)8bIjA*=B}?ORXi*S>oy)hc zpvj;N`4d}D-y<xn22On{x@tFGV#{{OpmDVZ<Db&4UH!V=ApU5PSY(7wgX@~jzqz+C z#*l&MHV*>_@iYOWXJpqGGF=%kbAV6EaDKJYMWozYd2C#?dC%S-d;b8{n&#3*xKxf{ ztH3PBpvS+hX+6wQL?c-QfjL|f2Jfu^Nvhk(i1%Z7KX;#hLz<;)Wo0wMQtC*Ge9}7} z3G~G{O7|!Y!{tHA8NeUoTNCNF0w8Vj$si0!Jq-XrHEW?O+r~o$`>s#kAC*CKCA>(G zv$2`fw$MB0)|oz*>lu4#U71Egfu5g@I?GhCx*kRBQb#cPM2<FqM{jxniqhZBT%x`~ z$>+b)wypg0aU625?#ae+_|-FOEse8W2>=otE)H@}@~NPjTXqr8CRvv~I%cUNTRM2i zO-eh9V5##0qo@T(uNCd-bkk`Hp-C)?S@FhCwt4ig2DVVP*aWk4&#plqg?f*KHB)h? z3mNf0kOXbG;|uGLV_4LR8ajK4L`f8q<j25Z02n>F$?H$IxRyJE8%kOx`+)n^V0@tP z$Pu?4!{jG&j`+#OD%?`XB9c)KIUgYy$JV*VI&ZZ@A^!kJW>)?^zz^1@M`<mYyu!PA z##jFUuSzb9#zbxj?mfPTHCa5SjG*Q7lfVP?6ae7y%mbFe#yfw7F37*X{Hc!|g#Fd& zP4eXa=%5e((fZih{h$)J;tY?t2S3bIb4BDPV!CHkU!0y#{{UW=@n<drM3Xi$9wq2~ z{i`HCr!<G`-~ydcr{6w<oL9}Bp%GST1C(an%8=MBJ;gz+Yu2&p$NpI|{G~zXKb338 zBt}8~qh)hvayl`=Q}_d3Y2u5IB4$*O5Dsz|1fOw}T2qU19nx20t)J}@Nx2J$#tGyf z@HLU*1AiXPiY7?Vf`hO<{{RZq(O@?ahHwT(^2ghcUwXfDq+UqgTyqIa@;2b8ADHI5 z)D%ux%IxMeq|^LFrpoRo*|24l4a&>F>-DY!P>WIUj<li|kX$Qyke&j%j?IDTU2<vq zrjdIzz#DSC58hMA^fig%yB$L5zqKymGnQYK;%*1w&2>TD7YCu}x<;rqriWUx+Dxm9 zfIo&nMsr^8r0X)n0J(X`VT41Regx*eEAW21b)nrgwhT~THXC<mJq>%mhjm%3$JjL% zb$pxvd0^aw$7<5#R7-Q|I~`6Vx<e^B8-T$^`rwc2S=Q-3r!$i=o;4?ie14cUPe;5F z+dz^ho|*gP@&+@>Kgzf4uB^P)p4J6y=Q{}h02<a0Lz3qyd1-q0SdJp-PImp!syz-y zO=!b+r&&an(^*LzrA`QJ?l~PfHLrCv;sv?+V?an~<N%x>PIx}}qQ=%)RyksLULtt& z5Ecab9E#JGFb0NcAXiByhB7ix$@70I&AOY*WrW<xCgFmi`2*jMYf|=WSXtn|c^saF ze&{ueJ&nAV5v&tJqmEgy$G#}`V0qTHZT4%47D89e`=z<vp4qN)SGUsTh(Qy|?aN`X z1$t+SY%ZpOXNlGPz7Lnl92)W6N;yCTq4NU`=oGULeB@M04FTQ7b|ZDk3M3n`jF4-x z^KPy!(3{a6y5k_H1pXcC&ulItXrN9}lxICiKI6FjYqYbwxswsxPU<=jEkdat{b|8! z1HR7FnNr#zHsPK5AmDpqur6(j4?IZZ4m_f~0zK=tdyAOux7(xKKW~{=1M;jJp>-O5 z`DBETnEbg1KBl&Pg6EuSw%TIA`@n!cLq`@1HwV`kuPxNH&$Qd3+Pgyu%8=n%vDe!b z?$?r9#WYtJq6s395%)uFKVM$8$#{Y`x`+J=E%xYo1sOi$02<0#6&+dg&b#5wMpPo! z6qEtPj(Hdr;(E@3e+=T{&^litmRVvr3PI`UYwTN{8s+2J6dp!bo$a@9#(nFA*E~CZ zh@M);(nf)lDBy$FHOzFSZI2r8x5PIjXx<`?ZSI2w^Od}~<^%iWj0X244z=wzULkm) zw|M-uYjVz^&f>uL2Y^SoBv*-ehrw&A%PPrefH4fEh7b@rJ$XFzBNfJ4&EkIrJB>Q^ zWVerzZ7K7R{{Xs93jGHj)Ct8s&$qOH7ELXgnpsdt>Q2Df?mdPpy3;&Lh)fbewOv8Z z?B_V?z^{mWFY#AKyMME4U}@LH?Zn?KZb|4H9DY^qx=y(?vU&HLnk)ku7$`A}5JCBe z<v|Xc=zHX{K|6V0KbDKo^(6D_Ow=^^b!|Bf%PepwZQnjeOjjqTYBq$M>yl+IxKIuk z869~(mDpQfz|l08kgG(*Vez;r^y)u445XTRo&n-$Z{;7^u$C}GxrX)5S2^|I_OF}# zRi{ZUyqAXFMPgrR{v41-IQ1E?xb^FSwbBJlKg#DBCqK%*X7P&J+6ir-g||rMd;WAA z;oi1B()d*b(R?)09?dawutyY#MirG;Xd^s=19QpEcj{%+VDsUT)LNsYu0~E65Puwz zT$jSAB);$lp6!O(tF=R5_8942jkF?598rOfFgiEP7?F(U+v!@tXhhbhjLD!Ic@`@< zR$>5c&&KAH=Er<i^}mMom|<0NfhEpFZMm1_KHuS7_lNap9#?qPw<|I>PUXmKamQXO z)MAcjwKLpYn`oG12%G0W;44J!p)g5G=aQzP)|SxOM(%uuZNzYQZsYN;j!kCv%TF*& z(YuvKkjPgk${P)q;j#}a(~d!{YySWb>GPx-)3C%FzV91`4mS?uoO50i;*DPLTSx?v zua?Is*xY)8q#Wd6`%!%?RF$Q4d^P(^{0=@Fy1Mv*@b(CsK$v}}Q`2H!n@yHdOV1Dk zn`drWMoKerk<;SCclMT!#@Z<_E+hexD|IrvXQmZ$PI|8#fO=xTUG!}$#TxbSWN$7E z6$;8o*hV&#Cuv*~4tj%KXT$G;pAGyMcXfHB=<r$JdA@X$?A+wYa={p%kvCw49FE5s ztlo^DGvGgkU$TC`;@5**@%-<gSY5F4EeAJK$JChyN0L23$?0Eu_z&T~hx`R;ai-`N z5nH4T3do~*bpr&K;FdM-_Bw8#s66)4OeB#;=-h$3fzuwI<dat5u$IcgOxwm87kA1s zNj|jgea&MnH8Ol5Wj*E83u1Phk+Ihq&tQFXj@>Krr&`qfN%2$TCy4B=r9avlo~+t! zx0c01q!RfK$J#j5ocBJRYxQSO@ipd=6xa8~E$ywpdFEqj<D3q)@sI51;5b|3hl+k6 z{I^yXcegR=_&HN8>lt1~82%oZ@9HUgIQB2vd7oMMU*T7Q{1f1*JPV`gu)Vk0yqnvJ z5ngMVUxtyTBP2281CR~}JxAxq#t((pKMD11C&k_zy^*YSN1g~3aJLaUL{?Z&P-2k? zLb%5y<0NDCYB^P5xQbndLZpHR$S^q1u<MUn_*3@Kx^oYJ{3l~>(rPPrs7nRB3?55| zm19uA05YZ2fKM3Dr!F-0(1}!c+`0QIYI>*qAiAEE(n<C$GhCQiM1v~~Q^q88T$6<@ z^JB3m*1oaTb!g&?cB2?cLMy>AAtV5de8bc6ub(~;XfpgY_<3pJtx61Q6PLc!j4iul z^W$t;$0juZ6W`ERWvR*iq~pz)NhHV=i5sTT$2kCfD`!pgIj1LcV^r}qwX#Glou^<v z@{EigqdvK=V%A+tSB(wqj;bVOiV}9^0p~yds;^|XHrDTI@v8yzDclN-9-|pKuF~H? zx3myUH#r!WapUHboN{^&m2p;DncQZbrOMZSc(f@8MQm<Uj`=mMa|8jTO-WKzVTDvt zoPGl}qjxRTu`F`RZLI*t)Z_u&@!qG?w97I6k!0^NL7b3sN8{R<8Cq_gXJoB&3{oRV zc?zHcJ@Q)>(%D>DLL`#n+U9J1-K1kbTyfs4+O>tgZf3UgIr)Zj`g&B;+r@GgTLNJX z{0cA*e;NSIg5+H+@@ckZVng$^t%lFA$2Ffel+r6ZqB7(VufaL=t(&Q4u!=ihkd_?n z9Gv>r7OQ!9ZK2F%!gXT5bpCYiQe94N`o`AEZP7p^%K+iqAY^vq)Kaz8;f-QwgpR+! zppT%)K9!kuaUIJvl1R;y%w&+c0Q+%T`d`{0g{8LMSZyb0`Mz$PobW0nQM+yZtdSR< z3>P0NHb`I0o_`9pZKTaUkvP4$3FqX62N=QjHJ28pu1mhs&e~rrx9=sHet&#>Q)kld z8s-eiFO}PA<&S<dz!e}nO`R-mQYHD-SPw0}?gxCH`KWcRM&U1AB>9?Ox}ybx{RVjV ztM>=Pj^Z`*<&q);As`LkT=cF^`a6j(OtHeY>Zjx@jN{zyIr>lq<9n!WS&_m(6stzZ zC;CxuHH?=w>1hLblDEvxcwf@7E*cr)UGfm|^()SO@sV0KR^~}O^6!)bW>;wbbOF=q z&t+$G5{Z<^SENIFZXJlJ<G8rFkLQIXh>w(_ZO%QgaZzZZ-rPKFQCXWinKP14Zg{S< zQi}FD1UhWdq!Fkift-=g6amG}sl{~hS}LOd00~f}5$&3vRj{|ytkL9<iDVf_k&Ay$ zD&3XEt*nxLp}fL`=20I)dy$@%&g+_mvSMg1T0p0goSuIU1pr#JZABtko>hei`>NQG z`y7syFP3f!-AM^@-zaU_$JdY6scDi&bGqS}&Y^lceA)b~dd}`YvA8&mBgq5br}d$3 z1>3V~wrgn`Gq%tH2O}q%WJV_`Y-47XfKtbeqNt_Yh5@8h5e?4R<DP&H=kcw5F_z~u z%%lVUtT4fI>+QyCR;)~os4;2;XMm%r?rU$ucd2cw{@`QnKOs_hB=r47a*{-<9#+c( z$^qQ|b$SrdCz_z)KxJXqc;pY#vzk^F&vS}axrbvQ62eOL!6W#G7{(4hwAf*H1-3L? z<L?&8{A#_O#Fw^~cgQw1+GbAw06O(QjcTm&EQ+nuAFXmpS(Jz~<dB9k%bu#dll~P1 z__fT<lC91ZWRJr&cINS&gmF5Zq;44L{xrzrSz2gHi4Q>FVDs(i_|%NM6mrQDNRhD& zpo5W5@+bcQo=y3R!c=&FcA<b%kM*5@4%F?^{{XfB0DmY0|I+%ed~rkmiY9bpk{kMS zQQc}$7=$-|WGV*%{!o7r&S|DMxS!4dNe~&}?WkpzWX!TzB*jn23-lhn4Sd<^34M7D zwwzuWBT=`FtXa?3Jq37;^Stv)WqrUL`<n0kL3Xk!X(A93GE{W@_^vAM+8Lqq;$ElM z&>w2;!=xsY++7;^FjLJaM?lAe?eAWJph%B$AyzS$JmGg|KF6olyoGN9s1Tiv&#?r0 z5-Xz6wLN0m?OaUqZBp6DBmvsHY@X&ain;7-q+3gSab<YVzp9)LR~^O&AB}LnA<}KF zP9Y*RDslrJx%9{BUWeiRVKo(LFP*;8v=(MgihF0*IIRs!NrK@^50IXzCJ4d)?@mo= z6=Wpre0kzaSQ=%DIfA*y-NPJ?M;Wf4O4hE6TWhmA$t9-P<W)Jy9-xt1KM*uaI0>Bt zE5*p%In8*P06Ia%zP`NEFH}b(c{8xefDhx%P(61)KzuW--e0^D#@6vcCLE)c%M5>a zoL8k?CfL?EA=$ic%qY12Rr0^XE9td1RF+7>!a>WfNgmi7eJkiIE187xd6yEjAmjI7 zfCuB&x2AJVB<zhlyGdfl+8D_LVVE+2N8%~hw)%Ct#d&P*v4Q@q9;UiEu49$swhRm{ z{{Sr{7|*q2&*sN2^|unDbSg5WXSb=Xnz06IYSLdwuxWP5KJUHeHVFL1PkCh%T?D(d zjU^x)jzA;diqVc3#oTvtS-e1F<=cWu_6Lfqs~g!_yoluqBMqU40<JGYTRFW)PDtJS zw+Nqkm0PGBdVALsE}WiZvThuV79V(n?OxFqooIBzH5W^4fCF`6-=EgFuM+9ji3<yK zec%AH<OL%hlr?dq$sJ_0oz<C656W|be-3NE(=K+}EJ}<7pLn;gvBhvqrq**K$L11q z^K;Lwb#~X68xh9v2plc~<I}xER%1IWWU#b}VpT~bCp?0B{vxkmTiR+iiy57xDlquq z5$Rm?SN7&lEBB@PWby$&fUOtOE*V7ZOlVJ2&QG}FmB8Y4Z8{LMTM1HlK^RmSKbX(w zT)wD+J0ynOY>{!WG1@+v^{-8bNV|etiKkVLJ;M@2KrFt6dE@b{eL}-em>KsdVgCR< zhXi{cU&5k8PRE$458h#!$y_;OyA6)jj9wX~7VRkv0A5#u2OV?oUF6o6H!w6Xyb<F# zVa9%@x%GnLC1nw-tYhbJ0Udi9$;B1zBzZQy;mD+vwV1f_<L$dDN&G92zVK9ca&49{ ztG7Qe+J3e404~=JYdlh9V*yAbpTJ<%ZEr>>h>dRip*#AM?d?j)J*1C_d_&<)Ivbf) zZRC-Tau3V~J?oOQhs0hVMw>{~WrbAaK{+zVk3cdpUtMZiRiebpI7Der3(Fh~_rU93 zapEl>Oq$hN#@NRy@JQ#MQ^z8UBS=(}IL!~@{ISh%7l`3DPl7i_OT^@Q@HXeKdiTv6 z;svg+rokSQG&3TGG9*%D{c(VE_*cumAn^0V{%#X+!2=!f>0Au@E!1)?-LlC82G01I zqha`|1MtO5MAhEs-s4`j5i7+j5XY5P4Tc=x=LB^0uQl<d+_z;|xsR5Uk<)`;G+TT} z@z#W6g6GTBV2hOAp<nSJ62~3uYUkq2TAizZY+B83;SXQCllW3E54rkv@OoF5;m(bD z6uWm5zrWpK?_SY8>A0PxvI1333J$E~XZZtOC-73xSbQ?k<3jQz<IRo0>VyJ#=hnTp z17RE!G09m0FSPvYfO5cg^v}I*4T+xY@X@8f^G$iA$0A(D%)561430@8udXX|N7VHJ zI$GUqj%d%7D4Bo{yPOVq>DHe2Hup+XNW@T<IDCMrx#X!nz3W~rPr|krC8oJctA#>Z zG>lGPDpv$ybDZO^8P7FBj^=NVJV#~WjWuoAlIrg9N?KamkXfHRGaunldCm@N%C!Fg z3&E%tyVtzAZvqC9OMI$Wf=KmYj8|g55b*}Q>EcMO;~I=)#ACYJs|*1C?ltLyj=gjE zRJZ;Q*X%@>Q(W0WfC@rgjtT3|&<!g-g->vvDRk*=t~7|e<PjXloufT5oE~d5-P}mo zl|)iXyu#x<SRT2^&2zerui}V<#iq{$yD`2M9eZ$V&o$k5#&JSsdt^^2e7i@lJv04j za<eASWP4R`ZZ0Hid6h_4hpGGl=~6F<t)RE_mhqbj<+{1reqQG_@txnqzlgtRbho&U zCsiXJSz<pjD+<hbv&1(GCZ~UKF=oS}w#3Of`=k78NXJb_Yo6Kh3&*c{VA^h-ER3wg zHZsec5sn8vYuP_%e-25ad^kzmDqU)m%XZ`^Je&Y?js`zk@l7AYmp0)YU7}IZmj~{G z*f|F|#eI?Bv|CRJ*v1R9{g=#Q-GOeSf$K@4S{7*}fur+DMwgZVe7kyJj^Njd-f1>E zuZ!aGQPZQj)GaL+PqS$sCJT7jwiJ=UFP1Hg@&|0!vUq~zHk{JJPUXhJN8TO#*P2*a znPh0;1(=L8^u}^d1#L+Z<k69F9sdBFAyti-ZYD+r@woilah`GBw=|t6?X9xvSRhOu zO68foKs`J0S_@!pzywGYe&`1$9mh^{R)y4(Ot$hy&?60@!g2xQIqmhW=Iya9+{;Zq zTWiTIw1f7)E<EHY87<qV2il;xp5?DC?R86ClConA_))jNv8`sl)wKJq$oA2z{^~YS zmOFc%MoF$-?8|egc@jIiK1{RlNIB0L&w9zbh}4QUMQeMOXY&bY(O+;1J#ca>(6ne3 z^w_ssNx|tG6OcVm{{ULJtrpZtdY)G4cDVCH-~fY;!@s3^PNi>sDhqukKeEEO9%OIV zxyZ*Kogfmz{>aMuboPE&WAeG=pRW~HR<^hoF~w^lK>+?FKQnuKRQ8uKK`;@aat_vK zIqY-W)})tOvw@?7%Sj~U=Wiy0k}#x>;@Ko*khFu9W+U>hdh1-AMI>b!UHx)$s(%4p z9-}krr8Zbgv)hhEaxVm~&X%w>ylB{QCv)kV3Nm6BF{7D*+{67|-T^&1t?LBT0Gi_F zK#3Zn1<&3C>Co17)#Q*(9kas4v6*E?+!XfX9+j&F`m9@0H=0LMLk0Or1bX+WkQ-M1 zB!+VEtY%Gs4XWQa@~Rh5i-u_u8ExYLV}5>6NFKcNO^94iGQ4)%0};HAs&VaF{{U%7 zqZYC`k*0908zF$}*9W+%0}bq8w~u6x1Ws74bC*8&KDC!{GZP{{QGdLk5`V(C?~H1# z6p*GMV$C!B%-QG%0~oGS>sPnEZza^m6mZ~if&FL#1da<t4Jrvs9oPf^0N0x9Ebi^~ z5Ee*0rT!huw-^=3hq$;<ELlpg13ha}1k~*!S?3JuMsj(<`V8iPI_95FTQ#>4CA{IV zB!2$jcOLn!qV{ND(~ZgU<hdvUh{z!E*PMP;$LJC`xO<D$^A*=SW`wX&k<gMbJ*v*5 zcWpbnrQm@Vv{7Jz>4T9#9Q5|umaAmYK|04NJBV1y<oXuris$8;=GIT_`HWyO0fRb~ z3_lF!wQsKNEo~a+3%Ok+1UtKIAAsa%@UAYx$4%5HK60@ovWx*IKIcEufIUA;g4aos zIhiB56AZL*5tJNr?kj&=v$b^+TB7-KrtEY-PW8-ax=xjCbhlUOZ~}vmEEA60`&Uyn z&W?y7w38<U?_hFs+a13;Yj82G?#-Qqkw~$=;u!q7Ja94j)a!e>t=?v5mB>irJY#pY zVBJVuG$=tO!QKZ&t?O$jB%T?eS1t08>~qMiWK4~jWEPv@J6(eipr5<zRxFx3Nb_*u zmR8|?PfE!NBHT5!yIM`nAvomcw_3k!K52F)N~r73cYlRRC#85U;_By5HpoLuX4~0W zvGTTf>0Q=*)D0r+;Pg4q@~<KAjrv*Ys~l%7bOds4;1W6hmFjaDyx8Z7#v@ij4??*F z;Ckc=<*Jf;7PcqaG8lkHFbeQT*V>Wrlb~lTUyZ>0x%?@DK+S>_2Oa+a!j@Db6$*+0 zJngJTOs{U^=eIcr2aMCRYySW|Kk?d(yl;Zr*Nk8qPu%*+{{VeZ2mjOhdg47zQ!qp3 z#s*3&fI$4JlFe$!eBjW{juEm)(=~CUNeeHLJLjR!)%?M%tx{Ceto+x7(6}n3xHvyd zdsogVw$D)EyhC)-+!bC_3=@pw^{P5_%Pb<_dZ^Ah`A_mSQc0Daqy&aNTcQ57I!iUv zM+_lJ&T>E%+eb!Bosm~lw~5NfaH_&Hg<;d{fGZL><h7Kwtk{3yE091x%vXJJr&(T= ziW{A?fwjj_5BOI#u322^$>o^Xvf%lt^C=%+TI!Q&*&N)>o#%%2S>u!<5ipq*WK-Ff zzH8ZHy0O-6<BwB~qq$YuuB79RoB>}bXxg({>X1f~U>t5e4^i0H)Os|sON+ZEf>j`H zNeMXoh~};mY?mf+ek<_YkEhKqnH|R;b+fx~PNY}Ko+8pNyx=jgrsYt*cp|>=@fErq zDoGrO+l|G3WdNUYJJ-p+Gqk!nlGS|oic^rFF9+Wyx)K*NG5#EQo_#x17V?&S1Lh)} z9zFdl?5_*zP-@mC<_Q46WL7P>=eBx&mHEpR<;IcX#l10z<7W!_a6riE#eLoI=TVnX zjs-wBg&Z*(6b|)p(UC26XSLhC!mMz!xsN#jfwX^9R&_{A#~;}|#CZ88A&xz9ny(g^ z*An@&+@ll!06dS~Kb{D!d8Uri*X%cJyO@HK1E0>exsBPKZFi|fbsU0MkgypN^NPEy zz2)-6Rmm}Mz;FpQmw4?gq2ZB=jBpP^J-_<(Ev49kOdY|9`Ga>Hb5^z$%*)*d_E!qw zq*4JmY#++8?l6%^9?DW<A$Kvz`fzJf`%=ES)Q{Nh-at7ccNiRcd)BY^a6&H=ZL_e; zAq9y(`8_`hWk-7+N3YvRBPk3~BQOW&J5CSvu6ows#Or9wCuQ6l3JLT)=DkZ^v$mBa zW>lTzBO8VSx3@LsH$im!y}~?Ufcx1}f2C(OTSup9RxNLF8I0v+0Jia*A6)v@?xAyM zEMnFvyw@x@fTRVkH%ru{yk(2ca#W5Z9FON+rkRU<McGnhE{7gr9YF^;$F(FwWp=Rh zB9b3DVs!zBe6PPF9M>Cpr(8S?vnZ7T;n$+|>@!`Kp>uC>bLO&aXW`G&wtedsDc;@Z zx3`$b7qH17`scM48aWG8v=8N?mQ@&R5J&5Z=k*Jj;gVRqxK#=H02m}!Np~%!wDK>O zB~ox$E)GvWbRX9>jd~+U7Uv4>oP{Hx`c%mwnEG|6nF~80StM_sMm_srcBrnb?xB=V zb9ESm4W$(P#2>Fu%9(WXK`6epa^Z2lCFpbft4clB(JMs^O(ds<obUz@1CGA+n?Q1x z(CIS8HN0sgs)Hfjh6z02ao6#$F4XNHyn&a=nm0dm4paf}ftvO0V#eamKQ8ylK_^v= zfrHLDC(^iC^q9Pj@m)JAWR=`8pU#nSJX>7S<eEsnSl8vr-@(A+uj5=r_JMHG%d%7v zo(>Li@7ld@U(<fsEwWq1BQpXCX5f3~xuw&lv<+%)k)5-GTowIsO2c~Yc#fyx$o#d; zs^=VW`t{<x`&-i|5Ph0g{oIa`l6fF89M{$Mej1uo4=ts#7Rfu<FweFPah@pf+sd)V z%?Md0iA-uc7aW7_){7%rIy3Zd;fhJ3cvDHzWR1iY*6I<@3Nkk1zH^M%P2w|WrfAxQ z{lVDFXE;eg`AKyke>%6J>Gy5nT{ydLh}woyqi|p{K>n5GU$h3h7K!k_^77o1zG!SE ziMSw3^A=IhE4H+X(9Q077lUmqr-IK;*R>fWYk1JAG=(Dpi6oWoan5t?j&eQYOFC<| zG>w8ZC+3W2<x()gv7EQ3YWT~-`jwlqT)o5Hpvr`Qmmm$lhg$ky!hR#SQ(>ptGUR-v zV>>XQ90n&j0+^3lN&HQzOtWbQD`-eDw5~TZ5HrXl8RM;9lfokQ6}i{0=9XyCGhy(Y z`Y%k>npN{p3r8KpBw)7o1RaN`VbAGLxYe#w_9cLMkh+B4!yw~~o|yX8z*3XB&T5_+ z(cukw3qO<-aYo~QIOL9%<oflMv)TDp#bAy@oD8rB>U-C?Uq@{eO$5{KD0gLf1fJhY z;C0O+ZzfMaWMtw<z+hFBbDv}FTDYPve4*k!K2=io?HQ5Z01?w~9-Znv7f8J~D{!v4 z4hy2LIKdrr{VS*ODmAvFa|~n&e9x6|7yv*%)vKUI<jNZ2L=rYO0^i5<$EIo;9nQ+$ z1hMes%{dDq2W01#J5C2qGt$0~(qRdvTH7+a#TBe5>_*)0O~;R7ab63kh}%T_PD5|F zwt=uPNoi9!_P{mtZR^~_0!<S!02K?5m&;rX{*^Wr&Kty5v8l6$D2u{?k_o{lJPcMP z^`*=axwEosQMV<-ZpiiMKPpcV{kG>(mh!?<<baB*a;i@`{VOKY&sg0wjd61`e;TkM z2e27Ats|zwu-LxZ(Zpt%lVQk|4ZwXzAC_xtQg1rRAcdY;VH=u77!r@CZ_L%(OEWFE zkcs6q$afh?37_dzU`4gJ5Zy$~@sqLDzD7S^N~b3GC8LwRcxSe^xSlnHLRhdTkba%b zV`(!<bOiERyh}3@qkc{S``E|nQ)|;+O?A3h(JkG;0za7Np8o*-YT3|qkh+38g0W<B z+%V&(UX+A&Iy8D@R}AsA$tfdpo{}G2_Wf(owCi~6*E5z8K3SBuN9Wt}s<w8(*%KUb zNg!f3FvbV?)H-gTb2RdK?YQ%VUIsD;IXx%<ytiAsVstlIWC6ZaBc^+?#d9y_F<7E# zp8DKiqp<3Jo|V*3bhZ;dn&_yoeE5tEwP9Y~Y0n#8CA%cfMgw6;A5b!AEo^7r-(HrE zG?+Z;^14KJfKNX4k*Ou^<FA<{afQJncYk`jKASjKy|NxyatjaQYUTAScHCnUDkBe^ z$k@;LP&0bpPqeX(r@FgIWzJ5+)b_yVkylpk%tS)l&pW93Wn=5uU{@Dx9y_GBu^wi5 z!ezlAcVaqz72N5fc#=DdgSC!u5N<tv0jZE0R`Fa|*_qlinFd!QkbUZ%#qgR~<+*$@ zBjv_Gz&}dZxwh19_IXamCnIoiwYqv9)k|1kvBMf$+n|w#3o$1i{XeZp95$h-Yco$Y zutz7C{2>qwf$C3c=F<Dhxodg$#8_nR`IxumT~*|hGQ77hAOIW%`Bab%aT^R%DH5EV zK3D^c`_KjJIi77mA1*-}9)3~~`{#^Ts9eD!S;j54vK1dTNFKPZD$y?HSmB=Fs>*Y> zZ~%e$SD@IzGuo_oGN@G>Q8K5V-4E+P7Okz_0DHC@WKaePILBZ+)_$=wTiI=f7mZ{) zLo$wdJ+WO$`$}1cwi_8n(!c}y*AwE4?NS>=kj7=iZI&&c(-Z;Z*KZ`w_L96QfPVNm zZb#&6r|{jqix+l{ow(W)I8r@0=D5f&=DWX7J`{;JE_ZTBZ^MfAyGMaS4>lEwFbFZe zNIvQp`p^ctYIhcvfcd0Ai?oLT5>IZGfpH*>E-r3Ze8cx@fH5HRobiv!w&l2%-T+w= zbdc@@kHe?qTwb!v45l_F27GQELg$}qq`HAdN$!qew~5NOIX?B?+e!92BoZ7d0h69{ z>_7VT&S~nCLT28~k&N%?0UTDXz0`LREI~_bKR!lsdvjHCWNOPC8|0lE=a6LqI3t2G zO+L;40COJK!{cLQ_a3zjGI<Q@k&H+%Oouog$G14AS=&cA+N=%Ct-eOj$~N#YKOb5N zF4`oYYe5$!TsdLRF<$d$aU8bs&cRqV0w?!4=O?~v!c_ox;(+2oo#Y&JuT=28$hp!; z{{Ws@R_=QG)>DdmmbNz^_fi#HvN_3ag-@~PBBhdK546bGIKVX&w^B6FvPP_cZRNKA z07F5Pxqp>g>(G4<dgWsnRoIdhi1nk(llzZWQ|C~3F!miO+fVo3@}vj<)cQ#TVSMN= z3lq-Kl7E=48^l+UzG86f>IpwLAIiJ8nIny)fyc-<6&*OQB-StPE+>-UD8M}ZE5V?< zo~^8m*cC1a3XPl-*QP%zxdTcbHoAbujHe7vKp>B&HCoP9p5Jfxk3ER3%gG~qCAUR8 zQy9l5KDFNBlhEdzp2f>`xVk_rR4c#8cXTWD$6C3l>9_G%D#Rrc4qblo)Vf=&F-EdH zVPqg+oDp55D;%@3+fTk&`Bf|m;~hQgNWsZmQI3a#>ajJpr4_~8D=MH2cJYez&kgI* zOB`_8ETSnKX9w>7AN_i7i8RaU;ttm8vr4Qm1~T2fc&{_F)UNbkjWH|b$~OdVSDbXm zBDaJRLX=gHt^{j;rJFfm6G0#W)xP%!KZSVLi@ZB!7?ExK*ony)A1L(BeLZWm(e+!q zxKc#9P>u5dI5-@0&%IBr!Y-_)g(goq(2!i<aoeG-3{Qr9Q>R;nc&2|biOJdlJBLn2 zE7|@C_?X<<Lt}9&nGxC(pGC$q&wArLai$9|IYfJ0sLlofEDuBf0M@Pt;l`*m4;BcC zA#NFPgy#%8k@cZHMD{;YG-Zk6g=df<=V?>5N7Q%x>!Z{xEv{l;BrCWZzDCYS@6(F% zpABj^m)dkM3_;X_ROdX7+~U1IN4t_(q5l9!F~|X3%0TBm2<Ep&1wXLe-C8f77>vT> z7|1vP^*!r4=I+82YpJ4fDi1H@%Vc%@I#*G3s$bkR+F_6+%5pN9$ovgrNnxtJy0mW! z6*+cg>^-@vxEMNo*ml%oX`@91l|GrRfpu@GqnoKptB;k3^7_<MUFq9Ai?1={$4HlG zKZ_L<Ew#0=jvYGIMp=1n@*%<X02V*ztx_sGnL2{$vBFkEEA3JjV9EN|j_V{O@hyl5 zyq9SK4V>4u-bEd--!|yvbF?XLe=WY=wd39`lRs(uK1_=1hGT(@9N=S<QzU64qVUb+ zmnl4UA0(@QTNniK-n}x`&PgryT+FX<FgJip0)0+;SBQ9ed(x_{Gh4@iTaHN`>(jLT zav3ua4uFnWoR%PtoOhxtTN%jcWz*8yR7Kj(dhX+&)~M^Rs<Bw@mf0h01|e|3`=3g& zs#?9$#eW!)nlu4bdCw!SPvuqSitc4Qt=U2Wxwmq2?TXS=`weVzcb0cCPGpy4ug23K zADHy6ZhLrMLm`xY@a5wlF<f@)E3ee-R^~Og255GXm;Uho06nV?9V<<@GsKWPk<Rw( zn$A(P)Mqszv)LlV;x;TkR$LrbW!{{jMAOdEoMjyez&_v#(woauz7d{Icd*F)DUn>f zkTt!lkkaRVSYQvXD+%rfVq0mm5^wIyxnGxX+%h}!SlYIMet~7VgftQ<*ke$ux7P!l z)x+mmq_JGx+wA~=0R~Ubn{{VymXKaSc`Tbp+-WxFJ-Il=No!-wF02~fWhDxT$pB|L z2fjO3Gd-ipD<K;jsNEg`T>F~uZ>5dF%9Unpl2|bveJPuCOXNhJLib2Uz`%fgi2nfV z&@-RYwA*<Oe$x<M>Hy&Vc%;|#DCJm;Cf$r|n{q};+&TQKwnfw6#535!tVTfyNI$J) zO=E90!rWTH87k|yXds5qa8FE7H72$`wD6pAj|*ug8QaOYvsBs+LX>^1bK4~6rF_Br zQtGk`Z4<(AkO-Db>4`bsS1P&s3i@lnm(jPw>lh>f0>>!FeZoxOAA0!H<L$&(o;*}1 zd&g#wDs|Y$B!YeYYPtyd_St5J;w8vYfW(fOIX<=N-W=5Cis>@$%eZA%jEsSt_OCjE z)!Ie#vWl1tI^%ABmELI*t+Xw0gP{Q6Wb7xe=ZZ>NY8<WXeUaf?_nu*69KU6RAnnWN z92frp>sNgZppG`SBkh}caGhfWj^nT7E9buk&2+P*k1MjNQpgTY(mzV~&kX7SLwDp` zl@mB7Uayhc=~}`_nI@j5td{y+%ztcB&E*n{Y=b1a_deZggYgl(yS148of<fX04D@7 z<EA@TutjTS_9l|i*)FioM)gp0k6QEnY+OfTL~(h@q&%{j6m2_(PC3p2HM@5t`6pip zHN$atYGW+rKm?F*0QB~*m%_FhZO!UU<nLg`Lk`S&8SBMI;>($3z29#TWHUJp!39SE zgP-Eb;+^5UYuC93)U+tXmMh6P9X-CaHUrW%O)lZ4@ZG%9Cflu66=>Iyv>}t8nZ<kI zm8Ve@<atr=!N&w-kMb+S>}=Thm%)hJ=7n_$9z}D{?ycN#Yt^-uj>_I|B3WK00zHh} z4tEkb;QRjojY~jt7T3{BCDo((R*3@^4r3~0<X{7nU1f&9r!~9Sv^!^!GO`2mkTLCC zIMrTT_FIS;SSqMei@n>R>&0zqRvKi$TtVmC6;Qa5Fh+SD`{Jo=1ebHzLo3|eMGVgx z5~}2t&$u0H&UKw!t6TDsz*U%{CmYX9eB2CYjD9t*sY<chnBat$WWgloJoV{ZIr8O| z-gr=XAboTB^Gulyvz70RXuDMoc?ffyj@kaT=r%FK8k=yJ6EPdNZQG0wT$<!8rMZUT zO9@4~gYzf=4EM%At#^7sA!G6%?Jmf{$y4&~BOGV?&`G7HYHw*VjZ~FwxmGzVp7_OR zTxxd{T%m_-Nh0keZU?CJ6!`6>Z#H?ziEyDxCp*v(N9$2qUEED<A8ARVVyZfC^vVAK z>sFD9Xb0*VCGEU2TqDHsvf#>doSu9ADw}}?n3O8Y<#B>J$F6I3JFP{cShY)J`GFii z<D44j7WN6>%4l4-%mN_j7x~mrO^aI?S29Topj(0@Dp(^Q(?9)c=9&xZ>Ee#~poy}4 z#yASu<LmlX?d9N_{KGwzmdF<y-vx+1gSXbQ?`^DZ5;($L+&DZZ-9SBmJXE2bZLO1A zTehB(D%=(MS`okp9sZQnv(xm(iWX5Migz(m%){62`kKIjr)We|mXBc^M%!fc8SB6X zx|>@#EF+poCT1ZsmT%q?aohg@uSI~mbxkJu7I2cqBxh*Ae4oXL7_NU=*6$bn9@%G? z%i))X#(tdE)Nt!iTE{c0NL!YT-wXli!B*z5Z)du;4HR-bGP%o3umRBb_NizN7WHQH z3rZvgP;#d^8TP8yT3g;ZwYP1bbqh(z<M~wAd4>rmnrS60N#0v=EIl|JRgDo0*4}gi zV)L-apdk15qS^u7Y1%cpk_&mHw~&km%W$~$_O8s>X}Wu~cD6R{ynqe1T<7sW;aIwU zmvebHmvQBUiAUT{8|rgiEU{c^@vQnuj{tIt*(A~ey~d|+r^O|c%@jqWXDj!JBeC>8 z)#H9E)%=Tn{{Uz#HnNppd~^EN_lUK-sNPhL#}dk<zmo=f@spP8UN^7ZTxwId%*d#6 z>V857Jvr_u18c+P32%&&$L7e&7B+VIP6m2o`q!}O`aal2)}FpYDu#7Yo#gfe_V&eb zo)55ej^0R>Zi6VuYz6^=`t<d$M!C4UwX~09g5O~X7a946aa55liyZT#Ni6DEFZ{E9 z2sr*#=en)c#qOYp;aQ(K=RZo+*R?4g+D5!%f-{xjiw@mukkXyh8H`3RG~rmBwmSa+ zN}*kyt-X}C-bSAMyuUNV4h8@u0@&z&mC$+7JEUx(Srl#&@V`v=HAWk0ugc%VCMg<W z^0?gHI{hkmRtGGyutil1f&5tSK(0u#!*MBSob6s2NbB^i2yY=+q+3XKGad=*G1rV$ z9X`qzn7m}?k@Ijd_2-(=v5pH#La8&dt7TW_{b&G=^)4KU*JOZ$pHetA={^g*0%@(v z<VbKwy?KqSQ#36rff){PKZ~A~(|CI3TP;@7Rr`ePe|*%+F2lL&CfX^MO{6DLwYeXG z^{JzRX>GteSRmkZsHIj(S)n1-j_ss+`_+cG$_Z5YWc5I6j!Mj=Qhn%lu0Zr0(|612 z)ny&y8&1>DdVieWi&8WH)%R_B>8$LefpZaW?{y;vyslXRwi1SHkb7`{TI#$-c#SN{ zDR=VFsNK)wUVD{mRaJHvZgOxjUPUPCdX*&0@ZKy-xS`w!LB(|uP3Axoe#Ujk{{RZ+ zqqdC2mT#LG2R%hdZKW;@E?A~H&2?`hnVJ_@sToKvc0LH_7$e`UdOwF(MtDleAQH0% zQn@Fm*1VJVmdwp`7E<4i1xcmq(?O^t=;2ZzbAgPEjxk+uR#rt=`aAnA=3VIGNZ0Xj zKN|Agb3?YhXEw-Uc$<>R%K&rJ{{XLDzK!D8<%meElDX)5VEc;IlH*O(*`(B+@>J)^ z#^2MuP}*Flm1)|nmMZpAM$3sZtgDi9?buhYLl&Q?+C*C9M>m#oLg#=hk)OhlLuf9o zu9@O?C+9yn6|1Jp9-j!c8{f<X43Ky~tp_RQzYw%Lne5*6!rVvYC{|yTsT>bn*T-7) zl4;t6I)rM`%LKcW{{Xwk>J5H?>edlzma<!`vZzoBlg3x_uZ+BZrAub>THL`VJjU4I zWpKcZ=h}ci+4z6r%lULG0ip=Wk|dFffWswDdi&S5cz*j-W-X|N+^QVy01udtO87(I zKZ|E=5*<AiWRlWGw{I}xk@rvHGHdOB4y3mqyq0CRk=1rIO}MxnK^~Q?9n55`&aU@P zo-3KP2&ZIc83rGnuzOW4*CkUW-aV&#HdVh7{{Yvnp4Q&lDVFDSiE>yk2jxEGbnQ_1 zg3|7IUPVV$&(23f!Rd<8qE1`;Lg9D}9z5+`pzi#6JW||Q+{+?d9Z4c%z>SH>&+?+h z<A)Z}k>)V@KpU8SYgsK|lWX0XOPqt17$1kN3F~5`xs7*c1coJR=bg|JO15x+D)QeI zDV-M16h$DC@=4U=@axTcPucX_7|igQOS!;Y1Cw4?;=6mvwFbC!D+$H{{{RYe+|oxQ zq{S*-BJL6<FnQgKW49IQ7G_wk1ERLv{G~<)FnH)|&K+d)RvT6^?%?(#@$|0#B$CP$ ziaWEs0DeK{1NiZZQTUAHb(%%I?{78S21%H?F061AR-9K>>&?W_o!k`+NeA+-Pe|A9 z?PYl4-8@cyb;lLb&1nSZ?H^{^9A^uGkHWfJu-3;Xdbf9%CI}}0A<FFU_}4!^i#lpA z<~`J`-+OC(n8*0nq1>$LYS3E8<Q50_jDX;NVz?bV%^l7rx_MS6JN&Xv?|z&C_|`Q$ zjO6FC7R19mWJp;4?3}g_PJjB<kqwO6Y=!)XFt&2U^vNBuUCya!f_9ee+Csy6xEMab z<2|brQ?<L)Jf*UbtXSn-;1lbg{<U+pa5-4g#ywH@*++37*A7AN&m4VgQ+AysDlM#8 z3TF#0))}m8nD6}UhHxT?;mA;MeT_cyCWhWOx1EkZi1j)83RXayy^PC(ZxTqQSRy;N z=gSI5UI88J0xd4i-&BDld0U=}Lv#99Q*#!X3h(n}60qk0kXZf|pAMI0B+GLw#7@63 z$RMcosc6u@1+KkoI?rty%MykFTrLkiIR60a)oaZ%_Gl98$`Q(^dn|t`JpTYH>Fwvg z^E|J%dGNnLMoQpz=Cq}eCDT^Q)(!Kpavl2Q144@>v#jt0t3QP-N0<uO%ZZoxvi|@# zKERw;%DywV{?yksO-k-$GS4)EJ);b&8vtk63i@Z^#Kknv5L>?nEs~DI=F5&h-N)x% z6XH!a*`v7Bt>OrIVq5r&5^yQFyA2)_e`PwIsUVOZpbU{s(w)%x4lrFv<nh#TU2dfV zT*dZ#NW@Yw7(EMT-?ekrmUGz5Vw7fV=V)yHwT`_`s(%UVej2~Lw3;MUFrFF6IrA~b zGI5T*E8P4o;y>*R!(j{p47gU2xXADLSIm|dmZoUqxMIu?D5UTD0xR15F*GZd@+XN< zg+_i0lg>Vv6|FQySgp^cEg;fky8AL*#dm?1^2Ts|JJ&hl87(y1YoQEIB6S;sD8UWe zkyUgJL&%Y$(-J7yWt(uxEsS;iYJ1yAE~1j}Sh)L0m*p`lZ2*sMYog~V@_&o2AYDmv z?7Ij8FrXj=9CY;cuDifbJo7<ri;$&@FkEnW&nKrfljDf=wbj-r;wc*pIqu$^*LmPG zrv07S?cK$={n-HXj(-|_%cvb)!G>QQct#C9fD1jyah#3j0!MFCitRi}e`yqFbVcyk zU=G~zjPyNu{HxFY&|Aj-8hdzrg_i1c0E~RX1A+&xMr+YE>pOX8RFr0BRPxU3AkSmR zI3WIXo462ZI-QL6l3T|LMI1md56ffN;CHNjZ&zlG&5Ug`vi#2RoueO(YWX&nB1tts z3yf_LbDv^;KMIdfy3^x`Y>wfhVaWt+QhOdbqTq0|PM5ak;L93eb~s)!-yMCckdglY zw6zq9BOjKX3FsMpO>gSgmiBPV1)Mg($U)qTxsN^h;<@NW;kmfFv4sR;J*|<CnB$Nr zyA_VI2pU5pHT+jw6@PHN0qyTyzMXq>6j7|u2qbnGRa|bsI6r~Kar!QWe=I+3z9AJt zfQhsNj{S3*?CvkDqgaZh#z@BuM(ki`98q>lMgIWp8_NZb*5HqvmMlgvNy+u=P>oW} zZl;tNLr6y4Hg@}-1y2^0FNmxS?xI@WGF6oB?StQn%b!h(&?E(85O$Ca?B8CTS4BO9 zEN0f?S8Ym3CSce@7D|lw$mvyov!v6ne$i%UxK;;hA>e1;wC{$dbm*YdQ7xj!%`VUZ z9;cemw$`Axl-ojQzLqQ|^!?mL&p64?rDs=i*2YG)rChw`b74G;LhhSCcO8$da+g}3 zu4Zdg3vUvTpp)}3`hmrEH&>U}cULVpn+mAr4>(|-z<)}?zA&tb1kkhtAL~I3PkPSd zHgwH9$b_pEXK|hQ+{Zn$pXpjQ(A~vy1m05m2IaBpeJh=YYlu}QhDDAtz$-D_#~y=< z=pj0rpDr74USIiW<S`)oaf)$zlG&+oG||lOZ@PKH^<G2&0DvEja@vb)H!v2|5nxOZ z=Nysgn(b|LeL^S!nq`HPC5rv*d-Kw`KM`D+EToLS_1~O%nIw$+Rmv*kN1R^FRhHPD z@vcIKIKVyYtkUf6B(qo}I7J{8R6K@0yQ_+o5?i?x@?A#W`*F>B29JF$tSJa~k&eTJ zPsgr(svlvUR+k>2jThNgMvVu|7-2h)e43g$E{aWhm|0+Bg^h8a)}Ny4cPVfnvx;d; z@areaqp7HM8|8vVVv-_APZ8rFj->SBqz)g%diCABQU>#_rI~Uc{jW}y<1w3^S5ibF z=aJhEM+ZLQy5AdWvT6_V-N~^}?&C4?aonBUdVV#V;WddbW|AGkTq_godgI?T0qh<f z@Z8~EX(m`ogjG%3O6~UVT^!Td+D^jca4-a!$Sw52u75{~o>H>GENsA^GDqAG<AI9j z{8!?4(ya};GCMky$6OM6gI2aCMGZ$#w7t}u?4Zp1unS}${{R~9tgS4P)n~cb#A7I{ zkDKe0$KhNKl$Ms7dNiBmn&S*X9Axw8ab2Wpqfh?;O8wg(xHH$=>r9h&<6oKWO`Kt) zKrjOEpmfPNsy5I(2(Gc4gvK(%8<~$$*X08|(`{|7Qr!*Ifnw_D?X{SCbUFG~g~h{* z6t=gJK(dga9$ZJ9^y34H0KXZxHmYIbNf>~!43W34eXBy{zRhSZSN9IX1cUu6B}2yX zLYrcZaz@@<sT}0`6ItP=nU%J-#$ve6bI-0nr2uHy%{9dDERG+X3~`b4twfZhn}dZ2 zLC3$PP*WYe&#)5b3I=chJ^EG2quC=pyk<pE1F_(9&P7~x&<|<1d8N~>?w|$m2IK8d zoXY`Qjr*0yE9qF?6TUL(xQvsu;IF6QT~);D@a>fV_r@!RuH~(W3nKu-X~@kPU-oIn z-X;oE@Ia>hl|6r2$Y=l7?{#U|ODUPuhZqca&UxuvP4>}jHudYrO6av%UVC!UD#qYs zZwH|CtN|2lZanR~opMHa0D4!SWVSmeCv#rT3z)1SmVLjx85tY}^v-I<jMnKpJmZHr zL)lxg{IgJL`jx1*k57<iZ_2E5f&A;ROBiIZCt|v%&ABjtK9x~=l2yIaZcVCPpx+Vu z!A|f`uR)5+y0Q@f_TdKuBX1)=jc(Z+na|ndSz?LE17OIlo6Q{COo|`OcflpN!2GMO z=ISY<&sK3^Du@VB!*c_`EnWWrhILywq*lF-RcCAlj22P#13%KSHLnlLoXPT*=u^_T z=x>gRD~FYr0FRlNWPTJ~gp#rJv>J@19?~cTkvB$ceA)KoidOK{w}oUXu(%u;Tjn2M zYVyAf_=0=1M|+_dEH<eb1b@ANYuDn`wOd32FD^uJoT`^^r)s2ev!A@N)UB;VV;FKn z2014lp7ryWk0KIBG-N!3Y2J)>k=u?AYwByc{@&>=;<PFh<S5$N{OiR0Z=#2_`(k;G zyLQsxa6Y-_n2(<R5o-fT@r&s$3^B~X!w?t}qaFQoU!(d4p*7XK5b5R}rEnyXLJ_-? zI2`n^%ouIIvV3c%+!t-L$V0LoGlRu_{oqv9Z)1k$TjYiqQwWj)jDj$8p199yxIIXc z)b{w%uVXecyP2Hr^CIO}-#k=;!s5`ar1w`Zw}8%esrJV|omtcGC;KsnX?Im5m0T%4 zqnecJcN&Gaku|h|z{cer0PkHExtm2zK5J_UT2Ckf;E=@bIpBH>Ra?PnbNkP=w&9XM zIQ=TXk1EnZcBdo<Qn?`Gr?qkyP+Z4zA!z3)L2oe^<@$4qEI96No4X*9qfI29S0O}X zl76|bH1Qs%dS`gu3Pe<p<xj6{<C^MIRh1${mK;X;8=T`F=L7oJ72*}Snn;!ynrnFS zWSJP1$I#N2=8`fjt>BsC5C)!RZL)ct6Fl+jThZFyY1d*n<(51H(Q=0*cFkpJjw5K{ ziDnHeZ}TG=-Hrf0oolzWyOzNo=4gD*2s_&jKMWkyt;n%j)?0%k77&JUv@)?$dLBkS ztJ3VRVv5ETySF8=$y1&_yjKYy*@VhtR#`4aT#=oJuLB(ZmC;$Kl|s%EP#f-$!0}yC z)oLc}Rk+lx#M4Obe6R|tpPL`>8s>zrcWFA>xPn4{QH*4I;=5_)W@m=d$}4@Nhby=f z`d1@qVRe6c?Rh+2dXf<l3W7eltmd__(aS!aeQr)}o;E9kk&tusJPOWBVH*)F#%Xhv zJyyCcUdHy+t6aLCPXY7#Rsyxv(ZmRm$VUV=JC4{j&n0Fv4(>^1)CIf5H?ojbMih=a zW35|&2;<oH<r|<};GfJ^_06k8J+zZCSH{+1&p7qsu$u1K<aKGDXHr7%90T>N+XFjM z)%7cV&=}*19DJdOE%+Uyn#ZvU@;o<^vlGeS5(n4Px{X@G)wIETc_?6nWCM~9JqJ_y z*D7PMFXq~bt~UYqM#GOnIjOza?$1EdZFJu(C6bcI<qS@F{Hu1(#Xos8gLKfG1sr}L zRvwe6UTOC6!D|w1lbx);I5_TWqL^Pv1-!EPZ1)GOeGmiOj`gB;74FPW14(Z+{{V=r zSxk6IS<$#56@FjHU{{{pX?B;8#~h0+h^Gq6!sD+s+I%=ilj2v1EghZWdwB=Xf(hCN zF~v~SzSJX>EM8X3?T9u33&*EgcV#zY#QZ^`c~Y=T=!qN%$6dakTJyGsM7&9EA_OVG zW^7lo>be%CcFU+oZV?n^_mR`vj<w}itrS->3x-sSIe##m?e^oXLy9$}C$YbzP7Uzc z50tnhIKj((NEPXx7i*iV*oz0bMP}Z_gNy=mjQ$nldUe$BZk7p0kOA`ckj?oKUWeg5 zQtm${H!?`FOM`$EjoBNx9kWd<GUoI>{{TR~u!Lq?wS`ICQbDzWJP%ygMRhC<2$tD# z8Xw*>E<w*gIR_`{T$hLSof_%miq6&UXGJ1S(UMf5#zzOgIj+X$+Q!ZaFK*mD+l-8Y zT#WFccp0l~<vwuuhi4*caZCdzn!{+@jDyGFUEhJAWrgF4NxZaHMs{2&ZVy0DY}YyC z@&uOmGfK}hpP2)Hs{o7)`kzc!ci@=X8%Uv=FtXf<26qrJM;wl(rRoK1`OEm<!cu9h zUgkTQBt&p<a*PSV$86W8Ybr19lFrxVF}Cf^#vJD(+*goYTodtkO1Hs>>`iZN8!ku< zB0f(i+P$(HS*~uRx|45Rs=4X!nyBw!BEHe1zl?`ILq?fqLyg5pr+n8DeG!g4L*=k# z1Q$32^VDX#7M9l5>EyUF&poc-S9i&^f%&n{ab9Pt-zCkdyRvj>v$eYAmpt*#(ht_H zpgE0Jc<t_dle7sAa}da5=shaVmv)dbTZMDGkC<&zGC21&hLQ`J;|g}fBr2exB|!H$ z&TG)LyHPZ1uRMhEOiUG*j+;r(Kgy6D7Kv!`dD7XUMI(L8t2sW{BW-jRacZiYHCvA@ z<jSmMDuNFlnWuPm+gOg;;K^=ewg5OLV2r#ToR8&Nw)!=V<j`K5n^ZB4<~O+CDd3!X zew9fI#fv)&hPPW+x`s62Fyrq5o->|5D$3Mtyoe-EwZgGRag`bT0nJY>;$As|+iHn9 z!~CAT4>|hRD#>*dHL(#k=H;+(pnLQ`rCM!hg2>W`TC>?Dj4_+H3As+vFa|S>oK{W7 zw*iayP2~c1t2|*c!|A)Ht}AB2Ev@dvw-=sSF*Xcol?RVwT&}GFz6zI@hIoM`5+w4- z8SX$85^apC*Kz6>4<m1pz~c+Ia2<ON^@Dr`?6I=IP(%ojf_DL1w>XJ|#|V}{pn?e? z0r>T-voz#QDk)ZF$;jFTNCm0;K#tL2Vv2CCP6$4@{{ZXPcV%;7Vm!FwQz|Y?%EX-e z99Jc!PrQGng}Celd~$mZoY!hX%?hkBJaQ=F76W!YPi#;t5y7p-_V)=fd1imOkpe*; zfSgyFc#BeZe>TqDV?cJYG5N#$@m`&)ThBebYZHV*Fm`;uUDz1|+*gcv?#=XQB66*8 z@?fhL<CBa6IX=9KrzSHiPZzg0@kO+cn+4Q*gTd>?dQP1nx495VB$h*k#6zau{VUHj zC%tQhj@AVpXAEW~dU8m|B-gm;`h~8TnX}UgxRpR>Qc35pCq9)o16*7~ac&Lf`*l^u z+-+t(x#Ov>C&ZT)AKB8pmeNX92s;T<NbAoy#~)hlE?QG;{{U-cKtbAN{_*5<?Oqe& z+l$+KX#`h@WpH-Jl5oWJ=N<a~6cmx<l4??2>K<A}GJ;M^9H1B@t}EC)4`XLy?)Ouk z5npnycmSMruP@VLTX7)0l$oSb0;%~yKD<}0X?Cq|95Y+b8wN!LD{ka*jANYrcoYLi zb$NHD=-wZ{*RJIu5<S^sM-TTv9_GGj)2?RmHmYy+B32Op0J;Etp?l})$2HM>Y4JV9 z_OqpYJhYIsR(5Hr>P>*COqmeJob$92Px7w2!kPt&S=-DQ%zGHq_j$l1u|2)9_|>h9 zZgjdzgtd{T;x!{@@V3x9XE`605h=c&Hv2T9Jm<_KAY;CGtJ6r4s4}9?SnLCKRew$g z6{l@-u|)fVykumBY$}gHNcoSp1u!qb`g~ECq>s&VK3SgxDD?x^@v3VDmBg<snu0z# z!w;AN*mL?;3!+{{9B!L7$?}5O-lsjgS6OLqq%W3{s>aGXwolFU0Ga?#H6-7+<{@@R zfyvG}K7)$CX+6Gtpf>qH?kBTitKHi?Wnhhq2IRH~-Sp{D7%!~1D=cAU#s&*F;&`A7 zmUDSFrDkN3;#WKURX@%-r^P&zM)2<^9X6bC??~2{_YkR8P#fiLooWem^I-w`%Cle- z^`@PG^e+S787}9vEENv}0|Yk}=+na9SsC1Wcjmk=!V+7<szD)brddj`-JVB&o$KD^ z3mQmwf=Auw*177Xa@NIlc|xd^pkOgh-haHGomOk<VIdu3kmK(W{{ZWz{{U&f{PK_T z{{ZV(Fwg(j^c{ZPZFZ`&INQ<Adsm-3Z$-!*>(V?^)j!?-mFB1JFVxoqI_J4Ip3NQF zgg6cIk?JeEw~c;i6DW_v1tey;J1_YAPpx_dxBAibrya%6t)<*uSzR<X)5#i##|NBx zS9xQq-o4~D^Ej0xaHn_#;=KCT{y*RTx_^y!79a8xw*LUGqxl-@I%BcY>u{_}-dQ6K z2VU4U<eo2rH=Sc)!^U%7!Qv0P-}TUcBVJ45-}w5E{<$CNK&7#P;MpaPZ?mhmz}Nuu zj%(>{0{-eJTR{7xIQzpSax3P42>$@b@B9z?*U`E!{C+3Ue=%0JIG*QmG-$ep*HIv1 z6o==64S7ez8%UBgble6Rc7yWcuTR3gYUln!BmMFJ0HB)k{{V=8@$R4SFXU;6_(R6Z zA=T`piz-Cw03AA-`;WkItab{@Tjz}w<m6!B{{Tw(d&gh$6SMySuZ8|*zUJ^Z{DY$Z z0Cqp*Dx<kFK7-IFa<gX#%PSiH0J=thrFU9xn{6eyxoz9mC+~6jSD*NQ?%e+X*G2yT z&`o-VmHz-B{{X?OrEF&-g1fMRyxW1fPnq{swyEpsP}#>F!+CNq%Y(+|$LecN=lw*# z_0#$q&eMO^{&d1SoNd6}=R`@{5hVwq6~)M{cFtsX1#`-sxgLac#dfp*0Ia(I0D*rx z<6`^L{xvc*krPcWouv7ej8TckeLj_DIUnq7(lL~fA1}`!RV_pQK*Rm&w7>D<-|l}( zmANNXrfWCyXwhmc;7!mFqa1YNyR8!X_f?IJ<m}vz6aYV!d5)3)03yU+?;q!0iQzx> zP5$t%sBBD*k5sX?vVh4FWtehU=dEX3*u!&ZU5hI!mjy5mM;&qRTe|-M{Cba|to=X! zJfHr#xAdr<<69nYE`_JfIFdzXd=}hT<eZAj)gg)ti4qhEumT7i*IWMpEeHFj{{TW2 z%WGfp_c!~Zx#zgfT05JGeE8*Z>LM-y?Aw%Q@-@ur)3vUkmQaNe%7C{V=j&VWf2_Fw z0ImoB07F{lwf_M54O{;J*D+a02uB;FKR1*{(0Lz-u7gjrwz}PQDh=e02Ltl1fBsp` z^%d1<{{Zpq-%<RkR^WA(Rxn)z^8p@TBWcICKb>?^*+XS(6gJsKknU_A_%)NH^gmDQ zT^0WT`m2BQ*w%ozK78;!%+UN<yYk$E@3D@0sb%BzuD0<*>(>HLoF&Z419ZS5xX%WE z$yvDn0It9LFs|0?{zKPa_4haP6cr<z)$J1L<SfpCUl>N(F@we{#5_}J5VxLjo$()+ zr|ZRg)%RI{-~NWYKgD11<9YS|v=>Wr%=9fv;jV8E6Cey7y|OFTw2O&mYiymrG9Q<Z z<67}u1O7fQeuMdn_B{*lTK!Z107F_9=Cj!Pd%@t^q}Nd~4v6^dI_I}FucrmM(%Vv< zcie9{a!JR`2N^Y!;g9(Ax&HuhzpZO%{{ZqXxBKV+07Dg0IZqn#{?ti!qW<b^1CCDP z$v&pNd%_kr?cpnXw_Va(=gTDY+{>EsABaD8=l;8|^RH^~fBcCn_<!h2QU`<BT+es$ zZc#gvvsbsr;H(Gpuc0-m-rfcgU~Z8Vp7<dD0EK+%qyGTOT)6)Lu)H71tLSeK{{X4~ z0JhQp0O%&FJAsL-O365$Fcsp9a3Fo|*l$tIc{Ztf^H|;wE;I_jfP-&S&qH3RtbOS} z>yP+Xk6-@)$CdZ{!}*HRKw)VpuJOqEBzc4aNg2o;Gw)uZqUpBTfDPD;F&{Utt$CJ@ z(LdmS`Vg;N(Z2G3x_^~O8_|fuO4|V|7{+o(L)$p5O$y4<%thft=V4pr10N$CjB*bZ zn7``h{{XII`5N8OKjqW?;r{@jJzCfZ?(TlhR@6lth=3yRT#z~Mkylw(??i7f1@e@# z?HJ@7bNSXzr~d#SU+^LPjci;00LTu1`sRO_rUwIabwAtSw^rL6jHtnQ$T|N23c0Q6 zmi89{TXv8YUIQ^BpK8mv{{WD+ul;pT<ZGqzKi@y@ikSeqR-J0WWFY0&c^Tt1#_F;& z>Wyr8KzIX=_2@c3`l$Z^>zV$Q=X#g?ha3Ix>rw^xf3Y$0)mZKWE;4!_OjlI}+^0`# zfq=>eFgO|G>s*!p0Ny|EFY-0gTmJya9#5eDVt_4OJcZ$aIaO>9I*fuz9`)j15Z+6B z$gJab+T<%aIdU`W)3>F2Wykz|TYXpm06{h4KN7#>F$eqO`OpTphHWAHWuq`I!_4K5 zPfGS(J<KpmGJW8H0o%QKABKP9?$_%l@~>L7{{W9EzJvK10OoZIxo6TPxSI~^o}(kL zrFgH1*=M*D6hviCHVh7x>=%FX0iW^yb>o`n-~RxBe@Y8m7(O11OiYrJ$7>#ah_71E z@bbfI6;UO0LQr7g*eSvK6JAr{pSXYBZ~X{Ys`!8YKdpc3tY7*uXa}+4Nemi~i9Ao@ z8~B!c#?@|Z!F-iS?LwqsMtH#fb?ngUH;_kSkjPPFIAu@(#?i(r$8`Sy`1tWB{f9Ll z=C$gh?&-f@{ghR$j1NWAZK0P<nn@Hhaz=X}TFCoLTv<%>Dmj21nH@3ot?eiNK{I|e z%)|cx9v7cO{ON%004v-5x<wBG%m6e0GcioJI0O^Nr%Kcyz_8035bmx^XOqok_<R0A ziT?myf1#~uztr3R0J#4E&VV%~zL{+7r@%4*PFpAX)-~p#d?S)`Gcj}bPu(9uS7-kK zkZqsQznx=V{{Y9K`p^ePvr8OSyIlO>Gjrah)n$}K2xHZMn5w7!U6=cUrqzGdek=Uw z0``qFzNG|g1{Q6@j&ok`B7=fBIPLVW0?|L@Ef@WD{{Ra2_vk*K(y^#o*uQRp;6`Fe X#ZBce`NcBV{ypRRe_EcoMt}d=rmsgY literal 0 HcmV?d00001 From 48ad7e338b7bc3aad8b1f873ba8b90c3f527b7be Mon Sep 17 00:00:00 2001 From: Vitaly Terentyev <vitaly.terentyev@akvelon.com> Date: Tue, 17 Oct 2023 17:58:09 +0400 Subject: [PATCH 159/435] Fix registry (#29013) --- .github/workflows/beam_Publish_Beam_SDK_Snapshots.yml | 2 +- .github/workflows/beam_Publish_Docker_Snapshots.yml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/.github/workflows/beam_Publish_Beam_SDK_Snapshots.yml b/.github/workflows/beam_Publish_Beam_SDK_Snapshots.yml index e2cfbd9f0a45..9bc268bbe9c2 100644 --- a/.github/workflows/beam_Publish_Beam_SDK_Snapshots.yml +++ b/.github/workflows/beam_Publish_Beam_SDK_Snapshots.yml @@ -45,7 +45,7 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} - docker_registry: gcr.io/apache-beam-testing/beam-sdk + docker_registry: gcr.io jobs: beam_Publish_Beam_SDK_Snapshots: diff --git a/.github/workflows/beam_Publish_Docker_Snapshots.yml b/.github/workflows/beam_Publish_Docker_Snapshots.yml index ce42829598b2..cf2520f4635d 100644 --- a/.github/workflows/beam_Publish_Docker_Snapshots.yml +++ b/.github/workflows/beam_Publish_Docker_Snapshots.yml @@ -45,7 +45,7 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} - docker_registry: gcr.io/apache-beam-testing/beam_portability + docker_registry: gcr.io jobs: beam_Publish_Docker_Snapshots: @@ -84,7 +84,7 @@ jobs: gradle-command: :runners:spark:3:job-server:container:dockerPush arguments: | -Pdocker-repository-root=gcr.io/apache-beam-testing/beam_portability \ - -Pdocker-tag-list=latest + -Pdocker-tag-list=latest \ - name: run Publish Docker Snapshots script for Flink uses: ./.github/actions/gradle-command-self-hosted-action with: From 2a764dd2a05ca05196d4cf558ec41dda939d12c8 Mon Sep 17 00:00:00 2001 From: Vitaly Terentyev <vitaly.terentyev@akvelon.com> Date: Tue, 17 Oct 2023 18:44:58 +0400 Subject: [PATCH 160/435] Change upload test report step (#29036) --- .../workflows/beam_PreCommit_Java_PVR_Flink_Batch.yml | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/.github/workflows/beam_PreCommit_Java_PVR_Flink_Batch.yml b/.github/workflows/beam_PreCommit_Java_PVR_Flink_Batch.yml index a029e071ca80..283e0c959d09 100644 --- a/.github/workflows/beam_PreCommit_Java_PVR_Flink_Batch.yml +++ b/.github/workflows/beam_PreCommit_Java_PVR_Flink_Batch.yml @@ -99,10 +99,9 @@ jobs: with: name: JUnit Test Results path: "**/build/reports/tests/" - - name: Publish JUnit Test Results - uses: EnricoMi/publish-unit-test-result-action@v2 - if: always() + - name: Upload test report + uses: actions/upload-artifact@v3 with: - commit: '${{ env.prsha || env.GITHUB_SHA }}' - comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/build/test-results/**/*.xml' \ No newline at end of file + name: java-code-coverage-report + path: "**/build/test-results/**/*.xml" +# TODO: Investigate 'Max retries exceeded' issue with EnricoMi/publish-unit-test-result-action@v2. \ No newline at end of file From 0850a6c9523a2bf1de30434e3d47e355ae999fc9 Mon Sep 17 00:00:00 2001 From: Alexey Romanenko <aromanenko.dev@gmail.com> Date: Fri, 4 Aug 2023 17:11:12 +0200 Subject: [PATCH 161/435] Remove Avro-related code from Java SDK "core" module --- CHANGES.md | 10 +- sdks/java/core/build.gradle | 6 +- .../org/apache/beam/sdk/coders/AvroCoder.java | 820 ------- .../beam/sdk/coders/AvroGenericCoder.java | 40 - .../java/org/apache/beam/sdk/io/AvroIO.java | 2031 ----------------- .../beam/sdk/io/AvroSchemaIOProvider.java | 157 -- .../java/org/apache/beam/sdk/io/AvroSink.java | 167 -- .../org/apache/beam/sdk/io/AvroSource.java | 773 ------- .../beam/sdk/io/ConstantAvroDestination.java | 156 -- .../apache/beam/sdk/io/CountingSource.java | 30 +- .../beam/sdk/io/DynamicAvroDestinations.java | 61 - .../sdk/io/SerializableAvroCodecFactory.java | 119 - .../beam/sdk/schemas/AvroRecordSchema.java | 64 - .../apache/beam/sdk/schemas/io/Providers.java | 27 +- .../AvroPayloadSerializerProvider.java | 47 - .../sdk/schemas/utils/AvroByteBuddyUtils.java | 149 -- .../beam/sdk/schemas/utils/AvroUtils.java | 1396 ----------- .../apache/beam/sdk/coders/AvroCoderTest.java | 1106 --------- .../beam/sdk/coders/AvroCoderTestPojo.java | 51 - .../beam/sdk/coders/CoderRegistryTest.java | 4 +- .../beam/sdk/coders/DefaultCoderTest.java | 6 +- .../beam/sdk/coders/MockDefaultCoder.java | 57 + .../org/apache/beam/sdk/io/AvroIOTest.java | 1627 ------------- .../beam/sdk/io/AvroSchemaIOProviderTest.java | 173 -- .../apache/beam/sdk/io/AvroSourceTest.java | 843 ------- .../java/org/apache/beam/sdk/io/ReadTest.java | 3 +- .../io/SerializableAvroCodecFactoryTest.java | 93 - .../beam/sdk/schemas/AvroSchemaTest.java | 496 ---- .../beam/sdk/schemas/SchemaCoderTest.java | 55 - .../io/AvroPayloadSerializerProviderTest.java | 64 - .../sdk/schemas/transforms/ConvertTest.java | 14 - .../sdk/schemas/utils/AvroGenerators.java | 220 -- .../beam/sdk/schemas/utils/AvroUtilsTest.java | 914 -------- .../beam/sdk/schemas/utils/TestJavaBeans.java | 12 +- .../avro/coders/DefaultCoderTest.java | 61 - sdks/java/extensions/timeseries/build.gradle | 1 + sdks/java/extensions/zetasketch/build.gradle | 1 + sdks/java/harness/build.gradle | 6 + sdks/java/io/clickhouse/build.gradle | 1 + sdks/java/io/contextualtextio/build.gradle | 1 + .../PubsubReadSchemaTransformProvider.java | 2 +- .../PubsubWriteSchemaTransformProvider.java | 2 +- .../beam/sdk/io/gcp/pubsub/PubsubIOTest.java | 2 +- sdks/java/io/kafka/build.gradle | 2 +- .../apache/beam/sdk/io/kafka/KafkaIOTest.java | 2 +- .../test/unit/read/SnowflakeIOReadTest.java | 2 +- sdks/java/io/thrift/build.gradle | 1 + 47 files changed, 129 insertions(+), 11746 deletions(-) delete mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/coders/AvroCoder.java delete mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/coders/AvroGenericCoder.java delete mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroIO.java delete mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSchemaIOProvider.java delete mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSink.java delete mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSource.java delete mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/io/ConstantAvroDestination.java delete mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/io/DynamicAvroDestinations.java delete mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/io/SerializableAvroCodecFactory.java delete mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/AvroRecordSchema.java delete mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/io/payloads/AvroPayloadSerializerProvider.java delete mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/AvroByteBuddyUtils.java delete mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/AvroUtils.java delete mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/coders/AvroCoderTest.java delete mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/coders/AvroCoderTestPojo.java create mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/coders/MockDefaultCoder.java delete mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroIOTest.java delete mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroSchemaIOProviderTest.java delete mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroSourceTest.java delete mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/io/SerializableAvroCodecFactoryTest.java delete mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/AvroSchemaTest.java delete mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/io/AvroPayloadSerializerProviderTest.java delete mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/AvroGenerators.java delete mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/AvroUtilsTest.java delete mode 100644 sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/coders/DefaultCoderTest.java diff --git a/CHANGES.md b/CHANGES.md index 5db1b11c595a..0f1530a78fe9 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -57,8 +57,10 @@ ## Highlights -* New highly anticipated feature X added to Python SDK ([#X](https://github.com/apache/beam/issues/X)). -* New highly anticipated feature Y added to Java SDK ([#Y](https://github.com/apache/beam/issues/Y)). +* Previously deprecated Avro-dependent code (Beam Release 2.46.0) has been finally removed from Java SDK "core" package. +Please, use `beam-sdks-java-extensions-avro` instead. This will allow to easily update Avro version in user code without +potential breaking changes in Beam "core" since the Beam Avro extension already supports the latest Avro versions and +should handle this. ([#25252](https://github.com/apache/beam/issues/25252)). ## I/Os @@ -71,6 +73,10 @@ ## Breaking Changes * X behavior was changed ([#X](https://github.com/apache/beam/issues/X)). +* `org.apache.beam.sdk.io.CountingSource.CounterMark` uses custom `CounterMarkCoder` as a default coder since all Avro-dependent +classes finally moved to `extensions/avro`. In case if it's still required to use `AvroCoder` for `CounterMark`, then, +as a workaround, a copy of "old" `CountingSource` class should be placed into a project code and used directly +([#25252](https://github.com/apache/beam/issues/25252)). ## Deprecations diff --git a/sdks/java/core/build.gradle b/sdks/java/core/build.gradle index a0f9b739dac6..7c788eaba49d 100644 --- a/sdks/java/core/build.gradle +++ b/sdks/java/core/build.gradle @@ -36,7 +36,6 @@ applyJavaNature( relocate "org.antlr.v4", getJavaRelocatedPath("org.antlr.v4") }, ) -applyAvroNature() applyAntlrNature() generateGrammarSource { @@ -89,7 +88,6 @@ dependencies { shadow library.java.jackson_annotations shadow library.java.jackson_databind shadow library.java.slf4j_api - shadow library.java.avro shadow library.java.snappy_java shadow library.java.joda_time implementation enforcedPlatform(library.java.google_cloud_platform_libraries_bom) @@ -114,7 +112,6 @@ dependencies { shadowTest "com.esotericsoftware.kryo:kryo:2.21" shadowTest library.java.quickcheck_core shadowTest library.java.quickcheck_generators - shadowTest library.java.avro_tests shadowTest library.java.zstd_jni shadowTest library.java.commons_logging shadowTest library.java.log4j @@ -124,5 +121,6 @@ dependencies { } project.tasks.compileTestJava { - options.compilerArgs += ['-Xlint:-rawtypes'] // generated avro uses rawtypes without suppression + // TODO: fix other places with warnings in tests and delete this option + options.compilerArgs += ['-Xlint:-rawtypes'] } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/AvroCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/AvroCoder.java deleted file mode 100644 index ee252ba70f85..000000000000 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/AvroCoder.java +++ /dev/null @@ -1,820 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.coders; - -import com.google.errorprone.annotations.FormatMethod; -import com.google.errorprone.annotations.FormatString; -import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.io.Serializable; -import java.lang.reflect.Field; -import java.util.ArrayList; -import java.util.Collection; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.Set; -import java.util.SortedMap; -import java.util.SortedSet; -import org.apache.avro.AvroRuntimeException; -import org.apache.avro.Conversion; -import org.apache.avro.LogicalType; -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericDatumReader; -import org.apache.avro.generic.GenericDatumWriter; -import org.apache.avro.generic.GenericRecord; -import org.apache.avro.generic.IndexedRecord; -import org.apache.avro.io.BinaryDecoder; -import org.apache.avro.io.BinaryEncoder; -import org.apache.avro.io.DatumReader; -import org.apache.avro.io.DatumWriter; -import org.apache.avro.io.DecoderFactory; -import org.apache.avro.io.EncoderFactory; -import org.apache.avro.reflect.AvroEncode; -import org.apache.avro.reflect.AvroName; -import org.apache.avro.reflect.AvroSchema; -import org.apache.avro.reflect.ReflectData; -import org.apache.avro.reflect.ReflectDatumReader; -import org.apache.avro.reflect.ReflectDatumWriter; -import org.apache.avro.reflect.Union; -import org.apache.avro.specific.SpecificData; -import org.apache.avro.specific.SpecificDatumReader; -import org.apache.avro.specific.SpecificDatumWriter; -import org.apache.avro.specific.SpecificRecord; -import org.apache.avro.util.ClassUtils; -import org.apache.avro.util.Utf8; -import org.apache.beam.sdk.util.EmptyOnDeserializationThreadLocal; -import org.apache.beam.sdk.values.TypeDescriptor; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Supplier; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Suppliers; -import org.checkerframework.checker.nullness.qual.Nullable; -import org.joda.time.DateTime; -import org.joda.time.DateTimeZone; - -/** - * A {@link Coder} using Avro binary format. - * - * <p>Each instance of {@code AvroCoder<T>} encapsulates an Avro schema for objects of type {@code - * T}. - * - * <p>The Avro schema may be provided explicitly via {@link AvroCoder#of(Class, Schema)} or omitted - * via {@link AvroCoder#of(Class)}, in which case it will be inferred using Avro's {@link - * org.apache.avro.reflect.ReflectData}. - * - * <p>For complete details about schema generation and how it can be controlled please see the - * {@link org.apache.avro.reflect} package. Only concrete classes with a no-argument constructor can - * be mapped to Avro records. All inherited fields that are not static or transient are included. - * Fields are not permitted to be null unless annotated by {@link Nullable} or a {@link Union} - * schema containing {@code "null"}. - * - * <p>To use, specify the {@code Coder} type on a PCollection: - * - * <pre>{@code - * PCollection<MyCustomElement> records = - * input.apply(...) - * .setCoder(AvroCoder.of(MyCustomElement.class)); - * }</pre> - * - * <p>or annotate the element class using {@code @DefaultCoder}. - * - * <pre>{@code @DefaultCoder(AvroCoder.class) - * public class MyCustomElement { - * ... - * } - * }</pre> - * - * <p>The implementation attempts to determine if the Avro encoding of the given type will satisfy - * the criteria of {@link Coder#verifyDeterministic} by inspecting both the type and the Schema - * provided or generated by Avro. Only coders that are deterministic can be used in {@link - * org.apache.beam.sdk.transforms.GroupByKey} operations. - * - * @param <T> the type of elements handled by this coder - * @deprecated Avro related classes are deprecated in module <code>beam-sdks-java-core</code> and - * will be eventually removed. Please, migrate to a new module <code> - * beam-sdks-java-extensions-avro</code> by importing <code> - * org.apache.beam.sdk.extensions.avro.coders.AvroCoder</code> instead of this one. - */ -@SuppressWarnings({ - "nullness" // TODO(https://github.com/apache/beam/issues/20497) -}) -@Deprecated -public class AvroCoder<T> extends CustomCoder<T> { - - /** - * Returns an {@code AvroCoder} instance for the provided element type. - * - * @param <T> the element type - */ - public static <T> AvroCoder<T> of(TypeDescriptor<T> type) { - return of(type, true); - } - - /** - * Returns an {@code AvroCoder} instance for the provided element type, respecting whether to use - * Avro's Reflect* or Specific* suite for encoding and decoding. - * - * @param <T> the element type - */ - public static <T> AvroCoder<T> of(TypeDescriptor<T> type, boolean useReflectApi) { - @SuppressWarnings("unchecked") - Class<T> clazz = (Class<T>) type.getRawType(); - return of(clazz, useReflectApi); - } - - /** - * Returns an {@code AvroCoder} instance for the provided element class. - * - * @param <T> the element type - */ - public static <T> AvroCoder<T> of(Class<T> clazz) { - return of(clazz, true); - } - - /** - * Returns an {@code AvroGenericCoder} instance for the Avro schema. The implicit type is - * GenericRecord. - */ - public static AvroGenericCoder of(Schema schema) { - return AvroGenericCoder.of(schema); - } - - /** - * Returns an {@code AvroCoder} instance for the given class, respecting whether to use Avro's - * Reflect* or Specific* suite for encoding and decoding. - * - * @param <T> the element type - */ - public static <T> AvroCoder<T> of(Class<T> type, boolean useReflectApi) { - ClassLoader cl = type.getClassLoader(); - SpecificData data = useReflectApi ? new ReflectData(cl) : new SpecificData(cl); - return of(type, data.getSchema(type), useReflectApi); - } - - /** - * Returns an {@code AvroCoder} instance for the provided element type using the provided Avro - * schema. - * - * <p>The schema must correspond to the type provided. - * - * @param <T> the element type - */ - public static <T> AvroCoder<T> of(Class<T> type, Schema schema) { - return of(type, schema, true); - } - - /** - * Returns an {@code AvroCoder} instance for the given class and schema, respecting whether to use - * Avro's Reflect* or Specific* suite for encoding and decoding. - * - * @param <T> the element type - */ - public static <T> AvroCoder<T> of(Class<T> type, Schema schema, boolean useReflectApi) { - return new AvroCoder<>(type, schema, useReflectApi); - } - - /** - * Returns a {@link CoderProvider} which uses the {@link AvroCoder} if possible for all types. - * - * <p>It is unsafe to register this as a {@link CoderProvider} because Avro will reflectively - * accept dangerous types such as {@link Object}. - * - * <p>This method is invoked reflectively from {@link DefaultCoder}. - */ - @SuppressWarnings("unused") - public static CoderProvider getCoderProvider() { - return new AvroCoderProvider(); - } - - /** - * A {@link CoderProvider} that constructs an {@link AvroCoder} for Avro compatible classes. - * - * <p>It is unsafe to register this as a {@link CoderProvider} because Avro will reflectively - * accept dangerous types such as {@link Object}. - */ - static class AvroCoderProvider extends CoderProvider { - @Override - public <T> Coder<T> coderFor( - TypeDescriptor<T> typeDescriptor, List<? extends Coder<?>> componentCoders) - throws CannotProvideCoderException { - try { - return AvroCoder.of(typeDescriptor); - } catch (AvroRuntimeException e) { - throw new CannotProvideCoderException( - String.format("%s is not compatible with Avro", typeDescriptor), e); - } - } - } - - private final Class<T> type; - private final boolean useReflectApi; - private final SerializableSchemaSupplier schemaSupplier; - private final TypeDescriptor<T> typeDescriptor; - - private final List<String> nonDeterministicReasons; - - // Factories allocated by .get() are thread-safe and immutable. - private static final EncoderFactory ENCODER_FACTORY = EncoderFactory.get(); - private static final DecoderFactory DECODER_FACTORY = DecoderFactory.get(); - - /** - * A {@link Serializable} object that holds the {@link String} version of a {@link Schema}. This - * is paired with the {@link SerializableSchemaSupplier} via {@link Serializable}'s usage of the - * {@link #readResolve} method. - */ - private static class SerializableSchemaString implements Serializable { - private final String schema; - - private SerializableSchemaString(String schema) { - this.schema = schema; - } - - private Object readResolve() throws IOException, ClassNotFoundException { - return new SerializableSchemaSupplier(new Schema.Parser().parse(schema)); - } - } - - /** - * A {@link Serializable} object that delegates to the {@link SerializableSchemaString} via {@link - * Serializable}'s usage of the {@link #writeReplace} method. Kryo doesn't utilize Java's - * serialization and hence is able to encode the {@link Schema} object directly. - */ - private static class SerializableSchemaSupplier implements Serializable, Supplier<Schema> { - // writeReplace makes this object serializable. This is a limitation of FindBugs as discussed - // here: - // http://stackoverflow.com/questions/26156523/is-writeobject-not-neccesary-using-the-serialization-proxy-pattern - @SuppressFBWarnings("SE_BAD_FIELD") - private final Schema schema; - - private SerializableSchemaSupplier(Schema schema) { - this.schema = schema; - } - - private Object writeReplace() { - return new SerializableSchemaString(schema.toString()); - } - - @Override - public Schema get() { - return schema; - } - } - - /** - * A {@link Serializable} object that lazily supplies a {@link ReflectData} built from the - * appropriate {@link ClassLoader} for the type encoded by this {@link AvroCoder}. - */ - private static class SerializableReflectDataSupplier - implements Serializable, Supplier<ReflectData> { - - private final Class<?> clazz; - - private SerializableReflectDataSupplier(Class<?> clazz) { - this.clazz = clazz; - } - - @Override - public ReflectData get() { - ReflectData reflectData = new ReflectData(clazz.getClassLoader()); - reflectData.addLogicalTypeConversion(new JodaTimestampConversion()); - return reflectData; - } - } - - // Cache the old encoder/decoder and let the factories reuse them when possible. To be threadsafe, - // these are ThreadLocal. This code does not need to be re-entrant as AvroCoder does not use - // an inner coder. - private final EmptyOnDeserializationThreadLocal<BinaryDecoder> decoder; - private final EmptyOnDeserializationThreadLocal<BinaryEncoder> encoder; - private final EmptyOnDeserializationThreadLocal<DatumWriter<T>> writer; - private final EmptyOnDeserializationThreadLocal<DatumReader<T>> reader; - - // Lazily re-instantiated after deserialization - private final Supplier<ReflectData> reflectData; - - protected AvroCoder(Class<T> type, Schema schema) { - this(type, schema, false); - } - - protected AvroCoder(Class<T> type, Schema schema, boolean useReflectApi) { - this.type = type; - this.useReflectApi = useReflectApi; - this.schemaSupplier = new SerializableSchemaSupplier(schema); - typeDescriptor = TypeDescriptor.of(type); - nonDeterministicReasons = new AvroDeterminismChecker().check(TypeDescriptor.of(type), schema); - - // Decoder and Encoder start off null for each thread. They are allocated and potentially - // reused inside encode/decode. - this.decoder = new EmptyOnDeserializationThreadLocal<>(); - this.encoder = new EmptyOnDeserializationThreadLocal<>(); - - this.reflectData = Suppliers.memoize(new SerializableReflectDataSupplier(getType())); - - // Reader and writer are allocated once per thread per Coder - this.reader = - new EmptyOnDeserializationThreadLocal<DatumReader<T>>() { - private final AvroCoder<T> myCoder = AvroCoder.this; - - @Override - public DatumReader<T> initialValue() { - if (myCoder.getType().equals(GenericRecord.class)) { - return new GenericDatumReader<>(myCoder.getSchema()); - } else if (SpecificRecord.class.isAssignableFrom(myCoder.getType()) && !useReflectApi) { - return new SpecificDatumReader<>(myCoder.getType()); - } - return new ReflectDatumReader<>( - myCoder.getSchema(), myCoder.getSchema(), myCoder.reflectData.get()); - } - }; - - this.writer = - new EmptyOnDeserializationThreadLocal<DatumWriter<T>>() { - private final AvroCoder<T> myCoder = AvroCoder.this; - - @Override - public DatumWriter<T> initialValue() { - if (myCoder.getType().equals(GenericRecord.class)) { - return new GenericDatumWriter<>(myCoder.getSchema()); - } else if (SpecificRecord.class.isAssignableFrom(myCoder.getType()) && !useReflectApi) { - return new SpecificDatumWriter<>(myCoder.getType()); - } - return new ReflectDatumWriter<>(myCoder.getSchema(), myCoder.reflectData.get()); - } - }; - } - - /** Returns the type this coder encodes/decodes. */ - public Class<T> getType() { - return type; - } - - public boolean useReflectApi() { - return useReflectApi; - } - - @Override - public void encode(T value, OutputStream outStream) throws IOException { - // Get a BinaryEncoder instance from the ThreadLocal cache and attempt to reuse it. - BinaryEncoder encoderInstance = ENCODER_FACTORY.directBinaryEncoder(outStream, encoder.get()); - // Save the potentially-new instance for reuse later. - encoder.set(encoderInstance); - writer.get().write(value, encoderInstance); - // Direct binary encoder does not buffer any data and need not be flushed. - } - - @Override - public T decode(InputStream inStream) throws IOException { - // Get a BinaryDecoder instance from the ThreadLocal cache and attempt to reuse it. - BinaryDecoder decoderInstance = DECODER_FACTORY.directBinaryDecoder(inStream, decoder.get()); - // Save the potentially-new instance for later. - decoder.set(decoderInstance); - return reader.get().read(null, decoderInstance); - } - - /** - * @throws NonDeterministicException when the type may not be deterministically encoded using the - * given {@link Schema}, the {@code directBinaryEncoder}, and the {@link ReflectDatumWriter} - * or {@link GenericDatumWriter}. - */ - @Override - public void verifyDeterministic() throws NonDeterministicException { - if (!nonDeterministicReasons.isEmpty()) { - throw new NonDeterministicException(this, nonDeterministicReasons); - } - } - - /** Returns the schema used by this coder. */ - public Schema getSchema() { - return schemaSupplier.get(); - } - - @Override - public TypeDescriptor<T> getEncodedTypeDescriptor() { - return typeDescriptor; - } - - /** - * Helper class encapsulating the various pieces of state maintained by the recursive walk used - * for checking if the encoding will be deterministic. - */ - private static class AvroDeterminismChecker { - - // Reasons that the original type are not deterministic. This accumulates - // the actual output. - private List<String> reasons = new ArrayList<>(); - - // Types that are currently "open". Used to make sure we don't have any - // recursive types. Note that we assume that all occurrences of a given type - // are equal, rather than tracking pairs of type + schema. - private Set<TypeDescriptor<?>> activeTypes = new HashSet<>(); - - // Similarly to how we record active types, we record the schemas we visit - // to make sure we don't encounter recursive fields. - private Set<Schema> activeSchemas = new HashSet<>(); - - /** Report an error in the current context. */ - @FormatMethod - private void reportError(String context, @FormatString String fmt, Object... args) { - String message = String.format(fmt, args); - reasons.add(context + ": " + message); - } - - /** - * Classes that are serialized by Avro as a String include - * - * <ul> - * <li>Subtypes of CharSequence (including String, Avro's mutable Utf8, etc.) - * <li>Several predefined classes (BigDecimal, BigInteger, URI, URL) - * <li>Classes annotated with @Stringable (uses their #toString() and a String constructor) - * </ul> - * - * <p>Rather than determine which of these cases are deterministic, we list some classes that - * definitely are, and treat any others as non-deterministic. - */ - private static final Set<Class<?>> DETERMINISTIC_STRINGABLE_CLASSES = new HashSet<>(); - - static { - // CharSequences: - DETERMINISTIC_STRINGABLE_CLASSES.add(String.class); - DETERMINISTIC_STRINGABLE_CLASSES.add(Utf8.class); - - // Explicitly Stringable: - DETERMINISTIC_STRINGABLE_CLASSES.add(java.math.BigDecimal.class); - DETERMINISTIC_STRINGABLE_CLASSES.add(java.math.BigInteger.class); - DETERMINISTIC_STRINGABLE_CLASSES.add(java.net.URI.class); - DETERMINISTIC_STRINGABLE_CLASSES.add(java.net.URL.class); - - // Classes annotated with @Stringable: - } - - /** Return true if the given type token is a subtype of *any* of the listed parents. */ - private static boolean isSubtypeOf(TypeDescriptor<?> type, Class<?>... parents) { - for (Class<?> parent : parents) { - if (type.isSubtypeOf(TypeDescriptor.of(parent))) { - return true; - } - } - return false; - } - - protected AvroDeterminismChecker() {} - - // The entry point for the check. Should not be recursively called. - public List<String> check(TypeDescriptor<?> type, Schema schema) { - recurse(type.getRawType().getName(), type, schema); - return reasons; - } - - // This is the method that should be recursively called. It sets up the path - // and visited types correctly. - private void recurse(String context, TypeDescriptor<?> type, Schema schema) { - if (type.getRawType().isAnnotationPresent(AvroSchema.class)) { - reportError(context, "Custom schemas are not supported -- remove @AvroSchema."); - return; - } - - if (!activeTypes.add(type)) { - reportError(context, "%s appears recursively", type); - return; - } - - // If the record isn't a true class, but rather a GenericRecord, SpecificRecord, etc. - // with a specified schema, then we need to make the decision based on the generated - // implementations. - if (isSubtypeOf(type, IndexedRecord.class)) { - checkIndexedRecord(context, schema, null); - } else { - doCheck(context, type, schema); - } - - activeTypes.remove(type); - } - - private void doCheck(String context, TypeDescriptor<?> type, Schema schema) { - switch (schema.getType()) { - case ARRAY: - checkArray(context, type, schema); - break; - case ENUM: - // Enums should be deterministic, since they depend only on the ordinal. - break; - case FIXED: - // Depending on the implementation of GenericFixed, we don't know how - // the given field will be encoded. So, we assume that it isn't - // deterministic. - reportError(context, "FIXED encodings are not guaranteed to be deterministic"); - break; - case MAP: - checkMap(context, type, schema); - break; - case RECORD: - if (!(type.getType() instanceof Class)) { - reportError(context, "Cannot determine type from generic %s due to erasure", type); - return; - } - checkRecord(type, schema); - break; - case UNION: - checkUnion(context, type, schema); - break; - case STRING: - checkString(context, type); - break; - case BOOLEAN: - case BYTES: - case DOUBLE: - case INT: - case FLOAT: - case LONG: - case NULL: - // For types that Avro encodes using one of the above primitives, we assume they are - // deterministic. - break; - default: - // In any other case (eg., new types added to Avro) we cautiously return - // false. - reportError(context, "Unknown schema type %s may be non-deterministic", schema.getType()); - break; - } - } - - private void checkString(String context, TypeDescriptor<?> type) { - // For types that are encoded as strings, we need to make sure they're in an approved - // list. For other types that are annotated @Stringable, Avro will just use the - // #toString() methods, which has no guarantees of determinism. - if (!DETERMINISTIC_STRINGABLE_CLASSES.contains(type.getRawType())) { - reportError(context, "%s may not have deterministic #toString()", type); - } - } - - private static final Schema AVRO_NULL_SCHEMA = Schema.create(Schema.Type.NULL); - - private void checkUnion(String context, TypeDescriptor<?> type, Schema schema) { - final List<Schema> unionTypes = schema.getTypes(); - - if (!type.getRawType().isAnnotationPresent(Union.class)) { - // First check for @Nullable field, which shows up as a union of field type and null. - if (unionTypes.size() == 2 && unionTypes.contains(AVRO_NULL_SCHEMA)) { - // Find the Schema that is not NULL and recursively check that it is deterministic. - Schema nullableFieldSchema = - unionTypes.get(0).equals(AVRO_NULL_SCHEMA) ? unionTypes.get(1) : unionTypes.get(0); - doCheck(context, type, nullableFieldSchema); - return; - } - - // Otherwise report a schema error. - reportError(context, "Expected type %s to have @Union annotation", type); - return; - } - - // Errors associated with this union will use the base class as their context. - String baseClassContext = type.getRawType().getName(); - - // For a union, we need to make sure that each possible instantiation is deterministic. - for (Schema concrete : unionTypes) { - @SuppressWarnings("unchecked") - TypeDescriptor<?> unionType = TypeDescriptor.of(ReflectData.get().getClass(concrete)); - - recurse(baseClassContext, unionType, concrete); - } - } - - private void checkRecord(TypeDescriptor<?> type, Schema schema) { - // For a record, we want to make sure that all the fields are deterministic. - Class<?> clazz = type.getRawType(); - for (Schema.Field fieldSchema : schema.getFields()) { - Field field = getField(clazz, fieldSchema.name()); - String fieldContext = field.getDeclaringClass().getName() + "#" + field.getName(); - - if (field.isAnnotationPresent(AvroEncode.class)) { - reportError( - fieldContext, "Custom encoders may be non-deterministic -- remove @AvroEncode"); - continue; - } - - if (!IndexedRecord.class.isAssignableFrom(field.getType()) - && field.isAnnotationPresent(AvroSchema.class)) { - // TODO: We should be able to support custom schemas on POJO fields, but we shouldn't - // need to, so we just allow it in the case of IndexedRecords. - reportError( - fieldContext, "Custom schemas are only supported for subtypes of IndexedRecord."); - continue; - } - - TypeDescriptor<?> fieldType = type.resolveType(field.getGenericType()); - recurse(fieldContext, fieldType, fieldSchema.schema()); - } - } - - private void checkIndexedRecord( - String context, Schema schema, @Nullable String specificClassStr) { - - if (!activeSchemas.add(schema)) { - reportError(context, "%s appears recursively", schema.getName()); - return; - } - - switch (schema.getType()) { - case ARRAY: - // Generic Records use GenericData.Array to implement arrays, which is - // essentially an ArrayList, and therefore ordering is deterministic. - // The array is thus deterministic if the elements are deterministic. - checkIndexedRecord(context, schema.getElementType(), null); - break; - case ENUM: - // Enums are deterministic because they encode as a single integer. - break; - case FIXED: - // In the case of GenericRecords, FIXED is deterministic because it - // encodes/decodes as a Byte[]. - break; - case MAP: - reportError( - context, - "GenericRecord and SpecificRecords use a HashMap to represent MAPs," - + " so it is non-deterministic"); - break; - case RECORD: - for (Schema.Field field : schema.getFields()) { - checkIndexedRecord( - schema.getName() + "." + field.name(), - field.schema(), - field.getProp(SpecificData.CLASS_PROP)); - } - break; - case STRING: - // GenericDatumWriter#findStringClass will use a CharSequence or a String - // for each string, so it is deterministic. - - // SpecificCompiler#getStringType will use java.lang.String, org.apache.avro.util.Utf8, - // or java.lang.CharSequence, unless SpecificData.CLASS_PROP overrides that. - if (specificClassStr != null) { - Class<?> specificClass; - try { - specificClass = ClassUtils.forName(specificClassStr); - if (!DETERMINISTIC_STRINGABLE_CLASSES.contains(specificClass)) { - reportError( - context, - "Specific class %s is not known to be deterministic", - specificClassStr); - } - } catch (ClassNotFoundException e) { - reportError( - context, "Specific class %s is not known to be deterministic", specificClassStr); - } - } - break; - case UNION: - for (Schema subschema : schema.getTypes()) { - checkIndexedRecord(subschema.getName(), subschema, null); - } - break; - case BOOLEAN: - case BYTES: - case DOUBLE: - case INT: - case FLOAT: - case LONG: - case NULL: - // For types that Avro encodes using one of the above primitives, we assume they are - // deterministic. - break; - default: - reportError(context, "Unknown schema type %s may be non-deterministic", schema.getType()); - break; - } - - activeSchemas.remove(schema); - } - - private void checkMap(String context, TypeDescriptor<?> type, Schema schema) { - if (!isSubtypeOf(type, SortedMap.class)) { - reportError(context, "%s may not be deterministically ordered", type); - } - - // Avro (currently) asserts that all keys are strings. - // In case that changes, we double check that the key was a string: - Class<?> keyType = type.resolveType(Map.class.getTypeParameters()[0]).getRawType(); - if (!String.class.equals(keyType)) { - reportError(context, "map keys should be Strings, but was %s", keyType); - } - - recurse(context, type.resolveType(Map.class.getTypeParameters()[1]), schema.getValueType()); - } - - private void checkArray(String context, TypeDescriptor<?> type, Schema schema) { - TypeDescriptor<?> elementType = null; - if (type.isArray()) { - // The type is an array (with ordering)-> deterministic iff the element is deterministic. - elementType = type.getComponentType(); - } else if (isSubtypeOf(type, Collection.class)) { - if (isSubtypeOf(type, List.class, SortedSet.class)) { - // Ordered collection -> deterministic iff the element is deterministic - elementType = type.resolveType(Collection.class.getTypeParameters()[0]); - } else { - // Not an ordered collection -> not deterministic - reportError(context, "%s may not be deterministically ordered", type); - return; - } - } else { - // If it was an unknown type encoded as an array, be conservative and assume - // that we don't know anything about the order. - reportError(context, "encoding %s as an ARRAY was unexpected", type); - return; - } - - // If we get here, it's either a deterministically-ordered Collection, or - // an array. Either way, the type is deterministic iff the element type is - // deterministic. - recurse(context, elementType, schema.getElementType()); - } - - /** - * Extract a field from a class. We need to look at the declared fields so that we can see - * private fields. We may need to walk up to the parent to get classes from the parent. - */ - private static Field getField(Class<?> originalClazz, String name) { - Class<?> clazz = originalClazz; - while (clazz != null) { - for (Field field : clazz.getDeclaredFields()) { - AvroName avroName = field.getAnnotation(AvroName.class); - if (avroName != null && name.equals(avroName.value())) { - return field; - } else if (avroName == null && name.equals(field.getName())) { - return field; - } - } - clazz = clazz.getSuperclass(); - } - - throw new IllegalArgumentException("Unable to get field " + name + " from " + originalClazz); - } - } - - @Override - public boolean equals(@Nullable Object other) { - if (other == this) { - return true; - } - if (!(other instanceof AvroCoder)) { - return false; - } - AvroCoder<?> that = (AvroCoder<?>) other; - return Objects.equals(this.schemaSupplier.get(), that.schemaSupplier.get()) - && Objects.equals(this.typeDescriptor, that.typeDescriptor) - && this.useReflectApi == that.useReflectApi; - } - - @Override - public int hashCode() { - return Objects.hash(schemaSupplier.get(), typeDescriptor, useReflectApi); - } - - /** - * Conversion for DateTime. - * - * <p>This is a copy from Avro 1.8's TimestampConversion, which is renamed in Avro 1.9. Defining - * own copy gives flexibility for Beam Java SDK to work with Avro 1.8 and 1.9 at runtime. - * - * @see <a href="https://issues.apache.org/jira/browse/BEAM-9144">BEAM-9144: Beam's own Avro - * TimeConversion class in beam-sdk-java-core</a> - */ - public static class JodaTimestampConversion extends Conversion<DateTime> { - @Override - public Class<DateTime> getConvertedType() { - return DateTime.class; - } - - @Override - public String getLogicalTypeName() { - return "timestamp-millis"; - } - - @Override - public DateTime fromLong(Long millisFromEpoch, Schema schema, LogicalType type) { - return new DateTime(millisFromEpoch, DateTimeZone.UTC); - } - - @Override - public Long toLong(DateTime timestamp, Schema schema, LogicalType type) { - return timestamp.getMillis(); - } - } -} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/AvroGenericCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/AvroGenericCoder.java deleted file mode 100644 index 7d90206ce4c5..000000000000 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/AvroGenericCoder.java +++ /dev/null @@ -1,40 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.coders; - -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericRecord; - -/** - * AvroCoder specialisation for GenericRecord. - * - * @deprecated Avro related classes are deprecated in module <code>beam-sdks-java-core</code> and - * will be eventually removed. Please, migrate to a new module <code> - * beam-sdks-java-extensions-avro</code> by importing <code> - * org.apache.beam.sdk.extensions.avro.coders.AvroGenericCoder</code> instead of this one. - */ -@Deprecated -public class AvroGenericCoder extends AvroCoder<GenericRecord> { - AvroGenericCoder(Schema schema) { - super(GenericRecord.class, schema); - } - - public static AvroGenericCoder of(Schema schema) { - return new AvroGenericCoder(schema); - } -} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroIO.java deleted file mode 100644 index 5593d5e70ea9..000000000000 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroIO.java +++ /dev/null @@ -1,2031 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.io; - -import static org.apache.beam.sdk.io.FileIO.ReadMatches.DirectoryTreatment; -import static org.apache.beam.sdk.io.ReadAllViaFileBasedSource.ReadFileRangesFnExceptionHandler; -import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; -import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull; - -import com.google.auto.value.AutoValue; -import java.io.IOException; -import java.io.Serializable; -import java.nio.channels.Channels; -import java.nio.channels.WritableByteChannel; -import java.util.Map; -import org.apache.avro.Schema; -import org.apache.avro.file.CodecFactory; -import org.apache.avro.file.DataFileConstants; -import org.apache.avro.file.DataFileWriter; -import org.apache.avro.generic.GenericDatumWriter; -import org.apache.avro.generic.GenericRecord; -import org.apache.avro.generic.IndexedRecord; -import org.apache.avro.reflect.ReflectData; -import org.apache.avro.reflect.ReflectDatumWriter; -import org.apache.beam.sdk.coders.AvroCoder; -import org.apache.beam.sdk.coders.CannotProvideCoderException; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.CoderRegistry; -import org.apache.beam.sdk.coders.StringUtf8Coder; -import org.apache.beam.sdk.io.FileBasedSink.FilenamePolicy; -import org.apache.beam.sdk.io.FileIO.MatchConfiguration; -import org.apache.beam.sdk.io.FileIO.ReadableFile; -import org.apache.beam.sdk.io.fs.EmptyMatchTreatment; -import org.apache.beam.sdk.io.fs.ResourceId; -import org.apache.beam.sdk.options.ValueProvider; -import org.apache.beam.sdk.options.ValueProvider.NestedValueProvider; -import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider; -import org.apache.beam.sdk.schemas.utils.AvroUtils; -import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.transforms.SerializableFunction; -import org.apache.beam.sdk.transforms.SerializableFunctions; -import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition; -import org.apache.beam.sdk.transforms.display.DisplayData; -import org.apache.beam.sdk.values.PBegin; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.PDone; -import org.apache.beam.sdk.values.TypeDescriptors; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Function; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Supplier; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Suppliers; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps; -import org.checkerframework.checker.nullness.qual.Nullable; -import org.joda.time.Duration; - -/** - * {@link PTransform}s for reading and writing Avro files. - * - * <h2>Reading Avro files</h2> - * - * <p>To read a {@link PCollection} from one or more Avro files with the same schema known at - * pipeline construction time, use {@link #read}, using {@link AvroIO.Read#from} to specify the - * filename or filepattern to read from. If the filepatterns to be read are themselves in a {@link - * PCollection} you can use {@link FileIO} to match them and {@link AvroIO#readFiles} to read them. - * If the schema is unknown at pipeline construction time, use {@link #parseGenericRecords} or - * {@link #parseFilesGenericRecords}. - * - * <p>Many configuration options below apply to several or all of these transforms. - * - * <p>See {@link FileSystems} for information on supported file systems and filepatterns. - * - * <h3>Filepattern expansion and watching</h3> - * - * <p>By default, the filepatterns are expanded only once. {@link Read#watchForNewFiles} or the - * combination of {@link FileIO.Match#continuously(Duration, TerminationCondition)} and {@link - * AvroIO#readFiles(Class)} allow streaming of new files matching the filepattern(s). - * - * <p>By default, {@link #read} prohibits filepatterns that match no files, and {@link - * AvroIO#readFiles(Class)} allows them in case the filepattern contains a glob wildcard character. - * Use {@link Read#withEmptyMatchTreatment} or {@link - * FileIO.Match#withEmptyMatchTreatment(EmptyMatchTreatment)} plus {@link AvroIO#readFiles(Class)} - * to configure this behavior. - * - * <h3>Reading records of a known schema</h3> - * - * <p>To read specific records, such as Avro-generated classes, use {@link #read(Class)}. To read - * {@link GenericRecord GenericRecords}, use {@link #readGenericRecords(Schema)} which takes a - * {@link Schema} object, or {@link #readGenericRecords(String)} which takes an Avro schema in a - * JSON-encoded string form. An exception will be thrown if a record doesn't match the specified - * schema. Likewise, to read a {@link PCollection} of filepatterns, apply {@link FileIO} matching - * plus {@link #readFilesGenericRecords}. - * - * <p>For example: - * - * <pre>{@code - * Pipeline p = ...; - * - * // Read Avro-generated classes from files on GCS - * PCollection<AvroAutoGenClass> records = - * p.apply(AvroIO.read(AvroAutoGenClass.class).from("gs://my_bucket/path/to/records-*.avro")); - * - * // Read GenericRecord's of the given schema from files on GCS - * Schema schema = new Schema.Parser().parse(new File("schema.avsc")); - * PCollection<GenericRecord> records = - * p.apply(AvroIO.readGenericRecords(schema) - * .from("gs://my_bucket/path/to/records-*.avro")); - * }</pre> - * - * <h3>Reading records of an unknown schema</h3> - * - * <p>To read records from files whose schema is unknown at pipeline construction time or differs - * between files, use {@link #parseGenericRecords} - in this case, you will need to specify a - * parsing function for converting each {@link GenericRecord} into a value of your custom type. - * Likewise, to read a {@link PCollection} of filepatterns with unknown schema, use {@link FileIO} - * matching plus {@link #parseFilesGenericRecords(SerializableFunction)}. - * - * <p>For example: - * - * <pre>{@code - * Pipeline p = ...; - * - * PCollection<Foo> records = - * p.apply(AvroIO.parseGenericRecords(new SerializableFunction<GenericRecord, Foo>() { - * public Foo apply(GenericRecord record) { - * // If needed, access the schema of the record using record.getSchema() - * return ...; - * } - * })); - * }</pre> - * - * <h3>Reading from a {@link PCollection} of filepatterns</h3> - * - * <pre>{@code - * Pipeline p = ...; - * - * PCollection<String> filepatterns = p.apply(...); - * PCollection<AvroAutoGenClass> records = - * filepatterns.apply(AvroIO.readAll(AvroAutoGenClass.class)); - * PCollection<AvroAutoGenClass> records = - * filepatterns - * .apply(FileIO.matchAll()) - * .apply(FileIO.readMatches()) - * .apply(AvroIO.readFiles(AvroAutoGenClass.class)); - * PCollection<GenericRecord> genericRecords = - * filepatterns.apply(AvroIO.readGenericRecords(schema)); - * PCollection<Foo> records = - * filepatterns - * .apply(FileIO.matchAll()) - * .apply(FileIO.readMatches()) - * .apply(AvroIO.parseFilesGenericRecords(new SerializableFunction...); - * }</pre> - * - * <h3>Streaming new files matching a filepattern</h3> - * - * <pre>{@code - * Pipeline p = ...; - * - * PCollection<AvroAutoGenClass> lines = p.apply(AvroIO - * .read(AvroAutoGenClass.class) - * .from("gs://my_bucket/path/to/records-*.avro") - * .watchForNewFiles( - * // Check for new files every minute - * Duration.standardMinutes(1), - * // Stop watching the filepattern if no new files appear within an hour - * afterTimeSinceNewOutput(Duration.standardHours(1)))); - * }</pre> - * - * <h3>Reading a very large number of files</h3> - * - * <p>If it is known that the filepattern will match a very large number of files (e.g. tens of - * thousands or more), use {@link Read#withHintMatchesManyFiles} for better performance and - * scalability. Note that it may decrease performance if the filepattern matches only a small number - * of files. - * - * <h3>Inferring Beam schemas from Avro files</h3> - * - * <p>If you want to use SQL or schema based operations on an Avro-based PCollection, you must - * configure the read transform to infer the Beam schema and automatically setup the Beam related - * coders by doing: - * - * <pre>{@code - * PCollection<AvroAutoGenClass> records = - * p.apply(AvroIO.read(...).from(...).withBeamSchemas(true)); - * }</pre> - * - * <h3>Inferring Beam schemas from Avro PCollections</h3> - * - * <p>If you created an Avro-based PCollection by other means e.g. reading records from Kafka or as - * the output of another PTransform, you may be interested on making your PCollection schema-aware - * so you can use the Schema-based APIs or Beam's SqlTransform. - * - * <p>If you are using Avro specific records (generated classes from an Avro schema), you can - * register a schema provider for the specific Avro class to make any PCollection of these objects - * schema-aware. - * - * <pre>{@code - * pipeline.getSchemaRegistry().registerSchemaProvider(AvroAutoGenClass.class, AvroAutoGenClass.getClassSchema()); - * }</pre> - * - * You can also manually set an Avro-backed Schema coder for a PCollection using {@link - * org.apache.beam.sdk.schemas.utils.AvroUtils#schemaCoder(Class, Schema)} to make it schema-aware. - * - * <pre>{@code - * PCollection<AvroAutoGenClass> records = ... - * AvroCoder<AvroAutoGenClass> coder = (AvroCoder<AvroAutoGenClass>) users.getCoder(); - * records.setCoder(AvroUtils.schemaCoder(coder.getType(), coder.getSchema())); - * }</pre> - * - * <p>If you are using GenericRecords you may need to set a specific Beam schema coder for each - * PCollection to match their internal Avro schema. - * - * <pre>{@code - * org.apache.avro.Schema avroSchema = ... - * PCollection<GenericRecord> records = ... - * records.setCoder(AvroUtils.schemaCoder(avroSchema)); - * }</pre> - * - * <h2>Writing Avro files</h2> - * - * <p>To write a {@link PCollection} to one or more Avro files, use {@link AvroIO.Write}, using - * {@code AvroIO.write().to(String)} to specify the output filename prefix. The default {@link - * DefaultFilenamePolicy} will use this prefix, in conjunction with a {@link ShardNameTemplate} (set - * via {@link Write#withShardNameTemplate(String)}) and optional filename suffix (set via {@link - * Write#withSuffix(String)}, to generate output filenames in a sharded way. You can override this - * default write filename policy using {@link Write#to(FileBasedSink.FilenamePolicy)} to specify a - * custom file naming policy. - * - * <p>By default, {@link AvroIO.Write} produces output files that are compressed using the {@link - * org.apache.avro.file.Codec CodecFactory.snappyCodec()}. This default can be changed or overridden - * using {@link AvroIO.Write#withCodec}. - * - * <h3>Writing specific or generic records</h3> - * - * <p>To write specific records, such as Avro-generated classes, use {@link #write(Class)}. To write - * {@link GenericRecord GenericRecords}, use either {@link #writeGenericRecords(Schema)} which takes - * a {@link Schema} object, or {@link #writeGenericRecords(String)} which takes a schema in a - * JSON-encoded string form. An exception will be thrown if a record doesn't match the specified - * schema. - * - * <p>For example: - * - * <pre>{@code - * // A simple Write to a local file (only runs locally): - * PCollection<AvroAutoGenClass> records = ...; - * records.apply(AvroIO.write(AvroAutoGenClass.class).to("/path/to/file.avro")); - * - * // A Write to a sharded GCS file (runs locally and using remote execution): - * Schema schema = new Schema.Parser().parse(new File("schema.avsc")); - * PCollection<GenericRecord> records = ...; - * records.apply("WriteToAvro", AvroIO.writeGenericRecords(schema) - * .to("gs://my_bucket/path/to/numbers") - * .withSuffix(".avro")); - * }</pre> - * - * <h3>Writing windowed or unbounded data</h3> - * - * <p>By default, all input is put into the global window before writing. If per-window writes are - * desired - for example, when using a streaming runner - {@link AvroIO.Write#withWindowedWrites()} - * will cause windowing and triggering to be preserved. When producing windowed writes with a - * streaming runner that supports triggers, the number of output shards must be set explicitly using - * {@link AvroIO.Write#withNumShards(int)}; some runners may set this for you to a runner-chosen - * value, so you may need not set it yourself. A {@link FileBasedSink.FilenamePolicy} must be set, - * and unique windows and triggers must produce unique filenames. - * - * <h3>Writing data to multiple destinations</h3> - * - * <p>The following shows a more-complex example of AvroIO.Write usage, generating dynamic file - * destinations as well as a dynamic Avro schema per file. In this example, a PCollection of user - * events (e.g. actions on a website) is written out to Avro files. Each event contains the user id - * as an integer field. We want events for each user to go into a specific directory for that user, - * and each user's data should be written with a specific schema for that user; a side input is - * used, so the schema can be calculated in a different stage. - * - * <pre>{@code - * // This is the user class that controls dynamic destinations for this avro write. The input to - * // AvroIO.Write will be UserEvent, and we will be writing GenericRecords to the file (in order - * // to have dynamic schemas). Everything is per userid, so we define a dynamic destination type - * // of Integer. - * class UserDynamicAvroDestinations - * extends DynamicAvroDestinations<UserEvent, Integer, GenericRecord> { - * private final PCollectionView<Map<Integer, String>> userToSchemaMap; - * public UserDynamicAvroDestinations( PCollectionView<Map<Integer, String>> userToSchemaMap) { - * this.userToSchemaMap = userToSchemaMap; - * } - * public GenericRecord formatRecord(UserEvent record) { - * return formatUserRecord(record, getSchema(record.getUserId())); - * } - * public Schema getSchema(Integer userId) { - * return new Schema.Parser().parse(sideInput(userToSchemaMap).get(userId)); - * } - * public Integer getDestination(UserEvent record) { - * return record.getUserId(); - * } - * public Integer getDefaultDestination() { - * return 0; - * } - * public FilenamePolicy getFilenamePolicy(Integer userId) { - * return DefaultFilenamePolicy.fromParams(new Params().withBaseFilename(baseDir + "/user-" - * + userId + "/events")); - * } - * public List<PCollectionView<?>> getSideInputs() { - * return ImmutableList.<PCollectionView<?>>of(userToSchemaMap); - * } - * } - * PCollection<UserEvents> events = ...; - * PCollectionView<Map<Integer, String>> userToSchemaMap = events.apply( - * "ComputePerUserSchemas", new ComputePerUserSchemas()); - * events.apply("WriteAvros", AvroIO.<Integer>writeCustomTypeToGenericRecords() - * .to(new UserDynamicAvroDestinations(userToSchemaMap))); - * }</pre> - * - * @deprecated Avro related classes are deprecated in module <code>beam-sdks-java-core</code> and - * will be eventually removed. Please, migrate to a new module <code> - * beam-sdks-java-extensions-avro</code> by importing <code> - * org.apache.beam.sdk.extensions.avro.io.AvroIO</code> instead of this one. - */ -@SuppressWarnings({ - "nullness" // TODO(https://github.com/apache/beam/issues/20497) -}) -@Deprecated -public class AvroIO { - /** - * Reads records of the given type from an Avro file (or multiple Avro files matching a pattern). - * - * <p>The schema must be specified using one of the {@code withSchema} functions. - */ - public static <T> Read<T> read(Class<T> recordClass) { - return new AutoValue_AvroIO_Read.Builder<T>() - .setMatchConfiguration(MatchConfiguration.create(EmptyMatchTreatment.DISALLOW)) - .setRecordClass(recordClass) - .setSchema(ReflectData.get().getSchema(recordClass)) - .setInferBeamSchema(false) - .setHintMatchesManyFiles(false) - .build(); - } - - /** - * Like {@link #read}, but reads each file in a {@link PCollection} of {@link ReadableFile}, - * returned by {@link FileIO#readMatches}. - * - * <p>You can read {@link GenericRecord} by using {@code #readFiles(GenericRecord.class)} or - * {@code #readFiles(new Schema.Parser().parse(schema))} if the schema is a String. - */ - public static <T> ReadFiles<T> readFiles(Class<T> recordClass) { - return new AutoValue_AvroIO_ReadFiles.Builder<T>() - .setRecordClass(recordClass) - .setSchema(ReflectData.get().getSchema(recordClass)) - .setInferBeamSchema(false) - .setDesiredBundleSizeBytes(DEFAULT_BUNDLE_SIZE_BYTES) - .setUsesReshuffle(ReadAllViaFileBasedSource.DEFAULT_USES_RESHUFFLE) - .setFileExceptionHandler(new ReadFileRangesFnExceptionHandler()) - .build(); - } - - /** - * Like {@link #read}, but reads each filepattern in the input {@link PCollection}. - * - * @deprecated You can achieve The functionality of {@link #readAll} using {@link FileIO} matching - * plus {@link #readFiles(Class)}. This is the preferred method to make composition explicit. - * {@link ReadAll} will not receive upgrades and will be removed in a future version of Beam. - */ - @Deprecated - public static <T> ReadAll<T> readAll(Class<T> recordClass) { - return new AutoValue_AvroIO_ReadAll.Builder<T>() - .setMatchConfiguration(MatchConfiguration.create(EmptyMatchTreatment.ALLOW_IF_WILDCARD)) - .setRecordClass(recordClass) - .setSchema(ReflectData.get().getSchema(recordClass)) - .setInferBeamSchema(false) - .setDesiredBundleSizeBytes(DEFAULT_BUNDLE_SIZE_BYTES) - .build(); - } - - /** Reads Avro file(s) containing records of the specified schema. */ - public static Read<GenericRecord> readGenericRecords(Schema schema) { - return new AutoValue_AvroIO_Read.Builder<GenericRecord>() - .setMatchConfiguration(MatchConfiguration.create(EmptyMatchTreatment.DISALLOW)) - .setRecordClass(GenericRecord.class) - .setSchema(schema) - .setInferBeamSchema(false) - .setHintMatchesManyFiles(false) - .build(); - } - - /** - * Like {@link #readGenericRecords(Schema)}, but for a {@link PCollection} of {@link - * ReadableFile}, for example, returned by {@link FileIO#readMatches}. - */ - public static ReadFiles<GenericRecord> readFilesGenericRecords(Schema schema) { - return new AutoValue_AvroIO_ReadFiles.Builder<GenericRecord>() - .setRecordClass(GenericRecord.class) - .setSchema(schema) - .setInferBeamSchema(false) - .setDesiredBundleSizeBytes(DEFAULT_BUNDLE_SIZE_BYTES) - .setUsesReshuffle(ReadAllViaFileBasedSource.DEFAULT_USES_RESHUFFLE) - .setFileExceptionHandler(new ReadFileRangesFnExceptionHandler()) - .build(); - } - - /** - * Like {@link #readGenericRecords(Schema)}, but for a {@link PCollection} of {@link - * ReadableFile}, for example, returned by {@link FileIO#readMatches}. - * - * @deprecated You can achieve The functionality of {@link #readAllGenericRecords(Schema)} using - * {@link FileIO} matching plus {@link #readFilesGenericRecords(Schema)}. This is the - * preferred method to make composition explicit. {@link ReadAll} will not receive upgrades - * and will be removed in a future version of Beam. - */ - @Deprecated - public static ReadAll<GenericRecord> readAllGenericRecords(Schema schema) { - return new AutoValue_AvroIO_ReadAll.Builder<GenericRecord>() - .setMatchConfiguration(MatchConfiguration.create(EmptyMatchTreatment.ALLOW_IF_WILDCARD)) - .setRecordClass(GenericRecord.class) - .setSchema(schema) - .setInferBeamSchema(false) - .setDesiredBundleSizeBytes(DEFAULT_BUNDLE_SIZE_BYTES) - .build(); - } - - /** - * Reads Avro file(s) containing records of the specified schema. The schema is specified as a - * JSON-encoded string. - */ - public static Read<GenericRecord> readGenericRecords(String schema) { - return readGenericRecords(new Schema.Parser().parse(schema)); - } - - /** Like {@link #readGenericRecords(String)}, but for {@link ReadableFile} collections. */ - public static ReadFiles<GenericRecord> readFilesGenericRecords(String schema) { - return readFilesGenericRecords(new Schema.Parser().parse(schema)); - } - - /** - * Like {@link #readGenericRecords(String)}, but reads each filepattern in the input {@link - * PCollection}. - * - * @deprecated You can achieve The functionality of {@link #readAllGenericRecords(String)} using - * {@link FileIO} matching plus {@link #readFilesGenericRecords(String)}. This is the - * preferred method to make composition explicit. {@link ReadAll} will not receive upgrades - * and will be removed in a future version of Beam. - */ - @Deprecated - public static ReadAll<GenericRecord> readAllGenericRecords(String schema) { - return readAllGenericRecords(new Schema.Parser().parse(schema)); - } - - /** - * Reads Avro file(s) containing records of an unspecified schema and converting each record to a - * custom type. - */ - public static <T> Parse<T> parseGenericRecords(SerializableFunction<GenericRecord, T> parseFn) { - return new AutoValue_AvroIO_Parse.Builder<T>() - .setMatchConfiguration(MatchConfiguration.create(EmptyMatchTreatment.DISALLOW)) - .setParseFn(parseFn) - .setHintMatchesManyFiles(false) - .build(); - } - - /** - * Like {@link #parseGenericRecords(SerializableFunction)}, but reads each {@link ReadableFile} in - * the input {@link PCollection}. - */ - public static <T> ParseFiles<T> parseFilesGenericRecords( - SerializableFunction<GenericRecord, T> parseFn) { - return new AutoValue_AvroIO_ParseFiles.Builder<T>() - .setParseFn(parseFn) - .setDesiredBundleSizeBytes(DEFAULT_BUNDLE_SIZE_BYTES) - .setUsesReshuffle(ReadAllViaFileBasedSource.DEFAULT_USES_RESHUFFLE) - .setFileExceptionHandler(new ReadFileRangesFnExceptionHandler()) - .build(); - } - - /** - * Like {@link #parseGenericRecords(SerializableFunction)}, but reads each filepattern in the - * input {@link PCollection}. - * - * @deprecated You can achieve The functionality of {@link - * #parseAllGenericRecords(SerializableFunction)} using {@link FileIO} matching plus {@link - * #parseFilesGenericRecords(SerializableFunction)} ()}. This is the preferred method to make - * composition explicit. {@link ParseAll} will not receive upgrades and will be removed in a - * future version of Beam. - */ - @Deprecated - public static <T> ParseAll<T> parseAllGenericRecords( - SerializableFunction<GenericRecord, T> parseFn) { - return new AutoValue_AvroIO_ParseAll.Builder<T>() - .setMatchConfiguration(MatchConfiguration.create(EmptyMatchTreatment.ALLOW_IF_WILDCARD)) - .setParseFn(parseFn) - .setDesiredBundleSizeBytes(DEFAULT_BUNDLE_SIZE_BYTES) - .build(); - } - - /** - * Writes a {@link PCollection} to an Avro file (or multiple Avro files matching a sharding - * pattern). - */ - public static <T> Write<T> write(Class<T> recordClass) { - return new Write<>( - AvroIO.<T, T>defaultWriteBuilder() - .setGenericRecords(false) - .setSchema(ReflectData.get().getSchema(recordClass)) - .build()); - } - - /** Writes Avro records of the specified schema. */ - public static Write<GenericRecord> writeGenericRecords(Schema schema) { - return new Write<>( - AvroIO.<GenericRecord, GenericRecord>defaultWriteBuilder() - .setGenericRecords(true) - .setSchema(schema) - .build()); - } - - /** - * A {@link PTransform} that writes a {@link PCollection} to an avro file (or multiple avro files - * matching a sharding pattern), with each element of the input collection encoded into its own - * record of type OutputT. - * - * <p>This version allows you to apply {@link AvroIO} writes to a PCollection of a custom type - * {@link UserT}. A format mechanism that converts the input type {@link UserT} to the output type - * that will be written to the file must be specified. If using a custom {@link - * DynamicAvroDestinations} object this is done using {@link - * DynamicAvroDestinations#formatRecord}, otherwise the {@link - * AvroIO.TypedWrite#withFormatFunction} can be used to specify a format function. - * - * <p>The advantage of using a custom type is that is it allows a user-provided {@link - * DynamicAvroDestinations} object, set via {@link AvroIO.Write#to(DynamicAvroDestinations)} to - * examine the custom type when choosing a destination. - * - * <p>If the output type is {@link GenericRecord} use {@link #writeCustomTypeToGenericRecords()} - * instead. - */ - public static <UserT, OutputT> TypedWrite<UserT, Void, OutputT> writeCustomType() { - return AvroIO.<UserT, OutputT>defaultWriteBuilder().setGenericRecords(false).build(); - } - - /** - * Similar to {@link #writeCustomType()}, but specialized for the case where the output type is - * {@link GenericRecord}. A schema must be specified either in {@link - * DynamicAvroDestinations#getSchema} or if not using dynamic destinations, by using {@link - * TypedWrite#withSchema(Schema)}. - */ - public static <UserT> TypedWrite<UserT, Void, GenericRecord> writeCustomTypeToGenericRecords() { - return AvroIO.<UserT, GenericRecord>defaultWriteBuilder().setGenericRecords(true).build(); - } - - /** - * Writes Avro records of the specified schema. The schema is specified as a JSON-encoded string. - */ - public static Write<GenericRecord> writeGenericRecords(String schema) { - return writeGenericRecords(new Schema.Parser().parse(schema)); - } - - private static <UserT, OutputT> TypedWrite.Builder<UserT, Void, OutputT> defaultWriteBuilder() { - return new AutoValue_AvroIO_TypedWrite.Builder<UserT, Void, OutputT>() - .setFilenameSuffix(null) - .setShardTemplate(null) - .setNumShards(0) - .setCodec(TypedWrite.DEFAULT_SERIALIZABLE_CODEC) - .setMetadata(ImmutableMap.of()) - .setWindowedWrites(false) - .setNoSpilling(false) - .setSyncInterval(DataFileConstants.DEFAULT_SYNC_INTERVAL); - } - - private static <T> PCollection<T> setBeamSchema( - PCollection<T> pc, Class<T> clazz, @Nullable Schema schema) { - return pc.setCoder(AvroUtils.schemaCoder(clazz, schema)); - } - - /** - * 64MB is a reasonable value that allows to amortize the cost of opening files, but is not so - * large as to exhaust a typical runner's maximum amount of output per ProcessElement call. - */ - private static final long DEFAULT_BUNDLE_SIZE_BYTES = 64 * 1024 * 1024L; - - /** Implementation of {@link #read} and {@link #readGenericRecords}. */ - @AutoValue - public abstract static class Read<T> extends PTransform<PBegin, PCollection<T>> { - - abstract @Nullable ValueProvider<String> getFilepattern(); - - abstract MatchConfiguration getMatchConfiguration(); - - abstract @Nullable Class<T> getRecordClass(); - - abstract @Nullable Schema getSchema(); - - abstract boolean getInferBeamSchema(); - - abstract boolean getHintMatchesManyFiles(); - - abstract Builder<T> toBuilder(); - - @AutoValue.Builder - abstract static class Builder<T> { - abstract Builder<T> setFilepattern(ValueProvider<String> filepattern); - - abstract Builder<T> setMatchConfiguration(MatchConfiguration matchConfiguration); - - abstract Builder<T> setRecordClass(Class<T> recordClass); - - abstract Builder<T> setSchema(Schema schema); - - abstract Builder<T> setInferBeamSchema(boolean infer); - - abstract Builder<T> setHintMatchesManyFiles(boolean hintManyFiles); - - abstract Read<T> build(); - } - - /** - * Reads from the given filename or filepattern. - * - * <p>If it is known that the filepattern will match a very large number of files (at least tens - * of thousands), use {@link #withHintMatchesManyFiles} for better performance and scalability. - */ - public Read<T> from(ValueProvider<String> filepattern) { - return toBuilder().setFilepattern(filepattern).build(); - } - - /** Like {@link #from(ValueProvider)}. */ - public Read<T> from(String filepattern) { - return from(StaticValueProvider.of(filepattern)); - } - - /** Sets the {@link MatchConfiguration}. */ - public Read<T> withMatchConfiguration(MatchConfiguration matchConfiguration) { - return toBuilder().setMatchConfiguration(matchConfiguration).build(); - } - - /** Configures whether or not a filepattern matching no files is allowed. */ - public Read<T> withEmptyMatchTreatment(EmptyMatchTreatment treatment) { - return withMatchConfiguration(getMatchConfiguration().withEmptyMatchTreatment(treatment)); - } - - /** - * Continuously watches for new files matching the filepattern, polling it at the given - * interval, until the given termination condition is reached. The returned {@link PCollection} - * is unbounded. If {@code matchUpdatedFiles} is set, also watches for files with timestamp - * change. - * - * <p>This works only in runners supporting splittable {@link - * org.apache.beam.sdk.transforms.DoFn}. - */ - public Read<T> watchForNewFiles( - Duration pollInterval, - TerminationCondition<String, ?> terminationCondition, - boolean matchUpdatedFiles) { - return withMatchConfiguration( - getMatchConfiguration() - .continuously(pollInterval, terminationCondition, matchUpdatedFiles)); - } - - /** - * Same as {@link Read#watchForNewFiles(Duration, TerminationCondition, boolean)} with {@code - * matchUpdatedFiles=false}. - */ - public Read<T> watchForNewFiles( - Duration pollInterval, TerminationCondition<String, ?> terminationCondition) { - return watchForNewFiles(pollInterval, terminationCondition, false); - } - - /** - * Hints that the filepattern specified in {@link #from(String)} matches a very large number of - * files. - * - * <p>This hint may cause a runner to execute the transform differently, in a way that improves - * performance for this case, but it may worsen performance if the filepattern matches only a - * small number of files (e.g., in a runner that supports dynamic work rebalancing, it will - * happen less efficiently within individual files). - */ - public Read<T> withHintMatchesManyFiles() { - return toBuilder().setHintMatchesManyFiles(true).build(); - } - - /** - * If set to true, a Beam schema will be inferred from the AVRO schema. This allows the output - * to be used by SQL and by the schema-transform library. - */ - public Read<T> withBeamSchemas(boolean withBeamSchemas) { - return toBuilder().setInferBeamSchema(withBeamSchemas).build(); - } - - @Override - @SuppressWarnings("unchecked") - public PCollection<T> expand(PBegin input) { - checkNotNull(getFilepattern(), "filepattern"); - checkNotNull(getSchema(), "schema"); - - if (getMatchConfiguration().getWatchInterval() == null && !getHintMatchesManyFiles()) { - PCollection<T> read = - input.apply( - "Read", - org.apache.beam.sdk.io.Read.from( - createSource( - getFilepattern(), - getMatchConfiguration().getEmptyMatchTreatment(), - getRecordClass(), - getSchema(), - null))); - return getInferBeamSchema() ? setBeamSchema(read, getRecordClass(), getSchema()) : read; - } - - // All other cases go through FileIO + ReadFiles - ReadFiles<T> readFiles = - (getRecordClass() == GenericRecord.class) - ? (ReadFiles<T>) readFilesGenericRecords(getSchema()) - : readFiles(getRecordClass()); - return input - .apply("Create filepattern", Create.ofProvider(getFilepattern(), StringUtf8Coder.of())) - .apply("Match All", FileIO.matchAll().withConfiguration(getMatchConfiguration())) - .apply( - "Read Matches", - FileIO.readMatches().withDirectoryTreatment(DirectoryTreatment.PROHIBIT)) - .apply("Via ReadFiles", readFiles); - } - - @Override - public void populateDisplayData(DisplayData.Builder builder) { - super.populateDisplayData(builder); - builder - .add( - DisplayData.item("inferBeamSchema", getInferBeamSchema()) - .withLabel("Infer Beam Schema")) - .addIfNotNull(DisplayData.item("schema", String.valueOf(getSchema()))) - .addIfNotNull(DisplayData.item("recordClass", getRecordClass()).withLabel("Record Class")) - .addIfNotNull( - DisplayData.item("filePattern", getFilepattern()).withLabel("Input File Pattern")) - .include("matchConfiguration", getMatchConfiguration()); - } - - @SuppressWarnings("unchecked") - private static <T> AvroSource<T> createSource( - ValueProvider<String> filepattern, - EmptyMatchTreatment emptyMatchTreatment, - Class<T> recordClass, - Schema schema, - AvroSource.@Nullable DatumReaderFactory<T> readerFactory) { - AvroSource<?> source = - AvroSource.from(filepattern).withEmptyMatchTreatment(emptyMatchTreatment); - - if (readerFactory != null) { - source = source.withDatumReaderFactory(readerFactory); - } - return recordClass == GenericRecord.class - ? (AvroSource<T>) source.withSchema(schema) - : source.withSchema(recordClass); - } - } - - ///////////////////////////////////////////////////////////////////////////// - - /** Implementation of {@link #readFiles}. */ - @AutoValue - public abstract static class ReadFiles<T> - extends PTransform<PCollection<ReadableFile>, PCollection<T>> { - - abstract @Nullable Class<T> getRecordClass(); - - abstract @Nullable Schema getSchema(); - - abstract boolean getUsesReshuffle(); - - abstract ReadFileRangesFnExceptionHandler getFileExceptionHandler(); - - abstract long getDesiredBundleSizeBytes(); - - abstract boolean getInferBeamSchema(); - - abstract AvroSource.@Nullable DatumReaderFactory<T> getDatumReaderFactory(); - - abstract Builder<T> toBuilder(); - - @AutoValue.Builder - abstract static class Builder<T> { - abstract Builder<T> setRecordClass(Class<T> recordClass); - - abstract Builder<T> setSchema(Schema schema); - - abstract Builder<T> setUsesReshuffle(boolean usesReshuffle); - - abstract Builder<T> setFileExceptionHandler( - ReadFileRangesFnExceptionHandler exceptionHandler); - - abstract Builder<T> setDesiredBundleSizeBytes(long desiredBundleSizeBytes); - - abstract Builder<T> setInferBeamSchema(boolean infer); - - abstract Builder<T> setDatumReaderFactory(AvroSource.DatumReaderFactory<T> factory); - - abstract ReadFiles<T> build(); - } - - /** - * Set a value for the bundle size for parallel reads. Default is 64 MB. You may want to use a - * lower value (e.g. 1 MB) for streaming applications. - */ - public ReadFiles<T> withDesiredBundleSizeBytes(long desiredBundleSizeBytes) { - return toBuilder().setDesiredBundleSizeBytes(desiredBundleSizeBytes).build(); - } - - /** Specifies if a Reshuffle should run before file reads occur. */ - public ReadFiles<T> withUsesReshuffle(boolean usesReshuffle) { - return toBuilder().setUsesReshuffle(usesReshuffle).build(); - } - - /** Specifies if exceptions should be logged only for streaming pipelines. */ - public ReadFiles<T> withFileExceptionHandler( - ReadFileRangesFnExceptionHandler exceptionHandler) { - return toBuilder().setFileExceptionHandler(exceptionHandler).build(); - } - - /** - * If set to true, a Beam schema will be inferred from the AVRO schema. This allows the output - * to be used by SQL and by the schema-transform library. - */ - public ReadFiles<T> withBeamSchemas(boolean withBeamSchemas) { - return toBuilder().setInferBeamSchema(withBeamSchemas).build(); - } - - public ReadFiles<T> withDatumReaderFactory(AvroSource.DatumReaderFactory<T> factory) { - return toBuilder().setDatumReaderFactory(factory).build(); - } - - @Override - public PCollection<T> expand(PCollection<ReadableFile> input) { - checkNotNull(getSchema(), "schema"); - PCollection<T> read = - input.apply( - "Read all via FileBasedSource", - new ReadAllViaFileBasedSource<>( - getDesiredBundleSizeBytes(), - new CreateSourceFn<>( - getRecordClass(), getSchema().toString(), getDatumReaderFactory()), - AvroCoder.of(getRecordClass(), getSchema()), - getUsesReshuffle(), - getFileExceptionHandler())); - return getInferBeamSchema() ? setBeamSchema(read, getRecordClass(), getSchema()) : read; - } - - @Override - public void populateDisplayData(DisplayData.Builder builder) { - super.populateDisplayData(builder); - builder - .add( - DisplayData.item("inferBeamSchema", getInferBeamSchema()) - .withLabel("Infer Beam Schema")) - .addIfNotNull(DisplayData.item("schema", String.valueOf(getSchema()))) - .addIfNotNull( - DisplayData.item("recordClass", getRecordClass()).withLabel("Record Class")); - } - } - - ///////////////////////////////////////////////////////////////////////////// - - /** - * Implementation of {@link #readAll}. - * - * @deprecated See {@link #readAll(Class)} for details. - */ - @Deprecated - @AutoValue - public abstract static class ReadAll<T> extends PTransform<PCollection<String>, PCollection<T>> { - abstract MatchConfiguration getMatchConfiguration(); - - abstract @Nullable Class<T> getRecordClass(); - - abstract @Nullable Schema getSchema(); - - abstract long getDesiredBundleSizeBytes(); - - abstract boolean getInferBeamSchema(); - - abstract Builder<T> toBuilder(); - - @AutoValue.Builder - abstract static class Builder<T> { - abstract Builder<T> setMatchConfiguration(MatchConfiguration matchConfiguration); - - abstract Builder<T> setRecordClass(Class<T> recordClass); - - abstract Builder<T> setSchema(Schema schema); - - abstract Builder<T> setDesiredBundleSizeBytes(long desiredBundleSizeBytes); - - abstract Builder<T> setInferBeamSchema(boolean infer); - - abstract ReadAll<T> build(); - } - - /** Sets the {@link MatchConfiguration}. */ - public ReadAll<T> withMatchConfiguration(MatchConfiguration configuration) { - return toBuilder().setMatchConfiguration(configuration).build(); - } - - /** Like {@link Read#withEmptyMatchTreatment}. */ - public ReadAll<T> withEmptyMatchTreatment(EmptyMatchTreatment treatment) { - return withMatchConfiguration(getMatchConfiguration().withEmptyMatchTreatment(treatment)); - } - - /** Like {@link Read#watchForNewFiles}. */ - public ReadAll<T> watchForNewFiles( - Duration pollInterval, TerminationCondition<String, ?> terminationCondition) { - return withMatchConfiguration( - getMatchConfiguration().continuously(pollInterval, terminationCondition)); - } - - /** - * Set a value for the bundle size for parallel reads. Default is 64 MB. You may want to use a - * lower value (e.g. 1 MB) for streaming applications. - */ - public ReadAll<T> withDesiredBundleSizeBytes(long desiredBundleSizeBytes) { - return toBuilder().setDesiredBundleSizeBytes(desiredBundleSizeBytes).build(); - } - - /** - * If set to true, a Beam schema will be inferred from the AVRO schema. This allows the output - * to be used by SQL and by the schema-transform library. - */ - public ReadAll<T> withBeamSchemas(boolean withBeamSchemas) { - return toBuilder().setInferBeamSchema(withBeamSchemas).build(); - } - - @Override - public PCollection<T> expand(PCollection<String> input) { - checkNotNull(getSchema(), "schema"); - PCollection<T> read = - input - .apply(FileIO.matchAll().withConfiguration(getMatchConfiguration())) - .apply(FileIO.readMatches().withDirectoryTreatment(DirectoryTreatment.PROHIBIT)) - .apply(readFiles(getRecordClass())); - return getInferBeamSchema() ? setBeamSchema(read, getRecordClass(), getSchema()) : read; - } - - @Override - public void populateDisplayData(DisplayData.Builder builder) { - super.populateDisplayData(builder); - builder - .add( - DisplayData.item("inferBeamSchema", getInferBeamSchema()) - .withLabel("Infer Beam Schema")) - .addIfNotNull(DisplayData.item("schema", String.valueOf(getSchema()))) - .addIfNotNull(DisplayData.item("recordClass", getRecordClass()).withLabel("Record Class")) - .include("matchConfiguration", getMatchConfiguration()); - } - } - - private static class CreateSourceFn<T> - implements SerializableFunction<String, FileBasedSource<T>> { - private final Class<T> recordClass; - private final Supplier<Schema> schemaSupplier; - private final AvroSource.DatumReaderFactory<T> readerFactory; - - CreateSourceFn( - Class<T> recordClass, String jsonSchema, AvroSource.DatumReaderFactory<T> readerFactory) { - this.recordClass = recordClass; - this.schemaSupplier = - Suppliers.memoize( - Suppliers.compose(new JsonToSchema(), Suppliers.ofInstance(jsonSchema))); - this.readerFactory = readerFactory; - } - - @Override - public FileBasedSource<T> apply(String input) { - return Read.createSource( - StaticValueProvider.of(input), - EmptyMatchTreatment.DISALLOW, - recordClass, - schemaSupplier.get(), - readerFactory); - } - - private static class JsonToSchema implements Function<String, Schema>, Serializable { - @Override - public Schema apply(String input) { - return new Schema.Parser().parse(input); - } - } - } - - ///////////////////////////////////////////////////////////////////////////// - - /** Implementation of {@link #parseGenericRecords}. */ - @AutoValue - public abstract static class Parse<T> extends PTransform<PBegin, PCollection<T>> { - - abstract @Nullable ValueProvider<String> getFilepattern(); - - abstract MatchConfiguration getMatchConfiguration(); - - abstract SerializableFunction<GenericRecord, T> getParseFn(); - - abstract @Nullable Coder<T> getCoder(); - - abstract boolean getHintMatchesManyFiles(); - - abstract Builder<T> toBuilder(); - - @AutoValue.Builder - abstract static class Builder<T> { - abstract Builder<T> setFilepattern(ValueProvider<String> filepattern); - - abstract Builder<T> setMatchConfiguration(MatchConfiguration matchConfiguration); - - abstract Builder<T> setParseFn(SerializableFunction<GenericRecord, T> parseFn); - - abstract Builder<T> setCoder(Coder<T> coder); - - abstract Builder<T> setHintMatchesManyFiles(boolean hintMatchesManyFiles); - - abstract Parse<T> build(); - } - - /** Reads from the given filename or filepattern. */ - public Parse<T> from(String filepattern) { - return from(StaticValueProvider.of(filepattern)); - } - - /** Like {@link #from(String)}. */ - public Parse<T> from(ValueProvider<String> filepattern) { - return toBuilder().setFilepattern(filepattern).build(); - } - - /** Sets the {@link MatchConfiguration}. */ - public Parse<T> withMatchConfiguration(MatchConfiguration configuration) { - return toBuilder().setMatchConfiguration(configuration).build(); - } - - /** Like {@link Read#withEmptyMatchTreatment}. */ - public Parse<T> withEmptyMatchTreatment(EmptyMatchTreatment treatment) { - return withMatchConfiguration(getMatchConfiguration().withEmptyMatchTreatment(treatment)); - } - - /** Like {@link Read#watchForNewFiles}. */ - public Parse<T> watchForNewFiles( - Duration pollInterval, TerminationCondition<String, ?> terminationCondition) { - return withMatchConfiguration( - getMatchConfiguration().continuously(pollInterval, terminationCondition)); - } - - /** Sets a coder for the result of the parse function. */ - public Parse<T> withCoder(Coder<T> coder) { - return toBuilder().setCoder(coder).build(); - } - - /** Like {@link Read#withHintMatchesManyFiles()}. */ - public Parse<T> withHintMatchesManyFiles() { - return toBuilder().setHintMatchesManyFiles(true).build(); - } - - @Override - public PCollection<T> expand(PBegin input) { - checkNotNull(getFilepattern(), "filepattern"); - Coder<T> coder = inferCoder(getCoder(), getParseFn(), input.getPipeline().getCoderRegistry()); - - if (getMatchConfiguration().getWatchInterval() == null && !getHintMatchesManyFiles()) { - return input.apply( - org.apache.beam.sdk.io.Read.from( - AvroSource.from(getFilepattern()).withParseFn(getParseFn(), coder))); - } - - // All other cases go through FileIO + ParseFilesGenericRecords. - return input - .apply("Create filepattern", Create.ofProvider(getFilepattern(), StringUtf8Coder.of())) - .apply("Match All", FileIO.matchAll().withConfiguration(getMatchConfiguration())) - .apply( - "Read Matches", - FileIO.readMatches().withDirectoryTreatment(DirectoryTreatment.PROHIBIT)) - .apply("Via ParseFiles", parseFilesGenericRecords(getParseFn()).withCoder(coder)); - } - - private static <T> Coder<T> inferCoder( - @Nullable Coder<T> explicitCoder, - SerializableFunction<GenericRecord, T> parseFn, - CoderRegistry coderRegistry) { - if (explicitCoder != null) { - return explicitCoder; - } - // If a coder was not specified explicitly, infer it from parse fn. - try { - return coderRegistry.getCoder(TypeDescriptors.outputOf(parseFn)); - } catch (CannotProvideCoderException e) { - throw new IllegalArgumentException( - "Unable to infer coder for output of parseFn. Specify it explicitly using withCoder().", - e); - } - } - - @Override - public void populateDisplayData(DisplayData.Builder builder) { - super.populateDisplayData(builder); - builder - .addIfNotNull( - DisplayData.item("filePattern", getFilepattern()).withLabel("Input File Pattern")) - .add(DisplayData.item("parseFn", getParseFn().getClass()).withLabel("Parse function")) - .include("matchConfiguration", getMatchConfiguration()); - } - } - - ///////////////////////////////////////////////////////////////////////////// - - /** Implementation of {@link #parseFilesGenericRecords}. */ - @AutoValue - public abstract static class ParseFiles<T> - extends PTransform<PCollection<ReadableFile>, PCollection<T>> { - abstract SerializableFunction<GenericRecord, T> getParseFn(); - - abstract @Nullable Coder<T> getCoder(); - - abstract boolean getUsesReshuffle(); - - abstract ReadFileRangesFnExceptionHandler getFileExceptionHandler(); - - abstract long getDesiredBundleSizeBytes(); - - abstract Builder<T> toBuilder(); - - @AutoValue.Builder - abstract static class Builder<T> { - abstract Builder<T> setParseFn(SerializableFunction<GenericRecord, T> parseFn); - - abstract Builder<T> setCoder(Coder<T> coder); - - abstract Builder<T> setUsesReshuffle(boolean usesReshuffle); - - abstract Builder<T> setFileExceptionHandler( - ReadFileRangesFnExceptionHandler exceptionHandler); - - abstract Builder<T> setDesiredBundleSizeBytes(long desiredBundleSizeBytes); - - abstract ParseFiles<T> build(); - } - - /** Specifies the coder for the result of the {@code parseFn}. */ - public ParseFiles<T> withCoder(Coder<T> coder) { - return toBuilder().setCoder(coder).build(); - } - - /** Specifies if a Reshuffle should run before file reads occur. */ - public ParseFiles<T> withUsesReshuffle(boolean usesReshuffle) { - return toBuilder().setUsesReshuffle(usesReshuffle).build(); - } - - /** Specifies if exceptions should be logged only for streaming pipelines. */ - public ParseFiles<T> withFileExceptionHandler( - ReadFileRangesFnExceptionHandler exceptionHandler) { - return toBuilder().setFileExceptionHandler(exceptionHandler).build(); - } - - /** - * Set a value for the bundle size for parallel reads. Default is 64 MB. You may want to use a - * lower value (e.g. 1 MB) for streaming applications. - */ - public ParseFiles<T> withDesiredBundleSizeBytes(long desiredBundleSizeBytes) { - return toBuilder().setDesiredBundleSizeBytes(desiredBundleSizeBytes).build(); - } - - @Override - public PCollection<T> expand(PCollection<ReadableFile> input) { - final Coder<T> coder = - Parse.inferCoder(getCoder(), getParseFn(), input.getPipeline().getCoderRegistry()); - final SerializableFunction<GenericRecord, T> parseFn = getParseFn(); - final SerializableFunction<String, FileBasedSource<T>> createSource = - new CreateParseSourceFn<>(parseFn, coder); - return input.apply( - "Parse Files via FileBasedSource", - new ReadAllViaFileBasedSource<>( - getDesiredBundleSizeBytes(), - createSource, - coder, - getUsesReshuffle(), - getFileExceptionHandler())); - } - - @Override - public void populateDisplayData(DisplayData.Builder builder) { - super.populateDisplayData(builder); - builder.add(DisplayData.item("parseFn", getParseFn().getClass()).withLabel("Parse function")); - } - - private static class CreateParseSourceFn<T> - implements SerializableFunction<String, FileBasedSource<T>> { - private final SerializableFunction<GenericRecord, T> parseFn; - private final Coder<T> coder; - - CreateParseSourceFn(SerializableFunction<GenericRecord, T> parseFn, Coder<T> coder) { - this.parseFn = parseFn; - this.coder = coder; - } - - @Override - public FileBasedSource<T> apply(String input) { - return AvroSource.from(input).withParseFn(parseFn, coder); - } - } - } - - ///////////////////////////////////////////////////////////////////////////// - - /** - * Implementation of {@link #parseAllGenericRecords}. - * - * @deprecated See {@link #parseAllGenericRecords(SerializableFunction)} for details. - */ - @Deprecated - @AutoValue - public abstract static class ParseAll<T> extends PTransform<PCollection<String>, PCollection<T>> { - abstract MatchConfiguration getMatchConfiguration(); - - abstract SerializableFunction<GenericRecord, T> getParseFn(); - - abstract @Nullable Coder<T> getCoder(); - - abstract long getDesiredBundleSizeBytes(); - - abstract Builder<T> toBuilder(); - - @AutoValue.Builder - abstract static class Builder<T> { - abstract Builder<T> setMatchConfiguration(MatchConfiguration matchConfiguration); - - abstract Builder<T> setParseFn(SerializableFunction<GenericRecord, T> parseFn); - - abstract Builder<T> setCoder(Coder<T> coder); - - abstract Builder<T> setDesiredBundleSizeBytes(long desiredBundleSizeBytes); - - abstract ParseAll<T> build(); - } - - /** Sets the {@link MatchConfiguration}. */ - public ParseAll<T> withMatchConfiguration(MatchConfiguration configuration) { - return toBuilder().setMatchConfiguration(configuration).build(); - } - - /** Like {@link Read#withEmptyMatchTreatment}. */ - public ParseAll<T> withEmptyMatchTreatment(EmptyMatchTreatment treatment) { - return withMatchConfiguration(getMatchConfiguration().withEmptyMatchTreatment(treatment)); - } - - /** Like {@link Read#watchForNewFiles(Duration, TerminationCondition, boolean)}. */ - public ParseAll<T> watchForNewFiles( - Duration pollInterval, - TerminationCondition<String, ?> terminationCondition, - boolean matchUpdatedFiles) { - return withMatchConfiguration( - getMatchConfiguration() - .continuously(pollInterval, terminationCondition, matchUpdatedFiles)); - } - - /** Like {@link Read#watchForNewFiles(Duration, TerminationCondition)}. */ - public ParseAll<T> watchForNewFiles( - Duration pollInterval, TerminationCondition<String, ?> terminationCondition) { - return watchForNewFiles(pollInterval, terminationCondition, false); - } - - /** Specifies the coder for the result of the {@code parseFn}. */ - public ParseAll<T> withCoder(Coder<T> coder) { - return toBuilder().setCoder(coder).build(); - } - - /** - * Set a value for the bundle size for parallel reads. Default is 64 MB. You may want to use a - * lower value (e.g. 1 MB) for streaming applications. - */ - public ParseAll<T> withDesiredBundleSizeBytes(long desiredBundleSizeBytes) { - return toBuilder().setDesiredBundleSizeBytes(desiredBundleSizeBytes).build(); - } - - @Override - public PCollection<T> expand(PCollection<String> input) { - return input - .apply(FileIO.matchAll().withConfiguration(getMatchConfiguration())) - .apply(FileIO.readMatches().withDirectoryTreatment(DirectoryTreatment.PROHIBIT)) - .apply( - "Parse all via FileBasedSource", - parseFilesGenericRecords(getParseFn()).withCoder(getCoder())); - } - - @Override - public void populateDisplayData(DisplayData.Builder builder) { - super.populateDisplayData(builder); - builder - .add(DisplayData.item("parseFn", getParseFn().getClass()).withLabel("Parse function")) - .include("matchConfiguration", getMatchConfiguration()); - } - } - - ///////////////////////////////////////////////////////////////////////////// - - /** Implementation of {@link #write}. */ - @AutoValue - public abstract static class TypedWrite<UserT, DestinationT, OutputT> - extends PTransform<PCollection<UserT>, WriteFilesResult<DestinationT>> { - static final CodecFactory DEFAULT_CODEC = CodecFactory.snappyCodec(); - static final SerializableAvroCodecFactory DEFAULT_SERIALIZABLE_CODEC = - new SerializableAvroCodecFactory(DEFAULT_CODEC); - - abstract @Nullable SerializableFunction<UserT, OutputT> getFormatFunction(); - - abstract @Nullable ValueProvider<ResourceId> getFilenamePrefix(); - - abstract @Nullable String getShardTemplate(); - - abstract @Nullable String getFilenameSuffix(); - - abstract @Nullable ValueProvider<ResourceId> getTempDirectory(); - - abstract int getNumShards(); - - abstract boolean getGenericRecords(); - - abstract int getSyncInterval(); - - abstract @Nullable Schema getSchema(); - - abstract boolean getWindowedWrites(); - - abstract boolean getNoSpilling(); - - abstract @Nullable FilenamePolicy getFilenamePolicy(); - - abstract @Nullable DynamicAvroDestinations<UserT, DestinationT, OutputT> - getDynamicDestinations(); - - abstract AvroSink.@Nullable DatumWriterFactory<OutputT> getDatumWriterFactory(); - - /** - * The codec used to encode the blocks in the Avro file. String value drawn from those in - * https://avro.apache.org/docs/1.7.7/api/java/org/apache/avro/file/CodecFactory.html - */ - abstract SerializableAvroCodecFactory getCodec(); - /** Avro file metadata. */ - abstract ImmutableMap<String, Object> getMetadata(); - - abstract Builder<UserT, DestinationT, OutputT> toBuilder(); - - @AutoValue.Builder - abstract static class Builder<UserT, DestinationT, OutputT> { - abstract Builder<UserT, DestinationT, OutputT> setFormatFunction( - @Nullable SerializableFunction<UserT, OutputT> formatFunction); - - abstract Builder<UserT, DestinationT, OutputT> setFilenamePrefix( - ValueProvider<ResourceId> filenamePrefix); - - abstract Builder<UserT, DestinationT, OutputT> setFilenameSuffix( - @Nullable String filenameSuffix); - - abstract Builder<UserT, DestinationT, OutputT> setTempDirectory( - ValueProvider<ResourceId> tempDirectory); - - abstract Builder<UserT, DestinationT, OutputT> setNumShards(int numShards); - - abstract Builder<UserT, DestinationT, OutputT> setShardTemplate( - @Nullable String shardTemplate); - - abstract Builder<UserT, DestinationT, OutputT> setGenericRecords(boolean genericRecords); - - abstract Builder<UserT, DestinationT, OutputT> setSyncInterval(int syncInterval); - - abstract Builder<UserT, DestinationT, OutputT> setSchema(Schema schema); - - abstract Builder<UserT, DestinationT, OutputT> setWindowedWrites(boolean windowedWrites); - - abstract Builder<UserT, DestinationT, OutputT> setNoSpilling(boolean noSpilling); - - abstract Builder<UserT, DestinationT, OutputT> setFilenamePolicy( - FilenamePolicy filenamePolicy); - - abstract Builder<UserT, DestinationT, OutputT> setCodec(SerializableAvroCodecFactory codec); - - abstract Builder<UserT, DestinationT, OutputT> setMetadata( - ImmutableMap<String, Object> metadata); - - abstract Builder<UserT, DestinationT, OutputT> setDynamicDestinations( - DynamicAvroDestinations<UserT, DestinationT, OutputT> dynamicDestinations); - - abstract Builder<UserT, DestinationT, OutputT> setDatumWriterFactory( - AvroSink.DatumWriterFactory<OutputT> datumWriterFactory); - - abstract TypedWrite<UserT, DestinationT, OutputT> build(); - } - - /** - * Writes to file(s) with the given output prefix. See {@link FileSystems} for information on - * supported file systems. - * - * <p>The name of the output files will be determined by the {@link FilenamePolicy} used. - * - * <p>By default, a {@link DefaultFilenamePolicy} will build output filenames using the - * specified prefix, a shard name template (see {@link #withShardNameTemplate(String)}, and a - * common suffix (if supplied using {@link #withSuffix(String)}). This default can be overridden - * using {@link #to(FilenamePolicy)}. - */ - public TypedWrite<UserT, DestinationT, OutputT> to(String outputPrefix) { - return to(FileBasedSink.convertToFileResourceIfPossible(outputPrefix)); - } - - /** - * Writes to file(s) with the given output prefix. See {@link FileSystems} for information on - * supported file systems. This prefix is used by the {@link DefaultFilenamePolicy} to generate - * filenames. - * - * <p>By default, a {@link DefaultFilenamePolicy} will build output filenames using the - * specified prefix, a shard name template (see {@link #withShardNameTemplate(String)}, and a - * common suffix (if supplied using {@link #withSuffix(String)}). This default can be overridden - * using {@link #to(FilenamePolicy)}. - * - * <p>This default policy can be overridden using {@link #to(FilenamePolicy)}, in which case - * {@link #withShardNameTemplate(String)} and {@link #withSuffix(String)} should not be set. - * Custom filename policies do not automatically see this prefix - you should explicitly pass - * the prefix into your {@link FilenamePolicy} object if you need this. - * - * <p>If {@link #withTempDirectory} has not been called, this filename prefix will be used to - * infer a directory for temporary files. - */ - public TypedWrite<UserT, DestinationT, OutputT> to(ResourceId outputPrefix) { - return toResource(StaticValueProvider.of(outputPrefix)); - } - - private static class OutputPrefixToResourceId - implements SerializableFunction<String, ResourceId> { - @Override - public ResourceId apply(String input) { - return FileBasedSink.convertToFileResourceIfPossible(input); - } - } - - /** Like {@link #to(String)}. */ - public TypedWrite<UserT, DestinationT, OutputT> to(ValueProvider<String> outputPrefix) { - return toResource( - NestedValueProvider.of( - outputPrefix, - // The function cannot be created as an anonymous class here since the enclosed class - // may contain unserializable members. - new OutputPrefixToResourceId())); - } - - /** Like {@link #to(ResourceId)}. */ - public TypedWrite<UserT, DestinationT, OutputT> toResource( - ValueProvider<ResourceId> outputPrefix) { - return toBuilder().setFilenamePrefix(outputPrefix).build(); - } - - /** - * Writes to files named according to the given {@link FileBasedSink.FilenamePolicy}. A - * directory for temporary files must be specified using {@link #withTempDirectory}. - */ - public TypedWrite<UserT, DestinationT, OutputT> to(FilenamePolicy filenamePolicy) { - return toBuilder().setFilenamePolicy(filenamePolicy).build(); - } - - /** - * Use a {@link DynamicAvroDestinations} object to vend {@link FilenamePolicy} objects. These - * objects can examine the input record when creating a {@link FilenamePolicy}. A directory for - * temporary files must be specified using {@link #withTempDirectory}. - * - * @deprecated Use {@link FileIO#write()} or {@link FileIO#writeDynamic()} instead. - */ - @Deprecated - public <NewDestinationT> TypedWrite<UserT, NewDestinationT, OutputT> to( - DynamicAvroDestinations<UserT, NewDestinationT, OutputT> dynamicDestinations) { - return toBuilder() - .setDynamicDestinations((DynamicAvroDestinations) dynamicDestinations) - .build(); - } - - /** - * Sets the approximate number of uncompressed bytes to write in each block for the AVRO - * container format. - */ - public TypedWrite<UserT, DestinationT, OutputT> withSyncInterval(int syncInterval) { - return toBuilder().setSyncInterval(syncInterval).build(); - } - - /** - * Sets the output schema. Can only be used when the output type is {@link GenericRecord} and - * when not using {@link #to(DynamicAvroDestinations)}. - */ - public TypedWrite<UserT, DestinationT, OutputT> withSchema(Schema schema) { - return toBuilder().setSchema(schema).build(); - } - - /** - * Specifies a format function to convert {@link UserT} to the output type. If {@link - * #to(DynamicAvroDestinations)} is used, {@link DynamicAvroDestinations#formatRecord} must be - * used instead. - */ - public TypedWrite<UserT, DestinationT, OutputT> withFormatFunction( - @Nullable SerializableFunction<UserT, OutputT> formatFunction) { - return toBuilder().setFormatFunction(formatFunction).build(); - } - - /** Set the base directory used to generate temporary files. */ - public TypedWrite<UserT, DestinationT, OutputT> withTempDirectory( - ValueProvider<ResourceId> tempDirectory) { - return toBuilder().setTempDirectory(tempDirectory).build(); - } - - /** Set the base directory used to generate temporary files. */ - public TypedWrite<UserT, DestinationT, OutputT> withTempDirectory(ResourceId tempDirectory) { - return withTempDirectory(StaticValueProvider.of(tempDirectory)); - } - - /** - * Uses the given {@link ShardNameTemplate} for naming output files. This option may only be - * used when using one of the default filename-prefix to() overrides. - * - * <p>See {@link DefaultFilenamePolicy} for how the prefix, shard name template, and suffix are - * used. - */ - public TypedWrite<UserT, DestinationT, OutputT> withShardNameTemplate(String shardTemplate) { - return toBuilder().setShardTemplate(shardTemplate).build(); - } - - /** - * Configures the filename suffix for written files. This option may only be used when using one - * of the default filename-prefix to() overrides. - * - * <p>See {@link DefaultFilenamePolicy} for how the prefix, shard name template, and suffix are - * used. - */ - public TypedWrite<UserT, DestinationT, OutputT> withSuffix(String filenameSuffix) { - return toBuilder().setFilenameSuffix(filenameSuffix).build(); - } - - /** - * Configures the number of output shards produced overall (when using unwindowed writes) or - * per-window (when using windowed writes). - * - * <p>For unwindowed writes, constraining the number of shards is likely to reduce the - * performance of a pipeline. Setting this value is not recommended unless you require a - * specific number of output files. - * - * @param numShards the number of shards to use, or 0 to let the system decide. - */ - public TypedWrite<UserT, DestinationT, OutputT> withNumShards(int numShards) { - checkArgument(numShards >= 0); - return toBuilder().setNumShards(numShards).build(); - } - - /** - * Forces a single file as output and empty shard name template. This option is only compatible - * with unwindowed writes. - * - * <p>For unwindowed writes, constraining the number of shards is likely to reduce the - * performance of a pipeline. Setting this value is not recommended unless you require a - * specific number of output files. - * - * <p>This is equivalent to {@code .withNumShards(1).withShardNameTemplate("")} - */ - public TypedWrite<UserT, DestinationT, OutputT> withoutSharding() { - return withNumShards(1).withShardNameTemplate(""); - } - - /** - * Preserves windowing of input elements and writes them to files based on the element's window. - * - * <p>If using {@link #to(FileBasedSink.FilenamePolicy)}. Filenames will be generated using - * {@link FilenamePolicy#windowedFilename}. See also {@link WriteFiles#withWindowedWrites()}. - */ - public TypedWrite<UserT, DestinationT, OutputT> withWindowedWrites() { - return toBuilder().setWindowedWrites(true).build(); - } - - /** See {@link WriteFiles#withNoSpilling()}. */ - public TypedWrite<UserT, DestinationT, OutputT> withNoSpilling() { - return toBuilder().setNoSpilling(true).build(); - } - - /** Writes to Avro file(s) compressed using specified codec. */ - public TypedWrite<UserT, DestinationT, OutputT> withCodec(CodecFactory codec) { - return toBuilder().setCodec(new SerializableAvroCodecFactory(codec)).build(); - } - - /** - * Specifies a {@link AvroSink.DatumWriterFactory} to use for creating {@link - * org.apache.avro.io.DatumWriter} instances. - */ - public TypedWrite<UserT, DestinationT, OutputT> withDatumWriterFactory( - AvroSink.DatumWriterFactory<OutputT> datumWriterFactory) { - return toBuilder().setDatumWriterFactory(datumWriterFactory).build(); - } - - /** - * Writes to Avro file(s) with the specified metadata. - * - * <p>Supported value types are String, Long, and byte[]. - */ - public TypedWrite<UserT, DestinationT, OutputT> withMetadata(Map<String, Object> metadata) { - Map<String, String> badKeys = Maps.newLinkedHashMap(); - for (Map.Entry<String, Object> entry : metadata.entrySet()) { - Object v = entry.getValue(); - if (!(v instanceof String || v instanceof Long || v instanceof byte[])) { - badKeys.put(entry.getKey(), v.getClass().getSimpleName()); - } - } - checkArgument( - badKeys.isEmpty(), - "Metadata value type must be one of String, Long, or byte[]. Found %s", - badKeys); - return toBuilder().setMetadata(ImmutableMap.copyOf(metadata)).build(); - } - - DynamicAvroDestinations<UserT, DestinationT, OutputT> resolveDynamicDestinations() { - DynamicAvroDestinations<UserT, DestinationT, OutputT> dynamicDestinations = - getDynamicDestinations(); - if (dynamicDestinations == null) { - // In this case DestinationT is Void. - FilenamePolicy usedFilenamePolicy = getFilenamePolicy(); - if (usedFilenamePolicy == null) { - usedFilenamePolicy = - DefaultFilenamePolicy.fromStandardParameters( - getFilenamePrefix(), - getShardTemplate(), - getFilenameSuffix(), - getWindowedWrites()); - } - dynamicDestinations = - (DynamicAvroDestinations<UserT, DestinationT, OutputT>) - constantDestinations( - usedFilenamePolicy, - getSchema(), - getMetadata(), - getCodec().getCodec(), - getFormatFunction(), - getDatumWriterFactory()); - } - return dynamicDestinations; - } - - @Override - public WriteFilesResult<DestinationT> expand(PCollection<UserT> input) { - checkArgument( - getFilenamePrefix() != null || getTempDirectory() != null, - "Need to set either the filename prefix or the tempDirectory of a AvroIO.Write " - + "transform."); - if (getFilenamePolicy() != null) { - checkArgument( - getShardTemplate() == null && getFilenameSuffix() == null, - "shardTemplate and filenameSuffix should only be used with the default " - + "filename policy"); - } - if (getDynamicDestinations() != null) { - checkArgument( - getFormatFunction() == null, - "A format function should not be specified " - + "with DynamicDestinations. Use DynamicDestinations.formatRecord instead"); - } else { - checkArgument( - getSchema() != null, "Unless using DynamicDestinations, .withSchema() is required."); - } - - ValueProvider<ResourceId> tempDirectory = getTempDirectory(); - if (tempDirectory == null) { - tempDirectory = getFilenamePrefix(); - } - WriteFiles<UserT, DestinationT, OutputT> write = - WriteFiles.to( - new AvroSink<>( - tempDirectory, - resolveDynamicDestinations(), - getGenericRecords(), - getSyncInterval())); - if (getNumShards() > 0) { - write = write.withNumShards(getNumShards()); - } - if (getWindowedWrites()) { - write = write.withWindowedWrites(); - } - if (getNoSpilling()) { - write = write.withNoSpilling(); - } - return input.apply("Write", write); - } - - @Override - public void populateDisplayData(DisplayData.Builder builder) { - super.populateDisplayData(builder); - resolveDynamicDestinations().populateDisplayData(builder); - builder - .addIfNotDefault( - DisplayData.item("numShards", getNumShards()).withLabel("Maximum Output Shards"), 0) - .addIfNotNull( - DisplayData.item("tempDirectory", getTempDirectory()) - .withLabel("Directory for temporary files")); - } - } - - /** - * This class is used as the default return value of {@link AvroIO#write} - * - * <p>All methods in this class delegate to the appropriate method of {@link AvroIO.TypedWrite}. - * This class exists for backwards compatibility, and will be removed in Beam 3.0. - */ - public static class Write<T> extends PTransform<PCollection<T>, PDone> { - @VisibleForTesting final TypedWrite<T, ?, T> inner; - - Write(TypedWrite<T, ?, T> inner) { - this.inner = inner; - } - - /** See {@link TypedWrite#to(String)}. */ - public Write<T> to(String outputPrefix) { - return new Write<>( - inner - .to(FileBasedSink.convertToFileResourceIfPossible(outputPrefix)) - .withFormatFunction(SerializableFunctions.identity())); - } - - /** See {@link TypedWrite#to(ResourceId)} . */ - public Write<T> to(ResourceId outputPrefix) { - return new Write<>( - inner.to(outputPrefix).withFormatFunction(SerializableFunctions.identity())); - } - - /** See {@link TypedWrite#to(ValueProvider)}. */ - public Write<T> to(ValueProvider<String> outputPrefix) { - return new Write<>( - inner.to(outputPrefix).withFormatFunction(SerializableFunctions.identity())); - } - - /** See {@link TypedWrite#to(ResourceId)}. */ - public Write<T> toResource(ValueProvider<ResourceId> outputPrefix) { - return new Write<>( - inner.toResource(outputPrefix).withFormatFunction(SerializableFunctions.identity())); - } - - /** See {@link TypedWrite#to(FilenamePolicy)}. */ - public Write<T> to(FilenamePolicy filenamePolicy) { - return new Write<>( - inner.to(filenamePolicy).withFormatFunction(SerializableFunctions.identity())); - } - - /** - * See {@link TypedWrite#to(DynamicAvroDestinations)}. - * - * @deprecated Use {@link FileIO#write()} or {@link FileIO#writeDynamic()} instead. - */ - @Deprecated - public Write<T> to(DynamicAvroDestinations<T, ?, T> dynamicDestinations) { - return new Write<>(inner.to(dynamicDestinations).withFormatFunction(null)); - } - - /** See {@link TypedWrite#withSyncInterval}. */ - public Write<T> withSyncInterval(int syncInterval) { - return new Write<>(inner.withSyncInterval(syncInterval)); - } - - /** See {@link TypedWrite#withSchema}. */ - public Write<T> withSchema(Schema schema) { - return new Write<>(inner.withSchema(schema)); - } - - /** See {@link TypedWrite#withTempDirectory(ValueProvider)}. */ - public Write<T> withTempDirectory(ValueProvider<ResourceId> tempDirectory) { - return new Write<>(inner.withTempDirectory(tempDirectory)); - } - - /** See {@link TypedWrite#withTempDirectory(ResourceId)}. */ - public Write<T> withTempDirectory(ResourceId tempDirectory) { - return new Write<>(inner.withTempDirectory(tempDirectory)); - } - - /** See {@link TypedWrite#withShardNameTemplate}. */ - public Write<T> withShardNameTemplate(String shardTemplate) { - return new Write<>(inner.withShardNameTemplate(shardTemplate)); - } - - /** See {@link TypedWrite#withSuffix}. */ - public Write<T> withSuffix(String filenameSuffix) { - return new Write<>(inner.withSuffix(filenameSuffix)); - } - - /** See {@link TypedWrite#withNumShards}. */ - public Write<T> withNumShards(int numShards) { - return new Write<>(inner.withNumShards(numShards)); - } - - /** See {@link TypedWrite#withoutSharding}. */ - public Write<T> withoutSharding() { - return new Write<>(inner.withoutSharding()); - } - - /** See {@link TypedWrite#withWindowedWrites}. */ - public Write<T> withWindowedWrites() { - return new Write<>(inner.withWindowedWrites()); - } - - /** See {@link TypedWrite#withCodec}. */ - public Write<T> withCodec(CodecFactory codec) { - return new Write<>(inner.withCodec(codec)); - } - - /** See {@link TypedWrite#withDatumWriterFactory}. */ - public Write<T> withDatumWriterFactory(AvroSink.DatumWriterFactory<T> datumWriterFactory) { - return new Write<>(inner.withDatumWriterFactory(datumWriterFactory)); - } - - /** - * Specify that output filenames are wanted. - * - * <p>The nested {@link TypedWrite}transform always has access to output filenames, however due - * to backwards-compatibility concerns, {@link Write} cannot return them. This method simply - * returns the inner {@link TypedWrite} transform which has {@link WriteFilesResult} as its - * output type, allowing access to output files. - * - * <p>The supplied {@code DestinationT} type must be: the same as that supplied in {@link - * #to(DynamicAvroDestinations)} if that method was used, or {@code Void} otherwise. - */ - public <DestinationT> TypedWrite<T, DestinationT, T> withOutputFilenames() { - return (TypedWrite) inner; - } - - /** See {@link TypedWrite#withMetadata} . */ - public Write<T> withMetadata(Map<String, Object> metadata) { - return new Write<>(inner.withMetadata(metadata)); - } - - @Override - public PDone expand(PCollection<T> input) { - input.apply(inner); - return PDone.in(input.getPipeline()); - } - - @Override - public void populateDisplayData(DisplayData.Builder builder) { - inner.populateDisplayData(builder); - } - } - - /** - * Returns a {@link DynamicAvroDestinations} that always returns the same {@link FilenamePolicy}, - * schema, metadata, and codec. - */ - public static <UserT, OutputT> DynamicAvroDestinations<UserT, Void, OutputT> constantDestinations( - FilenamePolicy filenamePolicy, - Schema schema, - Map<String, Object> metadata, - CodecFactory codec, - SerializableFunction<UserT, OutputT> formatFunction) { - return constantDestinations(filenamePolicy, schema, metadata, codec, formatFunction, null); - } - - /** - * Returns a {@link DynamicAvroDestinations} that always returns the same {@link FilenamePolicy}, - * schema, metadata, and codec. - */ - public static <UserT, OutputT> DynamicAvroDestinations<UserT, Void, OutputT> constantDestinations( - FilenamePolicy filenamePolicy, - Schema schema, - Map<String, Object> metadata, - CodecFactory codec, - SerializableFunction<UserT, OutputT> formatFunction, - AvroSink.@Nullable DatumWriterFactory<OutputT> datumWriterFactory) { - return new ConstantAvroDestination<>( - filenamePolicy, schema, metadata, codec, formatFunction, datumWriterFactory); - } - ///////////////////////////////////////////////////////////////////////////// - - /** - * Formats an element of a user type into a record with the given schema. - * - * @deprecated Users can achieve the same by providing this transform in a {@link - * org.apache.beam.sdk.transforms.ParDo} before using write in AvroIO {@link #write(Class)}. - */ - @Deprecated - public interface RecordFormatter<ElementT> extends Serializable { - GenericRecord formatRecord(ElementT element, Schema schema); - } - - /** - * A {@link Sink} for use with {@link FileIO#write} and {@link FileIO#writeDynamic}, writing - * elements of the given generated class, like {@link #write(Class)}. - */ - public static <ElementT> Sink<ElementT> sink(final Class<ElementT> clazz) { - return new AutoValue_AvroIO_Sink.Builder<ElementT>() - .setJsonSchema(ReflectData.get().getSchema(clazz).toString()) - .setMetadata(ImmutableMap.of()) - .setCodec(TypedWrite.DEFAULT_SERIALIZABLE_CODEC) - .build(); - } - - /** - * A {@link Sink} for use with {@link FileIO#write} and {@link FileIO#writeDynamic}, writing - * elements with a given (common) schema, like {@link #writeGenericRecords(Schema)}. - */ - public static <ElementT extends IndexedRecord> Sink<ElementT> sink(Schema schema) { - return sink(schema.toString()); - } - - /** - * A {@link Sink} for use with {@link FileIO#write} and {@link FileIO#writeDynamic}, writing - * elements with a given (common) schema, like {@link #writeGenericRecords(String)}. - */ - public static <ElementT extends IndexedRecord> Sink<ElementT> sink(String jsonSchema) { - return new AutoValue_AvroIO_Sink.Builder<ElementT>() - .setJsonSchema(jsonSchema) - .setMetadata(ImmutableMap.of()) - .setCodec(TypedWrite.DEFAULT_SERIALIZABLE_CODEC) - .build(); - } - - /** - * A {@link Sink} for use with {@link FileIO#write} and {@link FileIO#writeDynamic}, writing - * elements by converting each one to a {@link GenericRecord} with a given (common) schema, like - * {@link #writeCustomTypeToGenericRecords()}. - * - * @deprecated RecordFormatter will be removed in future versions. - */ - @Deprecated - public static <ElementT> Sink<ElementT> sinkViaGenericRecords( - Schema schema, RecordFormatter<ElementT> formatter) { - return new AutoValue_AvroIO_Sink.Builder<ElementT>() - .setRecordFormatter(formatter) - .setJsonSchema(schema.toString()) - .setMetadata(ImmutableMap.of()) - .setCodec(TypedWrite.DEFAULT_SERIALIZABLE_CODEC) - .build(); - } - - /** Implementation of {@link #sink} and {@link #sinkViaGenericRecords}. */ - @AutoValue - public abstract static class Sink<ElementT> implements FileIO.Sink<ElementT> { - /** @deprecated RecordFormatter will be removed in future versions. */ - @Deprecated - abstract @Nullable RecordFormatter<ElementT> getRecordFormatter(); - - abstract @Nullable String getJsonSchema(); - - abstract Map<String, Object> getMetadata(); - - abstract SerializableAvroCodecFactory getCodec(); - - abstract Builder<ElementT> toBuilder(); - - @AutoValue.Builder - abstract static class Builder<ElementT> { - /** @deprecated RecordFormatter will be removed in future versions. */ - @Deprecated - abstract Builder<ElementT> setRecordFormatter(RecordFormatter<ElementT> formatter); - - abstract Builder<ElementT> setJsonSchema(String jsonSchema); - - abstract Builder<ElementT> setMetadata(Map<String, Object> metadata); - - abstract Builder<ElementT> setCodec(SerializableAvroCodecFactory codec); - - abstract Sink<ElementT> build(); - } - - /** Specifies to put the given metadata into each generated file. By default, empty. */ - public Sink<ElementT> withMetadata(Map<String, Object> metadata) { - return toBuilder().setMetadata(metadata).build(); - } - - /** - * Specifies to use the given {@link CodecFactory} for each generated file. By default, {@code - * CodecFactory.snappyCodec()}. - */ - public Sink<ElementT> withCodec(CodecFactory codec) { - return toBuilder().setCodec(new SerializableAvroCodecFactory(codec)).build(); - } - - private transient @Nullable Schema schema; - private transient @Nullable DataFileWriter<ElementT> reflectWriter; - private transient @Nullable DataFileWriter<GenericRecord> genericWriter; - - @Override - public void open(WritableByteChannel channel) throws IOException { - this.schema = new Schema.Parser().parse(getJsonSchema()); - DataFileWriter<?> writer; - if (getRecordFormatter() == null) { - writer = reflectWriter = new DataFileWriter<>(new ReflectDatumWriter<>(schema)); - } else { - writer = genericWriter = new DataFileWriter<>(new GenericDatumWriter<>(schema)); - } - writer.setCodec(getCodec().getCodec()); - for (Map.Entry<String, Object> entry : getMetadata().entrySet()) { - Object v = entry.getValue(); - if (v instanceof String) { - writer.setMeta(entry.getKey(), (String) v); - } else if (v instanceof Long) { - writer.setMeta(entry.getKey(), (Long) v); - } else if (v instanceof byte[]) { - writer.setMeta(entry.getKey(), (byte[]) v); - } else { - throw new IllegalStateException( - "Metadata value type must be one of String, Long, or byte[]. Found " - + v.getClass().getSimpleName()); - } - } - writer.create(schema, Channels.newOutputStream(channel)); - } - - @Override - public void write(ElementT element) throws IOException { - if (getRecordFormatter() == null) { - reflectWriter.append(element); - } else { - genericWriter.append(getRecordFormatter().formatRecord(element, schema)); - } - } - - @Override - public void flush() throws IOException { - MoreObjects.firstNonNull(reflectWriter, genericWriter).flush(); - } - } - - /** Disallow construction of utility class. */ - private AvroIO() {} -} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSchemaIOProvider.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSchemaIOProvider.java deleted file mode 100644 index 43498235992a..000000000000 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSchemaIOProvider.java +++ /dev/null @@ -1,157 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.io; - -import com.google.auto.service.AutoService; -import java.io.Serializable; -import org.apache.avro.generic.GenericRecord; -import org.apache.beam.sdk.annotations.Internal; -import org.apache.beam.sdk.io.AvroIO.Write; -import org.apache.beam.sdk.schemas.Schema; -import org.apache.beam.sdk.schemas.Schema.FieldType; -import org.apache.beam.sdk.schemas.io.SchemaIO; -import org.apache.beam.sdk.schemas.io.SchemaIOProvider; -import org.apache.beam.sdk.schemas.transforms.Convert; -import org.apache.beam.sdk.schemas.utils.AvroUtils; -import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.transforms.windowing.FixedWindows; -import org.apache.beam.sdk.transforms.windowing.Window; -import org.apache.beam.sdk.values.PBegin; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.PCollection.IsBounded; -import org.apache.beam.sdk.values.PDone; -import org.apache.beam.sdk.values.POutput; -import org.apache.beam.sdk.values.Row; -import org.checkerframework.checker.nullness.qual.Nullable; -import org.joda.time.Duration; - -/** - * An implementation of {@link SchemaIOProvider} for reading and writing Avro files with {@link - * AvroIO}. - * - * @deprecated Avro related classes are deprecated in module <code>beam-sdks-java-core</code> and - * will be eventually removed. Please, migrate to a new module <code> - * beam-sdks-java-extensions-avro</code> by importing <code> - * org.apache.beam.sdk.extensions.avro.io.AvroSchemaIOProvider</code> instead of this one. - */ -@Internal -@AutoService(SchemaIOProvider.class) -@SuppressWarnings({ - "nullness" // TODO(https://github.com/apache/beam/issues/20497) -}) -@Deprecated -public class AvroSchemaIOProvider implements SchemaIOProvider { - /** Returns an id that uniquely represents this IO. */ - @Override - public String identifier() { - return "avro"; - } - - /** - * Returns the expected schema of the configuration object. Note this is distinct from the schema - * of the data source itself. No configuration expected for Avro. - */ - @Override - public Schema configurationSchema() { - return Schema.builder().addNullableField("writeWindowSizeSeconds", FieldType.INT64).build(); - } - - /** - * Produce a SchemaIO given a String representing the data's location, the schema of the data that - * resides there, and some IO-specific configuration object. - */ - @Override - public AvroSchemaIO from(String location, Row configuration, Schema dataSchema) { - return new AvroSchemaIO(location, dataSchema, configuration); - } - - @Override - public boolean requiresDataSchema() { - return true; - } - - @Override - public PCollection.IsBounded isBounded() { - // This supports streaming now as well but there's no option for this. The move to - // SchemaTransform will remove the need to provide this. - return PCollection.IsBounded.BOUNDED; - } - - /** An abstraction to create schema aware IOs. */ - private static class AvroSchemaIO implements SchemaIO, Serializable { - protected final Schema dataSchema; - protected final String location; - protected final @Nullable Duration windowSize; - - private AvroSchemaIO(String location, Schema dataSchema, Row configuration) { - this.dataSchema = dataSchema; - this.location = location; - if (configuration.getInt64("writeWindowSizeSeconds") != null) { - windowSize = Duration.standardSeconds(configuration.getInt64("writeWindowSizeSeconds")); - } else { - windowSize = null; - } - } - - @Override - public Schema schema() { - return dataSchema; - } - - @Override - public PTransform<PBegin, PCollection<Row>> buildReader() { - return new PTransform<PBegin, PCollection<Row>>() { - @Override - public PCollection<Row> expand(PBegin begin) { - return begin - .apply( - "AvroIORead", - AvroIO.readGenericRecords(AvroUtils.toAvroSchema(dataSchema, null, null)) - .withBeamSchemas(true) - .from(location)) - .apply("ToRows", Convert.toRows()); - } - }; - } - - @Override - public PTransform<PCollection<Row>, POutput> buildWriter() { - return new PTransform<PCollection<Row>, POutput>() { - @Override - public PDone expand(PCollection<Row> input) { - PCollection<GenericRecord> asRecords = - input.apply("ToGenericRecords", Convert.to(GenericRecord.class)); - Write<GenericRecord> avroWrite = - AvroIO.writeGenericRecords(AvroUtils.toAvroSchema(dataSchema, null, null)) - .to(location); - if (input.isBounded() == IsBounded.UNBOUNDED || windowSize != null) { - asRecords = - asRecords.apply( - Window.into( - FixedWindows.of( - windowSize == null ? Duration.standardMinutes(1) : windowSize))); - avroWrite = avroWrite.withWindowedWrites().withNumShards(1); - } else { - avroWrite = avroWrite.withoutSharding(); - } - return asRecords.apply("AvroIOWrite", avroWrite); - } - }; - } - } -} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSink.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSink.java deleted file mode 100644 index bc92113925cd..000000000000 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSink.java +++ /dev/null @@ -1,167 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.io; - -import java.io.Serializable; -import java.nio.channels.Channels; -import java.nio.channels.WritableByteChannel; -import java.util.Map; -import org.apache.avro.Schema; -import org.apache.avro.file.CodecFactory; -import org.apache.avro.file.DataFileWriter; -import org.apache.avro.generic.GenericDatumWriter; -import org.apache.avro.io.DatumWriter; -import org.apache.avro.reflect.ReflectDatumWriter; -import org.apache.beam.sdk.io.fs.ResourceId; -import org.apache.beam.sdk.options.ValueProvider; -import org.apache.beam.sdk.util.MimeTypes; -import org.checkerframework.checker.nullness.qual.Nullable; - -/** - * A {@link FileBasedSink} for Avro files. - * - * @deprecated Avro related classes are deprecated in module <code>beam-sdks-java-core</code> and - * will be eventually removed. Please, migrate to a new module <code> - * beam-sdks-java-extensions-avro</code> by importing <code> - * org.apache.beam.sdk.extensions.avro.io.AvroSink</code> instead of this one. - */ -@SuppressWarnings({ - "nullness" // TODO(https://github.com/apache/beam/issues/20497) -}) -@Deprecated -public class AvroSink<UserT, DestinationT, OutputT> - extends FileBasedSink<UserT, DestinationT, OutputT> { - private final boolean genericRecords; - private final int syncInterval; - - @FunctionalInterface - public interface DatumWriterFactory<T> extends Serializable { - DatumWriter<T> apply(Schema writer); - } - - AvroSink( - ValueProvider<ResourceId> outputPrefix, - DynamicAvroDestinations<UserT, DestinationT, OutputT> dynamicDestinations, - boolean genericRecords, - int syncInterval) { - // Avro handles compression internally using the codec. - super(outputPrefix, dynamicDestinations, Compression.UNCOMPRESSED); - this.genericRecords = genericRecords; - this.syncInterval = syncInterval; - } - - @Override - public DynamicAvroDestinations<UserT, DestinationT, OutputT> getDynamicDestinations() { - return (DynamicAvroDestinations<UserT, DestinationT, OutputT>) super.getDynamicDestinations(); - } - - @Override - public WriteOperation<DestinationT, OutputT> createWriteOperation() { - return new AvroWriteOperation<>(this, genericRecords, syncInterval); - } - - /** A {@link WriteOperation WriteOperation} for Avro files. */ - private static class AvroWriteOperation<DestinationT, OutputT> - extends WriteOperation<DestinationT, OutputT> { - private final DynamicAvroDestinations<?, DestinationT, OutputT> dynamicDestinations; - private final boolean genericRecords; - private final int syncInterval; - - private AvroWriteOperation( - AvroSink<?, DestinationT, OutputT> sink, boolean genericRecords, int syncInterval) { - super(sink); - this.dynamicDestinations = sink.getDynamicDestinations(); - this.genericRecords = genericRecords; - this.syncInterval = syncInterval; - } - - @Override - public Writer<DestinationT, OutputT> createWriter() throws Exception { - return new AvroWriter<>(this, dynamicDestinations, genericRecords, syncInterval); - } - } - - /** A {@link Writer Writer} for Avro files. */ - private static class AvroWriter<DestinationT, OutputT> extends Writer<DestinationT, OutputT> { - - // Initialized in prepareWrite - private @Nullable DataFileWriter<OutputT> dataFileWriter; - - private final DynamicAvroDestinations<?, DestinationT, OutputT> dynamicDestinations; - private final boolean genericRecords; - private final int syncInterval; - - public AvroWriter( - WriteOperation<DestinationT, OutputT> writeOperation, - DynamicAvroDestinations<?, DestinationT, OutputT> dynamicDestinations, - boolean genericRecords, - int syncInterval) { - super(writeOperation, MimeTypes.BINARY); - this.dynamicDestinations = dynamicDestinations; - this.genericRecords = genericRecords; - this.syncInterval = syncInterval; - } - - @SuppressWarnings("deprecation") // uses internal test functionality. - @Override - protected void prepareWrite(WritableByteChannel channel) throws Exception { - DestinationT destination = getDestination(); - CodecFactory codec = dynamicDestinations.getCodec(destination); - Schema schema = dynamicDestinations.getSchema(destination); - Map<String, Object> metadata = dynamicDestinations.getMetadata(destination); - DatumWriter<OutputT> datumWriter; - DatumWriterFactory<OutputT> datumWriterFactory = - dynamicDestinations.getDatumWriterFactory(destination); - - if (datumWriterFactory == null) { - datumWriter = - genericRecords ? new GenericDatumWriter<>(schema) : new ReflectDatumWriter<>(schema); - } else { - datumWriter = datumWriterFactory.apply(schema); - } - - dataFileWriter = new DataFileWriter<>(datumWriter).setCodec(codec); - for (Map.Entry<String, Object> entry : metadata.entrySet()) { - Object v = entry.getValue(); - if (v instanceof String) { - dataFileWriter.setMeta(entry.getKey(), (String) v); - } else if (v instanceof Long) { - dataFileWriter.setMeta(entry.getKey(), (Long) v); - } else if (v instanceof byte[]) { - dataFileWriter.setMeta(entry.getKey(), (byte[]) v); - } else { - throw new IllegalStateException( - "Metadata value type must be one of String, Long, or byte[]. Found " - + v.getClass().getSimpleName()); - } - } - dataFileWriter.setSyncInterval(syncInterval); - dataFileWriter.create(schema, Channels.newOutputStream(channel)); - } - - @Override - public void write(OutputT value) throws Exception { - dataFileWriter.append(value); - } - - @Override - protected void finishWrite() throws Exception { - dataFileWriter.flush(); - } - } -} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSource.java deleted file mode 100644 index f19375a7bcdc..000000000000 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSource.java +++ /dev/null @@ -1,773 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.io; - -import static org.apache.beam.sdk.io.FileBasedSource.Mode.SINGLE_FILE_OR_SUBRANGE; -import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; -import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull; -import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; - -import java.io.IOException; -import java.io.InputStream; -import java.io.InvalidObjectException; -import java.io.ObjectInputStream; -import java.io.ObjectStreamException; -import java.io.Serializable; -import java.nio.ByteBuffer; -import java.nio.channels.Channels; -import java.nio.channels.ReadableByteChannel; -import java.nio.channels.SeekableByteChannel; -import java.nio.charset.StandardCharsets; -import java.util.Arrays; -import java.util.Iterator; -import java.util.Map; -import java.util.WeakHashMap; -import javax.annotation.concurrent.GuardedBy; -import org.apache.avro.Schema; -import org.apache.avro.file.DataFileConstants; -import org.apache.avro.file.DataFileReader; -import org.apache.avro.file.SeekableInput; -import org.apache.avro.generic.GenericDatumReader; -import org.apache.avro.generic.GenericRecord; -import org.apache.avro.io.BinaryDecoder; -import org.apache.avro.io.DatumReader; -import org.apache.avro.io.DecoderFactory; -import org.apache.avro.reflect.ReflectData; -import org.apache.avro.reflect.ReflectDatumReader; -import org.apache.beam.sdk.PipelineRunner; -import org.apache.beam.sdk.coders.AvroCoder; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.io.fs.EmptyMatchTreatment; -import org.apache.beam.sdk.io.fs.MatchResult.Metadata; -import org.apache.beam.sdk.io.fs.ResourceId; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.options.ValueProvider; -import org.apache.beam.sdk.transforms.SerializableFunction; -import org.apache.beam.sdk.util.VarInt; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; -import org.checkerframework.checker.nullness.qual.Nullable; - -// CHECKSTYLE.OFF: JavadocStyle -/** - * Do not use in pipelines directly: most users should use {@link AvroIO.Read}. - * - * <p>A {@link FileBasedSource} for reading Avro files. - * - * <p>To read a {@link PCollection} of objects from one or more Avro files, use {@link - * AvroSource#from} to specify the path(s) of the files to read. The {@link AvroSource} that is - * returned will read objects of type {@link GenericRecord} with the schema(s) that were written at - * file creation. To further configure the {@link AvroSource} to read with a user-defined schema, or - * to return records of a type other than {@link GenericRecord}, use {@link - * AvroSource#withSchema(Schema)} (using an Avro {@link Schema}), {@link - * AvroSource#withSchema(String)} (using a JSON schema), or {@link AvroSource#withSchema(Class)} (to - * return objects of the Avro-generated class specified). - * - * <p>An {@link AvroSource} can be read from using the {@link Read} transform. For example: - * - * <pre>{@code - * AvroSource<MyType> source = AvroSource.from(file.toPath()).withSchema(MyType.class); - * PCollection<MyType> records = Read.from(mySource); - * }</pre> - * - * <p>This class's implementation is based on the <a - * href="https://avro.apache.org/docs/1.7.7/spec.html">Avro 1.7.7</a> specification and implements - * parsing of some parts of Avro Object Container Files. The rationale for doing so is that the Avro - * API does not provide efficient ways of computing the precise offsets of blocks within a file, - * which is necessary to support dynamic work rebalancing. However, whenever it is possible to use - * the Avro API in a way that supports maintaining precise offsets, this class uses the Avro API. - * - * <p>Avro Object Container files store records in blocks. Each block contains a collection of - * records. Blocks may be encoded (e.g., with bzip2, deflate, snappy, etc.). Blocks are delineated - * from one another by a 16-byte sync marker. - * - * <p>An {@link AvroSource} for a subrange of a single file contains records in the blocks such that - * the start offset of the block is greater than or equal to the start offset of the source and less - * than the end offset of the source. - * - * <p>To use XZ-encoded Avro files, please include an explicit dependency on {@code xz-1.8.jar}, - * which has been marked as optional in the Maven {@code sdk/pom.xml}. - * - * <pre>{@code - * <dependency> - * <groupId>org.tukaani</groupId> - * <artifactId>xz</artifactId> - * <version>1.8</version> - * </dependency> - * }</pre> - * - * <h3>Permissions</h3> - * - * <p>Permission requirements depend on the {@link PipelineRunner} that is used to execute the - * pipeline. Please refer to the documentation of corresponding {@link PipelineRunner}s for more - * details. - * - * @param <T> The type of records to be read from the source. - * @deprecated Avro related classes are deprecated in module <code>beam-sdks-java-core</code> and - * will be eventually removed. Please, migrate to a new module <code> - * beam-sdks-java-extensions-avro</code> by importing <code> - * org.apache.beam.sdk.extensions.avro.io.AvroSource</code> instead of this one. - */ -// CHECKSTYLE.ON: JavadocStyle - -@SuppressWarnings({ - "nullness" // TODO(https://github.com/apache/beam/issues/20497) -}) -@Deprecated -public class AvroSource<T> extends BlockBasedSource<T> { - // Default minimum bundle size (chosen as two default-size Avro blocks to attempt to - // ensure that every source has at least one block of records). - // The default sync interval is 64k. - private static final long DEFAULT_MIN_BUNDLE_SIZE = 2L * DataFileConstants.DEFAULT_SYNC_INTERVAL; - - @FunctionalInterface - public interface DatumReaderFactory<T> extends Serializable { - DatumReader<T> apply(Schema writer, Schema reader); - } - - private static final DatumReaderFactory<?> GENERIC_DATUM_READER_FACTORY = GenericDatumReader::new; - - private static final DatumReaderFactory<?> REFLECT_DATUM_READER_FACTORY = ReflectDatumReader::new; - - // Use cases of AvroSource are: - // 1) AvroSource<GenericRecord> Reading GenericRecord records with a specified schema. - // 2) AvroSource<Foo> Reading records of a generated Avro class Foo. - // 3) AvroSource<T> Reading GenericRecord records with an unspecified schema - // and converting them to type T. - // | Case 1 | Case 2 | Case 3 | - // type | GenericRecord | Foo | GenericRecord | - // readerSchemaString | non-null | non-null | null | - // parseFn | null | null | non-null | - // outputCoder | null | null | non-null | - // readerFactory | either | either | either | - private static class Mode<T> implements Serializable { - private final Class<?> type; - - // The JSON schema used to decode records. - private @Nullable String readerSchemaString; - - private final @Nullable SerializableFunction<GenericRecord, T> parseFn; - - private final @Nullable Coder<T> outputCoder; - - private final @Nullable DatumReaderFactory<?> readerFactory; - - private Mode( - Class<?> type, - @Nullable String readerSchemaString, - @Nullable SerializableFunction<GenericRecord, T> parseFn, - @Nullable Coder<T> outputCoder, - @Nullable DatumReaderFactory<?> readerFactory) { - this.type = type; - this.readerSchemaString = internSchemaString(readerSchemaString); - this.parseFn = parseFn; - this.outputCoder = outputCoder; - this.readerFactory = readerFactory; - } - - private void readObject(ObjectInputStream is) throws IOException, ClassNotFoundException { - is.defaultReadObject(); - readerSchemaString = internSchemaString(readerSchemaString); - } - - private Coder<T> getOutputCoder() { - if (parseFn == null) { - return AvroCoder.of((Class<T>) type, internOrParseSchemaString(readerSchemaString)); - } else { - return outputCoder; - } - } - - private void validate() { - if (parseFn == null) { - checkArgument( - readerSchemaString != null, - "schema must be specified using withSchema() when not using a parse fn"); - } - } - - private Mode<T> withReaderFactory(DatumReaderFactory<?> factory) { - return new Mode<>(type, readerSchemaString, parseFn, outputCoder, factory); - } - - private DatumReader<?> createReader(Schema writerSchema, Schema readerSchema) { - DatumReaderFactory<?> factory = this.readerFactory; - if (factory == null) { - factory = - (type == GenericRecord.class) - ? GENERIC_DATUM_READER_FACTORY - : REFLECT_DATUM_READER_FACTORY; - } - return factory.apply(writerSchema, readerSchema); - } - } - - private static Mode<GenericRecord> readGenericRecordsWithSchema( - String schema, @Nullable DatumReaderFactory<?> factory) { - return new Mode<>(GenericRecord.class, schema, null, null, factory); - } - - private static <T> Mode<T> readGeneratedClasses( - Class<T> clazz, @Nullable DatumReaderFactory<?> factory) { - return new Mode<>(clazz, ReflectData.get().getSchema(clazz).toString(), null, null, factory); - } - - private static <T> Mode<T> parseGenericRecords( - SerializableFunction<GenericRecord, T> parseFn, - Coder<T> outputCoder, - @Nullable DatumReaderFactory<?> factory) { - return new Mode<>(GenericRecord.class, null, parseFn, outputCoder, factory); - } - - private final Mode<T> mode; - - /** - * Reads from the given file name or pattern ("glob"). The returned source needs to be further - * configured by calling {@link #withSchema} to return a type other than {@link GenericRecord}. - */ - public static AvroSource<GenericRecord> from(ValueProvider<String> fileNameOrPattern) { - return new AvroSource<>( - fileNameOrPattern, - EmptyMatchTreatment.DISALLOW, - DEFAULT_MIN_BUNDLE_SIZE, - readGenericRecordsWithSchema(null /* will need to be specified in withSchema */, null)); - } - - public static AvroSource<GenericRecord> from(Metadata metadata) { - return new AvroSource<>( - metadata, - DEFAULT_MIN_BUNDLE_SIZE, - 0, - metadata.sizeBytes(), - readGenericRecordsWithSchema(null /* will need to be specified in withSchema */, null)); - } - - /** Like {@link #from(ValueProvider)}. */ - public static AvroSource<GenericRecord> from(String fileNameOrPattern) { - return from(ValueProvider.StaticValueProvider.of(fileNameOrPattern)); - } - - public AvroSource<T> withEmptyMatchTreatment(EmptyMatchTreatment emptyMatchTreatment) { - return new AvroSource<>( - getFileOrPatternSpecProvider(), emptyMatchTreatment, getMinBundleSize(), mode); - } - - /** Reads files containing records that conform to the given schema. */ - public AvroSource<GenericRecord> withSchema(String schema) { - checkArgument(schema != null, "schema can not be null"); - return new AvroSource<>( - getFileOrPatternSpecProvider(), - getEmptyMatchTreatment(), - getMinBundleSize(), - readGenericRecordsWithSchema(schema, mode.readerFactory)); - } - - /** Like {@link #withSchema(String)}. */ - public AvroSource<GenericRecord> withSchema(Schema schema) { - checkArgument(schema != null, "schema can not be null"); - return withSchema(schema.toString()); - } - - /** Reads files containing records of the given class. */ - public <X> AvroSource<X> withSchema(Class<X> clazz) { - checkArgument(clazz != null, "clazz can not be null"); - if (getMode() == SINGLE_FILE_OR_SUBRANGE) { - return new AvroSource<>( - getSingleFileMetadata(), - getMinBundleSize(), - getStartOffset(), - getEndOffset(), - readGeneratedClasses(clazz, mode.readerFactory)); - } - return new AvroSource<>( - getFileOrPatternSpecProvider(), - getEmptyMatchTreatment(), - getMinBundleSize(), - readGeneratedClasses(clazz, mode.readerFactory)); - } - - /** - * Reads {@link GenericRecord} of unspecified schema and maps them to instances of a custom type - * using the given {@code parseFn} and encoded using the given coder. - */ - public <X> AvroSource<X> withParseFn( - SerializableFunction<GenericRecord, X> parseFn, Coder<X> coder) { - checkArgument(parseFn != null, "parseFn can not be null"); - checkArgument(coder != null, "coder can not be null"); - if (getMode() == SINGLE_FILE_OR_SUBRANGE) { - return new AvroSource<>( - getSingleFileMetadata(), - getMinBundleSize(), - getStartOffset(), - getEndOffset(), - parseGenericRecords(parseFn, coder, mode.readerFactory)); - } - return new AvroSource<>( - getFileOrPatternSpecProvider(), - getEmptyMatchTreatment(), - getMinBundleSize(), - parseGenericRecords(parseFn, coder, mode.readerFactory)); - } - - /** - * Sets the minimum bundle size. Refer to {@link OffsetBasedSource} for a description of {@code - * minBundleSize} and its use. - */ - public AvroSource<T> withMinBundleSize(long minBundleSize) { - if (getMode() == SINGLE_FILE_OR_SUBRANGE) { - return new AvroSource<>( - getSingleFileMetadata(), minBundleSize, getStartOffset(), getEndOffset(), mode); - } - return new AvroSource<>( - getFileOrPatternSpecProvider(), getEmptyMatchTreatment(), minBundleSize, mode); - } - - public AvroSource<T> withDatumReaderFactory(DatumReaderFactory<?> factory) { - Mode<T> newMode = mode.withReaderFactory(factory); - if (getMode() == SINGLE_FILE_OR_SUBRANGE) { - return new AvroSource<>( - getSingleFileMetadata(), getMinBundleSize(), getStartOffset(), getEndOffset(), newMode); - } - return new AvroSource<>( - getFileOrPatternSpecProvider(), getEmptyMatchTreatment(), getMinBundleSize(), newMode); - } - - /** Constructor for FILEPATTERN mode. */ - private AvroSource( - ValueProvider<String> fileNameOrPattern, - EmptyMatchTreatment emptyMatchTreatment, - long minBundleSize, - Mode<T> mode) { - super(fileNameOrPattern, emptyMatchTreatment, minBundleSize); - this.mode = mode; - } - - /** Constructor for SINGLE_FILE_OR_SUBRANGE mode. */ - private AvroSource( - Metadata metadata, long minBundleSize, long startOffset, long endOffset, Mode<T> mode) { - super(metadata, minBundleSize, startOffset, endOffset); - this.mode = mode; - } - - @Override - public void validate() { - super.validate(); - mode.validate(); - } - - /** - * Used by the Dataflow worker. Do not introduce new usages. Do not delete without confirming that - * Dataflow ValidatesRunner tests pass. - * - * @deprecated Used by Dataflow worker - */ - @Deprecated - public BlockBasedSource<T> createForSubrangeOfFile(String fileName, long start, long end) - throws IOException { - return createForSubrangeOfFile(FileSystems.matchSingleFileSpec(fileName), start, end); - } - - @Override - public BlockBasedSource<T> createForSubrangeOfFile(Metadata fileMetadata, long start, long end) { - return new AvroSource<>(fileMetadata, getMinBundleSize(), start, end, mode); - } - - @Override - protected BlockBasedReader<T> createSingleFileReader(PipelineOptions options) { - return new AvroReader<>(this); - } - - @Override - public Coder<T> getOutputCoder() { - return mode.getOutputCoder(); - } - - @VisibleForTesting - @Nullable - String getReaderSchemaString() { - return mode.readerSchemaString; - } - - /** Avro file metadata. */ - @VisibleForTesting - static class AvroMetadata { - private final byte[] syncMarker; - private final String codec; - private final String schemaString; - - AvroMetadata(byte[] syncMarker, String codec, String schemaString) { - this.syncMarker = checkNotNull(syncMarker, "syncMarker"); - this.codec = checkNotNull(codec, "codec"); - this.schemaString = internSchemaString(checkNotNull(schemaString, "schemaString")); - } - - /** - * The JSON-encoded <a href="https://avro.apache.org/docs/1.7.7/spec.html#schemas">schema</a> - * string for the file. - */ - public String getSchemaString() { - return schemaString; - } - - /** - * The <a href="https://avro.apache.org/docs/1.7.7/spec.html#Required+Codecs">codec</a> of the - * file. - */ - public String getCodec() { - return codec; - } - - /** - * The 16-byte sync marker for the file. See the documentation for <a - * href="https://avro.apache.org/docs/1.7.7/spec.html#Object+Container+Files">Object Container - * File</a> for more information. - */ - public byte[] getSyncMarker() { - return syncMarker; - } - } - - /** - * Reads the {@link AvroMetadata} from the header of an Avro file. - * - * <p>This method parses the header of an Avro <a - * href="https://avro.apache.org/docs/1.7.7/spec.html#Object+Container+Files">Object Container - * File</a>. - * - * @throws IOException if the file is an invalid format. - */ - @VisibleForTesting - static AvroMetadata readMetadataFromFile(ResourceId fileResource) throws IOException { - String codec = null; - String schemaString = null; - byte[] syncMarker; - try (InputStream stream = Channels.newInputStream(FileSystems.open(fileResource))) { - BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(stream, null); - - // The header of an object container file begins with a four-byte magic number, followed - // by the file metadata (including the schema and codec), encoded as a map. Finally, the - // header ends with the file's 16-byte sync marker. - // See https://avro.apache.org/docs/1.7.7/spec.html#Object+Container+Files for details on - // the encoding of container files. - - // Read the magic number. - byte[] magic = new byte[DataFileConstants.MAGIC.length]; - decoder.readFixed(magic); - if (!Arrays.equals(magic, DataFileConstants.MAGIC)) { - throw new IOException("Missing Avro file signature: " + fileResource); - } - - // Read the metadata to find the codec and schema. - ByteBuffer valueBuffer = ByteBuffer.allocate(512); - long numRecords = decoder.readMapStart(); - while (numRecords > 0) { - for (long recordIndex = 0; recordIndex < numRecords; recordIndex++) { - String key = decoder.readString(); - // readBytes() clears the buffer and returns a buffer where: - // - position is the start of the bytes read - // - limit is the end of the bytes read - valueBuffer = decoder.readBytes(valueBuffer); - byte[] bytes = new byte[valueBuffer.remaining()]; - valueBuffer.get(bytes); - if (key.equals(DataFileConstants.CODEC)) { - codec = new String(bytes, StandardCharsets.UTF_8); - } else if (key.equals(DataFileConstants.SCHEMA)) { - schemaString = new String(bytes, StandardCharsets.UTF_8); - } - } - numRecords = decoder.mapNext(); - } - if (codec == null) { - codec = DataFileConstants.NULL_CODEC; - } - - // Finally, read the sync marker. - syncMarker = new byte[DataFileConstants.SYNC_SIZE]; - decoder.readFixed(syncMarker); - } - checkState(schemaString != null, "No schema present in Avro file metadata %s", fileResource); - return new AvroMetadata(syncMarker, codec, schemaString); - } - - // A logical reference cache used to store schemas and schema strings to allow us to - // "intern" values and reduce the number of copies of equivalent objects. - private static final Map<String, Schema> schemaLogicalReferenceCache = new WeakHashMap<>(); - private static final Map<String, String> schemaStringLogicalReferenceCache = new WeakHashMap<>(); - - // We avoid String.intern() because depending on the JVM, these may be added to the PermGenSpace - // which we want to avoid otherwise we could run out of PermGenSpace. - private static synchronized String internSchemaString(String schema) { - String internSchema = schemaStringLogicalReferenceCache.get(schema); - if (internSchema != null) { - return internSchema; - } - schemaStringLogicalReferenceCache.put(schema, schema); - return schema; - } - - static synchronized Schema internOrParseSchemaString(String schemaString) { - Schema schema = schemaLogicalReferenceCache.get(schemaString); - if (schema != null) { - return schema; - } - Schema.Parser parser = new Schema.Parser(); - schema = parser.parse(schemaString); - schemaLogicalReferenceCache.put(schemaString, schema); - return schema; - } - - // Reading the object from Java serialization typically does not go through the constructor, - // we use readResolve to replace the constructed instance with one which uses the constructor - // allowing us to intern any schemas. - @SuppressWarnings("unused") - private Object readResolve() throws ObjectStreamException { - switch (getMode()) { - case SINGLE_FILE_OR_SUBRANGE: - return new AvroSource<>( - getSingleFileMetadata(), getMinBundleSize(), getStartOffset(), getEndOffset(), mode); - case FILEPATTERN: - return new AvroSource<>( - getFileOrPatternSpecProvider(), getEmptyMatchTreatment(), getMinBundleSize(), mode); - default: - throw new InvalidObjectException( - String.format("Unknown mode %s for AvroSource %s", getMode(), this)); - } - } - - /** - * A {@link BlockBasedSource.Block} of Avro records. - * - * @param <T> The type of records stored in the block. - */ - static class AvroBlock<T> extends Block<T> { - - // The current record in the block. Initialized in readNextRecord. - private @Nullable T currentRecord; - - // The index of the current record in the block. - private long currentRecordIndex = 0; - - private final Iterator<?> iterator; - - private final SerializableFunction<GenericRecord, T> parseFn; - - private final long numRecordsInBlock; - - AvroBlock( - Iterator<?> iter, SerializableFunction<GenericRecord, T> parseFn, long numRecordsInBlock) { - this.iterator = iter; - this.parseFn = parseFn; - this.numRecordsInBlock = numRecordsInBlock; - } - - @Override - public T getCurrentRecord() { - return currentRecord; - } - - @Override - public boolean readNextRecord() { - if (currentRecordIndex >= numRecordsInBlock) { - return false; - } - - Object record = iterator.next(); - currentRecord = (parseFn == null) ? ((T) record) : parseFn.apply((GenericRecord) record); - currentRecordIndex++; - return true; - } - - @Override - public double getFractionOfBlockConsumed() { - return ((double) currentRecordIndex) / numRecordsInBlock; - } - } - - /** - * A {@link BlockBasedSource.BlockBasedReader} for reading blocks from Avro files. - * - * <p>An Avro Object Container File consists of a header followed by a 16-bit sync marker and then - * a sequence of blocks, where each block begins with two encoded longs representing the total - * number of records in the block and the block's size in bytes, followed by the block's - * (optionally-encoded) records. Each block is terminated by a 16-bit sync marker. - * - * @param <T> The type of records contained in the block. - */ - public static class AvroReader<T> extends BlockBasedReader<T> { - - private static class SeekableChannelInput implements SeekableInput { - - private final SeekableByteChannel channel; - private final InputStream input; - - SeekableChannelInput(SeekableByteChannel channel) { - this.channel = channel; - this.input = Channels.newInputStream(channel); - } - - @Override - public void seek(long p) throws IOException { - channel.position(p); - } - - @Override - public long tell() throws IOException { - return channel.position(); - } - - @Override - public long length() throws IOException { - return channel.size(); - } - - @Override - public int read(byte[] b, int off, int len) throws IOException { - return input.read(b, off, len); - } - - @Override - public void close() throws IOException { - channel.close(); - } - } - - // The current block. - // Initialized in readNextRecord. - private @Nullable AvroBlock<T> currentBlock; - - private @Nullable DataFileReader<?> dataFileReader; - - // A lock used to synchronize block offsets for getRemainingParallelism - private final Object progressLock = new Object(); - - // Offset of the current block. - @GuardedBy("progressLock") - private long currentBlockOffset = 0; - - // Size of the current block. - @GuardedBy("progressLock") - private long currentBlockSizeBytes = 0; - - /** Reads Avro records of type {@code T} from the specified source. */ - public AvroReader(AvroSource<T> source) { - super(source); - } - - @Override - public synchronized AvroSource<T> getCurrentSource() { - return (AvroSource<T>) super.getCurrentSource(); - } - - // Precondition: the stream is positioned after the sync marker in the current (about to be - // previous) block. currentBlockSize equals the size of the current block, or zero if this - // reader was just started. - // - // Postcondition: same as above, but for the new current (formerly next) block. - @Override - public boolean readNextBlock() { - if (!dataFileReader.hasNext()) { - return false; - } - - long headerLength = - (long) VarInt.getLength(dataFileReader.getBlockCount()) - + VarInt.getLength(dataFileReader.getBlockSize()) - + DataFileConstants.SYNC_SIZE; - - currentBlock = - new AvroBlock<>( - dataFileReader, getCurrentSource().mode.parseFn, dataFileReader.getBlockCount()); - - // Atomically update both the position and offset of the new block. - synchronized (progressLock) { - currentBlockOffset = dataFileReader.previousSync(); - // Total block size includes the header, block content, and trailing sync marker. - currentBlockSizeBytes = dataFileReader.getBlockSize() + headerLength; - } - - return true; - } - - @Override - public AvroBlock<T> getCurrentBlock() { - return currentBlock; - } - - @Override - public long getCurrentBlockOffset() { - synchronized (progressLock) { - return currentBlockOffset; - } - } - - @Override - public long getCurrentBlockSize() { - synchronized (progressLock) { - return currentBlockSizeBytes; - } - } - - @Override - public long getSplitPointsRemaining() { - if (isDone()) { - return 0; - } - synchronized (progressLock) { - if (currentBlockOffset + currentBlockSizeBytes >= getCurrentSource().getEndOffset()) { - // This block is known to be the last block in the range. - return 1; - } - } - return super.getSplitPointsRemaining(); - } - - // Postcondition: the stream is positioned at the beginning of the first block after the start - // of the current source, and currentBlockOffset is that position. Additionally, - // currentBlockSizeBytes will be set to 0 indicating that the previous block was empty. - @Override - protected void startReading(ReadableByteChannel channel) throws IOException { - SeekableChannelInput seekableChannelInput = - new SeekableChannelInput((SeekableByteChannel) channel); - // the channel needs to be at the beginning of the file in order for the DataFileReader to - // read the header, etc, we'll seek it back to where it should be after creating the DFR. - seekableChannelInput.seek(0); - - Schema readerSchema = null; - String readerSchemaString = this.getCurrentSource().getReaderSchemaString(); - if (readerSchemaString != null) { - readerSchema = AvroSource.internOrParseSchemaString(readerSchemaString); - } - // the DataFileReader will call setSchema with the writer schema when created. - DatumReader<?> reader = this.getCurrentSource().mode.createReader(readerSchema, readerSchema); - - dataFileReader = new DataFileReader<>(seekableChannelInput, reader); - - long startOffset = getCurrentSource().getStartOffset(); - if (startOffset != 0) { - // the start offset may be in the middle of a sync marker, by rewinding SYNC_SIZE bytes we - // ensure that we won't miss the block if so. - dataFileReader.sync(Math.max(0, startOffset - DataFileConstants.SYNC_SIZE)); - } - - synchronized (progressLock) { - currentBlockOffset = dataFileReader.previousSync(); - currentBlockSizeBytes = 0; - } - } - } -} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/ConstantAvroDestination.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/ConstantAvroDestination.java deleted file mode 100644 index dc7fee0d7ad5..000000000000 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/ConstantAvroDestination.java +++ /dev/null @@ -1,156 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.io; - -import java.io.Serializable; -import java.util.Map; -import org.apache.avro.Schema; -import org.apache.avro.file.CodecFactory; -import org.apache.beam.sdk.io.FileBasedSink.FilenamePolicy; -import org.apache.beam.sdk.transforms.SerializableFunction; -import org.apache.beam.sdk.transforms.display.DisplayData; -import org.apache.beam.sdk.transforms.display.HasDisplayData; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Function; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Supplier; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Suppliers; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.io.BaseEncoding; -import org.checkerframework.checker.nullness.qual.Nullable; - -/** - * Always returns a constant {@link FilenamePolicy}, {@link Schema}, metadata, and codec. - * - * @deprecated Avro related classes are deprecated in module <code>beam-sdks-java-core</code> and - * will be eventually removed. Please, migrate to a new module <code> - * beam-sdks-java-extensions-avro</code> by importing <code> - * org.apache.beam.sdk.extensions.avro.io.ConstantAvroDestination</code> instead of this one. - */ -@SuppressWarnings({ - "nullness" // TODO(https://github.com/apache/beam/issues/20497) -}) -@Deprecated -class ConstantAvroDestination<UserT, OutputT> - extends DynamicAvroDestinations<UserT, Void, OutputT> { - private static class SchemaFunction implements Serializable, Function<String, Schema> { - @Override - public Schema apply(String input) { - return new Schema.Parser().parse(input); - } - } - - // This should be a multiple of 4 to not get a partial encoded byte. - private static final int METADATA_BYTES_MAX_LENGTH = 40; - private final FilenamePolicy filenamePolicy; - private final Supplier<Schema> schema; - private final Map<String, Object> metadata; - private final SerializableAvroCodecFactory codec; - private final SerializableFunction<UserT, OutputT> formatFunction; - private final AvroSink.DatumWriterFactory<OutputT> datumWriterFactory; - - private class Metadata implements HasDisplayData { - @Override - public void populateDisplayData(DisplayData.Builder builder) { - for (Map.Entry<String, Object> entry : metadata.entrySet()) { - DisplayData.Type type = DisplayData.inferType(entry.getValue()); - if (type != null) { - builder.add(DisplayData.item(entry.getKey(), type, entry.getValue())); - } else { - String base64 = BaseEncoding.base64().encode((byte[]) entry.getValue()); - String repr = - base64.length() <= METADATA_BYTES_MAX_LENGTH - ? base64 - : base64.substring(0, METADATA_BYTES_MAX_LENGTH) + "..."; - builder.add(DisplayData.item(entry.getKey(), repr)); - } - } - } - } - - public ConstantAvroDestination( - FilenamePolicy filenamePolicy, - Schema schema, - Map<String, Object> metadata, - CodecFactory codec, - SerializableFunction<UserT, OutputT> formatFunction) { - this(filenamePolicy, schema, metadata, codec, formatFunction, null); - } - - public ConstantAvroDestination( - FilenamePolicy filenamePolicy, - Schema schema, - Map<String, Object> metadata, - CodecFactory codec, - SerializableFunction<UserT, OutputT> formatFunction, - AvroSink.@Nullable DatumWriterFactory<OutputT> datumWriterFactory) { - this.filenamePolicy = filenamePolicy; - this.schema = Suppliers.compose(new SchemaFunction(), Suppliers.ofInstance(schema.toString())); - this.metadata = metadata; - this.codec = new SerializableAvroCodecFactory(codec); - this.formatFunction = formatFunction; - this.datumWriterFactory = datumWriterFactory; - } - - @Override - public OutputT formatRecord(UserT record) { - return formatFunction.apply(record); - } - - @Override - public @Nullable Void getDestination(UserT element) { - return (Void) null; - } - - @Override - public @Nullable Void getDefaultDestination() { - return (Void) null; - } - - @Override - public FilenamePolicy getFilenamePolicy(Void destination) { - return filenamePolicy; - } - - @Override - public Schema getSchema(Void destination) { - return schema.get(); - } - - @Override - public Map<String, Object> getMetadata(Void destination) { - return metadata; - } - - @Override - public CodecFactory getCodec(Void destination) { - return codec.getCodec(); - } - - @Override - public AvroSink.@Nullable DatumWriterFactory<OutputT> getDatumWriterFactory(Void destination) { - return datumWriterFactory; - } - - @Override - public void populateDisplayData(DisplayData.Builder builder) { - filenamePolicy.populateDisplayData(builder); - builder.add(DisplayData.item("schema", schema.get().toString()).withLabel("Record Schema")); - builder.addIfNotDefault( - DisplayData.item("codec", codec.getCodec().toString()).withLabel("Avro Compression Codec"), - AvroIO.TypedWrite.DEFAULT_SERIALIZABLE_CODEC.toString()); - builder.include("Metadata", new Metadata()); - } -} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingSource.java index 23896c8cc962..9d30efb2f113 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingSource.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingSource.java @@ -20,13 +20,19 @@ import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull; +import java.io.DataInputStream; +import java.io.DataOutputStream; import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; import java.util.List; import java.util.NoSuchElementException; import java.util.Objects; -import org.apache.beam.sdk.coders.AvroCoder; +import org.apache.beam.sdk.coders.BigEndianLongCoder; import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CustomCoder; import org.apache.beam.sdk.coders.DefaultCoder; +import org.apache.beam.sdk.coders.InstantCoder; import org.apache.beam.sdk.coders.VarLongCoder; import org.apache.beam.sdk.io.UnboundedSource.UnboundedReader; import org.apache.beam.sdk.metrics.Counter; @@ -354,7 +360,7 @@ public UnboundedReader<Long> createReader(PipelineOptions options, CounterMark c @Override public Coder<CountingSource.CounterMark> getCheckpointMarkCoder() { - return AvroCoder.of(CountingSource.CounterMark.class); + return new CounterMarkCoder(); } @Override @@ -485,7 +491,7 @@ public long getSplitBacklogBytes() { * The checkpoint for an unbounded {@link CountingSource} is simply the last value produced. The * associated source object encapsulates the information needed to produce the next value. */ - @DefaultCoder(AvroCoder.class) + @DefaultCoder(CounterMarkCoder.class) public static class CounterMark implements UnboundedSource.CheckpointMark { /** The last value emitted. */ private final long lastEmitted; @@ -519,4 +525,22 @@ private CounterMark() { @Override public void finalizeCheckpoint() throws IOException {} } + + /** A custom coder for {@code CounterMark}. */ + public static class CounterMarkCoder extends CustomCoder<CounterMark> { + @Override + public void encode(CounterMark value, OutputStream outStream) throws IOException { + DataOutputStream stream = new DataOutputStream(outStream); + BigEndianLongCoder.of().encode(value.lastEmitted, stream); + InstantCoder.of().encode(value.startTime, stream); + } + + @Override + public CounterMark decode(InputStream inStream) throws IOException { + DataInputStream stream = new DataInputStream(inStream); + long lastEmitted = BigEndianLongCoder.of().decode(stream); + Instant startTime = InstantCoder.of().decode(stream); + return new CounterMark(lastEmitted, startTime); + } + } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/DynamicAvroDestinations.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/DynamicAvroDestinations.java deleted file mode 100644 index 8094fe30d853..000000000000 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/DynamicAvroDestinations.java +++ /dev/null @@ -1,61 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.io; - -import java.util.Map; -import org.apache.avro.Schema; -import org.apache.avro.file.CodecFactory; -import org.apache.beam.sdk.io.FileBasedSink.DynamicDestinations; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; -import org.checkerframework.checker.nullness.qual.Nullable; - -/** - * A specialization of {@link DynamicDestinations} for {@link AvroIO}. In addition to dynamic file - * destinations, this allows specifying other AVRO properties (schema, metadata, codec, datum - * writer) per destination. - * - * @deprecated Avro related classes are deprecated in module <code>beam-sdks-java-core</code> and - * will be eventually removed. Please, migrate to a new module <code> - * beam-sdks-java-extensions-avro</code> by importing <code> - * org.apache.beam.sdk.extensions.avro.io.DynamicAvroDestinations</code> instead of this one. - */ -@Deprecated -public abstract class DynamicAvroDestinations<UserT, DestinationT, OutputT> - extends DynamicDestinations<UserT, DestinationT, OutputT> { - /** Return an AVRO schema for a given destination. */ - public abstract Schema getSchema(DestinationT destination); - - /** Return AVRO file metadata for a given destination. */ - public Map<String, Object> getMetadata(DestinationT destination) { - return ImmutableMap.of(); - } - - /** Return an AVRO codec for a given destination. */ - public CodecFactory getCodec(DestinationT destination) { - return AvroIO.TypedWrite.DEFAULT_CODEC; - } - - /** - * Return a {@link AvroSink.DatumWriterFactory} for a given destination. If provided, it will be - * used to created {@link org.apache.avro.io.DatumWriter} instances as required. - */ - public AvroSink.@Nullable DatumWriterFactory<OutputT> getDatumWriterFactory( - DestinationT destinationT) { - return null; - } -} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/SerializableAvroCodecFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/SerializableAvroCodecFactory.java deleted file mode 100644 index 29e14ae06668..000000000000 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/SerializableAvroCodecFactory.java +++ /dev/null @@ -1,119 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.io; - -import static org.apache.avro.file.DataFileConstants.BZIP2_CODEC; -import static org.apache.avro.file.DataFileConstants.DEFLATE_CODEC; -import static org.apache.avro.file.DataFileConstants.NULL_CODEC; -import static org.apache.avro.file.DataFileConstants.SNAPPY_CODEC; -import static org.apache.avro.file.DataFileConstants.XZ_CODEC; -import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull; -import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; - -import java.io.Externalizable; -import java.io.IOException; -import java.io.ObjectInput; -import java.io.ObjectOutput; -import java.util.Arrays; -import java.util.List; -import java.util.regex.Matcher; -import java.util.regex.Pattern; -import org.apache.avro.file.CodecFactory; -import org.checkerframework.checker.nullness.qual.Nullable; - -/** - * A wrapper that allows {@link org.apache.avro.file.CodecFactory}s to be serialized using Java's - * standard serialization mechanisms. - * - * @deprecated Avro related classes are deprecated in module <code>beam-sdks-java-core</code> and - * will be eventually removed. Please, migrate to a new module <code> - * beam-sdks-java-extensions-avro</code> by importing <code> - * org.apache.beam.sdk.extensions.avro.io.SerializableAvroCodecFactory</code> instead of this - * one. - */ -@SuppressWarnings({ - "nullness" // TODO(https://github.com/apache/beam/issues/20497) -}) -@Deprecated -class SerializableAvroCodecFactory implements Externalizable { - private static final long serialVersionUID = 7445324844109564303L; - private static final List<String> noOptAvroCodecs = - Arrays.asList(NULL_CODEC, SNAPPY_CODEC, BZIP2_CODEC); - private static final Pattern deflatePattern = Pattern.compile(DEFLATE_CODEC + "-(?<level>-?\\d)"); - private static final Pattern xzPattern = Pattern.compile(XZ_CODEC + "-(?<level>\\d)"); - - private @Nullable CodecFactory codecFactory; - - // For java.io.Externalizable - public SerializableAvroCodecFactory() {} - - public SerializableAvroCodecFactory(CodecFactory codecFactory) { - checkNotNull(codecFactory, "Codec can't be null"); - checkState(checkIsSupportedCodec(codecFactory), "%s is not supported", codecFactory); - this.codecFactory = codecFactory; - } - - private boolean checkIsSupportedCodec(CodecFactory codecFactory) { - final String codecStr = codecFactory.toString(); - return noOptAvroCodecs.contains(codecStr) - || deflatePattern.matcher(codecStr).matches() - || xzPattern.matcher(codecStr).matches(); - } - - @Override - public void writeExternal(ObjectOutput out) throws IOException { - out.writeUTF(codecFactory.toString()); - } - - @Override - public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { - final String codecStr = in.readUTF(); - - switch (codecStr) { - case NULL_CODEC: - case SNAPPY_CODEC: - case BZIP2_CODEC: - codecFactory = CodecFactory.fromString(codecStr); - return; - } - - Matcher deflateMatcher = deflatePattern.matcher(codecStr); - if (deflateMatcher.find()) { - codecFactory = CodecFactory.deflateCodec(Integer.parseInt(deflateMatcher.group("level"))); - return; - } - - Matcher xzMatcher = xzPattern.matcher(codecStr); - if (xzMatcher.find()) { - codecFactory = CodecFactory.xzCodec(Integer.parseInt(xzMatcher.group("level"))); - return; - } - - throw new IllegalStateException(codecStr + " is not supported"); - } - - public CodecFactory getCodec() { - return codecFactory; - } - - @Override - public String toString() { - checkNotNull(codecFactory, "Inner CodecFactory is null, please use non default constructor"); - return codecFactory.toString(); - } -} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/AvroRecordSchema.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/AvroRecordSchema.java deleted file mode 100644 index 19027cd4527f..000000000000 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/AvroRecordSchema.java +++ /dev/null @@ -1,64 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.schemas; - -import static org.apache.beam.sdk.schemas.utils.AvroUtils.toBeamSchema; - -import java.util.List; -import org.apache.avro.reflect.ReflectData; -import org.apache.beam.sdk.schemas.utils.AvroUtils; -import org.apache.beam.sdk.values.TypeDescriptor; - -/** - * A {@link SchemaProvider} for AVRO generated SpecificRecords and POJOs. - * - * <p>This provider infers a schema from generated SpecificRecord objects, and creates schemas and - * rows that bind to the appropriate fields. This provider also infers schemas from Java POJO - * objects, creating a schema that matches that inferred by the AVRO libraries. - * - * @deprecated Avro related classes are deprecated in module <code>beam-sdks-java-core</code> and - * will be eventually removed. Please, migrate to a new module <code> - * beam-sdks-java-extensions-avro</code> by importing <code> - * org.apache.beam.sdk.extensions.avro.schemas.AvroRecordSchema</code> instead of this one. - */ -@SuppressWarnings({ - "rawtypes" // TODO(https://github.com/apache/beam/issues/20447) -}) -@Deprecated -public class AvroRecordSchema extends GetterBasedSchemaProvider { - @Override - public <T> Schema schemaFor(TypeDescriptor<T> typeDescriptor) { - return toBeamSchema(ReflectData.get().getSchema(typeDescriptor.getRawType())); - } - - @Override - public List<FieldValueGetter> fieldValueGetters(Class<?> targetClass, Schema schema) { - return AvroUtils.getGetters(targetClass, schema); - } - - @Override - public List<FieldValueTypeInformation> fieldValueTypeInformations( - Class<?> targetClass, Schema schema) { - return AvroUtils.getFieldTypes(targetClass, schema); - } - - @Override - public SchemaUserTypeCreator schemaTypeCreator(Class<?> targetClass, Schema schema) { - return AvroUtils.getCreator(targetClass, schema); - } -} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/io/Providers.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/io/Providers.java index 35b89bb70180..ed3abcd1ba83 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/io/Providers.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/io/Providers.java @@ -40,27 +40,12 @@ private Providers() {} public static <T extends Identifyable> Map<String, T> loadProviders(Class<T> klass) { Map<String, T> providers = new HashMap<>(); for (T provider : ServiceLoader.load(klass)) { - // Avro provider is treated as a special case since two Avro providers may want to be loaded - - // from "core" (deprecated) and from "extensions/avro" (actual) - but only one must succeed. - // TODO: we won't need this check once all Avro providers from "core" will be - // removed - if (provider.identifier().equals("avro")) { - // Avro provider from "extensions/avro" must have a priority. - if (provider.getClass().getName().startsWith("org.apache.beam.sdk.extensions.avro")) { - // Load Avro provider from "extensions/avro" by any case. - providers.put(provider.identifier(), provider); - } else { - // Load Avro provider from "core" if it was not loaded from Avro extension before. - providers.putIfAbsent(provider.identifier(), provider); - } - } else { - checkState( - !providers.containsKey(provider.identifier()), - "Duplicate providers exist with identifier `%s` for class %s.", - provider.identifier(), - klass); - providers.put(provider.identifier(), provider); - } + checkState( + !providers.containsKey(provider.identifier()), + "Duplicate providers exist with identifier `%s` for class %s.", + provider.identifier(), + klass); + providers.put(provider.identifier(), provider); } return providers; } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/io/payloads/AvroPayloadSerializerProvider.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/io/payloads/AvroPayloadSerializerProvider.java deleted file mode 100644 index e8f99b33c0dd..000000000000 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/io/payloads/AvroPayloadSerializerProvider.java +++ /dev/null @@ -1,47 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.schemas.io.payloads; - -import com.google.auto.service.AutoService; -import java.util.Map; -import org.apache.beam.sdk.annotations.Internal; -import org.apache.beam.sdk.schemas.Schema; -import org.apache.beam.sdk.schemas.utils.AvroUtils; - -/** - * @deprecated Avro related classes are deprecated in module <code>beam-sdks-java-core</code> and - * will be eventually removed. Please, migrate to a new module <code> - * beam-sdks-java-extensions-avro</code> by importing <code> - * org.apache.beam.sdk.extensions.avro.schemas.io.payloads.AvroPayloadSerializerProvider</code> - * instead of this one. - */ -@Internal -@Deprecated -@AutoService(PayloadSerializerProvider.class) -public class AvroPayloadSerializerProvider implements PayloadSerializerProvider { - @Override - public String identifier() { - return "avro"; - } - - @Override - public PayloadSerializer getSerializer(Schema schema, Map<String, Object> tableParams) { - return PayloadSerializer.of( - AvroUtils.getRowToAvroBytesFunction(schema), AvroUtils.getAvroBytesToRowFunction(schema)); - } -} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/AvroByteBuddyUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/AvroByteBuddyUtils.java deleted file mode 100644 index ab17907f0b13..000000000000 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/AvroByteBuddyUtils.java +++ /dev/null @@ -1,149 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.schemas.utils; - -import static org.apache.beam.sdk.util.ByteBuddyUtils.getClassLoadingStrategy; - -import java.lang.reflect.Constructor; -import java.lang.reflect.InvocationTargetException; -import java.lang.reflect.Type; -import java.util.Map; -import net.bytebuddy.ByteBuddy; -import net.bytebuddy.asm.AsmVisitorWrapper; -import net.bytebuddy.description.type.TypeDescription.ForLoadedType; -import net.bytebuddy.dynamic.DynamicType; -import net.bytebuddy.implementation.MethodCall; -import net.bytebuddy.implementation.bytecode.StackManipulation; -import net.bytebuddy.implementation.bytecode.assign.TypeCasting; -import net.bytebuddy.implementation.bytecode.collection.ArrayAccess; -import net.bytebuddy.implementation.bytecode.constant.IntegerConstant; -import net.bytebuddy.implementation.bytecode.member.MethodVariableAccess; -import net.bytebuddy.jar.asm.ClassWriter; -import net.bytebuddy.matcher.ElementMatchers; -import org.apache.avro.specific.SpecificRecord; -import org.apache.beam.sdk.schemas.Schema; -import org.apache.beam.sdk.schemas.SchemaUserTypeCreator; -import org.apache.beam.sdk.schemas.utils.AvroUtils.AvroTypeConversionFactory; -import org.apache.beam.sdk.schemas.utils.ByteBuddyUtils.InjectPackageStrategy; -import org.apache.beam.sdk.schemas.utils.ByteBuddyUtils.TypeConversion; -import org.apache.beam.sdk.schemas.utils.ByteBuddyUtils.TypeConversionsFactory; -import org.apache.beam.sdk.schemas.utils.ReflectUtils.ClassWithSchema; -import org.apache.beam.sdk.util.common.ReflectHelpers; -import org.apache.beam.sdk.values.TypeDescriptor; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps; - -/** - * @deprecated Avro related classes are deprecated in module <code>beam-sdks-java-core</code> and - * will be eventually removed. Please, migrate to a new module <code> - * beam-sdks-java-extensions-avro</code> by importing <code> - * org.apache.beam.sdk.extensions.avro.schemas.utils.AvroByteBuddyUtils</code> instead of this - * one. - */ -@SuppressWarnings({ - "nullness", // TODO(https://github.com/apache/beam/issues/20497) - "rawtypes" -}) -@Deprecated -class AvroByteBuddyUtils { - private static final ByteBuddy BYTE_BUDDY = new ByteBuddy(); - - // Cache the generated constructors. - private static final Map<ClassWithSchema, SchemaUserTypeCreator> CACHED_CREATORS = - Maps.newConcurrentMap(); - - static <T extends SpecificRecord> SchemaUserTypeCreator getCreator( - Class<T> clazz, Schema schema) { - return CACHED_CREATORS.computeIfAbsent( - ClassWithSchema.create(clazz, schema), c -> createCreator(clazz, schema)); - } - - private static <T> SchemaUserTypeCreator createCreator(Class<T> clazz, Schema schema) { - Constructor baseConstructor = null; - Constructor[] constructors = clazz.getDeclaredConstructors(); - for (Constructor constructor : constructors) { - // TODO: This assumes that Avro only generates one constructor with this many fields. - if (constructor.getParameterCount() == schema.getFieldCount()) { - baseConstructor = constructor; - } - } - if (baseConstructor == null) { - throw new RuntimeException("No matching constructor found for class " + clazz); - } - - // Generate a method call to create and invoke the SpecificRecord's constructor. . - MethodCall construct = MethodCall.construct(baseConstructor); - for (int i = 0; i < baseConstructor.getParameterTypes().length; ++i) { - Class<?> baseType = baseConstructor.getParameterTypes()[i]; - construct = construct.with(readAndConvertParameter(baseType, i), baseType); - } - - try { - DynamicType.Builder<SchemaUserTypeCreator> builder = - BYTE_BUDDY - .with(new InjectPackageStrategy(clazz)) - .subclass(SchemaUserTypeCreator.class) - .method(ElementMatchers.named("create")) - .intercept(construct); - - return builder - .visit(new AsmVisitorWrapper.ForDeclaredMethods().writerFlags(ClassWriter.COMPUTE_FRAMES)) - .make() - .load( - ReflectHelpers.findClassLoader(clazz.getClassLoader()), - getClassLoadingStrategy(clazz)) - .getLoaded() - .getDeclaredConstructor() - .newInstance(); - } catch (InstantiationException - | IllegalAccessException - | NoSuchMethodException - | InvocationTargetException e) { - throw new RuntimeException( - "Unable to generate a getter for class " + clazz + " with schema " + schema); - } - } - - private static StackManipulation readAndConvertParameter( - Class<?> constructorParameterType, int index) { - TypeConversionsFactory typeConversionsFactory = new AvroTypeConversionFactory(); - - // The types in the AVRO-generated constructor might be the types returned by Beam's Row class, - // so we have to convert the types used by Beam's Row class. - // We know that AVRO generates constructor parameters in the same order as fields - // in the schema, so we can just add the parameters sequentially. - TypeConversion<Type> convertType = typeConversionsFactory.createTypeConversion(true); - - // Map the AVRO-generated type to the one Beam will use. - ForLoadedType convertedType = - new ForLoadedType((Class) convertType.convert(TypeDescriptor.of(constructorParameterType))); - - // This will run inside the generated creator. Read the parameter and convert it to the - // type required by the SpecificRecord constructor. - StackManipulation readParameter = - new StackManipulation.Compound( - MethodVariableAccess.REFERENCE.loadFrom(1), - IntegerConstant.forValue(index), - ArrayAccess.REFERENCE.load(), - TypeCasting.to(convertedType)); - - // Convert to the parameter accepted by the SpecificRecord constructor. - return typeConversionsFactory - .createSetterConversions(readParameter) - .convert(TypeDescriptor.of(constructorParameterType)); - } -} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/AvroUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/AvroUtils.java deleted file mode 100644 index cd69f139ae61..000000000000 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/AvroUtils.java +++ /dev/null @@ -1,1396 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.schemas.utils; - -import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; -import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull; - -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; -import java.lang.reflect.Method; -import java.math.BigDecimal; -import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.stream.Collectors; -import javax.annotation.Nonnull; -import net.bytebuddy.description.type.TypeDescription.ForLoadedType; -import net.bytebuddy.implementation.bytecode.Duplication; -import net.bytebuddy.implementation.bytecode.StackManipulation; -import net.bytebuddy.implementation.bytecode.StackManipulation.Compound; -import net.bytebuddy.implementation.bytecode.TypeCreation; -import net.bytebuddy.implementation.bytecode.assign.TypeCasting; -import net.bytebuddy.implementation.bytecode.member.MethodInvocation; -import net.bytebuddy.matcher.ElementMatchers; -import org.apache.avro.AvroRuntimeException; -import org.apache.avro.Conversions; -import org.apache.avro.LogicalType; -import org.apache.avro.LogicalTypes; -import org.apache.avro.Schema.Type; -import org.apache.avro.generic.GenericData; -import org.apache.avro.generic.GenericFixed; -import org.apache.avro.generic.GenericRecord; -import org.apache.avro.generic.GenericRecordBuilder; -import org.apache.avro.reflect.AvroIgnore; -import org.apache.avro.reflect.AvroName; -import org.apache.avro.reflect.ReflectData; -import org.apache.avro.specific.SpecificData; -import org.apache.avro.specific.SpecificRecord; -import org.apache.avro.util.Utf8; -import org.apache.beam.sdk.coders.AvroCoder; -import org.apache.beam.sdk.coders.AvroCoder.JodaTimestampConversion; -import org.apache.beam.sdk.schemas.AvroRecordSchema; -import org.apache.beam.sdk.schemas.FieldValueGetter; -import org.apache.beam.sdk.schemas.FieldValueTypeInformation; -import org.apache.beam.sdk.schemas.Schema; -import org.apache.beam.sdk.schemas.Schema.Field; -import org.apache.beam.sdk.schemas.Schema.FieldType; -import org.apache.beam.sdk.schemas.Schema.TypeName; -import org.apache.beam.sdk.schemas.SchemaCoder; -import org.apache.beam.sdk.schemas.SchemaUserTypeCreator; -import org.apache.beam.sdk.schemas.logicaltypes.EnumerationType; -import org.apache.beam.sdk.schemas.logicaltypes.FixedBytes; -import org.apache.beam.sdk.schemas.logicaltypes.FixedString; -import org.apache.beam.sdk.schemas.logicaltypes.OneOfType; -import org.apache.beam.sdk.schemas.logicaltypes.SqlTypes; -import org.apache.beam.sdk.schemas.logicaltypes.VariableBytes; -import org.apache.beam.sdk.schemas.logicaltypes.VariableString; -import org.apache.beam.sdk.schemas.utils.ByteBuddyUtils.ConvertType; -import org.apache.beam.sdk.schemas.utils.ByteBuddyUtils.ConvertValueForGetter; -import org.apache.beam.sdk.schemas.utils.ByteBuddyUtils.ConvertValueForSetter; -import org.apache.beam.sdk.schemas.utils.ByteBuddyUtils.TypeConversion; -import org.apache.beam.sdk.schemas.utils.ByteBuddyUtils.TypeConversionsFactory; -import org.apache.beam.sdk.transforms.SerializableFunction; -import org.apache.beam.sdk.transforms.SimpleFunction; -import org.apache.beam.sdk.values.Row; -import org.apache.beam.sdk.values.TypeDescriptor; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.CaseFormat; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Strings; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps; -import org.checkerframework.checker.nullness.qual.Nullable; -import org.joda.time.Days; -import org.joda.time.Duration; -import org.joda.time.Instant; -import org.joda.time.ReadableInstant; - -/** - * Utils to convert AVRO records to Beam rows. Imposes a mapping between common avro types and Beam - * portable schemas (https://s.apache.org/beam-schemas): - * - * <pre> - * Avro Beam Field Type - * INT <-----> INT32 - * LONG <-----> INT64 - * FLOAT <-----> FLOAT - * DOUBLE <-----> DOUBLE - * BOOLEAN <-----> BOOLEAN - * STRING <-----> STRING - * BYTES <-----> BYTES - * <------ LogicalType(urn="beam:logical_type:var_bytes:v1") - * FIXED <-----> LogicalType(urn="beam:logical_type:fixed_bytes:v1") - * ARRAY <-----> ARRAY - * ENUM <-----> LogicalType(EnumerationType) - * MAP <-----> MAP - * RECORD <-----> ROW - * UNION <-----> LogicalType(OneOfType) - * LogicalTypes.Date <-----> LogicalType(DATE) - * <------ LogicalType(urn="beam:logical_type:date:v1") - * LogicalTypes.TimestampMillis <-----> DATETIME - * LogicalTypes.Decimal <-----> DECIMAL - * </pre> - * - * For SQL CHAR/VARCHAR types, an Avro schema - * - * <pre> - * LogicalType({"type":"string","logicalType":"char","maxLength":MAX_LENGTH}) or - * LogicalType({"type":"string","logicalType":"varchar","maxLength":MAX_LENGTH}) - * </pre> - * - * is used. - * - * @deprecated Avro related classes are deprecated in module <code>beam-sdks-java-core</code> and - * will be eventually removed. Please, migrate to a new module <code> - * beam-sdks-java-extensions-avro</code> by importing <code> - * org.apache.beam.sdk.extensions.avro.schemas.utils.AvroUtils</code> instead of this one. - */ -@SuppressWarnings({ - "nullness", // TODO(https://github.com/apache/beam/issues/20497) - "rawtypes" -}) -@Deprecated -public class AvroUtils { - static { - // This works around a bug in the Avro library (AVRO-1891) around SpecificRecord's handling - // of DateTime types. - SpecificData.get().addLogicalTypeConversion(new JodaTimestampConversion()); - GenericData.get().addLogicalTypeConversion(new JodaTimestampConversion()); - } - - /** Unwrap an AVRO schema into the base type an whether it is nullable. */ - public static class TypeWithNullability { - final org.apache.avro.Schema type; - final boolean nullable; - - public static TypeWithNullability create(org.apache.avro.Schema avroSchema) { - return new TypeWithNullability(avroSchema); - } - - TypeWithNullability(org.apache.avro.Schema avroSchema) { - if (avroSchema.getType() == org.apache.avro.Schema.Type.UNION) { - List<org.apache.avro.Schema> types = avroSchema.getTypes(); - - // optional fields in AVRO have form of: - // {"name": "foo", "type": ["null", "something"]} - - // don't need recursion because nested unions aren't supported in AVRO - List<org.apache.avro.Schema> nonNullTypes = - types.stream() - .filter(x -> x.getType() != org.apache.avro.Schema.Type.NULL) - .collect(Collectors.toList()); - - if (nonNullTypes.size() == types.size() || nonNullTypes.isEmpty()) { - // union without `null` or all 'null' union, keep as is. - type = avroSchema; - nullable = false; - } else if (nonNullTypes.size() > 1) { - type = org.apache.avro.Schema.createUnion(nonNullTypes); - nullable = true; - } else { - // One non-null type. - type = nonNullTypes.get(0); - nullable = true; - } - } else { - type = avroSchema; - nullable = false; - } - } - - public Boolean isNullable() { - return nullable; - } - - public org.apache.avro.Schema getType() { - return type; - } - } - - /** Wrapper for fixed byte fields. */ - public static class FixedBytesField { - private final int size; - - private FixedBytesField(int size) { - this.size = size; - } - - /** Create a {@link FixedBytesField} with the specified size. */ - public static FixedBytesField withSize(int size) { - return new FixedBytesField(size); - } - - /** Create a {@link FixedBytesField} from a Beam {@link FieldType}. */ - public static @Nullable FixedBytesField fromBeamFieldType(FieldType fieldType) { - if (fieldType.getTypeName().isLogicalType() - && fieldType.getLogicalType().getIdentifier().equals(FixedBytes.IDENTIFIER)) { - int length = fieldType.getLogicalType(FixedBytes.class).getLength(); - return new FixedBytesField(length); - } else { - return null; - } - } - - /** Create a {@link FixedBytesField} from an AVRO type. */ - public static @Nullable FixedBytesField fromAvroType(org.apache.avro.Schema type) { - if (type.getType().equals(Type.FIXED)) { - return new FixedBytesField(type.getFixedSize()); - } else { - return null; - } - } - - /** Get the size. */ - public int getSize() { - return size; - } - - /** Convert to a Beam type. */ - public FieldType toBeamType() { - return Schema.FieldType.logicalType(FixedBytes.of(size)); - } - - /** Convert to an AVRO type. */ - public org.apache.avro.Schema toAvroType(String name, String namespace) { - return org.apache.avro.Schema.createFixed(name, null, namespace, size); - } - } - - public static class AvroConvertType extends ConvertType { - public AvroConvertType(boolean returnRawType) { - super(returnRawType); - } - - @Override - protected java.lang.reflect.Type convertDefault(TypeDescriptor<?> type) { - if (type.isSubtypeOf(TypeDescriptor.of(GenericFixed.class))) { - return byte[].class; - } else { - return super.convertDefault(type); - } - } - } - - public static class AvroConvertValueForGetter extends ConvertValueForGetter { - AvroConvertValueForGetter(StackManipulation readValue) { - super(readValue); - } - - @Override - protected TypeConversionsFactory getFactory() { - return new AvroTypeConversionFactory(); - } - - @Override - protected StackManipulation convertDefault(TypeDescriptor<?> type) { - if (type.isSubtypeOf(TypeDescriptor.of(GenericFixed.class))) { - // Generate the following code: - // return value.bytes(); - return new Compound( - readValue, - MethodInvocation.invoke( - new ForLoadedType(GenericFixed.class) - .getDeclaredMethods() - .filter( - ElementMatchers.named("bytes") - .and(ElementMatchers.returns(new ForLoadedType(byte[].class)))) - .getOnly())); - } - return super.convertDefault(type); - } - } - - public static class AvroConvertValueForSetter extends ConvertValueForSetter { - AvroConvertValueForSetter(StackManipulation readValue) { - super(readValue); - } - - @Override - protected TypeConversionsFactory getFactory() { - return new AvroTypeConversionFactory(); - } - - @Override - protected StackManipulation convertDefault(TypeDescriptor<?> type) { - final ForLoadedType byteArrayType = new ForLoadedType(byte[].class); - if (type.isSubtypeOf(TypeDescriptor.of(GenericFixed.class))) { - // Generate the following code: - // return new T((byte[]) value); - ForLoadedType loadedType = new ForLoadedType(type.getRawType()); - return new Compound( - TypeCreation.of(loadedType), - Duplication.SINGLE, - // Load the parameter and cast it to a byte[]. - readValue, - TypeCasting.to(byteArrayType), - // Create a new instance that wraps this byte[]. - MethodInvocation.invoke( - loadedType - .getDeclaredMethods() - .filter( - ElementMatchers.isConstructor() - .and(ElementMatchers.takesArguments(byteArrayType))) - .getOnly())); - } - return super.convertDefault(type); - } - } - - static class AvroTypeConversionFactory implements TypeConversionsFactory { - - @Override - public TypeConversion<java.lang.reflect.Type> createTypeConversion(boolean returnRawTypes) { - return new AvroConvertType(returnRawTypes); - } - - @Override - public TypeConversion<StackManipulation> createGetterConversions(StackManipulation readValue) { - return new AvroConvertValueForGetter(readValue); - } - - @Override - public TypeConversion<StackManipulation> createSetterConversions(StackManipulation readValue) { - return new AvroConvertValueForSetter(readValue); - } - } - - /** Get Beam Field from avro Field. */ - public static Schema.Field toBeamField(org.apache.avro.Schema.Field field) { - TypeWithNullability nullableType = new TypeWithNullability(field.schema()); - FieldType beamFieldType = toFieldType(nullableType); - return Field.of(field.name(), beamFieldType); - } - - /** Get Avro Field from Beam Field. */ - public static org.apache.avro.Schema.Field toAvroField(Schema.Field field, String namespace) { - org.apache.avro.Schema fieldSchema = - getFieldSchema(field.getType(), field.getName(), namespace); - return new org.apache.avro.Schema.Field( - field.getName(), fieldSchema, field.getDescription(), (Object) null); - } - - private AvroUtils() {} - - /** - * Converts AVRO schema to Beam row schema. - * - * @param schema schema of type RECORD - */ - public static Schema toBeamSchema(org.apache.avro.Schema schema) { - Schema.Builder builder = Schema.builder(); - - for (org.apache.avro.Schema.Field field : schema.getFields()) { - Field beamField = toBeamField(field); - if (field.doc() != null) { - beamField = beamField.withDescription(field.doc()); - } - builder.addField(beamField); - } - - return builder.build(); - } - - /** Converts a Beam Schema into an AVRO schema. */ - public static org.apache.avro.Schema toAvroSchema( - Schema beamSchema, @Nullable String name, @Nullable String namespace) { - final String schemaName = Strings.isNullOrEmpty(name) ? "topLevelRecord" : name; - final String schemaNamespace = namespace == null ? "" : namespace; - String childNamespace = - !"".equals(schemaNamespace) ? schemaNamespace + "." + schemaName : schemaName; - List<org.apache.avro.Schema.Field> fields = Lists.newArrayList(); - for (Schema.Field field : beamSchema.getFields()) { - org.apache.avro.Schema.Field recordField = toAvroField(field, childNamespace); - fields.add(recordField); - } - return org.apache.avro.Schema.createRecord(schemaName, null, schemaNamespace, false, fields); - } - - public static org.apache.avro.Schema toAvroSchema(Schema beamSchema) { - return toAvroSchema(beamSchema, null, null); - } - - /** - * Strict conversion from AVRO to Beam, strict because it doesn't do widening or narrowing during - * conversion. If Schema is not provided, one is inferred from the AVRO schema. - */ - public static Row toBeamRowStrict(GenericRecord record, @Nullable Schema schema) { - if (schema == null) { - schema = toBeamSchema(record.getSchema()); - } - - Row.Builder builder = Row.withSchema(schema); - org.apache.avro.Schema avroSchema = record.getSchema(); - - for (Schema.Field field : schema.getFields()) { - Object value = record.get(field.getName()); - org.apache.avro.Schema fieldAvroSchema = avroSchema.getField(field.getName()).schema(); - builder.addValue(convertAvroFieldStrict(value, fieldAvroSchema, field.getType())); - } - - return builder.build(); - } - - /** - * Convert from a Beam Row to an AVRO GenericRecord. The Avro Schema is inferred from the Beam - * schema on the row. - */ - public static GenericRecord toGenericRecord(Row row) { - return toGenericRecord(row, null); - } - - /** - * Convert from a Beam Row to an AVRO GenericRecord. If a Schema is not provided, one is inferred - * from the Beam schema on the row. - */ - public static GenericRecord toGenericRecord( - Row row, org.apache.avro.@Nullable Schema avroSchema) { - Schema beamSchema = row.getSchema(); - // Use the provided AVRO schema if present, otherwise infer an AVRO schema from the row - // schema. - if (avroSchema != null && avroSchema.getFields().size() != beamSchema.getFieldCount()) { - throw new IllegalArgumentException( - "AVRO schema doesn't match row schema. Row schema " - + beamSchema - + ". AVRO schema + " - + avroSchema); - } - if (avroSchema == null) { - avroSchema = toAvroSchema(beamSchema); - } - - GenericRecordBuilder builder = new GenericRecordBuilder(avroSchema); - for (int i = 0; i < beamSchema.getFieldCount(); ++i) { - Schema.Field field = beamSchema.getField(i); - builder.set( - field.getName(), - genericFromBeamField( - field.getType(), avroSchema.getField(field.getName()).schema(), row.getValue(i))); - } - return builder.build(); - } - - @SuppressWarnings("unchecked") - public static <T> SerializableFunction<T, Row> getToRowFunction( - Class<T> clazz, org.apache.avro.@Nullable Schema schema) { - if (GenericRecord.class.equals(clazz)) { - Schema beamSchema = toBeamSchema(schema); - return (SerializableFunction<T, Row>) getGenericRecordToRowFunction(beamSchema); - } else { - return new AvroRecordSchema().toRowFunction(TypeDescriptor.of(clazz)); - } - } - - @SuppressWarnings("unchecked") - public static <T> SerializableFunction<Row, T> getFromRowFunction(Class<T> clazz) { - return GenericRecord.class.equals(clazz) - ? (SerializableFunction<Row, T>) getRowToGenericRecordFunction(null) - : new AvroRecordSchema().fromRowFunction(TypeDescriptor.of(clazz)); - } - - public static @Nullable <T> Schema getSchema( - Class<T> clazz, org.apache.avro.@Nullable Schema schema) { - if (schema != null) { - return schema.getType().equals(Type.RECORD) ? toBeamSchema(schema) : null; - } - if (GenericRecord.class.equals(clazz)) { - throw new IllegalArgumentException("No schema provided for getSchema(GenericRecord)"); - } - return new AvroRecordSchema().schemaFor(TypeDescriptor.of(clazz)); - } - - /** Returns a function mapping encoded AVRO {@link GenericRecord}s to Beam {@link Row}s. */ - public static SimpleFunction<byte[], Row> getAvroBytesToRowFunction(Schema beamSchema) { - return new AvroBytesToRowFn(beamSchema); - } - - private static class AvroBytesToRowFn extends SimpleFunction<byte[], Row> { - private final AvroCoder<GenericRecord> coder; - private final Schema beamSchema; - - AvroBytesToRowFn(Schema beamSchema) { - org.apache.avro.Schema avroSchema = toAvroSchema(beamSchema); - coder = AvroCoder.of(avroSchema); - this.beamSchema = beamSchema; - } - - @Override - public Row apply(byte[] bytes) { - try { - ByteArrayInputStream inputStream = new ByteArrayInputStream(bytes); - GenericRecord record = coder.decode(inputStream); - return AvroUtils.toBeamRowStrict(record, beamSchema); - } catch (Exception e) { - throw new AvroRuntimeException( - "Could not decode avro record from given bytes " - + new String(bytes, StandardCharsets.UTF_8), - e); - } - } - } - - /** Returns a function mapping Beam {@link Row}s to encoded AVRO {@link GenericRecord}s. */ - public static SimpleFunction<Row, byte[]> getRowToAvroBytesFunction(Schema beamSchema) { - return new RowToAvroBytesFn(beamSchema); - } - - private static class RowToAvroBytesFn extends SimpleFunction<Row, byte[]> { - private final transient org.apache.avro.Schema avroSchema; - private final AvroCoder<GenericRecord> coder; - - RowToAvroBytesFn(Schema beamSchema) { - avroSchema = toAvroSchema(beamSchema); - coder = AvroCoder.of(avroSchema); - } - - @Override - public byte[] apply(Row row) { - try { - GenericRecord record = toGenericRecord(row, avroSchema); - ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); - coder.encode(record, outputStream); - return outputStream.toByteArray(); - } catch (Exception e) { - throw new AvroRuntimeException( - String.format("Could not encode avro from given row: %s", row), e); - } - } - } - - /** - * Returns a function mapping AVRO {@link GenericRecord}s to Beam {@link Row}s for use in {@link - * org.apache.beam.sdk.values.PCollection#setSchema}. - */ - public static SerializableFunction<GenericRecord, Row> getGenericRecordToRowFunction( - @Nullable Schema schema) { - return new GenericRecordToRowFn(schema); - } - - private static class GenericRecordToRowFn implements SerializableFunction<GenericRecord, Row> { - private final Schema schema; - - GenericRecordToRowFn(Schema schema) { - this.schema = schema; - } - - @Override - public Row apply(GenericRecord input) { - return toBeamRowStrict(input, schema); - } - - @Override - public boolean equals(@Nullable Object other) { - if (this == other) { - return true; - } - if (other == null || getClass() != other.getClass()) { - return false; - } - GenericRecordToRowFn that = (GenericRecordToRowFn) other; - return Objects.equals(this.schema, that.schema); - } - - @Override - public int hashCode() { - return Objects.hash(schema); - } - } - - /** - * Returns a function mapping Beam {@link Row}s to AVRO {@link GenericRecord}s for use in {@link - * org.apache.beam.sdk.values.PCollection#setSchema}. - */ - public static SerializableFunction<Row, GenericRecord> getRowToGenericRecordFunction( - org.apache.avro.@Nullable Schema avroSchema) { - return new RowToGenericRecordFn(avroSchema); - } - - private static class RowToGenericRecordFn implements SerializableFunction<Row, GenericRecord> { - private transient org.apache.avro.Schema avroSchema; - - RowToGenericRecordFn(org.apache.avro.@Nullable Schema avroSchema) { - this.avroSchema = avroSchema; - } - - @Override - public GenericRecord apply(Row input) { - return toGenericRecord(input, avroSchema); - } - - @Override - public boolean equals(@Nullable Object other) { - if (this == other) { - return true; - } - if (other == null || getClass() != other.getClass()) { - return false; - } - RowToGenericRecordFn that = (RowToGenericRecordFn) other; - return Objects.equals(this.avroSchema, that.avroSchema); - } - - @Override - public int hashCode() { - return Objects.hash(avroSchema); - } - - private void writeObject(ObjectOutputStream out) throws IOException { - final String avroSchemaAsString = (avroSchema == null) ? null : avroSchema.toString(); - out.writeObject(avroSchemaAsString); - } - - private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { - final String avroSchemaAsString = (String) in.readObject(); - avroSchema = - (avroSchemaAsString == null) - ? null - : new org.apache.avro.Schema.Parser().parse(avroSchemaAsString); - } - } - - /** - * Returns an {@code SchemaCoder} instance for the provided element type. - * - * @param <T> the element type - */ - public static <T> SchemaCoder<T> schemaCoder(TypeDescriptor<T> type) { - @SuppressWarnings("unchecked") - Class<T> clazz = (Class<T>) type.getRawType(); - org.apache.avro.Schema avroSchema = new ReflectData(clazz.getClassLoader()).getSchema(clazz); - Schema beamSchema = toBeamSchema(avroSchema); - return SchemaCoder.of( - beamSchema, type, getToRowFunction(clazz, avroSchema), getFromRowFunction(clazz)); - } - - /** - * Returns an {@code SchemaCoder} instance for the provided element class. - * - * @param <T> the element type - */ - public static <T> SchemaCoder<T> schemaCoder(Class<T> clazz) { - return schemaCoder(TypeDescriptor.of(clazz)); - } - - /** - * Returns an {@code SchemaCoder} instance for the Avro schema. The implicit type is - * GenericRecord. - */ - public static SchemaCoder<GenericRecord> schemaCoder(org.apache.avro.Schema schema) { - Schema beamSchema = toBeamSchema(schema); - return SchemaCoder.of( - beamSchema, - TypeDescriptor.of(GenericRecord.class), - getGenericRecordToRowFunction(beamSchema), - getRowToGenericRecordFunction(schema)); - } - - /** - * Returns an {@code SchemaCoder} instance for the provided element type using the provided Avro - * schema. - * - * <p>If the type argument is GenericRecord, the schema may be arbitrary. Otherwise, the schema - * must correspond to the type provided. - * - * @param <T> the element type - */ - public static <T> SchemaCoder<T> schemaCoder(Class<T> clazz, org.apache.avro.Schema schema) { - return SchemaCoder.of( - getSchema(clazz, schema), - TypeDescriptor.of(clazz), - getToRowFunction(clazz, schema), - getFromRowFunction(clazz)); - } - - /** - * Returns an {@code SchemaCoder} instance based on the provided AvroCoder for the element type. - * - * @param <T> the element type - */ - public static <T> SchemaCoder<T> schemaCoder(AvroCoder<T> avroCoder) { - return schemaCoder(avroCoder.getType(), avroCoder.getSchema()); - } - - private static final class AvroSpecificRecordFieldValueTypeSupplier - implements FieldValueTypeSupplier { - @Override - public List<FieldValueTypeInformation> get(Class<?> clazz) { - throw new RuntimeException("Unexpected call."); - } - - @Override - public List<FieldValueTypeInformation> get(Class<?> clazz, Schema schema) { - Map<String, String> mapping = getMapping(schema); - List<Method> methods = ReflectUtils.getMethods(clazz); - List<FieldValueTypeInformation> types = Lists.newArrayList(); - for (int i = 0; i < methods.size(); ++i) { - Method method = methods.get(i); - if (ReflectUtils.isGetter(method)) { - FieldValueTypeInformation fieldValueTypeInformation = - FieldValueTypeInformation.forGetter(method, i); - String name = mapping.get(fieldValueTypeInformation.getName()); - if (name != null) { - types.add(fieldValueTypeInformation.withName(name)); - } - } - } - - // Return the list ordered by the schema fields. - return StaticSchemaInference.sortBySchema(types, schema); - } - - private Map<String, String> getMapping(Schema schema) { - Map<String, String> mapping = Maps.newHashMap(); - for (Field field : schema.getFields()) { - String fieldName = field.getName(); - String getter; - if (fieldName.contains("_")) { - if (Character.isLowerCase(fieldName.charAt(0))) { - // field_name -> fieldName - getter = CaseFormat.LOWER_UNDERSCORE.to(CaseFormat.LOWER_CAMEL, fieldName); - } else { - // FIELD_NAME -> fIELDNAME - // must remove underscore and then convert to match compiled Avro schema getter name - getter = CaseFormat.UPPER_CAMEL.to(CaseFormat.LOWER_CAMEL, fieldName.replace("_", "")); - } - } else if (Character.isUpperCase(fieldName.charAt(0))) { - // FieldName -> fieldName - getter = CaseFormat.UPPER_CAMEL.to(CaseFormat.LOWER_CAMEL, fieldName); - } else { - // If the field is in camel case already, then it's the identity mapping. - getter = fieldName; - } - mapping.put(getter, fieldName); - // The Avro compiler might add a $ at the end of a getter to disambiguate. - mapping.put(getter + "$", fieldName); - } - return mapping; - } - } - - private static final class AvroPojoFieldValueTypeSupplier implements FieldValueTypeSupplier { - @Override - public List<FieldValueTypeInformation> get(Class<?> clazz) { - List<java.lang.reflect.Field> classFields = ReflectUtils.getFields(clazz); - Map<String, FieldValueTypeInformation> types = Maps.newHashMap(); - for (int i = 0; i < classFields.size(); ++i) { - java.lang.reflect.Field f = classFields.get(i); - if (!f.isAnnotationPresent(AvroIgnore.class)) { - FieldValueTypeInformation typeInformation = FieldValueTypeInformation.forField(f, i); - AvroName avroname = f.getAnnotation(AvroName.class); - if (avroname != null) { - typeInformation = typeInformation.withName(avroname.value()); - } - types.put(typeInformation.getName(), typeInformation); - } - } - return Lists.newArrayList(types.values()); - } - } - - /** Get field types for an AVRO-generated SpecificRecord or a POJO. */ - public static <T> List<FieldValueTypeInformation> getFieldTypes(Class<T> clazz, Schema schema) { - if (TypeDescriptor.of(clazz).isSubtypeOf(TypeDescriptor.of(SpecificRecord.class))) { - return JavaBeanUtils.getFieldTypes( - clazz, schema, new AvroSpecificRecordFieldValueTypeSupplier()); - } else { - return POJOUtils.getFieldTypes(clazz, schema, new AvroPojoFieldValueTypeSupplier()); - } - } - - /** Get generated getters for an AVRO-generated SpecificRecord or a POJO. */ - public static <T> List<FieldValueGetter> getGetters(Class<T> clazz, Schema schema) { - if (TypeDescriptor.of(clazz).isSubtypeOf(TypeDescriptor.of(SpecificRecord.class))) { - return JavaBeanUtils.getGetters( - clazz, - schema, - new AvroSpecificRecordFieldValueTypeSupplier(), - new AvroTypeConversionFactory()); - } else { - return POJOUtils.getGetters( - clazz, schema, new AvroPojoFieldValueTypeSupplier(), new AvroTypeConversionFactory()); - } - } - - /** Get an object creator for an AVRO-generated SpecificRecord. */ - public static <T> SchemaUserTypeCreator getCreator(Class<T> clazz, Schema schema) { - if (TypeDescriptor.of(clazz).isSubtypeOf(TypeDescriptor.of(SpecificRecord.class))) { - return AvroByteBuddyUtils.getCreator((Class<? extends SpecificRecord>) clazz, schema); - } else { - return POJOUtils.getSetFieldCreator( - clazz, schema, new AvroPojoFieldValueTypeSupplier(), new AvroTypeConversionFactory()); - } - } - - /** Converts AVRO schema to Beam field. */ - private static Schema.FieldType toFieldType(TypeWithNullability type) { - Schema.FieldType fieldType = null; - org.apache.avro.Schema avroSchema = type.type; - - LogicalType logicalType = LogicalTypes.fromSchema(avroSchema); - if (logicalType != null) { - if (logicalType instanceof LogicalTypes.Decimal) { - fieldType = FieldType.DECIMAL; - } else if (logicalType instanceof LogicalTypes.TimestampMillis) { - // TODO: There is a desire to move Beam schema DATETIME to a micros representation. When - // this is done, this logical type needs to be changed. - fieldType = FieldType.DATETIME; - } else if (logicalType instanceof LogicalTypes.Date) { - fieldType = FieldType.DATETIME; - } - } - - if (fieldType == null) { - switch (type.type.getType()) { - case RECORD: - fieldType = Schema.FieldType.row(toBeamSchema(avroSchema)); - break; - - case ENUM: - fieldType = FieldType.logicalType(EnumerationType.create(type.type.getEnumSymbols())); - break; - - case ARRAY: - Schema.FieldType elementType = - toFieldType(new TypeWithNullability(avroSchema.getElementType())); - fieldType = Schema.FieldType.array(elementType); - break; - - case MAP: - fieldType = - Schema.FieldType.map( - Schema.FieldType.STRING, - toFieldType(new TypeWithNullability(avroSchema.getValueType()))); - break; - - case FIXED: - fieldType = FixedBytesField.fromAvroType(type.type).toBeamType(); - break; - - case STRING: - fieldType = Schema.FieldType.STRING; - break; - - case BYTES: - fieldType = Schema.FieldType.BYTES; - break; - - case INT: - fieldType = Schema.FieldType.INT32; - break; - - case LONG: - fieldType = Schema.FieldType.INT64; - break; - - case FLOAT: - fieldType = Schema.FieldType.FLOAT; - break; - - case DOUBLE: - fieldType = Schema.FieldType.DOUBLE; - break; - - case BOOLEAN: - fieldType = Schema.FieldType.BOOLEAN; - break; - - case UNION: - fieldType = - FieldType.logicalType( - OneOfType.create( - avroSchema.getTypes().stream() - .map(x -> Field.of(x.getName(), toFieldType(new TypeWithNullability(x)))) - .collect(Collectors.toList()))); - break; - case NULL: - throw new IllegalArgumentException("Can't convert 'null' to FieldType"); - - default: - throw new AssertionError("Unexpected AVRO Schema.Type: " + avroSchema.getType()); - } - } - fieldType = fieldType.withNullable(type.nullable); - return fieldType; - } - - private static org.apache.avro.Schema getFieldSchema( - Schema.FieldType fieldType, String fieldName, String namespace) { - org.apache.avro.Schema baseType; - switch (fieldType.getTypeName()) { - case BYTE: - case INT16: - case INT32: - baseType = org.apache.avro.Schema.create(Type.INT); - break; - - case INT64: - baseType = org.apache.avro.Schema.create(Type.LONG); - break; - - case DECIMAL: - baseType = - LogicalTypes.decimal(Integer.MAX_VALUE) - .addToSchema(org.apache.avro.Schema.create(Type.BYTES)); - break; - - case FLOAT: - baseType = org.apache.avro.Schema.create(Type.FLOAT); - break; - - case DOUBLE: - baseType = org.apache.avro.Schema.create(Type.DOUBLE); - break; - - case STRING: - baseType = org.apache.avro.Schema.create(Type.STRING); - break; - - case DATETIME: - // TODO: There is a desire to move Beam schema DATETIME to a micros representation. When - // this is done, this logical type needs to be changed. - baseType = - LogicalTypes.timestampMillis().addToSchema(org.apache.avro.Schema.create(Type.LONG)); - break; - - case BOOLEAN: - baseType = org.apache.avro.Schema.create(Type.BOOLEAN); - break; - - case BYTES: - baseType = org.apache.avro.Schema.create(Type.BYTES); - break; - - case LOGICAL_TYPE: - String identifier = fieldType.getLogicalType().getIdentifier(); - if (FixedBytes.IDENTIFIER.equals(identifier)) { - FixedBytesField fixedBytesField = - checkNotNull(FixedBytesField.fromBeamFieldType(fieldType)); - baseType = fixedBytesField.toAvroType("fixed", namespace + "." + fieldName); - } else if (VariableBytes.IDENTIFIER.equals(identifier)) { - // treat VARBINARY as bytes as that is what avro supports - baseType = org.apache.avro.Schema.create(Type.BYTES); - } else if (FixedString.IDENTIFIER.equals(identifier) - || "CHAR".equals(identifier) - || "NCHAR".equals(identifier)) { - baseType = - buildHiveLogicalTypeSchema("char", (int) fieldType.getLogicalType().getArgument()); - } else if (VariableString.IDENTIFIER.equals(identifier) - || "NVARCHAR".equals(identifier) - || "VARCHAR".equals(identifier) - || "LONGNVARCHAR".equals(identifier) - || "LONGVARCHAR".equals(identifier)) { - baseType = - buildHiveLogicalTypeSchema("varchar", (int) fieldType.getLogicalType().getArgument()); - } else if (EnumerationType.IDENTIFIER.equals(identifier)) { - EnumerationType enumerationType = fieldType.getLogicalType(EnumerationType.class); - baseType = - org.apache.avro.Schema.createEnum(fieldName, "", "", enumerationType.getValues()); - } else if (OneOfType.IDENTIFIER.equals(identifier)) { - OneOfType oneOfType = fieldType.getLogicalType(OneOfType.class); - baseType = - org.apache.avro.Schema.createUnion( - oneOfType.getOneOfSchema().getFields().stream() - .map(x -> getFieldSchema(x.getType(), x.getName(), namespace)) - .collect(Collectors.toList())); - } else if ("DATE".equals(identifier) || SqlTypes.DATE.getIdentifier().equals(identifier)) { - baseType = LogicalTypes.date().addToSchema(org.apache.avro.Schema.create(Type.INT)); - } else if ("TIME".equals(identifier)) { - baseType = LogicalTypes.timeMillis().addToSchema(org.apache.avro.Schema.create(Type.INT)); - } else { - throw new RuntimeException( - "Unhandled logical type " + fieldType.getLogicalType().getIdentifier()); - } - break; - - case ARRAY: - case ITERABLE: - baseType = - org.apache.avro.Schema.createArray( - getFieldSchema(fieldType.getCollectionElementType(), fieldName, namespace)); - break; - - case MAP: - if (fieldType.getMapKeyType().getTypeName().isStringType()) { - // Avro only supports string keys in maps. - baseType = - org.apache.avro.Schema.createMap( - getFieldSchema(fieldType.getMapValueType(), fieldName, namespace)); - } else { - throw new IllegalArgumentException("Avro only supports maps with string keys"); - } - break; - - case ROW: - baseType = toAvroSchema(fieldType.getRowSchema(), fieldName, namespace); - break; - - default: - throw new IllegalArgumentException("Unexpected type " + fieldType); - } - return fieldType.getNullable() ? ReflectData.makeNullable(baseType) : baseType; - } - - private static @Nullable Object genericFromBeamField( - Schema.FieldType fieldType, org.apache.avro.Schema avroSchema, @Nullable Object value) { - TypeWithNullability typeWithNullability = new TypeWithNullability(avroSchema); - if (!fieldType.getNullable().equals(typeWithNullability.nullable)) { - throw new IllegalArgumentException( - "FieldType " - + fieldType - + " and AVRO schema " - + avroSchema - + " don't have matching nullability"); - } - - if (value == null) { - return value; - } - - switch (fieldType.getTypeName()) { - case BYTE: - case INT16: - case INT32: - case INT64: - case FLOAT: - case DOUBLE: - case BOOLEAN: - return value; - - case STRING: - return new Utf8((String) value); - - case DECIMAL: - BigDecimal decimal = (BigDecimal) value; - LogicalType logicalType = typeWithNullability.type.getLogicalType(); - return new Conversions.DecimalConversion().toBytes(decimal, null, logicalType); - - case DATETIME: - if (typeWithNullability.type.getType() == Type.INT) { - ReadableInstant instant = (ReadableInstant) value; - return (int) Days.daysBetween(Instant.EPOCH, instant).getDays(); - } else if (typeWithNullability.type.getType() == Type.LONG) { - ReadableInstant instant = (ReadableInstant) value; - return (long) instant.getMillis(); - } else { - throw new IllegalArgumentException( - "Can't represent " + fieldType + " as " + typeWithNullability.type.getType()); - } - - case BYTES: - return ByteBuffer.wrap((byte[]) value); - - case LOGICAL_TYPE: - String identifier = fieldType.getLogicalType().getIdentifier(); - if (FixedBytes.IDENTIFIER.equals(identifier)) { - FixedBytesField fixedBytesField = - checkNotNull(FixedBytesField.fromBeamFieldType(fieldType)); - byte[] byteArray = (byte[]) value; - if (byteArray.length != fixedBytesField.getSize()) { - throw new IllegalArgumentException("Incorrectly sized byte array."); - } - return GenericData.get().createFixed(null, (byte[]) value, typeWithNullability.type); - } else if (VariableBytes.IDENTIFIER.equals(identifier)) { - return GenericData.get().createFixed(null, (byte[]) value, typeWithNullability.type); - } else if (FixedString.IDENTIFIER.equals(identifier) - || "CHAR".equals(identifier) - || "NCHAR".equals(identifier)) { - return new Utf8((String) value); - } else if (VariableString.IDENTIFIER.equals(identifier) - || "NVARCHAR".equals(identifier) - || "VARCHAR".equals(identifier) - || "LONGNVARCHAR".equals(identifier) - || "LONGVARCHAR".equals(identifier)) { - return new Utf8((String) value); - } else if (EnumerationType.IDENTIFIER.equals(identifier)) { - EnumerationType enumerationType = fieldType.getLogicalType(EnumerationType.class); - return GenericData.get() - .createEnum( - enumerationType.toString((EnumerationType.Value) value), - typeWithNullability.type); - } else if (OneOfType.IDENTIFIER.equals(identifier)) { - OneOfType oneOfType = fieldType.getLogicalType(OneOfType.class); - OneOfType.Value oneOfValue = (OneOfType.Value) value; - FieldType innerFieldType = oneOfType.getFieldType(oneOfValue); - if (typeWithNullability.nullable && oneOfValue.getValue() == null) { - return null; - } else { - return genericFromBeamField( - innerFieldType.withNullable(false), - typeWithNullability.type.getTypes().get(oneOfValue.getCaseType().getValue()), - oneOfValue.getValue()); - } - } else if ("DATE".equals(identifier)) { - // "Date" is backed by joda.time.Instant - return Days.daysBetween(Instant.EPOCH, (Instant) value).getDays(); - } else if (SqlTypes.DATE.getIdentifier().equals(identifier)) { - // portable SqlTypes.DATE is backed by java.time.LocalDate - return ((java.time.LocalDate) value).toEpochDay(); - } else if ("TIME".equals(identifier)) { - return (int) ((Instant) value).getMillis(); - } else { - throw new RuntimeException("Unhandled logical type " + identifier); - } - - case ARRAY: - case ITERABLE: - Iterable iterable = (Iterable) value; - List<Object> translatedArray = Lists.newArrayListWithExpectedSize(Iterables.size(iterable)); - - for (Object arrayElement : iterable) { - translatedArray.add( - genericFromBeamField( - fieldType.getCollectionElementType(), - typeWithNullability.type.getElementType(), - arrayElement)); - } - return translatedArray; - - case MAP: - Map map = Maps.newHashMap(); - Map<Object, Object> valueMap = (Map<Object, Object>) value; - for (Map.Entry entry : valueMap.entrySet()) { - Utf8 key = new Utf8((String) entry.getKey()); - map.put( - key, - genericFromBeamField( - fieldType.getMapValueType(), - typeWithNullability.type.getValueType(), - entry.getValue())); - } - return map; - - case ROW: - return toGenericRecord((Row) value, typeWithNullability.type); - - default: - throw new IllegalArgumentException("Unsupported type " + fieldType); - } - } - - /** - * Strict conversion from AVRO to Beam, strict because it doesn't do widening or narrowing during - * conversion. - * - * @param value {@link GenericRecord} or any nested value - * @param avroSchema schema for value - * @param fieldType target beam field type - * @return value converted for {@link Row} - */ - @SuppressWarnings("unchecked") - public static @Nullable Object convertAvroFieldStrict( - @Nullable Object value, - @Nonnull org.apache.avro.Schema avroSchema, - @Nonnull Schema.FieldType fieldType) { - if (value == null) { - return null; - } - - TypeWithNullability type = new TypeWithNullability(avroSchema); - LogicalType logicalType = LogicalTypes.fromSchema(type.type); - if (logicalType != null) { - if (logicalType instanceof LogicalTypes.Decimal) { - ByteBuffer byteBuffer = (ByteBuffer) value; - BigDecimal bigDecimal = - new Conversions.DecimalConversion() - .fromBytes(byteBuffer.duplicate(), type.type, logicalType); - return convertDecimal(bigDecimal, fieldType); - } else if (logicalType instanceof LogicalTypes.TimestampMillis) { - if (value instanceof ReadableInstant) { - return convertDateTimeStrict(((ReadableInstant) value).getMillis(), fieldType); - } else { - return convertDateTimeStrict((Long) value, fieldType); - } - } else if (logicalType instanceof LogicalTypes.Date) { - if (value instanceof ReadableInstant) { - int epochDays = Days.daysBetween(Instant.EPOCH, (ReadableInstant) value).getDays(); - return convertDateStrict(epochDays, fieldType); - } else if (value instanceof java.time.LocalDate) { - return convertDateStrict((int) ((java.time.LocalDate) value).toEpochDay(), fieldType); - } else { - return convertDateStrict((Integer) value, fieldType); - } - } - } - - switch (type.type.getType()) { - case FIXED: - return convertFixedStrict((GenericFixed) value, fieldType); - - case BYTES: - return convertBytesStrict((ByteBuffer) value, fieldType); - - case STRING: - return convertStringStrict((CharSequence) value, fieldType); - - case INT: - return convertIntStrict((Integer) value, fieldType); - - case LONG: - return convertLongStrict((Long) value, fieldType); - - case FLOAT: - return convertFloatStrict((Float) value, fieldType); - - case DOUBLE: - return convertDoubleStrict((Double) value, fieldType); - - case BOOLEAN: - return convertBooleanStrict((Boolean) value, fieldType); - - case RECORD: - return convertRecordStrict((GenericRecord) value, fieldType); - - case ENUM: - // enums are either Java enums, or GenericEnumSymbol, - // they don't share common interface, but override toString() - return convertEnumStrict(value, fieldType); - - case ARRAY: - return convertArrayStrict((List<Object>) value, type.type.getElementType(), fieldType); - - case MAP: - return convertMapStrict( - (Map<CharSequence, Object>) value, type.type.getValueType(), fieldType); - - case UNION: - return convertUnionStrict(value, type.type, fieldType); - - case NULL: - throw new IllegalArgumentException("Can't convert 'null' to non-nullable field"); - - default: - throw new AssertionError("Unexpected AVRO Schema.Type: " + type.type.getType()); - } - } - - private static Object convertRecordStrict(GenericRecord record, Schema.FieldType fieldType) { - checkTypeName(fieldType.getTypeName(), Schema.TypeName.ROW, "record"); - return toBeamRowStrict(record, fieldType.getRowSchema()); - } - - private static Object convertBytesStrict(ByteBuffer bb, Schema.FieldType fieldType) { - checkTypeName(fieldType.getTypeName(), Schema.TypeName.BYTES, "bytes"); - - byte[] bytes = new byte[bb.remaining()]; - bb.duplicate().get(bytes); - return bytes; - } - - private static Object convertFixedStrict(GenericFixed fixed, Schema.FieldType fieldType) { - checkTypeName(fieldType.getTypeName(), TypeName.LOGICAL_TYPE, "fixed"); - checkArgument(FixedBytes.IDENTIFIER.equals(fieldType.getLogicalType().getIdentifier())); - return fixed.bytes().clone(); // clone because GenericFixed is mutable - } - - private static Object convertStringStrict(CharSequence value, Schema.FieldType fieldType) { - checkTypeName(fieldType.getTypeName(), Schema.TypeName.STRING, "string"); - return value.toString(); - } - - private static Object convertIntStrict(Integer value, Schema.FieldType fieldType) { - checkTypeName(fieldType.getTypeName(), Schema.TypeName.INT32, "int"); - return value; - } - - private static Object convertLongStrict(Long value, Schema.FieldType fieldType) { - checkTypeName(fieldType.getTypeName(), Schema.TypeName.INT64, "long"); - return value; - } - - private static Object convertDecimal(BigDecimal value, Schema.FieldType fieldType) { - checkTypeName(fieldType.getTypeName(), TypeName.DECIMAL, "decimal"); - return value; - } - - private static Object convertDateStrict(Integer epochDays, Schema.FieldType fieldType) { - checkTypeName(fieldType.getTypeName(), TypeName.DATETIME, "date"); - return Instant.EPOCH.plus(Duration.standardDays(epochDays)); - } - - private static Object convertDateTimeStrict(Long value, Schema.FieldType fieldType) { - checkTypeName(fieldType.getTypeName(), TypeName.DATETIME, "dateTime"); - return new Instant(value); - } - - private static Object convertFloatStrict(Float value, Schema.FieldType fieldType) { - checkTypeName(fieldType.getTypeName(), Schema.TypeName.FLOAT, "float"); - return value; - } - - private static Object convertDoubleStrict(Double value, Schema.FieldType fieldType) { - checkTypeName(fieldType.getTypeName(), Schema.TypeName.DOUBLE, "double"); - return value; - } - - private static Object convertBooleanStrict(Boolean value, Schema.FieldType fieldType) { - checkTypeName(fieldType.getTypeName(), Schema.TypeName.BOOLEAN, "boolean"); - return value; - } - - private static Object convertEnumStrict(Object value, Schema.FieldType fieldType) { - checkTypeName(fieldType.getTypeName(), TypeName.LOGICAL_TYPE, "enum"); - checkArgument(fieldType.getLogicalType().getIdentifier().equals(EnumerationType.IDENTIFIER)); - EnumerationType enumerationType = fieldType.getLogicalType(EnumerationType.class); - return enumerationType.valueOf(value.toString()); - } - - private static Object convertUnionStrict( - Object value, org.apache.avro.Schema unionAvroSchema, Schema.FieldType fieldType) { - checkTypeName(fieldType.getTypeName(), TypeName.LOGICAL_TYPE, "oneOfType"); - checkArgument(fieldType.getLogicalType().getIdentifier().equals(OneOfType.IDENTIFIER)); - OneOfType oneOfType = fieldType.getLogicalType(OneOfType.class); - int fieldNumber = GenericData.get().resolveUnion(unionAvroSchema, value); - FieldType baseFieldType = oneOfType.getOneOfSchema().getField(fieldNumber).getType(); - Object convertedValue = - convertAvroFieldStrict(value, unionAvroSchema.getTypes().get(fieldNumber), baseFieldType); - return oneOfType.createValue(fieldNumber, convertedValue); - } - - private static Object convertArrayStrict( - List<Object> values, org.apache.avro.Schema elemAvroSchema, Schema.FieldType fieldType) { - checkTypeName(fieldType.getTypeName(), Schema.TypeName.ARRAY, "array"); - - List<Object> ret = new ArrayList<>(values.size()); - Schema.FieldType elemFieldType = fieldType.getCollectionElementType(); - - for (Object value : values) { - ret.add(convertAvroFieldStrict(value, elemAvroSchema, elemFieldType)); - } - - return ret; - } - - private static Object convertMapStrict( - Map<CharSequence, Object> values, - org.apache.avro.Schema valueAvroSchema, - Schema.FieldType fieldType) { - checkTypeName(fieldType.getTypeName(), Schema.TypeName.MAP, "map"); - checkNotNull(fieldType.getMapKeyType()); - checkNotNull(fieldType.getMapValueType()); - - if (!fieldType.getMapKeyType().equals(Schema.FieldType.STRING)) { - throw new IllegalArgumentException( - "Can't convert 'string' map keys to " + fieldType.getMapKeyType()); - } - - Map<Object, Object> ret = new HashMap<>(); - - for (Map.Entry<CharSequence, Object> value : values.entrySet()) { - ret.put( - convertStringStrict(value.getKey(), fieldType.getMapKeyType()), - convertAvroFieldStrict(value.getValue(), valueAvroSchema, fieldType.getMapValueType())); - } - - return ret; - } - - private static void checkTypeName(Schema.TypeName got, Schema.TypeName expected, String label) { - checkArgument( - got.equals(expected), "Can't convert '%s' to %s, expected: %s", label, got, expected); - } - - /** - * Helper factory to build Avro Logical types schemas for SQL *CHAR types. This method <a - * href="https://github.com/apache/hive/blob/5d268834a5f5278ea76399f8af0d0ab043ae0b45/serde/src/java/org/apache/hadoop/hive/serde2/avro/TypeInfoToSchema.java#L110-L121">represents - * the logical as Hive does</a>. - */ - private static org.apache.avro.Schema buildHiveLogicalTypeSchema( - String hiveLogicalType, int size) { - String schemaJson = - String.format( - "{\"type\": \"string\", \"logicalType\": \"%s\", \"maxLength\": %s}", - hiveLogicalType, size); - return new org.apache.avro.Schema.Parser().parse(schemaJson); - } -} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/AvroCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/AvroCoderTest.java deleted file mode 100644 index 2508a4d36361..000000000000 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/AvroCoderTest.java +++ /dev/null @@ -1,1106 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.coders; - -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.containsString; -import static org.hamcrest.Matchers.equalTo; -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - -import com.esotericsoftware.kryo.Kryo; -import com.esotericsoftware.kryo.io.Input; -import com.esotericsoftware.kryo.io.Output; -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.Collection; -import java.util.HashSet; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.SortedMap; -import java.util.SortedSet; -import java.util.TreeMap; -import java.util.TreeSet; -import org.apache.avro.AvroRuntimeException; -import org.apache.avro.Schema; -import org.apache.avro.SchemaBuilder; -import org.apache.avro.generic.GenericData; -import org.apache.avro.generic.GenericRecord; -import org.apache.avro.reflect.AvroName; -import org.apache.avro.reflect.AvroSchema; -import org.apache.avro.reflect.ReflectData; -import org.apache.avro.reflect.Stringable; -import org.apache.avro.reflect.Union; -import org.apache.avro.specific.SpecificData; -import org.apache.avro.specific.SpecificRecord; -import org.apache.avro.util.Utf8; -import org.apache.beam.sdk.coders.Coder.Context; -import org.apache.beam.sdk.coders.Coder.NonDeterministicException; -import org.apache.beam.sdk.schemas.TestAvro; -import org.apache.beam.sdk.schemas.TestAvroNested; -import org.apache.beam.sdk.schemas.TestEnum; -import org.apache.beam.sdk.schemas.fixed4; -import org.apache.beam.sdk.testing.CoderProperties; -import org.apache.beam.sdk.testing.InterceptingUrlClassLoader; -import org.apache.beam.sdk.testing.NeedsRunner; -import org.apache.beam.sdk.testing.PAssert; -import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.InstanceBuilder; -import org.apache.beam.sdk.util.SerializableUtils; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.TypeDescriptor; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; -import org.checkerframework.checker.nullness.qual.Nullable; -import org.hamcrest.Description; -import org.hamcrest.Matcher; -import org.hamcrest.Matchers; -import org.hamcrest.TypeSafeMatcher; -import org.joda.time.DateTime; -import org.joda.time.DateTimeZone; -import org.joda.time.LocalDate; -import org.junit.Rule; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; -import org.objenesis.strategy.StdInstantiatorStrategy; - -/** Tests for {@link AvroCoder}. */ -@RunWith(JUnit4.class) -public class AvroCoderTest { - - public static final DateTime DATETIME_A = - new DateTime().withDate(1994, 10, 31).withZone(DateTimeZone.UTC); - public static final DateTime DATETIME_B = - new DateTime().withDate(1997, 4, 25).withZone(DateTimeZone.UTC); - private static final TestAvroNested AVRO_NESTED_SPECIFIC_RECORD = new TestAvroNested(true, 42); - private static final TestAvro AVRO_SPECIFIC_RECORD = - new TestAvro( - true, - 43, - 44L, - 44.1f, - 44.2d, - "mystring", - ByteBuffer.wrap(new byte[] {1, 2, 3, 4}), - new fixed4(new byte[] {1, 2, 3, 4}), - new LocalDate(1979, 3, 14), - new DateTime().withDate(1979, 3, 14).withTime(1, 2, 3, 4), - TestEnum.abc, - AVRO_NESTED_SPECIFIC_RECORD, - ImmutableList.of(AVRO_NESTED_SPECIFIC_RECORD, AVRO_NESTED_SPECIFIC_RECORD), - ImmutableMap.of("k1", AVRO_NESTED_SPECIFIC_RECORD, "k2", AVRO_NESTED_SPECIFIC_RECORD)); - - @DefaultCoder(AvroCoder.class) - private static class Pojo { - public String text; - public int count; - - @AvroSchema("{\"type\": \"long\", \"logicalType\": \"timestamp-millis\"}") - public DateTime timestamp; - - // Empty constructor required for Avro decoding. - @SuppressWarnings("unused") - public Pojo() {} - - public Pojo(String text, int count, DateTime timestamp) { - this.text = text; - this.count = count; - this.timestamp = timestamp; - } - - // auto-generated - @Override - public boolean equals(@Nullable Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - - Pojo pojo = (Pojo) o; - - if (count != pojo.count) { - return false; - } - if (text != null ? !text.equals(pojo.text) : pojo.text != null) { - return false; - } - if (timestamp != null ? !timestamp.equals(pojo.timestamp) : pojo.timestamp != null) { - return false; - } - - return true; - } - - @Override - public int hashCode() { - return 0; - } - - @Override - public String toString() { - return "Pojo{" - + "text='" - + text - + '\'' - + ", count=" - + count - + ", timestamp=" - + timestamp - + '}'; - } - } - - private static class GetTextFn extends DoFn<Pojo, String> { - @ProcessElement - public void processElement(ProcessContext c) { - c.output(c.element().text); - } - } - - @Rule public TestPipeline pipeline = TestPipeline.create(); - - @Test - public void testAvroCoderEncoding() throws Exception { - AvroCoder<Pojo> coder = AvroCoder.of(Pojo.class); - CoderProperties.coderSerializable(coder); - AvroCoder<Pojo> copy = SerializableUtils.clone(coder); - - Pojo pojo = new Pojo("foo", 3, DATETIME_A); - Pojo equalPojo = new Pojo("foo", 3, DATETIME_A); - Pojo otherPojo = new Pojo("bar", -19, DATETIME_B); - CoderProperties.coderConsistentWithEquals(coder, pojo, equalPojo); - CoderProperties.coderConsistentWithEquals(copy, pojo, equalPojo); - CoderProperties.coderConsistentWithEquals(coder, pojo, otherPojo); - CoderProperties.coderConsistentWithEquals(copy, pojo, otherPojo); - } - - /** - * Tests that {@link AvroCoder} works around issues in Avro where cache classes might be from the - * wrong ClassLoader, causing confusing "Cannot cast X to X" error messages. - */ - @SuppressWarnings("ReturnValueIgnored") - @Test - public void testTwoClassLoaders() throws Exception { - ClassLoader contextClassLoader = Thread.currentThread().getContextClassLoader(); - ClassLoader loader1 = - new InterceptingUrlClassLoader(contextClassLoader, AvroCoderTestPojo.class.getName()); - ClassLoader loader2 = - new InterceptingUrlClassLoader(contextClassLoader, AvroCoderTestPojo.class.getName()); - - Class<?> pojoClass1 = loader1.loadClass(AvroCoderTestPojo.class.getName()); - Class<?> pojoClass2 = loader2.loadClass(AvroCoderTestPojo.class.getName()); - - Object pojo1 = InstanceBuilder.ofType(pojoClass1).withArg(String.class, "hello").build(); - Object pojo2 = InstanceBuilder.ofType(pojoClass2).withArg(String.class, "goodbye").build(); - - // Confirm incompatibility - try { - pojoClass2.cast(pojo1); - fail("Expected ClassCastException; without it, this test is vacuous"); - } catch (ClassCastException e) { - // g2g - } - - // The first coder is expected to populate the Avro SpecificData cache - // The second coder is expected to be corrupted if the caching is done wrong. - AvroCoder<Object> avroCoder1 = (AvroCoder) AvroCoder.of(pojoClass1); - AvroCoder<Object> avroCoder2 = (AvroCoder) AvroCoder.of(pojoClass2); - - Object cloned1 = CoderUtils.clone(avroCoder1, pojo1); - Object cloned2 = CoderUtils.clone(avroCoder2, pojo2); - - // Confirming that the uncorrupted coder is fine - pojoClass1.cast(cloned1); - - // Confirmed to fail prior to the fix - pojoClass2.cast(cloned2); - } - - /** - * Confirm that we can serialize and deserialize an AvroCoder object and still decode after. - * (https://github.com/apache/beam/issues/18022). - * - * @throws Exception - */ - @Test - public void testTransientFieldInitialization() throws Exception { - Pojo value = new Pojo("Hello", 42, DATETIME_A); - AvroCoder<Pojo> coder = AvroCoder.of(Pojo.class); - - // Serialization of object - ByteArrayOutputStream bos = new ByteArrayOutputStream(); - ObjectOutputStream out = new ObjectOutputStream(bos); - out.writeObject(coder); - - // De-serialization of object - ByteArrayInputStream bis = new ByteArrayInputStream(bos.toByteArray()); - ObjectInputStream in = new ObjectInputStream(bis); - AvroCoder<Pojo> copied = (AvroCoder<Pojo>) in.readObject(); - - CoderProperties.coderDecodeEncodeEqual(copied, value); - } - - /** - * Confirm that we can serialize and deserialize an AvroCoder object using Kryo. (BEAM-626). - * - * @throws Exception - */ - @Test - public void testKryoSerialization() throws Exception { - Pojo value = new Pojo("Hello", 42, DATETIME_A); - AvroCoder<Pojo> coder = AvroCoder.of(Pojo.class); - - // Kryo instantiation - Kryo kryo = new Kryo(); - kryo.setInstantiatorStrategy(new StdInstantiatorStrategy()); - - // Serialization of object without any memoization - ByteArrayOutputStream coderWithoutMemoizationBos = new ByteArrayOutputStream(); - try (Output output = new Output(coderWithoutMemoizationBos)) { - kryo.writeObject(output, coder); - } - - // Force thread local memoization to store values. - CoderProperties.coderDecodeEncodeEqual(coder, value); - - // Serialization of object with memoized fields - ByteArrayOutputStream coderWithMemoizationBos = new ByteArrayOutputStream(); - try (Output output = new Output(coderWithMemoizationBos)) { - kryo.writeObject(output, coder); - } - - // Copy empty and memoized variants of the Coder - ByteArrayInputStream bisWithoutMemoization = - new ByteArrayInputStream(coderWithoutMemoizationBos.toByteArray()); - AvroCoder<Pojo> copiedWithoutMemoization = - (AvroCoder<Pojo>) kryo.readObject(new Input(bisWithoutMemoization), AvroCoder.class); - ByteArrayInputStream bisWithMemoization = - new ByteArrayInputStream(coderWithMemoizationBos.toByteArray()); - AvroCoder<Pojo> copiedWithMemoization = - (AvroCoder<Pojo>) kryo.readObject(new Input(bisWithMemoization), AvroCoder.class); - - CoderProperties.coderDecodeEncodeEqual(copiedWithoutMemoization, value); - CoderProperties.coderDecodeEncodeEqual(copiedWithMemoization, value); - } - - @Test - public void testPojoEncoding() throws Exception { - Pojo value = new Pojo("Hello", 42, DATETIME_A); - AvroCoder<Pojo> coder = AvroCoder.of(Pojo.class); - - CoderProperties.coderDecodeEncodeEqual(coder, value); - } - - @Test - public void testSpecificRecordEncoding() throws Exception { - AvroCoder<TestAvro> coder = - AvroCoder.of(TestAvro.class, AVRO_SPECIFIC_RECORD.getSchema(), false); - - assertTrue(SpecificRecord.class.isAssignableFrom(coder.getType())); - CoderProperties.coderDecodeEncodeEqual(coder, AVRO_SPECIFIC_RECORD); - } - - @Test - public void testReflectRecordEncoding() throws Exception { - AvroCoder<TestAvro> coder = AvroCoder.of(TestAvro.class, true); - AvroCoder<TestAvro> coderWithSchema = - AvroCoder.of(TestAvro.class, AVRO_SPECIFIC_RECORD.getSchema(), true); - - assertTrue(SpecificRecord.class.isAssignableFrom(coder.getType())); - assertTrue(SpecificRecord.class.isAssignableFrom(coderWithSchema.getType())); - - CoderProperties.coderDecodeEncodeEqual(coder, AVRO_SPECIFIC_RECORD); - CoderProperties.coderDecodeEncodeEqual(coderWithSchema, AVRO_SPECIFIC_RECORD); - } - - @Test - public void testDisableReflectionEncoding() { - try { - AvroCoder.of(Pojo.class, false); - fail("When userReclectApi is disable, schema should not be generated through reflection"); - } catch (AvroRuntimeException e) { - String message = - "avro.shaded.com.google.common.util.concurrent.UncheckedExecutionException: " - + "org.apache.avro.AvroRuntimeException: " - + "Not a Specific class: class org.apache.beam.sdk.coders.AvroCoderTest$Pojo"; - assertEquals(message, e.getMessage()); - } - } - - @Test - public void testGenericRecordEncoding() throws Exception { - String schemaString = - "{\"namespace\": \"example.avro\",\n" - + " \"type\": \"record\",\n" - + " \"name\": \"User\",\n" - + " \"fields\": [\n" - + " {\"name\": \"name\", \"type\": \"string\"},\n" - + " {\"name\": \"favorite_number\", \"type\": [\"int\", \"null\"]},\n" - + " {\"name\": \"favorite_color\", \"type\": [\"string\", \"null\"]}\n" - + " ]\n" - + "}"; - Schema schema = new Schema.Parser().parse(schemaString); - - GenericRecord before = new GenericData.Record(schema); - before.put("name", "Bob"); - before.put("favorite_number", 256); - // Leave favorite_color null - - AvroCoder<GenericRecord> coder = AvroCoder.of(GenericRecord.class, schema); - - CoderProperties.coderDecodeEncodeEqual(coder, before); - assertEquals(schema, coder.getSchema()); - } - - @Test - public void testEncodingNotBuffered() throws Exception { - // This test ensures that the coder doesn't read ahead and buffer data. - // Reading ahead causes a problem if the stream consists of records of different - // types. - Pojo before = new Pojo("Hello", 42, DATETIME_A); - - AvroCoder<Pojo> coder = AvroCoder.of(Pojo.class); - SerializableCoder<Integer> intCoder = SerializableCoder.of(Integer.class); - - ByteArrayOutputStream outStream = new ByteArrayOutputStream(); - - Context context = Context.NESTED; - coder.encode(before, outStream, context); - intCoder.encode(10, outStream, context); - - ByteArrayInputStream inStream = new ByteArrayInputStream(outStream.toByteArray()); - - Pojo after = coder.decode(inStream, context); - assertEquals(before, after); - - Integer intAfter = intCoder.decode(inStream, context); - assertEquals(Integer.valueOf(10), intAfter); - } - - @Test - @Category(NeedsRunner.class) - public void testDefaultCoder() throws Exception { - // Use MyRecord as input and output types without explicitly specifying - // a coder (this uses the default coders, which may not be AvroCoder). - PCollection<String> output = - pipeline - .apply(Create.of(new Pojo("hello", 1, DATETIME_A), new Pojo("world", 2, DATETIME_B))) - .apply(ParDo.of(new GetTextFn())); - - PAssert.that(output).containsInAnyOrder("hello", "world"); - pipeline.run(); - } - - @Test - public void testAvroCoderIsSerializable() throws Exception { - AvroCoder<Pojo> coder = AvroCoder.of(Pojo.class); - - // Check that the coder is serializable using the regular JSON approach. - SerializableUtils.ensureSerializable(coder); - } - - @Test - public void testAvroSpecificCoderIsSerializable() throws Exception { - AvroCoder<TestAvro> coder = AvroCoder.of(TestAvro.class, false); - - // Check that the coder is serializable using the regular JSON approach. - SerializableUtils.ensureSerializable(coder); - } - - private void assertDeterministic(AvroCoder<?> coder) { - try { - coder.verifyDeterministic(); - } catch (NonDeterministicException e) { - fail("Expected " + coder + " to be deterministic, but got:\n" + e); - } - } - - private void assertNonDeterministic(AvroCoder<?> coder, Matcher<String> reason1) { - try { - coder.verifyDeterministic(); - fail("Expected " + coder + " to be non-deterministic."); - } catch (NonDeterministicException e) { - assertThat(e.getReasons(), Matchers.iterableWithSize(1)); - assertThat(e.getReasons(), Matchers.contains(reason1)); - } - } - - @Test - public void testDeterministicInteger() { - assertDeterministic(AvroCoder.of(Integer.class)); - } - - @Test - public void testDeterministicInt() { - assertDeterministic(AvroCoder.of(int.class)); - } - - private static class SimpleDeterministicClass { - @SuppressWarnings("unused") - private Integer intField; - - @SuppressWarnings("unused") - private char charField; - - @SuppressWarnings("unused") - private Integer[] intArray; - - @SuppressWarnings("unused") - private Utf8 utf8field; - } - - @Test - public void testDeterministicSimple() { - assertDeterministic(AvroCoder.of(SimpleDeterministicClass.class)); - } - - private static class UnorderedMapClass { - @SuppressWarnings("unused") - private Map<String, String> mapField; - } - - private Matcher<String> reason(final String prefix, final String messagePart) { - return new TypeSafeMatcher<String>(String.class) { - @Override - public void describeTo(Description description) { - description.appendText( - String.format("Reason starting with '%s:' containing '%s'", prefix, messagePart)); - } - - @Override - protected boolean matchesSafely(String item) { - return item.startsWith(prefix + ":") && item.contains(messagePart); - } - }; - } - - private Matcher<String> reasonClass(Class<?> clazz, String message) { - return reason(clazz.getName(), message); - } - - private Matcher<String> reasonField(Class<?> clazz, String field, String message) { - return reason(clazz.getName() + "#" + field, message); - } - - @Test - public void testDeterministicUnorderedMap() { - assertNonDeterministic( - AvroCoder.of(UnorderedMapClass.class), - reasonField( - UnorderedMapClass.class, - "mapField", - "java.util.Map<java.lang.String, java.lang.String> " - + "may not be deterministically ordered")); - } - - private static class NonDeterministicArray { - @SuppressWarnings("unused") - private UnorderedMapClass[] arrayField; - } - - @Test - public void testDeterministicNonDeterministicArray() { - assertNonDeterministic( - AvroCoder.of(NonDeterministicArray.class), - reasonField( - UnorderedMapClass.class, - "mapField", - "java.util.Map<java.lang.String, java.lang.String>" - + " may not be deterministically ordered")); - } - - private static class SubclassOfUnorderedMapClass extends UnorderedMapClass {} - - @Test - public void testDeterministicNonDeterministicChild() { - // Super class has non deterministic fields. - assertNonDeterministic( - AvroCoder.of(SubclassOfUnorderedMapClass.class), - reasonField(UnorderedMapClass.class, "mapField", "may not be deterministically ordered")); - } - - private static class SubclassHidingParent extends UnorderedMapClass { - @SuppressWarnings("unused") - @AvroName("mapField2") // AvroName is not enough - private int mapField; - } - - @Test - public void testAvroProhibitsShadowing() { - // This test verifies that Avro won't serialize a class with two fields of - // the same name. This is important for our error reporting, and also how - // we lookup a field. - try { - ReflectData.get().getSchema(SubclassHidingParent.class); - fail("Expected AvroTypeException"); - } catch (AvroRuntimeException e) { - assertThat(e.getMessage(), containsString("mapField")); - assertThat(e.getMessage(), containsString("two fields named")); - } - } - - private static class FieldWithAvroName { - @AvroName("name") - @SuppressWarnings("unused") - private int someField; - } - - @Test - public void testDeterministicWithAvroName() { - assertDeterministic(AvroCoder.of(FieldWithAvroName.class)); - } - - @Test - public void testDeterminismSortedMap() { - assertDeterministic(AvroCoder.of(StringSortedMapField.class)); - } - - private static class StringSortedMapField { - @SuppressWarnings("unused") - SortedMap<String, String> sortedMapField; - } - - @Test - public void testDeterminismTreeMapValue() { - // The value is non-deterministic, so we should fail. - assertNonDeterministic( - AvroCoder.of(TreeMapNonDetValue.class), - reasonField( - UnorderedMapClass.class, - "mapField", - "java.util.Map<java.lang.String, java.lang.String> " - + "may not be deterministically ordered")); - } - - private static class TreeMapNonDetValue { - @SuppressWarnings("unused") - TreeMap<String, NonDeterministicArray> nonDeterministicField; - } - - @Test - public void testDeterminismUnorderedMap() { - // LinkedHashMap is not deterministically ordered, so we should fail. - assertNonDeterministic( - AvroCoder.of(LinkedHashMapField.class), - reasonField( - LinkedHashMapField.class, - "nonDeterministicMap", - "java.util.LinkedHashMap<java.lang.String, java.lang.String> " - + "may not be deterministically ordered")); - } - - private static class LinkedHashMapField { - @SuppressWarnings("unused") - LinkedHashMap<String, String> nonDeterministicMap; - } - - @Test - public void testDeterminismCollection() { - assertNonDeterministic( - AvroCoder.of(StringCollection.class), - reasonField( - StringCollection.class, - "stringCollection", - "java.util.Collection<java.lang.String> may not be deterministically ordered")); - } - - private static class StringCollection { - @SuppressWarnings("unused") - Collection<String> stringCollection; - } - - @Test - public void testDeterminismList() { - assertDeterministic(AvroCoder.of(StringList.class)); - assertDeterministic(AvroCoder.of(StringArrayList.class)); - } - - private static class StringList { - @SuppressWarnings("unused") - List<String> stringCollection; - } - - private static class StringArrayList { - @SuppressWarnings("unused") - ArrayList<String> stringCollection; - } - - @Test - public void testDeterminismSet() { - assertDeterministic(AvroCoder.of(StringSortedSet.class)); - assertDeterministic(AvroCoder.of(StringTreeSet.class)); - assertNonDeterministic( - AvroCoder.of(StringHashSet.class), - reasonField( - StringHashSet.class, - "stringCollection", - "java.util.HashSet<java.lang.String> may not be deterministically ordered")); - } - - private static class StringSortedSet { - @SuppressWarnings("unused") - SortedSet<String> stringCollection; - } - - private static class StringTreeSet { - @SuppressWarnings("unused") - TreeSet<String> stringCollection; - } - - private static class StringHashSet { - @SuppressWarnings("unused") - HashSet<String> stringCollection; - } - - @Test - public void testDeterminismCollectionValue() { - assertNonDeterministic( - AvroCoder.of(OrderedSetOfNonDetValues.class), - reasonField(UnorderedMapClass.class, "mapField", "may not be deterministically ordered")); - assertNonDeterministic( - AvroCoder.of(ListOfNonDetValues.class), - reasonField(UnorderedMapClass.class, "mapField", "may not be deterministically ordered")); - } - - private static class OrderedSetOfNonDetValues { - @SuppressWarnings("unused") - SortedSet<UnorderedMapClass> set; - } - - private static class ListOfNonDetValues { - @SuppressWarnings("unused") - List<UnorderedMapClass> set; - } - - @Test - public void testDeterminismUnion() { - assertDeterministic(AvroCoder.of(DeterministicUnionBase.class)); - assertNonDeterministic( - AvroCoder.of(NonDeterministicUnionBase.class), - reasonField(UnionCase3.class, "mapField", "may not be deterministically ordered")); - } - - @Test - public void testDeterminismStringable() { - assertDeterministic(AvroCoder.of(String.class)); - assertNonDeterministic( - AvroCoder.of(StringableClass.class), - reasonClass(StringableClass.class, "may not have deterministic #toString()")); - } - - @Stringable - private static class StringableClass {} - - @Test - public void testDeterminismCyclicClass() { - assertNonDeterministic( - AvroCoder.of(Cyclic.class), - reasonField(Cyclic.class, "cyclicField", "appears recursively")); - assertNonDeterministic( - AvroCoder.of(CyclicField.class), - reasonField(Cyclic.class, "cyclicField", Cyclic.class.getName() + " appears recursively")); - assertNonDeterministic( - AvroCoder.of(IndirectCycle1.class), - reasonField( - IndirectCycle2.class, - "field2", - IndirectCycle1.class.getName() + " appears recursively")); - } - - private static class Cyclic { - @SuppressWarnings("unused") - int intField; - - @SuppressWarnings("unused") - Cyclic cyclicField; - } - - private static class CyclicField { - @SuppressWarnings("unused") - Cyclic cyclicField2; - } - - private static class IndirectCycle1 { - @SuppressWarnings("unused") - IndirectCycle2 field1; - } - - private static class IndirectCycle2 { - @SuppressWarnings("unused") - IndirectCycle1 field2; - } - - @Test - public void testDeterminismHasGenericRecord() { - assertDeterministic(AvroCoder.of(HasGenericRecord.class)); - } - - private static class HasGenericRecord { - @AvroSchema( - "{\"name\": \"bar\", \"type\": \"record\", \"fields\": [" - + "{\"name\": \"foo\", \"type\": \"int\"}]}") - GenericRecord genericRecord; - } - - @Test - public void testDeterminismHasCustomSchema() { - assertNonDeterministic( - AvroCoder.of(HasCustomSchema.class), - reasonField( - HasCustomSchema.class, - "withCustomSchema", - "Custom schemas are only supported for subtypes of IndexedRecord.")); - } - - private static class HasCustomSchema { - @AvroSchema( - "{\"name\": \"bar\", \"type\": \"record\", \"fields\": [" - + "{\"name\": \"foo\", \"type\": \"int\"}]}") - int withCustomSchema; - } - - @Test - public void testAvroCoderTreeMapDeterminism() throws Exception, NonDeterministicException { - TreeMapField size1 = new TreeMapField(); - TreeMapField size2 = new TreeMapField(); - - // Different order for entries - size1.field.put("hello", "world"); - size1.field.put("another", "entry"); - - size2.field.put("another", "entry"); - size2.field.put("hello", "world"); - - AvroCoder<TreeMapField> coder = AvroCoder.of(TreeMapField.class); - coder.verifyDeterministic(); - - ByteArrayOutputStream outStream1 = new ByteArrayOutputStream(); - ByteArrayOutputStream outStream2 = new ByteArrayOutputStream(); - - Context context = Context.NESTED; - coder.encode(size1, outStream1, context); - coder.encode(size2, outStream2, context); - - assertArrayEquals(outStream1.toByteArray(), outStream2.toByteArray()); - } - - private static class TreeMapField { - private TreeMap<String, String> field = new TreeMap<>(); - } - - @Union({UnionCase1.class, UnionCase2.class}) - private abstract static class DeterministicUnionBase {} - - @Union({UnionCase1.class, UnionCase2.class, UnionCase3.class}) - private abstract static class NonDeterministicUnionBase {} - - private static class UnionCase1 extends DeterministicUnionBase {} - - private static class UnionCase2 extends DeterministicUnionBase { - @SuppressWarnings("unused") - String field; - } - - private static class UnionCase3 extends NonDeterministicUnionBase { - @SuppressWarnings("unused") - private Map<String, String> mapField; - } - - @Test - public void testAvroCoderSimpleSchemaDeterminism() { - assertDeterministic(AvroCoder.of(SchemaBuilder.record("someRecord").fields().endRecord())); - assertDeterministic( - AvroCoder.of( - SchemaBuilder.record("someRecord") - .fields() - .name("int") - .type() - .intType() - .noDefault() - .endRecord())); - assertDeterministic( - AvroCoder.of( - SchemaBuilder.record("someRecord") - .fields() - .name("string") - .type() - .stringType() - .noDefault() - .endRecord())); - - assertNonDeterministic( - AvroCoder.of( - SchemaBuilder.record("someRecord") - .fields() - .name("map") - .type() - .map() - .values() - .stringType() - .noDefault() - .endRecord()), - reason("someRecord.map", "HashMap to represent MAPs")); - - assertDeterministic( - AvroCoder.of( - SchemaBuilder.record("someRecord") - .fields() - .name("array") - .type() - .array() - .items() - .stringType() - .noDefault() - .endRecord())); - - assertDeterministic( - AvroCoder.of( - SchemaBuilder.record("someRecord") - .fields() - .name("enum") - .type() - .enumeration("anEnum") - .symbols("s1", "s2") - .enumDefault("s1") - .endRecord())); - - assertDeterministic( - AvroCoder.of( - SchemaBuilder.unionOf() - .intType() - .and() - .record("someRecord") - .fields() - .nullableString("someField", "") - .endRecord() - .endUnion())); - } - - @Test - public void testAvroCoderStrings() { - // Custom Strings in Records - assertDeterministic( - AvroCoder.of( - SchemaBuilder.record("someRecord") - .fields() - .name("string") - .prop(SpecificData.CLASS_PROP, "java.lang.String") - .type() - .stringType() - .noDefault() - .endRecord())); - assertNonDeterministic( - AvroCoder.of( - SchemaBuilder.record("someRecord") - .fields() - .name("string") - .prop(SpecificData.CLASS_PROP, "unknownString") - .type() - .stringType() - .noDefault() - .endRecord()), - reason("someRecord.string", "unknownString is not known to be deterministic")); - - // Custom Strings in Unions - assertNonDeterministic( - AvroCoder.of( - SchemaBuilder.unionOf() - .intType() - .and() - .record("someRecord") - .fields() - .name("someField") - .prop(SpecificData.CLASS_PROP, "unknownString") - .type() - .stringType() - .noDefault() - .endRecord() - .endUnion()), - reason("someRecord.someField", "unknownString is not known to be deterministic")); - } - - @Test - public void testAvroCoderNestedRecords() { - // Nested Record - assertDeterministic( - AvroCoder.of( - SchemaBuilder.record("nestedRecord") - .fields() - .name("subRecord") - .type() - .record("subRecord") - .fields() - .name("innerField") - .type() - .stringType() - .noDefault() - .endRecord() - .noDefault() - .endRecord())); - } - - @Test - public void testAvroCoderCyclicRecords() { - // Recursive record - assertNonDeterministic( - AvroCoder.of( - SchemaBuilder.record("cyclicRecord") - .fields() - .name("cycle") - .type("cyclicRecord") - .noDefault() - .endRecord()), - reason("cyclicRecord.cycle", "cyclicRecord appears recursively")); - } - - private static class NullableField { - @SuppressWarnings("unused") - private @Nullable String nullable; - } - - @Test - public void testNullableField() { - assertDeterministic(AvroCoder.of(NullableField.class)); - } - - private static class NullableNonDeterministicField { - @SuppressWarnings("unused") - private @Nullable NonDeterministicArray nullableNonDetArray; - } - - private static class NullableCyclic { - @SuppressWarnings("unused") - private @Nullable NullableCyclic nullableNullableCyclicField; - } - - private static class NullableCyclicField { - @SuppressWarnings("unused") - private @Nullable Cyclic nullableCyclicField; - } - - @Test - public void testNullableNonDeterministicField() { - assertNonDeterministic( - AvroCoder.of(NullableCyclic.class), - reasonField( - NullableCyclic.class, - "nullableNullableCyclicField", - NullableCyclic.class.getName() + " appears recursively")); - assertNonDeterministic( - AvroCoder.of(NullableCyclicField.class), - reasonField(Cyclic.class, "cyclicField", Cyclic.class.getName() + " appears recursively")); - assertNonDeterministic( - AvroCoder.of(NullableNonDeterministicField.class), - reasonField(UnorderedMapClass.class, "mapField", " may not be deterministically ordered")); - } - - /** - * Tests that a parameterized class can have an automatically generated schema if the generic - * field is annotated with a union tag. - */ - @Test - public void testGenericClassWithUnionAnnotation() throws Exception { - // Cast is safe as long as the same coder is used for encoding and decoding. - @SuppressWarnings({"unchecked", "rawtypes"}) - AvroCoder<GenericWithAnnotation<String>> coder = - (AvroCoder) AvroCoder.of(GenericWithAnnotation.class); - - assertThat( - coder.getSchema().getField("onlySomeTypesAllowed").schema().getType(), - equalTo(Schema.Type.UNION)); - - CoderProperties.coderDecodeEncodeEqual(coder, new GenericWithAnnotation<>("hello")); - } - - private static class GenericWithAnnotation<T> { - @AvroSchema("[\"string\", \"int\"]") - private T onlySomeTypesAllowed; - - public GenericWithAnnotation(T value) { - onlySomeTypesAllowed = value; - } - - // For deserialization only - @SuppressWarnings("unused") - protected GenericWithAnnotation() {} - - @Override - public boolean equals(@Nullable Object other) { - return other instanceof GenericWithAnnotation - && onlySomeTypesAllowed.equals(((GenericWithAnnotation<?>) other).onlySomeTypesAllowed); - } - - @Override - public int hashCode() { - return Objects.hash(getClass(), onlySomeTypesAllowed); - } - } - - @Test - public void testAvroCoderForGenerics() throws Exception { - Schema fooSchema = AvroCoder.of(Foo.class).getSchema(); - Schema schema = - new Schema.Parser() - .parse( - "{" - + "\"type\":\"record\"," - + "\"name\":\"SomeGeneric\"," - + "\"namespace\":\"ns\"," - + "\"fields\":[" - + " {\"name\":\"foo\", \"type\":" - + fooSchema.toString() - + "}" - + "]}"); - @SuppressWarnings("rawtypes") - AvroCoder<SomeGeneric> coder = AvroCoder.of(SomeGeneric.class, schema); - - assertNonDeterministic(coder, reasonField(SomeGeneric.class, "foo", "erasure")); - } - - @Test - public void testEncodedTypeDescriptor() throws Exception { - AvroCoder<Pojo> coder = AvroCoder.of(Pojo.class); - assertThat(coder.getEncodedTypeDescriptor(), equalTo(TypeDescriptor.of(Pojo.class))); - } - - private static class SomeGeneric<T> { - @SuppressWarnings("unused") - private T foo; - } - - private static class Foo { - @SuppressWarnings("unused") - String id; - } -} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/AvroCoderTestPojo.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/AvroCoderTestPojo.java deleted file mode 100644 index d170e6447b89..000000000000 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/AvroCoderTestPojo.java +++ /dev/null @@ -1,51 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.coders; - -import java.util.Objects; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; -import org.checkerframework.checker.nullness.qual.Nullable; - -/** A Pojo at the top level for use in tests. */ -class AvroCoderTestPojo { - - public String text; - - // Empty constructor required for Avro decoding. - @SuppressWarnings("unused") - public AvroCoderTestPojo() {} - - public AvroCoderTestPojo(String text) { - this.text = text; - } - - @Override - public boolean equals(@Nullable Object other) { - return (other instanceof AvroCoderTestPojo) && ((AvroCoderTestPojo) other).text.equals(text); - } - - @Override - public int hashCode() { - return Objects.hash(AvroCoderTestPojo.class, text); - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this).add("text", text).toString(); - } -} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderRegistryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderRegistryTest.java index 5a8d7e83c442..36966c2d35d6 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderRegistryTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderRegistryTest.java @@ -463,7 +463,7 @@ public void testCoderPrecedence() throws Exception { CoderRegistry registry = CoderRegistry.createDefault(); // DefaultCoder precedes CoderProviderRegistrar - assertEquals(AvroCoder.of(MyValueA.class), registry.getCoder(MyValueA.class)); + assertEquals(MockDefaultCoder.of(MyValueA.class), registry.getCoder(MyValueA.class)); // CoderProviderRegistrar precedes SerializableCoder assertEquals(MyValueBCoder.INSTANCE, registry.getCoder(MyValueB.class)); @@ -472,7 +472,7 @@ public void testCoderPrecedence() throws Exception { assertEquals(SerializableCoder.of(MyValueC.class), registry.getCoder(MyValueC.class)); } - @DefaultCoder(AvroCoder.class) + @DefaultCoder(MockDefaultCoder.class) private static class MyValueA implements Serializable {} private static class MyValueB implements Serializable {} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DefaultCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DefaultCoderTest.java index d120ec07f571..62d6d7e1d049 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DefaultCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DefaultCoderTest.java @@ -39,7 +39,7 @@ public class DefaultCoderTest { @Rule public ExpectedException thrown = ExpectedException.none(); - @DefaultCoder(AvroCoder.class) + @DefaultCoder(MockDefaultCoder.class) private static class AvroRecord {} private static class SerializableBase implements Serializable {} @@ -111,7 +111,7 @@ public <T> Coder<T> coderFor( public void testCodersWithoutComponents() throws Exception { CoderRegistry registry = CoderRegistry.createDefault(); registry.registerCoderProvider(new DefaultCoderProvider()); - assertThat(registry.getCoder(AvroRecord.class), instanceOf(AvroCoder.class)); + assertThat(registry.getCoder(AvroRecord.class), instanceOf(MockDefaultCoder.class)); assertThat(registry.getCoder(SerializableRecord.class), instanceOf(SerializableCoder.class)); assertThat(registry.getCoder(CustomRecord.class), instanceOf(CustomSerializableCoder.class)); assertThat( @@ -125,7 +125,7 @@ public void testDefaultCoderInCollection() throws Exception { Coder<List<AvroRecord>> avroRecordCoder = registry.getCoder(new TypeDescriptor<List<AvroRecord>>() {}); assertThat(avroRecordCoder, instanceOf(ListCoder.class)); - assertThat(((ListCoder) avroRecordCoder).getElemCoder(), instanceOf(AvroCoder.class)); + assertThat(((ListCoder) avroRecordCoder).getElemCoder(), instanceOf(MockDefaultCoder.class)); assertThat( registry.getCoder(new TypeDescriptor<List<SerializableRecord>>() {}), Matchers.equalTo(ListCoder.of(SerializableCoder.of(SerializableRecord.class)))); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/MockDefaultCoder.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/MockDefaultCoder.java new file mode 100644 index 000000000000..4f20e86bf6bb --- /dev/null +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/MockDefaultCoder.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.coders; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.List; +import org.apache.beam.sdk.values.TypeDescriptor; + +/** + * Used only for tests. + * + * @param <T> + */ +class MockDefaultCoder<T> extends CustomCoder<T> { + private static final MockDefaultCoder INSTANCE = new MockDefaultCoder(); + + @Override + public void encode(T value, OutputStream outStream) throws IOException {} + + @Override + public T decode(InputStream inStream) throws IOException { + return null; + } + + public static <T> MockDefaultCoder<T> of(Class<T> clazz) { + return INSTANCE; + } + + public static CoderProvider getCoderProvider() { + return new MockAvroCoderProvider(); + } + + static class MockAvroCoderProvider extends CoderProvider { + @Override + public <T> Coder<T> coderFor( + TypeDescriptor<T> typeDescriptor, List<? extends Coder<?>> componentCoders) { + return (Coder<T>) MockDefaultCoder.INSTANCE; + } + } +} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroIOTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroIOTest.java deleted file mode 100644 index 0126dfb0c02f..000000000000 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroIOTest.java +++ /dev/null @@ -1,1627 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.io; - -import static org.apache.avro.file.DataFileConstants.SNAPPY_CODEC; -import static org.apache.beam.sdk.io.Compression.AUTO; -import static org.apache.beam.sdk.io.fs.ResolveOptions.StandardResolveOptions.RESOLVE_FILE; -import static org.apache.beam.sdk.transforms.Contextful.fn; -import static org.apache.beam.sdk.transforms.Requirements.requiresSideInputs; -import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; -import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects.firstNonNull; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.containsInAnyOrder; -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - -import java.io.File; -import java.io.FileInputStream; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.io.Serializable; -import java.nio.file.Files; -import java.nio.file.Path; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.Random; -import java.util.stream.Collectors; -import org.apache.avro.Schema; -import org.apache.avro.SchemaBuilder; -import org.apache.avro.file.CodecFactory; -import org.apache.avro.file.DataFileReader; -import org.apache.avro.file.DataFileStream; -import org.apache.avro.generic.GenericData; -import org.apache.avro.generic.GenericDatumReader; -import org.apache.avro.generic.GenericDatumWriter; -import org.apache.avro.generic.GenericRecord; -import org.apache.avro.generic.GenericRecordBuilder; -import org.apache.avro.io.DatumWriter; -import org.apache.avro.io.Encoder; -import org.apache.avro.reflect.ReflectData; -import org.apache.avro.reflect.ReflectDatumReader; -import org.apache.beam.sdk.coders.AvroCoder; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.CoderException; -import org.apache.beam.sdk.coders.DefaultCoder; -import org.apache.beam.sdk.coders.KvCoder; -import org.apache.beam.sdk.coders.StringUtf8Coder; -import org.apache.beam.sdk.io.FileBasedSink.FilenamePolicy; -import org.apache.beam.sdk.io.FileBasedSink.OutputFileHints; -import org.apache.beam.sdk.io.fs.ResourceId; -import org.apache.beam.sdk.options.ValueProvider; -import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider; -import org.apache.beam.sdk.testing.NeedsRunner; -import org.apache.beam.sdk.testing.PAssert; -import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.testing.TestStream; -import org.apache.beam.sdk.testing.UsesTestStream; -import org.apache.beam.sdk.testing.UsesUnboundedSplittableParDo; -import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.transforms.MapElements; -import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.transforms.SerializableFunction; -import org.apache.beam.sdk.transforms.SimpleFunction; -import org.apache.beam.sdk.transforms.View; -import org.apache.beam.sdk.transforms.Watch; -import org.apache.beam.sdk.transforms.display.DisplayData; -import org.apache.beam.sdk.transforms.windowing.AfterPane; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.transforms.windowing.FixedWindows; -import org.apache.beam.sdk.transforms.windowing.IntervalWindow; -import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.transforms.windowing.Repeatedly; -import org.apache.beam.sdk.transforms.windowing.Window; -import org.apache.beam.sdk.util.SerializableUtils; -import org.apache.beam.sdk.values.KV; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.PCollectionView; -import org.apache.beam.sdk.values.TimestampedValue; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ArrayListMultimap; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterators; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Multimap; -import org.checkerframework.checker.nullness.qual.Nullable; -import org.joda.time.Duration; -import org.joda.time.Instant; -import org.junit.Rule; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.junit.rules.ExpectedException; -import org.junit.rules.TemporaryFolder; -import org.junit.rules.Timeout; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; -import org.junit.runners.Parameterized; - -/** Tests for AvroIO Read and Write transforms. */ -@SuppressWarnings({ - "rawtypes", // TODO(https://github.com/apache/beam/issues/20447) -}) -public class AvroIOTest implements Serializable { - /** Unit tests. */ - @RunWith(JUnit4.class) - public static class SimpleTests implements Serializable { - @Test - public void testAvroIOGetName() { - assertEquals("AvroIO.Read", AvroIO.read(String.class).from("/tmp/foo*/baz").getName()); - assertEquals("AvroIO.Write", AvroIO.write(String.class).to("/tmp/foo/baz").getName()); - } - - @Test - public void testWriteWithDefaultCodec() { - AvroIO.Write<String> write = AvroIO.write(String.class).to("/tmp/foo/baz"); - assertEquals(CodecFactory.snappyCodec().toString(), write.inner.getCodec().toString()); - } - - @Test - public void testWriteWithCustomCodec() { - AvroIO.Write<String> write = - AvroIO.write(String.class).to("/tmp/foo/baz").withCodec(CodecFactory.snappyCodec()); - assertEquals(SNAPPY_CODEC, write.inner.getCodec().toString()); - } - - @Test - public void testWriteWithSerDeCustomDeflateCodec() { - AvroIO.Write<String> write = - AvroIO.write(String.class).to("/tmp/foo/baz").withCodec(CodecFactory.deflateCodec(9)); - - assertEquals( - CodecFactory.deflateCodec(9).toString(), - SerializableUtils.clone(write.inner.getCodec()).getCodec().toString()); - } - - @Test - public void testWriteWithSerDeCustomXZCodec() { - AvroIO.Write<String> write = - AvroIO.write(String.class).to("/tmp/foo/baz").withCodec(CodecFactory.xzCodec(9)); - - assertEquals( - CodecFactory.xzCodec(9).toString(), - SerializableUtils.clone(write.inner.getCodec()).getCodec().toString()); - } - - @Test - public void testReadDisplayData() { - AvroIO.Read<String> read = AvroIO.read(String.class).from("/foo.*"); - - DisplayData displayData = DisplayData.from(read); - assertThat(displayData, hasDisplayItem("filePattern", "/foo.*")); - } - } - - /** NeedsRunner tests. */ - @RunWith(Parameterized.class) - @Category(NeedsRunner.class) - public static class NeedsRunnerTests implements Serializable { - @Rule public transient TestPipeline writePipeline = TestPipeline.create(); - - @Rule public transient TestPipeline readPipeline = TestPipeline.create(); - - @Rule public transient TestPipeline windowedAvroWritePipeline = TestPipeline.create(); - - @Rule public transient TemporaryFolder tmpFolder = new TemporaryFolder(); - - @Rule public transient ExpectedException expectedException = ExpectedException.none(); - - @Rule public transient Timeout globalTimeout = Timeout.seconds(1200); - - @Parameterized.Parameters(name = "{index}: {0}") - public static Collection<Object[]> params() { - return Arrays.asList(new Object[][] {{true}, {false}}); - } - - @Parameterized.Parameter public boolean withBeamSchemas; - - @DefaultCoder(AvroCoder.class) - static class GenericClass { - int intField; - String stringField; - - GenericClass() {} - - GenericClass(int intField, String stringField) { - this.intField = intField; - this.stringField = stringField; - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(getClass()) - .add("intField", intField) - .add("stringField", stringField) - .toString(); - } - - @Override - public int hashCode() { - return Objects.hash(intField, stringField); - } - - @Override - public boolean equals(@Nullable Object other) { - if (other == null || !(other instanceof GenericClass)) { - return false; - } - GenericClass o = (GenericClass) other; - return intField == o.intField && Objects.equals(stringField, o.stringField); - } - } - - private static class ParseGenericClass - implements SerializableFunction<GenericRecord, GenericClass> { - @Override - public GenericClass apply(GenericRecord input) { - return new GenericClass((int) input.get("intField"), input.get("stringField").toString()); - } - - @Test - public void testWriteDisplayData() { - AvroIO.Write<GenericClass> write = - AvroIO.write(GenericClass.class) - .to("/foo") - .withShardNameTemplate("-SS-of-NN-") - .withSuffix("bar") - .withNumShards(100) - .withCodec(CodecFactory.deflateCodec(6)); - - DisplayData displayData = DisplayData.from(write); - - assertThat(displayData, hasDisplayItem("filePrefix", "/foo")); - assertThat(displayData, hasDisplayItem("shardNameTemplate", "-SS-of-NN-")); - assertThat(displayData, hasDisplayItem("fileSuffix", "bar")); - assertThat( - displayData, - hasDisplayItem( - "schema", - "{\"type\":\"record\",\"name\":\"GenericClass\",\"namespace\":\"org.apache.beam.sdk.io" - + ".AvroIOTest$\",\"fields\":[{\"name\":\"intField\",\"type\":\"int\"}," - + "{\"name\":\"stringField\",\"type\":\"string\"}]}")); - assertThat(displayData, hasDisplayItem("numShards", 100)); - assertThat(displayData, hasDisplayItem("codec", CodecFactory.deflateCodec(6).toString())); - } - } - - private enum Sharding { - RUNNER_DETERMINED, - WITHOUT_SHARDING, - FIXED_3_SHARDS - } - - private enum WriteMethod { - AVROIO_WRITE, - AVROIO_SINK_WITH_CLASS, - AVROIO_SINK_WITH_SCHEMA, - /** @deprecated Test code for the deprecated {AvroIO.RecordFormatter}. */ - @Deprecated - AVROIO_SINK_WITH_FORMATTER - } - - private static final String SCHEMA_STRING = - "{\"namespace\": \"example.avro\",\n" - + " \"type\": \"record\",\n" - + " \"name\": \"AvroGeneratedUser\",\n" - + " \"fields\": [\n" - + " {\"name\": \"name\", \"type\": \"string\"},\n" - + " {\"name\": \"favorite_number\", \"type\": [\"int\", \"null\"]},\n" - + " {\"name\": \"favorite_color\", \"type\": [\"string\", \"null\"]}\n" - + " ]\n" - + "}"; - - private static final Schema SCHEMA = new Schema.Parser().parse(SCHEMA_STRING); - - @Test - @Category(NeedsRunner.class) - public void testWriteThenReadJavaClass() throws Throwable { - List<GenericClass> values = - ImmutableList.of(new GenericClass(3, "hi"), new GenericClass(5, "bar")); - File outputFile = tmpFolder.newFile("output.avro"); - - writePipeline - .apply(Create.of(values)) - .apply( - AvroIO.write(GenericClass.class) - .to(writePipeline.newProvider(outputFile.getAbsolutePath())) - .withoutSharding()); - writePipeline.run(); - - PAssert.that( - readPipeline.apply( - "Read", - AvroIO.read(GenericClass.class) - .withBeamSchemas(withBeamSchemas) - .from(readPipeline.newProvider(outputFile.getAbsolutePath())))) - .containsInAnyOrder(values); - - readPipeline.run(); - } - - @Test - @Category(NeedsRunner.class) - public void testReadWithFilename() throws Throwable { - List<GenericClass> values = - ImmutableList.of(new GenericClass(3, "hi"), new GenericClass(5, "bar")); - File outputFile = tmpFolder.newFile("output.avro"); - - writePipeline - .apply(Create.of(values)) - .apply( - AvroIO.write(GenericClass.class) - .to(writePipeline.newProvider(outputFile.getAbsolutePath())) - .withoutSharding()); - writePipeline.run(); - - SerializableFunction<String, ? extends FileBasedSource<GenericClass>> createSource = - input -> - AvroSource.from(ValueProvider.StaticValueProvider.of(input)) - .withSchema(GenericClass.class); - - final PCollection<KV<String, GenericClass>> lines = - readPipeline - .apply(Create.of(Collections.singletonList(outputFile.getAbsolutePath()))) - .apply(FileIO.matchAll()) - .apply(FileIO.readMatches().withCompression(AUTO)) - .apply( - new ReadAllViaFileBasedSourceWithFilename<>( - 10, - createSource, - KvCoder.of(StringUtf8Coder.of(), AvroCoder.of(GenericClass.class)))); - - PAssert.that(lines) - .containsInAnyOrder( - values.stream() - .map(v -> KV.of(outputFile.getAbsolutePath(), v)) - .collect(Collectors.toList())); - readPipeline.run(); - } - - @Test - @Category(NeedsRunner.class) - public void testWriteThenReadCustomType() throws Throwable { - List<Long> values = Arrays.asList(0L, 1L, 2L); - File outputFile = tmpFolder.newFile("output.avro"); - - writePipeline - .apply(Create.of(values)) - .apply( - AvroIO.<Long, GenericClass>writeCustomType() - .to(writePipeline.newProvider(outputFile.getAbsolutePath())) - .withFormatFunction(new CreateGenericClass()) - .withSchema(ReflectData.get().getSchema(GenericClass.class)) - .withoutSharding()); - writePipeline.run(); - - PAssert.that( - readPipeline - .apply( - "Read", - AvroIO.read(GenericClass.class) - .withBeamSchemas(withBeamSchemas) - .from(readPipeline.newProvider(outputFile.getAbsolutePath()))) - .apply( - MapElements.via( - new SimpleFunction<GenericClass, Long>() { - @Override - public Long apply(GenericClass input) { - return (long) input.intField; - } - }))) - .containsInAnyOrder(values); - - readPipeline.run(); - } - - private <T extends GenericRecord> void testWriteThenReadGeneratedClass( - AvroIO.Write<T> writeTransform, AvroIO.Read<T> readTransform) throws Exception { - File outputFile = tmpFolder.newFile("output.avro"); - - List<T> values = - ImmutableList.of( - (T) new AvroGeneratedUser("Bob", 256, null), - (T) new AvroGeneratedUser("Alice", 128, null), - (T) new AvroGeneratedUser("Ted", null, "white")); - - writePipeline - .apply(Create.of(values)) - .apply( - writeTransform - .to(writePipeline.newProvider(outputFile.getAbsolutePath())) - .withoutSharding()); - writePipeline.run(); - - PAssert.that( - readPipeline.apply( - "Read", - readTransform.from(readPipeline.newProvider(outputFile.getAbsolutePath())))) - .containsInAnyOrder(values); - - readPipeline.run(); - } - - @Test - @Category(NeedsRunner.class) - public void testWriteThenReadGeneratedClassWithClass() throws Throwable { - testWriteThenReadGeneratedClass( - AvroIO.write(AvroGeneratedUser.class), - AvroIO.read(AvroGeneratedUser.class).withBeamSchemas(withBeamSchemas)); - } - - @Test - @Category(NeedsRunner.class) - public void testWriteThenReadGeneratedClassWithSchema() throws Throwable { - testWriteThenReadGeneratedClass( - AvroIO.writeGenericRecords(SCHEMA), - AvroIO.readGenericRecords(SCHEMA).withBeamSchemas(withBeamSchemas)); - } - - @Test - @Category(NeedsRunner.class) - public void testWriteThenReadGeneratedClassWithSchemaString() throws Throwable { - testWriteThenReadGeneratedClass( - AvroIO.writeGenericRecords(SCHEMA.toString()), - AvroIO.readGenericRecords(SCHEMA.toString()).withBeamSchemas(withBeamSchemas)); - } - - @Test - @Category(NeedsRunner.class) - public void testWriteSingleFileThenReadUsingAllMethods() throws Throwable { - List<GenericClass> values = - ImmutableList.of(new GenericClass(3, "hi"), new GenericClass(5, "bar")); - File outputFile = tmpFolder.newFile("output.avro"); - - writePipeline - .apply(Create.of(values)) - .apply( - AvroIO.write(GenericClass.class).to(outputFile.getAbsolutePath()).withoutSharding()); - writePipeline.run(); - - // Test the same data using all versions of read(). - PCollection<String> path = - readPipeline.apply("Create path", Create.of(outputFile.getAbsolutePath())); - PAssert.that( - readPipeline.apply( - "Read", - AvroIO.read(GenericClass.class) - .withBeamSchemas(withBeamSchemas) - .from(outputFile.getAbsolutePath()))) - .containsInAnyOrder(values); - PAssert.that( - readPipeline.apply( - "Read withHintMatchesManyFiles", - AvroIO.read(GenericClass.class) - .withBeamSchemas(withBeamSchemas) - .from(outputFile.getAbsolutePath()) - .withHintMatchesManyFiles())) - .containsInAnyOrder(values); - PAssert.that( - path.apply("MatchAllReadFiles", FileIO.matchAll()) - .apply("ReadMatchesReadFiles", FileIO.readMatches().withCompression(AUTO)) - .apply( - "ReadFiles", - AvroIO.readFiles(GenericClass.class) - .withBeamSchemas(withBeamSchemas) - .withDesiredBundleSizeBytes(10))) - .containsInAnyOrder(values); - PAssert.that( - path.apply( - "ReadAll", - AvroIO.readAll(GenericClass.class) - .withBeamSchemas(withBeamSchemas) - .withDesiredBundleSizeBytes(10))) - .containsInAnyOrder(values); - PAssert.that( - readPipeline.apply( - "Parse", - AvroIO.parseGenericRecords(new ParseGenericClass()) - .from(outputFile.getAbsolutePath()) - .withCoder(AvroCoder.of(GenericClass.class)))) - .containsInAnyOrder(values); - PAssert.that( - readPipeline.apply( - "Parse withHintMatchesManyFiles", - AvroIO.parseGenericRecords(new ParseGenericClass()) - .from(outputFile.getAbsolutePath()) - .withCoder(AvroCoder.of(GenericClass.class)) - .withHintMatchesManyFiles())) - .containsInAnyOrder(values); - PAssert.that( - path.apply("MatchAllParseFilesGenericRecords", FileIO.matchAll()) - .apply( - "ReadMatchesParseFilesGenericRecords", - FileIO.readMatches() - .withDirectoryTreatment(FileIO.ReadMatches.DirectoryTreatment.PROHIBIT)) - .apply( - "ParseFilesGenericRecords", - AvroIO.parseFilesGenericRecords(new ParseGenericClass()) - .withCoder(AvroCoder.of(GenericClass.class)) - .withUsesReshuffle(false) - .withDesiredBundleSizeBytes(10))) - .containsInAnyOrder(values); - PAssert.that( - path.apply("MatchAllParseFilesGenericRecordsWithShuffle", FileIO.matchAll()) - .apply( - "ReadMatchesParseFilesGenericRecordsWithShuffle", - FileIO.readMatches() - .withDirectoryTreatment(FileIO.ReadMatches.DirectoryTreatment.PROHIBIT)) - .apply( - "ParseFilesGenericRecordsWithShuffle", - AvroIO.parseFilesGenericRecords(new ParseGenericClass()) - .withCoder(AvroCoder.of(GenericClass.class)) - .withUsesReshuffle(true) - .withDesiredBundleSizeBytes(10))) - .containsInAnyOrder(values); - PAssert.that( - path.apply( - "ParseAllGenericRecords", - AvroIO.parseAllGenericRecords(new ParseGenericClass()) - .withCoder(AvroCoder.of(GenericClass.class)) - .withDesiredBundleSizeBytes(10))) - .containsInAnyOrder(values); - - readPipeline.run(); - } - - @Test - @Category(NeedsRunner.class) - public void testWriteThenReadMultipleFilepatterns() { - List<GenericClass> firstValues = new ArrayList<>(); - List<GenericClass> secondValues = new ArrayList<>(); - for (int i = 0; i < 10; ++i) { - firstValues.add(new GenericClass(i, "a" + i)); - secondValues.add(new GenericClass(i, "b" + i)); - } - writePipeline - .apply("Create first", Create.of(firstValues)) - .apply( - "Write first", - AvroIO.write(GenericClass.class) - .to(tmpFolder.getRoot().getAbsolutePath() + "/first") - .withNumShards(2)); - writePipeline - .apply("Create second", Create.of(secondValues)) - .apply( - "Write second", - AvroIO.write(GenericClass.class) - .to(tmpFolder.getRoot().getAbsolutePath() + "/second") - .withNumShards(3)); - writePipeline.run(); - - // Test readFiles(), readAll(), parseFilesGenericRecords() and parseAllGenericRecords(). - PCollection<String> paths = - readPipeline.apply( - "Create paths", - Create.of( - tmpFolder.getRoot().getAbsolutePath() + "/first*", - tmpFolder.getRoot().getAbsolutePath() + "/second*")); - PAssert.that( - paths - .apply("MatchAllReadFiles", FileIO.matchAll()) - .apply("ReadMatchesReadFiles", FileIO.readMatches().withCompression(AUTO)) - .apply( - "ReadFiles", - AvroIO.readFiles(GenericClass.class) - .withBeamSchemas(withBeamSchemas) - .withDesiredBundleSizeBytes(10))) - .containsInAnyOrder(Iterables.concat(firstValues, secondValues)); - PAssert.that( - paths.apply( - "ReadAll", - AvroIO.readAll(GenericClass.class) - .withBeamSchemas(withBeamSchemas) - .withDesiredBundleSizeBytes(10))) - .containsInAnyOrder(Iterables.concat(firstValues, secondValues)); - PAssert.that( - paths - .apply("MatchAllParseFilesGenericRecords", FileIO.matchAll()) - .apply( - "ReadMatchesParseFilesGenericRecords", - FileIO.readMatches() - .withDirectoryTreatment(FileIO.ReadMatches.DirectoryTreatment.PROHIBIT)) - .apply( - "ParseFilesGenericRecords", - AvroIO.parseFilesGenericRecords(new ParseGenericClass()) - .withCoder(AvroCoder.of(GenericClass.class)) - .withDesiredBundleSizeBytes(10))) - .containsInAnyOrder(Iterables.concat(firstValues, secondValues)); - PAssert.that( - paths.apply( - "ParseAllGenericRecords", - AvroIO.parseAllGenericRecords(new ParseGenericClass()) - .withCoder(AvroCoder.of(GenericClass.class)) - .withDesiredBundleSizeBytes(10))) - .containsInAnyOrder(Iterables.concat(firstValues, secondValues)); - - readPipeline.run(); - } - - private static class CreateGenericClass extends SimpleFunction<Long, GenericClass> { - @Override - public GenericClass apply(Long i) { - return new GenericClass(i.intValue(), "value" + i); - } - } - - @Test - @Category({NeedsRunner.class, UsesUnboundedSplittableParDo.class}) - public void testContinuouslyWriteAndReadMultipleFilepatterns() { - SimpleFunction<Long, GenericClass> mapFn = new CreateGenericClass(); - List<GenericClass> firstValues = new ArrayList<>(); - List<GenericClass> secondValues = new ArrayList<>(); - for (int i = 0; i < 7; ++i) { - (i < 3 ? firstValues : secondValues).add(mapFn.apply((long) i)); - } - // Configure windowing of the input so that it fires every time a new element is generated, - // so that files are written continuously. - Window<Long> window = - Window.<Long>into(FixedWindows.of(Duration.millis(100))) - .withAllowedLateness(Duration.ZERO) - .triggering(Repeatedly.forever(AfterPane.elementCountAtLeast(1))) - .discardingFiredPanes(); - readPipeline - .apply("Sequence first", GenerateSequence.from(0).to(3).withRate(1, Duration.millis(300))) - .apply("Window first", window) - .apply("Map first", MapElements.via(mapFn)) - .apply( - "Write first", - AvroIO.write(GenericClass.class) - .to(tmpFolder.getRoot().getAbsolutePath() + "/first") - .withNumShards(2) - .withWindowedWrites()); - readPipeline - .apply( - "Sequence second", GenerateSequence.from(3).to(7).withRate(1, Duration.millis(300))) - .apply("Window second", window) - .apply("Map second", MapElements.via(mapFn)) - .apply( - "Write second", - AvroIO.write(GenericClass.class) - .to(tmpFolder.getRoot().getAbsolutePath() + "/second") - .withNumShards(3) - .withWindowedWrites()); - - // Test read(), readFiles(), readAll(), parse(), parseFilesGenericRecords() and - // parseAllGenericRecords() with watchForNewFiles(). - PAssert.that( - readPipeline.apply( - "Read", - AvroIO.read(GenericClass.class) - .withBeamSchemas(withBeamSchemas) - .from(tmpFolder.getRoot().getAbsolutePath() + "/first*") - .watchForNewFiles( - Duration.millis(100), - Watch.Growth.afterTimeSinceNewOutput(Duration.standardSeconds(3))))) - .containsInAnyOrder(firstValues); - PAssert.that( - readPipeline.apply( - "Parse", - AvroIO.parseGenericRecords(new ParseGenericClass()) - .from(tmpFolder.getRoot().getAbsolutePath() + "/first*") - .watchForNewFiles( - Duration.millis(100), - Watch.Growth.afterTimeSinceNewOutput(Duration.standardSeconds(3))))) - .containsInAnyOrder(firstValues); - - PCollection<String> paths = - readPipeline.apply( - "Create paths", - Create.of( - tmpFolder.getRoot().getAbsolutePath() + "/first*", - tmpFolder.getRoot().getAbsolutePath() + "/second*")); - PAssert.that( - paths - .apply( - "Match All Read files", - FileIO.matchAll() - .continuously( - Duration.millis(100), - Watch.Growth.afterTimeSinceNewOutput(Duration.standardSeconds(3)))) - .apply( - "Read Matches Read files", - FileIO.readMatches() - .withDirectoryTreatment(FileIO.ReadMatches.DirectoryTreatment.PROHIBIT)) - .apply( - "Read files", - AvroIO.readFiles(GenericClass.class) - .withBeamSchemas(withBeamSchemas) - .withDesiredBundleSizeBytes(10))) - .containsInAnyOrder(Iterables.concat(firstValues, secondValues)); - PAssert.that( - paths.apply( - "Read all", - AvroIO.readAll(GenericClass.class) - .withBeamSchemas(withBeamSchemas) - .watchForNewFiles( - Duration.millis(100), - Watch.Growth.afterTimeSinceNewOutput(Duration.standardSeconds(3))) - .withDesiredBundleSizeBytes(10))) - .containsInAnyOrder(Iterables.concat(firstValues, secondValues)); - PAssert.that( - paths - .apply( - "Match All ParseFilesGenericRecords", - FileIO.matchAll() - .continuously( - Duration.millis(100), - Watch.Growth.afterTimeSinceNewOutput(Duration.standardSeconds(3)))) - .apply( - "Match Matches ParseFilesGenericRecords", - FileIO.readMatches() - .withDirectoryTreatment(FileIO.ReadMatches.DirectoryTreatment.PROHIBIT)) - .apply( - "ParseFilesGenericRecords", - AvroIO.parseFilesGenericRecords(new ParseGenericClass()) - .withCoder(AvroCoder.of(GenericClass.class)) - .withDesiredBundleSizeBytes(10))) - .containsInAnyOrder(Iterables.concat(firstValues, secondValues)); - PAssert.that( - paths.apply( - "ParseAllGenericRecords", - AvroIO.parseAllGenericRecords(new ParseGenericClass()) - .withCoder(AvroCoder.of(GenericClass.class)) - .watchForNewFiles( - Duration.millis(100), - Watch.Growth.afterTimeSinceNewOutput(Duration.standardSeconds(3))) - .withDesiredBundleSizeBytes(10))) - .containsInAnyOrder(Iterables.concat(firstValues, secondValues)); - readPipeline.run(); - } - - @Test - @SuppressWarnings("unchecked") - @Category(NeedsRunner.class) - public void testCompressedWriteAndReadASingleFile() throws Throwable { - List<GenericClass> values = - ImmutableList.of(new GenericClass(3, "hi"), new GenericClass(5, "bar")); - File outputFile = tmpFolder.newFile("output.avro"); - - writePipeline - .apply(Create.of(values)) - .apply( - AvroIO.write(GenericClass.class) - .to(outputFile.getAbsolutePath()) - .withoutSharding() - .withCodec(CodecFactory.deflateCodec(9))); - writePipeline.run(); - - PAssert.that( - readPipeline.apply( - AvroIO.read(GenericClass.class) - .withBeamSchemas(withBeamSchemas) - .from(outputFile.getAbsolutePath()))) - .containsInAnyOrder(values); - readPipeline.run(); - - try (DataFileStream dataFileStream = - new DataFileStream(new FileInputStream(outputFile), new GenericDatumReader())) { - assertEquals("deflate", dataFileStream.getMetaString("avro.codec")); - } - } - - @Test - @SuppressWarnings("unchecked") - @Category(NeedsRunner.class) - public void testWriteThenReadASingleFileWithNullCodec() throws Throwable { - List<GenericClass> values = - ImmutableList.of(new GenericClass(3, "hi"), new GenericClass(5, "bar")); - File outputFile = tmpFolder.newFile("output.avro"); - - writePipeline - .apply(Create.of(values)) - .apply( - AvroIO.write(GenericClass.class) - .to(outputFile.getAbsolutePath()) - .withoutSharding() - .withCodec(CodecFactory.nullCodec())); - writePipeline.run(); - - PAssert.that( - readPipeline.apply( - AvroIO.read(GenericClass.class) - .withBeamSchemas(withBeamSchemas) - .from(outputFile.getAbsolutePath()))) - .containsInAnyOrder(values); - readPipeline.run(); - - try (DataFileStream dataFileStream = - new DataFileStream(new FileInputStream(outputFile), new GenericDatumReader())) { - assertEquals("null", dataFileStream.getMetaString("avro.codec")); - } - } - - @DefaultCoder(AvroCoder.class) - static class GenericClassV2 { - int intField; - String stringField; - @org.apache.avro.reflect.Nullable String nullableField; - - GenericClassV2() {} - - GenericClassV2(int intValue, String stringValue, String nullableValue) { - this.intField = intValue; - this.stringField = stringValue; - this.nullableField = nullableValue; - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(getClass()) - .add("intField", intField) - .add("stringField", stringField) - .add("nullableField", nullableField) - .toString(); - } - - @Override - public int hashCode() { - return Objects.hash(intField, stringField, nullableField); - } - - @Override - public boolean equals(@Nullable Object other) { - if (!(other instanceof GenericClassV2)) { - return false; - } - GenericClassV2 o = (GenericClassV2) other; - return intField == o.intField - && Objects.equals(stringField, o.stringField) - && Objects.equals(nullableField, o.nullableField); - } - } - - /** - * Tests that {@code AvroIO} can read an upgraded version of an old class, as long as the schema - * resolution process succeeds. This test covers the case when a new, {@code @Nullable} field - * has been added. - * - * <p>For more information, see http://avro.apache.org/docs/1.7.7/spec.html#Schema+Resolution - */ - @Test - @Category(NeedsRunner.class) - public void testWriteThenReadSchemaUpgrade() throws Throwable { - List<GenericClass> values = - ImmutableList.of(new GenericClass(3, "hi"), new GenericClass(5, "bar")); - File outputFile = tmpFolder.newFile("output.avro"); - - writePipeline - .apply(Create.of(values)) - .apply( - AvroIO.write(GenericClass.class).to(outputFile.getAbsolutePath()).withoutSharding()); - writePipeline.run(); - - List<GenericClassV2> expected = - ImmutableList.of(new GenericClassV2(3, "hi", null), new GenericClassV2(5, "bar", null)); - - PAssert.that( - readPipeline.apply( - AvroIO.read(GenericClassV2.class) - .withBeamSchemas(withBeamSchemas) - .from(outputFile.getAbsolutePath()))) - .containsInAnyOrder(expected); - readPipeline.run(); - } - - private static class WindowedFilenamePolicy extends FilenamePolicy { - final ResourceId outputFilePrefix; - - WindowedFilenamePolicy(ResourceId outputFilePrefix) { - this.outputFilePrefix = outputFilePrefix; - } - - @Override - public ResourceId windowedFilename( - int shardNumber, - int numShards, - BoundedWindow window, - PaneInfo paneInfo, - OutputFileHints outputFileHints) { - String filenamePrefix = - outputFilePrefix.isDirectory() ? "" : firstNonNull(outputFilePrefix.getFilename(), ""); - - IntervalWindow interval = (IntervalWindow) window; - String windowStr = - String.format("%s-%s", interval.start().toString(), interval.end().toString()); - String filename = - String.format( - "%s-%s-%s-of-%s-pane-%s%s%s.avro", - filenamePrefix, - windowStr, - shardNumber, - numShards, - paneInfo.getIndex(), - paneInfo.isLast() ? "-last" : "", - outputFileHints.getSuggestedFilenameSuffix()); - return outputFilePrefix.getCurrentDirectory().resolve(filename, RESOLVE_FILE); - } - - @Override - public ResourceId unwindowedFilename( - int shardNumber, int numShards, OutputFileHints outputFileHints) { - throw new UnsupportedOperationException("Expecting windowed outputs only"); - } - - @Override - public void populateDisplayData(DisplayData.Builder builder) { - builder.add( - DisplayData.item("fileNamePrefix", outputFilePrefix.toString()) - .withLabel("File Name Prefix")); - } - } - - @Test - @Category({NeedsRunner.class, UsesTestStream.class}) - public void testWriteWindowed() throws Throwable { - testWindowedAvroIOWriteUsingMethod(WriteMethod.AVROIO_WRITE); - } - - @Test - @Category({NeedsRunner.class, UsesTestStream.class}) - public void testWindowedAvroIOWriteViaSink() throws Throwable { - testWindowedAvroIOWriteUsingMethod(WriteMethod.AVROIO_SINK_WITH_CLASS); - } - - void testWindowedAvroIOWriteUsingMethod(WriteMethod method) throws IOException { - Path baseDir = Files.createTempDirectory(tmpFolder.getRoot().toPath(), "testwrite"); - final String baseFilename = baseDir.resolve("prefix").toString(); - - Instant base = new Instant(0); - ArrayList<GenericClass> allElements = new ArrayList<>(); - ArrayList<TimestampedValue<GenericClass>> firstWindowElements = new ArrayList<>(); - ArrayList<Instant> firstWindowTimestamps = - Lists.newArrayList( - base.plus(Duration.ZERO), base.plus(Duration.standardSeconds(10)), - base.plus(Duration.standardSeconds(20)), base.plus(Duration.standardSeconds(30))); - - Random random = new Random(); - for (int i = 0; i < 100; ++i) { - GenericClass item = new GenericClass(i, String.valueOf(i)); - allElements.add(item); - firstWindowElements.add( - TimestampedValue.of( - item, firstWindowTimestamps.get(random.nextInt(firstWindowTimestamps.size())))); - } - - ArrayList<TimestampedValue<GenericClass>> secondWindowElements = new ArrayList<>(); - ArrayList<Instant> secondWindowTimestamps = - Lists.newArrayList( - base.plus(Duration.standardSeconds(60)), base.plus(Duration.standardSeconds(70)), - base.plus(Duration.standardSeconds(80)), base.plus(Duration.standardSeconds(90))); - for (int i = 100; i < 200; ++i) { - GenericClass item = new GenericClass(i, String.valueOf(i)); - allElements.add(new GenericClass(i, String.valueOf(i))); - secondWindowElements.add( - TimestampedValue.of( - item, secondWindowTimestamps.get(random.nextInt(secondWindowTimestamps.size())))); - } - - TimestampedValue<GenericClass>[] firstWindowArray = - firstWindowElements.toArray(new TimestampedValue[100]); - TimestampedValue<GenericClass>[] secondWindowArray = - secondWindowElements.toArray(new TimestampedValue[100]); - - TestStream<GenericClass> values = - TestStream.create(AvroCoder.of(GenericClass.class)) - .advanceWatermarkTo(new Instant(0)) - .addElements( - firstWindowArray[0], - Arrays.copyOfRange(firstWindowArray, 1, firstWindowArray.length)) - .advanceWatermarkTo(new Instant(0).plus(Duration.standardMinutes(1))) - .addElements( - secondWindowArray[0], - Arrays.copyOfRange(secondWindowArray, 1, secondWindowArray.length)) - .advanceWatermarkToInfinity(); - - final PTransform<PCollection<GenericClass>, WriteFilesResult<Void>> write; - switch (method) { - case AVROIO_WRITE: - { - FilenamePolicy policy = - new WindowedFilenamePolicy( - FileBasedSink.convertToFileResourceIfPossible(baseFilename)); - write = - AvroIO.write(GenericClass.class) - .to(policy) - .withTempDirectory( - StaticValueProvider.of( - FileSystems.matchNewResource(baseDir.toString(), true))) - .withWindowedWrites() - .withNumShards(2) - .withOutputFilenames(); - break; - } - - case AVROIO_SINK_WITH_CLASS: - { - write = - FileIO.<GenericClass>write() - .via(AvroIO.sink(GenericClass.class)) - .to(baseDir.toString()) - .withPrefix("prefix") - .withSuffix(".avro") - .withTempDirectory(baseDir.toString()) - .withNumShards(2); - break; - } - - default: - throw new UnsupportedOperationException(); - } - windowedAvroWritePipeline - .apply(values) - .apply(Window.into(FixedWindows.of(Duration.standardMinutes(1)))) - .apply(write); - windowedAvroWritePipeline.run(); - - // Validate that the data written matches the expected elements in the expected order - List<File> expectedFiles = new ArrayList<>(); - for (int shard = 0; shard < 2; shard++) { - for (int window = 0; window < 2; window++) { - Instant windowStart = new Instant(0).plus(Duration.standardMinutes(window)); - IntervalWindow iw = new IntervalWindow(windowStart, Duration.standardMinutes(1)); - String baseAndWindow = baseFilename + "-" + iw.start() + "-" + iw.end(); - switch (method) { - case AVROIO_WRITE: - expectedFiles.add(new File(baseAndWindow + "-" + shard + "-of-2-pane-0-last.avro")); - break; - case AVROIO_SINK_WITH_CLASS: - expectedFiles.add(new File(baseAndWindow + "-0000" + shard + "-of-00002.avro")); - break; - default: - throw new UnsupportedOperationException("Unknown write method " + method); - } - } - } - - List<GenericClass> actualElements = new ArrayList<>(); - for (File outputFile : expectedFiles) { - assertTrue("Expected output file " + outputFile.getAbsolutePath(), outputFile.exists()); - try (DataFileReader<GenericClass> reader = - new DataFileReader<>( - outputFile, - new ReflectDatumReader<>(ReflectData.get().getSchema(GenericClass.class)))) { - Iterators.addAll(actualElements, reader); - } - outputFile.delete(); - } - assertThat(actualElements, containsInAnyOrder(allElements.toArray())); - } - - private static final String SCHEMA_TEMPLATE_STRING = - "{\"namespace\": \"example.avro\",\n" - + " \"type\": \"record\",\n" - + " \"name\": \"$$TestTemplateSchema\",\n" - + " \"fields\": [\n" - + " {\"name\": \"$$full\", \"type\": \"string\"},\n" - + " {\"name\": \"$$suffix\", \"type\": [\"string\", \"null\"]}\n" - + " ]\n" - + "}"; - - private static String schemaFromPrefix(String prefix) { - return SCHEMA_TEMPLATE_STRING.replace("$$", prefix); - } - - private static GenericRecord createRecord(String record, String prefix, Schema schema) { - GenericRecord genericRecord = new GenericData.Record(schema); - genericRecord.put(prefix + "full", record); - genericRecord.put(prefix + "suffix", record.substring(1)); - return genericRecord; - } - - private static class TestDynamicDestinations - extends DynamicAvroDestinations<String, String, GenericRecord> { - final ResourceId baseDir; - final PCollectionView<Map<String, String>> schemaView; - - TestDynamicDestinations(ResourceId baseDir, PCollectionView<Map<String, String>> schemaView) { - this.baseDir = baseDir; - this.schemaView = schemaView; - } - - @Override - public Schema getSchema(String destination) { - // Return a per-destination schema. - String schema = sideInput(schemaView).get(destination); - return new Schema.Parser().parse(schema); - } - - @Override - public List<PCollectionView<?>> getSideInputs() { - return ImmutableList.of(schemaView); - } - - @Override - public GenericRecord formatRecord(String record) { - String prefix = record.substring(0, 1); - return createRecord(record, prefix, getSchema(prefix)); - } - - @Override - public String getDestination(String element) { - // Destination is based on first character of string. - return element.substring(0, 1); - } - - @Override - public String getDefaultDestination() { - return ""; - } - - @Override - public FilenamePolicy getFilenamePolicy(String destination) { - return DefaultFilenamePolicy.fromStandardParameters( - StaticValueProvider.of(baseDir.resolve("file_" + destination, RESOLVE_FILE)), - "-SSSSS-of-NNNNN", - ".avro", - false); - } - } - - /** - * Example of a {@link Coder} for a collection of Avro records with different schemas. - * - * <p>All the schemas are known at pipeline construction, and are keyed internally on the prefix - * character (lower byte only for UTF-8 data). - */ - private static class AvroMultiplexCoder extends Coder<GenericRecord> { - - /** Lookup table for the possible schemas, keyed on the prefix character. */ - private final Map<Character, AvroCoder<GenericRecord>> coderMap = Maps.newHashMap(); - - protected AvroMultiplexCoder(Map<String, String> schemaMap) { - for (Map.Entry<String, String> entry : schemaMap.entrySet()) { - coderMap.put( - entry.getKey().charAt(0), AvroCoder.of(new Schema.Parser().parse(entry.getValue()))); - } - } - - @Override - public void encode(GenericRecord value, OutputStream outStream) throws IOException { - char prefix = value.getSchema().getName().charAt(0); - outStream.write(prefix); // Only reads and writes the low byte. - coderMap.get(prefix).encode(value, outStream); - } - - @Override - public GenericRecord decode(InputStream inStream) throws CoderException, IOException { - char prefix = (char) inStream.read(); - return coderMap.get(prefix).decode(inStream); - } - - @Override - public List<? extends Coder<?>> getCoderArguments() { - return Collections.emptyList(); - } - - @Override - public void verifyDeterministic() throws NonDeterministicException { - for (AvroCoder<GenericRecord> internalCoder : coderMap.values()) { - internalCoder.verifyDeterministic(); - } - } - } - - private void testDynamicDestinationsUnwindowedWithSharding( - WriteMethod writeMethod, Sharding sharding) throws Exception { - final ResourceId baseDir = - FileSystems.matchNewResource( - Files.createTempDirectory(tmpFolder.getRoot().toPath(), "testDynamicDestinations") - .toString(), - true); - - List<String> elements = Lists.newArrayList("aaaa", "aaab", "baaa", "baab", "caaa", "caab"); - Multimap<String, GenericRecord> expectedElements = ArrayListMultimap.create(); - Map<String, String> schemaMap = Maps.newHashMap(); - for (String element : elements) { - String prefix = element.substring(0, 1); - String jsonSchema = schemaFromPrefix(prefix); - schemaMap.put(prefix, jsonSchema); - expectedElements.put( - prefix, createRecord(element, prefix, new Schema.Parser().parse(jsonSchema))); - } - final PCollectionView<Map<String, String>> schemaView = - writePipeline.apply("createSchemaView", Create.of(schemaMap)).apply(View.asMap()); - - PCollection<String> input = - writePipeline.apply("createInput", Create.of(elements).withCoder(StringUtf8Coder.of())); - - switch (writeMethod) { - case AVROIO_WRITE: - { - AvroIO.TypedWrite<String, String, GenericRecord> write = - AvroIO.<String>writeCustomTypeToGenericRecords() - .to(new TestDynamicDestinations(baseDir, schemaView)) - .withTempDirectory(baseDir); - - switch (sharding) { - case RUNNER_DETERMINED: - break; - case WITHOUT_SHARDING: - write = write.withoutSharding(); - break; - case FIXED_3_SHARDS: - write = write.withNumShards(3); - break; - default: - throw new IllegalArgumentException("Unknown sharding " + sharding); - } - - input.apply(write); - break; - } - - case AVROIO_SINK_WITH_SCHEMA: - { - FileIO.Write<String, GenericRecord> write = - FileIO.<String, GenericRecord>writeDynamic() - .by( - fn( - (element, c) -> { - c.sideInput(schemaView); // Ignore result - return element.getSchema().getName().substring(0, 1); - }, - requiresSideInputs(schemaView))) - .via( - fn( - (dest, c) -> { - Schema schema = - new Schema.Parser().parse(c.sideInput(schemaView).get(dest)); - return AvroIO.sink(schema); - }, - requiresSideInputs(schemaView))) - .to(baseDir.toString()) - .withNaming( - fn( - (dest, c) -> { - c.sideInput(schemaView); // Ignore result - return FileIO.Write.defaultNaming("file_" + dest, ".avro"); - }, - requiresSideInputs(schemaView))) - .withTempDirectory(baseDir.toString()) - .withDestinationCoder(StringUtf8Coder.of()) - .withIgnoreWindowing(); - switch (sharding) { - case RUNNER_DETERMINED: - break; - case WITHOUT_SHARDING: - write = write.withNumShards(1); - break; - case FIXED_3_SHARDS: - write = write.withNumShards(3); - break; - default: - throw new IllegalArgumentException("Unknown sharding " + sharding); - } - - MapElements<String, GenericRecord> toRecord = - MapElements.via( - new SimpleFunction<String, GenericRecord>() { - @Override - public GenericRecord apply(String element) { - String prefix = element.substring(0, 1); - GenericRecord record = - new GenericData.Record( - new Schema.Parser().parse(schemaFromPrefix(prefix))); - record.put(prefix + "full", element); - record.put(prefix + "suffix", element.substring(1)); - return record; - } - }); - - input.apply(toRecord).setCoder(new AvroMultiplexCoder(schemaMap)).apply(write); - break; - } - - case AVROIO_SINK_WITH_FORMATTER: - { - final AvroIO.RecordFormatter<String> formatter = - (element, schema) -> { - String prefix = element.substring(0, 1); - GenericRecord record = new GenericData.Record(schema); - record.put(prefix + "full", element); - record.put(prefix + "suffix", element.substring(1)); - return record; - }; - FileIO.Write<String, String> write = - FileIO.<String, String>writeDynamic() - .by( - fn( - (element, c) -> { - c.sideInput(schemaView); // Ignore result - return element.substring(0, 1); - }, - requiresSideInputs(schemaView))) - .via( - fn( - (dest, c) -> { - Schema schema = - new Schema.Parser().parse(c.sideInput(schemaView).get(dest)); - return AvroIO.sinkViaGenericRecords(schema, formatter); - }, - requiresSideInputs(schemaView))) - .to(baseDir.toString()) - .withNaming( - fn( - (dest, c) -> { - c.sideInput(schemaView); // Ignore result - return FileIO.Write.defaultNaming("file_" + dest, ".avro"); - }, - requiresSideInputs(schemaView))) - .withTempDirectory(baseDir.toString()) - .withDestinationCoder(StringUtf8Coder.of()) - .withIgnoreWindowing(); - switch (sharding) { - case RUNNER_DETERMINED: - break; - case WITHOUT_SHARDING: - write = write.withNumShards(1); - break; - case FIXED_3_SHARDS: - write = write.withNumShards(3); - break; - default: - throw new IllegalArgumentException("Unknown sharding " + sharding); - } - - input.apply(write); - break; - } - default: - throw new UnsupportedOperationException("Unknown write method " + writeMethod); - } - - writePipeline.run(); - - // Validate that the data written matches the expected elements in the expected order. - - for (String prefix : expectedElements.keySet()) { - String shardPattern; - switch (sharding) { - case RUNNER_DETERMINED: - shardPattern = "-*"; - break; - case WITHOUT_SHARDING: - shardPattern = "-00000-of-00001"; - break; - case FIXED_3_SHARDS: - shardPattern = "-*-of-00003"; - break; - default: - throw new IllegalArgumentException("Unknown sharding " + sharding); - } - String expectedFilepattern = - baseDir.resolve("file_" + prefix + shardPattern + ".avro", RESOLVE_FILE).toString(); - - PCollection<GenericRecord> records = - readPipeline.apply( - "read_" + prefix, - AvroIO.readGenericRecords(schemaFromPrefix(prefix)) - .withBeamSchemas(withBeamSchemas) - .from(expectedFilepattern)); - PAssert.that(records).containsInAnyOrder(expectedElements.get(prefix)); - } - readPipeline.run(); - } - - @Test - @Category(NeedsRunner.class) - public void testDynamicDestinationsRunnerDeterminedSharding() throws Exception { - testDynamicDestinationsUnwindowedWithSharding( - WriteMethod.AVROIO_WRITE, Sharding.RUNNER_DETERMINED); - } - - @Test - @Category(NeedsRunner.class) - public void testDynamicDestinationsWithoutSharding() throws Exception { - testDynamicDestinationsUnwindowedWithSharding( - WriteMethod.AVROIO_WRITE, Sharding.WITHOUT_SHARDING); - } - - @Test - @Category(NeedsRunner.class) - public void testDynamicDestinationsWithNumShards() throws Exception { - testDynamicDestinationsUnwindowedWithSharding( - WriteMethod.AVROIO_WRITE, Sharding.FIXED_3_SHARDS); - } - - @Test - @Category(NeedsRunner.class) - public void testDynamicDestinationsViaSinkRunnerDeterminedSharding() throws Exception { - testDynamicDestinationsUnwindowedWithSharding( - WriteMethod.AVROIO_SINK_WITH_SCHEMA, Sharding.RUNNER_DETERMINED); - } - - @Test - @Category(NeedsRunner.class) - public void testDynamicDestinationsViaSinkWithoutSharding() throws Exception { - testDynamicDestinationsUnwindowedWithSharding( - WriteMethod.AVROIO_SINK_WITH_SCHEMA, Sharding.WITHOUT_SHARDING); - } - - @Test - @Category(NeedsRunner.class) - public void testDynamicDestinationsViaSinkWithNumShards() throws Exception { - testDynamicDestinationsUnwindowedWithSharding( - WriteMethod.AVROIO_SINK_WITH_SCHEMA, Sharding.FIXED_3_SHARDS); - } - - @Test - @Category(NeedsRunner.class) - public void testDynamicDestinationsViaSinkWithFormatterRunnerDeterminedSharding() - throws Exception { - testDynamicDestinationsUnwindowedWithSharding( - WriteMethod.AVROIO_SINK_WITH_FORMATTER, Sharding.RUNNER_DETERMINED); - } - - @Test - @Category(NeedsRunner.class) - public void testDynamicDestinationsViaSinkWithFormatterWithoutSharding() throws Exception { - testDynamicDestinationsUnwindowedWithSharding( - WriteMethod.AVROIO_SINK_WITH_FORMATTER, Sharding.WITHOUT_SHARDING); - } - - @Test - @Category(NeedsRunner.class) - public void testDynamicDestinationsViaSinkWithFormatterWithNumShards() throws Exception { - testDynamicDestinationsUnwindowedWithSharding( - WriteMethod.AVROIO_SINK_WITH_FORMATTER, Sharding.FIXED_3_SHARDS); - } - - @Test - @SuppressWarnings("unchecked") - @Category(NeedsRunner.class) - public void testMetadata() throws Exception { - List<GenericClass> values = - ImmutableList.of(new GenericClass(3, "hi"), new GenericClass(5, "bar")); - File outputFile = tmpFolder.newFile("output.avro"); - - writePipeline - .apply(Create.of(values)) - .apply( - AvroIO.write(GenericClass.class) - .to(outputFile.getAbsolutePath()) - .withoutSharding() - .withMetadata( - ImmutableMap.of( - "stringKey", - "stringValue", - "longKey", - 100L, - "bytesKey", - "bytesValue".getBytes(Charsets.UTF_8)))); - writePipeline.run(); - - try (DataFileStream dataFileStream = - new DataFileStream(new FileInputStream(outputFile), new GenericDatumReader())) { - assertEquals("stringValue", dataFileStream.getMetaString("stringKey")); - assertEquals(100L, dataFileStream.getMetaLong("longKey")); - assertArrayEquals( - "bytesValue".getBytes(Charsets.UTF_8), dataFileStream.getMeta("bytesKey")); - } - } - - // using AvroCoder#createDatumReader for tests. - private void runTestWrite(String[] expectedElements, int numShards) throws IOException { - File baseOutputFile = new File(tmpFolder.getRoot(), "prefix"); - String outputFilePrefix = baseOutputFile.getAbsolutePath(); - - AvroIO.Write<String> write = - AvroIO.write(String.class).to(outputFilePrefix).withSuffix(".avro"); - if (numShards > 1) { - write = write.withNumShards(numShards); - } else { - write = write.withoutSharding(); - } - writePipeline.apply(Create.of(ImmutableList.copyOf(expectedElements))).apply(write); - writePipeline.run(); - - String shardNameTemplate = - firstNonNull( - write.inner.getShardTemplate(), - DefaultFilenamePolicy.DEFAULT_UNWINDOWED_SHARD_TEMPLATE); - - assertTestOutputs(expectedElements, numShards, outputFilePrefix, shardNameTemplate); - } - - static void assertTestOutputs( - String[] expectedElements, int numShards, String outputFilePrefix, String shardNameTemplate) - throws IOException { - // Validate that the data written matches the expected elements in the expected order - List<File> expectedFiles = new ArrayList<>(); - for (int i = 0; i < numShards; i++) { - expectedFiles.add( - new File( - DefaultFilenamePolicy.constructName( - FileBasedSink.convertToFileResourceIfPossible(outputFilePrefix), - shardNameTemplate, - ".avro", - i, - numShards, - null, - null) - .toString())); - } - - List<String> actualElements = new ArrayList<>(); - for (File outputFile : expectedFiles) { - assertTrue("Expected output file " + outputFile.getName(), outputFile.exists()); - try (DataFileReader<String> reader = - new DataFileReader<>( - outputFile, new ReflectDatumReader(ReflectData.get().getSchema(String.class)))) { - Iterators.addAll(actualElements, reader); - } - } - assertThat(actualElements, containsInAnyOrder(expectedElements)); - } - - @Test - @Category(NeedsRunner.class) - public void testAvroSinkWrite() throws Exception { - String[] expectedElements = new String[] {"first", "second", "third"}; - - runTestWrite(expectedElements, 1); - } - - @Test - @Category(NeedsRunner.class) - public void testAvroSinkShardedWrite() throws Exception { - String[] expectedElements = new String[] {"first", "second", "third", "fourth", "fifth"}; - - runTestWrite(expectedElements, 4); - } - - @Test - @Category(NeedsRunner.class) - public void testAvroSinkWriteWithCustomFactory() throws Exception { - Integer[] expectedElements = new Integer[] {1, 2, 3, 4, 5}; - - File baseOutputFile = new File(tmpFolder.getRoot(), "prefix"); - String outputFilePrefix = baseOutputFile.getAbsolutePath(); - - Schema recordSchema = SchemaBuilder.record("root").fields().requiredInt("i1").endRecord(); - - AvroIO.TypedWrite<Integer, Void, Integer> write = - AvroIO.<Integer, Integer>writeCustomType() - .to(outputFilePrefix) - .withSchema(recordSchema) - .withFormatFunction(f -> f) - .withDatumWriterFactory( - f -> - new DatumWriter<Integer>() { - private DatumWriter<GenericRecord> inner = new GenericDatumWriter<>(f); - - @Override - public void setSchema(Schema schema) { - inner.setSchema(schema); - } - - @Override - public void write(Integer datum, Encoder out) throws IOException { - GenericRecord record = - new GenericRecordBuilder(f).set("i1", datum).build(); - inner.write(record, out); - } - }) - .withSuffix(".avro"); - - write = write.withoutSharding(); - - writePipeline.apply(Create.of(ImmutableList.copyOf(expectedElements))).apply(write); - writePipeline.run(); - - File expectedFile = - new File( - DefaultFilenamePolicy.constructName( - FileBasedSink.convertToFileResourceIfPossible(outputFilePrefix), - "", - ".avro", - 1, - 1, - null, - null) - .toString()); - - assertTrue("Expected output file " + expectedFile.getName(), expectedFile.exists()); - DataFileReader<GenericRecord> dataFileReader = - new DataFileReader<>(expectedFile, new GenericDatumReader<>(recordSchema)); - - List<GenericRecord> actualRecords = new ArrayList<>(); - Iterators.addAll(actualRecords, dataFileReader); - - GenericRecord[] expectedRecords = - Arrays.stream(expectedElements) - .map(i -> new GenericRecordBuilder(recordSchema).set("i1", i).build()) - .toArray(GenericRecord[]::new); - - assertThat(actualRecords, containsInAnyOrder(expectedRecords)); - } - - // TODO: for Write only, test withSuffix, - // withShardNameTemplate and withoutSharding. - } -} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroSchemaIOProviderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroSchemaIOProviderTest.java deleted file mode 100644 index cf68633dc7ae..000000000000 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroSchemaIOProviderTest.java +++ /dev/null @@ -1,173 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.io; - -import static org.junit.Assert.assertEquals; - -import java.io.File; -import java.time.Duration; -import java.util.Arrays; -import java.util.List; -import org.apache.beam.sdk.coders.RowCoder; -import org.apache.beam.sdk.io.fs.MatchResult; -import org.apache.beam.sdk.schemas.Schema; -import org.apache.beam.sdk.schemas.io.SchemaIO; -import org.apache.beam.sdk.testing.NeedsRunner; -import org.apache.beam.sdk.testing.PAssert; -import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.testing.TestStream; -import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.Row; -import org.apache.beam.sdk.values.TimestampedValue; -import org.joda.time.Instant; -import org.junit.Rule; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -/** Test for AvroSchemaIOProvider. */ -@RunWith(JUnit4.class) -public class AvroSchemaIOProviderTest { - @Rule public TestPipeline writePipeline = TestPipeline.create(); - @Rule public TestPipeline readPipeline = TestPipeline.create(); - @Rule public TemporaryFolder tempFolder = new TemporaryFolder(); - - private static final Schema SCHEMA = - Schema.builder().addInt64Field("age").addStringField("age_str").build(); - - private Row createRow(long l) { - return Row.withSchema(SCHEMA).addValues(l, Long.valueOf(l).toString()).build(); - } - - @Test - @Category({NeedsRunner.class}) - public void testWriteAndReadTable() { - File destinationFile = new File(tempFolder.getRoot(), "person-info.avro"); - - AvroSchemaIOProvider provider = new AvroSchemaIOProvider(); - Row configuration = Row.withSchema(provider.configurationSchema()).addValue(null).build(); - SchemaIO io = provider.from(destinationFile.getAbsolutePath(), configuration, SCHEMA); - - List<Row> rowsList = Arrays.asList(createRow(1L), createRow(3L), createRow(4L)); - PCollection<Row> rows = - writePipeline.apply("Create", Create.of(rowsList).withCoder(RowCoder.of(SCHEMA))); - rows.apply(io.buildWriter()); - writePipeline.run(); - - PCollection<Row> read = readPipeline.begin().apply(io.buildReader()); - PAssert.that(read).containsInAnyOrder(rowsList); - readPipeline.run(); - } - - @Test - @Category({NeedsRunner.class}) - public void testStreamingWriteDefault() throws Exception { - File destinationFile = new File(tempFolder.getRoot(), "person-info"); - - AvroSchemaIOProvider provider = new AvroSchemaIOProvider(); - Row config = Row.withSchema(provider.configurationSchema()).addValue(null).build(); - SchemaIO writeIO = provider.from(destinationFile.getAbsolutePath(), config, SCHEMA); - - TestStream<Row> createEvents = - TestStream.create(RowCoder.of(SCHEMA)) - .addElements(TimestampedValue.of(createRow(1L), new Instant(1L))) - .addElements(TimestampedValue.of(createRow(2L), Instant.ofEpochSecond(120L))) - .advanceWatermarkToInfinity(); - - writePipeline.apply("create", createEvents).apply("write", writeIO.buildWriter()); - writePipeline.run(); - - // Verify we wrote two files. - String wildcardPath = destinationFile.getAbsolutePath() + "*"; - MatchResult result = FileSystems.match(wildcardPath); - assertEquals(2, result.metadata().size()); - - // Verify results of the files. - SchemaIO readIO = provider.from(wildcardPath, config, SCHEMA); - PCollection<Row> read = readPipeline.begin().apply("read", readIO.buildReader()); - PAssert.that(read).containsInAnyOrder(createRow(1L), createRow(2L)); - readPipeline.run(); - } - - @Test - @Category({NeedsRunner.class}) - public void testStreamingCustomWindowSize() throws Exception { - File destinationFile = new File(tempFolder.getRoot(), "person-info"); - - AvroSchemaIOProvider provider = new AvroSchemaIOProvider(); - Row config = - Row.withSchema(provider.configurationSchema()) - .addValue(Duration.ofMinutes(4).getSeconds()) - .build(); - SchemaIO writeIO = provider.from(destinationFile.getAbsolutePath(), config, SCHEMA); - - TestStream<Row> createEvents = - TestStream.create(RowCoder.of(SCHEMA)) - .addElements(TimestampedValue.of(createRow(1L), new Instant(1L))) - .addElements(TimestampedValue.of(createRow(2L), Instant.ofEpochSecond(120L))) - .advanceWatermarkToInfinity(); - - writePipeline.apply("create", createEvents).apply("write", writeIO.buildWriter()); - writePipeline.run(); - - // Verify we wrote one file. - String wildcardPath = destinationFile.getAbsolutePath() + "*"; - MatchResult result = FileSystems.match(wildcardPath); - assertEquals(1, result.metadata().size()); - - // Verify results of the files. - SchemaIO readIO = provider.from(wildcardPath, config, SCHEMA); - PCollection<Row> read = readPipeline.begin().apply("read", readIO.buildReader()); - PAssert.that(read).containsInAnyOrder(createRow(1L), createRow(2L)); - readPipeline.run(); - } - - @Test - @Category({NeedsRunner.class}) - public void testBatchCustomWindowSize() throws Exception { - File destinationFile = new File(tempFolder.getRoot(), "person-info"); - - AvroSchemaIOProvider provider = new AvroSchemaIOProvider(); - Row config = - Row.withSchema(provider.configurationSchema()) - .addValue(Duration.ofMinutes(4).getSeconds()) - .build(); - SchemaIO writeIO = provider.from(destinationFile.getAbsolutePath(), config, SCHEMA); - - List<Row> rowsList = Arrays.asList(createRow(1L), createRow(3L), createRow(4L)); - PCollection<Row> rows = - writePipeline.apply("Create", Create.of(rowsList).withCoder(RowCoder.of(SCHEMA))); - - rows.apply("write", writeIO.buildWriter()); - writePipeline.run(); - - // Verify we wrote one file. - String wildcardPath = destinationFile.getAbsolutePath() + "*"; - MatchResult result = FileSystems.match(wildcardPath); - assertEquals(1, result.metadata().size()); - - // Verify results of the files. - SchemaIO readIO = provider.from(wildcardPath, config, SCHEMA); - PCollection<Row> read = readPipeline.begin().apply("read", readIO.buildReader()); - PAssert.that(read).containsInAnyOrder(rowsList); - readPipeline.run(); - } -} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroSourceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroSourceTest.java deleted file mode 100644 index 577fdb19f013..000000000000 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroSourceTest.java +++ /dev/null @@ -1,843 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.io; - -import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.containsInAnyOrder; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertSame; -import static org.junit.Assert.assertTrue; - -import java.io.File; -import java.io.FileOutputStream; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; -import java.util.NoSuchElementException; -import java.util.Objects; -import java.util.Random; -import java.util.stream.Collectors; -import org.apache.avro.Schema; -import org.apache.avro.file.CodecFactory; -import org.apache.avro.file.DataFileConstants; -import org.apache.avro.file.DataFileWriter; -import org.apache.avro.generic.GenericDatumReader; -import org.apache.avro.generic.GenericDatumWriter; -import org.apache.avro.generic.GenericRecord; -import org.apache.avro.io.DatumWriter; -import org.apache.avro.io.Decoder; -import org.apache.avro.reflect.AvroDefault; -import org.apache.avro.reflect.ReflectData; -import org.apache.avro.reflect.ReflectDatumWriter; -import org.apache.beam.sdk.coders.AvroCoder; -import org.apache.beam.sdk.coders.DefaultCoder; -import org.apache.beam.sdk.io.AvroSource.AvroMetadata; -import org.apache.beam.sdk.io.BlockBasedSource.BlockBasedReader; -import org.apache.beam.sdk.io.BoundedSource.BoundedReader; -import org.apache.beam.sdk.io.fs.MatchResult.Metadata; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.testing.SourceTestUtils; -import org.apache.beam.sdk.transforms.display.DisplayData; -import org.apache.beam.sdk.util.SerializableUtils; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; -import org.checkerframework.checker.nullness.qual.Nullable; -import org.hamcrest.Matchers; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -/** Tests for AvroSource. */ -@RunWith(JUnit4.class) -public class AvroSourceTest { - @Rule public TemporaryFolder tmpFolder = new TemporaryFolder(); - - @Rule public ExpectedException expectedException = ExpectedException.none(); - - private enum SyncBehavior { - SYNC_REGULAR, // Sync at regular, user defined intervals - SYNC_RANDOM, // Sync at random intervals - SYNC_DEFAULT // Sync at default intervals (i.e., no manual syncing). - } - - private static final int DEFAULT_RECORD_COUNT = 1000; - - /** - * Generates an input Avro file containing the given records in the temporary directory and - * returns the full path of the file. - */ - private <T> String generateTestFile( - String filename, - List<T> elems, - SyncBehavior syncBehavior, - int syncInterval, - AvroCoder<T> coder, - String codec) - throws IOException { - Random random = new Random(0); - File tmpFile = tmpFolder.newFile(filename); - String path = tmpFile.toString(); - - FileOutputStream os = new FileOutputStream(tmpFile); - DatumWriter<T> datumWriter = - coder.getType().equals(GenericRecord.class) - ? new GenericDatumWriter<>(coder.getSchema()) - : new ReflectDatumWriter<>(coder.getSchema()); - try (DataFileWriter<T> writer = new DataFileWriter<>(datumWriter)) { - writer.setCodec(CodecFactory.fromString(codec)); - writer.create(coder.getSchema(), os); - - int recordIndex = 0; - int syncIndex = syncBehavior == SyncBehavior.SYNC_RANDOM ? random.nextInt(syncInterval) : 0; - - for (T elem : elems) { - writer.append(elem); - recordIndex++; - - switch (syncBehavior) { - case SYNC_REGULAR: - if (recordIndex == syncInterval) { - recordIndex = 0; - writer.sync(); - } - break; - case SYNC_RANDOM: - if (recordIndex == syncIndex) { - recordIndex = 0; - writer.sync(); - syncIndex = random.nextInt(syncInterval); - } - break; - case SYNC_DEFAULT: - default: - } - } - } - return path; - } - - @Test - public void testReadWithDifferentCodecs() throws Exception { - // Test reading files generated using all codecs. - String[] codecs = { - DataFileConstants.NULL_CODEC, - DataFileConstants.BZIP2_CODEC, - DataFileConstants.DEFLATE_CODEC, - DataFileConstants.SNAPPY_CODEC, - DataFileConstants.XZ_CODEC, - }; - // As Avro's default block size is 64KB, write 64K records to ensure at least one full block. - // We could make this smaller than 64KB assuming each record is at least B bytes, but then the - // test could silently stop testing the failure condition from BEAM-422. - List<Bird> expected = createRandomRecords(1 << 16); - - for (String codec : codecs) { - String filename = - generateTestFile( - codec, expected, SyncBehavior.SYNC_DEFAULT, 0, AvroCoder.of(Bird.class), codec); - AvroSource<Bird> source = AvroSource.from(filename).withSchema(Bird.class); - List<Bird> actual = SourceTestUtils.readFromSource(source, null); - assertThat(expected, containsInAnyOrder(actual.toArray())); - } - } - - @Test - public void testSplitAtFraction() throws Exception { - // A reduced dataset is enough here. - List<FixedRecord> expected = createFixedRecords(DEFAULT_RECORD_COUNT); - // Create an AvroSource where each block is 1/10th of the total set of records. - String filename = - generateTestFile( - "tmp.avro", - expected, - SyncBehavior.SYNC_REGULAR, - DEFAULT_RECORD_COUNT / 10 /* max records per block */, - AvroCoder.of(FixedRecord.class), - DataFileConstants.NULL_CODEC); - File file = new File(filename); - - AvroSource<FixedRecord> source = AvroSource.from(filename).withSchema(FixedRecord.class); - List<? extends BoundedSource<FixedRecord>> splits = source.split(file.length() / 3, null); - for (BoundedSource<FixedRecord> subSource : splits) { - int items = SourceTestUtils.readFromSource(subSource, null).size(); - // Shouldn't split while unstarted. - SourceTestUtils.assertSplitAtFractionFails(subSource, 0, 0.0, null); - SourceTestUtils.assertSplitAtFractionFails(subSource, 0, 0.7, null); - SourceTestUtils.assertSplitAtFractionSucceedsAndConsistent(subSource, 1, 0.7, null); - SourceTestUtils.assertSplitAtFractionSucceedsAndConsistent( - subSource, DEFAULT_RECORD_COUNT / 100, 0.7, null); - SourceTestUtils.assertSplitAtFractionSucceedsAndConsistent( - subSource, DEFAULT_RECORD_COUNT / 10, 0.1, null); - SourceTestUtils.assertSplitAtFractionFails( - subSource, DEFAULT_RECORD_COUNT / 10 + 1, 0.1, null); - SourceTestUtils.assertSplitAtFractionFails(subSource, DEFAULT_RECORD_COUNT / 3, 0.3, null); - SourceTestUtils.assertSplitAtFractionFails(subSource, items, 0.9, null); - SourceTestUtils.assertSplitAtFractionFails(subSource, items, 1.0, null); - SourceTestUtils.assertSplitAtFractionSucceedsAndConsistent(subSource, items, 0.999, null); - } - } - - @Test - public void testGetProgressFromUnstartedReader() throws Exception { - List<FixedRecord> records = createFixedRecords(DEFAULT_RECORD_COUNT); - String filename = - generateTestFile( - "tmp.avro", - records, - SyncBehavior.SYNC_DEFAULT, - 1000, - AvroCoder.of(FixedRecord.class), - DataFileConstants.NULL_CODEC); - File file = new File(filename); - - AvroSource<FixedRecord> source = AvroSource.from(filename).withSchema(FixedRecord.class); - try (BoundedSource.BoundedReader<FixedRecord> reader = source.createReader(null)) { - assertEquals(Double.valueOf(0.0), reader.getFractionConsumed()); - } - - List<? extends BoundedSource<FixedRecord>> splits = source.split(file.length() / 3, null); - for (BoundedSource<FixedRecord> subSource : splits) { - try (BoundedSource.BoundedReader<FixedRecord> reader = subSource.createReader(null)) { - assertEquals(Double.valueOf(0.0), reader.getFractionConsumed()); - } - } - } - - @Test - public void testProgress() throws Exception { - // 5 records, 2 per block. - List<FixedRecord> records = createFixedRecords(5); - String filename = - generateTestFile( - "tmp.avro", - records, - SyncBehavior.SYNC_REGULAR, - 2, - AvroCoder.of(FixedRecord.class), - DataFileConstants.NULL_CODEC); - - AvroSource<FixedRecord> source = AvroSource.from(filename).withSchema(FixedRecord.class); - try (BoundedSource.BoundedReader<FixedRecord> readerOrig = source.createReader(null)) { - assertThat(readerOrig, Matchers.instanceOf(BlockBasedReader.class)); - BlockBasedReader<FixedRecord> reader = (BlockBasedReader<FixedRecord>) readerOrig; - - // Before starting - assertEquals(0.0, reader.getFractionConsumed(), 1e-6); - assertEquals(0, reader.getSplitPointsConsumed()); - assertEquals(BoundedReader.SPLIT_POINTS_UNKNOWN, reader.getSplitPointsRemaining()); - - // First 2 records are in the same block. - assertTrue(reader.start()); - assertTrue(reader.isAtSplitPoint()); - assertEquals(0, reader.getSplitPointsConsumed()); - assertEquals(BoundedReader.SPLIT_POINTS_UNKNOWN, reader.getSplitPointsRemaining()); - // continued - assertTrue(reader.advance()); - assertFalse(reader.isAtSplitPoint()); - assertEquals(0, reader.getSplitPointsConsumed()); - assertEquals(BoundedReader.SPLIT_POINTS_UNKNOWN, reader.getSplitPointsRemaining()); - - // Second block -> parallelism consumed becomes 1. - assertTrue(reader.advance()); - assertTrue(reader.isAtSplitPoint()); - assertEquals(1, reader.getSplitPointsConsumed()); - assertEquals(BoundedReader.SPLIT_POINTS_UNKNOWN, reader.getSplitPointsRemaining()); - // continued - assertTrue(reader.advance()); - assertFalse(reader.isAtSplitPoint()); - assertEquals(1, reader.getSplitPointsConsumed()); - assertEquals(BoundedReader.SPLIT_POINTS_UNKNOWN, reader.getSplitPointsRemaining()); - - // Third and final block -> parallelism consumed becomes 2, remaining becomes 1. - assertTrue(reader.advance()); - assertTrue(reader.isAtSplitPoint()); - assertEquals(2, reader.getSplitPointsConsumed()); - assertEquals(1, reader.getSplitPointsRemaining()); - - // Done - assertFalse(reader.advance()); - assertEquals(3, reader.getSplitPointsConsumed()); - assertEquals(0, reader.getSplitPointsRemaining()); - assertEquals(1.0, reader.getFractionConsumed(), 1e-6); - } - } - - @Test - public void testProgressEmptySource() throws Exception { - // 0 records, 20 per block. - List<FixedRecord> records = Collections.emptyList(); - String filename = - generateTestFile( - "tmp.avro", - records, - SyncBehavior.SYNC_REGULAR, - 2, - AvroCoder.of(FixedRecord.class), - DataFileConstants.NULL_CODEC); - - AvroSource<FixedRecord> source = AvroSource.from(filename).withSchema(FixedRecord.class); - try (BoundedSource.BoundedReader<FixedRecord> readerOrig = source.createReader(null)) { - assertThat(readerOrig, Matchers.instanceOf(BlockBasedReader.class)); - BlockBasedReader<FixedRecord> reader = (BlockBasedReader<FixedRecord>) readerOrig; - - // before starting - assertEquals(0.0, reader.getFractionConsumed(), 1e-6); - assertEquals(0, reader.getSplitPointsConsumed()); - assertEquals(BoundedReader.SPLIT_POINTS_UNKNOWN, reader.getSplitPointsRemaining()); - - // confirm empty - assertFalse(reader.start()); - - // after reading empty source - assertEquals(0, reader.getSplitPointsConsumed()); - assertEquals(0, reader.getSplitPointsRemaining()); - assertEquals(1.0, reader.getFractionConsumed(), 1e-6); - } - } - - @Test - public void testGetCurrentFromUnstartedReader() throws Exception { - List<FixedRecord> records = createFixedRecords(DEFAULT_RECORD_COUNT); - String filename = - generateTestFile( - "tmp.avro", - records, - SyncBehavior.SYNC_DEFAULT, - 1000, - AvroCoder.of(FixedRecord.class), - DataFileConstants.NULL_CODEC); - - AvroSource<FixedRecord> source = AvroSource.from(filename).withSchema(FixedRecord.class); - try (BlockBasedSource.BlockBasedReader<FixedRecord> reader = - (BlockBasedSource.BlockBasedReader<FixedRecord>) source.createReader(null)) { - assertEquals(null, reader.getCurrentBlock()); - - expectedException.expect(NoSuchElementException.class); - expectedException.expectMessage("No block has been successfully read from"); - reader.getCurrent(); - } - } - - @Test - public void testSplitAtFractionExhaustive() throws Exception { - // A small-sized input is sufficient, because the test verifies that splitting is non-vacuous. - List<FixedRecord> expected = createFixedRecords(20); - String filename = - generateTestFile( - "tmp.avro", - expected, - SyncBehavior.SYNC_REGULAR, - 5, - AvroCoder.of(FixedRecord.class), - DataFileConstants.NULL_CODEC); - - AvroSource<FixedRecord> source = AvroSource.from(filename).withSchema(FixedRecord.class); - SourceTestUtils.assertSplitAtFractionExhaustive(source, null); - } - - @Test - public void testSplitsWithSmallBlocks() throws Exception { - PipelineOptions options = PipelineOptionsFactory.create(); - // Test reading from an object file with many small random-sized blocks. - // The file itself doesn't have to be big; we can use a decreased record count. - List<Bird> expected = createRandomRecords(DEFAULT_RECORD_COUNT); - String filename = - generateTestFile( - "tmp.avro", - expected, - SyncBehavior.SYNC_RANDOM, - DEFAULT_RECORD_COUNT / 20 /* max records/block */, - AvroCoder.of(Bird.class), - DataFileConstants.NULL_CODEC); - File file = new File(filename); - - // Small minimum bundle size - AvroSource<Bird> source = - AvroSource.from(filename).withSchema(Bird.class).withMinBundleSize(100L); - - // Assert that the source produces the expected records - assertEquals(expected, SourceTestUtils.readFromSource(source, options)); - - List<? extends BoundedSource<Bird>> splits; - int nonEmptySplits; - - // Split with the minimum bundle size - splits = source.split(100L, options); - assertTrue(splits.size() > 2); - SourceTestUtils.assertSourcesEqualReferenceSource(source, splits, options); - nonEmptySplits = 0; - for (BoundedSource<Bird> subSource : splits) { - if (SourceTestUtils.readFromSource(subSource, options).size() > 0) { - nonEmptySplits += 1; - } - } - assertTrue(nonEmptySplits > 2); - - // Split with larger bundle size - splits = source.split(file.length() / 4, options); - assertTrue(splits.size() > 2); - SourceTestUtils.assertSourcesEqualReferenceSource(source, splits, options); - nonEmptySplits = 0; - for (BoundedSource<Bird> subSource : splits) { - if (SourceTestUtils.readFromSource(subSource, options).size() > 0) { - nonEmptySplits += 1; - } - } - assertTrue(nonEmptySplits > 2); - - // Split with the file length - splits = source.split(file.length(), options); - assertTrue(splits.size() == 1); - SourceTestUtils.assertSourcesEqualReferenceSource(source, splits, options); - } - - @Test - public void testMultipleFiles() throws Exception { - String baseName = "tmp-"; - List<Bird> expected = new ArrayList<>(); - for (int i = 0; i < 10; i++) { - List<Bird> contents = createRandomRecords(DEFAULT_RECORD_COUNT / 10); - expected.addAll(contents); - generateTestFile( - baseName + i, - contents, - SyncBehavior.SYNC_DEFAULT, - 0, - AvroCoder.of(Bird.class), - DataFileConstants.NULL_CODEC); - } - - AvroSource<Bird> source = - AvroSource.from(new File(tmpFolder.getRoot().toString(), baseName + "*").toString()) - .withSchema(Bird.class); - List<Bird> actual = SourceTestUtils.readFromSource(source, null); - assertThat(actual, containsInAnyOrder(expected.toArray())); - } - - @Test - public void testCreationWithSchema() throws Exception { - List<Bird> expected = createRandomRecords(100); - String filename = - generateTestFile( - "tmp.avro", - expected, - SyncBehavior.SYNC_DEFAULT, - 0, - AvroCoder.of(Bird.class), - DataFileConstants.NULL_CODEC); - - // Create a source with a schema object - Schema schema = ReflectData.get().getSchema(Bird.class); - AvroSource<GenericRecord> source = AvroSource.from(filename).withSchema(schema); - List<GenericRecord> records = SourceTestUtils.readFromSource(source, null); - assertEqualsWithGeneric(expected, records); - - // Create a source with a JSON schema - String schemaString = ReflectData.get().getSchema(Bird.class).toString(); - source = AvroSource.from(filename).withSchema(schemaString); - records = SourceTestUtils.readFromSource(source, null); - assertEqualsWithGeneric(expected, records); - } - - @Test - public void testSchemaUpdate() throws Exception { - List<Bird> birds = createRandomRecords(100); - String filename = - generateTestFile( - "tmp.avro", - birds, - SyncBehavior.SYNC_DEFAULT, - 0, - AvroCoder.of(Bird.class), - DataFileConstants.NULL_CODEC); - - AvroSource<FancyBird> source = AvroSource.from(filename).withSchema(FancyBird.class); - List<FancyBird> actual = SourceTestUtils.readFromSource(source, null); - - List<FancyBird> expected = new ArrayList<>(); - for (Bird bird : birds) { - expected.add( - new FancyBird( - bird.number, bird.species, bird.quality, bird.quantity, null, "MAXIMUM OVERDRIVE")); - } - - assertThat(actual, containsInAnyOrder(expected.toArray())); - } - - @Test - public void testSchemaStringIsInterned() throws Exception { - List<Bird> birds = createRandomRecords(100); - String filename = - generateTestFile( - "tmp.avro", - birds, - SyncBehavior.SYNC_DEFAULT, - 0, - AvroCoder.of(Bird.class), - DataFileConstants.NULL_CODEC); - Metadata fileMetadata = FileSystems.matchSingleFileSpec(filename); - String schema = AvroSource.readMetadataFromFile(fileMetadata.resourceId()).getSchemaString(); - // Add "" to the schema to make sure it is not interned. - AvroSource<GenericRecord> sourceA = AvroSource.from(filename).withSchema("" + schema); - AvroSource<GenericRecord> sourceB = AvroSource.from(filename).withSchema("" + schema); - assertSame(sourceA.getReaderSchemaString(), sourceB.getReaderSchemaString()); - - // Ensure that deserialization still goes through interning - AvroSource<GenericRecord> sourceC = SerializableUtils.clone(sourceB); - assertSame(sourceA.getReaderSchemaString(), sourceC.getReaderSchemaString()); - } - - @Test - public void testParseFn() throws Exception { - List<Bird> expected = createRandomRecords(100); - String filename = - generateTestFile( - "tmp.avro", - expected, - SyncBehavior.SYNC_DEFAULT, - 0, - AvroCoder.of(Bird.class), - DataFileConstants.NULL_CODEC); - - AvroSource<Bird> source = - AvroSource.from(filename) - .withParseFn( - input -> - new Bird( - (long) input.get("number"), - input.get("species").toString(), - input.get("quality").toString(), - (long) input.get("quantity")), - AvroCoder.of(Bird.class)); - List<Bird> actual = SourceTestUtils.readFromSource(source, null); - assertThat(actual, containsInAnyOrder(expected.toArray())); - } - - @Test - public void testDatumReaderFactoryWithGenericRecord() throws Exception { - List<Bird> inputBirds = createRandomRecords(100); - - String filename = - generateTestFile( - "tmp.avro", - inputBirds, - SyncBehavior.SYNC_DEFAULT, - 0, - AvroCoder.of(Bird.class), - DataFileConstants.NULL_CODEC); - - AvroSource.DatumReaderFactory<GenericRecord> factory = - (writer, reader) -> - new GenericDatumReader<GenericRecord>(writer, reader) { - @Override - protected Object readString(Object old, Decoder in) throws IOException { - return super.readString(old, in) + "_custom"; - } - }; - - AvroSource<Bird> source = - AvroSource.from(filename) - .withParseFn( - input -> - new Bird( - (long) input.get("number"), - input.get("species").toString(), - input.get("quality").toString(), - (long) input.get("quantity")), - AvroCoder.of(Bird.class)) - .withDatumReaderFactory(factory); - List<Bird> actual = SourceTestUtils.readFromSource(source, null); - List<Bird> expected = - inputBirds.stream() - .map(b -> new Bird(b.number, b.species + "_custom", b.quality + "_custom", b.quantity)) - .collect(Collectors.toList()); - - assertThat(actual, containsInAnyOrder(expected.toArray())); - } - - private void assertEqualsWithGeneric(List<Bird> expected, List<GenericRecord> actual) { - assertEquals(expected.size(), actual.size()); - for (int i = 0; i < expected.size(); i++) { - Bird fixed = expected.get(i); - GenericRecord generic = actual.get(i); - assertEquals(fixed.number, generic.get("number")); - assertEquals(fixed.quality, generic.get("quality").toString()); // From Avro util.Utf8 - assertEquals(fixed.quantity, generic.get("quantity")); - assertEquals(fixed.species, generic.get("species").toString()); - } - } - - @Test - public void testDisplayData() { - AvroSource<Bird> source = - AvroSource.from("foobar.txt").withSchema(Bird.class).withMinBundleSize(1234); - - DisplayData displayData = DisplayData.from(source); - assertThat(displayData, hasDisplayItem("filePattern", "foobar.txt")); - assertThat(displayData, hasDisplayItem("minBundleSize", 1234)); - } - - @Test - public void testReadMetadataWithCodecs() throws Exception { - // Test reading files generated using all codecs. - String[] codecs = { - DataFileConstants.NULL_CODEC, - DataFileConstants.BZIP2_CODEC, - DataFileConstants.DEFLATE_CODEC, - DataFileConstants.SNAPPY_CODEC, - DataFileConstants.XZ_CODEC - }; - List<Bird> expected = createRandomRecords(DEFAULT_RECORD_COUNT); - - for (String codec : codecs) { - String filename = - generateTestFile( - codec, expected, SyncBehavior.SYNC_DEFAULT, 0, AvroCoder.of(Bird.class), codec); - - Metadata fileMeta = FileSystems.matchSingleFileSpec(filename); - AvroMetadata metadata = AvroSource.readMetadataFromFile(fileMeta.resourceId()); - assertEquals(codec, metadata.getCodec()); - } - } - - @Test - public void testReadSchemaString() throws Exception { - List<Bird> expected = createRandomRecords(DEFAULT_RECORD_COUNT); - String codec = DataFileConstants.NULL_CODEC; - String filename = - generateTestFile( - codec, expected, SyncBehavior.SYNC_DEFAULT, 0, AvroCoder.of(Bird.class), codec); - Metadata fileMeta = FileSystems.matchSingleFileSpec(filename); - AvroMetadata metadata = AvroSource.readMetadataFromFile(fileMeta.resourceId()); - // By default, parse validates the schema, which is what we want. - Schema schema = new Schema.Parser().parse(metadata.getSchemaString()); - assertEquals(4, schema.getFields().size()); - } - - @Test - public void testCreateFromMetadata() throws Exception { - List<Bird> expected = createRandomRecords(DEFAULT_RECORD_COUNT); - String codec = DataFileConstants.NULL_CODEC; - String filename = - generateTestFile( - codec, expected, SyncBehavior.SYNC_DEFAULT, 0, AvroCoder.of(Bird.class), codec); - Metadata fileMeta = FileSystems.matchSingleFileSpec(filename); - - AvroSource<GenericRecord> source = AvroSource.from(fileMeta); - AvroSource<Bird> sourceWithSchema = source.withSchema(Bird.class); - AvroSource<Bird> sourceWithSchemaWithMinBundleSize = sourceWithSchema.withMinBundleSize(1234); - - assertEquals(FileBasedSource.Mode.SINGLE_FILE_OR_SUBRANGE, source.getMode()); - assertEquals(FileBasedSource.Mode.SINGLE_FILE_OR_SUBRANGE, sourceWithSchema.getMode()); - assertEquals( - FileBasedSource.Mode.SINGLE_FILE_OR_SUBRANGE, sourceWithSchemaWithMinBundleSize.getMode()); - } - - /** - * Class that will encode to a fixed size: 16 bytes. - * - * <p>Each object has a 15-byte array. Avro encodes an object of this type as a byte array, so - * each encoded object will consist of 1 byte that encodes the length of the array, followed by 15 - * bytes. - */ - @DefaultCoder(AvroCoder.class) - public static class FixedRecord { - private byte[] value = new byte[15]; - - public FixedRecord() { - this(0); - } - - public FixedRecord(int i) { - value[0] = (byte) i; - value[1] = (byte) (i >> 8); - value[2] = (byte) (i >> 16); - value[3] = (byte) (i >> 24); - } - - public int asInt() { - return value[0] | (value[1] << 8) | (value[2] << 16) | (value[3] << 24); - } - - @Override - public boolean equals(@Nullable Object o) { - if (o instanceof FixedRecord) { - FixedRecord other = (FixedRecord) o; - return this.asInt() == other.asInt(); - } - return false; - } - - @Override - public int hashCode() { - return toString().hashCode(); - } - - @Override - public String toString() { - return Integer.toString(this.asInt()); - } - } - - /** Create a list of count 16-byte records. */ - private static List<FixedRecord> createFixedRecords(int count) { - List<FixedRecord> records = new ArrayList<>(); - for (int i = 0; i < count; i++) { - records.add(new FixedRecord(i)); - } - return records; - } - - /** Class used as the record type in tests. */ - @DefaultCoder(AvroCoder.class) - static class Bird { - long number; - String species; - String quality; - long quantity; - - public Bird() {} - - public Bird(long number, String species, String quality, long quantity) { - this.number = number; - this.species = species; - this.quality = quality; - this.quantity = quantity; - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(Bird.class) - .addValue(number) - .addValue(species) - .addValue(quantity) - .addValue(quality) - .toString(); - } - - @Override - public boolean equals(@Nullable Object obj) { - if (obj instanceof Bird) { - Bird other = (Bird) obj; - return Objects.equals(species, other.species) - && Objects.equals(quality, other.quality) - && quantity == other.quantity - && number == other.number; - } - return false; - } - - @Override - public int hashCode() { - return Objects.hash(number, species, quality, quantity); - } - } - - /** - * Class used as the record type in tests. - * - * <p>Contains nullable fields and fields with default values. Can be read using a file written - * with the Bird schema. - */ - @DefaultCoder(AvroCoder.class) - public static class FancyBird { - long number; - String species; - String quality; - long quantity; - - @org.apache.avro.reflect.Nullable String habitat; - - @AvroDefault("\"MAXIMUM OVERDRIVE\"") - String fancinessLevel; - - public FancyBird() {} - - public FancyBird( - long number, - String species, - String quality, - long quantity, - String habitat, - String fancinessLevel) { - this.number = number; - this.species = species; - this.quality = quality; - this.quantity = quantity; - this.habitat = habitat; - this.fancinessLevel = fancinessLevel; - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(FancyBird.class) - .addValue(number) - .addValue(species) - .addValue(quality) - .addValue(quantity) - .addValue(habitat) - .addValue(fancinessLevel) - .toString(); - } - - @Override - public boolean equals(@Nullable Object obj) { - if (obj instanceof FancyBird) { - FancyBird other = (FancyBird) obj; - return Objects.equals(species, other.species) - && Objects.equals(quality, other.quality) - && quantity == other.quantity - && number == other.number - && Objects.equals(fancinessLevel, other.fancinessLevel) - && Objects.equals(habitat, other.habitat); - } - return false; - } - - @Override - public int hashCode() { - return Objects.hash(number, species, quality, quantity, habitat, fancinessLevel); - } - } - - /** Create a list of n random records. */ - private static List<Bird> createRandomRecords(long n) { - String[] qualities = { - "miserable", "forelorn", "fidgity", "squirrelly", "fanciful", "chipper", "lazy" - }; - String[] species = {"pigeons", "owls", "gulls", "hawks", "robins", "jays"}; - Random random = new Random(0); - - List<Bird> records = new ArrayList<>(); - for (long i = 0; i < n; i++) { - Bird bird = new Bird(); - bird.quality = qualities[random.nextInt(qualities.length)]; - bird.species = species[random.nextInt(species.length)]; - bird.number = i; - bird.quantity = random.nextLong(); - records.add(bird); - } - return records; - } -} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/ReadTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/ReadTest.java index 04d9b8f12f7c..aa528c4f08f4 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/ReadTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/ReadTest.java @@ -38,7 +38,6 @@ import java.util.function.Consumer; import java.util.stream.Collectors; import java.util.stream.LongStream; -import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.CustomCoder; @@ -343,7 +342,7 @@ public Coder<Long> getOutputCoder() { @Override public Coder<CounterMark> getCheckpointMarkCoder() { - return AvroCoder.of(CountingSource.CounterMark.class); + return new CountingSource.CounterMarkCoder(); } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/SerializableAvroCodecFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/SerializableAvroCodecFactoryTest.java deleted file mode 100644 index 4383a16dd28c..000000000000 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/SerializableAvroCodecFactoryTest.java +++ /dev/null @@ -1,93 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.io; - -import static org.apache.avro.file.DataFileConstants.BZIP2_CODEC; -import static org.apache.avro.file.DataFileConstants.DEFLATE_CODEC; -import static org.apache.avro.file.DataFileConstants.NULL_CODEC; -import static org.apache.avro.file.DataFileConstants.SNAPPY_CODEC; -import static org.apache.avro.file.DataFileConstants.XZ_CODEC; -import static org.junit.Assert.assertEquals; - -import java.util.Arrays; -import java.util.List; -import org.apache.avro.file.CodecFactory; -import org.apache.beam.sdk.util.SerializableUtils; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -/** Tests of SerializableAvroCodecFactory. */ -@RunWith(JUnit4.class) -public class SerializableAvroCodecFactoryTest { - private final List<String> avroCodecs = - Arrays.asList(NULL_CODEC, SNAPPY_CODEC, DEFLATE_CODEC, XZ_CODEC, BZIP2_CODEC); - - @Test - public void testDefaultCodecsIn() throws Exception { - for (String codec : avroCodecs) { - SerializableAvroCodecFactory codecFactory = - new SerializableAvroCodecFactory(CodecFactory.fromString(codec)); - - assertEquals(CodecFactory.fromString(codec).toString(), codecFactory.getCodec().toString()); - } - } - - @Test - public void testDefaultCodecsSerDe() throws Exception { - for (String codec : avroCodecs) { - SerializableAvroCodecFactory codecFactory = - new SerializableAvroCodecFactory(CodecFactory.fromString(codec)); - - SerializableAvroCodecFactory serdeC = SerializableUtils.clone(codecFactory); - - assertEquals(CodecFactory.fromString(codec).toString(), serdeC.getCodec().toString()); - } - } - - @Test - public void testDeflateCodecSerDeWithLevels() throws Exception { - for (int i = 0; i < 10; ++i) { - SerializableAvroCodecFactory codecFactory = - new SerializableAvroCodecFactory(CodecFactory.deflateCodec(i)); - - SerializableAvroCodecFactory serdeC = SerializableUtils.clone(codecFactory); - - assertEquals(CodecFactory.deflateCodec(i).toString(), serdeC.getCodec().toString()); - } - } - - @Test - public void testXZCodecSerDeWithLevels() throws Exception { - for (int i = 0; i < 10; ++i) { - SerializableAvroCodecFactory codecFactory = - new SerializableAvroCodecFactory(CodecFactory.xzCodec(i)); - - SerializableAvroCodecFactory serdeC = SerializableUtils.clone(codecFactory); - - assertEquals(CodecFactory.xzCodec(i).toString(), serdeC.getCodec().toString()); - } - } - - @Test(expected = NullPointerException.class) - public void testNullCodecToString() throws Exception { - // use default CTR (available cause Serializable) - SerializableAvroCodecFactory codec = new SerializableAvroCodecFactory(); - assertEquals("null", codec.toString()); - } -} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/AvroSchemaTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/AvroSchemaTest.java deleted file mode 100644 index c32dbd9944a1..000000000000 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/AvroSchemaTest.java +++ /dev/null @@ -1,496 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.schemas; - -import static org.junit.Assert.assertEquals; - -import java.nio.ByteBuffer; -import java.util.Arrays; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import org.apache.avro.generic.GenericData; -import org.apache.avro.generic.GenericRecord; -import org.apache.avro.generic.GenericRecordBuilder; -import org.apache.avro.reflect.AvroIgnore; -import org.apache.avro.reflect.AvroName; -import org.apache.avro.reflect.AvroSchema; -import org.apache.avro.util.Utf8; -import org.apache.beam.sdk.schemas.Schema.FieldType; -import org.apache.beam.sdk.schemas.logicaltypes.EnumerationType; -import org.apache.beam.sdk.schemas.logicaltypes.FixedBytes; -import org.apache.beam.sdk.schemas.transforms.Group; -import org.apache.beam.sdk.schemas.utils.AvroUtils; -import org.apache.beam.sdk.testing.PAssert; -import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.testing.ValidatesRunner; -import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.transforms.SerializableFunction; -import org.apache.beam.sdk.util.SerializableUtils; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.Row; -import org.apache.beam.sdk.values.TypeDescriptor; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; -import org.checkerframework.checker.nullness.qual.Nullable; -import org.joda.time.DateTime; -import org.joda.time.DateTimeZone; -import org.joda.time.Days; -import org.joda.time.LocalDate; -import org.junit.Rule; -import org.junit.Test; -import org.junit.experimental.categories.Category; - -/** Tests for AVRO schema classes. */ -public class AvroSchemaTest { - /** A test POJO that corresponds to our AVRO schema. */ - public static class AvroSubPojo { - @AvroName("BOOL_NON_NULLABLE") - public boolean boolNonNullable; - - @AvroName("int") - @org.apache.avro.reflect.Nullable - public Integer anInt; - - public AvroSubPojo(boolean boolNonNullable, Integer anInt) { - this.boolNonNullable = boolNonNullable; - this.anInt = anInt; - } - - public AvroSubPojo() {} - - @Override - public boolean equals(@Nullable Object o) { - if (this == o) { - return true; - } - if (!(o instanceof AvroSubPojo)) { - return false; - } - AvroSubPojo that = (AvroSubPojo) o; - return boolNonNullable == that.boolNonNullable && Objects.equals(anInt, that.anInt); - } - - @Override - public int hashCode() { - return Objects.hash(boolNonNullable, anInt); - } - - @Override - public String toString() { - return "AvroSubPojo{" + "boolNonNullable=" + boolNonNullable + ", anInt=" + anInt + '}'; - } - } - - /** A test POJO that corresponds to our AVRO schema. */ - public static class AvroPojo { - public @AvroName("bool_non_nullable") boolean boolNonNullable; - - @org.apache.avro.reflect.Nullable - public @AvroName("int") Integer anInt; - - @org.apache.avro.reflect.Nullable - public @AvroName("long") Long aLong; - - @AvroName("float") - @org.apache.avro.reflect.Nullable - public Float aFloat; - - @AvroName("double") - @org.apache.avro.reflect.Nullable - public Double aDouble; - - @org.apache.avro.reflect.Nullable public String string; - @org.apache.avro.reflect.Nullable public ByteBuffer bytes; - - @AvroSchema("{\"type\": \"fixed\", \"size\": 4, \"name\": \"fixed4\"}") - public byte[] fixed; - - @AvroSchema("{\"type\": \"int\", \"logicalType\": \"date\"}") - public LocalDate date; - - @AvroSchema("{\"type\": \"long\", \"logicalType\": \"timestamp-millis\"}") - public DateTime timestampMillis; - - @AvroSchema("{\"name\": \"TestEnum\", \"type\": \"enum\", \"symbols\": [\"abc\",\"cde\"] }") - public TestEnum testEnum; - - @org.apache.avro.reflect.Nullable public AvroSubPojo row; - @org.apache.avro.reflect.Nullable public List<AvroSubPojo> array; - @org.apache.avro.reflect.Nullable public Map<String, AvroSubPojo> map; - @AvroIgnore String extraField; - - @Override - public boolean equals(@Nullable Object o) { - if (this == o) { - return true; - } - if (!(o instanceof AvroPojo)) { - return false; - } - AvroPojo avroPojo = (AvroPojo) o; - return boolNonNullable == avroPojo.boolNonNullable - && Objects.equals(anInt, avroPojo.anInt) - && Objects.equals(aLong, avroPojo.aLong) - && Objects.equals(aFloat, avroPojo.aFloat) - && Objects.equals(aDouble, avroPojo.aDouble) - && Objects.equals(string, avroPojo.string) - && Objects.equals(bytes, avroPojo.bytes) - && Arrays.equals(fixed, avroPojo.fixed) - && Objects.equals(date, avroPojo.date) - && Objects.equals(timestampMillis, avroPojo.timestampMillis) - && Objects.equals(testEnum, avroPojo.testEnum) - && Objects.equals(row, avroPojo.row) - && Objects.equals(array, avroPojo.array) - && Objects.equals(map, avroPojo.map); - } - - @Override - public int hashCode() { - return Objects.hash( - boolNonNullable, - anInt, - aLong, - aFloat, - aDouble, - string, - bytes, - Arrays.hashCode(fixed), - date, - timestampMillis, - testEnum, - row, - array, - map); - } - - public AvroPojo( - boolean boolNonNullable, - int anInt, - long aLong, - float aFloat, - double aDouble, - String string, - ByteBuffer bytes, - byte[] fixed, - LocalDate date, - DateTime timestampMillis, - TestEnum testEnum, - AvroSubPojo row, - List<AvroSubPojo> array, - Map<String, AvroSubPojo> map) { - this.boolNonNullable = boolNonNullable; - this.anInt = anInt; - this.aLong = aLong; - this.aFloat = aFloat; - this.aDouble = aDouble; - this.string = string; - this.bytes = bytes; - this.fixed = fixed; - this.date = date; - this.timestampMillis = timestampMillis; - this.testEnum = testEnum; - this.row = row; - this.array = array; - this.map = map; - this.extraField = ""; - } - - public AvroPojo() {} - - @Override - public String toString() { - return "AvroPojo{" - + "boolNonNullable=" - + boolNonNullable - + ", anInt=" - + anInt - + ", aLong=" - + aLong - + ", aFloat=" - + aFloat - + ", aDouble=" - + aDouble - + ", string='" - + string - + '\'' - + ", bytes=" - + bytes - + ", fixed=" - + Arrays.toString(fixed) - + ", date=" - + date - + ", timestampMillis=" - + timestampMillis - + ", testEnum=" - + testEnum - + ", row=" - + row - + ", array=" - + array - + ", map=" - + map - + ", extraField='" - + extraField - + '\'' - + '}'; - } - } - - private static final Schema SUBSCHEMA = - Schema.builder() - .addField("BOOL_NON_NULLABLE", FieldType.BOOLEAN) - .addNullableField("int", FieldType.INT32) - .build(); - private static final FieldType SUB_TYPE = FieldType.row(SUBSCHEMA).withNullable(true); - - private static final EnumerationType TEST_ENUM_TYPE = EnumerationType.create("abc", "cde"); - - private static final Schema SCHEMA = - Schema.builder() - .addField("bool_non_nullable", FieldType.BOOLEAN) - .addNullableField("int", FieldType.INT32) - .addNullableField("long", FieldType.INT64) - .addNullableField("float", FieldType.FLOAT) - .addNullableField("double", FieldType.DOUBLE) - .addNullableField("string", FieldType.STRING) - .addNullableField("bytes", FieldType.BYTES) - .addField("fixed", FieldType.logicalType(FixedBytes.of(4))) - .addField("date", FieldType.DATETIME) - .addField("timestampMillis", FieldType.DATETIME) - .addField("TestEnum", FieldType.logicalType(TEST_ENUM_TYPE)) - .addNullableField("row", SUB_TYPE) - .addNullableField("array", FieldType.array(SUB_TYPE)) - .addNullableField("map", FieldType.map(FieldType.STRING, SUB_TYPE)) - .build(); - - private static final Schema POJO_SCHEMA = - Schema.builder() - .addField("bool_non_nullable", FieldType.BOOLEAN) - .addNullableField("int", FieldType.INT32) - .addNullableField("long", FieldType.INT64) - .addNullableField("float", FieldType.FLOAT) - .addNullableField("double", FieldType.DOUBLE) - .addNullableField("string", FieldType.STRING) - .addNullableField("bytes", FieldType.BYTES) - .addField("fixed", FieldType.logicalType(FixedBytes.of(4))) - .addField("date", FieldType.DATETIME) - .addField("timestampMillis", FieldType.DATETIME) - .addField("testEnum", FieldType.logicalType(TEST_ENUM_TYPE)) - .addNullableField("row", SUB_TYPE) - .addNullableField("array", FieldType.array(SUB_TYPE.withNullable(false))) - .addNullableField("map", FieldType.map(FieldType.STRING, SUB_TYPE.withNullable(false))) - .build(); - - private static final byte[] BYTE_ARRAY = new byte[] {1, 2, 3, 4}; - private static final DateTime DATE_TIME = - new DateTime().withDate(1979, 3, 14).withTime(1, 2, 3, 4); - private static final LocalDate DATE = new LocalDate(1979, 3, 14); - private static final TestAvroNested AVRO_NESTED_SPECIFIC_RECORD = new TestAvroNested(true, 42); - private static final TestAvro AVRO_SPECIFIC_RECORD = - new TestAvro( - true, - 43, - 44L, - (float) 44.1, - (double) 44.2, - "mystring", - ByteBuffer.wrap(BYTE_ARRAY), - new fixed4(BYTE_ARRAY), - DATE, - DATE_TIME, - TestEnum.abc, - AVRO_NESTED_SPECIFIC_RECORD, - ImmutableList.of(AVRO_NESTED_SPECIFIC_RECORD, AVRO_NESTED_SPECIFIC_RECORD), - ImmutableMap.of("k1", AVRO_NESTED_SPECIFIC_RECORD, "k2", AVRO_NESTED_SPECIFIC_RECORD)); - private static final GenericRecord AVRO_NESTED_GENERIC_RECORD = - new GenericRecordBuilder(TestAvroNested.SCHEMA$) - .set("BOOL_NON_NULLABLE", true) - .set("int", 42) - .build(); - private static final GenericRecord AVRO_GENERIC_RECORD = - new GenericRecordBuilder(TestAvro.SCHEMA$) - .set("bool_non_nullable", true) - .set("int", 43) - .set("long", 44L) - .set("float", (float) 44.1) - .set("double", (double) 44.2) - .set("string", new Utf8("mystring")) - .set("bytes", ByteBuffer.wrap(BYTE_ARRAY)) - .set( - "fixed", - GenericData.get() - .createFixed( - null, BYTE_ARRAY, org.apache.avro.Schema.createFixed("fixed4", "", "", 4))) - .set("date", (int) Days.daysBetween(new LocalDate(1970, 1, 1), DATE).getDays()) - .set("timestampMillis", DATE_TIME.getMillis()) - .set("TestEnum", TestEnum.abc) - .set("row", AVRO_NESTED_GENERIC_RECORD) - .set("array", ImmutableList.of(AVRO_NESTED_GENERIC_RECORD, AVRO_NESTED_GENERIC_RECORD)) - .set( - "map", - ImmutableMap.of( - new Utf8("k1"), AVRO_NESTED_GENERIC_RECORD, - new Utf8("k2"), AVRO_NESTED_GENERIC_RECORD)) - .build(); - - private static final Row NESTED_ROW = Row.withSchema(SUBSCHEMA).addValues(true, 42).build(); - private static final Row ROW = - Row.withSchema(SCHEMA) - .addValues( - true, - 43, - 44L, - (float) 44.1, - (double) 44.2, - "mystring", - ByteBuffer.wrap(BYTE_ARRAY), - BYTE_ARRAY, - DATE.toDateTimeAtStartOfDay(DateTimeZone.UTC), - DATE_TIME, - TEST_ENUM_TYPE.valueOf("abc"), - NESTED_ROW, - ImmutableList.of(NESTED_ROW, NESTED_ROW), - ImmutableMap.of("k1", NESTED_ROW, "k2", NESTED_ROW)) - .build(); - - @Test - public void testSpecificRecordSchema() { - assertEquals(SCHEMA, new AvroRecordSchema().schemaFor(TypeDescriptor.of(TestAvro.class))); - } - - @Test - public void testPojoSchema() { - assertEquals(POJO_SCHEMA, new AvroRecordSchema().schemaFor(TypeDescriptor.of(AvroPojo.class))); - } - - @Test - public void testSpecificRecordToRow() { - SerializableFunction<TestAvro, Row> toRow = - new AvroRecordSchema().toRowFunction(TypeDescriptor.of(TestAvro.class)); - assertEquals(ROW, toRow.apply(AVRO_SPECIFIC_RECORD)); - } - - @Test - public void testRowToSpecificRecord() { - SerializableFunction<Row, TestAvro> fromRow = - new AvroRecordSchema().fromRowFunction(TypeDescriptor.of(TestAvro.class)); - assertEquals(AVRO_SPECIFIC_RECORD, fromRow.apply(ROW)); - } - - @Test - public void testGenericRecordToRow() { - SerializableFunction<GenericRecord, Row> toRow = - AvroUtils.getGenericRecordToRowFunction(SCHEMA); - assertEquals(ROW, toRow.apply(AVRO_GENERIC_RECORD)); - } - - @Test - public void testRowToGenericRecord() { - SerializableFunction<Row, GenericRecord> fromRow = - AvroUtils.getRowToGenericRecordFunction(TestAvro.SCHEMA$); - assertEquals(AVRO_GENERIC_RECORD, fromRow.apply(ROW)); - } - - private static final AvroSubPojo SUB_POJO = new AvroSubPojo(true, 42); - private static final AvroPojo AVRO_POJO = - new AvroPojo( - true, - 43, - 44L, - (float) 44.1, - (double) 44.2, - "mystring", - ByteBuffer.wrap(BYTE_ARRAY), - BYTE_ARRAY, - DATE, - DATE_TIME, - TestEnum.abc, - SUB_POJO, - ImmutableList.of(SUB_POJO, SUB_POJO), - ImmutableMap.of("k1", SUB_POJO, "k2", SUB_POJO)); - - private static final Row ROW_FOR_POJO = - Row.withSchema(POJO_SCHEMA) - .addValues( - true, - 43, - 44L, - (float) 44.1, - (double) 44.2, - "mystring", - ByteBuffer.wrap(BYTE_ARRAY), - BYTE_ARRAY, - DATE.toDateTimeAtStartOfDay(DateTimeZone.UTC), - DATE_TIME, - TEST_ENUM_TYPE.valueOf("abc"), - NESTED_ROW, - ImmutableList.of(NESTED_ROW, NESTED_ROW), - ImmutableMap.of("k1", NESTED_ROW, "k2", NESTED_ROW)) - .build(); - - @Test - public void testPojoRecordToRow() { - SerializableFunction<AvroPojo, Row> toRow = - new AvroRecordSchema().toRowFunction(TypeDescriptor.of(AvroPojo.class)); - assertEquals(ROW_FOR_POJO, toRow.apply(AVRO_POJO)); - } - - @Test - public void testRowToPojo() { - SerializableFunction<Row, AvroPojo> fromRow = - new AvroRecordSchema().fromRowFunction(TypeDescriptor.of(AvroPojo.class)); - assertEquals(AVRO_POJO, fromRow.apply(ROW_FOR_POJO)); - } - - @Test - public void testPojoRecordToRowSerializable() { - SerializableUtils.ensureSerializableRoundTrip( - new AvroRecordSchema().toRowFunction(TypeDescriptor.of(AvroPojo.class))); - } - - @Test - public void testPojoRecordFromRowSerializable() { - SerializableUtils.ensureSerializableRoundTrip( - new AvroRecordSchema().fromRowFunction(TypeDescriptor.of(AvroPojo.class))); - } - - @Rule public final transient TestPipeline pipeline = TestPipeline.create(); - - @Test - @Category(ValidatesRunner.class) - public void testAvroPipelineGroupBy() { - PCollection<Row> input = pipeline.apply(Create.of(ROW_FOR_POJO).withRowSchema(POJO_SCHEMA)); - - PCollection<Row> output = input.apply(Group.byFieldNames("string")); - Schema keySchema = Schema.builder().addStringField("string").build(); - Schema outputSchema = - Schema.builder() - .addRowField("key", keySchema) - .addIterableField("value", FieldType.row(POJO_SCHEMA)) - .build(); - PAssert.that(output) - .containsInAnyOrder( - Row.withSchema(outputSchema) - .addValue(Row.withSchema(keySchema).addValue("mystring").build()) - .addIterable(ImmutableList.of(ROW_FOR_POJO)) - .build()); - - pipeline.run(); - } -} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaCoderTest.java index 962c8f718349..cc57d382af0c 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaCoderTest.java @@ -26,7 +26,6 @@ import java.util.Collection; import java.util.Objects; import java.util.function.Supplier; -import org.apache.avro.reflect.AvroSchema; import org.apache.beam.sdk.coders.Coder.NonDeterministicException; import org.apache.beam.sdk.coders.RowCoder; import org.apache.beam.sdk.schemas.Schema.Field; @@ -196,43 +195,6 @@ public int hashCode() { } } - @DefaultSchema(AvroRecordSchema.class) - private static class SimpleAvro { - public String string; - public Integer int32; - public Long int64; - - @AvroSchema("{\"type\": \"long\", \"logicalType\": \"timestamp-millis\"}") - public DateTime datetime; - - public SimpleAvro(String string, Integer int32, Long int64, DateTime datetime) { - this.string = string; - this.int32 = int32; - this.int64 = int64; - this.datetime = datetime; - } - - @Override - public boolean equals(@Nullable Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - SimpleAvro that = (SimpleAvro) o; - return string.equals(that.string) - && int32.equals(that.int32) - && int64.equals(that.int64) - && datetime.equals(that.datetime); - } - - @Override - public int hashCode() { - return Objects.hash(string, int32, int64, datetime); - } - } - private static final SchemaRegistry REGISTRY = SchemaRegistry.createDefault(); private static SchemaCoder coderFrom(TypeDescriptor typeDescriptor) throws NoSuchSchemaException { @@ -316,23 +278,6 @@ public static Collection<Object[]> data() throws NoSuchSchemaException { new DateTime().withDate(1989, 3, 14).withTime(10, 30, 0, 0))), true }, - new Object[] { - coderFrom(TypeDescriptor.of(SimpleAvro.class)), - ImmutableList.<Supplier<Object>>of( - () -> - new SimpleAvro( - "foo", - 9001, - 0L, - new DateTime().withDate(1979, 3, 14).withTime(10, 30, 0, 0)), - () -> - new SimpleAvro( - "bar", - 9002, - 1L, - new DateTime().withDate(1989, 3, 14).withTime(10, 30, 0, 0))), - true - }, new Object[] { RowCoder.of(LOGICAL_NANOS_SCHEMA), ImmutableList.<Supplier<Object>>of( diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/io/AvroPayloadSerializerProviderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/io/AvroPayloadSerializerProviderTest.java deleted file mode 100644 index 2fbcc76f18ff..000000000000 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/io/AvroPayloadSerializerProviderTest.java +++ /dev/null @@ -1,64 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.schemas.io; - -import static org.junit.Assert.assertEquals; - -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import org.apache.avro.generic.GenericRecord; -import org.apache.avro.generic.GenericRecordBuilder; -import org.apache.beam.sdk.coders.AvroCoder; -import org.apache.beam.sdk.schemas.Schema; -import org.apache.beam.sdk.schemas.io.payloads.AvroPayloadSerializerProvider; -import org.apache.beam.sdk.schemas.utils.AvroUtils; -import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -@RunWith(JUnit4.class) -public class AvroPayloadSerializerProviderTest { - private static final Schema SCHEMA = - Schema.builder().addInt64Field("abc").addStringField("xyz").build(); - private static final org.apache.avro.Schema AVRO_SCHEMA = AvroUtils.toAvroSchema(SCHEMA); - private static final AvroCoder<GenericRecord> AVRO_CODER = AvroCoder.of(AVRO_SCHEMA); - private static final Row DESERIALIZED = - Row.withSchema(SCHEMA).withFieldValue("abc", 3L).withFieldValue("xyz", "qqq").build(); - private static final GenericRecord SERIALIZED = - new GenericRecordBuilder(AVRO_SCHEMA).set("abc", 3L).set("xyz", "qqq").build(); - - private final AvroPayloadSerializerProvider provider = new AvroPayloadSerializerProvider(); - - @Test - public void serialize() throws Exception { - byte[] bytes = provider.getSerializer(SCHEMA, ImmutableMap.of()).serialize(DESERIALIZED); - GenericRecord record = AVRO_CODER.decode(new ByteArrayInputStream(bytes)); - assertEquals(3L, record.get("abc")); - assertEquals("qqq", record.get("xyz").toString()); - } - - @Test - public void deserialize() throws Exception { - ByteArrayOutputStream os = new ByteArrayOutputStream(); - AVRO_CODER.encode(SERIALIZED, os); - Row row = provider.getSerializer(SCHEMA, ImmutableMap.of()).deserialize(os.toByteArray()); - assertEquals(DESERIALIZED, row); - } -} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/ConvertTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/ConvertTest.java index 603482edaaa0..32d32e8918eb 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/ConvertTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/ConvertTest.java @@ -20,12 +20,10 @@ import java.util.Arrays; import java.util.Map; import java.util.Objects; -import org.apache.avro.generic.GenericRecord; import org.apache.beam.sdk.schemas.JavaFieldSchema; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.Schema.FieldType; import org.apache.beam.sdk.schemas.annotations.DefaultSchema; -import org.apache.beam.sdk.schemas.utils.AvroUtils; import org.apache.beam.sdk.testing.NeedsRunner; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; @@ -134,9 +132,6 @@ public int hashCode() { .addValue(ImmutableMap.of("first", EXPECTED_ROW1_NESTED, "second", EXPECTED_ROW1_NESTED)) .build(); - private static final GenericRecord EXPECTED_GENERICRECORD1 = - AvroUtils.toGenericRecord(EXPECTED_ROW1, AvroUtils.toAvroSchema(EXPECTED_SCHEMA1)); - /** Test outer POJO. Different but equivalent schema. * */ @DefaultSchema(JavaFieldSchema.class) public static class POJO2 { @@ -248,13 +243,4 @@ public void testFromRowsUnboxingPrimitive() { PAssert.that(longs).containsInAnyOrder((Long) EXPECTED_ROW1.getValue("field2")); pipeline.run(); } - - @Test - @Category(NeedsRunner.class) - public void testToGenericRecords() { - PCollection<GenericRecord> records = - pipeline.apply(Create.of(new POJO1())).apply(Convert.to(GenericRecord.class)); - PAssert.that(records).containsInAnyOrder(EXPECTED_GENERICRECORD1); - pipeline.run(); - } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/AvroGenerators.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/AvroGenerators.java deleted file mode 100644 index 5b3127a80ab5..000000000000 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/AvroGenerators.java +++ /dev/null @@ -1,220 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.schemas.utils; - -import com.pholser.junit.quickcheck.generator.GenerationStatus; -import com.pholser.junit.quickcheck.generator.Generator; -import com.pholser.junit.quickcheck.random.SourceOfRandomness; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.function.Function; -import java.util.stream.Collectors; -import java.util.stream.IntStream; -import org.apache.avro.Schema; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Joiner; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ObjectArrays; - -/** QuickCheck generators for AVRO. */ -class AvroGenerators { - - /** Generates arbitrary AVRO schemas. */ - public static class SchemaGenerator extends BaseSchemaGenerator { - - public static final SchemaGenerator INSTANCE = new SchemaGenerator(); - - private static final ImmutableList<Schema.Type> PRIMITIVE_TYPES = - ImmutableList.of( - Schema.Type.STRING, - Schema.Type.BYTES, - Schema.Type.INT, - Schema.Type.LONG, - Schema.Type.FLOAT, - Schema.Type.DOUBLE, - Schema.Type.BOOLEAN); - - private static final ImmutableList<Schema.Type> ALL_TYPES = - ImmutableList.<Schema.Type>builder() - .addAll(PRIMITIVE_TYPES) - .add(Schema.Type.FIXED) - .add(Schema.Type.ENUM) - .add(Schema.Type.RECORD) - .add(Schema.Type.ARRAY) - .add(Schema.Type.MAP) - .add(Schema.Type.UNION) - .add(Schema.Type.ARRAY) - .build(); - - private static final int MAX_NESTING = 10; - - @Override - public Schema generate(SourceOfRandomness random, GenerationStatus status) { - Schema.Type type; - - if (nesting(status) >= MAX_NESTING) { - type = random.choose(PRIMITIVE_TYPES); - } else { - type = random.choose(ALL_TYPES); - } - - if (PRIMITIVE_TYPES.contains(type)) { - return Schema.create(type); - } else { - nestingInc(status); - - if (type == Schema.Type.FIXED) { - int size = random.choose(Arrays.asList(1, 5, 12)); - return Schema.createFixed("fixed_" + branch(status), "", "", size); - } else if (type == Schema.Type.UNION) { - // only nullable fields, everything else isn't supported in row conversion code - return UnionSchemaGenerator.INSTANCE.generate(random, status); - } else if (type == Schema.Type.ENUM) { - return EnumSchemaGenerator.INSTANCE.generate(random, status); - } else if (type == Schema.Type.RECORD) { - return RecordSchemaGenerator.INSTANCE.generate(random, status); - } else if (type == Schema.Type.MAP) { - return Schema.createMap(generate(random, status)); - } else if (type == Schema.Type.ARRAY) { - return Schema.createArray(generate(random, status)); - } else { - throw new AssertionError("Unexpected AVRO type: " + type); - } - } - } - } - - public static class RecordSchemaGenerator extends BaseSchemaGenerator { - - public static final RecordSchemaGenerator INSTANCE = new RecordSchemaGenerator(); - - @Override - public Schema generate(SourceOfRandomness random, GenerationStatus status) { - List<Schema.Field> fields = - IntStream.range(0, random.nextInt(0, status.size()) + 1) - .mapToObj( - i -> { - // deterministically avoid collisions in record names - branchPush(status, String.valueOf(i)); - Schema.Field field = - createField(i, SchemaGenerator.INSTANCE.generate(random, status)); - branchPop(status); - return field; - }) - .collect(Collectors.toList()); - - return Schema.createRecord("record_" + branch(status), "", "example", false, fields); - } - - private Schema.Field createField(int i, Schema schema) { - return new Schema.Field("field_" + i, schema, null, (Object) null); - } - } - - static class UnionSchemaGenerator extends BaseSchemaGenerator { - - public static final UnionSchemaGenerator INSTANCE = new UnionSchemaGenerator(); - - @Override - public Schema generate(SourceOfRandomness random, GenerationStatus status) { - Map<String, Schema> schemaMap = - IntStream.range(0, random.nextInt(0, status.size()) + 1) - .mapToObj( - i -> { - // deterministically avoid collisions in record names - branchPush(status, String.valueOf(i)); - Schema schema = - SchemaGenerator.INSTANCE - // nested unions aren't supported in AVRO - .filter(x -> x.getType() != Schema.Type.UNION) - .generate(random, status); - branchPop(status); - return schema; - }) - // AVRO requires uniqueness by full name - .collect(Collectors.toMap(Schema::getFullName, Function.identity(), (x, y) -> x)); - - List<Schema> schemas = new ArrayList<>(schemaMap.values()); - - if (random.nextBoolean()) { - org.apache.avro.Schema nullSchema = org.apache.avro.Schema.create(Schema.Type.NULL); - schemas.add(nullSchema); - Collections.shuffle(schemas, random.toJDKRandom()); - } - - return Schema.createUnion(schemas); - } - } - - static class EnumSchemaGenerator extends BaseSchemaGenerator { - - public static final EnumSchemaGenerator INSTANCE = new EnumSchemaGenerator(); - - private static final Schema FRUITS = - Schema.createEnum("Fruit", "", "example", Arrays.asList("banana", "apple", "pear")); - - private static final Schema STATUS = - Schema.createEnum("Status", "", "example", Arrays.asList("OK", "ERROR", "WARNING")); - - @Override - public Schema generate(final SourceOfRandomness random, final GenerationStatus status) { - return random.choose(Arrays.asList(FRUITS, STATUS)); - } - } - - abstract static class BaseSchemaGenerator extends Generator<Schema> { - - private static final GenerationStatus.Key<Integer> NESTING_KEY = - new GenerationStatus.Key<>("nesting", Integer.class); - - private static final GenerationStatus.Key<String[]> BRANCH_KEY = - new GenerationStatus.Key<>("branch", String[].class); - - BaseSchemaGenerator() { - super(org.apache.avro.Schema.class); - } - - void branchPush(GenerationStatus status, String value) { - String[] current = status.valueOf(BRANCH_KEY).orElse(new String[0]); - String[] next = ObjectArrays.concat(current, value); - - status.setValue(BRANCH_KEY, next); - } - - void branchPop(GenerationStatus status) { - String[] current = status.valueOf(BRANCH_KEY).orElse(new String[0]); - String[] next = Arrays.copyOf(current, current.length - 1); - - status.setValue(BRANCH_KEY, next); - } - - String branch(GenerationStatus status) { - return Joiner.on("_").join(status.valueOf(BRANCH_KEY).orElse(new String[0])); - } - - int nesting(GenerationStatus status) { - return status.valueOf(NESTING_KEY).orElse(0); - } - - void nestingInc(GenerationStatus status) { - status.setValue(NESTING_KEY, nesting(status) + 1); - } - } -} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/AvroUtilsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/AvroUtilsTest.java deleted file mode 100644 index 5abffd308650..000000000000 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/AvroUtilsTest.java +++ /dev/null @@ -1,914 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.schemas.utils; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; - -import com.pholser.junit.quickcheck.From; -import com.pholser.junit.quickcheck.Property; -import com.pholser.junit.quickcheck.runner.JUnitQuickcheck; -import java.math.BigDecimal; -import java.nio.ByteBuffer; -import java.sql.JDBCType; -import java.util.List; -import java.util.Map; -import org.apache.avro.Conversions; -import org.apache.avro.LogicalType; -import org.apache.avro.LogicalTypes; -import org.apache.avro.RandomData; -import org.apache.avro.Schema.Type; -import org.apache.avro.generic.GenericRecord; -import org.apache.avro.generic.GenericRecordBuilder; -import org.apache.avro.reflect.ReflectData; -import org.apache.avro.util.Utf8; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.coders.AvroCoder; -import org.apache.beam.sdk.io.AvroGeneratedUser; -import org.apache.beam.sdk.schemas.Schema; -import org.apache.beam.sdk.schemas.Schema.Field; -import org.apache.beam.sdk.schemas.Schema.FieldType; -import org.apache.beam.sdk.schemas.logicaltypes.EnumerationType; -import org.apache.beam.sdk.schemas.logicaltypes.OneOfType; -import org.apache.beam.sdk.schemas.logicaltypes.SqlTypes; -import org.apache.beam.sdk.schemas.utils.AvroGenerators.RecordSchemaGenerator; -import org.apache.beam.sdk.schemas.utils.AvroUtils.TypeWithNullability; -import org.apache.beam.sdk.testing.CoderProperties; -import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.transforms.SimpleFunction; -import org.apache.beam.sdk.util.SerializableUtils; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps; -import org.checkerframework.checker.nullness.qual.NonNull; -import org.checkerframework.checker.nullness.qual.Nullable; -import org.joda.time.DateTime; -import org.joda.time.DateTimeZone; -import org.joda.time.Days; -import org.joda.time.Instant; -import org.joda.time.LocalTime; -import org.junit.Test; -import org.junit.runner.RunWith; - -/** Tests for conversion between AVRO records and Beam rows. */ -@RunWith(JUnitQuickcheck.class) -@SuppressWarnings({ - "rawtypes", // TODO(https://github.com/apache/beam/issues/20447) -}) -public class AvroUtilsTest { - - private static final org.apache.avro.Schema NULL_SCHEMA = - org.apache.avro.Schema.create(Type.NULL); - - @Property(trials = 1000) - @SuppressWarnings("unchecked") - public void supportsAnyAvroSchema( - @From(RecordSchemaGenerator.class) org.apache.avro.Schema avroSchema) { - - Schema schema = AvroUtils.toBeamSchema(avroSchema); - Iterable iterable = new RandomData(avroSchema, 10); - List<GenericRecord> records = Lists.newArrayList((Iterable<GenericRecord>) iterable); - - for (GenericRecord record : records) { - AvroUtils.toBeamRowStrict(record, schema); - } - } - - @Property(trials = 1000) - @SuppressWarnings("unchecked") - public void avroToBeamRoundTrip( - @From(RecordSchemaGenerator.class) org.apache.avro.Schema avroSchema) { - - Schema schema = AvroUtils.toBeamSchema(avroSchema); - Iterable iterable = new RandomData(avroSchema, 10); - List<GenericRecord> records = Lists.newArrayList((Iterable<GenericRecord>) iterable); - - for (GenericRecord record : records) { - Row row = AvroUtils.toBeamRowStrict(record, schema); - GenericRecord out = AvroUtils.toGenericRecord(row, avroSchema); - assertEquals(record, out); - } - } - - @Test - public void testUnwrapNullableSchema() { - org.apache.avro.Schema avroSchema = - org.apache.avro.Schema.createUnion( - org.apache.avro.Schema.create(Type.NULL), org.apache.avro.Schema.create(Type.STRING)); - - TypeWithNullability typeWithNullability = new TypeWithNullability(avroSchema); - assertTrue(typeWithNullability.nullable); - assertEquals(org.apache.avro.Schema.create(Type.STRING), typeWithNullability.type); - } - - @Test - public void testUnwrapNullableSchemaReordered() { - org.apache.avro.Schema avroSchema = - org.apache.avro.Schema.createUnion( - org.apache.avro.Schema.create(Type.STRING), org.apache.avro.Schema.create(Type.NULL)); - - TypeWithNullability typeWithNullability = new TypeWithNullability(avroSchema); - assertTrue(typeWithNullability.nullable); - assertEquals(org.apache.avro.Schema.create(Type.STRING), typeWithNullability.type); - } - - @Test - public void testUnwrapNullableSchemaToUnion() { - org.apache.avro.Schema avroSchema = - org.apache.avro.Schema.createUnion( - org.apache.avro.Schema.create(Type.STRING), - org.apache.avro.Schema.create(Type.LONG), - org.apache.avro.Schema.create(Type.NULL)); - - TypeWithNullability typeWithNullability = new TypeWithNullability(avroSchema); - assertTrue(typeWithNullability.nullable); - assertEquals( - org.apache.avro.Schema.createUnion( - org.apache.avro.Schema.create(Type.STRING), org.apache.avro.Schema.create(Type.LONG)), - typeWithNullability.type); - } - - @Test - public void testNullableArrayFieldToBeamArrayField() { - org.apache.avro.Schema.Field avroField = - new org.apache.avro.Schema.Field( - "arrayField", - ReflectData.makeNullable( - org.apache.avro.Schema.createArray(org.apache.avro.Schema.create(Type.INT))), - "", - null); - - Field expectedBeamField = Field.nullable("arrayField", FieldType.array(FieldType.INT32)); - - Field beamField = AvroUtils.toBeamField(avroField); - assertEquals(expectedBeamField, beamField); - } - - @Test - public void testNullableBeamArrayFieldToAvroField() { - Field beamField = Field.nullable("arrayField", FieldType.array(FieldType.INT32)); - - org.apache.avro.Schema.Field expectedAvroField = - new org.apache.avro.Schema.Field( - "arrayField", - ReflectData.makeNullable( - org.apache.avro.Schema.createArray(org.apache.avro.Schema.create(Type.INT))), - "", - null); - - org.apache.avro.Schema.Field avroField = AvroUtils.toAvroField(beamField, "ignored"); - assertEquals(expectedAvroField, avroField); - } - - private static List<org.apache.avro.Schema.Field> getAvroSubSchemaFields() { - List<org.apache.avro.Schema.Field> fields = Lists.newArrayList(); - fields.add( - new org.apache.avro.Schema.Field( - "bool", org.apache.avro.Schema.create(Type.BOOLEAN), "", null)); - fields.add( - new org.apache.avro.Schema.Field("int", org.apache.avro.Schema.create(Type.INT), "", null)); - return fields; - } - - private static org.apache.avro.Schema getAvroSubSchema(String name) { - return org.apache.avro.Schema.createRecord( - name, null, "topLevelRecord", false, getAvroSubSchemaFields()); - } - - private static org.apache.avro.Schema getAvroSchema() { - List<org.apache.avro.Schema.Field> fields = Lists.newArrayList(); - fields.add( - new org.apache.avro.Schema.Field( - "bool", org.apache.avro.Schema.create(Type.BOOLEAN), "", (Object) null)); - fields.add( - new org.apache.avro.Schema.Field( - "int", org.apache.avro.Schema.create(Type.INT), "", (Object) null)); - fields.add( - new org.apache.avro.Schema.Field( - "long", org.apache.avro.Schema.create(Type.LONG), "", (Object) null)); - fields.add( - new org.apache.avro.Schema.Field( - "float", org.apache.avro.Schema.create(Type.FLOAT), "", (Object) null)); - fields.add( - new org.apache.avro.Schema.Field( - "double", org.apache.avro.Schema.create(Type.DOUBLE), "", (Object) null)); - fields.add( - new org.apache.avro.Schema.Field( - "string", org.apache.avro.Schema.create(Type.STRING), "", (Object) null)); - fields.add( - new org.apache.avro.Schema.Field( - "bytes", org.apache.avro.Schema.create(Type.BYTES), "", (Object) null)); - fields.add( - new org.apache.avro.Schema.Field( - "decimal", - LogicalTypes.decimal(Integer.MAX_VALUE) - .addToSchema(org.apache.avro.Schema.create(Type.BYTES)), - "", - (Object) null)); - fields.add( - new org.apache.avro.Schema.Field( - "timestampMillis", - LogicalTypes.timestampMillis().addToSchema(org.apache.avro.Schema.create(Type.LONG)), - "", - (Object) null)); - fields.add(new org.apache.avro.Schema.Field("row", getAvroSubSchema("row"), "", (Object) null)); - fields.add( - new org.apache.avro.Schema.Field( - "array", - org.apache.avro.Schema.createArray(getAvroSubSchema("array")), - "", - (Object) null)); - fields.add( - new org.apache.avro.Schema.Field( - "map", org.apache.avro.Schema.createMap(getAvroSubSchema("map")), "", (Object) null)); - return org.apache.avro.Schema.createRecord("topLevelRecord", null, null, false, fields); - } - - private static Schema getBeamSubSchema() { - return new Schema.Builder() - .addField(Field.of("bool", FieldType.BOOLEAN)) - .addField(Field.of("int", FieldType.INT32)) - .build(); - } - - private Schema getBeamSchema() { - Schema subSchema = getBeamSubSchema(); - return new Schema.Builder() - .addField(Field.of("bool", FieldType.BOOLEAN)) - .addField(Field.of("int", FieldType.INT32)) - .addField(Field.of("long", FieldType.INT64)) - .addField(Field.of("float", FieldType.FLOAT)) - .addField(Field.of("double", FieldType.DOUBLE)) - .addField(Field.of("string", FieldType.STRING)) - .addField(Field.of("bytes", FieldType.BYTES)) - .addField(Field.of("decimal", FieldType.DECIMAL)) - .addField(Field.of("timestampMillis", FieldType.DATETIME)) - .addField(Field.of("row", FieldType.row(subSchema))) - .addField(Field.of("array", FieldType.array(FieldType.row(subSchema)))) - .addField(Field.of("map", FieldType.map(FieldType.STRING, FieldType.row(subSchema)))) - .build(); - } - - private static final byte[] BYTE_ARRAY = new byte[] {1, 2, 3, 4}; - private static final DateTime DATE_TIME = - new DateTime().withDate(1979, 3, 14).withTime(1, 2, 3, 4).withZone(DateTimeZone.UTC); - private static final BigDecimal BIG_DECIMAL = new BigDecimal(3600); - - private Row getBeamRow() { - Row subRow = Row.withSchema(getBeamSubSchema()).addValues(true, 42).build(); - return Row.withSchema(getBeamSchema()) - .addValue(true) - .addValue(43) - .addValue(44L) - .addValue((float) 44.1) - .addValue((double) 44.2) - .addValue("string") - .addValue(BYTE_ARRAY) - .addValue(BIG_DECIMAL) - .addValue(DATE_TIME) - .addValue(subRow) - .addValue(ImmutableList.of(subRow, subRow)) - .addValue(ImmutableMap.of("k1", subRow, "k2", subRow)) - .build(); - } - - private static GenericRecord getSubGenericRecord(String name) { - return new GenericRecordBuilder(getAvroSubSchema(name)) - .set("bool", true) - .set("int", 42) - .build(); - } - - private static GenericRecord getGenericRecord() { - - LogicalType decimalType = - LogicalTypes.decimal(Integer.MAX_VALUE) - .addToSchema(org.apache.avro.Schema.create(Type.BYTES)) - .getLogicalType(); - ByteBuffer encodedDecimal = - new Conversions.DecimalConversion().toBytes(BIG_DECIMAL, null, decimalType); - - return new GenericRecordBuilder(getAvroSchema()) - .set("bool", true) - .set("int", 43) - .set("long", 44L) - .set("float", (float) 44.1) - .set("double", (double) 44.2) - .set("string", new Utf8("string")) - .set("bytes", ByteBuffer.wrap(BYTE_ARRAY)) - .set("decimal", encodedDecimal) - .set("timestampMillis", DATE_TIME.getMillis()) - .set("row", getSubGenericRecord("row")) - .set("array", ImmutableList.of(getSubGenericRecord("array"), getSubGenericRecord("array"))) - .set( - "map", - ImmutableMap.of( - new Utf8("k1"), - getSubGenericRecord("map"), - new Utf8("k2"), - getSubGenericRecord("map"))) - .build(); - } - - @Test - public void testFromAvroSchema() { - assertEquals(getBeamSchema(), AvroUtils.toBeamSchema(getAvroSchema())); - } - - @Test - public void testFromBeamSchema() { - Schema beamSchema = getBeamSchema(); - org.apache.avro.Schema avroSchema = AvroUtils.toAvroSchema(beamSchema); - assertEquals(getAvroSchema(), avroSchema); - } - - @Test - public void testAvroSchemaFromBeamSchemaCanBeParsed() { - org.apache.avro.Schema convertedSchema = AvroUtils.toAvroSchema(getBeamSchema()); - org.apache.avro.Schema validatedSchema = - new org.apache.avro.Schema.Parser().parse(convertedSchema.toString()); - assertEquals(convertedSchema, validatedSchema); - } - - @Test - public void testAvroSchemaFromBeamSchemaWithFieldCollisionCanBeParsed() { - - // Two similar schemas, the only difference is the "street" field type in the nested record. - Schema contact = - new Schema.Builder() - .addField(Field.of("name", FieldType.STRING)) - .addField( - Field.of( - "address", - FieldType.row( - new Schema.Builder() - .addField(Field.of("street", FieldType.STRING)) - .addField(Field.of("city", FieldType.STRING)) - .build()))) - .build(); - - Schema contactMultiline = - new Schema.Builder() - .addField(Field.of("name", FieldType.STRING)) - .addField( - Field.of( - "address", - FieldType.row( - new Schema.Builder() - .addField(Field.of("street", FieldType.array(FieldType.STRING))) - .addField(Field.of("city", FieldType.STRING)) - .build()))) - .build(); - - // Ensure that no collisions happen between two sibling fields with same-named child fields - // (with different schemas, between a parent field and a sub-record field with the same name, - // and artificially with the generated field name. - Schema beamSchema = - new Schema.Builder() - .addField(Field.of("home", FieldType.row(contact))) - .addField(Field.of("work", FieldType.row(contactMultiline))) - .addField(Field.of("address", FieldType.row(contact))) - .addField(Field.of("topLevelRecord", FieldType.row(contactMultiline))) - .build(); - - org.apache.avro.Schema convertedSchema = AvroUtils.toAvroSchema(beamSchema); - org.apache.avro.Schema validatedSchema = - new org.apache.avro.Schema.Parser().parse(convertedSchema.toString()); - assertEquals(convertedSchema, validatedSchema); - } - - @Test - public void testNullableFieldInAvroSchema() { - List<org.apache.avro.Schema.Field> fields = Lists.newArrayList(); - fields.add( - new org.apache.avro.Schema.Field( - "int", ReflectData.makeNullable(org.apache.avro.Schema.create(Type.INT)), "", null)); - fields.add( - new org.apache.avro.Schema.Field( - "array", - org.apache.avro.Schema.createArray( - ReflectData.makeNullable(org.apache.avro.Schema.create(Type.BYTES))), - "", - null)); - fields.add( - new org.apache.avro.Schema.Field( - "map", - org.apache.avro.Schema.createMap( - ReflectData.makeNullable(org.apache.avro.Schema.create(Type.INT))), - "", - null)); - fields.add( - new org.apache.avro.Schema.Field( - "enum", - ReflectData.makeNullable( - org.apache.avro.Schema.createEnum( - "fruit", "", "", ImmutableList.of("banana", "apple", "pear"))), - "", - null)); - - org.apache.avro.Schema avroSchema = - org.apache.avro.Schema.createRecord("topLevelRecord", null, null, false, fields); - - Schema expectedSchema = - Schema.builder() - .addNullableField("int", FieldType.INT32) - .addArrayField("array", FieldType.BYTES.withNullable(true)) - .addMapField("map", FieldType.STRING, FieldType.INT32.withNullable(true)) - .addField( - "enum", - FieldType.logicalType(EnumerationType.create("banana", "apple", "pear")) - .withNullable(true)) - .build(); - assertEquals(expectedSchema, AvroUtils.toBeamSchema(avroSchema)); - - Map<String, Object> nullMap = Maps.newHashMap(); - nullMap.put("k1", null); - GenericRecord genericRecord = - new GenericRecordBuilder(avroSchema) - .set("int", null) - .set("array", Lists.newArrayList((Object) null)) - .set("map", nullMap) - .set("enum", null) - .build(); - Row expectedRow = - Row.withSchema(expectedSchema) - .addValue(null) - .addValue(Lists.newArrayList((Object) null)) - .addValue(nullMap) - .addValue(null) - .build(); - assertEquals(expectedRow, AvroUtils.toBeamRowStrict(genericRecord, expectedSchema)); - } - - @Test - public void testNullableFieldsInBeamSchema() { - Schema beamSchema = - Schema.builder() - .addNullableField("int", FieldType.INT32) - .addArrayField("array", FieldType.INT32.withNullable(true)) - .addMapField("map", FieldType.STRING, FieldType.INT32.withNullable(true)) - .build(); - - List<org.apache.avro.Schema.Field> fields = Lists.newArrayList(); - fields.add( - new org.apache.avro.Schema.Field( - "int", ReflectData.makeNullable(org.apache.avro.Schema.create(Type.INT)), "", null)); - fields.add( - new org.apache.avro.Schema.Field( - "array", - org.apache.avro.Schema.createArray( - ReflectData.makeNullable(org.apache.avro.Schema.create(Type.INT))), - "", - null)); - fields.add( - new org.apache.avro.Schema.Field( - "map", - org.apache.avro.Schema.createMap( - ReflectData.makeNullable(org.apache.avro.Schema.create(Type.INT))), - "", - null)); - org.apache.avro.Schema avroSchema = - org.apache.avro.Schema.createRecord("topLevelRecord", null, null, false, fields); - assertEquals(avroSchema, AvroUtils.toAvroSchema(beamSchema)); - - Map<Utf8, Object> nullMapUtf8 = Maps.newHashMap(); - nullMapUtf8.put(new Utf8("k1"), null); - Map<String, Object> nullMapString = Maps.newHashMap(); - nullMapString.put("k1", null); - - GenericRecord expectedGenericRecord = - new GenericRecordBuilder(avroSchema) - .set("int", null) - .set("array", Lists.newArrayList((Object) null)) - .set("map", nullMapUtf8) - .build(); - Row row = - Row.withSchema(beamSchema) - .addValue(null) - .addValue(Lists.newArrayList((Object) null)) - .addValue(nullMapString) - .build(); - assertEquals(expectedGenericRecord, AvroUtils.toGenericRecord(row, avroSchema)); - } - - @Test - public void testUnionFieldInAvroSchema() { - - List<org.apache.avro.Schema.Field> fields = Lists.newArrayList(); - List<org.apache.avro.Schema> unionFields = Lists.newArrayList(); - - unionFields.add(org.apache.avro.Schema.create(Type.INT)); - unionFields.add(org.apache.avro.Schema.create(Type.STRING)); - - fields.add( - new org.apache.avro.Schema.Field( - "union", org.apache.avro.Schema.createUnion(unionFields), "", null)); - org.apache.avro.Schema avroSchema = - org.apache.avro.Schema.createRecord("topLevelRecord", null, null, false, fields); - OneOfType oneOfType = - OneOfType.create(Field.of("int", FieldType.INT32), Field.of("string", FieldType.STRING)); - - Schema expectedSchema = Schema.builder().addLogicalTypeField("union", oneOfType).build(); - assertEquals(expectedSchema, AvroUtils.toBeamSchema(avroSchema)); - GenericRecord genericRecord = new GenericRecordBuilder(avroSchema).set("union", 23423).build(); - Row expectedRow = - Row.withSchema(expectedSchema).addValue(oneOfType.createValue(0, 23423)).build(); - assertEquals(expectedRow, AvroUtils.toBeamRowStrict(genericRecord, expectedSchema)); - } - - @Test - public void testUnionFieldInBeamSchema() { - OneOfType oneOfType = - OneOfType.create(Field.of("int", FieldType.INT32), Field.of("string", FieldType.STRING)); - - Schema beamSchema = Schema.builder().addLogicalTypeField("union", oneOfType).build(); - List<org.apache.avro.Schema.Field> fields = Lists.newArrayList(); - List<org.apache.avro.Schema> unionFields = Lists.newArrayList(); - - unionFields.add(org.apache.avro.Schema.create(Type.INT)); - unionFields.add(org.apache.avro.Schema.create(Type.STRING)); - fields.add( - new org.apache.avro.Schema.Field( - "union", org.apache.avro.Schema.createUnion(unionFields), "", null)); - org.apache.avro.Schema avroSchema = - org.apache.avro.Schema.createRecord("topLevelRecord", null, null, false, fields); - GenericRecord expectedGenericRecord = - new GenericRecordBuilder(avroSchema).set("union", 23423).build(); - Row row = Row.withSchema(beamSchema).addValue(oneOfType.createValue(0, 23423)).build(); - assertEquals(expectedGenericRecord, AvroUtils.toGenericRecord(row, avroSchema)); - } - - @Test - public void testJdbcLogicalVarCharRowDataToAvroSchema() { - String expectedAvroSchemaJson = - "{ " - + " \"name\": \"topLevelRecord\", " - + " \"type\": \"record\", " - + " \"fields\": [{ " - + " \"name\": \"my_varchar_field\", " - + " \"type\": {\"type\": \"string\", \"logicalType\": \"varchar\", \"maxLength\": 10}" - + " }, " - + " { " - + " \"name\": \"my_longvarchar_field\", " - + " \"type\": {\"type\": \"string\", \"logicalType\": \"varchar\", \"maxLength\": 50}" - + " }, " - + " { " - + " \"name\": \"my_nvarchar_field\", " - + " \"type\": {\"type\": \"string\", \"logicalType\": \"varchar\", \"maxLength\": 10}" - + " }, " - + " { " - + " \"name\": \"my_longnvarchar_field\", " - + " \"type\": {\"type\": \"string\", \"logicalType\": \"varchar\", \"maxLength\": 50}" - + " }, " - + " { " - + " \"name\": \"fixed_length_char_field\", " - + " \"type\": {\"type\": \"string\", \"logicalType\": \"char\", \"maxLength\": 25}" - + " } " - + " ] " - + "}"; - - Schema beamSchema = - Schema.builder() - .addField( - Field.of( - "my_varchar_field", FieldType.logicalType(JdbcType.StringType.varchar(10)))) - .addField( - Field.of( - "my_longvarchar_field", - FieldType.logicalType(JdbcType.StringType.longvarchar(50)))) - .addField( - Field.of( - "my_nvarchar_field", FieldType.logicalType(JdbcType.StringType.nvarchar(10)))) - .addField( - Field.of( - "my_longnvarchar_field", - FieldType.logicalType(JdbcType.StringType.longnvarchar(50)))) - .addField( - Field.of( - "fixed_length_char_field", - FieldType.logicalType(JdbcType.StringType.fixedLengthChar(25)))) - .build(); - - assertEquals( - new org.apache.avro.Schema.Parser().parse(expectedAvroSchemaJson), - AvroUtils.toAvroSchema(beamSchema)); - } - - @Test - public void testJdbcLogicalVarCharRowDataToGenericRecord() { - Schema beamSchema = - Schema.builder() - .addField( - Field.of( - "my_varchar_field", FieldType.logicalType(JdbcType.StringType.varchar(10)))) - .addField( - Field.of( - "my_longvarchar_field", - FieldType.logicalType(JdbcType.StringType.longvarchar(50)))) - .addField( - Field.of( - "my_nvarchar_field", FieldType.logicalType(JdbcType.StringType.nvarchar(10)))) - .addField( - Field.of( - "my_longnvarchar_field", - FieldType.logicalType(JdbcType.StringType.longnvarchar(50)))) - .build(); - - Row rowData = - Row.withSchema(beamSchema) - .addValue("varchar_value") - .addValue("longvarchar_value") - .addValue("nvarchar_value") - .addValue("longnvarchar_value") - .build(); - - org.apache.avro.Schema avroSchema = AvroUtils.toAvroSchema(beamSchema); - GenericRecord expectedRecord = - new GenericRecordBuilder(avroSchema) - .set("my_varchar_field", "varchar_value") - .set("my_longvarchar_field", "longvarchar_value") - .set("my_nvarchar_field", "nvarchar_value") - .set("my_longnvarchar_field", "longnvarchar_value") - .build(); - - assertEquals(expectedRecord, AvroUtils.toGenericRecord(rowData, avroSchema)); - } - - @Test - public void testJdbcLogicalDateAndTimeRowDataToAvroSchema() { - String expectedAvroSchemaJson = - "{ " - + " \"name\": \"topLevelRecord\", " - + " \"type\": \"record\", " - + " \"fields\": [{ " - + " \"name\": \"my_date_field\", " - + " \"type\": { \"type\": \"int\", \"logicalType\": \"date\" }" - + " }, " - + " { " - + " \"name\": \"my_time_field\", " - + " \"type\": { \"type\": \"int\", \"logicalType\": \"time-millis\" }" - + " }" - + " ] " - + "}"; - - Schema beamSchema = - Schema.builder() - .addField(Field.of("my_date_field", FieldType.logicalType(JdbcType.DATE))) - .addField(Field.of("my_time_field", FieldType.logicalType(JdbcType.TIME))) - .build(); - - assertEquals( - new org.apache.avro.Schema.Parser().parse(expectedAvroSchemaJson), - AvroUtils.toAvroSchema(beamSchema)); - } - - @Test - public void testJdbcLogicalDateAndTimeRowDataToGenericRecord() { - // Test Fixed clock at - DateTime testDateTime = DateTime.parse("2021-05-29T11:15:16.234Z"); - - Schema beamSchema = - Schema.builder() - .addField(Field.of("my_date_field", FieldType.logicalType(JdbcType.DATE))) - .addField(Field.of("my_time_field", FieldType.logicalType(JdbcType.TIME))) - .build(); - - Row rowData = - Row.withSchema(beamSchema) - .addValue(testDateTime.toLocalDate().toDateTime(LocalTime.MIDNIGHT).toInstant()) - .addValue(Instant.ofEpochMilli(testDateTime.toLocalTime().millisOfDay().get())) - .build(); - - int daysFromEpoch = - Days.daysBetween( - Instant.EPOCH, - testDateTime.toLocalDate().toDateTime(LocalTime.MIDNIGHT).toInstant()) - .getDays(); - int timeSinceMidNight = testDateTime.toLocalTime().getMillisOfDay(); - - org.apache.avro.Schema avroSchema = AvroUtils.toAvroSchema(beamSchema); - GenericRecord expectedRecord = - new GenericRecordBuilder(avroSchema) - .set("my_date_field", daysFromEpoch) - .set("my_time_field", timeSinceMidNight) - .build(); - - assertEquals(expectedRecord, AvroUtils.toGenericRecord(rowData, avroSchema)); - } - - @Test - public void testSqlTypesToGenericRecord() { - // SqlTypes to LogicalTypes.date conversion is one direction - java.time.LocalDate localDate = java.time.LocalDate.of(1979, 3, 14); - - Schema beamSchema = - Schema.builder() - .addField(Field.of("local_date", FieldType.logicalType(SqlTypes.DATE))) - .build(); - - Row rowData = Row.withSchema(beamSchema).addValue(localDate).build(); - - org.apache.avro.Schema avroSchema = AvroUtils.toAvroSchema(beamSchema); - GenericRecord expectedRecord = - new GenericRecordBuilder(avroSchema).set("local_date", localDate.toEpochDay()).build(); - - assertEquals(expectedRecord, AvroUtils.toGenericRecord(rowData, avroSchema)); - } - - @Test - public void testBeamRowToGenericRecord() { - GenericRecord genericRecord = AvroUtils.toGenericRecord(getBeamRow(), null); - assertEquals(getAvroSchema(), genericRecord.getSchema()); - assertEquals(getGenericRecord(), genericRecord); - } - - @Test - public void testBeamRowToGenericRecordInferSchema() { - GenericRecord genericRecord = AvroUtils.toGenericRecord(getBeamRow()); - assertEquals(getAvroSchema(), genericRecord.getSchema()); - assertEquals(getGenericRecord(), genericRecord); - } - - @Test - public void testRowToGenericRecordFunction() { - SerializableUtils.ensureSerializable(AvroUtils.getRowToGenericRecordFunction(NULL_SCHEMA)); - SerializableUtils.ensureSerializable(AvroUtils.getRowToGenericRecordFunction(null)); - } - - @Test - public void testGenericRecordToBeamRow() { - GenericRecord genericRecord = getGenericRecord(); - Row row = AvroUtils.toBeamRowStrict(getGenericRecord(), null); - assertEquals(getBeamRow(), row); - - // Alternatively, a timestamp-millis logical type can have a joda datum. - genericRecord.put("timestampMillis", new DateTime(genericRecord.get("timestampMillis"))); - row = AvroUtils.toBeamRowStrict(getGenericRecord(), null); - assertEquals(getBeamRow(), row); - } - - @Test - public void testGenericRecordToRowFunction() { - SerializableUtils.ensureSerializable(AvroUtils.getGenericRecordToRowFunction(Schema.of())); - SerializableUtils.ensureSerializable(AvroUtils.getGenericRecordToRowFunction(null)); - } - - @Test - public void testAvroSchemaCoders() { - Pipeline pipeline = Pipeline.create(); - org.apache.avro.Schema schema = - org.apache.avro.Schema.createRecord( - "TestSubRecord", - "TestSubRecord doc", - "org.apache.beam.sdk.schemas.utils", - false, - getAvroSubSchemaFields()); - GenericRecord record = - new GenericRecordBuilder(getAvroSubSchema("simple")) - .set("bool", true) - .set("int", 42) - .build(); - - PCollection<GenericRecord> records = - pipeline.apply(Create.of(record).withCoder(AvroCoder.of(schema))); - assertFalse(records.hasSchema()); - records.setCoder(AvroUtils.schemaCoder(schema)); - assertTrue(records.hasSchema()); - CoderProperties.coderSerializable(records.getCoder()); - - AvroGeneratedUser user = new AvroGeneratedUser("foo", 42, "green"); - PCollection<AvroGeneratedUser> users = - pipeline.apply(Create.of(user).withCoder(AvroCoder.of(AvroGeneratedUser.class))); - assertFalse(users.hasSchema()); - users.setCoder(AvroUtils.schemaCoder((AvroCoder<AvroGeneratedUser>) users.getCoder())); - assertTrue(users.hasSchema()); - CoderProperties.coderSerializable(users.getCoder()); - } - - @Test - public void testAvroBytesToRowAndRowToAvroBytesFunctions() { - Schema schema = - Schema.builder() - .addInt32Field("f_int") - .addInt64Field("f_long") - .addDoubleField("f_double") - .addStringField("f_string") - .build(); - - SimpleFunction<Row, byte[]> toBytesFn = AvroUtils.getRowToAvroBytesFunction(schema); - SimpleFunction<byte[], Row> toRowFn = AvroUtils.getAvroBytesToRowFunction(schema); - - Row row = Row.withSchema(schema).attachValues(1, 1L, 1d, "string"); - - byte[] serializedRow = toBytesFn.apply(row); - Row deserializedRow = toRowFn.apply(serializedRow); - - assertEquals(row, deserializedRow); - } - - @Test - public void testNullSchemas() { - assertEquals( - AvroUtils.getFromRowFunction(GenericRecord.class), - AvroUtils.getFromRowFunction(GenericRecord.class)); - } - - /** Helper class that simulate JDBC Logical types. */ - private static class JdbcType<T> implements Schema.LogicalType<T, T> { - - private static final JdbcType<Instant> DATE = - new JdbcType<>(JDBCType.DATE, FieldType.STRING, FieldType.DATETIME, ""); - private static final JdbcType<Instant> TIME = - new JdbcType<>(JDBCType.TIME, FieldType.STRING, FieldType.DATETIME, ""); - - private final String identifier; - private final FieldType argumentType; - private final FieldType baseType; - private final Object argument; - - private static class StringType extends JdbcType<String> { - - private static StringType fixedLengthChar(int size) { - return new StringType(JDBCType.CHAR, size); - } - - private static StringType varchar(int size) { - return new StringType(JDBCType.VARCHAR, size); - } - - private static StringType longvarchar(int size) { - return new StringType(JDBCType.LONGVARCHAR, size); - } - - private static StringType nvarchar(int size) { - return new StringType(JDBCType.NVARCHAR, size); - } - - private static StringType longnvarchar(int size) { - return new StringType(JDBCType.LONGNVARCHAR, size); - } - - private StringType(JDBCType type, int size) { - super(type, FieldType.INT32, FieldType.STRING, size); - } - } - - private JdbcType( - JDBCType jdbcType, FieldType argumentType, FieldType baseType, Object argument) { - this.identifier = jdbcType.getName(); - this.argumentType = argumentType; - this.baseType = baseType; - this.argument = argument; - } - - @Override - public String getIdentifier() { - return identifier; - } - - @Override - public @Nullable FieldType getArgumentType() { - return argumentType; - } - - @Override - public FieldType getBaseType() { - return baseType; - } - - @Override - @SuppressWarnings("TypeParameterUnusedInFormals") - public <T1> @Nullable T1 getArgument() { - return (T1) argument; - } - - @Override - public @NonNull T toBaseType(@NonNull T input) { - return input; - } - - @Override - public @NonNull T toInputType(@NonNull T base) { - return base; - } - } -} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/TestJavaBeans.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/TestJavaBeans.java index 2e616aa2321b..b5ad6f989d9e 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/TestJavaBeans.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/TestJavaBeans.java @@ -1362,13 +1362,13 @@ public int hashCode() { @DefaultSchema(JavaBeanSchema.class) public static class ParameterNullableBean { - @org.apache.avro.reflect.Nullable private Float value; + @Nullable private Float value; - public @org.apache.avro.reflect.Nullable Float getValue() { + public @Nullable Float getValue() { return value; } - public void setValue(@org.apache.avro.reflect.Nullable Float value) { + public void setValue(@Nullable Float value) { this.value = value; } } @@ -1379,14 +1379,14 @@ public void setValue(@org.apache.avro.reflect.Nullable Float value) { @DefaultSchema(JavaBeanSchema.class) public static class FieldWithDescriptionBean { - @org.apache.avro.reflect.Nullable private Float value; + @Nullable private Float value; @SchemaFieldDescription("This value is the value stored in the object as a float.") - public @org.apache.avro.reflect.Nullable Float getValue() { + public @Nullable Float getValue() { return value; } - public void setValue(@org.apache.avro.reflect.Nullable Float value) { + public void setValue(@Nullable Float value) { this.value = value; } } diff --git a/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/coders/DefaultCoderTest.java b/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/coders/DefaultCoderTest.java deleted file mode 100644 index 82991f191887..000000000000 --- a/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/coders/DefaultCoderTest.java +++ /dev/null @@ -1,61 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.extensions.avro.coders; - -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.instanceOf; - -import java.util.List; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.CoderRegistry; -import org.apache.beam.sdk.coders.DefaultCoder; -import org.apache.beam.sdk.coders.DefaultCoder.DefaultCoderProviderRegistrar.DefaultCoderProvider; -import org.apache.beam.sdk.coders.ListCoder; -import org.apache.beam.sdk.values.TypeDescriptor; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -/** Tests for {@link DefaultCoder}. */ -@RunWith(JUnit4.class) -public class DefaultCoderTest { - - @Rule public ExpectedException thrown = ExpectedException.none(); - - @DefaultCoder(AvroCoder.class) - private static class AvroRecord {} - - @Test - public void testCodersWithoutComponents() throws Exception { - CoderRegistry registry = CoderRegistry.createDefault(); - registry.registerCoderProvider(new DefaultCoderProvider()); - assertThat(registry.getCoder(AvroRecord.class), instanceOf(AvroCoder.class)); - } - - @Test - public void testDefaultCoderInCollection() throws Exception { - CoderRegistry registry = CoderRegistry.createDefault(); - registry.registerCoderProvider(new DefaultCoderProvider()); - Coder<List<AvroRecord>> avroRecordCoder = - registry.getCoder(new TypeDescriptor<List<AvroRecord>>() {}); - assertThat(avroRecordCoder, instanceOf(ListCoder.class)); - assertThat(((ListCoder) avroRecordCoder).getElemCoder(), instanceOf(AvroCoder.class)); - } -} diff --git a/sdks/java/extensions/timeseries/build.gradle b/sdks/java/extensions/timeseries/build.gradle index 79d3957e6197..86bf89d72920 100644 --- a/sdks/java/extensions/timeseries/build.gradle +++ b/sdks/java/extensions/timeseries/build.gradle @@ -29,4 +29,5 @@ dependencies { implementation project(path: ":sdks:java:core", configuration: "shadow") testImplementation library.java.junit testRuntimeOnly project(path: ":runners:direct-java", configuration: "shadow") + testImplementation project(path: ":sdks:java:extensions:avro", configuration: "testRuntimeMigration") } diff --git a/sdks/java/extensions/zetasketch/build.gradle b/sdks/java/extensions/zetasketch/build.gradle index 3c745408e937..bb532ad08aa1 100644 --- a/sdks/java/extensions/zetasketch/build.gradle +++ b/sdks/java/extensions/zetasketch/build.gradle @@ -42,6 +42,7 @@ dependencies { testImplementation project(":sdks:java:extensions:google-cloud-platform-core") testImplementation library.java.google_api_services_bigquery testImplementation library.java.proto_google_cloud_bigquery_storage_v1 + testImplementation project(path: ":sdks:java:extensions:avro", configuration: "testRuntimeMigration") testRuntimeOnly library.java.slf4j_simple testRuntimeOnly project(path: ":runners:direct-java", configuration: "shadow") testRuntimeOnly project(":runners:google-cloud-dataflow-java") diff --git a/sdks/java/harness/build.gradle b/sdks/java/harness/build.gradle index 25d6b2ac4040..50eed2f2573b 100644 --- a/sdks/java/harness/build.gradle +++ b/sdks/java/harness/build.gradle @@ -47,18 +47,24 @@ applyJavaNature( automaticModuleName: 'org.apache.beam.fn.harness', testShadowJar: true, shadowJarValidationExcludes: [ + "avro/shaded/com/google/**", + "com/thoughtworks/paranamer/**", "junit/**", "io/github/classgraph/**", "nonapi/io/github/classgraph/**", + "org/apache/avro/**", "org/apache/beam/fn/harness/**", "org/apache/beam/model/fnexecution/**", "org/apache/beam/runners/core/**", "org/apache/beam/runners/core/construction/**", "org/apache/beam/sdk/extensions/avro/**", "org/apache/beam/sdk/fn/**", + "org/apache/commons/**", "org/checkerframework/**", + "org/codehaus/jackson/**", "org/hamcrest/**", "org/junit/**", + "org/tukaani/xz/**", ], shadowClosure: { diff --git a/sdks/java/io/clickhouse/build.gradle b/sdks/java/io/clickhouse/build.gradle index d61dcbe2660a..d711fb7fa316 100644 --- a/sdks/java/io/clickhouse/build.gradle +++ b/sdks/java/io/clickhouse/build.gradle @@ -66,6 +66,7 @@ dependencies { testImplementation library.java.testcontainers_clickhouse testRuntimeOnly library.java.slf4j_jdk14 testRuntimeOnly project(path: ":runners:direct-java", configuration: "shadow") + testImplementation project(path: ":sdks:java:extensions:avro", configuration: "testRuntimeMigration") } processTestResources { diff --git a/sdks/java/io/contextualtextio/build.gradle b/sdks/java/io/contextualtextio/build.gradle index c54ce15bb6c0..0557a1dfa259 100644 --- a/sdks/java/io/contextualtextio/build.gradle +++ b/sdks/java/io/contextualtextio/build.gradle @@ -33,6 +33,7 @@ dependencies { implementation library.java.vendored_grpc_1_54_0 testImplementation project(path: ":sdks:java:core", configuration: "shadowTest") + testImplementation library.java.commons_compress testImplementation library.java.guava_testlib testImplementation library.java.junit testImplementation library.java.hamcrest diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformProvider.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformProvider.java index 8cdd6aa9ab1b..c1f6b2b31754 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformProvider.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformProvider.java @@ -24,6 +24,7 @@ import java.util.Collections; import java.util.List; import java.util.Set; +import org.apache.beam.sdk.extensions.avro.schemas.utils.AvroUtils; import org.apache.beam.sdk.io.gcp.pubsub.PubsubTestClient.PubsubTestClientFactory; import org.apache.beam.sdk.metrics.Counter; import org.apache.beam.sdk.metrics.Metrics; @@ -31,7 +32,6 @@ import org.apache.beam.sdk.schemas.transforms.SchemaTransform; import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider; import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider; -import org.apache.beam.sdk.schemas.utils.AvroUtils; import org.apache.beam.sdk.schemas.utils.JsonUtils; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.ParDo; diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubWriteSchemaTransformProvider.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubWriteSchemaTransformProvider.java index 1d11eeaca6ca..6187f6f79d3e 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubWriteSchemaTransformProvider.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubWriteSchemaTransformProvider.java @@ -27,11 +27,11 @@ import java.util.Objects; import java.util.Set; import javax.annotation.Nullable; +import org.apache.beam.sdk.extensions.avro.schemas.utils.AvroUtils; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.transforms.SchemaTransform; import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider; import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider; -import org.apache.beam.sdk.schemas.utils.AvroUtils; import org.apache.beam.sdk.schemas.utils.JsonUtils; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.ParDo; diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIOTest.java index c11a071ab0eb..f7f9f5f91b74 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIOTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIOTest.java @@ -51,10 +51,10 @@ import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; +import org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser; import org.apache.beam.sdk.extensions.protobuf.Proto3SchemaMessages.Primitive; import org.apache.beam.sdk.extensions.protobuf.ProtoCoder; import org.apache.beam.sdk.extensions.protobuf.ProtoDomain; -import org.apache.beam.sdk.io.AvroGeneratedUser; import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.IncomingMessage; import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.OutgoingMessage; import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.SubscriptionPath; diff --git a/sdks/java/io/kafka/build.gradle b/sdks/java/io/kafka/build.gradle index 63eba6edcd8f..61209aa50928 100644 --- a/sdks/java/io/kafka/build.gradle +++ b/sdks/java/io/kafka/build.gradle @@ -82,7 +82,7 @@ dependencies { provided library.java.everit_json_schema testImplementation project(path: ":sdks:java:core", configuration: "shadowTest") testImplementation project(":sdks:java:io:synthetic") - testImplementation project(":sdks:java:extensions:avro") + testImplementation project(path: ":sdks:java:extensions:avro", configuration: "testRuntimeMigration") testImplementation project(path: ":sdks:java:io:common", configuration: "testRuntimeMigration") testImplementation project(path: ":sdks:java:testing:test-utils", configuration: "testRuntimeMigration") // For testing Cross-language transforms diff --git a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java index a686c5f1ae9a..52ab3e20f793 100644 --- a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java +++ b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java @@ -68,7 +68,7 @@ import org.apache.beam.sdk.coders.BigEndianLongCoder; import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.coders.VarLongCoder; -import org.apache.beam.sdk.io.AvroGeneratedUser; +import org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser; import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.io.UnboundedSource; import org.apache.beam.sdk.io.UnboundedSource.UnboundedReader; diff --git a/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/read/SnowflakeIOReadTest.java b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/read/SnowflakeIOReadTest.java index 06cbd99f8e03..df70fd87aac9 100644 --- a/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/read/SnowflakeIOReadTest.java +++ b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/read/SnowflakeIOReadTest.java @@ -24,7 +24,7 @@ import org.apache.avro.generic.GenericRecordBuilder; import org.apache.beam.sdk.Pipeline.PipelineExecutionException; import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; -import org.apache.beam.sdk.io.AvroGeneratedUser; +import org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser; import org.apache.beam.sdk.io.snowflake.SnowflakeIO; import org.apache.beam.sdk.io.snowflake.services.SnowflakeServices; import org.apache.beam.sdk.io.snowflake.test.FakeSnowflakeBasicDataSource; diff --git a/sdks/java/io/thrift/build.gradle b/sdks/java/io/thrift/build.gradle index 6ee3314a74ff..47c8111a0257 100644 --- a/sdks/java/io/thrift/build.gradle +++ b/sdks/java/io/thrift/build.gradle @@ -42,6 +42,7 @@ dependencies { testImplementation library.java.junit testRuntimeOnly library.java.slf4j_jdk14 testRuntimeOnly project(path: ":runners:direct-java", configuration: "shadow") + testImplementation project(path: ":sdks:java:extensions:avro", configuration: "testRuntimeMigration") } /* Removed due to lack of Thrift on Jenkins workers. From bde60059f0987eb4cd0d6ac6a43135daaf0eb3f8 Mon Sep 17 00:00:00 2001 From: Anand Inguva <34158215+AnandInguva@users.noreply.github.com> Date: Tue, 17 Oct 2023 18:12:44 +0000 Subject: [PATCH 162/435] Log warning when gen_protos is not found (#29025) * Log warning when gen_protos is not found * Fix lint --- sdks/python/setup.py | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/sdks/python/setup.py b/sdks/python/setup.py index 4a05544526fc..83cbb98d3ddf 100644 --- a/sdks/python/setup.py +++ b/sdks/python/setup.py @@ -151,6 +151,11 @@ def cythonize(*args, **kwargs): 'pandas>=1.4.3,!=1.5.0,!=1.5.1,<1.6;python_version>="3.8"', ] +def find_by_ext(root_dir, ext): + for root, _, files in os.walk(root_dir): + for file in files: + if file.endswith(ext): + yield os.path.realpath(os.path.join(root, file)) # We must generate protos after setup_requires are installed. def generate_protos_first(): @@ -158,6 +163,19 @@ def generate_protos_first(): # Pyproject toml build happens in isolated environemnts. In those envs, # gen_protos is unable to get imported. so we run a subprocess call. cwd = os.path.abspath(os.path.dirname(__file__)) + # when pip install <>.tar.gz gets called, if gen_protos.py is not available + # in the sdist,then the proto files would have already been generated. So we + # skip proto generation in that case. + if not os.path.exists(os.path.join(cwd, 'gen_protos.py')): + # make sure we already generated protos + pb2_files = list(find_by_ext(os.path.join( + cwd, 'apache_beam', 'portability', 'api'), '_pb2.py')) + if not pb2_files: + raise RuntimeError('protobuf files are not generated. ' + 'Please generate pb2 files') + + warnings.warn('Skipping proto generation as they are already generated.') + return out = subprocess.run([ sys.executable, os.path.join(cwd, 'gen_protos.py'), From 0b50a173608c9264d682f0b4c941adbf6362f506 Mon Sep 17 00:00:00 2001 From: Vitaly Terentyev <vitaly.terentyev@akvelon.com> Date: Tue, 17 Oct 2023 22:38:28 +0400 Subject: [PATCH 163/435] Fix num workers (#29006) * Fix num workers * Remove groovy file * Return groovy file, add InfluxDB envs --- .github/workflows/beam_Inference_Python_Benchmarks_Dataflow.yml | 2 ++ ..._Pytorch_Imagenet_Classification_Resnet_152_Tesla_T4_GPU.txt | 2 +- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/.github/workflows/beam_Inference_Python_Benchmarks_Dataflow.yml b/.github/workflows/beam_Inference_Python_Benchmarks_Dataflow.yml index b6c10b6ef60b..97099ec525f1 100644 --- a/.github/workflows/beam_Inference_Python_Benchmarks_Dataflow.yml +++ b/.github/workflows/beam_Inference_Python_Benchmarks_Dataflow.yml @@ -47,6 +47,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} jobs: beam_Inference_Python_Benchmarks_Dataflow: diff --git a/.github/workflows/load-tests-job-configs/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Imagenet_Classification_Resnet_152_Tesla_T4_GPU.txt b/.github/workflows/load-tests-job-configs/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Imagenet_Classification_Resnet_152_Tesla_T4_GPU.txt index c1b7e273c6e8..e79369befad7 100644 --- a/.github/workflows/load-tests-job-configs/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Imagenet_Classification_Resnet_152_Tesla_T4_GPU.txt +++ b/.github/workflows/load-tests-job-configs/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Imagenet_Classification_Resnet_152_Tesla_T4_GPU.txt @@ -16,7 +16,7 @@ --region=us-central1 --machine_type=n1-standard-2 ---num_workers=75 +--num_workers=30 --disk_size_gb=50 --autoscaling_algorithm=NONE --staging_location=gs://temp-storage-for-perf-tests/loadtests From 5f1659af9ab55282a252646c73725ef7994f6f94 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 17 Oct 2023 15:09:48 -0400 Subject: [PATCH 164/435] Bump github.com/aws/aws-sdk-go-v2/feature/s3/manager in /sdks (#29032) Bumps [github.com/aws/aws-sdk-go-v2/feature/s3/manager](https://github.com/aws/aws-sdk-go-v2) from 1.11.90 to 1.11.91. - [Release notes](https://github.com/aws/aws-sdk-go-v2/releases) - [Commits](https://github.com/aws/aws-sdk-go-v2/compare/feature/s3/manager/v1.11.90...feature/s3/manager/v1.11.91) --- updated-dependencies: - dependency-name: github.com/aws/aws-sdk-go-v2/feature/s3/manager dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] <support@github.com> Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 4 ++-- sdks/go.sum | 8 ++++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 1fe7a2a82525..8f6b18f0c443 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -31,9 +31,9 @@ require ( cloud.google.com/go/spanner v1.50.0 cloud.google.com/go/storage v1.33.0 github.com/aws/aws-sdk-go-v2 v1.21.2 - github.com/aws/aws-sdk-go-v2/config v1.18.45 + github.com/aws/aws-sdk-go-v2/config v1.19.0 github.com/aws/aws-sdk-go-v2/credentials v1.13.43 - github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.90 + github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.91 github.com/aws/aws-sdk-go-v2/service/s3 v1.40.2 github.com/aws/smithy-go v1.15.0 github.com/docker/go-connections v0.4.0 diff --git a/sdks/go.sum b/sdks/go.sum index d535c2dda116..014c84b15989 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -86,8 +86,8 @@ github.com/aws/aws-sdk-go-v2 v1.21.2/go.mod h1:ErQhvNuEMhJjweavOYhxVkn2RUx7kQXVA github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.4.14 h1:Sc82v7tDQ/vdU1WtuSyzZ1I7y/68j//HJ6uozND1IDs= github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.4.14/go.mod h1:9NCTOURS8OpxvoAVHq79LK81/zC78hfRWFn+aL0SPcY= github.com/aws/aws-sdk-go-v2/config v1.5.0/go.mod h1:RWlPOAW3E3tbtNAqTwvSW54Of/yP3oiZXMI0xfUdjyA= -github.com/aws/aws-sdk-go-v2/config v1.18.45 h1:Aka9bI7n8ysuwPeFdm77nfbyHCAKQ3z9ghB3S/38zes= -github.com/aws/aws-sdk-go-v2/config v1.18.45/go.mod h1:ZwDUgFnQgsazQTnWfeLWk5GjeqTQTL8lMkoE1UXzxdE= +github.com/aws/aws-sdk-go-v2/config v1.19.0 h1:AdzDvwH6dWuVARCl3RTLGRc4Ogy+N7yLFxVxXe1ClQ0= +github.com/aws/aws-sdk-go-v2/config v1.19.0/go.mod h1:ZwDUgFnQgsazQTnWfeLWk5GjeqTQTL8lMkoE1UXzxdE= github.com/aws/aws-sdk-go-v2/credentials v1.3.1/go.mod h1:r0n73xwsIVagq8RsxmZbGSRQFj9As3je72C2WzUIToc= github.com/aws/aws-sdk-go-v2/credentials v1.13.43 h1:LU8vo40zBlo3R7bAvBVy/ku4nxGEyZe9N8MqAeFTzF8= github.com/aws/aws-sdk-go-v2/credentials v1.13.43/go.mod h1:zWJBz1Yf1ZtX5NGax9ZdNjhhI4rgjfgsyk6vTY1yfVg= @@ -95,8 +95,8 @@ github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.3.0/go.mod h1:2LAuqPx1I6jNfaGDu github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.13.13 h1:PIktER+hwIG286DqXyvVENjgLTAwGgoeriLDD5C+YlQ= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.13.13/go.mod h1:f/Ib/qYjhV2/qdsf79H3QP/eRE4AkVyEf6sk7XfZ1tg= github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.3.2/go.mod h1:qaqQiHSrOUVOfKe6fhgQ6UzhxjwqVW8aHNegd6Ws4w4= -github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.90 h1:mtJRt80k1oGw7QQPluAx8AZ6u16MyCA2di/lMhagZ7I= -github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.90/go.mod h1:lYwZTkeMQWPvNU+u7oYArdNhQ8EKiSGU76jVv0w2GH4= +github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.91 h1:haAyxKHwoE+y/TJt+qHcPQf1dCViyyGbWcKjjYUllTE= +github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.91/go.mod h1:ACQ6ta5YFlfSOz2c9A+EVYawLxFMZ0rI3Q0A0tGieKo= github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.43 h1:nFBQlGtkbPzp/NjZLuFxRqmT91rLJkgvsEQs68h962Y= github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.43/go.mod h1:auo+PiyLl0n1l8A0e8RIeR8tOzYPfZZH/JNlrJ8igTQ= github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.37 h1:JRVhO25+r3ar2mKGP7E0LDl8K9/G36gjlqca5iQbaqc= From 3097f50230a4292cd133566c746aa02a3d2a9a28 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 17 Oct 2023 15:31:03 -0400 Subject: [PATCH 165/435] Bump cloud.google.com/go/datastore from 1.14.0 to 1.15.0 in /sdks (#28910) Bumps [cloud.google.com/go/datastore](https://github.com/googleapis/google-cloud-go) from 1.14.0 to 1.15.0. - [Release notes](https://github.com/googleapis/google-cloud-go/releases) - [Changelog](https://github.com/googleapis/google-cloud-go/blob/main/documentai/CHANGES.md) - [Commits](https://github.com/googleapis/google-cloud-go/compare/kms/v1.14.0...kms/v1.15.0) --- updated-dependencies: - dependency-name: cloud.google.com/go/datastore dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] <support@github.com> Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 2 +- sdks/go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 8f6b18f0c443..51b6ac696e9b 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -25,7 +25,7 @@ go 1.20 require ( cloud.google.com/go/bigquery v1.56.0 cloud.google.com/go/bigtable v1.20.0 - cloud.google.com/go/datastore v1.14.0 + cloud.google.com/go/datastore v1.15.0 cloud.google.com/go/profiler v0.3.1 cloud.google.com/go/pubsub v1.33.0 cloud.google.com/go/spanner v1.50.0 diff --git a/sdks/go.sum b/sdks/go.sum index 014c84b15989..7c3af169cee4 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -24,8 +24,8 @@ cloud.google.com/go/compute/metadata v0.2.3/go.mod h1:VAV5nSsACxMJvgaAuX6Pk2Aawl cloud.google.com/go/datacatalog v1.17.1 h1:qGWrlYvWtK+8jD1jhwq5BsGoSr7S4/LOroV7LwXi00g= cloud.google.com/go/datastore v1.0.0/go.mod h1:LXYbyblFSglQ5pkeyhO+Qmw7ukd3C+pD7TKLgZqpHYE= cloud.google.com/go/datastore v1.1.0/go.mod h1:umbIZjpQpHh4hmRpGhH4tLFup+FVzqBi1b3c64qFpCk= -cloud.google.com/go/datastore v1.14.0 h1:Mq0ApTRdLW3/dyiw+DkjTk0+iGIUvkbzaC8sfPwWTH4= -cloud.google.com/go/datastore v1.14.0/go.mod h1:GAeStMBIt9bPS7jMJA85kgkpsMkvseWWXiaHya9Jes8= +cloud.google.com/go/datastore v1.15.0 h1:0P9WcsQeTWjuD1H14JIY7XQscIPQ4Laje8ti96IC5vg= +cloud.google.com/go/datastore v1.15.0/go.mod h1:GAeStMBIt9bPS7jMJA85kgkpsMkvseWWXiaHya9Jes8= cloud.google.com/go/iam v1.1.2 h1:gacbrBdWcoVmGLozRuStX45YKvJtzIjJdAolzUs1sm4= cloud.google.com/go/iam v1.1.2/go.mod h1:A5avdyVL2tCppe4unb0951eI9jreack+RJ0/d+KUZOU= cloud.google.com/go/kms v1.15.2 h1:lh6qra6oC4AyWe5fUUUBe/S27k12OHAleOOOw6KakdE= From ce28a36d23f790a79aa3523f892cb7c4505831f8 Mon Sep 17 00:00:00 2001 From: Yi Hu <yathu@google.com> Date: Tue, 17 Oct 2023 15:31:23 -0400 Subject: [PATCH 166/435] Add cleanup stale k8s namespace to CleanUpGCPResource workflow (#29038) --- .../beam_CancelStaleDataflowJobs.yml | 2 - .../workflows/beam_CleanUpGCPResources.yml | 2 - .../beam_CleanUpPrebuiltSDKImages.yml | 2 - .test-infra/tools/build.gradle | 12 ++++- .../tools/stale_k8s_workload_cleaner.sh | 49 +++++++++++++++++++ 5 files changed, 60 insertions(+), 7 deletions(-) create mode 100755 .test-infra/tools/stale_k8s_workload_cleaner.sh diff --git a/.github/workflows/beam_CancelStaleDataflowJobs.yml b/.github/workflows/beam_CancelStaleDataflowJobs.yml index 0733e1ceb943..9d1eb8fcb943 100644 --- a/.github/workflows/beam_CancelStaleDataflowJobs.yml +++ b/.github/workflows/beam_CancelStaleDataflowJobs.yml @@ -18,8 +18,6 @@ name: Cancel Stale Dataflow Jobs on: - issue_comment: - types: [created] schedule: - cron: '0 */4 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_CleanUpGCPResources.yml b/.github/workflows/beam_CleanUpGCPResources.yml index 31ca81ec2a73..5b1272fe219f 100644 --- a/.github/workflows/beam_CleanUpGCPResources.yml +++ b/.github/workflows/beam_CleanUpGCPResources.yml @@ -18,8 +18,6 @@ name: Clean Up GCP Resources on: - issue_comment: - types: [created] schedule: - cron: '0 0 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_CleanUpPrebuiltSDKImages.yml b/.github/workflows/beam_CleanUpPrebuiltSDKImages.yml index cc0facae57c6..389f6cd73385 100644 --- a/.github/workflows/beam_CleanUpPrebuiltSDKImages.yml +++ b/.github/workflows/beam_CleanUpPrebuiltSDKImages.yml @@ -18,8 +18,6 @@ name: Clean Up Prebuilt SDK Images on: - issue_comment: - types: [created] schedule: - cron: '0 0 * * *' workflow_dispatch: diff --git a/.test-infra/tools/build.gradle b/.test-infra/tools/build.gradle index ea4a12a24fc6..5ed6ebfc18f6 100644 --- a/.test-infra/tools/build.gradle +++ b/.test-infra/tools/build.gradle @@ -24,6 +24,16 @@ task removeStaleSDKContainerImages(type: Exec) { commandLine './stale_dataflow_prebuilt_image_cleaner.sh' } -task cleanupOtherStaleResources(type: Exec) { +task removeStaleBqDatasets(type: Exec) { commandLine './stale_bq_datasets_cleaner.sh' } + +task removeStaleK8sWorkload(type: Exec) { + commandLine './stale_k8s_workload_cleaner.sh' +} + +task cleanupOtherStaleResources { + // declared as finalizedBy dependency so that other task continue even if one dep task fails + finalizedBy tasks.removeStaleBqDatasets + finalizedBy tasks.removeStaleK8sWorkload +} diff --git a/.test-infra/tools/stale_k8s_workload_cleaner.sh b/.test-infra/tools/stale_k8s_workload_cleaner.sh new file mode 100755 index 000000000000..f4d6f9b88324 --- /dev/null +++ b/.test-infra/tools/stale_k8s_workload_cleaner.sh @@ -0,0 +1,49 @@ +#!/usr/bin/env bash +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +# Deletes stale and old BQ datasets that are left after tests. +# + +set -euo pipefail + +# Clean up the stale kubernetes workload of given cluster + +PROJECT=apache-beam-testing +LOCATION=us-central1-a +CLUSTER=io-datastores + +function should_teardown() { + if [[ $1 =~ ^([0-9]+)([a-z]) ]]; then + local time_scale=${BASH_REMATCH[1]} + local time_unit=${BASH_REMATCH[2]} + # cutoff = 8 h + if [ $time_unit == y ] || [ $time_unit == d ]; then + return 0 + elif [ $time_unit == h ] && [ $time_scale -ge 8 ]; then + return 0 + fi + fi + return 1 +} + +gcloud container clusters get-credentials io-datastores --zone us-central1-a --project apache-beam-testing + +while read NAME STATUS AGE; do + if [[ $NAME =~ ^beam-.+test ]] && should_teardown $AGE; then + kubectl delete namespace $NAME + fi +done < <( kubectl get namespaces --context=gke_${PROJECT}_${LOCATION}_${CLUSTER} ) From 587fa177b69698db8c747bcdd739020f9e6f92ce Mon Sep 17 00:00:00 2001 From: Vitaly Terentyev <vitaly.terentyev@akvelon.com> Date: Tue, 17 Oct 2023 23:39:36 +0400 Subject: [PATCH 167/435] Fix beam_LoadTests_Go_GBK_Dataflow_Batch job (#28960) * Fix prepare test arguments step * Fix license headers * Refactoring --- .../beam_LoadTests_Go_GBK_Dataflow_Batch.yml | 43 ++++++++++--------- ...00b.txt => go_GBK_Dataflow_Batch_100b.txt} | 9 +--- ...kb.txt => go_GBK_Dataflow_Batch_100kb.txt} | 9 +--- ..._10b.txt => go_GBK_Dataflow_Batch_10b.txt} | 9 +--- ...txt => go_GBK_Dataflow_Batch_Fanout_4.txt} | 9 +--- ...txt => go_GBK_Dataflow_Batch_Fanout_8.txt} | 9 +--- ...o_GBK_Dataflow_Batch_Reiteration_10KB.txt} | 9 +--- ...go_GBK_Dataflow_Batch_Reiteration_2MB.txt} | 9 +--- 8 files changed, 37 insertions(+), 69 deletions(-) rename .github/workflows/load-tests-job-configs/{config_GBK_Go_Batch_100b.txt => go_GBK_Dataflow_Batch_100b.txt} (78%) rename .github/workflows/load-tests-job-configs/{config_GBK_Go_Batch_100kb.txt => go_GBK_Dataflow_Batch_100kb.txt} (78%) rename .github/workflows/load-tests-job-configs/{config_GBK_Go_Batch_10b.txt => go_GBK_Dataflow_Batch_10b.txt} (78%) rename .github/workflows/load-tests-job-configs/{config_GBK_Go_Batch_Fanout_4.txt => go_GBK_Dataflow_Batch_Fanout_4.txt} (78%) rename .github/workflows/load-tests-job-configs/{config_GBK_Go_Batch_Fanout_8.txt => go_GBK_Dataflow_Batch_Fanout_8.txt} (78%) rename .github/workflows/load-tests-job-configs/{config_GBK_Go_Batch_Reiteration_10KB.txt => go_GBK_Dataflow_Batch_Reiteration_10KB.txt} (78%) rename .github/workflows/load-tests-job-configs/{config_GBK_Go_Batch_Reiteration_2MB.txt => go_GBK_Dataflow_Batch_Reiteration_2MB.txt} (78%) diff --git a/.github/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml index 30f8ef076d70..9656c14ed999 100644 --- a/.github/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml @@ -67,19 +67,22 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - - name: Prepare configs - #Reads config files, excludes comments, appends current date to the job_name parameter - id: set_configs - shell: bash - run: | - CURDATE=$(date '+%m%d%H%M%S' --utc) - CONFIG_ARR=('config_GBK_Go_Batch_10b.txt' 'config_GBK_Go_Batch_100b.txt' 'config_GBK_Go_Batch_100b.txt' 'config_GBK_Go_Batch_Fanout_4.txt' 'config_GBK_Go_Batch_Fanout_8.txt' 'config_GBK_Go_Batch_Reiteration_10KB.txt', 'config_GBK_Go_Batch_Reiteration_2MB.txt') - for INDEX in ${!CONFIG_ARR[@]} - do - CURCONFIG=$(grep -v "^#.*" ./.github/workflows/load-tests-job-configs/${CONFIG_ARR[INDEX]} | tr '\n' ' ') - CURCONFIG=$(echo "${CURCONFIG/load-tests-go-dataflow-batch-gbk-$((INDEX + 1))-/load-tests-go-dataflow-batch-gbk-$((INDEX + 1))-$CURDATE}") - echo "prepared_config_$((INDEX + 1))=$CURCONFIG" >> $GITHUB_OUTPUT - done + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: go + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/go_GBK_Dataflow_Batch_10b.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/go_GBK_Dataflow_Batch_100b.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/go_GBK_Dataflow_Batch_100kb.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/go_GBK_Dataflow_Batch_Fanout_4.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/go_GBK_Dataflow_Batch_Fanout_8.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/go_GBK_Dataflow_Batch_Reiteration_10KB.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/go_GBK_Dataflow_Batch_Reiteration_2MB.txt + - name: get current time + run: echo "NOW_UTC=$(date '+%m%d%H%M%S' --utc)" >> $GITHUB_ENV + # The env variables are created and populated in the test-arguments-action as "<github.job>_test_arguments_<argument_file_paths_index>" - name: run GBK Dataflow Batch Go Load Test 1 (10 b records) uses: ./.github/actions/gradle-command-self-hosted-action with: @@ -87,7 +90,7 @@ jobs: arguments: | -PloadTest.mainClass=group_by_key \ -Prunner=DataflowRunner \ - '-PloadTest.args=${{ steps.set_configs.outputs.prepared_config_1 }}' \ + '-PloadTest.args=${{ env.beam_LoadTests_Go_GBK_Dataflow_Batch_test_arguments_1 }} --job_name=load-tests-go-dataflow-batch-gbk-1-${{env.NOW_UTC}}' \ - name: run GBK Dataflow Batch Go Load Test 2 (100 b records) uses: ./.github/actions/gradle-command-self-hosted-action with: @@ -95,7 +98,7 @@ jobs: arguments: | -PloadTest.mainClass=group_by_key \ -Prunner=DataflowRunner \ - '-PloadTest.args=${{ steps.set_configs.outputs.prepared_config_2 }}' \ + '-PloadTest.args=${{ env.beam_LoadTests_Go_GBK_Dataflow_Batch_test_arguments_2 }} --job_name=load-tests-go-dataflow-batch-gbk-2-${{env.NOW_UTC}}' \ - name: run GBK Dataflow Batch Go Load Test 3 (100 kb records) uses: ./.github/actions/gradle-command-self-hosted-action with: @@ -103,7 +106,7 @@ jobs: arguments: | -PloadTest.mainClass=group_by_key \ -Prunner=DataflowRunner \ - '-PloadTest.args=${{ steps.set_configs.outputs.prepared_config_3 }}' \ + '-PloadTest.args=${{ env.beam_LoadTests_Go_GBK_Dataflow_Batch_test_arguments_3 }} --job_name=load-tests-go-dataflow-batch-gbk-3-${{env.NOW_UTC}}' \ - name: run GBK Dataflow Batch Go Load Test 4 (fanout 4) uses: ./.github/actions/gradle-command-self-hosted-action with: @@ -111,7 +114,7 @@ jobs: arguments: | -PloadTest.mainClass=group_by_key \ -Prunner=DataflowRunner \ - '-PloadTest.args=${{ steps.set_configs.outputs.prepared_config_4 }}' \ + '-PloadTest.args=${{ env.beam_LoadTests_Go_GBK_Dataflow_Batch_test_arguments_4 }} --job_name=load-tests-go-dataflow-batch-gbk-4-${{env.NOW_UTC}}' \ - name: run GBK Dataflow Batch Go Load Test 5 (fanout 8) uses: ./.github/actions/gradle-command-self-hosted-action with: @@ -119,7 +122,7 @@ jobs: arguments: | -PloadTest.mainClass=group_by_key \ -Prunner=DataflowRunner \ - '-PloadTest.args=${{ steps.set_configs.outputs.prepared_config_5 }}' \ + '-PloadTest.args=${{ env.beam_LoadTests_Go_GBK_Dataflow_Batch_test_arguments_5 }} --job_name=load-tests-go-dataflow-batch-gbk-5-${{env.NOW_UTC}}' \ - name: run GBK Dataflow Batch Go Load Test 6 (reiterate 4 times 10 kb) uses: ./.github/actions/gradle-command-self-hosted-action with: @@ -127,7 +130,7 @@ jobs: arguments: | -PloadTest.mainClass=group_by_key \ -Prunner=DataflowRunner \ - '-PloadTest.args=${{ steps.set_configs.outputs.prepared_config_6 }}' \ + '-PloadTest.args=${{ env.beam_LoadTests_Go_GBK_Dataflow_Batch_test_arguments_6 }} --job_name=load-tests-go-dataflow-batch-gbk-6-${{env.NOW_UTC}}' \ - name: run GBK Dataflow Batch Go Load Test 7 (reiterate 4 times 2 mb) uses: ./.github/actions/gradle-command-self-hosted-action with: @@ -135,4 +138,4 @@ jobs: arguments: | -PloadTest.mainClass=group_by_key \ -Prunner=DataflowRunner \ - '-PloadTest.args=${{ steps.set_configs.outputs.prepared_config_7 }}' \ No newline at end of file + '-PloadTest.args=${{ env.beam_LoadTests_Go_GBK_Dataflow_Batch_test_arguments_7 }} --job_name=load-tests-go-dataflow-batch-gbk-7-${{env.NOW_UTC}}' \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/config_GBK_Go_Batch_100b.txt b/.github/workflows/load-tests-job-configs/go_GBK_Dataflow_Batch_100b.txt similarity index 78% rename from .github/workflows/load-tests-job-configs/config_GBK_Go_Batch_100b.txt rename to .github/workflows/load-tests-job-configs/go_GBK_Dataflow_Batch_100b.txt index f3ebed91b6ad..b5266fa83047 100644 --- a/.github/workflows/load-tests-job-configs/config_GBK_Go_Batch_100b.txt +++ b/.github/workflows/load-tests-job-configs/go_GBK_Dataflow_Batch_100b.txt @@ -1,4 +1,3 @@ -############################################################################### # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file # distributed with this work for additional information @@ -13,10 +12,8 @@ # distributed under the License is distributed on an "AS IS" BASIS, # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. # See the License for the specific language governing permissions and -# limitations under the License. -############################################################################### ---job_name=load-tests-go-dataflow-batch-gbk-2- ---project=apache-beam-testing +# limitations under the License. + --region=us-central1 --temp_location=gs://temp-storage-for-perf-tests/loadtests --staging_location=gs://temp-storage-for-perf-tests/loadtests @@ -29,6 +26,4 @@ --autoscaling_algorithm=NONE --environment_type=DOCKER --environment_config=gcr.io/apache-beam-testing/beam-sdk/beam_go_sdk:latest ---influx_db_name=beam_test_metrics ---influx_hostname=http://10.128.0.96:8086 --runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/config_GBK_Go_Batch_100kb.txt b/.github/workflows/load-tests-job-configs/go_GBK_Dataflow_Batch_100kb.txt similarity index 78% rename from .github/workflows/load-tests-job-configs/config_GBK_Go_Batch_100kb.txt rename to .github/workflows/load-tests-job-configs/go_GBK_Dataflow_Batch_100kb.txt index e5007c7d5b90..072ab494515c 100644 --- a/.github/workflows/load-tests-job-configs/config_GBK_Go_Batch_100kb.txt +++ b/.github/workflows/load-tests-job-configs/go_GBK_Dataflow_Batch_100kb.txt @@ -1,4 +1,3 @@ -############################################################################### # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file # distributed with this work for additional information @@ -13,10 +12,8 @@ # distributed under the License is distributed on an "AS IS" BASIS, # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. # See the License for the specific language governing permissions and -# limitations under the License. -############################################################################### ---job_name=load-tests-go-dataflow-batch-gbk-3- ---project=apache-beam-testing +# limitations under the License. + --region=us-central1 --temp_location=gs://temp-storage-for-perf-tests/loadtests --staging_location=gs://temp-storage-for-perf-tests/loadtests @@ -29,6 +26,4 @@ --autoscaling_algorithm=NONE --environment_type=DOCKER --environment_config=gcr.io/apache-beam-testing/beam-sdk/beam_go_sdk:latest ---influx_db_name=beam_test_metrics ---influx_hostname=http://10.128.0.96:8086 --runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/config_GBK_Go_Batch_10b.txt b/.github/workflows/load-tests-job-configs/go_GBK_Dataflow_Batch_10b.txt similarity index 78% rename from .github/workflows/load-tests-job-configs/config_GBK_Go_Batch_10b.txt rename to .github/workflows/load-tests-job-configs/go_GBK_Dataflow_Batch_10b.txt index 7683eac5cb93..2e69dd0b457b 100644 --- a/.github/workflows/load-tests-job-configs/config_GBK_Go_Batch_10b.txt +++ b/.github/workflows/load-tests-job-configs/go_GBK_Dataflow_Batch_10b.txt @@ -1,4 +1,3 @@ -############################################################################### # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file # distributed with this work for additional information @@ -13,10 +12,8 @@ # distributed under the License is distributed on an "AS IS" BASIS, # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. # See the License for the specific language governing permissions and -# limitations under the License. -############################################################################### ---job_name=load-tests-go-dataflow-batch-gbk-1- ---project=apache-beam-testing +# limitations under the License. + --region=us-central1 --temp_location=gs://temp-storage-for-perf-tests/loadtests --staging_location=gs://temp-storage-for-perf-tests/loadtests @@ -29,6 +26,4 @@ --autoscaling_algorithm=NONE --environment_type=DOCKER --environment_config=gcr.io/apache-beam-testing/beam-sdk/beam_go_sdk:latest ---influx_db_name=beam_test_metrics ---influx_hostname=http://10.128.0.96:8086 --runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/config_GBK_Go_Batch_Fanout_4.txt b/.github/workflows/load-tests-job-configs/go_GBK_Dataflow_Batch_Fanout_4.txt similarity index 78% rename from .github/workflows/load-tests-job-configs/config_GBK_Go_Batch_Fanout_4.txt rename to .github/workflows/load-tests-job-configs/go_GBK_Dataflow_Batch_Fanout_4.txt index 5792b3bf0b95..6371123142d8 100644 --- a/.github/workflows/load-tests-job-configs/config_GBK_Go_Batch_Fanout_4.txt +++ b/.github/workflows/load-tests-job-configs/go_GBK_Dataflow_Batch_Fanout_4.txt @@ -1,4 +1,3 @@ -############################################################################### # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file # distributed with this work for additional information @@ -13,10 +12,8 @@ # distributed under the License is distributed on an "AS IS" BASIS, # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. # See the License for the specific language governing permissions and -# limitations under the License. -############################################################################### ---job_name=load-tests-go-dataflow-batch-gbk-4- ---project=apache-beam-testing +# limitations under the License. + --region=us-central1 --temp_location=gs://temp-storage-for-perf-tests/loadtests --staging_location=gs://temp-storage-for-perf-tests/loadtests @@ -29,6 +26,4 @@ --autoscaling_algorithm=NONE --environment_type=DOCKER --environment_config=gcr.io/apache-beam-testing/beam-sdk/beam_go_sdk:latest ---influx_db_name=beam_test_metrics ---influx_hostname=http://10.128.0.96:8086 --runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/config_GBK_Go_Batch_Fanout_8.txt b/.github/workflows/load-tests-job-configs/go_GBK_Dataflow_Batch_Fanout_8.txt similarity index 78% rename from .github/workflows/load-tests-job-configs/config_GBK_Go_Batch_Fanout_8.txt rename to .github/workflows/load-tests-job-configs/go_GBK_Dataflow_Batch_Fanout_8.txt index 369fb25aa0e1..77d5f2e0162b 100644 --- a/.github/workflows/load-tests-job-configs/config_GBK_Go_Batch_Fanout_8.txt +++ b/.github/workflows/load-tests-job-configs/go_GBK_Dataflow_Batch_Fanout_8.txt @@ -1,4 +1,3 @@ -############################################################################### # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file # distributed with this work for additional information @@ -13,10 +12,8 @@ # distributed under the License is distributed on an "AS IS" BASIS, # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. # See the License for the specific language governing permissions and -# limitations under the License. -############################################################################### ---job_name=load-tests-go-dataflow-batch-gbk-5- ---project=apache-beam-testing +# limitations under the License. + --region=us-central1 --temp_location=gs://temp-storage-for-perf-tests/loadtests --staging_location=gs://temp-storage-for-perf-tests/loadtests @@ -29,6 +26,4 @@ --autoscaling_algorithm=NONE --environment_type=DOCKER --environment_config=gcr.io/apache-beam-testing/beam-sdk/beam_go_sdk:latest ---influx_db_name=beam_test_metrics ---influx_hostname=http://10.128.0.96:8086 --runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/config_GBK_Go_Batch_Reiteration_10KB.txt b/.github/workflows/load-tests-job-configs/go_GBK_Dataflow_Batch_Reiteration_10KB.txt similarity index 78% rename from .github/workflows/load-tests-job-configs/config_GBK_Go_Batch_Reiteration_10KB.txt rename to .github/workflows/load-tests-job-configs/go_GBK_Dataflow_Batch_Reiteration_10KB.txt index 9eb878d4e9fb..7b27693ed8cc 100644 --- a/.github/workflows/load-tests-job-configs/config_GBK_Go_Batch_Reiteration_10KB.txt +++ b/.github/workflows/load-tests-job-configs/go_GBK_Dataflow_Batch_Reiteration_10KB.txt @@ -1,4 +1,3 @@ -############################################################################### # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file # distributed with this work for additional information @@ -13,10 +12,8 @@ # distributed under the License is distributed on an "AS IS" BASIS, # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. # See the License for the specific language governing permissions and -# limitations under the License. -############################################################################### ---job_name=load-tests-go-dataflow-batch-gbk-6 ---project=apache-beam-testing +# limitations under the License. + --region=us-central1 --temp_location=gs://temp-storage-for-perf-tests/loadtests --staging_location=gs://temp-storage-for-perf-tests/loadtests @@ -29,6 +26,4 @@ --autoscaling_algorithm=NONE --environment_type=DOCKER --environment_config=gcr.io/apache-beam-testing/beam-sdk/beam_go_sdk:latest ---influx_db_name=beam_test_metrics ---influx_hostname=http://10.128.0.96:8086 --runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/config_GBK_Go_Batch_Reiteration_2MB.txt b/.github/workflows/load-tests-job-configs/go_GBK_Dataflow_Batch_Reiteration_2MB.txt similarity index 78% rename from .github/workflows/load-tests-job-configs/config_GBK_Go_Batch_Reiteration_2MB.txt rename to .github/workflows/load-tests-job-configs/go_GBK_Dataflow_Batch_Reiteration_2MB.txt index aa26473ca433..40971e546454 100644 --- a/.github/workflows/load-tests-job-configs/config_GBK_Go_Batch_Reiteration_2MB.txt +++ b/.github/workflows/load-tests-job-configs/go_GBK_Dataflow_Batch_Reiteration_2MB.txt @@ -1,4 +1,3 @@ -############################################################################### # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file # distributed with this work for additional information @@ -13,10 +12,8 @@ # distributed under the License is distributed on an "AS IS" BASIS, # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. # See the License for the specific language governing permissions and -# limitations under the License. -############################################################################### ---job_name=load-tests-go-dataflow-batch-gbk-7 ---project=apache-beam-testing +# limitations under the License. + --region=us-central1 --temp_location=gs://temp-storage-for-perf-tests/loadtests --staging_location=gs://temp-storage-for-perf-tests/loadtests @@ -29,6 +26,4 @@ --autoscaling_algorithm=NONE --environment_type=DOCKER --environment_config=gcr.io/apache-beam-testing/beam-sdk/beam_go_sdk:latest ---influx_db_name=beam_test_metrics ---influx_hostname=http://10.128.0.96:8086 --runner=DataflowRunner \ No newline at end of file From 67c30a447cf57138ff9b3bd2204e447a08f114fd Mon Sep 17 00:00:00 2001 From: Aleksandr Dudko <116064902+aleksandr-dudko@users.noreply.github.com> Date: Tue, 17 Oct 2023 23:41:02 +0400 Subject: [PATCH 168/435] Fix bug causing job failure (#28982) * Fix bug causing runs failure * Fix name for PreCommit Java PVR Flink Docker --- .../beam_PostCommit_Java_PVR_Spark_Batch.yml | 21 ++++++++++++++----- .../beam_PreCommit_Java_PVR_Flink_Docker.yml | 2 +- 2 files changed, 17 insertions(+), 6 deletions(-) diff --git a/.github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml b/.github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml index c240ac5abf26..2df2b39cd143 100644 --- a/.github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml +++ b/.github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml @@ -59,8 +59,7 @@ jobs: job_phrase: [Run Java Spark PortableValidatesRunner Batch] if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || - github.event.comment.body == 'Run Java Spark PortableValidatesRunner Batch' + github.event_name == 'schedule' steps: - uses: actions/checkout@v4 - name: Setup repository @@ -69,12 +68,16 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: run PostCommit Java PortableValidatesRunner Spark Batch script + env: + CLOUDSDK_CONFIG: ${{ env.KUBELET_GCLOUD_CONFIG_PATH}} uses: ./.github/actions/gradle-command-self-hosted-action with: gradle-command: | - :runners:spark:3:job-server:validatesPortableRunnerBatch - :runners:spark:3:job-server:validatesPortableRunnerDocker + :runners:spark:3:job-server:validatesPortableRunnerBatch \ + :runners:spark:3:job-server:validatesPortableRunnerDocker \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 if: failure() @@ -85,4 +88,12 @@ jobs: uses: EnricoMi/publish-unit-test-result-action@v2 if: always() with: - files: '**/build/test-results/**/*.xml' \ No newline at end of file + large_files: true + commit: '${{ env.prsha || env.GITHUB_SHA }}' + comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} + files: '**/build/test-results/**/*.xml' + - name: Archive SpotBugs Results + uses: actions/upload-artifact@v3 + with: + name: SpotBugs Results + path: "**/build/reports/spotbugs/*.html" \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml b/.github/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml index ac6b9e0471a4..79794904b487 100644 --- a/.github/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml +++ b/.github/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml @@ -69,7 +69,7 @@ env: GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} jobs: - beam_PreCommit_Java: + beam_PreCommit_Java_PVR_Flink_Docker: name: ${{matrix.job_name}} (${{matrix.job_phrase}}) runs-on: [self-hosted, ubuntu-20.04, main] strategy: From 416a4f268a27be9e02971b0efbce523ff4fc4644 Mon Sep 17 00:00:00 2001 From: Damon <damondouglas@users.noreply.github.com> Date: Tue, 17 Oct 2023 13:53:00 -0700 Subject: [PATCH 169/435] [RRIO] [Call] Create CallShouldBackoff and default implementation (#28952) * Create CallShouldBackoff and default implementation * Fix test class comment. * Update javadoc * Update per PR comments * Patch code comment --- .../requestresponseio/CallShouldBackoff.java | 33 ++++++ ...uldBackoffBasedOnRejectionProbability.java | 103 ++++++++++++++++++ ...ackoffBasedOnRejectionProbabilityTest.java | 92 ++++++++++++++++ 3 files changed, 228 insertions(+) create mode 100644 sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/CallShouldBackoff.java create mode 100644 sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/CallShouldBackoffBasedOnRejectionProbability.java create mode 100644 sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponseio/CallShouldBackoffBasedOnRejectionProbabilityTest.java diff --git a/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/CallShouldBackoff.java b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/CallShouldBackoff.java new file mode 100644 index 000000000000..9a0c42de5b1f --- /dev/null +++ b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/CallShouldBackoff.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.io.requestresponseio; + +import java.io.Serializable; + +/** Informs whether a call to an API should backoff. */ +public interface CallShouldBackoff<ResponseT> extends Serializable { + + /** Update the state of whether to backoff using information about the exception. */ + void update(UserCodeExecutionException exception); + + /** Update the state of whether to backoff using information about the response. */ + void update(ResponseT response); + + /** Report whether to backoff. */ + boolean value(); +} diff --git a/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/CallShouldBackoffBasedOnRejectionProbability.java b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/CallShouldBackoffBasedOnRejectionProbability.java new file mode 100644 index 000000000000..8298809fca8d --- /dev/null +++ b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/CallShouldBackoffBasedOnRejectionProbability.java @@ -0,0 +1,103 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.io.requestresponseio; + +import org.checkerframework.checker.nullness.qual.Nullable; + +/** Reports whether to apply backoff based on https://sre.google/sre-book/handling-overload/. */ +class CallShouldBackoffBasedOnRejectionProbability<ResponseT> + implements CallShouldBackoff<ResponseT> { + + // Default multiplier value recommended by https://sre.google/sre-book/handling-overload/ + private static final double DEFAULT_MULTIPLIER = 2.0; + + // The threshold is the value that the rejection probability must exceed in order to report a + // value() of true. If null, then the computation relies on a random value. + private @Nullable Double threshold; + + // The multiplier drives the impact of accepts on the rejection probability. See <a + // https://sre.google/sre-book/handling-overload/ for details. + private final double multiplier; + + // The number of total requests called to a remote API. + private double requests = 0; + + // The number of total accepts called to a remote API. + private double accepts = 0; + + /** Instantiate class with the {@link #DEFAULT_MULTIPLIER}. */ + CallShouldBackoffBasedOnRejectionProbability() { + this(DEFAULT_MULTIPLIER); + } + + /** + * Instantiates class with the provided multiplier. The multiplier drives the impact of accepts on + * the rejection probability. See https://sre.google/sre-book/handling-overload/ for details. + */ + CallShouldBackoffBasedOnRejectionProbability(double multiplier) { + this.multiplier = multiplier; + } + + /** + * Setter for the threshold that overrides usage of a random value. The threshold is the value + * (within range [0, 1)) that {@link #getRejectionProbability()} must exceed in order to report a + * value() of true. + */ + CallShouldBackoffBasedOnRejectionProbability<ResponseT> setThreshold(double threshold) { + this.threshold = threshold; + return this; + } + + /** Update the state of whether to backoff using information about the exception. */ + @Override + public void update(UserCodeExecutionException exception) { + this.requests++; + } + + /** Update the state of whether to backoff using information about the response. */ + @Override + public void update(ResponseT response) { + this.requests++; + this.accepts++; + } + + /** Provide a threshold to evaluate backoff. */ + double getThreshold() { + if (this.threshold != null) { + return this.threshold; + } + return Math.random(); + } + + /** + * Compute the probability of API call rejection based on + * https://sre.google/sre-book/handling-overload/. + */ + double getRejectionProbability() { + double numerator = requests - multiplier * accepts; + double denominator = requests + 1; + double ratio = numerator / denominator; + return Math.max(0, ratio); + } + + /** Report whether to backoff. */ + @Override + public boolean value() { + return getRejectionProbability() > getThreshold(); + } +} diff --git a/sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponseio/CallShouldBackoffBasedOnRejectionProbabilityTest.java b/sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponseio/CallShouldBackoffBasedOnRejectionProbabilityTest.java new file mode 100644 index 000000000000..b1e8347a25cf --- /dev/null +++ b/sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponseio/CallShouldBackoffBasedOnRejectionProbabilityTest.java @@ -0,0 +1,92 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.io.requestresponseio; + +import static org.junit.Assert.assertEquals; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Tests for {@link CallShouldBackoffBasedOnRejectionProbability}. */ +@RunWith(JUnit4.class) +public class CallShouldBackoffBasedOnRejectionProbabilityTest { + + @Test + public void testValue() { + for (Case caze : CASES) { + CallShouldBackoffBasedOnRejectionProbability<String> shouldBackoff = instance(); + for (boolean ar : caze.acceptRejects) { + if (ar) { + shouldBackoff.update(""); + } else { + shouldBackoff.update(new UserCodeExecutionException("")); + } + } + assertEquals(caze.toString(), caze.wantPReject, shouldBackoff.getRejectionProbability(), 0.1); + assertEquals(caze.toString(), caze.wantValue, shouldBackoff.value()); + } + } + + private static final List<Case> CASES = + Arrays.asList( + of(0, false), + of(0, false, true, true, true, true, true, true, true, true, true, true, true), + of(0, false, true), + of(0.5, false, false), + of(0.91, true, false, false, false, false, false, false, false, false, false, false)); + + private static Case of(double wantPReject, boolean wantValue, boolean... acceptRejects) { + List<Boolean> list = new ArrayList<>(); + for (boolean ar : acceptRejects) { + list.add(ar); + } + return new Case(list, wantPReject, wantValue); + } + + private static class Case { + private final List<Boolean> acceptRejects; + private final double wantPReject; + private final boolean wantValue; + + Case(List<Boolean> acceptRejects, double wantPReject, boolean wantValue) { + this.acceptRejects = acceptRejects; + this.wantPReject = wantPReject; + this.wantValue = wantValue; + } + + @Override + public String toString() { + return "Case{" + + "acceptRejects=" + + acceptRejects + + ", wantPReject=" + + wantPReject + + ", wantValue=" + + wantValue + + '}'; + } + } + + CallShouldBackoffBasedOnRejectionProbability<String> instance() { + return new CallShouldBackoffBasedOnRejectionProbability<String>().setThreshold(0.5); + } +} From eb7997145e4645aaa36ce33fc21924081c1f3735 Mon Sep 17 00:00:00 2001 From: Damon <damondouglas@users.noreply.github.com> Date: Tue, 17 Oct 2023 14:14:37 -0700 Subject: [PATCH 170/435] [RRIO] [Cache] Stub CacheRead transform (#29011) * Stub CacheRead * Run spotlessApply --- .../beam/io/requestresponseio/CacheRead.java | 121 ++++++++++++++++++ 1 file changed, 121 insertions(+) create mode 100644 sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/CacheRead.java diff --git a/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/CacheRead.java b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/CacheRead.java new file mode 100644 index 000000000000..6154873e506d --- /dev/null +++ b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/CacheRead.java @@ -0,0 +1,121 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.io.requestresponseio; + +import com.google.auto.value.AutoValue; +import java.util.Map; +import org.apache.beam.io.requestresponseio.CacheRead.Result; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionTuple; +import org.apache.beam.sdk.values.PInput; +import org.apache.beam.sdk.values.POutput; +import org.apache.beam.sdk.values.PValue; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; + +/** + * {@link CacheRead} reads associated {@link ResponseT} types from {@link RequestT} types, if any + * exist. + */ +class CacheRead<RequestT, ResponseT> + extends PTransform<PCollection<RequestT>, Result<RequestT, ResponseT>> { + + private static final TupleTag<ApiIOError> FAILURE_TAG = new TupleTag<ApiIOError>() {}; + + // TODO(damondouglas): remove suppress warnings after instance utilized. + @SuppressWarnings({"unused"}) + private final Configuration<RequestT, ResponseT> configuration; + + private CacheRead(Configuration<RequestT, ResponseT> configuration) { + this.configuration = configuration; + } + + /** Configuration details for {@link CacheRead}. */ + @AutoValue + abstract static class Configuration<RequestT, ResponseT> { + + static <RequestT, ResponseT> Builder<RequestT, ResponseT> builder() { + return new AutoValue_CacheRead_Configuration.Builder<>(); + } + + abstract Builder<RequestT, ResponseT> toBuilder(); + + @AutoValue.Builder + abstract static class Builder<RequestT, ResponseT> { + + abstract Configuration<RequestT, ResponseT> build(); + } + } + + @Override + public Result<RequestT, ResponseT> expand(PCollection<RequestT> input) { + return Result.of( + new TupleTag<KV<RequestT, ResponseT>>() {}, PCollectionTuple.empty(input.getPipeline())); + } + + /** + * The {@link Result} of reading RequestT {@link PCollection} elements yielding ResponseT {@link + * PCollection} elements. + */ + static class Result<RequestT, ResponseT> implements POutput { + + static <RequestT, ResponseT> Result<RequestT, ResponseT> of( + TupleTag<KV<RequestT, ResponseT>> responseTag, PCollectionTuple pct) { + return new Result<>(responseTag, pct); + } + + private final Pipeline pipeline; + private final TupleTag<KV<RequestT, ResponseT>> responseTag; + private final PCollection<KV<RequestT, ResponseT>> responses; + private final PCollection<ApiIOError> failures; + + private Result(TupleTag<KV<RequestT, ResponseT>> responseTag, PCollectionTuple pct) { + this.pipeline = pct.getPipeline(); + this.responseTag = responseTag; + this.responses = pct.get(responseTag); + this.failures = pct.get(FAILURE_TAG); + } + + PCollection<KV<RequestT, ResponseT>> getResponses() { + return responses; + } + + PCollection<ApiIOError> getFailures() { + return failures; + } + + @Override + public Pipeline getPipeline() { + return this.pipeline; + } + + @Override + public Map<TupleTag<?>, PValue> expand() { + return ImmutableMap.of( + responseTag, responses, + FAILURE_TAG, failures); + } + + @Override + public void finishSpecifyingOutput( + String transformName, PInput input, PTransform<?, ?> transform) {} + } +} From 89115954f67d5e819df46d0428a8b274a9a54e8c Mon Sep 17 00:00:00 2001 From: Damon <damondouglas@users.noreply.github.com> Date: Tue, 17 Oct 2023 18:54:34 -0700 Subject: [PATCH 171/435] Rename rrio package (#29040) --- sdks/java/io/rrio/build.gradle | 2 +- .../io/{requestresponseio => requestresponse}/ApiIOError.java | 2 +- .../io/{requestresponseio => requestresponse}/CacheRead.java | 4 ++-- .../io/{requestresponseio => requestresponse}/CacheWrite.java | 4 ++-- .../CallShouldBackoff.java | 2 +- .../CallShouldBackoffBasedOnRejectionProbability.java | 2 +- .../io/{requestresponseio => requestresponse}/Caller.java | 2 +- .../RequestResponseIO.java | 4 ++-- .../{requestresponseio => requestresponse}/SetupTeardown.java | 2 +- .../UserCodeExecutionException.java | 2 +- .../UserCodeQuotaException.java | 2 +- .../UserCodeTimeoutException.java | 2 +- .../{requestresponseio => requestresponse}/package-info.java | 2 +- .../CallShouldBackoffBasedOnRejectionProbabilityTest.java | 2 +- .../io/{requestresponseio => requestresponse}/CallerTest.java | 2 +- .../SetupTeardownTest.java | 2 +- 16 files changed, 19 insertions(+), 19 deletions(-) rename sdks/java/io/rrio/src/main/java/org/apache/beam/io/{requestresponseio => requestresponse}/ApiIOError.java (97%) rename sdks/java/io/rrio/src/main/java/org/apache/beam/io/{requestresponseio => requestresponse}/CacheRead.java (97%) rename sdks/java/io/rrio/src/main/java/org/apache/beam/io/{requestresponseio => requestresponse}/CacheWrite.java (97%) rename sdks/java/io/rrio/src/main/java/org/apache/beam/io/{requestresponseio => requestresponse}/CallShouldBackoff.java (96%) rename sdks/java/io/rrio/src/main/java/org/apache/beam/io/{requestresponseio => requestresponse}/CallShouldBackoffBasedOnRejectionProbability.java (98%) rename sdks/java/io/rrio/src/main/java/org/apache/beam/io/{requestresponseio => requestresponse}/Caller.java (96%) rename sdks/java/io/rrio/src/main/java/org/apache/beam/io/{requestresponseio => requestresponse}/RequestResponseIO.java (98%) rename sdks/java/io/rrio/src/main/java/org/apache/beam/io/{requestresponseio => requestresponse}/SetupTeardown.java (96%) rename sdks/java/io/rrio/src/main/java/org/apache/beam/io/{requestresponseio => requestresponse}/UserCodeExecutionException.java (96%) rename sdks/java/io/rrio/src/main/java/org/apache/beam/io/{requestresponseio => requestresponse}/UserCodeQuotaException.java (96%) rename sdks/java/io/rrio/src/main/java/org/apache/beam/io/{requestresponseio => requestresponse}/UserCodeTimeoutException.java (96%) rename sdks/java/io/rrio/src/main/java/org/apache/beam/io/{requestresponseio => requestresponse}/package-info.java (95%) rename sdks/java/io/rrio/src/test/java/org/apache/beam/io/{requestresponseio => requestresponse}/CallShouldBackoffBasedOnRejectionProbabilityTest.java (98%) rename sdks/java/io/rrio/src/test/java/org/apache/beam/io/{requestresponseio => requestresponse}/CallerTest.java (98%) rename sdks/java/io/rrio/src/test/java/org/apache/beam/io/{requestresponseio => requestresponse}/SetupTeardownTest.java (98%) diff --git a/sdks/java/io/rrio/build.gradle b/sdks/java/io/rrio/build.gradle index d65df370e0ca..d7d5c8817d06 100644 --- a/sdks/java/io/rrio/build.gradle +++ b/sdks/java/io/rrio/build.gradle @@ -18,7 +18,7 @@ plugins { id 'org.apache.beam.module' } applyJavaNature( - automaticModuleName: 'org.apache.beam.sdk.io.rrio' + automaticModuleName: 'org.apache.beam.sdk.io.requestresponse' ) description = "Apache Beam :: SDKS :: Java :: IO :: RequestResponseIO (RRIO)" diff --git a/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/ApiIOError.java b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/ApiIOError.java similarity index 97% rename from sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/ApiIOError.java rename to sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/ApiIOError.java index b7c5524e8237..5936c5dd84b0 100644 --- a/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/ApiIOError.java +++ b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/ApiIOError.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.io.requestresponseio; +package org.apache.beam.io.requestresponse; import com.google.auto.value.AutoValue; import org.apache.beam.sdk.schemas.AutoValueSchema; diff --git a/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/CacheRead.java b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/CacheRead.java similarity index 97% rename from sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/CacheRead.java rename to sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/CacheRead.java index 6154873e506d..3765d25370a6 100644 --- a/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/CacheRead.java +++ b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/CacheRead.java @@ -15,11 +15,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.io.requestresponseio; +package org.apache.beam.io.requestresponse; import com.google.auto.value.AutoValue; import java.util.Map; -import org.apache.beam.io.requestresponseio.CacheRead.Result; +import org.apache.beam.io.requestresponse.CacheRead.Result; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.values.KV; diff --git a/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/CacheWrite.java b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/CacheWrite.java similarity index 97% rename from sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/CacheWrite.java rename to sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/CacheWrite.java index 0fb14af67c35..25249c3e41b4 100644 --- a/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/CacheWrite.java +++ b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/CacheWrite.java @@ -15,11 +15,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.io.requestresponseio; +package org.apache.beam.io.requestresponse; import com.google.auto.value.AutoValue; import java.util.Map; -import org.apache.beam.io.requestresponseio.CacheWrite.Result; +import org.apache.beam.io.requestresponse.CacheWrite.Result; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.values.KV; diff --git a/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/CallShouldBackoff.java b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/CallShouldBackoff.java similarity index 96% rename from sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/CallShouldBackoff.java rename to sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/CallShouldBackoff.java index 9a0c42de5b1f..1d093f2efb12 100644 --- a/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/CallShouldBackoff.java +++ b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/CallShouldBackoff.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.io.requestresponseio; +package org.apache.beam.io.requestresponse; import java.io.Serializable; diff --git a/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/CallShouldBackoffBasedOnRejectionProbability.java b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/CallShouldBackoffBasedOnRejectionProbability.java similarity index 98% rename from sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/CallShouldBackoffBasedOnRejectionProbability.java rename to sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/CallShouldBackoffBasedOnRejectionProbability.java index 8298809fca8d..62a7990d21ee 100644 --- a/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/CallShouldBackoffBasedOnRejectionProbability.java +++ b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/CallShouldBackoffBasedOnRejectionProbability.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.io.requestresponseio; +package org.apache.beam.io.requestresponse; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/Caller.java b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/Caller.java similarity index 96% rename from sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/Caller.java rename to sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/Caller.java index 32b514c43a15..da636c863740 100644 --- a/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/Caller.java +++ b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/Caller.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.io.requestresponseio; +package org.apache.beam.io.requestresponse; import java.io.Serializable; diff --git a/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/RequestResponseIO.java b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/RequestResponseIO.java similarity index 98% rename from sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/RequestResponseIO.java rename to sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/RequestResponseIO.java index 2ff0d50f68d5..de7d26aab4bd 100644 --- a/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/RequestResponseIO.java +++ b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/RequestResponseIO.java @@ -15,11 +15,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.io.requestresponseio; +package org.apache.beam.io.requestresponse; import com.google.auto.value.AutoValue; import java.util.Map; -import org.apache.beam.io.requestresponseio.RequestResponseIO.Result; +import org.apache.beam.io.requestresponse.RequestResponseIO.Result; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.values.PCollection; diff --git a/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/SetupTeardown.java b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/SetupTeardown.java similarity index 96% rename from sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/SetupTeardown.java rename to sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/SetupTeardown.java index 2bdc8113d98e..be1b03105c3d 100644 --- a/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/SetupTeardown.java +++ b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/SetupTeardown.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.io.requestresponseio; +package org.apache.beam.io.requestresponse; import java.io.Serializable; diff --git a/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/UserCodeExecutionException.java b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/UserCodeExecutionException.java similarity index 96% rename from sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/UserCodeExecutionException.java rename to sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/UserCodeExecutionException.java index 3a4c002f52e8..be545b6da66a 100644 --- a/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/UserCodeExecutionException.java +++ b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/UserCodeExecutionException.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.io.requestresponseio; +package org.apache.beam.io.requestresponse; /** Base {@link Exception} for signaling errors in user custom code. */ public class UserCodeExecutionException extends Exception { diff --git a/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/UserCodeQuotaException.java b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/UserCodeQuotaException.java similarity index 96% rename from sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/UserCodeQuotaException.java rename to sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/UserCodeQuotaException.java index f16f078927f8..c513a5371da7 100644 --- a/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/UserCodeQuotaException.java +++ b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/UserCodeQuotaException.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.io.requestresponseio; +package org.apache.beam.io.requestresponse; /** * Extends {@link UserCodeQuotaException} to allow the user custom code to specifically signal a diff --git a/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/UserCodeTimeoutException.java b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/UserCodeTimeoutException.java similarity index 96% rename from sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/UserCodeTimeoutException.java rename to sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/UserCodeTimeoutException.java index 22b067449858..869b8a51b73f 100644 --- a/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/UserCodeTimeoutException.java +++ b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/UserCodeTimeoutException.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.io.requestresponseio; +package org.apache.beam.io.requestresponse; /** An extension of {@link UserCodeQuotaException} to specifically signal a user code timeout. */ public class UserCodeTimeoutException extends UserCodeExecutionException { diff --git a/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/package-info.java b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/package-info.java similarity index 95% rename from sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/package-info.java rename to sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/package-info.java index cd9c11c13f86..abaea0a58b32 100644 --- a/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/package-info.java +++ b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/package-info.java @@ -17,4 +17,4 @@ */ /** Package provides Beam I/O transform support for safely reading from and writing to Web APIs. */ -package org.apache.beam.io.requestresponseio; +package org.apache.beam.io.requestresponse; diff --git a/sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponseio/CallShouldBackoffBasedOnRejectionProbabilityTest.java b/sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponse/CallShouldBackoffBasedOnRejectionProbabilityTest.java similarity index 98% rename from sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponseio/CallShouldBackoffBasedOnRejectionProbabilityTest.java rename to sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponse/CallShouldBackoffBasedOnRejectionProbabilityTest.java index b1e8347a25cf..40aaa48c2692 100644 --- a/sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponseio/CallShouldBackoffBasedOnRejectionProbabilityTest.java +++ b/sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponse/CallShouldBackoffBasedOnRejectionProbabilityTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.io.requestresponseio; +package org.apache.beam.io.requestresponse; import static org.junit.Assert.assertEquals; diff --git a/sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponseio/CallerTest.java b/sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponse/CallerTest.java similarity index 98% rename from sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponseio/CallerTest.java rename to sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponse/CallerTest.java index 0ba2d93c5411..93f3de474c58 100644 --- a/sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponseio/CallerTest.java +++ b/sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponse/CallerTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.io.requestresponseio; +package org.apache.beam.io.requestresponse; import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; import static org.junit.Assert.assertEquals; diff --git a/sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponseio/SetupTeardownTest.java b/sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponse/SetupTeardownTest.java similarity index 98% rename from sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponseio/SetupTeardownTest.java rename to sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponse/SetupTeardownTest.java index 9ef2f88a29c5..eade6588955d 100644 --- a/sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponseio/SetupTeardownTest.java +++ b/sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponse/SetupTeardownTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.io.requestresponseio; +package org.apache.beam.io.requestresponse; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThrows; From 574255a04116a7b188393a08e7d1e9c94d412b5c Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 18 Oct 2023 09:19:50 -0400 Subject: [PATCH 172/435] Bump google.golang.org/grpc from 1.58.3 to 1.59.0 in /sdks (#29047) Bumps [google.golang.org/grpc](https://github.com/grpc/grpc-go) from 1.58.3 to 1.59.0. - [Release notes](https://github.com/grpc/grpc-go/releases) - [Commits](https://github.com/grpc/grpc-go/compare/v1.58.3...v1.59.0) --- updated-dependencies: - dependency-name: google.golang.org/grpc dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] <support@github.com> Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 2 +- sdks/go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 51b6ac696e9b..192802f77eaa 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -59,7 +59,7 @@ require ( golang.org/x/text v0.13.0 google.golang.org/api v0.147.0 google.golang.org/genproto v0.0.0-20231002182017-d307bd883b97 - google.golang.org/grpc v1.58.3 + google.golang.org/grpc v1.59.0 google.golang.org/protobuf v1.31.0 gopkg.in/retry.v1 v1.0.3 gopkg.in/yaml.v2 v2.4.0 diff --git a/sdks/go.sum b/sdks/go.sum index 7c3af169cee4..457fe2bf1ffe 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -713,8 +713,8 @@ google.golang.org/grpc v1.26.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8 google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= google.golang.org/grpc v1.27.1/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= google.golang.org/grpc v1.33.2/go.mod h1:JMHMWHQWaTccqQQlmk3MJZS+GWXOdAesneDmEnv2fbc= -google.golang.org/grpc v1.58.3 h1:BjnpXut1btbtgN/6sp+brB2Kbm2LjNXnidYujAVbSoQ= -google.golang.org/grpc v1.58.3/go.mod h1:tgX3ZQDlNJGU96V6yHh1T/JeoBQ2TXdr43YbYSsCJk0= +google.golang.org/grpc v1.59.0 h1:Z5Iec2pjwb+LEOqzpB2MR12/eKFhDPhuqW91O+4bwUk= +google.golang.org/grpc v1.59.0/go.mod h1:aUPDwccQo6OTjy7Hct4AfBPD1GptF4fyUjIkQ9YtF98= google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM= From 1cb87c34dbc48dc4ceb048037ce350ce7346116c Mon Sep 17 00:00:00 2001 From: magicgoody <131876064+magicgoody@users.noreply.github.com> Date: Wed, 18 Oct 2023 20:23:40 +0600 Subject: [PATCH 173/435] old py, java, spotbugs plugins fix (#29008) --- ...formanceTests_WordCountIT_PythonVersions.yml | 8 +++++++- ...eam_PostCommit_Java_Examples_Dataflow_V2.yml | 2 ++ ...ostCommit_Java_Examples_Dataflow_V2_Java.yml | 2 ++ ...beam_PostCommit_Java_PVR_Flink_Streaming.yml | 13 +++++++++++++ .../beam_PostCommit_Java_PVR_Samza.yml | 2 ++ ...eam_PostCommit_Java_PVR_Spark3_Streaming.yml | 4 +++- .../beam_PostCommit_Java_PVR_Spark_Batch.yml | 2 +- .../beam_PostCommit_PortableJar_Flink.yml | 1 + .../beam_PostCommit_PortableJar_Spark.yml | 1 + .github/workflows/beam_PostCommit_Python.yml | 1 + .../workflows/beam_PostCommit_Python_Arm.yml | 1 + ...beam_PostCommit_Python_Examples_Dataflow.yml | 1 + .../beam_PostCommit_Python_Examples_Direct.yml | 1 + .../beam_PostCommit_Python_Examples_Flink.yml | 1 + .../beam_PostCommit_Python_Examples_Spark.yml | 1 + .../beam_PostCommit_Python_MongoDBIO_IT.yml | 1 + ...ommit_Python_ValidatesContainer_Dataflow.yml | 1 + ...thon_ValidatesContainer_Dataflow_With_RC.yml | 1 + ...stCommit_Python_ValidatesRunner_Dataflow.yml | 1 + ..._PostCommit_Python_ValidatesRunner_Flink.yml | 1 + ..._PostCommit_Python_ValidatesRunner_Samza.yml | 1 + ..._PostCommit_Python_ValidatesRunner_Spark.yml | 1 + ...eam_PostCommit_Python_Xlang_Gcp_Dataflow.yml | 2 +- .../beam_PostCommit_Python_Xlang_Gcp_Direct.yml | 2 +- .../beam_PostCommit_Sickbay_Python.yml | 1 + .../beam_PostCommit_TransformService_Direct.yml | 2 +- .../workflows/beam_PostCommit_XVR_Direct.yml | 13 ++++++++++--- .github/workflows/beam_PostCommit_XVR_Flink.yml | 13 ++++++++++--- ..._PostCommit_XVR_JavaUsingPython_Dataflow.yml | 13 ++++++++++--- ...stCommit_XVR_PythonUsingJavaSQL_Dataflow.yml | 2 +- ..._PostCommit_XVR_PythonUsingJava_Dataflow.yml | 2 +- .github/workflows/beam_PostCommit_XVR_Samza.yml | 13 ++++++++++--- .../workflows/beam_PostCommit_XVR_Spark3.yml | 13 ++++++++++--- .github/workflows/beam_PreCommit_Java.yml | 9 ++++++++- ...mmit_Java_Amazon-Web-Services2_IO_Direct.yml | 9 ++++++++- ...ommit_Java_Amazon-Web-Services_IO_Direct.yml | 9 ++++++++- .../beam_PreCommit_Java_Amqp_IO_Direct.yml | 9 ++++++++- .../beam_PreCommit_Java_Azure_IO_Direct.yml | 9 ++++++++- .../beam_PreCommit_Java_Cassandra_IO_Direct.yml | 9 ++++++++- .../beam_PreCommit_Java_Cdap_IO_Direct.yml | 9 ++++++++- ...beam_PreCommit_Java_Clickhouse_IO_Direct.yml | 9 ++++++++- .../beam_PreCommit_Java_Csv_IO_Direct.yml | 9 ++++++++- .../beam_PreCommit_Java_Debezium_IO_Direct.yml | 9 ++++++++- ...m_PreCommit_Java_ElasticSearch_IO_Direct.yml | 9 ++++++++- ...mit_Java_File-schema-transform_IO_Direct.yml | 9 ++++++++- .../beam_PreCommit_Java_GCP_IO_Direct.yml | 9 ++++++++- .../beam_PreCommit_Java_HBase_IO_Direct.yml | 9 ++++++++- .../beam_PreCommit_Java_HCatalog_IO_Direct.yml | 9 ++++++++- .../beam_PreCommit_Java_Hadoop_IO_Direct.yml | 9 ++++++++- .../beam_PreCommit_Java_IOs_Direct.yml | 9 ++++++++- .../beam_PreCommit_Java_InfluxDb_IO_Direct.yml | 9 ++++++++- .../beam_PreCommit_Java_JDBC_IO_Direct.yml | 9 ++++++++- .../beam_PreCommit_Java_Jms_IO_Direct.yml | 9 ++++++++- .../beam_PreCommit_Java_Kafka_IO_Direct.yml | 9 ++++++++- .../beam_PreCommit_Java_Kinesis_IO_Direct.yml | 9 ++++++++- .../beam_PreCommit_Java_Kudu_IO_Direct.yml | 9 ++++++++- .../beam_PreCommit_Java_MongoDb_IO_Direct.yml | 9 ++++++++- .../beam_PreCommit_Java_Mqtt_IO_Direct.yml | 9 ++++++++- .../beam_PreCommit_Java_Neo4j_IO_Direct.yml | 9 ++++++++- .../beam_PreCommit_Java_Parquet_IO_Direct.yml | 9 ++++++++- .../beam_PreCommit_Java_Pulsar_IO_Direct.yml | 9 ++++++++- .../beam_PreCommit_Java_RabbitMq_IO_Direct.yml | 9 ++++++++- .../beam_PreCommit_Java_Redis_IO_Direct.yml | 9 ++++++++- ...eam_PreCommit_Java_SingleStore_IO_Direct.yml | 9 ++++++++- .../beam_PreCommit_Java_Snowflake_IO_Direct.yml | 9 ++++++++- .../beam_PreCommit_Java_Solr_IO_Direct.yml | 9 ++++++++- .../beam_PreCommit_Java_Splunk_IO_Direct.yml | 9 ++++++++- .../beam_PreCommit_Java_Thrift_IO_Direct.yml | 9 ++++++++- .../beam_PreCommit_Java_Tika_IO_Direct.yml | 9 ++++++++- .github/workflows/beam_PreCommit_Python.yml | 1 + .../beam_PreCommit_Python_Coverage.yml | 1 + .../beam_PreCommit_Python_Dataframes.yml | 1 + .../beam_PreCommit_Python_Examples.yml | 1 + .../beam_PreCommit_Python_Integration.yml | 1 + .../beam_PreCommit_Python_PVR_Flink.yml | 1 + .../workflows/beam_PreCommit_Python_Runners.yml | 1 + .../beam_PreCommit_Python_Transforms.yml | 1 + .github/workflows/beam_PreCommit_SQL.yml | 7 +++++++ .github/workflows/beam_PreCommit_SQL_Java11.yml | 3 ++- .github/workflows/beam_PreCommit_SQL_Java17.yml | 17 ++++++++++++----- ...m_Python_ValidatesContainer_Dataflow_ARM.yml | 1 + 81 files changed, 419 insertions(+), 65 deletions(-) diff --git a/.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml b/.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml index 4fffb9389147..ad9d8f713398 100644 --- a/.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml +++ b/.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml @@ -100,4 +100,10 @@ jobs: arguments: | --info \ -Ptest=apache_beam/examples/wordcount_it_test.py::WordCountIT::test_wordcount_it \ - "-Ptest-pipeline-options=${{ env.beam_PerformanceTests_WordCountIT_PythonVersions_test_arguments_1 }}" \ No newline at end of file + "-Ptest-pipeline-options=${{ env.beam_PerformanceTests_WordCountIT_PythonVersions_test_arguments_1 }}" + - name: Archive code coverage results + uses: actions/upload-artifact@v3 + if: always() + with: + name: python-code-coverage-report + path: "**/pytest*.xml" \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml index aadb14b50ef2..ab72872d322f 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml @@ -84,4 +84,6 @@ jobs: uses: EnricoMi/publish-unit-test-result-action@v2 if: always() with: + commit: '${{ env.prsha || env.GITHUB_SHA }}' + comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml index 1277153e20b2..151b09469c5a 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml @@ -98,4 +98,6 @@ jobs: uses: EnricoMi/publish-unit-test-result-action@v2 if: always() with: + commit: '${{ env.prsha || env.GITHUB_SHA }}' + comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml b/.github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml index 954aa045e6b7..050a6fd7289f 100644 --- a/.github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml +++ b/.github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml @@ -73,3 +73,16 @@ jobs: uses: ./.github/actions/gradle-command-self-hosted-action with: gradle-command: runners:flink:1.15:job-server:validatesPortableRunnerStreaming + - name: Archive JUnit Test Results + uses: actions/upload-artifact@v3 + if: failure() + with: + name: JUnit Test Results + path: "**/build/reports/tests/" + - name: Publish JUnit Test Results + uses: EnricoMi/publish-unit-test-result-action@v2 + if: always() + with: + commit: '${{ env.prsha || env.GITHUB_SHA }}' + comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} + files: '**/build/test-results/**/*.xml' \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Java_PVR_Samza.yml b/.github/workflows/beam_PostCommit_Java_PVR_Samza.yml index 54016ea45cd3..62ff353157b8 100644 --- a/.github/workflows/beam_PostCommit_Java_PVR_Samza.yml +++ b/.github/workflows/beam_PostCommit_Java_PVR_Samza.yml @@ -85,4 +85,6 @@ jobs: uses: EnricoMi/publish-unit-test-result-action@v2 if: always() with: + commit: '${{ env.prsha || env.GITHUB_SHA }}' + comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml b/.github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml index 550b794bc16c..1114ae06ec32 100644 --- a/.github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml +++ b/.github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml @@ -83,4 +83,6 @@ jobs: uses: EnricoMi/publish-unit-test-result-action@v2 if: always() with: - files: '**/build/test-results/**/*.xml' + commit: '${{ env.prsha || env.GITHUB_SHA }}' + comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} + files: '**/build/test-results/**/*.xml' \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml b/.github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml index 2df2b39cd143..2d506eb1d42d 100644 --- a/.github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml +++ b/.github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml @@ -96,4 +96,4 @@ jobs: uses: actions/upload-artifact@v3 with: name: SpotBugs Results - path: "**/build/reports/spotbugs/*.html" \ No newline at end of file + path: "**/build/reports/spotbugs/*.html" diff --git a/.github/workflows/beam_PostCommit_PortableJar_Flink.yml b/.github/workflows/beam_PostCommit_PortableJar_Flink.yml index b7c67645699a..854371af6ef7 100644 --- a/.github/workflows/beam_PostCommit_PortableJar_Flink.yml +++ b/.github/workflows/beam_PostCommit_PortableJar_Flink.yml @@ -81,6 +81,7 @@ jobs: -PpythonVersion=3.8 \ - name: Archive code coverage results uses: actions/upload-artifact@v3 + if: always() with: name: python-code-coverage-report path: "**/pytest*.xml" \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_PortableJar_Spark.yml b/.github/workflows/beam_PostCommit_PortableJar_Spark.yml index 06360db4d82d..fbafbbda4608 100644 --- a/.github/workflows/beam_PostCommit_PortableJar_Spark.yml +++ b/.github/workflows/beam_PostCommit_PortableJar_Spark.yml @@ -81,6 +81,7 @@ jobs: -PpythonVersion=3.8 \ - name: Archive code coverage results uses: actions/upload-artifact@v3 + if: always() with: name: python-code-coverage-report path: "**/pytest*.xml" \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Python.yml b/.github/workflows/beam_PostCommit_Python.yml index 22c15dec1bed..5444fe204453 100644 --- a/.github/workflows/beam_PostCommit_Python.yml +++ b/.github/workflows/beam_PostCommit_Python.yml @@ -96,6 +96,7 @@ jobs: CLOUDSDK_CONFIG: ${{ env.KUBELET_GCLOUD_CONFIG_PATH}} - name: Archive code coverage results uses: actions/upload-artifact@v3 + if: always() with: name: python-code-coverage-report path: "**/pytest*.xml" \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Python_Arm.yml b/.github/workflows/beam_PostCommit_Python_Arm.yml index bd7b7158db34..50e70e85edc0 100644 --- a/.github/workflows/beam_PostCommit_Python_Arm.yml +++ b/.github/workflows/beam_PostCommit_Python_Arm.yml @@ -112,6 +112,7 @@ jobs: USER: github-actions - name: Archive code coverage results uses: actions/upload-artifact@v3 + if: always() with: name: python-code-coverage-report path: "**/pytest*.xml" diff --git a/.github/workflows/beam_PostCommit_Python_Examples_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_Examples_Dataflow.yml index 329eec386ec0..941ee9994436 100644 --- a/.github/workflows/beam_PostCommit_Python_Examples_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_Examples_Dataflow.yml @@ -81,6 +81,7 @@ jobs: -PpythonVersion=3.11 \ - name: Archive code coverage results uses: actions/upload-artifact@v3 + if: always() with: name: python-code-coverage-report path: "**/pytest*.xml" \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Python_Examples_Direct.yml b/.github/workflows/beam_PostCommit_Python_Examples_Direct.yml index b1d39903a405..a54981bde887 100644 --- a/.github/workflows/beam_PostCommit_Python_Examples_Direct.yml +++ b/.github/workflows/beam_PostCommit_Python_Examples_Direct.yml @@ -88,6 +88,7 @@ jobs: -PpythonVersion=${{ matrix.python_version }} \ - name: Archive code coverage results uses: actions/upload-artifact@v3 + if: always() with: name: python-code-coverage-report path: "**/pytest*.xml" \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Python_Examples_Flink.yml b/.github/workflows/beam_PostCommit_Python_Examples_Flink.yml index 0c3017af34b3..2cc2ff69aed9 100644 --- a/.github/workflows/beam_PostCommit_Python_Examples_Flink.yml +++ b/.github/workflows/beam_PostCommit_Python_Examples_Flink.yml @@ -88,6 +88,7 @@ jobs: -PpythonVersion=${{ matrix.python_version }} \ - name: Archive code coverage results uses: actions/upload-artifact@v3 + if: always() with: name: python-code-coverage-report path: "**/pytest*.xml" \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Python_Examples_Spark.yml b/.github/workflows/beam_PostCommit_Python_Examples_Spark.yml index 46315ae80e54..4c8b39f5f116 100644 --- a/.github/workflows/beam_PostCommit_Python_Examples_Spark.yml +++ b/.github/workflows/beam_PostCommit_Python_Examples_Spark.yml @@ -88,6 +88,7 @@ jobs: -PpythonVersion=${{ matrix.python_version }} \ - name: Archive code coverage results uses: actions/upload-artifact@v3 + if: always() with: name: python-code-coverage-report path: "**/pytest*.xml" \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml b/.github/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml index 5a58f11e2dbc..f58632bad758 100644 --- a/.github/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml +++ b/.github/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml @@ -80,6 +80,7 @@ jobs: -PpythonVersion=3.11 \ - name: Archive code coverage results uses: actions/upload-artifact@v3 + if: always() with: name: python-code-coverage-report path: "**/pytest*.xml" \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml index 5b491c4c3e50..364f1455e955 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml @@ -90,6 +90,7 @@ jobs: -PpythonVersion=${{ matrix.python_version }} \ - name: Archive code coverage results uses: actions/upload-artifact@v3 + if: always() with: name: python-code-coverage-report path: "**/pytest*.xml" \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml b/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml index a94b4f234e6e..164577228e4f 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml @@ -91,6 +91,7 @@ jobs: -PpythonVersion=${{ matrix.python_version }} \ - name: Archive code coverage results uses: actions/upload-artifact@v3 + if: always() with: name: python-code-coverage-report path: "**/pytest*.xml" diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml index efb9833ba248..c9256c538594 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml @@ -96,6 +96,7 @@ jobs: -PpythonVersion=${{ matrix.python_version }} \ - name: Archive code coverage results uses: actions/upload-artifact@v3 + if: always() with: name: python-code-coverage-report path: "**/pytest*.xml" \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml index dfb1b15c8312..f6e6b3a182ef 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml @@ -90,6 +90,7 @@ jobs: -PpythonVersion=${{ matrix.python_version }} \ - name: Archive code coverage results uses: actions/upload-artifact@v3 + if: always() with: name: python-code-coverage-report path: "**/pytest*.xml" \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml index a8e4495e3cd8..805640999a57 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml @@ -88,6 +88,7 @@ jobs: -PpythonVersion=${{ matrix.python_version }} \ - name: Archive code coverage results uses: actions/upload-artifact@v3 + if: always() with: name: python-code-coverage-report path: "**/pytest*.xml" \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml index d5c88efe9074..fb8021030113 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml @@ -88,6 +88,7 @@ jobs: -PpythonVersion=${{ matrix.python_version }} \ - name: Archive code coverage results uses: actions/upload-artifact@v3 + if: always() with: name: python-code-coverage-report path: "**/pytest*.xml" \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml index 3a5934fe7aa2..cdfae228a53e 100644 --- a/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml @@ -82,5 +82,5 @@ jobs: uses: actions/upload-artifact@v3 if: always() with: - name: archiveJunit + name: python-code-coverage-report path: "**/pytest*.xml" \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml b/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml index b1fcfc52352d..d58098ed3c72 100644 --- a/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml +++ b/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml @@ -81,5 +81,5 @@ jobs: uses: actions/upload-artifact@v3 if: always() with: - name: archiveJunit + name: python-code-coverage-report path: "**/pytest*.xml" \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Sickbay_Python.yml b/.github/workflows/beam_PostCommit_Sickbay_Python.yml index 914f35bcd47c..c8fa9759c65d 100644 --- a/.github/workflows/beam_PostCommit_Sickbay_Python.yml +++ b/.github/workflows/beam_PostCommit_Sickbay_Python.yml @@ -91,6 +91,7 @@ jobs: -PpythonVersion=${{ matrix.python_version }} \ - name: Archive code coverage results uses: actions/upload-artifact@v3 + if: always() with: name: python-code-coverage-report path: "**/pytest*.xml" \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_TransformService_Direct.yml b/.github/workflows/beam_PostCommit_TransformService_Direct.yml index 44ebcb83fb35..8be0ebf3986f 100644 --- a/.github/workflows/beam_PostCommit_TransformService_Direct.yml +++ b/.github/workflows/beam_PostCommit_TransformService_Direct.yml @@ -92,5 +92,5 @@ jobs: uses: actions/upload-artifact@v3 if: always() with: - name: archiveJunit + name: python-code-coverage-report path: "**/pytest*.xml" \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_XVR_Direct.yml b/.github/workflows/beam_PostCommit_XVR_Direct.yml index e5771b4f9d44..dea36ed312e0 100644 --- a/.github/workflows/beam_PostCommit_XVR_Direct.yml +++ b/.github/workflows/beam_PostCommit_XVR_Direct.yml @@ -94,9 +94,16 @@ jobs: arguments: | -PpythonVersion=${{ matrix.python_version }} \ -PskipNonPythonTask=false \ - - name: Archive code coverage results + - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 + if: failure() + with: + name: JUnit Test Results + path: "**/build/reports/tests/" + - name: Publish JUnit Test Results + uses: EnricoMi/publish-unit-test-result-action@v2 if: always() with: - name: archiveJunit - path: "**/build/test-results/**/*.xml" \ No newline at end of file + commit: '${{ env.prsha || env.GITHUB_SHA }}' + comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} + files: '**/build/test-results/**/*.xml' \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_XVR_Flink.yml b/.github/workflows/beam_PostCommit_XVR_Flink.yml index 5bbde0e86b94..d076b7c341e9 100644 --- a/.github/workflows/beam_PostCommit_XVR_Flink.yml +++ b/.github/workflows/beam_PostCommit_XVR_Flink.yml @@ -95,9 +95,16 @@ jobs: arguments: | -PpythonVersion=${{ matrix.python_version }} \ -PskipNonPythonTask=false \ - - name: Archive code coverage results + - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 + if: failure() + with: + name: JUnit Test Results + path: "**/build/reports/tests/" + - name: Publish JUnit Test Results + uses: EnricoMi/publish-unit-test-result-action@v2 if: always() with: - name: archiveJunit - path: "**/build/test-results/**/*.xml" \ No newline at end of file + commit: '${{ env.prsha || env.GITHUB_SHA }}' + comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} + files: '**/build/test-results/**/*.xml' \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml b/.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml index 52021e547de7..79731b9cd085 100644 --- a/.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml @@ -82,9 +82,16 @@ jobs: gradle-command: :runners:google-cloud-dataflow-java:validatesCrossLanguageRunnerJavaUsingPython arguments: | -PpythonVersion=${{ matrix.python_version }} \ - - name: Archive code coverage results + - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 + if: failure() + with: + name: JUnit Test Results + path: "**/build/reports/tests/" + - name: Publish JUnit Test Results + uses: EnricoMi/publish-unit-test-result-action@v2 if: always() with: - name: archiveJunit - path: "**/build/test-results/**/*.xml" \ No newline at end of file + commit: '${{ env.prsha || env.GITHUB_SHA }}' + comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} + files: '**/build/test-results/**/*.xml' \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml b/.github/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml index 6bb2ecf62d83..8fdade48da41 100644 --- a/.github/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml @@ -83,5 +83,5 @@ jobs: uses: actions/upload-artifact@v3 if: always() with: - name: archiveJunit + name: python-code-coverage-report path: "**/pytest*.xml" \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml b/.github/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml index 924eb525dbf7..c0d2c90d027a 100644 --- a/.github/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml @@ -86,5 +86,5 @@ jobs: uses: actions/upload-artifact@v3 if: always() with: - name: archiveJunit + name: python-code-coverage-report path: "**/pytest*.xml" \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_XVR_Samza.yml b/.github/workflows/beam_PostCommit_XVR_Samza.yml index b00daae8f31a..681169c91d6b 100644 --- a/.github/workflows/beam_PostCommit_XVR_Samza.yml +++ b/.github/workflows/beam_PostCommit_XVR_Samza.yml @@ -94,9 +94,16 @@ jobs: arguments: | -PpythonVersion=${{ matrix.python_version }} \ -PskipNonPythonTask=false \ - - name: Archive code coverage results + - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 + if: failure() + with: + name: JUnit Test Results + path: "**/build/reports/tests/" + - name: Publish JUnit Test Results + uses: EnricoMi/publish-unit-test-result-action@v2 if: always() with: - name: archiveJunit - path: "**/build/test-results/**/*.xml" \ No newline at end of file + commit: '${{ env.prsha || env.GITHUB_SHA }}' + comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} + files: '**/build/test-results/**/*.xml' \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_XVR_Spark3.yml b/.github/workflows/beam_PostCommit_XVR_Spark3.yml index 55275c85b563..962ea0228784 100644 --- a/.github/workflows/beam_PostCommit_XVR_Spark3.yml +++ b/.github/workflows/beam_PostCommit_XVR_Spark3.yml @@ -94,9 +94,16 @@ jobs: arguments: | -PpythonVersion=${{ matrix.python_version }} \ -PskipNonPythonTask=false \ - - name: Archive code coverage results + - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 + if: failure() + with: + name: JUnit Test Results + path: "**/build/reports/tests/" + - name: Publish JUnit Test Results + uses: EnricoMi/publish-unit-test-result-action@v2 if: always() with: - name: archiveJunit - path: "**/build/test-results/**/*.xml" \ No newline at end of file + commit: '${{ env.prsha || env.GITHUB_SHA }}' + comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} + files: '**/build/test-results/**/*.xml' \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Java.yml b/.github/workflows/beam_PreCommit_Java.yml index fdd51e703584..f302d12e9617 100644 --- a/.github/workflows/beam_PreCommit_Java.yml +++ b/.github/workflows/beam_PreCommit_Java.yml @@ -192,6 +192,13 @@ jobs: files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results uses: actions/upload-artifact@v3 + if: always() with: name: SpotBugs Results - path: "**/build/reports/spotbugs/*.html" + path: '**/build/reports/spotbugs/*.html' + - name: Publish SpotBugs Results + uses: jwgmeligmeyling/spotbugs-github-action@v1.2 + if: always() + with: + name: Publish SpotBugs + path: '**/build/reports/spotbugs/*.html' \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Java_Amazon-Web-Services2_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Amazon-Web-Services2_IO_Direct.yml index 22877bc543f9..67b27a06462d 100644 --- a/.github/workflows/beam_PreCommit_Java_Amazon-Web-Services2_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Amazon-Web-Services2_IO_Direct.yml @@ -128,6 +128,13 @@ jobs: files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results uses: actions/upload-artifact@v3 + if: always() with: name: SpotBugs Results - path: "**/build/reports/spotbugs/*.html" \ No newline at end of file + path: '**/build/reports/spotbugs/*.html' + - name: Publish SpotBugs Results + uses: jwgmeligmeyling/spotbugs-github-action@v1.2 + if: always() + with: + name: Publish SpotBugs + path: '**/build/reports/spotbugs/*.html' \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Java_Amazon-Web-Services_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Amazon-Web-Services_IO_Direct.yml index 22c114a7c476..c8e56e928cb8 100644 --- a/.github/workflows/beam_PreCommit_Java_Amazon-Web-Services_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Amazon-Web-Services_IO_Direct.yml @@ -128,6 +128,13 @@ jobs: files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results uses: actions/upload-artifact@v3 + if: always() with: name: SpotBugs Results - path: "**/build/reports/spotbugs/*.html" \ No newline at end of file + path: '**/build/reports/spotbugs/*.html' + - name: Publish SpotBugs Results + uses: jwgmeligmeyling/spotbugs-github-action@v1.2 + if: always() + with: + name: Publish SpotBugs + path: '**/build/reports/spotbugs/*.html' \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Java_Amqp_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Amqp_IO_Direct.yml index 82f11ccdd223..050ff91493a0 100644 --- a/.github/workflows/beam_PreCommit_Java_Amqp_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Amqp_IO_Direct.yml @@ -103,6 +103,13 @@ jobs: files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results uses: actions/upload-artifact@v3 + if: always() with: name: SpotBugs Results - path: "**/build/reports/spotbugs/*.html" \ No newline at end of file + path: '**/build/reports/spotbugs/*.html' + - name: Publish SpotBugs Results + uses: jwgmeligmeyling/spotbugs-github-action@v1.2 + if: always() + with: + name: Publish SpotBugs + path: '**/build/reports/spotbugs/*.html' \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml index 16d1f64b9dcd..cbd33a766a99 100644 --- a/.github/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml @@ -121,6 +121,13 @@ jobs: files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results uses: actions/upload-artifact@v3 + if: always() with: name: SpotBugs Results - path: "**/build/reports/spotbugs/*.html" \ No newline at end of file + path: '**/build/reports/spotbugs/*.html' + - name: Publish SpotBugs Results + uses: jwgmeligmeyling/spotbugs-github-action@v1.2 + if: always() + with: + name: Publish SpotBugs + path: '**/build/reports/spotbugs/*.html' \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml index fd3f4fd68970..2b9066c1d27a 100644 --- a/.github/workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml @@ -103,6 +103,13 @@ jobs: files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results uses: actions/upload-artifact@v3 + if: always() with: name: SpotBugs Results - path: "**/build/reports/spotbugs/*.html" \ No newline at end of file + path: '**/build/reports/spotbugs/*.html' + - name: Publish SpotBugs Results + uses: jwgmeligmeyling/spotbugs-github-action@v1.2 + if: always() + with: + name: Publish SpotBugs + path: '**/build/reports/spotbugs/*.html' \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml index 7161f766d7ff..29447d880dee 100644 --- a/.github/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml @@ -107,6 +107,13 @@ jobs: files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results uses: actions/upload-artifact@v3 + if: always() with: name: SpotBugs Results - path: "**/build/reports/spotbugs/*.html" \ No newline at end of file + path: '**/build/reports/spotbugs/*.html' + - name: Publish SpotBugs Results + uses: jwgmeligmeyling/spotbugs-github-action@v1.2 + if: always() + with: + name: Publish SpotBugs + path: '**/build/reports/spotbugs/*.html' \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml index aa8d7aff5e0c..4086f690af4e 100644 --- a/.github/workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml @@ -103,6 +103,13 @@ jobs: files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results uses: actions/upload-artifact@v3 + if: always() with: name: SpotBugs Results - path: "**/build/reports/spotbugs/*.html" \ No newline at end of file + path: '**/build/reports/spotbugs/*.html' + - name: Publish SpotBugs Results + uses: jwgmeligmeyling/spotbugs-github-action@v1.2 + if: always() + with: + name: Publish SpotBugs + path: '**/build/reports/spotbugs/*.html' \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml index c84472e16e98..878951b97186 100644 --- a/.github/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml @@ -103,6 +103,13 @@ jobs: files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results uses: actions/upload-artifact@v3 + if: always() with: name: SpotBugs Results - path: "**/build/reports/spotbugs/*.html" \ No newline at end of file + path: '**/build/reports/spotbugs/*.html' + - name: Publish SpotBugs Results + uses: jwgmeligmeyling/spotbugs-github-action@v1.2 + if: always() + with: + name: Publish SpotBugs + path: '**/build/reports/spotbugs/*.html' \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml index b855b8280f50..281a9ce4a0ad 100644 --- a/.github/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml @@ -112,6 +112,13 @@ jobs: files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results uses: actions/upload-artifact@v3 + if: always() with: name: SpotBugs Results - path: "**/build/reports/spotbugs/*.html" \ No newline at end of file + path: '**/build/reports/spotbugs/*.html' + - name: Publish SpotBugs Results + uses: jwgmeligmeyling/spotbugs-github-action@v1.2 + if: always() + with: + name: Publish SpotBugs + path: '**/build/reports/spotbugs/*.html' \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml index 7cca73a505a4..f275d6276e00 100644 --- a/.github/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml @@ -116,6 +116,13 @@ jobs: files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results uses: actions/upload-artifact@v3 + if: always() with: name: SpotBugs Results - path: "**/build/reports/spotbugs/*.html" \ No newline at end of file + path: '**/build/reports/spotbugs/*.html' + - name: Publish SpotBugs Results + uses: jwgmeligmeyling/spotbugs-github-action@v1.2 + if: always() + with: + name: Publish SpotBugs + path: '**/build/reports/spotbugs/*.html' \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Java_File-schema-transform_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_File-schema-transform_IO_Direct.yml index 549543ccef89..2ad4de133228 100644 --- a/.github/workflows/beam_PreCommit_Java_File-schema-transform_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_File-schema-transform_IO_Direct.yml @@ -104,6 +104,13 @@ jobs: files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results uses: actions/upload-artifact@v3 + if: always() with: name: SpotBugs Results - path: "**/build/reports/spotbugs/*.html" \ No newline at end of file + path: '**/build/reports/spotbugs/*.html' + - name: Publish SpotBugs Results + uses: jwgmeligmeyling/spotbugs-github-action@v1.2 + if: always() + with: + name: Publish SpotBugs + path: '**/build/reports/spotbugs/*.html' \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml index b9068de73539..0c832aecf749 100644 --- a/.github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml @@ -125,6 +125,13 @@ jobs: files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results uses: actions/upload-artifact@v3 + if: always() with: name: SpotBugs Results - path: "**/build/reports/spotbugs/*.html" \ No newline at end of file + path: '**/build/reports/spotbugs/*.html' + - name: Publish SpotBugs Results + uses: jwgmeligmeyling/spotbugs-github-action@v1.2 + if: always() + with: + name: Publish SpotBugs + path: '**/build/reports/spotbugs/*.html' \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml index 2f5e04f3f29b..4e1e300bb925 100644 --- a/.github/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml @@ -105,6 +105,13 @@ jobs: files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results uses: actions/upload-artifact@v3 + if: always() with: name: SpotBugs Results - path: "**/build/reports/spotbugs/*.html" \ No newline at end of file + path: '**/build/reports/spotbugs/*.html' + - name: Publish SpotBugs Results + uses: jwgmeligmeyling/spotbugs-github-action@v1.2 + if: always() + with: + name: Publish SpotBugs + path: '**/build/reports/spotbugs/*.html' \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml index b95d111e1879..93eacb82cf2b 100644 --- a/.github/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml @@ -105,6 +105,13 @@ jobs: files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results uses: actions/upload-artifact@v3 + if: always() with: name: SpotBugs Results - path: "**/build/reports/spotbugs/*.html" \ No newline at end of file + path: '**/build/reports/spotbugs/*.html' + - name: Publish SpotBugs Results + uses: jwgmeligmeyling/spotbugs-github-action@v1.2 + if: always() + with: + name: Publish SpotBugs + path: '**/build/reports/spotbugs/*.html' \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml index 43f870711f4e..2a61bcfb3809 100644 --- a/.github/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml @@ -143,6 +143,13 @@ jobs: files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results uses: actions/upload-artifact@v3 + if: always() with: name: SpotBugs Results - path: "**/build/reports/spotbugs/*.html" \ No newline at end of file + path: '**/build/reports/spotbugs/*.html' + - name: Publish SpotBugs Results + uses: jwgmeligmeyling/spotbugs-github-action@v1.2 + if: always() + with: + name: Publish SpotBugs + path: '**/build/reports/spotbugs/*.html' \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Java_IOs_Direct.yml b/.github/workflows/beam_PreCommit_Java_IOs_Direct.yml index 7696ad7f6059..7936c391a644 100644 --- a/.github/workflows/beam_PreCommit_Java_IOs_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_IOs_Direct.yml @@ -104,6 +104,13 @@ jobs: files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results uses: actions/upload-artifact@v3 + if: always() with: name: SpotBugs Results - path: "**/build/reports/spotbugs/*.html" \ No newline at end of file + path: '**/build/reports/spotbugs/*.html' + - name: Publish SpotBugs Results + uses: jwgmeligmeyling/spotbugs-github-action@v1.2 + if: always() + with: + name: Publish SpotBugs + path: '**/build/reports/spotbugs/*.html' \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml index 2174f836f2a9..607a202b4494 100644 --- a/.github/workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml @@ -103,6 +103,13 @@ jobs: files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results uses: actions/upload-artifact@v3 + if: always() with: name: SpotBugs Results - path: "**/build/reports/spotbugs/*.html" \ No newline at end of file + path: '**/build/reports/spotbugs/*.html' + - name: Publish SpotBugs Results + uses: jwgmeligmeyling/spotbugs-github-action@v1.2 + if: always() + with: + name: Publish SpotBugs + path: '**/build/reports/spotbugs/*.html' \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml index b364d2d1cc1a..eab48ce8a0f3 100644 --- a/.github/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml @@ -110,6 +110,13 @@ jobs: files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results uses: actions/upload-artifact@v3 + if: always() with: name: SpotBugs Results - path: "**/build/reports/spotbugs/*.html" \ No newline at end of file + path: '**/build/reports/spotbugs/*.html' + - name: Publish SpotBugs Results + uses: jwgmeligmeyling/spotbugs-github-action@v1.2 + if: always() + with: + name: Publish SpotBugs + path: '**/build/reports/spotbugs/*.html' \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml index 05b081f4e737..aebe9984ab73 100644 --- a/.github/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml @@ -110,6 +110,13 @@ jobs: files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results uses: actions/upload-artifact@v3 + if: always() with: name: SpotBugs Results - path: "**/build/reports/spotbugs/*.html" \ No newline at end of file + path: '**/build/reports/spotbugs/*.html' + - name: Publish SpotBugs Results + uses: jwgmeligmeyling/spotbugs-github-action@v1.2 + if: always() + with: + name: Publish SpotBugs + path: '**/build/reports/spotbugs/*.html' \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml index 667f00e234a0..8b7b35922d37 100644 --- a/.github/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml @@ -112,6 +112,13 @@ jobs: files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results uses: actions/upload-artifact@v3 + if: always() with: name: SpotBugs Results - path: "**/build/reports/spotbugs/*.html" \ No newline at end of file + path: '**/build/reports/spotbugs/*.html' + - name: Publish SpotBugs Results + uses: jwgmeligmeyling/spotbugs-github-action@v1.2 + if: always() + with: + name: Publish SpotBugs + path: '**/build/reports/spotbugs/*.html' \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml index 6abd50dc6c4c..be29e3b87495 100644 --- a/.github/workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml @@ -135,6 +135,13 @@ jobs: files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results uses: actions/upload-artifact@v3 + if: always() with: name: SpotBugs Results - path: "**/build/reports/spotbugs/*.html" \ No newline at end of file + path: '**/build/reports/spotbugs/*.html' + - name: Publish SpotBugs Results + uses: jwgmeligmeyling/spotbugs-github-action@v1.2 + if: always() + with: + name: Publish SpotBugs + path: '**/build/reports/spotbugs/*.html' \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml index 0263ce70cf50..12cb1b2aa230 100644 --- a/.github/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml @@ -103,6 +103,13 @@ jobs: files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results uses: actions/upload-artifact@v3 + if: always() with: name: SpotBugs Results - path: "**/build/reports/spotbugs/*.html" \ No newline at end of file + path: '**/build/reports/spotbugs/*.html' + - name: Publish SpotBugs Results + uses: jwgmeligmeyling/spotbugs-github-action@v1.2 + if: always() + with: + name: Publish SpotBugs + path: '**/build/reports/spotbugs/*.html' \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml index dfbc30e82161..19ca7a24a27a 100644 --- a/.github/workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml @@ -103,6 +103,13 @@ jobs: files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results uses: actions/upload-artifact@v3 + if: always() with: name: SpotBugs Results - path: "**/build/reports/spotbugs/*.html" \ No newline at end of file + path: '**/build/reports/spotbugs/*.html' + - name: Publish SpotBugs Results + uses: jwgmeligmeyling/spotbugs-github-action@v1.2 + if: always() + with: + name: Publish SpotBugs + path: '**/build/reports/spotbugs/*.html' \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml index 13c874737f0b..5e9c8ecf9a48 100644 --- a/.github/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml @@ -103,6 +103,13 @@ jobs: files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results uses: actions/upload-artifact@v3 + if: always() with: name: SpotBugs Results - path: "**/build/reports/spotbugs/*.html" \ No newline at end of file + path: '**/build/reports/spotbugs/*.html' + - name: Publish SpotBugs Results + uses: jwgmeligmeyling/spotbugs-github-action@v1.2 + if: always() + with: + name: Publish SpotBugs + path: '**/build/reports/spotbugs/*.html' \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml index 27345be9adc6..0dc3f2c946ad 100644 --- a/.github/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml @@ -112,6 +112,13 @@ jobs: files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results uses: actions/upload-artifact@v3 + if: always() with: name: SpotBugs Results - path: "**/build/reports/spotbugs/*.html" \ No newline at end of file + path: '**/build/reports/spotbugs/*.html' + - name: Publish SpotBugs Results + uses: jwgmeligmeyling/spotbugs-github-action@v1.2 + if: always() + with: + name: Publish SpotBugs + path: '**/build/reports/spotbugs/*.html' \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml index 4847e9b2a01b..a0f021bb82d5 100644 --- a/.github/workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml @@ -103,6 +103,13 @@ jobs: files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results uses: actions/upload-artifact@v3 + if: always() with: name: SpotBugs Results - path: "**/build/reports/spotbugs/*.html" \ No newline at end of file + path: '**/build/reports/spotbugs/*.html' + - name: Publish SpotBugs Results + uses: jwgmeligmeyling/spotbugs-github-action@v1.2 + if: always() + with: + name: Publish SpotBugs + path: '**/build/reports/spotbugs/*.html' \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml index 810a787ac7b3..ef22e19ca514 100644 --- a/.github/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml @@ -121,6 +121,13 @@ jobs: files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results uses: actions/upload-artifact@v3 + if: always() with: name: SpotBugs Results - path: "**/build/reports/spotbugs/*.html" \ No newline at end of file + path: '**/build/reports/spotbugs/*.html' + - name: Publish SpotBugs Results + uses: jwgmeligmeyling/spotbugs-github-action@v1.2 + if: always() + with: + name: Publish SpotBugs + path: '**/build/reports/spotbugs/*.html' \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml index 174df8a37a48..f03b38b18257 100644 --- a/.github/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml @@ -103,6 +103,13 @@ jobs: files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results uses: actions/upload-artifact@v3 + if: always() with: name: SpotBugs Results - path: "**/build/reports/spotbugs/*.html" \ No newline at end of file + path: '**/build/reports/spotbugs/*.html' + - name: Publish SpotBugs Results + uses: jwgmeligmeyling/spotbugs-github-action@v1.2 + if: always() + with: + name: Publish SpotBugs + path: '**/build/reports/spotbugs/*.html' \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml index a0e2dfeb9b58..21efdf4b574c 100644 --- a/.github/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml @@ -103,6 +103,13 @@ jobs: files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results uses: actions/upload-artifact@v3 + if: always() with: name: SpotBugs Results - path: "**/build/reports/spotbugs/*.html" \ No newline at end of file + path: '**/build/reports/spotbugs/*.html' + - name: Publish SpotBugs Results + uses: jwgmeligmeyling/spotbugs-github-action@v1.2 + if: always() + with: + name: Publish SpotBugs + path: '**/build/reports/spotbugs/*.html' \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml index eb2a8521c597..4e98c1fee708 100644 --- a/.github/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml @@ -105,6 +105,13 @@ jobs: files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results uses: actions/upload-artifact@v3 + if: always() with: name: SpotBugs Results - path: "**/build/reports/spotbugs/*.html" \ No newline at end of file + path: '**/build/reports/spotbugs/*.html' + - name: Publish SpotBugs Results + uses: jwgmeligmeyling/spotbugs-github-action@v1.2 + if: always() + with: + name: Publish SpotBugs + path: '**/build/reports/spotbugs/*.html' \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml index 14fc532c2f3f..4f7cfb6b8316 100644 --- a/.github/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml @@ -114,6 +114,13 @@ jobs: files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results uses: actions/upload-artifact@v3 + if: always() with: name: SpotBugs Results - path: "**/build/reports/spotbugs/*.html" \ No newline at end of file + path: '**/build/reports/spotbugs/*.html' + - name: Publish SpotBugs Results + uses: jwgmeligmeyling/spotbugs-github-action@v1.2 + if: always() + with: + name: Publish SpotBugs + path: '**/build/reports/spotbugs/*.html' \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml index 94db93a74b8c..4a30cfbf3f40 100644 --- a/.github/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml @@ -103,6 +103,13 @@ jobs: files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results uses: actions/upload-artifact@v3 + if: always() with: name: SpotBugs Results - path: "**/build/reports/spotbugs/*.html" \ No newline at end of file + path: '**/build/reports/spotbugs/*.html' + - name: Publish SpotBugs Results + uses: jwgmeligmeyling/spotbugs-github-action@v1.2 + if: always() + with: + name: Publish SpotBugs + path: '**/build/reports/spotbugs/*.html' \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml index 53a933cff922..786276aeb885 100644 --- a/.github/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml @@ -103,6 +103,13 @@ jobs: files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results uses: actions/upload-artifact@v3 + if: always() with: name: SpotBugs Results - path: "**/build/reports/spotbugs/*.html" \ No newline at end of file + path: '**/build/reports/spotbugs/*.html' + - name: Publish SpotBugs Results + uses: jwgmeligmeyling/spotbugs-github-action@v1.2 + if: always() + with: + name: Publish SpotBugs + path: '**/build/reports/spotbugs/*.html' \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml index 1ae098097cc9..64cd4a755e73 100644 --- a/.github/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml @@ -103,6 +103,13 @@ jobs: files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results uses: actions/upload-artifact@v3 + if: always() with: name: SpotBugs Results - path: "**/build/reports/spotbugs/*.html" \ No newline at end of file + path: '**/build/reports/spotbugs/*.html' + - name: Publish SpotBugs Results + uses: jwgmeligmeyling/spotbugs-github-action@v1.2 + if: always() + with: + name: Publish SpotBugs + path: '**/build/reports/spotbugs/*.html' \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml index 98b6654c6d5e..e6bed237a808 100644 --- a/.github/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml @@ -103,6 +103,13 @@ jobs: files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results uses: actions/upload-artifact@v3 + if: always() with: name: SpotBugs Results - path: "**/build/reports/spotbugs/*.html" \ No newline at end of file + path: '**/build/reports/spotbugs/*.html' + - name: Publish SpotBugs Results + uses: jwgmeligmeyling/spotbugs-github-action@v1.2 + if: always() + with: + name: Publish SpotBugs + path: '**/build/reports/spotbugs/*.html' \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Python.yml b/.github/workflows/beam_PreCommit_Python.yml index 1ee602e02c0c..26efbad6ca0c 100644 --- a/.github/workflows/beam_PreCommit_Python.yml +++ b/.github/workflows/beam_PreCommit_Python.yml @@ -100,6 +100,7 @@ jobs: -PuseWheelDistribution - name: Archive code coverage results uses: actions/upload-artifact@v3 + if: always() with: name: python-code-coverage-report path: "**/pytest*.xml" \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Python_Coverage.yml b/.github/workflows/beam_PreCommit_Python_Coverage.yml index 6bc3e3911921..68319e6140e4 100644 --- a/.github/workflows/beam_PreCommit_Python_Coverage.yml +++ b/.github/workflows/beam_PreCommit_Python_Coverage.yml @@ -90,6 +90,7 @@ jobs: -PuseWheelDistribution - name: Archive code coverage results uses: actions/upload-artifact@v3 + if: always() with: name: python-code-coverage-report path: "**/pytest*.xml" \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Python_Dataframes.yml b/.github/workflows/beam_PreCommit_Python_Dataframes.yml index 5750d020607b..f1464e7fe5b9 100644 --- a/.github/workflows/beam_PreCommit_Python_Dataframes.yml +++ b/.github/workflows/beam_PreCommit_Python_Dataframes.yml @@ -100,6 +100,7 @@ jobs: -PuseWheelDistribution - name: Archive code coverage results uses: actions/upload-artifact@v3 + if: always() with: name: python-code-coverage-report path: "**/pytest*.xml" \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Python_Examples.yml b/.github/workflows/beam_PreCommit_Python_Examples.yml index 39d65e7833ff..1cfdb9f09c33 100644 --- a/.github/workflows/beam_PreCommit_Python_Examples.yml +++ b/.github/workflows/beam_PreCommit_Python_Examples.yml @@ -100,6 +100,7 @@ jobs: -PuseWheelDistribution - name: Archive code coverage results uses: actions/upload-artifact@v3 + if: always() with: name: python-code-coverage-report path: "**/pytest*.xml" \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Python_Integration.yml b/.github/workflows/beam_PreCommit_Python_Integration.yml index a4a46815a775..b00330cbfe76 100644 --- a/.github/workflows/beam_PreCommit_Python_Integration.yml +++ b/.github/workflows/beam_PreCommit_Python_Integration.yml @@ -106,6 +106,7 @@ jobs: -PpythonVersion=${{ matrix.python_version }} \ - name: Archive code coverage results uses: actions/upload-artifact@v3 + if: always() with: name: python-code-coverage-report path: "**/pytest*.xml" diff --git a/.github/workflows/beam_PreCommit_Python_PVR_Flink.yml b/.github/workflows/beam_PreCommit_Python_PVR_Flink.yml index 6649958b4791..94655f959fd7 100644 --- a/.github/workflows/beam_PreCommit_Python_PVR_Flink.yml +++ b/.github/workflows/beam_PreCommit_Python_PVR_Flink.yml @@ -113,6 +113,7 @@ jobs: -PpythonVersion=3.11 \ - name: Archive code coverage results uses: actions/upload-artifact@v3 + if: always() with: name: python-code-coverage-report path: '**/pytest*.xml' diff --git a/.github/workflows/beam_PreCommit_Python_Runners.yml b/.github/workflows/beam_PreCommit_Python_Runners.yml index a76b04be8fa2..5a65faf6e8d3 100644 --- a/.github/workflows/beam_PreCommit_Python_Runners.yml +++ b/.github/workflows/beam_PreCommit_Python_Runners.yml @@ -100,6 +100,7 @@ jobs: -PuseWheelDistribution - name: Archive code coverage results uses: actions/upload-artifact@v3 + if: always() with: name: python-code-coverage-report path: "**/pytest*.xml" \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Python_Transforms.yml b/.github/workflows/beam_PreCommit_Python_Transforms.yml index e82fec205d83..197f77a38048 100644 --- a/.github/workflows/beam_PreCommit_Python_Transforms.yml +++ b/.github/workflows/beam_PreCommit_Python_Transforms.yml @@ -100,6 +100,7 @@ jobs: -PuseWheelDistribution - name: Archive code coverage results uses: actions/upload-artifact@v3 + if: always() with: name: python-code-coverage-report path: "**/pytest*.xml" \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_SQL.yml b/.github/workflows/beam_PreCommit_SQL.yml index 3e32c9653ce5..3dd36ce050af 100644 --- a/.github/workflows/beam_PreCommit_SQL.yml +++ b/.github/workflows/beam_PreCommit_SQL.yml @@ -106,6 +106,13 @@ jobs: files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results uses: actions/upload-artifact@v3 + if: always() with: name: SpotBugs Results + path: '**/build/reports/spotbugs/*.html' + - name: Publish SpotBugs Results + uses: jwgmeligmeyling/spotbugs-github-action@v1.2 + if: always() + with: + name: Publish SpotBugs path: '**/build/reports/spotbugs/*.html' \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_SQL_Java11.yml b/.github/workflows/beam_PreCommit_SQL_Java11.yml index 1f22c71cb920..0f8c032a598f 100644 --- a/.github/workflows/beam_PreCommit_SQL_Java11.yml +++ b/.github/workflows/beam_PreCommit_SQL_Java11.yml @@ -121,6 +121,7 @@ jobs: files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results uses: actions/upload-artifact@v3 + if: always() with: name: SpotBugs Results path: '**/build/reports/spotbugs/*.html' @@ -128,5 +129,5 @@ jobs: uses: jwgmeligmeyling/spotbugs-github-action@v1.2 if: always() with: - name: SpotBugs + name: Publish SpotBugs path: '**/build/reports/spotbugs/*.html' \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_SQL_Java17.yml b/.github/workflows/beam_PreCommit_SQL_Java17.yml index 3e216f9cd4c8..4b67db10f5fc 100644 --- a/.github/workflows/beam_PreCommit_SQL_Java17.yml +++ b/.github/workflows/beam_PreCommit_SQL_Java17.yml @@ -110,15 +110,22 @@ jobs: with: name: JUnit Test Results path: "**/build/reports/tests/" + - name: Publish JUnit Test Results + uses: EnricoMi/publish-unit-test-result-action@v2 + if: always() + with: + commit: '${{ env.prsha || env.GITHUB_SHA }}' + comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} + files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results uses: actions/upload-artifact@v3 + if: always() with: name: SpotBugs Results path: '**/build/reports/spotbugs/*.html' - - name: Publish JUnit Test Results - uses: EnricoMi/publish-unit-test-result-action@v2 + - name: Publish SpotBugs Results + uses: jwgmeligmeyling/spotbugs-github-action@v1.2 if: always() with: - commit: '${{ env.prsha || env.GITHUB_SHA }}' - comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/build/test-results/**/*.xml' \ No newline at end of file + name: Publish SpotBugs + path: '**/build/reports/spotbugs/*.html' \ No newline at end of file diff --git a/.github/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml b/.github/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml index c6352a102fc0..2bb4328eaf26 100644 --- a/.github/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml +++ b/.github/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml @@ -115,6 +115,7 @@ jobs: USER: github-actions - name: Archive code coverage results uses: actions/upload-artifact@v3 + if: always() with: name: python-code-coverage-report path: "**/pytest*.xml" From 4e6f10b5be51f90f9410b2a16652fb2c71625b77 Mon Sep 17 00:00:00 2001 From: Israel Herraiz <ihr@google.com> Date: Wed, 18 Oct 2023 16:59:35 +0200 Subject: [PATCH 174/435] Add withAppProfileId to BigtableIO Read and Write (#28864) --- .../beam/sdk/io/gcp/bigtable/BigtableIO.java | 44 +++++++++++++++++++ .../sdk/io/gcp/bigtable/BigtableIOTest.java | 6 ++- 2 files changed, 49 insertions(+), 1 deletion(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java index ef868e8bf7c9..ad978e95016a 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java @@ -457,6 +457,25 @@ public Read withTableId(String tableId) { return withTableId(StaticValueProvider.of(tableId)); } + /** + * Returns a new {@link BigtableIO.Read} that will read using the specified app profile id. + * + * <p>Does not modify this object. + */ + public Read withAppProfileId(ValueProvider<String> appProfileId) { + BigtableConfig config = getBigtableConfig(); + return toBuilder().setBigtableConfig(config.withAppProfileId(appProfileId)).build(); + } + + /** + * Returns a new {@link BigtableIO.Read} that will read using the specified app profile id. + * + * <p>Does not modify this object. + */ + public Read withAppProfileId(String appProfileId) { + return withAppProfileId(StaticValueProvider.of(appProfileId)); + } + /** * WARNING: Should be used only to specify additional parameters for connection to the Cloud * Bigtable, instanceId and projectId should be provided over {@link #withInstanceId} and {@link @@ -837,6 +856,31 @@ public Write withTableId(String tableId) { return withTableId(StaticValueProvider.of(tableId)); } + /** + * Returns a new {@link BigtableIO.Write} that will write using the specified app profile id. + * + * <p>Remember that in order to use single-row transactions, this must use a single-cluster + * routing policy. + * + * <p>Does not modify this object. + */ + public Write withAppProfileId(ValueProvider<String> appProfileId) { + BigtableConfig config = getBigtableConfig(); + return toBuilder().setBigtableConfig(config.withAppProfileId(appProfileId)).build(); + } + + /** + * Returns a new {@link BigtableIO.Write} that will write using the specified app profile id. + * + * <p>Remember that in order to use single-row transactions, this must use a single-cluster + * routing policy. + * + * <p>Does not modify this object. + */ + public Write withAppProfileId(String appProfileId) { + return withAppProfileId(StaticValueProvider.of(appProfileId)); + } + /** * WARNING: Should be used only to specify additional parameters for connection to the Cloud * Bigtable, instanceId and projectId should be provided over {@link #withInstanceId} and {@link diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java index f6224cc24b2f..714dc9f8619d 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java @@ -219,11 +219,13 @@ public void testReadBuildsCorrectly() { .withTableId("table") .withInstanceId("instance") .withProjectId("project") + .withAppProfileId("app-profile") .withBigtableOptionsConfigurator(PORT_CONFIGURATOR); assertEquals("options_project", read.getBigtableOptions().getProjectId()); assertEquals("options_instance", read.getBigtableOptions().getInstanceId()); assertEquals("instance", read.getBigtableConfig().getInstanceId().get()); assertEquals("project", read.getBigtableConfig().getProjectId().get()); + assertEquals("app-profile", read.getBigtableConfig().getAppProfileId().get()); assertEquals("table", read.getTableId()); assertEquals(PORT_CONFIGURATOR, read.getBigtableConfig().getBigtableOptionsConfigurator()); } @@ -373,12 +375,14 @@ public void testWriteBuildsCorrectly() { .withBigtableOptions(BIGTABLE_OPTIONS) .withTableId("table") .withInstanceId("instance") - .withProjectId("project"); + .withProjectId("project") + .withAppProfileId("app-profile"); assertEquals("table", write.getBigtableWriteOptions().getTableId().get()); assertEquals("options_project", write.getBigtableOptions().getProjectId()); assertEquals("options_instance", write.getBigtableOptions().getInstanceId()); assertEquals("instance", write.getBigtableConfig().getInstanceId().get()); assertEquals("project", write.getBigtableConfig().getProjectId().get()); + assertEquals("app-profile", write.getBigtableConfig().getAppProfileId().get()); } @Test From 5fbf65afd97f23568fe3c78c20197ad3b0354ece Mon Sep 17 00:00:00 2001 From: Pranav Bhandari <bhandari.pranav22@gmail.com> Date: Wed, 18 Oct 2023 11:07:51 -0400 Subject: [PATCH 175/435] Add `requirementsFile` parameter to LaunchConfig. (#28732) * Add requirementsFile parameter to LaunchConfig. * Install requirements in virtualenv for python jobs. --- .../beam/it/common/PipelineLauncher.java | 16 ++++++++++++++ .../gcp/dataflow/DefaultPipelineLauncher.java | 22 +++++++++++++++---- 2 files changed, 34 insertions(+), 4 deletions(-) diff --git a/it/common/src/main/java/org/apache/beam/it/common/PipelineLauncher.java b/it/common/src/main/java/org/apache/beam/it/common/PipelineLauncher.java index 8777bbec6c40..6d1aeae21dd4 100644 --- a/it/common/src/main/java/org/apache/beam/it/common/PipelineLauncher.java +++ b/it/common/src/main/java/org/apache/beam/it/common/PipelineLauncher.java @@ -121,6 +121,7 @@ class LaunchConfig { private final @Nullable String specPath; private final @Nullable Sdk sdk; private final @Nullable String executable; + private final @Nullable String requirementsFile; private final @Nullable Pipeline pipeline; private LaunchConfig(Builder builder) { @@ -130,6 +131,7 @@ private LaunchConfig(Builder builder) { this.specPath = builder.specPath; this.sdk = builder.sdk; this.executable = builder.executable; + this.requirementsFile = builder.requirementsFile; this.pipeline = builder.pipeline; } @@ -161,6 +163,10 @@ public ImmutableMap<String, Object> environment() { return executable; } + public @Nullable String requirementsFile() { + return requirementsFile; + } + public @Nullable Pipeline pipeline() { return pipeline; } @@ -185,6 +191,7 @@ public static final class Builder { private Map<String, String> parameters; private Sdk sdk; private String executable; + private String requirementsFile; private Pipeline pipeline; private Builder(String jobName, String specPath) { @@ -243,6 +250,15 @@ public Builder setExecutable(String executable) { return this; } + public @Nullable String getRequirementsFile() { + return requirementsFile; + } + + public Builder setRequirementsFile(String requirementsFile) { + this.requirementsFile = requirementsFile; + return this; + } + public @Nullable Pipeline getPipeline() { return pipeline; } diff --git a/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/dataflow/DefaultPipelineLauncher.java b/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/dataflow/DefaultPipelineLauncher.java index ad2dcafc007b..3d43618821aa 100644 --- a/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/dataflow/DefaultPipelineLauncher.java +++ b/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/dataflow/DefaultPipelineLauncher.java @@ -360,11 +360,22 @@ public LaunchInfo launch(String project, String region, LaunchConfig options) th options.executable() != null, "Cannot launch a dataflow job " + "without executable specified. Please specify executable and try again!"); + if (options.requirementsFile() != null) { + // install requirements + cmd.add( + "virtualenv . && source ./bin/activate && pip3 install -r " + + options.requirementsFile()); + cmd.add("&&"); + } LOG.info("Using the executable at {}", options.executable()); cmd.add("python3"); cmd.add(options.executable()); cmd.addAll(extractOptions(project, region, options)); - jobId = executeCommandAndParseResponse(cmd); + if (options.requirementsFile() != null) { + cmd.add("&&"); + cmd.add("deactivate"); + } + jobId = executeCommandAndParseResponse(String.join(" ", cmd)); break; case GO: checkState( @@ -376,7 +387,7 @@ public LaunchInfo launch(String project, String region, LaunchConfig options) th cmd.add("run"); cmd.add(options.executable()); cmd.addAll(extractOptions(project, region, options)); - jobId = executeCommandAndParseResponse(cmd); + jobId = executeCommandAndParseResponse(String.join(" ", cmd)); break; default: throw new RuntimeException( @@ -441,10 +452,13 @@ private List<String> extractOptions(String project, String region, LaunchConfig } /** Executes the specified command and parses the response to get the Job ID. */ - private String executeCommandAndParseResponse(List<String> cmd) throws IOException { - Process process = new ProcessBuilder().command(cmd).redirectErrorStream(true).start(); + private String executeCommandAndParseResponse(String cmd) throws IOException { + LOG.info("Running command: {}", cmd); + Process process = + new ProcessBuilder().command("/bin/bash", "-c", cmd).redirectErrorStream(true).start(); String output = new String(ByteStreams.toByteArray(process.getInputStream()), StandardCharsets.UTF_8); + LOG.info(output); Matcher m = JOB_ID_PATTERN.matcher(output); if (!m.find()) { throw new RuntimeException( From 9745ea224976af888049d5553a7190f9c5877fbd Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 18 Oct 2023 11:11:47 -0400 Subject: [PATCH 176/435] Bump urllib3 from 1.26.17 to 1.26.18 in /sdks/python/container/py310 (#29045) Bumps [urllib3](https://github.com/urllib3/urllib3) from 1.26.17 to 1.26.18. - [Release notes](https://github.com/urllib3/urllib3/releases) - [Changelog](https://github.com/urllib3/urllib3/blob/main/CHANGES.rst) - [Commits](https://github.com/urllib3/urllib3/compare/1.26.17...1.26.18) --- updated-dependencies: - dependency-name: urllib3 dependency-type: direct:production ... Signed-off-by: dependabot[bot] <support@github.com> Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/python/container/py310/base_image_requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/container/py310/base_image_requirements.txt b/sdks/python/container/py310/base_image_requirements.txt index 548c56371706..82b210005feb 100644 --- a/sdks/python/container/py310/base_image_requirements.txt +++ b/sdks/python/container/py310/base_image_requirements.txt @@ -134,7 +134,7 @@ tqdm==4.66.1 typing_extensions==4.8.0 tzlocal==5.0.1 uritemplate==4.1.1 -urllib3==1.26.17 +urllib3==1.26.18 websocket-client==1.6.3 wrapt==1.15.0 zstandard==0.21.0 From dbc5c4e15b1378125e610286d005e56ee605b96b Mon Sep 17 00:00:00 2001 From: Ritesh Ghorse <riteshghorse@gmail.com> Date: Wed, 18 Oct 2023 11:48:47 -0400 Subject: [PATCH 177/435] [Python] Fail WindowInto when there are no inputs to it (#28946) --- sdks/python/apache_beam/runners/common.py | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/sdks/python/apache_beam/runners/common.py b/sdks/python/apache_beam/runners/common.py index ed0dc2d9a0c1..1cd0a3044663 100644 --- a/sdks/python/apache_beam/runners/common.py +++ b/sdks/python/apache_beam/runners/common.py @@ -1929,6 +1929,12 @@ def validate_transform(transform_id): raise ValueError( "Incompatible input coder %s and output coder %s for transform %s" % (transform_id, input_coder, output_coder)) + elif transform_proto.spec.urn == common_urns.primitives.ASSIGN_WINDOWS.urn: + if not transform_proto.inputs: + raise ValueError("Missing input for transform: %s" % transform_proto) + elif transform_proto.spec.urn == common_urns.primitives.PAR_DO.urn: + if not transform_proto.inputs: + raise ValueError("Missing input for transform: %s" % transform_proto) for t in transform_proto.subtransforms: validate_transform(t) From 130ad8b3ec6f1bab3d7c4e6bd5bb6111730a24f2 Mon Sep 17 00:00:00 2001 From: JayajP <jayajp@google.com> Date: Wed, 18 Oct 2023 09:02:11 -0700 Subject: [PATCH 178/435] Add MetricsContainer::getPerWorker{Counter|Histogram} (#28903) (#28923) * Add MetricsContainer::getPerWorker{Counter|Histogram} * Add comments to NoOp{Counter|Histogram}. --- .../worker/DataflowMetricsContainer.java | 13 +++++ .../worker/StreamingDataflowWorker.java | 5 ++ .../worker/StreamingStepMetricsContainer.java | 35 +++++++++++++ .../StreamingStepMetricsContainerTest.java | 22 ++++++++ .../beam/sdk/metrics/MetricsContainer.java | 16 ++++++ .../apache/beam/sdk/metrics/NoOpCounter.java | 51 +++++++++++++++++++ .../beam/sdk/metrics/NoOpHistogram.java | 42 +++++++++++++++ 7 files changed, 184 insertions(+) create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/NoOpCounter.java create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/NoOpHistogram.java diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowMetricsContainer.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowMetricsContainer.java index 81517129c8e9..c3e4fb1388b0 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowMetricsContainer.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowMetricsContainer.java @@ -22,9 +22,11 @@ import org.apache.beam.sdk.metrics.Counter; import org.apache.beam.sdk.metrics.Distribution; import org.apache.beam.sdk.metrics.Gauge; +import org.apache.beam.sdk.metrics.Histogram; import org.apache.beam.sdk.metrics.MetricName; import org.apache.beam.sdk.metrics.MetricsContainer; import org.apache.beam.sdk.metrics.MetricsEnvironment; +import org.apache.beam.sdk.util.HistogramData; /** * An implementation of {@link MetricsContainer} that reads the current execution state (tracked in @@ -56,6 +58,11 @@ public Counter getCounter(MetricName metricName) { return getCurrentContainer().getCounter(metricName); } + @Override + public Counter getPerWorkerCounter(MetricName metricName) { + return getCurrentContainer().getPerWorkerCounter(metricName); + } + @Override public Distribution getDistribution(MetricName metricName) { return getCurrentContainer().getDistribution(metricName); @@ -65,4 +72,10 @@ public Distribution getDistribution(MetricName metricName) { public Gauge getGauge(MetricName metricName) { return getCurrentContainer().getGauge(metricName); } + + @Override + public Histogram getPerWorkerHistogram( + MetricName metricName, HistogramData.BucketType bucketType) { + return getCurrentContainer().getPerWorkerHistogram(metricName, bucketType); + } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java index 32b272c67280..4c1693d61387 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java @@ -465,6 +465,11 @@ public static void main(String[] args) throws Exception { // metrics. MetricsEnvironment.setProcessWideContainer(new MetricsLogger(null)); + // When enabled, the Pipeline will record Per-Worker metrics that will be piped to WMW. + StreamingStepMetricsContainer.setEnablePerWorkerMetrics( + options.isEnableStreamingEngine() + && DataflowRunner.hasExperiment(options, "enable_per_worker_metrics")); + JvmInitializers.runBeforeProcessing(options); worker.startStatusPages(); worker.start(); diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingStepMetricsContainer.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingStepMetricsContainer.java index 8c5b9c2f2b66..875a2d649ece 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingStepMetricsContainer.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingStepMetricsContainer.java @@ -24,13 +24,17 @@ import javax.annotation.Nonnull; import org.apache.beam.runners.core.metrics.DistributionData; import org.apache.beam.runners.core.metrics.GaugeCell; +import org.apache.beam.runners.core.metrics.HistogramCell; import org.apache.beam.runners.core.metrics.MetricsMap; import org.apache.beam.sdk.metrics.Counter; import org.apache.beam.sdk.metrics.Distribution; import org.apache.beam.sdk.metrics.Gauge; +import org.apache.beam.sdk.metrics.Histogram; import org.apache.beam.sdk.metrics.MetricKey; import org.apache.beam.sdk.metrics.MetricName; import org.apache.beam.sdk.metrics.MetricsContainer; +import org.apache.beam.sdk.util.HistogramData; +import org.apache.beam.sdk.values.KV; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Function; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Predicates; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.FluentIterable; @@ -47,14 +51,22 @@ public class StreamingStepMetricsContainer implements MetricsContainer { private final String stepName; + private static Boolean enablePerWorkerMetrics; + private MetricsMap<MetricName, DeltaCounterCell> counters = new MetricsMap<>(DeltaCounterCell::new); + private MetricsMap<MetricName, DeltaCounterCell> perWorkerCounters = + new MetricsMap<>(DeltaCounterCell::new); + private MetricsMap<MetricName, GaugeCell> gauges = new MetricsMap<>(GaugeCell::new); private MetricsMap<MetricName, DeltaDistributionCell> distributions = new MetricsMap<>(DeltaDistributionCell::new); + private MetricsMap<KV<MetricName, HistogramData.BucketType>, HistogramCell> perWorkerHistograms = + new MetricsMap<>(HistogramCell::new); + private StreamingStepMetricsContainer(String stepName) { this.stepName = stepName; } @@ -73,6 +85,15 @@ public Counter getCounter(MetricName metricName) { return counters.get(metricName); } + @Override + public Counter getPerWorkerCounter(MetricName metricName) { + if (enablePerWorkerMetrics) { + return perWorkerCounters.get(metricName); + } else { + return MetricsContainer.super.getPerWorkerCounter(metricName); + } + } + @Override public Distribution getDistribution(MetricName metricName) { return distributions.get(metricName); @@ -83,6 +104,16 @@ public Gauge getGauge(MetricName metricName) { return gauges.get(metricName); } + @Override + public Histogram getPerWorkerHistogram( + MetricName metricName, HistogramData.BucketType bucketType) { + if (enablePerWorkerMetrics) { + return perWorkerHistograms.get(KV.of(metricName, bucketType)); + } else { + return MetricsContainer.super.getPerWorkerHistogram(metricName, bucketType); + } + } + public Iterable<CounterUpdate> extractUpdates() { return counterUpdates().append(distributionUpdates()); } @@ -142,4 +173,8 @@ public static Iterable<CounterUpdate> extractMetricUpdates( .getContainers() .transformAndConcat(StreamingStepMetricsContainer::extractUpdates); } + + public static void setEnablePerWorkerMetrics(Boolean enablePerWorkerMetrics) { + StreamingStepMetricsContainer.enablePerWorkerMetrics = enablePerWorkerMetrics; + } } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingStepMetricsContainerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingStepMetricsContainerTest.java index 1a4c43905d20..9e6d45a2351b 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingStepMetricsContainerTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingStepMetricsContainerTest.java @@ -20,6 +20,7 @@ import static org.apache.beam.runners.dataflow.worker.counters.DataflowCounterUpdateExtractor.longToSplitInt; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.not; import static org.hamcrest.Matchers.sameInstance; @@ -33,6 +34,9 @@ import org.apache.beam.sdk.metrics.Distribution; import org.apache.beam.sdk.metrics.MetricName; import org.apache.beam.sdk.metrics.MetricsContainer; +import org.apache.beam.sdk.metrics.NoOpCounter; +import org.apache.beam.sdk.metrics.NoOpHistogram; +import org.apache.beam.sdk.util.HistogramData; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -178,4 +182,22 @@ public void testDistributionUpdateExtraction() { .setMin(longToSplitInt(3)) .setSum(longToSplitInt(3))))); } + + @Test + public void testPerWorkerMetrics() { + StreamingStepMetricsContainer.setEnablePerWorkerMetrics(false); + MetricsContainer metricsContainer = registry.getContainer("test_step"); + assertThat( + metricsContainer.getPerWorkerCounter(name1), sameInstance(NoOpCounter.getInstance())); + HistogramData.BucketType testBucket = HistogramData.LinearBuckets.of(1, 1, 1); + assertThat( + metricsContainer.getPerWorkerHistogram(name1, testBucket), + sameInstance(NoOpHistogram.getInstance())); + + StreamingStepMetricsContainer.setEnablePerWorkerMetrics(true); + assertThat(metricsContainer.getPerWorkerCounter(name1), not(instanceOf(NoOpCounter.class))); + assertThat( + metricsContainer.getPerWorkerHistogram(name1, testBucket), + not(instanceOf(NoOpHistogram.class))); + } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricsContainer.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricsContainer.java index e93f8677b814..f48b9195c37c 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricsContainer.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricsContainer.java @@ -33,6 +33,14 @@ public interface MetricsContainer extends Serializable { */ Counter getCounter(MetricName metricName); + /** + * Return the {@link Counter} that should be used for implementing the given per-worker {@code metricName) + * in this container. + */ + default Counter getPerWorkerCounter(MetricName metricName) { + return NoOpCounter.getInstance(); + } + /** * Return the {@link Distribution} that should be used for implementing the given {@code * metricName} in this container. @@ -52,6 +60,14 @@ public interface MetricsContainer extends Serializable { default Histogram getHistogram(MetricName metricName, HistogramData.BucketType bucketType) { throw new RuntimeException("Histogram metric is not supported yet."); } + /** + * Return the {@link Histogram} that should be used for implementing the given per-worker {@code + * metricName} in this container. + */ + default Histogram getPerWorkerHistogram( + MetricName metricName, HistogramData.BucketType bucketType) { + return NoOpHistogram.getInstance(); + } /** Return the cumulative values for any metrics in this container as MonitoringInfos. */ default Iterable<MetricsApi.MonitoringInfo> getMonitoringInfos() { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/NoOpCounter.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/NoOpCounter.java new file mode 100644 index 000000000000..ab4fa685f9c2 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/NoOpCounter.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.metrics; + +/** + * A no-op implementation of Counter. This class exists to provide a default if an implementation of + * MetricsContainer does not override a Counter getter. + */ +public class NoOpCounter implements Counter { + + private static final NoOpCounter singleton = new NoOpCounter(); + private static final MetricName name = MetricName.named(NoOpCounter.class, "singleton"); + + private NoOpCounter() {} + + @Override + public void inc() {} + + @Override + public void inc(long n) {} + + @Override + public void dec() {} + + @Override + public void dec(long n) {} + + @Override + public MetricName getName() { + return name; + } + + public static NoOpCounter getInstance() { + return singleton; + } +} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/NoOpHistogram.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/NoOpHistogram.java new file mode 100644 index 000000000000..a088223ffe2b --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/NoOpHistogram.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.metrics; + +/** + * A no-op implementation of Histogram. This class exists to provide a default if an implementation + * of MetricsContainer does not override a Histogram getter. + */ +public class NoOpHistogram implements Histogram { + + private static final NoOpHistogram singleton = new NoOpHistogram(); + private static final MetricName name = MetricName.named(NoOpHistogram.class, "singleton"); + + private NoOpHistogram() {} + + @Override + public void update(double value) {} + + @Override + public MetricName getName() { + return name; + } + + public static NoOpHistogram getInstance() { + return singleton; + } +} From 4f7763bf2dc262436d8333ea00f0c09c4935b2e3 Mon Sep 17 00:00:00 2001 From: Damon <damondouglas@users.noreply.github.com> Date: Wed, 18 Oct 2023 09:13:15 -0700 Subject: [PATCH 179/435] [RRIO] [Throttle] Stub Enqueue PTransform (#29053) --- .../io/requestresponse/ThrottleEnqueue.java | 61 +++++++++++++++++++ 1 file changed, 61 insertions(+) create mode 100644 sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/ThrottleEnqueue.java diff --git a/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/ThrottleEnqueue.java b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/ThrottleEnqueue.java new file mode 100644 index 000000000000..505ef86be48b --- /dev/null +++ b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/ThrottleEnqueue.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.io.requestresponse; + +import com.google.auto.value.AutoValue; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.TypeDescriptor; + +/** + * {@link ThrottleEnqueue} enqueues {@link RequestT} elements yielding an {@link ApiIOError} {@link + * PCollection} of any enqueue errors. + */ +class ThrottleEnqueue<RequestT> extends PTransform<PCollection<RequestT>, PCollection<ApiIOError>> { + + @SuppressWarnings({"unused"}) + private final Configuration<RequestT> configuration; + + private ThrottleEnqueue(Configuration<RequestT> configuration) { + this.configuration = configuration; + } + + /** Configuration details for {@link ThrottleEnqueue}. */ + @AutoValue + abstract static class Configuration<RequestT> { + + static <RequestT> Builder<RequestT> builder() { + return new AutoValue_ThrottleEnqueue_Configuration.Builder<>(); + } + + abstract Builder<RequestT> toBuilder(); + + @AutoValue.Builder + abstract static class Builder<RequestT> { + + abstract Configuration<RequestT> build(); + } + } + + @Override + public PCollection<ApiIOError> expand(PCollection<RequestT> input) { + // TODO(damondouglas): expand in a future PR. + return input.getPipeline().apply(Create.empty(TypeDescriptor.of(ApiIOError.class))); + } +} From 695dd4116efa2a47dc68ceac65803d115e77845a Mon Sep 17 00:00:00 2001 From: Vitaly Terentyev <vitaly.terentyev@akvelon.com> Date: Wed, 18 Oct 2023 20:57:09 +0400 Subject: [PATCH 180/435] Fix comment phrase and name (#29052) --- .github/workflows/README.md | 2 +- .github/workflows/beam_CancelStaleDataflowJobs.yml | 2 +- .github/workflows/beam_CleanUpGCPResources.yml | 2 +- .github/workflows/beam_CleanUpPrebuiltSDKImages.yml | 2 +- ...Tests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml | 6 +++--- ...s_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml | 6 +++--- ...eam_PerformanceTests_WordCountIT_PythonVersions.yml | 8 ++++---- .github/workflows/beam_PostCommit_Java.yml | 2 +- .../workflows/beam_PostCommit_Java_Avro_Versions.yml | 2 +- .../beam_PostCommit_Java_BigQueryEarlyRollout.yml | 2 +- .github/workflows/beam_PostCommit_Java_DataflowV1.yml | 2 +- .github/workflows/beam_PostCommit_Java_DataflowV2.yml | 2 +- .../beam_PostCommit_Java_Examples_Dataflow.yml | 2 +- .../beam_PostCommit_Java_Examples_Dataflow_ARM.yml | 6 +++--- .../beam_PostCommit_Java_Examples_Dataflow_Java.yml | 2 +- .../beam_PostCommit_Java_Examples_Dataflow_V2.yml | 2 +- .../beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml | 6 +++--- .../workflows/beam_PostCommit_Java_Examples_Direct.yml | 2 +- .../workflows/beam_PostCommit_Java_Examples_Flink.yml | 2 +- .../workflows/beam_PostCommit_Java_Examples_Spark.yml | 2 +- .../workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml | 2 +- .../beam_PostCommit_Java_Nexmark_Dataflow.yml | 2 +- .../beam_PostCommit_Java_Nexmark_Dataflow_V2.yml | 2 +- .../beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml | 10 +++++----- .../workflows/beam_PostCommit_Java_Nexmark_Direct.yml | 2 +- .../workflows/beam_PostCommit_Java_Nexmark_Flink.yml | 2 +- .../workflows/beam_PostCommit_Java_Nexmark_Spark.yml | 2 +- .../beam_PostCommit_Java_PVR_Flink_Streaming.yml | 2 +- .github/workflows/beam_PostCommit_Java_PVR_Samza.yml | 2 +- .../beam_PostCommit_Java_PVR_Spark3_Streaming.yml | 2 +- .../workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml | 2 +- .github/workflows/beam_PostCommit_Java_Sickbay.yml | 2 +- .../beam_PostCommit_Java_SingleStoreIO_IT.yml | 2 +- .../beam_PostCommit_Java_ValidatesRunner_Dataflow.yml | 2 +- ...mmit_Java_ValidatesRunner_Dataflow_JavaVersions.yml | 6 +++--- ...tCommit_Java_ValidatesRunner_Dataflow_Streaming.yml | 2 +- ...eam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml | 2 +- ...mmit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml | 2 +- .../beam_PostCommit_Java_ValidatesRunner_Direct.yml | 2 +- ...Commit_Java_ValidatesRunner_Direct_JavaVersions.yml | 6 +++--- ...am_PostCommit_Java_ValidatesRunner_Flink_Java11.yml | 2 +- ...am_PostCommit_Java_ValidatesRunner_Spark_Java11.yml | 2 +- .github/workflows/beam_PostCommit_Javadoc.yml | 2 +- .github/workflows/beam_PostCommit_Python.yml | 4 ++-- .github/workflows/beam_PostCommit_Python_Arm.yml | 4 ++-- .../beam_PostCommit_Python_Nexmark_Direct.yml | 2 +- .github/workflows/beam_PostCommit_SQL.yml | 2 +- .github/workflows/beam_PostCommit_Sickbay_Python.yml | 8 ++++---- .../beam_PostCommit_TransformService_Direct.yml | 2 +- .github/workflows/beam_PostCommit_XVR_Direct.yml | 2 +- .github/workflows/beam_PostCommit_XVR_Flink.yml | 2 +- .../beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml | 2 +- .../beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml | 2 +- .github/workflows/beam_PostCommit_XVR_Samza.yml | 2 +- .github/workflows/beam_PostCommit_XVR_Spark3.yml | 2 +- .github/workflows/beam_PreCommit_CommunityMetrics.yml | 4 ++-- .github/workflows/beam_PreCommit_GoPortable.yml | 2 +- .github/workflows/beam_PreCommit_GoPrism.yml | 2 +- .github/workflows/beam_PreCommit_ItFramework.yml | 4 ++-- .github/workflows/beam_PreCommit_Java.yml | 2 +- .../beam_PreCommit_Java_Examples_Dataflow.yml | 4 ++-- .../beam_PreCommit_Java_Examples_Dataflow_Java11.yml | 6 +++--- .../workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml | 2 +- .../workflows/beam_PreCommit_Java_Spark3_Versions.yml | 4 ++-- .github/workflows/beam_PreCommit_Kotlin_Examples.yml | 4 ++-- .github/workflows/beam_PreCommit_Portable_Python.yml | 6 +++--- .github/workflows/beam_PreCommit_Python.yml | 2 +- .github/workflows/beam_PreCommit_PythonDocker.yml | 2 +- .github/workflows/beam_PreCommit_PythonDocs.yml | 4 ++-- .github/workflows/beam_PreCommit_PythonFormatter.yml | 4 ++-- .github/workflows/beam_PreCommit_PythonLint.yml | 2 +- .github/workflows/beam_PreCommit_Python_Coverage.yml | 4 ++-- .github/workflows/beam_PreCommit_Python_Dataframes.yml | 2 +- .github/workflows/beam_PreCommit_Python_Examples.yml | 2 +- .../workflows/beam_PreCommit_Python_Integration.yml | 2 +- .github/workflows/beam_PreCommit_Python_Runners.yml | 2 +- .github/workflows/beam_PreCommit_Python_Transforms.yml | 2 +- .github/workflows/beam_PreCommit_RAT.yml | 2 +- .github/workflows/beam_PreCommit_Spotless.yml | 2 +- .github/workflows/beam_PreCommit_Typescript.yml | 4 ++-- .github/workflows/beam_PreCommit_Website.yml | 2 +- .github/workflows/beam_PreCommit_Website_Stage_GCS.yml | 2 +- .github/workflows/beam_PreCommit_Whitespace.yml | 2 +- .../beam_Python_ValidatesContainer_Dataflow_ARM.yml | 2 +- .github/workflows/beam_Release_NightlySnapshot.yml | 7 +++---- .../workflows/beam_Release_Python_NightlySnapshot.yml | 6 +++--- 86 files changed, 127 insertions(+), 128 deletions(-) diff --git a/.github/workflows/README.md b/.github/workflows/README.md index daf873a11099..3bd759106207 100644 --- a/.github/workflows/README.md +++ b/.github/workflows/README.md @@ -85,7 +85,7 @@ jobs: - name: Setup repository uses: ./.github/actions/setup-action with: - comment_phrase: ${{matrix.job_phrase}} + comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }} ${{ matrix.python_version }}) ``` diff --git a/.github/workflows/beam_CancelStaleDataflowJobs.yml b/.github/workflows/beam_CancelStaleDataflowJobs.yml index 9d1eb8fcb943..63e780c2fef5 100644 --- a/.github/workflows/beam_CancelStaleDataflowJobs.yml +++ b/.github/workflows/beam_CancelStaleDataflowJobs.yml @@ -50,7 +50,7 @@ permissions: jobs: beam_CancelStaleDataflowJobs: - name: ${{matrix.job_name}} (${{matrix.job_phrase}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 120 strategy: diff --git a/.github/workflows/beam_CleanUpGCPResources.yml b/.github/workflows/beam_CleanUpGCPResources.yml index 5b1272fe219f..9aa92f0003c4 100644 --- a/.github/workflows/beam_CleanUpGCPResources.yml +++ b/.github/workflows/beam_CleanUpGCPResources.yml @@ -50,7 +50,7 @@ permissions: jobs: beam_CleanUpGCPResources: - name: ${{matrix.job_name}} (${{matrix.job_phrase}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 strategy: diff --git a/.github/workflows/beam_CleanUpPrebuiltSDKImages.yml b/.github/workflows/beam_CleanUpPrebuiltSDKImages.yml index 389f6cd73385..345624f063bb 100644 --- a/.github/workflows/beam_CleanUpPrebuiltSDKImages.yml +++ b/.github/workflows/beam_CleanUpPrebuiltSDKImages.yml @@ -50,7 +50,7 @@ permissions: jobs: beam_CleanUpPrebuiltSDKImages: - name: ${{matrix.job_name}} (${{matrix.job_phrase}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 strategy: diff --git a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml index 8c990cbb231a..2960fafe6b32 100644 --- a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml +++ b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml @@ -57,7 +57,7 @@ jobs: contains(github.event.comment.body, 'CoGBK Dataflow V2 Batch')) runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 720 - name: ${{matrix.job_name}} (${{matrix.job_phrase_1}} ${{matrix.java_version}} ${{matrix.job_phrase_2}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase_1 }} ${{ matrix.java_version }} ${{ matrix.job_phrase_2 }}) strategy: fail-fast: false matrix: @@ -70,9 +70,9 @@ jobs: - name: Setup repository uses: ./.github/actions/setup-action with: - comment_phrase: ${{ matrix.job_phrase_1 }} ${{matrix.java_version}} ${{ matrix.job_phrase_2 }} + comment_phrase: ${{ matrix.job_phrase_1 }} ${{ matrix.java_version }} ${{ matrix.job_phrase_2 }} github_token: ${{ secrets.GITHUB_TOKEN }} - github_job: ${{ matrix.job_name }} ${{ matrix.job_phrase_1 }} ${{matrix.java_version}} ${{ matrix.job_phrase_2 }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase_1 }} ${{ matrix.java_version }} ${{ matrix.job_phrase_2 }}) - name: Setup environment uses: ./.github/actions/setup-environment-action with: diff --git a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml index a6e28740b3c4..113927dba340 100644 --- a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml +++ b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml @@ -57,7 +57,7 @@ jobs: contains(github.event.comment.body, 'CoGBK Dataflow V2 Streaming')) runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 240 - name: ${{matrix.job_name}} (${{matrix.job_phrase_1}} ${{matrix.java_version}} ${{matrix.job_phrase_2}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase_1 }} ${{ matrix.java_version }} ${{ matrix.job_phrase_2 }}) strategy: fail-fast: false matrix: @@ -70,9 +70,9 @@ jobs: - name: Setup repository uses: ./.github/actions/setup-action with: - comment_phrase: ${{ matrix.job_phrase_1 }} ${{matrix.java_version}} ${{ matrix.job_phrase_2 }} + comment_phrase: ${{ matrix.job_phrase_1 }} ${{ matrix.java_version }} ${{ matrix.job_phrase_2 }} github_token: ${{ secrets.GITHUB_TOKEN }} - github_job: ${{ matrix.job_name }} ${{ matrix.job_phrase_1 }} ${{matrix.java_version}} ${{ matrix.job_phrase_2 }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase_1 }} ${{ matrix.java_version }} ${{ matrix.job_phrase_2 }}) - name: Setup environment uses: ./.github/actions/setup-environment-action with: diff --git a/.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml b/.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml index ad9d8f713398..9e901cc81926 100644 --- a/.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml +++ b/.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml @@ -56,7 +56,7 @@ jobs: endswith(github.event.comment.body, 'WordCountIT Performance Test')) runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 - name: ${{ matrix.job_name }} (${{ matrix.job_phrase_1 }}${{matrix.python_version}} ${{ matrix.job_phrase_2 }}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase_1 }} ${{ matrix.python_version }} ${{ matrix.job_phrase_2 }}) strategy: fail-fast: false matrix: @@ -69,13 +69,13 @@ jobs: - name: Setup repository uses: ./.github/actions/setup-action with: - comment_phrase: ${{ matrix.job_phrase_1 }}${{matrix.python_version}} ${{ matrix.job_phrase_2 }} + comment_phrase: ${{ matrix.job_phrase_1 }} ${{ matrix.python_version }} ${{ matrix.job_phrase_2 }} github_token: ${{ secrets.GITHUB_TOKEN }} - github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase_1 }}${{matrix.python_version}} ${{ matrix.job_phrase_2 }}) + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase_1 }} ${{ matrix.python_version }} ${{ matrix.job_phrase_2 }}) - name: Setup environment uses: ./.github/actions/setup-environment-action with: - python-version: ${{matrix.python_version}} + python-version: ${{ matrix.python_version }} - name: Set PY_VER_CLEAN id: set_py_ver_clean run: | diff --git a/.github/workflows/beam_PostCommit_Java.yml b/.github/workflows/beam_PostCommit_Java.yml index 601a438d5a4b..fb029e8402fa 100644 --- a/.github/workflows/beam_PostCommit_Java.yml +++ b/.github/workflows/beam_PostCommit_Java.yml @@ -50,7 +50,7 @@ env: jobs: beam_PostCommit_Java: - name: ${{matrix.job_name}} (${{matrix.job_phrase}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 240 strategy: diff --git a/.github/workflows/beam_PostCommit_Java_Avro_Versions.yml b/.github/workflows/beam_PostCommit_Java_Avro_Versions.yml index 69313a568f32..ef451b85f728 100644 --- a/.github/workflows/beam_PostCommit_Java_Avro_Versions.yml +++ b/.github/workflows/beam_PostCommit_Java_Avro_Versions.yml @@ -50,7 +50,7 @@ env: jobs: beam_PostCommit_Java_Avro_Versions: - name: ${{matrix.job_name}} (${{matrix.job_phrase}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 240 strategy: diff --git a/.github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml b/.github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml index 42330b5fc760..b281e550e562 100644 --- a/.github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml +++ b/.github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml @@ -50,7 +50,7 @@ env: jobs: beam_PostCommit_Java_BigQueryEarlyRollout: - name: ${{matrix.job_name}} (${{matrix.job_phrase}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 strategy: diff --git a/.github/workflows/beam_PostCommit_Java_DataflowV1.yml b/.github/workflows/beam_PostCommit_Java_DataflowV1.yml index e9dcdcb874eb..7515da1cad7b 100644 --- a/.github/workflows/beam_PostCommit_Java_DataflowV1.yml +++ b/.github/workflows/beam_PostCommit_Java_DataflowV1.yml @@ -50,7 +50,7 @@ env: jobs: beam_PostCommit_Java_DataflowV1: - name: ${{matrix.job_name}} (${{matrix.job_phrase}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 240 strategy: diff --git a/.github/workflows/beam_PostCommit_Java_DataflowV2.yml b/.github/workflows/beam_PostCommit_Java_DataflowV2.yml index 70423a06c56f..2360feeb7d3f 100644 --- a/.github/workflows/beam_PostCommit_Java_DataflowV2.yml +++ b/.github/workflows/beam_PostCommit_Java_DataflowV2.yml @@ -50,7 +50,7 @@ env: jobs: beam_PostCommit_Java_DataflowV2: - name: ${{matrix.job_name}} (${{matrix.job_phrase}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 240 strategy: diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow.yml index a302e631a40c..a1e2a0089029 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow.yml @@ -52,7 +52,7 @@ env: jobs: beam_PostCommit_Java_Examples_Dataflow_V2: - name: ${{matrix.job_name}} (${{matrix.job_phrase}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 180 strategy: diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml index 2e2adeea7cb0..c4d414fc32e2 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml @@ -63,7 +63,7 @@ permissions: jobs: beam_PostCommit_Java_Examples__Dataflow_ARM: - name: ${{matrix.job_name}} (${{matrix.job_phrase}} ${{matrix.java_version}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }} ${{ matrix.java_version }}) runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 120 strategy: @@ -82,9 +82,9 @@ jobs: - name: Setup repository uses: ./.github/actions/setup-action with: - comment_phrase: ${{ matrix.job_phrase }} ${{matrix.java_version}} + comment_phrase: ${{ matrix.job_phrase }} ${{ matrix.java_version }} github_token: ${{ secrets.GITHUB_TOKEN }} - github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }} ${{matrix.java_version}}) + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }} ${{ matrix.java_version }}) - name: Setup environment uses: ./.github/actions/setup-environment-action with: diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml index 2f798f71a624..f9a87b26ab83 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml @@ -68,7 +68,7 @@ jobs: - name: Setup repository uses: ./.github/actions/setup-action with: - comment_phrase: ${{ matrix.job_phrase }} ${{matrix.java_version}} + comment_phrase: ${{ matrix.job_phrase }} ${{ matrix.java_version }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }} ${{ matrix.java_version }}) - name: Set up Java${{ matrix.java_version }} diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml index ab72872d322f..28f6ae4d2053 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml @@ -50,7 +50,7 @@ env: jobs: beam_PostCommit_Java_Examples_Dataflow_V2: - name: ${{matrix.job_name}} (${{matrix.job_phrase}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 180 strategy: diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml index 151b09469c5a..16f48df03292 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml @@ -50,7 +50,7 @@ env: jobs: beam_PostCommit_Java_Examples_Dataflow_V2_Java: - name: ${{matrix.job_name}} (${{matrix.job_phrase_1}}${{matrix.job_phrase_2}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase_1 }} ${{ matrix.java_version }} ${{ matrix.job_phrase_2 }}) runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 180 strategy: @@ -70,9 +70,9 @@ jobs: - name: Setup repository uses: ./.github/actions/setup-action with: - comment_phrase: ${{ matrix.job_phrase_1 }} ${{matrix.java_version}} ${{ matrix.job_phrase_2 }} + comment_phrase: ${{ matrix.job_phrase_1 }} ${{ matrix.java_version }} ${{ matrix.job_phrase_2 }} github_token: ${{ secrets.GITHUB_TOKEN }} - github_job: ${{ matrix.job_name }} ${{ matrix.job_phrase_1 }} ${{matrix.java_version}} ${{ matrix.job_phrase_2 }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase_1 }} ${{ matrix.java_version }} ${{ matrix.job_phrase_2 }}) - name: Set up Java${{ matrix.java_version }} uses: actions/setup-java@v3.8.0 with: diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Direct.yml b/.github/workflows/beam_PostCommit_Java_Examples_Direct.yml index f8c5835f5aca..329093731273 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Direct.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Direct.yml @@ -50,7 +50,7 @@ env: jobs: beam_PostCommit_Java_Examples_Direct: - name: ${{matrix.job_name}} (${{matrix.job_phrase}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 120 strategy: diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Flink.yml b/.github/workflows/beam_PostCommit_Java_Examples_Flink.yml index 908e7d4286f3..db368031d9f0 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Flink.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Flink.yml @@ -50,7 +50,7 @@ env: jobs: beam_PostCommit_Java_Examples_Flink: - name: ${{matrix.job_name}} (${{matrix.job_phrase}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 120 strategy: diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Spark.yml b/.github/workflows/beam_PostCommit_Java_Examples_Spark.yml index 0449186e1ed0..f7d12a9860da 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Spark.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Spark.yml @@ -50,7 +50,7 @@ env: jobs: beam_PostCommit_Java_Examples_Spark: - name: ${{matrix.job_name}} (${{matrix.job_phrase}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 120 strategy: diff --git a/.github/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml b/.github/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml index ba525e3dd424..8d834a247e6b 100644 --- a/.github/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml +++ b/.github/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml @@ -50,7 +50,7 @@ permissions: jobs: beam_PostCommit_Java_InfluxDbIO_IT: - name: ${{matrix.job_name}} (${{matrix.job_phrase}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 strategy: diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml index db15117d3cba..99e5fb8e677b 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml @@ -73,7 +73,7 @@ env: jobs: beam_PostCommit_Java_Nexmark_Dataflow: - name: ${{matrix.job_name}} (${{matrix.job_phrase}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 240 strategy: diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml index 4865e1f3cb6e..aba92fd8bb07 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml @@ -74,7 +74,7 @@ env: jobs: beam_PostCommit_Java_Nexmark_Dataflow_V2: - name: ${{matrix.job_name}} (${{matrix.job_phrase}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 240 strategy: diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml index 1c58bf2510a2..72e1f736e917 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml @@ -73,7 +73,7 @@ env: jobs: beam_PostCommit_Java_Nexmark_Dataflow_V2_Java: - name: ${{matrix.job_name}} (${{matrix.job_phrase_1}} ${{matrix.java_version}} ${{matrix.job_phrase_2}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase_1 }} ${{ matrix.java_version }} ${{ matrix.job_phrase_2 }}) runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 240 strategy: @@ -94,15 +94,15 @@ jobs: - name: Setup repository uses: ./.github/actions/setup-action with: - comment_phrase: ${{ matrix.job_phrase_1 }} ${{matrix.java_version}} ${{ matrix.job_phrase_2 }} + comment_phrase: ${{ matrix.job_phrase_1 }} ${{ matrix.java_version }} ${{ matrix.job_phrase_2 }} github_token: ${{ secrets.GITHUB_TOKEN }} - github_job: ${{ matrix.job_name }} ${{ matrix.job_phrase_1 }} ${{matrix.java_version}} ${{ matrix.job_phrase_2 }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase_1 }} ${{ matrix.java_version }} ${{ matrix.job_phrase_2 }}) - name: Set up Java${{ matrix.java_version }} uses: actions/setup-java@v3.8.0 with: distribution: 'temurin' java-version: ${{ matrix.java_version }} - - name: run PostCommit Java ${{matrix.java_version}} Nexmark Dataflow V2 (streaming = ${{ matrix.streaming }}) script + - name: run PostCommit Java ${{ matrix.java_version }} Nexmark Dataflow V2 (streaming = ${{ matrix.streaming }}) script uses: ./.github/actions/gradle-command-self-hosted-action with: gradle-command: :sdks:java:testing:nexmark:run @@ -111,4 +111,4 @@ jobs: -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ -Pnexmark.runner.version=V2 \ -Pnexmark.runner=:runners:google-cloud-dataflow-java \ - '${{ env.GRADLE_COMMAND_ARGUMENTS }}--influxTags={"runnerVersion":"V2","javaVersion":"${{matrix.java_version}}"}--streaming=${{ matrix.streaming }}' \ No newline at end of file + '${{ env.GRADLE_COMMAND_ARGUMENTS }}--influxTags={"runnerVersion":"V2","javaVersion":"${{ matrix.java_version }}"}--streaming=${{ matrix.streaming }}' \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Direct.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Direct.yml index 3f230b0b78b8..c2e7433f0478 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Direct.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Direct.yml @@ -68,7 +68,7 @@ env: jobs: beam_PostCommit_Java_Nexmark_Direct: - name: ${{matrix.job_name}} (${{matrix.job_phrase}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 240 strategy: diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Flink.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Flink.yml index a525a65e1ad2..e80c5a5b2802 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Flink.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Flink.yml @@ -67,7 +67,7 @@ env: jobs: beam_PostCommit_Java_Nexmark_Flink: - name: ${{matrix.job_name}} (${{matrix.job_phrase}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 240 strategy: diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Spark.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Spark.yml index 6427a3ca33d9..0ccd751ac980 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Spark.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Spark.yml @@ -67,7 +67,7 @@ env: jobs: beam_PostCommit_Java_Nexmark_Spark: - name: ${{matrix.job_name}} (${{matrix.job_phrase}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 240 strategy: diff --git a/.github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml b/.github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml index 050a6fd7289f..209fb2acf23a 100644 --- a/.github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml +++ b/.github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml @@ -50,7 +50,7 @@ env: jobs: beam_PostCommit_Java_PVR_Flink_Streaming: - name: ${{matrix.job_name}} (${{matrix.job_phrase}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 120 strategy: diff --git a/.github/workflows/beam_PostCommit_Java_PVR_Samza.yml b/.github/workflows/beam_PostCommit_Java_PVR_Samza.yml index 62ff353157b8..f6f0140a6ec7 100644 --- a/.github/workflows/beam_PostCommit_Java_PVR_Samza.yml +++ b/.github/workflows/beam_PostCommit_Java_PVR_Samza.yml @@ -50,7 +50,7 @@ env: jobs: beam_PostCommit_Java_PVR_Samza: - name: ${{matrix.job_name}} (${{matrix.job_phrase}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 120 strategy: diff --git a/.github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml b/.github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml index 1114ae06ec32..4fff363d989a 100644 --- a/.github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml +++ b/.github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml @@ -50,7 +50,7 @@ env: jobs: beam_PostCommit_Java_PVR_Spark3_Streaming: - name: ${{matrix.job_name}} (${{matrix.job_phrase}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 120 strategy: diff --git a/.github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml b/.github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml index 2d506eb1d42d..d4db6f01518e 100644 --- a/.github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml +++ b/.github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml @@ -50,7 +50,7 @@ env: jobs: beam_PostCommit_Java_PVR_Spark_Batch: - name: ${{matrix.job_name}} (${{matrix.job_phrase}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 240 strategy: diff --git a/.github/workflows/beam_PostCommit_Java_Sickbay.yml b/.github/workflows/beam_PostCommit_Java_Sickbay.yml index 037bd610ab0a..3478e8902e08 100644 --- a/.github/workflows/beam_PostCommit_Java_Sickbay.yml +++ b/.github/workflows/beam_PostCommit_Java_Sickbay.yml @@ -50,7 +50,7 @@ env: jobs: beam_PostCommit_Java_Sickbay: - name: ${{matrix.job_name}} (${{matrix.job_phrase}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 120 strategy: diff --git a/.github/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml b/.github/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml index e79a525c3157..43b7d0bb1d81 100644 --- a/.github/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml +++ b/.github/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml @@ -50,7 +50,7 @@ env: jobs: beam_PostCommit_Java_SingleStoreIO_IT: - name: ${{matrix.job_name}} (${{matrix.job_phrase}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 strategy: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml index 1ddca5169180..8d8609ad96d6 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml @@ -50,7 +50,7 @@ env: jobs: beam_PostCommit_Java_ValidatesRunner_Dataflow: - name: ${{matrix.job_name}} (${{matrix.job_phrase}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 480 strategy: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml index b50c02a0e31b..5b151e535f97 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml @@ -50,7 +50,7 @@ env: jobs: beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions: - name: ${{matrix.job_name}} (${{matrix.job_phrase}} ${{matrix.java_version}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }} ${{ matrix.java_version }}) runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 480 strategy: @@ -68,9 +68,9 @@ jobs: - name: Setup repository uses: ./.github/actions/setup-action with: - comment_phrase: ${{ matrix.job_phrase }} ${{matrix.java_version}} + comment_phrase: ${{ matrix.job_phrase }} ${{ matrix.java_version }} github_token: ${{ secrets.GITHUB_TOKEN }} - github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) ${{matrix.java_version}} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) ${{ matrix.java_version }} - name: Set up Java${{ matrix.java_version }} uses: actions/setup-java@v3.8.0 with: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml index 0224bfd9dab5..2a61431c81af 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml @@ -50,7 +50,7 @@ env: jobs: beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming: - name: ${{matrix.job_name}} (${{matrix.job_phrase}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 720 strategy: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml index c15db0de5baf..4d3664618d9e 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml @@ -50,7 +50,7 @@ env: jobs: beam_PostCommit_Java_ValidatesRunner_Dataflow_V2: - name: ${{matrix.job_name}} (${{matrix.job_phrase}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 390 strategy: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml index 768dcdf4fb57..10725d6a72d9 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml @@ -50,7 +50,7 @@ env: jobs: beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming: - name: ${{matrix.job_name}} (${{matrix.job_phrase}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 510 strategy: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml index 9cb613f2a1df..5bf14340e0d1 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml @@ -50,7 +50,7 @@ env: jobs: beam_PostCommit_Java_ValidatesRunner_Direct: - name: ${{matrix.job_name}} (${{matrix.job_phrase}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 180 strategy: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml index f2cbebf1407a..e71c47266eac 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml @@ -50,7 +50,7 @@ env: jobs: beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions: - name: ${{matrix.job_name}} (${{matrix.job_phrase}} ${{matrix.java_version}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }} ${{ matrix.java_version }}) runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 480 strategy: @@ -68,9 +68,9 @@ jobs: - name: Setup repository uses: ./.github/actions/setup-action with: - comment_phrase: ${{ matrix.job_phrase }} ${{matrix.java_version}} + comment_phrase: ${{ matrix.job_phrase }} ${{ matrix.java_version }} github_token: ${{ secrets.GITHUB_TOKEN }} - github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) ${{matrix.java_version}} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) ${{ matrix.java_version }} - name: Set up Java${{ matrix.java_version }} uses: actions/setup-java@v3.8.0 with: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml index af9b06f20d45..eaeef09bb781 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml @@ -50,7 +50,7 @@ env: jobs: beam_PostCommit_Java_ValidatesRunner_Flink_Java11: - name: ${{matrix.job_name}} (${{matrix.job_phrase}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 270 strategy: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml index 44d0cd11d784..3456f24594a6 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml @@ -50,7 +50,7 @@ env: jobs: beam_PostCommit_Java_ValidatesRunner_Spark_Java11: - name: ${{matrix.job_name}} (${{matrix.job_phrase}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 270 strategy: diff --git a/.github/workflows/beam_PostCommit_Javadoc.yml b/.github/workflows/beam_PostCommit_Javadoc.yml index 4a38b9d22a3b..deab0f879f52 100644 --- a/.github/workflows/beam_PostCommit_Javadoc.yml +++ b/.github/workflows/beam_PostCommit_Javadoc.yml @@ -50,7 +50,7 @@ env: jobs: beam_PostCommit_Javadoc: - name: ${{matrix.job_name}} (${{matrix.job_phrase}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 240 strategy: diff --git a/.github/workflows/beam_PostCommit_Python.yml b/.github/workflows/beam_PostCommit_Python.yml index 5444fe204453..e6e5ebd93d30 100644 --- a/.github/workflows/beam_PostCommit_Python.yml +++ b/.github/workflows/beam_PostCommit_Python.yml @@ -50,7 +50,7 @@ env: jobs: beam_PostCommit_Python: - name: ${{matrix.job_name}} (${{matrix.job_phrase}} ${{matrix.python_version}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }} ${{ matrix.python_version }}) runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 240 strategy: @@ -74,7 +74,7 @@ jobs: - name: Install Python uses: actions/setup-python@v4 with: - python-version: ${{matrix.python_version}} + python-version: ${{ matrix.python_version }} - name: Install docker compose run: | sudo curl -L https://github.com/docker/compose/releases/download/1.22.0/docker-compose-$(uname -s)-$(uname -m) -o /usr/local/bin/docker-compose diff --git a/.github/workflows/beam_PostCommit_Python_Arm.yml b/.github/workflows/beam_PostCommit_Python_Arm.yml index 50e70e85edc0..f3d2b2463ef2 100644 --- a/.github/workflows/beam_PostCommit_Python_Arm.yml +++ b/.github/workflows/beam_PostCommit_Python_Arm.yml @@ -52,7 +52,7 @@ env: jobs: beam_PostCommit_Python_Arm: - name: ${{matrix.job_name}} (${{matrix.job_phrase}} ${{matrix.python_version}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }} ${{ matrix.python_version }}) runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 240 strategy: @@ -76,7 +76,7 @@ jobs: - name: Install Python uses: actions/setup-python@v4 with: - python-version: ${{matrix.python_version}} + python-version: ${{ matrix.python_version }} - name: Install docker compose run: | sudo curl -L https://github.com/docker/compose/releases/download/1.22.0/docker-compose-$(uname -s)-$(uname -m) -o /usr/local/bin/docker-compose diff --git a/.github/workflows/beam_PostCommit_Python_Nexmark_Direct.yml b/.github/workflows/beam_PostCommit_Python_Nexmark_Direct.yml index b04a392c6770..1fbe9a47296a 100644 --- a/.github/workflows/beam_PostCommit_Python_Nexmark_Direct.yml +++ b/.github/workflows/beam_PostCommit_Python_Nexmark_Direct.yml @@ -87,7 +87,7 @@ env: jobs: beam_PostCommit_Python_Nexmark_Direct: - name: ${{matrix.job_name}} (${{matrix.job_phrase}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 240 strategy: diff --git a/.github/workflows/beam_PostCommit_SQL.yml b/.github/workflows/beam_PostCommit_SQL.yml index 42b1a69fcc14..776f7ce1210b 100644 --- a/.github/workflows/beam_PostCommit_SQL.yml +++ b/.github/workflows/beam_PostCommit_SQL.yml @@ -50,7 +50,7 @@ env: jobs: beam_PostCommit_SQL: - name: ${{matrix.job_name}} (${{matrix.job_phrase}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 240 strategy: diff --git a/.github/workflows/beam_PostCommit_Sickbay_Python.yml b/.github/workflows/beam_PostCommit_Sickbay_Python.yml index c8fa9759c65d..99effb751357 100644 --- a/.github/workflows/beam_PostCommit_Sickbay_Python.yml +++ b/.github/workflows/beam_PostCommit_Sickbay_Python.yml @@ -50,7 +50,7 @@ env: jobs: beam_PostCommit_Sickbay_Python: - name: ${{matrix.job_name}} (${{matrix.job_phrase_1}} ${{matrix.python_version}} ${{matrix.job_phrase_2}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase_1 }} ${{ matrix.python_version }} ${{ matrix.job_phrase_2 }}) runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 180 strategy: @@ -70,13 +70,13 @@ jobs: - name: Setup repository uses: ./.github/actions/setup-action with: - comment_phrase: ${{ matrix.job_phrase_1 }} ${{matrix.python_version}} ${{ matrix.job_phrase_2 }} + comment_phrase: ${{ matrix.job_phrase_1 }} ${{ matrix.python_version }} ${{ matrix.job_phrase_2 }} github_token: ${{ secrets.GITHUB_TOKEN }} - github_job: ${{ matrix.job_name }} ${{ matrix.job_phrase_1 }} ${{matrix.python_version}} ${{ matrix.job_phrase_2 }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase_1 }} ${{ matrix.python_version }} ${{ matrix.job_phrase_2 }}) - name: Install Python uses: actions/setup-python@v4 with: - python-version: ${{matrix.python_version}} + python-version: ${{ matrix.python_version }} - name: Set PY_VER_CLEAN id: set_py_ver_clean run: | diff --git a/.github/workflows/beam_PostCommit_TransformService_Direct.yml b/.github/workflows/beam_PostCommit_TransformService_Direct.yml index 8be0ebf3986f..4cc1ccf99c22 100644 --- a/.github/workflows/beam_PostCommit_TransformService_Direct.yml +++ b/.github/workflows/beam_PostCommit_TransformService_Direct.yml @@ -54,7 +54,7 @@ jobs: github.event.comment.body == 'Run TransformService_Direct PostCommit' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 120 - name: ${{matrix.job_name}} (${{matrix.job_phrase}} ${{ matrix.python_version }}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }} ${{ matrix.python_version }}) strategy: matrix: job_name: ["beam_PostCommit_TransformService_Direct"] diff --git a/.github/workflows/beam_PostCommit_XVR_Direct.yml b/.github/workflows/beam_PostCommit_XVR_Direct.yml index dea36ed312e0..51e36053ea96 100644 --- a/.github/workflows/beam_PostCommit_XVR_Direct.yml +++ b/.github/workflows/beam_PostCommit_XVR_Direct.yml @@ -54,7 +54,7 @@ jobs: github.event.comment.body == 'Run XVR_Direct PostCommit' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 120 - name: ${{matrix.job_name}} (${{matrix.job_phrase}} ${{ matrix.python_version }}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }} ${{ matrix.python_version }}) strategy: matrix: job_name: ["beam_PostCommit_XVR_Direct"] diff --git a/.github/workflows/beam_PostCommit_XVR_Flink.yml b/.github/workflows/beam_PostCommit_XVR_Flink.yml index d076b7c341e9..9df8b1c8e776 100644 --- a/.github/workflows/beam_PostCommit_XVR_Flink.yml +++ b/.github/workflows/beam_PostCommit_XVR_Flink.yml @@ -55,7 +55,7 @@ jobs: github.event.comment.body == 'Run XVR_Flink PostCommit' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 - name: ${{matrix.job_name}} (${{matrix.job_phrase}} ${{ matrix.python_version }}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }} ${{ matrix.python_version }}) strategy: matrix: job_name: ["beam_PostCommit_XVR_Flink"] diff --git a/.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml b/.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml index 79731b9cd085..93e495c5eb15 100644 --- a/.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml @@ -54,7 +54,7 @@ jobs: github.event.comment.body == 'Run XVR_JavaUsingPython_Dataflow PostCommit' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 - name: ${{matrix.job_name}} (${{matrix.job_phrase}} ${{ matrix.python_version }}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }} ${{ matrix.python_version }}) strategy: matrix: job_name: ["beam_PostCommit_XVR_JavaUsingPython_Dataflow"] diff --git a/.github/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml b/.github/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml index c0d2c90d027a..9284c3698824 100644 --- a/.github/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml @@ -54,7 +54,7 @@ jobs: github.event.comment.body == 'Run XVR_PythonUsingJava_Dataflow PostCommit' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 - name: ${{matrix.job_name}} (${{matrix.job_phrase}} ${{ matrix.python_version }}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }} ${{ matrix.python_version }}) strategy: matrix: job_name: ["beam_PostCommit_XVR_PythonUsingJava_Dataflow"] diff --git a/.github/workflows/beam_PostCommit_XVR_Samza.yml b/.github/workflows/beam_PostCommit_XVR_Samza.yml index 681169c91d6b..f053f28ac190 100644 --- a/.github/workflows/beam_PostCommit_XVR_Samza.yml +++ b/.github/workflows/beam_PostCommit_XVR_Samza.yml @@ -54,7 +54,7 @@ jobs: github.event.comment.body == 'Run XVR_Samza PostCommit' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 - name: ${{matrix.job_name}} (${{matrix.job_phrase}} ${{ matrix.python_version }}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }} ${{ matrix.python_version }}) strategy: matrix: job_name: ["beam_PostCommit_XVR_Samza"] diff --git a/.github/workflows/beam_PostCommit_XVR_Spark3.yml b/.github/workflows/beam_PostCommit_XVR_Spark3.yml index 962ea0228784..3b6d5bdad551 100644 --- a/.github/workflows/beam_PostCommit_XVR_Spark3.yml +++ b/.github/workflows/beam_PostCommit_XVR_Spark3.yml @@ -54,7 +54,7 @@ jobs: github.event.comment.body == 'Run XVR_Spark3 PostCommit' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 - name: ${{matrix.job_name}} (${{matrix.job_phrase}} ${{ matrix.python_version }}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }} ${{ matrix.python_version }}) strategy: matrix: job_name: ["beam_PostCommit_XVR_Spark3"] diff --git a/.github/workflows/beam_PreCommit_CommunityMetrics.yml b/.github/workflows/beam_PreCommit_CommunityMetrics.yml index b3f100525bda..9341b1e8877f 100644 --- a/.github/workflows/beam_PreCommit_CommunityMetrics.yml +++ b/.github/workflows/beam_PreCommit_CommunityMetrics.yml @@ -57,7 +57,7 @@ env: jobs: beam_PreCommit_CommunityMetrics: - name: ${{matrix.job_name}} (${{matrix.job_phrase}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) timeout-minutes: 120 runs-on: [self-hosted, ubuntu-20.04, main] strategy: @@ -75,7 +75,7 @@ jobs: - name: Setup repository uses: ./.github/actions/setup-action with: - comment_phrase: ${{matrix.job_phrase}} + comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - name: Setup environment diff --git a/.github/workflows/beam_PreCommit_GoPortable.yml b/.github/workflows/beam_PreCommit_GoPortable.yml index 0ab59224aae6..3034285731a5 100644 --- a/.github/workflows/beam_PreCommit_GoPortable.yml +++ b/.github/workflows/beam_PreCommit_GoPortable.yml @@ -57,7 +57,7 @@ permissions: jobs: beam_PreCommit_GoPortable: - name: ${{matrix.job_name}} (${{ matrix.job_phrase }}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) runs-on: [self-hosted, ubuntu-20.04, main] strategy: matrix: diff --git a/.github/workflows/beam_PreCommit_GoPrism.yml b/.github/workflows/beam_PreCommit_GoPrism.yml index 1b0eecdcf8dc..9e403d13a2ef 100644 --- a/.github/workflows/beam_PreCommit_GoPrism.yml +++ b/.github/workflows/beam_PreCommit_GoPrism.yml @@ -57,7 +57,7 @@ permissions: jobs: beam_PreCommit_GoPrism: - name: ${{matrix.job_name}} (${{ matrix.job_phrase }}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) runs-on: [self-hosted, ubuntu-20.04, main] strategy: matrix: diff --git a/.github/workflows/beam_PreCommit_ItFramework.yml b/.github/workflows/beam_PreCommit_ItFramework.yml index b1000d9feea2..ef384def0d5f 100644 --- a/.github/workflows/beam_PreCommit_ItFramework.yml +++ b/.github/workflows/beam_PreCommit_ItFramework.yml @@ -61,7 +61,7 @@ permissions: statuses: read jobs: beam_PreCommit_ItFramework: - name: ${{matrix.job_name}} (${{matrix.job_phrase}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) runs-on: [self-hosted, ubuntu-20.04, main] strategy: matrix: @@ -78,7 +78,7 @@ jobs: - name: Setup repository uses: ./.github/actions/setup-action with: - comment_phrase: ${{matrix.job_phrase}} + comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - name: Setup environment diff --git a/.github/workflows/beam_PreCommit_Java.yml b/.github/workflows/beam_PreCommit_Java.yml index f302d12e9617..0b971462f334 100644 --- a/.github/workflows/beam_PreCommit_Java.yml +++ b/.github/workflows/beam_PreCommit_Java.yml @@ -148,7 +148,7 @@ env: jobs: beam_PreCommit_Java: - name: ${{matrix.job_name}} (${{matrix.job_phrase}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) timeout-minutes: 180 runs-on: [self-hosted, ubuntu-20.04, main] strategy: diff --git a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow.yml b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow.yml index 500b386f41d6..c68a85a0bc21 100644 --- a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow.yml +++ b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow.yml @@ -71,7 +71,7 @@ permissions: statuses: read jobs: beam_PreCommit_Java_Examples_Dataflow: - name: ${{matrix.job_name}} (${{matrix.job_phrase}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) timeout-minutes: 60 runs-on: [self-hosted, ubuntu-20.04, main] strategy: @@ -89,7 +89,7 @@ jobs: - name: Setup repository uses: ./.github/actions/setup-action with: - comment_phrase: ${{matrix.job_phrase}} + comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - name: Setup environment diff --git a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml index 65ed3b922439..032d13152eba 100644 --- a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml +++ b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml @@ -72,7 +72,7 @@ permissions: jobs: beam_PreCommit_Java_Examples_Dataflow_Java11: - name: ${{matrix.job_name}} (${{matrix.job_phrase}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) runs-on: [self-hosted, ubuntu-20.04, main] strategy: matrix: @@ -90,9 +90,9 @@ jobs: - name: Setup repository uses: ./.github/actions/setup-action with: - comment_phrase: ${{matrix.job_phrase}} + comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} - github_job: ${{ matrix.job_name }} (${{matrix.job_phrase}}) + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - name: Authenticate on GCP uses: google-github-actions/setup-gcloud@v0 with: diff --git a/.github/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml b/.github/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml index 79794904b487..e41f59f67205 100644 --- a/.github/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml +++ b/.github/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml @@ -70,7 +70,7 @@ env: jobs: beam_PreCommit_Java_PVR_Flink_Docker: - name: ${{matrix.job_name}} (${{matrix.job_phrase}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) runs-on: [self-hosted, ubuntu-20.04, main] strategy: fail-fast: false diff --git a/.github/workflows/beam_PreCommit_Java_Spark3_Versions.yml b/.github/workflows/beam_PreCommit_Java_Spark3_Versions.yml index 0a8096bf88b4..1809f11d8c7c 100644 --- a/.github/workflows/beam_PreCommit_Java_Spark3_Versions.yml +++ b/.github/workflows/beam_PreCommit_Java_Spark3_Versions.yml @@ -62,7 +62,7 @@ env: jobs: beam_PreCommit_Java_Spark3_Versions: - name: ${{matrix.job_name}} (${{ matrix.job_phrase }}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) runs-on: [self-hosted, ubuntu-20.04, main] strategy: matrix: @@ -80,7 +80,7 @@ jobs: - name: Setup repository uses: ./.github/actions/setup-action with: - comment_phrase: ${{matrix.job_phrase}} + comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - name: Authenticate on GCP diff --git a/.github/workflows/beam_PreCommit_Kotlin_Examples.yml b/.github/workflows/beam_PreCommit_Kotlin_Examples.yml index 2c16cd8e4749..8fe1673d9c55 100644 --- a/.github/workflows/beam_PreCommit_Kotlin_Examples.yml +++ b/.github/workflows/beam_PreCommit_Kotlin_Examples.yml @@ -72,7 +72,7 @@ env: jobs: beam_PreCommit_Kotlin_Examples: - name: ${{matrix.job_name}} (${{matrix.job_phrase}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) timeout-minutes: 120 runs-on: [self-hosted, ubuntu-20.04, main] strategy: @@ -90,7 +90,7 @@ jobs: - name: Setup repository uses: ./.github/actions/setup-action with: - comment_phrase: ${{matrix.job_phrase}} + comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - name: Install Java diff --git a/.github/workflows/beam_PreCommit_Portable_Python.yml b/.github/workflows/beam_PreCommit_Portable_Python.yml index bd35b1d0c1ef..e8221f7f23f5 100644 --- a/.github/workflows/beam_PreCommit_Portable_Python.yml +++ b/.github/workflows/beam_PreCommit_Portable_Python.yml @@ -76,7 +76,7 @@ env: jobs: beam_PreCommit_Portable_Python: - name: ${{matrix.job_name}} (${{matrix.job_phrase}} ${{matrix.python_version}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }} ${{ matrix.python_version }}) timeout-minutes: 120 runs-on: ['self-hosted', ubuntu-20.04, main] strategy: @@ -95,7 +95,7 @@ jobs: - name: Setup repository uses: ./.github/actions/setup-action with: - comment_phrase: ${{matrix.job_phrase}} ${{matrix.python_version}} + comment_phrase: ${{ matrix.job_phrase }} ${{ matrix.python_version }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }} ${{ matrix.python_version }}) - name: Install Java @@ -109,7 +109,7 @@ jobs: uses: actions/setup-python@v4 with: python-version: | - ${{matrix.python_version}} + ${{ matrix.python_version }} 3.8 - name: Set PY_VER_CLEAN id: set_py_ver_clean diff --git a/.github/workflows/beam_PreCommit_Python.yml b/.github/workflows/beam_PreCommit_Python.yml index 26efbad6ca0c..d65a8e643eb4 100644 --- a/.github/workflows/beam_PreCommit_Python.yml +++ b/.github/workflows/beam_PreCommit_Python.yml @@ -56,7 +56,7 @@ env: jobs: beam_PreCommit_Python: - name: ${{matrix.job_name}} (${{matrix.job_phrase}} ${{matrix.python_version}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }} ${{ matrix.python_version }}) runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 180 strategy: diff --git a/.github/workflows/beam_PreCommit_PythonDocker.yml b/.github/workflows/beam_PreCommit_PythonDocker.yml index 507600e6a32b..dc5ee73e198d 100644 --- a/.github/workflows/beam_PreCommit_PythonDocker.yml +++ b/.github/workflows/beam_PreCommit_PythonDocker.yml @@ -75,7 +75,7 @@ jobs: - name: Setup repository uses: ./.github/actions/setup-action with: - comment_phrase: ${{matrix.job_phrase}} ${{ matrix.python_version }} + comment_phrase: ${{ matrix.job_phrase }} ${{ matrix.python_version }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }} ${{ matrix.python_version }}) - name: Setup environment diff --git a/.github/workflows/beam_PreCommit_PythonDocs.yml b/.github/workflows/beam_PreCommit_PythonDocs.yml index 63e398aa5d4c..896d954386b5 100644 --- a/.github/workflows/beam_PreCommit_PythonDocs.yml +++ b/.github/workflows/beam_PreCommit_PythonDocs.yml @@ -57,7 +57,7 @@ env: jobs: beam_PreCommit_PythonDocs: - name: ${{matrix.job_name}} (${{matrix.job_phrase}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) runs-on: [self-hosted, ubuntu-20.04, main] strategy: matrix: @@ -75,7 +75,7 @@ jobs: - name: Setup repository uses: ./.github/actions/setup-action with: - comment_phrase: ${{matrix.job_phrase}} + comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - name: Setup environment diff --git a/.github/workflows/beam_PreCommit_PythonFormatter.yml b/.github/workflows/beam_PreCommit_PythonFormatter.yml index 8795b71ade70..e9c6787f6ee9 100644 --- a/.github/workflows/beam_PreCommit_PythonFormatter.yml +++ b/.github/workflows/beam_PreCommit_PythonFormatter.yml @@ -56,7 +56,7 @@ env: jobs: beam_PreCommit_PythonFormatter: - name: ${{matrix.job_name}} (${{matrix.job_phrase}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) runs-on: [self-hosted, ubuntu-20.04, main] strategy: matrix: @@ -74,7 +74,7 @@ jobs: - name: Setup repository uses: ./.github/actions/setup-action with: - comment_phrase: ${{matrix.job_phrase}} + comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - name: Setup environment diff --git a/.github/workflows/beam_PreCommit_PythonLint.yml b/.github/workflows/beam_PreCommit_PythonLint.yml index 2e75b61c10e1..2a1f186f2418 100644 --- a/.github/workflows/beam_PreCommit_PythonLint.yml +++ b/.github/workflows/beam_PreCommit_PythonLint.yml @@ -56,7 +56,7 @@ env: jobs: beam_PreCommit_PythonLint: - name: ${{matrix.job_name}} (${{ matrix.job_phrase }}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) runs-on: [self-hosted, ubuntu-20.04, main] strategy: matrix: diff --git a/.github/workflows/beam_PreCommit_Python_Coverage.yml b/.github/workflows/beam_PreCommit_Python_Coverage.yml index 68319e6140e4..ef8f9ee1bf1d 100644 --- a/.github/workflows/beam_PreCommit_Python_Coverage.yml +++ b/.github/workflows/beam_PreCommit_Python_Coverage.yml @@ -56,7 +56,7 @@ env: jobs: beam_PreCommit_Python_Coverage: - name: ${{matrix.job_name}} (${{matrix.job_phrase}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) runs-on: [self-hosted, ubuntu-20.04, main] strategy: matrix: @@ -74,7 +74,7 @@ jobs: - name: Setup repository uses: ./.github/actions/setup-action with: - comment_phrase: ${{matrix.job_phrase}} + comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - name: Setup environment diff --git a/.github/workflows/beam_PreCommit_Python_Dataframes.yml b/.github/workflows/beam_PreCommit_Python_Dataframes.yml index f1464e7fe5b9..b2292e3484a3 100644 --- a/.github/workflows/beam_PreCommit_Python_Dataframes.yml +++ b/.github/workflows/beam_PreCommit_Python_Dataframes.yml @@ -56,7 +56,7 @@ env: jobs: beam_PreCommit_Python_Dataframes: - name: ${{matrix.job_name}} (${{ matrix.job_phrase}} ${{ matrix.python_version}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase}} ${{ matrix.python_version}}) runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 180 strategy: diff --git a/.github/workflows/beam_PreCommit_Python_Examples.yml b/.github/workflows/beam_PreCommit_Python_Examples.yml index 1cfdb9f09c33..96d714973610 100644 --- a/.github/workflows/beam_PreCommit_Python_Examples.yml +++ b/.github/workflows/beam_PreCommit_Python_Examples.yml @@ -56,7 +56,7 @@ env: jobs: beam_PreCommit_Python_Examples: - name: ${{matrix.job_name}} (${{matrix.job_phrase}} ${{ matrix.python_version }}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }} ${{ matrix.python_version }}) runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 180 strategy: diff --git a/.github/workflows/beam_PreCommit_Python_Integration.yml b/.github/workflows/beam_PreCommit_Python_Integration.yml index b00330cbfe76..afb11abaabc9 100644 --- a/.github/workflows/beam_PreCommit_Python_Integration.yml +++ b/.github/workflows/beam_PreCommit_Python_Integration.yml @@ -56,7 +56,7 @@ env: jobs: beam_PreCommit_Python_Integration: - name: ${{matrix.job_name}} (${{matrix.job_phrase}} ${{matrix.python_version}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }} ${{ matrix.python_version }}) runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 180 strategy: diff --git a/.github/workflows/beam_PreCommit_Python_Runners.yml b/.github/workflows/beam_PreCommit_Python_Runners.yml index 5a65faf6e8d3..e3ad7675e983 100644 --- a/.github/workflows/beam_PreCommit_Python_Runners.yml +++ b/.github/workflows/beam_PreCommit_Python_Runners.yml @@ -56,7 +56,7 @@ env: jobs: beam_PreCommit_Python_Runners: - name: ${{matrix.job_name}} (${{matrix.job_phrase}} ${{matrix.python_version}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }} ${{ matrix.python_version }}) runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 180 strategy: diff --git a/.github/workflows/beam_PreCommit_Python_Transforms.yml b/.github/workflows/beam_PreCommit_Python_Transforms.yml index 197f77a38048..b62d4b4665ef 100644 --- a/.github/workflows/beam_PreCommit_Python_Transforms.yml +++ b/.github/workflows/beam_PreCommit_Python_Transforms.yml @@ -56,7 +56,7 @@ env: jobs: beam_PreCommit_Python_Transforms: - name: ${{matrix.job_name}} (${{matrix.job_phrase}} ${{matrix.python_version}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }} ${{ matrix.python_version }}) runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 180 strategy: diff --git a/.github/workflows/beam_PreCommit_RAT.yml b/.github/workflows/beam_PreCommit_RAT.yml index 314a0cf55b53..3d907410b5cd 100644 --- a/.github/workflows/beam_PreCommit_RAT.yml +++ b/.github/workflows/beam_PreCommit_RAT.yml @@ -55,7 +55,7 @@ env: jobs: beam_PreCommit_RAT: - name: ${{matrix.job_name}} (${{ matrix.job_phrase }}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) runs-on: [self-hosted, ubuntu-20.04, main] strategy: matrix: diff --git a/.github/workflows/beam_PreCommit_Spotless.yml b/.github/workflows/beam_PreCommit_Spotless.yml index 855c9b75d345..f3fc3794bdf3 100644 --- a/.github/workflows/beam_PreCommit_Spotless.yml +++ b/.github/workflows/beam_PreCommit_Spotless.yml @@ -69,7 +69,7 @@ env: jobs: beam_PreCommit_Spotless: - name: ${{matrix.job_name}} (${{matrix.job_phrase}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) timeout-minutes: 120 runs-on: [self-hosted, ubuntu-20.04, main] strategy: diff --git a/.github/workflows/beam_PreCommit_Typescript.yml b/.github/workflows/beam_PreCommit_Typescript.yml index 70ea8bf9963a..803d51e7e9a4 100644 --- a/.github/workflows/beam_PreCommit_Typescript.yml +++ b/.github/workflows/beam_PreCommit_Typescript.yml @@ -58,7 +58,7 @@ permissions: statuses: read jobs: beam_PreCommit_Typescript: - name: ${{matrix.job_name}} (${{matrix.job_phrase}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) timeout-minutes: 120 runs-on: [self-hosted, ubuntu-20.04, main] strategy: @@ -76,7 +76,7 @@ jobs: - name: Setup repository uses: ./.github/actions/setup-action with: - comment_phrase: ${{matrix.job_phrase}} + comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - name: Setup environment diff --git a/.github/workflows/beam_PreCommit_Website.yml b/.github/workflows/beam_PreCommit_Website.yml index b268ed6668ef..bce51e0c929a 100644 --- a/.github/workflows/beam_PreCommit_Website.yml +++ b/.github/workflows/beam_PreCommit_Website.yml @@ -57,7 +57,7 @@ env: jobs: beam_PreCommit_Website: - name: ${{matrix.job_name}} (${{ matrix.job_phrase}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase}}) runs-on: [self-hosted, ubuntu-20.04, small] strategy: matrix: diff --git a/.github/workflows/beam_PreCommit_Website_Stage_GCS.yml b/.github/workflows/beam_PreCommit_Website_Stage_GCS.yml index 88e02f0525ae..19d96eb8a1ce 100644 --- a/.github/workflows/beam_PreCommit_Website_Stage_GCS.yml +++ b/.github/workflows/beam_PreCommit_Website_Stage_GCS.yml @@ -59,7 +59,7 @@ permissions: jobs: beam_PreCommit_Website_Stage_GCS: - name: ${{matrix.job_name}} (${{matrix.job_phrase}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) runs-on: [self-hosted, ubuntu-20.04, main] strategy: matrix: diff --git a/.github/workflows/beam_PreCommit_Whitespace.yml b/.github/workflows/beam_PreCommit_Whitespace.yml index 5f430779eef9..148de272b622 100644 --- a/.github/workflows/beam_PreCommit_Whitespace.yml +++ b/.github/workflows/beam_PreCommit_Whitespace.yml @@ -56,7 +56,7 @@ env: jobs: beam_PreCommit_Whitespace: - name: ${{matrix.job_name}} (${{ matrix.job_phrase }}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) runs-on: [self-hosted, ubuntu-20.04, main] strategy: matrix: diff --git a/.github/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml b/.github/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml index 2bb4328eaf26..eaf3f2c5ca1a 100644 --- a/.github/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml +++ b/.github/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml @@ -52,7 +52,7 @@ env: jobs: beam_Python_ValidatesContainer_Dataflow_ARM: - name: ${{matrix.job_name}} (${{matrix.job_phrase}} ${{ matrix.python_version }}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }} ${{ matrix.python_version }}) strategy: fail-fast: false matrix: diff --git a/.github/workflows/beam_Release_NightlySnapshot.yml b/.github/workflows/beam_Release_NightlySnapshot.yml index a4be830cd3c5..bf22344c4199 100644 --- a/.github/workflows/beam_Release_NightlySnapshot.yml +++ b/.github/workflows/beam_Release_NightlySnapshot.yml @@ -43,7 +43,7 @@ env: jobs: beam_Release_NightlySnapshot: - name: ${{matrix.job_name}} (${{matrix.job_phrase}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) runs-on: [self-hosted, ubuntu-20.04, main] strategy: matrix: @@ -52,15 +52,14 @@ jobs: if: | github.event_name == 'workflow_dispatch' || github.event_name == 'schedule' - steps: - uses: actions/checkout@v4 - name: Setup repository uses: ./.github/actions/setup-action with: - github_job: ${{matrix.job_name}} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) github_token: ${{ secrets.GITHUB_TOKEN }} - comment_phrase: "Release Nightly Snapshot" + comment_phrase: ${{ matrix.job_phrase }} - name: Setup environment uses: ./.github/actions/setup-environment-action with: diff --git a/.github/workflows/beam_Release_Python_NightlySnapshot.yml b/.github/workflows/beam_Release_Python_NightlySnapshot.yml index 62019c536969..a9f4ac5d2d14 100644 --- a/.github/workflows/beam_Release_Python_NightlySnapshot.yml +++ b/.github/workflows/beam_Release_Python_NightlySnapshot.yml @@ -43,7 +43,7 @@ env: jobs: beam_Release_Python_NightlySnapshot: - name: ${{matrix.job_name}} (${{matrix.job_phrase}}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) runs-on: [self-hosted, ubuntu-20.04, main] strategy: matrix: @@ -57,9 +57,9 @@ jobs: - name: Setup repository uses: ./.github/actions/setup-action with: - github_job: ${{matrix.job_name}} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) github_token: ${{ secrets.GITHUB_TOKEN }} - comment_phrase: ${{matrix.job_phrase}} + comment_phrase: ${{ matrix.job_phrase }} - name: Setup environment uses: ./.github/actions/setup-environment-action with: From d7039bca9f276a8e1a6fa010b384b0915e195f1c Mon Sep 17 00:00:00 2001 From: Svetak Sundhar <svetaksundhar@google.com> Date: Wed, 18 Oct 2023 20:54:26 +0000 Subject: [PATCH 181/435] Update `google_api_services_healthcare` [10/23] (#29055) * Create HealthcareUtils file with shared resources * revert * Update Healthcare client on Groovy --- .../main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index 85d4e3de65a1..46968e0e5ad6 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -716,7 +716,7 @@ class BeamModulePlugin implements Plugin<Project> { // Keep version consistent with the version in google_cloud_resourcemanager, managed by google_cloud_platform_libraries_bom google_api_services_cloudresourcemanager : "com.google.apis:google-api-services-cloudresourcemanager:v1-rev20230806-$google_clients_version", google_api_services_dataflow : "com.google.apis:google-api-services-dataflow:v1b3-rev20220920-$google_clients_version", - google_api_services_healthcare : "com.google.apis:google-api-services-healthcare:v1-rev20230830-$google_clients_version", + google_api_services_healthcare : "com.google.apis:google-api-services-healthcare:v1-rev20231003-$google_clients_version", google_api_services_pubsub : "com.google.apis:google-api-services-pubsub:v1-rev20220904-$google_clients_version", // Keep version consistent with the version in google_cloud_nio, managed by google_cloud_platform_libraries_bom google_api_services_storage : "com.google.apis:google-api-services-storage:v1-rev20230907-$google_clients_version", From 964710f4f9d31cc8493a633e823ea1b3500faead Mon Sep 17 00:00:00 2001 From: Damon <damondouglas@users.noreply.github.com> Date: Wed, 18 Oct 2023 14:00:26 -0700 Subject: [PATCH 182/435] [RRIO] [Call] stub the Call transform (#29060) --- .../apache/beam/io/requestresponse/Call.java | 116 ++++++++++++++++++ 1 file changed, 116 insertions(+) create mode 100644 sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/Call.java diff --git a/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/Call.java b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/Call.java new file mode 100644 index 000000000000..4f854ea69c7e --- /dev/null +++ b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/Call.java @@ -0,0 +1,116 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.io.requestresponse; + +import com.google.auto.value.AutoValue; +import java.util.Map; +import org.apache.beam.io.requestresponse.Call.Result; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionTuple; +import org.apache.beam.sdk.values.PInput; +import org.apache.beam.sdk.values.POutput; +import org.apache.beam.sdk.values.PValue; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; + +/** + * {@link Call} transforms a {@link RequestT} {@link PCollection} into a {@link ResponseT} {@link + * PCollection} and {@link ApiIOError} {@link PCollection}, both wrapped in a {@link Result}. + */ +class Call<RequestT, ResponseT> extends PTransform<PCollection<RequestT>, Result<ResponseT>> { + + private static final TupleTag<ApiIOError> FAILURE_TAG = new TupleTag<ApiIOError>() {}; + + // TODO(damondouglas): remove suppress warnings when configuration utilized in future PR. + @SuppressWarnings({"unused"}) + private final Configuration<RequestT, ResponseT> configuration; + + private Call(Configuration<RequestT, ResponseT> configuration) { + this.configuration = configuration; + } + + /** Configuration details for {@link Call}. */ + @AutoValue + abstract static class Configuration<RequestT, ResponseT> { + + static <RequestT, ResponseT> Builder<RequestT, ResponseT> builder() { + return new AutoValue_Call_Configuration.Builder<>(); + } + + abstract Builder<RequestT, ResponseT> toBuilder(); + + @AutoValue.Builder + abstract static class Builder<RequestT, ResponseT> { + + abstract Configuration<RequestT, ResponseT> build(); + } + } + + @Override + public Result<ResponseT> expand(PCollection<RequestT> input) { + return Result.of(new TupleTag<ResponseT>() {}, PCollectionTuple.empty(input.getPipeline())); + } + + /** + * The {@link Result} of processing request {@link PCollection} into response {@link PCollection}. + */ + static class Result<ResponseT> implements POutput { + + static <ResponseT> Result<ResponseT> of(TupleTag<ResponseT> responseTag, PCollectionTuple pct) { + return new Result<>(responseTag, pct); + } + + private final Pipeline pipeline; + private final TupleTag<ResponseT> responseTag; + private final PCollection<ResponseT> responses; + private final PCollection<ApiIOError> failures; + + private Result(TupleTag<ResponseT> responseTag, PCollectionTuple pct) { + this.pipeline = pct.getPipeline(); + this.responseTag = responseTag; + this.responses = pct.get(responseTag); + this.failures = pct.get(FAILURE_TAG); + } + + public PCollection<ResponseT> getResponses() { + return responses; + } + + public PCollection<ApiIOError> getFailures() { + return failures; + } + + @Override + public Pipeline getPipeline() { + return this.pipeline; + } + + @Override + public Map<TupleTag<?>, PValue> expand() { + return ImmutableMap.of( + responseTag, responses, + FAILURE_TAG, failures); + } + + @Override + public void finishSpecifyingOutput( + String transformName, PInput input, PTransform<?, ?> transform) {} + } +} From 266721bc001394c5c222d769279fb2a1d7fd9442 Mon Sep 17 00:00:00 2001 From: Damon <damondouglas@users.noreply.github.com> Date: Wed, 18 Oct 2023 14:00:47 -0700 Subject: [PATCH 183/435] [RRIO] [Throttle] stub the refresh quota transform (#29057) --- .../requestresponse/ThrottleRefreshQuota.java | 55 +++++++++++++++++++ 1 file changed, 55 insertions(+) create mode 100644 sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/ThrottleRefreshQuota.java diff --git a/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/ThrottleRefreshQuota.java b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/ThrottleRefreshQuota.java new file mode 100644 index 000000000000..57e57528db4b --- /dev/null +++ b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/ThrottleRefreshQuota.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.io.requestresponse; + +import com.google.auto.value.AutoValue; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.TypeDescriptor; +import org.joda.time.Instant; + +/** + * {@link ThrottleRefreshQuota} refreshes a quota per {@link Instant} processing events emitting any + * errors into an {@link ApiIOError} {@link PCollection}. + */ +class ThrottleRefreshQuota extends PTransform<PCollection<Instant>, PCollection<ApiIOError>> { + + // TODO: remove suppress warnings after configuration utilized. + @SuppressWarnings({"unused"}) + private final Configuration configuration; + + private ThrottleRefreshQuota(Configuration configuration) { + this.configuration = configuration; + } + + @Override + public PCollection<ApiIOError> expand(PCollection<Instant> input) { + // TODO(damondouglas): expand in a later PR. + return input.getPipeline().apply(Create.empty(TypeDescriptor.of(ApiIOError.class))); + } + + @AutoValue + abstract static class Configuration { + + @AutoValue.Builder + abstract static class Builder { + abstract Configuration build(); + } + } +} From 9fdc59b2c0f9a54457d9cde3117e9f0c104d59f1 Mon Sep 17 00:00:00 2001 From: Damon <damondouglas@users.noreply.github.com> Date: Wed, 18 Oct 2023 14:01:02 -0700 Subject: [PATCH 184/435] [RRIO] [Throttle] stub the dequeue transform (#29056) --- .../io/requestresponse/ThrottleDequeue.java | 101 ++++++++++++++++++ 1 file changed, 101 insertions(+) create mode 100644 sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/ThrottleDequeue.java diff --git a/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/ThrottleDequeue.java b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/ThrottleDequeue.java new file mode 100644 index 000000000000..085b13b5e112 --- /dev/null +++ b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/ThrottleDequeue.java @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.io.requestresponse; + +import com.google.auto.value.AutoValue; +import java.util.Map; +import org.apache.beam.io.requestresponse.ThrottleDequeue.Result; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionTuple; +import org.apache.beam.sdk.values.PInput; +import org.apache.beam.sdk.values.POutput; +import org.apache.beam.sdk.values.PValue; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; +import org.joda.time.Instant; + +/** + * {@link ThrottleDequeue} dequeues {@link RequestT} elements at a fixed rate yielding a {@link + * Result} containing the dequeued {@link RequestT} {@link PCollection} and a {@link ApiIOError} + * {@link PCollection} of any errors. + */ +class ThrottleDequeue<RequestT> extends PTransform<PCollection<Instant>, Result<RequestT>> { + + private static final TupleTag<ApiIOError> FAILURE_TAG = new TupleTag<ApiIOError>() {}; + + // TODO(damondouglas): remove suppress warnings after instance utilized. + @SuppressWarnings({"unused"}) + private final Configuration<RequestT> configuration; + + private ThrottleDequeue(Configuration<RequestT> configuration) { + this.configuration = configuration; + } + + @Override + public Result<RequestT> expand(PCollection<Instant> input) { + // TODO(damondouglas): expand in a future PR. + return new Result<>(new TupleTag<RequestT>() {}, PCollectionTuple.empty(input.getPipeline())); + } + + @AutoValue + abstract static class Configuration<RequestT> { + + @AutoValue.Builder + abstract static class Builder<RequestT> { + abstract Configuration<RequestT> build(); + } + } + + /** The {@link Result} of dequeuing {@link RequestT}s. */ + static class Result<RequestT> implements POutput { + + static <RequestT> Result<RequestT> of(TupleTag<RequestT> requestsTag, PCollectionTuple pct) { + return new Result<>(requestsTag, pct); + } + + private final Pipeline pipeline; + private final TupleTag<RequestT> requestsTag; + private final PCollection<RequestT> requests; + private final PCollection<ApiIOError> failures; + + private Result(TupleTag<RequestT> requestsTag, PCollectionTuple pct) { + this.pipeline = pct.getPipeline(); + this.requestsTag = requestsTag; + this.requests = pct.get(requestsTag); + this.failures = pct.get(FAILURE_TAG); + } + + @Override + public Pipeline getPipeline() { + return pipeline; + } + + @Override + public Map<TupleTag<?>, PValue> expand() { + return ImmutableMap.of( + requestsTag, requests, + FAILURE_TAG, failures); + } + + @Override + public void finishSpecifyingOutput( + String transformName, PInput input, PTransform<?, ?> transform) {} + } +} From 9aaf9c213e6df38917e62c581dfb43fe7c112e5d Mon Sep 17 00:00:00 2001 From: JayajP <jayajp@google.com> Date: Wed, 18 Oct 2023 16:45:09 -0700 Subject: [PATCH 185/435] Implement java exponential histograms (#28903) (#28995) * Implement java exponential histograms (#28903) * Address comments * Address comments --- .../apache/beam/sdk/util/HistogramData.java | 194 ++++++++++++++++++ .../beam/sdk/util/HistogramDataTest.java | 132 ++++++++++++ 2 files changed, 326 insertions(+) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/HistogramData.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/HistogramData.java index b28e1cfd5af2..cca3a440334e 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/HistogramData.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/HistogramData.java @@ -23,6 +23,7 @@ import java.util.Arrays; import java.util.Objects; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.math.DoubleMath; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.math.IntMath; import org.checkerframework.checker.nullness.qual.Nullable; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -77,6 +78,55 @@ public static HistogramData linear(double start, double width, int numBuckets) { return new HistogramData(LinearBuckets.of(start, width, numBuckets)); } + /** + * Returns a histogram object wiht exponential boundaries. The input parameter {@code scale} + * determines a coefficient 'base' which species bucket boundaries. + * + * <pre> + * base = 2**(2**(-scale)) e.g. + * scale=1 => base=2**(1/2)=sqrt(2) + * scale=0 => base=2**(1)=2 + * scale=-1 => base=2**(2)=4 + * </pre> + * + * This bucketing strategy makes it simple/numerically stable to compute bucket indexes for + * datapoints. + * + * <pre> + * Bucket boundaries are given by the following table where n=numBuckets. + * | 'Bucket Index' | Bucket Boundaries | + * |---------------|---------------------| + * | Underflow | (-inf, 0) | + * | 0 | [0, base) | + * | 1 | [base, base^2) | + * | 2 | [base^2, base^3) | + * | i | [base^i, base^(i+1))| + * | n-1 | [base^(n-1), base^n)| + * | Overflow | [base^n, inf) | + * </pre> + * + * <pre> + * Example scale/boundaries: + * When scale=1, buckets 0,1,2...i have lowerbounds 0, 2^(1/2), 2^(2/2), ... 2^(i/2). + * When scale=0, buckets 0,1,2...i have lowerbounds 0, 2, 2^2, ... 2^(i). + * When scale=-1, buckets 0,1,2...i have lowerbounds 0, 4, 4^2, ... 4^(i). + * </pre> + * + * Scale parameter is similar to <a + * href="https://opentelemetry.io/docs/specs/otel/metrics/data-model/#exponentialhistogram"> + * OpenTelemetry's notion of ExponentialHistogram</a>. Bucket boundaries are modified to make them + * compatible with GCP's exponential histogram. + * + * @param numBuckets The number of buckets. Clipped so that the largest bucket's lower bound is + * not greater than 2^32-1 (uint32 max). + * @param scale Integer between [-3, 3] which determines bucket boundaries. Larger values imply + * more fine grained buckets. + * @return a new Histogram instance. + */ + public static HistogramData exponential(int scale, int numBuckets) { + return new HistogramData(ExponentialBuckets.of(scale, numBuckets)); + } + public void record(double... values) { for (double value : values) { record(value); @@ -227,6 +277,150 @@ public interface BucketType extends Serializable { double getAccumulatedBucketSize(int endIndex); } + @AutoValue + public abstract static class ExponentialBuckets implements BucketType { + + // Minimum scale factor. Bucket boundaries can grow at a rate of at most: 2^(2^3)=2^8=256 + private static final int MINIMUM_SCALE = -3; + + // Minimum scale factor. Bucket boundaries must grow at a rate of at least 2^(2^-3)=2^(1/8) + private static final int MAXIMUM_SCALE = 3; + + // Maximum number of buckets that is supported when 'scale' is zero. + private static final int ZERO_SCALE_MAX_NUM_BUCKETS = 32; + + public abstract double getBase(); + + public abstract int getScale(); + + /** + * Set to 2**scale which is equivalent to 1/log_2(base). Precomputed to use in {@code + * getBucketIndexPositiveScale} + */ + public abstract double getInvLog2GrowthFactor(); + + @Override + public abstract int getNumBuckets(); + + /* Precomputed since this value is used everytime a datapoint is recorded. */ + @Override + public abstract double getRangeTo(); + + public static ExponentialBuckets of(int scale, int numBuckets) { + if (scale < MINIMUM_SCALE) { + throw new IllegalArgumentException( + String.format("Scale should be greater than %d: %d", MINIMUM_SCALE, scale)); + } + + if (scale > MAXIMUM_SCALE) { + throw new IllegalArgumentException( + String.format("Scale should be less than %d: %d", MAXIMUM_SCALE, scale)); + } + if (numBuckets <= 0) { + throw new IllegalArgumentException( + String.format("numBuckets should be positive: %d", numBuckets)); + } + + double invLog2GrowthFactor = Math.pow(2, scale); + double base = Math.pow(2, Math.pow(2, -scale)); + int clippedNumBuckets = ExponentialBuckets.computeNumberOfBuckets(scale, numBuckets); + double rangeTo = Math.pow(base, clippedNumBuckets); + return new AutoValue_HistogramData_ExponentialBuckets( + base, scale, invLog2GrowthFactor, clippedNumBuckets, rangeTo); + } + + /** + * numBuckets is clipped so that the largest bucket's lower bound is not greater than 2^32-1 + * (uint32 max). This value is log_base(2^32) which simplifies as follows: + * + * <pre> + * log_base(2^32) + * = log_2(2^32)/log_2(base) + * = 32/(2**-scale) + * = 32*(2**scale) + * </pre> + */ + private static int computeNumberOfBuckets(int scale, int inputNumBuckets) { + if (scale == 0) { + // When base=2 then the bucket at index 31 contains [2^31, 2^32). + return Math.min(ZERO_SCALE_MAX_NUM_BUCKETS, inputNumBuckets); + } else if (scale > 0) { + // When scale is positive 32*(2**scale) is equivalent to a right bit-shift. + return Math.min(inputNumBuckets, ZERO_SCALE_MAX_NUM_BUCKETS << scale); + } else { + // When scale is negative 32*(2**scale) is equivalent to a left bit-shift. + return Math.min(inputNumBuckets, ZERO_SCALE_MAX_NUM_BUCKETS >> -scale); + } + } + + @Override + public int getBucketIndex(double value) { + if (value < getBase()) { + return 0; + } + + // When scale is non-positive, 'base' and 'bucket boundaries' will be integers. + // In this scenario `value` and `floor(value)` will belong to the same bucket. + int index; + if (getScale() > 0) { + index = getBucketIndexPositiveScale(value); + } else if (getScale() < 0) { + index = getBucketIndexNegativeScale(DoubleMath.roundToInt(value, RoundingMode.FLOOR)); + } else { + index = getBucketIndexZeroScale(DoubleMath.roundToInt(value, RoundingMode.FLOOR)); + } + // Ensure that a valid index is returned in the off chance of a numerical instability error. + return Math.max(Math.min(index, getNumBuckets() - 1), 0); + } + + private int getBucketIndexZeroScale(int value) { + return IntMath.log2(value, RoundingMode.FLOOR); + } + + private int getBucketIndexNegativeScale(int value) { + return getBucketIndexZeroScale(value) >> (-getScale()); + } + + // This method is valid for all 'scale' values but we fallback to more effecient methods for + // non-positive scales. + // For a value>base we would like to find an i s.t. : + // base^i <= value < base^(i+1) + // i <= log_base(value) < i+1 + // i = floor(log_base(value)) + // i = floor(log_2(value)/log_2(base)) + private int getBucketIndexPositiveScale(double value) { + return DoubleMath.roundToInt( + getInvLog2GrowthFactor() * DoubleMath.log2(value), RoundingMode.FLOOR); + } + + @Override + public double getBucketSize(int index) { + if (index < 0) { + return 0; + } + if (index == 0) { + return getBase(); + } + + // bucketSize = (base)^(i+1) - (base)^i + // = (base)^i(base - 1) + return Math.pow(getBase(), index) * (getBase() - 1); + } + + @Override + public double getAccumulatedBucketSize(int endIndex) { + if (endIndex < 0) { + return 0; + } + return Math.pow(getBase(), endIndex + 1); + } + + @Override + public double getRangeFrom() { + return 0; + } + } + @AutoValue public abstract static class LinearBuckets implements BucketType { public abstract double getStart(); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/HistogramDataTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/HistogramDataTest.java index b6e4d989a8f3..bfad087ecfa5 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/HistogramDataTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/HistogramDataTest.java @@ -18,8 +18,10 @@ package org.apache.beam.sdk.util; import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.closeTo; import static org.hamcrest.Matchers.equalTo; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.math.IntMath; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -200,4 +202,134 @@ public void testIncrementBucketCountByIndex() { assertThat(data.getTopBucketCount(), equalTo(4L)); assertThat(data.getTotalCount(), equalTo(10L)); } + + // The following tests cover exponential buckets. + @Test + public void testExponentialBuckets_PostiveScaleRecord() { + // Buckets will be: + // Index Range + // Underflow (-inf, 0) + // 0 [0, sqrt(2)) + // 1 [sqrt(2), 2) + // i [2^(i/2), 2^((i+1)/2)) + HistogramData data = HistogramData.exponential(1, 40); + + data.record(-1); + assertThat(data.getBottomBucketCount(), equalTo(1L)); + + data.record(0, 1); + assertThat(data.getCount(0), equalTo(2L)); + + data.record(2); + assertThat(data.getTotalCount(), equalTo(4L)); + assertThat(data.getCount(2), equalTo(1L)); + + // 10th bucket contains range [2^5, 2^5.5) ~= [32, 45.25) + for (int i = 32; i <= 45; i++) { + data.record(i); + } + assertThat(data.getCount(10), equalTo(14L)); + + // 30th bucket contains range [2^15, 2^15.5) ~= [32768, 46340.9) + for (int i = 32768; i < 32768 + 100; i++) { + data.record(i); + } + assertThat(data.getCount(30), equalTo(100L)); + for (int i = 46340; i > 46340 - 100; i--) { + data.record(i); + } + assertThat(data.getCount(30), equalTo(200L)); + } + + @Test + public void testExponentialBuckets_ZeroScaleRecord() { + // Buckets will be: + // Index Range + // Underflow (-inf, 0) + // 0 [0, 2) + // 1 [2, 2^2] + // i [2^i, 2^(i+1)) + HistogramData data = HistogramData.exponential(0, 20); + + data.record(-1); + assertThat(data.getBottomBucketCount(), equalTo(1L)); + + data.record(0, 1); + assertThat(data.getCount(0), equalTo(2L)); + + data.record(4, 5, 6, 7); + assertThat(data.getCount(2), equalTo(4L)); + + for (int i = 32; i < 64; i++) { + data.record(i); + } + assertThat(data.getCount(5), equalTo(32L)); + + for (int i = IntMath.pow(2, 16); i < IntMath.pow(2, 16) + 100; i++) { + data.record(i); + } + assertThat(data.getCount(16), equalTo(100L)); + + Long expectedTotalCount = Long.valueOf(100 + 32 + 4 + 2 + 1); + assertThat(data.getTotalCount(), equalTo(expectedTotalCount)); + } + + @Test + public void testExponentialBuckets_NegativeScalesRecord() { + // Buckets will be: + // Index Range + // Underflow (-inf, 0) + // 0 [0, 4) + // 1 [4, 4^2] + // i [4^i, 4^(i+1)) + HistogramData data = HistogramData.exponential(-1, 20); + + data.record(-1); + assertThat(data.getBottomBucketCount(), equalTo(1L)); + + data.record(0, 1, 2); + assertThat(data.getCount(0), equalTo(3L)); + + data.record(16, 17, 32, 33, 62, 63); + assertThat(data.getCount(2), equalTo(6L)); + + for (int i = IntMath.pow(4, 5); i < IntMath.pow(4, 5) + 20; i++) { + data.record(i); + } + assertThat(data.getCount(5), equalTo(20L)); + + Long expectedTotalCount = Long.valueOf(20 + 6 + 3 + 1); + assertThat(data.getTotalCount(), equalTo(expectedTotalCount)); + } + + @Test + public void testExponentialBuckets_BucketSize() { + HistogramData zeroScaleBucket = HistogramData.exponential(0, 20); + assertThat(zeroScaleBucket.getBucketType().getBucketSize(0), equalTo(2.0)); + // 10th bucket contains [2^10, 2^11). + assertThat(zeroScaleBucket.getBucketType().getBucketSize(10), equalTo(1024.0)); + + HistogramData positiveScaleBucket = HistogramData.exponential(1, 20); + assertThat(positiveScaleBucket.getBucketType().getBucketSize(0), equalTo(Math.sqrt(2))); + // 10th bucket contains [2^5, 2^5.5). + assertThat(positiveScaleBucket.getBucketType().getBucketSize(10), closeTo(13.2, .1)); + + HistogramData negativeScaleBucket = HistogramData.exponential(-1, 20); + assertThat(negativeScaleBucket.getBucketType().getBucketSize(0), equalTo(4.0)); + // 10th bucket contains [2^20, 2^22). + assertThat(negativeScaleBucket.getBucketType().getBucketSize(10), equalTo(3145728.0)); + } + + @Test + public void testExponentialBuckets_NumBuckets() { + // Validate that numBuckets clipping WAI. + HistogramData zeroScaleBucket = HistogramData.exponential(0, 200); + assertThat(zeroScaleBucket.getBucketType().getNumBuckets(), equalTo(32)); + + HistogramData positiveScaleBucket = HistogramData.exponential(3, 500); + assertThat(positiveScaleBucket.getBucketType().getNumBuckets(), equalTo(32 * 8)); + + HistogramData negativeScaleBucket = HistogramData.exponential(-3, 500); + assertThat(negativeScaleBucket.getBucketType().getNumBuckets(), equalTo(4)); + } } From 885f3c9600d8c4e1495a916adf865e8b33f26c14 Mon Sep 17 00:00:00 2001 From: magicgoody <131876064+magicgoody@users.noreply.github.com> Date: Thu, 19 Oct 2023 09:04:19 +0600 Subject: [PATCH 186/435] Adjust a schedule to run workflows in subgroups (#29049) * cron time update * cron fix --- .github/workflows/beam_Java_LoadTests_Combine_Smoke.yml | 2 +- .github/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml | 2 +- .github/workflows/beam_LoadTests_Go_CoGBK_Flink_batch.yml | 2 +- .github/workflows/beam_LoadTests_Go_Combine_Dataflow_Batch.yml | 2 +- .github/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml | 2 +- .github/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml | 2 +- .github/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml | 2 +- .github/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml | 2 +- .github/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml | 2 +- .../workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml | 2 +- .github/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml | 2 +- .github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml | 2 +- .../workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml | 2 +- ...beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml | 2 +- ..._LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml | 2 +- ...beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml | 2 +- .../workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml | 2 +- .../beam_LoadTests_Java_Combine_Dataflow_Streaming.yml | 2 +- ...am_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml | 2 +- .github/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml | 2 +- .../workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml | 2 +- .../beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml | 2 +- .../beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml | 2 +- .../beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml | 2 +- .../beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml | 2 +- .../beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch.yml | 2 +- .github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml | 2 +- .../workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml | 2 +- ...beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions.yml | 2 +- ..._LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions.yml | 2 +- ...beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml | 2 +- .../workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml | 2 +- .../beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml | 2 +- .github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml | 2 +- .../workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml | 2 +- .../beam_LoadTests_Python_Combine_Dataflow_Streaming.yml | 2 +- .github/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml | 2 +- .../workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml | 2 +- .github/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml | 2 +- .../workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml | 2 +- .github/workflows/beam_LoadTests_Python_GBK_Flink_Batch.yml | 2 +- .../beam_LoadTests_Python_GBK_reiterate_Dataflow_Batch.yml | 2 +- .../beam_LoadTests_Python_GBK_reiterate_Dataflow_Streaming.yml | 2 +- .../workflows/beam_LoadTests_Python_ParDo_Dataflow_Batch.yml | 2 +- .../beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml | 2 +- .github/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml | 2 +- .../workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml | 2 +- .../beam_LoadTests_Python_SideInput_Dataflow_Batch.yml | 2 +- .github/workflows/beam_PerformanceTests_AvroIOIT.yml | 2 +- .github/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml | 2 +- .../beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml | 2 +- .../beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml | 2 +- .../beam_PerformanceTests_BigQueryIO_Streaming_Java.yml | 2 +- .../workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml | 2 +- .../beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml | 2 +- .github/workflows/beam_PerformanceTests_Cdap.yml | 2 +- .github/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml | 2 +- .../beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml | 2 +- .github/workflows/beam_PerformanceTests_HadoopFormat.yml | 2 +- .github/workflows/beam_PerformanceTests_JDBC.yml | 2 +- .github/workflows/beam_PerformanceTests_Kafka_IO.yml | 2 +- .github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml | 2 +- .../workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml | 2 +- .github/workflows/beam_PerformanceTests_MongoDBIO_IT.yml | 2 +- .github/workflows/beam_PerformanceTests_ParquetIOIT.yml | 2 +- .github/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml | 2 +- .../beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml | 2 +- .../beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml | 2 +- .../beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml | 2 +- .../beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml | 2 +- .github/workflows/beam_PerformanceTests_SparkReceiver_IO.yml | 2 +- .github/workflows/beam_PerformanceTests_TFRecordIOIT.yml | 2 +- .github/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml | 2 +- .github/workflows/beam_PerformanceTests_TextIOIT.yml | 2 +- .github/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml | 2 +- .github/workflows/beam_PerformanceTests_TextIOIT_Python.yml | 2 +- .../beam_PerformanceTests_WordCountIT_PythonVersions.yml | 2 +- .github/workflows/beam_PerformanceTests_XmlIOIT.yml | 2 +- .github/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml | 2 +- .../workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml | 2 +- .github/workflows/beam_PostCommit_Go.yml | 2 +- .github/workflows/beam_PostCommit_Go_Dataflow_ARM.yml | 2 +- .github/workflows/beam_PostCommit_Go_VR_Flink.yml | 2 +- .github/workflows/beam_PostCommit_Go_VR_Samza.yml | 2 +- .github/workflows/beam_PostCommit_Go_VR_Spark.yml | 2 +- .github/workflows/beam_PostCommit_Java.yml | 2 +- .github/workflows/beam_PostCommit_Java_Avro_Versions.yml | 2 +- .github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml | 2 +- .github/workflows/beam_PostCommit_Java_DataflowV1.yml | 2 +- .github/workflows/beam_PostCommit_Java_DataflowV2.yml | 2 +- .github/workflows/beam_PostCommit_Java_Examples_Dataflow.yml | 2 +- .../workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml | 2 +- .../workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml | 2 +- .github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml | 2 +- .../beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml | 2 +- .github/workflows/beam_PostCommit_Java_Examples_Direct.yml | 2 +- .github/workflows/beam_PostCommit_Java_Examples_Flink.yml | 2 +- .github/workflows/beam_PostCommit_Java_Examples_Spark.yml | 2 +- .github/workflows/beam_PostCommit_Java_Hadoop_Versions.yml | 2 +- .github/workflows/beam_PostCommit_Java_IO_Performance_Tests.yml | 2 +- .github/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml | 2 +- .github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml | 2 +- .github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml | 2 +- .github/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml | 2 +- .github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml | 2 +- .github/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml | 2 +- .github/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml | 2 +- .github/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml | 2 +- .github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml | 2 +- .../workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml | 2 +- .github/workflows/beam_PostCommit_Java_Nexmark_Direct.yml | 2 +- .github/workflows/beam_PostCommit_Java_Nexmark_Flink.yml | 2 +- .github/workflows/beam_PostCommit_Java_Nexmark_Spark.yml | 2 +- .github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml | 2 +- .github/workflows/beam_PostCommit_Java_PVR_Samza.yml | 2 +- .github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml | 2 +- .github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml | 2 +- .github/workflows/beam_PostCommit_Java_Sickbay.yml | 2 +- .github/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml | 2 +- .github/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml | 2 +- .github/workflows/beam_PostCommit_Java_Tpcds_Flink.yml | 2 +- .github/workflows/beam_PostCommit_Java_Tpcds_Spark.yml | 2 +- .../workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml | 2 +- ...am_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml | 2 +- .../beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml | 2 +- .../beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml | 2 +- ...am_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml | 2 +- .../workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml | 2 +- ...beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml | 2 +- .../workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml | 2 +- .../beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml | 2 +- .../workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml | 2 +- .../workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml | 2 +- ...PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml | 2 +- .../beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml | 2 +- .../workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml | 2 +- .github/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml | 2 +- .github/workflows/beam_PostCommit_Javadoc.yml | 2 +- .github/workflows/beam_PostCommit_PortableJar_Flink.yml | 2 +- .github/workflows/beam_PostCommit_PortableJar_Spark.yml | 2 +- .github/workflows/beam_PostCommit_Python.yml | 2 +- .github/workflows/beam_PostCommit_Python_Arm.yml | 2 +- .github/workflows/beam_PostCommit_Python_Examples_Dataflow.yml | 2 +- .github/workflows/beam_PostCommit_Python_Examples_Direct.yml | 2 +- .github/workflows/beam_PostCommit_Python_Examples_Flink.yml | 2 +- .github/workflows/beam_PostCommit_Python_Examples_Spark.yml | 2 +- .github/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml | 2 +- .github/workflows/beam_PostCommit_Python_Nexmark_Direct.yml | 2 +- .../beam_PostCommit_Python_ValidatesContainer_Dataflow.yml | 2 +- ...am_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml | 2 +- .../beam_PostCommit_Python_ValidatesRunner_Dataflow.yml | 2 +- .../workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml | 2 +- .../workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml | 2 +- .../workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml | 2 +- .github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml | 2 +- .github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml | 2 +- .github/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml | 2 +- .github/workflows/beam_PostCommit_SQL.yml | 2 +- .github/workflows/beam_PostCommit_Sickbay_Python.yml | 2 +- .github/workflows/beam_PostCommit_TransformService_Direct.yml | 2 +- .github/workflows/beam_PostCommit_Website_Publish.yml | 2 +- .github/workflows/beam_PostCommit_Website_Test.yml | 2 +- .github/workflows/beam_PostCommit_XVR_Direct.yml | 2 +- .github/workflows/beam_PostCommit_XVR_Flink.yml | 2 +- .github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml | 2 +- .../workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml | 2 +- .../beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml | 2 +- .../workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml | 2 +- .github/workflows/beam_PostCommit_XVR_Samza.yml | 2 +- .github/workflows/beam_PostCommit_XVR_Spark3.yml | 2 +- .github/workflows/beam_PreCommit_CommunityMetrics.yml | 2 +- .github/workflows/beam_PreCommit_Go.yml | 2 +- .github/workflows/beam_PreCommit_GoPortable.yml | 2 +- .github/workflows/beam_PreCommit_GoPrism.yml | 2 +- .github/workflows/beam_PreCommit_ItFramework.yml | 2 +- .github/workflows/beam_PreCommit_Java.yml | 2 +- .../beam_PreCommit_Java_Amazon-Web-Services2_IO_Direct.yml | 2 +- .../beam_PreCommit_Java_Amazon-Web-Services_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Amqp_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml | 2 +- .../workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Examples_Dataflow.yml | 2 +- .../workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml | 2 +- .../workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml | 2 +- .../beam_PreCommit_Java_File-schema-transform_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Flink_Versions.yml | 2 +- .github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_PVR_Flink_Batch.yml | 2 +- .github/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml | 2 +- .github/workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Spark3_Versions.yml | 2 +- .github/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Kotlin_Examples.yml | 2 +- .github/workflows/beam_PreCommit_Portable_Python.yml | 2 +- .github/workflows/beam_PreCommit_Python.yml | 2 +- .github/workflows/beam_PreCommit_PythonDocker.yml | 2 +- .github/workflows/beam_PreCommit_PythonDocs.yml | 2 +- .github/workflows/beam_PreCommit_PythonFormatter.yml | 2 +- .github/workflows/beam_PreCommit_PythonLint.yml | 2 +- .github/workflows/beam_PreCommit_Python_Coverage.yml | 2 +- .github/workflows/beam_PreCommit_Python_Dataframes.yml | 2 +- .github/workflows/beam_PreCommit_Python_Examples.yml | 2 +- .github/workflows/beam_PreCommit_Python_Integration.yml | 2 +- .github/workflows/beam_PreCommit_Python_PVR_Flink.yml | 2 +- .github/workflows/beam_PreCommit_Python_Runners.yml | 2 +- .github/workflows/beam_PreCommit_Python_Transforms.yml | 2 +- .github/workflows/beam_PreCommit_RAT.yml | 2 +- .github/workflows/beam_PreCommit_SQL.yml | 2 +- .github/workflows/beam_PreCommit_SQL_Java11.yml | 2 +- .github/workflows/beam_PreCommit_SQL_Java17.yml | 2 +- .github/workflows/beam_PreCommit_Spotless.yml | 2 +- .github/workflows/beam_PreCommit_Typescript.yml | 2 +- .github/workflows/beam_PreCommit_Website.yml | 2 +- .github/workflows/beam_PreCommit_Website_Stage_GCS.yml | 2 +- .github/workflows/beam_PreCommit_Whitespace.yml | 2 +- 240 files changed, 240 insertions(+), 240 deletions(-) diff --git a/.github/workflows/beam_Java_LoadTests_Combine_Smoke.yml b/.github/workflows/beam_Java_LoadTests_Combine_Smoke.yml index 6660caab2e3f..39fdf5a40080 100644 --- a/.github/workflows/beam_Java_LoadTests_Combine_Smoke.yml +++ b/.github/workflows/beam_Java_LoadTests_Combine_Smoke.yml @@ -19,7 +19,7 @@ on: # issue_comment: # types: [created] # schedule: - # - cron: '1 1 * * *' + # - cron: '10 12 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml index f50408e09b43..5efd4b749910 100644 --- a/.github/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml @@ -17,7 +17,7 @@ name: Load Tests CoGBK Dataflow Batch Go on: schedule: - - cron: '40 23 * * *' + - cron: '10 12 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_LoadTests_Go_CoGBK_Flink_batch.yml b/.github/workflows/beam_LoadTests_Go_CoGBK_Flink_batch.yml index 49894879a915..659c7ae0fc29 100644 --- a/.github/workflows/beam_LoadTests_Go_CoGBK_Flink_batch.yml +++ b/.github/workflows/beam_LoadTests_Go_CoGBK_Flink_batch.yml @@ -17,7 +17,7 @@ name: LoadTests Go CoGBK Flink Batch on: schedule: - - cron: '10 14 * * *' + - cron: '10 12 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_LoadTests_Go_Combine_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Go_Combine_Dataflow_Batch.yml index 365f48fa1be2..2abaa3240152 100644 --- a/.github/workflows/beam_LoadTests_Go_Combine_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_Combine_Dataflow_Batch.yml @@ -17,7 +17,7 @@ name: Load Tests Combine Dataflow Batch Go on: schedule: - - cron: '40 23 * * *' + - cron: '10 12 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml index 7cab058f5eeb..03d72fbe6f13 100644 --- a/.github/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml @@ -17,7 +17,7 @@ name: LoadTests Go Combine Flink Batch on: schedule: - - cron: '40 6 * * *' + - cron: '10 12 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml index 9656c14ed999..33850101e918 100644 --- a/.github/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml @@ -17,7 +17,7 @@ name: Load Tests GBK Dataflow Batch Go on: schedule: - - cron: '40 23 * * *' + - cron: '50 12 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml index 41eb61efc580..4ab802ced6ca 100644 --- a/.github/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml @@ -17,7 +17,7 @@ name: LoadTests Go GBK Flink Batch on: schedule: - - cron: '20 1 * * *' + - cron: '50 12 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml index c1aa013e6ede..058e15783bd8 100644 --- a/.github/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml @@ -17,7 +17,7 @@ name: Load Tests ParDo Dataflow Batch Go on: schedule: - - cron: '15 18 * * *' + - cron: '50 12 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml index a99422287a5b..3fa326c733bd 100644 --- a/.github/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml @@ -17,7 +17,7 @@ name: LoadTests Go ParDo Flink Batch on: schedule: - - cron: '40 2 * * *' + - cron: '50 12 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml index 38373e98a4cc..65b71bb79649 100644 --- a/.github/workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml @@ -17,7 +17,7 @@ name: Load Tests SideInput Dataflow Batch Go on: schedule: - - cron: '40 23 * * *' + - cron: '50 12 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml index 079aac5ad7f3..72162de5155b 100644 --- a/.github/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml @@ -17,7 +17,7 @@ name: LoadTests Go SideInput Flink Batch on: schedule: - - cron: '40 23 * * *' + - cron: '10 13 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml index 8c05e76a0fdd..ae50db426a81 100644 --- a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml @@ -19,7 +19,7 @@ on: issue_comment: types: [created] schedule: - - cron: '30 9 * * *' + - cron: '10 13 * * *' workflow_dispatch: # Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml index 91ba9639c9c8..84250af98aa1 100644 --- a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml @@ -17,7 +17,7 @@ name: Load Tests CoGBK Dataflow Streaming Java on: schedule: - - cron: '50 10 * * *' + - cron: '10 13 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml index 2960fafe6b32..a35c3bee7f96 100644 --- a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml +++ b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml @@ -19,7 +19,7 @@ on: issue_comment: types: [created] schedule: - - cron: '0 10 * * *' + - cron: '10 13 * * *' workflow_dispatch: # Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml index 113927dba340..ea7fba6a3e5e 100644 --- a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml +++ b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml @@ -19,7 +19,7 @@ on: issue_comment: types: [created] schedule: - - cron: '30 10 * * *' + - cron: '10 13 * * *' workflow_dispatch: # Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml b/.github/workflows/beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml index 1c39a54e7ca1..dd61eb79fb9b 100644 --- a/.github/workflows/beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml +++ b/.github/workflows/beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml @@ -19,7 +19,7 @@ on: issue_comment: types: [created] schedule: - - cron: '0 11 * * *' + - cron: '50 13 * * *' workflow_dispatch: # Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml index de317b4a87ca..b87c2c11f1d4 100644 --- a/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml @@ -17,7 +17,7 @@ name: Load Tests Combine Dataflow Batch Java on: schedule: - - cron: '35 7 * * *' + - cron: '50 13 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Streaming.yml index 13b07f16b259..5d372505bd30 100644 --- a/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Streaming.yml @@ -17,7 +17,7 @@ name: LoadTests Java Combine Dataflow Streaming on: schedule: - - cron: '25 14 * * *' + - cron: '50 13 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml b/.github/workflows/beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml index bae659d14d09..62b701b95f50 100644 --- a/.github/workflows/beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml +++ b/.github/workflows/beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml @@ -17,7 +17,7 @@ name: LoadTests Java Combine SparkStructuredStreaming Batch on: schedule: - - cron: '15 18 * * *' + - cron: '50 13 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml index e3aca91c1e76..9eece637f8f4 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml @@ -17,7 +17,7 @@ name: LoadTests Java GBK Dataflow Batch on: schedule: - - cron: '10 6 * * *' + - cron: '50 13 * * *' workflow_dispatch: # Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml index f28c7a147b94..d2850b4aec58 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml @@ -17,7 +17,7 @@ name: LoadTests Java GBK Dataflow Streaming on: schedule: - - cron: '50 6 * * *' + - cron: '10 14 * * *' workflow_dispatch: # Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml index a7f60be0a2ce..bc5760d99180 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml @@ -17,7 +17,7 @@ name: LoadTests Java GBK Dataflow V2 Batch Java11 on: schedule: - - cron: '10 7 * * *' + - cron: '10 14 * * *' workflow_dispatch: # Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml index 88de449d9729..82929ad6783a 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml @@ -17,7 +17,7 @@ name: LoadTests Java GBK Dataflow V2 Batch Java17 on: schedule: - - cron: '50 7 * * *' + - cron: '10 14 * * *' workflow_dispatch: # Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml index 39166855e39f..c9b278d5eed5 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml @@ -17,7 +17,7 @@ name: LoadTests Java GBK Dataflow V2 Streaming Java11 on: schedule: - - cron: '50 8 * * *' + - cron: '10 14 * * *' workflow_dispatch: # Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml index 4f76b3f45eb9..afedbd706b10 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml @@ -17,7 +17,7 @@ name: LoadTests Java GBK Dataflow V2 Streaming Java17 on: schedule: - - cron: '50 9 * * *' + - cron: '10 14 * * *' workflow_dispatch: # Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch.yml b/.github/workflows/beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch.yml index 13fe4df9906d..0ebfed7d477c 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch.yml @@ -17,7 +17,7 @@ name: LoadTests Java GBK SparkStructuredStreaming Batch on: schedule: - - cron: '10 10 * * *' + - cron: '50 14 * * *' workflow_dispatch: # Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml index 1434521f7c28..df9d3d19c485 100644 --- a/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml @@ -17,7 +17,7 @@ name: LoadTests Java ParDo Dataflow Batch on: schedule: - - cron: '55 9 * * *' + - cron: '50 14 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml index aabc12c366b5..4505ac0953c7 100644 --- a/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml @@ -17,7 +17,7 @@ name: LoadTests Java ParDo Dataflow Streaming on: schedule: - - cron: '10 11 * * *' + - cron: '50 14 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions.yml b/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions.yml index eebc96a63959..0203169f96b3 100644 --- a/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions.yml +++ b/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions.yml @@ -19,7 +19,7 @@ on: issue_comment: types: [created] schedule: - - cron: '40 16 * * *' + - cron: '50 14 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions.yml b/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions.yml index dcec86f8b4e0..45ddd688cdcc 100644 --- a/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions.yml +++ b/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions.yml @@ -19,7 +19,7 @@ on: issue_comment: types: [created] schedule: - - cron: '30 21 * * *' + - cron: '50 14 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml b/.github/workflows/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml index b5fc5792e0ff..fceb880c24b2 100644 --- a/.github/workflows/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml +++ b/.github/workflows/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml @@ -17,7 +17,7 @@ name: LoadTests Java ParDo SparkStructuredStreaming Batch on: schedule: - - cron: '25 8 * * *' + - cron: '10 15 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml index 5f63bdb9c3b5..ab28ffead25e 100644 --- a/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml @@ -19,7 +19,7 @@ on: issue_comment: types: [created] schedule: - - cron: '30 11 * * *' + - cron: '10 15 * * *' workflow_dispatch: # Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml index 4e25d81bfcbc..d25a79cdd9a1 100644 --- a/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml @@ -19,7 +19,7 @@ on: issue_comment: types: [created] schedule: - - cron: '30 11 * * *' + - cron: '10 15 * * *' workflow_dispatch: # Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml index 17851d5e57e1..73e28cd36d5c 100644 --- a/.github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml @@ -19,7 +19,7 @@ on: issue_comment: types: [created] schedule: - - cron: '40 12 * * *' + - cron: '10 15 * * *' workflow_dispatch: # Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml index 15143708c365..a40a4a5dada0 100644 --- a/.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml @@ -17,7 +17,7 @@ name: Load Tests Combine Dataflow Batch Python on: schedule: - - cron: '40 5 * * *' + - cron: '10 15 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Streaming.yml index 005bddfa49ac..8bbd47a74944 100644 --- a/.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Streaming.yml @@ -19,7 +19,7 @@ on: issue_comment: types: [created] schedule: - - cron: '10 9 * * *' + - cron: '50 15 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml index 6f7d2a8caa0b..f1e9424cd6ca 100644 --- a/.github/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml @@ -17,7 +17,7 @@ name: LoadTests Python Combine Flink Batch on: schedule: - - cron: '10 6 * * *' + - cron: '50 15 * * *' workflow_dispatch: # Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml b/.github/workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml index 636dd20c148b..c7b8b45aa8f9 100644 --- a/.github/workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml @@ -17,7 +17,7 @@ name: LoadTests Python Combine Flink Streaming on: schedule: - - cron: '10 7 * * *' + - cron: '50 15 * * *' workflow_dispatch: # Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml index d9d9564d3bfe..537b14125cd7 100644 --- a/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml @@ -17,7 +17,7 @@ name: LoadTests Python GBK Dataflow Batch on: schedule: - - cron: '10 2 * * *' + - cron: '50 15 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml index cf252f0758e0..6d0b2ca6f995 100644 --- a/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml @@ -17,7 +17,7 @@ name: LoadTests Python GBK Dataflow Streaming on: schedule: - - cron: '10 4 * * *' + - cron: '10 16 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_LoadTests_Python_GBK_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Python_GBK_Flink_Batch.yml index cba71c0bd526..29101355fa3e 100644 --- a/.github/workflows/beam_LoadTests_Python_GBK_Flink_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_GBK_Flink_Batch.yml @@ -19,7 +19,7 @@ on: # issue_comment: # types: [created] # schedule: - # - cron: '1 1 * * *' + # - cron: '10 16 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Batch.yml index febc54c0c23e..97afaa800576 100644 --- a/.github/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Batch.yml @@ -19,7 +19,7 @@ on: issue_comment: types: [created] schedule: - - cron: '10 1 * * *' + - cron: '10 16 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Streaming.yml index a904c74d24b4..92df9e3ab924 100644 --- a/.github/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Streaming.yml @@ -19,7 +19,7 @@ on: issue_comment: types: [created] schedule: - - cron: '10 5 * * *' + - cron: '10 16 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Batch.yml index c16bafd34b2c..e2da23682346 100644 --- a/.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Batch.yml @@ -19,7 +19,7 @@ on: issue_comment: types: [created] schedule: - - cron: '30 13 * * *' + - cron: '10 16 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml index 9d4bcdba6092..23b5e69f45bf 100644 --- a/.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml @@ -19,7 +19,7 @@ on: issue_comment: types: [created] schedule: - - cron: '30 16 * * *' + - cron: '50 16 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml index 7a450fadbdb2..029194b040d4 100644 --- a/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml @@ -19,7 +19,7 @@ on: issue_comment: types: [created] schedule: - - cron: '10 14 * * *' + - cron: '50 16 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml b/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml index 37824b28ce0c..85f641bed4d2 100644 --- a/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml @@ -19,7 +19,7 @@ on: issue_comment: types: [created] schedule: - - cron: '15 15 * * *' + - cron: '50 16 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_LoadTests_Python_SideInput_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Python_SideInput_Dataflow_Batch.yml index 91e4c6a10514..15d2aa6f5c14 100644 --- a/.github/workflows/beam_LoadTests_Python_SideInput_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_SideInput_Dataflow_Batch.yml @@ -19,7 +19,7 @@ on: issue_comment: types: [created] schedule: - - cron: '30 13 * * *' + - cron: '50 16 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PerformanceTests_AvroIOIT.yml b/.github/workflows/beam_PerformanceTests_AvroIOIT.yml index cc3799cd59d3..8257eea64050 100644 --- a/.github/workflows/beam_PerformanceTests_AvroIOIT.yml +++ b/.github/workflows/beam_PerformanceTests_AvroIOIT.yml @@ -17,7 +17,7 @@ name: Performance Tests AvroIOIT on: schedule: - - cron: '10 1/13 * * *' + - cron: '10 9/12 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml b/.github/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml index 3bf97098213e..6ff0ae616b9d 100644 --- a/.github/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml +++ b/.github/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml @@ -17,7 +17,7 @@ name: Performance Tests AvroIOIT HDFS on: schedule: - - cron: '10 1/13 * * *' + - cron: '10 9/12 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml b/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml index a79c35704a8f..35e231313b73 100644 --- a/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml +++ b/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml @@ -17,7 +17,7 @@ name: Performance Tests BigQueryIO Batch Java Avro on: schedule: - - cron: '10 1,13 * * *' + - cron: '10 9/12 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml b/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml index d59b94253991..f4f40917f439 100644 --- a/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml +++ b/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml @@ -17,7 +17,7 @@ name: Performance Tests BigQueryIO Batch Java Json on: schedule: - - cron: '30 8,20 * * *' + - cron: '10 9/12 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml b/.github/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml index 775fe5abe938..4fea44cc99ee 100644 --- a/.github/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml +++ b/.github/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml @@ -17,7 +17,7 @@ name: Performance Tests BigQueryIO Streaming Java on: schedule: - - cron: '20 15,22 * * *' + - cron: '50 9/12 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml b/.github/workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml index c51e9c7d7442..176e619ff45f 100644 --- a/.github/workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml +++ b/.github/workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml @@ -17,7 +17,7 @@ name: PerformanceTests BiqQueryIO Read Python on: schedule: - - cron: '0 2 * * *' + - cron: '30 9 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml b/.github/workflows/beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml index c1d25b21e14e..86c3d972560f 100644 --- a/.github/workflows/beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml +++ b/.github/workflows/beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml @@ -17,7 +17,7 @@ name: PerformanceTests BiqQueryIO Write Python Batch on: schedule: - - cron: '0 1 * * *' + - cron: '30 9 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PerformanceTests_Cdap.yml b/.github/workflows/beam_PerformanceTests_Cdap.yml index d8cd8f8fdbe5..1b4c2086834f 100644 --- a/.github/workflows/beam_PerformanceTests_Cdap.yml +++ b/.github/workflows/beam_PerformanceTests_Cdap.yml @@ -17,7 +17,7 @@ name: PerformanceTests Cdap on: schedule: - - cron: '13 4/12 * * *' + - cron: '50 9/12 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml b/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml index cb1b77c84db4..7705cd132e95 100644 --- a/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml +++ b/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml @@ -17,7 +17,7 @@ name: PerformanceTests Compressed TextIOIT on: schedule: - - cron: '10 1/12 * * *' + - cron: '50 9/12 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml b/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml index bea11acf259a..daecc89ef85a 100644 --- a/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml +++ b/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml @@ -17,7 +17,7 @@ name: PerformanceTests Compressed TextIOIT HDFS on: schedule: - - cron: '50 1/12 * * *' + - cron: '50 9/12 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PerformanceTests_HadoopFormat.yml b/.github/workflows/beam_PerformanceTests_HadoopFormat.yml index b65387a857ee..ee42cc207e6f 100644 --- a/.github/workflows/beam_PerformanceTests_HadoopFormat.yml +++ b/.github/workflows/beam_PerformanceTests_HadoopFormat.yml @@ -17,7 +17,7 @@ name: PerformanceTests HadoopFormat on: schedule: - - cron: '16 7/12 * * *' + - cron: '10 10/12 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PerformanceTests_JDBC.yml b/.github/workflows/beam_PerformanceTests_JDBC.yml index 897fc461fd19..1fb6b7f74bb9 100644 --- a/.github/workflows/beam_PerformanceTests_JDBC.yml +++ b/.github/workflows/beam_PerformanceTests_JDBC.yml @@ -17,7 +17,7 @@ name: PerformanceTests JDBC on: schedule: - - cron: '30 1,13 * * *' + - cron: '10 10/12 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PerformanceTests_Kafka_IO.yml b/.github/workflows/beam_PerformanceTests_Kafka_IO.yml index 149be8587237..314297c53ef0 100644 --- a/.github/workflows/beam_PerformanceTests_Kafka_IO.yml +++ b/.github/workflows/beam_PerformanceTests_Kafka_IO.yml @@ -17,7 +17,7 @@ name: PerformanceTests Kafka IO on: schedule: - - cron: '30 2,14 * * *' + - cron: '10 10/12 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml b/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml index 0736b9d55b0c..f077a7c96f85 100644 --- a/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml +++ b/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml @@ -17,7 +17,7 @@ name: PerformanceTests ManyFiles TextIOIT on: schedule: - - cron: '10 2/12 * * *' + - cron: '10 10/12 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml b/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml index 90a6dbdd76df..748d5f30996b 100644 --- a/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml +++ b/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml @@ -17,7 +17,7 @@ name: PerformanceTests ManyFiles TextIOIT HDFS on: schedule: - - cron: '50 2/12 * * *' + - cron: '10 10/12 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PerformanceTests_MongoDBIO_IT.yml b/.github/workflows/beam_PerformanceTests_MongoDBIO_IT.yml index 187706e6c071..4bc36024f59b 100644 --- a/.github/workflows/beam_PerformanceTests_MongoDBIO_IT.yml +++ b/.github/workflows/beam_PerformanceTests_MongoDBIO_IT.yml @@ -17,7 +17,7 @@ name: PerformanceTests MongoDBIO IT on: schedule: - - cron: '14 5/12 * * *' + - cron: '50 10/12 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PerformanceTests_ParquetIOIT.yml b/.github/workflows/beam_PerformanceTests_ParquetIOIT.yml index df0f9137acf2..799acfc4f7f2 100644 --- a/.github/workflows/beam_PerformanceTests_ParquetIOIT.yml +++ b/.github/workflows/beam_PerformanceTests_ParquetIOIT.yml @@ -17,7 +17,7 @@ name: PerformanceTests ParquetIOIT on: schedule: - - cron: '10 3/12 * * *' + - cron: '50 10/12 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml b/.github/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml index 2cd7ac021df9..41f1d2135217 100644 --- a/.github/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml +++ b/.github/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml @@ -17,7 +17,7 @@ name: PerformanceTests ParquetIOIT HDFS on: schedule: - - cron: '50 3/12 * * *' + - cron: '50 10/12 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml b/.github/workflows/beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml index 4f652caa74c1..c252e716830c 100644 --- a/.github/workflows/beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml +++ b/.github/workflows/beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml @@ -17,7 +17,7 @@ name: PerformanceTests PubsubIOIT Python Streaming on: schedule: - - cron: '11 2 * * *' + - cron: '30 10 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml b/.github/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml index 370e038a6bca..0623c3717215 100644 --- a/.github/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml +++ b/.github/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml @@ -17,7 +17,7 @@ name: PerformanceTests SQLBigQueryIO Batch Java on: schedule: - - cron: '0 7,19 * * *' + - cron: '10 11/12 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml b/.github/workflows/beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml index 4a951e720e37..2a4a3ff18e39 100644 --- a/.github/workflows/beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml +++ b/.github/workflows/beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml @@ -17,7 +17,7 @@ name: PerformanceTests SpannerIO Read 2GB Python on: schedule: - - cron: '30 4 * * *' + - cron: '30 10 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml b/.github/workflows/beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml index 17d4b1cb9072..833d7efac926 100644 --- a/.github/workflows/beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml +++ b/.github/workflows/beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml @@ -17,7 +17,7 @@ name: PerformanceTests SpannerIO Write 2GB Python Batch on: schedule: - - cron: '0 5 * * *' + - cron: '30 11 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PerformanceTests_SparkReceiver_IO.yml b/.github/workflows/beam_PerformanceTests_SparkReceiver_IO.yml index 5a8a26ca1f61..6c75c93e8426 100644 --- a/.github/workflows/beam_PerformanceTests_SparkReceiver_IO.yml +++ b/.github/workflows/beam_PerformanceTests_SparkReceiver_IO.yml @@ -17,7 +17,7 @@ name: PerformanceTests SparkReceiver IO on: schedule: - - cron: '15 6/12 * * *' + - cron: '10 11/12 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PerformanceTests_TFRecordIOIT.yml b/.github/workflows/beam_PerformanceTests_TFRecordIOIT.yml index 8c1f89b999a5..65c58c87ddbd 100644 --- a/.github/workflows/beam_PerformanceTests_TFRecordIOIT.yml +++ b/.github/workflows/beam_PerformanceTests_TFRecordIOIT.yml @@ -17,7 +17,7 @@ name: PerformanceTests TFRecordIOIT on: schedule: - - cron: '10 4/12 * * *' + - cron: '50 11/12 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml b/.github/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml index 0e0da80e62a8..0cc337901211 100644 --- a/.github/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml +++ b/.github/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml @@ -19,7 +19,7 @@ on: # TODO(https://github.com/apache/beam/issues/18796) TFRecord performance test is failing only when running on hdfs. # We need to fix this before enabling this job on jenkins. # schedule: - # - cron: '17 8/20 * * *' + # - cron: '50 11/12 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PerformanceTests_TextIOIT.yml b/.github/workflows/beam_PerformanceTests_TextIOIT.yml index 32bb4821208c..b1b0338ba92f 100644 --- a/.github/workflows/beam_PerformanceTests_TextIOIT.yml +++ b/.github/workflows/beam_PerformanceTests_TextIOIT.yml @@ -17,7 +17,7 @@ name: PerformanceTests TextIOIT on: schedule: - - cron: '0 7,19 * * *' + - cron: '10 11/12 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml b/.github/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml index 94cda3343d5e..666d049cb4ee 100644 --- a/.github/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml +++ b/.github/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml @@ -17,7 +17,7 @@ name: PerformanceTests TextIOIT HDFS on: schedule: - - cron: '30 7,19 * * *' + - cron: '10 11/12 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PerformanceTests_TextIOIT_Python.yml b/.github/workflows/beam_PerformanceTests_TextIOIT_Python.yml index f49a573c3d91..fa612f8b2401 100644 --- a/.github/workflows/beam_PerformanceTests_TextIOIT_Python.yml +++ b/.github/workflows/beam_PerformanceTests_TextIOIT_Python.yml @@ -17,7 +17,7 @@ name: PerformanceTests TextIOIT Python on: schedule: - - cron: '0 8,20 * * *' + - cron: '30 11 * * *' workflow_dispatch: # Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml b/.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml index 9e901cc81926..65d7c564da07 100644 --- a/.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml +++ b/.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml @@ -17,7 +17,7 @@ name: PerformanceTests WordCountIT PythonVersions on: schedule: - - cron: '12 3 * * *' + - cron: '50 11/12 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PerformanceTests_XmlIOIT.yml b/.github/workflows/beam_PerformanceTests_XmlIOIT.yml index 43451bc095eb..0423b7ba1a71 100644 --- a/.github/workflows/beam_PerformanceTests_XmlIOIT.yml +++ b/.github/workflows/beam_PerformanceTests_XmlIOIT.yml @@ -17,7 +17,7 @@ name: PerformanceTests XmlIOIT on: schedule: - - cron: '30 4/12 * * *' + - cron: '50 11/12 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml b/.github/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml index 32ffbd63cc79..0197e8cb9b5f 100644 --- a/.github/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml +++ b/.github/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml @@ -17,7 +17,7 @@ name: PerformanceTests XmlIOIT HDFS on: schedule: - - cron: '50 4/12 * * *' + - cron: '50 11/12 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml b/.github/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml index 4d830223a5fa..aaffa3ca1dc1 100644 --- a/.github/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml +++ b/.github/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml @@ -17,7 +17,7 @@ name: PerformanceTests xlang KafkaIO Python on: schedule: - - cron: '10 5 * * *' + - cron: '30 11 * * *' workflow_dispatch: # Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Go.yml b/.github/workflows/beam_PostCommit_Go.yml index e373d4f623bc..27d101275fbd 100644 --- a/.github/workflows/beam_PostCommit_Go.yml +++ b/.github/workflows/beam_PostCommit_Go.yml @@ -17,7 +17,7 @@ name: PostCommit Go on: schedule: - - cron: '0 */6 * * *' + - cron: '30 3/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Go_Dataflow_ARM.yml b/.github/workflows/beam_PostCommit_Go_Dataflow_ARM.yml index 4a2f00a64668..83eb29497426 100644 --- a/.github/workflows/beam_PostCommit_Go_Dataflow_ARM.yml +++ b/.github/workflows/beam_PostCommit_Go_Dataflow_ARM.yml @@ -23,7 +23,7 @@ on: issue_comment: types: [created] schedule: - - cron: '0 */6 * * *' + - cron: '30 3/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Go_VR_Flink.yml b/.github/workflows/beam_PostCommit_Go_VR_Flink.yml index 6f43cd461f25..83393343d889 100644 --- a/.github/workflows/beam_PostCommit_Go_VR_Flink.yml +++ b/.github/workflows/beam_PostCommit_Go_VR_Flink.yml @@ -17,7 +17,7 @@ name: PostCommit Go VR Flink on: schedule: - - cron: '0 */6 * * *' + - cron: '30 3/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Go_VR_Samza.yml b/.github/workflows/beam_PostCommit_Go_VR_Samza.yml index f317615914ce..372e357dd06c 100644 --- a/.github/workflows/beam_PostCommit_Go_VR_Samza.yml +++ b/.github/workflows/beam_PostCommit_Go_VR_Samza.yml @@ -17,7 +17,7 @@ name: PostCommit Go VR Samza on: schedule: - - cron: '0 */6 * * *' + - cron: '30 3/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Go_VR_Spark.yml b/.github/workflows/beam_PostCommit_Go_VR_Spark.yml index 98abb1004b91..b902919006ad 100644 --- a/.github/workflows/beam_PostCommit_Go_VR_Spark.yml +++ b/.github/workflows/beam_PostCommit_Go_VR_Spark.yml @@ -17,7 +17,7 @@ name: PostCommit Go VR Spark on: schedule: - - cron: '0 */6 * * *' + - cron: '30 3/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java.yml b/.github/workflows/beam_PostCommit_Java.yml index fb029e8402fa..3f56b2c4d85a 100644 --- a/.github/workflows/beam_PostCommit_Java.yml +++ b/.github/workflows/beam_PostCommit_Java.yml @@ -19,7 +19,7 @@ name: PostCommit Java on: schedule: - - cron: '0 */6 * * *' + - cron: '0 5/6 * * *' workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_Avro_Versions.yml b/.github/workflows/beam_PostCommit_Java_Avro_Versions.yml index ef451b85f728..02f309c2f739 100644 --- a/.github/workflows/beam_PostCommit_Java_Avro_Versions.yml +++ b/.github/workflows/beam_PostCommit_Java_Avro_Versions.yml @@ -19,7 +19,7 @@ name: PostCommit Java Avro Versions on: schedule: - - cron: '0 */6 * * *' + - cron: '30 3/6 * * *' workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml b/.github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml index b281e550e562..5edfe37320e2 100644 --- a/.github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml +++ b/.github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml @@ -19,7 +19,7 @@ name: PostCommit Java BigQueryEarlyRollout on: schedule: - - cron: '0 */6 * * *' + - cron: '30 3/6 * * *' workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_DataflowV1.yml b/.github/workflows/beam_PostCommit_Java_DataflowV1.yml index 7515da1cad7b..53c0b0b1db39 100644 --- a/.github/workflows/beam_PostCommit_Java_DataflowV1.yml +++ b/.github/workflows/beam_PostCommit_Java_DataflowV1.yml @@ -19,7 +19,7 @@ name: PostCommit Java Dataflow V1 on: schedule: - - cron: '0 */6 * * *' + - cron: '30 3/6 * * *' workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_DataflowV2.yml b/.github/workflows/beam_PostCommit_Java_DataflowV2.yml index 2360feeb7d3f..6c8d9aa45153 100644 --- a/.github/workflows/beam_PostCommit_Java_DataflowV2.yml +++ b/.github/workflows/beam_PostCommit_Java_DataflowV2.yml @@ -19,7 +19,7 @@ name: PostCommit Java Dataflow V2 on: schedule: - - cron: '0 */6 * * *' + - cron: '30 3/6 * * *' workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow.yml index a1e2a0089029..03e94979c51f 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow.yml @@ -21,7 +21,7 @@ on: issue_comment: types: [created] schedule: - - cron: '0 */6 * * *' + - cron: '45 3/6 * * *' workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml index c4d414fc32e2..c401c27922fb 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml @@ -32,7 +32,7 @@ on: issue_comment: types: [created] schedule: - - cron: '0 */6 * * *' + - cron: '45 3/6 * * *' workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml index f9a87b26ab83..6b92c188c985 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml @@ -19,7 +19,7 @@ name: PostCommit Java Examples Dataflow Java on: schedule: - - cron: '0 */6 * * *' + - cron: '45 3/6 * * *' workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml index 28f6ae4d2053..1f817a3acf0a 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml @@ -19,7 +19,7 @@ name: PostCommit Java Examples Dataflow V2 on: schedule: - - cron: '0 */6 * * *' + - cron: '45 3/6 * * *' workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml index 16f48df03292..13506595fe1a 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml @@ -19,7 +19,7 @@ name: PostCommit Java Examples Dataflow V2 Java on: schedule: - - cron: '0 */6 * * *' + - cron: '45 3/6 * * *' workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Direct.yml b/.github/workflows/beam_PostCommit_Java_Examples_Direct.yml index 329093731273..b0f350809d2d 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Direct.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Direct.yml @@ -19,7 +19,7 @@ name: PostCommit Java Examples Direct on: schedule: - - cron: '0 */6 * * *' + - cron: '45 3/6 * * *' workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Flink.yml b/.github/workflows/beam_PostCommit_Java_Examples_Flink.yml index db368031d9f0..a86160e9d700 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Flink.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Flink.yml @@ -19,7 +19,7 @@ name: PostCommit Java Examples Flink on: schedule: - - cron: '0 */6 * * *' + - cron: '45 3/6 * * *' workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Spark.yml b/.github/workflows/beam_PostCommit_Java_Examples_Spark.yml index f7d12a9860da..757008684524 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Spark.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Spark.yml @@ -19,7 +19,7 @@ name: PostCommit Java Examples Spark on: schedule: - - cron: '0 */6 * * *' + - cron: '45 3/6 * * *' workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_Hadoop_Versions.yml b/.github/workflows/beam_PostCommit_Java_Hadoop_Versions.yml index 65ade4bf63d4..29e975628275 100644 --- a/.github/workflows/beam_PostCommit_Java_Hadoop_Versions.yml +++ b/.github/workflows/beam_PostCommit_Java_Hadoop_Versions.yml @@ -17,7 +17,7 @@ name: PostCommit Java Hadoop Versions on: schedule: - - cron: '0 */6 * * *' + - cron: '45 3/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_IO_Performance_Tests.yml b/.github/workflows/beam_PostCommit_Java_IO_Performance_Tests.yml index 649e48fb889d..8775c9643bd3 100644 --- a/.github/workflows/beam_PostCommit_Java_IO_Performance_Tests.yml +++ b/.github/workflows/beam_PostCommit_Java_IO_Performance_Tests.yml @@ -23,7 +23,7 @@ on: issue_comment: types: [created] schedule: - - cron: '0 */6 * * *' + - cron: '0 4/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml b/.github/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml index 8d834a247e6b..fe0cfaf578ec 100644 --- a/.github/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml +++ b/.github/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml @@ -19,7 +19,7 @@ name: Java InfluxDbIO Integration Test on: schedule: - - cron: '0 */6 * * *' + - cron: '0 4/6 * * *' workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml index 4c52984bcaa3..1fd46b95e087 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml @@ -17,7 +17,7 @@ name: PostCommit Java Jpms Dataflow Java11 on: schedule: - - cron: '0 */6 * * *' + - cron: '0 4/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml index 404292a83402..69b5fc23e07a 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml @@ -17,7 +17,7 @@ name: PostCommit Java Jpms Dataflow Java17 on: schedule: - - cron: '0 */6 * * *' + - cron: '0 4/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml index 5046430d31b6..1dec7b5f54f5 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml @@ -17,7 +17,7 @@ name: PostCommit Java Jpms Direct Java11 on: schedule: - - cron: '0 */6 * * *' + - cron: '0 4/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml index 8c4a78aea663..946e34675385 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml @@ -17,7 +17,7 @@ name: PostCommit Java Jpms Direct Java17 on: schedule: - - cron: '0 */6 * * *' + - cron: '0 4/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml index 2dcf6a4444f5..62656517038f 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml @@ -17,7 +17,7 @@ name: PostCommit Java Jpms Flink Java11 on: schedule: - - cron: '0 */6 * * *' + - cron: '0 4/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml index 01680a55026a..745bec641c24 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml @@ -17,7 +17,7 @@ name: PostCommit Java Jpms Spark Java11 on: schedule: - - cron: '0 */6 * * *' + - cron: '0 4/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml index 99e5fb8e677b..8a3ff9a9bb1c 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml @@ -19,7 +19,7 @@ name: PostCommit Java Nexmark Dataflow on: schedule: - - cron: '0 */6 * * *' + - cron: '15 4/6 * * *' workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml index aba92fd8bb07..428ea231e777 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml @@ -19,7 +19,7 @@ name: PostCommit Java Nexmark Dataflow V2 on: schedule: - - cron: '0 */6 * * *' + - cron: '15 4/6 * * *' workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml index 72e1f736e917..2bd36200b217 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml @@ -19,7 +19,7 @@ name: PostCommit Java Nexmark Dataflow V2 Java on: schedule: - - cron: '0 */6 * * *' + - cron: '0 4/6 * * *' workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Direct.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Direct.yml index c2e7433f0478..a1842c73fe53 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Direct.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Direct.yml @@ -19,7 +19,7 @@ name: PostCommit Java Nexmark Direct on: schedule: - - cron: '0 */6 * * *' + - cron: '15 4/6 * * *' workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Flink.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Flink.yml index e80c5a5b2802..76278de19ff2 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Flink.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Flink.yml @@ -19,7 +19,7 @@ name: PostCommit Java Nexmark Flink on: schedule: - - cron: '0 */6 * * *' + - cron: '15 4/6 * * *' workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Spark.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Spark.yml index 0ccd751ac980..110cb20c90cf 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Spark.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Spark.yml @@ -19,7 +19,7 @@ name: PostCommit Java Nexmark Spark on: schedule: - - cron: '0 */6 * * *' + - cron: '15 4/6 * * *' workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml b/.github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml index 209fb2acf23a..021378596e73 100644 --- a/.github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml +++ b/.github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml @@ -19,7 +19,7 @@ name: PostCommit Java PVR Flink Streaming on: schedule: - - cron: '0 */6 * * *' + - cron: '15 4/6 * * *' workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_PVR_Samza.yml b/.github/workflows/beam_PostCommit_Java_PVR_Samza.yml index f6f0140a6ec7..cf87530639c3 100644 --- a/.github/workflows/beam_PostCommit_Java_PVR_Samza.yml +++ b/.github/workflows/beam_PostCommit_Java_PVR_Samza.yml @@ -19,7 +19,7 @@ name: PostCommit Java PVR Samza on: schedule: - - cron: '0 */6 * * *' + - cron: '15 4/6 * * *' workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml b/.github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml index 4fff363d989a..79eadd753562 100644 --- a/.github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml +++ b/.github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml @@ -19,7 +19,7 @@ name: PostCommit Java PVR Spark3 Streaming on: schedule: - - cron: '0 */6 * * *' + - cron: '15 4/6 * * *' workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml b/.github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml index d4db6f01518e..d7461e3236be 100644 --- a/.github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml +++ b/.github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml @@ -19,7 +19,7 @@ name: PostCommit Java PVR Spark Batch on: schedule: - - cron: '0 */6 * * *' + - cron: '15 4/6 * * *' workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_Sickbay.yml b/.github/workflows/beam_PostCommit_Java_Sickbay.yml index 3478e8902e08..18ef48b92f70 100644 --- a/.github/workflows/beam_PostCommit_Java_Sickbay.yml +++ b/.github/workflows/beam_PostCommit_Java_Sickbay.yml @@ -19,7 +19,7 @@ name: PostCommit Java Sickbay on: schedule: - - cron: '0 0 * * *' + - cron: '30 4/6 * * *' workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml b/.github/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml index 43b7d0bb1d81..5af333e645b0 100644 --- a/.github/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml +++ b/.github/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml @@ -19,7 +19,7 @@ name: PostCommit Java SingleStoreIO IT on: schedule: - - cron: '0 */23 * * *' + - cron: '30 4/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event permissions: diff --git a/.github/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml b/.github/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml index 9cb67bbdf3c2..d6b63752eede 100644 --- a/.github/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml @@ -17,7 +17,7 @@ name: PostCommit Java Tpcds Dataflow on: schedule: - - cron: '0 */6 * * *' + - cron: '30 4/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_Tpcds_Flink.yml b/.github/workflows/beam_PostCommit_Java_Tpcds_Flink.yml index 6864576f651e..d2ba0cafc01f 100644 --- a/.github/workflows/beam_PostCommit_Java_Tpcds_Flink.yml +++ b/.github/workflows/beam_PostCommit_Java_Tpcds_Flink.yml @@ -17,7 +17,7 @@ name: PostCommit Java Tpcds Flink on: schedule: - - cron: '0 */6 * * *' + - cron: '30 4/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_Tpcds_Spark.yml b/.github/workflows/beam_PostCommit_Java_Tpcds_Spark.yml index be4aef55f714..d93f0e1a8b52 100644 --- a/.github/workflows/beam_PostCommit_Java_Tpcds_Spark.yml +++ b/.github/workflows/beam_PostCommit_Java_Tpcds_Spark.yml @@ -17,7 +17,7 @@ name: PostCommit Java Tpcds Spark on: schedule: - - cron: '0 */6 * * *' + - cron: '30 4/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml index 8d8609ad96d6..bd7f29720fa8 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml @@ -19,7 +19,7 @@ name: PostCommit Java ValidatesRunner Dataflow on: schedule: - - cron: '0 */6 * * *' + - cron: '30 4/6 * * *' workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml index 5b151e535f97..67bc4e6a3994 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml @@ -19,7 +19,7 @@ name: PostCommit Java ValidatesRunner Dataflow JavaVersions on: schedule: - - cron: '0 */6 * * *' + - cron: '30 4/6 * * *' workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml index 2a61431c81af..116f8f6e4a20 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml @@ -19,7 +19,7 @@ name: PostCommit Java ValidatesRunner Dataflow Streaming on: schedule: - - cron: '0 */6 * * *' + - cron: '30 4/6 * * *' workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml index 4d3664618d9e..55064692a718 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml @@ -19,7 +19,7 @@ name: PostCommit Java ValidatesRunner Dataflow V2 on: schedule: - - cron: '0 */8 * * *' + - cron: '30 6/8 * * *' workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml index 10725d6a72d9..8314db686ccd 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml @@ -19,7 +19,7 @@ name: PostCommit Java ValidatesRunner Dataflow V2 Streaming on: schedule: - - cron: '0 */8 * * *' + - cron: '30 6/8 * * *' workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml index 5bf14340e0d1..af7a4379ad5e 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml @@ -19,7 +19,7 @@ name: PostCommit Java ValidatesRunner Direct on: schedule: - - cron: '0 */6 * * *' + - cron: '45 4/6 * * *' workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml index e71c47266eac..c9da0d7bd604 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml @@ -19,7 +19,7 @@ name: PostCommit Java ValidatesRunner Direct JavaVersions on: schedule: - - cron: '0 */6 * * *' + - cron: '30 4/6 * * *' workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml index 6df33f411b8d..7202b6fa33ae 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml @@ -17,7 +17,7 @@ name: PostCommit Java ValidatesRunner Flink on: schedule: - - cron: '0 */6 * * *' + - cron: '45 4/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml index eaeef09bb781..2f2e4f15abe6 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml @@ -19,7 +19,7 @@ name: PostCommit Java ValidatesRunner Flink Java11 on: schedule: - - cron: '0 */6 * * *' + - cron: '45 4/6 * * *' workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml index cc68b3c7cc67..b577b584f0ad 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml @@ -17,7 +17,7 @@ name: PostCommit Java ValidatesRunner Samza on: schedule: - - cron: '0 */6 * * *' + - cron: '45 4/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml index b1c0395dc87a..3465ff953da7 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml @@ -17,7 +17,7 @@ name: PostCommit Java ValidatesRunner Spark on: schedule: - - cron: '0 */6 * * *' + - cron: '45 4/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml index 2047e5a448a2..6df7d3ac35e9 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml @@ -17,7 +17,7 @@ name: PostCommit Java ValidatesRunner SparkStructuredStreaming on: schedule: - - cron: '0 */6 * * *' + - cron: '45 4/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml index 3456f24594a6..cc9f51113124 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml @@ -19,7 +19,7 @@ name: PostCommit Java ValidatesRunner Spark Java11 on: schedule: - - cron: '0 */6 * * *' + - cron: '45 4/6 * * *' workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml index 72ad389087f1..3bcaff465f1d 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml @@ -17,7 +17,7 @@ name: PostCommit Java ValidatesRunner Twister2 on: schedule: - - cron: '0 */6 * * *' + - cron: '45 4/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml index 3296edfd3e62..9ce3655c6fc9 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml @@ -17,7 +17,7 @@ name: PostCommit Java ValidatesRunner ULR on: schedule: - - cron: '0 */6 * * *' + - cron: '45 4/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Javadoc.yml b/.github/workflows/beam_PostCommit_Javadoc.yml index deab0f879f52..a70b37c10a9e 100644 --- a/.github/workflows/beam_PostCommit_Javadoc.yml +++ b/.github/workflows/beam_PostCommit_Javadoc.yml @@ -19,7 +19,7 @@ name: PostCommit Javadoc on: schedule: - - cron: '0 */6 * * *' + - cron: '0 5/6 * * *' workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_PortableJar_Flink.yml b/.github/workflows/beam_PostCommit_PortableJar_Flink.yml index 854371af6ef7..ef27d1601e2b 100644 --- a/.github/workflows/beam_PostCommit_PortableJar_Flink.yml +++ b/.github/workflows/beam_PostCommit_PortableJar_Flink.yml @@ -17,7 +17,7 @@ name: PostCommit PortableJar Flink on: schedule: - - cron: '0 */6 * * *' + - cron: '0 5/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_PortableJar_Spark.yml b/.github/workflows/beam_PostCommit_PortableJar_Spark.yml index fbafbbda4608..a84384bdb865 100644 --- a/.github/workflows/beam_PostCommit_PortableJar_Spark.yml +++ b/.github/workflows/beam_PostCommit_PortableJar_Spark.yml @@ -17,7 +17,7 @@ name: PostCommit PortableJar Spark on: schedule: - - cron: '0 */6 * * *' + - cron: '0 5/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Python.yml b/.github/workflows/beam_PostCommit_Python.yml index e6e5ebd93d30..ffd2979c3a92 100644 --- a/.github/workflows/beam_PostCommit_Python.yml +++ b/.github/workflows/beam_PostCommit_Python.yml @@ -19,7 +19,7 @@ name: PostCommit Python on: schedule: - - cron: '0 */6 * * *' + - cron: '30 5/6 * * *' workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Python_Arm.yml b/.github/workflows/beam_PostCommit_Python_Arm.yml index f3d2b2463ef2..cf41ed19d256 100644 --- a/.github/workflows/beam_PostCommit_Python_Arm.yml +++ b/.github/workflows/beam_PostCommit_Python_Arm.yml @@ -21,7 +21,7 @@ on: issue_comment: types: [created] schedule: - - cron: '0 */6 * * *' + - cron: '0 5/6 * * *' workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Python_Examples_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_Examples_Dataflow.yml index 941ee9994436..61fbb1f80e71 100644 --- a/.github/workflows/beam_PostCommit_Python_Examples_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_Examples_Dataflow.yml @@ -17,7 +17,7 @@ name: PostCommit Python Examples Dataflow on: schedule: - - cron: '0 */6 * * *' + - cron: '0 5/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Python_Examples_Direct.yml b/.github/workflows/beam_PostCommit_Python_Examples_Direct.yml index a54981bde887..c8855b2b1274 100644 --- a/.github/workflows/beam_PostCommit_Python_Examples_Direct.yml +++ b/.github/workflows/beam_PostCommit_Python_Examples_Direct.yml @@ -17,7 +17,7 @@ name: PostCommit Python Examples Direct on: schedule: - - cron: '0 */6 * * *' + - cron: '0 5/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Python_Examples_Flink.yml b/.github/workflows/beam_PostCommit_Python_Examples_Flink.yml index 2cc2ff69aed9..46df55646776 100644 --- a/.github/workflows/beam_PostCommit_Python_Examples_Flink.yml +++ b/.github/workflows/beam_PostCommit_Python_Examples_Flink.yml @@ -17,7 +17,7 @@ name: PostCommit Python Examples Flink on: schedule: - - cron: '0 */6 * * *' + - cron: '0 5/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Python_Examples_Spark.yml b/.github/workflows/beam_PostCommit_Python_Examples_Spark.yml index 4c8b39f5f116..498bc4f747a4 100644 --- a/.github/workflows/beam_PostCommit_Python_Examples_Spark.yml +++ b/.github/workflows/beam_PostCommit_Python_Examples_Spark.yml @@ -17,7 +17,7 @@ name: PostCommit Python Examples Spark on: schedule: - - cron: '0 */6 * * *' + - cron: '0 5/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml b/.github/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml index f58632bad758..d8e9934609fa 100644 --- a/.github/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml +++ b/.github/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml @@ -17,7 +17,7 @@ name: PostCommit Python MongoDBIO IT on: schedule: - - cron: '0 */6 * * *' + - cron: '15 5/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Python_Nexmark_Direct.yml b/.github/workflows/beam_PostCommit_Python_Nexmark_Direct.yml index 1fbe9a47296a..c63bcf2ecad1 100644 --- a/.github/workflows/beam_PostCommit_Python_Nexmark_Direct.yml +++ b/.github/workflows/beam_PostCommit_Python_Nexmark_Direct.yml @@ -19,7 +19,7 @@ name: PostCommit Python Nexmark Direct on: schedule: - - cron: '0 */6 * * *' + - cron: '15 5/6 * * *' workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml index 364f1455e955..1bd547ab3774 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml @@ -17,7 +17,7 @@ name: PostCommit Python ValidatesContainer Dataflow on: schedule: - - cron: '0 */6 * * *' + - cron: '15 5/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml b/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml index 164577228e4f..2c1b1c45227a 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml @@ -17,7 +17,7 @@ name: PostCommit Python ValidatesContainer Dataflow With RC on: schedule: - - cron: '0 */6 * * *' + - cron: '15 5/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml index c9256c538594..b01ea81f9ac5 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml @@ -17,7 +17,7 @@ name: PostCommit Python ValidatesRunner Dataflow on: schedule: - - cron: '0 */6 * * *' + - cron: '15 5/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml index f6e6b3a182ef..79a7550755d1 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml @@ -17,7 +17,7 @@ name: PostCommit Python ValidatesRunner Flink on: schedule: - - cron: '0 */6 * * *' + - cron: '15 5/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml index 805640999a57..300cee6c3520 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml @@ -17,7 +17,7 @@ name: PostCommit Python ValidatesRunner Samza on: schedule: - - cron: '0 */6 * * *' + - cron: '15 5/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml index fb8021030113..c1f55590b8ed 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml @@ -17,7 +17,7 @@ name: PostCommit Python ValidatesRunner Spark on: schedule: - - cron: '0 */6 * * *' + - cron: '15 5/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml index cdfae228a53e..e4023479c32f 100644 --- a/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml @@ -17,7 +17,7 @@ name: PostCommit Python Xlang Gcp Dataflow on: schedule: - - cron: '0 */6 * * *' + - cron: '15 5/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml b/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml index d58098ed3c72..ec734641b6b6 100644 --- a/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml +++ b/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml @@ -17,7 +17,7 @@ name: PostCommit Python Xlang Gcp Direct on: schedule: - - cron: '0 */6 * * *' + - cron: '30 5/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml index c0b8b2c993c4..9a4a30098764 100644 --- a/.github/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml @@ -17,7 +17,7 @@ name: PostCommit Python Xlang IO Dataflow on: schedule: - - cron: '0 */6 * * *' + - cron: '30 5/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_SQL.yml b/.github/workflows/beam_PostCommit_SQL.yml index 776f7ce1210b..1ad8e0567cfd 100644 --- a/.github/workflows/beam_PostCommit_SQL.yml +++ b/.github/workflows/beam_PostCommit_SQL.yml @@ -19,7 +19,7 @@ name: PostCommit SQL on: schedule: - - cron: '0 */6 * * *' + - cron: '30 5/6 * * *' workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Sickbay_Python.yml b/.github/workflows/beam_PostCommit_Sickbay_Python.yml index 99effb751357..5e8b717d0ef9 100644 --- a/.github/workflows/beam_PostCommit_Sickbay_Python.yml +++ b/.github/workflows/beam_PostCommit_Sickbay_Python.yml @@ -19,7 +19,7 @@ name: PostCommit Sickbay Python on: schedule: - - cron: '0 0 * * *' + - cron: '0 8 * * *' workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_TransformService_Direct.yml b/.github/workflows/beam_PostCommit_TransformService_Direct.yml index 4cc1ccf99c22..916cd3b34e75 100644 --- a/.github/workflows/beam_PostCommit_TransformService_Direct.yml +++ b/.github/workflows/beam_PostCommit_TransformService_Direct.yml @@ -17,7 +17,7 @@ name: PostCommit TransformService Direct on: schedule: - - cron: '0 */6 * * *' + - cron: '30 5/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Website_Publish.yml b/.github/workflows/beam_PostCommit_Website_Publish.yml index 834c3d043afe..e5759e390211 100644 --- a/.github/workflows/beam_PostCommit_Website_Publish.yml +++ b/.github/workflows/beam_PostCommit_Website_Publish.yml @@ -17,7 +17,7 @@ name: PostCommit Website Publish on: schedule: - - cron: '0 */6 * * *' + - cron: '30 5/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Website_Test.yml b/.github/workflows/beam_PostCommit_Website_Test.yml index 6f2229ff76ff..a73d971f4b68 100644 --- a/.github/workflows/beam_PostCommit_Website_Test.yml +++ b/.github/workflows/beam_PostCommit_Website_Test.yml @@ -17,7 +17,7 @@ name: PostCommit Website Test on: schedule: - - cron: '0 */6 * * *' + - cron: '30 5/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_XVR_Direct.yml b/.github/workflows/beam_PostCommit_XVR_Direct.yml index 51e36053ea96..8a0c4691b85c 100644 --- a/.github/workflows/beam_PostCommit_XVR_Direct.yml +++ b/.github/workflows/beam_PostCommit_XVR_Direct.yml @@ -17,7 +17,7 @@ name: PostCommit XVR Direct on: schedule: - - cron: '0 */6 * * *' + - cron: '30 5/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_XVR_Flink.yml b/.github/workflows/beam_PostCommit_XVR_Flink.yml index 9df8b1c8e776..3e5a58892755 100644 --- a/.github/workflows/beam_PostCommit_XVR_Flink.yml +++ b/.github/workflows/beam_PostCommit_XVR_Flink.yml @@ -17,7 +17,7 @@ name: PostCommit XVR Flink on: schedule: - - cron: '0 */6 * * *' + - cron: '30 5/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml b/.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml index 439453364e22..64c87c92c5ed 100644 --- a/.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml @@ -17,7 +17,7 @@ name: PostCommit XVR GoUsingJava Dataflow on: schedule: - - cron: '0 */6 * * *' + - cron: '45 5/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml b/.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml index 93e495c5eb15..4f3cf65ce5dd 100644 --- a/.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml @@ -17,7 +17,7 @@ name: PostCommit XVR JavaUsingPython Dataflow on: schedule: - - cron: '0 */6 * * *' + - cron: '45 5/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml b/.github/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml index 8fdade48da41..be4fcb9cec5a 100644 --- a/.github/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml @@ -17,7 +17,7 @@ name: PostCommit XVR PythonUsingJavaSQL Dataflow on: schedule: - - cron: '0 */6 * * *' + - cron: '45 5/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml b/.github/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml index 9284c3698824..0318f732d978 100644 --- a/.github/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml @@ -17,7 +17,7 @@ name: PostCommit XVR PythonUsingJava Dataflow on: schedule: - - cron: '0 */6 * * *' + - cron: '45 5/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_XVR_Samza.yml b/.github/workflows/beam_PostCommit_XVR_Samza.yml index f053f28ac190..486a45159db1 100644 --- a/.github/workflows/beam_PostCommit_XVR_Samza.yml +++ b/.github/workflows/beam_PostCommit_XVR_Samza.yml @@ -17,7 +17,7 @@ name: PostCommit XVR Samza on: schedule: - - cron: '0 */6 * * *' + - cron: '45 5/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_XVR_Spark3.yml b/.github/workflows/beam_PostCommit_XVR_Spark3.yml index 3b6d5bdad551..5524f66c6afc 100644 --- a/.github/workflows/beam_PostCommit_XVR_Spark3.yml +++ b/.github/workflows/beam_PostCommit_XVR_Spark3.yml @@ -17,7 +17,7 @@ name: PostCommit XVR Spark3 on: schedule: - - cron: '0 */6 * * *' + - cron: '45 5/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_CommunityMetrics.yml b/.github/workflows/beam_PreCommit_CommunityMetrics.yml index 9341b1e8877f..d192c6d164d9 100644 --- a/.github/workflows/beam_PreCommit_CommunityMetrics.yml +++ b/.github/workflows/beam_PreCommit_CommunityMetrics.yml @@ -26,7 +26,7 @@ on: issue_comment: types: [created] schedule: - - cron: '0 */6 * * *' + - cron: '0 1/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_Go.yml b/.github/workflows/beam_PreCommit_Go.yml index c202c30d5a4b..127fc9751335 100644 --- a/.github/workflows/beam_PreCommit_Go.yml +++ b/.github/workflows/beam_PreCommit_Go.yml @@ -26,7 +26,7 @@ on: issue_comment: types: [created] schedule: - - cron: '0 */6 * * *' + - cron: '0 1/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_GoPortable.yml b/.github/workflows/beam_PreCommit_GoPortable.yml index 3034285731a5..f89b9e0a204e 100644 --- a/.github/workflows/beam_PreCommit_GoPortable.yml +++ b/.github/workflows/beam_PreCommit_GoPortable.yml @@ -26,7 +26,7 @@ on: issue_comment: types: [created] schedule: - - cron: '0 */6 * * *' + - cron: '0 1/6 * * *' workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PreCommit_GoPrism.yml b/.github/workflows/beam_PreCommit_GoPrism.yml index 9e403d13a2ef..99e117933772 100644 --- a/.github/workflows/beam_PreCommit_GoPrism.yml +++ b/.github/workflows/beam_PreCommit_GoPrism.yml @@ -26,7 +26,7 @@ on: issue_comment: types: [created] schedule: - - cron: '0 */6 * * *' + - cron: '0 1/6 * * *' workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PreCommit_ItFramework.yml b/.github/workflows/beam_PreCommit_ItFramework.yml index ef384def0d5f..b9757eae35d7 100644 --- a/.github/workflows/beam_PreCommit_ItFramework.yml +++ b/.github/workflows/beam_PreCommit_ItFramework.yml @@ -31,7 +31,7 @@ on: issue_comment: types: [created] schedule: - - cron: '10 */6 * * *' + - cron: '0 1/6 * * *' workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PreCommit_Java.yml b/.github/workflows/beam_PreCommit_Java.yml index 0b971462f334..eed88f79fb00 100644 --- a/.github/workflows/beam_PreCommit_Java.yml +++ b/.github/workflows/beam_PreCommit_Java.yml @@ -117,7 +117,7 @@ on: issue_comment: types: [created] schedule: - - cron: '0 */6 * * *' + - cron: '30 2/6 * * *' workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PreCommit_Java_Amazon-Web-Services2_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Amazon-Web-Services2_IO_Direct.yml index 67b27a06462d..689591fb3dcd 100644 --- a/.github/workflows/beam_PreCommit_Java_Amazon-Web-Services2_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Amazon-Web-Services2_IO_Direct.yml @@ -47,7 +47,7 @@ on: issue_comment: types: [created] schedule: - - cron: '0 */6 * * *' + - cron: '0 1/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_Java_Amazon-Web-Services_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Amazon-Web-Services_IO_Direct.yml index c8e56e928cb8..f6d3501194ff 100644 --- a/.github/workflows/beam_PreCommit_Java_Amazon-Web-Services_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Amazon-Web-Services_IO_Direct.yml @@ -47,7 +47,7 @@ on: issue_comment: types: [created] schedule: - - cron: '0 */6 * * *' + - cron: '0 1/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_Java_Amqp_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Amqp_IO_Direct.yml index 050ff91493a0..8fd2a799e916 100644 --- a/.github/workflows/beam_PreCommit_Java_Amqp_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Amqp_IO_Direct.yml @@ -29,7 +29,7 @@ on: issue_comment: types: [created] schedule: - - cron: '0 */6 * * *' + - cron: '15 1/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml index cbd33a766a99..17379856ec9a 100644 --- a/.github/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml @@ -47,7 +47,7 @@ on: issue_comment: types: [created] schedule: - - cron: '0 */6 * * *' + - cron: '15 1/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml index 2b9066c1d27a..d016125f1d20 100644 --- a/.github/workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml @@ -29,7 +29,7 @@ on: issue_comment: types: [created] schedule: - - cron: '0 */6 * * *' + - cron: '15 1/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml index 29447d880dee..e9e1c0ef51b3 100644 --- a/.github/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml @@ -33,7 +33,7 @@ on: issue_comment: types: [created] schedule: - - cron: '0 */6 * * *' + - cron: '15 1/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml index 4086f690af4e..37b2ca0f2a23 100644 --- a/.github/workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml @@ -29,7 +29,7 @@ on: issue_comment: types: [created] schedule: - - cron: '0 */6 * * *' + - cron: '15 1/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml index 878951b97186..9e6f07a278f9 100644 --- a/.github/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml @@ -29,7 +29,7 @@ on: issue_comment: types: [created] schedule: - - cron: '0 */6 * * *' + - cron: '15 1/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml index 281a9ce4a0ad..db07d2bbacf3 100644 --- a/.github/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml @@ -29,7 +29,7 @@ on: issue_comment: types: [created] schedule: - - cron: '0 */6 * * *' + - cron: '15 1/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml index f275d6276e00..93e4f54bdff5 100644 --- a/.github/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml @@ -31,7 +31,7 @@ on: issue_comment: types: [created] schedule: - - cron: '0 */6 * * *' + - cron: '30 1/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow.yml b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow.yml index c68a85a0bc21..fa7c3f0f1771 100644 --- a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow.yml +++ b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow.yml @@ -41,7 +41,7 @@ on: issue_comment: types: [created] schedule: - - cron: '0 */6 * * *' + - cron: '30 1/6 * * *' workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml index 032d13152eba..0e2614f5be73 100644 --- a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml +++ b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml @@ -41,7 +41,7 @@ on: issue_comment: types: [created] schedule: - - cron: '0 */6 * * *' + - cron: '30 1/6 * * *' workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml index 64809809dbd8..ba5d22041f27 100644 --- a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml +++ b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml @@ -39,7 +39,7 @@ on: issue_comment: types: [created] schedule: - - cron: '0 */6 * * *' + - cron: '30 1/6 * * *' workflow_dispatch: # Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_Java_File-schema-transform_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_File-schema-transform_IO_Direct.yml index 2ad4de133228..bd6a573b8141 100644 --- a/.github/workflows/beam_PreCommit_Java_File-schema-transform_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_File-schema-transform_IO_Direct.yml @@ -29,7 +29,7 @@ on: issue_comment: types: [created] schedule: - - cron: '0 */6 * * *' + - cron: '30 1/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_Java_Flink_Versions.yml b/.github/workflows/beam_PreCommit_Java_Flink_Versions.yml index 6590583925e1..e64e5880bcef 100644 --- a/.github/workflows/beam_PreCommit_Java_Flink_Versions.yml +++ b/.github/workflows/beam_PreCommit_Java_Flink_Versions.yml @@ -33,7 +33,7 @@ on: issue_comment: types: [created] schedule: - - cron: '20 */6 * * *' + - cron: '30 1/6 * * *' workflow_dispatch: permissions: diff --git a/.github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml index 0c832aecf749..37417ba0e210 100644 --- a/.github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml @@ -47,7 +47,7 @@ on: issue_comment: types: [created] schedule: - - cron: '0 */6 * * *' + - cron: '30 1/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml index 4e1e300bb925..8ba07fb062ca 100644 --- a/.github/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml @@ -31,7 +31,7 @@ on: issue_comment: types: [created] schedule: - - cron: '0 */6 * * *' + - cron: '45 1/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml index 93eacb82cf2b..6aaf3fa9eb0a 100644 --- a/.github/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml @@ -31,7 +31,7 @@ on: issue_comment: types: [created] schedule: - - cron: '0 */6 * * *' + - cron: '45 1/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml index 2a61bcfb3809..7a9f3cafe355 100644 --- a/.github/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml @@ -55,7 +55,7 @@ on: issue_comment: types: [created] schedule: - - cron: '0 */6 * * *' + - cron: '45 1/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml index 607a202b4494..8a840f92379e 100644 --- a/.github/workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml @@ -29,7 +29,7 @@ on: issue_comment: types: [created] schedule: - - cron: '0 */6 * * *' + - cron: '45 1/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml index eab48ce8a0f3..4d953028f496 100644 --- a/.github/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml @@ -29,7 +29,7 @@ on: issue_comment: types: [created] schedule: - - cron: '0 */6 * * *' + - cron: '45 1/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml index aebe9984ab73..fadebc4063c9 100644 --- a/.github/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml @@ -29,7 +29,7 @@ on: issue_comment: types: [created] schedule: - - cron: '0 */6 * * *' + - cron: '45 1/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml index 8b7b35922d37..3564e1a685df 100644 --- a/.github/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml @@ -37,7 +37,7 @@ on: issue_comment: types: [created] schedule: - - cron: '0 */6 * * *' + - cron: '45 1/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml index be29e3b87495..6da5325fd50f 100644 --- a/.github/workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml @@ -47,7 +47,7 @@ on: issue_comment: types: [created] schedule: - - cron: '0 */6 * * *' + - cron: '0 2/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml index 12cb1b2aa230..a2e4e11c72d8 100644 --- a/.github/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml @@ -29,7 +29,7 @@ on: issue_comment: types: [created] schedule: - - cron: '0 */6 * * *' + - cron: '0 2/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml index 19ca7a24a27a..a7489a597d1e 100644 --- a/.github/workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml @@ -29,7 +29,7 @@ on: issue_comment: types: [created] schedule: - - cron: '0 */6 * * *' + - cron: '0 2/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml index 5e9c8ecf9a48..867496e8e674 100644 --- a/.github/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml @@ -29,7 +29,7 @@ on: issue_comment: types: [created] schedule: - - cron: '0 */6 * * *' + - cron: '0 2/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml index 0dc3f2c946ad..01f03e85edb7 100644 --- a/.github/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml @@ -31,7 +31,7 @@ on: issue_comment: types: [created] schedule: - - cron: '0 */6 * * *' + - cron: '0 2/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_Java_PVR_Flink_Batch.yml b/.github/workflows/beam_PreCommit_Java_PVR_Flink_Batch.yml index 283e0c959d09..792ff47608d6 100644 --- a/.github/workflows/beam_PreCommit_Java_PVR_Flink_Batch.yml +++ b/.github/workflows/beam_PreCommit_Java_PVR_Flink_Batch.yml @@ -35,7 +35,7 @@ on: issue_comment: types: [created] schedule: - - cron: '0 */6 * * *' + - cron: '15 2/6 * * *' workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml b/.github/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml index e41f59f67205..5839a59d8e95 100644 --- a/.github/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml +++ b/.github/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml @@ -39,7 +39,7 @@ on: issue_comment: types: [created] schedule: - - cron: '0 */6 * * *' + - cron: '15 2/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml index a0f021bb82d5..80f6110a1143 100644 --- a/.github/workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml @@ -29,7 +29,7 @@ on: issue_comment: types: [created] schedule: - - cron: '0 */6 * * *' + - cron: '0 2/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml index ef22e19ca514..38cd15be1dd2 100644 --- a/.github/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml @@ -47,7 +47,7 @@ on: issue_comment: types: [created] schedule: - - cron: '0 */6 * * *' + - cron: '0 2/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml index f03b38b18257..64a03f93b0b6 100644 --- a/.github/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml @@ -29,7 +29,7 @@ on: issue_comment: types: [created] schedule: - - cron: '0 */6 * * *' + - cron: '15 2/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml index 21efdf4b574c..18cc54ddffcb 100644 --- a/.github/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml @@ -29,7 +29,7 @@ on: issue_comment: types: [created] schedule: - - cron: '0 */6 * * *' + - cron: '15 2/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml index 4e98c1fee708..c692cd24c979 100644 --- a/.github/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml @@ -31,7 +31,7 @@ on: issue_comment: types: [created] schedule: - - cron: '0 */6 * * *' + - cron: '15 2/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml index 4f7cfb6b8316..383f7b2fe32c 100644 --- a/.github/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml @@ -33,7 +33,7 @@ on: issue_comment: types: [created] schedule: - - cron: '0 */6 * * *' + - cron: '15 2/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml index 4a30cfbf3f40..61350d767b1e 100644 --- a/.github/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml @@ -29,7 +29,7 @@ on: issue_comment: types: [created] schedule: - - cron: '0 */6 * * *' + - cron: '15 2/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_Java_Spark3_Versions.yml b/.github/workflows/beam_PreCommit_Java_Spark3_Versions.yml index 1809f11d8c7c..9ff70bf1d321 100644 --- a/.github/workflows/beam_PreCommit_Java_Spark3_Versions.yml +++ b/.github/workflows/beam_PreCommit_Java_Spark3_Versions.yml @@ -31,7 +31,7 @@ on: issue_comment: types: [created] schedule: - - cron: '0 */6 * * *' + - cron: '30 2/6 * * *' workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml index 786276aeb885..39d159d9e3fc 100644 --- a/.github/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml @@ -29,7 +29,7 @@ on: issue_comment: types: [created] schedule: - - cron: '0 */6 * * *' + - cron: '30 2/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml index 64cd4a755e73..ac9fe2d38c91 100644 --- a/.github/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml @@ -29,7 +29,7 @@ on: issue_comment: types: [created] schedule: - - cron: '0 */6 * * *' + - cron: '30 2/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml index e6bed237a808..a27637da6aae 100644 --- a/.github/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml @@ -29,7 +29,7 @@ on: issue_comment: types: [created] schedule: - - cron: '0 */6 * * *' + - cron: '30 2/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_Kotlin_Examples.yml b/.github/workflows/beam_PreCommit_Kotlin_Examples.yml index 8fe1673d9c55..513487d0bf3b 100644 --- a/.github/workflows/beam_PreCommit_Kotlin_Examples.yml +++ b/.github/workflows/beam_PreCommit_Kotlin_Examples.yml @@ -41,7 +41,7 @@ on: issue_comment: types: [created] schedule: - - cron: '0 */6 * * *' + - cron: '30 2/6 * * *' workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PreCommit_Portable_Python.yml b/.github/workflows/beam_PreCommit_Portable_Python.yml index e8221f7f23f5..85816cc62a59 100644 --- a/.github/workflows/beam_PreCommit_Portable_Python.yml +++ b/.github/workflows/beam_PreCommit_Portable_Python.yml @@ -45,7 +45,7 @@ on: issue_comment: types: [created] schedule: - - cron: '0 */6 * * *' + - cron: '30 2/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_Python.yml b/.github/workflows/beam_PreCommit_Python.yml index d65a8e643eb4..03cba671eae2 100644 --- a/.github/workflows/beam_PreCommit_Python.yml +++ b/.github/workflows/beam_PreCommit_Python.yml @@ -25,7 +25,7 @@ on: branches: ['master', 'release-*'] paths: [ "model/**","sdks/python/**","release/**",".github/workflows/beam_PreCommit_Python.yml"] schedule: - - cron: '0 */6 * * *' + - cron: '0 3/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_PythonDocker.yml b/.github/workflows/beam_PreCommit_PythonDocker.yml index dc5ee73e198d..6578c733818e 100644 --- a/.github/workflows/beam_PreCommit_PythonDocker.yml +++ b/.github/workflows/beam_PreCommit_PythonDocker.yml @@ -25,7 +25,7 @@ on: branches: ['master', 'release-*'] paths: [ "model/**","sdks/python/**","release/**",".github/workflows/beam_PreCommit_PythonDocker.yml"] schedule: - - cron: '0 */6 * * *' + - cron: '0 3/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_PythonDocs.yml b/.github/workflows/beam_PreCommit_PythonDocs.yml index 896d954386b5..0666f13cc747 100644 --- a/.github/workflows/beam_PreCommit_PythonDocs.yml +++ b/.github/workflows/beam_PreCommit_PythonDocs.yml @@ -26,7 +26,7 @@ on: branches: ['master', 'release-*'] paths: ["sdks/python/**",".github/workflows/beam_PreCommit_PythonDocs.yml"] schedule: - - cron: '0 */6 * * *' + - cron: '0 3/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_PythonFormatter.yml b/.github/workflows/beam_PreCommit_PythonFormatter.yml index e9c6787f6ee9..2f22fc0d3d85 100644 --- a/.github/workflows/beam_PreCommit_PythonFormatter.yml +++ b/.github/workflows/beam_PreCommit_PythonFormatter.yml @@ -25,7 +25,7 @@ on: branches: ['master', 'release-*'] paths: [ "sdks/python/apache_beam/**",".github/workflows/beam_PreCommit_PythonFormatter.yml"] schedule: - - cron: '0 */6 * * *' + - cron: '0 3/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_PythonLint.yml b/.github/workflows/beam_PreCommit_PythonLint.yml index 2a1f186f2418..f24dcb4fa671 100644 --- a/.github/workflows/beam_PreCommit_PythonLint.yml +++ b/.github/workflows/beam_PreCommit_PythonLint.yml @@ -25,7 +25,7 @@ on: branches: ['master', 'release-*'] paths: ["sdks/python/**","release/**",".github/workflows/beam_PreCommit_PythonLint.yml"] schedule: - - cron: '0 */6 * * *' + - cron: '0 3/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_Python_Coverage.yml b/.github/workflows/beam_PreCommit_Python_Coverage.yml index ef8f9ee1bf1d..adabcbb47529 100644 --- a/.github/workflows/beam_PreCommit_Python_Coverage.yml +++ b/.github/workflows/beam_PreCommit_Python_Coverage.yml @@ -25,7 +25,7 @@ on: branches: ['master', 'release-*'] paths: [ "model/**","sdks/python/**","release/**", ".github/workflows/beam_PreCommit_Python_Coverage.yml"] schedule: - - cron: '0 */6 * * *' + - cron: '45 2/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_Python_Dataframes.yml b/.github/workflows/beam_PreCommit_Python_Dataframes.yml index b2292e3484a3..1fff65c00ed5 100644 --- a/.github/workflows/beam_PreCommit_Python_Dataframes.yml +++ b/.github/workflows/beam_PreCommit_Python_Dataframes.yml @@ -25,7 +25,7 @@ on: branches: ['master', 'release-*'] paths: [ "model/**","sdks/python/**","release/**",".github/workflows/beam_PreCommit_Python_Dataframes.yml"] schedule: - - cron: '0 */6 * * *' + - cron: '45 2/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_Python_Examples.yml b/.github/workflows/beam_PreCommit_Python_Examples.yml index 96d714973610..0db33b91c0d2 100644 --- a/.github/workflows/beam_PreCommit_Python_Examples.yml +++ b/.github/workflows/beam_PreCommit_Python_Examples.yml @@ -25,7 +25,7 @@ on: branches: ['master', 'release-*'] paths: [ "model/**","sdks/python/**","release/**",".github/workflows/beam_PreCommit_Python_Examples.yml"] schedule: - - cron: '0 */6 * * *' + - cron: '45 2/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_Python_Integration.yml b/.github/workflows/beam_PreCommit_Python_Integration.yml index afb11abaabc9..76653f00b06c 100644 --- a/.github/workflows/beam_PreCommit_Python_Integration.yml +++ b/.github/workflows/beam_PreCommit_Python_Integration.yml @@ -25,7 +25,7 @@ on: branches: ['master', 'release-*'] paths: ["model/**", "sdks/python/**", "release/**", ".github/workflows/beam_PreCommit_Python_Integration.yml"] schedule: - - cron: '0 */6 * * *' + - cron: '45 2/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_Python_PVR_Flink.yml b/.github/workflows/beam_PreCommit_Python_PVR_Flink.yml index 94655f959fd7..8b93fdc06d77 100644 --- a/.github/workflows/beam_PreCommit_Python_PVR_Flink.yml +++ b/.github/workflows/beam_PreCommit_Python_PVR_Flink.yml @@ -47,7 +47,7 @@ on: - 'runners/reference/**' - '.github/workflows/beam_PreCommit_Python_PVR_Flink.yml' schedule: - - cron: '* */6 * * *' + - cron: '45 2/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_Python_Runners.yml b/.github/workflows/beam_PreCommit_Python_Runners.yml index e3ad7675e983..f2f548291a25 100644 --- a/.github/workflows/beam_PreCommit_Python_Runners.yml +++ b/.github/workflows/beam_PreCommit_Python_Runners.yml @@ -25,7 +25,7 @@ on: branches: ['master', 'release-*'] paths: [ "model/**","sdks/python/**","release/**",".github/workflows/beam_PreCommit_Python_Runners.yml"] schedule: - - cron: '0 */6 * * *' + - cron: '45 2/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_Python_Transforms.yml b/.github/workflows/beam_PreCommit_Python_Transforms.yml index b62d4b4665ef..c60657d6d9e7 100644 --- a/.github/workflows/beam_PreCommit_Python_Transforms.yml +++ b/.github/workflows/beam_PreCommit_Python_Transforms.yml @@ -25,7 +25,7 @@ on: branches: ['master', 'release-*'] paths: [ "model/**","sdks/python/**","release/**",".github/workflows/beam_PreCommit_Python_Transforms.yml"] schedule: - - cron: '0 */6 * * *' + - cron: '45 2/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_RAT.yml b/.github/workflows/beam_PreCommit_RAT.yml index 3d907410b5cd..200e69f94bbf 100644 --- a/.github/workflows/beam_PreCommit_RAT.yml +++ b/.github/workflows/beam_PreCommit_RAT.yml @@ -24,7 +24,7 @@ on: issue_comment: types: [created] schedule: - - cron: '0 */6 * * *' + - cron: '0 3/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_SQL.yml b/.github/workflows/beam_PreCommit_SQL.yml index 3dd36ce050af..800f62b666cd 100644 --- a/.github/workflows/beam_PreCommit_SQL.yml +++ b/.github/workflows/beam_PreCommit_SQL.yml @@ -26,7 +26,7 @@ on: issue_comment: types: [created] schedule: - - cron: '0 */6 * * *' + - cron: '15 3/6 * * *' workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PreCommit_SQL_Java11.yml b/.github/workflows/beam_PreCommit_SQL_Java11.yml index 0f8c032a598f..caee133c2cb9 100644 --- a/.github/workflows/beam_PreCommit_SQL_Java11.yml +++ b/.github/workflows/beam_PreCommit_SQL_Java11.yml @@ -26,7 +26,7 @@ on: issue_comment: types: [created] schedule: - - cron: '0 */6 * * *' + - cron: '15 3/6 * * *' workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PreCommit_SQL_Java17.yml b/.github/workflows/beam_PreCommit_SQL_Java17.yml index 4b67db10f5fc..6ad36b2573e9 100644 --- a/.github/workflows/beam_PreCommit_SQL_Java17.yml +++ b/.github/workflows/beam_PreCommit_SQL_Java17.yml @@ -26,7 +26,7 @@ on: issue_comment: types: [created] schedule: - - cron: '0 */6 * * *' + - cron: '15 3/6 * * *' workflow_dispatch: # Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_Spotless.yml b/.github/workflows/beam_PreCommit_Spotless.yml index f3fc3794bdf3..32475f30486b 100644 --- a/.github/workflows/beam_PreCommit_Spotless.yml +++ b/.github/workflows/beam_PreCommit_Spotless.yml @@ -38,7 +38,7 @@ on: issue_comment: types: [created] schedule: - - cron: '0 */6 * * *' + - cron: '0 3/6 * * *' workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PreCommit_Typescript.yml b/.github/workflows/beam_PreCommit_Typescript.yml index 803d51e7e9a4..ba087e70cf85 100644 --- a/.github/workflows/beam_PreCommit_Typescript.yml +++ b/.github/workflows/beam_PreCommit_Typescript.yml @@ -28,7 +28,7 @@ on: issue_comment: types: [created] schedule: - - cron: '0 */6 * * *' + - cron: '15 3/6 * * *' workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PreCommit_Website.yml b/.github/workflows/beam_PreCommit_Website.yml index bce51e0c929a..6d8c96cd7545 100644 --- a/.github/workflows/beam_PreCommit_Website.yml +++ b/.github/workflows/beam_PreCommit_Website.yml @@ -26,7 +26,7 @@ on: issue_comment: types: [created] schedule: - - cron: '0 */6 * * *' + - cron: '15 3/6 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_Website_Stage_GCS.yml b/.github/workflows/beam_PreCommit_Website_Stage_GCS.yml index 19d96eb8a1ce..e2570e20414b 100644 --- a/.github/workflows/beam_PreCommit_Website_Stage_GCS.yml +++ b/.github/workflows/beam_PreCommit_Website_Stage_GCS.yml @@ -26,7 +26,7 @@ on: issue_comment: types: [created] schedule: - - cron: '0 */6 * * *' + - cron: '15 3/6 * * *' workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PreCommit_Whitespace.yml b/.github/workflows/beam_PreCommit_Whitespace.yml index 148de272b622..2c7294cb224e 100644 --- a/.github/workflows/beam_PreCommit_Whitespace.yml +++ b/.github/workflows/beam_PreCommit_Whitespace.yml @@ -26,7 +26,7 @@ on: issue_comment: types: [created] schedule: - - cron: '0 */6 * * *' + - cron: '15 3/6 * * *' workflow_dispatch: permissions: From 91207bc65dd4da2d53096df9204f226aae191bff Mon Sep 17 00:00:00 2001 From: magicgoody <131876064+magicgoody@users.noreply.github.com> Date: Thu, 19 Oct 2023 09:05:21 +0600 Subject: [PATCH 187/435] Disable comment trigger for Performance and Load Tests (#29035) * reference PR 28915 issue_comment removed * post commit issue_comment fix * revert changes for postcommit --- .github/workflows/beam_Java_LoadTests_Combine_Smoke.yml | 2 -- .github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml | 2 -- ...beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml | 2 -- ..._LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml | 2 -- ...beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml | 2 -- .github/workflows/beam_LoadTests_Java_GBK_Smoke.yml | 2 -- ...beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions.yml | 2 -- ..._LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions.yml | 2 -- .../workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml | 2 -- .../beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml | 2 -- .github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml | 2 -- .../beam_LoadTests_Python_Combine_Dataflow_Streaming.yml | 2 -- .github/workflows/beam_LoadTests_Python_GBK_Flink_Batch.yml | 2 -- .../beam_LoadTests_Python_GBK_reiterate_Dataflow_Batch.yml | 2 -- .../beam_LoadTests_Python_GBK_reiterate_Dataflow_Streaming.yml | 2 -- .../workflows/beam_LoadTests_Python_ParDo_Dataflow_Batch.yml | 2 -- .../beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml | 2 -- .github/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml | 2 -- .../workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml | 2 -- .../beam_LoadTests_Python_SideInput_Dataflow_Batch.yml | 2 -- .github/workflows/beam_LoadTests_Python_Smoke.yml | 2 -- 21 files changed, 42 deletions(-) diff --git a/.github/workflows/beam_Java_LoadTests_Combine_Smoke.yml b/.github/workflows/beam_Java_LoadTests_Combine_Smoke.yml index 39fdf5a40080..9ec96e51d813 100644 --- a/.github/workflows/beam_Java_LoadTests_Combine_Smoke.yml +++ b/.github/workflows/beam_Java_LoadTests_Combine_Smoke.yml @@ -16,8 +16,6 @@ name: LoadTests Java Combine Smoke on: - # issue_comment: - # types: [created] # schedule: # - cron: '10 12 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml index ae50db426a81..b1183364b459 100644 --- a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml @@ -16,8 +16,6 @@ name: LoadTests Java CoGBK Dataflow Batch on: - issue_comment: - types: [created] schedule: - cron: '10 13 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml index a35c3bee7f96..79f9880094da 100644 --- a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml +++ b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml @@ -16,8 +16,6 @@ name: LoadTests Java CoGBK Dataflow V2 Batch JavaVersions on: - issue_comment: - types: [created] schedule: - cron: '10 13 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml index ea7fba6a3e5e..5024ae3e7f1f 100644 --- a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml +++ b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml @@ -16,8 +16,6 @@ name: LoadTests Java CoGBK Dataflow V2 Streaming JavaVersions on: - issue_comment: - types: [created] schedule: - cron: '10 13 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml b/.github/workflows/beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml index dd61eb79fb9b..7ee5243555a1 100644 --- a/.github/workflows/beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml +++ b/.github/workflows/beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml @@ -16,8 +16,6 @@ name: LoadTests Java CoGBK SparkStructuredStreaming Batch on: - issue_comment: - types: [created] schedule: - cron: '50 13 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Smoke.yml b/.github/workflows/beam_LoadTests_Java_GBK_Smoke.yml index 59cdad972d78..15096bb8e4aa 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_Smoke.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_Smoke.yml @@ -16,8 +16,6 @@ name: LoadTests Java GBK Smoke on: - issue_comment: - types: [created] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions.yml b/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions.yml index 0203169f96b3..c9342847d806 100644 --- a/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions.yml +++ b/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions.yml @@ -16,8 +16,6 @@ name: LoadTests Java ParDo Dataflow V2 Batch JavaVersions on: - issue_comment: - types: [created] schedule: - cron: '50 14 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions.yml b/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions.yml index 45ddd688cdcc..1c2751de4b9f 100644 --- a/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions.yml +++ b/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions.yml @@ -16,8 +16,6 @@ name: LoadTests Java ParDo Dataflow V2 Streaming JavaVersions on: - issue_comment: - types: [created] schedule: - cron: '50 14 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml index ab28ffead25e..e502ab83993d 100644 --- a/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml @@ -16,8 +16,6 @@ name: LoadTests Python CoGBK Dataflow Batch on: - issue_comment: - types: [created] schedule: - cron: '10 15 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml index d25a79cdd9a1..6b509e43741a 100644 --- a/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml @@ -16,8 +16,6 @@ name: LoadTests Python CoGBK Dataflow Streaming on: - issue_comment: - types: [created] schedule: - cron: '10 15 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml index 73e28cd36d5c..321769672742 100644 --- a/.github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml @@ -16,8 +16,6 @@ name: LoadTests Python CoGBK Dataflow Flink Batch on: - issue_comment: - types: [created] schedule: - cron: '10 15 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Streaming.yml index 8bbd47a74944..df8ff7a5596c 100644 --- a/.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Streaming.yml @@ -16,8 +16,6 @@ name: LoadTests Python Combine Dataflow Streaming on: - issue_comment: - types: [created] schedule: - cron: '50 15 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Python_GBK_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Python_GBK_Flink_Batch.yml index 29101355fa3e..f153eb754a31 100644 --- a/.github/workflows/beam_LoadTests_Python_GBK_Flink_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_GBK_Flink_Batch.yml @@ -16,8 +16,6 @@ name: LoadTests Python GBK Flink Batch on: - # issue_comment: - # types: [created] # schedule: # - cron: '10 16 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Batch.yml index 97afaa800576..c262061ae87d 100644 --- a/.github/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Batch.yml @@ -16,8 +16,6 @@ name: LoadTests Python GBK reiterate Dataflow Batch on: - issue_comment: - types: [created] schedule: - cron: '10 16 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Streaming.yml index 92df9e3ab924..e0921e72558e 100644 --- a/.github/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Streaming.yml @@ -16,8 +16,6 @@ name: LoadTests Python GBK reiterate Dataflow Streaming on: - issue_comment: - types: [created] schedule: - cron: '10 16 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Batch.yml index e2da23682346..caff1480d0ec 100644 --- a/.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Batch.yml @@ -16,8 +16,6 @@ name: LoadTests Python ParDo Dataflow Batch on: - issue_comment: - types: [created] schedule: - cron: '10 16 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml index 23b5e69f45bf..212800cbbd74 100644 --- a/.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml @@ -16,8 +16,6 @@ name: LoadTests Python ParDo Dataflow Streaming on: - issue_comment: - types: [created] schedule: - cron: '50 16 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml index 029194b040d4..9287f4fa2745 100644 --- a/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml @@ -16,8 +16,6 @@ name: LoadTests Python ParDo Flink Batch on: - issue_comment: - types: [created] schedule: - cron: '50 16 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml b/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml index 85f641bed4d2..ea478f35a2a8 100644 --- a/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml @@ -16,8 +16,6 @@ name: LoadTests Python ParDo Flink Streaming on: - issue_comment: - types: [created] schedule: - cron: '50 16 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Python_SideInput_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Python_SideInput_Dataflow_Batch.yml index 15d2aa6f5c14..5632ce720921 100644 --- a/.github/workflows/beam_LoadTests_Python_SideInput_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_SideInput_Dataflow_Batch.yml @@ -16,8 +16,6 @@ name: LoadTests Python SideInput Dataflow Batch on: - issue_comment: - types: [created] schedule: - cron: '50 16 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Python_Smoke.yml b/.github/workflows/beam_LoadTests_Python_Smoke.yml index 720561d09e06..3a8d5b2e40b8 100644 --- a/.github/workflows/beam_LoadTests_Python_Smoke.yml +++ b/.github/workflows/beam_LoadTests_Python_Smoke.yml @@ -16,8 +16,6 @@ name: LoadTests Python Smoke on: - issue_comment: - types: [created] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event From 8fb23eef6dcf6fa8c42975f2d434492623b349ff Mon Sep 17 00:00:00 2001 From: Bruno Volpato <bvolpato@google.com> Date: Wed, 18 Oct 2023 23:11:37 -0400 Subject: [PATCH 188/435] [Security] Bump org.json:json due to CVE-2022-45688 (#28962) --- .../groovy/org/apache/beam/gradle/BeamModulePlugin.groovy | 4 ++-- .../java/org/apache/beam/sdk/schemas/utils/JsonUtils.java | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index 46968e0e5ad6..b13fd00dc249 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -602,7 +602,7 @@ class BeamModulePlugin implements Plugin<Project> { def jackson_version = "2.14.1" def jaxb_api_version = "2.3.3" def jsr305_version = "3.0.2" - def everit_json_version = "1.14.1" + def everit_json_version = "1.14.2" def kafka_version = "2.4.1" def log4j2_version = "2.20.0" def nemo_version = "0.1" @@ -809,7 +809,7 @@ class BeamModulePlugin implements Plugin<Project> { joda_time : "joda-time:joda-time:2.10.10", jsonassert : "org.skyscreamer:jsonassert:1.5.0", jsr305 : "com.google.code.findbugs:jsr305:$jsr305_version", - json_org : "org.json:json:20220320", // Keep in sync with everit-json-schema / google_cloud_platform_libraries_bom transitive deps. + json_org : "org.json:json:20230618", // Keep in sync with everit-json-schema / google_cloud_platform_libraries_bom transitive deps. everit_json_schema : "com.github.erosb:everit-json-schema:${everit_json_version}", junit : "junit:junit:4.13.1", jupiter_api : "org.junit.jupiter:junit-jupiter-api:$jupiter_version", diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/JsonUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/JsonUtils.java index a724664ceaf1..18f5813c6cc9 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/JsonUtils.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/JsonUtils.java @@ -58,7 +58,7 @@ * }</pre> * * <p><b>Note:</b> This functionality has been tested with {@code everit-json-schema} version - * 1.14.1. + * 1.14.2. * * <h3>JSON-Schema supported features</h3> * From 66c01f035630553a09240e3bbbb74dede79cebe9 Mon Sep 17 00:00:00 2001 From: Rebecca Szper <98840847+rszper@users.noreply.github.com> Date: Thu, 19 Oct 2023 06:12:09 -0700 Subject: [PATCH 189/435] Add link to the Dataflow service options page (#29068) * Add link to the Dataflow service options page * Fix link format * Small text edit --- .../runners/dataflow/options/DataflowPipelineOptions.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptions.java index 985e1736dcb0..8cc812cfa17b 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptions.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptions.java @@ -112,7 +112,10 @@ public interface DataflowPipelineOptions */ @Description( "Service options are set by the user and configure the service. This " - + "decouples service side feature availability from the Apache Beam release cycle.") + + "decouples service side feature availability from the Apache Beam release cycle. " + + "For a list of service options, see " + + "https://cloud.google.com/dataflow/docs/reference/service-options " + + "in the Dataflow documentation.") List<String> getDataflowServiceOptions(); void setDataflowServiceOptions(List<String> options); From 12e193370bec7f214328c313d6e97692088c441c Mon Sep 17 00:00:00 2001 From: Vlado Djerek <rage.bubblegum@gmail.com> Date: Thu, 19 Oct 2023 15:15:49 +0200 Subject: [PATCH 190/435] Arc network update (#29058) * Update arc terraform to allow for coloaction in the default network.Allow usage of reserved ip. Allow usage of existing SA * sync beam env * move aditional runners to load based scaling --- .../arc/config/arc_autoscaler.tpl | 2 +- .../arc/config/arc_deployment.tpl | 3 +++ .../arc/environments/beam.env | 12 ++++++++---- .../gh-actions-self-hosted-runners/arc/gke.tf | 16 ++++++++++++++-- .../arc/kubernetes.tf | 1 + .../arc/locals.tf | 8 +++----- .../arc/network.tf | 10 ++++++++-- .../arc/outputs.tf | 3 +-- .../arc/provider.tf | 6 +++--- .../arc/variables.tf | 17 +++++++++++++++++ 10 files changed, 59 insertions(+), 19 deletions(-) diff --git a/.github/gh-actions-self-hosted-runners/arc/config/arc_autoscaler.tpl b/.github/gh-actions-self-hosted-runners/arc/config/arc_autoscaler.tpl index f6da0aff038a..4b04c5ad8eb1 100644 --- a/.github/gh-actions-self-hosted-runners/arc/config/arc_autoscaler.tpl +++ b/.github/gh-actions-self-hosted-runners/arc/config/arc_autoscaler.tpl @@ -27,7 +27,7 @@ spec: name: ${name} minReplicas: ${min_runners} maxReplicas: ${max_runners} - %{~ if webhook_scaling == "true" ~} + %{~ if webhook_scaling ~} scaleUpTriggers: - githubEvent: workflowJob: {} diff --git a/.github/gh-actions-self-hosted-runners/arc/config/arc_deployment.tpl b/.github/gh-actions-self-hosted-runners/arc/config/arc_deployment.tpl index 6234571c55a3..71f8da3d9df1 100644 --- a/.github/gh-actions-self-hosted-runners/arc/config/arc_deployment.tpl +++ b/.github/gh-actions-self-hosted-runners/arc/config/arc_deployment.tpl @@ -22,6 +22,9 @@ metadata: name: ${name} spec: template: + metadata: + annotations: + cluster-autoscaler.kubernetes.io/safe-to-evict: "false" spec: %{~ if selector == true ~} nodeSelector: diff --git a/.github/gh-actions-self-hosted-runners/arc/environments/beam.env b/.github/gh-actions-self-hosted-runners/arc/environments/beam.env index 9de66b628c89..95c09e1cfad3 100644 --- a/.github/gh-actions-self-hosted-runners/arc/environments/beam.env +++ b/.github/gh-actions-self-hosted-runners/arc/environments/beam.env @@ -20,7 +20,7 @@ project_id = "apache-beam-testing" region = "us-central1" zone = "us-central1-b" -environment = "beam" +environment = "beam-prod" ingress_domain = "action.beam.apache.org" organization = "apache" repository = "beam" @@ -28,6 +28,10 @@ github_app_id_secret_name = "gh-app_id" github_app_install_id_secret_name = "gh-app_installation_id" github_private_key_secret_name = "gh-pem_key" deploy_webhook = "true" +existing_vpc_name = "default" +existing_ip_name = "beam-arc-webhook-ip" +subnetwork_cidr_range = "10.119.0.0/20" +service_account_id = "beam-github-actions@apache-beam-testing.iam.gserviceaccount.com" runner_group = "beam" main_runner = { name = "main-runner" @@ -37,7 +41,7 @@ main_runner = { max_node_count = "24" min_replicas = "1" max_replicas = "200" - webhook_scaling = true + webhook_scaling = false disk_size_gb = 200 requests = { cpu = "2" @@ -52,7 +56,7 @@ additional_runner_pools = [{ max_node_count = "10" min_replicas = "1" max_replicas = "10" - webhook_scaling = "true" + webhook_scaling = false requests = { cpu = "1500m" memory = "5Gi" @@ -69,7 +73,7 @@ additional_runner_pools = [{ max_node_count = "10" min_replicas = "1" max_replicas = "10" - webhook_scaling = "true" + webhook_scaling = false requests = { cpu = "7.5" memory = "5Gi" diff --git a/.github/gh-actions-self-hosted-runners/arc/gke.tf b/.github/gh-actions-self-hosted-runners/arc/gke.tf index bfb048885570..45421ad38b47 100644 --- a/.github/gh-actions-self-hosted-runners/arc/gke.tf +++ b/.github/gh-actions-self-hosted-runners/arc/gke.tf @@ -21,7 +21,7 @@ resource "google_container_cluster" "actions-runner-gke" { project = var.project_id location = var.zone initial_node_count = 1 - network = google_compute_network.actions-runner-network.id + network = data.google_compute_network.actions-runner-network.id subnetwork = google_compute_subnetwork.actions-runner-subnetwork.id remove_default_node_pool = true @@ -45,6 +45,7 @@ resource "google_container_node_pool" "main-actions-runner-pool" { oauth_scopes = [ "https://www.googleapis.com/auth/cloud-platform" ] + service_account = data.google_service_account.service_account.email tags = ["actions-runner-pool"] } } @@ -72,6 +73,7 @@ resource "google_container_node_pool" "additional_runner_pools" { oauth_scopes = [ "https://www.googleapis.com/auth/cloud-platform" ] + service_account = data.google_service_account.service_account.email tags = ["actions-runner-pool"] labels = { "runner-pool" = each.value.name @@ -90,5 +92,15 @@ resource "google_container_node_pool" "additional_runner_pools" { resource "google_compute_global_address" "actions-runner-ip" { - name = "${var.environment}-actions-runner-ip" + count = var.deploy_webhook == "true" && var.existing_ip_name == "" ? 1 : 0 + name = "${var.environment}-actions-runner-ip" +} + +data "google_compute_global_address" "actions-runner-ip" { + count = var.deploy_webhook == "true" ? 1 : 0 + name = var.existing_ip_name == "" ? google_compute_global_address.actions-runner-ip[0].name : var.existing_ip_name +} + +data google_service_account "service_account" { + account_id = var.service_account_id } \ No newline at end of file diff --git a/.github/gh-actions-self-hosted-runners/arc/kubernetes.tf b/.github/gh-actions-self-hosted-runners/arc/kubernetes.tf index bafb653896d7..0a36e1fa2ba6 100644 --- a/.github/gh-actions-self-hosted-runners/arc/kubernetes.tf +++ b/.github/gh-actions-self-hosted-runners/arc/kubernetes.tf @@ -27,6 +27,7 @@ resource "kubectl_manifest" "arc_autoscaler" { depends_on = [helm_release.arc] } resource "kubectl_manifest" "arc_webhook_certificate" { + count = var.deploy_webhook != "false" ? 1 : 0 yaml_body = templatefile("config/arc_certificate.tpl", { ingress_domain = var.ingress_domain }) override_namespace = "arc" depends_on = [helm_release.arc] diff --git a/.github/gh-actions-self-hosted-runners/arc/locals.tf b/.github/gh-actions-self-hosted-runners/arc/locals.tf index 170193b8b6b6..a69d069ab865 100644 --- a/.github/gh-actions-self-hosted-runners/arc/locals.tf +++ b/.github/gh-actions-self-hosted-runners/arc/locals.tf @@ -19,20 +19,18 @@ locals { - subnetwork_cidr_range = "10.128.0.0/20" arc_values = { - "githubWebhookServer.enabled" = "true" + "githubWebhookServer.enabled" = "${var.deploy_webhook}" "authSecret.create" = "true" "authSecret.github_app_id" = data.google_secret_manager_secret_version.github_app_id.secret_data "authSecret.github_app_installation_id" = data.google_secret_manager_secret_version.github_app_install_id.secret_data "authSecret.github_app_private_key" = data.google_secret_manager_secret_version.github_private_key.secret_data - "githubWebhookServer.ingress.enabled" = "true" + "githubWebhookServer.ingress.enabled" = "${var.deploy_webhook}" "githubWebhookServer.ingress.hosts[0].host" = var.ingress_domain "githubWebhookServer.ingress.hosts[0].paths[0].path" = "/" "githubWebhookServer.ingress.hosts[0].paths[0].pathType" = "ImplementationSpecific" "githubWebhookServer.service.type" = "NodePort" - #"githubWebhookServer.ingress.tls[0].hosts[0]" = var.ingress_domain - "githubWebhookServer.ingress.annotations.kubernetes\\.io/ingress\\.global-static-ip-name" = google_compute_global_address.actions-runner-ip.name + "githubWebhookServer.ingress.annotations.kubernetes\\.io/ingress\\.global-static-ip-name" = var.deploy_webhook != "false" ? data.google_compute_global_address.actions-runner-ip[0].name : "not-configured" "githubWebhookServer.ingress.annotations.networking\\.gke\\.io/managed-certificates" = "managed-cert" "githubWebhookServer.ingress.annotations.kubernetes\\.io/ingress\\.class" = "gce" } diff --git a/.github/gh-actions-self-hosted-runners/arc/network.tf b/.github/gh-actions-self-hosted-runners/arc/network.tf index fb7c23a7a3c6..1bc685641337 100644 --- a/.github/gh-actions-self-hosted-runners/arc/network.tf +++ b/.github/gh-actions-self-hosted-runners/arc/network.tf @@ -18,15 +18,21 @@ # resource "google_compute_network" "actions-runner-network" { + count = var.existing_vpc_name == "" ? 1 : 0 project = var.project_id name = "${var.environment}-actions-runner-network" auto_create_subnetworks = false } +data "google_compute_network" "actions-runner-network" { + name = var.existing_vpc_name == "" ? google_compute_network.actions-runner-network[0].name : var.existing_vpc_name + project = var.project_id +} + resource "google_compute_subnetwork" "actions-runner-subnetwork" { - ip_cidr_range = local.subnetwork_cidr_range + ip_cidr_range = var.subnetwork_cidr_range name = "${var.environment}-actions-runner-subnetwork" - network = google_compute_network.actions-runner-network.id + network = data.google_compute_network.actions-runner-network.id region = var.region project = var.project_id } diff --git a/.github/gh-actions-self-hosted-runners/arc/outputs.tf b/.github/gh-actions-self-hosted-runners/arc/outputs.tf index 1e805ca74ce1..f7450911aaf7 100644 --- a/.github/gh-actions-self-hosted-runners/arc/outputs.tf +++ b/.github/gh-actions-self-hosted-runners/arc/outputs.tf @@ -24,9 +24,8 @@ output "cluster_endpoint" { value = google_container_cluster.actions-runner-gke.endpoint } output "ingress_ip" { - value = google_compute_global_address.actions-runner-ip.address + value = var.deploy_webhook != "false" ? data.google_compute_global_address.actions-runner-ip[0].address : "Not Configured" } - output "get_kubeconfig_command" { value = "gcloud container clusters get-credentials ${google_container_cluster.actions-runner-gke.name} --region ${var.zone} --project ${var.project_id}" } diff --git a/.github/gh-actions-self-hosted-runners/arc/provider.tf b/.github/gh-actions-self-hosted-runners/arc/provider.tf index 11aa604fb288..dc557b62a559 100644 --- a/.github/gh-actions-self-hosted-runners/arc/provider.tf +++ b/.github/gh-actions-self-hosted-runners/arc/provider.tf @@ -19,7 +19,7 @@ terraform { backend "gcs" { - prefix = "test-state" + prefix = "prod" } required_providers { @@ -28,8 +28,8 @@ terraform { version = "~> 4.62.0" } kubectl = { - source = "gavinbunney/kubectl" - version = ">= 1.7.0" + source = "alekc/kubectl" + version = ">= 2.0.2" } } } diff --git a/.github/gh-actions-self-hosted-runners/arc/variables.tf b/.github/gh-actions-self-hosted-runners/arc/variables.tf index 43f51938b7d1..3caeffe5a523 100644 --- a/.github/gh-actions-self-hosted-runners/arc/variables.tf +++ b/.github/gh-actions-self-hosted-runners/arc/variables.tf @@ -58,6 +58,23 @@ variable "deploy_webhook" { description = "Enable Github Webhook deployment. use this if the Github App has permissions to create webhooks" default = "false" } +variable "existing_vpc_name" { + description = "Name of existing VPC to use for deployment" + default = "" +} +variable "existing_ip_name" { + description = "Name of existing IP to use for ingress" + default = "" +} +variable "subnetwork_cidr_range" { + description = "CIDR range for subnetwork" + default = "10.128.0.0/20" + +} +variable "service_account_id" { + description = "ID of service account to use for deployment. This can be Name, full Email or Fully Qualified Path" + default = "" +} variable "runner_group" { description = "value for the runner group label" default = "" From fec07c11af2c2d387837567d4d4c4a05d30e045c Mon Sep 17 00:00:00 2001 From: Danny McCormick <dannymccormick@google.com> Date: Thu, 19 Oct 2023 09:26:10 -0400 Subject: [PATCH 191/435] Update base image requirements in preparation for next release (#29061) --- .../py310/base_image_requirements.txt | 54 +++++++++--------- .../py311/base_image_requirements.txt | 52 ++++++++--------- .../py38/base_image_requirements.txt | 54 +++++++++--------- .../py39/base_image_requirements.txt | 56 ++++++++++--------- 4 files changed, 114 insertions(+), 102 deletions(-) diff --git a/sdks/python/container/py310/base_image_requirements.txt b/sdks/python/container/py310/base_image_requirements.txt index 82b210005feb..70b6f9018905 100644 --- a/sdks/python/container/py310/base_image_requirements.txt +++ b/sdks/python/container/py310/base_image_requirements.txt @@ -24,10 +24,11 @@ attrs==23.1.0 beautifulsoup4==4.12.2 bs4==0.0.1 +build==1.0.3 cachetools==5.3.1 certifi==2023.7.22 -cffi==1.15.1 -charset-normalizer==3.2.0 +cffi==1.16.0 +charset-normalizer==3.3.0 click==8.1.7 cloudpickle==2.2.1 crcmod==1.7 @@ -40,17 +41,17 @@ docker==6.1.3 docopt==0.6.2 exceptiongroup==1.1.3 execnet==2.0.2 -fastavro==1.8.3 +fastavro==1.8.4 fasteners==0.19 freezegun==1.2.2 future==0.18.3 -google-api-core==2.11.1 -google-api-python-client==2.100.0 +google-api-core==2.12.0 +google-api-python-client==2.104.0 google-apitools==0.5.31 -google-auth==2.23.0 +google-auth==2.23.3 google-auth-httplib2==0.1.1 -google-cloud-aiplatform==1.33.1 -google-cloud-bigquery==3.11.4 +google-cloud-aiplatform==1.35.0 +google-cloud-bigquery==3.12.0 google-cloud-bigquery-storage==2.22.0 google-cloud-bigtable==2.21.0 google-cloud-core==2.3.3 @@ -63,20 +64,20 @@ google-cloud-pubsublite==1.8.3 google-cloud-recommendations-ai==0.10.5 google-cloud-resource-manager==1.10.4 google-cloud-spanner==3.40.1 -google-cloud-storage==2.11.0 +google-cloud-storage==2.12.0 google-cloud-videointelligence==2.11.4 google-cloud-vision==3.4.4 google-crc32c==1.5.0 google-resumable-media==2.6.0 -googleapis-common-protos==1.60.0 -greenlet==2.0.2 +googleapis-common-protos==1.61.0 +greenlet==3.0.0 grpc-google-iam-v1==0.12.6 -grpcio==1.58.0 -grpcio-status==1.58.0 +grpcio==1.59.0 +grpcio-status==1.59.0 guppy3==3.1.3 -hdfs==2.7.2 +hdfs==2.7.3 httplib2==0.22.0 -hypothesis==6.87.0 +hypothesis==6.88.1 idna==3.4 iniconfig==2.0.0 joblib==1.3.2 @@ -88,15 +89,15 @@ nose==1.3.7 numpy==1.24.4 oauth2client==4.1.3 objsize==0.6.1 -orjson==3.9.7 +orjson==3.9.9 overrides==6.5.0 -packaging==23.1 +packaging==23.2 pandas==1.5.3 parameterized==0.9.0 pluggy==1.3.0 proto-plus==1.22.3 -protobuf==4.24.3 -psycopg2-binary==2.9.7 +protobuf==4.24.4 +psycopg2-binary==2.9.9 pyarrow==11.0.0 pyasn1==0.5.0 pyasn1-modules==0.3.0 @@ -107,20 +108,21 @@ pyjsparser==2.7.1 pymongo==4.5.0 PyMySQL==1.1.0 pyparsing==3.1.1 +pyproject_hooks==1.0.0 pytest==7.4.2 -pytest-timeout==2.1.0 +pytest-timeout==2.2.0 pytest-xdist==3.3.1 python-dateutil==2.8.2 python-snappy==0.6.1 pytz==2023.3.post1 PyYAML==6.0.1 -regex==2023.8.8 +regex==2023.10.3 requests==2.31.0 requests-mock==1.11.0 rsa==4.9 scikit-learn==1.3.1 -scipy==1.11.2 -Shapely==1.8.5.post1 +scipy==1.11.3 +shapely==2.0.2 six==1.16.0 sortedcontainers==2.4.0 soupsieve==2.5 @@ -132,9 +134,9 @@ threadpoolctl==3.2.0 tomli==2.0.1 tqdm==4.66.1 typing_extensions==4.8.0 -tzlocal==5.0.1 +tzlocal==5.1 uritemplate==4.1.1 -urllib3==1.26.18 -websocket-client==1.6.3 +urllib3==2.0.7 +websocket-client==1.6.4 wrapt==1.15.0 zstandard==0.21.0 diff --git a/sdks/python/container/py311/base_image_requirements.txt b/sdks/python/container/py311/base_image_requirements.txt index 2e5d834926bd..c293817ed6a8 100644 --- a/sdks/python/container/py311/base_image_requirements.txt +++ b/sdks/python/container/py311/base_image_requirements.txt @@ -24,10 +24,11 @@ attrs==23.1.0 beautifulsoup4==4.12.2 bs4==0.0.1 +build==1.0.3 cachetools==5.3.1 certifi==2023.7.22 -cffi==1.15.1 -charset-normalizer==3.2.0 +cffi==1.16.0 +charset-normalizer==3.3.0 click==8.1.7 cloudpickle==2.2.1 crcmod==1.7 @@ -39,16 +40,16 @@ dnspython==2.4.2 docker==6.1.3 docopt==0.6.2 execnet==2.0.2 -fastavro==1.8.3 +fastavro==1.8.4 fasteners==0.19 freezegun==1.2.2 future==0.18.3 -google-api-core==2.11.1 +google-api-core==2.12.0 google-apitools==0.5.31 -google-auth==2.23.0 +google-auth==2.23.3 google-auth-httplib2==0.1.1 -google-cloud-aiplatform==1.33.1 -google-cloud-bigquery==3.11.4 +google-cloud-aiplatform==1.35.0 +google-cloud-bigquery==3.12.0 google-cloud-bigquery-storage==2.22.0 google-cloud-bigtable==2.21.0 google-cloud-core==2.3.3 @@ -60,20 +61,20 @@ google-cloud-pubsublite==1.8.3 google-cloud-recommendations-ai==0.10.5 google-cloud-resource-manager==1.10.4 google-cloud-spanner==3.40.1 -google-cloud-storage==2.11.0 +google-cloud-storage==2.12.0 google-cloud-videointelligence==2.11.4 google-cloud-vision==3.4.4 google-crc32c==1.5.0 google-resumable-media==2.6.0 -googleapis-common-protos==1.60.0 -greenlet==2.0.2 +googleapis-common-protos==1.61.0 +greenlet==3.0.0 grpc-google-iam-v1==0.12.6 -grpcio==1.58.0 -grpcio-status==1.58.0 +grpcio==1.59.0 +grpcio-status==1.59.0 guppy3==3.1.3 -hdfs==2.7.2 +hdfs==2.7.3 httplib2==0.22.0 -hypothesis==6.87.0 +hypothesis==6.88.1 idna==3.4 iniconfig==2.0.0 joblib==1.3.2 @@ -85,15 +86,15 @@ nose==1.3.7 numpy==1.24.4 oauth2client==4.1.3 objsize==0.6.1 -orjson==3.9.7 +orjson==3.9.9 overrides==6.5.0 -packaging==23.1 +packaging==23.2 pandas==1.5.3 parameterized==0.9.0 pluggy==1.3.0 proto-plus==1.22.3 -protobuf==4.24.3 -psycopg2-binary==2.9.7 +protobuf==4.24.4 +psycopg2-binary==2.9.9 pyarrow==11.0.0 pyasn1==0.5.0 pyasn1-modules==0.3.0 @@ -104,19 +105,20 @@ pyjsparser==2.7.1 pymongo==4.5.0 PyMySQL==1.1.0 pyparsing==3.1.1 +pyproject_hooks==1.0.0 pytest==7.4.2 -pytest-timeout==2.1.0 +pytest-timeout==2.2.0 pytest-xdist==3.3.1 python-dateutil==2.8.2 pytz==2023.3.post1 PyYAML==6.0.1 -regex==2023.8.8 +regex==2023.10.3 requests==2.31.0 requests-mock==1.11.0 rsa==4.9 scikit-learn==1.3.1 -scipy==1.11.2 -Shapely==1.8.5.post1 +scipy==1.11.3 +shapely==2.0.2 six==1.16.0 sortedcontainers==2.4.0 soupsieve==2.5 @@ -127,8 +129,8 @@ testcontainers==3.7.1 threadpoolctl==3.2.0 tqdm==4.66.1 typing_extensions==4.8.0 -tzlocal==5.0.1 -urllib3==1.26.16 -websocket-client==1.6.3 +tzlocal==5.1 +urllib3==2.0.7 +websocket-client==1.6.4 wrapt==1.15.0 zstandard==0.21.0 diff --git a/sdks/python/container/py38/base_image_requirements.txt b/sdks/python/container/py38/base_image_requirements.txt index ed5d35fc6457..a46ec4573f8f 100644 --- a/sdks/python/container/py38/base_image_requirements.txt +++ b/sdks/python/container/py38/base_image_requirements.txt @@ -25,10 +25,11 @@ attrs==23.1.0 backports.zoneinfo==0.2.1 beautifulsoup4==4.12.2 bs4==0.0.1 +build==1.0.3 cachetools==5.3.1 certifi==2023.7.22 -cffi==1.15.1 -charset-normalizer==3.2.0 +cffi==1.16.0 +charset-normalizer==3.3.0 click==8.1.7 cloudpickle==2.2.1 crcmod==1.7 @@ -41,17 +42,17 @@ docker==6.1.3 docopt==0.6.2 exceptiongroup==1.1.3 execnet==2.0.2 -fastavro==1.8.3 +fastavro==1.8.4 fasteners==0.19 freezegun==1.2.2 future==0.18.3 -google-api-core==2.11.1 -google-api-python-client==2.100.0 +google-api-core==2.12.0 +google-api-python-client==2.104.0 google-apitools==0.5.31 -google-auth==2.23.0 +google-auth==2.23.3 google-auth-httplib2==0.1.1 -google-cloud-aiplatform==1.33.1 -google-cloud-bigquery==3.11.4 +google-cloud-aiplatform==1.35.0 +google-cloud-bigquery==3.12.0 google-cloud-bigquery-storage==2.22.0 google-cloud-bigtable==2.21.0 google-cloud-core==2.3.3 @@ -64,21 +65,22 @@ google-cloud-pubsublite==1.8.3 google-cloud-recommendations-ai==0.10.5 google-cloud-resource-manager==1.10.4 google-cloud-spanner==3.40.1 -google-cloud-storage==2.11.0 +google-cloud-storage==2.12.0 google-cloud-videointelligence==2.11.4 google-cloud-vision==3.4.4 google-crc32c==1.5.0 google-resumable-media==2.6.0 -googleapis-common-protos==1.60.0 -greenlet==2.0.2 +googleapis-common-protos==1.61.0 +greenlet==3.0.0 grpc-google-iam-v1==0.12.6 -grpcio==1.58.0 -grpcio-status==1.58.0 +grpcio==1.59.0 +grpcio-status==1.59.0 guppy3==3.1.3 -hdfs==2.7.2 +hdfs==2.7.3 httplib2==0.22.0 -hypothesis==6.87.0 +hypothesis==6.88.1 idna==3.4 +importlib-metadata==6.8.0 iniconfig==2.0.0 joblib==1.3.2 Js2Py==0.74 @@ -89,15 +91,15 @@ nose==1.3.7 numpy==1.24.4 oauth2client==4.1.3 objsize==0.6.1 -orjson==3.9.7 +orjson==3.9.9 overrides==6.5.0 -packaging==23.1 +packaging==23.2 pandas==1.5.3 parameterized==0.9.0 pluggy==1.3.0 proto-plus==1.22.3 -protobuf==4.24.3 -psycopg2-binary==2.9.7 +protobuf==4.24.4 +psycopg2-binary==2.9.9 pyarrow==11.0.0 pyasn1==0.5.0 pyasn1-modules==0.3.0 @@ -108,20 +110,21 @@ pyjsparser==2.7.1 pymongo==4.5.0 PyMySQL==1.1.0 pyparsing==3.1.1 +pyproject_hooks==1.0.0 pytest==7.4.2 -pytest-timeout==2.1.0 +pytest-timeout==2.2.0 pytest-xdist==3.3.1 python-dateutil==2.8.2 python-snappy==0.6.1 pytz==2023.3.post1 PyYAML==6.0.1 -regex==2023.8.8 +regex==2023.10.3 requests==2.31.0 requests-mock==1.11.0 rsa==4.9 scikit-learn==1.3.1 scipy==1.10.1 -Shapely==1.8.5.post1 +shapely==2.0.2 six==1.16.0 sortedcontainers==2.4.0 soupsieve==2.5 @@ -133,9 +136,10 @@ threadpoolctl==3.2.0 tomli==2.0.1 tqdm==4.66.1 typing_extensions==4.8.0 -tzlocal==5.0.1 +tzlocal==5.1 uritemplate==4.1.1 -urllib3==1.26.17 -websocket-client==1.6.3 +urllib3==2.0.7 +websocket-client==1.6.4 wrapt==1.15.0 +zipp==3.17.0 zstandard==0.21.0 diff --git a/sdks/python/container/py39/base_image_requirements.txt b/sdks/python/container/py39/base_image_requirements.txt index ff6ba0945e14..4809b8300267 100644 --- a/sdks/python/container/py39/base_image_requirements.txt +++ b/sdks/python/container/py39/base_image_requirements.txt @@ -24,10 +24,11 @@ attrs==23.1.0 beautifulsoup4==4.12.2 bs4==0.0.1 +build==1.0.3 cachetools==5.3.1 certifi==2023.7.22 -cffi==1.15.1 -charset-normalizer==3.2.0 +cffi==1.16.0 +charset-normalizer==3.3.0 click==8.1.7 cloudpickle==2.2.1 crcmod==1.7 @@ -40,17 +41,17 @@ docker==6.1.3 docopt==0.6.2 exceptiongroup==1.1.3 execnet==2.0.2 -fastavro==1.8.3 +fastavro==1.8.4 fasteners==0.19 freezegun==1.2.2 future==0.18.3 -google-api-core==2.11.1 -google-api-python-client==2.100.0 +google-api-core==2.12.0 +google-api-python-client==2.104.0 google-apitools==0.5.31 -google-auth==2.23.0 +google-auth==2.23.3 google-auth-httplib2==0.1.1 -google-cloud-aiplatform==1.33.1 -google-cloud-bigquery==3.11.4 +google-cloud-aiplatform==1.35.0 +google-cloud-bigquery==3.12.0 google-cloud-bigquery-storage==2.22.0 google-cloud-bigtable==2.21.0 google-cloud-core==2.3.3 @@ -63,21 +64,22 @@ google-cloud-pubsublite==1.8.3 google-cloud-recommendations-ai==0.10.5 google-cloud-resource-manager==1.10.4 google-cloud-spanner==3.40.1 -google-cloud-storage==2.11.0 +google-cloud-storage==2.12.0 google-cloud-videointelligence==2.11.4 google-cloud-vision==3.4.4 google-crc32c==1.5.0 google-resumable-media==2.6.0 -googleapis-common-protos==1.60.0 -greenlet==2.0.2 +googleapis-common-protos==1.61.0 +greenlet==3.0.0 grpc-google-iam-v1==0.12.6 -grpcio==1.58.0 -grpcio-status==1.58.0 +grpcio==1.59.0 +grpcio-status==1.59.0 guppy3==3.1.3 -hdfs==2.7.2 +hdfs==2.7.3 httplib2==0.22.0 -hypothesis==6.87.0 +hypothesis==6.88.1 idna==3.4 +importlib-metadata==6.8.0 iniconfig==2.0.0 joblib==1.3.2 Js2Py==0.74 @@ -88,15 +90,15 @@ nose==1.3.7 numpy==1.24.4 oauth2client==4.1.3 objsize==0.6.1 -orjson==3.9.7 +orjson==3.9.9 overrides==6.5.0 -packaging==23.1 +packaging==23.2 pandas==1.5.3 parameterized==0.9.0 pluggy==1.3.0 proto-plus==1.22.3 -protobuf==4.24.3 -psycopg2-binary==2.9.7 +protobuf==4.24.4 +psycopg2-binary==2.9.9 pyarrow==11.0.0 pyasn1==0.5.0 pyasn1-modules==0.3.0 @@ -107,20 +109,21 @@ pyjsparser==2.7.1 pymongo==4.5.0 PyMySQL==1.1.0 pyparsing==3.1.1 +pyproject_hooks==1.0.0 pytest==7.4.2 -pytest-timeout==2.1.0 +pytest-timeout==2.2.0 pytest-xdist==3.3.1 python-dateutil==2.8.2 python-snappy==0.6.1 pytz==2023.3.post1 PyYAML==6.0.1 -regex==2023.8.8 +regex==2023.10.3 requests==2.31.0 requests-mock==1.11.0 rsa==4.9 scikit-learn==1.3.1 -scipy==1.11.2 -Shapely==1.8.5.post1 +scipy==1.11.3 +shapely==2.0.2 six==1.16.0 sortedcontainers==2.4.0 soupsieve==2.5 @@ -132,9 +135,10 @@ threadpoolctl==3.2.0 tomli==2.0.1 tqdm==4.66.1 typing_extensions==4.8.0 -tzlocal==5.0.1 +tzlocal==5.1 uritemplate==4.1.1 -urllib3==1.26.16 -websocket-client==1.6.3 +urllib3==2.0.7 +websocket-client==1.6.4 wrapt==1.15.0 +zipp==3.17.0 zstandard==0.21.0 From f0e0c2fc8a9a74bb051c922ef0f42c3ff494ee65 Mon Sep 17 00:00:00 2001 From: Aleksandr Dudko <116064902+aleksandr-dudko@users.noreply.github.com> Date: Thu, 19 Oct 2023 17:34:13 +0400 Subject: [PATCH 192/435] Fix file paths for test arguments (#29034) --- .../beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml | 8 ++++---- .../beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml | 8 ++++---- .../beam_LoadTests_Python_ParDo_Flink_Batch.yml | 6 +++--- .../beam_LoadTests_Python_ParDo_Flink_Streaming.yml | 10 +++++----- .../python_CoGBK_Dataflow_Batch_100b_Multiple_Keys.txt | 0 .../python_CoGBK_Dataflow_Batch_100b_Single_Key.txt | 0 .../python_CoGBK_Dataflow_Batch_10kB.txt | 0 .../python_CoGBK_Dataflow_Batch_2MB.txt | 0 ...n_CoGBK_Dataflow_Flink_Batch_100b_Multiple_Keys.txt | 0 ...thon_CoGBK_Dataflow_Flink_Batch_100b_Single_Key.txt | 0 .../python_CoGBK_Dataflow_Flink_Batch_10kB.txt | 0 ...hon_CoGBK_Dataflow_Streaming_100b_Multiple_Keys.txt | 0 ...python_CoGBK_Dataflow_Streaming_100b_Single_Key.txt | 0 .../python_CoGBK_Dataflow_Streaming_10kB.txt | 0 .../python_CoGBK_Dataflow_Streaming_2MB.txt | 0 15 files changed, 16 insertions(+), 16 deletions(-) rename .github/workflows/load-tests-job-configs/{python-cogbk => }/python_CoGBK_Dataflow_Batch_100b_Multiple_Keys.txt (100%) rename .github/workflows/load-tests-job-configs/{python-cogbk => }/python_CoGBK_Dataflow_Batch_100b_Single_Key.txt (100%) rename .github/workflows/load-tests-job-configs/{python-cogbk => }/python_CoGBK_Dataflow_Batch_10kB.txt (100%) rename .github/workflows/load-tests-job-configs/{python-cogbk => }/python_CoGBK_Dataflow_Batch_2MB.txt (100%) rename .github/workflows/load-tests-job-configs/{python-cogbk => }/python_CoGBK_Dataflow_Flink_Batch_100b_Multiple_Keys.txt (100%) rename .github/workflows/load-tests-job-configs/{python-cogbk => }/python_CoGBK_Dataflow_Flink_Batch_100b_Single_Key.txt (100%) rename .github/workflows/load-tests-job-configs/{python-cogbk => }/python_CoGBK_Dataflow_Flink_Batch_10kB.txt (100%) rename .github/workflows/load-tests-job-configs/{python-cogbk => }/python_CoGBK_Dataflow_Streaming_100b_Multiple_Keys.txt (100%) rename .github/workflows/load-tests-job-configs/{python-cogbk => }/python_CoGBK_Dataflow_Streaming_100b_Single_Key.txt (100%) rename .github/workflows/load-tests-job-configs/{python-cogbk => }/python_CoGBK_Dataflow_Streaming_10kB.txt (100%) rename .github/workflows/load-tests-job-configs/{python-cogbk => }/python_CoGBK_Dataflow_Streaming_2MB.txt (100%) diff --git a/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml index e502ab83993d..9112246d89c0 100644 --- a/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml @@ -77,10 +77,10 @@ jobs: test-type: load test-language: python argument-file-paths: | - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Batch_100b_Single_Key.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Batch_100b_Multiple_Keys.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Batch_10kB.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Batch_2MB.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_CoGBK_Dataflow_Batch_100b_Single_Key.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_CoGBK_Dataflow_Batch_100b_Multiple_Keys.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_CoGBK_Dataflow_Batch_10kB.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_CoGBK_Dataflow_Batch_2MB.txt - name: Set current datetime id: datetime run: | diff --git a/.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml index 212800cbbd74..795c4c390a82 100644 --- a/.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml @@ -77,10 +77,10 @@ jobs: test-type: load test-language: python argument-file-paths: | - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python-pardo/python_ParDo_Dataflow_Streaming_10_Iterations.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python-pardo/python_ParDo_Dataflow_Streaming_200_Iterations.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python-pardo/python_ParDo_Dataflow_Streaming_10_Counters.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python-pardo/python_ParDo_Dataflow_Streaming_100_Counters.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_ParDo_Dataflow_Streaming_10_Iterations.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_ParDo_Dataflow_Streaming_200_Iterations.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_ParDo_Dataflow_Streaming_10_Counters.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_ParDo_Dataflow_Streaming_100_Counters.txt - name: Set current datetime id: datetime run: | diff --git a/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml index 9287f4fa2745..032a221d83db 100644 --- a/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml @@ -87,9 +87,9 @@ jobs: test-type: load test-language: python argument-file-paths: | - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python-pardo/python_ParDo_Flink_Batch_10_Iterations.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python-pardo/python_ParDo_Flink_Batch_200_Iterations.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python-pardo/python_ParDo_Flink_Batch_10_Counters.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_ParDo_Flink_Batch_10_Iterations.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_ParDo_Flink_Batch_200_Iterations.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_ParDo_Flink_Batch_10_Counters.txt - name: Start Flink with parallelism 5 env: FLINK_NUM_WORKERS: 5 diff --git a/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml b/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml index ea478f35a2a8..916ff208c910 100644 --- a/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml @@ -87,11 +87,11 @@ jobs: test-type: load test-language: python argument-file-paths: | - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python-pardo/python_ParDo_Flink_Streaming_10_Iterations.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python-pardo/python_ParDo_Flink_Streaming_200_Iterations.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python-pardo/python_ParDo_Flink_Streaming_10_Counters.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python-pardo/python_ParDo_Flink_Streaming_100_Counters.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python-pardo/python_ParDo_Flink_Streaming_5_Iterations.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_ParDo_Flink_Streaming_10_Iterations.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_ParDo_Flink_Streaming_200_Iterations.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_ParDo_Flink_Streaming_10_Counters.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_ParDo_Flink_Streaming_100_Counters.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_ParDo_Flink_Streaming_5_Iterations.txt - name: Start Flink with parallelism 5 env: FLINK_NUM_WORKERS: 5 diff --git a/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Batch_100b_Multiple_Keys.txt b/.github/workflows/load-tests-job-configs/python_CoGBK_Dataflow_Batch_100b_Multiple_Keys.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Batch_100b_Multiple_Keys.txt rename to .github/workflows/load-tests-job-configs/python_CoGBK_Dataflow_Batch_100b_Multiple_Keys.txt diff --git a/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Batch_100b_Single_Key.txt b/.github/workflows/load-tests-job-configs/python_CoGBK_Dataflow_Batch_100b_Single_Key.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Batch_100b_Single_Key.txt rename to .github/workflows/load-tests-job-configs/python_CoGBK_Dataflow_Batch_100b_Single_Key.txt diff --git a/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Batch_10kB.txt b/.github/workflows/load-tests-job-configs/python_CoGBK_Dataflow_Batch_10kB.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Batch_10kB.txt rename to .github/workflows/load-tests-job-configs/python_CoGBK_Dataflow_Batch_10kB.txt diff --git a/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Batch_2MB.txt b/.github/workflows/load-tests-job-configs/python_CoGBK_Dataflow_Batch_2MB.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Batch_2MB.txt rename to .github/workflows/load-tests-job-configs/python_CoGBK_Dataflow_Batch_2MB.txt diff --git a/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Flink_Batch_100b_Multiple_Keys.txt b/.github/workflows/load-tests-job-configs/python_CoGBK_Dataflow_Flink_Batch_100b_Multiple_Keys.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Flink_Batch_100b_Multiple_Keys.txt rename to .github/workflows/load-tests-job-configs/python_CoGBK_Dataflow_Flink_Batch_100b_Multiple_Keys.txt diff --git a/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Flink_Batch_100b_Single_Key.txt b/.github/workflows/load-tests-job-configs/python_CoGBK_Dataflow_Flink_Batch_100b_Single_Key.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Flink_Batch_100b_Single_Key.txt rename to .github/workflows/load-tests-job-configs/python_CoGBK_Dataflow_Flink_Batch_100b_Single_Key.txt diff --git a/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Flink_Batch_10kB.txt b/.github/workflows/load-tests-job-configs/python_CoGBK_Dataflow_Flink_Batch_10kB.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Flink_Batch_10kB.txt rename to .github/workflows/load-tests-job-configs/python_CoGBK_Dataflow_Flink_Batch_10kB.txt diff --git a/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Streaming_100b_Multiple_Keys.txt b/.github/workflows/load-tests-job-configs/python_CoGBK_Dataflow_Streaming_100b_Multiple_Keys.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Streaming_100b_Multiple_Keys.txt rename to .github/workflows/load-tests-job-configs/python_CoGBK_Dataflow_Streaming_100b_Multiple_Keys.txt diff --git a/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Streaming_100b_Single_Key.txt b/.github/workflows/load-tests-job-configs/python_CoGBK_Dataflow_Streaming_100b_Single_Key.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Streaming_100b_Single_Key.txt rename to .github/workflows/load-tests-job-configs/python_CoGBK_Dataflow_Streaming_100b_Single_Key.txt diff --git a/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Streaming_10kB.txt b/.github/workflows/load-tests-job-configs/python_CoGBK_Dataflow_Streaming_10kB.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Streaming_10kB.txt rename to .github/workflows/load-tests-job-configs/python_CoGBK_Dataflow_Streaming_10kB.txt diff --git a/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Streaming_2MB.txt b/.github/workflows/load-tests-job-configs/python_CoGBK_Dataflow_Streaming_2MB.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Streaming_2MB.txt rename to .github/workflows/load-tests-job-configs/python_CoGBK_Dataflow_Streaming_2MB.txt From d25905eb115e981486b2c175768b03def4fd0a4b Mon Sep 17 00:00:00 2001 From: Yi Hu <yathu@google.com> Date: Thu, 19 Oct 2023 09:57:05 -0400 Subject: [PATCH 193/435] Fix Python PreCommit GHA recommendations_ai teardown (#29067) --- .../apache_beam/ml/gcp/recommendations_ai_test_it.py | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/sdks/python/apache_beam/ml/gcp/recommendations_ai_test_it.py b/sdks/python/apache_beam/ml/gcp/recommendations_ai_test_it.py index 0ae68a298421..9f739de7883d 100644 --- a/sdks/python/apache_beam/ml/gcp/recommendations_ai_test_it.py +++ b/sdks/python/apache_beam/ml/gcp/recommendations_ai_test_it.py @@ -70,9 +70,12 @@ def extract_prediction(response): recommendationengine is None, "Recommendations AI dependencies not installed.") class RecommendationAIIT(unittest.TestCase): + test_ran = False + def test_create_catalog_item(self): with TestPipeline(is_integration_test=True) as p: + RecommendationAIIT.test_ran = True output = ( p | 'Create data' >> beam.Create([CATALOG_ITEM]) | 'Create CatalogItem' >> @@ -85,6 +88,7 @@ def test_create_user_event(self): USER_EVENT = {"event_type": "page-visit", "user_info": {"visitor_id": "1"}} with TestPipeline(is_integration_test=True) as p: + RecommendationAIIT.test_ran = True output = ( p | 'Create data' >> beam.Create([USER_EVENT]) | 'Create UserEvent' >> recommendations_ai.WriteUserEvent(project=GCP_TEST_PROJECT) @@ -96,6 +100,7 @@ def test_predict(self): USER_EVENT = {"event_type": "page-visit", "user_info": {"visitor_id": "1"}} with TestPipeline(is_integration_test=True) as p: + RecommendationAIIT.test_ran = True output = ( p | 'Create data' >> beam.Create([USER_EVENT]) | 'Predict UserEvent' >> recommendations_ai.PredictUserEvent( @@ -106,6 +111,9 @@ def test_predict(self): @classmethod def tearDownClass(cls): + if not cls.test_ran: + raise unittest.SkipTest('all test skipped') + client = recommendationengine.CatalogServiceClient() parent = ( f'projects/{GCP_TEST_PROJECT}/locations/' From 423ea2360a44dbd4eae4f1d55657e71c035d4a9b Mon Sep 17 00:00:00 2001 From: Vitaly Terentyev <vitaly.terentyev@akvelon.com> Date: Thu, 19 Oct 2023 17:59:56 +0400 Subject: [PATCH 194/435] Add InfluxDB credentials (#29071) --- .github/workflows/beam_CloudML_Benchmarks_Dataflow.yml | 2 ++ .github/workflows/beam_Java_LoadTests_Combine_Smoke.yml | 2 ++ .github/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml | 2 ++ .github/workflows/beam_LoadTests_Go_CoGBK_Flink_batch.yml | 2 ++ .github/workflows/beam_LoadTests_Go_Combine_Dataflow_Batch.yml | 2 ++ .github/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml | 2 ++ .github/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml | 2 ++ .github/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml | 2 ++ .github/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml | 2 ++ .github/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml | 2 ++ .../workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml | 2 ++ .github/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml | 2 ++ .github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml | 2 ++ .../workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml | 2 ++ ...beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml | 2 ++ ..._LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml | 2 ++ ...beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml | 2 ++ .../workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml | 2 ++ .../beam_LoadTests_Java_Combine_Dataflow_Streaming.yml | 2 ++ ...am_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml | 2 ++ .github/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml | 2 ++ .../workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml | 2 ++ .../beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml | 2 ++ .../beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml | 2 ++ .../beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml | 2 ++ .../beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml | 2 ++ .github/workflows/beam_LoadTests_Java_GBK_Smoke.yml | 2 ++ .../beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch.yml | 2 ++ .github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml | 2 ++ .../workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml | 2 ++ ...beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions.yml | 2 ++ ..._LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions.yml | 2 ++ ...beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml | 2 ++ .../workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml | 2 ++ .../beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml | 2 ++ .github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml | 2 ++ .../workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml | 2 ++ .../beam_LoadTests_Python_Combine_Dataflow_Streaming.yml | 2 ++ .github/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml | 2 ++ .../workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml | 2 ++ .../beam_LoadTests_Python_FnApiRunner_Microbenchmark.yml | 2 ++ .github/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml | 2 ++ .../workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml | 2 ++ .github/workflows/beam_LoadTests_Python_GBK_Flink_Batch.yml | 2 ++ .../beam_LoadTests_Python_GBK_reiterate_Dataflow_Batch.yml | 2 ++ .../beam_LoadTests_Python_GBK_reiterate_Dataflow_Streaming.yml | 2 ++ .../workflows/beam_LoadTests_Python_ParDo_Dataflow_Batch.yml | 2 ++ .../beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml | 2 ++ .github/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml | 2 ++ .../workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml | 2 ++ .../beam_LoadTests_Python_SideInput_Dataflow_Batch.yml | 2 ++ .github/workflows/beam_LoadTests_Python_Smoke.yml | 2 ++ .github/workflows/beam_PerformanceTests_AvroIOIT.yml | 2 ++ .github/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml | 2 ++ .../beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml | 2 ++ .../beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml | 2 ++ .../beam_PerformanceTests_BigQueryIO_Streaming_Java.yml | 2 ++ .../workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml | 2 ++ .../beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml | 2 ++ .github/workflows/beam_PerformanceTests_Cdap.yml | 2 ++ .github/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml | 2 ++ .../beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml | 2 ++ .github/workflows/beam_PerformanceTests_HadoopFormat.yml | 2 ++ .github/workflows/beam_PerformanceTests_JDBC.yml | 2 ++ .github/workflows/beam_PerformanceTests_Kafka_IO.yml | 2 ++ .github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml | 2 ++ .../workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml | 2 ++ .github/workflows/beam_PerformanceTests_MongoDBIO_IT.yml | 2 ++ .github/workflows/beam_PerformanceTests_ParquetIOIT.yml | 2 ++ .github/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml | 2 ++ .../beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml | 2 ++ .../beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml | 2 ++ .../beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml | 2 ++ .../beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml | 2 ++ .github/workflows/beam_PerformanceTests_SparkReceiver_IO.yml | 2 ++ .github/workflows/beam_PerformanceTests_TFRecordIOIT.yml | 2 ++ .github/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml | 2 ++ .github/workflows/beam_PerformanceTests_TextIOIT.yml | 2 ++ .github/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml | 2 ++ .github/workflows/beam_PerformanceTests_TextIOIT_Python.yml | 2 ++ .../beam_PerformanceTests_WordCountIT_PythonVersions.yml | 2 ++ .github/workflows/beam_PerformanceTests_XmlIOIT.yml | 2 ++ .github/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml | 2 ++ .../workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml | 2 ++ 84 files changed, 168 insertions(+) diff --git a/.github/workflows/beam_CloudML_Benchmarks_Dataflow.yml b/.github/workflows/beam_CloudML_Benchmarks_Dataflow.yml index c34d9b5118e6..df26f1a5cfc2 100644 --- a/.github/workflows/beam_CloudML_Benchmarks_Dataflow.yml +++ b/.github/workflows/beam_CloudML_Benchmarks_Dataflow.yml @@ -47,6 +47,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} jobs: beam_CloudML_Benchmarks_Dataflow: diff --git a/.github/workflows/beam_Java_LoadTests_Combine_Smoke.yml b/.github/workflows/beam_Java_LoadTests_Combine_Smoke.yml index 9ec96e51d813..2eb7f061ce99 100644 --- a/.github/workflows/beam_Java_LoadTests_Combine_Smoke.yml +++ b/.github/workflows/beam_Java_LoadTests_Combine_Smoke.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} jobs: beam_Java_LoadTests_Combine_Smoke: diff --git a/.github/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml index 5efd4b749910..4a7b6949ee44 100644 --- a/.github/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} jobs: beam_LoadTests_Go_CoGBK_Dataflow_Batch: diff --git a/.github/workflows/beam_LoadTests_Go_CoGBK_Flink_batch.yml b/.github/workflows/beam_LoadTests_Go_CoGBK_Flink_batch.yml index 659c7ae0fc29..f3a4cba5fce1 100644 --- a/.github/workflows/beam_LoadTests_Go_CoGBK_Flink_batch.yml +++ b/.github/workflows/beam_LoadTests_Go_CoGBK_Flink_batch.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} GCLOUD_ZONE: us-central1-a CLUSTER_NAME: beam-loadtests-go-cogbk-flink-batch-${{ github.run_id }} GCS_BUCKET: gs://beam-flink-cluster diff --git a/.github/workflows/beam_LoadTests_Go_Combine_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Go_Combine_Dataflow_Batch.yml index 2abaa3240152..9568d0cfcff6 100644 --- a/.github/workflows/beam_LoadTests_Go_Combine_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_Combine_Dataflow_Batch.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} jobs: beam_LoadTests_Go_Combine_Dataflow_Batch: diff --git a/.github/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml index 03d72fbe6f13..f368707f1bd9 100644 --- a/.github/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} GCLOUD_ZONE: us-central1-a CLUSTER_NAME: beam-loadtests-go-combine-flink-batch-${{ github.run_id }} GCS_BUCKET: gs://beam-flink-cluster diff --git a/.github/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml index 33850101e918..01493d467b35 100644 --- a/.github/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} jobs: beam_LoadTests_Go_GBK_Dataflow_Batch: diff --git a/.github/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml index 4ab802ced6ca..53982b4175f4 100644 --- a/.github/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} GCLOUD_ZONE: us-central1-a CLUSTER_NAME: beam-loadtests-go-gbk-flink-batch-${{ github.run_id }} GCS_BUCKET: gs://beam-flink-cluster diff --git a/.github/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml index 058e15783bd8..73e21c4d6c56 100644 --- a/.github/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} jobs: beam_LoadTests_Go_ParDo_Dataflow_Batch: diff --git a/.github/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml index 3fa326c733bd..9fe817f01566 100644 --- a/.github/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} GCLOUD_ZONE: us-central1-a CLUSTER_NAME: beam-loadtests-go-pardo-flink-batch-${{ github.run_id }} GCS_BUCKET: gs://beam-flink-cluster diff --git a/.github/workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml index 65b71bb79649..4aa8d7f7c3de 100644 --- a/.github/workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} jobs: beam_LoadTests_Go_SideInput_Dataflow_Batch: diff --git a/.github/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml index 72162de5155b..53c100fff2c8 100644 --- a/.github/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} GCLOUD_ZONE: us-central1-a CLUSTER_NAME: beam-loadtests-go-sideinput-flink-batch-${{ github.run_id }} GCS_BUCKET: gs://beam-flink-cluster diff --git a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml index b1183364b459..705d8a85fa65 100644 --- a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} jobs: beam_LoadTests_Java_CoGBK_Dataflow_Batch: diff --git a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml index 84250af98aa1..4ecca6231401 100644 --- a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} jobs: beam_LoadTests_Java_CoGBK_Dataflow_Streaming: diff --git a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml index 79f9880094da..fea2eb18ff14 100644 --- a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml +++ b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} jobs: beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions: diff --git a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml index 5024ae3e7f1f..b81ea53ab5fd 100644 --- a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml +++ b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} jobs: beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions: diff --git a/.github/workflows/beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml b/.github/workflows/beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml index 7ee5243555a1..f4b4b6d9946e 100644 --- a/.github/workflows/beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml +++ b/.github/workflows/beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} jobs: beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch: diff --git a/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml index b87c2c11f1d4..1702d6c75184 100644 --- a/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} jobs: beam_LoadTests_Java_Combine_Dataflow_Batch: diff --git a/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Streaming.yml index 5d372505bd30..e0e033939d3f 100644 --- a/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Streaming.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} beam_LoadTests_Java_Combine_Dataflow_Streaming_test_arguments_1: '' beam_LoadTests_Java_Combine_Dataflow_Streaming_test_arguments_2: '' beam_LoadTests_Java_Combine_Dataflow_Streaming_test_arguments_3: '' diff --git a/.github/workflows/beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml b/.github/workflows/beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml index 62b701b95f50..26c1a4975a7c 100644 --- a/.github/workflows/beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml +++ b/.github/workflows/beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch_test_arguments_1: '' beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch_test_arguments_2: '' beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch_test_arguments_3: '' diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml index 9eece637f8f4..fd86521e592d 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} jobs: beam_LoadTests_Java_GBK_Dataflow_Batch: diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml index d2850b4aec58..63bc91bb1599 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} jobs: beam_LoadTests_Java_GBK_Dataflow_Streaming: diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml index bc5760d99180..3a8d47ccb642 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} jobs: beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11: diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml index 82929ad6783a..414b2e0d660c 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} jobs: beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17: diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml index c9b278d5eed5..bfd8dfe09d9c 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} jobs: beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11: diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml index afedbd706b10..22df3fe88d26 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} jobs: beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17: diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Smoke.yml b/.github/workflows/beam_LoadTests_Java_GBK_Smoke.yml index 15096bb8e4aa..d41ee990a6ae 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_Smoke.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_Smoke.yml @@ -43,6 +43,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} jobs: beam_LoadTests_Java_GBK_Smoke: diff --git a/.github/workflows/beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch.yml b/.github/workflows/beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch.yml index 0ebfed7d477c..9f057bba3d3e 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} jobs: beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch: diff --git a/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml index df9d3d19c485..3b19c6a6fa70 100644 --- a/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} beam_LoadTests_Java_ParDo_Dataflow_Batch_test_arguments_1: '' beam_LoadTests_Java_ParDo_Dataflow_Batch_test_arguments_2: '' beam_LoadTests_Java_ParDo_Dataflow_Batch_test_arguments_3: '' diff --git a/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml index 4505ac0953c7..1f91207fee55 100644 --- a/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} beam_LoadTests_Java_ParDo_Dataflow_Streaming_test_arguments_1: '' beam_LoadTests_Java_ParDo_Dataflow_Streaming_test_arguments_2: '' beam_LoadTests_Java_ParDo_Dataflow_Streaming_test_arguments_3: '' diff --git a/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions.yml b/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions.yml index c9342847d806..a283310ff1d0 100644 --- a/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions.yml +++ b/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} jobs: beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions: diff --git a/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions.yml b/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions.yml index 1c2751de4b9f..45ea975d1580 100644 --- a/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions.yml +++ b/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} jobs: beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions: diff --git a/.github/workflows/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml b/.github/workflows/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml index fceb880c24b2..45700ca56937 100644 --- a/.github/workflows/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml +++ b/.github/workflows/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch_test_arguments_1: '' beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch_test_arguments_2: '' beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch_test_arguments_3: '' diff --git a/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml index 9112246d89c0..bc1b41461bcc 100644 --- a/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} jobs: beam_LoadTests_Python_CoGBK_Dataflow_Batch: diff --git a/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml index 6b509e43741a..15d36839de1f 100644 --- a/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} jobs: beam_LoadTests_Python_CoGBK_Dataflow_Streaming: diff --git a/.github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml index 321769672742..e06ed4f94e4f 100644 --- a/.github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} GCLOUD_ZONE: us-central1-a CLUSTER_NAME: beam-loadtests-python-cogbk-flink-batch-${{ github.run_id }} GCS_BUCKET: gs://beam-flink-cluster diff --git a/.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml index a40a4a5dada0..c042cafab864 100644 --- a/.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} jobs: beam_LoadTests_Python_Combine_Dataflow_Batch: diff --git a/.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Streaming.yml index df8ff7a5596c..c9f05083cfa8 100644 --- a/.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Streaming.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} jobs: beam_LoadTests_Python_Combine_Dataflow_Streaming: diff --git a/.github/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml index f1e9424cd6ca..3ab3e410faef 100644 --- a/.github/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} GCLOUD_ZONE: us-central1-a CLUSTER_NAME: beam-loadtests-py-cmb-flink-batch-${{ github.run_id }} GCS_BUCKET: gs://beam-flink-cluster diff --git a/.github/workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml b/.github/workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml index c7b8b45aa8f9..d90862ddef3c 100644 --- a/.github/workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} GCLOUD_ZONE: us-central1-a CLUSTER_NAME: beam-loadtests-py-cmb-flink-streaming-${{ github.run_id }} GCS_BUCKET: gs://beam-flink-cluster diff --git a/.github/workflows/beam_LoadTests_Python_FnApiRunner_Microbenchmark.yml b/.github/workflows/beam_LoadTests_Python_FnApiRunner_Microbenchmark.yml index 3077376a4da9..1483f59472ec 100644 --- a/.github/workflows/beam_LoadTests_Python_FnApiRunner_Microbenchmark.yml +++ b/.github/workflows/beam_LoadTests_Python_FnApiRunner_Microbenchmark.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} jobs: beam_LoadTests_Python_FnApiRunner_Microbenchmark: diff --git a/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml index 537b14125cd7..92a9d82b79aa 100644 --- a/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} jobs: beam_LoadTests_Python_GBK_Dataflow_Batch: diff --git a/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml index 6d0b2ca6f995..9a123f7b4aac 100644 --- a/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} jobs: beam_LoadTests_Python_GBK_Dataflow_Streaming: diff --git a/.github/workflows/beam_LoadTests_Python_GBK_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Python_GBK_Flink_Batch.yml index f153eb754a31..4074a0a7417c 100644 --- a/.github/workflows/beam_LoadTests_Python_GBK_Flink_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_GBK_Flink_Batch.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} GCLOUD_ZONE: us-central1-a CLUSTER_NAME: beam-loadtests-py-gbk-flk-batch-${{ github.run_id }} GCS_BUCKET: gs://beam-flink-cluster diff --git a/.github/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Batch.yml index c262061ae87d..556e1746f3ee 100644 --- a/.github/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Batch.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} jobs: beam_LoadTests_Python_GBK_reiterate_Dataflow_Batch: diff --git a/.github/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Streaming.yml index e0921e72558e..a1f37bae5133 100644 --- a/.github/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Streaming.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} jobs: beam_LoadTests_Python_GBK_reiterate_Dataflow_Streaming: diff --git a/.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Batch.yml index caff1480d0ec..472d494cd87b 100644 --- a/.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Batch.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} jobs: beam_LoadTests_Python_ParDo_Dataflow_Batch: diff --git a/.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml index 795c4c390a82..cbcd28149aea 100644 --- a/.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} jobs: beam_LoadTests_Python_ParDo_Dataflow_Streaming: diff --git a/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml index 032a221d83db..d78fe22b2858 100644 --- a/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} GCLOUD_ZONE: us-central1-a CLUSTER_NAME: beam-loadtests-python-pardo-flink-batch-${{ github.run_id }} GCS_BUCKET: gs://beam-flink-cluster diff --git a/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml b/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml index 916ff208c910..e30ab4d699cc 100644 --- a/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} GCLOUD_ZONE: us-central1-a CLUSTER_NAME: beam-loadtests-python-pardo-flink-stream-${{ github.run_id }} GCS_BUCKET: gs://beam-flink-cluster diff --git a/.github/workflows/beam_LoadTests_Python_SideInput_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Python_SideInput_Dataflow_Batch.yml index 5632ce720921..f62f72343357 100644 --- a/.github/workflows/beam_LoadTests_Python_SideInput_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_SideInput_Dataflow_Batch.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} jobs: beam_LoadTests_Python_SideInput_Dataflow_Batch: diff --git a/.github/workflows/beam_LoadTests_Python_Smoke.yml b/.github/workflows/beam_LoadTests_Python_Smoke.yml index 3a8d5b2e40b8..77d4d65bfe9a 100644 --- a/.github/workflows/beam_LoadTests_Python_Smoke.yml +++ b/.github/workflows/beam_LoadTests_Python_Smoke.yml @@ -43,6 +43,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} jobs: beam_LoadTests_Python_Smoke: diff --git a/.github/workflows/beam_PerformanceTests_AvroIOIT.yml b/.github/workflows/beam_PerformanceTests_AvroIOIT.yml index 8257eea64050..67b00a98ce8e 100644 --- a/.github/workflows/beam_PerformanceTests_AvroIOIT.yml +++ b/.github/workflows/beam_PerformanceTests_AvroIOIT.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} beam_PerformanceTests_AvroIOIT_test_arguments_1: '' jobs: diff --git a/.github/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml b/.github/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml index 6ff0ae616b9d..868d1b283636 100644 --- a/.github/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml +++ b/.github/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} beam_PerformanceTests_AvroIOIT_HDFS_test_arguments_1: '' jobs: diff --git a/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml b/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml index 35e231313b73..cd56c66c5a04 100644 --- a/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml +++ b/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} jobs: beam_PerformanceTests_BigQueryIO_Batch_Java_Avro: diff --git a/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml b/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml index f4f40917f439..2e2e00d40ab4 100644 --- a/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml +++ b/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} jobs: beam_PerformanceTests_BigQueryIO_Batch_Java_Json: diff --git a/.github/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml b/.github/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml index 4fea44cc99ee..95c4fa86b4e0 100644 --- a/.github/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml +++ b/.github/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} jobs: beam_PerformanceTests_BigQueryIO_Streaming_Java: diff --git a/.github/workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml b/.github/workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml index 176e619ff45f..a574cdc4bdb5 100644 --- a/.github/workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml +++ b/.github/workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} beam_PerformanceTests_BiqQueryIO_Read_Python_test_arguments_1: '' jobs: diff --git a/.github/workflows/beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml b/.github/workflows/beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml index 86c3d972560f..c906c6efc575 100644 --- a/.github/workflows/beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml +++ b/.github/workflows/beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} beam_PerformanceTests_BiqQueryIO_Write_Python_Batch_test_arguments_1: '' jobs: diff --git a/.github/workflows/beam_PerformanceTests_Cdap.yml b/.github/workflows/beam_PerformanceTests_Cdap.yml index 1b4c2086834f..804ea0a8f454 100644 --- a/.github/workflows/beam_PerformanceTests_Cdap.yml +++ b/.github/workflows/beam_PerformanceTests_Cdap.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} beam_PerformanceTests_Cdap_test_arguments_1: '' jobs: diff --git a/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml b/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml index 7705cd132e95..cd1f2433284b 100644 --- a/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml +++ b/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} jobs: beam_PerformanceTests_Compressed_TextIOIT: diff --git a/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml b/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml index daecc89ef85a..885d6c793c72 100644 --- a/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml +++ b/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} jobs: beam_PerformanceTests_Compressed_TextIOIT_HDFS: diff --git a/.github/workflows/beam_PerformanceTests_HadoopFormat.yml b/.github/workflows/beam_PerformanceTests_HadoopFormat.yml index ee42cc207e6f..060bb24811c3 100644 --- a/.github/workflows/beam_PerformanceTests_HadoopFormat.yml +++ b/.github/workflows/beam_PerformanceTests_HadoopFormat.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} beam_PerformanceTests_HadoopFormat_test_arguments_1: '' jobs: diff --git a/.github/workflows/beam_PerformanceTests_JDBC.yml b/.github/workflows/beam_PerformanceTests_JDBC.yml index 1fb6b7f74bb9..8d7dba3eb05e 100644 --- a/.github/workflows/beam_PerformanceTests_JDBC.yml +++ b/.github/workflows/beam_PerformanceTests_JDBC.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} beam_PerformanceTests_JDBC_test_arguments_1: '' jobs: diff --git a/.github/workflows/beam_PerformanceTests_Kafka_IO.yml b/.github/workflows/beam_PerformanceTests_Kafka_IO.yml index 314297c53ef0..a69f3400f80a 100644 --- a/.github/workflows/beam_PerformanceTests_Kafka_IO.yml +++ b/.github/workflows/beam_PerformanceTests_Kafka_IO.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} beam_PerformanceTests_Kafka_IO_test_arguments_1: '' beam_PerformanceTests_Kafka_IO_test_arguments_2: '' diff --git a/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml b/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml index f077a7c96f85..0559195a5a7d 100644 --- a/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml +++ b/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} beam_PerformanceTests_ManyFiles_TextIOIT_test_arguments_1: '' jobs: diff --git a/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml b/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml index 748d5f30996b..33db6119c066 100644 --- a/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml +++ b/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} beam_PerformanceTests_ManyFiles_TextIOIT_HDFS_test_arguments_1: '' jobs: diff --git a/.github/workflows/beam_PerformanceTests_MongoDBIO_IT.yml b/.github/workflows/beam_PerformanceTests_MongoDBIO_IT.yml index 4bc36024f59b..3b874d819d54 100644 --- a/.github/workflows/beam_PerformanceTests_MongoDBIO_IT.yml +++ b/.github/workflows/beam_PerformanceTests_MongoDBIO_IT.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} beam_PerformanceTests_MongoDBIO_IT_test_arguments_1: '' jobs: diff --git a/.github/workflows/beam_PerformanceTests_ParquetIOIT.yml b/.github/workflows/beam_PerformanceTests_ParquetIOIT.yml index 799acfc4f7f2..2c822bb3a12f 100644 --- a/.github/workflows/beam_PerformanceTests_ParquetIOIT.yml +++ b/.github/workflows/beam_PerformanceTests_ParquetIOIT.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} jobs: beam_PerformanceTests_ParquetIOIT: diff --git a/.github/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml b/.github/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml index 41f1d2135217..ac3e194efd46 100644 --- a/.github/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml +++ b/.github/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} jobs: beam_PerformanceTests_ParquetIOIT_HDFS: diff --git a/.github/workflows/beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml b/.github/workflows/beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml index c252e716830c..31e0c995aded 100644 --- a/.github/workflows/beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml +++ b/.github/workflows/beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} beam_PerformanceTests_PubsubIOIT_Python_Streaming_test_arguments_1: '' jobs: diff --git a/.github/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml b/.github/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml index 0623c3717215..8eab9e68a3e1 100644 --- a/.github/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml +++ b/.github/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} beam_PerformanceTests_SQLBigQueryIO_Batch_Java_test_arguments_1: '' jobs: diff --git a/.github/workflows/beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml b/.github/workflows/beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml index 2a4a3ff18e39..ddea3f6a92ee 100644 --- a/.github/workflows/beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml +++ b/.github/workflows/beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} beam_PerformanceTests_SpannerIO_Read_2GB_Python_test_arguments_1: '' jobs: diff --git a/.github/workflows/beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml b/.github/workflows/beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml index 833d7efac926..34066b890eeb 100644 --- a/.github/workflows/beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml +++ b/.github/workflows/beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch_test_arguments_1: '' jobs: diff --git a/.github/workflows/beam_PerformanceTests_SparkReceiver_IO.yml b/.github/workflows/beam_PerformanceTests_SparkReceiver_IO.yml index 6c75c93e8426..daab6a20a546 100644 --- a/.github/workflows/beam_PerformanceTests_SparkReceiver_IO.yml +++ b/.github/workflows/beam_PerformanceTests_SparkReceiver_IO.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} beam_PerformanceTests_SparkReceiver_IO_test_arguments_1: '' jobs: diff --git a/.github/workflows/beam_PerformanceTests_TFRecordIOIT.yml b/.github/workflows/beam_PerformanceTests_TFRecordIOIT.yml index 65c58c87ddbd..7c2a1011694b 100644 --- a/.github/workflows/beam_PerformanceTests_TFRecordIOIT.yml +++ b/.github/workflows/beam_PerformanceTests_TFRecordIOIT.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} beam_PerformanceTests_TFRecordIOIT_test_arguments_1: '' jobs: diff --git a/.github/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml b/.github/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml index 0cc337901211..32f31a29567b 100644 --- a/.github/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml +++ b/.github/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml @@ -47,6 +47,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} beam_PerformanceTests_TFRecordIOIT_HDFS_test_arguments_1: '' jobs: diff --git a/.github/workflows/beam_PerformanceTests_TextIOIT.yml b/.github/workflows/beam_PerformanceTests_TextIOIT.yml index b1b0338ba92f..477a8a08cac7 100644 --- a/.github/workflows/beam_PerformanceTests_TextIOIT.yml +++ b/.github/workflows/beam_PerformanceTests_TextIOIT.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} jobs: beam_PerformanceTests_TextIOIT: diff --git a/.github/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml b/.github/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml index 666d049cb4ee..8d1683cd1e35 100644 --- a/.github/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml +++ b/.github/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} jobs: beam_PerformanceTests_TextIOIT_HDFS: diff --git a/.github/workflows/beam_PerformanceTests_TextIOIT_Python.yml b/.github/workflows/beam_PerformanceTests_TextIOIT_Python.yml index fa612f8b2401..64d4cad53327 100644 --- a/.github/workflows/beam_PerformanceTests_TextIOIT_Python.yml +++ b/.github/workflows/beam_PerformanceTests_TextIOIT_Python.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} jobs: beam_PerformanceTests_TextIOIT_Python: diff --git a/.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml b/.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml index 65d7c564da07..dee49eb0b5e3 100644 --- a/.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml +++ b/.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} beam_PerformanceTests_WordCountIT_PythonVersions_test_arguments_1: '' jobs: diff --git a/.github/workflows/beam_PerformanceTests_XmlIOIT.yml b/.github/workflows/beam_PerformanceTests_XmlIOIT.yml index 0423b7ba1a71..33849aecf994 100644 --- a/.github/workflows/beam_PerformanceTests_XmlIOIT.yml +++ b/.github/workflows/beam_PerformanceTests_XmlIOIT.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} beam_PerformanceTests_XmlIOIT_test_arguments_1: '' jobs: diff --git a/.github/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml b/.github/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml index 0197e8cb9b5f..ff130dc4d53d 100644 --- a/.github/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml +++ b/.github/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} beam_PerformanceTests_XmlIOIT_HDFS_test_arguments_1: '' jobs: diff --git a/.github/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml b/.github/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml index aaffa3ca1dc1..bc62013c649d 100644 --- a/.github/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml +++ b/.github/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml @@ -45,6 +45,8 @@ env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} jobs: beam_PerformanceTests_xlang_KafkaIO_Python: From 4f4a58a2325e6517dd27ffc8548f825a1de9079f Mon Sep 17 00:00:00 2001 From: Andrey Devyatkin <andrey.9evyatkin@gmail.com> Date: Thu, 19 Oct 2023 16:39:41 +0200 Subject: [PATCH 195/435] fix workflows (#29009) --- ...Commit_Java_ValidatesRunner_Dataflow_JavaVersions.yml | 8 +++----- .../beam_PostCommit_Java_ValidatesRunner_Direct.yml | 4 +--- ...stCommit_Java_ValidatesRunner_Direct_JavaVersions.yml | 8 +++----- .../beam_PostCommit_Java_ValidatesRunner_Flink.yml | 1 + ...beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml | 9 ++++----- ...beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml | 8 +++----- .../beam_PostCommit_Java_ValidatesRunner_ULR.yml | 4 +--- .../workflows/beam_PostCommit_Python_Examples_Flink.yml | 2 +- ...eam_PostCommit_Python_ValidatesContainer_Dataflow.yml | 4 +++- ...Commit_Python_ValidatesContainer_Dataflow_With_RC.yml | 4 +++- 10 files changed, 23 insertions(+), 29 deletions(-) diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml index 67bc4e6a3994..a6afa04f565d 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml @@ -79,11 +79,9 @@ jobs: ${{ matrix.java_version }} 8 - name: run jar Java${{ matrix.java_version }} script - uses: ./.github/actions/gradle-command-self-hosted-action - with: - gradle-command: :runners:google-cloud-dataflow-java:testJar :runners:google-cloud-dataflow-java:worker:shadowJar - arguments: | - -Dorg.gradle.java.home=$JAVA_HOME_8_X64 \ + run: | + ./gradlew runners:google-cloud-dataflow-java:testJar :runners:google-cloud-dataflow-java:worker:shadowJar \ + -Dorg.gradle.java.home=$JAVA_HOME_8_X64 - name: run validatesRunner Java${{ matrix.java_version }} script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml index af7a4379ad5e..cf62a7894658 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml @@ -75,9 +75,7 @@ jobs: distribution: 'zulu' java-version: '8' - name: run validatesRunner script - uses: ./.github/actions/gradle-command-self-hosted-action - with: - gradle-command: :runners:direct-java:validatesRunner + run: ./gradlew :runners:direct-java:validatesRunner - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 if: failure() diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml index c9da0d7bd604..2d879fa7d631 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml @@ -79,11 +79,9 @@ jobs: ${{ matrix.java_version }} 8 - name: run jar Java${{ matrix.java_version }} script - uses: ./.github/actions/gradle-command-self-hosted-action - with: - gradle-command: :runners:direct-java:shadowJar :runners:direct-java:shadowTestJar - arguments: | - -Dorg.gradle.java.home=$JAVA_HOME_8_X64 \ + run: | + ./gradlew :runners:direct-java:shadowJar :runners:direct-java:shadowTestJar \ + -Dorg.gradle.java.home=$JAVA_HOME_8_X64 - name: run validatesRunner Java${{ matrix.java_version }} script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml index 7202b6fa33ae..03dd89592a73 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml @@ -86,6 +86,7 @@ jobs: uses: EnricoMi/publish-unit-test-result-action@v2 if: always() with: + large_files: true commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml index 2f2e4f15abe6..e96f8efb9aeb 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml @@ -77,11 +77,9 @@ jobs: 11 8 - name: run jar Java8 script - uses: ./.github/actions/gradle-command-self-hosted-action - with: - gradle-command: :runners:flink:1.15:jar :runners:flink:1.15:testJar - arguments: | - -Dorg.gradle.java.home=$JAVA_HOME_8_X64 \ + run: | + ./gradlew :runners:flink:1.15:jar :runners:flink:1.15:testJar \ + -Dorg.gradle.java.home=$JAVA_HOME_8_X64 - name: run validatesRunner Java11 script uses: ./.github/actions/gradle-command-self-hosted-action with: @@ -107,6 +105,7 @@ jobs: uses: EnricoMi/publish-unit-test-result-action@v2 if: always() with: + large_files: true commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml index cc9f51113124..1f82a4568274 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml @@ -77,11 +77,9 @@ jobs: 11 8 - name: run jar Java8 script - uses: ./.github/actions/gradle-command-self-hosted-action - with: - gradle-command: :runners:spark:3:jar :runners:spark:3:testJar - arguments: | - -Dorg.gradle.java.home=$JAVA_HOME_8_X64 \ + run: | + ./gradlew :runners:spark:3:jar :runners:spark:3:testJar \ + -Dorg.gradle.java.home=$JAVA_HOME_8_X64 - name: run validatesRunner Java11 script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml index 9ce3655c6fc9..9d42ecda4f38 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml @@ -77,9 +77,7 @@ jobs: with: python-version: '3.8' - name: run ulrLoopbackValidatesRunner script - uses: ./.github/actions/gradle-command-self-hosted-action - with: - gradle-command: :runners:portability:java:ulrLoopbackValidatesRunner + run: ./gradlew :runners:portability:java:ulrLoopbackValidatesRunner - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 if: failure() diff --git a/.github/workflows/beam_PostCommit_Python_Examples_Flink.yml b/.github/workflows/beam_PostCommit_Python_Examples_Flink.yml index 46df55646776..2e4db6192501 100644 --- a/.github/workflows/beam_PostCommit_Python_Examples_Flink.yml +++ b/.github/workflows/beam_PostCommit_Python_Examples_Flink.yml @@ -53,7 +53,7 @@ jobs: github.event_name == 'workflow_dispatch' || startsWith(github.event.comment.body, 'Run Python Examples_Flink') runs-on: [self-hosted, ubuntu-20.04, main] - timeout-minutes: 120 + timeout-minutes: 240 name: ${{ matrix.job_name }} (${{ matrix.job_phrase }} ${{ matrix.python_version }}) strategy: fail-fast: false diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml index 1bd547ab3774..51b22fa66a0e 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml @@ -72,7 +72,9 @@ jobs: - name: Setup environment uses: ./.github/actions/setup-environment-action with: - java-version: 8 + java-version: | + 11 + 8 python-version: ${{ matrix.python_version }} - name: Set PY_VER_CLEAN id: set_py_ver_clean diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml b/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml index 2c1b1c45227a..33fa6b53e23f 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml @@ -72,7 +72,9 @@ jobs: - name: Setup environment uses: ./.github/actions/setup-environment-action with: - java-version: 8 + java-version: | + 11 + 8 python-version: ${{ matrix.python_version }} - name: Set PY_VER_CLEAN id: set_py_ver_clean From 95be6b9bd004152dba0b1451c0acde316a5a63c8 Mon Sep 17 00:00:00 2001 From: Bruno Volpato <bvolpato@google.com> Date: Thu, 19 Oct 2023 10:42:14 -0400 Subject: [PATCH 196/435] [JdbcIO] Add fetchSize to the schema provider for partitioned reads (#29015) --- .../src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java | 6 ++++++ .../org/apache/beam/sdk/io/jdbc/JdbcSchemaIOProvider.java | 6 ++++++ 2 files changed, 12 insertions(+) diff --git a/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java b/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java index f8dad23d1fbd..e2a4a8e1072a 100644 --- a/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java +++ b/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java @@ -1288,6 +1288,12 @@ public ReadWithPartitions<T, PartitionColumnT> withPartitionColumn(String partit return toBuilder().setPartitionColumn(partitionColumn).build(); } + /** The number of rows to fetch from the database in the same {@link ResultSet} round-trip. */ + public ReadWithPartitions<T, PartitionColumnT> withFetchSize(int fetchSize) { + checkArgument(fetchSize > 0, "fetchSize can not be less than 1"); + return toBuilder().setFetchSize(fetchSize).build(); + } + /** Data output type is {@link Row}, and schema is auto-inferred from the database. */ public ReadWithPartitions<T, PartitionColumnT> withRowOutput() { return toBuilder().setUseBeamSchema(true).build(); diff --git a/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcSchemaIOProvider.java b/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcSchemaIOProvider.java index c68b33a02607..4b5dc0d7e24a 100644 --- a/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcSchemaIOProvider.java +++ b/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcSchemaIOProvider.java @@ -134,6 +134,12 @@ public PCollection<Row> expand(PBegin input) { if (partitions != null) { readRows = readRows.withNumPartitions(partitions); } + + @Nullable Short fetchSize = config.getInt16("fetchSize"); + if (fetchSize != null) { + readRows = readRows.withFetchSize(fetchSize); + } + return input.apply(readRows); } else { From 84deb33e38f20d29c8563a907f221963ccc72106 Mon Sep 17 00:00:00 2001 From: Yi Hu <yathu@google.com> Date: Thu, 19 Oct 2023 10:43:57 -0400 Subject: [PATCH 197/435] Add 'release/trigger_all_tests.json' to trigger path of mass comment PR (#29064) * Add 'release/trigger_all_tests.json' to trigger path of mass comment PR * Create GItHub Action PreCommit * Create missing Google-ads IO PreCommit * Create missing RequestResponse IO PreCommit * fix leftover * Fix leftover in phrase * Update contributor-docs/release-guide.md Co-authored-by: Danny McCormick <dannymccormick@google.com> --------- Co-authored-by: Danny McCormick <dannymccormick@google.com> --- .github/build.gradle | 61 +++++++++ .github/workflows/README.md | 3 + .../beam_PostCommit_BeamMetrics_Publish.yml | 1 - .../beam_PreCommit_CommunityMetrics.yml | 2 +- .github/workflows/beam_PreCommit_GHA.yml | 89 ++++++++++++++ .github/workflows/beam_PreCommit_Go.yml | 2 +- .../workflows/beam_PreCommit_GoPortable.yml | 2 +- .github/workflows/beam_PreCommit_GoPrism.yml | 2 +- .../workflows/beam_PreCommit_ItFramework.yml | 1 + .github/workflows/beam_PreCommit_Java.yml | 1 + ...it_Java_Amazon-Web-Services2_IO_Direct.yml | 1 + ...mit_Java_Amazon-Web-Services_IO_Direct.yml | 1 + .../beam_PreCommit_Java_Amqp_IO_Direct.yml | 1 + .../beam_PreCommit_Java_Azure_IO_Direct.yml | 1 + ...eam_PreCommit_Java_Cassandra_IO_Direct.yml | 1 + .../beam_PreCommit_Java_Cdap_IO_Direct.yml | 1 + ...am_PreCommit_Java_Clickhouse_IO_Direct.yml | 1 + .../beam_PreCommit_Java_Csv_IO_Direct.yml | 1 + ...beam_PreCommit_Java_Debezium_IO_Direct.yml | 1 + ...PreCommit_Java_ElasticSearch_IO_Direct.yml | 1 + .../beam_PreCommit_Java_Examples_Dataflow.yml | 1 + ...reCommit_Java_Examples_Dataflow_Java11.yml | 1 + ...reCommit_Java_Examples_Dataflow_Java17.yml | 1 + ...t_Java_File-schema-transform_IO_Direct.yml | 1 + .../beam_PreCommit_Java_Flink_Versions.yml | 1 + .../beam_PreCommit_Java_GCP_IO_Direct.yml | 1 + ...am_PreCommit_Java_Google-ads_IO_Direct.yml | 116 ++++++++++++++++++ .../beam_PreCommit_Java_HBase_IO_Direct.yml | 1 + ...beam_PreCommit_Java_HCatalog_IO_Direct.yml | 1 + .../beam_PreCommit_Java_Hadoop_IO_Direct.yml | 1 + .../beam_PreCommit_Java_IOs_Direct.yml | 1 + ...beam_PreCommit_Java_InfluxDb_IO_Direct.yml | 1 + .../beam_PreCommit_Java_JDBC_IO_Direct.yml | 1 + .../beam_PreCommit_Java_Jms_IO_Direct.yml | 1 + .../beam_PreCommit_Java_Kafka_IO_Direct.yml | 1 + .../beam_PreCommit_Java_Kinesis_IO_Direct.yml | 1 + .../beam_PreCommit_Java_Kudu_IO_Direct.yml | 1 + .../beam_PreCommit_Java_MongoDb_IO_Direct.yml | 1 + .../beam_PreCommit_Java_Mqtt_IO_Direct.yml | 1 + .../beam_PreCommit_Java_Neo4j_IO_Direct.yml | 1 + .../beam_PreCommit_Java_PVR_Flink_Batch.yml | 1 + .../beam_PreCommit_Java_PVR_Flink_Docker.yml | 1 + .../beam_PreCommit_Java_Parquet_IO_Direct.yml | 1 + .../beam_PreCommit_Java_Pulsar_IO_Direct.yml | 1 + ...beam_PreCommit_Java_RabbitMq_IO_Direct.yml | 1 + .../beam_PreCommit_Java_Redis_IO_Direct.yml | 1 + ...eCommit_Java_RequestResponse_IO_Direct.yml | 116 ++++++++++++++++++ ...m_PreCommit_Java_SingleStore_IO_Direct.yml | 1 + ...eam_PreCommit_Java_Snowflake_IO_Direct.yml | 1 + .../beam_PreCommit_Java_Solr_IO_Direct.yml | 1 + .../beam_PreCommit_Java_Spark3_Versions.yml | 1 + .../beam_PreCommit_Java_Splunk_IO_Direct.yml | 1 + .../beam_PreCommit_Java_Thrift_IO_Direct.yml | 1 + .../beam_PreCommit_Java_Tika_IO_Direct.yml | 1 + .../beam_PreCommit_Kotlin_Examples.yml | 1 + .../beam_PreCommit_Portable_Python.yml | 1 + .github/workflows/beam_PreCommit_Python.yml | 2 +- .../workflows/beam_PreCommit_PythonDocker.yml | 2 +- .../workflows/beam_PreCommit_PythonDocs.yml | 2 +- .../beam_PreCommit_PythonFormatter.yml | 2 +- .../workflows/beam_PreCommit_PythonLint.yml | 2 +- .../beam_PreCommit_Python_Coverage.yml | 2 +- .../beam_PreCommit_Python_Dataframes.yml | 2 +- .../beam_PreCommit_Python_Examples.yml | 2 +- .../beam_PreCommit_Python_Integration.yml | 2 +- .../beam_PreCommit_Python_PVR_Flink.yml | 1 + .../beam_PreCommit_Python_Runners.yml | 2 +- .../beam_PreCommit_Python_Transforms.yml | 2 +- .github/workflows/beam_PreCommit_SQL.yml | 2 +- .../workflows/beam_PreCommit_SQL_Java11.yml | 2 +- .../workflows/beam_PreCommit_SQL_Java17.yml | 2 +- .github/workflows/beam_PreCommit_Spotless.yml | 1 + .../workflows/beam_PreCommit_Typescript.yml | 2 +- .github/workflows/beam_PreCommit_Website.yml | 2 +- .../beam_PreCommit_Website_Stage_GCS.yml | 2 +- .../workflows/beam_PreCommit_Whitespace.yml | 2 +- contributor-docs/release-guide.md | 6 +- release/src/main/scripts/mass_comment.py | 21 +--- .../src/main/scripts/verify_release_build.sh | 8 +- settings.gradle.kts | 2 + 80 files changed, 467 insertions(+), 48 deletions(-) create mode 100644 .github/build.gradle create mode 100644 .github/workflows/beam_PreCommit_GHA.yml create mode 100644 .github/workflows/beam_PreCommit_Java_Google-ads_IO_Direct.yml create mode 100644 .github/workflows/beam_PreCommit_Java_RequestResponse_IO_Direct.yml diff --git a/.github/build.gradle b/.github/build.gradle new file mode 100644 index 000000000000..acfe7f3686a9 --- /dev/null +++ b/.github/build.gradle @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * License); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an AS IS BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +buildscript { + repositories { + mavenCentral() + } + dependencies { + classpath group: 'org.yaml', name: 'snakeyaml', version: '2.2' + } +} + +/** check that yml are valid */ +task check { + doLast { + fileTree("${project.projectDir}/workflows").matching { + include "*.yml" + include "*.yaml" + }.each { + def fname = it.getName() + // attempt load yml to make sure its valid + def workflow = new org.yaml.snakeyaml.Yaml().load(it.newInputStream()) + + // additional guards for running all tests functionality + // TODO(yathu) expand this also to post commits prior teardown Jenkins postcommits + if ( fname.startsWith("beam_PreCommit")) { + List paths + try { + paths = workflow.getAt(true).pull_request_target.paths as List + } catch (Exception e) { + throw new GradleException("Fail to get the trigger path for ${fname}. " + + "Make sure precommit has a pull_request_target trigger.", e) + } + // precommit should triggered by this specific file + if (paths != null && !paths.contains('release/trigger_all_tests.json')) { + throw new GradleException("Error validating ${fname}: " + + "Please add 'release/trigger_all_tests.json' to the trigger path for release verification run properly") + } + } + } + } +} + +task preCommit { + dependsOn check +} \ No newline at end of file diff --git a/.github/workflows/README.md b/.github/workflows/README.md index 3bd759106207..c17a6614acbb 100644 --- a/.github/workflows/README.md +++ b/.github/workflows/README.md @@ -278,6 +278,7 @@ Please note that jobs with matrix need to have matrix element in the comment. Ex | [ PostCommit XVR Samza ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Samza.yml) | N/A |`Run XVR_Samza PostCommit`| [![.github/workflows/beam_PostCommit_XVR_Samza](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Samza.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Samza.yml) | | [ PostCommit XVR Spark3 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Spark3.yml) | N/A |`Run XVR_Spark3 PostCommit`| [![.github/workflows/beam_PostCommit_XVR_Spark3](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Spark3.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Spark3.yml) | | [ PreCommit Community Metrics ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_CommunityMetrics.yml) | N/A |`Run CommunityMetrics PreCommit`| [![.github/workflows/beam_PreCommit_CommunityMetrics.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_CommunityMetrics.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_CommunityMetrics.yml) | +| [ PreCommit GHA ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_GHA.yml) | N/A |`Run GHA PreCommit`| [![.github/workflows/beam_PreCommit_GHA.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_GHA.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_GHA.yml) | | [ PreCommit Go ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Go.yml) | N/A |`Run Go PreCommit`| [![.github/workflows/beam_PreCommit_Go.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Go.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Go.yml) | | [ PreCommit Java ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java.yml) | N/A |`Run Java PreCommit`| [![.github/workflows/beam_PreCommit_Java.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java.yml) | | [ PreCommit Java Amazon Web Services IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Amazon-Web-Services_IO_Direct.yml) | N/A |`Run Java_Amazon-Web-Services_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Amazon-Web-Services_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Amazon-Web-Services_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Amazon-Web-Services_IO_Direct.yml) | @@ -293,6 +294,7 @@ Please note that jobs with matrix need to have matrix element in the comment. Ex | [ PreCommit Java Examples Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Examples_Dataflow.yml) | N/A |`Run Java_Examples_Dataflow PreCommit`| [![.github/workflows/beam_PreCommit_Java_Examples_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Examples_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Examples_Dataflow.yml) | | [ PreCommit Java Flink Versions ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Flink_Versions.yml) | N/A |`Run Java_Flink_Versions PreCommit`| [![.github/workflows/beam_PreCommit_Java_Flink_Versions.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Flink_Versions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Flink_Versions.yml) | | [ PreCommit Java GCP IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml) | N/A |`Run Java_GCP_IO_Direct PreCommit`| [![.github\workflows\beam_PreCommit_Java_GCP_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml) | +| [ PreCommit Java Google-ads IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Google-ads_IO_Direct.yml) | N/A |`Run Java_Google-ads_IO_Direct PreCommit`| [![.github\workflows\beam_PreCommit_Java_Google-ads_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Google-ads_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Google-ads_IO_Direct.yml) | | [ PreCommit Java Examples Dataflow Java11 ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml) | N/A | `Run Java_Examples_Dataflow_Java11 PreCommit` | [![.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml) | | [ PreCommit Java Examples Dataflow Java17 ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml) | N/A | `Run Java_Examples_Dataflow_Java17 PreCommit` | [![.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml) | | [ PreCommit Java File-schema-transform IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_File-schema-transform_IO_Direct.yml) | N/A |`Run Java_File-schema-transform_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_File-schema-transform_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_File-schema-transform_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_File-schema-transform_IO_Direct.yml) | @@ -315,6 +317,7 @@ Please note that jobs with matrix need to have matrix element in the comment. Ex | [ PreCommit Java PVR Flink Docker ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml) | N/A |`Run Java_PVR_Flink_Docker PreCommit`| [![.github/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml) | | [ PreCommit Java RabbitMq IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml) | N/A |`Run Java_RabbitMq_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml) | | [ PreCommit Java Redis IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml) | N/A |`Run Java_Redis_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml) | +| [ PreCommit Java RequestResponse IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_RequestResponse_IO_Direct.yml) | N/A |`Run Java_RequestResponse_IO_Direct PreCommit`| [![.github\workflows\beam_RequestResponse_Java_RequestResponse_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_RequestResponse_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_RequestResponse_IO_Direct.yml) | | [ PreCommit Java SingleStore IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml) | N/A |`Run Java_SingleStore_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml) | | [ PreCommit Java Snowflake IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml) | N/A |`Run Java_Snowflake_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml) | | [ PreCommit Java Solr IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml) | N/A |`Run Java_Solr_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml) | diff --git a/.github/workflows/beam_PostCommit_BeamMetrics_Publish.yml b/.github/workflows/beam_PostCommit_BeamMetrics_Publish.yml index 177704a179a1..643a159dc21c 100644 --- a/.github/workflows/beam_PostCommit_BeamMetrics_Publish.yml +++ b/.github/workflows/beam_PostCommit_BeamMetrics_Publish.yml @@ -19,7 +19,6 @@ on: push: tags: ['v*'] branches: ['master', 'release-*'] - paths: ['.github/workflows/beam_PostCommit_BeamMetrics_Publish.yml'] schedule: - cron: '24 2 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PreCommit_CommunityMetrics.yml b/.github/workflows/beam_PreCommit_CommunityMetrics.yml index d192c6d164d9..46afb4017882 100644 --- a/.github/workflows/beam_PreCommit_CommunityMetrics.yml +++ b/.github/workflows/beam_PreCommit_CommunityMetrics.yml @@ -22,7 +22,7 @@ on: paths: ['.test-infra/metrics/**', 'buildSrc/build.gradle.kts', '.github/workflows/beam_PreCommit_CommunityMetrics.yml'] pull_request_target: branches: ['master', 'release-*'] - paths: ['.test-infra/metrics/**', 'buildSrc/build.gradle.kts'] + paths: ['.test-infra/metrics/**', 'buildSrc/build.gradle.kts', 'release/trigger_all_tests.json'] issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_GHA.yml b/.github/workflows/beam_PreCommit_GHA.yml new file mode 100644 index 000000000000..79970bcfaa00 --- /dev/null +++ b/.github/workflows/beam_PreCommit_GHA.yml @@ -0,0 +1,89 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: PreCommit GHA + +on: + push: + tags: ['v*'] + branches: ['master', 'release-*'] + paths: ['.github/**/*.yml'] + pull_request_target: + branches: ['master', 'release-*' ] + paths: ['.github/**/*.yml', 'release/trigger_all_tests.json'] + issue_comment: + types: [created] + schedule: + - cron: '0 */6 * * *' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_PreCommit_GHA: + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + runs-on: [self-hosted, ubuntu-20.04, main] + strategy: + matrix: + job_name: [beam_PreCommit_GHA] + job_phrase: [Run GHA PreCommit] + timeout-minutes: 30 + if: | + github.event_name == 'push' || + github.event_name == 'pull_request_target' || + github.event_name == 'schedule' || + github.event_name == 'workflow_dispatch' || + github.event.comment.body == 'Run GHA PreCommit' + steps: + - uses: actions/checkout@v4 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action + with: + java-version: 8 + go-version: 1.21 + - name: run GHA PreCommit script + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :beam-test-gha:preCommit \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Go.yml b/.github/workflows/beam_PreCommit_Go.yml index 127fc9751335..57239d5a399d 100644 --- a/.github/workflows/beam_PreCommit_Go.yml +++ b/.github/workflows/beam_PreCommit_Go.yml @@ -22,7 +22,7 @@ on: paths: ['model/**', 'sdks/go.**', 'release/**', '.github/workflows/beam_PreCommit_Go.yml'] pull_request_target: branches: ['master', 'release-*' ] - paths: ['model/**', 'sdks/go.**', 'release/**'] + paths: ['model/**', 'sdks/go.**', 'release/**', 'release/trigger_all_tests.json'] issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_GoPortable.yml b/.github/workflows/beam_PreCommit_GoPortable.yml index f89b9e0a204e..213d0b25999a 100644 --- a/.github/workflows/beam_PreCommit_GoPortable.yml +++ b/.github/workflows/beam_PreCommit_GoPortable.yml @@ -22,7 +22,7 @@ on: paths: ['model/**', 'sdks/go.**', 'release/**','.github/workflows/beam_PreCommit_GoPortable.yml'] pull_request_target: branches: ['master', 'release-*'] - paths: ['model/**', 'sdks/go.**', 'release/**'] + paths: ['model/**', 'sdks/go.**', 'release/**', 'release/trigger_all_tests.json'] issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_GoPrism.yml b/.github/workflows/beam_PreCommit_GoPrism.yml index 99e117933772..17fece4bc722 100644 --- a/.github/workflows/beam_PreCommit_GoPrism.yml +++ b/.github/workflows/beam_PreCommit_GoPrism.yml @@ -22,7 +22,7 @@ on: paths: ['model/**', 'sdks/go.**', 'release/**','.github/workflows/beam_PreCommit_GoPrism.yml'] pull_request_target: branches: ['master', 'release-*'] - paths: ['model/**', 'sdks/go.**', 'release/**'] + paths: ['model/**', 'sdks/go.**', 'release/**', 'release/trigger_all_tests.json'] issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_ItFramework.yml b/.github/workflows/beam_PreCommit_ItFramework.yml index b9757eae35d7..8a5179099db5 100644 --- a/.github/workflows/beam_PreCommit_ItFramework.yml +++ b/.github/workflows/beam_PreCommit_ItFramework.yml @@ -28,6 +28,7 @@ on: branches: ['master', 'release-*'] paths: - 'it/**' + - 'release/trigger_all_tests.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java.yml b/.github/workflows/beam_PreCommit_Java.yml index eed88f79fb00..ec62a64ed970 100644 --- a/.github/workflows/beam_PreCommit_Java.yml +++ b/.github/workflows/beam_PreCommit_Java.yml @@ -75,6 +75,7 @@ on: - 'examples/java/**' - 'examples/kotlin/**' - 'release/**' + - 'release/trigger_all_tests.json' - '!sdks/java/extensions/avro/**' - '!sdks/java/extensions/sql/**' - '!sdks/java/io/amazon-web-services/**' diff --git a/.github/workflows/beam_PreCommit_Java_Amazon-Web-Services2_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Amazon-Web-Services2_IO_Direct.yml index 689591fb3dcd..0a117ec9cf9b 100644 --- a/.github/workflows/beam_PreCommit_Java_Amazon-Web-Services2_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Amazon-Web-Services2_IO_Direct.yml @@ -37,6 +37,7 @@ on: - "sdks/java/io/amazon-web-services2/**" - "sdks/java/io/common/**" - "sdks/java/core/src/main/**" + - 'release/trigger_all_tests.json' - "build.gradle" - "buildSrc/**" - "gradle/**" diff --git a/.github/workflows/beam_PreCommit_Java_Amazon-Web-Services_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Amazon-Web-Services_IO_Direct.yml index f6d3501194ff..2a8061657fc5 100644 --- a/.github/workflows/beam_PreCommit_Java_Amazon-Web-Services_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Amazon-Web-Services_IO_Direct.yml @@ -37,6 +37,7 @@ on: - "sdks/java/io/amazon-web-services/**" - "sdks/java/io/common/**" - "sdks/java/core/src/main/**" + - 'release/trigger_all_tests.json' - "build.gradle" - "buildSrc/**" - "gradle/**" diff --git a/.github/workflows/beam_PreCommit_Java_Amqp_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Amqp_IO_Direct.yml index 8fd2a799e916..093afbe05834 100644 --- a/.github/workflows/beam_PreCommit_Java_Amqp_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Amqp_IO_Direct.yml @@ -26,6 +26,7 @@ on: branches: ['master', 'release-*'] paths: - "sdks/java/io/amqp/**" + - 'release/trigger_all_tests.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml index 17379856ec9a..013bd587d3d9 100644 --- a/.github/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml @@ -37,6 +37,7 @@ on: - "sdks/java/io/azure/**" - "sdks/java/io/common/**" - "sdks/java/core/src/main/**" + - 'release/trigger_all_tests.json' - "build.gradle" - "buildSrc/**" - "gradle/**" diff --git a/.github/workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml index d016125f1d20..7713c026ea8c 100644 --- a/.github/workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml @@ -26,6 +26,7 @@ on: branches: ['master', 'release-*'] paths: - "sdks/java/io/cassandra/**" + - 'release/trigger_all_tests.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml index e9e1c0ef51b3..08650c314f24 100644 --- a/.github/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml @@ -30,6 +30,7 @@ on: - "sdks/java/io/cdap/**" - "sdks/java/io/hadoop-common/**" - "sdks/java/io/hadoop-format/**" + - 'release/trigger_all_tests.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml index 37b2ca0f2a23..9d2cbcffb407 100644 --- a/.github/workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml @@ -26,6 +26,7 @@ on: branches: ['master', 'release-*'] paths: - "sdks/java/io/clickhouse/**" + - 'release/trigger_all_tests.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml index 9e6f07a278f9..e2b331c235ba 100644 --- a/.github/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml @@ -26,6 +26,7 @@ on: branches: ['master', 'release-*'] paths: - "sdks/java/io/csv/**" + - 'release/trigger_all_tests.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml index db07d2bbacf3..5497926c3054 100644 --- a/.github/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml @@ -26,6 +26,7 @@ on: branches: ['master', 'release-*'] paths: - "sdks/java/io/debezium/**" + - 'release/trigger_all_tests.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml index 93e4f54bdff5..ad02187c7316 100644 --- a/.github/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml @@ -28,6 +28,7 @@ on: paths: - "sdks/java/io/elasticsearch/**" - "sdks/java/io/elasticsearch-tests/**" + - 'release/trigger_all_tests.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow.yml b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow.yml index fa7c3f0f1771..fc099666bb15 100644 --- a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow.yml +++ b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow.yml @@ -38,6 +38,7 @@ on: - 'examples/java/**' - 'examples/kotlin/**' - 'release/**' + - 'release/trigger_all_tests.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml index 0e2614f5be73..8b584bc64122 100644 --- a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml +++ b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml @@ -38,6 +38,7 @@ on: - 'examples/java/**' - 'examples/kotlin/**' - 'release/**' + - 'release/trigger_all_tests.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml index ba5d22041f27..e4e4b62a2e8a 100644 --- a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml +++ b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml @@ -36,6 +36,7 @@ on: - 'examples/java/**' - 'examples/kotlin/**' - 'release/**' + - 'release/trigger_all_tests.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_File-schema-transform_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_File-schema-transform_IO_Direct.yml index bd6a573b8141..aeb9ed6d5056 100644 --- a/.github/workflows/beam_PreCommit_Java_File-schema-transform_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_File-schema-transform_IO_Direct.yml @@ -26,6 +26,7 @@ on: branches: ['master', 'release-*'] paths: - "sdks/java/io/file-schema-transform/**" + - 'release/trigger_all_tests.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_Flink_Versions.yml b/.github/workflows/beam_PreCommit_Java_Flink_Versions.yml index e64e5880bcef..9850183ba4a0 100644 --- a/.github/workflows/beam_PreCommit_Java_Flink_Versions.yml +++ b/.github/workflows/beam_PreCommit_Java_Flink_Versions.yml @@ -30,6 +30,7 @@ on: - 'model/**' - 'runners/flink/**' - 'release/**' + - 'release/trigger_all_tests.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml index 37417ba0e210..dc937f2abcd3 100644 --- a/.github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml @@ -36,6 +36,7 @@ on: paths: - "runners/core-construction-java/**" - "runners/core-java/**" + - 'release/trigger_all_tests.json' - "sdks/java/core/src/main/**" - "sdks/java/extensions/arrow/**" - "sdks/java/extensions/google-cloud-platform-core/**" diff --git a/.github/workflows/beam_PreCommit_Java_Google-ads_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Google-ads_IO_Direct.yml new file mode 100644 index 000000000000..559da13a430a --- /dev/null +++ b/.github/workflows/beam_PreCommit_Java_Google-ads_IO_Direct.yml @@ -0,0 +1,116 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: PreCommit Java Google-ads IO Direct + +on: + push: + tags: ['v*'] + branches: ['master', 'release-*'] + paths: + - "sdks/java/io/google-ads/**" + - ".github/workflows/beam_PreCommit_Java_Google-ads_IO_Direct.yml" + pull_request_target: + branches: ['master', 'release-*'] + paths: + - "sdks/java/io/google-ads/**" + - 'release/trigger_all_tests.json' + issue_comment: + types: [created] + schedule: + - cron: '0 */6 * * *' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: write + checks: write + contents: read + deployments: read + id-token: none + issues: write + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_PreCommit_Java_Google-ads_IO_Direct: + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_PreCommit_Java_Google-ads_IO_Direct"] + job_phrase: ["Run Java_Google-ads_IO_Direct PreCommit"] + timeout-minutes: 60 + if: | + github.event_name == 'push' || + github.event_name == 'pull_request_target' || + github.event_name == 'schedule' || + github.event_name == 'workflow_dispatch' || + github.event.comment.body == 'Run Java_Google-ads_IO_Direct PreCommit' + runs-on: [self-hosted, ubuntu-20.04, main] + steps: + - uses: actions/checkout@v4 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: run Google-ads IO build script + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:io:google-ads:build + arguments: | + -PdisableSpotlessCheck=true \ + -PdisableCheckStyle=true \ + - name: Archive JUnit Test Results + uses: actions/upload-artifact@v3 + if: failure() + with: + name: JUnit Test Results + path: "**/build/reports/tests/" + - name: Publish JUnit Test Results + uses: EnricoMi/publish-unit-test-result-action@v2 + if: always() + with: + commit: '${{ env.prsha || env.GITHUB_SHA }}' + comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} + files: '**/build/test-results/**/*.xml' + - name: Archive SpotBugs Results + uses: actions/upload-artifact@v3 + if: always() + with: + name: SpotBugs Results + path: '**/build/reports/spotbugs/*.html' + - name: Publish SpotBugs Results + uses: jwgmeligmeyling/spotbugs-github-action@v1.2 + if: always() + with: + name: Publish SpotBugs + path: '**/build/reports/spotbugs/*.html' \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml index 8ba07fb062ca..c71c464c933e 100644 --- a/.github/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml @@ -28,6 +28,7 @@ on: paths: - "sdks/java/io/hbase/**" - "sdks/java/io/hadoop-common/**" + - 'release/trigger_all_tests.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml index 6aaf3fa9eb0a..ed99e58ca0eb 100644 --- a/.github/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml @@ -28,6 +28,7 @@ on: paths: - "sdks/java/io/hcatalog/**" - "sdks/java/io/hadoop-common/**" + - 'release/trigger_all_tests.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml index 7a9f3cafe355..4f90aceef9e6 100644 --- a/.github/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml @@ -52,6 +52,7 @@ on: - "sdks/java/testing/test-utils/**" - "sdks/java/io/hadoop-common/**" - "sdks/java/io/hadoop-format/**" + - 'release/trigger_all_tests.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_IOs_Direct.yml b/.github/workflows/beam_PreCommit_Java_IOs_Direct.yml index 7936c391a644..1137d9fec9ed 100644 --- a/.github/workflows/beam_PreCommit_Java_IOs_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_IOs_Direct.yml @@ -28,6 +28,7 @@ on: paths: - "sdks/java/io/common/**" - "sdks/java/core/src/main/**" + - 'release/trigger_all_tests.json' issue_comment: types: [created] workflow_dispatch: diff --git a/.github/workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml index 8a840f92379e..ac35176e551a 100644 --- a/.github/workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml @@ -26,6 +26,7 @@ on: branches: ['master', 'release-*'] paths: - "sdks/java/io/influxdb/**" + - 'release/trigger_all_tests.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml index 4d953028f496..d2ef342e53ca 100644 --- a/.github/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml @@ -26,6 +26,7 @@ on: branches: ['master', 'release-*'] paths: - "sdks/java/io/jdbc/**" + - 'release/trigger_all_tests.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml index fadebc4063c9..223c90cd6b73 100644 --- a/.github/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml @@ -26,6 +26,7 @@ on: branches: ['master', 'release-*'] paths: - "sdks/java/io/jms/**" + - 'release/trigger_all_tests.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml index 3564e1a685df..61ecc1d304c9 100644 --- a/.github/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml @@ -34,6 +34,7 @@ on: - "sdks/java/expansion-service/**" - "sdks/java/io/synthetic/**" - "sdks/java/io/expansion-service/**" + - 'release/trigger_all_tests.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml index 6da5325fd50f..8cca3951ddd9 100644 --- a/.github/workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml @@ -44,6 +44,7 @@ on: - "gradlew" - "gradle.bat" - "settings.gradle.kts" + - 'release/trigger_all_tests.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml index a2e4e11c72d8..bc3d257d1db0 100644 --- a/.github/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml @@ -26,6 +26,7 @@ on: branches: ['master', 'release-*'] paths: - "sdks/java/io/kudu/**" + - 'release/trigger_all_tests.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml index a7489a597d1e..b8d28c129d3d 100644 --- a/.github/workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml @@ -26,6 +26,7 @@ on: branches: ['master', 'release-*'] paths: - "sdks/java/io/mongodb/**" + - 'release/trigger_all_tests.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml index 867496e8e674..7b19949ee655 100644 --- a/.github/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml @@ -26,6 +26,7 @@ on: branches: ['master', 'release-*'] paths: - "sdks/java/io/mqtt/**" + - 'release/trigger_all_tests.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml index 01f03e85edb7..26b4637598b7 100644 --- a/.github/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml @@ -28,6 +28,7 @@ on: paths: - "sdks/java/io/neo4j/**" - "sdks/java/testing/test-utils/**" + - 'release/trigger_all_tests.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_PVR_Flink_Batch.yml b/.github/workflows/beam_PreCommit_Java_PVR_Flink_Batch.yml index 792ff47608d6..602bb8750ae5 100644 --- a/.github/workflows/beam_PreCommit_Java_PVR_Flink_Batch.yml +++ b/.github/workflows/beam_PreCommit_Java_PVR_Flink_Batch.yml @@ -32,6 +32,7 @@ on: - 'runners/flink/**' - 'runners/java-fn-execution/**' - 'sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/**' + - 'release/trigger_all_tests.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml b/.github/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml index 5839a59d8e95..b2f0c66c3348 100644 --- a/.github/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml +++ b/.github/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml @@ -36,6 +36,7 @@ on: - 'sdks/java/harness/**' - 'runners/flink/**' - 'runners/java-fn-execution/**' + - 'release/trigger_all_tests.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml index 80f6110a1143..4a725129faa1 100644 --- a/.github/workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml @@ -26,6 +26,7 @@ on: branches: ['master', 'release-*'] paths: - "sdks/java/io/parquet/**" + - 'release/trigger_all_tests.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml index 38cd15be1dd2..cac0872d902d 100644 --- a/.github/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml @@ -37,6 +37,7 @@ on: - "sdks/java/io/pulsar/**" - "sdks/java/io/common/**" - "sdks/java/core/src/main/**" + - 'release/trigger_all_tests.json' - "build.gradle" - "buildSrc/**" - "gradle/**" diff --git a/.github/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml index 64a03f93b0b6..4983a84a1267 100644 --- a/.github/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml @@ -26,6 +26,7 @@ on: branches: ['master', 'release-*'] paths: - "sdks/java/io/rabbitmq/**" + - 'release/trigger_all_tests.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml index 18cc54ddffcb..51970fcdff1a 100644 --- a/.github/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml @@ -26,6 +26,7 @@ on: branches: ['master', 'release-*'] paths: - "sdks/java/io/redis/**" + - 'release/trigger_all_tests.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_RequestResponse_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_RequestResponse_IO_Direct.yml new file mode 100644 index 000000000000..1da99d7cc525 --- /dev/null +++ b/.github/workflows/beam_PreCommit_Java_RequestResponse_IO_Direct.yml @@ -0,0 +1,116 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: PreCommit Java RequestResponse IO Direct + +on: + push: + tags: ['v*'] + branches: ['master', 'release-*'] + paths: + - "sdks/java/io/rrio/**" + - ".github/workflows/beam_PreCommit_Java_RequestResponse_IO_Direct.yml" + pull_request_target: + branches: ['master', 'release-*'] + paths: + - "sdks/java/io/rrio/**" + - 'release/trigger_all_tests.json' + issue_comment: + types: [created] + schedule: + - cron: '0 */6 * * *' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: write + checks: write + contents: read + deployments: read + id-token: none + issues: write + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_PreCommit_Java_RequestResponse_IO_Direct: + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_PreCommit_Java_RequestResponse_IO_Direct"] + job_phrase: ["Run Java_RequestResponse_IO_Direct PreCommit"] + timeout-minutes: 60 + if: | + github.event_name == 'push' || + github.event_name == 'pull_request_target' || + github.event_name == 'schedule' || + github.event_name == 'workflow_dispatch' || + github.event.comment.body == 'Run Java_RequestResponse_IO_Direct PreCommit' + runs-on: [self-hosted, ubuntu-20.04, main] + steps: + - uses: actions/checkout@v4 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: run RequestResponse IO build script + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:io:RequestResponse:build + arguments: | + -PdisableSpotlessCheck=true \ + -PdisableCheckStyle=true \ + - name: Archive JUnit Test Results + uses: actions/upload-artifact@v3 + if: failure() + with: + name: JUnit Test Results + path: "**/build/reports/tests/" + - name: Publish JUnit Test Results + uses: EnricoMi/publish-unit-test-result-action@v2 + if: always() + with: + commit: '${{ env.prsha || env.GITHUB_SHA }}' + comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} + files: '**/build/test-results/**/*.xml' + - name: Archive SpotBugs Results + uses: actions/upload-artifact@v3 + if: always() + with: + name: SpotBugs Results + path: '**/build/reports/spotbugs/*.html' + - name: Publish SpotBugs Results + uses: jwgmeligmeyling/spotbugs-github-action@v1.2 + if: always() + with: + name: Publish SpotBugs + path: '**/build/reports/spotbugs/*.html' \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml index c692cd24c979..1cae0b51872e 100644 --- a/.github/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml @@ -28,6 +28,7 @@ on: paths: - "sdks/java/io/singlestore/**" - "sdks/java/testing/test-utils/**" + - 'release/trigger_all_tests.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml index 383f7b2fe32c..7d28989b7690 100644 --- a/.github/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml @@ -30,6 +30,7 @@ on: - "sdks/java/io/snowflake/**" - "sdks/java/extensions/google-cloud-platform-core/**" - "sdks/java/testing/test-utils/**" + - 'release/trigger_all_tests.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml index 61350d767b1e..b0096415b063 100644 --- a/.github/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml @@ -26,6 +26,7 @@ on: branches: ['master', 'release-*'] paths: - "sdks/java/io/solr/**" + - 'release/trigger_all_tests.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_Spark3_Versions.yml b/.github/workflows/beam_PreCommit_Java_Spark3_Versions.yml index 9ff70bf1d321..42b9a244008d 100644 --- a/.github/workflows/beam_PreCommit_Java_Spark3_Versions.yml +++ b/.github/workflows/beam_PreCommit_Java_Spark3_Versions.yml @@ -28,6 +28,7 @@ on: branches: ['master', 'release-*'] paths: - 'runners/spark/**' + - 'release/trigger_all_tests.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml index 39d159d9e3fc..57adc64576cf 100644 --- a/.github/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml @@ -26,6 +26,7 @@ on: branches: ['master', 'release-*'] paths: - "sdks/java/io/splunk/**" + - 'release/trigger_all_tests.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml index ac9fe2d38c91..88eabd1e5c8a 100644 --- a/.github/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml @@ -26,6 +26,7 @@ on: branches: ['master', 'release-*'] paths: - "sdks/java/io/thrift/**" + - 'release/trigger_all_tests.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml index a27637da6aae..62f96aec254e 100644 --- a/.github/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml @@ -26,6 +26,7 @@ on: branches: ['master', 'release-*'] paths: - "sdks/java/io/tika/**" + - 'release/trigger_all_tests.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Kotlin_Examples.yml b/.github/workflows/beam_PreCommit_Kotlin_Examples.yml index 513487d0bf3b..f3e6e36a3c25 100644 --- a/.github/workflows/beam_PreCommit_Kotlin_Examples.yml +++ b/.github/workflows/beam_PreCommit_Kotlin_Examples.yml @@ -38,6 +38,7 @@ on: - 'runners/direct-java/**' - 'examples/kotlin/**' - 'release/**' + - 'release/trigger_all_tests.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Portable_Python.yml b/.github/workflows/beam_PreCommit_Portable_Python.yml index 85816cc62a59..c1302468a715 100644 --- a/.github/workflows/beam_PreCommit_Portable_Python.yml +++ b/.github/workflows/beam_PreCommit_Portable_Python.yml @@ -42,6 +42,7 @@ on: - 'runners/reference/**' - 'sdks/python/**' - 'release/**' + - 'release/trigger_all_tests.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Python.yml b/.github/workflows/beam_PreCommit_Python.yml index 03cba671eae2..d507a8e3f072 100644 --- a/.github/workflows/beam_PreCommit_Python.yml +++ b/.github/workflows/beam_PreCommit_Python.yml @@ -17,7 +17,7 @@ name: PreCommit Python on: pull_request_target: branches: [ "master", "release-*" ] - paths: [ "model/**","sdks/python/**","release/**"] + paths: [ "model/**","sdks/python/**","release/**", 'release/trigger_all_tests.json'] issue_comment: types: [created] push: diff --git a/.github/workflows/beam_PreCommit_PythonDocker.yml b/.github/workflows/beam_PreCommit_PythonDocker.yml index 6578c733818e..a84ba8fd3a26 100644 --- a/.github/workflows/beam_PreCommit_PythonDocker.yml +++ b/.github/workflows/beam_PreCommit_PythonDocker.yml @@ -17,7 +17,7 @@ name: PreCommit Python Docker on: pull_request_target: branches: [ "master", "release-*" ] - paths: [ "model/**","sdks/python/**","release/**"] + paths: [ "model/**","sdks/python/**","release/**", 'release/trigger_all_tests.json'] issue_comment: types: [created] push: diff --git a/.github/workflows/beam_PreCommit_PythonDocs.yml b/.github/workflows/beam_PreCommit_PythonDocs.yml index 0666f13cc747..3f641681d91c 100644 --- a/.github/workflows/beam_PreCommit_PythonDocs.yml +++ b/.github/workflows/beam_PreCommit_PythonDocs.yml @@ -18,7 +18,7 @@ name: PreCommit Python Docs on: pull_request_target: branches: [ "master", "release-*" ] - paths: ["sdks/python/**"] + paths: ["sdks/python/**", 'release/trigger_all_tests.json'] issue_comment: types: [created] push: diff --git a/.github/workflows/beam_PreCommit_PythonFormatter.yml b/.github/workflows/beam_PreCommit_PythonFormatter.yml index 2f22fc0d3d85..70f13a2ee736 100644 --- a/.github/workflows/beam_PreCommit_PythonFormatter.yml +++ b/.github/workflows/beam_PreCommit_PythonFormatter.yml @@ -17,7 +17,7 @@ name: PreCommit Python Formatter on: pull_request_target: branches: [ "master", "release-*" ] - paths: [ "sdks/python/apache_beam/**"] + paths: [ "sdks/python/apache_beam/**", 'release/trigger_all_tests.json'] issue_comment: types: [created] push: diff --git a/.github/workflows/beam_PreCommit_PythonLint.yml b/.github/workflows/beam_PreCommit_PythonLint.yml index f24dcb4fa671..199640d7b9ac 100644 --- a/.github/workflows/beam_PreCommit_PythonLint.yml +++ b/.github/workflows/beam_PreCommit_PythonLint.yml @@ -17,7 +17,7 @@ name: PreCommit Python Lint on: pull_request_target: branches: [ "master", "release-*" ] - paths: ["sdks/python/**","release/**"] + paths: ["sdks/python/**","release/**", 'release/trigger_all_tests.json'] issue_comment: types: [created] push: diff --git a/.github/workflows/beam_PreCommit_Python_Coverage.yml b/.github/workflows/beam_PreCommit_Python_Coverage.yml index adabcbb47529..2127f08ea591 100644 --- a/.github/workflows/beam_PreCommit_Python_Coverage.yml +++ b/.github/workflows/beam_PreCommit_Python_Coverage.yml @@ -17,7 +17,7 @@ name: PreCommit Python Coverage on: pull_request_target: branches: [ "master", "release-*" ] - paths: [ "model/**","sdks/python/**","release/**"] + paths: [ "model/**","sdks/python/**","release/**", 'release/trigger_all_tests.json'] issue_comment: types: [created] push: diff --git a/.github/workflows/beam_PreCommit_Python_Dataframes.yml b/.github/workflows/beam_PreCommit_Python_Dataframes.yml index 1fff65c00ed5..9211e2527c6a 100644 --- a/.github/workflows/beam_PreCommit_Python_Dataframes.yml +++ b/.github/workflows/beam_PreCommit_Python_Dataframes.yml @@ -17,7 +17,7 @@ name: PreCommit Python Dataframes on: pull_request_target: branches: [ "master", "release-*" ] - paths: [ "model/**","sdks/python/**","release/**"] + paths: [ "model/**","sdks/python/**","release/**", 'release/trigger_all_tests.json'] issue_comment: types: [created] push: diff --git a/.github/workflows/beam_PreCommit_Python_Examples.yml b/.github/workflows/beam_PreCommit_Python_Examples.yml index 0db33b91c0d2..164734f0c8e2 100644 --- a/.github/workflows/beam_PreCommit_Python_Examples.yml +++ b/.github/workflows/beam_PreCommit_Python_Examples.yml @@ -17,7 +17,7 @@ name: PreCommit Python Examples on: pull_request_target: branches: [ "master", "release-*" ] - paths: [ "model/**","sdks/python/**","release/**"] + paths: [ "model/**","sdks/python/**","release/**", 'release/trigger_all_tests.json'] issue_comment: types: [created] push: diff --git a/.github/workflows/beam_PreCommit_Python_Integration.yml b/.github/workflows/beam_PreCommit_Python_Integration.yml index 76653f00b06c..f9c32a12bf5a 100644 --- a/.github/workflows/beam_PreCommit_Python_Integration.yml +++ b/.github/workflows/beam_PreCommit_Python_Integration.yml @@ -17,7 +17,7 @@ name: PreCommit Python Integration on: pull_request_target: branches: [ "master", "release-*" ] - paths: ["model/**", "sdks/python/**", "release/**"] + paths: ["model/**", "sdks/python/**", "release/**", 'release/trigger_all_tests.json'] issue_comment: types: [created] push: diff --git a/.github/workflows/beam_PreCommit_Python_PVR_Flink.yml b/.github/workflows/beam_PreCommit_Python_PVR_Flink.yml index 8b93fdc06d77..06f18651f117 100644 --- a/.github/workflows/beam_PreCommit_Python_PVR_Flink.yml +++ b/.github/workflows/beam_PreCommit_Python_PVR_Flink.yml @@ -29,6 +29,7 @@ on: - 'runners/flink/**' - 'runners/java-fn-execution/**' - 'runners/reference/**' + - 'release/trigger_all_tests.json' issue_comment: types: [created] push: diff --git a/.github/workflows/beam_PreCommit_Python_Runners.yml b/.github/workflows/beam_PreCommit_Python_Runners.yml index f2f548291a25..d1702905f5d7 100644 --- a/.github/workflows/beam_PreCommit_Python_Runners.yml +++ b/.github/workflows/beam_PreCommit_Python_Runners.yml @@ -17,7 +17,7 @@ name: PreCommit Python Runners on: pull_request_target: branches: [ "master", "release-*" ] - paths: [ "model/**","sdks/python/**","release/**"] + paths: [ "model/**","sdks/python/**","release/**", 'release/trigger_all_tests.json'] issue_comment: types: [created] push: diff --git a/.github/workflows/beam_PreCommit_Python_Transforms.yml b/.github/workflows/beam_PreCommit_Python_Transforms.yml index c60657d6d9e7..7422d7dc0f44 100644 --- a/.github/workflows/beam_PreCommit_Python_Transforms.yml +++ b/.github/workflows/beam_PreCommit_Python_Transforms.yml @@ -17,7 +17,7 @@ name: PreCommit Python Transforms on: pull_request_target: branches: [ "master", "release-*" ] - paths: [ "model/**","sdks/python/**","release/**"] + paths: [ "model/**","sdks/python/**","release/**", 'release/trigger_all_tests.json'] issue_comment: types: [created] push: diff --git a/.github/workflows/beam_PreCommit_SQL.yml b/.github/workflows/beam_PreCommit_SQL.yml index 800f62b666cd..1fe577d759e9 100644 --- a/.github/workflows/beam_PreCommit_SQL.yml +++ b/.github/workflows/beam_PreCommit_SQL.yml @@ -22,7 +22,7 @@ on: paths: ['sdks/java/extensions/sql/**','.github/workflows/beam_PreCommit_SQL.yml'] pull_request_target: branches: ['master', 'release-*'] - paths: [sdks/java/extensions/sql/**] + paths: ['sdks/java/extensions/sql/**', 'release/trigger_all_tests.json'] issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_SQL_Java11.yml b/.github/workflows/beam_PreCommit_SQL_Java11.yml index caee133c2cb9..c48ad0dd8063 100644 --- a/.github/workflows/beam_PreCommit_SQL_Java11.yml +++ b/.github/workflows/beam_PreCommit_SQL_Java11.yml @@ -22,7 +22,7 @@ on: paths: ['sdks/java/extensions/sql/**','.github/workflows/beam_PreCommit_SQL_Java11.yml'] pull_request_target: branches: ['master', 'release-*'] - paths: [sdks/java/extensions/sql/**] + paths: ['sdks/java/extensions/sql/**', 'release/trigger_all_tests.json'] issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_SQL_Java17.yml b/.github/workflows/beam_PreCommit_SQL_Java17.yml index 6ad36b2573e9..1ea901947a98 100644 --- a/.github/workflows/beam_PreCommit_SQL_Java17.yml +++ b/.github/workflows/beam_PreCommit_SQL_Java17.yml @@ -22,7 +22,7 @@ on: paths: ['sdks/java/extensions/sql/**','.github/workflows/beam_PreCommit_SQL_Java17.yml'] pull_request_target: branches: ['master', 'release-*'] - paths: [sdks/java/extensions/sql/**] + paths: ['sdks/java/extensions/sql/**', 'release/trigger_all_tests.json'] issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Spotless.yml b/.github/workflows/beam_PreCommit_Spotless.yml index 32475f30486b..a637addf8628 100644 --- a/.github/workflows/beam_PreCommit_Spotless.yml +++ b/.github/workflows/beam_PreCommit_Spotless.yml @@ -35,6 +35,7 @@ on: - 'examples/java/**' - 'examples/kotlin/**' - '.test-infra/jenkins/' + - 'release/trigger_all_tests.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Typescript.yml b/.github/workflows/beam_PreCommit_Typescript.yml index ba087e70cf85..5459ebe89522 100644 --- a/.github/workflows/beam_PreCommit_Typescript.yml +++ b/.github/workflows/beam_PreCommit_Typescript.yml @@ -24,7 +24,7 @@ on: paths: ['sdks/python/apache_beam/runners/interactive/extensions/**', '.github/workflows/beam_PreCommit_Typescript.yml'] pull_request_target: branches: ['master', 'release-*'] - paths: ['sdks/python/apache_beam/runners/interactive/extensions/**'] + paths: ['sdks/python/apache_beam/runners/interactive/extensions/**', 'release/trigger_all_tests.json'] issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Website.yml b/.github/workflows/beam_PreCommit_Website.yml index 6d8c96cd7545..3eb60c2cd9ef 100644 --- a/.github/workflows/beam_PreCommit_Website.yml +++ b/.github/workflows/beam_PreCommit_Website.yml @@ -22,7 +22,7 @@ on: paths: ['website/**','.github/workflows/beam_PreCommit_Website.yml'] pull_request_target: branches: ['master', 'release-*'] - paths: ['website/**'] + paths: ['website/**', 'release/trigger_all_tests.json'] issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Website_Stage_GCS.yml b/.github/workflows/beam_PreCommit_Website_Stage_GCS.yml index e2570e20414b..e89d092be16c 100644 --- a/.github/workflows/beam_PreCommit_Website_Stage_GCS.yml +++ b/.github/workflows/beam_PreCommit_Website_Stage_GCS.yml @@ -22,7 +22,7 @@ on: paths: ['website/**','.github/workflows/beam_PreCommit_Website_Stage_GCS.yml'] pull_request_target: branches: ['master', 'release-*'] - paths: ['website/**'] + paths: ['website/**', 'release/trigger_all_tests.json'] issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Whitespace.yml b/.github/workflows/beam_PreCommit_Whitespace.yml index 2c7294cb224e..dff494c81370 100644 --- a/.github/workflows/beam_PreCommit_Whitespace.yml +++ b/.github/workflows/beam_PreCommit_Whitespace.yml @@ -22,7 +22,7 @@ on: paths: ['**.md', '**.build.gradle', 'build.gradle.kts', '.github/workflows/beam_PreCommit_Whitespace.yml'] pull_request_target: branches: ['master', 'release-*'] - paths: ['**.md', '**.build.gradle', 'build.gradle.kts'] + paths: ['**.md', '**.build.gradle', 'build.gradle.kts', 'release/trigger_all_tests.json'] issue_comment: types: [created] schedule: diff --git a/contributor-docs/release-guide.md b/contributor-docs/release-guide.md index 189bc61e15e6..8b8d29127c99 100644 --- a/contributor-docs/release-guide.md +++ b/contributor-docs/release-guide.md @@ -367,9 +367,11 @@ issues that would block the creation of the release candidate. ``` (cd release/src/main/scripts && ./verify_release_build.sh) ``` - 4. Trigger all Jenkins PostCommit jobs from the PR created by the previous step. + 4. Trigger all Github Action and Jenkins PostCommit jobs from the PR created by the previous step. + For GitHub Action jobs, they should be triggered by the pull_request_target event of a specific placeholder file + added to the PR (`release/trigger_all_tests.json`), so no additional action should be needed. You can run [mass_comment.py](https://github.com/apache/beam/blob/master/release/src/main/scripts/mass_comment.py) to do that. - Or manually add one trigger phrase per PR comment. + Or manually add one trigger phrase per PR comment for Jenkins tests, or rerun the workflow for GitHub Action tests. See [jenkins_jobs.txt](https://github.com/apache/beam/blob/master/release/src/main/scripts/jenkins_jobs.txt) for a full list of phrases. diff --git a/release/src/main/scripts/mass_comment.py b/release/src/main/scripts/mass_comment.py index 6d020f2af8e3..7dec20dbbba5 100644 --- a/release/src/main/scripts/mass_comment.py +++ b/release/src/main/scripts/mass_comment.py @@ -125,21 +125,6 @@ def getRemainingComments(accessToken, pr, initialComments): remainingComments.append(comment) return remainingComments -def getGithubActionsTriggerCommands(dirname): - ''' - Returns all trigger commands that will start PostCommit Dataflow ARM Github Actions test suites. - ''' - gha_trigger_commands = [] - - with open(os.path.join(dirname, 'github_actions_jobs.txt')) as file: - comments = [line.strip() for line in file if len(line.strip()) > 0] - - for i in range(len(comments)): - parts = comments[i].split(',') - gha_trigger_commands.append((parts[0], parts[1])) - - return gha_trigger_commands - ################################################################################ if __name__ == '__main__': ''' @@ -157,9 +142,6 @@ def getGithubActionsTriggerCommands(dirname): parts = comments[i].split(',') comments[i] = (parts[0], parts[1]) - gha_comments = getGithubActionsTriggerCommands(dirname) - comments.extend(gha_comments) - if not probeGitHubIsUp(): print("GitHub is unavailable, skipping fetching data.") exit() @@ -170,7 +152,8 @@ def getGithubActionsTriggerCommands(dirname): pr = input("Enter the Beam PR number to test (e.g. 11403): ") subjectId = getSubjectId(accessToken, pr) - + + # TODO(yathu): also auto rerun failed GitHub Action workflow remainingComments = getRemainingComments(accessToken, pr, comments) if len(remainingComments) == 0: print('Jobs have been started for all comments. If you would like to retry all jobs, create a new commit before running this script.') diff --git a/release/src/main/scripts/verify_release_build.sh b/release/src/main/scripts/verify_release_build.sh index 214c65cc9ef6..51008d40831f 100755 --- a/release/src/main/scripts/verify_release_build.sh +++ b/release/src/main/scripts/verify_release_build.sh @@ -136,9 +136,9 @@ if [[ ! -z `which hub` ]]; then # Without changing to dev version, the dataflow pipeline will fail because of non-existed worker containers. # Note that dataflow worker containers should be built after RC has been built. bash "$SCRIPT_DIR"/set_version.sh "$RELEASE_VER" --git-add - # In case the version string was not changed, append a newline to CHANGES.md - echo "" >> CHANGES.md - git add CHANGES.md + # add a file that will trigger all relevant GHA workflows. Need to be .json extension to be excluded from RAT check + echo "{}" > release/trigger_all_tests.json + git add release/trigger_all_tests.json git commit -m "Changed version.py and gradle.properties to python dev version to create a test PR" --quiet git push -f ${GITHUB_USERNAME} ${WORKING_BRANCH} --quiet @@ -147,6 +147,6 @@ if [[ ! -z `which hub` ]]; then You can run many tests automatically using release/src/main/scripts/mass_comment.py." echo "" - echo "[NOTE]: Please make sure all test targets have been invoked." + echo "[NOTE]: Please make sure all test targets (GHA and Jenkins) have been invoked." echo "Please check the test results. If there is any failure, follow the policy in release guide." fi diff --git a/settings.gradle.kts b/settings.gradle.kts index 1c851da92e60..d833a323212c 100644 --- a/settings.gradle.kts +++ b/settings.gradle.kts @@ -331,6 +331,8 @@ include("beam-test-tools") project(":beam-test-tools").projectDir = file(".test-infra/tools") include("beam-test-jenkins") project(":beam-test-jenkins").projectDir = file(".test-infra/jenkins") +include("beam-test-gha") +project(":beam-test-gha").projectDir = file(".github") include("beam-validate-runner") project(":beam-validate-runner").projectDir = file(".test-infra/validate-runner") include("com.google.api.gax.batching") From 949f12eb8099a056f2c1c8e00ee5cd1c6192278a Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 19 Oct 2023 10:59:59 -0400 Subject: [PATCH 198/435] Bump cloud.google.com/go/profiler from 0.3.1 to 0.4.0 in /sdks (#29070) Bumps [cloud.google.com/go/profiler](https://github.com/googleapis/google-cloud-go) from 0.3.1 to 0.4.0. - [Release notes](https://github.com/googleapis/google-cloud-go/releases) - [Changelog](https://github.com/googleapis/google-cloud-go/blob/main/CHANGES.md) - [Commits](https://github.com/googleapis/google-cloud-go/compare/talent/v0.3.1...v0.4.0) --- updated-dependencies: - dependency-name: cloud.google.com/go/profiler dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] <support@github.com> Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 4 ++-- sdks/go.sum | 8 ++++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 192802f77eaa..686a399bd80f 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -26,7 +26,7 @@ require ( cloud.google.com/go/bigquery v1.56.0 cloud.google.com/go/bigtable v1.20.0 cloud.google.com/go/datastore v1.15.0 - cloud.google.com/go/profiler v0.3.1 + cloud.google.com/go/profiler v0.4.0 cloud.google.com/go/pubsub v1.33.0 cloud.google.com/go/spanner v1.50.0 cloud.google.com/go/storage v1.33.0 @@ -128,7 +128,7 @@ require ( github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da // indirect github.com/golang/snappy v0.0.4 // indirect github.com/google/flatbuffers v2.0.8+incompatible // indirect - github.com/google/pprof v0.0.0-20221103000818-d260c55eee4c // indirect + github.com/google/pprof v0.0.0-20230602150820-91b7bce49751 // indirect github.com/google/renameio/v2 v2.0.0 // indirect github.com/google/s2a-go v0.1.7 // indirect github.com/googleapis/enterprise-certificate-proxy v0.3.1 // indirect diff --git a/sdks/go.sum b/sdks/go.sum index 457fe2bf1ffe..6428fbd82c9f 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -31,8 +31,8 @@ cloud.google.com/go/iam v1.1.2/go.mod h1:A5avdyVL2tCppe4unb0951eI9jreack+RJ0/d+K cloud.google.com/go/kms v1.15.2 h1:lh6qra6oC4AyWe5fUUUBe/S27k12OHAleOOOw6KakdE= cloud.google.com/go/longrunning v0.5.1 h1:Fr7TXftcqTudoyRJa113hyaqlGdiBQkp0Gq7tErFDWI= cloud.google.com/go/longrunning v0.5.1/go.mod h1:spvimkwdz6SPWKEt/XBij79E9fiTkHSQl/fRUUQJYJc= -cloud.google.com/go/profiler v0.3.1 h1:b5got9Be9Ia0HVvyt7PavWxXEht15B9lWnigdvHtxOc= -cloud.google.com/go/profiler v0.3.1/go.mod h1:GsG14VnmcMFQ9b+kq71wh3EKMZr3WRMgLzNiFRpW7tE= +cloud.google.com/go/profiler v0.4.0 h1:ZeRDZbsOBDyRG0OiK0Op1/XWZ3xeLwJc9zjkzczUxyY= +cloud.google.com/go/profiler v0.4.0/go.mod h1:RvPlm4dilIr3oJtAOeFQU9Lrt5RoySHSDj4pTd6TWeU= cloud.google.com/go/pubsub v1.0.1/go.mod h1:R0Gpsv3s54REJCy4fxDixWD93lHJMoZTyQ2kNxGRt3I= cloud.google.com/go/pubsub v1.1.0/go.mod h1:EwwdRX2sKPjnvnqCa270oGRyludottCI76h+R3AArQw= cloud.google.com/go/pubsub v1.2.0/go.mod h1:jhfEVHT8odbXTkndysNHCcx0awwzvfOlguIAii9o8iA= @@ -269,8 +269,8 @@ github.com/google/pprof v0.0.0-20181206194817-3ea8567a2e57/go.mod h1:zfwlbNMJ+OI github.com/google/pprof v0.0.0-20190515194954-54271f7e092f/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= github.com/google/pprof v0.0.0-20191218002539-d4f498aebedc/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= github.com/google/pprof v0.0.0-20200212024743-f11f1df84d12/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= -github.com/google/pprof v0.0.0-20221103000818-d260c55eee4c h1:lvddKcYTQ545ADhBujtIJmqQrZBDsGo7XIMbAQe/sNY= -github.com/google/pprof v0.0.0-20221103000818-d260c55eee4c/go.mod h1:dDKJzRmX4S37WGHujM7tX//fmj1uioxKzKxz3lo4HJo= +github.com/google/pprof v0.0.0-20230602150820-91b7bce49751 h1:hR7/MlvK23p6+lIw9SN1TigNLn9ZnF3W4SYRKq2gAHs= +github.com/google/pprof v0.0.0-20230602150820-91b7bce49751/go.mod h1:Jh3hGz2jkYak8qXPD19ryItVnUgpgeqzdkY/D0EaeuA= github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= github.com/google/renameio/v2 v2.0.0 h1:UifI23ZTGY8Tt29JbYFiuyIU3eX+RNFtUwefq9qAhxg= github.com/google/renameio/v2 v2.0.0/go.mod h1:BtmJXm5YlszgC+TD4HOEEUFgkJP3nLxehU6hfe7jRt4= From 68e9c997a9085b0cb045238ae406d534011e7c21 Mon Sep 17 00:00:00 2001 From: Danny McCormick <dannymccormick@google.com> Date: Thu, 19 Oct 2023 11:50:15 -0400 Subject: [PATCH 199/435] Move Java Google BOM to best effort post-release task (#29062) * Move Java Google BOM to best effort post-release task * Wording --- contributor-docs/release-guide.md | 46 ++++++++++++++++++++++++------- 1 file changed, 36 insertions(+), 10 deletions(-) diff --git a/contributor-docs/release-guide.md b/contributor-docs/release-guide.md index 8b8d29127c99..e8e8ead0902e 100644 --- a/contributor-docs/release-guide.md +++ b/contributor-docs/release-guide.md @@ -261,16 +261,6 @@ for each release. See if https://go.dev/doc/devel/release has a newer release. Update throughout Beam. See example at https://github.com/apache/beam/pull/27900/files -#### Update the Java BOM - -Google releases a BOM that pins compatible versions of their Java libraries. -Ideally, this update happens as far in advance of the release as possible, such -as just after a release. But if that was not done, consider doing it before -cutting the release branch. - -To do so, follow instructions at -https://github.com/apache/beam/blob/master/contributor-docs/java-dependency-upgrades.md. - ### Cut the release branch > **Note** @@ -1358,6 +1348,42 @@ Also, update [the Wikipedia article on Apache Beam](https://en.wikipedia.org/wik At the end of the release, go to the GitHub milestones page and mark the recently released version as closed. +#### Update the Java BOM + +Google releases a BOM that pins compatible versions of their Java libraries. +After the release, try updating the BOM to the latest version. + +To do so, create a draft PR and run test suites following the instructions at +https://github.com/apache/beam/blob/master/contributor-docs/java-dependency-upgrades.md. + +Triage the test failures and rerun any tests that seem potentially unrelated to the upgrade. +If there are no test failures due to the BOM upgrade, request review and merge the PR as normal. + +If there are test failures due to the BOM upgrade, email the dev list and ask for a volunteer to take the update forward. +It is not your responsibility to fix the BOM issues or to find a volunteer (though you are welcome to take it forward). +If nobody volunteers, that is OK and this issue can roll forward to the next release. +You can optionally use the following template for your email to the dev list: + +``` +From: Release Manager +To: dev@beam.apache.org +Subject: Java BOM Update X.Y.Z + +Hi everyone, + +Following the instructions in https://github.com/apache/beam/blob/master/contributor-docs/release-guide.md#post-release-tasks +I've attempted to update the Java Google BOM and have run into test issues caused by the upgrade [1]. +Since the Java Google BOM update is best effort for a release manager, I'm handing this piece off to the community. +If you would like to volunteer to help, you can get started by following the instructions in +https://github.com/apache/beam/blob/master/contributor-docs/java-dependency-upgrades.md#google-cloud-related-dependency-upgrades +otherwise this will roll over to the next release. + +Thanks, +Release Manager + +[1] https://github.com/apache/beam/pull/123 +``` + ### Update Beam Playground After new Beam Release is published, Beam Playground can be updated following the steps below: From 1885251330952546d1a74d31d7a0324445ac2335 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw <robertwb@gmail.com> Date: Thu, 19 Oct 2023 13:38:48 -0700 Subject: [PATCH 200/435] Add jsonschema dependency. --- sdks/python/container/py310/base_image_requirements.txt | 2 ++ sdks/python/container/py311/base_image_requirements.txt | 2 ++ sdks/python/container/py38/base_image_requirements.txt | 2 ++ sdks/python/container/py39/base_image_requirements.txt | 2 ++ sdks/python/setup.py | 1 + 5 files changed, 9 insertions(+) diff --git a/sdks/python/container/py310/base_image_requirements.txt b/sdks/python/container/py310/base_image_requirements.txt index 548c56371706..07e03a539bb0 100644 --- a/sdks/python/container/py310/base_image_requirements.txt +++ b/sdks/python/container/py310/base_image_requirements.txt @@ -81,6 +81,8 @@ idna==3.4 iniconfig==2.0.0 joblib==1.3.2 Js2Py==0.74 +jsonschema==4.19.1 +jsonschema-specifications==2023.7.1 mmh3==4.0.1 mock==5.1.0 nltk==3.8.1 diff --git a/sdks/python/container/py311/base_image_requirements.txt b/sdks/python/container/py311/base_image_requirements.txt index 2e5d834926bd..18f55e5d6927 100644 --- a/sdks/python/container/py311/base_image_requirements.txt +++ b/sdks/python/container/py311/base_image_requirements.txt @@ -78,6 +78,8 @@ idna==3.4 iniconfig==2.0.0 joblib==1.3.2 Js2Py==0.74 +jsonschema==4.19.1 +jsonschema-specifications==2023.7.1 mmh3==4.0.1 mock==5.1.0 nltk==3.8.1 diff --git a/sdks/python/container/py38/base_image_requirements.txt b/sdks/python/container/py38/base_image_requirements.txt index ed5d35fc6457..e474ab69c64c 100644 --- a/sdks/python/container/py38/base_image_requirements.txt +++ b/sdks/python/container/py38/base_image_requirements.txt @@ -82,6 +82,8 @@ idna==3.4 iniconfig==2.0.0 joblib==1.3.2 Js2Py==0.74 +jsonschema==4.19.1 +jsonschema-specifications==2023.7.1 mmh3==4.0.1 mock==5.1.0 nltk==3.8.1 diff --git a/sdks/python/container/py39/base_image_requirements.txt b/sdks/python/container/py39/base_image_requirements.txt index ff6ba0945e14..5d92e9601071 100644 --- a/sdks/python/container/py39/base_image_requirements.txt +++ b/sdks/python/container/py39/base_image_requirements.txt @@ -81,6 +81,8 @@ idna==3.4 iniconfig==2.0.0 joblib==1.3.2 Js2Py==0.74 +jsonschema==4.19.1 +jsonschema-specifications==2023.7.1 mmh3==4.0.1 mock==5.1.0 nltk==3.8.1 diff --git a/sdks/python/setup.py b/sdks/python/setup.py index 4a05544526fc..32b623c0b40f 100644 --- a/sdks/python/setup.py +++ b/sdks/python/setup.py @@ -261,6 +261,7 @@ def get_portability_package_data(): 'hdfs>=2.1.0,<3.0.0', 'httplib2>=0.8,<0.23.0', 'js2py>=0.74,<1', + 'jsonschema>=4.0.0,<5.0.0', # numpy can have breaking changes in minor versions. # Use a strict upper bound. 'numpy>=1.14.3,<1.25.0', # Update pyproject.toml as well. From b402fafd44f08db06700b33b8414f4bcb0b6a00a Mon Sep 17 00:00:00 2001 From: Robert Bradshaw <robertwb@gmail.com> Date: Thu, 19 Oct 2023 13:46:02 -0700 Subject: [PATCH 201/435] Validate incoming JSON data from pubsub. As well as good practice, not doing so may result in much more obscure errors (e.g. during encoding) downstream. --- sdks/python/apache_beam/yaml/json_utils.py | 39 +++++++++++++++++++- sdks/python/apache_beam/yaml/yaml_io.py | 10 ++++- sdks/python/apache_beam/yaml/yaml_io_test.py | 32 ++++++++++++++++ 3 files changed, 77 insertions(+), 4 deletions(-) diff --git a/sdks/python/apache_beam/yaml/json_utils.py b/sdks/python/apache_beam/yaml/json_utils.py index e2cb03dc96a0..5b0f5d74fa77 100644 --- a/sdks/python/apache_beam/yaml/json_utils.py +++ b/sdks/python/apache_beam/yaml/json_utils.py @@ -24,6 +24,9 @@ from typing import Any from typing import Callable from typing import Dict +from typing import Optional + +import jsonschema import apache_beam as beam from apache_beam.portability.api import schema_pb2 @@ -131,15 +134,47 @@ def json_to_row(beam_type: schema_pb2.FieldType) -> Callable[[Any], Any]: raise ValueError(f"Unrecognized type_info: {type_info!r}") -def json_parser(beam_schema: schema_pb2.Schema) -> Callable[[bytes], beam.Row]: +def json_parser( + beam_schema: schema_pb2.Schema, + json_schema: Optional[Dict[str, + Any]] = None) -> Callable[[bytes], beam.Row]: """Returns a callable converting Json strings to Beam rows of the given type. The input to the returned callable is expected to conform to the Json schema corresponding to this Beam type. """ + if json_schema is None: + validate_fn = None + else: + cls = jsonschema.validators.validator_for(json_schema) + cls.check_schema(json_schema) + validate_fn = _PicklableFromConstructor(lambda: jsonschema.validators.validator_for(json_schema)( + json_schema).validate) + to_row = json_to_row( schema_pb2.FieldType(row_type=schema_pb2.RowType(schema=beam_schema))) - return lambda s: to_row(json.loads(s)) + + def parse(s: bytes): + o = json.loads(s) + if validate_fn is not None: + validate_fn(o) + return to_row(o) + + return parse + + +class _PicklableFromConstructor: + def __init__(self, constructor): + self._constructor = constructor + self._value = None + + def __call__(self, o): + if self._value is None: + self._value = self._constructor() + return self._value(o) + + def __getstate__(self): + return {'_constructor': self._constructor, '_value': None} def row_to_json(beam_type: schema_pb2.FieldType) -> Callable[[Any], Any]: diff --git a/sdks/python/apache_beam/yaml/yaml_io.py b/sdks/python/apache_beam/yaml/yaml_io.py index b2bf150fa558..bf4009719b80 100644 --- a/sdks/python/apache_beam/yaml/yaml_io.py +++ b/sdks/python/apache_beam/yaml/yaml_io.py @@ -148,7 +148,7 @@ def _create_parser( lambda payload: beam.Row(payload=payload)) elif format == 'json': beam_schema = json_utils.json_schema_to_beam_schema(schema) - return beam_schema, json_utils.json_parser(beam_schema) + return beam_schema, json_utils.json_parser(beam_schema, schema) elif format == 'avro': beam_schema = avroio.avro_schema_to_beam_schema(schema) covert_to_row = avroio.avro_dict_to_beam_row(schema, beam_schema) @@ -215,6 +215,8 @@ def read_from_pubsub( - raw: Produces records with a single `payload` field whose contents are the raw bytes of the pubsub message. + - avro: Parses records with a given avro schema. + - json: Parses records with a given json schema. schema: Schema specification for the given format. attributes: List of attribute keys whose values will be flattened into the @@ -309,8 +311,12 @@ def write_to_pubsub( formats are - raw: Expects a message with a single field (excluding - attribute-related fields )whose contents are used as the raw bytes + attribute-related fields) whose contents are used as the raw bytes of the pubsub message. + - avro: Encodes records with a given avro schema, which may be inferred + from the input PCollection schema. + - json: Formats records with a given json schema, which may be inferred + from the input PCollection schema. schema: Schema specification for the given format. attributes: List of attribute keys whose values will be pulled out as diff --git a/sdks/python/apache_beam/yaml/yaml_io_test.py b/sdks/python/apache_beam/yaml/yaml_io_test.py index 7071860a7bf1..10aec8e8256d 100644 --- a/sdks/python/apache_beam/yaml/yaml_io_test.py +++ b/sdks/python/apache_beam/yaml/yaml_io_test.py @@ -307,6 +307,38 @@ def test_read_json_without_error_handling(self): some_int: {type: integer} ''') + def test_read_json_with_bad_schema(self): + with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions( + pickle_library='cloudpickle')) as p: + with mock.patch( + 'apache_beam.io.ReadFromPubSub', + FakeReadFromPubSub(topic='my_topic', + messages=[PubsubMessage('{"some_int": 123}', + attributes={}), + PubsubMessage('{"some_int": "NOT"}', + attributes={})])): + result = p | YamlTransform( + ''' + type: ReadFromPubSub + config: + topic: my_topic + format: json + schema: + type: object + properties: + some_int: {type: integer} + error_handling: + output: errors + ''') + assert_that( + result['good'], + equal_to([beam.Row(some_int=123)]), + label='CheckGood') + assert_that( + result['errors'] | beam.Map(lambda error: error.element), + equal_to(['{"some_int": "NOT"}']), + label='CheckErrors') + def test_simple_write(self): with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions( pickle_library='cloudpickle')) as p: From 39678b6acd7121a893059236f5d0cb6a61aef9c0 Mon Sep 17 00:00:00 2001 From: Rebecca Szper <98840847+rszper@users.noreply.github.com> Date: Thu, 19 Oct 2023 13:57:23 -0700 Subject: [PATCH 202/435] Remove command prompts from Go quickstart commands (#29079) --- .../www/site/content/en/get-started/quickstart-go.md | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/website/www/site/content/en/get-started/quickstart-go.md b/website/www/site/content/en/get-started/quickstart-go.md index 2f0bad49659c..dd8f3ba586c6 100644 --- a/website/www/site/content/en/get-started/quickstart-go.md +++ b/website/www/site/content/en/get-started/quickstart-go.md @@ -28,7 +28,7 @@ If you're interested in contributing to the Apache Beam Go codebase, see the [Co The Beam SDK for Go requires `go` version 1.20 or newer. It can be downloaded [here](https://golang.org/). Check what go version you have by running: {{< highlight >}} -$ go version +go version {{< /highlight >}} If you are unfamiliar with Go, see the [Get Started With Go Tutorial](https://go.dev/doc/tutorial/getting-started). @@ -43,12 +43,12 @@ required arguments described in the examples. For example, to run `wordcount`, run: {{< runner direct >}} -$ go run github.com/apache/beam/sdks/v2/go/examples/wordcount@latest --input "gs://apache-beam-samples/shakespeare/kinglear.txt" --output counts -$ less counts +go run github.com/apache/beam/sdks/v2/go/examples/wordcount@latest --input "gs://apache-beam-samples/shakespeare/kinglear.txt" --output counts +less counts {{< /runner >}} {{< runner dataflow >}} -$ go run github.com/apache/beam/sdks/v2/go/examples/wordcount@latest --input gs://dataflow-samples/shakespeare/kinglear.txt \ +go run github.com/apache/beam/sdks/v2/go/examples/wordcount@latest --input gs://dataflow-samples/shakespeare/kinglear.txt \ --output gs://<your-gcs-bucket>/counts \ --runner dataflow \ --project your-gcp-project \ @@ -60,10 +60,10 @@ $ go run github.com/apache/beam/sdks/v2/go/examples/wordcount@latest --input gs: {{< runner spark >}} # Build and run the Spark job server from Beam source. # -PsparkMasterUrl is optional. If it is unset the job will be run inside an embedded Spark cluster. -$ ./gradlew :runners:spark:3:job-server:runShadow -PsparkMasterUrl=spark://localhost:7077 +./gradlew :runners:spark:3:job-server:runShadow -PsparkMasterUrl=spark://localhost:7077 # In a separate terminal, run: -$ go run github.com/apache/beam/sdks/v2/go/examples/wordcount@latest --input <PATH_TO_INPUT_FILE> \ +go run github.com/apache/beam/sdks/v2/go/examples/wordcount@latest --input <PATH_TO_INPUT_FILE> \ --output counts \ --runner spark \ --endpoint localhost:8099 From 67a39a50df3f677d9d1e8c8519f80b222c1101fb Mon Sep 17 00:00:00 2001 From: "gabry.wu" <gabrywu@apache.org> Date: Fri, 20 Oct 2023 05:56:41 +0800 Subject: [PATCH 203/435] make BeamKafkaTable.createKafkaRead to be protected, and we can override it appending options to KafkaIO.Read (#29051) --- .../sdk/extensions/sql/meta/provider/kafka/BeamKafkaTable.java | 2 +- .../sdk/extensions/sql/meta/provider/kafka/KafkaTestTable.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/BeamKafkaTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/BeamKafkaTable.java index f1ec20831a4c..ab1817f6d75c 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/BeamKafkaTable.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/BeamKafkaTable.java @@ -110,7 +110,7 @@ public PCollection<Row> buildIOReader(PBegin begin) { .setRowSchema(getSchema()); } - KafkaIO.Read<byte[], byte[]> createKafkaRead() { + protected KafkaIO.Read<byte[], byte[]> createKafkaRead() { KafkaIO.Read<byte[], byte[]> kafkaRead; if (topics != null) { kafkaRead = diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/KafkaTestTable.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/KafkaTestTable.java index 44b4dbe21aca..158b0345bd8b 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/KafkaTestTable.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/KafkaTestTable.java @@ -61,7 +61,7 @@ public KafkaTestTable(Schema beamSchema, List<String> topics, int partitionsPerT } @Override - KafkaIO.Read<byte[], byte[]> createKafkaRead() { + protected KafkaIO.Read<byte[], byte[]> createKafkaRead() { return super.createKafkaRead().withConsumerFactoryFn(this::mkMockConsumer); } From 2e676a369ad98d66103c5cbe450ec7796681a1e9 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw <robertwb@gmail.com> Date: Thu, 19 Oct 2023 15:16:08 -0700 Subject: [PATCH 204/435] Regenrate python base deps files. --- .../py310/base_image_requirements.txt | 58 +++++++++--------- .../py311/base_image_requirements.txt | 56 +++++++++-------- .../py38/base_image_requirements.txt | 60 +++++++++++-------- .../py39/base_image_requirements.txt | 60 ++++++++++--------- 4 files changed, 128 insertions(+), 106 deletions(-) diff --git a/sdks/python/container/py310/base_image_requirements.txt b/sdks/python/container/py310/base_image_requirements.txt index 07e03a539bb0..d603ac16c0e2 100644 --- a/sdks/python/container/py310/base_image_requirements.txt +++ b/sdks/python/container/py310/base_image_requirements.txt @@ -24,10 +24,11 @@ attrs==23.1.0 beautifulsoup4==4.12.2 bs4==0.0.1 +build==1.0.3 cachetools==5.3.1 certifi==2023.7.22 -cffi==1.15.1 -charset-normalizer==3.2.0 +cffi==1.16.0 +charset-normalizer==3.3.0 click==8.1.7 cloudpickle==2.2.1 crcmod==1.7 @@ -40,17 +41,17 @@ docker==6.1.3 docopt==0.6.2 exceptiongroup==1.1.3 execnet==2.0.2 -fastavro==1.8.3 +fastavro==1.8.4 fasteners==0.19 freezegun==1.2.2 future==0.18.3 -google-api-core==2.11.1 -google-api-python-client==2.100.0 +google-api-core==2.12.0 +google-api-python-client==2.104.0 google-apitools==0.5.31 -google-auth==2.23.0 +google-auth==2.23.3 google-auth-httplib2==0.1.1 -google-cloud-aiplatform==1.33.1 -google-cloud-bigquery==3.11.4 +google-cloud-aiplatform==1.35.0 +google-cloud-bigquery==3.12.0 google-cloud-bigquery-storage==2.22.0 google-cloud-bigtable==2.21.0 google-cloud-core==2.3.3 @@ -63,20 +64,20 @@ google-cloud-pubsublite==1.8.3 google-cloud-recommendations-ai==0.10.5 google-cloud-resource-manager==1.10.4 google-cloud-spanner==3.40.1 -google-cloud-storage==2.11.0 +google-cloud-storage==2.12.0 google-cloud-videointelligence==2.11.4 -google-cloud-vision==3.4.4 +google-cloud-vision==3.4.5 google-crc32c==1.5.0 google-resumable-media==2.6.0 -googleapis-common-protos==1.60.0 -greenlet==2.0.2 +googleapis-common-protos==1.61.0 +greenlet==3.0.0 grpc-google-iam-v1==0.12.6 -grpcio==1.58.0 -grpcio-status==1.58.0 +grpcio==1.59.0 +grpcio-status==1.59.0 guppy3==3.1.3 -hdfs==2.7.2 +hdfs==2.7.3 httplib2==0.22.0 -hypothesis==6.87.0 +hypothesis==6.88.1 idna==3.4 iniconfig==2.0.0 joblib==1.3.2 @@ -90,15 +91,15 @@ nose==1.3.7 numpy==1.24.4 oauth2client==4.1.3 objsize==0.6.1 -orjson==3.9.7 +orjson==3.9.9 overrides==6.5.0 -packaging==23.1 +packaging==23.2 pandas==1.5.3 parameterized==0.9.0 pluggy==1.3.0 proto-plus==1.22.3 -protobuf==4.24.3 -psycopg2-binary==2.9.7 +protobuf==4.24.4 +psycopg2-binary==2.9.9 pyarrow==11.0.0 pyasn1==0.5.0 pyasn1-modules==0.3.0 @@ -109,20 +110,23 @@ pyjsparser==2.7.1 pymongo==4.5.0 PyMySQL==1.1.0 pyparsing==3.1.1 +pyproject_hooks==1.0.0 pytest==7.4.2 -pytest-timeout==2.1.0 +pytest-timeout==2.2.0 pytest-xdist==3.3.1 python-dateutil==2.8.2 python-snappy==0.6.1 pytz==2023.3.post1 PyYAML==6.0.1 -regex==2023.8.8 +referencing==0.30.2 +regex==2023.10.3 requests==2.31.0 requests-mock==1.11.0 +rpds-py==0.10.6 rsa==4.9 scikit-learn==1.3.1 -scipy==1.11.2 -Shapely==1.8.5.post1 +scipy==1.11.3 +shapely==2.0.2 six==1.16.0 sortedcontainers==2.4.0 soupsieve==2.5 @@ -134,9 +138,9 @@ threadpoolctl==3.2.0 tomli==2.0.1 tqdm==4.66.1 typing_extensions==4.8.0 -tzlocal==5.0.1 +tzlocal==5.1 uritemplate==4.1.1 -urllib3==1.26.17 -websocket-client==1.6.3 +urllib3==2.0.7 +websocket-client==1.6.4 wrapt==1.15.0 zstandard==0.21.0 diff --git a/sdks/python/container/py311/base_image_requirements.txt b/sdks/python/container/py311/base_image_requirements.txt index 18f55e5d6927..6f5d72aa35d3 100644 --- a/sdks/python/container/py311/base_image_requirements.txt +++ b/sdks/python/container/py311/base_image_requirements.txt @@ -24,10 +24,11 @@ attrs==23.1.0 beautifulsoup4==4.12.2 bs4==0.0.1 +build==1.0.3 cachetools==5.3.1 certifi==2023.7.22 -cffi==1.15.1 -charset-normalizer==3.2.0 +cffi==1.16.0 +charset-normalizer==3.3.0 click==8.1.7 cloudpickle==2.2.1 crcmod==1.7 @@ -39,16 +40,16 @@ dnspython==2.4.2 docker==6.1.3 docopt==0.6.2 execnet==2.0.2 -fastavro==1.8.3 +fastavro==1.8.4 fasteners==0.19 freezegun==1.2.2 future==0.18.3 -google-api-core==2.11.1 +google-api-core==2.12.0 google-apitools==0.5.31 -google-auth==2.23.0 +google-auth==2.23.3 google-auth-httplib2==0.1.1 -google-cloud-aiplatform==1.33.1 -google-cloud-bigquery==3.11.4 +google-cloud-aiplatform==1.35.0 +google-cloud-bigquery==3.12.0 google-cloud-bigquery-storage==2.22.0 google-cloud-bigtable==2.21.0 google-cloud-core==2.3.3 @@ -60,20 +61,20 @@ google-cloud-pubsublite==1.8.3 google-cloud-recommendations-ai==0.10.5 google-cloud-resource-manager==1.10.4 google-cloud-spanner==3.40.1 -google-cloud-storage==2.11.0 +google-cloud-storage==2.12.0 google-cloud-videointelligence==2.11.4 -google-cloud-vision==3.4.4 +google-cloud-vision==3.4.5 google-crc32c==1.5.0 google-resumable-media==2.6.0 -googleapis-common-protos==1.60.0 -greenlet==2.0.2 +googleapis-common-protos==1.61.0 +greenlet==3.0.0 grpc-google-iam-v1==0.12.6 -grpcio==1.58.0 -grpcio-status==1.58.0 +grpcio==1.59.0 +grpcio-status==1.59.0 guppy3==3.1.3 -hdfs==2.7.2 +hdfs==2.7.3 httplib2==0.22.0 -hypothesis==6.87.0 +hypothesis==6.88.1 idna==3.4 iniconfig==2.0.0 joblib==1.3.2 @@ -87,15 +88,15 @@ nose==1.3.7 numpy==1.24.4 oauth2client==4.1.3 objsize==0.6.1 -orjson==3.9.7 +orjson==3.9.9 overrides==6.5.0 -packaging==23.1 +packaging==23.2 pandas==1.5.3 parameterized==0.9.0 pluggy==1.3.0 proto-plus==1.22.3 -protobuf==4.24.3 -psycopg2-binary==2.9.7 +protobuf==4.24.4 +psycopg2-binary==2.9.9 pyarrow==11.0.0 pyasn1==0.5.0 pyasn1-modules==0.3.0 @@ -106,19 +107,22 @@ pyjsparser==2.7.1 pymongo==4.5.0 PyMySQL==1.1.0 pyparsing==3.1.1 +pyproject_hooks==1.0.0 pytest==7.4.2 -pytest-timeout==2.1.0 +pytest-timeout==2.2.0 pytest-xdist==3.3.1 python-dateutil==2.8.2 pytz==2023.3.post1 PyYAML==6.0.1 -regex==2023.8.8 +referencing==0.30.2 +regex==2023.10.3 requests==2.31.0 requests-mock==1.11.0 +rpds-py==0.10.6 rsa==4.9 scikit-learn==1.3.1 -scipy==1.11.2 -Shapely==1.8.5.post1 +scipy==1.11.3 +shapely==2.0.2 six==1.16.0 sortedcontainers==2.4.0 soupsieve==2.5 @@ -129,8 +133,8 @@ testcontainers==3.7.1 threadpoolctl==3.2.0 tqdm==4.66.1 typing_extensions==4.8.0 -tzlocal==5.0.1 -urllib3==1.26.16 -websocket-client==1.6.3 +tzlocal==5.1 +urllib3==2.0.7 +websocket-client==1.6.4 wrapt==1.15.0 zstandard==0.21.0 diff --git a/sdks/python/container/py38/base_image_requirements.txt b/sdks/python/container/py38/base_image_requirements.txt index e474ab69c64c..8e3a7ad4940f 100644 --- a/sdks/python/container/py38/base_image_requirements.txt +++ b/sdks/python/container/py38/base_image_requirements.txt @@ -25,10 +25,11 @@ attrs==23.1.0 backports.zoneinfo==0.2.1 beautifulsoup4==4.12.2 bs4==0.0.1 +build==1.0.3 cachetools==5.3.1 certifi==2023.7.22 -cffi==1.15.1 -charset-normalizer==3.2.0 +cffi==1.16.0 +charset-normalizer==3.3.0 click==8.1.7 cloudpickle==2.2.1 crcmod==1.7 @@ -41,17 +42,17 @@ docker==6.1.3 docopt==0.6.2 exceptiongroup==1.1.3 execnet==2.0.2 -fastavro==1.8.3 +fastavro==1.8.4 fasteners==0.19 freezegun==1.2.2 future==0.18.3 -google-api-core==2.11.1 -google-api-python-client==2.100.0 +google-api-core==2.12.0 +google-api-python-client==2.104.0 google-apitools==0.5.31 -google-auth==2.23.0 +google-auth==2.23.3 google-auth-httplib2==0.1.1 -google-cloud-aiplatform==1.33.1 -google-cloud-bigquery==3.11.4 +google-cloud-aiplatform==1.35.0 +google-cloud-bigquery==3.12.0 google-cloud-bigquery-storage==2.22.0 google-cloud-bigtable==2.21.0 google-cloud-core==2.3.3 @@ -64,21 +65,23 @@ google-cloud-pubsublite==1.8.3 google-cloud-recommendations-ai==0.10.5 google-cloud-resource-manager==1.10.4 google-cloud-spanner==3.40.1 -google-cloud-storage==2.11.0 +google-cloud-storage==2.12.0 google-cloud-videointelligence==2.11.4 -google-cloud-vision==3.4.4 +google-cloud-vision==3.4.5 google-crc32c==1.5.0 google-resumable-media==2.6.0 -googleapis-common-protos==1.60.0 -greenlet==2.0.2 +googleapis-common-protos==1.61.0 +greenlet==3.0.0 grpc-google-iam-v1==0.12.6 -grpcio==1.58.0 -grpcio-status==1.58.0 +grpcio==1.59.0 +grpcio-status==1.59.0 guppy3==3.1.3 -hdfs==2.7.2 +hdfs==2.7.3 httplib2==0.22.0 -hypothesis==6.87.0 +hypothesis==6.88.1 idna==3.4 +importlib-metadata==6.8.0 +importlib-resources==6.1.0 iniconfig==2.0.0 joblib==1.3.2 Js2Py==0.74 @@ -91,15 +94,16 @@ nose==1.3.7 numpy==1.24.4 oauth2client==4.1.3 objsize==0.6.1 -orjson==3.9.7 +orjson==3.9.9 overrides==6.5.0 -packaging==23.1 +packaging==23.2 pandas==1.5.3 parameterized==0.9.0 +pkgutil_resolve_name==1.3.10 pluggy==1.3.0 proto-plus==1.22.3 -protobuf==4.24.3 -psycopg2-binary==2.9.7 +protobuf==4.24.4 +psycopg2-binary==2.9.9 pyarrow==11.0.0 pyasn1==0.5.0 pyasn1-modules==0.3.0 @@ -110,20 +114,23 @@ pyjsparser==2.7.1 pymongo==4.5.0 PyMySQL==1.1.0 pyparsing==3.1.1 +pyproject_hooks==1.0.0 pytest==7.4.2 -pytest-timeout==2.1.0 +pytest-timeout==2.2.0 pytest-xdist==3.3.1 python-dateutil==2.8.2 python-snappy==0.6.1 pytz==2023.3.post1 PyYAML==6.0.1 -regex==2023.8.8 +referencing==0.30.2 +regex==2023.10.3 requests==2.31.0 requests-mock==1.11.0 +rpds-py==0.10.6 rsa==4.9 scikit-learn==1.3.1 scipy==1.10.1 -Shapely==1.8.5.post1 +shapely==2.0.2 six==1.16.0 sortedcontainers==2.4.0 soupsieve==2.5 @@ -135,9 +142,10 @@ threadpoolctl==3.2.0 tomli==2.0.1 tqdm==4.66.1 typing_extensions==4.8.0 -tzlocal==5.0.1 +tzlocal==5.1 uritemplate==4.1.1 -urllib3==1.26.17 -websocket-client==1.6.3 +urllib3==2.0.7 +websocket-client==1.6.4 wrapt==1.15.0 +zipp==3.17.0 zstandard==0.21.0 diff --git a/sdks/python/container/py39/base_image_requirements.txt b/sdks/python/container/py39/base_image_requirements.txt index 5d92e9601071..14e077b79223 100644 --- a/sdks/python/container/py39/base_image_requirements.txt +++ b/sdks/python/container/py39/base_image_requirements.txt @@ -24,10 +24,11 @@ attrs==23.1.0 beautifulsoup4==4.12.2 bs4==0.0.1 +build==1.0.3 cachetools==5.3.1 certifi==2023.7.22 -cffi==1.15.1 -charset-normalizer==3.2.0 +cffi==1.16.0 +charset-normalizer==3.3.0 click==8.1.7 cloudpickle==2.2.1 crcmod==1.7 @@ -40,17 +41,17 @@ docker==6.1.3 docopt==0.6.2 exceptiongroup==1.1.3 execnet==2.0.2 -fastavro==1.8.3 +fastavro==1.8.4 fasteners==0.19 freezegun==1.2.2 future==0.18.3 -google-api-core==2.11.1 -google-api-python-client==2.100.0 +google-api-core==2.12.0 +google-api-python-client==2.104.0 google-apitools==0.5.31 -google-auth==2.23.0 +google-auth==2.23.3 google-auth-httplib2==0.1.1 -google-cloud-aiplatform==1.33.1 -google-cloud-bigquery==3.11.4 +google-cloud-aiplatform==1.35.0 +google-cloud-bigquery==3.12.0 google-cloud-bigquery-storage==2.22.0 google-cloud-bigtable==2.21.0 google-cloud-core==2.3.3 @@ -63,21 +64,22 @@ google-cloud-pubsublite==1.8.3 google-cloud-recommendations-ai==0.10.5 google-cloud-resource-manager==1.10.4 google-cloud-spanner==3.40.1 -google-cloud-storage==2.11.0 +google-cloud-storage==2.12.0 google-cloud-videointelligence==2.11.4 -google-cloud-vision==3.4.4 +google-cloud-vision==3.4.5 google-crc32c==1.5.0 google-resumable-media==2.6.0 -googleapis-common-protos==1.60.0 -greenlet==2.0.2 +googleapis-common-protos==1.61.0 +greenlet==3.0.0 grpc-google-iam-v1==0.12.6 -grpcio==1.58.0 -grpcio-status==1.58.0 +grpcio==1.59.0 +grpcio-status==1.59.0 guppy3==3.1.3 -hdfs==2.7.2 +hdfs==2.7.3 httplib2==0.22.0 -hypothesis==6.87.0 +hypothesis==6.88.1 idna==3.4 +importlib-metadata==6.8.0 iniconfig==2.0.0 joblib==1.3.2 Js2Py==0.74 @@ -90,15 +92,15 @@ nose==1.3.7 numpy==1.24.4 oauth2client==4.1.3 objsize==0.6.1 -orjson==3.9.7 +orjson==3.9.9 overrides==6.5.0 -packaging==23.1 +packaging==23.2 pandas==1.5.3 parameterized==0.9.0 pluggy==1.3.0 proto-plus==1.22.3 -protobuf==4.24.3 -psycopg2-binary==2.9.7 +protobuf==4.24.4 +psycopg2-binary==2.9.9 pyarrow==11.0.0 pyasn1==0.5.0 pyasn1-modules==0.3.0 @@ -109,20 +111,23 @@ pyjsparser==2.7.1 pymongo==4.5.0 PyMySQL==1.1.0 pyparsing==3.1.1 +pyproject_hooks==1.0.0 pytest==7.4.2 -pytest-timeout==2.1.0 +pytest-timeout==2.2.0 pytest-xdist==3.3.1 python-dateutil==2.8.2 python-snappy==0.6.1 pytz==2023.3.post1 PyYAML==6.0.1 -regex==2023.8.8 +referencing==0.30.2 +regex==2023.10.3 requests==2.31.0 requests-mock==1.11.0 +rpds-py==0.10.6 rsa==4.9 scikit-learn==1.3.1 -scipy==1.11.2 -Shapely==1.8.5.post1 +scipy==1.11.3 +shapely==2.0.2 six==1.16.0 sortedcontainers==2.4.0 soupsieve==2.5 @@ -134,9 +139,10 @@ threadpoolctl==3.2.0 tomli==2.0.1 tqdm==4.66.1 typing_extensions==4.8.0 -tzlocal==5.0.1 +tzlocal==5.1 uritemplate==4.1.1 -urllib3==1.26.16 -websocket-client==1.6.3 +urllib3==2.0.7 +websocket-client==1.6.4 wrapt==1.15.0 +zipp==3.17.0 zstandard==0.21.0 From 7632d48c3bdcf2309e8e4d43bb282e6ba38072c8 Mon Sep 17 00:00:00 2001 From: Yi Hu <yathu@google.com> Date: Thu, 19 Oct 2023 18:42:58 -0400 Subject: [PATCH 205/435] TImeout StreamingDataflowWorkerTest when test stuck (#29065) --- .../runners/dataflow/worker/StreamingDataflowWorkerTest.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java index 24e6e2795c68..c4042e37c3bd 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java @@ -178,6 +178,7 @@ import org.junit.Test; import org.junit.rules.ErrorCollector; import org.junit.rules.TestRule; +import org.junit.rules.Timeout; import org.junit.runner.Description; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -254,6 +255,7 @@ public Long get() { return idGenerator.getAndIncrement(); } }; + @Rule public transient Timeout globalTimeout = Timeout.seconds(600); @Rule public BlockingFn blockingFn = new BlockingFn(); @Rule public TestRule restoreMDC = new RestoreDataflowLoggingMDC(); @Rule public ErrorCollector errorCollector = new ErrorCollector(); From c6ab5fe272f74749713a5ccf7c21a398d017a606 Mon Sep 17 00:00:00 2001 From: martin trieu <martinkt@google.com> Date: Fri, 20 Oct 2023 05:20:37 -0700 Subject: [PATCH 206/435] Refactor StateFetcher (#28755) Refactor and cleanup of StateFetcher in preparation for future changes --- .../runners/dataflow/worker/StateFetcher.java | 291 ------------------ .../worker/StreamingDataflowWorker.java | 11 +- .../worker/StreamingModeExecutionContext.java | 114 ++++--- .../worker/StreamingSideInputFetcher.java | 8 +- .../worker/streaming/sideinput/SideInput.java | 50 +++ .../streaming/sideinput/SideInputCache.java | 113 +++++++ .../streaming/sideinput/SideInputState.java | 25 ++ .../sideinput/SideInputStateFetcher.java | 245 +++++++++++++++ .../worker/StreamingDataflowWorkerTest.java | 6 +- .../StreamingModeExecutionContextTest.java | 9 +- .../StreamingSideInputDoFnRunnerTest.java | 2 +- .../worker/StreamingSideInputFetcherTest.java | 2 +- .../sideinput/SideInputStateFetcherTest.java} | 170 ++++++---- 13 files changed, 626 insertions(+), 420 deletions(-) delete mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StateFetcher.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/sideinput/SideInput.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/sideinput/SideInputCache.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/sideinput/SideInputState.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/sideinput/SideInputStateFetcher.java rename runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/{StateFetcherTest.java => streaming/sideinput/SideInputStateFetcherTest.java} (67%) diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StateFetcher.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StateFetcher.java deleted file mode 100644 index 0cbcd2e83012..000000000000 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StateFetcher.java +++ /dev/null @@ -1,291 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.runners.dataflow.worker; - -import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; - -import java.io.Closeable; -import java.util.Collections; -import java.util.Objects; -import java.util.Set; -import java.util.concurrent.Callable; -import java.util.concurrent.TimeUnit; -import org.apache.beam.runners.core.InMemoryMultimapSideInputView; -import org.apache.beam.runners.dataflow.worker.windmill.Windmill; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.IterableCoder; -import org.apache.beam.sdk.coders.KvCoder; -import org.apache.beam.sdk.transforms.Materializations; -import org.apache.beam.sdk.transforms.Materializations.IterableView; -import org.apache.beam.sdk.transforms.Materializations.MultimapView; -import org.apache.beam.sdk.transforms.ViewFn; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.ByteStringOutputStream; -import org.apache.beam.sdk.values.PCollectionView; -import org.apache.beam.sdk.values.TupleTag; -import org.apache.beam.sdk.values.WindowingStrategy; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Optional; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Supplier; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.Cache; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.CacheBuilder; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.Weigher; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; -import org.checkerframework.checker.nullness.qual.Nullable; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** Class responsible for fetching state from the windmill server. */ -@SuppressWarnings({ - "rawtypes", // TODO(https://github.com/apache/beam/issues/20447) - "nullness" // TODO(https://github.com/apache/beam/issues/20497) -}) -class StateFetcher { - private static final Set<String> SUPPORTED_MATERIALIZATIONS = - ImmutableSet.of( - Materializations.ITERABLE_MATERIALIZATION_URN, - Materializations.MULTIMAP_MATERIALIZATION_URN); - - private static final Logger LOG = LoggerFactory.getLogger(StateFetcher.class); - - private Cache<SideInputId, SideInputCacheEntry> sideInputCache; - private MetricTrackingWindmillServerStub server; - private long bytesRead = 0L; - - public StateFetcher(MetricTrackingWindmillServerStub server) { - this( - server, - CacheBuilder.newBuilder() - .maximumWeight(100000000 /* 100 MB */) - .expireAfterWrite(1, TimeUnit.MINUTES) - .weigher((Weigher<SideInputId, SideInputCacheEntry>) (id, entry) -> entry.size()) - .build()); - } - - public StateFetcher( - MetricTrackingWindmillServerStub server, - Cache<SideInputId, SideInputCacheEntry> sideInputCache) { - this.server = server; - this.sideInputCache = sideInputCache; - } - - /** Returns a view of the underlying cache that keeps track of bytes read separately. */ - public StateFetcher byteTrackingView() { - return new StateFetcher(server, sideInputCache); - } - - public long getBytesRead() { - return bytesRead; - } - - /** Indicates the caller's knowledge of whether a particular side input has been computed. */ - public enum SideInputState { - CACHED_IN_WORKITEM, - KNOWN_READY, - UNKNOWN; - } - - /** - * Fetch the given side input, storing it in a process-level cache. - * - * <p>If state is KNOWN_READY, attempt to fetch the data regardless of whether a not-ready entry - * was cached. - * - * <p>Returns {@literal null} if the side input was not ready, {@literal Optional.absent()} if the - * side input was null, and {@literal Optional.present(...)} if the side input was non-null. - */ - public @Nullable <T, SideWindowT extends BoundedWindow> Optional<T> fetchSideInput( - final PCollectionView<T> view, - final SideWindowT sideWindow, - final String stateFamily, - SideInputState state, - final Supplier<Closeable> scopedReadStateSupplier) { - final SideInputId id = new SideInputId(view.getTagInternal(), sideWindow); - - Callable<SideInputCacheEntry> fetchCallable = - () -> { - @SuppressWarnings("unchecked") - WindowingStrategy<?, SideWindowT> sideWindowStrategy = - (WindowingStrategy<?, SideWindowT>) view.getWindowingStrategyInternal(); - - Coder<SideWindowT> windowCoder = sideWindowStrategy.getWindowFn().windowCoder(); - - ByteStringOutputStream windowStream = new ByteStringOutputStream(); - windowCoder.encode(sideWindow, windowStream, Coder.Context.OUTER); - - @SuppressWarnings("unchecked") - Windmill.GlobalDataRequest request = - Windmill.GlobalDataRequest.newBuilder() - .setDataId( - Windmill.GlobalDataId.newBuilder() - .setTag(view.getTagInternal().getId()) - .setVersion(windowStream.toByteString()) - .build()) - .setStateFamily(stateFamily) - .setExistenceWatermarkDeadline( - WindmillTimeUtils.harnessToWindmillTimestamp( - sideWindowStrategy - .getTrigger() - .getWatermarkThatGuaranteesFiring(sideWindow))) - .build(); - - Windmill.GlobalData data; - try (Closeable scope = scopedReadStateSupplier.get()) { - data = server.getSideInputData(request); - } - - bytesRead += data.getSerializedSize(); - - checkState( - SUPPORTED_MATERIALIZATIONS.contains(view.getViewFn().getMaterialization().getUrn()), - "Only materializations of type %s supported, received %s", - SUPPORTED_MATERIALIZATIONS, - view.getViewFn().getMaterialization().getUrn()); - - Iterable<?> rawData; - if (data.getIsReady()) { - if (data.getData().size() > 0) { - rawData = - IterableCoder.of(view.getCoderInternal()) - .decode(data.getData().newInput(), Coder.Context.OUTER); - } else { - rawData = Collections.emptyList(); - } - - switch (view.getViewFn().getMaterialization().getUrn()) { - case Materializations.ITERABLE_MATERIALIZATION_URN: - { - ViewFn<IterableView, T> viewFn = (ViewFn<IterableView, T>) view.getViewFn(); - return SideInputCacheEntry.ready( - viewFn.apply(() -> rawData), data.getData().size()); - } - case Materializations.MULTIMAP_MATERIALIZATION_URN: - { - ViewFn<MultimapView, T> viewFn = (ViewFn<MultimapView, T>) view.getViewFn(); - Coder<?> keyCoder = ((KvCoder<?, ?>) view.getCoderInternal()).getKeyCoder(); - return SideInputCacheEntry.ready( - viewFn.apply( - InMemoryMultimapSideInputView.fromIterable(keyCoder, (Iterable) rawData)), - data.getData().size()); - } - default: - throw new IllegalStateException( - String.format( - "Unknown side input materialization format requested '%s'", - view.getViewFn().getMaterialization().getUrn())); - } - } else { - return SideInputCacheEntry.notReady(); - } - }; - - try { - if (state == SideInputState.KNOWN_READY) { - SideInputCacheEntry entry = sideInputCache.getIfPresent(id); - if (entry == null) { - return sideInputCache.get(id, fetchCallable).getValue(); - } else if (!entry.isReady()) { - // Invalidate the existing not-ready entry. This must be done atomically - // so that another thread doesn't replace the entry with a ready entry, which - // would then be deleted here. - synchronized (entry) { - SideInputCacheEntry newEntry = sideInputCache.getIfPresent(id); - if (newEntry != null && !newEntry.isReady()) { - sideInputCache.invalidate(id); - } - } - - return sideInputCache.get(id, fetchCallable).getValue(); - } else { - return entry.getValue(); - } - } else { - return sideInputCache.get(id, fetchCallable).getValue(); - } - } catch (Exception e) { - LOG.error("Fetch failed: ", e); - throw new RuntimeException("Exception while fetching side input: ", e); - } - } - - /** Struct representing a side input for a particular window. */ - static class SideInputId { - private final TupleTag<?> tag; - private final BoundedWindow window; - - public SideInputId(TupleTag<?> tag, BoundedWindow window) { - this.tag = tag; - this.window = window; - } - - @Override - public boolean equals(@Nullable Object other) { - if (other instanceof SideInputId) { - SideInputId otherId = (SideInputId) other; - return tag.equals(otherId.tag) && window.equals(otherId.window); - } - return false; - } - - @Override - public int hashCode() { - return Objects.hash(tag, window); - } - } - - /** - * Entry in the side input cache that stores the value (null if not ready), and the encoded size - * of the value. - */ - static class SideInputCacheEntry { - private final boolean ready; - private final Object value; - private final int encodedSize; - - private SideInputCacheEntry(boolean ready, Object value, int encodedSize) { - this.ready = ready; - this.value = value; - this.encodedSize = encodedSize; - } - - public static SideInputCacheEntry ready(Object value, int encodedSize) { - return new SideInputCacheEntry(true, value, encodedSize); - } - - public static SideInputCacheEntry notReady() { - return new SideInputCacheEntry(false, null, 0); - } - - public boolean isReady() { - return ready; - } - - /** - * Returns {@literal null} if the side input was not ready, {@literal Optional.absent()} if the - * side input was null, and {@literal Optional.present(...)} if the side input was non-null. - */ - public @Nullable <T> Optional<T> getValue() { - @SuppressWarnings("unchecked") - T typed = (T) value; - return ready ? Optional.fromNullable(typed) : null; - } - - public int size() { - return encodedSize; - } - } -} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java index 4c1693d61387..77f5205cf7e9 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java @@ -94,6 +94,7 @@ import org.apache.beam.runners.dataflow.worker.streaming.WeightedBoundedQueue; import org.apache.beam.runners.dataflow.worker.streaming.Work; import org.apache.beam.runners.dataflow.worker.streaming.Work.State; +import org.apache.beam.runners.dataflow.worker.streaming.sideinput.SideInputStateFetcher; import org.apache.beam.runners.dataflow.worker.util.BoundedQueueExecutor; import org.apache.beam.runners.dataflow.worker.util.MemoryMonitor; import org.apache.beam.runners.dataflow.worker.util.common.worker.ElementCounter; @@ -228,7 +229,7 @@ public class StreamingDataflowWorker { private final Thread commitThread; private final AtomicLong activeCommitBytes = new AtomicLong(); private final AtomicBoolean running = new AtomicBoolean(); - private final StateFetcher stateFetcher; + private final SideInputStateFetcher sideInputStateFetcher; private final StreamingDataflowWorkerOptions options; private final boolean windmillServiceEnabled; private final long clientId; @@ -406,7 +407,7 @@ public void run() { this.metricTrackingWindmillServer = new MetricTrackingWindmillServerStub(windmillServer, memoryMonitor, windmillServiceEnabled); this.metricTrackingWindmillServer.start(); - this.stateFetcher = new StateFetcher(metricTrackingWindmillServer); + this.sideInputStateFetcher = new SideInputStateFetcher(metricTrackingWindmillServer); this.clientId = clientIdGenerator.nextLong(); for (MapTask mapTask : mapTasks) { @@ -1078,7 +1079,7 @@ public void close() { } }; }); - StateFetcher localStateFetcher = stateFetcher.byteTrackingView(); + SideInputStateFetcher localSideInputStateFetcher = sideInputStateFetcher.byteTrackingView(); // If the read output KVs, then we can decode Windmill's byte key into a userland // key object and provide it to the execution context for use with per-key state. @@ -1114,7 +1115,7 @@ public void close() { outputDataWatermark, synchronizedProcessingTime, stateReader, - localStateFetcher, + localSideInputStateFetcher, outputBuilder); // Blocks while executing work. @@ -1184,7 +1185,7 @@ public void close() { shuffleBytesRead += message.getSerializedSize(); } } - long stateBytesRead = stateReader.getBytesRead() + localStateFetcher.getBytesRead(); + long stateBytesRead = stateReader.getBytesRead() + localSideInputStateFetcher.getBytesRead(); windmillShuffleBytesRead.addValue(shuffleBytesRead); windmillStateBytesRead.addValue(stateBytesRead); windmillStateBytesWritten.addValue(stateBytesWritten); diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java index c8fa6e6dfb78..d630601c28a3 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java @@ -30,6 +30,7 @@ import java.util.List; import java.util.Map; import java.util.NavigableSet; +import java.util.Optional; import java.util.Set; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicLong; @@ -45,6 +46,9 @@ import org.apache.beam.runners.dataflow.worker.counters.CounterFactory; import org.apache.beam.runners.dataflow.worker.counters.NameContext; import org.apache.beam.runners.dataflow.worker.profiler.ScopedProfiler.ProfileScope; +import org.apache.beam.runners.dataflow.worker.streaming.sideinput.SideInput; +import org.apache.beam.runners.dataflow.worker.streaming.sideinput.SideInputState; +import org.apache.beam.runners.dataflow.worker.streaming.sideinput.SideInputStateFetcher; import org.apache.beam.runners.dataflow.worker.windmill.Windmill; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GlobalDataId; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GlobalDataRequest; @@ -62,7 +66,7 @@ import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Optional; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Supplier; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.FluentIterable; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.HashBasedTable; @@ -86,7 +90,7 @@ public class StreamingModeExecutionContext extends DataflowExecutionContext<Step private static final Logger LOG = LoggerFactory.getLogger(StreamingModeExecutionContext.class); private final String computationId; - private final Map<TupleTag<?>, Map<BoundedWindow, Object>> sideInputCache; + private final Map<TupleTag<?>, Map<BoundedWindow, SideInput<?>>> sideInputCache; // Per-key cache of active Reader objects in use by this process. private final ImmutableMap<String, String> stateNameMap; private final WindmillStateCache.ForComputation stateCache; @@ -104,7 +108,7 @@ public class StreamingModeExecutionContext extends DataflowExecutionContext<Step private Windmill.WorkItem work; private WindmillComputationKey computationKey; - private StateFetcher stateFetcher; + private SideInputStateFetcher sideInputStateFetcher; private Windmill.WorkItemCommitRequest.Builder outputBuilder; private UnboundedSource.UnboundedReader<?> activeReader; private volatile long backlogBytes; @@ -145,20 +149,20 @@ public void start( @Nullable Instant outputDataWatermark, @Nullable Instant synchronizedProcessingTime, WindmillStateReader stateReader, - StateFetcher stateFetcher, + SideInputStateFetcher sideInputStateFetcher, Windmill.WorkItemCommitRequest.Builder outputBuilder) { this.key = key; this.work = work; this.computationKey = WindmillComputationKey.create(computationId, work.getKey(), work.getShardingKey()); - this.stateFetcher = stateFetcher; + this.sideInputStateFetcher = sideInputStateFetcher; this.outputBuilder = outputBuilder; this.sideInputCache.clear(); clearSinkFullHint(); Instant processingTime = Instant.now(); // Ensure that the processing time is greater than any fired processing time - // timers. Otherwise a trigger could ignore the timer and orphan the window. + // timers. Otherwise, a trigger could ignore the timer and orphan the window. for (Windmill.Timer timer : work.getTimers().getTimersList()) { if (timer.getType() == Windmill.Timer.Type.REALTIME) { Instant inferredFiringTime = @@ -208,42 +212,67 @@ protected SideInputReader getSideInputReaderForViews( return StreamingModeSideInputReader.of(views, this); } + @SuppressWarnings("deprecation") + private <T> TupleTag<?> getInternalTag(PCollectionView<T> view) { + return view.getTagInternal(); + } + /** * Fetches the requested sideInput, and maintains a view of the cache that doesn't remove items * until the active work item is finished. * - * <p>If the side input was not ready, throws {@code IllegalStateException} if the state is - * {@literal CACHED_IN_WORKITEM} or returns null otherwise. - * - * <p>If the side input was ready and null, returns {@literal Optional.absent()}. If the side - * input was ready and non-null returns {@literal Optional.present(...)}. + * <p>If the side input was not cached, throws {@code IllegalStateException} if the state is + * {@literal CACHED_IN_WORK_ITEM} or returns {@link SideInput<T>} which contains {@link + * Optional<T>}. */ - private @Nullable <T> Optional<T> fetchSideInput( + private <T> SideInput<T> fetchSideInput( + PCollectionView<T> view, + BoundedWindow sideInputWindow, + @Nullable String stateFamily, + SideInputState state, + @Nullable Supplier<Closeable> scopedReadStateSupplier) { + TupleTag<?> viewInternalTag = getInternalTag(view); + Map<BoundedWindow, SideInput<?>> tagCache = + sideInputCache.computeIfAbsent(viewInternalTag, k -> new HashMap<>()); + + @SuppressWarnings("unchecked") + Optional<SideInput<T>> cachedSideInput = + Optional.ofNullable((SideInput<T>) tagCache.get(sideInputWindow)); + + if (cachedSideInput.isPresent()) { + return cachedSideInput.get(); + } + + if (state == SideInputState.CACHED_IN_WORK_ITEM) { + throw new IllegalStateException( + "Expected side input to be cached. Tag: " + viewInternalTag.getId()); + } + + return fetchSideInputFromWindmill( + view, + sideInputWindow, + Preconditions.checkNotNull(stateFamily), + state, + Preconditions.checkNotNull(scopedReadStateSupplier), + tagCache); + } + + private <T> SideInput<T> fetchSideInputFromWindmill( PCollectionView<T> view, BoundedWindow sideInputWindow, String stateFamily, - StateFetcher.SideInputState state, - Supplier<Closeable> scopedReadStateSupplier) { - Map<BoundedWindow, Object> tagCache = - sideInputCache.computeIfAbsent(view.getTagInternal(), k -> new HashMap<>()); + SideInputState state, + Supplier<Closeable> scopedReadStateSupplier, + Map<BoundedWindow, SideInput<?>> tagCache) { + SideInput<T> fetched = + sideInputStateFetcher.fetchSideInput( + view, sideInputWindow, stateFamily, state, scopedReadStateSupplier); - if (tagCache.containsKey(sideInputWindow)) { - @SuppressWarnings("unchecked") - T typed = (T) tagCache.get(sideInputWindow); - return Optional.fromNullable(typed); - } else { - if (state == StateFetcher.SideInputState.CACHED_IN_WORKITEM) { - throw new IllegalStateException( - "Expected side input to be cached. Tag: " + view.getTagInternal().getId()); - } - Optional<T> fetched = - stateFetcher.fetchSideInput( - view, sideInputWindow, stateFamily, state, scopedReadStateSupplier); - if (fetched != null) { - tagCache.put(sideInputWindow, fetched.orNull()); - } - return fetched; + if (fetched.isReady()) { + tagCache.put(sideInputWindow, fetched); } + + return fetched; } public Iterable<Windmill.GlobalDataId> getSideInputNotifications() { @@ -378,8 +407,7 @@ String getStateFamily(NameContext nameContext) { interface StreamingModeStepContext { - boolean issueSideInputFetch( - PCollectionView<?> view, BoundedWindow w, StateFetcher.SideInputState s); + boolean issueSideInputFetch(PCollectionView<?> view, BoundedWindow w, SideInputState s); void addBlockingSideInput(Windmill.GlobalDataRequest blocked); @@ -412,10 +440,7 @@ public static class StreamingModeExecutionState // 2. The reporting thread calls extractUpdate which reads the current sum *AND* sets it to 0. private final AtomicLong totalMillisInState = new AtomicLong(); - // The worker that created this state. Used to report lulls back to the worker. - @SuppressWarnings("unused") // Affects a public api - private final StreamingDataflowWorker worker; - + @SuppressWarnings("unused") public StreamingModeExecutionState( NameContext nameContext, String stateName, @@ -424,7 +449,6 @@ public StreamingModeExecutionState( StreamingDataflowWorker worker) { // TODO: Take in the requesting step name and side input index for streaming. super(nameContext, stateName, null, null, metricsContainer, profileScope); - this.worker = worker; } /** @@ -513,8 +537,7 @@ public UserStepContext(StreamingModeExecutionContext.StepContext wrapped) { } @Override - public boolean issueSideInputFetch( - PCollectionView<?> view, BoundedWindow w, StateFetcher.SideInputState s) { + public boolean issueSideInputFetch(PCollectionView<?> view, BoundedWindow w, SideInputState s) { return wrapped.issueSideInputFetch(view, w, s); } @@ -609,9 +632,10 @@ public <T> T get(PCollectionView<T> view, BoundedWindow window) { view, window, null /* unused stateFamily */, - StateFetcher.SideInputState.CACHED_IN_WORKITEM, + SideInputState.CACHED_IN_WORK_ITEM, null /* unused readStateSupplier */) - .orNull(); + .value() + .orElse(null); } @Override @@ -883,10 +907,10 @@ public <T, W extends BoundedWindow> void writePCollectionViewData( /** Fetch the given side input asynchronously and return true if it is present. */ @Override public boolean issueSideInputFetch( - PCollectionView<?> view, BoundedWindow mainInputWindow, StateFetcher.SideInputState state) { + PCollectionView<?> view, BoundedWindow mainInputWindow, SideInputState state) { BoundedWindow sideInputWindow = view.getWindowMappingFn().getSideInputWindow(mainInputWindow); return fetchSideInput(view, sideInputWindow, stateFamily, state, scopedReadStateSupplier) - != null; + .isReady(); } /** Note that there is data on the current key that is blocked on the given side input. */ diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputFetcher.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputFetcher.java index 2b551acd2d8c..4f585e1c01b6 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputFetcher.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputFetcher.java @@ -33,6 +33,7 @@ import org.apache.beam.runners.core.TimerInternals.TimerData; import org.apache.beam.runners.core.TimerInternals.TimerDataCoder; import org.apache.beam.runners.core.TimerInternals.TimerDataCoderV2; +import org.apache.beam.runners.dataflow.worker.streaming.sideinput.SideInputState; import org.apache.beam.runners.dataflow.worker.windmill.Windmill; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GlobalDataRequest; import org.apache.beam.sdk.coders.AtomicCoder; @@ -135,8 +136,7 @@ public Set<W> getReadyWindows() { W window = entry.getKey(); boolean allSideInputsCached = true; for (PCollectionView<?> view : sideInputViews.values()) { - if (!stepContext.issueSideInputFetch( - view, window, StateFetcher.SideInputState.KNOWN_READY)) { + if (!stepContext.issueSideInputFetch(view, window, SideInputState.KNOWN_READY)) { Windmill.GlobalDataRequest request = buildGlobalDataRequest(view, window); stepContext.addBlockingSideInput(request); windowBlockedSet.add(request); @@ -192,7 +192,7 @@ public boolean storeIfBlocked(WindowedValue<InputT> elem) { Set<Windmill.GlobalDataRequest> blocked = blockedMap().get(window); if (blocked == null) { for (PCollectionView<?> view : sideInputViews.values()) { - if (!stepContext.issueSideInputFetch(view, window, StateFetcher.SideInputState.UNKNOWN)) { + if (!stepContext.issueSideInputFetch(view, window, SideInputState.UNKNOWN)) { if (blocked == null) { blocked = new HashSet<>(); blockedMap().put(window, blocked); @@ -222,7 +222,7 @@ public boolean storeIfBlocked(TimerData timer) { boolean blocked = false; for (PCollectionView<?> view : sideInputViews.values()) { - if (!stepContext.issueSideInputFetch(view, window, StateFetcher.SideInputState.UNKNOWN)) { + if (!stepContext.issueSideInputFetch(view, window, SideInputState.UNKNOWN)) { blocked = true; } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/sideinput/SideInput.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/sideinput/SideInput.java new file mode 100644 index 000000000000..04eecadc1e5c --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/sideinput/SideInput.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.dataflow.worker.streaming.sideinput; + +import com.google.auto.value.AutoValue; +import java.util.Optional; +import javax.annotation.Nullable; + +/** + * Entry in the side input cache that stores the value and the encoded size of the value. + * + * <p>Can be in 1 of 3 states: + * + * <ul> + * <li>Ready with a <T> value. + * <li>Ready with no value, represented as {@link Optional<T>} + * <li>Not ready. + * </ul> + */ +@AutoValue +public abstract class SideInput<T> { + static <T> SideInput<T> ready(@Nullable T value, int encodedSize) { + return new AutoValue_SideInput<>(true, Optional.ofNullable(value), encodedSize); + } + + static <T> SideInput<T> notReady() { + return new AutoValue_SideInput<>(false, Optional.empty(), 0); + } + + public abstract boolean isReady(); + + public abstract Optional<T> value(); + + public abstract int size(); +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/sideinput/SideInputCache.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/sideinput/SideInputCache.java new file mode 100644 index 000000000000..721c477435ef --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/sideinput/SideInputCache.java @@ -0,0 +1,113 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.dataflow.worker.streaming.sideinput; + +import com.google.auto.value.AutoValue; +import com.google.errorprone.annotations.CheckReturnValue; +import java.util.Optional; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.Cache; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.CacheBuilder; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.Weigher; + +/** + * Wrapper around {@code Cache<SideInputId, SideInput>} that mostly delegates to the underlying + * cache, but adds threadsafe functionality to invalidate and load entries that are not ready. + * + * @implNote Returned values are explicitly cast, because the {@link #sideInputCache} holds wildcard + * types of all objects. + */ +@CheckReturnValue +final class SideInputCache { + + private static final long MAXIMUM_CACHE_WEIGHT = 100000000; /* 100 MB */ + private static final long CACHE_ENTRY_EXPIRY_MINUTES = 1L; + + private final Cache<Key<?>, SideInput<?>> sideInputCache; + + SideInputCache(Cache<Key<?>, SideInput<?>> sideInputCache) { + this.sideInputCache = sideInputCache; + } + + static SideInputCache create() { + return new SideInputCache( + CacheBuilder.newBuilder() + .maximumWeight(MAXIMUM_CACHE_WEIGHT) + .expireAfterWrite(CACHE_ENTRY_EXPIRY_MINUTES, TimeUnit.MINUTES) + .weigher((Weigher<Key<?>, SideInput<?>>) (id, entry) -> entry.size()) + .build()); + } + + synchronized <T> SideInput<T> invalidateThenLoadNewEntry( + Key<T> key, Callable<SideInput<T>> cacheLoaderFn) throws ExecutionException { + // Invalidate the existing not-ready entry. This must be done atomically + // so that another thread doesn't replace the entry with a ready entry, which + // would then be deleted here. + Optional<SideInput<T>> newEntry = getIfPresentUnchecked(key); + if (newEntry.isPresent() && !newEntry.get().isReady()) { + sideInputCache.invalidate(key); + } + + return getUnchecked(key, cacheLoaderFn); + } + + <T> Optional<SideInput<T>> get(Key<T> key) { + return getIfPresentUnchecked(key); + } + + <T> SideInput<T> getOrLoad(Key<T> key, Callable<SideInput<T>> cacheLoaderFn) + throws ExecutionException { + return getUnchecked(key, cacheLoaderFn); + } + + @SuppressWarnings({ + "unchecked" // cacheLoaderFn loads SideInput<T>, and key is of type T, so value for Key is + // always SideInput<T>. + }) + private <T> SideInput<T> getUnchecked(Key<T> key, Callable<SideInput<T>> cacheLoaderFn) + throws ExecutionException { + return (SideInput<T>) sideInputCache.get(key, cacheLoaderFn); + } + + @SuppressWarnings({ + "unchecked" // cacheLoaderFn loads SideInput<T>, and key is of type T, so value for Key is + // always SideInput<T>. + }) + private <T> Optional<SideInput<T>> getIfPresentUnchecked(Key<T> key) { + return Optional.ofNullable((SideInput<T>) sideInputCache.getIfPresent(key)); + } + + @AutoValue + abstract static class Key<T> { + static <T> Key<T> create( + TupleTag<?> tag, BoundedWindow window, TypeDescriptor<T> typeDescriptor) { + return new AutoValue_SideInputCache_Key<>(tag, window, typeDescriptor); + } + + abstract TupleTag<?> tag(); + + abstract BoundedWindow window(); + + abstract TypeDescriptor<T> typeDescriptor(); + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/sideinput/SideInputState.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/sideinput/SideInputState.java new file mode 100644 index 000000000000..d7af10d29e1f --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/sideinput/SideInputState.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.dataflow.worker.streaming.sideinput; + +/** Indicates the caller's knowledge of whether a particular side input has been computed. */ +public enum SideInputState { + CACHED_IN_WORK_ITEM, + KNOWN_READY, + UNKNOWN +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/sideinput/SideInputStateFetcher.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/sideinput/SideInputStateFetcher.java new file mode 100644 index 000000000000..aa61c4219353 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/sideinput/SideInputStateFetcher.java @@ -0,0 +1,245 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.dataflow.worker.streaming.sideinput; + +import static org.apache.beam.sdk.transforms.Materializations.ITERABLE_MATERIALIZATION_URN; +import static org.apache.beam.sdk.transforms.Materializations.MULTIMAP_MATERIALIZATION_URN; +import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; + +import java.io.Closeable; +import java.io.IOException; +import java.util.Collections; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.Callable; +import javax.annotation.concurrent.NotThreadSafe; +import org.apache.beam.runners.core.InMemoryMultimapSideInputView; +import org.apache.beam.runners.dataflow.worker.MetricTrackingWindmillServerStub; +import org.apache.beam.runners.dataflow.worker.WindmillTimeUtils; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GlobalData; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.IterableCoder; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.transforms.Materializations.IterableView; +import org.apache.beam.sdk.transforms.Materializations.MultimapView; +import org.apache.beam.sdk.transforms.ViewFn; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.util.ByteStringOutputStream; +import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowingStrategy; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Supplier; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Class responsible for fetching state from the windmill server. */ +@NotThreadSafe +public class SideInputStateFetcher { + private static final Logger LOG = LoggerFactory.getLogger(SideInputStateFetcher.class); + + private static final Set<String> SUPPORTED_MATERIALIZATIONS = + ImmutableSet.of(ITERABLE_MATERIALIZATION_URN, MULTIMAP_MATERIALIZATION_URN); + + private final SideInputCache sideInputCache; + private final MetricTrackingWindmillServerStub server; + private long bytesRead = 0L; + + public SideInputStateFetcher(MetricTrackingWindmillServerStub server) { + this(server, SideInputCache.create()); + } + + SideInputStateFetcher(MetricTrackingWindmillServerStub server, SideInputCache sideInputCache) { + this.server = server; + this.sideInputCache = sideInputCache; + } + + private static <T> Iterable<?> decodeRawData(PCollectionView<T> view, GlobalData data) + throws IOException { + return !data.getData().isEmpty() + ? IterableCoder.of(getCoder(view)).decode(data.getData().newInput()) + : Collections.emptyList(); + } + + @SuppressWarnings({ + "deprecation" // Required as part of the SideInputCacheKey, and not exposed. + }) + private static <T> TupleTag<?> getInternalTag(PCollectionView<T> view) { + return view.getTagInternal(); + } + + @SuppressWarnings("deprecation") + private static <T> ViewFn<?, T> getViewFn(PCollectionView<T> view) { + return view.getViewFn(); + } + + @SuppressWarnings({ + "deprecation" // The view's internal coder is required to decode the raw data. + }) + private static <T> Coder<?> getCoder(PCollectionView<T> view) { + return view.getCoderInternal(); + } + + /** Returns a view of the underlying cache that keeps track of bytes read separately. */ + public SideInputStateFetcher byteTrackingView() { + return new SideInputStateFetcher(server, sideInputCache); + } + + public long getBytesRead() { + return bytesRead; + } + + /** + * Fetch the given side input, storing it in a process-level cache. + * + * <p>If state is KNOWN_READY, attempt to fetch the data regardless of whether a not-ready entry + * was cached. + */ + public <T> SideInput<T> fetchSideInput( + PCollectionView<T> view, + BoundedWindow sideWindow, + String stateFamily, + SideInputState state, + Supplier<Closeable> scopedReadStateSupplier) { + Callable<SideInput<T>> loadSideInputFromWindmill = + () -> loadSideInputFromWindmill(view, sideWindow, stateFamily, scopedReadStateSupplier); + SideInputCache.Key<T> sideInputCacheKey = + SideInputCache.Key.create( + getInternalTag(view), sideWindow, getViewFn(view).getTypeDescriptor()); + + try { + if (state == SideInputState.KNOWN_READY) { + Optional<SideInput<T>> existingCacheEntry = sideInputCache.get(sideInputCacheKey); + if (!existingCacheEntry.isPresent()) { + return sideInputCache.getOrLoad(sideInputCacheKey, loadSideInputFromWindmill); + } + + if (!existingCacheEntry.get().isReady()) { + return sideInputCache.invalidateThenLoadNewEntry( + sideInputCacheKey, loadSideInputFromWindmill); + } + + return existingCacheEntry.get(); + } + + return sideInputCache.getOrLoad(sideInputCacheKey, loadSideInputFromWindmill); + } catch (Exception e) { + LOG.error("Fetch failed: ", e); + throw new RuntimeException("Exception while fetching side input: ", e); + } + } + + private <T, SideWindowT extends BoundedWindow> GlobalData fetchGlobalDataFromWindmill( + PCollectionView<T> view, + SideWindowT sideWindow, + String stateFamily, + Supplier<Closeable> scopedReadStateSupplier) + throws IOException { + @SuppressWarnings({ + "deprecation", // Internal windowStrategy is required to fetch side input data from Windmill. + "unchecked" // Internal windowing strategy matches WindowingStrategy<?, SideWindowT>. + }) + WindowingStrategy<?, SideWindowT> sideWindowStrategy = + (WindowingStrategy<?, SideWindowT>) view.getWindowingStrategyInternal(); + + Coder<SideWindowT> windowCoder = sideWindowStrategy.getWindowFn().windowCoder(); + + ByteStringOutputStream windowStream = new ByteStringOutputStream(); + windowCoder.encode(sideWindow, windowStream); + + Windmill.GlobalDataRequest request = + Windmill.GlobalDataRequest.newBuilder() + .setDataId( + Windmill.GlobalDataId.newBuilder() + .setTag(getInternalTag(view).getId()) + .setVersion(windowStream.toByteString()) + .build()) + .setStateFamily(stateFamily) + .setExistenceWatermarkDeadline( + WindmillTimeUtils.harnessToWindmillTimestamp( + sideWindowStrategy.getTrigger().getWatermarkThatGuaranteesFiring(sideWindow))) + .build(); + + try (Closeable ignored = scopedReadStateSupplier.get()) { + return server.getSideInputData(request); + } + } + + private <T> SideInput<T> loadSideInputFromWindmill( + PCollectionView<T> view, + BoundedWindow sideWindow, + String stateFamily, + Supplier<Closeable> scopedReadStateSupplier) + throws IOException { + validateViewMaterialization(view); + GlobalData data = + fetchGlobalDataFromWindmill(view, sideWindow, stateFamily, scopedReadStateSupplier); + bytesRead += data.getSerializedSize(); + return data.getIsReady() ? createSideInputCacheEntry(view, data) : SideInput.notReady(); + } + + private <T> void validateViewMaterialization(PCollectionView<T> view) { + String materializationUrn = getViewFn(view).getMaterialization().getUrn(); + checkState( + SUPPORTED_MATERIALIZATIONS.contains(materializationUrn), + "Only materialization's of type %s supported, received %s", + SUPPORTED_MATERIALIZATIONS, + materializationUrn); + } + + private <T> SideInput<T> createSideInputCacheEntry(PCollectionView<T> view, GlobalData data) + throws IOException { + Iterable<?> rawData = decodeRawData(view, data); + switch (getViewFn(view).getMaterialization().getUrn()) { + case ITERABLE_MATERIALIZATION_URN: + { + @SuppressWarnings({ + "unchecked", // ITERABLE_MATERIALIZATION_URN has ViewFn<IterableView, T>. + "rawtypes" // TODO(https://github.com/apache/beam/issues/20447) + }) + ViewFn<IterableView, T> viewFn = (ViewFn<IterableView, T>) getViewFn(view); + return SideInput.ready(viewFn.apply(() -> rawData), data.getData().size()); + } + case MULTIMAP_MATERIALIZATION_URN: + { + @SuppressWarnings({ + "unchecked", // MULTIMAP_MATERIALIZATION_URN has ViewFn<MultimapView, T>. + "rawtypes" // TODO(https://github.com/apache/beam/issues/20447) + }) + ViewFn<MultimapView, T> viewFn = (ViewFn<MultimapView, T>) getViewFn(view); + Coder<?> keyCoder = ((KvCoder<?, ?>) getCoder(view)).getKeyCoder(); + + @SuppressWarnings({ + "unchecked", // Safe since multimap rawData is of type Iterable<KV<K, V>> + "rawtypes", // TODO(https://github.com/apache/beam/issues/20447) + }) + T multimapSideInputValue = + viewFn.apply( + InMemoryMultimapSideInputView.fromIterable(keyCoder, (Iterable) rawData)); + return SideInput.ready(multimapSideInputValue, data.getData().size()); + } + default: + { + throw new IllegalStateException( + "Unknown side input materialization format requested: " + + getViewFn(view).getMaterialization().getUrn()); + } + } + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java index c4042e37c3bd..fdec36d688e9 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java @@ -281,11 +281,7 @@ private static CounterUpdate getCounter(Iterable<CounterUpdate> counters, String } static Work createMockWork(long workToken) { - return Work.create( - Windmill.WorkItem.newBuilder().setKey(ByteString.EMPTY).setWorkToken(workToken).build(), - Instant::now, - Collections.emptyList(), - work -> {}); + return createMockWork(workToken, work -> {}); } static Work createMockWork(long workToken, Consumer<Work> processWorkFn) { diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContextTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContextTest.java index 6620dbdaab79..9991520d593b 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContextTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContextTest.java @@ -55,6 +55,7 @@ import org.apache.beam.runners.dataflow.worker.counters.NameContext; import org.apache.beam.runners.dataflow.worker.profiler.ScopedProfiler.NoopProfileScope; import org.apache.beam.runners.dataflow.worker.profiler.ScopedProfiler.ProfileScope; +import org.apache.beam.runners.dataflow.worker.streaming.sideinput.SideInputStateFetcher; import org.apache.beam.runners.dataflow.worker.windmill.Windmill; import org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateCache; import org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateReader; @@ -83,10 +84,10 @@ @RunWith(JUnit4.class) public class StreamingModeExecutionContextTest { - @Mock private StateFetcher stateFetcher; + @Mock private SideInputStateFetcher sideInputStateFetcher; @Mock private WindmillStateReader stateReader; - private StreamingModeExecutionStateRegistry executionStateRegistry = + private final StreamingModeExecutionStateRegistry executionStateRegistry = new StreamingModeExecutionStateRegistry(null); private StreamingModeExecutionContext executionContext; DataflowWorkerHarnessOptions options; @@ -133,7 +134,7 @@ public void testTimerInternalsSetTimer() { null, // output watermark null, // synchronized processing time stateReader, - stateFetcher, + sideInputStateFetcher, outputBuilder); TimerInternals timerInternals = stepContext.timerInternals(); @@ -183,7 +184,7 @@ public void testTimerInternalsProcessingTimeSkew() { null, // output watermark null, // synchronized processing time stateReader, - stateFetcher, + sideInputStateFetcher, outputBuilder); TimerInternals timerInternals = stepContext.timerInternals(); assertTrue(timerTimestamp.isBefore(timerInternals.currentProcessingTime())); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunnerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunnerTest.java index 05e0ff417615..3c121ab27f76 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunnerTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunnerTest.java @@ -39,7 +39,7 @@ import org.apache.beam.runners.core.SideInputReader; import org.apache.beam.runners.core.StateInternals; import org.apache.beam.runners.core.StateNamespaces; -import org.apache.beam.runners.dataflow.worker.StateFetcher.SideInputState; +import org.apache.beam.runners.dataflow.worker.streaming.sideinput.SideInputState; import org.apache.beam.runners.dataflow.worker.util.ListOutputManager; import org.apache.beam.runners.dataflow.worker.windmill.Windmill; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GlobalDataRequest; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputFetcherTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputFetcherTest.java index 9ce462be3211..a7196613fbb1 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputFetcherTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputFetcherTest.java @@ -31,7 +31,7 @@ import org.apache.beam.runners.core.StateInternals; import org.apache.beam.runners.core.StateNamespaces; import org.apache.beam.runners.core.TimerInternals.TimerData; -import org.apache.beam.runners.dataflow.worker.StateFetcher.SideInputState; +import org.apache.beam.runners.dataflow.worker.streaming.sideinput.SideInputState; import org.apache.beam.runners.dataflow.worker.windmill.Windmill; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.state.BagState; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StateFetcherTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/sideinput/SideInputStateFetcherTest.java similarity index 67% rename from runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StateFetcherTest.java rename to runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/sideinput/SideInputStateFetcherTest.java index 13d8a9bd3ffb..daf814618791 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StateFetcherTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/sideinput/SideInputStateFetcherTest.java @@ -15,11 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.dataflow.worker; +package org.apache.beam.runners.dataflow.worker.streaming.sideinput; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.contains; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNull; import static org.mockito.Matchers.any; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -27,10 +29,10 @@ import static org.mockito.Mockito.when; import java.io.Closeable; -import java.util.Arrays; +import java.util.Collections; import java.util.List; import java.util.concurrent.TimeUnit; -import org.apache.beam.runners.dataflow.worker.StateFetcher.SideInputState; +import org.apache.beam.runners.dataflow.worker.MetricTrackingWindmillServerStub; import org.apache.beam.runners.dataflow.worker.windmill.Windmill; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.ListCoder; @@ -56,14 +58,16 @@ import org.mockito.Mock; import org.mockito.MockitoAnnotations; -/** Unit tests for {@link StateFetcher}. */ +/** Unit tests for {@link SideInputStateFetcher}. */ +// TODO: Add tests with different encoded windows to verify version is correctly plumbed. +@SuppressWarnings("deprecation") @RunWith(JUnit4.class) -public class StateFetcherTest { +public class SideInputStateFetcherTest { private static final String STATE_FAMILY = "state"; - @Mock MetricTrackingWindmillServerStub server; + @Mock private MetricTrackingWindmillServerStub server; - @Mock Supplier<Closeable> readStateSupplier; + @Mock private Supplier<Closeable> readStateSupplier; @Before public void setUp() { @@ -72,10 +76,11 @@ public void setUp() { @Test public void testFetchGlobalDataBasic() throws Exception { - StateFetcher fetcher = new StateFetcher(server); + SideInputStateFetcher fetcher = new SideInputStateFetcher(server); ByteStringOutputStream stream = new ByteStringOutputStream(); - ListCoder.of(StringUtf8Coder.of()).encode(Arrays.asList("data"), stream, Coder.Context.OUTER); + ListCoder.of(StringUtf8Coder.of()) + .encode(Collections.singletonList("data"), stream, Coder.Context.OUTER); ByteString encodedIterable = stream.toByteString(); PCollectionView<String> view = @@ -87,17 +92,29 @@ public void testFetchGlobalDataBasic() throws Exception { // then the data is already cached. when(server.getSideInputData(any(Windmill.GlobalDataRequest.class))) .thenReturn( - buildGlobalDataResponse(tag, ByteString.EMPTY, false, null), - buildGlobalDataResponse(tag, ByteString.EMPTY, true, encodedIterable)); + buildGlobalDataResponse(tag, false, null), + buildGlobalDataResponse(tag, true, encodedIterable)); + + assertFalse( + fetcher + .fetchSideInput( + view, + GlobalWindow.INSTANCE, + STATE_FAMILY, + SideInputState.UNKNOWN, + readStateSupplier) + .isReady()); + + assertFalse( + fetcher + .fetchSideInput( + view, + GlobalWindow.INSTANCE, + STATE_FAMILY, + SideInputState.UNKNOWN, + readStateSupplier) + .isReady()); - assertEquals( - null, - fetcher.fetchSideInput( - view, GlobalWindow.INSTANCE, STATE_FAMILY, SideInputState.UNKNOWN, readStateSupplier)); - assertEquals( - null, - fetcher.fetchSideInput( - view, GlobalWindow.INSTANCE, STATE_FAMILY, SideInputState.UNKNOWN, readStateSupplier)); assertEquals( "data", fetcher @@ -107,7 +124,8 @@ public void testFetchGlobalDataBasic() throws Exception { STATE_FAMILY, SideInputState.KNOWN_READY, readStateSupplier) - .orNull()); + .value() + .orElse(null)); assertEquals( "data", fetcher @@ -117,18 +135,20 @@ public void testFetchGlobalDataBasic() throws Exception { STATE_FAMILY, SideInputState.KNOWN_READY, readStateSupplier) - .orNull()); + .value() + .orElse(null)); - verify(server, times(2)).getSideInputData(buildGlobalDataRequest(tag, ByteString.EMPTY)); + verify(server, times(2)).getSideInputData(buildGlobalDataRequest(tag)); verifyNoMoreInteractions(server); } @Test public void testFetchGlobalDataNull() throws Exception { - StateFetcher fetcher = new StateFetcher(server); + SideInputStateFetcher fetcher = new SideInputStateFetcher(server); ByteStringOutputStream stream = new ByteStringOutputStream(); - ListCoder.of(VoidCoder.of()).encode(Arrays.asList((Void) null), stream, Coder.Context.OUTER); + ListCoder.of(VoidCoder.of()) + .encode(Collections.singletonList(null), stream, Coder.Context.OUTER); ByteString encodedIterable = stream.toByteString(); PCollectionView<Void> view = @@ -140,19 +160,28 @@ public void testFetchGlobalDataNull() throws Exception { // then the data is already cached. when(server.getSideInputData(any(Windmill.GlobalDataRequest.class))) .thenReturn( - buildGlobalDataResponse(tag, ByteString.EMPTY, false, null), - buildGlobalDataResponse(tag, ByteString.EMPTY, true, encodedIterable)); + buildGlobalDataResponse(tag, false, null), + buildGlobalDataResponse(tag, true, encodedIterable)); - assertEquals( - null, - fetcher.fetchSideInput( - view, GlobalWindow.INSTANCE, STATE_FAMILY, SideInputState.UNKNOWN, readStateSupplier)); - assertEquals( - null, - fetcher.fetchSideInput( - view, GlobalWindow.INSTANCE, STATE_FAMILY, SideInputState.UNKNOWN, readStateSupplier)); - assertEquals( - null, + assertFalse( + fetcher + .fetchSideInput( + view, + GlobalWindow.INSTANCE, + STATE_FAMILY, + SideInputState.UNKNOWN, + readStateSupplier) + .isReady()); + assertFalse( + fetcher + .fetchSideInput( + view, + GlobalWindow.INSTANCE, + STATE_FAMILY, + SideInputState.UNKNOWN, + readStateSupplier) + .isReady()); + assertNull( fetcher .fetchSideInput( view, @@ -160,9 +189,9 @@ public void testFetchGlobalDataNull() throws Exception { STATE_FAMILY, SideInputState.KNOWN_READY, readStateSupplier) - .orNull()); - assertEquals( - null, + .value() + .orElse(null)); + assertNull( fetcher .fetchSideInput( view, @@ -170,9 +199,10 @@ public void testFetchGlobalDataNull() throws Exception { STATE_FAMILY, SideInputState.KNOWN_READY, readStateSupplier) - .orNull()); + .value() + .orElse(null)); - verify(server, times(2)).getSideInputData(buildGlobalDataRequest(tag, ByteString.EMPTY)); + verify(server, times(2)).getSideInputData(buildGlobalDataRequest(tag)); verifyNoMoreInteractions(server); } @@ -181,15 +211,14 @@ public void testFetchGlobalDataCacheOverflow() throws Exception { Coder<List<String>> coder = ListCoder.of(StringUtf8Coder.of()); ByteStringOutputStream stream = new ByteStringOutputStream(); - coder.encode(Arrays.asList("data1"), stream, Coder.Context.OUTER); + coder.encode(Collections.singletonList("data1"), stream, Coder.Context.OUTER); ByteString encodedIterable1 = stream.toByteStringAndReset(); - coder.encode(Arrays.asList("data2"), stream, Coder.Context.OUTER); + coder.encode(Collections.singletonList("data2"), stream, Coder.Context.OUTER); ByteString encodedIterable2 = stream.toByteString(); - Cache<StateFetcher.SideInputId, StateFetcher.SideInputCacheEntry> cache = - CacheBuilder.newBuilder().build(); + Cache<SideInputCache.Key<?>, SideInput<?>> cache = CacheBuilder.newBuilder().build(); - StateFetcher fetcher = new StateFetcher(server, cache); + SideInputStateFetcher fetcher = new SideInputStateFetcher(server, new SideInputCache(cache)); PCollectionView<String> view1 = TestPipeline.create().apply(Create.empty(StringUtf8Coder.of())).apply(View.asSingleton()); @@ -204,9 +233,9 @@ public void testFetchGlobalDataCacheOverflow() throws Exception { // then view 1 again twice. when(server.getSideInputData(any(Windmill.GlobalDataRequest.class))) .thenReturn( - buildGlobalDataResponse(tag1, ByteString.EMPTY, true, encodedIterable1), - buildGlobalDataResponse(tag2, ByteString.EMPTY, true, encodedIterable2), - buildGlobalDataResponse(tag1, ByteString.EMPTY, true, encodedIterable1)); + buildGlobalDataResponse(tag1, true, encodedIterable1), + buildGlobalDataResponse(tag2, true, encodedIterable2), + buildGlobalDataResponse(tag1, true, encodedIterable1)); assertEquals( "data1", @@ -217,7 +246,8 @@ public void testFetchGlobalDataCacheOverflow() throws Exception { STATE_FAMILY, SideInputState.UNKNOWN, readStateSupplier) - .orNull()); + .value() + .orElse(null)); assertEquals( "data2", fetcher @@ -227,7 +257,8 @@ public void testFetchGlobalDataCacheOverflow() throws Exception { STATE_FAMILY, SideInputState.UNKNOWN, readStateSupplier) - .orNull()); + .value() + .orElse(null)); cache.invalidateAll(); assertEquals( "data1", @@ -238,7 +269,8 @@ public void testFetchGlobalDataCacheOverflow() throws Exception { STATE_FAMILY, SideInputState.UNKNOWN, readStateSupplier) - .orNull()); + .value() + .orElse(null)); assertEquals( "data1", fetcher @@ -248,7 +280,8 @@ public void testFetchGlobalDataCacheOverflow() throws Exception { STATE_FAMILY, SideInputState.UNKNOWN, readStateSupplier) - .orNull()); + .value() + .orElse(null)); ArgumentCaptor<Windmill.GlobalDataRequest> captor = ArgumentCaptor.forClass(Windmill.GlobalDataRequest.class); @@ -259,14 +292,14 @@ public void testFetchGlobalDataCacheOverflow() throws Exception { assertThat( captor.getAllValues(), contains( - buildGlobalDataRequest(tag1, ByteString.EMPTY), - buildGlobalDataRequest(tag2, ByteString.EMPTY), - buildGlobalDataRequest(tag1, ByteString.EMPTY))); + buildGlobalDataRequest(tag1), + buildGlobalDataRequest(tag2), + buildGlobalDataRequest(tag1))); } @Test public void testEmptyFetchGlobalData() throws Exception { - StateFetcher fetcher = new StateFetcher(server); + SideInputStateFetcher fetcher = new SideInputStateFetcher(server); ByteString encodedIterable = ByteString.EMPTY; @@ -280,7 +313,7 @@ public void testEmptyFetchGlobalData() throws Exception { // Test three calls in a row. First, data is not ready, then data is ready, // then the data is already cached. when(server.getSideInputData(any(Windmill.GlobalDataRequest.class))) - .thenReturn(buildGlobalDataResponse(tag, ByteString.EMPTY, true, encodedIterable)); + .thenReturn(buildGlobalDataResponse(tag, true, encodedIterable)); assertEquals( 0L, @@ -292,17 +325,22 @@ public void testEmptyFetchGlobalData() throws Exception { STATE_FAMILY, SideInputState.UNKNOWN, readStateSupplier) - .orNull()); + .value() + .orElse(null)); - verify(server).getSideInputData(buildGlobalDataRequest(tag, ByteString.EMPTY)); + verify(server).getSideInputData(buildGlobalDataRequest(tag)); verifyNoMoreInteractions(server); } - private Windmill.GlobalData buildGlobalDataResponse( - String tag, ByteString version, boolean isReady, ByteString data) { + private static Windmill.GlobalData buildGlobalDataResponse( + String tag, boolean isReady, ByteString data) { Windmill.GlobalData.Builder builder = Windmill.GlobalData.newBuilder() - .setDataId(Windmill.GlobalDataId.newBuilder().setTag(tag).setVersion(version).build()); + .setDataId( + Windmill.GlobalDataId.newBuilder() + .setTag(tag) + .setVersion(ByteString.EMPTY) + .build()); if (isReady) { builder.setIsReady(true).setData(data); @@ -312,7 +350,7 @@ private Windmill.GlobalData buildGlobalDataResponse( return builder.build(); } - private Windmill.GlobalDataRequest buildGlobalDataRequest(String tag, ByteString version) { + private static Windmill.GlobalDataRequest buildGlobalDataRequest(String tag, ByteString version) { Windmill.GlobalDataId id = Windmill.GlobalDataId.newBuilder().setTag(tag).setVersion(version).build(); @@ -323,4 +361,8 @@ private Windmill.GlobalDataRequest buildGlobalDataRequest(String tag, ByteString TimeUnit.MILLISECONDS.toMicros(GlobalWindow.INSTANCE.maxTimestamp().getMillis())) .build(); } + + private static Windmill.GlobalDataRequest buildGlobalDataRequest(String tag) { + return buildGlobalDataRequest(tag, ByteString.EMPTY); + } } From 641b95df45d138cff32f5a43f00d174b7cd1eef1 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 20 Oct 2023 09:16:14 -0400 Subject: [PATCH 207/435] Bump google.golang.org/api from 0.147.0 to 0.148.0 in /sdks (#29086) Bumps [google.golang.org/api](https://github.com/googleapis/google-api-go-client) from 0.147.0 to 0.148.0. - [Release notes](https://github.com/googleapis/google-api-go-client/releases) - [Changelog](https://github.com/googleapis/google-api-go-client/blob/main/CHANGES.md) - [Commits](https://github.com/googleapis/google-api-go-client/compare/v0.147.0...v0.148.0) --- updated-dependencies: - dependency-name: google.golang.org/api dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] <support@github.com> Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 4 ++-- sdks/go.sum | 8 ++++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 686a399bd80f..b990ee33ea09 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -57,7 +57,7 @@ require ( golang.org/x/sync v0.4.0 golang.org/x/sys v0.13.0 golang.org/x/text v0.13.0 - google.golang.org/api v0.147.0 + google.golang.org/api v0.148.0 google.golang.org/genproto v0.0.0-20231002182017-d307bd883b97 google.golang.org/grpc v1.59.0 google.golang.org/protobuf v1.31.0 @@ -171,5 +171,5 @@ require ( golang.org/x/xerrors v0.0.0-20220907171357-04be3eba64a2 // indirect google.golang.org/appengine v1.6.7 // indirect google.golang.org/genproto/googleapis/api v0.0.0-20231002182017-d307bd883b97 // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20231009173412-8bfb1ae86b6c // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20231012201019-e917dd12ba7a // indirect ) diff --git a/sdks/go.sum b/sdks/go.sum index 6428fbd82c9f..d4dd5e1ae8e4 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -671,8 +671,8 @@ google.golang.org/api v0.14.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsb google.golang.org/api v0.15.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI= google.golang.org/api v0.17.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= google.golang.org/api v0.18.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= -google.golang.org/api v0.147.0 h1:Can3FaQo9LlVqxJCodNmeZW/ib3/qKAY3rFeXiHo5gc= -google.golang.org/api v0.147.0/go.mod h1:pQ/9j83DcmPd/5C9e2nFOdjjNkDZ1G+zkbK2uvdkJMs= +google.golang.org/api v0.148.0 h1:HBq4TZlN4/1pNcu0geJZ/Q50vIwIXT532UIMYoo0vOs= +google.golang.org/api v0.148.0/go.mod h1:8/TBgwaKjfqTdacOJrOv2+2Q6fBDU1uHKK06oGSkxzU= google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/appengine v1.5.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= @@ -702,8 +702,8 @@ google.golang.org/genproto v0.0.0-20231002182017-d307bd883b97 h1:SeZZZx0cP0fqUyA google.golang.org/genproto v0.0.0-20231002182017-d307bd883b97/go.mod h1:t1VqOqqvce95G3hIDCT5FeO3YUc6Q4Oe24L/+rNMxRk= google.golang.org/genproto/googleapis/api v0.0.0-20231002182017-d307bd883b97 h1:W18sezcAYs+3tDZX4F80yctqa12jcP1PUS2gQu1zTPU= google.golang.org/genproto/googleapis/api v0.0.0-20231002182017-d307bd883b97/go.mod h1:iargEX0SFPm3xcfMI0d1domjg0ZF4Aa0p2awqyxhvF0= -google.golang.org/genproto/googleapis/rpc v0.0.0-20231009173412-8bfb1ae86b6c h1:jHkCUWkseRf+W+edG5hMzr/Uh1xkDREY4caybAq4dpY= -google.golang.org/genproto/googleapis/rpc v0.0.0-20231009173412-8bfb1ae86b6c/go.mod h1:4cYg8o5yUbm77w8ZX00LhMVNl/YVBFJRYWDc0uYWMs0= +google.golang.org/genproto/googleapis/rpc v0.0.0-20231012201019-e917dd12ba7a h1:a2MQQVoTo96JC9PMGtGBymLp7+/RzpFc2yX/9WfFg1c= +google.golang.org/genproto/googleapis/rpc v0.0.0-20231012201019-e917dd12ba7a/go.mod h1:4cYg8o5yUbm77w8ZX00LhMVNl/YVBFJRYWDc0uYWMs0= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.20.1/go.mod h1:10oTOabMzJvdu6/UiuZezV6QK5dSlG84ov/aaiqXj38= google.golang.org/grpc v1.21.1/go.mod h1:oYelfM1adQP15Ek0mdvEgi9Df8B9CZIaU1084ijfRaM= From 7610d46f7456ccd7c311aed4c21105fcce27b7b5 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw <robertwb@gmail.com> Date: Fri, 20 Oct 2023 06:17:11 -0700 Subject: [PATCH 208/435] Remove warning in dataframes io. (#29084) --- sdks/python/apache_beam/dataframe/io.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/dataframe/io.py b/sdks/python/apache_beam/dataframe/io.py index eb1b1b5ec15c..fedc40c60714 100644 --- a/sdks/python/apache_beam/dataframe/io.py +++ b/sdks/python/apache_beam/dataframe/io.py @@ -281,8 +281,9 @@ def expand(self, root): if not self.binary: handle = TextIOWrapper(handle) if self.incremental: - sample = next( - self.reader(handle, *self.args, **dict(self.kwargs, chunksize=100))) + with self.reader(handle, *self.args, **dict(self.kwargs, + chunksize=100)) as stream: + sample = next(stream) else: sample = self.reader(handle, *self.args, **self.kwargs) From 0a11c04ee1d1e9aa781d09f96c19aa31ce5e9762 Mon Sep 17 00:00:00 2001 From: Vlado Djerek <rage.bubblegum@gmail.com> Date: Fri, 20 Oct 2023 16:28:14 +0200 Subject: [PATCH 209/435] beam_performance_singlestoreio (#28986) * beam_performance_singlestoreio * Update .github/workflows/beam_PerformanceTests_SingleStoreIO.yml Add coment about the test-argumets-action Co-authored-by: Andrey Devyatkin <andrey.9evyatkin@gmail.com> * remove fail-fast and remove comment line in config * config cleanup * influx creds and move to main pool --------- Co-authored-by: Andrey Devyatkin <andrey.9evyatkin@gmail.com> --- .../beam_PerformanceTests_SingleStoreIO.yml | 121 ++++++++++++++++++ .../config_PerformanceTests_SingleStoreIO.txt | 23 ++++ 2 files changed, 144 insertions(+) create mode 100644 .github/workflows/beam_PerformanceTests_SingleStoreIO.yml create mode 100644 .github/workflows/performance-tests-job-configs/config_PerformanceTests_SingleStoreIO.txt diff --git a/.github/workflows/beam_PerformanceTests_SingleStoreIO.yml b/.github/workflows/beam_PerformanceTests_SingleStoreIO.yml new file mode 100644 index 000000000000..c95434c2ede5 --- /dev/null +++ b/.github/workflows/beam_PerformanceTests_SingleStoreIO.yml @@ -0,0 +1,121 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +name: Performance Test Java SingleStoreIO + +on: + issue_comment: + types: [created] + schedule: + - cron: '0 */12 * * *' + workflow_dispatch: +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} + +jobs: + beam_PerformanceTests_SingleStoreIO: + name: ${{matrix.job_name}} (${{matrix.job_phrase}}) + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 100 + strategy: + matrix: + job_name: [beam_PerformanceTests_SingleStoreIO] + job_phrase: [Run Java SingleStoreIO Performance Test] + if: | + github.event_name == 'push' || + github.event_name == 'schedule' || + github.event_name == 'workflow_dispatch' || + github.event.comment.body == 'Run Java SingleStoreIO Performance Test' + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Authenticate on GCP + id: auth + uses: google-github-actions/auth@v1 + with: + credentials_json: ${{ secrets.GCP_SA_KEY }} + project_id: ${{ secrets.GCP_PROJECT_ID }} + - name: Set k8s access + uses: ./.github/actions/setup-k8s-access + with: + cluster_name: io-datastores + k8s_namespace: ${{ matrix.job_name }}-${{ github.run_id }} + - name: Install Singlestore operator + run: | + kubectl apply -f ${{github.workspace}}/.test-infra/kubernetes/singlestore/sdb-rbac.yaml + kubectl apply -f ${{github.workspace}}/.test-infra/kubernetes/singlestore/sdb-cluster-crd.yaml + kubectl apply -f ${{github.workspace}}/.test-infra/kubernetes/singlestore/sdb-operator.yaml + kubectl wait --for=condition=Ready pod -l name=sdb-operator --timeout=300s + - name: Install Singlestore cluster + id: install_singlestore + run: | + kubectl apply -f ${{github.workspace}}/.test-infra/kubernetes/singlestore/sdb-cluster.yaml + kubectl wait --for=jsonpath='{.status.phase}'=Running memsqlclusters.memsql.com --all --timeout=300s + kubectl wait svc/svc-sdb-cluster-ddl --for=jsonpath='{.status.loadBalancer.ingress[0].ip}' --timeout=300s + loadbalancer_IP=$(kubectl get svc svc-sdb-cluster-ddl -o jsonpath='{.status.loadBalancer.ingress[0].ip}') + echo lb_ip=$loadbalancer_IP >> $GITHUB_OUTPUT + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: performance + test-language: java + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/performance-tests-job-configs/config_PerformanceTests_SingleStoreIO.txt + arguments: | + --singleStoreServerName=${{ steps.install_singlestore.outputs.lb_ip }} + # The env variable is created and populated in the test-arguments-action as "<github.job>_test_arguments_<argument_file_paths_index>" + - name: Run Java SingleStore IO Performance Test + id: run_java_singlestore_io_performance_test + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:io:singlestore:integrationTest + arguments: | + --tests org.apache.beam.sdk.io.singlestore.SingleStoreIOPerformanceIT \ + --info \ + -DintegrationTestRunner=dataflow \ + -DintegrationTestPipelineOptions='[${{ env.beam_PerformanceTests_SingleStoreIO_test_arguments_1 }}]' \ No newline at end of file diff --git a/.github/workflows/performance-tests-job-configs/config_PerformanceTests_SingleStoreIO.txt b/.github/workflows/performance-tests-job-configs/config_PerformanceTests_SingleStoreIO.txt new file mode 100644 index 000000000000..909042791c9f --- /dev/null +++ b/.github/workflows/performance-tests-job-configs/config_PerformanceTests_SingleStoreIO.txt @@ -0,0 +1,23 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--tempRoot=gs://temp-storage-for-perf-tests +--runner=DataflowRunner +--singleStoreUsername=admin +--singleStorePassword=secretpass +--singleStorePort=3306 +--numberOfRecords=5000000 +--influxMeasurement=singlestoreioit_results From e7a6405800a83dd16437b8b1b372e020e010a042 Mon Sep 17 00:00:00 2001 From: Yi Hu <yathu@google.com> Date: Fri, 20 Oct 2023 11:39:04 -0400 Subject: [PATCH 210/435] Setup Java 21 container (#28918) * Setup Java 21 container (#28833) * Add Java21 container * fix spotless * Update website * Fix jamm * align configs in recent change * incremental fixes * add more comments for when jpms test enabled --- ...a_CoGBK_Dataflow_V2_Batch_JavaVersions.yml | 8 ++-- ...GBK_Dataflow_V2_Streaming_JavaVersions.yml | 8 ++-- ...ests_Java_GBK_Dataflow_V2_Batch_Java11.yml | 14 +++---- ...ests_Java_GBK_Dataflow_V2_Batch_Java17.yml | 14 +++---- ..._Java_GBK_Dataflow_V2_Streaming_Java11.yml | 14 +++---- ..._Java_GBK_Dataflow_V2_Streaming_Java17.yml | 14 +++---- ...a_ParDo_Dataflow_V2_Batch_JavaVersions.yml | 8 ++-- ...rDo_Dataflow_V2_Streaming_JavaVersions.yml | 8 ++-- ..._PostCommit_Java_Examples_Dataflow_ARM.yml | 2 +- ...tCommit_Java_Examples_Dataflow_V2_Java.yml | 2 +- ...m_PostCommit_Java_Jpms_Dataflow_Java17.yml | 2 +- ...eam_PostCommit_Java_Jpms_Direct_Java17.yml | 2 +- ...stCommit_Java_Nexmark_Dataflow_V2_Java.yml | 2 +- ...eam_PostCommit_TransformService_Direct.yml | 2 +- ...reCommit_Java_Examples_Dataflow_Java11.yml | 2 +- ...reCommit_Java_Examples_Dataflow_Java17.yml | 2 +- .../workflows/beam_PreCommit_SQL_Java11.yml | 2 +- .../workflows/beam_PreCommit_SQL_Java17.yml | 2 +- .test-infra/jenkins/JavaTestProperties.groovy | 7 +++- .test-infra/jenkins/NexmarkBuilder.groovy | 4 +- ..._LoadTests_CoGBK_Dataflow_V2_Java11.groovy | 2 +- ..._LoadTests_CoGBK_Dataflow_V2_Java17.groovy | 2 +- ...ob_LoadTests_GBK_Dataflow_V2_Java11.groovy | 2 +- ...ob_LoadTests_GBK_Dataflow_V2_Java17.groovy | 2 +- ..._LoadTests_ParDo_Dataflow_V2_Java11.groovy | 2 +- ..._LoadTests_ParDo_Dataflow_V2_Java17.groovy | 2 +- ...it_Java_Examples_Dataflow_V2_Java11.groovy | 2 +- ...it_Java_Examples_Dataflow_V2_Java17.groovy | 2 +- ...ostCommit_Java_Jpms_Dataflow_Java17.groovy | 2 +- ..._PostCommit_Java_Jpms_Direct_Java17.groovy | 2 +- ..._PostCommit_TransformService_Direct.groovy | 2 +- build.gradle.kts | 14 +++---- .../beam/gradle/BeamModulePlugin.groovy | 37 ++++++++++++++++++- .../core/construction/Environments.java | 6 ++- .../core/construction/EnvironmentsTest.java | 6 ++- .../arm/build.gradle | 6 +-- .../google-cloud-dataflow-java/build.gradle | 6 +-- .../beam/runners/dataflow/DataflowRunner.java | 7 ++-- runners/spark/spark_runner.gradle | 5 ++- sdks/java/container/Dockerfile | 3 +- sdks/java/container/agent/build.gradle | 14 ++++++- sdks/java/container/common.gradle | 28 +++++++++++--- sdks/java/container/java21/build.gradle | 30 +++++++++++++++ sdks/java/container/java21/option-jamm.json | 12 ++++++ sdks/java/testing/jpms-tests/build.gradle | 25 +++++++------ sdks/java/testing/test-utils/build.gradle | 29 +++++---------- .../jvmverification/JvmVerification.java | 16 +++++++- settings.gradle.kts | 1 + .../www/site/content/en/roadmap/java-sdk.md | 6 +-- 49 files changed, 255 insertions(+), 137 deletions(-) create mode 100644 sdks/java/container/java21/build.gradle create mode 100644 sdks/java/container/java21/option-jamm.json diff --git a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml index fea2eb18ff14..327e007d5437 100644 --- a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml +++ b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml @@ -95,7 +95,7 @@ jobs: with: gradle-command: :sdks:java:testing:load-tests:run arguments: | - -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -PtestJavaVersion=${{ matrix.java_version }} \ -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ @@ -105,7 +105,7 @@ jobs: with: gradle-command: :sdks:java:testing:load-tests:run arguments: | - -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -PtestJavaVersion=${{ matrix.java_version }} \ -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ @@ -115,7 +115,7 @@ jobs: with: gradle-command: :sdks:java:testing:load-tests:run arguments: | - -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -PtestJavaVersion=${{ matrix.java_version }} \ -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ @@ -125,7 +125,7 @@ jobs: with: gradle-command: :sdks:java:testing:load-tests:run arguments: | - -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -PtestJavaVersion=${{ matrix.java_version }} \ -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ diff --git a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml index b81ea53ab5fd..0abe99ce83d2 100644 --- a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml +++ b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml @@ -95,7 +95,7 @@ jobs: with: gradle-command: :sdks:java:testing:load-tests:run arguments: | - -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -PtestJavaVersion=${{ matrix.java_version }} \ -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ @@ -105,7 +105,7 @@ jobs: with: gradle-command: :sdks:java:testing:load-tests:run arguments: | - -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -PtestJavaVersion=${{ matrix.java_version }} \ -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ @@ -115,7 +115,7 @@ jobs: with: gradle-command: :sdks:java:testing:load-tests:run arguments: | - -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -PtestJavaVersion=${{ matrix.java_version }} \ -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ @@ -125,7 +125,7 @@ jobs: with: gradle-command: :sdks:java:testing:load-tests:run arguments: | - -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -PtestJavaVersion=${{ matrix.java_version }} \ -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml index 3a8d47ccb642..39490d59fc22 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml @@ -95,7 +95,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava11 \ + -PtestJavaVersion=11 \ -Pjava11Home=$JAVA_HOME_11_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11_test_arguments_1 }}' \ - name: run Load test 2GB of 100B records @@ -106,7 +106,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava11 \ + -PtestJavaVersion=11 \ -Pjava11Home=$JAVA_HOME_11_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11_test_arguments_2 }}' \ - name: run Load test 2GB of 100kB records @@ -117,7 +117,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava11 \ + -PtestJavaVersion=11 \ -Pjava11Home=$JAVA_HOME_11_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11_test_arguments_3 }}' \ - name: run Load test fanout 4 times with 2GB 10-byte records total @@ -128,7 +128,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava11 \ + -PtestJavaVersion=11 \ -Pjava11Home=$JAVA_HOME_11_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11_test_arguments_4 }}' \ - name: run Load test fanout 8 times with 2GB 10-byte records total @@ -139,7 +139,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava11 \ + -PtestJavaVersion=11 \ -Pjava11Home=$JAVA_HOME_11_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11_test_arguments_5 }}' \ - name: run Load test reiterate 4 times 10kB values @@ -150,7 +150,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava11 \ + -PtestJavaVersion=11 \ -Pjava11Home=$JAVA_HOME_11_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11_test_arguments_6 }}' \ - name: run Load test reiterate 4 times 2MB values @@ -161,6 +161,6 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava11 \ + -PtestJavaVersion=11 \ -Pjava11Home=$JAVA_HOME_11_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11_test_arguments_7 }}' \ No newline at end of file diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml index 414b2e0d660c..bb4e039ec8c2 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml @@ -97,7 +97,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava17 \ + -PtestJavaVersion=17 \ -Pjava17Home=$JAVA_HOME_17_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17_test_arguments_1 }}' \ - name: run Load test 2GB of 100B records @@ -108,7 +108,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava17 \ + -PtestJavaVersion=17 \ -Pjava17Home=$JAVA_HOME_17_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17_test_arguments_2 }}' \ - name: run Load test 2GB of 100kB records @@ -119,7 +119,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava17 \ + -PtestJavaVersion=17 \ -Pjava17Home=$JAVA_HOME_17_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17_test_arguments_3 }}' \ - name: run Load test fanout 4 times with 2GB 10-byte records total @@ -130,7 +130,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava17 \ + -PtestJavaVersion=17 \ -Pjava17Home=$JAVA_HOME_17_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17_test_arguments_4 }}' \ - name: run Load test fanout 8 times with 2GB 10-byte records total @@ -141,7 +141,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava17 \ + -PtestJavaVersion=17 \ -Pjava17Home=$JAVA_HOME_17_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17_test_arguments_5 }}' \ - name: run Load test reiterate 4 times 10kB values @@ -152,7 +152,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava17 \ + -PtestJavaVersion=17 \ -Pjava17Home=$JAVA_HOME_17_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17_test_arguments_6 }}' \ - name: run Load test reiterate 4 times 2MB values @@ -163,6 +163,6 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava17 \ + -PtestJavaVersion=17 \ -Pjava17Home=$JAVA_HOME_17_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17_test_arguments_7 }}' \ No newline at end of file diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml index bfd8dfe09d9c..60cf635457de 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml @@ -95,7 +95,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava11 \ + -PtestJavaVersion=11 \ -Pjava11Home=$JAVA_HOME_11_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11_test_arguments_1 }}' \ - name: run Load test 2GB of 100B records @@ -106,7 +106,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava11 \ + -PtestJavaVersion=11 \ -Pjava11Home=$JAVA_HOME_11_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11_test_arguments_2 }}' \ - name: run Load test 2GB of 100kB records @@ -117,7 +117,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava11 \ + -PtestJavaVersion=11 \ -Pjava11Home=$JAVA_HOME_11_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11_test_arguments_3 }}' \ - name: run Load test fanout 4 times with 2GB 10-byte records total @@ -128,7 +128,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava11 \ + -PtestJavaVersion=11 \ -Pjava11Home=$JAVA_HOME_11_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11_test_arguments_4 }}' \ - name: run Load test fanout 8 times with 2GB 10-byte records total @@ -139,7 +139,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava11 \ + -PtestJavaVersion=11 \ -Pjava11Home=$JAVA_HOME_11_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11_test_arguments_5 }}' \ - name: run Load test reiterate 4 times 10kB values @@ -150,7 +150,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava11 \ + -PtestJavaVersion=11 \ -Pjava11Home=$JAVA_HOME_11_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11_test_arguments_6 }}' \ - name: run Load test reiterate 4 times 2MB values @@ -161,6 +161,6 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava11 \ + -PtestJavaVersion=11 \ -Pjava11Home=$JAVA_HOME_11_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11_test_arguments_7 }}' \ No newline at end of file diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml index 22df3fe88d26..24d218c152ed 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml @@ -97,7 +97,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava17 \ + -PtestJavaVersion=17 \ -Pjava17Home=$JAVA_HOME_17_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17_test_arguments_1 }}' \ - name: run Load test 2GB of 100B records @@ -108,7 +108,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava17 \ + -PtestJavaVersion=17 \ -Pjava17Home=$JAVA_HOME_17_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17_test_arguments_2 }}' \ - name: run Load test 2GB of 100kB records @@ -119,7 +119,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava17 \ + -PtestJavaVersion=17 \ -Pjava17Home=$JAVA_HOME_17_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17_test_arguments_3 }}' \ - name: run Load test fanout 4 times with 2GB 10-byte records total @@ -130,7 +130,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava17 \ + -PtestJavaVersion=17 \ -Pjava17Home=$JAVA_HOME_17_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17_test_arguments_4 }}' \ - name: run Load test fanout 8 times with 2GB 10-byte records total @@ -141,7 +141,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava17 \ + -PtestJavaVersion=17 \ -Pjava17Home=$JAVA_HOME_17_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17_test_arguments_5 }}' \ - name: run Load test reiterate 4 times 10kB values @@ -152,7 +152,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava17 \ + -PtestJavaVersion=17 \ -Pjava17Home=$JAVA_HOME_17_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17_test_arguments_6 }}' \ - name: run Load test reiterate 4 times 2MB values @@ -163,6 +163,6 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava17 \ + -PtestJavaVersion=17 \ -Pjava17Home=$JAVA_HOME_17_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17_test_arguments_7 }}' \ No newline at end of file diff --git a/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions.yml b/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions.yml index a283310ff1d0..66b5b5a8837f 100644 --- a/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions.yml +++ b/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions.yml @@ -98,7 +98,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.ParDoLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -PtestJavaVersion=${{ matrix.java_version }} \ -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions_test_arguments_1 }} --appName=load_tests_Java${{ matrix.java_version }}_Dataflow_V2_batch_ParDo_1' \ - name: run ParDo Dataflow V2 Batch Java Load Test 2 (200 times) @@ -109,7 +109,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.ParDoLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -PtestJavaVersion=${{ matrix.java_version }} \ -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions_test_arguments_2 }} --appName=load_tests_Java${{ matrix.java_version }}_Dataflow_V2_batch_ParDo_2' \ - name: run ParDo Dataflow V2 Batch Java Load Test 3 (10 counters) @@ -120,7 +120,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.ParDoLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -PtestJavaVersion=${{ matrix.java_version }} \ -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions_test_arguments_3 }} --appName=load_tests_Java${{ matrix.java_version }}_Dataflow_V2_batch_ParDo_3' \ - name: run ParDo Dataflow V2 Batch Java Load Test 4 (100 counters) @@ -131,6 +131,6 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.ParDoLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -PtestJavaVersion=${{ matrix.java_version }} \ -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions_test_arguments_4 }} --appName=load_tests_Java${{ matrix.java_version }}_Dataflow_V2_batch_ParDo_4' \ No newline at end of file diff --git a/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions.yml b/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions.yml index 45ea975d1580..bd8987d4ee0d 100644 --- a/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions.yml +++ b/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions.yml @@ -98,7 +98,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.ParDoLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -PtestJavaVersion=${{ matrix.java_version }} \ -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions_test_arguments_1 }} --appName=load_tests_Java${{ matrix.java_version }}_Dataflow_V2_streaming_ParDo_1' \ - name: run ParDo Dataflow V2 Streaming Java Load Test 2 (200 times) @@ -109,7 +109,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.ParDoLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -PtestJavaVersion=${{ matrix.java_version }} \ -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions_test_arguments_2 }} --appName=load_tests_Java${{ matrix.java_version }}_Dataflow_V2_streaming_ParDo_2' \ - name: run ParDo Dataflow V2 Streaming Java Load Test 3 (10 counters) @@ -120,7 +120,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.ParDoLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -PtestJavaVersion=${{ matrix.java_version }} \ -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions_test_arguments_3 }} --appName=load_tests_Java${{ matrix.java_version }}_Dataflow_V2_streaming_ParDo_3' \ - name: run ParDo Dataflow V2 Streaming Java Load Test 4 (100 counters) @@ -131,6 +131,6 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.ParDoLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -PtestJavaVersion=${{ matrix.java_version }} \ -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions_test_arguments_4 }} --appName=load_tests_Java${{ matrix.java_version }}_Dataflow_V2_streaming_ParDo_4' \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml index c401c27922fb..09d90d5df1a8 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml @@ -111,7 +111,7 @@ jobs: gradle-command: :runners:google-cloud-dataflow-java:arm:examplesJavaRunnerV2IntegrationTestARM max-workers: 12 arguments: | - -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -PtestJavaVersion=${{ matrix.java_version }} \ -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ -Pcontainer-architecture-list=arm64,amd64 \ -Ppush-containers \ diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml index 13506595fe1a..411cc65944fb 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml @@ -86,7 +86,7 @@ jobs: -PdisableSpotlessCheck=true \ -PdisableCheckStyle=true \ -PskipCheckerFramework \ - -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -PtestJavaVersion=${{ matrix.java_version }} \ -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml index 69b5fc23e07a..ff27bae8c527 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml @@ -80,7 +80,7 @@ jobs: gradle-command: :sdks:java:testing:jpms-tests:dataflowRunnerIntegrationTest arguments: -PskipCheckerFramework - -PcompileAndRunTestsWithJava17 + -PtestJavaVersion=17 -Pjava17Home=$JAVA_HOME_17_X64 - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml index 946e34675385..748a74d2ec15 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml @@ -80,7 +80,7 @@ jobs: gradle-command: :sdks:java:testing:jpms-tests:directRunnerIntegrationTest arguments: -PskipCheckerFramework - -PcompileAndRunTestsWithJava17 + -PtestJavaVersion=17 -Pjava17Home=$JAVA_HOME_17_X64 - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml index 2bd36200b217..076c030d91ea 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml @@ -107,7 +107,7 @@ jobs: with: gradle-command: :sdks:java:testing:nexmark:run arguments: | - -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -PtestJavaVersion=${{ matrix.java_version }} \ -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ -Pnexmark.runner.version=V2 \ -Pnexmark.runner=:runners:google-cloud-dataflow-java \ diff --git a/.github/workflows/beam_PostCommit_TransformService_Direct.yml b/.github/workflows/beam_PostCommit_TransformService_Direct.yml index 916cd3b34e75..4c9bb3aa9302 100644 --- a/.github/workflows/beam_PostCommit_TransformService_Direct.yml +++ b/.github/workflows/beam_PostCommit_TransformService_Direct.yml @@ -84,7 +84,7 @@ jobs: with: gradle-command: :sdks:python:test-suites:direct:xlang:transformServicePythonUsingJava arguments: | - -PcompileAndRunTestsWithJava11 \ + -PtestJavaVersion=11 \ -Pjava11Home=$JAVA_HOME_11_X64 \ -PuseWheelDistribution \ -PpythonVersion=${{ matrix.python_version }} \ diff --git a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml index 8b584bc64122..b12590c01603 100644 --- a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml +++ b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml @@ -116,7 +116,7 @@ jobs: -PdisableSpotlessCheck=true \ -PdisableCheckStyle=true \ -PskipCheckerFramework \ - -PcompileAndRunTestsWithJava11 \ + -PtestJavaVersion=11 \ -Pjava11Home=$JAVA_HOME_11_X64 \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 diff --git a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml index e4e4b62a2e8a..d4a55aa3a824 100644 --- a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml +++ b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml @@ -116,7 +116,7 @@ jobs: arguments: | -PdisableSpotlessCheck=true \ -PdisableCheckStyle=true \ - -PcompileAndRunTestsWithJava17 \ + -PtestJavaVersion=17 \ -PskipCheckerFramework \ -Pjava17Home=$JAVA_HOME_17_X64 \ max-workers: 12 diff --git a/.github/workflows/beam_PreCommit_SQL_Java11.yml b/.github/workflows/beam_PreCommit_SQL_Java11.yml index c48ad0dd8063..4acbecc8d0fc 100644 --- a/.github/workflows/beam_PreCommit_SQL_Java11.yml +++ b/.github/workflows/beam_PreCommit_SQL_Java11.yml @@ -103,7 +103,7 @@ jobs: arguments: | -PdisableSpotlessCheck=true \ -PdisableCheckStyle=true \ - -PcompileAndRunTestsWithJava11 \ + -PtestJavaVersion=11 \ -PskipCheckerFramework \ -Pjava11Home=$JAVA_HOME_11_X64 \ - name: Archive JUnit Test Results diff --git a/.github/workflows/beam_PreCommit_SQL_Java17.yml b/.github/workflows/beam_PreCommit_SQL_Java17.yml index 1ea901947a98..a5899fff4837 100644 --- a/.github/workflows/beam_PreCommit_SQL_Java17.yml +++ b/.github/workflows/beam_PreCommit_SQL_Java17.yml @@ -101,7 +101,7 @@ jobs: arguments: | -PdisableSpotlessCheck=true \ -PdisableCheckStyle=true \ - -PcompileAndRunTestsWithJava17 \ + -PtestJavaVersion=17 \ -PskipCheckerFramework \ -Pjava17Home=$JAVA_HOME_17_X64 \ - name: Archive JUnit Test Results diff --git a/.test-infra/jenkins/JavaTestProperties.groovy b/.test-infra/jenkins/JavaTestProperties.groovy index ce7446a6e71d..5403cee5cf9a 100644 --- a/.test-infra/jenkins/JavaTestProperties.groovy +++ b/.test-infra/jenkins/JavaTestProperties.groovy @@ -17,5 +17,10 @@ */ class JavaTestProperties { - final static List<String> SUPPORTED_CONTAINER_TASKS = ['java8', 'java11', 'java17'] + final static List<String> SUPPORTED_CONTAINER_TASKS = [ + 'java8', + 'java11', + 'java17', + 'java21' + ] } diff --git a/.test-infra/jenkins/NexmarkBuilder.groovy b/.test-infra/jenkins/NexmarkBuilder.groovy index 044b0cbb9561..69fa3dcc4277 100644 --- a/.test-infra/jenkins/NexmarkBuilder.groovy +++ b/.test-infra/jenkins/NexmarkBuilder.groovy @@ -145,7 +145,7 @@ class NexmarkBuilder { rootBuildScriptDir(commonJobProperties.checkoutDir) tasks(':sdks:java:testing:nexmark:run') commonJobProperties.setGradleSwitches(delegate) - switches("-PcompileAndRunTestsWithJava11") + switches("-PtestJavaVersion=11") switches("-Pjava11Home=${commonJobProperties.JAVA_11_HOME}") switches("-Pnexmark.runner=${runner.getDependencyBySDK(sdk)}") switches("-Pnexmark.args=\"${parseOptions(options)}\"") @@ -168,7 +168,7 @@ class NexmarkBuilder { rootBuildScriptDir(commonJobProperties.checkoutDir) tasks(':sdks:java:testing:nexmark:run') commonJobProperties.setGradleSwitches(delegate) - switches("-PcompileAndRunTestsWithJava17") + switches("-PtestJavaVersion=17") switches("-Pjava17Home=${commonJobProperties.JAVA_17_HOME}") switches("-Pnexmark.runner=${runner.getDependencyBySDK(sdk)}") switches("-Pnexmark.args=\"${parseOptions(options)}\"") diff --git a/.test-infra/jenkins/job_LoadTests_CoGBK_Dataflow_V2_Java11.groovy b/.test-infra/jenkins/job_LoadTests_CoGBK_Dataflow_V2_Java11.groovy index fc7f39d28a0d..55501db4429d 100644 --- a/.test-infra/jenkins/job_LoadTests_CoGBK_Dataflow_V2_Java11.groovy +++ b/.test-infra/jenkins/job_LoadTests_CoGBK_Dataflow_V2_Java11.groovy @@ -188,7 +188,7 @@ def loadTestConfigurations = { mode, isStreaming -> def final JOB_SPECIFIC_SWITCHES = [ '-Prunner.version="V2"', - '-PcompileAndRunTestsWithJava11', + '-PtestJavaVersion=11', "-Pjava11Home=${commonJobProperties.JAVA_11_HOME}" ] diff --git a/.test-infra/jenkins/job_LoadTests_CoGBK_Dataflow_V2_Java17.groovy b/.test-infra/jenkins/job_LoadTests_CoGBK_Dataflow_V2_Java17.groovy index ca8c6689ad0f..8fb09fd07448 100644 --- a/.test-infra/jenkins/job_LoadTests_CoGBK_Dataflow_V2_Java17.groovy +++ b/.test-infra/jenkins/job_LoadTests_CoGBK_Dataflow_V2_Java17.groovy @@ -188,7 +188,7 @@ def loadTestConfigurations = { mode, isStreaming -> def final JOB_SPECIFIC_SWITCHES = [ '-Prunner.version="V2"', - '-PcompileAndRunTestsWithJava17', + '-PtestJavaVersion=17', "-Pjava17Home=${commonJobProperties.JAVA_17_HOME}" ] diff --git a/.test-infra/jenkins/job_LoadTests_GBK_Dataflow_V2_Java11.groovy b/.test-infra/jenkins/job_LoadTests_GBK_Dataflow_V2_Java11.groovy index cc2d5d2e5554..2191e448fade 100644 --- a/.test-infra/jenkins/job_LoadTests_GBK_Dataflow_V2_Java11.groovy +++ b/.test-infra/jenkins/job_LoadTests_GBK_Dataflow_V2_Java11.groovy @@ -253,7 +253,7 @@ def loadTestConfigurations = { mode, isStreaming -> def final JOB_SPECIFIC_SWITCHES = [ '-Prunner.version="V2"', - '-PcompileAndRunTestsWithJava11', + '-PtestJavaVersion=11', "-Pjava11Home=${commonJobProperties.JAVA_11_HOME}" ] diff --git a/.test-infra/jenkins/job_LoadTests_GBK_Dataflow_V2_Java17.groovy b/.test-infra/jenkins/job_LoadTests_GBK_Dataflow_V2_Java17.groovy index 7405f9154b83..2520f68f0178 100644 --- a/.test-infra/jenkins/job_LoadTests_GBK_Dataflow_V2_Java17.groovy +++ b/.test-infra/jenkins/job_LoadTests_GBK_Dataflow_V2_Java17.groovy @@ -253,7 +253,7 @@ def loadTestConfigurations = { mode, isStreaming -> def final JOB_SPECIFIC_SWITCHES = [ '-Prunner.version="V2"', - '-PcompileAndRunTestsWithJava17', + '-PtestJavaVersion=17', "-Pjava17Home=${commonJobProperties.JAVA_17_HOME}" ] diff --git a/.test-infra/jenkins/job_LoadTests_ParDo_Dataflow_V2_Java11.groovy b/.test-infra/jenkins/job_LoadTests_ParDo_Dataflow_V2_Java11.groovy index a4535d52e6cf..b7154e840e6a 100644 --- a/.test-infra/jenkins/job_LoadTests_ParDo_Dataflow_V2_Java11.groovy +++ b/.test-infra/jenkins/job_LoadTests_ParDo_Dataflow_V2_Java11.groovy @@ -160,7 +160,7 @@ def commonLoadTestConfig = { jobType, isStreaming -> def final JOB_SPECIFIC_SWITCHES = [ '-Prunner.version="V2"', - '-PcompileAndRunTestsWithJava11', + '-PtestJavaVersion=11', "-Pjava11Home=${commonJobProperties.JAVA_11_HOME}" ] diff --git a/.test-infra/jenkins/job_LoadTests_ParDo_Dataflow_V2_Java17.groovy b/.test-infra/jenkins/job_LoadTests_ParDo_Dataflow_V2_Java17.groovy index f0f2179ebb3b..df6c66e02aed 100644 --- a/.test-infra/jenkins/job_LoadTests_ParDo_Dataflow_V2_Java17.groovy +++ b/.test-infra/jenkins/job_LoadTests_ParDo_Dataflow_V2_Java17.groovy @@ -160,7 +160,7 @@ def commonLoadTestConfig = { jobType, isStreaming -> def final JOB_SPECIFIC_SWITCHES = [ '-Prunner.version="V2"', - '-PcompileAndRunTestsWithJava17', + '-PtestJavaVersion=17', "-Pjava17Home=${commonJobProperties.JAVA_17_HOME}" ] diff --git a/.test-infra/jenkins/job_PostCommit_Java_Examples_Dataflow_V2_Java11.groovy b/.test-infra/jenkins/job_PostCommit_Java_Examples_Dataflow_V2_Java11.groovy index 6687ae0e6f8a..6229f7c48a72 100644 --- a/.test-infra/jenkins/job_PostCommit_Java_Examples_Dataflow_V2_Java11.groovy +++ b/.test-infra/jenkins/job_PostCommit_Java_Examples_Dataflow_V2_Java11.groovy @@ -43,7 +43,7 @@ PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_Examples_Dataflow_V2_ja commonJobProperties.setGradleSwitches(delegate, 3 * Runtime.runtime.availableProcessors()) switches '-PdisableSpotlessCheck=true' switches '-PdisableCheckStyle=true' - switches '-PcompileAndRunTestsWithJava11' + switches '-PtestJavaVersion=11' switches '-PskipCheckerFramework' switches "-Pjava11Home=${commonJobProperties.JAVA_11_HOME}" } diff --git a/.test-infra/jenkins/job_PostCommit_Java_Examples_Dataflow_V2_Java17.groovy b/.test-infra/jenkins/job_PostCommit_Java_Examples_Dataflow_V2_Java17.groovy index b275fe9276d9..7e52a7e09789 100644 --- a/.test-infra/jenkins/job_PostCommit_Java_Examples_Dataflow_V2_Java17.groovy +++ b/.test-infra/jenkins/job_PostCommit_Java_Examples_Dataflow_V2_Java17.groovy @@ -43,7 +43,7 @@ PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_Examples_Dataflow_V2_ja commonJobProperties.setGradleSwitches(delegate, 3 * Runtime.runtime.availableProcessors()) switches '-PdisableSpotlessCheck=true' switches '-PdisableCheckStyle=true' - switches '-PcompileAndRunTestsWithJava17' + switches '-PtestJavaVersion=17' switches '-PskipCheckerFramework' switches "-Pjava17Home=${commonJobProperties.JAVA_17_HOME}" } diff --git a/.test-infra/jenkins/job_PostCommit_Java_Jpms_Dataflow_Java17.groovy b/.test-infra/jenkins/job_PostCommit_Java_Jpms_Dataflow_Java17.groovy index 4e26c164319e..f518985ca7a8 100644 --- a/.test-infra/jenkins/job_PostCommit_Java_Jpms_Dataflow_Java17.groovy +++ b/.test-infra/jenkins/job_PostCommit_Java_Jpms_Dataflow_Java17.groovy @@ -42,7 +42,7 @@ PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_Jpms_Dataflow_Java17', tasks(':sdks:java:testing:jpms-tests:dataflowRunnerIntegrationTest') commonJobProperties.setGradleSwitches(delegate) switches("-PskipCheckerFramework") - switches("-PcompileAndRunTestsWithJava17") + switches("-PtestJavaVersion=17") switches("-Pjava17Home=${commonJobProperties.JAVA_17_HOME}") // Specify maven home on Jenkins, needed by Maven archetype integration tests. switches('-Pmaven_home=/home/jenkins/tools/maven/apache-maven-3.5.4') diff --git a/.test-infra/jenkins/job_PostCommit_Java_Jpms_Direct_Java17.groovy b/.test-infra/jenkins/job_PostCommit_Java_Jpms_Direct_Java17.groovy index f31373ecaada..04c31389ecbe 100644 --- a/.test-infra/jenkins/job_PostCommit_Java_Jpms_Direct_Java17.groovy +++ b/.test-infra/jenkins/job_PostCommit_Java_Jpms_Direct_Java17.groovy @@ -42,7 +42,7 @@ PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_Jpms_Direct_Java17', 'R tasks(':sdks:java:testing:jpms-tests:directRunnerIntegrationTest') commonJobProperties.setGradleSwitches(delegate) switches("-PskipCheckerFramework") - switches("-PcompileAndRunTestsWithJava17") + switches("-PtestJavaVersion=17") switches("-Pjava17Home=${commonJobProperties.JAVA_17_HOME}") // Specify maven home on Jenkins, needed by Maven archetype integration tests. switches('-Pmaven_home=/home/jenkins/tools/maven/apache-maven-3.5.4') diff --git a/.test-infra/jenkins/job_PostCommit_TransformService_Direct.groovy b/.test-infra/jenkins/job_PostCommit_TransformService_Direct.groovy index 0d7f58e71706..03d29069a52c 100644 --- a/.test-infra/jenkins/job_PostCommit_TransformService_Direct.groovy +++ b/.test-infra/jenkins/job_PostCommit_TransformService_Direct.groovy @@ -43,7 +43,7 @@ PostcommitJobBuilder.postCommitJob('beam_PostCommit_TransformService_Direct', rootBuildScriptDir(commonJobProperties.checkoutDir) tasks(':sdks:python:test-suites:direct:xlang:transformServicePythonUsingJava') commonJobProperties.setGradleSwitches(delegate) - switches '-PcompileAndRunTestsWithJava11' + switches '-PtestJavaVersion=11' switches "-Pjava11Home=${commonJobProperties.JAVA_11_HOME}" switches("-PuseWheelDistribution") switches("-PpythonVersion=${pythonVersion}") diff --git a/build.gradle.kts b/build.gradle.kts index ea1b4e6784e3..5bcfbb3ed06e 100644 --- a/build.gradle.kts +++ b/build.gradle.kts @@ -712,14 +712,12 @@ if (project.hasProperty("javaLinkageArtifactIds")) { } } } -if (project.hasProperty("compileAndRunTestsWithJava11")) { - tasks.getByName("javaPreCommitPortabilityApi").dependsOn(":sdks:java:testing:test-utils:verifyJavaVersion") - tasks.getByName("javaExamplesDataflowPrecommit").dependsOn(":sdks:java:testing:test-utils:verifyJavaVersion") - tasks.getByName("sqlPreCommit").dependsOn(":sdks:java:testing:test-utils:verifyJavaVersion") -} else if (project.hasProperty("compileAndRunTestsWithJava17")) { - tasks.getByName("javaPreCommitPortabilityApi").dependsOn(":sdks:java:testing:test-utils:verifyJavaVersion17") - tasks.getByName("javaExamplesDataflowPrecommit").dependsOn(":sdks:java:testing:test-utils:verifyJavaVersion17") - tasks.getByName("sqlPreCommit").dependsOn(":sdks:java:testing:test-utils:verifyJavaVersion17") +if (project.hasProperty("testJavaVersion")) { + var testVer = project.property("testJavaVersion") + + tasks.getByName("javaPreCommitPortabilityApi").dependsOn(":sdks:java:testing:test-utils:verifyJavaVersion$testVer") + tasks.getByName("javaExamplesDataflowPrecommit").dependsOn(":sdks:java:testing:test-utils:verifyJavaVersion$testVer") + tasks.getByName("sqlPreCommit").dependsOn(":sdks:java:testing:test-utils:verifyJavaVersion$testVer") } else { allprojects { tasks.withType(Test::class).configureEach { diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index b13fd00dc249..b32a41f80279 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -946,6 +946,29 @@ class BeamModulePlugin implements Plugin<Project> { ] } + project.ext.setJava21Options = { CompileOptions options -> + def java21Home = project.findProperty("java21Home") + options.fork = true + options.forkOptions.javaHome = java21Home as File + options.compilerArgs += ['-Xlint:-path'] + // Error prone requires some packages to be exported/opened for Java 17+ + // Disabling checks since this property is only used for Jenkins tests + // https://github.com/tbroyer/gradle-errorprone-plugin#jdk-16-support + options.errorprone.errorproneArgs.add("-XepDisableAllChecks") + options.forkOptions.jvmArgs += [ + "-J--add-exports=jdk.compiler/com.sun.tools.javac.api=ALL-UNNAMED", + "-J--add-exports=jdk.compiler/com.sun.tools.javac.file=ALL-UNNAMED", + "-J--add-exports=jdk.compiler/com.sun.tools.javac.main=ALL-UNNAMED", + "-J--add-exports=jdk.compiler/com.sun.tools.javac.model=ALL-UNNAMED", + "-J--add-exports=jdk.compiler/com.sun.tools.javac.parser=ALL-UNNAMED", + "-J--add-exports=jdk.compiler/com.sun.tools.javac.processing=ALL-UNNAMED", + "-J--add-exports=jdk.compiler/com.sun.tools.javac.tree=ALL-UNNAMED", + "-J--add-exports=jdk.compiler/com.sun.tools.javac.util=ALL-UNNAMED", + "-J--add-opens=jdk.compiler/com.sun.tools.javac.code=ALL-UNNAMED", + "-J--add-opens=jdk.compiler/com.sun.tools.javac.comp=ALL-UNNAMED" + ] + } + project.ext.repositories = { maven { name "testPublicationLocal" @@ -1492,7 +1515,7 @@ class BeamModulePlugin implements Plugin<Project> { options.errorprone.errorproneArgs.add("-Xep:Slf4jLoggerShouldBeNonStatic:OFF") } - if (project.hasProperty("compileAndRunTestsWithJava11")) { + if (project.findProperty('testJavaVersion') == "11") { def java11Home = project.findProperty("java11Home") project.tasks.compileTestJava { options.fork = true @@ -1504,7 +1527,7 @@ class BeamModulePlugin implements Plugin<Project> { useJUnit() executable = "${java11Home}/bin/java" } - } else if (project.hasProperty("compileAndRunTestsWithJava17")) { + } else if (project.findProperty('testJavaVersion') == "17") { def java17Home = project.findProperty("java17Home") project.tasks.compileTestJava { setCompileAndRuntimeJavaVersion(options.compilerArgs, '17') @@ -1514,6 +1537,16 @@ class BeamModulePlugin implements Plugin<Project> { useJUnit() executable = "${java17Home}/bin/java" } + } else if (project.findProperty('testJavaVersion') == "21") { + def java21Home = project.findProperty("java21Home") + project.tasks.compileTestJava { + setCompileAndRuntimeJavaVersion(options.compilerArgs, '21') + project.ext.setJava17Options(options) + } + project.tasks.withType(Test).configureEach { + useJUnit() + executable = "${java21Home}/bin/java" + } } if (configuration.shadowClosure) { diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Environments.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Environments.java index 31a555989afd..f531b5be344d 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Environments.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Environments.java @@ -94,7 +94,8 @@ public class Environments { public enum JavaVersion { java8("java", "1.8", 8), java11("java11", "11", 11), - java17("java17", "17", 17); + java17("java17", "17", 17), + java21("java21", "21", 21); // Legacy name, as used in container image private final String legacyName; @@ -119,6 +120,7 @@ public String specification() { return this.specification; } + /** Return the LTS java version given the Java specification version. */ public static JavaVersion forSpecification(String specification) { for (JavaVersion ver : JavaVersion.values()) { if (ver.specification.equals(specification)) { @@ -137,7 +139,7 @@ public static JavaVersion forSpecification(String specification) { } } LOG.warn( - "unsupported Java version: {}, falling back to: {}", + "Unsupported Java version: {}, falling back to: {}", specification, fallback.specification); return fallback; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/EnvironmentsTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/EnvironmentsTest.java index ae429fb1fe6d..b71a654f1031 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/EnvironmentsTest.java +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/EnvironmentsTest.java @@ -291,6 +291,8 @@ public void testLtsJavaVersion() { assertEquals("java11", JavaVersion.java11.legacyName()); assertEquals(JavaVersion.java17, JavaVersion.forSpecification("17")); assertEquals("java17", JavaVersion.java17.legacyName()); + assertEquals(JavaVersion.java21, JavaVersion.forSpecification("21")); + assertEquals("java21", JavaVersion.java21.legacyName()); } @Test @@ -303,7 +305,9 @@ public void testNonLtsJavaVersion() { assertEquals(JavaVersion.java17, JavaVersion.forSpecification("15")); assertEquals(JavaVersion.java17, JavaVersion.forSpecification("16")); assertEquals(JavaVersion.java17, JavaVersion.forSpecification("18")); - assertEquals(JavaVersion.java17, JavaVersion.forSpecification("19")); + assertEquals(JavaVersion.java21, JavaVersion.forSpecification("19")); + assertEquals(JavaVersion.java21, JavaVersion.forSpecification("20")); + assertEquals(JavaVersion.java21, JavaVersion.forSpecification("21")); } @Test(expected = UnsupportedOperationException.class) diff --git a/runners/google-cloud-dataflow-java/arm/build.gradle b/runners/google-cloud-dataflow-java/arm/build.gradle index e79eeedcd828..71cbc7c58e86 100644 --- a/runners/google-cloud-dataflow-java/arm/build.gradle +++ b/runners/google-cloud-dataflow-java/arm/build.gradle @@ -76,10 +76,8 @@ dependencies { } def javaVer = "java8" -if(project.hasProperty('compileAndRunTestsWithJava17')) { - javaVer = "java17" -} else if(project.hasProperty('compileAndRunTestsWithJava11')) { - javaVer = "java11" +if (project.hasProperty('testJavaVersion')) { + javaVer = "java${project.getProperty('testJavaVersion')}" } def dataflowProject = project.findProperty('dataflowProject') ?: 'apache-beam-testing' def dataflowRegion = project.findProperty('dataflowRegion') ?: 'us-central1' diff --git a/runners/google-cloud-dataflow-java/build.gradle b/runners/google-cloud-dataflow-java/build.gradle index adc1f2e09bc4..e4f34687d31c 100644 --- a/runners/google-cloud-dataflow-java/build.gradle +++ b/runners/google-cloud-dataflow-java/build.gradle @@ -279,10 +279,8 @@ def createRunnerV2ValidatesRunnerTest = { Map args -> // task ordering such that the registry doesn't get cleaned up prior to task completion. def buildAndPushDockerJavaContainer = tasks.register("buildAndPushDockerJavaContainer") { def javaVer = "java8" - if(project.hasProperty('compileAndRunTestsWithJava17')) { - javaVer = "java17" - } else if(project.hasProperty('compileAndRunTestsWithJava11')) { - javaVer = "java11" + if(project.hasProperty('testJavaVersion')) { + javaVer = "java${project.getProperty('testJavaVersion')}" } dependsOn ":sdks:java:container:${javaVer}:docker" def defaultDockerImageName = containerImageName( diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java index 26548038a1df..891b4c0454c9 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java @@ -398,10 +398,9 @@ && isServiceEndpoint(dataflowOptions.getDataflowEndpoint())) { // Adding the Java version to the SDK name for user's and support convenience. String agentJavaVer = "(JRE 8 environment)"; - if (Environments.getJavaVersion() == Environments.JavaVersion.java17) { - agentJavaVer = "(JRE 17 environment)"; - } else if (Environments.getJavaVersion() == Environments.JavaVersion.java11) { - agentJavaVer = "(JRE 11 environment)"; + if (Environments.getJavaVersion() != Environments.JavaVersion.java8) { + agentJavaVer = + String.format("(JRE %s environment)", Environments.getJavaVersion().specification()); } DataflowRunnerInfo dataflowRunnerInfo = DataflowRunnerInfo.getDataflowRunnerInfo(); diff --git a/runners/spark/spark_runner.gradle b/runners/spark/spark_runner.gradle index d0dbe453ddfb..74013de6107d 100644 --- a/runners/spark/spark_runner.gradle +++ b/runners/spark/spark_runner.gradle @@ -63,8 +63,9 @@ def sparkTestProperties(overrides = [:]) { def sparkTestJvmArgs() { - // run tests with Java 17 using -PcompileAndRunTestsWithJava17 -Pjava17Home=??? - if (project.hasProperty("compileAndRunTestsWithJava17")) { + // run tests with Java 17 using -PtestJavaVersion=17 -Pjava17Home=??? + if (project.hasProperty('testJavaVersion') && + project.getProperty('testJavaVersion') in ['17', '21']) { return [ "--add-opens=java.base/sun.nio.ch=ALL-UNNAMED", // add-opens below required for Kryo FieldSerializer / SparkRunnerKryoRegistratorTest diff --git a/sdks/java/container/Dockerfile b/sdks/java/container/Dockerfile index e0fa8d4a0a6f..9c266ea132b8 100644 --- a/sdks/java/container/Dockerfile +++ b/sdks/java/container/Dockerfile @@ -15,8 +15,9 @@ # See the License for the specific language governing permissions and # limitations under the License. ############################################################################### +ARG base_image ARG java_version -FROM eclipse-temurin:${java_version} +FROM ${base_image}:${java_version} LABEL Author "Apache Beam <dev@beam.apache.org>" ARG TARGETOS ARG TARGETARCH diff --git a/sdks/java/container/agent/build.gradle b/sdks/java/container/agent/build.gradle index df3780e45446..044d175257cc 100644 --- a/sdks/java/container/agent/build.gradle +++ b/sdks/java/container/agent/build.gradle @@ -20,10 +20,13 @@ plugins { id 'org.apache.beam.module' } +// the order is intended here if (project.hasProperty('java11Home')) { javaVersion = "1.11" } else if (project.hasProperty('java17Home')) { javaVersion = "1.17" +} else if (project.hasProperty('java21Home')) { + javaVersion = "1.21" } applyJavaNature( @@ -42,6 +45,7 @@ jar { } } +// the order is intended here if (project.hasProperty('java11Home')) { def java11Home = project.findProperty('java11Home') project.tasks.withType(JavaCompile) { @@ -53,6 +57,14 @@ if (project.hasProperty('java11Home')) { project.tasks.withType(JavaCompile) { setJava17Options(options) + checkerFramework { + skipCheckerFramework = true + } + } +} else if (project.hasProperty('java21Home')) { + project.tasks.withType(JavaCompile) { + setJava21Options(options) + checkerFramework { skipCheckerFramework = true } @@ -62,7 +74,7 @@ if (project.hasProperty('java11Home')) { // Module classes requires JDK > 8 project.tasks.each { it.onlyIf { - project.hasProperty('java11Home') || project.hasProperty('java17Home') + project.hasProperty('java11Home') || project.hasProperty('java17Home') || project.hasProperty('java21Home') || JavaVersion.VERSION_1_8.compareTo(JavaVersion.current()) < 0 } } diff --git a/sdks/java/container/common.gradle b/sdks/java/container/common.gradle index cc427494ed6e..6be531bc5e80 100644 --- a/sdks/java/container/common.gradle +++ b/sdks/java/container/common.gradle @@ -29,6 +29,7 @@ applyDockerNature() if (!project.hasProperty('imageJavaVersion')) { throw new GradleException('imageJavaVersion project property must be set') } +def javaBaseImage = project.findProperty('javaBaseImage') ?: 'eclipse-temurin' def imageJavaVersion = project.findProperty('imageJavaVersion') description = "Apache Beam :: SDKs :: Java :: Container :: Java ${imageJavaVersion} Container" @@ -71,19 +72,19 @@ task copySdkHarnessLauncher(type: Copy) { } task copyJavaThirdPartyLicenses(type: Copy) { - from("${project(':sdks:java:container').buildDir}/target/third_party_licenses") + from project(':sdks:java:container').layout.buildDirectory.dir('target/third_party_licenses') into "build/target/third_party_licenses" dependsOn ':sdks:java:container:pullLicenses' } task copyGolangLicenses(type: Copy) { - from "${project(':release:go-licenses:java').buildDir}/output" + from project(':release:go-licenses:java').layout.buildDirectory.dir('output') into "build/target/go-licenses" dependsOn ':release:go-licenses:java:createLicenses' } task copyJdkOptions(type: Copy) { - if (imageJavaVersion == "17" || imageJavaVersion == "11") { + if (["11", "17", "21"].contains(imageJavaVersion)) { from "option-jamm.json" } from "java${imageJavaVersion}-security.properties" @@ -96,11 +97,25 @@ task skipPullLicenses(type: Exec) { args "-c", "mkdir -p build/target/go-licenses build/target/options build/target/third_party_licenses && touch build/target/go-licenses/skip && touch build/target/third_party_licenses/skip" } +// Java11+ container depends on the java agent project. To compile it, need a compatible JDK version: +// lower bound 11 and upper bound imageJavaVersion task validateJavaHome { - if (imageJavaVersion == "11" || imageJavaVersion == "17") { + def requiredForVer = ["11", "17", "21"] + if (requiredForVer.contains(imageJavaVersion)) { doFirst { - if (!project.hasProperty('java17Home') && !project.hasProperty('java11Home')) { - throw new GradleException('java17Home or java11Home property required. Re-run with -Pjava17Home or -Pjava11Home') + boolean propertyFound = false + // enable to build agent with compatible java versions (11-requiredForVer) + for (def checkVer : requiredForVer) { + if (project.hasProperty("java${checkVer}Home")) { + propertyFound = true + } + if (checkVer == imageJavaVersion) { + // cannot build agent with a higher version than the docker java ver + break + } + } + if (!propertyFound) { + throw new GradleException("java${imageJavaVersion}Home or compatible properties required for imageJavaVersion=${imageJavaVersion}. Re-run with -Pjava${imageJavaVersion}Home") } } } @@ -124,6 +139,7 @@ docker { buildArgs([ 'pull_licenses': project.rootProject.hasProperty(["docker-pull-licenses"]) || project.rootProject.hasProperty(["isRelease"]), + 'base_image': javaBaseImage, 'java_version': imageJavaVersion, ]) buildx useBuildx diff --git a/sdks/java/container/java21/build.gradle b/sdks/java/container/java21/build.gradle new file mode 100644 index 000000000000..038064102dcb --- /dev/null +++ b/sdks/java/container/java21/build.gradle @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * License); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an AS IS BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +project.ext { + // TODO(https://github.com/apache/beam/issues/28120) switch to temurin once available + javaBaseImage = 'openjdk' + imageJavaVersion = '21' +} + +// Load the main build script which contains all build logic. +apply from: "../common.gradle" + +dependencies { + dockerDependency project(path: ":sdks:java:container:agent") +} \ No newline at end of file diff --git a/sdks/java/container/java21/option-jamm.json b/sdks/java/container/java21/option-jamm.json new file mode 100644 index 000000000000..5647ff66be5c --- /dev/null +++ b/sdks/java/container/java21/option-jamm.json @@ -0,0 +1,12 @@ +{ + "name": "jamm", + "enabled": true, + "options": { + "java_arguments": [ + "--add-modules=jamm", + "--module-path=/opt/apache/beam/jars/jamm.jar", + "--add-opens=java.base/java.lang=jamm", + "--add-opens=java.base/java.util=jamm" + ] + } +} \ No newline at end of file diff --git a/sdks/java/testing/jpms-tests/build.gradle b/sdks/java/testing/jpms-tests/build.gradle index 6321f874c903..429fc063027f 100644 --- a/sdks/java/testing/jpms-tests/build.gradle +++ b/sdks/java/testing/jpms-tests/build.gradle @@ -23,10 +23,10 @@ plugins { } // overwrite javaVersion before applyJavaNature -if (project.hasProperty("compileAndRunTestsWithJava17")) { - javaVersion = '1.17' +if (project.hasProperty("testJavaVersion")) { + javaVersion = "1.${project.getProperty('testJavaVersion')}" as String } else { - javaVersion = '1.11' + javaVersion = "1.11" } applyJavaNature( @@ -42,13 +42,14 @@ ext.summary = "E2E test for Java 9 modules" // direct compileJava to use specified java version. project.tasks.compileJava { - if (project.hasProperty("compileAndRunTestsWithJava11")) { + if (project.hasProperty('testJavaVersion')) { options.fork = true - options.forkOptions.javaHome = project.findProperty("java11Home") as File - } else if (project.hasProperty("compileAndRunTestsWithJava17")) { - options.fork = true - options.forkOptions.javaHome = project.findProperty("java17Home") as File - setJava17Options(options) + options.forkOptions.javaHome = project.findProperty("java${project.getProperty('testJavaVersion')}Home") as File + if (project.getProperty('testJavaVersion') == '17') { + setJava17Options(options) + } else if (project.getProperty('testJavaVersion') == '21') { + setJava21Options(options) + } } } @@ -117,10 +118,12 @@ plugins.withType(JavaPlugin).configureEach{ } } -// JPMS requires JDK > 8 +// JPMS requires JDK > 8. Test tasks enabled when either +// (i) testJavaVersion property specified (assumed to be >8) or; +// (ii) current Java version is greater than 8 project.tasks.each { it.onlyIf { - project.hasProperty("compileAndRunTestsWithJava17") + project.hasProperty('testJavaVersion') || JavaVersion.VERSION_1_8.compareTo(JavaVersion.current()) < 0 } } diff --git a/sdks/java/testing/test-utils/build.gradle b/sdks/java/testing/test-utils/build.gradle index 50c815dd57f7..6e30693d8894 100644 --- a/sdks/java/testing/test-utils/build.gradle +++ b/sdks/java/testing/test-utils/build.gradle @@ -43,24 +43,15 @@ dependencies { testRuntimeOnly project(path: ":runners:direct-java", configuration: "shadowTest") } -task verifyJavaVersion(type: Test) { - filter { - includeTestsMatching 'org.apache.beam.sdk.testutils.jvmverification.JvmVerification.verifyCodeIsCompiledWithJava8' - includeTestsMatching 'org.apache.beam.sdk.testutils.jvmverification.JvmVerification.verifyTestCodeIsCompiledWithJava11' - includeTestsMatching 'org.apache.beam.sdk.testutils.jvmverification.JvmVerification.verifyRunningJVMVersionIs11' - } - doLast { - println 'Java verified' +['11', '17', '21'].each { + tasks.create(name: "verifyJavaVersion${it}", type: Test) { + filter { + includeTestsMatching "org.apache.beam.sdk.testutils.jvmverification.JvmVerification.verifyCodeIsCompiledWithJava8" + includeTestsMatching "org.apache.beam.sdk.testutils.jvmverification.JvmVerification.verifyTestCodeIsCompiledWithJava${it}" + includeTestsMatching "org.apache.beam.sdk.testutils.jvmverification.JvmVerification.verifyRunningJVMVersionIs${it}" + } + doLast { + println 'Java verified' + } } } - -task verifyJavaVersion17(type: Test) { - filter { - includeTestsMatching 'org.apache.beam.sdk.testutils.jvmverification.JvmVerification.verifyCodeIsCompiledWithJava8' - includeTestsMatching 'org.apache.beam.sdk.testutils.jvmverification.JvmVerification.verifyTestCodeIsCompiledWithJava17' - includeTestsMatching 'org.apache.beam.sdk.testutils.jvmverification.JvmVerification.verifyRunningJVMVersionIs17' - } - doLast { - println 'Java verified' - } -} \ No newline at end of file diff --git a/sdks/java/testing/test-utils/src/test/java/org/apache/beam/sdk/testutils/jvmverification/JvmVerification.java b/sdks/java/testing/test-utils/src/test/java/org/apache/beam/sdk/testutils/jvmverification/JvmVerification.java index ad29e8b6a1d6..a6b5d6dca6c1 100644 --- a/sdks/java/testing/test-utils/src/test/java/org/apache/beam/sdk/testutils/jvmverification/JvmVerification.java +++ b/sdks/java/testing/test-utils/src/test/java/org/apache/beam/sdk/testutils/jvmverification/JvmVerification.java @@ -20,6 +20,7 @@ import static org.apache.beam.sdk.testutils.jvmverification.JvmVerification.Java.v11; import static org.apache.beam.sdk.testutils.jvmverification.JvmVerification.Java.v17; import static org.apache.beam.sdk.testutils.jvmverification.JvmVerification.Java.v1_8; +import static org.apache.beam.sdk.testutils.jvmverification.JvmVerification.Java.v21; import static org.junit.Assert.assertEquals; import java.io.IOException; @@ -39,6 +40,7 @@ public class JvmVerification { versionMapping.put("0034", v1_8); versionMapping.put("0037", v11); versionMapping.put("003d", v17); + versionMapping.put("0041", v21); } // bytecode @@ -62,6 +64,11 @@ public void verifyTestCodeIsCompiledWithJava17() throws IOException { assertEquals(v17, getByteCodeVersion(JvmVerification.class)); } + @Test + public void verifyTestCodeIsCompiledWithJava21() throws IOException { + assertEquals(v21, getByteCodeVersion(JvmVerification.class)); + } + // jvm @Test public void verifyRunningJVMVersionIs11() { @@ -75,6 +82,12 @@ public void verifyRunningJVMVersionIs17() { assertEquals(v17.name, version); } + @Test + public void verifyRunningJVMVersionIs21() { + final String version = getJavaSpecification(); + assertEquals(v21.name, version); + } + private static <T> Java getByteCodeVersion(final Class<T> clazz) throws IOException { final InputStream stream = clazz.getClassLoader().getResourceAsStream(clazz.getName().replace(".", "/") + ".class"); @@ -91,7 +104,8 @@ private static String getJavaSpecification() { enum Java { v1_8("1.8"), v11("11"), - v17("17"); + v17("17"), + v21("21"); final String name; diff --git a/settings.gradle.kts b/settings.gradle.kts index d833a323212c..e8e374eb6e19 100644 --- a/settings.gradle.kts +++ b/settings.gradle.kts @@ -175,6 +175,7 @@ include(":sdks:java:container:agent") include(":sdks:java:container:java8") include(":sdks:java:container:java11") include(":sdks:java:container:java17") +include(":sdks:java:container:java21") include(":sdks:java:core") include(":sdks:java:core:jmh") include(":sdks:java:expansion-service") diff --git a/website/www/site/content/en/roadmap/java-sdk.md b/website/www/site/content/en/roadmap/java-sdk.md index b65424b57a3d..a1c85e139193 100644 --- a/website/www/site/content/en/roadmap/java-sdk.md +++ b/website/www/site/content/en/roadmap/java-sdk.md @@ -17,9 +17,9 @@ limitations under the License. # Java SDK Roadmap -## Next Java LTS version support (Java 17) +## Next Java LTS version support (Java 21) Work to support the next LTS release of Java is in progress. For more details -about the scope and info on the various tasks please see the JIRA ticket. +about the scope and info on the various tasks please see the GitHub Issue. -- JIRA: [BEAM-12240](https://issues.apache.org/jira/browse/BEAM-12240) +- GitHub: [#28120](https://github.com/apache/beam/issues/28120) From b97c7dee5aad99ad0fcf0ba53369811026f02012 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw <robertwb@gmail.com> Date: Fri, 20 Oct 2023 09:33:38 -0700 Subject: [PATCH 211/435] yapf --- sdks/python/apache_beam/yaml/json_utils.py | 5 +++-- sdks/python/apache_beam/yaml/yaml_io_test.py | 14 +++++++------- 2 files changed, 10 insertions(+), 9 deletions(-) diff --git a/sdks/python/apache_beam/yaml/json_utils.py b/sdks/python/apache_beam/yaml/json_utils.py index 5b0f5d74fa77..e11d18720617 100644 --- a/sdks/python/apache_beam/yaml/json_utils.py +++ b/sdks/python/apache_beam/yaml/json_utils.py @@ -148,8 +148,9 @@ def json_parser( else: cls = jsonschema.validators.validator_for(json_schema) cls.check_schema(json_schema) - validate_fn = _PicklableFromConstructor(lambda: jsonschema.validators.validator_for(json_schema)( - json_schema).validate) + validate_fn = _PicklableFromConstructor( + lambda: jsonschema.validators.validator_for(json_schema) + (json_schema).validate) to_row = json_to_row( schema_pb2.FieldType(row_type=schema_pb2.RowType(schema=beam_schema))) diff --git a/sdks/python/apache_beam/yaml/yaml_io_test.py b/sdks/python/apache_beam/yaml/yaml_io_test.py index 10aec8e8256d..d463430ce2e0 100644 --- a/sdks/python/apache_beam/yaml/yaml_io_test.py +++ b/sdks/python/apache_beam/yaml/yaml_io_test.py @@ -310,13 +310,13 @@ def test_read_json_without_error_handling(self): def test_read_json_with_bad_schema(self): with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions( pickle_library='cloudpickle')) as p: - with mock.patch( - 'apache_beam.io.ReadFromPubSub', - FakeReadFromPubSub(topic='my_topic', - messages=[PubsubMessage('{"some_int": 123}', - attributes={}), - PubsubMessage('{"some_int": "NOT"}', - attributes={})])): + with mock.patch('apache_beam.io.ReadFromPubSub', + FakeReadFromPubSub( + topic='my_topic', + messages=[PubsubMessage('{"some_int": 123}', + attributes={}), + PubsubMessage('{"some_int": "NOT"}', + attributes={})])): result = p | YamlTransform( ''' type: ReadFromPubSub From 6b927423f02eed1ea05aebfaf0084639467cd8e9 Mon Sep 17 00:00:00 2001 From: Rebecca Szper <98840847+rszper@users.noreply.github.com> Date: Fri, 20 Oct 2023 10:51:33 -0700 Subject: [PATCH 212/435] Editorial pass on content (#29094) --- .../io/built-in/google-bigquery.md | 39 +++++----- .../sdks/python-unrecoverable-errors.md | 78 +++++++++++-------- .../www/site/content/en/get-started/_index.md | 26 ++++--- .../site/content/en/get-started/downloads.md | 2 +- 4 files changed, 82 insertions(+), 63 deletions(-) diff --git a/website/www/site/content/en/documentation/io/built-in/google-bigquery.md b/website/www/site/content/en/documentation/io/built-in/google-bigquery.md index 26ca0baec0cf..769b05741345 100644 --- a/website/www/site/content/en/documentation/io/built-in/google-bigquery.md +++ b/website/www/site/content/en/documentation/io/built-in/google-bigquery.md @@ -261,7 +261,7 @@ BigQuery's exported JSON format. {{< paragraph class="language-py" >}} ***Note:*** `BigQuerySource()` is deprecated as of Beam SDK 2.25.0. Before 2.25.0, to read from -a BigQuery table using the Beam SDK, you will apply a `Read` transform on a `BigQuerySource`. For example, +a BigQuery table using the Beam SDK, apply a `Read` transform on a `BigQuerySource`. For example, `beam.io.Read(beam.io.BigQuerySource(table_spec))`. {{< /paragraph >}} @@ -397,8 +397,8 @@ for the destination table(s): whether the destination table must exist or can be created by the write operation. * The destination table's write disposition. The write disposition specifies - whether the data you write will replace an existing table, append rows to an - existing table, or write only to an empty table. + whether the data you write replaces an existing table, appends rows to an + existing table, or writes only to an empty table. In addition, if your write operation creates a new BigQuery table, you must also supply a table schema for the destination table. @@ -512,7 +512,7 @@ use a string that contains a JSON-serialized `TableSchema` object. To create a table schema in Python, you can either use a `TableSchema` object, or use a string that defines a list of fields. Single string based schemas do not support nested fields, repeated fields, or specifying a BigQuery mode for -fields (the mode will always be set to `NULLABLE`). +fields (the mode is always set to `NULLABLE`). {{< /paragraph >}} #### Using a TableSchema @@ -539,7 +539,7 @@ To create and use a table schema as a `TableSchema` object, follow these steps. 2. Create and append a `TableFieldSchema` object for each field in your table. -3. Next, use the `schema` parameter to provide your table schema when you apply +3. Use the `schema` parameter to provide your table schema when you apply a write transform. Set the parameter’s value to the `TableSchema` object. {{< /paragraph >}} @@ -728,8 +728,8 @@ The following examples use this `PCollection` that contains quotes. The `writeTableRows` method writes a `PCollection` of BigQuery `TableRow` objects to a BigQuery table. Each element in the `PCollection` represents a single row in the table. This example uses `writeTableRows` to write elements to a -`PCollection<TableRow>`. The write operation creates a table if needed; if the -table already exists, it will be replaced. +`PCollection<TableRow>`. The write operation creates a table if needed. If the +table already exists, it is replaced. {{< /paragraph >}} {{< highlight java >}} @@ -745,7 +745,7 @@ table already exists, it will be replaced. {{< paragraph class="language-py" >}} The following example code shows how to apply a `WriteToBigQuery` transform to write a `PCollection` of dictionaries to a BigQuery table. The write operation -creates a table if needed; if the table already exists, it will be replaced. +creates a table if needed. If the table already exists, it is replaced. {{< /paragraph >}} {{< highlight py >}} @@ -759,8 +759,8 @@ The `write` transform writes a `PCollection` of custom typed objects to a BigQue table. Use `.withFormatFunction(SerializableFunction)` to provide a formatting function that converts each input element in the `PCollection` into a `TableRow`. This example uses `write` to write a `PCollection<String>`. The -write operation creates a table if needed; if the table already exists, it will -be replaced. +write operation creates a table if needed. If the table already exists, it is +replaced. {{< /paragraph >}} {{< highlight java >}} @@ -786,7 +786,7 @@ BigQuery Storage Write API for Python SDK currently has some limitations on supp {{< /paragraph >}} {{< paragraph class="language-py" >}} -**Note:** If you want to run WriteToBigQuery with Storage Write API from the source code, you need to run `./gradlew :sdks:java:io:google-cloud-platform:expansion-service:build` to build the expansion-service jar. If you are running from a released Beam SDK, the jar will already be included. +**Note:** If you want to run WriteToBigQuery with Storage Write API from the source code, you need to run `./gradlew :sdks:java:io:google-cloud-platform:expansion-service:build` to build the expansion-service jar. If you are running from a released Beam SDK, the jar is already included. **Note:** Auto sharding is not currently supported for Python's Storage Write API exactly-once mode on DataflowRunner. @@ -877,32 +877,33 @@ Similar to streaming inserts, `STORAGE_WRITE_API` supports dynamically determini the number of parallel streams to write to BigQuery (starting 2.42.0). You can explicitly enable this using [`withAutoSharding`](https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.Write.html#withAutoSharding--). -***Note:*** `STORAGE_WRITE_API` will default to dynamic sharding when +`STORAGE_WRITE_API` defaults to dynamic sharding when `numStorageWriteApiStreams` is set to 0 or is unspecified. -***Note:*** Auto sharding with `STORAGE_WRITE_API` is supported on Dataflow's legacy runner, but **not** on Runner V2 +***Note:*** Auto sharding with `STORAGE_WRITE_API` is supported by Dataflow, but **not** on Runner v2. {{< /paragraph >}} When using `STORAGE_WRITE_API`, the `PCollection` returned by [`WriteResult.getFailedStorageApiInserts`](https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/gcp/bigquery/WriteResult.html#getFailedStorageApiInserts--) -will contain the rows that failed to be written to the Storage Write API sink. +contains the rows that failed to be written to the Storage Write API sink. #### At-least-once semantics If your use case allows for potential duplicate records in the target table, you can use the [`STORAGE_API_AT_LEAST_ONCE`](https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.Write.Method.html#STORAGE_API_AT_LEAST_ONCE) -method. Because this method doesn’t persist the records to be written to -BigQuery into its shuffle storage (needed to provide the exactly-once semantics -of the `STORAGE_WRITE_API` method), it is cheaper and results in lower latency -for most pipelines. If you use `STORAGE_API_AT_LEAST_ONCE`, you don’t need to +method. This method doesn’t persist the records to be written to +BigQuery into its shuffle storage, which is needed to provide the exactly-once semantics +of the `STORAGE_WRITE_API` method. Therefore, for most pipelines, using this method is often +less expensive and results in lower latency. +If you use `STORAGE_API_AT_LEAST_ONCE`, you don’t need to specify the number of streams, and you can’t specify the triggering frequency. Auto sharding is not applicable for `STORAGE_API_AT_LEAST_ONCE`. When using `STORAGE_API_AT_LEAST_ONCE`, the `PCollection` returned by [`WriteResult.getFailedStorageApiInserts`](https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/gcp/bigquery/WriteResult.html#getFailedStorageApiInserts--) -will contain the rows that failed to be written to the Storage Write API sink. +contains the rows that failed to be written to the Storage Write API sink. #### Quotas diff --git a/website/www/site/content/en/documentation/sdks/python-unrecoverable-errors.md b/website/www/site/content/en/documentation/sdks/python-unrecoverable-errors.md index 4e5d94ce8a8d..4fbb739e7ec7 100644 --- a/website/www/site/content/en/documentation/sdks/python-unrecoverable-errors.md +++ b/website/www/site/content/en/documentation/sdks/python-unrecoverable-errors.md @@ -16,46 +16,58 @@ See the License for the specific language governing permissions and limitations under the License. --> -# Unrecoverable Errors in Beam Python +# Unrecoverable errors in Beam Python -## What is an Unrecoverable Error? +Unrecoverable errors are issues that occur at job start-up time and +prevent jobs from ever running successfully. The problem usually stems +from a misconfiguration. This page provides context about +common errors and troubleshooting information. -An unrecoverable error is an issue at job start-up time that will -prevent a job from ever running successfully, usually due to some kind -of misconfiguration. Solving these issues when they occur is key to -successfully running a Beam Python pipeline. +## Job submission or Python runtime version mismatch {#python-version-mismatch} -## Common Unrecoverable Errors +If the Python version that you use to submit your job doesn't match the +Python version used to build the worker container, the job doesn't run. +The job fails immediately after job submission. -### Job Submission/Runtime Python Version Mismatch +To resolve this issue, ensure that the Python version used to submit the job +matches the Python container version. -If the Python version used for job submission does not match the -Python version used to build the worker container, the job will not -execute. Ensure that the Python version being used for job submission -and the container Python version match. +## Dependency resolution failures with pip {#dependency-resolution-failures} -### PIP Dependency Resolution Failures +During worker start-up, the worker might fail and, depending on the +runner, try to restart. -During worker start-up, dependencies are checked and installed in -the worker container before accepting work. If a pipeline requires -additional dependencies not already present in the runtime environment, -they are installed here. If there’s an issue during this process -(e.g. a dependency version cannot be found, or a worker cannot -connect to PyPI) the worker will fail and may try to restart -depending on the runner. Ensure that dependency versions provided in -your requirements.txt file exist and can be installed locally before -submitting jobs. +Before workers accept work, dependencies are checked and installed in +the worker container. If a pipeline requires +dependencies not already present in the runtime environment, +they are installed at this time. +When a problem occurs during this process, you might encounter +dependency resolution failures. -### Dependency Version Mismatches +Examples of problems include the following: -When additional dependencies like `torch`, `transformers`, etc. are not -specified via a requirements_file or preinstalled in a custom container -then the worker might fail to deserialize (unpickle) the user code. -This can result in `ModuleNotFound` errors. If dependencies are installed -but their versions don't match the versions in submission environment, -pipeline might have `AttributeError` messages. +- A dependency version can't be found. +- A worker can't connect to PyPI. -Ensure that the required dependencies at runtime and in the submission -environment are the same along with their versions. For better visibility, -debug logs are added specifying the dependencies at both stages starting in -Beam 2.52.0. For more information, see: https://beam.apache.org/documentation/sdks/python-pipeline-dependencies/#control-dependencies \ No newline at end of file +To resolve this issue, before submitting your job, ensure that the +dependency versions provided in your `requirements.txt` file exist +and that you can install them locally. + +## Dependency version mismatches {#dependency-version} + +When your pipeline has dependency version mismatches, you might +see `ModuleNotFound` errors or `AttributeError` messages. + + - The `ModuleNotFound` errors occur when additional dependencies, + such as `torch` and `transformers`, are neither specified in a + `requirements_file` nor preinstalled in a custom container. + In this case, the worker might fail to deserialize (unpickle) the user code. + +- Your pipeline might have `AttributeError` messages when dependencies + are installed but their versions don't match the versions in submission environment. + +To resolve these problems, ensure that the required dependencies and their versions are the same +at runtime and in the submission environment. To help you identify these issues, +in Apache Beam 2.52.0 and later versions, debug logs specify the dependencies at both stages. +For more information, see +[Control the dependencies the pipeline uses](https://beam.apache.org/documentation/sdks/python-pipeline-dependencies/#control-dependencies). \ No newline at end of file diff --git a/website/www/site/content/en/get-started/_index.md b/website/www/site/content/en/get-started/_index.md index c436129b066a..8aa6ff626c42 100644 --- a/website/www/site/content/en/get-started/_index.md +++ b/website/www/site/content/en/get-started/_index.md @@ -21,17 +21,18 @@ limitations under the License. # Get Started with Apache Beam -Learn to use Beam to create data processing pipelines that run on supported processing back-ends: +Learn how to use Beam to create data processing pipelines that run on supported processing back-ends. -## [Tour of Beam](https://tour.beam.apache.org) +## Tour of Beam -Learn Beam with an interactive tour with learning topics covering core Beam concepts -from simple ones to more advanced ones. +[Learn Beam with an interactive tour](https://tour.beam.apache.org). +Topics include core Beam concepts, from simple to advanced. You can try examples, do exercises, and solve challenges along the learning journey. -## [Beam Overview](/get-started/beam-overview) +## Beam Overview -Learn about the Beam model, the currently available Beam SDKs and Runners, and Beam's native I/O connectors. +Read the [Apache Beam Overview](/get-started/beam-overview) to learn about the Beam model, +the currently available Beam SDKs and runners, and Beam's native I/O connectors. ## Quickstarts for Java, Python, Go, and TypeScript @@ -49,10 +50,15 @@ See detailed walkthroughs of complete Beam pipelines. - [WordCount](/get-started/wordcount-example): Simple example pipelines that demonstrate basic Beam programming, including debugging and testing - [Mobile Gaming](/get-started/mobile-gaming-example): A series of more advanced pipelines that demonstrate use cases in the mobile gaming domain -## [Downloads and Releases](/get-started/downloads) +## Downloads and Releases -Find download links and information on the latest Beam releases, including versioning and release notes. +Find download links and information about the latest Beam releases, including versioning and release notes, +on the [Apache Beam Downloads](/get-started/downloads) page. -## [Support](/get-started/support) +## Support -Find resources, such as mailing lists and issue tracking, to help you use Beam. Ask questions and discuss topics via [Stack Overflow](https://stackoverflow.com/questions/tagged/apache-beam) or on Beam's [Slack Channel](https://apachebeam.slack.com). +- Find resources to help you use Beam, such as mailing lists and issue tracking, + on the [Support](/get-started/support) page. + - Ask questions and discuss topics on + [Stack Overflow](https://stackoverflow.com/questions/tagged/apache-beam) + or in the Beam [Slack Channel](https://apachebeam.slack.com). diff --git a/website/www/site/content/en/get-started/downloads.md b/website/www/site/content/en/get-started/downloads.md index b564a5801cd8..cc71f3101eb1 100644 --- a/website/www/site/content/en/get-started/downloads.md +++ b/website/www/site/content/en/get-started/downloads.md @@ -19,7 +19,7 @@ See the License for the specific language governing permissions and limitations under the License. --> -# Apache Beam&#8482; Downloads +# Apache Beam<sup>®</sup> Downloads > Beam SDK {{< param release_latest >}} is the latest released version. From b85c40bbdc879681d4a57908ed972d6d3a25e54f Mon Sep 17 00:00:00 2001 From: Yi Hu <yathu@google.com> Date: Fri, 20 Oct 2023 15:06:20 -0400 Subject: [PATCH 213/435] Fix rrio PreCommit build configs (#29097) --- .github/workflows/beam_PreCommit_Java.yml | 4 ++++ .../beam_PreCommit_Java_RequestResponse_IO_Direct.yml | 2 +- .test-infra/jenkins/job_PreCommit_Java.groovy | 1 + build.gradle.kts | 2 +- 4 files changed, 7 insertions(+), 2 deletions(-) diff --git a/.github/workflows/beam_PreCommit_Java.yml b/.github/workflows/beam_PreCommit_Java.yml index ec62a64ed970..e1a3361c0cf3 100644 --- a/.github/workflows/beam_PreCommit_Java.yml +++ b/.github/workflows/beam_PreCommit_Java.yml @@ -40,6 +40,7 @@ on: - '!sdks/java/io/elasticsearch/**' - '!sdks/java/io/elasticsearch-tests/**' - '!sdks/java/io/file-schema-transform/**' + - '!sdks/java/io/google-ads/**' - '!sdks/java/io/google-cloud-platform/**' - '!sdks/java/io/hadoop-common/**' - '!sdks/java/io/hadoop-file-system/**' @@ -59,6 +60,7 @@ on: - '!sdks/java/io/pulsar/**' - '!sdks/java/io/rabbitmq/**' - '!sdks/java/io/redis/**' + - '!sdks/java/io/rrio/**' - '!sdks/java/io/singlestore/**' - '!sdks/java/io/snowflake/**' - '!sdks/java/io/solr/**' @@ -90,6 +92,7 @@ on: - '!sdks/java/io/elasticsearch/**' - '!sdks/java/io/elasticsearch-tests/**' - '!sdks/java/io/file-schema-transform/**' + - '!sdks/java/io/google-ads/**' - '!sdks/java/io/google-cloud-platform/**' - '!sdks/java/io/hadoop-common/**' - '!sdks/java/io/hadoop-file-system/**' @@ -109,6 +112,7 @@ on: - '!sdks/java/io/pulsar/**' - '!sdks/java/io/rabbitmq/**' - '!sdks/java/io/redis/**' + - '!sdks/java/io/rrio/**' - '!sdks/java/io/singlestore/**' - '!sdks/java/io/snowflake/**' - '!sdks/java/io/solr/**' diff --git a/.github/workflows/beam_PreCommit_Java_RequestResponse_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_RequestResponse_IO_Direct.yml index 1da99d7cc525..27de388a6a65 100644 --- a/.github/workflows/beam_PreCommit_Java_RequestResponse_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_RequestResponse_IO_Direct.yml @@ -85,7 +85,7 @@ jobs: - name: run RequestResponse IO build script uses: ./.github/actions/gradle-command-self-hosted-action with: - gradle-command: :sdks:java:io:RequestResponse:build + gradle-command: :sdks:java:io:rrio:build arguments: | -PdisableSpotlessCheck=true \ -PdisableCheckStyle=true \ diff --git a/.test-infra/jenkins/job_PreCommit_Java.groovy b/.test-infra/jenkins/job_PreCommit_Java.groovy index 41a3b418a015..d1acb1ac7315 100644 --- a/.test-infra/jenkins/job_PreCommit_Java.groovy +++ b/.test-infra/jenkins/job_PreCommit_Java.groovy @@ -54,6 +54,7 @@ def excludePaths = [ 'io/pulsar', 'io/rabbitmq', 'io/redis', + 'io/rrio', 'io/singlestore', 'io/snowflake', 'io/solr', diff --git a/build.gradle.kts b/build.gradle.kts index 5bcfbb3ed06e..a97b73757249 100644 --- a/build.gradle.kts +++ b/build.gradle.kts @@ -294,7 +294,6 @@ tasks.register("javaPreCommit") { dependsOn(":sdks:java:fn-execution:build") dependsOn(":sdks:java:harness:build") dependsOn(":sdks:java:harness:jmh:build") - dependsOn(":sdks:java:io:azure:build") dependsOn(":sdks:java:io:bigquery-io-perf-tests:build") dependsOn(":sdks:java:io:common:build") dependsOn(":sdks:java:io:contextualtextio:build") @@ -351,6 +350,7 @@ tasks.register("javaioPreCommit") { dependsOn(":sdks:java:io:parquet:build") dependsOn(":sdks:java:io:rabbitmq:build") dependsOn(":sdks:java:io:redis:build") + dependsOn(":sdks:java:io:rrio:build") dependsOn(":sdks:java:io:singlestore:build") dependsOn(":sdks:java:io:solr:build") dependsOn(":sdks:java:io:splunk:build") From 1284d769892ddbba13be2fca8daa8a387c71c122 Mon Sep 17 00:00:00 2001 From: Pranav Bhandari <bhandari.pranav22@gmail.com> Date: Fri, 20 Oct 2023 16:19:04 -0400 Subject: [PATCH 214/435] Fix various issues in SplunkIO (#28825) * Fix GZIP compression in HttpEventPublisher. * Add checks to make sure the provided URL is valid. * Fix issue with DefaultCoder in AutoValue generated classes. * Add support for Splunk `fields` metadata. Also fix Coder issues for SplunkEvent. * Address comments. --- .../beam/sdk/coders/CoderProviders.java | 7 +- .../apache/beam/sdk/coders/DefaultCoder.java | 8 + sdks/java/io/splunk/build.gradle | 1 + .../sdk/io/splunk/HttpEventPublisher.java | 9 + .../beam/sdk/io/splunk/SplunkEvent.java | 21 +- .../beam/sdk/io/splunk/SplunkEventCoder.java | 206 ++++++++++++++++ .../beam/sdk/io/splunk/SplunkEventWriter.java | 65 ++++- .../apache/beam/sdk/io/splunk/SplunkIO.java | 1 - .../sdk/io/splunk/SplunkEventCoderTest.java | 228 ++++++++++++++++++ .../beam/sdk/io/splunk/SplunkEventTest.java | 5 + .../sdk/io/splunk/SplunkEventWriterTest.java | 63 ++++- .../beam/sdk/io/splunk/SplunkIOTest.java | 12 +- 12 files changed, 603 insertions(+), 23 deletions(-) create mode 100644 sdks/java/io/splunk/src/main/java/org/apache/beam/sdk/io/splunk/SplunkEventCoder.java create mode 100644 sdks/java/io/splunk/src/test/java/org/apache/beam/sdk/io/splunk/SplunkEventCoderTest.java diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderProviders.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderProviders.java index 8e47f4f2bc9c..e0a3199d0c69 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderProviders.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderProviders.java @@ -178,7 +178,12 @@ public CoderProviderForCoder(TypeDescriptor<?> type, Coder<?> coder) { @Override public <T> Coder<T> coderFor(TypeDescriptor<T> type, List<? extends Coder<?>> componentCoders) throws CannotProvideCoderException { - if (!this.type.equals(type)) { + boolean isTypeEqual = this.type.equals(type); + boolean isAutoValueConcrete = + type.getRawType().getName().contains("AutoValue_") + && this.type.getRawType().isAssignableFrom(type.getRawType()); + + if (!isTypeEqual && !isAutoValueConcrete) { throw new CannotProvideCoderException( String.format( "Unable to provide coder for %s, this factory can only provide coders for %s", diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DefaultCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DefaultCoder.java index 52718fcde2af..782a77cde685 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DefaultCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DefaultCoder.java @@ -88,6 +88,14 @@ public <T> Coder<T> coderFor( Class<?> clazz = typeDescriptor.getRawType(); DefaultCoder defaultAnnotation = clazz.getAnnotation(DefaultCoder.class); + if (defaultAnnotation == null) { + // check if the superclass has DefaultCoder annotation if the class is generated using + // AutoValue + if (clazz.getName().contains("AutoValue_")) { + clazz = clazz.getSuperclass(); + defaultAnnotation = clazz.getAnnotation(DefaultCoder.class); + } + } if (defaultAnnotation == null) { throw new CannotProvideCoderException( String.format("Class %s does not have a @DefaultCoder annotation.", clazz.getName())); diff --git a/sdks/java/io/splunk/build.gradle b/sdks/java/io/splunk/build.gradle index dd1b15e10dde..41a7a409e890 100644 --- a/sdks/java/io/splunk/build.gradle +++ b/sdks/java/io/splunk/build.gradle @@ -37,6 +37,7 @@ dependencies { implementation library.java.joda_time implementation library.java.slf4j_api implementation library.java.vendored_guava_32_1_2_jre + implementation library.java.commons_io testImplementation library.java.junit testImplementation group: 'org.mock-server', name: 'mockserver-junit-rule', version: '5.10.0' testImplementation group: 'org.mock-server', name: 'mockserver-client-java', version: '5.10.0' diff --git a/sdks/java/io/splunk/src/main/java/org/apache/beam/sdk/io/splunk/HttpEventPublisher.java b/sdks/java/io/splunk/src/main/java/org/apache/beam/sdk/io/splunk/HttpEventPublisher.java index 6c5537990bdf..f34fcb7c4e0e 100644 --- a/sdks/java/io/splunk/src/main/java/org/apache/beam/sdk/io/splunk/HttpEventPublisher.java +++ b/sdks/java/io/splunk/src/main/java/org/apache/beam/sdk/io/splunk/HttpEventPublisher.java @@ -22,9 +22,11 @@ import com.google.api.client.http.ByteArrayContent; import com.google.api.client.http.GZipEncoding; import com.google.api.client.http.GenericUrl; +import com.google.api.client.http.HttpBackOffIOExceptionHandler; import com.google.api.client.http.HttpBackOffUnsuccessfulResponseHandler; import com.google.api.client.http.HttpBackOffUnsuccessfulResponseHandler.BackOffRequired; import com.google.api.client.http.HttpContent; +import com.google.api.client.http.HttpIOExceptionHandler; import com.google.api.client.http.HttpMediaType; import com.google.api.client.http.HttpRequest; import com.google.api.client.http.HttpRequestFactory; @@ -139,6 +141,9 @@ HttpResponse execute(List<SplunkEvent> events) throws IOException { responseHandler.setBackOffRequired(BackOffRequired.ON_SERVER_ERROR); request.setUnsuccessfulResponseHandler(responseHandler); + HttpIOExceptionHandler ioExceptionHandler = + new HttpBackOffIOExceptionHandler(getConfiguredBackOff()); + request.setIOExceptionHandler(ioExceptionHandler); setHeaders(request, token()); return request.execute(); @@ -180,6 +185,10 @@ void close() throws IOException { */ private void setHeaders(HttpRequest request, String token) { request.getHeaders().setAuthorization(String.format(AUTHORIZATION_SCHEME, token)); + + if (enableGzipHttpCompression()) { + request.getHeaders().setContentEncoding("gzip"); + } } /** diff --git a/sdks/java/io/splunk/src/main/java/org/apache/beam/sdk/io/splunk/SplunkEvent.java b/sdks/java/io/splunk/src/main/java/org/apache/beam/sdk/io/splunk/SplunkEvent.java index 7dd78e1754b4..177900a2d09a 100644 --- a/sdks/java/io/splunk/src/main/java/org/apache/beam/sdk/io/splunk/SplunkEvent.java +++ b/sdks/java/io/splunk/src/main/java/org/apache/beam/sdk/io/splunk/SplunkEvent.java @@ -20,9 +20,9 @@ import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull; import com.google.auto.value.AutoValue; +import com.google.gson.JsonObject; import com.google.gson.annotations.SerializedName; -import org.apache.beam.sdk.schemas.AutoValueSchema; -import org.apache.beam.sdk.schemas.annotations.DefaultSchema; +import org.apache.beam.sdk.coders.DefaultCoder; import org.checkerframework.checker.nullness.qual.Nullable; /** @@ -39,7 +39,7 @@ * <li>index * </ul> */ -@DefaultSchema(AutoValueSchema.class) +@DefaultCoder(SplunkEventCoder.class) @AutoValue public abstract class SplunkEvent { @@ -59,6 +59,8 @@ public static Builder newBuilder() { public abstract @Nullable String index(); + public abstract @Nullable JsonObject fields(); + public abstract @Nullable String event(); /** A builder class for creating a {@link SplunkEvent}. */ @@ -75,6 +77,8 @@ public abstract static class Builder { abstract Builder setIndex(String index); + abstract Builder setFields(JsonObject fields); + abstract Builder setEvent(String event); abstract String event(); @@ -136,6 +140,17 @@ public Builder withIndex(String index) { return setIndex(index); } + /** + * Assigns fields value to the event metadata. + * + * @param fields fields value to assign + */ + public Builder withFields(JsonObject fields) { + checkNotNull(fields, "withFields(fields) called with null input."); + + return setFields(fields); + } + /** * Assigns the event payload to be sent to the HEC endpoint. * diff --git a/sdks/java/io/splunk/src/main/java/org/apache/beam/sdk/io/splunk/SplunkEventCoder.java b/sdks/java/io/splunk/src/main/java/org/apache/beam/sdk/io/splunk/SplunkEventCoder.java new file mode 100644 index 000000000000..35d5314ae9ee --- /dev/null +++ b/sdks/java/io/splunk/src/main/java/org/apache/beam/sdk/io/splunk/SplunkEventCoder.java @@ -0,0 +1,206 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.splunk; + +import com.google.gson.Gson; +import com.google.gson.JsonObject; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import org.apache.beam.sdk.coders.AtomicCoder; +import org.apache.beam.sdk.coders.BigEndianLongCoder; +import org.apache.beam.sdk.coders.CoderException; +import org.apache.beam.sdk.coders.CoderProvider; +import org.apache.beam.sdk.coders.CoderProviders; +import org.apache.beam.sdk.coders.NullableCoder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.commons.io.IOUtils; + +/** A {@link org.apache.beam.sdk.coders.Coder} for {@link SplunkEvent} objects. */ +public class SplunkEventCoder extends AtomicCoder<SplunkEvent> { + + private static final SplunkEventCoder SPLUNK_EVENT_CODER = new SplunkEventCoder(); + + private static final TypeDescriptor<SplunkEvent> TYPE_DESCRIPTOR = + new TypeDescriptor<SplunkEvent>() {}; + private static final StringUtf8Coder STRING_UTF_8_CODER = StringUtf8Coder.of(); + private static final NullableCoder<String> STRING_NULLABLE_CODER = + NullableCoder.of(STRING_UTF_8_CODER); + private static final NullableCoder<Long> LONG_NULLABLE_CODER = + NullableCoder.of(BigEndianLongCoder.of()); + + private static final Gson GSON = new Gson(); + + // Version markers must be >= 2. + private static final int VERSION_3 = 3; + + public static SplunkEventCoder of() { + return SPLUNK_EVENT_CODER; + } + + public static CoderProvider getCoderProvider() { + return CoderProviders.forCoder(TYPE_DESCRIPTOR, SplunkEventCoder.of()); + } + + @Override + @SuppressWarnings("nullness") + public void encode(SplunkEvent value, OutputStream out) throws IOException { + out.write(VERSION_3); + + LONG_NULLABLE_CODER.encode(value.time(), out); + STRING_NULLABLE_CODER.encode(value.host(), out); + STRING_NULLABLE_CODER.encode(value.source(), out); + STRING_NULLABLE_CODER.encode(value.sourceType(), out); + STRING_NULLABLE_CODER.encode(value.index(), out); + String fields = value.fields() == null ? null : value.fields().toString(); + STRING_NULLABLE_CODER.encode(fields, out); + STRING_UTF_8_CODER.encode(value.event(), out); + } + + @Override + public SplunkEvent decode(InputStream in) throws CoderException, IOException { + SplunkEvent.Builder builder = SplunkEvent.newBuilder(); + + int v = in.read(); + + // Versions 1 and 2 of this coder had no version marker field, but 1st byte in the serialized + // data was always 0 or 1 (present/not present indicator for a nullable field). + // So here we assume if the first byte is >= 2 then it's the version marker. + + if (v >= 2) { + decodeWithVersion(v, in, builder); + } else { + // It's impossible to distinguish between V1 and V2 without re-reading portions of the input + // stream twice (and without the version marker), so we must have a ByteArrayInputStream copy, + // which is guaranteed to support mark()/reset(). + + ByteArrayOutputStream os = new ByteArrayOutputStream(); + os.write(v); + IOUtils.copy(in, os); + ByteArrayInputStream streamCopy = new ByteArrayInputStream(os.toByteArray()); + + decodeVersion1or2(streamCopy, builder); + } + + return builder.build(); + } + + private void decodeWithVersion(int version, InputStream in, SplunkEvent.Builder builder) + throws IOException { + + decodeCommonFields(in, builder); + + if (version >= VERSION_3) { + String fields = STRING_NULLABLE_CODER.decode(in); + if (fields != null) { + builder.withFields(GSON.fromJson(fields, JsonObject.class)); + } + + String event = STRING_UTF_8_CODER.decode(in); + builder.withEvent(event); + } + } + + private void decodeVersion1or2(ByteArrayInputStream in, SplunkEvent.Builder builder) + throws IOException { + + decodeCommonFields(in, builder); + + in.mark(Integer.MAX_VALUE); + + // The following fields may be different between V1 and V2. + + // V1 format: <... common fields...> <event length> <event string> + // V2 format: <... common fields...> <fields present indicator byte 0/1> + // <fields length, if present> <fields string> <event length> <event string> + + // We try to read this as V2 first. If any exception, fall back to V1. + + // Note: it's impossible to incorrectly parse V1 data with V2 decoder (potentially causing + // corrupted fields in the message). If we try that and the 1st byte is: + // - 2 or more: decoding fails because V2 expects it to be either 0 or 1 (present indicator). + // - 1: this means the "event" string length is 1, so we have only 1 more byte in the stream. + // V2 decoding fails with EOF assuming 1 is the "fields" string length and reading + // at least 1 more byte. + // - 0: this means the "event" string is empty, so we have no more bytes in the stream. + // V2 decoding fails with EOF assuming 0 is the "fields" string length and reading + // the next "event" field. + + JsonObject fields = null; + String event; + + try { + // Assume V2 first. + String fieldsString = STRING_NULLABLE_CODER.decode(in); + if (fieldsString != null) { + fields = GSON.fromJson(fieldsString, JsonObject.class); + } + event = STRING_UTF_8_CODER.decode(in); + } catch (CoderException e) { + // If failed, reset the stream and parse as V1. + in.reset(); + event = STRING_UTF_8_CODER.decode(in); + } + + if (fields != null) { + builder.withFields(fields); + } + builder.withEvent(event); + } + + private void decodeCommonFields(InputStream in, SplunkEvent.Builder builder) throws IOException { + Long time = LONG_NULLABLE_CODER.decode(in); + if (time != null) { + builder.withTime(time); + } + + String host = STRING_NULLABLE_CODER.decode(in); + if (host != null) { + builder.withHost(host); + } + + String source = STRING_NULLABLE_CODER.decode(in); + if (source != null) { + builder.withSource(source); + } + + String sourceType = STRING_NULLABLE_CODER.decode(in); + if (sourceType != null) { + builder.withSourceType(sourceType); + } + + String index = STRING_NULLABLE_CODER.decode(in); + if (index != null) { + builder.withIndex(index); + } + } + + @Override + public TypeDescriptor<SplunkEvent> getEncodedTypeDescriptor() { + return TYPE_DESCRIPTOR; + } + + @Override + public void verifyDeterministic() throws NonDeterministicException { + throw new NonDeterministicException( + this, "SplunkEvent can hold arbitrary instances, which may be non-deterministic."); + } +} diff --git a/sdks/java/io/splunk/src/main/java/org/apache/beam/sdk/io/splunk/SplunkEventWriter.java b/sdks/java/io/splunk/src/main/java/org/apache/beam/sdk/io/splunk/SplunkEventWriter.java index 8ec2a064ee0d..615d4e932f4d 100644 --- a/sdks/java/io/splunk/src/main/java/org/apache/beam/sdk/io/splunk/SplunkEventWriter.java +++ b/sdks/java/io/splunk/src/main/java/org/apache/beam/sdk/io/splunk/SplunkEventWriter.java @@ -33,8 +33,9 @@ import java.security.KeyStoreException; import java.security.NoSuchAlgorithmException; import java.security.cert.CertificateException; -import java.time.Instant; import java.util.List; +import java.util.regex.Matcher; +import java.util.regex.Pattern; import org.apache.beam.repackaged.core.org.apache.commons.compress.utils.IOUtils; import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.io.fs.MatchResult; @@ -53,8 +54,11 @@ import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.values.KV; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.net.InetAddresses; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.net.InternetDomainName; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Duration; import org.slf4j.Logger; @@ -70,7 +74,7 @@ }) abstract class SplunkEventWriter extends DoFn<KV<Integer, SplunkEvent>, SplunkWriteError> { - private static final Integer DEFAULT_BATCH_COUNT = 1; + private static final Integer DEFAULT_BATCH_COUNT = 10; private static final Boolean DEFAULT_DISABLE_CERTIFICATE_VALIDATION = false; private static final Boolean DEFAULT_ENABLE_BATCH_LOGS = true; private static final Boolean DEFAULT_ENABLE_GZIP_HTTP_COMPRESSION = true; @@ -98,6 +102,13 @@ abstract class SplunkEventWriter extends DoFn<KV<Integer, SplunkEvent>, SplunkWr private static final String COUNT_STATE_NAME = "count"; private static final String TIME_ID_NAME = "expiry"; + private static final Pattern URL_PATTERN = Pattern.compile("^http(s?)://([^:]+)(:[0-9]+)?$"); + + @VisibleForTesting + protected static final String INVALID_URL_FORMAT_MESSAGE = + "Invalid url format. Url format should match PROTOCOL://HOST[:PORT], where PORT is optional. " + + "Supported Protocols are http and https. eg: http://hostname:8088"; + @StateId(BUFFER_STATE_NAME) private final StateSpec<BagState<SplunkEvent>> buffer = StateSpecs.bag(); @@ -139,6 +150,7 @@ static Builder newBuilder() { public void setup() { checkArgument(url().isAccessible(), "url is required for writing events."); + checkArgument(isValidUrlFormat(url().get()), INVALID_URL_FORMAT_MESSAGE); checkArgument(token().isAccessible(), "Access token is required for writing events."); // Either user supplied or default batchCount. @@ -287,7 +299,7 @@ private void flush( response = publisher.execute(events); if (!response.isSuccessStatusCode()) { - UNSUCCESSFUL_WRITE_LATENCY_MS.update(System.nanoTime() - startTime); + UNSUCCESSFUL_WRITE_LATENCY_MS.update(nanosToMillis(System.nanoTime() - startTime)); FAILED_WRITES.inc(countState.read()); int statusCode = response.getStatusCode(); if (statusCode >= 400 && statusCode < 500) { @@ -305,7 +317,7 @@ private void flush( events, response.getStatusMessage(), response.getStatusCode(), receiver); } else { - SUCCESSFUL_WRITE_LATENCY_MS.update(Instant.now().toEpochMilli() - startTime); + SUCCESSFUL_WRITE_LATENCY_MS.update(nanosToMillis(System.nanoTime() - startTime)); SUCCESS_WRITES.inc(countState.read()); VALID_REQUESTS.inc(); SUCCESSFUL_WRITE_BATCH_SIZE.update(countState.read()); @@ -321,7 +333,7 @@ private void flush( e.getStatusCode(), e.getContent(), e.getStatusMessage()); - UNSUCCESSFUL_WRITE_LATENCY_MS.update(System.nanoTime() - startTime); + UNSUCCESSFUL_WRITE_LATENCY_MS.update(nanosToMillis(System.nanoTime() - startTime)); FAILED_WRITES.inc(countState.read()); int statusCode = e.getStatusCode(); if (statusCode >= 400 && statusCode < 500) { @@ -336,7 +348,7 @@ private void flush( } catch (IOException ioe) { LOG.error("Error writing to Splunk: {}", ioe.getMessage()); - UNSUCCESSFUL_WRITE_LATENCY_MS.update(System.nanoTime() - startTime); + UNSUCCESSFUL_WRITE_LATENCY_MS.update(nanosToMillis(System.nanoTime() - startTime)); FAILED_WRITES.inc(countState.read()); INVALID_REQUESTS.inc(); @@ -350,8 +362,21 @@ private void flush( bufferState.clear(); countState.clear(); - if (response != null) { - response.disconnect(); + // We've observed cases where errors at this point can cause the pipeline to keep retrying + // the same events over and over (e.g. from Dataflow Runner's Pub/Sub implementation). Since + // the events have either been published or wrapped for error handling, we can safely + // ignore this error, though there may or may not be a leak of some type depending on + // HttpResponse's implementation. However, any potential leak would still happen if we let + // the exception fall through, so this isn't considered a major issue. + try { + if (response != null) { + response.ignore(); + } + } catch (IOException e) { + LOG.warn( + "Error ignoring response from Splunk. Messages should still have published, but there" + + " might be a connection leak.", + e); } } } @@ -426,6 +451,26 @@ public static byte[] getCertFromGcsAsBytes(String filePath) throws IOException { } } + @VisibleForTesting + static boolean isValidUrlFormat(String url) { + Matcher matcher = URL_PATTERN.matcher(url); + if (matcher.find()) { + String host = matcher.group(2); + return InetAddresses.isInetAddress(host) || InternetDomainName.isValid(host); + } + return false; + } + + /** + * Converts Nanoseconds to Milliseconds. + * + * @param ns time in nanoseconds + * @return time in milliseconds + */ + private static long nanosToMillis(long ns) { + return Math.round(((double) ns) / 1e6); + } + @AutoValue.Builder abstract static class Builder { @@ -458,6 +503,9 @@ abstract Builder setDisableCertificateValidation( */ Builder withUrl(ValueProvider<String> url) { checkArgument(url != null, "withURL(url) called with null input."); + if (url.isAccessible()) { + checkArgument(isValidUrlFormat(url.get()), INVALID_URL_FORMAT_MESSAGE); + } return setUrl(url); } @@ -469,6 +517,7 @@ Builder withUrl(ValueProvider<String> url) { */ Builder withUrl(String url) { checkArgument(url != null, "withURL(url) called with null input."); + checkArgument(isValidUrlFormat(url), INVALID_URL_FORMAT_MESSAGE); return setUrl(ValueProvider.StaticValueProvider.of(url)); } diff --git a/sdks/java/io/splunk/src/main/java/org/apache/beam/sdk/io/splunk/SplunkIO.java b/sdks/java/io/splunk/src/main/java/org/apache/beam/sdk/io/splunk/SplunkIO.java index bd1e716951d4..2127cc55752d 100644 --- a/sdks/java/io/splunk/src/main/java/org/apache/beam/sdk/io/splunk/SplunkIO.java +++ b/sdks/java/io/splunk/src/main/java/org/apache/beam/sdk/io/splunk/SplunkIO.java @@ -159,7 +159,6 @@ public PCollection<SplunkWriteError> expand(PCollection<SplunkEvent> input) { .withRootCaCertificatePath(rootCaCertificatePath()) .withEnableBatchLogs(enableBatchLogs()) .withEnableGzipHttpCompression(enableGzipHttpCompression()); - ; SplunkEventWriter writer = builder.build(); LOG.info("SplunkEventWriter configured"); diff --git a/sdks/java/io/splunk/src/test/java/org/apache/beam/sdk/io/splunk/SplunkEventCoderTest.java b/sdks/java/io/splunk/src/test/java/org/apache/beam/sdk/io/splunk/SplunkEventCoderTest.java new file mode 100644 index 000000000000..8267e406960a --- /dev/null +++ b/sdks/java/io/splunk/src/test/java/org/apache/beam/sdk/io/splunk/SplunkEventCoderTest.java @@ -0,0 +1,228 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.splunk; + +import static org.junit.Assert.assertEquals; + +import com.google.gson.JsonObject; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.nio.charset.StandardCharsets; +import org.apache.commons.codec.DecoderException; +import org.apache.commons.codec.binary.Hex; +import org.junit.Test; + +/** Unit tests for {@link SplunkEventCoder} class. */ +public class SplunkEventCoderTest { + + /** + * Test whether {@link SplunkEventCoder} is able to encode/decode a {@link SplunkEvent} correctly. + * + * @throws IOException + */ + @Test + public void testEncodeDecode() throws IOException { + + String event = "test-event"; + String host = "test-host"; + String index = "test-index"; + String source = "test-source"; + String sourceType = "test-source-type"; + Long time = 123456789L; + + SplunkEvent actualEvent = + SplunkEvent.newBuilder() + .withEvent(event) + .withHost(host) + .withIndex(index) + .withSource(source) + .withSourceType(sourceType) + .withTime(time) + .build(); + + SplunkEventCoder coder = SplunkEventCoder.of(); + try (ByteArrayOutputStream bos = new ByteArrayOutputStream()) { + coder.encode(actualEvent, bos); + try (ByteArrayInputStream bin = new ByteArrayInputStream(bos.toByteArray())) { + SplunkEvent decodedEvent = coder.decode(bin); + assertEquals(decodedEvent, actualEvent); + } + } + } + + /** + * Test whether {@link SplunkEventCoder} is able to encode/decode a {@link SplunkEvent} with + * metadata 'fields'. + * + * @throws IOException + */ + @Test + public void testEncodeDecodeFields() throws IOException { + + String event = "test-event"; + JsonObject fields = new JsonObject(); + fields.addProperty("test-key", "test-value"); + + SplunkEvent actualEvent = SplunkEvent.newBuilder().withEvent(event).withFields(fields).build(); + + SplunkEventCoder coder = SplunkEventCoder.of(); + try (ByteArrayOutputStream bos = new ByteArrayOutputStream()) { + coder.encode(actualEvent, bos); + try (ByteArrayInputStream bin = new ByteArrayInputStream(bos.toByteArray())) { + SplunkEvent decodedEvent = coder.decode(bin); + assertEquals(decodedEvent, actualEvent); + } + } + } + + /** + * Tests whether {@link SplunkEventCoder} is able to decode a {@link SplunkEvent} encoded using + * the older coder version 1 (commit f0ff6cc). + */ + @Test + public void testBackwardsCompatibility_canDecodeVersion1() throws IOException, DecoderException { + + SplunkEvent expectedEvent = + SplunkEvent.newBuilder() + .withEvent("e") + .withHost("h") + .withIndex("i") + .withSource("s") + .withSourceType("st") + .withTime(1234L) + .build(); + + String hex = "0100000000000004d2010168010173010273740101690165"; + SplunkEvent actualEvent = SplunkEventCoder.of().decode(fromHex(hex)); + + assertEquals(expectedEvent, actualEvent); + } + + /** + * Tests whether {@link SplunkEventCoder} is able to decode a {@link SplunkEvent} encoded using + * the older coder version 1 (commit f0ff6cc) and having an empty "event" field. + * + * <p>An empty field is encoded as <code>00</code>, which may look like the present/not present + * marker for the "fields" field in V2. + */ + @Test + public void testBackwardsCompatibility_canDecodeVersion1withEmptyEvent() + throws IOException, DecoderException { + + SplunkEvent expectedEvent = + SplunkEvent.newBuilder() + .withEvent("") + .withHost("h") + .withIndex("i") + .withSource("s") + .withSourceType("st") + .withTime(1234L) + .build(); + + String hex = "0100000000000004d20101680101730102737401016900"; + SplunkEvent actualEvent = SplunkEventCoder.of().decode(fromHex(hex)); + + assertEquals(expectedEvent, actualEvent); + } + + /** + * Tests whether {@link SplunkEventCoder} is able to decode a {@link SplunkEvent} encoded using + * the older coder version 1 (commit f0ff6cc) and having the "event" field of length 1. + * + * <p>This is a special case when "event" is of length 1 and the first character code is 00. This + * is encoded as byte sequence 01 00 by V1 coder, which can be treated as an empty "fields" field + * by V2 decoder. + */ + @Test + public void testBackwardsCompatibility_canDecodeVersion1withEventLength1() + throws IOException, DecoderException { + + SplunkEvent expectedEvent = + SplunkEvent.newBuilder() + .withEvent(new String(new byte[] {0}, StandardCharsets.UTF_8)) + .withHost("h") + .withIndex("i") + .withSource("s") + .withSourceType("st") + .withTime(1234L) + .build(); + + String hex = "0100000000000004d2010168010173010273740101690100"; + SplunkEvent actualEvent = SplunkEventCoder.of().decode(fromHex(hex)); + + assertEquals(expectedEvent, actualEvent); + } + + /** + * Tests whether {@link SplunkEventCoder} is able to decode a {@link SplunkEvent} encoded using + * the older coder version 2 (commit 5e53040), without the newly added "fields" field. + */ + @Test + public void testBackwardsCompatibility_canDecodeVersion2() throws IOException, DecoderException { + + SplunkEvent expectedEvent = + SplunkEvent.newBuilder() + .withEvent("e") + .withHost("h") + .withIndex("i") + .withSource("s") + .withSourceType("st") + .withTime(1234L) + .build(); + + String hex = "0100000000000004d201016801017301027374010169000165"; + SplunkEvent actualEvent = SplunkEventCoder.of().decode(fromHex(hex)); + + assertEquals(expectedEvent, actualEvent); + } + + /** + * Tests whether {@link SplunkEventCoder} is able to decode a {@link SplunkEvent} encoded using + * the older coder version 2 (commit 5e53040), with the newly added "fields" field. + */ + @Test + public void testBackwardsCompatibility_canDecodeVersion2withFields() + throws IOException, DecoderException { + + JsonObject fields = new JsonObject(); + fields.addProperty("k", "v"); + + SplunkEvent expectedEvent = + SplunkEvent.newBuilder() + .withEvent("e") + .withHost("h") + .withIndex("i") + .withSource("s") + .withSourceType("st") + .withTime(1234L) + .withFields(fields) + .build(); + + String hex = "0100000000000004d20101680101730102737401016901097b226b223a2276227d0165"; + SplunkEvent actualEvent = SplunkEventCoder.of().decode(fromHex(hex)); + + assertEquals(expectedEvent, actualEvent); + } + + private static InputStream fromHex(String hex) throws DecoderException { + byte[] b = Hex.decodeHex(hex); + return new ByteArrayInputStream(b); + } +} diff --git a/sdks/java/io/splunk/src/test/java/org/apache/beam/sdk/io/splunk/SplunkEventTest.java b/sdks/java/io/splunk/src/test/java/org/apache/beam/sdk/io/splunk/SplunkEventTest.java index 29769526d248..749086bac435 100644 --- a/sdks/java/io/splunk/src/test/java/org/apache/beam/sdk/io/splunk/SplunkEventTest.java +++ b/sdks/java/io/splunk/src/test/java/org/apache/beam/sdk/io/splunk/SplunkEventTest.java @@ -20,6 +20,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; +import com.google.gson.JsonObject; import org.junit.Test; /** Unit tests for {@link SplunkEvent} class. */ @@ -34,6 +35,8 @@ public void testEquals() { String source = "test-source"; String sourceType = "test-source-type"; Long time = 123456789L; + JsonObject fields = new JsonObject(); + fields.addProperty("test-key", "test-value"); SplunkEvent actualEvent = SplunkEvent.newBuilder() @@ -43,6 +46,7 @@ public void testEquals() { .withSource(source) .withSourceType(sourceType) .withTime(time) + .withFields(fields) .create(); assertEquals( @@ -53,6 +57,7 @@ public void testEquals() { .withSource(source) .withSourceType(sourceType) .withTime(time) + .withFields(fields) .create(), actualEvent); diff --git a/sdks/java/io/splunk/src/test/java/org/apache/beam/sdk/io/splunk/SplunkEventWriterTest.java b/sdks/java/io/splunk/src/test/java/org/apache/beam/sdk/io/splunk/SplunkEventWriterTest.java index 3633844ab6d2..f4d8c1a5e137 100644 --- a/sdks/java/io/splunk/src/test/java/org/apache/beam/sdk/io/splunk/SplunkEventWriterTest.java +++ b/sdks/java/io/splunk/src/test/java/org/apache/beam/sdk/io/splunk/SplunkEventWriterTest.java @@ -18,6 +18,7 @@ package org.apache.beam.sdk.io.splunk; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -62,6 +63,21 @@ public static void setup() { private MockServerClient mockServerClient; + @Test + public void testMissingURLProtocol() { + assertFalse(SplunkEventWriter.isValidUrlFormat("test-url")); + } + + @Test + public void testInvalidURL() { + assertFalse(SplunkEventWriter.isValidUrlFormat("http://1.2.3")); + } + + @Test + public void testValidURL() { + assertTrue(SplunkEventWriter.isValidUrlFormat("http://test-url")); + } + @Test public void eventWriterMissingURL() { @@ -71,13 +87,51 @@ public void eventWriterMissingURL() { assertTrue(thrown.getMessage().contains("url needs to be provided")); } + @Test + public void eventWriterMissingURLProtocol() { + Exception thrown = + assertThrows( + IllegalArgumentException.class, + () -> SplunkEventWriter.newBuilder().withUrl("test-url").build()); + + assertTrue(thrown.getMessage().contains(SplunkEventWriter.INVALID_URL_FORMAT_MESSAGE)); + } + + /** Test building {@link SplunkEventWriter} with an invalid URL. */ + @Test + public void eventWriterInvalidURL() { + Exception thrown = + assertThrows( + IllegalArgumentException.class, + () -> SplunkEventWriter.newBuilder().withUrl("http://1.2.3").build()); + + assertTrue(thrown.getMessage().contains(SplunkEventWriter.INVALID_URL_FORMAT_MESSAGE)); + } + + /** + * Test building {@link SplunkEventWriter} with the 'services/collector/event' path appended to + * the URL. + */ + @Test + public void eventWriterFullEndpoint() { + Exception thrown = + assertThrows( + IllegalArgumentException.class, + () -> + SplunkEventWriter.newBuilder() + .withUrl("http://test-url:8088/services/collector/event") + .build()); + + assertTrue(thrown.getMessage().contains(SplunkEventWriter.INVALID_URL_FORMAT_MESSAGE)); + } + @Test public void eventWriterMissingToken() { Exception thrown = assertThrows( NullPointerException.class, - () -> SplunkEventWriter.newBuilder().withUrl("test-url").build()); + () -> SplunkEventWriter.newBuilder().withUrl("http://test-url").build()); assertTrue(thrown.getMessage().contains("token needs to be provided")); } @@ -86,7 +140,7 @@ public void eventWriterMissingToken() { public void eventWriterDefaultBatchCountAndValidation() { SplunkEventWriter writer = - SplunkEventWriter.newBuilder().withUrl("test-url").withToken("test-token").build(); + SplunkEventWriter.newBuilder().withUrl("http://test-url").withToken("test-token").build(); assertNull(writer.inputBatchCount()); assertNull(writer.disableCertificateValidation()); @@ -99,7 +153,7 @@ public void eventWriterCustomBatchCountAndValidation() { Boolean certificateValidation = false; SplunkEventWriter writer = SplunkEventWriter.newBuilder() - .withUrl("test-url") + .withUrl("http://test-url") .withToken("test-token") .withInputBatchCount(StaticValueProvider.of(batchCount)) .withDisableCertificateValidation(StaticValueProvider.of(certificateValidation)) @@ -144,7 +198,6 @@ public void successfulSplunkWriteSingleBatchTest() { PCollection<SplunkWriteError> actual = pipeline .apply("Create Input data", Create.of(testEvents)) - // .withCoder(KvCoder.of(BigEndianIntegerCoder.of(), SplunkEventCoder.of()))) .apply( "SplunkEventWriter", ParDo.of( @@ -200,7 +253,6 @@ public void successfulSplunkWriteMultiBatchTest() { PCollection<SplunkWriteError> actual = pipeline .apply("Create Input data", Create.of(testEvents)) - // .withCoder(KvCoder.of(BigEndianIntegerCoder.of(), SplunkEventCoder.of()))) .apply( "SplunkEventWriter", ParDo.of( @@ -246,7 +298,6 @@ public void failedSplunkWriteSingleBatchTest() { PCollection<SplunkWriteError> actual = pipeline .apply("Create Input data", Create.of(testEvents)) - // .withCoder(KvCoder.of(BigEndianIntegerCoder.of(), SplunkEventCoder.of()))) .apply( "SplunkEventWriter", ParDo.of( diff --git a/sdks/java/io/splunk/src/test/java/org/apache/beam/sdk/io/splunk/SplunkIOTest.java b/sdks/java/io/splunk/src/test/java/org/apache/beam/sdk/io/splunk/SplunkIOTest.java index 32c98513ea24..d2cfd59aace2 100644 --- a/sdks/java/io/splunk/src/test/java/org/apache/beam/sdk/io/splunk/SplunkIOTest.java +++ b/sdks/java/io/splunk/src/test/java/org/apache/beam/sdk/io/splunk/SplunkIOTest.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.io.splunk; +import com.google.gson.JsonObject; import java.util.List; import org.apache.beam.sdk.testing.NeedsRunner; import org.apache.beam.sdk.testing.PAssert; @@ -63,7 +64,8 @@ public void successfulSplunkIOMultiBatchNoParallelismTest() { int testPort = mockServerRule.getPort(); String url = Joiner.on(':').join("http://localhost", testPort); String token = "test-token"; - + JsonObject fields = new JsonObject(); + fields.addProperty("customfield", 1); List<SplunkEvent> testEvents = ImmutableList.of( SplunkEvent.newBuilder() @@ -73,6 +75,7 @@ public void successfulSplunkIOMultiBatchNoParallelismTest() { .withSource("test-source-1") .withSourceType("test-source-type-1") .withTime(12345L) + .withFields(fields) .create(), SplunkEvent.newBuilder() .withEvent("test-event-2") @@ -81,11 +84,12 @@ public void successfulSplunkIOMultiBatchNoParallelismTest() { .withSource("test-source-2") .withSourceType("test-source-type-2") .withTime(12345L) + .withFields(fields) .create()); PCollection<SplunkWriteError> actual = pipeline - .apply("Create Input data", Create.of(testEvents)) // .withCoder(SplunkEventCoder.of())) + .apply("Create Input data", Create.of(testEvents)) .apply( "SplunkIO", SplunkIO.write(url, token).withParallelism(1).withBatchCount(testEvents.size())); @@ -132,7 +136,7 @@ public void successfulSplunkIOMultiBatchParallelismTest() { PCollection<SplunkWriteError> actual = pipeline - .apply("Create Input data", Create.of(testEvents)) // .withCoder(SplunkEventCoder.of())) + .apply("Create Input data", Create.of(testEvents)) .apply( "SplunkIO", SplunkIO.write(url, token) @@ -182,7 +186,7 @@ public void successfulSplunkIOSingleBatchParallelismTest() { PCollection<SplunkWriteError> actual = pipeline - .apply("Create Input data", Create.of(testEvents)) // .withCoder(SplunkEventCoder.of())) + .apply("Create Input data", Create.of(testEvents)) .apply( "SplunkIO", SplunkIO.write(url, token).withParallelism(testParallelism).withBatchCount(1)); From fd1b87849c05bb0e4092f80a574cbb7ab3dc667c Mon Sep 17 00:00:00 2001 From: Vitaly Terentyev <vitaly.terentyev@akvelon.com> Date: Sat, 21 Oct 2023 00:21:42 +0400 Subject: [PATCH 215/435] Refactor prepare arguments step for Github actions (#29090) * Refactor prepare arguments step * Rename perf/load/ pipeline options folders --- .../beam_CloudML_Benchmarks_Dataflow.yml | 2 +- ...m_Inference_Python_Benchmarks_Dataflow.yml | 10 ++--- .../beam_Java_LoadTests_Combine_Smoke.yml | 6 +-- ...beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml | 8 ++-- .../beam_LoadTests_Go_CoGBK_Flink_batch.yml | 6 +-- ...am_LoadTests_Go_Combine_Dataflow_Batch.yml | 31 +++++++-------- .../beam_LoadTests_Go_Combine_Flink_Batch.yml | 6 +-- .../beam_LoadTests_Go_GBK_Dataflow_Batch.yml | 14 +++---- .../beam_LoadTests_Go_GBK_Flink_Batch.yml | 12 +++--- ...beam_LoadTests_Go_ParDo_Dataflow_Batch.yml | 8 ++-- .../beam_LoadTests_Go_ParDo_Flink_Batch.yml | 8 ++-- ..._LoadTests_Go_SideInput_Dataflow_Batch.yml | 4 +- ...eam_LoadTests_Go_SideInput_Flink_Batch.yml | 4 +- ...am_LoadTests_Java_CoGBK_Dataflow_Batch.yml | 8 ++-- ...oadTests_Java_CoGBK_Dataflow_Streaming.yml | 33 ++++++++-------- ...a_CoGBK_Dataflow_V2_Batch_JavaVersions.yml | 8 ++-- ...GBK_Dataflow_V2_Streaming_JavaVersions.yml | 8 ++-- ...a_CoGBK_SparkStructuredStreaming_Batch.yml | 8 ++-- ..._LoadTests_Java_Combine_Dataflow_Batch.yml | 6 +-- ...dTests_Java_Combine_Dataflow_Streaming.yml | 10 ++--- ...Combine_SparkStructuredStreaming_Batch.yml | 10 ++--- ...beam_LoadTests_Java_GBK_Dataflow_Batch.yml | 14 +++---- ..._LoadTests_Java_GBK_Dataflow_Streaming.yml | 14 +++---- ...ests_Java_GBK_Dataflow_V2_Batch_Java11.yml | 14 +++---- ...ests_Java_GBK_Dataflow_V2_Batch_Java17.yml | 14 +++---- ..._Java_GBK_Dataflow_V2_Streaming_Java11.yml | 14 +++---- ..._Java_GBK_Dataflow_V2_Streaming_Java17.yml | 14 +++---- .../beam_LoadTests_Java_GBK_Smoke.yml | 8 ++-- ...ava_GBK_SparkStructuredStreaming_Batch.yml | 14 +++---- ...am_LoadTests_Java_ParDo_Dataflow_Batch.yml | 13 +++---- ...oadTests_Java_ParDo_Dataflow_Streaming.yml | 13 +++---- ...a_ParDo_Dataflow_V2_Batch_JavaVersions.yml | 8 ++-- ...rDo_Dataflow_V2_Streaming_JavaVersions.yml | 8 ++-- ...a_ParDo_SparkStructuredStreaming_Batch.yml | 13 +++---- ..._LoadTests_Python_CoGBK_Dataflow_Batch.yml | 8 ++-- ...dTests_Python_CoGBK_Dataflow_Streaming.yml | 8 ++-- ...eam_LoadTests_Python_CoGBK_Flink_Batch.yml | 6 +-- ...oadTests_Python_Combine_Dataflow_Batch.yml | 31 +++++++-------- ...ests_Python_Combine_Dataflow_Streaming.yml | 6 +-- ...m_LoadTests_Python_Combine_Flink_Batch.yml | 6 +-- ...adTests_Python_Combine_Flink_Streaming.yml | 4 +- ...ests_Python_FnApiRunner_Microbenchmark.yml | 2 +- ...am_LoadTests_Python_GBK_Dataflow_Batch.yml | 10 ++--- ...oadTests_Python_GBK_Dataflow_Streaming.yml | 2 +- .../beam_LoadTests_Python_GBK_Flink_Batch.yml | 10 ++--- ...ts_Python_GBK_reiterate_Dataflow_Batch.yml | 4 +- ...ython_GBK_reiterate_Dataflow_Streaming.yml | 4 +- ..._LoadTests_Python_ParDo_Dataflow_Batch.yml | 8 ++-- ...dTests_Python_ParDo_Dataflow_Streaming.yml | 8 ++-- ...eam_LoadTests_Python_ParDo_Flink_Batch.yml | 6 +-- ...LoadTests_Python_ParDo_Flink_Streaming.yml | 10 ++--- ...dTests_Python_SideInput_Dataflow_Batch.yml | 20 +++++----- .../workflows/beam_LoadTests_Python_Smoke.yml | 4 +- .../beam_PerformanceTests_AvroIOIT.yml | 4 +- .../beam_PerformanceTests_AvroIOIT_HDFS.yml | 4 +- ...rmanceTests_BigQueryIO_Batch_Java_Avro.yml | 20 +++++----- ...rmanceTests_BigQueryIO_Batch_Java_Json.yml | 20 +++++----- ...ormanceTests_BigQueryIO_Streaming_Java.yml | 20 +++++----- ...erformanceTests_BiqQueryIO_Read_Python.yml | 4 +- ...nceTests_BiqQueryIO_Write_Python_Batch.yml | 4 +- .../workflows/beam_PerformanceTests_Cdap.yml | 4 +- ...m_PerformanceTests_Compressed_TextIOIT.yml | 2 +- ...formanceTests_Compressed_TextIOIT_HDFS.yml | 2 +- .../beam_PerformanceTests_HadoopFormat.yml | 4 +- .../workflows/beam_PerformanceTests_JDBC.yml | 4 +- .../beam_PerformanceTests_Kafka_IO.yml | 7 ++-- ...am_PerformanceTests_ManyFiles_TextIOIT.yml | 4 +- ...rformanceTests_ManyFiles_TextIOIT_HDFS.yml | 4 +- .../beam_PerformanceTests_MongoDBIO_IT.yml | 4 +- .../beam_PerformanceTests_ParquetIOIT.yml | 2 +- ...beam_PerformanceTests_ParquetIOIT_HDFS.yml | 2 +- ...manceTests_PubsubIOIT_Python_Streaming.yml | 4 +- ...formanceTests_SQLBigQueryIO_Batch_Java.yml | 4 +- ...ormanceTests_SpannerIO_Read_2GB_Python.yml | 4 +- ...Tests_SpannerIO_Write_2GB_Python_Batch.yml | 4 +- ...beam_PerformanceTests_SparkReceiver_IO.yml | 4 +- .../beam_PerformanceTests_TFRecordIOIT.yml | 4 +- ...eam_PerformanceTests_TFRecordIOIT_HDFS.yml | 4 +- .../beam_PerformanceTests_TextIOIT.yml | 2 +- .../beam_PerformanceTests_TextIOIT_HDFS.yml | 2 +- .../beam_PerformanceTests_TextIOIT_Python.yml | 2 +- ...rmanceTests_WordCountIT_PythonVersions.yml | 4 +- .../beam_PerformanceTests_XmlIOIT.yml | 4 +- .../beam_PerformanceTests_XmlIOIT_HDFS.yml | 4 +- ..._PerformanceTests_xlang_KafkaIO_Python.yml | 2 +- ..._CloudML_Benchmarks_Dataflow_arguments.txt | 0 ...rch_Imagenet_Classification_Resnet_152.txt | 0 ...Classification_Resnet_152_Tesla_T4_GPU.txt | 0 ...ch_Language_Modeling_Bert_Base_Uncased.txt | 0 ...h_Language_Modeling_Bert_Large_Uncased.txt | 0 ...torch_Vision_Classification_Resnet_101.txt | 0 .../go_CoGBK_Dataflow_Batch_MultipleKey.txt} | 0 ...CoGBK_Dataflow_Batch_Reiteration_10KB.txt} | 0 ..._CoGBK_Dataflow_Batch_Reiteration_2MB.txt} | 0 .../go_CoGBK_Dataflow_Batch_SingleKey.txt} | 0 .../go_CoGBK_Flink_Batch_MultipleKey.txt | 0 .../go_CoGBK_Flink_Batch_Reiteration_10KB.txt | 0 .../go_CoGBK_Flink_Batch_Reiteration_2MB.txt | 0 .../go_Combine_Dataflow_Batch_10b.txt} | 9 +---- .../go_Combine_Dataflow_Batch_Fanout_4.txt} | 9 +---- .../go_Combine_Dataflow_Batch_Fanout_8.txt} | 9 +---- .../go_Combine_Flink_Batch_10b.txt | 0 .../go_Combine_Flink_Batch_Fanout_4.txt | 0 .../go_Combine_Flink_Batch_Fanout_8.txt | 0 .../go_GBK_Dataflow_Batch_100b.txt | 0 .../go_GBK_Dataflow_Batch_100kb.txt | 0 .../go_GBK_Dataflow_Batch_10b.txt | 0 .../go_GBK_Dataflow_Batch_Fanout_4.txt | 0 .../go_GBK_Dataflow_Batch_Fanout_8.txt | 0 ...go_GBK_Dataflow_Batch_Reiteration_10KB.txt | 0 .../go_GBK_Dataflow_Batch_Reiteration_2MB.txt | 0 .../go_GBK_Flink_Batch_100b.txt | 0 .../go_GBK_Flink_Batch_100kb.txt | 0 .../go_GBK_Flink_Batch_10b.txt | 0 .../go_GBK_Flink_Batch_Fanout_4.txt | 0 .../go_GBK_Flink_Batch_Fanout_8.txt | 0 .../go_GBK_Flink_Batch_Reiteration_10KB.txt | 0 .../go_ParDo_Dataflow_Batch_100_Counters.txt} | 0 .../go_ParDo_Dataflow_Batch_10_Counters.txt} | 0 ...go_ParDo_Dataflow_Batch_10_Iterations.txt} | 0 ...o_ParDo_Dataflow_Batch_200_Iterations.txt} | 0 .../go_ParDo_Flink_Batch_100_counters.txt | 0 .../go_ParDo_Flink_Batch_10_counters.txt | 0 .../go_ParDo_Flink_Batch_10_times.txt | 0 .../go_ParDo_Flink_Batch_200_times.txt | 0 ...deInput_Dataflow_Batch_First_Iterable.txt} | 0 .../go_SideInput_Dataflow_Batch_Iterable.txt} | 0 ...o_SideInput_Flink_Batch_First_Iterable.txt | 0 .../go_SideInput_Flink_Batch_Iterable.txt | 0 ...oGBK_Dataflow_Batch_100b_Multiple_Keys.txt | 0 ...a_CoGBK_Dataflow_Batch_100b_Single_Key.txt | 0 .../java_CoGBK_Dataflow_Batch_10kB.txt | 0 .../java_CoGBK_Dataflow_Batch_2MB.txt | 0 ...BK_Dataflow_Streaming_2GB_MultipleKey.txt} | 8 +--- ...taflow_Streaming_2GB_Reiteration_10KB.txt} | 8 +--- ...ataflow_Streaming_2GB_Reiteration_2MB.txt} | 8 +--- ...oGBK_Dataflow_Streaming_2GB_SingleKey.txt} | 8 +--- ...w_V2_Streaming_Java_100b_Multiple_Keys.txt | 0 ...flow_V2_Streaming_Java_100b_Single_Key.txt | 0 ..._CoGBK_Dataflow_V2_Streaming_Java_10kB.txt | 0 ...a_CoGBK_Dataflow_V2_Streaming_Java_2MB.txt | 0 ...uredStreaming_Batch_100b_Multiple_Keys.txt | 0 ...ucturedStreaming_Batch_100b_Single_Key.txt | 0 ...BK_SparkStructuredStreaming_Batch_10kB.txt | 0 ...GBK_SparkStructuredStreaming_Batch_2MB.txt | 0 .../java_Combine_Dataflow_Batch_10b.txt} | 0 .../java_Combine_Dataflow_Batch_Fanout_4.txt} | 0 .../java_Combine_Dataflow_Batch_Fanout_8.txt} | 0 .../java_Combine_Dataflow_Streaming_10b.txt | 0 ...va_Combine_Dataflow_Streaming_Fanout_4.txt | 0 ...va_Combine_Dataflow_Streaming_Fanout_8.txt | 0 ...ine_SparkStructuredStreaming_Batch_10b.txt | 0 ...parkStructuredStreaming_Batch_Fanout_4.txt | 0 ...parkStructuredStreaming_Batch_Fanout_8.txt | 0 ...GBK_Dataflow_Batch_2GB_of_100B_records.txt | 0 ...BK_Dataflow_Batch_2GB_of_100kB_records.txt | 0 ..._GBK_Dataflow_Batch_2GB_of_10B_records.txt | 0 ...4_times_with_2GB_10-byte_records_total.txt | 0 ...8_times_with_2GB_10-byte_records_total.txt | 0 ...ow_Batch_reiterate_4_times_10kB_values.txt | 0 ...low_Batch_reiterate_4_times_2MB_values.txt | 0 ...Dataflow_Streaming_2GB_of_100B_records.txt | 0 ...ataflow_Streaming_2GB_of_100kB_records.txt | 0 ..._Dataflow_Streaming_2GB_of_10B_records.txt | 0 ...4_times_with_2GB_10-byte_records_total.txt | 0 ...8_times_with_2GB_10-byte_records_total.txt | 0 ...treaming_reiterate_4_times_10kB_values.txt | 0 ...Streaming_reiterate_4_times_2MB_values.txt | 0 ...ow_V2_Batch_Java11_2GB_of_100B_records.txt | 0 ...w_V2_Batch_Java11_2GB_of_100kB_records.txt | 0 ...low_V2_Batch_Java11_2GB_of_10B_records.txt | 0 ...4_times_with_2GB_10-byte_records_total.txt | 0 ...8_times_with_2GB_10-byte_records_total.txt | 0 ...h_Java11_reiterate_4_times_10kB_values.txt | 0 ...ch_Java11_reiterate_4_times_2MB_values.txt | 0 ...ow_V2_Batch_Java17_2GB_of_100B_records.txt | 0 ...w_V2_Batch_Java17_2GB_of_100kB_records.txt | 0 ...low_V2_Batch_Java17_2GB_of_10B_records.txt | 0 ...4_times_with_2GB_10-byte_records_total.txt | 0 ...8_times_with_2GB_10-byte_records_total.txt | 0 ...h_Java17_reiterate_4_times_10kB_values.txt | 0 ...ch_Java17_reiterate_4_times_2MB_values.txt | 0 ...2_Streaming_Java11_2GB_of_100B_records.txt | 0 ..._Streaming_Java11_2GB_of_100kB_records.txt | 0 ...V2_Streaming_Java11_2GB_of_10B_records.txt | 0 ...4_times_with_2GB_10-byte_records_total.txt | 0 ...8_times_with_2GB_10-byte_records_total.txt | 0 ...g_Java11_reiterate_4_times_10kB_values.txt | 0 ...ng_Java11_reiterate_4_times_2MB_values.txt | 0 ...2_Streaming_Java17_2GB_of_100B_records.txt | 0 ..._Streaming_Java17_2GB_of_100kB_records.txt | 0 ...V2_Streaming_Java17_2GB_of_10B_records.txt | 0 ...4_times_with_2GB_10-byte_records_total.txt | 0 ...8_times_with_2GB_10-byte_records_total.txt | 0 ...g_Java17_reiterate_4_times_10kB_values.txt | 0 ...ng_Java17_reiterate_4_times_2MB_values.txt | 0 ...edStreaming_Batch_2GB_of_100B_records.txt} | 0 ...dStreaming_Batch_2GB_of_100kB_records.txt} | 0 ...redStreaming_Batch_2GB_of_10B_records.txt} | 0 ..._times_with_2GB_10-byte_records_total.txt} | 0 ..._times_with_2GB_10-byte_records_total.txt} | 0 ...g_Batch_reiterate_4_times_10kB_values.txt} | 0 ...ng_Batch_reiterate_4_times_2MB_values.txt} | 0 ...e_CombineLoadTest_load_test_Dataflow-1.txt | 0 ...e_CombineLoadTest_load_test_Dataflow-2.txt | 0 ...e_CombineLoadTest_load_test_Dataflow-3.txt | 0 ...java_ParDo_Dataflow_Batch_100_counters.txt | 0 .../java_ParDo_Dataflow_Batch_10_counters.txt | 0 .../java_ParDo_Dataflow_Batch_10_times.txt | 0 .../java_ParDo_Dataflow_Batch_200_times.txt | 0 ..._ParDo_Dataflow_Streaming_100_counters.txt | 0 ...a_ParDo_Dataflow_Streaming_10_counters.txt | 0 ...java_ParDo_Dataflow_Streaming_10_times.txt | 0 ...ava_ParDo_Dataflow_Streaming_200_times.txt | 0 ...low_V2_Batch_JavaVersions_100_counters.txt | 0 ...flow_V2_Batch_JavaVersions_10_counters.txt | 0 ...ataflow_V2_Batch_JavaVersions_10_times.txt | 0 ...taflow_V2_Batch_JavaVersions_200_times.txt | 0 ...V2_Streaming_JavaVersions_100_counters.txt | 0 ..._V2_Streaming_JavaVersions_10_counters.txt | 0 ...low_V2_Streaming_JavaVersions_10_times.txt | 0 ...ow_V2_Streaming_JavaVersions_200_times.txt | 0 ...StructuredStreaming_Batch_100_counters.txt | 0 ...kStructuredStreaming_Batch_10_counters.txt | 0 ...parkStructuredStreaming_Batch_10_times.txt | 0 ...arkStructuredStreaming_Batch_200_times.txt | 0 .../java_Smoke_GroupByKey_Dataflow.txt | 0 .../java_Smoke_GroupByKey_Direct.txt | 0 .../java_Smoke_GroupByKey_Flink.txt | 0 .../java_Smoke_GroupByKey_Spark.txt | 0 ...oGBK_Dataflow_Batch_100b_Multiple_Keys.txt | 0 ...n_CoGBK_Dataflow_Batch_100b_Single_Key.txt | 0 .../python_CoGBK_Dataflow_Batch_10kB.txt | 0 .../python_CoGBK_Dataflow_Batch_2MB.txt | 0 ...ataflow_Flink_Batch_100b_Multiple_Keys.txt | 0 ...K_Dataflow_Flink_Batch_100b_Single_Key.txt | 0 ...python_CoGBK_Dataflow_Flink_Batch_10kB.txt | 0 ..._Dataflow_Streaming_100b_Multiple_Keys.txt | 0 ...GBK_Dataflow_Streaming_100b_Single_Key.txt | 0 .../python_CoGBK_Dataflow_Streaming_10kB.txt | 0 .../python_CoGBK_Dataflow_Streaming_2MB.txt | 0 ...python_Combine_Dataflow_Batch_2GB_10b.txt} | 9 +---- ...n_Combine_Dataflow_Batch_2GB_Fanout_4.txt} | 7 +--- ...n_Combine_Dataflow_Batch_2GB_Fanout_8.txt} | 7 +--- ...Dataflow_Streaming_2GB_10_byte_records.txt | 0 ...ombine_Dataflow_Streaming_2GB_Fanout_4.txt | 0 ...ombine_Dataflow_Streaming_2GB_Fanout_8.txt | 0 ...ombine_Flink_Batch_2GB_10_byte_records.txt | 0 ...ython_Combine_Flink_Batch_2GB_Fanout_4.txt | 0 ...ython_Combine_Flink_Batch_2GB_Fanout_8.txt | 0 ...ne_Flink_Streaming_2GB_10_byte_records.txt | 0 ...n_Combine_Flink_Streaming_2GB_Fanout_4.txt | 0 ...n_Combine_Flink_Streaming_2GB_Fanout_8.txt | 0 .../python_FnApiRunner_Microbenchmark.txt} | 0 ...GBK_Dataflow_Batch_2GB_of_100B_records.txt | 0 ...BK_Dataflow_Batch_2GB_of_100kB_records.txt | 0 ..._GBK_Dataflow_Batch_2GB_of_10B_records.txt | 0 ...4_times_with_2GB_10-byte_records_total.txt | 0 ...8_times_with_2GB_10-byte_records_total.txt | 0 ...Dataflow_Streaming_2GB_of_100B_records.txt | 0 ...ataflow_Streaming_2GB_of_100kB_records.txt | 0 ..._Dataflow_Streaming_2GB_of_10B_records.txt | 0 ...4_times_with_2GB_10-byte_records_total.txt | 0 ...8_times_with_2GB_10-byte_records_total.txt | 0 ...on_GBK_Flink_Batch_2GB_of_100B_records.txt | 0 ...hon_GBK_Flink_Batch_2GB_of_10B_records.txt | 0 ...4_times_with_2GB_10-byte_records_total.txt | 0 ...8_times_with_2GB_10-byte_records_total.txt | 0 ...nk_Batch_reiterate_4_times_10kB_values.txt | 0 ...ow_Batch_reiterate_4_times_10kB_values.txt | 0 ...low_Batch_reiterate_4_times_2MB_values.txt | 0 ...treaming_reiterate_4_times_10kB_values.txt | 0 ...Streaming_reiterate_4_times_2MB_values.txt | 0 ...thon_ParDo_Dataflow_Batch_100_Counters.txt | 0 ...ython_ParDo_Dataflow_Batch_10_Counters.txt | 0 ...hon_ParDo_Dataflow_Batch_10_Iterations.txt | 0 ...on_ParDo_Dataflow_Batch_200_Iterations.txt | 0 ..._ParDo_Dataflow_Streaming_100_Counters.txt | 0 ...n_ParDo_Dataflow_Streaming_10_Counters.txt | 0 ...ParDo_Dataflow_Streaming_10_Iterations.txt | 0 ...arDo_Dataflow_Streaming_200_Iterations.txt | 0 .../python_ParDo_Flink_Batch_10_Counters.txt | 0 ...python_ParDo_Flink_Batch_10_Iterations.txt | 0 ...ython_ParDo_Flink_Batch_200_Iterations.txt | 0 ...hon_ParDo_Flink_Streaming_100_Counters.txt | 0 ...thon_ParDo_Flink_Streaming_10_Counters.txt | 0 ...on_ParDo_Flink_Streaming_10_Iterations.txt | 0 ...n_ParDo_Flink_Streaming_200_Iterations.txt | 0 ...hon_ParDo_Flink_Streaming_5_Iterations.txt | 0 ...w_Batch_10gb_1000window_first_iterable.txt | 0 ...ataflow_Batch_10gb_1000window_iterable.txt | 0 ...flow_Batch_10gb_1window_first_iterable.txt | 0 ...t_Dataflow_Batch_10gb_1window_iterable.txt | 0 ...Batch_1gb_1000window_1key_percent_dict.txt | 0 ...atch_1gb_1000window_99key_percent_dict.txt | 0 ...ow_Batch_1gb_1window_1key_percent_dict.txt | 0 ...w_Batch_1gb_1window_99key_percent_dict.txt | 0 ..._Dataflow_Batch_1gb_1window_first_list.txt | 0 ...eInput_Dataflow_Batch_1gb_1window_list.txt | 0 .../python_Smoke_GroupByKey_Dataflow.txt | 0 .../python_Smoke_GroupByKey_Direct.txt | 0 .../config_BigQueryIO_Batch_Java_Avro.txt | 38 ------------------ .../config_BigQueryIO_Batch_Java_Json.txt | 38 ------------------ .../config_BigQueryIO_Streaming_Java.txt | 39 ------------------- .../JDBC.txt | 0 .../SQLBigQueryIO_Batch_Java.txt | 0 .../TFRecordIOIT.txt} | 0 .../TFRecordIOIT_HDFS.txt | 0 .../avroIOIT.txt} | 0 .../avroIOIT_HDFS.txt} | 0 .../bigQueryIO_Batch_Java_Avro.txt | 29 ++++++++++++++ .../bigQueryIO_Batch_Java_Json.txt | 29 ++++++++++++++ .../bigQueryIO_Streaming_Java.txt | 30 ++++++++++++++ .../biqQueryIO_Read_Python.txt | 0 .../biqQueryIO_Write_Python_Batch.txt | 0 .../cdap.txt | 0 .../compressed_TextIOIT.txt} | 0 .../compressed_TextIOIT_HDFS.txt} | 0 .../hadoopFormat.txt | 0 .../kafka_IO_Batch.txt | 0 .../kafka_IO_Streaming.txt | 0 .../manyFiles_TextIOIT.txt} | 0 .../manyFiles_TextIOIT_HDFS.txt} | 0 .../mongoDBIO_IT.txt | 0 .../parquetIOIT.txt} | 0 .../parquetIOIT_HDFS.txt} | 0 .../pubsubIOIT_Python_Streaming.txt | 0 .../spannerIO_Read_2GB_Python.txt | 0 .../spannerIO_Write_2GB_Python.txt | 0 .../sparkReceiver_IO.txt | 0 .../textIOIT.txt | 0 .../textIOIT_HDFS.txt | 0 .../textIOIT_Python.txt | 0 .../wordCountIT_Python.txt | 0 .../xlang_KafkaIO_Python.txt | 0 .../xmlIOIT.txt} | 0 .../xmlIOIT_HDFS.txt} | 0 337 files changed, 446 insertions(+), 530 deletions(-) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/beam_CloudML_Benchmarks_Dataflow_arguments.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Imagenet_Classification_Resnet_152.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Imagenet_Classification_Resnet_152_Tesla_T4_GPU.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Language_Modeling_Bert_Base_Uncased.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Language_Modeling_Bert_Large_Uncased.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Vision_Classification_Resnet_101.txt (100%) rename .github/workflows/{load-tests-job-configs/config_CoGBK_Go_Batch_MultipleKey.txt => load-tests-pipeline-options/go_CoGBK_Dataflow_Batch_MultipleKey.txt} (100%) rename .github/workflows/{load-tests-job-configs/config_CoGBK_Go_Batch_Reiteration_10KB.txt => load-tests-pipeline-options/go_CoGBK_Dataflow_Batch_Reiteration_10KB.txt} (100%) rename .github/workflows/{load-tests-job-configs/config_CoGBK_Go_Batch_Reiteration_2MB.txt => load-tests-pipeline-options/go_CoGBK_Dataflow_Batch_Reiteration_2MB.txt} (100%) rename .github/workflows/{load-tests-job-configs/config_CoGBK_Go_Batch_SingleKey.txt => load-tests-pipeline-options/go_CoGBK_Dataflow_Batch_SingleKey.txt} (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/go_CoGBK_Flink_Batch_MultipleKey.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/go_CoGBK_Flink_Batch_Reiteration_10KB.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/go_CoGBK_Flink_Batch_Reiteration_2MB.txt (100%) rename .github/workflows/{load-tests-job-configs/config_Combine_Go_Batch_10b.txt => load-tests-pipeline-options/go_Combine_Dataflow_Batch_10b.txt} (78%) rename .github/workflows/{load-tests-job-configs/config_Combine_Go_Batch_Fanout_4.txt => load-tests-pipeline-options/go_Combine_Dataflow_Batch_Fanout_4.txt} (78%) rename .github/workflows/{load-tests-job-configs/config_Combine_Go_Batch_Fanout_8.txt => load-tests-pipeline-options/go_Combine_Dataflow_Batch_Fanout_8.txt} (78%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/go_Combine_Flink_Batch_10b.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/go_Combine_Flink_Batch_Fanout_4.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/go_Combine_Flink_Batch_Fanout_8.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/go_GBK_Dataflow_Batch_100b.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/go_GBK_Dataflow_Batch_100kb.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/go_GBK_Dataflow_Batch_10b.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/go_GBK_Dataflow_Batch_Fanout_4.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/go_GBK_Dataflow_Batch_Fanout_8.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/go_GBK_Dataflow_Batch_Reiteration_10KB.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/go_GBK_Dataflow_Batch_Reiteration_2MB.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/go_GBK_Flink_Batch_100b.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/go_GBK_Flink_Batch_100kb.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/go_GBK_Flink_Batch_10b.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/go_GBK_Flink_Batch_Fanout_4.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/go_GBK_Flink_Batch_Fanout_8.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/go_GBK_Flink_Batch_Reiteration_10KB.txt (100%) rename .github/workflows/{load-tests-job-configs/config_ParDo_Go_Batch_100_Counters.txt => load-tests-pipeline-options/go_ParDo_Dataflow_Batch_100_Counters.txt} (100%) rename .github/workflows/{load-tests-job-configs/config_ParDo_Go_Batch_10_Counters.txt => load-tests-pipeline-options/go_ParDo_Dataflow_Batch_10_Counters.txt} (100%) rename .github/workflows/{load-tests-job-configs/config_ParDo_Go_Batch_10_Iterations.txt => load-tests-pipeline-options/go_ParDo_Dataflow_Batch_10_Iterations.txt} (100%) rename .github/workflows/{load-tests-job-configs/config_ParDo_Go_Batch_200_Iterations.txt => load-tests-pipeline-options/go_ParDo_Dataflow_Batch_200_Iterations.txt} (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/go_ParDo_Flink_Batch_100_counters.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/go_ParDo_Flink_Batch_10_counters.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/go_ParDo_Flink_Batch_10_times.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/go_ParDo_Flink_Batch_200_times.txt (100%) rename .github/workflows/{load-tests-job-configs/config_SideInput_Go_Batch_Dataflow_First_Iterable.txt => load-tests-pipeline-options/go_SideInput_Dataflow_Batch_First_Iterable.txt} (100%) rename .github/workflows/{load-tests-job-configs/config_SideInput_Go_Batch_Dataflow_Iterable.txt => load-tests-pipeline-options/go_SideInput_Dataflow_Batch_Iterable.txt} (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/go_SideInput_Flink_Batch_First_Iterable.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/go_SideInput_Flink_Batch_Iterable.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_CoGBK_Dataflow_Batch_100b_Multiple_Keys.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_CoGBK_Dataflow_Batch_100b_Single_Key.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_CoGBK_Dataflow_Batch_10kB.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_CoGBK_Dataflow_Batch_2MB.txt (100%) rename .github/workflows/{load-tests-job-configs/config_CoGBK_Java_Streaming_2GB_MultipleKey.txt => load-tests-pipeline-options/java_CoGBK_Dataflow_Streaming_2GB_MultipleKey.txt} (81%) rename .github/workflows/{load-tests-job-configs/config_CoGBK_Java_Streaming_2GB_Reiteration_10KB.txt => load-tests-pipeline-options/java_CoGBK_Dataflow_Streaming_2GB_Reiteration_10KB.txt} (81%) rename .github/workflows/{load-tests-job-configs/config_CoGBK_Java_Streaming_2GB_Reiteration_2MB.txt => load-tests-pipeline-options/java_CoGBK_Dataflow_Streaming_2GB_Reiteration_2MB.txt} (81%) rename .github/workflows/{load-tests-job-configs/config_CoGBK_Java_Streaming_2GB_SingleKey.txt => load-tests-pipeline-options/java_CoGBK_Dataflow_Streaming_2GB_SingleKey.txt} (81%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_CoGBK_Dataflow_V2_Streaming_Java_100b_Multiple_Keys.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_CoGBK_Dataflow_V2_Streaming_Java_100b_Single_Key.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_CoGBK_Dataflow_V2_Streaming_Java_10kB.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_CoGBK_Dataflow_V2_Streaming_Java_2MB.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_CoGBK_SparkStructuredStreaming_Batch_100b_Multiple_Keys.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_CoGBK_SparkStructuredStreaming_Batch_100b_Single_Key.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_CoGBK_SparkStructuredStreaming_Batch_10kB.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_CoGBK_SparkStructuredStreaming_Batch_2MB.txt (100%) rename .github/workflows/{load-tests-job-configs/config_Combine_Java_Dataflow_Batch_10b.txt => load-tests-pipeline-options/java_Combine_Dataflow_Batch_10b.txt} (100%) rename .github/workflows/{load-tests-job-configs/config_Combine_Java_Dataflow_Batch_Fanout_4.txt => load-tests-pipeline-options/java_Combine_Dataflow_Batch_Fanout_4.txt} (100%) rename .github/workflows/{load-tests-job-configs/config_Combine_Java_Dataflow_Batch_Fanout_8.txt => load-tests-pipeline-options/java_Combine_Dataflow_Batch_Fanout_8.txt} (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_Combine_Dataflow_Streaming_10b.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_Combine_Dataflow_Streaming_Fanout_4.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_Combine_Dataflow_Streaming_Fanout_8.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_Combine_SparkStructuredStreaming_Batch_10b.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_Combine_SparkStructuredStreaming_Batch_Fanout_4.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_Combine_SparkStructuredStreaming_Batch_Fanout_8.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_GBK_Dataflow_Batch_2GB_of_100B_records.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_GBK_Dataflow_Batch_2GB_of_100kB_records.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_GBK_Dataflow_Batch_2GB_of_10B_records.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_GBK_Dataflow_Batch_fanout_4_times_with_2GB_10-byte_records_total.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_GBK_Dataflow_Batch_fanout_8_times_with_2GB_10-byte_records_total.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_GBK_Dataflow_Batch_reiterate_4_times_10kB_values.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_GBK_Dataflow_Batch_reiterate_4_times_2MB_values.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_GBK_Dataflow_Streaming_2GB_of_100B_records.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_GBK_Dataflow_Streaming_2GB_of_100kB_records.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_GBK_Dataflow_Streaming_2GB_of_10B_records.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_GBK_Dataflow_Streaming_fanout_4_times_with_2GB_10-byte_records_total.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_GBK_Dataflow_Streaming_fanout_8_times_with_2GB_10-byte_records_total.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_GBK_Dataflow_Streaming_reiterate_4_times_10kB_values.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_GBK_Dataflow_Streaming_reiterate_4_times_2MB_values.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_GBK_Dataflow_V2_Batch_Java11_2GB_of_100B_records.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_GBK_Dataflow_V2_Batch_Java11_2GB_of_100kB_records.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_GBK_Dataflow_V2_Batch_Java11_2GB_of_10B_records.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_GBK_Dataflow_V2_Batch_Java11_fanout_4_times_with_2GB_10-byte_records_total.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_GBK_Dataflow_V2_Batch_Java11_fanout_8_times_with_2GB_10-byte_records_total.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_GBK_Dataflow_V2_Batch_Java11_reiterate_4_times_10kB_values.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_GBK_Dataflow_V2_Batch_Java11_reiterate_4_times_2MB_values.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_GBK_Dataflow_V2_Batch_Java17_2GB_of_100B_records.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_GBK_Dataflow_V2_Batch_Java17_2GB_of_100kB_records.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_GBK_Dataflow_V2_Batch_Java17_2GB_of_10B_records.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_GBK_Dataflow_V2_Batch_Java17_fanout_4_times_with_2GB_10-byte_records_total.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_GBK_Dataflow_V2_Batch_Java17_fanout_8_times_with_2GB_10-byte_records_total.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_GBK_Dataflow_V2_Batch_Java17_reiterate_4_times_10kB_values.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_GBK_Dataflow_V2_Batch_Java17_reiterate_4_times_2MB_values.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_GBK_Dataflow_V2_Streaming_Java11_2GB_of_100B_records.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_GBK_Dataflow_V2_Streaming_Java11_2GB_of_100kB_records.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_GBK_Dataflow_V2_Streaming_Java11_2GB_of_10B_records.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_GBK_Dataflow_V2_Streaming_Java11_fanout_4_times_with_2GB_10-byte_records_total.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_GBK_Dataflow_V2_Streaming_Java11_fanout_8_times_with_2GB_10-byte_records_total.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_GBK_Dataflow_V2_Streaming_Java11_reiterate_4_times_10kB_values.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_GBK_Dataflow_V2_Streaming_Java11_reiterate_4_times_2MB_values.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_GBK_Dataflow_V2_Streaming_Java17_2GB_of_100B_records.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_GBK_Dataflow_V2_Streaming_Java17_2GB_of_100kB_records.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_GBK_Dataflow_V2_Streaming_Java17_2GB_of_10B_records.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_GBK_Dataflow_V2_Streaming_Java17_fanout_4_times_with_2GB_10-byte_records_total.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_GBK_Dataflow_V2_Streaming_Java17_fanout_8_times_with_2GB_10-byte_records_total.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_GBK_Dataflow_V2_Streaming_Java17_reiterate_4_times_10kB_values.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_GBK_Dataflow_V2_Streaming_Java17_reiterate_4_times_2MB_values.txt (100%) rename .github/workflows/{load-tests-job-configs/Java_GBK_SparkStructuredStreaming_Batch_2GB_of_100B_records.txt => load-tests-pipeline-options/java_GBK_SparkStructuredStreaming_Batch_2GB_of_100B_records.txt} (100%) rename .github/workflows/{load-tests-job-configs/Java_GBK_SparkStructuredStreaming_Batch_2GB_of_100kB_records.txt => load-tests-pipeline-options/java_GBK_SparkStructuredStreaming_Batch_2GB_of_100kB_records.txt} (100%) rename .github/workflows/{load-tests-job-configs/Java_GBK_SparkStructuredStreaming_Batch_2GB_of_10B_records.txt => load-tests-pipeline-options/java_GBK_SparkStructuredStreaming_Batch_2GB_of_10B_records.txt} (100%) rename .github/workflows/{load-tests-job-configs/Java_GBK_SparkStructuredStreaming_Batch_fanout_4_times_with_2GB_10-byte_records_total.txt => load-tests-pipeline-options/java_GBK_SparkStructuredStreaming_Batch_fanout_4_times_with_2GB_10-byte_records_total.txt} (100%) rename .github/workflows/{load-tests-job-configs/Java_GBK_SparkStructuredStreaming_Batch_fanout_8_times_with_2GB_10-byte_records_total.txt => load-tests-pipeline-options/java_GBK_SparkStructuredStreaming_Batch_fanout_8_times_with_2GB_10-byte_records_total.txt} (100%) rename .github/workflows/{load-tests-job-configs/Java_GBK_SparkStructuredStreaming_Batch_reiterate_4_times_10kB_values.txt => load-tests-pipeline-options/java_GBK_SparkStructuredStreaming_Batch_reiterate_4_times_10kB_values.txt} (100%) rename .github/workflows/{load-tests-job-configs/Java_GBK_SparkStructuredStreaming_Batch_reiterate_4_times_2MB_values.txt => load-tests-pipeline-options/java_GBK_SparkStructuredStreaming_Batch_reiterate_4_times_2MB_values.txt} (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_LoadTests_Combine_Smoke_CombineLoadTest_load_test_Dataflow-1.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_LoadTests_Combine_Smoke_CombineLoadTest_load_test_Dataflow-2.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_LoadTests_Combine_Smoke_CombineLoadTest_load_test_Dataflow-3.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_ParDo_Dataflow_Batch_100_counters.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_ParDo_Dataflow_Batch_10_counters.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_ParDo_Dataflow_Batch_10_times.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_ParDo_Dataflow_Batch_200_times.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_ParDo_Dataflow_Streaming_100_counters.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_ParDo_Dataflow_Streaming_10_counters.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_ParDo_Dataflow_Streaming_10_times.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_ParDo_Dataflow_Streaming_200_times.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_ParDo_Dataflow_V2_Batch_JavaVersions_100_counters.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_ParDo_Dataflow_V2_Batch_JavaVersions_10_counters.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_ParDo_Dataflow_V2_Batch_JavaVersions_10_times.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_ParDo_Dataflow_V2_Batch_JavaVersions_200_times.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_ParDo_Dataflow_V2_Streaming_JavaVersions_100_counters.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_ParDo_Dataflow_V2_Streaming_JavaVersions_10_counters.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_ParDo_Dataflow_V2_Streaming_JavaVersions_10_times.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_ParDo_Dataflow_V2_Streaming_JavaVersions_200_times.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_ParDo_SparkStructuredStreaming_Batch_100_counters.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_ParDo_SparkStructuredStreaming_Batch_10_counters.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_ParDo_SparkStructuredStreaming_Batch_10_times.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_ParDo_SparkStructuredStreaming_Batch_200_times.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_Smoke_GroupByKey_Dataflow.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_Smoke_GroupByKey_Direct.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_Smoke_GroupByKey_Flink.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/java_Smoke_GroupByKey_Spark.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_CoGBK_Dataflow_Batch_100b_Multiple_Keys.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_CoGBK_Dataflow_Batch_100b_Single_Key.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_CoGBK_Dataflow_Batch_10kB.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_CoGBK_Dataflow_Batch_2MB.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_CoGBK_Dataflow_Flink_Batch_100b_Multiple_Keys.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_CoGBK_Dataflow_Flink_Batch_100b_Single_Key.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_CoGBK_Dataflow_Flink_Batch_10kB.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_CoGBK_Dataflow_Streaming_100b_Multiple_Keys.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_CoGBK_Dataflow_Streaming_100b_Single_Key.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_CoGBK_Dataflow_Streaming_10kB.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_CoGBK_Dataflow_Streaming_2MB.txt (100%) rename .github/workflows/{load-tests-job-configs/config_Combine_Python_Batch_2GB_10b.txt => load-tests-pipeline-options/python_Combine_Dataflow_Batch_2GB_10b.txt} (77%) rename .github/workflows/{load-tests-job-configs/config_Combine_Python_Batch_2GB_Fanout_4.txt => load-tests-pipeline-options/python_Combine_Dataflow_Batch_2GB_Fanout_4.txt} (79%) rename .github/workflows/{load-tests-job-configs/config_Combine_Python_Batch_2GB_Fanout_8.txt => load-tests-pipeline-options/python_Combine_Dataflow_Batch_2GB_Fanout_8.txt} (79%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_Combine_Dataflow_Streaming_2GB_10_byte_records.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_Combine_Dataflow_Streaming_2GB_Fanout_4.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_Combine_Dataflow_Streaming_2GB_Fanout_8.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_Combine_Flink_Batch_2GB_10_byte_records.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_Combine_Flink_Batch_2GB_Fanout_4.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_Combine_Flink_Batch_2GB_Fanout_8.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_Combine_Flink_Streaming_2GB_10_byte_records.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_Combine_Flink_Streaming_2GB_Fanout_4.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_Combine_Flink_Streaming_2GB_Fanout_8.txt (100%) rename .github/workflows/{load-tests-job-configs/config_FnApiRunner_Python_Microbenchmark.txt => load-tests-pipeline-options/python_FnApiRunner_Microbenchmark.txt} (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_GBK_Dataflow_Batch_2GB_of_100B_records.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_GBK_Dataflow_Batch_2GB_of_100kB_records.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_GBK_Dataflow_Batch_2GB_of_10B_records.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_GBK_Dataflow_Batch_fanout_4_times_with_2GB_10-byte_records_total.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_GBK_Dataflow_Batch_fanout_8_times_with_2GB_10-byte_records_total.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_GBK_Dataflow_Streaming_2GB_of_100B_records.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_GBK_Dataflow_Streaming_2GB_of_100kB_records.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_GBK_Dataflow_Streaming_2GB_of_10B_records.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_GBK_Dataflow_Streaming_fanout_4_times_with_2GB_10-byte_records_total.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_GBK_Dataflow_Streaming_fanout_8_times_with_2GB_10-byte_records_total.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_GBK_Flink_Batch_2GB_of_100B_records.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_GBK_Flink_Batch_2GB_of_10B_records.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_GBK_Flink_Batch_fanout_4_times_with_2GB_10-byte_records_total.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_GBK_Flink_Batch_fanout_8_times_with_2GB_10-byte_records_total.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_GBK_Flink_Batch_reiterate_4_times_10kB_values.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_GBK_reiterate_Dataflow_Batch_reiterate_4_times_10kB_values.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_GBK_reiterate_Dataflow_Batch_reiterate_4_times_2MB_values.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_GBK_reiterate_Dataflow_Streaming_reiterate_4_times_10kB_values.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_GBK_reiterate_Dataflow_Streaming_reiterate_4_times_2MB_values.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_ParDo_Dataflow_Batch_100_Counters.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_ParDo_Dataflow_Batch_10_Counters.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_ParDo_Dataflow_Batch_10_Iterations.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_ParDo_Dataflow_Batch_200_Iterations.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_ParDo_Dataflow_Streaming_100_Counters.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_ParDo_Dataflow_Streaming_10_Counters.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_ParDo_Dataflow_Streaming_10_Iterations.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_ParDo_Dataflow_Streaming_200_Iterations.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_ParDo_Flink_Batch_10_Counters.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_ParDo_Flink_Batch_10_Iterations.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_ParDo_Flink_Batch_200_Iterations.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_ParDo_Flink_Streaming_100_Counters.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_ParDo_Flink_Streaming_10_Counters.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_ParDo_Flink_Streaming_10_Iterations.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_ParDo_Flink_Streaming_200_Iterations.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_ParDo_Flink_Streaming_5_Iterations.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_SideInput_Dataflow_Batch_10gb_1000window_first_iterable.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_SideInput_Dataflow_Batch_10gb_1000window_iterable.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_SideInput_Dataflow_Batch_10gb_1window_first_iterable.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_SideInput_Dataflow_Batch_10gb_1window_iterable.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_SideInput_Dataflow_Batch_1gb_1000window_1key_percent_dict.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_SideInput_Dataflow_Batch_1gb_1000window_99key_percent_dict.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_SideInput_Dataflow_Batch_1gb_1window_1key_percent_dict.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_SideInput_Dataflow_Batch_1gb_1window_99key_percent_dict.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_SideInput_Dataflow_Batch_1gb_1window_first_list.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_SideInput_Dataflow_Batch_1gb_1window_list.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_Smoke_GroupByKey_Dataflow.txt (100%) rename .github/workflows/{load-tests-job-configs => load-tests-pipeline-options}/python_Smoke_GroupByKey_Direct.txt (100%) delete mode 100644 .github/workflows/performance-tests-job-configs/config_BigQueryIO_Batch_Java_Avro.txt delete mode 100644 .github/workflows/performance-tests-job-configs/config_BigQueryIO_Batch_Java_Json.txt delete mode 100644 .github/workflows/performance-tests-job-configs/config_BigQueryIO_Streaming_Java.txt rename .github/workflows/{performance-tests-job-configs => performance-tests-pipeline-options}/JDBC.txt (100%) rename .github/workflows/{performance-tests-job-configs => performance-tests-pipeline-options}/SQLBigQueryIO_Batch_Java.txt (100%) rename .github/workflows/{performance-tests-job-configs/config_TFRecordIOIT.txt => performance-tests-pipeline-options/TFRecordIOIT.txt} (100%) rename .github/workflows/{performance-tests-job-configs => performance-tests-pipeline-options}/TFRecordIOIT_HDFS.txt (100%) rename .github/workflows/{performance-tests-job-configs/config_AvroIOIT.txt => performance-tests-pipeline-options/avroIOIT.txt} (100%) rename .github/workflows/{performance-tests-job-configs/config_AvroIOIT_HDFS.txt => performance-tests-pipeline-options/avroIOIT_HDFS.txt} (100%) create mode 100644 .github/workflows/performance-tests-pipeline-options/bigQueryIO_Batch_Java_Avro.txt create mode 100644 .github/workflows/performance-tests-pipeline-options/bigQueryIO_Batch_Java_Json.txt create mode 100644 .github/workflows/performance-tests-pipeline-options/bigQueryIO_Streaming_Java.txt rename .github/workflows/{performance-tests-job-configs => performance-tests-pipeline-options}/biqQueryIO_Read_Python.txt (100%) rename .github/workflows/{performance-tests-job-configs => performance-tests-pipeline-options}/biqQueryIO_Write_Python_Batch.txt (100%) rename .github/workflows/{performance-tests-job-configs => performance-tests-pipeline-options}/cdap.txt (100%) rename .github/workflows/{performance-tests-job-configs/config_Compressed_TextIOIT.txt => performance-tests-pipeline-options/compressed_TextIOIT.txt} (100%) rename .github/workflows/{performance-tests-job-configs/config_Compressed_TextIOIT_HDFS.txt => performance-tests-pipeline-options/compressed_TextIOIT_HDFS.txt} (100%) rename .github/workflows/{performance-tests-job-configs => performance-tests-pipeline-options}/hadoopFormat.txt (100%) rename .github/workflows/{performance-tests-job-configs => performance-tests-pipeline-options}/kafka_IO_Batch.txt (100%) rename .github/workflows/{performance-tests-job-configs => performance-tests-pipeline-options}/kafka_IO_Streaming.txt (100%) rename .github/workflows/{performance-tests-job-configs/config_ManyFiles_TextIOIT.txt => performance-tests-pipeline-options/manyFiles_TextIOIT.txt} (100%) rename .github/workflows/{performance-tests-job-configs/config_ManyFiles_TextIOIT_HDFS.txt => performance-tests-pipeline-options/manyFiles_TextIOIT_HDFS.txt} (100%) rename .github/workflows/{performance-tests-job-configs => performance-tests-pipeline-options}/mongoDBIO_IT.txt (100%) rename .github/workflows/{performance-tests-job-configs/config_ParquetIOIT.txt => performance-tests-pipeline-options/parquetIOIT.txt} (100%) rename .github/workflows/{performance-tests-job-configs/config_ParquetIOIT_HDFS.txt => performance-tests-pipeline-options/parquetIOIT_HDFS.txt} (100%) rename .github/workflows/{performance-tests-job-configs => performance-tests-pipeline-options}/pubsubIOIT_Python_Streaming.txt (100%) rename .github/workflows/{performance-tests-job-configs => performance-tests-pipeline-options}/spannerIO_Read_2GB_Python.txt (100%) rename .github/workflows/{performance-tests-job-configs => performance-tests-pipeline-options}/spannerIO_Write_2GB_Python.txt (100%) rename .github/workflows/{performance-tests-job-configs => performance-tests-pipeline-options}/sparkReceiver_IO.txt (100%) rename .github/workflows/{performance-tests-job-configs => performance-tests-pipeline-options}/textIOIT.txt (100%) rename .github/workflows/{performance-tests-job-configs => performance-tests-pipeline-options}/textIOIT_HDFS.txt (100%) rename .github/workflows/{performance-tests-job-configs => performance-tests-pipeline-options}/textIOIT_Python.txt (100%) rename .github/workflows/{performance-tests-job-configs => performance-tests-pipeline-options}/wordCountIT_Python.txt (100%) rename .github/workflows/{performance-tests-job-configs => performance-tests-pipeline-options}/xlang_KafkaIO_Python.txt (100%) rename .github/workflows/{performance-tests-job-configs/config_XmlIOIT.txt => performance-tests-pipeline-options/xmlIOIT.txt} (100%) rename .github/workflows/{performance-tests-job-configs/config_XmlIOIT_HDFS.txt => performance-tests-pipeline-options/xmlIOIT_HDFS.txt} (100%) diff --git a/.github/workflows/beam_CloudML_Benchmarks_Dataflow.yml b/.github/workflows/beam_CloudML_Benchmarks_Dataflow.yml index df26f1a5cfc2..fceae349a08d 100644 --- a/.github/workflows/beam_CloudML_Benchmarks_Dataflow.yml +++ b/.github/workflows/beam_CloudML_Benchmarks_Dataflow.yml @@ -83,7 +83,7 @@ jobs: test-type: load test-language: python argument-file-paths: | - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/beam_CloudML_Benchmarks_Dataflow_arguments.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/beam_CloudML_Benchmarks_Dataflow_arguments.txt # The env variables are created and populated in the test-arguments-action as "<github.job>_test_arguments_<argument_file_paths_index>" - name: run TFT Criteo Benchmarks uses: ./.github/actions/gradle-command-self-hosted-action diff --git a/.github/workflows/beam_Inference_Python_Benchmarks_Dataflow.yml b/.github/workflows/beam_Inference_Python_Benchmarks_Dataflow.yml index 97099ec525f1..16ef684b0589 100644 --- a/.github/workflows/beam_Inference_Python_Benchmarks_Dataflow.yml +++ b/.github/workflows/beam_Inference_Python_Benchmarks_Dataflow.yml @@ -81,11 +81,11 @@ jobs: test-type: load test-language: python argument-file-paths: | - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Vision_Classification_Resnet_101.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Imagenet_Classification_Resnet_152.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Language_Modeling_Bert_Base_Uncased.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Language_Modeling_Bert_Large_Uncased.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Imagenet_Classification_Resnet_152_Tesla_T4_GPU.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Vision_Classification_Resnet_101.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Imagenet_Classification_Resnet_152.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Language_Modeling_Bert_Base_Uncased.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Language_Modeling_Bert_Large_Uncased.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Imagenet_Classification_Resnet_152_Tesla_T4_GPU.txt # The env variables are created and populated in the test-arguments-action as "<github.job>_test_arguments_<argument_file_paths_index>" - name: get current time run: echo "NOW_UTC=$(date '+%m%d%H%M%S' --utc)" >> $GITHUB_ENV diff --git a/.github/workflows/beam_Java_LoadTests_Combine_Smoke.yml b/.github/workflows/beam_Java_LoadTests_Combine_Smoke.yml index 2eb7f061ce99..19d5b0f7174f 100644 --- a/.github/workflows/beam_Java_LoadTests_Combine_Smoke.yml +++ b/.github/workflows/beam_Java_LoadTests_Combine_Smoke.yml @@ -75,9 +75,9 @@ jobs: test-type: load test-language: java argument-file-paths: | - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_LoadTests_Combine_Smoke_CombineLoadTest_load_test_Dataflow-1.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_LoadTests_Combine_Smoke_CombineLoadTest_load_test_Dataflow-2.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_LoadTests_Combine_Smoke_CombineLoadTest_load_test_Dataflow-3.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_LoadTests_Combine_Smoke_CombineLoadTest_load_test_Dataflow-1.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_LoadTests_Combine_Smoke_CombineLoadTest_load_test_Dataflow-2.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_LoadTests_Combine_Smoke_CombineLoadTest_load_test_Dataflow-3.txt # The env variables are created and populated in the test-arguments-action as "<github.job>_test_arguments_<argument_file_paths_index>" - name: run CombineLoadTest load test Dataflow-1 uses: ./.github/actions/gradle-command-self-hosted-action diff --git a/.github/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml index 4a7b6949ee44..dac3733da4a9 100644 --- a/.github/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml @@ -75,10 +75,10 @@ jobs: test-type: load test-language: go argument-file-paths: | - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/config_CoGBK_Go_Batch_SingleKey.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/config_CoGBK_Go_Batch_MultipleKey.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/config_CoGBK_Go_Batch_Reiteration_10KB.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/config_CoGBK_Go_Batch_Reiteration_2MB.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/go_CoGBK_Dataflow_Batch_SingleKey.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/go_CoGBK_Dataflow_Batch_MultipleKey.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/go_CoGBK_Dataflow_Batch_Reiteration_10KB.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/go_CoGBK_Dataflow_Batch_Reiteration_2MB.txt arguments: | --job_name=load-tests-go-dataflow-batch-cogbk-$(date '+%m%d%H%M%S' --utc) - name: run CoGBK Dataflow Batch Go Load Test 1 (single key) diff --git a/.github/workflows/beam_LoadTests_Go_CoGBK_Flink_batch.yml b/.github/workflows/beam_LoadTests_Go_CoGBK_Flink_batch.yml index f3a4cba5fce1..05b61f285cd7 100644 --- a/.github/workflows/beam_LoadTests_Go_CoGBK_Flink_batch.yml +++ b/.github/workflows/beam_LoadTests_Go_CoGBK_Flink_batch.yml @@ -85,9 +85,9 @@ jobs: test-type: load test-language: go argument-file-paths: | - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/go_CoGBK_Flink_Batch_MultipleKey.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/go_CoGBK_Flink_Batch_Reiteration_10KB.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/go_CoGBK_Flink_Batch_Reiteration_2MB.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/go_CoGBK_Flink_Batch_MultipleKey.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/go_CoGBK_Flink_Batch_Reiteration_10KB.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/go_CoGBK_Flink_Batch_Reiteration_2MB.txt arguments: | --job_name=load-tests-go-flink-batch-cogbk-$(date '+%m%d%H%M%S' --utc) - name: Start Flink with parallelism 5 diff --git a/.github/workflows/beam_LoadTests_Go_Combine_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Go_Combine_Dataflow_Batch.yml index 9568d0cfcff6..cb198bbc233f 100644 --- a/.github/workflows/beam_LoadTests_Go_Combine_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_Combine_Dataflow_Batch.yml @@ -69,19 +69,18 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - - name: Prepare configs - #Reads config files, excludes comments, appends current date to the job_name parameter - id: set_configs - shell: bash - run: | - CURDATE=$(date '+%m%d%H%M%S' --utc) - CONFIG_ARR=('config_Combine_Go_Batch_10b.txt' 'config_Combine_Go_Batch_Fanout_4.txt' 'config_Combine_Go_Batch_Fanout_8.txt') - for INDEX in ${!CONFIG_ARR[@]} - do - CURCONFIG=$(grep -v "^#.*" ./.github/workflows/load-tests-job-configs/${CONFIG_ARR[INDEX]} | tr '\n' ' ') - CURCONFIG=$(echo "${CURCONFIG/load-tests-go-dataflow-batch-combine-$((INDEX + 1))-/load-tests-go-dataflow-batch-combine-$((INDEX + 1))-$CURDATE}") - echo "prepared_config_$((INDEX + 1))=$CURCONFIG" >> $GITHUB_OUTPUT - done + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: go + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/go_Combine_Dataflow_Batch_10b.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/go_Combine_Dataflow_Batch_Fanout_4.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/go_Combine_Dataflow_Batch_Fanout_8.txt + # The env variables are created and populated in the test-arguments-action as "<github.job>_test_arguments_<argument_file_paths_index>" + - name: get current time + run: echo "NOW_UTC=$(date '+%m%d%H%M%S' --utc)" >> $GITHUB_ENV - name: run Combine Dataflow Batch Go Load Test 1 (single key) uses: ./.github/actions/gradle-command-self-hosted-action with: @@ -89,7 +88,7 @@ jobs: arguments: | -PloadTest.mainClass=combine \ -Prunner=DataflowRunner \ - '-PloadTest.args=${{ steps.set_configs.outputs.prepared_config_1 }}' \ + '-PloadTest.args=${{ env.beam_LoadTests_Go_Combine_Dataflow_Batch_test_arguments_1 }} --job_name=load-tests-go-dataflow-batch-combine-1-${{env.NOW_UTC}}' \ - name: run Combine Dataflow Batch Go Load Test 2 (multiple keys) uses: ./.github/actions/gradle-command-self-hosted-action with: @@ -97,7 +96,7 @@ jobs: arguments: | -PloadTest.mainClass=combine \ -Prunner=DataflowRunner \ - '-PloadTest.args=${{ steps.set_configs.outputs.prepared_config_2 }}' \ + '-PloadTest.args=${{ env.beam_LoadTests_Go_Combine_Dataflow_Batch_test_arguments_2 }} --job_name=load-tests-go-dataflow-batch-combine-2-${{env.NOW_UTC}}' \ - name: run Combine Dataflow Batch Go Load Test 3 (reiterate 10KB) uses: ./.github/actions/gradle-command-self-hosted-action with: @@ -105,4 +104,4 @@ jobs: arguments: | -PloadTest.mainClass=combine \ -Prunner=DataflowRunner \ - '-PloadTest.args=${{ steps.set_configs.outputs.prepared_config_3 }}' \ No newline at end of file + '-PloadTest.args=${{ env.beam_LoadTests_Go_Combine_Dataflow_Batch_test_arguments_3 }} --job_name=load-tests-go-dataflow-batch-combine-3-${{env.NOW_UTC}}' \ No newline at end of file diff --git a/.github/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml index f368707f1bd9..f5c9c97652ee 100644 --- a/.github/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml @@ -85,9 +85,9 @@ jobs: test-type: load test-language: go argument-file-paths: | - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/go_Combine_Flink_Batch_10b.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/go_Combine_Flink_Batch_Fanout_4.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/go_Combine_Flink_Batch_Fanout_8.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/go_Combine_Flink_Batch_10b.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/go_Combine_Flink_Batch_Fanout_4.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/go_Combine_Flink_Batch_Fanout_8.txt arguments: | --job_name=load-tests-go-flink-batch-combine-$(date '+%m%d%H%M%S' --utc) - name: Start Flink with parallelism 5 diff --git a/.github/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml index 01493d467b35..9778b542bacb 100644 --- a/.github/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml @@ -75,13 +75,13 @@ jobs: test-type: load test-language: go argument-file-paths: | - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/go_GBK_Dataflow_Batch_10b.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/go_GBK_Dataflow_Batch_100b.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/go_GBK_Dataflow_Batch_100kb.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/go_GBK_Dataflow_Batch_Fanout_4.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/go_GBK_Dataflow_Batch_Fanout_8.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/go_GBK_Dataflow_Batch_Reiteration_10KB.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/go_GBK_Dataflow_Batch_Reiteration_2MB.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/go_GBK_Dataflow_Batch_10b.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/go_GBK_Dataflow_Batch_100b.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/go_GBK_Dataflow_Batch_100kb.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/go_GBK_Dataflow_Batch_Fanout_4.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/go_GBK_Dataflow_Batch_Fanout_8.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/go_GBK_Dataflow_Batch_Reiteration_10KB.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/go_GBK_Dataflow_Batch_Reiteration_2MB.txt - name: get current time run: echo "NOW_UTC=$(date '+%m%d%H%M%S' --utc)" >> $GITHUB_ENV # The env variables are created and populated in the test-arguments-action as "<github.job>_test_arguments_<argument_file_paths_index>" diff --git a/.github/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml index 53982b4175f4..37ae1e419c59 100644 --- a/.github/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml @@ -85,12 +85,12 @@ jobs: test-type: load test-language: go argument-file-paths: | - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/go_GBK_Flink_Batch_10b.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/go_GBK_Flink_Batch_100b.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/go_GBK_Flink_Batch_100kb.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/go_GBK_Flink_Batch_Fanout_4.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/go_GBK_Flink_Batch_Fanout_8.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/go_GBK_Flink_Batch_Reiteration_10KB.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/go_GBK_Flink_Batch_10b.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/go_GBK_Flink_Batch_100b.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/go_GBK_Flink_Batch_100kb.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/go_GBK_Flink_Batch_Fanout_4.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/go_GBK_Flink_Batch_Fanout_8.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/go_GBK_Flink_Batch_Reiteration_10KB.txt arguments: | --job_name=load-tests-go-flink-batch-gbk-$(date '+%m%d%H%M%S' --utc) - name: Start Flink with parallelism 5 diff --git a/.github/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml index 73e21c4d6c56..99292c88757a 100644 --- a/.github/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml @@ -75,10 +75,10 @@ jobs: test-type: load test-language: go argument-file-paths: | - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/config_ParDo_Go_Batch_10_Iterations.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/config_ParDo_Go_Batch_200_Iterations.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/config_ParDo_Go_Batch_10_Counters.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/config_ParDo_Go_Batch_100_Counters.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/go_ParDo_Dataflow_Batch_10_Iterations.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/go_ParDo_Dataflow_Batch_200_Iterations.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/go_ParDo_Dataflow_Batch_10_Counters.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/go_ParDo_Dataflow_Batch_100_Counters.txt arguments: | --job_name=load-tests-go-dataflow-batch-pardo-$(date '+%m%d%H%M%S' --utc) - name: run ParDo Dataflow Batch Go Load Test 1 (10 iterations) diff --git a/.github/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml index 9fe817f01566..666799bfc655 100644 --- a/.github/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml @@ -85,10 +85,10 @@ jobs: test-type: load test-language: go argument-file-paths: | - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/go_ParDo_Flink_Batch_10_times.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/go_ParDo_Flink_Batch_200_times.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/go_ParDo_Flink_Batch_10_counters.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/go_ParDo_Flink_Batch_100_counters.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/go_ParDo_Flink_Batch_10_times.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/go_ParDo_Flink_Batch_200_times.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/go_ParDo_Flink_Batch_10_counters.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/go_ParDo_Flink_Batch_100_counters.txt arguments: | --job_name=load-tests-go-flink-batch-pardo-$(date '+%m%d%H%M%S' --utc) - name: Start Flink with parallelism 5 diff --git a/.github/workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml index 4aa8d7f7c3de..123e839d46bb 100644 --- a/.github/workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml @@ -75,8 +75,8 @@ jobs: test-type: load test-language: go argument-file-paths: | - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/config_SideInput_Go_Batch_Dataflow_First_Iterable.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/config_SideInput_Go_Batch_Dataflow_Iterable.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/go_SideInput_Dataflow_Batch_First_Iterable.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/go_SideInput_Dataflow_Batch_Iterable.txt arguments: | --job_name=load-tests-go-dataflow-batch-sideinput-$(date '+%m%d%H%M%S' --utc) - name: run SideInput Dataflow Batch Go Load Test 1 (first iterable) diff --git a/.github/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml index 53c100fff2c8..592af7e86676 100644 --- a/.github/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml @@ -85,8 +85,8 @@ jobs: test-type: load test-language: go argument-file-paths: | - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/go_SideInput_Flink_Batch_First_Iterable.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/go_SideInput_Flink_Batch_Iterable.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/go_SideInput_Flink_Batch_First_Iterable.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/go_SideInput_Flink_Batch_Iterable.txt arguments: | --job_name=load-tests-go-flink-batch-sideinput-$(date '+%m%d%H%M%S' --utc) - name: Start Flink with parallelism 10 diff --git a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml index 705d8a85fa65..8677f6f81f71 100644 --- a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml @@ -75,10 +75,10 @@ jobs: test-type: load test-language: java argument-file-paths: | - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_Batch_100b_Single_Key.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_Batch_100b_Multiple_Keys.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_Batch_10kB.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_Batch_2MB.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_CoGBK_Dataflow_Batch_100b_Single_Key.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_CoGBK_Dataflow_Batch_100b_Multiple_Keys.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_CoGBK_Dataflow_Batch_10kB.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_CoGBK_Dataflow_Batch_2MB.txt # The env variables are created and populated in the test-arguments-action as "<github.job>_test_arguments_<argument_file_paths_index>" - name: run CoGBK 2GB 100 byte records - single key uses: ./.github/actions/gradle-command-self-hosted-action diff --git a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml index 4ecca6231401..bdd0adadb11d 100644 --- a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml @@ -59,7 +59,7 @@ jobs: name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) strategy: matrix: - job_name: [ "beam_LoadTests_Java_CoGBK_Dataflow_Streaming" ] + job_name: ["beam_LoadTests_Java_CoGBK_Dataflow_Streaming"] job_phrase: ["Run Load Tests Java CoGBK Dataflow Streaming"] steps: - uses: actions/checkout@v4 @@ -69,18 +69,17 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - - name: Prepare configs - id: set_configs - shell: bash - run: | - CURCONFIG=$(grep -v "^#.*" ./.github/workflows/load-tests-job-configs/config_CoGBK_Java_Streaming_2GB_SingleKey.txt | tr '\n' ' ') - echo "prepared_config_1=$CURCONFIG" >> $GITHUB_OUTPUT - CURCONFIG=$(grep -v "^#.*" ./.github/workflows/load-tests-job-configs/config_CoGBK_Java_Streaming_2GB_MultipleKey.txt | tr '\n' ' ') - echo "prepared_config_2=$CURCONFIG" >> $GITHUB_OUTPUT - CURCONFIG=$(grep -v "^#.*" ./.github/workflows/load-tests-job-configs/config_CoGBK_Java_Streaming_2GB_Reiteration_10KB.txt | tr '\n' ' ') - echo "prepared_config_3=$CURCONFIG" >> $GITHUB_OUTPUT - CURCONFIG=$(grep -v "^#.*" ./.github/workflows/load-tests-job-configs/config_CoGBK_Java_Streaming_2GB_Reiteration_2MB.txt | tr '\n' ' ') - echo "prepared_config_4=$CURCONFIG" >> $GITHUB_OUTPUT + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: java + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_CoGBK_Dataflow_Streaming_2GB_SingleKey.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_CoGBK_Dataflow_Streaming_2GB_MultipleKey.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_CoGBK_Dataflow_Streaming_2GB_Reiteration_10KB.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_CoGBK_Dataflow_Streaming_2GB_Reiteration_2MB.txt + # The env variables are created and populated in the test-arguments-action as "<github.job>_test_arguments_<argument_file_paths_index>" - name: run CoGBK Dataflow Streaming Java Load Test 1 (single key) uses: ./.github/actions/gradle-command-self-hosted-action with: @@ -88,7 +87,7 @@ jobs: arguments: | -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ - '-PloadTest.args=${{ steps.set_configs.outputs.prepared_config_1 }}' \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_CoGBK_Dataflow_Streaming_test_arguments_1 }}' \ - name: run CoGBK Dataflow Streaming Java Load Test 2 (multiple key) uses: ./.github/actions/gradle-command-self-hosted-action with: @@ -96,7 +95,7 @@ jobs: arguments: | -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ - '-PloadTest.args=${{ steps.set_configs.outputs.prepared_config_2 }}' \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_CoGBK_Dataflow_Streaming_test_arguments_2 }}' \ - name: run CoGBK Dataflow Streaming Java Load Test 3 (reiteration 10KB value) uses: ./.github/actions/gradle-command-self-hosted-action with: @@ -104,7 +103,7 @@ jobs: arguments: | -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ - '-PloadTest.args=${{ steps.set_configs.outputs.prepared_config_3 }}' \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_CoGBK_Dataflow_Streaming_test_arguments_3 }}' \ - name: run CoGBK Dataflow Streaming Java Load Test 4 (reiteration 2MB value) uses: ./.github/actions/gradle-command-self-hosted-action with: @@ -112,7 +111,7 @@ jobs: arguments: | -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ - '-PloadTest.args=${{ steps.set_configs.outputs.prepared_config_4 }}' \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_CoGBK_Dataflow_Streaming_test_arguments_4 }}' \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 if: failure() diff --git a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml index 327e007d5437..8c785595ffec 100644 --- a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml +++ b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml @@ -83,10 +83,10 @@ jobs: test-type: load test-language: java argument-file-paths: | - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_Batch_100b_Single_Key.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_Batch_100b_Multiple_Keys.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_Batch_10kB.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_Batch_2MB.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_CoGBK_Dataflow_Batch_100b_Single_Key.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_CoGBK_Dataflow_Batch_100b_Multiple_Keys.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_CoGBK_Dataflow_Batch_10kB.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_CoGBK_Dataflow_Batch_2MB.txt arguments: | --influxTags={\"runnerVersion\":\"v2\",\"jdk\":\"java${{ matrix.java_version }}\"} # The env variables are created and populated in the test-arguments-action as "<github.job>_test_arguments_<argument_file_paths_index>" diff --git a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml index 0abe99ce83d2..0c620f02750d 100644 --- a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml +++ b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml @@ -83,10 +83,10 @@ jobs: test-type: load test-language: java argument-file-paths: | - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_V2_Streaming_Java_100b_Single_Key.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_V2_Streaming_Java_100b_Multiple_Keys.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_V2_Streaming_Java_10kB.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_V2_Streaming_Java_2MB.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_CoGBK_Dataflow_V2_Streaming_Java_100b_Single_Key.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_CoGBK_Dataflow_V2_Streaming_Java_100b_Multiple_Keys.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_CoGBK_Dataflow_V2_Streaming_Java_10kB.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_CoGBK_Dataflow_V2_Streaming_Java_2MB.txt arguments: | --influxTags={\"runnerVersion\":\"v2\",\"jdk\":\"java${{ matrix.java_version }}\"} # The env variables are created and populated in the test-arguments-action as "<github.job>_test_arguments_<argument_file_paths_index>" diff --git a/.github/workflows/beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml b/.github/workflows/beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml index f4b4b6d9946e..7f1935267e95 100644 --- a/.github/workflows/beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml +++ b/.github/workflows/beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml @@ -75,10 +75,10 @@ jobs: test-type: load test-language: java argument-file-paths: | - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_CoGBK_SparkStructuredStreaming_Batch_100b_Single_Key.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_CoGBK_SparkStructuredStreaming_Batch_100b_Multiple_Keys.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_CoGBK_SparkStructuredStreaming_Batch_10kB.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_CoGBK_SparkStructuredStreaming_Batch_2MB.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_CoGBK_SparkStructuredStreaming_Batch_100b_Single_Key.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_CoGBK_SparkStructuredStreaming_Batch_100b_Multiple_Keys.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_CoGBK_SparkStructuredStreaming_Batch_10kB.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_CoGBK_SparkStructuredStreaming_Batch_2MB.txt # The env variables are created and populated in the test-arguments-action as "<github.job>_test_arguments_<argument_file_paths_index>" - name: run CoGBK 2GB 100 byte records - single key uses: ./.github/actions/gradle-command-self-hosted-action diff --git a/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml index 1702d6c75184..02f5da33afaf 100644 --- a/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml @@ -75,9 +75,9 @@ jobs: test-type: load test-language: java argument-file-paths: | - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/config_Combine_Java_Dataflow_Batch_10b.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/config_Combine_Java_Dataflow_Batch_Fanout_4.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/config_Combine_Java_Dataflow_Batch_Fanout_8.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_Combine_Dataflow_Batch_10b.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_Combine_Dataflow_Batch_Fanout_4.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_Combine_Dataflow_Batch_Fanout_8.txt - name: run Combine Dataflow Batch Java Load Test 1 (10 b records) uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Streaming.yml index e0e033939d3f..d6520646fcdd 100644 --- a/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Streaming.yml @@ -47,9 +47,6 @@ env: GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} - beam_LoadTests_Java_Combine_Dataflow_Streaming_test_arguments_1: '' - beam_LoadTests_Java_Combine_Dataflow_Streaming_test_arguments_2: '' - beam_LoadTests_Java_Combine_Dataflow_Streaming_test_arguments_3: '' jobs: beam_LoadTests_Java_Combine_Dataflow_Streaming: @@ -78,9 +75,10 @@ jobs: test-type: load test-language: java argument-file-paths: | - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_Combine_Dataflow_Streaming_10b.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_Combine_Dataflow_Streaming_Fanout_4.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_Combine_Dataflow_Streaming_Fanout_8.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_Combine_Dataflow_Streaming_10b.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_Combine_Dataflow_Streaming_Fanout_4.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_Combine_Dataflow_Streaming_Fanout_8.txt + # The env variables are created and populated in the test-arguments-action as "<github.job>_test_arguments_<argument_file_paths_index>" - name: run Combine Dataflow Streaming Java Load Test 1 (10 b records) uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml b/.github/workflows/beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml index 26c1a4975a7c..d4862ad45c07 100644 --- a/.github/workflows/beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml +++ b/.github/workflows/beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml @@ -47,9 +47,6 @@ env: GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} - beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch_test_arguments_1: '' - beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch_test_arguments_2: '' - beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch_test_arguments_3: '' jobs: beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch: @@ -78,9 +75,10 @@ jobs: test-type: load test-language: java argument-file-paths: | - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_Combine_SparkStructuredStreaming_Batch_10b.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_Combine_SparkStructuredStreaming_Batch_Fanout_4.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_Combine_SparkStructuredStreaming_Batch_Fanout_8.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_Combine_SparkStructuredStreaming_Batch_10b.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_Combine_SparkStructuredStreaming_Batch_Fanout_4.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_Combine_SparkStructuredStreaming_Batch_Fanout_8.txt + # The env variables are created and populated in the test-arguments-action as "<github.job>_test_arguments_<argument_file_paths_index>" - name: run Combine SparkStructuredStreaming Batch Java Load Test 1 (10b records) uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml index fd86521e592d..403deba2f0a5 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml @@ -75,13 +75,13 @@ jobs: test-type: load test-language: java argument-file-paths: | - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_2GB_of_10B_records.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_2GB_of_100B_records.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_2GB_of_100kB_records.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_fanout_4_times_with_2GB_10-byte_records_total.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_fanout_8_times_with_2GB_10-byte_records_total.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_reiterate_4_times_10kB_values.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_reiterate_4_times_2MB_values.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_Batch_2GB_of_10B_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_Batch_2GB_of_100B_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_Batch_2GB_of_100kB_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_Batch_fanout_4_times_with_2GB_10-byte_records_total.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_Batch_fanout_8_times_with_2GB_10-byte_records_total.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_Batch_reiterate_4_times_10kB_values.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_Batch_reiterate_4_times_2MB_values.txt # The env variables are created and populated in the test-arguments-action as "<github.job>_test_arguments_<argument_file_paths_index>" - name: run Load test 2GB of 10B records uses: ./.github/actions/gradle-command-self-hosted-action diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml index 63bc91bb1599..483c06cf821a 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml @@ -75,13 +75,13 @@ jobs: test-type: load test-language: java argument-file-paths: | - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_2GB_of_10B_records.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_2GB_of_100B_records.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_2GB_of_100kB_records.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_fanout_4_times_with_2GB_10-byte_records_total.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_fanout_8_times_with_2GB_10-byte_records_total.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_reiterate_4_times_10kB_values.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_reiterate_4_times_2MB_values.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_Streaming_2GB_of_10B_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_Streaming_2GB_of_100B_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_Streaming_2GB_of_100kB_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_Streaming_fanout_4_times_with_2GB_10-byte_records_total.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_Streaming_fanout_8_times_with_2GB_10-byte_records_total.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_Streaming_reiterate_4_times_10kB_values.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_Streaming_reiterate_4_times_2MB_values.txt # The env variables are created and populated in the test-arguments-action as "<github.job>_test_arguments_<argument_file_paths_index>" - name: run Load test 2GB of 10B records uses: ./.github/actions/gradle-command-self-hosted-action diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml index 39490d59fc22..fc5d072c3711 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml @@ -79,13 +79,13 @@ jobs: test-type: load test-language: java argument-file-paths: | - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_2GB_of_10B_records.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_2GB_of_100B_records.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_2GB_of_100kB_records.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_fanout_4_times_with_2GB_10-byte_records_total.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_fanout_8_times_with_2GB_10-byte_records_total.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_reiterate_4_times_10kB_values.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_reiterate_4_times_2MB_values.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Batch_Java11_2GB_of_10B_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Batch_Java11_2GB_of_100B_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Batch_Java11_2GB_of_100kB_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Batch_Java11_fanout_4_times_with_2GB_10-byte_records_total.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Batch_Java11_fanout_8_times_with_2GB_10-byte_records_total.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Batch_Java11_reiterate_4_times_10kB_values.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Batch_Java11_reiterate_4_times_2MB_values.txt # The env variables are created and populated in the test-arguments-action as "<github.job>_test_arguments_<argument_file_paths_index>" - name: run Load test 2GB of 10B records uses: ./.github/actions/gradle-command-self-hosted-action diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml index bb4e039ec8c2..fb249d2ade50 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml @@ -81,13 +81,13 @@ jobs: test-type: load test-language: java argument-file-paths: | - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_2GB_of_10B_records.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_2GB_of_100B_records.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_2GB_of_100kB_records.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_fanout_4_times_with_2GB_10-byte_records_total.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_fanout_8_times_with_2GB_10-byte_records_total.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_reiterate_4_times_10kB_values.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_reiterate_4_times_2MB_values.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Batch_Java17_2GB_of_10B_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Batch_Java17_2GB_of_100B_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Batch_Java17_2GB_of_100kB_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Batch_Java17_fanout_4_times_with_2GB_10-byte_records_total.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Batch_Java17_fanout_8_times_with_2GB_10-byte_records_total.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Batch_Java17_reiterate_4_times_10kB_values.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Batch_Java17_reiterate_4_times_2MB_values.txt # The env variables are created and populated in the test-arguments-action as "<github.job>_test_arguments_<argument_file_paths_index>" - name: run Load test 2GB of 10B records uses: ./.github/actions/gradle-command-self-hosted-action diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml index 60cf635457de..1ce6c4d7d378 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml @@ -79,13 +79,13 @@ jobs: test-type: load test-language: java argument-file-paths: | - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_2GB_of_10B_records.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_2GB_of_100B_records.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_2GB_of_100kB_records.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_fanout_4_times_with_2GB_10-byte_records_total.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_fanout_8_times_with_2GB_10-byte_records_total.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_reiterate_4_times_10kB_values.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_reiterate_4_times_2MB_values.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Streaming_Java11_2GB_of_10B_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Streaming_Java11_2GB_of_100B_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Streaming_Java11_2GB_of_100kB_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Streaming_Java11_fanout_4_times_with_2GB_10-byte_records_total.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Streaming_Java11_fanout_8_times_with_2GB_10-byte_records_total.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Streaming_Java11_reiterate_4_times_10kB_values.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Streaming_Java11_reiterate_4_times_2MB_values.txt # The env variables are created and populated in the test-arguments-action as "<github.job>_test_arguments_<argument_file_paths_index>" - name: run Load test 2GB of 10B records uses: ./.github/actions/gradle-command-self-hosted-action diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml index 24d218c152ed..ecffc9923f02 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml @@ -81,13 +81,13 @@ jobs: test-type: load test-language: java argument-file-paths: | - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_2GB_of_10B_records.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_2GB_of_100B_records.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_2GB_of_100kB_records.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_fanout_4_times_with_2GB_10-byte_records_total.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_fanout_8_times_with_2GB_10-byte_records_total.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_reiterate_4_times_10kB_values.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_reiterate_4_times_2MB_values.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Streaming_Java17_2GB_of_10B_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Streaming_Java17_2GB_of_100B_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Streaming_Java17_2GB_of_100kB_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Streaming_Java17_fanout_4_times_with_2GB_10-byte_records_total.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Streaming_Java17_fanout_8_times_with_2GB_10-byte_records_total.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Streaming_Java17_reiterate_4_times_10kB_values.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Streaming_Java17_reiterate_4_times_2MB_values.txt # The env variables are created and populated in the test-arguments-action as "<github.job>_test_arguments_<argument_file_paths_index>" - name: run Load test 2GB of 10B records uses: ./.github/actions/gradle-command-self-hosted-action diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Smoke.yml b/.github/workflows/beam_LoadTests_Java_GBK_Smoke.yml index d41ee990a6ae..251291163efe 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_Smoke.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_Smoke.yml @@ -72,10 +72,10 @@ jobs: test-type: load test-language: java argument-file-paths: | - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_Smoke_GroupByKey_Direct.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_Smoke_GroupByKey_Dataflow.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_Smoke_GroupByKey_Flink.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_Smoke_GroupByKey_Spark.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_Smoke_GroupByKey_Direct.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_Smoke_GroupByKey_Dataflow.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_Smoke_GroupByKey_Flink.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_Smoke_GroupByKey_Spark.txt - name: Set current datetime id: datetime run: | diff --git a/.github/workflows/beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch.yml b/.github/workflows/beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch.yml index 9f057bba3d3e..de66760e5298 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch.yml @@ -75,13 +75,13 @@ jobs: test-type: load test-language: java argument-file-paths: | - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/Java_GBK_SparkStructuredStreaming_Batch_2GB_of_10B_records.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/Java_GBK_SparkStructuredStreaming_Batch_2GB_of_100B_records.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/Java_GBK_SparkStructuredStreaming_Batch_2GB_of_100kB_records.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/Java_GBK_SparkStructuredStreaming_Batch_fanout_4_times_with_2GB_10-byte_records_total.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/Java_GBK_SparkStructuredStreaming_Batch_fanout_8_times_with_2GB_10-byte_records_total.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/Java_GBK_SparkStructuredStreaming_Batch_reiterate_4_times_10kB_values.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/Java_GBK_SparkStructuredStreaming_Batch_reiterate_4_times_2MB_values.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_GBK_SparkStructuredStreaming_Batch_2GB_of_10B_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_GBK_SparkStructuredStreaming_Batch_2GB_of_100B_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_GBK_SparkStructuredStreaming_Batch_2GB_of_100kB_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_GBK_SparkStructuredStreaming_Batch_fanout_4_times_with_2GB_10-byte_records_total.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_GBK_SparkStructuredStreaming_Batch_fanout_8_times_with_2GB_10-byte_records_total.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_GBK_SparkStructuredStreaming_Batch_reiterate_4_times_10kB_values.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_GBK_SparkStructuredStreaming_Batch_reiterate_4_times_2MB_values.txt # The env variables are created and populated in the test-arguments-action as "<github.job>_test_arguments_<argument_file_paths_index>" - name: run Load test 2GB of 10B records uses: ./.github/actions/gradle-command-self-hosted-action diff --git a/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml index 3b19c6a6fa70..ebc30fb0efa4 100644 --- a/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml @@ -47,10 +47,6 @@ env: GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} - beam_LoadTests_Java_ParDo_Dataflow_Batch_test_arguments_1: '' - beam_LoadTests_Java_ParDo_Dataflow_Batch_test_arguments_2: '' - beam_LoadTests_Java_ParDo_Dataflow_Batch_test_arguments_3: '' - beam_LoadTests_Java_ParDo_Dataflow_Batch_test_arguments_4: '' jobs: beam_LoadTests_Java_ParDo_Dataflow_Batch: @@ -79,10 +75,11 @@ jobs: test-type: load test-language: java argument-file-paths: | - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_Batch_10_times.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_Batch_200_times.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_Batch_10_counters.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_Batch_100_counters.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_ParDo_Dataflow_Batch_10_times.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_ParDo_Dataflow_Batch_200_times.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_ParDo_Dataflow_Batch_10_counters.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_ParDo_Dataflow_Batch_100_counters.txt + # The env variables are created and populated in the test-arguments-action as "<github.job>_test_arguments_<argument_file_paths_index>" - name: run ParDo Dataflow Batch Java Load Test 1 (10 times) uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml index 1f91207fee55..6dec9beca4bf 100644 --- a/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml @@ -47,10 +47,6 @@ env: GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} - beam_LoadTests_Java_ParDo_Dataflow_Streaming_test_arguments_1: '' - beam_LoadTests_Java_ParDo_Dataflow_Streaming_test_arguments_2: '' - beam_LoadTests_Java_ParDo_Dataflow_Streaming_test_arguments_3: '' - beam_LoadTests_Java_ParDo_Dataflow_Streaming_test_arguments_4: '' jobs: beam_LoadTests_Java_ParDo_Dataflow_Streaming: @@ -79,10 +75,11 @@ jobs: test-type: load test-language: java argument-file-paths: | - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_Streaming_10_times.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_Streaming_200_times.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_Streaming_10_counters.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_Streaming_100_counters.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_ParDo_Dataflow_Streaming_10_times.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_ParDo_Dataflow_Streaming_200_times.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_ParDo_Dataflow_Streaming_10_counters.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_ParDo_Dataflow_Streaming_100_counters.txt + # The env variables are created and populated in the test-arguments-action as "<github.job>_test_arguments_<argument_file_paths_index>" - name: run ParDo Dataflow Streaming Java Load Test 1 (10 times) uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions.yml b/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions.yml index 66b5b5a8837f..812541087ea7 100644 --- a/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions.yml +++ b/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions.yml @@ -83,10 +83,10 @@ jobs: test-type: load test-language: java argument-file-paths: | - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_V2_Batch_JavaVersions_10_times.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_V2_Batch_JavaVersions_200_times.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_V2_Batch_JavaVersions_10_counters.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_V2_Batch_JavaVersions_100_counters.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_ParDo_Dataflow_V2_Batch_JavaVersions_10_times.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_ParDo_Dataflow_V2_Batch_JavaVersions_200_times.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_ParDo_Dataflow_V2_Batch_JavaVersions_10_counters.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_ParDo_Dataflow_V2_Batch_JavaVersions_100_counters.txt arguments: | --influxTags={\"runnerVersion\":\"v2\",\"jdk\":\"java${{ matrix.java_version }}\"} # The env variables are created and populated in the test-arguments-action as "<github.job>_test_arguments_<argument_file_paths_index>" diff --git a/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions.yml b/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions.yml index bd8987d4ee0d..64013955ad22 100644 --- a/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions.yml +++ b/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions.yml @@ -83,10 +83,10 @@ jobs: test-type: load test-language: java argument-file-paths: | - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_V2_Streaming_JavaVersions_10_times.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_V2_Streaming_JavaVersions_200_times.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_V2_Streaming_JavaVersions_10_counters.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_V2_Streaming_JavaVersions_100_counters.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_ParDo_Dataflow_V2_Streaming_JavaVersions_10_times.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_ParDo_Dataflow_V2_Streaming_JavaVersions_200_times.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_ParDo_Dataflow_V2_Streaming_JavaVersions_10_counters.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_ParDo_Dataflow_V2_Streaming_JavaVersions_100_counters.txt arguments: | --influxTags={\"runnerVersion\":\"v2\",\"jdk\":\"java${{ matrix.java_version }}\"} # The env variables are created and populated in the test-arguments-action as "<github.job>_test_arguments_<argument_file_paths_index>" diff --git a/.github/workflows/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml b/.github/workflows/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml index 45700ca56937..26c1c88dd893 100644 --- a/.github/workflows/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml +++ b/.github/workflows/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml @@ -47,10 +47,6 @@ env: GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} - beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch_test_arguments_1: '' - beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch_test_arguments_2: '' - beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch_test_arguments_3: '' - beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch_test_arguments_4: '' jobs: beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch: @@ -79,10 +75,11 @@ jobs: test-type: load test-language: java argument-file-paths: | - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_ParDo_SparkStructuredStreaming_Batch_10_times.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_ParDo_SparkStructuredStreaming_Batch_200_times.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_ParDo_SparkStructuredStreaming_Batch_10_counters.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_ParDo_SparkStructuredStreaming_Batch_100_counters.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_ParDo_SparkStructuredStreaming_Batch_10_times.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_ParDo_SparkStructuredStreaming_Batch_200_times.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_ParDo_SparkStructuredStreaming_Batch_10_counters.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/java_ParDo_SparkStructuredStreaming_Batch_100_counters.txt + # The env variables are created and populated in the test-arguments-action as "<github.job>_test_arguments_<argument_file_paths_index>" - name: run ParDo SparkStructuredStreaming Batch Java Load Test 1 (10 times) uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml index bc1b41461bcc..504d512802aa 100644 --- a/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml @@ -79,10 +79,10 @@ jobs: test-type: load test-language: python argument-file-paths: | - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_CoGBK_Dataflow_Batch_100b_Single_Key.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_CoGBK_Dataflow_Batch_100b_Multiple_Keys.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_CoGBK_Dataflow_Batch_10kB.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_CoGBK_Dataflow_Batch_2MB.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_CoGBK_Dataflow_Batch_100b_Single_Key.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_CoGBK_Dataflow_Batch_100b_Multiple_Keys.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_CoGBK_Dataflow_Batch_10kB.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_CoGBK_Dataflow_Batch_2MB.txt - name: Set current datetime id: datetime run: | diff --git a/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml index 15d36839de1f..ccd8c371e583 100644 --- a/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml @@ -79,10 +79,10 @@ jobs: test-type: load test-language: python argument-file-paths: | - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Streaming_100b_Single_Key.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Streaming_100b_Multiple_Keys.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Streaming_10kB.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Streaming_2MB.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python-cogbk/python_CoGBK_Dataflow_Streaming_100b_Single_Key.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python-cogbk/python_CoGBK_Dataflow_Streaming_100b_Multiple_Keys.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python-cogbk/python_CoGBK_Dataflow_Streaming_10kB.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python-cogbk/python_CoGBK_Dataflow_Streaming_2MB.txt - name: Set current datetime id: datetime run: | diff --git a/.github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml index e06ed4f94e4f..cdf0c0953165 100644 --- a/.github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml @@ -89,9 +89,9 @@ jobs: test-type: load test-language: python argument-file-paths: | - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Flink_Batch_100b_Single_Key.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Flink_Batch_100b_Multiple_Keys.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Flink_Batch_10kB.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python-cogbk/python_CoGBK_Dataflow_Flink_Batch_100b_Single_Key.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python-cogbk/python_CoGBK_Dataflow_Flink_Batch_100b_Multiple_Keys.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python-cogbk/python_CoGBK_Dataflow_Flink_Batch_10kB.txt - name: Start Flink with parallelism 5 env: FLINK_NUM_WORKERS: 5 diff --git a/.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml index c042cafab864..2a1220c3b885 100644 --- a/.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml @@ -73,19 +73,18 @@ jobs: uses: actions/setup-python@v4 with: python-version: '3.8' - - name: Prepare configs - #Reads config files, excludes comments, appends current date to the job_name parameter - id: set_configs - shell: bash - run: | - CURDATE=$(date '+%m%d%H%M%S' --utc) - CONFIG_ARR=('config_Combine_Python_Batch_2GB_10b.txt' 'config_Combine_Python_Batch_2GB_Fanout_4.txt' 'config_Combine_Python_Batch_2GB_Fanout_8.txt') - for INDEX in ${!CONFIG_ARR[@]} - do - CURCONFIG=$(grep -v "^#.*" ./.github/workflows/load-tests-job-configs/${CONFIG_ARR[INDEX]} | tr '\n' ' ') - CURCONFIG=$(echo "${CURCONFIG/load-tests-python-dataflow-batch-combine-$((INDEX + 1))-/load-tests-python-dataflow-batch-combine-$((INDEX + 1))-$CURDATE}") - echo "prepared_config_$((INDEX + 1))=$CURCONFIG" >> $GITHUB_OUTPUT - done + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: python + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_Combine_Dataflow_Batch_2GB_10b.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_Combine_Dataflow_Batch_2GB_Fanout_4.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_Combine_Dataflow_Batch_2GB_Fanout_8.txt + # The env variables are created and populated in the test-arguments-action as "<github.job>_test_arguments_<argument_file_paths_index>" + - name: get current time + run: echo "NOW_UTC=$(date '+%m%d%H%M%S' --utc)" >> $GITHUB_ENV - name: run Combine Dataflow Batch Python Load Test 1 (10 bytes records) uses: ./.github/actions/gradle-command-self-hosted-action with: @@ -94,7 +93,7 @@ jobs: -PloadTest.mainClass=apache_beam.testing.load_tests.combine_test \ -Prunner=DataflowRunner \ -PpythonVersion=3.8 \ - '-PloadTest.args=${{ steps.set_configs.outputs.prepared_config_1 }}' \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_Combine_Dataflow_Batch_test_arguments_1 }} --job_name=load-tests-python-dataflow-batch-combine-1-${{env.NOW_UTC}}' \ - name: run Combine Dataflow Batch Python Load Test 2 (fanout 4) uses: ./.github/actions/gradle-command-self-hosted-action with: @@ -103,7 +102,7 @@ jobs: -PloadTest.mainClass=apache_beam.testing.load_tests.combine_test \ -Prunner=DataflowRunner \ -PpythonVersion=3.8 \ - '-PloadTest.args=${{ steps.set_configs.outputs.prepared_config_2 }}' \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_Combine_Dataflow_Batch_test_arguments_2 }} --job_name=load-tests-python-dataflow-batch-combine-2-${{env.NOW_UTC}}' \ - name: run Combine Dataflow Batch Python Load Test 3 (fanout 8) uses: ./.github/actions/gradle-command-self-hosted-action with: @@ -112,4 +111,4 @@ jobs: -PloadTest.mainClass=apache_beam.testing.load_tests.combine_test \ -Prunner=DataflowRunner \ -PpythonVersion=3.8 \ - '-PloadTest.args=${{ steps.set_configs.outputs.prepared_config_3 }}' \ No newline at end of file + '-PloadTest.args=${{ env.beam_LoadTests_Python_Combine_Dataflow_Batch_test_arguments_3 }} --job_name=load-tests-python-dataflow-batch-combine-3-${{env.NOW_UTC}}' \ No newline at end of file diff --git a/.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Streaming.yml index c9f05083cfa8..1dabfa77673c 100644 --- a/.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Streaming.yml @@ -79,9 +79,9 @@ jobs: test-type: load test-language: python argument-file-paths: | - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_Combine_Dataflow_Streaming_2GB_10_byte_records.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_Combine_Dataflow_Streaming_2GB_Fanout_4.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_Combine_Dataflow_Streaming_2GB_Fanout_8.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_Combine_Dataflow_Streaming_2GB_10_byte_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_Combine_Dataflow_Streaming_2GB_Fanout_4.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_Combine_Dataflow_Streaming_2GB_Fanout_8.txt - name: get current time run: echo "NOW_UTC=$(date '+%m%d%H%M%S' --utc)" >> $GITHUB_ENV # The env variables are created and populated in the test-arguments-action as "<github.job>_test_arguments_<argument_file_paths_index>" diff --git a/.github/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml index 3ab3e410faef..358a9a5378d9 100644 --- a/.github/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml @@ -89,9 +89,9 @@ jobs: test-type: load test-language: python argument-file-paths: | - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_Combine_Flink_Batch_2GB_10_byte_records.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_Combine_Flink_Batch_2GB_Fanout_4.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_Combine_Flink_Batch_2GB_Fanout_8.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_Combine_Flink_Batch_2GB_10_byte_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_Combine_Flink_Batch_2GB_Fanout_4.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_Combine_Flink_Batch_2GB_Fanout_8.txt - name: Start Flink with parallelism 5 env: FLINK_NUM_WORKERS: 5 diff --git a/.github/workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml b/.github/workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml index d90862ddef3c..8da2bd586045 100644 --- a/.github/workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml @@ -89,8 +89,8 @@ jobs: test-type: load test-language: python argument-file-paths: | - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_Combine_Flink_Streaming_2GB_Fanout_4.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_Combine_Flink_Streaming_2GB_Fanout_8.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_Combine_Flink_Streaming_2GB_Fanout_4.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_Combine_Flink_Streaming_2GB_Fanout_8.txt - name: Start Flink with parallelism 16 env: FLINK_NUM_WORKERS: 16 diff --git a/.github/workflows/beam_LoadTests_Python_FnApiRunner_Microbenchmark.yml b/.github/workflows/beam_LoadTests_Python_FnApiRunner_Microbenchmark.yml index 1483f59472ec..674f37fd3481 100644 --- a/.github/workflows/beam_LoadTests_Python_FnApiRunner_Microbenchmark.yml +++ b/.github/workflows/beam_LoadTests_Python_FnApiRunner_Microbenchmark.yml @@ -79,7 +79,7 @@ jobs: test-type: load test-language: python argument-file-paths: | - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/config_FnApiRunner_Python_Microbenchmark.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_FnApiRunner_Microbenchmark.txt - name: run FnApiRunner Microbenchmark Python Load Test uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml index 92a9d82b79aa..d3af9666a782 100644 --- a/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml @@ -79,11 +79,11 @@ jobs: test-type: load test-language: python argument-file-paths: | - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_2GB_of_10B_records.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_2GB_of_100B_records.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_2GB_of_100kB_records.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_fanout_4_times_with_2GB_10-byte_records_total.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_fanout_8_times_with_2GB_10-byte_records_total.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_GBK_Dataflow_Batch_2GB_of_10B_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_GBK_Dataflow_Batch_2GB_of_100B_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_GBK_Dataflow_Batch_2GB_of_100kB_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_GBK_Dataflow_Batch_fanout_4_times_with_2GB_10-byte_records_total.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_GBK_Dataflow_Batch_fanout_8_times_with_2GB_10-byte_records_total.txt - name: get current time run: echo "NOW_UTC=$(date '+%m%d%H%M%S' --utc)" >> $GITHUB_ENV # The env variables are created and populated in the test-arguments-action as "<github.job>_test_arguments_<argument_file_paths_index>" diff --git a/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml index 9a123f7b4aac..ff796c2db1ef 100644 --- a/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml @@ -79,7 +79,7 @@ jobs: test-type: load test-language: python argument-file-paths: | - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Streaming_2GB_of_100kB_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_GBK_Dataflow_Streaming_2GB_of_100kB_records.txt - name: get current time run: echo "NOW_UTC=$(date '+%m%d%H%M%S' --utc)" >> $GITHUB_ENV # The env variable is created and populated in the test-arguments-action as "<github.job>_test_arguments_<argument_file_paths_index>" diff --git a/.github/workflows/beam_LoadTests_Python_GBK_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Python_GBK_Flink_Batch.yml index 4074a0a7417c..4674e4bb01ed 100644 --- a/.github/workflows/beam_LoadTests_Python_GBK_Flink_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_GBK_Flink_Batch.yml @@ -89,11 +89,11 @@ jobs: test-type: load test-language: python argument-file-paths: | - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_GBK_Flink_Batch_2GB_of_10B_records.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_GBK_Flink_Batch_2GB_of_100B_records.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_GBK_Flink_Batch_fanout_4_times_with_2GB_10-byte_records_total.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_GBK_Flink_Batch_fanout_8_times_with_2GB_10-byte_records_total.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_GBK_Flink_Batch_reiterate_4_times_10kB_values.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_GBK_Flink_Batch_2GB_of_10B_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_GBK_Flink_Batch_2GB_of_100B_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_GBK_Flink_Batch_fanout_4_times_with_2GB_10-byte_records_total.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_GBK_Flink_Batch_fanout_8_times_with_2GB_10-byte_records_total.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_GBK_Flink_Batch_reiterate_4_times_10kB_values.txt - name: get current time run: echo "NOW_UTC=$(date '+%m%d%H%M%S' --utc)" >> $GITHUB_ENV - name: Start Flink with parallelism 5 diff --git a/.github/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Batch.yml index 556e1746f3ee..b5062369c4d9 100644 --- a/.github/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Batch.yml @@ -79,8 +79,8 @@ jobs: test-type: load test-language: python argument-file-paths: | - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_GBK_reiterate_Dataflow_Batch_reiterate_4_times_10kB_values.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_GBK_reiterate_Dataflow_Batch_reiterate_4_times_2MB_values.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_GBK_reiterate_Dataflow_Batch_reiterate_4_times_10kB_values.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_GBK_reiterate_Dataflow_Batch_reiterate_4_times_2MB_values.txt - name: get current time run: echo "NOW_UTC=$(date '+%m%d%H%M%S' --utc)" >> $GITHUB_ENV # The env variables are created and populated in the test-arguments-action as "<github.job>_test_arguments_<argument_file_paths_index>" diff --git a/.github/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Streaming.yml index a1f37bae5133..25fb9fdd27ed 100644 --- a/.github/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Streaming.yml @@ -79,8 +79,8 @@ jobs: test-type: load test-language: python argument-file-paths: | - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_GBK_reiterate_Dataflow_Streaming_reiterate_4_times_10kB_values.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_GBK_reiterate_Dataflow_Streaming_reiterate_4_times_2MB_values.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_GBK_reiterate_Dataflow_Streaming_reiterate_4_times_10kB_values.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_GBK_reiterate_Dataflow_Streaming_reiterate_4_times_2MB_values.txt - name: get current time run: echo "NOW_UTC=$(date '+%m%d%H%M%S' --utc)" >> $GITHUB_ENV # The env variables are created and populated in the test-arguments-action as "<github.job>_test_arguments_<argument_file_paths_index>" diff --git a/.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Batch.yml index 472d494cd87b..303780ac8088 100644 --- a/.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Batch.yml @@ -79,10 +79,10 @@ jobs: test-type: load test-language: python argument-file-paths: | - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_ParDo_Dataflow_Batch_10_Iterations.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_ParDo_Dataflow_Batch_200_Iterations.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_ParDo_Dataflow_Batch_10_Counters.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_ParDo_Dataflow_Batch_100_Counters.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_ParDo_Dataflow_Batch_10_Iterations.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_ParDo_Dataflow_Batch_200_Iterations.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_ParDo_Dataflow_Batch_10_Counters.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_ParDo_Dataflow_Batch_100_Counters.txt - name: Set current datetime id: datetime run: | diff --git a/.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml index cbcd28149aea..c6c7bc322e93 100644 --- a/.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml @@ -79,10 +79,10 @@ jobs: test-type: load test-language: python argument-file-paths: | - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_ParDo_Dataflow_Streaming_10_Iterations.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_ParDo_Dataflow_Streaming_200_Iterations.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_ParDo_Dataflow_Streaming_10_Counters.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_ParDo_Dataflow_Streaming_100_Counters.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_ParDo_Dataflow_Streaming_10_Iterations.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_ParDo_Dataflow_Streaming_200_Iterations.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_ParDo_Dataflow_Streaming_10_Counters.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_ParDo_Dataflow_Streaming_100_Counters.txt - name: Set current datetime id: datetime run: | diff --git a/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml index d78fe22b2858..9c8d86289e6e 100644 --- a/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml @@ -89,9 +89,9 @@ jobs: test-type: load test-language: python argument-file-paths: | - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_ParDo_Flink_Batch_10_Iterations.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_ParDo_Flink_Batch_200_Iterations.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_ParDo_Flink_Batch_10_Counters.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_ParDo_Flink_Batch_10_Iterations.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_ParDo_Flink_Batch_200_Iterations.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_ParDo_Flink_Batch_10_Counters.txt - name: Start Flink with parallelism 5 env: FLINK_NUM_WORKERS: 5 diff --git a/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml b/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml index e30ab4d699cc..067cdd5cd525 100644 --- a/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml @@ -89,11 +89,11 @@ jobs: test-type: load test-language: python argument-file-paths: | - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_ParDo_Flink_Streaming_10_Iterations.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_ParDo_Flink_Streaming_200_Iterations.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_ParDo_Flink_Streaming_10_Counters.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_ParDo_Flink_Streaming_100_Counters.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_ParDo_Flink_Streaming_5_Iterations.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_ParDo_Flink_Streaming_10_Iterations.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_ParDo_Flink_Streaming_200_Iterations.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_ParDo_Flink_Streaming_10_Counters.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_ParDo_Flink_Streaming_100_Counters.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_ParDo_Flink_Streaming_5_Iterations.txt - name: Start Flink with parallelism 5 env: FLINK_NUM_WORKERS: 5 diff --git a/.github/workflows/beam_LoadTests_Python_SideInput_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Python_SideInput_Dataflow_Batch.yml index f62f72343357..3be652ab8891 100644 --- a/.github/workflows/beam_LoadTests_Python_SideInput_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_SideInput_Dataflow_Batch.yml @@ -79,16 +79,16 @@ jobs: test-type: load test-language: python argument-file-paths: | - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1window_1key_percent_dict.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1window_99key_percent_dict.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_10gb_1window_first_iterable.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_10gb_1window_iterable.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1window_first_list.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1window_list.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1000window_1key_percent_dict.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1000window_99key_percent_dict.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_10gb_1000window_first_iterable.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_10gb_1000window_iterable.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_SideInput_Dataflow_Batch_1gb_1window_1key_percent_dict.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_SideInput_Dataflow_Batch_1gb_1window_99key_percent_dict.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_SideInput_Dataflow_Batch_10gb_1window_first_iterable.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_SideInput_Dataflow_Batch_10gb_1window_iterable.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_SideInput_Dataflow_Batch_1gb_1window_first_list.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_SideInput_Dataflow_Batch_1gb_1window_list.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_SideInput_Dataflow_Batch_1gb_1000window_1key_percent_dict.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_SideInput_Dataflow_Batch_1gb_1000window_99key_percent_dict.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_SideInput_Dataflow_Batch_10gb_1000window_first_iterable.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_SideInput_Dataflow_Batch_10gb_1000window_iterable.txt - name: Set current datetime id: datetime run: | diff --git a/.github/workflows/beam_LoadTests_Python_Smoke.yml b/.github/workflows/beam_LoadTests_Python_Smoke.yml index 77d4d65bfe9a..8150956fe8db 100644 --- a/.github/workflows/beam_LoadTests_Python_Smoke.yml +++ b/.github/workflows/beam_LoadTests_Python_Smoke.yml @@ -76,8 +76,8 @@ jobs: test-type: load test-language: python argument-file-paths: | - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_Smoke_GroupByKey_Direct.txt - ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_Smoke_GroupByKey_Dataflow.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_Smoke_GroupByKey_Direct.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_Smoke_GroupByKey_Dataflow.txt - name: Set current datetime id: datetime run: | diff --git a/.github/workflows/beam_PerformanceTests_AvroIOIT.yml b/.github/workflows/beam_PerformanceTests_AvroIOIT.yml index 67b00a98ce8e..011f8a740fbb 100644 --- a/.github/workflows/beam_PerformanceTests_AvroIOIT.yml +++ b/.github/workflows/beam_PerformanceTests_AvroIOIT.yml @@ -47,7 +47,6 @@ env: GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} - beam_PerformanceTests_AvroIOIT_test_arguments_1: '' jobs: beam_PerformanceTests_AvroIOIT: @@ -76,9 +75,10 @@ jobs: test-type: performance test-language: java argument-file-paths: | - ${{ github.workspace }}/.github/workflows/performance-tests-job-configs/config_AvroIOIT.txt + ${{ github.workspace }}/.github/workflows/performance-tests-pipeline-options/avroIOIT.txt arguments: | --filenamePrefix=gs://temp-storage-for-perf-tests/${{ matrix.job_name }}/${{github.run_id}}/ + # The env variables are created and populated in the test-arguments-action as "<github.job>_test_arguments_<argument_file_paths_index>" - name: run integrationTest uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml b/.github/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml index 868d1b283636..c85c8d481434 100644 --- a/.github/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml +++ b/.github/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml @@ -47,7 +47,6 @@ env: GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} - beam_PerformanceTests_AvroIOIT_HDFS_test_arguments_1: '' jobs: beam_PerformanceTests_AvroIOIT_HDFS: @@ -94,10 +93,11 @@ jobs: test-type: performance test-language: java argument-file-paths: | - ${{ github.workspace }}/.github/workflows/performance-tests-job-configs/config_AvroIOIT_HDFS.txt + ${{ github.workspace }}/.github/workflows/performance-tests-pipeline-options/avroIOIT_HDFS.txt arguments: | --filenamePrefix=hdfs://${{ steps.install_hadoop.outputs.hadoop_IP }}:9000/TEXTIO_IT_ --hdfsConfiguration=[{\\\"fs.defaultFS\\\":\\\"hdfs:${{ steps.install_hadoop.outputs.hadoop_IP }}:9000\\\",\\\"dfs.replication\\\":1}] + # The env variables are created and populated in the test-arguments-action as "<github.job>_test_arguments_<argument_file_paths_index>" - name: run integrationTest uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml b/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml index cd56c66c5a04..0f5f69dcd320 100644 --- a/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml +++ b/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml @@ -69,14 +69,16 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - - name: Prepare config - id: set_config - shell: bash - run: | - CURDATE=$(date '+%m%d%H%M%S' --utc) - CURCONFIG=$(grep -v "^#.*" ./.github/workflows/performance-tests-job-configs/config_BigQueryIO_Batch_Java_Avro.txt | tr '\n' ' ') - CONFIGWITHDATE=$(echo "${CURCONFIG/bqio_write_10GB_java_avro_/bqio_write_10GB_java_avro_$CURDATE}") - echo "prepared_config=$CONFIGWITHDATE" >> $GITHUB_OUTPUT + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: performance + test-language: java + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/performance-tests-pipeline-options/bigQueryIO_Batch_Java_Avro.txt + arguments: | + --testBigQueryTable=bqio_write_10GB_java_avro_$(date '+%m%d%H%M%S' --utc) + # The env variables are created and populated in the test-arguments-action as "<github.job>_test_arguments_<argument_file_paths_index>" - name: run Java BigQueryIO Batch Avro Performance Test uses: ./.github/actions/gradle-command-self-hosted-action with: @@ -85,7 +87,7 @@ jobs: --tests org.apache.beam.sdk.bigqueryioperftests.BigQueryIOIT \ --info \ -DintegrationTestRunner=dataflow \ - -DintegrationTestPipelineOptions=${{ steps.set_config.outputs.prepared_config }} \ + -DintegrationTestPipelineOptions='[${{ env.beam_PerformanceTests_BigQueryIO_Batch_Java_Avro_test_arguments_1 }}]' \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 if: failure() diff --git a/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml b/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml index 2e2e00d40ab4..df2f406c4294 100644 --- a/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml +++ b/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml @@ -69,14 +69,16 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - - name: Prepare config - id: set_config - shell: bash - run: | - CURDATE=$(date '+%m%d%H%M%S' --utc) - CURCONFIG=$(grep -v "^#.*" ./.github/workflows/performance-tests-job-configs/config_BigQueryIO_Batch_Java_Json.txt | tr '\n' ' ') - CONFIGWITHDATE=$(echo "${CURCONFIG/bqio_write_10GB_java_json_/bqio_write_10GB_java_json_$CURDATE}") - echo "prepared_config=$CONFIGWITHDATE" >> $GITHUB_OUTPUT + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: performance + test-language: java + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/performance-tests-pipeline-options/bigQueryIO_Batch_Java_Json.txt + arguments: | + --testBigQueryTable=bqio_write_10GB_java_json_$(date '+%m%d%H%M%S' --utc) + # The env variables are created and populated in the test-arguments-action as "<github.job>_test_arguments_<argument_file_paths_index>" - name: run Java BigQueryIO Batch Json Performance Test uses: ./.github/actions/gradle-command-self-hosted-action with: @@ -85,7 +87,7 @@ jobs: --tests org.apache.beam.sdk.bigqueryioperftests.BigQueryIOIT \ --info \ -DintegrationTestRunner=dataflow \ - -DintegrationTestPipelineOptions=${{ steps.set_config.outputs.prepared_config }} \ + -DintegrationTestPipelineOptions='[${{ env.beam_PerformanceTests_BigQueryIO_Batch_Java_Json_test_arguments_1 }}]' \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 if: failure() diff --git a/.github/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml b/.github/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml index 95c4fa86b4e0..cc628012f1a4 100644 --- a/.github/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml +++ b/.github/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml @@ -69,14 +69,16 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - - name: Prepare config - id: set_config - shell: bash - run: | - CURDATE=$(date '+%m%d%H%M%S' --utc) - CURCONFIG=$(grep -v "^#.*" ./.github/workflows/performance-tests-job-configs/config_BigQueryIO_Streaming_Java.txt | tr '\n' ' ') - CONFIGWITHDATE=$(echo "${CURCONFIG/bqio_write_10GB_java_stream_/bqio_write_10GB_java_stream_$CURDATE}") - echo "prepared_config=$CONFIGWITHDATE" >> $GITHUB_OUTPUT + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: performance + test-language: java + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/performance-tests-pipeline-options/bigQueryIO_Streaming_Java.txt + arguments: | + --testBigQueryTable=bqio_write_10GB_java_stream_$(date '+%m%d%H%M%S' --utc) + # The env variables are created and populated in the test-arguments-action as "<github.job>_test_arguments_<argument_file_paths_index>" - name: run Java BigQueryIO Streaming Performance Test uses: ./.github/actions/gradle-command-self-hosted-action with: @@ -85,7 +87,7 @@ jobs: --tests org.apache.beam.sdk.bigqueryioperftests.BigQueryIOIT \ --info \ -DintegrationTestRunner=dataflow \ - -DintegrationTestPipelineOptions=${{ steps.set_config.outputs.prepared_config }} \ + -DintegrationTestPipelineOptions='[${{ env.beam_PerformanceTests_BigQueryIO_Streaming_Java_test_arguments_1 }}]' \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 if: failure() diff --git a/.github/workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml b/.github/workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml index a574cdc4bdb5..dd9477f091af 100644 --- a/.github/workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml +++ b/.github/workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml @@ -47,7 +47,6 @@ env: GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} - beam_PerformanceTests_BiqQueryIO_Read_Python_test_arguments_1: '' jobs: beam_PerformanceTests_BiqQueryIO_Read_Python: @@ -80,9 +79,10 @@ jobs: test-type: load test-language: python argument-file-paths: | - ${{ github.workspace }}/.github/workflows/performance-tests-job-configs/biqQueryIO_Read_Python.txt + ${{ github.workspace }}/.github/workflows/performance-tests-pipeline-options/biqQueryIO_Read_Python.txt arguments: | --job_name=performance-tests-bqio-read-python-10gb$(date '+%m%d%H%M%S' --utc) + # The env variables are created and populated in the test-arguments-action as "<github.job>_test_arguments_<argument_file_paths_index>" - name: Run BigQueryIO Read Performance Test Python uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml b/.github/workflows/beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml index c906c6efc575..32eb2c9301ca 100644 --- a/.github/workflows/beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml +++ b/.github/workflows/beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml @@ -47,7 +47,6 @@ env: GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} - beam_PerformanceTests_BiqQueryIO_Write_Python_Batch_test_arguments_1: '' jobs: beam_PerformanceTests_BiqQueryIO_Write_Python_Batch: @@ -80,9 +79,10 @@ jobs: test-type: load test-language: python argument-file-paths: | - ${{ github.workspace }}/.github/workflows/performance-tests-job-configs/biqQueryIO_Write_Python_Batch.txt + ${{ github.workspace }}/.github/workflows/performance-tests-pipeline-options/biqQueryIO_Write_Python_Batch.txt arguments: | --job_name=performance-tests-bqio-write-python-batch-10gb$(date '+%m%d%H%M%S' --utc) + # The env variables are created and populated in the test-arguments-action as "<github.job>_test_arguments_<argument_file_paths_index>" - name: run BigQueryIO Write Batch Python Performance Test uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PerformanceTests_Cdap.yml b/.github/workflows/beam_PerformanceTests_Cdap.yml index 804ea0a8f454..49184abfd4f3 100644 --- a/.github/workflows/beam_PerformanceTests_Cdap.yml +++ b/.github/workflows/beam_PerformanceTests_Cdap.yml @@ -47,7 +47,6 @@ env: GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} - beam_PerformanceTests_Cdap_test_arguments_1: '' jobs: beam_PerformanceTests_Cdap: @@ -94,9 +93,10 @@ jobs: test-type: performance test-language: java argument-file-paths: | - ${{ github.workspace }}/.github/workflows/performance-tests-job-configs/cdap.txt + ${{ github.workspace }}/.github/workflows/performance-tests-pipeline-options/cdap.txt arguments: | --postgresServerName=${{ steps.install_postgres.outputs.postgres_IP }} + # The env variables are created and populated in the test-arguments-action as "<github.job>_test_arguments_<argument_file_paths_index>" - name: run integrationTest uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml b/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml index cd1f2433284b..7ea42f197708 100644 --- a/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml +++ b/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml @@ -76,7 +76,7 @@ jobs: test-type: performance test-language: java argument-file-paths: | - ${{ github.workspace }}/.github/workflows/performance-tests-job-configs/config_Compressed_TextIOIT.txt + ${{ github.workspace }}/.github/workflows/performance-tests-pipeline-options/compressed_TextIOIT.txt arguments: | --filenamePrefix=gs://temp-storage-for-perf-tests/${{ matrix.job_name }}/${{github.run_id}}/ - name: run integrationTest diff --git a/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml b/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml index 885d6c793c72..38b28ea084ae 100644 --- a/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml +++ b/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml @@ -93,7 +93,7 @@ jobs: test-type: performance test-language: java argument-file-paths: | - ${{ github.workspace }}/.github/workflows/performance-tests-job-configs/config_Compressed_TextIOIT_HDFS.txt + ${{ github.workspace }}/.github/workflows/performance-tests-pipeline-options/compressed_TextIOIT_HDFS.txt arguments: | --filenamePrefix=hdfs://${{ steps.install_hadoop.outputs.hadoop_IP }}:9000/TEXTIO_IT_ --hdfsConfiguration=[{\\\"fs.defaultFS\\\":\\\"hdfs:${{ steps.install_hadoop.outputs.hadoop_IP }}:9000\\\",\\\"dfs.replication\\\":1}] diff --git a/.github/workflows/beam_PerformanceTests_HadoopFormat.yml b/.github/workflows/beam_PerformanceTests_HadoopFormat.yml index 060bb24811c3..9e6160007e9d 100644 --- a/.github/workflows/beam_PerformanceTests_HadoopFormat.yml +++ b/.github/workflows/beam_PerformanceTests_HadoopFormat.yml @@ -47,7 +47,6 @@ env: GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} - beam_PerformanceTests_HadoopFormat_test_arguments_1: '' jobs: beam_PerformanceTests_HadoopFormat: @@ -94,9 +93,10 @@ jobs: test-type: performance test-language: java argument-file-paths: | - ${{ github.workspace }}/.github/workflows/performance-tests-job-configs/hadoopFormat.txt + ${{ github.workspace }}/.github/workflows/performance-tests-pipeline-options/hadoopFormat.txt arguments: | --postgresServerName=${{ steps.install_postgres.outputs.postgres_IP }} + # The env variables are created and populated in the test-arguments-action as "<github.job>_test_arguments_<argument_file_paths_index>" - name: run integrationTest uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PerformanceTests_JDBC.yml b/.github/workflows/beam_PerformanceTests_JDBC.yml index 8d7dba3eb05e..206f7cd199b6 100644 --- a/.github/workflows/beam_PerformanceTests_JDBC.yml +++ b/.github/workflows/beam_PerformanceTests_JDBC.yml @@ -47,7 +47,6 @@ env: GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} - beam_PerformanceTests_JDBC_test_arguments_1: '' jobs: beam_PerformanceTests_JDBC: @@ -94,9 +93,10 @@ jobs: test-type: performance test-language: java argument-file-paths: | - ${{ github.workspace }}/.github/workflows/performance-tests-job-configs/JDBC.txt + ${{ github.workspace }}/.github/workflows/performance-tests-pipeline-options/JDBC.txt arguments: | --postgresServerName=${{ steps.postgres_for_dev.outputs.pfd_ip }} + # The env variables are created and populated in the test-arguments-action as "<github.job>_test_arguments_<argument_file_paths_index>" - name: Run Java JdbcIO Performance Test uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PerformanceTests_Kafka_IO.yml b/.github/workflows/beam_PerformanceTests_Kafka_IO.yml index a69f3400f80a..47cafaac9f81 100644 --- a/.github/workflows/beam_PerformanceTests_Kafka_IO.yml +++ b/.github/workflows/beam_PerformanceTests_Kafka_IO.yml @@ -47,8 +47,6 @@ env: GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} - beam_PerformanceTests_Kafka_IO_test_arguments_1: '' - beam_PerformanceTests_Kafka_IO_test_arguments_2: '' jobs: beam_PerformanceTests_Kafka_IO: @@ -98,10 +96,11 @@ jobs: test-type: performance test-language: java argument-file-paths: | - ${{ github.workspace }}/.github/workflows/performance-tests-job-configs/kafka_IO_Streaming.txt - ${{ github.workspace }}/.github/workflows/performance-tests-job-configs/kafka_IO_Batch.txt + ${{ github.workspace }}/.github/workflows/performance-tests-pipeline-options/kafka_IO_Streaming.txt + ${{ github.workspace }}/.github/workflows/performance-tests-pipeline-options/kafka_IO_Batch.txt arguments: | --kafkaBootstrapServerAddresses=${{ steps.set_brokers.outputs.KAFKA_SERVICE_BROKER_0 }}:${{ env.KAFKA_SERVICE_PORT }},${{ steps.set_brokers.outputs.KAFKA_SERVICE_BROKER_1 }}:${{ env.KAFKA_SERVICE_PORT }},${{ steps.set_brokers.outputs.KAFKA_SERVICE_BROKER_2 }}:${{ env.KAFKA_SERVICE_PORT }} + # The env variables are created and populated in the test-arguments-action as "<github.job>_test_arguments_<argument_file_paths_index>" - name: Run Java KafkaIO Performance Streaming Test uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml b/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml index 0559195a5a7d..a804f66740cf 100644 --- a/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml +++ b/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml @@ -47,7 +47,6 @@ env: GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} - beam_PerformanceTests_ManyFiles_TextIOIT_test_arguments_1: '' jobs: beam_PerformanceTests_ManyFiles_TextIOIT: @@ -76,9 +75,10 @@ jobs: test-type: performance test-language: java argument-file-paths: | - ${{ github.workspace }}/.github/workflows/performance-tests-job-configs/config_ManyFiles_TextIOIT.txt + ${{ github.workspace }}/.github/workflows/performance-tests-pipeline-options/manyFiles_TextIOIT.txt arguments: | --filenamePrefix=gs://temp-storage-for-perf-tests/${{ matrix.job_name }}/${{github.run_id}}/ + # The env variables are created and populated in the test-arguments-action as "<github.job>_test_arguments_<argument_file_paths_index>" - name: run integrationTest uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml b/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml index 33db6119c066..9510a98e1c19 100644 --- a/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml +++ b/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml @@ -47,7 +47,6 @@ env: GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} - beam_PerformanceTests_ManyFiles_TextIOIT_HDFS_test_arguments_1: '' jobs: beam_PerformanceTests_ManyFiles_TextIOIT_HDFS: @@ -93,10 +92,11 @@ jobs: test-type: performance test-language: java argument-file-paths: | - ${{ github.workspace }}/.github/workflows/performance-tests-job-configs/config_ManyFiles_TextIOIT_HDFS.txt + ${{ github.workspace }}/.github/workflows/performance-tests-pipeline-options/manyFiles_TextIOIT_HDFS.txt arguments: | --filenamePrefix=hdfs://${{ steps.install_hadoop.outputs.hadoop_IP }}:9000/TEXTIO_IT_ --hdfsConfiguration=[{\\\"fs.defaultFS\\\":\\\"hdfs:${{ steps.install_hadoop.outputs.hadoop_IP }}:9000\\\",\\\"dfs.replication\\\":1}] + # The env variables are created and populated in the test-arguments-action as "<github.job>_test_arguments_<argument_file_paths_index>" - name: run integrationTest uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PerformanceTests_MongoDBIO_IT.yml b/.github/workflows/beam_PerformanceTests_MongoDBIO_IT.yml index 3b874d819d54..4f3d15f29a17 100644 --- a/.github/workflows/beam_PerformanceTests_MongoDBIO_IT.yml +++ b/.github/workflows/beam_PerformanceTests_MongoDBIO_IT.yml @@ -47,7 +47,6 @@ env: GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} - beam_PerformanceTests_MongoDBIO_IT_test_arguments_1: '' jobs: beam_PerformanceTests_MongoDBIO_IT: @@ -94,9 +93,10 @@ jobs: test-type: performance test-language: java argument-file-paths: | - ${{ github.workspace }}/.github/workflows/performance-tests-job-configs/mongoDBIO_IT.txt + ${{ github.workspace }}/.github/workflows/performance-tests-pipeline-options/mongoDBIO_IT.txt arguments: | --mongoDBHostName=${{ steps.install_mongo.outputs.mongo_IP }} + # The env variables are created and populated in the test-arguments-action as "<github.job>_test_arguments_<argument_file_paths_index>" - name: run integrationTest uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PerformanceTests_ParquetIOIT.yml b/.github/workflows/beam_PerformanceTests_ParquetIOIT.yml index 2c822bb3a12f..879d622c70d6 100644 --- a/.github/workflows/beam_PerformanceTests_ParquetIOIT.yml +++ b/.github/workflows/beam_PerformanceTests_ParquetIOIT.yml @@ -76,7 +76,7 @@ jobs: test-type: performance test-language: java argument-file-paths: | - ${{ github.workspace }}/.github/workflows/performance-tests-job-configs/config_ParquetIOIT.txt + ${{ github.workspace }}/.github/workflows/performance-tests-pipeline-options/parquetIOIT.txt arguments: | --filenamePrefix=gs://temp-storage-for-perf-tests/${{ matrix.job_name }}/${{github.run_id}}/ - name: run integrationTest diff --git a/.github/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml b/.github/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml index ac3e194efd46..e9bc1f3b5db5 100644 --- a/.github/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml +++ b/.github/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml @@ -93,7 +93,7 @@ jobs: test-type: performance test-language: java argument-file-paths: | - ${{ github.workspace }}/.github/workflows/performance-tests-job-configs/config_ParquetIOIT_HDFS.txt + ${{ github.workspace }}/.github/workflows/performance-tests-pipeline-options/parquetIOIT_HDFS.txt arguments: | --filenamePrefix=hdfs://${{ steps.install_hadoop.outputs.hadoop_IP }}:9000/TEXTIO_IT_ --hdfsConfiguration=[{\\\"fs.defaultFS\\\":\\\"hdfs:${{ steps.install_hadoop.outputs.hadoop_IP }}:9000\\\",\\\"dfs.replication\\\":1}] diff --git a/.github/workflows/beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml b/.github/workflows/beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml index 31e0c995aded..17bb5ce81986 100644 --- a/.github/workflows/beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml +++ b/.github/workflows/beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml @@ -47,7 +47,6 @@ env: GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} - beam_PerformanceTests_PubsubIOIT_Python_Streaming_test_arguments_1: '' jobs: beam_PerformanceTests_PubsubIOIT_Python_Streaming: @@ -80,9 +79,10 @@ jobs: test-type: load test-language: python argument-file-paths: | - ${{ github.workspace }}/.github/workflows/performance-tests-job-configs/pubsubIOIT_Python_Streaming.txt + ${{ github.workspace }}/.github/workflows/performance-tests-pipeline-options/pubsubIOIT_Python_Streaming.txt arguments: | --job_name=performance-tests-psio-python-2gb$(date '+%m%d%H%M%S' --utc) + # The env variables are created and populated in the test-arguments-action as "<github.job>_test_arguments_<argument_file_paths_index>" - name: Run test uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml b/.github/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml index 8eab9e68a3e1..32297e41b74e 100644 --- a/.github/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml +++ b/.github/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml @@ -47,7 +47,6 @@ env: GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} - beam_PerformanceTests_SQLBigQueryIO_Batch_Java_test_arguments_1: '' jobs: beam_PerformanceTests_SQLBigQueryIO_Batch_Java: @@ -80,7 +79,8 @@ jobs: test-type: performance test-language: java argument-file-paths: | - ${{ github.workspace }}/.github/workflows/performance-tests-job-configs/SQLBigQueryIO_Batch_Java.txt + ${{ github.workspace }}/.github/workflows/performance-tests-pipeline-options/SQLBigQueryIO_Batch_Java.txt + # The env variables are created and populated in the test-arguments-action as "<github.job>_test_arguments_<argument_file_paths_index>" - name: Run SQLBigQueryIO Batch Performance Test Java uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml b/.github/workflows/beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml index ddea3f6a92ee..4f1b8357e9e3 100644 --- a/.github/workflows/beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml +++ b/.github/workflows/beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml @@ -47,7 +47,6 @@ env: GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} - beam_PerformanceTests_SpannerIO_Read_2GB_Python_test_arguments_1: '' jobs: beam_PerformanceTests_SpannerIO_Read_2GB_Python: @@ -80,9 +79,10 @@ jobs: test-type: load test-language: python argument-file-paths: | - ${{ github.workspace }}/.github/workflows/performance-tests-job-configs/spannerIO_Read_2GB_Python.txt + ${{ github.workspace }}/.github/workflows/performance-tests-pipeline-options/spannerIO_Read_2GB_Python.txt arguments: | --job_name=performance-tests-spanner-read-python-2gb$(date '+%m%d%H%M%S' --utc) + # The env variables are created and populated in the test-arguments-action as "<github.job>_test_arguments_<argument_file_paths_index>" - name: run Performance SpannerIO Read 2GB Test Python uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml b/.github/workflows/beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml index 34066b890eeb..07b336e53348 100644 --- a/.github/workflows/beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml +++ b/.github/workflows/beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml @@ -47,7 +47,6 @@ env: GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} - beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch_test_arguments_1: '' jobs: beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch: @@ -80,9 +79,10 @@ jobs: test-type: load test-language: python argument-file-paths: | - ${{ github.workspace }}/.github/workflows/performance-tests-job-configs/spannerIO_Write_2GB_Python.txt + ${{ github.workspace }}/.github/workflows/performance-tests-pipeline-options/spannerIO_Write_2GB_Python.txt arguments: | --job_name=performance-tests-spannerio-write-python-batch-2gb$(date '+%m%d%H%M%S' --utc) + # The env variables are created and populated in the test-arguments-action as "<github.job>_test_arguments_<argument_file_paths_index>" - name: run Performance SpannerIO Write 2GB Test Python uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PerformanceTests_SparkReceiver_IO.yml b/.github/workflows/beam_PerformanceTests_SparkReceiver_IO.yml index daab6a20a546..bf750acbd31c 100644 --- a/.github/workflows/beam_PerformanceTests_SparkReceiver_IO.yml +++ b/.github/workflows/beam_PerformanceTests_SparkReceiver_IO.yml @@ -47,7 +47,6 @@ env: GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} - beam_PerformanceTests_SparkReceiver_IO_test_arguments_1: '' jobs: beam_PerformanceTests_SparkReceiver_IO: @@ -94,9 +93,10 @@ jobs: test-type: performance test-language: java argument-file-paths: | - ${{ github.workspace }}/.github/workflows/performance-tests-job-configs/sparkReceiver_IO.txt + ${{ github.workspace }}/.github/workflows/performance-tests-pipeline-options/sparkReceiver_IO.txt arguments: | --rabbitMqBootstrapServerAddress=amqp://guest:guest@${{ steps.install_rabbitmq.outputs.rabbitmq_IP }}:5672 + # The env variables are created and populated in the test-arguments-action as "<github.job>_test_arguments_<argument_file_paths_index>" - name: run integrationTest uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PerformanceTests_TFRecordIOIT.yml b/.github/workflows/beam_PerformanceTests_TFRecordIOIT.yml index 7c2a1011694b..7088dab537d9 100644 --- a/.github/workflows/beam_PerformanceTests_TFRecordIOIT.yml +++ b/.github/workflows/beam_PerformanceTests_TFRecordIOIT.yml @@ -47,7 +47,6 @@ env: GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} - beam_PerformanceTests_TFRecordIOIT_test_arguments_1: '' jobs: beam_PerformanceTests_TFRecordIOIT: @@ -76,9 +75,10 @@ jobs: test-type: performance test-language: java argument-file-paths: | - ${{ github.workspace }}/.github/workflows/performance-tests-job-configs/config_TFRecordIOIT.txt + ${{ github.workspace }}/.github/workflows/performance-tests-pipeline-options/TFRecordIOIT.txt arguments: | --filenamePrefix=gs://temp-storage-for-perf-tests/${{ matrix.job_name }}/${{github.run_id}}/ + # The env variables are created and populated in the test-arguments-action as "<github.job>_test_arguments_<argument_file_paths_index>" - name: run integrationTest uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml b/.github/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml index 32f31a29567b..f49be834154b 100644 --- a/.github/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml +++ b/.github/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml @@ -49,7 +49,6 @@ env: GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} - beam_PerformanceTests_TFRecordIOIT_HDFS_test_arguments_1: '' jobs: beam_PerformanceTests_TFRecordIOIT_HDFS: @@ -96,10 +95,11 @@ jobs: test-type: performance test-language: java argument-file-paths: | - ${{ github.workspace }}/.github/workflows/performance-tests-job-configs/TFRecordIOIT_HDFS.txt + ${{ github.workspace }}/.github/workflows/performance-tests-pipeline-options/TFRecordIOIT_HDFS.txt arguments: | --filenamePrefix=hdfs://${{ steps.install_hadoop.outputs.hadoop_IP }}:9000/TEXTIO_IT_ --hdfsConfiguration=[{\\\"fs.defaultFS\\\":\\\"hdfs:${{ steps.install_hadoop.outputs.hadoop_IP }}:9000\\\",\\\"dfs.replication\\\":1}] + # The env variables are created and populated in the test-arguments-action as "<github.job>_test_arguments_<argument_file_paths_index>" - name: run integrationTest uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PerformanceTests_TextIOIT.yml b/.github/workflows/beam_PerformanceTests_TextIOIT.yml index 477a8a08cac7..8be4370b3e4f 100644 --- a/.github/workflows/beam_PerformanceTests_TextIOIT.yml +++ b/.github/workflows/beam_PerformanceTests_TextIOIT.yml @@ -75,7 +75,7 @@ jobs: test-type: performance test-language: java argument-file-paths: | - ${{ github.workspace }}/.github/workflows/performance-tests-job-configs/textIOIT.txt + ${{ github.workspace }}/.github/workflows/performance-tests-pipeline-options/textIOIT.txt arguments: | --filenamePrefix=gs://temp-storage-for-perf-tests/${{ matrix.job_name }}/${{github.run_id}} # The env variable is created and populated in the test-arguments-action as "beam_PerformanceTests_TextIOIT_test_arguments_1" diff --git a/.github/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml b/.github/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml index 8d1683cd1e35..e0f4b0365b6e 100644 --- a/.github/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml +++ b/.github/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml @@ -87,7 +87,7 @@ jobs: test-type: performance test-language: java argument-file-paths: | - ${{ github.workspace }}/.github/workflows/performance-tests-job-configs/textIOIT_HDFS.txt + ${{ github.workspace }}/.github/workflows/performance-tests-pipeline-options/textIOIT_HDFS.txt arguments: | --filenamePrefix=hdfs://${{ steps.install_hadoop.outputs.hadoop_IP }}:9000/TEXTIO_IT_ --hdfsConfiguration=[{\\\"fs.defaultFS\\\":\\\"hdfs:${{ steps.install_hadoop.outputs.hadoop_IP }}:9000\\\",\\\"dfs.replication\\\":1}] diff --git a/.github/workflows/beam_PerformanceTests_TextIOIT_Python.yml b/.github/workflows/beam_PerformanceTests_TextIOIT_Python.yml index 64d4cad53327..b5aecb97889b 100644 --- a/.github/workflows/beam_PerformanceTests_TextIOIT_Python.yml +++ b/.github/workflows/beam_PerformanceTests_TextIOIT_Python.yml @@ -79,7 +79,7 @@ jobs: test-type: load test-language: python argument-file-paths: | - ${{ github.workspace }}/.github/workflows/performance-tests-job-configs/textIOIT_Python.txt + ${{ github.workspace }}/.github/workflows/performance-tests-pipeline-options/textIOIT_Python.txt arguments: | --filename_prefix=gs://temp-storage-for-perf-tests/${{ matrix.job_name }}/${{github.run_id}} # The env variable is created and populated in the test-arguments-action as "beam_PerformanceTests_TextIOIT_Python_test_arguments_1" diff --git a/.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml b/.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml index dee49eb0b5e3..6df9ef42d60a 100644 --- a/.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml +++ b/.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml @@ -47,7 +47,6 @@ env: GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} - beam_PerformanceTests_WordCountIT_PythonVersions_test_arguments_1: '' jobs: beam_PerformanceTests_WordCountIT_PythonVersions: @@ -90,11 +89,12 @@ jobs: test-type: load test-language: python argument-file-paths: | - ${{ github.workspace }}/.github/workflows/performance-tests-job-configs/wordCountIT_Python.txt + ${{ github.workspace }}/.github/workflows/performance-tests-pipeline-options/wordCountIT_Python.txt arguments: | --job_name=performance-tests-wordcount-python${{steps.set_py_ver_clean.outputs.py_ver_clean}}-batch-1gb$(date '+%m%d%H%M%S' --utc) --metrics_table=wordcount_py${{steps.set_py_ver_clean.outputs.py_ver_clean}}_pkb_results --influx_measurement=wordcount_py${{steps.set_py_ver_clean.outputs.py_ver_clean}}_results + # The env variables are created and populated in the test-arguments-action as "<github.job>_test_arguments_<argument_file_paths_index>" - name: Run test uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PerformanceTests_XmlIOIT.yml b/.github/workflows/beam_PerformanceTests_XmlIOIT.yml index 33849aecf994..d610d362cdac 100644 --- a/.github/workflows/beam_PerformanceTests_XmlIOIT.yml +++ b/.github/workflows/beam_PerformanceTests_XmlIOIT.yml @@ -47,7 +47,6 @@ env: GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} - beam_PerformanceTests_XmlIOIT_test_arguments_1: '' jobs: beam_PerformanceTests_XmlIOIT: @@ -76,9 +75,10 @@ jobs: test-type: performance test-language: java argument-file-paths: | - ${{ github.workspace }}/.github/workflows/performance-tests-job-configs/config_XmlIOIT.txt + ${{ github.workspace }}/.github/workflows/performance-tests-pipeline-options/xmlIOIT.txt arguments: | --filenamePrefix=gs://temp-storage-for-perf-tests/${{ matrix.job_name }}/${{github.run_id}}/ + # The env variables are created and populated in the test-arguments-action as "<github.job>_test_arguments_<argument_file_paths_index>" - name: run integrationTest uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml b/.github/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml index ff130dc4d53d..6ceb38ca87a4 100644 --- a/.github/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml +++ b/.github/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml @@ -47,7 +47,6 @@ env: GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} - beam_PerformanceTests_XmlIOIT_HDFS_test_arguments_1: '' jobs: beam_PerformanceTests_XmlIOIT_HDFS: @@ -93,10 +92,11 @@ jobs: test-type: performance test-language: java argument-file-paths: | - ${{ github.workspace }}/.github/workflows/performance-tests-job-configs/config_XmlIOIT_HDFS.txt + ${{ github.workspace }}/.github/workflows/performance-tests-pipeline-options/xmlIOIT_HDFS.txt arguments: | --filenamePrefix=hdfs://${{ steps.install_hadoop.outputs.hadoop_IP }}:9000/TEXTIO_IT_ --hdfsConfiguration=[{\\\"fs.defaultFS\\\":\\\"hdfs:${{ steps.install_hadoop.outputs.hadoop_IP }}:9000\\\",\\\"dfs.replication\\\":1}] + # The env variables are created and populated in the test-arguments-action as "<github.job>_test_arguments_<argument_file_paths_index>" - name: run integrationTest uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml b/.github/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml index bc62013c649d..92787670d93c 100644 --- a/.github/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml +++ b/.github/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml @@ -101,7 +101,7 @@ jobs: test-type: load test-language: python argument-file-paths: | - ${{ github.workspace }}/.github/workflows/performance-tests-job-configs/xlang_KafkaIO_Python.txt + ${{ github.workspace }}/.github/workflows/performance-tests-pipeline-options/xlang_KafkaIO_Python.txt arguments: | --filename_prefix=gs://temp-storage-for-perf-tests/${{ matrix.job_name }}/${{github.run_id}}/ --bootstrap_servers=${{ steps.kafka_ip.outputs.KAFKA_BROKER_0 }}:32400,${{ steps.kafka_ip.outputs.KAFKA_BROKER_1 }}:32400,${{ steps.kafka_ip.outputs.KAFKA_BROKER_2 }}:32400 diff --git a/.github/workflows/load-tests-job-configs/beam_CloudML_Benchmarks_Dataflow_arguments.txt b/.github/workflows/load-tests-pipeline-options/beam_CloudML_Benchmarks_Dataflow_arguments.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/beam_CloudML_Benchmarks_Dataflow_arguments.txt rename to .github/workflows/load-tests-pipeline-options/beam_CloudML_Benchmarks_Dataflow_arguments.txt diff --git a/.github/workflows/load-tests-job-configs/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Imagenet_Classification_Resnet_152.txt b/.github/workflows/load-tests-pipeline-options/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Imagenet_Classification_Resnet_152.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Imagenet_Classification_Resnet_152.txt rename to .github/workflows/load-tests-pipeline-options/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Imagenet_Classification_Resnet_152.txt diff --git a/.github/workflows/load-tests-job-configs/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Imagenet_Classification_Resnet_152_Tesla_T4_GPU.txt b/.github/workflows/load-tests-pipeline-options/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Imagenet_Classification_Resnet_152_Tesla_T4_GPU.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Imagenet_Classification_Resnet_152_Tesla_T4_GPU.txt rename to .github/workflows/load-tests-pipeline-options/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Imagenet_Classification_Resnet_152_Tesla_T4_GPU.txt diff --git a/.github/workflows/load-tests-job-configs/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Language_Modeling_Bert_Base_Uncased.txt b/.github/workflows/load-tests-pipeline-options/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Language_Modeling_Bert_Base_Uncased.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Language_Modeling_Bert_Base_Uncased.txt rename to .github/workflows/load-tests-pipeline-options/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Language_Modeling_Bert_Base_Uncased.txt diff --git a/.github/workflows/load-tests-job-configs/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Language_Modeling_Bert_Large_Uncased.txt b/.github/workflows/load-tests-pipeline-options/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Language_Modeling_Bert_Large_Uncased.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Language_Modeling_Bert_Large_Uncased.txt rename to .github/workflows/load-tests-pipeline-options/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Language_Modeling_Bert_Large_Uncased.txt diff --git a/.github/workflows/load-tests-job-configs/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Vision_Classification_Resnet_101.txt b/.github/workflows/load-tests-pipeline-options/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Vision_Classification_Resnet_101.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Vision_Classification_Resnet_101.txt rename to .github/workflows/load-tests-pipeline-options/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Vision_Classification_Resnet_101.txt diff --git a/.github/workflows/load-tests-job-configs/config_CoGBK_Go_Batch_MultipleKey.txt b/.github/workflows/load-tests-pipeline-options/go_CoGBK_Dataflow_Batch_MultipleKey.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/config_CoGBK_Go_Batch_MultipleKey.txt rename to .github/workflows/load-tests-pipeline-options/go_CoGBK_Dataflow_Batch_MultipleKey.txt diff --git a/.github/workflows/load-tests-job-configs/config_CoGBK_Go_Batch_Reiteration_10KB.txt b/.github/workflows/load-tests-pipeline-options/go_CoGBK_Dataflow_Batch_Reiteration_10KB.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/config_CoGBK_Go_Batch_Reiteration_10KB.txt rename to .github/workflows/load-tests-pipeline-options/go_CoGBK_Dataflow_Batch_Reiteration_10KB.txt diff --git a/.github/workflows/load-tests-job-configs/config_CoGBK_Go_Batch_Reiteration_2MB.txt b/.github/workflows/load-tests-pipeline-options/go_CoGBK_Dataflow_Batch_Reiteration_2MB.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/config_CoGBK_Go_Batch_Reiteration_2MB.txt rename to .github/workflows/load-tests-pipeline-options/go_CoGBK_Dataflow_Batch_Reiteration_2MB.txt diff --git a/.github/workflows/load-tests-job-configs/config_CoGBK_Go_Batch_SingleKey.txt b/.github/workflows/load-tests-pipeline-options/go_CoGBK_Dataflow_Batch_SingleKey.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/config_CoGBK_Go_Batch_SingleKey.txt rename to .github/workflows/load-tests-pipeline-options/go_CoGBK_Dataflow_Batch_SingleKey.txt diff --git a/.github/workflows/load-tests-job-configs/go_CoGBK_Flink_Batch_MultipleKey.txt b/.github/workflows/load-tests-pipeline-options/go_CoGBK_Flink_Batch_MultipleKey.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/go_CoGBK_Flink_Batch_MultipleKey.txt rename to .github/workflows/load-tests-pipeline-options/go_CoGBK_Flink_Batch_MultipleKey.txt diff --git a/.github/workflows/load-tests-job-configs/go_CoGBK_Flink_Batch_Reiteration_10KB.txt b/.github/workflows/load-tests-pipeline-options/go_CoGBK_Flink_Batch_Reiteration_10KB.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/go_CoGBK_Flink_Batch_Reiteration_10KB.txt rename to .github/workflows/load-tests-pipeline-options/go_CoGBK_Flink_Batch_Reiteration_10KB.txt diff --git a/.github/workflows/load-tests-job-configs/go_CoGBK_Flink_Batch_Reiteration_2MB.txt b/.github/workflows/load-tests-pipeline-options/go_CoGBK_Flink_Batch_Reiteration_2MB.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/go_CoGBK_Flink_Batch_Reiteration_2MB.txt rename to .github/workflows/load-tests-pipeline-options/go_CoGBK_Flink_Batch_Reiteration_2MB.txt diff --git a/.github/workflows/load-tests-job-configs/config_Combine_Go_Batch_10b.txt b/.github/workflows/load-tests-pipeline-options/go_Combine_Dataflow_Batch_10b.txt similarity index 78% rename from .github/workflows/load-tests-job-configs/config_Combine_Go_Batch_10b.txt rename to .github/workflows/load-tests-pipeline-options/go_Combine_Dataflow_Batch_10b.txt index b9ad28105903..d97a3731bb97 100644 --- a/.github/workflows/load-tests-job-configs/config_Combine_Go_Batch_10b.txt +++ b/.github/workflows/load-tests-pipeline-options/go_Combine_Dataflow_Batch_10b.txt @@ -1,4 +1,3 @@ -############################################################################### # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file # distributed with this work for additional information @@ -13,10 +12,8 @@ # distributed under the License is distributed on an "AS IS" BASIS, # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. # See the License for the specific language governing permissions and -# limitations under the License. -############################################################################### ---job_name=load-tests-go-dataflow-batch-combine-1- ---project=apache-beam-testing +# limitations under the License. + --region=us-central1 --temp_location=gs://temp-storage-for-perf-tests/loadtests --staging_location=gs://temp-storage-for-perf-tests/loadtests @@ -29,6 +26,4 @@ --autoscaling_algorithm=NONE --environment_type=DOCKER --environment_config=gcr.io/apache-beam-testing/beam-sdk/beam_go_sdk:latest ---influx_db_name=beam_test_metrics ---influx_hostname=http://10.128.0.96:8086 --runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/config_Combine_Go_Batch_Fanout_4.txt b/.github/workflows/load-tests-pipeline-options/go_Combine_Dataflow_Batch_Fanout_4.txt similarity index 78% rename from .github/workflows/load-tests-job-configs/config_Combine_Go_Batch_Fanout_4.txt rename to .github/workflows/load-tests-pipeline-options/go_Combine_Dataflow_Batch_Fanout_4.txt index 5f3a18583270..a64b0c9e7c2c 100644 --- a/.github/workflows/load-tests-job-configs/config_Combine_Go_Batch_Fanout_4.txt +++ b/.github/workflows/load-tests-pipeline-options/go_Combine_Dataflow_Batch_Fanout_4.txt @@ -1,4 +1,3 @@ -############################################################################### # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file # distributed with this work for additional information @@ -13,10 +12,8 @@ # distributed under the License is distributed on an "AS IS" BASIS, # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. # See the License for the specific language governing permissions and -# limitations under the License. -############################################################################### ---job_name=load-tests-go-dataflow-batch-combine-2- ---project=apache-beam-testing +# limitations under the License. + --region=us-central1 --temp_location=gs://temp-storage-for-perf-tests/loadtests --staging_location=gs://temp-storage-for-perf-tests/loadtests @@ -29,6 +26,4 @@ --autoscaling_algorithm=NONE --environment_type=DOCKER --environment_config=gcr.io/apache-beam-testing/beam-sdk/beam_go_sdk:latest ---influx_db_name=beam_test_metrics ---influx_hostname=http://10.128.0.96:8086 --runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/config_Combine_Go_Batch_Fanout_8.txt b/.github/workflows/load-tests-pipeline-options/go_Combine_Dataflow_Batch_Fanout_8.txt similarity index 78% rename from .github/workflows/load-tests-job-configs/config_Combine_Go_Batch_Fanout_8.txt rename to .github/workflows/load-tests-pipeline-options/go_Combine_Dataflow_Batch_Fanout_8.txt index eba65b666a25..23ef62663b1c 100644 --- a/.github/workflows/load-tests-job-configs/config_Combine_Go_Batch_Fanout_8.txt +++ b/.github/workflows/load-tests-pipeline-options/go_Combine_Dataflow_Batch_Fanout_8.txt @@ -1,4 +1,3 @@ -############################################################################### # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file # distributed with this work for additional information @@ -13,10 +12,8 @@ # distributed under the License is distributed on an "AS IS" BASIS, # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. # See the License for the specific language governing permissions and -# limitations under the License. -############################################################################### ---job_name=load-tests-go-dataflow-batch-combine-3- ---project=apache-beam-testing +# limitations under the License. + --region=us-central1 --temp_location=gs://temp-storage-for-perf-tests/loadtests --staging_location=gs://temp-storage-for-perf-tests/loadtests @@ -29,6 +26,4 @@ --autoscaling_algorithm=NONE --environment_type=DOCKER --environment_config=gcr.io/apache-beam-testing/beam-sdk/beam_go_sdk:latest ---influx_db_name=beam_test_metrics ---influx_hostname=http://10.128.0.96:8086 --runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/go_Combine_Flink_Batch_10b.txt b/.github/workflows/load-tests-pipeline-options/go_Combine_Flink_Batch_10b.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/go_Combine_Flink_Batch_10b.txt rename to .github/workflows/load-tests-pipeline-options/go_Combine_Flink_Batch_10b.txt diff --git a/.github/workflows/load-tests-job-configs/go_Combine_Flink_Batch_Fanout_4.txt b/.github/workflows/load-tests-pipeline-options/go_Combine_Flink_Batch_Fanout_4.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/go_Combine_Flink_Batch_Fanout_4.txt rename to .github/workflows/load-tests-pipeline-options/go_Combine_Flink_Batch_Fanout_4.txt diff --git a/.github/workflows/load-tests-job-configs/go_Combine_Flink_Batch_Fanout_8.txt b/.github/workflows/load-tests-pipeline-options/go_Combine_Flink_Batch_Fanout_8.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/go_Combine_Flink_Batch_Fanout_8.txt rename to .github/workflows/load-tests-pipeline-options/go_Combine_Flink_Batch_Fanout_8.txt diff --git a/.github/workflows/load-tests-job-configs/go_GBK_Dataflow_Batch_100b.txt b/.github/workflows/load-tests-pipeline-options/go_GBK_Dataflow_Batch_100b.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/go_GBK_Dataflow_Batch_100b.txt rename to .github/workflows/load-tests-pipeline-options/go_GBK_Dataflow_Batch_100b.txt diff --git a/.github/workflows/load-tests-job-configs/go_GBK_Dataflow_Batch_100kb.txt b/.github/workflows/load-tests-pipeline-options/go_GBK_Dataflow_Batch_100kb.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/go_GBK_Dataflow_Batch_100kb.txt rename to .github/workflows/load-tests-pipeline-options/go_GBK_Dataflow_Batch_100kb.txt diff --git a/.github/workflows/load-tests-job-configs/go_GBK_Dataflow_Batch_10b.txt b/.github/workflows/load-tests-pipeline-options/go_GBK_Dataflow_Batch_10b.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/go_GBK_Dataflow_Batch_10b.txt rename to .github/workflows/load-tests-pipeline-options/go_GBK_Dataflow_Batch_10b.txt diff --git a/.github/workflows/load-tests-job-configs/go_GBK_Dataflow_Batch_Fanout_4.txt b/.github/workflows/load-tests-pipeline-options/go_GBK_Dataflow_Batch_Fanout_4.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/go_GBK_Dataflow_Batch_Fanout_4.txt rename to .github/workflows/load-tests-pipeline-options/go_GBK_Dataflow_Batch_Fanout_4.txt diff --git a/.github/workflows/load-tests-job-configs/go_GBK_Dataflow_Batch_Fanout_8.txt b/.github/workflows/load-tests-pipeline-options/go_GBK_Dataflow_Batch_Fanout_8.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/go_GBK_Dataflow_Batch_Fanout_8.txt rename to .github/workflows/load-tests-pipeline-options/go_GBK_Dataflow_Batch_Fanout_8.txt diff --git a/.github/workflows/load-tests-job-configs/go_GBK_Dataflow_Batch_Reiteration_10KB.txt b/.github/workflows/load-tests-pipeline-options/go_GBK_Dataflow_Batch_Reiteration_10KB.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/go_GBK_Dataflow_Batch_Reiteration_10KB.txt rename to .github/workflows/load-tests-pipeline-options/go_GBK_Dataflow_Batch_Reiteration_10KB.txt diff --git a/.github/workflows/load-tests-job-configs/go_GBK_Dataflow_Batch_Reiteration_2MB.txt b/.github/workflows/load-tests-pipeline-options/go_GBK_Dataflow_Batch_Reiteration_2MB.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/go_GBK_Dataflow_Batch_Reiteration_2MB.txt rename to .github/workflows/load-tests-pipeline-options/go_GBK_Dataflow_Batch_Reiteration_2MB.txt diff --git a/.github/workflows/load-tests-job-configs/go_GBK_Flink_Batch_100b.txt b/.github/workflows/load-tests-pipeline-options/go_GBK_Flink_Batch_100b.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/go_GBK_Flink_Batch_100b.txt rename to .github/workflows/load-tests-pipeline-options/go_GBK_Flink_Batch_100b.txt diff --git a/.github/workflows/load-tests-job-configs/go_GBK_Flink_Batch_100kb.txt b/.github/workflows/load-tests-pipeline-options/go_GBK_Flink_Batch_100kb.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/go_GBK_Flink_Batch_100kb.txt rename to .github/workflows/load-tests-pipeline-options/go_GBK_Flink_Batch_100kb.txt diff --git a/.github/workflows/load-tests-job-configs/go_GBK_Flink_Batch_10b.txt b/.github/workflows/load-tests-pipeline-options/go_GBK_Flink_Batch_10b.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/go_GBK_Flink_Batch_10b.txt rename to .github/workflows/load-tests-pipeline-options/go_GBK_Flink_Batch_10b.txt diff --git a/.github/workflows/load-tests-job-configs/go_GBK_Flink_Batch_Fanout_4.txt b/.github/workflows/load-tests-pipeline-options/go_GBK_Flink_Batch_Fanout_4.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/go_GBK_Flink_Batch_Fanout_4.txt rename to .github/workflows/load-tests-pipeline-options/go_GBK_Flink_Batch_Fanout_4.txt diff --git a/.github/workflows/load-tests-job-configs/go_GBK_Flink_Batch_Fanout_8.txt b/.github/workflows/load-tests-pipeline-options/go_GBK_Flink_Batch_Fanout_8.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/go_GBK_Flink_Batch_Fanout_8.txt rename to .github/workflows/load-tests-pipeline-options/go_GBK_Flink_Batch_Fanout_8.txt diff --git a/.github/workflows/load-tests-job-configs/go_GBK_Flink_Batch_Reiteration_10KB.txt b/.github/workflows/load-tests-pipeline-options/go_GBK_Flink_Batch_Reiteration_10KB.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/go_GBK_Flink_Batch_Reiteration_10KB.txt rename to .github/workflows/load-tests-pipeline-options/go_GBK_Flink_Batch_Reiteration_10KB.txt diff --git a/.github/workflows/load-tests-job-configs/config_ParDo_Go_Batch_100_Counters.txt b/.github/workflows/load-tests-pipeline-options/go_ParDo_Dataflow_Batch_100_Counters.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/config_ParDo_Go_Batch_100_Counters.txt rename to .github/workflows/load-tests-pipeline-options/go_ParDo_Dataflow_Batch_100_Counters.txt diff --git a/.github/workflows/load-tests-job-configs/config_ParDo_Go_Batch_10_Counters.txt b/.github/workflows/load-tests-pipeline-options/go_ParDo_Dataflow_Batch_10_Counters.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/config_ParDo_Go_Batch_10_Counters.txt rename to .github/workflows/load-tests-pipeline-options/go_ParDo_Dataflow_Batch_10_Counters.txt diff --git a/.github/workflows/load-tests-job-configs/config_ParDo_Go_Batch_10_Iterations.txt b/.github/workflows/load-tests-pipeline-options/go_ParDo_Dataflow_Batch_10_Iterations.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/config_ParDo_Go_Batch_10_Iterations.txt rename to .github/workflows/load-tests-pipeline-options/go_ParDo_Dataflow_Batch_10_Iterations.txt diff --git a/.github/workflows/load-tests-job-configs/config_ParDo_Go_Batch_200_Iterations.txt b/.github/workflows/load-tests-pipeline-options/go_ParDo_Dataflow_Batch_200_Iterations.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/config_ParDo_Go_Batch_200_Iterations.txt rename to .github/workflows/load-tests-pipeline-options/go_ParDo_Dataflow_Batch_200_Iterations.txt diff --git a/.github/workflows/load-tests-job-configs/go_ParDo_Flink_Batch_100_counters.txt b/.github/workflows/load-tests-pipeline-options/go_ParDo_Flink_Batch_100_counters.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/go_ParDo_Flink_Batch_100_counters.txt rename to .github/workflows/load-tests-pipeline-options/go_ParDo_Flink_Batch_100_counters.txt diff --git a/.github/workflows/load-tests-job-configs/go_ParDo_Flink_Batch_10_counters.txt b/.github/workflows/load-tests-pipeline-options/go_ParDo_Flink_Batch_10_counters.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/go_ParDo_Flink_Batch_10_counters.txt rename to .github/workflows/load-tests-pipeline-options/go_ParDo_Flink_Batch_10_counters.txt diff --git a/.github/workflows/load-tests-job-configs/go_ParDo_Flink_Batch_10_times.txt b/.github/workflows/load-tests-pipeline-options/go_ParDo_Flink_Batch_10_times.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/go_ParDo_Flink_Batch_10_times.txt rename to .github/workflows/load-tests-pipeline-options/go_ParDo_Flink_Batch_10_times.txt diff --git a/.github/workflows/load-tests-job-configs/go_ParDo_Flink_Batch_200_times.txt b/.github/workflows/load-tests-pipeline-options/go_ParDo_Flink_Batch_200_times.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/go_ParDo_Flink_Batch_200_times.txt rename to .github/workflows/load-tests-pipeline-options/go_ParDo_Flink_Batch_200_times.txt diff --git a/.github/workflows/load-tests-job-configs/config_SideInput_Go_Batch_Dataflow_First_Iterable.txt b/.github/workflows/load-tests-pipeline-options/go_SideInput_Dataflow_Batch_First_Iterable.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/config_SideInput_Go_Batch_Dataflow_First_Iterable.txt rename to .github/workflows/load-tests-pipeline-options/go_SideInput_Dataflow_Batch_First_Iterable.txt diff --git a/.github/workflows/load-tests-job-configs/config_SideInput_Go_Batch_Dataflow_Iterable.txt b/.github/workflows/load-tests-pipeline-options/go_SideInput_Dataflow_Batch_Iterable.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/config_SideInput_Go_Batch_Dataflow_Iterable.txt rename to .github/workflows/load-tests-pipeline-options/go_SideInput_Dataflow_Batch_Iterable.txt diff --git a/.github/workflows/load-tests-job-configs/go_SideInput_Flink_Batch_First_Iterable.txt b/.github/workflows/load-tests-pipeline-options/go_SideInput_Flink_Batch_First_Iterable.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/go_SideInput_Flink_Batch_First_Iterable.txt rename to .github/workflows/load-tests-pipeline-options/go_SideInput_Flink_Batch_First_Iterable.txt diff --git a/.github/workflows/load-tests-job-configs/go_SideInput_Flink_Batch_Iterable.txt b/.github/workflows/load-tests-pipeline-options/go_SideInput_Flink_Batch_Iterable.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/go_SideInput_Flink_Batch_Iterable.txt rename to .github/workflows/load-tests-pipeline-options/go_SideInput_Flink_Batch_Iterable.txt diff --git a/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_Batch_100b_Multiple_Keys.txt b/.github/workflows/load-tests-pipeline-options/java_CoGBK_Dataflow_Batch_100b_Multiple_Keys.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_Batch_100b_Multiple_Keys.txt rename to .github/workflows/load-tests-pipeline-options/java_CoGBK_Dataflow_Batch_100b_Multiple_Keys.txt diff --git a/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_Batch_100b_Single_Key.txt b/.github/workflows/load-tests-pipeline-options/java_CoGBK_Dataflow_Batch_100b_Single_Key.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_Batch_100b_Single_Key.txt rename to .github/workflows/load-tests-pipeline-options/java_CoGBK_Dataflow_Batch_100b_Single_Key.txt diff --git a/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_Batch_10kB.txt b/.github/workflows/load-tests-pipeline-options/java_CoGBK_Dataflow_Batch_10kB.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_Batch_10kB.txt rename to .github/workflows/load-tests-pipeline-options/java_CoGBK_Dataflow_Batch_10kB.txt diff --git a/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_Batch_2MB.txt b/.github/workflows/load-tests-pipeline-options/java_CoGBK_Dataflow_Batch_2MB.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_Batch_2MB.txt rename to .github/workflows/load-tests-pipeline-options/java_CoGBK_Dataflow_Batch_2MB.txt diff --git a/.github/workflows/load-tests-job-configs/config_CoGBK_Java_Streaming_2GB_MultipleKey.txt b/.github/workflows/load-tests-pipeline-options/java_CoGBK_Dataflow_Streaming_2GB_MultipleKey.txt similarity index 81% rename from .github/workflows/load-tests-job-configs/config_CoGBK_Java_Streaming_2GB_MultipleKey.txt rename to .github/workflows/load-tests-pipeline-options/java_CoGBK_Dataflow_Streaming_2GB_MultipleKey.txt index 5fd9518bc8d0..94cbd383a736 100644 --- a/.github/workflows/load-tests-job-configs/config_CoGBK_Java_Streaming_2GB_MultipleKey.txt +++ b/.github/workflows/load-tests-pipeline-options/java_CoGBK_Dataflow_Streaming_2GB_MultipleKey.txt @@ -1,4 +1,3 @@ -############################################################################### # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file # distributed with this work for additional information @@ -13,9 +12,8 @@ # distributed under the License is distributed on an "AS IS" BASIS, # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. # See the License for the specific language governing permissions and -# limitations under the License. -############################################################################### ---project=apache-beam-testing +# limitations under the License. + --region=us-central1 --appName=load_tests_Java_Dataflow_streaming_CoGBK_2 --tempLocation=gs://temp-storage-for-perf-tests/loadtests @@ -29,6 +27,4 @@ --streaming=true --inputWindowDurationSec=1200 --coInputWindowDurationSec=1200 ---influxDatabase=beam_test_metrics ---influxHost=http://10.128.0.96:8086 --runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/config_CoGBK_Java_Streaming_2GB_Reiteration_10KB.txt b/.github/workflows/load-tests-pipeline-options/java_CoGBK_Dataflow_Streaming_2GB_Reiteration_10KB.txt similarity index 81% rename from .github/workflows/load-tests-job-configs/config_CoGBK_Java_Streaming_2GB_Reiteration_10KB.txt rename to .github/workflows/load-tests-pipeline-options/java_CoGBK_Dataflow_Streaming_2GB_Reiteration_10KB.txt index 2840fe75d5af..6b16e4546ff6 100644 --- a/.github/workflows/load-tests-job-configs/config_CoGBK_Java_Streaming_2GB_Reiteration_10KB.txt +++ b/.github/workflows/load-tests-pipeline-options/java_CoGBK_Dataflow_Streaming_2GB_Reiteration_10KB.txt @@ -1,4 +1,3 @@ -############################################################################### # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file # distributed with this work for additional information @@ -13,9 +12,8 @@ # distributed under the License is distributed on an "AS IS" BASIS, # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. # See the License for the specific language governing permissions and -# limitations under the License. -############################################################################### ---project=apache-beam-testing +# limitations under the License. + --region=us-central1 --appName=load_tests_Java_Dataflow_streaming_CoGBK_3 --tempLocation=gs://temp-storage-for-perf-tests/loadtests @@ -29,6 +27,4 @@ --streaming=true --inputWindowDurationSec=1200 --coInputWindowDurationSec=1200 ---influxDatabase=beam_test_metrics ---influxHost=http://10.128.0.96:8086 --runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/config_CoGBK_Java_Streaming_2GB_Reiteration_2MB.txt b/.github/workflows/load-tests-pipeline-options/java_CoGBK_Dataflow_Streaming_2GB_Reiteration_2MB.txt similarity index 81% rename from .github/workflows/load-tests-job-configs/config_CoGBK_Java_Streaming_2GB_Reiteration_2MB.txt rename to .github/workflows/load-tests-pipeline-options/java_CoGBK_Dataflow_Streaming_2GB_Reiteration_2MB.txt index bcc8a36cf31f..d643e21d18cb 100644 --- a/.github/workflows/load-tests-job-configs/config_CoGBK_Java_Streaming_2GB_Reiteration_2MB.txt +++ b/.github/workflows/load-tests-pipeline-options/java_CoGBK_Dataflow_Streaming_2GB_Reiteration_2MB.txt @@ -1,4 +1,3 @@ -############################################################################### # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file # distributed with this work for additional information @@ -13,9 +12,8 @@ # distributed under the License is distributed on an "AS IS" BASIS, # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. # See the License for the specific language governing permissions and -# limitations under the License. -############################################################################### ---project=apache-beam-testing +# limitations under the License. + --region=us-central1 --appName=load_tests_Java_Dataflow_streaming_CoGBK_4 --tempLocation=gs://temp-storage-for-perf-tests/loadtests @@ -29,6 +27,4 @@ --streaming=true --inputWindowDurationSec=1200 --coInputWindowDurationSec=1200 ---influxDatabase=beam_test_metrics ---influxHost=http://10.128.0.96:8086 --runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/config_CoGBK_Java_Streaming_2GB_SingleKey.txt b/.github/workflows/load-tests-pipeline-options/java_CoGBK_Dataflow_Streaming_2GB_SingleKey.txt similarity index 81% rename from .github/workflows/load-tests-job-configs/config_CoGBK_Java_Streaming_2GB_SingleKey.txt rename to .github/workflows/load-tests-pipeline-options/java_CoGBK_Dataflow_Streaming_2GB_SingleKey.txt index afae1a1bd6bf..82c6612e920a 100644 --- a/.github/workflows/load-tests-job-configs/config_CoGBK_Java_Streaming_2GB_SingleKey.txt +++ b/.github/workflows/load-tests-pipeline-options/java_CoGBK_Dataflow_Streaming_2GB_SingleKey.txt @@ -1,4 +1,3 @@ -############################################################################### # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file # distributed with this work for additional information @@ -13,9 +12,8 @@ # distributed under the License is distributed on an "AS IS" BASIS, # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. # See the License for the specific language governing permissions and -# limitations under the License. -############################################################################### ---project=apache-beam-testing +# limitations under the License. + --region=us-central1 --appName=load_tests_Java_Dataflow_streaming_CoGBK_1 --tempLocation=gs://temp-storage-for-perf-tests/loadtests @@ -29,6 +27,4 @@ --streaming=true --inputWindowDurationSec=1200 --coInputWindowDurationSec=1200 ---influxDatabase=beam_test_metrics ---influxHost=http://10.128.0.96:8086 --runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_V2_Streaming_Java_100b_Multiple_Keys.txt b/.github/workflows/load-tests-pipeline-options/java_CoGBK_Dataflow_V2_Streaming_Java_100b_Multiple_Keys.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_V2_Streaming_Java_100b_Multiple_Keys.txt rename to .github/workflows/load-tests-pipeline-options/java_CoGBK_Dataflow_V2_Streaming_Java_100b_Multiple_Keys.txt diff --git a/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_V2_Streaming_Java_100b_Single_Key.txt b/.github/workflows/load-tests-pipeline-options/java_CoGBK_Dataflow_V2_Streaming_Java_100b_Single_Key.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_V2_Streaming_Java_100b_Single_Key.txt rename to .github/workflows/load-tests-pipeline-options/java_CoGBK_Dataflow_V2_Streaming_Java_100b_Single_Key.txt diff --git a/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_V2_Streaming_Java_10kB.txt b/.github/workflows/load-tests-pipeline-options/java_CoGBK_Dataflow_V2_Streaming_Java_10kB.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_V2_Streaming_Java_10kB.txt rename to .github/workflows/load-tests-pipeline-options/java_CoGBK_Dataflow_V2_Streaming_Java_10kB.txt diff --git a/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_V2_Streaming_Java_2MB.txt b/.github/workflows/load-tests-pipeline-options/java_CoGBK_Dataflow_V2_Streaming_Java_2MB.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_V2_Streaming_Java_2MB.txt rename to .github/workflows/load-tests-pipeline-options/java_CoGBK_Dataflow_V2_Streaming_Java_2MB.txt diff --git a/.github/workflows/load-tests-job-configs/java_CoGBK_SparkStructuredStreaming_Batch_100b_Multiple_Keys.txt b/.github/workflows/load-tests-pipeline-options/java_CoGBK_SparkStructuredStreaming_Batch_100b_Multiple_Keys.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_CoGBK_SparkStructuredStreaming_Batch_100b_Multiple_Keys.txt rename to .github/workflows/load-tests-pipeline-options/java_CoGBK_SparkStructuredStreaming_Batch_100b_Multiple_Keys.txt diff --git a/.github/workflows/load-tests-job-configs/java_CoGBK_SparkStructuredStreaming_Batch_100b_Single_Key.txt b/.github/workflows/load-tests-pipeline-options/java_CoGBK_SparkStructuredStreaming_Batch_100b_Single_Key.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_CoGBK_SparkStructuredStreaming_Batch_100b_Single_Key.txt rename to .github/workflows/load-tests-pipeline-options/java_CoGBK_SparkStructuredStreaming_Batch_100b_Single_Key.txt diff --git a/.github/workflows/load-tests-job-configs/java_CoGBK_SparkStructuredStreaming_Batch_10kB.txt b/.github/workflows/load-tests-pipeline-options/java_CoGBK_SparkStructuredStreaming_Batch_10kB.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_CoGBK_SparkStructuredStreaming_Batch_10kB.txt rename to .github/workflows/load-tests-pipeline-options/java_CoGBK_SparkStructuredStreaming_Batch_10kB.txt diff --git a/.github/workflows/load-tests-job-configs/java_CoGBK_SparkStructuredStreaming_Batch_2MB.txt b/.github/workflows/load-tests-pipeline-options/java_CoGBK_SparkStructuredStreaming_Batch_2MB.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_CoGBK_SparkStructuredStreaming_Batch_2MB.txt rename to .github/workflows/load-tests-pipeline-options/java_CoGBK_SparkStructuredStreaming_Batch_2MB.txt diff --git a/.github/workflows/load-tests-job-configs/config_Combine_Java_Dataflow_Batch_10b.txt b/.github/workflows/load-tests-pipeline-options/java_Combine_Dataflow_Batch_10b.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/config_Combine_Java_Dataflow_Batch_10b.txt rename to .github/workflows/load-tests-pipeline-options/java_Combine_Dataflow_Batch_10b.txt diff --git a/.github/workflows/load-tests-job-configs/config_Combine_Java_Dataflow_Batch_Fanout_4.txt b/.github/workflows/load-tests-pipeline-options/java_Combine_Dataflow_Batch_Fanout_4.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/config_Combine_Java_Dataflow_Batch_Fanout_4.txt rename to .github/workflows/load-tests-pipeline-options/java_Combine_Dataflow_Batch_Fanout_4.txt diff --git a/.github/workflows/load-tests-job-configs/config_Combine_Java_Dataflow_Batch_Fanout_8.txt b/.github/workflows/load-tests-pipeline-options/java_Combine_Dataflow_Batch_Fanout_8.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/config_Combine_Java_Dataflow_Batch_Fanout_8.txt rename to .github/workflows/load-tests-pipeline-options/java_Combine_Dataflow_Batch_Fanout_8.txt diff --git a/.github/workflows/load-tests-job-configs/java_Combine_Dataflow_Streaming_10b.txt b/.github/workflows/load-tests-pipeline-options/java_Combine_Dataflow_Streaming_10b.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_Combine_Dataflow_Streaming_10b.txt rename to .github/workflows/load-tests-pipeline-options/java_Combine_Dataflow_Streaming_10b.txt diff --git a/.github/workflows/load-tests-job-configs/java_Combine_Dataflow_Streaming_Fanout_4.txt b/.github/workflows/load-tests-pipeline-options/java_Combine_Dataflow_Streaming_Fanout_4.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_Combine_Dataflow_Streaming_Fanout_4.txt rename to .github/workflows/load-tests-pipeline-options/java_Combine_Dataflow_Streaming_Fanout_4.txt diff --git a/.github/workflows/load-tests-job-configs/java_Combine_Dataflow_Streaming_Fanout_8.txt b/.github/workflows/load-tests-pipeline-options/java_Combine_Dataflow_Streaming_Fanout_8.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_Combine_Dataflow_Streaming_Fanout_8.txt rename to .github/workflows/load-tests-pipeline-options/java_Combine_Dataflow_Streaming_Fanout_8.txt diff --git a/.github/workflows/load-tests-job-configs/java_Combine_SparkStructuredStreaming_Batch_10b.txt b/.github/workflows/load-tests-pipeline-options/java_Combine_SparkStructuredStreaming_Batch_10b.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_Combine_SparkStructuredStreaming_Batch_10b.txt rename to .github/workflows/load-tests-pipeline-options/java_Combine_SparkStructuredStreaming_Batch_10b.txt diff --git a/.github/workflows/load-tests-job-configs/java_Combine_SparkStructuredStreaming_Batch_Fanout_4.txt b/.github/workflows/load-tests-pipeline-options/java_Combine_SparkStructuredStreaming_Batch_Fanout_4.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_Combine_SparkStructuredStreaming_Batch_Fanout_4.txt rename to .github/workflows/load-tests-pipeline-options/java_Combine_SparkStructuredStreaming_Batch_Fanout_4.txt diff --git a/.github/workflows/load-tests-job-configs/java_Combine_SparkStructuredStreaming_Batch_Fanout_8.txt b/.github/workflows/load-tests-pipeline-options/java_Combine_SparkStructuredStreaming_Batch_Fanout_8.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_Combine_SparkStructuredStreaming_Batch_Fanout_8.txt rename to .github/workflows/load-tests-pipeline-options/java_Combine_SparkStructuredStreaming_Batch_Fanout_8.txt diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_2GB_of_100B_records.txt b/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_Batch_2GB_of_100B_records.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_2GB_of_100B_records.txt rename to .github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_Batch_2GB_of_100B_records.txt diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_2GB_of_100kB_records.txt b/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_Batch_2GB_of_100kB_records.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_2GB_of_100kB_records.txt rename to .github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_Batch_2GB_of_100kB_records.txt diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_2GB_of_10B_records.txt b/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_Batch_2GB_of_10B_records.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_2GB_of_10B_records.txt rename to .github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_Batch_2GB_of_10B_records.txt diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_fanout_4_times_with_2GB_10-byte_records_total.txt b/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_Batch_fanout_4_times_with_2GB_10-byte_records_total.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_fanout_4_times_with_2GB_10-byte_records_total.txt rename to .github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_Batch_fanout_4_times_with_2GB_10-byte_records_total.txt diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_fanout_8_times_with_2GB_10-byte_records_total.txt b/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_Batch_fanout_8_times_with_2GB_10-byte_records_total.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_fanout_8_times_with_2GB_10-byte_records_total.txt rename to .github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_Batch_fanout_8_times_with_2GB_10-byte_records_total.txt diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_reiterate_4_times_10kB_values.txt b/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_Batch_reiterate_4_times_10kB_values.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_reiterate_4_times_10kB_values.txt rename to .github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_Batch_reiterate_4_times_10kB_values.txt diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_reiterate_4_times_2MB_values.txt b/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_Batch_reiterate_4_times_2MB_values.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_reiterate_4_times_2MB_values.txt rename to .github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_Batch_reiterate_4_times_2MB_values.txt diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_2GB_of_100B_records.txt b/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_Streaming_2GB_of_100B_records.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_2GB_of_100B_records.txt rename to .github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_Streaming_2GB_of_100B_records.txt diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_2GB_of_100kB_records.txt b/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_Streaming_2GB_of_100kB_records.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_2GB_of_100kB_records.txt rename to .github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_Streaming_2GB_of_100kB_records.txt diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_2GB_of_10B_records.txt b/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_Streaming_2GB_of_10B_records.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_2GB_of_10B_records.txt rename to .github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_Streaming_2GB_of_10B_records.txt diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_fanout_4_times_with_2GB_10-byte_records_total.txt b/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_Streaming_fanout_4_times_with_2GB_10-byte_records_total.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_fanout_4_times_with_2GB_10-byte_records_total.txt rename to .github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_Streaming_fanout_4_times_with_2GB_10-byte_records_total.txt diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_fanout_8_times_with_2GB_10-byte_records_total.txt b/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_Streaming_fanout_8_times_with_2GB_10-byte_records_total.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_fanout_8_times_with_2GB_10-byte_records_total.txt rename to .github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_Streaming_fanout_8_times_with_2GB_10-byte_records_total.txt diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_reiterate_4_times_10kB_values.txt b/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_Streaming_reiterate_4_times_10kB_values.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_reiterate_4_times_10kB_values.txt rename to .github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_Streaming_reiterate_4_times_10kB_values.txt diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_reiterate_4_times_2MB_values.txt b/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_Streaming_reiterate_4_times_2MB_values.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_reiterate_4_times_2MB_values.txt rename to .github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_Streaming_reiterate_4_times_2MB_values.txt diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_2GB_of_100B_records.txt b/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Batch_Java11_2GB_of_100B_records.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_2GB_of_100B_records.txt rename to .github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Batch_Java11_2GB_of_100B_records.txt diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_2GB_of_100kB_records.txt b/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Batch_Java11_2GB_of_100kB_records.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_2GB_of_100kB_records.txt rename to .github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Batch_Java11_2GB_of_100kB_records.txt diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_2GB_of_10B_records.txt b/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Batch_Java11_2GB_of_10B_records.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_2GB_of_10B_records.txt rename to .github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Batch_Java11_2GB_of_10B_records.txt diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_fanout_4_times_with_2GB_10-byte_records_total.txt b/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Batch_Java11_fanout_4_times_with_2GB_10-byte_records_total.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_fanout_4_times_with_2GB_10-byte_records_total.txt rename to .github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Batch_Java11_fanout_4_times_with_2GB_10-byte_records_total.txt diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_fanout_8_times_with_2GB_10-byte_records_total.txt b/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Batch_Java11_fanout_8_times_with_2GB_10-byte_records_total.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_fanout_8_times_with_2GB_10-byte_records_total.txt rename to .github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Batch_Java11_fanout_8_times_with_2GB_10-byte_records_total.txt diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_reiterate_4_times_10kB_values.txt b/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Batch_Java11_reiterate_4_times_10kB_values.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_reiterate_4_times_10kB_values.txt rename to .github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Batch_Java11_reiterate_4_times_10kB_values.txt diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_reiterate_4_times_2MB_values.txt b/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Batch_Java11_reiterate_4_times_2MB_values.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_reiterate_4_times_2MB_values.txt rename to .github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Batch_Java11_reiterate_4_times_2MB_values.txt diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_2GB_of_100B_records.txt b/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Batch_Java17_2GB_of_100B_records.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_2GB_of_100B_records.txt rename to .github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Batch_Java17_2GB_of_100B_records.txt diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_2GB_of_100kB_records.txt b/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Batch_Java17_2GB_of_100kB_records.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_2GB_of_100kB_records.txt rename to .github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Batch_Java17_2GB_of_100kB_records.txt diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_2GB_of_10B_records.txt b/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Batch_Java17_2GB_of_10B_records.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_2GB_of_10B_records.txt rename to .github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Batch_Java17_2GB_of_10B_records.txt diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_fanout_4_times_with_2GB_10-byte_records_total.txt b/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Batch_Java17_fanout_4_times_with_2GB_10-byte_records_total.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_fanout_4_times_with_2GB_10-byte_records_total.txt rename to .github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Batch_Java17_fanout_4_times_with_2GB_10-byte_records_total.txt diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_fanout_8_times_with_2GB_10-byte_records_total.txt b/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Batch_Java17_fanout_8_times_with_2GB_10-byte_records_total.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_fanout_8_times_with_2GB_10-byte_records_total.txt rename to .github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Batch_Java17_fanout_8_times_with_2GB_10-byte_records_total.txt diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_reiterate_4_times_10kB_values.txt b/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Batch_Java17_reiterate_4_times_10kB_values.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_reiterate_4_times_10kB_values.txt rename to .github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Batch_Java17_reiterate_4_times_10kB_values.txt diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_reiterate_4_times_2MB_values.txt b/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Batch_Java17_reiterate_4_times_2MB_values.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_reiterate_4_times_2MB_values.txt rename to .github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Batch_Java17_reiterate_4_times_2MB_values.txt diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_2GB_of_100B_records.txt b/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Streaming_Java11_2GB_of_100B_records.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_2GB_of_100B_records.txt rename to .github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Streaming_Java11_2GB_of_100B_records.txt diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_2GB_of_100kB_records.txt b/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Streaming_Java11_2GB_of_100kB_records.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_2GB_of_100kB_records.txt rename to .github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Streaming_Java11_2GB_of_100kB_records.txt diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_2GB_of_10B_records.txt b/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Streaming_Java11_2GB_of_10B_records.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_2GB_of_10B_records.txt rename to .github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Streaming_Java11_2GB_of_10B_records.txt diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_fanout_4_times_with_2GB_10-byte_records_total.txt b/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Streaming_Java11_fanout_4_times_with_2GB_10-byte_records_total.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_fanout_4_times_with_2GB_10-byte_records_total.txt rename to .github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Streaming_Java11_fanout_4_times_with_2GB_10-byte_records_total.txt diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_fanout_8_times_with_2GB_10-byte_records_total.txt b/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Streaming_Java11_fanout_8_times_with_2GB_10-byte_records_total.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_fanout_8_times_with_2GB_10-byte_records_total.txt rename to .github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Streaming_Java11_fanout_8_times_with_2GB_10-byte_records_total.txt diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_reiterate_4_times_10kB_values.txt b/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Streaming_Java11_reiterate_4_times_10kB_values.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_reiterate_4_times_10kB_values.txt rename to .github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Streaming_Java11_reiterate_4_times_10kB_values.txt diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_reiterate_4_times_2MB_values.txt b/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Streaming_Java11_reiterate_4_times_2MB_values.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_reiterate_4_times_2MB_values.txt rename to .github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Streaming_Java11_reiterate_4_times_2MB_values.txt diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_2GB_of_100B_records.txt b/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Streaming_Java17_2GB_of_100B_records.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_2GB_of_100B_records.txt rename to .github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Streaming_Java17_2GB_of_100B_records.txt diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_2GB_of_100kB_records.txt b/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Streaming_Java17_2GB_of_100kB_records.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_2GB_of_100kB_records.txt rename to .github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Streaming_Java17_2GB_of_100kB_records.txt diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_2GB_of_10B_records.txt b/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Streaming_Java17_2GB_of_10B_records.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_2GB_of_10B_records.txt rename to .github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Streaming_Java17_2GB_of_10B_records.txt diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_fanout_4_times_with_2GB_10-byte_records_total.txt b/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Streaming_Java17_fanout_4_times_with_2GB_10-byte_records_total.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_fanout_4_times_with_2GB_10-byte_records_total.txt rename to .github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Streaming_Java17_fanout_4_times_with_2GB_10-byte_records_total.txt diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_fanout_8_times_with_2GB_10-byte_records_total.txt b/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Streaming_Java17_fanout_8_times_with_2GB_10-byte_records_total.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_fanout_8_times_with_2GB_10-byte_records_total.txt rename to .github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Streaming_Java17_fanout_8_times_with_2GB_10-byte_records_total.txt diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_reiterate_4_times_10kB_values.txt b/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Streaming_Java17_reiterate_4_times_10kB_values.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_reiterate_4_times_10kB_values.txt rename to .github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Streaming_Java17_reiterate_4_times_10kB_values.txt diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_reiterate_4_times_2MB_values.txt b/.github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Streaming_Java17_reiterate_4_times_2MB_values.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_reiterate_4_times_2MB_values.txt rename to .github/workflows/load-tests-pipeline-options/java_GBK_Dataflow_V2_Streaming_Java17_reiterate_4_times_2MB_values.txt diff --git a/.github/workflows/load-tests-job-configs/Java_GBK_SparkStructuredStreaming_Batch_2GB_of_100B_records.txt b/.github/workflows/load-tests-pipeline-options/java_GBK_SparkStructuredStreaming_Batch_2GB_of_100B_records.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/Java_GBK_SparkStructuredStreaming_Batch_2GB_of_100B_records.txt rename to .github/workflows/load-tests-pipeline-options/java_GBK_SparkStructuredStreaming_Batch_2GB_of_100B_records.txt diff --git a/.github/workflows/load-tests-job-configs/Java_GBK_SparkStructuredStreaming_Batch_2GB_of_100kB_records.txt b/.github/workflows/load-tests-pipeline-options/java_GBK_SparkStructuredStreaming_Batch_2GB_of_100kB_records.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/Java_GBK_SparkStructuredStreaming_Batch_2GB_of_100kB_records.txt rename to .github/workflows/load-tests-pipeline-options/java_GBK_SparkStructuredStreaming_Batch_2GB_of_100kB_records.txt diff --git a/.github/workflows/load-tests-job-configs/Java_GBK_SparkStructuredStreaming_Batch_2GB_of_10B_records.txt b/.github/workflows/load-tests-pipeline-options/java_GBK_SparkStructuredStreaming_Batch_2GB_of_10B_records.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/Java_GBK_SparkStructuredStreaming_Batch_2GB_of_10B_records.txt rename to .github/workflows/load-tests-pipeline-options/java_GBK_SparkStructuredStreaming_Batch_2GB_of_10B_records.txt diff --git a/.github/workflows/load-tests-job-configs/Java_GBK_SparkStructuredStreaming_Batch_fanout_4_times_with_2GB_10-byte_records_total.txt b/.github/workflows/load-tests-pipeline-options/java_GBK_SparkStructuredStreaming_Batch_fanout_4_times_with_2GB_10-byte_records_total.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/Java_GBK_SparkStructuredStreaming_Batch_fanout_4_times_with_2GB_10-byte_records_total.txt rename to .github/workflows/load-tests-pipeline-options/java_GBK_SparkStructuredStreaming_Batch_fanout_4_times_with_2GB_10-byte_records_total.txt diff --git a/.github/workflows/load-tests-job-configs/Java_GBK_SparkStructuredStreaming_Batch_fanout_8_times_with_2GB_10-byte_records_total.txt b/.github/workflows/load-tests-pipeline-options/java_GBK_SparkStructuredStreaming_Batch_fanout_8_times_with_2GB_10-byte_records_total.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/Java_GBK_SparkStructuredStreaming_Batch_fanout_8_times_with_2GB_10-byte_records_total.txt rename to .github/workflows/load-tests-pipeline-options/java_GBK_SparkStructuredStreaming_Batch_fanout_8_times_with_2GB_10-byte_records_total.txt diff --git a/.github/workflows/load-tests-job-configs/Java_GBK_SparkStructuredStreaming_Batch_reiterate_4_times_10kB_values.txt b/.github/workflows/load-tests-pipeline-options/java_GBK_SparkStructuredStreaming_Batch_reiterate_4_times_10kB_values.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/Java_GBK_SparkStructuredStreaming_Batch_reiterate_4_times_10kB_values.txt rename to .github/workflows/load-tests-pipeline-options/java_GBK_SparkStructuredStreaming_Batch_reiterate_4_times_10kB_values.txt diff --git a/.github/workflows/load-tests-job-configs/Java_GBK_SparkStructuredStreaming_Batch_reiterate_4_times_2MB_values.txt b/.github/workflows/load-tests-pipeline-options/java_GBK_SparkStructuredStreaming_Batch_reiterate_4_times_2MB_values.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/Java_GBK_SparkStructuredStreaming_Batch_reiterate_4_times_2MB_values.txt rename to .github/workflows/load-tests-pipeline-options/java_GBK_SparkStructuredStreaming_Batch_reiterate_4_times_2MB_values.txt diff --git a/.github/workflows/load-tests-job-configs/java_LoadTests_Combine_Smoke_CombineLoadTest_load_test_Dataflow-1.txt b/.github/workflows/load-tests-pipeline-options/java_LoadTests_Combine_Smoke_CombineLoadTest_load_test_Dataflow-1.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_LoadTests_Combine_Smoke_CombineLoadTest_load_test_Dataflow-1.txt rename to .github/workflows/load-tests-pipeline-options/java_LoadTests_Combine_Smoke_CombineLoadTest_load_test_Dataflow-1.txt diff --git a/.github/workflows/load-tests-job-configs/java_LoadTests_Combine_Smoke_CombineLoadTest_load_test_Dataflow-2.txt b/.github/workflows/load-tests-pipeline-options/java_LoadTests_Combine_Smoke_CombineLoadTest_load_test_Dataflow-2.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_LoadTests_Combine_Smoke_CombineLoadTest_load_test_Dataflow-2.txt rename to .github/workflows/load-tests-pipeline-options/java_LoadTests_Combine_Smoke_CombineLoadTest_load_test_Dataflow-2.txt diff --git a/.github/workflows/load-tests-job-configs/java_LoadTests_Combine_Smoke_CombineLoadTest_load_test_Dataflow-3.txt b/.github/workflows/load-tests-pipeline-options/java_LoadTests_Combine_Smoke_CombineLoadTest_load_test_Dataflow-3.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_LoadTests_Combine_Smoke_CombineLoadTest_load_test_Dataflow-3.txt rename to .github/workflows/load-tests-pipeline-options/java_LoadTests_Combine_Smoke_CombineLoadTest_load_test_Dataflow-3.txt diff --git a/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_Batch_100_counters.txt b/.github/workflows/load-tests-pipeline-options/java_ParDo_Dataflow_Batch_100_counters.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_ParDo_Dataflow_Batch_100_counters.txt rename to .github/workflows/load-tests-pipeline-options/java_ParDo_Dataflow_Batch_100_counters.txt diff --git a/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_Batch_10_counters.txt b/.github/workflows/load-tests-pipeline-options/java_ParDo_Dataflow_Batch_10_counters.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_ParDo_Dataflow_Batch_10_counters.txt rename to .github/workflows/load-tests-pipeline-options/java_ParDo_Dataflow_Batch_10_counters.txt diff --git a/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_Batch_10_times.txt b/.github/workflows/load-tests-pipeline-options/java_ParDo_Dataflow_Batch_10_times.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_ParDo_Dataflow_Batch_10_times.txt rename to .github/workflows/load-tests-pipeline-options/java_ParDo_Dataflow_Batch_10_times.txt diff --git a/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_Batch_200_times.txt b/.github/workflows/load-tests-pipeline-options/java_ParDo_Dataflow_Batch_200_times.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_ParDo_Dataflow_Batch_200_times.txt rename to .github/workflows/load-tests-pipeline-options/java_ParDo_Dataflow_Batch_200_times.txt diff --git a/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_Streaming_100_counters.txt b/.github/workflows/load-tests-pipeline-options/java_ParDo_Dataflow_Streaming_100_counters.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_ParDo_Dataflow_Streaming_100_counters.txt rename to .github/workflows/load-tests-pipeline-options/java_ParDo_Dataflow_Streaming_100_counters.txt diff --git a/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_Streaming_10_counters.txt b/.github/workflows/load-tests-pipeline-options/java_ParDo_Dataflow_Streaming_10_counters.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_ParDo_Dataflow_Streaming_10_counters.txt rename to .github/workflows/load-tests-pipeline-options/java_ParDo_Dataflow_Streaming_10_counters.txt diff --git a/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_Streaming_10_times.txt b/.github/workflows/load-tests-pipeline-options/java_ParDo_Dataflow_Streaming_10_times.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_ParDo_Dataflow_Streaming_10_times.txt rename to .github/workflows/load-tests-pipeline-options/java_ParDo_Dataflow_Streaming_10_times.txt diff --git a/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_Streaming_200_times.txt b/.github/workflows/load-tests-pipeline-options/java_ParDo_Dataflow_Streaming_200_times.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_ParDo_Dataflow_Streaming_200_times.txt rename to .github/workflows/load-tests-pipeline-options/java_ParDo_Dataflow_Streaming_200_times.txt diff --git a/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_V2_Batch_JavaVersions_100_counters.txt b/.github/workflows/load-tests-pipeline-options/java_ParDo_Dataflow_V2_Batch_JavaVersions_100_counters.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_ParDo_Dataflow_V2_Batch_JavaVersions_100_counters.txt rename to .github/workflows/load-tests-pipeline-options/java_ParDo_Dataflow_V2_Batch_JavaVersions_100_counters.txt diff --git a/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_V2_Batch_JavaVersions_10_counters.txt b/.github/workflows/load-tests-pipeline-options/java_ParDo_Dataflow_V2_Batch_JavaVersions_10_counters.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_ParDo_Dataflow_V2_Batch_JavaVersions_10_counters.txt rename to .github/workflows/load-tests-pipeline-options/java_ParDo_Dataflow_V2_Batch_JavaVersions_10_counters.txt diff --git a/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_V2_Batch_JavaVersions_10_times.txt b/.github/workflows/load-tests-pipeline-options/java_ParDo_Dataflow_V2_Batch_JavaVersions_10_times.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_ParDo_Dataflow_V2_Batch_JavaVersions_10_times.txt rename to .github/workflows/load-tests-pipeline-options/java_ParDo_Dataflow_V2_Batch_JavaVersions_10_times.txt diff --git a/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_V2_Batch_JavaVersions_200_times.txt b/.github/workflows/load-tests-pipeline-options/java_ParDo_Dataflow_V2_Batch_JavaVersions_200_times.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_ParDo_Dataflow_V2_Batch_JavaVersions_200_times.txt rename to .github/workflows/load-tests-pipeline-options/java_ParDo_Dataflow_V2_Batch_JavaVersions_200_times.txt diff --git a/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_V2_Streaming_JavaVersions_100_counters.txt b/.github/workflows/load-tests-pipeline-options/java_ParDo_Dataflow_V2_Streaming_JavaVersions_100_counters.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_ParDo_Dataflow_V2_Streaming_JavaVersions_100_counters.txt rename to .github/workflows/load-tests-pipeline-options/java_ParDo_Dataflow_V2_Streaming_JavaVersions_100_counters.txt diff --git a/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_V2_Streaming_JavaVersions_10_counters.txt b/.github/workflows/load-tests-pipeline-options/java_ParDo_Dataflow_V2_Streaming_JavaVersions_10_counters.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_ParDo_Dataflow_V2_Streaming_JavaVersions_10_counters.txt rename to .github/workflows/load-tests-pipeline-options/java_ParDo_Dataflow_V2_Streaming_JavaVersions_10_counters.txt diff --git a/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_V2_Streaming_JavaVersions_10_times.txt b/.github/workflows/load-tests-pipeline-options/java_ParDo_Dataflow_V2_Streaming_JavaVersions_10_times.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_ParDo_Dataflow_V2_Streaming_JavaVersions_10_times.txt rename to .github/workflows/load-tests-pipeline-options/java_ParDo_Dataflow_V2_Streaming_JavaVersions_10_times.txt diff --git a/.github/workflows/load-tests-job-configs/java_ParDo_Dataflow_V2_Streaming_JavaVersions_200_times.txt b/.github/workflows/load-tests-pipeline-options/java_ParDo_Dataflow_V2_Streaming_JavaVersions_200_times.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_ParDo_Dataflow_V2_Streaming_JavaVersions_200_times.txt rename to .github/workflows/load-tests-pipeline-options/java_ParDo_Dataflow_V2_Streaming_JavaVersions_200_times.txt diff --git a/.github/workflows/load-tests-job-configs/java_ParDo_SparkStructuredStreaming_Batch_100_counters.txt b/.github/workflows/load-tests-pipeline-options/java_ParDo_SparkStructuredStreaming_Batch_100_counters.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_ParDo_SparkStructuredStreaming_Batch_100_counters.txt rename to .github/workflows/load-tests-pipeline-options/java_ParDo_SparkStructuredStreaming_Batch_100_counters.txt diff --git a/.github/workflows/load-tests-job-configs/java_ParDo_SparkStructuredStreaming_Batch_10_counters.txt b/.github/workflows/load-tests-pipeline-options/java_ParDo_SparkStructuredStreaming_Batch_10_counters.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_ParDo_SparkStructuredStreaming_Batch_10_counters.txt rename to .github/workflows/load-tests-pipeline-options/java_ParDo_SparkStructuredStreaming_Batch_10_counters.txt diff --git a/.github/workflows/load-tests-job-configs/java_ParDo_SparkStructuredStreaming_Batch_10_times.txt b/.github/workflows/load-tests-pipeline-options/java_ParDo_SparkStructuredStreaming_Batch_10_times.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_ParDo_SparkStructuredStreaming_Batch_10_times.txt rename to .github/workflows/load-tests-pipeline-options/java_ParDo_SparkStructuredStreaming_Batch_10_times.txt diff --git a/.github/workflows/load-tests-job-configs/java_ParDo_SparkStructuredStreaming_Batch_200_times.txt b/.github/workflows/load-tests-pipeline-options/java_ParDo_SparkStructuredStreaming_Batch_200_times.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_ParDo_SparkStructuredStreaming_Batch_200_times.txt rename to .github/workflows/load-tests-pipeline-options/java_ParDo_SparkStructuredStreaming_Batch_200_times.txt diff --git a/.github/workflows/load-tests-job-configs/java_Smoke_GroupByKey_Dataflow.txt b/.github/workflows/load-tests-pipeline-options/java_Smoke_GroupByKey_Dataflow.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_Smoke_GroupByKey_Dataflow.txt rename to .github/workflows/load-tests-pipeline-options/java_Smoke_GroupByKey_Dataflow.txt diff --git a/.github/workflows/load-tests-job-configs/java_Smoke_GroupByKey_Direct.txt b/.github/workflows/load-tests-pipeline-options/java_Smoke_GroupByKey_Direct.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_Smoke_GroupByKey_Direct.txt rename to .github/workflows/load-tests-pipeline-options/java_Smoke_GroupByKey_Direct.txt diff --git a/.github/workflows/load-tests-job-configs/java_Smoke_GroupByKey_Flink.txt b/.github/workflows/load-tests-pipeline-options/java_Smoke_GroupByKey_Flink.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_Smoke_GroupByKey_Flink.txt rename to .github/workflows/load-tests-pipeline-options/java_Smoke_GroupByKey_Flink.txt diff --git a/.github/workflows/load-tests-job-configs/java_Smoke_GroupByKey_Spark.txt b/.github/workflows/load-tests-pipeline-options/java_Smoke_GroupByKey_Spark.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/java_Smoke_GroupByKey_Spark.txt rename to .github/workflows/load-tests-pipeline-options/java_Smoke_GroupByKey_Spark.txt diff --git a/.github/workflows/load-tests-job-configs/python_CoGBK_Dataflow_Batch_100b_Multiple_Keys.txt b/.github/workflows/load-tests-pipeline-options/python_CoGBK_Dataflow_Batch_100b_Multiple_Keys.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_CoGBK_Dataflow_Batch_100b_Multiple_Keys.txt rename to .github/workflows/load-tests-pipeline-options/python_CoGBK_Dataflow_Batch_100b_Multiple_Keys.txt diff --git a/.github/workflows/load-tests-job-configs/python_CoGBK_Dataflow_Batch_100b_Single_Key.txt b/.github/workflows/load-tests-pipeline-options/python_CoGBK_Dataflow_Batch_100b_Single_Key.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_CoGBK_Dataflow_Batch_100b_Single_Key.txt rename to .github/workflows/load-tests-pipeline-options/python_CoGBK_Dataflow_Batch_100b_Single_Key.txt diff --git a/.github/workflows/load-tests-job-configs/python_CoGBK_Dataflow_Batch_10kB.txt b/.github/workflows/load-tests-pipeline-options/python_CoGBK_Dataflow_Batch_10kB.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_CoGBK_Dataflow_Batch_10kB.txt rename to .github/workflows/load-tests-pipeline-options/python_CoGBK_Dataflow_Batch_10kB.txt diff --git a/.github/workflows/load-tests-job-configs/python_CoGBK_Dataflow_Batch_2MB.txt b/.github/workflows/load-tests-pipeline-options/python_CoGBK_Dataflow_Batch_2MB.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_CoGBK_Dataflow_Batch_2MB.txt rename to .github/workflows/load-tests-pipeline-options/python_CoGBK_Dataflow_Batch_2MB.txt diff --git a/.github/workflows/load-tests-job-configs/python_CoGBK_Dataflow_Flink_Batch_100b_Multiple_Keys.txt b/.github/workflows/load-tests-pipeline-options/python_CoGBK_Dataflow_Flink_Batch_100b_Multiple_Keys.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_CoGBK_Dataflow_Flink_Batch_100b_Multiple_Keys.txt rename to .github/workflows/load-tests-pipeline-options/python_CoGBK_Dataflow_Flink_Batch_100b_Multiple_Keys.txt diff --git a/.github/workflows/load-tests-job-configs/python_CoGBK_Dataflow_Flink_Batch_100b_Single_Key.txt b/.github/workflows/load-tests-pipeline-options/python_CoGBK_Dataflow_Flink_Batch_100b_Single_Key.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_CoGBK_Dataflow_Flink_Batch_100b_Single_Key.txt rename to .github/workflows/load-tests-pipeline-options/python_CoGBK_Dataflow_Flink_Batch_100b_Single_Key.txt diff --git a/.github/workflows/load-tests-job-configs/python_CoGBK_Dataflow_Flink_Batch_10kB.txt b/.github/workflows/load-tests-pipeline-options/python_CoGBK_Dataflow_Flink_Batch_10kB.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_CoGBK_Dataflow_Flink_Batch_10kB.txt rename to .github/workflows/load-tests-pipeline-options/python_CoGBK_Dataflow_Flink_Batch_10kB.txt diff --git a/.github/workflows/load-tests-job-configs/python_CoGBK_Dataflow_Streaming_100b_Multiple_Keys.txt b/.github/workflows/load-tests-pipeline-options/python_CoGBK_Dataflow_Streaming_100b_Multiple_Keys.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_CoGBK_Dataflow_Streaming_100b_Multiple_Keys.txt rename to .github/workflows/load-tests-pipeline-options/python_CoGBK_Dataflow_Streaming_100b_Multiple_Keys.txt diff --git a/.github/workflows/load-tests-job-configs/python_CoGBK_Dataflow_Streaming_100b_Single_Key.txt b/.github/workflows/load-tests-pipeline-options/python_CoGBK_Dataflow_Streaming_100b_Single_Key.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_CoGBK_Dataflow_Streaming_100b_Single_Key.txt rename to .github/workflows/load-tests-pipeline-options/python_CoGBK_Dataflow_Streaming_100b_Single_Key.txt diff --git a/.github/workflows/load-tests-job-configs/python_CoGBK_Dataflow_Streaming_10kB.txt b/.github/workflows/load-tests-pipeline-options/python_CoGBK_Dataflow_Streaming_10kB.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_CoGBK_Dataflow_Streaming_10kB.txt rename to .github/workflows/load-tests-pipeline-options/python_CoGBK_Dataflow_Streaming_10kB.txt diff --git a/.github/workflows/load-tests-job-configs/python_CoGBK_Dataflow_Streaming_2MB.txt b/.github/workflows/load-tests-pipeline-options/python_CoGBK_Dataflow_Streaming_2MB.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_CoGBK_Dataflow_Streaming_2MB.txt rename to .github/workflows/load-tests-pipeline-options/python_CoGBK_Dataflow_Streaming_2MB.txt diff --git a/.github/workflows/load-tests-job-configs/config_Combine_Python_Batch_2GB_10b.txt b/.github/workflows/load-tests-pipeline-options/python_Combine_Dataflow_Batch_2GB_10b.txt similarity index 77% rename from .github/workflows/load-tests-job-configs/config_Combine_Python_Batch_2GB_10b.txt rename to .github/workflows/load-tests-pipeline-options/python_Combine_Dataflow_Batch_2GB_10b.txt index a6dabb5e5086..29b301551283 100644 --- a/.github/workflows/load-tests-job-configs/config_Combine_Python_Batch_2GB_10b.txt +++ b/.github/workflows/load-tests-pipeline-options/python_Combine_Dataflow_Batch_2GB_10b.txt @@ -1,4 +1,3 @@ -############################################################################### # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file # distributed with this work for additional information @@ -13,10 +12,8 @@ # distributed under the License is distributed on an "AS IS" BASIS, # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. # See the License for the specific language governing permissions and -# limitations under the License. -############################################################################### ---job_name=load-tests-python-dataflow-batch-combine-1- ---project=apache-beam-testing +# limitations under the License. + --region=us-central1 --temp_location=gs://temp-storage-for-perf-tests/smoketests --publish_to_big_query=true @@ -27,6 +24,4 @@ --num_workers=5 --autoscaling_algorithm=NONE --top_count=20 ---influxDatabase=beam_test_metrics ---influxHost=http://10.128.0.96:8086 --runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/config_Combine_Python_Batch_2GB_Fanout_4.txt b/.github/workflows/load-tests-pipeline-options/python_Combine_Dataflow_Batch_2GB_Fanout_4.txt similarity index 79% rename from .github/workflows/load-tests-job-configs/config_Combine_Python_Batch_2GB_Fanout_4.txt rename to .github/workflows/load-tests-pipeline-options/python_Combine_Dataflow_Batch_2GB_Fanout_4.txt index 7639456296b6..7405d6b75a82 100644 --- a/.github/workflows/load-tests-job-configs/config_Combine_Python_Batch_2GB_Fanout_4.txt +++ b/.github/workflows/load-tests-pipeline-options/python_Combine_Dataflow_Batch_2GB_Fanout_4.txt @@ -1,4 +1,3 @@ -############################################################################### # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file # distributed with this work for additional information @@ -14,9 +13,7 @@ # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. # See the License for the specific language governing permissions and # limitations under the License. -############################################################################### ---job_name=load-tests-python-dataflow-batch-combine-2- ---project=apache-beam-testing + --region=us-central1 --temp_location=gs://temp-storage-for-perf-tests/smoketests --publish_to_big_query=true @@ -28,6 +25,4 @@ --autoscaling_algorithm=NONE --fanout=4 --top_count=20 ---influxDatabase=beam_test_metrics ---influxHost=http://10.128.0.96:8086 --runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/config_Combine_Python_Batch_2GB_Fanout_8.txt b/.github/workflows/load-tests-pipeline-options/python_Combine_Dataflow_Batch_2GB_Fanout_8.txt similarity index 79% rename from .github/workflows/load-tests-job-configs/config_Combine_Python_Batch_2GB_Fanout_8.txt rename to .github/workflows/load-tests-pipeline-options/python_Combine_Dataflow_Batch_2GB_Fanout_8.txt index e5d46791a83c..b45c4eb1bd4a 100644 --- a/.github/workflows/load-tests-job-configs/config_Combine_Python_Batch_2GB_Fanout_8.txt +++ b/.github/workflows/load-tests-pipeline-options/python_Combine_Dataflow_Batch_2GB_Fanout_8.txt @@ -1,4 +1,3 @@ -############################################################################### # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file # distributed with this work for additional information @@ -14,9 +13,7 @@ # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. # See the License for the specific language governing permissions and # limitations under the License. -############################################################################### ---job_name=load-tests-python-dataflow-batch-combine-3- ---project=apache-beam-testing + --region=us-central1 --temp_location=gs://temp-storage-for-perf-tests/smoketests --publish_to_big_query=true @@ -28,6 +25,4 @@ --autoscaling_algorithm=NONE --fanout=8 --top_count=20 ---influxDatabase=beam_test_metrics ---influxHost=http://10.128.0.96:8086 --runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_Combine_Dataflow_Streaming_2GB_10_byte_records.txt b/.github/workflows/load-tests-pipeline-options/python_Combine_Dataflow_Streaming_2GB_10_byte_records.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_Combine_Dataflow_Streaming_2GB_10_byte_records.txt rename to .github/workflows/load-tests-pipeline-options/python_Combine_Dataflow_Streaming_2GB_10_byte_records.txt diff --git a/.github/workflows/load-tests-job-configs/python_Combine_Dataflow_Streaming_2GB_Fanout_4.txt b/.github/workflows/load-tests-pipeline-options/python_Combine_Dataflow_Streaming_2GB_Fanout_4.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_Combine_Dataflow_Streaming_2GB_Fanout_4.txt rename to .github/workflows/load-tests-pipeline-options/python_Combine_Dataflow_Streaming_2GB_Fanout_4.txt diff --git a/.github/workflows/load-tests-job-configs/python_Combine_Dataflow_Streaming_2GB_Fanout_8.txt b/.github/workflows/load-tests-pipeline-options/python_Combine_Dataflow_Streaming_2GB_Fanout_8.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_Combine_Dataflow_Streaming_2GB_Fanout_8.txt rename to .github/workflows/load-tests-pipeline-options/python_Combine_Dataflow_Streaming_2GB_Fanout_8.txt diff --git a/.github/workflows/load-tests-job-configs/python_Combine_Flink_Batch_2GB_10_byte_records.txt b/.github/workflows/load-tests-pipeline-options/python_Combine_Flink_Batch_2GB_10_byte_records.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_Combine_Flink_Batch_2GB_10_byte_records.txt rename to .github/workflows/load-tests-pipeline-options/python_Combine_Flink_Batch_2GB_10_byte_records.txt diff --git a/.github/workflows/load-tests-job-configs/python_Combine_Flink_Batch_2GB_Fanout_4.txt b/.github/workflows/load-tests-pipeline-options/python_Combine_Flink_Batch_2GB_Fanout_4.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_Combine_Flink_Batch_2GB_Fanout_4.txt rename to .github/workflows/load-tests-pipeline-options/python_Combine_Flink_Batch_2GB_Fanout_4.txt diff --git a/.github/workflows/load-tests-job-configs/python_Combine_Flink_Batch_2GB_Fanout_8.txt b/.github/workflows/load-tests-pipeline-options/python_Combine_Flink_Batch_2GB_Fanout_8.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_Combine_Flink_Batch_2GB_Fanout_8.txt rename to .github/workflows/load-tests-pipeline-options/python_Combine_Flink_Batch_2GB_Fanout_8.txt diff --git a/.github/workflows/load-tests-job-configs/python_Combine_Flink_Streaming_2GB_10_byte_records.txt b/.github/workflows/load-tests-pipeline-options/python_Combine_Flink_Streaming_2GB_10_byte_records.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_Combine_Flink_Streaming_2GB_10_byte_records.txt rename to .github/workflows/load-tests-pipeline-options/python_Combine_Flink_Streaming_2GB_10_byte_records.txt diff --git a/.github/workflows/load-tests-job-configs/python_Combine_Flink_Streaming_2GB_Fanout_4.txt b/.github/workflows/load-tests-pipeline-options/python_Combine_Flink_Streaming_2GB_Fanout_4.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_Combine_Flink_Streaming_2GB_Fanout_4.txt rename to .github/workflows/load-tests-pipeline-options/python_Combine_Flink_Streaming_2GB_Fanout_4.txt diff --git a/.github/workflows/load-tests-job-configs/python_Combine_Flink_Streaming_2GB_Fanout_8.txt b/.github/workflows/load-tests-pipeline-options/python_Combine_Flink_Streaming_2GB_Fanout_8.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_Combine_Flink_Streaming_2GB_Fanout_8.txt rename to .github/workflows/load-tests-pipeline-options/python_Combine_Flink_Streaming_2GB_Fanout_8.txt diff --git a/.github/workflows/load-tests-job-configs/config_FnApiRunner_Python_Microbenchmark.txt b/.github/workflows/load-tests-pipeline-options/python_FnApiRunner_Microbenchmark.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/config_FnApiRunner_Python_Microbenchmark.txt rename to .github/workflows/load-tests-pipeline-options/python_FnApiRunner_Microbenchmark.txt diff --git a/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_2GB_of_100B_records.txt b/.github/workflows/load-tests-pipeline-options/python_GBK_Dataflow_Batch_2GB_of_100B_records.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_2GB_of_100B_records.txt rename to .github/workflows/load-tests-pipeline-options/python_GBK_Dataflow_Batch_2GB_of_100B_records.txt diff --git a/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_2GB_of_100kB_records.txt b/.github/workflows/load-tests-pipeline-options/python_GBK_Dataflow_Batch_2GB_of_100kB_records.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_2GB_of_100kB_records.txt rename to .github/workflows/load-tests-pipeline-options/python_GBK_Dataflow_Batch_2GB_of_100kB_records.txt diff --git a/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_2GB_of_10B_records.txt b/.github/workflows/load-tests-pipeline-options/python_GBK_Dataflow_Batch_2GB_of_10B_records.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_2GB_of_10B_records.txt rename to .github/workflows/load-tests-pipeline-options/python_GBK_Dataflow_Batch_2GB_of_10B_records.txt diff --git a/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_fanout_4_times_with_2GB_10-byte_records_total.txt b/.github/workflows/load-tests-pipeline-options/python_GBK_Dataflow_Batch_fanout_4_times_with_2GB_10-byte_records_total.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_fanout_4_times_with_2GB_10-byte_records_total.txt rename to .github/workflows/load-tests-pipeline-options/python_GBK_Dataflow_Batch_fanout_4_times_with_2GB_10-byte_records_total.txt diff --git a/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_fanout_8_times_with_2GB_10-byte_records_total.txt b/.github/workflows/load-tests-pipeline-options/python_GBK_Dataflow_Batch_fanout_8_times_with_2GB_10-byte_records_total.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_fanout_8_times_with_2GB_10-byte_records_total.txt rename to .github/workflows/load-tests-pipeline-options/python_GBK_Dataflow_Batch_fanout_8_times_with_2GB_10-byte_records_total.txt diff --git a/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Streaming_2GB_of_100B_records.txt b/.github/workflows/load-tests-pipeline-options/python_GBK_Dataflow_Streaming_2GB_of_100B_records.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_GBK_Dataflow_Streaming_2GB_of_100B_records.txt rename to .github/workflows/load-tests-pipeline-options/python_GBK_Dataflow_Streaming_2GB_of_100B_records.txt diff --git a/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Streaming_2GB_of_100kB_records.txt b/.github/workflows/load-tests-pipeline-options/python_GBK_Dataflow_Streaming_2GB_of_100kB_records.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_GBK_Dataflow_Streaming_2GB_of_100kB_records.txt rename to .github/workflows/load-tests-pipeline-options/python_GBK_Dataflow_Streaming_2GB_of_100kB_records.txt diff --git a/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Streaming_2GB_of_10B_records.txt b/.github/workflows/load-tests-pipeline-options/python_GBK_Dataflow_Streaming_2GB_of_10B_records.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_GBK_Dataflow_Streaming_2GB_of_10B_records.txt rename to .github/workflows/load-tests-pipeline-options/python_GBK_Dataflow_Streaming_2GB_of_10B_records.txt diff --git a/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Streaming_fanout_4_times_with_2GB_10-byte_records_total.txt b/.github/workflows/load-tests-pipeline-options/python_GBK_Dataflow_Streaming_fanout_4_times_with_2GB_10-byte_records_total.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_GBK_Dataflow_Streaming_fanout_4_times_with_2GB_10-byte_records_total.txt rename to .github/workflows/load-tests-pipeline-options/python_GBK_Dataflow_Streaming_fanout_4_times_with_2GB_10-byte_records_total.txt diff --git a/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Streaming_fanout_8_times_with_2GB_10-byte_records_total.txt b/.github/workflows/load-tests-pipeline-options/python_GBK_Dataflow_Streaming_fanout_8_times_with_2GB_10-byte_records_total.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_GBK_Dataflow_Streaming_fanout_8_times_with_2GB_10-byte_records_total.txt rename to .github/workflows/load-tests-pipeline-options/python_GBK_Dataflow_Streaming_fanout_8_times_with_2GB_10-byte_records_total.txt diff --git a/.github/workflows/load-tests-job-configs/python_GBK_Flink_Batch_2GB_of_100B_records.txt b/.github/workflows/load-tests-pipeline-options/python_GBK_Flink_Batch_2GB_of_100B_records.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_GBK_Flink_Batch_2GB_of_100B_records.txt rename to .github/workflows/load-tests-pipeline-options/python_GBK_Flink_Batch_2GB_of_100B_records.txt diff --git a/.github/workflows/load-tests-job-configs/python_GBK_Flink_Batch_2GB_of_10B_records.txt b/.github/workflows/load-tests-pipeline-options/python_GBK_Flink_Batch_2GB_of_10B_records.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_GBK_Flink_Batch_2GB_of_10B_records.txt rename to .github/workflows/load-tests-pipeline-options/python_GBK_Flink_Batch_2GB_of_10B_records.txt diff --git a/.github/workflows/load-tests-job-configs/python_GBK_Flink_Batch_fanout_4_times_with_2GB_10-byte_records_total.txt b/.github/workflows/load-tests-pipeline-options/python_GBK_Flink_Batch_fanout_4_times_with_2GB_10-byte_records_total.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_GBK_Flink_Batch_fanout_4_times_with_2GB_10-byte_records_total.txt rename to .github/workflows/load-tests-pipeline-options/python_GBK_Flink_Batch_fanout_4_times_with_2GB_10-byte_records_total.txt diff --git a/.github/workflows/load-tests-job-configs/python_GBK_Flink_Batch_fanout_8_times_with_2GB_10-byte_records_total.txt b/.github/workflows/load-tests-pipeline-options/python_GBK_Flink_Batch_fanout_8_times_with_2GB_10-byte_records_total.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_GBK_Flink_Batch_fanout_8_times_with_2GB_10-byte_records_total.txt rename to .github/workflows/load-tests-pipeline-options/python_GBK_Flink_Batch_fanout_8_times_with_2GB_10-byte_records_total.txt diff --git a/.github/workflows/load-tests-job-configs/python_GBK_Flink_Batch_reiterate_4_times_10kB_values.txt b/.github/workflows/load-tests-pipeline-options/python_GBK_Flink_Batch_reiterate_4_times_10kB_values.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_GBK_Flink_Batch_reiterate_4_times_10kB_values.txt rename to .github/workflows/load-tests-pipeline-options/python_GBK_Flink_Batch_reiterate_4_times_10kB_values.txt diff --git a/.github/workflows/load-tests-job-configs/python_GBK_reiterate_Dataflow_Batch_reiterate_4_times_10kB_values.txt b/.github/workflows/load-tests-pipeline-options/python_GBK_reiterate_Dataflow_Batch_reiterate_4_times_10kB_values.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_GBK_reiterate_Dataflow_Batch_reiterate_4_times_10kB_values.txt rename to .github/workflows/load-tests-pipeline-options/python_GBK_reiterate_Dataflow_Batch_reiterate_4_times_10kB_values.txt diff --git a/.github/workflows/load-tests-job-configs/python_GBK_reiterate_Dataflow_Batch_reiterate_4_times_2MB_values.txt b/.github/workflows/load-tests-pipeline-options/python_GBK_reiterate_Dataflow_Batch_reiterate_4_times_2MB_values.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_GBK_reiterate_Dataflow_Batch_reiterate_4_times_2MB_values.txt rename to .github/workflows/load-tests-pipeline-options/python_GBK_reiterate_Dataflow_Batch_reiterate_4_times_2MB_values.txt diff --git a/.github/workflows/load-tests-job-configs/python_GBK_reiterate_Dataflow_Streaming_reiterate_4_times_10kB_values.txt b/.github/workflows/load-tests-pipeline-options/python_GBK_reiterate_Dataflow_Streaming_reiterate_4_times_10kB_values.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_GBK_reiterate_Dataflow_Streaming_reiterate_4_times_10kB_values.txt rename to .github/workflows/load-tests-pipeline-options/python_GBK_reiterate_Dataflow_Streaming_reiterate_4_times_10kB_values.txt diff --git a/.github/workflows/load-tests-job-configs/python_GBK_reiterate_Dataflow_Streaming_reiterate_4_times_2MB_values.txt b/.github/workflows/load-tests-pipeline-options/python_GBK_reiterate_Dataflow_Streaming_reiterate_4_times_2MB_values.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_GBK_reiterate_Dataflow_Streaming_reiterate_4_times_2MB_values.txt rename to .github/workflows/load-tests-pipeline-options/python_GBK_reiterate_Dataflow_Streaming_reiterate_4_times_2MB_values.txt diff --git a/.github/workflows/load-tests-job-configs/python_ParDo_Dataflow_Batch_100_Counters.txt b/.github/workflows/load-tests-pipeline-options/python_ParDo_Dataflow_Batch_100_Counters.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_ParDo_Dataflow_Batch_100_Counters.txt rename to .github/workflows/load-tests-pipeline-options/python_ParDo_Dataflow_Batch_100_Counters.txt diff --git a/.github/workflows/load-tests-job-configs/python_ParDo_Dataflow_Batch_10_Counters.txt b/.github/workflows/load-tests-pipeline-options/python_ParDo_Dataflow_Batch_10_Counters.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_ParDo_Dataflow_Batch_10_Counters.txt rename to .github/workflows/load-tests-pipeline-options/python_ParDo_Dataflow_Batch_10_Counters.txt diff --git a/.github/workflows/load-tests-job-configs/python_ParDo_Dataflow_Batch_10_Iterations.txt b/.github/workflows/load-tests-pipeline-options/python_ParDo_Dataflow_Batch_10_Iterations.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_ParDo_Dataflow_Batch_10_Iterations.txt rename to .github/workflows/load-tests-pipeline-options/python_ParDo_Dataflow_Batch_10_Iterations.txt diff --git a/.github/workflows/load-tests-job-configs/python_ParDo_Dataflow_Batch_200_Iterations.txt b/.github/workflows/load-tests-pipeline-options/python_ParDo_Dataflow_Batch_200_Iterations.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_ParDo_Dataflow_Batch_200_Iterations.txt rename to .github/workflows/load-tests-pipeline-options/python_ParDo_Dataflow_Batch_200_Iterations.txt diff --git a/.github/workflows/load-tests-job-configs/python_ParDo_Dataflow_Streaming_100_Counters.txt b/.github/workflows/load-tests-pipeline-options/python_ParDo_Dataflow_Streaming_100_Counters.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_ParDo_Dataflow_Streaming_100_Counters.txt rename to .github/workflows/load-tests-pipeline-options/python_ParDo_Dataflow_Streaming_100_Counters.txt diff --git a/.github/workflows/load-tests-job-configs/python_ParDo_Dataflow_Streaming_10_Counters.txt b/.github/workflows/load-tests-pipeline-options/python_ParDo_Dataflow_Streaming_10_Counters.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_ParDo_Dataflow_Streaming_10_Counters.txt rename to .github/workflows/load-tests-pipeline-options/python_ParDo_Dataflow_Streaming_10_Counters.txt diff --git a/.github/workflows/load-tests-job-configs/python_ParDo_Dataflow_Streaming_10_Iterations.txt b/.github/workflows/load-tests-pipeline-options/python_ParDo_Dataflow_Streaming_10_Iterations.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_ParDo_Dataflow_Streaming_10_Iterations.txt rename to .github/workflows/load-tests-pipeline-options/python_ParDo_Dataflow_Streaming_10_Iterations.txt diff --git a/.github/workflows/load-tests-job-configs/python_ParDo_Dataflow_Streaming_200_Iterations.txt b/.github/workflows/load-tests-pipeline-options/python_ParDo_Dataflow_Streaming_200_Iterations.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_ParDo_Dataflow_Streaming_200_Iterations.txt rename to .github/workflows/load-tests-pipeline-options/python_ParDo_Dataflow_Streaming_200_Iterations.txt diff --git a/.github/workflows/load-tests-job-configs/python_ParDo_Flink_Batch_10_Counters.txt b/.github/workflows/load-tests-pipeline-options/python_ParDo_Flink_Batch_10_Counters.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_ParDo_Flink_Batch_10_Counters.txt rename to .github/workflows/load-tests-pipeline-options/python_ParDo_Flink_Batch_10_Counters.txt diff --git a/.github/workflows/load-tests-job-configs/python_ParDo_Flink_Batch_10_Iterations.txt b/.github/workflows/load-tests-pipeline-options/python_ParDo_Flink_Batch_10_Iterations.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_ParDo_Flink_Batch_10_Iterations.txt rename to .github/workflows/load-tests-pipeline-options/python_ParDo_Flink_Batch_10_Iterations.txt diff --git a/.github/workflows/load-tests-job-configs/python_ParDo_Flink_Batch_200_Iterations.txt b/.github/workflows/load-tests-pipeline-options/python_ParDo_Flink_Batch_200_Iterations.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_ParDo_Flink_Batch_200_Iterations.txt rename to .github/workflows/load-tests-pipeline-options/python_ParDo_Flink_Batch_200_Iterations.txt diff --git a/.github/workflows/load-tests-job-configs/python_ParDo_Flink_Streaming_100_Counters.txt b/.github/workflows/load-tests-pipeline-options/python_ParDo_Flink_Streaming_100_Counters.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_ParDo_Flink_Streaming_100_Counters.txt rename to .github/workflows/load-tests-pipeline-options/python_ParDo_Flink_Streaming_100_Counters.txt diff --git a/.github/workflows/load-tests-job-configs/python_ParDo_Flink_Streaming_10_Counters.txt b/.github/workflows/load-tests-pipeline-options/python_ParDo_Flink_Streaming_10_Counters.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_ParDo_Flink_Streaming_10_Counters.txt rename to .github/workflows/load-tests-pipeline-options/python_ParDo_Flink_Streaming_10_Counters.txt diff --git a/.github/workflows/load-tests-job-configs/python_ParDo_Flink_Streaming_10_Iterations.txt b/.github/workflows/load-tests-pipeline-options/python_ParDo_Flink_Streaming_10_Iterations.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_ParDo_Flink_Streaming_10_Iterations.txt rename to .github/workflows/load-tests-pipeline-options/python_ParDo_Flink_Streaming_10_Iterations.txt diff --git a/.github/workflows/load-tests-job-configs/python_ParDo_Flink_Streaming_200_Iterations.txt b/.github/workflows/load-tests-pipeline-options/python_ParDo_Flink_Streaming_200_Iterations.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_ParDo_Flink_Streaming_200_Iterations.txt rename to .github/workflows/load-tests-pipeline-options/python_ParDo_Flink_Streaming_200_Iterations.txt diff --git a/.github/workflows/load-tests-job-configs/python_ParDo_Flink_Streaming_5_Iterations.txt b/.github/workflows/load-tests-pipeline-options/python_ParDo_Flink_Streaming_5_Iterations.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_ParDo_Flink_Streaming_5_Iterations.txt rename to .github/workflows/load-tests-pipeline-options/python_ParDo_Flink_Streaming_5_Iterations.txt diff --git a/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_10gb_1000window_first_iterable.txt b/.github/workflows/load-tests-pipeline-options/python_SideInput_Dataflow_Batch_10gb_1000window_first_iterable.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_10gb_1000window_first_iterable.txt rename to .github/workflows/load-tests-pipeline-options/python_SideInput_Dataflow_Batch_10gb_1000window_first_iterable.txt diff --git a/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_10gb_1000window_iterable.txt b/.github/workflows/load-tests-pipeline-options/python_SideInput_Dataflow_Batch_10gb_1000window_iterable.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_10gb_1000window_iterable.txt rename to .github/workflows/load-tests-pipeline-options/python_SideInput_Dataflow_Batch_10gb_1000window_iterable.txt diff --git a/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_10gb_1window_first_iterable.txt b/.github/workflows/load-tests-pipeline-options/python_SideInput_Dataflow_Batch_10gb_1window_first_iterable.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_10gb_1window_first_iterable.txt rename to .github/workflows/load-tests-pipeline-options/python_SideInput_Dataflow_Batch_10gb_1window_first_iterable.txt diff --git a/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_10gb_1window_iterable.txt b/.github/workflows/load-tests-pipeline-options/python_SideInput_Dataflow_Batch_10gb_1window_iterable.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_10gb_1window_iterable.txt rename to .github/workflows/load-tests-pipeline-options/python_SideInput_Dataflow_Batch_10gb_1window_iterable.txt diff --git a/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1000window_1key_percent_dict.txt b/.github/workflows/load-tests-pipeline-options/python_SideInput_Dataflow_Batch_1gb_1000window_1key_percent_dict.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1000window_1key_percent_dict.txt rename to .github/workflows/load-tests-pipeline-options/python_SideInput_Dataflow_Batch_1gb_1000window_1key_percent_dict.txt diff --git a/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1000window_99key_percent_dict.txt b/.github/workflows/load-tests-pipeline-options/python_SideInput_Dataflow_Batch_1gb_1000window_99key_percent_dict.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1000window_99key_percent_dict.txt rename to .github/workflows/load-tests-pipeline-options/python_SideInput_Dataflow_Batch_1gb_1000window_99key_percent_dict.txt diff --git a/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1window_1key_percent_dict.txt b/.github/workflows/load-tests-pipeline-options/python_SideInput_Dataflow_Batch_1gb_1window_1key_percent_dict.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1window_1key_percent_dict.txt rename to .github/workflows/load-tests-pipeline-options/python_SideInput_Dataflow_Batch_1gb_1window_1key_percent_dict.txt diff --git a/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1window_99key_percent_dict.txt b/.github/workflows/load-tests-pipeline-options/python_SideInput_Dataflow_Batch_1gb_1window_99key_percent_dict.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1window_99key_percent_dict.txt rename to .github/workflows/load-tests-pipeline-options/python_SideInput_Dataflow_Batch_1gb_1window_99key_percent_dict.txt diff --git a/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1window_first_list.txt b/.github/workflows/load-tests-pipeline-options/python_SideInput_Dataflow_Batch_1gb_1window_first_list.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1window_first_list.txt rename to .github/workflows/load-tests-pipeline-options/python_SideInput_Dataflow_Batch_1gb_1window_first_list.txt diff --git a/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1window_list.txt b/.github/workflows/load-tests-pipeline-options/python_SideInput_Dataflow_Batch_1gb_1window_list.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1window_list.txt rename to .github/workflows/load-tests-pipeline-options/python_SideInput_Dataflow_Batch_1gb_1window_list.txt diff --git a/.github/workflows/load-tests-job-configs/python_Smoke_GroupByKey_Dataflow.txt b/.github/workflows/load-tests-pipeline-options/python_Smoke_GroupByKey_Dataflow.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_Smoke_GroupByKey_Dataflow.txt rename to .github/workflows/load-tests-pipeline-options/python_Smoke_GroupByKey_Dataflow.txt diff --git a/.github/workflows/load-tests-job-configs/python_Smoke_GroupByKey_Direct.txt b/.github/workflows/load-tests-pipeline-options/python_Smoke_GroupByKey_Direct.txt similarity index 100% rename from .github/workflows/load-tests-job-configs/python_Smoke_GroupByKey_Direct.txt rename to .github/workflows/load-tests-pipeline-options/python_Smoke_GroupByKey_Direct.txt diff --git a/.github/workflows/performance-tests-job-configs/config_BigQueryIO_Batch_Java_Avro.txt b/.github/workflows/performance-tests-job-configs/config_BigQueryIO_Batch_Java_Avro.txt deleted file mode 100644 index 5e7e53821231..000000000000 --- a/.github/workflows/performance-tests-job-configs/config_BigQueryIO_Batch_Java_Avro.txt +++ /dev/null @@ -1,38 +0,0 @@ -############################################################################### -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -############################################################################### -'["--tempLocation=gs://temp-storage-for-perf-tests/loadtests", -"--project=apache-beam-testing", -"--tempRoot=gs://temp-storage-for-perf-tests/loadtests", -"--writeMethod=FILE_LOADS", -"--writeFormat=AVRO", -"--testBigQueryDataset=beam_performance", -"--testBigQueryTable=bqio_write_10GB_java_avro_", -"--metricsBigQueryDataset=beam_performance", -"--metricsBigQueryTable=bqio_10GB_results_java_batch_avro", -"--influxMeasurement=bqio_10GB_results_java_batch_avro", -"--sourceOptions={ -\"numRecords\":\"10485760\", -\"keySizeBytes\":\"1\", -\"valueSizeBytes\":\"1024\" -}", -"--runner=DataflowRunner", -"--maxNumWorkers=5", -"--numWorkers=5", -"--autoscalingAlgorithm=NONE", -"--influxDatabase=beam_test_metrics", -"--influxHost=http://10.128.0.96:8086"]' \ No newline at end of file diff --git a/.github/workflows/performance-tests-job-configs/config_BigQueryIO_Batch_Java_Json.txt b/.github/workflows/performance-tests-job-configs/config_BigQueryIO_Batch_Java_Json.txt deleted file mode 100644 index 7bd9c30ae738..000000000000 --- a/.github/workflows/performance-tests-job-configs/config_BigQueryIO_Batch_Java_Json.txt +++ /dev/null @@ -1,38 +0,0 @@ -############################################################################### -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -############################################################################### -'["--tempLocation=gs://temp-storage-for-perf-tests/loadtests", -"--project=apache-beam-testing", -"--tempRoot=gs://temp-storage-for-perf-tests/loadtests", -"--writeMethod=FILE_LOADS", -"--writeFormat=JSON", -"--testBigQueryDataset=beam_performance", -"--testBigQueryTable=bqio_write_10GB_java_json_", -"--metricsBigQueryDataset=beam_performance", -"--metricsBigQueryTable=bqio_10GB_results_java_batch_json", -"--influxMeasurement=bqio_10GB_results_java_batch_json", -"--sourceOptions={ -\"numRecords\":\"10485760\", -\"keySizeBytes\":\"1\", -\"valueSizeBytes\":\"1024\" -}", -"--runner=DataflowRunner", -"--maxNumWorkers=5", -"--numWorkers=5", -"--autoscalingAlgorithm=NONE", -"--influxDatabase=beam_test_metrics", -"--influxHost=http://10.128.0.96:8086"]' \ No newline at end of file diff --git a/.github/workflows/performance-tests-job-configs/config_BigQueryIO_Streaming_Java.txt b/.github/workflows/performance-tests-job-configs/config_BigQueryIO_Streaming_Java.txt deleted file mode 100644 index 8bddea5fcb8b..000000000000 --- a/.github/workflows/performance-tests-job-configs/config_BigQueryIO_Streaming_Java.txt +++ /dev/null @@ -1,39 +0,0 @@ -############################################################################### -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -############################################################################### -'["--tempLocation=gs://temp-storage-for-perf-tests/loadtests", -"--project=apache-beam-testing", -"--tempRoot=gs://temp-storage-for-perf-tests/loadtests", -"--writeMethod=STREAMING_INSERTS", -"--writeFormat=JSON", -"--pipelineTimeout=1200", -"--testBigQueryDataset=beam_performance", -"--testBigQueryTable=bqio_write_10GB_java_stream_", -"--metricsBigQueryDataset=beam_performance", -"--metricsBigQueryTable=bqio_10GB_results_java_stream", -"--influxMeasurement=bqio_10GB_results_java_stream", -"--sourceOptions={ -\"numRecords\":\"10485760\", -\"keySizeBytes\":\"1\", -\"valueSizeBytes\":\"1024\" -}", -"--runner=DataflowRunner", -"--maxNumWorkers=5", -"--numWorkers=5", -"--autoscalingAlgorithm=NONE", -"--influxDatabase=beam_test_metrics", -"--influxHost=http://10.128.0.96:8086"]' \ No newline at end of file diff --git a/.github/workflows/performance-tests-job-configs/JDBC.txt b/.github/workflows/performance-tests-pipeline-options/JDBC.txt similarity index 100% rename from .github/workflows/performance-tests-job-configs/JDBC.txt rename to .github/workflows/performance-tests-pipeline-options/JDBC.txt diff --git a/.github/workflows/performance-tests-job-configs/SQLBigQueryIO_Batch_Java.txt b/.github/workflows/performance-tests-pipeline-options/SQLBigQueryIO_Batch_Java.txt similarity index 100% rename from .github/workflows/performance-tests-job-configs/SQLBigQueryIO_Batch_Java.txt rename to .github/workflows/performance-tests-pipeline-options/SQLBigQueryIO_Batch_Java.txt diff --git a/.github/workflows/performance-tests-job-configs/config_TFRecordIOIT.txt b/.github/workflows/performance-tests-pipeline-options/TFRecordIOIT.txt similarity index 100% rename from .github/workflows/performance-tests-job-configs/config_TFRecordIOIT.txt rename to .github/workflows/performance-tests-pipeline-options/TFRecordIOIT.txt diff --git a/.github/workflows/performance-tests-job-configs/TFRecordIOIT_HDFS.txt b/.github/workflows/performance-tests-pipeline-options/TFRecordIOIT_HDFS.txt similarity index 100% rename from .github/workflows/performance-tests-job-configs/TFRecordIOIT_HDFS.txt rename to .github/workflows/performance-tests-pipeline-options/TFRecordIOIT_HDFS.txt diff --git a/.github/workflows/performance-tests-job-configs/config_AvroIOIT.txt b/.github/workflows/performance-tests-pipeline-options/avroIOIT.txt similarity index 100% rename from .github/workflows/performance-tests-job-configs/config_AvroIOIT.txt rename to .github/workflows/performance-tests-pipeline-options/avroIOIT.txt diff --git a/.github/workflows/performance-tests-job-configs/config_AvroIOIT_HDFS.txt b/.github/workflows/performance-tests-pipeline-options/avroIOIT_HDFS.txt similarity index 100% rename from .github/workflows/performance-tests-job-configs/config_AvroIOIT_HDFS.txt rename to .github/workflows/performance-tests-pipeline-options/avroIOIT_HDFS.txt diff --git a/.github/workflows/performance-tests-pipeline-options/bigQueryIO_Batch_Java_Avro.txt b/.github/workflows/performance-tests-pipeline-options/bigQueryIO_Batch_Java_Avro.txt new file mode 100644 index 000000000000..922114e63d0e --- /dev/null +++ b/.github/workflows/performance-tests-pipeline-options/bigQueryIO_Batch_Java_Avro.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--tempRoot=gs://temp-storage-for-perf-tests/loadtests +--writeMethod=FILE_LOADS +--writeFormat=AVRO +--testBigQueryDataset=beam_performance +--metricsBigQueryDataset=beam_performance +--metricsBigQueryTable=bqio_10GB_results_java_batch_avro +--influxMeasurement=bqio_10GB_results_java_batch_avro +--sourceOptions={\"numRecords\":\"10485760\",\"keySizeBytes\":\"1\",\"valueSizeBytes\":\"1024\"} +--runner=DataflowRunner +--maxNumWorkers=5 +--numWorkers=5 +--autoscalingAlgorithm=NONE \ No newline at end of file diff --git a/.github/workflows/performance-tests-pipeline-options/bigQueryIO_Batch_Java_Json.txt b/.github/workflows/performance-tests-pipeline-options/bigQueryIO_Batch_Java_Json.txt new file mode 100644 index 000000000000..627368ec0e24 --- /dev/null +++ b/.github/workflows/performance-tests-pipeline-options/bigQueryIO_Batch_Java_Json.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--tempRoot=gs://temp-storage-for-perf-tests/loadtests +--writeMethod=FILE_LOADS +--writeFormat=JSON +--testBigQueryDataset=beam_performance +--metricsBigQueryDataset=beam_performance +--metricsBigQueryTable=bqio_10GB_results_java_batch_json +--influxMeasurement=bqio_10GB_results_java_batch_json +--sourceOptions={\"numRecords\":\"10485760\",\"keySizeBytes\":\"1\",\"valueSizeBytes\":\"1024\"} +--runner=DataflowRunner +--maxNumWorkers=5 +--numWorkers=5 +--autoscalingAlgorithm=NONE \ No newline at end of file diff --git a/.github/workflows/performance-tests-pipeline-options/bigQueryIO_Streaming_Java.txt b/.github/workflows/performance-tests-pipeline-options/bigQueryIO_Streaming_Java.txt new file mode 100644 index 000000000000..99bd30e25994 --- /dev/null +++ b/.github/workflows/performance-tests-pipeline-options/bigQueryIO_Streaming_Java.txt @@ -0,0 +1,30 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--tempRoot=gs://temp-storage-for-perf-tests/loadtests +--writeMethod=STREAMING_INSERTS +--writeFormat=JSON +--pipelineTimeout=1200 +--testBigQueryDataset=beam_performance +--metricsBigQueryDataset=beam_performance +--metricsBigQueryTable=bqio_10GB_results_java_stream +--influxMeasurement=bqio_10GB_results_java_stream +--sourceOptions={\"numRecords\":\"10485760\",\"keySizeBytes\":\"1\",\"valueSizeBytes\":\"1024\"} +--runner=DataflowRunner +--maxNumWorkers=5 +--numWorkers=5 +--autoscalingAlgorithm=NONE \ No newline at end of file diff --git a/.github/workflows/performance-tests-job-configs/biqQueryIO_Read_Python.txt b/.github/workflows/performance-tests-pipeline-options/biqQueryIO_Read_Python.txt similarity index 100% rename from .github/workflows/performance-tests-job-configs/biqQueryIO_Read_Python.txt rename to .github/workflows/performance-tests-pipeline-options/biqQueryIO_Read_Python.txt diff --git a/.github/workflows/performance-tests-job-configs/biqQueryIO_Write_Python_Batch.txt b/.github/workflows/performance-tests-pipeline-options/biqQueryIO_Write_Python_Batch.txt similarity index 100% rename from .github/workflows/performance-tests-job-configs/biqQueryIO_Write_Python_Batch.txt rename to .github/workflows/performance-tests-pipeline-options/biqQueryIO_Write_Python_Batch.txt diff --git a/.github/workflows/performance-tests-job-configs/cdap.txt b/.github/workflows/performance-tests-pipeline-options/cdap.txt similarity index 100% rename from .github/workflows/performance-tests-job-configs/cdap.txt rename to .github/workflows/performance-tests-pipeline-options/cdap.txt diff --git a/.github/workflows/performance-tests-job-configs/config_Compressed_TextIOIT.txt b/.github/workflows/performance-tests-pipeline-options/compressed_TextIOIT.txt similarity index 100% rename from .github/workflows/performance-tests-job-configs/config_Compressed_TextIOIT.txt rename to .github/workflows/performance-tests-pipeline-options/compressed_TextIOIT.txt diff --git a/.github/workflows/performance-tests-job-configs/config_Compressed_TextIOIT_HDFS.txt b/.github/workflows/performance-tests-pipeline-options/compressed_TextIOIT_HDFS.txt similarity index 100% rename from .github/workflows/performance-tests-job-configs/config_Compressed_TextIOIT_HDFS.txt rename to .github/workflows/performance-tests-pipeline-options/compressed_TextIOIT_HDFS.txt diff --git a/.github/workflows/performance-tests-job-configs/hadoopFormat.txt b/.github/workflows/performance-tests-pipeline-options/hadoopFormat.txt similarity index 100% rename from .github/workflows/performance-tests-job-configs/hadoopFormat.txt rename to .github/workflows/performance-tests-pipeline-options/hadoopFormat.txt diff --git a/.github/workflows/performance-tests-job-configs/kafka_IO_Batch.txt b/.github/workflows/performance-tests-pipeline-options/kafka_IO_Batch.txt similarity index 100% rename from .github/workflows/performance-tests-job-configs/kafka_IO_Batch.txt rename to .github/workflows/performance-tests-pipeline-options/kafka_IO_Batch.txt diff --git a/.github/workflows/performance-tests-job-configs/kafka_IO_Streaming.txt b/.github/workflows/performance-tests-pipeline-options/kafka_IO_Streaming.txt similarity index 100% rename from .github/workflows/performance-tests-job-configs/kafka_IO_Streaming.txt rename to .github/workflows/performance-tests-pipeline-options/kafka_IO_Streaming.txt diff --git a/.github/workflows/performance-tests-job-configs/config_ManyFiles_TextIOIT.txt b/.github/workflows/performance-tests-pipeline-options/manyFiles_TextIOIT.txt similarity index 100% rename from .github/workflows/performance-tests-job-configs/config_ManyFiles_TextIOIT.txt rename to .github/workflows/performance-tests-pipeline-options/manyFiles_TextIOIT.txt diff --git a/.github/workflows/performance-tests-job-configs/config_ManyFiles_TextIOIT_HDFS.txt b/.github/workflows/performance-tests-pipeline-options/manyFiles_TextIOIT_HDFS.txt similarity index 100% rename from .github/workflows/performance-tests-job-configs/config_ManyFiles_TextIOIT_HDFS.txt rename to .github/workflows/performance-tests-pipeline-options/manyFiles_TextIOIT_HDFS.txt diff --git a/.github/workflows/performance-tests-job-configs/mongoDBIO_IT.txt b/.github/workflows/performance-tests-pipeline-options/mongoDBIO_IT.txt similarity index 100% rename from .github/workflows/performance-tests-job-configs/mongoDBIO_IT.txt rename to .github/workflows/performance-tests-pipeline-options/mongoDBIO_IT.txt diff --git a/.github/workflows/performance-tests-job-configs/config_ParquetIOIT.txt b/.github/workflows/performance-tests-pipeline-options/parquetIOIT.txt similarity index 100% rename from .github/workflows/performance-tests-job-configs/config_ParquetIOIT.txt rename to .github/workflows/performance-tests-pipeline-options/parquetIOIT.txt diff --git a/.github/workflows/performance-tests-job-configs/config_ParquetIOIT_HDFS.txt b/.github/workflows/performance-tests-pipeline-options/parquetIOIT_HDFS.txt similarity index 100% rename from .github/workflows/performance-tests-job-configs/config_ParquetIOIT_HDFS.txt rename to .github/workflows/performance-tests-pipeline-options/parquetIOIT_HDFS.txt diff --git a/.github/workflows/performance-tests-job-configs/pubsubIOIT_Python_Streaming.txt b/.github/workflows/performance-tests-pipeline-options/pubsubIOIT_Python_Streaming.txt similarity index 100% rename from .github/workflows/performance-tests-job-configs/pubsubIOIT_Python_Streaming.txt rename to .github/workflows/performance-tests-pipeline-options/pubsubIOIT_Python_Streaming.txt diff --git a/.github/workflows/performance-tests-job-configs/spannerIO_Read_2GB_Python.txt b/.github/workflows/performance-tests-pipeline-options/spannerIO_Read_2GB_Python.txt similarity index 100% rename from .github/workflows/performance-tests-job-configs/spannerIO_Read_2GB_Python.txt rename to .github/workflows/performance-tests-pipeline-options/spannerIO_Read_2GB_Python.txt diff --git a/.github/workflows/performance-tests-job-configs/spannerIO_Write_2GB_Python.txt b/.github/workflows/performance-tests-pipeline-options/spannerIO_Write_2GB_Python.txt similarity index 100% rename from .github/workflows/performance-tests-job-configs/spannerIO_Write_2GB_Python.txt rename to .github/workflows/performance-tests-pipeline-options/spannerIO_Write_2GB_Python.txt diff --git a/.github/workflows/performance-tests-job-configs/sparkReceiver_IO.txt b/.github/workflows/performance-tests-pipeline-options/sparkReceiver_IO.txt similarity index 100% rename from .github/workflows/performance-tests-job-configs/sparkReceiver_IO.txt rename to .github/workflows/performance-tests-pipeline-options/sparkReceiver_IO.txt diff --git a/.github/workflows/performance-tests-job-configs/textIOIT.txt b/.github/workflows/performance-tests-pipeline-options/textIOIT.txt similarity index 100% rename from .github/workflows/performance-tests-job-configs/textIOIT.txt rename to .github/workflows/performance-tests-pipeline-options/textIOIT.txt diff --git a/.github/workflows/performance-tests-job-configs/textIOIT_HDFS.txt b/.github/workflows/performance-tests-pipeline-options/textIOIT_HDFS.txt similarity index 100% rename from .github/workflows/performance-tests-job-configs/textIOIT_HDFS.txt rename to .github/workflows/performance-tests-pipeline-options/textIOIT_HDFS.txt diff --git a/.github/workflows/performance-tests-job-configs/textIOIT_Python.txt b/.github/workflows/performance-tests-pipeline-options/textIOIT_Python.txt similarity index 100% rename from .github/workflows/performance-tests-job-configs/textIOIT_Python.txt rename to .github/workflows/performance-tests-pipeline-options/textIOIT_Python.txt diff --git a/.github/workflows/performance-tests-job-configs/wordCountIT_Python.txt b/.github/workflows/performance-tests-pipeline-options/wordCountIT_Python.txt similarity index 100% rename from .github/workflows/performance-tests-job-configs/wordCountIT_Python.txt rename to .github/workflows/performance-tests-pipeline-options/wordCountIT_Python.txt diff --git a/.github/workflows/performance-tests-job-configs/xlang_KafkaIO_Python.txt b/.github/workflows/performance-tests-pipeline-options/xlang_KafkaIO_Python.txt similarity index 100% rename from .github/workflows/performance-tests-job-configs/xlang_KafkaIO_Python.txt rename to .github/workflows/performance-tests-pipeline-options/xlang_KafkaIO_Python.txt diff --git a/.github/workflows/performance-tests-job-configs/config_XmlIOIT.txt b/.github/workflows/performance-tests-pipeline-options/xmlIOIT.txt similarity index 100% rename from .github/workflows/performance-tests-job-configs/config_XmlIOIT.txt rename to .github/workflows/performance-tests-pipeline-options/xmlIOIT.txt diff --git a/.github/workflows/performance-tests-job-configs/config_XmlIOIT_HDFS.txt b/.github/workflows/performance-tests-pipeline-options/xmlIOIT_HDFS.txt similarity index 100% rename from .github/workflows/performance-tests-job-configs/config_XmlIOIT_HDFS.txt rename to .github/workflows/performance-tests-pipeline-options/xmlIOIT_HDFS.txt From 96acc2cdd5d56d2d74b3147af09a57dfe5d51c58 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw <robertwb@gmail.com> Date: Fri, 20 Oct 2023 15:29:20 -0700 Subject: [PATCH 216/435] [YAML] Improved pipeline schema definition. (#29083) Slightly stricter definitions for catching more errors, as well as avoding the use of anyOf which often makes it difficult to deduce what the true error is. This does mean a pipeline must have a transform (or source/sink) block rather than simply be itself a list of transforms. --- sdks/python/apache_beam/yaml/README.md | 177 +++++++++--------- .../apache_beam/yaml/pipeline.schema.yaml | 48 ++++- 2 files changed, 128 insertions(+), 97 deletions(-) diff --git a/sdks/python/apache_beam/yaml/README.md b/sdks/python/apache_beam/yaml/README.md index 3ba78784c997..62c0d0eea162 100644 --- a/sdks/python/apache_beam/yaml/README.md +++ b/sdks/python/apache_beam/yaml/README.md @@ -166,41 +166,42 @@ Here we read two sources, join them, and write two outputs. ``` pipeline: - - type: ReadFromCsv - name: ReadLeft - config: - path: /path/to/left*.csv + transforms: + - type: ReadFromCsv + name: ReadLeft + config: + path: /path/to/left*.csv - - type: ReadFromCsv - name: ReadRight - config: - path: /path/to/right*.csv + - type: ReadFromCsv + name: ReadRight + config: + path: /path/to/right*.csv - - type: Sql - config: - query: select left.col1, right.col2 from left join right using (col3) - input: - left: ReadLeft - right: ReadRight - - - type: WriteToJson - name: WriteAll - input: Sql - config: - path: /path/to/all.json + - type: Sql + config: + query: select left.col1, right.col2 from left join right using (col3) + input: + left: ReadLeft + right: ReadRight - - type: Filter - name: FilterToBig - input: Sql - config: - language: python - keep: "col2 > 100" + - type: WriteToJson + name: WriteAll + input: Sql + config: + path: /path/to/all.json - - type: WriteToCsv - name: WriteBig - input: FilterToBig - config: - path: /path/to/big.csv + - type: Filter + name: FilterToBig + input: Sql + config: + language: python + keep: "col2 > 100" + + - type: WriteToCsv + name: WriteBig + input: FilterToBig + config: + path: /path/to/big.csv ``` One can, however, nest `chains` within a non-linear pipeline. @@ -209,49 +210,50 @@ that has a single input and contains its own sink. ``` pipeline: - - type: ReadFromCsv - name: ReadLeft - config: - path: /path/to/left*.csv + transforms: + - type: ReadFromCsv + name: ReadLeft + config: + path: /path/to/left*.csv - - type: ReadFromCsv - name: ReadRight - config: - path: /path/to/right*.csv + - type: ReadFromCsv + name: ReadRight + config: + path: /path/to/right*.csv - - type: Sql - config: - query: select left.col1, right.col2 from left join right using (col3) - input: - left: ReadLeft - right: ReadRight - - - type: WriteToJson - name: WriteAll - input: Sql - config: - path: /path/to/all.json + - type: Sql + config: + query: select left.col1, right.col2 from left join right using (col3) + input: + left: ReadLeft + right: ReadRight - - type: chain - name: ExtraProcessingForBigRows - input: Sql - transforms: - - type: Filter - config: - language: python - keep: "col2 > 100" - - type: Filter - config: - language: python - keep: "len(col1) > 10" - - type: Filter - config: - language: python - keep: "col1 > 'z'" - sink: - type: WriteToCsv + - type: WriteToJson + name: WriteAll + input: Sql config: - path: /path/to/big.csv + path: /path/to/all.json + + - type: chain + name: ExtraProcessingForBigRows + input: Sql + transforms: + - type: Filter + config: + language: python + keep: "col2 > 100" + - type: Filter + config: + language: python + keep: "len(col1) > 10" + - type: Filter + config: + language: python + keep: "col1 > 'z'" + sink: + type: WriteToCsv + config: + path: /path/to/big.csv ``` ## Windowing @@ -329,25 +331,26 @@ a join per window. ``` pipeline: - - type: ReadFromPubSub - name: ReadLeft - config: - topic: leftTopic + transforms: + - type: ReadFromPubSub + name: ReadLeft + config: + topic: leftTopic - - type: ReadFromPubSub - name: ReadRight - config: - topic: rightTopic + - type: ReadFromPubSub + name: ReadRight + config: + topic: rightTopic - - type: Sql - config: - query: select left.col1, right.col2 from left join right using (col3) - input: - left: ReadLeft - right: ReadRight - windowing: - type: fixed - size: 60 + - type: Sql + config: + query: select left.col1, right.col2 from left join right using (col3) + input: + left: ReadLeft + right: ReadRight + windowing: + type: fixed + size: 60 ``` For a transform with no inputs, the specified windowing is instead applied to diff --git a/sdks/python/apache_beam/yaml/pipeline.schema.yaml b/sdks/python/apache_beam/yaml/pipeline.schema.yaml index ef0d9fe0f262..e784531d9be1 100644 --- a/sdks/python/apache_beam/yaml/pipeline.schema.yaml +++ b/sdks/python/apache_beam/yaml/pipeline.schema.yaml @@ -15,7 +15,7 @@ # limitations under the License. # -$schema: 'http://json-schema.org/schema#' +$schema: 'http://json-schema.org/draft-07/schema#' $id: https://github.com/apache/beam/tree/master/sdks/python/apache_beam/yaml/pipeline.schema.yaml $defs: @@ -115,6 +115,23 @@ $defs: - $ref: '#/$defs/nestedTransform' - $ref: '#/$defs/implicitInputOutputs' + - if: + not: + anyOf: + - properties: { type: { const: composite }} + - properties: { type: { const: chain }} + then: + properties: + type: {} + name: {} + input: {} + output: {} + windowing: {} + config: { type: object } + __line__: {} + __uuid__: {} + additionalProperties: false + windowing: {} # TODO provider: @@ -128,27 +145,38 @@ $defs: properties: { __line__: {}} additionalProperties: type: string + config: { type: object } + additionalProperties: false required: - type - transforms + - config type: object properties: pipeline: - anyOf: - - type: array - items: - $ref: '#/$defs/transform' - - $ref: '#/$defs/transform' + allOf: + # These are the only top-level properties defined in pipeline. - type: object properties: - transforms: - type: array - items: - $ref: '#/$defs/transform' + type: { const: chain } + windowing: + $ref: '#/$defs/windowing' + transforms: {} + extra_transforms: {} + sink: {} + source: {} __line__: {} __uuid__: {} additionalProperties: false + # This defines the allowable contents of the attributes above. + - $ref: '#/$defs/nestedTransform' + # A chain-type transform, like a chain composite, must have implicit io. + - if: + properties: { type: { const: chain }} + required: [type] + then: + $ref: '#/$defs/implicitInputOutputs' providers: type: array items: From 97a52af980d521cfbeba1c3f272942cf39d555f1 Mon Sep 17 00:00:00 2001 From: Jeff Kinard <35542536+Polber@users.noreply.github.com> Date: Fri, 20 Oct 2023 19:08:58 -0400 Subject: [PATCH 217/435] [yaml] support gcs location for main.py yaml input file (#29014) --- sdks/python/apache_beam/yaml/main.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/yaml/main.py b/sdks/python/apache_beam/yaml/main.py index e2ec8df9cfc3..331b9e7b3616 100644 --- a/sdks/python/apache_beam/yaml/main.py +++ b/sdks/python/apache_beam/yaml/main.py @@ -20,6 +20,7 @@ import yaml import apache_beam as beam +from apache_beam.io.filesystems import FileSystems from apache_beam.typehints.schemas import LogicalType from apache_beam.typehints.schemas import MillisInstant from apache_beam.yaml import yaml_transform @@ -43,8 +44,8 @@ def _pipeline_spec_from_args(known_args): raise ValueError( "Exactly one of pipeline_spec or pipeline_spec_file must be set.") elif known_args.pipeline_spec_file: - with open(known_args.pipeline_spec_file) as fin: - pipeline_yaml = fin.read() + with FileSystems.open(known_args.pipeline_spec_file) as fin: + pipeline_yaml = fin.read().decode() elif known_args.pipeline_spec: pipeline_yaml = known_args.pipeline_spec else: From abce1ad34ebd34c58c34b505a5df89a54ec0dcb4 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw <robertwb@gmail.com> Date: Fri, 20 Oct 2023 16:26:49 -0700 Subject: [PATCH 218/435] Merge pull request #29093 [YAML] Don't require redundant input for YamlTransform. --- sdks/python/apache_beam/yaml/yaml_io_test.py | 6 ------ sdks/python/apache_beam/yaml/yaml_mapping_test.py | 4 ---- sdks/python/apache_beam/yaml/yaml_transform.py | 12 +++++++++++- sdks/python/apache_beam/yaml/yaml_transform_test.py | 12 +++++++++++- .../apache_beam/yaml/yaml_transform_unit_test.py | 2 -- sdks/python/apache_beam/yaml/yaml_udf_test.py | 8 -------- 6 files changed, 22 insertions(+), 22 deletions(-) diff --git a/sdks/python/apache_beam/yaml/yaml_io_test.py b/sdks/python/apache_beam/yaml/yaml_io_test.py index 7071860a7bf1..a0bd65c78ca5 100644 --- a/sdks/python/apache_beam/yaml/yaml_io_test.py +++ b/sdks/python/apache_beam/yaml/yaml_io_test.py @@ -319,7 +319,6 @@ def test_simple_write(self): | YamlTransform( ''' type: WriteToPubSub - input: input config: topic: my_topic format: raw @@ -341,7 +340,6 @@ def test_write_with_attribute(self): ]) | YamlTransform( ''' type: WriteToPubSub - input: input config: topic: my_topic format: raw @@ -364,7 +362,6 @@ def test_write_with_attribute_map(self): ]) | YamlTransform( ''' type: WriteToPubSub - input: input config: topic: my_topic format: raw @@ -384,7 +381,6 @@ def test_write_with_id_attribute(self): | YamlTransform( ''' type: WriteToPubSub - input: input config: topic: my_topic format: raw @@ -408,7 +404,6 @@ def test_write_avro(self): | YamlTransform( ''' type: WriteToPubSub - input: input config: topic: my_topic format: avro @@ -434,7 +429,6 @@ def test_write_json(self): ]) | YamlTransform( ''' type: WriteToPubSub - input: input config: topic: my_topic format: json diff --git a/sdks/python/apache_beam/yaml/yaml_mapping_test.py b/sdks/python/apache_beam/yaml/yaml_mapping_test.py index 55032aeae52e..a09214c26b6d 100644 --- a/sdks/python/apache_beam/yaml/yaml_mapping_test.py +++ b/sdks/python/apache_beam/yaml/yaml_mapping_test.py @@ -40,7 +40,6 @@ def test_basic(self): result = elements | YamlTransform( ''' type: MapToFields - input: input config: language: python fields: @@ -62,7 +61,6 @@ def test_drop(self): result = elements | YamlTransform( ''' type: MapToFields - input: input config: fields: {} append: true @@ -83,7 +81,6 @@ def test_filter(self): result = elements | YamlTransform( ''' type: Filter - input: input config: language: python keep: "rank > 0" @@ -106,7 +103,6 @@ def test_explode(self): result = elements | YamlTransform( ''' type: chain - input: input transforms: - type: MapToFields config: diff --git a/sdks/python/apache_beam/yaml/yaml_transform.py b/sdks/python/apache_beam/yaml/yaml_transform.py index fa30c1830809..7ab8da33f1a5 100644 --- a/sdks/python/apache_beam/yaml/yaml_transform.py +++ b/sdks/python/apache_beam/yaml/yaml_transform.py @@ -522,7 +522,7 @@ def is_not_output_of_last_transform(new_transforms, value): raise TypeError( f"Chain at {identify_object(spec)} missing transforms property.") has_explicit_outputs = 'output' in spec - composite_spec = normalize_inputs_outputs(spec) + composite_spec = normalize_inputs_outputs(tag_explicit_inputs(spec)) new_transforms = [] for ix, transform in enumerate(composite_spec['transforms']): if any(io in transform for io in ('input', 'output')): @@ -539,6 +539,8 @@ def is_not_output_of_last_transform(new_transforms, value): pass elif is_explicitly_empty(composite_spec['input']): transform['input'] = composite_spec['input'] + elif is_empty(composite_spec['input']): + del composite_spec['input'] else: transform['input'] = { key: key @@ -931,6 +933,7 @@ def __init__(self, spec, providers={}): # pylint: disable=dangerous-default-val self._providers = yaml_provider.merge_providers( providers, yaml_provider.standard_providers()) self._spec = preprocess(spec, known_transforms=self._providers.keys()) + self._was_chain = spec['type'] == 'chain' def expand(self, pcolls): if isinstance(pcolls, beam.pvalue.PBegin): @@ -939,8 +942,15 @@ def expand(self, pcolls): elif isinstance(pcolls, beam.PCollection): root = pcolls.pipeline pcolls = {'input': pcolls} + if not self._spec['input']: + self._spec['input'] = {'input': 'input'} + if self._was_chain and self._spec['transforms']: + # This should have been copied as part of the composite-to-chain. + self._spec['transforms'][0]['input'] = self._spec['input'] else: root = next(iter(pcolls.values())).pipeline + if not self._spec['input']: + self._spec['input'] = {name: name for name in pcolls.keys()} result = expand_transform( self._spec, Scope( diff --git a/sdks/python/apache_beam/yaml/yaml_transform_test.py b/sdks/python/apache_beam/yaml/yaml_transform_test.py index 63f2e0e7facd..05bbf4196473 100644 --- a/sdks/python/apache_beam/yaml/yaml_transform_test.py +++ b/sdks/python/apache_beam/yaml/yaml_transform_test.py @@ -43,6 +43,16 @@ def expand(self, p): | beam.Map(lambda x: beam.transforms.window.TimestampedValue(x, x))) +class CreateInts(beam.PTransform): + _yaml_requires_inputs = False + + def __init__(self, elements): + self._elements = elements + + def expand(self, p): + return p | beam.Create(self._elements) + + class SumGlobally(beam.PTransform): def expand(self, pcoll): return pcoll | beam.CombineGlobally(sum).without_defaults() @@ -65,7 +75,7 @@ def raise_on_big(element): TEST_PROVIDERS = { - 'CreateInts': lambda elements: beam.Create(elements), + 'CreateInts': CreateInts, 'CreateTimestamped': CreateTimestamped, 'SumGlobally': SumGlobally, 'SizeLimiter': SizeLimiter, diff --git a/sdks/python/apache_beam/yaml/yaml_transform_unit_test.py b/sdks/python/apache_beam/yaml/yaml_transform_unit_test.py index 5d5e5850fd73..d1886ba4dcfb 100644 --- a/sdks/python/apache_beam/yaml/yaml_transform_unit_test.py +++ b/sdks/python/apache_beam/yaml/yaml_transform_unit_test.py @@ -244,12 +244,10 @@ def test_chain_as_composite(self): expected = f''' type: composite name: Chain - input: {{}} transforms: - type: Create config: elements: [0,1,2] - input: {{}} - type: PyMap config: fn: 'lambda x: x*x' diff --git a/sdks/python/apache_beam/yaml/yaml_udf_test.py b/sdks/python/apache_beam/yaml/yaml_udf_test.py index 5e9faa08253c..42bdf6e0bd5b 100644 --- a/sdks/python/apache_beam/yaml/yaml_udf_test.py +++ b/sdks/python/apache_beam/yaml/yaml_udf_test.py @@ -55,7 +55,6 @@ def test_map_to_fields_filter_inline_js(self): result = elements | YamlTransform( ''' type: MapToFields - input: input config: language: javascript fields: @@ -79,7 +78,6 @@ def test_map_to_fields_filter_inline_py(self): result = elements | YamlTransform( ''' type: MapToFields - input: input config: language: python fields: @@ -103,7 +101,6 @@ def test_filter_inline_js(self): result = elements | YamlTransform( ''' type: Filter - input: input config: language: javascript keep: @@ -123,7 +120,6 @@ def test_filter_inline_py(self): result = elements | YamlTransform( ''' type: Filter - input: input config: language: python keep: @@ -143,7 +139,6 @@ def test_filter_expression_js(self): result = elements | YamlTransform( ''' type: Filter - input: input config: language: javascript keep: @@ -162,7 +157,6 @@ def test_filter_expression_py(self): result = elements | YamlTransform( ''' type: Filter - input: input config: language: python keep: @@ -194,7 +188,6 @@ def test_filter_inline_js_file(self): result = elements | YamlTransform( f''' type: Filter - input: input config: language: javascript keep: @@ -226,7 +219,6 @@ def g(x): result = elements | YamlTransform( f''' type: Filter - input: input config: language: python keep: From ffd44b446c40874eda83909cab59540365dd022f Mon Sep 17 00:00:00 2001 From: Andrey Devyatkin <andrey.9evyatkin@gmail.com> Date: Sat, 21 Oct 2023 04:17:30 +0200 Subject: [PATCH 219/435] Add workflows to README (#29096) * Add readme for PerformanceTests TextIOIT, JDBC, Kafka IO, SpannerIO, SQLBigQueryIO and BiqQueryIO Python * Update readme * PRs 28582 28584 28606 28581 * PR 28738 LoadTests_Java_GBK_Dataflow * Add readme for PostCommit Java Examples Dataflow V2 * Add readme for LoadTests Java CoGBK * Add readme for LoadTests Python CoGBK Dataflow * Add readme for LoadTests Python ParDo and SideInput * Add readme for LoadTests Smoke Python and Java * Update Readme * Update Readme * updated README * Update readme for Performance Tests BigQueryIO Write Python Batch * Remove Trigger Phrases for Load Tests and Performance tests * PR 28846 28730 28827 28861 28897 * update readme --------- Co-authored-by: aleksandr-dudko <aleksandr.dudko@akvelon.com> Co-authored-by: vitaly.terentyev <vitaly.terentyev@akvelon.com> Co-authored-by: magicgoody <magzhan.kulatayev@akvelon.com> --- .github/workflows/README.md | 172 +++++++++++++----- .../beam_PerformanceTests_SingleStoreIO.yml | 6 +- 2 files changed, 131 insertions(+), 47 deletions(-) diff --git a/.github/workflows/README.md b/.github/workflows/README.md index c17a6614acbb..ea497ae6404e 100644 --- a/.github/workflows/README.md +++ b/.github/workflows/README.md @@ -180,18 +180,98 @@ Please note that jobs with matrix need to have matrix element in the comment. Ex ```Run Python PreCommit (3.8)``` | Workflow name | Matrix | Trigger Phrase | Cron Status | |:-------------:|:------:|:--------------:|:-----------:| +| [ Cancel Stale Dataflow Jobs ](https://github.com/apache/beam/actions/workflows/beam_CancelStaleDataflowJobs.yml) | N/A | `Run Cancel Stale Dataflow Jobs` | [![.github/workflows/beam_CancelStaleDataflowJobs.yml](https://github.com/apache/beam/actions/workflows/beam_CancelStaleDataflowJobs.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_CancelStaleDataflowJobs.yml) | +| [ Clean Up GCP Resources ](https://github.com/apache/beam/actions/workflows/beam_CleanUpGCPResources.yml) | N/A | `Run Clean GCP Resources` | [![.github/workflows/beam_CleanUpGCPResources.yml](https://github.com/apache/beam/actions/workflows/beam_CleanUpGCPResources.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_CleanUpGCPResources.yml) | +| [ Clean Up Prebuilt SDK Images ](https://github.com/apache/beam/actions/workflows/beam_CleanUpPrebuiltSDKImages.yml) | N/A | `Run Clean Prebuilt Images` | [![.github/workflows/beam_beam_CleanUpPrebuiltSDKImages.yml](https://github.com/apache/beam/actions/workflows/beam_CleanUpPrebuiltSDKImages.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_CleanUpPrebuiltSDKImages.yml) | +| [ Cleanup Dataproc Resources ](https://github.com/apache/beam/actions/workflows/beam_CleanUpDataprocResources.yml) | N/A | N/A | [![.github/workflows/beam_CleanUpDataprocResources.yml](https://github.com/apache/beam/actions/workflows/beam_CleanUpDataprocResources.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_CleanUpDataprocResources.yml) +| [ CloudML Benchmarks Dataflow ](https://github.com/apache/beam/actions/workflows/beam_CloudML_Benchmarks_Dataflow.yml) | N/A |`Run TFT Criteo Benchmarks`| [![.github/workflows/beam_CloudML_Benchmarks_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_CloudML_Benchmarks_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_CloudML_Benchmarks_Dataflow.yml) +| [ Community Metrics Prober ](https://github.com/apache/beam/actions/workflows/beam_Prober_CommunityMetrics.yml) | N/A |`Run Community Metrics Prober`| [![.github/workflows/beam_Prober_CommunityMetrics.yml](https://github.com/apache/beam/actions/workflows/beam_Prober_CommunityMetrics.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_Prober_CommunityMetrics.yml) +| [ Inference Python Benchmarks Dataflow ](https://github.com/apache/beam/actions/workflows/beam_Inference_Python_Benchmarks_Dataflow.yml) | N/A |`Run Inference Benchmarks`| [![.github/workflows/beam_Inference_Python_Benchmarks_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_Inference_Python_Benchmarks_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_Inference_Python_Benchmarks_Dataflow.yml) | [ Java InfluxDbIO Integration Test ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml) | N/A |`Run Java InfluxDbIO_IT`| [![.github/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml) -| [ Load Tests GBK Dataflow Batch Go ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml) | N/A |`Run Load Tests Go GBK Dataflow Batch`| [![.github/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml) -| [ Load Tests CoGBK Dataflow Streaming Java ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml) | N/A |`Run Load Tests Java CoGBK Dataflow Streaming`| [![.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml) -| [ Load Tests Combine Dataflow Batch Python ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml) | N/A |`Run Load Tests Python Combine Dataflow Batch`| [![.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml) -| [ Load Tests Combine Dataflow Batch Python ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml) | N/A |`Run Load Tests Python Combine Dataflow Batch`| [![.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml) -| [ Load Tests FnApiRunner Microbenchmark Python ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_FnApiRunner_Microbenchmark.yml) | N/A |`Run Python Load Tests FnApiRunner Microbenchmark`| [![.github/workflows/beam_LoadTests_Python_FnApiRunner_Microbenchmark](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_FnApiRunner_Microbenchmark.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_FnApiRunner_Microbenchmark.yml) -| [ Load Tests ParDo Dataflow Batch Go ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml) | N/A |`Run Load Tests Go ParDo Dataflow Batch`| [![.github/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml) -| [ Performance Tests AvroIOIT HDFS ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml) | N/A |`Run Java AvroIO Performance Test HDFS`| [![.github/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml) -| [ Performance Tests AvroIOIT ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_AvroIOIT.yml) | N/A |`Run Java AvroIO Performance Test`| [![.github/workflows/beam_PerformanceTests_AvroIOIT.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_AvroIOIT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_AvroIOIT.yml) -| [ Performance Tests BigQueryIO Batch Java Avro ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml) | N/A |`Run BigQueryIO Batch Performance Test Java Avro`| [![.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml) -| [ Performance Tests BigQueryIO Batch Java Json ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml) | N/A |`Run BigQueryIO Batch Performance Test Java Json`| [![.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml) -| [ Performance Tests BigQueryIO Streaming Java ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml) | N/A |`Run BigQueryIO Streaming Performance Test Java`| [![.github/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml) +| [ Java JMH ](https://github.com/apache/beam/actions/workflows/beam_Java_JMH.yml) | N/A | N/A | [![.github/workflows/beam_Java_JMH.yml](https://github.com/apache/beam/actions/workflows/beam_Java_JMH.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_Java_JMH.yml) +| [ Load Tests GBK Dataflow Batch Go ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml) +| [ Load Tests CoGBK Dataflow Batch Go ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml) +| [ Load Tests CoGBK Dataflow Streaming Java ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml) +| [ Load Tests Combine Dataflow Batch Java ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml) +| [ Load Tests Combine Dataflow Batch Python ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml) +| [ Load Tests Combine Dataflow Batch Python ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml) +| [ Load Tests FnApiRunner Microbenchmark Python ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_FnApiRunner_Microbenchmark.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Python_FnApiRunner_Microbenchmark](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_FnApiRunner_Microbenchmark.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_FnApiRunner_Microbenchmark.yml) +| [ Load Tests Go CoGBK Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_CoGBK_Flink_Batch.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Go_CoGBK_Flink_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_CoGBK_Flink_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_CoGBK_Flink_Batch.yml) +| [ Load Tests Go Combine Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml) +| [ Load Tests Go GBK Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml) +| [ Load Tests Go ParDo Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml) +| [ Load Tests Go SideInput Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml) +| [ LoadTests Java CoGBK Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml) +| [ LoadTests Java CoGBK Dataflow V2 Batch JavaVersions ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml) | ['11','17'] | N/A | [![.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml) +| [ LoadTests Java CoGBK Dataflow V2 Streaming JavaVersions ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml) | ['11','17'] | N/A | [![.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml) +| [ LoadTests Java CoGBK SparkStructuredStreaming Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml) +| [ Load Tests Java Combine Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_Combine_Dataflow_Streaming.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_Combine_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_Combine_Dataflow_Streaming.yml) +| [ Load Tests Java Combine SparkStructuredStreaming Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml) +| [ Load Tests Java ParDo Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml) +| [ Load Tests Java ParDo Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml) +| [ LoadTests Java ParDo Dataflow V2 Batch JavaVersions ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions.yml) | ['11','17'] | N/A | [![.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions.yml) +| [ LoadTests Java ParDo Dataflow V2 Streaming JavaVersions ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions.yml) | ['11','17'] | N/A | [![.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions.yml) +| [ Load Tests Java ParDo SparkStructuredStreaming Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml) +| [ Load Tests ParDo Dataflow Batch Go ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml) +| [ Load Tests SideInput Dataflow Batch Go ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml) +| [ LoadTests Java Combine Smoke ](https://github.com/apache/beam/actions/workflows/beam_Java_LoadTests_Combine_Smoke.yml) | N/A | N/A | [![.github/workflows/beam_Java_LoadTests_Combine_Smoke.yml](https://github.com/apache/beam/actions/workflows/beam_Java_LoadTests_Combine_Smoke.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_Java_LoadTests_Combine_Smoke.yml) +| [ LoadTests Java GBK Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml) +| [ LoadTests Java GBK Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml) +| [ LoadTests Java GBK Dataflow V2 Batch Java11 ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml) +| [ LoadTests Java GBK Dataflow V2 Batch Java17 ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml) +| [ LoadTests Java GBK Dataflow V2 Streaming Java11 ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml) +| [ LoadTests Java GBK Dataflow V2 Streaming Java17 ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml) +| [ LoadTests Java GBK Smoke ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Smoke.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Java_GBK_Smoke.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Smoke.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Smoke.yml) +| [ LoadTests Python GBK reiterate Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Batch.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Batch.yml) +| [ LoadTests Python GBK reiterate Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Streaming.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Streaming.yml) +| [ LoadTests Python CoGBK Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml) +| [ LoadTests Python CoGBK Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml) +| [ LoadTests Python Combine Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Dataflow_Streaming.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Dataflow_Streaming.yml) +| [ LoadTests Python Combine Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml) +| [ LoadTests Python Combine Flink Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml) +| [ LoadTests Python GBK Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml) +| [ LoadTests Python GBK Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml) +| [ LoadTests Python GBK Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_Flink_Batch.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Python_GBK_Flink_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_Flink_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_Flink_Batch.yml) +| [ LoadTests Python CoGBK Dataflow Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml) +| [ LoadTests Python ParDo Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Dataflow_Batch.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Dataflow_Batch.yml) +| [ LoadTests Python ParDo Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml) +| [ LoadTests Python ParDo Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml) +| [ LoadTests Python ParDo Flink Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml) +| [ LoadTests Python SideInput Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_SideInput_Dataflow_Batch.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Python_SideInput_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_SideInput_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_SideInput_Dataflow_Batch.yml) +| [ LoadTests Python Smoke ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Smoke.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Python_Smoke.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Smoke.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Smoke.yml) +| [ Performance Tests AvroIOIT HDFS ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml) | N/A | N/A | [![.github/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml) +| [ Performance Tests AvroIOIT ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_AvroIOIT.yml) | N/A | N/A | [![.github/workflows/beam_PerformanceTests_AvroIOIT.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_AvroIOIT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_AvroIOIT.yml) +| [ Performance Tests BigQueryIO Batch Java Avro ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml) | N/A | N/A | [![.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml) +| [ Performance Tests BigQueryIO Batch Java Json ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml) | N/A | N/A | [![.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml) +| [ Performance Tests BigQueryIO Streaming Java ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml) | N/A | N/A | [![.github/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml) +| [ Performance Tests BigQueryIO Read Python ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml) | N/A | N/A | [![.github/workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml) +| [ Performance Tests BigQueryIO Write Python Batch ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml) | N/A | N/A | [![.github/workflows/beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml) +| [ PerformanceTests Cdap ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Cdap.yml) | N/A | N/A | [![.github/workflows/beam_PerformanceTests_Cdap.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Cdap.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Cdap.yml) +| [ PerformanceTests Compressed TextIOIT HDFS ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml) | N/A | N/A | [![.github/workflows/beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml) +| [ PerformanceTests Compressed TextIOIT ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml) | N/A | N/A | [![.github/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml) +| [ PerformanceTests HadoopFormat ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_HadoopFormat.yml) | N/A | N/A | [![.github/workflows/beam_PerformanceTests_HadoopFormat.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_HadoopFormat.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_HadoopFormat.yml) +| [ PerformanceTests JDBC ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_JDBC.yml) | N/A | N/A | [![.github/workflows/beam_PerformanceTests_JDBC.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_JDBC.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_JDBC.yml) +| [ PerformanceTests Kafka IO ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Kafka_IO.yml) | N/A | N/A | [![.github/workflows/beam_PerformanceTests_Kafka_IO.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Kafka_IO.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Kafka_IO.yml) +| [ PerformanceTests ManyFiles TextIOIT HDFS ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml) | N/A | N/A | [![.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml) +| [ PerformanceTests ManyFiles TextIOIT ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml) | N/A | N/A | [![.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml) +| [ PerformanceTests MongoDBIO IT ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_MongoDBIO_IT.yml) | N/A | N/A | [![.github/workflows/beam_PerformanceTests_MongoDBIO_IT.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_MongoDBIO_IT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_MongoDBIO_IT.yml) +| [ PerformanceTests ParquetIOIT HDFS ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml) | N/A | N/A | [![.github/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml) +| [ PerformanceTests ParquetIOIT ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ParquetIOIT.yml) | N/A | N/A | [![.github/workflows/beam_PerformanceTests_ParquetIOIT.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ParquetIOIT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ParquetIOIT.yml) +| [ PerformanceTests PubsubIOIT Python Streaming ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml) | N/A | N/A | [![.github/workflows/beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml) +| [ PerformanceTests SingleStoreIO ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SingleStoreIO.yml) | N/A | N/A | [![.github/workflows/beam_PerformanceTests_SingleStoreIO.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SingleStoreIO.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SingleStoreIO.yml) +| [ PerformanceTests SpannerIO Read 2GB Python ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml) | N/A | N/A | [![.github/workflows/beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml) +| [ PerformanceTests SpannerIO Write 2GB Python Batch ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml) | N/A | N/A | [![.github/workflows/beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml) +| [ PerformanceTests SparkReceiver IO ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SparkReceiver_IO.yml) | N/A | N/A | [![.github/workflows/beam_PerformanceTests_SparkReceiver_IO.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SparkReceiver_IO.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SparkReceiver_IO.yml) +| [ PerformanceTests SQLBigQueryIO Batch Java ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml) | N/A | N/A | [![.github/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml) +| [ PerformanceTests TextIOIT HDFS ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml) | N/A | N/A | [![.github/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml) +| [ PerformanceTests TextIOIT Python ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TextIOIT_Python.yml) | N/A | N/A | [![.github/workflows/beam_PerformanceTests_TextIOIT_Python.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TextIOIT_Python.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TextIOIT_Python.yml) +| [ PerformanceTests TextIOIT ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TextIOIT.yml) | N/A | N/A | [![.github/workflows/beam_PerformanceTests_TextIOIT.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TextIOIT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TextIOIT.yml) +| [ PerformanceTests TFRecordIOIT HDFS ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml) | N/A | N/A | [![.github/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml) +| [ PerformanceTests TFRecordIOIT ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TFRecordIOIT.yml) | N/A | N/A | [![.github/workflows/beam_PerformanceTests_TFRecordIOIT.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TFRecordIOIT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TFRecordIOIT.yml) +| [ PerformanceTests WordCountIT PythonVersions ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml) | ['3.8'] | N/A | [![.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml) +| [ PerformanceTests XmlIOIT HDFS ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml) | N/A | N/A | [![.github/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml) +| [ PerformanceTests XmlIOIT ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_XmlIOIT.yml) | N/A | N/A | [![.github/workflows/beam_PerformanceTests_XmlIOIT.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_XmlIOIT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_XmlIOIT.yml) +| [ PerformanceTests xlang KafkaIO Python ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml) | N/A | N/A | [![.github/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml) | [ PostCommit BeamMetrics Publish ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_BeamMetrics_Publish.yml) | N/A |`Run Beam Metrics Deployment`| [![.github/workflows/beam_PostCommit_BeamMetrics_Publish.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_BeamMetrics_Publish.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_BeamMetrics_Publish.yml) | [ PostCommit Go ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go.yml) | N/A |`Run Go PostCommit`| [![.github/workflows/beam_PostCommit_Go.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go.yml) | | [ PostCommit Go Dataflow ARM](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go_Dataflow_ARM.yml) | N/A |`Run Go PostCommit Dataflow ARM`| [![.github/workflows/beam_PostCommit_Go_Dataflow_ARM.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go_Dataflow_ARM.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go_Dataflow_ARM.yml) | @@ -204,36 +284,38 @@ Please note that jobs with matrix need to have matrix element in the comment. Ex | [ PostCommit Java Examples Dataflow ARM ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml) | ['8','11','17'] |`Run Java_Examples_Dataflow_ARM PostCommit (matrix_element)`| [![.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml) | | [ PostCommit Java Examples Dataflow](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow.yml) | N/A |`Run Java examples on Dataflow`| [![.github/workflows/beam_PostCommit_Java_Examples_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow.yml) | | [ PostCommit Java Examples Dataflow Java ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml) | ['11','17'] |`Run Java examples on Dataflow Java (matrix_element)`| [![.github/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml) | +| [ PostCommit Java Examples Dataflow V2 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml) | N/A |`Run Java Examples on Dataflow Runner V2`| [![.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml) | +| [ PostCommit Java Examples Dataflow V2 Java ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml) | ['11','17'] |`Run Java (matrix_element) Examples on Dataflow Runner V2`| [![.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml) | | [ PostCommit Java Examples Direct ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Direct.yml) | N/A |`Run Java Examples_Direct`| [![.github/workflows/beam_PostCommit_Java_Examples_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Direct.yml) | | [ PostCommit Java Examples Flink ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Flink.yml) | N/A |`Run Java Examples_Flink`| [![.github/workflows/beam_PostCommit_Java_Examples_Flink.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Flink.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Flink.yml) | | [ PostCommit Java Examples Spark ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Spark.yml) | N/A |`Run Java Examples_Spark`| [![.github/workflows/beam_PostCommit_Java_Examples_Spark.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Spark.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Spark.yml) | | [ PostCommit Java Hadoop Versions ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Hadoop_Versions.yml) | N/A |`Run PostCommit_Java_Hadoop_Versions`| [![.github/workflows/beam_PostCommit_Java_Hadoop_Versions.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Hadoop_Versions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Hadoop_Versions.yml) | -| [ PostCommit Java Jpms Dataflow Java11 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml) | N/A |`Run Jpms Dataflow Java 11 PostCommit`| [![.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml) | -| [ PostCommit Java Jpms Dataflow Java17 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml) | N/A |`Run Jpms Dataflow Java 17 PostCommit`| [![.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml) | -| [ PostCommit Java Jpms Direct Java11 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml) | N/A |`Run Jpms Direct Java 11 PostCommit`| [![.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java11](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml) | -| [ PostCommit Java Jpms Direct Java17 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml) | N/A |`Run Jpms Direct Java 17 PostCommit`| [![.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml) | -| [ PostCommit Java Jpms Flink Java11 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml) | N/A |`Run Jpms Flink Java 11 PostCommit`| [![.github/workflows/beam_PostCommit_Java_Jpms_Flink_Java11](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml) | -| [ PostCommit Java Jpms Spark Java11 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml) | N/A |`Run Jpms Spark Java 11 PostCommit`| [![.github/workflows/beam_PostCommit_Java_Jpms_Spark_Java11](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml) | +| [ PostCommit Java Jpms Dataflow Java11 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml) | N/A |`Run Jpms Dataflow Java 11 PostCommit`| [![.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml) | +| [ PostCommit Java Jpms Dataflow Java17 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml) | N/A |`Run Jpms Dataflow Java 17 PostCommit`| [![.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml) | +| [ PostCommit Java Jpms Direct Java11 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml) | N/A |`Run Jpms Direct Java 11 PostCommit`| [![.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml) | +| [ PostCommit Java Jpms Direct Java17 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml) | N/A |`Run Jpms Direct Java 17 PostCommit`| [![.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml) | +| [ PostCommit Java Jpms Flink Java11 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml) | N/A |`Run Jpms Flink Java 11 PostCommit`| [![.github/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml) | +| [ PostCommit Java Jpms Spark Java11 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml) | N/A |`Run Jpms Spark Java 11 PostCommit`| [![.github/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml) | | [ PostCommit Java Nexmark Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml) | N/A |`Run Dataflow Runner Nexmark Tests`| [![.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml) | | [ PostCommit Java Nexmark Dataflow V2 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml) | N/A |`Run Dataflow Runner V2 Nexmark Tests`| [![.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml) | | [ PostCommit Java Nexmark Dataflow V2 Java ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml) | ['11','17'] |`Run Dataflow Runner V2 Java (matrix) Nexmark Tests`| [![.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml) | | [ PostCommit Java Nexmark Direct ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Direct.yml) | N/A |`Run Direct Runner Nexmark Tests`| [![.github/workflows/beam_PostCommit_Java_Nexmark_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Direct.yml) | | [ PostCommit Java Nexmark Flink ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Flink.yml) | N/A |`Run Flink Runner Nexmark Tests`| [![.github/workflows/beam_PostCommit_Java_Nexmark_Flink.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Flink.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Flink.yml) | | [ PostCommit Java Nexmark Spark ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Spark.yml) | N/A |`Run Spark Runner Nexmark Tests`| [![.github/workflows/beam_PostCommit_Java_Nexmark_Spark.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Spark.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Spark.yml) | -| [ PostCommit Java PVR Flink Streaming ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml) | N/A |`Run Java Flink PortableValidatesRunner Streaming`| [![PostCommit Java PVR Flink Streaming](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml) | -| [ PostCommit Java PVR Samza ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Samza.yml) | N/A |`Run Java Samza PortableValidatesRunner`| [![PostCommit Java PVR Samza](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Samza.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Samza.yml) | -| [ PostCommit Java PVR Spark3 Streaming ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml) | N/A |`Run Java Spark v3 PortableValidatesRunner Streaming`| [![PostCommit Java PVR Spark3 Streaming](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml) | -| [ PostCommit Java PVR Spark Batch ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml) | N/A |`Run Java Spark PortableValidatesRunner Batch`| [![PostCommit Java PVR Spark Batch](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml) | +| [ PostCommit Java PVR Flink Streaming ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml) | N/A |`Run Java Flink PortableValidatesRunner Streaming`| [![.github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml) | +| [ PostCommit Java PVR Samza ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Samza.yml) | N/A |`Run Java Samza PortableValidatesRunner`| [![.github/workflows/beam_PostCommit_Java_PVR_Samza.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Samza.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Samza.yml) | +| [ PostCommit Java PVR Spark3 Streaming ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml) | N/A |`Run Java Spark v3 PortableValidatesRunner Streaming`| [![.github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml) | +| [ PostCommit Java PVR Spark Batch ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml) | N/A |`Run Java Spark PortableValidatesRunner Batch`| [![.github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml) | | [ PostCommit Java Sickbay ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Sickbay.yml) | N/A |`Run Java Sickbay`| [![.github/workflows/beam_PostCommit_Java_Sickbay.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Sickbay.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Sickbay.yml) | | [ PostCommit Java Tpcds Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml) | N/A |`Run Dataflow Runner Tpcds Tests`| [![.github/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml) | | [ PostCommit Java Tpcds Flink ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Tpcds_Flink.yml) | N/A |`Run Flink Runner Tpcds Tests`| [![.github/workflows/beam_PostCommit_Java_Tpcds_Flink.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Tpcds_Flink.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Tpcds_Flink.yml) | | [ PostCommit Java Tpcds Spark ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Tpcds_Spark.yml) | N/A |`Run Spark Runner Tpcds Tests`| [![.github/workflows/beam_PostCommit_Java_Tpcds_Spark.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Tpcds_Spark.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Tpcds_Spark.yml) | -| [ PostCommit Java ValidatesRunner Dataflow JavaVersions ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Java.yml) | ['11','17'] |`Run Dataflow ValidatesRunner Java (matrix_element)`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Java.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Java.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Java.yml) | +| [ PostCommit Java ValidatesRunner Dataflow JavaVersions ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml) | ['11','17'] |`Run Dataflow ValidatesRunner Java (matrix_element)`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml) | | [ PostCommit Java ValidatesRunner Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml) | N/A |`Run Dataflow Streaming ValidatesRunner`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml) | | [ PostCommit Java ValidatesRunner Dataflow V2 Streaming ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml) | N/A |`Run Java Dataflow V2 ValidatesRunner Streaming`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml) | | [ PostCommit Java ValidatesRunner Dataflow V2 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml) | N/A |`Run Java Dataflow V2 ValidatesRunner`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml) | | [ PostCommit Java ValidatesRunner Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml) | N/A |`Run Dataflow ValidatesRunner`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml) | -| [ PostCommit Java ValidatesRunner Direct JavaVersions ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_Java.yml) | ['11','17'] |`Run Direct ValidatesRunner Java (matrix_element)`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_Java.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_Java.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_Java.yml) | +| [ PostCommit Java ValidatesRunner Direct JavaVersions ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml) | ['11','17'] |`Run Direct ValidatesRunner Java (matrix_element)`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml) | | [ PostCommit Java ValidatesRunner Direct ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml) | N/A |`Run Direct ValidatesRunner`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml) | | [ PostCommit Java ValidatesRunner Flink Java11 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml) | N/A |`Run Flink ValidatesRunner Java 11`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml) | | [ PostCommit Java ValidatesRunner Flink ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml) | N/A |`Run Flink ValidatesRunner`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml) | @@ -257,10 +339,10 @@ Please note that jobs with matrix need to have matrix element in the comment. Ex | [ PostCommit Python Nexmark Direct ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Nexmark_Direct.yml) | N/A |`Run Python Direct Runner Nexmark Tests`| [![.github/workflows/beam_PostCommit_Python_Nexmark_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Nexmark_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Nexmark_Direct.yml) | | [ PostCommit Python ValidatesContainer Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml) | ['3.8','3.9','3.10','3.11'] |`Run Python Dataflow ValidatesContainer (matrix_element)`| [![.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml) | | [ PostCommit Python ValidatesContainer Dataflow With RC ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml) | ['3.8','3.9','3.10','3.11'] |`Run Python RC Dataflow ValidatesContainer (matrix_element)`| [![.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml) | -| [ PostCommit Python ValidatesRunner Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml) | ['3.8','3.11'] |`Run Python Dataflow ValidatesRunner (matrix_element)`| [![PostCommit Python ValidatesRunner Dataflow](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml) | -| [ PostCommit Python ValidatesRunner Flink ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml) | ['3.8','3.11'] |`Run Python Flink ValidatesRunner (matrix_element)`| [![PostCommit Python ValidatesRunner Flink](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml) | -| [ PostCommit Python ValidatesRunner Samza ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml) | ['3.8','3.11'] |`Run Python Samza ValidatesRunner (matrix_element)`| [![PostCommit Python ValidatesRunner Samza](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml) | -| [ PostCommit Python ValidatesRunner Spark ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml) | ['3.8','3.9','3.11'] |`Run Python Spark ValidatesRunner (matrix_element)`| [![PostCommit Python ValidatesRunner Spark](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml) | +| [ PostCommit Python ValidatesRunner Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml) | ['3.8','3.11'] |`Run Python Dataflow ValidatesRunner (matrix_element)`| [![.github/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml) | +| [ PostCommit Python ValidatesRunner Flink ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml) | ['3.8','3.11'] |`Run Python Flink ValidatesRunner (matrix_element)`| [![.github/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml) | +| [ PostCommit Python ValidatesRunner Samza ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml) | ['3.8','3.11'] |`Run Python Samza ValidatesRunner (matrix_element)`| [![.github/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml) | +| [ PostCommit Python ValidatesRunner Spark ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml) | ['3.8','3.9','3.11'] |`Run Python Spark ValidatesRunner (matrix_element)`| [![.github/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml) | | [ PostCommit Python Xlang Gcp Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml) | N/A |`Run Python_Xlang_Gcp_Dataflow PostCommit`| [![.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml) | | [ PostCommit Python Xlang Gcp Direct ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml) | N/A |`Run Python_Xlang_Gcp_Direct PostCommit`| [![.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml) | | [ PostCommit Python Xlang IO Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml) | N/A |`Run Python_Xlang_IO_Dataflow PostCommit`| [![.github/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml) | @@ -268,15 +350,15 @@ Please note that jobs with matrix need to have matrix element in the comment. Ex | [ PostCommit SQL ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_SQL.yml) | N/A |`Run SQL PostCommit`| [![.github/workflows/beam_PostCommit_SQL.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_SQL.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_SQL.yml) | | [ PostCommit TransformService Direct ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_TransformService_Direct.yml) | N/A |`Run TransformService_Direct PostCommit`| [![.github/workflows/beam_PostCommit_TransformService_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_TransformService_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_TransformService_Direct.yml) | [ PostCommit Website Publish ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Website_Publish.yml) | N/A | N/A | [![.github/workflows/beam_PostCommit_Website_Publish.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Website_Publish.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Website_Publish.yml) | -| [ PostCommit Website Test](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Website_Test.yml) | N/A |`Run Full Website Test`| [![.github/workflows/beam_PostCommit_Website_Test](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Website_Test.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Website_Test.yml) | -| [ PostCommit XVR GoUsingJava Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml) | N/A |`Run XVR_GoUsingJava_Dataflow PostCommit`| [![.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml) | -| [ PostCommit XVR Direct ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Direct.yml) | N/A |`Run XVR_Direct PostCommit`| [![.github/workflows/beam_PostCommit_XVR_Direct](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Direct.yml) | -| [ PostCommit XVR Flink ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Flink.yml) | N/A |`Run XVR_Flink PostCommit`| [![.github/workflows/beam_PostCommit_XVR_Flink](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Flink.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Flink.yml) | -| [ PostCommit XVR JavaUsingPython Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml) | N/A |`Run XVR_JavaUsingPython_Dataflow PostCommit`| [![.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml) | -| [ PostCommit XVR PythonUsingJava Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml) | N/A |`Run XVR_PythonUsingJava_Dataflow PostCommit`| [![.github/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml) | -| [ PostCommit XVR PythonUsingJavaSQL Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml) | N/A |`Run XVR_PythonUsingJavaSQL_Dataflow PostCommit`| [![.github/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml) | -| [ PostCommit XVR Samza ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Samza.yml) | N/A |`Run XVR_Samza PostCommit`| [![.github/workflows/beam_PostCommit_XVR_Samza](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Samza.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Samza.yml) | -| [ PostCommit XVR Spark3 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Spark3.yml) | N/A |`Run XVR_Spark3 PostCommit`| [![.github/workflows/beam_PostCommit_XVR_Spark3](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Spark3.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Spark3.yml) | +| [ PostCommit Website Test](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Website_Test.yml) | N/A |`Run Full Website Test`| [![.github/workflows/beam_PostCommit_Website_Test.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Website_Test.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Website_Test.yml) | +| [ PostCommit XVR GoUsingJava Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml) | N/A |`Run XVR_GoUsingJava_Dataflow PostCommit`| [![.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml) | +| [ PostCommit XVR Direct ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Direct.yml) | N/A |`Run XVR_Direct PostCommit`| [![.github/workflows/beam_PostCommit_XVR_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Direct.yml) | +| [ PostCommit XVR Flink ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Flink.yml) | N/A |`Run XVR_Flink PostCommit`| [![.github/workflows/beam_PostCommit_XVR_Flink.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Flink.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Flink.yml) | +| [ PostCommit XVR JavaUsingPython Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml) | N/A |`Run XVR_JavaUsingPython_Dataflow PostCommit`| [![.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml) | +| [ PostCommit XVR PythonUsingJava Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml) | N/A |`Run XVR_PythonUsingJava_Dataflow PostCommit`| [![.github/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml) | +| [ PostCommit XVR PythonUsingJavaSQL Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml) | N/A |`Run XVR_PythonUsingJavaSQL_Dataflow PostCommit`| [![.github/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml) | +| [ PostCommit XVR Samza ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Samza.yml) | N/A |`Run XVR_Samza PostCommit`| [![.github/workflows/beam_PostCommit_XVR_Samza.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Samza.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Samza.yml) | +| [ PostCommit XVR Spark3 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Spark3.yml) | N/A |`Run XVR_Spark3 PostCommit`| [![.github/workflows/beam_PostCommit_XVR_Spark3.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Spark3.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Spark3.yml) | | [ PreCommit Community Metrics ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_CommunityMetrics.yml) | N/A |`Run CommunityMetrics PreCommit`| [![.github/workflows/beam_PreCommit_CommunityMetrics.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_CommunityMetrics.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_CommunityMetrics.yml) | | [ PreCommit GHA ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_GHA.yml) | N/A |`Run GHA PreCommit`| [![.github/workflows/beam_PreCommit_GHA.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_GHA.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_GHA.yml) | | [ PreCommit Go ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Go.yml) | N/A |`Run Go PreCommit`| [![.github/workflows/beam_PreCommit_Go.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Go.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Go.yml) | @@ -292,12 +374,12 @@ Please note that jobs with matrix need to have matrix element in the comment. Ex | [ PreCommit Java Debezium IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml) | N/A |`Run Java_Debezium_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml) | | [ PreCommit Java ElasticSearch IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml) | N/A |`Run Java_ElasticSearch_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml) | | [ PreCommit Java Examples Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Examples_Dataflow.yml) | N/A |`Run Java_Examples_Dataflow PreCommit`| [![.github/workflows/beam_PreCommit_Java_Examples_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Examples_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Examples_Dataflow.yml) | -| [ PreCommit Java Flink Versions ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Flink_Versions.yml) | N/A |`Run Java_Flink_Versions PreCommit`| [![.github/workflows/beam_PreCommit_Java_Flink_Versions.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Flink_Versions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Flink_Versions.yml) | -| [ PreCommit Java GCP IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml) | N/A |`Run Java_GCP_IO_Direct PreCommit`| [![.github\workflows\beam_PreCommit_Java_GCP_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml) | -| [ PreCommit Java Google-ads IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Google-ads_IO_Direct.yml) | N/A |`Run Java_Google-ads_IO_Direct PreCommit`| [![.github\workflows\beam_PreCommit_Java_Google-ads_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Google-ads_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Google-ads_IO_Direct.yml) | | [ PreCommit Java Examples Dataflow Java11 ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml) | N/A | `Run Java_Examples_Dataflow_Java11 PreCommit` | [![.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml) | | [ PreCommit Java Examples Dataflow Java17 ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml) | N/A | `Run Java_Examples_Dataflow_Java17 PreCommit` | [![.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml) | | [ PreCommit Java File-schema-transform IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_File-schema-transform_IO_Direct.yml) | N/A |`Run Java_File-schema-transform_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_File-schema-transform_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_File-schema-transform_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_File-schema-transform_IO_Direct.yml) | +| [ PreCommit Java Flink Versions ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Flink_Versions.yml) | N/A |`Run Java_Flink_Versions PreCommit`| [![.github/workflows/beam_PreCommit_Java_Flink_Versions.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Flink_Versions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Flink_Versions.yml) | +| [ PreCommit Java GCP IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml) | N/A |`Run Java_GCP_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml) | +| [ PreCommit Java Google-ads IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Google-ads_IO_Direct.yml) | N/A |`Run Java_Google-ads_IO_Direct PreCommit`| [![.github\workflows\beam_PreCommit_Java_Google-ads_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Google-ads_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Google-ads_IO_Direct.yml) | | [ PreCommit Java Hadoop IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml) | N/A |`Run Java_Hadoop_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml) | | [ PreCommit Java HBase IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml) | N/A |`Run Java_HBase_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml) | | [ PreCommit Java HCatalog IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml) | N/A |`Run Java_HCatalog_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml) | @@ -344,12 +426,14 @@ Please note that jobs with matrix need to have matrix element in the comment. Ex | [ PreCommit SQL Java17 ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_SQL_Java17.yml) | N/A |`Run SQL_Java17 PreCommit`| [![.github/workflows/beam_PreCommit_SQL_Java17.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_SQL_Java17.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_SQL_Java17.yml) | | [ PreCommit Typescript ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Typescript.yml) | N/A |`Run Typescript PreCommit`| [![.github/workflows/beam_PreCommit_Typescript.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Typescript.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Typescript.yml) | | [ PreCommit Website ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Website.yml) | N/A |`Run Website PreCommit`| [![.github/workflows/beam_PreCommit_Website.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Website.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Website.yml) | -| [ PreCommit Website Stage GCS ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Website_Stage_GCS.yml) | N/A |`Run Website_Stage_GCS PreCommit`| [![PreCommit Website Stage GCS](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Website_Stage_GCS.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Website_Stage_GCS.yml) | +| [ PreCommit Website Stage GCS ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Website_Stage_GCS.yml) | N/A |`Run Website_Stage_GCS PreCommit`| [![.github/workflows/beam_PreCommit_Website_Stage_GCS.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Website_Stage_GCS.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Website_Stage_GCS.yml) | | [ PreCommit Whitespace ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Whitespace.yml) | N/A |`Run Whitespace PreCommit`| [![.github/workflows/beam_PreCommit_Whitespace.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Whitespace.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Whitespace.yml) | -| [ Python Validates Container Dataflow ARM ](https://github.com/apache/beam/actions/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml) | ['3.8','3.9','3.10','3.11'] | `Run Python ValidatesContainer Dataflow ARM (matrix_element)`| [![.github/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml](https://github.com/apache/beam/actions/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml) | +| [ Python Validates Container Dataflow ARM ](https://github.com/apache/beam/actions/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml) | ['3.8','3.9','3.10','3.11'] |beam_Python_ValidatesContainer_Dataflow_ARM.yml +| [![.github/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml](https://github.com/apache/beam/actions/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml) | | [ PreCommit GoPortable ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_GoPortable.yml) | N/A |`Run GoPortable PreCommit`| [![.github/workflows/beam_PreCommit_GoPortable.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_GoPortable.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_GoPortable.yml) | | [ PreCommit Kotlin Examples ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Kotlin_Examples.yml) | N/A | `Run Kotlin_Examples PreCommit` | [![.github/workflows/beam_PreCommit_Kotlin_Examples.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Kotlin_Examples.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Kotlin_Examples.yml) | | [ PreCommit Portable Python ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Portable_Python.yml) | ['3.8','3.11'] | `Run Portable_Python PreCommit` | [![.github/workflows/beam_PreCommit_Portable_Python.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Portable_Python.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Portable_Python.yml) | -| [ Cancel Stale Dataflow Jobs ](https://github.com/apache/beam/actions/workflows/beam_CancelStaleDataflowJobs.yml) | N/A | `Run Cancel Stale Dataflow Jobs` | [![.github/workflows/beam_CancelStaleDataflowJobs.yml](https://github.com/apache/beam/actions/workflows/beam_CancelStaleDataflowJobs.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_CancelStaleDataflowJobs.yml) | -| [ Clean Up GCP Resources ](https://github.com/apache/beam/actions/workflows/beam_CleanUpGCPResources.yml) | N/A | `Run Clean GCP Resources` | [![.github/workflows/beam_CleanUpGCPResources.yml](https://github.com/apache/beam/actions/workflows/beam_CleanUpGCPResources.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_CleanUpGCPResources.yml) | -| [ Clean Up Prebuilt SDK Images ](https://github.com/apache/beam/actions/workflows/beam_CleanUpPrebuiltSDKImages.yml) | N/A | `Run Clean Prebuilt Images` | [![.github/workflows/beam_beam_CleanUpPrebuiltSDKImages.yml](https://github.com/apache/beam/actions/workflows/beam_CleanUpPrebuiltSDKImages.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_CleanUpPrebuiltSDKImages.yml) | +| [ Publish Beam SDK Snapshots ](https://github.com/apache/beam/actions/workflows/beam_Publish_Beam_SDK_Snapshots.yml) | N/A | N/A | [![.github/workflows/beam_Publish_Beam_SDK_Snapshots.yml](https://github.com/apache/beam/actions/workflows/beam_Publish_Beam_SDK_Snapshots.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_Publish_Beam_SDK_Snapshots.yml) | +| [ Publish Docker Snapshots ](https://github.com/apache/beam/actions/workflows/beam_Publish_Docker_Snapshots.yml) | N/A | N/A | [![.github/workflows/beam_Publish_Docker_Snapshots.yml](https://github.com/apache/beam/actions/workflows/beam_Publish_Docker_Snapshots.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_Publish_Docker_Snapshots.yml) | +| [ Rotate IO-Datastores Cluster Credentials ](https://github.com/apache/beam/actions/workflows/beam_IODatastoresCredentialsRotation.yml) | N/A | N/A | [![.github/workflows/beam_IODatastoresCredentialsRotation.yml](https://github.com/apache/beam/actions/workflows/beam_IODatastoresCredentialsRotation.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_IODatastoresCredentialsRotation.yml) | +| [ Rotate Metrics Cluster Credentials ](https://github.com/apache/beam/actions/workflows/beam_MetricsCredentialsRotation.yml) | N/A | N/A | [![.github/workflows/beam_MetricsCredentialsRotation.yml](https://github.com/apache/beam/actions/workflows/beam_MetricsCredentialsRotation.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_MetricsCredentialsRotation.yml) | diff --git a/.github/workflows/beam_PerformanceTests_SingleStoreIO.yml b/.github/workflows/beam_PerformanceTests_SingleStoreIO.yml index c95434c2ede5..e7bf6cc91cd4 100644 --- a/.github/workflows/beam_PerformanceTests_SingleStoreIO.yml +++ b/.github/workflows/beam_PerformanceTests_SingleStoreIO.yml @@ -15,7 +15,7 @@ # specific language governing permissions and limitations # under the License. -name: Performance Test Java SingleStoreIO +name: PerformanceTests SingleStoreIO on: issue_comment: @@ -23,6 +23,7 @@ on: schedule: - cron: '0 */12 * * *' workflow_dispatch: + #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event permissions: actions: write @@ -39,10 +40,9 @@ permissions: security-events: read statuses: read - # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: From 7bd25a8e6d16f15dde12a2af4a0b6bc10a31110f Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sat, 21 Oct 2023 10:22:52 -0700 Subject: [PATCH 220/435] Bump cloud.google.com/go/spanner from 1.50.0 to 1.51.0 in /sdks (#29046) Bumps [cloud.google.com/go/spanner](https://github.com/googleapis/google-cloud-go) from 1.50.0 to 1.51.0. - [Release notes](https://github.com/googleapis/google-cloud-go/releases) - [Changelog](https://github.com/googleapis/google-cloud-go/blob/main/CHANGES.md) - [Commits](https://github.com/googleapis/google-cloud-go/compare/spanner/v1.50.0...spanner/v1.51.0) --- updated-dependencies: - dependency-name: cloud.google.com/go/spanner dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] <support@github.com> Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 2 +- sdks/go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index b990ee33ea09..6cd7b77be91d 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -28,7 +28,7 @@ require ( cloud.google.com/go/datastore v1.15.0 cloud.google.com/go/profiler v0.4.0 cloud.google.com/go/pubsub v1.33.0 - cloud.google.com/go/spanner v1.50.0 + cloud.google.com/go/spanner v1.51.0 cloud.google.com/go/storage v1.33.0 github.com/aws/aws-sdk-go-v2 v1.21.2 github.com/aws/aws-sdk-go-v2/config v1.19.0 diff --git a/sdks/go.sum b/sdks/go.sum index d4dd5e1ae8e4..27a55827c72b 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -38,8 +38,8 @@ cloud.google.com/go/pubsub v1.1.0/go.mod h1:EwwdRX2sKPjnvnqCa270oGRyludottCI76h+ cloud.google.com/go/pubsub v1.2.0/go.mod h1:jhfEVHT8odbXTkndysNHCcx0awwzvfOlguIAii9o8iA= cloud.google.com/go/pubsub v1.33.0 h1:6SPCPvWav64tj0sVX/+npCBKhUi/UjJehy9op/V3p2g= cloud.google.com/go/pubsub v1.33.0/go.mod h1:f+w71I33OMyxf9VpMVcZbnG5KSUkCOUHYpFd5U1GdRc= -cloud.google.com/go/spanner v1.50.0 h1:QrJFOpaxCXdXF+GkiruLz642PHxkdj68PbbnLw3O2Zw= -cloud.google.com/go/spanner v1.50.0/go.mod h1:eGj9mQGK8+hkgSVbHNQ06pQ4oS+cyc4tXXd6Dif1KoM= +cloud.google.com/go/spanner v1.51.0 h1:l3exhhsVMKsx1E7Xd1QajYSvHmI1KZoWPW5tRxIIdvQ= +cloud.google.com/go/spanner v1.51.0/go.mod h1:c5KNo5LQ1X5tJwma9rSQZsXNBDNvj4/n8BVc3LNahq0= cloud.google.com/go/storage v1.0.0/go.mod h1:IhtSnM/ZTZV8YYJWCY8RULGVqBDmpoyjwiyrjsg+URw= cloud.google.com/go/storage v1.5.0/go.mod h1:tpKbwo567HUNpVclU5sGELwQWBDZ8gh0ZeosJ0Rtdos= cloud.google.com/go/storage v1.6.0/go.mod h1:N7U0C8pVQ/+NIKOBQyamJIeKQKkZ+mxpohlUTyfDhBk= From 84c9ea0a48bc2b2098fc322142e6b0d978766c3e Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sat, 21 Oct 2023 10:32:15 -0700 Subject: [PATCH 221/435] Bump golang.org/x/net from 0.7.0 to 0.17.0 in /learning/katas/go (#28947) Bumps [golang.org/x/net](https://github.com/golang/net) from 0.7.0 to 0.17.0. - [Commits](https://github.com/golang/net/compare/v0.7.0...v0.17.0) --- updated-dependencies: - dependency-name: golang.org/x/net dependency-type: indirect ... Signed-off-by: dependabot[bot] <support@github.com> Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- learning/katas/go/go.mod | 2 +- learning/katas/go/go.sum | 20 ++++++++++++++++---- 2 files changed, 17 insertions(+), 5 deletions(-) diff --git a/learning/katas/go/go.mod b/learning/katas/go/go.mod index eb161611f8f1..3e6d7a207c61 100644 --- a/learning/katas/go/go.mod +++ b/learning/katas/go/go.mod @@ -20,6 +20,6 @@ go 1.14 require ( github.com/apache/beam/sdks/v2 v2.40.0 github.com/google/go-cmp v0.5.8 - golang.org/x/net v0.7.0 // indirect + golang.org/x/net v0.17.0 // indirect google.golang.org/genproto v0.0.0-20220815135757-37a418bb8959 // indirect ) diff --git a/learning/katas/go/go.sum b/learning/katas/go/go.sum index 65e59bbb40aa..a78aca00007f 100644 --- a/learning/katas/go/go.sum +++ b/learning/katas/go/go.sum @@ -781,6 +781,7 @@ golang.org/x/crypto v0.0.0-20201002170205-7f63de1d35b0/go.mod h1:LzIPMQfyMNhhGPh golang.org/x/crypto v0.0.0-20210322153248-0c34fe9e7dc2/go.mod h1:T9bdIzuCu7OtxOm1hfPfRQxPLYneinmdGuTeoZ9dtd4= golang.org/x/crypto v0.0.0-20210513164829-c07d793c2f9a/go.mod h1:P+XmwS30IXTQdn5tA2iutPOUgjI07+tq3H3K9MVA1s8= golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= +golang.org/x/crypto v0.14.0/go.mod h1:MVFd36DqK4CsrnJYDkBA3VC4m2GkXAM0PvzMCn4JQf4= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190306152737-a1d7652674e8/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190510132918-efd6b22b2522/go.mod h1:ZjyILWgesfNpC6sMxTJOJm9Kp84zZh5NQWvqDGG3Qr8= @@ -817,6 +818,7 @@ golang.org/x/mod v0.4.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.4.1/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.4.2/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91VN4djpZkiMVwK6gcyfeH4XE8wZrZaV4= +golang.org/x/mod v0.8.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180906233101-161cd47e91fd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= @@ -874,8 +876,10 @@ golang.org/x/net v0.0.0-20220425223048-2871e0cb64e4/go.mod h1:CfG3xpIq0wQ8r1q4Su golang.org/x/net v0.0.0-20220520000938-2e3eb7b945c2/go.mod h1:CfG3xpIq0wQ8r1q4Su4UZFWDARRcnwPjda9FqA0JpMk= golang.org/x/net v0.0.0-20220607020251-c690dde0001d/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= -golang.org/x/net v0.7.0 h1:rJrUqqhjsgNp7KqAIc25s9pZnjU7TUcSY7HcVZjdn1g= -golang.org/x/net v0.7.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= +golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= +golang.org/x/net v0.10.0/go.mod h1:0qNGK6F8kojg2nk9dLZ2mShWaEBan6FAoqfSigmmuDg= +golang.org/x/net v0.17.0 h1:pVaXccu2ozPjCXewfr1S7xza/zcXTity9cCdXQYSjIM= +golang.org/x/net v0.17.0/go.mod h1:NxSsAGuq816PNPmqtQdLE42eU2Fs7NoRIZrHJAlaCOE= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= @@ -911,6 +915,7 @@ golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20220513210516-0976fa681c29/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20220601150217-0de741cfad7f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.1.0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180909124046-d0be0721c37e/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= @@ -1011,11 +1016,15 @@ golang.org/x/sys v0.0.0-20220502124256-b6088ccd6cba/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20220503163025-988cb79eb6c6/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.5.0 h1:MUK/U/4lj1t1oPg0HfuXDN/Z1wv31ZJ/YcPiGccS4DU= golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.8.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.13.0 h1:Af8nKPmuFypiUBjVoU9V20FiaFXOcuZI21p0ycVYYGE= +golang.org/x/sys v0.13.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/term v0.5.0/go.mod h1:jMB1sMXY+tzblOD4FWmEbocvup2/aLOaQEp7JmGp78k= +golang.org/x/term v0.8.0/go.mod h1:xPskH00ivmX89bAKVGSKKtLOWNx2+17Eiy94tnKShWo= +golang.org/x/term v0.13.0/go.mod h1:LTmsnFJwVN6bCy1rVCoS+qHT1HhALEFxKncY3WNNh4U= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= @@ -1025,8 +1034,10 @@ golang.org/x/text v0.3.4/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.5/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= -golang.org/x/text v0.7.0 h1:4BRB4x83lYWy72KwLD/qYDuTu7q9PjSagHvijDw7cLo= golang.org/x/text v0.7.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= +golang.org/x/text v0.9.0/go.mod h1:e1OnstbJyHTd6l/uOt8jFFHp6TRDWZR/bV3emEE/zU8= +golang.org/x/text v0.13.0 h1:ablQoSUd0tRdKxZewP80B+BaqeKJuVhuRxj/dkrun3k= +golang.org/x/text v0.13.0/go.mod h1:TvPlkZtksWOMsz7fbANvkp4WM8x/WCo/om8BMLbz+aE= golang.org/x/time v0.0.0-20180412165947-fbb02b2291d2/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= @@ -1092,6 +1103,7 @@ golang.org/x/tools v0.1.3/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/tools v0.1.4/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/tools v0.1.5/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/tools v0.1.12/go.mod h1:hNGJHUnrk76NpqgfD5Aqm5Crs+Hm0VOH/i9J2+nxYbc= +golang.org/x/tools v0.6.0/go.mod h1:Xwgl3UAJ/d3gWutnCtw505GrjyAbvKui8lOU390QaIU= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= From 68fec9e1ac61eab8fff9c457f42251b2af1ee9bc Mon Sep 17 00:00:00 2001 From: Joar Wandborg <joar@wandborg.se> Date: Sat, 21 Oct 2023 19:33:57 +0200 Subject: [PATCH 222/435] Include ID and name of tracked thread in lull message (#29007) In order to aid debugging of lulls. --- .../runners/dataflow/worker/DataflowOperationContext.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowOperationContext.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowOperationContext.java index df520ebd3923..b2ab928bc996 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowOperationContext.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowOperationContext.java @@ -264,7 +264,12 @@ protected String getLullMessage(Thread trackedThread, Duration lullDuration) { .append(" for at least ") .append(formatDuration(lullDuration)) .append(" without outputting or completing in state ") - .append(getStateName()); + .append(getStateName()) + .append(" in thread ") + .append(trackedThread.getName()) + .append(" with id ") + .append(trackedThread.getId()); + message.append("\n"); message.append(getStackTraceForLullMessage(trackedThread.getStackTrace())); From f0f8f437d1ae6f0e69d9ae587e7ef8aa33e869b8 Mon Sep 17 00:00:00 2001 From: magicgoody <131876064+magicgoody@users.noreply.github.com> Date: Mon, 23 Oct 2023 03:33:46 +0600 Subject: [PATCH 223/435] Publish test results for PreCommit and PostCommit Python workflows (#29089) * python test results publish * add postcommit python test publish step --- ...rmanceTests_WordCountIT_PythonVersions.yml | 13 ++++++++++--- .../beam_PostCommit_PortableJar_Flink.yml | 13 ++++++++++--- .../beam_PostCommit_PortableJar_Spark.yml | 19 +++++++++++++------ .github/workflows/beam_PostCommit_Python.yml | 19 +++++++++++++------ .../workflows/beam_PostCommit_Python_Arm.yml | 19 +++++++++++++------ ...am_PostCommit_Python_Examples_Dataflow.yml | 19 +++++++++++++------ ...beam_PostCommit_Python_Examples_Direct.yml | 19 +++++++++++++------ .../beam_PostCommit_Python_Examples_Flink.yml | 19 +++++++++++++------ .../beam_PostCommit_Python_Examples_Spark.yml | 19 +++++++++++++------ .../beam_PostCommit_Python_MongoDBIO_IT.yml | 19 +++++++++++++------ ...mit_Python_ValidatesContainer_Dataflow.yml | 19 +++++++++++++------ ...on_ValidatesContainer_Dataflow_With_RC.yml | 19 +++++++++++++------ ...Commit_Python_ValidatesRunner_Dataflow.yml | 19 +++++++++++++------ ...ostCommit_Python_ValidatesRunner_Flink.yml | 19 +++++++++++++------ ...ostCommit_Python_ValidatesRunner_Samza.yml | 19 +++++++++++++------ ...ostCommit_Python_ValidatesRunner_Spark.yml | 19 +++++++++++++------ ...m_PostCommit_Python_Xlang_Gcp_Dataflow.yml | 19 +++++++++++++------ ...eam_PostCommit_Python_Xlang_Gcp_Direct.yml | 19 +++++++++++++------ ...am_PostCommit_Python_Xlang_IO_Dataflow.yml | 19 +++++++++++++------ .../beam_PostCommit_Sickbay_Python.yml | 19 +++++++++++++------ ...eam_PostCommit_TransformService_Direct.yml | 19 +++++++++++++------ ...Commit_XVR_PythonUsingJavaSQL_Dataflow.yml | 19 +++++++++++++------ ...ostCommit_XVR_PythonUsingJava_Dataflow.yml | 19 +++++++++++++------ .github/workflows/beam_PreCommit_Python.yml | 19 +++++++++++++------ .../beam_PreCommit_Python_Coverage.yml | 19 +++++++++++++------ .../beam_PreCommit_Python_Dataframes.yml | 19 +++++++++++++------ .../beam_PreCommit_Python_Examples.yml | 19 +++++++++++++------ .../beam_PreCommit_Python_Integration.yml | 19 +++++++++++++------ .../beam_PreCommit_Python_PVR_Flink.yml | 19 +++++++++++++------ .../beam_PreCommit_Python_Runners.yml | 19 +++++++++++++------ .../beam_PreCommit_Python_Transforms.yml | 19 +++++++++++++------ 31 files changed, 397 insertions(+), 180 deletions(-) diff --git a/.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml b/.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml index 6df9ef42d60a..7903bf033ada 100644 --- a/.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml +++ b/.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml @@ -103,9 +103,16 @@ jobs: --info \ -Ptest=apache_beam/examples/wordcount_it_test.py::WordCountIT::test_wordcount_it \ "-Ptest-pipeline-options=${{ env.beam_PerformanceTests_WordCountIT_PythonVersions_test_arguments_1 }}" - - name: Archive code coverage results + - name: Archive Python Test Results uses: actions/upload-artifact@v3 + if: failure() + with: + name: Python Test Results + path: '**/pytest*.xml' + - name: Publish Python Test Results + uses: EnricoMi/publish-unit-test-result-action@v2 if: always() with: - name: python-code-coverage-report - path: "**/pytest*.xml" \ No newline at end of file + commit: '${{ env.prsha || env.GITHUB_SHA }}' + comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} + files: '**/pytest*.xml' \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_PortableJar_Flink.yml b/.github/workflows/beam_PostCommit_PortableJar_Flink.yml index ef27d1601e2b..0140c16a85cf 100644 --- a/.github/workflows/beam_PostCommit_PortableJar_Flink.yml +++ b/.github/workflows/beam_PostCommit_PortableJar_Flink.yml @@ -79,9 +79,16 @@ jobs: gradle-command: :sdks:python:test-suites:portable:py38:testPipelineJarFlinkRunner arguments: | -PpythonVersion=3.8 \ - - name: Archive code coverage results + - name: Archive Python Test Results uses: actions/upload-artifact@v3 + if: failure() + with: + name: Python Test Results + path: '**/pytest*.xml' + - name: Publish Python Test Results + uses: EnricoMi/publish-unit-test-result-action@v2 if: always() with: - name: python-code-coverage-report - path: "**/pytest*.xml" \ No newline at end of file + commit: '${{ env.prsha || env.GITHUB_SHA }}' + comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} + files: '**/pytest*.xml' \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_PortableJar_Spark.yml b/.github/workflows/beam_PostCommit_PortableJar_Spark.yml index a84384bdb865..243effdf063d 100644 --- a/.github/workflows/beam_PostCommit_PortableJar_Spark.yml +++ b/.github/workflows/beam_PostCommit_PortableJar_Spark.yml @@ -23,12 +23,12 @@ on: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event permissions: actions: write - pull-requests: read - checks: read + pull-requests: write + checks: write contents: read deployments: read id-token: none - issues: read + issues: write discussions: read packages: read pages: read @@ -79,9 +79,16 @@ jobs: gradle-command: :sdks:python:test-suites:portable:py38:testPipelineJarSparkRunner arguments: | -PpythonVersion=3.8 \ - - name: Archive code coverage results + - name: Archive Python Test Results uses: actions/upload-artifact@v3 + if: failure() + with: + name: Python Test Results + path: '**/pytest*.xml' + - name: Publish Python Test Results + uses: EnricoMi/publish-unit-test-result-action@v2 if: always() with: - name: python-code-coverage-report - path: "**/pytest*.xml" \ No newline at end of file + commit: '${{ env.prsha || env.GITHUB_SHA }}' + comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} + files: '**/pytest*.xml' \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Python.yml b/.github/workflows/beam_PostCommit_Python.yml index ffd2979c3a92..e439b50d5313 100644 --- a/.github/workflows/beam_PostCommit_Python.yml +++ b/.github/workflows/beam_PostCommit_Python.yml @@ -30,12 +30,12 @@ concurrency: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event permissions: actions: write - pull-requests: read - checks: read + pull-requests: write + checks: write contents: read deployments: read id-token: none - issues: read + issues: write discussions: read packages: read pages: read @@ -94,9 +94,16 @@ jobs: -PpythonVersion=${{ matrix.python_version }} \ env: CLOUDSDK_CONFIG: ${{ env.KUBELET_GCLOUD_CONFIG_PATH}} - - name: Archive code coverage results + - name: Archive Python Test Results uses: actions/upload-artifact@v3 + if: failure() + with: + name: Python Test Results + path: '**/pytest*.xml' + - name: Publish Python Test Results + uses: EnricoMi/publish-unit-test-result-action@v2 if: always() with: - name: python-code-coverage-report - path: "**/pytest*.xml" \ No newline at end of file + commit: '${{ env.prsha || env.GITHUB_SHA }}' + comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} + files: '**/pytest*.xml' \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Python_Arm.yml b/.github/workflows/beam_PostCommit_Python_Arm.yml index cf41ed19d256..065c917f8687 100644 --- a/.github/workflows/beam_PostCommit_Python_Arm.yml +++ b/.github/workflows/beam_PostCommit_Python_Arm.yml @@ -32,12 +32,12 @@ concurrency: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event permissions: actions: write - pull-requests: read - checks: read + pull-requests: write + checks: write contents: read deployments: read id-token: none - issues: read + issues: write discussions: read packages: read pages: read @@ -110,9 +110,16 @@ jobs: CLOUDSDK_CONFIG: ${{ env.KUBELET_GCLOUD_CONFIG_PATH}} MULTIARCH_TAG: ${{ steps.set_tag.outputs.TAG }} USER: github-actions - - name: Archive code coverage results + - name: Archive Python Test Results uses: actions/upload-artifact@v3 + if: failure() + with: + name: Python Test Results + path: '**/pytest*.xml' + - name: Publish Python Test Results + uses: EnricoMi/publish-unit-test-result-action@v2 if: always() with: - name: python-code-coverage-report - path: "**/pytest*.xml" + commit: '${{ env.prsha || env.GITHUB_SHA }}' + comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} + files: '**/pytest*.xml' \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Python_Examples_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_Examples_Dataflow.yml index 61fbb1f80e71..b659b2522b19 100644 --- a/.github/workflows/beam_PostCommit_Python_Examples_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_Examples_Dataflow.yml @@ -23,12 +23,12 @@ on: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event permissions: actions: write - pull-requests: read - checks: read + pull-requests: write + checks: write contents: read deployments: read id-token: none - issues: read + issues: write discussions: read packages: read pages: read @@ -79,9 +79,16 @@ jobs: arguments: | -PuseWheelDistribution \ -PpythonVersion=3.11 \ - - name: Archive code coverage results + - name: Archive Python Test Results uses: actions/upload-artifact@v3 + if: failure() + with: + name: Python Test Results + path: '**/pytest*.xml' + - name: Publish Python Test Results + uses: EnricoMi/publish-unit-test-result-action@v2 if: always() with: - name: python-code-coverage-report - path: "**/pytest*.xml" \ No newline at end of file + commit: '${{ env.prsha || env.GITHUB_SHA }}' + comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} + files: '**/pytest*.xml' \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Python_Examples_Direct.yml b/.github/workflows/beam_PostCommit_Python_Examples_Direct.yml index c8855b2b1274..b7344be286e0 100644 --- a/.github/workflows/beam_PostCommit_Python_Examples_Direct.yml +++ b/.github/workflows/beam_PostCommit_Python_Examples_Direct.yml @@ -23,12 +23,12 @@ on: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event permissions: actions: write - pull-requests: read - checks: read + pull-requests: write + checks: write contents: read deployments: read id-token: none - issues: read + issues: write discussions: read packages: read pages: read @@ -86,9 +86,16 @@ jobs: gradle-command: :sdks:python:test-suites:direct:py${{steps.set_py_ver_clean.outputs.py_ver_clean}}:examples arguments: | -PpythonVersion=${{ matrix.python_version }} \ - - name: Archive code coverage results + - name: Archive Python Test Results uses: actions/upload-artifact@v3 + if: failure() + with: + name: Python Test Results + path: '**/pytest*.xml' + - name: Publish Python Test Results + uses: EnricoMi/publish-unit-test-result-action@v2 if: always() with: - name: python-code-coverage-report - path: "**/pytest*.xml" \ No newline at end of file + commit: '${{ env.prsha || env.GITHUB_SHA }}' + comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} + files: '**/pytest*.xml' \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Python_Examples_Flink.yml b/.github/workflows/beam_PostCommit_Python_Examples_Flink.yml index 2e4db6192501..af349c0ef368 100644 --- a/.github/workflows/beam_PostCommit_Python_Examples_Flink.yml +++ b/.github/workflows/beam_PostCommit_Python_Examples_Flink.yml @@ -23,12 +23,12 @@ on: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event permissions: actions: write - pull-requests: read - checks: read + pull-requests: write + checks: write contents: read deployments: read id-token: none - issues: read + issues: write discussions: read packages: read pages: read @@ -86,9 +86,16 @@ jobs: gradle-command: :sdks:python:test-suites:portable:py${{steps.set_py_ver_clean.outputs.py_ver_clean}}:flinkExamples arguments: | -PpythonVersion=${{ matrix.python_version }} \ - - name: Archive code coverage results + - name: Archive Python Test Results uses: actions/upload-artifact@v3 + if: failure() + with: + name: Python Test Results + path: '**/pytest*.xml' + - name: Publish Python Test Results + uses: EnricoMi/publish-unit-test-result-action@v2 if: always() with: - name: python-code-coverage-report - path: "**/pytest*.xml" \ No newline at end of file + commit: '${{ env.prsha || env.GITHUB_SHA }}' + comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} + files: '**/pytest*.xml' \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Python_Examples_Spark.yml b/.github/workflows/beam_PostCommit_Python_Examples_Spark.yml index 498bc4f747a4..66a240eec23b 100644 --- a/.github/workflows/beam_PostCommit_Python_Examples_Spark.yml +++ b/.github/workflows/beam_PostCommit_Python_Examples_Spark.yml @@ -23,12 +23,12 @@ on: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event permissions: actions: write - pull-requests: read - checks: read + pull-requests: write + checks: write contents: read deployments: read id-token: none - issues: read + issues: write discussions: read packages: read pages: read @@ -86,9 +86,16 @@ jobs: gradle-command: :sdks:python:test-suites:portable:py${{steps.set_py_ver_clean.outputs.py_ver_clean}}:sparkExamples arguments: | -PpythonVersion=${{ matrix.python_version }} \ - - name: Archive code coverage results + - name: Archive Python Test Results uses: actions/upload-artifact@v3 + if: failure() + with: + name: Python Test Results + path: '**/pytest*.xml' + - name: Publish Python Test Results + uses: EnricoMi/publish-unit-test-result-action@v2 if: always() with: - name: python-code-coverage-report - path: "**/pytest*.xml" \ No newline at end of file + commit: '${{ env.prsha || env.GITHUB_SHA }}' + comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} + files: '**/pytest*.xml' \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml b/.github/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml index d8e9934609fa..e406a58d01a5 100644 --- a/.github/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml +++ b/.github/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml @@ -23,12 +23,12 @@ on: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event permissions: actions: write - pull-requests: read - checks: read + pull-requests: write + checks: write contents: read deployments: read id-token: none - issues: read + issues: write discussions: read packages: read pages: read @@ -78,9 +78,16 @@ jobs: gradle-command: :sdks:python:test-suites:direct:py311:mongodbioIT arguments: | -PpythonVersion=3.11 \ - - name: Archive code coverage results + - name: Archive Python Test Results uses: actions/upload-artifact@v3 + if: failure() + with: + name: Python Test Results + path: '**/pytest*.xml' + - name: Publish Python Test Results + uses: EnricoMi/publish-unit-test-result-action@v2 if: always() with: - name: python-code-coverage-report - path: "**/pytest*.xml" \ No newline at end of file + commit: '${{ env.prsha || env.GITHUB_SHA }}' + comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} + files: '**/pytest*.xml' \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml index 51b22fa66a0e..e572c5c9198f 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml @@ -23,12 +23,12 @@ on: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event permissions: actions: write - pull-requests: read - checks: read + pull-requests: write + checks: write contents: read deployments: read id-token: none - issues: read + issues: write discussions: read packages: read pages: read @@ -90,9 +90,16 @@ jobs: gradle-command: :sdks:python:test-suites:dataflow:py${{steps.set_py_ver_clean.outputs.py_ver_clean}}:validatesContainer arguments: | -PpythonVersion=${{ matrix.python_version }} \ - - name: Archive code coverage results + - name: Archive Python Test Results uses: actions/upload-artifact@v3 + if: failure() + with: + name: Python Test Results + path: '**/pytest*.xml' + - name: Publish Python Test Results + uses: EnricoMi/publish-unit-test-result-action@v2 if: always() with: - name: python-code-coverage-report - path: "**/pytest*.xml" \ No newline at end of file + commit: '${{ env.prsha || env.GITHUB_SHA }}' + comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} + files: '**/pytest*.xml' \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml b/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml index 33fa6b53e23f..cfc5db3ff8e7 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml @@ -23,12 +23,12 @@ on: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event permissions: actions: write - pull-requests: read - checks: read + pull-requests: write + checks: write contents: read deployments: read id-token: none - issues: read + issues: write discussions: read packages: read pages: read @@ -91,9 +91,16 @@ jobs: arguments: | -PtestRCDependencies=true \ -PpythonVersion=${{ matrix.python_version }} \ - - name: Archive code coverage results + - name: Archive Python Test Results uses: actions/upload-artifact@v3 + if: failure() + with: + name: Python Test Results + path: '**/pytest*.xml' + - name: Publish Python Test Results + uses: EnricoMi/publish-unit-test-result-action@v2 if: always() with: - name: python-code-coverage-report - path: "**/pytest*.xml" + commit: '${{ env.prsha || env.GITHUB_SHA }}' + comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} + files: '**/pytest*.xml' \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml index b01ea81f9ac5..79b2269fa26c 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml @@ -23,12 +23,12 @@ on: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event permissions: actions: write - pull-requests: read - checks: read + pull-requests: write + checks: write contents: read deployments: read id-token: none - issues: read + issues: write discussions: read packages: read pages: read @@ -94,9 +94,16 @@ jobs: arguments: | -PuseWheelDistribution \ -PpythonVersion=${{ matrix.python_version }} \ - - name: Archive code coverage results + - name: Archive Python Test Results uses: actions/upload-artifact@v3 + if: failure() + with: + name: Python Test Results + path: '**/pytest*.xml' + - name: Publish Python Test Results + uses: EnricoMi/publish-unit-test-result-action@v2 if: always() with: - name: python-code-coverage-report - path: "**/pytest*.xml" \ No newline at end of file + commit: '${{ env.prsha || env.GITHUB_SHA }}' + comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} + files: '**/pytest*.xml' \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml index 79a7550755d1..367a5da28fd1 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml @@ -23,12 +23,12 @@ on: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event permissions: actions: write - pull-requests: read - checks: read + pull-requests: write + checks: write contents: read deployments: read id-token: none - issues: read + issues: write discussions: read packages: read pages: read @@ -88,9 +88,16 @@ jobs: gradle-command: :sdks:python:test-suites:portable:py${{steps.set_py_ver_clean.outputs.py_ver_clean}}:flinkValidatesRunner arguments: | -PpythonVersion=${{ matrix.python_version }} \ - - name: Archive code coverage results + - name: Archive Python Test Results uses: actions/upload-artifact@v3 + if: failure() + with: + name: Python Test Results + path: '**/pytest*.xml' + - name: Publish Python Test Results + uses: EnricoMi/publish-unit-test-result-action@v2 if: always() with: - name: python-code-coverage-report - path: "**/pytest*.xml" \ No newline at end of file + commit: '${{ env.prsha || env.GITHUB_SHA }}' + comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} + files: '**/pytest*.xml' \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml index 300cee6c3520..9fff3935639d 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml @@ -23,12 +23,12 @@ on: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event permissions: actions: write - pull-requests: read - checks: read + pull-requests: write + checks: write contents: read deployments: read id-token: none - issues: read + issues: write discussions: read packages: read pages: read @@ -86,9 +86,16 @@ jobs: gradle-command: :sdks:python:test-suites:portable:py${{steps.set_py_ver_clean.outputs.py_ver_clean}}:samzaValidatesRunner arguments: | -PpythonVersion=${{ matrix.python_version }} \ - - name: Archive code coverage results + - name: Archive Python Test Results uses: actions/upload-artifact@v3 + if: failure() + with: + name: Python Test Results + path: '**/pytest*.xml' + - name: Publish Python Test Results + uses: EnricoMi/publish-unit-test-result-action@v2 if: always() with: - name: python-code-coverage-report - path: "**/pytest*.xml" \ No newline at end of file + commit: '${{ env.prsha || env.GITHUB_SHA }}' + comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} + files: '**/pytest*.xml' \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml index c1f55590b8ed..67ec4b5af84d 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml @@ -23,12 +23,12 @@ on: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event permissions: actions: write - pull-requests: read - checks: read + pull-requests: write + checks: write contents: read deployments: read id-token: none - issues: read + issues: write discussions: read packages: read pages: read @@ -86,9 +86,16 @@ jobs: gradle-command: :sdks:python:test-suites:portable:py${{steps.set_py_ver_clean.outputs.py_ver_clean}}:sparkValidatesRunner arguments: | -PpythonVersion=${{ matrix.python_version }} \ - - name: Archive code coverage results + - name: Archive Python Test Results uses: actions/upload-artifact@v3 + if: failure() + with: + name: Python Test Results + path: '**/pytest*.xml' + - name: Publish Python Test Results + uses: EnricoMi/publish-unit-test-result-action@v2 if: always() with: - name: python-code-coverage-report - path: "**/pytest*.xml" \ No newline at end of file + commit: '${{ env.prsha || env.GITHUB_SHA }}' + comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} + files: '**/pytest*.xml' \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml index e4023479c32f..291de49c3bbb 100644 --- a/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml @@ -23,12 +23,12 @@ on: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event permissions: actions: write - pull-requests: read - checks: read + pull-requests: write + checks: write contents: read deployments: read id-token: none - issues: read + issues: write discussions: read packages: read pages: read @@ -78,9 +78,16 @@ jobs: with: gradle-command: :sdks:python:test-suites:dataflow:gcpCrossLanguagePostCommit arguments: -PuseWheelDistribution - - name: Archive code coverage results + - name: Archive Python Test Results uses: actions/upload-artifact@v3 + if: failure() + with: + name: Python Test Results + path: '**/pytest*.xml' + - name: Publish Python Test Results + uses: EnricoMi/publish-unit-test-result-action@v2 if: always() with: - name: python-code-coverage-report - path: "**/pytest*.xml" \ No newline at end of file + commit: '${{ env.prsha || env.GITHUB_SHA }}' + comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} + files: '**/pytest*.xml' \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml b/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml index ec734641b6b6..cf061f47e677 100644 --- a/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml +++ b/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml @@ -23,12 +23,12 @@ on: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event permissions: actions: write - pull-requests: read - checks: read + pull-requests: write + checks: write contents: read deployments: read id-token: none - issues: read + issues: write discussions: read packages: read pages: read @@ -77,9 +77,16 @@ jobs: uses: ./.github/actions/gradle-command-self-hosted-action with: gradle-command: :sdks:python:test-suites:direct:gcpCrossLanguagePostCommit - - name: Archive code coverage results + - name: Archive Python Test Results uses: actions/upload-artifact@v3 + if: failure() + with: + name: Python Test Results + path: '**/pytest*.xml' + - name: Publish Python Test Results + uses: EnricoMi/publish-unit-test-result-action@v2 if: always() with: - name: python-code-coverage-report - path: "**/pytest*.xml" \ No newline at end of file + commit: '${{ env.prsha || env.GITHUB_SHA }}' + comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} + files: '**/pytest*.xml' \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml index 9a4a30098764..069b9965517d 100644 --- a/.github/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml @@ -23,12 +23,12 @@ on: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event permissions: actions: write - pull-requests: read - checks: read + pull-requests: write + checks: write contents: read deployments: read id-token: none - issues: read + issues: write discussions: read packages: read pages: read @@ -80,9 +80,16 @@ jobs: arguments: | -PuseWheelDistribution \ -PkafkaBootstrapServer=10.128.0.40:9094,10.128.0.28:9094,10.128.0.165:9094 \ - - name: Archive code coverage results + - name: Archive Python Test Results uses: actions/upload-artifact@v3 + if: failure() + with: + name: Python Test Results + path: '**/pytest*.xml' + - name: Publish Python Test Results + uses: EnricoMi/publish-unit-test-result-action@v2 if: always() with: - name: archiveJunit - path: "**/pytest*.xml" \ No newline at end of file + commit: '${{ env.prsha || env.GITHUB_SHA }}' + comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} + files: '**/pytest*.xml' \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Sickbay_Python.yml b/.github/workflows/beam_PostCommit_Sickbay_Python.yml index 5e8b717d0ef9..7516a6003a2e 100644 --- a/.github/workflows/beam_PostCommit_Sickbay_Python.yml +++ b/.github/workflows/beam_PostCommit_Sickbay_Python.yml @@ -30,12 +30,12 @@ concurrency: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event permissions: actions: write - pull-requests: read - checks: read + pull-requests: write + checks: write contents: read deployments: read id-token: none - issues: read + issues: write discussions: read packages: read pages: read @@ -89,9 +89,16 @@ jobs: gradle-command: :sdks:python:test-suites:dataflow:py${{steps.set_py_ver_clean.outputs.py_ver_clean}}:postCommitSickbay arguments: | -PpythonVersion=${{ matrix.python_version }} \ - - name: Archive code coverage results + - name: Archive Python Test Results uses: actions/upload-artifact@v3 + if: failure() + with: + name: Python Test Results + path: '**/pytest*.xml' + - name: Publish Python Test Results + uses: EnricoMi/publish-unit-test-result-action@v2 if: always() with: - name: python-code-coverage-report - path: "**/pytest*.xml" \ No newline at end of file + commit: '${{ env.prsha || env.GITHUB_SHA }}' + comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} + files: '**/pytest*.xml' \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_TransformService_Direct.yml b/.github/workflows/beam_PostCommit_TransformService_Direct.yml index 4c9bb3aa9302..327b50f7840c 100644 --- a/.github/workflows/beam_PostCommit_TransformService_Direct.yml +++ b/.github/workflows/beam_PostCommit_TransformService_Direct.yml @@ -23,12 +23,12 @@ on: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event permissions: actions: write - pull-requests: read - checks: read + pull-requests: write + checks: write contents: read deployments: read id-token: none - issues: read + issues: write discussions: read packages: read pages: read @@ -88,9 +88,16 @@ jobs: -Pjava11Home=$JAVA_HOME_11_X64 \ -PuseWheelDistribution \ -PpythonVersion=${{ matrix.python_version }} \ - - name: Archive code coverage results + - name: Archive Python Test Results uses: actions/upload-artifact@v3 + if: failure() + with: + name: Python Test Results + path: '**/pytest*.xml' + - name: Publish Python Test Results + uses: EnricoMi/publish-unit-test-result-action@v2 if: always() with: - name: python-code-coverage-report - path: "**/pytest*.xml" \ No newline at end of file + commit: '${{ env.prsha || env.GITHUB_SHA }}' + comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} + files: '**/pytest*.xml' \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml b/.github/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml index be4fcb9cec5a..f000d1956f01 100644 --- a/.github/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml @@ -23,12 +23,12 @@ on: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event permissions: actions: write - pull-requests: read - checks: read + pull-requests: write + checks: write contents: read deployments: read id-token: none - issues: read + issues: write discussions: read packages: read pages: read @@ -79,9 +79,16 @@ jobs: gradle-command: :runners:google-cloud-dataflow-java:validatesCrossLanguageRunnerPythonUsingSql arguments: | -PpythonVersion=3.11 \ - - name: Archive code coverage results + - name: Archive Python Test Results uses: actions/upload-artifact@v3 + if: failure() + with: + name: Python Test Results + path: '**/pytest*.xml' + - name: Publish Python Test Results + uses: EnricoMi/publish-unit-test-result-action@v2 if: always() with: - name: python-code-coverage-report - path: "**/pytest*.xml" \ No newline at end of file + commit: '${{ env.prsha || env.GITHUB_SHA }}' + comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} + files: '**/pytest*.xml' \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml b/.github/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml index 0318f732d978..fdc3798999d9 100644 --- a/.github/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml @@ -23,12 +23,12 @@ on: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event permissions: actions: write - pull-requests: read - checks: read + pull-requests: write + checks: write contents: read deployments: read id-token: none - issues: read + issues: write discussions: read packages: read pages: read @@ -82,9 +82,16 @@ jobs: gradle-command: :runners:google-cloud-dataflow-java:validatesCrossLanguageRunnerPythonUsingJava arguments: | -PpythonVersion=${{ matrix.python_version }} \ - - name: Archive code coverage results + - name: Archive Python Test Results uses: actions/upload-artifact@v3 + if: failure() + with: + name: Python Test Results + path: '**/pytest*.xml' + - name: Publish Python Test Results + uses: EnricoMi/publish-unit-test-result-action@v2 if: always() with: - name: python-code-coverage-report - path: "**/pytest*.xml" \ No newline at end of file + commit: '${{ env.prsha || env.GITHUB_SHA }}' + comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} + files: '**/pytest*.xml' \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Python.yml b/.github/workflows/beam_PreCommit_Python.yml index d507a8e3f072..53cc2af4a07d 100644 --- a/.github/workflows/beam_PreCommit_Python.yml +++ b/.github/workflows/beam_PreCommit_Python.yml @@ -31,12 +31,12 @@ on: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event permissions: actions: write - pull-requests: read - checks: read + pull-requests: write + checks: write contents: read deployments: read id-token: none - issues: read + issues: write discussions: read packages: read pages: read @@ -98,9 +98,16 @@ jobs: -Pposargs="--ignore=apache_beam/dataframe/ --ignore=apache_beam/examples/ --ignore=apache_beam/runners/ --ignore=apache_beam/transforms/" \ -PpythonVersion=${{ matrix.python_version }} \ -PuseWheelDistribution - - name: Archive code coverage results + - name: Archive Python Test Results uses: actions/upload-artifact@v3 + if: failure() + with: + name: Python Test Results + path: '**/pytest*.xml' + - name: Publish Python Test Results + uses: EnricoMi/publish-unit-test-result-action@v2 if: always() with: - name: python-code-coverage-report - path: "**/pytest*.xml" \ No newline at end of file + commit: '${{ env.prsha || env.GITHUB_SHA }}' + comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} + files: '**/pytest*.xml' \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Python_Coverage.yml b/.github/workflows/beam_PreCommit_Python_Coverage.yml index 2127f08ea591..e04f1006c2af 100644 --- a/.github/workflows/beam_PreCommit_Python_Coverage.yml +++ b/.github/workflows/beam_PreCommit_Python_Coverage.yml @@ -31,12 +31,12 @@ on: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event permissions: actions: write - pull-requests: read - checks: read + pull-requests: write + checks: write contents: read deployments: read id-token: none - issues: read + issues: write discussions: read packages: read pages: read @@ -88,9 +88,16 @@ jobs: gradle-command: :sdks:python:test-suites:tox:py38:preCommitPyCoverage arguments: | -PuseWheelDistribution - - name: Archive code coverage results + - name: Archive Python Test Results uses: actions/upload-artifact@v3 + if: failure() + with: + name: Python Test Results + path: '**/pytest*.xml' + - name: Publish Python Test Results + uses: EnricoMi/publish-unit-test-result-action@v2 if: always() with: - name: python-code-coverage-report - path: "**/pytest*.xml" \ No newline at end of file + commit: '${{ env.prsha || env.GITHUB_SHA }}' + comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} + files: '**/pytest*.xml' \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Python_Dataframes.yml b/.github/workflows/beam_PreCommit_Python_Dataframes.yml index 9211e2527c6a..8024f8d53909 100644 --- a/.github/workflows/beam_PreCommit_Python_Dataframes.yml +++ b/.github/workflows/beam_PreCommit_Python_Dataframes.yml @@ -31,12 +31,12 @@ on: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event permissions: actions: write - pull-requests: read - checks: read + pull-requests: write + checks: write contents: read deployments: read id-token: none - issues: read + issues: write discussions: read packages: read pages: read @@ -98,9 +98,16 @@ jobs: -Pposargs=apache_beam/dataframe/ \ -PpythonVersion=${{ matrix.python_version }} \ -PuseWheelDistribution - - name: Archive code coverage results + - name: Archive Python Test Results uses: actions/upload-artifact@v3 + if: failure() + with: + name: Python Test Results + path: '**/pytest*.xml' + - name: Publish Python Test Results + uses: EnricoMi/publish-unit-test-result-action@v2 if: always() with: - name: python-code-coverage-report - path: "**/pytest*.xml" \ No newline at end of file + commit: '${{ env.prsha || env.GITHUB_SHA }}' + comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} + files: '**/pytest*.xml' \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Python_Examples.yml b/.github/workflows/beam_PreCommit_Python_Examples.yml index 164734f0c8e2..19839118dbff 100644 --- a/.github/workflows/beam_PreCommit_Python_Examples.yml +++ b/.github/workflows/beam_PreCommit_Python_Examples.yml @@ -31,12 +31,12 @@ on: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event permissions: actions: write - pull-requests: read - checks: read + pull-requests: write + checks: write contents: read deployments: read id-token: none - issues: read + issues: write discussions: read packages: read pages: read @@ -98,9 +98,16 @@ jobs: -Pposargs=apache_beam/examples/ \ -PpythonVersion=${{ matrix.python_version }} \ -PuseWheelDistribution - - name: Archive code coverage results + - name: Archive Python Test Results uses: actions/upload-artifact@v3 + if: failure() + with: + name: Python Test Results + path: '**/pytest*.xml' + - name: Publish Python Test Results + uses: EnricoMi/publish-unit-test-result-action@v2 if: always() with: - name: python-code-coverage-report - path: "**/pytest*.xml" \ No newline at end of file + commit: '${{ env.prsha || env.GITHUB_SHA }}' + comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} + files: '**/pytest*.xml' \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Python_Integration.yml b/.github/workflows/beam_PreCommit_Python_Integration.yml index f9c32a12bf5a..c8f22339565d 100644 --- a/.github/workflows/beam_PreCommit_Python_Integration.yml +++ b/.github/workflows/beam_PreCommit_Python_Integration.yml @@ -31,12 +31,12 @@ on: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event permissions: actions: write - pull-requests: read - checks: read + pull-requests: write + checks: write contents: read deployments: read id-token: none - issues: read + issues: write discussions: read packages: read pages: read @@ -104,9 +104,16 @@ jobs: arguments: | -PuseWheelDistribution \ -PpythonVersion=${{ matrix.python_version }} \ - - name: Archive code coverage results + - name: Archive Python Test Results uses: actions/upload-artifact@v3 + if: failure() + with: + name: Python Test Results + path: '**/pytest*.xml' + - name: Publish Python Test Results + uses: EnricoMi/publish-unit-test-result-action@v2 if: always() with: - name: python-code-coverage-report - path: "**/pytest*.xml" + commit: '${{ env.prsha || env.GITHUB_SHA }}' + comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} + files: '**/pytest*.xml' \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Python_PVR_Flink.yml b/.github/workflows/beam_PreCommit_Python_PVR_Flink.yml index 06f18651f117..f5457122e691 100644 --- a/.github/workflows/beam_PreCommit_Python_PVR_Flink.yml +++ b/.github/workflows/beam_PreCommit_Python_PVR_Flink.yml @@ -54,12 +54,12 @@ on: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event permissions: actions: write - pull-requests: read - checks: read + pull-requests: write + checks: write contents: read deployments: read id-token: none - issues: read + issues: write discussions: read packages: read pages: read @@ -112,9 +112,16 @@ jobs: gradle-command: :sdks:python:test-suites:portable:py311:flinkValidatesRunner arguments: | -PpythonVersion=3.11 \ - - name: Archive code coverage results + - name: Archive Python Test Results uses: actions/upload-artifact@v3 - if: always() + if: failure() with: - name: python-code-coverage-report + name: Python Test Results path: '**/pytest*.xml' + - name: Publish Python Test Results + uses: EnricoMi/publish-unit-test-result-action@v2 + if: always() + with: + commit: '${{ env.prsha || env.GITHUB_SHA }}' + comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} + files: '**/pytest*.xml' \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Python_Runners.yml b/.github/workflows/beam_PreCommit_Python_Runners.yml index d1702905f5d7..d622f9e6bbbd 100644 --- a/.github/workflows/beam_PreCommit_Python_Runners.yml +++ b/.github/workflows/beam_PreCommit_Python_Runners.yml @@ -31,12 +31,12 @@ on: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event permissions: actions: write - pull-requests: read - checks: read + pull-requests: write + checks: write contents: read deployments: read id-token: none - issues: read + issues: write discussions: read packages: read pages: read @@ -98,9 +98,16 @@ jobs: -Pposargs=apache_beam/runners/ \ -PpythonVersion=${{ matrix.python_version }} \ -PuseWheelDistribution - - name: Archive code coverage results + - name: Archive Python Test Results uses: actions/upload-artifact@v3 + if: failure() + with: + name: Python Test Results + path: '**/pytest*.xml' + - name: Publish Python Test Results + uses: EnricoMi/publish-unit-test-result-action@v2 if: always() with: - name: python-code-coverage-report - path: "**/pytest*.xml" \ No newline at end of file + commit: '${{ env.prsha || env.GITHUB_SHA }}' + comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} + files: '**/pytest*.xml' \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Python_Transforms.yml b/.github/workflows/beam_PreCommit_Python_Transforms.yml index 7422d7dc0f44..d034296038ea 100644 --- a/.github/workflows/beam_PreCommit_Python_Transforms.yml +++ b/.github/workflows/beam_PreCommit_Python_Transforms.yml @@ -31,12 +31,12 @@ on: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event permissions: actions: write - pull-requests: read - checks: read + pull-requests: write + checks: write contents: read deployments: read id-token: none - issues: read + issues: write discussions: read packages: read pages: read @@ -98,9 +98,16 @@ jobs: -Pposargs=apache_beam/transforms/ \ -PpythonVersion=${{ matrix.python_version }} \ -PuseWheelDistribution - - name: Archive code coverage results + - name: Archive Python Test Results uses: actions/upload-artifact@v3 + if: failure() + with: + name: Python Test Results + path: '**/pytest*.xml' + - name: Publish Python Test Results + uses: EnricoMi/publish-unit-test-result-action@v2 if: always() with: - name: python-code-coverage-report - path: "**/pytest*.xml" \ No newline at end of file + commit: '${{ env.prsha || env.GITHUB_SHA }}' + comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} + files: '**/pytest*.xml' \ No newline at end of file From 4b2aab56f0f225bbd92e4ced9f157c840782ef1d Mon Sep 17 00:00:00 2001 From: illoise <39356499+joy91227@users.noreply.github.com> Date: Sun, 22 Oct 2023 17:35:14 -0400 Subject: [PATCH 224/435] Update beam-sql-with-notebooks.md with working API link (#29095) The provided API to fetch covid data from has changed to: https://api.covidtracking.com/v1/states/current.json --- website/www/site/content/en/blog/beam-sql-with-notebooks.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/website/www/site/content/en/blog/beam-sql-with-notebooks.md b/website/www/site/content/en/blog/beam-sql-with-notebooks.md index 4f7c428613a1..d7d80f4db7f5 100644 --- a/website/www/site/content/en/blog/beam-sql-with-notebooks.md +++ b/website/www/site/content/en/blog/beam-sql-with-notebooks.md @@ -420,7 +420,7 @@ import json import requests # The covidtracking project has stopped collecting new data, current data ends on 2021-03-07 -json_current='https://covidtracking.com/api/v1/states/current.json' +json_current='https://api.covidtracking.com/v1/states/current.json' def get_json_data(url): with requests.Session() as session: From 1a27d2cee0c5ab226a50cc2dc53da4ad18cbde12 Mon Sep 17 00:00:00 2001 From: Kanishk Karanawat <kkaranawat@twitter.com> Date: Sat, 21 Oct 2023 15:20:32 -0700 Subject: [PATCH 225/435] flush buffer during drain operation for requiresStableInput operator (#28554) --- .../runners/flink/FlinkPipelineOptions.java | 8 ++ .../wrappers/streaming/DoFnOperator.java | 14 +++ .../wrappers/streaming/DoFnOperatorTest.java | 92 +++++++++++++++++++ .../flink_java_pipeline_options.html | 5 + .../flink_python_pipeline_options.html | 5 + 5 files changed, 124 insertions(+) diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java index 1e01514fe8b6..650768c7b44b 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java @@ -320,6 +320,14 @@ public interface FlinkPipelineOptions void setFileInputSplitMaxSizeMB(Long fileInputSplitMaxSizeMB); + @Description( + "Allow drain operation for flink pipelines that contain RequiresStableInput operator. Note that at time of draining," + + "the RequiresStableInput contract might be violated if there any processing related failures in the DoFn operator.") + @Default.Boolean(false) + Boolean getEnableStableInputDrain(); + + void setEnableStableInputDrain(Boolean enableStableInputDrain); + static FlinkPipelineOptions defaults() { return PipelineOptionsFactory.as(FlinkPipelineOptions.class); } diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java index 726ffb229188..0a9731da8b56 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java @@ -199,6 +199,12 @@ public class DoFnOperator<InputT, OutputT> /** If true, we must process elements only after a checkpoint is finished. */ final boolean requiresStableInput; + /** + * If both requiresStableInput and this parameter are true, we must flush the buffer during drain + * operation. + */ + final boolean enableStableInputDrain; + final int numConcurrentCheckpoints; private final boolean usesOnWindowExpiration; @@ -323,6 +329,8 @@ public DoFnOperator( + Math.max(0, flinkOptions.getMinPauseBetweenCheckpoints())); } + this.enableStableInputDrain = flinkOptions.getEnableStableInputDrain(); + this.numConcurrentCheckpoints = flinkOptions.getNumConcurrentCheckpoints(); this.finishBundleBeforeCheckpointing = flinkOptions.getFinishBundleBeforeCheckpointing(); @@ -626,6 +634,12 @@ void flushData() throws Exception { while (bundleStarted) { invokeFinishBundle(); } + if (requiresStableInput && enableStableInputDrain) { + // Flush any buffered events here before draining the pipeline. Note that this is best-effort + // and requiresStableInput contract might be violated in cases where buffer processing fails. + bufferingDoFnRunner.checkpointCompleted(Long.MAX_VALUE); + updateOutputWatermark(); + } if (currentOutputWatermark < Long.MAX_VALUE) { throw new RuntimeException( String.format( diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperatorTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperatorTest.java index 722d32b309c4..17cc16cc76e0 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperatorTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperatorTest.java @@ -2015,6 +2015,98 @@ public void finishBundle(FinishBundleContext context) { WindowedValue.valueInGlobalWindow("finishBundle"))); } + @Test + public void testExactlyOnceBufferingFlushDuringDrain() throws Exception { + FlinkPipelineOptions options = FlinkPipelineOptions.defaults(); + options.setMaxBundleSize(2L); + options.setCheckpointingInterval(1L); + options.setEnableStableInputDrain(true); + + TupleTag<String> outputTag = new TupleTag<>("main-output"); + WindowedValue.ValueOnlyWindowedValueCoder<String> windowedValueCoder = + WindowedValue.getValueOnlyCoder(StringUtf8Coder.of()); + + numStartBundleCalled = 0; + DoFn<String, String> doFn = + new DoFn<String, String>() { + @StartBundle + public void startBundle(StartBundleContext context) { + numStartBundleCalled += 1; + } + + @ProcessElement + // Use RequiresStableInput to force buffering elements + @RequiresStableInput + public void processElement(ProcessContext context) { + context.output(context.element()); + } + + @FinishBundle + public void finishBundle(FinishBundleContext context) { + context.output( + "finishBundle", BoundedWindow.TIMESTAMP_MIN_VALUE, GlobalWindow.INSTANCE); + } + }; + + DoFnOperator.MultiOutputOutputManagerFactory<String> outputManagerFactory = + new DoFnOperator.MultiOutputOutputManagerFactory<>( + outputTag, + WindowedValue.getFullCoder(StringUtf8Coder.of(), GlobalWindow.Coder.INSTANCE), + new SerializablePipelineOptions(options)); + + Supplier<DoFnOperator<String, String>> doFnOperatorSupplier = + () -> + new DoFnOperator<>( + doFn, + "stepName", + windowedValueCoder, + Collections.emptyMap(), + outputTag, + Collections.emptyList(), + outputManagerFactory, + WindowingStrategy.globalDefault(), + new HashMap<>(), /* side-input mapping */ + Collections.emptyList(), /* side inputs */ + options, + null, + null, + DoFnSchemaInformation.create(), + Collections.emptyMap()); + + DoFnOperator<String, String> doFnOperator = doFnOperatorSupplier.get(); + OneInputStreamOperatorTestHarness<WindowedValue<String>, WindowedValue<String>> testHarness = + new OneInputStreamOperatorTestHarness<>(doFnOperator); + + testHarness.open(); + + testHarness.processElement(new StreamRecord<>(WindowedValue.valueInGlobalWindow("a"))); + testHarness.processElement(new StreamRecord<>(WindowedValue.valueInGlobalWindow("b"))); + + assertThat(Iterables.size(testHarness.getOutput()), is(0)); + assertThat(numStartBundleCalled, is(0)); + + // Simulate pipeline drain scenario + OperatorSubtaskState backup = testHarness.snapshot(0, 0); + doFnOperator.flushData(); + + assertThat(numStartBundleCalled, is(1)); + assertThat( + stripStreamRecordFromWindowedValue(testHarness.getOutput()), + contains( + WindowedValue.valueInGlobalWindow("a"), + WindowedValue.valueInGlobalWindow("b"), + WindowedValue.valueInGlobalWindow("finishBundle"))); + + doFnOperator = doFnOperatorSupplier.get(); + testHarness = new OneInputStreamOperatorTestHarness<>(doFnOperator); + testHarness.open(); + + doFnOperator.notifyCheckpointComplete(0L); + + assertThat(numStartBundleCalled, is(1)); + assertThat(stripStreamRecordFromWindowedValue(testHarness.getOutput()), emptyIterable()); + } + @Test public void testExactlyOnceBufferingKeyed() throws Exception { FlinkPipelineOptions options = FlinkPipelineOptions.defaults(); diff --git a/website/www/site/layouts/shortcodes/flink_java_pipeline_options.html b/website/www/site/layouts/shortcodes/flink_java_pipeline_options.html index ba8b597aaeeb..87d69ee60fe3 100644 --- a/website/www/site/layouts/shortcodes/flink_java_pipeline_options.html +++ b/website/www/site/layouts/shortcodes/flink_java_pipeline_options.html @@ -52,6 +52,11 @@ <td>Disable Beam metrics in Flink Runner</td> <td>Default: <code>false</code></td> </tr> +<tr> + <td><code>enableStableInputDrain</code></td> + <td>Allow drain operation for flink pipelines that contain RequiresStableInput operator. Note that at time of draining,the RequiresStableInput contract might be violated if there any processing related failures in the DoFn operator.</td> + <td>Default: <code>false</code></td> +</tr> <tr> <td><code>executionModeForBatch</code></td> <td>Flink mode for data exchange of batch pipelines. Reference {@link org.apache.flink.api.common.ExecutionMode}. Set this to BATCH_FORCED if pipelines get blocked, see https://issues.apache.org/jira/browse/FLINK-10672</td> diff --git a/website/www/site/layouts/shortcodes/flink_python_pipeline_options.html b/website/www/site/layouts/shortcodes/flink_python_pipeline_options.html index 5293f35e6a1e..27ae27ad05a3 100644 --- a/website/www/site/layouts/shortcodes/flink_python_pipeline_options.html +++ b/website/www/site/layouts/shortcodes/flink_python_pipeline_options.html @@ -52,6 +52,11 @@ <td>Disable Beam metrics in Flink Runner</td> <td>Default: <code>false</code></td> </tr> +<tr> + <td><code>enable_stable_input_drain</code></td> + <td>Allow drain operation for flink pipelines that contain RequiresStableInput operator. Note that at time of draining,the RequiresStableInput contract might be violated if there any processing related failures in the DoFn operator.</td> + <td>Default: <code>false</code></td> +</tr> <tr> <td><code>execution_mode_for_batch</code></td> <td>Flink mode for data exchange of batch pipelines. Reference {@link org.apache.flink.api.common.ExecutionMode}. Set this to BATCH_FORCED if pipelines get blocked, see https://issues.apache.org/jira/browse/FLINK-10672</td> From c5608241839104d91c57ff4e8e7dcec6906afd3f Mon Sep 17 00:00:00 2001 From: Andrey Devyatkin <andrey.9evyatkin@gmail.com> Date: Mon, 23 Oct 2023 16:16:23 +0200 Subject: [PATCH 226/435] Split workflows into subgroups in README (#29101) --- .github/workflows/README.md | 357 ++++++++++-------- ...beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml | 2 +- ...am_LoadTests_Go_Combine_Dataflow_Batch.yml | 2 +- .../beam_LoadTests_Go_GBK_Dataflow_Batch.yml | 2 +- ...beam_LoadTests_Go_ParDo_Dataflow_Batch.yml | 2 +- ..._LoadTests_Go_SideInput_Dataflow_Batch.yml | 2 +- ...oadTests_Java_CoGBK_Dataflow_Streaming.yml | 2 +- ..._LoadTests_Java_Combine_Dataflow_Batch.yml | 2 +- ...eam_LoadTests_Python_CoGBK_Flink_Batch.yml | 2 +- ...oadTests_Python_Combine_Dataflow_Batch.yml | 2 +- ...ests_Python_FnApiRunner_Microbenchmark.yml | 2 +- .../beam_PerformanceTests_AvroIOIT.yml | 2 +- .../beam_PerformanceTests_AvroIOIT_HDFS.yml | 2 +- ...rmanceTests_BigQueryIO_Batch_Java_Avro.yml | 2 +- ...rmanceTests_BigQueryIO_Batch_Java_Json.yml | 2 +- ...ormanceTests_BigQueryIO_Streaming_Java.yml | 2 +- .../beam_PostCommit_Java_InfluxDbIO_IT.yml | 2 +- 17 files changed, 206 insertions(+), 183 deletions(-) diff --git a/.github/workflows/README.md b/.github/workflows/README.md index ea497ae6404e..003d9f0561e9 100644 --- a/.github/workflows/README.md +++ b/.github/workflows/README.md @@ -178,100 +178,90 @@ You can do this by changing runs-on: [self-hosted, ubuntu-20.04, main] (self-hos # Workflows Please note that jobs with matrix need to have matrix element in the comment. Example: ```Run Python PreCommit (3.8)``` + +### PreCommit Jobs + +| Workflow name | Matrix | Trigger Phrase | Cron Status | +|:-------------:|:------:|:--------------:|:-----------:| +| [ PreCommit Community Metrics ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_CommunityMetrics.yml) | N/A |`Run CommunityMetrics PreCommit`| [![.github/workflows/beam_PreCommit_CommunityMetrics.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_CommunityMetrics.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_CommunityMetrics.yml) | +| [ PreCommit GHA ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_GHA.yml) | N/A |`Run GHA PreCommit`| [![.github/workflows/beam_PreCommit_GHA.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_GHA.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_GHA.yml) | +| [ PreCommit Go ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Go.yml) | N/A |`Run Go PreCommit`| [![.github/workflows/beam_PreCommit_Go.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Go.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Go.yml) | +| [ PreCommit GoPortable ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_GoPortable.yml) | N/A |`Run GoPortable PreCommit`| [![.github/workflows/beam_PreCommit_GoPortable.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_GoPortable.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_GoPortable.yml) | +| [ PreCommit Java ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java.yml) | N/A |`Run Java PreCommit`| [![.github/workflows/beam_PreCommit_Java.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java.yml) | +| [ PreCommit Java Amazon Web Services IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Amazon-Web-Services_IO_Direct.yml) | N/A |`Run Java_Amazon-Web-Services_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Amazon-Web-Services_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Amazon-Web-Services_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Amazon-Web-Services_IO_Direct.yml) | +| [ PreCommit Java Amazon Web Services2 IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Amazon-Web-Services2_IO_Direct.yml) | N/A |`Run Java_Amazon-Web-Services2_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Amazon-Web-Services2_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Amazon-Web-Services2_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Amazon-Web-Services2_IO_Direct.yml) | +| [ PreCommit Java Amqp IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Amqp_IO_Direct.yml) | N/A |`Run Java_Amqp_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Amqp_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Amqp_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Amqp_IO_Direct.yml) | +| [ PreCommit Java Azure IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml) | N/A |`Run Java_Azure_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml) | +| [ PreCommit Java Cassandra IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml) | N/A |`Run Java_Cassandra_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml) | +| [ PreCommit Java Cdap IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml) | N/A |`Run Java_Cdap_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml) | +| [ PreCommit Java Clickhouse IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml) | N/A |`Run Java_Clickhouse_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml) | +| [ PreCommit Java Csv IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml) | N/A |`Run Java_Csv_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml) | +| [ PreCommit Java Debezium IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml) | N/A |`Run Java_Debezium_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml) | +| [ PreCommit Java ElasticSearch IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml) | N/A |`Run Java_ElasticSearch_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml) | +| [ PreCommit Java Examples Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Examples_Dataflow.yml) | N/A |`Run Java_Examples_Dataflow PreCommit`| [![.github/workflows/beam_PreCommit_Java_Examples_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Examples_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Examples_Dataflow.yml) | +| [ PreCommit Java Examples Dataflow Java11 ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml) | N/A | `Run Java_Examples_Dataflow_Java11 PreCommit` | [![.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml) | +| [ PreCommit Java Examples Dataflow Java17 ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml) | N/A | `Run Java_Examples_Dataflow_Java17 PreCommit` | [![.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml) | +| [ PreCommit Java File-schema-transform IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_File-schema-transform_IO_Direct.yml) | N/A |`Run Java_File-schema-transform_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_File-schema-transform_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_File-schema-transform_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_File-schema-transform_IO_Direct.yml) | +| [ PreCommit Java Flink Versions ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Flink_Versions.yml) | N/A |`Run Java_Flink_Versions PreCommit`| [![.github/workflows/beam_PreCommit_Java_Flink_Versions.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Flink_Versions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Flink_Versions.yml) | +| [ PreCommit Java GCP IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml) | N/A |`Run Java_GCP_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml) | +| [ PreCommit Java Google-ads IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Google-ads_IO_Direct.yml) | N/A |`Run Java_Google-ads_IO_Direct PreCommit`| [![.github\workflows\beam_PreCommit_Java_Google-ads_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Google-ads_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Google-ads_IO_Direct.yml) | +| [ PreCommit Java Hadoop IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml) | N/A |`Run Java_Hadoop_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml) | +| [ PreCommit Java HBase IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml) | N/A |`Run Java_HBase_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml) | +| [ PreCommit Java HCatalog IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml) | N/A |`Run Java_HCatalog_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml) | +| [ PreCommit Java Kafka IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml) | N/A |`Run Java_Kafka_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml) | +| [ PreCommit Java InfluxDb IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml) | N/A |`Run Java_InfluxDb_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml) | +| [ PreCommit Java IOs Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_IOs_Direct.yml) | N/A |`Run Java_IOs_Direct PreCommit`| N/A | +| [ PreCommit Java JDBC IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml) | N/A |`Run Java_JDBC_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml) | +| [ PreCommit Java Jms IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml) | N/A |`Run Java_Jms_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml) | +| [ PreCommit Java Kinesis IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml) | N/A |`Run Java_Kinesis_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml) | +| [ PreCommit Java Kudu IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml) | N/A |`Run Java_Kudu_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml) | +| [ PreCommit Java MongoDb IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml) | N/A |`Run Java_MongoDb_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml) | +| [ PreCommit Java Mqtt IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml) | N/A |`Run Java_Mqtt_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml) | +| [ PreCommit Java Neo4j IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml) | N/A |`Run Java_Neo4j_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml) | +| [ PreCommit Java Parquet IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml) | N/A |`Run Java_Parquet_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml) | +| [ PreCommit Java Pulsar IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml) | N/A |`Run Java_Pulsar_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml) | +| [ PreCommit Java PVR Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_PVR_Flink_Batch.yml) | N/A |`Run Java_PVR_Flink_Batch PreCommit`| [![.github/workflows/beam_PreCommit_Java_PVR_Flink_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_PVR_Flink_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_PVR_Flink_Batch.yml) | +| [ PreCommit Java PVR Flink Docker ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml) | N/A |`Run Java_PVR_Flink_Docker PreCommit`| [![.github/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml) | +| [ PreCommit Java RabbitMq IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml) | N/A |`Run Java_RabbitMq_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml) | +| [ PreCommit Java Redis IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml) | N/A |`Run Java_Redis_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml) | +| [ PreCommit Java RequestResponse IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_RequestResponse_IO_Direct.yml) | N/A |`Run Java_RequestResponse_IO_Direct PreCommit`| [![.github\workflows\beam_RequestResponse_Java_RequestResponse_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_RequestResponse_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_RequestResponse_IO_Direct.yml) | +| [ PreCommit Java SingleStore IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml) | N/A |`Run Java_SingleStore_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml) | +| [ PreCommit Java Snowflake IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml) | N/A |`Run Java_Snowflake_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml) | +| [ PreCommit Java Solr IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml) | N/A |`Run Java_Solr_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml) | +| [ PreCommit Java Spark3 Versions ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Spark3_Versions.yml) | N/A | `Run Java_Spark3_Versions PreCommit` | [![.github/workflows/beam_PreCommit_Java_Spark3_Versions.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Spark3_Versions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Spark3_Versions.yml) | +| [ PreCommit Java Splunk IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml) | N/A |`Run Java_Splunk_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml) | +| [ PreCommit Java Thrift IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml) | N/A |`Run Java_Thrift_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml) | +| [ PreCommit Java Tika IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml) | N/A |`Run Java_Tika_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml) | +| [ PreCommit Kotlin Examples ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Kotlin_Examples.yml) | N/A | `Run Kotlin_Examples PreCommit` | [![.github/workflows/beam_PreCommit_Kotlin_Examples.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Kotlin_Examples.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Kotlin_Examples.yml) | +| [ PreCommit Portable Python ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Portable_Python.yml) | ['3.8','3.11'] | `Run Portable_Python PreCommit` | [![.github/workflows/beam_PreCommit_Portable_Python.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Portable_Python.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Portable_Python.yml) | +| [ PreCommit Python ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python.yml) | ['3.8','3.9','3.10','3.11'] | `Run Python PreCommit (matrix_element)` | [![.github/workflows/beam_PreCommit_Python.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python.yml) | +| [ PreCommit Python Coverage ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Coverage.yml) | N/A | `Run Python_Coverage PreCommit`| [![.github/workflows/beam_PreCommit_Python_Coverage.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Coverage.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Coverage.yml) | +| [ PreCommit Python Dataframes ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Dataframes.yml) | ['3.8','3.9','3.10','3.11'] | `Run Python_Dataframes PreCommit (matrix_element)`| [![.github/workflows/beam_PreCommit_Python_Dataframes.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Dataframes.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Dataframes.yml) | +| [ PreCommit Python Docker ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_PythonDocker.yml) | ['3.8','3.9','3.10','3.11'] | `Run PythonDocker PreCommit (matrix_element)`| [![.github/workflows/beam_PreCommit_PythonDocker.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_PythonDocker.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_PythonDocker.yml) | +| [ PreCommit Python Docs ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_PythonDocs.yml) | N/A | `Run PythonDocs PreCommit`| [![.github/workflows/beam_PreCommit_PythonDocs.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_PythonDocs.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_PythonDocs.yml) | +| [ PreCommit Python Examples ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Examples.yml) | ['3.8','3.9','3.10','3.11'] | `Run Python_Examples PreCommit (matrix_element)` | [![.github/workflows/beam_PreCommit_Python_Examples.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Examples.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Examples.yml) | +| [ PreCommit Python Formatter ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_PythonFormatter.yml) | N/A | `Run PythonFormatter PreCommit`| [![.github/workflows/beam_PreCommit_PythonFormatter.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_PythonFormatter.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_PythonFormatter.yml) | +| [ PreCommit Python Integration](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Integration.yml) | ['3.8','3.11'] | `Run Python_Integration PreCommit (matrix_element)` | [![.github/workflows/beam_PreCommit_Python_Integration.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Integration.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Integration.yml) | +| [ PreCommit Python Lint ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_PythonLint.yml) | N/A | `Run PythonLint PreCommit` | [![.github/workflows/beam_PreCommit_PythonLint.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_PythonLint.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_PythonLint.yml) | +| [ PreCommit Python PVR Flink ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_PVR_Flink.yml) | N/A | `Run Python_PVR_Flink PreCommit` | [![.github/workflows/beam_PreCommit_Python_PVR_Flink.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_PVR_Flink.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_PVR_Flink.yml) | +| [ PreCommit Python Runners ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Runners.yml) | ['3.8','3.9','3.10','3.11'] | `Run Python_Runners PreCommit (matrix_element)`| [![.github/workflows/beam_PreCommit_Python_Runners.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Runners.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Runners.yml) | +| [ PreCommit Python Transforms ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Transforms.yml) | ['3.8','3.9','3.10','3.11'] | `Run Python_Transforms PreCommit (matrix_element)`| [![.github/workflows/beam_PreCommit_Python_Transforms.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Transforms.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Transforms.yml) | +| [ PreCommit RAT ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_RAT.yml) | N/A | `Run RAT PreCommit` | [![.github/workflows/beam_PreCommit_RAT.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_RAT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_RAT.yml) | +| [ PreCommit Spotless ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Spotless.yml) | N/A | `Run Spotless PreCommit` | [![.github/workflows/beam_PreCommit_Spotless.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Spotless.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Spotless.yml) | +| [ PreCommit SQL ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_SQL.yml) | N/A |`Run SQL PreCommit`| [![.github/workflows/beam_PreCommit_SQL.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_SQL.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_SQL.yml) | +| [ PreCommit SQL Java11 ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_SQL_Java11.yml) | N/A |`Run SQL_Java11 PreCommit`| [![.github/workflows/beam_PreCommit_SQL_Java11.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_SQL_Java11.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_SQL_Java11.yml) | +| [ PreCommit SQL Java17 ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_SQL_Java17.yml) | N/A |`Run SQL_Java17 PreCommit`| [![.github/workflows/beam_PreCommit_SQL_Java17.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_SQL_Java17.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_SQL_Java17.yml) | +| [ PreCommit Typescript ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Typescript.yml) | N/A |`Run Typescript PreCommit`| [![.github/workflows/beam_PreCommit_Typescript.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Typescript.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Typescript.yml) | +| [ PreCommit Website ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Website.yml) | N/A |`Run Website PreCommit`| [![.github/workflows/beam_PreCommit_Website.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Website.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Website.yml) | +| [ PreCommit Website Stage GCS ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Website_Stage_GCS.yml) | N/A |`Run Website_Stage_GCS PreCommit`| [![.github/workflows/beam_PreCommit_Website_Stage_GCS.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Website_Stage_GCS.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Website_Stage_GCS.yml) | +| [ PreCommit Whitespace ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Whitespace.yml) | N/A |`Run Whitespace PreCommit`| [![.github/workflows/beam_PreCommit_Whitespace.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Whitespace.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Whitespace.yml) | + +### PostCommit Jobs + | Workflow name | Matrix | Trigger Phrase | Cron Status | |:-------------:|:------:|:--------------:|:-----------:| -| [ Cancel Stale Dataflow Jobs ](https://github.com/apache/beam/actions/workflows/beam_CancelStaleDataflowJobs.yml) | N/A | `Run Cancel Stale Dataflow Jobs` | [![.github/workflows/beam_CancelStaleDataflowJobs.yml](https://github.com/apache/beam/actions/workflows/beam_CancelStaleDataflowJobs.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_CancelStaleDataflowJobs.yml) | -| [ Clean Up GCP Resources ](https://github.com/apache/beam/actions/workflows/beam_CleanUpGCPResources.yml) | N/A | `Run Clean GCP Resources` | [![.github/workflows/beam_CleanUpGCPResources.yml](https://github.com/apache/beam/actions/workflows/beam_CleanUpGCPResources.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_CleanUpGCPResources.yml) | -| [ Clean Up Prebuilt SDK Images ](https://github.com/apache/beam/actions/workflows/beam_CleanUpPrebuiltSDKImages.yml) | N/A | `Run Clean Prebuilt Images` | [![.github/workflows/beam_beam_CleanUpPrebuiltSDKImages.yml](https://github.com/apache/beam/actions/workflows/beam_CleanUpPrebuiltSDKImages.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_CleanUpPrebuiltSDKImages.yml) | -| [ Cleanup Dataproc Resources ](https://github.com/apache/beam/actions/workflows/beam_CleanUpDataprocResources.yml) | N/A | N/A | [![.github/workflows/beam_CleanUpDataprocResources.yml](https://github.com/apache/beam/actions/workflows/beam_CleanUpDataprocResources.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_CleanUpDataprocResources.yml) -| [ CloudML Benchmarks Dataflow ](https://github.com/apache/beam/actions/workflows/beam_CloudML_Benchmarks_Dataflow.yml) | N/A |`Run TFT Criteo Benchmarks`| [![.github/workflows/beam_CloudML_Benchmarks_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_CloudML_Benchmarks_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_CloudML_Benchmarks_Dataflow.yml) -| [ Community Metrics Prober ](https://github.com/apache/beam/actions/workflows/beam_Prober_CommunityMetrics.yml) | N/A |`Run Community Metrics Prober`| [![.github/workflows/beam_Prober_CommunityMetrics.yml](https://github.com/apache/beam/actions/workflows/beam_Prober_CommunityMetrics.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_Prober_CommunityMetrics.yml) -| [ Inference Python Benchmarks Dataflow ](https://github.com/apache/beam/actions/workflows/beam_Inference_Python_Benchmarks_Dataflow.yml) | N/A |`Run Inference Benchmarks`| [![.github/workflows/beam_Inference_Python_Benchmarks_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_Inference_Python_Benchmarks_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_Inference_Python_Benchmarks_Dataflow.yml) -| [ Java InfluxDbIO Integration Test ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml) | N/A |`Run Java InfluxDbIO_IT`| [![.github/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml) | [ Java JMH ](https://github.com/apache/beam/actions/workflows/beam_Java_JMH.yml) | N/A | N/A | [![.github/workflows/beam_Java_JMH.yml](https://github.com/apache/beam/actions/workflows/beam_Java_JMH.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_Java_JMH.yml) -| [ Load Tests GBK Dataflow Batch Go ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml) -| [ Load Tests CoGBK Dataflow Batch Go ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml) -| [ Load Tests CoGBK Dataflow Streaming Java ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml) -| [ Load Tests Combine Dataflow Batch Java ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml) -| [ Load Tests Combine Dataflow Batch Python ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml) -| [ Load Tests Combine Dataflow Batch Python ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml) -| [ Load Tests FnApiRunner Microbenchmark Python ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_FnApiRunner_Microbenchmark.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Python_FnApiRunner_Microbenchmark](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_FnApiRunner_Microbenchmark.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_FnApiRunner_Microbenchmark.yml) -| [ Load Tests Go CoGBK Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_CoGBK_Flink_Batch.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Go_CoGBK_Flink_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_CoGBK_Flink_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_CoGBK_Flink_Batch.yml) -| [ Load Tests Go Combine Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml) -| [ Load Tests Go GBK Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml) -| [ Load Tests Go ParDo Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml) -| [ Load Tests Go SideInput Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml) -| [ LoadTests Java CoGBK Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml) -| [ LoadTests Java CoGBK Dataflow V2 Batch JavaVersions ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml) | ['11','17'] | N/A | [![.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml) -| [ LoadTests Java CoGBK Dataflow V2 Streaming JavaVersions ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml) | ['11','17'] | N/A | [![.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml) -| [ LoadTests Java CoGBK SparkStructuredStreaming Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml) -| [ Load Tests Java Combine Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_Combine_Dataflow_Streaming.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_Combine_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_Combine_Dataflow_Streaming.yml) -| [ Load Tests Java Combine SparkStructuredStreaming Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml) -| [ Load Tests Java ParDo Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml) -| [ Load Tests Java ParDo Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml) -| [ LoadTests Java ParDo Dataflow V2 Batch JavaVersions ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions.yml) | ['11','17'] | N/A | [![.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions.yml) -| [ LoadTests Java ParDo Dataflow V2 Streaming JavaVersions ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions.yml) | ['11','17'] | N/A | [![.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions.yml) -| [ Load Tests Java ParDo SparkStructuredStreaming Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml) -| [ Load Tests ParDo Dataflow Batch Go ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml) -| [ Load Tests SideInput Dataflow Batch Go ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml) -| [ LoadTests Java Combine Smoke ](https://github.com/apache/beam/actions/workflows/beam_Java_LoadTests_Combine_Smoke.yml) | N/A | N/A | [![.github/workflows/beam_Java_LoadTests_Combine_Smoke.yml](https://github.com/apache/beam/actions/workflows/beam_Java_LoadTests_Combine_Smoke.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_Java_LoadTests_Combine_Smoke.yml) -| [ LoadTests Java GBK Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml) -| [ LoadTests Java GBK Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml) -| [ LoadTests Java GBK Dataflow V2 Batch Java11 ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml) -| [ LoadTests Java GBK Dataflow V2 Batch Java17 ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml) -| [ LoadTests Java GBK Dataflow V2 Streaming Java11 ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml) -| [ LoadTests Java GBK Dataflow V2 Streaming Java17 ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml) -| [ LoadTests Java GBK Smoke ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Smoke.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Java_GBK_Smoke.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Smoke.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Smoke.yml) -| [ LoadTests Python GBK reiterate Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Batch.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Batch.yml) -| [ LoadTests Python GBK reiterate Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Streaming.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Streaming.yml) -| [ LoadTests Python CoGBK Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml) -| [ LoadTests Python CoGBK Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml) -| [ LoadTests Python Combine Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Dataflow_Streaming.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Dataflow_Streaming.yml) -| [ LoadTests Python Combine Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml) -| [ LoadTests Python Combine Flink Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml) -| [ LoadTests Python GBK Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml) -| [ LoadTests Python GBK Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml) -| [ LoadTests Python GBK Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_Flink_Batch.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Python_GBK_Flink_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_Flink_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_Flink_Batch.yml) -| [ LoadTests Python CoGBK Dataflow Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml) -| [ LoadTests Python ParDo Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Dataflow_Batch.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Dataflow_Batch.yml) -| [ LoadTests Python ParDo Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml) -| [ LoadTests Python ParDo Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml) -| [ LoadTests Python ParDo Flink Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml) -| [ LoadTests Python SideInput Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_SideInput_Dataflow_Batch.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Python_SideInput_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_SideInput_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_SideInput_Dataflow_Batch.yml) -| [ LoadTests Python Smoke ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Smoke.yml) | N/A | N/A | [![.github/workflows/beam_LoadTests_Python_Smoke.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Smoke.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Smoke.yml) -| [ Performance Tests AvroIOIT HDFS ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml) | N/A | N/A | [![.github/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml) -| [ Performance Tests AvroIOIT ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_AvroIOIT.yml) | N/A | N/A | [![.github/workflows/beam_PerformanceTests_AvroIOIT.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_AvroIOIT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_AvroIOIT.yml) -| [ Performance Tests BigQueryIO Batch Java Avro ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml) | N/A | N/A | [![.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml) -| [ Performance Tests BigQueryIO Batch Java Json ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml) | N/A | N/A | [![.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml) -| [ Performance Tests BigQueryIO Streaming Java ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml) | N/A | N/A | [![.github/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml) -| [ Performance Tests BigQueryIO Read Python ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml) | N/A | N/A | [![.github/workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml) -| [ Performance Tests BigQueryIO Write Python Batch ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml) | N/A | N/A | [![.github/workflows/beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml) -| [ PerformanceTests Cdap ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Cdap.yml) | N/A | N/A | [![.github/workflows/beam_PerformanceTests_Cdap.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Cdap.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Cdap.yml) -| [ PerformanceTests Compressed TextIOIT HDFS ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml) | N/A | N/A | [![.github/workflows/beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml) -| [ PerformanceTests Compressed TextIOIT ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml) | N/A | N/A | [![.github/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml) -| [ PerformanceTests HadoopFormat ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_HadoopFormat.yml) | N/A | N/A | [![.github/workflows/beam_PerformanceTests_HadoopFormat.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_HadoopFormat.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_HadoopFormat.yml) -| [ PerformanceTests JDBC ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_JDBC.yml) | N/A | N/A | [![.github/workflows/beam_PerformanceTests_JDBC.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_JDBC.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_JDBC.yml) -| [ PerformanceTests Kafka IO ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Kafka_IO.yml) | N/A | N/A | [![.github/workflows/beam_PerformanceTests_Kafka_IO.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Kafka_IO.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Kafka_IO.yml) -| [ PerformanceTests ManyFiles TextIOIT HDFS ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml) | N/A | N/A | [![.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml) -| [ PerformanceTests ManyFiles TextIOIT ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml) | N/A | N/A | [![.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml) -| [ PerformanceTests MongoDBIO IT ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_MongoDBIO_IT.yml) | N/A | N/A | [![.github/workflows/beam_PerformanceTests_MongoDBIO_IT.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_MongoDBIO_IT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_MongoDBIO_IT.yml) -| [ PerformanceTests ParquetIOIT HDFS ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml) | N/A | N/A | [![.github/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml) -| [ PerformanceTests ParquetIOIT ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ParquetIOIT.yml) | N/A | N/A | [![.github/workflows/beam_PerformanceTests_ParquetIOIT.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ParquetIOIT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ParquetIOIT.yml) -| [ PerformanceTests PubsubIOIT Python Streaming ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml) | N/A | N/A | [![.github/workflows/beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml) -| [ PerformanceTests SingleStoreIO ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SingleStoreIO.yml) | N/A | N/A | [![.github/workflows/beam_PerformanceTests_SingleStoreIO.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SingleStoreIO.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SingleStoreIO.yml) -| [ PerformanceTests SpannerIO Read 2GB Python ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml) | N/A | N/A | [![.github/workflows/beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml) -| [ PerformanceTests SpannerIO Write 2GB Python Batch ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml) | N/A | N/A | [![.github/workflows/beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml) -| [ PerformanceTests SparkReceiver IO ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SparkReceiver_IO.yml) | N/A | N/A | [![.github/workflows/beam_PerformanceTests_SparkReceiver_IO.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SparkReceiver_IO.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SparkReceiver_IO.yml) -| [ PerformanceTests SQLBigQueryIO Batch Java ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml) | N/A | N/A | [![.github/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml) -| [ PerformanceTests TextIOIT HDFS ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml) | N/A | N/A | [![.github/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml) -| [ PerformanceTests TextIOIT Python ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TextIOIT_Python.yml) | N/A | N/A | [![.github/workflows/beam_PerformanceTests_TextIOIT_Python.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TextIOIT_Python.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TextIOIT_Python.yml) -| [ PerformanceTests TextIOIT ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TextIOIT.yml) | N/A | N/A | [![.github/workflows/beam_PerformanceTests_TextIOIT.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TextIOIT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TextIOIT.yml) -| [ PerformanceTests TFRecordIOIT HDFS ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml) | N/A | N/A | [![.github/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml) -| [ PerformanceTests TFRecordIOIT ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TFRecordIOIT.yml) | N/A | N/A | [![.github/workflows/beam_PerformanceTests_TFRecordIOIT.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TFRecordIOIT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TFRecordIOIT.yml) -| [ PerformanceTests WordCountIT PythonVersions ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml) | ['3.8'] | N/A | [![.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml) -| [ PerformanceTests XmlIOIT HDFS ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml) | N/A | N/A | [![.github/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml) -| [ PerformanceTests XmlIOIT ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_XmlIOIT.yml) | N/A | N/A | [![.github/workflows/beam_PerformanceTests_XmlIOIT.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_XmlIOIT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_XmlIOIT.yml) -| [ PerformanceTests xlang KafkaIO Python ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml) | N/A | N/A | [![.github/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml) +| [ Publish Beam SDK Snapshots ](https://github.com/apache/beam/actions/workflows/beam_Publish_Beam_SDK_Snapshots.yml) | N/A | N/A | [![.github/workflows/beam_Publish_Beam_SDK_Snapshots.yml](https://github.com/apache/beam/actions/workflows/beam_Publish_Beam_SDK_Snapshots.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_Publish_Beam_SDK_Snapshots.yml) | | [ PostCommit BeamMetrics Publish ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_BeamMetrics_Publish.yml) | N/A |`Run Beam Metrics Deployment`| [![.github/workflows/beam_PostCommit_BeamMetrics_Publish.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_BeamMetrics_Publish.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_BeamMetrics_Publish.yml) | [ PostCommit Go ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go.yml) | N/A |`Run Go PostCommit`| [![.github/workflows/beam_PostCommit_Go.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go.yml) | | [ PostCommit Go Dataflow ARM](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go_Dataflow_ARM.yml) | N/A |`Run Go PostCommit Dataflow ARM`| [![.github/workflows/beam_PostCommit_Go_Dataflow_ARM.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go_Dataflow_ARM.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go_Dataflow_ARM.yml) | @@ -290,6 +280,7 @@ Please note that jobs with matrix need to have matrix element in the comment. Ex | [ PostCommit Java Examples Flink ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Flink.yml) | N/A |`Run Java Examples_Flink`| [![.github/workflows/beam_PostCommit_Java_Examples_Flink.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Flink.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Flink.yml) | | [ PostCommit Java Examples Spark ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Spark.yml) | N/A |`Run Java Examples_Spark`| [![.github/workflows/beam_PostCommit_Java_Examples_Spark.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Spark.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Spark.yml) | | [ PostCommit Java Hadoop Versions ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Hadoop_Versions.yml) | N/A |`Run PostCommit_Java_Hadoop_Versions`| [![.github/workflows/beam_PostCommit_Java_Hadoop_Versions.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Hadoop_Versions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Hadoop_Versions.yml) | +| [ PostCommit Java InfluxDbIO Integration Test ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml) | N/A |`Run Java InfluxDbIO_IT`| [![.github/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml) | [ PostCommit Java Jpms Dataflow Java11 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml) | N/A |`Run Jpms Dataflow Java 11 PostCommit`| [![.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml) | | [ PostCommit Java Jpms Dataflow Java17 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml) | N/A |`Run Jpms Dataflow Java 17 PostCommit`| [![.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml) | | [ PostCommit Java Jpms Direct Java11 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml) | N/A |`Run Jpms Direct Java 11 PostCommit`| [![.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml) | @@ -359,81 +350,113 @@ Please note that jobs with matrix need to have matrix element in the comment. Ex | [ PostCommit XVR PythonUsingJavaSQL Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml) | N/A |`Run XVR_PythonUsingJavaSQL_Dataflow PostCommit`| [![.github/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml) | | [ PostCommit XVR Samza ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Samza.yml) | N/A |`Run XVR_Samza PostCommit`| [![.github/workflows/beam_PostCommit_XVR_Samza.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Samza.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Samza.yml) | | [ PostCommit XVR Spark3 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Spark3.yml) | N/A |`Run XVR_Spark3 PostCommit`| [![.github/workflows/beam_PostCommit_XVR_Spark3.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Spark3.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Spark3.yml) | -| [ PreCommit Community Metrics ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_CommunityMetrics.yml) | N/A |`Run CommunityMetrics PreCommit`| [![.github/workflows/beam_PreCommit_CommunityMetrics.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_CommunityMetrics.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_CommunityMetrics.yml) | -| [ PreCommit GHA ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_GHA.yml) | N/A |`Run GHA PreCommit`| [![.github/workflows/beam_PreCommit_GHA.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_GHA.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_GHA.yml) | -| [ PreCommit Go ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Go.yml) | N/A |`Run Go PreCommit`| [![.github/workflows/beam_PreCommit_Go.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Go.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Go.yml) | -| [ PreCommit Java ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java.yml) | N/A |`Run Java PreCommit`| [![.github/workflows/beam_PreCommit_Java.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java.yml) | -| [ PreCommit Java Amazon Web Services IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Amazon-Web-Services_IO_Direct.yml) | N/A |`Run Java_Amazon-Web-Services_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Amazon-Web-Services_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Amazon-Web-Services_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Amazon-Web-Services_IO_Direct.yml) | -| [ PreCommit Java Amazon Web Services2 IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Amazon-Web-Services2_IO_Direct.yml) | N/A |`Run Java_Amazon-Web-Services2_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Amazon-Web-Services2_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Amazon-Web-Services2_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Amazon-Web-Services2_IO_Direct.yml) | -| [ PreCommit Java Amqp IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Amqp_IO_Direct.yml) | N/A |`Run Java_Amqp_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Amqp_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Amqp_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Amqp_IO_Direct.yml) | -| [ PreCommit Java Azure IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml) | N/A |`Run Java_Azure_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml) | -| [ PreCommit Java Cassandra IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml) | N/A |`Run Java_Cassandra_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml) | -| [ PreCommit Java Cdap IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml) | N/A |`Run Java_Cdap_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml) | -| [ PreCommit Java Clickhouse IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml) | N/A |`Run Java_Clickhouse_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml) | -| [ PreCommit Java Csv IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml) | N/A |`Run Java_Csv_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml) | -| [ PreCommit Java Debezium IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml) | N/A |`Run Java_Debezium_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml) | -| [ PreCommit Java ElasticSearch IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml) | N/A |`Run Java_ElasticSearch_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml) | -| [ PreCommit Java Examples Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Examples_Dataflow.yml) | N/A |`Run Java_Examples_Dataflow PreCommit`| [![.github/workflows/beam_PreCommit_Java_Examples_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Examples_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Examples_Dataflow.yml) | -| [ PreCommit Java Examples Dataflow Java11 ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml) | N/A | `Run Java_Examples_Dataflow_Java11 PreCommit` | [![.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml) | -| [ PreCommit Java Examples Dataflow Java17 ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml) | N/A | `Run Java_Examples_Dataflow_Java17 PreCommit` | [![.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml) | -| [ PreCommit Java File-schema-transform IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_File-schema-transform_IO_Direct.yml) | N/A |`Run Java_File-schema-transform_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_File-schema-transform_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_File-schema-transform_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_File-schema-transform_IO_Direct.yml) | -| [ PreCommit Java Flink Versions ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Flink_Versions.yml) | N/A |`Run Java_Flink_Versions PreCommit`| [![.github/workflows/beam_PreCommit_Java_Flink_Versions.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Flink_Versions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Flink_Versions.yml) | -| [ PreCommit Java GCP IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml) | N/A |`Run Java_GCP_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml) | -| [ PreCommit Java Google-ads IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Google-ads_IO_Direct.yml) | N/A |`Run Java_Google-ads_IO_Direct PreCommit`| [![.github\workflows\beam_PreCommit_Java_Google-ads_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Google-ads_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Google-ads_IO_Direct.yml) | -| [ PreCommit Java Hadoop IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml) | N/A |`Run Java_Hadoop_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml) | -| [ PreCommit Java HBase IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml) | N/A |`Run Java_HBase_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml) | -| [ PreCommit Java HCatalog IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml) | N/A |`Run Java_HCatalog_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml) | -| [ PreCommit Java Kafka IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml) | N/A |`Run Java_Kafka_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml) | -| [ PreCommit Java InfluxDb IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml) | N/A |`Run Java_InfluxDb_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml) | -| [ PreCommit Java IOs Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_IOs_Direct.yml) | N/A |`Run Java_IOs_Direct PreCommit`| N/A | -| [ PreCommit Java JDBC IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml) | N/A |`Run Java_JDBC_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml) | -| [ PreCommit Java Jms IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml) | N/A |`Run Java_Jms_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml) | -| [ PreCommit Java Kinesis IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml) | N/A |`Run Java_Kinesis_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml) | -| [ PreCommit Java Kudu IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml) | N/A |`Run Java_Kudu_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml) | -| [ PreCommit Java MongoDb IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml) | N/A |`Run Java_MongoDb_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml) | -| [ PreCommit Java Mqtt IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml) | N/A |`Run Java_Mqtt_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml) | -| [ PreCommit Java Neo4j IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml) | N/A |`Run Java_Neo4j_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml) | -| [ PreCommit Java Parquet IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml) | N/A |`Run Java_Parquet_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml) | -| [ PreCommit Java Pulsar IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml) | N/A |`Run Java_Pulsar_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml) | -| [ PreCommit Java PVR Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_PVR_Flink_Batch.yml) | N/A |`Run Java_PVR_Flink_Batch PreCommit`| [![.github/workflows/beam_PreCommit_Java_PVR_Flink_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_PVR_Flink_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_PVR_Flink_Batch.yml) | -| [ PreCommit Java PVR Flink Docker ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml) | N/A |`Run Java_PVR_Flink_Docker PreCommit`| [![.github/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml) | -| [ PreCommit Java RabbitMq IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml) | N/A |`Run Java_RabbitMq_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml) | -| [ PreCommit Java Redis IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml) | N/A |`Run Java_Redis_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml) | -| [ PreCommit Java RequestResponse IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_RequestResponse_IO_Direct.yml) | N/A |`Run Java_RequestResponse_IO_Direct PreCommit`| [![.github\workflows\beam_RequestResponse_Java_RequestResponse_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_RequestResponse_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_RequestResponse_IO_Direct.yml) | -| [ PreCommit Java SingleStore IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml) | N/A |`Run Java_SingleStore_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml) | -| [ PreCommit Java Snowflake IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml) | N/A |`Run Java_Snowflake_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml) | -| [ PreCommit Java Solr IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml) | N/A |`Run Java_Solr_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml) | -| [ PreCommit Java Spark3 Versions ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Spark3_Versions.yml) | N/A | `Run Java_Spark3_Versions PreCommit` | [![.github/workflows/beam_PreCommit_Java_Spark3_Versions.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Spark3_Versions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Spark3_Versions.yml) | -| [ PreCommit Java Splunk IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml) | N/A |`Run Java_Splunk_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml) | -| [ PreCommit Java Thrift IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml) | N/A |`Run Java_Thrift_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml) | -| [ PreCommit Java Tika IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml) | N/A |`Run Java_Tika_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml) | -| [ PreCommit Python ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python.yml) | ['3.8','3.9','3.10','3.11'] | `Run Python PreCommit (matrix_element)` | [![.github/workflows/beam_PreCommit_Python.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python.yml) | -| [ PreCommit Python Coverage ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Coverage.yml) | N/A | `Run Python_Coverage PreCommit`| [![.github/workflows/beam_PreCommit_Python_Coverage.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Coverage.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Coverage.yml) | -| [ PreCommit Python Dataframes ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Dataframes.yml) | ['3.8','3.9','3.10','3.11'] | `Run Python_Dataframes PreCommit (matrix_element)`| [![.github/workflows/beam_PreCommit_Python_Dataframes.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Dataframes.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Dataframes.yml) | -| [ PreCommit Python Docker ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_PythonDocker.yml) | ['3.8','3.9','3.10','3.11'] | `Run PythonDocker PreCommit (matrix_element)`| [![.github/workflows/beam_PreCommit_PythonDocker.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_PythonDocker.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_PythonDocker.yml) | -| [ PreCommit Python Docs ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_PythonDocs.yml) | N/A | `Run PythonDocs PreCommit`| [![.github/workflows/beam_PreCommit_PythonDocs.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_PythonDocs.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_PythonDocs.yml) | -| [ PreCommit Python Examples ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Examples.yml) | ['3.8','3.9','3.10','3.11'] | `Run Python_Examples PreCommit (matrix_element)` | [![.github/workflows/beam_PreCommit_Python_Examples.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Examples.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Examples.yml) | -| [ PreCommit Python Formatter ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_PythonFormatter.yml) | N/A | `Run PythonFormatter PreCommit`| [![.github/workflows/beam_PreCommit_PythonFormatter.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_PythonFormatter.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_PythonFormatter.yml) | -| [ PreCommit Python Integration](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Integration.yml) | ['3.8','3.11'] | `Run Python_Integration PreCommit (matrix_element)` | [![.github/workflows/beam_PreCommit_Python_Integration.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Integration.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Integration.yml) | -| [ PreCommit Python Lint ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_PythonLint.yml) | N/A | `Run PythonLint PreCommit` | [![.github/workflows/beam_PreCommit_PythonLint.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_PythonLint.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_PythonLint.yml) | -| [ PreCommit Python PVR Flink ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_PVR_Flink.yml) | N/A | `Run Python_PVR_Flink PreCommit` | [![.github/workflows/beam_PreCommit_Python_PVR_Flink.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_PVR_Flink.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_PVR_Flink.yml) | -| [ PreCommit Python Runners ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Runners.yml) | ['3.8','3.9','3.10','3.11'] | `Run Python_Runners PreCommit (matrix_element)`| [![.github/workflows/beam_PreCommit_Python_Runners.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Runners.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Runners.yml) | -| [ PreCommit Python Transforms ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Transforms.yml) | ['3.8','3.9','3.10','3.11'] | `Run Python_Transforms PreCommit (matrix_element)`| [![.github/workflows/beam_PreCommit_Python_Transforms.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Transforms.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Transforms.yml) | -| [ PreCommit RAT ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_RAT.yml) | N/A | `Run RAT PreCommit` | [![.github/workflows/beam_PreCommit_RAT.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_RAT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_RAT.yml) | -| [ PreCommit Spotless ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Spotless.yml) | N/A | `Run Spotless PreCommit` | [![.github/workflows/beam_PreCommit_Spotless.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Spotless.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Spotless.yml) | -| [ PreCommit SQL ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_SQL.yml) | N/A |`Run SQL PreCommit`| [![.github/workflows/beam_PreCommit_SQL.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_SQL.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_SQL.yml) | -| [ PreCommit SQL Java11 ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_SQL_Java11.yml) | N/A |`Run SQL_Java11 PreCommit`| [![.github/workflows/beam_PreCommit_SQL_Java11.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_SQL_Java11.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_SQL_Java11.yml) | -| [ PreCommit SQL Java17 ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_SQL_Java17.yml) | N/A |`Run SQL_Java17 PreCommit`| [![.github/workflows/beam_PreCommit_SQL_Java17.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_SQL_Java17.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_SQL_Java17.yml) | -| [ PreCommit Typescript ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Typescript.yml) | N/A |`Run Typescript PreCommit`| [![.github/workflows/beam_PreCommit_Typescript.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Typescript.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Typescript.yml) | -| [ PreCommit Website ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Website.yml) | N/A |`Run Website PreCommit`| [![.github/workflows/beam_PreCommit_Website.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Website.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Website.yml) | -| [ PreCommit Website Stage GCS ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Website_Stage_GCS.yml) | N/A |`Run Website_Stage_GCS PreCommit`| [![.github/workflows/beam_PreCommit_Website_Stage_GCS.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Website_Stage_GCS.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Website_Stage_GCS.yml) | -| [ PreCommit Whitespace ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Whitespace.yml) | N/A |`Run Whitespace PreCommit`| [![.github/workflows/beam_PreCommit_Whitespace.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Whitespace.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Whitespace.yml) | -| [ Python Validates Container Dataflow ARM ](https://github.com/apache/beam/actions/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml) | ['3.8','3.9','3.10','3.11'] |beam_Python_ValidatesContainer_Dataflow_ARM.yml -| [![.github/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml](https://github.com/apache/beam/actions/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml) | -| [ PreCommit GoPortable ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_GoPortable.yml) | N/A |`Run GoPortable PreCommit`| [![.github/workflows/beam_PreCommit_GoPortable.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_GoPortable.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_GoPortable.yml) | -| [ PreCommit Kotlin Examples ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Kotlin_Examples.yml) | N/A | `Run Kotlin_Examples PreCommit` | [![.github/workflows/beam_PreCommit_Kotlin_Examples.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Kotlin_Examples.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Kotlin_Examples.yml) | -| [ PreCommit Portable Python ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Portable_Python.yml) | ['3.8','3.11'] | `Run Portable_Python PreCommit` | [![.github/workflows/beam_PreCommit_Portable_Python.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Portable_Python.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Portable_Python.yml) | -| [ Publish Beam SDK Snapshots ](https://github.com/apache/beam/actions/workflows/beam_Publish_Beam_SDK_Snapshots.yml) | N/A | N/A | [![.github/workflows/beam_Publish_Beam_SDK_Snapshots.yml](https://github.com/apache/beam/actions/workflows/beam_Publish_Beam_SDK_Snapshots.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_Publish_Beam_SDK_Snapshots.yml) | -| [ Publish Docker Snapshots ](https://github.com/apache/beam/actions/workflows/beam_Publish_Docker_Snapshots.yml) | N/A | N/A | [![.github/workflows/beam_Publish_Docker_Snapshots.yml](https://github.com/apache/beam/actions/workflows/beam_Publish_Docker_Snapshots.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_Publish_Docker_Snapshots.yml) | +| [ Python Validates Container Dataflow ARM ](https://github.com/apache/beam/actions/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml) | ['3.8','3.9','3.10','3.11'] |`Run Python ValidatesContainer Dataflow ARM (matrix_element)`|[![.github/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml](https://github.com/apache/beam/actions/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml) | + +### PerformanceTests Jobs + +| Workflow name | Matrix | Trigger Phrase | Cron Status | +|:-------------:|:------:|:--------------:|:-----------:| +| [ CloudML Benchmarks Dataflow ](https://github.com/apache/beam/actions/workflows/beam_CloudML_Benchmarks_Dataflow.yml) | N/A |`Run TFT Criteo Benchmarks`| [![.github/workflows/beam_CloudML_Benchmarks_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_CloudML_Benchmarks_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_CloudML_Benchmarks_Dataflow.yml) +| [ Inference Python Benchmarks Dataflow ](https://github.com/apache/beam/actions/workflows/beam_Inference_Python_Benchmarks_Dataflow.yml) | N/A |`Run Inference Benchmarks`| [![.github/workflows/beam_Inference_Python_Benchmarks_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_Inference_Python_Benchmarks_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_Inference_Python_Benchmarks_Dataflow.yml) +| [ Performance Tests AvroIOIT HDFS ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml) | N/A |`Run Java AvroIO Performance Test HDFS`| [![.github/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml) +| [ Performance Tests AvroIOIT ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_AvroIOIT.yml) | N/A |`Run Java AvroIO Performance Test`| [![.github/workflows/beam_PerformanceTests_AvroIOIT.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_AvroIOIT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_AvroIOIT.yml) +| [ Performance Tests BigQueryIO Batch Java Avro ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml) | N/A |`Run BigQueryIO Batch Performance Test Java Avro`| [![.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml) +| [ Performance Tests BigQueryIO Batch Java Json ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml) | N/A |`Run BigQueryIO Batch Performance Test Java Json`| [![.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml) +| [ Performance Tests BigQueryIO Streaming Java ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml) | N/A |`Run BigQueryIO Streaming Performance Test Java`| [![.github/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml) +| [ Performance Tests BigQueryIO Read Python ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml) | N/A |`Run BigQueryIO Read Performance Test Python`| [![.github/workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml) +| [ Performance Tests BigQueryIO Write Python Batch ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml) | N/A |`Run BigQueryIO Write Performance Test Python`| [![.github/workflows/beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml) +| [ PerformanceTests Cdap ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Cdap.yml) | N/A |`Run Java CdapIO Performance Test`| [![.github/workflows/beam_PerformanceTests_Cdap.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Cdap.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Cdap.yml) +| [ PerformanceTests Compressed TextIOIT HDFS ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml) | N/A |`Run Java CompressedTextIO Performance Test HDFS`| [![.github/workflows/beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml) +| [ PerformanceTests Compressed TextIOIT ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml) | N/A |`Run Java CompressedTextIO Performance Test`| [![.github/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml) +| [ PerformanceTests HadoopFormat ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_HadoopFormat.yml) | N/A |`Run Java HadoopFormatIO Performance Test`| [![.github/workflows/beam_PerformanceTests_HadoopFormat.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_HadoopFormat.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_HadoopFormat.yml) +| [ PerformanceTests JDBC ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_JDBC.yml) | N/A |`Run Java JdbcIO Performance Test`| [![.github/workflows/beam_PerformanceTests_JDBC.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_JDBC.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_JDBC.yml) +| [ PerformanceTests Kafka IO ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Kafka_IO.yml) | N/A |`Run Java KafkaIO Performance Test`| [![.github/workflows/beam_PerformanceTests_Kafka_IO.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Kafka_IO.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Kafka_IO.yml) +| [ PerformanceTests ManyFiles TextIOIT HDFS ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml) | N/A |`Run Java ManyFilesTextIO Performance Test HDFS`| [![.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml) +| [ PerformanceTests ManyFiles TextIOIT ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml) | N/A |`Run Java ManyFilesTextIO Performance Test`| [![.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml) +| [ PerformanceTests MongoDBIO IT ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_MongoDBIO_IT.yml) | N/A |`Run Java MongoDBIO Performance Test`| [![.github/workflows/beam_PerformanceTests_MongoDBIO_IT.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_MongoDBIO_IT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_MongoDBIO_IT.yml) +| [ PerformanceTests ParquetIOIT HDFS ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml) | N/A |`Run Java ParquetIO Performance Test HDFS`| [![.github/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml) +| [ PerformanceTests ParquetIOIT ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ParquetIOIT.yml) | N/A |`Run Java ParquetIO Performance Test`| [![.github/workflows/beam_PerformanceTests_ParquetIOIT.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ParquetIOIT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ParquetIOIT.yml) +| [ PerformanceTests PubsubIOIT Python Streaming ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml) | N/A |`Run PubsubIO Performance Test Python`| [![.github/workflows/beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml) +| [ PerformanceTests SingleStoreIO ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SingleStoreIO.yml) | N/A |`Run Java SingleStoreIO Performance Test`| [![.github/workflows/beam_PerformanceTests_SingleStoreIO.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SingleStoreIO.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SingleStoreIO.yml) +| [ PerformanceTests SpannerIO Read 2GB Python ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml) | N/A |`Run SpannerIO Read 2GB Performance Test Python`| [![.github/workflows/beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml) +| [ PerformanceTests SpannerIO Write 2GB Python Batch ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml) | N/A |`Run SpannerIO Write 2GB Performance Test Python Batch`| [![.github/workflows/beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml) +| [ PerformanceTests SparkReceiver IO ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SparkReceiver_IO.yml) | N/A |`Run Java SparkReceiverIO Performance Test`| [![.github/workflows/beam_PerformanceTests_SparkReceiver_IO.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SparkReceiver_IO.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SparkReceiver_IO.yml) +| [ PerformanceTests SQLBigQueryIO Batch Java ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml) | N/A |`Run SQLBigQueryIO Batch Performance Test Java`| [![.github/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml) +| [ PerformanceTests TextIOIT HDFS ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml) | N/A |`Run Java TextIO Performance Test HDFS`| [![.github/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml) +| [ PerformanceTests TextIOIT Python ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TextIOIT_Python.yml) | N/A |`Run Python TextIO Performance Test`| [![.github/workflows/beam_PerformanceTests_TextIOIT_Python.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TextIOIT_Python.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TextIOIT_Python.yml) +| [ PerformanceTests TextIOIT ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TextIOIT.yml) | N/A |`Run Java TextIO Performance Test`| [![.github/workflows/beam_PerformanceTests_TextIOIT.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TextIOIT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TextIOIT.yml) +| [ PerformanceTests TFRecordIOIT HDFS ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml) | N/A |`Run Java TFRecordIO Performance Test HDFS`| [![.github/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml) +| [ PerformanceTests TFRecordIOIT ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TFRecordIOIT.yml) | N/A |`Run Java TFRecordIO Performance Test`| [![.github/workflows/beam_PerformanceTests_TFRecordIOIT.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TFRecordIOIT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TFRecordIOIT.yml) +| [ PerformanceTests WordCountIT PythonVersions ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml) | ['3.8'] |`Run Python (matrix_element) WordCountIT Performance Test`| [![.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml) +| [ PerformanceTests XmlIOIT HDFS ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml) | N/A |`Run Java XmlIO Performance Test HDFS`| [![.github/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml) +| [ PerformanceTests XmlIOIT ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_XmlIOIT.yml) | N/A |`Run Java XmlIO Performance Test`| [![.github/workflows/beam_PerformanceTests_XmlIOIT.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_XmlIOIT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_XmlIOIT.yml) +| [ PerformanceTests xlang KafkaIO Python ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml) | N/A |`Run Python xlang KafkaIO Performance Test`| [![.github/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml) + +### LoadTests Jobs + +| Workflow name | Matrix | Trigger Phrase | Cron Status | +|:-------------:|:------:|:--------------:|:-----------:| +| [ LoadTests Go CoGBK Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml) | N/A |`Run LoadTests Go CoGBK Dataflow Batch`| [![.github/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml) +| [ LoadTests Go CoGBK Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_CoGBK_Flink_batch.yml) | N/A |`Run Load Tests Go CoGBK Flink Batch`| [![.github/workflows/beam_LoadTests_Go_CoGBK_Flink_batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_CoGBK_Flink_batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_CoGBK_Flink_batch.yml) +| [ LoadTests Go Combine Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_Combine_Dataflow_Batch.yml) | N/A |`Run Load Tests Go Combine Dataflow Batch`| [![.github/workflows/beam_LoadTests_Go_Combine_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_Combine_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_Combine_Dataflow_Batch.yml) +| [ LoadTests Go Combine Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml) | N/A |`Run Load Tests Go Combine Flink Batch`| [![.github/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml) +| [ LoadTests Go GBK Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml) | N/A |`Run Load Tests Go GBK Dataflow Batch`| [![.github/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml) +| [ LoadTests Go GBK Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml) | N/A |`Run Load Tests Go GBK Flink Batch`| [![.github/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml) +| [ LoadTests Go ParDo Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml) | N/A |`Run Load Tests Go ParDo Dataflow Batch`| [![.github/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml) +| [ LoadTests Go ParDo Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml) | N/A |`Run Load Tests Go ParDo Flink Batch`| [![.github/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml) +| [ LoadTests Go SideInput Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml) | N/A |`Run Load Tests Go SideInput Dataflow Batch`| [![.github/workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml) +| [ LoadTests Go SideInput Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml) | N/A |`Run Load Tests Go SideInput Flink Batch`| [![.github/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml) +| [ LoadTests Java CoGBK Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml) | N/A |`Run Load Tests Java CoGBK Dataflow Batch`| [![.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml) +| [ LoadTests Java CoGBK Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml) | N/A |`Run Load Tests Java CoGBK Dataflow Streaming`| [![.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml) +| [ LoadTests Java CoGBK Dataflow V2 Batch JavaVersions ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml) | ['11','17'] |`Run Load Tests Java (matrix_element) CoGBK Dataflow V2 Batch`| [![.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml) +| [ LoadTests Java CoGBK Dataflow V2 Streaming JavaVersions ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml) | ['11','17'] |`Run Load Tests Java (matrix_element) CoGBK Dataflow V2 Streaming`| [![.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml) +| [ LoadTests Java CoGBK SparkStructuredStreaming Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml) | N/A |`Run Load Tests Java CoGBK SparkStructuredStreaming Batch`| [![.github/workflows/beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml) +| [ LoadTests Java Combine Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml) | N/A |`Run Load Tests Java Combine Dataflow Batch`| [![.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml) +| [ LoadTests Java Combine Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_Combine_Dataflow_Streaming.yml) | N/A |`Run Load Tests Java Combine Dataflow Streaming`| [![.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_Combine_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_Combine_Dataflow_Streaming.yml) +| [ LoadTests Java Combine SparkStructuredStreaming Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml) | N/A |`Run Load Tests Java Combine SparkStructuredStreaming Batch`| [![.github/workflows/beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml) +| [ LoadTests Java GBK Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml) | N/A |`Run Load Tests Java GBK Dataflow Batch`| [![.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml) +| [ LoadTests Java GBK Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml) | N/A |`Run Load Tests Java GBK Dataflow Streaming`| [![.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml) +| [ LoadTests Java GBK Dataflow V2 Batch Java11 ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml) | N/A |`Run Load Tests Java 11 GBK Dataflow V2 Batch`| [![.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml) +| [ LoadTests Java GBK Dataflow V2 Batch Java17 ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml) | N/A |`Run Load Tests Java 17 GBK Dataflow V2 Batch`| [![.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml) +| [ LoadTests Java GBK Dataflow V2 Streaming Java11 ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml) | N/A |`Run Load Tests Java 11 GBK Dataflow V2 Streaming`| [![.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml) +| [ LoadTests Java GBK Dataflow V2 Streaming Java17 ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml) | N/A |`Run Load Tests Java 17 GBK Dataflow V2 Streaming`| [![.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml) +| [ LoadTests Java GBK Smoke ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Smoke.yml) | N/A |`Run Java Load Tests GBK Smoke`| [![.github/workflows/beam_LoadTests_Java_GBK_Smoke.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Smoke.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Smoke.yml) +| [ LoadTests Java GBK SparkStructuredStreaming Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch.yml) | N/A |`Run Load Tests Java GBK SparkStructuredStreaming Batch`| [![.github/workflows/beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch.yml) +| [ LoadTests Java ParDo Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml) | N/A |`Run Load Tests Java ParDo Dataflow Batch`| [![.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml) +| [ LoadTests Java ParDo Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml) | N/A |`Run Load Tests Java ParDo Dataflow Streaming`| [![.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml) +| [ LoadTests Java ParDo Dataflow V2 Batch JavaVersions ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions.yml) | ['11','17'] |`Run Load Tests Java (matrix_element) ParDo Dataflow V2 Batch`| [![.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions.yml) +| [ LoadTests Java ParDo Dataflow V2 Streaming JavaVersions ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions.yml) | ['11','17'] |`Run Load Tests Java (matrix_element) ParDo Dataflow V2 Streaming`| [![.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions.yml) +| [ LoadTests Java ParDo SparkStructuredStreaming Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml) | N/A |`Run Load Tests Java ParDo SparkStructuredStreaming Batch`| [![.github/workflows/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml) +| [ LoadTests Java Combine Smoke ](https://github.com/apache/beam/actions/workflows/beam_Java_LoadTests_Combine_Smoke.yml) | N/A | N/A | [![.github/workflows/beam_Java_LoadTests_Combine_Smoke.yml](https://github.com/apache/beam/actions/workflows/beam_Java_LoadTests_Combine_Smoke.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_Java_LoadTests_Combine_Smoke.yml) +| [ LoadTests Python CoGBK Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml) | N/A |`Run Load Tests Python CoGBK Dataflow Batch`| [![.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml) +| [ LoadTests Python CoGBK Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml) | N/A |`Run Load Tests Python CoGBK Dataflow Streaming`| [![.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml) +| [ LoadTests Python CoGBK Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml) | N/A |`Run Load Tests Python CoGBK Flink Batch`| [![.github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml) +| [ LoadTests Python Combine Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml) | N/A |`Run Load Tests Python Combine Dataflow Batch`| [![.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml) +| [ LoadTests Python Combine Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Dataflow_Streaming.yml) | N/A |`Run Load Tests Python Combine Dataflow Streaming`| [![.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Dataflow_Streaming.yml) +| [ LoadTests Python Combine Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml) | N/A |`Run Load Tests Python Combine Flink Batch`| [![.github/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml) +| [ LoadTests Python Combine Flink Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml) | N/A |`Run Load Tests Python Combine Flink Streaming`| [![.github/workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml) +| [ LoadTests Python FnApiRunner Microbenchmark ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_FnApiRunner_Microbenchmark.yml) | N/A |`Run Python Load Tests FnApiRunner Microbenchmark`| [![.github/workflows/beam_LoadTests_Python_FnApiRunner_Microbenchmark](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_FnApiRunner_Microbenchmark.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_FnApiRunner_Microbenchmark.yml) +| [ LoadTests Python GBK Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml) | N/A |`Run Load Tests Python GBK Dataflow Batch`| [![.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml) +| [ LoadTests Python GBK Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml) | N/A |`Run Load Tests Python GBK Dataflow Streaming`| [![.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml) +| [ LoadTests Python GBK Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_Flink_Batch.yml) | N/A |`Run Load Tests Python GBK Flink Batch`| [![.github/workflows/beam_LoadTests_Python_GBK_Flink_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_Flink_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_Flink_Batch.yml) +| [ LoadTests Python GBK reiterate Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Batch.yml) | N/A |`Run Load Tests Python GBK reiterate Dataflow Batch`| [![.github/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Batch.yml) +| [ LoadTests Python GBK reiterate Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Streaming.yml) | N/A |`Run Load Tests Python GBK reiterate Dataflow Streaming`| [![.github/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Streaming.yml) +| [ LoadTests Python ParDo Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Dataflow_Batch.yml) | N/A |`Run Load Tests Python ParDo Dataflow Batch`| [![.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Dataflow_Batch.yml) +| [ LoadTests Python ParDo Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml) | N/A |`Run Python Load Tests ParDo Dataflow Streaming`| [![.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml) +| [ LoadTests Python ParDo Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml) | N/A |`Run Load Tests Python ParDo Flink Batch`| [![.github/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml) +| [ LoadTests Python ParDo Flink Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml) | N/A |`Run Load Tests Python ParDo Flink Streaming`| [![.github/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml) +| [ LoadTests Python SideInput Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_SideInput_Dataflow_Batch.yml) | N/A |`Run Load Tests Python SideInput Dataflow Batch`| [![.github/workflows/beam_LoadTests_Python_SideInput_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_SideInput_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_SideInput_Dataflow_Batch.yml) +| [ LoadTests Python Smoke ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Smoke.yml) | N/A |`Run Python Load Tests Smoke`| [![.github/workflows/beam_LoadTests_Python_Smoke.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Smoke.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Smoke.yml) + +### Other Jobs + +| Workflow name | Matrix | Trigger Phrase | Cron Status | +|:-------------:|:------:|:--------------:|:-----------:| +| [ Cancel Stale Dataflow Jobs ](https://github.com/apache/beam/actions/workflows/beam_CancelStaleDataflowJobs.yml) | N/A | `Run Cancel Stale Dataflow Jobs` | [![.github/workflows/beam_CancelStaleDataflowJobs.yml](https://github.com/apache/beam/actions/workflows/beam_CancelStaleDataflowJobs.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_CancelStaleDataflowJobs.yml) | +| [ Clean Up GCP Resources ](https://github.com/apache/beam/actions/workflows/beam_CleanUpGCPResources.yml) | N/A | `Run Clean GCP Resources` | [![.github/workflows/beam_CleanUpGCPResources.yml](https://github.com/apache/beam/actions/workflows/beam_CleanUpGCPResources.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_CleanUpGCPResources.yml) | +| [ Clean Up Prebuilt SDK Images ](https://github.com/apache/beam/actions/workflows/beam_CleanUpPrebuiltSDKImages.yml) | N/A | `Run Clean Prebuilt Images` | [![.github/workflows/beam_beam_CleanUpPrebuiltSDKImages.yml](https://github.com/apache/beam/actions/workflows/beam_CleanUpPrebuiltSDKImages.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_CleanUpPrebuiltSDKImages.yml) | +| [ Cleanup Dataproc Resources ](https://github.com/apache/beam/actions/workflows/beam_CleanUpDataprocResources.yml) | N/A | N/A | [![.github/workflows/beam_CleanUpDataprocResources.yml](https://github.com/apache/beam/actions/workflows/beam_CleanUpDataprocResources.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_CleanUpDataprocResources.yml) +| [ Community Metrics Prober ](https://github.com/apache/beam/actions/workflows/beam_Prober_CommunityMetrics.yml) | N/A |`Run Community Metrics Prober`| [![.github/workflows/beam_Prober_CommunityMetrics.yml](https://github.com/apache/beam/actions/workflows/beam_Prober_CommunityMetrics.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_Prober_CommunityMetrics.yml) +| [ Publish Docker Snapshots ](https://github.com/apache/beam/actions/workflows/beam_Publish_Docker_Snapshots.yml) | N/A |`Publish Docker Snapshots`| [![.github/workflows/beam_Publish_Docker_Snapshots.yml](https://github.com/apache/beam/actions/workflows/beam_Publish_Docker_Snapshots.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_Publish_Docker_Snapshots.yml) | | [ Rotate IO-Datastores Cluster Credentials ](https://github.com/apache/beam/actions/workflows/beam_IODatastoresCredentialsRotation.yml) | N/A | N/A | [![.github/workflows/beam_IODatastoresCredentialsRotation.yml](https://github.com/apache/beam/actions/workflows/beam_IODatastoresCredentialsRotation.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_IODatastoresCredentialsRotation.yml) | | [ Rotate Metrics Cluster Credentials ](https://github.com/apache/beam/actions/workflows/beam_MetricsCredentialsRotation.yml) | N/A | N/A | [![.github/workflows/beam_MetricsCredentialsRotation.yml](https://github.com/apache/beam/actions/workflows/beam_MetricsCredentialsRotation.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_MetricsCredentialsRotation.yml) | diff --git a/.github/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml index dac3733da4a9..61758111f597 100644 --- a/.github/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml @@ -13,7 +13,7 @@ # See the License for the specific language governing permissions and # limitations under the License. -name: Load Tests CoGBK Dataflow Batch Go +name: LoadTests Go CoGBK Dataflow Batch on: schedule: diff --git a/.github/workflows/beam_LoadTests_Go_Combine_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Go_Combine_Dataflow_Batch.yml index cb198bbc233f..7fd8b8bcf813 100644 --- a/.github/workflows/beam_LoadTests_Go_Combine_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_Combine_Dataflow_Batch.yml @@ -13,7 +13,7 @@ # See the License for the specific language governing permissions and # limitations under the License. -name: Load Tests Combine Dataflow Batch Go +name: LoadTests Go Combine Dataflow Batch on: schedule: diff --git a/.github/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml index 9778b542bacb..fbabc8e46827 100644 --- a/.github/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml @@ -13,7 +13,7 @@ # See the License for the specific language governing permissions and # limitations under the License. -name: Load Tests GBK Dataflow Batch Go +name: LoadTests Go GBK Dataflow Batch on: schedule: diff --git a/.github/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml index 99292c88757a..4da33c5c9235 100644 --- a/.github/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml @@ -13,7 +13,7 @@ # See the License for the specific language governing permissions and # limitations under the License. -name: Load Tests ParDo Dataflow Batch Go +name: LoadTests Go ParDo Dataflow Batch on: schedule: diff --git a/.github/workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml index 123e839d46bb..876aaaab2ed4 100644 --- a/.github/workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml @@ -13,7 +13,7 @@ # See the License for the specific language governing permissions and # limitations under the License. -name: Load Tests SideInput Dataflow Batch Go +name: LoadTests Go SideInput Dataflow Batch on: schedule: diff --git a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml index bdd0adadb11d..ed1ae6b61e48 100644 --- a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml @@ -13,7 +13,7 @@ # See the License for the specific language governing permissions and # limitations under the License. -name: Load Tests CoGBK Dataflow Streaming Java +name: LoadTests Java CoGBK Dataflow Streaming on: schedule: diff --git a/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml index 02f5da33afaf..8c91375b47d1 100644 --- a/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml @@ -13,7 +13,7 @@ # See the License for the specific language governing permissions and # limitations under the License. -name: Load Tests Combine Dataflow Batch Java +name: LoadTests Java Combine Dataflow Batch on: schedule: diff --git a/.github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml index cdf0c0953165..d29f27fb86f9 100644 --- a/.github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml @@ -13,7 +13,7 @@ # See the License for the specific language governing permissions and # limitations under the License. -name: LoadTests Python CoGBK Dataflow Flink Batch +name: LoadTests Python CoGBK Flink Batch on: schedule: diff --git a/.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml index 2a1220c3b885..28bba715f4b5 100644 --- a/.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml @@ -13,7 +13,7 @@ # See the License for the specific language governing permissions and # limitations under the License. -name: Load Tests Combine Dataflow Batch Python +name: LoadTests Python Combine Dataflow Batch on: schedule: diff --git a/.github/workflows/beam_LoadTests_Python_FnApiRunner_Microbenchmark.yml b/.github/workflows/beam_LoadTests_Python_FnApiRunner_Microbenchmark.yml index 674f37fd3481..8d111a9f4e64 100644 --- a/.github/workflows/beam_LoadTests_Python_FnApiRunner_Microbenchmark.yml +++ b/.github/workflows/beam_LoadTests_Python_FnApiRunner_Microbenchmark.yml @@ -13,7 +13,7 @@ # See the License for the specific language governing permissions and # limitations under the License. -name: Load Tests FnApiRunner Microbenchmark Python +name: LoadTests Python FnApiRunner Microbenchmark on: schedule: diff --git a/.github/workflows/beam_PerformanceTests_AvroIOIT.yml b/.github/workflows/beam_PerformanceTests_AvroIOIT.yml index 011f8a740fbb..4c6ac900c544 100644 --- a/.github/workflows/beam_PerformanceTests_AvroIOIT.yml +++ b/.github/workflows/beam_PerformanceTests_AvroIOIT.yml @@ -13,7 +13,7 @@ # See the License for the specific language governing permissions and # limitations under the License. -name: Performance Tests AvroIOIT +name: PerformanceTests AvroIOIT on: schedule: diff --git a/.github/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml b/.github/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml index c85c8d481434..d3f881e9e07b 100644 --- a/.github/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml +++ b/.github/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml @@ -13,7 +13,7 @@ # See the License for the specific language governing permissions and # limitations under the License. -name: Performance Tests AvroIOIT HDFS +name: PerformanceTests AvroIOIT HDFS on: schedule: diff --git a/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml b/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml index 0f5f69dcd320..599e1804011f 100644 --- a/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml +++ b/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml @@ -13,7 +13,7 @@ # See the License for the specific language governing permissions and # limitations under the License. -name: Performance Tests BigQueryIO Batch Java Avro +name: PerformanceTests BigQueryIO Batch Java Avro on: schedule: diff --git a/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml b/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml index df2f406c4294..a4811864164c 100644 --- a/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml +++ b/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml @@ -13,7 +13,7 @@ # See the License for the specific language governing permissions and # limitations under the License. -name: Performance Tests BigQueryIO Batch Java Json +name: PerformanceTests BigQueryIO Batch Java Json on: schedule: diff --git a/.github/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml b/.github/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml index cc628012f1a4..8d0b3adb9141 100644 --- a/.github/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml +++ b/.github/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml @@ -13,7 +13,7 @@ # See the License for the specific language governing permissions and # limitations under the License. -name: Performance Tests BigQueryIO Streaming Java +name: PerformanceTests BigQueryIO Streaming Java on: schedule: diff --git a/.github/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml b/.github/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml index fe0cfaf578ec..4147d1099c8d 100644 --- a/.github/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml +++ b/.github/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml @@ -15,7 +15,7 @@ # specific language governing permissions and limitations # under the License. -name: Java InfluxDbIO Integration Test +name: PostCommit Java InfluxDbIO Integration Test on: schedule: From 189f531c2148fa1fb906d253eaf1e66aaa6a7ff6 Mon Sep 17 00:00:00 2001 From: Yi Hu <yathu@google.com> Date: Mon, 23 Oct 2023 10:17:44 -0400 Subject: [PATCH 227/435] Publish Java21 container for Publish SDK container GitHub Action (#29103) --- .github/workflows/beam_Publish_Beam_SDK_Snapshots.yml | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/.github/workflows/beam_Publish_Beam_SDK_Snapshots.yml b/.github/workflows/beam_Publish_Beam_SDK_Snapshots.yml index 9bc268bbe9c2..9e48325b89ab 100644 --- a/.github/workflows/beam_Publish_Beam_SDK_Snapshots.yml +++ b/.github/workflows/beam_Publish_Beam_SDK_Snapshots.yml @@ -60,7 +60,16 @@ jobs: matrix: job_name: ["beam_Publish_Beam_SDK_Snapshots"] job_phrase: ["N/A"] - container_task: ["go:container", "java:container:java8", "java:container:java11", "java:container:java17", "python:container:py38", "python:container:py39", "python:container:py310", "python:container:py311"] + container_task: + - "go:container" + - "java:container:java8" + - "java:container:java11" + - "java:container:java17" + - "java:container:java21" + - "python:container:py38" + - "python:container:py39" + - "python:container:py310" + - "python:container:py311" steps: - uses: actions/checkout@v3 - name: Setup repository From ca96cb5ea4220b4fe5d59cc51c750610f71974fc Mon Sep 17 00:00:00 2001 From: Aleksandr Dudko <116064902+aleksandr-dudko@users.noreply.github.com> Date: Mon, 23 Oct 2023 18:49:26 +0400 Subject: [PATCH 228/435] Add Setup environment actions (#29026) Co-authored-by: Andrey Devyatkin <andrey.9evyatkin@gmail.com> --- .github/actions/setup-action/action.yml | 4 +--- .../beam_CancelStaleDataflowJobs.yml | 2 ++ .../workflows/beam_CleanUpGCPResources.yml | 2 ++ .../beam_CleanUpPrebuiltSDKImages.yml | 2 ++ .../beam_IODatastoresCredentialsRotation.yml | 2 ++ .github/workflows/beam_Java_JMH.yml | 2 ++ .../beam_Java_LoadTests_Combine_Smoke.yml | 2 ++ ...beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml | 2 ++ .../beam_LoadTests_Go_CoGBK_Flink_batch.yml | 2 ++ ...am_LoadTests_Go_Combine_Dataflow_Batch.yml | 2 ++ .../beam_LoadTests_Go_GBK_Dataflow_Batch.yml | 2 ++ .../beam_LoadTests_Go_GBK_Flink_Batch.yml | 2 ++ ...beam_LoadTests_Go_ParDo_Dataflow_Batch.yml | 2 ++ .../beam_LoadTests_Go_ParDo_Flink_Batch.yml | 2 ++ ..._LoadTests_Go_SideInput_Dataflow_Batch.yml | 2 ++ ...eam_LoadTests_Go_SideInput_Flink_Batch.yml | 2 ++ ...am_LoadTests_Java_CoGBK_Dataflow_Batch.yml | 2 ++ ...oadTests_Java_CoGBK_Dataflow_Streaming.yml | 2 ++ ...a_CoGBK_SparkStructuredStreaming_Batch.yml | 2 ++ ..._LoadTests_Java_Combine_Dataflow_Batch.yml | 2 ++ ...dTests_Java_Combine_Dataflow_Streaming.yml | 2 ++ ...Combine_SparkStructuredStreaming_Batch.yml | 2 ++ ...beam_LoadTests_Java_GBK_Dataflow_Batch.yml | 2 ++ ..._LoadTests_Java_GBK_Dataflow_Streaming.yml | 2 ++ .../beam_LoadTests_Java_GBK_Smoke.yml | 2 ++ ...ava_GBK_SparkStructuredStreaming_Batch.yml | 2 ++ ...am_LoadTests_Java_ParDo_Dataflow_Batch.yml | 2 ++ ...oadTests_Java_ParDo_Dataflow_Streaming.yml | 2 ++ ...a_ParDo_SparkStructuredStreaming_Batch.yml | 2 ++ ...eam_LoadTests_Python_CoGBK_Flink_Batch.yml | 6 +++--- ...oadTests_Python_Combine_Dataflow_Batch.yml | 6 +++--- ...ests_Python_FnApiRunner_Microbenchmark.yml | 6 +++--- .../beam_MetricsCredentialsRotation.yml | 2 ++ .../beam_PerformanceTests_AvroIOIT.yml | 2 ++ .../beam_PerformanceTests_AvroIOIT_HDFS.yml | 2 ++ ...rmanceTests_BigQueryIO_Batch_Java_Avro.yml | 2 ++ ...rmanceTests_BigQueryIO_Batch_Java_Json.yml | 2 ++ ...ormanceTests_BigQueryIO_Streaming_Java.yml | 2 ++ .../workflows/beam_PerformanceTests_Cdap.yml | 2 ++ ...m_PerformanceTests_Compressed_TextIOIT.yml | 2 ++ ...formanceTests_Compressed_TextIOIT_HDFS.yml | 2 ++ .../beam_PerformanceTests_HadoopFormat.yml | 2 ++ .../workflows/beam_PerformanceTests_JDBC.yml | 2 ++ .../beam_PerformanceTests_Kafka_IO.yml | 2 ++ ...am_PerformanceTests_ManyFiles_TextIOIT.yml | 2 ++ ...rformanceTests_ManyFiles_TextIOIT_HDFS.yml | 2 ++ .../beam_PerformanceTests_MongoDBIO_IT.yml | 2 ++ .../beam_PerformanceTests_ParquetIOIT.yml | 2 ++ ...beam_PerformanceTests_ParquetIOIT_HDFS.yml | 2 ++ ...beam_PerformanceTests_SparkReceiver_IO.yml | 2 ++ .../beam_PerformanceTests_TFRecordIOIT.yml | 2 ++ ...eam_PerformanceTests_TFRecordIOIT_HDFS.yml | 2 ++ .../beam_PerformanceTests_TextIOIT.yml | 2 ++ .../beam_PerformanceTests_TextIOIT_HDFS.yml | 2 ++ .../beam_PerformanceTests_XmlIOIT.yml | 2 ++ .../beam_PerformanceTests_XmlIOIT_HDFS.yml | 2 ++ .../beam_PostCommit_BeamMetrics_Publish.yml | 2 ++ .github/workflows/beam_PostCommit_Go.yml | 2 ++ .../workflows/beam_PostCommit_Go_VR_Flink.yml | 2 ++ .../workflows/beam_PostCommit_Go_VR_Spark.yml | 2 ++ .github/workflows/beam_PostCommit_Java.yml | 2 ++ .../beam_PostCommit_Java_Avro_Versions.yml | 2 ++ ...m_PostCommit_Java_BigQueryEarlyRollout.yml | 2 ++ .../beam_PostCommit_Java_DataflowV1.yml | 5 ++--- .../beam_PostCommit_Java_DataflowV2.yml | 2 ++ ...beam_PostCommit_Java_Examples_Dataflow.yml | 2 ++ ...PostCommit_Java_Examples_Dataflow_Java.yml | 5 ++--- ...m_PostCommit_Java_Examples_Dataflow_V2.yml | 2 ++ ...tCommit_Java_Examples_Dataflow_V2_Java.yml | 5 ++--- .../beam_PostCommit_Java_Examples_Direct.yml | 7 +++---- .../beam_PostCommit_Java_Examples_Spark.yml | 7 +++---- .../beam_PostCommit_Java_Hadoop_Versions.yml | 7 +++---- .../beam_PostCommit_Java_InfluxDbIO_IT.yml | 2 ++ ...m_PostCommit_Java_Jpms_Dataflow_Java11.yml | 7 +++---- ...m_PostCommit_Java_Jpms_Dataflow_Java17.yml | 5 ++--- ...eam_PostCommit_Java_Jpms_Direct_Java11.yml | 7 +++---- ...eam_PostCommit_Java_Jpms_Direct_Java17.yml | 5 ++--- ...beam_PostCommit_Java_Jpms_Flink_Java11.yml | 7 +++---- ...beam_PostCommit_Java_Jpms_Spark_Java11.yml | 7 +++---- .../beam_PostCommit_Java_Nexmark_Dataflow.yml | 2 ++ ...am_PostCommit_Java_Nexmark_Dataflow_V2.yml | 2 ++ ...stCommit_Java_Nexmark_Dataflow_V2_Java.yml | 5 ++--- .../beam_PostCommit_Java_Nexmark_Direct.yml | 2 ++ .../beam_PostCommit_Java_Nexmark_Flink.yml | 2 ++ .../beam_PostCommit_Java_Nexmark_Spark.yml | 2 ++ ...am_PostCommit_Java_PVR_Flink_Streaming.yml | 2 ++ .../beam_PostCommit_Java_PVR_Samza.yml | 2 ++ ...m_PostCommit_Java_PVR_Spark3_Streaming.yml | 2 ++ .../beam_PostCommit_Java_Sickbay.yml | 2 ++ .../beam_PostCommit_Java_SingleStoreIO_IT.yml | 2 ++ .../beam_PostCommit_Java_Tpcds_Dataflow.yml | 2 ++ .../beam_PostCommit_Java_Tpcds_Flink.yml | 2 ++ .../beam_PostCommit_Java_Tpcds_Spark.yml | 2 ++ ...stCommit_Java_ValidatesRunner_Dataflow.yml | 7 +++---- ..._ValidatesRunner_Dataflow_JavaVersions.yml | 5 ++--- ...ava_ValidatesRunner_Dataflow_Streaming.yml | 7 +++---- ...ommit_Java_ValidatesRunner_Dataflow_V2.yml | 7 +++---- ..._ValidatesRunner_Dataflow_V2_Streaming.yml | 7 +++---- ...PostCommit_Java_ValidatesRunner_Direct.yml | 7 +++---- ...va_ValidatesRunner_Direct_JavaVersions.yml | 5 ++--- ...mmit_Java_ValidatesRunner_Flink_Java11.yml | 5 ++--- ..._PostCommit_Java_ValidatesRunner_Samza.yml | 7 +++---- ..._PostCommit_Java_ValidatesRunner_Spark.yml | 7 +++---- ...lidatesRunner_SparkStructuredStreaming.yml | 7 +++---- ...mmit_Java_ValidatesRunner_Spark_Java11.yml | 5 ++--- ...stCommit_Java_ValidatesRunner_Twister2.yml | 7 +++---- ...am_PostCommit_Java_ValidatesRunner_ULR.yml | 12 ++++-------- .github/workflows/beam_PostCommit_Javadoc.yml | 2 ++ .../beam_PostCommit_PortableJar_Flink.yml | 4 ++-- .../beam_PostCommit_PortableJar_Spark.yml | 4 ++-- .github/workflows/beam_PostCommit_Python.yml | 4 ++-- .../workflows/beam_PostCommit_Python_Arm.yml | 4 ++-- .../beam_PostCommit_Python_Nexmark_Direct.yml | 4 ++-- ...m_PostCommit_Python_Xlang_Gcp_Dataflow.yml | 4 ++-- ...eam_PostCommit_Python_Xlang_Gcp_Direct.yml | 4 ++-- ...am_PostCommit_Python_Xlang_IO_Dataflow.yml | 4 ++-- .github/workflows/beam_PostCommit_SQL.yml | 2 ++ .../beam_PostCommit_Sickbay_Python.yml | 4 ++-- ...eam_PostCommit_TransformService_Direct.yml | 10 +++------- .../beam_PostCommit_Website_Publish.yml | 2 ++ .../beam_PostCommit_Website_Test.yml | 2 ++ .../workflows/beam_PostCommit_XVR_Direct.yml | 4 ++-- .../workflows/beam_PostCommit_XVR_Flink.yml | 4 ++-- ...am_PostCommit_XVR_GoUsingJava_Dataflow.yml | 6 +++--- ...ostCommit_XVR_JavaUsingPython_Dataflow.yml | 4 ++-- ...Commit_XVR_PythonUsingJavaSQL_Dataflow.yml | 4 ++-- ...ostCommit_XVR_PythonUsingJava_Dataflow.yml | 4 ++-- .../workflows/beam_PostCommit_XVR_Samza.yml | 4 ++-- .../workflows/beam_PostCommit_XVR_Spark3.yml | 4 ++-- .github/workflows/beam_PreCommit_Java.yml | 2 ++ ...it_Java_Amazon-Web-Services2_IO_Direct.yml | 2 ++ ...mit_Java_Amazon-Web-Services_IO_Direct.yml | 2 ++ .../beam_PreCommit_Java_Amqp_IO_Direct.yml | 2 ++ .../beam_PreCommit_Java_Azure_IO_Direct.yml | 2 ++ ...eam_PreCommit_Java_Cassandra_IO_Direct.yml | 2 ++ .../beam_PreCommit_Java_Cdap_IO_Direct.yml | 2 ++ ...am_PreCommit_Java_Clickhouse_IO_Direct.yml | 2 ++ .../beam_PreCommit_Java_Csv_IO_Direct.yml | 2 ++ ...beam_PreCommit_Java_Debezium_IO_Direct.yml | 2 ++ ...PreCommit_Java_ElasticSearch_IO_Direct.yml | 2 ++ ...reCommit_Java_Examples_Dataflow_Java11.yml | 7 +++---- ...reCommit_Java_Examples_Dataflow_Java17.yml | 5 ++--- ...t_Java_File-schema-transform_IO_Direct.yml | 2 ++ .../beam_PreCommit_Java_GCP_IO_Direct.yml | 2 ++ .../beam_PreCommit_Java_HBase_IO_Direct.yml | 2 ++ ...beam_PreCommit_Java_HCatalog_IO_Direct.yml | 2 ++ .../beam_PreCommit_Java_Hadoop_IO_Direct.yml | 2 ++ .../beam_PreCommit_Java_IOs_Direct.yml | 2 ++ ...beam_PreCommit_Java_InfluxDb_IO_Direct.yml | 2 ++ .../beam_PreCommit_Java_JDBC_IO_Direct.yml | 2 ++ .../beam_PreCommit_Java_Jms_IO_Direct.yml | 2 ++ .../beam_PreCommit_Java_Kafka_IO_Direct.yml | 2 ++ .../beam_PreCommit_Java_Kinesis_IO_Direct.yml | 2 ++ .../beam_PreCommit_Java_Kudu_IO_Direct.yml | 2 ++ .../beam_PreCommit_Java_MongoDb_IO_Direct.yml | 2 ++ .../beam_PreCommit_Java_Mqtt_IO_Direct.yml | 2 ++ .../beam_PreCommit_Java_Neo4j_IO_Direct.yml | 2 ++ .../beam_PreCommit_Java_PVR_Flink_Batch.yml | 2 ++ .../beam_PreCommit_Java_PVR_Flink_Docker.yml | 2 ++ .../beam_PreCommit_Java_Parquet_IO_Direct.yml | 2 ++ .../beam_PreCommit_Java_Pulsar_IO_Direct.yml | 2 ++ ...beam_PreCommit_Java_RabbitMq_IO_Direct.yml | 2 ++ .../beam_PreCommit_Java_Redis_IO_Direct.yml | 2 ++ ...m_PreCommit_Java_SingleStore_IO_Direct.yml | 2 ++ ...eam_PreCommit_Java_Snowflake_IO_Direct.yml | 2 ++ .../beam_PreCommit_Java_Solr_IO_Direct.yml | 2 ++ .../beam_PreCommit_Java_Spark3_Versions.yml | 9 +++------ .../beam_PreCommit_Java_Splunk_IO_Direct.yml | 2 ++ .../beam_PreCommit_Java_Thrift_IO_Direct.yml | 2 ++ .../beam_PreCommit_Java_Tika_IO_Direct.yml | 2 ++ .../beam_PreCommit_Kotlin_Examples.yml | 9 +++------ .../beam_PreCommit_Portable_Python.yml | 12 +++--------- .../beam_PreCommit_Python_PVR_Flink.yml | 4 ++-- .github/workflows/beam_PreCommit_SQL.yml | 7 +++---- .../workflows/beam_PreCommit_SQL_Java11.yml | 17 +++++------------ .../workflows/beam_PreCommit_SQL_Java17.yml | 19 ++++++------------- .github/workflows/beam_PreCommit_Spotless.yml | 2 ++ .../beam_Prober_CommunityMetrics.yml | 2 ++ .../beam_Publish_Docker_Snapshots.yml | 2 ++ 179 files changed, 392 insertions(+), 215 deletions(-) diff --git a/.github/actions/setup-action/action.yml b/.github/actions/setup-action/action.yml index da69dd9a97dd..743e89a931da 100644 --- a/.github/actions/setup-action/action.yml +++ b/.github/actions/setup-action/action.yml @@ -69,6 +69,4 @@ runs: - name: expose gcloud path shell: bash run: | - echo KUBELET_GCLOUD_CONFIG_PATH=/var/lib/kubelet/pods/$POD_UID/volumes/kubernetes.io~empty-dir/gcloud >> $GITHUB_ENV - - name: Setup environment - uses: ./.github/actions/setup-environment-action + echo KUBELET_GCLOUD_CONFIG_PATH=/var/lib/kubelet/pods/$POD_UID/volumes/kubernetes.io~empty-dir/gcloud >> $GITHUB_ENV \ No newline at end of file diff --git a/.github/workflows/beam_CancelStaleDataflowJobs.yml b/.github/workflows/beam_CancelStaleDataflowJobs.yml index 63e780c2fef5..46ff76df6b51 100644 --- a/.github/workflows/beam_CancelStaleDataflowJobs.yml +++ b/.github/workflows/beam_CancelStaleDataflowJobs.yml @@ -69,6 +69,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: Authenticate on GCP id: auth uses: google-github-actions/auth@v1 diff --git a/.github/workflows/beam_CleanUpGCPResources.yml b/.github/workflows/beam_CleanUpGCPResources.yml index 9aa92f0003c4..a7267ad1e0e0 100644 --- a/.github/workflows/beam_CleanUpGCPResources.yml +++ b/.github/workflows/beam_CleanUpGCPResources.yml @@ -69,6 +69,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: Authenticate on GCP id: auth uses: google-github-actions/auth@v1 diff --git a/.github/workflows/beam_CleanUpPrebuiltSDKImages.yml b/.github/workflows/beam_CleanUpPrebuiltSDKImages.yml index 345624f063bb..f327f09c0118 100644 --- a/.github/workflows/beam_CleanUpPrebuiltSDKImages.yml +++ b/.github/workflows/beam_CleanUpPrebuiltSDKImages.yml @@ -69,6 +69,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: Authenticate on GCP id: auth uses: google-github-actions/auth@v1 diff --git a/.github/workflows/beam_IODatastoresCredentialsRotation.yml b/.github/workflows/beam_IODatastoresCredentialsRotation.yml index 36e6b238cdfc..7a402e426fa7 100644 --- a/.github/workflows/beam_IODatastoresCredentialsRotation.yml +++ b/.github/workflows/beam_IODatastoresCredentialsRotation.yml @@ -66,6 +66,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: Starting credential rotation run: | gcloud container clusters update io-datastores --start-credential-rotation --zone=us-central1-a --quiet diff --git a/.github/workflows/beam_Java_JMH.yml b/.github/workflows/beam_Java_JMH.yml index 07beb1dadba1..a25c3fafaf99 100644 --- a/.github/workflows/beam_Java_JMH.yml +++ b/.github/workflows/beam_Java_JMH.yml @@ -58,6 +58,8 @@ jobs: name: "beam_Java_JMH" steps: - uses: actions/checkout@v3 + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: run the Java JMH micro-benchmark harness suite uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_Java_LoadTests_Combine_Smoke.yml b/.github/workflows/beam_Java_LoadTests_Combine_Smoke.yml index 19d5b0f7174f..b252f4e7b5c2 100644 --- a/.github/workflows/beam_Java_LoadTests_Combine_Smoke.yml +++ b/.github/workflows/beam_Java_LoadTests_Combine_Smoke.yml @@ -69,6 +69,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: Prepare test arguments uses: ./.github/actions/test-arguments-action with: diff --git a/.github/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml index 61758111f597..e7e5b90107d2 100644 --- a/.github/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml @@ -69,6 +69,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: Prepare test arguments uses: ./.github/actions/test-arguments-action with: diff --git a/.github/workflows/beam_LoadTests_Go_CoGBK_Flink_batch.yml b/.github/workflows/beam_LoadTests_Go_CoGBK_Flink_batch.yml index 05b61f285cd7..feec436bcd57 100644 --- a/.github/workflows/beam_LoadTests_Go_CoGBK_Flink_batch.yml +++ b/.github/workflows/beam_LoadTests_Go_CoGBK_Flink_batch.yml @@ -79,6 +79,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: Prepare test arguments uses: ./.github/actions/test-arguments-action with: diff --git a/.github/workflows/beam_LoadTests_Go_Combine_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Go_Combine_Dataflow_Batch.yml index 7fd8b8bcf813..93b65ac99c6c 100644 --- a/.github/workflows/beam_LoadTests_Go_Combine_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_Combine_Dataflow_Batch.yml @@ -69,6 +69,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: Prepare test arguments uses: ./.github/actions/test-arguments-action with: diff --git a/.github/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml index fbabc8e46827..476338d3df71 100644 --- a/.github/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml @@ -69,6 +69,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: Prepare test arguments uses: ./.github/actions/test-arguments-action with: diff --git a/.github/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml index 37ae1e419c59..6f17a4ba6bb3 100644 --- a/.github/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml @@ -79,6 +79,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: Prepare test arguments uses: ./.github/actions/test-arguments-action with: diff --git a/.github/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml index 4da33c5c9235..e5b33baf568c 100644 --- a/.github/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml @@ -69,6 +69,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: Prepare test arguments uses: ./.github/actions/test-arguments-action with: diff --git a/.github/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml index 666799bfc655..ba544420a12f 100644 --- a/.github/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml @@ -79,6 +79,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: Prepare test arguments uses: ./.github/actions/test-arguments-action with: diff --git a/.github/workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml index 876aaaab2ed4..439bb47fbd06 100644 --- a/.github/workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml @@ -69,6 +69,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: Prepare test arguments uses: ./.github/actions/test-arguments-action with: diff --git a/.github/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml index 592af7e86676..5be573e0e0b2 100644 --- a/.github/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml @@ -79,6 +79,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: Prepare test arguments uses: ./.github/actions/test-arguments-action with: diff --git a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml index 8677f6f81f71..27e6a8ad912a 100644 --- a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml @@ -69,6 +69,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: Prepare test arguments uses: ./.github/actions/test-arguments-action with: diff --git a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml index ed1ae6b61e48..27cc98379bcf 100644 --- a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml @@ -69,6 +69,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: Prepare test arguments uses: ./.github/actions/test-arguments-action with: diff --git a/.github/workflows/beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml b/.github/workflows/beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml index 7f1935267e95..b77c0aefe95c 100644 --- a/.github/workflows/beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml +++ b/.github/workflows/beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml @@ -69,6 +69,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: Prepare test arguments uses: ./.github/actions/test-arguments-action with: diff --git a/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml index 8c91375b47d1..91ccdce3f397 100644 --- a/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml @@ -69,6 +69,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: Prepare test arguments uses: ./.github/actions/test-arguments-action with: diff --git a/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Streaming.yml index d6520646fcdd..243cb94d9d66 100644 --- a/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Streaming.yml @@ -69,6 +69,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: Prepare test arguments uses: ./.github/actions/test-arguments-action with: diff --git a/.github/workflows/beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml b/.github/workflows/beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml index d4862ad45c07..dcf6015f62a2 100644 --- a/.github/workflows/beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml +++ b/.github/workflows/beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml @@ -69,6 +69,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: Prepare test arguments uses: ./.github/actions/test-arguments-action with: diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml index 403deba2f0a5..25a48781e083 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml @@ -69,6 +69,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: Prepare test arguments uses: ./.github/actions/test-arguments-action with: diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml index 483c06cf821a..9d8ccf279a1c 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml @@ -69,6 +69,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: Prepare test arguments uses: ./.github/actions/test-arguments-action with: diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Smoke.yml b/.github/workflows/beam_LoadTests_Java_GBK_Smoke.yml index 251291163efe..cf31693a7884 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_Smoke.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_Smoke.yml @@ -66,6 +66,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: Prepare test arguments uses: ./.github/actions/test-arguments-action with: diff --git a/.github/workflows/beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch.yml b/.github/workflows/beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch.yml index de66760e5298..eeba909a13e9 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch.yml @@ -69,6 +69,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: Prepare test arguments uses: ./.github/actions/test-arguments-action with: diff --git a/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml index ebc30fb0efa4..5f2eaabae5d5 100644 --- a/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml @@ -69,6 +69,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: Prepare test arguments uses: ./.github/actions/test-arguments-action with: diff --git a/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml index 6dec9beca4bf..8279643c0d2d 100644 --- a/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml @@ -69,6 +69,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: Prepare test arguments uses: ./.github/actions/test-arguments-action with: diff --git a/.github/workflows/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml b/.github/workflows/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml index 26c1c88dd893..8e379324fc02 100644 --- a/.github/workflows/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml +++ b/.github/workflows/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml @@ -69,6 +69,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: Prepare test arguments uses: ./.github/actions/test-arguments-action with: diff --git a/.github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml index d29f27fb86f9..87de130bd4ed 100644 --- a/.github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml @@ -79,10 +79,10 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - - name: Install Python - uses: actions/setup-python@v4 + - name: Setup environment + uses: ./.github/actions/setup-environment-action with: - python-version: '3.8' + python-version: 3.8 - name: Prepare test arguments uses: ./.github/actions/test-arguments-action with: diff --git a/.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml index 28bba715f4b5..3aa9b35679e2 100644 --- a/.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml @@ -69,10 +69,10 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - - name: Install Python - uses: actions/setup-python@v4 + - name: Setup environment + uses: ./.github/actions/setup-environment-action with: - python-version: '3.8' + python-version: 3.8 - name: Prepare test arguments uses: ./.github/actions/test-arguments-action with: diff --git a/.github/workflows/beam_LoadTests_Python_FnApiRunner_Microbenchmark.yml b/.github/workflows/beam_LoadTests_Python_FnApiRunner_Microbenchmark.yml index 8d111a9f4e64..5755f25a47b1 100644 --- a/.github/workflows/beam_LoadTests_Python_FnApiRunner_Microbenchmark.yml +++ b/.github/workflows/beam_LoadTests_Python_FnApiRunner_Microbenchmark.yml @@ -69,10 +69,10 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - - name: Install Python - uses: actions/setup-python@v4 + - name: Setup environment + uses: ./.github/actions/setup-environment-action with: - python-version: '3.8' + python-version: 3.8 - name: Prepare test arguments uses: ./.github/actions/test-arguments-action with: diff --git a/.github/workflows/beam_MetricsCredentialsRotation.yml b/.github/workflows/beam_MetricsCredentialsRotation.yml index 9bd795f0c2a4..7b9727053aaa 100644 --- a/.github/workflows/beam_MetricsCredentialsRotation.yml +++ b/.github/workflows/beam_MetricsCredentialsRotation.yml @@ -66,6 +66,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: Starting credential rotation run: | gcloud container clusters update metrics --start-credential-rotation --zone=us-central1-a --quiet diff --git a/.github/workflows/beam_PerformanceTests_AvroIOIT.yml b/.github/workflows/beam_PerformanceTests_AvroIOIT.yml index 4c6ac900c544..8c781b867263 100644 --- a/.github/workflows/beam_PerformanceTests_AvroIOIT.yml +++ b/.github/workflows/beam_PerformanceTests_AvroIOIT.yml @@ -69,6 +69,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: Prepare test arguments uses: ./.github/actions/test-arguments-action with: diff --git a/.github/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml b/.github/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml index d3f881e9e07b..e137af01554a 100644 --- a/.github/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml +++ b/.github/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml @@ -69,6 +69,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: Authenticate on GCP id: auth uses: google-github-actions/auth@v1 diff --git a/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml b/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml index 599e1804011f..00a817c94e83 100644 --- a/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml +++ b/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml @@ -69,6 +69,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: Prepare test arguments uses: ./.github/actions/test-arguments-action with: diff --git a/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml b/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml index a4811864164c..6cb5e8f51fec 100644 --- a/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml +++ b/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml @@ -69,6 +69,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: Prepare test arguments uses: ./.github/actions/test-arguments-action with: diff --git a/.github/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml b/.github/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml index 8d0b3adb9141..f8c978cff142 100644 --- a/.github/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml +++ b/.github/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml @@ -69,6 +69,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: Prepare test arguments uses: ./.github/actions/test-arguments-action with: diff --git a/.github/workflows/beam_PerformanceTests_Cdap.yml b/.github/workflows/beam_PerformanceTests_Cdap.yml index 49184abfd4f3..b4e0b2e011ab 100644 --- a/.github/workflows/beam_PerformanceTests_Cdap.yml +++ b/.github/workflows/beam_PerformanceTests_Cdap.yml @@ -69,6 +69,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: Authenticate on GCP id: auth uses: google-github-actions/auth@v1 diff --git a/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml b/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml index 7ea42f197708..465a77b45088 100644 --- a/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml +++ b/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml @@ -69,6 +69,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action # The env variable is created and populated in the test-arguments-action as "<github.job>_test_arguments_<argument_file_paths_index>" - name: Prepare test arguments uses: ./.github/actions/test-arguments-action diff --git a/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml b/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml index 38b28ea084ae..f52adaaec4a1 100644 --- a/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml +++ b/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml @@ -69,6 +69,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: Authenticate on GCP id: auth uses: google-github-actions/auth@v1 diff --git a/.github/workflows/beam_PerformanceTests_HadoopFormat.yml b/.github/workflows/beam_PerformanceTests_HadoopFormat.yml index 9e6160007e9d..d303a000fddc 100644 --- a/.github/workflows/beam_PerformanceTests_HadoopFormat.yml +++ b/.github/workflows/beam_PerformanceTests_HadoopFormat.yml @@ -69,6 +69,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: Authenticate on GCP id: auth uses: google-github-actions/auth@v1 diff --git a/.github/workflows/beam_PerformanceTests_JDBC.yml b/.github/workflows/beam_PerformanceTests_JDBC.yml index 206f7cd199b6..f6f5fcebd45b 100644 --- a/.github/workflows/beam_PerformanceTests_JDBC.yml +++ b/.github/workflows/beam_PerformanceTests_JDBC.yml @@ -69,6 +69,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: Authenticate on GCP id: auth uses: google-github-actions/auth@v1 diff --git a/.github/workflows/beam_PerformanceTests_Kafka_IO.yml b/.github/workflows/beam_PerformanceTests_Kafka_IO.yml index 47cafaac9f81..772142a46ef8 100644 --- a/.github/workflows/beam_PerformanceTests_Kafka_IO.yml +++ b/.github/workflows/beam_PerformanceTests_Kafka_IO.yml @@ -71,6 +71,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: Set k8s access uses: ./.github/actions/setup-k8s-access with: diff --git a/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml b/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml index a804f66740cf..9482aae8fc0c 100644 --- a/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml +++ b/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml @@ -69,6 +69,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: Prepare test arguments uses: ./.github/actions/test-arguments-action with: diff --git a/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml b/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml index 9510a98e1c19..e8e52104e898 100644 --- a/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml +++ b/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml @@ -69,6 +69,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: Authenticate on GCP id: auth uses: google-github-actions/auth@v1 diff --git a/.github/workflows/beam_PerformanceTests_MongoDBIO_IT.yml b/.github/workflows/beam_PerformanceTests_MongoDBIO_IT.yml index 4f3d15f29a17..5b25bd838811 100644 --- a/.github/workflows/beam_PerformanceTests_MongoDBIO_IT.yml +++ b/.github/workflows/beam_PerformanceTests_MongoDBIO_IT.yml @@ -69,6 +69,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: Authenticate on GCP id: auth uses: google-github-actions/auth@v1 diff --git a/.github/workflows/beam_PerformanceTests_ParquetIOIT.yml b/.github/workflows/beam_PerformanceTests_ParquetIOIT.yml index 879d622c70d6..6ade6b2f88ef 100644 --- a/.github/workflows/beam_PerformanceTests_ParquetIOIT.yml +++ b/.github/workflows/beam_PerformanceTests_ParquetIOIT.yml @@ -69,6 +69,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action # The env variable is created and populated in the test-arguments-action as "<github.job>_test_arguments_<argument_file_paths_index>" - name: Prepare test arguments uses: ./.github/actions/test-arguments-action diff --git a/.github/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml b/.github/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml index e9bc1f3b5db5..05f1198d65c2 100644 --- a/.github/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml +++ b/.github/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml @@ -69,6 +69,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: Authenticate on GCP id: auth uses: google-github-actions/auth@v1 diff --git a/.github/workflows/beam_PerformanceTests_SparkReceiver_IO.yml b/.github/workflows/beam_PerformanceTests_SparkReceiver_IO.yml index bf750acbd31c..7aa71c3ef5a2 100644 --- a/.github/workflows/beam_PerformanceTests_SparkReceiver_IO.yml +++ b/.github/workflows/beam_PerformanceTests_SparkReceiver_IO.yml @@ -69,6 +69,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: Authenticate on GCP id: auth uses: google-github-actions/auth@v1 diff --git a/.github/workflows/beam_PerformanceTests_TFRecordIOIT.yml b/.github/workflows/beam_PerformanceTests_TFRecordIOIT.yml index 7088dab537d9..0d80693da073 100644 --- a/.github/workflows/beam_PerformanceTests_TFRecordIOIT.yml +++ b/.github/workflows/beam_PerformanceTests_TFRecordIOIT.yml @@ -69,6 +69,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: Prepare test arguments uses: ./.github/actions/test-arguments-action with: diff --git a/.github/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml b/.github/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml index f49be834154b..10099f75f967 100644 --- a/.github/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml +++ b/.github/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml @@ -71,6 +71,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: Authenticate on GCP id: auth uses: google-github-actions/auth@v1 diff --git a/.github/workflows/beam_PerformanceTests_TextIOIT.yml b/.github/workflows/beam_PerformanceTests_TextIOIT.yml index 8be4370b3e4f..2c92ec3eff23 100644 --- a/.github/workflows/beam_PerformanceTests_TextIOIT.yml +++ b/.github/workflows/beam_PerformanceTests_TextIOIT.yml @@ -69,6 +69,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: Prepare test arguments uses: ./.github/actions/test-arguments-action with: diff --git a/.github/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml b/.github/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml index e0f4b0365b6e..4caad4ff4114 100644 --- a/.github/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml +++ b/.github/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml @@ -69,6 +69,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: Set k8s access uses: ./.github/actions/setup-k8s-access with: diff --git a/.github/workflows/beam_PerformanceTests_XmlIOIT.yml b/.github/workflows/beam_PerformanceTests_XmlIOIT.yml index d610d362cdac..1dd1efc8b052 100644 --- a/.github/workflows/beam_PerformanceTests_XmlIOIT.yml +++ b/.github/workflows/beam_PerformanceTests_XmlIOIT.yml @@ -69,6 +69,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: Prepare test arguments uses: ./.github/actions/test-arguments-action with: diff --git a/.github/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml b/.github/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml index 6ceb38ca87a4..b03a69902354 100644 --- a/.github/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml +++ b/.github/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml @@ -69,6 +69,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: Authenticate on GCP id: auth uses: google-github-actions/auth@v1 diff --git a/.github/workflows/beam_PostCommit_BeamMetrics_Publish.yml b/.github/workflows/beam_PostCommit_BeamMetrics_Publish.yml index 643a159dc21c..9cff830828a2 100644 --- a/.github/workflows/beam_PostCommit_BeamMetrics_Publish.yml +++ b/.github/workflows/beam_PostCommit_BeamMetrics_Publish.yml @@ -72,6 +72,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: Authenticate on GCP uses: google-github-actions/setup-gcloud@v0 with: diff --git a/.github/workflows/beam_PostCommit_Go.yml b/.github/workflows/beam_PostCommit_Go.yml index 27d101275fbd..1a0a616006f0 100644 --- a/.github/workflows/beam_PostCommit_Go.yml +++ b/.github/workflows/beam_PostCommit_Go.yml @@ -67,6 +67,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: Set up Docker Buildx uses: docker/setup-buildx-action@v1 - name: Authenticate on GCP diff --git a/.github/workflows/beam_PostCommit_Go_VR_Flink.yml b/.github/workflows/beam_PostCommit_Go_VR_Flink.yml index 83393343d889..6434f57316e7 100644 --- a/.github/workflows/beam_PostCommit_Go_VR_Flink.yml +++ b/.github/workflows/beam_PostCommit_Go_VR_Flink.yml @@ -67,6 +67,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: run Go Flink ValidatesRunner script env: CLOUDSDK_CONFIG: ${{ env.KUBELET_GCLOUD_CONFIG_PATH}} diff --git a/.github/workflows/beam_PostCommit_Go_VR_Spark.yml b/.github/workflows/beam_PostCommit_Go_VR_Spark.yml index b902919006ad..7b80df1191c9 100644 --- a/.github/workflows/beam_PostCommit_Go_VR_Spark.yml +++ b/.github/workflows/beam_PostCommit_Go_VR_Spark.yml @@ -67,6 +67,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: run Go Spark ValidatesRunner script env: CLOUDSDK_CONFIG: ${{ env.KUBELET_GCLOUD_CONFIG_PATH}} diff --git a/.github/workflows/beam_PostCommit_Java.yml b/.github/workflows/beam_PostCommit_Java.yml index 3f56b2c4d85a..905cb3a7f253 100644 --- a/.github/workflows/beam_PostCommit_Java.yml +++ b/.github/workflows/beam_PostCommit_Java.yml @@ -69,6 +69,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: run PostCommit Java script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PostCommit_Java_Avro_Versions.yml b/.github/workflows/beam_PostCommit_Java_Avro_Versions.yml index 02f309c2f739..1cd59fe409a1 100644 --- a/.github/workflows/beam_PostCommit_Java_Avro_Versions.yml +++ b/.github/workflows/beam_PostCommit_Java_Avro_Versions.yml @@ -69,6 +69,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: run PostCommit Java Avro Versions script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml b/.github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml index 5edfe37320e2..33748db11cec 100644 --- a/.github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml +++ b/.github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml @@ -69,6 +69,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: Authenticate on GCP uses: google-github-actions/setup-gcloud@v0 with: diff --git a/.github/workflows/beam_PostCommit_Java_DataflowV1.yml b/.github/workflows/beam_PostCommit_Java_DataflowV1.yml index 53c0b0b1db39..a15730a3d4ed 100644 --- a/.github/workflows/beam_PostCommit_Java_DataflowV1.yml +++ b/.github/workflows/beam_PostCommit_Java_DataflowV1.yml @@ -69,10 +69,9 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - - name: Set up Java - uses: actions/setup-java@v3.8.0 + - name: Setup environment + uses: ./.github/actions/setup-environment-action with: - distribution: 'temurin' java-version: | 11 8 diff --git a/.github/workflows/beam_PostCommit_Java_DataflowV2.yml b/.github/workflows/beam_PostCommit_Java_DataflowV2.yml index 6c8d9aa45153..f90fc8f1f262 100644 --- a/.github/workflows/beam_PostCommit_Java_DataflowV2.yml +++ b/.github/workflows/beam_PostCommit_Java_DataflowV2.yml @@ -69,6 +69,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: run PostCommit Java Dataflow V2 script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow.yml index 03e94979c51f..2c2137482552 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow.yml @@ -71,6 +71,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: run PostCommit Java Examples Dataflow script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml index 6b92c188c985..ee6596adbe4c 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml @@ -71,10 +71,9 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} ${{ matrix.java_version }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }} ${{ matrix.java_version }}) - - name: Set up Java${{ matrix.java_version }} - uses: actions/setup-java@v3.8.0 + - name: Setup environment + uses: ./.github/actions/setup-environment-action with: - distribution: 'temurin' java-version: | ${{ matrix.java_version }} 8 diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml index 1f817a3acf0a..7283bbedca74 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml @@ -69,6 +69,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: run PostCommit Java Examples Dataflow V2 script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml index 411cc65944fb..1b0065901046 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml @@ -73,10 +73,9 @@ jobs: comment_phrase: ${{ matrix.job_phrase_1 }} ${{ matrix.java_version }} ${{ matrix.job_phrase_2 }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase_1 }} ${{ matrix.java_version }} ${{ matrix.job_phrase_2 }}) - - name: Set up Java${{ matrix.java_version }} - uses: actions/setup-java@v3.8.0 + - name: Setup environment + uses: ./.github/actions/setup-environment-action with: - distribution: 'temurin' java-version: ${{ matrix.java_version }} - name: run PostCommit Java Examples Dataflow V2 Java${{ matrix.java_version }} script uses: ./.github/actions/gradle-command-self-hosted-action diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Direct.yml b/.github/workflows/beam_PostCommit_Java_Examples_Direct.yml index b0f350809d2d..60f5c1da489f 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Direct.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Direct.yml @@ -69,11 +69,10 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - - name: Install Java - uses: actions/setup-java@v3.8.0 + - name: Setup environment + uses: ./.github/actions/setup-environment-action with: - distribution: 'zulu' - java-version: '8' + java-version: 8 - name: run examplesIntegrationTest script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Spark.yml b/.github/workflows/beam_PostCommit_Java_Examples_Spark.yml index 757008684524..e3f70bd8f69f 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Spark.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Spark.yml @@ -69,11 +69,10 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - - name: Install Java - uses: actions/setup-java@v3.8.0 + - name: Setup environment + uses: ./.github/actions/setup-environment-action with: - distribution: 'zulu' - java-version: '8' + java-version: 8 - name: run examplesIntegrationTest script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PostCommit_Java_Hadoop_Versions.yml b/.github/workflows/beam_PostCommit_Java_Hadoop_Versions.yml index 29e975628275..41eec12629d3 100644 --- a/.github/workflows/beam_PostCommit_Java_Hadoop_Versions.yml +++ b/.github/workflows/beam_PostCommit_Java_Hadoop_Versions.yml @@ -67,11 +67,10 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - - name: Install Java - uses: actions/setup-java@v3.8.0 + - name: Setup environment + uses: ./.github/actions/setup-environment-action with: - distribution: 'zulu' - java-version: '8' + java-version: 8 - name: run validatesRunner script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml b/.github/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml index 4147d1099c8d..c80ec6c0fbee 100644 --- a/.github/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml +++ b/.github/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml @@ -69,6 +69,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: Authenticate on GCP id: auth uses: google-github-actions/auth@v1 diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml index 1fd46b95e087..57dca871e59a 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml @@ -67,11 +67,10 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - - name: Set up Java 11 - uses: actions/setup-java@v3.11.0 + - name: Setup environment + uses: ./.github/actions/setup-environment-action with: - distribution: 'temurin' - java-version: '11' + java-version: 11 - name: run PostCommit Java Jpms Dataflow Java11 script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml index ff27bae8c527..02046068c025 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml @@ -67,10 +67,9 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - - name: Set up Java - uses: actions/setup-java@v3.11.0 + - name: Setup environment + uses: ./.github/actions/setup-environment-action with: - distribution: 'temurin' java-version: | 17 8 diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml index 1dec7b5f54f5..61148ace0be8 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml @@ -67,11 +67,10 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - - name: Set up Java 11 - uses: actions/setup-java@v3.11.0 + - name: Setup environment + uses: ./.github/actions/setup-environment-action with: - distribution: 'temurin' - java-version: '11' + java-version: 11 - name: run PostCommit Java Jpms Direct Java11 script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml index 748a74d2ec15..823cea06ada9 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml @@ -67,10 +67,9 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - - name: Set up Java - uses: actions/setup-java@v3.11.0 + - name: Setup environment + uses: ./.github/actions/setup-environment-action with: - distribution: 'temurin' java-version: | 17 8 diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml index 62656517038f..2b217630ebf4 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml @@ -67,11 +67,10 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - - name: Set up Java 11 - uses: actions/setup-java@v3.11.0 + - name: Setup environment + uses: ./.github/actions/setup-environment-action with: - distribution: 'temurin' - java-version: '11' + java-version: 11 - name: run PostCommit Java Jpms Flink Java11 script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml index 745bec641c24..4026ce9a22f4 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml @@ -67,11 +67,10 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - - name: Set up Java 11 - uses: actions/setup-java@v3.11.0 + - name: Setup environment + uses: ./.github/actions/setup-environment-action with: - distribution: 'temurin' - java-version: '11' + java-version: 11 - name: run PostCommit Java Jpms Spark Java11 script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml index 8a3ff9a9bb1c..d40723af05c9 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml @@ -95,6 +95,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: run PostCommit Java Nexmark Dataflow (${{ matrix.streaming }} ${{ matrix.queryLanguage }}) script if: matrix.queryLanguage != 'none' uses: ./.github/actions/gradle-command-self-hosted-action diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml index 428ea231e777..787ca05a065d 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml @@ -95,6 +95,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: run PostCommit Java Nexmark Dataflow V2 (streaming = ${{ matrix.streaming }}) script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml index 076c030d91ea..f06302c08513 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml @@ -97,10 +97,9 @@ jobs: comment_phrase: ${{ matrix.job_phrase_1 }} ${{ matrix.java_version }} ${{ matrix.job_phrase_2 }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase_1 }} ${{ matrix.java_version }} ${{ matrix.job_phrase_2 }}) - - name: Set up Java${{ matrix.java_version }} - uses: actions/setup-java@v3.8.0 + - name: Setup environment + uses: ./.github/actions/setup-environment-action with: - distribution: 'temurin' java-version: ${{ matrix.java_version }} - name: run PostCommit Java ${{ matrix.java_version }} Nexmark Dataflow V2 (streaming = ${{ matrix.streaming }}) script uses: ./.github/actions/gradle-command-self-hosted-action diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Direct.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Direct.yml index a1842c73fe53..837582a484b9 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Direct.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Direct.yml @@ -90,6 +90,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: run PostCommit Java Nexmark Direct (${{ matrix.streaming }} ${{ matrix.queryLanguage }}) script if: matrix.queryLanguage != 'none' uses: ./.github/actions/gradle-command-self-hosted-action diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Flink.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Flink.yml index 76278de19ff2..afcc9060491e 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Flink.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Flink.yml @@ -89,6 +89,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: run PostCommit Java Nexmark Flink (${{ matrix.streaming }} ${{ matrix.queryLanguage }}) script if: matrix.queryLanguage != 'none' uses: ./.github/actions/gradle-command-self-hosted-action diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Spark.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Spark.yml index 110cb20c90cf..937a843740a1 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Spark.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Spark.yml @@ -89,6 +89,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: run PostCommit Java Nexmark Spark (runner = ${{ matrix.runner }} queryLanguage = ${{ matrix.queryLanguage }}) script if: matrix.queryLanguage != 'none' uses: ./.github/actions/gradle-command-self-hosted-action diff --git a/.github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml b/.github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml index 021378596e73..de5253fe8516 100644 --- a/.github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml +++ b/.github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml @@ -69,6 +69,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: run PostCommit Java Flink PortableValidatesRunner Streaming script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PostCommit_Java_PVR_Samza.yml b/.github/workflows/beam_PostCommit_Java_PVR_Samza.yml index cf87530639c3..912cba9877a5 100644 --- a/.github/workflows/beam_PostCommit_Java_PVR_Samza.yml +++ b/.github/workflows/beam_PostCommit_Java_PVR_Samza.yml @@ -69,6 +69,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: run PostCommit Java Samza script env: CLOUDSDK_CONFIG: ${{ env.KUBELET_GCLOUD_CONFIG_PATH}} diff --git a/.github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml b/.github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml index 79eadd753562..b94e40037bdc 100644 --- a/.github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml +++ b/.github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml @@ -69,6 +69,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: run PostCommit Java PortableValidatesRunner Spark3 Streaming script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PostCommit_Java_Sickbay.yml b/.github/workflows/beam_PostCommit_Java_Sickbay.yml index 18ef48b92f70..a9e21f4c9f33 100644 --- a/.github/workflows/beam_PostCommit_Java_Sickbay.yml +++ b/.github/workflows/beam_PostCommit_Java_Sickbay.yml @@ -69,6 +69,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: run PostCommit Java Sickbay script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml b/.github/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml index 5af333e645b0..ba2c72ffa2cc 100644 --- a/.github/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml +++ b/.github/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml @@ -71,6 +71,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: Authenticate on GCP id: auth uses: google-github-actions/auth@v1 diff --git a/.github/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml b/.github/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml index d6b63752eede..5fb70ef56127 100644 --- a/.github/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml @@ -92,6 +92,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: run PostCommit Java Tpcds Dataflow script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PostCommit_Java_Tpcds_Flink.yml b/.github/workflows/beam_PostCommit_Java_Tpcds_Flink.yml index d2ba0cafc01f..fcd87f41326d 100644 --- a/.github/workflows/beam_PostCommit_Java_Tpcds_Flink.yml +++ b/.github/workflows/beam_PostCommit_Java_Tpcds_Flink.yml @@ -89,6 +89,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: run PostCommit Java Tpcds Flink script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PostCommit_Java_Tpcds_Spark.yml b/.github/workflows/beam_PostCommit_Java_Tpcds_Spark.yml index d93f0e1a8b52..dd1b3a18b8cd 100644 --- a/.github/workflows/beam_PostCommit_Java_Tpcds_Spark.yml +++ b/.github/workflows/beam_PostCommit_Java_Tpcds_Spark.yml @@ -88,6 +88,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: run PostCommit Java Tpcds Spark script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml index bd7f29720fa8..4d0357d9215b 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml @@ -69,11 +69,10 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - - name: Install Java - uses: actions/setup-java@v3.8.0 + - name: Setup environment + uses: ./.github/actions/setup-environment-action with: - distribution: 'zulu' - java-version: '8' + java-version: 8 - name: run validatesRunner script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml index a6afa04f565d..62f742f5b5b9 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml @@ -71,10 +71,9 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} ${{ matrix.java_version }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) ${{ matrix.java_version }} - - name: Set up Java${{ matrix.java_version }} - uses: actions/setup-java@v3.8.0 + - name: Setup environment + uses: ./.github/actions/setup-environment-action with: - distribution: 'temurin' java-version: | ${{ matrix.java_version }} 8 diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml index 116f8f6e4a20..175893f5b015 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml @@ -69,11 +69,10 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - - name: Install Java - uses: actions/setup-java@v3.8.0 + - name: Setup environment + uses: ./.github/actions/setup-environment-action with: - distribution: 'zulu' - java-version: '8' + java-version: 8 - name: run validatesRunnerStreaming script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml index 55064692a718..723d17cf094c 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml @@ -69,11 +69,10 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - - name: Install Java - uses: actions/setup-java@v3.8.0 + - name: Setup environment + uses: ./.github/actions/setup-environment-action with: - distribution: 'zulu' - java-version: '8' + java-version: 8 - name: run validatesRunnerV2 script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml index 8314db686ccd..d81f7cf2752d 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml @@ -69,11 +69,10 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - - name: Install Java - uses: actions/setup-java@v3.8.0 + - name: Setup environment + uses: ./.github/actions/setup-environment-action with: - distribution: 'zulu' - java-version: '8' + java-version: 8 - name: run validatesRunnerV2Streaming script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml index cf62a7894658..03453f0eb646 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml @@ -69,11 +69,10 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - - name: Install Java - uses: actions/setup-java@v3.8.0 + - name: Setup environment + uses: ./.github/actions/setup-environment-action with: - distribution: 'zulu' - java-version: '8' + java-version: 8 - name: run validatesRunner script run: ./gradlew :runners:direct-java:validatesRunner - name: Archive JUnit Test Results diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml index 2d879fa7d631..b16a20f6c210 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml @@ -71,10 +71,9 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} ${{ matrix.java_version }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) ${{ matrix.java_version }} - - name: Set up Java${{ matrix.java_version }} - uses: actions/setup-java@v3.8.0 + - name: Setup environment + uses: ./.github/actions/setup-environment-action with: - distribution: 'temurin' java-version: | ${{ matrix.java_version }} 8 diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml index e96f8efb9aeb..181483597a9f 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml @@ -69,10 +69,9 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - - name: Set up Java - uses: actions/setup-java@v3.8.0 + - name: Setup environment + uses: ./.github/actions/setup-environment-action with: - distribution: 'temurin' java-version: | 11 8 diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml index b577b584f0ad..986eab3595b9 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml @@ -67,11 +67,10 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - - name: Install Java - uses: actions/setup-java@v3.8.0 + - name: Setup environment + uses: ./.github/actions/setup-environment-action with: - distribution: 'zulu' - java-version: '8' + java-version: 8 - name: run validatesRunner script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml index 3465ff953da7..bc01dac79e96 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml @@ -67,11 +67,10 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - - name: Install Java - uses: actions/setup-java@v3.8.0 + - name: Setup environment + uses: ./.github/actions/setup-environment-action with: - distribution: 'zulu' - java-version: '8' + java-version: 8 - name: run validatesRunner script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml index 6df7d3ac35e9..404c15e65149 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml @@ -67,11 +67,10 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - - name: Install Java - uses: actions/setup-java@v3.8.0 + - name: Setup environment + uses: ./.github/actions/setup-environment-action with: - distribution: 'zulu' - java-version: '8' + java-version: 8 - name: run validatesStructuredStreamingRunnerBatch script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml index 1f82a4568274..d8249a17df8e 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml @@ -69,10 +69,9 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - - name: Set up Java - uses: actions/setup-java@v3.8.0 + - name: Setup environment + uses: ./.github/actions/setup-environment-action with: - distribution: 'temurin' java-version: | 11 8 diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml index 3bcaff465f1d..87db69c308ad 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml @@ -67,11 +67,10 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - - name: Install Java - uses: actions/setup-java@v3.8.0 + - name: Setup environment + uses: ./.github/actions/setup-environment-action with: - distribution: 'zulu' - java-version: '8' + java-version: 8 - name: run validatesRunner script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml index 9d42ecda4f38..4e3b9c49d471 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml @@ -67,15 +67,11 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - - name: Install Java - uses: actions/setup-java@v3.8.0 + - name: Setup environment + uses: ./.github/actions/setup-environment-action with: - distribution: 'zulu' - java-version: '8' - - name: Install Python - uses: actions/setup-python@v4 - with: - python-version: '3.8' + java-version: 8 + python-version: 3.8 - name: run ulrLoopbackValidatesRunner script run: ./gradlew :runners:portability:java:ulrLoopbackValidatesRunner - name: Archive JUnit Test Results diff --git a/.github/workflows/beam_PostCommit_Javadoc.yml b/.github/workflows/beam_PostCommit_Javadoc.yml index a70b37c10a9e..fe72554de6fb 100644 --- a/.github/workflows/beam_PostCommit_Javadoc.yml +++ b/.github/workflows/beam_PostCommit_Javadoc.yml @@ -69,6 +69,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: run aggregateJavadoc script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PostCommit_PortableJar_Flink.yml b/.github/workflows/beam_PostCommit_PortableJar_Flink.yml index 0140c16a85cf..a0d5f51d04a2 100644 --- a/.github/workflows/beam_PostCommit_PortableJar_Flink.yml +++ b/.github/workflows/beam_PostCommit_PortableJar_Flink.yml @@ -67,8 +67,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - - name: Install Python - uses: actions/setup-python@v4 + - name: Setup environment + uses: ./.github/actions/setup-environment-action with: python-version: 3.8 - name: run testPipelineJarFlinkRunner script diff --git a/.github/workflows/beam_PostCommit_PortableJar_Spark.yml b/.github/workflows/beam_PostCommit_PortableJar_Spark.yml index 243effdf063d..500c5919d213 100644 --- a/.github/workflows/beam_PostCommit_PortableJar_Spark.yml +++ b/.github/workflows/beam_PostCommit_PortableJar_Spark.yml @@ -67,8 +67,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - - name: Install Python - uses: actions/setup-python@v4 + - name: Setup environment + uses: ./.github/actions/setup-environment-action with: python-version: 3.8 - name: run testPipelineJarSparkRunner script diff --git a/.github/workflows/beam_PostCommit_Python.yml b/.github/workflows/beam_PostCommit_Python.yml index e439b50d5313..e3e4ce927519 100644 --- a/.github/workflows/beam_PostCommit_Python.yml +++ b/.github/workflows/beam_PostCommit_Python.yml @@ -71,8 +71,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} ${{ matrix.python_version }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }} ${{ matrix.python_version }}) - - name: Install Python - uses: actions/setup-python@v4 + - name: Setup environment + uses: ./.github/actions/setup-environment-action with: python-version: ${{ matrix.python_version }} - name: Install docker compose diff --git a/.github/workflows/beam_PostCommit_Python_Arm.yml b/.github/workflows/beam_PostCommit_Python_Arm.yml index 065c917f8687..f28d6b1da783 100644 --- a/.github/workflows/beam_PostCommit_Python_Arm.yml +++ b/.github/workflows/beam_PostCommit_Python_Arm.yml @@ -73,8 +73,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} ${{ matrix.python_version }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }} ${{ matrix.python_version }}) - - name: Install Python - uses: actions/setup-python@v4 + - name: Setup environment + uses: ./.github/actions/setup-environment-action with: python-version: ${{ matrix.python_version }} - name: Install docker compose diff --git a/.github/workflows/beam_PostCommit_Python_Nexmark_Direct.yml b/.github/workflows/beam_PostCommit_Python_Nexmark_Direct.yml index c63bcf2ecad1..9f00b6f163d1 100644 --- a/.github/workflows/beam_PostCommit_Python_Nexmark_Direct.yml +++ b/.github/workflows/beam_PostCommit_Python_Nexmark_Direct.yml @@ -112,8 +112,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - - name: Install Python - uses: actions/setup-python@v4 + - name: Setup environment + uses: ./.github/actions/setup-environment-action with: python-version: 3.8 - name: run Java Testing Nexmark (query ${{ matrix.query }}) diff --git a/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml index 291de49c3bbb..65d3f70d130b 100644 --- a/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml @@ -67,8 +67,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - - name: Install Python - uses: actions/setup-python@v4 + - name: Setup environment + uses: ./.github/actions/setup-environment-action with: python-version: | 3.8 diff --git a/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml b/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml index cf061f47e677..00bcabf3b6a4 100644 --- a/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml +++ b/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml @@ -67,8 +67,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - - name: Install Python - uses: actions/setup-python@v4 + - name: Setup environment + uses: ./.github/actions/setup-environment-action with: python-version: | 3.8 diff --git a/.github/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml index 069b9965517d..0e73b0e4bc82 100644 --- a/.github/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml @@ -67,8 +67,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - - name: Install Python - uses: actions/setup-python@v4 + - name: Setup environment + uses: ./.github/actions/setup-environment-action with: python-version: | 3.8 diff --git a/.github/workflows/beam_PostCommit_SQL.yml b/.github/workflows/beam_PostCommit_SQL.yml index 1ad8e0567cfd..0ac8af3c6bb8 100644 --- a/.github/workflows/beam_PostCommit_SQL.yml +++ b/.github/workflows/beam_PostCommit_SQL.yml @@ -69,6 +69,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: run PostCommit SQL script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PostCommit_Sickbay_Python.yml b/.github/workflows/beam_PostCommit_Sickbay_Python.yml index 7516a6003a2e..b01ee16ade13 100644 --- a/.github/workflows/beam_PostCommit_Sickbay_Python.yml +++ b/.github/workflows/beam_PostCommit_Sickbay_Python.yml @@ -73,8 +73,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase_1 }} ${{ matrix.python_version }} ${{ matrix.job_phrase_2 }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase_1 }} ${{ matrix.python_version }} ${{ matrix.job_phrase_2 }}) - - name: Install Python - uses: actions/setup-python@v4 + - name: Setup environment + uses: ./.github/actions/setup-environment-action with: python-version: ${{ matrix.python_version }} - name: Set PY_VER_CLEAN diff --git a/.github/workflows/beam_PostCommit_TransformService_Direct.yml b/.github/workflows/beam_PostCommit_TransformService_Direct.yml index 327b50f7840c..a180adf810bb 100644 --- a/.github/workflows/beam_PostCommit_TransformService_Direct.yml +++ b/.github/workflows/beam_PostCommit_TransformService_Direct.yml @@ -68,14 +68,10 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} ${{ matrix.python_version }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }} ${{ matrix.python_version }}) - - name: Set up Java 11 - uses: actions/setup-java@v3.11.0 - with: - distribution: 'temurin' - java-version: '11' - - name: Install Python - uses: actions/setup-python@v4 + - name: Setup environment + uses: ./.github/actions/setup-environment-action with: + java-version: 11 python-version: | 3.8 3.11 diff --git a/.github/workflows/beam_PostCommit_Website_Publish.yml b/.github/workflows/beam_PostCommit_Website_Publish.yml index e5759e390211..047f9564c37a 100644 --- a/.github/workflows/beam_PostCommit_Website_Publish.yml +++ b/.github/workflows/beam_PostCommit_Website_Publish.yml @@ -56,6 +56,8 @@ jobs: name: beam_PostCommit_Website_Publish steps: - uses: actions/checkout@v4 + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: run PostCommit Website Publish script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PostCommit_Website_Test.yml b/.github/workflows/beam_PostCommit_Website_Test.yml index a73d971f4b68..3782b9ca17dd 100644 --- a/.github/workflows/beam_PostCommit_Website_Test.yml +++ b/.github/workflows/beam_PostCommit_Website_Test.yml @@ -67,6 +67,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: run PostCommit Website Test script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PostCommit_XVR_Direct.yml b/.github/workflows/beam_PostCommit_XVR_Direct.yml index 8a0c4691b85c..c63969051a07 100644 --- a/.github/workflows/beam_PostCommit_XVR_Direct.yml +++ b/.github/workflows/beam_PostCommit_XVR_Direct.yml @@ -68,8 +68,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} ${{ matrix.python_version }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }} ${{ matrix.python_version }}) - - name: Install Python - uses: actions/setup-python@v4 + - name: Setup environment + uses: ./.github/actions/setup-environment-action with: python-version: | 3.8 diff --git a/.github/workflows/beam_PostCommit_XVR_Flink.yml b/.github/workflows/beam_PostCommit_XVR_Flink.yml index 3e5a58892755..ea6e0b55c764 100644 --- a/.github/workflows/beam_PostCommit_XVR_Flink.yml +++ b/.github/workflows/beam_PostCommit_XVR_Flink.yml @@ -69,8 +69,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} ${{ matrix.python_version }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }} ${{ matrix.python_version }}) - - name: Install Python - uses: actions/setup-python@v4 + - name: Setup environment + uses: ./.github/actions/setup-environment-action with: python-version: | 3.8 diff --git a/.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml b/.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml index 64c87c92c5ed..2d37e0522c93 100644 --- a/.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml @@ -67,10 +67,10 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - - name: Install Python - uses: actions/setup-python@v4 + - name: Setup environment + uses: ./.github/actions/setup-environment-action with: - python-version: '3.8' + python-version: 3.8 - name: run XVR GoUsingJava Dataflow script env: USER: github-actions diff --git a/.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml b/.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml index 4f3cf65ce5dd..494ab1e277d3 100644 --- a/.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml @@ -68,8 +68,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} ${{ matrix.python_version }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }} ${{ matrix.python_version }}) - - name: Install Python - uses: actions/setup-python@v4 + - name: Setup environment + uses: ./.github/actions/setup-environment-action with: python-version: | 3.8 diff --git a/.github/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml b/.github/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml index f000d1956f01..52adbc2ad824 100644 --- a/.github/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml @@ -67,8 +67,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - - name: Install Python - uses: actions/setup-python@v4 + - name: Setup environment + uses: ./.github/actions/setup-environment-action with: python-version: 3.11 - name: run PostCommit XVR PythonUsingJavaSQL Dataflow script diff --git a/.github/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml b/.github/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml index fdc3798999d9..796544539dc2 100644 --- a/.github/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml @@ -68,8 +68,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} ${{ matrix.python_version }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }} ${{ matrix.python_version }}) - - name: Install Python - uses: actions/setup-python@v4 + - name: Setup environment + uses: ./.github/actions/setup-environment-action with: python-version: | 3.8 diff --git a/.github/workflows/beam_PostCommit_XVR_Samza.yml b/.github/workflows/beam_PostCommit_XVR_Samza.yml index 486a45159db1..0f08490e415b 100644 --- a/.github/workflows/beam_PostCommit_XVR_Samza.yml +++ b/.github/workflows/beam_PostCommit_XVR_Samza.yml @@ -68,8 +68,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} ${{ matrix.python_version }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }} ${{ matrix.python_version }}) - - name: Install Python - uses: actions/setup-python@v4 + - name: Setup environment + uses: ./.github/actions/setup-environment-action with: python-version: | 3.8 diff --git a/.github/workflows/beam_PostCommit_XVR_Spark3.yml b/.github/workflows/beam_PostCommit_XVR_Spark3.yml index 5524f66c6afc..f49765381fad 100644 --- a/.github/workflows/beam_PostCommit_XVR_Spark3.yml +++ b/.github/workflows/beam_PostCommit_XVR_Spark3.yml @@ -68,8 +68,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} ${{ matrix.python_version }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }} ${{ matrix.python_version }}) - - name: Install Python - uses: actions/setup-python@v4 + - name: Setup environment + uses: ./.github/actions/setup-environment-action with: python-version: | 3.8 diff --git a/.github/workflows/beam_PreCommit_Java.yml b/.github/workflows/beam_PreCommit_Java.yml index e1a3361c0cf3..ea3340beebac 100644 --- a/.github/workflows/beam_PreCommit_Java.yml +++ b/.github/workflows/beam_PreCommit_Java.yml @@ -175,6 +175,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: run Java PreCommit script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PreCommit_Java_Amazon-Web-Services2_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Amazon-Web-Services2_IO_Direct.yml index 0a117ec9cf9b..4591fcd3dacc 100644 --- a/.github/workflows/beam_PreCommit_Java_Amazon-Web-Services2_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Amazon-Web-Services2_IO_Direct.yml @@ -100,6 +100,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: run Amazon-Web-Services2 IO build script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PreCommit_Java_Amazon-Web-Services_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Amazon-Web-Services_IO_Direct.yml index 2a8061657fc5..960c59587085 100644 --- a/.github/workflows/beam_PreCommit_Java_Amazon-Web-Services_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Amazon-Web-Services_IO_Direct.yml @@ -100,6 +100,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: run Amazon-Web-Services IO build script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PreCommit_Java_Amqp_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Amqp_IO_Direct.yml index 093afbe05834..e23ccea56d8b 100644 --- a/.github/workflows/beam_PreCommit_Java_Amqp_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Amqp_IO_Direct.yml @@ -82,6 +82,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: run Amqp IO build script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml index 013bd587d3d9..1b997178aa6a 100644 --- a/.github/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml @@ -100,6 +100,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: run Azure IO build script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml index 7713c026ea8c..e026a1536fb8 100644 --- a/.github/workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml @@ -82,6 +82,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: run Cassandra IO build script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml index 08650c314f24..b1c324720b3d 100644 --- a/.github/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml @@ -86,6 +86,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: run Cdap IO build script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml index 9d2cbcffb407..f20daacd89a8 100644 --- a/.github/workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml @@ -82,6 +82,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: run Clickhouse IO build script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml index e2b331c235ba..cf7f99605c29 100644 --- a/.github/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml @@ -82,6 +82,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: run Csv IO build script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml index 5497926c3054..ac68cc6040fb 100644 --- a/.github/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml @@ -82,6 +82,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: run Debezium IO build task uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml index ad02187c7316..e6176729b376 100644 --- a/.github/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml @@ -84,6 +84,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: run ElasticSearch IO build task uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml index b12590c01603..132665df05a0 100644 --- a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml +++ b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml @@ -103,11 +103,10 @@ jobs: export_default_credentials: true # The workflow installs java 11 and as default jvm. This is different from # PreCommit_Java_Examples_Dataflow_Java17 where the build system and sources are compiled with Java8 - - name: Set up Java - uses: actions/setup-java@v3.8.0 + - name: Setup environment + uses: ./.github/actions/setup-environment-action with: - distribution: 'temurin' - java-version: '11' + java-version: 11 - name: run javaExamplesDataflowPrecommit script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml index d4a55aa3a824..6ae9eaf2572d 100644 --- a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml +++ b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml @@ -94,10 +94,9 @@ jobs: github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) # The test requires Java 17 and Java 8 versions. # Java 8 is installed second because JAVA_HOME needs to point to Java 8. - - name: Set up Java 17 and 8 - uses: actions/setup-java@v3.11.0 + - name: Setup environment + uses: ./.github/actions/setup-environment-action with: - distribution: 'temurin' java-version: | 17 8 diff --git a/.github/workflows/beam_PreCommit_Java_File-schema-transform_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_File-schema-transform_IO_Direct.yml index aeb9ed6d5056..2446eb972c62 100644 --- a/.github/workflows/beam_PreCommit_Java_File-schema-transform_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_File-schema-transform_IO_Direct.yml @@ -82,6 +82,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: run File-schema-transform IO build script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml index dc937f2abcd3..e5ef36c00451 100644 --- a/.github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml @@ -100,6 +100,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: run PreCommit Java GCP IO Direct script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml index c71c464c933e..dce33b36b9ce 100644 --- a/.github/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml @@ -84,6 +84,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: run HBase IO build script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml index ed99e58ca0eb..85723824ac52 100644 --- a/.github/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml @@ -84,6 +84,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: run HCatalog IO build script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml index 4f90aceef9e6..ec3a0c5bd839 100644 --- a/.github/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml @@ -108,6 +108,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: run Hadoop IO build script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PreCommit_Java_IOs_Direct.yml b/.github/workflows/beam_PreCommit_Java_IOs_Direct.yml index 1137d9fec9ed..4a221a911009 100644 --- a/.github/workflows/beam_PreCommit_Java_IOs_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_IOs_Direct.yml @@ -82,6 +82,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: run Java IOs PreCommit script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml index ac35176e551a..67ee59b7c4ff 100644 --- a/.github/workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml @@ -82,6 +82,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: run InfluxDb IO build script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml index d2ef342e53ca..4b86f8b23038 100644 --- a/.github/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml @@ -82,6 +82,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: run JDBC IO build script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml index 223c90cd6b73..686cc759ca31 100644 --- a/.github/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml @@ -82,6 +82,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: run Jms IO build script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml index 61ecc1d304c9..86d59f7696ff 100644 --- a/.github/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml @@ -90,6 +90,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: run Kafka IO build script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml index 8cca3951ddd9..bfe0d50ec048 100644 --- a/.github/workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml @@ -100,6 +100,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: run Kinesis IO build script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml index bc3d257d1db0..55c81bb501a2 100644 --- a/.github/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml @@ -82,6 +82,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: run Kudu IO build script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml index b8d28c129d3d..563c8714fbf4 100644 --- a/.github/workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml @@ -82,6 +82,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: run MongoDb IO build script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml index 7b19949ee655..b85697d7e39f 100644 --- a/.github/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml @@ -82,6 +82,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: run Mqtt IO build script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml index 26b4637598b7..21f746ef2cec 100644 --- a/.github/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml @@ -84,6 +84,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: run Neo4j IO build script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PreCommit_Java_PVR_Flink_Batch.yml b/.github/workflows/beam_PreCommit_Java_PVR_Flink_Batch.yml index 602bb8750ae5..eb2f85b5e106 100644 --- a/.github/workflows/beam_PreCommit_Java_PVR_Flink_Batch.yml +++ b/.github/workflows/beam_PreCommit_Java_PVR_Flink_Batch.yml @@ -88,6 +88,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: run validatesPortableRunnerBatch script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml b/.github/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml index b2f0c66c3348..b3b5bd5b344b 100644 --- a/.github/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml +++ b/.github/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml @@ -93,6 +93,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: run PreCommit Java PVR Flink Docker script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml index 4a725129faa1..9e52618edf9f 100644 --- a/.github/workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml @@ -82,6 +82,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: run Parquet IO build script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml index cac0872d902d..b60f53cc31fc 100644 --- a/.github/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml @@ -100,6 +100,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: run Pulsar IO build script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml index 4983a84a1267..d917c2830f51 100644 --- a/.github/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml @@ -82,6 +82,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: run RabbitMq IO build script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml index 51970fcdff1a..91095a000e18 100644 --- a/.github/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml @@ -82,6 +82,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: run Redis IO build script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml index 1cae0b51872e..b380e96b813f 100644 --- a/.github/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml @@ -84,6 +84,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: run SingleStore IO build script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml index 7d28989b7690..fab5fd4be694 100644 --- a/.github/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml @@ -86,6 +86,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: run Snowflake IO build script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml index b0096415b063..06ed141281d0 100644 --- a/.github/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml @@ -82,6 +82,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: run Solr IO build script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PreCommit_Java_Spark3_Versions.yml b/.github/workflows/beam_PreCommit_Java_Spark3_Versions.yml index 42b9a244008d..aaab30164805 100644 --- a/.github/workflows/beam_PreCommit_Java_Spark3_Versions.yml +++ b/.github/workflows/beam_PreCommit_Java_Spark3_Versions.yml @@ -91,13 +91,10 @@ jobs: service_account_key: ${{ secrets.GCP_SA_KEY }} project_id: ${{ secrets.GCP_PROJECT_ID }} export_default_credentials: true - - name: Install Java - uses: actions/setup-java@v3.8.0 + - name: Setup environment + uses: ./.github/actions/setup-environment-action with: - distribution: 'zulu' - java-version: '8' - cache: 'gradle' - check-latest: true + java-version: 8 - name: run sparkVersionsTest script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml index 57adc64576cf..f31ad63d1cf5 100644 --- a/.github/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml @@ -82,6 +82,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: run Splunk IO build script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml index 88eabd1e5c8a..bb22fa8aaca4 100644 --- a/.github/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml @@ -82,6 +82,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: run Thrift IO build script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml index 62f96aec254e..1dbb51bdf2a7 100644 --- a/.github/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml @@ -82,6 +82,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: run Tika IO build script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PreCommit_Kotlin_Examples.yml b/.github/workflows/beam_PreCommit_Kotlin_Examples.yml index f3e6e36a3c25..613a082a74fd 100644 --- a/.github/workflows/beam_PreCommit_Kotlin_Examples.yml +++ b/.github/workflows/beam_PreCommit_Kotlin_Examples.yml @@ -94,13 +94,10 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - - name: Install Java - uses: actions/setup-java@v3.8.0 + - name: Setup environment + uses: ./.github/actions/setup-environment-action with: - distribution: 'zulu' - java-version: '8' - cache: 'gradle' - check-latest: true + java-version: 8 - name: run Kotlin Examples script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PreCommit_Portable_Python.yml b/.github/workflows/beam_PreCommit_Portable_Python.yml index c1302468a715..1540600b7205 100644 --- a/.github/workflows/beam_PreCommit_Portable_Python.yml +++ b/.github/workflows/beam_PreCommit_Portable_Python.yml @@ -99,16 +99,10 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} ${{ matrix.python_version }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }} ${{ matrix.python_version }}) - - name: Install Java - uses: actions/setup-java@v3.8.0 - with: - distribution: 'zulu' - java-version: '8' - cache: 'gradle' - check-latest: true - - name: Install Python - uses: actions/setup-python@v4 + - name: Setup environment + uses: ./.github/actions/setup-environment-action with: + java-version: 8 python-version: | ${{ matrix.python_version }} 3.8 diff --git a/.github/workflows/beam_PreCommit_Python_PVR_Flink.yml b/.github/workflows/beam_PreCommit_Python_PVR_Flink.yml index f5457122e691..2725cc383f2a 100644 --- a/.github/workflows/beam_PreCommit_Python_PVR_Flink.yml +++ b/.github/workflows/beam_PreCommit_Python_PVR_Flink.yml @@ -100,8 +100,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - - name: Install Python - uses: actions/setup-python@v4 + - name: Setup environment + uses: ./.github/actions/setup-environment-action with: python-version: 3.11 - name: run Python PVR Flink PreCommit script diff --git a/.github/workflows/beam_PreCommit_SQL.yml b/.github/workflows/beam_PreCommit_SQL.yml index 1fe577d759e9..f4a4e30a01ff 100644 --- a/.github/workflows/beam_PreCommit_SQL.yml +++ b/.github/workflows/beam_PreCommit_SQL.yml @@ -78,11 +78,10 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - - name: Set up Java - uses: actions/setup-java@v3.8.0 + - name: Setup environment + uses: ./.github/actions/setup-environment-action with: - distribution: 'temurin' - java-version: '11' + java-version: 11 - name: Build and Test uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PreCommit_SQL_Java11.yml b/.github/workflows/beam_PreCommit_SQL_Java11.yml index 4acbecc8d0fc..92b8fe366494 100644 --- a/.github/workflows/beam_PreCommit_SQL_Java11.yml +++ b/.github/workflows/beam_PreCommit_SQL_Java11.yml @@ -78,19 +78,12 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - - name: Set up Java - uses: actions/setup-java@v3.8.0 + - name: Setup environment + uses: ./.github/actions/setup-environment-action with: - distribution: 'temurin' - java-version: '11' - - name: Install Go - uses: actions/setup-go@v4 - with: - go-version: '1.21' - - name: Install Python - uses: actions/setup-python@v4 - with: - python-version: '3.8' + java-version: 11 + python-version: 3.8 + go-version: 1.21 - name: Install Flutter uses: subosito/flutter-action@v2 with: diff --git a/.github/workflows/beam_PreCommit_SQL_Java17.yml b/.github/workflows/beam_PreCommit_SQL_Java17.yml index a5899fff4837..7d6617177613 100644 --- a/.github/workflows/beam_PreCommit_SQL_Java17.yml +++ b/.github/workflows/beam_PreCommit_SQL_Java17.yml @@ -80,20 +80,13 @@ jobs: github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) # The test requires Java 17 and Java 8 versions. # Java 8 is installed second because JAVA_HOME needs to point to Java 8. - - name: Set up Java 17 - uses: actions/setup-java@v3.11.0 + - name: Setup environment + uses: ./.github/actions/setup-environment-action with: - distribution: 'temurin' - java-version: '17' - - name: Set up Java 8 - uses: actions/setup-java@v3.11.0 - with: - distribution: 'temurin' - java-version: '8' - - name: Install Python - uses: actions/setup-python@v4 - with: - python-version: '3.8' + java-version: | + 17 + 8 + python-version: 3.8 - name: Build and Test uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PreCommit_Spotless.yml b/.github/workflows/beam_PreCommit_Spotless.yml index a637addf8628..b4bd436a19f6 100644 --- a/.github/workflows/beam_PreCommit_Spotless.yml +++ b/.github/workflows/beam_PreCommit_Spotless.yml @@ -92,6 +92,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: run Spotless PreCommit script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_Prober_CommunityMetrics.yml b/.github/workflows/beam_Prober_CommunityMetrics.yml index d22b7db8309f..ce788aca216e 100644 --- a/.github/workflows/beam_Prober_CommunityMetrics.yml +++ b/.github/workflows/beam_Prober_CommunityMetrics.yml @@ -69,6 +69,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: Health check probes for the Community Metrics infrastructure uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_Publish_Docker_Snapshots.yml b/.github/workflows/beam_Publish_Docker_Snapshots.yml index cf2520f4635d..01b846eb65d0 100644 --- a/.github/workflows/beam_Publish_Docker_Snapshots.yml +++ b/.github/workflows/beam_Publish_Docker_Snapshots.yml @@ -68,6 +68,8 @@ jobs: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action - name: Authenticate on GCP uses: google-github-actions/setup-gcloud@v0 with: From a07d90eec0efb69490a3bf19df161fb441859d02 Mon Sep 17 00:00:00 2001 From: Damon <damondouglas@users.noreply.github.com> Date: Mon, 23 Oct 2023 09:13:52 -0700 Subject: [PATCH 229/435] [RRIO] Define and implement mock quota aware API (#28893) * Implement gRPC service with transcoder * WIP: defer integration tests for later * Make service executables * remove grpc gateway * Clean up and add messy integration test * Delete flakey integration tests that depend on Docker * Generate java proto * Generate python proto * Python is too much trouble for now * Finalize README * Fix rat and whitespace * Fix whitespace * Clean up per PR comments * Fix gcp logging not capturing severity --- .test-infra/mock-apis/README.md | 116 ++ .test-infra/mock-apis/buf.gen.yaml | 40 + .test-infra/mock-apis/buf.lock | 7 + .test-infra/mock-apis/buf.yaml | 20 + .test-infra/mock-apis/build.gradle | 44 + .test-infra/mock-apis/go.mod | 58 + .test-infra/mock-apis/go.sum | 214 +++ .../mock-apis/proto/echo/v1/echo.proto | 46 + .../src/main/go/cmd/service/echo/main.go | 148 ++ .../src/main/go/cmd/service/refresher/main.go | 121 ++ .../src/main/go/internal/cache/cache.go | 122 ++ .../src/main/go/internal/cache/doc.go | 17 + .../src/main/go/internal/cache/interface.go | 45 + .../src/main/go/internal/cache/redis.go | 59 + .../main/go/internal/environment/variable.go | 118 ++ .../go/internal/environment/variable_test.go | 312 ++++ .../src/main/go/internal/logging/logging.go | 137 ++ .../main/go/internal/logging/logging_test.go | 153 ++ .../src/main/go/internal/metric/doc.go | 17 + .../src/main/go/internal/metric/gcp.go | 77 + .../src/main/go/internal/metric/interface.go | 38 + .../main/go/internal/proto/echo/v1/echo.pb.go | 256 +++ .../go/internal/proto/echo/v1/echo_grpc.pb.go | 107 ++ .../src/main/go/internal/service/echo/echo.go | 185 +++ .../beam/testinfra/mockapis/echo/v1/Echo.java | 1447 +++++++++++++++++ .../mockapis/echo/v1/EchoServiceGrpc.java | 393 +++++ .../mockapis/echo/v1/package-info.java | 20 + build.gradle.kts | 4 + .../beam/checkstyle/suppressions.xml | 1 + settings.gradle.kts | 2 + 30 files changed, 4324 insertions(+) create mode 100644 .test-infra/mock-apis/README.md create mode 100644 .test-infra/mock-apis/buf.gen.yaml create mode 100644 .test-infra/mock-apis/buf.lock create mode 100644 .test-infra/mock-apis/buf.yaml create mode 100644 .test-infra/mock-apis/build.gradle create mode 100644 .test-infra/mock-apis/go.mod create mode 100644 .test-infra/mock-apis/go.sum create mode 100644 .test-infra/mock-apis/proto/echo/v1/echo.proto create mode 100644 .test-infra/mock-apis/src/main/go/cmd/service/echo/main.go create mode 100644 .test-infra/mock-apis/src/main/go/cmd/service/refresher/main.go create mode 100644 .test-infra/mock-apis/src/main/go/internal/cache/cache.go create mode 100644 .test-infra/mock-apis/src/main/go/internal/cache/doc.go create mode 100644 .test-infra/mock-apis/src/main/go/internal/cache/interface.go create mode 100644 .test-infra/mock-apis/src/main/go/internal/cache/redis.go create mode 100644 .test-infra/mock-apis/src/main/go/internal/environment/variable.go create mode 100644 .test-infra/mock-apis/src/main/go/internal/environment/variable_test.go create mode 100644 .test-infra/mock-apis/src/main/go/internal/logging/logging.go create mode 100644 .test-infra/mock-apis/src/main/go/internal/logging/logging_test.go create mode 100644 .test-infra/mock-apis/src/main/go/internal/metric/doc.go create mode 100644 .test-infra/mock-apis/src/main/go/internal/metric/gcp.go create mode 100644 .test-infra/mock-apis/src/main/go/internal/metric/interface.go create mode 100644 .test-infra/mock-apis/src/main/go/internal/proto/echo/v1/echo.pb.go create mode 100644 .test-infra/mock-apis/src/main/go/internal/proto/echo/v1/echo_grpc.pb.go create mode 100644 .test-infra/mock-apis/src/main/go/internal/service/echo/echo.go create mode 100644 .test-infra/mock-apis/src/main/java/org/apache/beam/testinfra/mockapis/echo/v1/Echo.java create mode 100644 .test-infra/mock-apis/src/main/java/org/apache/beam/testinfra/mockapis/echo/v1/EchoServiceGrpc.java create mode 100644 .test-infra/mock-apis/src/main/java/org/apache/beam/testinfra/mockapis/echo/v1/package-info.java diff --git a/.test-infra/mock-apis/README.md b/.test-infra/mock-apis/README.md new file mode 100644 index 000000000000..0165421dbb21 --- /dev/null +++ b/.test-infra/mock-apis/README.md @@ -0,0 +1,116 @@ +<!-- + Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. +--> + +# Overview + +This directory holds code and related artifacts to support API related +integration tests. + +## System review + +The diagram below summarizes the system design. Integration tests use an API +client that makes calls to a backend service. Prior to fulfilling the response, +the service checks and decrements a quota. Said quota persists in a backend +redis instance that is refreshed on an interval by the +[Refresher](./src/main/go/cmd/service/refresher). + +## Echo Service + +The [Echo Service](./src/main/go/cmd/service/echo) implements a simple gRPC +service that echos a payload. See [echo.proto](./proto/echo/v1/echo.proto) +for details. + +```mermaid +flowchart LR + echoClient --> echoSvc + subgraph "Integration Tests" + echoClient[Echo Client] + end + subgraph Backend + echoSvc[Echo Service&lt./src/main/go/cmd/service/echo&gt] + refresher[Refresher&lt./src/main/go/cmd/service/refresher&gt] + redis[redis://:6739] + refresher -- SetQuota&#40&ltstring&gt,&ltint64&gt,&lttime.Duration&gt&#41 --> redis + echoSvc -- DecrementQuota&#40&ltstring&gt&#41 --> redis + end +``` + +# Writing Integration Tests + +TODO: See https://github.com/apache/beam/issues/28859 + +# Development Dependencies + +| Dependency | Reason | +|-----------------------------------------------------|----------------------------------------------------------------------------------------| +| [go](https://go.dev) | For making code changes in this directory. See [go.mod](go.mod) for required version. | +| [buf](https://github.com/bufbuild/buf#installation) | Optional for when making changes to proto. | +| [ko](https://ko.build/install/) | To easily build Go container images. | + +# Testing + +## Unit + +To run unit tests in this project, execute the following command: + +``` +go test ./src/main/go/internal/... +``` + +## Integration + +TODO: See https://github.com/apache/beam/issues/28859 + +# Local Usage + +## Requirements + +To execute the services on your local machine, you'll need [redis](https://redis.io/docs/getting-started/installation/). + +## Execute services + +Follow these steps to run the services on your local machine. + + +1. Start redis + + Start redis using the following command. + ``` + redis-server + ``` + +1. Start the refresher service in a new terminal. + ``` + export CACHE_HOST=localhost:6379; \ + export QUOTA_ID=$(uuidgen); \ + export QUOTA_REFRESH_INTERVAL=10s; \ + export QUOTA_SIZE=100; \ + go run ./src/main/go/cmd/service/refresher + ``` +1. Start the echo service in a new terminal. + ``` + export HTTP_PORT=8080; \ + export GRPC_PORT=50051; \ + export CACHE_HOST=localhost:6379; \ + go run ./src/main/go/cmd/service/echo + ``` + +# Deployment + +TODO: See https://github.com/apache/beam/issues/28709 diff --git a/.test-infra/mock-apis/buf.gen.yaml b/.test-infra/mock-apis/buf.gen.yaml new file mode 100644 index 000000000000..31e57ff2da1e --- /dev/null +++ b/.test-infra/mock-apis/buf.gen.yaml @@ -0,0 +1,40 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# buf.gen.yaml configures proto stub generation using buf. +# +# Requirements: +# - go install google.golang.org/protobuf/cmd/protoc-gen-go@latest +# - go install google.golang.org/grpc/cmd/protoc-gen-go-grpc@latest +# - https://github.com/grpc/grpc-java/blob/master/compiler/README.md#grpc-java-codegen-plugin-for-protobuf-compiler +# - https://grpc.io/docs/languages/python/quickstart/#grpc-tools +# +# Usage: +# Open a terminal in the same directory as this file and run: +# +# buf generate +# +# See https://buf.build/docs/ for more details. + +version: v1 +plugins: +- name: go + out: src/main/go/internal +- name: go-grpc + out: src/main/go/internal +- name: java + out: src/main/java +- name: grpc-java + out: src/main/java \ No newline at end of file diff --git a/.test-infra/mock-apis/buf.lock b/.test-infra/mock-apis/buf.lock new file mode 100644 index 000000000000..1304ceb9973e --- /dev/null +++ b/.test-infra/mock-apis/buf.lock @@ -0,0 +1,7 @@ +# Generated by buf. DO NOT EDIT. +version: v1 +deps: + - remote: buf.build + owner: googleapis + repository: googleapis + commit: 28151c0d0a1641bf938a7672c500e01d diff --git a/.test-infra/mock-apis/buf.yaml b/.test-infra/mock-apis/buf.yaml new file mode 100644 index 000000000000..419e020247f9 --- /dev/null +++ b/.test-infra/mock-apis/buf.yaml @@ -0,0 +1,20 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# Configures buf to include various proto dependencies. +# See buf.build for details. +version: v1 +deps: +- buf.build/googleapis/googleapis \ No newline at end of file diff --git a/.test-infra/mock-apis/build.gradle b/.test-infra/mock-apis/build.gradle new file mode 100644 index 000000000000..64b7e8c614cc --- /dev/null +++ b/.test-infra/mock-apis/build.gradle @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * License); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an AS IS BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + plugins { + id 'org.apache.beam.module' + } + + applyJavaNature( + exportJavadoc: false, + publish: false, + ) + +description = "Apache Beam :: Test Infra :: Mock APIs" +ext.summary = "Supports API related integration tests." + +def guavaVersion = "31.1-jre" +def ioGrpcApiVersion = "1.53.0" +def protobufVersion = "1.55.1" +def protobufJavaVersion = "3.23.2" + + dependencies { + + // Required by autogenerated proto classes. + implementation "io.grpc:grpc-api:${ioGrpcApiVersion}" + implementation "com.google.guava:guava:${guavaVersion}" + implementation "io.grpc:grpc-protobuf:${protobufVersion}" + implementation "com.google.protobuf:protobuf-java:${protobufJavaVersion}" + implementation "io.grpc:grpc-stub:${protobufVersion}" + } \ No newline at end of file diff --git a/.test-infra/mock-apis/go.mod b/.test-infra/mock-apis/go.mod new file mode 100644 index 000000000000..6f88254dba81 --- /dev/null +++ b/.test-infra/mock-apis/go.mod @@ -0,0 +1,58 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// This module contains all Go code used for Beam's SDKs. This file is placed +// in this directory in order to cover the go code required for Java and Python +// containers, as well as the entire Go SDK. Placing this file in the repository +// root is not possible because it causes conflicts with a pre-existing vendor +// directory. +module github.com/apache/beam/test-infra/mock-apis + +go 1.21 + +require ( + cloud.google.com/go/logging v1.8.1 + cloud.google.com/go/monitoring v1.16.0 + github.com/google/go-cmp v0.5.9 + github.com/redis/go-redis/v9 v9.2.1 + google.golang.org/genproto/googleapis/api v0.0.0-20230822172742-b8732ec3820d + google.golang.org/grpc v1.58.2 + google.golang.org/protobuf v1.31.0 +) + +require ( + cloud.google.com/go v0.110.6 // indirect + cloud.google.com/go/compute v1.23.0 // indirect + cloud.google.com/go/compute/metadata v0.2.3 // indirect + cloud.google.com/go/longrunning v0.5.1 // indirect + github.com/cespare/xxhash/v2 v2.2.0 // indirect + github.com/dgryski/go-rendezvous v0.0.0-20200823014737-9f7001d12a5f // indirect + github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da // indirect + github.com/golang/protobuf v1.5.3 // indirect + github.com/google/s2a-go v0.1.4 // indirect + github.com/googleapis/enterprise-certificate-proxy v0.2.4 // indirect + github.com/googleapis/gax-go/v2 v2.12.0 // indirect + go.opencensus.io v0.24.0 // indirect + golang.org/x/crypto v0.13.0 // indirect + golang.org/x/net v0.15.0 // indirect + golang.org/x/oauth2 v0.12.0 // indirect + golang.org/x/sync v0.3.0 // indirect + golang.org/x/sys v0.12.0 // indirect + golang.org/x/text v0.13.0 // indirect + google.golang.org/api v0.128.0 // indirect + google.golang.org/appengine v1.6.7 // indirect + google.golang.org/genproto v0.0.0-20230803162519-f966b187b2e5 // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20230822172742-b8732ec3820d // indirect +) diff --git a/.test-infra/mock-apis/go.sum b/.test-infra/mock-apis/go.sum new file mode 100644 index 000000000000..56d3a2b193dc --- /dev/null +++ b/.test-infra/mock-apis/go.sum @@ -0,0 +1,214 @@ +cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= +cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= +cloud.google.com/go v0.110.6 h1:8uYAkj3YHTP/1iwReuHPxLSbdcyc+dSBbzFMrVwDR6Q= +cloud.google.com/go v0.110.6/go.mod h1:+EYjdK8e5RME/VY/qLCAtuyALQ9q67dvuum8i+H5xsI= +cloud.google.com/go/compute v1.23.0 h1:tP41Zoavr8ptEqaW6j+LQOnyBBhO7OkOMAGrgLopTwY= +cloud.google.com/go/compute v1.23.0/go.mod h1:4tCnrn48xsqlwSAiLf1HXMQk8CONslYbdiEZc9FEIbM= +cloud.google.com/go/compute/metadata v0.2.3 h1:mg4jlk7mCAj6xXp9UJ4fjI9VUI5rubuGBW5aJ7UnBMY= +cloud.google.com/go/compute/metadata v0.2.3/go.mod h1:VAV5nSsACxMJvgaAuX6Pk2AawlZn8kiOGuCv6gTkwuA= +cloud.google.com/go/iam v1.1.1 h1:lW7fzj15aVIXYHREOqjRBV9PsH0Z6u8Y46a1YGvQP4Y= +cloud.google.com/go/iam v1.1.1/go.mod h1:A5avdyVL2tCppe4unb0951eI9jreack+RJ0/d+KUZOU= +cloud.google.com/go/logging v1.8.1 h1:26skQWPeYhvIasWKm48+Eq7oUqdcdbwsCVwz5Ys0FvU= +cloud.google.com/go/logging v1.8.1/go.mod h1:TJjR+SimHwuC8MZ9cjByQulAMgni+RkXeI3wwctHJEI= +cloud.google.com/go/longrunning v0.5.1 h1:Fr7TXftcqTudoyRJa113hyaqlGdiBQkp0Gq7tErFDWI= +cloud.google.com/go/longrunning v0.5.1/go.mod h1:spvimkwdz6SPWKEt/XBij79E9fiTkHSQl/fRUUQJYJc= +cloud.google.com/go/monitoring v1.16.0 h1:rlndy4K8yknMY9JuGe2aK4SbCh21FXoCdX7SAGHmRgI= +cloud.google.com/go/monitoring v1.16.0/go.mod h1:Ptp15HgAyM1fNICAojDMoNc/wUmn67mLHQfyqbw+poY= +github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= +github.com/antihax/optional v1.0.0/go.mod h1:uupD/76wgC+ih3iEmQUL+0Ugr19nfwCT1kdvxnR2qWY= +github.com/bsm/ginkgo/v2 v2.12.0 h1:Ny8MWAHyOepLGlLKYmXG4IEkioBysk6GpaRTLC8zwWs= +github.com/bsm/ginkgo/v2 v2.12.0/go.mod h1:SwYbGRRDovPVboqFv0tPTcG1sN61LM1Z4ARdbAV9g4c= +github.com/bsm/gomega v1.27.10 h1:yeMWxP2pV2fG3FgAODIY8EiRE3dy0aeFYt4l7wh6yKA= +github.com/bsm/gomega v1.27.10/go.mod h1:JyEr/xRbxbtgWNi8tIEVPUYZ5Dzef52k01W3YH0H+O0= +github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= +github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= +github.com/cespare/xxhash/v2 v2.2.0 h1:DC2CZ1Ep5Y4k3ZQ899DldepgrayRUGE6BBZ/cd9Cj44= +github.com/cespare/xxhash/v2 v2.2.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= +github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= +github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= +github.com/cncf/udpa/go v0.0.0-20201120205902-5459f2c99403/go.mod h1:WmhPx2Nbnhtbo57+VJT5O0JRkEi1Wbu0z5j0R8u5Hbk= +github.com/cncf/udpa/go v0.0.0-20210930031921-04548b0d99d4/go.mod h1:6pvJx4me5XPnfI9Z40ddWsdw2W/uZgQLFXToKeRcDiI= +github.com/cncf/xds/go v0.0.0-20210805033703-aa0b78936158/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= +github.com/cncf/xds/go v0.0.0-20210922020428-25de7278fc84/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= +github.com/cncf/xds/go v0.0.0-20211011173535-cb28da3451f1/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= +github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= +github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= +github.com/dgryski/go-rendezvous v0.0.0-20200823014737-9f7001d12a5f h1:lO4WD4F/rVNCu3HqELle0jiPLLBs70cWOduZpkS1E78= +github.com/dgryski/go-rendezvous v0.0.0-20200823014737-9f7001d12a5f/go.mod h1:cuUVRXasLTGF7a8hSLbxyZXjz+1KgoB3wDUb6vlszIc= +github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= +github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= +github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98= +github.com/envoyproxy/go-control-plane v0.9.9-0.20201210154907-fd9021fe5dad/go.mod h1:cXg6YxExXjJnVBQHBLXeUAgxn2UodCpnH306RInaBQk= +github.com/envoyproxy/go-control-plane v0.9.10-0.20210907150352-cf90f659a021/go.mod h1:AFq3mo9L8Lqqiid3OhADV3RfLJnjiw63cSpi+fDTRC0= +github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= +github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= +github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= +github.com/golang/groupcache v0.0.0-20200121045136-8c9f03a8e57e/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= +github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da h1:oI5xCqsCo564l8iNU+DwB5epxmsaqB+rhGL0m5jtYqE= +github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= +github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= +github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= +github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= +github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= +github.com/golang/protobuf v1.3.3/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= +github.com/golang/protobuf v1.4.0-rc.1/go.mod h1:ceaxUfeHdC40wWswd/P6IGgMaK3YpKi5j83Wpe3EHw8= +github.com/golang/protobuf v1.4.0-rc.1.0.20200221234624-67d41d38c208/go.mod h1:xKAWHe0F5eneWXFV3EuXVDTCmh+JuBKY0li0aMyXATA= +github.com/golang/protobuf v1.4.0-rc.2/go.mod h1:LlEzMj4AhA7rCAGe4KMBDvJI+AwstrUpVNzEA03Pprs= +github.com/golang/protobuf v1.4.0-rc.4.0.20200313231945-b860323f09d0/go.mod h1:WU3c8KckQ9AFe+yFwt9sWVRKCVIyN9cPHBJSNnbL67w= +github.com/golang/protobuf v1.4.0/go.mod h1:jodUvKwWbYaEsadDk5Fwe5c77LiNKVO9IDvqG2KuDX0= +github.com/golang/protobuf v1.4.1/go.mod h1:U8fpvMrcmy5pZrNK1lt4xCsGvpyWQ/VVv6QDs8UjoX8= +github.com/golang/protobuf v1.4.2/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= +github.com/golang/protobuf v1.4.3/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= +github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk= +github.com/golang/protobuf v1.5.2/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY= +github.com/golang/protobuf v1.5.3 h1:KhyjKVUg7Usr/dYsdSqoFveMYd5ko72D+zANwlG1mmg= +github.com/golang/protobuf v1.5.3/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY= +github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= +github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= +github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= +github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.5.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.5.3/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.5.9 h1:O2Tfq5qg4qc4AmwVlvv0oLiVAGB7enBSJ2x2DqQFi38= +github.com/google/go-cmp v0.5.9/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= +github.com/google/s2a-go v0.1.4 h1:1kZ/sQM3srePvKs3tXAvQzo66XfcReoqFpIpIccE7Oc= +github.com/google/s2a-go v0.1.4/go.mod h1:Ej+mSEMGRnqRzjc7VtF+jdBwYG5fuJfiZ8ELkjEwM0A= +github.com/google/uuid v1.1.2/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= +github.com/googleapis/enterprise-certificate-proxy v0.2.4 h1:uGy6JWR/uMIILU8wbf+OkstIrNiMjGpEIyhx8f6W7s4= +github.com/googleapis/enterprise-certificate-proxy v0.2.4/go.mod h1:AwSRAtLfXpU5Nm3pW+v7rGDHp09LsPtGY9MduiEsR9k= +github.com/googleapis/gax-go/v2 v2.12.0 h1:A+gCJKdRfqXkr+BIRGtZLibNXf0m1f9E4HG56etFpas= +github.com/googleapis/gax-go/v2 v2.12.0/go.mod h1:y+aIqrI5eb1YGMVJfuV3185Ts/D7qKpsEkdD5+I6QGU= +github.com/grpc-ecosystem/grpc-gateway v1.16.0/go.mod h1:BDjrQk3hbvj6Nolgz8mAMFbcEtjT1g+wF4CSlocrBnw= +github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= +github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= +github.com/redis/go-redis/v9 v9.2.1 h1:WlYJg71ODF0dVspZZCpYmoF1+U1Jjk9Rwd7pq6QmlCg= +github.com/redis/go-redis/v9 v9.2.1/go.mod h1:hdY0cQFCN4fnSYT6TkisLufl/4W5UIXyv0b/CLO2V2M= +github.com/rogpeppe/fastuuid v1.2.0/go.mod h1:jVj6XXZzXRy/MSR5jhDC/2q6DgLz+nrA6LYCDYWNEvQ= +github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= +github.com/stretchr/objx v0.4.0/go.mod h1:YvHI0jy2hoMjB+UWwv71VJQ9isScKT/TqJzVSSt89Yw= +github.com/stretchr/objx v0.5.0/go.mod h1:Yh+to48EsGEfYuaHDzXPcE3xhTkx73EhmCGUpEOglKo= +github.com/stretchr/testify v1.5.1/go.mod h1:5W2xD1RspED5o8YsWQXVCued0rvSQ+mT+I5cxcmMvtA= +github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= +github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= +github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= +github.com/stretchr/testify v1.8.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= +github.com/yuin/goldmark v1.4.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5taEt/CY= +go.opencensus.io v0.24.0 h1:y73uSU6J157QMP2kn2r30vwW1A2W2WFwSCGnAVxeaD0= +go.opencensus.io v0.24.0/go.mod h1:vNK8G9p7aAivkbmorf4v+7Hgx+Zs0yY+0fOtgBfjQKo= +go.opentelemetry.io/proto/otlp v0.7.0/go.mod h1:PqfVotwruBrMGOCsRd/89rSnXhoiJIqeYNgFYFoEGnI= +golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= +golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= +golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= +golang.org/x/crypto v0.0.0-20220314234659-1baeb1ce4c0b/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= +golang.org/x/crypto v0.13.0 h1:mvySKfSWJ+UKUii46M40LOvyWfN0s2U+46/jDd0e6Ck= +golang.org/x/crypto v0.13.0/go.mod h1:y6Z2r+Rw4iayiXXAIxJIDAJ1zMW4yaTpebo8fPOliYc= +golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= +golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= +golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= +golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91VN4djpZkiMVwK6gcyfeH4XE8wZrZaV4= +golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20190108225652-1e06a53dbb7e/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190603091049-60506f45cf65/go.mod h1:HSz+uSET+XFnRR8LxR5pz3Of3rY3CfYBVs4xY44aLks= +golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20200822124328-c89045814202/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= +golang.org/x/net v0.0.0-20201110031124-69a78807bb2b/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= +golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= +golang.org/x/net v0.0.0-20211112202133-69e39bad7dc2/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= +golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= +golang.org/x/net v0.15.0 h1:ugBLEUaxABaB5AJqW9enI0ACdci2RUd4eP51NTBvuJ8= +golang.org/x/net v0.15.0/go.mod h1:idbUs1IY1+zTqbi8yxTbhexhEEk5ur9LInksu6HrEpk= +golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= +golang.org/x/oauth2 v0.0.0-20200107190931-bf48bf16ab8d/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= +golang.org/x/oauth2 v0.12.0 h1:smVPGxink+n1ZI5pkQa8y6fZT0RW0MgCO5bFpepy4B4= +golang.org/x/oauth2 v0.12.0/go.mod h1:A74bZ3aGXgCY0qaIC9Ahg6Lglin4AMAco8cIv9baba4= +golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.3.0 h1:ftCYgMx6zT/asHUrPw8BLLscYtGznsLAnjq5RH9P66E= +golang.org/x/sync v0.3.0/go.mod h1:FU7BRWz2tNW+3quACPkgCx/L+uEAv1htQ0V83Z9Rj+Y= +golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200323222414-85ca7c5b95cd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.12.0 h1:CM0HF96J0hcLAwsHPJZjfdNzs0gftsLfgKt57wWHJ0o= +golang.org/x/sys v0.12.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= +golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= +golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= +golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= +golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= +golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= +golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= +golang.org/x/text v0.3.8/go.mod h1:E6s5w1FMmriuDzIBO73fBruAKo1PCIq6d2Q6DHfQ8WQ= +golang.org/x/text v0.13.0 h1:ablQoSUd0tRdKxZewP80B+BaqeKJuVhuRxj/dkrun3k= +golang.org/x/text v0.13.0/go.mod h1:TvPlkZtksWOMsz7fbANvkp4WM8x/WCo/om8BMLbz+aE= +golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= +golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= +golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= +golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.1.12/go.mod h1:hNGJHUnrk76NpqgfD5Aqm5Crs+Hm0VOH/i9J2+nxYbc= +golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +google.golang.org/api v0.128.0 h1:RjPESny5CnQRn9V6siglged+DZCgfu9l6mO9dkX9VOg= +google.golang.org/api v0.128.0/go.mod h1:Y611qgqaE92On/7g65MQgxYul3c0rEB894kniWLY750= +google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= +google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= +google.golang.org/appengine v1.6.7 h1:FZR1q0exgwxzPzp/aF+VccGrSfxfPpkBqjIIEq3ru6c= +google.golang.org/appengine v1.6.7/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc= +google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= +google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= +google.golang.org/genproto v0.0.0-20200513103714-09dca8ec2884/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= +google.golang.org/genproto v0.0.0-20200526211855-cb27e3aa2013/go.mod h1:NbSheEEYHJ7i3ixzK3sjbqSGDJWnxyFXZblF3eUsNvo= +google.golang.org/genproto v0.0.0-20230803162519-f966b187b2e5 h1:L6iMMGrtzgHsWofoFcihmDEMYeDR9KN/ThbPWGrh++g= +google.golang.org/genproto v0.0.0-20230803162519-f966b187b2e5/go.mod h1:oH/ZOT02u4kWEp7oYBGYFFkCdKS/uYR9Z7+0/xuuFp8= +google.golang.org/genproto/googleapis/api v0.0.0-20230822172742-b8732ec3820d h1:DoPTO70H+bcDXcd39vOqb2viZxgqeBeSGtZ55yZU4/Q= +google.golang.org/genproto/googleapis/api v0.0.0-20230822172742-b8732ec3820d/go.mod h1:KjSP20unUpOx5kyQUFa7k4OJg0qeJ7DEZflGDu2p6Bk= +google.golang.org/genproto/googleapis/rpc v0.0.0-20230822172742-b8732ec3820d h1:uvYuEyMHKNt+lT4K3bN6fGswmK8qSvcreM3BwjDh+y4= +google.golang.org/genproto/googleapis/rpc v0.0.0-20230822172742-b8732ec3820d/go.mod h1:+Bk1OCOj40wS2hwAMA+aCW9ypzm63QTBBHp6lQ3p+9M= +google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= +google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= +google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= +google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= +google.golang.org/grpc v1.33.1/go.mod h1:fr5YgcSWrqhRRxogOsw7RzIpsmvOZ6IcH4kBYTpR3n0= +google.golang.org/grpc v1.33.2/go.mod h1:JMHMWHQWaTccqQQlmk3MJZS+GWXOdAesneDmEnv2fbc= +google.golang.org/grpc v1.36.0/go.mod h1:qjiiYl8FncCW8feJPdyg3v6XW24KsRHe+dy9BAGRRjU= +google.golang.org/grpc v1.45.0/go.mod h1:lN7owxKUQEqMfSyQikvvk5tf/6zMPsrK+ONuO11+0rQ= +google.golang.org/grpc v1.58.2 h1:SXUpjxeVF3FKrTYQI4f4KvbGD5u2xccdYdurwowix5I= +google.golang.org/grpc v1.58.2/go.mod h1:tgX3ZQDlNJGU96V6yHh1T/JeoBQ2TXdr43YbYSsCJk0= +google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= +google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= +google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM= +google.golang.org/protobuf v1.20.1-0.20200309200217-e05f789c0967/go.mod h1:A+miEFZTKqfCUM6K7xSMQL9OKL/b6hQv+e19PK+JZNE= +google.golang.org/protobuf v1.21.0/go.mod h1:47Nbq4nVaFHyn7ilMalzfO3qCViNmqZ2kzikPIcrTAo= +google.golang.org/protobuf v1.22.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= +google.golang.org/protobuf v1.23.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= +google.golang.org/protobuf v1.23.1-0.20200526195155-81db48ad09cc/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= +google.golang.org/protobuf v1.25.0/go.mod h1:9JNX74DMeImyA3h4bdi1ymwjUzf21/xIlbajtzgsN7c= +google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= +google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= +google.golang.org/protobuf v1.31.0 h1:g0LDEJHgrBl9N9r17Ru3sqWhkIx2NB67okBHPwC7hs8= +google.golang.org/protobuf v1.31.0/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= +gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= +gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v2 v2.2.3/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= +gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= +honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= +honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= diff --git a/.test-infra/mock-apis/proto/echo/v1/echo.proto b/.test-infra/mock-apis/proto/echo/v1/echo.proto new file mode 100644 index 000000000000..826dc0f233fd --- /dev/null +++ b/.test-infra/mock-apis/proto/echo/v1/echo.proto @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Protocol buffers describing a simple mock API that echos a request. + */ + +syntax = "proto3"; + +package proto.echo.v1; +option go_package = "proto/echo/v1"; +option java_package = "org.apache.beam.testinfra.mockapis.echo.v1"; + +// EchoService simulates a mock API that echos a request. +service EchoService { + + // Echo an EchoRequest payload in an EchoResponse. + rpc Echo(EchoRequest) returns (EchoResponse) {} +} + +// The request to echo a payload. +message EchoRequest { + string id = 1; + bytes payload = 2; +} + +// The response echo of a request payload. +message EchoResponse { + string id = 1; + bytes payload = 2; +} \ No newline at end of file diff --git a/.test-infra/mock-apis/src/main/go/cmd/service/echo/main.go b/.test-infra/mock-apis/src/main/go/cmd/service/echo/main.go new file mode 100644 index 000000000000..891468a603a1 --- /dev/null +++ b/.test-infra/mock-apis/src/main/go/cmd/service/echo/main.go @@ -0,0 +1,148 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// echo is an executable that runs the echov1.EchoService. +package main + +import ( + "context" + "fmt" + "log/slog" + "net" + "net/http" + "os" + "os/signal" + + gcplogging "cloud.google.com/go/logging" + "github.com/apache/beam/test-infra/mock-apis/src/main/go/internal/cache" + "github.com/apache/beam/test-infra/mock-apis/src/main/go/internal/environment" + "github.com/apache/beam/test-infra/mock-apis/src/main/go/internal/logging" + "github.com/apache/beam/test-infra/mock-apis/src/main/go/internal/service/echo" + "github.com/redis/go-redis/v9" + "google.golang.org/grpc" +) + +var ( + env = []environment.Variable{ + environment.CacheHost, + environment.GrpcPort, + environment.HttpPort, + } + + logger *slog.Logger + logAttrs []slog.Attr + opts = &logging.Options{ + Name: "echo", + } +) + +func init() { + for _, v := range env { + logAttrs = append(logAttrs, slog.Attr{ + Key: v.Key(), + Value: slog.StringValue(v.Value()), + }) + } +} + +func main() { + ctx := context.Background() + + if !environment.ProjectId.Missing() { + client, err := gcplogging.NewClient(ctx, environment.ProjectId.Value()) + if err != nil { + slog.LogAttrs(ctx, slog.LevelError, err.Error(), logAttrs...) + os.Exit(1) + } + + opts.Client = client + } + + logger = logging.New(opts) + + if err := run(ctx); err != nil { + logger.LogAttrs(ctx, slog.LevelError, err.Error(), logAttrs...) + os.Exit(1) + } +} + +func run(ctx context.Context) error { + ctx, cancel := signal.NotifyContext(ctx, os.Interrupt) + defer cancel() + + if err := environment.Missing(env...); err != nil { + return err + } + + grpcPort, err := environment.GrpcPort.Int() + if err != nil { + return err + } + grpcAddress := fmt.Sprintf(":%v", grpcPort) + + httpPort, err := environment.HttpPort.Int() + if err != nil { + return err + } + httpAddress := fmt.Sprintf(":%v", httpPort) + + s := grpc.NewServer() + defer s.GracefulStop() + + r := redis.NewClient(&redis.Options{ + Addr: environment.CacheHost.Value(), + }) + + echoOpts := &echo.Options{ + Decrementer: (*cache.RedisCache)(r), + LoggingAttrs: logAttrs, + Logger: logger, + // TODO(damondouglas): add GCP metrics client + // MetricsWriter: + } + + handler, err := echo.Register(s, echoOpts) + if err != nil { + return err + } + + logger.LogAttrs(ctx, slog.LevelInfo, "starting service", logAttrs...) + + lis, err := net.Listen("tcp", grpcAddress) + if err != nil { + return err + } + + errChan := make(chan error) + go func() { + if err := s.Serve(lis); err != nil { + errChan <- err + } + }() + + go func() { + if err := http.ListenAndServe(httpAddress, handler); err != nil { + errChan <- err + } + }() + + select { + case err := <-errChan: + return err + case <-ctx.Done(): + logger.LogAttrs(ctx, slog.LevelInfo, "shutting down", logAttrs...) + return nil + } +} diff --git a/.test-infra/mock-apis/src/main/go/cmd/service/refresher/main.go b/.test-infra/mock-apis/src/main/go/cmd/service/refresher/main.go new file mode 100644 index 000000000000..63e32671935c --- /dev/null +++ b/.test-infra/mock-apis/src/main/go/cmd/service/refresher/main.go @@ -0,0 +1,121 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// refresher is an executable that runs the cache.Refresher service. +package main + +import ( + "context" + "log/slog" + "os" + "os/signal" + + gcplogging "cloud.google.com/go/logging" + "github.com/apache/beam/test-infra/mock-apis/src/main/go/internal/cache" + "github.com/apache/beam/test-infra/mock-apis/src/main/go/internal/environment" + "github.com/apache/beam/test-infra/mock-apis/src/main/go/internal/logging" + "github.com/redis/go-redis/v9" +) + +var ( + env = []environment.Variable{ + environment.CacheHost, + environment.QuotaId, + environment.QuotaSize, + environment.QuotaRefreshInterval, + } + logger *slog.Logger + logAttrs []slog.Attr + opts = &logging.Options{ + Name: "refresher", + } +) + +func init() { + for _, v := range env { + logAttrs = append(logAttrs, slog.Attr{ + Key: v.Key(), + Value: slog.StringValue(v.Value()), + }) + } +} + +func main() { + ctx := context.Background() + + if !environment.ProjectId.Missing() { + client, err := gcplogging.NewClient(ctx, environment.ProjectId.Value()) + if err != nil { + slog.LogAttrs(ctx, slog.LevelError, err.Error(), logAttrs...) + os.Exit(1) + } + + opts.Client = client + } + + logger = logging.New(opts) + if err := run(ctx); err != nil { + logger.LogAttrs(ctx, slog.LevelError, err.Error(), logAttrs...) + os.Exit(1) + } +} + +func run(ctx context.Context) error { + ctx, cancel := signal.NotifyContext(ctx, os.Interrupt) + defer cancel() + + if err := environment.Missing(env...); err != nil { + return err + } + + size, err := environment.QuotaSize.UInt64() + if err != nil { + return err + } + + interval, err := environment.QuotaRefreshInterval.Duration() + if err != nil { + return err + } + + r := redis.NewClient(&redis.Options{ + Addr: environment.CacheHost.Value(), + }) + + opts := &cache.Options{ + Logger: logger, + Setter: (*cache.RedisCache)(r), + } + + ref, err := cache.NewRefresher(ctx, opts) + if err != nil { + return err + } + + errChan := make(chan error) + go func() { + if err := ref.Refresh(ctx, environment.QuotaId.Value(), size, interval); err != nil { + errChan <- err + } + }() + + select { + case err := <-errChan: + return err + case <-ctx.Done(): + return nil + } + +} diff --git a/.test-infra/mock-apis/src/main/go/internal/cache/cache.go b/.test-infra/mock-apis/src/main/go/internal/cache/cache.go new file mode 100644 index 000000000000..cab20ad998cc --- /dev/null +++ b/.test-infra/mock-apis/src/main/go/internal/cache/cache.go @@ -0,0 +1,122 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package cache + +import ( + "context" + "errors" + "fmt" + "log/slog" + "reflect" + "time" + + "github.com/apache/beam/test-infra/mock-apis/src/main/go/internal/logging" +) + +var ( + + // ErrNotExist is an error indicating that a resource does not exist + ErrNotExist = errors.New("resource does not exist") +) + +// IsNotExist is true when err is ErrNotExist. +func IsNotExist(err error) bool { + return errors.Is(err, ErrNotExist) +} + +// Options for running the Refresher. +type Options struct { + Setter UInt64Setter + Logger *slog.Logger +} + +// Refresher refreshes a value in a cache on a set interval. +type Refresher struct { + opts *Options + stop chan struct{} +} + +// NewRefresher instantiates a Refresher. +func NewRefresher(ctx context.Context, opts *Options) (*Refresher, error) { + if opts.Logger == nil { + opts.Logger = logging.New(&logging.Options{ + Name: reflect.TypeOf((*Refresher)(nil)).PkgPath(), + }) + } + + if opts.Setter == nil { + return nil, fmt.Errorf("%T.Setter is nil but required", opts) + } + + if err := opts.Setter.Alive(ctx); err != nil { + return nil, err + } + + ref := &Refresher{ + opts: opts, + } + + return ref, nil +} + +// Stop the Refresher. +func (ref *Refresher) Stop() { + ref.stop <- struct{}{} +} + +// Refresh the size of the associated key at an interval. +func (ref *Refresher) Refresh(ctx context.Context, key string, size uint64, interval time.Duration) error { + ctx, cancel := context.WithCancel(ctx) + defer cancel() + ref.stop = make(chan struct{}) + attrs := []slog.Attr{ + { + Key: "key", + Value: slog.StringValue(key), + }, + { + Key: "size", + Value: slog.Uint64Value(size), + }, + { + Key: "interval", + Value: slog.StringValue(interval.String()), + }, + } + + ref.opts.Logger.LogAttrs(ctx, slog.LevelInfo, "starting refresher service", attrs...) + + if err := ref.opts.Setter.Set(ctx, key, size, interval); err != nil { + return err + } + ref.opts.Logger.LogAttrs(ctx, slog.LevelDebug, "successful initial refresh", attrs...) + + tick := time.Tick(interval) + for { + select { + case <-tick: + if err := ref.opts.Setter.Set(ctx, key, size, interval); err != nil { + return err + } + ref.opts.Logger.LogAttrs(ctx, slog.LevelDebug, "refresh successful", attrs...) + case <-ref.stop: + ref.opts.Logger.LogAttrs(ctx, slog.LevelInfo, "stopping refresher service", attrs...) + return nil + case <-ctx.Done(): + return nil + } + } +} diff --git a/.test-infra/mock-apis/src/main/go/internal/cache/doc.go b/.test-infra/mock-apis/src/main/go/internal/cache/doc.go new file mode 100644 index 000000000000..c0f937aa8d59 --- /dev/null +++ b/.test-infra/mock-apis/src/main/go/internal/cache/doc.go @@ -0,0 +1,17 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// Package cache stores and retrieves data from a cache. +package cache diff --git a/.test-infra/mock-apis/src/main/go/internal/cache/interface.go b/.test-infra/mock-apis/src/main/go/internal/cache/interface.go new file mode 100644 index 000000000000..8266f7205885 --- /dev/null +++ b/.test-infra/mock-apis/src/main/go/internal/cache/interface.go @@ -0,0 +1,45 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package cache + +import ( + "context" + "time" +) + +// HealthChecker checks the health and availability of a resource. +type HealthChecker interface { + + // Alive checks whether the resource is healthy and available. + Alive(ctx context.Context) error +} + +// UInt64Setter associates a key with a value for an expiry time.Duration. +type UInt64Setter interface { + HealthChecker + + // Set a key with a value for an expiry time.Duration. + Set(ctx context.Context, key string, value uint64, expiry time.Duration) error +} + +// Decrementer decrements a value associated with a key. +type Decrementer interface { + HealthChecker + + // Decrement the value associated with a key; returns the value after + // decrementing it. + Decrement(ctx context.Context, key string) (int64, error) +} diff --git a/.test-infra/mock-apis/src/main/go/internal/cache/redis.go b/.test-infra/mock-apis/src/main/go/internal/cache/redis.go new file mode 100644 index 000000000000..51ad73061cb7 --- /dev/null +++ b/.test-infra/mock-apis/src/main/go/internal/cache/redis.go @@ -0,0 +1,59 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package cache + +import ( + "context" + "time" + + "github.com/redis/go-redis/v9" +) + +// Validate interface implementations +var _ UInt64Setter = &RedisCache{} +var _ Decrementer = &RedisCache{} +var _ HealthChecker = &RedisCache{} + +// RedisCache implements a Decrementer and a Refresher. +type RedisCache redis.Client + +// Set implements Refresher's Set method using a redis cache where expiry of 0 +// has no expiration. Returns any error from the redis client. +func (client *RedisCache) Set(ctx context.Context, key string, value uint64, expiry time.Duration) error { + r := (*redis.Client)(client) + return r.Set(ctx, key, value, expiry).Err() +} + +// Decrement implements Decrementer's Decrement method using a redis cache. +// Returns an error when the key does not exist or from the redis client. +func (client *RedisCache) Decrement(ctx context.Context, key string) (int64, error) { + r := (*redis.Client)(client) + v, err := r.Exists(ctx, key).Result() + if err != nil { + return -1, err + } + if v == 0 { + return -1, ErrNotExist + } + return r.Decr(ctx, key).Result() +} + +// Alive implements HealthChecker's Alive checking the availability of a +// redis cache. Returns an error if no successful connection. +func (client *RedisCache) Alive(ctx context.Context) error { + r := (*redis.Client)(client) + return r.Ping(ctx).Err() +} diff --git a/.test-infra/mock-apis/src/main/go/internal/environment/variable.go b/.test-infra/mock-apis/src/main/go/internal/environment/variable.go new file mode 100644 index 000000000000..b1e3a8ec4011 --- /dev/null +++ b/.test-infra/mock-apis/src/main/go/internal/environment/variable.go @@ -0,0 +1,118 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// Package environment provides helpers for interacting with environment variables. +package environment + +import ( + "fmt" + "os" + "strconv" + "strings" + "time" +) + +var ( + // HttpPort is the port to bind an HTTP service. + HttpPort Variable = "HTTP_PORT" + + // GrpcPort is the port to bind a gRPC service. + GrpcPort Variable = "GRPC_PORT" + + // CacheHost is the host address of the cache. + CacheHost Variable = "CACHE_HOST" + + // ProjectId is the ID of the Google Cloud host project. + ProjectId Variable = "PROJECT_ID" + + // QuotaId uniquely identifies a quota measure. + QuotaId Variable = "QUOTA_ID" + + // QuotaSize specifies the size of the quota. + QuotaSize Variable = "QUOTA_SIZE" + + // QuotaRefreshInterval configures how often a quota is refreshed. + QuotaRefreshInterval Variable = "QUOTA_REFRESH_INTERVAL" +) + +// Variable defines an environment variable via a string type alias. +// Variable's string defaultValue assigns the system environment variable key. +type Variable string + +// Default a default value to the system environment. +func (v Variable) Default(value string) error { + if v.Missing() { + return os.Setenv((string)(v), value) + } + return nil +} + +// MustDefault a default value to the system environment. Panics on error. +func (v Variable) MustDefault(value string) { + if err := v.Default(value); err != nil { + panic(err) + } +} + +// Missing reports whether the system environment variable is an empty string. +func (v Variable) Missing() bool { + return v.Value() == "" +} + +// Key returns the system environment variable key. +func (v Variable) Key() string { + return (string)(v) +} + +// Value returns the system environment variable value. +func (v Variable) Value() string { + return os.Getenv((string)(v)) +} + +// Int returns the system environment variable parsed as an int. +func (v Variable) Int() (int, error) { + return strconv.Atoi(v.Value()) +} + +// UInt64 returns the system environment variable value parsed as a uint64. +func (v Variable) UInt64() (uint64, error) { + return strconv.ParseUint(v.Value(), 10, 64) +} + +// Duration returns the system environment variable value parsed as time.Duration. +func (v Variable) Duration() (time.Duration, error) { + return time.ParseDuration(v.Value()) +} + +// KeyValue returns a concatenated string of the system environment variable's +// <key>=<defaultValue>. +func (v Variable) KeyValue() string { + return fmt.Sprintf("%s=%s", (string)(v), v.Value()) +} + +// Missing reports as an error listing all Variable among vars that are +// not assigned in the system environment. +func Missing(vars ...Variable) error { + var missing []string + for _, v := range vars { + if v.Missing() { + missing = append(missing, v.KeyValue()) + } + } + if len(missing) > 0 { + return fmt.Errorf("variables empty but expected from environment: %s", strings.Join(missing, "; ")) + } + return nil +} diff --git a/.test-infra/mock-apis/src/main/go/internal/environment/variable_test.go b/.test-infra/mock-apis/src/main/go/internal/environment/variable_test.go new file mode 100644 index 000000000000..b566f14d446b --- /dev/null +++ b/.test-infra/mock-apis/src/main/go/internal/environment/variable_test.go @@ -0,0 +1,312 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package environment + +import ( + "errors" + "os" + "testing" + + "github.com/google/go-cmp/cmp" +) + +func TestMissing(t *testing.T) { + type args struct { + vars []Variable + values []string + } + tests := []struct { + name string + args args + want error + }{ + { + name: "{}", + args: args{}, + }, + { + name: "{A=}", + args: args{ + vars: []Variable{ + "A", + }, + values: []string{ + "", + }, + }, + want: errors.New("variables empty but expected from environment: A="), + }, + { + name: "{A=1}", + args: args{ + vars: []Variable{ + "A", + }, + values: []string{ + "1", + }, + }, + want: nil, + }, + { + name: "{A=; B=}", + args: args{ + vars: []Variable{ + "A", + "B", + }, + values: []string{ + "", + "", + }, + }, + want: errors.New("variables empty but expected from environment: A=; B="), + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + var got, want string + clearVars(tt.args.vars...) + set(t, tt.args.vars, tt.args.values) + err := Missing(tt.args.vars...) + if err != nil { + got = err.Error() + } + if tt.want != nil { + want = tt.want.Error() + } + if diff := cmp.Diff(want, got); diff != "" { + t.Errorf("Missing() error returned unexpected difference in error messages (-want +got):\n%s", diff) + } + }) + } +} + +func TestVariable_Default(t *testing.T) { + type args struct { + setValue string + defaultValue string + } + tests := []struct { + name string + v Variable + args args + want string + }{ + { + name: "environment variable not set", + v: "A", + args: args{ + defaultValue: "1", + }, + want: "1", + }, + { + name: "environment variable default is overridden by set value", + v: "A", + args: args{ + setValue: "2", + defaultValue: "1", + }, + want: "2", + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + clearVars(tt.v) + if tt.args.setValue != "" { + set(t, []Variable{tt.v}, []string{tt.args.setValue}) + } + if err := tt.v.Default(tt.args.defaultValue); err != nil { + t.Fatalf("could not set default environment variable value during test execution: %v", err) + } + got := os.Getenv(tt.v.Key()) + if got != tt.want { + t.Errorf("Default() = %s, want %s", got, tt.want) + } + }) + } +} + +func TestVariable_KeyValue(t *testing.T) { + tests := []struct { + name string + v Variable + value string + want string + }{ + { + name: "environment variable not set", + v: "A", + want: "A=", + }, + { + name: "environment variable is set", + v: "A", + value: "1", + want: "A=1", + }, + } + for _, tt := range tests { + clearVars(tt.v) + t.Run(tt.name, func(t *testing.T) { + set(t, []Variable{tt.v}, []string{tt.value}) + got := tt.v.KeyValue() + if got != tt.want { + t.Errorf("KeyValue() = %s, want %s", got, tt.want) + } + }) + } +} + +func TestVariable_Missing(t *testing.T) { + type args struct { + setValue string + defaultValue string + } + tests := []struct { + name string + args args + v Variable + want bool + }{ + { + name: "no default and not set", + args: args{}, + v: "A", + want: true, + }, + { + name: "has default but not set", + args: args{ + defaultValue: "1", + }, + v: "A", + want: false, + }, + { + name: "no default but set", + args: args{ + setValue: "1", + }, + v: "A", + want: false, + }, + { + name: "has default and set", + args: args{ + setValue: "2", + defaultValue: "1", + }, + v: "A", + want: false, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + clearVars(tt.v) + if tt.args.defaultValue != "" { + if err := tt.v.Default(tt.args.defaultValue); err != nil { + t.Fatalf("could not set default environment variable value during test execution: %v", err) + } + } + if tt.args.setValue != "" { + set(t, []Variable{tt.v}, []string{tt.args.setValue}) + } + if got := tt.v.Missing(); got != tt.want { + t.Errorf("Missing() = %v, want %v", got, tt.want) + } + }) + } +} + +func TestVariable_Value(t *testing.T) { + type args struct { + setValue string + defaultValue string + } + tests := []struct { + name string + args args + v Variable + want string + }{ + { + name: "no default and not set", + args: args{}, + v: "A", + want: "", + }, + { + name: "has default but not set", + args: args{ + defaultValue: "1", + }, + v: "A", + want: "1", + }, + { + name: "no default but set", + args: args{ + setValue: "1", + }, + v: "A", + want: "1", + }, + { + name: "has default and set", + args: args{ + setValue: "2", + defaultValue: "1", + }, + v: "A", + want: "2", + }, + } + for _, tt := range tests { + clearVars(tt.v) + if tt.args.defaultValue != "" { + if err := tt.v.Default(tt.args.defaultValue); err != nil { + t.Fatalf("could not set default environment variable value during test execution: %v", err) + } + } + if tt.args.setValue != "" { + set(t, []Variable{tt.v}, []string{tt.args.setValue}) + } + t.Run(tt.name, func(t *testing.T) { + if got := tt.v.Value(); got != tt.want { + t.Errorf("Value() = %v, want %v", got, tt.want) + } + }) + } +} + +func clearVars(vars ...Variable) { + for _, k := range vars { + _ = os.Setenv(k.Key(), "") + } +} + +func set(t *testing.T, vars []Variable, values []string) { + if len(vars) != len(values) { + t.Fatalf("test cases should be configured with matching args.vars and args.values: len(tt.args.vars): %v != len(tt.args.values): %v", len(vars), len(values)) + } + for i := range vars { + key := vars[i].Key() + value := values[i] + _ = os.Setenv(key, value) + } +} diff --git a/.test-infra/mock-apis/src/main/go/internal/logging/logging.go b/.test-infra/mock-apis/src/main/go/internal/logging/logging.go new file mode 100644 index 000000000000..53cead40b0d8 --- /dev/null +++ b/.test-infra/mock-apis/src/main/go/internal/logging/logging.go @@ -0,0 +1,137 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// Package logging performs structured output of log entries. +package logging + +import ( + "context" + "encoding/json" + "io" + "log/slog" + "os" + "path" + "runtime" + "sync" + + "cloud.google.com/go/logging" + "cloud.google.com/go/logging/apiv2/loggingpb" +) + +// Options for the slog.Logger +type Options struct { + *slog.HandlerOptions + Name string + Writer io.Writer + Client *logging.Client +} + +// New instantiates a slog.Logger to output using Google Cloud logging entries. +// When running locally, output is JSON strings of Cloud logging entries and +// does not make any API calls to the service. When running in Google Cloud, +// logging entries are submitted to the Cloud logging service. +func New(opts *Options) *slog.Logger { + if opts.HandlerOptions == nil { + opts.HandlerOptions = &slog.HandlerOptions{} + } + + opts.AddSource = true + + if opts.Writer == nil { + opts.Writer = os.Stdout + } + + handler := &gcpHandler{ + name: opts.Name, + mu: &sync.Mutex{}, + out: opts.Writer, + JSONHandler: slog.NewJSONHandler(opts.Writer, opts.HandlerOptions), + } + + if opts.Client != nil { + handler.logger = opts.Client.Logger(path.Base(opts.Name)) + } + + return slog.New(handler) +} + +var _ slog.Handler = &gcpHandler{} + +type gcpHandler struct { + name string + *slog.JSONHandler + mu *sync.Mutex + out io.Writer + logger *logging.Logger +} + +func (g *gcpHandler) Enabled(ctx context.Context, level slog.Level) bool { + return g.JSONHandler.Enabled(ctx, level) +} + +func severity(lvl slog.Level) logging.Severity { + switch lvl { + case slog.LevelDebug: + return logging.Debug + case slog.LevelInfo: + return logging.Info + case slog.LevelWarn: + return logging.Warning + case slog.LevelError: + return logging.Error + } + return logging.Default +} + +func (g *gcpHandler) Handle(_ context.Context, record slog.Record) error { + payload := map[string]any{ + "message": record.Message, + } + record.Attrs(func(attr slog.Attr) bool { + payload[attr.Key] = attr.Value.Any() + return true + }) + fs := runtime.CallersFrames([]uintptr{record.PC}) + f, _ := fs.Next() + entry := logging.Entry{ + LogName: g.name, + Timestamp: record.Time, + Severity: severity(record.Level), + Payload: payload, + SourceLocation: &loggingpb.LogEntrySourceLocation{ + File: f.File, + Line: int64(f.Line), + }, + } + g.mu.Lock() + defer g.mu.Unlock() + if g.logger == nil { + return json.NewEncoder(g.out).Encode(entry) + } + + entry.LogName = "" + g.logger.Log(entry) + return g.logger.Flush() +} + +func (g *gcpHandler) WithAttrs(attrs []slog.Attr) slog.Handler { + h := g.JSONHandler + return h.WithAttrs(attrs) +} + +func (g *gcpHandler) WithGroup(name string) slog.Handler { + h := g.JSONHandler + return h.WithGroup(name) +} diff --git a/.test-infra/mock-apis/src/main/go/internal/logging/logging_test.go b/.test-infra/mock-apis/src/main/go/internal/logging/logging_test.go new file mode 100644 index 000000000000..87bfa160981c --- /dev/null +++ b/.test-infra/mock-apis/src/main/go/internal/logging/logging_test.go @@ -0,0 +1,153 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package logging_test + +import ( + "bytes" + "context" + "encoding/json" + "log/slog" + "runtime" + "testing" + "time" + + gcplogging "cloud.google.com/go/logging" + "cloud.google.com/go/logging/apiv2/loggingpb" + "github.com/apache/beam/test-infra/mock-apis/src/main/go/internal/logging" + "github.com/google/go-cmp/cmp" + "github.com/google/go-cmp/cmp/cmpopts" +) + +var ( + opts = []cmp.Option{ + cmpopts.IgnoreFields(loggingpb.LogEntrySourceLocation{}, "state", "sizeCache", "unknownFields"), + cmpopts.IgnoreFields(gcplogging.Entry{}, "Timestamp"), + } +) + +func Test_logger_Info(t *testing.T) { + type args struct { + message string + fields []slog.Attr + } + tests := []struct { + name string + args args + want gcplogging.Entry + }{ + { + name: "message only", + args: args{ + message: "hello log", + }, + want: gcplogging.Entry{ + LogName: "message only", + Severity: gcplogging.Info, + Payload: map[string]interface{}{ + "message": "hello log", + }, + }, + }, + { + name: "with flat fields", + args: args{ + message: "message with fields", + fields: []slog.Attr{ + { + Key: "string", + Value: slog.StringValue("a string"), + }, + { + Key: "int", + Value: slog.IntValue(1), + }, + { + Key: "bool", + Value: slog.BoolValue(true), + }, + { + Key: "float", + Value: slog.Float64Value(1.23456789), + }, + }, + }, + want: gcplogging.Entry{ + LogName: "with flat fields", + Severity: gcplogging.Info, + Payload: map[string]interface{}{ + "message": "message with fields", + "string": "a string", + "int": float64(1), + "bool": true, + "float": 1.23456789, + }, + }, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + buf := bytes.Buffer{} + l := logging.New(&logging.Options{ + Name: tt.name, + Writer: &buf, + }) + l.LogAttrs(context.Background(), slog.LevelInfo, tt.args.message, tt.args.fields...) + _, file, line, _ := runtime.Caller(0) + tt.want.SourceLocation = &loggingpb.LogEntrySourceLocation{ + File: file, + Line: int64(line) - 1, + } + var got gcplogging.Entry + if err := json.NewDecoder(&buf).Decode(&got); err != nil { + t.Fatal(err) + } + if diff := cmp.Diff(tt.want, got, opts...); diff != "" { + t.Errorf("LogAttrs(Info) yielded unexpected difference in log entry (-want, +got):\n%s", diff) + } + }) + } +} +func Test_logger_Error(t *testing.T) { + buf := bytes.Buffer{} + l := logging.New(&logging.Options{ + Name: "test logger error", + Writer: &buf, + }) + message := "some error" + fields := []slog.Attr{ + { + Key: "observed", + Value: slog.TimeValue(time.Unix(1000000000, 0)), + }, + } + l.LogAttrs(context.Background(), slog.LevelError, message, fields...) + _, file, line, _ := runtime.Caller(0) + var got gcplogging.Entry + if err := json.NewDecoder(&buf).Decode(&got); err != nil { + t.Fatal(err) + } + if diff := cmp.Diff(gcplogging.Entry{ + LogName: "test logger error", + Severity: gcplogging.Error, + Payload: map[string]any{"message": "some error", "observed": "2001-09-09T01:46:40Z"}, + SourceLocation: &loggingpb.LogEntrySourceLocation{ + File: file, + Line: int64(line) - 1, + }, + }, got, opts...); diff != "" { + t.Errorf("LogAttrs(Error) yielded unexpected difference in log entry (-want, +got):\n%s", diff) + } +} diff --git a/.test-infra/mock-apis/src/main/go/internal/metric/doc.go b/.test-infra/mock-apis/src/main/go/internal/metric/doc.go new file mode 100644 index 000000000000..43bfc771c198 --- /dev/null +++ b/.test-infra/mock-apis/src/main/go/internal/metric/doc.go @@ -0,0 +1,17 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// Package metric supports monitoring. +package metric diff --git a/.test-infra/mock-apis/src/main/go/internal/metric/gcp.go b/.test-infra/mock-apis/src/main/go/internal/metric/gcp.go new file mode 100644 index 000000000000..3d23d538955a --- /dev/null +++ b/.test-infra/mock-apis/src/main/go/internal/metric/gcp.go @@ -0,0 +1,77 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package metric + +import ( + "context" + "path" + + monitoring "cloud.google.com/go/monitoring/apiv3" + "cloud.google.com/go/monitoring/apiv3/v2/monitoringpb" + "google.golang.org/genproto/googleapis/api/metric" + "google.golang.org/genproto/googleapis/api/monitoredres" + "google.golang.org/protobuf/types/known/timestamppb" +) + +const ( + metricTypePrefix = "custom.googleapis.com" + monitoredResourceType = "generic_task" +) + +// GcpGauge implements a Writer for a Google Cloud gauge. +// See https://cloud.google.com/monitoring/api/v3/kinds-and-types#metric-kinds +type GcpGauge monitoring.MetricClient + +// Write to a Google Cloud monitoring gauge. +func (writer *GcpGauge) Write(ctx context.Context, name string, unit string, points ...*Point) error { + var mPts []*monitoringpb.Point + for _, p := range points { + t := timestamppb.New(p.Timestamp) + mPts = append(mPts, &monitoringpb.Point{ + Interval: &monitoringpb.TimeInterval{ + StartTime: t, + EndTime: t, + }, + Value: &monitoringpb.TypedValue{ + Value: &monitoringpb.TypedValue_Int64Value{ + Int64Value: p.Value, + }, + }, + }) + } + ts := timeseries(name, unit, metric.MetricDescriptor_GAUGE, mPts) + + client := (*monitoring.MetricClient)(writer) + return client.CreateTimeSeries(ctx, &monitoringpb.CreateTimeSeriesRequest{ + Name: name, + TimeSeries: []*monitoringpb.TimeSeries{ts}, + }) +} + +func timeseries(name string, unit string, kind metric.MetricDescriptor_MetricKind, points []*monitoringpb.Point) *monitoringpb.TimeSeries { + return &monitoringpb.TimeSeries{ + Metric: &metric.Metric{ + Type: path.Join(metricTypePrefix, name), + }, + Resource: &monitoredres.MonitoredResource{ + Type: monitoredResourceType, + }, + MetricKind: kind, + ValueType: metric.MetricDescriptor_INT64, + Unit: unit, + Points: points, + } +} diff --git a/.test-infra/mock-apis/src/main/go/internal/metric/interface.go b/.test-infra/mock-apis/src/main/go/internal/metric/interface.go new file mode 100644 index 000000000000..d0f7e385227e --- /dev/null +++ b/.test-infra/mock-apis/src/main/go/internal/metric/interface.go @@ -0,0 +1,38 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package metric + +import ( + "context" + "time" +) + +// Writer writes to a metric sink. +type Writer interface { + + // Write to a metric sink. + Write(ctx context.Context, name string, unit string, points ...*Point) error +} + +// Point models a metric data point. +type Point struct { + + // Timestamp of the metric data point. + Timestamp time.Time + + // Value of the metric data point. + Value int64 +} diff --git a/.test-infra/mock-apis/src/main/go/internal/proto/echo/v1/echo.pb.go b/.test-infra/mock-apis/src/main/go/internal/proto/echo/v1/echo.pb.go new file mode 100644 index 000000000000..97ced9227d7f --- /dev/null +++ b/.test-infra/mock-apis/src/main/go/internal/proto/echo/v1/echo.pb.go @@ -0,0 +1,256 @@ +// +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// +// Protocol buffers describing a simple mock API that echos a request. + +// Code generated by protoc-gen-go. DO NOT EDIT. +// versions: +// protoc-gen-go v1.28.1 +// protoc (unknown) +// source: proto/echo/v1/echo.proto + +package v1 + +import ( + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + reflect "reflect" + sync "sync" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +// The request to echo a payload. +type EchoRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + Payload []byte `protobuf:"bytes,2,opt,name=payload,proto3" json:"payload,omitempty"` +} + +func (x *EchoRequest) Reset() { + *x = EchoRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_proto_echo_v1_echo_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *EchoRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*EchoRequest) ProtoMessage() {} + +func (x *EchoRequest) ProtoReflect() protoreflect.Message { + mi := &file_proto_echo_v1_echo_proto_msgTypes[0] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use EchoRequest.ProtoReflect.Descriptor instead. +func (*EchoRequest) Descriptor() ([]byte, []int) { + return file_proto_echo_v1_echo_proto_rawDescGZIP(), []int{0} +} + +func (x *EchoRequest) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +func (x *EchoRequest) GetPayload() []byte { + if x != nil { + return x.Payload + } + return nil +} + +// The response echo of a request payload. +type EchoResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + Payload []byte `protobuf:"bytes,2,opt,name=payload,proto3" json:"payload,omitempty"` +} + +func (x *EchoResponse) Reset() { + *x = EchoResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_proto_echo_v1_echo_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *EchoResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*EchoResponse) ProtoMessage() {} + +func (x *EchoResponse) ProtoReflect() protoreflect.Message { + mi := &file_proto_echo_v1_echo_proto_msgTypes[1] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use EchoResponse.ProtoReflect.Descriptor instead. +func (*EchoResponse) Descriptor() ([]byte, []int) { + return file_proto_echo_v1_echo_proto_rawDescGZIP(), []int{1} +} + +func (x *EchoResponse) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +func (x *EchoResponse) GetPayload() []byte { + if x != nil { + return x.Payload + } + return nil +} + +var File_proto_echo_v1_echo_proto protoreflect.FileDescriptor + +var file_proto_echo_v1_echo_proto_rawDesc = []byte{ + 0x0a, 0x18, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x65, 0x63, 0x68, 0x6f, 0x2f, 0x76, 0x31, 0x2f, + 0x65, 0x63, 0x68, 0x6f, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x0d, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x65, 0x63, 0x68, 0x6f, 0x2e, 0x76, 0x31, 0x22, 0x37, 0x0a, 0x0b, 0x45, 0x63, 0x68, + 0x6f, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, 0x18, 0x0a, 0x07, 0x70, 0x61, 0x79, 0x6c, + 0x6f, 0x61, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, + 0x61, 0x64, 0x22, 0x38, 0x0a, 0x0c, 0x45, 0x63, 0x68, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, + 0x69, 0x64, 0x12, 0x18, 0x0a, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x0c, 0x52, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x32, 0x50, 0x0a, 0x0b, + 0x45, 0x63, 0x68, 0x6f, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x41, 0x0a, 0x04, 0x45, + 0x63, 0x68, 0x6f, 0x12, 0x1a, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x65, 0x63, 0x68, 0x6f, + 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x63, 0x68, 0x6f, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x1b, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x65, 0x63, 0x68, 0x6f, 0x2e, 0x76, 0x31, 0x2e, + 0x45, 0x63, 0x68, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x42, 0x3b, + 0x0a, 0x2a, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, + 0x6d, 0x2e, 0x74, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x66, 0x72, 0x61, 0x2e, 0x6d, 0x6f, 0x63, 0x6b, + 0x61, 0x70, 0x69, 0x73, 0x2e, 0x65, 0x63, 0x68, 0x6f, 0x2e, 0x76, 0x31, 0x5a, 0x0d, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2f, 0x65, 0x63, 0x68, 0x6f, 0x2f, 0x76, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x33, +} + +var ( + file_proto_echo_v1_echo_proto_rawDescOnce sync.Once + file_proto_echo_v1_echo_proto_rawDescData = file_proto_echo_v1_echo_proto_rawDesc +) + +func file_proto_echo_v1_echo_proto_rawDescGZIP() []byte { + file_proto_echo_v1_echo_proto_rawDescOnce.Do(func() { + file_proto_echo_v1_echo_proto_rawDescData = protoimpl.X.CompressGZIP(file_proto_echo_v1_echo_proto_rawDescData) + }) + return file_proto_echo_v1_echo_proto_rawDescData +} + +var file_proto_echo_v1_echo_proto_msgTypes = make([]protoimpl.MessageInfo, 2) +var file_proto_echo_v1_echo_proto_goTypes = []interface{}{ + (*EchoRequest)(nil), // 0: proto.echo.v1.EchoRequest + (*EchoResponse)(nil), // 1: proto.echo.v1.EchoResponse +} +var file_proto_echo_v1_echo_proto_depIdxs = []int32{ + 0, // 0: proto.echo.v1.EchoService.Echo:input_type -> proto.echo.v1.EchoRequest + 1, // 1: proto.echo.v1.EchoService.Echo:output_type -> proto.echo.v1.EchoResponse + 1, // [1:2] is the sub-list for method output_type + 0, // [0:1] is the sub-list for method input_type + 0, // [0:0] is the sub-list for extension type_name + 0, // [0:0] is the sub-list for extension extendee + 0, // [0:0] is the sub-list for field type_name +} + +func init() { file_proto_echo_v1_echo_proto_init() } +func file_proto_echo_v1_echo_proto_init() { + if File_proto_echo_v1_echo_proto != nil { + return + } + if !protoimpl.UnsafeEnabled { + file_proto_echo_v1_echo_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*EchoRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_proto_echo_v1_echo_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*EchoResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_proto_echo_v1_echo_proto_rawDesc, + NumEnums: 0, + NumMessages: 2, + NumExtensions: 0, + NumServices: 1, + }, + GoTypes: file_proto_echo_v1_echo_proto_goTypes, + DependencyIndexes: file_proto_echo_v1_echo_proto_depIdxs, + MessageInfos: file_proto_echo_v1_echo_proto_msgTypes, + }.Build() + File_proto_echo_v1_echo_proto = out.File + file_proto_echo_v1_echo_proto_rawDesc = nil + file_proto_echo_v1_echo_proto_goTypes = nil + file_proto_echo_v1_echo_proto_depIdxs = nil +} diff --git a/.test-infra/mock-apis/src/main/go/internal/proto/echo/v1/echo_grpc.pb.go b/.test-infra/mock-apis/src/main/go/internal/proto/echo/v1/echo_grpc.pb.go new file mode 100644 index 000000000000..3ce2bdeeeec1 --- /dev/null +++ b/.test-infra/mock-apis/src/main/go/internal/proto/echo/v1/echo_grpc.pb.go @@ -0,0 +1,107 @@ +// Code generated by protoc-gen-go-grpc. DO NOT EDIT. +// versions: +// - protoc-gen-go-grpc v1.2.0 +// - protoc (unknown) +// source: proto/echo/v1/echo.proto + +package v1 + +import ( + context "context" + grpc "google.golang.org/grpc" + codes "google.golang.org/grpc/codes" + status "google.golang.org/grpc/status" +) + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the grpc package it is being compiled against. +// Requires gRPC-Go v1.32.0 or later. +const _ = grpc.SupportPackageIsVersion7 + +// EchoServiceClient is the client API for EchoService service. +// +// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://pkg.go.dev/google.golang.org/grpc/?tab=doc#ClientConn.NewStream. +type EchoServiceClient interface { + // Echo an EchoRequest payload in an EchoResponse. + Echo(ctx context.Context, in *EchoRequest, opts ...grpc.CallOption) (*EchoResponse, error) +} + +type echoServiceClient struct { + cc grpc.ClientConnInterface +} + +func NewEchoServiceClient(cc grpc.ClientConnInterface) EchoServiceClient { + return &echoServiceClient{cc} +} + +func (c *echoServiceClient) Echo(ctx context.Context, in *EchoRequest, opts ...grpc.CallOption) (*EchoResponse, error) { + out := new(EchoResponse) + err := c.cc.Invoke(ctx, "/proto.echo.v1.EchoService/Echo", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +// EchoServiceServer is the server API for EchoService service. +// All implementations must embed UnimplementedEchoServiceServer +// for forward compatibility +type EchoServiceServer interface { + // Echo an EchoRequest payload in an EchoResponse. + Echo(context.Context, *EchoRequest) (*EchoResponse, error) + mustEmbedUnimplementedEchoServiceServer() +} + +// UnimplementedEchoServiceServer must be embedded to have forward compatible implementations. +type UnimplementedEchoServiceServer struct { +} + +func (UnimplementedEchoServiceServer) Echo(context.Context, *EchoRequest) (*EchoResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method Echo not implemented") +} +func (UnimplementedEchoServiceServer) mustEmbedUnimplementedEchoServiceServer() {} + +// UnsafeEchoServiceServer may be embedded to opt out of forward compatibility for this service. +// Use of this interface is not recommended, as added methods to EchoServiceServer will +// result in compilation errors. +type UnsafeEchoServiceServer interface { + mustEmbedUnimplementedEchoServiceServer() +} + +func RegisterEchoServiceServer(s grpc.ServiceRegistrar, srv EchoServiceServer) { + s.RegisterService(&EchoService_ServiceDesc, srv) +} + +func _EchoService_Echo_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(EchoRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(EchoServiceServer).Echo(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/proto.echo.v1.EchoService/Echo", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(EchoServiceServer).Echo(ctx, req.(*EchoRequest)) + } + return interceptor(ctx, in, info, handler) +} + +// EchoService_ServiceDesc is the grpc.ServiceDesc for EchoService service. +// It's only intended for direct use with grpc.RegisterService, +// and not to be introspected or modified (even as a copy) +var EchoService_ServiceDesc = grpc.ServiceDesc{ + ServiceName: "proto.echo.v1.EchoService", + HandlerType: (*EchoServiceServer)(nil), + Methods: []grpc.MethodDesc{ + { + MethodName: "Echo", + Handler: _EchoService_Echo_Handler, + }, + }, + Streams: []grpc.StreamDesc{}, + Metadata: "proto/echo/v1/echo.proto", +} diff --git a/.test-infra/mock-apis/src/main/go/internal/service/echo/echo.go b/.test-infra/mock-apis/src/main/go/internal/service/echo/echo.go new file mode 100644 index 000000000000..d958d18576e5 --- /dev/null +++ b/.test-infra/mock-apis/src/main/go/internal/service/echo/echo.go @@ -0,0 +1,185 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// Package echo contains the EchoService API implementation. +package echo + +import ( + "context" + "encoding/json" + "fmt" + "log/slog" + "net/http" + "path" + "reflect" + "time" + + "github.com/apache/beam/test-infra/mock-apis/src/main/go/internal/cache" + "github.com/apache/beam/test-infra/mock-apis/src/main/go/internal/logging" + "github.com/apache/beam/test-infra/mock-apis/src/main/go/internal/metric" + echov1 "github.com/apache/beam/test-infra/mock-apis/src/main/go/internal/proto/echo/v1" + "google.golang.org/grpc" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/health/grpc_health_v1" + "google.golang.org/grpc/status" +) + +const ( + metricsNamePrefix = "echo" + echoPath = "/proto.echo.v1.EchoService/Echo" + echoPathAlias = "/v1/echo" + healthPath = "/grpc.health.v1.Health/Check" + healthPathAlias = "/v1/healthz" +) + +type Options struct { + Decrementer cache.Decrementer + MetricsWriter metric.Writer + Logger *slog.Logger + LoggingAttrs []slog.Attr +} + +// Register a grpc.Server with the echov1.EchoService. Returns a http.Handler or error. +func Register(s *grpc.Server, opts *Options) (http.Handler, error) { + if opts.Logger == nil { + opts.Logger = logging.New(&logging.Options{ + Name: reflect.TypeOf((*echo)(nil)).PkgPath(), + }) + } + srv := &echo{ + opts: opts, + } + + echov1.RegisterEchoServiceServer(s, srv) + grpc_health_v1.RegisterHealthServer(s, srv) + + return srv, nil +} + +type echo struct { + echov1.UnimplementedEchoServiceServer + grpc_health_v1.UnimplementedHealthServer + opts *Options +} + +// ServeHTTP implements http.Handler, allowing echo to support HTTP clients in addition to gRPC. +func (srv *echo) ServeHTTP(w http.ResponseWriter, r *http.Request) { + switch r.URL.Path { + case echoPath, echoPathAlias: + srv.httpHandler(w, r) + case healthPath, healthPathAlias: + srv.checkHandler(w, r) + default: + http.Error(w, fmt.Sprintf("%s not found", r.URL.Path), http.StatusNotFound) + } +} + +// Check checks whether echo service's underlying decrementer is alive. +func (srv *echo) Check(ctx context.Context, _ *grpc_health_v1.HealthCheckRequest) (*grpc_health_v1.HealthCheckResponse, error) { + if err := srv.opts.Decrementer.Alive(ctx); err != nil { + return nil, err + } + return &grpc_health_v1.HealthCheckResponse{ + Status: grpc_health_v1.HealthCheckResponse_SERVING, + }, nil +} + +func (srv *echo) checkHandler(w http.ResponseWriter, r *http.Request) { + resp, err := srv.Check(r.Context(), nil) + if err != nil { + + http.Error(w, err.Error(), http.StatusInternalServerError) + return + } + if err := json.NewEncoder(w).Encode(resp); err != nil { + srv.opts.Logger.LogAttrs(context.Background(), slog.LevelError, err.Error(), srv.opts.LoggingAttrs...) + http.Error(w, err.Error(), http.StatusInternalServerError) + } +} + +// Watch the health of the echov1.EchoServiceServer. +func (srv *echo) Watch(request *grpc_health_v1.HealthCheckRequest, server grpc_health_v1.Health_WatchServer) error { + resp, err := srv.Check(server.Context(), request) + if err != nil { + srv.opts.Logger.LogAttrs(context.Background(), slog.LevelError, err.Error(), srv.opts.LoggingAttrs...) + return err + } + return server.Send(resp) +} + +// Echo a EchoRequest with a EchoResponse. Decrements an underlying quota identified by the id of the request. +// Returns a cache.IsNotExist if request's id does not map to a key in the cache. +// See cache.Refresher for how the cache refreshes the quota identified by the request id. +func (srv *echo) Echo(ctx context.Context, request *echov1.EchoRequest) (*echov1.EchoResponse, error) { + v, err := srv.opts.Decrementer.Decrement(ctx, request.Id) + if cache.IsNotExist(err) { + return nil, status.Errorf(codes.NotFound, "error: source not found: %s, err %v", request.Id, err) + } + if err != nil { + srv.opts.Logger.LogAttrs(context.Background(), slog.LevelError, err.Error(), srv.opts.LoggingAttrs...) + return nil, status.Errorf(codes.Internal, "error: encountered from cache for resource: %srv, err %v", request.Id, err) + } + + if err := srv.writeMetric(ctx, request.Id, v); err != nil { + return nil, err + } + + if v < 0 { + return nil, status.Errorf(codes.ResourceExhausted, "error: resource exhausted for: %srv", request.Id) + } + + return &echov1.EchoResponse{ + Id: request.Id, + Payload: request.Payload, + }, nil +} + +func (srv *echo) writeMetric(ctx context.Context, id string, value int64) error { + if srv.opts.MetricsWriter == nil { + return nil + } + if err := srv.opts.MetricsWriter.Write(ctx, path.Join(metricsNamePrefix, id), "unit", &metric.Point{ + Timestamp: time.Now(), + Value: value + 1, + }); err != nil { + srv.opts.Logger.LogAttrs(context.Background(), slog.LevelError, err.Error(), srv.opts.LoggingAttrs...) + } + return nil +} + +func (srv *echo) httpHandler(w http.ResponseWriter, r *http.Request) { + var body *echov1.EchoRequest + if err := json.NewDecoder(r.Body).Decode(&body); err != nil { + err = fmt.Errorf("error decoding request body, payload field of %T needs to be base64 encoded, error: %w", body, err) + srv.opts.Logger.LogAttrs(context.Background(), slog.LevelError, err.Error(), srv.opts.LoggingAttrs...) + http.Error(w, err.Error(), http.StatusBadRequest) + return + } + + resp, err := srv.Echo(r.Context(), body) + if status.Code(err) == http.StatusNotFound { + http.Error(w, err.Error(), http.StatusNotFound) + return + } + + if err != nil { + http.Error(w, err.Error(), http.StatusInternalServerError) + return + } + + if err := json.NewEncoder(w).Encode(resp); err != nil { + http.Error(w, err.Error(), http.StatusInternalServerError) + } +} diff --git a/.test-infra/mock-apis/src/main/java/org/apache/beam/testinfra/mockapis/echo/v1/Echo.java b/.test-infra/mock-apis/src/main/java/org/apache/beam/testinfra/mockapis/echo/v1/Echo.java new file mode 100644 index 000000000000..4652ff716b87 --- /dev/null +++ b/.test-infra/mock-apis/src/main/java/org/apache/beam/testinfra/mockapis/echo/v1/Echo.java @@ -0,0 +1,1447 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.testinfra.mockapis.echo.v1; + +@SuppressWarnings({ + "argument", + "assignment", + "initialization.fields.uninitialized", + "initialization.static.field.uninitialized", + "override.param", + "ClassTypeParameterName", + "ForbidNonVendoredGuava", + "JavadocStyle", + "LocalVariableName", + "MemberName", + "NeedBraces", + "MissingOverride", + "RedundantModifier", + "ReferenceEquality", + "UnusedVariable", +}) +public final class Echo { + private Echo() {} + + public static void registerAllExtensions(com.google.protobuf.ExtensionRegistryLite registry) {} + + public static void registerAllExtensions(com.google.protobuf.ExtensionRegistry registry) { + registerAllExtensions((com.google.protobuf.ExtensionRegistryLite) registry); + } + + public interface EchoRequestOrBuilder + extends + // @@protoc_insertion_point(interface_extends:proto.echo.v1.EchoRequest) + com.google.protobuf.MessageOrBuilder { + + /** + * <code>string id = 1 [json_name = "id"];</code> + * + * @return The id. + */ + java.lang.String getId(); + /** + * <code>string id = 1 [json_name = "id"];</code> + * + * @return The bytes for id. + */ + com.google.protobuf.ByteString getIdBytes(); + + /** + * <code>bytes payload = 2 [json_name = "payload"];</code> + * + * @return The payload. + */ + com.google.protobuf.ByteString getPayload(); + } + /** + * + * + * <pre> + * The request to echo a payload. + * </pre> + * + * Protobuf type {@code proto.echo.v1.EchoRequest} + */ + public static final class EchoRequest extends com.google.protobuf.GeneratedMessageV3 + implements + // @@protoc_insertion_point(message_implements:proto.echo.v1.EchoRequest) + EchoRequestOrBuilder { + private static final long serialVersionUID = 0L; + // Use EchoRequest.newBuilder() to construct. + private EchoRequest(com.google.protobuf.GeneratedMessageV3.Builder<?> builder) { + super(builder); + } + + private EchoRequest() { + id_ = ""; + payload_ = com.google.protobuf.ByteString.EMPTY; + } + + @java.lang.Override + @SuppressWarnings({"unused"}) + protected java.lang.Object newInstance(UnusedPrivateParameter unused) { + return new EchoRequest(); + } + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return org.apache.beam.testinfra.mockapis.echo.v1.Echo + .internal_static_proto_echo_v1_EchoRequest_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.beam.testinfra.mockapis.echo.v1.Echo + .internal_static_proto_echo_v1_EchoRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoRequest.class, + org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoRequest.Builder.class); + } + + public static final int ID_FIELD_NUMBER = 1; + + @SuppressWarnings("serial") + private volatile java.lang.Object id_ = ""; + /** + * <code>string id = 1 [json_name = "id"];</code> + * + * @return The id. + */ + @java.lang.Override + public java.lang.String getId() { + java.lang.Object ref = id_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + id_ = s; + return s; + } + } + /** + * <code>string id = 1 [json_name = "id"];</code> + * + * @return The bytes for id. + */ + @java.lang.Override + public com.google.protobuf.ByteString getIdBytes() { + java.lang.Object ref = id_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + id_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int PAYLOAD_FIELD_NUMBER = 2; + private com.google.protobuf.ByteString payload_ = com.google.protobuf.ByteString.EMPTY; + /** + * <code>bytes payload = 2 [json_name = "payload"];</code> + * + * @return The payload. + */ + @java.lang.Override + public com.google.protobuf.ByteString getPayload() { + return payload_; + } + + private byte memoizedIsInitialized = -1; + + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { + if (!com.google.protobuf.GeneratedMessageV3.isStringEmpty(id_)) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 1, id_); + } + if (!payload_.isEmpty()) { + output.writeBytes(2, payload_); + } + getUnknownFields().writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (!com.google.protobuf.GeneratedMessageV3.isStringEmpty(id_)) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, id_); + } + if (!payload_.isEmpty()) { + size += com.google.protobuf.CodedOutputStream.computeBytesSize(2, payload_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoRequest)) { + return super.equals(obj); + } + org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoRequest other = + (org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoRequest) obj; + + if (!getId().equals(other.getId())) return false; + if (!getPayload().equals(other.getPayload())) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + ID_FIELD_NUMBER; + hash = (53 * hash) + getId().hashCode(); + hash = (37 * hash) + PAYLOAD_FIELD_NUMBER; + hash = (53 * hash) + getPayload().hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoRequest parseFrom( + java.nio.ByteBuffer data) throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoRequest parseFrom( + java.nio.ByteBuffer data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoRequest parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoRequest parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoRequest parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoRequest parseFrom( + byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoRequest parseFrom( + java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoRequest parseFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + public static org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoRequest parseDelimitedFrom( + java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input); + } + + public static org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoRequest parseDelimitedFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input, extensionRegistry); + } + + public static org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoRequest parseFrom( + com.google.protobuf.CodedInputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoRequest parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder( + org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoRequest prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * + * + * <pre> + * The request to echo a payload. + * </pre> + * + * Protobuf type {@code proto.echo.v1.EchoRequest} + */ + public static final class Builder + extends com.google.protobuf.GeneratedMessageV3.Builder<Builder> + implements + // @@protoc_insertion_point(builder_implements:proto.echo.v1.EchoRequest) + org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoRequestOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return org.apache.beam.testinfra.mockapis.echo.v1.Echo + .internal_static_proto_echo_v1_EchoRequest_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.beam.testinfra.mockapis.echo.v1.Echo + .internal_static_proto_echo_v1_EchoRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoRequest.class, + org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoRequest.Builder.class); + } + + // Construct using org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoRequest.newBuilder() + private Builder() {} + + private Builder(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + } + + @java.lang.Override + public Builder clear() { + super.clear(); + bitField0_ = 0; + id_ = ""; + payload_ = com.google.protobuf.ByteString.EMPTY; + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { + return org.apache.beam.testinfra.mockapis.echo.v1.Echo + .internal_static_proto_echo_v1_EchoRequest_descriptor; + } + + @java.lang.Override + public org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoRequest + getDefaultInstanceForType() { + return org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoRequest.getDefaultInstance(); + } + + @java.lang.Override + public org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoRequest build() { + org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoRequest result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoRequest buildPartial() { + org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoRequest result = + new org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoRequest(this); + if (bitField0_ != 0) { + buildPartial0(result); + } + onBuilt(); + return result; + } + + private void buildPartial0( + org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoRequest result) { + int from_bitField0_ = bitField0_; + if (((from_bitField0_ & 0x00000001) != 0)) { + result.id_ = id_; + } + if (((from_bitField0_ & 0x00000002) != 0)) { + result.payload_ = payload_; + } + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.setField(field, value); + } + + @java.lang.Override + public Builder clearField(com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + + @java.lang.Override + public Builder clearOneof(com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + int index, + java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.addRepeatedField(field, value); + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoRequest) { + return mergeFrom((org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoRequest) other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoRequest other) { + if (other + == org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoRequest.getDefaultInstance()) + return this; + if (!other.getId().isEmpty()) { + id_ = other.id_; + bitField0_ |= 0x00000001; + onChanged(); + } + if (other.getPayload() != com.google.protobuf.ByteString.EMPTY) { + setPayload(other.getPayload()); + } + this.mergeUnknownFields(other.getUnknownFields()); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: + { + id_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000001; + break; + } // case 10 + case 18: + { + payload_ = input.readBytes(); + bitField0_ |= 0x00000002; + break; + } // case 18 + default: + { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.unwrapIOException(); + } finally { + onChanged(); + } // finally + return this; + } + + private int bitField0_; + + private java.lang.Object id_ = ""; + /** + * <code>string id = 1 [json_name = "id"];</code> + * + * @return The id. + */ + public java.lang.String getId() { + java.lang.Object ref = id_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + id_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * <code>string id = 1 [json_name = "id"];</code> + * + * @return The bytes for id. + */ + public com.google.protobuf.ByteString getIdBytes() { + java.lang.Object ref = id_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + id_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * <code>string id = 1 [json_name = "id"];</code> + * + * @param value The id to set. + * @return This builder for chaining. + */ + public Builder setId(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + id_ = value; + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + /** + * <code>string id = 1 [json_name = "id"];</code> + * + * @return This builder for chaining. + */ + public Builder clearId() { + id_ = getDefaultInstance().getId(); + bitField0_ = (bitField0_ & ~0x00000001); + onChanged(); + return this; + } + /** + * <code>string id = 1 [json_name = "id"];</code> + * + * @param value The bytes for id to set. + * @return This builder for chaining. + */ + public Builder setIdBytes(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + id_ = value; + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + + private com.google.protobuf.ByteString payload_ = com.google.protobuf.ByteString.EMPTY; + /** + * <code>bytes payload = 2 [json_name = "payload"];</code> + * + * @return The payload. + */ + @java.lang.Override + public com.google.protobuf.ByteString getPayload() { + return payload_; + } + /** + * <code>bytes payload = 2 [json_name = "payload"];</code> + * + * @param value The payload to set. + * @return This builder for chaining. + */ + public Builder setPayload(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + payload_ = value; + bitField0_ |= 0x00000002; + onChanged(); + return this; + } + /** + * <code>bytes payload = 2 [json_name = "payload"];</code> + * + * @return This builder for chaining. + */ + public Builder clearPayload() { + bitField0_ = (bitField0_ & ~0x00000002); + payload_ = getDefaultInstance().getPayload(); + onChanged(); + return this; + } + + @java.lang.Override + public final Builder setUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:proto.echo.v1.EchoRequest) + } + + // @@protoc_insertion_point(class_scope:proto.echo.v1.EchoRequest) + private static final org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoRequest + DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = new org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoRequest(); + } + + public static org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoRequest getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser<EchoRequest> PARSER = + new com.google.protobuf.AbstractParser<EchoRequest>() { + @java.lang.Override + public EchoRequest parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException() + .setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); + } + }; + + public static com.google.protobuf.Parser<EchoRequest> parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser<EchoRequest> getParserForType() { + return PARSER; + } + + @java.lang.Override + public org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoRequest getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + } + + public interface EchoResponseOrBuilder + extends + // @@protoc_insertion_point(interface_extends:proto.echo.v1.EchoResponse) + com.google.protobuf.MessageOrBuilder { + + /** + * <code>string id = 1 [json_name = "id"];</code> + * + * @return The id. + */ + java.lang.String getId(); + /** + * <code>string id = 1 [json_name = "id"];</code> + * + * @return The bytes for id. + */ + com.google.protobuf.ByteString getIdBytes(); + + /** + * <code>bytes payload = 2 [json_name = "payload"];</code> + * + * @return The payload. + */ + com.google.protobuf.ByteString getPayload(); + } + /** + * + * + * <pre> + * The response echo of a request payload. + * </pre> + * + * Protobuf type {@code proto.echo.v1.EchoResponse} + */ + public static final class EchoResponse extends com.google.protobuf.GeneratedMessageV3 + implements + // @@protoc_insertion_point(message_implements:proto.echo.v1.EchoResponse) + EchoResponseOrBuilder { + private static final long serialVersionUID = 0L; + // Use EchoResponse.newBuilder() to construct. + private EchoResponse(com.google.protobuf.GeneratedMessageV3.Builder<?> builder) { + super(builder); + } + + private EchoResponse() { + id_ = ""; + payload_ = com.google.protobuf.ByteString.EMPTY; + } + + @java.lang.Override + @SuppressWarnings({"unused"}) + protected java.lang.Object newInstance(UnusedPrivateParameter unused) { + return new EchoResponse(); + } + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return org.apache.beam.testinfra.mockapis.echo.v1.Echo + .internal_static_proto_echo_v1_EchoResponse_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.beam.testinfra.mockapis.echo.v1.Echo + .internal_static_proto_echo_v1_EchoResponse_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoResponse.class, + org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoResponse.Builder.class); + } + + public static final int ID_FIELD_NUMBER = 1; + + @SuppressWarnings("serial") + private volatile java.lang.Object id_ = ""; + /** + * <code>string id = 1 [json_name = "id"];</code> + * + * @return The id. + */ + @java.lang.Override + public java.lang.String getId() { + java.lang.Object ref = id_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + id_ = s; + return s; + } + } + /** + * <code>string id = 1 [json_name = "id"];</code> + * + * @return The bytes for id. + */ + @java.lang.Override + public com.google.protobuf.ByteString getIdBytes() { + java.lang.Object ref = id_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + id_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int PAYLOAD_FIELD_NUMBER = 2; + private com.google.protobuf.ByteString payload_ = com.google.protobuf.ByteString.EMPTY; + /** + * <code>bytes payload = 2 [json_name = "payload"];</code> + * + * @return The payload. + */ + @java.lang.Override + public com.google.protobuf.ByteString getPayload() { + return payload_; + } + + private byte memoizedIsInitialized = -1; + + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { + if (!com.google.protobuf.GeneratedMessageV3.isStringEmpty(id_)) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 1, id_); + } + if (!payload_.isEmpty()) { + output.writeBytes(2, payload_); + } + getUnknownFields().writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (!com.google.protobuf.GeneratedMessageV3.isStringEmpty(id_)) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, id_); + } + if (!payload_.isEmpty()) { + size += com.google.protobuf.CodedOutputStream.computeBytesSize(2, payload_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoResponse)) { + return super.equals(obj); + } + org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoResponse other = + (org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoResponse) obj; + + if (!getId().equals(other.getId())) return false; + if (!getPayload().equals(other.getPayload())) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + ID_FIELD_NUMBER; + hash = (53 * hash) + getId().hashCode(); + hash = (37 * hash) + PAYLOAD_FIELD_NUMBER; + hash = (53 * hash) + getPayload().hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoResponse parseFrom( + java.nio.ByteBuffer data) throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoResponse parseFrom( + java.nio.ByteBuffer data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoResponse parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoResponse parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoResponse parseFrom( + byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoResponse parseFrom( + byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoResponse parseFrom( + java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoResponse parseFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + public static org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoResponse parseDelimitedFrom( + java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input); + } + + public static org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoResponse parseDelimitedFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input, extensionRegistry); + } + + public static org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoResponse parseFrom( + com.google.protobuf.CodedInputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoResponse parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder( + org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoResponse prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * + * + * <pre> + * The response echo of a request payload. + * </pre> + * + * Protobuf type {@code proto.echo.v1.EchoResponse} + */ + public static final class Builder + extends com.google.protobuf.GeneratedMessageV3.Builder<Builder> + implements + // @@protoc_insertion_point(builder_implements:proto.echo.v1.EchoResponse) + org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoResponseOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return org.apache.beam.testinfra.mockapis.echo.v1.Echo + .internal_static_proto_echo_v1_EchoResponse_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.beam.testinfra.mockapis.echo.v1.Echo + .internal_static_proto_echo_v1_EchoResponse_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoResponse.class, + org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoResponse.Builder.class); + } + + // Construct using org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoResponse.newBuilder() + private Builder() {} + + private Builder(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + } + + @java.lang.Override + public Builder clear() { + super.clear(); + bitField0_ = 0; + id_ = ""; + payload_ = com.google.protobuf.ByteString.EMPTY; + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { + return org.apache.beam.testinfra.mockapis.echo.v1.Echo + .internal_static_proto_echo_v1_EchoResponse_descriptor; + } + + @java.lang.Override + public org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoResponse + getDefaultInstanceForType() { + return org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoResponse.getDefaultInstance(); + } + + @java.lang.Override + public org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoResponse build() { + org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoResponse buildPartial() { + org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoResponse result = + new org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoResponse(this); + if (bitField0_ != 0) { + buildPartial0(result); + } + onBuilt(); + return result; + } + + private void buildPartial0( + org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoResponse result) { + int from_bitField0_ = bitField0_; + if (((from_bitField0_ & 0x00000001) != 0)) { + result.id_ = id_; + } + if (((from_bitField0_ & 0x00000002) != 0)) { + result.payload_ = payload_; + } + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.setField(field, value); + } + + @java.lang.Override + public Builder clearField(com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + + @java.lang.Override + public Builder clearOneof(com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + int index, + java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.addRepeatedField(field, value); + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoResponse) { + return mergeFrom((org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoResponse) other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoResponse other) { + if (other + == org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoResponse.getDefaultInstance()) + return this; + if (!other.getId().isEmpty()) { + id_ = other.id_; + bitField0_ |= 0x00000001; + onChanged(); + } + if (other.getPayload() != com.google.protobuf.ByteString.EMPTY) { + setPayload(other.getPayload()); + } + this.mergeUnknownFields(other.getUnknownFields()); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: + { + id_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000001; + break; + } // case 10 + case 18: + { + payload_ = input.readBytes(); + bitField0_ |= 0x00000002; + break; + } // case 18 + default: + { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.unwrapIOException(); + } finally { + onChanged(); + } // finally + return this; + } + + private int bitField0_; + + private java.lang.Object id_ = ""; + /** + * <code>string id = 1 [json_name = "id"];</code> + * + * @return The id. + */ + public java.lang.String getId() { + java.lang.Object ref = id_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + id_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * <code>string id = 1 [json_name = "id"];</code> + * + * @return The bytes for id. + */ + public com.google.protobuf.ByteString getIdBytes() { + java.lang.Object ref = id_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + id_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * <code>string id = 1 [json_name = "id"];</code> + * + * @param value The id to set. + * @return This builder for chaining. + */ + public Builder setId(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + id_ = value; + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + /** + * <code>string id = 1 [json_name = "id"];</code> + * + * @return This builder for chaining. + */ + public Builder clearId() { + id_ = getDefaultInstance().getId(); + bitField0_ = (bitField0_ & ~0x00000001); + onChanged(); + return this; + } + /** + * <code>string id = 1 [json_name = "id"];</code> + * + * @param value The bytes for id to set. + * @return This builder for chaining. + */ + public Builder setIdBytes(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + id_ = value; + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + + private com.google.protobuf.ByteString payload_ = com.google.protobuf.ByteString.EMPTY; + /** + * <code>bytes payload = 2 [json_name = "payload"];</code> + * + * @return The payload. + */ + @java.lang.Override + public com.google.protobuf.ByteString getPayload() { + return payload_; + } + /** + * <code>bytes payload = 2 [json_name = "payload"];</code> + * + * @param value The payload to set. + * @return This builder for chaining. + */ + public Builder setPayload(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + payload_ = value; + bitField0_ |= 0x00000002; + onChanged(); + return this; + } + /** + * <code>bytes payload = 2 [json_name = "payload"];</code> + * + * @return This builder for chaining. + */ + public Builder clearPayload() { + bitField0_ = (bitField0_ & ~0x00000002); + payload_ = getDefaultInstance().getPayload(); + onChanged(); + return this; + } + + @java.lang.Override + public final Builder setUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:proto.echo.v1.EchoResponse) + } + + // @@protoc_insertion_point(class_scope:proto.echo.v1.EchoResponse) + private static final org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoResponse + DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = new org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoResponse(); + } + + public static org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoResponse + getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser<EchoResponse> PARSER = + new com.google.protobuf.AbstractParser<EchoResponse>() { + @java.lang.Override + public EchoResponse parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException() + .setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); + } + }; + + public static com.google.protobuf.Parser<EchoResponse> parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser<EchoResponse> getParserForType() { + return PARSER; + } + + @java.lang.Override + public org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoResponse + getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + } + + private static final com.google.protobuf.Descriptors.Descriptor + internal_static_proto_echo_v1_EchoRequest_descriptor; + private static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_proto_echo_v1_EchoRequest_fieldAccessorTable; + private static final com.google.protobuf.Descriptors.Descriptor + internal_static_proto_echo_v1_EchoResponse_descriptor; + private static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_proto_echo_v1_EchoResponse_fieldAccessorTable; + + public static com.google.protobuf.Descriptors.FileDescriptor getDescriptor() { + return descriptor; + } + + private static com.google.protobuf.Descriptors.FileDescriptor descriptor; + + static { + java.lang.String[] descriptorData = { + "\n\030proto/echo/v1/echo.proto\022\rproto.echo.v" + + "1\"7\n\013EchoRequest\022\016\n\002id\030\001 \001(\tR\002id\022\030\n\007payl" + + "oad\030\002 \001(\014R\007payload\"8\n\014EchoResponse\022\016\n\002id" + + "\030\001 \001(\tR\002id\022\030\n\007payload\030\002 \001(\014R\007payload2P\n\013" + + "EchoService\022A\n\004Echo\022\032.proto.echo.v1.Echo" + + "Request\032\033.proto.echo.v1.EchoResponse\"\000B;" + + "\n*org.apache.beam.testinfra.mockapis.ech" + + "o.v1Z\rproto/echo/v1b\006proto3" + }; + descriptor = + com.google.protobuf.Descriptors.FileDescriptor.internalBuildGeneratedFileFrom( + descriptorData, new com.google.protobuf.Descriptors.FileDescriptor[] {}); + internal_static_proto_echo_v1_EchoRequest_descriptor = getDescriptor().getMessageTypes().get(0); + internal_static_proto_echo_v1_EchoRequest_fieldAccessorTable = + new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_proto_echo_v1_EchoRequest_descriptor, + new java.lang.String[] { + "Id", "Payload", + }); + internal_static_proto_echo_v1_EchoResponse_descriptor = + getDescriptor().getMessageTypes().get(1); + internal_static_proto_echo_v1_EchoResponse_fieldAccessorTable = + new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_proto_echo_v1_EchoResponse_descriptor, + new java.lang.String[] { + "Id", "Payload", + }); + } + + // @@protoc_insertion_point(outer_class_scope) +} diff --git a/.test-infra/mock-apis/src/main/java/org/apache/beam/testinfra/mockapis/echo/v1/EchoServiceGrpc.java b/.test-infra/mock-apis/src/main/java/org/apache/beam/testinfra/mockapis/echo/v1/EchoServiceGrpc.java new file mode 100644 index 000000000000..14437899b69c --- /dev/null +++ b/.test-infra/mock-apis/src/main/java/org/apache/beam/testinfra/mockapis/echo/v1/EchoServiceGrpc.java @@ -0,0 +1,393 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.testinfra.mockapis.echo.v1; + +import static io.grpc.MethodDescriptor.generateFullMethodName; + +/** + * + * + * <pre> + * EchoService simulates a mock API that echos a request. + * </pre> + */ +@SuppressWarnings({ + "argument", + "assignment", + "initialization.fields.uninitialized", + "initialization.static.field.uninitialized", + "override.param", + "ClassTypeParameterName", + "ForbidNonVendoredGuava", + "JavadocStyle", + "LocalVariableName", + "MemberName", + "NeedBraces", + "MissingOverride", + "RedundantModifier", + "ReferenceEquality", + "UnusedVariable", +}) +@javax.annotation.Generated( + value = "by gRPC proto compiler (version 1.58.0)", + comments = "Source: proto/echo/v1/echo.proto") +@io.grpc.stub.annotations.GrpcGenerated +public final class EchoServiceGrpc { + + private EchoServiceGrpc() {} + + public static final java.lang.String SERVICE_NAME = "proto.echo.v1.EchoService"; + + // Static method descriptors that strictly reflect the proto. + private static volatile io.grpc.MethodDescriptor< + org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoRequest, + org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoResponse> + getEchoMethod; + + @io.grpc.stub.annotations.RpcMethod( + fullMethodName = SERVICE_NAME + '/' + "Echo", + requestType = org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoRequest.class, + responseType = org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoResponse.class, + methodType = io.grpc.MethodDescriptor.MethodType.UNARY) + public static io.grpc.MethodDescriptor< + org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoRequest, + org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoResponse> + getEchoMethod() { + io.grpc.MethodDescriptor< + org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoRequest, + org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoResponse> + getEchoMethod; + if ((getEchoMethod = EchoServiceGrpc.getEchoMethod) == null) { + synchronized (EchoServiceGrpc.class) { + if ((getEchoMethod = EchoServiceGrpc.getEchoMethod) == null) { + EchoServiceGrpc.getEchoMethod = + getEchoMethod = + io.grpc.MethodDescriptor + .<org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoRequest, + org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoResponse> + newBuilder() + .setType(io.grpc.MethodDescriptor.MethodType.UNARY) + .setFullMethodName(generateFullMethodName(SERVICE_NAME, "Echo")) + .setSampledToLocalTracing(true) + .setRequestMarshaller( + io.grpc.protobuf.ProtoUtils.marshaller( + org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoRequest + .getDefaultInstance())) + .setResponseMarshaller( + io.grpc.protobuf.ProtoUtils.marshaller( + org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoResponse + .getDefaultInstance())) + .setSchemaDescriptor(new EchoServiceMethodDescriptorSupplier("Echo")) + .build(); + } + } + } + return getEchoMethod; + } + + /** Creates a new async stub that supports all call types for the service */ + public static EchoServiceStub newStub(io.grpc.Channel channel) { + io.grpc.stub.AbstractStub.StubFactory<EchoServiceStub> factory = + new io.grpc.stub.AbstractStub.StubFactory<EchoServiceStub>() { + @java.lang.Override + public EchoServiceStub newStub(io.grpc.Channel channel, io.grpc.CallOptions callOptions) { + return new EchoServiceStub(channel, callOptions); + } + }; + return EchoServiceStub.newStub(factory, channel); + } + + /** + * Creates a new blocking-style stub that supports unary and streaming output calls on the service + */ + public static EchoServiceBlockingStub newBlockingStub(io.grpc.Channel channel) { + io.grpc.stub.AbstractStub.StubFactory<EchoServiceBlockingStub> factory = + new io.grpc.stub.AbstractStub.StubFactory<EchoServiceBlockingStub>() { + @java.lang.Override + public EchoServiceBlockingStub newStub( + io.grpc.Channel channel, io.grpc.CallOptions callOptions) { + return new EchoServiceBlockingStub(channel, callOptions); + } + }; + return EchoServiceBlockingStub.newStub(factory, channel); + } + + /** Creates a new ListenableFuture-style stub that supports unary calls on the service */ + public static EchoServiceFutureStub newFutureStub(io.grpc.Channel channel) { + io.grpc.stub.AbstractStub.StubFactory<EchoServiceFutureStub> factory = + new io.grpc.stub.AbstractStub.StubFactory<EchoServiceFutureStub>() { + @java.lang.Override + public EchoServiceFutureStub newStub( + io.grpc.Channel channel, io.grpc.CallOptions callOptions) { + return new EchoServiceFutureStub(channel, callOptions); + } + }; + return EchoServiceFutureStub.newStub(factory, channel); + } + + /** + * + * + * <pre> + * EchoService simulates a mock API that echos a request. + * </pre> + */ + public interface AsyncService { + + /** + * + * + * <pre> + * Echo an EchoRequest payload in an EchoResponse. + * </pre> + */ + default void echo( + org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoRequest request, + io.grpc.stub.StreamObserver<org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoResponse> + responseObserver) { + io.grpc.stub.ServerCalls.asyncUnimplementedUnaryCall(getEchoMethod(), responseObserver); + } + } + + /** + * Base class for the server implementation of the service EchoService. + * + * <pre> + * EchoService simulates a mock API that echos a request. + * </pre> + */ + public abstract static class EchoServiceImplBase + implements io.grpc.BindableService, AsyncService { + + @java.lang.Override + public final io.grpc.ServerServiceDefinition bindService() { + return EchoServiceGrpc.bindService(this); + } + } + + /** + * A stub to allow clients to do asynchronous rpc calls to service EchoService. + * + * <pre> + * EchoService simulates a mock API that echos a request. + * </pre> + */ + public static final class EchoServiceStub + extends io.grpc.stub.AbstractAsyncStub<EchoServiceStub> { + private EchoServiceStub(io.grpc.Channel channel, io.grpc.CallOptions callOptions) { + super(channel, callOptions); + } + + @java.lang.Override + protected EchoServiceStub build(io.grpc.Channel channel, io.grpc.CallOptions callOptions) { + return new EchoServiceStub(channel, callOptions); + } + + /** + * + * + * <pre> + * Echo an EchoRequest payload in an EchoResponse. + * </pre> + */ + public void echo( + org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoRequest request, + io.grpc.stub.StreamObserver<org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoResponse> + responseObserver) { + io.grpc.stub.ClientCalls.asyncUnaryCall( + getChannel().newCall(getEchoMethod(), getCallOptions()), request, responseObserver); + } + } + + /** + * A stub to allow clients to do synchronous rpc calls to service EchoService. + * + * <pre> + * EchoService simulates a mock API that echos a request. + * </pre> + */ + public static final class EchoServiceBlockingStub + extends io.grpc.stub.AbstractBlockingStub<EchoServiceBlockingStub> { + private EchoServiceBlockingStub(io.grpc.Channel channel, io.grpc.CallOptions callOptions) { + super(channel, callOptions); + } + + @java.lang.Override + protected EchoServiceBlockingStub build( + io.grpc.Channel channel, io.grpc.CallOptions callOptions) { + return new EchoServiceBlockingStub(channel, callOptions); + } + + /** + * + * + * <pre> + * Echo an EchoRequest payload in an EchoResponse. + * </pre> + */ + public org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoResponse echo( + org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoRequest request) { + return io.grpc.stub.ClientCalls.blockingUnaryCall( + getChannel(), getEchoMethod(), getCallOptions(), request); + } + } + + /** + * A stub to allow clients to do ListenableFuture-style rpc calls to service EchoService. + * + * <pre> + * EchoService simulates a mock API that echos a request. + * </pre> + */ + public static final class EchoServiceFutureStub + extends io.grpc.stub.AbstractFutureStub<EchoServiceFutureStub> { + private EchoServiceFutureStub(io.grpc.Channel channel, io.grpc.CallOptions callOptions) { + super(channel, callOptions); + } + + @java.lang.Override + protected EchoServiceFutureStub build( + io.grpc.Channel channel, io.grpc.CallOptions callOptions) { + return new EchoServiceFutureStub(channel, callOptions); + } + + /** + * + * + * <pre> + * Echo an EchoRequest payload in an EchoResponse. + * </pre> + */ + public com.google.common.util.concurrent.ListenableFuture< + org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoResponse> + echo(org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoRequest request) { + return io.grpc.stub.ClientCalls.futureUnaryCall( + getChannel().newCall(getEchoMethod(), getCallOptions()), request); + } + } + + private static final int METHODID_ECHO = 0; + + private static final class MethodHandlers<Req, Resp> + implements io.grpc.stub.ServerCalls.UnaryMethod<Req, Resp>, + io.grpc.stub.ServerCalls.ServerStreamingMethod<Req, Resp>, + io.grpc.stub.ServerCalls.ClientStreamingMethod<Req, Resp>, + io.grpc.stub.ServerCalls.BidiStreamingMethod<Req, Resp> { + private final AsyncService serviceImpl; + private final int methodId; + + MethodHandlers(AsyncService serviceImpl, int methodId) { + this.serviceImpl = serviceImpl; + this.methodId = methodId; + } + + @java.lang.Override + @java.lang.SuppressWarnings("unchecked") + public void invoke(Req request, io.grpc.stub.StreamObserver<Resp> responseObserver) { + switch (methodId) { + case METHODID_ECHO: + serviceImpl.echo( + (org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoRequest) request, + (io.grpc.stub.StreamObserver< + org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoResponse>) + responseObserver); + break; + default: + throw new AssertionError(); + } + } + + @java.lang.Override + @java.lang.SuppressWarnings("unchecked") + public io.grpc.stub.StreamObserver<Req> invoke( + io.grpc.stub.StreamObserver<Resp> responseObserver) { + switch (methodId) { + default: + throw new AssertionError(); + } + } + } + + public static final io.grpc.ServerServiceDefinition bindService(AsyncService service) { + return io.grpc.ServerServiceDefinition.builder(getServiceDescriptor()) + .addMethod( + getEchoMethod(), + io.grpc.stub.ServerCalls.asyncUnaryCall( + new MethodHandlers< + org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoRequest, + org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoResponse>( + service, METHODID_ECHO))) + .build(); + } + + private abstract static class EchoServiceBaseDescriptorSupplier + implements io.grpc.protobuf.ProtoFileDescriptorSupplier, + io.grpc.protobuf.ProtoServiceDescriptorSupplier { + EchoServiceBaseDescriptorSupplier() {} + + @java.lang.Override + public com.google.protobuf.Descriptors.FileDescriptor getFileDescriptor() { + return org.apache.beam.testinfra.mockapis.echo.v1.Echo.getDescriptor(); + } + + @java.lang.Override + public com.google.protobuf.Descriptors.ServiceDescriptor getServiceDescriptor() { + return getFileDescriptor().findServiceByName("EchoService"); + } + } + + private static final class EchoServiceFileDescriptorSupplier + extends EchoServiceBaseDescriptorSupplier { + EchoServiceFileDescriptorSupplier() {} + } + + private static final class EchoServiceMethodDescriptorSupplier + extends EchoServiceBaseDescriptorSupplier + implements io.grpc.protobuf.ProtoMethodDescriptorSupplier { + private final java.lang.String methodName; + + EchoServiceMethodDescriptorSupplier(java.lang.String methodName) { + this.methodName = methodName; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.MethodDescriptor getMethodDescriptor() { + return getServiceDescriptor().findMethodByName(methodName); + } + } + + private static volatile io.grpc.ServiceDescriptor serviceDescriptor; + + public static io.grpc.ServiceDescriptor getServiceDescriptor() { + io.grpc.ServiceDescriptor result = serviceDescriptor; + if (result == null) { + synchronized (EchoServiceGrpc.class) { + result = serviceDescriptor; + if (result == null) { + serviceDescriptor = + result = + io.grpc.ServiceDescriptor.newBuilder(SERVICE_NAME) + .setSchemaDescriptor(new EchoServiceFileDescriptorSupplier()) + .addMethod(getEchoMethod()) + .build(); + } + } + } + return result; + } +} diff --git a/.test-infra/mock-apis/src/main/java/org/apache/beam/testinfra/mockapis/echo/v1/package-info.java b/.test-infra/mock-apis/src/main/java/org/apache/beam/testinfra/mockapis/echo/v1/package-info.java new file mode 100644 index 000000000000..00b7fa2f70b7 --- /dev/null +++ b/.test-infra/mock-apis/src/main/java/org/apache/beam/testinfra/mockapis/echo/v1/package-info.java @@ -0,0 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** Autogenerated code supporting a quota aware gRPC endpoint client. */ +package org.apache.beam.testinfra.mockapis.echo.v1; diff --git a/build.gradle.kts b/build.gradle.kts index a97b73757249..b330bd07861e 100644 --- a/build.gradle.kts +++ b/build.gradle.kts @@ -54,6 +54,7 @@ tasks.rat { // Proto/grpc generated wrappers "**/apache_beam/portability/api/**/*_pb2*.py", "**/go/pkg/beam/**/*.pb.go", + "**/mock-apis/**/*.pb.go", // Ignore go.sum files, which don't permit headers "**/go.sum", @@ -198,6 +199,9 @@ tasks.rat { // Ignore typesciript package management. "sdks/typescript/package-lock.json", "sdks/typescript/node_modules/**/*", + + // Ignore buf autogenerated files. + "**/buf.lock", ) // Add .gitignore excludes to the Apache Rat exclusion list. We re-create the behavior diff --git a/sdks/java/build-tools/src/main/resources/beam/checkstyle/suppressions.xml b/sdks/java/build-tools/src/main/resources/beam/checkstyle/suppressions.xml index 7037f0543f4f..3f052508f2dd 100644 --- a/sdks/java/build-tools/src/main/resources/beam/checkstyle/suppressions.xml +++ b/sdks/java/build-tools/src/main/resources/beam/checkstyle/suppressions.xml @@ -87,6 +87,7 @@ <suppress id="ForbidNonVendoredGrpcProtobuf" files=".*it.*Client\.java" /> <suppress id="ForbidNonVendoredGrpcProtobuf" files=".*it.*LT\.java" /> <suppress id="ForbidNonVendoredGrpcProtobuf" files=".*it.*ResourceManagerTest\.java" /> + <suppress id="ForbidNonVendoredGrpcProtobuf" files=".*testinfra.*mockapis.*" /> <!-- Flink --> <!-- Checkstyle does not correctly detect package files across multiple source directories. --> diff --git a/settings.gradle.kts b/settings.gradle.kts index e8e374eb6e19..de83c11715c8 100644 --- a/settings.gradle.kts +++ b/settings.gradle.kts @@ -326,6 +326,8 @@ include(":runners:google-cloud-dataflow-java:worker:windmill") // no dots allowed for project paths include("beam-test-infra-metrics") project(":beam-test-infra-metrics").projectDir = file(".test-infra/metrics") +include("beam-test-infra-mock-apis") +project(":beam-test-infra-mock-apis").projectDir = file(".test-infra/mock-apis") include("beam-test-infra-pipelines") project(":beam-test-infra-pipelines").projectDir = file(".test-infra/pipelines") include("beam-test-tools") From 42586c4a8e7cf3994f2b7152d9380119841e8536 Mon Sep 17 00:00:00 2001 From: Shunping Huang <133698626+shunping-google@users.noreply.github.com> Date: Mon, 23 Oct 2023 13:59:21 -0400 Subject: [PATCH 230/435] Update the version of beam-master container image for python. (#29109) --- sdks/python/apache_beam/runners/dataflow/internal/names.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/runners/dataflow/internal/names.py b/sdks/python/apache_beam/runners/dataflow/internal/names.py index 579764aeb7c1..b39c4a2b7576 100644 --- a/sdks/python/apache_beam/runners/dataflow/internal/names.py +++ b/sdks/python/apache_beam/runners/dataflow/internal/names.py @@ -34,6 +34,6 @@ # Unreleased sdks use container image tag specified below. # Update this tag whenever there is a change that # requires changes to SDK harness container or SDK harness launcher. -BEAM_DEV_SDK_CONTAINER_TAG = 'beam-master-20231009' +BEAM_DEV_SDK_CONTAINER_TAG = 'beam-master-20231023' DATAFLOW_CONTAINER_IMAGE_REPOSITORY = 'gcr.io/cloud-dataflow/v1beta3' From f2df753cff330925c2d7655662bf50bb8fd1bbdb Mon Sep 17 00:00:00 2001 From: Devansh Modi <130608461+devanshmodi@users.noreply.github.com> Date: Mon, 23 Oct 2023 13:37:28 -0500 Subject: [PATCH 231/435] Heathcare : Hl7 messages to HL7v2 Store (HCAPI) example (#28873) * Adding examples to post messages hl7 messages to HCAPI Hl7v2 store Adding beam example on consuming HL7 message file from GCS bucket , transform it and post HL7 messages to HCAPI Hl7v2 store for further ingestion and consumption * Created using Colaboratory * Update examples/notebooks/healthcare/beam_post_hl7_messages_to_hcapi.ipynb Co-authored-by: Svetak Sundhar <svetaksundhar@google.com> * Update examples/notebooks/healthcare/beam_post_hl7_messages_to_hcapi.ipynb Co-authored-by: Svetak Sundhar <svetaksundhar@google.com> * Update examples/notebooks/healthcare/beam_post_hl7_messages_to_hcapi.ipynb Co-authored-by: Svetak Sundhar <svetaksundhar@google.com> * Created using Colaboratory * Created using Colaboratory * Created using Colaboratory * Created using Colaboratory * Created using Colaboratory --------- Co-authored-by: Svetak Sundhar <svetaksundhar@google.com> --- .../beam_post_hl7_messages_to_hcapi.ipynb | 528 ++++++++++++++++++ 1 file changed, 528 insertions(+) create mode 100644 examples/notebooks/healthcare/beam_post_hl7_messages_to_hcapi.ipynb diff --git a/examples/notebooks/healthcare/beam_post_hl7_messages_to_hcapi.ipynb b/examples/notebooks/healthcare/beam_post_hl7_messages_to_hcapi.ipynb new file mode 100644 index 000000000000..ab6b2d9233cb --- /dev/null +++ b/examples/notebooks/healthcare/beam_post_hl7_messages_to_hcapi.ipynb @@ -0,0 +1,528 @@ +{ + "nbformat": 4, + "nbformat_minor": 0, + "metadata": { + "colab": { + "provenance": [], + "private_outputs": true, + "toc_visible": true, + "include_colab_link": true + }, + "kernelspec": { + "name": "python3", + "display_name": "Python 3" + }, + "language_info": { + "name": "python" + } + }, + "cells": [ + { + "cell_type": "markdown", + "metadata": { + "id": "view-in-github", + "colab_type": "text" + }, + "source": [ + "<a href=\"https://colab.research.google.com/github/devanshmodi/beam/blob/devanshmodi-patch-healthcare-hl7-to-hcapi/examples/notebooks/healthcare/beam_post_hl7_messages_to_hcapi.ipynb\" target=\"_parent\"><img src=\"https://colab.research.google.com/assets/colab-badge.svg\" alt=\"Open In Colab\"/></a>" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "zQ_JXPR3RoFV" + }, + "outputs": [], + "source": [ + "# @title ###### Licensed to the Apache Software Foundation (ASF), Version 2.0 (the \"License\")\n", + "\n", + "# Licensed to the Apache Software Foundation (ASF) under one\n", + "# or more contributor license agreements. See the NOTICE file\n", + "# distributed with this work for additional information\n", + "# regarding copyright ownership. The ASF licenses this file\n", + "# to you under the Apache License, Version 2.0 (the\n", + "# \"License\"); you may not use this file except in compliance\n", + "# with the License. You may obtain a copy of the License at\n", + "#\n", + "# http://www.apache.org/licenses/LICENSE-2.0\n", + "#\n", + "# Unless required by applicable law or agreed to in writing,\n", + "# software distributed under the License is distributed on an\n", + "# \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY\n", + "# KIND, either express or implied. See the License for the\n", + "# specific language governing permissions and limitations\n", + "# under the License\n", + "\n", + "##################################\n", + "# Author: Devansh Modi #\n", + "##################################\n" + ] + }, + { + "cell_type": "markdown", + "source": [ + "**Highlevel Architecture**\n", + "\n", + "![Screenshot 2023-10-18 at 3.53.31 PM.png](data:image/png;base64,iVBORw0KGgoAAAANSUhEUgAAAn4AAACYCAYAAAB6UuTWAAABX2lDQ1BJQ0MgUHJvZmlsZQAAKJFtkD1Lw1AUht9oSyEWrCJODgUHFaq0SRfHWkWFgqFW/NjSNKZK0l7SiLj5H/zYHBVXB+mgg4u7IKjoooP4A4QstsRzGzWtei6H8/By3nvPPUBXVGXMDAGwKo6dn52Kr6yuxSNvENGPXgwjrGo1llGUHLXgu3aGeweB19txftd1KnkgCyiIjYeF47Onvb/9HSGW9JpGtUEpacx2ACFJrGw7jPMu8YBNQxHvczZ8PuVc9Pmy1VPIZ4lviGNaWS0RPxMnim260caWuaV9zcCnj+qVpUWqg5RDmMYMcnTiUCAhjRQmMUc7+t+TbnmyqIJhBzY2YKAMh9wZUhhM6MTzqEDDBBLEEpKUMt/17x0GWukVkC16ajTQNmPAhQv0nQfayAt95wi4Uphqqz+bFdxQbV2WfO6pA+FDz3tfBiJjQPPe8z7qntc8Abofyet+Arl9Y2Q/CPdhAAAAVmVYSWZNTQAqAAAACAABh2kABAAAAAEAAAAaAAAAAAADkoYABwAAABIAAABEoAIABAAAAAEAAAJ+oAMABAAAAAEAAACYAAAAAEFTQ0lJAAAAU2NyZWVuc2hvdLBKykEAAAHWaVRYdFhNTDpjb20uYWRvYmUueG1wAAAAAAA8eDp4bXBtZXRhIHhtbG5zOng9ImFkb2JlOm5zOm1ldGEvIiB4OnhtcHRrPSJYTVAgQ29yZSA2LjAuMCI+CiAgIDxyZGY6UkRGIHhtbG5zOnJkZj0iaHR0cDovL3d3dy53My5vcmcvMTk5OS8wMi8yMi1yZGYtc3ludGF4LW5zIyI+CiAgICAgIDxyZGY6RGVzY3JpcHRpb24gcmRmOmFib3V0PSIiCiAgICAgICAgICAgIHhtbG5zOmV4aWY9Imh0dHA6Ly9ucy5hZG9iZS5jb20vZXhpZi8xLjAvIj4KICAgICAgICAgPGV4aWY6UGl4ZWxZRGltZW5zaW9uPjE1MjwvZXhpZjpQaXhlbFlEaW1lbnNpb24+CiAgICAgICAgIDxleGlmOlBpeGVsWERpbWVuc2lvbj42Mzg8L2V4aWY6UGl4ZWxYRGltZW5zaW9uPgogICAgICAgICA8ZXhpZjpVc2VyQ29tbWVudD5TY3JlZW5zaG90PC9leGlmOlVzZXJDb21tZW50PgogICAgICA8L3JkZjpEZXNjcmlwdGlvbj4KICAgPC9yZGY6UkRGPgo8L3g6eG1wbWV0YT4K2MaQWgAAQABJREFUeAHtfQeYHNWZ7d89PVGjNNIo54wSIBEkECAhECByBoPBYIwz67Bre9drf7v7/N6u37e7ttf22s/GeE1wAJOTyEiAQKCMcs5Zo6yJ3f3OuTM1qml1z3TP9EjT0+f/pqerK9y6dapu3XP/+4dAWfmOqEmEgBAQAkJACAgBISAE2j0CwXZ/hbpAISAEhIAQEAJCQAgIAYeAiJ8eBCEgBISAEBACQkAIZAkCIn5ZcqN1mUJACAgBISAEhIAQEPHTMyAEhIAQEAJCQAgIgSxBQMQvS260LlMICAEhIASEgBAQAiJ+egaEgBAQAkJACAgBIZAlCIj4ZcmN1mUKASEgBISAEBACQkDET8+AEBACQkAICAEhIASyBAERvyy50bpMISAEhIAQEAJCQAiI+OkZEAJCQAgIASEgBIRAliAg4pclN1qXKQSEgBAQAkJACAgBET89A0JACAgBISAEhIAQyBIERPyy5EbrMoWAEBACQkAICAEhIOKnZ0AICAEhIASEgBAQAlmCgIhfltxoXaYQEAJCQAgIASEgBET89AwIASEgBISAEBACQiBLEBDxy5IbrcsUAkJACAgBISAEhICIn54BISAEhIAQEAJCQAhkCQIifllyo3WZQkAICAEhIASEgBAQ8dMzIASEgBAQAkJACAiBLEFAxC9LbrQuUwgIASEgBISAEBACIn56BoSAEBACQkAICAEhkCUIiPhlyY3WZQoBISAEhIAQEAJCQMRPz4AQEAJCQAgIASEgBLIEARG/LLnRukwhIASEgBAQAkJACIj46RkQAkJACAgBISAEhECWICDilyU3WpcpBISAEBACQkAICAERPz0DQkAICAEhIASEgBDIEgRE/LLkRusyhYAQEAJCQAgIASEg4qdnQAgIASEgBISAEBACWYKAiF+W3GhdphAQAkJACAgBISAEQoJACAgBIdDWEHh/zty2VqVWqc+Uiy9olXJVqBAQAkIgEQIifomQ0XohIAROCwIkfdddcctpOfepPul3v/9t++4/fvtUn1bnEwJCIIsREPHL4puvSxcCbRmBsvIdCasXjUYtf+9ay9+1DPsEzKLc1fvGj9q/uvXY5LZ737UbsbehmNpNUfzKCVokVGDh/GILdyyxnPwii+bmYx0+eYVu2QI8Kj2SLeQ2PWipFCEgBNKFgIhfupBUOUJACJwyBAKga0Vr37PAiz9tnXNGUGxhV7OSvmalg62q90ir7j0cv/tYdedeFs0vaJ3zqlQhIASEQCsjIOLXygCreCEgBNKPADV+0Q5RC/SAf9rhsFklzkGtXbqEbm+VB8x28rPM8pa+aHn5nc36jrXwkPOsaug5VtlvFLSAeek6o8oRAkJACJwSBET8TgnMOokQEALpRqCmd8jyphaZ7Tlqtq/CbC/mbQ/lgrDhteZN7abzpJWHzDZ8YDkbP7bClaOtcMRUO3je9RbtWprOs6gsISAEhECrIiDi16rwqnAhIARaC4FoHtRyQzDl2h+k71iN2YEqs4P47M6Blg5nPZBvVgUimG6JVpttX4LzrLfibUutfPpXrGbgSJgYplHjmO46qzwhIASEQB0CIn56FISAEMhMBAJQ6+WAbAVBAHPx3Qm/+4EADsHnsFnkcI0Fd2DbNlzeAUzJVqXZLq/mqIXWzrai8sN2/Jq/B/kbIfKXmU+Sai0EsgoBWrJIhIAQEAKZjQDfZFD0GU3uYIpn8MkIDoft3yRo566utug1x83Ogb1eKT450AqmTSKWs32xFb72M7P9e9JWqgoSAkLgZATohS9pOQLS+LUcQ5UgBIRAW0OARJCzvPzADDBQgh5jABxCzsPvg0cssBrfm/DZ3wX/yBhbINGwhbbMt65z/mhl13zVAnly+GgBmjpUCDRAYMMus4UborZjf9QqMGYrhAXHgNKATRwWsH7dGuyqH0kiIOKXJFDaTQgIgQxFwJvXwHegjghaT1zL+ZgOPnjQgmuwvB2vwkNgiNUggTWYNk7VQzhcbuGt8y1380qrHjreAkHZ+2Xo06JqtxEEyqui9vxHUVuyMQLz2RPt6eAxmPCWRe2jVVGbPCpoMyYErICmHpKkERDxSxoq7SgEhEBGI8C+wesfqOTDJ1iI7x7geVWwC9x/2AKb8HsPNHaHoFY4hp2rwRYjyWkEc3atsQ6r59rB/iPMCliwRAgIgeYgcBCWGX/9AE70OxGyyUf6/GVx/Sdro3a8KmBXn2vWMc0mvP5ztbfldkn8dh/y3u4Nb1dRfgDv/fjbGu7ZNn6VV0O1jU9saIrSTuhXcmXs0DbukmqRsQjwVcAPNYF8E5Kr9UZzq66ywF7MKW3B74NQER7EBmoDoYGwMAhiIolWWPW+9RY4tN8sH0aGCTqsRIdrvRAQAmhmaGLvfRqxLXua7uPCCLS+dnvEFnQO2pQxAQslN0bLepjbJfF7b03IXlnCOZ2GcuXYXBAmvOUzROasqbYP18Mw3ff8cwbp8S+GbVBpIx1QhlyfqikE2gQCdeTP1SVUbIFCGA51QtaOIV0tWpNvgf1Bi+7qYtH1ey249lMyw4TVzt+3xarKdlp1aZ+EmoqEB2uDEBACth22fOug6auBb1YyUo4x2podERvZL8d6lyRzhPZpl8SvPd/WDvlB65CfZItoz0Do2oRAKgiQ3FEC0N4FEXCZnwA/cAGmZs5t578O2AaHjwDU6tgWCGDat6SjRUecYRaabbZqHfYB8Us0fjyww3IO7raqCOySGGZGIgSEQEoIbNmLaEzlKR1iexG+adfBKIifa8ipHZyFe4v4ZdhN78TpKIkQEAI+BDi/Q9ddfuND0haExi4AzV0AKoAgyFv9NtjuBYrrPiB5RsMgdBYBkjR2Gvz2VOxcxisyfxJCBcIbJG9FreNHDbYXYN94vK7qiNUcPwi7QA7O9HoFCBIhkDQCbHkHj9HECa3QP9XVRAnHKgJ2BGSRx4v6NQEWNuvN1DRGbWYPTvPSTlEiBIQAESCh6wWux9cYPjTUo4bOkTmMkJx2D8TOufJyPUkh9637jsZjbtjsFxLCILw/vO6EPYub6cVCIdpibHMMRjATjMTBnKfiKSVCQAgkjQC53kcbK+3P85GNJ7ZtNVYKvPD79S60i0ZzYNfYjtpGBPgWlGQIAoV5gbhKhgypvqopBNKMAEkZXvQM1kdyR9IX4G8yrjqi1+JegFrEOASR5I+9VIeYXgY/o4jrl4q2AiVJhIAQAAJUbuSjnyvHuKmKmvUkhQoRHid/quQAE/FLDqcW7dWtOGAjeuW47FLxCqqEn8YnG5t21uigBzsefFqXtQhAKxDehquHax+JXhBTuPy4aV/aRGD6N8p2VefoRWJILZ+b1k22U+Hx0OAZCWWMcBOmpawIvZXH/1gVTiurB4oBSz+FQHIIjOufY907BmzHgWTbKAI6dwva0B5xBmjJnTLr9hLxOwW3vLQYD2VpyHIaeS7nb6pxCoRE1eGx+Sc7KifaXeuFQJYgANVA9BCuFanYakAEKV5/4X3ngAAGR+CDWC105ghy+pfEkPZ9JIL4DmIfkjs3LUwWV9dYob2zqqXgjmdgHVldjJD8leNEns0fysop6Gg1OdQUSoSAEEgVgSkjQja2X8h2H6oyhmtpSvJCATtvaMjGD1Cbaworb3vWE78Q3u8tjf1Tjb4hmQfUA70537lIRs+PRAgIgToEnHYNy001izCiwYYXY0d+fMLj6XlLG76c/rXfATqJcJrYmzpG4eHNZhUbwC03wb7PGfj5CsGiZ/NH8te1L0LB9ITSr444NtxTv4SAEGgCgZ6dA/b5qQW2cW/Y1u5C59qIsJmdNTBkd0zKs5JYs4tGjsv2TVlP/Pp0CdqQHjnOtqA5DwMJ35rdYdt5kL1I60ke7lRLCWrr1U4lC4FTj0Dg6DGQNXjQFkHjluOp91KsB8KuWGQXtIX4xAoJJaeFD4PMLYATyUJoFA/hnAU4F7QMDYSavwp4I3YbZJXd+4FPSvvQAB/9EAIpIDBtdMj+4foi+/nrFbZwY5zBFspiC5wyMs++dnmBnTsk66lMCujKucM6wjOvf1cQv2YO0GvQb2xvZdJHcyHmIqThq0QICIFaBILHQcIq9iBXE6Zoi8G8aArRzHYcF1MqG1aA5H2ETBzLQTCr6joXcszCGPLHdTmY4u0x0qJdusPET401LqZaKQSSQIAzcdeclWuDugft1cXV9scPjtm2MnS2HIhFIzakNGj3TSu2S8fk2sjeQfWNSWDq3yXraTKjLhxHWrTmkipq/MKNa6P9eDdrmXUrkH1fs7DTQe0cgUpq7BDAC9H7GXvZpV3jW62lvAuc0p7GtO8mdDS7YxoftXuYPXaaRp/mr6rncKsYiZh/hZwubmkFUL5ECGQxApzGPRN2e907RG3FqqO2bR1tedmuojb57BK77+KSJsObbUEWkLeW19gNE0PWVVPB9U9T1hO/jfvCtuMgmFtzX9QI6cBgk60pITC/Qmj8JEJACMRBANzP6NdBQkYCCN5V74SLziMl4SDuE3zeB4vcAsIXTtDuuB/JYTFUfbC9DXfqZccnXmvRfkMtGMr61yqAkQiB9CBQgGgWBcVoU0G0tRpq9nOssBjpV7G+Kdl/JGI/ev64PTc/aN+/ocgmDkJ0jVTfCU2dJAO3Z/0bigmh+TnhCtj27mIHxChqLi9te1ejGgmBVkCAU60kgNTEkfwxqgsjsPANl+yL/gj2fQHevvNxYDU6FZbZmPB8PKak0KrGXGkVZ0+1vKIOmuZtDDNty2oEOEPG/jYChQnDYCYjlTUwxyrtZueO61K/e5+uITsGm9qm+sXjVVEXD/C91dV268+O2P1T8+3BaflW2jGY1TbzWU/86p+kVlw4iHAPZUcjCUca5ZhqbqwRFCPahEQICIEkEaA2joSM4fcYtYUEkL4WiRQE1NgvxcansOOR1BpbNDffKsbfYgdm3mv5XbrCm1dOHUBTIgQaIECyt3lfxFbtDNvSLWHbc7gGyjsyv6bZXzWOXbkTFhc+W/rnl9TY2n1s4I1L2VEQTfSvlCMVEfvZrHKbtaTavnVVoV04Isd6dYZ9YLIDw8ZPlVFbRfxOwe3adShir3xKNUTqksNp3iRU2qmXrCOEQDtHgISOJJAafZI/vu38BJD9wXasmIvPHEztNt0H4YA6AeGLdO1hZdd9zaonTLb8Tp01xetho28h4EPg4PGovbAADhofVtoixKutDqfS0HwF+Rb3wcxiyVbfihQWV++ssa//4ajNPCvPbkcYmHMRAzDbQsGI+KXwwJyOXd007+k4sc4pBNoDAuxj4PvhYu1RmccQfXzrleFDG75HQfgqE6kCsU88CeVZzagLbO8dX7dgzz6Wz+ndbFQbxMNG64SAD4FDaHs/ebXCfj+nwo5iaratSBXI50uLK42ktFeXHBG/tnJjVI9aBDqlNvMk2ISAEIiHALV+JICcHdoM9jcfZG8xvECa0xcFQ1aTl2vFy+fCfvBCqxk0PN4ZtU4IZDUCNF/6/exKe2R2hR2rbE5Daz348uGNf91EavzybWD3FAd+rVetU1ayNH6nDOrUT5SLaV4+oBIhIATSgAA9f+n8sQFtag+M/5rbF1Udt4IFr7tPePRci0642A5PvxlZ4OhOLBECQoAILNsWsafmVdpxH+mjR+2wnrXp1YqofU9CKmCj9/HKI7Zx1wmbvlH9i+y8M6Bpb+L4fUei9vqn1Q2ml8f0zbW/vbrQzh+WYz06ZWd83HZJ/OADi7h8Jz8SQRAp/GWMdEJw6X1HUeEW1Pnh2SGbMChiEwbSBVEiBE4PAo/MybFD5QG7bEzExvU7xc8iT0dtH8Ov0O5vIDqQTmB927G8EO6/zsgcy82QnBXQ+m1cZkUH91gFHDyiJd2bUYoOEQJtA4Ev/k+uzUAbnY5PF2bEaYG8AcK1rSxcP77KB9u46dx8+/vriqwEzS7Zbq3sSI39sGyfbVyLfNx1Mql/jv2vW0ub7M8Xw5Hk3VW1doXsTx+8tNB59TKmXzaHdWmXxO/8oTU2oNvJnUvvLtUZdbNH9I7CE+nkxtepiBbriYVkj8LO9v6LwyJ9iaHSllOIwPMLc+ypj3OsX0nUZp4ZdiSwX9eTn++0VolED0k3nIMHmw17G3rxdYOzVSd8d4Hb3zpo6jZiQ3OrUn7YCt78C85RbeU3PmjWuQQFS4RA5iGwYnvQVu0I2n/MMrtoRMSuGBe2S0ef3Jc2dWURtKUV22saTPEOgabvm1cVof0nS/lqz1KYH7TczkUWKj1Rj7xORc7psSlFDjNeMQbupKF59r3rCu0sBITOZsLn3bd2Sfz6lUTwcJ14SLyLzbTv3ifCFjVZdT/Z83Ym6XvgEho3SYRA20FgW1nA/vBeyH7zjtn4/hGQwIhdNjqMKPxpriMD/XNq1yN8/uLZ93CqqR82dkPsl1JM/S7BijgDLf9hCZcrjlnBO89YsGN3O3rtZy2Ql+6LSXhmbRACaUWguk6v8P7aoPHzv180uxIEkNr6s5OcOWKMvaPIqkMCSOEEXA+EThnWMzXSx2NDcJwa3K2zndMfjlh1MrBrblIaw+6I1/fDm4rs2rND1hkaP0ktAu2S+GXTzSXho2YvVm47L2xXQ6uy82AaH3YUlcbSGlQ5pXKT3DnJ3RrUI5kf9eXWLyRzVNP7tKi4BAcnWN10ZZrYI2G5CTaUVzXcUFk3Hlm6NWgroGH48cshmw7NwuVjwy1/xthx7cYnHuGLva4gBogd4N07BgyxHyr1CQgb6tQs7V/lMQvNe8NCoyZYzeizmwzkzDzfxOEwp6HrpSFO9auxkLJCMskDktzNX5UmlxuU2eBHk4c2ukOLikpwcILVjdYjmY0Jy024oelSW3DoyYU3UZgXW7YcyvGXFufYswsY947aekwFY6A2uDRxASFks+GHT7O3VwWCKXMGq2NB4mf85Eq2bE1/aBfvvgDtW9IAARG/BnBkzo9EhM+7gicxpcaPRAi0FQRyE7xtmC+b8s7KoL25PGg5e2pf1Mu2BW1sKvaALId2fCRSXm+DxaYFRC8H5+wKJjYV9n8rQf7WYN1hHJlSOZhF3rLccpZ+YDUDhyN7SMdGT03N5+OIITjr36UdbBQobTxlCMQxjXfn9rSAuw4F7Ak8s1Q2nDmAmnp8oAnsHGMPyNzyfbsELQ/OiZWwoSWJ3Lw3bC8tqrabzsmz/NomfsquSydqiECCV3HDnfSr7SHAKVw6bfwOGr+j6KvW7mo4iroVGr/b8aGk2HelfLFxy4+7MuWi3QFpLCouGGktP84lJlV+UjvFKRyrWnBo/AJj1noj/5jVKf18Zn6OvQitQTyh0Tc1X5z2vWp8xIrKqu2Why150kerDsZHJ+GjTV/DpoAVSQp7PaRHtHEobCC+P0LvVAYCyOnipCVqhcs/tupJMyzaYUTtHFeCY/t0idr4CTDH+AIrH18SXkrCDfHL8a9twaH+YmqXfYX5Fk/er5lrki4z6R1PqnYza3byYa4KKdbj5FJOXtOsIps4KNHmu36dZ/HiK4fQDKihpsOHM80YE7ZRsEFPJGxKk4bn2guLqmz3odr99iJv7n+9Vm4bQQBH98kx2t8lkgDOd/GoUMIc9Vv2h23W0urapo5imOSgf0kQJl4km4lK1XoPgXYJEUcnNXGSq4fQ7yR+1DxI2s43GxpzG8ZKYV7UeTPRU3fCPbWdxsLNQUcCF22uvUIa0XdEDEDZ+MWip9+nA4GuHRp2Enw5M43TgG5RZ5JArUFvECHK+3NSqCGJHj8M1cLD+fi3tJFjispoXzsdbWsDyN86/N6Dl0e8HhG7xUpw43IL7Nhskf5DLRBK/IolBt3g3TiiV0NsYsvTbyFwuhDwHCFogsE2OnlYnA4pQeUuHR2CVjBkby6rcrZ+7MvW7Arbr9+qcPZ2XtnxDg8h9eGL3y62wgR27h+uq7FXV5wI75KDNluMQdvoviH77JR8m4RQLU05fsQ7b7asS/xWymAEXlqch9HAybrkK8eGoGJuaa9w6oCZs7rGPlxf43LPe2dlmJrHvgCv5e7s7U5IPBLo2f6J/J3ASUunH4FO8KOYOb62IxndN/mOJLbmARBHl5PXKyKdTZtl5ULtMBKjyL6Yil2HD0JD2F60u6Z4WrjSAnu2gZCCODZC/GKvR7+FwOlGgI+9NyibiBklauCnQ7tHrXyq0rkoYD+4sciWbq2xXXV5djl7wAweTWXxyMUUcdh5hsRv1Dx+ywGv4Z+o2codNSCalfa1K4rsb2bIhOIEMg2XmnE7GxbQFn/xgYmXD5CaQOa+zRShxq8G1+KfaivEqIYav8YklgRSG6g4fo0hpm2nAoGpo2gPFLYpCBORFiHx44dNutnNGgfmwlswFx69IQwWSdQ4T+WV6aYJoO0b3hOOH/h+Yxk0f4mnZr3ryjm4z6pB/BTU2UNE35mAwMDuURfChdo9OnK0VMb0Ddozf9PJPvPfR2zLvnC9l29Ly010PO2F9x+N2j8/fcxqYFv4rasKXHNOtH+2rm+XxK8930xqS1IREb5U0NK+rYXAnZPDsNfBW/l0CB03cjFnlIt4fSHYP+TgtcfcuiR3QZC5ILZzepcfDgwD7PDwoccv98sbCs/fydD4rQPRXFk75dtERp2cwweh8WuolT8dl65zCoFUEHjsi00PalIpj/uO6hO017/byX75ZiWyaFTZwWMRmGI1rjinRzCbXiKhfSBDtXhCBxJmCPGbRv389XKbODhkU88QzfFw8r6FiIdEBnyzT+pA43OJEMgwBApPtrxI3xWwSbAPCFFr1xkfaPBysBzkb36D3JH8wW7ILVOjx/29fiMAghdAT0TCx2W/BPCK7Dgdx6Lc4BZzdhfl2KcIB5MoJpBAdaVFIzFlJdhXq4VAe0ege8eA/fCGAvvKZfkIEB2GVg4x/hppHmyiXTBVnEgmDwvZpNH5buxG09udmPZ9Z2W1zd9wImg08wM/MqfSzh8KJxG8BiQnEBDxO4FFm1+i51IGzVS3eTxVwXaAALics8XLAdkj6QtCq5cD2x6n1cPb3pE9vOacJg9kzZvGdZdOzV4TEmBZKNcv7LCgtbBiEkn/hhPLUU4bN3/++URBWhIC7QQBNr1SEMDSkS2nHUN65LiwMCzTkxnj8uwHTx+32SuqnAeycyaBzd/63REb2z9BQ/UOzrLvlt+BLAOsOZfLvIDD8KAmUhAwlMWiLTRWalw6gPj5H/TG99ZWIZAlCDj+hn9RMjJOJ+M7CU6XFDpRtksW5utheCBXHcW5EpC/SCHcdUk6JUJACJwSBDilPGNsni3YWOOmk3lSav02I/SLiF/DWyDi1xCPVvnVE7YII3uFGs0RuBieT34njtiKUGGR15rTZbEn1G8hkAkIkOeR8IWRczeAD8kWNX+5CJ4cgfYvikYTpMYPH8aPiGC70wJSAxBD5uJeL05Qswdl9jh5q0f+OqCsmFFddTESAdMxRCIEhEBKCBwujyL+X9iuOzvHOqWYZq20U6BBHD9OA1fI1PYk/LOe+LEvcO/sZPqAk+DDCjxY9L71chLG2yUd6xgBPS+mc0lHuSpDCGQ8AiRgVPaxDUdB1CJlCBKID8XxOzZyeEUFaOeHqdtcOHjQ9i8XvwP4pho9iEI4unKfupeCKw+9xqFXodm7CA0dDhuxozOe+zhOXohj6h0+AhbpWorTybDI3QP9EwJJIMA+dOGmsP10Vrl9siFq087okDLxo/0g8wR7koexV+cUHSK9Y9vzd9YTvz5IKzO4e/ODPXJEsXZ32HYdYs/TesLYSlIgtB6+KrmdIsA+gBrBKHO5QThzy7ivJHWUAEggyR9dCD0nENrnOc0h1jGFwBEQvj07zD7FwRUngsbWFoD/bPoV+FdQR/6Ku1m0ex80WJQjEQJCoEkEqOX79VtV9v/eLrcDsJ/NRWcXO8ZqqpB3VtTY8wuq3PSut29HDMhG9M56muPBUf+d9YhQlTywG4gf3tnNEWr7dpL0HWrO0ckdQ4UE3depjJAIASGQRgQidUSuXmXPFwEaGriebYG6gCnb9uGzez9y96Kds8FTkRc7i8spZ2oawCMrho6zSPfe4JGxO2EfiRAQAg4BEjsSvvkbw/aj58ttyeYTc7JhuPz++8sVVlwQsAqEaqEmcHddEGge/OKyiG2DjS27RDbdbWVh+3BttYvh5wrHv1w0v8lIGzegmzpODxPvO+uJH+MJlVfXpkDzQEnlm/1AmC/9VhQSvsKsv1OtCLCKFgKuFwIMG0DydpDsYXk/Gh46JkSDb4gPQ50xiHqs6QXV/zV5Vj14rEW79cAMsjqchsDplxCoRYCEbyWmZR97v8qenFeJuH5oOz6JgM09+j7zMMaXnevC9tG6E0Qx3l4jeuUgZamyd8TDJuvpBD1+9iB5dLMFz+uxqoYPbbPLSnAgs43kw6NXIgSEQBoRiJLY4RW4AqqBTfjei7KPgvRRCdiYkz15YCWOLUC7j1HDVw4bZ5WjJlqgCI4lIn5pvFkqqj0hsGYXsms8U24fr69uFft45uz9/g1FmOZFe5achEDWEz96/FRA49eWpUjx+9ry7VHdMgkBNnUqEpZjTnY1Xn/bQeCq8eHYL5XxH/etiCF/HbtY5YSLLDx4OGyUZN8HhCRCIC4CfbsG7O4p+VaAvm3Oyqq0kb8ShFeahkwdd1+Yb5coY0dc7Lky64lfQmTSuOEoYgkxnhA9iONJeVWkUUPWjvJKigeb1gmBphFwhA4ErRy7rsXrbgWmfrajIaYjMxXLLkfZhWCT8OCtPnuKHb9kJlL/Fmuat+k7oz2yGIHeXQJ2+/l5dg5Sqr25LNd+926lbdx7Qs0ehLb88rHIuJEftKrqiC3dcNy27T3hWDWkd4GNH4Kg7Wh++HMEckC3oI0fELLx/XOsD8qXJEZAxC8xNmnbsg3pZLYdOPHQplIwyWIRHDskQkAIJIkASV0lGg6b3Ca0ndXQ7u3CdC7JX7qFGsRIgVWdOdnKbrnfcrqXyqkj3RirvHaJAPu2Eb2CcL7ItykjQvYwyN+z8yudkiQIb8vvXVcEAhe0A0dr7EdPlNm2DfS4qpVJ53WzH9zeHZYWtX0jnTMLoGRXdisPoca/Rfwax+e0by1Wto7Tfg9UgbaKgG9ARJJ3FL+PgeDtxPImuN7uxOstsX14Wi4q0qWrVU68wA595gHL6TcIKYKhUZRtX1qwVSHZgQAJ27gBOfbjOwrtqjNz7d9eLLfVO6PWrRjp3RCQmSSwoBMGb51hN1snRZ3zsS0Ya2LrbdZ3EwiI+DUB0One3BHu7BIhIATiIBDGMH8X2sdB9Bx0zNgB0rcXrzROv7a22W5hkVUMGWmVF06z8ktnWqg7tA8gffLkjXOftEoINIEAezlq60j8zhkSst/OrjbatlNyQfyG9uhik4YgDWKdDO6W66Z4vd/6Tg0BEb/U8Dqle+fSm1fTvKcUc50scxAI78+3nLehCSgD8eM0bmuTPWjyop2h4Rs20qpGn2UVEydbdPgoywUJdDH7pOnLnIdHNW2zCJR2DNg/XMNgmZLWQkDEr7WQTUO5RfmBhA4haSheRQiBjEWAmrXoEXQO20D6WlOKO1q4tKdV9elv1QOGwmN3mNUMHGzRPgMsWNTBgrkhaPmgeZQIASEgBDIEgXZJ/M4bGrZ+JScP//t2qa43Bs2E+zO8d9SOIhtA7JV0Kkwl7kQmXKnqKARSRABkq3L8+Vb9pYcsHDnhDXiisZyY7q2dMPKVzwblzwfla2BRaNmj8NCNFnawcKcuyNHbCTH5ii3SsZMZfgfgsevIHtK7aVrXh6kWhYAQyBgE2iXx618SMX5i5aQOIHaHNva7bxurj6ojBE4lAtddcUvjpyN5Yx5ev/hInH91KsuM61wfJ8JvSYT1Af/vVAqNs+/7c+bahRdNjrNFq4SAEBACrYdAuyR+mUbwWu/2qmQhkHkITLn4Avvu97+deRVPscYkfd/9x/Z/nSnCot2FQAMEaDrLvLt+CbVL5uK/wtZdDpSV70jDGLl1K6nShYAQEAJCQAgIgexDAGl7beW2iP1qVpVt2BW20Qj98g04f/RDwGZJ8xAQ8WsebjpKCAgBISAEhIAQOEUIrAD5m7cmYpedmWP9u2leryWwi/i1BD0dKwSEgBAQAkJACAiBDEJAutIMulmqqhAQAkJACAgBISAEWoKAiF9L0NOxQkAICAEhIASEgBDIIARE/DLoZqmqQkAICAEhIASEgBBoCQIifi1BT8cKASEgBISAEBACQiCDEBDxy6CbpaoKASEgBISAEBACQqAlCIj4tQQ9HSsEhIAQEAJCQAgIgQxCQMQvg26WqioEhIAQEAJCQAgIgZYgIOLXEvR0rBAQAkJACAgBISAEMggBEb8MulmqqhAQAkJACAgBISAEWoKAiF9L0NOxQkAICAEhIASEgBDIIARE/DLoZqmqQkAICAEhIASEgBBoCQIifi1BT8cKASEgBISAEBACQiCDEAhlUF1VVSEgBISAEBACQqAdIrBl4zbbtXOPBQJmffr1tr79e9vaVevtpWdes0uvuNjOnDg2rVf99J9etC0bt9pD3/mi5YRy0lp2Wy9MxK+t3yHVTwgIASEgBIRAO0WgurrGnnvyZVu3eoMZSJ9Fay/0wYc+Z+FwxCoqKq2mJpz2q6+uqnZlp73gDChQxC8DbpKqKASEgBAQAkKgPSIw74P5jvSdfe54mzbjIouEw7YZ2r9u3bva3t37TrpkagWpqcvLy7PBwwZa5y6d3D779x2wTeu32KAh/a1baYmVHy+3FZ+usZ69ulu/gX3dPps3bLWtW7Zb7769sk7L5wcyK4kfRxib1m+2fXvLrKAg3wYNHWBdS7r4cdGyEBACQkAICAEh0MoILF243PLy82z6lZdYbm4tJRk1Znjcsy5e8Km9+vybjrRFI1HLzcu1O++9GUSup+3cvstef/ltm3nDDEf8Dh8+6n6fO3mCI34rPl1tzz/1CqaSA+5TUJgf9xzZsDLriF/Z/gP21ydesP37yurVyh07FdtXv/2Aexiac9MPHTzsVNW33nW9FXUoak4ROkYICAEhIASEQFYhEIZ27/ChI9ajV2k96UsEQDQatbdmzbEuXTvbvQ/eYVWYqv3tzx+1OW99YLffc1Oiw+rXfzx3gVv+7AO3G/v8R3/75/pt2baQdcRv1gtv2f79ZXb1jTNszPhR7qEjceMogGpgagHHnnWG5WMEsmfXXtu6eYcNHTHIPWzHj5VDBb3F9u0ps05QL48YNcTyoTH88L1PbMe2XbZo/qfugRp/9hj3HCVSSW/dtN2OHDlq/aF+XrZkpYVgWHrG2BGuDsuXrrIOxR1s+MghbhSUbQ+krlcICAEhIASyA4GcnBxM2eZaJez4mpKy/QetqrIK/e5Qp2Ap6mDWpaSz7di+O+6hJIp+OVh2yDp17uicRrieDiSrV6z175I1y1lF/EjwNsM2YNCQAeaRM07xetO8y5eutCVQO5Pokfhx3zdfnW23dL7O2RH88fdP2SGMTkpwzD5oDEnY+CB+uniFe2AWzFtixcVFruzGVNIrl6+2xfOX4eEtNM/A9JMPFxkfVNolcCqapPDuz9+WNQ+iLlQICAEhIASyD4HSnt1t+7adRmJX0i2xyRUJIsXv6MH+NxRqSGOqKmtJ5FFM9fqlhDaDe/Ybp4gDwYDru/3bs2m5IWLt/MoPlB10V9gDxp6pygGMFvjQ9B/U12687RoLwRYhFw9cMCcIreB2o6buvi99xmn8klFJU8VNY9YLLznfngChpCv7pCnn2CXTL7THH3nStm3Z4UhgYVFhqlXV/kJACAgBISAEMgKB8y+caE//+UV76vFnndKEfev2rbvs+luualB/Ts/26tPT1q5eb1SyHD502M3YnTVxnNuvW/cS902bwYLCApv/0eIGx9OWf/vWnfYa7AC7YMaO/Xa2SlYFcC7Ew0BJRq3M/fyq4q5QKXM6ltO0v/zPh+3dN963SDTC3U4STyXdb0Afp5KmTUI8lTQ1j5Tupd3c93CosEkkae/Acx87etyt1z8hIASEgBAQAu0RgRFnDLMrr5kOG79cm/3WXDfLtmf33lob/JgLvv7Wmdardw97/ZW37eO5C23k6GHwBJ7i9uL6sWeeYXv37rdXnnvDBg7u52bVvCLOnDAWnsIltuiTpfYpTKxo6pWtklUaP7p4U8VLph9GXKCTgzYyiBCIIdTHlKNHjrlv/qMN4A23XW0Tzj3T5rw91xZ+vMQ6wJFjyrRJ9ft4C8mqpDmd7Bd6GFN4LokQEAJCQAgIgWxAgLNf/LDvZf/n9aH07v37f/lmPQScCr7nC7WOHTlQktBG0BP27dfefKVded1lbhU9hBn42ROGfXnwoXutorzCaQS5fuYNl3ubs+o7q4gfbQHOOf9s++TDhfbnx56xkWcMh41dlVXgYZt2+RSjDQBl7ux5NmT4IKPK2BNOE69esc6oxRs/YYwjj9U1NW4zVdCUxXDu6IoHk6OOxlTSbmf9EwJCQAgIASEgBOoRiFWG1G+IWfCIYcxq99MLCRNvG9dxGjjbJauIH2/2dIwAopiiXbtqg73xyjtOAzj+rFov3HFnjbaVy9bYquVrbSMCQZ4z6Sz74N157hmJIIL4h3M+dpG+OSIhAZx43pluG4keHTzef/cjKywqcMSPKumXnpnlVNLBQLCBStodpH9CQAgIASEgBISAEDjFCATKync09Hk+xRU4nacrP17hVMqxU75MEZMHewPa2/ml1usWx+TnnuRJxDQzx+GRSztCqpw9YayhWJW0t03fQkAIGGJqHrB5738Cs4nJLtxCujFhnLAnH38OZhrjbULdYC3VczDskic0yegJeyJP0++t17cQEAJCIBMQyDqNn/+mUDsXTzxbu9ht1PQxBEtcAdeLt60xlXTccrRSCGQZArPf/MDF02IU/stnTkv71TP8A1M/tcRZ6sWnZ7l6cVDnhYO4FOmlzrtgYtrrqwKFgBAQAq2JQEOVVmueSWULASEgBGIQKIehtUvOjvUMicQwRxSaVjAo+rGjx4yB0BneKBI54UXPZa7btGHLSYnWDx44ZBvWbYZzVsM4Xix3PwK0b0S6xpo6+1yuo+xDqKaNOIZB2hPJEOQF/d4/fcMe+s6D1rlzp3ozEO7P+jJUxJZN21wcTq+MePXkNXrXxggArCvjedYgfidjh+5G4HiJEDidCLAtMBatv52wvXEdZ8paIm+/Nsd+8n/+uzZ7VpyCWD63v/TMa3G2nt5VNOma/9Ei27Vjd4OKcGBJbPjh7ADbt1943Krla/yrTutyVmv8TivyOrkQEAK2EvkzSYSmX3mxS8e0fs1GY3iHY8eO2x9+86dahGg5AVMKxuG6856bYVJx3B7/HYOpH7ZgsNaz787P3Yxk7KX2ARyz6HVPQUZOu/La6TZgcH/3+xO8sN+f/ZErq0+/Xs47kFr8F/76qiOd2N2FlLjp9mucc5c7KM6/goICZ+rBrD0UksU//eFpl+mHZXTp0tnuuv9WF+A9Xj2LEJuT18YZAnZyNCFhQvoAbIH37dvv6ncVPBPPOqc2PlmcKmiVEGhVBBYgasXaVetdlgwvQDIHZoxm0advL2fL3twKkEzSnIqacwqdJpk84aY7rq0rMuq2V1dX1/1uG180GXnp2VoyOmTowAZp4ugT8Nas2Q0qyiQRzBBGeePld6wYTqCjxoxosM/p+iGN3+lCXucVAkLAOUWVdOsKR6qznX3s0kW1WXD80HznBw+54Oab8HLduGGzS5nIpOwM6/BFhGdgtht64DMzz3vvfGhnTRhnf/eDr9tQeOYzLhhJI4XxO7/zw4eM8bw4IqenPjs3dmgM+/C33/8aYmqWOAJae0TD/9TM/esPf2L/91/+C6kd99vM62tDQXz43sfGuGOf++Jn7Cvf/LwjrUzjyNSO8erplUrCyHrSOYx2juOQKvKb3/uyS9W4sg1pB7z66lsIxCLAdvTR+/OR8WpZAw07iR21XByEsX2xjcYThlZh+9mC+LgMykxNoycM1bJh7SbXpmNTq7H8lctWu22MpuFpIantXzBvsb339ofuWGrRKSSbLJ/vCGrmWS9vW6Jr8Orhfa/AdfBdMnrsSLyHtmDAd3KcXRK9B79+r0sLR80fZyzaomSMxo+qVKZ1YdBjBkQ+3cKHiGFhPvvAbS63brrrw9HFDqaxQYfAkcJgjDD8xuSMMchpIT7IzD84dMRg13GyHpzK2r1zr0t63X9QP+uBlDgSIdDWEOC0K1+6JD60v3MhkEDE/C9UZq6h8xVHyuxgOCV77qQJyKKzz+bANpAOVRQ3Jbxjj9MiDBs52GnkboDmzqVELK9N4UQiyM6EKaIoJF7b0AlQ+FLfimlarmPboxbSHyPM7YR/Z+Cl37lLR1sBTeUbr75rn//K3U5jwQ7hA3j1U6hFZJ7vK66+NG493U74R60jr43aPspgTCUz1ATjg1a0cDrNFah/QqCFCLCfCcOMgcK25BcGQp714lsuKHIl0qTNwSDrsw/c7vrn5SA97yDJAQdSDLRMzfzX/+5B1yf5y3jjlXcxbbqH6nlHAJnEoFefHm6XzchmxQgb3Ma4u1Mvu9AmX3yea6NMn8pMWnw/lJeXu2Nos//2a++5UGud0GdS+0++cMe9N7m6v46MHWtXDXTmIVE02ClTJ7lgzomuwV9PLpPAMgA07XrZ/hkBZOL5ZzXYjUGoGS+YswwkmDuRR3jYyCEN9mkLPzKG+HmqVAZRPh3E78nHnnVaCcb3o5CE8aEO4UanW6gyZmNxgoeencqZE8fWaxjWrt5gNDZ3GUjqto8eN9IYQua1l9526nh2PpxCopr+63/3BcUuSvdNUnktRoAaAQpHxn6vWb5gvSkRz+bP+yYZW/jJEnsdUyczrp7mpn9/819/cOV43vleRxXGKL8KcTo96Y0pqlhhh0ThNDLfKxxAUThNHCu08bvhtpludQTTVB/PXQCt4SGnoeNK71h+cxo3UT29cpmP2y/xiKZ/u5aFwKlG4Hf//Xj8U6JPeuf191wWjfu+fJcdQV7cX/z7b52W7yJ459NMgVOddNhi22Z/Ra0dB3l+YcBlKjDYT33pG/e5TZ52kO33a3/7BdeGf/WTR5CqbYMjfsuWrHCkj2YcDPpMclrcsYM79moEZM5DYgT2f8zesWTRMqcc8Zwst2zeZtchFRzDsTHMWmPX4K8nB5x7oHxiWtXeSBtX3LHYEcFY4kdTlZ3bdzlCyXpwcNcWJWOIXyx4ZNwkX7yhfCBo3zNkGEb0GEFTqPLlTaAmoQgjaD4InHbhjaB6mdM23IfrqE2jUPW7BVk9dkILwZs2dMQgJI3u6h7c9VA5sxy+6MeMH+m0a4NxPr/BOY3Qt+AhzsvLc6N3RgqnMM3bERiaD4D2jY2AIV9oxxTPq5haDZI+Pph8sJnObf++MteAWBbr/jKMXhkihqMr2lvshKEp68v6L1mwzB37mftucfvu3b1fpI/ASdoUAhyUeGTvs5+/zdWNhO2P//NX43SvR/yYhJ15rOfPW+T26QeyRGJI6Qyi5rQFWCYR69uvt3sf0MC6uLjI3kbHxE7hjntvdvvH+zcMJG3unHluqpXaBo7SKbGhnLhu04atbqTPARffP2xzTOXI1E9M4chzUWvH+jG94w6M9imx9XQr9U8IZAACl8+c6p5zVpXtdTPaAKUMZhLMskGt23/CEcMTzsxROEW7dNHyupzzFW4dyWEqwr6ZA6jaD7Tg6PsoXpv3NGke6eO2xej/1q3ZYLvQ9qpoIwiCynbJjB+UgeiDqSSh0LGqsWtwO9X98945vAZqEgsK8927ghzCr4jiYJachLMKU5EUgpyhLUrGEj+Czxcwp2aCCLFQDdLD0QRHELTd+f2v/+imQHlTaMdDAjfjmmnOFuDxR550pI9ToLPf+sAmXXgO8v1dZG/B24jqa8boOoQbyumkc6DKpWEmhQSTXoRDhg+0lctXO5sB5gMkkWNn8+rzbzriSaNVjnTuRIfDh5f7Lp6/zD3AfHhZVwZ7/uq3HnBqbP+DsWblOkdSb77zOrc/t3XvUZvLl8usAz0hL55+gSN4XNe3f29+ORLKcBMkp2wcXN+huHYk5HbQPyHQRhBgO+JLlCNokjlPGESdL8/9sKGjcHD3JJK3h+HFe+El5zuzhTxoE9imn0JsPrZhasqYsJ0DKY7mX33hTXvsd09iSraTex94Zcf7ZhuhIwWJ22MPr3dl+NM8+Y/hIO/5p15xHWEp2uT0Ky9xU1jnY+rnyOEjbiqa7ZrTQbyOsxPU01+mloVAW0aAJMkjL5y29IifU1pAKc7+zd9eSIgYu/alZ193bZeDLporPffkyylfJgdWnlBx44m3nnZ+fD94Qq9jegxzIHb1V2c4BQrt/fziHct1jV2D/xguc6BH8Qig+4F/NBG5AO8lTzjr5hFLb11b/M5Y4kcw2XEQ6BFnDLXf/fJxI2miLQANREkKL4ONz8Ah/e0/fvQLsPJOTrM374MFzk7ols9cZxzhk6x98uEiu/jSC3A8X/yFRnUxyRZtGui5xwf7leffQIyxqSepqlkPai/emjXHMf97H6zNI/jbnz8Km4cP6j1/OFVFLd9l6CxYFju3bVt31JM3lsOOhYSNqeM4yuEIy3vQ2IlRA3gAoxRKPLs9TluxjrRZePS3f3bn42/aAEqEQFtCgFp2fw5Or27X3HSF8eNpB/gSnXrZFNc2Qsi9SeFgjgbUfPHHxtxkm+Yndpv/XOdOPtv48eSsieOMHw6oCuGxG2eWN25dveOpHZwBe77Lr5oGDQLqhIGgJ4nq6a8POw5/5+FNeXll6FsItDUEqOzglCdnm0gIaZdHbR/tdDkIooKjtGc3q4R97YKPG5Kv2GsheaNNOvs6Vy7IZGPCwRoVNFTI0ASKWsfR0LB72kb24ZyajXUIiS2zsWvw78tZAGr2OJjju4lCbvDTf/s16ry6Qdv1E1R/GW1tOaO9ekl0RoJM0T6gZ+9SN9KoqKio15DR+JLkiRo2dgaU3ZiOpbwAmwOqqLmdpKxs/wGnfeADS7uGvz7xvOs83M5N/KPKmFNSnJ7l6IgdUxdMAXlTPd7hrCs7CW/en0bkfuH1UE1M8kphvVgutSPbMAVN8ToVqqjjCTuwz3/5bne9JMJPPPJUfXnx9tc6IdDWEWCb8Uifv66xpC/Zbf79/MvsCOKRPv8+jS1T2+61T/9+jdXTv5+WhUAmIXDrXTdY/wF9YVIxB9ryvxgDsdN7tlu3EmeqQROnPz/2jFO4kIwlEs9OjmGVOFXblJCAnXfBBChJdrtQTPTqpYkXFSvUxDPO3svPvY7B3YSmirJE1+A/cMWntaYlHofgNs7oDYJSid79mRh3M6M1ftTGeTZ9fqZNWzrepDWYDiKh4zQuDU0p3gNITRinZDwhWTt3cnd44MDmZ/bHzj6BMXvuuu9Wb5eE357hKDMEeELC5sU/8tZ1gN0RxV9Xb5v33QMEljZNHL0w7AQ/P/vxr73N9Zo+hraINZT1dqJ9ATWaDIBJzeJePJy9MH0tEQKZggAJ1Exo3kt7yCM9U+6Z6tl+EGD/EStXXHOp8eMJ+zPGz6xVUFQ7bZ03cLrx9quh/b7cQjDDYB9NMw1PqB3nxxP2Y7SJrapEGXXZtPwace7HoOl+oZnF9CsucR69fFd4feoDX7vHmXNxsMWBGE1JPIktk+sbuwbvOGYTipdR6Na7b/B2cT4GJKOJ5Fvf/2qiTadlfUYTv0SIMcQJbYB4I/r2hxYOow0+BBQaX3NEsAIq2vMvnOimkPYhrATVx4z5NWBQX5CtMbYa2jIvzk9xnR0BA02SzPUbUGtT552fqmoXimL1emf3R3sjavOofUtVzkU8M0b/fwoaR5JVPpjUWHpCT0BqDEno+LBz+RDOReNVdpJvIsQEnVw4VcxwMNSWlJR08Q7XtxDICARyMa3LQY9ECAiBto0AbWwLi2qdKv01zffZ6PnXx1tOVEa8fevXoUv3FDn167DgkUf/uqaWm3X+pgptw9vbJfEj2+8KslMbEqU2LAq1g1/+5v1GLz6OWubO+dj+/Ogz7taMGjPcfa9G0NS5cBqh0KZu2uUXueX+mMIlweLUKT2GPvfgnW69/x9tDV96Zpa9/srbzk185Ohh0DRO8e+S1DI1ldfceIV9As9eZhmgowg1it70MEdPdPxg2BaSW3pOUe18+VVT3QNPL2YvcwG1mDcjGrrfoDWpSmgnIdDGEdgDjThj5VE4AKLzVo9etY4eXEdnDcawZOy+VIW2to/86gkXf4uxwzjIYogJ5eVNFUntLwSEQFtEIFBWvgMOz+1LHodHHx0u6OFHzQHDMNAbj7/p8eMJjbm53T8lS6Nw6ga9dEzevvzm/nmI2+dNL/u3ecv0aGKoFY4gWio0IGUnFK8urmzcOQawpW2Sp9Hk+tr4fgFLZcTV0rrqeCFwKhFgho733/mofoqH7Z2DvdvvudF5+j38i8fc9NElIG6pCvPu/viff1YfJYDvjsNwJPPCzqRanvYXAkJACLQlBNqlxo9kjtO39O4NwGGCeQCp9eJ0rl+cMbd/BZYbM8SOt3/M4U47F7uuub+pyWtUwFA5pRsrCYli7I76LQQyHAHGsmT8vs2btrp8ubPfnGs3QPvOQR6nfEjiGF+T2TaOIJ4XzTeoPfcCN/Py6QFIL2KGcYrXni6cej7smGqzF9CQmxp4xuqkoxi1jAzm6gnLoZch4/uRiEqEgBAQAm0NgXZJ/K5A2BN6+tA9nKlZGAR5Bgw0OfUpEQJCoJ0hgAHQQKRIYqozxv+jlvwPv/mT09gxiCqXnVCVDy05s3TceQ+COuM3PQldyCQsM93STQgBxTRPfmHIJ9oNM4sAYwdSq48pBRf8lSSQBuU0DWECe6agikRrSeLF0y4wkkaJEGivCNAJMTaVKkOfMBIF7evpZcvYekwscA9CnTHQ+naEMYuV7nC09MfzjN3O38zwwdAxVM7QVp9tLp7QDIQOknQ6YbSPgYP6N5gRi3dMtq1rl8SPBI8vfIkQEALtGwHG6qKWjZ0JI/R7GT/iXfV3fvCQs39ldpyNGzY77R9JH+N0TkDqpyeQ/5PxOJl/tzGhKcU3/v7LLjsBiSMzBNFOmMcym8D1t17lUkExmDPjjPkzCzRWrrYJgUxDIF4qVZIzDpYug905iR9DvNCEivbqjEZBB8RYOWfSWY0SP4ZXY/ICZ4eFwRvJ393Q9rN8vzDQ8gtPv+pMvTjIo9z22RudrS+dMzn7dxPs3rNd2iXxy/abqusXAtmCAAOyUzj6n3DemXYpQjfFE3r/0TaXxJDEj9O7R+EIRWH0/a3wpGcWIHrjhyMnwjLFK4vraPbhdTrs1Bh4nR0cHcCe+8vLblqZmkeeR8QvEYpan20IMIfv6HEj6i+b7ZefkaOH25KFy5zNrhd67fix47Zy2VrE5S2xMxDInckY6Om/Ek6YzMPLdhtrw0utO232v/yN+53TI7WEzLHN+LwfvvcxAjEfdpE3GAWDWkNHRqGdZCBqmmZwX89UiiSSyQ/4WQYHrxFw2OTsITWaNCNjtA3OHsRLplB/gW10IWuJHz31qApmMnbeaEYZ98f1a879opctY+dRg8CRvkQICIHWReBu5Pnthyklv3MT7fpihW2d4n3T+cqz8+PLm7MEQ+HxT+E7oTGJNyXllcUUVl45hndA17ocoY2Vp21CINMRILHysu1wOZHQ/p4fChMVLFuyysXAY+xdZuMg2eqHoNAkZp8uXummiW+47eoGTpleGtJ4oVw4uKtGjt4N6zbZuDNH23Bo4Ck0wXA5ftG0SQAZPYPZgxirl0kcaE9PZ0rGwGXsXtoHMy0s4xAeRiYSDgr7wEeguqrG5RPnNfD877zxnjMP8Qd3dids4/+yljbezP8AAAdJSURBVPgx3Uts9ouzMd3DtGjNFRqAc/TvD+ScbFlSQyeLlPYTAicQIOHyk74TWxouMaA67X7mz1vkNpC88UU/d848249conxxMzUTxSNx7keS/3rDYYRZeyorqhzZKz9e4bQCGgAmCaB2y2gEmCbU+ElBSLiOHT0Gs6xaz3tq80j8Vi2rTYO2Et/UrI9A2/SEffY7r73n4tvGS2DA9I5/QZg2KmAYEm0GQrcxo9a1N1/pbASpDfRSInJKmnVg6DU6gzFlKz34Fy/41CZfdK47JR3DJl98HmLyjrWOcOJ65i8vOcL6FYSGK0B6x9/+4lEXNk7Ez7tDGfDdDTlx70FMPua/ZUw/GqAyowcJ3KeLVkAD2NWpcnkpfBgYHuWMsSPdlZHgbcSogt7DfCCoqo4VThutXbXBOiNPMOMHcuqHKmJ6AzIf79ARQ9zoJ5EaOrY8/RYCQqB5CDDI+pOPP4tp3IjLIuBNz1x13WUu5t9jDzNPd0GDhPOpnInvhpvvvNZNQf3pf5524Zw4kGxKe5jKObSvEGirCNDMgtpuCgkVp1wbFdjfcYqXg6Ux42pDrHEql9OqK5BqdTTCrrEcpnPzwqdxapXOVdTA3Q67vXge+KzDgw/dC63eJ25Kl23xc1+802nyYuvDlG8U9sMkhIOh+acw8YEnTBV5yfQL6sNGUWvIfvxXP/2924WDR05JZ5pkrcaPNwqOec4IlDHA+INZMjgFxCDIr7/8tvMK5DQQ5W2MMrrAjojEj2rfP8IQnGEbqO4tLy93SardjnX/uA8fOiZtZ8gJnoOjkOXI+8fk1iSMTGxN78K4amjYGkiEgBCIj8BF0yYbP/GE2Wq89Eze9NNo2AhRG8CXtj/vL7Pr8MMYnYUYwXuzvN7xLJ/t1xN69nrSo1dp/Xm4jpoFdjp0/qCHMOshEQLZgACnapl2jcJnvynitxaJEPbvK7MpUyfVEzsGYqd9H52iapMvmI1HFi0KwzA99cRzLpXpHffcdFJoNrdT3T+SSaZ047Tty8++bhvXb45L/LzEBmHY5lK8GUDW3xMue+nguK4Ig8Mq9Om33nV9/cyAf7t3XFv/zmrixzy+P/3XX7l7RMZPdW8ysmwJ8t+C9HFamKN6ehPSgJuGo5RwuMb+8tizdgwjgbvuv9UZjXJ0QQ9CppHjQ8llegRyeimeGjqZemgfISAEkkeARCwRGUsmRmeyZ/KMw5PdX/sJgWxDYB4crKhkmXDe+AaXPu7s0S5j1XJMwXJg5XLMQy/z1B+ft81IxDDijGGOyJHMMRUrNY1+efiXjzkPXtrWrsXULaV7Xb5vav1379zr+l62d4aAYjgmkkza+K1avtbtX2+j6341/EeHEGoqV0IrSaLLvh9ao/rMWg33bru/spr48UGYfuXFTsNHQ84XkXLtS/AGiidOK1i3wRmJYpmhGyixXnsfvb/A2S4wF3BPPLwU2gpQFsKAdcnC5fAmqjVAZ8yhAYP7uW36JwSEQHoRYAL3mTdcDg/c7uktWKUJASHQLAR2bNtlWzdvdzNqnqOGVxCdrBh3j964U6D9o9Ahi2FgKPSa54dCZ8xY4ldQmO+IGftrzsZdfOkFNgROHBROG9MbmAoXmmbddMc1dgPids564U03G0eHjQsuOb+BI4k70PePYeIYx5O2gQvmLXbaSs4kZJpkNfGjqtez2du2ZSdY/GoEgC1zdga8kZ7ql4bh/HjiqYhp50fyGCt8gJglZP5Hi135tDvg6IRCA1Z/2rhEQShjy9RvISAEUkeAbZFtTiIEhED6EeAMFj9+oSbMm/bl+hlXX+o+3j7MnOM3pfDWe9+fue8Wb9F908avsf39O999/21uWpievbFev3QGYb2qKrkNZh0Q5vL+6rcfcOFeqKn3T9t+AYHZY4WOX5zpu/Ka6S5dKolmc5zBYss91b+z2giFnnc0AuUoYO3q9c72h0bf1OBRBbwFsb3o1PEqRgR+8VK/0TOYU7bvvvlBfcJ47sdk7jdiJEH18XNPvuxsfmj/w05ow9pNLlYYY35tRvwgLy0bCSSdQVge95EIASEgBISAEBACqSFAG95Y0ueVwOllj/R56/jNmQE/6fNvi7uMsDB0LslE0sfryfnuP377n+JeWDtfScJHb1qmljlYdtDl6bzu5qusC3Js8gGglx5jAa1BtO9ikDIGd6QHD236OH1LDSAj9tP9nFpCBqakQ8cqBJekjQCdQpivk6njDqB8Gq0OgE3B+jWbbOH8Jc5rmESP5YUwoqGNIT2AeTzDwfhHTO38VujyhIAQEAJCQAgIgVOEQKCsfAdMJyXxEKAHIAkeRwNxBcjRozfV0QJtBGiDQHLpF9oy+NXQ/m1aFgJCQAgIASEgBIRASxEQ8WspgjpeCAgBISAEhIAQEAIZgkBW2/hlyD1SNYWAEBACQkAICAEhkBYERPzSAqMKEQJCQAgIASEgBIRA20dAxK/t3yPVUAgIASEgBISAEBACaUHg/wN+3YH5lsrX4wAAAABJRU5ErkJggg==)" + ], + "metadata": { + "id": "RL1LDp645ogr" + } + }, + { + "cell_type": "markdown", + "source": [ + "# **Post Hl7v2 messages to Google Cloud Healthcare API HL7v2 store pipeline**\n", + "\n", + "This example demonstrates how to set up an Apache Beam pipeline that reads a HL7 file from [Google Cloud Storage](https://https://cloud.google.com/storage), and calls the [Google Cloud Healthcare API Hl7v2 store to store Hl7 messages](https://cloud.google.com/healthcare-api/docs/how-tos/hl7v2-messages) to extract information from unstructured data. This application can be used in contexts such as reading raw Hl7 messages, if needed parse them or modify them as per your defined Hl7v2 store configurations and store data into Hl7v2 store.\n", + "\n", + "An Apache Beam pipeline is a pipeline that reads input data, transforms that data, and writes output data. It consists of PTransforms and PCollections. A PCollection represents a distributed data set that your Beam pipeline operates on. A PTransform represents a data processing operation, or a step, in your pipeline. It takes one or more PCollections as input, performs a processing function that you provide on the elements of that PCollection, and produces zero or more output PCollection objects.\n", + "\n", + "For details about Apache Beam pipelines, including PTransforms and PCollections, visit the [Beam Programming Guide](https://beam.apache.org/documentation/programming-guide/).\n", + "\n", + "You'll be able to use this notebook to explore the data in each PCollection." + ], + "metadata": { + "id": "wC9KRrlORwKu" + } + }, + { + "cell_type": "markdown", + "source": [ + "**What is an HL7v2 message?**\n", + "\n", + "HL7 Messages are used to transfer electronic data between disparate healthcare systems, each sending information about a particular event such as a patient admission.\n", + "\n", + "An HL7 message consists of one or more segments. Each segment is displayed on a different line of text. A carriage return character (\\r, which is 0D in hexadecimal) separates one segment from another.\n", + "\n", + "Each segment consists of one or more composites, also known as fields. A pipe (|) character is used to separate one composite from another. If a composite contains other composites, these sub-composites (or sub-fields) are normally separated by caret (^) characters.\n", + "\n" + ], + "metadata": { + "id": "AOVYgtyaqSxa" + } + }, + { + "cell_type": "markdown", + "source": [ + "***Sample HL7v2 Message***\n", + "\n", + "The below reference message shows a sample Hl7v2 messages seperated by \\r.\n", + "\n", + "**MSH|^~\\&|FROM_APP|FROM_FACILITY|TO_APP|TO_FACILITY|20150503223000||ADT^A01|20150503223000|P|2.5|\\r\n", + "EVN|A01|20110613083617|\\r\n", + "PID|1||21004053^^^^MRN||SULLY^BRIAN||19611209|M|||123 MAIN ST^^MOUNTAIN SPRINGS^CO^80439|\\r\n", + "PV1||I|H73 RM1^1^^HIGHWAY 73 CLINIC||||5148^MARY QUINN|||||||||Y||||||||||||||||||||||||||||20150503223000|**\n", + "\n", + "The file contains many such messages and the objective of this code will be to split and construct messages and POST it to Google Cloud HealthCare API HL7v2 store." + ], + "metadata": { + "id": "-lpbvwHmX1L5" + } + }, + { + "cell_type": "markdown", + "source": [ + "Lets install necessary packages" + ], + "metadata": { + "id": "81wCK9XnS6Sc" + } + }, + { + "cell_type": "code", + "source": [ + "!pip install apache-beam[gcp]" + ], + "metadata": { + "id": "Yv1phmRZS23c" + }, + "execution_count": null, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "**Google Cloud Authentication**\n", + "\n", + "As we are using Google Clous Storage and HealthCare API, we will be requiring tokens to make sure our connection is secure.\n", + "\n", + "Click [this](https://cloud.google.com/free) link to create a new Google Cloud Platform account\n" + ], + "metadata": { + "id": "3EcdPBczYQlB" + } + }, + { + "cell_type": "markdown", + "source": [ + "**GCP Setup**\n", + "1. Authenticate your notebook by `gcloud auth application-default login` in the Colab terminal.\n", + "\n", + "2. Run `gcloud config set project <YOUR-PROJECT>`\n", + "\n", + "Set the variables in the next cell based upon your project and preferences.\n", + "\n", + "Note that below, **us-central1** is hardcoded as the location. This is because of the limited number of [locations](https://cloud.google.com/healthcare-api/docs/how-tos/hl7v2-messages) the API currently supports." + ], + "metadata": { + "id": "tpePe_yOsdSJ" + } + }, + { + "cell_type": "markdown", + "source": [ + "Before running please set the following variables as arguments as mentioned below\n" + ], + "metadata": { + "id": "_1Q3mw1usnoE" + } + }, + { + "cell_type": "code", + "source": [ + "args = {'gcp_project':'xxx', #GCP project ID\n", + " 'gcp_region':'xxx', # GCP project region\n", + " 'temp_location':'gs://<YOUR Bucket>/tmp', #input location where your HL7 messages are stored in GCS bucket\n", + " 'input_file':'gs://<YOUR Bucket>/my_message.hl7', #input location where your HL7 messages are stored in GCS bucket\n", + " 'hcapi_project_id':'xxxxxx', #healthcare API project ID\n", + " 'hcapi_dataset':'xxxx', #healthcare dataset\n", + " 'hcapi_version':'v1', #healthcare API version by defualt v1\n", + " 'hcapi_location':'xxxx', #healthcare API configured location\n", + " 'hcapi_hl7_store':'xxx', #healthcare api hl7 store\n", + " 'hcapi_fhir_store':''}" + ], + "metadata": { + "id": "a722GbqdvgOX" + }, + "execution_count": null, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "**Google Cloud Healthcare (HCAPI) API Utils class**\n", + "\n", + "Below is the code snippet which describes the class having healthcare API connections and configurations. Basic functionality includes constructing the hcapi_url as per the input parameters, cleaning the HL7 message in a proper format and posting hl7v2 message to hl7v2 store. You can add more transformations as per your requirements." + ], + "metadata": { + "id": "NHzk8JIqxQoa" + } + }, + { + "cell_type": "code", + "source": [ + "import google.auth\n", + "import google.auth.transport.requests\n", + "import base64\n", + "import json\n", + "import hashlib\n", + "import requests\n", + "import logging\n", + "import apache_beam as beam\n", + "from apache_beam.options.pipeline_options import PipelineOptions\n", + "from apache_beam.options.pipeline_options import SetupOptions\n", + "from apache_beam.testing.test_pipeline import TestPipeline\n", + "import apache_beam.runners.interactive.interactive_beam as ib\n", + "from apache_beam import io\n", + "\n", + "logging.basicConfig(level=logging.INFO, format='%(asctime)s :: %(levelname)s :: %(message)s')\n", + "\n", + "class hcapi_cls:\n", + "\n", + " def __init__(self, args):\n", + " self.hcapi_hl7_store = str(args['hcapi_hl7_store'])\n", + " self.hcapi_project_id = str(args['hcapi_project_id'])\n", + " self.hcapi_version = str(args['hcapi_version'])\n", + " self.hcapi_location = str(args['hcapi_location'])\n", + " self.hcapi_dataset = str(args['hcapi_dataset'])\n", + " self.hcapi_fhir_store = str(args['hcapi_fhir_store'])\n", + " self.token = None\n", + "\n", + " def google_api_headers(self):\n", + " \"\"\" Function gets the token for the request \"\"\"\n", + " logging.info(\"fetching token and refreshing credentials\")\n", + " creds, project = google.auth.default()\n", + " auth_req = google.auth.transport.requests.Request()\n", + " creds.refresh(auth_req)\n", + " return {\n", + " \"Authorization\": f\"Bearer {creds.token}\",\n", + " \"Prefer\": \"handling=strict\"\n", + " }\n", + "\n", + " def hcapi_dataset_url(self, version=None, project=None, location=None, dataset=None):\n", + " \"\"\" This function creates base hcapi dataset url and returns it \"\"\"\n", + " base = 'https://healthcare.googleapis.com'\n", + " version = self.hcapi_version\n", + " project = self.hcapi_project_id\n", + " location = self.hcapi_location\n", + " dataset = self.hcapi_dataset\n", + " return f'{base}/{version}/projects/{project}/locations/{location}/datasets/{dataset}'\n", + "\n", + " def hcapi_get(self, url):\n", + " \"\"\" Function to send get request to HCAPI \"\"\"\n", + " response = requests.get(url, headers=self.google_api_headers())\n", + " if not response.ok:\n", + " raise Exception(f'Error with HC API get:\\n{response.text}')\n", + " return response.json()\n", + "\n", + " def hcapi_post(self, url, data):\n", + " \"\"\" Function to send post request to HCAPI \"\"\"\n", + " response = requests.post(url, headers=self.google_api_headers(), json=data)\n", + " if not response.ok:\n", + " raise Exception(f'Error with HC API post:\\n{response.text}')\n", + " return response.json()\n", + "\n", + " def hcapi_delete(self, url):\n", + " \"\"\" Function to send delete request to HCAPI \"\"\"\n", + " response = requests.delete(url, headers=self.google_api_headers())\n", + " if not response.ok:\n", + " raise Exception(f'Error with HC API get:\\n{response.text}')\n", + " return response.json()\n", + "\n", + " def hcapi_hl7_url(self, version=None, project=None, location=None, dataset=None, store=None):\n", + " \"\"\" This function creates hcapi hl7V2store url and returns the url \"\"\"\n", + " base_url = self.hcapi_dataset_url(version=version, project=project,\n", + " location=location, dataset=dataset)\n", + " hl7_store = self.hcapi_hl7_store\n", + " return f'{base_url}/hl7V2Stores/{hl7_store}'\n", + "\n", + " def get_hl7_message(self, message_id):\n", + " \"\"\" Function to get message from HL7v2 store using HCAPI URL \"\"\"\n", + " url = f'{self.hcapi_hl7_url()}/messages/{message_id}'\n", + " return self.hcapi_get(url)\n", + "\n", + " def post_hl7_message(self, payload):\n", + " \"\"\" Function to post messages to HL7v2 store \"\"\"\n", + " url = f'{self.hcapi_hl7_url()}/messages'\n", + " return self.hcapi_post(url, payload)\n", + "\n", + " def message_to_hl7_store(self, message):\n", + " \"\"\" Function to clean up Hl7 messages with \\r seperator before posting to HCAPI \"\"\"\n", + " messase =str(message)\n", + " message = message.replace('\\n', '\\r')\n", + " message = message.replace('\\\\r', '\\r')\n", + " message = message.replace('\\r\\r', '\\r')\n", + " encoded = base64.b64encode(str(message).encode())\n", + " payload = {\n", + " \"message\": {\n", + " \"data\": encoded.decode()\n", + " }\n", + " }\n", + " return self.post_hl7_message(payload)\n", + "\n", + " def hcapi_fhir_url(self, version=None, project=None, location=None, dataset=None, store=None):\n", + " \"\"\" This function creates hcapi fhir store url and returns it \"\"\"\n", + " base_url = self.hcapi_dataset_url(version=version, project=project,\n", + " location=location, dataset=dataset)\n", + " if store is None:\n", + " raise Exception('No FHIR store specified')\n", + " return f'{base_url}/fhirStores/{store}/fhir'\n", + "\n", + " def hcapi_fhir_request(self, store_key, query, data={}, method='GET'):\n", + " \"\"\" Function to send post request to HCAPI FHIR store \"\"\"\n", + " store = self.hcapi_fhir_store\n", + " if not store:\n", + " raise Exception(f\"Couldn't FHIR find store named {store_key} in config\")\n", + " url = self.hcapi_fhir_url(store=store)\n", + " url = f'{url}/{query}' if query else url\n", + " get = lambda q, d: self.hcapi_get(url)\n", + " post = lambda q, d: self.hcapi_post(url, data)\n", + " delete = lambda q, d: self.hcapi_delete(url)\n", + " return {'GET': get, 'POST': post, 'DELETE' : delete}[method](query, data)\n", + "\n" + ], + "metadata": { + "id": "H7g4_-rGS9P_" + }, + "execution_count": null, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "**Pipeline Setup**\n", + "\n", + "We will use InteractiveRunner in this notebook.\n", + "Following are the DoFn classes which carry out their respective operations" + ], + "metadata": { + "id": "lXnzAtbHyUd2" + } + }, + { + "cell_type": "markdown", + "source": [ + "The following class **BuildFileName** takes the file name from the element and converts its into string. You can enhance this class to construct GCS bucket URL, if your GCS bucket prefix remains constant." + ], + "metadata": { + "id": "TKnL8kxh3Kms" + } + }, + { + "cell_type": "code", + "source": [ + "class BuildFileName(beam.DoFn):\n", + " \"\"\" Class to get file name from variable and returns the filename \"\"\"\n", + " def process(self, element):\n", + " logging.info(\"processing the following file: {}\".format(element))\n", + " file_path = str(element)\n", + " yield file_path" + ], + "metadata": { + "id": "N01E3dQd3Jr3" + }, + "execution_count": null, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "The following class **BuildMessages** takes the GCS URL from the above class reads it, separates out each message, appends them into a list and return the list for the next class." + ], + "metadata": { + "id": "Jej68R8w3i2Z" + } + }, + { + "cell_type": "code", + "source": [ + "class BuildMessages(beam.DoFn):\n", + " \"\"\" Class to read file, clean and seperate messgaes based on MSH\"\"\"\n", + " def process(self, file_name):\n", + " try:\n", + " logging.info(\"starting to read file: {}\".format(file_name))\n", + " file = io.gcsio.GcsIO().open(filename=file_name, mode='r')\n", + " read_file = file.read()\n", + " new_file = str(read_file, encoding='utf-8').replace('\\n', '\\r')\n", + " logging.info(\"starting to seperate HL7 messages into list\")\n", + " messages=[]\n", + " for line in new_file.split('\\r'):\n", + " if line[:3] =='MSH':\n", + " messages.append(line)\n", + " else:\n", + " messages[-1]+= line\n", + "\n", + "\n", + " logging.info(\"total number of messages parsed are {}\".format(len(messages)))\n", + " return messages\n", + " except Exception as error:\n", + " logging.error(\"got the following error while processing : {}\".format('\\n'+str(error)))\n", + " raise Exception\n", + "\n", + "\n" + ], + "metadata": { + "id": "MC6tr_sGyNKG" + }, + "execution_count": null, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "The following class **PostToHL7V2Store** takes the messages return in the earlier class and POST each messages to Hl7v2 store ." + ], + "metadata": { + "id": "1hpuoUGA33jo" + } + }, + { + "cell_type": "code", + "source": [ + "class PostToHL7V2Store(beam.DoFn):\n", + " \"\"\" Class to read file, clean and seperate messgaes based on MSH\"\"\"\n", + " def process(self, element):\n", + " try:\n", + " logging.info(\"starting to prepare and post message\")\n", + " hl7v2_store_response = hcapi.message_to_hl7_store(element)\n", + " message_id = hl7v2_store_response['name'].split(\"/\")[-1]\n", + " logging.info(\"successfully posted message to Hl7v2 store with message id :- {}\".format(message_id))\n", + "\n", + " yield message_id\n", + " except Exception as error:\n", + " logging.error(\"got the following error while processing : {}\".format(error))\n", + " raise Exception" + ], + "metadata": { + "id": "lVjqYfb2330k" + }, + "execution_count": null, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "The following function sets up a beam pipeline with various other Pipeline options that will extracts messages from Hl7 text and post each hl7 message to hl7v2 store using Google Cloud Healthcare API (HCAPI) api methods.\n", + "\n", + "**\"|\"** is an overloaded operator that applies a PTransform to a PCollection to produce a new PCollection. Together with |, >> allows you to optionally name a PTransform.\n", + "\n", + "Usage:[PCollection] | [PTransform], **or** [PCollection] | [name] >> [PTransform]" + ], + "metadata": { + "id": "g5oJgXCk4O1a" + } + }, + { + "cell_type": "code", + "source": [ + "\n", + "import apache_beam.runners.interactive.interactive_beam as ib\n", + "def run(beam_args,argv=None,save_main_session=True):\n", + " runnertype = \"InteractiveRunner\"\n", + " project=beam_args['gcp_project']\n", + " region=beam_args['gcp_region']\n", + " temp_location=beam_args['temp_location']\n", + "\n", + " options = PipelineOptions(\n", + " flags=argv,\n", + " runner=runnertype,\n", + " project=project,\n", + " job_name=\"my-beam-hl7to-hcapi\",\n", + " temp_location=temp_location,\n", + " region=region)\n", + " beam_pipeline_options = PipelineOptions(beam_args)\n", + " beam_pipeline_options.view_as(SetupOptions).save_main_session = save_main_session\n", + " with beam.Pipeline(options=beam_pipeline_options) as pipeline:\n", + " file = (\n", + " pipeline\n", + " | 'reading filename' >> beam.Create([args_dict['input_file']])\n", + " | 'preparing file path' >> beam.ParDo(BuildFileName())\n", + " )\n", + " hl7_messages=(\n", + " file\n", + " | 'parsing hl7 messages' >> beam.ParDo(BuildMessages())\n", + " )\n", + " post_hl7_messages = (\n", + " hl7_messages\n", + " | \"posting to hl7v2 Store\" >> beam.ParDo(PostToHL7V2Store())\n", + " )\n", + "\n", + "\n", + " ib.show_graph(pipeline)\n", + "\n", + "\n", + "if __name__ == \"__main__\":\n", + " logging.getLogger().setLevel(logging.INFO)\n", + " args_dict = dict(args)\n", + " hcapi= hcapi_cls(args_dict)\n", + " run(beam_args=args_dict)" + ], + "metadata": { + "id": "Dynn2PDuyRBT" + }, + "execution_count": null, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "![Screenshot 2023-09-29 at 4.42.51 PM.png](data:image/png;base64,iVBORw0KGgoAAAANSUhEUgAAAM0AAALXCAYAAAAubsS9AAABYGlDQ1BJQ0MgUHJvZmlsZQAAKJFtkD9Lw1AUxU9tpdgWrCAuKnRToZaadHGsVVToEKriny1NYqok6SOJiJtujg7SzVUEXbWDDn4EQVBw0kH9AGIctMT7WrWteh+X++Nwz3v3XaAjJjNmhACYlmsXpicSS8srifAzujCIbkQRkRWHZSUpTy34ru3h3SDA6/Uov0uquHtC/9nJ285T6nU3ePy3vy0iquYoVD8oBYXZLhBIE0ubLuO8Tdxr01DE+5z1Bh9xLjb4ot4zX8gRXxHHlZKsEt8TJ4stut7CprGhfM3Ap49p1sIc1T7KAUxiCnk6CUgQkMEYxjFDO/rfk6l7ciiDYQs21qCjBJfcWVIYDGjEs7CgIIUksYA0pch3/XuHTU19BESTnhpuautx4NwDek6b2tADfecAuJSYbMs/mw14IWdVFBocrQKdFd9/WQTCI0Dt1vffq75fOwSCd+T1PgGPE2R36Tkc1wAAAFZlWElmTU0AKgAAAAgAAYdpAAQAAAABAAAAGgAAAAAAA5KGAAcAAAASAAAARKACAAQAAAABAAAAzaADAAQAAAABAAAC1wAAAABBU0NJSQAAAFNjcmVlbnNob3Sit7HCAAAB1mlUWHRYTUw6Y29tLmFkb2JlLnhtcAAAAAAAPHg6eG1wbWV0YSB4bWxuczp4PSJhZG9iZTpuczptZXRhLyIgeDp4bXB0az0iWE1QIENvcmUgNi4wLjAiPgogICA8cmRmOlJERiB4bWxuczpyZGY9Imh0dHA6Ly93d3cudzMub3JnLzE5OTkvMDIvMjItcmRmLXN5bnRheC1ucyMiPgogICAgICA8cmRmOkRlc2NyaXB0aW9uIHJkZjphYm91dD0iIgogICAgICAgICAgICB4bWxuczpleGlmPSJodHRwOi8vbnMuYWRvYmUuY29tL2V4aWYvMS4wLyI+CiAgICAgICAgIDxleGlmOlBpeGVsWURpbWVuc2lvbj43Mjc8L2V4aWY6UGl4ZWxZRGltZW5zaW9uPgogICAgICAgICA8ZXhpZjpQaXhlbFhEaW1lbnNpb24+MjA1PC9leGlmOlBpeGVsWERpbWVuc2lvbj4KICAgICAgICAgPGV4aWY6VXNlckNvbW1lbnQ+U2NyZWVuc2hvdDwvZXhpZjpVc2VyQ29tbWVudD4KICAgICAgPC9yZGY6RGVzY3JpcHRpb24+CiAgIDwvcmRmOlJERj4KPC94OnhtcG1ldGE+Cn1R2R4AAEAASURBVHgB7F0HmBTFEq5NBypZAUFMYCYpORlREUQJdwdHMmcxIL5njmDWZ0IUjASVpGIWFVCRjCCgZFDJiCAZbndn+vU/c30bbvfuNs2mqu+7m5me7uruv/qfrq6ZnbEJKcTCCDAC5UbAXu6cnJERYAQMBJg0PBAYgQgRYNJECBhnZwSYNDwGGIEIEXBGmD8u2Sd9qsdFDythBPK6WX/dTwppJnyk08TJGlucEYgJAZuNSN9dISYd0RROCmnQ0PzuDho/KmnVR4MVl0khBCZO1qngKk9SWpTUUYsrBQsjkG4IWO8QphtC3F5GIAgBJk0QIHzICJSFAJOmLIT4PCMQhACTJggQPmQEykLAruvmPRP/5zaxj3T/LRSptLKU8nlGIJMRsNuKQlggiCKJpmmEdPwhDVsQRh0DEP+8iniZDBT3jRFQCDj9Zw+73U74g4AgII/D4TAIg3zYV2TBefyp/IpUSGNhBDIZAYMhIIMa7F6v1yAJthAQR8022CoSqTT/WQY6/I8zGTjuW/Yi4MTgV8TAPv6cTqcx+LEPEqgZCEQCMfCHcx5P4B1ZpCEvtoqE2Qst9zxTEbBj4IMMGOhq0CMNZMGxcsnUDKTy+a97AA7SlR4c84wDFFgyEYHix2gKCwuN/rlcLmPwY8bwJwZIgRkIW5AK+yAJSKXyKYKpWQbpSGNhBDIJAef27dsNIoAMGOwgArb483e1QAB1DMIowX7VqlUJZFN5VFnkUXpVft4yAumOgF3ocsEvJCFk0MtuE1Qhx0lOh41cThkjMNKliyZ3cYzzyIcycslvbDWvm/bu3Vu89gFxQDxsmTDpPjy4/aEQMG5uYv2haV5jsMP1woBX6xq1r4ig61rxeZxD2UOHDhlb5MFshNkHsw1Ig/Ppsr6RzaWfZur08BNeOrWZuxivTZsFNWzlppeGW/8boEnyEfirbvJS87Pd1LOfhyZ+4vsB35atgka+q1HXfA917xMYlCluPO/EHQHzpkwMapXLBmKAJCASyOMvIFC6yOYtRC+9ptGqNb43W/27i2jZCkGLl/rSrOjP409r9MKrGr3+opM+et9Fu/cQ9brSQ2gPZN9+op3/En05BRcuM43/Jx6BmEmDJqoZBeQAedQW5xSpsJ/qAm4X5Nmp+VmBsDQ6w0ZbVufQm6/61nKJ7svmLYIeedJLg29z0GGHEZ1wnI2mfOKi7z9zUfVqZu0nN7DRHTdzoCXRtgjWHzg6is7CNdu5c2dw3pDHykUDWTDD+EfQFJlCFkzhxKKHIgJaeHRtm3Q7A5ISerBgkTmrHVvPVw3q73heoMmsbJOvJdm9FzAM3G43jRj5Ji1fvtxAJScnhwYM6E/NmzULi5LNZpezi3lfB7OKIo6aYTDrxOqeLflN0KgPNOp2qZ1WrxX0kXwxxwP/cVD7NnbDVXn9LY0WLtapahUbXdHHTud28A2svzYImvyFTj/+rNO+fWTouPWGwKvznPk6vTNGN3S3bGajrdsC3bAdOwV9+qVOa/8Q9MTDJmRLfzfTKlUiuvgCO709WqPlKwVdM8BBed199QM4rEvwToR/dhC1am6jFnImwyxRozpRvWNKuq5vjdKMMij72kidvvrWXMfgrgD6c8t1Djq7XWAdyOsvcOHC4YJZ7POvdYmZoCcfcdCId3RjLXdOezvde5dDegemJriDn3+t0Q8zBK1YJahdaxs9er+TDpczH+TP9YLGTdKpaWMb1TzKRu+P12jPXqLrr7JTm5Z2mr8QttJow0ZBffId1PUSX5vhwb87VqMZs+QtDK+gTh3t0na+us0aUvT/urWrhfq7997/ihtvuF78smCeWLVyuRg96j1x4YUdxby5s4vzqLxqu2H9n2LlypVCzkxiz5494sCBA0Le8xFyBhJyxpKTjRByFjKOjQP5L3+Ax/hTx6Vth42QiFY+ZPw1aVsoTjmr0Ni/9xGPWL9BN47fHesV8xfqolN3t3Huw4maoXLXbiFq1S8UNw8y2zHxEy3gPDJN+FgTlescEt9N08TBg0I886KvPpyH3m4Fpt4LurqRJNas08V5Xcy0+o0LRbMOhWLgYI+hB21dvFQ38uHfq7L90L/2D12sXqsX52l5bqF4/S1vcT7/Hei/8x6P0dbRH3rF3AW6+GGGVty/r741+4cybtkk1Im+KykNF5jkyhtN3SiHdtxyl6cY1+Fv+tqEfgNvlFmxymz7TXeaWH4/XSsug/4DjweHeAy8oRd4tLnALW643VcXdECgr3NPt/jPgx6x8Fdd3POwmUfpVv0obTte2s1W5VBpWRJ2jtTgX7J4kUEQEEalYTtixOvi66++CEjzP//nH+vEqlWrxI4dO8Tu3buFjKTJwXfQ2MpZR+APBPKXSEiDck//zxzIva40B8bylbrYf0CIvte4DeMo3b9IA8Bg9U4rNJJ+nm2S5PlXzIGAMjh/+39Mw+/8VxiD+JpbzGMUkpwvHthKL9JQTpEG6X+uN+sCkaEX8s4Ys51vvG3Wh26DMP4D+v7HzAHyyeeBmJgafP9feNXUhT4pURcQEFxJKNKUhQvKAiP0Cf2ArFpj9kdhjAsIzl+a5yMjSAASKZky1cQX9SkTjx1vpg243lOchosYdOGiBRn5rjdAD0gEnJBn3Z++/qp6Qm2TSZpi9+yv9evlFFuTqleXPoMUuGpYr3Ro3964cRluooTrBZcMIjtn/Pk/BaDcNKx5sB+N1K1jlsrrZrpVp51io/0HiD6YqFOtmjZasMgMt0rDUf0TzLUHXBm4b0vn5FCDE22EsDFcMAhcHMjHn2m0V7ps55/jc5Gw6G7ZXLoWv/hCuxVCvCWoYlHaicfbit2V00819fy93VBPh2QboP93GXlT0krqlo/BSrfSl6bOlbVFoKIsKQ8u6I9ysY4/1lR6Uv3AtlesKN2v33OMfIjSyRmZfl+uG/1RbahS2dxrdAaejjf34XZCGjfEzfGitJPMtPXSTYPADohOtu3oC5PDvatzNNHf2wUB01SWYtLgXktFNRJki0eNGk3L5NoGj9ece8451Lt3rzD9MJ9Pw0m1psE+iKOiaCo4gPRoRA0WZQToWLvONMCj9znoZunjh5PK0rDXDfQY4dr/3BGYD2FkCCJT8RDVTqULA/OCc+007UfdCFfD91+52qyzrDWJ0hHptry4BOsNbjvOH3WkjeQsT1O+1+mugU5q0shOM+f4LibBOnBcIadkqqtolMnrsCEgX34PO00Y5SqZOQ1Sii/9dY4+mrZs3UpyTWI0+/rrr6MX//cCNW7cmKpUqVJqVzCLqCia/8LfPwDgv1+qsnKePPxwM+O8X8xB6F/s4EGSi25B2/8R1OFijwzZ2uiz8a4Si2dlxNnzSurw1xfL/nuvO+mM02x0cXcP3f2A1wgYfPKBi9SVPRbdocqWB5dQ5YLTpJNB/a710Gdf6fTtpzlGKF7NTsF5Iz2GPYB50O08Qw2CBqkuxaSpW7cu1a9fn8aMfd9wzdDwpUuX0rJly6hRo0Zh+wEyYIbBH/aVawZXTblk2Aep4ilwKyrLyNV778Nt8AEt+WvMLLhJ+cyLGm3cJChXvroUs9QBSSZ/adLInGG+kVdSf5ER9wA3xP9cpPuIYh0rI2RffeSiW6530O/zcqh712LYI1VXZv7y4FKmEpkBETtEDLtcbKdqVc0SMrBSZlHgX5a0bmEz7ALb+QuifbhRm+pS7J6hoddecw298+67dMedg6iCdHyFXBv27VNAx/rfLAjqEYgCQoAYIA6IgjSEqxVxQCSci1YQqoX88ZfPaPL5UPrvnU56aKhX+sZuGYJ2EnzsCfIxk3Pa26QrYZPHJilelo+/wEVAiBOC0DCM062rgx4cohnu06NPeenKvg5aIMOkM2aZhoMRe/W0yzWdqcf/KQG1bkFYVsmGjeaeDDCoJOrR12MMurkyrI0Je+GvNsIaDWFnCVFYwRoM8pcM6zZrata/Tfr7kI1F57CvwuNbisLk5cEFAxuP4EAwK2Mdp46xpoDgggTB+uP8c3SaO18YIWSkff2dbqwdt/2NI5Kur1kG+4pY/jMGQvb+5+6+3WGEvK+/zUsI3Z8l+zftR0Eb5AVu6udp4LL5R8LU/m+/LRFz58wSa1avDBs1U3kRPZP3dYR8Wlrs2rVL7N+/34icIdyMP0kiI4KGrZJIomcvvuYtDmMiwnLfo75IF6IuCG0i6qL+EKqVATxDEHpFOBTnsEW0R4Vb+1/nMcK1y1bo4oyWZh7kQ5gVx4gUIST8rYxUqfAyziMqJH1y0f4iM+SMtIeGesSc+ZpA+BnHaOeYcWYEDfmRFvyHqNs/O3yYKGywRXRJRZMQrUK7XxvpwwGRL4SdZ87RjHYq3dfe6hFyABsh3XC4IMrl36b8K9wCGCAyqPQMvt+MfOX2N9NQH+yAdiEPwtTjP/IW44aw/oh3vEbYXmGAfE8+7xUzZmkCfcUx+oQQPAS3CVR92F50uVts3BQaD6NA0L9kRs9sGPzGZSDKf3a7Q0aJ3FS7dm1jNsGsggc28Sf7WbzF7KOk1xVmtG3C6ICJTp2OeIuI0Z9yFjpBRl2OKFrr+CuBi6QePUE6okGVjvDPQbTtb0GHH266fHieCzceYxW4g90LPHSXfBQGLhquyHhGbNduovse9dJj8kYhHttJlJSFS3nqBXZwz5T5QmFXHj2h8mBNs0YGdI6sQcbN0VB5wqXBo8C7nNPyBegAE+TAn3LD4KKpNOWu4difOOHAiCYdRGl4uo+UwTr8CYNzwYRBWu1avvLxIAx03jLIS5vk3fdLLlTE8NWxYpWDjq6NXImTsnApT83lwa48ekLlwSNAuH2QbhLzpR5rFZAFpABB1M8CkI40CNKRJ9vkoPTKENZ+4HGv8VgJrqi/LRP082xhrCP+e2f2YZIJYyAm0tiMxb256FduGQgCwuCGJ37NCeJkI2EwOF58ykn16mrGs29PPq8Zi2s833XD1Q66vIuafTJhGGVXH+TEYPJGwywhZwNsHUWzhKYhEhb4eD+IggkV5xzyJ53/Sge9Vq1ahuul3C+QBH+KQNkFqa+3devY6IUnnfLPF6XyneW9dEXA+e8uM2aKAY5BjxlChYcxS6h9dBBEQBryYR9haQQAKspnLkA+hJ7V7KIAUS6aIpRKz7YtwrosmYGA86ijjjJ6omYFbBVZVBqOVRpIoQgAooBUIBBEHWMfBMI5f9IhnYURSHcEnIfJSyAGOP4gIISaSdQsga1yt7DFeZABJFGEQjr+FLlwHvkU8dIdKG4/I6AQMBY0GNzGEwBF7hhcNDXgQQKQQ5EGZFBpyIM/nFPpUKzKYot0FkYgkxBwgiz+j/ajc3DBlGDQY7bAn5pNQAb/GQWkUjMV8qsyyMfCCGQaAsZMg+fElBsFMqg/RQ61VkEepOFPuWIqj5ptkEcRClsWRiDTEJB8MN9Rho6BABAMdkUUzELYx0wCYkD8XS7kVcRR+8ijSIh9FkYgkxBwqgGPLcgAYoAg2MdWzUI4D/L4zzYAAun4808HefyJlUmAcV8YAScGN/7UoFdbDHz/mQV5QA6sd0Am7PuX8ycP0lkYgUxFIGCmAVEgakbBPkgEEuAPrpqaRUASkAeCNBxDmDAGDPwvgxEwPh+oSKEGv5pt1GyC/vuTBenIg5lIlclgjLhrjEAAAnJSKXr2rGjWUC6ZmkmwxZ8ijSqtZhScY2EEsgkBkzGyx4os6LwihH8akyObhgX3tTQEiklTWqZEnZsoX9fKwgikGwJJIw3ebYw/ljAIaNPlCfl2C0fLMBk4OVkrA5tcq1h+2976GtNrgMEk+K3ShAkTKC8vP70ab3Frk0GcpMw0yeioxbaMW3WMVdygjJsivgsZNyhZUbYgwKTJFktzP+OGAJMmblCyomxBgEmTLZbmfsYNASZN3KBkRdmCAJMmWyzN/YwbAkyauEHJirIFASZNtlia+xk3BJg0cYOSFWULAkyabLE09zNuCDBp4gYlK8oWBJg02WJp7mfcEGDSxA1KVpQtCDBpssXS3M+4IcCkiRuUrChbEGDSZIuluZ9xQ4BJEzcoWVG2IMCkyRZLcz/jhgCTJm5QsqJsQYBJky2W5n7GDQEmTdygZEXZggCTJlsszf2MGwJMmrhByYqyBQEmTbZYmvsZNwSYNHGDkhVlCwJMmmyxNPczbggwaeIGJSvKFgSYNNliae5n3BBg0sQNSlaULQgwabLF0tzPuCHApIkblKwoWxBIykedsgXc8vazZcuWVFhYGJB91apVVLt2bapatWpxOr6FOmrUKGratGlxGu9Yj0BSPupkfTdTu8bLL7+chg4dSm63O6Ch69evDzhu1KgREyYAkeQcsHuWHNwDas3LyytBmIAM8sDlclG/fv2Ck/k4CQiwe5YE0ENVCRdtwYIFoU4Vp61Zs4YaNGhQfMw7yUGAZ5rk4F6i1r59+5LTGd5bbteuHROmBGrJSWDSJAf3ErXCRfN6vSXSkeBwOAikYkkNBNg9Sw07GK3o2LEjTZs2LWSLtm7dakTTQp7kREsR4JnGUrhLr6ygoKBEBoSZL7nkEiZMCWSSl8CkSR72JWrOz88vkabrOoUiU4mMnGAZAkway6Auu6Jq1apRjx49jDWMyo2ZBusdltRBgEmTOrYwWoLZRtM0Yx/3Znr16kVHHHFEirUyu5vDpEkx+4M0FSpUMFrl8Xh4lkkx+6A5TJoUMwru1fTu3ZtycnKMGSY3NzfFWsjNYdKk0Bj4409BM2bpdNyJucZjNW3bF9DUH3T6bbmgAwdTqKFZ3hS+T5PEATDle52+m67TzDmCFi0R8kln4WvNvrpEFUcROS8qTjv1ZBu1aWmn88+x06WdbHTUkbbic7xjHQJMGuuwNmr6bZmgt0dr9P4Enbb/I6hJIzud085Gzc+y02mn2Oi4ekQ1atho8F230ssvv0a79xBt3iJo9VpBi5cKmjVXp+/l7APpcZmDrupnp8u7sMNgAGLVP8FiCQJLf9dFv2s9giofEqc2KxRPPOcVq9boYeuWTwaEPbd3nxCjP9REp+5uQ1/Lc91i4ida2Px8Ir4IUHzVsbZQCNz/mNcY3M06uMUHE+M7uOf9oos+V5tkvLy3WyxfGZ6IodrGaZEjwKSJHLNyl1iwSBeYBSocVSheGu4td7loMn4/XTPqclYvFCPfTWxd0bQvk8qwM5wgP3jcJJ3anO+hqlWIls5x0R03OxJUk6m243l2mveDi+4aaKcbbvfSoHtDPzGd0EZkiXIOBCTA0MPf0ujWu7w08AYHvfp8+N/IJKBqQ+WYD3W64kYP9e/toDFvWl9/ovqVKnoZ0ThbYuS7JmEevtdJj92f2NklXNMH9LFTrZouurwAs42XiRMOqCjTeaaJErhQxT77SqduBR66/24nPfFwcgjj3y7cB7qkp4fuvMVBLz7N10d/bGLZZ9LEgp5f2XXybn7Lcz3U7VI7vTM8dQaoctVGvuKk669KPpH9IEvbXSZNnEx3aZ6Htm4j+mWGK04a46fmnoe99L9huhGQwA1UltgQYNLEhp9R+nW58L9FLvx//tZF7dukZkCy1XkeqnM00afjUo/UcTCBpSqYNDHCjQcpT2jopivk4vv5J1LHLQvuFh69uehyD00c7aK87qlJ7OA2p+oxkyZGyzz+tEYvvKrR+uU5xj2ZGNUltHjfa7y09g9Bc6fzbBML0HzJiQU9WXbYSM2ITuEmZqrLoFsdJB+7oc+/Nh/4TPX2pmr7mDQxWGbUB+aTyjdekx4wtmxuo4s72um9sUyaGMzOv9yMBbwJH2vU83IH1a2TPhGpfr0c9PHnGu3YGUvPs7tselwiU9BGe/cSffWtTt27pheEPS4z2/vlNzzbRDus0svi0fYyAeWmzzAH3cUd02eWAQyVKxF1PNdO035i0kQ7LJg0USI3d4GghqfZqXbN9CINuttO3kuaM59JE6XpeU0TLXDyl5jUtHH6EQb9PVO2e+VqfllHtLbnmSZK5Nb+QXRS/fQkzUkNzHbj7TcskSPApIkcM6PE1m1CRs2iLJzkYnXl4zQQ9IElcgSYNJFjZpTYI98SU6VKes40VYvavUdGAFkiR4BJEzlmRgmvJsiZpk/aqw+uFb0yOkoEsrcYkyZK21eoYJMv94uycJKLqXbLN9+yRIEAkyYK0FCkRjWiHf+m55pgx06z3dVlH1giR4BJEzlmRol6x9how8b0JM36jWan69VNzzVZlCaLWzEmTZRQnnKSjVasSk/SoN0VpXt54glMmmjMz6SJBjVZ5swmNlr4a3qS5pdFOp3VlAkTpen5iYBogWvX2k5b5H0OvNA83WTGLCF/ls2kidZuPNNEiVy71jaqeZTNeNI5ShVJKbZqjaAl8hGgC89n00drAEYuWuRkOfws4KNP0+vBx4mTdfkiQRt1kj9GY4kOAUYuOtyMUn3y5fuT5c+HZ85JH+KM/kCjvrLdLNEjwOhFjx2df7bd+DLZ8DfTgzQfTNQJ7tl1V6bpowwx2CqeRZk0MaJ5u/wagPzmDM2dn/oBgedf0aiffCl6w9M5CBCL2fkVTrGgV1T2vM4eOuwwoq8/Tt1XI708XL41R35+Y+ncHGrEpInJ6jzTxASfWXjIQw76Rr5s/LU3tThoi78KuGT3PKwZL2ZnwsSOL880sWNoaHjgMY2efMFL83/MoRZnpZb7gzdr4oO3+OgTS+wIMGlix7BYQ6fuHtq0heinb1xUo3pxclJ3Bt7tpRHv6DR3mouanZlaZE4qMDFUzu5ZDOAFFx01wkke+c3YvP6elPjZwCNPaPSafAPomJFOJkywsWI4ZtLEAF5w0aNr2+ijsS5avVZQZ/kxpZ3/Buew7viBxzV6/BkvDf+fkwry2MzxRJ7ds3iiWaQLz6PlytnG5bLRqDec1NziNc41t3jp3bEajXjZSTdczfdk4m1ivgTFG1Gpr9EZNvrxaxcdI1+80eJct2VRtZ9n69T8bA99+qVufIeGCZMA40qVTJrE4Epw1aZMdtH9g500cLDXcNfwgsFECCJjdz/gpbM7eWS9JCN48iO1Xdi0icAaOhnZRCFbpPeJRxxGNO2g/PhTmwvc1O9ab9yeVdu8RRAW+8ed7qbR8jPoWL98OclF9fnHZQm1Kq9pEgpvoPIPJ+n0yuua8UrYVs3tlNvNTl0uthvuXGDO8Efbtgv6dqqgyV/o9PFnmvHzhIE3OOjuOxx0uHwqgSXxCDBpEo9xiRrw8vQP5cOTGPjb/xHyW5g2aiZ/SYmPyB5bz0ZHVrdRhQpEXvmAwZ49gjbLez9r1glavFTQ7yvMh0NBtl49HXRlX3YWSgCc4AQmTYIBLkv9rLmCZs3V6dclwngCeeMmQTt3ERUemEoOeyWqWqO1sT5qcCJR44Z2at3CZjxdXblyWZr5fKIQYNIkCtkY9AohyG6304QJEyg/Pz8GTVw0EQjw3J4IVFlnRiPApMlo83LnEoEAkyYRqLLOjEaASZPR5uXOJQIBJk0iUGWdGY0AkyajzcudSwQCTJpEoMo6MxoBJk1Gm5c7lwgEmDSJQJV1ZjQCTJqMNi93LhEIMGkSgSrrzGgEmDQZbV7uXCIQYNIkAlXWmdEIMGky2rzcuUQgwKRJBKqsM6MRYNJktHm5c4lAgEmTCFRZZ0YjwKTJaPNy5xKBAJMmEaiyzoxGgEmT0eblziUCASZNIlBlnRmNAJMmo83LnUsEAkyaRKDKOjMaASZNRpuXO5cIBJg0iUCVdWY0AkyajDYvdy4RCDBpEoEq68xoBJg0GW1e7lwiEGDSJAJV1pnRCDBpMtq83LlEIMCkSQSqrDOjEWDSZLR5uXOJQIA/6pQIVCPU2aJFCyosLAwotXr1aqpVqxZVrVq1OB0feho9ejQ1bdq0OI13rEeASWM95iVqHDJkCA0dOpTcbneJc/4JjRs3piVLlvgn8X4SEGD3LAmgB1eJTwSWRRiXy0X9+vULLsrHSUCAZ5okgB6qypYtW9KCBQtCnSpOW7NmDTVo0KD4mHeSgwDPNMnBvUStmEWcTmeJdJXQrl07JowCI8lbJk2SDaCqz8vLI6/Xqw4DtggA9O3bNyCND5KHAJMmedgH1FyvXj264IILAtLUga7rBFKxpAYCTJrUsIPRij59+pRoDWaZzp07U+3atUuc44TkIMCkSQ7uIWsNNZtglundu3fI/JyYHASYNMnBPWSt1apVo549e5LD4Sg+j5kmFJmKM/CO5QgwaSyHvPQKcc9G0zQjE+7N9OrVi4444ojSC/FZSxFg0lgKd9mVYVapUKGCkdHj8fAsUzZkludg0lgOeekV4l5NQUEB5eTkGDNMbm5u6QX4rOUIhL+bFqemTPhEj5Om7FFT+5ie8rGaUdThnAHE+EVud7kMpLxuiZsPEv4Yja1K4NO7kUOQpSX21SWqOIrIeVGWAhB9tyeOcSWUNAmfadD1CaNdlN89ccyPHt7ULTlwYB8aNqxr6jYwBVsmBJG9auIv0jySU9D4aBKHmVPUMLJZTJoUtc15552Xoi3jZjFpeAwwAhEiwKSJEDDOzggwaXgMMAIRIsCkiRAwzs4IMGl4DDACESLApIkQMM7OCDBpeAwwAhEiwKSJEDDOzggwaXgMMAIRIsCkiRAwzs4IMGl4DDACESLApIkQMM7OCDBp4jgGvp2mU4MmbvppprU/vMN70597WaN+13qoYSs3XTfQS7/8Kp+TL5JpP+p09wNeo21z5lvbNtWG0rbyV930xTc63XKXl66+OfQLE0srb/U5Jk0cEf/jT0Hr5N/mLXFUWoYq+YYn6tTDQ1u3CXr/bRe98ZKTpkzV6Ybb5EiUgvP/7iL6brputK0MdZaexu9fIAcOEi39XdDrb2m0ZauP7ObZ1PvPpImjTW68xkEbV+RQQZ51sH72lU4/zJAzye3ma5/ObmenRT+76M1hLqNn+Olvrvzpb4/LrGtTeSD9ebZOz75kvnWnahWie+/yvbaqPOWTmSe1kEwmEnGq+5i6tjhpKp+aGbNMd6vO0b56jzrSRs2a+o6hyWEPPC6f9sTk2rRZUG5/r3xVlU+/LXWa52tUmD1Lfu4cpu6AZLz7e+YcnSZNlm+UzLXT6rWCPv1SN9wLXMEv7WTye/MWQR9O0qmifMtRw9Nt9NJwzTh3/VUO+QJxonfHajRjliCPV1Cnjna6oo+DcLXdf4Dom+90+aIKjZ4d4qTxH+n0tTw+9hgb3X+3g047xWe1BYuE9LE1mjVHUBV5FbxJ1n/h+Wb9+/aT0a4583S681YHPfWCJt8cQ/TaC05a+4egifJFIsfWI+rf20F79xF9K12ljz/T6LmhTqM+9On4Y2308L0OqnmUr06sS155QzNmDbS3TUs7tWhmozq1bVRPtrF6tQC4jL4OedZrrAVw5pEnzbUAXJ79so1oy3tvuKia70NqgQqKjr75XifMVnDvmp9pp9tvdlDlSiGzlhtDkAF6p/6g07xfBJ3cwEYP/tdBDU600V8bBHW42EN/bxc0+QuNNkl7XnuFI4DkK1cL+mCiRnPnCzq3g53uGWTaMHSrkpAqEixU+ZCQA7XMWsaM84p6pxUK5K9c55DoVuAWV97oMY6R9umXmpj2o1ac55SzCgX+cK7luYXC4xGic0+3+M+DHrHwV13c87BZ9qY75Qkp6hj56zcuFFfd5BEXXe4u1r9+g27km79QN9LGTdJEYaEQg+839az7UxfI0/4iswza2KRtodFW6Hx7tCaadTDbM/RZr6Hr6f95i8/j3IDrfXXmX+E28qh/Pfq6RZsL3GLffmH0FTrx16m7W8ycExo/9FO1Z94vupi7QBdffauJWvXNduzeo7QL8fjT8ioi9c2e59P15PNeQz/KAX+cP6NloTh40FfOf6+8GL403NS1YaMuduwU4uxObgNz6Nr+jy5efM08f/Mgj9G3XbvNWlA/cIV9YDdl32EjTDz92xJqX5cmhI6Jk319DJUv1jSKVUFZ5dGJ8pAGep550QQTBlaCwauMiTQMIBxjYBw6JMSmzbrYvEUXI9/1GiCrciARDIC8GPAQDEAc/zDDB+otd5mkGHSvSS4MJOTBQIJ8N82s76NPzTLQi/P4+325LvYfEGL5SjPv1B/MvIo0KH/9baZ+9EMJ2o62KVm81CQqSKYEFwLU8fd2U7dKD95emmf2yT8dBERZtE1JMGl+W2bWCfyU4EKFciBQOCkPhupipBV1+aGhJgZbtwVi+tzLgfWgblw4d/5r1r56rdnGXlcGXmDCtc0q0pg+RxJmuFBVHnG4mdq0sc9t6dXTTvVPsNGyFYIOHSI6WrorkF497PJNlER160gXRvrz74zRadUaQW07eoy/szt5DPfnlJNshiuAMpWK3I5GZ/j0Y+qHwCWDYEH968wcatXcRosWCxr5nul4w42A4LtLtWraCHrPOM1Ghx9Gxa5d0YsxjXzq32EVzT3kV3JWE5vhuiHUCpFXY2OL9ivp0NY0za7dKqX821K+DVWsZPzHZr+wtlCYLV8pjH7tLqXO8mA4fpSLVv+aY7jWcIHhkkLKE1WE66pcUdgdsn6DsUmZfymzpgmHCBaI8IkRypWui7E+Qd7ghaO86lO+JNIEabBIpF7Rwl0NXPn6ZKOeDhd7DT/7kgvtxjoFoVslWMPEIliz+EtruX6B4F7FQRl+PUwSERcJDBo1cPzzx2N/xSo5qciLyOypkeEVqu5gDDHoQZRuBRpdM0CuB8+z05LfNPJdEkwtwTYM1h2MU/D5ZB0HmS9ZzSi93kVLTAMjKhRODjvMRrPnCWOBHJxnw8Zgc/lygIwQBBUgs+bqxkL1hqvt9MpzTjr15PB1GgXi8A+z1YwpLmNGvKCrh26720tb5ML8s/Eu+QWBOFQQQsXhEi8EKtbKgEGwlIZXcF4cB2P4v2GacZMSF7DBtzkonN3KIk2oulIhLeVJgygaIi253UofPa1b2GjjJhkxet8vjikRxg2zL6f4TRNBqKuQbddLTChuHmRGobp3NetD1M0K+XO9GUV681Un3TfYQYtn5RQTORH1Kxf4safkDODHG7ipDw0NxLCs+v0x/GeHoMH3e6mljPypCxE8BH9RZClM/Hv9/KuN235KumcIgV7exRzEI94xDfjQPeYg3rHT7PuadYGGwFrk8691uv42r3F3+Sx5n2Laj4I2SCJN/TzQBYGfjZAw7kS/PVqnJo1s1Dff1F+1CmYWQXIxT+e0t9MDj5skmrdAl2FUm+FqgJwQrLEqFq1ZcLxnL/5LH9xvZsOMAdmzF1tz1lJX9393CWN9hCv7gOu9hnuJx1wwa1Y6QtCJx9uMthkKwvxD/yDb/hZUu5apX60dEJ4/qX5RWtGd9n92mIqu6uegR57w0phxGsmFN+XJN6DirvzkL3Sa9X0gXmaJwP/hMDwoMYHMXyho7HjTdk8+b25xE/bgQVvxuhR2xq0E3GpA/RBcJJXgSQaImsnMoxT4Hy4SEa90RETKGz1DaBH5EZ5F6BNbhB9/kaFVyJSpmpGOPPhDCBchTSXvjjXLq/OI4mzc5IsO5fY3o0MI7SJUi0gN8iAMqgR1qJDtZb3cYslvutEG6HxwiKc4AofjC7q6BULUkLdHm9E7pCMy9thTXvFqUX+QhmjYgkW6GDjYjCQhDSHyf3boYu8+URxeRbr/X//rzKieap//9ra7fbrQD0SbVLQOOtBHpCEyqHQijCvvGxlq0B4V5sd5YB0uvK3qLQ+Gz75k2gE4oL+IVmIfuE7/SRNuGQxT4XnUKV3igPA/7IqonorCoW3q1oFqR6itVdGzlAo5K9LgngxCrf4DPhRIodIQEkYIOFSoVhkcJPlL3nNRoc1gPdAhZ4ZiwXG4exfFmWLYwcAGqZat0MWvS3Tx82zNCHW/NcocfLjfkSjBQPvjL138uV4X2C9LyoshLgQghxKEv1UIGmnYhw3801TeaLdWkSYl3TNMwP53yyOZkBFu9b+7H6osfOrj6pluS6jz0OF/VxzH5QnjhtJVVhqeYjj/Uo/xgvjTT1VtMrdYa3zyuU7Vq6v0srRFfh5YnHBc5PrLwrBS0MfbEOzwF0TGSrOBf95U208p0qj1CnzxRMjf202tW7YSHVkjETVErhOPnOARmvcn6HTiCRqdLh/nQcQMj/bjERw8/KnuX0WuPf4lUhHD+PeyDI3RToXlLQd/tDxrmtdG+h45gZ/98BPhffny1q3ywQWAT6z8eqxp5LNN6nTSt3Pma8ZjPWothe21t5qPAyW9cUUNSHUM0Uyr3DNLPupUnu/T4KaexwxUGTTH9O/vIpXB/TJP40FL/xuUcLeCXYYylSQ4A9wxhGH9I3IJrjIi9amOIfDD92ky4qNO5bEM7oIHub3lKVbuPME+drkLWpgRF4pUJQxgSAcMrTCXGRy3oiaugxHIEASYNBliSO6GdQgwaazDmmvKEASYNBliSO6GdQgwaazDmmvKEASYNBliSO6GdQgwaazDmmvKEASYNBliSO6GdQgwaazDmmvKEASYNBliSO6GdQgwaazDmmvKEASYNBliSO6GdQhY8nsaPH06Qb6ulYURyAQEEk4aPLlbcFXRW/EyATGr+uCdKt/DIV9M5mhtVY0ZUQ/GW6Il4b+nSXQHMlG//D2VfCmifPHhhAmUn5+fiV1M6z7xmiatzceNTwYCTJpkoM51pjUCTJq0Nh83PhkIMGmSgTrXmdYIMGnS2nzc+GQgwKRJBupcZ1ojwKRJa/Nx45OBAJMmGahznWmNAJMmrc3HjU8GAkyaZKDOdaY1AkyatDYfNz4ZCDBpkoE615nWCDBp0tp83PhkIMCkSQbqXGdaI8CkSWvzceOTgQCTJhmoc51pjQCTJq3Nx41PBgJMmmSgznWmNQJMmrQ2Hzc+GQgwaZKBOteZ1ggwadLafNz4ZCDApEkG6lxnWiPApElr83Hjk4EAkyYZqHOdaY0AkyatzceNTwYCTJpkoM51pjUCTJq0Nh83PhkIMGmSgTrXmdYIMGnS2nzc+GQgwKRJBupcZ1ojwKRJa/Nx45OBAJMmGahznWmNAH/UKQXM17x5cyosLAxoyerVq6l27dpUpUqV4nR86GnMmDHUtGnT4jTesR6BhH8+0PoupV+NPXr0oCFDhpDb7Q5o/IYNGwKOGzduzIQJQCQ5B+yeJQf3gFrz8vJKECYggzxwuVzUr1+/4GQ+TgIC7J4lAfRQVbZq1Yrmz58f6lRx2po1a6hBgwbFx7yTHAR4pkkO7iVq7du3Lzmd4b3l9u3bM2FKoJacBCZNcnAvUStcNK/XWyIdCQgA9OnTJ+Q5TrQeASaN9ZiHrLFevXrUsWPHkOd0XSeQiiU1EGDSpIYdjFYUFBSUaA1mmc6dOxvh5xInOSEpCDBpkgJ76EpDzSaYZXr37h26AKcmBQEmTVJgD11ptWrVqGfPnuRwOIozYD8UmYoz8I7lCDBpLIe89Arz8/NJ0zQjE+7N4PiII44ovRCftRQBJo2lcJddGWaVihUrGhk9Hg/PMmVDZnkOJo3lkJdeIe7VYA2Tk5NDlSpVotzc3NIL8FnLEWDSWA55+ArX/SHop5k6HXt8rvFYTZt2ven76TotXSZo/4Hw5fiMtQjwYzTW4h1Q2zff6/TdNJ1mzhG0aImQRBG+8/uOIar4HpHzouK0U06yUZuWdjr/HDtd2slGNY+yFZ/jHesQYNJYh7VR09LfBb09WqP3J+j0zw5BTRvb6Zx2Nmp+pp1OO9VGx9UjqlHdRoMHD6SXXx5Gu/cQbd4iaPVaQYuXCpo1VxJNzj6Q7l0ddHU/O11+KTsMBiBW/RMsliCw5Ddd9L3GI6jyIXFa80Lx5PNesXqNHrbu6dOnhz23b78QY8Zp4pIecmqS+lqc4xYTPtbC5ucT8UWA4quOtYVC4L5Hvcbgbn62W3w4Kb6De/5CHxkv6+UWy1aEJ2KotnFa5AgwaSLHrNwlMKAxC1Q8qlC8/Lq33OWiyTj1B020Os8tHNUKxYh3EltXNO3LpDLsDCfID/5wok5tzvdQ9WpES+e46PabfHf5E1HlBefaae50F919u4NuvMNLd94b+onpRNSdbTo5EJAAiw9/U6NbB3vpthsd9Mpz4X8jk4CqDZVjx+k04AYP9evloLFvWV9/ovqVKnoZ0ThbYsQ7JmEeuddJj96f2NklXNP7F9ipVk0XXV6A2cbLxAkHVJTpPNNECVyoYp9+qVP3Ph564D9OGvpQcgjj365vp+rUqYeH7rzFQS8+zddHf2xi2WfSxIKeX9l18m5+i3M91KOrnd4enjoDVLlqI1520g1XJ5/IfpCl7S6TJk6m65Lrob+3Ey34yRUnjfFTc+/DGr3wqkZLZEDidHkDlSU2BJg0seFnlFYL/5+/dVH7NqkZkGwlI3lH1yL6bHzqkToOJrBUBZMmRrgPyAcpT2jopivl4yzPDU0dtyy4W1N/0OnCyz00YbT8jU731CR2cJtT9ZhJE6NlHntaoxeHabR+WY58hWyMyhJcvO+1Xlojn2Gb9wPPNrFAzZecWNCTZV8boRnRqVQnDLo56FYHyacU6LOvzAc+Y+x61hZn0sRg+vfe12m7fFL5hmvSA8aWzWzUqaOd0G6W6BFID2tH37+ElpRPFlPu5Q6qe3T6RKT69XbQJ59rxs8SEgpOBitn0kRp3D17ib7+Tt7MvCy9IOxR1N4vp/j94C1KDLK1WHpZPIWs9MMM08W5+IL0mWUAXyX5YpuO59lp+k/sokU7nJg0USI3d4GghqfjGa/0Ig262661nebMZ9JEaXpi0kSJ3NLfdWraKP0Ig+6e2dhGK1cLOnAwys5neTEmTZQDYO0fRCc1SE/SnFzU7nV/8romGvMzaaJBTZbZuk3IqFmUhZNcrE5Ru7fJPrBEjgCTJnLMjBJ75FtiqlRJz5mmalWz3YgAskSOAJMmcsyMEl5NkDNNn7RX7S56ZXSUCGRvMSZNlLavUMFGhwK/Yh6lJuuLqXbLN9+yRIEAkyYK0FCkhnxhxs5/03NNsHOn2e4a1aPsfJYXY9JEOQDqHWOjDRvTkzTrN5qdPqZOeq7JojRZ3IoxaaKEEu9VXrEqPUmDdleU7uWJJzBpojE/kyYa1GSZM5vYaOGv6UmaXxbpdFZTJkyUpucnAqIFDj9r3iLvc+CF5ukmP80S8mfZTJpo7cYzTZTItW1lo1ryUxd40jmdBI/P4BGgi85n00drN0YuWuRkue7ydU2TJqcXadBePGR6sfwxGkt0CDBy0eFmlOqTbzd+Pvzz7PQhzugPNfm6WjZ7DGbnNU0s4J13tp3atrLT8LfSgzQfyA9JrVoj6Nor0vRRhliMFceyfMmJEczbb3bQhxM1+fuU1A8IPP+KRv3lz50bns5BgFjMzq9wigW9orLnd/FQhQpE33ySuq9Gemm4RoPk5zd+m5vDpInR5jzTxAggij/+oIOmyJeNDxupxUFb/FWskhGzex7SjBez8ywTO74808SOoaHhgcc1kt/RpPk/5FAL+aqkVJILL/PQ3n1kfPQpldqVrm1h0sTRcvisxcZNRD9946Ija8RRcQyqBsqPS418V6c58itpzfgpgBiQ9BVl98yHRcx7o0c4SZO/s8kb4KFDh2JWF7OCh4dq9Jr8KtuYN51MmJjR9Clg0viwiHmvdi0bTRrrorXrBHWWn97YsTNmlVEruP8xjYY866XXX3RS71w2c9RAhijI7lkIUGJN+m25nG36e8jhsNF7bzgJr4O1SoSMfF9zi1e+elajka846fqr+J5MvLHnS1C8EZX6Gsn7ID9+7aJjjyGSnym3LKqGJxOan+2hz7+WLzmX36FhwiTAuFIlkyYxuBJcNdy3uf9uJ912t5cukUGCRN0A3bWbaPD9Xjq7k4fq1iGa/6OLLuvMpk2QaZk0iQJW6X3iYQfNmOKiQvk+gbYd3dT3Gi/F61m1TZsFYbF/3OluGjteN9YvX0x00YnHW+cOqn5m05bXNBZae9xHOr3yukaz5+lynWOn3G526nKxnRo3LP8g3/q3oG+nCpr8hW68/R9PLA+8wUF33+6gww6zsDNZXBWTJgnGx8vTP5ykGwP/7+2Cjq5tM0LCp51io2Pr2ejI6jbjsRyvfMBgzx5Bm7cQrZERucVLBS1baT4cemknO/Xq6aAr+rAbZrUJmTRWIx5U3+x5gmbN1enXJcJ4AnnjJiHfckNUeOB7GX2rTFVrtDZI1eBEkjOSnVq3sNH58unqSpWCFPGhZQgwaSyDuvwVCRk3ttvtNGHCBMrPzy9/Qc5pCQI8t1sCM1eSSQgwaTLJmtwXSxBg0lgCM1eSSQgwaTLJmtwXSxBg0lgCM1eSSQgwaTLJmtwXSxBg0lgCM1eSSQgwaTLJmtwXSxBg0lgCM1eSSQgwaTLJmtwXSxBg0lgCM1eSSQgwaTLJmtwXSxBg0lgCM1eSSQgwaTLJmtwXSxBg0lgCM1eSSQgwaTLJmtwXSxBg0lgCM1eSSQgwaTLJmtwXSxBg0lgCM1eSSQgwaTLJmtwXSxBg0lgCM1eSSQgwaTLJmtwXSxBg0lgCM1eSSQgwaTLJmtwXSxBg0lgCM1eSSQgwaTLJmtwXSxBg0lgCM1eSSQgwaTLJmtwXSxBg0lgCM1eSSQgwaTLJmtwXSxBg0lgCM1eSSQgwaTLJmtwXSxDgjzpZAnPplTRv3lx+yFZ+ydZP1qxZQ7Vq1aIqVaoUp+JDT2PHjqUmTZoUp/GO9Qg4ra+SawxGoEePHjRkyBByu90BpzZs2BBw3LhxYyZMACLJOWD3LDm4B9Sal5dXgjABGeSBy+Wifv36BSfzcRIQYPcsCaCHqrJVq1Y0f/78UKeK0+CyNWjQoPiYd5KDAM80ycG9RK19+/aVX3N2lEhXCe3bt2fCKDCSvGXSJNkAqnq4aJqmqcOALQIAffr0CUjjg+QhwKRJHvYBNderV486duwYkKYOdF0nkIolNRBg0qSGHYxWFBQUlGgNZpnOnTtT7dq1S5zjhOQgwKRJDu4haw01m2CW6d27d8j8nJgcBJg0ycE9ZK3VqlWjnj17BgQEEBzIz88PmZ8Tk4MAkyY5uIetFQRRAQHcm8Hx4YcfHjY/n7AeASaN9ZiXWiNctIoVKxp5PB4PBwBKRSs5J5k0ycE9bK1Op9NYw+Tk5FClSpUoNzc3bF4+kRwEonr2bPzHenJamyW11j4mVz5WM4o6nHMFMdaJNXrvnpHPG1E9RtPrCi999BkTJ5Hm1PfUJdvho8jmvCiR1WS1biEE6bsrRIxBVDMNasm93E4TRkddPOKGZluBgQMLaNiwS7Ot25b1FzN4n6s9UdUX+dwUVTVcKFIEQt2ziVQH508MAkyaxOAas9bzzjsvZh2sIDEIMGkSgytrzWAEmDQZbFzuWmIQYNIkBlfWmsEIMGky2LjctcQgwKRJDK6sNYMRYNJksHG5a4lBgEmTGFxZawYjwKTJYONy1xKDAJMmMbiy1gxGgEmTwcblriUGASZNYnBlrRmMAJMmg43LXUsMAhn9bL/8uQRd1ssjf2NPNGGUKzEI+mmdM1+nr77V6aNPdfp9Xo7fGd/ulq2CPv9ap8++0kn+SJMmf2i268/1gka+G/plgfJHnPTofRltKh9AabCX0ZbACyt/mqlT1ao2km9CIvkKsYTK6rWCXnpNo737wlezbz/Rzn+JvpyiU6eOvgaBME+9EJo0bVraJWnC6+Qz1iLgs5q19VpSG67kG1dWoOULchJOGHRoQIGDOrQtHdKTG9jojpsD39l86BDRmHE6TZnsot2bKlDhP+bf/m0VqHIlovwepeu0BEyupBiBjJ5p0MsqlYv7aslOeWYzkNlfdu8RNPZNJ53bIZAc8+foxqx1eZfAdP+yvG89AkHmi28DvF6imdLwkybLt0Tm2gnuy6df6oardOM1Drq0k28w/LVB0OQvdPrxZ532Sfem26V2uvUG84oMPT/M0OndsRo9O8RJI6Qrs+Q3QW+85JRfCrPR+I80+uZ7nbCGOa6ejQbf5qA6R9sIV/Dvf9Dp/Qkavf+Wy5htNm8x1xQLFwt68hEHjXhHN1y4c9rb6d67HAEz0iKZZ9hIjdb9KeiM02zUspmNGje00xFyjXTaKbZSwdqxU9CHk3T65judGpxoo4fucdBRR4YuU7uWjfAXLFgbNWlko5Pqlzyn8u7aTfTJ5xphi5dEjHxPo+UrBfXq4aDuXe20cbOgiZ/oNO8Xndq3sdPtNznI5qcOuGF9tXWboOZnyvNyFsTsBjlwkMJii/NwMWHbf3cJOuUkm2FPRfzS7ImykPLg++8uotff0mjhYulmS1tf0ccecHHBOHjvfdNGxx5jo2Zn2ujq/oEzuVlbHP/LlwtELPkDPAJ/ZcmYcV5R77RCQZUPicp1DoluBW5x5Y0e4xhpn36pGSp27RaiVv1CcfMgU+fETzQjz4cTzfMDrveVaXluoajf2NT51beaePJ5r7isl1u43UJs2Kgb9f22TBdy0IobbveV80jV+POvH7puucsjTjnL1Df8TW9xl+Yv1I02yAEp0L7zuriNY/TnpjvD9/3SPDNfk7aFAu2+oKt5nNtfNrBI0Fb0v1N3X5o6p7Ze2RRggv6Fk7826MXtAiboz32Peowt9F97q0ec0bJQ3P4fj4E/0t4d69MH3WjD3AW6gK1wHvkPHjRrDIctzv48WzN07vxXiEOHhOjc0y3eeNvUXZY9Ub48+K6X/YNt0GbkR1vRRjUu5AxttGHmHHOcDHnGK3pdGR5T1Ktk3EeasFWRDY9C5NU5cikvaaD5mRdNYzz+tM9Y4yaZpICBIDAAwHj+FTPP/gPmoIKxlSjAYBhNYrR4qW5sMVhARqRBnnvZKxb+qpsH8n/7i0ygQRglish/rjfzrVpjEsQf8PwrzHIYvJApU802DrrXT5F5KuC/Ig2MrASDHxcNJeUhDQYCMPl9uU+PKu+/3bvPxAo4yCu+IegXymLAYQBDMACRNnCw2X5cWHC8abNPP3BEGggEKQ1bXODQLwxcyJz5mnjlDbNceexZHnz7XuMubi/q+EXaFe2D/SBqHCmsD8hxgwtVeSQW0iTUPcOECFcG0rSxzyfoJd2I+x+z0bIVwnCh4DYsnZNjuDGbpDvxzhjz9VCY4pUcU9csf1lnu+FCwW2BXCwjUG+8rVGXXA+99IyzhPtx+GFKg2+r0o4/1tSh3J+/t/vybN1m7qMN9U+Q035TM+8/O3x5SttrKN05Jc2ly/D1d4L2H/Dhoc6F234sX5EFlwduYWlS6QjzLFzAalXNfdUvpFUt+s7tsdJthfzxl4np+I/NSF1uf+n7FomcNYw6d0tXD1Iatue0sxtu01nt3fS6dJMvOt8uXVezXHnsWRa+wOqDiTrVqmmjBYvMt8bIC6NhC6wJ8V3fdq3NPrU8V7bhRSdde4WDhsttosW3qEh0TX764VMjigTZt980YmW5YL9uoIduvMNL57Q3z/kVKfbDgxfaD/3XYfjqU6bqdHoLNw151itfIO5fsux9tCdYVDj4a3nfBbJmndnOThdGDhnus0QiCI8jmtYnP/K6yqpHfQt3xSp50ZZrl9lTXcV/KxfmEP7UWrI0bHtcZqebrnUY671O3T3Uo6/HGMiq/rLsWRa+a4vwfvQ+R3H75v2QQ2uXmG2sIF9XhgvBiJdNktw8yEuNWrtp3R++C61qS7y38bdKOVu4aIlpNCyOt/8jqMPFHjrsMBt9Nt5FZ8urWHnlyBo2+uErFz39mAne0Gc1ev6V0Pc7yqsT+e4a6KCCPDsNvNtLg+710u3/8Ro3GPsmYCAHtwuL9r+3C+rWJUL2Bysq5fhwiTXuJ60NMcg2bDQHXmnYSqfQuLrj5mw9uQBHgEe9R6w89iwLX/XO93m/lCTBQRmg+GeHIASIMLssmZ1jXDhXrRF0UTePDEyU0vE4nCr/6IxDZUoFomgYFLndzEHxzIuhDE+UAAAspUlEQVQabdyEY9P1QtSmvJJ/hce4s37PIAf9NjfHuHpOmhw7aQ7J6X+HdMWmfeEyrqg/fp1Dj8irXqhZKVRbS5o6VK7QaXDNMBDPbBJiCgxdJOJU5S4/9pRmRB2VggWLBD001MSvNGwH3ec1LnaIcsK1RmQRs/3uPUTlsWdZ+MLFxEyIyNjvy31oYhaGR7J5C5FclxhPYDRuaKOfvnHRzdc5jHGFJzMSKZaRBmFNJSPeMY2CMCykSmVzcLw8XKPvp+t04+2mD4vQKcKa8G9BMoj/OgfHmLFmzDJ1NzzdZkzZp57s65ZcCCMbbZaPr0AAOh5lgeCKBVHHqg6kPfC4lwA+QtO48qP9035EeBVnw4taMyidyBmchvAuZEvR1jgo+of2jfpAp3697OUiKK64EIS4lSDUDvGfRdSFSOW7qp8ZWh4zTjMeNcLgHHy/l3pf6aFnHjftUhq28oMG8rEfE3espXBTF+s+DPTy2LMsfOVXRui/d5reQ9uOboOICD13vMxDJ0nXXq1pX5O3BDDjwG2/+ALT7k2L1rsmCgn4X55IQ3CeSKJnw0aY0bNmHQqNcCa2iMogEqIEkR2kIzKCLSJVKjTc71qPuOIGX+gYEaEPikLRKI8IHCJTt93tMSIniJ4gIoS//tf5yiFkvGixbuRBPfhDBGfZCr04LIy0wfeb0ReEOVU+/y3qmj2vKFSnOlC0RRtUXkTt1qzTjZC2Sju7k1t88Y0m2lxgRqmQjrCwikBBDaJQSA9Xh3+V6/7UhYrWoQwie6hThceRhgjU2j8C8yFkjgjegkVmiB758Ae7qPAt6gmHLc5BB7BQEa4efd2GPpwrzZ6wCeouD76IeCLap9qH7Z33eIwQN+pBlA9tANawG7DwHxvIE05iiZ5F/QJ08Lc873LGlQDrgk/HuahtK5t8Gz6RioQFXwNwFa9ezZeK57RUdMiXGrgHfVj4b5I3LWsdZZPfdgk8H+3R/Y95qWIFG/WVV3xcnTErYcZ7U948xI1TRGsSIZgRsKZAoCQ46JGI+rA2wewNtxM3hv3dz9KwxTkEOLb9LeTWFmA31c7S7BkJvsD9Txn1O+F4W3E0FnVgtkMkbc9eaRsZUKpbp/zurHqXs6UvQFfARLKtKQd1aeJPGOQrizDIoyJTMHi8BK4KHp7cub6CMRhUSBr6EcKd9qPPFYpXnUoPwuGnnhy/vii94bYgyQnHha6vNGzVuVBPMqi6wtkzUnxx2wKud7DAhYPAJnhawCpJzOXSr/U7dpoHeHwlXWT7P2ZLpWsgH0VxyHUS1iVE8+XaBveWRo9MOGzpAlVU7Ux3fBNq/eFvIvxr3jx74nnNWPg+dn9Cq4zKiMGF7rzFYUSUvvhGp15yYQxBdOiaAQ4a+rCT1M3R4HJ8XD4E0h3fhK5psA7wmJwx0IQroB4GLB+8yc+FO89wRfx9/eS3KnNakCx8U3ZNc5j0z0M8xZJWFsedZ5bEIZCO+PpuaCQOF9bMCGQUAkyajDInd8YKBJg0VqDMdWQUAkyajDInd8YKBJg0VqDMdWQUAkyajDInd8YKBJg0VqDMdWQUAkyajDInd8YKBJg0VqDMdWQUAkyajDInd8YKBJg0VqDMdWQUAkyajDInd8YKBGJ6Th9PirIwAtmGQNSk+Ui+MQV/LIlBQHi/lz9HkG+pcLRJTAWsVeIb3a89o/o9DeOdWATkyyDk+wHsNGHCBMrPz09sZaw9YgR4TRMxZFwg2xFg0mT7COD+R4wAkyZiyLhAtiPApMn2EcD9jxgBJk3EkHGBbEeASZPtI4D7HzECTJqIIeMC2Y4AkybbRwD3P2IEmDQRQ8YFsh0BJk22jwDuf8QIMGkihowLZDsCTJpsHwHc/4gRYNJEDBkXyHYEmDTZPgK4/xEjwKSJGDIukO0IMGmyfQRw/yNGgEkTMWRcINsRYNJk+wjg/keMAJMmYsi4QLYjwKTJ9hHA/Y8YASZNxJBxgWxHgEmT7SOA+x8xAkyaiCHjAtmOAJMm20cA9z9iBJg0EUPGBbIdASZNto8A7n/ECDBpIoaMC2Q7AkyabB8B3P+IEWDSRAwZF8h2BJg02T4CuP8RI8CkiRgyLpDtCDBpsn0EcP8jRoA/6hQxZPEv0KxZMyosLAxQvHbtWqpZsyZVqVKlOB0fenr//fepSZMmxWm8Yz0CUX8+0PqmZm6NPXv2pCFDhpDb7Q7o5MaNGwOOGzduzIQJQCQ5B+yeJQf3gFrz8vJKECYggzxwuVzUr1+/4GQ+TgIC7J4lAfRQVbZq1Yrmz58f6lRxGly2+vXrFx/zTnIQ4JkmObiXqLVv377kcDhKpKuE9u3bM2EUGEneMmmSbABVPVw0TdPUYcAWAYA+ffoEpPFB8hBg9yx52Jeo+cILL6SpU6eWSEfCtm3bqFatWiHPcaK1CPBMYy3epdZWUFBQ4jxmmS5dujBhSiCTvAQmTfKwL1EzXLRg0XWdevXqFZzMx0lEgEmTRPCDq65WrRrl5uYGBAQQHMjPzw/OysdJRIBJk0TwQ1XtHxDAvRkQ5vDDDw+VldOShACTJknAh6sWpKlYsaJx2uPx8CwTDqgkpjNpkgh+qKqdTif17t2bcnJyqFKlSoRHbFhSCwEmTQrZY+06QT/+rFO943ONx2ratC2g76bptOR3Qfv2p1BDs7wpfJ8miQPg6+90gxQz5whatESQxyN8rdl3DFHFUUTOC4vTTm5gozYt7XT+OXa6tJONatW0FZ/jHesQYNJYh7VR05LfBL09WqP3J+i0Y6egM5vY6Zx2Nmp+pp1OO9VGx9UjqlHdRnfdNZBeeWUY7d5DtHmLoNVrBf26VNCsuTp9P10nIfnV7VI7Xd3fYWwt7kZ2VydYLEFg8VJd9LlaTiWVD4nTW7jFUy94xZq1eti6p0+fHvbc/gNCjB2nic493Ya+5ue4xfiPtLD5+UR8EZBXLJZEI3DvI15jcLeQg3vcpPgO7gWLdNHvWpOMXfPd4vfl4YmY6H5mi34OBCTQ0Zi/UFCLczz08nCNXnnOSfN/dFHv3PhC3vxMG419y0lTv3DR9n+IGrfx0Ih3Qj/4mcCuZpXq+Fowq6ArvbMfyDVLm/M9dGQNoqVzXHTbjeEf+y9dU/nOXiCDA3Omuei/dzropju9dMc93vIV5FwRI8CBgIghK7vAayM1Gni3l26/yUEvP2v9L8rfH69T/+s91DffQe+/bX39ZSOU3jmYNHG23xtva3TzIC89ep+THrkvsbNLaU3/Vt7f6dbbSz0vly/jYOKUBlXE55g0EUMWvsDkL3Tq0ddDD/7XSUMeTB5hVAtBnE7dPXTHzQ566RmecRQusW6ZNLEiWFQed/NbnOuhXHllf+u11BmgY6WrNkC6am+85KQbr0k+keMEd1LVMGniBH/nnh76ZwcZEbI4qYybmnsf0ej5lzVaOtdFp8sbqCyxIcCkiQ0/o/Rrb8qF/2Avzfwuh9q1Ts1B2VpG8mrVJPp8gisOPc5uFUyaGO2/Xz5IeUIjN13Vz07PDU0dtyy4W9N+1KnjZR6aMEr+RqcH32kIxieSYyZNJGiFyPvYUxq9+JpG65fnUJXKITKkUFK/a720ao1ISRcyhWAqsyl8ySkTovAZ8NDkMHlP5s5bHClPGPRi0EAHycdu6LOv9PCd4jNlIsCkKROi8BlGfaDLxb+QUan0gLHFWTbqdKGd3h3LpAlv1bLPpIe1y+5HUnJM+Fij3G4OqnN0ai7+Q4HSr5eDJn+hGWQPdZ7TykaASVM2RiFz7NlLhB+Rde+aXhD2uMxs75dT/H7wFrKHnBgOgfSyeLheJCF9+k+mi3PxBekzywCmSkcQXXienRBNY4kOASZNdLjRvAWCGp5uT8ufHLdrbae5C5g0UZqemDRRIrd0mU5NG6XXLKO62rSxjVauFnTgoErhbSQIMGkiQcsv79o/iE6SL7pIR8ELOiDr/uB1TTT2Y9JEg5oss3WroLpHR1k4ycXq1jFJs/VvJk00pmDSRIOaLIPoWZUq6TnTqG/f7pV9YIkcASZN5JgZJbyaIGeaPmmv2h3mG1JRIpI9xZg0Udq6QgUbHQr8inmUmqwvptpdIcf6ujOhRiZNlFasUY1o57/puSbYKV9SCKlePcrOZ3kxJk2UA6DeMTbasDE9SbN+o9npY+qm55osSpPFrRiTJkooTznJRitWpSdpVqwU8nMeNjrxeCZNNOZn0kSDmixzVhMb/fJrepJmwa+60f4ou571xZg0UQ6Bdm3stHWboKXyMxjpJjNmCmrfhmeZaO3GpIkSubat5KcujrLRV9+m1zNceHwGjwBddAGbPkrT87Nn0QKHct3lY/YffZpepJk4Wafa8rs2FzNpojY9X26iho6oT76d5i/U6efZ6UOc0R9o1LcXmz0Gs/NMEwt453WwU9tWdhr+ZnqQBh+SwsehrrsyTR9liMVYcSzLl5wYwcQrXz+cpNHseakfEHj+FY0GFDjojNM4CBCL2fkVTrGgV1T2/Es9hEdSvvkkdV/Eh9dM3XWfl36fl8OkidHmPNPECCCK42XnU6bqNGxEan5MCRGzex7W6MH/OJkwcbA3zzRxABEqHhyi0RPPeWneDznUsllquT94s+a+fURzp6fuTBgnM1iihkkTR5gv6eGRz6MR/fiNk446MjWIc+tdXnpzlE5z5VfSzmqaGm2KI+RJUcXuWRxhHzXSSfIzsZQ3wEsHD8VRcZSqHhqq0fC3NBrzppMJEyWGoYoxaUKhEmUabhpOGuOidX8KMj+9kbyI2n2PajT0WS+9/qKTevdkM0dp0pDF2D0LCUtsib8vx2zjIbvNRu+94aSWza1zizR5y+jaW7w0St7EfPNVJ9+Tic2UIUvzJSgkLLElNjzdRj9+7aJj6xG1Ot9Nr1oUVftppk4tzvbQl9/oxndo+CZmbHYMV5pJEw6ZGNNrSVcN920ekGHe2//jNb59OWtuYty1nf+ScQ/m3M4eqneM/BrbTy7qegmbNkYThi3OyIaFJj4nhj7koBlTXOTxELW/yE19rvYSZoR4CH45ilD38We46cOJuvFdTXzp7ITjrHMH49GPdNPBaxoLLTb+Y51eeV2jWXN1an6WnfK62anzRXbCGy/LK1vk+9a+nSZo8uc6Tf5So6Nr2ejWGxx09x0OqlihvFo4XywIMGliQS/KsphpPpAzAz6hvk2+sA9Rt2Zn2ui0U2x0rHz3QI0aNqogCaB5iXbvEbR5K9Ea+fXoxUv14p9Yw/3q1dMhnyVjZyFKM0RdjEkTNXTxKThnvjBmnl+XCOPTfhs3Cdq5i+jQ/u/ke9UqUdUabeno2jZqcCJR44Z2at3CRuefbacj5Nv/WZKDAJMmObiXWquQN0jtdjtNmDCB8vPzS83LJ61HgOd26zHnGtMcASZNmhuQm289Akwa6zHnGtMcASZNmhuQm289Akwa6zHnGtMcASZNmhuQm289Akwa6zHnGtMcASZNmhuQm289Akwa6zHnGtMcASZNmhuQm289Akwa6zHnGtMcASZNmhuQm289Akwa6zHnGtMcASZNmhuQm289Akwa6zHnGtMcASZNmhuQm289Akwa6zHnGtMcASZNmhuQm289Akwa6zHnGtMcASZNmhuQm289Akwa6zHnGtMcASZNmhuQm289Akwa6zHnGtMcASZNmhuQm289Akwa6zHnGtMcASZNmhuQm289Akwa6zHnGtMcASZNmhuQm289Akwa6zHnGtMcASZNmhuQm289Akwa6zHnGtMcASZNmhuQm289AvxRJ+sxL1HjWWedRYWFhQHpa9eupZo1a1KVKlWK0/Ghpw8++ICaNGlSnMY71iPApLEe8xI1Dh06lIYMGUJut7vEOf8EkGXx4sX+SbyfBATYPUsC6MFV5uXllUkYl8tF/fr1Cy7Kx0lAgGeaJIAeqspWrVrR/PnzQ50qToPLVr9+/eJj3kkOAjzTJAf3ErX27duXHA5HiXSV0L59eyaMAiPJWyZNkg2gqoeLpmmaOgzYIgAAUrGkBgJMmtSwA9WrV486duwYsjW6rhNIxZIaCDBpUsMORisKCgpKtAazTJcuXahWrVolznFCchBg0iQH95C15ufnl0jHLNO7d+8S6ZyQPASYNMnDvkTNVatWpdzcXHI6ncXnEBxg16wYjpTYYdKkhBl8jcBs4/V6jQTcm+nVqxcdfvjhvgy8l3QEmDRJN0FgAzCrVKxY0Uj0eDw8ywTCkxJHTJqUMIOvEXDHEBDIycmhSpUqUc+ePX0neS8lEPA5z2U0Z+JkXd5HKCMTn44LArXq9JSP1bxHHc65gsZ9pMdFJyspG4GC3PLNIeV+jAak6XWFR961tpVdO+eIGQFtd12yHzGabM4LY9bFCspGIK+bnca9V745pHy5/Or0/pvjd8S7iUJg4MACGjasS6LUs14/BPIHmIEXv6RSd8s3H5Wqgk8mAoFQ92wSUQ/rjBwBJk3kmFlS4txzz7WkHq4kcgSYNJFjxiWyHAEmTZYPAO5+5AgwaSLHjEtkOQJMmiwfANz9yBFg0kSOGZfIcgSYNFk+ALj7kSPApIkcMy6R5QgwabJ8AHD3I0eASRM5ZlwiyxFg0mT5AODuR44AkyZyzLhEliPApMnyAcDdjxyBiH8aEHkV1pb4dppON9/ppXdfd9I57dPnmiB/2UxTpur01bc6HTxIRvtDIffbMkGff63TJ59r1Kung+6+3Xwr5/SfdPpueugfrJ1+qo0GFIR/e2eoeuYu0OnN93T64y9Bp5xko5uvdVCTRmX/lmr1WkHPv6LR78sFHVuPqEMbu/wNFslfoRL17x1ZG0K1KxXS0p40QhDZ/Gz5x5+C1sm/zVushTe4HZHWfkASZenvgl5/S6NOHcOT/d9dgn5bptP8hYLyuvtquedhr5HmS/HtPfjfyAbruEk69bnGQ/WOsdHGTYKm/Uj0xtsa/fSNi85uF75tK1YJOr2F28iDOjdvEfTOGJ1mztHpnkGRtcHX+tTbC49A6rW1RIvkK8Ho0jxPwM+wb7zGQRtX5FBBnnVd+3m2Ts++FNtvwavKz9Dce1fZAwuD9poBgfkWLRZUKL/SsXBGDh3YVoEK/zH/fp1p/mDw8i6B+UsA6ZewZy/RHfd4afZUF21YnkObVubIGc3E8r8Pld7H5142z//vKSddcqHZzh+/dlGu/FXkho3y6uYnz7yoEXBLR7FuZCUAnaHPavT1dyWBP6au39STgHr9VW7aLCi3vzeAuP7nI9n3nzFLK+f3WjQjm9sj6NNxLjqrqY0OO4zkSznMP7h6tWraqMVZ5cfjyyk6YZZo09IcGnXr2OiBu02HZM78klj7t3PlapMYf633EQSu2TOPO2nzVl/OqT/odO8jkf1a0lc6+Xtxd8/2HyD6Rg7kCZ9o9OwQJ42XL4bAwD5WTvX33+2g007xGXDrNmH4zYuW6FTpCBu1bWWja69wGEZX0Cz5TdB772uGywUdzc600dX9HQTCPPKkCfxt//HS0bVs9PC9DlqzTtDET2R90p+GD41XiOGK9snnOl3exU6atPvYcbq8MgvpMjipmRxoSvBNpVfe0OiHGTrJt8EaA6dFMxvVqW0zXJXq1VROc/vXBkEdLvbQ39sFTf5Co03SHUH7lc7JX+iENdZ6ma/h6TYqyHUYAztQS+gjDMAPJmo0d76gczvYDfcGbQolrVuEPjF2vCbXMnbDfcXa5JEnNEluc0DndXdQj8vsBl5PPKfJF3kIurKfg87rYKMa1QP1YS1TWa5Jah7lwypUO4AvXLGrb/bI11C56NJOpp4GJ9poyIPmbPe9XHdd1E0u4KRg7fOhdAWfetRJVSqT/Boc0dujNZoj+3zgoKCmjex0w9V2qi1tC9m3n+jTL3WaM0+nO2910FMvaMZYee0Fp9HHb77X6bOvdMK4an6mnW6/2WG02ygcz3+inCJJIKjyoTJz3/OwvOzJfPir37hQXHWTR1x0ubRIUdr6DbqhY+Gvuqhc55BA/r9k2sQi/U3aFoqd/5rV7N4jjDwz52hGwpBnvKLXlW5j/xdZ/oyWhYZenF/6uy4+nKiJZh3MtKHPeo18U6ZqRjtQP3Rf0NUtrrzRbGOt+oXikF+XevR1izYXuMW+/UJ8+qXZX5Tr1N0tVBvMlpn/t/+jixdf8xptuHmQx8iza7d57tpbPaLeaYVixixNrF6ri8t6mRi8Ncpsl78e/33UB1yA3U13esQpZ5n9GTbCV+6HGWbbnnnRl+avA/srVulGu36ebWKHNGAB/dDtLw8O8YiCq01c/dPV/p69csUmyw2616OSQm63bDVtirz4AwbAyF+AhcL/jbe9YvY8TUgiC7lWM2zXuadbLFuhi3m/6AZ+wGL+Ql1g3LS/yMQQabAltqgHZZ983mvYSQYwxJhxpk0wPg4e9K899H5ef4/ofWXpffMvSf4Hpe2XlzTQgUGGzsC4Sm65yxyoAN4j24dOo1NeP7uj4yiHAQgZN8k0MkCDHDggxIDrfZ1TIBoni/5N/cEso0iDZHnFDdCLNOhBXXJRjUOxeKk5yJ7+n69BLc81B+zf2wMNbxQo+vfdNFP3cy/7yk2abKZN+NjXfww8ZeS1f4TXhzaBbOrCgUGGNHWxQLXlIc1TL3gFLgr++KLspXmmbX5d4msD+omLWDh5abjXaPvWbeHzqLK4AJ7dyawD7Uaf/ccB8j38hIn9nPk+fG6720zbtNlXBwgPHRgnGDP4wzH+fl+ui/1yPCxfqRs2RJp/2W4FZhtAoLIkUtIEzsNxmsIQXoQ0OsM3navoyYJFgpatEAS3q20rMxxp5ia66AKzOWPHmQvKdq3N8i3PdRvRG/jyw18s3aOsUEFp820PM19YSQi9KlFt+2eH6a7s2GluV63x+eMd2prt2bVblSrfdvzHZvvh2imBe3Pe2aa+b2VouTQ5/lgbKVew/gmmjvUbSitR8tyHk2RIukcgvsh16w2mm/TWKLONsAME66FQgjXbQ0O89MVEV7GbFCqfSjuuno1++MpF7wx3Gq7R3n2y3108hhum8gRvEXmEC45oHdZQSpTbifGCyBzsjzUaQuBnnGajw+X6De6+whtry7YdPcbf8pVmqHx3hLZTdZe2TQhpQlVYr2hxjsEJ/xoS7KM3LboPAKAxmI+VBhjxskmSmwd5qVFrN637wzeoQ9VT3rTgulsXLXy/+Ma8TwI9MBYGrRq4pen2X8TjHgUkuA4VrvUnZmk6Q+koKz/O414JyNC9a0nzXiwvTBicw0ZqxhoBg/WW600iBetGGLxHXw+99r/I7nmh31h3Ll+QUxxQuFNG5EAOf7EVgYYxAZsHC0jS+SKzD2uL7I4gR7CAULgoIeKn/lYuzCH8qYtEcJlYjkuiGou2Usri3gkEC+KjjjQzIoriL/J934ZBkVatqs1YxGNhvWR2DrWXN8kw2LCI/HeXf6n47OOqNWOKy1jUX9DVQ7fd7aUtckH52XiXcXOurFr8SXNM0dVy3oLAUYLBClEL27J0RnseQQ8MolA3dxHNGiQX0ZBRkjB4CSSCAsGCt6liQd9PBlPKc2P0p5k63fdoYEQMUczhknAQBEvUrKbqUphVrWLigntCuLfjL7hwQmoe5Z8auH/4YTaDdIpY/meDQ93+56Ldt4w0M2aZBOl6iV1OrWa15k1IH0iYXQAcrsi4yoz7SDPukDduaDNurN18ncMAPzj0Ga/X5f4pQ6Ug6ZuvOum+wQ5aPCvHIHlp4CrDI/KjpG2RW4mB5C/zfzGPW7fwuSD+5+O1//4E82kBXIRCyZV9TfwHygtDDzkbVToiMBdmBMwMjU630x0yAqUEkchB9wYSQ52DW/X0/7RiL0Kl+7vEVeS9KIjCTK5RjGO0ExdFyMw5vvGAY3Uvp6FsSzhp2tjE87GntIDZDEuBh4aabmi4stGkh29JNNqCyqh7KJjm3x6tG49h9M13GP7680+YVyCEeJVM/8kE7LmhPkO9Jt0IGAtTPlwLiHLjjqlrlkRoFqFLhJtxcw6y3u9m2ra/Tb27/GYodS9h579mflyRBlzvleWFDHnqNH2GvPchw5vBV0czt+//0TIcDUGoE3lxR3/QrU7D9x71gUYIS0PUYzIIw6q1jXHC79/2f8y8cK+UqFlVzdRI31J0z0Otx1RebIEB2oEbiuHkyBpmaB/ncZEIloef8Bru2wZ5Abvhdq/xd9VNXmrazk3yEzohBWsZSNd8j+EeqkyjPzTti36feLyZByF8yPsTdJo1Vye4xC88abbj5dc1I/SM87iAwkXGWMHNX4wDpMHjOHQIOUy5SobKMbOOkWvhy3p5jPXR4Pu9JCNi8h5Ryf6pclFvy4osqPORRM9y+5uRC4RvEeFCNAhhZ//wI8KM/xtmRsuQ/78PmSHar7/zRVQQ+UD0BToG3+8xIj+SIKpJ4qNPzegKIicImyKcq0K0KPfYU16BaBrqRx5EkxCWfmeMqRdpyI/w5t59orgs0v3/+l/ni9gVV160I+9vFIe5EcpFqBeCqBeiUqgbYXWEuvte4xaIooUSROj8Q/OI7iEa5J+GEDTarvqDPiIM7y+I4qHtZYVaFyzSjfC6f1nsq3Ctf//999f96YtuBZdF/xG1QrvQd2ULhJ5VNBBlED1EHuhFiFlhgpA4bISy9z9mhtsRVsdYQb0qKotywFNFVaET/VG44DzaEuo2AfIGS6TRs4hfgC72hAhPBVE2b4CHPvpUp+1/yMc65CxTuZIvGhSU1Xg4cblcyCHCdXIDm+GWqTy4OsNNw+yxf78IiKyoPLgSY7qvFuYKqPKVtcUVuq983mrUCJe80YcbacJoG2aK6wZ6jUdK1JokWBce59koo0wIdmBG9BdcGXHu1JPtxREx//Px3seiGjcBEYErS5Av2DUrq0xp5zEbImgC9w6z1L9yFj9RHuPGZbBgppBEKmFTuNqwBdp2hox24gmH8grqhb0wHjDzKTewrPJ4lzPWegl7AXpZDfA/rxrvnxa8D1DUHfTgc8onx9SsFovBeVRoNjg9kmNM++df6qH87na/sLQ56GAILKyrVw8/CEEU5Z4E1wuihSNbcN54HMP1OrJG+TTFkzCoUUUZld2Pqxe+HfhulX94WeXE4D315PBYq3yhtqj3hOOiKxtKX7g0c2ER7myU6X9vNwvC9y6vAaOsKi7FcHXD7AIf+8QTNDpdxv5hvF9+FYR7Knj48wj+gl9csM4EJUHORGxdgpuC+ykqUnb9bd5Sb2rFVlv8SuOG6GfjndTlYrvxTBvC2gVXe40banjYMPip4vjVzJrSEYG4zjRwU54b6jSealVgYE2SDoK7z/iDO4bwcdFnL9Oh6dxGixGI+5COt59sMR7G4pEJYzXq6VVfXN2z9Oo6t5YRiA4BJk10uHGpLEaASZPFxueuR4cAkyY63LhUFiPApMli43PXo0OASRMdblwqixFg0mSx8bnr0SHApIkONy6VxQgwabLY+Nz16BBg0kSHG5fKYgSYNFlsfO56dAgwaaLDjUtlMQIRP7A5Tr5mloURyGYEIiKNw2Gj/tfJnzmyJBwB4f1O1lGJbM62Ca+LKyDKK+VFJMH4lPsdAcEF+ThxCMgXP8h3DdhpwoQJlJ+fn7iKWHNUCPCaJirYuFA2I8CkyWbrc9+jQoBJExVsXCibEWDSZLP1ue9RIcCkiQo2LpTNCDBpstn63PeoEGDSRAUbF8pmBJg02Wx97ntUCDBpooKNC2UzAkyabLY+9z0qBJg0UcHGhbIZASZNNluf+x4VAkyaqGD7f3vnHh1Vccfx7737CCKEg8UUebVAPSCWQCIG0EAhpggBiiQxjxNppVpqUVE4FqkSa0l6eqCegxiUSqEHrYQSUKQKyENCSkFCQAgILUJiixZoq6GNFmT3Pjpz0w0k2Ud29ububu/v/pO9M/Obx+e338zs3LkzZGRnAiQaO3uf2i5EgEQjhI2M7EyARGNn71PbhQiQaISwkZGdCZBo7Ox9arsQARKNEDYysjMBEo2dvU9tFyJAohHCRkZ2JkCisbP3qe1CBEg0QtjIyM4ESDR29j61XYgAiUYIGxnZmQCJxs7ep7YLESDRCGEjIzsTINHY2fvUdiECJBohbGRkZwIkGjt7n9ouRIBEI4SNjOxMgERjZ+9T24UI0KFOQtjMNUpJScGVK1daZFpfX48ePXogMTGxOZwf9FReXo7k5OTmMPpgPQESjfXM25RYWlqKkpISeDyeNnHXBnCx1NbWXhtEn6NAgIZnUYDeukh+RGAowbhcLhQVFbU2pfsoEKCeJgrQ/RWZlpaGmpoaf1HNYXV1dRgwYEDzPX2IDgHqaaLDvU2pvBdxOBxtwn0B6enpJBgfjCj/JdFE2QG+4nNzc6Gqqu+2xV8+AVBYWNgijG6iR4BEEz32LUru3bs3MjMzW4T5bjRNAxcVXbFBgEQTG34walFQUNCmNryXycrKQlJSUps4CogOARJNdLj7LdVfb8J7mfz8fL/pKTA6BEg00eHut9Ru3bohJycHTqezOZ5PDvgTU3MC+mA5ARKN5ciDF8if2SiKYiTiz2by8vLQuXPn4EYUaykBEo2luEMXxnuVTp06GQm9Xi/1MqGRWZ6CRGM58uAF8uEYnxBwu93o0qULsrOzgxtQrOUESDSWIw9c4OkzOir3aujVN9tYVjNydAG279JQe1xH4+eB7SjGWgK0jMZa3i1K27Jdw85KDfsO6DhSq7OHm/rV+C/6AJ3WAM6rz24G9pcw6nYZ48dKmDJRxleTpKvp6ZNlBEg0lqFuKugo6zVWv6JibYWGi//SkTpMxtg7Jdw2XMbgQRL6Ma3c0F3C3LmPoqysDP9uBM6d13G6Tge33V+tYVdlk8CmTpIx8z4Hpk+lAYOlbtTpsoTAkWOann+/V0fXL/Vb0zz64qWKXveRFrDsPXv2BIy7fFnXyytUfXKux8gvNd2jr9uoBkxPEeYSgLnZUW7+CDxZ3CSWtHEeveINc7/c7x/V9Bk/aMo/K8ejf3AysBD91Y3CwidA/XoH9usHD7Ph1xgvylZqWP6cE9WVLtw73VzkKcMkvLrSicotLjRcBIaO8mDFav8LPzuwqbbK2lwP2gpd8MauXa9h5HgPknoAxw+48PCswMv+g+fUvthxY2S8964LC+Y5MXuugjnzmx6Qts+aUoVDgCYCwqHVzrTLX1bx6I8VPPYjB55ffHVJTDvNI05WziYZih70ojDXgfLfWF9+xA2I8QxINCY7aMUqFbPnKfjZU048s6Bje5dgVd/FprK/k6/gnilsMw4STjBUYceRaMJGFthg01sasou8KH7SiUVPR08wvhryZ0ATpnkx5yEHli2hHsfHJdK/JJpICf7P/ky9jhFj2Vqxe2SsWh47X1D+POg+NlRbsdSJhx6IvpBNwh3VbEg0JuGfON2LzxqAmiqXSTmal81PnlWx5HnVmJAYMphWEURKlkQTKUFm/+JKFY88oWD/LjdGp8Xml3JUhhc3spm8typiT9QmuMDSLEg0EeL+4j/A12/14PszZCwpiZ1hWetm7a7ScNdUL9avYe/oZNOThtZ8wrkn0YRDy0/aZ3+hYtlLKs6edKNrVz8JYiio6EEFH57WY3IIGUOYQlaF/uWERBQ4gc4WJfOh2eOzHTEvGN6KuQ87cOiIhs1btMCNopiQBEg0IREFTvDKWg2ffqZj1sz4wDgiRcLETBlrWL3pEicQH94Wb1+HWq7fpCJ3mgM39YzNH//+Gl+U58Cbb6v456fXvLvjLyGFBSRAogmIJnhEI3vP5Z2dmvHEPXjK2Ir1vXuzdTuJRtQzJBpBcvy1ZH59+6746WV4fa+/HsgcJ2P3H2iIxnmIXCQaEWrMpvqQjm/eIrNVzPElGt7cO0bKqK4h0Qi6HiQaQXLHT2hIHhp/guHNHcbqfYpt4nHpkmDjbW5GohH8AtR/BHxjQHyK5uaBTfWu/wv9rhFxP4lGhBqzOf93Hb16ChpH2azXTU2iufAPEo2IK0g0ItSYzedsH7LExPjsaXxn3/I20BU+ARJN+MwMC4XtUeaM05X2vnoHOENKkIh9zEg0gr5OSJDwZctTzAVzst7MV+8Et/Vl/z+USKIR9OIN3YGGhvj8TeCrd3fWBrrCJ0CiCZ+ZYdGnl4SP/xafojn7SVOje7M20BU+ARJN+MwMi0E3S/jzh/Epmj+d0thxHhL6f41EI+J+Eo0INWYzPFnC4aPxKRpe71RWf7rECJBoxLgZS1EusGc1x07En3D27tNx52gSjaDraRmNKDi+F0DSjRK27YivNVyn2Jubx09qyBxP/y9FfU/kRMkxu+lsI76Nm+NLNBs2aca5NhMyyPWiridyouSYXeG9Mg69r2Hv/vgRzqvrVBTlkdsjcDsNzyKB9610GXekyXhpVXyIhm/Kzg+HeuC7cbqUIRJnmWhL/3IihDmHbarxu40q3jsY+xMCz72gYkaBA7RhYGROpy2cIuNnWGdM9sLF9uDb/mbsbsS3dLmKeU8pOHHQTaKJ0OfU00QIkJuXFDuwY7eGMnbERixe/CHs/GdULJzvJMGY4CDqaUyAyLMoLlVRukRhp525kXZbbD0DyZjiNd7SPLA7dntCk9xgSTYkGhMxT8r24q9ngap3nGzf5NgQDj8VbfVvNVQzwfBVDHRFToCGZ5EzbM6Bn30J9r3MnaHg0uXm4Kh9WFiiGudvvvZrJwnGRC+QaEyEyXuXja85WW+jg/c60dyQb8FPVfz8lwpeXuY0/XBcE5HFZVYkGpPdNmQQW1rzhosJBhg70Wts9WRyEUGzU9hcxPd+qGDxUsU4XGrWTHomExSYQCSJRgBaKJNbmHCqtjmNpfejMjx44VfWzKpV/VHDCHYE+za28+fbG1z0EDOUowTjSTSC4EKZ8aHa1tddKGbTvI+x48n52Zf7DnTMygF+AtvjCxSMy/KiX1/gEDuNbfLd5NpQPhKNJ7Ki5Nppt2ihA/t2uqEyvaRP8CL/fgW8RzDjOvuxjqcXqeg3xIOK1zXj98vv17uYcGiWzAy+gfKgKedAZDognK8wXrZCNXqc1GEycqbJyJoghzWzde68jh3v6sbO/5u3asaJBY/McuCJOQ64aaOMDvBa2yxJNG2ZdHgIXxW9boPGvviasekgH8qlDpcwmL1C3bePhK90l5CQAPAf9Y2NOs5dAM6whZa1H2jg78PwfmTKJBn52Q4U5dNgocMd1qoAEk0rIFbfHjysY3+1hqPHdONov0/O6Wi4CFzxwNhXLZEdSdiTnX8zsL+EoUMkjLxdwvgxMq67zuqaUnk+Av8F13uFass06m8AAAAASUVORK5CYII=)" + ], + "metadata": { + "id": "tweQCiuX5RVK" + } + } + ] +} \ No newline at end of file From 0b246c4f39466e1351662afc7149def29e12d9dd Mon Sep 17 00:00:00 2001 From: Yi Hu <yathu@google.com> Date: Mon, 23 Oct 2023 15:55:40 -0400 Subject: [PATCH 232/435] Update the version of beam-master container image for python. (#29109) (#28516) Co-authored-by: Shunping Huang <133698626+shunping-google@users.noreply.github.com> --- .github/workflows/README.md | 516 ++++++++++++++++++------------------ 1 file changed, 258 insertions(+), 258 deletions(-) diff --git a/.github/workflows/README.md b/.github/workflows/README.md index 003d9f0561e9..5777a84b2a3f 100644 --- a/.github/workflows/README.md +++ b/.github/workflows/README.md @@ -183,280 +183,280 @@ Please note that jobs with matrix need to have matrix element in the comment. Ex | Workflow name | Matrix | Trigger Phrase | Cron Status | |:-------------:|:------:|:--------------:|:-----------:| -| [ PreCommit Community Metrics ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_CommunityMetrics.yml) | N/A |`Run CommunityMetrics PreCommit`| [![.github/workflows/beam_PreCommit_CommunityMetrics.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_CommunityMetrics.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_CommunityMetrics.yml) | -| [ PreCommit GHA ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_GHA.yml) | N/A |`Run GHA PreCommit`| [![.github/workflows/beam_PreCommit_GHA.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_GHA.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_GHA.yml) | -| [ PreCommit Go ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Go.yml) | N/A |`Run Go PreCommit`| [![.github/workflows/beam_PreCommit_Go.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Go.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Go.yml) | -| [ PreCommit GoPortable ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_GoPortable.yml) | N/A |`Run GoPortable PreCommit`| [![.github/workflows/beam_PreCommit_GoPortable.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_GoPortable.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_GoPortable.yml) | -| [ PreCommit Java ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java.yml) | N/A |`Run Java PreCommit`| [![.github/workflows/beam_PreCommit_Java.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java.yml) | -| [ PreCommit Java Amazon Web Services IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Amazon-Web-Services_IO_Direct.yml) | N/A |`Run Java_Amazon-Web-Services_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Amazon-Web-Services_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Amazon-Web-Services_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Amazon-Web-Services_IO_Direct.yml) | -| [ PreCommit Java Amazon Web Services2 IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Amazon-Web-Services2_IO_Direct.yml) | N/A |`Run Java_Amazon-Web-Services2_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Amazon-Web-Services2_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Amazon-Web-Services2_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Amazon-Web-Services2_IO_Direct.yml) | -| [ PreCommit Java Amqp IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Amqp_IO_Direct.yml) | N/A |`Run Java_Amqp_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Amqp_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Amqp_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Amqp_IO_Direct.yml) | -| [ PreCommit Java Azure IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml) | N/A |`Run Java_Azure_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml) | -| [ PreCommit Java Cassandra IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml) | N/A |`Run Java_Cassandra_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml) | -| [ PreCommit Java Cdap IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml) | N/A |`Run Java_Cdap_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml) | -| [ PreCommit Java Clickhouse IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml) | N/A |`Run Java_Clickhouse_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml) | -| [ PreCommit Java Csv IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml) | N/A |`Run Java_Csv_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml) | -| [ PreCommit Java Debezium IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml) | N/A |`Run Java_Debezium_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml) | -| [ PreCommit Java ElasticSearch IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml) | N/A |`Run Java_ElasticSearch_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml) | -| [ PreCommit Java Examples Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Examples_Dataflow.yml) | N/A |`Run Java_Examples_Dataflow PreCommit`| [![.github/workflows/beam_PreCommit_Java_Examples_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Examples_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Examples_Dataflow.yml) | -| [ PreCommit Java Examples Dataflow Java11 ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml) | N/A | `Run Java_Examples_Dataflow_Java11 PreCommit` | [![.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml) | -| [ PreCommit Java Examples Dataflow Java17 ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml) | N/A | `Run Java_Examples_Dataflow_Java17 PreCommit` | [![.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml) | -| [ PreCommit Java File-schema-transform IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_File-schema-transform_IO_Direct.yml) | N/A |`Run Java_File-schema-transform_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_File-schema-transform_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_File-schema-transform_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_File-schema-transform_IO_Direct.yml) | -| [ PreCommit Java Flink Versions ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Flink_Versions.yml) | N/A |`Run Java_Flink_Versions PreCommit`| [![.github/workflows/beam_PreCommit_Java_Flink_Versions.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Flink_Versions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Flink_Versions.yml) | -| [ PreCommit Java GCP IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml) | N/A |`Run Java_GCP_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml) | -| [ PreCommit Java Google-ads IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Google-ads_IO_Direct.yml) | N/A |`Run Java_Google-ads_IO_Direct PreCommit`| [![.github\workflows\beam_PreCommit_Java_Google-ads_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Google-ads_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Google-ads_IO_Direct.yml) | -| [ PreCommit Java Hadoop IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml) | N/A |`Run Java_Hadoop_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml) | -| [ PreCommit Java HBase IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml) | N/A |`Run Java_HBase_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml) | -| [ PreCommit Java HCatalog IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml) | N/A |`Run Java_HCatalog_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml) | -| [ PreCommit Java Kafka IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml) | N/A |`Run Java_Kafka_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml) | -| [ PreCommit Java InfluxDb IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml) | N/A |`Run Java_InfluxDb_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml) | +| [ PreCommit Community Metrics ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_CommunityMetrics.yml) | N/A |`Run CommunityMetrics PreCommit`| [![.github/workflows/beam_PreCommit_CommunityMetrics.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_CommunityMetrics.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_CommunityMetrics.yml?query=event%3Aschedule) | +| [ PreCommit GHA ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_GHA.yml) | N/A |`Run GHA PreCommit`| [![.github/workflows/beam_PreCommit_GHA.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_GHA.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_GHA.yml?query=event%3Aschedule) | +| [ PreCommit Go ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Go.yml) | N/A |`Run Go PreCommit`| [![.github/workflows/beam_PreCommit_Go.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Go.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Go.yml?query=event%3Aschedule) | +| [ PreCommit GoPortable ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_GoPortable.yml) | N/A |`Run GoPortable PreCommit`| [![.github/workflows/beam_PreCommit_GoPortable.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_GoPortable.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_GoPortable.yml?query=event%3Aschedule) | +| [ PreCommit Java ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java.yml) | N/A |`Run Java PreCommit`| [![.github/workflows/beam_PreCommit_Java.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java.yml?query=event%3Aschedule) | +| [ PreCommit Java Amazon Web Services IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Amazon-Web-Services_IO_Direct.yml) | N/A |`Run Java_Amazon-Web-Services_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Amazon-Web-Services_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Amazon-Web-Services_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Amazon-Web-Services_IO_Direct.yml?query=event%3Aschedule) | +| [ PreCommit Java Amazon Web Services2 IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Amazon-Web-Services2_IO_Direct.yml) | N/A |`Run Java_Amazon-Web-Services2_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Amazon-Web-Services2_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Amazon-Web-Services2_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Amazon-Web-Services2_IO_Direct.yml?query=event%3Aschedule) | +| [ PreCommit Java Amqp IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Amqp_IO_Direct.yml) | N/A |`Run Java_Amqp_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Amqp_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Amqp_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Amqp_IO_Direct.yml?query=event%3Aschedule) | +| [ PreCommit Java Azure IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml) | N/A |`Run Java_Azure_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml?query=event%3Aschedule) | +| [ PreCommit Java Cassandra IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml) | N/A |`Run Java_Cassandra_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml?query=event%3Aschedule) | +| [ PreCommit Java Cdap IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml) | N/A |`Run Java_Cdap_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml?query=event%3Aschedule) | +| [ PreCommit Java Clickhouse IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml) | N/A |`Run Java_Clickhouse_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml?query=event%3Aschedule) | +| [ PreCommit Java Csv IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml) | N/A |`Run Java_Csv_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml?query=event%3Aschedule) | +| [ PreCommit Java Debezium IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml) | N/A |`Run Java_Debezium_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml?query=event%3Aschedule) | +| [ PreCommit Java ElasticSearch IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml) | N/A |`Run Java_ElasticSearch_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml?query=event%3Aschedule) | +| [ PreCommit Java Examples Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Examples_Dataflow.yml) | N/A |`Run Java_Examples_Dataflow PreCommit`| [![.github/workflows/beam_PreCommit_Java_Examples_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Examples_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Examples_Dataflow.yml?query=event%3Aschedule) | +| [ PreCommit Java Examples Dataflow Java11 ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml) | N/A | `Run Java_Examples_Dataflow_Java11 PreCommit` | [![.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml?query=event%3Aschedule) | +| [ PreCommit Java Examples Dataflow Java17 ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml) | N/A | `Run Java_Examples_Dataflow_Java17 PreCommit` | [![.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml?query=event%3Aschedule) | +| [ PreCommit Java File-schema-transform IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_File-schema-transform_IO_Direct.yml) | N/A |`Run Java_File-schema-transform_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_File-schema-transform_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_File-schema-transform_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_File-schema-transform_IO_Direct.yml?query=event%3Aschedule) | +| [ PreCommit Java Flink Versions ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Flink_Versions.yml) | N/A |`Run Java_Flink_Versions PreCommit`| [![.github/workflows/beam_PreCommit_Java_Flink_Versions.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Flink_Versions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Flink_Versions.yml?query=event%3Aschedule) | +| [ PreCommit Java GCP IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml) | N/A |`Run Java_GCP_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml?query=event%3Aschedule) | +| [ PreCommit Java Google-ads IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Google-ads_IO_Direct.yml) | N/A |`Run Java_Google-ads_IO_Direct PreCommit`| [![.github\workflows\beam_PreCommit_Java_Google-ads_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Google-ads_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Google-ads_IO_Direct.yml?query=event%3Aschedule) | +| [ PreCommit Java Hadoop IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml) | N/A |`Run Java_Hadoop_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml?query=event%3Aschedule) | +| [ PreCommit Java HBase IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml) | N/A |`Run Java_HBase_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml?query=event%3Aschedule) | +| [ PreCommit Java HCatalog IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml) | N/A |`Run Java_HCatalog_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml?query=event%3Aschedule) | +| [ PreCommit Java Kafka IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml) | N/A |`Run Java_Kafka_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml?query=event%3Aschedule) | +| [ PreCommit Java InfluxDb IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml) | N/A |`Run Java_InfluxDb_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml?query=event%3Aschedule) | | [ PreCommit Java IOs Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_IOs_Direct.yml) | N/A |`Run Java_IOs_Direct PreCommit`| N/A | -| [ PreCommit Java JDBC IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml) | N/A |`Run Java_JDBC_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml) | -| [ PreCommit Java Jms IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml) | N/A |`Run Java_Jms_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml) | -| [ PreCommit Java Kinesis IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml) | N/A |`Run Java_Kinesis_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml) | -| [ PreCommit Java Kudu IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml) | N/A |`Run Java_Kudu_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml) | -| [ PreCommit Java MongoDb IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml) | N/A |`Run Java_MongoDb_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml) | -| [ PreCommit Java Mqtt IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml) | N/A |`Run Java_Mqtt_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml) | -| [ PreCommit Java Neo4j IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml) | N/A |`Run Java_Neo4j_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml) | -| [ PreCommit Java Parquet IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml) | N/A |`Run Java_Parquet_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml) | -| [ PreCommit Java Pulsar IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml) | N/A |`Run Java_Pulsar_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml) | -| [ PreCommit Java PVR Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_PVR_Flink_Batch.yml) | N/A |`Run Java_PVR_Flink_Batch PreCommit`| [![.github/workflows/beam_PreCommit_Java_PVR_Flink_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_PVR_Flink_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_PVR_Flink_Batch.yml) | -| [ PreCommit Java PVR Flink Docker ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml) | N/A |`Run Java_PVR_Flink_Docker PreCommit`| [![.github/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml) | -| [ PreCommit Java RabbitMq IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml) | N/A |`Run Java_RabbitMq_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml) | -| [ PreCommit Java Redis IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml) | N/A |`Run Java_Redis_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml) | -| [ PreCommit Java RequestResponse IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_RequestResponse_IO_Direct.yml) | N/A |`Run Java_RequestResponse_IO_Direct PreCommit`| [![.github\workflows\beam_RequestResponse_Java_RequestResponse_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_RequestResponse_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_RequestResponse_IO_Direct.yml) | -| [ PreCommit Java SingleStore IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml) | N/A |`Run Java_SingleStore_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml) | -| [ PreCommit Java Snowflake IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml) | N/A |`Run Java_Snowflake_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml) | -| [ PreCommit Java Solr IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml) | N/A |`Run Java_Solr_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml) | -| [ PreCommit Java Spark3 Versions ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Spark3_Versions.yml) | N/A | `Run Java_Spark3_Versions PreCommit` | [![.github/workflows/beam_PreCommit_Java_Spark3_Versions.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Spark3_Versions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Spark3_Versions.yml) | -| [ PreCommit Java Splunk IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml) | N/A |`Run Java_Splunk_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml) | -| [ PreCommit Java Thrift IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml) | N/A |`Run Java_Thrift_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml) | -| [ PreCommit Java Tika IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml) | N/A |`Run Java_Tika_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml) | -| [ PreCommit Kotlin Examples ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Kotlin_Examples.yml) | N/A | `Run Kotlin_Examples PreCommit` | [![.github/workflows/beam_PreCommit_Kotlin_Examples.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Kotlin_Examples.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Kotlin_Examples.yml) | -| [ PreCommit Portable Python ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Portable_Python.yml) | ['3.8','3.11'] | `Run Portable_Python PreCommit` | [![.github/workflows/beam_PreCommit_Portable_Python.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Portable_Python.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Portable_Python.yml) | -| [ PreCommit Python ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python.yml) | ['3.8','3.9','3.10','3.11'] | `Run Python PreCommit (matrix_element)` | [![.github/workflows/beam_PreCommit_Python.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python.yml) | -| [ PreCommit Python Coverage ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Coverage.yml) | N/A | `Run Python_Coverage PreCommit`| [![.github/workflows/beam_PreCommit_Python_Coverage.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Coverage.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Coverage.yml) | -| [ PreCommit Python Dataframes ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Dataframes.yml) | ['3.8','3.9','3.10','3.11'] | `Run Python_Dataframes PreCommit (matrix_element)`| [![.github/workflows/beam_PreCommit_Python_Dataframes.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Dataframes.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Dataframes.yml) | -| [ PreCommit Python Docker ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_PythonDocker.yml) | ['3.8','3.9','3.10','3.11'] | `Run PythonDocker PreCommit (matrix_element)`| [![.github/workflows/beam_PreCommit_PythonDocker.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_PythonDocker.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_PythonDocker.yml) | -| [ PreCommit Python Docs ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_PythonDocs.yml) | N/A | `Run PythonDocs PreCommit`| [![.github/workflows/beam_PreCommit_PythonDocs.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_PythonDocs.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_PythonDocs.yml) | -| [ PreCommit Python Examples ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Examples.yml) | ['3.8','3.9','3.10','3.11'] | `Run Python_Examples PreCommit (matrix_element)` | [![.github/workflows/beam_PreCommit_Python_Examples.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Examples.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Examples.yml) | -| [ PreCommit Python Formatter ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_PythonFormatter.yml) | N/A | `Run PythonFormatter PreCommit`| [![.github/workflows/beam_PreCommit_PythonFormatter.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_PythonFormatter.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_PythonFormatter.yml) | -| [ PreCommit Python Integration](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Integration.yml) | ['3.8','3.11'] | `Run Python_Integration PreCommit (matrix_element)` | [![.github/workflows/beam_PreCommit_Python_Integration.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Integration.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Integration.yml) | -| [ PreCommit Python Lint ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_PythonLint.yml) | N/A | `Run PythonLint PreCommit` | [![.github/workflows/beam_PreCommit_PythonLint.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_PythonLint.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_PythonLint.yml) | -| [ PreCommit Python PVR Flink ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_PVR_Flink.yml) | N/A | `Run Python_PVR_Flink PreCommit` | [![.github/workflows/beam_PreCommit_Python_PVR_Flink.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_PVR_Flink.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_PVR_Flink.yml) | -| [ PreCommit Python Runners ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Runners.yml) | ['3.8','3.9','3.10','3.11'] | `Run Python_Runners PreCommit (matrix_element)`| [![.github/workflows/beam_PreCommit_Python_Runners.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Runners.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Runners.yml) | -| [ PreCommit Python Transforms ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Transforms.yml) | ['3.8','3.9','3.10','3.11'] | `Run Python_Transforms PreCommit (matrix_element)`| [![.github/workflows/beam_PreCommit_Python_Transforms.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Transforms.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Transforms.yml) | -| [ PreCommit RAT ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_RAT.yml) | N/A | `Run RAT PreCommit` | [![.github/workflows/beam_PreCommit_RAT.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_RAT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_RAT.yml) | -| [ PreCommit Spotless ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Spotless.yml) | N/A | `Run Spotless PreCommit` | [![.github/workflows/beam_PreCommit_Spotless.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Spotless.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Spotless.yml) | -| [ PreCommit SQL ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_SQL.yml) | N/A |`Run SQL PreCommit`| [![.github/workflows/beam_PreCommit_SQL.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_SQL.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_SQL.yml) | -| [ PreCommit SQL Java11 ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_SQL_Java11.yml) | N/A |`Run SQL_Java11 PreCommit`| [![.github/workflows/beam_PreCommit_SQL_Java11.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_SQL_Java11.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_SQL_Java11.yml) | -| [ PreCommit SQL Java17 ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_SQL_Java17.yml) | N/A |`Run SQL_Java17 PreCommit`| [![.github/workflows/beam_PreCommit_SQL_Java17.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_SQL_Java17.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_SQL_Java17.yml) | -| [ PreCommit Typescript ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Typescript.yml) | N/A |`Run Typescript PreCommit`| [![.github/workflows/beam_PreCommit_Typescript.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Typescript.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Typescript.yml) | -| [ PreCommit Website ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Website.yml) | N/A |`Run Website PreCommit`| [![.github/workflows/beam_PreCommit_Website.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Website.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Website.yml) | -| [ PreCommit Website Stage GCS ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Website_Stage_GCS.yml) | N/A |`Run Website_Stage_GCS PreCommit`| [![.github/workflows/beam_PreCommit_Website_Stage_GCS.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Website_Stage_GCS.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Website_Stage_GCS.yml) | -| [ PreCommit Whitespace ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Whitespace.yml) | N/A |`Run Whitespace PreCommit`| [![.github/workflows/beam_PreCommit_Whitespace.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Whitespace.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Whitespace.yml) | +| [ PreCommit Java JDBC IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml) | N/A |`Run Java_JDBC_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml?query=event%3Aschedule) | +| [ PreCommit Java Jms IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml) | N/A |`Run Java_Jms_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml?query=event%3Aschedule) | +| [ PreCommit Java Kinesis IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml) | N/A |`Run Java_Kinesis_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml?query=event%3Aschedule) | +| [ PreCommit Java Kudu IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml) | N/A |`Run Java_Kudu_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml?query=event%3Aschedule) | +| [ PreCommit Java MongoDb IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml) | N/A |`Run Java_MongoDb_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml?query=event%3Aschedule) | +| [ PreCommit Java Mqtt IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml) | N/A |`Run Java_Mqtt_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml?query=event%3Aschedule) | +| [ PreCommit Java Neo4j IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml) | N/A |`Run Java_Neo4j_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml?query=event%3Aschedule) | +| [ PreCommit Java Parquet IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml) | N/A |`Run Java_Parquet_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml?query=event%3Aschedule) | +| [ PreCommit Java Pulsar IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml) | N/A |`Run Java_Pulsar_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml?query=event%3Aschedule) | +| [ PreCommit Java PVR Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_PVR_Flink_Batch.yml) | N/A |`Run Java_PVR_Flink_Batch PreCommit`| [![.github/workflows/beam_PreCommit_Java_PVR_Flink_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_PVR_Flink_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_PVR_Flink_Batch.yml?query=event%3Aschedule) | +| [ PreCommit Java PVR Flink Docker ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml) | N/A |`Run Java_PVR_Flink_Docker PreCommit`| [![.github/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml?query=event%3Aschedule) | +| [ PreCommit Java RabbitMq IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml) | N/A |`Run Java_RabbitMq_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml?query=event%3Aschedule) | +| [ PreCommit Java Redis IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml) | N/A |`Run Java_Redis_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml?query=event%3Aschedule) | +| [ PreCommit Java RequestResponse IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_RequestResponse_IO_Direct.yml) | N/A |`Run Java_RequestResponse_IO_Direct PreCommit`| [![.github\workflows\beam_RequestResponse_Java_RequestResponse_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_RequestResponse_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_RequestResponse_IO_Direct.yml?query=event%3Aschedule) | +| [ PreCommit Java SingleStore IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml) | N/A |`Run Java_SingleStore_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml?query=event%3Aschedule) | +| [ PreCommit Java Snowflake IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml) | N/A |`Run Java_Snowflake_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml?query=event%3Aschedule) | +| [ PreCommit Java Solr IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml) | N/A |`Run Java_Solr_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml?query=event%3Aschedule) | +| [ PreCommit Java Spark3 Versions ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Spark3_Versions.yml) | N/A | `Run Java_Spark3_Versions PreCommit` | [![.github/workflows/beam_PreCommit_Java_Spark3_Versions.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Spark3_Versions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Spark3_Versions.yml?query=event%3Aschedule) | +| [ PreCommit Java Splunk IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml) | N/A |`Run Java_Splunk_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml?query=event%3Aschedule) | +| [ PreCommit Java Thrift IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml) | N/A |`Run Java_Thrift_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml?query=event%3Aschedule) | +| [ PreCommit Java Tika IO Direct ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml) | N/A |`Run Java_Tika_IO_Direct PreCommit`| [![.github/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml?query=event%3Aschedule) | +| [ PreCommit Kotlin Examples ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Kotlin_Examples.yml) | N/A | `Run Kotlin_Examples PreCommit` | [![.github/workflows/beam_PreCommit_Kotlin_Examples.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Kotlin_Examples.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Kotlin_Examples.yml?query=event%3Aschedule) | +| [ PreCommit Portable Python ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Portable_Python.yml) | ['3.8','3.11'] | `Run Portable_Python PreCommit` | [![.github/workflows/beam_PreCommit_Portable_Python.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Portable_Python.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Portable_Python.yml?query=event%3Aschedule) | +| [ PreCommit Python ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python.yml) | ['3.8','3.9','3.10','3.11'] | `Run Python PreCommit (matrix_element)` | [![.github/workflows/beam_PreCommit_Python.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python.yml?query=event%3Aschedule) | +| [ PreCommit Python Coverage ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Coverage.yml) | N/A | `Run Python_Coverage PreCommit`| [![.github/workflows/beam_PreCommit_Python_Coverage.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Coverage.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Coverage.yml?query=event%3Aschedule) | +| [ PreCommit Python Dataframes ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Dataframes.yml) | ['3.8','3.9','3.10','3.11'] | `Run Python_Dataframes PreCommit (matrix_element)`| [![.github/workflows/beam_PreCommit_Python_Dataframes.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Dataframes.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Dataframes.yml?query=event%3Aschedule) | +| [ PreCommit Python Docker ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_PythonDocker.yml) | ['3.8','3.9','3.10','3.11'] | `Run PythonDocker PreCommit (matrix_element)`| [![.github/workflows/beam_PreCommit_PythonDocker.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_PythonDocker.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_PythonDocker.yml?query=event%3Aschedule) | +| [ PreCommit Python Docs ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_PythonDocs.yml) | N/A | `Run PythonDocs PreCommit`| [![.github/workflows/beam_PreCommit_PythonDocs.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_PythonDocs.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_PythonDocs.yml?query=event%3Aschedule) | +| [ PreCommit Python Examples ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Examples.yml) | ['3.8','3.9','3.10','3.11'] | `Run Python_Examples PreCommit (matrix_element)` | [![.github/workflows/beam_PreCommit_Python_Examples.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Examples.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Examples.yml?query=event%3Aschedule) | +| [ PreCommit Python Formatter ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_PythonFormatter.yml) | N/A | `Run PythonFormatter PreCommit`| [![.github/workflows/beam_PreCommit_PythonFormatter.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_PythonFormatter.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_PythonFormatter.yml?query=event%3Aschedule) | +| [ PreCommit Python Integration](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Integration.yml) | ['3.8','3.11'] | `Run Python_Integration PreCommit (matrix_element)` | [![.github/workflows/beam_PreCommit_Python_Integration.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Integration.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Integration.yml?query=event%3Aschedule) | +| [ PreCommit Python Lint ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_PythonLint.yml) | N/A | `Run PythonLint PreCommit` | [![.github/workflows/beam_PreCommit_PythonLint.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_PythonLint.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_PythonLint.yml?query=event%3Aschedule) | +| [ PreCommit Python PVR Flink ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_PVR_Flink.yml) | N/A | `Run Python_PVR_Flink PreCommit` | [![.github/workflows/beam_PreCommit_Python_PVR_Flink.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_PVR_Flink.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_PVR_Flink.yml?query=event%3Aschedule) | +| [ PreCommit Python Runners ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Runners.yml) | ['3.8','3.9','3.10','3.11'] | `Run Python_Runners PreCommit (matrix_element)`| [![.github/workflows/beam_PreCommit_Python_Runners.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Runners.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Runners.yml?query=event%3Aschedule) | +| [ PreCommit Python Transforms ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Transforms.yml) | ['3.8','3.9','3.10','3.11'] | `Run Python_Transforms PreCommit (matrix_element)`| [![.github/workflows/beam_PreCommit_Python_Transforms.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Transforms.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_Transforms.yml?query=event%3Aschedule) | +| [ PreCommit RAT ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_RAT.yml) | N/A | `Run RAT PreCommit` | [![.github/workflows/beam_PreCommit_RAT.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_RAT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_RAT.yml?query=event%3Aschedule) | +| [ PreCommit Spotless ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Spotless.yml) | N/A | `Run Spotless PreCommit` | [![.github/workflows/beam_PreCommit_Spotless.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Spotless.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Spotless.yml?query=event%3Aschedule) | +| [ PreCommit SQL ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_SQL.yml) | N/A |`Run SQL PreCommit`| [![.github/workflows/beam_PreCommit_SQL.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_SQL.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_SQL.yml?query=event%3Aschedule) | +| [ PreCommit SQL Java11 ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_SQL_Java11.yml) | N/A |`Run SQL_Java11 PreCommit`| [![.github/workflows/beam_PreCommit_SQL_Java11.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_SQL_Java11.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_SQL_Java11.yml?query=event%3Aschedule) | +| [ PreCommit SQL Java17 ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_SQL_Java17.yml) | N/A |`Run SQL_Java17 PreCommit`| [![.github/workflows/beam_PreCommit_SQL_Java17.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_SQL_Java17.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_SQL_Java17.yml?query=event%3Aschedule) | +| [ PreCommit Typescript ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Typescript.yml) | N/A |`Run Typescript PreCommit`| [![.github/workflows/beam_PreCommit_Typescript.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Typescript.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Typescript.yml?query=event%3Aschedule) | +| [ PreCommit Website ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Website.yml) | N/A |`Run Website PreCommit`| [![.github/workflows/beam_PreCommit_Website.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Website.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Website.yml?query=event%3Aschedule) | +| [ PreCommit Website Stage GCS ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Website_Stage_GCS.yml) | N/A |`Run Website_Stage_GCS PreCommit`| [![.github/workflows/beam_PreCommit_Website_Stage_GCS.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Website_Stage_GCS.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Website_Stage_GCS.yml?query=event%3Aschedule) | +| [ PreCommit Whitespace ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Whitespace.yml) | N/A |`Run Whitespace PreCommit`| [![.github/workflows/beam_PreCommit_Whitespace.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Whitespace.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Whitespace.yml?query=event%3Aschedule) | ### PostCommit Jobs | Workflow name | Matrix | Trigger Phrase | Cron Status | |:-------------:|:------:|:--------------:|:-----------:| -| [ Java JMH ](https://github.com/apache/beam/actions/workflows/beam_Java_JMH.yml) | N/A | N/A | [![.github/workflows/beam_Java_JMH.yml](https://github.com/apache/beam/actions/workflows/beam_Java_JMH.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_Java_JMH.yml) -| [ Publish Beam SDK Snapshots ](https://github.com/apache/beam/actions/workflows/beam_Publish_Beam_SDK_Snapshots.yml) | N/A | N/A | [![.github/workflows/beam_Publish_Beam_SDK_Snapshots.yml](https://github.com/apache/beam/actions/workflows/beam_Publish_Beam_SDK_Snapshots.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_Publish_Beam_SDK_Snapshots.yml) | -| [ PostCommit BeamMetrics Publish ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_BeamMetrics_Publish.yml) | N/A |`Run Beam Metrics Deployment`| [![.github/workflows/beam_PostCommit_BeamMetrics_Publish.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_BeamMetrics_Publish.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_BeamMetrics_Publish.yml) -| [ PostCommit Go ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go.yml) | N/A |`Run Go PostCommit`| [![.github/workflows/beam_PostCommit_Go.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go.yml) | -| [ PostCommit Go Dataflow ARM](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go_Dataflow_ARM.yml) | N/A |`Run Go PostCommit Dataflow ARM`| [![.github/workflows/beam_PostCommit_Go_Dataflow_ARM.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go_Dataflow_ARM.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go_Dataflow_ARM.yml) | -| [ PostCommit Go VR Flink](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go_VR_Flink.yml) | N/A |`Run Go Flink ValidatesRunner`| [![.github/workflows/beam_PostCommit_Go_VR_Flink.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go_VR_Flink.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go_VR_Flink.yml) | -| [ PostCommit Go VR Samza](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go_VR_Samza.yml) | N/A |`Run Go Samza ValidatesRunner`| [![.github/workflows/beam_PostCommit_Go_VR_Samza.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go_VR_Samza.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go_VR_Samza.yml) | -| [ PostCommit Go VR Spark](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go_VR_Spark.yml) | N/A |`Run Go Spark ValidatesRunner`| [![.github/workflows/beam_PostCommit_Go_VR_Spark.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go_VR_Spark.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go_VR_Spark.yml) | -| [ PostCommit Java Avro Versions ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Avro_Versions.yml) | N/A |`Run Java Avro Versions PostCommit`| [![.github/workflows/beam_PostCommit_Java_Avro_Versions.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Avro_Versions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Avro_Versions.yml) | -| [ PostCommit Java Dataflow V1 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_DataflowV1.yml) | N/A |`Run PostCommit_Java_Dataflow`| [![.github/workflows/beam_PostCommit_Java_DataflowV1.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_DataflowV1.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_DataflowV1.yml) | -| [ PostCommit Java Dataflow V2 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_DataflowV2.yml) | N/A |`Run PostCommit_Java_DataflowV2`| [![.github/workflows/beam_PostCommit_Java_DataflowV2.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_DataflowV2.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_DataflowV2.yml) | -| [ PostCommit Java Examples Dataflow ARM ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml) | ['8','11','17'] |`Run Java_Examples_Dataflow_ARM PostCommit (matrix_element)`| [![.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml) | -| [ PostCommit Java Examples Dataflow](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow.yml) | N/A |`Run Java examples on Dataflow`| [![.github/workflows/beam_PostCommit_Java_Examples_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow.yml) | -| [ PostCommit Java Examples Dataflow Java ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml) | ['11','17'] |`Run Java examples on Dataflow Java (matrix_element)`| [![.github/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml) | -| [ PostCommit Java Examples Dataflow V2 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml) | N/A |`Run Java Examples on Dataflow Runner V2`| [![.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml) | -| [ PostCommit Java Examples Dataflow V2 Java ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml) | ['11','17'] |`Run Java (matrix_element) Examples on Dataflow Runner V2`| [![.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml) | -| [ PostCommit Java Examples Direct ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Direct.yml) | N/A |`Run Java Examples_Direct`| [![.github/workflows/beam_PostCommit_Java_Examples_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Direct.yml) | -| [ PostCommit Java Examples Flink ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Flink.yml) | N/A |`Run Java Examples_Flink`| [![.github/workflows/beam_PostCommit_Java_Examples_Flink.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Flink.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Flink.yml) | -| [ PostCommit Java Examples Spark ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Spark.yml) | N/A |`Run Java Examples_Spark`| [![.github/workflows/beam_PostCommit_Java_Examples_Spark.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Spark.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Spark.yml) | -| [ PostCommit Java Hadoop Versions ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Hadoop_Versions.yml) | N/A |`Run PostCommit_Java_Hadoop_Versions`| [![.github/workflows/beam_PostCommit_Java_Hadoop_Versions.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Hadoop_Versions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Hadoop_Versions.yml) | -| [ PostCommit Java InfluxDbIO Integration Test ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml) | N/A |`Run Java InfluxDbIO_IT`| [![.github/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml) -| [ PostCommit Java Jpms Dataflow Java11 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml) | N/A |`Run Jpms Dataflow Java 11 PostCommit`| [![.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml) | -| [ PostCommit Java Jpms Dataflow Java17 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml) | N/A |`Run Jpms Dataflow Java 17 PostCommit`| [![.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml) | -| [ PostCommit Java Jpms Direct Java11 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml) | N/A |`Run Jpms Direct Java 11 PostCommit`| [![.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml) | -| [ PostCommit Java Jpms Direct Java17 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml) | N/A |`Run Jpms Direct Java 17 PostCommit`| [![.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml) | -| [ PostCommit Java Jpms Flink Java11 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml) | N/A |`Run Jpms Flink Java 11 PostCommit`| [![.github/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml) | -| [ PostCommit Java Jpms Spark Java11 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml) | N/A |`Run Jpms Spark Java 11 PostCommit`| [![.github/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml) | -| [ PostCommit Java Nexmark Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml) | N/A |`Run Dataflow Runner Nexmark Tests`| [![.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml) | -| [ PostCommit Java Nexmark Dataflow V2 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml) | N/A |`Run Dataflow Runner V2 Nexmark Tests`| [![.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml) | -| [ PostCommit Java Nexmark Dataflow V2 Java ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml) | ['11','17'] |`Run Dataflow Runner V2 Java (matrix) Nexmark Tests`| [![.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml) | -| [ PostCommit Java Nexmark Direct ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Direct.yml) | N/A |`Run Direct Runner Nexmark Tests`| [![.github/workflows/beam_PostCommit_Java_Nexmark_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Direct.yml) | -| [ PostCommit Java Nexmark Flink ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Flink.yml) | N/A |`Run Flink Runner Nexmark Tests`| [![.github/workflows/beam_PostCommit_Java_Nexmark_Flink.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Flink.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Flink.yml) | -| [ PostCommit Java Nexmark Spark ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Spark.yml) | N/A |`Run Spark Runner Nexmark Tests`| [![.github/workflows/beam_PostCommit_Java_Nexmark_Spark.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Spark.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Spark.yml) | -| [ PostCommit Java PVR Flink Streaming ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml) | N/A |`Run Java Flink PortableValidatesRunner Streaming`| [![.github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml) | -| [ PostCommit Java PVR Samza ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Samza.yml) | N/A |`Run Java Samza PortableValidatesRunner`| [![.github/workflows/beam_PostCommit_Java_PVR_Samza.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Samza.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Samza.yml) | -| [ PostCommit Java PVR Spark3 Streaming ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml) | N/A |`Run Java Spark v3 PortableValidatesRunner Streaming`| [![.github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml) | -| [ PostCommit Java PVR Spark Batch ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml) | N/A |`Run Java Spark PortableValidatesRunner Batch`| [![.github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml) | -| [ PostCommit Java Sickbay ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Sickbay.yml) | N/A |`Run Java Sickbay`| [![.github/workflows/beam_PostCommit_Java_Sickbay.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Sickbay.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Sickbay.yml) | -| [ PostCommit Java Tpcds Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml) | N/A |`Run Dataflow Runner Tpcds Tests`| [![.github/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml) | -| [ PostCommit Java Tpcds Flink ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Tpcds_Flink.yml) | N/A |`Run Flink Runner Tpcds Tests`| [![.github/workflows/beam_PostCommit_Java_Tpcds_Flink.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Tpcds_Flink.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Tpcds_Flink.yml) | -| [ PostCommit Java Tpcds Spark ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Tpcds_Spark.yml) | N/A |`Run Spark Runner Tpcds Tests`| [![.github/workflows/beam_PostCommit_Java_Tpcds_Spark.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Tpcds_Spark.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Tpcds_Spark.yml) | -| [ PostCommit Java ValidatesRunner Dataflow JavaVersions ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml) | ['11','17'] |`Run Dataflow ValidatesRunner Java (matrix_element)`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml) | -| [ PostCommit Java ValidatesRunner Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml) | N/A |`Run Dataflow Streaming ValidatesRunner`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml) | -| [ PostCommit Java ValidatesRunner Dataflow V2 Streaming ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml) | N/A |`Run Java Dataflow V2 ValidatesRunner Streaming`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml) | -| [ PostCommit Java ValidatesRunner Dataflow V2 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml) | N/A |`Run Java Dataflow V2 ValidatesRunner`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml) | -| [ PostCommit Java ValidatesRunner Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml) | N/A |`Run Dataflow ValidatesRunner`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml) | -| [ PostCommit Java ValidatesRunner Direct JavaVersions ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml) | ['11','17'] |`Run Direct ValidatesRunner Java (matrix_element)`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml) | -| [ PostCommit Java ValidatesRunner Direct ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml) | N/A |`Run Direct ValidatesRunner`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml) | -| [ PostCommit Java ValidatesRunner Flink Java11 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml) | N/A |`Run Flink ValidatesRunner Java 11`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml) | -| [ PostCommit Java ValidatesRunner Flink ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml) | N/A |`Run Flink ValidatesRunner`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml) | -| [ PostCommit Java ValidatesRunner Samza ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml) | N/A |`Run Samza ValidatesRunner`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml) | -| [ PostCommit Java ValidatesRunner Spark Java11 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml) | N/A |`Run Spark ValidatesRunner Java 11`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml) | -| [ PostCommit Java ValidatesRunner Spark ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml) | N/A |`Run Spark ValidatesRunner`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml) | -| [ PostCommit Java ValidatesRunner SparkStructuredStreaming ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml) | N/A |`Run Spark StructuredStreaming ValidatesRunner`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml) | -| [ PostCommit Java ValidatesRunner Twister2 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml) | N/A |`Run Twister2 ValidatesRunner`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml) | -| [ PostCommit Java ValidatesRunner ULR ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml) | N/A |`Run ULR Loopback ValidatesRunner`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml) | -| [ PostCommit Java ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java.yml) | N/A |`Run Java PostCommit`| [![.github/workflows/beam_PostCommit_Java.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java.yml) | -| [ PostCommit Javadoc ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Javadoc.yml) | N/A |`Run Javadoc PostCommit`| [![.github/workflows/beam_PostCommit_Javadoc.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Javadoc.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Javadoc.yml) | -| [ PostCommit PortableJar Flink ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_PortableJar_Flink.yml) | N/A |`Run PortableJar_Flink PostCommit`| [![.github/workflows/beam_PostCommit_PortableJar_Flink.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_PortableJar_Flink.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_PortableJar_Flink.yml) | -| [ PostCommit PortableJar Spark ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_PortableJar_Spark.yml) | N/A |`Run PortableJar_Spark PostCommit`| [![.github/workflows/beam_PostCommit_PortableJar_Spark.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_PortableJar_Spark.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_PortableJar_Spark.yml) | -| [ PostCommit Python ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python.yml) | ['3.8','3.9','3.10','3.11'] |`Run Python PostCommit (matrix_element)`| [![.github/workflows/beam_PostCommit_Python.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python.yml) | -| [ PostCommit Python Arm](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Arm.yml) | ['3.8','3.9','3.10','3.11'] |`Run Python PostCommit Arm (matrix_element)`| [![.github/workflows/beam_PostCommit_Python_Arm.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Arm.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Arm.yml) | -| [ PostCommit Python Examples Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Examples_Dataflow.yml) | N/A |`Run Python Examples_Dataflow`| [![.github/workflows/beam_PostCommit_Python_Examples_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Examples_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Examples_Dataflow.yml) | -| [ PostCommit Python Examples Direct ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Examples_Direct.yml) | ['3.8','3.9','3.10','3.11'] |`Run Python Examples_Direct (matrix_element)`| [![.github/workflows/beam_PostCommit_Python_Examples_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Examples_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Examples_Direct.yml) | -| [ PostCommit Python Examples Flink ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Examples_Flink.yml) | ['3.8','3.11'] |`Run Python Examples_Flink (matrix_element)`| [![.github/workflows/beam_PostCommit_Python_Examples_Flink.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Examples_Flink.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Examples_Flink.yml) | -| [ PostCommit Python Examples Spark ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Examples_Spark.yml) | ['3.8','3.11'] |`Run Python Examples_Spark (matrix_element)`| [![.github/workflows/beam_PostCommit_Python_Examples_Spark.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Examples_Spark.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Examples_Spark.yml) | -| [ PostCommit Python MongoDBIO IT ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml) | N/A |`Run Python MongoDBIO_IT`| [![.github/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml) | -| [ PostCommit Python Nexmark Direct ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Nexmark_Direct.yml) | N/A |`Run Python Direct Runner Nexmark Tests`| [![.github/workflows/beam_PostCommit_Python_Nexmark_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Nexmark_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Nexmark_Direct.yml) | -| [ PostCommit Python ValidatesContainer Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml) | ['3.8','3.9','3.10','3.11'] |`Run Python Dataflow ValidatesContainer (matrix_element)`| [![.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml) | -| [ PostCommit Python ValidatesContainer Dataflow With RC ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml) | ['3.8','3.9','3.10','3.11'] |`Run Python RC Dataflow ValidatesContainer (matrix_element)`| [![.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml) | -| [ PostCommit Python ValidatesRunner Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml) | ['3.8','3.11'] |`Run Python Dataflow ValidatesRunner (matrix_element)`| [![.github/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml) | -| [ PostCommit Python ValidatesRunner Flink ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml) | ['3.8','3.11'] |`Run Python Flink ValidatesRunner (matrix_element)`| [![.github/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml) | -| [ PostCommit Python ValidatesRunner Samza ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml) | ['3.8','3.11'] |`Run Python Samza ValidatesRunner (matrix_element)`| [![.github/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml) | -| [ PostCommit Python ValidatesRunner Spark ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml) | ['3.8','3.9','3.11'] |`Run Python Spark ValidatesRunner (matrix_element)`| [![.github/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml) | -| [ PostCommit Python Xlang Gcp Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml) | N/A |`Run Python_Xlang_Gcp_Dataflow PostCommit`| [![.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml) | -| [ PostCommit Python Xlang Gcp Direct ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml) | N/A |`Run Python_Xlang_Gcp_Direct PostCommit`| [![.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml) | -| [ PostCommit Python Xlang IO Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml) | N/A |`Run Python_Xlang_IO_Dataflow PostCommit`| [![.github/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml) | -| [ PostCommit Sickbay Python ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Sickbay_Python.yml) | ['3.8','3.9','3.10','3.11'] |`Run Python (matrix_element) PostCommit Sickbay`| [![.github/workflows/beam_PostCommit_Sickbay_Python.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Sickbay_Python.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Sickbay_Python.yml) | -| [ PostCommit SQL ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_SQL.yml) | N/A |`Run SQL PostCommit`| [![.github/workflows/beam_PostCommit_SQL.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_SQL.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_SQL.yml) | -| [ PostCommit TransformService Direct ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_TransformService_Direct.yml) | N/A |`Run TransformService_Direct PostCommit`| [![.github/workflows/beam_PostCommit_TransformService_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_TransformService_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_TransformService_Direct.yml) -| [ PostCommit Website Publish ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Website_Publish.yml) | N/A | N/A | [![.github/workflows/beam_PostCommit_Website_Publish.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Website_Publish.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Website_Publish.yml) | -| [ PostCommit Website Test](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Website_Test.yml) | N/A |`Run Full Website Test`| [![.github/workflows/beam_PostCommit_Website_Test.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Website_Test.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Website_Test.yml) | -| [ PostCommit XVR GoUsingJava Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml) | N/A |`Run XVR_GoUsingJava_Dataflow PostCommit`| [![.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml) | -| [ PostCommit XVR Direct ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Direct.yml) | N/A |`Run XVR_Direct PostCommit`| [![.github/workflows/beam_PostCommit_XVR_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Direct.yml) | -| [ PostCommit XVR Flink ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Flink.yml) | N/A |`Run XVR_Flink PostCommit`| [![.github/workflows/beam_PostCommit_XVR_Flink.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Flink.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Flink.yml) | -| [ PostCommit XVR JavaUsingPython Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml) | N/A |`Run XVR_JavaUsingPython_Dataflow PostCommit`| [![.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml) | -| [ PostCommit XVR PythonUsingJava Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml) | N/A |`Run XVR_PythonUsingJava_Dataflow PostCommit`| [![.github/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml) | -| [ PostCommit XVR PythonUsingJavaSQL Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml) | N/A |`Run XVR_PythonUsingJavaSQL_Dataflow PostCommit`| [![.github/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml) | -| [ PostCommit XVR Samza ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Samza.yml) | N/A |`Run XVR_Samza PostCommit`| [![.github/workflows/beam_PostCommit_XVR_Samza.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Samza.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Samza.yml) | -| [ PostCommit XVR Spark3 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Spark3.yml) | N/A |`Run XVR_Spark3 PostCommit`| [![.github/workflows/beam_PostCommit_XVR_Spark3.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Spark3.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Spark3.yml) | -| [ Python Validates Container Dataflow ARM ](https://github.com/apache/beam/actions/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml) | ['3.8','3.9','3.10','3.11'] |`Run Python ValidatesContainer Dataflow ARM (matrix_element)`|[![.github/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml](https://github.com/apache/beam/actions/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml) | - -### PerformanceTests Jobs +| [ PostCommit BeamMetrics Publish ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_BeamMetrics_Publish.yml) | N/A |`Run Beam Metrics Deployment`| [![.github/workflows/beam_PostCommit_BeamMetrics_Publish.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_BeamMetrics_Publish.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_BeamMetrics_Publish.yml?query=event%3Aschedule) +| [ PostCommit Go ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go.yml) | N/A |`Run Go PostCommit`| [![.github/workflows/beam_PostCommit_Go.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go.yml?query=event%3Aschedule) | +| [ PostCommit Go Dataflow ARM](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go_Dataflow_ARM.yml) | N/A |`Run Go PostCommit Dataflow ARM`| [![.github/workflows/beam_PostCommit_Go_Dataflow_ARM.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go_Dataflow_ARM.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go_Dataflow_ARM.yml?query=event%3Aschedule) | +| [ PostCommit Go VR Flink](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go_VR_Flink.yml) | N/A |`Run Go Flink ValidatesRunner`| [![.github/workflows/beam_PostCommit_Go_VR_Flink.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go_VR_Flink.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go_VR_Flink.yml?query=event%3Aschedule) | +| [ PostCommit Go VR Samza](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go_VR_Samza.yml) | N/A |`Run Go Samza ValidatesRunner`| [![.github/workflows/beam_PostCommit_Go_VR_Samza.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go_VR_Samza.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go_VR_Samza.yml?query=event%3Aschedule) | +| [ PostCommit Go VR Spark](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go_VR_Spark.yml) | N/A |`Run Go Spark ValidatesRunner`| [![.github/workflows/beam_PostCommit_Go_VR_Spark.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go_VR_Spark.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go_VR_Spark.yml?query=event%3Aschedule) | +| [ PostCommit Java Avro Versions ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Avro_Versions.yml) | N/A |`Run Java Avro Versions PostCommit`| [![.github/workflows/beam_PostCommit_Java_Avro_Versions.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Avro_Versions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Avro_Versions.yml?query=event%3Aschedule) | +| [ PostCommit Java Dataflow V1 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_DataflowV1.yml) | N/A |`Run PostCommit_Java_Dataflow`| [![.github/workflows/beam_PostCommit_Java_DataflowV1.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_DataflowV1.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_DataflowV1.yml?query=event%3Aschedule) | +| [ PostCommit Java Dataflow V2 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_DataflowV2.yml) | N/A |`Run PostCommit_Java_DataflowV2`| [![.github/workflows/beam_PostCommit_Java_DataflowV2.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_DataflowV2.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_DataflowV2.yml?query=event%3Aschedule) | +| [ PostCommit Java Examples Dataflow ARM ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml) | ['8','11','17'] |`Run Java_Examples_Dataflow_ARM PostCommit (matrix_element)`| [![.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml?query=event%3Aschedule) | +| [ PostCommit Java Examples Dataflow](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow.yml) | N/A |`Run Java examples on Dataflow`| [![.github/workflows/beam_PostCommit_Java_Examples_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow.yml?query=event%3Aschedule) | +| [ PostCommit Java Examples Dataflow Java ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml) | ['11','17'] |`Run Java examples on Dataflow Java (matrix_element)`| [![.github/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml?query=event%3Aschedule) | +| [ PostCommit Java Examples Dataflow V2 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml) | N/A |`Run Java Examples on Dataflow Runner V2`| [![.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml?query=event%3Aschedule) | +| [ PostCommit Java Examples Dataflow V2 Java ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml) | ['11','17'] |`Run Java (matrix_element) Examples on Dataflow Runner V2`| [![.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml?query=event%3Aschedule) | +| [ PostCommit Java Examples Direct ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Direct.yml) | N/A |`Run Java Examples_Direct`| [![.github/workflows/beam_PostCommit_Java_Examples_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Direct.yml?query=event%3Aschedule) | +| [ PostCommit Java Examples Flink ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Flink.yml) | N/A |`Run Java Examples_Flink`| [![.github/workflows/beam_PostCommit_Java_Examples_Flink.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Flink.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Flink.yml?query=event%3Aschedule) | +| [ PostCommit Java Examples Spark ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Spark.yml) | N/A |`Run Java Examples_Spark`| [![.github/workflows/beam_PostCommit_Java_Examples_Spark.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Spark.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Spark.yml?query=event%3Aschedule) | +| [ PostCommit Java Hadoop Versions ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Hadoop_Versions.yml) | N/A |`Run PostCommit_Java_Hadoop_Versions`| [![.github/workflows/beam_PostCommit_Java_Hadoop_Versions.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Hadoop_Versions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Hadoop_Versions.yml?query=event%3Aschedule) | +| [ PostCommit Java InfluxDbIO Integration Test ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml) | N/A |`Run Java InfluxDbIO_IT`| [![.github/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml?query=event%3Aschedule) +| [ PostCommit Java Jpms Dataflow Java11 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml) | N/A |`Run Jpms Dataflow Java 11 PostCommit`| [![.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml?query=event%3Aschedule) | +| [ PostCommit Java Jpms Dataflow Java17 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml) | N/A |`Run Jpms Dataflow Java 17 PostCommit`| [![.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml?query=event%3Aschedule) | +| [ PostCommit Java Jpms Direct Java11 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml) | N/A |`Run Jpms Direct Java 11 PostCommit`| [![.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml?query=event%3Aschedule) | +| [ PostCommit Java Jpms Direct Java17 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml) | N/A |`Run Jpms Direct Java 17 PostCommit`| [![.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml?query=event%3Aschedule) | +| [ PostCommit Java Jpms Flink Java11 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml) | N/A |`Run Jpms Flink Java 11 PostCommit`| [![.github/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml?query=event%3Aschedule) | +| [ PostCommit Java Jpms Spark Java11 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml) | N/A |`Run Jpms Spark Java 11 PostCommit`| [![.github/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml?query=event%3Aschedule) | +| [ PostCommit Java Nexmark Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml) | N/A |`Run Dataflow Runner Nexmark Tests`| [![.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml?query=event%3Aschedule) | +| [ PostCommit Java Nexmark Dataflow V2 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml) | N/A |`Run Dataflow Runner V2 Nexmark Tests`| [![.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml?query=event%3Aschedule) | +| [ PostCommit Java Nexmark Dataflow V2 Java ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml) | ['11','17'] |`Run Dataflow Runner V2 Java (matrix) Nexmark Tests`| [![.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml?query=event%3Aschedule) | +| [ PostCommit Java Nexmark Direct ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Direct.yml) | N/A |`Run Direct Runner Nexmark Tests`| [![.github/workflows/beam_PostCommit_Java_Nexmark_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Direct.yml?query=event%3Aschedule) | +| [ PostCommit Java Nexmark Flink ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Flink.yml) | N/A |`Run Flink Runner Nexmark Tests`| [![.github/workflows/beam_PostCommit_Java_Nexmark_Flink.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Flink.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Flink.yml?query=event%3Aschedule) | +| [ PostCommit Java Nexmark Spark ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Spark.yml) | N/A |`Run Spark Runner Nexmark Tests`| [![.github/workflows/beam_PostCommit_Java_Nexmark_Spark.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Spark.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Spark.yml?query=event%3Aschedule) | +| [ PostCommit Java PVR Flink Streaming ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml) | N/A |`Run Java Flink PortableValidatesRunner Streaming`| [![.github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml?query=event%3Aschedule) | +| [ PostCommit Java PVR Samza ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Samza.yml) | N/A |`Run Java Samza PortableValidatesRunner`| [![.github/workflows/beam_PostCommit_Java_PVR_Samza.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Samza.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Samza.yml?query=event%3Aschedule) | +| [ PostCommit Java PVR Spark3 Streaming ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml) | N/A |`Run Java Spark v3 PortableValidatesRunner Streaming`| [![.github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml?query=event%3Aschedule) | +| [ PostCommit Java PVR Spark Batch ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml) | N/A |`Run Java Spark PortableValidatesRunner Batch`| [![.github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml?query=event%3Aschedule) | +| [ PostCommit Java Sickbay ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Sickbay.yml) | N/A |`Run Java Sickbay`| [![.github/workflows/beam_PostCommit_Java_Sickbay.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Sickbay.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Sickbay.yml?query=event%3Aschedule) | +| [ PostCommit Java Tpcds Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml) | N/A |`Run Dataflow Runner Tpcds Tests`| [![.github/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml?query=event%3Aschedule) | +| [ PostCommit Java Tpcds Flink ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Tpcds_Flink.yml) | N/A |`Run Flink Runner Tpcds Tests`| [![.github/workflows/beam_PostCommit_Java_Tpcds_Flink.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Tpcds_Flink.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Tpcds_Flink.yml?query=event%3Aschedule) | +| [ PostCommit Java Tpcds Spark ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Tpcds_Spark.yml) | N/A |`Run Spark Runner Tpcds Tests`| [![.github/workflows/beam_PostCommit_Java_Tpcds_Spark.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Tpcds_Spark.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Tpcds_Spark.yml?query=event%3Aschedule) | +| [ PostCommit Java ValidatesRunner Dataflow JavaVersions ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml) | ['11','17'] |`Run Dataflow ValidatesRunner Java (matrix_element)`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml?query=event%3Aschedule) | +| [ PostCommit Java ValidatesRunner Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml) | N/A |`Run Dataflow Streaming ValidatesRunner`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml?query=event%3Aschedule) | +| [ PostCommit Java ValidatesRunner Dataflow V2 Streaming ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml) | N/A |`Run Java Dataflow V2 ValidatesRunner Streaming`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml?query=event%3Aschedule) | +| [ PostCommit Java ValidatesRunner Dataflow V2 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml) | N/A |`Run Java Dataflow V2 ValidatesRunner`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml?query=event%3Aschedule) | +| [ PostCommit Java ValidatesRunner Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml) | N/A |`Run Dataflow ValidatesRunner`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml?query=event%3Aschedule) | +| [ PostCommit Java ValidatesRunner Direct JavaVersions ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml) | ['11','17'] |`Run Direct ValidatesRunner Java (matrix_element)`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml?query=event%3Aschedule) | +| [ PostCommit Java ValidatesRunner Direct ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml) | N/A |`Run Direct ValidatesRunner`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml?query=event%3Aschedule) | +| [ PostCommit Java ValidatesRunner Flink Java11 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml) | N/A |`Run Flink ValidatesRunner Java 11`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml?query=event%3Aschedule) | +| [ PostCommit Java ValidatesRunner Flink ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml) | N/A |`Run Flink ValidatesRunner`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml?query=event%3Aschedule) | +| [ PostCommit Java ValidatesRunner Samza ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml) | N/A |`Run Samza ValidatesRunner`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml?query=event%3Aschedule) | +| [ PostCommit Java ValidatesRunner Spark Java11 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml) | N/A |`Run Spark ValidatesRunner Java 11`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml?query=event%3Aschedule) | +| [ PostCommit Java ValidatesRunner Spark ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml) | N/A |`Run Spark ValidatesRunner`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml?query=event%3Aschedule) | +| [ PostCommit Java ValidatesRunner SparkStructuredStreaming ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml) | N/A |`Run Spark StructuredStreaming ValidatesRunner`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml?query=event%3Aschedule) | +| [ PostCommit Java ValidatesRunner Twister2 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml) | N/A |`Run Twister2 ValidatesRunner`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml?query=event%3Aschedule) | +| [ PostCommit Java ValidatesRunner ULR ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml) | N/A |`Run ULR Loopback ValidatesRunner`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml?query=event%3Aschedule) | +| [ PostCommit Java ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java.yml) | N/A |`Run Java PostCommit`| [![.github/workflows/beam_PostCommit_Java.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java.yml?query=event%3Aschedule) | +| [ PostCommit Javadoc ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Javadoc.yml) | N/A |`Run Javadoc PostCommit`| [![.github/workflows/beam_PostCommit_Javadoc.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Javadoc.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Javadoc.yml?query=event%3Aschedule) | +| [ PostCommit PortableJar Flink ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_PortableJar_Flink.yml) | N/A |`Run PortableJar_Flink PostCommit`| [![.github/workflows/beam_PostCommit_PortableJar_Flink.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_PortableJar_Flink.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_PortableJar_Flink.yml?query=event%3Aschedule) | +| [ PostCommit PortableJar Spark ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_PortableJar_Spark.yml) | N/A |`Run PortableJar_Spark PostCommit`| [![.github/workflows/beam_PostCommit_PortableJar_Spark.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_PortableJar_Spark.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_PortableJar_Spark.yml?query=event%3Aschedule) | +| [ PostCommit Python ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python.yml) | ['3.8','3.9','3.10','3.11'] |`Run Python PostCommit (matrix_element)`| [![.github/workflows/beam_PostCommit_Python.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python.yml?query=event%3Aschedule) | +| [ PostCommit Python Arm](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Arm.yml) | ['3.8','3.9','3.10','3.11'] |`Run Python PostCommit Arm (matrix_element)`| [![.github/workflows/beam_PostCommit_Python_Arm.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Arm.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Arm.yml?query=event%3Aschedule) | +| [ PostCommit Python Examples Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Examples_Dataflow.yml) | N/A |`Run Python Examples_Dataflow`| [![.github/workflows/beam_PostCommit_Python_Examples_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Examples_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Examples_Dataflow.yml?query=event%3Aschedule) | +| [ PostCommit Python Examples Direct ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Examples_Direct.yml) | ['3.8','3.9','3.10','3.11'] |`Run Python Examples_Direct (matrix_element)`| [![.github/workflows/beam_PostCommit_Python_Examples_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Examples_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Examples_Direct.yml?query=event%3Aschedule) | +| [ PostCommit Python Examples Flink ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Examples_Flink.yml) | ['3.8','3.11'] |`Run Python Examples_Flink (matrix_element)`| [![.github/workflows/beam_PostCommit_Python_Examples_Flink.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Examples_Flink.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Examples_Flink.yml?query=event%3Aschedule) | +| [ PostCommit Python Examples Spark ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Examples_Spark.yml) | ['3.8','3.11'] |`Run Python Examples_Spark (matrix_element)`| [![.github/workflows/beam_PostCommit_Python_Examples_Spark.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Examples_Spark.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Examples_Spark.yml?query=event%3Aschedule) | +| [ PostCommit Python MongoDBIO IT ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml) | N/A |`Run Python MongoDBIO_IT`| [![.github/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml?query=event%3Aschedule) | +| [ PostCommit Python Nexmark Direct ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Nexmark_Direct.yml) | N/A |`Run Python Direct Runner Nexmark Tests`| [![.github/workflows/beam_PostCommit_Python_Nexmark_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Nexmark_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Nexmark_Direct.yml?query=event%3Aschedule) | +| [ PostCommit Python ValidatesContainer Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml) | ['3.8','3.9','3.10','3.11'] |`Run Python Dataflow ValidatesContainer (matrix_element)`| [![.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml?query=event%3Aschedule) | +| [ PostCommit Python ValidatesContainer Dataflow With RC ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml) | ['3.8','3.9','3.10','3.11'] |`Run Python RC Dataflow ValidatesContainer (matrix_element)`| [![.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml?query=event%3Aschedule) | +| [ PostCommit Python ValidatesRunner Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml) | ['3.8','3.11'] |`Run Python Dataflow ValidatesRunner (matrix_element)`| [![.github/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml?query=event%3Aschedule) | +| [ PostCommit Python ValidatesRunner Flink ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml) | ['3.8','3.11'] |`Run Python Flink ValidatesRunner (matrix_element)`| [![.github/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml?query=event%3Aschedule) | +| [ PostCommit Python ValidatesRunner Samza ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml) | ['3.8','3.11'] |`Run Python Samza ValidatesRunner (matrix_element)`| [![.github/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml?query=event%3Aschedule) | +| [ PostCommit Python ValidatesRunner Spark ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml) | ['3.8','3.9','3.11'] |`Run Python Spark ValidatesRunner (matrix_element)`| [![.github/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml?query=event%3Aschedule) | +| [ PostCommit Python Xlang Gcp Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml) | N/A |`Run Python_Xlang_Gcp_Dataflow PostCommit`| [![.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml?query=event%3Aschedule) | +| [ PostCommit Python Xlang Gcp Direct ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml) | N/A |`Run Python_Xlang_Gcp_Direct PostCommit`| [![.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml?query=event%3Aschedule) | +| [ PostCommit Python Xlang IO Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml) | N/A |`Run Python_Xlang_IO_Dataflow PostCommit`| [![.github/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml?query=event%3Aschedule) | +| [ PostCommit Sickbay Python ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Sickbay_Python.yml) | ['3.8','3.9','3.10','3.11'] |`Run Python (matrix_element) PostCommit Sickbay`| [![.github/workflows/beam_PostCommit_Sickbay_Python.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Sickbay_Python.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Sickbay_Python.yml?query=event%3Aschedule) | +| [ PostCommit SQL ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_SQL.yml) | N/A |`Run SQL PostCommit`| [![.github/workflows/beam_PostCommit_SQL.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_SQL.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_SQL.yml?query=event%3Aschedule) | +| [ PostCommit TransformService Direct ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_TransformService_Direct.yml) | N/A |`Run TransformService_Direct PostCommit`| [![.github/workflows/beam_PostCommit_TransformService_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_TransformService_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_TransformService_Direct.yml?query=event%3Aschedule) +| [ PostCommit Website Publish ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Website_Publish.yml) | N/A | N/A | [![.github/workflows/beam_PostCommit_Website_Publish.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Website_Publish.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Website_Publish.yml?query=event%3Aschedule) | +| [ PostCommit Website Test](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Website_Test.yml) | N/A |`Run Full Website Test`| [![.github/workflows/beam_PostCommit_Website_Test.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Website_Test.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Website_Test.yml?query=event%3Aschedule) | +| [ PostCommit XVR GoUsingJava Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml) | N/A |`Run XVR_GoUsingJava_Dataflow PostCommit`| [![.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml?query=event%3Aschedule) | +| [ PostCommit XVR Direct ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Direct.yml) | N/A |`Run XVR_Direct PostCommit`| [![.github/workflows/beam_PostCommit_XVR_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Direct.yml?query=event%3Aschedule) | +| [ PostCommit XVR Flink ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Flink.yml) | N/A |`Run XVR_Flink PostCommit`| [![.github/workflows/beam_PostCommit_XVR_Flink.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Flink.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Flink.yml?query=event%3Aschedule) | +| [ PostCommit XVR JavaUsingPython Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml) | N/A |`Run XVR_JavaUsingPython_Dataflow PostCommit`| [![.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml?query=event%3Aschedule) | +| [ PostCommit XVR PythonUsingJava Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml) | N/A |`Run XVR_PythonUsingJava_Dataflow PostCommit`| [![.github/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml?query=event%3Aschedule) | +| [ PostCommit XVR PythonUsingJavaSQL Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml) | N/A |`Run XVR_PythonUsingJavaSQL_Dataflow PostCommit`| [![.github/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml?query=event%3Aschedule) | +| [ PostCommit XVR Samza ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Samza.yml) | N/A |`Run XVR_Samza PostCommit`| [![.github/workflows/beam_PostCommit_XVR_Samza.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Samza.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Samza.yml?query=event%3Aschedule) | +| [ PostCommit XVR Spark3 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Spark3.yml) | N/A |`Run XVR_Spark3 PostCommit`| [![.github/workflows/beam_PostCommit_XVR_Spark3.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Spark3.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Spark3.yml?query=event%3Aschedule) | +| [ Python Validates Container Dataflow ARM ](https://github.com/apache/beam/actions/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml) | ['3.8','3.9','3.10','3.11'] |`Run Python ValidatesContainer Dataflow ARM (matrix_element)`|[![.github/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml](https://github.com/apache/beam/actions/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml?query=event%3Aschedule) | + +### PerformanceTests and Benchmark Jobs | Workflow name | Matrix | Trigger Phrase | Cron Status | |:-------------:|:------:|:--------------:|:-----------:| -| [ CloudML Benchmarks Dataflow ](https://github.com/apache/beam/actions/workflows/beam_CloudML_Benchmarks_Dataflow.yml) | N/A |`Run TFT Criteo Benchmarks`| [![.github/workflows/beam_CloudML_Benchmarks_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_CloudML_Benchmarks_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_CloudML_Benchmarks_Dataflow.yml) -| [ Inference Python Benchmarks Dataflow ](https://github.com/apache/beam/actions/workflows/beam_Inference_Python_Benchmarks_Dataflow.yml) | N/A |`Run Inference Benchmarks`| [![.github/workflows/beam_Inference_Python_Benchmarks_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_Inference_Python_Benchmarks_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_Inference_Python_Benchmarks_Dataflow.yml) -| [ Performance Tests AvroIOIT HDFS ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml) | N/A |`Run Java AvroIO Performance Test HDFS`| [![.github/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml) -| [ Performance Tests AvroIOIT ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_AvroIOIT.yml) | N/A |`Run Java AvroIO Performance Test`| [![.github/workflows/beam_PerformanceTests_AvroIOIT.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_AvroIOIT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_AvroIOIT.yml) -| [ Performance Tests BigQueryIO Batch Java Avro ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml) | N/A |`Run BigQueryIO Batch Performance Test Java Avro`| [![.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml) -| [ Performance Tests BigQueryIO Batch Java Json ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml) | N/A |`Run BigQueryIO Batch Performance Test Java Json`| [![.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml) -| [ Performance Tests BigQueryIO Streaming Java ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml) | N/A |`Run BigQueryIO Streaming Performance Test Java`| [![.github/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml) -| [ Performance Tests BigQueryIO Read Python ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml) | N/A |`Run BigQueryIO Read Performance Test Python`| [![.github/workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml) -| [ Performance Tests BigQueryIO Write Python Batch ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml) | N/A |`Run BigQueryIO Write Performance Test Python`| [![.github/workflows/beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml) -| [ PerformanceTests Cdap ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Cdap.yml) | N/A |`Run Java CdapIO Performance Test`| [![.github/workflows/beam_PerformanceTests_Cdap.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Cdap.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Cdap.yml) -| [ PerformanceTests Compressed TextIOIT HDFS ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml) | N/A |`Run Java CompressedTextIO Performance Test HDFS`| [![.github/workflows/beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml) -| [ PerformanceTests Compressed TextIOIT ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml) | N/A |`Run Java CompressedTextIO Performance Test`| [![.github/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml) -| [ PerformanceTests HadoopFormat ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_HadoopFormat.yml) | N/A |`Run Java HadoopFormatIO Performance Test`| [![.github/workflows/beam_PerformanceTests_HadoopFormat.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_HadoopFormat.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_HadoopFormat.yml) -| [ PerformanceTests JDBC ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_JDBC.yml) | N/A |`Run Java JdbcIO Performance Test`| [![.github/workflows/beam_PerformanceTests_JDBC.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_JDBC.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_JDBC.yml) -| [ PerformanceTests Kafka IO ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Kafka_IO.yml) | N/A |`Run Java KafkaIO Performance Test`| [![.github/workflows/beam_PerformanceTests_Kafka_IO.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Kafka_IO.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Kafka_IO.yml) -| [ PerformanceTests ManyFiles TextIOIT HDFS ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml) | N/A |`Run Java ManyFilesTextIO Performance Test HDFS`| [![.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml) -| [ PerformanceTests ManyFiles TextIOIT ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml) | N/A |`Run Java ManyFilesTextIO Performance Test`| [![.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml) -| [ PerformanceTests MongoDBIO IT ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_MongoDBIO_IT.yml) | N/A |`Run Java MongoDBIO Performance Test`| [![.github/workflows/beam_PerformanceTests_MongoDBIO_IT.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_MongoDBIO_IT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_MongoDBIO_IT.yml) -| [ PerformanceTests ParquetIOIT HDFS ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml) | N/A |`Run Java ParquetIO Performance Test HDFS`| [![.github/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml) -| [ PerformanceTests ParquetIOIT ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ParquetIOIT.yml) | N/A |`Run Java ParquetIO Performance Test`| [![.github/workflows/beam_PerformanceTests_ParquetIOIT.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ParquetIOIT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ParquetIOIT.yml) -| [ PerformanceTests PubsubIOIT Python Streaming ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml) | N/A |`Run PubsubIO Performance Test Python`| [![.github/workflows/beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml) -| [ PerformanceTests SingleStoreIO ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SingleStoreIO.yml) | N/A |`Run Java SingleStoreIO Performance Test`| [![.github/workflows/beam_PerformanceTests_SingleStoreIO.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SingleStoreIO.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SingleStoreIO.yml) -| [ PerformanceTests SpannerIO Read 2GB Python ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml) | N/A |`Run SpannerIO Read 2GB Performance Test Python`| [![.github/workflows/beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml) -| [ PerformanceTests SpannerIO Write 2GB Python Batch ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml) | N/A |`Run SpannerIO Write 2GB Performance Test Python Batch`| [![.github/workflows/beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml) -| [ PerformanceTests SparkReceiver IO ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SparkReceiver_IO.yml) | N/A |`Run Java SparkReceiverIO Performance Test`| [![.github/workflows/beam_PerformanceTests_SparkReceiver_IO.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SparkReceiver_IO.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SparkReceiver_IO.yml) -| [ PerformanceTests SQLBigQueryIO Batch Java ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml) | N/A |`Run SQLBigQueryIO Batch Performance Test Java`| [![.github/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml) -| [ PerformanceTests TextIOIT HDFS ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml) | N/A |`Run Java TextIO Performance Test HDFS`| [![.github/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml) -| [ PerformanceTests TextIOIT Python ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TextIOIT_Python.yml) | N/A |`Run Python TextIO Performance Test`| [![.github/workflows/beam_PerformanceTests_TextIOIT_Python.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TextIOIT_Python.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TextIOIT_Python.yml) -| [ PerformanceTests TextIOIT ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TextIOIT.yml) | N/A |`Run Java TextIO Performance Test`| [![.github/workflows/beam_PerformanceTests_TextIOIT.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TextIOIT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TextIOIT.yml) -| [ PerformanceTests TFRecordIOIT HDFS ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml) | N/A |`Run Java TFRecordIO Performance Test HDFS`| [![.github/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml) -| [ PerformanceTests TFRecordIOIT ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TFRecordIOIT.yml) | N/A |`Run Java TFRecordIO Performance Test`| [![.github/workflows/beam_PerformanceTests_TFRecordIOIT.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TFRecordIOIT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TFRecordIOIT.yml) -| [ PerformanceTests WordCountIT PythonVersions ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml) | ['3.8'] |`Run Python (matrix_element) WordCountIT Performance Test`| [![.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml) -| [ PerformanceTests XmlIOIT HDFS ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml) | N/A |`Run Java XmlIO Performance Test HDFS`| [![.github/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml) -| [ PerformanceTests XmlIOIT ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_XmlIOIT.yml) | N/A |`Run Java XmlIO Performance Test`| [![.github/workflows/beam_PerformanceTests_XmlIOIT.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_XmlIOIT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_XmlIOIT.yml) -| [ PerformanceTests xlang KafkaIO Python ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml) | N/A |`Run Python xlang KafkaIO Performance Test`| [![.github/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml) +| [ CloudML Benchmarks Dataflow ](https://github.com/apache/beam/actions/workflows/beam_CloudML_Benchmarks_Dataflow.yml) | N/A |`Run TFT Criteo Benchmarks`| [![.github/workflows/beam_CloudML_Benchmarks_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_CloudML_Benchmarks_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_CloudML_Benchmarks_Dataflow.yml?query=event%3Aschedule) +| [ Inference Python Benchmarks Dataflow ](https://github.com/apache/beam/actions/workflows/beam_Inference_Python_Benchmarks_Dataflow.yml) | N/A |`Run Inference Benchmarks`| [![.github/workflows/beam_Inference_Python_Benchmarks_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_Inference_Python_Benchmarks_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_Inference_Python_Benchmarks_Dataflow.yml?query=event%3Aschedule) +| [ Java JMH ](https://github.com/apache/beam/actions/workflows/beam_Java_JMH.yml) | N/A | N/A | [![.github/workflows/beam_Java_JMH.yml](https://github.com/apache/beam/actions/workflows/beam_Java_JMH.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_Java_JMH.yml?query=event%3Aschedule) +| [ Performance Tests AvroIOIT HDFS ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml) | N/A |`Run Java AvroIO Performance Test HDFS`| [![.github/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml?query=event%3Aschedule) +| [ Performance Tests AvroIOIT ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_AvroIOIT.yml) | N/A |`Run Java AvroIO Performance Test`| [![.github/workflows/beam_PerformanceTests_AvroIOIT.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_AvroIOIT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_AvroIOIT.yml?query=event%3Aschedule) +| [ Performance Tests BigQueryIO Batch Java Avro ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml) | N/A |`Run BigQueryIO Batch Performance Test Java Avro`| [![.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml?query=event%3Aschedule) +| [ Performance Tests BigQueryIO Batch Java Json ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml) | N/A |`Run BigQueryIO Batch Performance Test Java Json`| [![.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml?query=event%3Aschedule) +| [ Performance Tests BigQueryIO Streaming Java ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml) | N/A |`Run BigQueryIO Streaming Performance Test Java`| [![.github/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml?query=event%3Aschedule) +| [ Performance Tests BigQueryIO Read Python ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml) | N/A |`Run BigQueryIO Read Performance Test Python`| [![.github/workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml?query=event%3Aschedule) +| [ Performance Tests BigQueryIO Write Python Batch ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml) | N/A |`Run BigQueryIO Write Performance Test Python`| [![.github/workflows/beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml?query=event%3Aschedule) +| [ PerformanceTests Cdap ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Cdap.yml) | N/A |`Run Java CdapIO Performance Test`| [![.github/workflows/beam_PerformanceTests_Cdap.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Cdap.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Cdap.yml?query=event%3Aschedule) +| [ PerformanceTests Compressed TextIOIT HDFS ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml) | N/A |`Run Java CompressedTextIO Performance Test HDFS`| [![.github/workflows/beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml?query=event%3Aschedule) +| [ PerformanceTests Compressed TextIOIT ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml) | N/A |`Run Java CompressedTextIO Performance Test`| [![.github/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml?query=event%3Aschedule) +| [ PerformanceTests HadoopFormat ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_HadoopFormat.yml) | N/A |`Run Java HadoopFormatIO Performance Test`| [![.github/workflows/beam_PerformanceTests_HadoopFormat.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_HadoopFormat.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_HadoopFormat.yml?query=event%3Aschedule) +| [ PerformanceTests JDBC ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_JDBC.yml) | N/A |`Run Java JdbcIO Performance Test`| [![.github/workflows/beam_PerformanceTests_JDBC.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_JDBC.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_JDBC.yml?query=event%3Aschedule) +| [ PerformanceTests Kafka IO ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Kafka_IO.yml) | N/A |`Run Java KafkaIO Performance Test`| [![.github/workflows/beam_PerformanceTests_Kafka_IO.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Kafka_IO.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Kafka_IO.yml?query=event%3Aschedule) +| [ PerformanceTests ManyFiles TextIOIT HDFS ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml) | N/A |`Run Java ManyFilesTextIO Performance Test HDFS`| [![.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml?query=event%3Aschedule) +| [ PerformanceTests ManyFiles TextIOIT ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml) | N/A |`Run Java ManyFilesTextIO Performance Test`| [![.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml?query=event%3Aschedule) +| [ PerformanceTests MongoDBIO IT ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_MongoDBIO_IT.yml) | N/A |`Run Java MongoDBIO Performance Test`| [![.github/workflows/beam_PerformanceTests_MongoDBIO_IT.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_MongoDBIO_IT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_MongoDBIO_IT.yml?query=event%3Aschedule) +| [ PerformanceTests ParquetIOIT HDFS ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml) | N/A |`Run Java ParquetIO Performance Test HDFS`| [![.github/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml?query=event%3Aschedule) +| [ PerformanceTests ParquetIOIT ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ParquetIOIT.yml) | N/A |`Run Java ParquetIO Performance Test`| [![.github/workflows/beam_PerformanceTests_ParquetIOIT.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ParquetIOIT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ParquetIOIT.yml?query=event%3Aschedule) +| [ PerformanceTests PubsubIOIT Python Streaming ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml) | N/A |`Run PubsubIO Performance Test Python`| [![.github/workflows/beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml?query=event%3Aschedule) +| [ PerformanceTests SingleStoreIO ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SingleStoreIO.yml) | N/A |`Run Java SingleStoreIO Performance Test`| [![.github/workflows/beam_PerformanceTests_SingleStoreIO.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SingleStoreIO.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SingleStoreIO.yml?query=event%3Aschedule) +| [ PerformanceTests SpannerIO Read 2GB Python ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml) | N/A |`Run SpannerIO Read 2GB Performance Test Python`| [![.github/workflows/beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml?query=event%3Aschedule) +| [ PerformanceTests SpannerIO Write 2GB Python Batch ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml) | N/A |`Run SpannerIO Write 2GB Performance Test Python Batch`| [![.github/workflows/beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml?query=event%3Aschedule) +| [ PerformanceTests SparkReceiver IO ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SparkReceiver_IO.yml) | N/A |`Run Java SparkReceiverIO Performance Test`| [![.github/workflows/beam_PerformanceTests_SparkReceiver_IO.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SparkReceiver_IO.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SparkReceiver_IO.yml?query=event%3Aschedule) +| [ PerformanceTests SQLBigQueryIO Batch Java ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml) | N/A |`Run SQLBigQueryIO Batch Performance Test Java`| [![.github/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml?query=event%3Aschedule) +| [ PerformanceTests TextIOIT HDFS ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml) | N/A |`Run Java TextIO Performance Test HDFS`| [![.github/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml?query=event%3Aschedule) +| [ PerformanceTests TextIOIT Python ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TextIOIT_Python.yml) | N/A |`Run Python TextIO Performance Test`| [![.github/workflows/beam_PerformanceTests_TextIOIT_Python.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TextIOIT_Python.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TextIOIT_Python.yml?query=event%3Aschedule) +| [ PerformanceTests TextIOIT ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TextIOIT.yml) | N/A |`Run Java TextIO Performance Test`| [![.github/workflows/beam_PerformanceTests_TextIOIT.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TextIOIT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TextIOIT.yml?query=event%3Aschedule) +| [ PerformanceTests TFRecordIOIT HDFS ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml) | N/A |`Run Java TFRecordIO Performance Test HDFS`| [![.github/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml?query=event%3Aschedule) +| [ PerformanceTests TFRecordIOIT ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TFRecordIOIT.yml) | N/A |`Run Java TFRecordIO Performance Test`| [![.github/workflows/beam_PerformanceTests_TFRecordIOIT.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TFRecordIOIT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TFRecordIOIT.yml?query=event%3Aschedule) +| [ PerformanceTests WordCountIT PythonVersions ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml) | ['3.8'] |`Run Python (matrix_element) WordCountIT Performance Test`| [![.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml?query=event%3Aschedule) +| [ PerformanceTests XmlIOIT HDFS ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml) | N/A |`Run Java XmlIO Performance Test HDFS`| [![.github/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml?query=event%3Aschedule) +| [ PerformanceTests XmlIOIT ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_XmlIOIT.yml) | N/A |`Run Java XmlIO Performance Test`| [![.github/workflows/beam_PerformanceTests_XmlIOIT.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_XmlIOIT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_XmlIOIT.yml?query=event%3Aschedule) +| [ PerformanceTests xlang KafkaIO Python ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml) | N/A |`Run Python xlang KafkaIO Performance Test`| [![.github/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml?query=event%3Aschedule) ### LoadTests Jobs | Workflow name | Matrix | Trigger Phrase | Cron Status | |:-------------:|:------:|:--------------:|:-----------:| -| [ LoadTests Go CoGBK Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml) | N/A |`Run LoadTests Go CoGBK Dataflow Batch`| [![.github/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml) -| [ LoadTests Go CoGBK Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_CoGBK_Flink_batch.yml) | N/A |`Run Load Tests Go CoGBK Flink Batch`| [![.github/workflows/beam_LoadTests_Go_CoGBK_Flink_batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_CoGBK_Flink_batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_CoGBK_Flink_batch.yml) -| [ LoadTests Go Combine Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_Combine_Dataflow_Batch.yml) | N/A |`Run Load Tests Go Combine Dataflow Batch`| [![.github/workflows/beam_LoadTests_Go_Combine_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_Combine_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_Combine_Dataflow_Batch.yml) -| [ LoadTests Go Combine Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml) | N/A |`Run Load Tests Go Combine Flink Batch`| [![.github/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml) -| [ LoadTests Go GBK Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml) | N/A |`Run Load Tests Go GBK Dataflow Batch`| [![.github/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml) -| [ LoadTests Go GBK Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml) | N/A |`Run Load Tests Go GBK Flink Batch`| [![.github/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml) -| [ LoadTests Go ParDo Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml) | N/A |`Run Load Tests Go ParDo Dataflow Batch`| [![.github/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml) -| [ LoadTests Go ParDo Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml) | N/A |`Run Load Tests Go ParDo Flink Batch`| [![.github/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml) -| [ LoadTests Go SideInput Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml) | N/A |`Run Load Tests Go SideInput Dataflow Batch`| [![.github/workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml) -| [ LoadTests Go SideInput Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml) | N/A |`Run Load Tests Go SideInput Flink Batch`| [![.github/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml) -| [ LoadTests Java CoGBK Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml) | N/A |`Run Load Tests Java CoGBK Dataflow Batch`| [![.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml) -| [ LoadTests Java CoGBK Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml) | N/A |`Run Load Tests Java CoGBK Dataflow Streaming`| [![.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml) -| [ LoadTests Java CoGBK Dataflow V2 Batch JavaVersions ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml) | ['11','17'] |`Run Load Tests Java (matrix_element) CoGBK Dataflow V2 Batch`| [![.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml) -| [ LoadTests Java CoGBK Dataflow V2 Streaming JavaVersions ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml) | ['11','17'] |`Run Load Tests Java (matrix_element) CoGBK Dataflow V2 Streaming`| [![.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml) -| [ LoadTests Java CoGBK SparkStructuredStreaming Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml) | N/A |`Run Load Tests Java CoGBK SparkStructuredStreaming Batch`| [![.github/workflows/beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml) -| [ LoadTests Java Combine Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml) | N/A |`Run Load Tests Java Combine Dataflow Batch`| [![.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml) -| [ LoadTests Java Combine Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_Combine_Dataflow_Streaming.yml) | N/A |`Run Load Tests Java Combine Dataflow Streaming`| [![.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_Combine_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_Combine_Dataflow_Streaming.yml) -| [ LoadTests Java Combine SparkStructuredStreaming Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml) | N/A |`Run Load Tests Java Combine SparkStructuredStreaming Batch`| [![.github/workflows/beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml) -| [ LoadTests Java GBK Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml) | N/A |`Run Load Tests Java GBK Dataflow Batch`| [![.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml) -| [ LoadTests Java GBK Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml) | N/A |`Run Load Tests Java GBK Dataflow Streaming`| [![.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml) -| [ LoadTests Java GBK Dataflow V2 Batch Java11 ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml) | N/A |`Run Load Tests Java 11 GBK Dataflow V2 Batch`| [![.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml) -| [ LoadTests Java GBK Dataflow V2 Batch Java17 ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml) | N/A |`Run Load Tests Java 17 GBK Dataflow V2 Batch`| [![.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml) -| [ LoadTests Java GBK Dataflow V2 Streaming Java11 ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml) | N/A |`Run Load Tests Java 11 GBK Dataflow V2 Streaming`| [![.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml) -| [ LoadTests Java GBK Dataflow V2 Streaming Java17 ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml) | N/A |`Run Load Tests Java 17 GBK Dataflow V2 Streaming`| [![.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml) -| [ LoadTests Java GBK Smoke ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Smoke.yml) | N/A |`Run Java Load Tests GBK Smoke`| [![.github/workflows/beam_LoadTests_Java_GBK_Smoke.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Smoke.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Smoke.yml) -| [ LoadTests Java GBK SparkStructuredStreaming Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch.yml) | N/A |`Run Load Tests Java GBK SparkStructuredStreaming Batch`| [![.github/workflows/beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch.yml) -| [ LoadTests Java ParDo Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml) | N/A |`Run Load Tests Java ParDo Dataflow Batch`| [![.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml) -| [ LoadTests Java ParDo Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml) | N/A |`Run Load Tests Java ParDo Dataflow Streaming`| [![.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml) -| [ LoadTests Java ParDo Dataflow V2 Batch JavaVersions ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions.yml) | ['11','17'] |`Run Load Tests Java (matrix_element) ParDo Dataflow V2 Batch`| [![.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions.yml) -| [ LoadTests Java ParDo Dataflow V2 Streaming JavaVersions ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions.yml) | ['11','17'] |`Run Load Tests Java (matrix_element) ParDo Dataflow V2 Streaming`| [![.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions.yml) -| [ LoadTests Java ParDo SparkStructuredStreaming Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml) | N/A |`Run Load Tests Java ParDo SparkStructuredStreaming Batch`| [![.github/workflows/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml) -| [ LoadTests Java Combine Smoke ](https://github.com/apache/beam/actions/workflows/beam_Java_LoadTests_Combine_Smoke.yml) | N/A | N/A | [![.github/workflows/beam_Java_LoadTests_Combine_Smoke.yml](https://github.com/apache/beam/actions/workflows/beam_Java_LoadTests_Combine_Smoke.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_Java_LoadTests_Combine_Smoke.yml) -| [ LoadTests Python CoGBK Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml) | N/A |`Run Load Tests Python CoGBK Dataflow Batch`| [![.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml) -| [ LoadTests Python CoGBK Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml) | N/A |`Run Load Tests Python CoGBK Dataflow Streaming`| [![.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml) -| [ LoadTests Python CoGBK Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml) | N/A |`Run Load Tests Python CoGBK Flink Batch`| [![.github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml) -| [ LoadTests Python Combine Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml) | N/A |`Run Load Tests Python Combine Dataflow Batch`| [![.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml) -| [ LoadTests Python Combine Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Dataflow_Streaming.yml) | N/A |`Run Load Tests Python Combine Dataflow Streaming`| [![.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Dataflow_Streaming.yml) -| [ LoadTests Python Combine Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml) | N/A |`Run Load Tests Python Combine Flink Batch`| [![.github/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml) -| [ LoadTests Python Combine Flink Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml) | N/A |`Run Load Tests Python Combine Flink Streaming`| [![.github/workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml) -| [ LoadTests Python FnApiRunner Microbenchmark ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_FnApiRunner_Microbenchmark.yml) | N/A |`Run Python Load Tests FnApiRunner Microbenchmark`| [![.github/workflows/beam_LoadTests_Python_FnApiRunner_Microbenchmark](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_FnApiRunner_Microbenchmark.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_FnApiRunner_Microbenchmark.yml) -| [ LoadTests Python GBK Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml) | N/A |`Run Load Tests Python GBK Dataflow Batch`| [![.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml) -| [ LoadTests Python GBK Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml) | N/A |`Run Load Tests Python GBK Dataflow Streaming`| [![.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml) -| [ LoadTests Python GBK Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_Flink_Batch.yml) | N/A |`Run Load Tests Python GBK Flink Batch`| [![.github/workflows/beam_LoadTests_Python_GBK_Flink_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_Flink_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_Flink_Batch.yml) -| [ LoadTests Python GBK reiterate Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Batch.yml) | N/A |`Run Load Tests Python GBK reiterate Dataflow Batch`| [![.github/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Batch.yml) -| [ LoadTests Python GBK reiterate Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Streaming.yml) | N/A |`Run Load Tests Python GBK reiterate Dataflow Streaming`| [![.github/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Streaming.yml) -| [ LoadTests Python ParDo Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Dataflow_Batch.yml) | N/A |`Run Load Tests Python ParDo Dataflow Batch`| [![.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Dataflow_Batch.yml) -| [ LoadTests Python ParDo Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml) | N/A |`Run Python Load Tests ParDo Dataflow Streaming`| [![.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml) -| [ LoadTests Python ParDo Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml) | N/A |`Run Load Tests Python ParDo Flink Batch`| [![.github/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml) -| [ LoadTests Python ParDo Flink Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml) | N/A |`Run Load Tests Python ParDo Flink Streaming`| [![.github/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml) -| [ LoadTests Python SideInput Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_SideInput_Dataflow_Batch.yml) | N/A |`Run Load Tests Python SideInput Dataflow Batch`| [![.github/workflows/beam_LoadTests_Python_SideInput_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_SideInput_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_SideInput_Dataflow_Batch.yml) -| [ LoadTests Python Smoke ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Smoke.yml) | N/A |`Run Python Load Tests Smoke`| [![.github/workflows/beam_LoadTests_Python_Smoke.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Smoke.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Smoke.yml) +| [ LoadTests Go CoGBK Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml) | N/A |`Run LoadTests Go CoGBK Dataflow Batch`| [![.github/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml?query=event%3Aschedule) +| [ LoadTests Go CoGBK Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_CoGBK_Flink_batch.yml) | N/A |`Run Load Tests Go CoGBK Flink Batch`| [![.github/workflows/beam_LoadTests_Go_CoGBK_Flink_batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_CoGBK_Flink_batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_CoGBK_Flink_batch.yml?query=event%3Aschedule) +| [ LoadTests Go Combine Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_Combine_Dataflow_Batch.yml) | N/A |`Run Load Tests Go Combine Dataflow Batch`| [![.github/workflows/beam_LoadTests_Go_Combine_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_Combine_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_Combine_Dataflow_Batch.yml?query=event%3Aschedule) +| [ LoadTests Go Combine Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml) | N/A |`Run Load Tests Go Combine Flink Batch`| [![.github/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml?query=event%3Aschedule) +| [ LoadTests Go GBK Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml) | N/A |`Run Load Tests Go GBK Dataflow Batch`| [![.github/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml?query=event%3Aschedule) +| [ LoadTests Go GBK Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml) | N/A |`Run Load Tests Go GBK Flink Batch`| [![.github/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml?query=event%3Aschedule) +| [ LoadTests Go ParDo Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml) | N/A |`Run Load Tests Go ParDo Dataflow Batch`| [![.github/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml?query=event%3Aschedule) +| [ LoadTests Go ParDo Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml) | N/A |`Run Load Tests Go ParDo Flink Batch`| [![.github/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml?query=event%3Aschedule) +| [ LoadTests Go SideInput Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml) | N/A |`Run Load Tests Go SideInput Dataflow Batch`| [![.github/workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml?query=event%3Aschedule) +| [ LoadTests Go SideInput Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml) | N/A |`Run Load Tests Go SideInput Flink Batch`| [![.github/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml?query=event%3Aschedule) +| [ LoadTests Java CoGBK Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml) | N/A |`Run Load Tests Java CoGBK Dataflow Batch`| [![.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml?query=event%3Aschedule) +| [ LoadTests Java CoGBK Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml) | N/A |`Run Load Tests Java CoGBK Dataflow Streaming`| [![.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml?query=event%3Aschedule) +| [ LoadTests Java CoGBK Dataflow V2 Batch JavaVersions ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml) | ['11','17'] |`Run Load Tests Java (matrix_element) CoGBK Dataflow V2 Batch`| [![.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml?query=event%3Aschedule) +| [ LoadTests Java CoGBK Dataflow V2 Streaming JavaVersions ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml) | ['11','17'] |`Run Load Tests Java (matrix_element) CoGBK Dataflow V2 Streaming`| [![.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml?query=event%3Aschedule) +| [ LoadTests Java CoGBK SparkStructuredStreaming Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml) | N/A |`Run Load Tests Java CoGBK SparkStructuredStreaming Batch`| [![.github/workflows/beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml?query=event%3Aschedule) +| [ LoadTests Java Combine Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml) | N/A |`Run Load Tests Java Combine Dataflow Batch`| [![.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml?query=event%3Aschedule) +| [ LoadTests Java Combine Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_Combine_Dataflow_Streaming.yml) | N/A |`Run Load Tests Java Combine Dataflow Streaming`| [![.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_Combine_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_Combine_Dataflow_Streaming.yml?query=event%3Aschedule) +| [ LoadTests Java Combine SparkStructuredStreaming Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml) | N/A |`Run Load Tests Java Combine SparkStructuredStreaming Batch`| [![.github/workflows/beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml?query=event%3Aschedule) +| [ LoadTests Java GBK Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml) | N/A |`Run Load Tests Java GBK Dataflow Batch`| [![.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml?query=event%3Aschedule) +| [ LoadTests Java GBK Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml) | N/A |`Run Load Tests Java GBK Dataflow Streaming`| [![.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml?query=event%3Aschedule) +| [ LoadTests Java GBK Dataflow V2 Batch Java11 ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml) | N/A |`Run Load Tests Java 11 GBK Dataflow V2 Batch`| [![.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml?query=event%3Aschedule) +| [ LoadTests Java GBK Dataflow V2 Batch Java17 ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml) | N/A |`Run Load Tests Java 17 GBK Dataflow V2 Batch`| [![.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml?query=event%3Aschedule) +| [ LoadTests Java GBK Dataflow V2 Streaming Java11 ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml) | N/A |`Run Load Tests Java 11 GBK Dataflow V2 Streaming`| [![.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml?query=event%3Aschedule) +| [ LoadTests Java GBK Dataflow V2 Streaming Java17 ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml) | N/A |`Run Load Tests Java 17 GBK Dataflow V2 Streaming`| [![.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml?query=event%3Aschedule) +| [ LoadTests Java GBK Smoke ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Smoke.yml) | N/A |`Run Java Load Tests GBK Smoke`| [![.github/workflows/beam_LoadTests_Java_GBK_Smoke.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Smoke.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Smoke.yml?query=event%3Aschedule) +| [ LoadTests Java GBK SparkStructuredStreaming Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch.yml) | N/A |`Run Load Tests Java GBK SparkStructuredStreaming Batch`| [![.github/workflows/beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch.yml?query=event%3Aschedule) +| [ LoadTests Java ParDo Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml) | N/A |`Run Load Tests Java ParDo Dataflow Batch`| [![.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml?query=event%3Aschedule) +| [ LoadTests Java ParDo Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml) | N/A |`Run Load Tests Java ParDo Dataflow Streaming`| [![.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml?query=event%3Aschedule) +| [ LoadTests Java ParDo Dataflow V2 Batch JavaVersions ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions.yml) | ['11','17'] |`Run Load Tests Java (matrix_element) ParDo Dataflow V2 Batch`| [![.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions.yml?query=event%3Aschedule) +| [ LoadTests Java ParDo Dataflow V2 Streaming JavaVersions ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions.yml) | ['11','17'] |`Run Load Tests Java (matrix_element) ParDo Dataflow V2 Streaming`| [![.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions.yml?query=event%3Aschedule) +| [ LoadTests Java ParDo SparkStructuredStreaming Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml) | N/A |`Run Load Tests Java ParDo SparkStructuredStreaming Batch`| [![.github/workflows/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml?query=event%3Aschedule) +| [ LoadTests Java Combine Smoke ](https://github.com/apache/beam/actions/workflows/beam_Java_LoadTests_Combine_Smoke.yml) | N/A | N/A | [![.github/workflows/beam_Java_LoadTests_Combine_Smoke.yml](https://github.com/apache/beam/actions/workflows/beam_Java_LoadTests_Combine_Smoke.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_Java_LoadTests_Combine_Smoke.yml?query=event%3Aschedule) +| [ LoadTests Python CoGBK Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml) | N/A |`Run Load Tests Python CoGBK Dataflow Batch`| [![.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml?query=event%3Aschedule) +| [ LoadTests Python CoGBK Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml) | N/A |`Run Load Tests Python CoGBK Dataflow Streaming`| [![.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml?query=event%3Aschedule) +| [ LoadTests Python CoGBK Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml) | N/A |`Run Load Tests Python CoGBK Flink Batch`| [![.github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml?query=event%3Aschedule) +| [ LoadTests Python Combine Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml) | N/A |`Run Load Tests Python Combine Dataflow Batch`| [![.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml?query=event%3Aschedule) +| [ LoadTests Python Combine Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Dataflow_Streaming.yml) | N/A |`Run Load Tests Python Combine Dataflow Streaming`| [![.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Dataflow_Streaming.yml?query=event%3Aschedule) +| [ LoadTests Python Combine Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml) | N/A |`Run Load Tests Python Combine Flink Batch`| [![.github/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml?query=event%3Aschedule) +| [ LoadTests Python Combine Flink Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml) | N/A |`Run Load Tests Python Combine Flink Streaming`| [![.github/workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml?query=event%3Aschedule) +| [ LoadTests Python FnApiRunner Microbenchmark ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_FnApiRunner_Microbenchmark.yml) | N/A |`Run Python Load Tests FnApiRunner Microbenchmark`| [![.github/workflows/beam_LoadTests_Python_FnApiRunner_Microbenchmark](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_FnApiRunner_Microbenchmark.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_FnApiRunner_Microbenchmark.yml?query=event%3Aschedule) +| [ LoadTests Python GBK Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml) | N/A |`Run Load Tests Python GBK Dataflow Batch`| [![.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml?query=event%3Aschedule) +| [ LoadTests Python GBK Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml) | N/A |`Run Load Tests Python GBK Dataflow Streaming`| [![.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml?query=event%3Aschedule) +| [ LoadTests Python GBK Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_Flink_Batch.yml) | N/A |`Run Load Tests Python GBK Flink Batch`| [![.github/workflows/beam_LoadTests_Python_GBK_Flink_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_Flink_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_Flink_Batch.yml?query=event%3Aschedule) +| [ LoadTests Python GBK reiterate Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Batch.yml) | N/A |`Run Load Tests Python GBK reiterate Dataflow Batch`| [![.github/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Batch.yml?query=event%3Aschedule) +| [ LoadTests Python GBK reiterate Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Streaming.yml) | N/A |`Run Load Tests Python GBK reiterate Dataflow Streaming`| [![.github/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Streaming.yml?query=event%3Aschedule) +| [ LoadTests Python ParDo Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Dataflow_Batch.yml) | N/A |`Run Load Tests Python ParDo Dataflow Batch`| [![.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Dataflow_Batch.yml?query=event%3Aschedule) +| [ LoadTests Python ParDo Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml) | N/A |`Run Python Load Tests ParDo Dataflow Streaming`| [![.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml?query=event%3Aschedule) +| [ LoadTests Python ParDo Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml) | N/A |`Run Load Tests Python ParDo Flink Batch`| [![.github/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml?query=event%3Aschedule) +| [ LoadTests Python ParDo Flink Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml) | N/A |`Run Load Tests Python ParDo Flink Streaming`| [![.github/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml?query=event%3Aschedule) +| [ LoadTests Python SideInput Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_SideInput_Dataflow_Batch.yml) | N/A |`Run Load Tests Python SideInput Dataflow Batch`| [![.github/workflows/beam_LoadTests_Python_SideInput_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_SideInput_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_SideInput_Dataflow_Batch.yml?query=event%3Aschedule) +| [ LoadTests Python Smoke ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Smoke.yml) | N/A |`Run Python Load Tests Smoke`| [![.github/workflows/beam_LoadTests_Python_Smoke.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Smoke.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Smoke.yml?query=event%3Aschedule) ### Other Jobs | Workflow name | Matrix | Trigger Phrase | Cron Status | |:-------------:|:------:|:--------------:|:-----------:| -| [ Cancel Stale Dataflow Jobs ](https://github.com/apache/beam/actions/workflows/beam_CancelStaleDataflowJobs.yml) | N/A | `Run Cancel Stale Dataflow Jobs` | [![.github/workflows/beam_CancelStaleDataflowJobs.yml](https://github.com/apache/beam/actions/workflows/beam_CancelStaleDataflowJobs.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_CancelStaleDataflowJobs.yml) | -| [ Clean Up GCP Resources ](https://github.com/apache/beam/actions/workflows/beam_CleanUpGCPResources.yml) | N/A | `Run Clean GCP Resources` | [![.github/workflows/beam_CleanUpGCPResources.yml](https://github.com/apache/beam/actions/workflows/beam_CleanUpGCPResources.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_CleanUpGCPResources.yml) | -| [ Clean Up Prebuilt SDK Images ](https://github.com/apache/beam/actions/workflows/beam_CleanUpPrebuiltSDKImages.yml) | N/A | `Run Clean Prebuilt Images` | [![.github/workflows/beam_beam_CleanUpPrebuiltSDKImages.yml](https://github.com/apache/beam/actions/workflows/beam_CleanUpPrebuiltSDKImages.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_CleanUpPrebuiltSDKImages.yml) | -| [ Cleanup Dataproc Resources ](https://github.com/apache/beam/actions/workflows/beam_CleanUpDataprocResources.yml) | N/A | N/A | [![.github/workflows/beam_CleanUpDataprocResources.yml](https://github.com/apache/beam/actions/workflows/beam_CleanUpDataprocResources.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_CleanUpDataprocResources.yml) -| [ Community Metrics Prober ](https://github.com/apache/beam/actions/workflows/beam_Prober_CommunityMetrics.yml) | N/A |`Run Community Metrics Prober`| [![.github/workflows/beam_Prober_CommunityMetrics.yml](https://github.com/apache/beam/actions/workflows/beam_Prober_CommunityMetrics.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_Prober_CommunityMetrics.yml) -| [ Publish Docker Snapshots ](https://github.com/apache/beam/actions/workflows/beam_Publish_Docker_Snapshots.yml) | N/A |`Publish Docker Snapshots`| [![.github/workflows/beam_Publish_Docker_Snapshots.yml](https://github.com/apache/beam/actions/workflows/beam_Publish_Docker_Snapshots.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_Publish_Docker_Snapshots.yml) | -| [ Rotate IO-Datastores Cluster Credentials ](https://github.com/apache/beam/actions/workflows/beam_IODatastoresCredentialsRotation.yml) | N/A | N/A | [![.github/workflows/beam_IODatastoresCredentialsRotation.yml](https://github.com/apache/beam/actions/workflows/beam_IODatastoresCredentialsRotation.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_IODatastoresCredentialsRotation.yml) | -| [ Rotate Metrics Cluster Credentials ](https://github.com/apache/beam/actions/workflows/beam_MetricsCredentialsRotation.yml) | N/A | N/A | [![.github/workflows/beam_MetricsCredentialsRotation.yml](https://github.com/apache/beam/actions/workflows/beam_MetricsCredentialsRotation.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_MetricsCredentialsRotation.yml) | +| [ Cancel Stale Dataflow Jobs ](https://github.com/apache/beam/actions/workflows/beam_CancelStaleDataflowJobs.yml) | N/A | `Run Cancel Stale Dataflow Jobs` | [![.github/workflows/beam_CancelStaleDataflowJobs.yml](https://github.com/apache/beam/actions/workflows/beam_CancelStaleDataflowJobs.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_CancelStaleDataflowJobs.yml?query=event%3Aschedule) | +| [ Clean Up GCP Resources ](https://github.com/apache/beam/actions/workflows/beam_CleanUpGCPResources.yml) | N/A | `Run Clean GCP Resources` | [![.github/workflows/beam_CleanUpGCPResources.yml](https://github.com/apache/beam/actions/workflows/beam_CleanUpGCPResources.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_CleanUpGCPResources.yml?query=event%3Aschedule) | +| [ Clean Up Prebuilt SDK Images ](https://github.com/apache/beam/actions/workflows/beam_CleanUpPrebuiltSDKImages.yml) | N/A | `Run Clean Prebuilt Images` | [![.github/workflows/beam_beam_CleanUpPrebuiltSDKImages.yml](https://github.com/apache/beam/actions/workflows/beam_CleanUpPrebuiltSDKImages.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_CleanUpPrebuiltSDKImages.yml?query=event%3Aschedule) | +| [ Cleanup Dataproc Resources ](https://github.com/apache/beam/actions/workflows/beam_CleanUpDataprocResources.yml) | N/A | N/A | [![.github/workflows/beam_CleanUpDataprocResources.yml](https://github.com/apache/beam/actions/workflows/beam_CleanUpDataprocResources.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_CleanUpDataprocResources.yml?query=event%3Aschedule) +| [ Community Metrics Prober ](https://github.com/apache/beam/actions/workflows/beam_Prober_CommunityMetrics.yml) | N/A |`Run Community Metrics Prober`| [![.github/workflows/beam_Prober_CommunityMetrics.yml](https://github.com/apache/beam/actions/workflows/beam_Prober_CommunityMetrics.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_Prober_CommunityMetrics.yml?query=event%3Aschedule) +| [ Publish Beam SDK Snapshots ](https://github.com/apache/beam/actions/workflows/beam_Publish_Beam_SDK_Snapshots.yml) | N/A | N/A | [![.github/workflows/beam_Publish_Beam_SDK_Snapshots.yml](https://github.com/apache/beam/actions/workflows/beam_Publish_Beam_SDK_Snapshots.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_Publish_Beam_SDK_Snapshots.yml?query=event%3Aschedule) | +| [ Publish Docker Snapshots ](https://github.com/apache/beam/actions/workflows/beam_Publish_Docker_Snapshots.yml) | N/A |`Publish Docker Snapshots`| [![.github/workflows/beam_Publish_Docker_Snapshots.yml](https://github.com/apache/beam/actions/workflows/beam_Publish_Docker_Snapshots.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_Publish_Docker_Snapshots.yml?query=event%3Aschedule) | +| [ Rotate IO-Datastores Cluster Credentials ](https://github.com/apache/beam/actions/workflows/beam_IODatastoresCredentialsRotation.yml) | N/A | N/A | [![.github/workflows/beam_IODatastoresCredentialsRotation.yml](https://github.com/apache/beam/actions/workflows/beam_IODatastoresCredentialsRotation.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_IODatastoresCredentialsRotation.yml?query=event%3Aschedule) | +| [ Rotate Metrics Cluster Credentials ](https://github.com/apache/beam/actions/workflows/beam_MetricsCredentialsRotation.yml) | N/A | N/A | [![.github/workflows/beam_MetricsCredentialsRotation.yml](https://github.com/apache/beam/actions/workflows/beam_MetricsCredentialsRotation.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_MetricsCredentialsRotation.yml?query=event%3Aschedule) | From 683fbd638a762f871bb84aa33e3f925a0d752a02 Mon Sep 17 00:00:00 2001 From: Vivek Sumanth <61607315+viveksumanth@users.noreply.github.com> Date: Mon, 23 Oct 2023 13:01:46 -0700 Subject: [PATCH 233/435] Fix typos in HistogramData.java and HistogramDataTest.java (#29105) --- .../src/main/java/org/apache/beam/sdk/util/HistogramData.java | 4 ++-- .../test/java/org/apache/beam/sdk/util/HistogramDataTest.java | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/HistogramData.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/HistogramData.java index cca3a440334e..dd2193d9d335 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/HistogramData.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/HistogramData.java @@ -79,7 +79,7 @@ public static HistogramData linear(double start, double width, int numBuckets) { } /** - * Returns a histogram object wiht exponential boundaries. The input parameter {@code scale} + * Returns a histogram object with exponential boundaries. The input parameter {@code scale} * determines a coefficient 'base' which species bucket boundaries. * * <pre> @@ -381,7 +381,7 @@ private int getBucketIndexNegativeScale(int value) { return getBucketIndexZeroScale(value) >> (-getScale()); } - // This method is valid for all 'scale' values but we fallback to more effecient methods for + // This method is valid for all 'scale' values but we fallback to more efficient methods for // non-positive scales. // For a value>base we would like to find an i s.t. : // base^i <= value < base^(i+1) diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/HistogramDataTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/HistogramDataTest.java index bfad087ecfa5..133bf787de30 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/HistogramDataTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/HistogramDataTest.java @@ -205,7 +205,7 @@ public void testIncrementBucketCountByIndex() { // The following tests cover exponential buckets. @Test - public void testExponentialBuckets_PostiveScaleRecord() { + public void testExponentialBuckets_PositiveScaleRecord() { // Buckets will be: // Index Range // Underflow (-inf, 0) From f70cbc12053984b0d74d4c24d8b27de06dedb749 Mon Sep 17 00:00:00 2001 From: Damon <damondouglas@users.noreply.github.com> Date: Mon, 23 Oct 2023 15:05:36 -0700 Subject: [PATCH 234/435] [RRIO] [Throttle] Stub throttle without external transform (#29059) --- .../ThrottleWithoutExternalResource.java | 57 +++++++++++++++++++ 1 file changed, 57 insertions(+) create mode 100644 sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/ThrottleWithoutExternalResource.java diff --git a/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/ThrottleWithoutExternalResource.java b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/ThrottleWithoutExternalResource.java new file mode 100644 index 000000000000..0648a86f28eb --- /dev/null +++ b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/ThrottleWithoutExternalResource.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.io.requestresponse; + +import com.google.auto.value.AutoValue; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.values.PCollection; + +/** + * {@link ThrottleWithoutExternalResource} throttles a {@link RequestT} {@link PCollection} emitting + * a {@link RequestT} {@link PCollection} at a maximally configured rate, without using an external + * resource. + */ +// TODO(damondouglas): expand what "without external resource" means with respect to "with external +// resource" when the other throttle transforms implemented. +// See: https://github.com/apache/beam/issues/28932 +class ThrottleWithoutExternalResource<RequestT> + extends PTransform<PCollection<RequestT>, PCollection<RequestT>> { + + // TODO(damondouglas): remove suppress warnings when finally utilized in a future PR. + @SuppressWarnings({"unused"}) + private final Configuration<RequestT> configuration; + + private ThrottleWithoutExternalResource(Configuration<RequestT> configuration) { + this.configuration = configuration; + } + + @Override + public PCollection<RequestT> expand(PCollection<RequestT> input) { + // TODO(damondouglas): expand in a future PR. + return input; + } + + @AutoValue + abstract static class Configuration<RequestT> { + + @AutoValue.Builder + abstract static class Builder<RequestT> { + abstract Configuration<RequestT> build(); + } + } +} From 3f05945888e2e4aaf1dea03a1719192a037faa7b Mon Sep 17 00:00:00 2001 From: Robert Bradshaw <robertwb@gmail.com> Date: Mon, 23 Oct 2023 17:09:35 -0700 Subject: [PATCH 235/435] [YAML] Schema-producing Create. (#29030) Yaml's Create will now always create schema'd data. For non-row elements, this will be a single Row with an element field. --- sdks/python/apache_beam/yaml/yaml_provider.py | 9 ++++++- .../apache_beam/yaml/yaml_transform_test.py | 24 +++++++++++-------- 2 files changed, 22 insertions(+), 11 deletions(-) diff --git a/sdks/python/apache_beam/yaml/yaml_provider.py b/sdks/python/apache_beam/yaml/yaml_provider.py index 84399cd597b2..33c16380ece3 100644 --- a/sdks/python/apache_beam/yaml/yaml_provider.py +++ b/sdks/python/apache_beam/yaml/yaml_provider.py @@ -460,6 +460,13 @@ def create_transform(self, type, args, yaml_create_transform): } +def element_to_rows(e): + if isinstance(e, dict): + return dicts_to_rows(e) + else: + return beam.Row(element=dicts_to_rows(e)) + + def dicts_to_rows(o): if isinstance(o, dict): return beam.Row(**{k: dicts_to_rows(v) for k, v in o.items()}) @@ -487,7 +494,7 @@ def create(elements: Iterable[Any], reshuffle: bool = True): reshuffle (optional): Whether to introduce a reshuffle if there is more than one element in the collection. Defaults to True. """ - return beam.Create(dicts_to_rows(elements), reshuffle) + return beam.Create([element_to_rows(e) for e in elements], reshuffle) def with_schema(**args): # TODO: This is preliminary. diff --git a/sdks/python/apache_beam/yaml/yaml_transform_test.py b/sdks/python/apache_beam/yaml/yaml_transform_test.py index 05bbf4196473..ce608578b600 100644 --- a/sdks/python/apache_beam/yaml/yaml_transform_test.py +++ b/sdks/python/apache_beam/yaml/yaml_transform_test.py @@ -64,11 +64,11 @@ def __init__(self, limit, error_handling): self._error_handling = error_handling def expand(self, pcoll): - def raise_on_big(element): - if len(element) > self._limit: - raise ValueError(element) + def raise_on_big(row): + if len(row.element) > self._limit: + raise ValueError(row.element) else: - return element + return row.element good, bad = pcoll | beam.Map(raise_on_big).with_exception_handling() return {'small_elements': good, self._error_handling['output']: bad} @@ -211,7 +211,7 @@ def test_implicit_flatten(self): - type: PyMap input: [CreateBig, CreateSmall] config: - fn: "lambda x: x * x" + fn: "lambda x: x.element * x.element" output: PyMap ''', providers=TEST_PROVIDERS) @@ -273,7 +273,7 @@ def test_name_is_not_ambiguous(self): - type: PyMap name: PyMap config: - fn: "lambda elem: elem * elem" + fn: "lambda row: row.element * row.element" input: Create output: PyMap ''', @@ -431,11 +431,14 @@ def test_mapping_errors(self): - type: Create config: elements: [0, 1, 2, 4] - - type: PyMap + - type: MapToFields name: ToRow input: Create config: - fn: "lambda x: beam.Row(num=x, str='a' * x or 'bbb')" + language: python + fields: + num: element + str: "'a' * element or 'bbb'" - type: Filter input: ToRow config: @@ -595,7 +598,8 @@ class AnnotatingProvider(yaml_provider.InlineProvider): """ def __init__(self, name, transform_names): super().__init__({ - transform_name: lambda: beam.Map(lambda x: (x or ()) + (name, )) + transform_name: + lambda: beam.Map(lambda x: (x if type(x) == tuple else ()) + (name, )) for transform_name in transform_names.strip().split() }) self._name = name @@ -728,7 +732,7 @@ def __init__(self, a, b): def expand(self, pcoll): a = self._a b = self._b - return pcoll | beam.Map(lambda x: a * x + b) + return pcoll | beam.Map(lambda x: a * x.element + b) if __name__ == '__main__': From 9e21f3d07201c54f210d67742f71639045517530 Mon Sep 17 00:00:00 2001 From: Vlado Djerek <rage.bubblegum@gmail.com> Date: Tue, 24 Oct 2023 15:24:52 +0200 Subject: [PATCH 236/435] adding metrics report (#29010) * adding metrics report * change concurrency group Co-authored-by: Andrey Devyatkin <andrey.9evyatkin@gmail.com> * misc fixes * Update .github/workflows/beam_Metrics_Report.yml Co-authored-by: Andrey Devyatkin <andrey.9evyatkin@gmail.com> --------- Co-authored-by: Andrey Devyatkin <andrey.9evyatkin@gmail.com> --- .github/workflows/beam_Metrics_Report.yml | 114 ++++++++++++++++++++++ 1 file changed, 114 insertions(+) create mode 100644 .github/workflows/beam_Metrics_Report.yml diff --git a/.github/workflows/beam_Metrics_Report.yml b/.github/workflows/beam_Metrics_Report.yml new file mode 100644 index 000000000000..418b6bde43d3 --- /dev/null +++ b/.github/workflows/beam_Metrics_Report.yml @@ -0,0 +1,114 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +name: Beam Metrics Report + +on: + # issue_comment: + # types: [created] + # schedule: + # - cron: '0 */23 * * *' + workflow_dispatch: + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +jobs: + beam_Metrics_Report: + name: ${{matrix.job_name}} (${{matrix.job_phrase}}) + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 100 + strategy: + matrix: + job_name: [beam_Metrics_Report] + job_phrase: [Run Metrics Report] + if: | + github.event_name == 'schedule' || + github.event_name == 'workflow_dispatch' || + github.event.comment.body == 'Run Metrics Report' + steps: + - uses: actions/checkout@v3 + - name: Generate a token + id: generate_token + uses: actions/create-github-app-token@v1 + with: + app_id: ${{ secrets.GH_APP_ID }} + private_key: ${{ secrets.GH_APP_PRIVATE_KEY }} + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ steps.generate_token.outputs.token }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + steps_context: ${{ toJson(steps) }} + - name: Setup environment + uses: ./.github/actions/setup-environment-action + with: + python-version: 3.8 + - name: Run Metrics Report + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :beam-test-jenkins:generateMetricsReport + arguments: --info -PinfluxDb=beam_test_metrics -PinfluxHost='10.128.0.96' -PinfluxPort=8086 + env: + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} + - name: Archive Report + uses: actions/upload-artifact@v3 + with: + name: Metrics Report + path: "${{ github.workspace }}/.test-infra/jenkins/metrics_report/beam-metrics_report.html" + - name: Generate Date + run: | + date=$(date -u +"%Y-%m-%d") + echo "date=$date" >> $GITHUB_ENV + - name: Send mail + uses: dawidd6/action-send-mail@v3 + with: + server_address: smtp.gmail.com + server_port: 465 + secure: true + username: ${{ secrets.ISSUE_REPORT_SENDER_EMAIL_ADDRESS }} + password: ${{ secrets.ISSUE_REPORT_SENDER_EMAIL_PASSWORD }} + subject: Beam Metrics Report ${{ env.date }} + to: dannymccormick@google.com #dev@beam.apache.org + cc: vlado.djerek@akvelon.com + from: gactions@beam.apache.org # <user@example.com> + html_body: file://${{ github.workspace }}/.test-infra/jenkins/metrics_report/beam-metrics_report.html From bc06581e11085d02a52356681226ee8caac5a825 Mon Sep 17 00:00:00 2001 From: Yi Hu <yathu@google.com> Date: Tue, 24 Oct 2023 11:05:32 -0400 Subject: [PATCH 237/435] Enable run test on Java 21 (#28969) * upgrade gradle 8,3 -> 8.4 * Use temurin 21 base image as its available now * Setup Java21 JPMS postcommits * Setup Java21 Example PostCommit on DataflowV2 * Fix kotlin test --- .github/workflows/README.md | 1 + ...tCommit_Java_Examples_Dataflow_V2_Java.yml | 7 +- ...eam_PostCommit_Java_Jpms_Direct_Java21.yml | 97 +++++++++ .../beam/gradle/BeamModulePlugin.groovy | 190 ++++++++---------- examples/java/build.gradle | 4 + examples/kotlin/build.gradle | 9 +- gradle/wrapper/gradle-wrapper.properties | 2 +- gradlew | 14 +- sdks/java/container/agent/build.gradle | 9 +- sdks/java/container/java21/build.gradle | 2 - sdks/java/testing/jpms-tests/build.gradle | 4 +- 11 files changed, 209 insertions(+), 130 deletions(-) create mode 100644 .github/workflows/beam_PostCommit_Java_Jpms_Direct_Java21.yml diff --git a/.github/workflows/README.md b/.github/workflows/README.md index 5777a84b2a3f..18333212487c 100644 --- a/.github/workflows/README.md +++ b/.github/workflows/README.md @@ -283,6 +283,7 @@ Please note that jobs with matrix need to have matrix element in the comment. Ex | [ PostCommit Java Jpms Dataflow Java17 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml) | N/A |`Run Jpms Dataflow Java 17 PostCommit`| [![.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml?query=event%3Aschedule) | | [ PostCommit Java Jpms Direct Java11 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml) | N/A |`Run Jpms Direct Java 11 PostCommit`| [![.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml?query=event%3Aschedule) | | [ PostCommit Java Jpms Direct Java17 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml) | N/A |`Run Jpms Direct Java 17 PostCommit`| [![.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml?query=event%3Aschedule) | +| [ PostCommit Java Jpms Direct Java21 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Direct_Java21.yml) | N/A |`Run Jpms Direct Java21 PostCommit`| [![.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java21.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Direct_Java21.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Direct_Java21.yml?query=event%3Aschedule) | | [ PostCommit Java Jpms Flink Java11 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml) | N/A |`Run Jpms Flink Java 11 PostCommit`| [![.github/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml?query=event%3Aschedule) | | [ PostCommit Java Jpms Spark Java11 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml) | N/A |`Run Jpms Spark Java 11 PostCommit`| [![.github/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml?query=event%3Aschedule) | | [ PostCommit Java Nexmark Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml) | N/A |`Run Dataflow Runner Nexmark Tests`| [![.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml?query=event%3Aschedule) | diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml index 1b0065901046..fe8de2d2abe0 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml @@ -59,7 +59,7 @@ jobs: job_name: [beam_PostCommit_Java_Examples_Dataflow_V2_Java] job_phrase_1: [Run Java ] job_phrase_2: [Examples on Dataflow Runner V2] - java_version: ['11', '17'] + java_version: ['11', '17', '21'] if: | github.event_name == 'workflow_dispatch' || github.event_name == 'schedule' || @@ -76,7 +76,10 @@ jobs: - name: Setup environment uses: ./.github/actions/setup-environment-action with: - java-version: ${{ matrix.java_version }} + distribution: 'temurin' + java-version: | + ${{ matrix.java_version }} + 8 - name: run PostCommit Java Examples Dataflow V2 Java${{ matrix.java_version }} script uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java21.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java21.yml new file mode 100644 index 000000000000..fb93a1c72953 --- /dev/null +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java21.yml @@ -0,0 +1,97 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: PostCommit Java Jpms Direct Java21 + +on: + schedule: + - cron: '0 */6 * * *' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: write + checks: write + contents: read + deployments: read + id-token: none + issues: write + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_PostCommit_Java_Jpms_Direct_Java21: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Jpms Direct Java 21 PostCommit' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 240 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_PostCommit_Java_Jpms_Direct_Java21"] + job_phrase: ["Run Jpms Direct Java 21 PostCommit"] + steps: + - uses: actions/checkout@v4 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Set up Java + uses: actions/setup-java@v3.11.0 + with: + distribution: 'temurin' + java-version: | + 21 + 8 + - name: run PostCommit Java Jpms Direct Java21 script + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:jpms-tests:directRunnerIntegrationTest + arguments: + -PskipCheckerFramework + -PtestJavaVersion=21 + -Pjava21Home=$JAVA_HOME_21_X64 + - name: Archive JUnit Test Results + uses: actions/upload-artifact@v3 + if: failure() + with: + name: JUnit Test Results + path: "**/build/reports/tests/" + - name: Publish JUnit Test Results + uses: EnricoMi/publish-unit-test-result-action@v2 + if: always() + with: + commit: '${{ env.prsha || env.GITHUB_SHA }}' + comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} + files: '**/build/test-results/**/*.xml' \ No newline at end of file diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index b32a41f80279..3d7dd5d5aaf9 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -449,6 +449,21 @@ class BeamModulePlugin implements Plugin<Project> { return 'beam' + p.path.replace(':', '-') } + static def getSupportedJavaVersion() { + if (JavaVersion.current() == JavaVersion.VERSION_1_8) { + return 'java8' + } else if (JavaVersion.current() == JavaVersion.VERSION_11) { + return 'java11' + } else if (JavaVersion.current() == JavaVersion.VERSION_17) { + return 'java17' + } else if (JavaVersion.current() == JavaVersion.VERSION_21) { + return 'java21' + } else { + String exceptionMessage = "Your Java version is unsupported. You need Java version of 8, 11, 17 or 21 to get started, but your Java version is: " + JavaVersion.current(); + throw new GradleException(exceptionMessage) + } + } + /* * Set compile args for compiling and running in different java version by modifying the compiler args in place. * @@ -922,51 +937,63 @@ class BeamModulePlugin implements Plugin<Project> { + suffix) } - project.ext.setJava17Options = { CompileOptions options -> - def java17Home = project.findProperty("java17Home") - options.fork = true - options.forkOptions.javaHome = java17Home as File - options.compilerArgs += ['-Xlint:-path'] - // Error prone requires some packages to be exported/opened for Java 17 - // Disabling checks since this property is only used for Jenkins tests - // https://github.com/tbroyer/gradle-errorprone-plugin#jdk-16-support - options.errorprone.errorproneArgs.add("-XepDisableAllChecks") - // The -J prefix is needed to workaround https://github.com/gradle/gradle/issues/22747 - options.forkOptions.jvmArgs += [ - "-J--add-exports=jdk.compiler/com.sun.tools.javac.api=ALL-UNNAMED", - "-J--add-exports=jdk.compiler/com.sun.tools.javac.file=ALL-UNNAMED", - "-J--add-exports=jdk.compiler/com.sun.tools.javac.main=ALL-UNNAMED", - "-J--add-exports=jdk.compiler/com.sun.tools.javac.model=ALL-UNNAMED", - "-J--add-exports=jdk.compiler/com.sun.tools.javac.parser=ALL-UNNAMED", - "-J--add-exports=jdk.compiler/com.sun.tools.javac.processing=ALL-UNNAMED", - "-J--add-exports=jdk.compiler/com.sun.tools.javac.tree=ALL-UNNAMED", - "-J--add-exports=jdk.compiler/com.sun.tools.javac.util=ALL-UNNAMED", - "-J--add-opens=jdk.compiler/com.sun.tools.javac.code=ALL-UNNAMED", - "-J--add-opens=jdk.compiler/com.sun.tools.javac.comp=ALL-UNNAMED" - ] - } - - project.ext.setJava21Options = { CompileOptions options -> - def java21Home = project.findProperty("java21Home") - options.fork = true - options.forkOptions.javaHome = java21Home as File - options.compilerArgs += ['-Xlint:-path'] - // Error prone requires some packages to be exported/opened for Java 17+ - // Disabling checks since this property is only used for Jenkins tests - // https://github.com/tbroyer/gradle-errorprone-plugin#jdk-16-support - options.errorprone.errorproneArgs.add("-XepDisableAllChecks") - options.forkOptions.jvmArgs += [ - "-J--add-exports=jdk.compiler/com.sun.tools.javac.api=ALL-UNNAMED", - "-J--add-exports=jdk.compiler/com.sun.tools.javac.file=ALL-UNNAMED", - "-J--add-exports=jdk.compiler/com.sun.tools.javac.main=ALL-UNNAMED", - "-J--add-exports=jdk.compiler/com.sun.tools.javac.model=ALL-UNNAMED", - "-J--add-exports=jdk.compiler/com.sun.tools.javac.parser=ALL-UNNAMED", - "-J--add-exports=jdk.compiler/com.sun.tools.javac.processing=ALL-UNNAMED", - "-J--add-exports=jdk.compiler/com.sun.tools.javac.tree=ALL-UNNAMED", - "-J--add-exports=jdk.compiler/com.sun.tools.javac.util=ALL-UNNAMED", - "-J--add-opens=jdk.compiler/com.sun.tools.javac.code=ALL-UNNAMED", - "-J--add-opens=jdk.compiler/com.sun.tools.javac.comp=ALL-UNNAMED" - ] + // set compiler options for java version overrides to compile with a different java version + project.ext.setJavaVerOptions = { CompileOptions options, String ver -> + if (ver == '11') { + def java11Home = project.findProperty("java11Home") + options.fork = true + options.forkOptions.javaHome = java11Home as File + options.compilerArgs += ['-Xlint:-path'] + } else if (ver == '17') { + def java17Home = project.findProperty("java17Home") + options.fork = true + options.forkOptions.javaHome = java17Home as File + options.compilerArgs += ['-Xlint:-path'] + // Error prone requires some packages to be exported/opened for Java 17 + // Disabling checks since this property is only used for tests + // https://github.com/tbroyer/gradle-errorprone-plugin#jdk-16-support + options.errorprone.errorproneArgs.add("-XepDisableAllChecks") + // The -J prefix is needed to workaround https://github.com/gradle/gradle/issues/22747 + options.forkOptions.jvmArgs += [ + "-J--add-exports=jdk.compiler/com.sun.tools.javac.api=ALL-UNNAMED", + "-J--add-exports=jdk.compiler/com.sun.tools.javac.file=ALL-UNNAMED", + "-J--add-exports=jdk.compiler/com.sun.tools.javac.main=ALL-UNNAMED", + "-J--add-exports=jdk.compiler/com.sun.tools.javac.model=ALL-UNNAMED", + "-J--add-exports=jdk.compiler/com.sun.tools.javac.parser=ALL-UNNAMED", + "-J--add-exports=jdk.compiler/com.sun.tools.javac.processing=ALL-UNNAMED", + "-J--add-exports=jdk.compiler/com.sun.tools.javac.tree=ALL-UNNAMED", + "-J--add-exports=jdk.compiler/com.sun.tools.javac.util=ALL-UNNAMED", + "-J--add-opens=jdk.compiler/com.sun.tools.javac.code=ALL-UNNAMED", + "-J--add-opens=jdk.compiler/com.sun.tools.javac.comp=ALL-UNNAMED" + ] + } else if (ver == '21') { + def java21Home = project.findProperty("java21Home") + options.fork = true + options.forkOptions.javaHome = java21Home as File + options.compilerArgs += ['-Xlint:-path'] + // Error prone requires some packages to be exported/opened for Java 17+ + // Disabling checks since this property is only used for tests + options.errorprone.errorproneArgs.add("-XepDisableAllChecks") + options.forkOptions.jvmArgs += [ + "-J--add-exports=jdk.compiler/com.sun.tools.javac.api=ALL-UNNAMED", + "-J--add-exports=jdk.compiler/com.sun.tools.javac.file=ALL-UNNAMED", + "-J--add-exports=jdk.compiler/com.sun.tools.javac.main=ALL-UNNAMED", + "-J--add-exports=jdk.compiler/com.sun.tools.javac.model=ALL-UNNAMED", + "-J--add-exports=jdk.compiler/com.sun.tools.javac.parser=ALL-UNNAMED", + "-J--add-exports=jdk.compiler/com.sun.tools.javac.processing=ALL-UNNAMED", + "-J--add-exports=jdk.compiler/com.sun.tools.javac.tree=ALL-UNNAMED", + "-J--add-exports=jdk.compiler/com.sun.tools.javac.util=ALL-UNNAMED", + "-J--add-opens=jdk.compiler/com.sun.tools.javac.code=ALL-UNNAMED", + "-J--add-opens=jdk.compiler/com.sun.tools.javac.comp=ALL-UNNAMED" + ] + // TODO(https://github.com/apache/beam/issues/28963) + // upgrade checkerFramework to enable it in Java 21 + project.checkerFramework { + skipCheckerFramework = true + } + } else { + throw new GradleException("Unknown Java Version ${ver} for setting additional java options") + } } project.ext.repositories = { @@ -1515,37 +1542,20 @@ class BeamModulePlugin implements Plugin<Project> { options.errorprone.errorproneArgs.add("-Xep:Slf4jLoggerShouldBeNonStatic:OFF") } - if (project.findProperty('testJavaVersion') == "11") { - def java11Home = project.findProperty("java11Home") - project.tasks.compileTestJava { - options.fork = true - options.forkOptions.javaHome = java11Home as File - options.compilerArgs += ['-Xlint:-path'] - setCompileAndRuntimeJavaVersion(options.compilerArgs, '11') - } - project.tasks.withType(Test).configureEach { - useJUnit() - executable = "${java11Home}/bin/java" - } - } else if (project.findProperty('testJavaVersion') == "17") { - def java17Home = project.findProperty("java17Home") - project.tasks.compileTestJava { - setCompileAndRuntimeJavaVersion(options.compilerArgs, '17') - project.ext.setJava17Options(options) - } - project.tasks.withType(Test).configureEach { - useJUnit() - executable = "${java17Home}/bin/java" - } - } else if (project.findProperty('testJavaVersion') == "21") { - def java21Home = project.findProperty("java21Home") + // if specified test java version, modify the compile and runtime versions accordingly + if (project.hasProperty('testJavaVersion')) { + String ver = project.getProperty('testJavaVersion') + def testJavaHome = project.getProperty("java${ver}Home") + + // redirect java compiler to specified version for compileTestJava only project.tasks.compileTestJava { - setCompileAndRuntimeJavaVersion(options.compilerArgs, '21') - project.ext.setJava17Options(options) + setCompileAndRuntimeJavaVersion(options.compilerArgs, ver) + project.ext.setJavaVerOptions(options, ver) } + // redirect java runtime to specified version for running tests project.tasks.withType(Test).configureEach { useJUnit() - executable = "${java21Home}/bin/java" + executable = "${testJavaHome}/bin/java" } } @@ -2579,17 +2589,7 @@ class BeamModulePlugin implements Plugin<Project> { "java_expansion_service_allowlist_file": javaClassLookupAllowlistFile, ] def usesDataflowRunner = config.pythonPipelineOptions.contains("--runner=TestDataflowRunner") || config.pythonPipelineOptions.contains("--runner=DataflowRunner") - def javaContainerSuffix - if (JavaVersion.current() == JavaVersion.VERSION_1_8) { - javaContainerSuffix = 'java8' - } else if (JavaVersion.current() == JavaVersion.VERSION_11) { - javaContainerSuffix = 'java11' - } else if (JavaVersion.current() == JavaVersion.VERSION_17) { - javaContainerSuffix = 'java17' - } else { - String exceptionMessage = "Your Java version is unsupported. You need Java version of 8 or 11 or 17 to get started, but your Java version is: " + JavaVersion.current(); - throw new GradleException(exceptionMessage) - } + def javaContainerSuffix = getSupportedJavaVersion() // 1. Builds the chosen expansion service jar and launches it def setupTask = project.tasks.register(config.name+"Setup") { @@ -2692,17 +2692,7 @@ class BeamModulePlugin implements Plugin<Project> { ] def serviceArgs = project.project(':sdks:python').mapToArgString(expansionServiceOpts) def pythonContainerSuffix = project.project(':sdks:python').pythonVersion.replace('.', '') - def javaContainerSuffix - if (JavaVersion.current() == JavaVersion.VERSION_1_8) { - javaContainerSuffix = 'java8' - } else if (JavaVersion.current() == JavaVersion.VERSION_11) { - javaContainerSuffix = 'java11' - } else if (JavaVersion.current() == JavaVersion.VERSION_17) { - javaContainerSuffix = 'java17' - } else { - String exceptionMessage = "Your Java version is unsupported. You need Java version of 8 or 11 or 17 to get started, but your Java version is: " + JavaVersion.current(); - throw new GradleException(exceptionMessage) - } + def javaContainerSuffix = getSupportedJavaVersion() def setupTask = project.tasks.register(config.name+"Setup", Exec) { dependsOn ':sdks:java:container:'+javaContainerSuffix+':docker' dependsOn ':sdks:python:container:py'+pythonContainerSuffix+':docker' @@ -2875,17 +2865,7 @@ class BeamModulePlugin implements Plugin<Project> { ] def serviceArgs = project.project(':sdks:python').mapToArgString(transformServiceOpts) def pythonContainerSuffix = project.project(':sdks:python').pythonVersion.replace('.', '') - def javaContainerSuffix - if (JavaVersion.current() == JavaVersion.VERSION_1_8) { - javaContainerSuffix = 'java8' - } else if (JavaVersion.current() == JavaVersion.VERSION_11) { - javaContainerSuffix = 'java11' - } else if (JavaVersion.current() == JavaVersion.VERSION_17) { - javaContainerSuffix = 'java17' - } else { - String exceptionMessage = "Your Java version is unsupported. You need Java version of 8 or 11 or 17 to get started, but your Java version is: " + JavaVersion.current(); - throw new GradleException(exceptionMessage) - } + def javaContainerSuffix = getSupportedJavaVersion() // Transform service delivers transforms that refer to SDK harness containers with following sufixes. def transformServiceJavaContainerSuffix = 'java11' diff --git a/examples/java/build.gradle b/examples/java/build.gradle index 2e262e8de795..a43862ae801d 100644 --- a/examples/java/build.gradle +++ b/examples/java/build.gradle @@ -54,6 +54,10 @@ configurations.sparkRunnerPreCommit { dependencies { implementation enforcedPlatform(library.java.google_cloud_platform_libraries_bom) implementation library.java.vendored_guava_32_1_2_jre + if (project.findProperty('testJavaVersion') == '21' || JavaVersion.current().equals(JavaVersion.VERSION_21)) { + // this dependency is somehow needed for compile only under Java21 + compileOnly library.java.kafka + } implementation library.java.kafka_clients implementation project(path: ":sdks:java:core", configuration: "shadow") implementation project(":sdks:java:extensions:avro") diff --git a/examples/kotlin/build.gradle b/examples/kotlin/build.gradle index 98258401d588..829aefd44745 100644 --- a/examples/kotlin/build.gradle +++ b/examples/kotlin/build.gradle @@ -80,6 +80,9 @@ dependencies { for (String runner : preCommitRunners) { delegate.add(runner + "PreCommit", project(path: ":examples:kotlin", configuration: "testRuntimeMigration")) } + directRunnerPreCommit project(project.path) + flinkRunnerPreCommit project(project.path) + sparkRunnerPreCommit project(project.path) directRunnerPreCommit project(path: ":runners:direct-java", configuration: "shadow") flinkRunnerPreCommit project(":runners:flink:${project.ext.latestFlinkVersion}") sparkRunnerPreCommit project(":runners:spark:3") @@ -106,10 +109,7 @@ for (String runner : preCommitRunners) { "--runner=" + preCommitRunnerClass[runner], ] classpath = configurations."${runner}PreCommit" - include "**/WordCountIT.class" - if (!"sparkRunner".equals(runner)) { - include "**/WindowedWordCountIT.class" - } + include "**/kotlin/**/*Test.class" forkEvery 1 maxParallelForks 4 systemProperty "beamTestPipelineOptions", JsonOutput.toJson(preCommitBeamTestPipelineOptions) @@ -136,4 +136,3 @@ compileTestKotlin { repositories { mavenCentral() } - diff --git a/gradle/wrapper/gradle-wrapper.properties b/gradle/wrapper/gradle-wrapper.properties index ac72c34e8acc..3fa8f862f753 100644 --- a/gradle/wrapper/gradle-wrapper.properties +++ b/gradle/wrapper/gradle-wrapper.properties @@ -1,6 +1,6 @@ distributionBase=GRADLE_USER_HOME distributionPath=wrapper/dists -distributionUrl=https\://services.gradle.org/distributions/gradle-8.3-bin.zip +distributionUrl=https\://services.gradle.org/distributions/gradle-8.4-bin.zip networkTimeout=10000 validateDistributionUrl=true zipStoreBase=GRADLE_USER_HOME diff --git a/gradlew b/gradlew index 0adc8e1a5321..1aa94a426907 100755 --- a/gradlew +++ b/gradlew @@ -145,7 +145,7 @@ if ! "$cygwin" && ! "$darwin" && ! "$nonstop" ; then case $MAX_FD in #( max*) # In POSIX sh, ulimit -H is undefined. That's why the result is checked to see if it worked. - # shellcheck disable=SC3045 + # shellcheck disable=SC2039,SC3045 MAX_FD=$( ulimit -H -n ) || warn "Could not query maximum file descriptor limit" esac @@ -153,7 +153,7 @@ if ! "$cygwin" && ! "$darwin" && ! "$nonstop" ; then '' | soft) :;; #( *) # In POSIX sh, ulimit -n is undefined. That's why the result is checked to see if it worked. - # shellcheck disable=SC3045 + # shellcheck disable=SC2039,SC3045 ulimit -n "$MAX_FD" || warn "Could not set maximum file descriptor limit to $MAX_FD" esac @@ -202,11 +202,11 @@ fi # Add default JVM options here. You can also use JAVA_OPTS and GRADLE_OPTS to pass JVM options to this script. DEFAULT_JVM_OPTS='"-Xmx64m" "-Xms64m"' -# Collect all arguments for the java command; -# * $DEFAULT_JVM_OPTS, $JAVA_OPTS, and $GRADLE_OPTS can contain fragments of -# shell script including quotes and variable substitutions, so put them in -# double quotes to make sure that they get re-expanded; and -# * put everything else in single quotes, so that it's not re-expanded. +# Collect all arguments for the java command: +# * DEFAULT_JVM_OPTS, JAVA_OPTS, JAVA_OPTS, and optsEnvironmentVar are not allowed to contain shell fragments, +# and any embedded shellness will be escaped. +# * For example: A user cannot expect ${Hostname} to be expanded, as it is an environment variable and will be +# treated as '${Hostname}' itself on the command line. set -- \ "-Dorg.gradle.appname=$APP_BASE_NAME" \ diff --git a/sdks/java/container/agent/build.gradle b/sdks/java/container/agent/build.gradle index 044d175257cc..44338915bf28 100644 --- a/sdks/java/container/agent/build.gradle +++ b/sdks/java/container/agent/build.gradle @@ -47,15 +47,12 @@ jar { // the order is intended here if (project.hasProperty('java11Home')) { - def java11Home = project.findProperty('java11Home') project.tasks.withType(JavaCompile) { - options.fork = true - options.forkOptions.javaHome = java11Home as File - options.compilerArgs += ['-Xlint:-path'] + setJavaVerOptions(options, '11') } } else if (project.hasProperty('java17Home')) { project.tasks.withType(JavaCompile) { - setJava17Options(options) + setJavaVerOptions(options, '17') checkerFramework { skipCheckerFramework = true @@ -63,7 +60,7 @@ if (project.hasProperty('java11Home')) { } } else if (project.hasProperty('java21Home')) { project.tasks.withType(JavaCompile) { - setJava21Options(options) + setJavaVerOptions(options, '21') checkerFramework { skipCheckerFramework = true diff --git a/sdks/java/container/java21/build.gradle b/sdks/java/container/java21/build.gradle index 038064102dcb..75c84c0c7cdf 100644 --- a/sdks/java/container/java21/build.gradle +++ b/sdks/java/container/java21/build.gradle @@ -17,8 +17,6 @@ */ project.ext { - // TODO(https://github.com/apache/beam/issues/28120) switch to temurin once available - javaBaseImage = 'openjdk' imageJavaVersion = '21' } diff --git a/sdks/java/testing/jpms-tests/build.gradle b/sdks/java/testing/jpms-tests/build.gradle index 429fc063027f..9b881d7a2a9e 100644 --- a/sdks/java/testing/jpms-tests/build.gradle +++ b/sdks/java/testing/jpms-tests/build.gradle @@ -46,9 +46,9 @@ project.tasks.compileJava { options.fork = true options.forkOptions.javaHome = project.findProperty("java${project.getProperty('testJavaVersion')}Home") as File if (project.getProperty('testJavaVersion') == '17') { - setJava17Options(options) + setJavaVerOptions(options, '17') } else if (project.getProperty('testJavaVersion') == '21') { - setJava21Options(options) + setJavaVerOptions(options, '21') } } } From 0d95b4b400e94b2d47303c336852bb3192cf53b1 Mon Sep 17 00:00:00 2001 From: Jack McCluskey <34928439+jrmccluskey@users.noreply.github.com> Date: Tue, 24 Oct 2023 13:46:33 -0400 Subject: [PATCH 238/435] Increment Go build version to 1.21.3 (#29119) * Increment Go build version to 1.21.3 * Fix security fixes markdown links --- CHANGES.md | 3 ++- .../main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/CHANGES.md b/CHANGES.md index 0f1530a78fe9..6d67ccdcf95d 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -88,7 +88,8 @@ as a workaround, a copy of "old" `CountingSource` class should be placed into a * `watch_file_pattern` arg of the [RunInference](https://github.com/apache/beam/blob/104c10b3ee536a9a3ea52b4dbf62d86b669da5d9/sdks/python/apache_beam/ml/inference/base.py#L997) arg had no effect prior to 2.52.0. To use the behavior of arg `watch_file_pattern` prior to 2.52.0, follow the documentation at https://beam.apache.org/documentation/ml/side-input-updates/ and use `WatchFilePattern` PTransform as a SideInput. ([#28948](https://github.com/apache/beam/pulls/28948)) ## Security Fixes -* Fixed (CVE-YYYY-NNNN)[https://www.cve.org/CVERecord?id=CVE-YYYY-NNNN] (Java/Python/Go) ([#X](https://github.com/apache/beam/issues/X)). +* Fixed [CVE-YYYY-NNNN](https://www.cve.org/CVERecord?id=CVE-YYYY-NNNN) (Java/Python/Go) ([#X](https://github.com/apache/beam/issues/X)). +* Fixed [CVE-2023-39325](https://www.cve.org/CVERecord?id=CVE-2023-39325) (Java/Python/Go) ([#29118](https://github.com/apache/beam/issues/29118)). ## Known Issues diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index 3d7dd5d5aaf9..0b72499abadc 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -2197,7 +2197,7 @@ class BeamModulePlugin implements Plugin<Project> { def goRootDir = "${project.rootDir}/sdks/go" // This sets the whole project Go version. - project.ext.goVersion = "go1.21.2" + project.ext.goVersion = "go1.21.3" // Minor TODO: Figure out if we can pull out the GOCMD env variable after goPrepare script // completion, and avoid this GOBIN substitution. From 9b2ca42910c7fd57711104821024700d60904eb7 Mon Sep 17 00:00:00 2001 From: Danny McCormick <dannymccormick@google.com> Date: Tue, 24 Oct 2023 14:20:40 -0400 Subject: [PATCH 239/435] Fix flatten docs (#29120) * Fix flatten docs * Nest in function --- .../Core Transforms/Flatten/Flatten/task.py | 30 +++++++++++-------- 1 file changed, 18 insertions(+), 12 deletions(-) diff --git a/learning/katas/python/Core Transforms/Flatten/Flatten/task.py b/learning/katas/python/Core Transforms/Flatten/Flatten/task.py index ae0f5c81d958..e8639782c60c 100644 --- a/learning/katas/python/Core Transforms/Flatten/Flatten/task.py +++ b/learning/katas/python/Core Transforms/Flatten/Flatten/task.py @@ -18,7 +18,7 @@ # name: Flatten # description: Task from katas that merges two PCollections of words into a single PCollection. # multifile: false -# context_line: 31 +# context_line: 33 # categories: # - Flatten # complexity: BASIC @@ -26,16 +26,22 @@ # - merge # - strings -import apache_beam as beam - -with beam.Pipeline() as p: - - wordsStartingWithA = \ - p | 'Words starting with A' >> beam.Create(['apple', 'ant', 'arrow']) - - wordsStartingWithB = \ - p | 'Words starting with B' >> beam.Create(['ball', 'book', 'bow']) - - ((wordsStartingWithA, wordsStartingWithB) +def flatten(): + # [START flatten] + import apache_beam as beam + + with beam.Pipeline() as p: + + wordsStartingWithA = \ + p | 'Words starting with A' >> beam.Create(['apple', 'ant', 'arrow']) + + wordsStartingWithB = \ + p | 'Words starting with B' >> beam.Create(['ball', 'book', 'bow']) + + ((wordsStartingWithA, wordsStartingWithB) | beam.Flatten() | beam.LogElements()) + # [END flatten] + +if __name__ == '__main__': + flatten() From be2cd9dcdef0cf880749ff2ee81d6674a06a6355 Mon Sep 17 00:00:00 2001 From: Vlado Djerek <rage.bubblegum@gmail.com> Date: Tue, 24 Oct 2023 20:23:52 +0200 Subject: [PATCH 240/435] change report generator to be aware of GH and passtrough vars to tox (#29121) * change report generator to be aware of GH and passtrough vars to tox * Update .test-infra/jenkins/metrics_report/report_generator.py Rework the logic of github detection * remove github run id env from tox config as its not used --- .github/workflows/beam_Metrics_Report.yml | 9 +-------- .test-infra/jenkins/metrics_report/report_generator.py | 9 ++++++--- .test-infra/jenkins/metrics_report/tox.ini | 2 +- 3 files changed, 8 insertions(+), 12 deletions(-) diff --git a/.github/workflows/beam_Metrics_Report.yml b/.github/workflows/beam_Metrics_Report.yml index 418b6bde43d3..0c88171f50c1 100644 --- a/.github/workflows/beam_Metrics_Report.yml +++ b/.github/workflows/beam_Metrics_Report.yml @@ -65,19 +65,12 @@ jobs: github.event.comment.body == 'Run Metrics Report' steps: - uses: actions/checkout@v3 - - name: Generate a token - id: generate_token - uses: actions/create-github-app-token@v1 - with: - app_id: ${{ secrets.GH_APP_ID }} - private_key: ${{ secrets.GH_APP_PRIVATE_KEY }} - name: Setup repository uses: ./.github/actions/setup-action with: comment_phrase: ${{ matrix.job_phrase }} - github_token: ${{ steps.generate_token.outputs.token }} + github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - steps_context: ${{ toJson(steps) }} - name: Setup environment uses: ./.github/actions/setup-environment-action with: diff --git a/.test-infra/jenkins/metrics_report/report_generator.py b/.test-infra/jenkins/metrics_report/report_generator.py index 7923fcce9dab..bdaada04f30d 100644 --- a/.test-infra/jenkins/metrics_report/report_generator.py +++ b/.test-infra/jenkins/metrics_report/report_generator.py @@ -27,11 +27,14 @@ INFLUXDB_USER = os.getenv("INFLUXDB_USER") INFLUXDB_USER_PASSWORD = os.getenv("INFLUXDB_USER_PASSWORD") -WORKING_SPACE = os.getenv("WORKSPACE", "") +WORKING_SPACE = os.getenv("GITHUB_WORKSPACE", os.getenv("WORKSPACE", "")) +if "GITHUB_WORKSPACE" in os.environ: + path_prefix = "" +else: + path_prefix= "src/" PERF_DASHBOARDS = os.path.join( WORKING_SPACE, - "src/.test-infra/metrics/grafana/dashboards/perftests_metrics/") - + path_prefix+".test-infra/metrics/grafana/dashboards/perftests_metrics/") TABLE_FIELD_NAMES = [ "Measurement", "Metric", diff --git a/.test-infra/jenkins/metrics_report/tox.ini b/.test-infra/jenkins/metrics_report/tox.ini index dbf68016c57b..026db5dc4860 100644 --- a/.test-infra/jenkins/metrics_report/tox.ini +++ b/.test-infra/jenkins/metrics_report/tox.ini @@ -32,5 +32,5 @@ commands = python -m unittest dashboards_parser.py [testenv:py38-generate-report] deps = -r requirements.txt -passenv = WORKSPACE,INFLUXDB_USER,INFLUXDB_USER_PASSWORD +passenv = WORKSPACE,INFLUXDB_USER,INFLUXDB_USER_PASSWORD,GITHUB_WORKSPACE commands = python report_generator.py {posargs} From 02b84b8487a42900caef8099e23cd31c63b88238 Mon Sep 17 00:00:00 2001 From: Yi Hu <yathu@google.com> Date: Tue, 24 Oct 2023 14:57:31 -0400 Subject: [PATCH 241/435] Follow up Java21 examples (#29117) * Skip unsupported ver for -PtestJavaVersion option this fixes Java Examples ARM GHA on Java8 * Fix missing readme --- .github/workflows/README.md | 2 +- .github/workflows/beam_PostCommit_Java_Jpms_Direct_Java21.yml | 2 +- .../main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/.github/workflows/README.md b/.github/workflows/README.md index 18333212487c..c7db263120ef 100644 --- a/.github/workflows/README.md +++ b/.github/workflows/README.md @@ -271,7 +271,7 @@ Please note that jobs with matrix need to have matrix element in the comment. Ex | [ PostCommit Java Dataflow V2 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_DataflowV2.yml) | N/A |`Run PostCommit_Java_DataflowV2`| [![.github/workflows/beam_PostCommit_Java_DataflowV2.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_DataflowV2.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_DataflowV2.yml?query=event%3Aschedule) | | [ PostCommit Java Examples Dataflow ARM ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml) | ['8','11','17'] |`Run Java_Examples_Dataflow_ARM PostCommit (matrix_element)`| [![.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml?query=event%3Aschedule) | | [ PostCommit Java Examples Dataflow](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow.yml) | N/A |`Run Java examples on Dataflow`| [![.github/workflows/beam_PostCommit_Java_Examples_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow.yml?query=event%3Aschedule) | -| [ PostCommit Java Examples Dataflow Java ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml) | ['11','17'] |`Run Java examples on Dataflow Java (matrix_element)`| [![.github/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml?query=event%3Aschedule) | +| [ PostCommit Java Examples Dataflow Java ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml) | ['11','17','21'] |`Run Java examples on Dataflow Java (matrix_element)`| [![.github/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml?query=event%3Aschedule) | | [ PostCommit Java Examples Dataflow V2 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml) | N/A |`Run Java Examples on Dataflow Runner V2`| [![.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml?query=event%3Aschedule) | | [ PostCommit Java Examples Dataflow V2 Java ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml) | ['11','17'] |`Run Java (matrix_element) Examples on Dataflow Runner V2`| [![.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml?query=event%3Aschedule) | | [ PostCommit Java Examples Direct ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Direct.yml) | N/A |`Run Java Examples_Direct`| [![.github/workflows/beam_PostCommit_Java_Examples_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Direct.yml?query=event%3Aschedule) | diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java21.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java21.yml index fb93a1c72953..608d70b20cd9 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java21.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java21.yml @@ -38,7 +38,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index 0b72499abadc..5943c0c77d81 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -1543,7 +1543,7 @@ class BeamModulePlugin implements Plugin<Project> { } // if specified test java version, modify the compile and runtime versions accordingly - if (project.hasProperty('testJavaVersion')) { + if (['11', '17', '21'].contains(project.findProperty('testJavaVersion'))) { String ver = project.getProperty('testJavaVersion') def testJavaHome = project.getProperty("java${ver}Home") From ad11e0a664ac335b12c602b23c422cda3e91a627 Mon Sep 17 00:00:00 2001 From: Vlado Djerek <rage.bubblegum@gmail.com> Date: Tue, 24 Oct 2023 20:58:20 +0200 Subject: [PATCH 242/435] add auth to gcp step (#29123) --- .github/workflows/beam_PerformanceTests_Kafka_IO.yml | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/.github/workflows/beam_PerformanceTests_Kafka_IO.yml b/.github/workflows/beam_PerformanceTests_Kafka_IO.yml index 772142a46ef8..f19b0b844a98 100644 --- a/.github/workflows/beam_PerformanceTests_Kafka_IO.yml +++ b/.github/workflows/beam_PerformanceTests_Kafka_IO.yml @@ -73,6 +73,12 @@ jobs: github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - name: Setup environment uses: ./.github/actions/setup-environment-action + - name: Authenticate on GCP + id: auth + uses: google-github-actions/auth@v1 + with: + credentials_json: ${{ secrets.GCP_SA_KEY }} + project_id: ${{ secrets.GCP_PROJECT_ID }} - name: Set k8s access uses: ./.github/actions/setup-k8s-access with: From 9d142879fb28a3a5a4d4f46067a7873ae53c967f Mon Sep 17 00:00:00 2001 From: Vlado Djerek <rage.bubblegum@gmail.com> Date: Tue, 24 Oct 2023 21:21:22 +0200 Subject: [PATCH 243/435] remove subdir from test config paths (#29125) --- .../beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml | 8 ++++---- .../workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml | 6 +++--- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml index ccd8c371e583..9146c20226c3 100644 --- a/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml @@ -79,10 +79,10 @@ jobs: test-type: load test-language: python argument-file-paths: | - ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python-cogbk/python_CoGBK_Dataflow_Streaming_100b_Single_Key.txt - ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python-cogbk/python_CoGBK_Dataflow_Streaming_100b_Multiple_Keys.txt - ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python-cogbk/python_CoGBK_Dataflow_Streaming_10kB.txt - ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python-cogbk/python_CoGBK_Dataflow_Streaming_2MB.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_CoGBK_Dataflow_Streaming_100b_Single_Key.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_CoGBK_Dataflow_Streaming_100b_Multiple_Keys.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_CoGBK_Dataflow_Streaming_10kB.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_CoGBK_Dataflow_Streaming_2MB.txt - name: Set current datetime id: datetime run: | diff --git a/.github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml index 87de130bd4ed..3783f72cf9bc 100644 --- a/.github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml @@ -89,9 +89,9 @@ jobs: test-type: load test-language: python argument-file-paths: | - ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python-cogbk/python_CoGBK_Dataflow_Flink_Batch_100b_Single_Key.txt - ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python-cogbk/python_CoGBK_Dataflow_Flink_Batch_100b_Multiple_Keys.txt - ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python-cogbk/python_CoGBK_Dataflow_Flink_Batch_10kB.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_CoGBK_Dataflow_Flink_Batch_100b_Single_Key.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_CoGBK_Dataflow_Flink_Batch_100b_Multiple_Keys.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/python_CoGBK_Dataflow_Flink_Batch_10kB.txt - name: Start Flink with parallelism 5 env: FLINK_NUM_WORKERS: 5 From 0478cd2827cd003cb7511278bb9c5f13dd96e151 Mon Sep 17 00:00:00 2001 From: Vlado Djerek <rage.bubblegum@gmail.com> Date: Tue, 24 Oct 2023 21:23:16 +0200 Subject: [PATCH 244/435] move performance singlestoreio config to the right place (#29126) --- .github/workflows/beam_PerformanceTests_SingleStoreIO.yml | 2 +- .../config_PerformanceTests_SingleStoreIO.txt | 0 2 files changed, 1 insertion(+), 1 deletion(-) rename .github/workflows/{performance-tests-job-configs => performance-tests-pipeline-options}/config_PerformanceTests_SingleStoreIO.txt (100%) diff --git a/.github/workflows/beam_PerformanceTests_SingleStoreIO.yml b/.github/workflows/beam_PerformanceTests_SingleStoreIO.yml index e7bf6cc91cd4..c5c35b5807ab 100644 --- a/.github/workflows/beam_PerformanceTests_SingleStoreIO.yml +++ b/.github/workflows/beam_PerformanceTests_SingleStoreIO.yml @@ -105,7 +105,7 @@ jobs: test-type: performance test-language: java argument-file-paths: | - ${{ github.workspace }}/.github/workflows/performance-tests-job-configs/config_PerformanceTests_SingleStoreIO.txt + ${{ github.workspace }}/.github/workflows/performance-tests-pipeline-options/config_PerformanceTests_SingleStoreIO.txt arguments: | --singleStoreServerName=${{ steps.install_singlestore.outputs.lb_ip }} # The env variable is created and populated in the test-arguments-action as "<github.job>_test_arguments_<argument_file_paths_index>" diff --git a/.github/workflows/performance-tests-job-configs/config_PerformanceTests_SingleStoreIO.txt b/.github/workflows/performance-tests-pipeline-options/config_PerformanceTests_SingleStoreIO.txt similarity index 100% rename from .github/workflows/performance-tests-job-configs/config_PerformanceTests_SingleStoreIO.txt rename to .github/workflows/performance-tests-pipeline-options/config_PerformanceTests_SingleStoreIO.txt From 0f7698143261271cf0fdb527b833e73997f10748 Mon Sep 17 00:00:00 2001 From: Yi Hu <yathu@google.com> Date: Tue, 24 Oct 2023 15:24:15 -0400 Subject: [PATCH 245/435] Teardown Load test and IO Performance tests on Jenkins (#29092) --- ..._LoadTests_CoGBK_Dataflow_V2_Java11.groovy | 246 ----------- ..._LoadTests_CoGBK_Dataflow_V2_Java17.groovy | 246 ----------- ...GBK_Java_spark_structured_streaming.groovy | 172 -------- .../jenkins/job_LoadTests_Combine_Go.groovy | 127 ------ .../jenkins/job_LoadTests_Combine_Java.groovy | 162 -------- .../job_LoadTests_Combine_Java_Smoke.groovy | 85 ---- ...ine_Java_spark_structured_streaming.groovy | 123 ------ .../job_LoadTests_FnApiRunner_Python.groovy | 74 ---- ...ob_LoadTests_GBK_Dataflow_V2_Java11.groovy | 311 -------------- ...ob_LoadTests_GBK_Dataflow_V2_Java17.groovy | 311 -------------- .../job_LoadTests_GBK_Flink_Python.groovy | 172 -------- .../jenkins/job_LoadTests_GBK_Go.groovy | 227 ----------- .../jenkins/job_LoadTests_GBK_Java.groovy | 261 ------------ .../job_LoadTests_GBK_Java_Smoke.groovy | 85 ---- ...GBK_Java_spark_structured_streaming.groovy | 210 ---------- .../jenkins/job_LoadTests_GBK_Python.groovy | 209 ---------- .../job_LoadTests_GBK_Python_reiterate.groovy | 137 ------- ..._LoadTests_ParDo_Dataflow_V2_Java11.groovy | 217 ---------- ..._LoadTests_ParDo_Dataflow_V2_Java17.groovy | 217 ---------- .../job_LoadTests_ParDo_Flink_Go.groovy | 152 ------- .../jenkins/job_LoadTests_ParDo_Go.groovy | 155 ------- .../jenkins/job_LoadTests_ParDo_Java.groovy | 186 --------- ...rDo_Java_spark_structured_streaming.groovy | 148 ------- .../jenkins/job_LoadTests_ParDo_Python.groovy | 186 --------- .../jenkins/job_LoadTests_Python_Smoke.groovy | 71 ---- .../jenkins/job_LoadTests_SideInput_Go.groovy | 100 ----- .../job_LoadTests_SideInput_Python.groovy | 198 --------- .../job_LoadTests_coGBK_Flink_Python.groovy | 156 ------- .../jenkins/job_LoadTests_coGBK_Go.groovy | 179 -------- ...ob_PerformanceTests_BigQueryIO_Java.groovy | 145 ------- ..._PerformanceTests_BigQueryIO_Python.groovy | 110 ----- .../job_PerformanceTests_CdapIO.groovy | 72 ---- ...job_PerformanceTests_FileBasedIO_IT.groovy | 384 ------------------ ...PerformanceTests_FileBasedIO_Python.groovy | 82 ---- .../job_PerformanceTests_HadoopFormat.groovy | 73 ---- .../jenkins/job_PerformanceTests_JDBC.groovy | 73 ---- .../job_PerformanceTests_MongoDBIO_IT.groovy | 71 ---- ...ob_PerformanceTests_PubsubIO_Python.groovy | 72 ---- .../job_PerformanceTests_Python.groovy | 97 ----- .../job_PerformanceTests_SQLIO_Java.groovy | 67 --- .../job_PerformanceTests_SingleStoreIO.groovy | 86 ---- ...b_PerformanceTests_SpannerIO_Python.groovy | 112 ----- ...PerformanceTests_SparkReceiverIO_IT.groovy | 84 ---- 43 files changed, 6651 deletions(-) delete mode 100644 .test-infra/jenkins/job_LoadTests_CoGBK_Dataflow_V2_Java11.groovy delete mode 100644 .test-infra/jenkins/job_LoadTests_CoGBK_Dataflow_V2_Java17.groovy delete mode 100644 .test-infra/jenkins/job_LoadTests_CoGBK_Java_spark_structured_streaming.groovy delete mode 100644 .test-infra/jenkins/job_LoadTests_Combine_Go.groovy delete mode 100644 .test-infra/jenkins/job_LoadTests_Combine_Java.groovy delete mode 100644 .test-infra/jenkins/job_LoadTests_Combine_Java_Smoke.groovy delete mode 100644 .test-infra/jenkins/job_LoadTests_Combine_Java_spark_structured_streaming.groovy delete mode 100644 .test-infra/jenkins/job_LoadTests_FnApiRunner_Python.groovy delete mode 100644 .test-infra/jenkins/job_LoadTests_GBK_Dataflow_V2_Java11.groovy delete mode 100644 .test-infra/jenkins/job_LoadTests_GBK_Dataflow_V2_Java17.groovy delete mode 100644 .test-infra/jenkins/job_LoadTests_GBK_Flink_Python.groovy delete mode 100644 .test-infra/jenkins/job_LoadTests_GBK_Go.groovy delete mode 100644 .test-infra/jenkins/job_LoadTests_GBK_Java.groovy delete mode 100644 .test-infra/jenkins/job_LoadTests_GBK_Java_Smoke.groovy delete mode 100644 .test-infra/jenkins/job_LoadTests_GBK_Java_spark_structured_streaming.groovy delete mode 100644 .test-infra/jenkins/job_LoadTests_GBK_Python.groovy delete mode 100644 .test-infra/jenkins/job_LoadTests_GBK_Python_reiterate.groovy delete mode 100644 .test-infra/jenkins/job_LoadTests_ParDo_Dataflow_V2_Java11.groovy delete mode 100644 .test-infra/jenkins/job_LoadTests_ParDo_Dataflow_V2_Java17.groovy delete mode 100644 .test-infra/jenkins/job_LoadTests_ParDo_Flink_Go.groovy delete mode 100644 .test-infra/jenkins/job_LoadTests_ParDo_Go.groovy delete mode 100644 .test-infra/jenkins/job_LoadTests_ParDo_Java.groovy delete mode 100644 .test-infra/jenkins/job_LoadTests_ParDo_Java_spark_structured_streaming.groovy delete mode 100644 .test-infra/jenkins/job_LoadTests_ParDo_Python.groovy delete mode 100644 .test-infra/jenkins/job_LoadTests_Python_Smoke.groovy delete mode 100644 .test-infra/jenkins/job_LoadTests_SideInput_Go.groovy delete mode 100644 .test-infra/jenkins/job_LoadTests_SideInput_Python.groovy delete mode 100644 .test-infra/jenkins/job_LoadTests_coGBK_Flink_Python.groovy delete mode 100644 .test-infra/jenkins/job_LoadTests_coGBK_Go.groovy delete mode 100644 .test-infra/jenkins/job_PerformanceTests_BigQueryIO_Java.groovy delete mode 100644 .test-infra/jenkins/job_PerformanceTests_BigQueryIO_Python.groovy delete mode 100644 .test-infra/jenkins/job_PerformanceTests_CdapIO.groovy delete mode 100644 .test-infra/jenkins/job_PerformanceTests_FileBasedIO_IT.groovy delete mode 100644 .test-infra/jenkins/job_PerformanceTests_FileBasedIO_Python.groovy delete mode 100644 .test-infra/jenkins/job_PerformanceTests_HadoopFormat.groovy delete mode 100644 .test-infra/jenkins/job_PerformanceTests_JDBC.groovy delete mode 100644 .test-infra/jenkins/job_PerformanceTests_MongoDBIO_IT.groovy delete mode 100644 .test-infra/jenkins/job_PerformanceTests_PubsubIO_Python.groovy delete mode 100644 .test-infra/jenkins/job_PerformanceTests_Python.groovy delete mode 100644 .test-infra/jenkins/job_PerformanceTests_SQLIO_Java.groovy delete mode 100644 .test-infra/jenkins/job_PerformanceTests_SingleStoreIO.groovy delete mode 100644 .test-infra/jenkins/job_PerformanceTests_SpannerIO_Python.groovy delete mode 100644 .test-infra/jenkins/job_PerformanceTests_SparkReceiverIO_IT.groovy diff --git a/.test-infra/jenkins/job_LoadTests_CoGBK_Dataflow_V2_Java11.groovy b/.test-infra/jenkins/job_LoadTests_CoGBK_Dataflow_V2_Java11.groovy deleted file mode 100644 index 55501db4429d..000000000000 --- a/.test-infra/jenkins/job_LoadTests_CoGBK_Dataflow_V2_Java11.groovy +++ /dev/null @@ -1,246 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonJobProperties as commonJobProperties -import CommonTestProperties -import LoadTestsBuilder as loadTestsBuilder -import PhraseTriggeringPostCommitBuilder -import CronJobBuilder -import InfluxDBCredentialsHelper - -def loadTestConfigurations = { mode, isStreaming -> - [ - [ - title : 'Load test: CoGBK 2GB 100 byte records - single key', - test : 'org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - project : 'apache-beam-testing', - region : 'us-central1', - appName : "load_tests_Java11_Dataflow_V2_${mode}_CoGBK_1", - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${mode}_cogbk_1", - influxTags : """ - { - "runnerVersion": "v2", - "jdk": "java11" - } - """.trim().replaceAll("\\s", ""), - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 20000000, - "keySizeBytes": 10, - "valueSizeBytes": 90, - "numHotKeys": 1 - } - """.trim().replaceAll("\\s", ""), - coSourceOptions : """ - { - "numRecords": 2000000, - "keySizeBytes": 10, - "valueSizeBytes": 90, - "numHotKeys": 1000 - } - """.trim().replaceAll("\\s", ""), - iterations : 1, - numWorkers : 5, - autoscalingAlgorithm : "NONE", - streaming : isStreaming - ] - ], - [ - title : 'Load test: CoGBK 2GB 100 byte records - multiple keys', - test : 'org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - project : 'apache-beam-testing', - region : 'us-central1', - appName : "load_tests_Java11_Dataflow_V2_${mode}_CoGBK_2", - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${mode}_cogbk_2", - influxTags : """ - { - "runnerVersion": "v2", - "jdk": "java11" - } - """.trim().replaceAll("\\s", ""), - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 20000000, - "keySizeBytes": 10, - "valueSizeBytes": 90, - "numHotKeys": 5 - } - """.trim().replaceAll("\\s", ""), - coSourceOptions : """ - { - "numRecords": 2000000, - "keySizeBytes": 10, - "valueSizeBytes": 90, - "numHotKeys": 1000 - } - """.trim().replaceAll("\\s", ""), - iterations : 1, - numWorkers : 5, - autoscalingAlgorithm : "NONE", - streaming : isStreaming - ] - ], - [ - - title : 'Load test: CoGBK 2GB reiteration 10kB value', - test : 'org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - project : 'apache-beam-testing', - region : 'us-central1', - appName : "load_tests_Java11_Dataflow_V2_${mode}_CoGBK_3", - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${mode}_cogbk_3", - influxTags : """ - { - "runnerVersion": "v2", - "jdk": "java11" - } - """.trim().replaceAll("\\s", ""), - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 2000000, - "keySizeBytes": 10, - "valueSizeBytes": 90, - "numHotKeys": 200000 - } - """.trim().replaceAll("\\s", ""), - coSourceOptions : """ - { - "numRecords": 2000000, - "keySizeBytes": 10, - "valueSizeBytes": 90, - "numHotKeys": 1000 - } - """.trim().replaceAll("\\s", ""), - iterations : 4, - numWorkers : 5, - autoscalingAlgorithm : "NONE", - streaming : isStreaming - ] - - ], - [ - title : 'Load test: CoGBK 2GB reiteration 2MB value', - test : 'org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - project : 'apache-beam-testing', - region : 'us-central1', - appName : "load_tests_Java11_Dataflow_V2_${mode}_CoGBK_4", - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${mode}_cogbk_4", - influxTags : """ - { - "runnerVersion": "v2", - "jdk": "java11" - } - """.trim().replaceAll("\\s", ""), - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 2000000, - "keySizeBytes": 10, - "valueSizeBytes": 90, - "numHotKeys": 1000 - } - """.trim().replaceAll("\\s", ""), - coSourceOptions : """ - { - "numRecords": 2000000, - "keySizeBytes": 10, - "valueSizeBytes": 90, - "numHotKeys": 1000 - } - """.trim().replaceAll("\\s", ""), - iterations : 4, - numWorkers : 5, - autoscalingAlgorithm : "NONE", - streaming : isStreaming - ] - ] - ].each { test -> test.pipelineOptions.putAll(additionalPipelineArgs) } -} - -def final JOB_SPECIFIC_SWITCHES = [ - '-Prunner.version="V2"', - '-PtestJavaVersion=11', - "-Pjava11Home=${commonJobProperties.JAVA_11_HOME}" -] - -def streamingLoadTestJob = { scope, triggeringContext -> - scope.description('Runs Java 11 CoGBK load tests on Dataflow runner V2 in streaming mode') - commonJobProperties.setTopLevelMainJobProperties(scope, 'master', 240) - - for (testConfiguration in loadTestConfigurations('streaming', true)) { - testConfiguration.pipelineOptions << [inputWindowDurationSec: 1200, coInputWindowDurationSec: 1200] - loadTestsBuilder.loadTest(scope, testConfiguration.title, testConfiguration.runner, CommonTestProperties.SDK.JAVA, - testConfiguration.pipelineOptions, testConfiguration.test, JOB_SPECIFIC_SWITCHES) - } -} - -CronJobBuilder.cronJob('beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_Java11', 'H H * * *', this) { - additionalPipelineArgs = [ - influxDatabase: InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influxHost: InfluxDBCredentialsHelper.InfluxDBHostUrl, - ] - streamingLoadTestJob(delegate, CommonTestProperties.TriggeringContext.POST_COMMIT) -} - -PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_Java11', - 'Run Load Tests Java 11 CoGBK Dataflow V2 Streaming', - 'Load Tests Java 11 CoGBK Dataflow V2 Streaming suite', - this - ) { - additionalPipelineArgs = [:] - streamingLoadTestJob(delegate, CommonTestProperties.TriggeringContext.PR) - } - - -def batchLoadTestJob = { scope, triggeringContext -> - loadTestsBuilder.loadTests(scope, CommonTestProperties.SDK.JAVA, loadTestConfigurations('batch', false), - "CoGBK", "batch", JOB_SPECIFIC_SWITCHES) -} - -CronJobBuilder.cronJob('beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_Java11', 'H H * * *', this) { - additionalPipelineArgs = [ - influxDatabase: InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influxHost: InfluxDBCredentialsHelper.InfluxDBHostUrl, - ] - batchLoadTestJob(delegate, CommonTestProperties.TriggeringContext.POST_COMMIT) -} - -PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_Java11', - 'Run Load Tests Java 11 CoGBK Dataflow V2 Batch', - 'Load Tests Java 11 CoGBK Dataflow V2 Batch suite', - this - ) { - additionalPipelineArgs = [:] - batchLoadTestJob(delegate, CommonTestProperties.TriggeringContext.PR) - } diff --git a/.test-infra/jenkins/job_LoadTests_CoGBK_Dataflow_V2_Java17.groovy b/.test-infra/jenkins/job_LoadTests_CoGBK_Dataflow_V2_Java17.groovy deleted file mode 100644 index 8fb09fd07448..000000000000 --- a/.test-infra/jenkins/job_LoadTests_CoGBK_Dataflow_V2_Java17.groovy +++ /dev/null @@ -1,246 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonJobProperties as commonJobProperties -import CommonTestProperties -import LoadTestsBuilder as loadTestsBuilder -import PhraseTriggeringPostCommitBuilder -import CronJobBuilder -import InfluxDBCredentialsHelper - -def loadTestConfigurations = { mode, isStreaming -> - [ - [ - title : 'Load test: CoGBK 2GB 100 byte records - single key', - test : 'org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - project : 'apache-beam-testing', - region : 'us-central1', - appName : "load_tests_Java17_Dataflow_V2_${mode}_CoGBK_1", - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${mode}_cogbk_1", - influxTags : """ - { - "runnerVersion": "v2", - "jdk": "java17" - } - """.trim().replaceAll("\\s", ""), - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 20000000, - "keySizeBytes": 10, - "valueSizeBytes": 90, - "numHotKeys": 1 - } - """.trim().replaceAll("\\s", ""), - coSourceOptions : """ - { - "numRecords": 2000000, - "keySizeBytes": 10, - "valueSizeBytes": 90, - "numHotKeys": 1000 - } - """.trim().replaceAll("\\s", ""), - iterations : 1, - numWorkers : 5, - autoscalingAlgorithm : "NONE", - streaming : isStreaming - ] - ], - [ - title : 'Load test: CoGBK 2GB 100 byte records - multiple keys', - test : 'org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - project : 'apache-beam-testing', - region : 'us-central1', - appName : "load_tests_Java17_Dataflow_V2_${mode}_CoGBK_2", - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${mode}_cogbk_2", - influxTags : """ - { - "runnerVersion": "v2", - "jdk": "java17" - } - """.trim().replaceAll("\\s", ""), - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 20000000, - "keySizeBytes": 10, - "valueSizeBytes": 90, - "numHotKeys": 5 - } - """.trim().replaceAll("\\s", ""), - coSourceOptions : """ - { - "numRecords": 2000000, - "keySizeBytes": 10, - "valueSizeBytes": 90, - "numHotKeys": 1000 - } - """.trim().replaceAll("\\s", ""), - iterations : 1, - numWorkers : 5, - autoscalingAlgorithm : "NONE", - streaming : isStreaming - ] - ], - [ - - title : 'Load test: CoGBK 2GB reiteration 10kB value', - test : 'org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - project : 'apache-beam-testing', - region : 'us-central1', - appName : "load_tests_Java17_Dataflow_V2_${mode}_CoGBK_3", - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${mode}_cogbk_3", - influxTags : """ - { - "runnerVersion": "v2", - "jdk": "java17" - } - """.trim().replaceAll("\\s", ""), - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 2000000, - "keySizeBytes": 10, - "valueSizeBytes": 90, - "numHotKeys": 200000 - } - """.trim().replaceAll("\\s", ""), - coSourceOptions : """ - { - "numRecords": 2000000, - "keySizeBytes": 10, - "valueSizeBytes": 90, - "numHotKeys": 1000 - } - """.trim().replaceAll("\\s", ""), - iterations : 4, - numWorkers : 5, - autoscalingAlgorithm : "NONE", - streaming : isStreaming - ] - - ], - [ - title : 'Load test: CoGBK 2GB reiteration 2MB value', - test : 'org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - project : 'apache-beam-testing', - region : 'us-central1', - appName : "load_tests_Java17_Dataflow_V2_${mode}_CoGBK_4", - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${mode}_cogbk_4", - influxTags : """ - { - "runnerVersion": "v2", - "jdk": "java17" - } - """.trim().replaceAll("\\s", ""), - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 2000000, - "keySizeBytes": 10, - "valueSizeBytes": 90, - "numHotKeys": 1000 - } - """.trim().replaceAll("\\s", ""), - coSourceOptions : """ - { - "numRecords": 2000000, - "keySizeBytes": 10, - "valueSizeBytes": 90, - "numHotKeys": 1000 - } - """.trim().replaceAll("\\s", ""), - iterations : 4, - numWorkers : 5, - autoscalingAlgorithm : "NONE", - streaming : isStreaming - ] - ] - ].each { test -> test.pipelineOptions.putAll(additionalPipelineArgs) } -} - -def final JOB_SPECIFIC_SWITCHES = [ - '-Prunner.version="V2"', - '-PtestJavaVersion=17', - "-Pjava17Home=${commonJobProperties.JAVA_17_HOME}" -] - -def streamingLoadTestJob = { scope, triggeringContext -> - scope.description('Runs Java 17 CoGBK load tests on Dataflow runner V2 in streaming mode') - commonJobProperties.setTopLevelMainJobProperties(scope, 'master', 240) - - for (testConfiguration in loadTestConfigurations('streaming', true)) { - testConfiguration.pipelineOptions << [inputWindowDurationSec: 1200, coInputWindowDurationSec: 1200] - loadTestsBuilder.loadTest(scope, testConfiguration.title, testConfiguration.runner, CommonTestProperties.SDK.JAVA, - testConfiguration.pipelineOptions, testConfiguration.test, JOB_SPECIFIC_SWITCHES) - } -} - -CronJobBuilder.cronJob('beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_Java17', 'H H * * *', this) { - additionalPipelineArgs = [ - influxDatabase: InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influxHost: InfluxDBCredentialsHelper.InfluxDBHostUrl, - ] - streamingLoadTestJob(delegate, CommonTestProperties.TriggeringContext.POST_COMMIT) -} - -PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_Java17', - 'Run Load Tests Java 17 CoGBK Dataflow V2 Streaming', - 'Load Tests Java 17 CoGBK Dataflow V2 Streaming suite', - this - ) { - additionalPipelineArgs = [:] - streamingLoadTestJob(delegate, CommonTestProperties.TriggeringContext.PR) - } - - -def batchLoadTestJob = { scope, triggeringContext -> - loadTestsBuilder.loadTests(scope, CommonTestProperties.SDK.JAVA, loadTestConfigurations('batch', false), - "CoGBK", "batch", JOB_SPECIFIC_SWITCHES) -} - -CronJobBuilder.cronJob('beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_Java17', 'H H * * *', this) { - additionalPipelineArgs = [ - influxDatabase: InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influxHost: InfluxDBCredentialsHelper.InfluxDBHostUrl, - ] - batchLoadTestJob(delegate, CommonTestProperties.TriggeringContext.POST_COMMIT) -} - -PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_Java17', - 'Run Load Tests Java 17 CoGBK Dataflow V2 Batch', - 'Load Tests Java 17 CoGBK Dataflow V2 Batch suite', - this - ) { - additionalPipelineArgs = [:] - batchLoadTestJob(delegate, CommonTestProperties.TriggeringContext.PR) - } diff --git a/.test-infra/jenkins/job_LoadTests_CoGBK_Java_spark_structured_streaming.groovy b/.test-infra/jenkins/job_LoadTests_CoGBK_Java_spark_structured_streaming.groovy deleted file mode 100644 index 1d02dd7c6d81..000000000000 --- a/.test-infra/jenkins/job_LoadTests_CoGBK_Java_spark_structured_streaming.groovy +++ /dev/null @@ -1,172 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonTestProperties -import CronJobBuilder -import LoadTestsBuilder as loadTestsBuilder -import PhraseTriggeringPostCommitBuilder -import InfluxDBCredentialsHelper - -def loadTestConfigurations = { mode, isStreaming -> - [ - [ - title : 'Load test: CoGBK 2GB 100 byte records - single key', - test : 'org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest', - runner : CommonTestProperties.Runner.SPARK_STRUCTURED_STREAMING, - pipelineOptions: [ - project : 'apache-beam-testing', - appName : "load_tests_Java_SparkStructuredStreaming_${mode}_CoGBK_1", - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${mode}_cogbk_1", - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 20000000, - "keySizeBytes": 10, - "valueSizeBytes": 90, - "numHotKeys": 1 - } - """.trim().replaceAll("\\s", ""), - coSourceOptions : """ - { - "numRecords": 2000000, - "keySizeBytes": 10, - "valueSizeBytes": 90, - "numHotKeys": 1000 - } - """.trim().replaceAll("\\s", ""), - iterations : 1, - streaming : isStreaming - ] - ], - [ - title : 'Load test: CoGBK 2GB 100 byte records - multiple keys', - test : 'org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest', - runner : CommonTestProperties.Runner.SPARK_STRUCTURED_STREAMING, - pipelineOptions: [ - project : 'apache-beam-testing', - appName : "load_tests_Java_SparkStructuredStreaming_${mode}_CoGBK_2", - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${mode}_cogbk_2", - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 20000000, - "keySizeBytes": 10, - "valueSizeBytes": 90, - "numHotKeys": 5 - } - """.trim().replaceAll("\\s", ""), - coSourceOptions : """ - { - "numRecords": 2000000, - "keySizeBytes": 10, - "valueSizeBytes": 90, - "numHotKeys": 1000 - } - """.trim().replaceAll("\\s", ""), - iterations : 1, - streaming : isStreaming - ] - ], - [ - - title : 'Load test: CoGBK 2GB reiteration 10kB value', - test : 'org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest', - runner : CommonTestProperties.Runner.SPARK_STRUCTURED_STREAMING, - pipelineOptions: [ - project : 'apache-beam-testing', - appName : "load_tests_Java_SparkStructuredStreaming_${mode}_CoGBK_3", - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${mode}_cogbk_3", - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 2000000, - "keySizeBytes": 10, - "valueSizeBytes": 90, - "numHotKeys": 200000 - } - """.trim().replaceAll("\\s", ""), - coSourceOptions : """ - { - "numRecords": 2000000, - "keySizeBytes": 10, - "valueSizeBytes": 90, - "numHotKeys": 1000 - } - """.trim().replaceAll("\\s", ""), - iterations : 4, - streaming : isStreaming - ] - - ], - [ - title : 'Load test: CoGBK 2GB reiteration 2MB value', - test : 'org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest', - runner : CommonTestProperties.Runner.SPARK_STRUCTURED_STREAMING, - pipelineOptions: [ - project : 'apache-beam-testing', - appName : "load_tests_Java_SparkStructuredStreaming_${mode}_CoGBK_4", - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${mode}_cogbk_4", - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 2000000, - "keySizeBytes": 10, - "valueSizeBytes": 90, - "numHotKeys": 1000 - } - """.trim().replaceAll("\\s", ""), - coSourceOptions : """ - { - "numRecords": 2000000, - "keySizeBytes": 10, - "valueSizeBytes": 90, - "numHotKeys": 1000 - } - """.trim().replaceAll("\\s", ""), - iterations : 4, - streaming : isStreaming - ] - ] - ].each { test -> test.pipelineOptions.putAll(additionalPipelineArgs) } -} - -def batchLoadTestJob = { scope, triggeringContext -> - loadTestsBuilder.loadTests(scope, CommonTestProperties.SDK.JAVA, loadTestConfigurations('batch', false), "CoGBK", "batch") -} - -CronJobBuilder.cronJob('beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch', 'H H * * *', this) { - additionalPipelineArgs = [ - influxDatabase: InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influxHost: InfluxDBCredentialsHelper.InfluxDBHostUrl, - ] - batchLoadTestJob(delegate, CommonTestProperties.TriggeringContext.POST_COMMIT) -} - -PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch', - 'Run Load Tests Java CoGBK SparkStructuredStreaming Batch', - 'Load Tests Java CoGBK SparkStructuredStreaming Batch suite', - this - ) { - additionalPipelineArgs = [:] - batchLoadTestJob(delegate, CommonTestProperties.TriggeringContext.PR) - } diff --git a/.test-infra/jenkins/job_LoadTests_Combine_Go.groovy b/.test-infra/jenkins/job_LoadTests_Combine_Go.groovy deleted file mode 100644 index 0d6b7b8bb5b4..000000000000 --- a/.test-infra/jenkins/job_LoadTests_Combine_Go.groovy +++ /dev/null @@ -1,127 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonJobProperties as commonJobProperties -import CommonTestProperties -import LoadTestsBuilder as loadTestsBuilder -import PhraseTriggeringPostCommitBuilder -import InfluxDBCredentialsHelper - -import static LoadTestsBuilder.GO_SDK_CONTAINER - - -String now = new Date().format('MMddHHmmss', TimeZone.getTimeZone('UTC')) - -def batchScenarios = { - [ - [ - title : 'Combine Go Load test: 2GB of 10B records', - test : 'combine', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - job_name : "load-tests-go-dataflow-batch-combine-1-${now}", - project : 'apache-beam-testing', - region : 'us-central1', - temp_location : 'gs://temp-storage-for-perf-tests/loadtests', - staging_location : 'gs://temp-storage-for-perf-tests/loadtests', - influx_namespace : 'dataflow', - influx_measurement : 'go_batch_combine_1', - input_options : '\'{' + - '"num_records": 200000000,' + - '"key_size": 1,' + - '"value_size": 9}\'', - fanout : 1, - top_count : 20, - num_workers : 5, - autoscaling_algorithm: 'NONE', - environment_type : 'DOCKER', - environment_config : GO_SDK_CONTAINER, - ] - ], - [ - title : 'Combine Go Load test: fanout 4 times with 2GB 10-byte records total', - test : 'combine', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - job_name : "load-tests-go-dataflow-batch-combine-4-${now}", - project : 'apache-beam-testing', - region : 'us-central1', - temp_location : 'gs://temp-storage-for-perf-tests/loadtests', - staging_location : 'gs://temp-storage-for-perf-tests/loadtests', - influx_namespace : 'dataflow', - influx_measurement : 'go_batch_combine_4', - input_options : '\'{' + - '"num_records": 5000000,' + - '"key_size": 10,' + - '"value_size": 90}\'', - fanout : 4, - top_count : 20, - num_workers : 16, - autoscaling_algorithm: 'NONE', - environment_type : 'DOCKER', - environment_config : GO_SDK_CONTAINER, - ] - ], - [ - title : 'Combine Go Load test: fanout 8 times with 2GB 10-byte records total', - test : 'combine', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - job_name : "load-tests-go-dataflow-batch-combine-5-${now}", - project : 'apache-beam-testing', - region : 'us-central1', - temp_location : 'gs://temp-storage-for-perf-tests/loadtests', - staging_location : 'gs://temp-storage-for-perf-tests/loadtests', - influx_namespace : 'dataflow', - influx_measurement : 'go_batch_combine_5', - input_options : '\'{' + - '"num_records": 2500000,' + - '"key_size": 10,' + - '"value_size": 90}\'', - fanout : 8, - top_count : 20, - num_workers : 16, - autoscaling_algorithm: 'NONE', - environment_type : 'DOCKER', - environment_config : GO_SDK_CONTAINER, - ] - ], - ].each { test -> test.pipelineOptions.putAll(additionalPipelineArgs) } -} - -def loadTestJob = { scope, triggeringContext, mode -> - loadTestsBuilder.loadTests(scope, CommonTestProperties.SDK.GO, batchScenarios(), 'combine', mode) -} - -PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_LoadTests_Go_Combine_Dataflow_Batch', - 'Run Load Tests Go Combine Dataflow Batch', - 'Load Tests Go Combine Dataflow Batch suite', - this - ) { - additionalPipelineArgs = [:] - loadTestJob(delegate, CommonTestProperties.TriggeringContext.PR, 'batch') - } - -CronJobBuilder.cronJob('beam_LoadTests_Go_Combine_Dataflow_Batch', 'H H * * *', this) { - additionalPipelineArgs = [ - influx_db_name: InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influx_hostname: InfluxDBCredentialsHelper.InfluxDBHostUrl, - ] - loadTestJob(delegate, CommonTestProperties.TriggeringContext.POST_COMMIT, 'batch') -} diff --git a/.test-infra/jenkins/job_LoadTests_Combine_Java.groovy b/.test-infra/jenkins/job_LoadTests_Combine_Java.groovy deleted file mode 100644 index a4254ca21949..000000000000 --- a/.test-infra/jenkins/job_LoadTests_Combine_Java.groovy +++ /dev/null @@ -1,162 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - -import CommonJobProperties as commonJobProperties -import CommonTestProperties -import CronJobBuilder -import LoadTestsBuilder as loadTestsBuilder -import PhraseTriggeringPostCommitBuilder -import InfluxDBCredentialsHelper - -def commonLoadTestConfig = { jobType, isStreaming -> - [ - [ - title : 'Load test: 2GB of 10B records', - test : 'org.apache.beam.sdk.loadtests.CombineLoadTest', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - project : 'apache-beam-testing', - region : 'us-central1', - appName : "load_tests_Java_Dataflow_${jobType}_Combine_1", - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${jobType}_combine_1", - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 200000000, - "keySizeBytes": 1, - "valueSizeBytes": 9 - } - """.trim().replaceAll("\\s", ""), - fanout : 1, - iterations : 1, - topCount : 20, - numWorkers : 5, - autoscalingAlgorithm: "NONE", - perKeyCombiner : "TOP_LARGEST", - streaming : isStreaming - ] - ], - [ - title : 'Load test: fanout 4 times with 2GB 10-byte records total', - test : 'org.apache.beam.sdk.loadtests.CombineLoadTest', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - project : 'apache-beam-testing', - region : 'us-central1', - appName : "load_tests_Java_Dataflow_${jobType}_Combine_4", - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${jobType}_combine_4", - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 5000000, - "keySizeBytes": 10, - "valueSizeBytes": 90 - } - """.trim().replaceAll("\\s", ""), - fanout : 4, - iterations : 1, - topCount : 20, - numWorkers : 16, - autoscalingAlgorithm: "NONE", - perKeyCombiner : "TOP_LARGEST", - streaming : isStreaming - ] - ], - [ - title : 'Load test: fanout 8 times with 2GB 10-byte records total', - test : 'org.apache.beam.sdk.loadtests.CombineLoadTest', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - project : 'apache-beam-testing', - region : 'us-central1', - appName : "load_tests_Java_Dataflow_${jobType}_Combine_5", - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${jobType}_combine_5", - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 2500000, - "keySizeBytes": 10, - "valueSizeBytes": 90 - } - """.trim().replaceAll("\\s", ""), - fanout : 8, - iterations : 1, - topCount : 20, - numWorkers : 16, - autoscalingAlgorithm: "NONE", - perKeyCombiner : "TOP_LARGEST", - streaming : isStreaming - ] - ] - ].each { test -> test.pipelineOptions.putAll(additionalPipelineArgs) } -} - - -def batchLoadTestJob = { scope, triggeringContext -> - loadTestsBuilder.loadTests(scope, CommonTestProperties.SDK.JAVA, commonLoadTestConfig('batch', false), "Combine", "batch") -} - -def streamingLoadTestJob = {scope, triggeringContext -> - scope.description('Runs Java Combine load tests on Dataflow runner in streaming mode') - commonJobProperties.setTopLevelMainJobProperties(scope, 'master', 240) - - for (testConfiguration in commonLoadTestConfig('streaming', true)) { - testConfiguration.pipelineOptions << [inputWindowDurationSec: 1200] - loadTestsBuilder.loadTest(scope, testConfiguration.title, testConfiguration.runner, CommonTestProperties.SDK.JAVA, testConfiguration.pipelineOptions, testConfiguration.test) - } -} - -CronJobBuilder.cronJob('beam_LoadTests_Java_Combine_Dataflow_Batch', 'H H * * *', this) { - additionalPipelineArgs = [ - influxDatabase: InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influxHost: InfluxDBCredentialsHelper.InfluxDBHostUrl, - ] - batchLoadTestJob(delegate, CommonTestProperties.TriggeringContext.POST_COMMIT) -} - -CronJobBuilder.cronJob('beam_LoadTests_Java_Combine_Dataflow_Streaming', 'H H * * *', this) { - additionalPipelineArgs = [ - influxDatabase: InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influxHost: InfluxDBCredentialsHelper.InfluxDBHostUrl, - ] - streamingLoadTestJob(delegate, CommonTestProperties.TriggeringContext.POST_COMMIT) -} - -PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_LoadTests_Java_Combine_Dataflow_Batch', - 'Run Load Tests Java Combine Dataflow Batch', - 'Load Tests Java Combine Dataflow Batch suite', - this - ) { - additionalPipelineArgs = [:] - batchLoadTestJob(delegate, CommonTestProperties.TriggeringContext.PR) - } - -PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_LoadTests_Java_Combine_Dataflow_Streaming', - 'Run Load Tests Java Combine Dataflow Streaming', - 'Load Tests Java Combine Dataflow Streaming suite', - this - ) { - additionalPipelineArgs = [:] - streamingLoadTestJob(delegate, CommonTestProperties.TriggeringContext.PR) - } diff --git a/.test-infra/jenkins/job_LoadTests_Combine_Java_Smoke.groovy b/.test-infra/jenkins/job_LoadTests_Combine_Java_Smoke.groovy deleted file mode 100644 index fa22932716d9..000000000000 --- a/.test-infra/jenkins/job_LoadTests_Combine_Java_Smoke.groovy +++ /dev/null @@ -1,85 +0,0 @@ - -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonTestProperties -import LoadTestsBuilder as loadTestsBuilder -import PhraseTriggeringPostCommitBuilder - -import static LoadTestConfig.fromTemplate -import static LoadTestConfig.templateConfig - -def smokeTestConfigurations = { - def template = templateConfig { - title 'CombineLoadTest load test Dataflow-1' - test 'org.apache.beam.sdk.loadtests.CombineLoadTest' - dataflow() - pipelineOptions { - java() - appName 'smoke-dsl-java' - project 'apache-beam-testing' - tempLocation 'gs://temp-storage-for-perf-tests/smoketests' - numWorkers 5 - autoscalingAlgorithm 'NONE' - sourceOptions { - numRecords 100000 - splitPointFrequencyRecords 1 - } - stepOptions { - outputRecordsPerInputRecord 1 - preservesInputKeyDistribution true - } - specificParameters([ - fanout: 10, - iterations: 1 - ]) - } - } - [ - fromTemplate(template), - fromTemplate(template) { - title 'CombineLoadTest load test Dataflow-2' - pipelineOptions { - numWorkers 3 - specificParameters([ - fanout: 1 - ]) - } - }, - fromTemplate(template) { - title 'CombineLoadTest load test Dataflow-3' - pipelineOptions { - sourceOptions { - numRecords 20000 - } - } - }, - ] -} - - - -// Runs a tiny version load test suite to ensure nothing is broken. -PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_Java_LoadTests_Combine_Smoke', - 'Run Java Load Tests Combine Smoke', - 'Java Load Tests Combine Smoke', - this - ) { - loadTestsBuilder.loadTests(delegate, CommonTestProperties.SDK.JAVA, smokeTestConfigurations(), "Combine", "smoke") - } diff --git a/.test-infra/jenkins/job_LoadTests_Combine_Java_spark_structured_streaming.groovy b/.test-infra/jenkins/job_LoadTests_Combine_Java_spark_structured_streaming.groovy deleted file mode 100644 index ff1dc2bf190a..000000000000 --- a/.test-infra/jenkins/job_LoadTests_Combine_Java_spark_structured_streaming.groovy +++ /dev/null @@ -1,123 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonTestProperties -import CronJobBuilder -import LoadTestsBuilder as loadTestsBuilder -import PhraseTriggeringPostCommitBuilder -import InfluxDBCredentialsHelper - -def commonLoadTestConfig = { jobType, isStreaming -> - [ - [ - title : 'Load test: 2GB of 10B records', - test : 'org.apache.beam.sdk.loadtests.CombineLoadTest', - runner : CommonTestProperties.Runner.SPARK_STRUCTURED_STREAMING, - pipelineOptions: [ - project : 'apache-beam-testing', - appName : "load_tests_Java_SparkStructuredStreaming_${jobType}_Combine_1", - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${jobType}_combine_1", - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 200000000, - "keySizeBytes": 1, - "valueSizeBytes": 9 - } - """.trim().replaceAll("\\s", ""), - fanout : 1, - iterations : 1, - topCount : 20, - perKeyCombiner : "TOP_LARGEST", - streaming : isStreaming - ] - ], - [ - title : 'Load test: fanout 4 times with 2GB 10-byte records total', - test : 'org.apache.beam.sdk.loadtests.CombineLoadTest', - runner : CommonTestProperties.Runner.SPARK_STRUCTURED_STREAMING, - pipelineOptions: [ - project : 'apache-beam-testing', - appName : "load_tests_Java_SparkStructuredStreaming_${jobType}_Combine_4", - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${jobType}_combine_4", - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 5000000, - "keySizeBytes": 10, - "valueSizeBytes": 90 - } - """.trim().replaceAll("\\s", ""), - fanout : 4, - iterations : 1, - topCount : 20, - perKeyCombiner : "TOP_LARGEST", - streaming : isStreaming - ] - ], - [ - title : 'Load test: fanout 8 times with 2GB 10-byte records total', - test : 'org.apache.beam.sdk.loadtests.CombineLoadTest', - runner : CommonTestProperties.Runner.SPARK_STRUCTURED_STREAMING, - pipelineOptions: [ - project : 'apache-beam-testing', - appName : "load_tests_Java_SparkStructuredStreaming_${jobType}_Combine_5", - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${jobType}_combine_5", - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 2500000, - "keySizeBytes": 10, - "valueSizeBytes": 90 - } - """.trim().replaceAll("\\s", ""), - fanout : 8, - iterations : 1, - topCount : 20, - perKeyCombiner : "TOP_LARGEST", - streaming : isStreaming - ] - ] - ].each { test -> test.pipelineOptions.putAll(additionalPipelineArgs) } -} - - -def batchLoadTestJob = { scope, triggeringContext -> - loadTestsBuilder.loadTests(scope, CommonTestProperties.SDK.JAVA, commonLoadTestConfig('batch', false), "Combine", "batch") -} - -CronJobBuilder.cronJob('beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch', 'H H * * *', this) { - additionalPipelineArgs = [ - influxDatabase: InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influxHost: InfluxDBCredentialsHelper.InfluxDBHostUrl, - ] - batchLoadTestJob(delegate, CommonTestProperties.TriggeringContext.POST_COMMIT) -} - -PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch', - 'Run Load Tests Java Combine SparkStructuredStreaming Batch', - 'Load Tests Java Combine SparkStructuredStreaming Batch suite', - this - ) { - additionalPipelineArgs = [:] - batchLoadTestJob(delegate, CommonTestProperties.TriggeringContext.PR) - } diff --git a/.test-infra/jenkins/job_LoadTests_FnApiRunner_Python.groovy b/.test-infra/jenkins/job_LoadTests_FnApiRunner_Python.groovy deleted file mode 100644 index 538ef2be908c..000000000000 --- a/.test-infra/jenkins/job_LoadTests_FnApiRunner_Python.groovy +++ /dev/null @@ -1,74 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonJobProperties as commonJobProperties -import LoadTestsBuilder as loadTestsBuilder -import PhraseTriggeringPostCommitBuilder -import InfluxDBCredentialsHelper - -def now = new Date().format("MMddHHmmss", TimeZone.getTimeZone('UTC')) - -def loadTestConfigurations = { datasetName -> - [ - [ - title : 'FnApiRunner Python load test - microbenchmark', - test : 'apache_beam.testing.load_tests.microbenchmarks_test', - runner : CommonTestProperties.Runner.DIRECT, - pipelineOptions: [ - publish_to_big_query: true, - influx_measurement : 'python_direct_microbenchmarks', - project : 'apache-beam-testing', - metrics_dataset : datasetName, - metrics_table : 'python_direct_microbenchmarks', - input_options : '\'{}\'', - ] - ], - ] - .each { test -> test.pipelineOptions.putAll(additionalPipelineArgs) } -} - -def loadTestJob = { scope, triggeringContext -> - scope.description("Runs Python FnApiRunner Microbenchmark") - commonJobProperties.setTopLevelMainJobProperties(scope, 'master', 120) - - def datasetName = loadTestsBuilder.getBigQueryDataset('load_test', triggeringContext) - for (testConfiguration in loadTestConfigurations(datasetName)) { - loadTestsBuilder.loadTest(scope, testConfiguration.title, testConfiguration.runner, CommonTestProperties.SDK.PYTHON, testConfiguration.pipelineOptions, testConfiguration.test) - } -} - -PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_Python_LoadTests_FnApiRunner_Microbenchmark', - 'Run Python Load Tests FnApiRunner Microbenchmark', - 'Python Load Tests FnApiRunner Microbenchmark', - this - ) { - additionalPipelineArgs = [:] - loadTestJob(delegate, CommonTestProperties.TriggeringContext.PR) - } - - -// Run this job every 6 hours on a random minute. -CronJobBuilder.cronJob('beam_Python_LoadTests_FnApiRunner_Microbenchmark', 'H H/6 * * *', this) { - additionalPipelineArgs = [ - influx_db_name: InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influx_hostname: InfluxDBCredentialsHelper.InfluxDBHostUrl, - ] - loadTestJob(delegate, CommonTestProperties.TriggeringContext.POST_COMMIT) -} - diff --git a/.test-infra/jenkins/job_LoadTests_GBK_Dataflow_V2_Java11.groovy b/.test-infra/jenkins/job_LoadTests_GBK_Dataflow_V2_Java11.groovy deleted file mode 100644 index 2191e448fade..000000000000 --- a/.test-infra/jenkins/job_LoadTests_GBK_Dataflow_V2_Java11.groovy +++ /dev/null @@ -1,311 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonJobProperties as commonJobProperties -import CommonTestProperties -import LoadTestsBuilder as loadTestsBuilder -import PhraseTriggeringPostCommitBuilder -import CronJobBuilder -import InfluxDBCredentialsHelper - -def loadTestConfigurations = { mode, isStreaming -> - [ - [ - title : 'Load test: 2GB of 10B records', - test : 'org.apache.beam.sdk.loadtests.GroupByKeyLoadTest', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - project : 'apache-beam-testing', - region : 'us-central1', - appName : "load_tests_Java11_Dataflow_V2_${mode}_GBK_1", - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${mode}_gbk_1", - influxTags : """ - { - "runnerVersion": "v2", - "jdk": "java11" - } - """.trim().replaceAll("\\s", ""), - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 200000000, - "keySizeBytes": 1, - "valueSizeBytes": 9 - } - """.trim().replaceAll("\\s", ""), - fanout : 1, - iterations : 1, - numWorkers : 5, - autoscalingAlgorithm : "NONE", - streaming : isStreaming - ] - ], - [ - title : 'Load test: 2GB of 100B records', - test : 'org.apache.beam.sdk.loadtests.GroupByKeyLoadTest', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - project : 'apache-beam-testing', - region : 'us-central1', - appName : "load_tests_Java11_Dataflow_V2_${mode}_GBK_2", - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${mode}_gbk_2", - influxTags : """ - { - "runnerVersion": "v2", - "jdk": "java11" - } - """.trim().replaceAll("\\s", ""), - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 20000000, - "keySizeBytes": 10, - "valueSizeBytes": 90 - } - """.trim().replaceAll("\\s", ""), - fanout : 1, - iterations : 1, - numWorkers : 5, - autoscalingAlgorithm : "NONE", - streaming : isStreaming - ] - ], - [ - - title : 'Load test: 2GB of 100kB records', - test : 'org.apache.beam.sdk.loadtests.GroupByKeyLoadTest', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - project : 'apache-beam-testing', - region : 'us-central1', - appName : "load_tests_Java11_Dataflow_V2_${mode}_GBK_3", - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${mode}_gbk_3", - influxTags : """ - { - "runnerVersion": "v2", - "jdk": "java11" - } - """.trim().replaceAll("\\s", ""), - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 20000, - "keySizeBytes": 10000, - "valueSizeBytes": 90000 - } - """.trim().replaceAll("\\s", ""), - fanout : 1, - iterations : 1, - numWorkers : 5, - autoscalingAlgorithm : "NONE", - streaming : isStreaming - ] - - ], - [ - title : 'Load test: fanout 4 times with 2GB 10-byte records total', - test : 'org.apache.beam.sdk.loadtests.GroupByKeyLoadTest', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - project : 'apache-beam-testing', - region : 'us-central1', - appName : "load_tests_Java11_Dataflow_V2_${mode}_GBK_4", - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${mode}_gbk_4", - influxTags : """ - { - "runnerVersion": "v2", - "jdk": "java11" - } - """.trim().replaceAll("\\s", ""), - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 5000000, - "keySizeBytes": 10, - "valueSizeBytes": 90 - } - """.trim().replaceAll("\\s", ""), - fanout : 4, - iterations : 1, - numWorkers : 16, - autoscalingAlgorithm : "NONE", - streaming : isStreaming - ] - ], - [ - title : 'Load test: fanout 8 times with 2GB 10-byte records total', - test : 'org.apache.beam.sdk.loadtests.GroupByKeyLoadTest', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - project : 'apache-beam-testing', - region : 'us-central1', - appName : "load_tests_Java11_Dataflow_V2_${mode}_GBK_5", - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${mode}_gbk_5", - influxTags : """ - { - "runnerVersion": "v2", - "jdk": "java11" - } - """.trim().replaceAll("\\s", ""), - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 2500000, - "keySizeBytes": 10, - "valueSizeBytes": 90 - } - """.trim().replaceAll("\\s", ""), - fanout : 8, - iterations : 1, - numWorkers : 16, - autoscalingAlgorithm : "NONE", - streaming : isStreaming - ] - ], - [ - title : 'Load test: reiterate 4 times 10kB values', - test : 'org.apache.beam.sdk.loadtests.GroupByKeyLoadTest', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - project : 'apache-beam-testing', - region : 'us-central1', - appName : "load_tests_Java11_Dataflow_V2_${mode}_GBK_6", - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${mode}_gbk_6", - influxTags : """ - { - "runnerVersion": "v2", - "jdk": "java11" - } - """.trim().replaceAll("\\s", ""), - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 20000000, - "keySizeBytes": 10, - "valueSizeBytes": 90, - "numHotKeys": 200, - "hotKeyFraction": 1 - } - """.trim().replaceAll("\\s", ""), - fanout : 1, - iterations : 4, - numWorkers : 5, - autoscalingAlgorithm : "NONE", - streaming : isStreaming - ] - ], - [ - title : 'Load test: reiterate 4 times 2MB values', - test : 'org.apache.beam.sdk.loadtests.GroupByKeyLoadTest', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - project : 'apache-beam-testing', - region : 'us-central1', - appName : "load_tests_Java11_Dataflow_V2_${mode}_GBK_7", - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${mode}_gbk_7", - influxTags : """ - { - "runnerVersion": "v2", - "jdk": "java11" - } - """.trim().replaceAll("\\s", ""), - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 20000000, - "keySizeBytes": 10, - "valueSizeBytes": 90, - "numHotKeys": 10, - "hotKeyFraction": 1 - } - """.trim().replaceAll("\\s", ""), - fanout : 1, - iterations : 4, - numWorkers : 5, - autoscalingAlgorithm : "NONE", - streaming : isStreaming - ] - ] - ].each { test -> test.pipelineOptions.putAll(additionalPipelineArgs) } -} - -def final JOB_SPECIFIC_SWITCHES = [ - '-Prunner.version="V2"', - '-PtestJavaVersion=11', - "-Pjava11Home=${commonJobProperties.JAVA_11_HOME}" -] - -def streamingLoadTestJob = { scope, triggeringContext -> - scope.description('Runs Java 11 GBK load tests on Dataflow runner V2 in streaming mode') - commonJobProperties.setTopLevelMainJobProperties(scope, 'master', 240) - - for (testConfiguration in loadTestConfigurations('streaming', true)) { - testConfiguration.pipelineOptions << [inputWindowDurationSec: 1200] - loadTestsBuilder.loadTest(scope, testConfiguration.title, testConfiguration.runner, CommonTestProperties.SDK.JAVA, - testConfiguration.pipelineOptions, testConfiguration.test, JOB_SPECIFIC_SWITCHES) - } -} - -CronJobBuilder.cronJob('beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11', 'H H * * *', this) { - additionalPipelineArgs = [ - influxDatabase: InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influxHost: InfluxDBCredentialsHelper.InfluxDBHostUrl, - ] - streamingLoadTestJob(delegate, CommonTestProperties.TriggeringContext.POST_COMMIT) -} - -PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11', - 'Run Load Tests Java 11 GBK Dataflow V2 Streaming', - 'Load Tests Java 11 GBK Dataflow V2 Streaming suite', - this - ) { - additionalPipelineArgs = [:] - streamingLoadTestJob(delegate, CommonTestProperties.TriggeringContext.PR) - } - - -def batchLoadTestJob = { scope, triggeringContext -> - loadTestsBuilder.loadTests(scope, CommonTestProperties.SDK.JAVA, loadTestConfigurations('batch', false), - "GBK", "batch", JOB_SPECIFIC_SWITCHES) -} - -CronJobBuilder.cronJob('beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11', 'H H * * *', this) { - additionalPipelineArgs = [ - influxDatabase: InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influxHost: InfluxDBCredentialsHelper.InfluxDBHostUrl, - ] - batchLoadTestJob(delegate, CommonTestProperties.TriggeringContext.POST_COMMIT) -} - -PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11', - 'Run Load Tests Java 11 GBK Dataflow V2 Batch', - 'Load Tests Java 11 GBK Dataflow V2 Batch suite', - this - ) { - additionalPipelineArgs = [:] - batchLoadTestJob(delegate, CommonTestProperties.TriggeringContext.PR) - } diff --git a/.test-infra/jenkins/job_LoadTests_GBK_Dataflow_V2_Java17.groovy b/.test-infra/jenkins/job_LoadTests_GBK_Dataflow_V2_Java17.groovy deleted file mode 100644 index 2520f68f0178..000000000000 --- a/.test-infra/jenkins/job_LoadTests_GBK_Dataflow_V2_Java17.groovy +++ /dev/null @@ -1,311 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonJobProperties as commonJobProperties -import CommonTestProperties -import LoadTestsBuilder as loadTestsBuilder -import PhraseTriggeringPostCommitBuilder -import CronJobBuilder -import InfluxDBCredentialsHelper - -def loadTestConfigurations = { mode, isStreaming -> - [ - [ - title : 'Load test: 2GB of 10B records', - test : 'org.apache.beam.sdk.loadtests.GroupByKeyLoadTest', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - project : 'apache-beam-testing', - region : 'us-central1', - appName : "load_tests_Java17_Dataflow_V2_${mode}_GBK_1", - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${mode}_gbk_1", - influxTags : """ - { - "runnerVersion": "v2", - "jdk": "java17" - } - """.trim().replaceAll("\\s", ""), - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 200000000, - "keySizeBytes": 1, - "valueSizeBytes": 9 - } - """.trim().replaceAll("\\s", ""), - fanout : 1, - iterations : 1, - numWorkers : 5, - autoscalingAlgorithm : "NONE", - streaming : isStreaming - ] - ], - [ - title : 'Load test: 2GB of 100B records', - test : 'org.apache.beam.sdk.loadtests.GroupByKeyLoadTest', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - project : 'apache-beam-testing', - region : 'us-central1', - appName : "load_tests_Java17_Dataflow_V2_${mode}_GBK_2", - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${mode}_gbk_2", - influxTags : """ - { - "runnerVersion": "v2", - "jdk": "java17" - } - """.trim().replaceAll("\\s", ""), - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 20000000, - "keySizeBytes": 10, - "valueSizeBytes": 90 - } - """.trim().replaceAll("\\s", ""), - fanout : 1, - iterations : 1, - numWorkers : 5, - autoscalingAlgorithm : "NONE", - streaming : isStreaming - ] - ], - [ - - title : 'Load test: 2GB of 100kB records', - test : 'org.apache.beam.sdk.loadtests.GroupByKeyLoadTest', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - project : 'apache-beam-testing', - region : 'us-central1', - appName : "load_tests_Java17_Dataflow_V2_${mode}_GBK_3", - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${mode}_gbk_3", - influxTags : """ - { - "runnerVersion": "v2", - "jdk": "java17" - } - """.trim().replaceAll("\\s", ""), - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 20000, - "keySizeBytes": 10000, - "valueSizeBytes": 90000 - } - """.trim().replaceAll("\\s", ""), - fanout : 1, - iterations : 1, - numWorkers : 5, - autoscalingAlgorithm : "NONE", - streaming : isStreaming - ] - - ], - [ - title : 'Load test: fanout 4 times with 2GB 10-byte records total', - test : 'org.apache.beam.sdk.loadtests.GroupByKeyLoadTest', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - project : 'apache-beam-testing', - region : 'us-central1', - appName : "load_tests_Java17_Dataflow_V2_${mode}_GBK_4", - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${mode}_gbk_4", - influxTags : """ - { - "runnerVersion": "v2", - "jdk": "java17" - } - """.trim().replaceAll("\\s", ""), - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 5000000, - "keySizeBytes": 10, - "valueSizeBytes": 90 - } - """.trim().replaceAll("\\s", ""), - fanout : 4, - iterations : 1, - numWorkers : 16, - autoscalingAlgorithm : "NONE", - streaming : isStreaming - ] - ], - [ - title : 'Load test: fanout 8 times with 2GB 10-byte records total', - test : 'org.apache.beam.sdk.loadtests.GroupByKeyLoadTest', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - project : 'apache-beam-testing', - region : 'us-central1', - appName : "load_tests_Java17_Dataflow_V2_${mode}_GBK_5", - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${mode}_gbk_5", - influxTags : """ - { - "runnerVersion": "v2", - "jdk": "java17" - } - """.trim().replaceAll("\\s", ""), - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 2500000, - "keySizeBytes": 10, - "valueSizeBytes": 90 - } - """.trim().replaceAll("\\s", ""), - fanout : 8, - iterations : 1, - numWorkers : 16, - autoscalingAlgorithm : "NONE", - streaming : isStreaming - ] - ], - [ - title : 'Load test: reiterate 4 times 10kB values', - test : 'org.apache.beam.sdk.loadtests.GroupByKeyLoadTest', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - project : 'apache-beam-testing', - region : 'us-central1', - appName : "load_tests_Java17_Dataflow_V2_${mode}_GBK_6", - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${mode}_gbk_6", - influxTags : """ - { - "runnerVersion": "v2", - "jdk": "java17" - } - """.trim().replaceAll("\\s", ""), - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 20000000, - "keySizeBytes": 10, - "valueSizeBytes": 90, - "numHotKeys": 200, - "hotKeyFraction": 1 - } - """.trim().replaceAll("\\s", ""), - fanout : 1, - iterations : 4, - numWorkers : 5, - autoscalingAlgorithm : "NONE", - streaming : isStreaming - ] - ], - [ - title : 'Load test: reiterate 4 times 2MB values', - test : 'org.apache.beam.sdk.loadtests.GroupByKeyLoadTest', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - project : 'apache-beam-testing', - region : 'us-central1', - appName : "load_tests_Java17_Dataflow_V2_${mode}_GBK_7", - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${mode}_gbk_7", - influxTags : """ - { - "runnerVersion": "v2", - "jdk": "java17" - } - """.trim().replaceAll("\\s", ""), - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 20000000, - "keySizeBytes": 10, - "valueSizeBytes": 90, - "numHotKeys": 10, - "hotKeyFraction": 1 - } - """.trim().replaceAll("\\s", ""), - fanout : 1, - iterations : 4, - numWorkers : 5, - autoscalingAlgorithm : "NONE", - streaming : isStreaming - ] - ] - ].each { test -> test.pipelineOptions.putAll(additionalPipelineArgs) } -} - -def final JOB_SPECIFIC_SWITCHES = [ - '-Prunner.version="V2"', - '-PtestJavaVersion=17', - "-Pjava17Home=${commonJobProperties.JAVA_17_HOME}" -] - -def streamingLoadTestJob = { scope, triggeringContext -> - scope.description('Runs Java 17 GBK load tests on Dataflow runner V2 in streaming mode') - commonJobProperties.setTopLevelMainJobProperties(scope, 'master', 240) - - for (testConfiguration in loadTestConfigurations('streaming', true)) { - testConfiguration.pipelineOptions << [inputWindowDurationSec: 1200] - loadTestsBuilder.loadTest(scope, testConfiguration.title, testConfiguration.runner, CommonTestProperties.SDK.JAVA, - testConfiguration.pipelineOptions, testConfiguration.test, JOB_SPECIFIC_SWITCHES) - } -} - -CronJobBuilder.cronJob('beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17', 'H H * * *', this) { - additionalPipelineArgs = [ - influxDatabase: InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influxHost: InfluxDBCredentialsHelper.InfluxDBHostUrl, - ] - streamingLoadTestJob(delegate, CommonTestProperties.TriggeringContext.POST_COMMIT) -} - -PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17', - 'Run Load Tests Java 17 GBK Dataflow V2 Streaming', - 'Load Tests Java 17 GBK Dataflow V2 Streaming suite', - this - ) { - additionalPipelineArgs = [:] - streamingLoadTestJob(delegate, CommonTestProperties.TriggeringContext.PR) - } - - -def batchLoadTestJob = { scope, triggeringContext -> - loadTestsBuilder.loadTests(scope, CommonTestProperties.SDK.JAVA, loadTestConfigurations('batch', false), - "GBK", "batch", JOB_SPECIFIC_SWITCHES) -} - -CronJobBuilder.cronJob('beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17', 'H H * * *', this) { - additionalPipelineArgs = [ - influxDatabase: InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influxHost: InfluxDBCredentialsHelper.InfluxDBHostUrl, - ] - batchLoadTestJob(delegate, CommonTestProperties.TriggeringContext.POST_COMMIT) -} - -PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17', - 'Run Load Tests Java 17 GBK Dataflow V2 Batch', - 'Load Tests Java 17 GBK Dataflow V2 Batch suite', - this - ) { - additionalPipelineArgs = [:] - batchLoadTestJob(delegate, CommonTestProperties.TriggeringContext.PR) - } diff --git a/.test-infra/jenkins/job_LoadTests_GBK_Flink_Python.groovy b/.test-infra/jenkins/job_LoadTests_GBK_Flink_Python.groovy deleted file mode 100644 index 25e2647ebf32..000000000000 --- a/.test-infra/jenkins/job_LoadTests_GBK_Flink_Python.groovy +++ /dev/null @@ -1,172 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonJobProperties as commonJobProperties -import CommonTestProperties -import LoadTestsBuilder as loadTestsBuilder -import PhraseTriggeringPostCommitBuilder -import Flink -import InfluxDBCredentialsHelper - -import static LoadTestsBuilder.DOCKER_CONTAINER_REGISTRY -import static LoadTestsBuilder.DOCKER_BEAM_JOBSERVER -import static LoadTestsBuilder.DOCKER_BEAM_SDK_IMAGE - -String now = new Date().format("MMddHHmmss", TimeZone.getTimeZone('UTC')) - -def scenarios = { datasetName -> - [ - [ - title : 'Load test: 2GB of 10B records', - test : 'apache_beam.testing.load_tests.group_by_key_test', - runner : CommonTestProperties.Runner.PORTABLE, - pipelineOptions: [ - job_name : "load_tests_Python_Flink_Batch_GBK_1_${now}", - publish_to_big_query: true, - project : 'apache-beam-testing', - metrics_dataset : datasetName, - metrics_table : "python_flink_batch_GBK_1", - influx_measurement : 'python_batch_gkb_1', - input_options : '\'{"num_records": 200000000,"key_size": 1,"value_size":9}\'', - iterations : 1, - fanout : 1, - parallelism : 5, - job_endpoint : 'localhost:8099', - environment_type : 'DOCKER', - environment_config : "${DOCKER_CONTAINER_REGISTRY}/${DOCKER_BEAM_SDK_IMAGE}", - ] - ], - [ - title : 'Load test: 2GB of 100B records', - test : 'apache_beam.testing.load_tests.group_by_key_test', - runner : CommonTestProperties.Runner.PORTABLE, - pipelineOptions: [ - job_name : "load_tests_Python_Flink_Batch_GBK_2_${now}", - publish_to_big_query: true, - project : 'apache-beam-testing', - metrics_dataset : datasetName, - metrics_table : "python_flink_batch_GBK_2", - influx_measurement : 'python_batch_gbk_2', - input_options : '\'{"num_records": 20000000,"key_size": 10,"value_size":90}\'', - iterations : 1, - fanout : 1, - parallelism : 5, - job_endpoint : 'localhost:8099', - environment_type : 'DOCKER', - environment_config : "${DOCKER_CONTAINER_REGISTRY}/${DOCKER_BEAM_SDK_IMAGE}", - ] - ], - [ - title : 'Load test: fanout 4 times with 2GB 10-byte records total', - test : 'apache_beam.testing.load_tests.group_by_key_test', - runner : CommonTestProperties.Runner.PORTABLE, - pipelineOptions: [ - job_name : "load_tests_Python_Flink_Batch_GBK_4_${now}", - publish_to_big_query: true, - project : 'apache-beam-testing', - metrics_dataset : datasetName, - metrics_table : "python_flink_batch_GBK_4", - influx_measurement : 'python_batch_gbk_4', - input_options : '\'{"num_records": 5000000,"key_size": 10,"value_size":90}\'', - iterations : 1, - fanout : 4, - parallelism : 16, - job_endpoint : 'localhost:8099', - environment_type : 'DOCKER', - environment_config : "${DOCKER_CONTAINER_REGISTRY}/${DOCKER_BEAM_SDK_IMAGE}", - ] - ], - [ - title : 'Load test: fanout 8 times with 2GB 10-byte records total', - test : 'apache_beam.testing.load_tests.group_by_key_test', - runner : CommonTestProperties.Runner.PORTABLE, - pipelineOptions: [ - job_name : "load_tests_Python_Flink_Batch_GBK_5_${now}", - publish_to_big_query: true, - project : 'apache-beam-testing', - metrics_dataset : datasetName, - metrics_table : "python_flink_batch_GBK_5", - influx_measurement : 'python_batch_gbk_5', - input_options : '\'{"num_records":2500000,"key_size":10,"value_size":90,"algorithm":"lcg"}\'', - iterations : 1, - fanout : 8, - parallelism : 16, - job_endpoint : 'localhost:8099', - environment_type : 'DOCKER', - environment_config : "${DOCKER_CONTAINER_REGISTRY}/${DOCKER_BEAM_SDK_IMAGE}", - ] - ], - [ - title : 'Load test: reiterate 4 times 10kB values', - test : 'apache_beam.testing.load_tests.group_by_key_test', - runner : CommonTestProperties.Runner.PORTABLE, - pipelineOptions: [ - job_name : "load_tests_Python_Flink_Batch_GBK_6_${now}", - publish_to_big_query: true, - project : 'apache-beam-testing', - metrics_dataset : datasetName, - metrics_table : "python_flink_batch_GBK_6", - influx_measurement : 'python_batch_gbk_6', - input_options : '\'{"num_records":20000000,"key_size":10,"value_size":90,"num_hot_keys":200,"hot_key_fraction":1,"algorithm":"lcg"}\'', - iterations : 4, - fanout : 1, - parallelism : 5, - job_endpoint : 'localhost:8099', - environment_type : 'DOCKER', - environment_config : "${DOCKER_CONTAINER_REGISTRY}/${DOCKER_BEAM_SDK_IMAGE}", - ] - ], - ].each { test -> test.pipelineOptions.putAll(additionalPipelineArgs) } -} - -def loadTest = { scope, triggeringContext -> - def sdk = CommonTestProperties.SDK.PYTHON - - def datasetName = loadTestsBuilder.getBigQueryDataset('load_test', triggeringContext) - def numberOfWorkers = 16 - List<Map> testScenarios = scenarios(datasetName) - - def flink = new Flink(scope, 'beam_LoadTests_Python_GBK_Flink_Batch') - flink.setUp( - [ - "${DOCKER_CONTAINER_REGISTRY}/${DOCKER_BEAM_SDK_IMAGE}" - ], - numberOfWorkers, - "${DOCKER_BEAM_JOBSERVER}/beam_flink${CommonTestProperties.getFlinkVersion()}_job_server:latest") - - def configurations = testScenarios.findAll { it.pipelineOptions?.parallelism?.value == numberOfWorkers } - loadTestsBuilder.loadTests(scope, sdk, configurations, "GBK", "batch") - - numberOfWorkers = 5 - flink.scaleCluster(numberOfWorkers) - - configurations = testScenarios.findAll { it.pipelineOptions?.parallelism?.value == numberOfWorkers } - loadTestsBuilder.loadTests(scope, sdk, configurations, "GBK", "batch") -} - -PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_LoadTests_Python_GBK_Flink_Batch', - 'Run Load Tests Python GBK Flink Batch', - 'Load Tests Python GBK Flink Batch suite', - this - ) { - additionalPipelineArgs = [:] - loadTest(delegate, CommonTestProperties.TriggeringContext.PR) - } - -// TODO(https://github.com/apache/beam/issues/20146) Re-enable auto builds after these tests pass. diff --git a/.test-infra/jenkins/job_LoadTests_GBK_Go.groovy b/.test-infra/jenkins/job_LoadTests_GBK_Go.groovy deleted file mode 100644 index da31a692f61d..000000000000 --- a/.test-infra/jenkins/job_LoadTests_GBK_Go.groovy +++ /dev/null @@ -1,227 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonJobProperties as commonJobProperties -import CommonTestProperties -import LoadTestsBuilder as loadTestsBuilder -import PhraseTriggeringPostCommitBuilder -import InfluxDBCredentialsHelper - -import static LoadTestsBuilder.GO_SDK_CONTAINER - -String now = new Date().format('MMddHHmmss', TimeZone.getTimeZone('UTC')) - -def batchScenarios = { - [ - [ - title : 'Group By Key Go Load test: 2GB of 10B records', - test : 'group_by_key', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - job_name : "load-tests-go-dataflow-batch-gbk-1-${now}", - project : 'apache-beam-testing', - region : 'us-central1', - temp_location : 'gs://temp-storage-for-perf-tests/loadtests', - staging_location : 'gs://temp-storage-for-perf-tests/loadtests', - influx_namespace : 'dataflow', - influx_measurement : 'go_batch_gbk_1', - input_options : '\'{' + - '"num_records": 200000000,' + - '"key_size": 1,' + - '"value_size": 9}\'', - iterations : 1, - fanout : 1, - num_workers : 5, - autoscaling_algorithm: 'NONE', - environment_type : 'DOCKER', - environment_config : GO_SDK_CONTAINER, - ] - ], - [ - title : 'Group By Key Go Load test: 2GB of 100B records', - test : 'group_by_key', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - job_name : "load-tests-go-dataflow-batch-gbk-2-${now}", - project : 'apache-beam-testing', - region : 'us-central1', - temp_location : 'gs://temp-storage-for-perf-tests/loadtests', - staging_location : 'gs://temp-storage-for-perf-tests/loadtests', - influx_namespace : 'dataflow', - influx_measurement : 'go_batch_gbk_2', - input_options : '\'{' + - '"num_records": 20000000,' + - '"key_size": 10,' + - '"value_size": 90}\'', - iterations : 1, - fanout : 1, - num_workers : 5, - autoscaling_algorithm: 'NONE', - environment_type : 'DOCKER', - environment_config : GO_SDK_CONTAINER, - ] - ], - [ - title : 'Group By Key Go Load test: 2GB of 100kB records', - test : 'group_by_key', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - job_name : "load-tests-go-dataflow-batch-gbk-3-${now}", - project : 'apache-beam-testing', - region : 'us-central1', - temp_location : 'gs://temp-storage-for-perf-tests/loadtests', - staging_location : 'gs://temp-storage-for-perf-tests/loadtests', - influx_namespace : 'dataflow', - influx_measurement : 'go_batch_gbk_3', - input_options : '\'{' + - '"num_records": 20000,' + - '"key_size": 10000,' + - '"value_size": 90000}\'', - iterations : 1, - fanout : 1, - num_workers : 5, - autoscaling_algorithm: 'NONE', - environment_type : 'DOCKER', - environment_config : GO_SDK_CONTAINER, - ] - ], - [ - title : 'Group By Key Go Load test: fanout 4 times with 2GB 10-byte records total', - test : 'group_by_key', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - job_name : "load-tests-go-dataflow-batch-gbk-4-${now}", - project : 'apache-beam-testing', - region : 'us-central1', - temp_location : 'gs://temp-storage-for-perf-tests/loadtests', - staging_location : 'gs://temp-storage-for-perf-tests/loadtests', - influx_namespace : 'dataflow', - influx_measurement : 'go_batch_gbk_4', - input_options : '\'{' + - '"num_records": 5000000,' + - '"key_size": 10,' + - '"value_size": 90}\'', - iterations : 1, - fanout : 4, - num_workers : 16, - autoscaling_algorithm: 'NONE', - environment_type : 'DOCKER', - environment_config : GO_SDK_CONTAINER, - ] - ], - [ - title : 'Group By Key Go Load test: fanout 8 times with 2GB 10-byte records total', - test : 'group_by_key', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - job_name : "load-tests-go-dataflow-batch-gbk-5-${now}", - project : 'apache-beam-testing', - region : 'us-central1', - temp_location : 'gs://temp-storage-for-perf-tests/loadtests', - staging_location : 'gs://temp-storage-for-perf-tests/loadtests', - influx_namespace : 'dataflow', - influx_measurement : 'go_batch_gbk_5', - input_options : '\'{' + - '"num_records": 2500000,' + - '"key_size": 10,' + - '"value_size": 90}\'', - iterations : 1, - fanout : 8, - num_workers : 16, - autoscaling_algorithm: 'NONE', - environment_type : 'DOCKER', - environment_config : GO_SDK_CONTAINER, - ] - ], - [ - title : 'Group By Key Go Load test: reiterate 4 times 10kB values', - test : 'group_by_key', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - job_name : "load-tests-go-dataflow-batch-gbk-6-${now}", - project : 'apache-beam-testing', - region : 'us-central1', - temp_location : 'gs://temp-storage-for-perf-tests/loadtests', - staging_location : 'gs://temp-storage-for-perf-tests/loadtests', - influx_namespace : 'dataflow', - influx_measurement : 'go_batch_gbk_6', - input_options : '\'{' + - '"num_records": 20000000,' + - '"key_size": 10,' + - '"value_size": 90,' + - '"num_hot_keys": 200,' + - '"hot_key_fraction": 1}\'', - iterations : 4, - fanout : 1, - num_workers : 5, - autoscaling_algorithm: 'NONE', - environment_type : 'DOCKER', - environment_config : GO_SDK_CONTAINER, - ] - ], - [ - title : 'Group By Key Go Load test: reiterate 4 times 2MB values', - test : 'group_by_key', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - job_name : "load-tests-go-dataflow-batch-gbk-7-${now}", - project : 'apache-beam-testing', - region : 'us-central1', - temp_location : 'gs://temp-storage-for-perf-tests/loadtests', - staging_location : 'gs://temp-storage-for-perf-tests/loadtests', - influx_namespace : 'dataflow', - influx_measurement : 'go_batch_gbk_7', - input_options : '\'{' + - '"num_records": 20000000,' + - '"key_size": 10,' + - '"value_size": 90,' + - '"num_hot_keys": 10,' + - '"hot_key_fraction": 1}\'', - iterations : 4, - fanout : 1, - num_workers : 5, - autoscaling_algorithm: 'NONE', - environment_type : 'DOCKER', - environment_config : GO_SDK_CONTAINER, - ] - ], - ] - .each { test -> test.pipelineOptions.putAll(additionalPipelineArgs) } -} - -def loadTestJob = { scope, triggeringContext, mode -> - loadTestsBuilder.loadTests(scope, CommonTestProperties.SDK.GO, batchScenarios(), 'group_by_key', mode) -} - -PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_LoadTests_Go_GBK_Dataflow_Batch', - 'Run Load Tests Go GBK Dataflow Batch', - 'Load Tests Go GBK Dataflow Batch suite', - this - ) { - additionalPipelineArgs = [:] - loadTestJob(delegate, CommonTestProperties.TriggeringContext.PR, 'batch') - } - -CronJobBuilder.cronJob('beam_LoadTests_Go_GBK_Dataflow_Batch', 'H H * * *', this) { - additionalPipelineArgs = [ - influx_db_name: InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influx_hostname: InfluxDBCredentialsHelper.InfluxDBHostUrl, - ] - loadTestJob(delegate, CommonTestProperties.TriggeringContext.POST_COMMIT, 'batch') -} diff --git a/.test-infra/jenkins/job_LoadTests_GBK_Java.groovy b/.test-infra/jenkins/job_LoadTests_GBK_Java.groovy deleted file mode 100644 index 6427b5b12357..000000000000 --- a/.test-infra/jenkins/job_LoadTests_GBK_Java.groovy +++ /dev/null @@ -1,261 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonJobProperties as commonJobProperties -import CommonTestProperties -import LoadTestsBuilder as loadTestsBuilder -import PhraseTriggeringPostCommitBuilder -import CronJobBuilder -import InfluxDBCredentialsHelper - -def loadTestConfigurations = { mode, isStreaming -> - [ - [ - title : 'Load test: 2GB of 10B records', - test : 'org.apache.beam.sdk.loadtests.GroupByKeyLoadTest', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - project : 'apache-beam-testing', - region : 'us-central1', - appName : "load_tests_Java_Dataflow_${mode}_GBK_1", - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${mode}_gbk_1", - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 200000000, - "keySizeBytes": 1, - "valueSizeBytes": 9 - } - """.trim().replaceAll("\\s", ""), - fanout : 1, - iterations : 1, - numWorkers : 5, - autoscalingAlgorithm : "NONE", - streaming : isStreaming - ] - ], - [ - title : 'Load test: 2GB of 100B records', - test : 'org.apache.beam.sdk.loadtests.GroupByKeyLoadTest', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - project : 'apache-beam-testing', - region : 'us-central1', - appName : "load_tests_Java_Dataflow_${mode}_GBK_2", - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${mode}_gbk_2", - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 20000000, - "keySizeBytes": 10, - "valueSizeBytes": 90 - } - """.trim().replaceAll("\\s", ""), - fanout : 1, - iterations : 1, - numWorkers : 5, - autoscalingAlgorithm : "NONE", - streaming : isStreaming - ] - ], - [ - - title : 'Load test: 2GB of 100kB records', - test : 'org.apache.beam.sdk.loadtests.GroupByKeyLoadTest', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - project : 'apache-beam-testing', - region : 'us-central1', - appName : "load_tests_Java_Dataflow_${mode}_GBK_3", - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${mode}_gbk_3", - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 20000, - "keySizeBytes": 10000, - "valueSizeBytes": 90000 - } - """.trim().replaceAll("\\s", ""), - fanout : 1, - iterations : 1, - numWorkers : 5, - autoscalingAlgorithm : "NONE", - streaming : isStreaming - ] - - ], - [ - title : 'Load test: fanout 4 times with 2GB 10-byte records total', - test : 'org.apache.beam.sdk.loadtests.GroupByKeyLoadTest', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - project : 'apache-beam-testing', - region : 'us-central1', - appName : 'load_tests_Java_Dataflow_${mode}_GBK_4', - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${mode}_gbk_4", - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 5000000, - "keySizeBytes": 10, - "valueSizeBytes": 90 - } - """.trim().replaceAll("\\s", ""), - fanout : 4, - iterations : 1, - numWorkers : 16, - autoscalingAlgorithm : "NONE", - streaming : isStreaming - ] - ], - [ - title : 'Load test: fanout 8 times with 2GB 10-byte records total', - test : 'org.apache.beam.sdk.loadtests.GroupByKeyLoadTest', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - project : 'apache-beam-testing', - region : 'us-central1', - appName : "load_tests_Java_Dataflow_${mode}_GBK_5", - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${mode}_gbk_5", - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 2500000, - "keySizeBytes": 10, - "valueSizeBytes": 90 - } - """.trim().replaceAll("\\s", ""), - fanout : 8, - iterations : 1, - numWorkers : 16, - autoscalingAlgorithm : "NONE", - streaming : isStreaming - ] - ], - [ - title : 'Load test: reiterate 4 times 10kB values', - test : 'org.apache.beam.sdk.loadtests.GroupByKeyLoadTest', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - project : 'apache-beam-testing', - region : 'us-central1', - appName : "load_tests_Java_Dataflow_${mode}_GBK_6", - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${mode}_gbk_6", - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 20000000, - "keySizeBytes": 10, - "valueSizeBytes": 90, - "numHotKeys": 200, - "hotKeyFraction": 1 - } - """.trim().replaceAll("\\s", ""), - fanout : 1, - iterations : 4, - numWorkers : 5, - autoscalingAlgorithm : "NONE", - streaming : isStreaming - ] - ], - [ - title : 'Load test: reiterate 4 times 2MB values', - test : 'org.apache.beam.sdk.loadtests.GroupByKeyLoadTest', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - project : 'apache-beam-testing', - region : 'us-central1', - appName : "load_tests_Java_Dataflow_${mode}_GBK_7", - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${mode}_gbk_7", - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 20000000, - "keySizeBytes": 10, - "valueSizeBytes": 90, - "numHotKeys": 10, - "hotKeyFraction": 1 - } - """.trim().replaceAll("\\s", ""), - fanout : 1, - iterations : 4, - numWorkers : 5, - autoscalingAlgorithm : "NONE", - streaming : isStreaming - ] - ] - ].each { test -> test.pipelineOptions.putAll(additionalPipelineArgs) } -} - -def streamingLoadTestJob = { scope, triggeringContext -> - scope.description('Runs Java GBK load tests on Dataflow runner in streaming mode') - commonJobProperties.setTopLevelMainJobProperties(scope, 'master', 240) - - for (testConfiguration in loadTestConfigurations('streaming', true)) { - testConfiguration.pipelineOptions << [inputWindowDurationSec: 1200] - loadTestsBuilder.loadTest(scope, testConfiguration.title, testConfiguration.runner, CommonTestProperties.SDK.JAVA, testConfiguration.pipelineOptions, testConfiguration.test) - } -} - -CronJobBuilder.cronJob('beam_LoadTests_Java_GBK_Dataflow_Streaming', 'H H * * *', this) { - additionalPipelineArgs = [ - influxDatabase: InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influxHost: InfluxDBCredentialsHelper.InfluxDBHostUrl, - ] - streamingLoadTestJob(delegate, CommonTestProperties.TriggeringContext.POST_COMMIT) -} - -PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_LoadTests_Java_GBK_Dataflow_Streaming', - 'Run Load Tests Java GBK Dataflow Streaming', - 'Load Tests Java GBK Dataflow Streaming suite', - this - ) { - additionalPipelineArgs = [:] - streamingLoadTestJob(delegate, CommonTestProperties.TriggeringContext.PR) - } - - -def batchLoadTestJob = { scope, triggeringContext -> - loadTestsBuilder.loadTests(scope, CommonTestProperties.SDK.JAVA, loadTestConfigurations('batch', false), "GBK", "batch") -} - -CronJobBuilder.cronJob('beam_LoadTests_Java_GBK_Dataflow_Batch', 'H H * * *', this) { - additionalPipelineArgs = [ - influxDatabase: InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influxHost: InfluxDBCredentialsHelper.InfluxDBHostUrl, - ] - batchLoadTestJob(delegate, CommonTestProperties.TriggeringContext.POST_COMMIT) -} - -PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_LoadTests_Java_GBK_Dataflow_Batch', - 'Run Load Tests Java GBK Dataflow Batch', - 'Load Tests Java GBK Dataflow Batch suite', - this - ) { - additionalPipelineArgs = [:] - batchLoadTestJob(delegate, CommonTestProperties.TriggeringContext.PR) - } diff --git a/.test-infra/jenkins/job_LoadTests_GBK_Java_Smoke.groovy b/.test-infra/jenkins/job_LoadTests_GBK_Java_Smoke.groovy deleted file mode 100644 index 131c26efe2e5..000000000000 --- a/.test-infra/jenkins/job_LoadTests_GBK_Java_Smoke.groovy +++ /dev/null @@ -1,85 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonTestProperties -import LoadTestsBuilder as loadTestsBuilder -import PhraseTriggeringPostCommitBuilder - -def smokeTestConfigurations = { - [ - [ - title : 'GroupByKey load test Direct', - test : 'org.apache.beam.sdk.loadtests.GroupByKeyLoadTest', - runner : CommonTestProperties.Runner.DIRECT, - pipelineOptions: [ - sourceOptions : '{"numRecords":100000,"splitPointFrequencyRecords":1}', - stepOptions : '{"outputRecordsPerInputRecord":1,"preservesInputKeyDistribution":true}', - fanout : 10, - iterations : 1, - ] - ], - [ - title : 'GroupByKey load test Dataflow', - test : 'org.apache.beam.sdk.loadtests.GroupByKeyLoadTest', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - project : 'apache-beam-testing', - region : 'us-central1', - tempLocation : 'gs://temp-storage-for-perf-tests/smoketests', - sourceOptions : '{"numRecords":100000,"splitPointFrequencyRecords":1}', - stepOptions : '{"outputRecordsPerInputRecord":1,"preservesInputKeyDistribution":true}', - fanout : 10, - iterations : 1, - ] - ], - [ - title : 'GroupByKey load test Flink', - test : 'org.apache.beam.sdk.loadtests.GroupByKeyLoadTest', - runner : CommonTestProperties.Runner.FLINK, - pipelineOptions: [ - sourceOptions : '{"numRecords":100000,"splitPointFrequencyRecords":1}', - stepOptions : '{"outputRecordsPerInputRecord":1,"preservesInputKeyDistribution":true}', - fanout : 10, - iterations : 1, - ] - ], - [ - title : 'GroupByKey load test Spark', - test : 'org.apache.beam.sdk.loadtests.GroupByKeyLoadTest', - runner : CommonTestProperties.Runner.SPARK, - pipelineOptions: [ - sparkMaster : 'local[4]', - sourceOptions : '{"numRecords":100000,"splitPointFrequencyRecords":1}', - stepOptions : '{"outputRecordsPerInputRecord":1,"preservesInputKeyDistribution":true}', - fanout : 10, - iterations : 1, - ] - ] - ] -} - - -// Runs a tiny version load test suite to ensure nothing is broken. -PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_Java_LoadTests_GBK_Smoke', - 'Run Java Load Tests GBK Smoke', - 'Java Load Tests GBK Smoke', - this - ) { - loadTestsBuilder.loadTests(delegate, CommonTestProperties.SDK.JAVA, smokeTestConfigurations(), "GBK", "smoke") - } diff --git a/.test-infra/jenkins/job_LoadTests_GBK_Java_spark_structured_streaming.groovy b/.test-infra/jenkins/job_LoadTests_GBK_Java_spark_structured_streaming.groovy deleted file mode 100644 index 8e5ec703f3b0..000000000000 --- a/.test-infra/jenkins/job_LoadTests_GBK_Java_spark_structured_streaming.groovy +++ /dev/null @@ -1,210 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonTestProperties -import CronJobBuilder -import LoadTestsBuilder as loadTestsBuilder -import PhraseTriggeringPostCommitBuilder -import InfluxDBCredentialsHelper - -def loadTestConfigurations = { mode, isStreaming -> - [ - [ - title : 'Load test: 2GB of 10B records', - test : 'org.apache.beam.sdk.loadtests.GroupByKeyLoadTest', - runner : CommonTestProperties.Runner.SPARK_STRUCTURED_STREAMING, - pipelineOptions: [ - project : 'apache-beam-testing', - appName : "load_tests_Java_SparkStructuredStreaming_${mode}_GBK_1", - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${mode}_gbk_1", - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 200000000, - "keySizeBytes": 1, - "valueSizeBytes": 9 - } - """.trim().replaceAll("\\s", ""), - fanout : 1, - iterations : 1, - streaming : isStreaming - ] - ], - [ - title : 'Load test: 2GB of 100B records', - test : 'org.apache.beam.sdk.loadtests.GroupByKeyLoadTest', - runner : CommonTestProperties.Runner.SPARK_STRUCTURED_STREAMING, - pipelineOptions: [ - project : 'apache-beam-testing', - appName : "load_tests_Java_SparkStructuredStreaming_${mode}_GBK_2", - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${mode}_gbk_2", - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 20000000, - "keySizeBytes": 10, - "valueSizeBytes": 90 - } - """.trim().replaceAll("\\s", ""), - fanout : 1, - iterations : 1, - streaming : isStreaming - ] - ], - [ - - title : 'Load test: 2GB of 100kB records', - test : 'org.apache.beam.sdk.loadtests.GroupByKeyLoadTest', - runner : CommonTestProperties.Runner.SPARK_STRUCTURED_STREAMING, - pipelineOptions: [ - project : 'apache-beam-testing', - appName : "load_tests_Java_SparkStructuredStreaming_${mode}_GBK_3", - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${mode}_gbk_3", - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 20000, - "keySizeBytes": 10000, - "valueSizeBytes": 90000 - } - """.trim().replaceAll("\\s", ""), - fanout : 1, - iterations : 1, - streaming : isStreaming - ] - - ], - [ - title : 'Load test: fanout 4 times with 2GB 10-byte records total', - test : 'org.apache.beam.sdk.loadtests.GroupByKeyLoadTest', - runner : CommonTestProperties.Runner.SPARK_STRUCTURED_STREAMING, - pipelineOptions: [ - project : 'apache-beam-testing', - appName : 'load_tests_Java_SparkStructuredStreaming_${mode}_GBK_4', - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${mode}_gbk_4", - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 5000000, - "keySizeBytes": 10, - "valueSizeBytes": 90 - } - """.trim().replaceAll("\\s", ""), - fanout : 4, - iterations : 1, - streaming : isStreaming - ] - ], - [ - title : 'Load test: fanout 8 times with 2GB 10-byte records total', - test : 'org.apache.beam.sdk.loadtests.GroupByKeyLoadTest', - runner : CommonTestProperties.Runner.SPARK_STRUCTURED_STREAMING, - pipelineOptions: [ - project : 'apache-beam-testing', - appName : "load_tests_Java_SparkStructuredStreaming_${mode}_GBK_5", - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${mode}_gbk_5", - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 2500000, - "keySizeBytes": 10, - "valueSizeBytes": 90 - } - """.trim().replaceAll("\\s", ""), - fanout : 8, - iterations : 1, - streaming : isStreaming - ] - ], - [ - title : 'Load test: reiterate 4 times 10kB values', - test : 'org.apache.beam.sdk.loadtests.GroupByKeyLoadTest', - runner : CommonTestProperties.Runner.SPARK_STRUCTURED_STREAMING, - pipelineOptions: [ - project : 'apache-beam-testing', - appName : "load_tests_Java_SparkStructuredStreaming_${mode}_GBK_6", - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${mode}_gbk_6", - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 20000000, - "keySizeBytes": 10, - "valueSizeBytes": 90, - "numHotKeys": 200, - "hotKeyFraction": 1 - } - """.trim().replaceAll("\\s", ""), - fanout : 1, - iterations : 4, - streaming : isStreaming - ] - ], - [ - title : 'Load test: reiterate 4 times 2MB values', - test : 'org.apache.beam.sdk.loadtests.GroupByKeyLoadTest', - runner : CommonTestProperties.Runner.SPARK_STRUCTURED_STREAMING, - pipelineOptions: [ - project : 'apache-beam-testing', - appName : "load_tests_Java_SparkStructuredStreaming_${mode}_GBK_7", - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${mode}_gbk_7", - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 20000000, - "keySizeBytes": 10, - "valueSizeBytes": 90, - "numHotKeys": 10, - "hotKeyFraction": 1 - } - """.trim().replaceAll("\\s", ""), - fanout : 1, - iterations : 4, - streaming : isStreaming - ] - ] - ].each { test -> test.pipelineOptions.putAll(additionalPipelineArgs) } -} - -def batchLoadTestJob = { scope, triggeringContext -> - loadTestsBuilder.loadTests(scope, CommonTestProperties.SDK.JAVA, loadTestConfigurations('batch', false), "GBK", "batch") -} - -CronJobBuilder.cronJob('beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch', 'H H * * *', this) { - additionalPipelineArgs = [ - influxDatabase: InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influxHost: InfluxDBCredentialsHelper.InfluxDBHostUrl, - ] - batchLoadTestJob(delegate, CommonTestProperties.TriggeringContext.POST_COMMIT) -} - -PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch', - 'Run Load Tests Java GBK SparkStructuredStreaming Batch', - 'Load Tests Java GBK SparkStructuredStreaming Batch suite', - this - ) { - additionalPipelineArgs = [:] - batchLoadTestJob(delegate, CommonTestProperties.TriggeringContext.PR) - } diff --git a/.test-infra/jenkins/job_LoadTests_GBK_Python.groovy b/.test-infra/jenkins/job_LoadTests_GBK_Python.groovy deleted file mode 100644 index 8538e2bcc422..000000000000 --- a/.test-infra/jenkins/job_LoadTests_GBK_Python.groovy +++ /dev/null @@ -1,209 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import LoadTestsBuilder as loadTestsBuilder -import PhraseTriggeringPostCommitBuilder -import InfluxDBCredentialsHelper - -def now = new Date().format("MMddHHmmss", TimeZone.getTimeZone('UTC')) - -// TODO(https://github.com/apache/beam/issues/20403): Skipping some cases because they are too slow. -def TESTS_TO_SKIP = [ - 'load-tests-python-dataflow-streaming-gbk-1', - 'load-tests-python-dataflow-streaming-gbk-2', - 'load-tests-python-dataflow-streaming-gbk-4', - 'load-tests-python-dataflow-streaming-gbk-5', -] - -def loadTestConfigurations = { mode, datasetName -> - [ - [ - title : 'GroupByKey Python Load test: 2GB of 10B records', - test : 'apache_beam.testing.load_tests.group_by_key_test', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - job_name : "load-tests-python-dataflow-${mode}-gbk-1-${now}", - project : 'apache-beam-testing', - region : 'us-central1', - temp_location : 'gs://temp-storage-for-perf-tests/loadtests', - publish_to_big_query : true, - metrics_dataset : datasetName, - metrics_table : "python_dataflow_${mode}_gbk_1", - influx_measurement : "python_${mode}_gbk_1", - input_options : '\'{"num_records": 200000000,' + - '"key_size": 1,' + - '"value_size": 9,' + - '"algorithm": "lcg"}\'', - iterations : 1, - fanout : 1, - num_workers : 5, - autoscaling_algorithm: 'NONE', - ] - ], - [ - title : 'GroupByKey Python Load test: 2GB of 100B records', - test : 'apache_beam.testing.load_tests.group_by_key_test', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - job_name : "load-tests-python-dataflow-${mode}-gbk-2-${now}", - project : 'apache-beam-testing', - region : 'us-central1', - temp_location : 'gs://temp-storage-for-perf-tests/loadtests', - publish_to_big_query : true, - metrics_dataset : datasetName, - metrics_table : "python_dataflow_${mode}_gbk_2", - influx_measurement : "python_${mode}_gbk_2", - input_options : '\'{"num_records": 20000000,' + - '"key_size": 10,' + - '"value_size": 90,' + - '"algorithm": "lcg"}\'', - iterations : 1, - fanout : 1, - num_workers : 5, - autoscaling_algorithm: 'NONE', - ] - ], - [ - title : 'GroupByKey Python Load test: 2GB of 100kB records', - test : 'apache_beam.testing.load_tests.group_by_key_test', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - job_name : "load-tests-python-dataflow-${mode}-gbk-3-${now}", - project : 'apache-beam-testing', - region : 'us-central1', - temp_location : 'gs://temp-storage-for-perf-tests/loadtests', - publish_to_big_query : true, - metrics_dataset : datasetName, - metrics_table : "python_dataflow_${mode}_gbk_3", - influx_measurement : "python_${mode}_gbk_3", - input_options : '\'{"num_records": 20000,' + - '"key_size": 10000,' + - '"value_size": 90000,' + - '"algorithm": "lcg"}\'', - iterations : 1, - fanout : 1, - num_workers : 5, - autoscaling_algorithm: 'NONE', - ] - ], - [ - title : 'GroupByKey Python Load test: fanout 4 times with 2GB 10-byte records total', - test : 'apache_beam.testing.load_tests.group_by_key_test', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - job_name : "load-tests-python-dataflow-${mode}-gbk-4-${now}", - project : 'apache-beam-testing', - region : 'us-central1', - temp_location : 'gs://temp-storage-for-perf-tests/loadtests', - publish_to_big_query : true, - metrics_dataset : datasetName, - metrics_table : "python_dataflow_${mode}_gbk_4", - influx_measurement : "python_${mode}_gbk_4", - input_options : '\'{"num_records": 5000000,' + - '"key_size": 10,' + - '"value_size": 90,' + - '"algorithm": "lcg"}\'', - iterations : 1, - fanout : 4, - num_workers : 16, - autoscaling_algorithm: 'NONE', - ] - ], - [ - title : 'GroupByKey Python Load test: fanout 8 times with 2GB 10-byte records total', - test : 'apache_beam.testing.load_tests.group_by_key_test', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - job_name : "load-tests-python-dataflow-${mode}-gbk-5-${now}", - project : 'apache-beam-testing', - region : 'us-central1', - temp_location : 'gs://temp-storage-for-perf-tests/loadtests', - publish_to_big_query : true, - metrics_dataset : datasetName, - metrics_table : "python_dataflow_${mode}_gbk_5", - influx_measurement : "python_${mode}_gbk_5", - input_options : '\'{"num_records": 2500000,' + - '"key_size": 10,' + - '"value_size": 90,' + - '"algorithm": "lcg"}\'', - iterations : 1, - fanout : 8, - num_workers : 16, - autoscaling_algorithm: 'NONE', - ] - ], - ] - .each { test -> test.pipelineOptions.putAll(additionalPipelineArgs) } - .each { test -> (mode != 'streaming') ?: addStreamingOptions(test) } - .collectMany { test -> - TESTS_TO_SKIP.any { element -> test.pipelineOptions.job_name.startsWith(element) } ? []: [test] - } -} - -def addStreamingOptions(test) { - test.pipelineOptions << [ - streaming: null, - // Use the new Dataflow runner, which offers improved efficiency of Dataflow jobs. - // See https://cloud.google.com/dataflow/docs/guides/deploying-a-pipeline#dataflow-runner-v2 - // for more details. - experiments: 'use_runner_v2', - ] -} - -def loadTestJob = { scope, triggeringContext, mode -> - def datasetName = loadTestsBuilder.getBigQueryDataset('load_test', triggeringContext) - loadTestsBuilder.loadTests(scope, CommonTestProperties.SDK.PYTHON, - loadTestConfigurations(mode, datasetName), 'GBK', mode) -} - -PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_LoadTests_Python_GBK_Dataflow_Batch', - 'Run Load Tests Python GBK Dataflow Batch', - 'Load Tests Python GBK Dataflow Batch suite', - this - ) { - additionalPipelineArgs = [:] - loadTestJob(delegate, CommonTestProperties.TriggeringContext.PR, 'batch') - } - -CronJobBuilder.cronJob('beam_LoadTests_Python_GBK_Dataflow_Batch', 'H H * * *', this) { - additionalPipelineArgs = [ - influx_db_name: InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influx_hostname: InfluxDBCredentialsHelper.InfluxDBHostUrl, - ] - loadTestJob(delegate, CommonTestProperties.TriggeringContext.POST_COMMIT, 'batch') -} - -PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_LoadTests_Python_GBK_Dataflow_Streaming', - 'Run Load Tests Python GBK Dataflow Streaming', - 'Load Tests Python GBK Dataflow Streaming suite', - this - ) { - additionalPipelineArgs = [:] - loadTestJob(delegate, CommonTestProperties.TriggeringContext.PR, 'streaming') - } - -CronJobBuilder.cronJob('beam_LoadTests_Python_GBK_Dataflow_Streaming', 'H H * * *', this) { - additionalPipelineArgs = [ - influx_db_name: InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influx_hostname: InfluxDBCredentialsHelper.InfluxDBHostUrl, - ] - loadTestJob(delegate, CommonTestProperties.TriggeringContext.POST_COMMIT, 'streaming') -} - diff --git a/.test-infra/jenkins/job_LoadTests_GBK_Python_reiterate.groovy b/.test-infra/jenkins/job_LoadTests_GBK_Python_reiterate.groovy deleted file mode 100644 index b958dd8a907a..000000000000 --- a/.test-infra/jenkins/job_LoadTests_GBK_Python_reiterate.groovy +++ /dev/null @@ -1,137 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonJobProperties as commonJobProperties -import CommonTestProperties -import LoadTestsBuilder as loadTestsBuilder -import PhraseTriggeringPostCommitBuilder -import CronJobBuilder -import InfluxDBCredentialsHelper - -def now = new Date().format("MMddHHmmss", TimeZone.getTimeZone('UTC')) - -def loadTestConfigurations = { mode, datasetName -> - [ - [ - title : 'GroupByKey Python Load test: reiterate 4 times 10kB values', - test : 'apache_beam.testing.load_tests.group_by_key_test', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - project : 'apache-beam-testing', - region : 'us-central1', - job_name : "load-tests-python-dataflow-${mode}-gbk-6-${now}", - temp_location : 'gs://temp-storage-for-perf-tests/loadtests', - publish_to_big_query : true, - metrics_dataset : datasetName, - metrics_table : "python_dataflow_${mode}_gbk_6", - influx_measurement : "python_${mode}_gbk_6", - input_options : '\'{"num_records": 20000000,' + - '"key_size": 10,' + - '"value_size": 90,' + - '"num_hot_keys": 200,' + - '"hot_key_fraction": 1,' + - '"algorithm": "lcg"}\'', - iterations : 4, - fanout : 1, - num_workers : 5, - autoscaling_algorithm: 'NONE', - ] - ], - [ - title : 'GroupByKey Python Load test: reiterate 4 times 2MB values', - test : 'apache_beam.testing.load_tests.group_by_key_test', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - project : 'apache-beam-testing', - region : 'us-central1', - job_name : "load-tests-python-dataflow-${mode}-gbk-7-${now}", - temp_location : 'gs://temp-storage-for-perf-tests/loadtests', - publish_to_big_query : true, - metrics_dataset : datasetName, - metrics_table : "python_dataflow_${mode}_gbk_7", - influx_measurement : "python_${mode}_gbk_7", - input_options : '\'{"num_records": 20000000,' + - '"key_size": 10,' + - '"value_size": 90,' + - '"num_hot_keys": 10,' + - '"hot_key_fraction": 1,' + - '"algorithm": "lcg"}\'', - iterations : 4, - fanout : 1, - num_workers : 5, - autoscaling_algorithm: 'NONE', - ] - ] - ] - .each { test -> test.pipelineOptions.putAll(additionalPipelineArgs) } - .each { test -> (mode != 'streaming') ?: addStreamingOptions(test) } -} - -def addStreamingOptions(test) { - test.pipelineOptions << [ - streaming: null, - // Use the new Dataflow runner, which offers improved efficiency of Dataflow jobs. - // See https://cloud.google.com/dataflow/docs/guides/deploying-a-pipeline#dataflow-runner-v2 - // for more details. - experiments: 'use_runner_v2', - ] -} - -def loadTestJob = { scope, triggeringContext, mode -> - def datasetName = loadTestsBuilder.getBigQueryDataset('load_test', triggeringContext) - loadTestsBuilder.loadTests(scope, CommonTestProperties.SDK.PYTHON, - loadTestConfigurations(mode, datasetName), 'GBK reiterate', mode) -} - -CronJobBuilder.cronJob('beam_LoadTests_Python_GBK_reiterate_Dataflow_Batch', - 'H H * * *', this) { - additionalPipelineArgs = [ - influx_db_name: InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influx_hostname: InfluxDBCredentialsHelper.InfluxDBHostUrl, - ] - loadTestJob(delegate, CommonTestProperties.TriggeringContext.POST_COMMIT, 'batch') - } - -PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_LoadTests_Python_GBK_reiterate_Dataflow_Batch', - 'Run Load Tests Python GBK reiterate Dataflow Batch', - 'Load Tests Python GBK reiterate Dataflow Batch suite', - this - ) { - additionalPipelineArgs = [:] - loadTestJob(delegate, CommonTestProperties.TriggeringContext.PR, 'batch') - } - -CronJobBuilder.cronJob('beam_LoadTests_Python_GBK_reiterate_Dataflow_Streaming', - 'H H * * *', this) { - additionalPipelineArgs = [ - influx_db_name: InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influx_hostname: InfluxDBCredentialsHelper.InfluxDBHostUrl, - ] - loadTestJob(delegate, CommonTestProperties.TriggeringContext.POST_COMMIT, 'streaming') - } - -PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_LoadTests_Python_GBK_reiterate_Dataflow_Streaming', - 'Run Load Tests Python GBK reiterate Dataflow Streaming', - 'Load Tests Python GBK reiterate Dataflow Streaming suite', - this - ) { - additionalPipelineArgs = [:] - loadTestJob(delegate, CommonTestProperties.TriggeringContext.PR, 'streaming') - } diff --git a/.test-infra/jenkins/job_LoadTests_ParDo_Dataflow_V2_Java11.groovy b/.test-infra/jenkins/job_LoadTests_ParDo_Dataflow_V2_Java11.groovy deleted file mode 100644 index b7154e840e6a..000000000000 --- a/.test-infra/jenkins/job_LoadTests_ParDo_Dataflow_V2_Java11.groovy +++ /dev/null @@ -1,217 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonJobProperties as commonJobProperties -import CommonTestProperties -import LoadTestsBuilder as loadTestsBuilder -import PhraseTriggeringPostCommitBuilder -import CronJobBuilder -import InfluxDBCredentialsHelper - -def commonLoadTestConfig = { jobType, isStreaming -> - [ - [ - title : 'Load test: ParDo 2GB 100 byte records 10 times', - test : 'org.apache.beam.sdk.loadtests.ParDoLoadTest', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - project : 'apache-beam-testing', - region : 'us-central1', - appName : "load_tests_Java11_Dataflow_V2_${jobType}_ParDo_1", - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${jobType}_pardo_1", - influxTags : """ - { - "runnerVersion": "v2", - "jdk": "java11" - } - """.trim().replaceAll("\\s", ""), - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 20000000, - "keySizeBytes": 10, - "valueSizeBytes": 90 - } - """.trim().replaceAll("\\s", ""), - iterations : 10, - numberOfCounters : 1, - numberOfCounterOperations: 0, - numWorkers : 5, - autoscalingAlgorithm: "NONE", - streaming : isStreaming - ] - ], - [ - title : 'Load test: ParDo 2GB 100 byte records 200 times', - test : 'org.apache.beam.sdk.loadtests.ParDoLoadTest', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - project : 'apache-beam-testing', - region : 'us-central1', - appName : "load_tests_Java11_Dataflow_V2_${jobType}_ParDo_2", - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${jobType}_pardo_2", - influxTags : """ - { - "runnerVersion": "v2", - "jdk": "java11" - } - """.trim().replaceAll("\\s", ""), - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 20000000, - "keySizeBytes": 10, - "valueSizeBytes": 90 - } - """.trim().replaceAll("\\s", ""), - iterations : 200, - numberOfCounters : 1, - numberOfCounterOperations: 0, - numWorkers : 5, - autoscalingAlgorithm: "NONE", - streaming : isStreaming - ] - ], - [ - - title : 'Load test: ParDo 2GB 100 byte records 10 counters', - test : 'org.apache.beam.sdk.loadtests.ParDoLoadTest', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - project : 'apache-beam-testing', - region : 'us-central1', - appName : "load_tests_Java11_Dataflow_V2_${jobType}_ParDo_3", - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${jobType}_pardo_3", - influxTags : """ - { - "runnerVersion": "v2", - "jdk": "java11" - } - """.trim().replaceAll("\\s", ""), - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 20000000, - "keySizeBytes": 10, - "valueSizeBytes": 90 - } - """.trim().replaceAll("\\s", ""), - iterations : 1, - numberOfCounters : 1, - numberOfCounterOperations: 10, - numWorkers : 5, - autoscalingAlgorithm: "NONE", - streaming : isStreaming - ] - - ], - [ - title : 'Load test: ParDo 2GB 100 byte records 100 counters', - test : 'org.apache.beam.sdk.loadtests.ParDoLoadTest', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - project : 'apache-beam-testing', - region : 'us-central1', - appName : "load_tests_Java11_Dataflow_V2_${jobType}_ParDo_4", - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${jobType}_pardo_4", - influxTags : """ - { - "runnerVersion": "v2", - "jdk": "java11" - } - """.trim().replaceAll("\\s", ""), - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 20000000, - "keySizeBytes": 10, - "valueSizeBytes": 90 - } - """.trim().replaceAll("\\s", ""), - iterations : 1, - numberOfCounters : 1, - numberOfCounterOperations: 100, - numWorkers : 5, - autoscalingAlgorithm: "NONE", - streaming : isStreaming - ] - ] - ].each { test -> test.pipelineOptions.putAll(additionalPipelineArgs) } -} - -def final JOB_SPECIFIC_SWITCHES = [ - '-Prunner.version="V2"', - '-PtestJavaVersion=11', - "-Pjava11Home=${commonJobProperties.JAVA_11_HOME}" -] - -def batchLoadTestJob = { scope, triggeringContext -> - loadTestsBuilder.loadTests(scope, CommonTestProperties.SDK.JAVA, commonLoadTestConfig('batch', false), - "ParDo", "batch", JOB_SPECIFIC_SWITCHES) -} - -def streamingLoadTestJob = {scope, triggeringContext -> - scope.description('Runs Java 11 ParDo load tests on Dataflow runner V2 in streaming mode') - commonJobProperties.setTopLevelMainJobProperties(scope, 'master', 240) - - for (testConfiguration in commonLoadTestConfig('streaming', true)) { - testConfiguration.pipelineOptions << [inputWindowDurationSec: 1200] - loadTestsBuilder.loadTest(scope, testConfiguration.title, testConfiguration.runner, CommonTestProperties.SDK.JAVA, - testConfiguration.pipelineOptions, testConfiguration.test, JOB_SPECIFIC_SWITCHES) - } -} - -CronJobBuilder.cronJob('beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_Java11', 'H H * * *', this) { - additionalPipelineArgs = [ - influxDatabase: InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influxHost: InfluxDBCredentialsHelper.InfluxDBHostUrl, - ] - batchLoadTestJob(delegate, CommonTestProperties.TriggeringContext.POST_COMMIT) -} - -CronJobBuilder.cronJob('beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_Java11', 'H H * * *', this) { - additionalPipelineArgs = [ - influxDatabase: InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influxHost: InfluxDBCredentialsHelper.InfluxDBHostUrl, - ] - streamingLoadTestJob(delegate, CommonTestProperties.TriggeringContext.POST_COMMIT) -} - -PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_Java11', - 'Run Load Tests Java 11 ParDo Dataflow V2 Batch', - 'Load Tests Java 11 ParDo Dataflow V2 Batch suite', - this - ) { - additionalPipelineArgs = [:] - batchLoadTestJob(delegate, CommonTestProperties.TriggeringContext.PR) - } - -PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_Java11', - 'Run Load Tests Java 11 ParDo Dataflow V2 Streaming', - 'Load Tests Java 11 ParDo Dataflow V2 Streaming suite', - this - ) { - additionalPipelineArgs = [:] - streamingLoadTestJob(delegate, CommonTestProperties.TriggeringContext.PR) - } diff --git a/.test-infra/jenkins/job_LoadTests_ParDo_Dataflow_V2_Java17.groovy b/.test-infra/jenkins/job_LoadTests_ParDo_Dataflow_V2_Java17.groovy deleted file mode 100644 index df6c66e02aed..000000000000 --- a/.test-infra/jenkins/job_LoadTests_ParDo_Dataflow_V2_Java17.groovy +++ /dev/null @@ -1,217 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonJobProperties as commonJobProperties -import CommonTestProperties -import LoadTestsBuilder as loadTestsBuilder -import PhraseTriggeringPostCommitBuilder -import CronJobBuilder -import InfluxDBCredentialsHelper - -def commonLoadTestConfig = { jobType, isStreaming -> - [ - [ - title : 'Load test: ParDo 2GB 100 byte records 10 times', - test : 'org.apache.beam.sdk.loadtests.ParDoLoadTest', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - project : 'apache-beam-testing', - region : 'us-central1', - appName : "load_tests_Java17_Dataflow_V2_${jobType}_ParDo_1", - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${jobType}_pardo_1", - influxTags : """ - { - "runnerVersion": "v2", - "jdk": "java17" - } - """.trim().replaceAll("\\s", ""), - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 20000000, - "keySizeBytes": 10, - "valueSizeBytes": 90 - } - """.trim().replaceAll("\\s", ""), - iterations : 10, - numberOfCounters : 1, - numberOfCounterOperations: 0, - numWorkers : 5, - autoscalingAlgorithm: "NONE", - streaming : isStreaming - ] - ], - [ - title : 'Load test: ParDo 2GB 100 byte records 200 times', - test : 'org.apache.beam.sdk.loadtests.ParDoLoadTest', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - project : 'apache-beam-testing', - region : 'us-central1', - appName : "load_tests_Java17_Dataflow_V2_${jobType}_ParDo_2", - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${jobType}_pardo_2", - influxTags : """ - { - "runnerVersion": "v2", - "jdk": "java17" - } - """.trim().replaceAll("\\s", ""), - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 20000000, - "keySizeBytes": 10, - "valueSizeBytes": 90 - } - """.trim().replaceAll("\\s", ""), - iterations : 200, - numberOfCounters : 1, - numberOfCounterOperations: 0, - numWorkers : 5, - autoscalingAlgorithm: "NONE", - streaming : isStreaming - ] - ], - [ - - title : 'Load test: ParDo 2GB 100 byte records 10 counters', - test : 'org.apache.beam.sdk.loadtests.ParDoLoadTest', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - project : 'apache-beam-testing', - region : 'us-central1', - appName : "load_tests_Java17_Dataflow_V2_${jobType}_ParDo_3", - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${jobType}_pardo_3", - influxTags : """ - { - "runnerVersion": "v2", - "jdk": "java17" - } - """.trim().replaceAll("\\s", ""), - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 20000000, - "keySizeBytes": 10, - "valueSizeBytes": 90 - } - """.trim().replaceAll("\\s", ""), - iterations : 1, - numberOfCounters : 1, - numberOfCounterOperations: 10, - numWorkers : 5, - autoscalingAlgorithm: "NONE", - streaming : isStreaming - ] - - ], - [ - title : 'Load test: ParDo 2GB 100 byte records 100 counters', - test : 'org.apache.beam.sdk.loadtests.ParDoLoadTest', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - project : 'apache-beam-testing', - region : 'us-central1', - appName : "load_tests_Java17_Dataflow_V2_${jobType}_ParDo_4", - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${jobType}_pardo_4", - influxTags : """ - { - "runnerVersion": "v2", - "jdk": "java17" - } - """.trim().replaceAll("\\s", ""), - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 20000000, - "keySizeBytes": 10, - "valueSizeBytes": 90 - } - """.trim().replaceAll("\\s", ""), - iterations : 1, - numberOfCounters : 1, - numberOfCounterOperations: 100, - numWorkers : 5, - autoscalingAlgorithm: "NONE", - streaming : isStreaming - ] - ] - ].each { test -> test.pipelineOptions.putAll(additionalPipelineArgs) } -} - -def final JOB_SPECIFIC_SWITCHES = [ - '-Prunner.version="V2"', - '-PtestJavaVersion=17', - "-Pjava17Home=${commonJobProperties.JAVA_17_HOME}" -] - -def batchLoadTestJob = { scope, triggeringContext -> - loadTestsBuilder.loadTests(scope, CommonTestProperties.SDK.JAVA, commonLoadTestConfig('batch', false), - "ParDo", "batch", JOB_SPECIFIC_SWITCHES) -} - -def streamingLoadTestJob = {scope, triggeringContext -> - scope.description('Runs Java 17 ParDo load tests on Dataflow runner V2 in streaming mode') - commonJobProperties.setTopLevelMainJobProperties(scope, 'master', 240) - - for (testConfiguration in commonLoadTestConfig('streaming', true)) { - testConfiguration.pipelineOptions << [inputWindowDurationSec: 1200] - loadTestsBuilder.loadTest(scope, testConfiguration.title, testConfiguration.runner, CommonTestProperties.SDK.JAVA, - testConfiguration.pipelineOptions, testConfiguration.test, JOB_SPECIFIC_SWITCHES) - } -} - -CronJobBuilder.cronJob('beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_Java17', 'H H * * *', this) { - additionalPipelineArgs = [ - influxDatabase: InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influxHost: InfluxDBCredentialsHelper.InfluxDBHostUrl, - ] - batchLoadTestJob(delegate, CommonTestProperties.TriggeringContext.POST_COMMIT) -} - -CronJobBuilder.cronJob('beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_Java17', 'H H * * *', this) { - additionalPipelineArgs = [ - influxDatabase: InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influxHost: InfluxDBCredentialsHelper.InfluxDBHostUrl, - ] - streamingLoadTestJob(delegate, CommonTestProperties.TriggeringContext.POST_COMMIT) -} - -PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_Java17', - 'Run Load Tests Java 17 ParDo Dataflow V2 Batch', - 'Load Tests Java 17 ParDo Dataflow V2 Batch suite', - this - ) { - additionalPipelineArgs = [:] - batchLoadTestJob(delegate, CommonTestProperties.TriggeringContext.PR) - } - -PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_Java17', - 'Run Load Tests Java 17 ParDo Dataflow V2 Streaming', - 'Load Tests Java 17 ParDo Dataflow V2 Streaming suite', - this - ) { - additionalPipelineArgs = [:] - streamingLoadTestJob(delegate, CommonTestProperties.TriggeringContext.PR) - } diff --git a/.test-infra/jenkins/job_LoadTests_ParDo_Flink_Go.groovy b/.test-infra/jenkins/job_LoadTests_ParDo_Flink_Go.groovy deleted file mode 100644 index df20312f27b5..000000000000 --- a/.test-infra/jenkins/job_LoadTests_ParDo_Flink_Go.groovy +++ /dev/null @@ -1,152 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonJobProperties as commonJobProperties -import CommonTestProperties -import LoadTestsBuilder as loadTestsBuilder -import PhraseTriggeringPostCommitBuilder -import Flink -import InfluxDBCredentialsHelper - -import static LoadTestsBuilder.DOCKER_BEAM_JOBSERVER -import static LoadTestsBuilder.GO_SDK_CONTAINER - -String now = new Date().format("MMddHHmmss", TimeZone.getTimeZone('UTC')) - - -def batchScenarios = { - [ - [ - title : 'ParDo Go Load test: 20M 100 byte records 10 iterations', - test : 'pardo', - runner : CommonTestProperties.Runner.FLINK, - pipelineOptions: [ - job_name : "load-tests-go-flink-batch-pardo-1-${now}", - influx_measurement : 'go_batch_pardo_1', - influx_namespace : 'flink', - input_options : '\'{' + - '"num_records": 20000000,' + - '"key_size": 10,' + - '"value_size": 90}\'', - iterations : 10, - number_of_counter_operations: 0, - number_of_counters : 0, - parallelism : 5, - endpoint : 'localhost:8099', - environment_type : 'DOCKER', - environment_config : GO_SDK_CONTAINER, - ] - ], - [ - title : 'ParDo Go Load test: 20M 100 byte records 200 times', - test : 'pardo', - runner : CommonTestProperties.Runner.FLINK, - pipelineOptions: [ - job_name : "load-tests-go-flink-batch-pardo-2-${now}", - influx_measurement : 'go_batch_pardo_2', - influx_namespace : 'flink', - input_options : '\'{' + - '"num_records": 20000000,' + - '"key_size": 10,' + - '"value_size": 90}\'', - iterations : 200, - number_of_counter_operations: 0, - number_of_counters : 0, - parallelism : 5, - endpoint : 'localhost:8099', - environment_type : 'DOCKER', - environment_config : GO_SDK_CONTAINER, - ] - ], - [ - title : 'ParDo Go Load test: 20M 100 byte records 10 counters', - test : 'pardo', - runner : CommonTestProperties.Runner.FLINK, - pipelineOptions: [ - job_name : "load-tests-go-flink-batch-pardo-3-${now}", - influx_measurement : 'go_batch_pardo_3', - influx_namespace : 'flink', - input_options : '\'{' + - '"num_records": 20000000,' + - '"key_size": 10,' + - '"value_size": 90}\'', - iterations : 1, - number_of_counter_operations: 10, - number_of_counters : 1, - parallelism : 5, - endpoint : 'localhost:8099', - environment_type : 'DOCKER', - environment_config : GO_SDK_CONTAINER, - ] - ], - [ - title : 'ParDo Go Load test: 20M 100 byte records 100 counters', - test : 'pardo', - runner : CommonTestProperties.Runner.FLINK, - pipelineOptions: [ - job_name : "load-tests-go-flink-batch-pardo-4-${now}", - influx_measurement : 'go_batch_pardo_4', - influx_namespace : 'flink', - input_options : '\'{' + - '"num_records": 20000000,' + - '"key_size": 10,' + - '"value_size": 90}\'', - iterations : 1, - number_of_counter_operations: 100, - number_of_counters : 1, - parallelism : 5, - endpoint : 'localhost:8099', - environment_type : 'DOCKER', - environment_config : GO_SDK_CONTAINER, - ] - ], - ].each { test -> test.pipelineOptions.putAll(additionalPipelineArgs) } -} - -def loadTestJob = { scope, triggeringContext, mode -> - def numberOfWorkers = 5 - - Flink flink = new Flink(scope, "beam_LoadTests_Go_ParDo_Flink_${mode.capitalize()}") - flink.setUp( - [ - GO_SDK_CONTAINER - ], - numberOfWorkers, - "${DOCKER_BEAM_JOBSERVER}/beam_flink${CommonTestProperties.getFlinkVersion()}_job_server:latest") - - loadTestsBuilder.loadTests(scope, CommonTestProperties.SDK.GO, batchScenarios(), 'ParDo', mode) -} - -PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_LoadTests_Go_ParDo_Flink_Batch', - 'Run Load Tests Go ParDo Flink Batch', - 'Load Tests Go ParDo Flink Batch suite', - this - ) { - additionalPipelineArgs = [:] - loadTestJob(delegate, CommonTestProperties.TriggeringContext.PR, 'batch') - } - -CronJobBuilder.cronJob('beam_LoadTests_Go_ParDo_Flink_Batch', 'H H * * *', this) { - additionalPipelineArgs = [ - influx_db_name: InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influx_hostname: InfluxDBCredentialsHelper.InfluxDBHostUrl, - ] - // TODO(BEAM): Fix this test. - loadTestJob(delegate, CommonTestProperties.TriggeringContext.POST_COMMIT, 'batch') -} diff --git a/.test-infra/jenkins/job_LoadTests_ParDo_Go.groovy b/.test-infra/jenkins/job_LoadTests_ParDo_Go.groovy deleted file mode 100644 index a45a146ca93c..000000000000 --- a/.test-infra/jenkins/job_LoadTests_ParDo_Go.groovy +++ /dev/null @@ -1,155 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonJobProperties as commonJobProperties -import CommonTestProperties -import LoadTestsBuilder as loadTestsBuilder -import PhraseTriggeringPostCommitBuilder -import InfluxDBCredentialsHelper - -import static LoadTestsBuilder.GO_SDK_CONTAINER - -String now = new Date().format("MMddHHmmss", TimeZone.getTimeZone('UTC')) - - -def batchScenarios = { - [ - [ - title : 'ParDo Go Load test: 20M 100 byte records 10 iterations', - test : 'pardo', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - job_name : "load-tests-go-dataflow-batch-pardo-1-${now}", - project : 'apache-beam-testing', - region : 'us-central1', - temp_location : 'gs://temp-storage-for-perf-tests/loadtests', - staging_location : 'gs://temp-storage-for-perf-tests/loadtests', - influx_measurement : 'go_batch_pardo_1', - influx_namespace : 'dataflow', - input_options : '\'{' + - '"num_records": 20000000,' + - '"key_size": 10,' + - '"value_size": 90}\'', - iterations : 10, - number_of_counter_operations: 0, - number_of_counters : 0, - num_workers : 5, - autoscaling_algorithm: 'NONE', - environment_type : 'DOCKER', - environment_config : GO_SDK_CONTAINER, - ] - ], - [ - title : 'ParDo Go Load test: 20M 100 byte records 200 times', - test : 'pardo', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - job_name : "load-tests-go-dataflow-batch-pardo-2-${now}", - project : 'apache-beam-testing', - region : 'us-central1', - temp_location : 'gs://temp-storage-for-perf-tests/loadtests', - staging_location : 'gs://temp-storage-for-perf-tests/loadtests', - influx_measurement : 'go_batch_pardo_2', - influx_namespace : 'dataflow', - input_options : '\'{' + - '"num_records": 20000000,' + - '"key_size": 10,' + - '"value_size": 90}\'', - iterations : 200, - number_of_counter_operations: 0, - number_of_counters : 0, - num_workers : 5, - autoscaling_algorithm: 'NONE', - environment_type : 'DOCKER', - environment_config : GO_SDK_CONTAINER, - ] - ], - [ - title : 'ParDo Go Load test: 20M 100 byte records 10 counters', - test : 'pardo', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - job_name : "load-tests-go-dataflow-batch-pardo-3-${now}", - project : 'apache-beam-testing', - region : 'us-central1', - temp_location : 'gs://temp-storage-for-perf-tests/loadtests', - staging_location : 'gs://temp-storage-for-perf-tests/loadtests', - influx_measurement : 'go_batch_pardo_3', - influx_namespace : 'dataflow', - input_options : '\'{' + - '"num_records": 20000000,' + - '"key_size": 10,' + - '"value_size": 90}\'', - iterations : 1, - number_of_counter_operations: 10, - number_of_counters : 1, - num_workers : 5, - autoscaling_algorithm: 'NONE', - environment_type : 'DOCKER', - environment_config : GO_SDK_CONTAINER, - ] - ], - [ - title : 'ParDo Go Load test: 20M 100 byte records 100 counters', - test : 'pardo', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - job_name : "load-tests-go-dataflow-batch-pardo-4-${now}", - project : 'apache-beam-testing', - region : 'us-central1', - temp_location : 'gs://temp-storage-for-perf-tests/loadtests', - staging_location : 'gs://temp-storage-for-perf-tests/loadtests', - influx_measurement : 'go_batch_pardo_4', - influx_namespace : 'dataflow', - input_options : '\'{' + - '"num_records": 20000000,' + - '"key_size": 10,' + - '"value_size": 90}\'', - iterations : 1, - number_of_counter_operations: 100, - number_of_counters : 1, - num_workers : 5, - autoscaling_algorithm: 'NONE', - environment_type : 'DOCKER', - environment_config : GO_SDK_CONTAINER, - ] - ], - ].each { test -> test.pipelineOptions.putAll(additionalPipelineArgs) } -} - -def loadTestJob = { scope, triggeringContext, mode -> - loadTestsBuilder.loadTests(scope, CommonTestProperties.SDK.GO, batchScenarios(), 'ParDo', mode) -} - -PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_LoadTests_Go_ParDo_Dataflow_Batch', - 'Run Load Tests Go ParDo Dataflow Batch', - 'Load Tests Go ParDo Dataflow Batch suite', - this - ) { - additionalPipelineArgs = [:] - loadTestJob(delegate, CommonTestProperties.TriggeringContext.PR, 'batch') - } - -CronJobBuilder.cronJob('beam_LoadTests_Go_ParDo_Dataflow_Batch', 'H H * * *', this) { - additionalPipelineArgs = [ - influx_db_name: InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influx_hostname: InfluxDBCredentialsHelper.InfluxDBHostUrl, - ] - loadTestJob(delegate, CommonTestProperties.TriggeringContext.POST_COMMIT, 'batch') -} diff --git a/.test-infra/jenkins/job_LoadTests_ParDo_Java.groovy b/.test-infra/jenkins/job_LoadTests_ParDo_Java.groovy deleted file mode 100644 index 974ae6b4d4f3..000000000000 --- a/.test-infra/jenkins/job_LoadTests_ParDo_Java.groovy +++ /dev/null @@ -1,186 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonJobProperties as commonJobProperties -import CommonTestProperties -import LoadTestsBuilder as loadTestsBuilder -import PhraseTriggeringPostCommitBuilder -import CronJobBuilder -import InfluxDBCredentialsHelper - -def commonLoadTestConfig = { jobType, isStreaming -> - [ - [ - title : 'Load test: ParDo 2GB 100 byte records 10 times', - test : 'org.apache.beam.sdk.loadtests.ParDoLoadTest', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - project : 'apache-beam-testing', - region : 'us-central1', - appName : "load_tests_Java_Dataflow_${jobType}_ParDo_1", - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${jobType}_pardo_1", - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 20000000, - "keySizeBytes": 10, - "valueSizeBytes": 90 - } - """.trim().replaceAll("\\s", ""), - iterations : 10, - numberOfCounters : 1, - numberOfCounterOperations: 0, - numWorkers : 5, - autoscalingAlgorithm: "NONE", - streaming : isStreaming - ] - ], - [ - title : 'Load test: ParDo 2GB 100 byte records 200 times', - test : 'org.apache.beam.sdk.loadtests.ParDoLoadTest', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - project : 'apache-beam-testing', - region : 'us-central1', - appName : "load_tests_Java_Dataflow_${jobType}_ParDo_2", - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${jobType}_pardo_2", - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 20000000, - "keySizeBytes": 10, - "valueSizeBytes": 90 - } - """.trim().replaceAll("\\s", ""), - iterations : 200, - numberOfCounters : 1, - numberOfCounterOperations: 0, - numWorkers : 5, - autoscalingAlgorithm: "NONE", - streaming : isStreaming - ] - ], - [ - - title : 'Load test: ParDo 2GB 100 byte records 10 counters', - test : 'org.apache.beam.sdk.loadtests.ParDoLoadTest', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - project : 'apache-beam-testing', - region : 'us-central1', - appName : "load_tests_Java_Dataflow_${jobType}_ParDo_3", - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${jobType}_pardo_3", - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 20000000, - "keySizeBytes": 10, - "valueSizeBytes": 90 - } - """.trim().replaceAll("\\s", ""), - iterations : 1, - numberOfCounters : 1, - numberOfCounterOperations: 10, - numWorkers : 5, - autoscalingAlgorithm: "NONE", - streaming : isStreaming - ] - - ], - [ - title : 'Load test: ParDo 2GB 100 byte records 100 counters', - test : 'org.apache.beam.sdk.loadtests.ParDoLoadTest', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - project : 'apache-beam-testing', - region : 'us-central1', - appName : "load_tests_Java_Dataflow_${jobType}_ParDo_4", - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${jobType}_pardo_4", - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 20000000, - "keySizeBytes": 10, - "valueSizeBytes": 90 - } - """.trim().replaceAll("\\s", ""), - iterations : 1, - numberOfCounters : 1, - numberOfCounterOperations: 100, - numWorkers : 5, - autoscalingAlgorithm: "NONE", - streaming : isStreaming - ] - ] - ].each { test -> test.pipelineOptions.putAll(additionalPipelineArgs) } -} - - -def batchLoadTestJob = { scope, triggeringContext -> - loadTestsBuilder.loadTests(scope, CommonTestProperties.SDK.JAVA, commonLoadTestConfig('batch', false), "ParDo", "batch") -} - -def streamingLoadTestJob = {scope, triggeringContext -> - scope.description('Runs Java ParDo load tests on Dataflow runner in streaming mode') - commonJobProperties.setTopLevelMainJobProperties(scope, 'master', 240) - - for (testConfiguration in commonLoadTestConfig('streaming', true)) { - testConfiguration.pipelineOptions << [inputWindowDurationSec: 1200] - loadTestsBuilder.loadTest(scope, testConfiguration.title, testConfiguration.runner, CommonTestProperties.SDK.JAVA, testConfiguration.pipelineOptions, testConfiguration.test) - } -} - -CronJobBuilder.cronJob('beam_LoadTests_Java_ParDo_Dataflow_Batch', 'H H * * *', this) { - additionalPipelineArgs = [ - influxDatabase: InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influxHost: InfluxDBCredentialsHelper.InfluxDBHostUrl, - ] - batchLoadTestJob(delegate, CommonTestProperties.TriggeringContext.POST_COMMIT) -} - -CronJobBuilder.cronJob('beam_LoadTests_Java_ParDo_Dataflow_Streaming', 'H H * * *', this) { - additionalPipelineArgs = [ - influxDatabase: InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influxHost: InfluxDBCredentialsHelper.InfluxDBHostUrl, - ] - streamingLoadTestJob(delegate, CommonTestProperties.TriggeringContext.POST_COMMIT) -} - -PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_LoadTests_Java_ParDo_Dataflow_Batch', - 'Run Load Tests Java ParDo Dataflow Batch', - 'Load Tests Java ParDo Dataflow Batch suite', - this - ) { - additionalPipelineArgs = [:] - batchLoadTestJob(delegate, CommonTestProperties.TriggeringContext.PR) - } - -PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_LoadTests_Java_ParDo_Dataflow_Streaming', - 'Run Load Tests Java ParDo Dataflow Streaming', - 'Load Tests Java ParDo Dataflow Streaming suite', - this - ) { - additionalPipelineArgs = [:] - streamingLoadTestJob(delegate, CommonTestProperties.TriggeringContext.PR) - } diff --git a/.test-infra/jenkins/job_LoadTests_ParDo_Java_spark_structured_streaming.groovy b/.test-infra/jenkins/job_LoadTests_ParDo_Java_spark_structured_streaming.groovy deleted file mode 100644 index 99639636bf64..000000000000 --- a/.test-infra/jenkins/job_LoadTests_ParDo_Java_spark_structured_streaming.groovy +++ /dev/null @@ -1,148 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonTestProperties -import CronJobBuilder -import LoadTestsBuilder as loadTestsBuilder -import PhraseTriggeringPostCommitBuilder -import InfluxDBCredentialsHelper - -def commonLoadTestConfig = { jobType, isStreaming -> - [ - [ - title : 'Load test: ParDo 2GB 100 byte records 10 times', - test : 'org.apache.beam.sdk.loadtests.ParDoLoadTest', - runner : CommonTestProperties.Runner.SPARK_STRUCTURED_STREAMING, - pipelineOptions: [ - project : 'apache-beam-testing', - appName : "load_tests_Java_SparkStructuredStreaming_${jobType}_ParDo_1", - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${jobType}_pardo_1", - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 20000000, - "keySizeBytes": 10, - "valueSizeBytes": 90 - } - """.trim().replaceAll("\\s", ""), - iterations : 10, - numberOfCounters : 1, - numberOfCounterOperations: 0, - streaming : isStreaming - ] - ], - [ - title : 'Load test: ParDo 2GB 100 byte records 200 times', - test : 'org.apache.beam.sdk.loadtests.ParDoLoadTest', - runner : CommonTestProperties.Runner.SPARK_STRUCTURED_STREAMING, - pipelineOptions: [ - project : 'apache-beam-testing', - appName : "load_tests_Java_SparkStructuredStreaming_${jobType}_ParDo_2", - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${jobType}_pardo_2", - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 20000000, - "keySizeBytes": 10, - "valueSizeBytes": 90 - } - """.trim().replaceAll("\\s", ""), - iterations : 200, - numberOfCounters : 1, - numberOfCounterOperations: 0, - streaming : isStreaming - ] - ], - [ - - title : 'Load test: ParDo 2GB 100 byte records 10 counters', - test : 'org.apache.beam.sdk.loadtests.ParDoLoadTest', - runner : CommonTestProperties.Runner.SPARK_STRUCTURED_STREAMING, - pipelineOptions: [ - project : 'apache-beam-testing', - appName : "load_tests_Java_SparkStructuredStreaming_${jobType}_ParDo_3", - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${jobType}_pardo_3", - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 20000000, - "keySizeBytes": 10, - "valueSizeBytes": 90 - } - """.trim().replaceAll("\\s", ""), - iterations : 1, - numberOfCounters : 1, - numberOfCounterOperations: 10, - streaming : isStreaming - ] - - ], - [ - title : 'Load test: ParDo 2GB 100 byte records 100 counters', - test : 'org.apache.beam.sdk.loadtests.ParDoLoadTest', - runner : CommonTestProperties.Runner.SPARK_STRUCTURED_STREAMING, - pipelineOptions: [ - project : 'apache-beam-testing', - appName : "load_tests_Java_SparkStructuredStreaming_${jobType}_ParDo_4", - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${jobType}_pardo_4", - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 20000000, - "keySizeBytes": 10, - "valueSizeBytes": 90 - } - """.trim().replaceAll("\\s", ""), - iterations : 1, - numberOfCounters : 1, - numberOfCounterOperations: 100, - streaming : isStreaming - ] - ] - ].each { test -> test.pipelineOptions.putAll(additionalPipelineArgs) } -} - - -def batchLoadTestJob = { scope, triggeringContext -> - loadTestsBuilder.loadTests(scope, CommonTestProperties.SDK.JAVA, commonLoadTestConfig('batch', false), "ParDo", "batch") -} - - -CronJobBuilder.cronJob('beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch', 'H H * * *', this) { - additionalPipelineArgs = [ - influxDatabase: InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influxHost: InfluxDBCredentialsHelper.InfluxDBHostUrl, - ] - batchLoadTestJob(delegate, CommonTestProperties.TriggeringContext.POST_COMMIT) -} - - -PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch', - 'Run Load Tests Java ParDo SparkStructuredStreaming Batch', - 'Load Tests Java ParDo SparkStructuredStreaming Batch suite', - this - ) { - additionalPipelineArgs = [:] - batchLoadTestJob(delegate, CommonTestProperties.TriggeringContext.PR) - } - diff --git a/.test-infra/jenkins/job_LoadTests_ParDo_Python.groovy b/.test-infra/jenkins/job_LoadTests_ParDo_Python.groovy deleted file mode 100644 index 31a64fade319..000000000000 --- a/.test-infra/jenkins/job_LoadTests_ParDo_Python.groovy +++ /dev/null @@ -1,186 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonJobProperties as commonJobProperties -import LoadTestsBuilder as loadTestsBuilder -import PhraseTriggeringPostCommitBuilder -import InfluxDBCredentialsHelper - -def now = new Date().format("MMddHHmmss", TimeZone.getTimeZone('UTC')) - -def loadTestConfigurations = { mode, datasetName -> - [ - [ - title : 'ParDo Python Load test: 2GB 100 byte records 10 times', - test : 'apache_beam.testing.load_tests.pardo_test', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - job_name : "load-tests-python-dataflow-${mode}-pardo-1-${now}", - project : 'apache-beam-testing', - region : 'us-central1', - temp_location : 'gs://temp-storage-for-perf-tests/loadtests', - publish_to_big_query : true, - metrics_dataset : datasetName, - metrics_table : "python_dataflow_${mode}_pardo_1", - influx_measurement : "python_${mode}_pardo_1", - input_options : '\'{' + - '"num_records": 20000000,' + - '"key_size": 10,' + - '"value_size": 90,' + - '"algorithm": "lcg"}\'', - iterations : 10, - number_of_counter_operations: 0, - number_of_counters : 0, - num_workers : 5, - autoscaling_algorithm: 'NONE', - ] - ], - [ - title : 'ParDo Python Load test: 2GB 100 byte records 200 times', - test : 'apache_beam.testing.load_tests.pardo_test', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - job_name : "load-tests-python-dataflow-${mode}-pardo-2-${now}", - project : 'apache-beam-testing', - region : 'us-central1', - temp_location : 'gs://temp-storage-for-perf-tests/loadtests', - publish_to_big_query : true, - metrics_dataset : datasetName, - metrics_table : "python_dataflow_${mode}_pardo_2", - influx_measurement : "python_${mode}_pardo_2", - input_options : '\'{' + - '"num_records": 20000000,' + - '"key_size": 10,' + - '"value_size": 90,' + - '"algorithm": "lcg"}\'', - iterations : 200, - number_of_counter_operations: 0, - number_of_counters : 0, - num_workers : 5, - autoscaling_algorithm: 'NONE', - ] - ], - [ - title : 'ParDo Python Load test: 2GB 100 byte records 10 counters', - test : 'apache_beam.testing.load_tests.pardo_test', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - job_name : "load-tests-python-dataflow-${mode}-pardo-3-${now}", - project : 'apache-beam-testing', - region : 'us-central1', - temp_location : 'gs://temp-storage-for-perf-tests/loadtests', - publish_to_big_query : true, - metrics_dataset : datasetName, - metrics_table : "python_dataflow_${mode}_pardo_3", - influx_measurement : "python_${mode}_pardo_3", - input_options : '\'{' + - '"num_records": 20000000,' + - '"key_size": 10,' + - '"value_size": 90,' + - '"algorithm": "lcg"}\'', - iterations : 1, - number_of_counter_operations: 10, - number_of_counters : 1, - num_workers : 5, - autoscaling_algorithm: 'NONE', - ] - ], - [ - title : 'ParDo Python Load test: 2GB 100 byte records 100 counters', - test : 'apache_beam.testing.load_tests.pardo_test', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - job_name : "load-tests-python-dataflow-${mode}-pardo-4-${now}", - project : 'apache-beam-testing', - region : 'us-central1', - temp_location : 'gs://temp-storage-for-perf-tests/loadtests', - publish_to_big_query : true, - metrics_dataset : datasetName, - metrics_table : "python_dataflow_${mode}_pardo_4", - influx_measurement : "python_${mode}_pardo_4", - input_options : '\'{' + - '"num_records": 20000000,' + - '"key_size": 10,' + - '"value_size": 90,' + - '"algorithm": "lcg"}\'', - iterations : 1, - number_of_counter_operations: 100, - number_of_counters : 1, - num_workers : 5, - autoscaling_algorithm: 'NONE', - ] - ], - ].each { test -> test.pipelineOptions.putAll(additionalPipelineArgs) } - .each{ test -> (mode != 'streaming') ?: addStreamingOptions(test) } -} - -def addStreamingOptions(test) { - test.pipelineOptions << [ - streaming: null, - // Use the new Dataflow runner, which offers improved efficiency of Dataflow jobs. - // See https://cloud.google.com/dataflow/docs/guides/deploying-a-pipeline#dataflow-runner-v2 - // for more details. - experiments: 'use_runner_v2', - ] -} - -def loadTestJob = { scope, triggeringContext, jobType -> - scope.description("Runs Python ParDo load tests on Dataflow runner in ${jobType} mode") - commonJobProperties.setTopLevelMainJobProperties(scope, 'master', 200) - - def datasetName = loadTestsBuilder.getBigQueryDataset('load_test', triggeringContext) - for (testConfiguration in loadTestConfigurations(jobType, datasetName)) { - loadTestsBuilder.loadTest(scope, testConfiguration.title, testConfiguration.runner, CommonTestProperties.SDK.PYTHON, testConfiguration.pipelineOptions, testConfiguration.test) - } -} - -PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_LoadTests_Python_ParDo_Dataflow_Batch', - 'Run Load Tests Python ParDo Dataflow Batch', - 'Load Tests Python ParDo Dataflow Batch suite', - this - ) { - additionalPipelineArgs = [:] - loadTestJob(delegate, CommonTestProperties.TriggeringContext.PR, "batch") - } - -CronJobBuilder.cronJob('beam_LoadTests_Python_ParDo_Dataflow_Batch', 'H H * * *', this) { - additionalPipelineArgs = [ - influx_db_name: InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influx_hostname: InfluxDBCredentialsHelper.InfluxDBHostUrl, - ] - loadTestJob(delegate, CommonTestProperties.TriggeringContext.POST_COMMIT, "batch") -} - -PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_LoadTests_Python_ParDo_Dataflow_Streaming', - 'Run Python Load Tests ParDo Dataflow Streaming', - 'Load Tests Python ParDo Dataflow Streaming suite', - this - ) { - additionalPipelineArgs = [:] - loadTestJob(delegate, CommonTestProperties.TriggeringContext.PR, "streaming") - } - -CronJobBuilder.cronJob('beam_LoadTests_Python_ParDo_Dataflow_Streaming', 'H H * * *', this) { - additionalPipelineArgs = [ - influx_db_name: InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influx_hostname: InfluxDBCredentialsHelper.InfluxDBHostUrl, - ] - loadTestJob(delegate, CommonTestProperties.TriggeringContext.POST_COMMIT, "streaming") -} diff --git a/.test-infra/jenkins/job_LoadTests_Python_Smoke.groovy b/.test-infra/jenkins/job_LoadTests_Python_Smoke.groovy deleted file mode 100644 index 82a6e958021f..000000000000 --- a/.test-infra/jenkins/job_LoadTests_Python_Smoke.groovy +++ /dev/null @@ -1,71 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import LoadTestsBuilder as loadTestsBuilder -import PhraseTriggeringPostCommitBuilder - -def now = new Date().format("MMddHHmmss", TimeZone.getTimeZone('UTC')) - -def smokeTestConfigurations = { datasetName -> - [ - [ - title : 'GroupByKey Python load test Direct', - test : 'apache_beam.testing.load_tests.group_by_key_test', - runner : CommonTestProperties.Runner.DIRECT, - pipelineOptions: [ - publish_to_big_query: true, - project : 'apache-beam-testing', - metrics_dataset : datasetName, - metrics_table : 'python_direct_gbk', - input_options : '\'{"num_records": 100000,' + - '"key_size": 1,' + - '"value_size":1}\'', - - ] - ], - [ - title : 'GroupByKey Python load test Dataflow', - test : 'apache_beam.testing.load_tests.group_by_key_test', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - job_name : 'load-tests-python-dataflow-batch-gbk-smoke-' + now, - project : 'apache-beam-testing', - region : 'us-central1', - temp_location : 'gs://temp-storage-for-perf-tests/smoketests', - publish_to_big_query: true, - metrics_dataset : datasetName, - metrics_table : 'python_dataflow_gbk', - input_options : '\'{"num_records": 100000,' + - '"key_size": 1,' + - '"value_size":1}\'', - max_num_workers : 1, - ] - ], - ] -} - -// Runs a tiny version load test suite to ensure nothing is broken. -PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_Python_LoadTests_Smoke', - 'Run Python Load Tests Smoke', - 'Python Load Tests Smoke', - this - ) { - def datasetName = loadTestsBuilder.getBigQueryDataset('load_test_SMOKE', CommonTestProperties.TriggeringContext.PR) - loadTestsBuilder.loadTests(delegate, CommonTestProperties.SDK.PYTHON, smokeTestConfigurations(datasetName), "GBK", "smoke") - } diff --git a/.test-infra/jenkins/job_LoadTests_SideInput_Go.groovy b/.test-infra/jenkins/job_LoadTests_SideInput_Go.groovy deleted file mode 100644 index 35484d4cc3ea..000000000000 --- a/.test-infra/jenkins/job_LoadTests_SideInput_Go.groovy +++ /dev/null @@ -1,100 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonJobProperties as commonJobProperties -import CommonTestProperties -import LoadTestsBuilder as loadTestsBuilder -import PhraseTriggeringPostCommitBuilder -import InfluxDBCredentialsHelper - -import static LoadTestsBuilder.GO_SDK_CONTAINER - -String now = new Date().format('MMddHHmmss', TimeZone.getTimeZone('UTC')) - -def batchScenarios = { - [ - [ - title : 'SideInput Go Load test: 10gb-1kb-10workers-1window-first-iterable', - test : 'sideinput', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - job_name : "load-tests-go-dataflow-batch-sideinput-3-${now}", - project : 'apache-beam-testing', - region : 'us-central1', - temp_location : 'gs://temp-storage-for-perf-tests/loadtests', - staging_location : 'gs://temp-storage-for-perf-tests/loadtests', - influx_namespace : 'dataflow', - influx_measurement : 'go_batch_sideinput_3', - input_options : '\'{' + - '"num_records": 10000000,' + - '"key_size": 100,' + - '"value_size": 900}\'', - access_percentage: 1, - num_workers : 10, - autoscaling_algorithm: 'NONE', - environment_type : 'DOCKER', - environment_config : GO_SDK_CONTAINER, - ] - ], - [ - title : 'SideInput Go Load test: 10gb-1kb-10workers-1window-iterable', - test : 'sideinput', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - job_name : "load-tests-go-dataflow-batch-sideinput-4-${now}", - project : 'apache-beam-testing', - region : 'us-central1', - temp_location : 'gs://temp-storage-for-perf-tests/loadtests', - staging_location : 'gs://temp-storage-for-perf-tests/loadtests', - influx_namespace : 'dataflow', - influx_measurement : 'go_batch_sideinput_4', - input_options : '\'{' + - '"num_records": 10000000,' + - '"key_size": 100,' + - '"value_size": 900}\'', - num_workers : 10, - autoscaling_algorithm: 'NONE', - environment_type : 'DOCKER', - environment_config : GO_SDK_CONTAINER, - ] - ] - ] - .each { test -> test.pipelineOptions.putAll(additionalPipelineArgs) } -} - -def loadTestJob = { scope, triggeringContext, mode -> - loadTestsBuilder.loadTests(scope, CommonTestProperties.SDK.GO, batchScenarios(), 'sideinput', mode) -} - -PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_LoadTests_Go_SideInput_Dataflow_Batch', - 'Run Load Tests Go SideInput Dataflow Batch', - 'Load Tests Go SideInput Dataflow Batch suite', - this - ) { - additionalPipelineArgs = [:] - loadTestJob(delegate, CommonTestProperties.TriggeringContext.PR, 'batch') - } - -CronJobBuilder.cronJob('beam_LoadTests_Go_SideInput_Dataflow_Batch', 'H H * * *', this) { - additionalPipelineArgs = [ - influx_db_name: InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influx_hostname: InfluxDBCredentialsHelper.InfluxDBHostUrl, - ] - loadTestJob(delegate, CommonTestProperties.TriggeringContext.POST_COMMIT, 'batch') -} diff --git a/.test-infra/jenkins/job_LoadTests_SideInput_Python.groovy b/.test-infra/jenkins/job_LoadTests_SideInput_Python.groovy deleted file mode 100644 index f6655bfe4324..000000000000 --- a/.test-infra/jenkins/job_LoadTests_SideInput_Python.groovy +++ /dev/null @@ -1,198 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonJobProperties as commonJobProperties -import LoadTestsBuilder as loadTestsBuilder -import PhraseTriggeringPostCommitBuilder -import InfluxDBCredentialsHelper - -def now = new Date().format("MMddHHmmss", TimeZone.getTimeZone('UTC')) - -def fromTemplate = { mode, name, id, datasetName, testSpecificOptions -> - [ - title : "SideInput Python Load test: ${name}", - test : 'apache_beam.testing.load_tests.sideinput_test', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - job_name : "load-tests-python-dataflow-${mode}-sideinput-${id}-${now}", - project : 'apache-beam-testing', - region : 'us-central1', - temp_location : 'gs://temp-storage-for-perf-tests/loadtests', - publish_to_big_query : true, - metrics_dataset : datasetName, - metrics_table : "python_dataflow_${mode}_sideinput_${id}", - influx_measurement : "python_${mode}_sideinput_${id}", - num_workers : 10, - autoscaling_algorithm: 'NONE', - experiments : 'use_runner_v2', - ] << testSpecificOptions - ] -} - -def loadTestConfigurations = { mode, datasetName -> - [ - [ - name: '1gb-1kb-10workers-1window-1key-percent-dict', - testSpecificOptions: [ - input_options : '\'{' + - '"num_records": 1000000,' + - '"key_size": 100,' + - '"value_size": 900,' + - '"algorithm": "lcg"}\'', - side_input_type : 'dict', - access_percentage: 1, - ] - ], - [ - name: '1gb-1kb-10workers-1window-99key-percent-dict', - testSpecificOptions: [ - input_options : '\'{' + - '"num_records": 1000000,' + - '"key_size": 100,' + - '"value_size": 900,' + - '"algorithm": "lcg"}\'', - side_input_type : 'dict', - access_percentage: 99, - ] - ], - [ - name: '10gb-1kb-10workers-1window-first-iterable', - testSpecificOptions: [ - input_options : '\'{' + - '"num_records": 10000000,' + - '"key_size": 100,' + - '"value_size": 900,' + - '"algorithm": "lcg"}\'', - side_input_type : 'iter', - access_percentage: 1, - ] - ], - [ - name: '10gb-1kb-10workers-1window-iterable', - testSpecificOptions: [ - input_options : '\'{' + - '"num_records": 10000000,' + - '"key_size": 100,' + - '"value_size": 900,' + - '"algorithm": "lcg"}\'', - side_input_type : 'iter', - ] - ], - [ - name: '1gb-1kb-10workers-1window-first-list', - testSpecificOptions: [ - input_options : '\'{' + - '"num_records": 1000000,' + - '"key_size": 100,' + - '"value_size": 900,' + - '"algorithm": "lcg"}\'', - side_input_type : 'list', - access_percentage: 1, - ] - ], - [ - name: '1gb-1kb-10workers-1window-list', - testSpecificOptions: [ - input_options : '\'{' + - '"num_records": 1000000,' + - '"key_size": 100,' + - '"value_size": 900,' + - '"algorithm": "lcg"}\'', - side_input_type : 'list', - ] - ], - [ - name: '1gb-1kb-10workers-1000window-1key-percent-dict', - testSpecificOptions: [ - input_options : '\'{' + - '"num_records": 1000000,' + - '"key_size": 100,' + - '"value_size": 900,' + - '"algorithm": "lcg"}\'', - side_input_type : 'dict', - access_percentage: 1, - window_count : 1000, - ] - ], - [ - name: '1gb-1kb-10workers-1000window-99key-percent-dict', - testSpecificOptions: [ - input_options : '\'{' + - '"num_records": 1000000,' + - '"key_size": 100,' + - '"value_size": 900,' + - '"algorithm": "lcg"}\'', - side_input_type : 'dict', - access_percentage: 99, - window_count : 1000, - ] - ], - [ - name: '10gb-1kb-10workers-1000window-first-iterable', - testSpecificOptions: [ - input_options : '\'{' + - '"num_records": 10000000,' + - '"key_size": 100,' + - '"value_size": 900,' + - '"algorithm": "lcg"}\'', - side_input_type : 'iter', - access_percentage: 1, - window_count : 1000, - ] - ], - [ - name: '10gb-1kb-10workers-1000window-iterable', - testSpecificOptions: [ - input_options : '\'{' + - '"num_records": 10000000,' + - '"key_size": 100,' + - '"value_size": 900,' + - '"algorithm": "lcg"}\'', - side_input_type : 'iter', - window_count : 1000, - ] - ], - ].indexed().collect { index, it -> - fromTemplate(mode, it.name, index + 1, datasetName, it.testSpecificOptions << additionalPipelineArgs) - } -} - - -def loadTestJob = { scope, triggeringContext, mode -> - def datasetName = loadTestsBuilder.getBigQueryDataset('load_test', triggeringContext) - loadTestsBuilder.loadTests(scope, CommonTestProperties.SDK.PYTHON, - loadTestConfigurations(mode, datasetName), 'SideInput', mode) -} - -PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_LoadTests_Python_SideInput_Dataflow_Batch', - 'Run Load Tests Python SideInput Dataflow Batch', - 'Load Tests Python SideInput Dataflow Batch suite', - this - ) { - additionalPipelineArgs = [:] - loadTestJob(delegate, CommonTestProperties.TriggeringContext.PR, 'batch') - } - -CronJobBuilder.cronJob('beam_LoadTests_Python_SideInput_Dataflow_Batch', 'H H * * *', this) { - additionalPipelineArgs = [ - influx_db_name: InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influx_hostname: InfluxDBCredentialsHelper.InfluxDBHostUrl, - ] - loadTestJob(delegate, CommonTestProperties.TriggeringContext.POST_COMMIT, 'batch') -} diff --git a/.test-infra/jenkins/job_LoadTests_coGBK_Flink_Python.groovy b/.test-infra/jenkins/job_LoadTests_coGBK_Flink_Python.groovy deleted file mode 100644 index 9a0798f8107b..000000000000 --- a/.test-infra/jenkins/job_LoadTests_coGBK_Flink_Python.groovy +++ /dev/null @@ -1,156 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonJobProperties as commonJobProperties -import CommonTestProperties -import LoadTestsBuilder as loadTestsBuilder -import PhraseTriggeringPostCommitBuilder -import Flink -import InfluxDBCredentialsHelper - -import static LoadTestsBuilder.DOCKER_CONTAINER_REGISTRY -import static LoadTestsBuilder.DOCKER_BEAM_SDK_IMAGE -import static LoadTestsBuilder.DOCKER_BEAM_JOBSERVER - -String now = new Date().format("MMddHHmmss", TimeZone.getTimeZone('UTC')) - -def scenarios = { datasetName -> - [ - [ - title : 'CoGroupByKey Python Load test: 2GB of 100B records with a single key', - test : 'apache_beam.testing.load_tests.co_group_by_key_test', - runner : CommonTestProperties.Runner.PORTABLE, - pipelineOptions: [ - project : 'apache-beam-testing', - job_name : 'load-tests-python-flink-batch-cogbk-1-' + now, - temp_location : 'gs://temp-storage-for-perf-tests/loadtests', - publish_to_big_query : true, - metrics_dataset : datasetName, - metrics_table : "python_flink_batch_cogbk_1", - influx_measurement : 'python_batch_cogbk_1', - input_options : '\'{' + - '"num_records": 20000000,' + - '"key_size": 10,' + - '"value_size": 90,' + - '"num_hot_keys": 1,' + - '"hot_key_fraction": 1}\'', - co_input_options : '\'{' + - '"num_records": 2000000,' + - '"key_size": 10,' + - '"value_size": 90,' + - '"num_hot_keys": 1000,' + - '"hot_key_fraction": 1}\'', - iterations : 1, - parallelism : 5, - job_endpoint : 'localhost:8099', - environment_type : 'DOCKER', - environment_config : "${DOCKER_CONTAINER_REGISTRY}/${DOCKER_BEAM_SDK_IMAGE}", - ] - ], - [ - title : 'CoGroupByKey Python Load test: 2GB of 100B records with multiple keys', - test : 'apache_beam.testing.load_tests.co_group_by_key_test', - runner : CommonTestProperties.Runner.PORTABLE, - pipelineOptions: [ - project : 'apache-beam-testing', - job_name : 'load-tests-python-flink-batch-cogbk-2-' + now, - temp_location : 'gs://temp-storage-for-perf-tests/loadtests', - publish_to_big_query : true, - metrics_dataset : datasetName, - metrics_table : 'python_flink_batch_cogbk_2', - influx_measurement : 'python_batch_cogbk_2', - input_options : '\'{' + - '"num_records": 20000000,' + - '"key_size": 10,' + - '"value_size": 90,' + - '"num_hot_keys": 5,' + - '"hot_key_fraction": 1}\'', - co_input_options : '\'{' + - '"num_records": 2000000,' + - '"key_size": 10,' + - '"value_size": 90,' + - '"num_hot_keys": 1000,' + - '"hot_key_fraction": 1}\'', - iterations : 1, - parallelism : 5, - job_endpoint : 'localhost:8099', - environment_type : 'DOCKER', - environment_config : "${DOCKER_CONTAINER_REGISTRY}/${DOCKER_BEAM_SDK_IMAGE}", - ] - ], - [ - title : 'CoGroupByKey Python Load test: reiterate 4 times 10kB values', - test : 'apache_beam.testing.load_tests.co_group_by_key_test', - runner : CommonTestProperties.Runner.PORTABLE, - pipelineOptions: [ - project : 'apache-beam-testing', - job_name : 'load-tests-python-flink-batch-cogbk-3-' + now, - temp_location : 'gs://temp-storage-for-perf-tests/loadtests', - publish_to_big_query : true, - metrics_dataset : datasetName, - metrics_table : "python_flink_batch_cogbk_3", - influx_measurement : 'python_batch_cogbk_3', - input_options : '\'{' + - '"num_records": 20000000,' + - '"key_size": 10,' + - '"value_size": 90,' + - '"num_hot_keys": 200000,' + - '"hot_key_fraction": 1}\'', - co_input_options : '\'{' + - '"num_records": 2000000,' + - '"key_size": 10,' + - '"value_size": 90,' + - '"num_hot_keys": 1000,' + - '"hot_key_fraction": 1}\'', - iterations : 4, - parallelism : 5, - job_endpoint : 'localhost:8099', - environment_type : 'DOCKER', - environment_config : "${DOCKER_CONTAINER_REGISTRY}/${DOCKER_BEAM_SDK_IMAGE}", - ] - ], - ].each { test -> test.pipelineOptions.putAll(additionalPipelineArgs) } -} - -def loadTest = { scope, triggeringContext -> - def datasetName = loadTestsBuilder.getBigQueryDataset('load_test', triggeringContext) - def numberOfWorkers = 5 - List<Map> testScenarios = scenarios(datasetName) - - def flink = new Flink(scope, 'beam_LoadTests_Python_CoGBK_Flink_Batch') - flink.setUp( - [ - "${DOCKER_CONTAINER_REGISTRY}/${DOCKER_BEAM_SDK_IMAGE}" - ], - numberOfWorkers, - "${DOCKER_BEAM_JOBSERVER}/beam_flink${CommonTestProperties.getFlinkVersion()}_job_server:latest") - - loadTestsBuilder.loadTests(scope, CommonTestProperties.SDK.PYTHON, testScenarios, 'CoGBK', 'batch') -} - -PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_LoadTests_Python_CoGBK_Flink_Batch', - 'Run Load Tests Python CoGBK Flink Batch', - 'Load Tests Python CoGBK Flink Batch suite', - this - ) { - additionalPipelineArgs = [:] - loadTest(delegate, CommonTestProperties.TriggeringContext.PR) - } - -// TODO(https://github.com/apache/beam/issues/20146) Re-enable auto builds after these tests pass. diff --git a/.test-infra/jenkins/job_LoadTests_coGBK_Go.groovy b/.test-infra/jenkins/job_LoadTests_coGBK_Go.groovy deleted file mode 100644 index cab73e55968a..000000000000 --- a/.test-infra/jenkins/job_LoadTests_coGBK_Go.groovy +++ /dev/null @@ -1,179 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonJobProperties as commonJobProperties -import CommonTestProperties -import LoadTestsBuilder as loadTestsBuilder -import PhraseTriggeringPostCommitBuilder -import InfluxDBCredentialsHelper - -import static LoadTestsBuilder.GO_SDK_CONTAINER - -String now = new Date().format("MMddHHmmss", TimeZone.getTimeZone('UTC')) - -def batchScenarios = { - [ - [ - title : 'CoGroupByKey Go Load test: 2GB of 100B records with a single key', - test : 'cogbk', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - job_name : "load-tests-go-dataflow-batch-cogbk-1-${now}", - project : 'apache-beam-testing', - region : 'us-central1', - temp_location : 'gs://temp-storage-for-perf-tests/loadtests', - staging_location : 'gs://temp-storage-for-perf-tests/loadtests', - influx_measurement : 'go_batch_cogbk_1', - influx_namespace : 'dataflow', - input_options : '\'{' + - '"num_records": 20000000,' + - '"key_size": 10,' + - '"value_size": 90,' + - '"num_hot_keys": 1,' + - '"hot_key_fraction": 1}\'', - co_input_options : '\'{' + - '"num_records": 2000000,' + - '"key_size": 10,' + - '"value_size": 90,' + - '"num_hot_keys": 1000,' + - '"hot_key_fraction": 1}\'', - iterations : 1, - num_workers : 5, - autoscaling_algorithm: 'NONE', - environment_type : 'DOCKER', - environment_config : GO_SDK_CONTAINER, - ] - ], - [ - title : 'CoGroupByKey Go Load test: 2GB of 100B records with multiple keys', - test : 'cogbk', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - job_name : "load-tests-go-dataflow-batch-cogbk-2-${now}", - project : 'apache-beam-testing', - region : 'us-central1', - temp_location : 'gs://temp-storage-for-perf-tests/loadtests', - staging_location : 'gs://temp-storage-for-perf-tests/loadtests', - influx_measurement : 'go_batch_cogbk_2', - influx_namespace : 'dataflow', - input_options : '\'{' + - '"num_records": 20000000,' + - '"key_size": 10,' + - '"value_size": 90,' + - '"num_hot_keys": 5,' + - '"hot_key_fraction": 1}\'', - co_input_options : '\'{' + - '"num_records": 2000000,' + - '"key_size": 10,' + - '"value_size": 90,' + - '"num_hot_keys": 1000,' + - '"hot_key_fraction": 1}\'', - iterations : 1, - num_workers : 5, - autoscaling_algorithm: 'NONE', - environment_type : 'DOCKER', - environment_config : GO_SDK_CONTAINER, - ] - ], - [ - title : 'CoGroupByKey Go Load test: reiterate 4 times 10kB values', - test : 'cogbk', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - job_name : "load-tests-go-dataflow-batch-cogbk-3-${now}", - project : 'apache-beam-testing', - region : 'us-central1', - temp_location : 'gs://temp-storage-for-perf-tests/loadtests', - staging_location : 'gs://temp-storage-for-perf-tests/loadtests', - influx_measurement : 'go_batch_cogbk_3', - influx_namespace : 'dataflow', - input_options : '\'{' + - '"num_records": 20000000,' + - '"key_size": 10,' + - '"value_size": 90,' + - '"num_hot_keys": 200000,' + - '"hot_key_fraction": 1}\'', - co_input_options : '\'{' + - '"num_records": 2000000,' + - '"key_size": 10,' + - '"value_size": 90,' + - '"num_hot_keys": 1000,' + - '"hot_key_fraction": 1}\'', - iterations : 4, - num_workers : 5, - autoscaling_algorithm: 'NONE', - environment_type : 'DOCKER', - environment_config : GO_SDK_CONTAINER, - ] - ], - [ - title : 'CoGroupByKey Go Load test: reiterate 4 times 2MB values', - test : 'cogbk', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - job_name : "load-tests-go-dataflow-batch-cogbk-4-${now}", - project : 'apache-beam-testing', - region : 'us-central1', - temp_location : 'gs://temp-storage-for-perf-tests/loadtests', - staging_location : 'gs://temp-storage-for-perf-tests/loadtests', - influx_measurement : 'go_batch_cogbk_4', - influx_namespace : 'dataflow', - input_options : '\'{' + - '"num_records": 20000000,' + - '"key_size": 10,' + - '"value_size": 90,' + - '"num_hot_keys": 1000,' + - '"hot_key_fraction": 1}\'', - co_input_options : '\'{' + - '"num_records": 2000000,' + - '"key_size": 10,' + - '"value_size": 90,' + - '"num_hot_keys": 1000,' + - '"hot_key_fraction": 1}\'', - iterations : 4, - num_workers : 5, - autoscaling_algorithm: 'NONE', - environment_type : 'DOCKER', - environment_config : GO_SDK_CONTAINER, - ] - ], - ] - .each { test -> test.pipelineOptions.putAll(additionalPipelineArgs) } -} - -def loadTestJob = { scope, triggeringContext, mode -> - loadTestsBuilder.loadTests(scope, CommonTestProperties.SDK.GO, batchScenarios(), 'CoGBK', mode) -} - -PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_LoadTests_Go_CoGBK_Dataflow_Batch', - 'Run Load Tests Go CoGBK Dataflow Batch', - 'Load Tests Go CoGBK Dataflow Batch suite', - this - ) { - additionalPipelineArgs = [:] - loadTestJob(delegate, CommonTestProperties.TriggeringContext.PR, 'batch') - } - -CronJobBuilder.cronJob('beam_LoadTests_Go_CoGBK_Dataflow_batch', 'H H * * *', this) { - additionalPipelineArgs = [ - influx_db_name: InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influx_hostname: InfluxDBCredentialsHelper.InfluxDBHostUrl, - ] - loadTestJob(delegate, CommonTestProperties.TriggeringContext.POST_COMMIT, 'batch') -} diff --git a/.test-infra/jenkins/job_PerformanceTests_BigQueryIO_Java.groovy b/.test-infra/jenkins/job_PerformanceTests_BigQueryIO_Java.groovy deleted file mode 100644 index 270aea0b7334..000000000000 --- a/.test-infra/jenkins/job_PerformanceTests_BigQueryIO_Java.groovy +++ /dev/null @@ -1,145 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonJobProperties as common -import PhraseTriggeringPostCommitBuilder -import InfluxDBCredentialsHelper - -def now = new Date().format("MMddHHmmss", TimeZone.getTimeZone('UTC')) - -def jobConfigs = [ - [ - title : 'BigQueryIO Performance Test Streaming Java 10 GB', - triggerPhrase: 'Run BigQueryIO Streaming Performance Test Java', - name : 'beam_PerformanceTests_BiqQueryIO_Streaming_Java', - itClass : 'org.apache.beam.sdk.bigqueryioperftests.BigQueryIOIT', - properties: [ - project : 'apache-beam-testing', - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - tempRoot : 'gs://temp-storage-for-perf-tests/loadtests', - writeMethod : 'STREAMING_INSERTS', - writeFormat : 'JSON', - pipelineTimeout : '1200', - testBigQueryDataset : 'beam_performance', - testBigQueryTable : 'bqio_write_10GB_java_stream_' + now, - metricsBigQueryDataset: 'beam_performance', - metricsBigQueryTable : 'bqio_10GB_results_java_stream', - influxMeasurement : 'bqio_10GB_results_java_stream', - sourceOptions : """ - { - "numRecords": "10485760", - "keySizeBytes": "1", - "valueSizeBytes": "1024" - } - """.trim().replaceAll("\\s", ""), - runner : 'DataflowRunner', - maxNumWorkers : '5', - numWorkers : '5', - autoscalingAlgorithm : 'NONE', - ] - ], - [ - title : 'BigQueryIO Performance Test Batch Java 10 GB JSON', - triggerPhrase: 'Run BigQueryIO Batch Performance Test Java Json', - name : 'beam_PerformanceTests_BiqQueryIO_Batch_Java_Json', - itClass : 'org.apache.beam.sdk.bigqueryioperftests.BigQueryIOIT', - properties: [ - project : 'apache-beam-testing', - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - tempRoot : 'gs://temp-storage-for-perf-tests/loadtests', - writeMethod : 'FILE_LOADS', - writeFormat : 'JSON', - testBigQueryDataset : 'beam_performance', - testBigQueryTable : 'bqio_write_10GB_java_json_' + now, - metricsBigQueryDataset: 'beam_performance', - metricsBigQueryTable : 'bqio_10GB_results_java_batch_json', - influxMeasurement : 'bqio_10GB_results_java_batch_json', - sourceOptions : """ - { - "numRecords": "10485760", - "keySizeBytes": "1", - "valueSizeBytes": "1024" - } - """.trim().replaceAll("\\s", ""), - runner : "DataflowRunner", - maxNumWorkers : '5', - numWorkers : '5', - autoscalingAlgorithm : 'NONE', - ] - ], - [ - title : 'BigQueryIO Performance Test Batch Java 10 GB AVRO', - triggerPhrase: 'Run BigQueryIO Batch Performance Test Java Avro', - name : 'beam_PerformanceTests_BiqQueryIO_Batch_Java_Avro', - itClass : 'org.apache.beam.sdk.bigqueryioperftests.BigQueryIOIT', - properties: [ - project : 'apache-beam-testing', - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - tempRoot : 'gs://temp-storage-for-perf-tests/loadtests', - writeMethod : 'FILE_LOADS', - writeFormat : 'AVRO', - testBigQueryDataset : 'beam_performance', - testBigQueryTable : 'bqio_write_10GB_java_avro_' + now, - metricsBigQueryDataset: 'beam_performance', - metricsBigQueryTable : 'bqio_10GB_results_java_batch_avro', - influxMeasurement : 'bqio_10GB_results_java_batch_avro', - sourceOptions : """ - { - "numRecords": "10485760", - "keySizeBytes": "1", - "valueSizeBytes": "1024" - } - """.trim().replaceAll("\\s", ""), - runner : "DataflowRunner", - maxNumWorkers : '5', - numWorkers : '5', - autoscalingAlgorithm : 'NONE', - ] - ] -] - -jobConfigs.forEach { jobConfig -> createPostCommitJob(jobConfig)} - -private void createPostCommitJob(jobConfig) { - job(jobConfig.name) { - description(jobConfig.description) - common.setTopLevelMainJobProperties(delegate) - common.enablePhraseTriggeringFromPullRequest(delegate, jobConfig.title, jobConfig.triggerPhrase) - common.setAutoJob(delegate, 'H H/12 * * *') - publishers { - archiveJunit('**/build/test-results/**/*.xml') - } - InfluxDBCredentialsHelper.useCredentials(delegate) - additionalPipelineArgs = [ - influxDatabase: InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influxHost: InfluxDBCredentialsHelper.InfluxDBHostUrl, - ] - jobConfig.properties.putAll(additionalPipelineArgs) - - steps { - gradle { - rootBuildScriptDir(common.checkoutDir) - common.setGradleSwitches(delegate) - switches("--info") - switches("-DintegrationTestPipelineOptions=\'${common.joinOptionsWithNestedJsonValues(jobConfig.properties)}\'") - switches("-DintegrationTestRunner=dataflow") - tasks(":sdks:java:io:bigquery-io-perf-tests:integrationTest --tests ${jobConfig.itClass}") - } - } - } -} diff --git a/.test-infra/jenkins/job_PerformanceTests_BigQueryIO_Python.groovy b/.test-infra/jenkins/job_PerformanceTests_BigQueryIO_Python.groovy deleted file mode 100644 index 571410cb3563..000000000000 --- a/.test-infra/jenkins/job_PerformanceTests_BigQueryIO_Python.groovy +++ /dev/null @@ -1,110 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonJobProperties as commonJobProperties -import LoadTestsBuilder as loadTestsBuilder -import PhraseTriggeringPostCommitBuilder -import InfluxDBCredentialsHelper - -def now = new Date().format("MMddHHmmss", TimeZone.getTimeZone('UTC')) - -def bqio_read_test = [ - title : 'BigQueryIO Read Performance Test Python 10 GB', - test : 'apache_beam.io.gcp.bigquery_read_perf_test', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - job_name : 'performance-tests-bqio-read-python-10gb' + now, - project : 'apache-beam-testing', - region : 'us-central1', - temp_location : 'gs://temp-storage-for-perf-tests/loadtests', - input_dataset : 'beam_performance', - input_table : 'bqio_read_10GB', - publish_to_big_query : true, - metrics_dataset : 'beam_performance', - metrics_table : 'bqio_read_10GB_results', - influx_measurement : 'python_bqio_read_10GB_results', - influx_db_name : InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influx_hostname : InfluxDBCredentialsHelper.InfluxDBHostUrl, - input_options : '\'{' + - '"num_records": 10485760,' + - '"key_size": 1,' + - '"value_size": 1024,' + - '"algorithm": "lcg"}\'', - num_workers : 5, - autoscaling_algorithm: 'NONE', // Disable autoscale the worker pool. - ] -] - -def bqio_write_test = [ - title : 'BigQueryIO Write Performance Test Python Batch 10 GB', - test : 'apache_beam.io.gcp.bigquery_write_perf_test', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - job_name : 'performance-tests-bqio-write-python-batch-10gb' + now, - project : 'apache-beam-testing', - region : 'us-central1', - temp_location : 'gs://temp-storage-for-perf-tests/loadtests', - output_dataset : 'beam_performance', - output_table : 'bqio_write_10GB', - publish_to_big_query : true, - metrics_dataset : 'beam_performance', - metrics_table : 'bqio_write_10GB_results', - influx_measurement : 'python_bqio_write_10GB_results', - influx_db_name : InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influx_hostname : InfluxDBCredentialsHelper.InfluxDBHostUrl, - input_options : '\'{' + - '"num_records": 10485760,' + - '"key_size": 1,' + - '"value_size": 1024,' + - '"algorithm": "lcg"}\'', - num_workers : 5, - autoscaling_algorithm: 'NONE', // Disable autoscale the worker pool. - ] -] - -def executeJob = { scope, testConfig -> - commonJobProperties.setTopLevelMainJobProperties(scope, 'master', 240) - - loadTestsBuilder.loadTest(scope, testConfig.title, testConfig.runner, CommonTestProperties.SDK.PYTHON, testConfig.pipelineOptions, testConfig.test) -} - -PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_PerformanceTests_BiqQueryIO_Read_Python', - 'Run BigQueryIO Read Performance Test Python', - 'BigQueryIO Read Performance Test Python', - this - ) { - executeJob(delegate, bqio_read_test) - } - -CronJobBuilder.cronJob('beam_PerformanceTests_BiqQueryIO_Read_Python', 'H H * * *', this) { - executeJob(delegate, bqio_read_test) -} - -PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_PerformanceTests_BiqQueryIO_Write_Python_Batch', - 'Run BigQueryIO Write Performance Test Python Batch', - 'BigQueryIO Write Performance Test Python Batch', - this - ) { - executeJob(delegate, bqio_write_test) - } - -CronJobBuilder.cronJob('beam_PerformanceTests_BiqQueryIO_Write_Python_Batch', 'H H * * *', this) { - executeJob(delegate, bqio_write_test) -} diff --git a/.test-infra/jenkins/job_PerformanceTests_CdapIO.groovy b/.test-infra/jenkins/job_PerformanceTests_CdapIO.groovy deleted file mode 100644 index 94b4d677587b..000000000000 --- a/.test-infra/jenkins/job_PerformanceTests_CdapIO.groovy +++ /dev/null @@ -1,72 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonJobProperties as common -import Kubernetes -import InfluxDBCredentialsHelper - -String jobName = "beam_PerformanceTests_Cdap" - -job(jobName) { - common.setTopLevelMainJobProperties(delegate) - common.setAutoJob(delegate, 'H H/12 * * *') - common.enablePhraseTriggeringFromPullRequest( - delegate, - 'Java CdapIO Performance Test', - 'Run Java CdapIO Performance Test') - InfluxDBCredentialsHelper.useCredentials(delegate) - - String namespace = common.getKubernetesNamespace(jobName) - String kubeconfig = common.getKubeconfigLocationForNamespace(namespace) - Kubernetes k8s = Kubernetes.create(delegate, kubeconfig, namespace) - - k8s.apply(common.makePathAbsolute("src/.test-infra/kubernetes/postgres/postgres-service-for-local-dev.yml")) - String postgresHostName = "LOAD_BALANCER_IP" - k8s.loadBalancerIP("postgres-for-dev", postgresHostName) - - Map pipelineOptions = [ - tempRoot : 'gs://temp-storage-for-perf-tests', - project : 'apache-beam-testing', - runner : 'DataflowRunner', - numberOfRecords : '5000000', - bigQueryDataset : 'beam_performance', - bigQueryTable : 'cdapioit_results', - influxMeasurement : 'cdapioit_results', - influxDatabase : InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influxHost : InfluxDBCredentialsHelper.InfluxDBHostUrl, - postgresUsername : 'postgres', - postgresPassword : 'uuinkks', - postgresDatabaseName : 'postgres', - postgresServerName : "\$${postgresHostName}", - postgresSsl : false, - postgresPort : '5432', - numWorkers : '5', - autoscalingAlgorithm : 'NONE' - ] - - steps { - gradle { - rootBuildScriptDir(common.checkoutDir) - common.setGradleSwitches(delegate) - switches("--info") - switches("-DintegrationTestPipelineOptions=\'${common.joinPipelineOptions(pipelineOptions)}\'") - switches("-DintegrationTestRunner=dataflow") - tasks(":sdks:java:io:cdap:integrationTest --tests org.apache.beam.sdk.io.cdap.CdapIOIT") - } - } -} diff --git a/.test-infra/jenkins/job_PerformanceTests_FileBasedIO_IT.groovy b/.test-infra/jenkins/job_PerformanceTests_FileBasedIO_IT.groovy deleted file mode 100644 index c2208d8b00ad..000000000000 --- a/.test-infra/jenkins/job_PerformanceTests_FileBasedIO_IT.groovy +++ /dev/null @@ -1,384 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonJobProperties as common -import InfluxDBCredentialsHelper - -def jobs = [ - [ - name : 'beam_PerformanceTests_TextIOIT', - description : 'Runs performance tests for TextIOIT', - test : 'org.apache.beam.sdk.io.text.TextIOIT', - githubTitle : 'Java TextIO Performance Test', - githubTriggerPhrase: 'Run Java TextIO Performance Test', - pipelineOptions : [ - bigQueryDataset : 'beam_performance', - bigQueryTable : 'textioit_results', - influxMeasurement : 'textioit_results', - numberOfRecords : '25000000', - expectedHash : 'f8453256ccf861e8a312c125dfe0e436', - datasetSize : '1062290000', - numWorkers : '5', - autoscalingAlgorithm: 'NONE' - ] - ], - [ - name : 'beam_PerformanceTests_Compressed_TextIOIT', - description : 'Runs performance tests for TextIOIT with GZIP compression', - test : 'org.apache.beam.sdk.io.text.TextIOIT', - githubTitle : 'Java CompressedTextIO Performance Test', - githubTriggerPhrase: 'Run Java CompressedTextIO Performance Test', - pipelineOptions : [ - bigQueryDataset : 'beam_performance', - bigQueryTable : 'compressed_textioit_results', - influxMeasurement : 'compressed_textioit_results', - numberOfRecords : '450000000', - expectedHash : '8a3de973354abc6fba621c6797cc0f06', - datasetSize : '1097840000', - compressionType : 'GZIP', - numWorkers : '5', - autoscalingAlgorithm: 'NONE' - ] - ], - [ - name : 'beam_PerformanceTests_ManyFiles_TextIOIT', - description : 'Runs performance tests for TextIOIT with many output files', - test : 'org.apache.beam.sdk.io.text.TextIOIT', - githubTitle : 'Java ManyFilesTextIO Performance Test', - githubTriggerPhrase: 'Run Java ManyFilesTextIO Performance Test', - pipelineOptions : [ - bigQueryDataset : 'beam_performance', - bigQueryTable : 'many_files_textioit_results', - influxMeasurement : 'many_files_textioit_results', - reportGcsPerformanceMetrics: 'true', - gcsPerformanceMetrics : 'true', - numberOfRecords : '25000000', - expectedHash : 'f8453256ccf861e8a312c125dfe0e436', - datasetSize : '1062290000', - numberOfShards : '1000', - numWorkers : '5', - autoscalingAlgorithm : 'NONE' - ] - - ], - [ - name : 'beam_PerformanceTests_AvroIOIT', - description : 'Runs performance tests for AvroIOIT', - test : 'org.apache.beam.sdk.io.avro.AvroIOIT', - githubTitle : 'Java AvroIO Performance Test', - githubTriggerPhrase: 'Run Java AvroIO Performance Test', - pipelineOptions : [ - numberOfRecords : '225000000', - expectedHash : '2f9f5ca33ea464b25109c0297eb6aecb', - datasetSize : '1089730000', - bigQueryDataset : 'beam_performance', - bigQueryTable : 'avroioit_results', - influxMeasurement : 'avroioit_results', - numWorkers : '5', - autoscalingAlgorithm: 'NONE' - ] - ], - [ - name : 'beam_PerformanceTests_TFRecordIOIT', - description : 'Runs performance tests for beam_PerformanceTests_TFRecordIOIT', - test : 'org.apache.beam.sdk.io.tfrecord.TFRecordIOIT', - githubTitle : 'Java TFRecordIO Performance Test', - githubTriggerPhrase: 'Run Java TFRecordIO Performance Test', - pipelineOptions : [ - bigQueryDataset : 'beam_performance', - bigQueryTable : 'tfrecordioit_results', - influxMeasurement : 'tfrecordioit_results', - numberOfRecords : '18000000', - expectedHash : '543104423f8b6eb097acb9f111c19fe4', - datasetSize : '1019380000', - numWorkers : '5', - autoscalingAlgorithm: 'NONE' - ] - ], - [ - name : 'beam_PerformanceTests_XmlIOIT', - description : 'Runs performance tests for beam_PerformanceTests_XmlIOIT', - test : 'org.apache.beam.sdk.io.xml.XmlIOIT', - githubTitle : 'Java XmlIOPerformance Test', - githubTriggerPhrase: 'Run Java XmlIO Performance Test', - pipelineOptions : [ - bigQueryDataset : 'beam_performance', - bigQueryTable : 'xmlioit_results', - influxMeasurement : 'xmlioit_results', - numberOfRecords : '12000000', - expectedHash : 'b3b717e7df8f4878301b20f314512fb3', - datasetSize : '1076590000', - charset : 'UTF-8', - numWorkers : '5', - autoscalingAlgorithm: 'NONE' - ] - ], - [ - name : 'beam_PerformanceTests_ParquetIOIT', - description : 'Runs performance tests for beam_PerformanceTests_ParquetIOIT', - test : 'org.apache.beam.sdk.io.parquet.ParquetIOIT', - githubTitle : 'Java ParquetIOPerformance Test', - githubTriggerPhrase: 'Run Java ParquetIO Performance Test', - pipelineOptions : [ - bigQueryDataset : 'beam_performance', - bigQueryTable : 'parquetioit_results', - influxMeasurement : 'parquetioit_results', - numberOfRecords : '225000000', - expectedHash : '2f9f5ca33ea464b25109c0297eb6aecb', - datasetSize : '1087370000', - numWorkers : '5', - autoscalingAlgorithm: 'NONE' - ] - ], - [ - name : 'beam_PerformanceTests_TextIOIT_HDFS', - description : 'Runs performance tests for TextIOIT on HDFS', - test : 'org.apache.beam.sdk.io.text.TextIOIT', - githubTitle : 'Java TextIO Performance Test on HDFS', - githubTriggerPhrase: 'Run Java TextIO Performance Test HDFS', - pipelineOptions : [ - bigQueryDataset : 'beam_performance', - bigQueryTable : 'textioit_hdfs_results', - influxMeasurement : 'textioit_hdfs_results', - numberOfRecords : '25000000', - expectedHash : 'f8453256ccf861e8a312c125dfe0e436', - datasetSize : '1062290000', - numWorkers : '5', - autoscalingAlgorithm: 'NONE' - ] - - ], - [ - name : 'beam_PerformanceTests_Compressed_TextIOIT_HDFS', - description : 'Runs performance tests for TextIOIT with GZIP compression on HDFS', - test : 'org.apache.beam.sdk.io.text.TextIOIT', - githubTitle : 'Java CompressedTextIO Performance Test on HDFS', - githubTriggerPhrase: 'Run Java CompressedTextIO Performance Test HDFS', - pipelineOptions : [ - bigQueryDataset : 'beam_performance', - bigQueryTable : 'compressed_textioit_hdfs_results', - influxMeasurement : 'compressed_textioit_hdfs_results', - numberOfRecords : '450000000', - expectedHash : '8a3de973354abc6fba621c6797cc0f06', - datasetSize : '1097840000', - compressionType : 'GZIP', - numWorkers : '5', - autoscalingAlgorithm: 'NONE' - ] - ], - [ - name : 'beam_PerformanceTests_ManyFiles_TextIOIT_HDFS', - description : 'Runs performance tests for TextIOIT with many output files on HDFS', - test : 'org.apache.beam.sdk.io.text.TextIOIT', - githubTitle : 'Java ManyFilesTextIO Performance Test on HDFS', - githubTriggerPhrase: 'Run Java ManyFilesTextIO Performance Test HDFS', - pipelineOptions : [ - bigQueryDataset : 'beam_performance', - bigQueryTable : 'many_files_textioit_hdfs_results', - influxMeasurement : 'many_files_textioit_hdfs_results', - reportGcsPerformanceMetrics: 'true', - gcsPerformanceMetrics : 'true', - numberOfRecords : '25000000', - expectedHash : 'f8453256ccf861e8a312c125dfe0e436', - datasetSize : '1062290000', - numberOfShards : '1000', - numWorkers : '5', - autoscalingAlgorithm : 'NONE' - ] - - ], - [ - name : 'beam_PerformanceTests_AvroIOIT_HDFS', - description : 'Runs performance tests for AvroIOIT on HDFS', - test : 'org.apache.beam.sdk.io.avro.AvroIOIT', - githubTitle : 'Java AvroIO Performance Test on HDFS', - githubTriggerPhrase: 'Run Java AvroIO Performance Test HDFS', - pipelineOptions : [ - bigQueryDataset : 'beam_performance', - bigQueryTable : 'avroioit_hdfs_results', - influxMeasurement : 'avroioit_hdfs_results', - numberOfRecords : '225000000', - expectedHash : '2f9f5ca33ea464b25109c0297eb6aecb', - datasetSize : '1089730000', - numWorkers : '5', - autoscalingAlgorithm: 'NONE' - ] - ], - [ - name : 'beam_PerformanceTests_TFRecordIOIT_HDFS', - description : 'Runs performance tests for beam_PerformanceTests_TFRecordIOIT on HDFS', - test : 'org.apache.beam.sdk.io.tfrecord.TFRecordIOIT', - githubTitle : 'Java TFRecordIO Performance Test on HDFS', - githubTriggerPhrase: 'Run Java TFRecordIO Performance Test HDFS', - pipelineOptions : [ - numberOfRecords : '18000000', - expectedHash : '543104423f8b6eb097acb9f111c19fe4', - datasetSize : '1019380000', - numWorkers : '5', - autoscalingAlgorithm: 'NONE' - ] - ], - [ - name : 'beam_PerformanceTests_XmlIOIT_HDFS', - description : 'Runs performance tests for beam_PerformanceTests_XmlIOIT on HDFS', - test : 'org.apache.beam.sdk.io.xml.XmlIOIT', - githubTitle : 'Java XmlIOPerformance Test on HDFS', - githubTriggerPhrase: 'Run Java XmlIO Performance Test HDFS', - pipelineOptions : [ - bigQueryDataset : 'beam_performance', - bigQueryTable : 'xmlioit_hdfs_results', - influxMeasurement : 'xmlioit_hdfs_results', - numberOfRecords : '12000000', - expectedHash : 'b3b717e7df8f4878301b20f314512fb3', - datasetSize : '1076590000', - charset : 'UTF-8', - numWorkers : '5', - autoscalingAlgorithm: 'NONE' - ] - ], - [ - name : 'beam_PerformanceTests_ParquetIOIT_HDFS', - description : 'Runs performance tests for beam_PerformanceTests_ParquetIOIT on HDFS', - test : 'org.apache.beam.sdk.io.parquet.ParquetIOIT', - githubTitle : 'Java ParquetIOPerformance Test on HDFS', - githubTriggerPhrase: 'Run Java ParquetIO Performance Test HDFS', - pipelineOptions : [ - bigQueryDataset : 'beam_performance', - bigQueryTable : 'parquetioit_hdfs_results', - influxMeasurement : 'parquetioit_hdfs_results', - numberOfRecords : '225000000', - expectedHash : '2f9f5ca33ea464b25109c0297eb6aecb', - datasetSize : '1087370000', - numWorkers : '5', - autoscalingAlgorithm: 'NONE' - ] - ] -] - -jobs.findAll { - it.name in [ - 'beam_PerformanceTests_TextIOIT', - 'beam_PerformanceTests_Compressed_TextIOIT', - 'beam_PerformanceTests_ManyFiles_TextIOIT', - 'beam_PerformanceTests_AvroIOIT', - 'beam_PerformanceTests_TFRecordIOIT', - 'beam_PerformanceTests_XmlIOIT', - 'beam_PerformanceTests_ParquetIOIT' - ] -}.forEach { testJob -> createGCSFileBasedIOITTestJob(testJob) } - -private void createGCSFileBasedIOITTestJob(testJob) { - job(testJob.name) { - description(testJob.description) - common.setTopLevelMainJobProperties(delegate) - common.enablePhraseTriggeringFromPullRequest(delegate, testJob.githubTitle, testJob.githubTriggerPhrase) - common.setAutoJob(delegate, 'H H/12 * * *') - InfluxDBCredentialsHelper.useCredentials(delegate) - additionalPipelineArgs = [ - influxDatabase: InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influxHost: InfluxDBCredentialsHelper.InfluxDBHostUrl, - ] - testJob.pipelineOptions.putAll(additionalPipelineArgs) - - def dataflowSpecificOptions = [ - runner : 'DataflowRunner', - project : 'apache-beam-testing', - tempRoot : 'gs://temp-storage-for-perf-tests', - filenamePrefix: "gs://temp-storage-for-perf-tests/${testJob.name}/\${BUILD_ID}/", - ] - - Map allPipelineOptions = dataflowSpecificOptions << testJob.pipelineOptions - String runner = "dataflow" - String filesystem = "gcs" - String testTask = ":sdks:java:io:file-based-io-tests:integrationTest" - - steps { - gradle { - rootBuildScriptDir(common.checkoutDir) - common.setGradleSwitches(delegate) - switches("--info") - switches("-DintegrationTestPipelineOptions=\'${common.joinPipelineOptions(allPipelineOptions)}\'") - switches("-Dfilesystem=\'${filesystem}\'") - switches("-DintegrationTestRunner=\'${runner}\'") - tasks("${testTask} --tests ${testJob.test}") - } - } - } -} - -jobs.findAll { - it.name in [ - 'beam_PerformanceTests_TextIOIT_HDFS', - 'beam_PerformanceTests_Compressed_TextIOIT_HDFS', - 'beam_PerformanceTests_ManyFiles_TextIOIT_HDFS', - // TODO(https://github.com/apache/beam/issues/18796) TFRecord performance test is failing only when running on hdfs. - // We need to fix this before enabling this job on jenkins. - //'beam_PerformanceTests_TFRecordIOIT_HDFS', - 'beam_PerformanceTests_AvroIOIT_HDFS', - 'beam_PerformanceTests_XmlIOIT_HDFS', - 'beam_PerformanceTests_ParquetIOIT_HDFS' - ] -}.forEach { testJob -> createHDFSFileBasedIOITTestJob(testJob) } - -private void createHDFSFileBasedIOITTestJob(testJob) { - job(testJob.name) { - description(testJob.description) - common.setTopLevelMainJobProperties(delegate) - common.enablePhraseTriggeringFromPullRequest(delegate, testJob.githubTitle, testJob.githubTriggerPhrase) - common.setAutoJob(delegate, 'H H/12 * * *') - InfluxDBCredentialsHelper.useCredentials(delegate) - additionalPipelineArgs = [ - influxDatabase: InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influxHost: InfluxDBCredentialsHelper.InfluxDBHostUrl, - ] - testJob.pipelineOptions.putAll(additionalPipelineArgs) - - String namespace = common.getKubernetesNamespace(testJob.name) - String kubeconfig = common.getKubeconfigLocationForNamespace(namespace) - Kubernetes k8s = Kubernetes.create(delegate, kubeconfig, namespace) - - k8s.apply(common.makePathAbsolute("src/.test-infra/kubernetes/hadoop/LargeITCluster/hdfs-multi-datanode-cluster.yml")) - String hostName = "LOAD_BALANCER_IP" - k8s.loadBalancerIP("hadoop", hostName) - - Map additionalOptions = [ - runner : 'DataflowRunner', - project : 'apache-beam-testing', - tempRoot : 'gs://temp-storage-for-perf-tests', - hdfsConfiguration: /[{\\\"fs.defaultFS\\\":\\\"hdfs:$${hostName}:9000\\\",\\\"dfs.replication\\\":1}]/, - filenamePrefix : "hdfs://\$${hostName}:9000/TEXTIO_IT_" - ] - - Map allPipelineOptions = testJob.pipelineOptions << additionalOptions - String runner = "dataflow" - String filesystem = "hdfs" - String testTask = ":sdks:java:io:file-based-io-tests:integrationTest" - - steps { - gradle { - rootBuildScriptDir(common.checkoutDir) - common.setGradleSwitches(delegate) - switches("--info") - switches("-DintegrationTestPipelineOptions=\'${common.joinPipelineOptions(allPipelineOptions)}\'") - switches("-Dfilesystem=\'${filesystem}\'") - switches("-DintegrationTestRunner=\'${runner}\'") - tasks("${testTask} --tests ${testJob.test}") - } - } - } -} diff --git a/.test-infra/jenkins/job_PerformanceTests_FileBasedIO_Python.groovy b/.test-infra/jenkins/job_PerformanceTests_FileBasedIO_Python.groovy deleted file mode 100644 index 21fef2313895..000000000000 --- a/.test-infra/jenkins/job_PerformanceTests_FileBasedIO_Python.groovy +++ /dev/null @@ -1,82 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonJobProperties as common -import LoadTestsBuilder as loadTestsBuilder -import InfluxDBCredentialsHelper - -def now = new Date().format("MMddHHmmss", TimeZone.getTimeZone('UTC')) - -def jobs = [ - [ - name : 'beam_PerformanceTests_TextIOIT_Python', - description : 'Runs performance tests for Python TextIOIT', - test : 'apache_beam.io.filebasedio_perf_test', - githubTitle : 'Python TextIO Performance Test', - githubTriggerPhrase: 'Run Python TextIO Performance Test', - pipelineOptions : [ - publish_to_big_query : true, - metrics_dataset : 'beam_performance', - metrics_table : 'python_textio_1GB_results', - influx_measurement : 'python_textio_1GB_results', - test_class : 'TextIOPerfTest', - input_options : '\'{' + - '"num_records": 25000000,' + - '"key_size": 9,' + - '"value_size": 21,' + - '"algorithm": "lcg"}\'', - dataset_size : '1050000000', - num_workers : '5', - autoscaling_algorithm: 'NONE' - ] - ] -] - -jobs.findAll { - it.name in [ - 'beam_PerformanceTests_TextIOIT_Python', - ] -}.forEach { testJob -> createGCSFileBasedIOITTestJob(testJob) } - -private void createGCSFileBasedIOITTestJob(testJob) { - job(testJob.name) { - description(testJob.description) - common.setTopLevelMainJobProperties(delegate) - common.enablePhraseTriggeringFromPullRequest(delegate, testJob.githubTitle, testJob.githubTriggerPhrase) - common.setAutoJob(delegate, 'H H * * *') - InfluxDBCredentialsHelper.useCredentials(delegate) - additionalPipelineArgs = [ - influx_db_name: InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influx_hostname: InfluxDBCredentialsHelper.InfluxDBHostUrl, - ] - testJob.pipelineOptions.putAll(additionalPipelineArgs) - - def dataflowSpecificOptions = [ - runner : 'DataflowRunner', - project : 'apache-beam-testing', - region : 'us-central1', - temp_location : 'gs://temp-storage-for-perf-tests/', - filename_prefix : "gs://temp-storage-for-perf-tests/${testJob.name}/\${BUILD_ID}/", - ] - - Map allPipelineOptions = dataflowSpecificOptions << testJob.pipelineOptions - - loadTestsBuilder.loadTest( - delegate, testJob.name, CommonTestProperties.Runner.DATAFLOW, CommonTestProperties.SDK.PYTHON, allPipelineOptions, testJob.test) - } -} diff --git a/.test-infra/jenkins/job_PerformanceTests_HadoopFormat.groovy b/.test-infra/jenkins/job_PerformanceTests_HadoopFormat.groovy deleted file mode 100644 index 3c267e4a2da7..000000000000 --- a/.test-infra/jenkins/job_PerformanceTests_HadoopFormat.groovy +++ /dev/null @@ -1,73 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonJobProperties as common -import Kubernetes -import InfluxDBCredentialsHelper - -String jobName = "beam_PerformanceTests_HadoopFormat" - -job(jobName) { - common.setTopLevelMainJobProperties(delegate) - common.setAutoJob(delegate, 'H H/12 * * *') - common.enablePhraseTriggeringFromPullRequest( - delegate, - 'Java HadoopFormatIO Performance Test', - 'Run Java HadoopFormatIO Performance Test') - InfluxDBCredentialsHelper.useCredentials(delegate) - - String namespace = common.getKubernetesNamespace(jobName) - String kubeconfig = common.getKubeconfigLocationForNamespace(namespace) - Kubernetes k8s = Kubernetes.create(delegate, kubeconfig, namespace) - - k8s.apply(common.makePathAbsolute("src/.test-infra/kubernetes/postgres/postgres-service-for-local-dev.yml")) - String postgresHostName = "LOAD_BALANCER_IP" - k8s.loadBalancerIP("postgres-for-dev", postgresHostName) - - Map pipelineOptions = [ - tempRoot : 'gs://temp-storage-for-perf-tests', - project : 'apache-beam-testing', - runner : 'DataflowRunner', - numberOfRecords : '5000000', - bigQueryDataset : 'beam_performance', - bigQueryTable : 'hadoopformatioit_results', - influxMeasurement : 'hadoopformatioit_results', - influxDatabase : InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influxHost : InfluxDBCredentialsHelper.InfluxDBHostUrl, - postgresUsername : 'postgres', - postgresPassword : 'uuinkks', - postgresDatabaseName : 'postgres', - postgresServerName : "\$${postgresHostName}", - postgresSsl : false, - postgresPort : '5432', - numWorkers : '5', - autoscalingAlgorithm : 'NONE' - ] - - steps { - gradle { - rootBuildScriptDir(common.checkoutDir) - common.setGradleSwitches(delegate) - switches("--info") - switches("-DintegrationTestPipelineOptions=\'${common.joinPipelineOptions(pipelineOptions)}\'") - switches("-DintegrationTestRunner=dataflow") - tasks(":sdks:java:io:hadoop-format:integrationTest --tests org.apache.beam.sdk.io.hadoop.format.HadoopFormatIOIT") - } - } -} - diff --git a/.test-infra/jenkins/job_PerformanceTests_JDBC.groovy b/.test-infra/jenkins/job_PerformanceTests_JDBC.groovy deleted file mode 100644 index d4885ae85105..000000000000 --- a/.test-infra/jenkins/job_PerformanceTests_JDBC.groovy +++ /dev/null @@ -1,73 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonJobProperties as common -import Kubernetes -import InfluxDBCredentialsHelper - -String jobName = "beam_PerformanceTests_JDBC" - -job(jobName) { - common.setTopLevelMainJobProperties(delegate) - common.setAutoJob(delegate, 'H H/12 * * *') - common.enablePhraseTriggeringFromPullRequest( - delegate, - 'Java JdbcIO Performance Test', - 'Run Java JdbcIO Performance Test') - InfluxDBCredentialsHelper.useCredentials(delegate) - - String namespace = common.getKubernetesNamespace(jobName) - String kubeconfig = common.getKubeconfigLocationForNamespace(namespace) - Kubernetes k8s = Kubernetes.create(delegate, kubeconfig, namespace) - - k8s.apply(common.makePathAbsolute("src/.test-infra/kubernetes/postgres/postgres-service-for-local-dev.yml")) - String postgresHostName = "LOAD_BALANCER_IP" - k8s.loadBalancerIP("postgres-for-dev", postgresHostName) - - Map pipelineOptions = [ - tempRoot : 'gs://temp-storage-for-perf-tests', - project : 'apache-beam-testing', - runner : 'DataflowRunner', - numberOfRecords : '5000000', - bigQueryDataset : 'beam_performance', - bigQueryTable : 'jdbcioit_results', - influxMeasurement : 'jdbcioit_results', - influxDatabase : InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influxHost : InfluxDBCredentialsHelper.InfluxDBHostUrl, - postgresUsername : 'postgres', - postgresPassword : 'uuinkks', - postgresDatabaseName : 'postgres', - postgresServerName : "\$${postgresHostName}", - postgresSsl : false, - postgresPort : '5432', - autoscalingAlgorithm : 'NONE', - numWorkers : '5' - ] - - steps { - gradle { - rootBuildScriptDir(common.checkoutDir) - common.setGradleSwitches(delegate) - switches("--info") - switches("-DintegrationTestPipelineOptions=\'${common.joinPipelineOptions(pipelineOptions)}\'") - switches("-DintegrationTestRunner=dataflow") - tasks(":sdks:java:io:jdbc:integrationTest --tests org.apache.beam.sdk.io.jdbc.JdbcIOIT") - } - } -} - diff --git a/.test-infra/jenkins/job_PerformanceTests_MongoDBIO_IT.groovy b/.test-infra/jenkins/job_PerformanceTests_MongoDBIO_IT.groovy deleted file mode 100644 index 73ef6c4a53ea..000000000000 --- a/.test-infra/jenkins/job_PerformanceTests_MongoDBIO_IT.groovy +++ /dev/null @@ -1,71 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonJobProperties as common -import Kubernetes -import InfluxDBCredentialsHelper - -String jobName = "beam_PerformanceTests_MongoDBIO_IT" - -job(jobName) { - common.setTopLevelMainJobProperties(delegate) - common.setAutoJob(delegate,'H H/12 * * *') - common.enablePhraseTriggeringFromPullRequest( - delegate, - 'Java MongoDBIO Performance Test', - 'Run Java MongoDBIO Performance Test') - InfluxDBCredentialsHelper.useCredentials(delegate) - - String namespace = common.getKubernetesNamespace(jobName) - String kubeconfigPath = common.getKubeconfigLocationForNamespace(namespace) - Kubernetes k8s = Kubernetes.create(delegate, kubeconfigPath, namespace) - - k8s.apply(common.makePathAbsolute("src/.test-infra/kubernetes/mongodb/load-balancer/mongo.yml")) - String mongoHostName = "LOAD_BALANCER_IP" - k8s.loadBalancerIP("mongo-load-balancer-service", mongoHostName) - - Map pipelineOptions = [ - tempRoot : 'gs://temp-storage-for-perf-tests', - project : 'apache-beam-testing', - numberOfRecords : '10000000', - bigQueryDataset : 'beam_performance', - bigQueryTable : 'mongodbioit_results', - influxMeasurement : 'mongodbioit_results', - influxDatabase : InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influxHost : InfluxDBCredentialsHelper.InfluxDBHostUrl, - mongoDBDatabaseName : 'beam', - mongoDBHostName : "\$${mongoHostName}", - mongoDBPort : 27017, - mongoDBUsername : 'root', - mongoDBPassword : 'uuinkkS', - runner : 'DataflowRunner', - autoscalingAlgorithm: 'NONE', - numWorkers : '5' - ] - - steps { - gradle { - rootBuildScriptDir(common.checkoutDir) - common.setGradleSwitches(delegate) - switches("--info") - switches("-DintegrationTestPipelineOptions=\'${common.joinPipelineOptions(pipelineOptions)}\'") - switches("-DintegrationTestRunner=dataflow") - tasks(":sdks:java:io:mongodb:integrationTest --tests org.apache.beam.sdk.io.mongodb.MongoDBIOIT") - } - } -} diff --git a/.test-infra/jenkins/job_PerformanceTests_PubsubIO_Python.groovy b/.test-infra/jenkins/job_PerformanceTests_PubsubIO_Python.groovy deleted file mode 100644 index 8ec84dc9b5e4..000000000000 --- a/.test-infra/jenkins/job_PerformanceTests_PubsubIO_Python.groovy +++ /dev/null @@ -1,72 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonJobProperties as commonJobProperties -import LoadTestsBuilder as loadTestsBuilder -import PhraseTriggeringPostCommitBuilder - -import static java.util.UUID.randomUUID - -def now = new Date().format("MMddHHmmss", TimeZone.getTimeZone('UTC')) - -def psio_test = [ - title : 'PubsubIO Write Performance Test Python 2GB', - test : 'apache_beam.io.gcp.pubsub_io_perf_test', - runner : CommonTestProperties.Runner.TEST_DATAFLOW, - pipelineOptions: [ - job_name : 'performance-tests-psio-python-2gb' + now, - project : 'apache-beam-testing', - region : 'us-central1', - temp_location : 'gs://temp-storage-for-perf-tests/loadtests', - publish_to_big_query : true, - metrics_dataset : 'beam_performance', - metrics_table : 'psio_io_2GB_results', - influx_measurement : 'python_psio_2GB_results', - influx_db_name : InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influx_hostname : InfluxDBCredentialsHelper.InfluxDBHostUrl, - input_options : '\'{' + - '"num_records": 2097152,' + - '"key_size": 1,' + - '"value_size": 1024,' + - '"algorithm": "lcg"}\'', - num_workers : 5, - autoscaling_algorithm : 'NONE', // Disable autoscale the worker pool. - pubsub_namespace_prefix : 'pubsub_io_performance_', - wait_until_finish_duration: 1000 * 60 * 12, // in milliseconds - ] -] - -def executeJob = { scope, testConfig -> - commonJobProperties.setTopLevelMainJobProperties(scope, 'master', 240) - - loadTestsBuilder.loadTest(scope, testConfig.title, testConfig.runner, - CommonTestProperties.SDK.PYTHON, testConfig.pipelineOptions, testConfig.test) -} - -PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_PerformanceTests_PubsubIOIT_Python_Streaming', - 'Run PubsubIO Performance Test Python', - 'PubsubIO Performance Test Python', - this - ) { - executeJob(delegate, psio_test) - } - -CronJobBuilder.cronJob('beam_PerformanceTests_PubsubIOIT_Python_Streaming', 'H H * * *', this) { - executeJob(delegate, psio_test) -} diff --git a/.test-infra/jenkins/job_PerformanceTests_Python.groovy b/.test-infra/jenkins/job_PerformanceTests_Python.groovy deleted file mode 100644 index 04c8fc999530..000000000000 --- a/.test-infra/jenkins/job_PerformanceTests_Python.groovy +++ /dev/null @@ -1,97 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonJobProperties as commonJobProperties -import InfluxDBCredentialsHelper - -def now = new Date().format("MMddHHmmss", TimeZone.getTimeZone('UTC')) - -// Common pipeline args for Dataflow job. -def dataflowPipelineArgs = [ - project : 'apache-beam-testing', - region : 'us-central1', - staging_location: 'gs://temp-storage-for-end-to-end-tests/staging-it', - temp_location : 'gs://temp-storage-for-end-to-end-tests/temp-it', -] - -testConfigurations = [] -pythonVersions = ['38'] - -for (pythonVersion in pythonVersions) { - testConfigurations.add([ - jobName : "beam_PerformanceTests_WordCountIT_Py${pythonVersion}", - jobDescription : "Python SDK Performance Test - Run WordCountIT in Py${pythonVersion} with 1Gb files", - jobTriggerPhrase : "Run Python${pythonVersion} WordCountIT Performance Test", - test : "apache_beam/examples/wordcount_it_test.py::WordCountIT::test_wordcount_it", - gradleTaskName : ":sdks:python:test-suites:dataflow:py${pythonVersion}:runPerformanceTest", - pipelineOptions : dataflowPipelineArgs + [ - job_name : "performance-tests-wordcount-python${pythonVersion}-batch-1gb${now}", - runner : 'TestDataflowRunner', - publish_to_big_query : true, - metrics_dataset : 'beam_performance', - metrics_table : "wordcount_py${pythonVersion}_pkb_results", - influx_measurement : "wordcount_py${pythonVersion}_results", - influx_db_name : InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influx_hostname : InfluxDBCredentialsHelper.InfluxDBHostUrl, - input : "gs://apache-beam-samples/input_small_files/ascii_sort_1MB_input.0000*", // 1Gb - output : "gs://temp-storage-for-end-to-end-tests/py-it-cloud/output", - expect_checksum : "ea0ca2e5ee4ea5f218790f28d0b9fe7d09d8d710", - num_workers : '10', - autoscaling_algorithm: "NONE", // Disable autoscale the worker pool. - ] - ]) -} - -for (testConfig in testConfigurations) { - createPythonPerformanceTestJob(testConfig) -} - -private void createPythonPerformanceTestJob(Map testConfig) { - // This job runs the Beam Python performance tests - job(testConfig.jobName) { - // Set default Beam job properties. - commonJobProperties.setTopLevelMainJobProperties(delegate) - - InfluxDBCredentialsHelper.useCredentials(delegate) - - // Run job in postcommit, don't trigger every push. - commonJobProperties.setAutoJob(delegate, 'H H * * *') - - // Allows triggering this build against pull requests. - commonJobProperties.enablePhraseTriggeringFromPullRequest( - delegate, - testConfig.jobDescription, - testConfig.jobTriggerPhrase, - ) - - publishers { - archiveJunit('**/pytest*.xml') - } - - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - switches("--info") - switches("-Ptest-pipeline-options=\"${commonJobProperties.mapToArgString(testConfig.pipelineOptions)}\"") - switches("-Ptest=${testConfig.test}") - tasks(testConfig.gradleTaskName) - commonJobProperties.setGradleSwitches(delegate) - } - } - } -} diff --git a/.test-infra/jenkins/job_PerformanceTests_SQLIO_Java.groovy b/.test-infra/jenkins/job_PerformanceTests_SQLIO_Java.groovy deleted file mode 100644 index ceded537bb4c..000000000000 --- a/.test-infra/jenkins/job_PerformanceTests_SQLIO_Java.groovy +++ /dev/null @@ -1,67 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -import CommonJobProperties as common - -def jobConfigs = [ - [ - title : 'SQL BigQueryIO with push-down Batch Performance Test Java', - triggerPhrase: 'Run SQLBigQueryIO Batch Performance Test Java', - name : 'beam_PerformanceTests_SQLBigQueryIO_Batch_Java', - previousName : 'beam_SQLBigQueryIO_Batch_Performance_Test_Java/', - itClass : 'org.apache.beam.sdk.extensions.sql.meta.provider.bigquery.BigQueryIOPushDownIT', - properties: [ - project : 'apache-beam-testing', - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - tempRoot : 'gs://temp-storage-for-perf-tests/loadtests', - metricsBigQueryDataset: 'beam_performance', - metricsBigQueryTable : 'sql_bqio_read_java_batch', - runner : "DataflowRunner", - maxNumWorkers : '5', - numWorkers : '5', - autoscalingAlgorithm : 'NONE', - ] - ] -] - -jobConfigs.forEach { jobConfig -> createPostCommitJob(jobConfig)} - -private void createPostCommitJob(jobConfig) { - job(jobConfig.name) { - description(jobConfig.description) - common.setTopLevelMainJobProperties(delegate) - common.enablePhraseTriggeringFromPullRequest(delegate, jobConfig.title, jobConfig.triggerPhrase) - common.setAutoJob(delegate, 'H H/12 * * *') - if (jobConfig.containsKey('previousName')) { - previousNames(jobConfig.previousName) - } - publishers { - archiveJunit('**/build/test-results/**/*.xml') - } - - steps { - gradle { - rootBuildScriptDir(common.checkoutDir) - common.setGradleSwitches(delegate) - switches("--info") - switches("-DintegrationTestPipelineOptions=\'${common.joinOptionsWithNestedJsonValues(jobConfig.properties)}\'") - switches("-DintegrationTestRunner=dataflow") - tasks(":sdks:java:extensions:sql:perf-tests:integrationTest --tests ${jobConfig.itClass}") - } - } - } -} diff --git a/.test-infra/jenkins/job_PerformanceTests_SingleStoreIO.groovy b/.test-infra/jenkins/job_PerformanceTests_SingleStoreIO.groovy deleted file mode 100644 index a5d54f03213f..000000000000 --- a/.test-infra/jenkins/job_PerformanceTests_SingleStoreIO.groovy +++ /dev/null @@ -1,86 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonJobProperties as common -import Kubernetes -import InfluxDBCredentialsHelper - -String jobName = "beam_PerformanceTests_SingleStoreIO" - -void waitForPodWithLabel(job, Kubernetes k8s, String label) { - job.steps { - shell("${k8s.KUBERNETES_DIR}/singlestore/wait-for-pod-with-label.sh ${label} 600") - } -} - -void waitFor(job, Kubernetes k8s, String resource) { - job.steps { - shell("${k8s.KUBERNETES_DIR}/singlestore/wait-for.sh ${resource} 600") - } -} - -job(jobName) { - common.setTopLevelMainJobProperties(delegate) - common.setAutoJob(delegate,'H H/12 * * *') - common.enablePhraseTriggeringFromPullRequest( - delegate, - 'Java SingleStoreIO Performance Test', - 'Run Java SingleStoreIO Performance Test') - InfluxDBCredentialsHelper.useCredentials(delegate) - - - String namespace = common.getKubernetesNamespace(jobName) - String kubeconfigPath = common.getKubeconfigLocationForNamespace(namespace) - Kubernetes k8s = Kubernetes.create(delegate, kubeconfigPath, namespace) - - k8s.apply(common.makePathAbsolute("src/.test-infra/kubernetes/singlestore/sdb-rbac.yaml")) - k8s.apply(common.makePathAbsolute("src/.test-infra/kubernetes/singlestore/sdb-cluster-crd.yaml")) - k8s.apply(common.makePathAbsolute("src/.test-infra/kubernetes/singlestore/sdb-operator.yaml")) - waitForPodWithLabel(delegate, k8s, "sdb-operator") - - k8s.apply(common.makePathAbsolute("src/.test-infra/kubernetes/singlestore/sdb-cluster.yaml")) - waitFor(delegate, k8s, "memsqlclusters.memsql.com") - - String singlestoreHostName = "LOAD_BALANCER_IP" - k8s.loadBalancerIP("svc-sdb-cluster-ddl", singlestoreHostName) - - Map pipelineOptions = [ - tempRoot : 'gs://temp-storage-for-perf-tests', - project : 'apache-beam-testing', - runner : 'DataflowRunner', - singleStoreServerName : "\$${singlestoreHostName}", - singleStoreUsername : "admin", - singleStorePassword : "secretpass", - singleStorePort: "3306", - numberOfRecords: "5000000", - influxMeasurement : 'singlestoreioit_results', - influxDatabase : InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influxHost : InfluxDBCredentialsHelper.InfluxDBHostUrl - ] - - steps { - gradle { - rootBuildScriptDir(common.checkoutDir) - common.setGradleSwitches(delegate) - switches("--info") - switches("-DintegrationTestPipelineOptions=\'${common.joinPipelineOptions(pipelineOptions)}\'") - switches("-DintegrationTestRunner=dataflow") - tasks(":sdks:java:io:singlestore:integrationTest --tests org.apache.beam.sdk.io.singlestore.SingleStoreIOPerformanceIT") - } - } -} diff --git a/.test-infra/jenkins/job_PerformanceTests_SpannerIO_Python.groovy b/.test-infra/jenkins/job_PerformanceTests_SpannerIO_Python.groovy deleted file mode 100644 index ed7afff43b79..000000000000 --- a/.test-infra/jenkins/job_PerformanceTests_SpannerIO_Python.groovy +++ /dev/null @@ -1,112 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonJobProperties as commonJobProperties -import LoadTestsBuilder as loadTestsBuilder -import PhraseTriggeringPostCommitBuilder -import InfluxDBCredentialsHelper - -def now = new Date().format("MMddHHmmss", TimeZone.getTimeZone('UTC')) - -def spannerio_read_test_2gb = [ - title : 'SpannerIO Read Performance Test Python 2 GB', - test : 'apache_beam.io.gcp.experimental.spannerio_read_perf_test', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - job_name : 'performance-tests-spanner-read-python-2gb' + now, - project : 'apache-beam-testing', - // Run in us-west1 to colocate with beam-test spanner instance (BEAM-13222) - region : 'us-west1', - temp_location : 'gs://temp-storage-for-perf-tests/loadtests', - spanner_instance : 'beam-test', - spanner_database : 'pyspanner_read_2gb', - publish_to_big_query : true, - metrics_dataset : 'beam_performance', - metrics_table : 'pyspanner_read_2GB_results', - influx_measurement : 'python_spannerio_read_2GB_results', - influx_db_name : InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influx_hostname : InfluxDBCredentialsHelper.InfluxDBHostUrl, - input_options : '\'{' + - '"num_records": 2097152,' + - '"key_size": 1,' + - '"value_size": 1024,' + - '"algorithm": "lcg"}\'', - num_workers : 5, - autoscaling_algorithm: 'NONE', // Disable autoscale the worker pool. - ] -] - -def spannerio_write_test_2gb = [ - title : 'SpannerIO Write Performance Test Python Batch 2 GB', - test : 'apache_beam.io.gcp.experimental.spannerio_write_perf_test', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - job_name : 'performance-tests-spannerio-write-python-batch-2gb' + now, - project : 'apache-beam-testing', - // Run in us-west1 to colocate with beam-test spanner instance (BEAM-13222) - region : 'us-west1', - temp_location : 'gs://temp-storage-for-perf-tests/loadtests', - spanner_instance : 'beam-test', - spanner_database : 'pyspanner_write_2gb', - publish_to_big_query : true, - metrics_dataset : 'beam_performance', - metrics_table : 'pyspanner_write_2GB_results', - influx_measurement : 'python_spanner_write_2GB_results', - influx_db_name : InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influx_hostname : InfluxDBCredentialsHelper.InfluxDBHostUrl, - input_options : '\'{' + - '"num_records": 2097152,' + - '"key_size": 1,' + - '"value_size": 1024,' + - '"algorithm": "lcg"}\'', - num_workers : 5, - autoscaling_algorithm: 'NONE', // Disable autoscale the worker pool. - ] -] - -def executeJob = { scope, testConfig -> - commonJobProperties.setTopLevelMainJobProperties(scope, 'master', 480) - - loadTestsBuilder.loadTest(scope, testConfig.title, testConfig.runner, CommonTestProperties.SDK.PYTHON, testConfig.pipelineOptions, testConfig.test) -} - -PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_PerformanceTests_SpannerIO_Read_2GB_Python', - 'Run SpannerIO Read 2GB Performance Test Python', - 'SpannerIO Read 2GB Performance Test Python', - this - ) { - executeJob(delegate, spannerio_read_test_2gb) - } - -CronJobBuilder.cronJob('beam_PerformanceTests_SpannerIO_Read_2GB_Python', 'H H * * *', this) { - executeJob(delegate, spannerio_read_test_2gb) -} - -PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch', - 'Run SpannerIO Write 2GB Performance Test Python Batch', - 'SpannerIO Write 2GB Performance Test Python Batch', - this - ) { - executeJob(delegate, spannerio_write_test_2gb) - } - -CronJobBuilder.cronJob('beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch', 'H H * * *', this) { - executeJob(delegate, spannerio_write_test_2gb) -} diff --git a/.test-infra/jenkins/job_PerformanceTests_SparkReceiverIO_IT.groovy b/.test-infra/jenkins/job_PerformanceTests_SparkReceiverIO_IT.groovy deleted file mode 100644 index 6a2328749f05..000000000000 --- a/.test-infra/jenkins/job_PerformanceTests_SparkReceiverIO_IT.groovy +++ /dev/null @@ -1,84 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonJobProperties as common -import Kubernetes -import InfluxDBCredentialsHelper - -String jobName = "beam_PerformanceTests_SparkReceiver_IO" - -/** - * This job runs the SparkReceiver IO performance tests. - It runs on a RabbitMQ cluster that is build by applying the folder .test-infra/kubernetes/rabbit, - in an existing kubernetes cluster (DEFAULT_CLUSTER in Kubernetes.groovy). - The services created to run this test are: - Pods: 1 RabbitMq pods. - Services: 1 broker - When the performance tests finish all resources are cleaned up by a postBuild step in Kubernetes.groovy - */ -job(jobName) { - common.setTopLevelMainJobProperties(delegate, 'master', 120) - common.setAutoJob(delegate, 'H H/12 * * *') - common.enablePhraseTriggeringFromPullRequest( - delegate, - 'Java SparkReceiverIO Performance Test', - 'Run Java SparkReceiverIO Performance Test') - InfluxDBCredentialsHelper.useCredentials(delegate) - - String namespace = common.getKubernetesNamespace(jobName) - String kubeconfig = common.getKubeconfigLocationForNamespace(namespace) - Kubernetes k8s = Kubernetes.create(delegate, kubeconfig, namespace) - - k8s.apply(common.makePathAbsolute("src/.test-infra/kubernetes/rabbit/rabbitmq.yaml")) - String rabbitMqHostName = "LOAD_BALANCER_IP" - k8s.loadBalancerIP("rabbitmq", rabbitMqHostName) - - Map pipelineOptions = [ - tempRoot : 'gs://temp-storage-for-perf-tests', - project : 'apache-beam-testing', - runner : 'DataflowRunner', - sourceOptions : """ - { - "numRecords": "5000000", - "keySizeBytes": "1", - "valueSizeBytes": "90" - } - """.trim().replaceAll("\\s", ""), - bigQueryDataset : 'beam_performance', - bigQueryTable : 'sparkreceiverioit_results', - influxMeasurement : 'sparkreceiverioit_results', - influxDatabase : InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influxHost : InfluxDBCredentialsHelper.InfluxDBHostUrl, - rabbitMqBootstrapServerAddress: "amqp://guest:guest@\$${rabbitMqHostName}:5672", - streamName : 'rabbitMqTestStream', - readTimeout : '1800', - numWorkers : '1', - autoscalingAlgorithm : 'NONE' - ] - - steps { - gradle { - rootBuildScriptDir(common.checkoutDir) - common.setGradleSwitches(delegate) - switches("--info") - switches("-DintegrationTestPipelineOptions=\'${common.joinOptionsWithNestedJsonValues(pipelineOptions)}\'") - switches("-DintegrationTestRunner=dataflow") - tasks(":sdks:java:io:sparkreceiver:2:integrationTest --tests org.apache.beam.sdk.io.sparkreceiver.SparkReceiverIOIT") - } - } -} From 8a827e8c0182bddbb944161df2aa6fd5308c82c8 Mon Sep 17 00:00:00 2001 From: Vlado Djerek <rage.bubblegum@gmail.com> Date: Tue, 24 Oct 2023 21:54:45 +0200 Subject: [PATCH 246/435] auth fix (#29128) --- .../beam_PerformanceTests_xlang_KafkaIO_Python.yml | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/.github/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml b/.github/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml index 92787670d93c..2818b710a62b 100644 --- a/.github/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml +++ b/.github/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml @@ -73,6 +73,12 @@ jobs: uses: ./.github/actions/setup-environment-action with: python-version: 3.8 + - name: Authenticate on GCP + id: auth + uses: google-github-actions/auth@v1 + with: + credentials_json: ${{ secrets.GCP_SA_KEY }} + project_id: ${{ secrets.GCP_PROJECT_ID }} - name: Set k8s access uses: ./.github/actions/setup-k8s-access with: From 48722f1bfb4b0e81d29c8ba595944a3828af877f Mon Sep 17 00:00:00 2001 From: Hai Joey Tran <joey.tran@schrodinger.com> Date: Tue, 24 Oct 2023 19:54:16 -0400 Subject: [PATCH 247/435] [PYTHON] Add new `--auto_unique_labels` option to StandardOptions (#28984) --- .../apache_beam/options/pipeline_options.py | 8 +++++ sdks/python/apache_beam/pipeline.py | 33 +++++++++++++++---- sdks/python/apache_beam/pipeline_test.py | 27 +++++++++++++++ 3 files changed, 62 insertions(+), 6 deletions(-) diff --git a/sdks/python/apache_beam/options/pipeline_options.py b/sdks/python/apache_beam/options/pipeline_options.py index 3fbf7eff7dd6..76b776779cf9 100644 --- a/sdks/python/apache_beam/options/pipeline_options.py +++ b/sdks/python/apache_beam/options/pipeline_options.py @@ -515,6 +515,14 @@ def _add_argparse_args(cls, parser): 'at transform level. Interpretation of hints is defined by ' 'Beam runners.')) + parser.add_argument( + '--auto_unique_labels', + default=False, + action='store_true', + help='Whether to automatically generate unique transform labels ' + 'for every transform. The default behavior is to raise an ' + 'exception if a transform is created with a non-unique label.') + class CrossLanguageOptions(PipelineOptions): @classmethod diff --git a/sdks/python/apache_beam/pipeline.py b/sdks/python/apache_beam/pipeline.py index f52616307e7b..ed0736250d1f 100644 --- a/sdks/python/apache_beam/pipeline.py +++ b/sdks/python/apache_beam/pipeline.py @@ -54,6 +54,7 @@ import shutil import tempfile import unicodedata +import uuid from collections import defaultdict from typing import TYPE_CHECKING from typing import Any @@ -681,13 +682,20 @@ def apply( alter_label_if_ipython(transform, pvalueish) full_label = '/'.join( - [self._current_transform().full_label, label or - transform.label]).lstrip('/') + [self._current_transform().full_label, transform.label]).lstrip('/') if full_label in self.applied_labels: - raise RuntimeError( - 'A transform with label "%s" already exists in the pipeline. ' - 'To apply a transform with a specified label write ' - 'pvalue | "label" >> transform' % full_label) + auto_unique_labels = self._options.view_as( + StandardOptions).auto_unique_labels + if auto_unique_labels: + # If auto_unique_labels is set, we will append a unique suffix to the + # label to make it unique. + unique_label = self._generate_unique_label(transform) + return self.apply(transform, pvalueish, unique_label) + else: + raise RuntimeError( + 'A transform with label "%s" already exists in the pipeline. ' + 'To apply a transform with a specified label write ' + 'pvalue | "label" >> transform' % full_label) self.applied_labels.add(full_label) pvalueish, inputs = transform._extract_input_pvalues(pvalueish) @@ -763,6 +771,19 @@ def apply( self.transforms_stack.pop() return pvalueish_result + def _generate_unique_label( + self, + transform # type: str + ): + # type: (...) -> str + + """ + Given a transform, generate a unique label for it based on current label. + """ + unique_suffix = uuid.uuid4().hex[:6] + return '%s_%s' % (transform.label, unique_suffix) + + def _infer_result_type( self, transform, # type: ptransform.PTransform diff --git a/sdks/python/apache_beam/pipeline_test.py b/sdks/python/apache_beam/pipeline_test.py index c9ac4ce4c13d..113d1a99990c 100644 --- a/sdks/python/apache_beam/pipeline_test.py +++ b/sdks/python/apache_beam/pipeline_test.py @@ -22,6 +22,7 @@ import copy import platform import unittest +import uuid import mock import pytest @@ -266,6 +267,32 @@ def test_reuse_custom_transform_instance(self): 'pipeline. To apply a transform with a specified label write ' 'pvalue | "label" >> transform') + def test_auto_unique_labels(self): + + opts = PipelineOptions(["--auto_unique_labels"]) + with mock.patch.object(uuid, 'uuid4') as mock_uuid_gen: + mock_uuids = [mock.Mock(hex='UUID01XXX'), mock.Mock(hex='UUID02XXX')] + mock_uuid_gen.side_effect = mock_uuids + with TestPipeline(options=opts) as pipeline: + pcoll = pipeline | 'pcoll' >> Create([1, 2, 3]) + + def identity(x): + return x + + pcoll2 = pcoll | Map(identity) + pcoll3 = pcoll2 | Map(identity) + pcoll4 = pcoll3 | Map(identity) + assert_that(pcoll4, equal_to([1, 2, 3])) + + map_id_full_labels = { + label + for label in pipeline.applied_labels if "Map(identity)" in label + } + map_id_leaf_labels = {label.split(":")[-1] for label in map_id_full_labels} + # Only the first 6 chars of the UUID hex should be used + assert map_id_leaf_labels == set( + ["Map(identity)", "Map(identity)_UUID01", "Map(identity)_UUID02"]) + def test_reuse_cloned_custom_transform_instance(self): with TestPipeline() as pipeline: pcoll1 = pipeline | 'pc1' >> Create([1, 2, 3]) From 972a0fbb10f1eaeae433cd1db4202e1e81588ed5 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw <robertwb@gmail.com> Date: Tue, 24 Oct 2023 17:38:06 -0700 Subject: [PATCH 248/435] [YAML] Guard javascript UDFs with experimental feature enablement. --- sdks/python/apache_beam/transforms/core.py | 8 +++++ sdks/python/apache_beam/yaml/options.py | 36 +++++++++++++++++++ sdks/python/apache_beam/yaml/yaml_mapping.py | 7 ++++ sdks/python/apache_beam/yaml/yaml_udf_test.py | 12 ++++--- 4 files changed, 59 insertions(+), 4 deletions(-) create mode 100644 sdks/python/apache_beam/yaml/options.py diff --git a/sdks/python/apache_beam/transforms/core.py b/sdks/python/apache_beam/transforms/core.py index e980dccea744..6bf0e187ce91 100644 --- a/sdks/python/apache_beam/transforms/core.py +++ b/sdks/python/apache_beam/transforms/core.py @@ -2258,6 +2258,10 @@ def __init__(self, pcoll, exception_handling_args, upstream_errors=()): self._exception_handling_args = exception_handling_args self._upstream_errors = upstream_errors + @property + def pipeline(self): + return self._pvalue.pipeline + @property def element_type(self): return self._pcoll.element_type @@ -2324,6 +2328,10 @@ def __init__(self, pvalue, exception_handling_args=None): else: self._pvalue = _PValueWithErrors(pvalue, exception_handling_args) + @property + def pipeline(self): + return self._pvalue.pipeline + @property def element_type(self): return self._pvalue.element_type diff --git a/sdks/python/apache_beam/yaml/options.py b/sdks/python/apache_beam/yaml/options.py new file mode 100644 index 000000000000..e80141c40b1d --- /dev/null +++ b/sdks/python/apache_beam/yaml/options.py @@ -0,0 +1,36 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from apache_beam.options import pipeline_options + + +class YamlOptions(pipeline_options.PipelineOptions): + @classmethod + def _add_argparse_args(cls, parser): + parser.add_argument( + '--yaml_experimental_features', + dest='yaml_experimental_features', + action='append', + default=[], + help=('Enable yaml features ahead of them being declared stable.')) + + @classmethod + def check_enabled(cls, pipeline, feature, description=None): + if feature not in pipeline._options.view_as(cls).yaml_experimental_features: + raise ValueError( + f'{description or feature} unsupported because ' + f'{feature} is not set in --yaml_experimental_features option.') diff --git a/sdks/python/apache_beam/yaml/yaml_mapping.py b/sdks/python/apache_beam/yaml/yaml_mapping.py index 889f7f1ee309..62bc3a7ae56d 100644 --- a/sdks/python/apache_beam/yaml/yaml_mapping.py +++ b/sdks/python/apache_beam/yaml/yaml_mapping.py @@ -34,6 +34,7 @@ from apache_beam.typehints import trivial_inference from apache_beam.typehints.schemas import named_fields_from_element_type from apache_beam.utils import python_callable +from apache_beam.yaml import options from apache_beam.yaml import yaml_provider @@ -257,6 +258,9 @@ def with_exception_handling(self, **kwargs): @maybe_with_exception_handling_transform_fn def _PyJsFilter( pcoll, keep: Union[str, Dict[str, str]], language: Optional[str] = None): + if language == 'javascript': + options.YamlOptions.check_enabled(pcoll.pipeline, 'javascript') + try: input_schema = dict(named_fields_from_element_type(pcoll.element_type)) except (TypeError, ValueError) as exn: @@ -327,6 +331,9 @@ def normalize_fields(pcoll, fields, drop=(), append=False, language='generic'): def _PyJsMapToFields(pcoll, language='generic', **mapping_args): input_schema, fields = normalize_fields( pcoll, language=language, **mapping_args) + if language == 'javascript': + options.YamlOptions.check_enabled(pcoll.pipeline, 'javascript') + original_fields = list(input_schema.keys()) return pcoll | beam.Select( diff --git a/sdks/python/apache_beam/yaml/yaml_udf_test.py b/sdks/python/apache_beam/yaml/yaml_udf_test.py index 42bdf6e0bd5b..04a4a3bac963 100644 --- a/sdks/python/apache_beam/yaml/yaml_udf_test.py +++ b/sdks/python/apache_beam/yaml/yaml_udf_test.py @@ -50,7 +50,8 @@ def tearDown(self): def test_map_to_fields_filter_inline_js(self): with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions( - pickle_library='cloudpickle')) as p: + pickle_library='cloudpickle', yaml_experimental_features=['javascript' + ])) as p: elements = p | beam.Create(self.data) result = elements | YamlTransform( ''' @@ -96,7 +97,8 @@ def test_map_to_fields_filter_inline_py(self): def test_filter_inline_js(self): with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions( - pickle_library='cloudpickle')) as p: + pickle_library='cloudpickle', yaml_experimental_features=['javascript' + ])) as p: elements = p | beam.Create(self.data) result = elements | YamlTransform( ''' @@ -134,7 +136,8 @@ def test_filter_inline_py(self): def test_filter_expression_js(self): with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions( - pickle_library='cloudpickle')) as p: + pickle_library='cloudpickle', yaml_experimental_features=['javascript' + ])) as p: elements = p | beam.Create(self.data) result = elements | YamlTransform( ''' @@ -183,7 +186,8 @@ def test_filter_inline_js_file(self): self.fs.create(path).write(data.encode('utf8')) with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions( - pickle_library='cloudpickle')) as p: + pickle_library='cloudpickle', yaml_experimental_features=['javascript' + ])) as p: elements = p | beam.Create(self.data) result = elements | YamlTransform( f''' From 8b31859ff8ac17bf0ff24d368d4df014e8f72c11 Mon Sep 17 00:00:00 2001 From: Anand Inguva <34158215+AnandInguva@users.noreply.github.com> Date: Wed, 25 Oct 2023 01:16:42 +0000 Subject: [PATCH 249/435] [Python]Remove get_artifacts in MLTranform since artifacts are stored in artifact location (#29016) * Remove get_artifacts for MLTransform * Doc changes * Add snippet tests to MLTransform test suite * Fix tests * fix test values * Update CHANGES.md * add word count to file instead of dict * remove get_artifacts method * Remove redundant code * Change return type * Update tft.py --- CHANGES.md | 1 + .../elementwise/mltransform_test.py | 8 +- sdks/python/apache_beam/ml/transforms/base.py | 13 -- .../ml/transforms/handlers_test.py | 18 --- sdks/python/apache_beam/ml/transforms/tft.py | 82 ++----------- .../apache_beam/ml/transforms/tft_test.py | 112 ++++++------------ sdks/python/tox.ini | 9 +- .../en/documentation/ml/preprocess-data.md | 5 +- 8 files changed, 59 insertions(+), 189 deletions(-) diff --git a/CHANGES.md b/CHANGES.md index 6d67ccdcf95d..0eee0a350ae6 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -86,6 +86,7 @@ as a workaround, a copy of "old" `CountingSource` class should be placed into a * Fixed "Desired bundle size 0 bytes must be greater than 0" in Java SDK's BigtableIO.BigtableSource when you have more cores than bytes to read (Java) [#28793](https://github.com/apache/beam/issues/28793). * `watch_file_pattern` arg of the [RunInference](https://github.com/apache/beam/blob/104c10b3ee536a9a3ea52b4dbf62d86b669da5d9/sdks/python/apache_beam/ml/inference/base.py#L997) arg had no effect prior to 2.52.0. To use the behavior of arg `watch_file_pattern` prior to 2.52.0, follow the documentation at https://beam.apache.org/documentation/ml/side-input-updates/ and use `WatchFilePattern` PTransform as a SideInput. ([#28948](https://github.com/apache/beam/pulls/28948)) +* `MLTransform` doesn't output artifacts such as min, max and quantiles. Instead, `MLTransform` will add a feature to output these artifacts as human readable format - [#29017](https://github.com/apache/beam/issues/29017). For now, to use the artifacts such as min and max that were produced by the eariler `MLTransform`, use `read_artifact_location` of `MLTransform`, which reads artifacts that were produced earlier in a different `MLTransform` ([#29016](https://github.com/apache/beam/pull/29016/)) ## Security Fixes * Fixed [CVE-YYYY-NNNN](https://www.cve.org/CVERecord?id=CVE-YYYY-NNNN) (Java/Python/Go) ([#X](https://github.com/apache/beam/issues/X)). diff --git a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/mltransform_test.py b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/mltransform_test.py index 1d2197e35e4e..0db10718295b 100644 --- a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/mltransform_test.py +++ b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/mltransform_test.py @@ -31,7 +31,7 @@ import tensorflow_transform as tft # pylint: disable=unused-import from apache_beam.examples.snippets.transforms.elementwise.mltransform import mltransform_scale_to_0_1 from apache_beam.examples.snippets.transforms.elementwise.mltransform import mltransform_compute_and_apply_vocabulary - from apache_beam.examples.snippets.transforms.elementwise.mltransform import mltransform_compute_and_apply_vocabulary_with_non_columnar_data + from apache_beam.examples.snippets.transforms.elementwise.mltransform import mltransform_compute_and_apply_vocabulary_with_scalar except ImportError: raise unittest.SkipTest('tensorflow_transform is not installed.') @@ -46,8 +46,8 @@ def check_mltransform_compute_and_apply_vocab(): def check_mltransform_scale_to_0_1(): expected = '''[START mltransform_scale_to_0_1] -Row(x=array([0. , 0.5714286, 0.2857143], dtype=float32), x_max=array([8.], dtype=float32), x_min=array([1.], dtype=float32)) -Row(x=array([0.42857143, 0.14285715, 1. ], dtype=float32), x_max=array([8.], dtype=float32), x_min=array([1.], dtype=float32)) +Row(x=array([0. , 0.5714286, 0.2857143], dtype=float32)) +Row(x=array([0.42857143, 0.14285715, 1. ], dtype=float32)) [END mltransform_scale_to_0_1] '''.splitlines()[1:-1] return expected @@ -80,7 +80,7 @@ def test_mltransform_scale_to_0_1(self, mock_stdout): self.assertEqual(predicted, expected) def test_mltransform_compute_and_apply_vocab_scalar(self, mock_stdout): - mltransform_compute_and_apply_vocabulary_with_non_columnar_data() + mltransform_compute_and_apply_vocabulary_with_scalar() predicted = mock_stdout.getvalue().splitlines() expected = check_mltransform_compute_and_apply_vocabulary_with_scalar() self.assertEqual(predicted, expected) diff --git a/sdks/python/apache_beam/ml/transforms/base.py b/sdks/python/apache_beam/ml/transforms/base.py index a0bc4a906100..b3a30bb5f125 100644 --- a/sdks/python/apache_beam/ml/transforms/base.py +++ b/sdks/python/apache_beam/ml/transforms/base.py @@ -67,16 +67,6 @@ def apply_transform(self, data: OperationInputT, inputs: input data. """ - @abc.abstractmethod - def get_artifacts( - self, data: OperationInputT, - output_column_prefix: str) -> Optional[Dict[str, OperationOutputT]]: - """ - If the operation generates any artifacts, they can be returned from this - method. - """ - pass - def __call__(self, data: OperationInputT, output_column_name: str) -> Dict[str, OperationOutputT]: """ @@ -84,9 +74,6 @@ def __call__(self, data: OperationInputT, This method will invoke the apply() method of the class. """ transformed_data = self.apply_transform(data, output_column_name) - artifacts = self.get_artifacts(data, output_column_name) - if artifacts: - transformed_data = {**transformed_data, **artifacts} return transformed_data def get_counter(self): diff --git a/sdks/python/apache_beam/ml/transforms/handlers_test.py b/sdks/python/apache_beam/ml/transforms/handlers_test.py index 3342ec76cae5..327c8c76c0e9 100644 --- a/sdks/python/apache_beam/ml/transforms/handlers_test.py +++ b/sdks/python/apache_beam/ml/transforms/handlers_test.py @@ -58,14 +58,6 @@ def apply_transform(self, inputs, output_column_name, **kwargs): return {output_column_name: inputs * 10} -class _FakeOperationWithArtifacts(TFTOperation): - def apply_transform(self, inputs, output_column_name, **kwargs): - return {output_column_name: inputs} - - def get_artifacts(self, data, col_name): - return {'artifact': tf.convert_to_tensor([1])} - - class IntType(NamedTuple): x: int @@ -106,16 +98,6 @@ def test_tft_operation_preprocessing_fn( actual_result = process_handler.process_data_fn(inputs) self.assertDictEqual(actual_result, expected_result) - def test_preprocessing_fn_with_artifacts(self): - process_handler = handlers.TFTProcessHandler( - transforms=[_FakeOperationWithArtifacts(columns=['x'])], - artifact_location=self.artifact_location) - inputs = {'x': [1, 2, 3]} - preprocessing_fn = process_handler.process_data_fn - actual_result = preprocessing_fn(inputs) - expected_result = {'x': [1, 2, 3], 'artifact': tf.convert_to_tensor([1])} - self.assertDictEqual(actual_result, expected_result) - def test_input_type_from_schema_named_tuple_pcoll(self): data = [{'x': 1}] with beam.Pipeline() as p: diff --git a/sdks/python/apache_beam/ml/transforms/tft.py b/sdks/python/apache_beam/ml/transforms/tft.py index 1d492642cd60..c7b8ff015324 100644 --- a/sdks/python/apache_beam/ml/transforms/tft.py +++ b/sdks/python/apache_beam/ml/transforms/tft.py @@ -45,9 +45,7 @@ import tensorflow as tf import tensorflow_transform as tft from apache_beam.ml.transforms.base import BaseOperation -from tensorflow_transform import analyzers from tensorflow_transform import common_types -from tensorflow_transform import tf_utils __all__ = [ 'ComputeAndApplyVocabulary', @@ -77,6 +75,8 @@ def wrapper(fn): return wrapper +# TODO: https://github.com/apache/beam/pull/29016 +# Add support for outputting artifacts to a text file in human readable form. class TFTOperation(BaseOperation[common_types.TensorType, common_types.TensorType]): def __init__(self, columns: List[str]) -> None: @@ -95,13 +95,6 @@ def __init__(self, columns: List[str]) -> None: "Columns are not specified. Please specify the column for the " " op %s" % self.__class__.__name__) - def get_artifacts(self, data: common_types.TensorType, - col_name: str) -> Dict[str, common_types.TensorType]: - """ - Returns the artifacts generated by the operation. - """ - return {} - @tf.function def _split_string_with_delimiter(self, data, delimiter): """ @@ -240,15 +233,6 @@ def apply_transform( } return output_dict - def get_artifacts(self, data: common_types.TensorType, - col_name: str) -> Dict[str, common_types.TensorType]: - mean_var = tft.analyzers._mean_and_var(data) - shape = [tf.shape(data)[0], 1] - return { - col_name + '_mean': tf.broadcast_to(mean_var[0], shape), - col_name + '_var': tf.broadcast_to(mean_var[1], shape), - } - @register_input_dtype(float) class ScaleTo01(TFTOperation): @@ -280,14 +264,6 @@ def __init__( self.elementwise = elementwise self.name = name - def get_artifacts(self, data: common_types.TensorType, - col_name: str) -> Dict[str, common_types.TensorType]: - shape = [tf.shape(data)[0], 1] - return { - col_name + '_min': tf.broadcast_to(tft.min(data), shape), - col_name + '_max': tf.broadcast_to(tft.max(data), shape) - } - def apply_transform( self, data: common_types.TensorType, output_column_name: str) -> Dict[str, common_types.TensorType]: @@ -368,34 +344,6 @@ def __init__( self.elementwise = elementwise self.name = name - def get_artifacts(self, data: common_types.TensorType, - col_name: str) -> Dict[str, common_types.TensorType]: - num_buckets = self.num_buckets - epsilon = self.epsilon - elementwise = self.elementwise - - if num_buckets < 1: - raise ValueError('Invalid num_buckets %d' % num_buckets) - - if isinstance(data, (tf.SparseTensor, tf.RaggedTensor)) and elementwise: - raise ValueError( - 'bucketize requires `x` to be dense if `elementwise=True`') - - x_values = tf_utils.get_values(data) - - if epsilon is None: - # See explanation in args documentation for epsilon. - epsilon = min(1.0 / num_buckets, 0.01) - - quantiles = analyzers.quantiles( - x_values, num_buckets, epsilon, reduce_instance_dims=not elementwise) - shape = [ - tf.shape(data)[0], num_buckets - 1 if num_buckets > 1 else num_buckets - ] - # These quantiles are used as the bucket boundaries in the later stages. - # Should we change the prefix _quantiles to _bucket_boundaries? - return {col_name + '_quantiles': tf.broadcast_to(quantiles, shape)} - def apply_transform( self, data: common_types.TensorType, output_column_name: str) -> Dict[str, common_types.TensorType]: @@ -572,6 +520,7 @@ def __init__( ngram_range: Tuple[int, int] = (1, 1), ngrams_separator: Optional[str] = None, compute_word_count: bool = False, + key_vocab_filename: str = 'key_vocab_mapping', name: Optional[str] = None, ): """ @@ -592,9 +541,9 @@ def __init__( n-gram sizes. seperator: A string that will be inserted between each ngram. compute_word_count: A boolean that specifies whether to compute - the unique word count and add it as an artifact to the output. - Note that the count will be computed over the entire dataset so - it will be the same value for all inputs. + the unique word count over the entire dataset. Defaults to False. + key_vocab_filename: The file name for the key vocabulary file when + compute_word_count is True. name: A name for the operation (optional). Note that original order of the input may not be preserved. @@ -605,33 +554,26 @@ def __init__( self.ngrams_separator = ngrams_separator self.name = name self.split_string_by_delimiter = split_string_by_delimiter + self.key_vocab_filename = key_vocab_filename if compute_word_count: self.compute_word_count_fn = count_unqiue_words else: - self.compute_word_count_fn = lambda *args, **kwargs: {} + self.compute_word_count_fn = lambda *args, **kwargs: None if ngram_range != (1, 1) and not ngrams_separator: raise ValueError( 'ngrams_separator must be specified when ngram_range is not (1, 1)') - def get_artifacts(self, data: tf.SparseTensor, - col_name: str) -> Dict[str, tf.Tensor]: - return self.compute_word_count_fn(data, col_name) - def apply_transform(self, data: tf.SparseTensor, output_col_name: str): if self.split_string_by_delimiter: data = self._split_string_with_delimiter( data, self.split_string_by_delimiter) output = tft.bag_of_words( data, self.ngram_range, self.ngrams_separator, self.name) + # word counts are written to the key_vocab_filename + self.compute_word_count_fn(data, self.key_vocab_filename) return {output_col_name: output} -def count_unqiue_words(data: tf.SparseTensor, - output_col_name: str) -> Dict[str, tf.Tensor]: - keys, count = tft.count_per_key(data) - shape = [tf.shape(data)[0], tf.shape(keys)[0]] - return { - output_col_name + '_unique_elements': tf.broadcast_to(keys, shape), - output_col_name + '_counts': tf.broadcast_to(count, shape) - } +def count_unqiue_words(data: tf.SparseTensor, output_vocab_name: str) -> None: + tft.count_per_key(data, key_vocabulary_filename=output_vocab_name) diff --git a/sdks/python/apache_beam/ml/transforms/tft_test.py b/sdks/python/apache_beam/ml/transforms/tft_test.py index 41f59c868c3b..38ded6a809af 100644 --- a/sdks/python/apache_beam/ml/transforms/tft_test.py +++ b/sdks/python/apache_beam/ml/transforms/tft_test.py @@ -17,6 +17,7 @@ # pytype: skip-file +import os import shutil import tempfile import unittest @@ -38,31 +39,6 @@ if not tft: raise unittest.SkipTest('tensorflow_transform is not installed.') -z_score_expected = {'x_mean': 3.5, 'x_var': 2.9166666666666665} - - -def assert_z_score_artifacts(element): - element = element.as_dict() - assert 'x_mean' in element - assert 'x_var' in element - assert element['x_mean'] == z_score_expected['x_mean'] - assert element['x_var'] == z_score_expected['x_var'] - - -def assert_ScaleTo01_artifacts(element): - element = element.as_dict() - assert 'x_min' in element - assert 'x_max' in element - assert element['x_min'] == 1 - assert element['x_max'] == 6 - - -def assert_bucketize_artifacts(element): - element = element.as_dict() - assert 'x_quantiles' in element - assert np.array_equal( - element['x_quantiles'], np.array([3, 5], dtype=np.float32)) - class ScaleZScoreTest(unittest.TestCase): def setUp(self) -> None: @@ -100,7 +76,18 @@ def test_z_score(self): | "MLTransform" >> base.MLTransform( write_artifact_location=self.artifact_location).with_transform( tft.ScaleToZScore(columns=['x']))) - _ = (result | beam.Map(assert_z_score_artifacts)) + expected_data = [ + np.array([-1.46385], dtype=np.float32), + np.array([-0.87831], dtype=np.float32), + np.array([-0.29277], dtype=np.float32), + np.array([0.29277], dtype=np.float32), + np.array([0.87831], dtype=np.float32), + np.array([1.46385], dtype=np.float32), + ] + + actual_data = (result | beam.Map(lambda x: x.x)) + assert_that( + actual_data, equal_to(expected_data, equals_fn=np.array_equal)) def test_z_score_list_data(self): list_data = [{'x': [1, 2, 3]}, {'x': [4, 5, 6]}] @@ -111,7 +98,14 @@ def test_z_score_list_data(self): | "listMLTransform" >> base.MLTransform( write_artifact_location=self.artifact_location).with_transform( tft.ScaleToZScore(columns=['x']))) - _ = (list_result | beam.Map(assert_z_score_artifacts)) + + expected_data = [ + np.array([-1.46385, -0.87831, -0.29277], dtype=np.float32), + np.array([0.29277, 0.87831, 1.46385], dtype=np.float32) + ] + actual_data = (list_result | beam.Map(lambda x: x.x)) + assert_that( + actual_data, equal_to(expected_data, equals_fn=np.array_equal)) class ScaleTo01Test(unittest.TestCase): @@ -130,7 +124,6 @@ def test_ScaleTo01_list(self): | "MLTransform" >> base.MLTransform( write_artifact_location=self.artifact_location).with_transform( tft.ScaleTo01(columns=['x']))) - _ = (list_result | beam.Map(assert_ScaleTo01_artifacts)) expected_output = [ np.array([0, 0.2, 0.4], dtype=np.float32), @@ -150,7 +143,6 @@ def test_ScaleTo01(self): write_artifact_location=self.artifact_location).with_transform( tft.ScaleTo01(columns=['x']))) - _ = (result | beam.Map(assert_ScaleTo01_artifacts)) expected_output = ( np.array([0], dtype=np.float32), np.array([0.2], dtype=np.float32), @@ -179,7 +171,6 @@ def test_bucketize(self): | "MLTransform" >> base.MLTransform( write_artifact_location=self.artifact_location).with_transform( tft.Bucketize(columns=['x'], num_buckets=3))) - _ = (result | beam.Map(assert_bucketize_artifacts)) transformed_data = (result | beam.Map(lambda x: x.x)) expected_data = [ @@ -202,8 +193,6 @@ def test_bucketize_list(self): | "MLTransform" >> base.MLTransform( write_artifact_location=self.artifact_location).with_transform( tft.Bucketize(columns=['x'], num_buckets=3))) - _ = (list_result | beam.Map(assert_bucketize_artifacts)) - transformed_data = ( list_result | "TransformedColumnX" >> beam.Map(lambda ele: ele.x)) @@ -214,36 +203,6 @@ def test_bucketize_list(self): assert_that( transformed_data, equal_to(expected_data, equals_fn=np.array_equal)) - @parameterized.expand([ - (range(1, 10), [4, 7]), - (range(9, 0, -1), [4, 7]), - (range(19, 0, -1), [10]), - (range(1, 100), [25, 50, 75]), - # similar to the above but with odd number of elements - (range(1, 100, 2), [25, 51, 75]), - (range(99, 0, -1), range(10, 100, 10)) - ]) - def test_bucketize_boundaries(self, test_input, expected_boundaries): - # boundaries are outputted as artifacts for the Bucketize transform. - data = [{'x': [i]} for i in test_input] - num_buckets = len(expected_boundaries) + 1 - with beam.Pipeline() as p: - result = ( - p - | "Create" >> beam.Create(data) - | "MLTransform" >> base.MLTransform( - write_artifact_location=self.artifact_location).with_transform( - tft.Bucketize(columns=['x'], num_buckets=num_buckets))) - actual_boundaries = ( - result - | beam.Map(lambda x: x.as_dict()) - | beam.Map(lambda x: x['x_quantiles'])) - - def assert_boundaries(actual_boundaries): - assert np.array_equal(actual_boundaries, expected_boundaries) - - _ = (actual_boundaries | beam.Map(assert_boundaries)) - class ApplyBucketsTest(unittest.TestCase): def setUp(self) -> None: @@ -731,10 +690,6 @@ def test_bag_of_words_on_by_splitting_input_text(self): assert_that(result, equal_to(expected_data, equals_fn=np.array_equal)) def test_count_per_key_on_list(self): - def map_element_to_count(elements, counts): - d = {elements[i]: counts[i] for i in range(len(elements))} - return d - data = [{ 'x': ['I', 'like', 'pie', 'pie', 'pie'], }, { @@ -743,25 +698,28 @@ def map_element_to_count(elements, counts): 'x': ['Banana', 'Banana', 'Apple', 'Apple', 'Apple', 'Apple'] }] with beam.Pipeline() as p: - result = ( + _ = ( p | "Create" >> beam.Create(data) | "MLTransform" >> base.MLTransform( write_artifact_location=self.artifact_location, transforms=[ - tft.BagOfWords(columns=['x'], compute_word_count=True) + tft.BagOfWords( + columns=['x'], + compute_word_count=True, + key_vocab_filename='my_vocab') ])) - # the unique elements and counts are artifacts and will be - # stored in the result and same for all the elements in the - # PCollection. - result = result | beam.Map( - lambda x: map_element_to_count(x.x_unique_elements, x.x_counts)) + def validate_count_per_key(key_vocab_filename): + key_vocab_location = os.path.join( + self.artifact_location, 'transform_fn/assets', key_vocab_filename) + with open(key_vocab_location, 'r') as f: + key_vocab_list = [line.strip() for line in f] + return key_vocab_list - expected_data = [{ - b'Apple': 4, b'Banana': 2, b'I': 1, b'like': 1, b'pie': 4, b'yum': 2 - }] * 3 # since there are 3 elements in input. - assert_that(result, equal_to(expected_data)) + expected_data = ['2 yum', '4 Apple', '1 like', '1 I', '4 pie', '2 Banana'] + actual_data = validate_count_per_key('my_vocab') + self.assertEqual(expected_data, actual_data) if __name__ == '__main__': diff --git a/sdks/python/tox.ini b/sdks/python/tox.ini index 1e797d96074f..e4cf09cacba4 100644 --- a/sdks/python/tox.ini +++ b/sdks/python/tox.ini @@ -162,7 +162,7 @@ deps = holdup==1.8.0 extras = gcp -allowlist_externals = +allowlist_externals = bash echo sleep @@ -194,7 +194,7 @@ deps = extras = azure passenv = REQUESTS_CA_BUNDLE -allowlist_externals = +allowlist_externals = wget az bash @@ -311,11 +311,12 @@ commands = # Run all DataFrame API unit tests bash {toxinidir}/scripts/run_pytest.sh {envname} 'apache_beam/dataframe' -[testenv:py{38,39}-tft-113] +[testenv:py{38,39}-tft-{113,114}] deps = 113: tensorflow_transform>=1.13.0,<1.14.0 + 114: tensorflow_transform>=1.14.0,<1.15.0 commands = - bash {toxinidir}/scripts/run_pytest.sh {envname} 'apache_beam/ml/transforms' + bash {toxinidir}/scripts/run_pytest.sh {envname} 'apache_beam/ml/transforms apache_beam/examples/snippets/transforms/elementwise/mltransform_test.py' [testenv:py{38,39,310,311}-pytorch-{19,110,111,112,113}] deps = diff --git a/website/www/site/content/en/documentation/ml/preprocess-data.md b/website/www/site/content/en/documentation/ml/preprocess-data.md index cb79afff6036..2b291b9c75a5 100644 --- a/website/www/site/content/en/documentation/ml/preprocess-data.md +++ b/website/www/site/content/en/documentation/ml/preprocess-data.md @@ -105,7 +105,7 @@ artifacts. When you use the `write_artifact_location` parameter, the `MLTransform` class runs the specified transformations on the dataset and then creates artifacts from these transformations. The artifacts are stored in the location that you specify in -the `write_artifact_location` parameter or in the `MLTransform` output. +the `write_artifact_location` parameter. Write mode is useful when you want to store the results of your transformations for future use. For example, if you apply the same transformations on a @@ -120,8 +120,7 @@ The following examples demonstrate how write mode works. The `ComputeAndApplyVocabulary` transform outputs the indices of the vocabulary to the vocabulary file. - The `ScaleToZScore` transform calculates the mean and variance over the entire dataset - and then normalizes the entire dataset using the mean and variance. The - mean and variance are outputted by the `MLTransform` operation. + and then normalizes the entire dataset using the mean and variance. When you use the `write_artifact_location` parameter, these values are stored as a `tensorflow` graph in the location specified by the `write_artifact_location` parameter value. You can reuse the values in read mode From e4aef8c108316ed35913c308eceefe83614a1487 Mon Sep 17 00:00:00 2001 From: Danny McCormick <dannymccormick@google.com> Date: Wed, 25 Oct 2023 11:30:30 -0400 Subject: [PATCH 250/435] Restrict workflows to run on apache/beam (#29133) --- .github/workflows/beam_MetricsCredentialsRotation.yml | 5 +++-- .github/workflows/beam_Metrics_Report.yml | 5 +++-- .github/workflows/beam_PostCommit_BeamMetrics_Publish.yml | 5 +++-- .github/workflows/beam_Publish_Beam_SDK_Snapshots.yml | 5 +++-- .github/workflows/beam_Publish_Docker_Snapshots.yml | 5 +++-- .github/workflows/beam_Release_NightlySnapshot.yml | 5 +++-- .github/workflows/beam_Release_Python_NightlySnapshot.yml | 5 +++-- .github/workflows/build_runner_image.yml | 1 + 8 files changed, 22 insertions(+), 14 deletions(-) diff --git a/.github/workflows/beam_MetricsCredentialsRotation.yml b/.github/workflows/beam_MetricsCredentialsRotation.yml index 7b9727053aaa..839a8cba0ed0 100644 --- a/.github/workflows/beam_MetricsCredentialsRotation.yml +++ b/.github/workflows/beam_MetricsCredentialsRotation.yml @@ -49,8 +49,9 @@ env: jobs: beam_MetricsCredentialsRotation: if: | - github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' + (github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule') && + github.repository == 'apache/beam' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 name: ${{ matrix.job_name }} diff --git a/.github/workflows/beam_Metrics_Report.yml b/.github/workflows/beam_Metrics_Report.yml index 0c88171f50c1..9c4a540e7ef0 100644 --- a/.github/workflows/beam_Metrics_Report.yml +++ b/.github/workflows/beam_Metrics_Report.yml @@ -60,9 +60,10 @@ jobs: job_name: [beam_Metrics_Report] job_phrase: [Run Metrics Report] if: | - github.event_name == 'schedule' || + (github.event_name == 'schedule' || github.event_name == 'workflow_dispatch' || - github.event.comment.body == 'Run Metrics Report' + github.event.comment.body == 'Run Metrics Report') && + github.repository == 'apache/beam' steps: - uses: actions/checkout@v3 - name: Setup repository diff --git a/.github/workflows/beam_PostCommit_BeamMetrics_Publish.yml b/.github/workflows/beam_PostCommit_BeamMetrics_Publish.yml index 9cff830828a2..15ab008cec30 100644 --- a/.github/workflows/beam_PostCommit_BeamMetrics_Publish.yml +++ b/.github/workflows/beam_PostCommit_BeamMetrics_Publish.yml @@ -53,10 +53,11 @@ env: jobs: beam_PostCommit_BeamMetrics_Publish: if: | - github.event_name == 'push' || + (github.event_name == 'push' || github.event_name == 'workflow_dispatch' || github.event_name == 'schedule' || - github.event.comment.body == 'Run Beam Metrics Deployment' + github.event.comment.body == 'Run Beam Metrics Deployment') && + github.repository == 'apache/beam' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) diff --git a/.github/workflows/beam_Publish_Beam_SDK_Snapshots.yml b/.github/workflows/beam_Publish_Beam_SDK_Snapshots.yml index 9e48325b89ab..95f993bc33ac 100644 --- a/.github/workflows/beam_Publish_Beam_SDK_Snapshots.yml +++ b/.github/workflows/beam_Publish_Beam_SDK_Snapshots.yml @@ -50,8 +50,9 @@ env: jobs: beam_Publish_Beam_SDK_Snapshots: if: | - github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' + (github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule') && + github.repository == 'apache/beam' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 name: ${{ matrix.job_name }} (${{ matrix.container_task }}) diff --git a/.github/workflows/beam_Publish_Docker_Snapshots.yml b/.github/workflows/beam_Publish_Docker_Snapshots.yml index 01b846eb65d0..1abc26863067 100644 --- a/.github/workflows/beam_Publish_Docker_Snapshots.yml +++ b/.github/workflows/beam_Publish_Docker_Snapshots.yml @@ -50,9 +50,10 @@ env: jobs: beam_Publish_Docker_Snapshots: if: | - github.event_name == 'workflow_dispatch' || + (github.event_name == 'workflow_dispatch' || github.event_name == 'schedule' || - github.event.comment.body == 'Publish Docker Snapshots' + github.event.comment.body == 'Publish Docker Snapshots') && + github.repository == 'apache/beam' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) diff --git a/.github/workflows/beam_Release_NightlySnapshot.yml b/.github/workflows/beam_Release_NightlySnapshot.yml index bf22344c4199..735d0e466177 100644 --- a/.github/workflows/beam_Release_NightlySnapshot.yml +++ b/.github/workflows/beam_Release_NightlySnapshot.yml @@ -50,8 +50,9 @@ jobs: job_name: [beam_Release_NightlySnapshot] job_phrase: [Release Nightly Snapshot] if: | - github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' + (github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule') && + github.repository == 'apache/beam' steps: - uses: actions/checkout@v4 - name: Setup repository diff --git a/.github/workflows/beam_Release_Python_NightlySnapshot.yml b/.github/workflows/beam_Release_Python_NightlySnapshot.yml index a9f4ac5d2d14..fe132356fc53 100644 --- a/.github/workflows/beam_Release_Python_NightlySnapshot.yml +++ b/.github/workflows/beam_Release_Python_NightlySnapshot.yml @@ -49,8 +49,9 @@ jobs: matrix: job_name: [beam_Release_Python_NightlySnapshot] job_phrase: [Release Nightly Snapshot Python] - if: github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' + if: (github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule') && + github.repository == 'apache/beam' steps: - uses: actions/checkout@v4 diff --git a/.github/workflows/build_runner_image.yml b/.github/workflows/build_runner_image.yml index 6071d936958a..c0f4cacd86fe 100644 --- a/.github/workflows/build_runner_image.yml +++ b/.github/workflows/build_runner_image.yml @@ -30,6 +30,7 @@ env: docker_repo: apache-beam-testing/beam-github-actions/beam-arc-runner jobs: build-and-version-runner: + if: github.repository == 'apache/beam' env: working-directory: .github/gh-actions-self-hosted-runners/arc/images/ runs-on: [self-hosted, ubuntu-20.04] From 76ad5bc45af9b9435c0bb544d42a0e51f2905f84 Mon Sep 17 00:00:00 2001 From: Vitaly Terentyev <vitaly.terentyev@akvelon.com> Date: Wed, 25 Oct 2023 20:57:15 +0400 Subject: [PATCH 251/435] Add Send email step (#29088) --- .../beam_IODatastoresCredentialsRotation.yml | 18 +++++++++++++++++- .../beam_MetricsCredentialsRotation.yml | 18 +++++++++++++++++- 2 files changed, 34 insertions(+), 2 deletions(-) diff --git a/.github/workflows/beam_IODatastoresCredentialsRotation.yml b/.github/workflows/beam_IODatastoresCredentialsRotation.yml index 7a402e426fa7..0ededda00b39 100644 --- a/.github/workflows/beam_IODatastoresCredentialsRotation.yml +++ b/.github/workflows/beam_IODatastoresCredentialsRotation.yml @@ -77,4 +77,20 @@ jobs: - name: Completing the rotation run: | gcloud container clusters update io-datastores --complete-credential-rotation --zone=us-central1-a --quiet -# TODO: Send email to dev@beam.apache.org if something went wrong during credentials rotation \ No newline at end of file + - name: Generate Date + run: | + date=$(date -u +"%Y-%m-%d") + echo "date=$date" >> $GITHUB_ENV + - name: Send email + uses: dawidd6/action-send-mail@v3 + with: + server_address: smtp.gmail.com + server_port: 465 + secure: true + username: ${{ secrets.ISSUE_REPORT_SENDER_EMAIL_ADDRESS }} + password: ${{ secrets.ISSUE_REPORT_SENDER_EMAIL_PASSWORD }} + subject: Credentials Rotation Failure on IO-Datastores cluster (${{ env.date }}) + to: dev@beam.apache.org + from: gactions@beam.apache.org + body: | + Something went wrong during the automatic credentials rotation for IO-Datastores Cluster, performed at ${{ env.date }}. It may be necessary to check the state of the cluster certificates. For further details refer to the following links:\n * Failing job: https://github.com/apache/beam/actions/workflows/beam_IODatastoresCredentialsRotation.yml \n * Job configuration: https://github.com/apache/beam/blob/master/.github/workflows/beam_IODatastoresCredentialsRotation.yml \n * Cluster URL: https://pantheon.corp.google.com/kubernetes/clusters/details/us-central1-a/io-datastores/details?mods=dataflow_dev&project=apache-beam-testing \ No newline at end of file diff --git a/.github/workflows/beam_MetricsCredentialsRotation.yml b/.github/workflows/beam_MetricsCredentialsRotation.yml index 839a8cba0ed0..eda3ec355631 100644 --- a/.github/workflows/beam_MetricsCredentialsRotation.yml +++ b/.github/workflows/beam_MetricsCredentialsRotation.yml @@ -78,4 +78,20 @@ jobs: - name: Completing the rotation run: | gcloud container clusters update metrics --complete-credential-rotation --zone=us-central1-a --quiet -# TODO: Send email to dev@beam.apache.org if something went wrong during credentials rotation \ No newline at end of file + - name: Generate Date + run: | + date=$(date -u +"%Y-%m-%d") + echo "date=$date" >> $GITHUB_ENV + - name: Send email + uses: dawidd6/action-send-mail@v3 + with: + server_address: smtp.gmail.com + server_port: 465 + secure: true + username: ${{ secrets.ISSUE_REPORT_SENDER_EMAIL_ADDRESS }} + password: ${{ secrets.ISSUE_REPORT_SENDER_EMAIL_PASSWORD }} + subject: Credentials Rotation Failure on Metrics cluster (${{ env.date }}) + to: dev@beam.apache.org + from: gactions@beam.apache.org + body: | + Something went wrong during the automatic credentials rotation for Metrics Cluster, performed at ${{ env.date }}. It may be necessary to check the state of the cluster certificates. For further details refer to the following links:\n * Failing job: https://github.com/apache/beam/actions/workflows/beam_MetricsCredentialsRotation.yml \n * Job configuration: https://github.com/apache/beam/blob/master/.github/workflows/beam_MetricsCredentialsRotation.yml \n * Cluster URL: https://pantheon.corp.google.com/kubernetes/clusters/details/us-central1-a/metrics/details?mods=dataflow_dev&project=apache-beam-testing \ No newline at end of file From d6b34679e7ee5b12c652dcbc7daee1d95c9e1ff5 Mon Sep 17 00:00:00 2001 From: Yi Hu <yathu@google.com> Date: Wed, 25 Oct 2023 13:03:22 -0400 Subject: [PATCH 252/435] IgnoreExitValue for dataflow runner cleanUpDockerJavaImages (#29124) --- runners/google-cloud-dataflow-java/build.gradle | 1 + 1 file changed, 1 insertion(+) diff --git a/runners/google-cloud-dataflow-java/build.gradle b/runners/google-cloud-dataflow-java/build.gradle index e4f34687d31c..9e4d28ae5b4b 100644 --- a/runners/google-cloud-dataflow-java/build.gradle +++ b/runners/google-cloud-dataflow-java/build.gradle @@ -304,6 +304,7 @@ def cleanUpDockerJavaImages = tasks.register("cleanUpDockerJavaImages") { commandLine "docker", "rmi", "--force", "${dockerJavaImageName}" } exec { + ignoreExitValue true commandLine "gcloud", "--quiet", "container", "images", "untag", "${dockerJavaImageName}" } exec { From 99782138b62d167c80577ba8bc3231c3c0c952ee Mon Sep 17 00:00:00 2001 From: Yi Hu <yathu@google.com> Date: Wed, 25 Oct 2023 16:30:36 -0400 Subject: [PATCH 253/435] Publish Java21 container in release process (#29134) --- CHANGES.md | 2 ++ sdks/java/container/build.gradle | 1 + 2 files changed, 3 insertions(+) diff --git a/CHANGES.md b/CHANGES.md index 0eee0a350ae6..49b4fdfe89b0 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -61,6 +61,8 @@ Please, use `beam-sdks-java-extensions-avro` instead. This will allow to easily update Avro version in user code without potential breaking changes in Beam "core" since the Beam Avro extension already supports the latest Avro versions and should handle this. ([#25252](https://github.com/apache/beam/issues/25252)). +* Publishing Java 21 SDK container images now supported as part of Apache Beam release process. ([#28120](https://github.com/apache/beam/issues/28120)) + * Direct Runner and Dataflow Runner V2 support running pipeline on Java21. Support for other runners are planned in upcoming versions. ## I/Os diff --git a/sdks/java/container/build.gradle b/sdks/java/container/build.gradle index 4c4b6aaa31fd..0cafdf511854 100644 --- a/sdks/java/container/build.gradle +++ b/sdks/java/container/build.gradle @@ -83,4 +83,5 @@ task pushAll { dependsOn ":sdks:java:container:java8:docker" dependsOn ":sdks:java:container:java11:docker" dependsOn ":sdks:java:container:java17:docker" + dependsOn ":sdks:java:container:java21:docker" } From 208c0284b8287082d15f86d559457751669a7027 Mon Sep 17 00:00:00 2001 From: Jack McCluskey <34928439+jrmccluskey@users.noreply.github.com> Date: Wed, 25 Oct 2023 16:55:58 -0400 Subject: [PATCH 254/435] Add typehint support for collections.abc.Set types (#29137) * Add typehint support for collections.abc.Set types * Formatting * more formatting * Add extra test cases for collections interactions * formatting again * even more formatting * Reorder FrozenSet check * Gate type subscripting * last linting cleanup --- .../typehints/native_type_compatibility.py | 29 +++++++++++++++---- .../native_type_compatibility_test.py | 2 ++ .../apache_beam/typehints/typehints_test.py | 13 +++++++++ 3 files changed, 39 insertions(+), 5 deletions(-) diff --git a/sdks/python/apache_beam/typehints/native_type_compatibility.py b/sdks/python/apache_beam/typehints/native_type_compatibility.py index e916f34146f1..67cb2b8c3f03 100644 --- a/sdks/python/apache_beam/typehints/native_type_compatibility.py +++ b/sdks/python/apache_beam/typehints/native_type_compatibility.py @@ -45,6 +45,11 @@ frozenset: typing.FrozenSet, } +_CONVERTED_COLLECTIONS = [ + collections.abc.Set, + collections.abc.MutableSet, +] + def _get_args(typ): """Returns a list of arguments to the given type. @@ -148,6 +153,15 @@ def _match_is_union(user_type): return False +def match_is_set(user_type): + if _safe_issubclass(user_type, typing.Set): + return True + elif getattr(user_type, '__origin__', None) is not None: + return _safe_issubclass(user_type.__origin__, collections.abc.Set) + else: + return False + + def is_any(typ): return typ is typing.Any @@ -232,6 +246,7 @@ def convert_to_beam_type(typ): None) == 'collections.abc': typ = convert_collections_to_typing(typ) + typ_module = getattr(typ, '__module__', None) if isinstance(typ, typing.TypeVar): # This is a special case, as it's not parameterized by types. # Also, identity must be preserved through conversion (i.e. the same @@ -254,8 +269,13 @@ def convert_to_beam_type(typ): # TODO(https://github.com/apache/beam/issues/20076): Currently unhandled. _LOGGER.info('Converting NewType type hint to Any: "%s"', typ) return typehints.Any - elif getattr(typ, '__module__', None) != 'typing': - # Only translate types from the typing module. + elif (typ_module != 'typing') and (typ_module != 'collections.abc'): + # Only translate types from the typing and collections.abc modules. + return typ + if (typ_module == 'collections.abc' and + typ.__origin__ not in _CONVERTED_COLLECTIONS): + # TODO(https://github.com/apache/beam/issues/29135): + # Support more collections types return typ type_map = [ @@ -278,13 +298,12 @@ def convert_to_beam_type(typ): match=_match_issubclass(typing.List), arity=1, beam_type=typehints.List), - _TypeMapEntry( - match=_match_issubclass(typing.Set), arity=1, - beam_type=typehints.Set), + # FrozenSets are a specific instance of a set, so we check this first. _TypeMapEntry( match=_match_issubclass(typing.FrozenSet), arity=1, beam_type=typehints.FrozenSet), + _TypeMapEntry(match=match_is_set, arity=1, beam_type=typehints.Set), # NamedTuple is a subclass of Tuple, but it needs special handling. # We just convert it to Any for now. # This MUST appear before the entry for the normal Tuple. diff --git a/sdks/python/apache_beam/typehints/native_type_compatibility_test.py b/sdks/python/apache_beam/typehints/native_type_compatibility_test.py index b9280c57a393..23b9beadd336 100644 --- a/sdks/python/apache_beam/typehints/native_type_compatibility_test.py +++ b/sdks/python/apache_beam/typehints/native_type_compatibility_test.py @@ -171,6 +171,8 @@ def test_convert_to_beam_type_with_collections_types(self): 'mapping not caught', collections.abc.Mapping[str, int], collections.abc.Mapping[str, int]), + ('set', collections.abc.Set[str], typehints.Set[str]), + ('mutable set', collections.abc.MutableSet[int], typehints.Set[int]), ] for test_case in test_cases: diff --git a/sdks/python/apache_beam/typehints/typehints_test.py b/sdks/python/apache_beam/typehints/typehints_test.py index a1a1913bded7..7f8c322f9f40 100644 --- a/sdks/python/apache_beam/typehints/typehints_test.py +++ b/sdks/python/apache_beam/typehints/typehints_test.py @@ -19,6 +19,7 @@ # pytype: skip-file +import collections.abc import functools import sys import typing @@ -845,6 +846,18 @@ class SetHintTestCase(BaseSetHintTest.CommonTests): beam_type = typehints.Set string_type = 'Set' + def test_builtin_compatibility(self): + if sys.version_info >= (3, 9): + self.assertCompatible(set[int], collections.abc.Set[int]) + self.assertCompatible(set[int], collections.abc.MutableSet[int]) + + def test_collections_compatibility(self): + if sys.version_info >= (3, 9): + self.assertCompatible( + collections.abc.Set[int], collections.abc.MutableSet[int]) + self.assertCompatible( + collections.abc.MutableSet[int], collections.abc.Set[int]) + class FrozenSetHintTestCase(BaseSetHintTest.CommonTests): py_type = frozenset From ffb43321a0a4a646c6568336b8fba5787b1b64a0 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw <robertwb@gmail.com> Date: Wed, 25 Oct 2023 16:47:50 -0700 Subject: [PATCH 255/435] [YAML] Add optional output type parameter to mappings. (#29077) --- sdks/python/apache_beam/transforms/core.py | 11 ++- sdks/python/apache_beam/yaml/yaml_mapping.md | 44 ++++++++++++ sdks/python/apache_beam/yaml/yaml_mapping.py | 68 +++++++++++++++++-- .../apache_beam/yaml/yaml_mapping_test.py | 21 ++++++ 4 files changed, 136 insertions(+), 8 deletions(-) diff --git a/sdks/python/apache_beam/transforms/core.py b/sdks/python/apache_beam/transforms/core.py index 6bf0e187ce91..dec81905f201 100644 --- a/sdks/python/apache_beam/transforms/core.py +++ b/sdks/python/apache_beam/transforms/core.py @@ -3372,9 +3372,16 @@ def expand(self, pcoll): for name, expr in self._fields}))).as_result() def infer_output_type(self, input_type): + def extract_return_type(expr): + expr_hints = get_type_hints(expr) + if (expr_hints and expr_hints.has_simple_output_type() and + expr_hints.simple_output_type(None) != typehints.Any): + return expr_hints.simple_output_type(None) + else: + return trivial_inference.infer_return_type(expr, [input_type]) + return row_type.RowTypeConstraint.from_fields([ - (name, trivial_inference.infer_return_type(expr, [input_type])) - for (name, expr) in self._fields + (name, extract_return_type(expr)) for (name, expr) in self._fields ]) diff --git a/sdks/python/apache_beam/yaml/yaml_mapping.md b/sdks/python/apache_beam/yaml/yaml_mapping.md index 653b4abe8b89..e760b691e13d 100644 --- a/sdks/python/apache_beam/yaml/yaml_mapping.md +++ b/sdks/python/apache_beam/yaml/yaml_mapping.md @@ -200,3 +200,47 @@ criteria. This can be accomplished with a `Filter` transform, e.g. language: sql keep: "col2 > 0" ``` + +## Types + +Beam will try to infer the types involved in the mappings, but sometimes this +is not possible. In these cases one can explicitly denote the expected output +type, e.g. + +``` +- type: MapToFields + config: + language: python + fields: + new_col: + expression: "col1.upper()" + output_type: string +``` + +The expected type is given in json schema notation, with the addition that +a top-level basic types may be given as a literal string rather than requiring +a `{type: 'basic_type_name'}` nesting. + +``` +- type: MapToFields + config: + language: python + fields: + new_col: + expression: "col1.upper()" + output_type: string + another_col: + expression: "beam.Row(a=col1, b=[col2])" + output_type: + type: 'object' + properties: + a: + type: 'string' + b: + type: 'array' + items: + type: 'number' +``` + +This can be especially useful to resolve errors involving the inability to +handle the `beam:logical:pythonsdk_any:v1` type. diff --git a/sdks/python/apache_beam/yaml/yaml_mapping.py b/sdks/python/apache_beam/yaml/yaml_mapping.py index 62bc3a7ae56d..e217ab285844 100644 --- a/sdks/python/apache_beam/yaml/yaml_mapping.py +++ b/sdks/python/apache_beam/yaml/yaml_mapping.py @@ -30,10 +30,13 @@ import apache_beam as beam from apache_beam.io.filesystems import FileSystems +from apache_beam.portability.api import schema_pb2 from apache_beam.typehints import row_type +from apache_beam.typehints import schemas from apache_beam.typehints import trivial_inference from apache_beam.typehints.schemas import named_fields_from_element_type from apache_beam.utils import python_callable +from apache_beam.yaml import json_utils from apache_beam.yaml import options from apache_beam.yaml import yaml_provider @@ -121,11 +124,48 @@ def _expand_python_mapping_func( return python_callable.PythonCallableWithSource(source) +def _validator(beam_type: schema_pb2.FieldType) -> Callable[[Any], bool]: + """Returns a callable converting rows of the given type to Json objects.""" + type_info = beam_type.WhichOneof("type_info") + if type_info == "atomic_type": + if beam_type.atomic_type == schema_pb2.BOOLEAN: + return lambda x: isinstance(x, bool) + elif beam_type.atomic_type == schema_pb2.INT64: + return lambda x: isinstance(x, int) + elif beam_type.atomic_type == schema_pb2.DOUBLE: + return lambda x: isinstance(x, (int, float)) + elif beam_type.atomic_type == schema_pb2.STRING: + return lambda x: isinstance(x, str) + else: + raise ValueError( + f'Unknown or unsupported atomic type: {beam_type.atomic_type}') + elif type_info == "array_type": + element_validator = _validator(beam_type.array_type.element_type) + return lambda value: all(element_validator(e) for e in value) + elif type_info == "iterable_type": + element_validator = _validator(beam_type.iterable_type.element_type) + return lambda value: all(element_validator(e) for e in value) + elif type_info == "map_type": + key_validator = _validator(beam_type.map_type.key_type) + value_validator = _validator(beam_type.map_type.value_type) + return lambda value: all( + key_validator(k) and value_validator(v) for (k, v) in value.items()) + elif type_info == "row_type": + validators = { + field.name: _validator(field.type) + for field in beam_type.row_type.schema.fields + } + return lambda row: all( + validator(getattr(row, name)) + for (name, validator) in validators.items()) + else: + raise ValueError(f"Unrecognized type_info: {type_info!r}") + + def _as_callable(original_fields, expr, transform_name, language): if expr in original_fields: return expr - # TODO(yaml): support a type parameter # TODO(yaml): support an imports parameter # TODO(yaml): support a requirements parameter (possibly at a higher level) if isinstance(expr, str): @@ -133,20 +173,36 @@ def _as_callable(original_fields, expr, transform_name, language): if not isinstance(expr, dict): raise ValueError( f"Ambiguous expression type (perhaps missing quoting?): {expr}") - elif len(expr) != 1 and ('path' not in expr or 'name' not in expr): - raise ValueError(f"Ambiguous expression type: {list(expr.keys())}") - + explicit_type = expr.pop('output_type', None) _check_mapping_arguments(transform_name, **expr) if language == "javascript": - return _expand_javascript_mapping_func(original_fields, **expr) + func = _expand_javascript_mapping_func(original_fields, **expr) elif language == "python": - return _expand_python_mapping_func(original_fields, **expr) + func = _expand_python_mapping_func(original_fields, **expr) else: raise ValueError( f'Unknown language for mapping transform: {language}. ' 'Supported languages are "javascript" and "python."') + if explicit_type: + if isinstance(explicit_type, str): + explicit_type = {'type': explicit_type} + beam_type = json_utils.json_type_to_beam_type(explicit_type) + validator = _validator(beam_type) + + @beam.typehints.with_output_types(schemas.typing_from_runner_api(beam_type)) + def checking_func(row): + result = func(row) + if not validator(result): + raise TypeError(f'{result} violates schema {explicit_type}') + return result + + return checking_func + + else: + return func + def exception_handling_args(error_handling_spec): if error_handling_spec: diff --git a/sdks/python/apache_beam/yaml/yaml_mapping_test.py b/sdks/python/apache_beam/yaml/yaml_mapping_test.py index a09214c26b6d..0de2f7022550 100644 --- a/sdks/python/apache_beam/yaml/yaml_mapping_test.py +++ b/sdks/python/apache_beam/yaml/yaml_mapping_test.py @@ -132,6 +132,27 @@ def test_explode(self): beam.Row(a=3, b='y', c=.125, range=2), ])) + def test_validate_explicit_types(self): + with self.assertRaisesRegex(TypeError, r'.*violates schema.*'): + with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions( + pickle_library='cloudpickle')) as p: + elements = p | beam.Create([ + beam.Row(a=2, b='abc', c=.25), + beam.Row(a=3, b='xy', c=.125), + ]) + result = elements | YamlTransform( + ''' + type: MapToFields + input: input + config: + language: python + fields: + bad: + expression: "a + c" + output_type: string # This is a lie. + ''') + self.assertEqual(result.element_type._fields[0][1], str) + YamlMappingDocTest = createTestSuite( 'YamlMappingDocTest', From 8b8af7fee894cc8bf4c3196e8d7889be4c3012f7 Mon Sep 17 00:00:00 2001 From: Bruno Volpato <bvolpato@google.com> Date: Thu, 26 Oct 2023 08:56:19 -0400 Subject: [PATCH 256/435] Change level of initialization exception to error, make errors obvious (#29078) --- .../src/main/java/org/apache/beam/fn/harness/FnHarness.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java index e103da4d6007..cada9b12f60b 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java @@ -384,9 +384,10 @@ private BeamFnApi.ProcessBundleDescriptor loadDescriptor(String id) { } processBundleHandler.shutdown(); } catch (Exception e) { - System.out.println("Shutting down harness due to exception: " + e.toString()); + LOG.error("Shutting down harness due to exception", e); + e.printStackTrace(); } finally { - System.out.println("Shutting SDK harness down."); + LOG.info("Shutting SDK harness down."); executionStateSampler.stop(); executorService.shutdown(); } From 1ff014d61011d807a46a978dec3ebcbe7b946487 Mon Sep 17 00:00:00 2001 From: Danny McCormick <dannymccormick@google.com> Date: Thu, 26 Oct 2023 09:41:50 -0400 Subject: [PATCH 257/435] Move metrics report fully to GHA (#29136) * Move metrics report fully to GHA * Remove issue comment trigger * Remove issue comment trigger * Simplify --- .github/workflows/beam_Metrics_Report.yml | 26 ++----- .test-infra/jenkins/job_Metrics_Report.groovy | 77 ------------------- 2 files changed, 6 insertions(+), 97 deletions(-) delete mode 100644 .test-infra/jenkins/job_Metrics_Report.groovy diff --git a/.github/workflows/beam_Metrics_Report.yml b/.github/workflows/beam_Metrics_Report.yml index 9c4a540e7ef0..1502c1fa326c 100644 --- a/.github/workflows/beam_Metrics_Report.yml +++ b/.github/workflows/beam_Metrics_Report.yml @@ -18,10 +18,8 @@ name: Beam Metrics Report on: - # issue_comment: - # types: [created] - # schedule: - # - cron: '0 */23 * * *' + schedule: + - cron: '0 11 * * 2' workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs @@ -52,26 +50,15 @@ permissions: jobs: beam_Metrics_Report: - name: ${{matrix.job_name}} (${{matrix.job_phrase}}) + name: beam_Metrics_Report runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 - strategy: - matrix: - job_name: [beam_Metrics_Report] - job_phrase: [Run Metrics Report] if: | (github.event_name == 'schedule' || - github.event_name == 'workflow_dispatch' || - github.event.comment.body == 'Run Metrics Report') && + github.event_name == 'workflow_dispatch') && github.repository == 'apache/beam' steps: - uses: actions/checkout@v3 - - name: Setup repository - uses: ./.github/actions/setup-action - with: - comment_phrase: ${{ matrix.job_phrase }} - github_token: ${{ secrets.GITHUB_TOKEN }} - github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - name: Setup environment uses: ./.github/actions/setup-environment-action with: @@ -102,7 +89,6 @@ jobs: username: ${{ secrets.ISSUE_REPORT_SENDER_EMAIL_ADDRESS }} password: ${{ secrets.ISSUE_REPORT_SENDER_EMAIL_PASSWORD }} subject: Beam Metrics Report ${{ env.date }} - to: dannymccormick@google.com #dev@beam.apache.org - cc: vlado.djerek@akvelon.com - from: gactions@beam.apache.org # <user@example.com> + to: dev@beam.apache.org + from: beamactions@gmail.com html_body: file://${{ github.workspace }}/.test-infra/jenkins/metrics_report/beam-metrics_report.html diff --git a/.test-infra/jenkins/job_Metrics_Report.groovy b/.test-infra/jenkins/job_Metrics_Report.groovy deleted file mode 100644 index d1d358221213..000000000000 --- a/.test-infra/jenkins/job_Metrics_Report.groovy +++ /dev/null @@ -1,77 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonJobProperties as commonJobProperties -import InfluxDBCredentialsHelper - -job('beam_Metrics_Report') { - description('Runs Beam metrics report.') - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties( - delegate, 'master', 100, true, 'beam', false) - InfluxDBCredentialsHelper.useCredentials(delegate) - - def influxDb = InfluxDBCredentialsHelper.InfluxDBDatabaseName - def influxHost = InfluxDBCredentialsHelper.InfluxDBHost - def influxPort = InfluxDBCredentialsHelper.InfluxDBPort - - // Allows triggering this build against pull requests. - commonJobProperties.enablePhraseTriggeringFromPullRequest( - delegate, - 'Beam Metrics Report', - 'Run Metrics Report', - false - ) - - commonJobProperties.setAutoJob( - delegate, - '@weekly') - - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - commonJobProperties.setGradleSwitches(delegate) - switches("-PinfluxDb=${influxDb}") - switches("-PinfluxHost=${influxHost}") - switches("-PinfluxPort=${influxPort}") - tasks(':beam-test-jenkins:generateMetricsReport') - } - } - - def date = new Date().format('yyyy-MM-dd') - publishers { - extendedEmail { - triggers { - always { - recipientList('dev@beam.apache.org') - contentType('text/html') - subject("Beam Metrics Report (${date})") - content('''${FILE, path="src/.test-infra/jenkins/metrics_report/beam-metrics_report.html"}''') - } - } - } - archiveArtifacts { - pattern('src/.test-infra/jenkins/metrics_report/beam-metrics_report.html') - onlyIfSuccessful() - } - wsCleanup { - excludePattern('src/.test-infra/jenkins/metrics_report/beam-metrics_report.html') - } - } -} From 60c2ba56f60ccfd34384711302daf790cd697db7 Mon Sep 17 00:00:00 2001 From: tvalentyn <tvalentyn@users.noreply.github.com> Date: Thu, 26 Oct 2023 09:39:32 -0700 Subject: [PATCH 258/435] Set correct markers for the other test scenario. (#29141) --- .../apache_beam/ml/inference/vertex_ai_inference_it_test.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/ml/inference/vertex_ai_inference_it_test.py b/sdks/python/apache_beam/ml/inference/vertex_ai_inference_it_test.py index 168ab031abb1..7c96dbe8b847 100644 --- a/sdks/python/apache_beam/ml/inference/vertex_ai_inference_it_test.py +++ b/sdks/python/apache_beam/ml/inference/vertex_ai_inference_it_test.py @@ -65,8 +65,7 @@ def test_vertex_ai_run_flower_image_classification(self): test_pipeline.get_full_options_as_args(**extra_opts)) self.assertEqual(FileSystems().exists(output_file), True) - @pytest.mark.uses_vertex_ai - @pytest.mark.it_postcommit + @pytest.mark.vertex_ai_postcommit def test_vertex_ai_run_llm_text_classification(self): output_file = '/'.join([_OUTPUT_DIR, str(uuid.uuid4()), 'output.txt']) From a411a71bc464bb1b59e704a686f0a1264e37c9df Mon Sep 17 00:00:00 2001 From: Damon <damondouglas@users.noreply.github.com> Date: Thu, 26 Oct 2023 09:48:51 -0700 Subject: [PATCH 259/435] [RRIO] create RedisClient utility (#29081) * [RRIO] create RedisClient utility * Fix gradle command in GitHub workflow * Fix alignment of method names * Patch grammer in javadoc * Add code comment for JUnit4 ExternalResource rule * Revert GitHub workflow change * Validate rpush,lpop FIFO order --- sdks/java/io/rrio/build.gradle | 12 + .../beam/io/requestresponse/RedisClient.java | 188 ++++++++++++++++ .../io/requestresponse/RedisClientTestIT.java | 209 ++++++++++++++++++ .../RedisExternalResourcesRule.java | 71 ++++++ 4 files changed, 480 insertions(+) create mode 100644 sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/RedisClient.java create mode 100644 sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponse/RedisClientTestIT.java create mode 100644 sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponse/RedisExternalResourcesRule.java diff --git a/sdks/java/io/rrio/build.gradle b/sdks/java/io/rrio/build.gradle index d7d5c8817d06..6963fcb23ddf 100644 --- a/sdks/java/io/rrio/build.gradle +++ b/sdks/java/io/rrio/build.gradle @@ -24,13 +24,25 @@ applyJavaNature( description = "Apache Beam :: SDKS :: Java :: IO :: RequestResponseIO (RRIO)" ext.summary = "Support to read from and write to Web APIs" +var jedisVersion = "5.0.0" + dependencies { implementation project(path: ":sdks:java:core", configuration: "shadow") implementation library.java.joda_time implementation library.java.vendored_guava_32_1_2_jre + implementation "redis.clients:jedis:$jedisVersion" testImplementation project(path: ":sdks:java:core", configuration: "shadowTest") + testImplementation project(path: ":sdks:java:io:common", configuration: "testRuntimeMigration") testImplementation library.java.junit + testImplementation library.java.testcontainers_base + testRuntimeOnly project(path: ":runners:direct-java", configuration: "shadow") testRuntimeOnly library.java.slf4j_jdk14 +} + +task integrationTest(type: Test) { + group = "verification" + + include '**/*IT.class' } \ No newline at end of file diff --git a/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/RedisClient.java b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/RedisClient.java new file mode 100644 index 000000000000..a87f5c191e4b --- /dev/null +++ b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/RedisClient.java @@ -0,0 +1,188 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.io.requestresponse; + +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; + +import java.net.URI; +import java.nio.charset.StandardCharsets; +import org.apache.beam.sdk.transforms.DoFn; +import org.checkerframework.checker.nullness.qual.MonotonicNonNull; +import org.checkerframework.checker.nullness.qual.NonNull; +import org.joda.time.Duration; +import redis.clients.jedis.JedisPooled; +import redis.clients.jedis.exceptions.JedisException; + +/** + * {@link RedisClient} is a convenience class that supports operations needed for caching by various + * transforms in this package. It implements the {@link SetupTeardown} interface for ease-of-use + * within a {@link DoFn} context. Unlike the underlying {@link JedisPooled} client, {@link + * RedisClient} is {@link java.io.Serializable}. + */ +class RedisClient implements SetupTeardown { + + private final URI uri; + + private transient @MonotonicNonNull JedisPooled jedis; + + /** + * Instantiates a {@link RedisClient}. {@link URI} expected of the form: {@code + * redis://<host>:<port>}. + */ + RedisClient(URI uri) { + this.uri = uri; + } + + /** + * Decrement a value stored by the key, returning the resulting decremented value. Per Redis + * convention, sets the value to -1 for keys that do not exist. Naming of this method preserves + * that of the underlying {@link JedisPooled} client and performs a null check prior to execution. + */ + long decr(String key) throws UserCodeExecutionException { + try { + return getSafeClient().decr(key); + } catch (JedisException e) { + throw new UserCodeExecutionException(e); + } + } + + /** + * Get the long value stored by the key. Yields zero when key does not exist, keeping consistency + * with Redis convention. Consider using {@link #exists} to query key existance. + */ + long getLong(String key) throws UserCodeExecutionException { + try { + return getSafeClient().decrBy(key, 0L); + } catch (JedisException e) { + throw new UserCodeExecutionException(e); + } + } + + /** Query whether the key exists. */ + boolean exists(String key) throws UserCodeExecutionException { + try { + return getSafeClient().exists(key); + } catch (JedisException e) { + throw new UserCodeExecutionException(e); + } + } + + /** + * Increment a value stored by the key, returning the resulting decremented value. Sets the value + * to 1, if key does not exist, per Redis convention. Naming of this method preserves that of the + * underlying {@link JedisPooled} client and performs a null check prior to execution. + */ + long incr(String key) throws UserCodeExecutionException { + try { + return getSafeClient().incr(key); + } catch (JedisException e) { + throw new UserCodeExecutionException(e); + } + } + + /** + * Query the size of a list identified by the key. Returns 0 if key does not exist, per Redis + * convention. Naming of this method preserves that of the underlying {@link JedisPooled} client + * and performs a null check prior to execution. + */ + long llen(String key) throws UserCodeExecutionException { + try { + return getSafeClient().llen(key); + } catch (JedisException e) { + throw new UserCodeExecutionException(e); + } + } + + /** Query whether the Redis list is empty. Calls {@link #llen} to determine this. */ + boolean isEmpty(String key) throws UserCodeExecutionException { + return this.llen(key) == 0L; + } + + /** + * Pushes items to the back ('right') of the list. Naming of this method preserves that of the + * underlying {@link JedisPooled} client and performs a null check prior to execution. + */ + void rpush(String key, byte[]... items) throws UserCodeExecutionException { + try { + getSafeClient().rpush(key.getBytes(StandardCharsets.UTF_8), items); + } catch (JedisException e) { + throw new UserCodeExecutionException(e); + } + } + + /** + * Pops items from the front ('left') of the list. Naming of this method preserves that of the + * underlying {@link JedisPooled} client and performs a null check prior to execution. + */ + byte[] lpop(String key) throws UserCodeExecutionException { + try { + return getSafeClient().lpop(key.getBytes(StandardCharsets.UTF_8)); + } catch (JedisException e) { + throw new UserCodeExecutionException(e); + } + } + + /** + * Sets the key/value for a Duration expiry. Naming of this method preserves that of the + * underlying {@link JedisPooled} client and performs a null check prior to execution. + */ + void setex(byte[] key, byte[] value, @NonNull Duration expiry) throws UserCodeExecutionException { + try { + getSafeClient().setex(key, expiry.getStandardSeconds(), value); + } catch (JedisException e) { + throw new UserCodeExecutionException(e); + } + } + + /** + * Sets the key/value for a Duration expiry. Naming of this method preserves that of the + * underlying {@link JedisPooled} client and performs a null check prior to execution. + */ + void setex(String key, Long value, @NonNull Duration expiry) throws UserCodeExecutionException { + try { + getSafeClient().setex(key, expiry.getStandardSeconds(), String.valueOf(value)); + } catch (JedisException e) { + throw new UserCodeExecutionException(e); + } + } + + /** Overrides {@link SetupTeardown}'s {@link SetupTeardown#setup} method. */ + @Override + public void setup() throws UserCodeExecutionException { + try { + jedis = new JedisPooled(uri); + jedis.ping(); + } catch (JedisException e) { + String message = + String.format("Failed to connect to host: %s, error: %s", uri, e.getMessage()); + throw new UserCodeExecutionException(message, e); + } + } + + private @NonNull JedisPooled getSafeClient() { + return checkStateNotNull(jedis); + } + + /** Overrides {@link SetupTeardown}'s {@link SetupTeardown#teardown} method. */ + @Override + public void teardown() throws UserCodeExecutionException { + if (jedis != null) { + jedis.close(); + } + } +} diff --git a/sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponse/RedisClientTestIT.java b/sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponse/RedisClientTestIT.java new file mode 100644 index 000000000000..1fbb320a5f23 --- /dev/null +++ b/sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponse/RedisClientTestIT.java @@ -0,0 +1,209 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.io.requestresponse; + +import static org.apache.beam.sdk.io.common.SchemaAwareJavaBeans.allPrimitiveDataTypes; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertThrows; +import static org.junit.Assert.assertTrue; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.math.BigDecimal; +import java.net.URI; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.List; +import java.util.UUID; +import org.apache.beam.sdk.coders.SerializableCoder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.io.common.SchemaAwareJavaBeans.AllPrimitiveDataTypes; +import org.apache.beam.sdk.util.SerializableUtils; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; +import org.checkerframework.checker.nullness.qual.NonNull; +import org.joda.time.Duration; +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.testcontainers.containers.GenericContainer; +import org.testcontainers.utility.DockerImageName; + +/** Integration tests for {@link RedisClient}. */ +@RunWith(JUnit4.class) +public class RedisClientTestIT { + + private static final String CONTAINER_IMAGE_NAME = "redis:5.0.3-alpine"; + private static final Integer PORT = 6379; + + @Rule + public GenericContainer<?> redis = + new GenericContainer<>(DockerImageName.parse(CONTAINER_IMAGE_NAME)).withExposedPorts(PORT); + + @Rule + public RedisExternalResourcesRule externalClients = + new RedisExternalResourcesRule( + () -> { + redis.start(); + return URI.create( + String.format("redis://%s:%d", redis.getHost(), redis.getFirstMappedPort())); + }); + + @Test + public void canSerialize() { + SerializableUtils.serializeToByteArray(externalClients.getActualClient()); + } + + @Test + public void wrongHostURIThrowsException() { + URI uri = URI.create("redis://1.2.3.4:6379"); + RedisClient client = new RedisClient(uri); + UserCodeExecutionException got = assertThrows(UserCodeExecutionException.class, client::setup); + String expected = + "Failed to connect to host: redis://1.2.3.4:6379, error: Failed to connect to any host resolved for DNS name."; + assertEquals(expected, got.getMessage()); + } + + @Test + public void givenCustomTypeAndCoder_setex_doesNotCorruptData() + throws IOException, UserCodeExecutionException { + + String key = UUID.randomUUID().toString(); + StringUtf8Coder keyCoder = StringUtf8Coder.of(); + + AllPrimitiveDataTypes value = + allPrimitiveDataTypes(true, BigDecimal.ONE, 1.23456, 1.23456f, 1, 1L, "🦄🦄🦄🦄"); + SerializableCoder<@NonNull AllPrimitiveDataTypes> valueCoder = + SerializableCoder.of(AllPrimitiveDataTypes.class); + + ByteArrayOutputStream keyBuffer = new ByteArrayOutputStream(); + keyCoder.encode(key, keyBuffer); + ByteArrayOutputStream valueBuffer = new ByteArrayOutputStream(); + valueCoder.encode(value, valueBuffer); + + byte[] keyBytes = keyBuffer.toByteArray(); + + externalClients + .getActualClient() + .setex(keyBytes, valueBuffer.toByteArray(), Duration.standardHours(1L)); + byte[] storedValueBytes = externalClients.getValidatingClient().get(keyBytes); + AllPrimitiveDataTypes storedValue = + valueCoder.decode(new ByteArrayInputStream(storedValueBytes)); + + assertEquals(value, storedValue); + } + + @Test + public void setex_expiresDataWhenExpected() + throws UserCodeExecutionException, InterruptedException { + Duration expiry = Duration.standardSeconds(2L); + String key = UUID.randomUUID().toString(); + byte[] keyBytes = key.getBytes(StandardCharsets.UTF_8); + externalClients.getActualClient().setex(keyBytes, keyBytes, expiry); + assertTrue(externalClients.getValidatingClient().exists(keyBytes)); + assertTrue(externalClients.getValidatingClient().ttl(keyBytes) > 0L); + Thread.sleep(expiry.getMillis()); + assertFalse(externalClients.getValidatingClient().exists(keyBytes)); + } + + @Test + public void givenCustomTypeAndCoder_rpush_doesNotCorruptData() + throws IOException, UserCodeExecutionException { + String key = UUID.randomUUID().toString(); + + AllPrimitiveDataTypes value = + allPrimitiveDataTypes(true, BigDecimal.ONE, 1.23456, 1.23456f, 1, 1L, "🦄🦄🦄🦄"); + SerializableCoder<@NonNull AllPrimitiveDataTypes> valueCoder = + SerializableCoder.of(AllPrimitiveDataTypes.class); + + ByteArrayOutputStream valueBuffer = new ByteArrayOutputStream(); + valueCoder.encode(value, valueBuffer); + + assertEquals(0L, externalClients.getActualClient().llen(key)); + externalClients.getActualClient().rpush(key, valueBuffer.toByteArray()); + assertEquals(1L, externalClients.getActualClient().llen(key)); + + byte[] storedBytes = externalClients.getActualClient().lpop(key); + + AllPrimitiveDataTypes storedValue = valueCoder.decode(new ByteArrayInputStream(storedBytes)); + + assertEquals(value, storedValue); + assertEquals(0L, externalClients.getActualClient().llen(key)); + } + + @Test + public void rpushAndlPopYieldsFIFOOrder() throws UserCodeExecutionException { + String key = UUID.randomUUID().toString(); + List<String> want = ImmutableList.of("1", "2", "3", "4", "5"); + + for (String item : want) { + externalClients.getActualClient().rpush(key, item.getBytes(StandardCharsets.UTF_8)); + } + List<String> got = new ArrayList<>(); + while (!externalClients.getActualClient().isEmpty(key)) { + byte[] bytes = externalClients.getActualClient().lpop(key); + got.add(new String(bytes, StandardCharsets.UTF_8)); + } + + assertEquals(want, got); + } + + @Test + public void givenExpired_decr_yieldsNegativeOne_andNotExists() + throws InterruptedException, UserCodeExecutionException { + String key = UUID.randomUUID().toString(); + externalClients.getActualClient().setex(key, 100L, Duration.standardSeconds(1L)); + assertTrue(externalClients.getActualClient().exists(key)); + Thread.sleep(1500L); + assertFalse(externalClients.getActualClient().exists(key)); + assertEquals(-1L, externalClients.getActualClient().decr(key)); + assertEquals(-2L, externalClients.getActualClient().decr(key)); + assertEquals(-3L, externalClients.getActualClient().decr(key)); + + key = UUID.randomUUID().toString(); + externalClients.getActualClient().setex(key, -100L, Duration.standardSeconds(1L)); + assertTrue(externalClients.getActualClient().exists(key)); + Thread.sleep(1500L); + assertFalse(externalClients.getActualClient().exists(key)); + assertEquals(-1L, externalClients.getActualClient().decr(key)); + assertEquals(-2L, externalClients.getActualClient().decr(key)); + assertEquals(-3L, externalClients.getActualClient().decr(key)); + } + + @Test + public void setThenDecrThenIncr_yieldsExpectedValue() throws UserCodeExecutionException { + String key = UUID.randomUUID().toString(); + externalClients.getActualClient().setex(key, 100L, Duration.standardHours(1L)); + assertEquals(100L, externalClients.getActualClient().getLong(key)); + for (long i = 0; i < 100L; i++) { + externalClients.getActualClient().decr(key); + } + assertEquals(0L, externalClients.getActualClient().getLong(key)); + for (long i = 0; i < 100L; i++) { + externalClients.getActualClient().incr(key); + } + assertEquals(100L, externalClients.getActualClient().getLong(key)); + } + + @Test + public void givenKeyNotExists_getLong_yieldsZero() throws UserCodeExecutionException { + assertEquals(0L, externalClients.getActualClient().getLong(UUID.randomUUID().toString())); + } +} diff --git a/sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponse/RedisExternalResourcesRule.java b/sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponse/RedisExternalResourcesRule.java new file mode 100644 index 000000000000..5c77dde387b0 --- /dev/null +++ b/sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponse/RedisExternalResourcesRule.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.io.requestresponse; + +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; + +import java.net.URI; +import java.util.function.Supplier; +import org.checkerframework.checker.nullness.qual.MonotonicNonNull; +import org.checkerframework.checker.nullness.qual.NonNull; +import org.junit.rules.ExternalResource; +import redis.clients.jedis.JedisPooled; + +/** + * {@link org.junit.runners.JUnit4} {@link org.junit.Rule} for {@link JedisPooled} based clients. + */ +class RedisExternalResourcesRule extends ExternalResource { + + private final Supplier<URI> configurationSupplier; + + private @MonotonicNonNull JedisPooled validatingClient; + private @MonotonicNonNull RedisClient actualClient; + + RedisExternalResourcesRule(Supplier<URI> configurationSupplier) { + this.configurationSupplier = configurationSupplier; + } + + @Override + protected void before() throws Throwable { + URI uri = configurationSupplier.get(); + validatingClient = new JedisPooled(uri); + actualClient = new RedisClient(uri); + + validatingClient.ping(); + actualClient.setup(); + } + + @Override + protected void after() { + getValidatingClient().close(); + try { + getActualClient().teardown(); + } catch (UserCodeExecutionException e) { + throw new RuntimeException(e); + } + } + + @NonNull + JedisPooled getValidatingClient() { + return checkStateNotNull(validatingClient); + } + + public RedisClient getActualClient() { + return checkStateNotNull(actualClient); + } +} From 16d68c120fce1bed06bbf84c69996d21350ea8fc Mon Sep 17 00:00:00 2001 From: Jeff Kinard <35542536+Polber@users.noreply.github.com> Date: Thu, 26 Oct 2023 15:02:32 -0400 Subject: [PATCH 260/435] [yaml] Normalize JdbcIO (#28971) --- .../jdbc/JdbcReadSchemaTransformProvider.java | 13 +++++++ .../JdbcWriteSchemaTransformProvider.java | 39 ++++++++++++++++--- sdks/python/apache_beam/yaml/standard_io.yaml | 33 ++++++++++++++++ 3 files changed, 80 insertions(+), 5 deletions(-) diff --git a/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcReadSchemaTransformProvider.java b/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcReadSchemaTransformProvider.java index dbf12f35024a..3b504b1a90d4 100644 --- a/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcReadSchemaTransformProvider.java +++ b/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcReadSchemaTransformProvider.java @@ -38,6 +38,9 @@ * An implementation of {@link org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider} for * reading from JDBC connections using {@link org.apache.beam.sdk.io.jdbc.JdbcIO}. */ +@SuppressWarnings({ + "nullness" // TODO(https://github.com/apache/beam/issues/20497) +}) @AutoService(SchemaTransformProvider.class) public class JdbcReadSchemaTransformProvider extends TypedSchemaTransformProvider< @@ -80,6 +83,11 @@ protected JdbcIO.DataSourceConfiguration dataSourceConfiguration() { dsConfig = dsConfig.withConnectionInitSqls(initialSql); } + String driverJars = config.getDriverJars(); + if (driverJars != null) { + dsConfig = dsConfig.withDriverJars(config.getDriverJars()); + } + return dsConfig; } @@ -152,6 +160,9 @@ public abstract static class JdbcReadSchemaTransformConfiguration implements Ser @Nullable public abstract Boolean getOutputParallelization(); + @Nullable + public abstract String getDriverJars(); + public void validate() throws IllegalArgumentException { if (Strings.isNullOrEmpty(getDriverClassName())) { throw new IllegalArgumentException("JDBC Driver class name cannot be blank."); @@ -199,6 +210,8 @@ public abstract static class Builder { public abstract Builder setOutputParallelization(Boolean value); + public abstract Builder setDriverJars(String value); + public abstract JdbcReadSchemaTransformConfiguration build(); } } diff --git a/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcWriteSchemaTransformProvider.java b/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcWriteSchemaTransformProvider.java index cb9d79631ca8..e9f67969626e 100644 --- a/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcWriteSchemaTransformProvider.java +++ b/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcWriteSchemaTransformProvider.java @@ -29,6 +29,9 @@ import org.apache.beam.sdk.schemas.transforms.SchemaTransform; import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider; import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionRowTuple; import org.apache.beam.sdk.values.Row; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Strings; @@ -40,6 +43,9 @@ * An implementation of {@link org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider} for * writing to a JDBC connections using {@link org.apache.beam.sdk.io.jdbc.JdbcIO}. */ +@SuppressWarnings({ + "nullness" // TODO(https://github.com/apache/beam/issues/20497) +}) @AutoService(SchemaTransformProvider.class) public class JdbcWriteSchemaTransformProvider extends TypedSchemaTransformProvider< @@ -82,6 +88,11 @@ protected JdbcIO.DataSourceConfiguration dataSourceConfiguration() { dsConfig = dsConfig.withConnectionInitSqls(initialSql); } + String driverJars = config.getDriverJars(); + if (driverJars != null) { + dsConfig = dsConfig.withDriverJars(config.getDriverJars()); + } + return dsConfig; } @@ -92,7 +103,9 @@ protected String writeStatement(Schema schema) { } else { StringBuilder statement = new StringBuilder("INSERT INTO "); statement.append(config.getLocation()); - statement.append(" VALUES("); + statement.append(" ("); + statement.append(String.join(", ", schema.getFieldNames())); + statement.append(") VALUES("); for (int i = 0; i < schema.getFieldCount() - 1; i++) { statement.append("?, "); } @@ -101,19 +114,30 @@ protected String writeStatement(Schema schema) { } } + private static class NoOutputDoFn<T> extends DoFn<T, Row> { + @ProcessElement + public void process(ProcessContext c) {} + } + @Override public PCollectionRowTuple expand(PCollectionRowTuple input) { - JdbcIO.Write<Row> writeRows = + JdbcIO.WriteVoid<Row> writeRows = JdbcIO.<Row>write() .withDataSourceConfiguration(dataSourceConfiguration()) .withStatement(writeStatement(input.get("input").getSchema())) - .withPreparedStatementSetter(new JdbcUtil.BeamRowPreparedStatementSetter()); + .withPreparedStatementSetter(new JdbcUtil.BeamRowPreparedStatementSetter()) + .withResults(); Boolean autosharding = config.getAutosharding(); if (autosharding != null && autosharding) { writeRows = writeRows.withAutoSharding(); } - input.get("input").apply(writeRows); - return PCollectionRowTuple.empty(input.getPipeline()); + PCollection<Row> postWrite = + input + .get("input") + .apply(writeRows) + .apply("post-write", ParDo.of(new NoOutputDoFn<>())) + .setRowSchema(Schema.of()); + return PCollectionRowTuple.of("post_write", postWrite); } } @@ -164,6 +188,9 @@ public abstract static class JdbcWriteSchemaTransformConfiguration implements Se @Nullable public abstract Boolean getAutosharding(); + @Nullable + public abstract String getDriverJars(); + public void validate() throws IllegalArgumentException { if (Strings.isNullOrEmpty(getDriverClassName())) { throw new IllegalArgumentException("JDBC Driver class name cannot be blank."); @@ -211,6 +238,8 @@ public abstract Builder setConnectionInitSql( public abstract Builder setAutosharding(Boolean value); + public abstract Builder setDriverJars(String value); + public abstract JdbcWriteSchemaTransformConfiguration build(); } } diff --git a/sdks/python/apache_beam/yaml/standard_io.yaml b/sdks/python/apache_beam/yaml/standard_io.yaml index c4748483b04b..d796ecc28bc7 100644 --- a/sdks/python/apache_beam/yaml/standard_io.yaml +++ b/sdks/python/apache_beam/yaml/standard_io.yaml @@ -112,3 +112,36 @@ 'WriteToJson': 'beam:schematransform:org.apache.beam:json_write:v1' config: gradle_target: 'sdks:java:extensions:schemaio-expansion-service:shadowJar' + +- type: renaming + transforms: + 'ReadFromJdbc': 'ReadFromJdbc' + 'WriteToJdbc': 'WriteToJdbc' + config: + mappings: + 'ReadFromJdbc': + driver_class_name: 'driverClassName' + url: 'jdbcUrl' + username: 'username' + password: 'password' + table: 'location' + query: 'readQuery' + driver_jars: 'driverJars' + connection_properties: 'connectionProperties' + connection_init_sql: 'connectionInitSql' + 'WriteToJdbc': + driver_class_name: 'driverClassName' + url: 'jdbcUrl' + username: 'username' + password: 'password' + table: 'location' + driver_jars: 'driverJars' + connection_properties: 'connectionProperties' + connection_init_sql: 'connectionInitSql' + underlying_provider: + type: beamJar + transforms: + 'ReadFromJdbc': 'beam:schematransform:org.apache.beam:jdbc_read:v1' + 'WriteToJdbc': 'beam:schematransform:org.apache.beam:jdbc_write:v1' + config: + gradle_target: 'sdks:java:extensions:schemaio-expansion-service:shadowJar' From 32d1cc0a3816a0dd2164a91ecceaeabf3e921b86 Mon Sep 17 00:00:00 2001 From: Yi Hu <yathu@google.com> Date: Thu, 26 Oct 2023 16:30:33 -0400 Subject: [PATCH 261/435] Bump Dataflow dev container versions to beam-master-20231026 (#29153) --- runners/google-cloud-dataflow-java/build.gradle | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/runners/google-cloud-dataflow-java/build.gradle b/runners/google-cloud-dataflow-java/build.gradle index 9e4d28ae5b4b..69759fc76a6f 100644 --- a/runners/google-cloud-dataflow-java/build.gradle +++ b/runners/google-cloud-dataflow-java/build.gradle @@ -51,8 +51,8 @@ evaluationDependsOn(":sdks:java:container:java11") ext.dataflowLegacyEnvironmentMajorVersion = '8' ext.dataflowFnapiEnvironmentMajorVersion = '8' -ext.dataflowLegacyContainerVersion = 'beam-master-20230809' -ext.dataflowFnapiContainerVersion = 'beam-master-20230809' +ext.dataflowLegacyContainerVersion = 'beam-master-20231026' +ext.dataflowFnapiContainerVersion = 'beam-master-20231026' ext.dataflowContainerBaseRepository = 'gcr.io/cloud-dataflow/v1beta3' processResources { From f4574bc9721e82192e81bdb1f5e97c4b83d20fbc Mon Sep 17 00:00:00 2001 From: Yi Hu <yathu@google.com> Date: Thu, 26 Oct 2023 17:22:48 -0400 Subject: [PATCH 262/435] Fix cleanup dataproc cluster name pattern (#29154) --- .test-infra/dataproc/cleanup.sh | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/.test-infra/dataproc/cleanup.sh b/.test-infra/dataproc/cleanup.sh index e535fabb7776..3efb80d8613f 100755 --- a/.test-infra/dataproc/cleanup.sh +++ b/.test-infra/dataproc/cleanup.sh @@ -20,10 +20,7 @@ clustersList=( ) toDeleteList=( ) -generatedResources=("beam-loadtests-go-cogbk-flink" "beam-loadtests-python-cogbk-flink" \ -"beam-loadtests-go-combine-flink" "beam-loadtests-python-combine-flink" \ -"beam-loadtests-go-gbk-flink" "beam-loadtests-python-gbk-flink" \ -"beam-loadtests-go-pardo-flink" "beam-loadtests-python-pardo-flink" \ +generatedResources=("beam-loadtests-go-*-flink" "beam-loadtests-python-*-flink" \ "beam-postcommit-python-chicago" ) function deleteFilteredClusters(){ @@ -51,7 +48,7 @@ function filterClusters(){ if [[ $elapsedHours -ge 2 ]]; then for name in ${generatedResources[@]}; do # Only resources generated by the groovy jobs set are queued for deletion - if [[ "$cluster" == *"$name"* ]]; then + if [[ "$cluster" == *${name}* ]]; then toDeleteList+=( "$cluster" ) break fi From 3b890399e11d65695d04ae679e2fdf56555630a8 Mon Sep 17 00:00:00 2001 From: liferoad <huxiangqian@gmail.com> Date: Thu, 26 Oct 2023 22:14:58 -0400 Subject: [PATCH 263/435] Automatically switch to upload_graph when the graph is large (#28621) * Automatically Switch to upload_graph when the graph is large * fix the formats * updated the CHANGES.md * added one test * fixed the styles * fixed the styles * fixed the styles * addressed the comments --- CHANGES.md | 1 + .../beam/runners/dataflow/DataflowRunner.java | 24 +++++++++---- .../runners/dataflow/DataflowRunnerTest.java | 36 ++++++++++++++++++- 3 files changed, 53 insertions(+), 8 deletions(-) diff --git a/CHANGES.md b/CHANGES.md index 49b4fdfe89b0..a97035fdc297 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -71,6 +71,7 @@ should handle this. ([#25252](https://github.com/apache/beam/issues/25252)). ## New Features / Improvements * X feature added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). +* `upload_graph` as one of the Experiments options for DataflowRunner is no longer required when the graph is larger than 10MB for Java SDK ([PR#28621](https://github.com/apache/beam/pull/28621). ## Breaking Changes diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java index 891b4c0454c9..80b4e4cfd8b5 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java @@ -168,7 +168,6 @@ import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Joiner; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Strings; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Utf8; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; @@ -1330,15 +1329,26 @@ public DataflowPipelineJob run(Pipeline pipeline) { hooks.modifyEnvironmentBeforeSubmission(newJob.getEnvironment()); } + // enable upload_graph when the graph is too large + byte[] jobGraphBytes = DataflowPipelineTranslator.jobToString(newJob).getBytes(UTF_8); + int jobGraphByteSize = jobGraphBytes.length; + if (jobGraphByteSize >= CREATE_JOB_REQUEST_LIMIT_BYTES + && !hasExperiment(options, "upload_graph")) { + List<String> experiments = firstNonNull(options.getExperiments(), new ArrayList<>()); + experiments.add("upload_graph"); + options.setExperiments(ImmutableList.copyOf(experiments)); + LOG.info( + "The job graph size ({} in bytes) is larger than {}. Automatically add " + + "the upload_graph option to experiments.", + jobGraphByteSize, + CREATE_JOB_REQUEST_LIMIT_BYTES); + } + // Upload the job to GCS and remove the graph object from the API call. The graph // will be downloaded from GCS by the service. if (hasExperiment(options, "upload_graph")) { DataflowPackage stagedGraph = - options - .getStager() - .stageToFile( - DataflowPipelineTranslator.jobToString(newJob).getBytes(UTF_8), - DATAFLOW_GRAPH_FILE_NAME); + options.getStager().stageToFile(jobGraphBytes, DATAFLOW_GRAPH_FILE_NAME); newJob.getSteps().clear(); newJob.setStepsLocation(stagedGraph.getLocation()); } @@ -1398,7 +1408,7 @@ public DataflowPipelineJob run(Pipeline pipeline) { } catch (GoogleJsonResponseException e) { String errorMessages = "Unexpected errors"; if (e.getDetails() != null) { - if (Utf8.encodedLength(newJob.toString()) >= CREATE_JOB_REQUEST_LIMIT_BYTES) { + if (jobGraphByteSize >= CREATE_JOB_REQUEST_LIMIT_BYTES) { errorMessages = "The size of the serialized JSON representation of the pipeline " + "exceeds the allowable limit. " diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java index 078f25e0e38e..bcdea03dba2c 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java @@ -242,7 +242,7 @@ public void setUp() throws IOException { mockJobs = mock(Dataflow.Projects.Locations.Jobs.class); } - private Pipeline buildDataflowPipeline(DataflowPipelineOptions options) { + private static Pipeline buildDataflowPipeline(DataflowPipelineOptions options) { options.setStableUniqueNames(CheckEnabled.ERROR); options.setRunner(DataflowRunner.class); Pipeline p = Pipeline.create(options); @@ -256,6 +256,22 @@ private Pipeline buildDataflowPipeline(DataflowPipelineOptions options) { return p; } + private static Pipeline buildDataflowPipelineWithLargeGraph(DataflowPipelineOptions options) { + options.setStableUniqueNames(CheckEnabled.ERROR); + options.setRunner(DataflowRunner.class); + Pipeline p = Pipeline.create(options); + + for (int i = 0; i < 100; i++) { + p.apply("ReadMyFile_" + i, TextIO.read().from("gs://bucket/object")) + .apply("WriteMyFile_" + i, TextIO.write().to("gs://bucket/object")); + } + + // Enable the FileSystems API to know about gs:// URIs in this test. + FileSystems.setDefaultPipelineOptions(options); + + return p; + } + private static Dataflow buildMockDataflow(Dataflow.Projects.Locations.Jobs mockJobs) throws IOException { Dataflow mockDataflowClient = mock(Dataflow.class); @@ -824,6 +840,24 @@ public void testUploadGraph() throws IOException { .startsWith("gs://valid-bucket/temp/staging/dataflow_graph")); } + /** Test for automatically using upload_graph when the job graph is too large (>10MB). */ + @Test + public void testUploadGraphWithAutoUpload() throws IOException { + DataflowPipelineOptions options = buildPipelineOptions(); + Pipeline p = buildDataflowPipelineWithLargeGraph(options); + p.run(); + + ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class); + Mockito.verify(mockJobs).create(eq(PROJECT_ID), eq(REGION_ID), jobCaptor.capture()); + assertValidJob(jobCaptor.getValue()); + assertTrue(jobCaptor.getValue().getSteps().isEmpty()); + assertTrue( + jobCaptor + .getValue() + .getStepsLocation() + .startsWith("gs://valid-bucket/temp/staging/dataflow_graph")); + } + @Test public void testUpdateNonExistentPipeline() throws IOException { thrown.expect(IllegalArgumentException.class); From e98e37f15b898c55b63af680aff33f528d2ede38 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ferran=20Fern=C3=A1ndez=20Garrido?= <ffernandez.upc@gmail.com> Date: Fri, 27 Oct 2023 06:46:38 +0200 Subject: [PATCH 264/435] [YAML] Kafka Read Provider (#28865) * [YAML] Kafka provider --- ...KafkaReadSchemaTransformConfiguration.java | 2 +- .../KafkaReadSchemaTransformProvider.java | 54 ++++++++++++++++++- .../KafkaReadSchemaTransformProviderTest.java | 18 +++++++ sdks/python/apache_beam/yaml/standard_io.yaml | 21 ++++++++ 4 files changed, 92 insertions(+), 3 deletions(-) diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaReadSchemaTransformConfiguration.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaReadSchemaTransformConfiguration.java index f7e091580680..299a4f222e3a 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaReadSchemaTransformConfiguration.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaReadSchemaTransformConfiguration.java @@ -39,7 +39,7 @@ public abstract class KafkaReadSchemaTransformConfiguration { public static final Set<String> VALID_START_OFFSET_VALUES = Sets.newHashSet("earliest", "latest"); - public static final String VALID_FORMATS_STR = "AVRO,JSON"; + public static final String VALID_FORMATS_STR = "RAW,AVRO,JSON"; public static final Set<String> VALID_DATA_FORMATS = Sets.newHashSet(VALID_FORMATS_STR.split(",")); diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaReadSchemaTransformProvider.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaReadSchemaTransformProvider.java index 0c091bf9ba84..fcba5c355df1 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaReadSchemaTransformProvider.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaReadSchemaTransformProvider.java @@ -49,6 +49,7 @@ import org.apache.beam.sdk.transforms.DoFn.ProcessElement; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.transforms.SimpleFunction; import org.apache.beam.sdk.transforms.Values; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionRowTuple; @@ -112,16 +113,56 @@ protected SchemaTransform from(KafkaReadSchemaTransformConfiguration configurati consumerConfigs.put(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG, 100); consumerConfigs.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, autoOffsetReset); + String format = configuration.getFormat(); + + if (format != null && format.equals("RAW")) { + if (inputSchema != null) { + throw new IllegalArgumentException( + "To read from Kafka in RAW format, you can't provide a schema."); + } + Schema rawSchema = Schema.builder().addField("payload", Schema.FieldType.BYTES).build(); + SerializableFunction<byte[], Row> valueMapper = getRawBytesToRowFunction(rawSchema); + return new SchemaTransform() { + @Override + public PCollectionRowTuple expand(PCollectionRowTuple input) { + KafkaIO.Read<byte[], byte[]> kafkaRead = + KafkaIO.readBytes() + .withConsumerConfigUpdates(consumerConfigs) + .withConsumerFactoryFn(new ConsumerFactoryWithGcsTrustStores()) + .withTopic(configuration.getTopic()) + .withBootstrapServers(configuration.getBootstrapServers()); + if (isTest) { + kafkaRead = kafkaRead.withMaxReadTime(Duration.standardSeconds(testTimeoutSecs)); + } + + PCollection<byte[]> kafkaValues = + input.getPipeline().apply(kafkaRead.withoutMetadata()).apply(Values.create()); + + PCollectionTuple outputTuple = + kafkaValues.apply( + ParDo.of(new ErrorFn("Kafka-read-error-counter", valueMapper)) + .withOutputTags(OUTPUT_TAG, TupleTagList.of(ERROR_TAG))); + + return PCollectionRowTuple.of( + "output", + outputTuple.get(OUTPUT_TAG).setRowSchema(rawSchema), + "errors", + outputTuple.get(ERROR_TAG).setRowSchema(ERROR_SCHEMA)); + } + }; + } + if (inputSchema != null && !inputSchema.isEmpty()) { assert Strings.isNullOrEmpty(configuration.getConfluentSchemaRegistryUrl()) : "To read from Kafka, a schema must be provided directly or though Confluent " + "Schema Registry, but not both."; + final Schema beamSchema = - Objects.equals(configuration.getFormat(), "JSON") + Objects.equals(format, "JSON") ? JsonUtils.beamSchemaFromJsonSchema(inputSchema) : AvroUtils.toBeamSchema(new org.apache.avro.Schema.Parser().parse(inputSchema)); SerializableFunction<byte[], Row> valueMapper = - Objects.equals(configuration.getFormat(), "JSON") + Objects.equals(format, "JSON") ? JsonUtils.getJsonBytesToRowFunction(beamSchema) : AvroUtils.getAvroBytesToRowFunction(beamSchema); return new SchemaTransform() { @@ -193,6 +234,15 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) { } } + public static SerializableFunction<byte[], Row> getRawBytesToRowFunction(Schema rawSchema) { + return new SimpleFunction<byte[], Row>() { + @Override + public Row apply(byte[] input) { + return Row.withSchema(rawSchema).addValue(input).build(); + } + }; + } + @Override public String identifier() { return "beam:schematransform:org.apache.beam:kafka_read:v1"; diff --git a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaReadSchemaTransformProviderTest.java b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaReadSchemaTransformProviderTest.java index 6b9dde4dc952..367f0f7a535e 100644 --- a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaReadSchemaTransformProviderTest.java +++ b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaReadSchemaTransformProviderTest.java @@ -147,4 +147,22 @@ public void testBuildTransformWithJsonSchema() throws IOException { StandardCharsets.UTF_8)) .build()); } + + @Test + public void testBuildTransformWithRawFormat() throws IOException { + ServiceLoader<SchemaTransformProvider> serviceLoader = + ServiceLoader.load(SchemaTransformProvider.class); + List<SchemaTransformProvider> providers = + StreamSupport.stream(serviceLoader.spliterator(), false) + .filter(provider -> provider.getClass() == KafkaReadSchemaTransformProvider.class) + .collect(Collectors.toList()); + KafkaReadSchemaTransformProvider kafkaProvider = + (KafkaReadSchemaTransformProvider) providers.get(0); + kafkaProvider.from( + KafkaReadSchemaTransformConfiguration.builder() + .setTopic("anytopic") + .setBootstrapServers("anybootstrap") + .setFormat("RAW") + .build()); + } } diff --git a/sdks/python/apache_beam/yaml/standard_io.yaml b/sdks/python/apache_beam/yaml/standard_io.yaml index d796ecc28bc7..6499ffc7e55a 100644 --- a/sdks/python/apache_beam/yaml/standard_io.yaml +++ b/sdks/python/apache_beam/yaml/standard_io.yaml @@ -46,6 +46,27 @@ config: gradle_target: 'sdks:java:extensions:sql:expansion-service:shadowJar' +- type: renaming + transforms: + 'ReadFromKafka': 'ReadFromKafka' + config: + mappings: + 'ReadFromKafka': + 'schema': 'schema' + 'consumer_config': 'consumerConfigUpdates' + 'format': 'format' + 'topic': 'topic' + 'bootstrap_servers': 'bootstrapServers' + 'confluent_schema_registry_url': 'confluentSchemaRegistryUrl' + 'confluent_schema_registry_subject': 'confluentSchemaRegistrySubject' + 'auto_offset_reset_config': 'autoOffsetResetConfig' + underlying_provider: + type: beamJar + transforms: + 'ReadFromKafka': 'beam:schematransform:org.apache.beam:kafka_read:v1' + config: + gradle_target: 'sdks:java:io:expansion-service:shadowJar' + - type: python transforms: 'ReadFromBigQuery': 'apache_beam.yaml.yaml_io.read_from_bigquery' From ca21948d64d78902deebc16c00e5915c1a480877 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ferran=20Fern=C3=A1ndez=20Garrido?= <ffernandez.upc@gmail.com> Date: Fri, 27 Oct 2023 12:24:17 +0200 Subject: [PATCH 265/435] [YAML] - Kafka write and RAW format --- .../KafkaWriteSchemaTransformProvider.java | 36 ++++++++++++--- ...KafkaWriteSchemaTransformProviderTest.java | 44 +++++++++++++++++++ sdks/python/apache_beam/yaml/standard_io.yaml | 7 +++ 3 files changed, 81 insertions(+), 6 deletions(-) diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaWriteSchemaTransformProvider.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaWriteSchemaTransformProvider.java index 876ef9a49e8a..46a4d078f472 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaWriteSchemaTransformProvider.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaWriteSchemaTransformProvider.java @@ -41,6 +41,7 @@ import org.apache.beam.sdk.transforms.DoFn.ProcessElement; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.transforms.SimpleFunction; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionRowTuple; import org.apache.beam.sdk.values.PCollectionTuple; @@ -60,7 +61,7 @@ public class KafkaWriteSchemaTransformProvider extends TypedSchemaTransformProvider< KafkaWriteSchemaTransformProvider.KafkaWriteSchemaTransformConfiguration> { - public static final String SUPPORTED_FORMATS_STR = "JSON,AVRO"; + public static final String SUPPORTED_FORMATS_STR = "RAW,JSON,AVRO"; public static final Set<String> SUPPORTED_FORMATS = Sets.newHashSet(SUPPORTED_FORMATS_STR.split(",")); public static final TupleTag<Row> ERROR_TAG = new TupleTag<Row>() {}; @@ -71,6 +72,8 @@ public class KafkaWriteSchemaTransformProvider private static final Logger LOG = LoggerFactory.getLogger(KafkaWriteSchemaTransformProvider.class); + private static final String PAYLOAD = "payload"; + @Override protected @UnknownKeyFor @NonNull @Initialized Class<KafkaWriteSchemaTransformConfiguration> configurationClass() { @@ -131,10 +134,18 @@ public void finish() { @Override public PCollectionRowTuple expand(PCollectionRowTuple input) { Schema inputSchema = input.get("input").getSchema(); - final SerializableFunction<Row, byte[]> toBytesFn = - configuration.getFormat().equals("JSON") - ? JsonUtils.getRowToJsonBytesFunction(inputSchema) - : AvroUtils.getRowToAvroBytesFunction(inputSchema); + final SerializableFunction<Row, byte[]> toBytesFn; + if (configuration.getFormat().equals("RAW")) { + if (!inputSchema.hasField(PAYLOAD)) { + throw new IllegalArgumentException( + "To write to Kafka in RAW format, the input schema must provide the payload attribute."); + } + toBytesFn = getRowToRawBytesFunction(); + } else if (configuration.getFormat().equals("JSON")) { + toBytesFn = JsonUtils.getRowToJsonBytesFunction(inputSchema); + } else { + toBytesFn = AvroUtils.getRowToAvroBytesFunction(inputSchema); + } final Map<String, String> configOverrides = configuration.getProducerConfigUpdates(); PCollectionTuple outputTuple = @@ -163,6 +174,19 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) { } } + public static SerializableFunction<Row, byte[]> getRowToRawBytesFunction() { + return new SimpleFunction<Row, byte[]>() { + @Override + public byte[] apply(Row input) { + byte[] rawBytes = input.getBytes(PAYLOAD); + if (rawBytes == null) { + throw new NullPointerException(); + } + return rawBytes; + } + }; + } + @Override public @UnknownKeyFor @NonNull @Initialized String identifier() { return "beam:schematransform:org.apache.beam:kafka_write:v1"; @@ -193,7 +217,7 @@ public abstract static class KafkaWriteSchemaTransformConfiguration implements S @SchemaFieldDescription( "A list of host/port pairs to use for establishing the initial connection to the" + " Kafka cluster. The client will make use of all servers irrespective of which servers are specified" - + " here for bootstrapping—this list only impacts the initial hosts used to discover the full set" + + " here for bootstrapping—this list only impacts the initial hosts used to discover the full set" + " of servers. | Format: host1:port1,host2:port2,...") public abstract String getBootstrapServers(); diff --git a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaWriteSchemaTransformProviderTest.java b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaWriteSchemaTransformProviderTest.java index 8d01ebe8233c..279e67242a6d 100644 --- a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaWriteSchemaTransformProviderTest.java +++ b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaWriteSchemaTransformProviderTest.java @@ -17,6 +17,9 @@ */ package org.apache.beam.sdk.io.kafka; +import static org.apache.beam.sdk.io.kafka.KafkaWriteSchemaTransformProvider.getRowToRawBytesFunction; + +import java.io.UnsupportedEncodingException; import java.util.Arrays; import java.util.List; import org.apache.beam.sdk.io.kafka.KafkaWriteSchemaTransformProvider.KafkaWriteSchemaTransform.ErrorCounterFn; @@ -47,6 +50,9 @@ public class KafkaWriteSchemaTransformProviderTest { private static final Schema BEAMSCHEMA = Schema.of(Schema.Field.of("name", Schema.FieldType.STRING)); + + private static final Schema BEAMRAWSCHEMA = + Schema.of(Schema.Field.of("payload", Schema.FieldType.BYTES)); private static final Schema ERRORSCHEMA = KafkaWriteSchemaTransformProvider.ERROR_SCHEMA; private static final List<Row> ROWS = @@ -55,9 +61,27 @@ public class KafkaWriteSchemaTransformProviderTest { Row.withSchema(BEAMSCHEMA).withFieldValue("name", "b").build(), Row.withSchema(BEAMSCHEMA).withFieldValue("name", "c").build()); + private static final List<Row> RAW_ROWS; + + static { + try { + RAW_ROWS = + Arrays.asList( + Row.withSchema(BEAMRAWSCHEMA).withFieldValue("payload", "a".getBytes("UTF8")).build(), + Row.withSchema(BEAMRAWSCHEMA).withFieldValue("payload", "b".getBytes("UTF8")).build(), + Row.withSchema(BEAMRAWSCHEMA) + .withFieldValue("payload", "c".getBytes("UTF8")) + .build()); + } catch (UnsupportedEncodingException e) { + throw new RuntimeException(e); + } + } + final SerializableFunction<Row, byte[]> valueMapper = JsonUtils.getRowToJsonBytesFunction(BEAMSCHEMA); + final SerializableFunction<Row, byte[]> valueRawMapper = getRowToRawBytesFunction(); + @Rule public transient TestPipeline p = TestPipeline.create(); @Test @@ -79,4 +103,24 @@ public void testKafkaErrorFnSuccess() throws Exception { PAssert.that(output.get(OUTPUT_TAG)).containsInAnyOrder(msg); p.run().waitUntilFinish(); } + + @Test + public void testKafkaErrorFnRawSuccess() throws Exception { + List<KV<byte[], byte[]>> msg = + Arrays.asList( + KV.of(new byte[1], "a".getBytes("UTF8")), + KV.of(new byte[1], "b".getBytes("UTF8")), + KV.of(new byte[1], "c".getBytes("UTF8"))); + + PCollection<Row> input = p.apply(Create.of(RAW_ROWS)); + PCollectionTuple output = + input.apply( + ParDo.of(new ErrorCounterFn("Kafka-write-error-counter", valueRawMapper)) + .withOutputTags(OUTPUT_TAG, TupleTagList.of(ERROR_TAG))); + + output.get(ERROR_TAG).setRowSchema(ERRORSCHEMA); + + PAssert.that(output.get(OUTPUT_TAG)).containsInAnyOrder(msg); + p.run().waitUntilFinish(); + } } diff --git a/sdks/python/apache_beam/yaml/standard_io.yaml b/sdks/python/apache_beam/yaml/standard_io.yaml index 6499ffc7e55a..b8fe0660b127 100644 --- a/sdks/python/apache_beam/yaml/standard_io.yaml +++ b/sdks/python/apache_beam/yaml/standard_io.yaml @@ -49,6 +49,7 @@ - type: renaming transforms: 'ReadFromKafka': 'ReadFromKafka' + 'WriteToKafka': 'WriteToKafka' config: mappings: 'ReadFromKafka': @@ -60,10 +61,16 @@ 'confluent_schema_registry_url': 'confluentSchemaRegistryUrl' 'confluent_schema_registry_subject': 'confluentSchemaRegistrySubject' 'auto_offset_reset_config': 'autoOffsetResetConfig' + 'WriteToKafka': + 'format': 'format' + 'topic': 'topic' + 'bootstrap_servers': 'bootstrapServers' + 'producer_config_updates': 'ProducerConfigUpdates' underlying_provider: type: beamJar transforms: 'ReadFromKafka': 'beam:schematransform:org.apache.beam:kafka_read:v1' + 'WriteToKafka': 'beam:schematransform:org.apache.beam:kafka_write:v1' config: gradle_target: 'sdks:java:io:expansion-service:shadowJar' From 88fcfd5ac655b2f90f0116bced028a582eeaa762 Mon Sep 17 00:00:00 2001 From: Danny McCormick <dannymccormick@google.com> Date: Fri, 27 Oct 2023 08:16:13 -0400 Subject: [PATCH 266/435] First pass at sorting gha dashboard (#29151) --- .../GA-Post-Commits_status_dashboard.json | 12026 +++++++++++++++- .../metrics/sync/github/sync_workflows.py | 27 +- 2 files changed, 11885 insertions(+), 168 deletions(-) diff --git a/.test-infra/metrics/grafana/dashboards/GA-Post-Commits_status_dashboard.json b/.test-infra/metrics/grafana/dashboards/GA-Post-Commits_status_dashboard.json index 380b31dcd074..b576c0f03bc4 100644 --- a/.test-infra/metrics/grafana/dashboards/GA-Post-Commits_status_dashboard.json +++ b/.test-infra/metrics/grafana/dashboards/GA-Post-Commits_status_dashboard.json @@ -3,7 +3,10 @@ "list": [ { "builtIn": 1, - "datasource": "-- Grafana --", + "datasource": { + "type": "datasource", + "uid": "grafana" + }, "enable": true, "hide": true, "iconColor": "rgba(0, 211, 255, 1)", @@ -19,68 +22,59 @@ ] }, "editable": true, - "gnetId": null, + "fiscalYearStartMonth": 0, "graphTooltip": 0, + "id": 35, "links": [], + "liveNow": false, "panels": [ { - "id": 2, - "gridPos": { - "h": 32, - "w": 24, - "x": 0, - "y": 0 + "datasource": { + "type": "postgres", + "uid": "fK5onAOVz" }, - "type": "table", - "title": "GA Post-Commits Status", - "transformations": [ - { - "id": "merge", - "options": { - "reducers": [] - } - } - ], - "datasource": "BeamPSQL", - "pluginVersion": "9.2.0", - "links": [], "fieldConfig": { "defaults": { + "color": { + "mode": "thresholds" + }, "custom": { "align": "auto", - "displayMode": "auto", + "cellOptions": { + "type": "auto" + }, "inspect": false }, "mappings": [ { - "type": "value", "options": { "0": { + "color": "semi-dark-red", "index": 0, - "text": "Fail", - "color": "semi-dark-red" + "text": "Fail" }, "1": { + "color": "semi-dark-green", "index": 1, - "text": "Success", - "color": "semi-dark-green" + "text": "Success" }, "2": { + "color": "semi-dark-yellow", "index": 2, - "text": "Pending", - "color": "semi-dark-yellow" + "text": "Pending" }, "3": { + "color": "semi-dark-purple", "index": 3, - "text": "Cancelled", - "color": "semi-dark-purple" + "text": "Cancelled" }, "4": { + "color": "light-blue", "index": 4, - "text": "None", - "color": "light-blue" + "text": "None" } - } + }, + "type": "value" } ], "thresholds": { @@ -91,9 +85,6 @@ "value": null } ] - }, - "color": { - "mode": "thresholds" } }, "overrides": [ @@ -172,8 +163,8 @@ { "id": "color", "value": { - "mode": "fixed", - "fixedColor": "dark-green" + "fixedColor": "dark-green", + "mode": "fixed" } }, { @@ -194,27 +185,30 @@ "value": 0 }, { - "value": 1, - "color": "semi-dark-green" + "color": "semi-dark-green", + "value": 1 }, { - "value": 2, - "color": "semi-dark-yellow" + "color": "semi-dark-yellow", + "value": 2 }, { - "value": 3, - "color": "light-blue" + "color": "light-blue", + "value": 3 }, { - "value": 4, - "color": "purple" + "color": "purple", + "value": 4 } ] } }, { - "id": "custom.displayMode", - "value": "color-background-solid" + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } }, { "id": "custom.align", @@ -244,8 +238,8 @@ { "id": "color", "value": { - "mode": "fixed", - "fixedColor": "dark-green" + "fixedColor": "dark-green", + "mode": "fixed" } }, { @@ -266,27 +260,30 @@ "value": 0 }, { - "value": 1, - "color": "semi-dark-green" + "color": "semi-dark-green", + "value": 1 }, { - "value": 2, - "color": "semi-dark-yellow" + "color": "semi-dark-yellow", + "value": 2 }, { - "value": 3, - "color": "light-blue" + "color": "light-blue", + "value": 3 }, { - "value": 4, - "color": "purple" + "color": "purple", + "value": 4 } ] } }, { - "id": "custom.displayMode", - "value": "color-background-solid" + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } }, { "id": "custom.align", @@ -316,8 +313,8 @@ { "id": "color", "value": { - "mode": "fixed", - "fixedColor": "dark-green" + "fixedColor": "dark-green", + "mode": "fixed" } }, { @@ -338,27 +335,30 @@ "value": 0 }, { - "value": 1, - "color": "semi-dark-green" + "color": "semi-dark-green", + "value": 1 }, { - "value": 2, - "color": "semi-dark-yellow" + "color": "semi-dark-yellow", + "value": 2 }, { - "value": 3, - "color": "light-blue" + "color": "light-blue", + "value": 3 }, { - "value": 4, - "color": "purple" + "color": "purple", + "value": 4 } ] } }, { - "id": "custom.displayMode", - "value": "color-background-solid" + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } }, { "id": "custom.align", @@ -388,8 +388,8 @@ { "id": "color", "value": { - "mode": "fixed", - "fixedColor": "dark-green" + "fixedColor": "dark-green", + "mode": "fixed" } }, { @@ -410,27 +410,30 @@ "value": 0 }, { - "value": 1, - "color": "semi-dark-green" + "color": "semi-dark-green", + "value": 1 }, { - "value": 2, - "color": "semi-dark-yellow" + "color": "semi-dark-yellow", + "value": 2 }, { - "value": 3, - "color": "light-blue" + "color": "light-blue", + "value": 3 }, { - "value": 4, - "color": "purple" + "color": "purple", + "value": 4 } ] } }, { - "id": "custom.displayMode", - "value": "color-background-solid" + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } }, { "id": "custom.align", @@ -460,8 +463,8 @@ { "id": "color", "value": { - "mode": "fixed", - "fixedColor": "dark-green" + "fixedColor": "dark-green", + "mode": "fixed" } }, { @@ -482,27 +485,30 @@ "value": 0 }, { - "value": 1, - "color": "semi-dark-green" + "color": "semi-dark-green", + "value": 1 }, { - "value": 2, - "color": "semi-dark-yellow" + "color": "semi-dark-yellow", + "value": 2 }, { - "value": 3, - "color": "light-blue" + "color": "light-blue", + "value": 3 }, { - "value": 4, - "color": "purple" + "color": "purple", + "value": 4 } ] } }, { - "id": "custom.displayMode", - "value": "color-background-solid" + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } }, { "id": "custom.align", @@ -532,8 +538,8 @@ { "id": "color", "value": { - "mode": "fixed", - "fixedColor": "dark-green" + "fixedColor": "dark-green", + "mode": "fixed" } }, { @@ -554,27 +560,30 @@ "value": 0 }, { - "value": 1, - "color": "semi-dark-green" + "color": "semi-dark-green", + "value": 1 }, { - "value": 2, - "color": "semi-dark-yellow" + "color": "semi-dark-yellow", + "value": 2 }, { - "value": 3, - "color": "light-blue" + "color": "light-blue", + "value": 3 }, { - "value": 4, - "color": "purple" + "color": "purple", + "value": 4 } ] } }, { - "id": "custom.displayMode", - "value": "color-background-solid" + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } }, { "id": "custom.align", @@ -604,8 +613,8 @@ { "id": "color", "value": { - "mode": "fixed", - "fixedColor": "dark-green" + "fixedColor": "dark-green", + "mode": "fixed" } }, { @@ -626,27 +635,30 @@ "value": 0 }, { - "value": 1, - "color": "semi-dark-green" + "color": "semi-dark-green", + "value": 1 }, { - "value": 2, - "color": "semi-dark-yellow" + "color": "semi-dark-yellow", + "value": 2 }, { - "value": 3, - "color": "light-blue" + "color": "light-blue", + "value": 3 }, { - "value": 4, - "color": "purple" + "color": "purple", + "value": 4 } ] } }, { - "id": "custom.displayMode", - "value": "color-background-solid" + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } }, { "id": "custom.align", @@ -676,8 +688,8 @@ { "id": "color", "value": { - "mode": "fixed", - "fixedColor": "dark-green" + "fixedColor": "dark-green", + "mode": "fixed" } }, { @@ -698,27 +710,30 @@ "value": 0 }, { - "value": 1, - "color": "semi-dark-green" + "color": "semi-dark-green", + "value": 1 }, { - "value": 2, - "color": "semi-dark-yellow" + "color": "semi-dark-yellow", + "value": 2 }, { - "value": 3, - "color": "light-blue" + "color": "light-blue", + "value": 3 }, { - "value": 4, - "color": "purple" + "color": "purple", + "value": 4 } ] } }, { - "id": "custom.displayMode", - "value": "color-background-solid" + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } }, { "id": "custom.align", @@ -748,8 +763,8 @@ { "id": "color", "value": { - "mode": "fixed", - "fixedColor": "dark-green" + "fixedColor": "dark-green", + "mode": "fixed" } }, { @@ -770,27 +785,30 @@ "value": 0 }, { - "value": 1, - "color": "semi-dark-green" + "color": "semi-dark-green", + "value": 1 }, { - "value": 2, - "color": "semi-dark-yellow" + "color": "semi-dark-yellow", + "value": 2 }, { - "value": 3, - "color": "light-blue" + "color": "light-blue", + "value": 3 }, { - "value": 4, - "color": "purple" + "color": "purple", + "value": 4 } ] } }, { - "id": "custom.displayMode", - "value": "color-background-solid" + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } }, { "id": "custom.align", @@ -820,8 +838,8 @@ { "id": "color", "value": { - "mode": "fixed", - "fixedColor": "dark-green" + "fixedColor": "dark-green", + "mode": "fixed" } }, { @@ -842,27 +860,30 @@ "value": 0 }, { - "value": 1, - "color": "semi-dark-green" + "color": "semi-dark-green", + "value": 1 }, { - "value": 2, - "color": "semi-dark-yellow" + "color": "semi-dark-yellow", + "value": 2 }, { - "value": 3, - "color": "light-blue" + "color": "light-blue", + "value": 3 }, { - "value": 4, - "color": "purple" + "color": "purple", + "value": 4 } ] } }, { - "id": "custom.displayMode", - "value": "color-background-solid" + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } }, { "id": "custom.align", @@ -885,30 +906,46 @@ } ] }, + "gridPos": { + "h": 20, + "w": 24, + "x": 0, + "y": 0 + }, + "id": 2, + "links": [], "options": { - "showHeader": true, + "cellHeight": "sm", "footer": { - "show": false, + "countRows": false, + "fields": "", "reducer": [ "sum" ], - "fields": "" - } + "show": false + }, + "showHeader": true }, + "pluginVersion": "10.0.3", "targets": [ { "aggregation": "Last", "alias": "job", + "datasource": { + "type": "postgres", + "uid": "fK5onAOVz" + }, "decimals": 2, "displayAliasType": "Warning / Critical", "displayType": "Regular", "displayValueWithAlias": "Never", + "editorMode": "code", "format": "table", "group": [], "hide": false, "metricColumn": "none", "rawQuery": true, - "rawSql": "with workflows as (\n select\n *,\n case when run1 like 'success' then 1 when run1 like 'in_progress' then 2 when run1 like 'queued' then 2 when run1 like 'waiting' then 2 when run1 like 'cancelled' then 3 when run1 like 'None' then 4 else 0 end as run_1,\n case when run2 like 'success' then 1 when run2 like 'in_progress' then 2 when run2 like 'queued' then 2 when run2 like 'waiting' then 2 when run2 like 'cancelled' then 3 when run2 like 'None' then 4 else 0 end as run_2,\n case when run3 like 'success' then 1 when run3 like 'in_progress' then 2 when run3 like 'queued' then 2 when run3 like 'waiting' then 2 when run3 like 'cancelled' then 3 when run3 like 'None' then 4 else 0 end as run_3,\n case when run4 like 'success' then 1 when run4 like 'in_progress' then 2 when run4 like 'queued' then 2 when run4 like 'waiting' then 2 when run4 like 'cancelled' then 3 when run4 like 'None' then 4 else 0 end as run_4,\n case when run5 like 'success' then 1 when run5 like 'in_progress' then 2 when run5 like 'queued' then 2 when run5 like 'waiting' then 2 when run5 like 'cancelled' then 3 when run5 like 'None' then 4 else 0 end as run_5,\n case when run6 like 'success' then 1 when run6 like 'in_progress' then 2 when run6 like 'queued' then 2 when run6 like 'waiting' then 2 when run6 like 'cancelled' then 3 when run6 like 'None' then 4 else 0 end as run_6,\n case when run7 like 'success' then 1 when run7 like 'in_progress' then 2 when run7 like 'queued' then 2 when run7 like 'waiting' then 2 when run7 like 'cancelled' then 3 when run7 like 'None' then 4 else 0 end as run_7,\n case when run8 like 'success' then 1 when run8 like 'in_progress' then 2 when run8 like 'queued' then 2 when run8 like 'waiting' then 2 when run8 like 'cancelled' then 3 when run8 like 'None' then 4 else 0 end as run_8,\n case when run9 like 'success' then 1 when run9 like 'in_progress' then 2 when run9 like 'queued' then 2 when run9 like 'waiting' then 2 when run9 like 'cancelled' then 3 when run9 like 'None' then 4 else 0 end as run_9,\n case when run10 like 'success' then 1 when run10 like 'in_progress' then 2 when run10 like 'queued' then 2 when run10 like 'waiting' then 2 when run10 like 'cancelled' then 3 when run10 like 'None' then 4 else 0 end as run_10\n from\n github_workflows\n)\nselect\n job_name,\n job_yml_filename,\n run_1,\n run1Id,\n run_2,\n run2Id,\n run_3,\n run3Id,\n run_4,\n run4Id,\n run_5,\n run5Id,\n run_6,\n run6Id,\n run_7,\n run7Id,\n run_8,\n run8Id,\n run_9,\n run9Id,\n run_10,\n run10Id\nfrom\n workflows;", + "rawSql": "with workflows as (\n select\n *,\n case when run1 like 'success' then 1 when run1 like 'in_progress' then 2 when run1 like 'queued' then 2 when run1 like 'waiting' then 2 when run1 like 'cancelled' then 3 when run1 like 'None' then 4 else 0 end as run_1,\n case when run2 like 'success' then 1 when run2 like 'in_progress' then 2 when run2 like 'queued' then 2 when run2 like 'waiting' then 2 when run2 like 'cancelled' then 3 when run2 like 'None' then 4 else 0 end as run_2,\n case when run3 like 'success' then 1 when run3 like 'in_progress' then 2 when run3 like 'queued' then 2 when run3 like 'waiting' then 2 when run3 like 'cancelled' then 3 when run3 like 'None' then 4 else 0 end as run_3,\n case when run4 like 'success' then 1 when run4 like 'in_progress' then 2 when run4 like 'queued' then 2 when run4 like 'waiting' then 2 when run4 like 'cancelled' then 3 when run4 like 'None' then 4 else 0 end as run_4,\n case when run5 like 'success' then 1 when run5 like 'in_progress' then 2 when run5 like 'queued' then 2 when run5 like 'waiting' then 2 when run5 like 'cancelled' then 3 when run5 like 'None' then 4 else 0 end as run_5,\n case when run6 like 'success' then 1 when run6 like 'in_progress' then 2 when run6 like 'queued' then 2 when run6 like 'waiting' then 2 when run6 like 'cancelled' then 3 when run6 like 'None' then 4 else 0 end as run_6,\n case when run7 like 'success' then 1 when run7 like 'in_progress' then 2 when run7 like 'queued' then 2 when run7 like 'waiting' then 2 when run7 like 'cancelled' then 3 when run7 like 'None' then 4 else 0 end as run_7,\n case when run8 like 'success' then 1 when run8 like 'in_progress' then 2 when run8 like 'queued' then 2 when run8 like 'waiting' then 2 when run8 like 'cancelled' then 3 when run8 like 'None' then 4 else 0 end as run_8,\n case when run9 like 'success' then 1 when run9 like 'in_progress' then 2 when run9 like 'queued' then 2 when run9 like 'waiting' then 2 when run9 like 'cancelled' then 3 when run9 like 'None' then 4 else 0 end as run_9,\n case when run10 like 'success' then 1 when run10 like 'in_progress' then 2 when run10 like 'queued' then 2 when run10 like 'waiting' then 2 when run10 like 'cancelled' then 3 when run10 like 'None' then 4 else 0 end as run_10\n from\n github_workflows\n where\n dashboard_category = 'core_java'\n)\nselect\n job_name,\n job_yml_filename,\n run_1,\n run1Id,\n run_2,\n run2Id,\n run_3,\n run3Id,\n run_4,\n run4Id,\n run_5,\n run5Id,\n run_6,\n run6Id,\n run_7,\n run7Id,\n run_8,\n run8Id,\n run_9,\n run9Id,\n run_10,\n run10Id\nfrom\n workflows;", "refId": "A", "select": [ [ @@ -920,6 +957,23 @@ } ] ], + "sql": { + "columns": [ + { + "parameters": [], + "type": "function" + } + ], + "groupBy": [ + { + "property": { + "type": "string" + }, + "type": "groupBy" + } + ], + "limit": 50 + }, "timeColumn": "time", "units": "none", "valueHandler": "Number Threshold", @@ -931,10 +985,11649 @@ } ] } - ] + ], + "title": "Core Java Tests", + "transformations": [ + { + "id": "merge", + "options": { + "reducers": [] + } + } + ], + "type": "table" + }, + { + "datasource": { + "type": "postgres", + "uid": "fK5onAOVz" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "thresholds" + }, + "custom": { + "align": "auto", + "cellOptions": { + "type": "auto" + }, + "inspect": false + }, + "mappings": [ + { + "options": { + "0": { + "color": "semi-dark-red", + "index": 0, + "text": "Fail" + }, + "1": { + "color": "semi-dark-green", + "index": 1, + "text": "Success" + }, + "2": { + "color": "semi-dark-yellow", + "index": 2, + "text": "Pending" + }, + "3": { + "color": "semi-dark-purple", + "index": 3, + "text": "Cancelled" + }, + "4": { + "color": "light-blue", + "index": 4, + "text": "None" + } + }, + "type": "value" + } + ], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "semi-dark-red", + "value": null + } + ] + } + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "job_name" + }, + "properties": [ + { + "id": "unit", + "value": "short" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "links", + "value": [ + { + "targetBlank": true, + "title": "", + "url": "https://github.com/apache/beam/actions/${__data.fields.job_yml_filename}" + } + ] + }, + { + "id": "custom.align", + "value": "auto" + }, + { + "id": "mappings", + "value": [] + }, + { + "id": "displayName", + "value": "Job Name" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "job_yml_filename" + }, + "properties": [ + { + "id": "custom.hidden", + "value": true + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/run\\d+id/" + }, + "properties": [ + { + "id": "custom.hidden", + "value": true + }, + { + "id": "custom.align", + "value": "center" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_1" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run1id}" + } + ] + }, + { + "id": "displayName", + "value": "Run" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_2" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run2id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 2" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_3" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run3id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 3" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_4" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run4id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 4" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_5" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run5id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 5" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_6" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run6id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 6" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_7" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run7id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 7" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_8" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run8id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 8" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_9" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run9id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 9" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_10" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run10id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 10" + } + ] + } + ] + }, + "gridPos": { + "h": 20, + "w": 24, + "x": 0, + "y": 20 + }, + "id": 11, + "links": [], + "options": { + "cellHeight": "sm", + "footer": { + "countRows": false, + "fields": "", + "reducer": [ + "sum" + ], + "show": false + }, + "showHeader": true + }, + "pluginVersion": "10.0.3", + "targets": [ + { + "aggregation": "Last", + "alias": "job", + "datasource": { + "type": "postgres", + "uid": "fK5onAOVz" + }, + "decimals": 2, + "displayAliasType": "Warning / Critical", + "displayType": "Regular", + "displayValueWithAlias": "Never", + "editorMode": "code", + "format": "table", + "group": [], + "hide": false, + "metricColumn": "none", + "rawQuery": true, + "rawSql": "with workflows as (\n select\n *,\n case when run1 like 'success' then 1 when run1 like 'in_progress' then 2 when run1 like 'queued' then 2 when run1 like 'waiting' then 2 when run1 like 'cancelled' then 3 when run1 like 'None' then 4 else 0 end as run_1,\n case when run2 like 'success' then 1 when run2 like 'in_progress' then 2 when run2 like 'queued' then 2 when run2 like 'waiting' then 2 when run2 like 'cancelled' then 3 when run2 like 'None' then 4 else 0 end as run_2,\n case when run3 like 'success' then 1 when run3 like 'in_progress' then 2 when run3 like 'queued' then 2 when run3 like 'waiting' then 2 when run3 like 'cancelled' then 3 when run3 like 'None' then 4 else 0 end as run_3,\n case when run4 like 'success' then 1 when run4 like 'in_progress' then 2 when run4 like 'queued' then 2 when run4 like 'waiting' then 2 when run4 like 'cancelled' then 3 when run4 like 'None' then 4 else 0 end as run_4,\n case when run5 like 'success' then 1 when run5 like 'in_progress' then 2 when run5 like 'queued' then 2 when run5 like 'waiting' then 2 when run5 like 'cancelled' then 3 when run5 like 'None' then 4 else 0 end as run_5,\n case when run6 like 'success' then 1 when run6 like 'in_progress' then 2 when run6 like 'queued' then 2 when run6 like 'waiting' then 2 when run6 like 'cancelled' then 3 when run6 like 'None' then 4 else 0 end as run_6,\n case when run7 like 'success' then 1 when run7 like 'in_progress' then 2 when run7 like 'queued' then 2 when run7 like 'waiting' then 2 when run7 like 'cancelled' then 3 when run7 like 'None' then 4 else 0 end as run_7,\n case when run8 like 'success' then 1 when run8 like 'in_progress' then 2 when run8 like 'queued' then 2 when run8 like 'waiting' then 2 when run8 like 'cancelled' then 3 when run8 like 'None' then 4 else 0 end as run_8,\n case when run9 like 'success' then 1 when run9 like 'in_progress' then 2 when run9 like 'queued' then 2 when run9 like 'waiting' then 2 when run9 like 'cancelled' then 3 when run9 like 'None' then 4 else 0 end as run_9,\n case when run10 like 'success' then 1 when run10 like 'in_progress' then 2 when run10 like 'queued' then 2 when run10 like 'waiting' then 2 when run10 like 'cancelled' then 3 when run10 like 'None' then 4 else 0 end as run_10\n from\n github_workflows\n where\n dashboard_category = 'dataflow_java'\n)\nselect\n job_name,\n job_yml_filename,\n run_1,\n run1Id,\n run_2,\n run2Id,\n run_3,\n run3Id,\n run_4,\n run4Id,\n run_5,\n run5Id,\n run_6,\n run6Id,\n run_7,\n run7Id,\n run_8,\n run8Id,\n run_9,\n run9Id,\n run_10,\n run10Id\nfrom\n workflows;", + "refId": "A", + "select": [ + [ + { + "params": [ + "value" + ], + "type": "column" + } + ] + ], + "sql": { + "columns": [ + { + "parameters": [], + "type": "function" + } + ], + "groupBy": [ + { + "property": { + "type": "string" + }, + "type": "groupBy" + } + ], + "limit": 50 + }, + "timeColumn": "time", + "units": "none", + "valueHandler": "Number Threshold", + "where": [ + { + "name": "$__timeFilter", + "params": [], + "type": "macro" + } + ] + } + ], + "title": "Dataflow Java Tests", + "transformations": [ + { + "id": "merge", + "options": { + "reducers": [] + } + } + ], + "type": "table" + }, + { + "datasource": { + "type": "postgres", + "uid": "fK5onAOVz" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "thresholds" + }, + "custom": { + "align": "auto", + "cellOptions": { + "type": "auto" + }, + "inspect": false + }, + "mappings": [ + { + "options": { + "0": { + "color": "semi-dark-red", + "index": 0, + "text": "Fail" + }, + "1": { + "color": "semi-dark-green", + "index": 1, + "text": "Success" + }, + "2": { + "color": "semi-dark-yellow", + "index": 2, + "text": "Pending" + }, + "3": { + "color": "semi-dark-purple", + "index": 3, + "text": "Cancelled" + }, + "4": { + "color": "light-blue", + "index": 4, + "text": "None" + } + }, + "type": "value" + } + ], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "semi-dark-red", + "value": null + } + ] + } + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "job_name" + }, + "properties": [ + { + "id": "unit", + "value": "short" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "links", + "value": [ + { + "targetBlank": true, + "title": "", + "url": "https://github.com/apache/beam/actions/${__data.fields.job_yml_filename}" + } + ] + }, + { + "id": "custom.align", + "value": "auto" + }, + { + "id": "mappings", + "value": [] + }, + { + "id": "displayName", + "value": "Job Name" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "job_yml_filename" + }, + "properties": [ + { + "id": "custom.hidden", + "value": true + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/run\\d+id/" + }, + "properties": [ + { + "id": "custom.hidden", + "value": true + }, + { + "id": "custom.align", + "value": "center" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_1" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run1id}" + } + ] + }, + { + "id": "displayName", + "value": "Run" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_2" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run2id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 2" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_3" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run3id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 3" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_4" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run4id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 4" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_5" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run5id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 5" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_6" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run6id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 6" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_7" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run7id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 7" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_8" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run8id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 8" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_9" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run9id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 9" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_10" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run10id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 10" + } + ] + } + ] + }, + "gridPos": { + "h": 20, + "w": 24, + "x": 0, + "y": 40 + }, + "id": 10, + "links": [], + "options": { + "cellHeight": "sm", + "footer": { + "countRows": false, + "fields": "", + "reducer": [ + "sum" + ], + "show": false + }, + "showHeader": true + }, + "pluginVersion": "10.0.3", + "targets": [ + { + "aggregation": "Last", + "alias": "job", + "datasource": { + "type": "postgres", + "uid": "fK5onAOVz" + }, + "decimals": 2, + "displayAliasType": "Warning / Critical", + "displayType": "Regular", + "displayValueWithAlias": "Never", + "editorMode": "code", + "format": "table", + "group": [], + "hide": false, + "metricColumn": "none", + "rawQuery": true, + "rawSql": "with workflows as (\n select\n *,\n case when run1 like 'success' then 1 when run1 like 'in_progress' then 2 when run1 like 'queued' then 2 when run1 like 'waiting' then 2 when run1 like 'cancelled' then 3 when run1 like 'None' then 4 else 0 end as run_1,\n case when run2 like 'success' then 1 when run2 like 'in_progress' then 2 when run2 like 'queued' then 2 when run2 like 'waiting' then 2 when run2 like 'cancelled' then 3 when run2 like 'None' then 4 else 0 end as run_2,\n case when run3 like 'success' then 1 when run3 like 'in_progress' then 2 when run3 like 'queued' then 2 when run3 like 'waiting' then 2 when run3 like 'cancelled' then 3 when run3 like 'None' then 4 else 0 end as run_3,\n case when run4 like 'success' then 1 when run4 like 'in_progress' then 2 when run4 like 'queued' then 2 when run4 like 'waiting' then 2 when run4 like 'cancelled' then 3 when run4 like 'None' then 4 else 0 end as run_4,\n case when run5 like 'success' then 1 when run5 like 'in_progress' then 2 when run5 like 'queued' then 2 when run5 like 'waiting' then 2 when run5 like 'cancelled' then 3 when run5 like 'None' then 4 else 0 end as run_5,\n case when run6 like 'success' then 1 when run6 like 'in_progress' then 2 when run6 like 'queued' then 2 when run6 like 'waiting' then 2 when run6 like 'cancelled' then 3 when run6 like 'None' then 4 else 0 end as run_6,\n case when run7 like 'success' then 1 when run7 like 'in_progress' then 2 when run7 like 'queued' then 2 when run7 like 'waiting' then 2 when run7 like 'cancelled' then 3 when run7 like 'None' then 4 else 0 end as run_7,\n case when run8 like 'success' then 1 when run8 like 'in_progress' then 2 when run8 like 'queued' then 2 when run8 like 'waiting' then 2 when run8 like 'cancelled' then 3 when run8 like 'None' then 4 else 0 end as run_8,\n case when run9 like 'success' then 1 when run9 like 'in_progress' then 2 when run9 like 'queued' then 2 when run9 like 'waiting' then 2 when run9 like 'cancelled' then 3 when run9 like 'None' then 4 else 0 end as run_9,\n case when run10 like 'success' then 1 when run10 like 'in_progress' then 2 when run10 like 'queued' then 2 when run10 like 'waiting' then 2 when run10 like 'cancelled' then 3 when run10 like 'None' then 4 else 0 end as run_10\n from\n github_workflows\n where\n dashboard_category = 'spark_java'\n)\nselect\n job_name,\n job_yml_filename,\n run_1,\n run1Id,\n run_2,\n run2Id,\n run_3,\n run3Id,\n run_4,\n run4Id,\n run_5,\n run5Id,\n run_6,\n run6Id,\n run_7,\n run7Id,\n run_8,\n run8Id,\n run_9,\n run9Id,\n run_10,\n run10Id\nfrom\n workflows;", + "refId": "A", + "select": [ + [ + { + "params": [ + "value" + ], + "type": "column" + } + ] + ], + "sql": { + "columns": [ + { + "parameters": [], + "type": "function" + } + ], + "groupBy": [ + { + "property": { + "type": "string" + }, + "type": "groupBy" + } + ], + "limit": 50 + }, + "timeColumn": "time", + "units": "none", + "valueHandler": "Number Threshold", + "where": [ + { + "name": "$__timeFilter", + "params": [], + "type": "macro" + } + ] + } + ], + "title": "Spark Java Tests", + "transformations": [ + { + "id": "merge", + "options": { + "reducers": [] + } + } + ], + "type": "table" + }, + { + "datasource": { + "type": "postgres", + "uid": "fK5onAOVz" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "thresholds" + }, + "custom": { + "align": "auto", + "cellOptions": { + "type": "auto" + }, + "inspect": false + }, + "mappings": [ + { + "options": { + "0": { + "color": "semi-dark-red", + "index": 0, + "text": "Fail" + }, + "1": { + "color": "semi-dark-green", + "index": 1, + "text": "Success" + }, + "2": { + "color": "semi-dark-yellow", + "index": 2, + "text": "Pending" + }, + "3": { + "color": "semi-dark-purple", + "index": 3, + "text": "Cancelled" + }, + "4": { + "color": "light-blue", + "index": 4, + "text": "None" + } + }, + "type": "value" + } + ], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "semi-dark-red", + "value": null + } + ] + } + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "job_name" + }, + "properties": [ + { + "id": "unit", + "value": "short" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "links", + "value": [ + { + "targetBlank": true, + "title": "", + "url": "https://github.com/apache/beam/actions/${__data.fields.job_yml_filename}" + } + ] + }, + { + "id": "custom.align", + "value": "auto" + }, + { + "id": "mappings", + "value": [] + }, + { + "id": "displayName", + "value": "Job Name" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "job_yml_filename" + }, + "properties": [ + { + "id": "custom.hidden", + "value": true + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/run\\d+id/" + }, + "properties": [ + { + "id": "custom.hidden", + "value": true + }, + { + "id": "custom.align", + "value": "center" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_1" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run1id}" + } + ] + }, + { + "id": "displayName", + "value": "Run" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_2" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run2id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 2" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_3" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run3id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 3" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_4" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run4id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 4" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_5" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run5id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 5" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_6" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run6id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 6" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_7" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run7id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 7" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_8" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run8id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 8" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_9" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run9id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 9" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_10" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run10id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 10" + } + ] + } + ] + }, + "gridPos": { + "h": 20, + "w": 24, + "x": 0, + "y": 60 + }, + "id": 12, + "links": [], + "options": { + "cellHeight": "sm", + "footer": { + "countRows": false, + "fields": "", + "reducer": [ + "sum" + ], + "show": false + }, + "showHeader": true + }, + "pluginVersion": "10.0.3", + "targets": [ + { + "aggregation": "Last", + "alias": "job", + "datasource": { + "type": "postgres", + "uid": "fK5onAOVz" + }, + "decimals": 2, + "displayAliasType": "Warning / Critical", + "displayType": "Regular", + "displayValueWithAlias": "Never", + "editorMode": "code", + "format": "table", + "group": [], + "hide": false, + "metricColumn": "none", + "rawQuery": true, + "rawSql": "with workflows as (\n select\n *,\n case when run1 like 'success' then 1 when run1 like 'in_progress' then 2 when run1 like 'queued' then 2 when run1 like 'waiting' then 2 when run1 like 'cancelled' then 3 when run1 like 'None' then 4 else 0 end as run_1,\n case when run2 like 'success' then 1 when run2 like 'in_progress' then 2 when run2 like 'queued' then 2 when run2 like 'waiting' then 2 when run2 like 'cancelled' then 3 when run2 like 'None' then 4 else 0 end as run_2,\n case when run3 like 'success' then 1 when run3 like 'in_progress' then 2 when run3 like 'queued' then 2 when run3 like 'waiting' then 2 when run3 like 'cancelled' then 3 when run3 like 'None' then 4 else 0 end as run_3,\n case when run4 like 'success' then 1 when run4 like 'in_progress' then 2 when run4 like 'queued' then 2 when run4 like 'waiting' then 2 when run4 like 'cancelled' then 3 when run4 like 'None' then 4 else 0 end as run_4,\n case when run5 like 'success' then 1 when run5 like 'in_progress' then 2 when run5 like 'queued' then 2 when run5 like 'waiting' then 2 when run5 like 'cancelled' then 3 when run5 like 'None' then 4 else 0 end as run_5,\n case when run6 like 'success' then 1 when run6 like 'in_progress' then 2 when run6 like 'queued' then 2 when run6 like 'waiting' then 2 when run6 like 'cancelled' then 3 when run6 like 'None' then 4 else 0 end as run_6,\n case when run7 like 'success' then 1 when run7 like 'in_progress' then 2 when run7 like 'queued' then 2 when run7 like 'waiting' then 2 when run7 like 'cancelled' then 3 when run7 like 'None' then 4 else 0 end as run_7,\n case when run8 like 'success' then 1 when run8 like 'in_progress' then 2 when run8 like 'queued' then 2 when run8 like 'waiting' then 2 when run8 like 'cancelled' then 3 when run8 like 'None' then 4 else 0 end as run_8,\n case when run9 like 'success' then 1 when run9 like 'in_progress' then 2 when run9 like 'queued' then 2 when run9 like 'waiting' then 2 when run9 like 'cancelled' then 3 when run9 like 'None' then 4 else 0 end as run_9,\n case when run10 like 'success' then 1 when run10 like 'in_progress' then 2 when run10 like 'queued' then 2 when run10 like 'waiting' then 2 when run10 like 'cancelled' then 3 when run10 like 'None' then 4 else 0 end as run_10\n from\n github_workflows\n where\n dashboard_category = 'flink_java'\n)\nselect\n job_name,\n job_yml_filename,\n run_1,\n run1Id,\n run_2,\n run2Id,\n run_3,\n run3Id,\n run_4,\n run4Id,\n run_5,\n run5Id,\n run_6,\n run6Id,\n run_7,\n run7Id,\n run_8,\n run8Id,\n run_9,\n run9Id,\n run_10,\n run10Id\nfrom\n workflows;", + "refId": "A", + "select": [ + [ + { + "params": [ + "value" + ], + "type": "column" + } + ] + ], + "sql": { + "columns": [ + { + "parameters": [], + "type": "function" + } + ], + "groupBy": [ + { + "property": { + "type": "string" + }, + "type": "groupBy" + } + ], + "limit": 50 + }, + "timeColumn": "time", + "units": "none", + "valueHandler": "Number Threshold", + "where": [ + { + "name": "$__timeFilter", + "params": [], + "type": "macro" + } + ] + } + ], + "title": "Flink Java Tests", + "transformations": [ + { + "id": "merge", + "options": { + "reducers": [] + } + } + ], + "type": "table" + }, + { + "datasource": { + "type": "postgres", + "uid": "fK5onAOVz" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "thresholds" + }, + "custom": { + "align": "auto", + "cellOptions": { + "type": "auto" + }, + "inspect": false + }, + "mappings": [ + { + "options": { + "0": { + "color": "semi-dark-red", + "index": 0, + "text": "Fail" + }, + "1": { + "color": "semi-dark-green", + "index": 1, + "text": "Success" + }, + "2": { + "color": "semi-dark-yellow", + "index": 2, + "text": "Pending" + }, + "3": { + "color": "semi-dark-purple", + "index": 3, + "text": "Cancelled" + }, + "4": { + "color": "light-blue", + "index": 4, + "text": "None" + } + }, + "type": "value" + } + ], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "semi-dark-red", + "value": null + } + ] + } + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "job_name" + }, + "properties": [ + { + "id": "unit", + "value": "short" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "links", + "value": [ + { + "targetBlank": true, + "title": "", + "url": "https://github.com/apache/beam/actions/${__data.fields.job_yml_filename}" + } + ] + }, + { + "id": "custom.align", + "value": "auto" + }, + { + "id": "mappings", + "value": [] + }, + { + "id": "displayName", + "value": "Job Name" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "job_yml_filename" + }, + "properties": [ + { + "id": "custom.hidden", + "value": true + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/run\\d+id/" + }, + "properties": [ + { + "id": "custom.hidden", + "value": true + }, + { + "id": "custom.align", + "value": "center" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_1" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run1id}" + } + ] + }, + { + "id": "displayName", + "value": "Run" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_2" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run2id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 2" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_3" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run3id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 3" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_4" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run4id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 4" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_5" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run5id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 5" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_6" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run6id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 6" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_7" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run7id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 7" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_8" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run8id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 8" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_9" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run9id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 9" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_10" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run10id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 10" + } + ] + } + ] + }, + "gridPos": { + "h": 20, + "w": 24, + "x": 0, + "y": 80 + }, + "id": 3, + "links": [], + "options": { + "cellHeight": "sm", + "footer": { + "countRows": false, + "fields": "", + "reducer": [ + "sum" + ], + "show": false + }, + "showHeader": true + }, + "pluginVersion": "10.0.3", + "targets": [ + { + "aggregation": "Last", + "alias": "job", + "datasource": { + "type": "postgres", + "uid": "fK5onAOVz" + }, + "decimals": 2, + "displayAliasType": "Warning / Critical", + "displayType": "Regular", + "displayValueWithAlias": "Never", + "editorMode": "code", + "format": "table", + "group": [], + "hide": false, + "metricColumn": "none", + "rawQuery": true, + "rawSql": "with workflows as (\n select\n *,\n case when run1 like 'success' then 1 when run1 like 'in_progress' then 2 when run1 like 'queued' then 2 when run1 like 'waiting' then 2 when run1 like 'cancelled' then 3 when run1 like 'None' then 4 else 0 end as run_1,\n case when run2 like 'success' then 1 when run2 like 'in_progress' then 2 when run2 like 'queued' then 2 when run2 like 'waiting' then 2 when run2 like 'cancelled' then 3 when run2 like 'None' then 4 else 0 end as run_2,\n case when run3 like 'success' then 1 when run3 like 'in_progress' then 2 when run3 like 'queued' then 2 when run3 like 'waiting' then 2 when run3 like 'cancelled' then 3 when run3 like 'None' then 4 else 0 end as run_3,\n case when run4 like 'success' then 1 when run4 like 'in_progress' then 2 when run4 like 'queued' then 2 when run4 like 'waiting' then 2 when run4 like 'cancelled' then 3 when run4 like 'None' then 4 else 0 end as run_4,\n case when run5 like 'success' then 1 when run5 like 'in_progress' then 2 when run5 like 'queued' then 2 when run5 like 'waiting' then 2 when run5 like 'cancelled' then 3 when run5 like 'None' then 4 else 0 end as run_5,\n case when run6 like 'success' then 1 when run6 like 'in_progress' then 2 when run6 like 'queued' then 2 when run6 like 'waiting' then 2 when run6 like 'cancelled' then 3 when run6 like 'None' then 4 else 0 end as run_6,\n case when run7 like 'success' then 1 when run7 like 'in_progress' then 2 when run7 like 'queued' then 2 when run7 like 'waiting' then 2 when run7 like 'cancelled' then 3 when run7 like 'None' then 4 else 0 end as run_7,\n case when run8 like 'success' then 1 when run8 like 'in_progress' then 2 when run8 like 'queued' then 2 when run8 like 'waiting' then 2 when run8 like 'cancelled' then 3 when run8 like 'None' then 4 else 0 end as run_8,\n case when run9 like 'success' then 1 when run9 like 'in_progress' then 2 when run9 like 'queued' then 2 when run9 like 'waiting' then 2 when run9 like 'cancelled' then 3 when run9 like 'None' then 4 else 0 end as run_9,\n case when run10 like 'success' then 1 when run10 like 'in_progress' then 2 when run10 like 'queued' then 2 when run10 like 'waiting' then 2 when run10 like 'cancelled' then 3 when run10 like 'None' then 4 else 0 end as run_10\n from\n github_workflows\n where\n dashboard_category = 'core_python'\n)\nselect\n job_name,\n job_yml_filename,\n run_1,\n run1Id,\n run_2,\n run2Id,\n run_3,\n run3Id,\n run_4,\n run4Id,\n run_5,\n run5Id,\n run_6,\n run6Id,\n run_7,\n run7Id,\n run_8,\n run8Id,\n run_9,\n run9Id,\n run_10,\n run10Id\nfrom\n workflows;", + "refId": "A", + "select": [ + [ + { + "params": [ + "value" + ], + "type": "column" + } + ] + ], + "sql": { + "columns": [ + { + "parameters": [], + "type": "function" + } + ], + "groupBy": [ + { + "property": { + "type": "string" + }, + "type": "groupBy" + } + ], + "limit": 50 + }, + "timeColumn": "time", + "units": "none", + "valueHandler": "Number Threshold", + "where": [ + { + "name": "$__timeFilter", + "params": [], + "type": "macro" + } + ] + } + ], + "title": "Core Python Tests", + "transformations": [ + { + "id": "merge", + "options": { + "reducers": [] + } + } + ], + "type": "table" + }, + { + "datasource": { + "type": "postgres", + "uid": "fK5onAOVz" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "thresholds" + }, + "custom": { + "align": "auto", + "cellOptions": { + "type": "auto" + }, + "inspect": false + }, + "mappings": [ + { + "options": { + "0": { + "color": "semi-dark-red", + "index": 0, + "text": "Fail" + }, + "1": { + "color": "semi-dark-green", + "index": 1, + "text": "Success" + }, + "2": { + "color": "semi-dark-yellow", + "index": 2, + "text": "Pending" + }, + "3": { + "color": "semi-dark-purple", + "index": 3, + "text": "Cancelled" + }, + "4": { + "color": "light-blue", + "index": 4, + "text": "None" + } + }, + "type": "value" + } + ], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "semi-dark-red", + "value": null + } + ] + } + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "job_name" + }, + "properties": [ + { + "id": "unit", + "value": "short" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "links", + "value": [ + { + "targetBlank": true, + "title": "", + "url": "https://github.com/apache/beam/actions/${__data.fields.job_yml_filename}" + } + ] + }, + { + "id": "custom.align", + "value": "auto" + }, + { + "id": "mappings", + "value": [] + }, + { + "id": "displayName", + "value": "Job Name" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "job_yml_filename" + }, + "properties": [ + { + "id": "custom.hidden", + "value": true + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/run\\d+id/" + }, + "properties": [ + { + "id": "custom.hidden", + "value": true + }, + { + "id": "custom.align", + "value": "center" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_1" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run1id}" + } + ] + }, + { + "id": "displayName", + "value": "Run" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_2" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run2id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 2" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_3" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run3id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 3" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_4" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run4id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 4" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_5" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run5id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 5" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_6" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run6id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 6" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_7" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run7id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 7" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_8" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run8id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 8" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_9" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run9id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 9" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_10" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run10id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 10" + } + ] + } + ] + }, + "gridPos": { + "h": 20, + "w": 24, + "x": 0, + "y": 100 + }, + "id": 9, + "links": [], + "options": { + "cellHeight": "sm", + "footer": { + "countRows": false, + "fields": "", + "reducer": [ + "sum" + ], + "show": false + }, + "showHeader": true + }, + "pluginVersion": "10.0.3", + "targets": [ + { + "aggregation": "Last", + "alias": "job", + "datasource": { + "type": "postgres", + "uid": "fK5onAOVz" + }, + "decimals": 2, + "displayAliasType": "Warning / Critical", + "displayType": "Regular", + "displayValueWithAlias": "Never", + "editorMode": "code", + "format": "table", + "group": [], + "hide": false, + "metricColumn": "none", + "rawQuery": true, + "rawSql": "with workflows as (\n select\n *,\n case when run1 like 'success' then 1 when run1 like 'in_progress' then 2 when run1 like 'queued' then 2 when run1 like 'waiting' then 2 when run1 like 'cancelled' then 3 when run1 like 'None' then 4 else 0 end as run_1,\n case when run2 like 'success' then 1 when run2 like 'in_progress' then 2 when run2 like 'queued' then 2 when run2 like 'waiting' then 2 when run2 like 'cancelled' then 3 when run2 like 'None' then 4 else 0 end as run_2,\n case when run3 like 'success' then 1 when run3 like 'in_progress' then 2 when run3 like 'queued' then 2 when run3 like 'waiting' then 2 when run3 like 'cancelled' then 3 when run3 like 'None' then 4 else 0 end as run_3,\n case when run4 like 'success' then 1 when run4 like 'in_progress' then 2 when run4 like 'queued' then 2 when run4 like 'waiting' then 2 when run4 like 'cancelled' then 3 when run4 like 'None' then 4 else 0 end as run_4,\n case when run5 like 'success' then 1 when run5 like 'in_progress' then 2 when run5 like 'queued' then 2 when run5 like 'waiting' then 2 when run5 like 'cancelled' then 3 when run5 like 'None' then 4 else 0 end as run_5,\n case when run6 like 'success' then 1 when run6 like 'in_progress' then 2 when run6 like 'queued' then 2 when run6 like 'waiting' then 2 when run6 like 'cancelled' then 3 when run6 like 'None' then 4 else 0 end as run_6,\n case when run7 like 'success' then 1 when run7 like 'in_progress' then 2 when run7 like 'queued' then 2 when run7 like 'waiting' then 2 when run7 like 'cancelled' then 3 when run7 like 'None' then 4 else 0 end as run_7,\n case when run8 like 'success' then 1 when run8 like 'in_progress' then 2 when run8 like 'queued' then 2 when run8 like 'waiting' then 2 when run8 like 'cancelled' then 3 when run8 like 'None' then 4 else 0 end as run_8,\n case when run9 like 'success' then 1 when run9 like 'in_progress' then 2 when run9 like 'queued' then 2 when run9 like 'waiting' then 2 when run9 like 'cancelled' then 3 when run9 like 'None' then 4 else 0 end as run_9,\n case when run10 like 'success' then 1 when run10 like 'in_progress' then 2 when run10 like 'queued' then 2 when run10 like 'waiting' then 2 when run10 like 'cancelled' then 3 when run10 like 'None' then 4 else 0 end as run_10\n from\n github_workflows\n where\n dashboard_category = 'dataflow_python'\n)\nselect\n job_name,\n job_yml_filename,\n run_1,\n run1Id,\n run_2,\n run2Id,\n run_3,\n run3Id,\n run_4,\n run4Id,\n run_5,\n run5Id,\n run_6,\n run6Id,\n run_7,\n run7Id,\n run_8,\n run8Id,\n run_9,\n run9Id,\n run_10,\n run10Id\nfrom\n workflows;", + "refId": "A", + "select": [ + [ + { + "params": [ + "value" + ], + "type": "column" + } + ] + ], + "sql": { + "columns": [ + { + "parameters": [], + "type": "function" + } + ], + "groupBy": [ + { + "property": { + "type": "string" + }, + "type": "groupBy" + } + ], + "limit": 50 + }, + "timeColumn": "time", + "units": "none", + "valueHandler": "Number Threshold", + "where": [ + { + "name": "$__timeFilter", + "params": [], + "type": "macro" + } + ] + } + ], + "title": "Dataflow Python Tests", + "transformations": [ + { + "id": "merge", + "options": { + "reducers": [] + } + } + ], + "type": "table" + }, + { + "datasource": { + "type": "postgres", + "uid": "fK5onAOVz" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "thresholds" + }, + "custom": { + "align": "auto", + "cellOptions": { + "type": "auto" + }, + "inspect": false + }, + "mappings": [ + { + "options": { + "0": { + "color": "semi-dark-red", + "index": 0, + "text": "Fail" + }, + "1": { + "color": "semi-dark-green", + "index": 1, + "text": "Success" + }, + "2": { + "color": "semi-dark-yellow", + "index": 2, + "text": "Pending" + }, + "3": { + "color": "semi-dark-purple", + "index": 3, + "text": "Cancelled" + }, + "4": { + "color": "light-blue", + "index": 4, + "text": "None" + } + }, + "type": "value" + } + ], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "semi-dark-red", + "value": null + } + ] + } + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "job_name" + }, + "properties": [ + { + "id": "unit", + "value": "short" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "links", + "value": [ + { + "targetBlank": true, + "title": "", + "url": "https://github.com/apache/beam/actions/${__data.fields.job_yml_filename}" + } + ] + }, + { + "id": "custom.align", + "value": "auto" + }, + { + "id": "mappings", + "value": [] + }, + { + "id": "displayName", + "value": "Job Name" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "job_yml_filename" + }, + "properties": [ + { + "id": "custom.hidden", + "value": true + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/run\\d+id/" + }, + "properties": [ + { + "id": "custom.hidden", + "value": true + }, + { + "id": "custom.align", + "value": "center" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_1" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run1id}" + } + ] + }, + { + "id": "displayName", + "value": "Run" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_2" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run2id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 2" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_3" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run3id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 3" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_4" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run4id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 4" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_5" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run5id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 5" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_6" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run6id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 6" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_7" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run7id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 7" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_8" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run8id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 8" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_9" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run9id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 9" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_10" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run10id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 10" + } + ] + } + ] + }, + "gridPos": { + "h": 20, + "w": 24, + "x": 0, + "y": 120 + }, + "id": 7, + "links": [], + "options": { + "cellHeight": "sm", + "footer": { + "countRows": false, + "fields": "", + "reducer": [ + "sum" + ], + "show": false + }, + "showHeader": true + }, + "pluginVersion": "10.0.3", + "targets": [ + { + "aggregation": "Last", + "alias": "job", + "datasource": { + "type": "postgres", + "uid": "fK5onAOVz" + }, + "decimals": 2, + "displayAliasType": "Warning / Critical", + "displayType": "Regular", + "displayValueWithAlias": "Never", + "editorMode": "code", + "format": "table", + "group": [], + "hide": false, + "metricColumn": "none", + "rawQuery": true, + "rawSql": "with workflows as (\n select\n *,\n case when run1 like 'success' then 1 when run1 like 'in_progress' then 2 when run1 like 'queued' then 2 when run1 like 'waiting' then 2 when run1 like 'cancelled' then 3 when run1 like 'None' then 4 else 0 end as run_1,\n case when run2 like 'success' then 1 when run2 like 'in_progress' then 2 when run2 like 'queued' then 2 when run2 like 'waiting' then 2 when run2 like 'cancelled' then 3 when run2 like 'None' then 4 else 0 end as run_2,\n case when run3 like 'success' then 1 when run3 like 'in_progress' then 2 when run3 like 'queued' then 2 when run3 like 'waiting' then 2 when run3 like 'cancelled' then 3 when run3 like 'None' then 4 else 0 end as run_3,\n case when run4 like 'success' then 1 when run4 like 'in_progress' then 2 when run4 like 'queued' then 2 when run4 like 'waiting' then 2 when run4 like 'cancelled' then 3 when run4 like 'None' then 4 else 0 end as run_4,\n case when run5 like 'success' then 1 when run5 like 'in_progress' then 2 when run5 like 'queued' then 2 when run5 like 'waiting' then 2 when run5 like 'cancelled' then 3 when run5 like 'None' then 4 else 0 end as run_5,\n case when run6 like 'success' then 1 when run6 like 'in_progress' then 2 when run6 like 'queued' then 2 when run6 like 'waiting' then 2 when run6 like 'cancelled' then 3 when run6 like 'None' then 4 else 0 end as run_6,\n case when run7 like 'success' then 1 when run7 like 'in_progress' then 2 when run7 like 'queued' then 2 when run7 like 'waiting' then 2 when run7 like 'cancelled' then 3 when run7 like 'None' then 4 else 0 end as run_7,\n case when run8 like 'success' then 1 when run8 like 'in_progress' then 2 when run8 like 'queued' then 2 when run8 like 'waiting' then 2 when run8 like 'cancelled' then 3 when run8 like 'None' then 4 else 0 end as run_8,\n case when run9 like 'success' then 1 when run9 like 'in_progress' then 2 when run9 like 'queued' then 2 when run9 like 'waiting' then 2 when run9 like 'cancelled' then 3 when run9 like 'None' then 4 else 0 end as run_9,\n case when run10 like 'success' then 1 when run10 like 'in_progress' then 2 when run10 like 'queued' then 2 when run10 like 'waiting' then 2 when run10 like 'cancelled' then 3 when run10 like 'None' then 4 else 0 end as run_10\n from\n github_workflows\n where\n dashboard_category = 'spark_python'\n)\nselect\n job_name,\n job_yml_filename,\n run_1,\n run1Id,\n run_2,\n run2Id,\n run_3,\n run3Id,\n run_4,\n run4Id,\n run_5,\n run5Id,\n run_6,\n run6Id,\n run_7,\n run7Id,\n run_8,\n run8Id,\n run_9,\n run9Id,\n run_10,\n run10Id\nfrom\n workflows;", + "refId": "A", + "select": [ + [ + { + "params": [ + "value" + ], + "type": "column" + } + ] + ], + "sql": { + "columns": [ + { + "parameters": [], + "type": "function" + } + ], + "groupBy": [ + { + "property": { + "type": "string" + }, + "type": "groupBy" + } + ], + "limit": 50 + }, + "timeColumn": "time", + "units": "none", + "valueHandler": "Number Threshold", + "where": [ + { + "name": "$__timeFilter", + "params": [], + "type": "macro" + } + ] + } + ], + "title": "Spark Python Tests", + "transformations": [ + { + "id": "merge", + "options": { + "reducers": [] + } + } + ], + "type": "table" + }, + { + "datasource": { + "type": "postgres", + "uid": "fK5onAOVz" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "thresholds" + }, + "custom": { + "align": "auto", + "cellOptions": { + "type": "auto" + }, + "inspect": false + }, + "mappings": [ + { + "options": { + "0": { + "color": "semi-dark-red", + "index": 0, + "text": "Fail" + }, + "1": { + "color": "semi-dark-green", + "index": 1, + "text": "Success" + }, + "2": { + "color": "semi-dark-yellow", + "index": 2, + "text": "Pending" + }, + "3": { + "color": "semi-dark-purple", + "index": 3, + "text": "Cancelled" + }, + "4": { + "color": "light-blue", + "index": 4, + "text": "None" + } + }, + "type": "value" + } + ], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "semi-dark-red", + "value": null + } + ] + } + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "job_name" + }, + "properties": [ + { + "id": "unit", + "value": "short" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "links", + "value": [ + { + "targetBlank": true, + "title": "", + "url": "https://github.com/apache/beam/actions/${__data.fields.job_yml_filename}" + } + ] + }, + { + "id": "custom.align", + "value": "auto" + }, + { + "id": "mappings", + "value": [] + }, + { + "id": "displayName", + "value": "Job Name" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "job_yml_filename" + }, + "properties": [ + { + "id": "custom.hidden", + "value": true + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/run\\d+id/" + }, + "properties": [ + { + "id": "custom.hidden", + "value": true + }, + { + "id": "custom.align", + "value": "center" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_1" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run1id}" + } + ] + }, + { + "id": "displayName", + "value": "Run" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_2" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run2id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 2" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_3" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run3id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 3" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_4" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run4id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 4" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_5" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run5id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 5" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_6" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run6id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 6" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_7" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run7id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 7" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_8" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run8id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 8" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_9" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run9id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 9" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_10" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run10id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 10" + } + ] + } + ] + }, + "gridPos": { + "h": 20, + "w": 24, + "x": 0, + "y": 140 + }, + "id": 8, + "links": [], + "options": { + "cellHeight": "sm", + "footer": { + "countRows": false, + "fields": "", + "reducer": [ + "sum" + ], + "show": false + }, + "showHeader": true + }, + "pluginVersion": "10.0.3", + "targets": [ + { + "aggregation": "Last", + "alias": "job", + "datasource": { + "type": "postgres", + "uid": "fK5onAOVz" + }, + "decimals": 2, + "displayAliasType": "Warning / Critical", + "displayType": "Regular", + "displayValueWithAlias": "Never", + "editorMode": "code", + "format": "table", + "group": [], + "hide": false, + "metricColumn": "none", + "rawQuery": true, + "rawSql": "with workflows as (\n select\n *,\n case when run1 like 'success' then 1 when run1 like 'in_progress' then 2 when run1 like 'queued' then 2 when run1 like 'waiting' then 2 when run1 like 'cancelled' then 3 when run1 like 'None' then 4 else 0 end as run_1,\n case when run2 like 'success' then 1 when run2 like 'in_progress' then 2 when run2 like 'queued' then 2 when run2 like 'waiting' then 2 when run2 like 'cancelled' then 3 when run2 like 'None' then 4 else 0 end as run_2,\n case when run3 like 'success' then 1 when run3 like 'in_progress' then 2 when run3 like 'queued' then 2 when run3 like 'waiting' then 2 when run3 like 'cancelled' then 3 when run3 like 'None' then 4 else 0 end as run_3,\n case when run4 like 'success' then 1 when run4 like 'in_progress' then 2 when run4 like 'queued' then 2 when run4 like 'waiting' then 2 when run4 like 'cancelled' then 3 when run4 like 'None' then 4 else 0 end as run_4,\n case when run5 like 'success' then 1 when run5 like 'in_progress' then 2 when run5 like 'queued' then 2 when run5 like 'waiting' then 2 when run5 like 'cancelled' then 3 when run5 like 'None' then 4 else 0 end as run_5,\n case when run6 like 'success' then 1 when run6 like 'in_progress' then 2 when run6 like 'queued' then 2 when run6 like 'waiting' then 2 when run6 like 'cancelled' then 3 when run6 like 'None' then 4 else 0 end as run_6,\n case when run7 like 'success' then 1 when run7 like 'in_progress' then 2 when run7 like 'queued' then 2 when run7 like 'waiting' then 2 when run7 like 'cancelled' then 3 when run7 like 'None' then 4 else 0 end as run_7,\n case when run8 like 'success' then 1 when run8 like 'in_progress' then 2 when run8 like 'queued' then 2 when run8 like 'waiting' then 2 when run8 like 'cancelled' then 3 when run8 like 'None' then 4 else 0 end as run_8,\n case when run9 like 'success' then 1 when run9 like 'in_progress' then 2 when run9 like 'queued' then 2 when run9 like 'waiting' then 2 when run9 like 'cancelled' then 3 when run9 like 'None' then 4 else 0 end as run_9,\n case when run10 like 'success' then 1 when run10 like 'in_progress' then 2 when run10 like 'queued' then 2 when run10 like 'waiting' then 2 when run10 like 'cancelled' then 3 when run10 like 'None' then 4 else 0 end as run_10\n from\n github_workflows\n where\n dashboard_category = 'flink_python'\n)\nselect\n job_name,\n job_yml_filename,\n run_1,\n run1Id,\n run_2,\n run2Id,\n run_3,\n run3Id,\n run_4,\n run4Id,\n run_5,\n run5Id,\n run_6,\n run6Id,\n run_7,\n run7Id,\n run_8,\n run8Id,\n run_9,\n run9Id,\n run_10,\n run10Id\nfrom\n workflows;", + "refId": "A", + "select": [ + [ + { + "params": [ + "value" + ], + "type": "column" + } + ] + ], + "sql": { + "columns": [ + { + "parameters": [], + "type": "function" + } + ], + "groupBy": [ + { + "property": { + "type": "string" + }, + "type": "groupBy" + } + ], + "limit": 50 + }, + "timeColumn": "time", + "units": "none", + "valueHandler": "Number Threshold", + "where": [ + { + "name": "$__timeFilter", + "params": [], + "type": "macro" + } + ] + } + ], + "title": "Flink Python Tests", + "transformations": [ + { + "id": "merge", + "options": { + "reducers": [] + } + } + ], + "type": "table" + }, + { + "datasource": { + "type": "postgres", + "uid": "fK5onAOVz" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "thresholds" + }, + "custom": { + "align": "auto", + "cellOptions": { + "type": "auto" + }, + "inspect": false + }, + "mappings": [ + { + "options": { + "0": { + "color": "semi-dark-red", + "index": 0, + "text": "Fail" + }, + "1": { + "color": "semi-dark-green", + "index": 1, + "text": "Success" + }, + "2": { + "color": "semi-dark-yellow", + "index": 2, + "text": "Pending" + }, + "3": { + "color": "semi-dark-purple", + "index": 3, + "text": "Cancelled" + }, + "4": { + "color": "light-blue", + "index": 4, + "text": "None" + } + }, + "type": "value" + } + ], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "semi-dark-red", + "value": null + } + ] + } + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "job_name" + }, + "properties": [ + { + "id": "unit", + "value": "short" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "links", + "value": [ + { + "targetBlank": true, + "title": "", + "url": "https://github.com/apache/beam/actions/${__data.fields.job_yml_filename}" + } + ] + }, + { + "id": "custom.align", + "value": "auto" + }, + { + "id": "mappings", + "value": [] + }, + { + "id": "displayName", + "value": "Job Name" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "job_yml_filename" + }, + "properties": [ + { + "id": "custom.hidden", + "value": true + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/run\\d+id/" + }, + "properties": [ + { + "id": "custom.hidden", + "value": true + }, + { + "id": "custom.align", + "value": "center" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_1" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run1id}" + } + ] + }, + { + "id": "displayName", + "value": "Run" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_2" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run2id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 2" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_3" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run3id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 3" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_4" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run4id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 4" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_5" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run5id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 5" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_6" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run6id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 6" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_7" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run7id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 7" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_8" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run8id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 8" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_9" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run9id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 9" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_10" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run10id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 10" + } + ] + } + ] + }, + "gridPos": { + "h": 20, + "w": 24, + "x": 0, + "y": 160 + }, + "id": 13, + "links": [], + "options": { + "cellHeight": "sm", + "footer": { + "countRows": false, + "fields": "", + "reducer": [ + "sum" + ], + "show": false + }, + "showHeader": true + }, + "pluginVersion": "10.0.3", + "targets": [ + { + "aggregation": "Last", + "alias": "job", + "datasource": { + "type": "postgres", + "uid": "fK5onAOVz" + }, + "decimals": 2, + "displayAliasType": "Warning / Critical", + "displayType": "Regular", + "displayValueWithAlias": "Never", + "editorMode": "code", + "format": "table", + "group": [], + "hide": false, + "metricColumn": "none", + "rawQuery": true, + "rawSql": "with workflows as (\n select\n *,\n case when run1 like 'success' then 1 when run1 like 'in_progress' then 2 when run1 like 'queued' then 2 when run1 like 'waiting' then 2 when run1 like 'cancelled' then 3 when run1 like 'None' then 4 else 0 end as run_1,\n case when run2 like 'success' then 1 when run2 like 'in_progress' then 2 when run2 like 'queued' then 2 when run2 like 'waiting' then 2 when run2 like 'cancelled' then 3 when run2 like 'None' then 4 else 0 end as run_2,\n case when run3 like 'success' then 1 when run3 like 'in_progress' then 2 when run3 like 'queued' then 2 when run3 like 'waiting' then 2 when run3 like 'cancelled' then 3 when run3 like 'None' then 4 else 0 end as run_3,\n case when run4 like 'success' then 1 when run4 like 'in_progress' then 2 when run4 like 'queued' then 2 when run4 like 'waiting' then 2 when run4 like 'cancelled' then 3 when run4 like 'None' then 4 else 0 end as run_4,\n case when run5 like 'success' then 1 when run5 like 'in_progress' then 2 when run5 like 'queued' then 2 when run5 like 'waiting' then 2 when run5 like 'cancelled' then 3 when run5 like 'None' then 4 else 0 end as run_5,\n case when run6 like 'success' then 1 when run6 like 'in_progress' then 2 when run6 like 'queued' then 2 when run6 like 'waiting' then 2 when run6 like 'cancelled' then 3 when run6 like 'None' then 4 else 0 end as run_6,\n case when run7 like 'success' then 1 when run7 like 'in_progress' then 2 when run7 like 'queued' then 2 when run7 like 'waiting' then 2 when run7 like 'cancelled' then 3 when run7 like 'None' then 4 else 0 end as run_7,\n case when run8 like 'success' then 1 when run8 like 'in_progress' then 2 when run8 like 'queued' then 2 when run8 like 'waiting' then 2 when run8 like 'cancelled' then 3 when run8 like 'None' then 4 else 0 end as run_8,\n case when run9 like 'success' then 1 when run9 like 'in_progress' then 2 when run9 like 'queued' then 2 when run9 like 'waiting' then 2 when run9 like 'cancelled' then 3 when run9 like 'None' then 4 else 0 end as run_9,\n case when run10 like 'success' then 1 when run10 like 'in_progress' then 2 when run10 like 'queued' then 2 when run10 like 'waiting' then 2 when run10 like 'cancelled' then 3 when run10 like 'None' then 4 else 0 end as run_10\n from\n github_workflows\n where\n dashboard_category = 'core_go'\n)\nselect\n job_name,\n job_yml_filename,\n run_1,\n run1Id,\n run_2,\n run2Id,\n run_3,\n run3Id,\n run_4,\n run4Id,\n run_5,\n run5Id,\n run_6,\n run6Id,\n run_7,\n run7Id,\n run_8,\n run8Id,\n run_9,\n run9Id,\n run_10,\n run10Id\nfrom\n workflows;", + "refId": "A", + "select": [ + [ + { + "params": [ + "value" + ], + "type": "column" + } + ] + ], + "sql": { + "columns": [ + { + "parameters": [], + "type": "function" + } + ], + "groupBy": [ + { + "property": { + "type": "string" + }, + "type": "groupBy" + } + ], + "limit": 50 + }, + "timeColumn": "time", + "units": "none", + "valueHandler": "Number Threshold", + "where": [ + { + "name": "$__timeFilter", + "params": [], + "type": "macro" + } + ] + } + ], + "title": "Core Go Tests", + "transformations": [ + { + "id": "merge", + "options": { + "reducers": [] + } + } + ], + "type": "table" + }, + { + "datasource": { + "type": "postgres", + "uid": "fK5onAOVz" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "thresholds" + }, + "custom": { + "align": "auto", + "cellOptions": { + "type": "auto" + }, + "inspect": false + }, + "mappings": [ + { + "options": { + "0": { + "color": "semi-dark-red", + "index": 0, + "text": "Fail" + }, + "1": { + "color": "semi-dark-green", + "index": 1, + "text": "Success" + }, + "2": { + "color": "semi-dark-yellow", + "index": 2, + "text": "Pending" + }, + "3": { + "color": "semi-dark-purple", + "index": 3, + "text": "Cancelled" + }, + "4": { + "color": "light-blue", + "index": 4, + "text": "None" + } + }, + "type": "value" + } + ], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "semi-dark-red", + "value": null + } + ] + } + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "job_name" + }, + "properties": [ + { + "id": "unit", + "value": "short" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "links", + "value": [ + { + "targetBlank": true, + "title": "", + "url": "https://github.com/apache/beam/actions/${__data.fields.job_yml_filename}" + } + ] + }, + { + "id": "custom.align", + "value": "auto" + }, + { + "id": "mappings", + "value": [] + }, + { + "id": "displayName", + "value": "Job Name" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "job_yml_filename" + }, + "properties": [ + { + "id": "custom.hidden", + "value": true + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/run\\d+id/" + }, + "properties": [ + { + "id": "custom.hidden", + "value": true + }, + { + "id": "custom.align", + "value": "center" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_1" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run1id}" + } + ] + }, + { + "id": "displayName", + "value": "Run" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_2" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run2id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 2" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_3" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run3id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 3" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_4" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run4id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 4" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_5" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run5id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 5" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_6" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run6id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 6" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_7" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run7id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 7" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_8" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run8id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 8" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_9" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run9id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 9" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_10" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run10id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 10" + } + ] + } + ] + }, + "gridPos": { + "h": 20, + "w": 24, + "x": 0, + "y": 180 + }, + "id": 6, + "links": [], + "options": { + "cellHeight": "sm", + "footer": { + "countRows": false, + "fields": "", + "reducer": [ + "sum" + ], + "show": false + }, + "showHeader": true + }, + "pluginVersion": "10.0.3", + "targets": [ + { + "aggregation": "Last", + "alias": "job", + "datasource": { + "type": "postgres", + "uid": "fK5onAOVz" + }, + "decimals": 2, + "displayAliasType": "Warning / Critical", + "displayType": "Regular", + "displayValueWithAlias": "Never", + "editorMode": "code", + "format": "table", + "group": [], + "hide": false, + "metricColumn": "none", + "rawQuery": true, + "rawSql": "with workflows as (\n select\n *,\n case when run1 like 'success' then 1 when run1 like 'in_progress' then 2 when run1 like 'queued' then 2 when run1 like 'waiting' then 2 when run1 like 'cancelled' then 3 when run1 like 'None' then 4 else 0 end as run_1,\n case when run2 like 'success' then 1 when run2 like 'in_progress' then 2 when run2 like 'queued' then 2 when run2 like 'waiting' then 2 when run2 like 'cancelled' then 3 when run2 like 'None' then 4 else 0 end as run_2,\n case when run3 like 'success' then 1 when run3 like 'in_progress' then 2 when run3 like 'queued' then 2 when run3 like 'waiting' then 2 when run3 like 'cancelled' then 3 when run3 like 'None' then 4 else 0 end as run_3,\n case when run4 like 'success' then 1 when run4 like 'in_progress' then 2 when run4 like 'queued' then 2 when run4 like 'waiting' then 2 when run4 like 'cancelled' then 3 when run4 like 'None' then 4 else 0 end as run_4,\n case when run5 like 'success' then 1 when run5 like 'in_progress' then 2 when run5 like 'queued' then 2 when run5 like 'waiting' then 2 when run5 like 'cancelled' then 3 when run5 like 'None' then 4 else 0 end as run_5,\n case when run6 like 'success' then 1 when run6 like 'in_progress' then 2 when run6 like 'queued' then 2 when run6 like 'waiting' then 2 when run6 like 'cancelled' then 3 when run6 like 'None' then 4 else 0 end as run_6,\n case when run7 like 'success' then 1 when run7 like 'in_progress' then 2 when run7 like 'queued' then 2 when run7 like 'waiting' then 2 when run7 like 'cancelled' then 3 when run7 like 'None' then 4 else 0 end as run_7,\n case when run8 like 'success' then 1 when run8 like 'in_progress' then 2 when run8 like 'queued' then 2 when run8 like 'waiting' then 2 when run8 like 'cancelled' then 3 when run8 like 'None' then 4 else 0 end as run_8,\n case when run9 like 'success' then 1 when run9 like 'in_progress' then 2 when run9 like 'queued' then 2 when run9 like 'waiting' then 2 when run9 like 'cancelled' then 3 when run9 like 'None' then 4 else 0 end as run_9,\n case when run10 like 'success' then 1 when run10 like 'in_progress' then 2 when run10 like 'queued' then 2 when run10 like 'waiting' then 2 when run10 like 'cancelled' then 3 when run10 like 'None' then 4 else 0 end as run_10\n from\n github_workflows\n where\n dashboard_category = 'dataflow_go'\n)\nselect\n job_name,\n job_yml_filename,\n run_1,\n run1Id,\n run_2,\n run2Id,\n run_3,\n run3Id,\n run_4,\n run4Id,\n run_5,\n run5Id,\n run_6,\n run6Id,\n run_7,\n run7Id,\n run_8,\n run8Id,\n run_9,\n run9Id,\n run_10,\n run10Id\nfrom\n workflows;", + "refId": "A", + "select": [ + [ + { + "params": [ + "value" + ], + "type": "column" + } + ] + ], + "sql": { + "columns": [ + { + "parameters": [], + "type": "function" + } + ], + "groupBy": [ + { + "property": { + "type": "string" + }, + "type": "groupBy" + } + ], + "limit": 50 + }, + "timeColumn": "time", + "units": "none", + "valueHandler": "Number Threshold", + "where": [ + { + "name": "$__timeFilter", + "params": [], + "type": "macro" + } + ] + } + ], + "title": "Dataflow Go Tests", + "transformations": [ + { + "id": "merge", + "options": { + "reducers": [] + } + } + ], + "type": "table" + }, + { + "datasource": { + "type": "postgres", + "uid": "fK5onAOVz" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "thresholds" + }, + "custom": { + "align": "auto", + "cellOptions": { + "type": "auto" + }, + "inspect": false + }, + "mappings": [ + { + "options": { + "0": { + "color": "semi-dark-red", + "index": 0, + "text": "Fail" + }, + "1": { + "color": "semi-dark-green", + "index": 1, + "text": "Success" + }, + "2": { + "color": "semi-dark-yellow", + "index": 2, + "text": "Pending" + }, + "3": { + "color": "semi-dark-purple", + "index": 3, + "text": "Cancelled" + }, + "4": { + "color": "light-blue", + "index": 4, + "text": "None" + } + }, + "type": "value" + } + ], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "semi-dark-red", + "value": null + } + ] + } + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "job_name" + }, + "properties": [ + { + "id": "unit", + "value": "short" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "links", + "value": [ + { + "targetBlank": true, + "title": "", + "url": "https://github.com/apache/beam/actions/${__data.fields.job_yml_filename}" + } + ] + }, + { + "id": "custom.align", + "value": "auto" + }, + { + "id": "mappings", + "value": [] + }, + { + "id": "displayName", + "value": "Job Name" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "job_yml_filename" + }, + "properties": [ + { + "id": "custom.hidden", + "value": true + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/run\\d+id/" + }, + "properties": [ + { + "id": "custom.hidden", + "value": true + }, + { + "id": "custom.align", + "value": "center" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_1" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run1id}" + } + ] + }, + { + "id": "displayName", + "value": "Run" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_2" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run2id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 2" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_3" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run3id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 3" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_4" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run4id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 4" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_5" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run5id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 5" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_6" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run6id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 6" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_7" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run7id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 7" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_8" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run8id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 8" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_9" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run9id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 9" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_10" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run10id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 10" + } + ] + } + ] + }, + "gridPos": { + "h": 20, + "w": 24, + "x": 0, + "y": 200 + }, + "id": 5, + "links": [], + "options": { + "cellHeight": "sm", + "footer": { + "countRows": false, + "fields": "", + "reducer": [ + "sum" + ], + "show": false + }, + "showHeader": true + }, + "pluginVersion": "10.0.3", + "targets": [ + { + "aggregation": "Last", + "alias": "job", + "datasource": { + "type": "postgres", + "uid": "fK5onAOVz" + }, + "decimals": 2, + "displayAliasType": "Warning / Critical", + "displayType": "Regular", + "displayValueWithAlias": "Never", + "editorMode": "code", + "format": "table", + "group": [], + "hide": false, + "metricColumn": "none", + "rawQuery": true, + "rawSql": "with workflows as (\n select\n *,\n case when run1 like 'success' then 1 when run1 like 'in_progress' then 2 when run1 like 'queued' then 2 when run1 like 'waiting' then 2 when run1 like 'cancelled' then 3 when run1 like 'None' then 4 else 0 end as run_1,\n case when run2 like 'success' then 1 when run2 like 'in_progress' then 2 when run2 like 'queued' then 2 when run2 like 'waiting' then 2 when run2 like 'cancelled' then 3 when run2 like 'None' then 4 else 0 end as run_2,\n case when run3 like 'success' then 1 when run3 like 'in_progress' then 2 when run3 like 'queued' then 2 when run3 like 'waiting' then 2 when run3 like 'cancelled' then 3 when run3 like 'None' then 4 else 0 end as run_3,\n case when run4 like 'success' then 1 when run4 like 'in_progress' then 2 when run4 like 'queued' then 2 when run4 like 'waiting' then 2 when run4 like 'cancelled' then 3 when run4 like 'None' then 4 else 0 end as run_4,\n case when run5 like 'success' then 1 when run5 like 'in_progress' then 2 when run5 like 'queued' then 2 when run5 like 'waiting' then 2 when run5 like 'cancelled' then 3 when run5 like 'None' then 4 else 0 end as run_5,\n case when run6 like 'success' then 1 when run6 like 'in_progress' then 2 when run6 like 'queued' then 2 when run6 like 'waiting' then 2 when run6 like 'cancelled' then 3 when run6 like 'None' then 4 else 0 end as run_6,\n case when run7 like 'success' then 1 when run7 like 'in_progress' then 2 when run7 like 'queued' then 2 when run7 like 'waiting' then 2 when run7 like 'cancelled' then 3 when run7 like 'None' then 4 else 0 end as run_7,\n case when run8 like 'success' then 1 when run8 like 'in_progress' then 2 when run8 like 'queued' then 2 when run8 like 'waiting' then 2 when run8 like 'cancelled' then 3 when run8 like 'None' then 4 else 0 end as run_8,\n case when run9 like 'success' then 1 when run9 like 'in_progress' then 2 when run9 like 'queued' then 2 when run9 like 'waiting' then 2 when run9 like 'cancelled' then 3 when run9 like 'None' then 4 else 0 end as run_9,\n case when run10 like 'success' then 1 when run10 like 'in_progress' then 2 when run10 like 'queued' then 2 when run10 like 'waiting' then 2 when run10 like 'cancelled' then 3 when run10 like 'None' then 4 else 0 end as run_10\n from\n github_workflows\n where\n dashboard_category = 'spark_go'\n)\nselect\n job_name,\n job_yml_filename,\n run_1,\n run1Id,\n run_2,\n run2Id,\n run_3,\n run3Id,\n run_4,\n run4Id,\n run_5,\n run5Id,\n run_6,\n run6Id,\n run_7,\n run7Id,\n run_8,\n run8Id,\n run_9,\n run9Id,\n run_10,\n run10Id\nfrom\n workflows;", + "refId": "A", + "select": [ + [ + { + "params": [ + "value" + ], + "type": "column" + } + ] + ], + "sql": { + "columns": [ + { + "parameters": [], + "type": "function" + } + ], + "groupBy": [ + { + "property": { + "type": "string" + }, + "type": "groupBy" + } + ], + "limit": 50 + }, + "timeColumn": "time", + "units": "none", + "valueHandler": "Number Threshold", + "where": [ + { + "name": "$__timeFilter", + "params": [], + "type": "macro" + } + ] + } + ], + "title": "Spark Go Tests", + "transformations": [ + { + "id": "merge", + "options": { + "reducers": [] + } + } + ], + "type": "table" + }, + { + "datasource": { + "type": "postgres", + "uid": "fK5onAOVz" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "thresholds" + }, + "custom": { + "align": "auto", + "cellOptions": { + "type": "auto" + }, + "inspect": false + }, + "mappings": [ + { + "options": { + "0": { + "color": "semi-dark-red", + "index": 0, + "text": "Fail" + }, + "1": { + "color": "semi-dark-green", + "index": 1, + "text": "Success" + }, + "2": { + "color": "semi-dark-yellow", + "index": 2, + "text": "Pending" + }, + "3": { + "color": "semi-dark-purple", + "index": 3, + "text": "Cancelled" + }, + "4": { + "color": "light-blue", + "index": 4, + "text": "None" + } + }, + "type": "value" + } + ], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "semi-dark-red", + "value": null + } + ] + } + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "job_name" + }, + "properties": [ + { + "id": "unit", + "value": "short" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "links", + "value": [ + { + "targetBlank": true, + "title": "", + "url": "https://github.com/apache/beam/actions/${__data.fields.job_yml_filename}" + } + ] + }, + { + "id": "custom.align", + "value": "auto" + }, + { + "id": "mappings", + "value": [] + }, + { + "id": "displayName", + "value": "Job Name" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "job_yml_filename" + }, + "properties": [ + { + "id": "custom.hidden", + "value": true + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/run\\d+id/" + }, + "properties": [ + { + "id": "custom.hidden", + "value": true + }, + { + "id": "custom.align", + "value": "center" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_1" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run1id}" + } + ] + }, + { + "id": "displayName", + "value": "Run" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_2" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run2id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 2" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_3" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run3id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 3" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_4" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run4id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 4" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_5" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run5id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 5" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_6" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run6id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 6" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_7" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run7id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 7" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_8" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run8id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 8" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_9" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run9id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 9" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_10" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run10id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 10" + } + ] + } + ] + }, + "gridPos": { + "h": 20, + "w": 24, + "x": 0, + "y": 220 + }, + "id": 4, + "links": [], + "options": { + "cellHeight": "sm", + "footer": { + "countRows": false, + "fields": "", + "reducer": [ + "sum" + ], + "show": false + }, + "showHeader": true + }, + "pluginVersion": "10.0.3", + "targets": [ + { + "aggregation": "Last", + "alias": "job", + "datasource": { + "type": "postgres", + "uid": "fK5onAOVz" + }, + "decimals": 2, + "displayAliasType": "Warning / Critical", + "displayType": "Regular", + "displayValueWithAlias": "Never", + "editorMode": "code", + "format": "table", + "group": [], + "hide": false, + "metricColumn": "none", + "rawQuery": true, + "rawSql": "with workflows as (\n select\n *,\n case when run1 like 'success' then 1 when run1 like 'in_progress' then 2 when run1 like 'queued' then 2 when run1 like 'waiting' then 2 when run1 like 'cancelled' then 3 when run1 like 'None' then 4 else 0 end as run_1,\n case when run2 like 'success' then 1 when run2 like 'in_progress' then 2 when run2 like 'queued' then 2 when run2 like 'waiting' then 2 when run2 like 'cancelled' then 3 when run2 like 'None' then 4 else 0 end as run_2,\n case when run3 like 'success' then 1 when run3 like 'in_progress' then 2 when run3 like 'queued' then 2 when run3 like 'waiting' then 2 when run3 like 'cancelled' then 3 when run3 like 'None' then 4 else 0 end as run_3,\n case when run4 like 'success' then 1 when run4 like 'in_progress' then 2 when run4 like 'queued' then 2 when run4 like 'waiting' then 2 when run4 like 'cancelled' then 3 when run4 like 'None' then 4 else 0 end as run_4,\n case when run5 like 'success' then 1 when run5 like 'in_progress' then 2 when run5 like 'queued' then 2 when run5 like 'waiting' then 2 when run5 like 'cancelled' then 3 when run5 like 'None' then 4 else 0 end as run_5,\n case when run6 like 'success' then 1 when run6 like 'in_progress' then 2 when run6 like 'queued' then 2 when run6 like 'waiting' then 2 when run6 like 'cancelled' then 3 when run6 like 'None' then 4 else 0 end as run_6,\n case when run7 like 'success' then 1 when run7 like 'in_progress' then 2 when run7 like 'queued' then 2 when run7 like 'waiting' then 2 when run7 like 'cancelled' then 3 when run7 like 'None' then 4 else 0 end as run_7,\n case when run8 like 'success' then 1 when run8 like 'in_progress' then 2 when run8 like 'queued' then 2 when run8 like 'waiting' then 2 when run8 like 'cancelled' then 3 when run8 like 'None' then 4 else 0 end as run_8,\n case when run9 like 'success' then 1 when run9 like 'in_progress' then 2 when run9 like 'queued' then 2 when run9 like 'waiting' then 2 when run9 like 'cancelled' then 3 when run9 like 'None' then 4 else 0 end as run_9,\n case when run10 like 'success' then 1 when run10 like 'in_progress' then 2 when run10 like 'queued' then 2 when run10 like 'waiting' then 2 when run10 like 'cancelled' then 3 when run10 like 'None' then 4 else 0 end as run_10\n from\n github_workflows\n where\n dashboard_category = 'flink_go'\n)\nselect\n job_name,\n job_yml_filename,\n run_1,\n run1Id,\n run_2,\n run2Id,\n run_3,\n run3Id,\n run_4,\n run4Id,\n run_5,\n run5Id,\n run_6,\n run6Id,\n run_7,\n run7Id,\n run_8,\n run8Id,\n run_9,\n run9Id,\n run_10,\n run10Id\nfrom\n workflows;", + "refId": "A", + "select": [ + [ + { + "params": [ + "value" + ], + "type": "column" + } + ] + ], + "sql": { + "columns": [ + { + "parameters": [], + "type": "function" + } + ], + "groupBy": [ + { + "property": { + "type": "string" + }, + "type": "groupBy" + } + ], + "limit": 50 + }, + "timeColumn": "time", + "units": "none", + "valueHandler": "Number Threshold", + "where": [ + { + "name": "$__timeFilter", + "params": [], + "type": "macro" + } + ] + } + ], + "title": "Flink Go Tests", + "transformations": [ + { + "id": "merge", + "options": { + "reducers": [] + } + } + ], + "type": "table" + }, + { + "datasource": { + "type": "postgres", + "uid": "fK5onAOVz" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "thresholds" + }, + "custom": { + "align": "auto", + "cellOptions": { + "type": "auto" + }, + "inspect": false + }, + "mappings": [ + { + "options": { + "0": { + "color": "semi-dark-red", + "index": 0, + "text": "Fail" + }, + "1": { + "color": "semi-dark-green", + "index": 1, + "text": "Success" + }, + "2": { + "color": "semi-dark-yellow", + "index": 2, + "text": "Pending" + }, + "3": { + "color": "semi-dark-purple", + "index": 3, + "text": "Cancelled" + }, + "4": { + "color": "light-blue", + "index": 4, + "text": "None" + } + }, + "type": "value" + } + ], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "semi-dark-red", + "value": null + } + ] + } + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "job_name" + }, + "properties": [ + { + "id": "unit", + "value": "short" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "links", + "value": [ + { + "targetBlank": true, + "title": "", + "url": "https://github.com/apache/beam/actions/${__data.fields.job_yml_filename}" + } + ] + }, + { + "id": "custom.align", + "value": "auto" + }, + { + "id": "mappings", + "value": [] + }, + { + "id": "displayName", + "value": "Job Name" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "job_yml_filename" + }, + "properties": [ + { + "id": "custom.hidden", + "value": true + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/run\\d+id/" + }, + "properties": [ + { + "id": "custom.hidden", + "value": true + }, + { + "id": "custom.align", + "value": "center" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_1" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run1id}" + } + ] + }, + { + "id": "displayName", + "value": "Run" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_2" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run2id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 2" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_3" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run3id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 3" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_4" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run4id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 4" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_5" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run5id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 5" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_6" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run6id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 6" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_7" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run7id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 7" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_8" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run8id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 8" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_9" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run9id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 9" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "run_10" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-green", + "mode": "fixed" + } + }, + { + "id": "custom.hidden", + "value": false + }, + { + "id": "thresholds", + "value": { + "mode": "percentage", + "steps": [ + { + "color": "semi-dark-red", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + }, + { + "color": "semi-dark-yellow", + "value": 2 + }, + { + "color": "light-blue", + "value": 3 + }, + { + "color": "purple", + "value": 4 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "custom.align", + "value": "center" + }, + { + "id": "links", + "value": [ + { + "title": "", + "url": "${__data.fields.run10id}" + } + ] + }, + { + "id": "displayName", + "value": "Run 10" + } + ] + } + ] + }, + "gridPos": { + "h": 20, + "w": 24, + "x": 0, + "y": 240 + }, + "id": 14, + "links": [], + "options": { + "cellHeight": "sm", + "footer": { + "countRows": false, + "fields": "", + "reducer": [ + "sum" + ], + "show": false + }, + "showHeader": true + }, + "pluginVersion": "10.0.3", + "targets": [ + { + "aggregation": "Last", + "alias": "job", + "datasource": { + "type": "postgres", + "uid": "fK5onAOVz" + }, + "decimals": 2, + "displayAliasType": "Warning / Critical", + "displayType": "Regular", + "displayValueWithAlias": "Never", + "editorMode": "code", + "format": "table", + "group": [], + "hide": false, + "metricColumn": "none", + "rawQuery": true, + "rawSql": "with workflows as (\n select\n *,\n case when run1 like 'success' then 1 when run1 like 'in_progress' then 2 when run1 like 'queued' then 2 when run1 like 'waiting' then 2 when run1 like 'cancelled' then 3 when run1 like 'None' then 4 else 0 end as run_1,\n case when run2 like 'success' then 1 when run2 like 'in_progress' then 2 when run2 like 'queued' then 2 when run2 like 'waiting' then 2 when run2 like 'cancelled' then 3 when run2 like 'None' then 4 else 0 end as run_2,\n case when run3 like 'success' then 1 when run3 like 'in_progress' then 2 when run3 like 'queued' then 2 when run3 like 'waiting' then 2 when run3 like 'cancelled' then 3 when run3 like 'None' then 4 else 0 end as run_3,\n case when run4 like 'success' then 1 when run4 like 'in_progress' then 2 when run4 like 'queued' then 2 when run4 like 'waiting' then 2 when run4 like 'cancelled' then 3 when run4 like 'None' then 4 else 0 end as run_4,\n case when run5 like 'success' then 1 when run5 like 'in_progress' then 2 when run5 like 'queued' then 2 when run5 like 'waiting' then 2 when run5 like 'cancelled' then 3 when run5 like 'None' then 4 else 0 end as run_5,\n case when run6 like 'success' then 1 when run6 like 'in_progress' then 2 when run6 like 'queued' then 2 when run6 like 'waiting' then 2 when run6 like 'cancelled' then 3 when run6 like 'None' then 4 else 0 end as run_6,\n case when run7 like 'success' then 1 when run7 like 'in_progress' then 2 when run7 like 'queued' then 2 when run7 like 'waiting' then 2 when run7 like 'cancelled' then 3 when run7 like 'None' then 4 else 0 end as run_7,\n case when run8 like 'success' then 1 when run8 like 'in_progress' then 2 when run8 like 'queued' then 2 when run8 like 'waiting' then 2 when run8 like 'cancelled' then 3 when run8 like 'None' then 4 else 0 end as run_8,\n case when run9 like 'success' then 1 when run9 like 'in_progress' then 2 when run9 like 'queued' then 2 when run9 like 'waiting' then 2 when run9 like 'cancelled' then 3 when run9 like 'None' then 4 else 0 end as run_9,\n case when run10 like 'success' then 1 when run10 like 'in_progress' then 2 when run10 like 'queued' then 2 when run10 like 'waiting' then 2 when run10 like 'cancelled' then 3 when run10 like 'None' then 4 else 0 end as run_10\n from\n github_workflows\n where\n dashboard_category = 'misc'\n)\nselect\n job_name,\n job_yml_filename,\n run_1,\n run1Id,\n run_2,\n run2Id,\n run_3,\n run3Id,\n run_4,\n run4Id,\n run_5,\n run5Id,\n run_6,\n run6Id,\n run_7,\n run7Id,\n run_8,\n run8Id,\n run_9,\n run9Id,\n run_10,\n run10Id\nfrom\n workflows;", + "refId": "A", + "select": [ + [ + { + "params": [ + "value" + ], + "type": "column" + } + ] + ], + "sql": { + "columns": [ + { + "parameters": [], + "type": "function" + } + ], + "groupBy": [ + { + "property": { + "type": "string" + }, + "type": "groupBy" + } + ], + "limit": 50 + }, + "timeColumn": "time", + "units": "none", + "valueHandler": "Number Threshold", + "where": [ + { + "name": "$__timeFilter", + "params": [], + "type": "macro" + } + ] + } + ], + "title": "Misc Tests", + "transformations": [ + { + "id": "merge", + "options": { + "reducers": [] + } + } + ], + "type": "table" } ], - "schemaVersion": 30, + "refresh": "", + "schemaVersion": 38, "style": "dark", "tags": [], "templating": { @@ -948,5 +12641,6 @@ "timezone": "", "title": "GA Post-Commits Status", "uid": "CTYdoxP4z", - "version": 1 -} + "version": 2, + "weekStart": "" +} \ No newline at end of file diff --git a/.test-infra/metrics/sync/github/sync_workflows.py b/.test-infra/metrics/sync/github/sync_workflows.py index 25065878c4bd..1e1b3b004f62 100644 --- a/.test-infra/metrics/sync/github/sync_workflows.py +++ b/.test-infra/metrics/sync/github/sync_workflows.py @@ -55,6 +55,26 @@ def __init__(self, id, name, filename): self.filename = filename self.runs = [] +def get_dashboard_category(workflow_name): + workflow_name = workflow_name.lower() + lang = '' + if 'java' in workflow_name: + lang = 'java' + elif 'python' in workflow_name: + lang = 'python' + elif 'go' in workflow_name: + lang = 'go' + else: + # If no language found, toss in miscellaneous bucket + return 'misc' + if 'dataflow' in workflow_name: + return f'dataflow_{lang}' + if 'spark' in workflow_name: + return f'spark_{lang}' + if 'flink' in workflow_name: + return f'flink_{lang}' + return f'core_{lang}' + async def github_workflows_dashboard_sync(): print('Started') print('Updating table with recent workflow runs') @@ -247,7 +267,8 @@ def database_operations(connection, workflows): CREATE TABLE IF NOT EXISTS {workflows_table_name} ( workflow_id integer NOT NULL PRIMARY KEY, job_name text NOT NULL, - job_yml_filename text NOT NULL""" + job_yml_filename text NOT NULL, + dashboard_category text NOT NULL""" for i in range(int(GH_NUMBER_OF_WORKFLOW_RUNS_TO_FETCH)): create_table_query += f""", run{i+1} text, @@ -256,12 +277,14 @@ def database_operations(connection, workflows): cursor.execute(create_table_query) insert_query = f"INSERT INTO {workflows_table_name} VALUES " for workflow in workflows: + category = get_dashboard_category(workflow.name) row_insert =\ - f"(\'{workflow.id}\',\'{workflow.name}\',\'{workflow.filename}\'" + f"(\'{workflow.id}\',\'{workflow.name}\',\'{workflow.filename}\',\'{category}\'" for _, status, url in workflow.runs: row_insert += f",\'{status}\',\'{url}\'" insert_query += f"{row_insert})," insert_query = insert_query[:-1] + ";" + print(insert_query) cursor.execute(insert_query) cursor.close() connection.commit() From 97b14a16f365105dd14ca9c06088e86e1ce5fd1b Mon Sep 17 00:00:00 2001 From: Andrey Devyatkin <andrey.9evyatkin@gmail.com> Date: Fri, 27 Oct 2023 14:30:46 +0200 Subject: [PATCH 267/435] Added conditions to publish email (#29138) * Add Send email step * add consitions for publishing * add consitions for publishing --------- Co-authored-by: vitaly.terentyev <vitaly.terentyev@akvelon.com> --- .github/workflows/beam_IODatastoresCredentialsRotation.yml | 2 ++ .github/workflows/beam_MetricsCredentialsRotation.yml | 2 ++ 2 files changed, 4 insertions(+) diff --git a/.github/workflows/beam_IODatastoresCredentialsRotation.yml b/.github/workflows/beam_IODatastoresCredentialsRotation.yml index 0ededda00b39..17a32a4ecdff 100644 --- a/.github/workflows/beam_IODatastoresCredentialsRotation.yml +++ b/.github/workflows/beam_IODatastoresCredentialsRotation.yml @@ -78,11 +78,13 @@ jobs: run: | gcloud container clusters update io-datastores --complete-credential-rotation --zone=us-central1-a --quiet - name: Generate Date + if: failure() run: | date=$(date -u +"%Y-%m-%d") echo "date=$date" >> $GITHUB_ENV - name: Send email uses: dawidd6/action-send-mail@v3 + if: failure() with: server_address: smtp.gmail.com server_port: 465 diff --git a/.github/workflows/beam_MetricsCredentialsRotation.yml b/.github/workflows/beam_MetricsCredentialsRotation.yml index eda3ec355631..ea5aaeefed25 100644 --- a/.github/workflows/beam_MetricsCredentialsRotation.yml +++ b/.github/workflows/beam_MetricsCredentialsRotation.yml @@ -79,11 +79,13 @@ jobs: run: | gcloud container clusters update metrics --complete-credential-rotation --zone=us-central1-a --quiet - name: Generate Date + if: failure() run: | date=$(date -u +"%Y-%m-%d") echo "date=$date" >> $GITHUB_ENV - name: Send email uses: dawidd6/action-send-mail@v3 + if: failure() with: server_address: smtp.gmail.com server_port: 465 From db664d8125b3ed6cf989d9586c98df8dcdb585f0 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 27 Oct 2023 09:50:02 -0400 Subject: [PATCH 268/435] Bump github.com/google/uuid from 1.3.1 to 1.4.0 in /sdks (#29159) Bumps [github.com/google/uuid](https://github.com/google/uuid) from 1.3.1 to 1.4.0. - [Release notes](https://github.com/google/uuid/releases) - [Changelog](https://github.com/google/uuid/blob/master/CHANGELOG.md) - [Commits](https://github.com/google/uuid/compare/v1.3.1...v1.4.0) --- updated-dependencies: - dependency-name: github.com/google/uuid dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] <support@github.com> Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 2 +- sdks/go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 6cd7b77be91d..49968e3227d4 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -41,7 +41,7 @@ require ( github.com/go-sql-driver/mysql v1.7.1 github.com/golang/protobuf v1.5.3 // TODO(danoliveira): Fully replace this with google.golang.org/protobuf github.com/google/go-cmp v0.6.0 - github.com/google/uuid v1.3.1 + github.com/google/uuid v1.4.0 github.com/johannesboyne/gofakes3 v0.0.0-20221110173912-32fb85c5aed6 github.com/lib/pq v1.10.9 github.com/linkedin/goavro/v2 v2.12.0 diff --git a/sdks/go.sum b/sdks/go.sum index 27a55827c72b..05fc3076eb20 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -278,8 +278,8 @@ github.com/google/s2a-go v0.1.7 h1:60BLSyTrOV4/haCDW4zb1guZItoSq8foHCXrAnjBo/o= github.com/google/s2a-go v0.1.7/go.mod h1:50CgR4k1jNlWBu4UfS4AcfhVe1r6pdZPygJ3R8F0Qdw= github.com/google/uuid v1.1.2/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.2.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= -github.com/google/uuid v1.3.1 h1:KjJaJ9iWZ3jOFZIf1Lqf4laDRCasjl0BCmnEGxkdLb4= -github.com/google/uuid v1.3.1/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= +github.com/google/uuid v1.4.0 h1:MtMxsa51/r9yyhkyLsVeVt0B+BGQZzpQiTQ4eHZ8bc4= +github.com/google/uuid v1.4.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/googleapis/enterprise-certificate-proxy v0.3.1 h1:SBWmZhjUDRorQxrN0nwzf+AHBxnbFjViHQS4P0yVpmQ= github.com/googleapis/enterprise-certificate-proxy v0.3.1/go.mod h1:VLSiSSBs/ksPL8kq3OBOQ6WRI2QnaFynd1DCjZ62+V0= github.com/googleapis/gax-go/v2 v2.0.4/go.mod h1:0Wqv26UfaUD9n4G6kQubkQ+KchISgw+vpHVxEJEs9eg= From 7479aececf7ae1123bd73cf19b7a22e52c16e450 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 27 Oct 2023 10:12:50 -0400 Subject: [PATCH 269/435] Bump actions/setup-node from 3 to 4 (#29115) Bumps [actions/setup-node](https://github.com/actions/setup-node) from 3 to 4. - [Release notes](https://github.com/actions/setup-node/releases) - [Commits](https://github.com/actions/setup-node/compare/v3...v4) --- updated-dependencies: - dependency-name: actions/setup-node dependency-type: direct:production update-type: version-update:semver-major ... Signed-off-by: dependabot[bot] <support@github.com> Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .github/workflows/build_release_candidate.yml | 2 +- .github/workflows/pr-bot-new-prs.yml | 2 +- .github/workflows/pr-bot-pr-updates.yml | 2 +- .github/workflows/pr-bot-prs-needing-attention.yml | 2 +- .github/workflows/pr-bot-update-reviewers.yml | 2 +- .github/workflows/reportGenerator.yml | 2 +- .github/workflows/typescript_tests.yml | 6 +++--- 7 files changed, 9 insertions(+), 9 deletions(-) diff --git a/.github/workflows/build_release_candidate.yml b/.github/workflows/build_release_candidate.yml index 12f1537dac18..68456bb19fcb 100644 --- a/.github/workflows/build_release_candidate.yml +++ b/.github/workflows/build_release_candidate.yml @@ -291,7 +291,7 @@ jobs: with: python-version: '3.8' - name: Install node - uses: actions/setup-node@v3 + uses: actions/setup-node@v4 with: node-version: '16' - name: Install Java 8 diff --git a/.github/workflows/pr-bot-new-prs.yml b/.github/workflows/pr-bot-new-prs.yml index ef825e067b7d..0f17d662db9c 100644 --- a/.github/workflows/pr-bot-new-prs.yml +++ b/.github/workflows/pr-bot-new-prs.yml @@ -35,7 +35,7 @@ jobs: steps: - uses: actions/checkout@v4 - name: Setup Node - uses: actions/setup-node@v3 + uses: actions/setup-node@v4 with: node-version: 16 - name: Install pr-bot npm dependencies diff --git a/.github/workflows/pr-bot-pr-updates.yml b/.github/workflows/pr-bot-pr-updates.yml index f3734e0adcbf..02c8a2473ff3 100644 --- a/.github/workflows/pr-bot-pr-updates.yml +++ b/.github/workflows/pr-bot-pr-updates.yml @@ -40,7 +40,7 @@ jobs: with: ref: 'master' - name: Setup Node - uses: actions/setup-node@v3 + uses: actions/setup-node@v4 with: node-version: 16 - name: Install pr-bot npm dependencies diff --git a/.github/workflows/pr-bot-prs-needing-attention.yml b/.github/workflows/pr-bot-prs-needing-attention.yml index 9dff7c8565a4..95be91e8dcb4 100644 --- a/.github/workflows/pr-bot-prs-needing-attention.yml +++ b/.github/workflows/pr-bot-prs-needing-attention.yml @@ -35,7 +35,7 @@ jobs: steps: - uses: actions/checkout@v4 - name: Setup Node - uses: actions/setup-node@v3 + uses: actions/setup-node@v4 with: node-version: 16 - name: Install pr-bot npm dependencies diff --git a/.github/workflows/pr-bot-update-reviewers.yml b/.github/workflows/pr-bot-update-reviewers.yml index b4c41b66f9d6..ac80d1e346e6 100644 --- a/.github/workflows/pr-bot-update-reviewers.yml +++ b/.github/workflows/pr-bot-update-reviewers.yml @@ -35,7 +35,7 @@ jobs: steps: - uses: actions/checkout@v4 - name: Setup Node - uses: actions/setup-node@v3 + uses: actions/setup-node@v4 with: node-version: 16 - name: Install pr-bot npm dependencies diff --git a/.github/workflows/reportGenerator.yml b/.github/workflows/reportGenerator.yml index 8f6bccddcfad..e568e84f7d2d 100644 --- a/.github/workflows/reportGenerator.yml +++ b/.github/workflows/reportGenerator.yml @@ -28,7 +28,7 @@ jobs: steps: - uses: actions/checkout@v4 - name: Setup Node - uses: actions/setup-node@v3 + uses: actions/setup-node@v4 with: node-version: 16 - run: | diff --git a/.github/workflows/typescript_tests.yml b/.github/workflows/typescript_tests.yml index edbe8399e7d8..e71834f09ffb 100644 --- a/.github/workflows/typescript_tests.yml +++ b/.github/workflows/typescript_tests.yml @@ -50,7 +50,7 @@ jobs: persist-credentials: false submodules: recursive - name: Install node - uses: actions/setup-node@v3 + uses: actions/setup-node@v4 with: node-version: '16' - run: npm ci @@ -79,7 +79,7 @@ jobs: persist-credentials: false submodules: recursive - name: Install Node - uses: actions/setup-node@v3 + uses: actions/setup-node@v4 with: node-version: '16' - name: Install Python @@ -134,7 +134,7 @@ jobs: persist-credentials: false submodules: recursive - name: Install node - uses: actions/setup-node@v3 + uses: actions/setup-node@v4 with: node-version: '16' - name: Install python From dd853ce0d10fae46c578a1970bd2be5c7658f2e7 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 27 Oct 2023 10:13:17 -0400 Subject: [PATCH 270/435] Bump github.com/aws/aws-sdk-go-v2/config from 1.19.0 to 1.19.1 in /sdks (#29132) Bumps [github.com/aws/aws-sdk-go-v2/config](https://github.com/aws/aws-sdk-go-v2) from 1.19.0 to 1.19.1. - [Release notes](https://github.com/aws/aws-sdk-go-v2/releases) - [Changelog](https://github.com/aws/aws-sdk-go-v2/blob/v1.19.1/CHANGELOG.md) - [Commits](https://github.com/aws/aws-sdk-go-v2/compare/v1.19.0...v1.19.1) --- updated-dependencies: - dependency-name: github.com/aws/aws-sdk-go-v2/config dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] <support@github.com> Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 2 +- sdks/go.sum | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 49968e3227d4..9837c97ab1de 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -31,7 +31,7 @@ require ( cloud.google.com/go/spanner v1.51.0 cloud.google.com/go/storage v1.33.0 github.com/aws/aws-sdk-go-v2 v1.21.2 - github.com/aws/aws-sdk-go-v2/config v1.19.0 + github.com/aws/aws-sdk-go-v2/config v1.19.1 github.com/aws/aws-sdk-go-v2/credentials v1.13.43 github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.91 github.com/aws/aws-sdk-go-v2/service/s3 v1.40.2 diff --git a/sdks/go.sum b/sdks/go.sum index 05fc3076eb20..1b1390f57f66 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -86,8 +86,9 @@ github.com/aws/aws-sdk-go-v2 v1.21.2/go.mod h1:ErQhvNuEMhJjweavOYhxVkn2RUx7kQXVA github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.4.14 h1:Sc82v7tDQ/vdU1WtuSyzZ1I7y/68j//HJ6uozND1IDs= github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.4.14/go.mod h1:9NCTOURS8OpxvoAVHq79LK81/zC78hfRWFn+aL0SPcY= github.com/aws/aws-sdk-go-v2/config v1.5.0/go.mod h1:RWlPOAW3E3tbtNAqTwvSW54Of/yP3oiZXMI0xfUdjyA= -github.com/aws/aws-sdk-go-v2/config v1.19.0 h1:AdzDvwH6dWuVARCl3RTLGRc4Ogy+N7yLFxVxXe1ClQ0= github.com/aws/aws-sdk-go-v2/config v1.19.0/go.mod h1:ZwDUgFnQgsazQTnWfeLWk5GjeqTQTL8lMkoE1UXzxdE= +github.com/aws/aws-sdk-go-v2/config v1.19.1 h1:oe3vqcGftyk40icfLymhhhNysAwk0NfiwkDi2GTPMXs= +github.com/aws/aws-sdk-go-v2/config v1.19.1/go.mod h1:ZwDUgFnQgsazQTnWfeLWk5GjeqTQTL8lMkoE1UXzxdE= github.com/aws/aws-sdk-go-v2/credentials v1.3.1/go.mod h1:r0n73xwsIVagq8RsxmZbGSRQFj9As3je72C2WzUIToc= github.com/aws/aws-sdk-go-v2/credentials v1.13.43 h1:LU8vo40zBlo3R7bAvBVy/ku4nxGEyZe9N8MqAeFTzF8= github.com/aws/aws-sdk-go-v2/credentials v1.13.43/go.mod h1:zWJBz1Yf1ZtX5NGax9ZdNjhhI4rgjfgsyk6vTY1yfVg= From 919441e64c02c8412593e1bc71a53eb9f18da37a Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 27 Oct 2023 10:13:43 -0400 Subject: [PATCH 271/435] Bump google.golang.org/grpc in /.test-infra/mock-apis (#29142) Bumps [google.golang.org/grpc](https://github.com/grpc/grpc-go) from 1.58.2 to 1.58.3. - [Release notes](https://github.com/grpc/grpc-go/releases) - [Commits](https://github.com/grpc/grpc-go/compare/v1.58.2...v1.58.3) --- updated-dependencies: - dependency-name: google.golang.org/grpc dependency-type: direct:production ... Signed-off-by: dependabot[bot] <support@github.com> Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .test-infra/mock-apis/go.mod | 2 +- .test-infra/mock-apis/go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/.test-infra/mock-apis/go.mod b/.test-infra/mock-apis/go.mod index 6f88254dba81..ef2953f27c57 100644 --- a/.test-infra/mock-apis/go.mod +++ b/.test-infra/mock-apis/go.mod @@ -28,7 +28,7 @@ require ( github.com/google/go-cmp v0.5.9 github.com/redis/go-redis/v9 v9.2.1 google.golang.org/genproto/googleapis/api v0.0.0-20230822172742-b8732ec3820d - google.golang.org/grpc v1.58.2 + google.golang.org/grpc v1.58.3 google.golang.org/protobuf v1.31.0 ) diff --git a/.test-infra/mock-apis/go.sum b/.test-infra/mock-apis/go.sum index 56d3a2b193dc..74b587a72449 100644 --- a/.test-infra/mock-apis/go.sum +++ b/.test-infra/mock-apis/go.sum @@ -190,8 +190,8 @@ google.golang.org/grpc v1.33.1/go.mod h1:fr5YgcSWrqhRRxogOsw7RzIpsmvOZ6IcH4kBYTp google.golang.org/grpc v1.33.2/go.mod h1:JMHMWHQWaTccqQQlmk3MJZS+GWXOdAesneDmEnv2fbc= google.golang.org/grpc v1.36.0/go.mod h1:qjiiYl8FncCW8feJPdyg3v6XW24KsRHe+dy9BAGRRjU= google.golang.org/grpc v1.45.0/go.mod h1:lN7owxKUQEqMfSyQikvvk5tf/6zMPsrK+ONuO11+0rQ= -google.golang.org/grpc v1.58.2 h1:SXUpjxeVF3FKrTYQI4f4KvbGD5u2xccdYdurwowix5I= -google.golang.org/grpc v1.58.2/go.mod h1:tgX3ZQDlNJGU96V6yHh1T/JeoBQ2TXdr43YbYSsCJk0= +google.golang.org/grpc v1.58.3 h1:BjnpXut1btbtgN/6sp+brB2Kbm2LjNXnidYujAVbSoQ= +google.golang.org/grpc v1.58.3/go.mod h1:tgX3ZQDlNJGU96V6yHh1T/JeoBQ2TXdr43YbYSsCJk0= google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM= From 71a74b9e9be3bfbad11231b043819e7cea969f53 Mon Sep 17 00:00:00 2001 From: Andrei Gurau <andreigurau@google.com> Date: Fri, 27 Oct 2023 10:39:16 -0400 Subject: [PATCH 272/435] [Java][Dataflow Runner] Add precondition to make sure userAgent name and version are not empty (#29157) --- .../apache/beam/runners/dataflow/DataflowRunner.java | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java index 80b4e4cfd8b5..6449053194fe 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java @@ -403,11 +403,14 @@ && isServiceEndpoint(dataflowOptions.getDataflowEndpoint())) { } DataflowRunnerInfo dataflowRunnerInfo = DataflowRunnerInfo.getDataflowRunnerInfo(); + String userAgentName = dataflowRunnerInfo.getName(); + Preconditions.checkArgument( + !userAgentName.equals(""), "Dataflow runner's `name` property cannot be empty."); + String userAgentVersion = dataflowRunnerInfo.getVersion(); + Preconditions.checkArgument( + !userAgentVersion.equals(""), "Dataflow runner's `version` property cannot be empty."); String userAgent = - String.format( - "%s/%s%s", - dataflowRunnerInfo.getName(), dataflowRunnerInfo.getVersion(), agentJavaVer) - .replace(" ", "_"); + String.format("%s/%s%s", userAgentName, userAgentVersion, agentJavaVer).replace(" ", "_"); dataflowOptions.setUserAgent(userAgent); return new DataflowRunner(dataflowOptions); From aeea5c80e184b4e867605a1ae613a60f95f53104 Mon Sep 17 00:00:00 2001 From: Danny McCormick <dannymccormick@google.com> Date: Fri, 27 Oct 2023 11:06:20 -0400 Subject: [PATCH 273/435] Revert grafana data source (#29161) --- .../GA-Post-Commits_status_dashboard.json | 343 +----------------- 1 file changed, 14 insertions(+), 329 deletions(-) diff --git a/.test-infra/metrics/grafana/dashboards/GA-Post-Commits_status_dashboard.json b/.test-infra/metrics/grafana/dashboards/GA-Post-Commits_status_dashboard.json index b576c0f03bc4..52f4747688d4 100644 --- a/.test-infra/metrics/grafana/dashboards/GA-Post-Commits_status_dashboard.json +++ b/.test-infra/metrics/grafana/dashboards/GA-Post-Commits_status_dashboard.json @@ -3,10 +3,7 @@ "list": [ { "builtIn": 1, - "datasource": { - "type": "datasource", - "uid": "grafana" - }, + "datasource": "-- Grafana --", "enable": true, "hide": true, "iconColor": "rgba(0, 211, 255, 1)", @@ -29,10 +26,7 @@ "liveNow": false, "panels": [ { - "datasource": { - "type": "postgres", - "uid": "fK5onAOVz" - }, + "datasource": "BeamPSQL", "fieldConfig": { "defaults": { "color": { @@ -931,10 +925,6 @@ { "aggregation": "Last", "alias": "job", - "datasource": { - "type": "postgres", - "uid": "fK5onAOVz" - }, "decimals": 2, "displayAliasType": "Warning / Critical", "displayType": "Regular", @@ -957,23 +947,6 @@ } ] ], - "sql": { - "columns": [ - { - "parameters": [], - "type": "function" - } - ], - "groupBy": [ - { - "property": { - "type": "string" - }, - "type": "groupBy" - } - ], - "limit": 50 - }, "timeColumn": "time", "units": "none", "valueHandler": "Number Threshold", @@ -998,10 +971,7 @@ "type": "table" }, { - "datasource": { - "type": "postgres", - "uid": "fK5onAOVz" - }, + "datasource": "BeamPSQL", "fieldConfig": { "defaults": { "color": { @@ -1900,10 +1870,6 @@ { "aggregation": "Last", "alias": "job", - "datasource": { - "type": "postgres", - "uid": "fK5onAOVz" - }, "decimals": 2, "displayAliasType": "Warning / Critical", "displayType": "Regular", @@ -1926,23 +1892,6 @@ } ] ], - "sql": { - "columns": [ - { - "parameters": [], - "type": "function" - } - ], - "groupBy": [ - { - "property": { - "type": "string" - }, - "type": "groupBy" - } - ], - "limit": 50 - }, "timeColumn": "time", "units": "none", "valueHandler": "Number Threshold", @@ -1967,10 +1916,7 @@ "type": "table" }, { - "datasource": { - "type": "postgres", - "uid": "fK5onAOVz" - }, + "datasource": "BeamPSQL", "fieldConfig": { "defaults": { "color": { @@ -2869,10 +2815,6 @@ { "aggregation": "Last", "alias": "job", - "datasource": { - "type": "postgres", - "uid": "fK5onAOVz" - }, "decimals": 2, "displayAliasType": "Warning / Critical", "displayType": "Regular", @@ -2895,23 +2837,6 @@ } ] ], - "sql": { - "columns": [ - { - "parameters": [], - "type": "function" - } - ], - "groupBy": [ - { - "property": { - "type": "string" - }, - "type": "groupBy" - } - ], - "limit": 50 - }, "timeColumn": "time", "units": "none", "valueHandler": "Number Threshold", @@ -2936,10 +2861,7 @@ "type": "table" }, { - "datasource": { - "type": "postgres", - "uid": "fK5onAOVz" - }, + "datasource": "BeamPSQL", "fieldConfig": { "defaults": { "color": { @@ -3838,10 +3760,6 @@ { "aggregation": "Last", "alias": "job", - "datasource": { - "type": "postgres", - "uid": "fK5onAOVz" - }, "decimals": 2, "displayAliasType": "Warning / Critical", "displayType": "Regular", @@ -3864,23 +3782,6 @@ } ] ], - "sql": { - "columns": [ - { - "parameters": [], - "type": "function" - } - ], - "groupBy": [ - { - "property": { - "type": "string" - }, - "type": "groupBy" - } - ], - "limit": 50 - }, "timeColumn": "time", "units": "none", "valueHandler": "Number Threshold", @@ -3905,10 +3806,7 @@ "type": "table" }, { - "datasource": { - "type": "postgres", - "uid": "fK5onAOVz" - }, + "datasource": "BeamPSQL", "fieldConfig": { "defaults": { "color": { @@ -4807,10 +4705,6 @@ { "aggregation": "Last", "alias": "job", - "datasource": { - "type": "postgres", - "uid": "fK5onAOVz" - }, "decimals": 2, "displayAliasType": "Warning / Critical", "displayType": "Regular", @@ -4833,23 +4727,6 @@ } ] ], - "sql": { - "columns": [ - { - "parameters": [], - "type": "function" - } - ], - "groupBy": [ - { - "property": { - "type": "string" - }, - "type": "groupBy" - } - ], - "limit": 50 - }, "timeColumn": "time", "units": "none", "valueHandler": "Number Threshold", @@ -4874,10 +4751,7 @@ "type": "table" }, { - "datasource": { - "type": "postgres", - "uid": "fK5onAOVz" - }, + "datasource": "BeamPSQL", "fieldConfig": { "defaults": { "color": { @@ -5776,10 +5650,6 @@ { "aggregation": "Last", "alias": "job", - "datasource": { - "type": "postgres", - "uid": "fK5onAOVz" - }, "decimals": 2, "displayAliasType": "Warning / Critical", "displayType": "Regular", @@ -5802,23 +5672,6 @@ } ] ], - "sql": { - "columns": [ - { - "parameters": [], - "type": "function" - } - ], - "groupBy": [ - { - "property": { - "type": "string" - }, - "type": "groupBy" - } - ], - "limit": 50 - }, "timeColumn": "time", "units": "none", "valueHandler": "Number Threshold", @@ -5843,10 +5696,7 @@ "type": "table" }, { - "datasource": { - "type": "postgres", - "uid": "fK5onAOVz" - }, + "datasource": "BeamPSQL", "fieldConfig": { "defaults": { "color": { @@ -6745,10 +6595,6 @@ { "aggregation": "Last", "alias": "job", - "datasource": { - "type": "postgres", - "uid": "fK5onAOVz" - }, "decimals": 2, "displayAliasType": "Warning / Critical", "displayType": "Regular", @@ -6771,23 +6617,6 @@ } ] ], - "sql": { - "columns": [ - { - "parameters": [], - "type": "function" - } - ], - "groupBy": [ - { - "property": { - "type": "string" - }, - "type": "groupBy" - } - ], - "limit": 50 - }, "timeColumn": "time", "units": "none", "valueHandler": "Number Threshold", @@ -6812,10 +6641,7 @@ "type": "table" }, { - "datasource": { - "type": "postgres", - "uid": "fK5onAOVz" - }, + "datasource": "BeamPSQL", "fieldConfig": { "defaults": { "color": { @@ -7714,10 +7540,6 @@ { "aggregation": "Last", "alias": "job", - "datasource": { - "type": "postgres", - "uid": "fK5onAOVz" - }, "decimals": 2, "displayAliasType": "Warning / Critical", "displayType": "Regular", @@ -7740,23 +7562,6 @@ } ] ], - "sql": { - "columns": [ - { - "parameters": [], - "type": "function" - } - ], - "groupBy": [ - { - "property": { - "type": "string" - }, - "type": "groupBy" - } - ], - "limit": 50 - }, "timeColumn": "time", "units": "none", "valueHandler": "Number Threshold", @@ -7781,10 +7586,7 @@ "type": "table" }, { - "datasource": { - "type": "postgres", - "uid": "fK5onAOVz" - }, + "datasource": "BeamPSQL", "fieldConfig": { "defaults": { "color": { @@ -8683,10 +8485,6 @@ { "aggregation": "Last", "alias": "job", - "datasource": { - "type": "postgres", - "uid": "fK5onAOVz" - }, "decimals": 2, "displayAliasType": "Warning / Critical", "displayType": "Regular", @@ -8709,23 +8507,6 @@ } ] ], - "sql": { - "columns": [ - { - "parameters": [], - "type": "function" - } - ], - "groupBy": [ - { - "property": { - "type": "string" - }, - "type": "groupBy" - } - ], - "limit": 50 - }, "timeColumn": "time", "units": "none", "valueHandler": "Number Threshold", @@ -8750,10 +8531,7 @@ "type": "table" }, { - "datasource": { - "type": "postgres", - "uid": "fK5onAOVz" - }, + "datasource": "BeamPSQL", "fieldConfig": { "defaults": { "color": { @@ -9652,10 +9430,6 @@ { "aggregation": "Last", "alias": "job", - "datasource": { - "type": "postgres", - "uid": "fK5onAOVz" - }, "decimals": 2, "displayAliasType": "Warning / Critical", "displayType": "Regular", @@ -9678,23 +9452,6 @@ } ] ], - "sql": { - "columns": [ - { - "parameters": [], - "type": "function" - } - ], - "groupBy": [ - { - "property": { - "type": "string" - }, - "type": "groupBy" - } - ], - "limit": 50 - }, "timeColumn": "time", "units": "none", "valueHandler": "Number Threshold", @@ -9719,10 +9476,7 @@ "type": "table" }, { - "datasource": { - "type": "postgres", - "uid": "fK5onAOVz" - }, + "datasource": "BeamPSQL", "fieldConfig": { "defaults": { "color": { @@ -10621,10 +10375,6 @@ { "aggregation": "Last", "alias": "job", - "datasource": { - "type": "postgres", - "uid": "fK5onAOVz" - }, "decimals": 2, "displayAliasType": "Warning / Critical", "displayType": "Regular", @@ -10647,23 +10397,6 @@ } ] ], - "sql": { - "columns": [ - { - "parameters": [], - "type": "function" - } - ], - "groupBy": [ - { - "property": { - "type": "string" - }, - "type": "groupBy" - } - ], - "limit": 50 - }, "timeColumn": "time", "units": "none", "valueHandler": "Number Threshold", @@ -10688,10 +10421,7 @@ "type": "table" }, { - "datasource": { - "type": "postgres", - "uid": "fK5onAOVz" - }, + "datasource": "BeamPSQL", "fieldConfig": { "defaults": { "color": { @@ -11590,10 +11320,6 @@ { "aggregation": "Last", "alias": "job", - "datasource": { - "type": "postgres", - "uid": "fK5onAOVz" - }, "decimals": 2, "displayAliasType": "Warning / Critical", "displayType": "Regular", @@ -11616,23 +11342,6 @@ } ] ], - "sql": { - "columns": [ - { - "parameters": [], - "type": "function" - } - ], - "groupBy": [ - { - "property": { - "type": "string" - }, - "type": "groupBy" - } - ], - "limit": 50 - }, "timeColumn": "time", "units": "none", "valueHandler": "Number Threshold", @@ -11657,10 +11366,7 @@ "type": "table" }, { - "datasource": { - "type": "postgres", - "uid": "fK5onAOVz" - }, + "datasource": "BeamPSQL", "fieldConfig": { "defaults": { "color": { @@ -12559,10 +12265,6 @@ { "aggregation": "Last", "alias": "job", - "datasource": { - "type": "postgres", - "uid": "fK5onAOVz" - }, "decimals": 2, "displayAliasType": "Warning / Critical", "displayType": "Regular", @@ -12585,23 +12287,6 @@ } ] ], - "sql": { - "columns": [ - { - "parameters": [], - "type": "function" - } - ], - "groupBy": [ - { - "property": { - "type": "string" - }, - "type": "groupBy" - } - ], - "limit": 50 - }, "timeColumn": "time", "units": "none", "valueHandler": "Number Threshold", From 73f91238b815caae4ede1b44a1b95a2b5b650592 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 27 Oct 2023 11:06:40 -0400 Subject: [PATCH 274/435] Bump com.gradle.enterprise from 3.13.2 to 3.15.1 (#28913) Bumps com.gradle.enterprise from 3.13.2 to 3.15.1. --- updated-dependencies: - dependency-name: com.gradle.enterprise dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] <support@github.com> Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- settings.gradle.kts | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/settings.gradle.kts b/settings.gradle.kts index de83c11715c8..85ef793b8e12 100644 --- a/settings.gradle.kts +++ b/settings.gradle.kts @@ -24,7 +24,7 @@ pluginManagement { } plugins { - id("com.gradle.enterprise") version "3.13.2" + id("com.gradle.enterprise") version "3.15.1" id("com.gradle.common-custom-user-data-gradle-plugin") version "1.11.3" } From 7b38f743f78812fd240e2245abc490e6cb8997ed Mon Sep 17 00:00:00 2001 From: Jack McCluskey <34928439+jrmccluskey@users.noreply.github.com> Date: Fri, 27 Oct 2023 11:43:01 -0400 Subject: [PATCH 275/435] Add extra typehinting tests (#29148) * Add extra typehinting tests * Formatting * Update sdks/python/apache_beam/typehints/native_type_compatibility_test.py Co-authored-by: Danny McCormick <dannymccormick@google.com> * Restore wonky formatting --------- Co-authored-by: Danny McCormick <dannymccormick@google.com> --- .../native_type_compatibility_test.py | 45 +++++++++++++------ 1 file changed, 31 insertions(+), 14 deletions(-) diff --git a/sdks/python/apache_beam/typehints/native_type_compatibility_test.py b/sdks/python/apache_beam/typehints/native_type_compatibility_test.py index 23b9beadd336..9c2762dff710 100644 --- a/sdks/python/apache_beam/typehints/native_type_compatibility_test.py +++ b/sdks/python/apache_beam/typehints/native_type_compatibility_test.py @@ -20,6 +20,7 @@ # pytype: skip-file import collections.abc +import enum import sys import typing import unittest @@ -54,6 +55,11 @@ class _TestPair(typing.NamedTuple('TestTuple', [('first', T), ('second', T)]), pass +class _TestEnum(enum.Enum): + FOO = enum.auto() + BAR = enum.auto() + + class NativeTypeCompatibilityTest(unittest.TestCase): def test_convert_to_beam_type(self): test_cases = [ @@ -106,6 +112,7 @@ def test_convert_to_beam_type(self): typehints.List[_TestGeneric[int]]), ('nested generic with any', typing.List[_TestPair[typing.Any]], typehints.List[_TestPair[typing.Any]]), + ('raw enum', _TestEnum, _TestEnum), ] for test_case in test_cases: @@ -122,20 +129,22 @@ def test_convert_to_beam_type(self): def test_convert_to_beam_type_with_builtin_types(self): if sys.version_info >= (3, 9): - test_cases = [('builtin dict', dict[str, int], typehints.Dict[str, int]), - ('builtin list', list[str], typehints.List[str]), - ('builtin tuple', tuple[str], typehints.Tuple[str]), - ('builtin set', set[str], typehints.Set[str]), - ( - 'nested builtin', - dict[str, list[tuple[float]]], - typehints.Dict[str, - typehints.List[typehints.Tuple[float]]]), - ( - 'builtin nested tuple', - tuple[str, list], - typehints.Tuple[str, typehints.List[typehints.Any]], - )] + test_cases = [ + ('builtin dict', dict[str, int], typehints.Dict[str, int]), + ('builtin list', list[str], typehints.List[str]), + ('builtin tuple', tuple[str], + typehints.Tuple[str]), ('builtin set', set[str], typehints.Set[str]), + ('builtin frozenset', frozenset[int], typehints.FrozenSet[int]), + ( + 'nested builtin', + dict[str, list[tuple[float]]], + typehints.Dict[str, typehints.List[typehints.Tuple[float]]]), + ( + 'builtin nested tuple', + tuple[str, list], + typehints.Tuple[str, typehints.List[typehints.Any]], + ) + ] for test_case in test_cases: description = test_case[0] @@ -173,6 +182,14 @@ def test_convert_to_beam_type_with_collections_types(self): collections.abc.Mapping[str, int]), ('set', collections.abc.Set[str], typehints.Set[str]), ('mutable set', collections.abc.MutableSet[int], typehints.Set[int]), + ( + 'enum set', + collections.abc.Set[_TestEnum], + typehints.Set[_TestEnum]), + ( + 'enum mutable set', + collections.abc.MutableSet[_TestEnum], + typehints.Set[_TestEnum]) ] for test_case in test_cases: From 48f3c67054e036cc9ec0825575946953e12f0631 Mon Sep 17 00:00:00 2001 From: Jeffrey Kinard <jeff@thekinards.com> Date: Wed, 20 Sep 2023 09:17:22 -0400 Subject: [PATCH 276/435] Mark JS UDFs as experimental Signed-off-by: Jeffrey Kinard <jeff@thekinards.com> --- sdks/python/apache_beam/yaml/yaml_udf_test.py | 17 ++++++++++++++++- 1 file changed, 16 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/yaml/yaml_udf_test.py b/sdks/python/apache_beam/yaml/yaml_udf_test.py index 04a4a3bac963..900e5dc53351 100644 --- a/sdks/python/apache_beam/yaml/yaml_udf_test.py +++ b/sdks/python/apache_beam/yaml/yaml_udf_test.py @@ -57,7 +57,7 @@ def test_map_to_fields_filter_inline_js(self): ''' type: MapToFields config: - language: javascript + language: javascript-experimental fields: label: callable: "function label_map(x) {return x.label + 'x'}" @@ -104,8 +104,13 @@ def test_filter_inline_js(self): ''' type: Filter config: +<<<<<<< HEAD language: javascript keep: +======= + language: javascript-experimental + keep: +>>>>>>> 0ced28e35e (Mark JS UDFs as experimental) callable: "function filter(x) {return x.rank > 0}" ''') assert_that( @@ -143,8 +148,13 @@ def test_filter_expression_js(self): ''' type: Filter config: +<<<<<<< HEAD language: javascript keep: +======= + language: javascript-experimental + keep: +>>>>>>> 0ced28e35e (Mark JS UDFs as experimental) expression: "label.toUpperCase().indexOf('3') == -1 && conductor" ''') assert_that( @@ -193,8 +203,13 @@ def test_filter_inline_js_file(self): f''' type: Filter config: +<<<<<<< HEAD language: javascript keep: +======= + language: javascript-experimental + keep: +>>>>>>> 0ced28e35e (Mark JS UDFs as experimental) path: {path} name: "f" ''') From f166cfbd018d84cfb29f66b8145889e091c135b8 Mon Sep 17 00:00:00 2001 From: Jeffrey Kinard <jeff@thekinards.com> Date: Thu, 21 Sep 2023 12:41:19 -0400 Subject: [PATCH 277/435] [YAML] Add jsobject conversion to mapping Signed-off-by: Jeffrey Kinard <jeff@thekinards.com> --- sdks/python/apache_beam/transforms/core.py | 2 +- sdks/python/apache_beam/yaml/yaml_mapping.py | 95 +++++++++++- sdks/python/apache_beam/yaml/yaml_udf_test.py | 137 ++++++++++++------ 3 files changed, 183 insertions(+), 51 deletions(-) diff --git a/sdks/python/apache_beam/transforms/core.py b/sdks/python/apache_beam/transforms/core.py index dec81905f201..a6fd3b184d4c 100644 --- a/sdks/python/apache_beam/transforms/core.py +++ b/sdks/python/apache_beam/transforms/core.py @@ -2260,7 +2260,7 @@ def __init__(self, pcoll, exception_handling_args, upstream_errors=()): @property def pipeline(self): - return self._pvalue.pipeline + return self._pcoll.pipeline @property def element_type(self): diff --git a/sdks/python/apache_beam/yaml/yaml_mapping.py b/sdks/python/apache_beam/yaml/yaml_mapping.py index e217ab285844..197cf7483815 100644 --- a/sdks/python/apache_beam/yaml/yaml_mapping.py +++ b/sdks/python/apache_beam/yaml/yaml_mapping.py @@ -27,8 +27,31 @@ from typing import Union import js2py +from js2py.base import JsObjectWrapper +from js2py.base import PyJsArray +from js2py.base import PyJsArrayBuffer +from js2py.base import PyJsBoolean +from js2py.base import PyJsError +from js2py.base import PyJsFloat32Array +from js2py.base import PyJsFloat64Array +from js2py.base import PyJsInt16Array +from js2py.base import PyJsInt32Array +from js2py.base import PyJsInt8Array +from js2py.base import PyJsNull +from js2py.base import PyJsNumber +from js2py.base import PyJsObject +from js2py.base import PyJsString +from js2py.base import PyJsUint16Array +from js2py.base import PyJsUint32Array +from js2py.base import PyJsUint8Array +from js2py.base import PyJsUint8ClampedArray +from js2py.base import PyJsUndefined +from js2py.base import to_python +from js2py.constructors.jsdate import PyJsDate +from js2py.internals.simplex import JsException import apache_beam as beam +from apache_beam import Row from apache_beam.io.filesystems import FileSystems from apache_beam.portability.api import schema_pb2 from apache_beam.typehints import row_type @@ -39,6 +62,7 @@ from apache_beam.yaml import json_utils from apache_beam.yaml import options from apache_beam.yaml import yaml_provider +from apache_beam.yaml.yaml_provider import dicts_to_rows def _check_mapping_arguments( @@ -75,19 +99,77 @@ def __setstate__(self, state): self.__dict__.update(state) +def row_to_dict(row): + if (str(type(row).__name__).startswith('BeamSchema_') or + isinstance(row, Row)): + row = row._asdict() + if isinstance(row, dict): + for key, value in row.items(): + row[key] = row_to_dict(value) + elif not isinstance(row, str) and isinstance(row, Iterable): + row_list = list(row) + for idx in range(len(row_list)): + row_list[idx] = row_to_dict(row_list[idx]) + return row + + # TODO(yaml) Consider adding optional language version parameter to support # ECMAScript 5 and 6 def _expand_javascript_mapping_func( original_fields, expression=None, callable=None, path=None, name=None): + + js_array_type = ( + PyJsArray, + PyJsArrayBuffer, + PyJsInt8Array, + PyJsUint8Array, + PyJsUint8ClampedArray, + PyJsInt16Array, + PyJsUint16Array, + PyJsInt32Array, + PyJsUint32Array, + PyJsFloat32Array, + PyJsFloat64Array) + + def _js_object_to_py_object(obj): + if isinstance(obj, (PyJsNumber, PyJsString, PyJsBoolean)): + obj = to_python(obj) + elif isinstance(obj, js_array_type): + return [_js_object_to_py_object(value) for value in obj.to_list()] + elif isinstance(obj, PyJsDate): + obj = obj.to_utc_dt() + elif isinstance(obj, (PyJsNull, PyJsUndefined)): + return None + elif isinstance(obj, PyJsError): + raise RuntimeError(obj['message']) + elif isinstance(obj, PyJsObject): + return { + key: _js_object_to_py_object(value['value']) + for key, + value in obj.own.items() + } + elif isinstance(obj, JsObjectWrapper): + return _js_object_to_py_object(obj._obj) + + return obj + + def _catch_js_errors(func): + try: + result = func() + except JsException as e: + result = getattr(e, 'mes') + return result + if expression: args = ', '.join(original_fields) js_func = f'function fn({args}) {{return ({expression})}}' - js_callable = _CustomJsObjectWrapper(js2py.eval_js(js_func)) - return lambda __row__: js_callable(*__row__._asdict().values()) + js_expr_callable = _CustomJsObjectWrapper(js2py.eval_js(js_func)) + fn = lambda __row__: lambda: js_expr_callable( + *row_to_dict(__row__).values()) elif callable: js_callable = _CustomJsObjectWrapper(js2py.eval_js(callable)) - return lambda __row__: js_callable(__row__._asdict()) + fn = lambda __row__: lambda: js_callable(row_to_dict(__row__)) else: if not path.endswith('.js'): @@ -95,8 +177,11 @@ def _expand_javascript_mapping_func( udf_code = FileSystems.open(path).read().decode() js = js2py.EvalJs() js.eval(udf_code) - js_callable = _CustomJsObjectWrapper(getattr(js, name)) - return lambda __row__: js_callable(__row__._asdict()) + js_file_callable = _CustomJsObjectWrapper(getattr(js, name)) + fn = lambda __row__: lambda: js_file_callable(row_to_dict(__row__)) + + return lambda __row__: dicts_to_rows( + _js_object_to_py_object(_catch_js_errors(fn(__row__)))) def _expand_python_mapping_func( diff --git a/sdks/python/apache_beam/yaml/yaml_udf_test.py b/sdks/python/apache_beam/yaml/yaml_udf_test.py index 900e5dc53351..4b75f3c489fc 100644 --- a/sdks/python/apache_beam/yaml/yaml_udf_test.py +++ b/sdks/python/apache_beam/yaml/yaml_udf_test.py @@ -25,20 +25,26 @@ from apache_beam.options import pipeline_options from apache_beam.testing.util import assert_that from apache_beam.testing.util import equal_to +from apache_beam.yaml.yaml_mapping import row_to_dict +from apache_beam.yaml.yaml_provider import dicts_to_rows from apache_beam.yaml.yaml_transform import YamlTransform def AsRows(): - return beam.Map(lambda named_tuple: beam.Row(**named_tuple._asdict())) + return beam.Map(lambda named_tuple: dicts_to_rows(row_to_dict(named_tuple))) class YamlUDFMappingTest(unittest.TestCase): def __init__(self, method_name='runYamlMappingTest'): super().__init__(method_name) self.data = [ - beam.Row(label='11a', conductor=11, rank=0), - beam.Row(label='37a', conductor=37, rank=1), - beam.Row(label='389a', conductor=389, rank=2), + beam.Row( + label='11a', conductor=11, row=beam.Row(rank=0, values=[1, 2, 3])), + beam.Row( + label='37a', conductor=37, row=beam.Row(rank=1, values=[4, 5, 6])), + beam.Row( + label='389a', conductor=389, row=beam.Row(rank=2, values=[7, 8, + 9])), ] def setUp(self): @@ -57,19 +63,40 @@ def test_map_to_fields_filter_inline_js(self): ''' type: MapToFields config: - language: javascript-experimental + language: javascript fields: label: - callable: "function label_map(x) {return x.label + 'x'}" + callable: | + function label_map(x) { + return x.label + 'x' + } conductor: - callable: "function conductor_map(x) {return x.conductor + 1}" + callable: | + function conductor_map(x) { + return x.conductor + 1 + } + row: + callable: | + function row_map(x) { + x.row.values.push(x.row.rank + 10) + return x.row + } ''') assert_that( result, equal_to([ - beam.Row(label='11ax', conductor=12), - beam.Row(label='37ax', conductor=38), - beam.Row(label='389ax', conductor=390), + beam.Row( + label='11ax', + conductor=12, + row=beam.Row(rank=0, values=[1, 2, 3, 10])), + beam.Row( + label='37ax', + conductor=38, + row=beam.Row(rank=1, values=[4, 5, 6, 11])), + beam.Row( + label='389ax', + conductor=390, + row=beam.Row(rank=2, values=[7, 8, 9, 12])), ])) def test_map_to_fields_filter_inline_py(self): @@ -86,13 +113,15 @@ def test_map_to_fields_filter_inline_py(self): callable: "lambda x: x.label + 'x'" conductor: callable: "lambda x: x.conductor + 1" + sum: + callable: "lambda x: sum(x.row.values)" ''') assert_that( result, equal_to([ - beam.Row(label='11ax', conductor=12), - beam.Row(label='37ax', conductor=38), - beam.Row(label='389ax', conductor=390), + beam.Row(label='11ax', conductor=12, sum=6), + beam.Row(label='37ax', conductor=38, sum=15), + beam.Row(label='389ax', conductor=390, sum=24), ])) def test_filter_inline_js(self): @@ -104,20 +133,24 @@ def test_filter_inline_js(self): ''' type: Filter config: -<<<<<<< HEAD language: javascript keep: -======= - language: javascript-experimental - keep: ->>>>>>> 0ced28e35e (Mark JS UDFs as experimental) - callable: "function filter(x) {return x.rank > 0}" + callable: | + function filter(x) { + return x.row.rank > 0 + } ''') assert_that( result | AsRows(), equal_to([ - beam.Row(label='37a', conductor=37, rank=1), - beam.Row(label='389a', conductor=389, rank=2), + beam.Row( + label='37a', + conductor=37, + row=beam.Row(rank=1, values=[4, 5, 6])), + beam.Row( + label='389a', + conductor=389, + row=beam.Row(rank=2, values=[7, 8, 9])), ])) def test_filter_inline_py(self): @@ -130,13 +163,19 @@ def test_filter_inline_py(self): config: language: python keep: - callable: "lambda x: x.rank > 0" + callable: "lambda x: x.row.rank > 0" ''') assert_that( result | AsRows(), equal_to([ - beam.Row(label='37a', conductor=37, rank=1), - beam.Row(label='389a', conductor=389, rank=2), + beam.Row( + label='37a', + conductor=37, + row=beam.Row(rank=1, values=[4, 5, 6])), + beam.Row( + label='389a', + conductor=389, + row=beam.Row(rank=2, values=[7, 8, 9])), ])) def test_filter_expression_js(self): @@ -148,19 +187,17 @@ def test_filter_expression_js(self): ''' type: Filter config: -<<<<<<< HEAD language: javascript keep: -======= - language: javascript-experimental - keep: ->>>>>>> 0ced28e35e (Mark JS UDFs as experimental) - expression: "label.toUpperCase().indexOf('3') == -1 && conductor" + expression: "label.toUpperCase().indexOf('3') == -1 && row.rank < 1" ''') assert_that( result | AsRows(), equal_to([ - beam.Row(label='11a', conductor=11, rank=0), + beam.Row( + label='11a', + conductor=11, + row=beam.Row(rank=0, values=[1, 2, 3])), ])) def test_filter_expression_py(self): @@ -178,17 +215,20 @@ def test_filter_expression_py(self): assert_that( result | AsRows(), equal_to([ - beam.Row(label='11a', conductor=11, rank=0), + beam.Row( + label='11a', + conductor=11, + row=beam.Row(rank=0, values=[1, 2, 3])), ])) def test_filter_inline_js_file(self): data = ''' function f(x) { - return x.rank > 0 + return x.row.rank > 0 } function g(x) { - return x.rank > 1 + return x.row.rank > 1 } '''.replace(' ', '') @@ -203,30 +243,31 @@ def test_filter_inline_js_file(self): f''' type: Filter config: -<<<<<<< HEAD language: javascript keep: -======= - language: javascript-experimental - keep: ->>>>>>> 0ced28e35e (Mark JS UDFs as experimental) path: {path} name: "f" ''') assert_that( result | AsRows(), equal_to([ - beam.Row(label='37a', conductor=37, rank=1), - beam.Row(label='389a', conductor=389, rank=2), + beam.Row( + label='37a', + conductor=37, + row=beam.Row(rank=1, values=[4, 5, 6])), + beam.Row( + label='389a', + conductor=389, + row=beam.Row(rank=2, values=[7, 8, 9])), ])) def test_filter_inline_py_file(self): data = ''' def f(x): - return x.rank > 0 + return x.row.rank > 0 def g(x): - return x.rank > 1 + return x.row.rank > 1 '''.replace(' ', '') path = os.path.join(self.tmpdir, 'udf.py') @@ -247,8 +288,14 @@ def g(x): assert_that( result | AsRows(), equal_to([ - beam.Row(label='37a', conductor=37, rank=1), - beam.Row(label='389a', conductor=389, rank=2), + beam.Row( + label='37a', + conductor=37, + row=beam.Row(rank=1, values=[4, 5, 6])), + beam.Row( + label='389a', + conductor=389, + row=beam.Row(rank=2, values=[7, 8, 9])), ])) From 0d5469e76f08a0df6f146965e4a1a7a41cb5a949 Mon Sep 17 00:00:00 2001 From: Jeffrey Kinard <jeff@thekinards.com> Date: Fri, 27 Oct 2023 13:59:38 -0400 Subject: [PATCH 278/435] [YAML] add GCS support for external transform jars Signed-off-by: Jeffrey Kinard <jeff@thekinards.com> --- sdks/python/apache_beam/utils/subprocess_server.py | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/utils/subprocess_server.py b/sdks/python/apache_beam/utils/subprocess_server.py index f566c3ea2914..f6e214046f37 100644 --- a/sdks/python/apache_beam/utils/subprocess_server.py +++ b/sdks/python/apache_beam/utils/subprocess_server.py @@ -36,6 +36,7 @@ import grpc +from apache_beam.io.filesystems import FileSystems from apache_beam.version import __version__ as beam_version _LOGGER = logging.getLogger(__name__) @@ -272,7 +273,10 @@ def local_jar(cls, url, cache_dir=None): os.makedirs(cache_dir) # TODO: Clean up this cache according to some policy. try: - url_read = urlopen(url) + try: + url_read = FileSystems.open(url) + except ValueError: + url_read = urlopen(url) with open(cached_jar + '.tmp', 'wb') as jar_write: shutil.copyfileobj(url_read, jar_write, length=1 << 20) os.rename(cached_jar + '.tmp', cached_jar) From 6bb3ef2ec4d22f1c77cdb5885deeff8cbbb35007 Mon Sep 17 00:00:00 2001 From: Jack McCluskey <34928439+jrmccluskey@users.noreply.github.com> Date: Fri, 27 Oct 2023 14:49:13 -0400 Subject: [PATCH 279/435] Bump Python Postcommit to high memory runner (#29176) --- .github/workflows/beam_PostCommit_Python.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/beam_PostCommit_Python.yml b/.github/workflows/beam_PostCommit_Python.yml index e3e4ce927519..09e813e4a41f 100644 --- a/.github/workflows/beam_PostCommit_Python.yml +++ b/.github/workflows/beam_PostCommit_Python.yml @@ -51,7 +51,7 @@ env: jobs: beam_PostCommit_Python: name: ${{ matrix.job_name }} (${{ matrix.job_phrase }} ${{ matrix.python_version }}) - runs-on: [self-hosted, ubuntu-20.04, main] + runs-on: [self-hosted, ubuntu-20.04, highmem] timeout-minutes: 240 strategy: fail-fast: false From e8c455eacc2732ca6899673c479287db5a8fbd63 Mon Sep 17 00:00:00 2001 From: Anand Inguva <34158215+AnandInguva@users.noreply.github.com> Date: Fri, 27 Oct 2023 19:22:44 +0000 Subject: [PATCH 280/435] change extension of beam sdist from .zip to .tar.gz (#29108) --- .../actions/common-rc-validation/action.yaml | 10 +- .github/workflows/build_release_candidate.yml | 16 ++-- .github/workflows/run_rc_validation.yml | 93 +++++++++---------- contributor-docs/release-guide.md | 10 +- .../python_release_automation_utils.sh | 6 +- .../main/scripts/build_release_candidate.sh | 16 ++-- .../scripts/deploy_release_candidate_pypi.sh | 4 +- release/src/main/scripts/run_rc_validation.sh | 48 +++++----- sdks/python/scripts/run_snapshot_publish.sh | 6 +- 9 files changed, 104 insertions(+), 105 deletions(-) diff --git a/.github/actions/common-rc-validation/action.yaml b/.github/actions/common-rc-validation/action.yaml index 23efa93d1533..51738e138122 100644 --- a/.github/actions/common-rc-validation/action.yaml +++ b/.github/actions/common-rc-validation/action.yaml @@ -36,19 +36,19 @@ runs: shell: bash run: | echo "---------------------Downloading Python Staging RC----------------------------" - wget ${PYTHON_RC_DOWNLOAD_URL}/${RELEASE_VER}/python/apache-beam-${RELEASE_VER}.zip - wget ${PYTHON_RC_DOWNLOAD_URL}/${RELEASE_VER}/python/apache-beam-${RELEASE_VER}.zip.sha512 - if [[ ! -f apache-beam-$RELEASE_VER.zip ]]; then + wget ${PYTHON_RC_DOWNLOAD_URL}/${RELEASE_VER}/python/apache-beam-${RELEASE_VER}.tar.gz + wget ${PYTHON_RC_DOWNLOAD_URL}/${RELEASE_VER}/python/apache-beam-${RELEASE_VER}.tar.gz.sha512 + if [[ ! -f apache-beam-$RELEASE_VER.tar.gz ]]; then { echo "Fail to download Python Staging RC files." ;exit 1; } fi echo "--------------------------Verifying Hashes------------------------------------" - sha512sum -c apache-beam-${RELEASE_VER}.zip.sha512 + sha512sum -c apache-beam-${RELEASE_VER}.tar.gz.sha512 `which pip` install --upgrade pip `which pip` install --upgrade setuptools - name: Installing python SDK shell: bash - run: pip install apache-beam-${RELEASE_VER}.zip[gcp] + run: pip install apache-beam-${RELEASE_VER}.tar.gz[gcp] \ No newline at end of file diff --git a/.github/workflows/build_release_candidate.yml b/.github/workflows/build_release_candidate.yml index 68456bb19fcb..5c35aa744198 100644 --- a/.github/workflows/build_release_candidate.yml +++ b/.github/workflows/build_release_candidate.yml @@ -122,18 +122,18 @@ jobs: mkdir -p beam/${{ github.event.inputs.RELEASE }} cd beam/${{ github.event.inputs.RELEASE }} RC_DIR="beam-${{ github.event.inputs.RELEASE }}-RC${{ github.event.inputs.RC }}" - RC_ZIP="${RC_DIR}.zip" + RC_ZIP="${RC_DIR}.tar.gz" RELEASE_DIR="beam-${{ github.event.inputs.RELEASE }}" RC_TAG="v${{ github.event.inputs.RELEASE }}-RC${{ github.event.inputs.RC }}" - SOURCE_RELEASE_ZIP="apache-beam-${{ github.event.inputs.RELEASE }}-source-release.zip" + SOURCE_RELEASE_ZIP="apache-beam-${{ github.event.inputs.RELEASE }}-source-release.tar.gz" # Check whether there is an existing dist dir if (svn ls "${SOURCE_RELEASE_ZIP}"); then echo "Removing existing ${SOURCE_RELEASE_ZIP}." svn delete "${SOURCE_RELEASE_ZIP}" fi - echo "Downloading: https://github.com/apache/beam/archive/${RC_TAG}.zip" - wget https://github.com/apache/beam/archive/${RC_TAG}.zip -O "${RC_ZIP}" + echo "Downloading: https://github.com/apache/beam/archive/${RC_TAG}.tar.gz" + wget https://github.com/apache/beam/archive/${RC_TAG}.tar.gz -O "${RC_ZIP}" unzip "$RC_ZIP" rm "$RC_ZIP" @@ -197,11 +197,11 @@ jobs: cd "${RELEASE_DIR}"/python - echo "------Checking Hash Value for apache-beam-${RELEASE}.zip-----" - sha512sum -c "apache-beam-${RELEASE}.zip.sha512" + echo "------Checking Hash Value for apache-beam-${RELEASE}.tar.gz-----" + sha512sum -c "apache-beam-${RELEASE}.tar.gz.sha512" - echo "------Signing Source Release apache-beam-${RELEASE}.zip------" - gpg --local-user "${{steps.import_gpg.outputs.name}}" --armor --detach-sig "apache-beam-${RELEASE}.zip" + echo "------Signing Source Release apache-beam-${RELEASE}.tar.gz------" + gpg --local-user "${{steps.import_gpg.outputs.name}}" --armor --detach-sig "apache-beam-${RELEASE}.tar.gz" for artifact in *.whl; do echo "----------Checking Hash Value for ${artifact} wheel-----------" diff --git a/.github/workflows/run_rc_validation.yml b/.github/workflows/run_rc_validation.yml index 4902fee81016..35fb3ce05eae 100644 --- a/.github/workflows/run_rc_validation.yml +++ b/.github/workflows/run_rc_validation.yml @@ -17,14 +17,14 @@ # To learn more about GitHub Actions in Apache Beam check the CI.m -name: Run RC Validation +name: Run RC Validation on: workflow_dispatch: inputs: RELEASE_VER: description: Beam current Release Version required: true - default: 2.42.0 + default: 2.42.0 USER_GCS_BUCKET: description: Bucket to upload results required: true @@ -57,9 +57,9 @@ on: type: boolean required: true default: true -env: +env: RC_TAG: "v${{github.event.inputs.RELEASE_VER}}-RC${{github.event.inputs.RC_NUM}}" - RELEASE_VER: ${{github.event.inputs.RELEASE_VER}} + RELEASE_VER: ${{github.event.inputs.RELEASE_VER}} USER_GCP_PROJECT: apache-beam-testing PYTHON_RC_DOWNLOAD_URL: https://dist.apache.org/repos/dist/dev/beam USER_GCP_REGION: us-central1 @@ -100,19 +100,19 @@ jobs: - name: Comment on PR to Trigger Python ReleaseCandidate Test run: | gh pr comment "$GITHUB_PR_URL" --body "Run Python ReleaseCandidate" - + sql_taxi_with_dataflow: runs-on: [self-hosted,ubuntu-20.04] if: ${{github.event.inputs.RUN_SQL_TAXI_WITH_DATAFLOW == 'true'}} strategy: - matrix: + matrix: py_version: [3.8] steps: - name: Checkout code uses: actions/checkout@v4 - with: - ref: ${{env.RC_TAG}} - + with: + ref: ${{env.RC_TAG}} + - name: Install Python uses: actions/setup-python@v4 with: @@ -154,23 +154,23 @@ jobs: --num_workers 5 \ --output_topic projects/${USER_GCP_PROJECT}/topics/${SQL_TAXI_TOPIC} \ --beam_services="{\":sdks:java:extensions:sql:expansion-service:shadowJar\": \"${SQL_EXPANSION_SERVICE_JAR}\"}" \ - --sdk_location apache-beam-${RELEASE_VER}.zip || true + --sdk_location apache-beam-${RELEASE_VER}.tar.gz || true - name: Checking Results run: | gcloud pubsub subscriptions pull --project=${USER_GCP_PROJECT} --limit=5 ${SQL_TAXI_SUBSCRIPTION} gcloud pubsub subscriptions pull --project=${USER_GCP_PROJECT} --limit=5 ${SQL_TAXI_SUBSCRIPTION} - name: Removing Pub Sub Topic if: always() - run: | + run: | gcloud pubsub topics delete --project=${USER_GCP_PROJECT} ${SQL_TAXI_TOPIC} gcloud pubsub subscriptions delete --project=${USER_GCP_PROJECT} ${SQL_TAXI_SUBSCRIPTION} python_cross_validation: runs-on: [self-hosted,ubuntu-20.04] - if: ${{github.event.inputs.RUN_PYTHON_CROSS_VALIDATION == 'true'}} + if: ${{github.event.inputs.RUN_PYTHON_CROSS_VALIDATION == 'true'}} strategy: - matrix: + matrix: py_version: [3.8] steps: - name: Checkout code @@ -183,7 +183,7 @@ jobs: echo "====================Checking Environment & Variables=================" echo "" echo "running validations on release ${{github.event.inputs.RELEASE_VER}} RC${{github.event.inputs.RC_NUM}}." - - name: Install Kubectl + - name: Install Kubectl uses: azure/setup-kubectl@v3 - name: Setup Java JDK @@ -196,7 +196,7 @@ jobs: uses: actions/setup-python@v4 with: python-version: ${{matrix.py_version}} - + - name: Setting python env uses: ./.github/actions/common-rc-validation @@ -206,19 +206,19 @@ jobs: - name: Installing gcloud-auth-plugin run: sudo apt-get install google-cloud-sdk-gke-gcloud-auth-plugin - - name: Setting Kafka Cluster Name + - name: Setting Kafka Cluster Name run: | - echo "KAFKA_CLUSTER_NAME=xlang-kafka-cluster-$RANDOM">> $GITHUB_ENV - + echo "KAFKA_CLUSTER_NAME=xlang-kafka-cluster-$RANDOM">> $GITHUB_ENV + - name: Creating Kafka Cluster run: | gcloud container clusters create --project=${USER_GCP_PROJECT} --region=${USER_GCP_REGION} --no-enable-ip-alias $KAFKA_CLUSTER_NAME kubectl apply -R -f .test-infra/kubernetes/kafka-cluster - + - name: Waiting for Kafka cluster to be ready run: kubectl wait --for=condition=Ready pod/kafka-0 --timeout=1200s - - name: Start xlang Kafka Taxi with Dataflow Runner + - name: Start xlang Kafka Taxi with Dataflow Runner run: | echo "BOOTSTRAP_SERVERS=$(kubectl get svc outside-0 -o jsonpath='{.status.loadBalancer.ingress[0].ip}'):32400" >> $GITHUB_ENV echo "KAFKA_TAXI_DF_DATASET=${GITHUB_ACTOR}_python_validations_$(date +%m%d)_$RANDOM" >> $GITHUB_ENV @@ -243,16 +243,16 @@ jobs: --temp_location=${USER_GCS_BUCKET}/temp/ \ --with_metadata \ --beam_services="{\"sdks:java:io:expansion-service:shadowJar\": \"${KAFKA_EXPANSION_SERVICE_JAR}\"}" \ - --sdk_location apache-beam-${RELEASE_VER}.zip || true + --sdk_location apache-beam-${RELEASE_VER}.tar.gz || true - name: Checking executions results run: | - bq head -n 10 ${KAFKA_TAXI_DF_DATASET}.xlang_kafka_taxi + bq head -n 10 ${KAFKA_TAXI_DF_DATASET}.xlang_kafka_taxi - name: Remove BigQuery Dataset if: always() run: | bq rm -f ${KAFKA_TAXI_DF_DATASET}.xlang_kafka_taxi bq rm -f ${KAFKA_TAXI_DF_DATASET} - + - name: Delete Kafka Cluster if: always() run: gcloud container clusters delete --project=${USER_GCP_PROJECT} --region=${USER_GCP_REGION} --async -q $KAFKA_CLUSTER_NAME @@ -266,8 +266,8 @@ jobs: steps: - name: Sending PubSub name to env run: | - echo "SHARED_PUBSUB_TOPIC=leader_board-${GITHUB_ACTOR}-python-topic-$(date +%m%d)_$RANDOM" >> $GITHUB_ENV - - id: generate_pubsub_name + echo "SHARED_PUBSUB_TOPIC=leader_board-${GITHUB_ACTOR}-python-topic-$(date +%m%d)_$RANDOM" >> $GITHUB_ENV + - id: generate_pubsub_name run: | echo "::set-output name=pubsub::$SHARED_PUBSUB_TOPIC" - name: Creating Pub Sub Topics @@ -287,7 +287,7 @@ jobs: uses: actions/setup-python@v4 with: python-version: '3.8' - + - name: Setting python env uses: ./.github/actions/common-rc-validation with: @@ -343,14 +343,14 @@ jobs: run: | ls cd word-count-beam - timeout --preserve-status 50m mvn compile exec:java -Dexec.mainClass=org.apache.beam.examples.complete.game.injector.Injector -Dexec.args="${USER_GCP_PROJECT} ${{needs.generate_shared_pubsub.outputs.name}} none" || true - + timeout --preserve-status 50m mvn compile exec:java -Dexec.mainClass=org.apache.beam.examples.complete.game.injector.Injector -Dexec.args="${USER_GCP_PROJECT} ${{needs.generate_shared_pubsub.outputs.name}} none" || true + direct_runner_leaderboard: runs-on: [self-hosted, ubuntu-20.04] if: ${{github.event.inputs.RUN_DIRECT_RUNNER_TESTS == 'true' }} strategy: - matrix: + matrix: py_version: [3.8] needs: generate_shared_pubsub steps: @@ -363,13 +363,13 @@ jobs: uses: actions/setup-python@v4 with: python-version: ${{matrix.py_version}} - + - name: Setting python env uses: ./.github/actions/common-rc-validation with: RELEASE_VER: ${{env.RELEASE_VER}} PYTHON_RC_DOWNLOAD_URL: ${{env.PYTHON_RC_DOWNLOAD_URL}} - + - name: Exporting leaderboard Dataset Name run: echo "LEADERBOARD_DIRECT_DATASET=${GITHUB_ACTOR}_python_validations_$(date +%m%d)_$RANDOM" >> $GITHUB_ENV - name: Creating Dataset @@ -389,11 +389,11 @@ jobs: bq head -n 10 ${LEADERBOARD_DIRECT_DATASET}.leader_board_teams - name: Removing BigQuery Dataset if: always() - run: | + run: | bq rm -f ${LEADERBOARD_DIRECT_DATASET}.leader_board_users bq rm -f ${LEADERBOARD_DIRECT_DATASET}.leader_board_teams bq rm -f $LEADERBOARD_DIRECT_DATASET - + dataflow_runner_leaderboard: runs-on: [self-hosted,ubuntu-20.04] if: ${{github.event.inputs.RUN_DATAFLOW_RUNNER_TESTS=='true'}} @@ -411,13 +411,13 @@ jobs: uses: actions/setup-python@v4 with: python-version: ${{matrix.py_version}} - + - name: Setting python env uses: ./.github/actions/common-rc-validation with: RELEASE_VER: ${{env.RELEASE_VER}} PYTHON_RC_DOWNLOAD_URL: ${{env.PYTHON_RC_DOWNLOAD_URL}} - + - name: Exporting Dataflow Dataset Name run: echo "LEADERBOARD_DF_DATASET=${GITHUB_ACTOR}_python_validations_$(date +%m%d)_$RANDOM" >> $GITHUB_ENV - name: Creating Dataset @@ -434,18 +434,18 @@ jobs: --dataset ${LEADERBOARD_DF_DATASET} \ --runner DataflowRunner \ --temp_location=${USER_GCS_BUCKET}/temp/ \ - --sdk_location apache-beam-${RELEASE_VER}.zip || true + --sdk_location apache-beam-${RELEASE_VER}.tar.gz || true - name: Checking results run: | bq head -n 10 ${LEADERBOARD_DF_DATASET}.leader_board_users bq head -n 10 ${LEADERBOARD_DF_DATASET}.leader_board_teams - name: Removing BigQuery Dataset if: always() - run: | + run: | bq rm -f ${LEADERBOARD_DF_DATASET}.leader_board_users bq rm -f ${LEADERBOARD_DF_DATASET}.leader_board_teams bq rm -f $LEADERBOARD_DF_DATASET - + direct_runner_gamestats: runs-on: [self-hosted,ubuntu-20.04] @@ -463,13 +463,13 @@ jobs: uses: actions/setup-python@v4 with: python-version: ${{matrix.py_version}} - + - name: Setting python env uses: ./.github/actions/common-rc-validation with: RELEASE_VER: ${{env.RELEASE_VER}} PYTHON_RC_DOWNLOAD_URL: ${{env.PYTHON_RC_DOWNLOAD_URL}} - + - name: Exporting Gamestates Direct Dataset Name run: echo "GAMESTATS_DIRECT_DATASET=${GITHUB_ACTOR}_python_validations_$(date +%m%d)_$RANDOM" >> $GITHUB_ENV - name: Creating Dataset @@ -490,7 +490,7 @@ jobs: bq head -n 10 ${GAMESTATS_DIRECT_DATASET}.game_stats_sessions - name: Removing BigQuery Dataset if: always() - run: | + run: | bq rm -f ${GAMESTATS_DIRECT_DATASET}.game_stats_sessions bq rm -f ${GAMESTATS_DIRECT_DATASET}.game_stats_teams bq rm -f $GAMESTATS_DIRECT_DATASET @@ -512,13 +512,13 @@ jobs: uses: actions/setup-python@v4 with: python-version: ${{matrix.py_version}} - + - name: Setting python env uses: ./.github/actions/common-rc-validation with: RELEASE_VER: ${{env.RELEASE_VER}} PYTHON_RC_DOWNLOAD_URL: ${{env.PYTHON_RC_DOWNLOAD_URL}} - + - name: Exporting Gamestates Direct Dataset Name run: echo "GAMESTATS_DF_DATASET=${GITHUB_ACTOR}_python_validations_$(date +%m%d)_$RANDOM" >> $GITHUB_ENV - name: Creating Dataset @@ -535,7 +535,7 @@ jobs: --dataset ${GAMESTATS_DF_DATASET} \ --runner DataflowRunner \ --temp_location=${USER_GCS_BUCKET}/temp/ \ - --sdk_location apache-beam-${RELEASE_VER}.zip \ + --sdk_location apache-beam-${RELEASE_VER}.tar.gz \ --fixed_window_duration ${FIXED_WINDOW_DURATION} || true - name: Checking Results run: | @@ -545,9 +545,9 @@ jobs: if: always() run: | bq rm -f ${GAMESTATS_DF_DATASET}.game_stats_teams - bq rm -f ${GAMESTATS_DF_DATASET}.game_stats_sessions + bq rm -f ${GAMESTATS_DF_DATASET}.game_stats_sessions bq rm -f $GAMESTATS_DF_DATASET - + remove_shared_pubsub: runs-on: [self-hosted,ubuntu-20.04] needs: [java_injector, generate_shared_pubsub] @@ -555,4 +555,3 @@ jobs: steps: - name: Deleting Shared Pub Sub run: gcloud pubsub topics delete --project=${USER_GCP_PROJECT} ${{needs.generate_shared_pubsub.outputs.name}} - \ No newline at end of file diff --git a/contributor-docs/release-guide.md b/contributor-docs/release-guide.md index e8e8ead0902e..712e77a25b17 100644 --- a/contributor-docs/release-guide.md +++ b/contributor-docs/release-guide.md @@ -801,8 +801,8 @@ You can (optionally) also do additional verification by: signature/checksum files of Java artifacts may not contain filenames. Hence you might need to compare checksums/signatures manually or modify the files by appending the filenames.) -- [ ] Check signatures (e.g. `gpg --verify apache-beam-1.2.3-python.zip.asc - apache-beam-1.2.3-python.zip`) +- [ ] Check signatures (e.g. `gpg --verify apache-beam-1.2.3-python.tar.gz.asc + apache-beam-1.2.3-python.tar.gz`) - [ ] `grep` for legal headers in each file. - [ ] Run all jenkins suites and include links to passing tests in the voting email. @@ -1018,14 +1018,14 @@ write to BigQuery, and create a cluster of machines for running containers (for * **Verify the hashes** ``` - sha512sum -c apache-beam-2.5.0-python.zip.sha512 - sha512sum -c apache-beam-2.5.0-source-release.zip.sha512 + sha512sum -c apache-beam-2.5.0-python.tar.gz.sha512 + sha512sum -c apache-beam-2.5.0-source-release.tar.gz.sha512 ``` * **Build SDK** ``` sudo apt-get install unzip - unzip apache-beam-2.5.0-source-release.zip + unzip apache-beam-2.5.0-source-release.tar.gz python setup.py sdist ``` * **Setup virtual environment** diff --git a/release/src/main/python-release/python_release_automation_utils.sh b/release/src/main/python-release/python_release_automation_utils.sh index 2f5a9ac0a5db..337ece8ba643 100644 --- a/release/src/main/python-release/python_release_automation_utils.sh +++ b/release/src/main/python-release/python_release_automation_utils.sh @@ -97,7 +97,7 @@ function download_files() { wget -r -l2 --no-parent -nd -A "$BEAM_PYTHON_SDK_WHL*" $RC_STAGING_URL else - BEAM_PYTHON_SDK_ZIP="apache-beam-$VERSION.zip" + BEAM_PYTHON_SDK_ZIP="apache-beam-$VERSION.tar.gz" wget -r -l2 --no-parent -nd -A "$BEAM_PYTHON_SDK_ZIP*" $RC_STAGING_URL fi } @@ -128,7 +128,7 @@ function get_sha512_name() { if [[ $1 = *"wheel"* ]]; then echo $(ls | grep "/*.whl.sha512$") else - echo $(ls | grep "/*.zip.sha512$") + echo $(ls | grep "/*.tar.gz.sha512$") fi } @@ -142,7 +142,7 @@ function get_asc_name() { if [[ $1 = *"wheel"* ]]; then echo $(ls | grep "/*.whl.asc$") else - echo $(ls | grep "/*.zip.asc$") + echo $(ls | grep "/*.tar.gz.asc$") fi } diff --git a/release/src/main/scripts/build_release_candidate.sh b/release/src/main/scripts/build_release_candidate.sh index d0e6310f50aa..745d726d7655 100755 --- a/release/src/main/scripts/build_release_candidate.sh +++ b/release/src/main/scripts/build_release_candidate.sh @@ -193,19 +193,19 @@ if [[ $confirmation = "y" ]]; then echo "----------------Downloading Source Release-------------------" # GitHub strips the "v" from "v2.29.0" in naming zip and the dir inside it RC_DIR="beam-${RELEASE}-RC${RC_NUM}" - RC_ZIP="${RC_DIR}.zip" + RC_ZIP="${RC_DIR}.tar.gz" # We want to strip the -RC1 suffix from the directory name inside the zip RELEASE_DIR="beam-${RELEASE}" - SOURCE_RELEASE_ZIP="apache-beam-${RELEASE}-source-release.zip" + SOURCE_RELEASE_ZIP="apache-beam-${RELEASE}-source-release.tar.gz" # Check whether there is an existing dist dir if (svn ls "${SOURCE_RELEASE_ZIP}"); then echo "Removing existing ${SOURCE_RELEASE_ZIP}." svn delete "${SOURCE_RELEASE_ZIP}" fi - echo "Downloading: ${GIT_BEAM_ARCHIVE}/${RC_TAG}.zip" - wget ${GIT_BEAM_ARCHIVE}/${RC_TAG}.zip -O "${RC_ZIP}" + echo "Downloading: ${GIT_BEAM_ARCHIVE}/${RC_TAG}.tar.gz" + wget ${GIT_BEAM_ARCHIVE}/${RC_TAG}.tar.gz -O "${RC_ZIP}" unzip "$RC_ZIP" rm "$RC_ZIP" @@ -268,11 +268,11 @@ if [[ $confirmation = "y" ]]; then cd "${SVN_ARTIFACTS_DIR}" - echo "------Checking Hash Value for apache-beam-${RELEASE}.zip-----" - sha512sum -c "apache-beam-${RELEASE}.zip.sha512" + echo "------Checking Hash Value for apache-beam-${RELEASE}.tar.gz-----" + sha512sum -c "apache-beam-${RELEASE}.tar.gz.sha512" - echo "------Signing Source Release apache-beam-${RELEASE}.zip------" - gpg --local-user "${SIGNING_KEY}" --armor --detach-sig "apache-beam-${RELEASE}.zip" + echo "------Signing Source Release apache-beam-${RELEASE}.tar.gz------" + gpg --local-user "${SIGNING_KEY}" --armor --detach-sig "apache-beam-${RELEASE}.tar.gz" for artifact in *.whl; do echo "----------Checking Hash Value for ${artifact} wheel-----------" diff --git a/release/src/main/scripts/deploy_release_candidate_pypi.sh b/release/src/main/scripts/deploy_release_candidate_pypi.sh index d1fef8c60783..9ee90212dda0 100755 --- a/release/src/main/scripts/deploy_release_candidate_pypi.sh +++ b/release/src/main/scripts/deploy_release_candidate_pypi.sh @@ -141,8 +141,8 @@ python3 "${SCRIPT_DIR}/download_github_actions_artifacts.py" \ cd "${PYTHON_ARTIFACTS_DIR}" -echo "------Checking Hash Value for apache-beam-${RELEASE}rc${RC_NUMBER}.zip-----" -sha512sum -c "apache-beam-${RELEASE}rc${RC_NUMBER}.zip.sha512" +echo "------Checking Hash Value for apache-beam-${RELEASE}rc${RC_NUMBER}.tar.gz-----" +sha512sum -c "apache-beam-${RELEASE}rc${RC_NUMBER}.tar.gz.sha512" for artifact in *.whl; do echo "----------Checking Hash Value for ${artifact} wheel-----------" diff --git a/release/src/main/scripts/run_rc_validation.sh b/release/src/main/scripts/run_rc_validation.sh index 7f32c2979660..0f2bfe4aaec2 100755 --- a/release/src/main/scripts/run_rc_validation.sh +++ b/release/src/main/scripts/run_rc_validation.sh @@ -300,14 +300,14 @@ if [[ ("$python_leaderboard_direct" = true \ cd ${LOCAL_BEAM_DIR} echo "---------------------Downloading Python Staging RC----------------------------" - wget ${PYTHON_RC_DOWNLOAD_URL}/${RELEASE_VER}/python/apache-beam-${RELEASE_VER}.zip - wget ${PYTHON_RC_DOWNLOAD_URL}/${RELEASE_VER}/python/apache-beam-${RELEASE_VER}.zip.sha512 - if [[ ! -f apache-beam-${RELEASE_VER}.zip ]]; then + wget ${PYTHON_RC_DOWNLOAD_URL}/${RELEASE_VER}/python/apache-beam-${RELEASE_VER}.tar.gz + wget ${PYTHON_RC_DOWNLOAD_URL}/${RELEASE_VER}/python/apache-beam-${RELEASE_VER}.tar.gz.sha512 + if [[ ! -f apache-beam-${RELEASE_VER}.tar.gz ]]; then { echo "Fail to download Python Staging RC files." ;exit 1; } fi echo "--------------------------Verifying Hashes------------------------------------" - sha512sum -c apache-beam-${RELEASE_VER}.zip.sha512 + sha512sum -c apache-beam-${RELEASE_VER}.tar.gz.sha512 echo "--------------------------Updating ~/.m2/settings.xml-------------------------" cd ~ @@ -378,7 +378,7 @@ if [[ ("$python_leaderboard_direct" = true \ pip install --upgrade pip setuptools wheel echo "--------------------------Installing Python SDK-------------------------------" - pip install apache-beam-${RELEASE_VER}.zip[gcp] + pip install apache-beam-${RELEASE_VER}.tar.gz[gcp] echo "----------------Starting Leaderboard with DirectRunner-----------------------" if [[ "$python_leaderboard_direct" = true ]]; then @@ -434,7 +434,7 @@ if [[ ("$python_leaderboard_direct" = true \ --dataset ${LEADERBOARD_DF_DATASET} \ --runner DataflowRunner \ --temp_location=${USER_GCS_BUCKET}/temp/ \ - --sdk_location apache-beam-${RELEASE_VER}.zip; \ + --sdk_location apache-beam-${RELEASE_VER}.tar.gz; \ exec bash" echo "***************************************************************" @@ -509,7 +509,7 @@ if [[ ("$python_leaderboard_direct" = true \ --dataset ${GAMESTATS_DF_DATASET} \ --runner DataflowRunner \ --temp_location=${USER_GCS_BUCKET}/temp/ \ - --sdk_location apache-beam-${RELEASE_VER}.zip \ + --sdk_location apache-beam-${RELEASE_VER}.tar.gz \ --fixed_window_duration ${FIXED_WINDOW_DURATION}; exec bash" echo "***************************************************************" @@ -566,14 +566,14 @@ if [[ ("$python_xlang_quickstart" = true) \ cd ${LOCAL_BEAM_DIR} echo "---------------------Downloading Python Staging RC----------------------------" - wget ${PYTHON_RC_DOWNLOAD_URL}/${RELEASE_VER}/python/apache-beam-${RELEASE_VER}.zip - wget ${PYTHON_RC_DOWNLOAD_URL}/${RELEASE_VER}/python/apache-beam-${RELEASE_VER}.zip.sha512 - if [[ ! -f apache-beam-${RELEASE_VER}.zip ]]; then + wget ${PYTHON_RC_DOWNLOAD_URL}/${RELEASE_VER}/python/apache-beam-${RELEASE_VER}.tar.gz + wget ${PYTHON_RC_DOWNLOAD_URL}/${RELEASE_VER}/python/apache-beam-${RELEASE_VER}.tar.gz.sha512 + if [[ ! -f apache-beam-${RELEASE_VER}.tar.gz ]]; then { echo "Failed to download Python Staging RC files." ;exit 1; } fi echo "--------------------------Verifying Hashes------------------------------------" - sha512sum -c apache-beam-${RELEASE_VER}.zip.sha512 + sha512sum -c apache-beam-${RELEASE_VER}.tar.gz.sha512 `which pip` install --upgrade pip `which pip` install --upgrade setuptools @@ -593,7 +593,7 @@ if [[ ("$python_xlang_quickstart" = true) \ ln -s ${LOCAL_BEAM_DIR}/sdks beam_env_${py_version}/lib/sdks echo "--------------------------Installing Python SDK-------------------------------" - pip install apache-beam-${RELEASE_VER}.zip + pip install apache-beam-${RELEASE_VER}.tar.gz echo '************************************************************'; echo '* Running Python Multi-language Quickstart with DirectRunner'; @@ -672,14 +672,14 @@ if [[ ("$java_xlang_quickstart" = true) \ cd ${LOCAL_BEAM_DIR} echo "---------------------Downloading Python Staging RC----------------------------" - wget ${PYTHON_RC_DOWNLOAD_URL}/${RELEASE_VER}/python/apache-beam-${RELEASE_VER}.zip - wget ${PYTHON_RC_DOWNLOAD_URL}/${RELEASE_VER}/python/apache-beam-${RELEASE_VER}.zip.sha512 - if [[ ! -f apache-beam-${RELEASE_VER}.zip ]]; then + wget ${PYTHON_RC_DOWNLOAD_URL}/${RELEASE_VER}/python/apache-beam-${RELEASE_VER}.tar.gz + wget ${PYTHON_RC_DOWNLOAD_URL}/${RELEASE_VER}/python/apache-beam-${RELEASE_VER}.tar.gz.sha512 + if [[ ! -f apache-beam-${RELEASE_VER}.tar.gz ]]; then { echo "Failed to download Python Staging RC files." ;exit 1; } fi echo "--------------------------Verifying Hashes------------------------------------" - sha512sum -c apache-beam-${RELEASE_VER}.zip.sha512 + sha512sum -c apache-beam-${RELEASE_VER}.tar.gz.sha512 `which pip` install --upgrade pip `which pip` install --upgrade setuptools @@ -699,7 +699,7 @@ if [[ ("$java_xlang_quickstart" = true) \ ln -s ${LOCAL_BEAM_DIR}/sdks beam_env_${py_version}/lib/sdks echo "--------------------------Installing Python SDK-------------------------------" - pip install apache-beam-${RELEASE_VER}.zip[dataframe] + pip install apache-beam-${RELEASE_VER}.tar.gz[dataframe] # Deacrivating in the main shell. We will reactivate the virtual environment new shells # for the expansion service and the job server. @@ -768,14 +768,14 @@ if [[ ("$python_xlang_kafka_taxi_dataflow" = true cd ${LOCAL_BEAM_DIR} echo "---------------------Downloading Python Staging RC----------------------------" - wget ${PYTHON_RC_DOWNLOAD_URL}/${RELEASE_VER}/python/apache-beam-${RELEASE_VER}.zip - wget ${PYTHON_RC_DOWNLOAD_URL}/${RELEASE_VER}/python/apache-beam-${RELEASE_VER}.zip.sha512 - if [[ ! -f apache-beam-${RELEASE_VER}.zip ]]; then + wget ${PYTHON_RC_DOWNLOAD_URL}/${RELEASE_VER}/python/apache-beam-${RELEASE_VER}.tar.gz + wget ${PYTHON_RC_DOWNLOAD_URL}/${RELEASE_VER}/python/apache-beam-${RELEASE_VER}.tar.gz.sha512 + if [[ ! -f apache-beam-${RELEASE_VER}.tar.gz ]]; then { echo "Fail to download Python Staging RC files." ;exit 1; } fi echo "--------------------------Verifying Hashes------------------------------------" - sha512sum -c apache-beam-${RELEASE_VER}.zip.sha512 + sha512sum -c apache-beam-${RELEASE_VER}.tar.gz.sha512 `which pip` install --upgrade pip `which pip` install --upgrade setuptools @@ -807,7 +807,7 @@ if [[ ("$python_xlang_kafka_taxi_dataflow" = true ln -s ${LOCAL_BEAM_DIR}/sdks beam_env_${py_version}/lib/sdks echo "--------------------------Installing Python SDK-------------------------------" - pip install apache-beam-${RELEASE_VER}.zip[gcp] + pip install apache-beam-${RELEASE_VER}.tar.gz[gcp] echo "----------------Starting XLang Kafka Taxi with DataflowRunner---------------------" if [[ "$python_xlang_kafka_taxi_dataflow" = true ]]; then @@ -837,7 +837,7 @@ if [[ ("$python_xlang_kafka_taxi_dataflow" = true --temp_location=${USER_GCS_BUCKET}/temp/ \ --with_metadata \ --beam_services=\"{\\\"sdks:java:io:expansion-service:shadowJar\\\": \\\"${KAFKA_EXPANSION_SERVICE_JAR}\\\"}\" \ - --sdk_location apache-beam-${RELEASE_VER}.zip; \ + --sdk_location apache-beam-${RELEASE_VER}.tar.gz; \ exec bash" echo "***************************************************************" @@ -882,7 +882,7 @@ if [[ ("$python_xlang_kafka_taxi_dataflow" = true --temp_location=${USER_GCS_BUCKET}/temp/ \ --output_topic projects/${USER_GCP_PROJECT}/topics/${SQL_TAXI_TOPIC} \ --beam_services=\"{\\\":sdks:java:extensions:sql:expansion-service:shadowJar\\\": \\\"${SQL_EXPANSION_SERVICE_JAR}\\\"}\" \ - --sdk_location apache-beam-${RELEASE_VER}.zip; \ + --sdk_location apache-beam-${RELEASE_VER}.tar.gz; \ exec bash" echo "***************************************************************" diff --git a/sdks/python/scripts/run_snapshot_publish.sh b/sdks/python/scripts/run_snapshot_publish.sh index 6379e6f21084..bc379077349d 100755 --- a/sdks/python/scripts/run_snapshot_publish.sh +++ b/sdks/python/scripts/run_snapshot_publish.sh @@ -21,7 +21,7 @@ BUCKET=gs://beam-python-nightly-snapshots VERSION=$(awk '/__version__/{print $3}' $WORKSPACE/sdks/python/apache_beam/version.py) VERSION=$(echo $VERSION | cut -c 2- | rev | cut -c 2- | rev) time=$(date +"%Y-%m-%dT%H:%M:%S") -SNAPSHOT="apache-beam-$VERSION-$time.zip" +SNAPSHOT="apache-beam-$VERSION-$time.tar.gz" DEP_SNAPSHOT_ROOT="$BUCKET/dependency_requirements_snapshot" DEP_SNAPSHOT_FILE_NAME="beam-py-requirements-$time.txt" @@ -30,8 +30,8 @@ DEP_SNAPSHOT_FILE_NAME="beam-py-requirements-$time.txt" # and located under Gradle build directory. cd $WORKSPACE/sdks/python/build -# Rename the file to be apache-beam-{VERSION}-{datetime}.zip -for file in "apache-beam-$VERSION*.zip"; do +# Rename the file to be apache-beam-{VERSION}-{datetime}.tar.gz +for file in "apache-beam-$VERSION*.tar.gz"; do mv $file $SNAPSHOT done From 388fc394bbf82500bb25eced1bf1a3bcb026d2f7 Mon Sep 17 00:00:00 2001 From: Danny McCormick <dannymccormick@google.com> Date: Fri, 27 Oct 2023 16:03:31 -0400 Subject: [PATCH 281/435] Checkout repo for post step in website publish (#29166) * Checkout repo for post step in website publish * Disable cache * Still read from cache --- .github/actions/setup-environment-action/action.yml | 6 +++++- .github/workflows/beam_PostCommit_Website_Publish.yml | 5 ++++- 2 files changed, 9 insertions(+), 2 deletions(-) diff --git a/.github/actions/setup-environment-action/action.yml b/.github/actions/setup-environment-action/action.yml index 3452a16c132c..5c6151ca6e1f 100644 --- a/.github/actions/setup-environment-action/action.yml +++ b/.github/actions/setup-environment-action/action.yml @@ -30,6 +30,10 @@ inputs: required: false description: 'Install Go version' default: '' + disable-cache: + required: false + description: 'Whether to disable the gradle cache' + default: false runs: using: "composite" @@ -48,7 +52,7 @@ runs: - name: Setup Gradle uses: gradle/gradle-build-action@v2 with: - cache-read-only: false + cache-read-only: ${{ inputs.disable-cache }} - name: Install Go if: ${{ inputs.go-version != '' }} uses: actions/setup-go@v3 diff --git a/.github/workflows/beam_PostCommit_Website_Publish.yml b/.github/workflows/beam_PostCommit_Website_Publish.yml index 047f9564c37a..ef857d231094 100644 --- a/.github/workflows/beam_PostCommit_Website_Publish.yml +++ b/.github/workflows/beam_PostCommit_Website_Publish.yml @@ -58,8 +58,11 @@ jobs: - uses: actions/checkout@v4 - name: Setup environment uses: ./.github/actions/setup-environment-action + with: + disable-cache: true - name: run PostCommit Website Publish script uses: ./.github/actions/gradle-command-self-hosted-action with: gradle-command: :website:clean :website:publishWebsite - arguments: -PgitPublishRemote="https://github.com/apache/beam.git" \ No newline at end of file + arguments: -PgitPublishRemote="https://github.com/apache/beam.git" + - uses: actions/checkout@v4 # Extra checkout to make sure we're on master for post steps. From a256080035551440ac7998663d8d535eac0b525c Mon Sep 17 00:00:00 2001 From: Damon <damondouglas@users.noreply.github.com> Date: Fri, 27 Oct 2023 13:11:12 -0700 Subject: [PATCH 282/435] Infrastructure-as-Code deploys API & dependencies (#29155) --- .gitignore | 3 + .test-infra/mock-apis/README.md | 95 ++++++++++++++++++- .../kubernetes/echo/configmap.yaml | 30 ++++++ .../kubernetes/echo/deployment.yaml | 48 ++++++++++ .../kubernetes/echo/kustomization.yaml | 23 +++++ .../kubernetes/echo/service.yaml | 42 ++++++++ .../kubernetes/redis/kustomization.yaml | 22 +++++ .../kubernetes/redis/redis-values.yaml | 26 +++++ .../kubernetes/refresher/base/configmap.yaml | 37 ++++++++ .../kubernetes/refresher/base/deployment.yaml | 54 +++++++++++ .../refresher/base/kustomization.yaml | 22 +++++ .../configmap.yaml | 24 +++++ .../deployment.yaml | 27 ++++++ .../kustomization.yaml | 34 +++++++ .../configmap.yaml | 24 +++++ .../deployment.yaml | 27 ++++++ .../kustomization.yaml | 34 +++++++ .../configmap.yaml | 24 +++++ .../deployment.yaml | 27 ++++++ .../kustomization.yaml | 34 +++++++ .../terraform/.terraform.lock.hcl | 40 ++++++++ .../terraform/artifact_registry.tf | 29 ++++++ .../infrastructure/terraform/cluster.tf | 32 +++++++ .../mock-apis/infrastructure/terraform/iam.tf | 35 +++++++ .../infrastructure/terraform/prerequisites.tf | 59 ++++++++++++ .../infrastructure/terraform/provider.tf | 19 ++++ .../infrastructure/terraform/variables.tf | 44 +++++++++ 27 files changed, 914 insertions(+), 1 deletion(-) create mode 100644 .test-infra/mock-apis/infrastructure/kubernetes/echo/configmap.yaml create mode 100644 .test-infra/mock-apis/infrastructure/kubernetes/echo/deployment.yaml create mode 100644 .test-infra/mock-apis/infrastructure/kubernetes/echo/kustomization.yaml create mode 100644 .test-infra/mock-apis/infrastructure/kubernetes/echo/service.yaml create mode 100644 .test-infra/mock-apis/infrastructure/kubernetes/redis/kustomization.yaml create mode 100644 .test-infra/mock-apis/infrastructure/kubernetes/redis/redis-values.yaml create mode 100644 .test-infra/mock-apis/infrastructure/kubernetes/refresher/base/configmap.yaml create mode 100644 .test-infra/mock-apis/infrastructure/kubernetes/refresher/base/deployment.yaml create mode 100644 .test-infra/mock-apis/infrastructure/kubernetes/refresher/base/kustomization.yaml create mode 100644 .test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/123079b5-1e58-4b28-a185-66702e2b10c3/configmap.yaml create mode 100644 .test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/123079b5-1e58-4b28-a185-66702e2b10c3/deployment.yaml create mode 100644 .test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/123079b5-1e58-4b28-a185-66702e2b10c3/kustomization.yaml create mode 100644 .test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/e1064224-3671-46fe-971d-47887fac3d4c/configmap.yaml create mode 100644 .test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/e1064224-3671-46fe-971d-47887fac3d4c/deployment.yaml create mode 100644 .test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/e1064224-3671-46fe-971d-47887fac3d4c/kustomization.yaml create mode 100644 .test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/f588787b-28f8-4e5f-8335-f862379daf59/configmap.yaml create mode 100644 .test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/f588787b-28f8-4e5f-8335-f862379daf59/deployment.yaml create mode 100644 .test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/f588787b-28f8-4e5f-8335-f862379daf59/kustomization.yaml create mode 100644 .test-infra/mock-apis/infrastructure/terraform/.terraform.lock.hcl create mode 100644 .test-infra/mock-apis/infrastructure/terraform/artifact_registry.tf create mode 100644 .test-infra/mock-apis/infrastructure/terraform/cluster.tf create mode 100644 .test-infra/mock-apis/infrastructure/terraform/iam.tf create mode 100644 .test-infra/mock-apis/infrastructure/terraform/prerequisites.tf create mode 100644 .test-infra/mock-apis/infrastructure/terraform/provider.tf create mode 100644 .test-infra/mock-apis/infrastructure/terraform/variables.tf diff --git a/.gitignore b/.gitignore index d69995de84ec..0852e63dbd3e 100644 --- a/.gitignore +++ b/.gitignore @@ -143,3 +143,6 @@ playground/cloudfunction.zip # Exception to .gitignore .test-infra/pipelines related files !.test-infra/pipelines/**/apache-beam-testing.tfvars + +# Ignore .test-infra/mock-apis related files +.test-infra/mock-apis/**/charts/ diff --git a/.test-infra/mock-apis/README.md b/.test-infra/mock-apis/README.md index 0165421dbb21..df34757b770d 100644 --- a/.test-infra/mock-apis/README.md +++ b/.test-infra/mock-apis/README.md @@ -113,4 +113,97 @@ Follow these steps to run the services on your local machine. # Deployment -TODO: See https://github.com/apache/beam/issues/28709 +The following has already been performed for the `apache-beam-testing` project +and only needs to be done for a different Google Cloud project. + +To deploy the APIs and dependent services, run the following commands. + +## 1. Provision dependent resources in Google Cloud. + +``` +terraform -chdir=infrastructure/terraform init +terraform -chdir=infrastructure/terraform apply -var-file=apache-beam-testing.tfvars +``` + +## 2. Set the KO_DOCKER_REPO environment variable. + +After the terraform module completes, you will need to set the following: + +``` +export KO_DOCKER_REPO=<region>-docker.pkg.dev/<project>/<repository> +``` + +where: + +- `region` - is the GCP compute region +- `project` - is the GCP project id i.e. `apache-beam-testing` +- `repository` - is the repository name created by the terraform module. To +find this run: +`gcloud artifacts repositories list --project=<project> --location=<region>`. +For example, +`gcloud artifacts repositories list --project=apache-beam-testing --location=us-west1` + +## 3. Connect to the Kubernetes cluster + +Run the following command to setup credentials to the Kubernetes cluster. + +``` +gcloud container clusters get-credentials <cluster> --region <region> --project <project> +``` + +where: +- `region` - is the GCP compute region +- `project` - is the GCP project id i.e. `apache-beam-testing` +- `<cluster>` - is the name of the cluster created by the terraform module. +You can find this by running `gcloud container clusters list --project=<project> --region=<region>` + +## 4. Provision the Redis instance + +``` +kubectl kustomize --enable-helm infrastructure/kubernetes/redis | kubectl apply -f - +``` + +**You will initially see "Unschedulable" while the cluster is applying the helm +chart. It's important to wait until the helm chart completely provisions resources +before proceeding. Using Google Kubernetes Engine (GKE) autopilot may take some +time before this autoscales appropriately. ** + +## 5. Provision the Echo service + +Run the following command to provision the Echo service. + +``` +kubectl kustomize infrastructure/kubernetes/echo | ko resolve -f - | kubectl apply -f - +``` + +Like previously, you may see "Does not have minimum availability" message +showing on the status. It may take some time for GKE autopilot +to scale the node pool. + +## 6. Provision the Refresher services + +The Refresher service relies on [kustomize](https://kustomize.io) overlays +which are located at [infrastructure/kubernetes/refresher/overlays](infrastructure/kubernetes/refresher/overlays). + +Each folder contained in [infrastructure/kubernetes/refresher/overlays](infrastructure/kubernetes/refresher/overlays) +corresponds to an individual Refresher instance that is identified by the UUID. +You will need to deploy each one individually. + +For example: +``` +kubectl kustomize infrastructure/kubernetes/refresher/overlays/f588787b-28f8-4e5f-8335-f862379daf59 | ko resolve -f - | kubectl apply -f - +``` + +Like previously, you may see "Does not have minimum availability" message +showing on the status. It may take some time for GKE autopilot +to scale the node pool. + +## Additional note for creating a new Refresher service instance + +Each Refresher service instance relies on a unique UUID, where +the [kustomize](https://kustomize.io) overlay replaces in the +[infrastructure/kubernetes/refresher/base](infrastructure/kubernetes/refresher/base) +template. + +You can copy the entire folder and paste into a new one with a unique UUID +and then perform a find-replace of the old UUID with the new one. diff --git a/.test-infra/mock-apis/infrastructure/kubernetes/echo/configmap.yaml b/.test-infra/mock-apis/infrastructure/kubernetes/echo/configmap.yaml new file mode 100644 index 000000000000..831bca3e67c0 --- /dev/null +++ b/.test-infra/mock-apis/infrastructure/kubernetes/echo/configmap.yaml @@ -0,0 +1,30 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# Configures the Echo Service ConfigMap. +# See https://github.com/apache/beam/blob/master/.test-infra/mock-apis/src/main/go/cmd/service/echo/main.go +# for details on the Echo service executable and +# https://github.com/apache/beam/blob/master/.test-infra/mock-apis/src/main/go/internal/environment/variable.go +# for details on various environment variables. + +apiVersion: v1 +kind: ConfigMap +metadata: + name: echo +data: + HTTP_PORT: "8080" + GRPC_PORT: "50051" + # See .test-infra/mock-apis/infrastructure/kubernetes/redis + CACHE_HOST: redis-master.default.svc.cluster.local:6379 diff --git a/.test-infra/mock-apis/infrastructure/kubernetes/echo/deployment.yaml b/.test-infra/mock-apis/infrastructure/kubernetes/echo/deployment.yaml new file mode 100644 index 000000000000..55f246362d4b --- /dev/null +++ b/.test-infra/mock-apis/infrastructure/kubernetes/echo/deployment.yaml @@ -0,0 +1,48 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# Configures the Echo Deployment on the Kubernetes cluster. +# See https://github.com/apache/beam/blob/master/.test-infra/mock-apis/src/main/go/cmd/service/echo/main.go +# for details on the Echo service executable. +# Assumes usage of https://ko.build/ to resolve the manifest: +# export KO_DOCKER_REPO=<location>-docker.pkg.dev/<project>/<repository> +# kubectl kustomize .test-infra/mock-apis/infrastructure/echo | ko resolve -f - | kubectl apply -f - +# See .test-infra/mock-apis/README.md for details + +apiVersion: apps/v1 +kind: Deployment +metadata: + name: echo + labels: + app: echo +spec: + replicas: 3 + selector: + matchLabels: + app: echo + template: + metadata: + labels: + app: echo + spec: + containers: + - name: echo + # Prefixed with ko:// to resolve with the ko utility. + # See https://ko.build/features/k8s/ for details. + image: ko://github.com/apache/beam/test-infra/mock-apis/src/main/go/cmd/service/echo + imagePullPolicy: IfNotPresent + envFrom: + - configMapRef: + name: echo diff --git a/.test-infra/mock-apis/infrastructure/kubernetes/echo/kustomization.yaml b/.test-infra/mock-apis/infrastructure/kubernetes/echo/kustomization.yaml new file mode 100644 index 000000000000..2bfdec218afc --- /dev/null +++ b/.test-infra/mock-apis/infrastructure/kubernetes/echo/kustomization.yaml @@ -0,0 +1,23 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# Configures the provisioning of the Echo service on the Kubernetes cluster. +# See https://github.com/apache/beam/blob/master/.test-infra/mock-apis/src/main/go/cmd/service/echo/main.go +# for details on the Echo service executable. + +resources: +- configmap.yaml +- deployment.yaml +- service.yaml diff --git a/.test-infra/mock-apis/infrastructure/kubernetes/echo/service.yaml b/.test-infra/mock-apis/infrastructure/kubernetes/echo/service.yaml new file mode 100644 index 000000000000..da9a811146e5 --- /dev/null +++ b/.test-infra/mock-apis/infrastructure/kubernetes/echo/service.yaml @@ -0,0 +1,42 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# Configures the provisioning of the Echo Service on the Kubernetes cluster. +# See https://github.com/apache/beam/blob/master/.test-infra/mock-apis/src/main/go/cmd/service/echo/main.go +# for details on the Echo service executable. + +apiVersion: v1 +kind: Service +metadata: + name: echo + annotations: + # Configures the LoadBalancer to assign an internal private IP + # instead of an external private IP. + # See https://cloud.google.com/kubernetes-engine/docs/how-to/internal-load-balancing + networking.gke.io/load-balancer-type: "Internal" +spec: + type: LoadBalancer + externalTrafficPolicy: Cluster + selector: + app: echo + # Ports must match the environment variables assigned in the ConfigMap/echo. + # See configmap.yaml. + ports: + - port: 50051 + name: grpc + targetPort: 50051 + - port: 8080 + name: http + targetPort: 8080 diff --git a/.test-infra/mock-apis/infrastructure/kubernetes/redis/kustomization.yaml b/.test-infra/mock-apis/infrastructure/kubernetes/redis/kustomization.yaml new file mode 100644 index 000000000000..9d3c3256b696 --- /dev/null +++ b/.test-infra/mock-apis/infrastructure/kubernetes/redis/kustomization.yaml @@ -0,0 +1,22 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# Configures a Redis instance using https://bitnami.com/stack/redis/helm. +helmCharts: + - name: redis + releaseName: redis + repo: https://charts.bitnami.com/bitnami + version: 18.1.5 + valuesFile: redis-values.yaml diff --git a/.test-infra/mock-apis/infrastructure/kubernetes/redis/redis-values.yaml b/.test-infra/mock-apis/infrastructure/kubernetes/redis/redis-values.yaml new file mode 100644 index 000000000000..147534d4a22a --- /dev/null +++ b/.test-infra/mock-apis/infrastructure/kubernetes/redis/redis-values.yaml @@ -0,0 +1,26 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# Assigns values to the https://bitnami.com/stack/redis/helm chart. +auth: + # The cluster is used for testing only in a private Google Kubernetes Engine + # (GKE) cluster. So setting enabled to false delegates role based + # access control to Google Cloud Identity and Access Management (IAM). + enabled: false + + # We set sentinel to false, since we do not need high availability. + # See https://developer.redis.com/operate/redis-at-scale/high-availability/understanding-sentinels/ + # for more details on the sentinel mode. + sentinel: false \ No newline at end of file diff --git a/.test-infra/mock-apis/infrastructure/kubernetes/refresher/base/configmap.yaml b/.test-infra/mock-apis/infrastructure/kubernetes/refresher/base/configmap.yaml new file mode 100644 index 000000000000..eebb099b23b5 --- /dev/null +++ b/.test-infra/mock-apis/infrastructure/kubernetes/refresher/base/configmap.yaml @@ -0,0 +1,37 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# Configures the Refresher ConfigMap. +# See https://github.com/apache/beam/blob/master/.test-infra/mock-apis/src/main/go/cmd/service/refresher/main.go +# Designed for use with kustomize patch overlays. +# See https://kubectl.docs.kubernetes.io/references/kustomize/kustomization/patches/ + +apiVersion: v1 +kind: ConfigMap +metadata: + name: refresher + labels: + app.kubernetes.io/name: refresher + + # targeted for overlay replacement + quota-id: quota-id-value +data: + CACHE_HOST: redis-master.default.svc.cluster.local:6379 + + # targeted for overlay replacement + QUOTA_ID: quota-id-value + QUOTA_SIZE: "100" + QUOTA_REFRESH_INTERVAL: "10s" + diff --git a/.test-infra/mock-apis/infrastructure/kubernetes/refresher/base/deployment.yaml b/.test-infra/mock-apis/infrastructure/kubernetes/refresher/base/deployment.yaml new file mode 100644 index 000000000000..d89c4f0601f6 --- /dev/null +++ b/.test-infra/mock-apis/infrastructure/kubernetes/refresher/base/deployment.yaml @@ -0,0 +1,54 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# Configures the Refresher Deployment. +# See https://github.com/apache/beam/blob/master/.test-infra/mock-apis/src/main/go/cmd/service/refresher/main.go +# Designed for use with kustomize patch overlays. +# See https://kubectl.docs.kubernetes.io/references/kustomize/kustomization/patches/ + +apiVersion: apps/v1 +kind: Deployment +metadata: + # name created using kustomize nameSuffix as refresher-<quota-id-value> + name: refresher + + labels: + app.kubernetes.io/name: refresher + + # targeted for overlay replacement + quota-id: quota-id-value +spec: + replicas: 1 + selector: + matchLabels: + app.kubernetes.io/name: refresher + + # targeted for overlay replacement + quota-id: quota-id-value + template: + metadata: + labels: + app.kubernetes.io/name: refresher + + # targeted for overlay replacement + quota-id: quota-id-value + spec: + containers: + - name: refresher + image: ko://github.com/apache/beam/test-infra/mock-apis/src/main/go/cmd/service/refresher + imagePullPolicy: IfNotPresent + envFrom: + - configMapRef: + name: refresher \ No newline at end of file diff --git a/.test-infra/mock-apis/infrastructure/kubernetes/refresher/base/kustomization.yaml b/.test-infra/mock-apis/infrastructure/kubernetes/refresher/base/kustomization.yaml new file mode 100644 index 000000000000..8dedf1dd485d --- /dev/null +++ b/.test-infra/mock-apis/infrastructure/kubernetes/refresher/base/kustomization.yaml @@ -0,0 +1,22 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# Configures the provisioning of the Refresher service on the Kubernetes cluster. +# See https://github.com/apache/beam/blob/master/.test-infra/mock-apis/src/main/go/cmd/service/refresher/main.go +# for details on the Echo service executable. + +resources: +- configmap.yaml +- deployment.yaml diff --git a/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/123079b5-1e58-4b28-a185-66702e2b10c3/configmap.yaml b/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/123079b5-1e58-4b28-a185-66702e2b10c3/configmap.yaml new file mode 100644 index 000000000000..bf87b0646ea6 --- /dev/null +++ b/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/123079b5-1e58-4b28-a185-66702e2b10c3/configmap.yaml @@ -0,0 +1,24 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# Configures patch for ../base/configmap.yaml +# See https://kubectl.docs.kubernetes.io/references/kustomize/kustomization/patches/ + +- op: replace + path: /metadata/labels/quota-id + value: 123079b5-1e58-4b28-a185-66702e2b10c3 +- op: replace + path: /data/QUOTA_ID + value: 123079b5-1e58-4b28-a185-66702e2b10c3 diff --git a/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/123079b5-1e58-4b28-a185-66702e2b10c3/deployment.yaml b/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/123079b5-1e58-4b28-a185-66702e2b10c3/deployment.yaml new file mode 100644 index 000000000000..7ad531ce7d8b --- /dev/null +++ b/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/123079b5-1e58-4b28-a185-66702e2b10c3/deployment.yaml @@ -0,0 +1,27 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# Configures patch for ../base/deployment.yaml +# See https://kubectl.docs.kubernetes.io/references/kustomize/kustomization/patches/ + +- op: replace + path: /metadata/labels/quota-id + value: 123079b5-1e58-4b28-a185-66702e2b10c3 +- op: replace + path: /spec/selector/matchLabels/quota-id + value: 123079b5-1e58-4b28-a185-66702e2b10c3 +- op: replace + path: /spec/template/metadata/labels/quota-id + value: 123079b5-1e58-4b28-a185-66702e2b10c3 diff --git a/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/123079b5-1e58-4b28-a185-66702e2b10c3/kustomization.yaml b/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/123079b5-1e58-4b28-a185-66702e2b10c3/kustomization.yaml new file mode 100644 index 000000000000..496e53544630 --- /dev/null +++ b/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/123079b5-1e58-4b28-a185-66702e2b10c3/kustomization.yaml @@ -0,0 +1,34 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# Configures the overlay for .test-infra/mock-apis/infrastructure/kubernetes/refresher/base +# Using the Quota Id: +# 123079b5-1e58-4b28-a185-66702e2b10c3 + +resources: +- ../../base + +nameSuffix: -123079b5-1e58-4b28-a185-66702e2b10c3 + +patches: +- path: configmap.yaml + target: + kind: ConfigMap + name: refresher + +- path: deployment.yaml + target: + kind: Deployment + name: refresher diff --git a/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/e1064224-3671-46fe-971d-47887fac3d4c/configmap.yaml b/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/e1064224-3671-46fe-971d-47887fac3d4c/configmap.yaml new file mode 100644 index 000000000000..71b19b257f2b --- /dev/null +++ b/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/e1064224-3671-46fe-971d-47887fac3d4c/configmap.yaml @@ -0,0 +1,24 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# Configures patch for ../base/configmap.yaml +# See https://kubectl.docs.kubernetes.io/references/kustomize/kustomization/patches/ + +- op: replace + path: /metadata/labels/quota-id + value: e1064224-3671-46fe-971d-47887fac3d4c +- op: replace + path: /data/QUOTA_ID + value: e1064224-3671-46fe-971d-47887fac3d4c diff --git a/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/e1064224-3671-46fe-971d-47887fac3d4c/deployment.yaml b/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/e1064224-3671-46fe-971d-47887fac3d4c/deployment.yaml new file mode 100644 index 000000000000..9f13ec4b7844 --- /dev/null +++ b/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/e1064224-3671-46fe-971d-47887fac3d4c/deployment.yaml @@ -0,0 +1,27 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# Configures patch for ../base/deployment.yaml +# See https://kubectl.docs.kubernetes.io/references/kustomize/kustomization/patches/ + +- op: replace + path: /metadata/labels/quota-id + value: e1064224-3671-46fe-971d-47887fac3d4c +- op: replace + path: /spec/selector/matchLabels/quota-id + value: e1064224-3671-46fe-971d-47887fac3d4c +- op: replace + path: /spec/template/metadata/labels/quota-id + value: e1064224-3671-46fe-971d-47887fac3d4c diff --git a/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/e1064224-3671-46fe-971d-47887fac3d4c/kustomization.yaml b/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/e1064224-3671-46fe-971d-47887fac3d4c/kustomization.yaml new file mode 100644 index 000000000000..3dd6ff160abc --- /dev/null +++ b/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/e1064224-3671-46fe-971d-47887fac3d4c/kustomization.yaml @@ -0,0 +1,34 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# Configures the overlay for .test-infra/mock-apis/infrastructure/kubernetes/refresher/base +# Using the Quota Id: +# e1064224-3671-46fe-971d-47887fac3d4c + +resources: +- ../../base + +nameSuffix: -e1064224-3671-46fe-971d-47887fac3d4c + +patches: +- path: configmap.yaml + target: + kind: ConfigMap + name: refresher + +- path: deployment.yaml + target: + kind: Deployment + name: refresher diff --git a/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/f588787b-28f8-4e5f-8335-f862379daf59/configmap.yaml b/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/f588787b-28f8-4e5f-8335-f862379daf59/configmap.yaml new file mode 100644 index 000000000000..b6a12f7f133d --- /dev/null +++ b/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/f588787b-28f8-4e5f-8335-f862379daf59/configmap.yaml @@ -0,0 +1,24 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# Configures patch for ../base/configmap.yaml +# See https://kubectl.docs.kubernetes.io/references/kustomize/kustomization/patches/ + +- op: replace + path: /metadata/labels/quota-id + value: f588787b-28f8-4e5f-8335-f862379daf59 +- op: replace + path: /data/QUOTA_ID + value: f588787b-28f8-4e5f-8335-f862379daf59 diff --git a/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/f588787b-28f8-4e5f-8335-f862379daf59/deployment.yaml b/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/f588787b-28f8-4e5f-8335-f862379daf59/deployment.yaml new file mode 100644 index 000000000000..214ed9634a82 --- /dev/null +++ b/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/f588787b-28f8-4e5f-8335-f862379daf59/deployment.yaml @@ -0,0 +1,27 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# Configures patch for ../base/deployment.yaml +# See https://kubectl.docs.kubernetes.io/references/kustomize/kustomization/patches/ + +- op: replace + path: /metadata/labels/quota-id + value: f588787b-28f8-4e5f-8335-f862379daf59 +- op: replace + path: /spec/selector/matchLabels/quota-id + value: f588787b-28f8-4e5f-8335-f862379daf59 +- op: replace + path: /spec/template/metadata/labels/quota-id + value: f588787b-28f8-4e5f-8335-f862379daf59 diff --git a/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/f588787b-28f8-4e5f-8335-f862379daf59/kustomization.yaml b/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/f588787b-28f8-4e5f-8335-f862379daf59/kustomization.yaml new file mode 100644 index 000000000000..5198e4238198 --- /dev/null +++ b/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/f588787b-28f8-4e5f-8335-f862379daf59/kustomization.yaml @@ -0,0 +1,34 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# Configures the overlay for .test-infra/mock-apis/infrastructure/kubernetes/refresher/base +# Using the Quota Id: +# f588787b-28f8-4e5f-8335-f862379daf59 + +resources: +- ../../base + +nameSuffix: -f588787b-28f8-4e5f-8335-f862379daf59 + +patches: +- path: configmap.yaml + target: + kind: ConfigMap + name: refresher + +- path: deployment.yaml + target: + kind: Deployment + name: refresher diff --git a/.test-infra/mock-apis/infrastructure/terraform/.terraform.lock.hcl b/.test-infra/mock-apis/infrastructure/terraform/.terraform.lock.hcl new file mode 100644 index 000000000000..03be0106251e --- /dev/null +++ b/.test-infra/mock-apis/infrastructure/terraform/.terraform.lock.hcl @@ -0,0 +1,40 @@ +# This file is maintained automatically by "terraform init". +# Manual edits may be lost in future updates. + +provider "registry.terraform.io/hashicorp/google" { + version = "5.3.0" + hashes = [ + "h1:FGOSaAf5Fcw2GVPlGMlytcmhDxnSt3f2PfAewDS9km4=", + "zh:17849daec20cd82de916c897c730285267c62b5291bc24cd3fbdac5d10be746a", + "zh:1bab50e2eb7382e7342095417a1119e65dee1b62a5c0d93f8df724be4421c3fd", + "zh:3a800e3ea8de0d2b3b69f3256461878a5e0a6cfd0801fd762a087578ad42a207", + "zh:3dc70168baa91f6815a7e1885c4e29cadd2c67f41d9267a9278b6626c8fac594", + "zh:4000c3e16ea1bc3b5636ec18dba080135a90c0d4365597331ead9f30860041af", + "zh:58d812b8869158b2bf9c4a1a9676b6283a1914104234e8e70c36d4e1985abded", + "zh:908ff6a2a144ee76f4b68ce88164533343b2f860b8ee510107ff8e026856f5c1", + "zh:b606b6516151a947b7d9485cf330366b9c1b439677f8732cae6677cc3dc0a71f", + "zh:b623cda8316699b40db50081f79e361935d6b66b07d9dd607ed3598e51a8ffdf", + "zh:e99693fc83a8017dab5136d41a688777bb1e76076e837f2039fd6d69fe5dcfc4", + "zh:f569b65999264a9416862bca5cd2a6177d94ccb0424f3a4ef424428912b9cb3c", + "zh:fd7a7e58aa0baa9f3dd05ec693a2849ed8f724c34b8c42b3cbc4919399e622cd", + ] +} + +provider "registry.terraform.io/hashicorp/random" { + version = "3.5.1" + hashes = [ + "h1:VSnd9ZIPyfKHOObuQCaKfnjIHRtR7qTw19Rz8tJxm+k=", + "zh:04e3fbd610cb52c1017d282531364b9c53ef72b6bc533acb2a90671957324a64", + "zh:119197103301ebaf7efb91df8f0b6e0dd31e6ff943d231af35ee1831c599188d", + "zh:4d2b219d09abf3b1bb4df93d399ed156cadd61f44ad3baf5cf2954df2fba0831", + "zh:6130bdde527587bbe2dcaa7150363e96dbc5250ea20154176d82bc69df5d4ce3", + "zh:6cc326cd4000f724d3086ee05587e7710f032f94fc9af35e96a386a1c6f2214f", + "zh:78d5eefdd9e494defcb3c68d282b8f96630502cac21d1ea161f53cfe9bb483b3", + "zh:b6d88e1d28cf2dfa24e9fdcc3efc77adcdc1c3c3b5c7ce503a423efbdd6de57b", + "zh:ba74c592622ecbcef9dc2a4d81ed321c4e44cddf7da799faa324da9bf52a22b2", + "zh:c7c5cde98fe4ef1143bd1b3ec5dc04baf0d4cc3ca2c5c7d40d17c0e9b2076865", + "zh:dac4bad52c940cd0dfc27893507c1e92393846b024c5a9db159a93c534a3da03", + "zh:de8febe2a2acd9ac454b844a4106ed295ae9520ef54dc8ed2faf29f12716b602", + "zh:eab0d0495e7e711cca367f7d4df6e322e6c562fc52151ec931176115b83ed014", + ] +} diff --git a/.test-infra/mock-apis/infrastructure/terraform/artifact_registry.tf b/.test-infra/mock-apis/infrastructure/terraform/artifact_registry.tf new file mode 100644 index 000000000000..347ab8631088 --- /dev/null +++ b/.test-infra/mock-apis/infrastructure/terraform/artifact_registry.tf @@ -0,0 +1,29 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +resource "google_artifact_registry_repository" "default" { + depends_on = [google_project_service.required] + format = "DOCKER" + repository_id = local.resource_name + location = var.region +} + +// Bind the node pool service account to the roles/artifactregistry.reader role. +resource "google_artifact_registry_repository_iam_member" "default" { + depends_on = [google_project_service.required] + member = "serviceAccount:${google_service_account.node_pool.email}" + repository = google_artifact_registry_repository.default.id + role = "roles/artifactregistry.reader" +} diff --git a/.test-infra/mock-apis/infrastructure/terraform/cluster.tf b/.test-infra/mock-apis/infrastructure/terraform/cluster.tf new file mode 100644 index 000000000000..e2a3c734ebcb --- /dev/null +++ b/.test-infra/mock-apis/infrastructure/terraform/cluster.tf @@ -0,0 +1,32 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// Provision the Kubernetes cluster. +resource "google_container_cluster" "default" { + depends_on = [google_project_service.required] + deletion_protection = false + name = local.resource_name + location = var.region + enable_autopilot = true + private_cluster_config { + enable_private_nodes = true + enable_private_endpoint = false + } + network = data.google_compute_network.default.id + subnetwork = data.google_compute_network.default.id + node_config { + service_account = google_service_account.node_pool.email + } +} diff --git a/.test-infra/mock-apis/infrastructure/terraform/iam.tf b/.test-infra/mock-apis/infrastructure/terraform/iam.tf new file mode 100644 index 000000000000..7946e2fa687d --- /dev/null +++ b/.test-infra/mock-apis/infrastructure/terraform/iam.tf @@ -0,0 +1,35 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// Provision a custom service account for the node pool. +resource "google_service_account" "node_pool" { + depends_on = [google_project_service.required] + account_id = local.resource_name +} + +// Bind minimally permissive IAM roles to the node pool service account. +// See https://cloud.google.com/kubernetes-engine/docs/how-to/hardening-your-cluster#permissions +resource "google_project_iam_member" "node_pool" { + for_each = toset([ + "roles/logging.logWriter", + "roles/monitoring.metricWriter", + "roles/monitoring.viewer", + "roles/stackdriver.resourceMetadata.writer", + "roles/autoscaling.metricsWriter" + ]) + member = "serviceAccount:${google_service_account.node_pool.email}" + project = var.project + role = each.key +} diff --git a/.test-infra/mock-apis/infrastructure/terraform/prerequisites.tf b/.test-infra/mock-apis/infrastructure/terraform/prerequisites.tf new file mode 100644 index 000000000000..ae1534c67102 --- /dev/null +++ b/.test-infra/mock-apis/infrastructure/terraform/prerequisites.tf @@ -0,0 +1,59 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +locals { + resource_name = "${var.resource_name_prefix}-${random_string.postfix.result}" +} + +resource "google_project_service" "required" { + for_each = toset([ + "artifactregistry", + "cloudresourcemanager", + "container", + "iam", + ]) + service = "${each.key}.googleapis.com" + disable_on_destroy = false +} + +resource "random_string" "postfix" { + length = 6 + special = false + upper = false +} + +// Query the VPC network. +data "google_compute_network" "default" { + name = var.network +} + +// Query valid subnetwork configuration. +data "google_compute_subnetwork" "default" { + name = var.subnetwork + region = var.region + lifecycle { + postcondition { + condition = self.private_ip_google_access + error_message = "The subnetwork: regions/${var.region}/subnetworks/${var.subnetwork} in projects/${var.project}/networks/${var.network} does not have private google access enabled" + } + } +} + +// Query valid existence of the router. +data "google_compute_router" "default" { + name = var.router + region = var.region + network = data.google_compute_network.default.id +} diff --git a/.test-infra/mock-apis/infrastructure/terraform/provider.tf b/.test-infra/mock-apis/infrastructure/terraform/provider.tf new file mode 100644 index 000000000000..313d2f19fad0 --- /dev/null +++ b/.test-infra/mock-apis/infrastructure/terraform/provider.tf @@ -0,0 +1,19 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +provider "google" { + project = var.project + region = var.region +} \ No newline at end of file diff --git a/.test-infra/mock-apis/infrastructure/terraform/variables.tf b/.test-infra/mock-apis/infrastructure/terraform/variables.tf new file mode 100644 index 000000000000..bd73851e1b11 --- /dev/null +++ b/.test-infra/mock-apis/infrastructure/terraform/variables.tf @@ -0,0 +1,44 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +variable "project" { + type = string + description = "The Google Cloud (GCP) project ID within which this module provisions resources" +} + +variable "region" { + type = string + description = "The Compute region which which this module provisions resources" +} + +variable "resource_name_prefix" { + type = string + description = "The prefix to apply when naming resources followed by a random string" +} + +variable "router" { + type = string + description = "The name of the Compute Network Router" +} + +variable "network" { + type = string + description = "The Virtual Private Cloud (VPC) network ID" +} + +variable "subnetwork" { + type = string + description = "The Virtual Private Cloud (VPC) subnetwork ID" +} From 717295e25f3210d2de79a5e261029607644c8f09 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 27 Oct 2023 14:04:44 -0700 Subject: [PATCH 283/435] Bump github.com/fsouza/fake-gcs-server from 1.47.5 to 1.47.6 in /sdks (#29104) Bumps [github.com/fsouza/fake-gcs-server](https://github.com/fsouza/fake-gcs-server) from 1.47.5 to 1.47.6. - [Release notes](https://github.com/fsouza/fake-gcs-server/releases) - [Commits](https://github.com/fsouza/fake-gcs-server/compare/v1.47.5...v1.47.6) --- updated-dependencies: - dependency-name: github.com/fsouza/fake-gcs-server dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] <support@github.com> Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 4 ++-- sdks/go.sum | 8 ++++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 9837c97ab1de..e1f648254b83 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -67,7 +67,7 @@ require ( ) require ( - github.com/fsouza/fake-gcs-server v1.47.5 + github.com/fsouza/fake-gcs-server v1.47.6 golang.org/x/exp v0.0.0-20230807204917-050eac23e9de ) @@ -88,7 +88,7 @@ require ( cloud.google.com/go v0.110.8 // indirect cloud.google.com/go/compute v1.23.0 // indirect cloud.google.com/go/compute/metadata v0.2.3 // indirect - cloud.google.com/go/iam v1.1.2 // indirect + cloud.google.com/go/iam v1.1.3 // indirect cloud.google.com/go/longrunning v0.5.1 // indirect github.com/Azure/go-ansiterm v0.0.0-20210617225240-d185dfc1b5a1 // indirect github.com/Microsoft/go-winio v0.6.1 // indirect diff --git a/sdks/go.sum b/sdks/go.sum index 1b1390f57f66..f55f0bd3bc2d 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -26,8 +26,8 @@ cloud.google.com/go/datastore v1.0.0/go.mod h1:LXYbyblFSglQ5pkeyhO+Qmw7ukd3C+pD7 cloud.google.com/go/datastore v1.1.0/go.mod h1:umbIZjpQpHh4hmRpGhH4tLFup+FVzqBi1b3c64qFpCk= cloud.google.com/go/datastore v1.15.0 h1:0P9WcsQeTWjuD1H14JIY7XQscIPQ4Laje8ti96IC5vg= cloud.google.com/go/datastore v1.15.0/go.mod h1:GAeStMBIt9bPS7jMJA85kgkpsMkvseWWXiaHya9Jes8= -cloud.google.com/go/iam v1.1.2 h1:gacbrBdWcoVmGLozRuStX45YKvJtzIjJdAolzUs1sm4= -cloud.google.com/go/iam v1.1.2/go.mod h1:A5avdyVL2tCppe4unb0951eI9jreack+RJ0/d+KUZOU= +cloud.google.com/go/iam v1.1.3 h1:18tKG7DzydKWUnLjonWcJO6wjSCAtzh4GcRKlH/Hrzc= +cloud.google.com/go/iam v1.1.3/go.mod h1:3khUlaBXfPKKe7huYgEpDn6FtgRyMEqbkvBxrQyY5SE= cloud.google.com/go/kms v1.15.2 h1:lh6qra6oC4AyWe5fUUUBe/S27k12OHAleOOOw6KakdE= cloud.google.com/go/longrunning v0.5.1 h1:Fr7TXftcqTudoyRJa113hyaqlGdiBQkp0Gq7tErFDWI= cloud.google.com/go/longrunning v0.5.1/go.mod h1:spvimkwdz6SPWKEt/XBij79E9fiTkHSQl/fRUUQJYJc= @@ -192,8 +192,8 @@ github.com/form3tech-oss/jwt-go v3.2.2+incompatible/go.mod h1:pbq4aXjuKjdthFRnoD github.com/frankban/quicktest v1.2.2/go.mod h1:Qh/WofXFeiAFII1aEBu529AtJo6Zg2VHscnEsbBnJ20= github.com/frankban/quicktest v1.11.3 h1:8sXhOn0uLys67V8EsXLc6eszDs8VXWxL3iRvebPhedY= github.com/frankban/quicktest v1.11.3/go.mod h1:wRf/ReqHper53s+kmmSZizM8NamnL3IM0I9ntUbOk+k= -github.com/fsouza/fake-gcs-server v1.47.5 h1:o+wL01s01j/2OdkIaduDogXw2bZveq9TFb8f+BqEHtM= -github.com/fsouza/fake-gcs-server v1.47.5/go.mod h1:PhN8F1rHAOCL5jWyXcw8nPfLfHnka6D9fT7ctL9nbkA= +github.com/fsouza/fake-gcs-server v1.47.6 h1:/d/879q/Os9Zc5gyV3QVLfZoajN1KcWucf2zYCFeFxs= +github.com/fsouza/fake-gcs-server v1.47.6/go.mod h1:ApSXKexpG1BUXJ4f2tNCxvhTKwCPFqFLBDW2UNQDODE= github.com/go-gl/glfw v0.0.0-20190409004039-e6da0acd62b1/go.mod h1:vR7hzQXu2zJy9AVAgeJqvqgH9Q5CA+iKCZ2gyEVpxRU= github.com/go-gl/glfw/v3.3/glfw v0.0.0-20191125211704-12ad95a8df72/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= github.com/go-gl/glfw/v3.3/glfw v0.0.0-20200222043503-6f7a984d4dc4/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= From 23ba59be7bcc4e82415ecb40f2e397f439d54e86 Mon Sep 17 00:00:00 2001 From: Naireen <naireenhussain@google.com> Date: Fri, 27 Oct 2023 21:10:09 +0000 Subject: [PATCH 284/435] add html suffix to worker page dumps so chrome can render them correctly --- .../beam/runners/dataflow/worker/StreamingDataflowWorker.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java index 7110fee29362..156de23b346b 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java @@ -600,7 +600,8 @@ public void run() { + options.getWorkerId() + "_" + page.pageName() - + timestamp) + + timestamp + + ".html") .replaceAll("/", "_")); writer = new PrintWriter(outputFile, UTF_8.name()); page.captureData(writer); From 46cdcf4bf713bfb595dad91c880092b2f7a98223 Mon Sep 17 00:00:00 2001 From: Jeffrey Kinard <jeff@thekinards.com> Date: Fri, 27 Oct 2023 17:46:25 -0400 Subject: [PATCH 285/435] address initial comments Signed-off-by: Jeffrey Kinard <jeff@thekinards.com> --- sdks/python/apache_beam/yaml/yaml_mapping.py | 80 +++++++------------- 1 file changed, 28 insertions(+), 52 deletions(-) diff --git a/sdks/python/apache_beam/yaml/yaml_mapping.py b/sdks/python/apache_beam/yaml/yaml_mapping.py index 197cf7483815..7fdc7e30b4e4 100644 --- a/sdks/python/apache_beam/yaml/yaml_mapping.py +++ b/sdks/python/apache_beam/yaml/yaml_mapping.py @@ -27,31 +27,11 @@ from typing import Union import js2py -from js2py.base import JsObjectWrapper -from js2py.base import PyJsArray -from js2py.base import PyJsArrayBuffer -from js2py.base import PyJsBoolean -from js2py.base import PyJsError -from js2py.base import PyJsFloat32Array -from js2py.base import PyJsFloat64Array -from js2py.base import PyJsInt16Array -from js2py.base import PyJsInt32Array -from js2py.base import PyJsInt8Array -from js2py.base import PyJsNull -from js2py.base import PyJsNumber -from js2py.base import PyJsObject -from js2py.base import PyJsString -from js2py.base import PyJsUint16Array -from js2py.base import PyJsUint32Array -from js2py.base import PyJsUint8Array -from js2py.base import PyJsUint8ClampedArray -from js2py.base import PyJsUndefined -from js2py.base import to_python -from js2py.constructors.jsdate import PyJsDate -from js2py.internals.simplex import JsException +from js2py import base +from js2py.constructors import jsdate +from js2py.internals import simplex import apache_beam as beam -from apache_beam import Row from apache_beam.io.filesystems import FileSystems from apache_beam.portability.api import schema_pb2 from apache_beam.typehints import row_type @@ -100,16 +80,13 @@ def __setstate__(self, state): def row_to_dict(row): - if (str(type(row).__name__).startswith('BeamSchema_') or - isinstance(row, Row)): + if ((isinstance(row, tuple) and hasattr(row, '_asdict')) or + isinstance(row, beam.Row)): row = row._asdict() if isinstance(row, dict): - for key, value in row.items(): - row[key] = row_to_dict(value) + return {key: row_to_dict(value) for key, value in row.items()} elif not isinstance(row, str) and isinstance(row, Iterable): - row_list = list(row) - for idx in range(len(row_list)): - row_list[idx] = row_to_dict(row_list[idx]) + return [row_to_dict(value) for value in list(row)] return row @@ -119,36 +96,35 @@ def _expand_javascript_mapping_func( original_fields, expression=None, callable=None, path=None, name=None): js_array_type = ( - PyJsArray, - PyJsArrayBuffer, - PyJsInt8Array, - PyJsUint8Array, - PyJsUint8ClampedArray, - PyJsInt16Array, - PyJsUint16Array, - PyJsInt32Array, - PyJsUint32Array, - PyJsFloat32Array, - PyJsFloat64Array) + base.PyJsArray, + base.PyJsArrayBuffer, + base.PyJsInt8Array, + base.PyJsUint8Array, + base.PyJsUint8ClampedArray, + base.PyJsInt16Array, + base.PyJsUint16Array, + base.PyJsInt32Array, + base.PyJsUint32Array, + base.PyJsFloat32Array, + base.PyJsFloat64Array) def _js_object_to_py_object(obj): - if isinstance(obj, (PyJsNumber, PyJsString, PyJsBoolean)): - obj = to_python(obj) + if isinstance(obj, (base.PyJsNumber, base.PyJsString, base.PyJsBoolean)): + return base.to_python(obj) elif isinstance(obj, js_array_type): return [_js_object_to_py_object(value) for value in obj.to_list()] - elif isinstance(obj, PyJsDate): - obj = obj.to_utc_dt() - elif isinstance(obj, (PyJsNull, PyJsUndefined)): + elif isinstance(obj, jsdate.PyJsDate): + return obj.to_utc_dt() + elif isinstance(obj, (base.PyJsNull, base.PyJsUndefined)): return None - elif isinstance(obj, PyJsError): + elif isinstance(obj, base.PyJsError): raise RuntimeError(obj['message']) - elif isinstance(obj, PyJsObject): + elif isinstance(obj, base.PyJsObject): return { key: _js_object_to_py_object(value['value']) - for key, - value in obj.own.items() + for (key, value) in obj.own.items() } - elif isinstance(obj, JsObjectWrapper): + elif isinstance(obj, base.JsObjectWrapper): return _js_object_to_py_object(obj._obj) return obj @@ -156,7 +132,7 @@ def _js_object_to_py_object(obj): def _catch_js_errors(func): try: result = func() - except JsException as e: + except simplex.JsException as e: result = getattr(e, 'mes') return result From aab3c64566ffd51d8330e5787b72c124689453ea Mon Sep 17 00:00:00 2001 From: Svetak Sundhar <svetaksundhar@google.com> Date: Fri, 27 Oct 2023 18:07:45 -0400 Subject: [PATCH 286/435] [Go] BigTableIO support on Website (#29152) * Create HealthcareUtils file with shared resources * revert * Update Website * remove versioning --- .../www/site/content/en/documentation/io/connectors.md | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/website/www/site/content/en/documentation/io/connectors.md b/website/www/site/content/en/documentation/io/connectors.md index 59b8898aa226..0a17954d9028 100644 --- a/website/www/site/content/en/documentation/io/connectors.md +++ b/website/www/site/content/en/documentation/io/connectors.md @@ -559,7 +559,13 @@ This table provides a consolidated, at-a-glance overview of the available built- ✔ <a href="https://beam.apache.org/releases/pydoc/current/apache_beam.io.gcp.bigtableio.html">via X-language</a> </td> - <td>Not available</td> + <td class="present"> + ✔ + <a href="https://pkg.go.dev/github.com/apache/beam/sdks/v2/go/pkg/beam/io/bigtableio">native</a> (sink) + <br> + ✔ + <a href="https://pkg.go.dev/github.com/apache/beam/sdks/v2/go/pkg/beam/io/xlang/bigtableio">via X-language</a> + </td> <td>Not available</td> <td class="present">✔</td> <td class="present">✔</td> From 1f7e882e22e41842bf23422007dbff7394b6c6ec Mon Sep 17 00:00:00 2001 From: Julien Tournay <boudhevil@gmail.com> Date: Sat, 28 Oct 2023 00:09:20 +0200 Subject: [PATCH 287/435] [Flink Runner] Add UseDataStreamForBatch option to Flink runner to enable batch execution on DataStream API (#28614) Co-authored-by: Jiangjie Qin <jqin@jqin-mn1.internal.linkedin.cn> Co-authored-by: tvalentyn <tvalentyn@users.noreply.github.com> --- CHANGES.md | 4 +- .../AbstractStreamOperatorCompat.java | 16 +++- .../AbstractStreamOperatorCompat.java | 16 +++- runners/flink/flink_runner.gradle | 8 ++ .../FlinkPipelineExecutionEnvironment.java | 9 +- .../runners/flink/FlinkPipelineOptions.java | 6 +- .../FlinkStreamingPipelineTranslator.java | 5 +- .../FlinkStreamingTransformTranslators.java | 79 +++++++++++------- .../FlinkStreamingTranslationContext.java | 9 +- .../flink/FlinkTransformOverrides.java | 2 +- .../VersionDependentFlinkPipelineOptions.java | 33 ++++++++ .../wrappers/streaming/DoFnOperator.java | 37 ++++++++- .../streaming/SingletonKeyedWorkItem.java | 5 ++ .../flink/FlinkExecutionEnvironmentsTest.java | 82 +++++++++++-------- ...FlinkPipelineExecutionEnvironmentTest.java | 81 ++++++++++-------- .../flink/FlinkPipelineOptionsTest.java | 1 + .../FlinkStreamingPipelineTranslatorTest.java | 4 +- ...linkStreamingTransformTranslatorsTest.java | 44 +++++----- .../runners/flink/FlinkSubmissionTest.java | 15 ++++ .../flink/ReadSourceStreamingTest.java | 14 +++- 20 files changed, 325 insertions(+), 145 deletions(-) create mode 100644 runners/flink/src/main/java/org/apache/beam/runners/flink/VersionDependentFlinkPipelineOptions.java diff --git a/CHANGES.md b/CHANGES.md index a97035fdc297..6561cc2b56df 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -70,7 +70,9 @@ should handle this. ([#25252](https://github.com/apache/beam/issues/25252)). ## New Features / Improvements -* X feature added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). +* Add `UseDataStreamForBatch` pipeline option to the Flink runner. When it is set to true, Flink runner will run batch + jobs using the DataStream API. By default the option is set to false, so the batch jobs are still executed + using the DataSet API. * `upload_graph` as one of the Experiments options for DataflowRunner is no longer required when the graph is larger than 10MB for Java SDK ([PR#28621](https://github.com/apache/beam/pull/28621). ## Breaking Changes diff --git a/runners/flink/1.12/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/AbstractStreamOperatorCompat.java b/runners/flink/1.12/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/AbstractStreamOperatorCompat.java index bb794e04398d..5072e6b2459f 100644 --- a/runners/flink/1.12/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/AbstractStreamOperatorCompat.java +++ b/runners/flink/1.12/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/AbstractStreamOperatorCompat.java @@ -20,6 +20,7 @@ import org.apache.flink.streaming.api.operators.AbstractStreamOperator; import org.apache.flink.streaming.api.operators.InternalTimeServiceManager; import org.apache.flink.streaming.api.operators.InternalTimeServiceManagerImpl; +import org.apache.flink.streaming.api.operators.sorted.state.BatchExecutionInternalTimeServiceManager; /** Compatibility layer for {@link AbstractStreamOperator} breaking changes. */ public abstract class AbstractStreamOperatorCompat<OutputT> @@ -44,9 +45,18 @@ protected int numProcessingTimeTimers() { return getTimeServiceManager() .map( manager -> { - final InternalTimeServiceManagerImpl<?> cast = - (InternalTimeServiceManagerImpl<?>) getTimeServiceManagerCompat(); - return cast.numProcessingTimeTimers(); + InternalTimeServiceManager<?> tsm = getTimeServiceManagerCompat(); + if (tsm instanceof InternalTimeServiceManagerImpl) { + final InternalTimeServiceManagerImpl<?> cast = + (InternalTimeServiceManagerImpl<?>) getTimeServiceManagerCompat(); + return cast.numProcessingTimeTimers(); + } else if (tsm instanceof BatchExecutionInternalTimeServiceManager) { + return 0; + } else { + throw new IllegalStateException( + String.format( + "Unknown implementation of InternalTimerServiceManager. %s", tsm)); + } }) .orElse(0); } diff --git a/runners/flink/1.14/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/AbstractStreamOperatorCompat.java b/runners/flink/1.14/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/AbstractStreamOperatorCompat.java index 3b64612d6d19..d8740964fda9 100644 --- a/runners/flink/1.14/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/AbstractStreamOperatorCompat.java +++ b/runners/flink/1.14/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/AbstractStreamOperatorCompat.java @@ -20,6 +20,7 @@ import org.apache.flink.streaming.api.operators.AbstractStreamOperator; import org.apache.flink.streaming.api.operators.InternalTimeServiceManager; import org.apache.flink.streaming.api.operators.InternalTimeServiceManagerImpl; +import org.apache.flink.streaming.api.operators.sorted.state.BatchExecutionInternalTimeServiceManager; /** Compatibility layer for {@link AbstractStreamOperator} breaking changes. */ public abstract class AbstractStreamOperatorCompat<OutputT> @@ -44,9 +45,18 @@ protected int numProcessingTimeTimers() { return getTimeServiceManager() .map( manager -> { - final InternalTimeServiceManagerImpl<?> cast = - (InternalTimeServiceManagerImpl<?>) getTimeServiceManagerCompat(); - return cast.numProcessingTimeTimers(); + InternalTimeServiceManager<?> tsm = getTimeServiceManagerCompat(); + if (tsm instanceof InternalTimeServiceManagerImpl) { + final InternalTimeServiceManagerImpl<?> cast = + (InternalTimeServiceManagerImpl<?>) getTimeServiceManagerCompat(); + return cast.numProcessingTimeTimers(); + } else if (tsm instanceof BatchExecutionInternalTimeServiceManager) { + return 0; + } else { + throw new IllegalStateException( + String.format( + "Unknown implementation of InternalTimerServiceManager. %s", tsm)); + } }) .orElse(0); } diff --git a/runners/flink/flink_runner.gradle b/runners/flink/flink_runner.gradle index b1a459337e51..c510b346d5d0 100644 --- a/runners/flink/flink_runner.gradle +++ b/runners/flink/flink_runner.gradle @@ -237,6 +237,7 @@ class ValidatesRunnerConfig { String name boolean streaming boolean checkpointing + boolean useDataStreamForBatch ArrayList<String> sickbayTests } @@ -255,6 +256,7 @@ def createValidatesRunnerTask(Map m) { description = "Validates the ${runnerType} runner" def pipelineOptionsArray = ["--runner=TestFlinkRunner", "--streaming=${config.streaming}", + "--useDataStreamForBatch=${config.useDataStreamForBatch}", "--parallelism=2", ] if (config.checkpointing) { @@ -314,12 +316,17 @@ def createValidatesRunnerTask(Map m) { excludeTestsMatching 'org.apache.beam.sdk.testing.TestStreamTest.testFirstElementLate' // https://github.com/apache/beam/issues/20844 excludeTestsMatching 'org.apache.beam.sdk.testing.TestStreamTest.testLateDataAccumulating' + if (!config.streaming) { + // FlinkBatchExecutionInternalTimeService does not support timer registration on timer firing. + excludeTestsMatching 'org.apache.beam.sdk.transforms.ParDoTest$TimestampTests.testOnTimerTimestampSkew' + } } } } } createValidatesRunnerTask(name: "validatesRunnerBatch", streaming: false, sickbayTests: sickbayTests) +createValidatesRunnerTask(name: "validatesRunnerBatchWithDataStream", streaming: false, useDataStreamForBatch: true, sickbayTests: sickbayTests) createValidatesRunnerTask(name: "validatesRunnerStreaming", streaming: true, sickbayTests: sickbayTests) // We specifically have a variant which runs with checkpointing enabled for the // tests that require it since running a checkpoint variant is significantly @@ -332,6 +339,7 @@ tasks.register('validatesRunner') { group = 'Verification' description "Validates Flink runner" dependsOn validatesRunnerBatch + dependsOn validatesRunnerBatchWithDataStream dependsOn validatesRunnerStreaming dependsOn validatesRunnerStreamingCheckpointing } diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java index 7961bea6069d..12ed3603264a 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java @@ -27,6 +27,7 @@ import org.apache.beam.sdk.metrics.MetricsOptions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.flink.api.common.JobExecutionResult; +import org.apache.flink.api.common.RuntimeExecutionMode; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.core.execution.JobClient; import org.apache.flink.runtime.jobgraph.JobGraph; @@ -101,13 +102,17 @@ public void translate(Pipeline pipeline) { prepareFilesToStageForRemoteClusterExecution(options); FlinkPipelineTranslator translator; - if (options.isStreaming()) { + if (options.isStreaming() || options.getUseDataStreamForBatch()) { this.flinkStreamEnv = FlinkExecutionEnvironments.createStreamExecutionEnvironment(options); if (hasUnboundedOutput && !flinkStreamEnv.getCheckpointConfig().isCheckpointingEnabled()) { LOG.warn( "UnboundedSources present which rely on checkpointing, but checkpointing is disabled."); } - translator = new FlinkStreamingPipelineTranslator(flinkStreamEnv, options); + translator = + new FlinkStreamingPipelineTranslator(flinkStreamEnv, options, options.isStreaming()); + if (!options.isStreaming()) { + flinkStreamEnv.setRuntimeMode(RuntimeExecutionMode.BATCH); + } } else { this.flinkBatchEnv = FlinkExecutionEnvironments.createBatchExecutionEnvironment(options); translator = new FlinkBatchPipelineTranslator(flinkBatchEnv, options); diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java index 650768c7b44b..f0514c69891b 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java @@ -32,7 +32,11 @@ * requiring flink on the classpath (e.g. to use with the direct runner). */ public interface FlinkPipelineOptions - extends PipelineOptions, ApplicationNameOptions, StreamingOptions, FileStagingOptions { + extends PipelineOptions, + ApplicationNameOptions, + StreamingOptions, + FileStagingOptions, + VersionDependentFlinkPipelineOptions { String AUTO = "[auto]"; String PIPELINED = "PIPELINED"; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPipelineTranslator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPipelineTranslator.java index e9f3f7fe9176..ffc7da97cd02 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPipelineTranslator.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPipelineTranslator.java @@ -81,8 +81,9 @@ class FlinkStreamingPipelineTranslator extends FlinkPipelineTranslator { private int depth = 0; - public FlinkStreamingPipelineTranslator(StreamExecutionEnvironment env, PipelineOptions options) { - this.streamingContext = new FlinkStreamingTranslationContext(env, options); + public FlinkStreamingPipelineTranslator( + StreamExecutionEnvironment env, PipelineOptions options, boolean isStreaming) { + this.streamingContext = new FlinkStreamingTranslationContext(env, options, isStreaming); } @Override diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslators.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslators.java index 6d42d0c3b485..f3901fde03ba 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslators.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslators.java @@ -38,9 +38,7 @@ import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.runners.core.construction.SplittableParDo; import org.apache.beam.runners.core.construction.TransformPayloadTranslatorRegistrar; -import org.apache.beam.runners.core.construction.UnboundedReadFromBoundedSource.BoundedToUnboundedSourceAdapter; import org.apache.beam.runners.flink.translation.functions.FlinkAssignWindows; -import org.apache.beam.runners.flink.translation.functions.ImpulseSourceFunction; import org.apache.beam.runners.flink.translation.types.CoderTypeInformation; import org.apache.beam.runners.flink.translation.wrappers.streaming.DoFnOperator; import org.apache.beam.runners.flink.translation.wrappers.streaming.KvToByteBufferKeySelector; @@ -54,6 +52,9 @@ import org.apache.beam.runners.flink.translation.wrappers.streaming.io.DedupingOperator; import org.apache.beam.runners.flink.translation.wrappers.streaming.io.TestStreamSource; import org.apache.beam.runners.flink.translation.wrappers.streaming.io.UnboundedSourceWrapper; +import org.apache.beam.runners.flink.translation.wrappers.streaming.io.source.FlinkSource; +import org.apache.beam.runners.flink.translation.wrappers.streaming.io.source.bounded.FlinkBoundedSource; +import org.apache.beam.runners.flink.translation.wrappers.streaming.io.source.unbounded.FlinkUnboundedSource; import org.apache.beam.sdk.coders.ByteArrayCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; @@ -96,6 +97,7 @@ import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.api.common.functions.RichFlatMapFunction; import org.apache.flink.api.common.functions.RichMapFunction; @@ -220,16 +222,19 @@ public void translateNode( context.getExecutionEnvironment().getMaxParallelism() > 0 ? context.getExecutionEnvironment().getMaxParallelism() : context.getExecutionEnvironment().getParallelism(); - UnboundedSourceWrapper<T, ?> sourceWrapper = - new UnboundedSourceWrapper<>( - fullName, context.getPipelineOptions(), rawSource, parallelism); + + FlinkUnboundedSource<T> unboundedSource = + FlinkSource.unbounded( + transform.getName(), + rawSource, + new SerializablePipelineOptions(context.getPipelineOptions()), + parallelism); nonDedupSource = context .getExecutionEnvironment() - .addSource(sourceWrapper) - .name(fullName) - .uid(fullName) - .returns(withIdTypeInfo); + .fromSource( + unboundedSource, WatermarkStrategy.noWatermarks(), fullName, withIdTypeInfo) + .uid(fullName); if (rawSource.requiresDeduping()) { source = @@ -303,15 +308,24 @@ void translateNode(Impulse transform, FlinkStreamingTranslationContext context) WindowedValue.getFullCoder(ByteArrayCoder.of(), GlobalWindow.Coder.INSTANCE), context.getPipelineOptions()); - long shutdownAfterIdleSourcesMs = - context - .getPipelineOptions() - .as(FlinkPipelineOptions.class) - .getShutdownSourcesAfterIdleMs(); + FlinkBoundedSource<byte[]> impulseSource; + WatermarkStrategy<WindowedValue<byte[]>> watermarkStrategy; + if (context.isStreaming()) { + long shutdownAfterIdleSourcesMs = + context + .getPipelineOptions() + .as(FlinkPipelineOptions.class) + .getShutdownSourcesAfterIdleMs(); + impulseSource = FlinkSource.unboundedImpulse(shutdownAfterIdleSourcesMs); + watermarkStrategy = WatermarkStrategy.forMonotonousTimestamps(); + } else { + impulseSource = FlinkSource.boundedImpulse(); + watermarkStrategy = WatermarkStrategy.noWatermarks(); + } SingleOutputStreamOperator<WindowedValue<byte[]>> source = context .getExecutionEnvironment() - .addSource(new ImpulseSourceFunction(shutdownAfterIdleSourcesMs), "Impulse") + .fromSource(impulseSource, watermarkStrategy, "Impulse") .returns(typeInfo); context.setOutputDataStream(context.getOutput(transform), source); @@ -330,7 +344,8 @@ private static class ReadSourceTranslator<T> @Override void translateNode( PTransform<PBegin, PCollection<T>> transform, FlinkStreamingTranslationContext context) { - if (context.getOutput(transform).isBounded().equals(PCollection.IsBounded.BOUNDED)) { + if (ReadTranslation.sourceIsBounded(context.getCurrentTransform()) + == PCollection.IsBounded.BOUNDED) { boundedTranslator.translateNode(transform, context); } else { unboundedTranslator.translateNode(transform, context); @@ -361,24 +376,26 @@ public void translateNode( } String fullName = getCurrentTransformName(context); - UnboundedSource<T, ?> adaptedRawSource = new BoundedToUnboundedSourceAdapter<>(rawSource); + int parallelism = + context.getExecutionEnvironment().getMaxParallelism() > 0 + ? context.getExecutionEnvironment().getMaxParallelism() + : context.getExecutionEnvironment().getParallelism(); + + FlinkBoundedSource<T> flinkBoundedSource = + FlinkSource.bounded( + transform.getName(), + rawSource, + new SerializablePipelineOptions(context.getPipelineOptions()), + parallelism); + DataStream<WindowedValue<T>> source; try { - int parallelism = - context.getExecutionEnvironment().getMaxParallelism() > 0 - ? context.getExecutionEnvironment().getMaxParallelism() - : context.getExecutionEnvironment().getParallelism(); - UnboundedSourceWrapperNoValueWithRecordId<T, ?> sourceWrapper = - new UnboundedSourceWrapperNoValueWithRecordId<>( - new UnboundedSourceWrapper<>( - fullName, context.getPipelineOptions(), adaptedRawSource, parallelism)); source = context .getExecutionEnvironment() - .addSource(sourceWrapper) - .name(fullName) - .uid(fullName) - .returns(outputTypeInfo); + .fromSource( + flinkBoundedSource, WatermarkStrategy.noWatermarks(), fullName, outputTypeInfo) + .uid(fullName); } catch (Exception e) { throw new RuntimeException("Error while translating BoundedSource: " + rawSource, e); } @@ -545,7 +562,9 @@ static <InputT, OutputT> void translateParDo( KeySelector<WindowedValue<InputT>, ?> keySelector = null; boolean stateful = false; DoFnSignature signature = DoFnSignatures.getSignature(doFn.getClass()); - if (signature.stateDeclarations().size() > 0 || signature.timerDeclarations().size() > 0) { + if (!signature.stateDeclarations().isEmpty() + || !signature.timerDeclarations().isEmpty() + || !signature.timerFamilyDeclarations().isEmpty()) { // Based on the fact that the signature is stateful, DoFnSignatures ensures // that it is also keyed keyCoder = ((KvCoder) input.getCoder()).getKeyCoder(); diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTranslationContext.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTranslationContext.java index 9791eaeb4ac1..0a89bd18172b 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTranslationContext.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTranslationContext.java @@ -51,6 +51,7 @@ class FlinkStreamingTranslationContext { private final StreamExecutionEnvironment env; private final PipelineOptions options; + private final boolean isStreaming; /** * Keeps a mapping between the output value of the PTransform and the Flink Operator that produced @@ -62,9 +63,11 @@ class FlinkStreamingTranslationContext { private AppliedPTransform<?, ?, ?> currentTransform; - public FlinkStreamingTranslationContext(StreamExecutionEnvironment env, PipelineOptions options) { + public FlinkStreamingTranslationContext( + StreamExecutionEnvironment env, PipelineOptions options, boolean isStreaming) { this.env = checkNotNull(env); this.options = checkNotNull(options); + this.isStreaming = isStreaming; } public StreamExecutionEnvironment getExecutionEnvironment() { @@ -75,6 +78,10 @@ public PipelineOptions getPipelineOptions() { return options; } + public boolean isStreaming() { + return isStreaming; + } + @SuppressWarnings("unchecked") public <T> DataStream<T> getInputDataStream(PValue value) { return (DataStream<T>) dataStreams.get(value); diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkTransformOverrides.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkTransformOverrides.java index b53864d968c7..69ad58253b8e 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkTransformOverrides.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkTransformOverrides.java @@ -36,7 +36,7 @@ class FlinkTransformOverrides { static List<PTransformOverride> getDefaultOverrides(FlinkPipelineOptions options) { ImmutableList.Builder<PTransformOverride> builder = ImmutableList.builder(); - if (options.isStreaming()) { + if (options.isStreaming() || options.getUseDataStreamForBatch()) { builder .add( PTransformOverride.of( diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/VersionDependentFlinkPipelineOptions.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/VersionDependentFlinkPipelineOptions.java new file mode 100644 index 000000000000..48ee15501156 --- /dev/null +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/VersionDependentFlinkPipelineOptions.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.flink; + +import org.apache.beam.sdk.options.Default; +import org.apache.beam.sdk.options.Description; +import org.apache.beam.sdk.options.PipelineOptions; + +public interface VersionDependentFlinkPipelineOptions extends PipelineOptions { + + @Description( + "When set to true, the batch job execution will use DataStream API. " + + "Otherwise, the batch job execution will use the legacy DataSet API.") + @Default.Boolean(false) + Boolean getUseDataStreamForBatch(); + + void setUseDataStreamForBatch(Boolean useDataStreamForBatch); +} diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java index 0a9731da8b56..63f5ede00242 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java @@ -110,10 +110,12 @@ import org.apache.flink.streaming.api.operators.InternalTimeServiceManager; import org.apache.flink.streaming.api.operators.InternalTimer; import org.apache.flink.streaming.api.operators.InternalTimerService; +import org.apache.flink.streaming.api.operators.InternalTimerServiceImpl; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.operators.Output; import org.apache.flink.streaming.api.operators.Triggerable; import org.apache.flink.streaming.api.operators.TwoInputStreamOperator; +import org.apache.flink.streaming.api.operators.sorted.state.BatchExecutionInternalTimeService; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; @@ -686,6 +688,7 @@ protected final void setBundleFinishedCallback(Runnable callback) { @Override public final void processElement(StreamRecord<WindowedValue<InputT>> streamRecord) { checkInvokeStartBundle(); + LOG.trace("Processing element {} in {}", streamRecord.getValue().getValue(), doFn.getClass()); long oldHold = keyCoder != null ? keyedStateInternals.minWatermarkHoldMs() : -1L; doFnRunner.processElement(streamRecord.getValue()); checkInvokeFinishBundleByCount(); @@ -768,6 +771,7 @@ public final void processElement2(StreamRecord<RawUnionValue> streamRecord) thro @Override public final void processWatermark(Watermark mark) throws Exception { + LOG.trace("Processing watermark {} in {}", mark.getTimestamp(), doFn.getClass()); processWatermark1(mark); } @@ -1456,8 +1460,10 @@ private void populateOutputTimestampQueue(InternalTimerService<TimerData> timerS BiConsumerWithException<TimerData, Long, Exception> consumer = (timerData, stamp) -> keyedStateInternals.addWatermarkHoldUsage(timerData.getOutputTimestamp()); - timerService.forEachEventTimeTimer(consumer); - timerService.forEachProcessingTimeTimer(consumer); + if (timerService instanceof InternalTimerServiceImpl) { + timerService.forEachEventTimeTimer(consumer); + timerService.forEachProcessingTimeTimer(consumer); + } } private String constructTimerId(String timerFamilyId, String timerId) { @@ -1508,6 +1514,7 @@ public void setTimer(TimerData timer) { } private void registerTimer(TimerData timer, String contextTimerId) throws Exception { + LOG.debug("Registering timer {}", timer); pendingTimersById.put(contextTimerId, timer); long time = timer.getTimestamp().getMillis(); switch (timer.getDomain()) { @@ -1618,7 +1625,31 @@ public Instant currentProcessingTime() { @Override public Instant currentInputWatermarkTime() { - return new Instant(getEffectiveInputWatermark()); + if (timerService instanceof BatchExecutionInternalTimeService) { + // In batch mode, this method will only either return BoundedWindow.TIMESTAMP_MIN_VALUE, + // or BoundedWindow.TIMESTAMP_MAX_VALUE. + // + // For batch execution mode, the currentInputWatermark variable will never be updated + // until all the records are processed. However, every time when a record with a new + // key arrives, the Flink timer service watermark will be set to + // MAX_WATERMARK(LONG.MAX_VALUE) so that all the timers associated with the current + // key can fire. After that the Flink timer service watermark will be reset to + // LONG.MIN_VALUE, so the next key will start from a fresh env as if the previous + // records of a different key never existed. So the watermark is either Long.MIN_VALUE + // or long MAX_VALUE. So we should just use the Flink time service watermark in batch mode. + // + // In Flink the watermark ranges from + // [LONG.MIN_VALUE (-9223372036854775808), LONG.MAX_VALUE (9223372036854775807)] while the + // beam + // watermark range is [BoundedWindow.TIMESTAMP_MIN_VALUE (-9223372036854775), + // BoundedWindow.TIMESTAMP_MAX_VALUE (9223372036854775)]. To ensure the timestamp visible to + // the users follow the Beam convention, we just use the Beam range instead. + return timerService.currentWatermark() == Long.MAX_VALUE + ? new Instant(Long.MAX_VALUE) + : BoundedWindow.TIMESTAMP_MIN_VALUE; + } else { + return new Instant(getEffectiveInputWatermark()); + } } @Override diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItem.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItem.java index c4d82cb5c8ad..6f2f473feddc 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItem.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItem.java @@ -51,4 +51,9 @@ public Iterable<TimerInternals.TimerData> timersIterable() { public Iterable<WindowedValue<ElemT>> elementsIterable() { return Collections.singletonList(value); } + + @Override + public String toString() { + return String.format("{%s, [%s]}", key, value); + } } diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkExecutionEnvironmentsTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkExecutionEnvironmentsTest.java index 49d317d46ced..ec44d279586d 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkExecutionEnvironmentsTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkExecutionEnvironmentsTest.java @@ -29,6 +29,8 @@ import java.io.InputStream; import java.net.InetSocketAddress; import java.nio.file.Files; +import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.Map; @@ -49,17 +51,33 @@ import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; import org.powermock.reflect.Whitebox; /** Tests for {@link FlinkExecutionEnvironments}. */ +@RunWith(Parameterized.class) public class FlinkExecutionEnvironmentsTest { @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); @Rule public ExpectedException expectedException = ExpectedException.none(); + @Parameterized.Parameter public boolean useDataStreamForBatch; + + @Parameterized.Parameters(name = "UseDataStreamForBatch = {0}") + public static Collection<Object[]> useDataStreamForBatchJobValues() { + return Arrays.asList(new Object[][] {{false}, {true}}); + } + + private FlinkPipelineOptions getDefaultPipelineOptions() { + FlinkPipelineOptions options = FlinkPipelineOptions.defaults(); + options.setUseDataStreamForBatch(useDataStreamForBatch); + return options; + } + @Test public void shouldSetParallelismBatch() { - FlinkPipelineOptions options = FlinkPipelineOptions.defaults(); + FlinkPipelineOptions options = getDefaultPipelineOptions(); options.setRunner(TestFlinkRunner.class); options.setParallelism(42); @@ -71,7 +89,7 @@ public void shouldSetParallelismBatch() { @Test public void shouldSetParallelismStreaming() { - FlinkPipelineOptions options = FlinkPipelineOptions.defaults(); + FlinkPipelineOptions options = getDefaultPipelineOptions(); options.setRunner(TestFlinkRunner.class); options.setParallelism(42); @@ -84,7 +102,7 @@ public void shouldSetParallelismStreaming() { @Test public void shouldSetMaxParallelismStreaming() { - FlinkPipelineOptions options = FlinkPipelineOptions.defaults(); + FlinkPipelineOptions options = getDefaultPipelineOptions(); options.setRunner(TestFlinkRunner.class); options.setMaxParallelism(42); @@ -99,7 +117,7 @@ public void shouldSetMaxParallelismStreaming() { public void shouldInferParallelismFromEnvironmentBatch() throws IOException { String flinkConfDir = extractFlinkConfig(); - FlinkPipelineOptions options = FlinkPipelineOptions.defaults(); + FlinkPipelineOptions options = getDefaultPipelineOptions(); options.setRunner(TestFlinkRunner.class); options.setFlinkMaster("host:80"); @@ -115,7 +133,7 @@ public void shouldInferParallelismFromEnvironmentBatch() throws IOException { public void shouldInferParallelismFromEnvironmentStreaming() throws IOException { String confDir = extractFlinkConfig(); - FlinkPipelineOptions options = FlinkPipelineOptions.defaults(); + FlinkPipelineOptions options = getDefaultPipelineOptions(); options.setRunner(TestFlinkRunner.class); options.setFlinkMaster("host:80"); @@ -129,7 +147,7 @@ public void shouldInferParallelismFromEnvironmentStreaming() throws IOException @Test public void shouldFallbackToDefaultParallelismBatch() { - FlinkPipelineOptions options = FlinkPipelineOptions.defaults(); + FlinkPipelineOptions options = getDefaultPipelineOptions(); options.setRunner(TestFlinkRunner.class); options.setFlinkMaster("host:80"); @@ -141,7 +159,7 @@ public void shouldFallbackToDefaultParallelismBatch() { @Test public void shouldFallbackToDefaultParallelismStreaming() { - FlinkPipelineOptions options = FlinkPipelineOptions.defaults(); + FlinkPipelineOptions options = getDefaultPipelineOptions(); options.setRunner(TestFlinkRunner.class); options.setFlinkMaster("host:80"); @@ -154,7 +172,7 @@ public void shouldFallbackToDefaultParallelismStreaming() { @Test public void useDefaultParallelismFromContextBatch() { - FlinkPipelineOptions options = FlinkPipelineOptions.defaults(); + FlinkPipelineOptions options = getDefaultPipelineOptions(); options.setRunner(TestFlinkRunner.class); ExecutionEnvironment bev = FlinkExecutionEnvironments.createBatchExecutionEnvironment(options); @@ -166,7 +184,7 @@ public void useDefaultParallelismFromContextBatch() { @Test public void useDefaultParallelismFromContextStreaming() { - FlinkPipelineOptions options = FlinkPipelineOptions.defaults(); + FlinkPipelineOptions options = getDefaultPipelineOptions(); options.setRunner(TestFlinkRunner.class); StreamExecutionEnvironment sev = @@ -179,7 +197,7 @@ public void useDefaultParallelismFromContextStreaming() { @Test public void shouldParsePortForRemoteEnvironmentBatch() { - FlinkPipelineOptions options = FlinkPipelineOptions.defaults(); + FlinkPipelineOptions options = getDefaultPipelineOptions(); options.setRunner(FlinkRunner.class); options.setFlinkMaster("host:1234"); @@ -191,7 +209,7 @@ public void shouldParsePortForRemoteEnvironmentBatch() { @Test public void shouldParsePortForRemoteEnvironmentStreaming() { - FlinkPipelineOptions options = FlinkPipelineOptions.defaults(); + FlinkPipelineOptions options = getDefaultPipelineOptions(); options.setRunner(FlinkRunner.class); options.setFlinkMaster("host:1234"); @@ -204,7 +222,7 @@ public void shouldParsePortForRemoteEnvironmentStreaming() { @Test public void shouldAllowPortOmissionForRemoteEnvironmentBatch() { - FlinkPipelineOptions options = FlinkPipelineOptions.defaults(); + FlinkPipelineOptions options = getDefaultPipelineOptions(); options.setRunner(FlinkRunner.class); options.setFlinkMaster("host"); @@ -216,7 +234,7 @@ public void shouldAllowPortOmissionForRemoteEnvironmentBatch() { @Test public void shouldAllowPortOmissionForRemoteEnvironmentStreaming() { - FlinkPipelineOptions options = FlinkPipelineOptions.defaults(); + FlinkPipelineOptions options = getDefaultPipelineOptions(); options.setRunner(FlinkRunner.class); options.setFlinkMaster("host"); @@ -229,7 +247,7 @@ public void shouldAllowPortOmissionForRemoteEnvironmentStreaming() { @Test public void shouldTreatAutoAndEmptyHostTheSameBatch() { - FlinkPipelineOptions options = FlinkPipelineOptions.defaults(); + FlinkPipelineOptions options = getDefaultPipelineOptions(); options.setRunner(FlinkRunner.class); ExecutionEnvironment sev = FlinkExecutionEnvironments.createBatchExecutionEnvironment(options); @@ -243,7 +261,7 @@ public void shouldTreatAutoAndEmptyHostTheSameBatch() { @Test public void shouldTreatAutoAndEmptyHostTheSameStreaming() { - FlinkPipelineOptions options = FlinkPipelineOptions.defaults(); + FlinkPipelineOptions options = getDefaultPipelineOptions(); options.setRunner(FlinkRunner.class); StreamExecutionEnvironment sev = @@ -259,7 +277,7 @@ public void shouldTreatAutoAndEmptyHostTheSameStreaming() { @Test public void shouldDetectMalformedPortBatch() { - FlinkPipelineOptions options = FlinkPipelineOptions.defaults(); + FlinkPipelineOptions options = getDefaultPipelineOptions(); options.setRunner(FlinkRunner.class); options.setFlinkMaster("host:p0rt"); @@ -271,7 +289,7 @@ public void shouldDetectMalformedPortBatch() { @Test public void shouldDetectMalformedPortStreaming() { - FlinkPipelineOptions options = FlinkPipelineOptions.defaults(); + FlinkPipelineOptions options = getDefaultPipelineOptions(); options.setRunner(FlinkRunner.class); options.setFlinkMaster("host:p0rt"); @@ -283,7 +301,7 @@ public void shouldDetectMalformedPortStreaming() { @Test public void shouldSupportIPv4Batch() { - FlinkPipelineOptions options = FlinkPipelineOptions.defaults(); + FlinkPipelineOptions options = getDefaultPipelineOptions(); options.setRunner(FlinkRunner.class); options.setFlinkMaster("192.168.1.1:1234"); @@ -297,7 +315,7 @@ public void shouldSupportIPv4Batch() { @Test public void shouldSupportIPv4Streaming() { - FlinkPipelineOptions options = FlinkPipelineOptions.defaults(); + FlinkPipelineOptions options = getDefaultPipelineOptions(); options.setRunner(FlinkRunner.class); options.setFlinkMaster("192.168.1.1:1234"); @@ -311,7 +329,7 @@ public void shouldSupportIPv4Streaming() { @Test public void shouldSupportIPv6Batch() { - FlinkPipelineOptions options = FlinkPipelineOptions.defaults(); + FlinkPipelineOptions options = getDefaultPipelineOptions(); options.setRunner(FlinkRunner.class); options.setFlinkMaster("[FE80:CD00:0000:0CDE:1257:0000:211E:729C]:1234"); @@ -326,7 +344,7 @@ public void shouldSupportIPv6Batch() { @Test public void shouldSupportIPv6Streaming() { - FlinkPipelineOptions options = FlinkPipelineOptions.defaults(); + FlinkPipelineOptions options = getDefaultPipelineOptions(); options.setRunner(FlinkRunner.class); options.setFlinkMaster("[FE80:CD00:0000:0CDE:1257:0000:211E:729C]:1234"); @@ -342,7 +360,7 @@ public void shouldSupportIPv6Streaming() { @Test public void shouldRemoveHttpProtocolFromHostBatch() { - FlinkPipelineOptions options = FlinkPipelineOptions.defaults(); + FlinkPipelineOptions options = getDefaultPipelineOptions(); options.setRunner(FlinkRunner.class); for (String flinkMaster : @@ -358,7 +376,7 @@ public void shouldRemoveHttpProtocolFromHostBatch() { @Test public void shouldRemoveHttpProtocolFromHostStreaming() { - FlinkPipelineOptions options = FlinkPipelineOptions.defaults(); + FlinkPipelineOptions options = getDefaultPipelineOptions(); options.setRunner(FlinkRunner.class); for (String flinkMaster : @@ -382,7 +400,7 @@ private String extractFlinkConfig() throws IOException { @Test public void shouldAutoSetIdleSourcesFlagWithoutCheckpointing() { // Checkpointing disabled, shut down sources immediately - FlinkPipelineOptions options = FlinkPipelineOptions.defaults(); + FlinkPipelineOptions options = getDefaultPipelineOptions(); FlinkExecutionEnvironments.createStreamExecutionEnvironment(options); assertThat(options.getShutdownSourcesAfterIdleMs(), is(0L)); } @@ -390,7 +408,7 @@ public void shouldAutoSetIdleSourcesFlagWithoutCheckpointing() { @Test public void shouldAutoSetIdleSourcesFlagWithCheckpointing() { // Checkpointing is enabled, never shut down sources - FlinkPipelineOptions options = FlinkPipelineOptions.defaults(); + FlinkPipelineOptions options = getDefaultPipelineOptions(); options.setCheckpointingInterval(1000L); FlinkExecutionEnvironments.createStreamExecutionEnvironment(options); assertThat(options.getShutdownSourcesAfterIdleMs(), is(Long.MAX_VALUE)); @@ -399,7 +417,7 @@ public void shouldAutoSetIdleSourcesFlagWithCheckpointing() { @Test public void shouldAcceptExplicitlySetIdleSourcesFlagWithoutCheckpointing() { // Checkpointing disabled, accept flag - FlinkPipelineOptions options = FlinkPipelineOptions.defaults(); + FlinkPipelineOptions options = getDefaultPipelineOptions(); options.setShutdownSourcesAfterIdleMs(42L); FlinkExecutionEnvironments.createStreamExecutionEnvironment(options); assertThat(options.getShutdownSourcesAfterIdleMs(), is(42L)); @@ -408,7 +426,7 @@ public void shouldAcceptExplicitlySetIdleSourcesFlagWithoutCheckpointing() { @Test public void shouldAcceptExplicitlySetIdleSourcesFlagWithCheckpointing() { // Checkpointing enable, still accept flag - FlinkPipelineOptions options = FlinkPipelineOptions.defaults(); + FlinkPipelineOptions options = getDefaultPipelineOptions(); options.setCheckpointingInterval(1000L); options.setShutdownSourcesAfterIdleMs(42L); FlinkExecutionEnvironments.createStreamExecutionEnvironment(options); @@ -418,7 +436,7 @@ public void shouldAcceptExplicitlySetIdleSourcesFlagWithCheckpointing() { @Test public void shouldSetSavepointRestoreForRemoteStreaming() { String path = "fakePath"; - FlinkPipelineOptions options = FlinkPipelineOptions.defaults(); + FlinkPipelineOptions options = getDefaultPipelineOptions(); options.setRunner(TestFlinkRunner.class); options.setFlinkMaster("host:80"); options.setSavepointPath(path); @@ -432,7 +450,7 @@ public void shouldSetSavepointRestoreForRemoteStreaming() { @Test public void shouldFailOnUnknownStateBackend() { - FlinkPipelineOptions options = FlinkPipelineOptions.defaults(); + FlinkPipelineOptions options = getDefaultPipelineOptions(); options.setStreaming(true); options.setStateBackend("unknown"); options.setStateBackendStoragePath("/path"); @@ -445,7 +463,7 @@ public void shouldFailOnUnknownStateBackend() { @Test public void shouldFailOnNoStoragePathProvided() { - FlinkPipelineOptions options = FlinkPipelineOptions.defaults(); + FlinkPipelineOptions options = getDefaultPipelineOptions(); options.setStreaming(true); options.setStateBackend("unknown"); @@ -457,7 +475,7 @@ public void shouldFailOnNoStoragePathProvided() { @Test public void shouldCreateFileSystemStateBackend() { - FlinkPipelineOptions options = FlinkPipelineOptions.defaults(); + FlinkPipelineOptions options = getDefaultPipelineOptions(); options.setStreaming(true); options.setStateBackend("fileSystem"); options.setStateBackendStoragePath(temporaryFolder.getRoot().toURI().toString()); @@ -470,7 +488,7 @@ public void shouldCreateFileSystemStateBackend() { @Test public void shouldCreateRocksDbStateBackend() { - FlinkPipelineOptions options = FlinkPipelineOptions.defaults(); + FlinkPipelineOptions options = getDefaultPipelineOptions(); options.setStreaming(true); options.setStateBackend("rocksDB"); options.setStateBackendStoragePath(temporaryFolder.getRoot().toURI().toString()); diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironmentTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironmentTest.java index d8c4c6f6c8ec..676e35d4bc0f 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironmentTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironmentTest.java @@ -28,6 +28,7 @@ import static org.hamcrest.core.Every.everyItem; import static org.junit.Assert.assertThrows; import static org.junit.Assert.fail; +import static org.junit.Assume.assumeFalse; import java.io.ByteArrayOutputStream; import java.io.File; @@ -38,6 +39,8 @@ import java.net.MalformedURLException; import java.net.URL; import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; import java.util.List; import java.util.stream.Collectors; import org.apache.beam.runners.core.construction.PTransformMatchers; @@ -68,13 +71,13 @@ import org.junit.Test; import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; +import org.junit.runners.Parameterized; import org.mockito.ArgumentCaptor; import org.mockito.Mockito; import org.powermock.reflect.Whitebox; /** Tests for {@link FlinkPipelineExecutionEnvironment}. */ -@RunWith(JUnit4.class) +@RunWith(Parameterized.class) @SuppressWarnings({ "rawtypes", // TODO(https://github.com/apache/beam/issues/20447) }) @@ -82,9 +85,22 @@ public class FlinkPipelineExecutionEnvironmentTest implements Serializable { @Rule public transient TemporaryFolder tmpFolder = new TemporaryFolder(); + @Parameterized.Parameter public boolean useDataStreamForBatch; + + @Parameterized.Parameters(name = "UseDataStreamForBatch = {0}") + public static Collection<Object[]> useDataStreamForBatchJobValues() { + return Arrays.asList(new Object[][] {{false}, {true}}); + } + + private FlinkPipelineOptions getDefaultPipelineOptions() { + FlinkPipelineOptions options = FlinkPipelineOptions.defaults(); + options.setUseDataStreamForBatch(useDataStreamForBatch); + return options; + } + @Test public void shouldRecognizeAndTranslateStreamingPipeline() { - FlinkPipelineOptions options = FlinkPipelineOptions.defaults(); + FlinkPipelineOptions options = getDefaultPipelineOptions(); options.setRunner(TestFlinkRunner.class); options.setFlinkMaster("[auto]"); @@ -136,6 +152,8 @@ public void shouldNotPrepareFilesToStageWhenFlinkMasterIsSetToAuto() throws IOEx @Test public void shouldNotPrepareFilesToStagewhenFlinkMasterIsSetToCollection() throws IOException { + // StreamingExecutionEnv does not support "collection" mode. + assumeFalse(useDataStreamForBatch); FlinkPipelineOptions options = testPreparingResourcesToStage("[collection]"); assertThat(options.getFilesToStage().size(), is(2)); @@ -152,7 +170,7 @@ public void shouldNotPrepareFilesToStageWhenFlinkMasterIsSetToLocal() throws IOE @Test public void shouldUseDefaultTempLocationIfNoneSet() { - FlinkPipelineOptions options = FlinkPipelineOptions.defaults(); + FlinkPipelineOptions options = getDefaultPipelineOptions(); options.setRunner(TestFlinkRunner.class); options.setFlinkMaster("clusterAddress"); @@ -168,42 +186,33 @@ public void shouldUseDefaultTempLocationIfNoneSet() { @Test public void shouldUsePreparedFilesOnRemoteEnvironment() throws Exception { - FlinkPipelineOptions options = FlinkPipelineOptions.defaults(); - options.setRunner(TestFlinkRunner.class); - options.setFlinkMaster("clusterAddress"); - - FlinkPipelineExecutionEnvironment flinkEnv = new FlinkPipelineExecutionEnvironment(options); - - Pipeline pipeline = Pipeline.create(options); - flinkEnv.translate(pipeline); - - ExecutionEnvironment executionEnvironment = flinkEnv.getBatchExecutionEnvironment(); - assertThat(executionEnvironment, instanceOf(RemoteEnvironment.class)); - - List<URL> jarFiles = getJars(executionEnvironment); - - List<URL> urlConvertedStagedFiles = convertFilesToURLs(options.getFilesToStage()); - - assertThat(jarFiles, is(urlConvertedStagedFiles)); + shouldUsePreparedFilesOnRemoteStreamEnvironment(true); + shouldUsePreparedFilesOnRemoteStreamEnvironment(false); } - @Test - public void shouldUsePreparedFilesOnRemoteStreamEnvironment() throws Exception { - FlinkPipelineOptions options = FlinkPipelineOptions.defaults(); + public void shouldUsePreparedFilesOnRemoteStreamEnvironment(boolean streamingMode) + throws Exception { + FlinkPipelineOptions options = getDefaultPipelineOptions(); options.setRunner(TestFlinkRunner.class); options.setFlinkMaster("clusterAddress"); - options.setStreaming(true); + options.setStreaming(streamingMode); FlinkPipelineExecutionEnvironment flinkEnv = new FlinkPipelineExecutionEnvironment(options); Pipeline pipeline = Pipeline.create(options); flinkEnv.translate(pipeline); - StreamExecutionEnvironment streamExecutionEnvironment = - flinkEnv.getStreamExecutionEnvironment(); - assertThat(streamExecutionEnvironment, instanceOf(RemoteStreamEnvironment.class)); - - List<URL> jarFiles = getJars(streamExecutionEnvironment); + List<URL> jarFiles; + if (streamingMode || options.getUseDataStreamForBatch()) { + StreamExecutionEnvironment streamExecutionEnvironment = + flinkEnv.getStreamExecutionEnvironment(); + assertThat(streamExecutionEnvironment, instanceOf(RemoteStreamEnvironment.class)); + jarFiles = getJars(streamExecutionEnvironment); + } else { + ExecutionEnvironment executionEnvironment = flinkEnv.getBatchExecutionEnvironment(); + assertThat(executionEnvironment, instanceOf(RemoteEnvironment.class)); + jarFiles = getJars(executionEnvironment); + } List<URL> urlConvertedStagedFiles = convertFilesToURLs(options.getFilesToStage()); @@ -214,7 +223,7 @@ public void shouldUsePreparedFilesOnRemoteStreamEnvironment() throws Exception { public void shouldUseTransformOverrides() { boolean[] testParameters = {true, false}; for (boolean streaming : testParameters) { - FlinkPipelineOptions options = FlinkPipelineOptions.defaults(); + FlinkPipelineOptions options = getDefaultPipelineOptions(); options.setStreaming(streaming); options.setRunner(FlinkRunner.class); FlinkPipelineExecutionEnvironment flinkEnv = new FlinkPipelineExecutionEnvironment(options); @@ -234,7 +243,7 @@ public void shouldUseTransformOverrides() { @Test public void shouldProvideParallelismToTransformOverrides() { - FlinkPipelineOptions options = FlinkPipelineOptions.defaults(); + FlinkPipelineOptions options = getDefaultPipelineOptions(); options.setStreaming(true); options.setRunner(FlinkRunner.class); FlinkPipelineExecutionEnvironment flinkEnv = new FlinkPipelineExecutionEnvironment(options); @@ -278,7 +287,7 @@ public boolean matches(Object actual) { @Test public void shouldUseStreamingTransformOverridesWithUnboundedSources() { - FlinkPipelineOptions options = FlinkPipelineOptions.defaults(); + FlinkPipelineOptions options = getDefaultPipelineOptions(); // no explicit streaming mode set options.setRunner(FlinkRunner.class); FlinkPipelineExecutionEnvironment flinkEnv = new FlinkPipelineExecutionEnvironment(options); @@ -303,7 +312,7 @@ public void shouldUseStreamingTransformOverridesWithUnboundedSources() { @Test public void testTranslationModeOverrideWithUnboundedSources() { - FlinkPipelineOptions options = FlinkPipelineOptions.defaults(); + FlinkPipelineOptions options = getDefaultPipelineOptions(); options.setRunner(FlinkRunner.class); options.setStreaming(false); @@ -319,7 +328,7 @@ public void testTranslationModeOverrideWithUnboundedSources() { public void testTranslationModeNoOverrideWithoutUnboundedSources() { boolean[] testArgs = new boolean[] {true, false}; for (boolean streaming : testArgs) { - FlinkPipelineOptions options = FlinkPipelineOptions.defaults(); + FlinkPipelineOptions options = getDefaultPipelineOptions(); options.setRunner(FlinkRunner.class); options.setStreaming(streaming); @@ -408,7 +417,7 @@ private FlinkPipelineOptions testPreparingResourcesToStage( private FlinkPipelineOptions setPipelineOptions( String flinkMaster, String tempLocation, List<String> filesToStage) { - FlinkPipelineOptions options = FlinkPipelineOptions.defaults(); + FlinkPipelineOptions options = getDefaultPipelineOptions(); options.setRunner(TestFlinkRunner.class); options.setFlinkMaster(flinkMaster); options.setTempLocation(tempLocation); diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkPipelineOptionsTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkPipelineOptionsTest.java index c2d9163aacc9..da8c560690a6 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkPipelineOptionsTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkPipelineOptionsTest.java @@ -94,6 +94,7 @@ public void testDefaults() { assertThat(options.getMaxBundleSize(), is(1000L)); assertThat(options.getMaxBundleTimeMills(), is(1000L)); assertThat(options.getExecutionModeForBatch(), is(ExecutionMode.PIPELINED.name())); + assertThat(options.getUseDataStreamForBatch(), is(false)); assertThat(options.getSavepointPath(), is(nullValue())); assertThat(options.getAllowNonRestoredState(), is(false)); assertThat(options.getDisableMetrics(), is(false)); diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkStreamingPipelineTranslatorTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkStreamingPipelineTranslatorTest.java index 5d56e6ddbf67..84f1dc3c6457 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkStreamingPipelineTranslatorTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkStreamingPipelineTranslatorTest.java @@ -156,7 +156,7 @@ public void testStatefulParDoAfterCombineChaining() { private JobGraph getStatefulParDoAfterCombineChainingJobGraph(boolean stablePartitioning) { final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); final FlinkStreamingPipelineTranslator translator = - new FlinkStreamingPipelineTranslator(env, PipelineOptionsFactory.create()); + new FlinkStreamingPipelineTranslator(env, PipelineOptionsFactory.create(), true); final PipelineOptions pipelineOptions = PipelineOptionsFactory.create(); pipelineOptions.setRunner(FlinkRunner.class); final Pipeline pipeline = Pipeline.create(pipelineOptions); @@ -188,7 +188,7 @@ public void testStatefulParDoAfterGroupByKeyChaining() { private JobGraph getStatefulParDoAfterGroupByKeyChainingJobGraph(boolean stablePartitioning) { final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); final FlinkStreamingPipelineTranslator translator = - new FlinkStreamingPipelineTranslator(env, PipelineOptionsFactory.create()); + new FlinkStreamingPipelineTranslator(env, PipelineOptionsFactory.create(), true); final PipelineOptions pipelineOptions = PipelineOptionsFactory.create(); pipelineOptions.setRunner(FlinkRunner.class); final Pipeline pipeline = Pipeline.create(pipelineOptions); diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslatorsTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslatorsTest.java index 451070c1c164..d5d34b59214b 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslatorsTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslatorsTest.java @@ -29,8 +29,8 @@ import java.util.Map; import org.apache.beam.runners.core.construction.PTransformTranslation; import org.apache.beam.runners.core.construction.SplittableParDo; -import org.apache.beam.runners.flink.FlinkStreamingTransformTranslators.UnboundedSourceWrapperNoValueWithRecordId; -import org.apache.beam.runners.flink.translation.wrappers.streaming.io.UnboundedSourceWrapper; +import org.apache.beam.runners.flink.translation.wrappers.streaming.io.source.FlinkSource; +import org.apache.beam.runners.flink.translation.wrappers.streaming.io.source.bounded.FlinkBoundedSource; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.StringUtf8Coder; @@ -49,8 +49,8 @@ import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.transformations.LegacySourceTransformation; import org.apache.flink.streaming.api.transformations.OneInputTransformation; +import org.apache.flink.streaming.api.transformations.SourceTransformation; import org.checkerframework.checker.nullness.qual.Nullable; import org.junit.Test; @@ -76,11 +76,10 @@ public void readSourceTranslatorBoundedWithMaxParallelism() { Object sourceTransform = applyReadSourceTransform(transform, PCollection.IsBounded.BOUNDED, env); - UnboundedSourceWrapperNoValueWithRecordId source = - (UnboundedSourceWrapperNoValueWithRecordId) - ((LegacySourceTransformation<?>) sourceTransform).getOperator().getUserFunction(); + FlinkBoundedSource<?> source = + (FlinkBoundedSource<?>) ((SourceTransformation<?, ?, ?>) sourceTransform).getSource(); - assertEquals(maxParallelism, source.getUnderlyingSource().getSplitSources().size()); + assertEquals(maxParallelism, source.getNumSplits()); } @Test @@ -96,11 +95,10 @@ public void readSourceTranslatorBoundedWithoutMaxParallelism() { Object sourceTransform = applyReadSourceTransform(transform, PCollection.IsBounded.BOUNDED, env); - UnboundedSourceWrapperNoValueWithRecordId source = - (UnboundedSourceWrapperNoValueWithRecordId) - ((LegacySourceTransformation<?>) sourceTransform).getOperator().getUserFunction(); + FlinkBoundedSource<?> source = + (FlinkBoundedSource<?>) ((SourceTransformation<?, ?, ?>) sourceTransform).getSource(); - assertEquals(parallelism, source.getUnderlyingSource().getSplitSources().size()); + assertEquals(parallelism, source.getNumSplits()); } @Test @@ -119,13 +117,12 @@ public void readSourceTranslatorUnboundedWithMaxParallelism() { (OneInputTransformation) applyReadSourceTransform(transform, PCollection.IsBounded.UNBOUNDED, env); - UnboundedSourceWrapper source = - (UnboundedSourceWrapper) - ((LegacySourceTransformation<?>) Iterables.getOnlyElement(sourceTransform.getInputs())) - .getOperator() - .getUserFunction(); + FlinkSource<?, ?> source = + (FlinkSource<?, ?>) + ((SourceTransformation<?, ?, ?>) Iterables.getOnlyElement(sourceTransform.getInputs())) + .getSource(); - assertEquals(maxParallelism, source.getSplitSources().size()); + assertEquals(maxParallelism, source.getNumSplits()); } @Test @@ -142,13 +139,12 @@ public void readSourceTranslatorUnboundedWithoutMaxParallelism() { (OneInputTransformation) applyReadSourceTransform(transform, PCollection.IsBounded.UNBOUNDED, env); - UnboundedSourceWrapper source = - (UnboundedSourceWrapper) - ((LegacySourceTransformation<?>) Iterables.getOnlyElement(sourceTransform.getInputs())) - .getOperator() - .getUserFunction(); + FlinkSource<?, ?> source = + (FlinkSource<?, ?>) + ((SourceTransformation<?, ?, ?>) Iterables.getOnlyElement(sourceTransform.getInputs())) + .getSource(); - assertEquals(parallelism, source.getSplitSources().size()); + assertEquals(parallelism, source.getNumSplits()); } private Object applyReadSourceTransform( @@ -157,7 +153,7 @@ private Object applyReadSourceTransform( FlinkStreamingPipelineTranslator.StreamTransformTranslator<PTransform<?, ?>> translator = getReadSourceTranslator(); FlinkStreamingTranslationContext ctx = - new FlinkStreamingTranslationContext(env, PipelineOptionsFactory.create()); + new FlinkStreamingTranslationContext(env, PipelineOptionsFactory.create(), true); Pipeline pipeline = Pipeline.create(); PCollection<String> pc = diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkSubmissionTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkSubmissionTest.java index 601dbc66b1a2..b502e1129ee2 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkSubmissionTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkSubmissionTest.java @@ -72,6 +72,8 @@ public class FlinkSubmissionTest { /** Counter which keeps track of the number of jobs submitted. */ private static int expectedNumberOfJobs; + public static boolean useDataStreamForBatch; + @BeforeClass public static void beforeClass() throws Exception { Configuration config = new Configuration(); @@ -104,6 +106,12 @@ public void testSubmissionBatch() throws Exception { runSubmission(false, false); } + @Test + public void testSubmissionBatchUseDataStream() throws Exception { + FlinkSubmissionTest.useDataStreamForBatch = true; + runSubmission(false, false); + } + @Test public void testSubmissionStreaming() throws Exception { runSubmission(false, true); @@ -114,6 +122,12 @@ public void testDetachedSubmissionBatch() throws Exception { runSubmission(true, false); } + @Test + public void testDetachedSubmissionBatchUseDataStream() throws Exception { + FlinkSubmissionTest.useDataStreamForBatch = true; + runSubmission(true, false); + } + @Test public void testDetachedSubmissionStreaming() throws Exception { runSubmission(true, true); @@ -164,6 +178,7 @@ private void waitUntilJobIsCompleted() throws Exception { /** The Flink program which is executed by the CliFrontend. */ public static void main(String[] args) { FlinkPipelineOptions options = FlinkPipelineOptions.defaults(); + options.setUseDataStreamForBatch(useDataStreamForBatch); options.setRunner(FlinkRunner.class); options.setStreaming(streaming); options.setParallelism(1); diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/ReadSourceStreamingTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/ReadSourceStreamingTest.java index 8da44d4b3a83..b8dc52f6cd4b 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/ReadSourceStreamingTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/ReadSourceStreamingTest.java @@ -56,13 +56,19 @@ public void postSubmit() throws Exception { } @Test - public void testProgram() throws Exception { - runProgram(resultPath); + public void testStreaming() { + runProgram(resultPath, true); } - private static void runProgram(String resultPath) { + @Test + public void testBatch() { + runProgram(resultPath, false); + } + + private static void runProgram(String resultPath, boolean streaming) { - Pipeline p = FlinkTestPipeline.createForStreaming(); + Pipeline p = + streaming ? FlinkTestPipeline.createForStreaming() : FlinkTestPipeline.createForBatch(); p.apply(GenerateSequence.from(0).to(10)) .apply( From c2816c8d9770df3d30eec15a2563c48cb66a66b3 Mon Sep 17 00:00:00 2001 From: Robert Burke <lostluck@users.noreply.github.com> Date: Sat, 28 Oct 2023 09:11:01 -0700 Subject: [PATCH 288/435] remove anon functions from minimal wordcount (#29179) Co-authored-by: lostluck <13907733+lostluck@users.noreply.github.com> --- .../minimal_wordcount/minimal_wordcount.go | 34 ++++++++++++++----- 1 file changed, 25 insertions(+), 9 deletions(-) diff --git a/sdks/go/examples/minimal_wordcount/minimal_wordcount.go b/sdks/go/examples/minimal_wordcount/minimal_wordcount.go index f5f22cae1d65..83cb390a6093 100644 --- a/sdks/go/examples/minimal_wordcount/minimal_wordcount.go +++ b/sdks/go/examples/minimal_wordcount/minimal_wordcount.go @@ -27,6 +27,7 @@ // // Concepts: // +// 0. Registering transforms with Beam. // 1. Reading data from text files // 2. Specifying 'inline' transforms // 3. Counting items in a PCollection @@ -62,6 +63,7 @@ import ( "github.com/apache/beam/sdks/v2/go/pkg/beam" "github.com/apache/beam/sdks/v2/go/pkg/beam/io/textio" + "github.com/apache/beam/sdks/v2/go/pkg/beam/register" "github.com/apache/beam/sdks/v2/go/pkg/beam/runners/prism" "github.com/apache/beam/sdks/v2/go/pkg/beam/transforms/stats" @@ -71,6 +73,26 @@ import ( var wordRE = regexp.MustCompile(`[a-zA-Z]+('[a-z])?`) +func splitWords(line string, emit func(string)) { + for _, word := range wordRE.FindAllString(line, -1) { + emit(word) + } +} + +func formatCounts(w string, c int) string { + return fmt.Sprintf("%s: %v", w, c) +} + +// Concept #0: Transform functions executed by Beam need to be registered +// so they can be executed by portable runners. We use the register package +// in an init block to inform Beam of the functions we will be using, so +// it can access them on workers. +func init() { + register.Function2x0(splitWords) + register.Function2x1(formatCounts) + register.Emitter1[string]() +} + func main() { // beam.Init() is an initialization hook that must be called on startup. beam.Init() @@ -91,15 +113,11 @@ func main() { lines := textio.Read(s, "gs://apache-beam-samples/shakespeare/kinglear.txt") // Concept #2: Invoke a ParDo transform on our PCollection of text lines. - // This ParDo invokes a DoFn (defined in-line) on each element that + // This ParDo invokes a DoFn (registered earlier) on each element that // tokenizes the text line into individual words. The ParDo returns a // PCollection of type string, where each element is an individual word in // Shakespeare's collected texts. - words := beam.ParDo(s, func(line string, emit func(string)) { - for _, word := range wordRE.FindAllString(line, -1) { - emit(word) - } - }, lines) + words := beam.ParDo(s, splitWords, lines) // Concept #3: Invoke the stats.Count transform on our PCollection of // individual words. The Count transform returns a new PCollection of @@ -110,9 +128,7 @@ func main() { // Use a ParDo to format our PCollection of word counts into a printable // string, suitable for writing to an output file. When each element // produces exactly one element, the DoFn can simply return it. - formatted := beam.ParDo(s, func(w string, c int) string { - return fmt.Sprintf("%s: %v", w, c) - }, counted) + formatted := beam.ParDo(s, formatCounts, counted) // Concept #4: Invoke textio.Write at the end of the pipeline to write // the contents of a PCollection (in this case, our PCollection of From b5794fc27b89dd8c550fdac7879e798f155520ad Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 30 Oct 2023 10:59:40 -0400 Subject: [PATCH 289/435] Bump github.com/docker/docker in /sdks (#29193) Bumps [github.com/docker/docker](https://github.com/docker/docker) from 24.0.6+incompatible to 24.0.7+incompatible. - [Release notes](https://github.com/docker/docker/releases) - [Commits](https://github.com/docker/docker/compare/v24.0.6...v24.0.7) --- updated-dependencies: - dependency-name: github.com/docker/docker dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] <support@github.com> Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 2 +- sdks/go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index e1f648254b83..64e319040dbf 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -118,7 +118,7 @@ require ( github.com/containerd/containerd v1.7.6 // indirect github.com/cpuguy83/dockercfg v0.3.1 // indirect github.com/docker/distribution v2.8.2+incompatible // indirect - github.com/docker/docker v24.0.6+incompatible // but required to resolve issue docker has with go1.20 + github.com/docker/docker v24.0.7+incompatible // but required to resolve issue docker has with go1.20 github.com/docker/go-units v0.5.0 // indirect github.com/envoyproxy/go-control-plane v0.11.1 // indirect github.com/envoyproxy/protoc-gen-validate v1.0.2 // indirect diff --git a/sdks/go.sum b/sdks/go.sum index f55f0bd3bc2d..3e2cd5dff4d1 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -168,8 +168,8 @@ github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/docker/distribution v2.8.2+incompatible h1:T3de5rq0dB1j30rp0sA2rER+m322EBzniBPB6ZIzuh8= github.com/docker/distribution v2.8.2+incompatible/go.mod h1:J2gT2udsDAN96Uj4KfcMRqY0/ypR+oyYUYmja8H+y+w= -github.com/docker/docker v24.0.6+incompatible h1:hceabKCtUgDqPu+qm0NgsaXf28Ljf4/pWFL7xjWWDgE= -github.com/docker/docker v24.0.6+incompatible/go.mod h1:eEKB0N0r5NX/I1kEveEz05bcu8tLC/8azJZsviup8Sk= +github.com/docker/docker v24.0.7+incompatible h1:Wo6l37AuwP3JaMnZa226lzVXGA3F9Ig1seQen0cKYlM= +github.com/docker/docker v24.0.7+incompatible/go.mod h1:eEKB0N0r5NX/I1kEveEz05bcu8tLC/8azJZsviup8Sk= github.com/docker/go-connections v0.4.0 h1:El9xVISelRB7BuFusrZozjnkIM5YnzCViNKohAFqRJQ= github.com/docker/go-connections v0.4.0/go.mod h1:Gbd7IOopHjR8Iph03tsViu4nIes5XhDvyHbTtUxmeec= github.com/docker/go-units v0.4.0/go.mod h1:fgPhTUdO+D/Jk86RDLlptpiXQzgHJF7gydDDbaIK4Dk= From a93fa513181456f499e6d467f40da67fc42eda59 Mon Sep 17 00:00:00 2001 From: Robert Burke <lostluck@users.noreply.github.com> Date: Mon, 30 Oct 2023 08:00:58 -0700 Subject: [PATCH 290/435] minor prism fixes (#29183) --- .../prism/internal/jobservices/metrics.go | 2 +- .../beam/runners/prism/internal/web/web.go | 9 ++++++ .../runners/prism/internal/worker/bundle.go | 30 +++++++++---------- 3 files changed, 25 insertions(+), 16 deletions(-) diff --git a/sdks/go/pkg/beam/runners/prism/internal/jobservices/metrics.go b/sdks/go/pkg/beam/runners/prism/internal/jobservices/metrics.go index 6db16191de93..e0caec55881e 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/jobservices/metrics.go +++ b/sdks/go/pkg/beam/runners/prism/internal/jobservices/metrics.go @@ -475,8 +475,8 @@ func (m *metricsStore) AddShortIDs(resp *fnpb.MonitoringInfosMetadataResponse) { m.shortIDsToKeys[short] = key } for d, payloads := range m.unprocessedPayloads { - m.contributeMetrics(durability(d), payloads) m.unprocessedPayloads[d] = nil + m.contributeMetrics(durability(d), payloads) } } diff --git a/sdks/go/pkg/beam/runners/prism/internal/web/web.go b/sdks/go/pkg/beam/runners/prism/internal/web/web.go index b7afad35aeee..765f0b50c836 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/web/web.go +++ b/sdks/go/pkg/beam/runners/prism/internal/web/web.go @@ -230,6 +230,9 @@ func (h *jobDetailsHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) { var msecMets []string // TODO: Figure out where uniquename or id is being used in prism. It should be all global transform ID to faciliate lookups. + for _, msec := range msecs[id] { + msecMets = append(msecMets, fmt.Sprintf("\n- %+v", msec.Result())) + } for _, msec := range msecs[pt.GetUniqueName()] { msecMets = append(msecMets, fmt.Sprintf("\n- %+v", msec.Result())) } @@ -239,6 +242,12 @@ func (h *jobDetailsHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) { } var userMetrics []string + for _, ctr := range counters[id] { + userMetrics = append(userMetrics, fmt.Sprintf("\n- %s.%s: %v", ctr.Namespace(), ctr.Name(), ctr.Result())) + } + for _, dist := range distributions[id] { + userMetrics = append(userMetrics, fmt.Sprintf("\n- %s.%s: %+v", dist.Namespace(), dist.Name(), dist.Result())) + } for _, ctr := range counters[pt.GetUniqueName()] { userMetrics = append(userMetrics, fmt.Sprintf("\n- %s.%s: %v", ctr.Namespace(), ctr.Name(), ctr.Result())) } diff --git a/sdks/go/pkg/beam/runners/prism/internal/worker/bundle.go b/sdks/go/pkg/beam/runners/prism/internal/worker/bundle.go index 573bdf4aeb9d..fab8cbc141f0 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/worker/bundle.go +++ b/sdks/go/pkg/beam/runners/prism/internal/worker/bundle.go @@ -16,6 +16,7 @@ package worker import ( + "bytes" "context" "fmt" "sync/atomic" @@ -126,22 +127,21 @@ func (b *B) ProcessOn(ctx context.Context, wk *W) <-chan struct{} { } // TODO: make batching decisions. - for i, d := range b.InputData { - select { - case wk.DataReqs <- &fnpb.Elements{ - Data: []*fnpb.Elements_Data{ - { - InstructionId: b.InstID, - TransformId: b.InputTransformID, - Data: d, - IsLast: i+1 == len(b.InputData), - }, + dataBuf := bytes.Join(b.InputData, []byte{}) + select { + case wk.DataReqs <- &fnpb.Elements{ + Data: []*fnpb.Elements_Data{ + { + InstructionId: b.InstID, + TransformId: b.InputTransformID, + Data: dataBuf, + IsLast: true, }, - }: - case <-ctx.Done(): - b.DataDone() - return b.DataWait - } + }, + }: + case <-ctx.Done(): + b.DataDone() + return b.DataWait } return b.DataWait } From 472ed9f8d8645fea8aadace94ea561aa511b2d70 Mon Sep 17 00:00:00 2001 From: Robert Burke <lostluck@users.noreply.github.com> Date: Mon, 30 Oct 2023 08:48:00 -0700 Subject: [PATCH 291/435] [Go Docs] Remove outdated examples of anonymous functions from pardo, partition, and filter. (#29181) --- sdks/go/pkg/beam/pardo.go | 46 ++++++++++++-------- sdks/go/pkg/beam/partition.go | 13 ++++++ sdks/go/pkg/beam/transforms/filter/filter.go | 22 +++++++--- 3 files changed, 57 insertions(+), 24 deletions(-) diff --git a/sdks/go/pkg/beam/pardo.go b/sdks/go/pkg/beam/pardo.go index d18945834d6d..629ce329c9ba 100644 --- a/sdks/go/pkg/beam/pardo.go +++ b/sdks/go/pkg/beam/pardo.go @@ -157,11 +157,14 @@ func ParDo0(s Scope, dofn any, col PCollection, opts ...Option) { // struct may also define Setup, StartBundle, FinishBundle and Teardown methods. // The struct is JSON-serialized and may contain construction-time values. // +// Functions and types used as DoFns must be registered with beam using the +// beam `register` package, so they may execute on distributed workers. +// Functions must not be anonymous or closures, or they will fail at execution time. +// // Conceptually, when a ParDo transform is executed, the elements of the input // PCollection are first divided up into some number of "bundles". These are -// farmed off to distributed worker machines (or run locally, if using the -// direct runner). For each bundle of input elements processing proceeds as -// follows: +// farmed off to distributed worker machines (or locally on a local runner instance). +// For each bundle of input elements processing proceeds as follows: // // - If a struct, a fresh instance of the argument DoFn is created on a // worker from json serialization, and the Setup method is called on this @@ -187,10 +190,11 @@ func ParDo0(s Scope, dofn any, col PCollection, opts ...Option) { // // For example: // +// func stringLen(word string) int { return len(word) } +// func init() { register.Function1x1(stringLen) } +// // words := beam.ParDo(s, &Foo{...}, ...) -// lengths := beam.ParDo(s, func (word string) int) { -// return len(word) -// }, words) +// lengths := beam.ParDo(s, stringLen, words) // // Each output element has the same timestamp and is in the same windows as its // corresponding input element. The timestamp can be accessed and/or emitted by @@ -207,28 +211,34 @@ func ParDo0(s Scope, dofn any, col PCollection, opts ...Option) { // options, and their contents accessible to each of the DoFn operations. For // example: // +// func filterLessThanCutoff(word string, cutoff int, emit func(string)) { +// if len(word) < cutoff { +// emit(word) +// } +// } +// func init() { register.Function3x0(filterLessThanCutoff) } +// // words := ... // cufoff := ... // Singleton PCollection<int> -// smallWords := beam.ParDo(s, func (word string, cutoff int, emit func(string)) { -// if len(word) < cutoff { -// emit(word) -// } -// }, words, beam.SideInput{Input: cutoff}) +// smallWords := beam.ParDo(s, filterLessThanCutoff, words, beam.SideInput{Input: cutoff}) // // # Additional Outputs // // Optionally, a ParDo transform can produce zero or multiple output // PCollections. Note the use of ParDo2 to specfic 2 outputs. For example: // +// func partitionAtCutoff(word string, cutoff int, small, big func(string)) { +// if len(word) < cutoff { +// small(word) +// } else { +// big(word) +// } +// } +// func init() { register.Function4x0(partitionAtCutoff) } +// // words := ... // cufoff := ... // Singleton PCollection<int> -// small, big := beam.ParDo2(s, func (word string, cutoff int, small, big func(string)) { -// if len(word) < cutoff { -// small(word) -// } else { -// big(word) -// } -// }, words, beam.SideInput{Input: cutoff}) +// small, big := beam.ParDo2(s, partitionAtCutoff, words, beam.SideInput{Input: cutoff}) // // By default, the Coders for the elements of each output PCollections is // inferred from the concrete type. diff --git a/sdks/go/pkg/beam/partition.go b/sdks/go/pkg/beam/partition.go index 37498ddbc0bd..1c79965ea63b 100644 --- a/sdks/go/pkg/beam/partition.go +++ b/sdks/go/pkg/beam/partition.go @@ -39,6 +39,19 @@ var ( // // A PartitionFn has the signature `func(T) int.` // +// func lenToTen(s string) int { +// if len(s) > 9 { +// return 10 +// } +// return len(s) +// } +// +// // Partition functions must be registered with Beam, and must not be closures. +// func init() { register.Function1x1(lenToTen) } +// +// // The number of partitions goes up to 11 since we can return 0 through 10 +// wordsByLength := beam.Partition(s, 11, lenToTen, inputStrings) +// // T is permitted to be a KV. func Partition(s Scope, n int, fn any, col PCollection) []PCollection { s = s.Scope(fmt.Sprintf("Partition(%v)", n)) diff --git a/sdks/go/pkg/beam/transforms/filter/filter.go b/sdks/go/pkg/beam/transforms/filter/filter.go index 997eec5eb4ef..699ec9c4c792 100644 --- a/sdks/go/pkg/beam/transforms/filter/filter.go +++ b/sdks/go/pkg/beam/transforms/filter/filter.go @@ -40,10 +40,15 @@ var ( // the filter function returns false. It returns a PCollection of the same type // as the input. For example: // +// func lessThanThree(s string) bool { +// return len(s) < 3 +// } +// +// // Filter functions must be registered with Beam, and must not be closures. +// func init() { register.Function1x1(lessThanThree) } +// // words := beam.Create(s, "a", "b", "long", "alsolong") -// short := filter.Include(s, words, func(s string) bool { -// return len(s) < 3 -// }) +// short := filter.Include(s, words, lessThanThree) // // Here, "short" will contain "a" and "b" at runtime. func Include(s beam.Scope, col beam.PCollection, fn any) beam.PCollection { @@ -58,10 +63,15 @@ func Include(s beam.Scope, col beam.PCollection, fn any) beam.PCollection { // the filter function returns true. It returns a PCollection of the same type // as the input. For example: // +// func lessThanThree(s string) bool { +// return len(s) < 3 +// } +// +// // Filter functions must be registered with Beam, and must not be closures. +// func init() { register.Function1x1(lessThanThree) } +// // words := beam.Create(s, "a", "b", "long", "alsolong") -// long := filter.Exclude(s, words, func(s string) bool { -// return len(s) < 3 -// }) +// long := filter.Exclude(s, words, lessThanThree) // // Here, "long" will contain "long" and "alsolong" at runtime. func Exclude(s beam.Scope, col beam.PCollection, fn any) beam.PCollection { From 84e6ceb2f0c9906fc1d64f630a320ea7c2126e92 Mon Sep 17 00:00:00 2001 From: Robert Burke <lostluck@users.noreply.github.com> Date: Mon, 30 Oct 2023 08:48:54 -0700 Subject: [PATCH 292/435] increase received message size to max for Job Management. (#29178) Co-authored-by: lostluck <13907733+lostluck@users.noreply.github.com> --- .../go/pkg/beam/runners/prism/internal/jobservices/server.go | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/sdks/go/pkg/beam/runners/prism/internal/jobservices/server.go b/sdks/go/pkg/beam/runners/prism/internal/jobservices/server.go index bf2db814813c..647e9ad96283 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/jobservices/server.go +++ b/sdks/go/pkg/beam/runners/prism/internal/jobservices/server.go @@ -17,6 +17,7 @@ package jobservices import ( "fmt" + "math" "net" "sync" @@ -60,7 +61,9 @@ func NewServer(port int, execute func(*Job)) *Server { execute: execute, } slog.Info("Serving JobManagement", slog.String("endpoint", s.Endpoint())) - var opts []grpc.ServerOption + opts := []grpc.ServerOption{ + grpc.MaxRecvMsgSize(math.MaxInt32), + } s.server = grpc.NewServer(opts...) jobpb.RegisterJobServiceServer(s.server, s) jobpb.RegisterArtifactStagingServiceServer(s.server, s) From 2f8eebb7fa7a439eecba604eacaf3611ae6f5fb6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Johanna=20=C3=96jeling?= <51084516+johannaojeling@users.noreply.github.com> Date: Mon, 30 Oct 2023 16:53:37 +0100 Subject: [PATCH 293/435] Remove anonymous function from fileio.ReadMatches example (#29186) --- sdks/go/pkg/beam/io/fileio/example_test.go | 12 +----------- 1 file changed, 1 insertion(+), 11 deletions(-) diff --git a/sdks/go/pkg/beam/io/fileio/example_test.go b/sdks/go/pkg/beam/io/fileio/example_test.go index 1763dccb0712..ddf11af56554 100644 --- a/sdks/go/pkg/beam/io/fileio/example_test.go +++ b/sdks/go/pkg/beam/io/fileio/example_test.go @@ -67,19 +67,9 @@ func ExampleReadMatches() { beam.Init() p, s := beam.NewPipelineWithRoot() - pairFn := func(ctx context.Context, file fileio.ReadableFile, emit func(string, string)) error { - contents, err := file.ReadString(ctx) - if err != nil { - return err - } - emit(file.Metadata.Path, contents) - return nil - } - matches := fileio.MatchFiles(s, "gs://path/to/*.gz") files := fileio.ReadMatches(s, matches) - pairs := beam.ParDo(s, pairFn, files) - debug.Print(s, pairs) + debug.Print(s, files) if err := beamx.Run(context.Background(), p); err != nil { log.Fatalf("Failed to execute job: %v", err) From 8a28100c47b15cf04ec76aba2e1d00fd760aeebd Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 30 Oct 2023 13:33:31 -0400 Subject: [PATCH 294/435] Bump commons-cli:commons-cli from 1.5.0 to 1.6.0 (#29192) Bumps commons-cli:commons-cli from 1.5.0 to 1.6.0. --- updated-dependencies: - dependency-name: commons-cli:commons-cli dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] <support@github.com> Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- release/build.gradle.kts | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/release/build.gradle.kts b/release/build.gradle.kts index abea131423a6..abb34d8605ad 100644 --- a/release/build.gradle.kts +++ b/release/build.gradle.kts @@ -29,8 +29,8 @@ val library = project.extensions.extraProperties["library"] as Map<String, Map<S dependencies { implementation(library.getValue("groovy").getValue("groovy_all")) - implementation("commons-cli:commons-cli:1.5.0") - permitUnusedDeclared("commons-cli:commons-cli:1.5.0") // BEAM-11761 + implementation("commons-cli:commons-cli:1.6.0") + permitUnusedDeclared("commons-cli:commons-cli:1.6.0") // BEAM-11761 } task("runJavaExamplesValidationTask") { From 551544ef779bce2acd8d076cc02e323c7bfcd005 Mon Sep 17 00:00:00 2001 From: Jeffrey Kinard <jeff@thekinards.com> Date: Mon, 30 Oct 2023 15:44:51 -0400 Subject: [PATCH 295/435] Refactor js function construction Signed-off-by: Jeffrey Kinard <jeff@thekinards.com> --- sdks/python/apache_beam/yaml/yaml_mapping.py | 55 ++++++++++--------- sdks/python/apache_beam/yaml/yaml_udf_test.py | 5 +- 2 files changed, 31 insertions(+), 29 deletions(-) diff --git a/sdks/python/apache_beam/yaml/yaml_mapping.py b/sdks/python/apache_beam/yaml/yaml_mapping.py index 7fdc7e30b4e4..7e6acc424d5b 100644 --- a/sdks/python/apache_beam/yaml/yaml_mapping.py +++ b/sdks/python/apache_beam/yaml/yaml_mapping.py @@ -79,15 +79,16 @@ def __setstate__(self, state): self.__dict__.update(state) -def row_to_dict(row): - if ((isinstance(row, tuple) and hasattr(row, '_asdict')) or - isinstance(row, beam.Row)): - row = row._asdict() - if isinstance(row, dict): - return {key: row_to_dict(value) for key, value in row.items()} - elif not isinstance(row, str) and isinstance(row, Iterable): - return [row_to_dict(value) for value in list(row)] - return row +def py_value_to_js_dict(py_value): + if ((isinstance(py_value, tuple) and hasattr(py_value, '_asdict')) or + isinstance(py_value, beam.Row)): + py_value = py_value._asdict() + if isinstance(py_value, dict): + return {key: py_value_to_js_dict(value) for key, value in py_value.items()} + elif not isinstance(py_value, str) and isinstance(py_value, Iterable): + return [py_value_to_js_dict(value) for value in list(py_value)] + else: + return py_value # TODO(yaml) Consider adding optional language version parameter to support @@ -129,23 +130,15 @@ def _js_object_to_py_object(obj): return obj - def _catch_js_errors(func): - try: - result = func() - except simplex.JsException as e: - result = getattr(e, 'mes') - return result - if expression: - args = ', '.join(original_fields) - js_func = f'function fn({args}) {{return ({expression})}}' - js_expr_callable = _CustomJsObjectWrapper(js2py.eval_js(js_func)) - fn = lambda __row__: lambda: js_expr_callable( - *row_to_dict(__row__).values()) + source = '\n'.join(['function(row) {'] + [ + f' {name} = row.{name}' + for name in original_fields if name in expression + ] + [' return (' + expression + ')'] + ['}']) + js_func = _CustomJsObjectWrapper(js2py.eval_js(source)) elif callable: - js_callable = _CustomJsObjectWrapper(js2py.eval_js(callable)) - fn = lambda __row__: lambda: js_callable(row_to_dict(__row__)) + js_func = _CustomJsObjectWrapper(js2py.eval_js(callable)) else: if not path.endswith('.js'): @@ -153,11 +146,19 @@ def _catch_js_errors(func): udf_code = FileSystems.open(path).read().decode() js = js2py.EvalJs() js.eval(udf_code) - js_file_callable = _CustomJsObjectWrapper(getattr(js, name)) - fn = lambda __row__: lambda: js_file_callable(row_to_dict(__row__)) + js_func = _CustomJsObjectWrapper(getattr(js, name)) - return lambda __row__: dicts_to_rows( - _js_object_to_py_object(_catch_js_errors(fn(__row__)))) + def js_wrapper(row): + row_as_dict = py_value_to_js_dict(row) + try: + js_result = js_func(row_as_dict) + except simplex.JsException as exn: + raise RuntimeError( + f"Error evaluating javascript expression: " + f"{exn.mes['message']}") from exn + return dicts_to_rows(_js_object_to_py_object(js_result)) + + return js_wrapper def _expand_python_mapping_func( diff --git a/sdks/python/apache_beam/yaml/yaml_udf_test.py b/sdks/python/apache_beam/yaml/yaml_udf_test.py index 4b75f3c489fc..5f5ee1147ded 100644 --- a/sdks/python/apache_beam/yaml/yaml_udf_test.py +++ b/sdks/python/apache_beam/yaml/yaml_udf_test.py @@ -25,13 +25,14 @@ from apache_beam.options import pipeline_options from apache_beam.testing.util import assert_that from apache_beam.testing.util import equal_to -from apache_beam.yaml.yaml_mapping import row_to_dict +from apache_beam.yaml.yaml_mapping import py_value_to_js_dict from apache_beam.yaml.yaml_provider import dicts_to_rows from apache_beam.yaml.yaml_transform import YamlTransform def AsRows(): - return beam.Map(lambda named_tuple: dicts_to_rows(row_to_dict(named_tuple))) + return beam.Map( + lambda named_tuple: dicts_to_rows(py_value_to_js_dict(named_tuple))) class YamlUDFMappingTest(unittest.TestCase): From 28f6e3398bf9f470d9f44cfc5991435ba0c52b08 Mon Sep 17 00:00:00 2001 From: Danny McCormick <dannymccormick@google.com> Date: Mon, 30 Oct 2023 17:03:46 -0400 Subject: [PATCH 296/435] Don't assume env vars are set in model handler (#29200) * Don't assume env vars are set in model handler * Patch notebook --- examples/notebooks/beam-ml/run_custom_inference.ipynb | 1 + sdks/python/apache_beam/ml/inference/base.py | 10 +++++----- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/examples/notebooks/beam-ml/run_custom_inference.ipynb b/examples/notebooks/beam-ml/run_custom_inference.ipynb index df81ae5af56f..a66c5847de0e 100644 --- a/examples/notebooks/beam-ml/run_custom_inference.ipynb +++ b/examples/notebooks/beam-ml/run_custom_inference.ipynb @@ -356,6 +356,7 @@ " model_name: The spaCy model name. Default is en_core_web_sm.\n", " \"\"\"\n", " self._model_name = model_name\n", + " self._env_vars = {}\n", "\n", " def load_model(self) -> Language:\n", " \"\"\"Loads and initializes a model for processing.\"\"\"\n", diff --git a/sdks/python/apache_beam/ml/inference/base.py b/sdks/python/apache_beam/ml/inference/base.py index 45c5078c13cf..fc8ac59a1fb7 100644 --- a/sdks/python/apache_beam/ml/inference/base.py +++ b/sdks/python/apache_beam/ml/inference/base.py @@ -512,7 +512,7 @@ def __init__( 'postprocessing functions defined into a keyed model handler. All ' 'pre/postprocessing functions must be defined on the outer model' 'handler.') - self._env_vars = unkeyed._env_vars + self._env_vars = getattr(unkeyed, '_env_vars', {}) self._unkeyed = unkeyed return @@ -553,7 +553,7 @@ def __init__( 'overriding the KeyedModelHandler.batch_elements_kwargs() method.', hints, batch_kwargs) - env_vars = mh._env_vars + env_vars = getattr(mh, '_env_vars', {}) if len(env_vars) > 0: logging.warning( 'mh %s defines the following _env_vars which will be ignored %s. ' @@ -816,7 +816,7 @@ def __init__(self, unkeyed: ModelHandler[ExampleT, PredictionT, ModelT]): 'pre/postprocessing functions must be defined on the outer model' 'handler.') self._unkeyed = unkeyed - self._env_vars = unkeyed._env_vars + self._env_vars = getattr(unkeyed, '_env_vars', {}) def load_model(self) -> ModelT: return self._unkeyed.load_model() @@ -895,7 +895,7 @@ def __init__( preprocess_fn: the preprocessing function to use. """ self._base = base - self._env_vars = base._env_vars + self._env_vars = getattr(base, '_env_vars', {}) self._preprocess_fn = preprocess_fn def load_model(self) -> ModelT: @@ -951,7 +951,7 @@ def __init__( postprocess_fn: the preprocessing function to use. """ self._base = base - self._env_vars = base._env_vars + self._env_vars = getattr(base, '_env_vars', {}) self._postprocess_fn = postprocess_fn def load_model(self) -> ModelT: From 41369a71df983bcd0584792889e202894e38b362 Mon Sep 17 00:00:00 2001 From: edman124 <ed12cheng@gmail.com> Date: Mon, 30 Oct 2023 17:01:13 -0700 Subject: [PATCH 297/435] Report Outstanding Bundles and Bytes (#29041) * New active threads metric and initial tests * create unit tests for max active threads metric * remove test filter * fix formatting with spotless apply * revert format changes * revert format change straggler * revert format change straggler * remove unnecessary comment * synchronize threads in unit tests * fix formatting with spotless apply * remove comments and rename counter * fix formatting with spotless apply * fix tests for StreamingDataflowWorker change and fixed createMockWork * Add outstanding bytes and bundle metrics + outstanding bytes test + refactor metric unit tests * report outstand bundles and byte metrics * Add variable for max bytes outstanding * remove duplicate test --- .../worker/DataflowSystemMetrics.java | 4 + .../worker/StreamingDataflowWorker.java | 24 +++ .../worker/util/BoundedQueueExecutor.java | 16 ++ .../worker/StreamingDataflowWorkerTest.java | 186 +++++++++++++++--- 4 files changed, 207 insertions(+), 23 deletions(-) diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowSystemMetrics.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowSystemMetrics.java index ee2a04af9982..640febc616ba 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowSystemMetrics.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowSystemMetrics.java @@ -42,6 +42,10 @@ public enum StreamingSystemCounterNames { TIME_AT_MAX_ACTIVE_THREADS("dataflow_time_at_max_active_threads"), ACTIVE_THREADS("dataflow_active_threads"), TOTAL_ALLOCATED_THREADS("dataflow_total_allocated_threads"), + OUTSTANDING_BYTES("dataflow_outstanding_bytes"), + MAX_OUTSTANDING_BYTES("dataflow_max_outstanding_bytes"), + OUTSTANDING_BUNDLES("dataflow_outstanding_bundles"), + MAX_OUTSTANDING_BUNDLES("dataflow_max_outstanding_bundles"), WINDMILL_QUOTA_THROTTLING("dataflow_streaming_engine_throttled_msecs"), MEMORY_THRASHING("dataflow_streaming_engine_user_worker_thrashing"); diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java index 77f5205cf7e9..811250ee785c 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java @@ -251,6 +251,10 @@ public class StreamingDataflowWorker { private final Counter<Long, Long> timeAtMaxActiveThreads; private final Counter<Integer, Integer> activeThreads; private final Counter<Integer, Integer> totalAllocatedThreads; + private final Counter<Long, Long> outstandingBytes; + private final Counter<Long, Long> maxOutstandingBytes; + private final Counter<Long, Long> outstandingBundles; + private final Counter<Long, Long> maxOutstandingBundles; private final Counter<Integer, Integer> windmillMaxObservedWorkItemCommitBytes; private final Counter<Integer, Integer> memoryThrashing; private final boolean publishCounters; @@ -337,6 +341,18 @@ public class StreamingDataflowWorker { StreamingSystemCounterNames.TIME_AT_MAX_ACTIVE_THREADS.counterName()); this.activeThreads = pendingCumulativeCounters.intSum(StreamingSystemCounterNames.ACTIVE_THREADS.counterName()); + this.outstandingBytes = + pendingCumulativeCounters.longSum( + StreamingSystemCounterNames.OUTSTANDING_BYTES.counterName()); + this.maxOutstandingBytes = + pendingCumulativeCounters.longSum( + StreamingSystemCounterNames.MAX_OUTSTANDING_BYTES.counterName()); + this.outstandingBundles = + pendingCumulativeCounters.longSum( + StreamingSystemCounterNames.OUTSTANDING_BUNDLES.counterName()); + this.maxOutstandingBundles = + pendingCumulativeCounters.longSum( + StreamingSystemCounterNames.MAX_OUTSTANDING_BUNDLES.counterName()); this.totalAllocatedThreads = pendingCumulativeCounters.intSum( StreamingSystemCounterNames.TOTAL_ALLOCATED_THREADS.counterName()); @@ -1721,6 +1737,14 @@ private void updateThreadMetrics() { activeThreads.addValue(workUnitExecutor.activeCount()); totalAllocatedThreads.getAndReset(); totalAllocatedThreads.addValue(chooseMaximumNumberOfThreads()); + outstandingBytes.getAndReset(); + outstandingBytes.addValue(workUnitExecutor.bytesOutstanding()); + maxOutstandingBytes.getAndReset(); + maxOutstandingBytes.addValue(workUnitExecutor.maximumBytesOutstanding()); + outstandingBundles.getAndReset(); + outstandingBundles.addValue(workUnitExecutor.elementsOutstanding()); + maxOutstandingBundles.getAndReset(); + maxOutstandingBundles.addValue(workUnitExecutor.maximumElementsOutstanding()); } @VisibleForTesting diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BoundedQueueExecutor.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BoundedQueueExecutor.java index a160b0e6ad03..dcff1f73f10f 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BoundedQueueExecutor.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BoundedQueueExecutor.java @@ -123,6 +123,22 @@ public int activeCount() { return activeCount.intValue(); } + public long bytesOutstanding() { + return bytesOutstanding; + } + + public long elementsOutstanding() { + return elementsOutstanding; + } + + public long maximumBytesOutstanding() { + return maximumBytesOutstanding; + } + + public long maximumElementsOutstanding() { + return maximumElementsOutstanding; + } + public String summaryHtml() { monitor.enter(); try { diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java index fdec36d688e9..6826607513d9 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java @@ -80,6 +80,7 @@ import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.function.Consumer; @@ -226,6 +227,7 @@ public class StreamingDataflowWorkerTest { private static final ByteString DEFAULT_KEY_BYTES = ByteString.copyFromUtf8(DEFAULT_KEY_STRING); private static final String DEFAULT_DATA_STRING = "data"; private static final String DEFAULT_DESTINATION_STREAM_ID = "out"; + private static final long MAXIMUM_BYTES_OUTSTANDING = 10000000; private static final Function<GetDataRequest, GetDataResponse> EMPTY_DATA_RESPONDER = (GetDataRequest request) -> { GetDataResponse.Builder builder = GetDataResponse.newBuilder(); @@ -2747,7 +2749,7 @@ public void testMaxThreadMetric() throws Exception { threadExpiration, TimeUnit.SECONDS, maxThreads, - 10000000, + MAXIMUM_BYTES_OUTSTANDING, new ThreadFactoryBuilder() .setNameFormat("DataflowWorkUnits-%d") .setDaemon(true) @@ -2791,12 +2793,14 @@ public void testMaxThreadMetric() throws Exception { executor.shutdown(); } - volatile boolean stop = false; - @Test public void testActiveThreadMetric() throws Exception { int maxThreads = 5; int threadExpirationSec = 60; + CountDownLatch processStart1 = new CountDownLatch(2); + CountDownLatch processStart2 = new CountDownLatch(3); + CountDownLatch processStart3 = new CountDownLatch(4); + AtomicBoolean stop = new AtomicBoolean(false); // setting up actual implementation of executor instead of mocking to keep track of // active thread count. BoundedQueueExecutor executor = @@ -2805,7 +2809,7 @@ public void testActiveThreadMetric() throws Exception { threadExpirationSec, TimeUnit.SECONDS, maxThreads, - 10000000, + MAXIMUM_BYTES_OUTSTANDING, new ThreadFactoryBuilder() .setNameFormat("DataflowWorkUnits-%d") .setDaemon(true) @@ -2823,11 +2827,11 @@ public void testActiveThreadMetric() throws Exception { Consumer<Work> sleepProcessWorkFn = unused -> { - synchronized (this) { - this.notify(); - } + processStart1.countDown(); + processStart2.countDown(); + processStart3.countDown(); int count = 0; - while (!stop) { + while (!stop.get()) { count += 1; } }; @@ -2840,27 +2844,163 @@ public void testActiveThreadMetric() throws Exception { assertEquals(0, executor.activeCount()); assertTrue(computationState.activateWork(key1Shard1, m2)); - synchronized (this) { - executor.execute(m2, m2.getWorkItem().getSerializedSize()); - this.wait(); - // Seems current executor executes the initial work item twice - this.wait(); - } + // activate work starts executing work if no other work is queued for that shard + executor.execute(m2, m2.getWorkItem().getSerializedSize()); + processStart1.await(); assertEquals(2, executor.activeCount()); assertTrue(computationState.activateWork(key1Shard1, m3)); assertTrue(computationState.activateWork(key1Shard1, m4)); - synchronized (this) { - executor.execute(m3, m3.getWorkItem().getSerializedSize()); - this.wait(); - } + executor.execute(m3, m3.getWorkItem().getSerializedSize()); + processStart2.await(); + assertEquals(3, executor.activeCount()); - synchronized (this) { - executor.execute(m4, m4.getWorkItem().getSerializedSize()); - this.wait(); - } + executor.execute(m4, m4.getWorkItem().getSerializedSize()); + processStart3.await(); assertEquals(4, executor.activeCount()); - stop = true; + stop.set(true); + executor.shutdown(); + } + + @Test + public void testOutstandingBytesMetric() throws Exception { + int maxThreads = 5; + int threadExpirationSec = 60; + CountDownLatch processStart1 = new CountDownLatch(2); + CountDownLatch processStart2 = new CountDownLatch(3); + CountDownLatch processStart3 = new CountDownLatch(4); + AtomicBoolean stop = new AtomicBoolean(false); + // setting up actual implementation of executor instead of mocking to keep track of + // active thread count. + BoundedQueueExecutor executor = + new BoundedQueueExecutor( + maxThreads, + threadExpirationSec, + TimeUnit.SECONDS, + maxThreads, + MAXIMUM_BYTES_OUTSTANDING, + new ThreadFactoryBuilder() + .setNameFormat("DataflowWorkUnits-%d") + .setDaemon(true) + .build()); + + ComputationState computationState = + new ComputationState( + "computation", + defaultMapTask(Arrays.asList(makeSourceInstruction(StringUtf8Coder.of()))), + executor, + ImmutableMap.of(), + null); + + ShardedKey key1Shard1 = ShardedKey.create(ByteString.copyFromUtf8("key1"), 1); + Consumer<Work> sleepProcessWorkFn = + unused -> { + processStart1.countDown(); + processStart2.countDown(); + processStart3.countDown(); + int count = 0; + while (!stop.get()) { + count += 1; + } + }; + + Work m2 = createMockWork(2, sleepProcessWorkFn); + + Work m3 = createMockWork(3, sleepProcessWorkFn); + + Work m4 = createMockWork(4, sleepProcessWorkFn); + assertEquals(0, executor.bytesOutstanding()); + + long bytes = m2.getWorkItem().getSerializedSize(); + assertTrue(computationState.activateWork(key1Shard1, m2)); + // activate work starts executing work if no other work is queued for that shard + bytes += m2.getWorkItem().getSerializedSize(); + executor.execute(m2, m2.getWorkItem().getSerializedSize()); + processStart1.await(); + assertEquals(bytes, executor.bytesOutstanding()); + + assertTrue(computationState.activateWork(key1Shard1, m3)); + assertTrue(computationState.activateWork(key1Shard1, m4)); + + bytes += m3.getWorkItem().getSerializedSize(); + executor.execute(m3, m3.getWorkItem().getSerializedSize()); + processStart2.await(); + assertEquals(bytes, executor.bytesOutstanding()); + + bytes += m4.getWorkItem().getSerializedSize(); + executor.execute(m4, m4.getWorkItem().getSerializedSize()); + processStart3.await(); + assertEquals(bytes, executor.bytesOutstanding()); + stop.set(true); + executor.shutdown(); + } + + @Test + public void testOutstandingBundlesMetric() throws Exception { + int maxThreads = 5; + int threadExpirationSec = 60; + CountDownLatch processStart1 = new CountDownLatch(2); + CountDownLatch processStart2 = new CountDownLatch(3); + CountDownLatch processStart3 = new CountDownLatch(4); + AtomicBoolean stop = new AtomicBoolean(false); + // setting up actual implementation of executor instead of mocking to keep track of + // active thread count. + BoundedQueueExecutor executor = + new BoundedQueueExecutor( + maxThreads, + threadExpirationSec, + TimeUnit.SECONDS, + maxThreads, + MAXIMUM_BYTES_OUTSTANDING, + new ThreadFactoryBuilder() + .setNameFormat("DataflowWorkUnits-%d") + .setDaemon(true) + .build()); + + ComputationState computationState = + new ComputationState( + "computation", + defaultMapTask(Arrays.asList(makeSourceInstruction(StringUtf8Coder.of()))), + executor, + ImmutableMap.of(), + null); + + ShardedKey key1Shard1 = ShardedKey.create(ByteString.copyFromUtf8("key1"), 1); + Consumer<Work> sleepProcessWorkFn = + unused -> { + processStart1.countDown(); + processStart2.countDown(); + processStart3.countDown(); + int count = 0; + while (!stop.get()) { + count += 1; + } + }; + + Work m2 = createMockWork(2, sleepProcessWorkFn); + + Work m3 = createMockWork(3, sleepProcessWorkFn); + + Work m4 = createMockWork(4, sleepProcessWorkFn); + assertEquals(0, executor.elementsOutstanding()); + + assertTrue(computationState.activateWork(key1Shard1, m2)); + // activate work starts executing work if no other work is queued for that shard + executor.execute(m2, m2.getWorkItem().getSerializedSize()); + processStart1.await(); + assertEquals(2, executor.elementsOutstanding()); + + assertTrue(computationState.activateWork(key1Shard1, m3)); + assertTrue(computationState.activateWork(key1Shard1, m4)); + + executor.execute(m3, m3.getWorkItem().getSerializedSize()); + processStart2.await(); + assertEquals(3, executor.elementsOutstanding()); + + executor.execute(m4, m4.getWorkItem().getSerializedSize()); + processStart3.await(); + assertEquals(4, executor.elementsOutstanding()); + stop.set(true); executor.shutdown(); } From d5fc02479c25cec4b98322495c3cb4fd866cc746 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw <robertwb@gmail.com> Date: Mon, 30 Oct 2023 17:18:47 -0700 Subject: [PATCH 298/435] [YAML] Add a basic aggregating transform to Beam Yaml. (#29167) This follows the spirit of the schema-aware GroupBy operations in Java, Python, Typescript, etc. Syntactic sugar is provided for simplified forms. This is currently guarded as experimental until we've fully vetted the API. --- sdks/python/apache_beam/yaml/readme_test.py | 23 +- sdks/python/apache_beam/yaml/yaml_combine.md | 166 ++++++++++++++ sdks/python/apache_beam/yaml/yaml_combine.py | 205 ++++++++++++++++++ .../apache_beam/yaml/yaml_combine_test.py | 173 +++++++++++++++ sdks/python/apache_beam/yaml/yaml_mapping.py | 67 +----- sdks/python/apache_beam/yaml/yaml_provider.py | 41 ++++ .../python/apache_beam/yaml/yaml_transform.py | 12 +- 7 files changed, 619 insertions(+), 68 deletions(-) create mode 100644 sdks/python/apache_beam/yaml/yaml_combine.md create mode 100644 sdks/python/apache_beam/yaml/yaml_combine.py create mode 100644 sdks/python/apache_beam/yaml/yaml_combine_test.py diff --git a/sdks/python/apache_beam/yaml/readme_test.py b/sdks/python/apache_beam/yaml/readme_test.py index d918d18e11dd..7f2d193bf35f 100644 --- a/sdks/python/apache_beam/yaml/readme_test.py +++ b/sdks/python/apache_beam/yaml/readme_test.py @@ -26,13 +26,13 @@ import tempfile import unittest +import mock import yaml from yaml.loader import SafeLoader import apache_beam as beam from apache_beam.options.pipeline_options import PipelineOptions from apache_beam.typehints import trivial_inference -from apache_beam.yaml import yaml_mapping from apache_beam.yaml import yaml_provider from apache_beam.yaml import yaml_transform @@ -200,7 +200,10 @@ def test(self): if write in test_yaml: spec = replace_recursive(spec, write, 'path', env.output_file()) modified_yaml = yaml.dump(spec) - options = {'pickle_library': 'cloudpickle'} + options = { + 'pickle_library': 'cloudpickle', + 'yaml_experimental_features': ['Combine'] + } if RENDER_DIR is not None: options['runner'] = 'apache_beam.runners.render.RenderRunner' options['render_output'] = [ @@ -208,13 +211,12 @@ def test(self): ] options['render_leaf_composite_nodes'] = ['.*'] test_provider = TestProvider(TEST_TRANSFORMS) - test_sql_mapping_provider = yaml_mapping.SqlMappingProvider(test_provider) - p = beam.Pipeline(options=PipelineOptions(**options)) - yaml_transform.expand_pipeline( - p, - modified_yaml, - yaml_provider.merge_providers( - [test_provider, test_sql_mapping_provider])) + with mock.patch( + 'apache_beam.yaml.yaml_provider.SqlBackedProvider.sql_provider', + lambda self: test_provider): + p = beam.Pipeline(options=PipelineOptions(**options)) + yaml_transform.expand_pipeline( + p, modified_yaml, yaml_provider.merge_providers([test_provider])) if test_type == 'BUILD': return p.run().wait_until_finish() @@ -270,6 +272,9 @@ def createTestSuite(name, path): 'ErrorHandlingTest', os.path.join(os.path.dirname(__file__), 'yaml_errors.md')) +CombineTest = createTestSuite( + 'CombineTest', os.path.join(os.path.dirname(__file__), 'yaml_combine.md')) + if __name__ == '__main__': parser = argparse.ArgumentParser() parser.add_argument('--render_dir', default=None) diff --git a/sdks/python/apache_beam/yaml/yaml_combine.md b/sdks/python/apache_beam/yaml/yaml_combine.md new file mode 100644 index 000000000000..e2fef304fb0a --- /dev/null +++ b/sdks/python/apache_beam/yaml/yaml_combine.md @@ -0,0 +1,166 @@ +<!-- + Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. +--> + +# Beam YAML Aggregations + +Beam YAML has EXPERIMENTAL ability to do aggregations to group and combine +values across records. The is accomplished via the `Combine` transform type. +Currently `Combine` needs to be in the `yaml_experimental_features` +option to use this transform. + +For example, one can write + +``` +- type: Combine + config: + group_by: col1 + combine: + total: + value: col2 + fn: + type: sum +``` + +If the function has no configuration requirements, it can be provided directly +as a string + +``` +- type: Combine + config: + group_by: col1 + combine: + total: + value: col2 + fn: sum +``` + +This can be simplified further if the output field name is the same as the input +field name + +``` +- type: Combine + config: + group_by: col1 + combine: + col2: sum +``` + +One can aggregate over may fields at once + +``` +- type: Combine + config: + group_by: col1 + combine: + col2: sum + col3: max +``` + +and/or group by more than one field + +``` +- type: Combine + config: + group_by: [col1, col2] + combine: + col3: sum +``` + +or none at all (which will result in a global combine with a single output) + +``` +- type: Combine + config: + group_by: [] + combine: + col2: sum + col3: max +``` + +## Windowed aggregation + +As with all transforms, `Combine` can take a windowing parameter + +``` +- type: Combine + windowing: + type: fixed + size: 60 + config: + group_by: col1 + combine: + col2: sum + col3: max +``` + +If no windowing specification is provided, it inherits the windowing +parameters from upstream, e.g. + +``` +- type: WindowInto + windowing: + type: fixed + size: 60 +- type: Combine + config: + group_by: col1 + combine: + col2: sum + col3: max +``` + +is equivalent to the previous example. + + +## Custom aggregation functions + +One can use aggregation functions defined in Python by setting the language +parameter. + +``` +- type: Combine + config: + language: python + group_by: col1 + combine: + biggest: + value: "col2 + col2" + fn: + type: 'apache_beam.transforms.combiners.TopCombineFn' + config: + n: 10 +``` + +## SQL-style aggregations + +By setting the language to SQL, one can provide full SQL snippets as the +combine fn. + +``` +- type: Combine + config: + language: sql + group_by: col1 + combine: + num_values: "count(*)" + total: "sum(col2)" +``` + +One can of course also use the `Sql` transform type and provide a query +directly. diff --git a/sdks/python/apache_beam/yaml/yaml_combine.py b/sdks/python/apache_beam/yaml/yaml_combine.py new file mode 100644 index 000000000000..ef4974cff351 --- /dev/null +++ b/sdks/python/apache_beam/yaml/yaml_combine.py @@ -0,0 +1,205 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +"""This module defines the basic Combine operation.""" + +from typing import Any +from typing import Iterable +from typing import Mapping +from typing import Optional + +import apache_beam as beam +from apache_beam import typehints +from apache_beam.typehints import row_type +from apache_beam.typehints import trivial_inference +from apache_beam.typehints.decorators import get_type_hints +from apache_beam.typehints.schemas import named_fields_from_element_type +from apache_beam.utils import python_callable +from apache_beam.yaml import options +from apache_beam.yaml import yaml_mapping +from apache_beam.yaml import yaml_provider + +BUILTIN_COMBINE_FNS = { + 'sum': sum, + 'max': max, + 'min': min, + 'all': all, + 'any': any, + 'mean': beam.transforms.combiners.MeanCombineFn(), + 'count': beam.transforms.combiners.CountCombineFn(), +} + + +def normalize_combine(spec): + """Expands various shorthand specs for combine (which can otherwise be quite + verbose for simple cases.) We do this here so that it doesn't need to be done + per language. The following are all equivalent:: + + dest: fn_type + + dest: + value: dest + fn: fn_type + + dest: + value: dest + fn: + type: fn_type + """ + from apache_beam.yaml.yaml_transform import SafeLineLoader + if spec['type'] == 'Combine': + config = spec.get('config') + if isinstance(config.get('group_by'), str): + config['group_by'] = [config['group_by']] + + def normalize_agg(dest, agg): + if isinstance(agg, str): + agg = {'fn': agg} + if 'value' not in agg and spec.get('language') != 'sql': + agg['value'] = dest + if isinstance(agg['fn'], str): + agg['fn'] = {'type': agg['fn']} + return agg + + if 'combine' not in config: + raise ValueError('Missing combine parameter in Combine config.') + config['combine'] = { + dest: normalize_agg(dest, agg) + for (dest, + agg) in SafeLineLoader.strip_metadata(config['combine']).items() + } + return spec + + +class PyJsYamlCombine(beam.PTransform): + def __init__( + self, + group_by: Iterable[str], + combine: Mapping[str, Mapping[str, Any]], + language: Optional[str] = None): + self._group_by = group_by + self._combine = combine + self._language = language + + def expand(self, pcoll): + options.YamlOptions.check_enabled(pcoll.pipeline, 'Combine') + input_types = dict(named_fields_from_element_type(pcoll.element_type)) + all_fields = list(input_types.keys()) + unknown_keys = set(self._group_by) - set(all_fields) + if unknown_keys: + raise ValueError(f'Unknown grouping columns: {list(unknown_keys)}') + + def create_combine_fn(fn_spec): + if 'type' not in fn_spec: + raise ValueError(f'CombineFn spec missing type: {fn_spec}') + elif fn_spec['type'] in BUILTIN_COMBINE_FNS: + return BUILTIN_COMBINE_FNS[fn_spec['type']] + elif self._language == 'python': + # TODO(yaml): Support output_type here as well. + fn = python_callable.PythonCallableWithSource.load_from_source( + fn_spec['type']) + if 'config' in fn_spec: + fn = fn(**fn_spec['config']) + return fn + else: + raise TypeError('Unknown CombineFn: {fn_spec}') + + def extract_return_type(expr): + if isinstance(expr, str) and expr in input_types: + return input_types[expr] + expr_hints = get_type_hints(expr) + if (expr_hints and expr_hints.has_simple_output_type() and + expr_hints.simple_output_type(None) != typehints.Any): + return expr_hints.simple_output_type(None) + elif callable(expr): + return trivial_inference.infer_return_type(expr, [pcoll.element_type]) + else: + return Any + + # TODO(yaml): Support error handling. + transform = beam.GroupBy(*self._group_by) + output_types = [(k, input_types[k]) for k in self._group_by] + + for output, agg in self._combine.items(): + expr = yaml_mapping._as_callable( + all_fields, agg['value'], 'Combine', self._language) + fn = create_combine_fn(agg['fn']) + transform = transform.aggregate_field(expr, fn, output) + + # TODO(yaml): See if this logic can be pushed into GroupBy itself. + expr_type = extract_return_type(expr) + print('expr', expr, 'expr_type', expr_type) + if isinstance(fn, beam.CombineFn): + # TODO(yaml): Better inference on CombineFns whose outputs types are + # functions of their input types + combined_type = extract_return_type(fn) + elif fn in (sum, min, max): + combined_type = expr_type + elif fn in (any, all): + combined_type = bool + else: + combined_type = Any + output_types.append((output, combined_type)) + + return pcoll | transform.with_output_types( + row_type.RowTypeConstraint.from_fields(output_types)) + + +@beam.ptransform.ptransform_fn +def _SqlCombineTransform( + pcoll, sql_transform_constructor, group_by, combine, language=None): + options.YamlOptions.check_enabled(pcoll.pipeline, 'Combine') + all_fields = [ + x for x, _ in named_fields_from_element_type(pcoll.element_type) + ] + unknown_keys = set(group_by) - set(all_fields) + if unknown_keys: + raise ValueError(f'Unknown grouping columns: {list(unknown_keys)}') + + def combine_col(dest, fn_spec): + if 'value' in fn_spec or 'config' in fn_spec['fn']: + expr = '%s(%s)' % ( + fn_spec['fn']['type'], + ', '.join([fn_spec['value']] + + list(fn_spec['fn'].get('config', {}).values()))) + else: + expr = fn_spec['fn']['type'] + return f'{expr} as {dest}' + + return pcoll | sql_transform_constructor( + 'SELECT %s FROM PCOLLECTION GROUP BY %s' % ( + ', '.join( + list(group_by) + + [combine_col(dest, fn_spec) + for dest, fn_spec in combine.items()]), + ', '.join(group_by), + )) + + +def create_combine_providers(): + return [ + yaml_provider.InlineProvider({ + 'Combine-generic': PyJsYamlCombine, + 'Combine-python': PyJsYamlCombine, + 'Combine-javascript': PyJsYamlCombine, + }), + yaml_provider.SqlBackedProvider({ + 'Combine-generic': _SqlCombineTransform, + 'Combine-sql': _SqlCombineTransform, + 'Combine-calcite': _SqlCombineTransform, + }), + ] diff --git a/sdks/python/apache_beam/yaml/yaml_combine_test.py b/sdks/python/apache_beam/yaml/yaml_combine_test.py new file mode 100644 index 000000000000..ef696c89379f --- /dev/null +++ b/sdks/python/apache_beam/yaml/yaml_combine_test.py @@ -0,0 +1,173 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import logging +import unittest + +import apache_beam as beam +from apache_beam.testing.util import assert_that +from apache_beam.testing.util import equal_to +from apache_beam.yaml.yaml_transform import YamlTransform + +DATA = [ + beam.Row(a='x', b=1, c=101), + beam.Row(a='x', b=1, c=102), + beam.Row(a='y', b=1, c=103), + beam.Row(a='y', b=2, c=104), +] + + +class YamlCombineTest(unittest.TestCase): + def test_multiple_aggregations(self): + with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions( + pickle_library='cloudpickle', yaml_experimental_features=['Combine' + ])) as p: + elements = p | beam.Create(DATA) + result = elements | YamlTransform( + ''' + type: Combine + config: + group_by: a + combine: + b: sum + c: max + ''') + assert_that( + result | beam.Map(lambda x: beam.Row(**x._asdict())), + equal_to([ + beam.Row(a='x', b=2, c=102), + beam.Row(a='y', b=3, c=104), + ])) + + def test_multiple_keys(self): + with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions( + pickle_library='cloudpickle', yaml_experimental_features=['Combine' + ])) as p: + elements = p | beam.Create(DATA) + result = elements | YamlTransform( + ''' + type: Combine + config: + group_by: [a, b] + combine: + c: sum + ''') + assert_that( + result | beam.Map(lambda x: beam.Row(**x._asdict())), + equal_to([ + beam.Row(a='x', b=1, c=203), + beam.Row(a='y', b=1, c=103), + beam.Row(a='y', b=2, c=104), + ])) + + def test_no_keys(self): + with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions( + pickle_library='cloudpickle', yaml_experimental_features=['Combine' + ])) as p: + elements = p | beam.Create(DATA) + result = elements | YamlTransform( + ''' + type: Combine + config: + group_by: [] + combine: + c: sum + ''') + assert_that( + result | beam.Map(lambda x: beam.Row(**x._asdict())), + equal_to([ + beam.Row(c=410), + ])) + + def test_multiple_combines(self): + with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions( + pickle_library='cloudpickle', yaml_experimental_features=['Combine' + ])) as p: + elements = p | beam.Create(DATA) + result = elements | YamlTransform( + ''' + type: Combine + config: + group_by: a + combine: + min_c: + fn: min + value: c + max_c: + fn: max + value: c + ''') + assert_that( + result | beam.Map(lambda x: beam.Row(**x._asdict())), + equal_to([ + beam.Row(a='x', min_c=101, max_c=102), + beam.Row(a='y', min_c=103, max_c=104), + ])) + + def test_expression(self): + with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions( + pickle_library='cloudpickle', yaml_experimental_features=['Combine' + ])) as p: + elements = p | beam.Create(DATA) + result = elements | YamlTransform( + ''' + type: Combine + config: + language: python + group_by: a + combine: + max: + fn: max + value: b + c + ''') + assert_that( + result | beam.Map(lambda x: beam.Row(**x._asdict())), + equal_to([ + beam.Row(a='x', max=103), + beam.Row(a='y', max=106), + ])) + + def test_config(self): + with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions( + pickle_library='cloudpickle', yaml_experimental_features=['Combine' + ])) as p: + elements = p | beam.Create(DATA) + result = elements | YamlTransform( + ''' + type: Combine + config: + language: python + group_by: b + combine: + biggest: + fn: + type: 'apache_beam.transforms.combiners.TopCombineFn' + config: + n: 2 + value: c + ''') + assert_that( + result | beam.Map(lambda x: beam.Row(**x._asdict())), + equal_to([ + beam.Row(b=1, biggest=[103, 102]), + beam.Row(b=2, biggest=[104]), + ])) + + +if __name__ == '__main__': + logging.getLogger().setLevel(logging.INFO) + unittest.main() diff --git a/sdks/python/apache_beam/yaml/yaml_mapping.py b/sdks/python/apache_beam/yaml/yaml_mapping.py index e217ab285844..501c7a5c57be 100644 --- a/sdks/python/apache_beam/yaml/yaml_mapping.py +++ b/sdks/python/apache_beam/yaml/yaml_mapping.py @@ -21,7 +21,6 @@ from typing import Callable from typing import Collection from typing import Dict -from typing import Iterable from typing import Mapping from typing import Optional from typing import Union @@ -399,62 +398,14 @@ def _PyJsMapToFields(pcoll, language='generic', **mapping_args): }) -class SqlMappingProvider(yaml_provider.Provider): - def __init__(self, sql_provider=None): - if sql_provider is None: - sql_provider = yaml_provider.beam_jar( - urns={'Sql': 'beam:external:java:sql:v1'}, - gradle_target='sdks:java:extensions:sql:expansion-service:shadowJar') - self._sql_provider = sql_provider - - def available(self): - return self._sql_provider.available() - - def cache_artifacts(self): - return self._sql_provider.cache_artifacts() - - def provided_transforms(self) -> Iterable[str]: - return [ - 'Filter-sql', - 'Filter-calcite', - 'MapToFields-sql', - 'MapToFields-calcite' - ] - - def create_transform( - self, - typ: str, - args: Mapping[str, Any], - yaml_create_transform: Callable[ - [Mapping[str, Any], Iterable[beam.PCollection]], beam.PTransform] - ) -> beam.PTransform: - if typ.startswith('Filter-'): - return _SqlFilterTransform( - self._sql_provider, yaml_create_transform, **args) - if typ.startswith('MapToFields-'): - return _SqlMapToFieldsTransform( - self._sql_provider, yaml_create_transform, **args) - else: - raise NotImplementedError(typ) - - def underlying_provider(self): - return self._sql_provider - - def to_json(self): - return {'type': "SqlMappingProvider"} - - @beam.ptransform.ptransform_fn -def _SqlFilterTransform( - pcoll, sql_provider, yaml_create_transform, keep, language): - return pcoll | sql_provider.create_transform( - 'Sql', {'query': f'SELECT * FROM PCOLLECTION WHERE {keep}'}, - yaml_create_transform) +def _SqlFilterTransform(pcoll, sql_transform_constructor, keep, language): + return pcoll | sql_transform_constructor( + f'SELECT * FROM PCOLLECTION WHERE {keep}') @beam.ptransform.ptransform_fn -def _SqlMapToFieldsTransform( - pcoll, sql_provider, yaml_create_transform, **mapping_args): +def _SqlMapToFieldsTransform(pcoll, sql_transform_constructor, **mapping_args): _, fields = normalize_fields(pcoll, **mapping_args) def extract_expr(name, v): @@ -470,8 +421,7 @@ def extract_expr(name, v): for (name, expr) in fields.items() ] query = "SELECT " + ", ".join(selects) + " FROM PCOLLECTION" - return pcoll | sql_provider.create_transform( - 'Sql', {'query': query}, yaml_create_transform) + return pcoll | sql_transform_constructor(query) def create_mapping_providers(): @@ -487,5 +437,10 @@ def create_mapping_providers(): 'MapToFields-javascript': _PyJsMapToFields, 'MapToFields-generic': _PyJsMapToFields, }), - SqlMappingProvider(), + yaml_provider.SqlBackedProvider({ + 'Filter-sql': _SqlFilterTransform, + 'Filter-calcite': _SqlFilterTransform, + 'MapToFields-sql': _SqlMapToFieldsTransform, + 'MapToFields-calcite': _SqlMapToFieldsTransform, + }), ] diff --git a/sdks/python/apache_beam/yaml/yaml_provider.py b/sdks/python/apache_beam/yaml/yaml_provider.py index 33c16380ece3..a21db09b50ca 100644 --- a/sdks/python/apache_beam/yaml/yaml_provider.py +++ b/sdks/python/apache_beam/yaml/yaml_provider.py @@ -453,6 +453,45 @@ def create_transform(self, type, args, yaml_create_transform): return self._transform_factories[type](yaml_create_transform, **args) +class SqlBackedProvider(Provider): + def __init__( + self, + transforms: Mapping[str, Callable[..., beam.PTransform]], + sql_provider: Optional[Provider] = None): + self._transforms = transforms + if sql_provider is None: + sql_provider = beam_jar( + urns={'Sql': 'beam:external:java:sql:v1'}, + gradle_target='sdks:java:extensions:sql:expansion-service:shadowJar') + self._sql_provider = sql_provider + + def sql_provider(self): + return self._sql_provider + + def provided_transforms(self): + return self._transforms.keys() + + def available(self): + return self.sql_provider().available() + + def cache_artifacts(self): + return self.sql_provider().cache_artifacts() + + def underlying_provider(self): + return self.sql_provider() + + def to_json(self): + return {'type': "SqlBackedProvider"} + + def create_transform( + self, typ: str, args: Mapping[str, Any], + yaml_create_transform: Any) -> beam.PTransform: + return self._transforms[typ]( + lambda query: self.sql_provider().create_transform( + 'Sql', {'query': query}, yaml_create_transform), + **args) + + PRIMITIVE_NAMES_TO_ATOMIC_TYPE = { py_type.__name__: schema_type for (py_type, schema_type) in schemas.PRIMITIVE_TO_ATOMIC_TYPE.items() @@ -781,6 +820,7 @@ def merge_providers(*provider_sets): def standard_providers(): + from apache_beam.yaml.yaml_combine import create_combine_providers from apache_beam.yaml.yaml_mapping import create_mapping_providers from apache_beam.yaml.yaml_io import io_providers with open(os.path.join(os.path.dirname(__file__), @@ -790,6 +830,7 @@ def standard_providers(): return merge_providers( create_builtin_provider(), create_mapping_providers(), + create_combine_providers(), io_providers(), parse_providers(standard_providers)) diff --git a/sdks/python/apache_beam/yaml/yaml_transform.py b/sdks/python/apache_beam/yaml/yaml_transform.py index 7ab8da33f1a5..ca63834e283f 100644 --- a/sdks/python/apache_beam/yaml/yaml_transform.py +++ b/sdks/python/apache_beam/yaml/yaml_transform.py @@ -34,6 +34,7 @@ import apache_beam as beam from apache_beam.transforms.fully_qualified_named_transform import FullyQualifiedNamedTransform from apache_beam.yaml import yaml_provider +from apache_beam.yaml.yaml_combine import normalize_combine __all__ = ["YamlTransform"] @@ -885,7 +886,7 @@ def ensure_transforms_have_providers(spec): return spec def preprocess_langauges(spec): - if spec['type'] in ('Filter', 'MapToFields'): + if spec['type'] in ('Filter', 'MapToFields', 'Combine'): language = spec.get('config', {}).get('language', 'generic') new_type = spec['type'] + '-' + language if known_transforms and new_type not in known_transforms: @@ -900,6 +901,7 @@ def preprocess_langauges(spec): for phase in [ ensure_transforms_have_types, + normalize_combine, preprocess_langauges, ensure_transforms_have_providers, preprocess_source_sink, @@ -951,14 +953,18 @@ def expand(self, pcolls): root = next(iter(pcolls.values())).pipeline if not self._spec['input']: self._spec['input'] = {name: name for name in pcolls.keys()} + python_provider = yaml_provider.InlineProvider({}) result = expand_transform( self._spec, Scope( root, pcolls, - transforms=[], + transforms=[self._spec], providers=self._providers, - input_providers={})) + input_providers={ + pcoll: python_provider + for pcoll in pcolls.values() + })) if len(result) == 1: return only_element(result.values()) else: From 8648e583c843d4c1af4497cb8ab2fc758622495a Mon Sep 17 00:00:00 2001 From: reuvenlax <relax@google.com> Date: Mon, 30 Oct 2023 17:39:40 -0700 Subject: [PATCH 299/435] Merge pull request #29114: Support default values in storage-api sink --- .../sdk/io/gcp/bigquery/AppendClientInfo.java | 7 +- .../beam/sdk/io/gcp/bigquery/BigQueryIO.java | 26 +- .../sdk/io/gcp/bigquery/BigQueryServices.java | 6 +- .../io/gcp/bigquery/BigQueryServicesImpl.java | 7 +- .../sdk/io/gcp/bigquery/StorageApiLoads.java | 17 +- .../StorageApiWriteRecordsInconsistent.java | 9 +- .../StorageApiWriteUnshardedRecords.java | 17 +- .../StorageApiWritesShardedRecords.java | 24 +- .../bigquery/TableRowToStorageApiProto.java | 23 +- .../io/gcp/testing/FakeDatasetService.java | 6 +- .../StorageApiSinkDefaultValuesIT.java | 317 ++++++++++++++++++ 11 files changed, 432 insertions(+), 27 deletions(-) create mode 100644 sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiSinkDefaultValuesIT.java diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/AppendClientInfo.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/AppendClientInfo.java index 46c25d47e7a8..9210f305eca7 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/AppendClientInfo.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/AppendClientInfo.java @@ -20,6 +20,7 @@ import com.google.api.services.bigquery.model.TableRow; import com.google.auto.value.AutoValue; import com.google.auto.value.extension.memoized.Memoized; +import com.google.cloud.bigquery.storage.v1.AppendRowsRequest; import com.google.cloud.bigquery.storage.v1.TableSchema; import com.google.protobuf.ByteString; import com.google.protobuf.DescriptorProtos; @@ -106,7 +107,8 @@ public AppendClientInfo withNoAppendClient() { public AppendClientInfo withAppendClient( BigQueryServices.DatasetService datasetService, Supplier<String> getStreamName, - boolean useConnectionPool) + boolean useConnectionPool, + AppendRowsRequest.MissingValueInterpretation missingValueInterpretation) throws Exception { if (getStreamAppendClient() != null) { return this; @@ -115,7 +117,8 @@ public AppendClientInfo withAppendClient( return toBuilder() .setStreamName(streamName) .setStreamAppendClient( - datasetService.getStreamAppendClient(streamName, getDescriptor(), useConnectionPool)) + datasetService.getStreamAppendClient( + streamName, getDescriptor(), useConnectionPool, missingValueInterpretation)) .build(); } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java index ce8ddb683d1e..cf62ffed3b9d 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java @@ -37,6 +37,7 @@ import com.google.api.services.bigquery.model.TableSchema; import com.google.api.services.bigquery.model.TimePartitioning; import com.google.auto.value.AutoValue; +import com.google.cloud.bigquery.storage.v1.AppendRowsRequest; import com.google.cloud.bigquery.storage.v1.CreateReadSessionRequest; import com.google.cloud.bigquery.storage.v1.DataFormat; import com.google.cloud.bigquery.storage.v1.ReadSession; @@ -2143,6 +2144,8 @@ public static <T> Write<T> write() { .setMaxRetryJobs(1000) .setPropagateSuccessfulStorageApiWrites(false) .setDirectWriteProtos(true) + .setDefaultMissingValueInterpretation( + AppendRowsRequest.MissingValueInterpretation.DEFAULT_VALUE) .build(); } @@ -2327,6 +2330,8 @@ public enum Method { abstract @Nullable List<String> getPrimaryKey(); + abstract AppendRowsRequest.MissingValueInterpretation getDefaultMissingValueInterpretation(); + abstract Boolean getOptimizeWrites(); abstract Boolean getUseBeamSchema(); @@ -2429,6 +2434,9 @@ abstract Builder<T> setPropagateSuccessfulStorageApiWrites( abstract Builder<T> setPrimaryKey(@Nullable List<String> primaryKey); + abstract Builder<T> setDefaultMissingValueInterpretation( + AppendRowsRequest.MissingValueInterpretation missingValueInterpretation); + abstract Builder<T> setOptimizeWrites(Boolean optimizeWrites); abstract Builder<T> setUseBeamSchema(Boolean useBeamSchema); @@ -2964,6 +2972,21 @@ public Write<T> withPrimaryKey(List<String> primaryKey) { return toBuilder().setPrimaryKey(primaryKey).build(); } + /** + * Specify how missing values should be interpreted when there is a default value in the schema. + * Options are to take the default value or to write an explicit null (not an option of the + * field is also required.). Note: this is only used when using one of the storage write API + * insert methods. + */ + public Write<T> withDefaultMissingValueInterpretation( + AppendRowsRequest.MissingValueInterpretation missingValueInterpretation) { + checkArgument( + missingValueInterpretation == AppendRowsRequest.MissingValueInterpretation.DEFAULT_VALUE + || missingValueInterpretation + == AppendRowsRequest.MissingValueInterpretation.NULL_VALUE); + return toBuilder().setDefaultMissingValueInterpretation(missingValueInterpretation).build(); + } + /** * If true, enables new codepaths that are expected to use less resources while writing to * BigQuery. Not enabled by default in order to maintain backwards compatibility. @@ -3685,7 +3708,8 @@ private <DestinationT> WriteResult continueExpandTyped( getAutoSchemaUpdate(), getIgnoreUnknownValues(), getPropagateSuccessfulStorageApiWrites(), - getRowMutationInformationFn() != null); + getRowMutationInformationFn() != null, + getDefaultMissingValueInterpretation()); return input.apply("StorageApiLoads", storageApiLoads); } else { throw new RuntimeException("Unexpected write method " + method); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServices.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServices.java index 1cc9049a542d..c9c96eb35f3f 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServices.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServices.java @@ -30,6 +30,7 @@ import com.google.api.services.bigquery.model.Table; import com.google.api.services.bigquery.model.TableReference; import com.google.api.services.bigquery.model.TableRow; +import com.google.cloud.bigquery.storage.v1.AppendRowsRequest; import com.google.cloud.bigquery.storage.v1.AppendRowsResponse; import com.google.cloud.bigquery.storage.v1.BatchCommitWriteStreamsResponse; import com.google.cloud.bigquery.storage.v1.CreateReadSessionRequest; @@ -213,7 +214,10 @@ WriteStream createWriteStream(String tableUrn, WriteStream.Type type) * first. */ StreamAppendClient getStreamAppendClient( - String streamName, DescriptorProtos.DescriptorProto descriptor, boolean useConnectionPool) + String streamName, + DescriptorProtos.DescriptorProto descriptor, + boolean useConnectionPool, + AppendRowsRequest.MissingValueInterpretation missingValueInterpretation) throws Exception; /** Flush a given stream up to the given offset. The stream must have type BUFFERED. */ diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java index 1b6cc555511d..b6d5eefe715e 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java @@ -61,6 +61,7 @@ import com.google.api.services.bigquery.model.TableRow; import com.google.auth.Credentials; import com.google.auth.http.HttpCredentialsAdapter; +import com.google.cloud.bigquery.storage.v1.AppendRowsRequest; import com.google.cloud.bigquery.storage.v1.AppendRowsResponse; import com.google.cloud.bigquery.storage.v1.BatchCommitWriteStreamsRequest; import com.google.cloud.bigquery.storage.v1.BatchCommitWriteStreamsResponse; @@ -1352,7 +1353,10 @@ public WriteStream createWriteStream(String tableUrn, WriteStream.Type type) @Override public StreamAppendClient getStreamAppendClient( - String streamName, DescriptorProtos.DescriptorProto descriptor, boolean useConnectionPool) + String streamName, + DescriptorProtos.DescriptorProto descriptor, + boolean useConnectionPool, + AppendRowsRequest.MissingValueInterpretation missingValueInterpretation) throws Exception { ProtoSchema protoSchema = ProtoSchema.newBuilder().setProtoDescriptor(descriptor).build(); @@ -1384,6 +1388,7 @@ public StreamAppendClient getStreamAppendClient( .setMaxInflightRequests(storageWriteMaxInflightRequests) .setMaxInflightBytes(storageWriteMaxInflightBytes) .setTraceId(traceId) + .setDefaultMissingValueInterpretation(missingValueInterpretation) .build(); return new StreamAppendClient() { private int pins = 0; diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiLoads.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiLoads.java index f9f57f71ba2c..0227b8020129 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiLoads.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiLoads.java @@ -18,6 +18,7 @@ package org.apache.beam.sdk.io.gcp.bigquery; import com.google.api.services.bigquery.model.TableRow; +import com.google.cloud.bigquery.storage.v1.AppendRowsRequest; import java.nio.ByteBuffer; import java.util.concurrent.ThreadLocalRandom; import javax.annotation.Nullable; @@ -63,9 +64,10 @@ public class StorageApiLoads<DestinationT, ElementT> private final boolean allowAutosharding; private final boolean autoUpdateSchema; private final boolean ignoreUnknownValues; - private final boolean usesCdc; + private final AppendRowsRequest.MissingValueInterpretation defaultMissingValueInterpretation; + public StorageApiLoads( Coder<DestinationT> destinationCoder, StorageApiDynamicDestinations<ElementT, DestinationT> dynamicDestinations, @@ -80,7 +82,8 @@ public StorageApiLoads( boolean autoUpdateSchema, boolean ignoreUnknownValues, boolean propagateSuccessfulStorageApiWrites, - boolean usesCdc) { + boolean usesCdc, + AppendRowsRequest.MissingValueInterpretation defaultMissingValueInterpretation) { this.destinationCoder = destinationCoder; this.dynamicDestinations = dynamicDestinations; this.rowUpdateFn = rowUpdateFn; @@ -97,6 +100,7 @@ public StorageApiLoads( this.successfulWrittenRowsTag = new TupleTag<>("successfulPublishedRowsTag"); } this.usesCdc = usesCdc; + this.defaultMissingValueInterpretation = defaultMissingValueInterpretation; } public TupleTag<BigQueryStorageApiInsertError> getFailedRowsTag() { @@ -156,7 +160,8 @@ public WriteResult expandInconsistent( ignoreUnknownValues, createDisposition, kmsKey, - usesCdc)); + usesCdc, + defaultMissingValueInterpretation)); PCollection<BigQueryStorageApiInsertError> insertErrors = PCollectionList.of(convertMessagesResult.get(failedRowsTag)) @@ -243,7 +248,8 @@ public WriteResult expandTriggered( failedRowsTag, successfulWrittenRowsTag, autoUpdateSchema, - ignoreUnknownValues)); + ignoreUnknownValues, + defaultMissingValueInterpretation)); PCollection<BigQueryStorageApiInsertError> insertErrors = PCollectionList.of(convertMessagesResult.get(failedRowsTag)) @@ -331,7 +337,8 @@ public WriteResult expandUntriggered( ignoreUnknownValues, createDisposition, kmsKey, - usesCdc)); + usesCdc, + defaultMissingValueInterpretation)); PCollection<BigQueryStorageApiInsertError> insertErrors = PCollectionList.of(convertMessagesResult.get(failedRowsTag)) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWriteRecordsInconsistent.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWriteRecordsInconsistent.java index 6a9997fffdb4..022ee1fbed08 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWriteRecordsInconsistent.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWriteRecordsInconsistent.java @@ -18,6 +18,7 @@ package org.apache.beam.sdk.io.gcp.bigquery; import com.google.api.services.bigquery.model.TableRow; +import com.google.cloud.bigquery.storage.v1.AppendRowsRequest; import javax.annotation.Nullable; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.PTransform; @@ -49,6 +50,7 @@ public class StorageApiWriteRecordsInconsistent<DestinationT, ElementT> private final BigQueryIO.Write.CreateDisposition createDisposition; private final @Nullable String kmsKey; private final boolean usesCdc; + private final AppendRowsRequest.MissingValueInterpretation defaultMissingValueInterpretation; public StorageApiWriteRecordsInconsistent( StorageApiDynamicDestinations<ElementT, DestinationT> dynamicDestinations, @@ -61,7 +63,8 @@ public StorageApiWriteRecordsInconsistent( boolean ignoreUnknownValues, BigQueryIO.Write.CreateDisposition createDisposition, @Nullable String kmsKey, - boolean usesCdc) { + boolean usesCdc, + AppendRowsRequest.MissingValueInterpretation defaultMissingValueInterpretation) { this.dynamicDestinations = dynamicDestinations; this.bqServices = bqServices; this.failedRowsTag = failedRowsTag; @@ -73,6 +76,7 @@ public StorageApiWriteRecordsInconsistent( this.createDisposition = createDisposition; this.kmsKey = kmsKey; this.usesCdc = usesCdc; + this.defaultMissingValueInterpretation = defaultMissingValueInterpretation; } @Override @@ -103,7 +107,8 @@ public PCollectionTuple expand(PCollection<KV<DestinationT, StorageApiWritePaylo ignoreUnknownValues, createDisposition, kmsKey, - usesCdc)) + usesCdc, + defaultMissingValueInterpretation)) .withOutputTags(finalizeTag, tupleTagList) .withSideInputs(dynamicDestinations.getSideInputs())); result.get(failedRowsTag).setCoder(failedRowsCoder); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWriteUnshardedRecords.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWriteUnshardedRecords.java index f54522c2ca59..df79ece207f0 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWriteUnshardedRecords.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWriteUnshardedRecords.java @@ -23,6 +23,7 @@ import com.google.api.core.ApiFuture; import com.google.api.core.ApiFutures; import com.google.api.services.bigquery.model.TableRow; +import com.google.cloud.bigquery.storage.v1.AppendRowsRequest; import com.google.cloud.bigquery.storage.v1.AppendRowsResponse; import com.google.cloud.bigquery.storage.v1.Exceptions; import com.google.cloud.bigquery.storage.v1.ProtoRows; @@ -109,6 +110,7 @@ public class StorageApiWriteUnshardedRecords<DestinationT, ElementT> private final BigQueryIO.Write.CreateDisposition createDisposition; private final @Nullable String kmsKey; private final boolean usesCdc; + private final AppendRowsRequest.MissingValueInterpretation defaultMissingValueInterpretation; /** * The Guava cache object is thread-safe. However our protocol requires that client pin the @@ -166,7 +168,8 @@ public StorageApiWriteUnshardedRecords( boolean ignoreUnknownValues, BigQueryIO.Write.CreateDisposition createDisposition, @Nullable String kmsKey, - boolean usesCdc) { + boolean usesCdc, + AppendRowsRequest.MissingValueInterpretation defaultMissingValueInterpretation) { this.dynamicDestinations = dynamicDestinations; this.bqServices = bqServices; this.failedRowsTag = failedRowsTag; @@ -178,6 +181,7 @@ public StorageApiWriteUnshardedRecords( this.createDisposition = createDisposition; this.kmsKey = kmsKey; this.usesCdc = usesCdc; + this.defaultMissingValueInterpretation = defaultMissingValueInterpretation; } @Override @@ -210,7 +214,8 @@ public PCollectionTuple expand(PCollection<KV<DestinationT, StorageApiWritePaylo ignoreUnknownValues, createDisposition, kmsKey, - usesCdc)) + usesCdc, + defaultMissingValueInterpretation)) .withOutputTags(finalizeTag, tupleTagList) .withSideInputs(dynamicDestinations.getSideInputs())); @@ -240,6 +245,7 @@ static class WriteRecordsDoFn<DestinationT extends @NonNull Object, ElementT> private final BigQueryIO.Write.CreateDisposition createDisposition; private final @Nullable String kmsKey; private final boolean usesCdc; + private final AppendRowsRequest.MissingValueInterpretation defaultMissingValueInterpretation; static class AppendRowsContext extends RetryManager.Operation.Context<AppendRowsResponse> { long offset; @@ -390,7 +396,8 @@ AppendClientInfo generateClient(@Nullable TableSchema updatedSchema) throws Exce .withAppendClient( Preconditions.checkStateNotNull(maybeDatasetService), () -> streamName, - usingMultiplexing)); + usingMultiplexing, + defaultMissingValueInterpretation)); Preconditions.checkStateNotNull(appendClientInfo.get().getStreamAppendClient()); return null; }, @@ -839,7 +846,8 @@ void postFlush() { boolean ignoreUnknownValues, BigQueryIO.Write.CreateDisposition createDisposition, @Nullable String kmsKey, - boolean usesCdc) { + boolean usesCdc, + AppendRowsRequest.MissingValueInterpretation defaultMissingValueInterpretation) { this.messageConverters = new TwoLevelMessageConverterCache<>(operationName); this.dynamicDestinations = dynamicDestinations; this.bqServices = bqServices; @@ -855,6 +863,7 @@ void postFlush() { this.createDisposition = createDisposition; this.kmsKey = kmsKey; this.usesCdc = usesCdc; + this.defaultMissingValueInterpretation = defaultMissingValueInterpretation; } boolean shouldFlush() { diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWritesShardedRecords.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWritesShardedRecords.java index efcf87eac7a3..77d9e8023a05 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWritesShardedRecords.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWritesShardedRecords.java @@ -22,6 +22,7 @@ import com.google.api.core.ApiFuture; import com.google.api.core.ApiFutures; import com.google.api.services.bigquery.model.TableRow; +import com.google.cloud.bigquery.storage.v1.AppendRowsRequest; import com.google.cloud.bigquery.storage.v1.AppendRowsResponse; import com.google.cloud.bigquery.storage.v1.Exceptions; import com.google.cloud.bigquery.storage.v1.Exceptions.StreamFinalizedException; @@ -125,6 +126,7 @@ public class StorageApiWritesShardedRecords<DestinationT extends @NonNull Object private final Coder<BigQueryStorageApiInsertError> failedRowsCoder; private final boolean autoUpdateSchema; private final boolean ignoreUnknownValues; + private final AppendRowsRequest.MissingValueInterpretation defaultMissingValueInterpretation; private final Duration streamIdleTime = DEFAULT_STREAM_IDLE_TIME; private final TupleTag<BigQueryStorageApiInsertError> failedRowsTag; @@ -217,7 +219,8 @@ public StorageApiWritesShardedRecords( TupleTag<BigQueryStorageApiInsertError> failedRowsTag, @Nullable TupleTag<TableRow> successfulRowsTag, boolean autoUpdateSchema, - boolean ignoreUnknownValues) { + boolean ignoreUnknownValues, + AppendRowsRequest.MissingValueInterpretation defaultMissingValueInterpretation) { this.dynamicDestinations = dynamicDestinations; this.createDisposition = createDisposition; this.kmsKey = kmsKey; @@ -229,6 +232,7 @@ public StorageApiWritesShardedRecords( this.succussfulRowsCoder = successfulRowsCoder; this.autoUpdateSchema = autoUpdateSchema; this.ignoreUnknownValues = ignoreUnknownValues; + this.defaultMissingValueInterpretation = defaultMissingValueInterpretation; } @Override @@ -494,7 +498,11 @@ public void process( client.unpin(); client.close(); })) - .withAppendClient(datasetService, getOrCreateStream, false); + .withAppendClient( + datasetService, + getOrCreateStream, + false, + defaultMissingValueInterpretation); // This pin is "owned" by the cache. Preconditions.checkStateNotNull(info.getStreamAppendClient()).pin(); return info; @@ -554,7 +562,11 @@ public void process( appendClientInfo.set( appendClientInfo .get() - .withAppendClient(datasetService, getOrCreateStream, false)); + .withAppendClient( + datasetService, + getOrCreateStream, + false, + defaultMissingValueInterpretation)); StreamAppendClient streamAppendClient = Preconditions.checkArgumentNotNull( appendClientInfo.get().getStreamAppendClient()); @@ -599,7 +611,11 @@ public void process( appendClientInfo.set( appendClientInfo .get() - .withAppendClient(datasetService, getOrCreateStream, false)); + .withAppendClient( + datasetService, + getOrCreateStream, + false, + defaultMissingValueInterpretation)); return Preconditions.checkStateNotNull(appendClientInfo.get().getStreamAppendClient()) .appendRows(context.offset, context.protoRows); } catch (Exception e) { diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowToStorageApiProto.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowToStorageApiProto.java index c31886da6144..4d714aaaf777 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowToStorageApiProto.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowToStorageApiProto.java @@ -213,11 +213,22 @@ private static String getPrettyFieldName(SchemaInformation schema) { .put(TableFieldSchema.Type.JSON, "JSON") .build(); - public static TableFieldSchema.Mode modeToProtoMode(String mode) { - return Optional.ofNullable(mode) - .map(Mode::valueOf) - .map(m -> MODE_MAP_JSON_PROTO.get(m)) - .orElse(TableFieldSchema.Mode.NULLABLE); + public static TableFieldSchema.Mode modeToProtoMode( + @Nullable String defaultValueExpression, String mode) { + TableFieldSchema.Mode resultMode = + Optional.ofNullable(mode) + .map(Mode::valueOf) + .map(MODE_MAP_JSON_PROTO::get) + .orElse(TableFieldSchema.Mode.NULLABLE); + if (defaultValueExpression == null) { + return resultMode; + } else { + // If there is a default value expression, treat this field as if it were nullable or + // repeated. + return resultMode.equals(TableFieldSchema.Mode.REPEATED) + ? resultMode + : TableFieldSchema.Mode.NULLABLE; + } } public static String protoModeToJsonMode(TableFieldSchema.Mode protoMode) { @@ -310,7 +321,7 @@ public static TableFieldSchema tableFieldToProtoTableField( if (field.getMaxLength() != null) { builder.setMaxLength(field.getMaxLength()); } - builder.setMode(modeToProtoMode(field.getMode())); + builder.setMode(modeToProtoMode(field.getDefaultValueExpression(), field.getMode())); if (field.getPrecision() != null) { builder.setPrecision(field.getPrecision()); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/FakeDatasetService.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/FakeDatasetService.java index 347a3513d896..f26c38d1e3c8 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/FakeDatasetService.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/FakeDatasetService.java @@ -32,6 +32,7 @@ import com.google.api.services.bigquery.model.TableReference; import com.google.api.services.bigquery.model.TableRow; import com.google.api.services.bigquery.model.TableSchema; +import com.google.cloud.bigquery.storage.v1.AppendRowsRequest; import com.google.cloud.bigquery.storage.v1.AppendRowsResponse; import com.google.cloud.bigquery.storage.v1.BatchCommitWriteStreamsResponse; import com.google.cloud.bigquery.storage.v1.Exceptions; @@ -600,7 +601,10 @@ public WriteStream getWriteStream(String streamName) { @Override public StreamAppendClient getStreamAppendClient( - String streamName, DescriptorProtos.DescriptorProto descriptor, boolean useConnectionPool) + String streamName, + DescriptorProtos.DescriptorProto descriptor, + boolean useConnectionPool, + AppendRowsRequest.MissingValueInterpretation missingValueInterpretation) throws Exception { return new StreamAppendClient() { private Descriptor protoDescriptor; diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiSinkDefaultValuesIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiSinkDefaultValuesIT.java new file mode 100644 index 000000000000..87c3659fa081 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiSinkDefaultValuesIT.java @@ -0,0 +1,317 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.gcp.bigquery; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.is; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; + +import com.google.api.services.bigquery.model.Table; +import com.google.api.services.bigquery.model.TableFieldSchema; +import com.google.api.services.bigquery.model.TableReference; +import com.google.api.services.bigquery.model.TableRow; +import com.google.api.services.bigquery.model.TableSchema; +import com.google.cloud.bigquery.storage.v1.AppendRowsRequest; +import java.io.IOException; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.function.Function; +import java.util.stream.Collectors; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; +import org.apache.beam.sdk.io.gcp.testing.BigqueryClient; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.util.Preconditions; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; +import org.hamcrest.Matchers; +import org.joda.time.Duration; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Ignore; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class StorageApiSinkDefaultValuesIT { + private static final BigqueryClient BQ_CLIENT = + new BigqueryClient("StorageApiSinkDefaultValuesIT"); + private static final String PROJECT = + TestPipeline.testingPipelineOptions().as(GcpOptions.class).getProject(); + private static final String BIG_QUERY_DATASET_ID = + "storage_api_sink_default_values" + System.nanoTime(); + + private static String bigQueryLocation; + + @BeforeClass + public static void setUpTestEnvironment() throws IOException, InterruptedException { + // Create one BQ dataset for all test cases. + bigQueryLocation = + TestPipeline.testingPipelineOptions().as(TestBigQueryOptions.class).getBigQueryLocation(); + BQ_CLIENT.createNewDataset(PROJECT, BIG_QUERY_DATASET_ID, null, bigQueryLocation); + } + + @AfterClass + public static void cleanup() { + BQ_CLIENT.deleteDataset(PROJECT, BIG_QUERY_DATASET_ID); + } + + private static String createAndGetTablespec(TableSchema tableSchema) + throws IOException, InterruptedException { + String tableName = "table" + System.nanoTime(); + TableReference tableReference = + new TableReference() + .setProjectId(PROJECT) + .setDatasetId(BIG_QUERY_DATASET_ID) + .setTableId(tableName); + BQ_CLIENT.createNewTable( + PROJECT, + BIG_QUERY_DATASET_ID, + new Table().setSchema(tableSchema).setTableReference(tableReference)); + return PROJECT + "." + BIG_QUERY_DATASET_ID + "." + tableName; + } + + @Test + public void testMissingValueSchemaKnownTakeDefault() throws IOException, InterruptedException { + runTest(true, true, false); + } + + @Test + public void testMissingRequiredValueSchemaKnownTakeDefault() + throws IOException, InterruptedException { + runTest(true, true, true); + } + + @Test + public void testMissingRequiredValueSchemaKnownTakeNull() + throws IOException, InterruptedException { + runTest(true, false, true); + } + + @Test + public void testMissingRequiredValueSchemaUnknownTakeDefault() + throws IOException, InterruptedException { + runTest(false, true, true); + } + + @Test + public void testMissingValueSchemaUnknownTakeDefault() throws IOException, InterruptedException { + + runTest(false, true, false); + } + + @Test + public void testMissingValueSchemaKnownTakeNull() throws IOException, InterruptedException { + runTest(true, false, false); + } + + @Test + @Ignore // This currently appears broke in BigQuery. + public void testMissingValueSchemaUnknownTakeNull() throws IOException, InterruptedException { + runTest(false, false, false); + } + + public void runTest( + boolean sinkKnowsDefaultFields, boolean takeDefault, boolean defaultFieldsRequired) + throws IOException, InterruptedException { + boolean expectDeadLetter = !takeDefault && defaultFieldsRequired; + TableSchema bqSchema; + if (defaultFieldsRequired) { + bqSchema = + new TableSchema() + .setFields( + ImmutableList.of( + new TableFieldSchema().setName("id").setType("STRING"), + new TableFieldSchema().setName("key2").setType("STRING"), + new TableFieldSchema().setName("value").setType("STRING"), + new TableFieldSchema() + .setName("defaultrepeated") + .setType("STRING") + .setMode("REPEATED") + .setDefaultValueExpression("['a','b', 'c']"), + new TableFieldSchema() + .setName("defaultliteral") + .setType("INT64") + .setDefaultValueExpression("42") + .setMode("REQUIRED"), + new TableFieldSchema() + .setName("defaulttime") + .setType("TIMESTAMP") + .setDefaultValueExpression("CURRENT_TIMESTAMP()") + .setMode("REQUIRED"))); + } else { + bqSchema = + new TableSchema() + .setFields( + ImmutableList.of( + new TableFieldSchema().setName("id").setType("STRING"), + new TableFieldSchema().setName("key2").setType("STRING"), + new TableFieldSchema().setName("value").setType("STRING"), + new TableFieldSchema() + .setName("defaultrepeated") + .setType("STRING") + .setMode("REPEATED") + .setDefaultValueExpression("['a','b', 'c']"), + new TableFieldSchema() + .setName("defaultliteral") + .setType("INT64") + .setDefaultValueExpression("42"), + new TableFieldSchema() + .setName("defaulttime") + .setType("TIMESTAMP") + .setDefaultValueExpression("CURRENT_TIMESTAMP()"))); + } + + TableSchema sinkSchema = bqSchema; + if (!sinkKnowsDefaultFields) { + sinkSchema = + new TableSchema() + .setFields( + bqSchema.getFields().stream() + .filter(tfs -> tfs.getDefaultValueExpression() == null) + .collect(Collectors.toList())); + } + final TableRow row1 = + new TableRow() + .set("id", "row1") + .set("key2", "bar0") + .set("value", "1") + .set("defaultliteral", 12) + .set("defaultrepeated", Lists.newArrayList("foo", "bar")); + final TableRow row2 = new TableRow().set("id", "row2").set("key2", "bar1").set("value", "1"); + final TableRow row3 = new TableRow().set("id", "row3").set("key2", "bar2").set("value", "2"); + + List<TableRow> tableRows = Lists.newArrayList(row1, row2, row3); + + String tableSpec = createAndGetTablespec(bqSchema); + Pipeline p = Pipeline.create(); + + BigQueryIO.Write<TableRow> write = + BigQueryIO.writeTableRows() + .to(tableSpec) + .withSchema(sinkSchema) + .withNumStorageWriteApiStreams(2) + .ignoreUnknownValues() + .withTriggeringFrequency(Duration.standardSeconds(1)) + .withMethod(BigQueryIO.Write.Method.STORAGE_WRITE_API) + .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_NEVER); + if (!takeDefault) { + write = + write.withDefaultMissingValueInterpretation( + AppendRowsRequest.MissingValueInterpretation.NULL_VALUE); + } + WriteResult writeResult = + p.apply("Create rows", Create.of(tableRows)) + .setIsBoundedInternal(PCollection.IsBounded.UNBOUNDED) + .apply("write", write); + if (expectDeadLetter) { + PAssert.that(writeResult.getFailedStorageApiInserts()) + .satisfies( + (SerializableFunction<Iterable<BigQueryStorageApiInsertError>, Void>) + input -> { + assertThat(Lists.newArrayList(input).size(), is(3)); + // assertThat(input, containsInAnyOrder(tableRows)); + return null; + }); + } + p.run(); + + if (!expectDeadLetter) { + Map<String, TableRow> queryResponse = + BQ_CLIENT + .queryUnflattened( + String.format("SELECT * FROM %s", tableSpec), + PROJECT, + true, + true, + bigQueryLocation) + .stream() + .collect(Collectors.toMap(tr -> (String) tr.get("id"), Function.identity())); + assertThat(queryResponse.size(), equalTo(3)); + + TableRow resultRow1 = Preconditions.checkArgumentNotNull(queryResponse.get("row1")); + TableRow resultRow2 = Preconditions.checkArgumentNotNull(queryResponse.get("row2")); + TableRow resultRow3 = Preconditions.checkArgumentNotNull(queryResponse.get("row3")); + + if (sinkKnowsDefaultFields) { + assertThat(resultRow1.get("defaultliteral"), equalTo("12")); + assertThat( + (Collection<String>) resultRow1.get("defaultrepeated"), + containsInAnyOrder("foo", "bar")); + if (takeDefault) { + assertNotNull(resultRow1.get("defaulttime")); + assertNotNull(resultRow2.get("defaulttime")); + assertThat(resultRow2.get("defaultliteral"), equalTo("42")); + assertThat( + (Collection<String>) resultRow2.get("defaultrepeated"), + containsInAnyOrder("a", "b", "c")); + assertNotNull(resultRow3.get("defaulttime")); + assertThat(resultRow3.get("defaultliteral"), equalTo("42")); + assertThat( + (Collection<String>) resultRow3.get("defaultrepeated"), + containsInAnyOrder("a", "b", "c")); + } else { + assertNull(resultRow1.get("defaulttime")); + assertNull(resultRow2.get("defaulttime")); + assertNull(resultRow2.get("defaultliteral")); + assertThat((Collection<String>) resultRow2.get("defaultrepeated"), Matchers.empty()); + assertNull(resultRow3.get("defaulttime")); + assertNull(resultRow3.get("defaultliteral")); + assertThat((Collection<String>) resultRow3.get("defaultrepeated"), Matchers.empty()); + } + } else { + if (takeDefault) { + assertNotNull(resultRow1.get("defaulttime")); + assertThat(resultRow1.get("defaultliteral"), equalTo("42")); + assertThat( + (Collection<String>) resultRow1.get("defaultrepeated"), + containsInAnyOrder("a", "b", "c")); + assertNotNull(resultRow2.get("defaulttime")); + assertThat(resultRow2.get("defaultliteral"), equalTo("42")); + assertThat( + (Collection<String>) resultRow2.get("defaultrepeated"), + containsInAnyOrder("a", "b", "c")); + assertNotNull(resultRow3.get("defaulttime")); + assertThat(resultRow3.get("defaultliteral"), equalTo("42")); + assertThat( + (Collection<String>) resultRow3.get("defaultrepeated"), + containsInAnyOrder("a", "b", "c")); + } else { + assertNull(resultRow1.get("defaulttime")); + assertNull(resultRow1.get("defaultliteral")); + assertThat((Collection<String>) resultRow1.get("defaultrepeated"), Matchers.empty()); + assertNull(resultRow2.get("defaulttime")); + assertNull(resultRow2.get("defaultliteral")); + assertThat((Collection<String>) resultRow2.get("defaultrepeated"), Matchers.empty()); + assertNull(resultRow3.get("defaulttime")); + assertNull(resultRow3.get("defaultliteral")); + assertThat((Collection<String>) resultRow3.get("defaultrepeated"), Matchers.empty()); + } + } + } + } +} From 9a83620305529d41fa33144679370d27358c0d18 Mon Sep 17 00:00:00 2001 From: Yi Hu <yathu@google.com> Date: Tue, 31 Oct 2023 00:07:03 -0400 Subject: [PATCH 300/435] Rename --host to --fireStoreHost (#29201) * Rename --host to --fireStoreHost --- CHANGES.md | 1 + sdks/java/io/google-cloud-platform/build.gradle | 8 ++++---- .../beam/sdk/io/gcp/firestore/FirestoreOptions.java | 4 ++-- .../gcp/firestore/FirestoreStatefulComponentFactory.java | 2 +- .../sdk/io/gcp/firestore/it/FirestoreTestingHelper.java | 2 +- 5 files changed, 9 insertions(+), 8 deletions(-) diff --git a/CHANGES.md b/CHANGES.md index 6561cc2b56df..f34ffef79722 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -82,6 +82,7 @@ should handle this. ([#25252](https://github.com/apache/beam/issues/25252)). classes finally moved to `extensions/avro`. In case if it's still required to use `AvroCoder` for `CounterMark`, then, as a workaround, a copy of "old" `CountingSource` class should be placed into a project code and used directly ([#25252](https://github.com/apache/beam/issues/25252)). +* Renamed `host` to `firestoreHost` in `FirestoreOptions` to avoid potential conflict of command line arguments (Java) ([#29201](https://github.com/apache/beam/pull/29201)). ## Deprecations diff --git a/sdks/java/io/google-cloud-platform/build.gradle b/sdks/java/io/google-cloud-platform/build.gradle index c4a508680186..d66122e4d107 100644 --- a/sdks/java/io/google-cloud-platform/build.gradle +++ b/sdks/java/io/google-cloud-platform/build.gradle @@ -186,13 +186,13 @@ task integrationTest(type: Test, dependsOn: processTestResources) { def gcpProject = project.findProperty('gcpProject') ?: 'apache-beam-testing' def gcpTempRoot = project.findProperty('gcpTempRoot') ?: 'gs://temp-storage-for-end-to-end-tests' def firestoreDb = project.findProperty('firestoreDb') ?: 'firestoredb' - def host = project.findProperty('host') ?: 'batch-firestore.googleapis.com:443' + def firestoreHost = project.findProperty('firestoreHost') ?: 'batch-firestore.googleapis.com:443' systemProperty "beamTestPipelineOptions", JsonOutput.toJson([ "--runner=DirectRunner", "--project=${gcpProject}", "--tempRoot=${gcpTempRoot}", "--firestoreDb=${firestoreDb}", - "--host=${host}", + "--firestoreHost=${firestoreHost}", ]) // Disable Gradle cache: these ITs interact with live service that should always be considered "out of date" @@ -220,14 +220,14 @@ task integrationTestKms(type: Test) { def gcpTempRoot = project.findProperty('gcpTempRootKms') ?: 'gs://temp-storage-for-end-to-end-tests-cmek' def dataflowKmsKey = project.findProperty('dataflowKmsKey') ?: "projects/apache-beam-testing/locations/global/keyRings/beam-it/cryptoKeys/test" def firestoreDb = project.findProperty('firestoreDb') ?: 'firestoredb' - def host = project.findProperty('host') ?: 'batch-firestore.googleapis.com:443' + def firestoreHost = project.findProperty('firestoreHost') ?: 'batch-firestore.googleapis.com:443' systemProperty "beamTestPipelineOptions", JsonOutput.toJson([ "--runner=DirectRunner", "--project=${gcpProject}", "--tempRoot=${gcpTempRoot}", "--dataflowKmsKey=${dataflowKmsKey}", "--firestoreDb=${firestoreDb}", - "--host=${host}", + "--firestoreHost=${firestoreHost}", ]) // Disable Gradle cache: these ITs interact with live service that should always be considered "out of date" diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/FirestoreOptions.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/FirestoreOptions.java index 1be6568372d9..a292a106e51f 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/FirestoreOptions.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/FirestoreOptions.java @@ -66,7 +66,7 @@ public interface FirestoreOptions extends PipelineOptions { */ @Description("Firestore endpoint (host and port)") @Default.String("batch-firestore.googleapis.com:443") - String getHost(); + String getFirestoreHost(); /** * Define a host port pair to allow connecting to a Cloud Firestore instead of the default live @@ -74,5 +74,5 @@ public interface FirestoreOptions extends PipelineOptions { * * @param host the host and port to connect to */ - void setHost(String host); + void setFirestoreHost(String host); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/FirestoreStatefulComponentFactory.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/FirestoreStatefulComponentFactory.java index 21c29c485d1e..4e8c11f7072c 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/FirestoreStatefulComponentFactory.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/FirestoreStatefulComponentFactory.java @@ -93,7 +93,7 @@ FirestoreStub getFirestoreStub(PipelineOptions options) { GcpOptions gcpOptions = options.as(GcpOptions.class); builder .setCredentialsProvider(FixedCredentialsProvider.create(gcpOptions.getGcpCredential())) - .setEndpoint(firestoreOptions.getHost()); + .setEndpoint(firestoreOptions.getFirestoreHost()); headers.put( "x-goog-request-params", "project_id=" diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/firestore/it/FirestoreTestingHelper.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/firestore/it/FirestoreTestingHelper.java index d8c55d44f3c8..a57dd688d4af 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/firestore/it/FirestoreTestingHelper.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/firestore/it/FirestoreTestingHelper.java @@ -134,7 +134,7 @@ public FirestoreTestingHelper(CleanupMode cleanupMode) { .setCredentials(gcpOptions.getGcpCredential()) .setProjectId(gcpOptions.getProject()) .setDatabaseId(firestoreBeamOptions.getFirestoreDb()) - .setHost(firestoreBeamOptions.getHost()) + .setHost(firestoreBeamOptions.getFirestoreHost()) .build(); fs = firestoreOptions.getService(); rpc = (FirestoreRpc) firestoreOptions.getRpc(); From 714badb821b7acfb9801a69dd0aff08bc84f6b4f Mon Sep 17 00:00:00 2001 From: Yi Hu <yathu@google.com> Date: Tue, 31 Oct 2023 00:13:09 -0400 Subject: [PATCH 301/435] Better naming and logging to investigate leaking bigtable resources (#29203) --- sdks/python/apache_beam/io/gcp/bigtableio_it_test.py | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/sdks/python/apache_beam/io/gcp/bigtableio_it_test.py b/sdks/python/apache_beam/io/gcp/bigtableio_it_test.py index f61e346cff9f..6cdd0bbeecf6 100644 --- a/sdks/python/apache_beam/io/gcp/bigtableio_it_test.py +++ b/sdks/python/apache_beam/io/gcp/bigtableio_it_test.py @@ -96,7 +96,7 @@ def tearDown(self): self.table.delete() self.instance.delete() except HttpError: - _LOGGER.debug( + _LOGGER.warning( "Failed to clean up table [%s] and instance [%s]", self.table.table_id, self.instance.instance_id) @@ -160,8 +160,8 @@ def setUpClass(cls): cls.args = cls.test_pipeline.get_full_options_as_args() cls.expansion_service = ('localhost:%s' % os.environ.get('EXPANSION_PORT')) - instance_id = '%s-%s-%s' % ( - cls.INSTANCE, str(int(time.time())), secrets.token_hex(3)) + timestr = "".join(filter(str.isdigit, str(datetime.datetime.utcnow()))) + instance_id = '%s-%s-%s' % (cls.INSTANCE, timestr, secrets.token_hex(3)) cls.client = client.Client(admin=True, project=cls.project) # create cluster and instance @@ -190,7 +190,7 @@ def tearDown(self): _LOGGER.info("Deleting table [%s]", self.table.table_id) self.table.delete() except HttpError: - _LOGGER.debug("Failed to clean up table [%s]", self.table.table_id) + _LOGGER.warning("Failed to clean up table [%s]", self.table.table_id) @classmethod def tearDownClass(cls): @@ -198,7 +198,7 @@ def tearDownClass(cls): _LOGGER.info("Deleting instance [%s]", cls.instance.instance_id) cls.instance.delete() except HttpError: - _LOGGER.debug( + _LOGGER.warning( "Failed to clean up instance [%s]", cls.instance.instance_id) def run_pipeline(self, rows): From 524a7bfef40f7d642423f24dc0b8d7f6d2a0835b Mon Sep 17 00:00:00 2001 From: Jan Lukavsky <je.ik@seznam.cz> Date: Fri, 27 Oct 2023 14:55:56 +0200 Subject: [PATCH 302/435] [runners-spark] Do not set accTimestamp to null in SparkCombineFn (#28256) --- .../spark/translation/SparkCombineFn.java | 18 +++++------- .../spark/translation/SparkCombineFnTest.java | 29 +++++++++++++++++++ 2 files changed, 37 insertions(+), 10 deletions(-) diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkCombineFn.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkCombineFn.java index ddf4b12bae13..1075ae0d2a7d 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkCombineFn.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkCombineFn.java @@ -41,7 +41,6 @@ import org.apache.beam.runners.spark.util.SideInputBroadcast; import org.apache.beam.runners.spark.util.SparkSideInputReader; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.IterableCoder; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.CombineWithContext; @@ -101,7 +100,7 @@ void add(WindowedValue<InputT> value, SparkCombineFn<InputT, ValueT, AccumT, ?> throws Exception; /** - * Merge other acccumulator into this one. + * Merge other accumulator into this one. * * @param other the other accumulator to merge */ @@ -173,7 +172,7 @@ static <InputT, ValueT, AccumT> SingleWindowWindowedAccumulator<InputT, ValueT, return new SingleWindowWindowedAccumulator<>(toValue); } - static <InputT, ValueT, AccumT> WindowedAccumulator<InputT, ValueT, AccumT, ?> create( + static <InputT, ValueT, AccumT> SingleWindowWindowedAccumulator<InputT, ValueT, AccumT> create( Function<InputT, ValueT> toValue, WindowedValue<AccumT> accumulator) { return new SingleWindowWindowedAccumulator<>(toValue, accumulator); } @@ -191,10 +190,7 @@ static <InputT, ValueT, AccumT> SingleWindowWindowedAccumulator<InputT, ValueT, Function<InputT, ValueT> toValue, WindowedValue<AccumT> accumulator) { this.toValue = toValue; this.windowAccumulator = accumulator.getValue(); - this.accTimestamp = - accumulator.getTimestamp().equals(BoundedWindow.TIMESTAMP_MIN_VALUE) - ? null - : accumulator.getTimestamp(); + this.accTimestamp = accumulator.getTimestamp(); this.accWindow = getWindow(accumulator); } @@ -247,7 +243,7 @@ public void merge( @Override public Collection<WindowedValue<AccumT>> extractOutput() { if (windowAccumulator != null) { - return Arrays.asList( + return Collections.singletonList( WindowedValue.of( windowAccumulator, accTimestamp, accWindow, PaneInfo.ON_TIME_AND_ONLY_FIRING)); } @@ -516,7 +512,8 @@ static class WindowedAccumulatorCoder<InputT, ValueT, AccumT> @Override public void encode(WindowedAccumulator<InputT, ValueT, AccumT, ?> value, OutputStream outStream) - throws CoderException, IOException { + throws IOException { + if (type.isMapBased()) { wrap.encode(((MapBasedWindowedAccumulator<?, ?, AccumT, ?>) value).map.values(), outStream); } else { @@ -536,7 +533,8 @@ public void encode(WindowedAccumulator<InputT, ValueT, AccumT, ?> value, OutputS @Override public WindowedAccumulator<InputT, ValueT, AccumT, ?> decode(InputStream inStream) - throws CoderException, IOException { + throws IOException { + if (type.isMapBased()) { return WindowedAccumulator.create(toValue, type, wrap.decode(inStream), windowComparator); } diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkCombineFnTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkCombineFnTest.java index 295b7ef2b948..9cb4b44c897c 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkCombineFnTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkCombineFnTest.java @@ -36,6 +36,7 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.Sessions; import org.apache.beam.sdk.transforms.windowing.SlidingWindows; +import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.CombineFnUtil; import org.apache.beam.sdk.util.WindowedValue; @@ -219,6 +220,34 @@ public void testSlidingCombineFnExplode() throws Exception { result); } + @Test + public void testGlobalWindowMergeAccumulatorsWithEarliestCombiner() throws Exception { + SparkCombineFn<KV<String, Integer>, Integer, Long, Long> sparkCombineFn = + SparkCombineFn.keyed( + combineFn, + opts, + Collections.emptyMap(), + WindowingStrategy.globalDefault().withTimestampCombiner(TimestampCombiner.EARLIEST)); + + Instant ts = BoundedWindow.TIMESTAMP_MIN_VALUE; + WindowedValue<KV<String, Integer>> first = input("key", 1, ts); + WindowedValue<KV<String, Integer>> second = input("key", 2, ts); + WindowedValue<KV<String, Integer>> third = input("key", 3, ts); + WindowedValue<Long> accumulator = WindowedValue.valueInGlobalWindow(0L); + SparkCombineFn.SingleWindowWindowedAccumulator<KV<String, Integer>, Integer, Long> acc1 = + SparkCombineFn.SingleWindowWindowedAccumulator.create(KV::getValue, accumulator); + SparkCombineFn.SingleWindowWindowedAccumulator<KV<String, Integer>, Integer, Long> acc2 = + SparkCombineFn.SingleWindowWindowedAccumulator.create(KV::getValue, accumulator); + SparkCombineFn.SingleWindowWindowedAccumulator<KV<String, Integer>, Integer, Long> acc3 = + SparkCombineFn.SingleWindowWindowedAccumulator.create(KV::getValue, accumulator); + acc1.add(first, sparkCombineFn); + acc2.add(second, sparkCombineFn); + acc3.merge(acc1, sparkCombineFn); + acc3.merge(acc2, sparkCombineFn); + acc3.add(third, sparkCombineFn); + assertEquals(6, (long) Iterables.getOnlyElement(sparkCombineFn.extractOutput(acc3)).getValue()); + } + private static Combine.CombineFn<Integer, Long, Long> getSumFn() { return new Combine.CombineFn<Integer, Long, Long>() { From ba714221d5efaea58a59bccaad2eaeef70bd4ec4 Mon Sep 17 00:00:00 2001 From: martin trieu <martinkt@google.com> Date: Tue, 31 Oct 2023 02:37:10 -0700 Subject: [PATCH 303/435] organize and refactor GrpcWindmillServer. (#29156) organize and refactor GrpcWindmillServer to prepare for Streaming Engine Client changes. --- .../worker/build.gradle | 3 + .../MetricTrackingWindmillServerStub.java | 4 +- .../worker/StreamingDataflowWorker.java | 6 +- .../StreamingDataflowWorkerOptions.java | 2 +- .../worker/windmill/WindmillServerBase.java | 8 +- .../worker/windmill/WindmillServerStub.java | 8 +- .../{ => client}/AbstractWindmillStream.java | 3 +- .../windmill/{ => client}/WindmillStream.java | 21 +- .../{ => client}/WindmillStreamPool.java | 2 +- .../grpc}/AppendableInputStream.java | 2 +- .../grpc}/GetWorkTimingInfosTracker.java | 2 +- .../grpc}/GrpcCommitWorkStream.java | 9 +- .../grpc}/GrpcDeadlineClientInterceptor.java | 2 +- .../client/grpc/GrpcDispatcherClient.java | 136 ++++ .../grpc}/GrpcGetDataStream.java | 13 +- .../grpc}/GrpcGetDataStreamRequests.java | 2 +- .../grpc}/GrpcGetWorkStream.java | 29 +- .../grpc}/GrpcGetWorkerMetadataStream.java | 9 +- .../client/grpc/GrpcWindmillServer.java | 355 ++++++++++ .../grpc/GrpcWindmillStreamFactory.java | 227 +++++++ .../grpc/auth/VendoredCredentialsAdapter.java | 81 +++ ...endoredRequestMetadataCallbackAdapter.java | 51 ++ .../grpc/observers}/DirectStreamObserver.java | 2 +- .../ForwardingClientResponseObserver.java | 2 +- .../observers}/StreamObserverFactory.java | 2 +- .../grpc/stubs/WindmillChannelFactory.java | 137 ++++ .../grpc/stubs/WindmillStubFactory.java | 73 +++ .../StreamingEngineThrottleTimers.java | 41 ++ .../throttling}/ThrottleTimer.java | 6 +- .../grpcclient/GrpcWindmillServer.java | 607 ------------------ .../windmill/work/WorkItemReceiver.java | 34 + .../windmill/work/budget/GetWorkBudget.java | 98 +++ .../dataflow/worker/FakeWindmillServer.java | 24 +- .../{ => client}/WindmillStreamPoolTest.java | 2 +- .../GrpcGetWorkerMetadataStreamTest.java | 9 +- .../grpc}/GrpcWindmillServerTest.java | 15 +- .../work/budget/GetWorkBudgetTest.java | 72 +++ 37 files changed, 1413 insertions(+), 686 deletions(-) rename runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/{ => client}/AbstractWindmillStream.java (98%) rename runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/{ => client}/WindmillStream.java (84%) rename runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/{ => client}/WindmillStreamPool.java (99%) rename runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/{grpcclient => client/grpc}/AppendableInputStream.java (98%) rename runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/{grpcclient => client/grpc}/GetWorkTimingInfosTracker.java (99%) rename runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/{grpcclient => client/grpc}/GrpcCommitWorkStream.java (96%) rename runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/{grpcclient => client/grpc}/GrpcDeadlineClientInterceptor.java (97%) create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDispatcherClient.java rename runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/{grpcclient => client/grpc}/GrpcGetDataStream.java (95%) rename runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/{grpcclient => client/grpc}/GrpcGetDataStreamRequests.java (98%) rename runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/{grpcclient => client/grpc}/GrpcGetWorkStream.java (89%) rename runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/{grpcclient => client/grpc}/GrpcGetWorkerMetadataStream.java (93%) create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillServer.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillStreamFactory.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/auth/VendoredCredentialsAdapter.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/auth/VendoredRequestMetadataCallbackAdapter.java rename runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/{ => client/grpc/observers}/DirectStreamObserver.java (98%) rename runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/{ => client/grpc/observers}/ForwardingClientResponseObserver.java (96%) rename runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/{ => client/grpc/observers}/StreamObserverFactory.java (97%) create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/stubs/WindmillChannelFactory.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/stubs/WindmillStubFactory.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/throttling/StreamingEngineThrottleTimers.java rename runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/{grpcclient => client/throttling}/ThrottleTimer.java (94%) delete mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GrpcWindmillServer.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/WorkItemReceiver.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/GetWorkBudget.java rename runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/{ => client}/WindmillStreamPoolTest.java (99%) rename runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/{grpcclient => client/grpc}/GrpcGetWorkerMetadataStreamTest.java (96%) rename runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/{grpcclient => client/grpc}/GrpcWindmillServerTest.java (98%) create mode 100644 runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/GetWorkBudgetTest.java diff --git a/runners/google-cloud-dataflow-java/worker/build.gradle b/runners/google-cloud-dataflow-java/worker/build.gradle index ce06063c9b52..1ca9eba2b482 100644 --- a/runners/google-cloud-dataflow-java/worker/build.gradle +++ b/runners/google-cloud-dataflow-java/worker/build.gradle @@ -89,6 +89,9 @@ applyJavaNature( // Allow slf4j implementation worker for logging during pipeline execution "org/slf4j/impl/**" ], + generatedClassPatterns: [ + /^org\.apache\.beam\.runners\.dataflow\.worker\.windmill.*/ + ], shadowClosure: { // Each included dependency must also include all of its necessary transitive dependencies // or have them provided by the users pipeline during job submission. Typically a users diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/MetricTrackingWindmillServerStub.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/MetricTrackingWindmillServerStub.java index 33b55647213a..0e929249b3a1 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/MetricTrackingWindmillServerStub.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/MetricTrackingWindmillServerStub.java @@ -29,8 +29,8 @@ import org.apache.beam.runners.dataflow.worker.windmill.Windmill; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.KeyedGetDataRequest; import org.apache.beam.runners.dataflow.worker.windmill.WindmillServerStub; -import org.apache.beam.runners.dataflow.worker.windmill.WindmillStream.GetDataStream; -import org.apache.beam.runners.dataflow.worker.windmill.WindmillStreamPool; +import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.GetDataStream; +import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStreamPool; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.SettableFuture; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Duration; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java index 811250ee785c..11849e8b8c4c 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java @@ -104,9 +104,9 @@ import org.apache.beam.runners.dataflow.worker.windmill.Windmill.LatencyAttribution; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkItemCommitRequest; import org.apache.beam.runners.dataflow.worker.windmill.WindmillServerStub; -import org.apache.beam.runners.dataflow.worker.windmill.WindmillStream.CommitWorkStream; -import org.apache.beam.runners.dataflow.worker.windmill.WindmillStream.GetWorkStream; -import org.apache.beam.runners.dataflow.worker.windmill.WindmillStreamPool; +import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.CommitWorkStream; +import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.GetWorkStream; +import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStreamPool; import org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateCache; import org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateReader; import org.apache.beam.sdk.coders.Coder; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/options/StreamingDataflowWorkerOptions.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/options/StreamingDataflowWorkerOptions.java index cc5b3302b01b..bacfa1eef63b 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/options/StreamingDataflowWorkerOptions.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/options/StreamingDataflowWorkerOptions.java @@ -21,7 +21,7 @@ import org.apache.beam.runners.dataflow.options.DataflowWorkerHarnessOptions; import org.apache.beam.runners.dataflow.worker.windmill.WindmillServerStub; import org.apache.beam.runners.dataflow.worker.windmill.appliance.JniWindmillApplianceServer; -import org.apache.beam.runners.dataflow.worker.windmill.grpcclient.GrpcWindmillServer; +import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.GrpcWindmillServer; import org.apache.beam.sdk.options.Default; import org.apache.beam.sdk.options.DefaultValueFactory; import org.apache.beam.sdk.options.Description; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/WindmillServerBase.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/WindmillServerBase.java index fe81eece1383..8caa79cd3f76 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/WindmillServerBase.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/WindmillServerBase.java @@ -19,10 +19,10 @@ import java.io.IOException; import java.util.Set; -import org.apache.beam.runners.dataflow.worker.windmill.WindmillStream.CommitWorkStream; -import org.apache.beam.runners.dataflow.worker.windmill.WindmillStream.GetDataStream; -import org.apache.beam.runners.dataflow.worker.windmill.WindmillStream.GetWorkStream; -import org.apache.beam.runners.dataflow.worker.windmill.WindmillStream.GetWorkStream.WorkItemReceiver; +import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.CommitWorkStream; +import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.GetDataStream; +import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.GetWorkStream; +import org.apache.beam.runners.dataflow.worker.windmill.work.WorkItemReceiver; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.net.HostAndPort; /** diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/WindmillServerStub.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/WindmillServerStub.java index 1bb5359e06f4..c327e68d7e91 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/WindmillServerStub.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/WindmillServerStub.java @@ -21,10 +21,10 @@ import java.io.PrintWriter; import java.util.Set; import org.apache.beam.runners.dataflow.worker.status.StatusDataProvider; -import org.apache.beam.runners.dataflow.worker.windmill.WindmillStream.CommitWorkStream; -import org.apache.beam.runners.dataflow.worker.windmill.WindmillStream.GetDataStream; -import org.apache.beam.runners.dataflow.worker.windmill.WindmillStream.GetWorkStream; -import org.apache.beam.runners.dataflow.worker.windmill.WindmillStream.GetWorkStream.WorkItemReceiver; +import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.CommitWorkStream; +import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.GetDataStream; +import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.GetWorkStream; +import org.apache.beam.runners.dataflow.worker.windmill.work.WorkItemReceiver; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.net.HostAndPort; /** Stub for communicating with a Windmill server. */ diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/AbstractWindmillStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java similarity index 98% rename from runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/AbstractWindmillStream.java rename to runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java index ea7efff7a06d..4e47676989a6 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/AbstractWindmillStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.dataflow.worker.windmill; +package org.apache.beam.runners.dataflow.worker.windmill.client; import java.io.IOException; import java.io.PrintWriter; @@ -30,6 +30,7 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; import java.util.function.Supplier; +import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.observers.StreamObserverFactory; import org.apache.beam.sdk.util.BackOff; import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.Status; import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.StatusRuntimeException; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/WindmillStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStream.java similarity index 84% rename from runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/WindmillStream.java rename to runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStream.java index 4dd4164fc4ef..fa1f797a1911 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/WindmillStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStream.java @@ -15,15 +15,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.dataflow.worker.windmill; +package org.apache.beam.runners.dataflow.worker.windmill.client; -import java.util.Collection; import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; import java.util.function.Consumer; import javax.annotation.concurrent.ThreadSafe; -import org.checkerframework.checker.nullness.qual.Nullable; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill; +import org.apache.beam.runners.dataflow.worker.windmill.work.budget.GetWorkBudget; import org.joda.time.Instant; /** Superclass for streams returned by streaming Windmill methods. */ @@ -41,16 +41,11 @@ public interface WindmillStream { /** Handle representing a stream of GetWork responses. */ @ThreadSafe interface GetWorkStream extends WindmillStream { - /** Functional interface for receiving WorkItems. */ - @FunctionalInterface - interface WorkItemReceiver { - void receiveWork( - String computation, - @Nullable Instant inputDataWatermark, - @Nullable Instant synchronizedProcessingTime, - Windmill.WorkItem workItem, - Collection<Windmill.LatencyAttribution> getWorkStreamLatencies); - } + /** Adjusts the {@link GetWorkBudget} for the stream. */ + void adjustBudget(long itemsDelta, long bytesDelta); + + /** Returns the remaining in-flight {@link GetWorkBudget}. */ + GetWorkBudget remainingBudget(); } /** Interface for streaming GetDataRequests to Windmill. */ diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/WindmillStreamPool.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStreamPool.java similarity index 99% rename from runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/WindmillStreamPool.java rename to runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStreamPool.java index 9cd4ab0ea4a5..9f1b67edc1e0 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/WindmillStreamPool.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStreamPool.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.dataflow.worker.windmill; +package org.apache.beam.runners.dataflow.worker.windmill.client; import java.util.ArrayList; import java.util.HashMap; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/AppendableInputStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/AppendableInputStream.java similarity index 98% rename from runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/AppendableInputStream.java rename to runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/AppendableInputStream.java index dbd3613ee4c2..6a0d0a63d5a9 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/AppendableInputStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/AppendableInputStream.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.dataflow.worker.windmill.grpcclient; +package org.apache.beam.runners.dataflow.worker.windmill.client.grpc; import java.io.IOException; import java.io.InputStream; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GetWorkTimingInfosTracker.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GetWorkTimingInfosTracker.java similarity index 99% rename from runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GetWorkTimingInfosTracker.java rename to runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GetWorkTimingInfosTracker.java index e6710993af9b..221b18be164c 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GetWorkTimingInfosTracker.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GetWorkTimingInfosTracker.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.dataflow.worker.windmill.grpcclient; +package org.apache.beam.runners.dataflow.worker.windmill.client.grpc; import java.util.ArrayList; import java.util.Collection; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GrpcCommitWorkStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcCommitWorkStream.java similarity index 96% rename from runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GrpcCommitWorkStream.java rename to runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcCommitWorkStream.java index 1bba40805dec..5d0a5085fe1b 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GrpcCommitWorkStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcCommitWorkStream.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.dataflow.worker.windmill.grpcclient; +package org.apache.beam.runners.dataflow.worker.windmill.client.grpc; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull; @@ -27,15 +27,16 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.function.Consumer; import java.util.function.Function; -import org.apache.beam.runners.dataflow.worker.windmill.AbstractWindmillStream; -import org.apache.beam.runners.dataflow.worker.windmill.StreamObserverFactory; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.CommitStatus; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.JobHeader; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.StreamingCommitRequestChunk; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.StreamingCommitResponse; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.StreamingCommitWorkRequest; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkItemCommitRequest; -import org.apache.beam.runners.dataflow.worker.windmill.WindmillStream.CommitWorkStream; +import org.apache.beam.runners.dataflow.worker.windmill.client.AbstractWindmillStream; +import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.CommitWorkStream; +import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.observers.StreamObserverFactory; +import org.apache.beam.runners.dataflow.worker.windmill.client.throttling.ThrottleTimer; import org.apache.beam.sdk.util.BackOff; import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GrpcDeadlineClientInterceptor.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDeadlineClientInterceptor.java similarity index 97% rename from runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GrpcDeadlineClientInterceptor.java rename to runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDeadlineClientInterceptor.java index 6b0e19cbb480..629006e23596 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GrpcDeadlineClientInterceptor.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDeadlineClientInterceptor.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.dataflow.worker.windmill.grpcclient; +package org.apache.beam.runners.dataflow.worker.windmill.client.grpc; import java.util.concurrent.TimeUnit; import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.CallOptions; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDispatcherClient.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDispatcherClient.java new file mode 100644 index 000000000000..ef9156f9c050 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDispatcherClient.java @@ -0,0 +1,136 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.dataflow.worker.windmill.client.grpc; + +import static org.apache.beam.runners.dataflow.worker.windmill.client.grpc.stubs.WindmillChannelFactory.LOCALHOST; +import static org.apache.beam.runners.dataflow.worker.windmill.client.grpc.stubs.WindmillChannelFactory.localhostChannel; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Random; +import java.util.Set; +import javax.annotation.concurrent.GuardedBy; +import javax.annotation.concurrent.ThreadSafe; +import org.apache.beam.runners.dataflow.worker.windmill.CloudWindmillServiceV1Alpha1Grpc; +import org.apache.beam.runners.dataflow.worker.windmill.CloudWindmillServiceV1Alpha1Grpc.CloudWindmillServiceV1Alpha1Stub; +import org.apache.beam.runners.dataflow.worker.windmill.WindmillServiceAddress; +import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.stubs.WindmillStubFactory; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.net.HostAndPort; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Manages endpoints and stubs for connecting to the Windmill Dispatcher. */ +@ThreadSafe +class GrpcDispatcherClient { + private static final Logger LOG = LoggerFactory.getLogger(GrpcDispatcherClient.class); + private final WindmillStubFactory windmillStubFactory; + + @GuardedBy("this") + private final List<CloudWindmillServiceV1Alpha1Stub> dispatcherStubs; + + @GuardedBy("this") + private final Set<HostAndPort> dispatcherEndpoints; + + @GuardedBy("this") + private final Random rand; + + private GrpcDispatcherClient( + WindmillStubFactory windmillStubFactory, + List<CloudWindmillServiceV1Alpha1Stub> dispatcherStubs, + Set<HostAndPort> dispatcherEndpoints, + Random rand) { + this.windmillStubFactory = windmillStubFactory; + this.dispatcherStubs = dispatcherStubs; + this.dispatcherEndpoints = dispatcherEndpoints; + this.rand = rand; + } + + static GrpcDispatcherClient create(WindmillStubFactory windmillStubFactory) { + return new GrpcDispatcherClient( + windmillStubFactory, new ArrayList<>(), new HashSet<>(), new Random()); + } + + @VisibleForTesting + static GrpcDispatcherClient forTesting( + WindmillStubFactory windmillGrpcStubFactory, + List<CloudWindmillServiceV1Alpha1Stub> dispatcherStubs, + Set<HostAndPort> dispatcherEndpoints) { + Preconditions.checkArgument(dispatcherEndpoints.size() == dispatcherStubs.size()); + return new GrpcDispatcherClient( + windmillGrpcStubFactory, dispatcherStubs, dispatcherEndpoints, new Random()); + } + + synchronized CloudWindmillServiceV1Alpha1Stub getDispatcherStub() { + Preconditions.checkState( + !dispatcherStubs.isEmpty(), "windmillServiceEndpoint has not been set"); + + return (dispatcherStubs.size() == 1 + ? dispatcherStubs.get(0) + : dispatcherStubs.get(rand.nextInt(dispatcherStubs.size()))); + } + + synchronized boolean isReady() { + return !dispatcherStubs.isEmpty(); + } + + synchronized void consumeWindmillDispatcherEndpoints( + ImmutableSet<HostAndPort> dispatcherEndpoints) { + Preconditions.checkArgument( + dispatcherEndpoints != null && !dispatcherEndpoints.isEmpty(), + "Cannot set dispatcher endpoints to nothing."); + if (this.dispatcherEndpoints.equals(dispatcherEndpoints)) { + // The endpoints are equal don't recreate the stubs. + return; + } + + LOG.info("Creating a new windmill stub, endpoints: {}", dispatcherEndpoints); + if (!this.dispatcherEndpoints.isEmpty()) { + LOG.info("Previous windmill stub endpoints: {}", this.dispatcherEndpoints); + } + + resetDispatcherEndpoints(dispatcherEndpoints); + } + + private synchronized void resetDispatcherEndpoints( + ImmutableSet<HostAndPort> newDispatcherEndpoints) { + LOG.info("Initializing Streaming Engine GRPC client for endpoints: {}", newDispatcherEndpoints); + this.dispatcherStubs.clear(); + this.dispatcherEndpoints.clear(); + this.dispatcherEndpoints.addAll(newDispatcherEndpoints); + + dispatcherEndpoints.stream() + .map(this::createDispatcherStubForWindmillService) + .forEach(dispatcherStubs::add); + } + + private CloudWindmillServiceV1Alpha1Stub createDispatcherStubForWindmillService( + HostAndPort endpoint) { + if (LOCALHOST.equals(endpoint.getHost())) { + return CloudWindmillServiceV1Alpha1Grpc.newStub(localhostChannel(endpoint.getPort())); + } + + // Use an in-process stub if testing. + return windmillStubFactory.getKind() == WindmillStubFactory.Kind.IN_PROCESS + ? windmillStubFactory.inProcess().get() + : windmillStubFactory.remote().apply(WindmillServiceAddress.create(endpoint)); + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GrpcGetDataStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStream.java similarity index 95% rename from runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GrpcGetDataStream.java rename to runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStream.java index 238cc771dce8..ea9cd7f0fa32 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GrpcGetDataStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStream.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.dataflow.worker.windmill.grpcclient; +package org.apache.beam.runners.dataflow.worker.windmill.client.grpc; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Verify.verify; @@ -33,8 +33,6 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicLong; import java.util.function.Function; -import org.apache.beam.runners.dataflow.worker.windmill.AbstractWindmillStream; -import org.apache.beam.runners.dataflow.worker.windmill.StreamObserverFactory; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.ComputationGetDataRequest; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GlobalData; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GlobalDataRequest; @@ -43,9 +41,12 @@ import org.apache.beam.runners.dataflow.worker.windmill.Windmill.KeyedGetDataResponse; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.StreamingGetDataRequest; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.StreamingGetDataResponse; -import org.apache.beam.runners.dataflow.worker.windmill.WindmillStream.GetDataStream; -import org.apache.beam.runners.dataflow.worker.windmill.grpcclient.GrpcGetDataStreamRequests.QueuedBatch; -import org.apache.beam.runners.dataflow.worker.windmill.grpcclient.GrpcGetDataStreamRequests.QueuedRequest; +import org.apache.beam.runners.dataflow.worker.windmill.client.AbstractWindmillStream; +import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.GetDataStream; +import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.GrpcGetDataStreamRequests.QueuedBatch; +import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.GrpcGetDataStreamRequests.QueuedRequest; +import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.observers.StreamObserverFactory; +import org.apache.beam.runners.dataflow.worker.windmill.client.throttling.ThrottleTimer; import org.apache.beam.sdk.util.BackOff; import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; import org.joda.time.Instant; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GrpcGetDataStreamRequests.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStreamRequests.java similarity index 98% rename from runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GrpcGetDataStreamRequests.java rename to runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStreamRequests.java index 7da7b13958b9..cda9537127d9 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GrpcGetDataStreamRequests.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStreamRequests.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.dataflow.worker.windmill.grpcclient; +package org.apache.beam.runners.dataflow.worker.windmill.client.grpc; import com.google.auto.value.AutoOneOf; import java.util.ArrayList; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GrpcGetWorkStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetWorkStream.java similarity index 89% rename from runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GrpcGetWorkStream.java rename to runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetWorkStream.java index 4660fe25b13b..d7d9bfddffb0 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GrpcGetWorkStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetWorkStream.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.dataflow.worker.windmill.grpcclient; +package org.apache.beam.runners.dataflow.worker.windmill.client.grpc; import java.io.IOException; import java.io.PrintWriter; @@ -27,16 +27,18 @@ import java.util.function.Function; import javax.annotation.Nullable; import org.apache.beam.runners.dataflow.worker.WindmillTimeUtils; -import org.apache.beam.runners.dataflow.worker.windmill.AbstractWindmillStream; -import org.apache.beam.runners.dataflow.worker.windmill.StreamObserverFactory; import org.apache.beam.runners.dataflow.worker.windmill.Windmill; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GetWorkRequest; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.LatencyAttribution; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.StreamingGetWorkRequest; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.StreamingGetWorkRequestExtension; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.StreamingGetWorkResponseChunk; -import org.apache.beam.runners.dataflow.worker.windmill.WindmillStream.GetWorkStream; -import org.apache.beam.runners.dataflow.worker.windmill.WindmillStream.GetWorkStream.WorkItemReceiver; +import org.apache.beam.runners.dataflow.worker.windmill.client.AbstractWindmillStream; +import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.GetWorkStream; +import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.observers.StreamObserverFactory; +import org.apache.beam.runners.dataflow.worker.windmill.client.throttling.ThrottleTimer; +import org.apache.beam.runners.dataflow.worker.windmill.work.WorkItemReceiver; +import org.apache.beam.runners.dataflow.worker.windmill.work.budget.GetWorkBudget; import org.apache.beam.sdk.util.BackOff; import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; @@ -44,7 +46,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -final class GrpcGetWorkStream +public final class GrpcGetWorkStream extends AbstractWindmillStream<StreamingGetWorkRequest, StreamingGetWorkResponseChunk> implements GetWorkStream { @@ -79,7 +81,7 @@ private GrpcGetWorkStream( this.inflightBytes = new AtomicLong(); } - static GrpcGetWorkStream create( + public static GrpcGetWorkStream create( Function< StreamObserver<StreamingGetWorkResponseChunk>, StreamObserver<StreamingGetWorkRequest>> @@ -190,6 +192,19 @@ protected void startThrottleTimer() { getWorkThrottleTimer.start(); } + @Override + public void adjustBudget(long itemsDelta, long bytesDelta) { + // no-op + } + + @Override + public GetWorkBudget remainingBudget() { + return GetWorkBudget.builder() + .setBytes(request.getMaxBytes() - inflightBytes.get()) + .setItems(request.getMaxItems() - inflightMessages.get()) + .build(); + } + private class WorkItemBuffer { private final GetWorkTimingInfosTracker workTimingInfosTracker; private String computation; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GrpcGetWorkerMetadataStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetWorkerMetadataStream.java similarity index 93% rename from runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GrpcGetWorkerMetadataStream.java rename to runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetWorkerMetadataStream.java index 427fd412ec7f..a403feddb450 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GrpcGetWorkerMetadataStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetWorkerMetadataStream.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.dataflow.worker.windmill.grpcclient; +package org.apache.beam.runners.dataflow.worker.windmill.client.grpc; import com.google.errorprone.annotations.concurrent.GuardedBy; import java.io.PrintWriter; @@ -23,13 +23,14 @@ import java.util.Set; import java.util.function.Consumer; import java.util.function.Function; -import org.apache.beam.runners.dataflow.worker.windmill.AbstractWindmillStream; -import org.apache.beam.runners.dataflow.worker.windmill.StreamObserverFactory; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.JobHeader; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkerMetadataRequest; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkerMetadataResponse; import org.apache.beam.runners.dataflow.worker.windmill.WindmillEndpoints; -import org.apache.beam.runners.dataflow.worker.windmill.WindmillStream.GetWorkerMetadataStream; +import org.apache.beam.runners.dataflow.worker.windmill.client.AbstractWindmillStream; +import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.GetWorkerMetadataStream; +import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.observers.StreamObserverFactory; +import org.apache.beam.runners.dataflow.worker.windmill.client.throttling.ThrottleTimer; import org.apache.beam.sdk.util.BackOff; import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; import org.slf4j.Logger; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillServer.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillServer.java new file mode 100644 index 000000000000..3a881df71462 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillServer.java @@ -0,0 +1,355 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.dataflow.worker.windmill.client.grpc; + +import static org.apache.beam.runners.dataflow.worker.windmill.client.grpc.stubs.WindmillChannelFactory.LOCALHOST; +import static org.apache.beam.runners.dataflow.worker.windmill.client.grpc.stubs.WindmillChannelFactory.inProcessChannel; +import static org.apache.beam.runners.dataflow.worker.windmill.client.grpc.stubs.WindmillChannelFactory.localhostChannel; + +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; +import java.io.IOException; +import java.io.PrintWriter; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.function.Supplier; +import javax.annotation.Nullable; +import org.apache.beam.runners.dataflow.worker.options.StreamingDataflowWorkerOptions; +import org.apache.beam.runners.dataflow.worker.windmill.CloudWindmillServiceV1Alpha1Grpc; +import org.apache.beam.runners.dataflow.worker.windmill.CloudWindmillServiceV1Alpha1Grpc.CloudWindmillServiceV1Alpha1Stub; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.CommitWorkRequest; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.CommitWorkResponse; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GetConfigRequest; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GetConfigResponse; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GetDataRequest; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GetDataResponse; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GetWorkRequest; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GetWorkResponse; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.JobHeader; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.ReportStatsRequest; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.ReportStatsResponse; +import org.apache.beam.runners.dataflow.worker.windmill.WindmillApplianceGrpc; +import org.apache.beam.runners.dataflow.worker.windmill.WindmillServerStub; +import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.CommitWorkStream; +import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.GetDataStream; +import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.GetWorkStream; +import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.stubs.WindmillStubFactory; +import org.apache.beam.runners.dataflow.worker.windmill.client.throttling.StreamingEngineThrottleTimers; +import org.apache.beam.runners.dataflow.worker.windmill.work.WorkItemReceiver; +import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.util.BackOff; +import org.apache.beam.sdk.util.BackOffUtils; +import org.apache.beam.sdk.util.FluentBackoff; +import org.apache.beam.sdk.util.Sleeper; +import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.Channel; +import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ManagedChannel; +import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.StatusRuntimeException; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Splitter; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Sets; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.net.HostAndPort; +import org.joda.time.Duration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** gRPC client for communicating with Streaming Engine. */ +@SuppressFBWarnings({ + // Very likely real potentials for bugs. + "JLM_JSR166_UTILCONCURRENT_MONITORENTER", // https://github.com/apache/beam/issues/19273 + "IS2_INCONSISTENT_SYNC" // https://github.com/apache/beam/issues/19271 +}) +@SuppressWarnings("nullness") // TODO(https://github.com/apache/beam/issues/20497 +public final class GrpcWindmillServer extends WindmillServerStub { + private static final Logger LOG = LoggerFactory.getLogger(GrpcWindmillServer.class); + private static final int DEFAULT_LOG_EVERY_N_FAILURES = 20; + private static final Duration MIN_BACKOFF = Duration.millis(1); + private static final Duration MAX_BACKOFF = Duration.standardSeconds(30); + private static final int NO_HEALTH_CHECK = -1; + private static final String GRPC_LOCALHOST = "grpc:localhost"; + + private final GrpcWindmillStreamFactory windmillStreamFactory; + private final GrpcDispatcherClient dispatcherClient; + private final StreamingDataflowWorkerOptions options; + private final StreamingEngineThrottleTimers throttleTimers; + private Duration maxBackoff; + private @Nullable WindmillApplianceGrpc.WindmillApplianceBlockingStub syncApplianceStub; + + private GrpcWindmillServer( + StreamingDataflowWorkerOptions options, GrpcDispatcherClient grpcDispatcherClient) { + this.options = options; + this.throttleTimers = StreamingEngineThrottleTimers.create(); + this.maxBackoff = MAX_BACKOFF; + this.windmillStreamFactory = + GrpcWindmillStreamFactory.of( + JobHeader.newBuilder() + .setJobId(options.getJobId()) + .setProjectId(options.getProject()) + .setWorkerId(options.getWorkerId()) + .build()) + .setWindmillMessagesBetweenIsReadyChecks( + options.getWindmillMessagesBetweenIsReadyChecks()) + .setMaxBackOffSupplier(() -> maxBackoff) + .setLogEveryNStreamFailures( + options.getWindmillServiceStreamingLogEveryNStreamFailures()) + .setStreamingRpcBatchLimit(options.getWindmillServiceStreamingRpcBatchLimit()) + .build(); + windmillStreamFactory.scheduleHealthChecks( + options.getWindmillServiceStreamingRpcHealthCheckPeriodMs()); + + this.dispatcherClient = grpcDispatcherClient; + this.syncApplianceStub = null; + } + + private static StreamingDataflowWorkerOptions testOptions(boolean enableStreamingEngine) { + StreamingDataflowWorkerOptions options = + PipelineOptionsFactory.create().as(StreamingDataflowWorkerOptions.class); + options.setProject("project"); + options.setJobId("job"); + options.setWorkerId("worker"); + List<String> experiments = + options.getExperiments() == null ? new ArrayList<>() : options.getExperiments(); + if (enableStreamingEngine) { + experiments.add(GcpOptions.STREAMING_ENGINE_EXPERIMENT); + } + options.setExperiments(experiments); + + options.setWindmillServiceStreamingRpcBatchLimit(Integer.MAX_VALUE); + options.setWindmillServiceStreamingRpcHealthCheckPeriodMs(NO_HEALTH_CHECK); + options.setWindmillServiceStreamingLogEveryNStreamFailures(DEFAULT_LOG_EVERY_N_FAILURES); + + return options; + } + + /** Create new instance of {@link GrpcWindmillServer}. */ + public static GrpcWindmillServer create(StreamingDataflowWorkerOptions workerOptions) + throws IOException { + + GrpcWindmillServer grpcWindmillServer = + new GrpcWindmillServer( + workerOptions, + GrpcDispatcherClient.create( + WindmillStubFactory.remoteStubFactory( + workerOptions.getWindmillServiceRpcChannelAliveTimeoutSec(), + workerOptions.getGcpCredential()))); + if (workerOptions.getWindmillServiceEndpoint() != null) { + grpcWindmillServer.configureWindmillServiceEndpoints(); + } else if (!workerOptions.isEnableStreamingEngine() + && workerOptions.getLocalWindmillHostport() != null) { + grpcWindmillServer.configureLocalHost(); + } + + return grpcWindmillServer; + } + + @VisibleForTesting + static GrpcWindmillServer newTestInstance(String name) { + ManagedChannel inProcessChannel = inProcessChannel(name); + CloudWindmillServiceV1Alpha1Stub stub = + CloudWindmillServiceV1Alpha1Grpc.newStub(inProcessChannel); + List<CloudWindmillServiceV1Alpha1Stub> dispatcherStubs = Lists.newArrayList(stub); + Set<HostAndPort> dispatcherEndpoints = Sets.newHashSet(HostAndPort.fromHost(name)); + GrpcDispatcherClient dispatcherClient = + GrpcDispatcherClient.forTesting( + WindmillStubFactory.inProcessStubFactory(name, unused -> inProcessChannel), + dispatcherStubs, + dispatcherEndpoints); + return new GrpcWindmillServer(testOptions(/* enableStreamingEngine= */ true), dispatcherClient); + } + + @VisibleForTesting + static GrpcWindmillServer newApplianceTestInstance(Channel channel) { + GrpcWindmillServer testServer = + new GrpcWindmillServer( + testOptions(/* enableStreamingEngine= */ false), + // No-op, Appliance does not use Dispatcher to call Streaming Engine. + GrpcDispatcherClient.create(WindmillStubFactory.inProcessStubFactory("test"))); + testServer.syncApplianceStub = createWindmillApplianceStubWithDeadlineInterceptor(channel); + return testServer; + } + + private static WindmillApplianceGrpc.WindmillApplianceBlockingStub + createWindmillApplianceStubWithDeadlineInterceptor(Channel channel) { + return WindmillApplianceGrpc.newBlockingStub(channel) + .withInterceptors(GrpcDeadlineClientInterceptor.withDefaultUnaryRpcDeadline()); + } + + private static UnsupportedOperationException unsupportedUnaryRequestInStreamingEngineException( + String rpcName) { + return new UnsupportedOperationException( + String.format("Unary %s calls are not supported in Streaming Engine.", rpcName)); + } + + private void configureWindmillServiceEndpoints() { + Set<HostAndPort> endpoints = new HashSet<>(); + for (String endpoint : Splitter.on(',').split(options.getWindmillServiceEndpoint())) { + endpoints.add( + HostAndPort.fromString(endpoint).withDefaultPort(options.getWindmillServicePort())); + } + + dispatcherClient.consumeWindmillDispatcherEndpoints(ImmutableSet.copyOf(endpoints)); + } + + private void configureLocalHost() { + int portStart = options.getLocalWindmillHostport().lastIndexOf(':'); + String endpoint = options.getLocalWindmillHostport().substring(0, portStart); + Preconditions.checkState(GRPC_LOCALHOST.equals(endpoint)); + int port = Integer.parseInt(options.getLocalWindmillHostport().substring(portStart + 1)); + dispatcherClient.consumeWindmillDispatcherEndpoints( + ImmutableSet.of(HostAndPort.fromParts(LOCALHOST, port))); + initializeLocalHost(port); + } + + @Override + public void setWindmillServiceEndpoints(Set<HostAndPort> endpoints) { + dispatcherClient.consumeWindmillDispatcherEndpoints(ImmutableSet.copyOf(endpoints)); + } + + @Override + public boolean isReady() { + return dispatcherClient.isReady(); + } + + private synchronized void initializeLocalHost(int port) { + this.maxBackoff = Duration.millis(500); + if (options.isEnableStreamingEngine()) { + dispatcherClient.consumeWindmillDispatcherEndpoints( + ImmutableSet.of(HostAndPort.fromParts(LOCALHOST, port))); + } else { + this.syncApplianceStub = + createWindmillApplianceStubWithDeadlineInterceptor(localhostChannel(port)); + } + } + + @Override + public void appendSummaryHtml(PrintWriter writer) { + windmillStreamFactory.appendSummaryHtml(writer); + } + + private <ResponseT> ResponseT callWithBackoff(Supplier<ResponseT> function) { + // Configure backoff to retry calls forever, with a maximum sane retry interval. + BackOff backoff = + FluentBackoff.DEFAULT.withInitialBackoff(MIN_BACKOFF).withMaxBackoff(maxBackoff).backoff(); + + int rpcErrors = 0; + while (true) { + try { + return function.get(); + } catch (StatusRuntimeException e) { + try { + if (++rpcErrors % 20 == 0) { + LOG.warn( + "Many exceptions calling gRPC. Last exception: {} with status {}", + e, + e.getStatus()); + } + if (!BackOffUtils.next(Sleeper.DEFAULT, backoff)) { + throw new RpcException(e); + } + } catch (IOException | InterruptedException i) { + if (i instanceof InterruptedException) { + Thread.currentThread().interrupt(); + } + RpcException rpcException = new RpcException(e); + rpcException.addSuppressed(i); + throw rpcException; + } + } + } + } + + @Override + public GetWorkResponse getWork(GetWorkRequest request) { + if (syncApplianceStub != null) { + return callWithBackoff(() -> syncApplianceStub.getWork(request)); + } + + throw new RpcException(unsupportedUnaryRequestInStreamingEngineException("GetWork")); + } + + @Override + public GetDataResponse getData(GetDataRequest request) { + if (syncApplianceStub != null) { + return callWithBackoff(() -> syncApplianceStub.getData(request)); + } + + throw new RpcException(unsupportedUnaryRequestInStreamingEngineException("GetData")); + } + + @Override + public CommitWorkResponse commitWork(CommitWorkRequest request) { + if (syncApplianceStub != null) { + return callWithBackoff(() -> syncApplianceStub.commitWork(request)); + } + throw new RpcException(unsupportedUnaryRequestInStreamingEngineException("CommitWork")); + } + + @Override + public GetWorkStream getWorkStream(GetWorkRequest request, WorkItemReceiver receiver) { + return windmillStreamFactory.createGetWorkStream( + dispatcherClient.getDispatcherStub(), + GetWorkRequest.newBuilder(request) + .setJobId(options.getJobId()) + .setProjectId(options.getProject()) + .setWorkerId(options.getWorkerId()) + .build(), + throttleTimers.getWorkThrottleTimer(), + receiver); + } + + @Override + public GetDataStream getDataStream() { + return windmillStreamFactory.createGetDataStream( + dispatcherClient.getDispatcherStub(), throttleTimers.getDataThrottleTimer()); + } + + @Override + public CommitWorkStream commitWorkStream() { + return windmillStreamFactory.createCommitWorkStream( + dispatcherClient.getDispatcherStub(), throttleTimers.commitWorkThrottleTimer()); + } + + @Override + public GetConfigResponse getConfig(GetConfigRequest request) { + if (syncApplianceStub != null) { + return callWithBackoff(() -> syncApplianceStub.getConfig(request)); + } + + throw new RpcException( + new UnsupportedOperationException("GetConfig not supported in Streaming Engine.")); + } + + @Override + public ReportStatsResponse reportStats(ReportStatsRequest request) { + if (syncApplianceStub != null) { + return callWithBackoff(() -> syncApplianceStub.reportStats(request)); + } + + throw new RpcException( + new UnsupportedOperationException("ReportStats not supported in Streaming Engine.")); + } + + @Override + public long getAndResetThrottleTime() { + return throttleTimers.getAndResetThrottleTime(); + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillStreamFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillStreamFactory.java new file mode 100644 index 000000000000..e474ebf18b29 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillStreamFactory.java @@ -0,0 +1,227 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.dataflow.worker.windmill.client.grpc; + +import static org.apache.beam.runners.dataflow.worker.windmill.client.AbstractWindmillStream.DEFAULT_STREAM_RPC_DEADLINE_SECONDS; + +import com.google.auto.value.AutoBuilder; +import java.io.PrintWriter; +import java.util.Set; +import java.util.Timer; +import java.util.TimerTask; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Consumer; +import java.util.function.Supplier; +import javax.annotation.concurrent.ThreadSafe; +import org.apache.beam.runners.dataflow.worker.status.StatusDataProvider; +import org.apache.beam.runners.dataflow.worker.windmill.CloudWindmillServiceV1Alpha1Grpc.CloudWindmillServiceV1Alpha1Stub; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GetWorkRequest; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.JobHeader; +import org.apache.beam.runners.dataflow.worker.windmill.WindmillEndpoints; +import org.apache.beam.runners.dataflow.worker.windmill.client.AbstractWindmillStream; +import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.CommitWorkStream; +import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.GetDataStream; +import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.GetWorkStream; +import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.GetWorkerMetadataStream; +import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.observers.StreamObserverFactory; +import org.apache.beam.runners.dataflow.worker.windmill.client.throttling.ThrottleTimer; +import org.apache.beam.runners.dataflow.worker.windmill.work.WorkItemReceiver; +import org.apache.beam.sdk.util.BackOff; +import org.apache.beam.sdk.util.FluentBackoff; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Suppliers; +import org.joda.time.Duration; +import org.joda.time.Instant; + +/** + * Creates gRPC streaming connections to Windmill Service. Maintains a set of all currently opened + * RPC streams for health check/heartbeat requests to keep the streams alive. + */ +@ThreadSafe +public final class GrpcWindmillStreamFactory implements StatusDataProvider { + private static final Duration MIN_BACKOFF = Duration.millis(1); + private static final Duration DEFAULT_MAX_BACKOFF = Duration.standardSeconds(30); + private static final int DEFAULT_LOG_EVERY_N_STREAM_FAILURES = 1; + private static final int DEFAULT_STREAMING_RPC_BATCH_LIMIT = Integer.MAX_VALUE; + private static final int DEFAULT_WINDMILL_MESSAGES_BETWEEN_IS_READY_CHECKS = 1; + + private final JobHeader jobHeader; + private final int logEveryNStreamFailures; + private final int streamingRpcBatchLimit; + private final int windmillMessagesBetweenIsReadyChecks; + private final Supplier<BackOff> grpcBackOff; + private final Set<AbstractWindmillStream<?, ?>> streamRegistry; + private final AtomicLong streamIdGenerator; + + GrpcWindmillStreamFactory( + JobHeader jobHeader, + int logEveryNStreamFailures, + int streamingRpcBatchLimit, + int windmillMessagesBetweenIsReadyChecks, + Supplier<Duration> maxBackOffSupplier) { + this.jobHeader = jobHeader; + this.logEveryNStreamFailures = logEveryNStreamFailures; + this.streamingRpcBatchLimit = streamingRpcBatchLimit; + this.windmillMessagesBetweenIsReadyChecks = windmillMessagesBetweenIsReadyChecks; + // Configure backoff to retry calls forever, with a maximum sane retry interval. + this.grpcBackOff = + Suppliers.memoize( + () -> + FluentBackoff.DEFAULT + .withInitialBackoff(MIN_BACKOFF) + .withMaxBackoff(maxBackOffSupplier.get()) + .backoff()); + this.streamRegistry = ConcurrentHashMap.newKeySet(); + this.streamIdGenerator = new AtomicLong(); + } + + /** + * Returns a new {@link Builder} for {@link GrpcWindmillStreamFactory} with default values set for + * the given {@link JobHeader}. + */ + public static GrpcWindmillStreamFactory.Builder of(JobHeader jobHeader) { + return new AutoBuilder_GrpcWindmillStreamFactory_Builder() + .setJobHeader(jobHeader) + .setWindmillMessagesBetweenIsReadyChecks(DEFAULT_WINDMILL_MESSAGES_BETWEEN_IS_READY_CHECKS) + .setMaxBackOffSupplier(() -> DEFAULT_MAX_BACKOFF) + .setLogEveryNStreamFailures(DEFAULT_LOG_EVERY_N_STREAM_FAILURES) + .setStreamingRpcBatchLimit(DEFAULT_STREAMING_RPC_BATCH_LIMIT); + } + + private static CloudWindmillServiceV1Alpha1Stub withDeadline( + CloudWindmillServiceV1Alpha1Stub stub) { + // Deadlines are absolute points in time, so generate a new one everytime this function is + // called. + return stub.withDeadlineAfter( + AbstractWindmillStream.DEFAULT_STREAM_RPC_DEADLINE_SECONDS, TimeUnit.SECONDS); + } + + public GetWorkStream createGetWorkStream( + CloudWindmillServiceV1Alpha1Stub stub, + GetWorkRequest request, + ThrottleTimer getWorkThrottleTimer, + WorkItemReceiver processWorkItem) { + return GrpcGetWorkStream.create( + responseObserver -> withDeadline(stub).getWorkStream(responseObserver), + request, + grpcBackOff.get(), + newStreamObserverFactory(), + streamRegistry, + logEveryNStreamFailures, + getWorkThrottleTimer, + processWorkItem); + } + + public GetDataStream createGetDataStream( + CloudWindmillServiceV1Alpha1Stub stub, ThrottleTimer getDataThrottleTimer) { + return GrpcGetDataStream.create( + responseObserver -> withDeadline(stub).getDataStream(responseObserver), + grpcBackOff.get(), + newStreamObserverFactory(), + streamRegistry, + logEveryNStreamFailures, + getDataThrottleTimer, + jobHeader, + streamIdGenerator, + streamingRpcBatchLimit); + } + + public CommitWorkStream createCommitWorkStream( + CloudWindmillServiceV1Alpha1Stub stub, ThrottleTimer commitWorkThrottleTimer) { + return GrpcCommitWorkStream.create( + responseObserver -> withDeadline(stub).commitWorkStream(responseObserver), + grpcBackOff.get(), + newStreamObserverFactory(), + streamRegistry, + logEveryNStreamFailures, + commitWorkThrottleTimer, + jobHeader, + streamIdGenerator, + streamingRpcBatchLimit); + } + + public GetWorkerMetadataStream createGetWorkerMetadataStream( + CloudWindmillServiceV1Alpha1Stub stub, + ThrottleTimer getWorkerMetadataThrottleTimer, + Consumer<WindmillEndpoints> onNewWindmillEndpoints) { + return GrpcGetWorkerMetadataStream.create( + responseObserver -> withDeadline(stub).getWorkerMetadataStream(responseObserver), + grpcBackOff.get(), + newStreamObserverFactory(), + streamRegistry, + logEveryNStreamFailures, + jobHeader, + 0, + getWorkerMetadataThrottleTimer, + onNewWindmillEndpoints); + } + + private StreamObserverFactory newStreamObserverFactory() { + return StreamObserverFactory.direct( + DEFAULT_STREAM_RPC_DEADLINE_SECONDS * 2, windmillMessagesBetweenIsReadyChecks); + } + + /** + * Schedules streaming RPC health checks to run on a background daemon thread, which will be + * cleaned up when the JVM shutdown. + */ + public void scheduleHealthChecks(int healthCheckInterval) { + if (healthCheckInterval < 0) { + return; + } + + new Timer("WindmillHealthCheckTimer") + .schedule( + new TimerTask() { + @Override + public void run() { + Instant reportThreshold = Instant.now().minus(Duration.millis(healthCheckInterval)); + for (AbstractWindmillStream<?, ?> stream : streamRegistry) { + stream.maybeSendHealthCheck(reportThreshold); + } + } + }, + 0, + healthCheckInterval); + } + + @Override + public void appendSummaryHtml(PrintWriter writer) { + writer.write("Active Streams:<br>"); + for (AbstractWindmillStream<?, ?> stream : streamRegistry) { + stream.appendSummaryHtml(writer); + writer.write("<br>"); + } + } + + @AutoBuilder(ofClass = GrpcWindmillStreamFactory.class) + interface Builder { + Builder setJobHeader(JobHeader jobHeader); + + Builder setLogEveryNStreamFailures(int logEveryNStreamFailures); + + Builder setStreamingRpcBatchLimit(int streamingRpcBatchLimit); + + Builder setWindmillMessagesBetweenIsReadyChecks(int windmillMessagesBetweenIsReadyChecks); + + Builder setMaxBackOffSupplier(Supplier<Duration> maxBackOff); + + GrpcWindmillStreamFactory build(); + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/auth/VendoredCredentialsAdapter.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/auth/VendoredCredentialsAdapter.java new file mode 100644 index 000000000000..23f6fb801a4f --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/auth/VendoredCredentialsAdapter.java @@ -0,0 +1,81 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.dataflow.worker.windmill.client.grpc.auth; + +import java.io.IOException; +import java.net.URI; +import java.util.List; +import java.util.Map; +import java.util.concurrent.Executor; + +/** + * Create a wrapper around credentials that delegates to the underlying {@link + * com.google.auth.Credentials}. Note that this class should override every method that is not final + * and not static and call the delegate directly. + * + * <p>TODO: Replace this with an auto generated proxy which calls the underlying implementation + * delegate to reduce maintenance burden. + */ +public class VendoredCredentialsAdapter + extends org.apache.beam.vendor.grpc.v1p54p0.com.google.auth.Credentials { + + private final com.google.auth.Credentials credentials; + + public VendoredCredentialsAdapter(com.google.auth.Credentials credentials) { + this.credentials = credentials; + } + + @Override + public String getAuthenticationType() { + return credentials.getAuthenticationType(); + } + + @Override + public Map<String, List<String>> getRequestMetadata() throws IOException { + return credentials.getRequestMetadata(); + } + + @Override + public void getRequestMetadata( + final URI uri, + Executor executor, + final org.apache.beam.vendor.grpc.v1p54p0.com.google.auth.RequestMetadataCallback callback) { + credentials.getRequestMetadata( + uri, executor, new VendoredRequestMetadataCallbackAdapter(callback)); + } + + @Override + public Map<String, List<String>> getRequestMetadata(URI uri) throws IOException { + return credentials.getRequestMetadata(uri); + } + + @Override + public boolean hasRequestMetadata() { + return credentials.hasRequestMetadata(); + } + + @Override + public boolean hasRequestMetadataOnly() { + return credentials.hasRequestMetadataOnly(); + } + + @Override + public void refresh() throws IOException { + credentials.refresh(); + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/auth/VendoredRequestMetadataCallbackAdapter.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/auth/VendoredRequestMetadataCallbackAdapter.java new file mode 100644 index 000000000000..8b1b695287e7 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/auth/VendoredRequestMetadataCallbackAdapter.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.dataflow.worker.windmill.client.grpc.auth; + +import java.util.List; +import java.util.Map; + +/** + * Create a wrapper around credentials callback that delegates to the underlying vendored {@link + * com.google.auth.RequestMetadataCallback}. Note that this class should override every method that + * is not final and not static and call the delegate directly. + * + * <p>TODO: Replace this with an auto generated proxy which calls the underlying implementation + * delegate to reduce maintenance burden. + */ +public class VendoredRequestMetadataCallbackAdapter + implements com.google.auth.RequestMetadataCallback { + + private final org.apache.beam.vendor.grpc.v1p54p0.com.google.auth.RequestMetadataCallback + callback; + + VendoredRequestMetadataCallbackAdapter( + org.apache.beam.vendor.grpc.v1p54p0.com.google.auth.RequestMetadataCallback callback) { + this.callback = callback; + } + + @Override + public void onSuccess(Map<String, List<String>> metadata) { + callback.onSuccess(metadata); + } + + @Override + public void onFailure(Throwable exception) { + callback.onFailure(exception); + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/DirectStreamObserver.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/DirectStreamObserver.java similarity index 98% rename from runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/DirectStreamObserver.java rename to runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/DirectStreamObserver.java index 3c7798126e59..5fb22476ab3a 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/DirectStreamObserver.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/DirectStreamObserver.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.dataflow.worker.windmill; +package org.apache.beam.runners.dataflow.worker.windmill.client.grpc.observers; import java.util.concurrent.Phaser; import java.util.concurrent.TimeUnit; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/ForwardingClientResponseObserver.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/ForwardingClientResponseObserver.java similarity index 96% rename from runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/ForwardingClientResponseObserver.java rename to runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/ForwardingClientResponseObserver.java index a1f80598d89a..007717d03b58 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/ForwardingClientResponseObserver.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/ForwardingClientResponseObserver.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.dataflow.worker.windmill; +package org.apache.beam.runners.dataflow.worker.windmill.client.grpc.observers; import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.ClientCallStreamObserver; import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.ClientResponseObserver; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/StreamObserverFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/StreamObserverFactory.java similarity index 97% rename from runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/StreamObserverFactory.java rename to runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/StreamObserverFactory.java index e0878b7b0b91..e3f12687638d 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/StreamObserverFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/StreamObserverFactory.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.dataflow.worker.windmill; +package org.apache.beam.runners.dataflow.worker.windmill.client.grpc.observers; import java.util.function.Function; import org.apache.beam.sdk.fn.stream.AdvancingPhaser; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/stubs/WindmillChannelFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/stubs/WindmillChannelFactory.java new file mode 100644 index 000000000000..48cf8ff3f761 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/stubs/WindmillChannelFactory.java @@ -0,0 +1,137 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.dataflow.worker.windmill.client.grpc.stubs; + +import java.net.Inet6Address; +import java.net.InetSocketAddress; +import java.util.concurrent.TimeUnit; +import javax.net.ssl.SSLException; +import org.apache.beam.runners.dataflow.worker.windmill.WindmillServiceAddress; +import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.Channel; +import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ManagedChannel; +import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.inprocess.InProcessChannelBuilder; +import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.netty.GrpcSslContexts; +import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.netty.NegotiationType; +import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.netty.NettyChannelBuilder; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.net.HostAndPort; + +/** Utility class used to create different RPC Channels. */ +public final class WindmillChannelFactory { + public static final String LOCALHOST = "localhost"; + private static final int DEFAULT_GRPC_PORT = 443; + + private WindmillChannelFactory() {} + + public static ManagedChannel inProcessChannel(String channelName) { + return InProcessChannelBuilder.forName(channelName).directExecutor().build(); + } + + public static Channel localhostChannel(int port) { + return NettyChannelBuilder.forAddress(LOCALHOST, port) + .maxInboundMessageSize(Integer.MAX_VALUE) + .negotiationType(NegotiationType.PLAINTEXT) + .build(); + } + + static Channel remoteChannel( + WindmillServiceAddress windmillServiceAddress, int windmillServiceRpcChannelTimeoutSec) { + switch (windmillServiceAddress.getKind()) { + case IPV6: + return remoteChannel(windmillServiceAddress.ipv6(), windmillServiceRpcChannelTimeoutSec); + case GCP_SERVICE_ADDRESS: + return remoteChannel( + windmillServiceAddress.gcpServiceAddress(), windmillServiceRpcChannelTimeoutSec); + // switch is exhaustive will never happen. + default: + throw new UnsupportedOperationException( + "Only IPV6 and GCP_SERVICE_ADDRESS are supported WindmillServiceAddresses."); + } + } + + public static Channel remoteChannel( + HostAndPort endpoint, int windmillServiceRpcChannelTimeoutSec) { + try { + return createRemoteChannel( + NettyChannelBuilder.forAddress(endpoint.getHost(), endpoint.getPort()), + windmillServiceRpcChannelTimeoutSec); + } catch (SSLException sslException) { + throw new WindmillChannelCreationException(endpoint, sslException); + } + } + + public static Channel remoteChannel( + Inet6Address directEndpoint, int port, int windmillServiceRpcChannelTimeoutSec) { + try { + return createRemoteChannel( + NettyChannelBuilder.forAddress(new InetSocketAddress(directEndpoint, port)), + windmillServiceRpcChannelTimeoutSec); + } catch (SSLException sslException) { + throw new WindmillChannelCreationException(directEndpoint.toString(), sslException); + } + } + + public static Channel remoteChannel( + Inet6Address directEndpoint, int windmillServiceRpcChannelTimeoutSec) { + try { + return createRemoteChannel( + NettyChannelBuilder.forAddress(new InetSocketAddress(directEndpoint, DEFAULT_GRPC_PORT)), + windmillServiceRpcChannelTimeoutSec); + } catch (SSLException sslException) { + throw new WindmillChannelCreationException(directEndpoint.toString(), sslException); + } + } + + @SuppressWarnings("nullness") + private static Channel createRemoteChannel( + NettyChannelBuilder channelBuilder, int windmillServiceRpcChannelTimeoutSec) + throws SSLException { + if (windmillServiceRpcChannelTimeoutSec > 0) { + channelBuilder + .keepAliveTime(windmillServiceRpcChannelTimeoutSec, TimeUnit.SECONDS) + .keepAliveTimeout(windmillServiceRpcChannelTimeoutSec, TimeUnit.SECONDS) + .keepAliveWithoutCalls(true); + } + + return channelBuilder + .flowControlWindow(10 * 1024 * 1024) + .maxInboundMessageSize(Integer.MAX_VALUE) + .maxInboundMetadataSize(1024 * 1024) + .negotiationType(NegotiationType.TLS) + // Set ciphers(null) to not use GCM, which is disabled for Dataflow + // due to it being horribly slow. + .sslContext(GrpcSslContexts.forClient().ciphers(null).build()) + .build(); + } + + public static class WindmillChannelCreationException extends IllegalStateException { + private WindmillChannelCreationException(HostAndPort endpoint, SSLException sourceException) { + super( + String.format( + "Exception thrown when trying to create channel to endpoint={host:%s; port:%d}", + endpoint.getHost(), endpoint.getPort()), + sourceException); + } + + WindmillChannelCreationException(String directEndpoint, Throwable sourceException) { + super( + String.format( + "Exception thrown when trying to create channel to endpoint={%s}", directEndpoint), + sourceException); + } + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/stubs/WindmillStubFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/stubs/WindmillStubFactory.java new file mode 100644 index 000000000000..0c7719b0bc13 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/stubs/WindmillStubFactory.java @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.dataflow.worker.windmill.client.grpc.stubs; + +import static org.apache.beam.runners.dataflow.worker.windmill.client.grpc.stubs.WindmillChannelFactory.remoteChannel; + +import com.google.auth.Credentials; +import com.google.auto.value.AutoOneOf; +import java.util.function.Function; +import java.util.function.Supplier; +import org.apache.beam.runners.dataflow.worker.windmill.CloudWindmillServiceV1Alpha1Grpc; +import org.apache.beam.runners.dataflow.worker.windmill.CloudWindmillServiceV1Alpha1Grpc.CloudWindmillServiceV1Alpha1Stub; +import org.apache.beam.runners.dataflow.worker.windmill.WindmillServiceAddress; +import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.auth.VendoredCredentialsAdapter; +import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ManagedChannel; +import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.auth.MoreCallCredentials; + +/** + * Used to create stubs to talk to Streaming Engine. Stubs are either in-process for testing, or + * remote. + */ +@AutoOneOf(WindmillStubFactory.Kind.class) +public abstract class WindmillStubFactory { + + public static WindmillStubFactory inProcessStubFactory( + String testName, Function<String, ManagedChannel> channelFactory) { + return AutoOneOf_WindmillStubFactory.inProcess( + () -> CloudWindmillServiceV1Alpha1Grpc.newStub(channelFactory.apply(testName))); + } + + public static WindmillStubFactory inProcessStubFactory(String testName) { + return AutoOneOf_WindmillStubFactory.inProcess( + () -> + CloudWindmillServiceV1Alpha1Grpc.newStub( + WindmillChannelFactory.inProcessChannel(testName))); + } + + public static WindmillStubFactory remoteStubFactory( + int rpcChannelTimeoutSec, Credentials gcpCredentials) { + return AutoOneOf_WindmillStubFactory.remote( + directEndpoint -> + CloudWindmillServiceV1Alpha1Grpc.newStub( + remoteChannel(directEndpoint, rpcChannelTimeoutSec)) + .withCallCredentials( + MoreCallCredentials.from(new VendoredCredentialsAdapter(gcpCredentials)))); + } + + public abstract Kind getKind(); + + public abstract Supplier<CloudWindmillServiceV1Alpha1Stub> inProcess(); + + public abstract Function<WindmillServiceAddress, CloudWindmillServiceV1Alpha1Stub> remote(); + + public enum Kind { + IN_PROCESS, + REMOTE + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/throttling/StreamingEngineThrottleTimers.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/throttling/StreamingEngineThrottleTimers.java new file mode 100644 index 000000000000..6b8dd2720374 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/throttling/StreamingEngineThrottleTimers.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.dataflow.worker.windmill.client.throttling; + +import com.google.auto.value.AutoValue; + +@AutoValue +public abstract class StreamingEngineThrottleTimers { + + public static StreamingEngineThrottleTimers create() { + return new AutoValue_StreamingEngineThrottleTimers( + new ThrottleTimer(), new ThrottleTimer(), new ThrottleTimer()); + } + + public long getAndResetThrottleTime() { + return getWorkThrottleTimer().getAndResetThrottleTime() + + getDataThrottleTimer().getAndResetThrottleTime() + + commitWorkThrottleTimer().getAndResetThrottleTime(); + } + + public abstract ThrottleTimer getWorkThrottleTimer(); + + public abstract ThrottleTimer getDataThrottleTimer(); + + public abstract ThrottleTimer commitWorkThrottleTimer(); +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/ThrottleTimer.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/throttling/ThrottleTimer.java similarity index 94% rename from runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/ThrottleTimer.java rename to runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/throttling/ThrottleTimer.java index 237339aff399..f660112721ba 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/ThrottleTimer.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/throttling/ThrottleTimer.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.dataflow.worker.windmill.grpcclient; +package org.apache.beam.runners.dataflow.worker.windmill.client.throttling; import org.joda.time.Instant; @@ -25,7 +25,7 @@ * CommitWork are both blocked for x, totalTime will be 2x. However, if 2 GetWork streams are both * blocked for x totalTime will be x. All methods are thread safe. */ -class ThrottleTimer { +public final class ThrottleTimer { // This is -1 if not currently being throttled or the time in // milliseconds when throttling for this type started. private long startTime = -1; @@ -36,7 +36,7 @@ class ThrottleTimer { /** * Starts the timer if it has not been started and does nothing if it has already been started. */ - synchronized void start() { + public synchronized void start() { if (!throttled()) { // This timer is not started yet so start it now. startTime = Instant.now().getMillis(); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GrpcWindmillServer.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GrpcWindmillServer.java deleted file mode 100644 index 19cb90297df5..000000000000 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GrpcWindmillServer.java +++ /dev/null @@ -1,607 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.runners.dataflow.worker.windmill.grpcclient; - -import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; -import java.io.IOException; -import java.io.PrintWriter; -import java.net.URI; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Random; -import java.util.Set; -import java.util.Timer; -import java.util.TimerTask; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.Executor; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicLong; -import java.util.function.Supplier; -import org.apache.beam.runners.dataflow.worker.options.StreamingDataflowWorkerOptions; -import org.apache.beam.runners.dataflow.worker.windmill.AbstractWindmillStream; -import org.apache.beam.runners.dataflow.worker.windmill.CloudWindmillServiceV1Alpha1Grpc; -import org.apache.beam.runners.dataflow.worker.windmill.StreamObserverFactory; -import org.apache.beam.runners.dataflow.worker.windmill.Windmill.CommitWorkRequest; -import org.apache.beam.runners.dataflow.worker.windmill.Windmill.CommitWorkResponse; -import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GetConfigRequest; -import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GetConfigResponse; -import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GetDataRequest; -import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GetDataResponse; -import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GetWorkRequest; -import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GetWorkResponse; -import org.apache.beam.runners.dataflow.worker.windmill.Windmill.JobHeader; -import org.apache.beam.runners.dataflow.worker.windmill.Windmill.ReportStatsRequest; -import org.apache.beam.runners.dataflow.worker.windmill.Windmill.ReportStatsResponse; -import org.apache.beam.runners.dataflow.worker.windmill.WindmillApplianceGrpc; -import org.apache.beam.runners.dataflow.worker.windmill.WindmillServerStub; -import org.apache.beam.runners.dataflow.worker.windmill.WindmillStream.CommitWorkStream; -import org.apache.beam.runners.dataflow.worker.windmill.WindmillStream.GetDataStream; -import org.apache.beam.runners.dataflow.worker.windmill.WindmillStream.GetWorkStream; -import org.apache.beam.runners.dataflow.worker.windmill.WindmillStream.GetWorkStream.WorkItemReceiver; -import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; -import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.util.BackOff; -import org.apache.beam.sdk.util.BackOffUtils; -import org.apache.beam.sdk.util.FluentBackoff; -import org.apache.beam.sdk.util.Sleeper; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.Channel; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.StatusRuntimeException; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.auth.MoreCallCredentials; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.inprocess.InProcessChannelBuilder; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.netty.GrpcSslContexts; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.netty.NegotiationType; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.netty.NettyChannelBuilder; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Splitter; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.net.HostAndPort; -import org.joda.time.Duration; -import org.joda.time.Instant; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** gRPC client for communicating with Streaming Engine. */ -// Very likely real potential for bugs - https://github.com/apache/beam/issues/19273 -// Very likely real potential for bugs - https://github.com/apache/beam/issues/19271 -@SuppressFBWarnings({"JLM_JSR166_UTILCONCURRENT_MONITORENTER", "IS2_INCONSISTENT_SYNC"}) -@SuppressWarnings({ - "nullness" // TODO(https://github.com/apache/beam/issues/20497) -}) -public final class GrpcWindmillServer extends WindmillServerStub { - private static final Logger LOG = LoggerFactory.getLogger(GrpcWindmillServer.class); - - // If a connection cannot be established, gRPC will fail fast so this deadline can be relatively - // high. - private static final long DEFAULT_STREAM_RPC_DEADLINE_SECONDS = 300; - private static final int DEFAULT_LOG_EVERY_N_FAILURES = 20; - private static final String LOCALHOST = "localhost"; - private static final Duration MIN_BACKOFF = Duration.millis(1); - private static final Duration MAX_BACKOFF = Duration.standardSeconds(30); - private static final AtomicLong nextId = new AtomicLong(0); - private static final int NO_HEALTH_CHECK = -1; - - private final StreamingDataflowWorkerOptions options; - private final int streamingRpcBatchLimit; - private final List<CloudWindmillServiceV1Alpha1Grpc.CloudWindmillServiceV1Alpha1Stub> stubList; - private final ThrottleTimer getWorkThrottleTimer; - private final ThrottleTimer getDataThrottleTimer; - private final ThrottleTimer commitWorkThrottleTimer; - private final Random rand; - private final Set<AbstractWindmillStream<?, ?>> streamRegistry; - private ImmutableSet<HostAndPort> endpoints; - private int logEveryNStreamFailures; - private Duration maxBackoff = MAX_BACKOFF; - private WindmillApplianceGrpc.WindmillApplianceBlockingStub syncApplianceStub = null; - - private GrpcWindmillServer(StreamingDataflowWorkerOptions options) { - this.options = options; - this.streamingRpcBatchLimit = options.getWindmillServiceStreamingRpcBatchLimit(); - this.stubList = new ArrayList<>(); - this.logEveryNStreamFailures = options.getWindmillServiceStreamingLogEveryNStreamFailures(); - this.endpoints = ImmutableSet.of(); - this.getWorkThrottleTimer = new ThrottleTimer(); - this.getDataThrottleTimer = new ThrottleTimer(); - this.commitWorkThrottleTimer = new ThrottleTimer(); - this.rand = new Random(); - this.streamRegistry = Collections.newSetFromMap(new ConcurrentHashMap<>()); - } - - private static StreamingDataflowWorkerOptions testOptions(boolean enableStreamingEngine) { - StreamingDataflowWorkerOptions options = - PipelineOptionsFactory.create().as(StreamingDataflowWorkerOptions.class); - options.setProject("project"); - options.setJobId("job"); - options.setWorkerId("worker"); - List<String> experiments = - options.getExperiments() == null ? new ArrayList<>() : options.getExperiments(); - if (enableStreamingEngine) { - experiments.add(GcpOptions.STREAMING_ENGINE_EXPERIMENT); - } - options.setExperiments(experiments); - - options.setWindmillServiceStreamingRpcBatchLimit(Integer.MAX_VALUE); - options.setWindmillServiceStreamingRpcHealthCheckPeriodMs(NO_HEALTH_CHECK); - options.setWindmillServiceStreamingLogEveryNStreamFailures(DEFAULT_LOG_EVERY_N_FAILURES); - - return options; - } - - /** Create new instance of {@link GrpcWindmillServer}. */ - public static GrpcWindmillServer create(StreamingDataflowWorkerOptions workerOptions) - throws IOException { - GrpcWindmillServer grpcWindmillServer = new GrpcWindmillServer(workerOptions); - if (workerOptions.getWindmillServiceEndpoint() != null) { - grpcWindmillServer.configureWindmillServiceEndpoints(); - } else if (!workerOptions.isEnableStreamingEngine() - && workerOptions.getLocalWindmillHostport() != null) { - grpcWindmillServer.configureLocalHost(); - } - - if (workerOptions.getWindmillServiceStreamingRpcHealthCheckPeriodMs() > 0) { - grpcWindmillServer.scheduleHealthCheckTimer( - workerOptions, () -> grpcWindmillServer.streamRegistry); - } - - return grpcWindmillServer; - } - - @VisibleForTesting - static GrpcWindmillServer newTestInstance(String name) { - GrpcWindmillServer testServer = - new GrpcWindmillServer(testOptions(/* enableStreamingEngine= */ true)); - testServer.stubList.add(CloudWindmillServiceV1Alpha1Grpc.newStub(inProcessChannel(name))); - return testServer; - } - - @VisibleForTesting - static GrpcWindmillServer newApplianceTestInstance(Channel channel) { - GrpcWindmillServer testServer = - new GrpcWindmillServer(testOptions(/* enableStreamingEngine= */ false)); - testServer.syncApplianceStub = createWindmillApplianceStubWithDeadlineInterceptor(channel); - return testServer; - } - - private static WindmillApplianceGrpc.WindmillApplianceBlockingStub - createWindmillApplianceStubWithDeadlineInterceptor(Channel channel) { - return WindmillApplianceGrpc.newBlockingStub(channel) - .withInterceptors(GrpcDeadlineClientInterceptor.withDefaultUnaryRpcDeadline()); - } - - private static Channel inProcessChannel(String name) { - return InProcessChannelBuilder.forName(name).directExecutor().build(); - } - - private static Channel localhostChannel(int port) { - return NettyChannelBuilder.forAddress(LOCALHOST, port) - .maxInboundMessageSize(Integer.MAX_VALUE) - .negotiationType(NegotiationType.PLAINTEXT) - .build(); - } - - private static UnsupportedOperationException unsupportedUnaryRequestInStreamingEngineException( - String rpcName) { - return new UnsupportedOperationException( - String.format("Unary %s calls are not supported in Streaming Engine.", rpcName)); - } - - private void scheduleHealthCheckTimer( - StreamingDataflowWorkerOptions options, Supplier<Set<AbstractWindmillStream<?, ?>>> streams) { - new Timer("WindmillHealthCheckTimer") - .schedule( - new HealthCheckTimerTask(options, streams), - 0, - options.getWindmillServiceStreamingRpcHealthCheckPeriodMs()); - } - - private void configureWindmillServiceEndpoints() throws IOException { - Set<HostAndPort> endpoints = new HashSet<>(); - for (String endpoint : Splitter.on(',').split(options.getWindmillServiceEndpoint())) { - endpoints.add( - HostAndPort.fromString(endpoint).withDefaultPort(options.getWindmillServicePort())); - } - initializeWindmillService(endpoints); - } - - private void configureLocalHost() { - int portStart = options.getLocalWindmillHostport().lastIndexOf(':'); - String endpoint = options.getLocalWindmillHostport().substring(0, portStart); - assert ("grpc:localhost".equals(endpoint)); - int port = Integer.parseInt(options.getLocalWindmillHostport().substring(portStart + 1)); - this.endpoints = ImmutableSet.of(HostAndPort.fromParts(LOCALHOST, port)); - initializeLocalHost(port); - } - - @Override - public synchronized void setWindmillServiceEndpoints(Set<HostAndPort> endpoints) - throws IOException { - Preconditions.checkNotNull(endpoints); - if (endpoints.equals(this.endpoints)) { - // The endpoints are equal don't recreate the stubs. - return; - } - LOG.info("Creating a new windmill stub, endpoints: {}", endpoints); - if (this.endpoints != null) { - LOG.info("Previous windmill stub endpoints: {}", this.endpoints); - } - initializeWindmillService(endpoints); - } - - @Override - public synchronized boolean isReady() { - return !stubList.isEmpty(); - } - - private synchronized void initializeLocalHost(int port) { - this.logEveryNStreamFailures = 1; - this.maxBackoff = Duration.millis(500); - Channel channel = localhostChannel(port); - if (options.isEnableStreamingEngine()) { - this.stubList.add(CloudWindmillServiceV1Alpha1Grpc.newStub(channel)); - } else { - this.syncApplianceStub = createWindmillApplianceStubWithDeadlineInterceptor(channel); - } - } - - private synchronized void initializeWindmillService(Set<HostAndPort> endpoints) - throws IOException { - LOG.info("Initializing Streaming Engine GRPC client for endpoints: {}", endpoints); - this.stubList.clear(); - this.endpoints = ImmutableSet.copyOf(endpoints); - for (HostAndPort endpoint : this.endpoints) { - if (LOCALHOST.equals(endpoint.getHost())) { - initializeLocalHost(endpoint.getPort()); - } else { - this.stubList.add( - CloudWindmillServiceV1Alpha1Grpc.newStub(remoteChannel(endpoint)) - .withCallCredentials( - MoreCallCredentials.from( - new VendoredCredentialsAdapter(options.getGcpCredential())))); - } - } - } - - private Channel remoteChannel(HostAndPort endpoint) throws IOException { - NettyChannelBuilder builder = - NettyChannelBuilder.forAddress(endpoint.getHost(), endpoint.getPort()); - int timeoutSec = options.getWindmillServiceRpcChannelAliveTimeoutSec(); - if (timeoutSec > 0) { - builder - .keepAliveTime(timeoutSec, TimeUnit.SECONDS) - .keepAliveTimeout(timeoutSec, TimeUnit.SECONDS) - .keepAliveWithoutCalls(true); - } - return builder - .flowControlWindow(10 * 1024 * 1024) - .maxInboundMessageSize(Integer.MAX_VALUE) - .maxInboundMetadataSize(1024 * 1024) - .negotiationType(NegotiationType.TLS) - // Set ciphers(null) to not use GCM, which is disabled for Dataflow - // due to it being horribly slow. - .sslContext(GrpcSslContexts.forClient().ciphers(null).build()) - .build(); - } - - /** - * Stubs returned from this method do not (and should not) have {@link - * org.apache.beam.vendor.grpc.v1p54p0.io.grpc.Deadline}(s) set since they represent an absolute - * point in time. {@link org.apache.beam.vendor.grpc.v1p54p0.io.grpc.Deadline}(s) should not be - * treated as a timeout which represents a relative point in time. - * - * @see <a href=https://grpc.io/blog/deadlines/>Official gRPC deadline documentation for more - * details.</a> - */ - private synchronized CloudWindmillServiceV1Alpha1Grpc.CloudWindmillServiceV1Alpha1Stub stub() { - if (stubList.isEmpty()) { - throw new RuntimeException("windmillServiceEndpoint has not been set"); - } - - return stubList.size() == 1 ? stubList.get(0) : stubList.get(rand.nextInt(stubList.size())); - } - - @Override - public void appendSummaryHtml(PrintWriter writer) { - writer.write("Active Streams:<br>"); - for (AbstractWindmillStream<?, ?> stream : streamRegistry) { - stream.appendSummaryHtml(writer); - writer.write("<br>"); - } - } - - // Configure backoff to retry calls forever, with a maximum sane retry interval. - private BackOff grpcBackoff() { - return FluentBackoff.DEFAULT - .withInitialBackoff(MIN_BACKOFF) - .withMaxBackoff(maxBackoff) - .backoff(); - } - - private <ResponseT> ResponseT callWithBackoff(Supplier<ResponseT> function) { - BackOff backoff = grpcBackoff(); - int rpcErrors = 0; - while (true) { - try { - return function.get(); - } catch (StatusRuntimeException e) { - try { - if (++rpcErrors % 20 == 0) { - LOG.warn( - "Many exceptions calling gRPC. Last exception: {} with status {}", - e, - e.getStatus()); - } - if (!BackOffUtils.next(Sleeper.DEFAULT, backoff)) { - throw new RpcException(e); - } - } catch (IOException | InterruptedException i) { - if (i instanceof InterruptedException) { - Thread.currentThread().interrupt(); - } - RpcException rpcException = new RpcException(e); - rpcException.addSuppressed(i); - throw rpcException; - } - } - } - } - - @Override - public GetWorkResponse getWork(GetWorkRequest request) { - if (syncApplianceStub != null) { - return callWithBackoff(() -> syncApplianceStub.getWork(request)); - } - - throw new RpcException(unsupportedUnaryRequestInStreamingEngineException("GetWork")); - } - - @Override - public GetDataResponse getData(GetDataRequest request) { - if (syncApplianceStub != null) { - return callWithBackoff(() -> syncApplianceStub.getData(request)); - } - - throw new RpcException(unsupportedUnaryRequestInStreamingEngineException("GetData")); - } - - @Override - public CommitWorkResponse commitWork(CommitWorkRequest request) { - if (syncApplianceStub != null) { - return callWithBackoff(() -> syncApplianceStub.commitWork(request)); - } - throw new RpcException(unsupportedUnaryRequestInStreamingEngineException("CommitWork")); - } - - private StreamObserverFactory newStreamObserverFactory() { - return StreamObserverFactory.direct( - DEFAULT_STREAM_RPC_DEADLINE_SECONDS * 2, options.getWindmillMessagesBetweenIsReadyChecks()); - } - - @Override - public GetWorkStream getWorkStream(GetWorkRequest request, WorkItemReceiver receiver) { - GetWorkRequest getWorkRequest = - GetWorkRequest.newBuilder(request) - .setJobId(options.getJobId()) - .setProjectId(options.getProject()) - .setWorkerId(options.getWorkerId()) - .build(); - - return GrpcGetWorkStream.create( - responseObserver -> - stub() - // Deadlines are absolute points in time, so generate a new one everytime this - // function is called. - .withDeadlineAfter( - AbstractWindmillStream.DEFAULT_STREAM_RPC_DEADLINE_SECONDS, TimeUnit.SECONDS) - .getWorkStream(responseObserver), - getWorkRequest, - grpcBackoff(), - newStreamObserverFactory(), - streamRegistry, - logEveryNStreamFailures, - getWorkThrottleTimer, - receiver); - } - - @Override - public GetDataStream getDataStream() { - return GrpcGetDataStream.create( - responseObserver -> - stub() - // Deadlines are absolute points in time, so generate a new one everytime this - // function is called. - .withDeadlineAfter( - AbstractWindmillStream.DEFAULT_STREAM_RPC_DEADLINE_SECONDS, TimeUnit.SECONDS) - .getDataStream(responseObserver), - grpcBackoff(), - newStreamObserverFactory(), - streamRegistry, - logEveryNStreamFailures, - getDataThrottleTimer, - makeHeader(), - nextId, - streamingRpcBatchLimit); - } - - @Override - public CommitWorkStream commitWorkStream() { - return GrpcCommitWorkStream.create( - responseObserver -> - stub() - // Deadlines are absolute points in time, so generate a new one everytime this - // function is called. - .withDeadlineAfter( - AbstractWindmillStream.DEFAULT_STREAM_RPC_DEADLINE_SECONDS, TimeUnit.SECONDS) - .commitWorkStream(responseObserver), - grpcBackoff(), - newStreamObserverFactory(), - streamRegistry, - logEveryNStreamFailures, - commitWorkThrottleTimer, - makeHeader(), - nextId, - streamingRpcBatchLimit); - } - - @Override - public GetConfigResponse getConfig(GetConfigRequest request) { - if (syncApplianceStub != null) { - return callWithBackoff(() -> syncApplianceStub.getConfig(request)); - } - - throw new RpcException( - new UnsupportedOperationException("GetConfig not supported in Streaming Engine.")); - } - - @Override - public ReportStatsResponse reportStats(ReportStatsRequest request) { - if (syncApplianceStub != null) { - return callWithBackoff(() -> syncApplianceStub.reportStats(request)); - } - - throw new RpcException( - new UnsupportedOperationException("ReportStats not supported in Streaming Engine.")); - } - - @Override - public long getAndResetThrottleTime() { - return getWorkThrottleTimer.getAndResetThrottleTime() - + getDataThrottleTimer.getAndResetThrottleTime() - + commitWorkThrottleTimer.getAndResetThrottleTime(); - } - - private JobHeader makeHeader() { - return JobHeader.newBuilder() - .setJobId(options.getJobId()) - .setProjectId(options.getProject()) - .setWorkerId(options.getWorkerId()) - .build(); - } - - /** - * Create a wrapper around credentials callback that delegates to the underlying vendored {@link - * com.google.auth.RequestMetadataCallback}. Note that this class should override every method - * that is not final and not static and call the delegate directly. - * - * <p>TODO: Replace this with an auto generated proxy which calls the underlying implementation - * delegate to reduce maintenance burden. - */ - private static class VendoredRequestMetadataCallbackAdapter - implements com.google.auth.RequestMetadataCallback { - - private final org.apache.beam.vendor.grpc.v1p54p0.com.google.auth.RequestMetadataCallback - callback; - - private VendoredRequestMetadataCallbackAdapter( - org.apache.beam.vendor.grpc.v1p54p0.com.google.auth.RequestMetadataCallback callback) { - this.callback = callback; - } - - @Override - public void onSuccess(Map<String, List<String>> metadata) { - callback.onSuccess(metadata); - } - - @Override - public void onFailure(Throwable exception) { - callback.onFailure(exception); - } - } - - /** - * Create a wrapper around credentials that delegates to the underlying {@link - * com.google.auth.Credentials}. Note that this class should override every method that is not - * final and not static and call the delegate directly. - * - * <p>TODO: Replace this with an auto generated proxy which calls the underlying implementation - * delegate to reduce maintenance burden. - */ - private static class VendoredCredentialsAdapter - extends org.apache.beam.vendor.grpc.v1p54p0.com.google.auth.Credentials { - - private final com.google.auth.Credentials credentials; - - private VendoredCredentialsAdapter(com.google.auth.Credentials credentials) { - this.credentials = credentials; - } - - @Override - public String getAuthenticationType() { - return credentials.getAuthenticationType(); - } - - @Override - public Map<String, List<String>> getRequestMetadata() throws IOException { - return credentials.getRequestMetadata(); - } - - @Override - public void getRequestMetadata( - final URI uri, - Executor executor, - final org.apache.beam.vendor.grpc.v1p54p0.com.google.auth.RequestMetadataCallback - callback) { - credentials.getRequestMetadata( - uri, executor, new VendoredRequestMetadataCallbackAdapter(callback)); - } - - @Override - public Map<String, List<String>> getRequestMetadata(URI uri) throws IOException { - return credentials.getRequestMetadata(uri); - } - - @Override - public boolean hasRequestMetadata() { - return credentials.hasRequestMetadata(); - } - - @Override - public boolean hasRequestMetadataOnly() { - return credentials.hasRequestMetadataOnly(); - } - - @Override - public void refresh() throws IOException { - credentials.refresh(); - } - } - - private static class HealthCheckTimerTask extends TimerTask { - private final StreamingDataflowWorkerOptions options; - private final Supplier<Set<AbstractWindmillStream<?, ?>>> streams; - - public HealthCheckTimerTask( - StreamingDataflowWorkerOptions options, - Supplier<Set<AbstractWindmillStream<?, ?>>> streams) { - this.options = options; - this.streams = streams; - } - - @Override - public void run() { - Instant reportThreshold = - Instant.now() - .minus(Duration.millis(options.getWindmillServiceStreamingRpcHealthCheckPeriodMs())); - for (AbstractWindmillStream<?, ?> stream : streams.get()) { - stream.maybeSendHealthCheck(reportThreshold); - } - } - } -} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/WorkItemReceiver.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/WorkItemReceiver.java new file mode 100644 index 000000000000..307dfdfa17b3 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/WorkItemReceiver.java @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.dataflow.worker.windmill.work; + +import java.util.Collection; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.joda.time.Instant; + +/** Functional interface for receiving WorkItems. */ +@FunctionalInterface +public interface WorkItemReceiver { + void receiveWork( + String computation, + @Nullable Instant inputDataWatermark, + @Nullable Instant synchronizedProcessingTime, + Windmill.WorkItem workItem, + Collection<Windmill.LatencyAttribution> getWorkStreamLatencies); +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/GetWorkBudget.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/GetWorkBudget.java new file mode 100644 index 000000000000..0038e3e9cc60 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/GetWorkBudget.java @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.dataflow.worker.windmill.work.budget; + +import com.google.auto.value.AutoValue; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GetWorkRequest; +import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; + +/** + * Budget of items and bytes for fetching {@link + * org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkItem}(s) via {@link + * WindmillStream.GetWorkStream}. Used to control how "much" work is returned from Windmill. + */ +@AutoValue +public abstract class GetWorkBudget { + public static GetWorkBudget.Builder builder() { + return new AutoValue_GetWorkBudget.Builder(); + } + + /** {@link GetWorkBudget} of 0. */ + public static GetWorkBudget noBudget() { + return builder().setItems(0).setBytes(0).build(); + } + + public static GetWorkBudget from(GetWorkRequest getWorkRequest) { + return builder() + .setItems(getWorkRequest.getMaxItems()) + .setBytes(getWorkRequest.getMaxBytes()) + .build(); + } + + /** + * Adds the given bytes and items or the current budget, returning a new {@link GetWorkBudget}. + * Does not drop below 0. + */ + public GetWorkBudget add(long items, long bytes) { + Preconditions.checkArgument(items >= 0 && bytes >= 0); + return GetWorkBudget.builder().setBytes(bytes() + bytes).setItems(items() + items).build(); + } + + public GetWorkBudget add(GetWorkBudget other) { + return add(other.items(), other.bytes()); + } + + /** + * Subtracts the given bytes and items or the current budget, returning a new {@link + * GetWorkBudget}. Does not drop below 0. + */ + public GetWorkBudget subtract(long items, long bytes) { + Preconditions.checkArgument(items >= 0 && bytes >= 0); + return GetWorkBudget.builder().setBytes(bytes() - bytes).setItems(items() - items).build(); + } + + public GetWorkBudget subtract(GetWorkBudget other) { + return subtract(other.items(), other.bytes()); + } + + /** Budget of bytes for GetWork. Does not drop below 0. */ + public abstract long bytes(); + + /** Budget of items for GetWork. Does not drop below 0. */ + public abstract long items(); + + @AutoValue.Builder + public abstract static class Builder { + public abstract Builder setBytes(long bytes); + + public abstract Builder setItems(long budget); + + abstract long items(); + + abstract long bytes(); + + abstract GetWorkBudget autoBuild(); + + public final GetWorkBudget build() { + setItems(Math.max(0, items())); + setBytes(Math.max(0, bytes())); + return autoBuild(); + } + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/FakeWindmillServer.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/FakeWindmillServer.java index 4700217dc8a4..092f5e59a13c 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/FakeWindmillServer.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/FakeWindmillServer.java @@ -53,9 +53,11 @@ import org.apache.beam.runners.dataflow.worker.windmill.Windmill.LatencyAttribution.State; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkItemCommitRequest; import org.apache.beam.runners.dataflow.worker.windmill.WindmillServerStub; -import org.apache.beam.runners.dataflow.worker.windmill.WindmillStream.CommitWorkStream; -import org.apache.beam.runners.dataflow.worker.windmill.WindmillStream.GetDataStream; -import org.apache.beam.runners.dataflow.worker.windmill.WindmillStream.GetWorkStream; +import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.CommitWorkStream; +import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.GetDataStream; +import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.GetWorkStream; +import org.apache.beam.runners.dataflow.worker.windmill.work.WorkItemReceiver; +import org.apache.beam.runners.dataflow.worker.windmill.work.budget.GetWorkBudget; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.net.HostAndPort; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.Uninterruptibles; import org.joda.time.Duration; @@ -198,8 +200,7 @@ public long getAndResetThrottleTime() { } @Override - public GetWorkStream getWorkStream( - Windmill.GetWorkRequest request, GetWorkStream.WorkItemReceiver receiver) { + public GetWorkStream getWorkStream(Windmill.GetWorkRequest request, WorkItemReceiver receiver) { LOG.debug("getWorkStream: {}", request.toString()); Instant startTime = Instant.now(); final CountDownLatch done = new CountDownLatch(1); @@ -209,6 +210,19 @@ public void close() { done.countDown(); } + @Override + public void adjustBudget(long itemsDelta, long bytesDelta) { + // no-op. + } + + @Override + public GetWorkBudget remainingBudget() { + return GetWorkBudget.builder() + .setItems(request.getMaxItems()) + .setBytes(request.getMaxBytes()) + .build(); + } + @Override public boolean awaitTermination(int time, TimeUnit unit) throws InterruptedException { while (done.getCount() > 0) { diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/WindmillStreamPoolTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStreamPoolTest.java similarity index 99% rename from runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/WindmillStreamPoolTest.java rename to runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStreamPoolTest.java index 9924bb7d2b2b..264540531bf8 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/WindmillStreamPoolTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStreamPoolTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.dataflow.worker.windmill; +package org.apache.beam.runners.dataflow.worker.windmill.client; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GrpcGetWorkerMetadataStreamTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetWorkerMetadataStreamTest.java similarity index 96% rename from runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GrpcGetWorkerMetadataStreamTest.java rename to runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetWorkerMetadataStreamTest.java index 45ed3381a8bf..e3b07bf7aa4d 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GrpcGetWorkerMetadataStreamTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetWorkerMetadataStreamTest.java @@ -15,10 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.dataflow.worker.windmill.grpcclient; +package org.apache.beam.runners.dataflow.worker.windmill.client.grpc; import static com.google.common.truth.Truth.assertThat; -import static org.apache.beam.runners.dataflow.worker.windmill.AbstractWindmillStream.DEFAULT_STREAM_RPC_DEADLINE_SECONDS; +import static org.apache.beam.runners.dataflow.worker.windmill.client.AbstractWindmillStream.DEFAULT_STREAM_RPC_DEADLINE_SECONDS; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import static org.mockito.Mockito.verify; @@ -33,13 +33,14 @@ import java.util.function.Consumer; import java.util.stream.Collectors; import javax.annotation.Nullable; -import org.apache.beam.runners.dataflow.worker.windmill.AbstractWindmillStream; import org.apache.beam.runners.dataflow.worker.windmill.CloudWindmillServiceV1Alpha1Grpc; -import org.apache.beam.runners.dataflow.worker.windmill.StreamObserverFactory; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.JobHeader; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkerMetadataRequest; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkerMetadataResponse; import org.apache.beam.runners.dataflow.worker.windmill.WindmillEndpoints; +import org.apache.beam.runners.dataflow.worker.windmill.client.AbstractWindmillStream; +import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.observers.StreamObserverFactory; +import org.apache.beam.runners.dataflow.worker.windmill.client.throttling.ThrottleTimer; import org.apache.beam.sdk.util.FluentBackoff; import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ManagedChannel; import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.Server; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GrpcWindmillServerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillServerTest.java similarity index 98% rename from runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GrpcWindmillServerTest.java rename to runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillServerTest.java index 53afc6990e43..d9f4b72716cb 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GrpcWindmillServerTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillServerTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.dataflow.worker.windmill.grpcclient; +package org.apache.beam.runners.dataflow.worker.windmill.client.grpc; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; @@ -68,9 +68,9 @@ import org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkItem; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkItemCommitRequest; import org.apache.beam.runners.dataflow.worker.windmill.WindmillApplianceGrpc; -import org.apache.beam.runners.dataflow.worker.windmill.WindmillStream.CommitWorkStream; -import org.apache.beam.runners.dataflow.worker.windmill.WindmillStream.GetDataStream; -import org.apache.beam.runners.dataflow.worker.windmill.WindmillStream.GetWorkStream; +import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.CommitWorkStream; +import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.GetDataStream; +import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.GetWorkStream; import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.CallOptions; import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.Channel; @@ -99,10 +99,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -/** - * Unit tests for {@link - * org.apache.beam.runners.dataflow.worker.windmill.grpcclient.GrpcWindmillServer}. - */ +/** Unit tests for {@link GrpcWindmillServer}. */ @RunWith(JUnit4.class) @SuppressWarnings({ "rawtypes", // TODO(https://github.com/apache/beam/issues/20447) @@ -114,7 +111,7 @@ public class GrpcWindmillServerTest { private final MutableHandlerRegistry serviceRegistry = new MutableHandlerRegistry(); @Rule public ErrorCollector errorCollector = new ErrorCollector(); private Server server; - private org.apache.beam.runners.dataflow.worker.windmill.grpcclient.GrpcWindmillServer client; + private GrpcWindmillServer client; private int remainingErrors = 20; @Before diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/GetWorkBudgetTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/GetWorkBudgetTest.java new file mode 100644 index 000000000000..76d508397850 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/GetWorkBudgetTest.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.dataflow.worker.windmill.work.budget; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThrows; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class GetWorkBudgetTest { + + @Test + public void testCreateWithNoBudget() { + GetWorkBudget getWorkBudget = GetWorkBudget.noBudget(); + assertEquals(0, getWorkBudget.items()); + assertEquals(0, getWorkBudget.bytes()); + } + + @Test + public void testBuild_itemsAndBytesNeverBelowZero() { + GetWorkBudget getWorkBudget = GetWorkBudget.builder().setItems(-10).setBytes(-10).build(); + assertEquals(0, getWorkBudget.items()); + assertEquals(0, getWorkBudget.bytes()); + } + + @Test + public void testAdd_doesNotAllowNegativeParameters() { + GetWorkBudget getWorkBudget = GetWorkBudget.builder().setItems(1).setBytes(1).build(); + assertThrows(IllegalArgumentException.class, () -> getWorkBudget.add(-1, -1)); + } + + @Test + public void testSubtract_itemsAndBytesNeverBelowZero() { + GetWorkBudget getWorkBudget = GetWorkBudget.builder().setItems(1).setBytes(1).build(); + GetWorkBudget subtracted = getWorkBudget.subtract(10, 10); + assertEquals(0, subtracted.items()); + assertEquals(0, subtracted.bytes()); + } + + @Test + public void testSubtractGetWorkBudget_itemsAndBytesNeverBelowZero() { + GetWorkBudget getWorkBudget = GetWorkBudget.builder().setItems(1).setBytes(1).build(); + GetWorkBudget subtracted = + getWorkBudget.subtract(GetWorkBudget.builder().setItems(10).setBytes(10).build()); + assertEquals(0, subtracted.items()); + assertEquals(0, subtracted.bytes()); + } + + @Test + public void testSubtract_doesNotAllowNegativeParameters() { + GetWorkBudget getWorkBudget = GetWorkBudget.builder().setItems(1).setBytes(1).build(); + assertThrows(IllegalArgumentException.class, () -> getWorkBudget.subtract(-1, -1)); + } +} From 400ef71e7194535828d49f12e5311388a763e62b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ferran=20Fern=C3=A1ndez=20Garrido?= <ffernandez.upc@gmail.com> Date: Tue, 31 Oct 2023 11:15:22 +0100 Subject: [PATCH 304/435] [YAML] - Exactly one field + fix typo --- .../kafka/KafkaWriteSchemaTransformProvider.java | 16 +++++++--------- .../KafkaWriteSchemaTransformProviderTest.java | 2 +- 2 files changed, 8 insertions(+), 10 deletions(-) diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaWriteSchemaTransformProvider.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaWriteSchemaTransformProvider.java index 46a4d078f472..1dbe37791c05 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaWriteSchemaTransformProvider.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaWriteSchemaTransformProvider.java @@ -72,8 +72,6 @@ public class KafkaWriteSchemaTransformProvider private static final Logger LOG = LoggerFactory.getLogger(KafkaWriteSchemaTransformProvider.class); - private static final String PAYLOAD = "payload"; - @Override protected @UnknownKeyFor @NonNull @Initialized Class<KafkaWriteSchemaTransformConfiguration> configurationClass() { @@ -136,11 +134,11 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) { Schema inputSchema = input.get("input").getSchema(); final SerializableFunction<Row, byte[]> toBytesFn; if (configuration.getFormat().equals("RAW")) { - if (!inputSchema.hasField(PAYLOAD)) { - throw new IllegalArgumentException( - "To write to Kafka in RAW format, the input schema must provide the payload attribute."); + int numFields = inputSchema.getFields().size(); + if (numFields != 1) { + throw new IllegalArgumentException("Expecting exactly one field, found " + numFields); } - toBytesFn = getRowToRawBytesFunction(); + toBytesFn = getRowToRawBytesFunction(inputSchema.getField(0).getName()); } else if (configuration.getFormat().equals("JSON")) { toBytesFn = JsonUtils.getRowToJsonBytesFunction(inputSchema); } else { @@ -174,11 +172,11 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) { } } - public static SerializableFunction<Row, byte[]> getRowToRawBytesFunction() { + public static SerializableFunction<Row, byte[]> getRowToRawBytesFunction(String rowFieldName) { return new SimpleFunction<Row, byte[]>() { @Override public byte[] apply(Row input) { - byte[] rawBytes = input.getBytes(PAYLOAD); + byte[] rawBytes = input.getBytes(rowFieldName); if (rawBytes == null) { throw new NullPointerException(); } @@ -217,7 +215,7 @@ public abstract static class KafkaWriteSchemaTransformConfiguration implements S @SchemaFieldDescription( "A list of host/port pairs to use for establishing the initial connection to the" + " Kafka cluster. The client will make use of all servers irrespective of which servers are specified" - + " here for bootstrapping—this list only impacts the initial hosts used to discover the full set" + + " here for bootstrapping—this list only impacts the initial hosts used to discover the full set" + " of servers. | Format: host1:port1,host2:port2,...") public abstract String getBootstrapServers(); diff --git a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaWriteSchemaTransformProviderTest.java b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaWriteSchemaTransformProviderTest.java index 279e67242a6d..3a7769f42d72 100644 --- a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaWriteSchemaTransformProviderTest.java +++ b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaWriteSchemaTransformProviderTest.java @@ -80,7 +80,7 @@ public class KafkaWriteSchemaTransformProviderTest { final SerializableFunction<Row, byte[]> valueMapper = JsonUtils.getRowToJsonBytesFunction(BEAMSCHEMA); - final SerializableFunction<Row, byte[]> valueRawMapper = getRowToRawBytesFunction(); + final SerializableFunction<Row, byte[]> valueRawMapper = getRowToRawBytesFunction("payload"); @Rule public transient TestPipeline p = TestPipeline.create(); From 181296dec2a6d6973e7aaa17f2cb0286c9308457 Mon Sep 17 00:00:00 2001 From: Jing <lscmirror@gmail.com> Date: Tue, 31 Oct 2023 06:17:59 -0700 Subject: [PATCH 305/435] Support DatabaseID in Datastore beam connector (#27987) * Fix routing header issue in Firestore Beam connector * Apply the lint * Support multi-db in datastore beam * Support multi-db in datastore beam * Remove databaseRef in WriteFn * retrigger checks * resolve comments * resolve comments * resolve comments * resolve comments * lint * Remove the PipelineOption reference * Update comment * Update ValueProviders usage --------- Co-authored-by: Sichen Liu <sichenliu@google.com> --- .../sdk/io/gcp/datastore/DatastoreV1.java | 349 +++++++++++++++--- .../sdk/io/gcp/datastore/DatastoreV1Test.java | 94 ++++- .../sdk/io/gcp/datastore/SplitQueryFnIT.java | 17 +- .../beam/sdk/io/gcp/datastore/V1ReadIT.java | 26 +- .../beam/sdk/io/gcp/datastore/V1TestUtil.java | 59 ++- .../beam/sdk/io/gcp/datastore/V1WriteIT.java | 11 +- 6 files changed, 458 insertions(+), 98 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java index cdd003abae22..b94186b9893d 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java @@ -126,8 +126,9 @@ * <p>To read a {@link PCollection} from a query to Cloud Datastore, use {@link DatastoreV1#read} * and its methods {@link DatastoreV1.Read#withProjectId} and {@link DatastoreV1.Read#withQuery} to * specify the project to query and the query to read from. You can optionally provide a namespace - * to query within using {@link DatastoreV1.Read#withNamespace}. You could also optionally specify - * how many splits you want for the query using {@link DatastoreV1.Read#withNumQuerySplits}. + * to query within using {@link DatastoreV1.Read#withDatabase} or {@link + * DatastoreV1.Read#withNamespace}. You could also optionally specify how many splits you want for + * the query using {@link DatastoreV1.Read#withNumQuerySplits}. * * <p>For example: * @@ -135,12 +136,14 @@ * // Read a query from Datastore * PipelineOptions options = PipelineOptionsFactory.fromArgs(args).create(); * Query query = ...; + * String databaseId = "..."; * String projectId = "..."; * * Pipeline p = Pipeline.create(options); * PCollection<Entity> entities = p.apply( * DatastoreIO.v1().read() * .withProjectId(projectId) + * .withDatabaseId(databaseId) * .withQuery(query)); * }</pre> * @@ -156,7 +159,7 @@ * * <pre>{@code * PCollection<Entity> entities = ...; - * entities.apply(DatastoreIO.v1().write().withProjectId(projectId)); + * entities.apply(DatastoreIO.v1().write().withProjectId(projectId).withDatabaseId(databaseId)); * p.run(); * }</pre> * @@ -165,7 +168,7 @@ * * <pre>{@code * PCollection<Entity> entities = ...; - * entities.apply(DatastoreIO.v1().deleteEntity().withProjectId(projectId)); + * entities.apply(DatastoreIO.v1().deleteEntity().withProjectId(projectId).withDatabaseId(databaseId)); * p.run(); * }</pre> * @@ -174,7 +177,7 @@ * * <pre>{@code * PCollection<Entity> entities = ...; - * entities.apply(DatastoreIO.v1().deleteKey().withProjectId(projectId)); + * entities.apply(DatastoreIO.v1().deleteKey().withProjectId(projectId).withDatabaseId(databaseId)); * p.run(); * }</pre> * @@ -275,6 +278,9 @@ public class DatastoreV1 { Code.PERMISSION_DENIED, Code.UNAUTHENTICATED); + /** Database ID for the default database. */ + private static final String DEFAULT_DATABASE = ""; + /** * Returns an empty {@link DatastoreV1.Read} builder. Configure the source {@code projectId}, * {@code query}, and optionally {@code namespace} and {@code numQuerySplits} using {@link @@ -313,6 +319,8 @@ public abstract static class Read extends PTransform<PBegin, PCollection<Entity> public abstract @Nullable ValueProvider<String> getProjectId(); + public abstract @Nullable ValueProvider<String> getDatabaseId(); + public abstract @Nullable Query getQuery(); public abstract @Nullable ValueProvider<String> getLiteralGqlQuery(); @@ -334,6 +342,8 @@ public abstract static class Read extends PTransform<PBegin, PCollection<Entity> abstract static class Builder { abstract Builder setProjectId(ValueProvider<String> projectId); + abstract Builder setDatabaseId(ValueProvider<String> databaseId); + abstract Builder setQuery(Query query); abstract Builder setLiteralGqlQuery(ValueProvider<String> literalGqlQuery); @@ -354,10 +364,16 @@ abstract static class Builder { * size from Cloud Datastore. */ static int getEstimatedNumSplits( - Datastore datastore, Query query, @Nullable String namespace, @Nullable Instant readTime) { + Datastore datastore, + String projectId, + String databaseId, + Query query, + @Nullable String namespace, + @Nullable Instant readTime) { int numSplits; try { - long estimatedSizeBytes = getEstimatedSizeBytes(datastore, query, namespace, readTime); + long estimatedSizeBytes = + getEstimatedSizeBytes(datastore, projectId, databaseId, query, namespace, readTime); LOG.info("Estimated size bytes for the query is: {}", estimatedSizeBytes); numSplits = (int) @@ -378,7 +394,11 @@ static int getEstimatedNumSplits( * table. */ private static long queryLatestStatisticsTimestamp( - Datastore datastore, @Nullable String namespace, @Nullable Instant readTime) + Datastore datastore, + String projectId, + String databaseId, + @Nullable String namespace, + @Nullable Instant readTime) throws DatastoreException { Query.Builder query = Query.newBuilder(); // Note: namespace either being null or empty represents the default namespace, in which @@ -390,7 +410,8 @@ private static long queryLatestStatisticsTimestamp( } query.addOrder(makeOrder("timestamp", DESCENDING)); query.setLimit(Int32Value.newBuilder().setValue(1)); - RunQueryRequest request = makeRequest(query.build(), namespace, readTime); + RunQueryRequest request = + makeRequest(projectId, databaseId, query.build(), namespace, readTime); RunQueryResponse response = datastore.runQuery(request); QueryResultBatch batch = response.getBatch(); @@ -406,9 +427,15 @@ private static long queryLatestStatisticsTimestamp( * readTime specified, the latest statistics at or before readTime is retrieved. */ private static Entity getLatestTableStats( - String ourKind, @Nullable String namespace, Datastore datastore, @Nullable Instant readTime) + String projectId, + String databaseId, + String ourKind, + @Nullable String namespace, + Datastore datastore, + @Nullable Instant readTime) throws DatastoreException { - long latestTimestamp = queryLatestStatisticsTimestamp(datastore, namespace, readTime); + long latestTimestamp = + queryLatestStatisticsTimestamp(datastore, projectId, databaseId, namespace, readTime); LOG.info("Latest stats timestamp for kind {} is {}", ourKind, latestTimestamp); Query.Builder queryBuilder = Query.newBuilder(); @@ -423,7 +450,8 @@ private static Entity getLatestTableStats( makeFilter("kind_name", EQUAL, makeValue(ourKind).build()).build(), makeFilter("timestamp", EQUAL, makeValue(latestTimestamp).build()).build())); - RunQueryRequest request = makeRequest(queryBuilder.build(), namespace, readTime); + RunQueryRequest request = + makeRequest(projectId, databaseId, queryBuilder.build(), namespace, readTime); long now = System.currentTimeMillis(); RunQueryResponse response = datastore.runQuery(request); @@ -447,10 +475,16 @@ private static Entity getLatestTableStats( * <p>See https://cloud.google.com/datastore/docs/concepts/stats. */ static long getEstimatedSizeBytes( - Datastore datastore, Query query, @Nullable String namespace, @Nullable Instant readTime) + Datastore datastore, + String projectId, + String databaseId, + Query query, + @Nullable String namespace, + @Nullable Instant readTime) throws DatastoreException { String ourKind = query.getKind(0).getName(); - Entity entity = getLatestTableStats(ourKind, namespace, datastore, readTime); + Entity entity = + getLatestTableStats(projectId, databaseId, ourKind, namespace, datastore, readTime); return entity.getPropertiesOrThrow("entity_bytes").getIntegerValue(); } @@ -470,9 +504,18 @@ private static PartitionId.Builder forNamespace(@Nullable String namespace) { * the requested {@code readTime}. */ static RunQueryRequest makeRequest( - Query query, @Nullable String namespace, @Nullable Instant readTime) { + String projectId, + String databaseId, + Query query, + @Nullable String namespace, + @Nullable Instant readTime) { RunQueryRequest.Builder request = - RunQueryRequest.newBuilder().setQuery(query).setPartitionId(forNamespace(namespace)); + RunQueryRequest.newBuilder() + .setProjectId(projectId) + .setDatabaseId(databaseId) + .setQuery(query) + .setPartitionId( + forNamespace(namespace).setProjectId(projectId).setDatabaseId(databaseId)); if (readTime != null) { Timestamp readTimeProto = Timestamps.fromMillis(readTime.getMillis()); request.setReadOptions(ReadOptions.newBuilder().setReadTime(readTimeProto).build()); @@ -486,11 +529,18 @@ static RunQueryRequest makeRequest( * at the requested {@code readTime}. */ static RunQueryRequest makeRequest( - GqlQuery gqlQuery, @Nullable String namespace, @Nullable Instant readTime) { + String projectId, + String databaseId, + GqlQuery gqlQuery, + @Nullable String namespace, + @Nullable Instant readTime) { RunQueryRequest.Builder request = RunQueryRequest.newBuilder() + .setProjectId(projectId) + .setDatabaseId(databaseId) .setGqlQuery(gqlQuery) - .setPartitionId(forNamespace(namespace)); + .setPartitionId( + forNamespace(namespace).setProjectId(projectId).setDatabaseId(databaseId)); if (readTime != null) { Timestamp readTimeProto = Timestamps.fromMillis(readTime.getMillis()); request.setReadOptions(ReadOptions.newBuilder().setReadTime(readTimeProto).build()); @@ -504,6 +554,8 @@ static RunQueryRequest makeRequest( * namespace}. */ private static List<Query> splitQuery( + String projectId, + String databaseId, Query query, @Nullable String namespace, Datastore datastore, @@ -512,7 +564,8 @@ private static List<Query> splitQuery( @Nullable Instant readTime) throws DatastoreException { // If namespace is set, include it in the split request so splits are calculated accordingly. - PartitionId partitionId = forNamespace(namespace).build(); + PartitionId partitionId = + forNamespace(namespace).setProjectId(projectId).setDatabaseId(databaseId).build(); if (readTime != null) { Timestamp readTimeProto = Timestamps.fromMillis(readTime.getMillis()); return querySplitter.getSplits(query, partitionId, numSplits, datastore, readTimeProto); @@ -535,12 +588,18 @@ private static List<Query> splitQuery( */ @VisibleForTesting static Query translateGqlQueryWithLimitCheck( - String gql, Datastore datastore, String namespace, @Nullable Instant readTime) + String gql, + Datastore datastore, + String projectId, + String databaseId, + String namespace, + @Nullable Instant readTime) throws DatastoreException { String gqlQueryWithZeroLimit = gql + " LIMIT 0"; try { Query translatedQuery = - translateGqlQuery(gqlQueryWithZeroLimit, datastore, namespace, readTime); + translateGqlQuery( + gqlQueryWithZeroLimit, datastore, projectId, databaseId, namespace, readTime); // Clear the limit that we set. return translatedQuery.toBuilder().clearLimit().build(); } catch (DatastoreException e) { @@ -551,7 +610,7 @@ static Query translateGqlQueryWithLimitCheck( LOG.warn("Failed to translate Gql query '{}': {}", gqlQueryWithZeroLimit, e.getMessage()); LOG.warn("User query might have a limit already set, so trying without zero limit"); // Retry without the zero limit. - return translateGqlQuery(gql, datastore, namespace, readTime); + return translateGqlQuery(gql, datastore, projectId, databaseId, namespace, readTime); } else { throw e; } @@ -560,13 +619,27 @@ static Query translateGqlQueryWithLimitCheck( /** Translates a gql query string to {@link Query}. */ private static Query translateGqlQuery( - String gql, Datastore datastore, String namespace, @Nullable Instant readTime) + String gql, + Datastore datastore, + String projectId, + String databaseId, + String namespace, + @Nullable Instant readTime) throws DatastoreException { GqlQuery gqlQuery = GqlQuery.newBuilder().setQueryString(gql).setAllowLiterals(true).build(); - RunQueryRequest req = makeRequest(gqlQuery, namespace, readTime); + RunQueryRequest req = makeRequest(projectId, databaseId, gqlQuery, namespace, readTime); return datastore.runQuery(req).getQuery(); } + /** + * Returns a new {@link DatastoreV1.Read} that reads from the Cloud Datastore for the specified + * database. + */ + public DatastoreV1.Read withDatabaseId(String databaseId) { + checkArgument(databaseId != null, "databaseId can not be null"); + return toBuilder().setDatabaseId(StaticValueProvider.of(databaseId)).build(); + } + /** * Returns a new {@link DatastoreV1.Read} that reads from the Cloud Datastore for the specified * project. @@ -677,13 +750,24 @@ public DatastoreV1.Read withReadTime(Instant readTime) { public long getNumEntities( PipelineOptions options, String ourKind, @Nullable String namespace) { try { - V1Options v1Options = V1Options.from(getProjectId(), getNamespace(), getLocalhost()); + V1Options v1Options = + V1Options.from(getProjectId(), getDatabaseId(), getNamespace(), getLocalhost()); V1DatastoreFactory datastoreFactory = new V1DatastoreFactory(); Datastore datastore = datastoreFactory.getDatastore( - options, v1Options.getProjectId(), v1Options.getLocalhost()); - - Entity entity = getLatestTableStats(ourKind, namespace, datastore, getReadTime()); + options, + v1Options.getProjectId(), + v1Options.getDatabaseId(), + v1Options.getLocalhost()); + + Entity entity = + getLatestTableStats( + v1Options.getProjectId(), + v1Options.getDatabaseId(), + ourKind, + namespace, + datastore, + getReadTime()); return entity.getPropertiesOrThrow("count").getIntegerValue(); } catch (Exception e) { return -1; @@ -704,7 +788,8 @@ public PCollection<Entity> expand(PBegin input) { getQuery() == null || getLiteralGqlQuery() == null, "withQuery() and withLiteralGqlQuery() are exclusive"); - V1Options v1Options = V1Options.from(getProjectId(), getNamespace(), getLocalhost()); + V1Options v1Options = + V1Options.from(getProjectId(), getDatabaseId(), getNamespace(), getLocalhost()); /* * This composite transform involves the following steps: @@ -748,6 +833,7 @@ public void populateDisplayData(DisplayData.Builder builder) { String query = getQuery() == null ? null : getQuery().toString(); builder .addIfNotNull(DisplayData.item("projectId", getProjectId()).withLabel("ProjectId")) + .addIfNotNull(DisplayData.item("databaseId", getDatabaseId()).withLabel("DatabaseId")) .addIfNotNull(DisplayData.item("namespace", getNamespace()).withLabel("Namespace")) .addIfNotNull(DisplayData.item("query", query).withLabel("Query")) .addIfNotNull(DisplayData.item("gqlQuery", getLiteralGqlQuery()).withLabel("GqlQuery")) @@ -757,30 +843,46 @@ public void populateDisplayData(DisplayData.Builder builder) { @VisibleForTesting static class V1Options implements HasDisplayData, Serializable { private final ValueProvider<String> project; + private final ValueProvider<String> database; private final @Nullable ValueProvider<String> namespace; private final @Nullable String localhost; private V1Options( - ValueProvider<String> project, ValueProvider<String> namespace, String localhost) { + ValueProvider<String> project, + ValueProvider<String> database, + ValueProvider<String> namespace, + String localhost) { this.project = project; + this.database = database; this.namespace = namespace; this.localhost = localhost; } - public static V1Options from(String projectId, String namespace, String localhost) { + public static V1Options from( + String projectId, ValueProvider<String> databaseId, String namespace, String localhost) { return from( - StaticValueProvider.of(projectId), StaticValueProvider.of(namespace), localhost); + StaticValueProvider.of(projectId), + databaseId, + StaticValueProvider.of(namespace), + localhost); } public static V1Options from( - ValueProvider<String> project, ValueProvider<String> namespace, String localhost) { - return new V1Options(project, namespace, localhost); + ValueProvider<String> project, + ValueProvider<String> databaseId, + ValueProvider<String> namespace, + String localhost) { + return new V1Options(project, databaseId, namespace, localhost); } public String getProjectId() { return project.get(); } + public String getDatabaseId() { + return database == null ? DEFAULT_DATABASE : database.get(); + } + public @Nullable String getNamespace() { return namespace == null ? null : namespace.get(); } @@ -789,6 +891,10 @@ public ValueProvider<String> getProjectValueProvider() { return project; } + public ValueProvider<String> getDatabaseValueProvider() { + return database; + } + public @Nullable ValueProvider<String> getNamespaceValueProvider() { return namespace; } @@ -802,6 +908,8 @@ public void populateDisplayData(DisplayData.Builder builder) { builder .addIfNotNull( DisplayData.item("projectId", getProjectValueProvider()).withLabel("ProjectId")) + .addIfNotNull( + DisplayData.item("databaseId", getDatabaseValueProvider()).withLabel("DatabaseId")) .addIfNotNull( DisplayData.item("namespace", getNamespaceValueProvider()).withLabel("Namespace")); } @@ -833,7 +941,10 @@ static class GqlQueryTranslateFn extends DoFn<String, Query> { public void startBundle(StartBundleContext c) throws Exception { datastore = datastoreFactory.getDatastore( - c.getPipelineOptions(), v1Options.getProjectId(), v1Options.getLocalhost()); + c.getPipelineOptions(), + v1Options.getProjectId(), + v1Options.getDatabaseId(), + v1Options.getLocalhost()); } @ProcessElement @@ -842,7 +953,12 @@ public void processElement(ProcessContext c) throws Exception { LOG.info("User query: '{}'", gqlQuery); Query query = translateGqlQueryWithLimitCheck( - gqlQuery, datastore, v1Options.getNamespace(), readTime); + gqlQuery, + datastore, + v1Options.getProjectId(), + v1Options.getDatabaseId(), + v1Options.getNamespace(), + readTime); LOG.info("User gql query translated to Query({})", query); c.output(query); } @@ -890,7 +1006,10 @@ public SplitQueryFn(V1Options options, int numSplits, @Nullable Instant readTime public void startBundle(StartBundleContext c) throws Exception { datastore = datastoreFactory.getDatastore( - c.getPipelineOptions(), options.getProjectId(), options.getLocalhost()); + c.getPipelineOptions(), + options.getProjectId(), + options.getDatabaseId(), + options.getLocalhost()); querySplitter = datastoreFactory.getQuerySplitter(); } @@ -908,7 +1027,13 @@ public void processElement(ProcessContext c) throws Exception { // Compute the estimated numSplits if numSplits is not specified by the user. if (numSplits <= 0) { estimatedNumSplits = - getEstimatedNumSplits(datastore, query, options.getNamespace(), readTime); + getEstimatedNumSplits( + datastore, + options.getProjectId(), + options.getDatabaseId(), + query, + options.getNamespace(), + readTime); } else { estimatedNumSplits = numSplits; } @@ -918,6 +1043,8 @@ public void processElement(ProcessContext c) throws Exception { try { querySplits = splitQuery( + options.getProjectId(), + options.getDatabaseId(), query, options.getNamespace(), datastore, @@ -985,7 +1112,10 @@ public ReadFn(V1Options options, @Nullable Instant readTime) { public void startBundle(StartBundleContext c) throws Exception { datastore = datastoreFactory.getDatastore( - c.getPipelineOptions(), options.getProjectId(), options.getLocalhost()); + c.getPipelineOptions(), + options.getProjectId(), + options.getDatabaseId(), + options.getLocalhost()); } private RunQueryResponse runQueryWithRetries(RunQueryRequest request) throws Exception { @@ -1045,7 +1175,13 @@ public void processElement(ProcessContext context) throws Exception { queryBuilder.setStartCursor(currentBatch.getEndCursor()); } - RunQueryRequest request = makeRequest(queryBuilder.build(), namespace, readTime); + RunQueryRequest request = + makeRequest( + options.getProjectId(), + options.getDatabaseId(), + queryBuilder.build(), + namespace, + readTime); RunQueryResponse response = runQueryWithRetries(request); currentBatch = response.getBatch(); @@ -1128,21 +1264,42 @@ public static class Write extends Mutate<Entity> { @Nullable String localhost, boolean throttleRampup, ValueProvider<Integer> hintNumWorkers) { - super(projectId, localhost, new UpsertFn(), throttleRampup, hintNumWorkers); + super(projectId, null, localhost, new UpsertFn(), throttleRampup, hintNumWorkers); + } + + Write( + @Nullable ValueProvider<String> projectId, + @Nullable ValueProvider<String> databaseId, + @Nullable String localhost, + boolean throttleRampup, + ValueProvider<Integer> hintNumWorkers) { + super(projectId, databaseId, localhost, new UpsertFn(), throttleRampup, hintNumWorkers); } - /** Returns a new {@link Write} that writes to the Cloud Datastore for the specified project. */ + /** Returns a new {@link Write} that writes to the Cloud Datastore for the default database. */ public Write withProjectId(String projectId) { checkArgument(projectId != null, "projectId can not be null"); return withProjectId(StaticValueProvider.of(projectId)); } + /** Returns a new {@link Write} that writes to the Cloud Datastore for the database id. */ + public Write withDatabaseId(String databaseId) { + checkArgument(databaseId != null, "databaseId can not be null"); + return withDatabaseId(StaticValueProvider.of(databaseId)); + } + /** Same as {@link Write#withProjectId(String)} but with a {@link ValueProvider}. */ public Write withProjectId(ValueProvider<String> projectId) { checkArgument(projectId != null, "projectId can not be null"); return new Write(projectId, localhost, throttleRampup, hintNumWorkers); } + /** Same as {@link Write#withDatabaseId(String)} but with a {@link ValueProvider}. */ + public Write withDatabaseId(ValueProvider<String> databaseId) { + checkArgument(databaseId != null, "databaseId can not be null"); + return new Write(projectId, databaseId, localhost, throttleRampup, hintNumWorkers); + } + /** * Returns a new {@link Write} that writes to the Cloud Datastore Emulator running locally on * the specified host port. @@ -1188,7 +1345,16 @@ public static class DeleteEntity extends Mutate<Entity> { @Nullable String localhost, boolean throttleRampup, ValueProvider<Integer> hintNumWorkers) { - super(projectId, localhost, new DeleteEntityFn(), throttleRampup, hintNumWorkers); + super(projectId, null, localhost, new DeleteEntityFn(), throttleRampup, hintNumWorkers); + } + + DeleteEntity( + @Nullable ValueProvider<String> projectId, + @Nullable ValueProvider<String> databaseId, + @Nullable String localhost, + boolean throttleRampup, + ValueProvider<Integer> hintNumWorkers) { + super(projectId, databaseId, localhost, new DeleteEntityFn(), throttleRampup, hintNumWorkers); } /** @@ -1200,12 +1366,27 @@ public DeleteEntity withProjectId(String projectId) { return withProjectId(StaticValueProvider.of(projectId)); } + /** + * Returns a new {@link DeleteEntity} that deletes entities from the Cloud Datastore for the + * specified database. + */ + public DeleteEntity withDatabaseId(String databaseId) { + checkArgument(databaseId != null, "databaseId can not be null"); + return withDatabaseId(StaticValueProvider.of(databaseId)); + } + /** Same as {@link DeleteEntity#withProjectId(String)} but with a {@link ValueProvider}. */ public DeleteEntity withProjectId(ValueProvider<String> projectId) { checkArgument(projectId != null, "projectId can not be null"); return new DeleteEntity(projectId, localhost, throttleRampup, hintNumWorkers); } + /** Same as {@link DeleteEntity#withDatabaseId(String)} but with a {@link ValueProvider}. */ + public DeleteEntity withDatabaseId(ValueProvider<String> databaseId) { + checkArgument(databaseId != null, "databaseId can not be null"); + return new DeleteEntity(projectId, databaseId, localhost, throttleRampup, hintNumWorkers); + } + /** * Returns a new {@link DeleteEntity} that deletes entities from the Cloud Datastore Emulator * running locally on the specified host port. @@ -1253,7 +1434,16 @@ public static class DeleteKey extends Mutate<Key> { @Nullable String localhost, boolean throttleRampup, ValueProvider<Integer> hintNumWorkers) { - super(projectId, localhost, new DeleteKeyFn(), throttleRampup, hintNumWorkers); + super(projectId, null, localhost, new DeleteKeyFn(), throttleRampup, hintNumWorkers); + } + + DeleteKey( + @Nullable ValueProvider<String> projectId, + @Nullable ValueProvider<String> databaseId, + @Nullable String localhost, + boolean throttleRampup, + ValueProvider<Integer> hintNumWorkers) { + super(projectId, databaseId, localhost, new DeleteKeyFn(), throttleRampup, hintNumWorkers); } /** @@ -1265,6 +1455,15 @@ public DeleteKey withProjectId(String projectId) { return withProjectId(StaticValueProvider.of(projectId)); } + /** + * Returns a new {@link DeleteKey} that deletes entities from the Cloud Datastore for the + * specified database. + */ + public DeleteKey withDatabaseId(String databaseId) { + checkArgument(databaseId != null, "databaseId can not be null"); + return withDatabaseId(StaticValueProvider.of(databaseId)); + } + /** * Returns a new {@link DeleteKey} that deletes entities from the Cloud Datastore Emulator * running locally on the specified host port. @@ -1280,6 +1479,12 @@ public DeleteKey withProjectId(ValueProvider<String> projectId) { return new DeleteKey(projectId, localhost, throttleRampup, hintNumWorkers); } + /** Same as {@link DeleteKey#withDatabaseId(String)} but with a {@link ValueProvider}. */ + public DeleteKey withDatabaseId(ValueProvider<String> databaseId) { + checkArgument(databaseId != null, "databaseId can not be null"); + return new DeleteKey(projectId, databaseId, localhost, throttleRampup, hintNumWorkers); + } + /** Returns a new {@link DeleteKey} that does not throttle during ramp-up. */ public DeleteKey withRampupThrottlingDisabled() { return new DeleteKey(projectId, localhost, false, hintNumWorkers); @@ -1312,6 +1517,7 @@ public DeleteKey withHintNumWorkers(ValueProvider<Integer> hintNumWorkers) { private abstract static class Mutate<T> extends PTransform<PCollection<T>, PDone> { protected ValueProvider<String> projectId; + protected ValueProvider<String> databaseId; protected @Nullable String localhost; protected boolean throttleRampup; protected ValueProvider<Integer> hintNumWorkers; @@ -1326,11 +1532,13 @@ private abstract static class Mutate<T> extends PTransform<PCollection<T>, PDone */ Mutate( @Nullable ValueProvider<String> projectId, + @Nullable ValueProvider<String> databaseId, @Nullable String localhost, SimpleFunction<T, Mutation> mutationFn, boolean throttleRampup, ValueProvider<Integer> hintNumWorkers) { this.projectId = projectId; + this.databaseId = databaseId; this.localhost = localhost; this.throttleRampup = throttleRampup; this.hintNumWorkers = hintNumWorkers; @@ -1372,7 +1580,14 @@ public PCollectionView<Instant> expand(PBegin input) { ParDo.of(rampupThrottlingFn).withSideInputs(startTimestampView)); } intermediateOutput.apply( - "Write Mutation to Datastore", ParDo.of(new DatastoreWriterFn(projectId, localhost))); + "Write Mutation to Datastore", + ParDo.of( + new DatastoreWriterFn( + projectId, + databaseId, + localhost, + new V1DatastoreFactory(), + new WriteBatcherImpl()))); return PDone.in(input.getPipeline()); } @@ -1390,6 +1605,7 @@ public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); builder .addIfNotNull(DisplayData.item("projectId", projectId).withLabel("Output Project")) + .addIfNotNull(DisplayData.item("databaseId", databaseId).withLabel("Output Database")) .include("mutationFn", mutationFn); if (rampupThrottlingFn != null) { builder.include("rampupThrottlingFn", rampupThrottlingFn); @@ -1399,6 +1615,10 @@ public void populateDisplayData(DisplayData.Builder builder) { public String getProjectId() { return projectId.get(); } + + public String getDatabaseId() { + return databaseId.get(); + } } /** Determines batch sizes for commit RPCs. */ @@ -1480,6 +1700,7 @@ static class DatastoreWriterFn extends DoFn<Mutation, Void> { private static final Logger LOG = LoggerFactory.getLogger(DatastoreWriterFn.class); private final ValueProvider<String> projectId; + private final ValueProvider<String> databaseId; private final @Nullable String localhost; private transient Datastore datastore; private final V1DatastoreFactory datastoreFactory; @@ -1511,13 +1732,14 @@ static class DatastoreWriterFn extends DoFn<Mutation, Void> { DatastoreWriterFn(String projectId, @Nullable String localhost) { this( StaticValueProvider.of(projectId), + null, localhost, new V1DatastoreFactory(), new WriteBatcherImpl()); } DatastoreWriterFn(ValueProvider<String> projectId, @Nullable String localhost) { - this(projectId, localhost, new V1DatastoreFactory(), new WriteBatcherImpl()); + this(projectId, null, localhost, new V1DatastoreFactory(), new WriteBatcherImpl()); } @VisibleForTesting @@ -1526,7 +1748,18 @@ static class DatastoreWriterFn extends DoFn<Mutation, Void> { @Nullable String localhost, V1DatastoreFactory datastoreFactory, WriteBatcher writeBatcher) { + this(projectId, null, localhost, datastoreFactory, writeBatcher); + } + + @VisibleForTesting + DatastoreWriterFn( + ValueProvider<String> projectId, + ValueProvider<String> databaseId, + @Nullable String localhost, + V1DatastoreFactory datastoreFactory, + WriteBatcher writeBatcher) { this.projectId = checkNotNull(projectId, "projectId"); + this.databaseId = databaseId; this.localhost = localhost; this.datastoreFactory = datastoreFactory; this.writeBatcher = writeBatcher; @@ -1534,7 +1767,10 @@ static class DatastoreWriterFn extends DoFn<Mutation, Void> { @StartBundle public void startBundle(StartBundleContext c) { - datastore = datastoreFactory.getDatastore(c.getPipelineOptions(), projectId.get(), localhost); + String databaseIdOrDefaultDatabase = databaseId == null ? DEFAULT_DATABASE : databaseId.get(); + datastore = + datastoreFactory.getDatastore( + c.getPipelineOptions(), projectId.get(), databaseIdOrDefaultDatabase, localhost); writeBatcher.start(); if (adaptiveThrottler == null) { // Initialize throttler at first use, because it is not serializable. @@ -1602,11 +1838,14 @@ private synchronized void flushBatch() batchSize.update(mutations.size()); + String databaseIdOrDefaultDatabase = databaseId == null ? DEFAULT_DATABASE : databaseId.get(); while (true) { // Batch upsert entities. CommitRequest.Builder commitRequest = CommitRequest.newBuilder(); commitRequest.addAllMutations(mutations); commitRequest.setMode(CommitRequest.Mode.NON_TRANSACTIONAL); + commitRequest.setProjectId(projectId.get()); + commitRequest.setDatabaseId(databaseIdOrDefaultDatabase); long startTime = System.currentTimeMillis(), endTime; if (adaptiveThrottler.throttleRequest(startTime)) { @@ -1628,6 +1867,7 @@ private synchronized void flushBatch() ServiceCallMetric serviceCallMetric = new ServiceCallMetric(MonitoringInfoConstants.Urns.API_REQUEST_COUNT, baseLabels); try { + datastore.commit(commitRequest.build()); endTime = System.currentTimeMillis(); serviceCallMetric.call("ok"); @@ -1768,8 +2008,9 @@ public void populateDisplayData(DisplayData.Builder builder) { static class V1DatastoreFactory implements Serializable { /** Builds a Cloud Datastore client for the given pipeline options and project. */ - public Datastore getDatastore(PipelineOptions pipelineOptions, String projectId) { - return getDatastore(pipelineOptions, projectId, null); + public Datastore getDatastore( + PipelineOptions pipelineOptions, String projectId, String databaseId) { + return getDatastore(pipelineOptions, projectId, databaseId, null); } /** @@ -1777,7 +2018,10 @@ public Datastore getDatastore(PipelineOptions pipelineOptions, String projectId) * locahost. */ public Datastore getDatastore( - PipelineOptions pipelineOptions, String projectId, @Nullable String localhost) { + PipelineOptions pipelineOptions, + String projectId, + String databaseId, + @Nullable String localhost) { Credentials credential = pipelineOptions.as(GcpOptions.class).getGcpCredential(); // Add Beam version to user agent header. @@ -1797,7 +2041,10 @@ public Datastore getDatastore( } DatastoreOptions.Builder builder = - new DatastoreOptions.Builder().projectId(projectId).initializer(initializer); + new DatastoreOptions.Builder() + .projectId(projectId) + .databaseId(databaseId) + .initializer(initializer); if (localhost != null) { builder.localHost(localhost); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java index 3280e17998e4..ab6f5567dd9a 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java @@ -125,6 +125,7 @@ @RunWith(Enclosed.class) public class DatastoreV1Test { private static final String PROJECT_ID = "testProject"; + private static final String DATABASE_ID = ""; private static final String NAMESPACE = "testNamespace"; private static final String KIND = "testKind"; private static final Query QUERY; @@ -138,7 +139,7 @@ public class DatastoreV1Test { Query.Builder q = Query.newBuilder(); q.addKindBuilder().setName(KIND); QUERY = q.build(); - V_1_OPTIONS = V1Options.from(PROJECT_ID, NAMESPACE, null); + V_1_OPTIONS = V1Options.from(PROJECT_ID, StaticValueProvider.of(DATABASE_ID), NAMESPACE, null); } @Mock protected Datastore mockDatastore; @@ -153,6 +154,9 @@ public void setUp() { when(mockDatastoreFactory.getDatastore(any(PipelineOptions.class), any(String.class), any())) .thenReturn(mockDatastore); + when(mockDatastoreFactory.getDatastore( + any(PipelineOptions.class), any(String.class), any(String.class), any())) + .thenReturn(mockDatastore); when(mockDatastoreFactory.getQuerySplitter()).thenReturn(mockQuerySplitter); // Setup the ProcessWideContainer for testing metrics are set. MetricsContainerImpl container = new MetricsContainerImpl(null); @@ -167,10 +171,12 @@ public void testBuildRead() throws Exception { DatastoreIO.v1() .read() .withProjectId(PROJECT_ID) + .withDatabaseId(DATABASE_ID) .withQuery(QUERY) .withNamespace(NAMESPACE); assertEquals(QUERY, read.getQuery()); assertEquals(PROJECT_ID, read.getProjectId().get()); + assertEquals(DATABASE_ID, read.getDatabaseId().get()); assertEquals(NAMESPACE, read.getNamespace().get()); } @@ -180,11 +186,13 @@ public void testBuildReadWithReadTime() throws Exception { DatastoreIO.v1() .read() .withProjectId(PROJECT_ID) + .withDatabaseId(DATABASE_ID) .withQuery(QUERY) .withReadTime(TIMESTAMP); assertEquals(TIMESTAMP, read.getReadTime()); assertEquals(QUERY, read.getQuery()); assertEquals(PROJECT_ID, read.getProjectId().get()); + assertEquals(DATABASE_ID, read.getDatabaseId().get()); } @Test @@ -193,10 +201,12 @@ public void testBuildReadWithGqlQuery() throws Exception { DatastoreIO.v1() .read() .withProjectId(PROJECT_ID) + .withDatabaseId(DATABASE_ID) .withLiteralGqlQuery(GQL_QUERY) .withNamespace(NAMESPACE); assertEquals(GQL_QUERY, read.getLiteralGqlQuery().get()); assertEquals(PROJECT_ID, read.getProjectId().get()); + assertEquals(DATABASE_ID, read.getDatabaseId().get()); assertEquals(NAMESPACE, read.getNamespace().get()); } @@ -210,10 +220,12 @@ public void testBuildReadAlt() throws Exception { .withQuery(QUERY) .withNamespace(NAMESPACE) .withProjectId(PROJECT_ID) + .withDatabaseId(DATABASE_ID) .withLocalhost(LOCALHOST); assertEquals(TIMESTAMP, read.getReadTime()); assertEquals(QUERY, read.getQuery()); assertEquals(PROJECT_ID, read.getProjectId().get()); + assertEquals(DATABASE_ID, read.getDatabaseId().get()); assertEquals(NAMESPACE, read.getNamespace().get()); assertEquals(LOCALHOST, read.getLocalhost()); } @@ -224,6 +236,7 @@ public void testReadValidationFailsQueryAndGqlQuery() throws Exception { DatastoreIO.v1() .read() .withProjectId(PROJECT_ID) + .withDatabaseId(DATABASE_ID) .withLiteralGqlQuery(GQL_QUERY) .withQuery(QUERY); @@ -257,6 +270,7 @@ public void testReadDisplayData() { DatastoreIO.v1() .read() .withProjectId(PROJECT_ID) + .withDatabaseId(DATABASE_ID) .withQuery(QUERY) .withNamespace(NAMESPACE) .withReadTime(TIMESTAMP); @@ -264,6 +278,7 @@ public void testReadDisplayData() { DisplayData displayData = DisplayData.from(read); assertThat(displayData, hasDisplayItem("projectId", PROJECT_ID)); + assertThat(displayData, hasDisplayItem("databaseId", DATABASE_ID)); assertThat(displayData, hasDisplayItem("query", QUERY.toString())); assertThat(displayData, hasDisplayItem("namespace", NAMESPACE)); assertThat(displayData, hasDisplayItem("readTime", TIMESTAMP)); @@ -275,6 +290,7 @@ public void testReadDisplayDataWithGqlQuery() { DatastoreIO.v1() .read() .withProjectId(PROJECT_ID) + .withDatabaseId(DATABASE_ID) .withLiteralGqlQuery(GQL_QUERY) .withNamespace(NAMESPACE) .withReadTime(TIMESTAMP); @@ -295,6 +311,7 @@ public void testSourcePrimitiveDisplayData() { DatastoreIO.v1() .read() .withProjectId(PROJECT_ID) + .withDatabaseId(DATABASE_ID) .withQuery(Query.newBuilder().build()) .withNumQuerySplits(numSplits); @@ -313,29 +330,34 @@ public void testSourcePrimitiveDisplayData() { @Test public void testWriteDisplayData() { - Write write = DatastoreIO.v1().write().withProjectId(PROJECT_ID); + Write write = DatastoreIO.v1().write().withProjectId(PROJECT_ID).withDatabaseId(DATABASE_ID); DisplayData displayData = DisplayData.from(write); assertThat(displayData, hasDisplayItem("projectId", PROJECT_ID)); + assertThat(displayData, hasDisplayItem("databaseId", DATABASE_ID)); } @Test public void testDeleteEntityDisplayData() { - DeleteEntity deleteEntity = DatastoreIO.v1().deleteEntity().withProjectId(PROJECT_ID); + DeleteEntity deleteEntity = + DatastoreIO.v1().deleteEntity().withProjectId(PROJECT_ID).withDatabaseId(DATABASE_ID); DisplayData displayData = DisplayData.from(deleteEntity); assertThat(displayData, hasDisplayItem("projectId", PROJECT_ID)); + assertThat(displayData, hasDisplayItem("databaseId", DATABASE_ID)); } @Test public void testDeleteKeyDisplayData() { - DeleteKey deleteKey = DatastoreIO.v1().deleteKey().withProjectId(PROJECT_ID); + DeleteKey deleteKey = + DatastoreIO.v1().deleteKey().withProjectId(PROJECT_ID).withDatabaseId(DATABASE_ID); DisplayData displayData = DisplayData.from(deleteKey); assertThat(displayData, hasDisplayItem("projectId", PROJECT_ID)); + assertThat(displayData, hasDisplayItem("databaseId", DATABASE_ID)); } @Test @@ -600,6 +622,8 @@ private void datastoreWriterFnTest(int numMutations) throws Exception { while (start < numMutations) { int end = Math.min(numMutations, start + DatastoreV1.DATASTORE_BATCH_UPDATE_ENTITIES_START); CommitRequest.Builder commitRequest = CommitRequest.newBuilder(); + commitRequest.setProjectId(PROJECT_ID); + commitRequest.setDatabaseId(DATABASE_ID); commitRequest.setMode(CommitRequest.Mode.NON_TRANSACTIONAL); commitRequest.addAllMutations(mutations.subList(start, end)); // Verify all the batch requests were made with the expected mutations. @@ -644,6 +668,8 @@ public void testDatastoreWriterFnWithLargeEntities() throws Exception { while (start < mutations.size()) { int end = Math.min(mutations.size(), start + entitiesPerRpc); CommitRequest.Builder commitRequest = CommitRequest.newBuilder(); + commitRequest.setProjectId(PROJECT_ID); + commitRequest.setDatabaseId(DATABASE_ID); commitRequest.setMode(CommitRequest.Mode.NON_TRANSACTIONAL); commitRequest.addAllMutations(mutations.subList(start, end)); // Verify all the batch requests were made with the expected mutations. @@ -681,12 +707,16 @@ public void testDatastoreWriterFnWithDuplicateEntities() throws Exception { CommitRequest.Builder commitRequest = CommitRequest.newBuilder(); commitRequest.setMode(CommitRequest.Mode.NON_TRANSACTIONAL); commitRequest.addAllMutations(mutations.subList(0, 2)); + commitRequest.setProjectId(PROJECT_ID); + commitRequest.setDatabaseId(DATABASE_ID); verify(mockDatastore, times(1)).commit(commitRequest.build()); // second invocation has key [0, 2] because the second 0 triggered a flush batch commitRequest = CommitRequest.newBuilder(); commitRequest.setMode(CommitRequest.Mode.NON_TRANSACTIONAL); commitRequest.addAllMutations(mutations.subList(2, 4)); + commitRequest.setProjectId(PROJECT_ID); + commitRequest.setDatabaseId(DATABASE_ID); verify(mockDatastore, times(1)).commit(commitRequest.build()); verifyMetricWasSet("BatchDatastoreWrite", "ok", "", 2); } @@ -841,17 +871,26 @@ public void testEstimatedSizeBytes() throws Exception { long timestamp = 1234L; RunQueryRequest latestTimestampRequest = - makeRequest(makeLatestTimestampQuery(NAMESPACE), NAMESPACE, readTime); + makeRequest( + PROJECT_ID, DATABASE_ID, makeLatestTimestampQuery(NAMESPACE), NAMESPACE, readTime); RunQueryResponse latestTimestampResponse = makeLatestTimestampResponse(timestamp); // Per Kind statistics request and response RunQueryRequest statRequest = - makeRequest(makeStatKindQuery(NAMESPACE, timestamp), NAMESPACE, readTime); + makeRequest( + PROJECT_ID, + DATABASE_ID, + makeStatKindQuery(NAMESPACE, timestamp), + NAMESPACE, + readTime); RunQueryResponse statResponse = makeStatKindResponse(entityBytes); when(mockDatastore.runQuery(latestTimestampRequest)).thenReturn(latestTimestampResponse); when(mockDatastore.runQuery(statRequest)).thenReturn(statResponse); - assertEquals(entityBytes, getEstimatedSizeBytes(mockDatastore, QUERY, NAMESPACE, readTime)); + assertEquals( + entityBytes, + getEstimatedSizeBytes( + mockDatastore, PROJECT_ID, DATABASE_ID, QUERY, NAMESPACE, readTime)); verify(mockDatastore, times(1)).runQuery(latestTimestampRequest); verify(mockDatastore, times(1)).runQuery(statRequest); } @@ -917,12 +956,18 @@ public void testSplitQueryFnWithoutNumSplits() throws Exception { long timestamp = 1234L; RunQueryRequest latestTimestampRequest = - makeRequest(makeLatestTimestampQuery(NAMESPACE), NAMESPACE, readTime); + makeRequest( + PROJECT_ID, DATABASE_ID, makeLatestTimestampQuery(NAMESPACE), NAMESPACE, readTime); RunQueryResponse latestTimestampResponse = makeLatestTimestampResponse(timestamp); // Per Kind statistics request and response RunQueryRequest statRequest = - makeRequest(makeStatKindQuery(NAMESPACE, timestamp), NAMESPACE, readTime); + makeRequest( + PROJECT_ID, + DATABASE_ID, + makeStatKindQuery(NAMESPACE, timestamp), + NAMESPACE, + readTime); RunQueryResponse statResponse = makeStatKindResponse(entityBytes); when(mockDatastore.runQuery(latestTimestampRequest)).thenReturn(latestTimestampResponse); @@ -1030,9 +1075,11 @@ public void testTranslateGqlQueryWithLimit() throws Exception { GqlQuery gqlQueryWithZeroLimit = GqlQuery.newBuilder().setQueryString(gqlWithZeroLimit).setAllowLiterals(true).build(); - RunQueryRequest gqlRequest = makeRequest(gqlQuery, V_1_OPTIONS.getNamespace(), readTime); + RunQueryRequest gqlRequest = + makeRequest(PROJECT_ID, DATABASE_ID, gqlQuery, V_1_OPTIONS.getNamespace(), readTime); RunQueryRequest gqlRequestWithZeroLimit = - makeRequest(gqlQueryWithZeroLimit, V_1_OPTIONS.getNamespace(), readTime); + makeRequest( + PROJECT_ID, DATABASE_ID, gqlQueryWithZeroLimit, V_1_OPTIONS.getNamespace(), readTime); when(mockDatastore.runQuery(gqlRequestWithZeroLimit)) .thenThrow( new DatastoreException( @@ -1044,7 +1091,8 @@ public void testTranslateGqlQueryWithLimit() throws Exception { when(mockDatastore.runQuery(gqlRequest)) .thenReturn(RunQueryResponse.newBuilder().setQuery(QUERY).build()); assertEquals( - translateGqlQueryWithLimitCheck(gql, mockDatastore, V_1_OPTIONS.getNamespace(), readTime), + translateGqlQueryWithLimitCheck( + gql, mockDatastore, PROJECT_ID, DATABASE_ID, V_1_OPTIONS.getNamespace(), readTime), QUERY); verify(mockDatastore, times(1)).runQuery(gqlRequest); verify(mockDatastore, times(1)).runQuery(gqlRequestWithZeroLimit); @@ -1058,11 +1106,18 @@ public void testTranslateGqlQueryWithNoLimit() throws Exception { GqlQuery.newBuilder().setQueryString(gqlWithZeroLimit).setAllowLiterals(true).build(); RunQueryRequest gqlRequestWithZeroLimit = - makeRequest(gqlQueryWithZeroLimit, V_1_OPTIONS.getNamespace(), readTime); + makeRequest( + PROJECT_ID, DATABASE_ID, gqlQueryWithZeroLimit, V_1_OPTIONS.getNamespace(), readTime); when(mockDatastore.runQuery(gqlRequestWithZeroLimit)) .thenReturn(RunQueryResponse.newBuilder().setQuery(QUERY).build()); assertEquals( - translateGqlQueryWithLimitCheck(gql, mockDatastore, V_1_OPTIONS.getNamespace(), readTime), + translateGqlQueryWithLimitCheck( + gql, + mockDatastore, + V_1_OPTIONS.getProjectId(), + V_1_OPTIONS.getDatabaseId(), + V_1_OPTIONS.getNamespace(), + readTime), QUERY); verify(mockDatastore, times(1)).runQuery(gqlRequestWithZeroLimit); } @@ -1074,13 +1129,20 @@ public void testTranslateGqlQueryWithException() throws Exception { GqlQuery gqlQueryWithZeroLimit = GqlQuery.newBuilder().setQueryString(gqlWithZeroLimit).setAllowLiterals(true).build(); RunQueryRequest gqlRequestWithZeroLimit = - makeRequest(gqlQueryWithZeroLimit, V_1_OPTIONS.getNamespace(), readTime); + makeRequest( + PROJECT_ID, DATABASE_ID, gqlQueryWithZeroLimit, V_1_OPTIONS.getNamespace(), readTime); when(mockDatastore.runQuery(gqlRequestWithZeroLimit)) .thenThrow(new RuntimeException("TestException")); thrown.expect(RuntimeException.class); thrown.expectMessage("TestException"); - translateGqlQueryWithLimitCheck(gql, mockDatastore, V_1_OPTIONS.getNamespace(), readTime); + translateGqlQueryWithLimitCheck( + gql, + mockDatastore, + V_1_OPTIONS.getProjectId(), + V_1_OPTIONS.getDatabaseId(), + V_1_OPTIONS.getNamespace(), + readTime); } } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/SplitQueryFnIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/SplitQueryFnIT.java index ea00821f3604..913e05c86129 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/SplitQueryFnIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/SplitQueryFnIT.java @@ -24,6 +24,7 @@ import java.util.List; import org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.Read.SplitQueryFn; import org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.Read.V1Options; +import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider; import org.apache.beam.sdk.transforms.DoFnTester; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Duration; @@ -58,29 +59,32 @@ public class SplitQueryFnIT { @Test public void testSplitQueryFnWithLargeDataset() throws Exception { String projectId = "apache-beam-testing"; + String databaseId = ""; String kind = "sort_1G"; String namespace = null; // Num splits is computed based on the entity_bytes size of the input_sort_1G kind reported by // Datastore stats. int expectedNumSplits = 32; - testSplitQueryFn(projectId, kind, namespace, expectedNumSplits, null); - testSplitQueryFn(projectId, kind, namespace, expectedNumSplits, readTime); + testSplitQueryFn(projectId, databaseId, kind, namespace, expectedNumSplits, null); + testSplitQueryFn(projectId, databaseId, kind, namespace, expectedNumSplits, readTime); } /** Tests {@link SplitQueryFn} to fallback to NUM_QUERY_SPLITS_MIN for a small dataset. */ @Test public void testSplitQueryFnWithSmallDataset() throws Exception { String projectId = "apache-beam-testing"; + String databaseId = ""; String kind = "shakespeare"; String namespace = null; int expectedNumSplits = NUM_QUERY_SPLITS_MIN; - testSplitQueryFn(projectId, kind, namespace, expectedNumSplits, null); - testSplitQueryFn(projectId, kind, namespace, expectedNumSplits, readTime); + testSplitQueryFn(projectId, databaseId, kind, namespace, expectedNumSplits, null); + testSplitQueryFn(projectId, databaseId, kind, namespace, expectedNumSplits, readTime); } /** A helper method to test {@link SplitQueryFn} to generate the expected number of splits. */ private void testSplitQueryFn( String projectId, + String databaseId, String kind, @Nullable String namespace, int expectedNumSplits, @@ -90,7 +94,10 @@ private void testSplitQueryFn( query.addKindBuilder().setName(kind); SplitQueryFn splitQueryFn = - new SplitQueryFn(V1Options.from(projectId, namespace, null), 0, readTime); + new SplitQueryFn( + V1Options.from(projectId, StaticValueProvider.of(databaseId), namespace, null), + 0, + readTime); DoFnTester<Query, Query> doFnTester = DoFnTester.of(splitQueryFn); List<Query> queries = doFnTester.processBundle(query.build()); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1ReadIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1ReadIT.java index 249cadd48866..5918ec203d9a 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1ReadIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1ReadIT.java @@ -31,6 +31,7 @@ import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; import org.apache.beam.sdk.io.gcp.datastore.V1TestUtil.UpsertMutationBuilder; import org.apache.beam.sdk.io.gcp.datastore.V1TestUtil.V1TestWriter; +import org.apache.beam.sdk.io.gcp.firestore.FirestoreOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; @@ -48,6 +49,7 @@ public class V1ReadIT { private V1TestOptions options; private String project; + private String database; private String ancestor; private final long numEntitiesBeforeReadTime = 600; private final long totalNumEntities = 1000; @@ -58,9 +60,11 @@ public void setup() throws Exception { PipelineOptionsFactory.register(V1TestOptions.class); options = TestPipeline.testingPipelineOptions().as(V1TestOptions.class); project = TestPipeline.testingPipelineOptions().as(GcpOptions.class).getProject(); + database = TestPipeline.testingPipelineOptions().as(FirestoreOptions.class).getFirestoreDb(); + ancestor = UUID.randomUUID().toString(); // Create entities and write them to datastore - writeEntitiesToDatastore(options, project, ancestor, 0, numEntitiesBeforeReadTime); + writeEntitiesToDatastore(options, project, database, ancestor, 0, numEntitiesBeforeReadTime); Thread.sleep(1000); readTime = Instant.now(); @@ -68,12 +72,12 @@ public void setup() throws Exception { long moreEntitiesToWrite = totalNumEntities - numEntitiesBeforeReadTime; writeEntitiesToDatastore( - options, project, ancestor, numEntitiesBeforeReadTime, moreEntitiesToWrite); + options, project, database, ancestor, numEntitiesBeforeReadTime, moreEntitiesToWrite); } @After public void tearDown() throws Exception { - deleteAllEntities(options, project, ancestor); + deleteAllEntities(options, project, database, ancestor); } /** @@ -93,6 +97,7 @@ public void testE2EV1Read() throws Exception { DatastoreIO.v1() .read() .withProjectId(project) + .withDatabaseId(database) .withQuery(query) .withNamespace(options.getNamespace()); @@ -108,6 +113,7 @@ public void testE2EV1Read() throws Exception { DatastoreIO.v1() .read() .withProjectId(project) + .withDatabaseId(database) .withQuery(query) .withNamespace(options.getNamespace()) .withReadTime(readTime); @@ -152,6 +158,7 @@ private void testE2EV1ReadWithGQLQuery(long limit) throws Exception { DatastoreIO.v1() .read() .withProjectId(project) + .withDatabaseId(database) .withLiteralGqlQuery(gqlQuery) .withNamespace(options.getNamespace()); @@ -167,6 +174,7 @@ private void testE2EV1ReadWithGQLQuery(long limit) throws Exception { DatastoreIO.v1() .read() .withProjectId(project) + .withDatabaseId(database) .withLiteralGqlQuery(gqlQuery) .withNamespace(options.getNamespace()) .withReadTime(readTime); @@ -181,11 +189,17 @@ private void testE2EV1ReadWithGQLQuery(long limit) throws Exception { // Creates entities and write them to datastore private static void writeEntitiesToDatastore( - V1TestOptions options, String project, String ancestor, long valueOffset, long numEntities) + V1TestOptions options, + String project, + String database, + String ancestor, + long valueOffset, + long numEntities) throws Exception { - Datastore datastore = getDatastore(options, project); + Datastore datastore = getDatastore(options, project, database); // Write test entities to datastore - V1TestWriter writer = new V1TestWriter(datastore, new UpsertMutationBuilder()); + V1TestWriter writer = + new V1TestWriter(datastore, project, database, new UpsertMutationBuilder()); Key ancestorKey = makeAncestorKey(options.getNamespace(), options.getKind(), ancestor); for (long i = 0; i < numEntities; i++) { diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestUtil.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestUtil.java index e3057c4b90ca..0cb3ffac3222 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestUtil.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestUtil.java @@ -137,10 +137,12 @@ public void processElement(ProcessContext c) throws Exception { } /** Build a new datastore client. */ - static Datastore getDatastore(PipelineOptions pipelineOptions, String projectId) { + static Datastore getDatastore( + PipelineOptions pipelineOptions, String projectId, String databaseId) { Credentials credential = pipelineOptions.as(GcpOptions.class).getGcpCredential(); HttpRequestInitializer initializer; if (credential != null) { + initializer = new ChainingHttpRequestInitializer( new HttpCredentialsAdapter(credential), new RetryHttpRequestInitializer()); @@ -149,14 +151,22 @@ static Datastore getDatastore(PipelineOptions pipelineOptions, String projectId) } DatastoreOptions.Builder builder = - new DatastoreOptions.Builder().projectId(projectId).initializer(initializer); + new DatastoreOptions.Builder() + .projectId(projectId) + .databaseId(databaseId) + .initializer(initializer); return DatastoreFactory.get().create(builder.build()); } /** Build a datastore query request. */ - private static RunQueryRequest makeRequest(Query query, @Nullable String namespace) { - RunQueryRequest.Builder requestBuilder = RunQueryRequest.newBuilder().setQuery(query); + private static RunQueryRequest makeRequest( + String projectId, String databaseId, Query query, @Nullable String namespace) { + RunQueryRequest.Builder requestBuilder = + RunQueryRequest.newBuilder() + .setQuery(query) + .setProjectId(projectId) + .setDatabaseId(databaseId); if (namespace != null) { requestBuilder.getPartitionIdBuilder().setNamespaceId(namespace); } @@ -164,14 +174,16 @@ private static RunQueryRequest makeRequest(Query query, @Nullable String namespa } /** Delete all entities with the given ancestor. */ - static void deleteAllEntities(V1TestOptions options, String project, String ancestor) - throws Exception { - Datastore datastore = getDatastore(options, project); + static void deleteAllEntities( + V1TestOptions options, String project, String database, String ancestor) throws Exception { + Datastore datastore = getDatastore(options, project, database); Query query = V1TestUtil.makeAncestorKindQuery(options.getKind(), options.getNamespace(), ancestor); - V1TestReader reader = new V1TestReader(datastore, query, options.getNamespace()); - V1TestWriter writer = new V1TestWriter(datastore, new DeleteMutationBuilder()); + V1TestReader reader = + new V1TestReader(datastore, project, database, query, options.getNamespace()); + V1TestWriter writer = + new V1TestWriter(datastore, project, database, new DeleteMutationBuilder()); long numEntities = 0; while (reader.advance()) { @@ -185,14 +197,15 @@ static void deleteAllEntities(V1TestOptions options, String project, String ance } /** Returns the total number of entities for the given datastore. */ - static long countEntities(V1TestOptions options, String project, String ancestor) + static long countEntities(V1TestOptions options, String project, String database, String ancestor) throws Exception { // Read from datastore. - Datastore datastore = V1TestUtil.getDatastore(options, project); + Datastore datastore = V1TestUtil.getDatastore(options, project, database); Query query = V1TestUtil.makeAncestorKindQuery(options.getKind(), options.getNamespace(), ancestor); - V1TestReader reader = new V1TestReader(datastore, query, options.getNamespace()); + V1TestReader reader = + new V1TestReader(datastore, project, database, query, options.getNamespace()); long numEntitiesRead = 0; while (reader.advance()) { @@ -247,12 +260,17 @@ static boolean isValidKey(Key key) { return (lastElement.getId() != 0 || !lastElement.getName().isEmpty()); } + private final String projectId; + private final String databaseId; private final Datastore datastore; private final MutationBuilder mutationBuilder; private final List<Entity> entities = new ArrayList<>(); - V1TestWriter(Datastore datastore, MutationBuilder mutationBuilder) { + V1TestWriter( + Datastore datastore, String projectId, String databaseId, MutationBuilder mutationBuilder) { this.datastore = datastore; + this.projectId = projectId; + this.databaseId = databaseId; this.mutationBuilder = mutationBuilder; } @@ -295,6 +313,8 @@ private void flushBatch() throws DatastoreException, IOException, InterruptedExc commitRequest.addMutations(mutationBuilder.apply(entity)); } commitRequest.setMode(CommitRequest.Mode.NON_TRANSACTIONAL); + commitRequest.setProjectId(projectId); + commitRequest.setDatabaseId(databaseId); datastore.commit(commitRequest.build()); // Break if the commit threw no exception. break; @@ -318,6 +338,8 @@ private void flushBatch() throws DatastoreException, IOException, InterruptedExc static class V1TestReader { private static final int QUERY_BATCH_LIMIT = 500; private final Datastore datastore; + private final String projectId; + private final String databaseId; private final Query query; private final @Nullable String namespace; private boolean moreResults; @@ -326,8 +348,15 @@ static class V1TestReader { private QueryResultBatch currentBatch; private Entity currentEntity; - V1TestReader(Datastore datastore, Query query, @Nullable String namespace) { + V1TestReader( + Datastore datastore, + String projectId, + String databaseId, + Query query, + @Nullable String namespace) { this.datastore = datastore; + this.projectId = projectId; + this.databaseId = databaseId; this.query = query; this.namespace = namespace; } @@ -362,7 +391,7 @@ private Iterator<EntityResult> getIteratorAndMoveCursor() throws DatastoreExcept query.setStartCursor(currentBatch.getEndCursor()); } - RunQueryRequest request = makeRequest(query.build(), namespace); + RunQueryRequest request = makeRequest(projectId, databaseId, query.build(), namespace); RunQueryResponse response = datastore.runQuery(request); currentBatch = response.getBatch(); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1WriteIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1WriteIT.java index 4f794a03e58d..0062208630f6 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1WriteIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1WriteIT.java @@ -56,6 +56,7 @@ public class V1WriteIT { private V1TestOptions options; private String project; + private String database = ""; private String ancestor; private final long numEntities = 1000; @@ -83,12 +84,12 @@ public void testE2EV1Write() throws Exception { ParDo.of( new V1TestUtil.CreateEntityFn( options.getKind(), options.getNamespace(), ancestor, 0))) - .apply(DatastoreIO.v1().write().withProjectId(project)); + .apply(DatastoreIO.v1().write().withProjectId(project).withDatabaseId(database)); p.run(); // Count number of entities written to datastore. - long numEntitiesWritten = countEntities(options, project, ancestor); + long numEntitiesWritten = countEntities(options, project, database, ancestor); assertEquals(numEntities, numEntitiesWritten); } @@ -185,18 +186,18 @@ public void testE2EV1WriteWithLargeEntities() throws Exception { ParDo.of( new V1TestUtil.CreateEntityFn( options.getKind(), options.getNamespace(), ancestor, rawPropertySize))) - .apply(DatastoreIO.v1().write().withProjectId(project)); + .apply(DatastoreIO.v1().write().withProjectId(project).withDatabaseId(database)); p.run(); // Count number of entities written to datastore. - long numEntitiesWritten = countEntities(options, project, ancestor); + long numEntitiesWritten = countEntities(options, project, database, ancestor); assertEquals(numLargeEntities, numEntitiesWritten); } @After public void tearDown() throws Exception { - deleteAllEntities(options, project, ancestor); + deleteAllEntities(options, project, database, ancestor); } } From 7d81f0a4f35561276ee3b4fef01e0f538c4c97f1 Mon Sep 17 00:00:00 2001 From: Danny McCormick <dannymccormick@google.com> Date: Tue, 31 Oct 2023 09:26:17 -0400 Subject: [PATCH 306/435] Add empty outputs to AMR notebook (#29205) --- .../beam-ml/automatic_model_refresh.ipynb | 165 +++++++++++++++--- 1 file changed, 144 insertions(+), 21 deletions(-) diff --git a/examples/notebooks/beam-ml/automatic_model_refresh.ipynb b/examples/notebooks/beam-ml/automatic_model_refresh.ipynb index 3bafa4f07887..cf05979c5b33 100644 --- a/examples/notebooks/beam-ml/automatic_model_refresh.ipynb +++ b/examples/notebooks/beam-ml/automatic_model_refresh.ipynb @@ -52,7 +52,13 @@ "id": "OsFaZscKSPvo" }, "execution_count": null, - "outputs": [] + "outputs": [{ + "output_type": "stream", + "name": "stdout", + "text": [ + "\n" + ] + }] }, { "cell_type": "markdown", @@ -70,7 +76,14 @@ ], "metadata": { "id": "ZUSiAR62SgO8" - } + }, + "outputs": [{ + "output_type": "stream", + "name": "stdout", + "text": [ + "\n" + ] + }] }, { "cell_type": "markdown", @@ -85,7 +98,14 @@ ], "metadata": { "id": "tBtqF5UpKJNZ" - } + }, + "outputs": [{ + "output_type": "stream", + "name": "stdout", + "text": [ + "\n" + ] + }] }, { "cell_type": "markdown", @@ -97,7 +117,14 @@ ], "metadata": { "id": "SPuXFowiTpWx" - } + }, + "outputs": [{ + "output_type": "stream", + "name": "stdout", + "text": [ + "\n" + ] + }] }, { "cell_type": "code", @@ -105,7 +132,13 @@ "metadata": { "id": "1RyTYsFEIOlA" }, - "outputs": [], + "outputs": [{ + "output_type": "stream", + "name": "stdout", + "text": [ + "\n" + ] + }], "source": [ "!pip install apache_beam[gcp]>=2.46.0 --quiet\n", "!pip install tensorflow --quiet\n", @@ -140,7 +173,13 @@ "id": "Rs4cwwNrIV9H" }, "execution_count": null, - "outputs": [] + "outputs": [{ + "output_type": "stream", + "name": "stdout", + "text": [ + "\n" + ] + }] }, { "cell_type": "code", @@ -156,7 +195,13 @@ "id": "jAKpPcmmGm03" }, "execution_count": null, - "outputs": [] + "outputs": [{ + "output_type": "stream", + "name": "stdout", + "text": [ + "\n" + ] + }] }, { "cell_type": "markdown", @@ -216,7 +261,13 @@ "id": "wWjbnq6X-4uE" }, "execution_count": null, - "outputs": [] + "outputs": [{ + "output_type": "stream", + "name": "stdout", + "text": [ + "\n" + ] + }] }, { "cell_type": "markdown", @@ -239,7 +290,13 @@ "id": "lEy4PkluWbdm" }, "execution_count": null, - "outputs": [] + "outputs": [{ + "output_type": "stream", + "name": "stdout", + "text": [ + "\n" + ] + }] }, { "cell_type": "markdown", @@ -267,7 +324,13 @@ "id": "ibkWiwVNvyrn" }, "execution_count": null, - "outputs": [] + "outputs": [{ + "output_type": "stream", + "name": "stdout", + "text": [ + "\n" + ] + }] }, { "cell_type": "code", @@ -279,7 +342,13 @@ "id": "kkSnsxwUk-Sp" }, "execution_count": null, - "outputs": [] + "outputs": [{ + "output_type": "stream", + "name": "stdout", + "text": [ + "\n" + ] + }] }, { "cell_type": "markdown", @@ -306,7 +375,13 @@ "id": "dU5imgTt-8Ne" }, "execution_count": null, - "outputs": [] + "outputs": [{ + "output_type": "stream", + "name": "stdout", + "text": [ + "\n" + ] + }] }, { "cell_type": "code", @@ -329,7 +404,13 @@ "id": "6V5tJxO6-gyt" }, "execution_count": null, - "outputs": [] + "outputs": [{ + "output_type": "stream", + "name": "stdout", + "text": [ + "\n" + ] + }] }, { "cell_type": "code", @@ -341,7 +422,13 @@ "id": "GpdKk72O_NXT" }, "execution_count": null, - "outputs": [] + "outputs": [{ + "output_type": "stream", + "name": "stdout", + "text": [ + "\n" + ] + }] }, { "cell_type": "markdown", @@ -385,7 +472,13 @@ "id": "vUFStz66_Tbb" }, "execution_count": null, - "outputs": [] + "outputs": [{ + "output_type": "stream", + "name": "stdout", + "text": [ + "\n" + ] + }] }, { "cell_type": "markdown", @@ -418,7 +511,13 @@ "id": "dGg11TpV_aV6" }, "execution_count": null, - "outputs": [] + "outputs": [{ + "output_type": "stream", + "name": "stdout", + "text": [ + "\n" + ] + }] }, { "cell_type": "markdown", @@ -452,7 +551,13 @@ "id": "_AjvvexJ_hUq" }, "execution_count": null, - "outputs": [] + "outputs": [{ + "output_type": "stream", + "name": "stdout", + "text": [ + "\n" + ] + }] }, { "cell_type": "markdown", @@ -476,7 +581,13 @@ "id": "9TB76fo-_vZJ" }, "execution_count": null, - "outputs": [] + "outputs": [{ + "output_type": "stream", + "name": "stdout", + "text": [ + "\n" + ] + }] }, { "cell_type": "markdown", @@ -501,7 +612,13 @@ "id": "FpUfNBSWH9Xy" }, "execution_count": null, - "outputs": [] + "outputs": [{ + "output_type": "stream", + "name": "stdout", + "text": [ + "\n" + ] + }] }, { "cell_type": "markdown", @@ -524,7 +641,13 @@ "id": "wd0VJLeLEWBU" }, "execution_count": null, - "outputs": [] + "outputs": [{ + "output_type": "stream", + "name": "stdout", + "text": [ + "\n" + ] + }] } ] -} \ No newline at end of file +} From 97354ee83c208454a5e7b4a026ed2437a9621dfb Mon Sep 17 00:00:00 2001 From: Sam Whittle <samuelw@google.com> Date: Fri, 8 Sep 2023 13:21:47 +0200 Subject: [PATCH 307/435] Change WindmillStateReader to not batch OrderedListFetches for the same family and tag. Fix issue with MultimapState delayed fetches due to batching. --- .../windmill/state/WindmillStateReader.java | 112 ++++++++++++------ .../worker/windmill/state/WrappedFuture.java | 4 +- .../state/WindmillStateReaderTest.java | 96 +++++++++------ 3 files changed, 137 insertions(+), 75 deletions(-) diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateReader.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateReader.java index 07d652992c1c..c28939c59ee2 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateReader.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateReader.java @@ -317,47 +317,88 @@ private void delayUnbatchableMultimapFetches( } } - public void startBatchAndBlock() { - // First, drain work out of the pending lookups into a set. These will be the items we fetch. + private void delayUnbatchableOrderedListFetches( + List<StateTag<?>> orderedListTags, HashSet<StateTag<?>> toFetch) { + // Each KeyedGetDataRequest can have at most 1 TagOrderedListRequest per <tag, state_family> + // pair, thus we need to delay unbatchable ordered list requests of the same stateFamily and tag + // into later batches. + + Map<String, Map<ByteString, List<StateTag<?>>>> groupedTags = + orderedListTags.stream() + .collect( + Collectors.groupingBy( + StateTag::getStateFamily, Collectors.groupingBy(StateTag::getTag))); + + for (Map<ByteString, List<StateTag<?>>> familyTags : groupedTags.values()) { + for (List<StateTag<?>> tags : familyTags.values()) { + StateTag<?> first = tags.remove(0); + toFetch.add(first); + // Add the rest of the reads for the state family and tags back to pending. + pendingLookups.addAll(tags); + } + } + } + + private HashSet<StateTag<?>> buildFetchSet() { HashSet<StateTag<?>> toFetch = Sets.newHashSet(); - try { - List<StateTag<?>> multimapTags = Lists.newArrayList(); - while (!pendingLookups.isEmpty()) { - StateTag<?> stateTag = pendingLookups.poll(); - if (stateTag == null) { - break; - } - if (stateTag.getKind() == Kind.MULTIMAP_ALL - || stateTag.getKind() == Kind.MULTIMAP_SINGLE_ENTRY) { - multimapTags.add(stateTag); - continue; - } - if (!toFetch.add(stateTag)) { - throw new IllegalStateException("Duplicate tags being fetched."); - } + List<StateTag<?>> multimapTags = Lists.newArrayList(); + List<StateTag<?>> orderedListTags = Lists.newArrayList(); + while (!pendingLookups.isEmpty()) { + StateTag<?> stateTag = pendingLookups.poll(); + if (stateTag == null) { + break; } - if (!multimapTags.isEmpty()) { - delayUnbatchableMultimapFetches(multimapTags, toFetch); + if (stateTag.getKind() == Kind.MULTIMAP_ALL + || stateTag.getKind() == Kind.MULTIMAP_SINGLE_ENTRY) { + multimapTags.add(stateTag); + continue; + } + if (stateTag.getKind() == Kind.ORDERED_LIST) { + orderedListTags.add(stateTag); + continue; + } + + if (!toFetch.add(stateTag)) { + throw new IllegalStateException("Duplicate tags being fetched."); } + } + if (!multimapTags.isEmpty()) { + delayUnbatchableMultimapFetches(multimapTags, toFetch); + } + if (!orderedListTags.isEmpty()) { + delayUnbatchableOrderedListFetches(orderedListTags, toFetch); + } + return toFetch; + } - // If we failed to drain anything, some other thread pulled it off the queue. We have no work - // to do. + public void performReads() { + while (true) { + HashSet<StateTag<?>> toFetch = buildFetchSet(); if (toFetch.isEmpty()) { return; } - - KeyedGetDataResponse response = tryGetDataFromWindmill(toFetch); - - // Removes tags from toFetch as they are processed. - consumeResponse(response, toFetch); - } catch (Exception e) { - // Set up all the remaining futures for this key to throw an exception. This ensures that if - // the exception is caught that all futures have been completed and do not block. - for (StateTag<?> stateTag : toFetch) { - waiting.get(stateTag).future.setException(e); + try { + KeyedGetDataResponse response = tryGetDataFromWindmill(toFetch); + // Removes tags from toFetch as they are processed. + consumeResponse(response, toFetch); + if (!toFetch.isEmpty()) { + throw new IllegalStateException( + "Didn't receive responses for all pending fetches. Missing: " + toFetch); + } + } catch (Exception e) { + // Set up all the remaining futures for this key to throw an exception. This ensures that if + // the exception is caught that all futures have been completed and do not block. + for (StateTag<?> stateTag : toFetch) { + waiting.get(stateTag).future.setException(e); + } + // Also setup futures that may have been added back if they were not batched. + while (true) { + @Nullable StateTag<?> stateTag = pendingLookups.poll(); + if (stateTag == null) break; + waiting.get(stateTag).future.setException(e); + } + throw new RuntimeException(e); } - - throw new RuntimeException(e); } } @@ -643,11 +684,6 @@ private void consumeResponse(KeyedGetDataResponse response, Set<StateTag<?>> toF consumeMultimapSingleEntry(entry, entryTag); } } - - if (!toFetch.isEmpty()) { - throw new IllegalStateException( - "Didn't receive responses for all pending fetches. Missing: " + toFetch); - } } /** The deserialized values in {@code bag} as a read-only array list. */ diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WrappedFuture.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WrappedFuture.java index 035f6ec8e93d..7e894524bef3 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WrappedFuture.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WrappedFuture.java @@ -45,7 +45,7 @@ public WrappedFuture(WindmillStateReader reader, Future<T> delegate) { public T get() throws InterruptedException, ExecutionException { if (!delegate().isDone() && reader != null) { // Only one thread per reader, so no race here. - reader.startBatchAndBlock(); + reader.performReads(); } reader = null; return super.get(); @@ -56,7 +56,7 @@ public T get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException { if (!delegate().isDone() && reader != null) { // Only one thread per reader, so no race here. - reader.startBatchAndBlock(); + reader.performReads(); } reader = null; return super.get(timeout, unit); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateReaderTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateReaderTest.java index b8c4803a8f34..430e31ee04ff 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateReaderTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateReaderTest.java @@ -87,6 +87,8 @@ public class WindmillStateReaderTest { private static final ByteString STATE_KEY_2 = ByteString.copyFromUtf8("key2"); private static final String STATE_FAMILY = "family"; + private static final String STATE_FAMILY2 = "family2"; + private static void assertNoReader(Object obj) throws Exception { WindmillStateTestUtils.assertNoReference(obj, WindmillStateReader.class); } @@ -993,15 +995,19 @@ public void testReadSortedList() throws Exception { public void testReadSortedListRanges() throws Exception { Future<Iterable<TimestampedValue<Integer>>> future1 = underTest.orderedListFuture(Range.closedOpen(0L, 5L), STATE_KEY_1, STATE_FAMILY, INT_CODER); + // Should be put into a subsequent batch as it has the same key and state family. Future<Iterable<TimestampedValue<Integer>>> future2 = underTest.orderedListFuture(Range.closedOpen(5L, 6L), STATE_KEY_1, STATE_FAMILY, INT_CODER); Future<Iterable<TimestampedValue<Integer>>> future3 = underTest.orderedListFuture( - Range.closedOpen(6L, 10L), STATE_KEY_1, STATE_FAMILY, INT_CODER); + Range.closedOpen(6L, 10L), STATE_KEY_2, STATE_FAMILY, INT_CODER); + Future<Iterable<TimestampedValue<Integer>>> future4 = + underTest.orderedListFuture( + Range.closedOpen(11L, 12L), STATE_KEY_2, STATE_FAMILY2, INT_CODER); Mockito.verifyNoMoreInteractions(mockWindmill); // Fetch the entire list. - Windmill.KeyedGetDataRequest.Builder expectedRequest = + Windmill.KeyedGetDataRequest.Builder expectedRequest1 = Windmill.KeyedGetDataRequest.newBuilder() .setKey(DATA_KEY) .setShardingKey(SHARDING_KEY) @@ -1015,18 +1021,31 @@ public void testReadSortedListRanges() throws Exception { .setFetchMaxBytes(WindmillStateReader.MAX_ORDERED_LIST_BYTES)) .addSortedListsToFetch( Windmill.TagSortedListFetchRequest.newBuilder() - .setTag(STATE_KEY_1) + .setTag(STATE_KEY_2) .setStateFamily(STATE_FAMILY) - .addFetchRanges(SortedListRange.newBuilder().setStart(5).setLimit(6)) + .addFetchRanges(SortedListRange.newBuilder().setStart(6).setLimit(10)) .setFetchMaxBytes(WindmillStateReader.MAX_ORDERED_LIST_BYTES)) + .addSortedListsToFetch( + Windmill.TagSortedListFetchRequest.newBuilder() + .setTag(STATE_KEY_2) + .setStateFamily(STATE_FAMILY2) + .addFetchRanges(SortedListRange.newBuilder().setStart(11).setLimit(12)) + .setFetchMaxBytes(WindmillStateReader.MAX_ORDERED_LIST_BYTES)); + + Windmill.KeyedGetDataRequest.Builder expectedRequest2 = + Windmill.KeyedGetDataRequest.newBuilder() + .setKey(DATA_KEY) + .setShardingKey(SHARDING_KEY) + .setWorkToken(WORK_TOKEN) + .setMaxBytes(WindmillStateReader.MAX_KEY_BYTES) .addSortedListsToFetch( Windmill.TagSortedListFetchRequest.newBuilder() .setTag(STATE_KEY_1) .setStateFamily(STATE_FAMILY) - .addFetchRanges(SortedListRange.newBuilder().setStart(6).setLimit(10)) + .addFetchRanges(SortedListRange.newBuilder().setStart(5).setLimit(6)) .setFetchMaxBytes(WindmillStateReader.MAX_ORDERED_LIST_BYTES)); - Windmill.KeyedGetDataResponse.Builder response = + Windmill.KeyedGetDataResponse.Builder response1 = Windmill.KeyedGetDataResponse.newBuilder() .setKey(DATA_KEY) .addTagSortedLists( @@ -1038,41 +1057,41 @@ public void testReadSortedListRanges() throws Exception { .addFetchRanges(SortedListRange.newBuilder().setStart(0).setLimit(5))) .addTagSortedLists( Windmill.TagSortedListFetchResponse.newBuilder() - .setTag(STATE_KEY_1) + .setTag(STATE_KEY_2) .setStateFamily(STATE_FAMILY) .addEntries( - SortedListEntry.newBuilder().setValue(intData(6)).setSortKey(6000).setId(5)) - .addEntries( - SortedListEntry.newBuilder().setValue(intData(7)).setSortKey(7000).setId(7)) - .addFetchRanges(SortedListRange.newBuilder().setStart(5).setLimit(6))) + SortedListEntry.newBuilder().setValue(intData(8)).setSortKey(8000).setId(8)) + .addFetchRanges(SortedListRange.newBuilder().setStart(6).setLimit(10))) + .addTagSortedLists( + Windmill.TagSortedListFetchResponse.newBuilder() + .setTag(STATE_KEY_2) + .setStateFamily(STATE_FAMILY2) + .addFetchRanges(SortedListRange.newBuilder().setStart(11).setLimit(12))); + + Windmill.KeyedGetDataResponse.Builder response2 = + Windmill.KeyedGetDataResponse.newBuilder() + .setKey(DATA_KEY) .addTagSortedLists( Windmill.TagSortedListFetchResponse.newBuilder() .setTag(STATE_KEY_1) .setStateFamily(STATE_FAMILY) .addEntries( - SortedListEntry.newBuilder().setValue(intData(8)).setSortKey(8000).setId(8)) - .addFetchRanges(SortedListRange.newBuilder().setStart(6).setLimit(10))); - - Mockito.when(mockWindmill.getStateData(COMPUTATION, expectedRequest.build())) - .thenReturn(response.build()); + SortedListEntry.newBuilder().setValue(intData(6)).setSortKey(6000).setId(5)) + .addEntries( + SortedListEntry.newBuilder().setValue(intData(7)).setSortKey(7000).setId(7)) + .addFetchRanges(SortedListRange.newBuilder().setStart(5).setLimit(6))); - { - Iterable<TimestampedValue<Integer>> results = future1.get(); - Mockito.verify(mockWindmill).getStateData(COMPUTATION, expectedRequest.build()); - for (TimestampedValue<Integer> unused : results) { - // Iterate over the results to force loading all the pages. - } - Mockito.verifyNoMoreInteractions(mockWindmill); - assertThat(results, Matchers.contains(TimestampedValue.of(5, Instant.ofEpochMilli(5)))); - assertNoReader(future1); - } + Mockito.when(mockWindmill.getStateData(COMPUTATION, expectedRequest1.build())) + .thenReturn(response1.build()); + Mockito.when(mockWindmill.getStateData(COMPUTATION, expectedRequest2.build())) + .thenReturn(response2.build()); + // Trigger reads of batching. By fetching future2 which is not part of the first batch we ensure + // that all batches are fetched. { Iterable<TimestampedValue<Integer>> results = future2.get(); - Mockito.verify(mockWindmill).getStateData(COMPUTATION, expectedRequest.build()); - for (TimestampedValue<Integer> unused : results) { - // Iterate over the results to force loading all the pages. - } + Mockito.verify(mockWindmill).getStateData(COMPUTATION, expectedRequest1.build()); + Mockito.verify(mockWindmill).getStateData(COMPUTATION, expectedRequest2.build()); Mockito.verifyNoMoreInteractions(mockWindmill); assertThat( results, @@ -1082,16 +1101,23 @@ public void testReadSortedListRanges() throws Exception { assertNoReader(future2); } + { + Iterable<TimestampedValue<Integer>> results = future1.get(); + assertThat(results, Matchers.contains(TimestampedValue.of(5, Instant.ofEpochMilli(5)))); + assertNoReader(future1); + } + { Iterable<TimestampedValue<Integer>> results = future3.get(); - Mockito.verify(mockWindmill).getStateData(COMPUTATION, expectedRequest.build()); - for (TimestampedValue<Integer> unused : results) { - // Iterate over the results to force loading all the pages. - } - Mockito.verifyNoMoreInteractions(mockWindmill); assertThat(results, Matchers.contains(TimestampedValue.of(8, Instant.ofEpochMilli(8)))); assertNoReader(future3); } + + { + Iterable<TimestampedValue<Integer>> results = future4.get(); + assertThat(results, Matchers.emptyIterable()); + assertNoReader(future4); + } } @Test From c7dc5bd3d0d5388c4417bc7f5fbea2b0cf93e02f Mon Sep 17 00:00:00 2001 From: Danny McCormick <dannymccormick@google.com> Date: Tue, 31 Oct 2023 09:52:01 -0400 Subject: [PATCH 308/435] Add in missing space between dependency and comment --- examples/notebooks/beam-ml/run_inference_windowing.ipynb | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/notebooks/beam-ml/run_inference_windowing.ipynb b/examples/notebooks/beam-ml/run_inference_windowing.ipynb index 02b1d42f4221..a2ba3ad8dde5 100644 --- a/examples/notebooks/beam-ml/run_inference_windowing.ipynb +++ b/examples/notebooks/beam-ml/run_inference_windowing.ipynb @@ -95,7 +95,7 @@ "source": [ "!pip install apache-beam==2.47.0\n", "!pip install xgboost", - "# You may need to install a different version of Datatable directly depending on environment", + " # You may need to install a different version of Datatable directly depending on environment", "!pip install datatable" ], "metadata": { From 9a97ec841c1feaa5b3156a5969e16de626ab5258 Mon Sep 17 00:00:00 2001 From: Danny McCormick <dannymccormick@google.com> Date: Tue, 31 Oct 2023 10:24:32 -0400 Subject: [PATCH 309/435] Fix notebook installs (#29211) --- examples/notebooks/beam-ml/run_inference_windowing.ipynb | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/examples/notebooks/beam-ml/run_inference_windowing.ipynb b/examples/notebooks/beam-ml/run_inference_windowing.ipynb index a2ba3ad8dde5..27c56acdbd11 100644 --- a/examples/notebooks/beam-ml/run_inference_windowing.ipynb +++ b/examples/notebooks/beam-ml/run_inference_windowing.ipynb @@ -93,9 +93,9 @@ { "cell_type": "code", "source": [ - "!pip install apache-beam==2.47.0\n", - "!pip install xgboost", - " # You may need to install a different version of Datatable directly depending on environment", + "!pip install apache-beam>=2.47.0\n", + "!pip install xgboost\n", + "# You may need to install a different version of Datatable directly depending on environment\n", "!pip install datatable" ], "metadata": { From 9fcd0c9184479e5d3211d82747fd4f56e708cbe5 Mon Sep 17 00:00:00 2001 From: Tony Tang <tonycantang@gmail.com> Date: Tue, 31 Oct 2023 10:45:30 -0400 Subject: [PATCH 310/435] Add Cloud Bigtable Change Stream integration tests (#29127) * Add Cloud Bigtable Change Stream integration tests Change-Id: I68a877d5686f1898686b18491c6b4aff5e699862 * Add default value to bigtable environment endpoint Change-Id: I490fca7ba2f24b15faa288d6f2c3f209db59f948 * Move Bigtable options to main and register it automatically so we can set instanceId Change-Id: I4d5e5347dbba09a0e1ee170b8aa911d2f0a772ef * Remove bigtableProject from BigtableTestOptions. There was no way to set it explicitly prior to this Change-Id: I5ee3c663d3120ee85970ae5f24a962b9535323b3 * Add comment explaining why we build the test pipeline in a different package Change-Id: I228dc61ca0b27131cd38a3ab24f136d0f924d9f7 * Change instanceID to bigtableInstanceId to clarify the value we're specifying Change-Id: Ic66c4c061ed2f5979f6a530905e3cbbddd238f18 * Change BigtableChangeStreamTestOptions to use more specific field names to avoid conflicts Change-Id: I489850e07812058e8c7ebb3c9878eae9d4bc9f06 --- .../io/google-cloud-platform/build.gradle | 2 + .../BigtableChangeStreamTestOptions.java | 30 ++ .../common/GcpIoPipelineOptionsRegistrar.java | 2 + .../io/gcp/bigtable/BigtableTestUtils.java | 27 +- .../it/BigtableChangeStreamIT.java | 361 ++++++++++++++++++ ...BigtableClientIntegrationTestOverride.java | 83 ++++ 6 files changed, 504 insertions(+), 1 deletion(-) create mode 100644 sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/changestreams/BigtableChangeStreamTestOptions.java create mode 100644 sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/changestreams/it/BigtableChangeStreamIT.java create mode 100644 sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/changestreams/it/BigtableClientIntegrationTestOverride.java diff --git a/sdks/java/io/google-cloud-platform/build.gradle b/sdks/java/io/google-cloud-platform/build.gradle index d66122e4d107..b0122035a015 100644 --- a/sdks/java/io/google-cloud-platform/build.gradle +++ b/sdks/java/io/google-cloud-platform/build.gradle @@ -187,12 +187,14 @@ task integrationTest(type: Test, dependsOn: processTestResources) { def gcpTempRoot = project.findProperty('gcpTempRoot') ?: 'gs://temp-storage-for-end-to-end-tests' def firestoreDb = project.findProperty('firestoreDb') ?: 'firestoredb' def firestoreHost = project.findProperty('firestoreHost') ?: 'batch-firestore.googleapis.com:443' + def bigtableChangeStreamInstanceId = project.findProperty('bigtableChangeStreamInstanceId') ?: 'beam-test' systemProperty "beamTestPipelineOptions", JsonOutput.toJson([ "--runner=DirectRunner", "--project=${gcpProject}", "--tempRoot=${gcpTempRoot}", "--firestoreDb=${firestoreDb}", "--firestoreHost=${firestoreHost}", + "--bigtableChangeStreamInstanceId=${bigtableChangeStreamInstanceId}", ]) // Disable Gradle cache: these ITs interact with live service that should always be considered "out of date" diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/changestreams/BigtableChangeStreamTestOptions.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/changestreams/BigtableChangeStreamTestOptions.java new file mode 100644 index 000000000000..71303a0e84ac --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/changestreams/BigtableChangeStreamTestOptions.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.gcp.bigtable.changestreams; + +import org.apache.beam.sdk.options.Default; +import org.apache.beam.sdk.options.Description; +import org.apache.beam.sdk.testing.TestPipelineOptions; + +public interface BigtableChangeStreamTestOptions extends TestPipelineOptions { + @Description("Instance ID for Bigtable Change Stream") + @Default.String("beam-test") + String getBigtableChangeStreamInstanceId(); + + void setBigtableChangeStreamInstanceId(String value); +} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/common/GcpIoPipelineOptionsRegistrar.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/common/GcpIoPipelineOptionsRegistrar.java index f1ff827fc633..6cfc03c9eaa7 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/common/GcpIoPipelineOptionsRegistrar.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/common/GcpIoPipelineOptionsRegistrar.java @@ -21,6 +21,7 @@ import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryOptions; import org.apache.beam.sdk.io.gcp.bigquery.TestBigQueryOptions; +import org.apache.beam.sdk.io.gcp.bigtable.changestreams.BigtableChangeStreamTestOptions; import org.apache.beam.sdk.io.gcp.firestore.FirestoreOptions; import org.apache.beam.sdk.io.gcp.pubsub.PubsubOptions; import org.apache.beam.sdk.options.PipelineOptions; @@ -38,6 +39,7 @@ public Iterable<Class<? extends PipelineOptions>> getPipelineOptions() { .add(PubsubOptions.class) .add(FirestoreOptions.class) .add(TestBigQueryOptions.class) + .add(BigtableChangeStreamTestOptions.class) .build(); } } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableTestUtils.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableTestUtils.java index c35b7c54c4d9..6bd2f3b25b3c 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableTestUtils.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableTestUtils.java @@ -31,12 +31,14 @@ import com.google.bigtable.v2.Mutation; import com.google.protobuf.ByteString; import java.util.List; +import org.apache.beam.sdk.io.gcp.bigtable.changestreams.dao.BigtableClientOverride; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.values.KV; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.primitives.Longs; +import org.joda.time.Instant; -class BigtableTestUtils { +public class BigtableTestUtils { static final String BOOL_COLUMN = "boolColumn"; static final String LONG_COLUMN = "longColumn"; @@ -144,4 +146,27 @@ private static Cell createCell(ByteString value, long timestamp, String... label } return builder.build(); } + + // We have to build the pipeline at this package level and not changestreams package because + // endTime is package private and we can only create a pipeline with endTime here. Setting endTime + // allows the tests to predictably terminate. + public static BigtableIO.ReadChangeStream buildTestPipelineInput( + String projectId, + String instanceId, + String tableId, + String appProfileId, + String metadataTableName, + Instant startTime, + Instant endTime, + BigtableClientOverride clientOverride) { + return BigtableIO.readChangeStream() + .withProjectId(projectId) + .withInstanceId(instanceId) + .withTableId(tableId) + .withAppProfileId(appProfileId) + .withMetadataTableTableId(metadataTableName) + .withStartTime(startTime) + .withEndTime(endTime) + .withBigtableClientOverride(clientOverride); + } } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/changestreams/it/BigtableChangeStreamIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/changestreams/it/BigtableChangeStreamIT.java new file mode 100644 index 000000000000..e6455cbfd581 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/changestreams/it/BigtableChangeStreamIT.java @@ -0,0 +1,361 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.gcp.bigtable.changestreams.it; + +import com.google.api.gax.batching.Batcher; +import com.google.bigtable.v2.MutateRowsRequest; +import com.google.cloud.bigtable.admin.v2.BigtableTableAdminClient; +import com.google.cloud.bigtable.admin.v2.BigtableTableAdminSettings; +import com.google.cloud.bigtable.admin.v2.models.CreateTableRequest; +import com.google.cloud.bigtable.admin.v2.models.UpdateTableRequest; +import com.google.cloud.bigtable.data.v2.BigtableDataClient; +import com.google.cloud.bigtable.data.v2.BigtableDataSettings; +import com.google.cloud.bigtable.data.v2.models.ChangeStreamMutation; +import com.google.cloud.bigtable.data.v2.models.Range; +import com.google.cloud.bigtable.data.v2.models.RowMutationEntry; +import com.google.cloud.bigtable.data.v2.stub.EnhancedBigtableStubSettings; +import com.google.protobuf.ByteString; +import java.io.IOException; +import java.util.Arrays; +import java.util.stream.Collectors; +import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; +import org.apache.beam.sdk.io.common.IOITHelper; +import org.apache.beam.sdk.io.gcp.bigtable.BigtableTestUtils; +import org.apache.beam.sdk.io.gcp.bigtable.changestreams.BigtableChangeStreamTestOptions; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** End-to-end tests of Bigtable Change Stream. */ +@SuppressWarnings("FutureReturnValueIgnored") +@RunWith(JUnit4.class) +public class BigtableChangeStreamIT { + private static final Logger LOG = LoggerFactory.getLogger(BigtableChangeStreamIT.class); + private static final String COLUMN_FAMILY1 = "CF"; + private static final String COLUMN_FAMILY2 = "CF2"; + private static final String COLUMN_QUALIFIER = "CQ"; + private static String projectId; + private static String instanceId; + private static String tableId; + private static String appProfileId; + private static String metadataTableId; + private static BigtableTableAdminClient adminClient; + private static BigtableDataClient dataClient; + private static BigtableClientIntegrationTestOverride bigtableClientOverride; + private static Batcher<RowMutationEntry, Void> mutationBatcher; + private static BigtableChangeStreamTestOptions options; + private transient TestPipeline pipeline; + + @BeforeClass + public static void beforeClass() throws IOException { + options = IOITHelper.readIOTestPipelineOptions(BigtableChangeStreamTestOptions.class); + LOG.info("Pipeline options: {}", options); + projectId = options.as(GcpOptions.class).getProject(); + instanceId = options.getBigtableChangeStreamInstanceId(); + + long randomId = Instant.now().getMillis(); + tableId = "beam-change-stream-test-" + randomId; + metadataTableId = "beam-change-stream-test-md-" + randomId; + appProfileId = "default"; + + bigtableClientOverride = new BigtableClientIntegrationTestOverride(); + LOG.info(bigtableClientOverride.toString()); + + BigtableDataSettings.Builder dataSettingsBuilder = BigtableDataSettings.newBuilder(); + BigtableTableAdminSettings.Builder tableAdminSettingsBuilder = + BigtableTableAdminSettings.newBuilder(); + dataSettingsBuilder.setProjectId(projectId); + tableAdminSettingsBuilder.setProjectId(projectId); + dataSettingsBuilder.setInstanceId(instanceId); + tableAdminSettingsBuilder.setInstanceId(instanceId); + dataSettingsBuilder.setAppProfileId(appProfileId); + // TODO: Remove this later. But for now, disable direct path. + dataSettingsBuilder + .stubSettings() + .setTransportChannelProvider( + EnhancedBigtableStubSettings.defaultGrpcTransportProviderBuilder() + .setAttemptDirectPath(false) + .build()); + + bigtableClientOverride.updateDataClientSettings(dataSettingsBuilder); + bigtableClientOverride.updateTableAdminClientSettings(tableAdminSettingsBuilder); + + // These clients are used to modify the table and write to it + dataClient = BigtableDataClient.create(dataSettingsBuilder.build()); + adminClient = BigtableTableAdminClient.create(tableAdminSettingsBuilder.build()); + + // Create change stream enabled table + adminClient.createTable( + CreateTableRequest.of(tableId) + .addChangeStreamRetention(org.threeten.bp.Duration.ofDays(1)) + .addFamily(COLUMN_FAMILY1) + .addFamily(COLUMN_FAMILY2)); + + mutationBatcher = dataClient.newBulkMutationBatcher(tableId); + } + + @Before + public void before() { + pipeline = TestPipeline.fromOptions(options).enableAbandonedNodeEnforcement(false); + } + + @AfterClass + public static void afterClass() { + if (adminClient != null) { + if (adminClient.exists(tableId)) { + adminClient.updateTable(UpdateTableRequest.of(tableId).disableChangeStreamRetention()); + adminClient.deleteTable(tableId); + adminClient.deleteTable(metadataTableId); + } + adminClient.close(); + } + if (dataClient != null) { + dataClient.close(); + } + } + + @Test + public void testReadBigtableChangeStream() throws InterruptedException { + Instant startTime = Instant.now(); + String rowKey = "rowKeySetCell"; + RowMutationEntry setCellEntry = + RowMutationEntry.create(rowKey).setCell(COLUMN_FAMILY1, COLUMN_QUALIFIER, "cell value 1"); + mutationBatcher.add(setCellEntry); + mutationBatcher.flush(); + Instant endTime = Instant.now().plus(Duration.standardSeconds(1)); + + PCollection<MutateRowsRequest.Entry> changeStream = buildPipeline(startTime, endTime); + PAssert.that(changeStream).containsInAnyOrder(setCellEntry.toProto()); + pipeline.run().waitUntilFinish(); + } + + @Test + public void testDeleteRow() throws InterruptedException { + Instant startTime = Instant.now(); + String rowKeyToDelete = "rowKeyToDelete"; + RowMutationEntry setCellMutationToDelete = + RowMutationEntry.create(rowKeyToDelete) + .setCell(COLUMN_FAMILY1, COLUMN_QUALIFIER, "cell value 1"); + RowMutationEntry deleteRowMutation = RowMutationEntry.create(rowKeyToDelete).deleteRow(); + mutationBatcher.add(setCellMutationToDelete); + mutationBatcher.flush(); + mutationBatcher.add(deleteRowMutation); + mutationBatcher.flush(); + Instant endTime = Instant.now().plus(Duration.standardSeconds(1)); + + PCollection<MutateRowsRequest.Entry> changeStream = buildPipeline(startTime, endTime); + PAssert.that(changeStream) + .containsInAnyOrder( + setCellMutationToDelete.toProto(), + // Delete row becomes one deleteFamily per family + RowMutationEntry.create(rowKeyToDelete) + .deleteFamily(COLUMN_FAMILY1) + .deleteFamily(COLUMN_FAMILY2) + .toProto()); + pipeline.run().waitUntilFinish(); + } + + @Test + public void testDeleteColumnFamily() throws InterruptedException { + Instant startTime = Instant.now(); + String cellValue = "cell value 1"; + String rowKeyMultiFamily = "rowKeyMultiFamily"; + RowMutationEntry setCells = + RowMutationEntry.create(rowKeyMultiFamily) + .setCell(COLUMN_FAMILY1, COLUMN_QUALIFIER, cellValue) + .setCell(COLUMN_FAMILY2, COLUMN_QUALIFIER, cellValue); + mutationBatcher.add(setCells); + mutationBatcher.flush(); + RowMutationEntry deleteCF2 = + RowMutationEntry.create(rowKeyMultiFamily).deleteFamily(COLUMN_FAMILY2); + mutationBatcher.add(deleteCF2); + mutationBatcher.flush(); + Instant endTime = Instant.now().plus(Duration.standardSeconds(1)); + + PCollection<MutateRowsRequest.Entry> changeStream = buildPipeline(startTime, endTime); + PAssert.that(changeStream).containsInAnyOrder(setCells.toProto(), deleteCF2.toProto()); + pipeline.run().waitUntilFinish(); + } + + @Test + public void testDeleteCell() throws InterruptedException { + Instant startTime = Instant.now(); + String cellValue = "cell value 1"; + String rowKeyMultiCell = "rowKeyMultiCell"; + RowMutationEntry setCells = + RowMutationEntry.create(rowKeyMultiCell) + .setCell(COLUMN_FAMILY1, COLUMN_QUALIFIER, cellValue) + .setCell(COLUMN_FAMILY1, "CQ2", cellValue); + mutationBatcher.add(setCells); + mutationBatcher.flush(); + RowMutationEntry deleteCQ2 = + RowMutationEntry.create(rowKeyMultiCell) + // need to set timestamp range to make change stream output match + .deleteCells( + COLUMN_FAMILY1, + ByteString.copyFromUtf8("CQ2"), + Range.TimestampRange.create( + startTime.getMillis() * 1000, + startTime.plus(Duration.standardMinutes(2)).getMillis() * 1000)); + mutationBatcher.add(deleteCQ2); + mutationBatcher.flush(); + Instant endTime = Instant.now().plus(Duration.standardSeconds(1)); + + PCollection<MutateRowsRequest.Entry> changeStream = buildPipeline(startTime, endTime); + PAssert.that(changeStream).containsInAnyOrder(setCells.toProto(), deleteCQ2.toProto()); + pipeline.run().waitUntilFinish(); + } + + @Test + public void testComplexMutation() throws InterruptedException { + Instant startTime = Instant.now(); + String rowKey = "rowKeyComplex"; + // We'll delete this in the next mutation + RowMutationEntry setCell = + RowMutationEntry.create(rowKey).setCell(COLUMN_FAMILY1, COLUMN_QUALIFIER, "cell value 1"); + mutationBatcher.add(setCell); + mutationBatcher.flush(); + RowMutationEntry complexMutation = + RowMutationEntry.create(rowKey) + .setCell(COLUMN_FAMILY1, "CQ2", "cell value 2") + .setCell(COLUMN_FAMILY1, "CQ3", "cell value 3") + // need to set timestamp range to make change stream output match + .deleteCells( + COLUMN_FAMILY1, + ByteString.copyFromUtf8(COLUMN_QUALIFIER), + Range.TimestampRange.create( + startTime.getMillis() * 1000, + startTime.plus(Duration.standardMinutes(2)).getMillis() * 1000)); + mutationBatcher.add(complexMutation); + mutationBatcher.flush(); + Instant endTime = Instant.now().plus(Duration.standardSeconds(1)); + + PCollection<MutateRowsRequest.Entry> changeStream = buildPipeline(startTime, endTime); + PAssert.that(changeStream).containsInAnyOrder(setCell.toProto(), complexMutation.toProto()); + pipeline.run().waitUntilFinish(); + } + + @Test + public void testLargeMutation() throws InterruptedException { + Instant startTime = Instant.now(); + // test set cell w size > 1MB so it triggers chunking + char[] chars = new char[1024 * 1500]; + Arrays.fill(chars, '\u200B'); // zero-width space + String largeString = String.valueOf(chars); + String rowKeyLargeCell = "rowKeyLargeCell"; + RowMutationEntry setLargeCell = + RowMutationEntry.create(rowKeyLargeCell) + .setCell(COLUMN_FAMILY1, COLUMN_QUALIFIER, largeString); + mutationBatcher.add(setLargeCell); + mutationBatcher.flush(); + Instant endTime = Instant.now().plus(Duration.standardSeconds(1)); + + PCollection<MutateRowsRequest.Entry> changeStream = buildPipeline(startTime, endTime); + PAssert.that(changeStream).containsInAnyOrder(setLargeCell.toProto()); + pipeline.run().waitUntilFinish(); + } + + @Test + public void testManyMutations() throws InterruptedException { + Instant startTime = Instant.now(); + // test set cell w size > 1MB so it triggers chunking + char[] chars = new char[1024 * 3]; + Arrays.fill(chars, '\u200B'); // zero-width space + String largeString = String.valueOf(chars); + + ImmutableList.Builder<RowMutationEntry> originalWrites = ImmutableList.builder(); + for (int i = 0; i < 100; ++i) { + String rowKey = "rowKey" + i; + // SetCell. + RowMutationEntry setLargeCell = + RowMutationEntry.create(rowKey).setCell(COLUMN_FAMILY1, COLUMN_QUALIFIER, largeString); + // DeleteFamily. + RowMutationEntry deleteFamily = RowMutationEntry.create(rowKey).deleteFamily(COLUMN_FAMILY1); + // DeleteCells. + RowMutationEntry deleteCells = + RowMutationEntry.create(rowKey) + // need to set timestamp range to make change stream output match + .deleteCells( + COLUMN_FAMILY1, + ByteString.copyFromUtf8(COLUMN_QUALIFIER), + Range.TimestampRange.create( + startTime.getMillis() * 1000, + startTime.plus(Duration.standardMinutes(2)).getMillis() * 1000)); + // Apply the mutations. + originalWrites.add(setLargeCell); + mutationBatcher.add(setLargeCell); + mutationBatcher.flush(); + + originalWrites.add(deleteFamily); + mutationBatcher.add(deleteFamily); + mutationBatcher.flush(); + + originalWrites.add(deleteCells); + mutationBatcher.add(deleteCells); + mutationBatcher.flush(); + } + Instant endTime = Instant.now().plus(Duration.standardSeconds(1)); + + PCollection<MutateRowsRequest.Entry> changeStream = buildPipeline(startTime, endTime); + PAssert.that(changeStream) + .containsInAnyOrder( + originalWrites.build().stream() + .map(RowMutationEntry::toProto) + .collect(Collectors.toList())); + pipeline.run().waitUntilFinish(); + } + + private PCollection<MutateRowsRequest.Entry> buildPipeline(Instant startTime, Instant endTime) { + return pipeline + .apply( + BigtableTestUtils.buildTestPipelineInput( + projectId, + instanceId, + tableId, + appProfileId, + metadataTableId, + startTime, + endTime, + bigtableClientOverride)) + .apply(ParDo.of(new ConvertToEntry())); + } + + private static class ConvertToEntry + extends DoFn<KV<ByteString, ChangeStreamMutation>, MutateRowsRequest.Entry> { + @ProcessElement + public void processElement( + @Element KV<ByteString, ChangeStreamMutation> element, + OutputReceiver<MutateRowsRequest.Entry> out) { + out.output(element.getValue().toRowMutationEntry().toProto()); + } + } +} diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/changestreams/it/BigtableClientIntegrationTestOverride.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/changestreams/it/BigtableClientIntegrationTestOverride.java new file mode 100644 index 000000000000..0d6766aa20df --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/changestreams/it/BigtableClientIntegrationTestOverride.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.gcp.bigtable.changestreams.it; + +import com.google.cloud.bigtable.admin.v2.BigtableInstanceAdminSettings; +import com.google.cloud.bigtable.admin.v2.BigtableTableAdminSettings; +import com.google.cloud.bigtable.data.v2.BigtableDataSettings; +import com.google.errorprone.annotations.CheckReturnValue; +import java.io.Serializable; +import org.apache.beam.sdk.io.gcp.bigtable.changestreams.dao.BigtableClientOverride; + +/** Implements BigtableClientOverride to override data and admin endpoints. */ +@CheckReturnValue +final class BigtableClientIntegrationTestOverride implements Serializable, BigtableClientOverride { + private static final long serialVersionUID = 4188505491566837311L; + + // The address of the admin API endpoint. + private static final String ADMIN_ENDPOINT_ENV_VAR = + getenv("BIGTABLE_ENV_ADMIN_ENDPOINT", "bigtableadmin.googleapis.com:443"); + // The address of the data API endpoint. + private static final String DATA_ENDPOINT_ENV_VAR = + getenv("BIGTABLE_ENV_DATA_ENDPOINT", "bigtable.googleapis.com:443"); + + private final String adminEndpoint; + private final String dataEndpoint; + + @Override + public String toString() { + return "BigtableClientIntegrationTestOverride{" + + "adminEndpoint=" + + adminEndpoint + + ", dataEndpoint=" + + dataEndpoint + + "}"; + } + + /** Applies the test environment settings to the builder. */ + @Override + public void updateInstanceAdminClientSettings(BigtableInstanceAdminSettings.Builder builder) { + builder.stubSettings().setEndpoint(adminEndpoint); + } + + /** Applies the test environment settings to the builder. */ + @Override + public void updateTableAdminClientSettings(BigtableTableAdminSettings.Builder builder) { + builder.stubSettings().setEndpoint(adminEndpoint); + } + + /** Applies the test environment settings to the builder. */ + @Override + public void updateDataClientSettings(BigtableDataSettings.Builder builder) { + builder.stubSettings().setEndpoint(dataEndpoint); + } + + /** Returns the value of the environment variable, or default string if not found. */ + private static String getenv(String name, String defaultValue) { + final String value = System.getenv(name); + if (value != null) { + return value; + } + return defaultValue; + } + + BigtableClientIntegrationTestOverride() { + adminEndpoint = ADMIN_ENDPOINT_ENV_VAR; + dataEndpoint = DATA_ENDPOINT_ENV_VAR; + } +} From 3f434f07053d5a73be202cb6d7d70abca200f463 Mon Sep 17 00:00:00 2001 From: Yi Hu <yathu@google.com> Date: Tue, 31 Oct 2023 11:47:26 -0400 Subject: [PATCH 311/435] Add Dataflow Example Java21 PreCommit (#29170) * Add Dataflow Example Java21 PreCommit * Note that support is experimental for the first version --- ...reCommit_Java_Examples_Dataflow_Java21.yml | 139 ++++++++++++++++++ CHANGES.md | 2 +- .../examples/build.gradle | 8 +- 3 files changed, 145 insertions(+), 4 deletions(-) create mode 100644 .github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java21.yml diff --git a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java21.yml b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java21.yml new file mode 100644 index 000000000000..030c3086cc1c --- /dev/null +++ b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java21.yml @@ -0,0 +1,139 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: PreCommit Java Examples Dataflow Java21 + +on: + push: + tags: ['v*'] + branches: ['master', 'release-*'] + paths: + - 'model/**' + - 'sdks/java/**' + - 'runners/google-cloud-dataflow-java/**' + - 'examples/java/**' + - 'examples/kotlin/**' + - 'release/**' + - '.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java21.yml' + pull_request_target: + branches: ['master', 'release-*'] + paths: + - 'model/**' + - 'sdks/java/**' + - 'runners/google-cloud-dataflow-java/**' + - 'examples/java/**' + - 'examples/kotlin/**' + - 'release/**' + - 'release/trigger_all_tests.json' + issue_comment: + types: [created] + schedule: + - cron: '30 1/6 * * *' + workflow_dispatch: + +# Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: write + checks: write + contents: read + deployments: read + id-token: none + issues: write + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_PreCommit_Java_Examples_Dataflow_Java21: + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_PreCommit_Java_Examples_Dataflow_Java21"] + job_phrase: ["Run Java_Examples_Dataflow_Java21 PreCommit"] + timeout-minutes: 60 + if: | + github.event_name == 'push' || + github.event_name == 'pull_request_target' || + github.event_name == 'schedule' || + github.event_name == 'workflow_dispatch' || + github.event.comment.body == 'Run Java_Examples_Dataflow_Java21 PreCommit' + runs-on: [self-hosted, ubuntu-20.04, main] + steps: + - uses: actions/checkout@v4 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + # The test requires Java 21 and Java 8 versions. + # Java 8 is installed second because JAVA_HOME needs to point to Java 8. + - name: Setup environment + uses: ./.github/actions/setup-environment-action + with: + java-version: | + 21 + 8 + - name: Clean + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :clean + arguments: | + -PdisableSpotlessCheck=true \ + -PdisableCheckStyle=true \ + -PskipCheckerFramework \ + - name: Build and Test + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :runners:google-cloud-dataflow-java:examples:preCommit + arguments: | + -PdisableSpotlessCheck=true \ + -PdisableCheckStyle=true \ + -PtestJavaVersion=21 \ + -PskipCheckerFramework \ + -Pjava21Home=$JAVA_HOME_21_X64 \ + max-workers: 12 + - name: Archive JUnit Test Results + uses: actions/upload-artifact@v3 + if: failure() + with: + name: JUnit Test Results + path: "**/build/reports/tests/" + - name: Publish JUnit Test Results + uses: EnricoMi/publish-unit-test-result-action@v2 + if: always() + with: + commit: '${{ env.prsha || env.GITHUB_SHA }}' + comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} + files: '**/build/test-results/**/*.xml' + - name: Archive SpotBugs Results + uses: actions/upload-artifact@v3 + with: + name: SpotBugs Results + path: '**/build/reports/spotbugs/*.html' \ No newline at end of file diff --git a/CHANGES.md b/CHANGES.md index f34ffef79722..1625ce7aee90 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -62,7 +62,7 @@ Please, use `beam-sdks-java-extensions-avro` instead. This will allow to easily potential breaking changes in Beam "core" since the Beam Avro extension already supports the latest Avro versions and should handle this. ([#25252](https://github.com/apache/beam/issues/25252)). * Publishing Java 21 SDK container images now supported as part of Apache Beam release process. ([#28120](https://github.com/apache/beam/issues/28120)) - * Direct Runner and Dataflow Runner V2 support running pipeline on Java21. Support for other runners are planned in upcoming versions. + * Direct Runner and Dataflow Runner support running pipelines on Java21 (experimental until tests fully setup). For other runners (Flink, Spark, Samza, etc) support status depend on runner projects. ## I/Os diff --git a/runners/google-cloud-dataflow-java/examples/build.gradle b/runners/google-cloud-dataflow-java/examples/build.gradle index 20bc50dea5a7..96f8c07992dd 100644 --- a/runners/google-cloud-dataflow-java/examples/build.gradle +++ b/runners/google-cloud-dataflow-java/examples/build.gradle @@ -98,9 +98,11 @@ def commonConfig = { Map args -> "--region=${gcpRegion}", "--tempRoot=${actualGcsTempRoot}", "--runner=TestDataflowRunner", - "--dataflowWorkerJar=${actualDataflowWorkerJar}", - "--workerHarnessContainerImage=${actualWorkerHarnessContainerImage}" - ] + additionalOptions + "--dataflowWorkerJar=${actualDataflowWorkerJar}"] + if (actualWorkerHarnessContainerImage) { + preCommitBeamTestPipelineOptions += "--workerHarnessContainerImage=${actualWorkerHarnessContainerImage}" + } + preCommitBeamTestPipelineOptions += additionalOptions systemProperty "beamTestPipelineOptions", JsonOutput.toJson(preCommitBeamTestPipelineOptions) } } From e8849a424e72c8cd1a7cc641cd7bb5d8e17c5783 Mon Sep 17 00:00:00 2001 From: Bruno Volpato <bvolpato@google.com> Date: Tue, 31 Oct 2023 12:25:31 -0400 Subject: [PATCH 312/435] Inventory - Update temp files retention to 24h, run more frequently (#29216) --- .test-infra/jenkins/job_Inventory.groovy | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.test-infra/jenkins/job_Inventory.groovy b/.test-infra/jenkins/job_Inventory.groovy index 4723dfdf97e1..e115603d0d42 100644 --- a/.test-infra/jenkins/job_Inventory.groovy +++ b/.test-infra/jenkins/job_Inventory.groovy @@ -32,7 +32,7 @@ nums.each { commonJobProperties.setTopLevelMainJobProperties(delegate) // Sets that this is a cron job. - commonJobProperties.setCronJob(delegate, '45 6,18 * * *') + commonJobProperties.setCronJob(delegate, '45 */8 * * *') // Allows triggering this build against pull requests. commonJobProperties.enablePhraseTriggeringFromPullRequest( @@ -50,7 +50,7 @@ nums.each { } stringParam { name("tmp_unaccessed_for") - defaultValue("48") + defaultValue("24") description("Files from /tmp dir that were not accessed for last `tmp_unaccessed_for` hours will be deleted.") trim(true) } From e2a7ea66fa23497efe570e7d78758523c68c5339 Mon Sep 17 00:00:00 2001 From: Bruno Volpato <bvolpato@google.com> Date: Tue, 31 Oct 2023 12:52:27 -0400 Subject: [PATCH 313/435] [Security] Bump org.json version due to CVE-2023-5072 (#29207) --- .../main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index 5943c0c77d81..2d2ac32d815d 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -824,7 +824,7 @@ class BeamModulePlugin implements Plugin<Project> { joda_time : "joda-time:joda-time:2.10.10", jsonassert : "org.skyscreamer:jsonassert:1.5.0", jsr305 : "com.google.code.findbugs:jsr305:$jsr305_version", - json_org : "org.json:json:20230618", // Keep in sync with everit-json-schema / google_cloud_platform_libraries_bom transitive deps. + json_org : "org.json:json:20231013", // Keep in sync with everit-json-schema / google_cloud_platform_libraries_bom transitive deps. everit_json_schema : "com.github.erosb:everit-json-schema:${everit_json_version}", junit : "junit:junit:4.13.1", jupiter_api : "org.junit.jupiter:junit-jupiter-api:$jupiter_version", From aed3457d9a2d9913c59f8d94663d2e787393ec7a Mon Sep 17 00:00:00 2001 From: Robert Bradshaw <robertwb@gmail.com> Date: Tue, 31 Oct 2023 12:43:45 -0700 Subject: [PATCH 314/435] Add apache_beam[yaml] options. --- sdks/python/setup.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/sdks/python/setup.py b/sdks/python/setup.py index 01354868774c..e0bc6197e93b 100644 --- a/sdks/python/setup.py +++ b/sdks/python/setup.py @@ -398,6 +398,9 @@ def get_portability_package_data(): 'dask >= 2022.6', 'distributed >= 2022.6', ], + 'yaml': [ + 'pyyaml>=3.12,<7.0.0', + ] + dataframe_dependency }, zip_safe=False, # PyPI package information. From 8fcdf6a2975642ac81dacdffe2735cbae776f128 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw <robertwb@gmail.com> Date: Tue, 31 Oct 2023 12:45:17 -0700 Subject: [PATCH 315/435] Update readme to refere to new options. --- sdks/python/apache_beam/yaml/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/yaml/README.md b/sdks/python/apache_beam/yaml/README.md index 62c0d0eea162..a3a2a4616144 100644 --- a/sdks/python/apache_beam/yaml/README.md +++ b/sdks/python/apache_beam/yaml/README.md @@ -483,7 +483,7 @@ The Beam yaml parser is currently included as part of the Apache Beam Python SDK This can be installed (e.g. within a virtual environment) as ``` -pip install apache_beam +pip install apache_beam[yaml] ``` In addition, several of the provided transforms (such as SQL) are implemented From a036a18117059e3fbcc06b7e6fbc2f8c67c74d4e Mon Sep 17 00:00:00 2001 From: Yi Hu <yathu@google.com> Date: Tue, 31 Oct 2023 15:51:59 -0400 Subject: [PATCH 316/435] Delete on exit beam-artifact temp file (#29219) --- .../org/apache/beam/runners/core/construction/External.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java index 534a2b5fe0e6..93a1ade474a5 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java @@ -19,6 +19,7 @@ import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; +import java.io.File; import java.io.FileOutputStream; import java.io.IOException; import java.nio.file.Files; @@ -391,7 +392,8 @@ private static List<RunnerApi.ArtifactInformation> resolveArtifacts( .build()) .getReplacementsList()) { Path path = Files.createTempFile("beam-artifact", ""); - try (FileOutputStream fout = new FileOutputStream(path.toFile())) { + File artifactFile = path.toFile(); + try (FileOutputStream fout = new FileOutputStream(artifactFile)) { for (Iterator<ArtifactApi.GetArtifactResponse> it = retrievalStub.getArtifact( ArtifactApi.GetArtifactRequest.newBuilder().setArtifact(artifact).build()); @@ -409,6 +411,8 @@ private static List<RunnerApi.ArtifactInformation> resolveArtifacts( .build() .toByteString()) .build()); + // Delete beam-artifact temp File on program exit + artifactFile.deleteOnExit(); } return resolved; } From 99e16d3f65fcd5910480d6e6cefe9a425f1fa5b6 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw <robertwb@gmail.com> Date: Tue, 31 Oct 2023 12:57:36 -0700 Subject: [PATCH 317/435] Also include gcp for ease of use. --- sdks/python/apache_beam/yaml/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/yaml/README.md b/sdks/python/apache_beam/yaml/README.md index a3a2a4616144..247b42b6839a 100644 --- a/sdks/python/apache_beam/yaml/README.md +++ b/sdks/python/apache_beam/yaml/README.md @@ -483,7 +483,7 @@ The Beam yaml parser is currently included as part of the Apache Beam Python SDK This can be installed (e.g. within a virtual environment) as ``` -pip install apache_beam[yaml] +pip install apache_beam[yaml,gcp] ``` In addition, several of the provided transforms (such as SQL) are implemented From e4bd6183b70c4bcb9db1b86aafaa3be0a0e75a7a Mon Sep 17 00:00:00 2001 From: Jeff Kinard <35542536+Polber@users.noreply.github.com> Date: Tue, 31 Oct 2023 16:24:36 -0400 Subject: [PATCH 318/435] [YAML] Add JDBC branding transforms (#29171) Add specific jdbc io's for mysql, postgres, oracle and sql server. --- .../schemaio-expansion-service/build.gradle | 4 ++ sdks/python/apache_beam/yaml/standard_io.yaml | 41 +++++++++++++++++++ sdks/python/apache_beam/yaml/yaml_provider.py | 18 +++++++- 3 files changed, 62 insertions(+), 1 deletion(-) diff --git a/sdks/java/extensions/schemaio-expansion-service/build.gradle b/sdks/java/extensions/schemaio-expansion-service/build.gradle index 246c0c155cbd..ae6599dfbb99 100644 --- a/sdks/java/extensions/schemaio-expansion-service/build.gradle +++ b/sdks/java/extensions/schemaio-expansion-service/build.gradle @@ -46,6 +46,10 @@ dependencies { implementation project(path: ":sdks:java:core", configuration: "shadow") implementation library.java.vendored_grpc_1_54_0 implementation library.java.vendored_guava_32_1_2_jre + // Stage default drivers for JdbcIO schema transforms + implementation 'mysql:mysql-connector-java:8.0.22' + implementation 'com.oracle.database.jdbc:ojdbc8:23.2.0.0' + implementation 'com.microsoft.sqlserver:mssql-jdbc:12.2.0.jre11' testImplementation library.java.junit testImplementation library.java.mockito_core } diff --git a/sdks/python/apache_beam/yaml/standard_io.yaml b/sdks/python/apache_beam/yaml/standard_io.yaml index 6499ffc7e55a..236046abb9b7 100644 --- a/sdks/python/apache_beam/yaml/standard_io.yaml +++ b/sdks/python/apache_beam/yaml/standard_io.yaml @@ -138,6 +138,14 @@ transforms: 'ReadFromJdbc': 'ReadFromJdbc' 'WriteToJdbc': 'WriteToJdbc' + 'ReadFromMySql': 'ReadFromJdbc' + 'WriteToMySql': 'WriteToJdbc' + 'ReadFromPostgres': 'ReadFromJdbc' + 'WriteToPostgres': 'WriteToJdbc' + 'ReadFromOracle': 'ReadFromJdbc' + 'WriteToOracle': 'WriteToJdbc' + 'ReadFromSqlServer': 'ReadFromJdbc' + 'WriteToSqlServer': 'WriteToJdbc' config: mappings: 'ReadFromJdbc': @@ -159,10 +167,43 @@ driver_jars: 'driverJars' connection_properties: 'connectionProperties' connection_init_sql: 'connectionInitSql' + 'ReadFromMySql': 'ReadFromJdbc' + 'WriteToMySql': 'WriteToJdbc' + 'ReadFromPostgres': 'ReadFromJdbc' + 'WriteToPostgres': 'WriteToJdbc' + 'ReadFromOracle': 'ReadFromJdbc' + 'WriteToOracle': 'WriteToJdbc' + 'ReadFromSqlServer': 'ReadFromJdbc' + 'WriteToSqlServer': 'WriteToJdbc' + defaults: + 'ReadFromMySql': + driverClassName: 'com.mysql.jdbc.Driver' + 'WriteToMySql': + driverClassName: 'com.mysql.jdbc.Driver' + 'ReadFromPostgres': + driverClassName: 'org.postgresql.Driver' + 'WriteToPostgres': + driverClassName: 'org.postgresql.Driver' + 'ReadFromOracle': + driverClassName: 'oracle.jdbc.driver.OracleDriver' + 'WriteToOracle': + driverClassName: 'oracle.jdbc.driver.OracleDriver' + 'ReadFromSqlServer': + driverClassName: 'com.microsoft.sqlserver.jdbc.SQLServerDriver' + 'WriteToSqlServer': + driverClassName: 'com.microsoft.sqlserver.jdbc.SQLServerDriver' underlying_provider: type: beamJar transforms: 'ReadFromJdbc': 'beam:schematransform:org.apache.beam:jdbc_read:v1' 'WriteToJdbc': 'beam:schematransform:org.apache.beam:jdbc_write:v1' + 'ReadFromMySql': 'beam:schematransform:org.apache.beam:jdbc_read:v1' + 'WriteToMySql': 'beam:schematransform:org.apache.beam:jdbc_write:v1' + 'ReadFromPostgres': 'beam:schematransform:org.apache.beam:jdbc_read:v1' + 'WriteToPostgres': 'beam:schematransform:org.apache.beam:jdbc_write:v1' + 'ReadFromOracle': 'beam:schematransform:org.apache.beam:jdbc_read:v1' + 'WriteToOracle': 'beam:schematransform:org.apache.beam:jdbc_write:v1' + 'ReadFromSqlServer': 'beam:schematransform:org.apache.beam:jdbc_read:v1' + 'WriteToSqlServer': 'beam:schematransform:org.apache.beam:jdbc_write:v1' config: gradle_target: 'sdks:java:extensions:schemaio-expansion-service:shadowJar' diff --git a/sdks/python/apache_beam/yaml/yaml_provider.py b/sdks/python/apache_beam/yaml/yaml_provider.py index a21db09b50ca..2d7471e506b0 100644 --- a/sdks/python/apache_beam/yaml/yaml_provider.py +++ b/sdks/python/apache_beam/yaml/yaml_provider.py @@ -739,9 +739,25 @@ def __init__(self, transforms, mappings, underlying_provider, defaults=None): for transform in transforms.keys(): if transform not in mappings: raise ValueError(f'Missing transform {transform} in mappings.') - self._mappings = mappings + self._mappings = self.expand_mappings(mappings) self._defaults = defaults or {} + @staticmethod + def expand_mappings(mappings): + if not isinstance(mappings, dict): + raise ValueError( + "RenamingProvider mappings must be dict of transform " + "mappings.") + for key, value in mappings.items(): + if isinstance(value, str): + if value not in mappings.keys(): + raise ValueError( + "RenamingProvider transform mappings must be dict or " + "specify transform that has mappings within same " + "provider.") + mappings[key] = mappings[value] + return mappings + def available(self) -> bool: return self._underlying_provider.available() From f2f88c5f3a17cb4859c6b24844669b94b0c9b779 Mon Sep 17 00:00:00 2001 From: Yi Hu <yathu@google.com> Date: Tue, 31 Oct 2023 16:25:18 -0400 Subject: [PATCH 319/435] Fix launcher artifact gradle configuration name conflict (#29224) * Rename sdkHarnessLauncher in java container and python container gradle project --- sdks/java/container/build.gradle | 4 ++-- sdks/java/container/common.gradle | 6 +++--- sdks/python/container/build.gradle | 4 ++-- sdks/python/container/common.gradle | 8 ++++---- 4 files changed, 11 insertions(+), 11 deletions(-) diff --git a/sdks/java/container/build.gradle b/sdks/java/container/build.gradle index 0cafdf511854..ac4104638b65 100644 --- a/sdks/java/container/build.gradle +++ b/sdks/java/container/build.gradle @@ -31,7 +31,7 @@ description = "Apache Beam :: SDKs :: Java :: Container" configurations { dockerDependency - sdkHarnessLauncher + javaHarnessLauncher } dependencies { @@ -76,7 +76,7 @@ task downloadCloudProfilerAgent(type: Exec) { } artifacts { - sdkHarnessLauncher file: file('./build/target'), builtBy: goBuild + javaHarnessLauncher file: file('./build/target'), builtBy: goBuild } task pushAll { diff --git a/sdks/java/container/common.gradle b/sdks/java/container/common.gradle index 6be531bc5e80..e4488d20f4c7 100644 --- a/sdks/java/container/common.gradle +++ b/sdks/java/container/common.gradle @@ -36,13 +36,13 @@ description = "Apache Beam :: SDKs :: Java :: Container :: Java ${imageJavaVersi configurations { dockerDependency - sdkHarnessLauncher + javaHarnessLauncher pulledLicenses } dependencies { dockerDependency project(path: ":sdks:java:container", configuration: "dockerDependency") - sdkHarnessLauncher project(path: ":sdks:java:container", configuration: "sdkHarnessLauncher") + javaHarnessLauncher project(path: ":sdks:java:container", configuration: "javaHarnessLauncher") } task copyDockerfileDependencies(type: Copy) { @@ -67,7 +67,7 @@ task copySdkHarnessLauncher(type: Copy) { // if licenses are required, they should be present before this task run. mustRunAfter ":sdks:java:container:pullLicenses" - from configurations.sdkHarnessLauncher + from configurations.javaHarnessLauncher into "build/target" } diff --git a/sdks/python/container/build.gradle b/sdks/python/container/build.gradle index 06b1ea918c7f..161d343b303a 100644 --- a/sdks/python/container/build.gradle +++ b/sdks/python/container/build.gradle @@ -25,7 +25,7 @@ int max_python_version=11 configurations { sdkSourceTarball - sdkHarnessLauncher + pythonHarnessLauncher } dependencies { @@ -82,5 +82,5 @@ tasks.register("generatePythonRequirementsAll") { } artifacts { - sdkHarnessLauncher file: file('./build/target/launcher'), builtBy: goBuild + pythonHarnessLauncher file: file('./build/target/launcher'), builtBy: goBuild } diff --git a/sdks/python/container/common.gradle b/sdks/python/container/common.gradle index bb706aa5c5d8..efee0fbd2112 100644 --- a/sdks/python/container/common.gradle +++ b/sdks/python/container/common.gradle @@ -22,12 +22,12 @@ description = "Apache Beam :: SDKs :: Python :: Container :: Python ${pythonVers configurations { sdkSourceTarball - sdkHarnessLauncher + pythonHarnessLauncher } dependencies { sdkSourceTarball project(path: ":sdks:python", configuration: "distTarBall") - sdkHarnessLauncher project(path: ":sdks:python:container", configuration: "sdkHarnessLauncher") + pythonHarnessLauncher project(path: ":sdks:python:container", configuration: "pythonHarnessLauncher") } def generatePythonRequirements = tasks.register("generatePythonRequirements") { @@ -59,9 +59,9 @@ def copyLicenseScripts = tasks.register("copyLicenseScripts", Copy){ } def copyLauncherDependencies = tasks.register("copyLauncherDependencies", Copy) { - from configurations.sdkHarnessLauncher + from configurations.pythonHarnessLauncher into "build/target/launcher" - if(configurations.sdkHarnessLauncher.isEmpty()) { + if(configurations.pythonHarnessLauncher.isEmpty()) { throw new StopExecutionException(); } } From 9101dc1f842266522843b8aaa983d0932cb6df90 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw <robertwb@gmail.com> Date: Tue, 31 Oct 2023 14:02:36 -0700 Subject: [PATCH 320/435] Avoid potential null pointer exceptions in logging. --- .../apache/beam/fn/harness/logging/BeamFnLoggingClient.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/logging/BeamFnLoggingClient.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/logging/BeamFnLoggingClient.java index 8fa074b04768..c16296be717d 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/logging/BeamFnLoggingClient.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/logging/BeamFnLoggingClient.java @@ -408,11 +408,15 @@ public void publish(LogRecord record) { if (severity == null) { return; } + if (record == null) { + return; + } + String messageString = getFormatter().formatMessage(record); BeamFnApi.LogEntry.Builder builder = BeamFnApi.LogEntry.newBuilder() .setSeverity(severity) - .setMessage(getFormatter().formatMessage(record)) + .setMessage(messageString == null ? "null" : messageString) .setThread(Integer.toString(record.getThreadID())) .setTimestamp( Timestamp.newBuilder() From d329a7e9b85d729a474395abfe1975c0f3649550 Mon Sep 17 00:00:00 2001 From: Anand Inguva <34158215+AnandInguva@users.noreply.github.com> Date: Tue, 31 Oct 2023 22:21:15 +0000 Subject: [PATCH 321/435] [Python]Enable state cache to 100 MB (#28781) * change state_cache_mb to 100 MB as default * Address comments * Reword help of pipeline option * Reword help of pipeline option * Fix doc string * reword docstring * Set default in the pipeline options * Reword documentation * Add warning * Address comments * Update CHANGES.md --- CHANGES.md | 1 + .../apache_beam/options/pipeline_options.py | 16 ++++++++++++ .../runners/portability/portable_runner.py | 3 ++- .../runners/worker/sdk_worker_main.py | 26 ++++++++++++------- .../runners/worker/sdk_worker_main_test.py | 13 ++++++++++ 5 files changed, 48 insertions(+), 11 deletions(-) diff --git a/CHANGES.md b/CHANGES.md index 1625ce7aee90..366f3236df9b 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -74,6 +74,7 @@ should handle this. ([#25252](https://github.com/apache/beam/issues/25252)). jobs using the DataStream API. By default the option is set to false, so the batch jobs are still executed using the DataSet API. * `upload_graph` as one of the Experiments options for DataflowRunner is no longer required when the graph is larger than 10MB for Java SDK ([PR#28621](https://github.com/apache/beam/pull/28621). +* state amd side input cache has been enabled to a default of 100 MB. Use `--max_cache_memory_usage_mb=X` to provide cache size for the user state API and side inputs. (Python) ([#28770](https://github.com/apache/beam/issues/28770)). ## Breaking Changes diff --git a/sdks/python/apache_beam/options/pipeline_options.py b/sdks/python/apache_beam/options/pipeline_options.py index 76b776779cf9..bbdafb540976 100644 --- a/sdks/python/apache_beam/options/pipeline_options.py +++ b/sdks/python/apache_beam/options/pipeline_options.py @@ -1135,6 +1135,22 @@ def _add_argparse_args(cls, parser): dest='min_cpu_platform', type=str, help='GCE minimum CPU platform. Default is determined by GCP.') + parser.add_argument( + '--max_cache_memory_usage_mb', + dest='max_cache_memory_usage_mb', + type=int, + default=100, + help=( + 'Size of the SDK Harness cache to store user state and side ' + 'inputs in MB. Default is 100MB. If the cache is full, least ' + 'recently used elements will be evicted. This cache is per ' + 'each SDK Harness instance. SDK Harness is a component ' + 'responsible for executing the user code and communicating with ' + 'the runner. Depending on the runner, there may be more than one ' + 'SDK Harness process running on the same worker node. Increasing ' + 'cache size might improve performance of some pipelines, but can ' + 'lead to an increase in memory consumption and OOM errors if ' + 'workers are not appropriately provisioned.')) def validate(self, validator): errors = [] diff --git a/sdks/python/apache_beam/runners/portability/portable_runner.py b/sdks/python/apache_beam/runners/portability/portable_runner.py index 9ff03ec1d061..ab5ee9fff6f9 100644 --- a/sdks/python/apache_beam/runners/portability/portable_runner.py +++ b/sdks/python/apache_beam/runners/portability/portable_runner.py @@ -415,7 +415,8 @@ def start_and_replace_loopback_environments(pipeline, options): portable_options.environment_config, server = ( worker_pool_main.BeamFnExternalWorkerPoolServicer.start( state_cache_size= - sdk_worker_main._get_state_cache_size(experiments), + sdk_worker_main._get_state_cache_size_bytes( + options=options), data_buffer_time_limit_ms= sdk_worker_main._get_data_buffer_time_limit_ms(experiments), use_process=use_loopback_process_worker)) diff --git a/sdks/python/apache_beam/runners/worker/sdk_worker_main.py b/sdks/python/apache_beam/runners/worker/sdk_worker_main.py index 1e44a998ba05..1af0071edc14 100644 --- a/sdks/python/apache_beam/runners/worker/sdk_worker_main.py +++ b/sdks/python/apache_beam/runners/worker/sdk_worker_main.py @@ -36,6 +36,7 @@ from apache_beam.options.pipeline_options import PipelineOptions from apache_beam.options.pipeline_options import ProfilingOptions from apache_beam.options.pipeline_options import SetupOptions +from apache_beam.options.pipeline_options import WorkerOptions from apache_beam.options.value_provider import RuntimeValueProvider from apache_beam.portability.api import endpoints_pb2 from apache_beam.runners.internal import names @@ -159,7 +160,8 @@ def create_harness(environment, dry_run=False): control_address=control_service_descriptor.url, status_address=status_service_descriptor.url, worker_id=_worker_id, - state_cache_size=_get_state_cache_size(experiments), + state_cache_size=_get_state_cache_size_bytes( + options=sdk_pipeline_options), data_buffer_time_limit_ms=_get_data_buffer_time_limit_ms(experiments), profiler_factory=profiler.Profile.factory_from_options( sdk_pipeline_options.view_as(ProfilingOptions)), @@ -239,24 +241,28 @@ def _parse_pipeline_options(options_json): return PipelineOptions.from_dictionary(_load_pipeline_options(options_json)) -def _get_state_cache_size(experiments): - """Defines the upper number of state items to cache. - - Note: state_cache_size is an experimental flag and might not be available in - future releases. +def _get_state_cache_size_bytes(options): + """Return the maximum size of state cache in bytes. Returns: - an int indicating the maximum number of megabytes to cache. - Default is 100 MB + an int indicating the maximum number of bytes to cache. """ - + max_cache_memory_usage_mb = options.view_as( + WorkerOptions).max_cache_memory_usage_mb + # to maintain backward compatibility + experiments = options.view_as(DebugOptions).experiments or [] for experiment in experiments: # There should only be 1 match so returning from the loop if re.match(r'state_cache_size=', experiment): + _LOGGER.warning( + '--experiments=state_cache_size=X is deprecated and will be removed ' + 'in future releases.' + 'Please use --max_cache_memory_usage_mb=X to set the cache size for ' + 'user state API and side inputs.') return int( re.match(r'state_cache_size=(?P<state_cache_size>.*)', experiment).group('state_cache_size')) << 20 - return 100 << 20 + return max_cache_memory_usage_mb << 20 def _get_data_buffer_time_limit_ms(experiments): diff --git a/sdks/python/apache_beam/runners/worker/sdk_worker_main_test.py b/sdks/python/apache_beam/runners/worker/sdk_worker_main_test.py index 00e09840787f..498a07b70e9e 100644 --- a/sdks/python/apache_beam/runners/worker/sdk_worker_main_test.py +++ b/sdks/python/apache_beam/runners/worker/sdk_worker_main_test.py @@ -234,6 +234,19 @@ def test_gcp_profiler_uses_job_name_when_enabled_as_experiment(self): sdk_worker_main._start_profiler(gcp_profiler_name, "version") sdk_worker_main._start_profiler.assert_called_with("sample_job", "version") + @unittest.mock.patch.dict(os.environ, {"JOB_NAME": "sample_job"}, clear=True) + def test_pipeline_option_max_cache_memory_usage_mb(self): + options = PipelineOptions(flags=['--max_cache_memory_usage_mb=50']) + + cache_size = sdk_worker_main._get_state_cache_size_bytes(options) + self.assertEqual(cache_size, 50 << 20) + + @unittest.mock.patch.dict(os.environ, {"JOB_NAME": "sample_job"}, clear=True) + def test_pipeline_option_max_cache_memory_usage_mb_with_experiments(self): + options = PipelineOptions(flags=['--experiments=state_cache_size=50']) + cache_size = sdk_worker_main._get_state_cache_size_bytes(options) + self.assertEqual(cache_size, 50 << 20) + if __name__ == '__main__': logging.getLogger().setLevel(logging.INFO) From 96d6f2d07e8afb7eaabab2fe415b54f93b15da18 Mon Sep 17 00:00:00 2001 From: Vlado Djerek <rage.bubblegum@gmail.com> Date: Tue, 31 Oct 2023 23:29:08 +0100 Subject: [PATCH 322/435] prevent performance cdap on forks (#29194) * prevent performance cdap on forks * update all scheduled runs to include fork prevention conditonal * add conditional to publish and release jobs --------- Co-authored-by: Vlado Djerek <djerek.vlado6@gmail.com> --- .github/workflows/README.md | 4 ++-- .github/workflows/beam_CancelStaleDataflowJobs.yml | 2 +- .github/workflows/beam_CleanUpDataprocResources.yml | 2 +- .github/workflows/beam_CleanUpGCPResources.yml | 2 +- .github/workflows/beam_CleanUpPrebuiltSDKImages.yml | 2 +- .github/workflows/beam_CloudML_Benchmarks_Dataflow.yml | 2 +- .github/workflows/beam_IODatastoresCredentialsRotation.yml | 2 +- .../beam_Inference_Python_Benchmarks_Dataflow.yml | 2 +- .github/workflows/beam_Java_JMH.yml | 2 +- .github/workflows/beam_Java_LoadTests_Combine_Smoke.yml | 2 +- .../workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml | 2 +- .github/workflows/beam_LoadTests_Go_CoGBK_Flink_batch.yml | 2 +- .../workflows/beam_LoadTests_Go_Combine_Dataflow_Batch.yml | 2 +- .../workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml | 2 +- .github/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml | 2 +- .github/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml | 2 +- .../workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml | 2 +- .github/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml | 2 +- .../beam_LoadTests_Go_SideInput_Dataflow_Batch.yml | 2 +- .../workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml | 2 +- .../workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml | 2 +- .../beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml | 2 +- ...LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml | 2 +- ...Tests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml | 2 +- ...LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml | 2 +- .../beam_LoadTests_Java_Combine_Dataflow_Batch.yml | 2 +- .../beam_LoadTests_Java_Combine_Dataflow_Streaming.yml | 2 +- ...adTests_Java_Combine_SparkStructuredStreaming_Batch.yml | 2 +- .../workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml | 2 +- .../beam_LoadTests_Java_GBK_Dataflow_Streaming.yml | 2 +- .../beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml | 2 +- .../beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml | 2 +- ...eam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml | 2 +- ...eam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml | 2 +- ...m_LoadTests_Java_GBK_SparkStructuredStreaming_Batch.yml | 2 +- .../workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml | 2 +- .../beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml | 2 +- ...LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions.yml | 2 +- ...Tests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions.yml | 2 +- ...LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml | 2 +- .../beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml | 2 +- .../beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml | 2 +- .../workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml | 2 +- .../beam_LoadTests_Python_Combine_Dataflow_Batch.yml | 2 +- .../beam_LoadTests_Python_Combine_Dataflow_Streaming.yml | 2 +- .../beam_LoadTests_Python_Combine_Flink_Batch.yml | 2 +- .../beam_LoadTests_Python_Combine_Flink_Streaming.yml | 2 +- .../beam_LoadTests_Python_FnApiRunner_Microbenchmark.yml | 2 +- .../workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml | 2 +- .../beam_LoadTests_Python_GBK_Dataflow_Streaming.yml | 2 +- .../workflows/beam_LoadTests_Python_GBK_Flink_Batch.yml | 2 +- .../beam_LoadTests_Python_GBK_reiterate_Dataflow_Batch.yml | 2 +- ...m_LoadTests_Python_GBK_reiterate_Dataflow_Streaming.yml | 2 +- .../beam_LoadTests_Python_ParDo_Dataflow_Batch.yml | 2 +- .../beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml | 2 +- .../workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml | 2 +- .../beam_LoadTests_Python_ParDo_Flink_Streaming.yml | 2 +- .../beam_LoadTests_Python_SideInput_Dataflow_Batch.yml | 2 +- .github/workflows/beam_MetricsCredentialsRotation.yml | 5 ++--- .github/workflows/beam_Metrics_Report.yml | 6 +++--- .github/workflows/beam_PerformanceTests_AvroIOIT.yml | 2 +- .github/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml | 2 +- .../beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml | 2 +- .../beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml | 2 +- .../beam_PerformanceTests_BigQueryIO_Streaming_Java.yml | 2 +- .../beam_PerformanceTests_BiqQueryIO_Read_Python.yml | 2 +- ...beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml | 2 +- .github/workflows/beam_PerformanceTests_Cdap.yml | 2 +- .../beam_PerformanceTests_Compressed_TextIOIT.yml | 2 +- .../beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml | 2 +- .github/workflows/beam_PerformanceTests_HadoopFormat.yml | 2 +- .github/workflows/beam_PerformanceTests_JDBC.yml | 2 +- .github/workflows/beam_PerformanceTests_Kafka_IO.yml | 2 +- .../workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml | 2 +- .../beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml | 2 +- .github/workflows/beam_PerformanceTests_MongoDBIO_IT.yml | 2 +- .github/workflows/beam_PerformanceTests_ParquetIOIT.yml | 2 +- .../workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml | 2 +- .../beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml | 2 +- .../beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml | 2 +- .github/workflows/beam_PerformanceTests_SingleStoreIO.yml | 2 +- .../beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml | 2 +- ...m_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml | 2 +- .../workflows/beam_PerformanceTests_SparkReceiver_IO.yml | 2 +- .github/workflows/beam_PerformanceTests_TFRecordIOIT.yml | 2 +- .../workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml | 2 +- .github/workflows/beam_PerformanceTests_TextIOIT.yml | 2 +- .github/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml | 2 +- .../workflows/beam_PerformanceTests_TextIOIT_Python.yml | 2 +- .../beam_PerformanceTests_WordCountIT_PythonVersions.yml | 2 +- .github/workflows/beam_PerformanceTests_XmlIOIT.yml | 2 +- .github/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml | 2 +- .../beam_PerformanceTests_xlang_KafkaIO_Python.yml | 2 +- .github/workflows/beam_PostCommit_BeamMetrics_Publish.yml | 7 +++---- .github/workflows/beam_PostCommit_Go.yml | 2 +- .github/workflows/beam_PostCommit_Go_Dataflow_ARM.yml | 2 +- .github/workflows/beam_PostCommit_Go_VR_Flink.yml | 2 +- .github/workflows/beam_PostCommit_Go_VR_Samza.yml | 2 +- .github/workflows/beam_PostCommit_Go_VR_Spark.yml | 2 +- .github/workflows/beam_PostCommit_Java.yml | 2 +- .github/workflows/beam_PostCommit_Java_Avro_Versions.yml | 2 +- .../beam_PostCommit_Java_BigQueryEarlyRollout.yml | 2 +- .github/workflows/beam_PostCommit_Java_DataflowV1.yml | 2 +- .github/workflows/beam_PostCommit_Java_DataflowV2.yml | 2 +- .../workflows/beam_PostCommit_Java_Examples_Dataflow.yml | 2 +- .../beam_PostCommit_Java_Examples_Dataflow_ARM.yml | 2 +- .../beam_PostCommit_Java_Examples_Dataflow_Java.yml | 2 +- .../beam_PostCommit_Java_Examples_Dataflow_V2.yml | 2 +- .../beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml | 2 +- .github/workflows/beam_PostCommit_Java_Examples_Direct.yml | 2 +- .github/workflows/beam_PostCommit_Java_Examples_Flink.yml | 2 +- .github/workflows/beam_PostCommit_Java_Examples_Spark.yml | 2 +- .github/workflows/beam_PostCommit_Java_Hadoop_Versions.yml | 2 +- .../beam_PostCommit_Java_IO_Performance_Tests.yml | 2 +- .github/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml | 2 +- .../beam_PostCommit_Java_Jpms_Dataflow_Java11.yml | 2 +- .../beam_PostCommit_Java_Jpms_Dataflow_Java17.yml | 2 +- .../workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml | 2 +- .../workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml | 2 +- .../workflows/beam_PostCommit_Java_Jpms_Direct_Java21.yml | 2 +- .../workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml | 2 +- .../workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml | 2 +- .../workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml | 2 +- .../workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml | 2 +- .../beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml | 2 +- .github/workflows/beam_PostCommit_Java_Nexmark_Direct.yml | 2 +- .github/workflows/beam_PostCommit_Java_Nexmark_Flink.yml | 2 +- .github/workflows/beam_PostCommit_Java_Nexmark_Spark.yml | 2 +- .../workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml | 2 +- .github/workflows/beam_PostCommit_Java_PVR_Samza.yml | 2 +- .../beam_PostCommit_Java_PVR_Spark3_Streaming.yml | 2 +- .github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml | 2 +- .github/workflows/beam_PostCommit_Java_Sickbay.yml | 2 +- .../workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml | 2 +- .github/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml | 2 +- .github/workflows/beam_PostCommit_Java_Tpcds_Flink.yml | 2 +- .github/workflows/beam_PostCommit_Java_Tpcds_Spark.yml | 2 +- .../beam_PostCommit_Java_ValidatesRunner_Dataflow.yml | 2 +- ...stCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml | 2 +- ..._PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml | 2 +- .../beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml | 2 +- ...stCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml | 2 +- .../beam_PostCommit_Java_ValidatesRunner_Direct.yml | 2 +- ...PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml | 2 +- .../beam_PostCommit_Java_ValidatesRunner_Flink.yml | 2 +- .../beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml | 2 +- .../beam_PostCommit_Java_ValidatesRunner_Samza.yml | 2 +- .../beam_PostCommit_Java_ValidatesRunner_Spark.yml | 2 +- ...ommit_Java_ValidatesRunner_SparkStructuredStreaming.yml | 2 +- .../beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml | 2 +- .../beam_PostCommit_Java_ValidatesRunner_Twister2.yml | 2 +- .../workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml | 2 +- .github/workflows/beam_PostCommit_Javadoc.yml | 2 +- .github/workflows/beam_PostCommit_PortableJar_Flink.yml | 2 +- .github/workflows/beam_PostCommit_PortableJar_Spark.yml | 2 +- .github/workflows/beam_PostCommit_Python.yml | 2 +- .github/workflows/beam_PostCommit_Python_Arm.yml | 2 +- .../workflows/beam_PostCommit_Python_Examples_Dataflow.yml | 2 +- .../workflows/beam_PostCommit_Python_Examples_Direct.yml | 2 +- .../workflows/beam_PostCommit_Python_Examples_Flink.yml | 2 +- .../workflows/beam_PostCommit_Python_Examples_Spark.yml | 2 +- .github/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml | 2 +- .../workflows/beam_PostCommit_Python_Nexmark_Direct.yml | 2 +- .../beam_PostCommit_Python_ValidatesContainer_Dataflow.yml | 2 +- ...stCommit_Python_ValidatesContainer_Dataflow_With_RC.yml | 2 +- .../beam_PostCommit_Python_ValidatesRunner_Dataflow.yml | 2 +- .../beam_PostCommit_Python_ValidatesRunner_Flink.yml | 2 +- .../beam_PostCommit_Python_ValidatesRunner_Samza.yml | 2 +- .../beam_PostCommit_Python_ValidatesRunner_Spark.yml | 2 +- .../beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml | 2 +- .../workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml | 2 +- .../workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml | 2 +- .github/workflows/beam_PostCommit_SQL.yml | 2 +- .github/workflows/beam_PostCommit_Sickbay_Python.yml | 2 +- .../workflows/beam_PostCommit_TransformService_Direct.yml | 2 +- .github/workflows/beam_PostCommit_Website_Publish.yml | 2 +- .github/workflows/beam_PostCommit_Website_Test.yml | 2 +- .github/workflows/beam_PostCommit_XVR_Direct.yml | 2 +- .github/workflows/beam_PostCommit_XVR_Flink.yml | 2 +- .../workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml | 2 +- .../beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml | 2 +- .../beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml | 2 +- .../beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml | 2 +- .github/workflows/beam_PostCommit_XVR_Samza.yml | 2 +- .github/workflows/beam_PostCommit_XVR_Spark3.yml | 2 +- .github/workflows/beam_PreCommit_CommunityMetrics.yml | 2 +- .github/workflows/beam_PreCommit_GHA.yml | 2 +- .github/workflows/beam_PreCommit_Go.yml | 2 +- .github/workflows/beam_PreCommit_GoPortable.yml | 2 +- .github/workflows/beam_PreCommit_GoPrism.yml | 2 +- .github/workflows/beam_PreCommit_ItFramework.yml | 2 +- .github/workflows/beam_PreCommit_Java.yml | 2 +- .../beam_PreCommit_Java_Amazon-Web-Services2_IO_Direct.yml | 2 +- .../beam_PreCommit_Java_Amazon-Web-Services_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Amqp_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml | 2 +- .../workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml | 2 +- .../workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml | 2 +- .../workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml | 2 +- .../beam_PreCommit_Java_ElasticSearch_IO_Direct.yml | 2 +- .../workflows/beam_PreCommit_Java_Examples_Dataflow.yml | 2 +- .../beam_PreCommit_Java_Examples_Dataflow_Java11.yml | 2 +- .../beam_PreCommit_Java_Examples_Dataflow_Java17.yml | 2 +- ...beam_PreCommit_Java_File-schema-transform_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Flink_Versions.yml | 2 +- .github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml | 2 +- .../workflows/beam_PreCommit_Java_Google-ads_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml | 2 +- .../workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml | 2 +- .../workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml | 2 +- .../workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml | 2 +- .../workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_PVR_Flink_Batch.yml | 2 +- .github/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml | 2 +- .../workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml | 2 +- .../workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml | 2 +- .../beam_PreCommit_Java_RequestResponse_IO_Direct.yml | 2 +- .../beam_PreCommit_Java_SingleStore_IO_Direct.yml | 2 +- .../workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Spark3_Versions.yml | 2 +- .github/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml | 2 +- .github/workflows/beam_PreCommit_Kotlin_Examples.yml | 2 +- .github/workflows/beam_PreCommit_Portable_Python.yml | 2 +- .github/workflows/beam_PreCommit_Python.yml | 2 +- .github/workflows/beam_PreCommit_PythonDocker.yml | 2 +- .github/workflows/beam_PreCommit_PythonDocs.yml | 2 +- .github/workflows/beam_PreCommit_PythonFormatter.yml | 2 +- .github/workflows/beam_PreCommit_PythonLint.yml | 2 +- .github/workflows/beam_PreCommit_Python_Coverage.yml | 2 +- .github/workflows/beam_PreCommit_Python_Dataframes.yml | 2 +- .github/workflows/beam_PreCommit_Python_Examples.yml | 2 +- .github/workflows/beam_PreCommit_Python_Integration.yml | 2 +- .github/workflows/beam_PreCommit_Python_PVR_Flink.yml | 2 +- .github/workflows/beam_PreCommit_Python_Runners.yml | 2 +- .github/workflows/beam_PreCommit_Python_Transforms.yml | 2 +- .github/workflows/beam_PreCommit_RAT.yml | 2 +- .github/workflows/beam_PreCommit_SQL.yml | 2 +- .github/workflows/beam_PreCommit_SQL_Java11.yml | 2 +- .github/workflows/beam_PreCommit_SQL_Java17.yml | 2 +- .github/workflows/beam_PreCommit_Spotless.yml | 2 +- .github/workflows/beam_PreCommit_Typescript.yml | 2 +- .github/workflows/beam_PreCommit_Website.yml | 2 +- .github/workflows/beam_PreCommit_Website_Stage_GCS.yml | 2 +- .github/workflows/beam_PreCommit_Whitespace.yml | 2 +- .github/workflows/beam_Prober_CommunityMetrics.yml | 2 +- .github/workflows/beam_Publish_Beam_SDK_Snapshots.yml | 5 ++--- .github/workflows/beam_Publish_Docker_Snapshots.yml | 7 +++---- .../beam_Python_ValidatesContainer_Dataflow_ARM.yml | 2 +- .github/workflows/beam_Release_NightlySnapshot.yml | 5 ++--- .github/workflows/beam_Release_Python_NightlySnapshot.yml | 7 +++---- 264 files changed, 276 insertions(+), 282 deletions(-) diff --git a/.github/workflows/README.md b/.github/workflows/README.md index c7db263120ef..e8ec3ce18323 100644 --- a/.github/workflows/README.md +++ b/.github/workflows/README.md @@ -49,7 +49,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Job Phrase' steps: @@ -78,7 +78,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || startsWith(github.event.comment.body, 'Run Job With Matrix') steps: - uses: actions/checkout@v3 diff --git a/.github/workflows/beam_CancelStaleDataflowJobs.yml b/.github/workflows/beam_CancelStaleDataflowJobs.yml index 46ff76df6b51..78cfe67da851 100644 --- a/.github/workflows/beam_CancelStaleDataflowJobs.yml +++ b/.github/workflows/beam_CancelStaleDataflowJobs.yml @@ -58,7 +58,7 @@ jobs: job_name: [beam_CancelStaleDataflowJobs] job_phrase: [Run Cancel Stale Dataflow Jobs] if: | - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Cancel Stale Dataflow Jobs' steps: diff --git a/.github/workflows/beam_CleanUpDataprocResources.yml b/.github/workflows/beam_CleanUpDataprocResources.yml index b6081e1891e6..7ab5029902a1 100644 --- a/.github/workflows/beam_CleanUpDataprocResources.yml +++ b/.github/workflows/beam_CleanUpDataprocResources.yml @@ -50,7 +50,7 @@ jobs: beam_CleanUpDataprocResources: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' + (github.event_name == 'schedule' && github.repository == 'apache/beam') runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 name: "beam_CleanUpDataprocResources" diff --git a/.github/workflows/beam_CleanUpGCPResources.yml b/.github/workflows/beam_CleanUpGCPResources.yml index a7267ad1e0e0..3097df8ecf7d 100644 --- a/.github/workflows/beam_CleanUpGCPResources.yml +++ b/.github/workflows/beam_CleanUpGCPResources.yml @@ -58,7 +58,7 @@ jobs: job_name: [beam_CleanUpGCPResources] job_phrase: [Run Clean GCP Resources] if: | - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Clean GCP Resources' steps: diff --git a/.github/workflows/beam_CleanUpPrebuiltSDKImages.yml b/.github/workflows/beam_CleanUpPrebuiltSDKImages.yml index f327f09c0118..81ece47832f6 100644 --- a/.github/workflows/beam_CleanUpPrebuiltSDKImages.yml +++ b/.github/workflows/beam_CleanUpPrebuiltSDKImages.yml @@ -58,7 +58,7 @@ jobs: job_name: [beam_CleanUpPrebuiltSDKImages] job_phrase: [Run Clean Prebuilt Images] if: | - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Clean Prebuilt Images' steps: diff --git a/.github/workflows/beam_CloudML_Benchmarks_Dataflow.yml b/.github/workflows/beam_CloudML_Benchmarks_Dataflow.yml index fceae349a08d..65e388c8a729 100644 --- a/.github/workflows/beam_CloudML_Benchmarks_Dataflow.yml +++ b/.github/workflows/beam_CloudML_Benchmarks_Dataflow.yml @@ -54,7 +54,7 @@ jobs: beam_CloudML_Benchmarks_Dataflow: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run TFT Criteo Benchmarks' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 360 diff --git a/.github/workflows/beam_IODatastoresCredentialsRotation.yml b/.github/workflows/beam_IODatastoresCredentialsRotation.yml index 17a32a4ecdff..c24d3e52ca71 100644 --- a/.github/workflows/beam_IODatastoresCredentialsRotation.yml +++ b/.github/workflows/beam_IODatastoresCredentialsRotation.yml @@ -50,7 +50,7 @@ jobs: beam_IODatastoresCredentialsRotation: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' + (github.event_name == 'schedule' && github.repository == 'apache/beam') runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 name: ${{ matrix.job_name }} diff --git a/.github/workflows/beam_Inference_Python_Benchmarks_Dataflow.yml b/.github/workflows/beam_Inference_Python_Benchmarks_Dataflow.yml index 16ef684b0589..3bf1c101646e 100644 --- a/.github/workflows/beam_Inference_Python_Benchmarks_Dataflow.yml +++ b/.github/workflows/beam_Inference_Python_Benchmarks_Dataflow.yml @@ -54,7 +54,7 @@ jobs: beam_Inference_Python_Benchmarks_Dataflow: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Inference Benchmarks' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 900 diff --git a/.github/workflows/beam_Java_JMH.yml b/.github/workflows/beam_Java_JMH.yml index a25c3fafaf99..f7cc8838f911 100644 --- a/.github/workflows/beam_Java_JMH.yml +++ b/.github/workflows/beam_Java_JMH.yml @@ -52,7 +52,7 @@ jobs: beam_Java_JMH: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' + (github.event_name == 'schedule' && github.repository == 'apache/beam') runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 900 name: "beam_Java_JMH" diff --git a/.github/workflows/beam_Java_LoadTests_Combine_Smoke.yml b/.github/workflows/beam_Java_LoadTests_Combine_Smoke.yml index b252f4e7b5c2..5fb71d01ced2 100644 --- a/.github/workflows/beam_Java_LoadTests_Combine_Smoke.yml +++ b/.github/workflows/beam_Java_LoadTests_Combine_Smoke.yml @@ -52,7 +52,7 @@ jobs: beam_Java_LoadTests_Combine_Smoke: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Java Load Tests Combine Smoke' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 720 diff --git a/.github/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml index e7e5b90107d2..2124bee4c9e9 100644 --- a/.github/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml @@ -52,7 +52,7 @@ jobs: beam_LoadTests_Go_CoGBK_Dataflow_Batch: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Load Tests Go CoGBK Dataflow Batch' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 720 diff --git a/.github/workflows/beam_LoadTests_Go_CoGBK_Flink_batch.yml b/.github/workflows/beam_LoadTests_Go_CoGBK_Flink_batch.yml index feec436bcd57..93e062925f39 100644 --- a/.github/workflows/beam_LoadTests_Go_CoGBK_Flink_batch.yml +++ b/.github/workflows/beam_LoadTests_Go_CoGBK_Flink_batch.yml @@ -62,7 +62,7 @@ jobs: beam_LoadTests_Go_CoGBK_Flink_Batch: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Load Tests Go CoGBK Flink Batch' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 720 diff --git a/.github/workflows/beam_LoadTests_Go_Combine_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Go_Combine_Dataflow_Batch.yml index 93b65ac99c6c..d927c16ffa39 100644 --- a/.github/workflows/beam_LoadTests_Go_Combine_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_Combine_Dataflow_Batch.yml @@ -52,7 +52,7 @@ jobs: beam_LoadTests_Go_Combine_Dataflow_Batch: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Load Tests Go Combine Dataflow Batch' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 720 diff --git a/.github/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml index f5c9c97652ee..3ec1097e64af 100644 --- a/.github/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml @@ -62,7 +62,7 @@ jobs: beam_LoadTests_Go_Combine_Flink_Batch: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Load Tests Go Combine Flink Batch' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 720 diff --git a/.github/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml index 476338d3df71..0b682ebdf552 100644 --- a/.github/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml @@ -52,7 +52,7 @@ jobs: beam_LoadTests_Go_GBK_Dataflow_Batch: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Load Tests Go GBK Dataflow Batch' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 720 diff --git a/.github/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml index 6f17a4ba6bb3..d0870f417452 100644 --- a/.github/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml @@ -62,7 +62,7 @@ jobs: beam_LoadTests_Go_GBK_Flink_Batch: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Load Tests Go GBK Flink Batch' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 720 diff --git a/.github/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml index e5b33baf568c..47b2c51471f3 100644 --- a/.github/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml @@ -52,7 +52,7 @@ jobs: beam_LoadTests_Go_ParDo_Dataflow_Batch: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Load Tests Go ParDo Dataflow Batch' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 720 diff --git a/.github/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml index ba544420a12f..c6929905d429 100644 --- a/.github/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml @@ -62,7 +62,7 @@ jobs: beam_LoadTests_Go_ParDo_Flink_Batch: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Load Tests Go ParDo Flink Batch' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 720 diff --git a/.github/workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml index 439bb47fbd06..181365e2d561 100644 --- a/.github/workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml @@ -52,7 +52,7 @@ jobs: beam_LoadTests_Go_SideInput_Dataflow_Batch: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Load Tests Go SideInput Dataflow Batch' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 720 diff --git a/.github/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml index 5be573e0e0b2..955c54c238ca 100644 --- a/.github/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml @@ -62,7 +62,7 @@ jobs: beam_LoadTests_Go_SideInput_Flink_Batch: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Load Tests Go SideInput Flink Batch' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 720 diff --git a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml index 27e6a8ad912a..265e3dfc9d38 100644 --- a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml @@ -52,7 +52,7 @@ jobs: beam_LoadTests_Java_CoGBK_Dataflow_Batch: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Load Tests Java CoGBK Dataflow Batch' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 720 diff --git a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml index 27cc98379bcf..ffb38e34a454 100644 --- a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml @@ -52,7 +52,7 @@ jobs: beam_LoadTests_Java_CoGBK_Dataflow_Streaming: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Load Tests Java CoGBK Dataflow Streaming' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 240 diff --git a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml index 8c785595ffec..b1cc0bc7b147 100644 --- a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml +++ b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml @@ -52,7 +52,7 @@ jobs: beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || (contains(github.event.comment.body, 'Run Load Tests Java') && contains(github.event.comment.body, 'CoGBK Dataflow V2 Batch')) runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml index 0c620f02750d..2b38f2e96482 100644 --- a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml +++ b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml @@ -52,7 +52,7 @@ jobs: beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || (contains(github.event.comment.body, 'Run Load Tests Java') && contains(github.event.comment.body, 'CoGBK Dataflow V2 Streaming')) runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml b/.github/workflows/beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml index b77c0aefe95c..1fd32911dc7f 100644 --- a/.github/workflows/beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml +++ b/.github/workflows/beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml @@ -52,7 +52,7 @@ jobs: beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Load Tests Java CoGBK SparkStructuredStreaming Batch' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 720 diff --git a/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml index 91ccdce3f397..53c7a000d385 100644 --- a/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml @@ -52,7 +52,7 @@ jobs: beam_LoadTests_Java_Combine_Dataflow_Batch: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Load Tests Java Combine Dataflow Batch' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 720 diff --git a/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Streaming.yml index 243cb94d9d66..a19db5ff9011 100644 --- a/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Streaming.yml @@ -52,7 +52,7 @@ jobs: beam_LoadTests_Java_Combine_Dataflow_Streaming: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Load Tests Java Combine Dataflow Streaming' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 240 diff --git a/.github/workflows/beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml b/.github/workflows/beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml index dcf6015f62a2..1f9a5a7a9639 100644 --- a/.github/workflows/beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml +++ b/.github/workflows/beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml @@ -52,7 +52,7 @@ jobs: beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Load Tests Java Combine SparkStructuredStreaming Batch' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 720 diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml index 25a48781e083..c9ab32def649 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml @@ -52,7 +52,7 @@ jobs: beam_LoadTests_Java_GBK_Dataflow_Batch: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Load Tests Java GBK Dataflow Batch' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 240 diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml index 9d8ccf279a1c..c73c7f084437 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml @@ -52,7 +52,7 @@ jobs: beam_LoadTests_Java_GBK_Dataflow_Streaming: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Load Tests Java GBK Dataflow Streaming' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 240 diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml index fc5d072c3711..a2e60076e19a 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml @@ -52,7 +52,7 @@ jobs: beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Load Tests Java 11 GBK Dataflow V2 Batch' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 240 diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml index fb249d2ade50..7a658b2cfdc5 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml @@ -52,7 +52,7 @@ jobs: beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Load Tests Java 17 GBK Dataflow V2 Batch' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 240 diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml index 1ce6c4d7d378..6595c9b00e89 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml @@ -52,7 +52,7 @@ jobs: beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Load Tests Java 11 GBK Dataflow V2 Streaming' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 240 diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml index ecffc9923f02..33f5c26a86cb 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml @@ -52,7 +52,7 @@ jobs: beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Load Tests Java 17 GBK Dataflow V2 Streaming' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 240 diff --git a/.github/workflows/beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch.yml b/.github/workflows/beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch.yml index eeba909a13e9..95f1ed712e21 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch.yml @@ -52,7 +52,7 @@ jobs: beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Load Tests Java GBK SparkStructuredStreaming Batch' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 diff --git a/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml index 5f2eaabae5d5..4b0453152a29 100644 --- a/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml @@ -52,7 +52,7 @@ jobs: beam_LoadTests_Java_ParDo_Dataflow_Batch: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Load Tests Java ParDo Dataflow Batch' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 720 diff --git a/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml index 8279643c0d2d..e1644a068570 100644 --- a/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml @@ -52,7 +52,7 @@ jobs: beam_LoadTests_Java_ParDo_Dataflow_Streaming: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Load Tests Java ParDo Dataflow Streaming' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 240 diff --git a/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions.yml b/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions.yml index 812541087ea7..0993409d5122 100644 --- a/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions.yml +++ b/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions.yml @@ -52,7 +52,7 @@ jobs: beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || (contains(github.event.comment.body, 'Run Load Tests Java') && contains(github.event.comment.body, 'ParDo Dataflow V2 Batch')) runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions.yml b/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions.yml index 64013955ad22..24b32d5f2197 100644 --- a/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions.yml +++ b/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions.yml @@ -52,7 +52,7 @@ jobs: beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || (contains(github.event.comment.body, 'Run Load Tests Java') && contains(github.event.comment.body, 'ParDo Dataflow V2 Streaming')) runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml b/.github/workflows/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml index 8e379324fc02..c5972a7c5e93 100644 --- a/.github/workflows/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml +++ b/.github/workflows/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml @@ -52,7 +52,7 @@ jobs: beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Load Tests Java ParDo SparkStructuredStreaming Batch' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 720 diff --git a/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml index 504d512802aa..ffbd362a1eab 100644 --- a/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml @@ -52,7 +52,7 @@ jobs: beam_LoadTests_Python_CoGBK_Dataflow_Batch: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Load Tests Python CoGBK Dataflow Batch' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 720 diff --git a/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml index 9146c20226c3..f569237c7fb4 100644 --- a/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml @@ -52,7 +52,7 @@ jobs: beam_LoadTests_Python_CoGBK_Dataflow_Streaming: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Load Tests Python CoGBK Dataflow Streaming' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 720 diff --git a/.github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml index 3783f72cf9bc..2493f14585b9 100644 --- a/.github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml @@ -62,7 +62,7 @@ jobs: beam_LoadTests_Python_CoGBK_Flink_Batch: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Load Tests Python CoGBK Flink Batch' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 720 diff --git a/.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml index 3aa9b35679e2..3e3b9be9754f 100644 --- a/.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml @@ -52,7 +52,7 @@ jobs: beam_LoadTests_Python_Combine_Dataflow_Batch: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Load Tests Python Combine Dataflow Batch' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 720 diff --git a/.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Streaming.yml index 1dabfa77673c..746e6bc19300 100644 --- a/.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Streaming.yml @@ -52,7 +52,7 @@ jobs: beam_LoadTests_Python_Combine_Dataflow_Streaming: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Load Tests Python Combine Dataflow Streaming' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 720 diff --git a/.github/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml index 358a9a5378d9..815f3dbc50d8 100644 --- a/.github/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml @@ -62,7 +62,7 @@ jobs: beam_LoadTests_Python_Combine_Flink_Batch: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Load Tests Python Combine Flink Batch' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 720 diff --git a/.github/workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml b/.github/workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml index 8da2bd586045..24fdce175f2c 100644 --- a/.github/workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml @@ -62,7 +62,7 @@ jobs: beam_LoadTests_Python_Combine_Flink_Streaming: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Load Tests Python Combine Flink Streaming' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 720 diff --git a/.github/workflows/beam_LoadTests_Python_FnApiRunner_Microbenchmark.yml b/.github/workflows/beam_LoadTests_Python_FnApiRunner_Microbenchmark.yml index 5755f25a47b1..1169a45dfc2d 100644 --- a/.github/workflows/beam_LoadTests_Python_FnApiRunner_Microbenchmark.yml +++ b/.github/workflows/beam_LoadTests_Python_FnApiRunner_Microbenchmark.yml @@ -52,7 +52,7 @@ jobs: beam_LoadTests_Python_FnApiRunner_Microbenchmark: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Python Load Tests FnApiRunner Microbenchmark' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 120 diff --git a/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml index d3af9666a782..4631c2b31088 100644 --- a/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml @@ -52,7 +52,7 @@ jobs: beam_LoadTests_Python_GBK_Dataflow_Batch: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Load Tests Python GBK Dataflow Batch' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 720 diff --git a/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml index ff796c2db1ef..a28532f9d71c 100644 --- a/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml @@ -52,7 +52,7 @@ jobs: beam_LoadTests_Python_GBK_Dataflow_Streaming: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Load Tests Python GBK Dataflow Streaming' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 720 diff --git a/.github/workflows/beam_LoadTests_Python_GBK_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Python_GBK_Flink_Batch.yml index 4674e4bb01ed..d2a99b3711e0 100644 --- a/.github/workflows/beam_LoadTests_Python_GBK_Flink_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_GBK_Flink_Batch.yml @@ -62,7 +62,7 @@ jobs: beam_LoadTests_Python_GBK_Flink_Batch: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Load Tests Python GBK Flink Batch' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 720 diff --git a/.github/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Batch.yml index b5062369c4d9..e08b99c1f678 100644 --- a/.github/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Batch.yml @@ -52,7 +52,7 @@ jobs: beam_LoadTests_Python_GBK_reiterate_Dataflow_Batch: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Load Tests Python GBK reiterate Dataflow Batch' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 720 diff --git a/.github/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Streaming.yml index 25fb9fdd27ed..9028dedf876d 100644 --- a/.github/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Streaming.yml @@ -52,7 +52,7 @@ jobs: beam_LoadTests_Python_GBK_reiterate_Dataflow_Streaming: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Load Tests Python GBK reiterate Dataflow Streaming' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 720 diff --git a/.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Batch.yml index 303780ac8088..dc3738e83bf2 100644 --- a/.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Batch.yml @@ -52,7 +52,7 @@ jobs: beam_LoadTests_Python_ParDo_Dataflow_Batch: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Load Tests Python ParDo Dataflow Batch' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 200 diff --git a/.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml index c6c7bc322e93..447460fecf0d 100644 --- a/.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml @@ -52,7 +52,7 @@ jobs: beam_LoadTests_Python_ParDo_Dataflow_Streaming: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Python Load Tests ParDo Dataflow Streaming' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 200 diff --git a/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml index 9c8d86289e6e..97211b0f0207 100644 --- a/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml @@ -62,7 +62,7 @@ jobs: beam_LoadTests_Python_ParDo_Flink_Batch: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Load Tests Python ParDo Flink Batch' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 720 diff --git a/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml b/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml index 067cdd5cd525..46437e765388 100644 --- a/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml @@ -62,7 +62,7 @@ jobs: beam_LoadTests_Python_ParDo_Flink_Streaming: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Load Tests Python ParDo Flink Streaming' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 720 diff --git a/.github/workflows/beam_LoadTests_Python_SideInput_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Python_SideInput_Dataflow_Batch.yml index 3be652ab8891..08e5567e6a0e 100644 --- a/.github/workflows/beam_LoadTests_Python_SideInput_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_SideInput_Dataflow_Batch.yml @@ -52,7 +52,7 @@ jobs: beam_LoadTests_Python_SideInput_Dataflow_Batch: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Load Tests Python SideInput Dataflow Batch' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 720 diff --git a/.github/workflows/beam_MetricsCredentialsRotation.yml b/.github/workflows/beam_MetricsCredentialsRotation.yml index ea5aaeefed25..777477fe2057 100644 --- a/.github/workflows/beam_MetricsCredentialsRotation.yml +++ b/.github/workflows/beam_MetricsCredentialsRotation.yml @@ -49,9 +49,8 @@ env: jobs: beam_MetricsCredentialsRotation: if: | - (github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule') && - github.repository == 'apache/beam' + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 name: ${{ matrix.job_name }} diff --git a/.github/workflows/beam_Metrics_Report.yml b/.github/workflows/beam_Metrics_Report.yml index 1502c1fa326c..8ed0c66480f0 100644 --- a/.github/workflows/beam_Metrics_Report.yml +++ b/.github/workflows/beam_Metrics_Report.yml @@ -54,9 +54,9 @@ jobs: runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 if: | - (github.event_name == 'schedule' || - github.event_name == 'workflow_dispatch') && - github.repository == 'apache/beam' + ((github.event_name == 'schedule' && github.repository == 'apache/beam') || + github.event_name == 'workflow_dispatch' + steps: - uses: actions/checkout@v3 - name: Setup environment diff --git a/.github/workflows/beam_PerformanceTests_AvroIOIT.yml b/.github/workflows/beam_PerformanceTests_AvroIOIT.yml index 8c781b867263..18bd56855c82 100644 --- a/.github/workflows/beam_PerformanceTests_AvroIOIT.yml +++ b/.github/workflows/beam_PerformanceTests_AvroIOIT.yml @@ -52,7 +52,7 @@ jobs: beam_PerformanceTests_AvroIOIT: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Java AvroIO Performance Test' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 diff --git a/.github/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml b/.github/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml index e137af01554a..dab1d9d6e942 100644 --- a/.github/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml +++ b/.github/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml @@ -52,7 +52,7 @@ jobs: beam_PerformanceTests_AvroIOIT_HDFS: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Java AvroIO Performance Test HDFS' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 diff --git a/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml b/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml index 00a817c94e83..8727b2387403 100644 --- a/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml +++ b/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml @@ -52,7 +52,7 @@ jobs: beam_PerformanceTests_BigQueryIO_Batch_Java_Avro: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run BigQueryIO Batch Performance Test Java Avro' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 diff --git a/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml b/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml index 6cb5e8f51fec..a231d00f5ede 100644 --- a/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml +++ b/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml @@ -52,7 +52,7 @@ jobs: beam_PerformanceTests_BigQueryIO_Batch_Java_Json: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run BigQueryIO Batch Performance Test Java Json' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 diff --git a/.github/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml b/.github/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml index f8c978cff142..7259fb1838f2 100644 --- a/.github/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml +++ b/.github/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml @@ -52,7 +52,7 @@ jobs: beam_PerformanceTests_BigQueryIO_Streaming_Java: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run BigQueryIO Streaming Performance Test Java' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 diff --git a/.github/workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml b/.github/workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml index dd9477f091af..11fa89767d61 100644 --- a/.github/workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml +++ b/.github/workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml @@ -52,7 +52,7 @@ jobs: beam_PerformanceTests_BiqQueryIO_Read_Python: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run BigQueryIO Read Performance Test Python' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 240 diff --git a/.github/workflows/beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml b/.github/workflows/beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml index 32eb2c9301ca..fa7a3a78d5d1 100644 --- a/.github/workflows/beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml +++ b/.github/workflows/beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml @@ -52,7 +52,7 @@ jobs: beam_PerformanceTests_BiqQueryIO_Write_Python_Batch: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run BigQueryIO Write Performance Test Python' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 240 diff --git a/.github/workflows/beam_PerformanceTests_Cdap.yml b/.github/workflows/beam_PerformanceTests_Cdap.yml index b4e0b2e011ab..b0d29bbf02b5 100644 --- a/.github/workflows/beam_PerformanceTests_Cdap.yml +++ b/.github/workflows/beam_PerformanceTests_Cdap.yml @@ -52,7 +52,7 @@ jobs: beam_PerformanceTests_Cdap: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Java CdapIO Performance Test' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 diff --git a/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml b/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml index 465a77b45088..3b68bdeeb509 100644 --- a/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml +++ b/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml @@ -52,7 +52,7 @@ jobs: beam_PerformanceTests_Compressed_TextIOIT: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Java CompressedTextIO Performance Test' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 diff --git a/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml b/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml index f52adaaec4a1..414889a159c4 100644 --- a/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml +++ b/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml @@ -52,7 +52,7 @@ jobs: beam_PerformanceTests_Compressed_TextIOIT_HDFS: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Java CompressedTextIO Performance Test HDFS' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 diff --git a/.github/workflows/beam_PerformanceTests_HadoopFormat.yml b/.github/workflows/beam_PerformanceTests_HadoopFormat.yml index d303a000fddc..002dd9865728 100644 --- a/.github/workflows/beam_PerformanceTests_HadoopFormat.yml +++ b/.github/workflows/beam_PerformanceTests_HadoopFormat.yml @@ -52,7 +52,7 @@ jobs: beam_PerformanceTests_HadoopFormat: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Java HadoopFormatIO Performance Test' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 diff --git a/.github/workflows/beam_PerformanceTests_JDBC.yml b/.github/workflows/beam_PerformanceTests_JDBC.yml index f6f5fcebd45b..caa2955b6c23 100644 --- a/.github/workflows/beam_PerformanceTests_JDBC.yml +++ b/.github/workflows/beam_PerformanceTests_JDBC.yml @@ -52,7 +52,7 @@ jobs: beam_PerformanceTests_JDBC: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Java JdbcIO Performance Test' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 diff --git a/.github/workflows/beam_PerformanceTests_Kafka_IO.yml b/.github/workflows/beam_PerformanceTests_Kafka_IO.yml index f19b0b844a98..ac96a50efcb7 100644 --- a/.github/workflows/beam_PerformanceTests_Kafka_IO.yml +++ b/.github/workflows/beam_PerformanceTests_Kafka_IO.yml @@ -52,7 +52,7 @@ jobs: beam_PerformanceTests_Kafka_IO: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Java KafkaIO Performance Test' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 120 diff --git a/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml b/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml index 9482aae8fc0c..3b6e54bc7b92 100644 --- a/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml +++ b/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml @@ -52,7 +52,7 @@ jobs: beam_PerformanceTests_ManyFiles_TextIOIT: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Java ManyFilesTextIO Performance Test' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 diff --git a/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml b/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml index e8e52104e898..8a34dae133aa 100644 --- a/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml +++ b/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml @@ -52,7 +52,7 @@ jobs: beam_PerformanceTests_ManyFiles_TextIOIT_HDFS: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Java ManyFilesTextIO Performance Test HDFS' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 diff --git a/.github/workflows/beam_PerformanceTests_MongoDBIO_IT.yml b/.github/workflows/beam_PerformanceTests_MongoDBIO_IT.yml index 5b25bd838811..6d4a9f4e2b88 100644 --- a/.github/workflows/beam_PerformanceTests_MongoDBIO_IT.yml +++ b/.github/workflows/beam_PerformanceTests_MongoDBIO_IT.yml @@ -52,7 +52,7 @@ jobs: beam_PerformanceTests_MongoDBIO_IT: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Java MongoDBIO Performance Test' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 diff --git a/.github/workflows/beam_PerformanceTests_ParquetIOIT.yml b/.github/workflows/beam_PerformanceTests_ParquetIOIT.yml index 6ade6b2f88ef..d2d045f6ba02 100644 --- a/.github/workflows/beam_PerformanceTests_ParquetIOIT.yml +++ b/.github/workflows/beam_PerformanceTests_ParquetIOIT.yml @@ -52,7 +52,7 @@ jobs: beam_PerformanceTests_ParquetIOIT: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Java ParquetIO Performance Test' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 diff --git a/.github/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml b/.github/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml index 05f1198d65c2..e5a85d06c20a 100644 --- a/.github/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml +++ b/.github/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml @@ -52,7 +52,7 @@ jobs: beam_PerformanceTests_ParquetIOIT_HDFS: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Java ParquetIO Performance Test HDFS' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 diff --git a/.github/workflows/beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml b/.github/workflows/beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml index 17bb5ce81986..8a1d8196178b 100644 --- a/.github/workflows/beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml +++ b/.github/workflows/beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml @@ -52,7 +52,7 @@ jobs: beam_PerformanceTests_PubsubIOIT_Python_Streaming: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run PubsubIO Performance Test Python' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 240 diff --git a/.github/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml b/.github/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml index 32297e41b74e..0fdd94bcfe53 100644 --- a/.github/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml +++ b/.github/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml @@ -52,7 +52,7 @@ jobs: beam_PerformanceTests_SQLBigQueryIO_Batch_Java: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run SQLBigQueryIO Batch Performance Test Java' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 diff --git a/.github/workflows/beam_PerformanceTests_SingleStoreIO.yml b/.github/workflows/beam_PerformanceTests_SingleStoreIO.yml index c5c35b5807ab..cc1a0b24a60a 100644 --- a/.github/workflows/beam_PerformanceTests_SingleStoreIO.yml +++ b/.github/workflows/beam_PerformanceTests_SingleStoreIO.yml @@ -63,7 +63,7 @@ jobs: job_phrase: [Run Java SingleStoreIO Performance Test] if: | github.event_name == 'push' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Java SingleStoreIO Performance Test' steps: diff --git a/.github/workflows/beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml b/.github/workflows/beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml index 4f1b8357e9e3..8e77a0edd66f 100644 --- a/.github/workflows/beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml +++ b/.github/workflows/beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml @@ -52,7 +52,7 @@ jobs: beam_PerformanceTests_SpannerIO_Read_2GB_Python: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run SpannerIO Read 2GB Performance Test Python' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 480 diff --git a/.github/workflows/beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml b/.github/workflows/beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml index 07b336e53348..92664757e70a 100644 --- a/.github/workflows/beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml +++ b/.github/workflows/beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml @@ -52,7 +52,7 @@ jobs: beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run SpannerIO Write 2GB Performance Test Python Batch' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 480 diff --git a/.github/workflows/beam_PerformanceTests_SparkReceiver_IO.yml b/.github/workflows/beam_PerformanceTests_SparkReceiver_IO.yml index 7aa71c3ef5a2..33d8122775d9 100644 --- a/.github/workflows/beam_PerformanceTests_SparkReceiver_IO.yml +++ b/.github/workflows/beam_PerformanceTests_SparkReceiver_IO.yml @@ -52,7 +52,7 @@ jobs: beam_PerformanceTests_SparkReceiver_IO: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Java SparkReceiverIO Performance Test' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 120 diff --git a/.github/workflows/beam_PerformanceTests_TFRecordIOIT.yml b/.github/workflows/beam_PerformanceTests_TFRecordIOIT.yml index 0d80693da073..84438ff31584 100644 --- a/.github/workflows/beam_PerformanceTests_TFRecordIOIT.yml +++ b/.github/workflows/beam_PerformanceTests_TFRecordIOIT.yml @@ -52,7 +52,7 @@ jobs: beam_PerformanceTests_TFRecordIOIT: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Java TFRecordIO Performance Test' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 diff --git a/.github/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml b/.github/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml index 10099f75f967..6cc273bbe9c0 100644 --- a/.github/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml +++ b/.github/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml @@ -54,7 +54,7 @@ jobs: beam_PerformanceTests_TFRecordIOIT_HDFS: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Java TFRecordIO Performance Test HDFS' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 diff --git a/.github/workflows/beam_PerformanceTests_TextIOIT.yml b/.github/workflows/beam_PerformanceTests_TextIOIT.yml index 2c92ec3eff23..96b20ad3a6f6 100644 --- a/.github/workflows/beam_PerformanceTests_TextIOIT.yml +++ b/.github/workflows/beam_PerformanceTests_TextIOIT.yml @@ -52,7 +52,7 @@ jobs: beam_PerformanceTests_TextIOIT: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Java TextIO Performance Test' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 diff --git a/.github/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml b/.github/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml index 4caad4ff4114..aca9739c3fb1 100644 --- a/.github/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml +++ b/.github/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml @@ -52,7 +52,7 @@ jobs: beam_PerformanceTests_TextIOIT_HDFS: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Java TextIO Performance Test HDFS' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 diff --git a/.github/workflows/beam_PerformanceTests_TextIOIT_Python.yml b/.github/workflows/beam_PerformanceTests_TextIOIT_Python.yml index b5aecb97889b..2bc8bdbb194c 100644 --- a/.github/workflows/beam_PerformanceTests_TextIOIT_Python.yml +++ b/.github/workflows/beam_PerformanceTests_TextIOIT_Python.yml @@ -52,7 +52,7 @@ jobs: beam_PerformanceTests_TextIOIT_Python: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Python TextIO Performance Test' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 diff --git a/.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml b/.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml index 7903bf033ada..b3e606fcc390 100644 --- a/.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml +++ b/.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml @@ -52,7 +52,7 @@ jobs: beam_PerformanceTests_WordCountIT_PythonVersions: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || (startswith(github.event.comment.body, 'Run Python') && endswith(github.event.comment.body, 'WordCountIT Performance Test')) runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PerformanceTests_XmlIOIT.yml b/.github/workflows/beam_PerformanceTests_XmlIOIT.yml index 1dd1efc8b052..cf83d2ea4015 100644 --- a/.github/workflows/beam_PerformanceTests_XmlIOIT.yml +++ b/.github/workflows/beam_PerformanceTests_XmlIOIT.yml @@ -52,7 +52,7 @@ jobs: beam_PerformanceTests_XmlIOIT: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Java XmlIO Performance Test' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 diff --git a/.github/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml b/.github/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml index b03a69902354..3740a30f129f 100644 --- a/.github/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml +++ b/.github/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml @@ -52,7 +52,7 @@ jobs: beam_PerformanceTests_XmlIOIT_HDFS: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Java XmlIO Performance Test HDFS' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 diff --git a/.github/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml b/.github/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml index 2818b710a62b..c6c60e165721 100644 --- a/.github/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml +++ b/.github/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml @@ -52,7 +52,7 @@ jobs: beam_PerformanceTests_xlang_KafkaIO_Python: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Python xlang KafkaIO Performance Test' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 240 diff --git a/.github/workflows/beam_PostCommit_BeamMetrics_Publish.yml b/.github/workflows/beam_PostCommit_BeamMetrics_Publish.yml index 15ab008cec30..7695b7f0ca30 100644 --- a/.github/workflows/beam_PostCommit_BeamMetrics_Publish.yml +++ b/.github/workflows/beam_PostCommit_BeamMetrics_Publish.yml @@ -53,11 +53,10 @@ env: jobs: beam_PostCommit_BeamMetrics_Publish: if: | - (github.event_name == 'push' || + github.event_name == 'push' || github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || - github.event.comment.body == 'Run Beam Metrics Deployment') && - github.repository == 'apache/beam' + (github.event_name == 'schedule' && github.repository == 'apache/beam') || + github.event.comment.body == 'Run Beam Metrics Deployment' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) diff --git a/.github/workflows/beam_PostCommit_Go.yml b/.github/workflows/beam_PostCommit_Go.yml index 1a0a616006f0..8664b386cc37 100644 --- a/.github/workflows/beam_PostCommit_Go.yml +++ b/.github/workflows/beam_PostCommit_Go.yml @@ -50,7 +50,7 @@ jobs: beam_PostCommit_Go: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Go PostCommit' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 300 diff --git a/.github/workflows/beam_PostCommit_Go_Dataflow_ARM.yml b/.github/workflows/beam_PostCommit_Go_Dataflow_ARM.yml index 83eb29497426..dcc1b018f914 100644 --- a/.github/workflows/beam_PostCommit_Go_Dataflow_ARM.yml +++ b/.github/workflows/beam_PostCommit_Go_Dataflow_ARM.yml @@ -57,7 +57,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Go PostCommit Dataflow ARM' runs-on: [self-hosted, ubuntu-20.04, main] name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) diff --git a/.github/workflows/beam_PostCommit_Go_VR_Flink.yml b/.github/workflows/beam_PostCommit_Go_VR_Flink.yml index 6434f57316e7..1d12aed4e798 100644 --- a/.github/workflows/beam_PostCommit_Go_VR_Flink.yml +++ b/.github/workflows/beam_PostCommit_Go_VR_Flink.yml @@ -50,7 +50,7 @@ jobs: beam_PostCommit_Go_VR_Flink: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Go Flink ValidatesRunner' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 diff --git a/.github/workflows/beam_PostCommit_Go_VR_Samza.yml b/.github/workflows/beam_PostCommit_Go_VR_Samza.yml index 372e357dd06c..6d1f0bb3ef90 100644 --- a/.github/workflows/beam_PostCommit_Go_VR_Samza.yml +++ b/.github/workflows/beam_PostCommit_Go_VR_Samza.yml @@ -50,7 +50,7 @@ jobs: beam_PostCommit_Go_VR_Samza: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Go Samza ValidatesRunner' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 diff --git a/.github/workflows/beam_PostCommit_Go_VR_Spark.yml b/.github/workflows/beam_PostCommit_Go_VR_Spark.yml index 7b80df1191c9..2008db6a1bbf 100644 --- a/.github/workflows/beam_PostCommit_Go_VR_Spark.yml +++ b/.github/workflows/beam_PostCommit_Go_VR_Spark.yml @@ -50,7 +50,7 @@ jobs: beam_PostCommit_Go_VR_Spark: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Go Spark ValidatesRunner' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 diff --git a/.github/workflows/beam_PostCommit_Java.yml b/.github/workflows/beam_PostCommit_Java.yml index 905cb3a7f253..a60e3c7f24f1 100644 --- a/.github/workflows/beam_PostCommit_Java.yml +++ b/.github/workflows/beam_PostCommit_Java.yml @@ -59,7 +59,7 @@ jobs: job_phrase: [Run Java PostCommit] if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Java PostCommit' steps: - uses: actions/checkout@v4 diff --git a/.github/workflows/beam_PostCommit_Java_Avro_Versions.yml b/.github/workflows/beam_PostCommit_Java_Avro_Versions.yml index 1cd59fe409a1..31107c24f989 100644 --- a/.github/workflows/beam_PostCommit_Java_Avro_Versions.yml +++ b/.github/workflows/beam_PostCommit_Java_Avro_Versions.yml @@ -59,7 +59,7 @@ jobs: job_phrase: [Run Java Avro Versions PostCommit] if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Java Avro Versions PostCommit' steps: - uses: actions/checkout@v4 diff --git a/.github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml b/.github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml index 33748db11cec..20f3ecdcde8b 100644 --- a/.github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml +++ b/.github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml @@ -59,7 +59,7 @@ jobs: job_phrase: [Run Java BigQueryEarlyRollout PostCommit] if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Java BigQueryEarlyRollout PostCommit' steps: - uses: actions/checkout@v4 diff --git a/.github/workflows/beam_PostCommit_Java_DataflowV1.yml b/.github/workflows/beam_PostCommit_Java_DataflowV1.yml index a15730a3d4ed..90ee56c475f5 100644 --- a/.github/workflows/beam_PostCommit_Java_DataflowV1.yml +++ b/.github/workflows/beam_PostCommit_Java_DataflowV1.yml @@ -59,7 +59,7 @@ jobs: job_phrase: [Run PostCommit_Java_Dataflow] if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run PostCommit_Java_Dataflow' steps: - uses: actions/checkout@v4 diff --git a/.github/workflows/beam_PostCommit_Java_DataflowV2.yml b/.github/workflows/beam_PostCommit_Java_DataflowV2.yml index f90fc8f1f262..066c5933ec72 100644 --- a/.github/workflows/beam_PostCommit_Java_DataflowV2.yml +++ b/.github/workflows/beam_PostCommit_Java_DataflowV2.yml @@ -59,7 +59,7 @@ jobs: job_phrase: [Run PostCommit_Java_DataflowV2] if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run PostCommit_Java_DataflowV2' steps: - uses: actions/checkout@v4 diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow.yml index 2c2137482552..d050b9b1a8c4 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow.yml @@ -61,7 +61,7 @@ jobs: job_phrase: [Run Java examples on Dataflow] if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Java Examples on Dataflow' steps: - uses: actions/checkout@v4 diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml index 09d90d5df1a8..f8f8ffe617f3 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml @@ -74,7 +74,7 @@ jobs: java_version: ['8','11','17'] if: | github.event_name == 'push' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github. event_name == 'workflow_dispatch' || startswith(github.event.comment.body, 'Run Java_Examples_Dataflow_ARM PostCommit') steps: diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml index ee6596adbe4c..685b3aa01e4c 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml @@ -61,7 +61,7 @@ jobs: java_version: ['11','17'] if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || startswith(github.event.comment.body, 'Run Java examples on Dataflow Java') steps: - uses: actions/checkout@v4 diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml index 7283bbedca74..9e2b37ab4570 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml @@ -59,7 +59,7 @@ jobs: job_phrase: [Run Java Examples on Dataflow Runner V2] if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Java Examples on Dataflow Runner V2' steps: - uses: actions/checkout@v4 diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml index fe8de2d2abe0..a752c939b994 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml @@ -62,7 +62,7 @@ jobs: java_version: ['11', '17', '21'] if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || (contains(github.event.comment.body, 'Run Java') && contains(github.event.comment.body, 'Examples on Dataflow Runner V2')) steps: diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Direct.yml b/.github/workflows/beam_PostCommit_Java_Examples_Direct.yml index 60f5c1da489f..897660cb97cf 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Direct.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Direct.yml @@ -59,7 +59,7 @@ jobs: job_phrase: [Run Java Examples_Direct] if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Java Examples_Direct' steps: - uses: actions/checkout@v4 diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Flink.yml b/.github/workflows/beam_PostCommit_Java_Examples_Flink.yml index a86160e9d700..b03575950cc4 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Flink.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Flink.yml @@ -59,7 +59,7 @@ jobs: job_phrase: [Run Java Examples_Flink] if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Java Examples_Flink' steps: - uses: actions/checkout@v4 diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Spark.yml b/.github/workflows/beam_PostCommit_Java_Examples_Spark.yml index e3f70bd8f69f..42e265e8ae35 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Spark.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Spark.yml @@ -59,7 +59,7 @@ jobs: job_phrase: [Run Java Examples_Spark] if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Java Examples_Spark' steps: - uses: actions/checkout@v4 diff --git a/.github/workflows/beam_PostCommit_Java_Hadoop_Versions.yml b/.github/workflows/beam_PostCommit_Java_Hadoop_Versions.yml index 41eec12629d3..f031f87c159e 100644 --- a/.github/workflows/beam_PostCommit_Java_Hadoop_Versions.yml +++ b/.github/workflows/beam_PostCommit_Java_Hadoop_Versions.yml @@ -57,7 +57,7 @@ jobs: job_phrase: [Run PostCommit_Java_Hadoop_Versions] if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run PostCommit_Java_Hadoop_Versions' steps: - uses: actions/checkout@v4 diff --git a/.github/workflows/beam_PostCommit_Java_IO_Performance_Tests.yml b/.github/workflows/beam_PostCommit_Java_IO_Performance_Tests.yml index 8775c9643bd3..bbb3ced69b70 100644 --- a/.github/workflows/beam_PostCommit_Java_IO_Performance_Tests.yml +++ b/.github/workflows/beam_PostCommit_Java_IO_Performance_Tests.yml @@ -57,7 +57,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Java PostCommit IO Performance Tests' runs-on: [self-hosted, ubuntu-20.04, main] name: ${{ matrix.job_name }} (${{ matrix.job_phrase }} ${{ matrix.test_case }}) diff --git a/.github/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml b/.github/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml index c80ec6c0fbee..331a5bdaa305 100644 --- a/.github/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml +++ b/.github/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml @@ -58,7 +58,7 @@ jobs: job_name: [beam_PostCommit_Java_InfluxDbIO_IT] job_phrase: [Run Java InfluxDbIO_IT] if: | - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Java InfluxDbIO_IT' steps: diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml index 57dca871e59a..13e67f0d1123 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml @@ -50,7 +50,7 @@ jobs: beam_PostCommit_Java_Jpms_Dataflow_Java11: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Jpms Dataflow Java 11 PostCommit' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 240 diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml index 02046068c025..72a5af15f357 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml @@ -50,7 +50,7 @@ jobs: beam_PostCommit_Java_Jpms_Dataflow_Java17: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Jpms Dataflow Java 17 PostCommit' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 240 diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml index 61148ace0be8..09190a51cb62 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml @@ -50,7 +50,7 @@ jobs: beam_PostCommit_Java_Jpms_Direct_Java11: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Jpms Direct Java 11 PostCommit' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 240 diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml index 823cea06ada9..ed1bf0898020 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml @@ -50,7 +50,7 @@ jobs: beam_PostCommit_Java_Jpms_Direct_Java17: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Jpms Direct Java 17 PostCommit' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 240 diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java21.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java21.yml index 608d70b20cd9..d300e5ed50b2 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java21.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java21.yml @@ -50,7 +50,7 @@ jobs: beam_PostCommit_Java_Jpms_Direct_Java21: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Jpms Direct Java 21 PostCommit' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 240 diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml index 2b217630ebf4..b83d229f02f7 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml @@ -50,7 +50,7 @@ jobs: beam_PostCommit_Java_Jpms_Flink_Java11: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Jpms Flink Java 11 PostCommit' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 240 diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml index 4026ce9a22f4..830a2c0b2f1e 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml @@ -50,7 +50,7 @@ jobs: beam_PostCommit_Java_Jpms_Spark_Java11: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Jpms Spark Java 11 PostCommit' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 240 diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml index d40723af05c9..5bf6676e4268 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml @@ -85,7 +85,7 @@ jobs: queryLanguage: [sql, zetasql, none] if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Dataflow Runner Nexmark Tests' steps: - uses: actions/checkout@v4 diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml index 787ca05a065d..c8de83d4c4cc 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml @@ -85,7 +85,7 @@ jobs: streaming: [false, true] if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Dataflow Runner V2 Nexmark Tests' steps: - uses: actions/checkout@v4 diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml index f06302c08513..4000f1ce5bd1 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml @@ -86,7 +86,7 @@ jobs: java_version: ['11','17'] if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || (contains(github.event.comment.body, 'Run Dataflow Runner V2 Java') && contains(github.event.comment.body, 'Nexmark Tests')) steps: diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Direct.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Direct.yml index 837582a484b9..d6376cfb6ed7 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Direct.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Direct.yml @@ -80,7 +80,7 @@ jobs: queryLanguage: [sql, zetasql, none] if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Direct Runner Nexmark Tests' steps: - uses: actions/checkout@v4 diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Flink.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Flink.yml index afcc9060491e..57a8562c534a 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Flink.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Flink.yml @@ -79,7 +79,7 @@ jobs: queryLanguage: [sql, zetasql, none] if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Flink Runner Nexmark Tests' steps: - uses: actions/checkout@v4 diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Spark.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Spark.yml index 937a843740a1..fad07f56ce83 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Spark.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Spark.yml @@ -79,7 +79,7 @@ jobs: queryLanguage: [sql, zetasql, none] if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Spark Runner Nexmark Tests' steps: - uses: actions/checkout@v4 diff --git a/.github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml b/.github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml index de5253fe8516..979d3937ad59 100644 --- a/.github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml +++ b/.github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml @@ -59,7 +59,7 @@ jobs: job_phrase: [Run Java Flink PortableValidatesRunner Streaming] if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Java Flink PortableValidatesRunner Streaming' steps: - uses: actions/checkout@v4 diff --git a/.github/workflows/beam_PostCommit_Java_PVR_Samza.yml b/.github/workflows/beam_PostCommit_Java_PVR_Samza.yml index 912cba9877a5..cebadc256695 100644 --- a/.github/workflows/beam_PostCommit_Java_PVR_Samza.yml +++ b/.github/workflows/beam_PostCommit_Java_PVR_Samza.yml @@ -59,7 +59,7 @@ jobs: job_phrase: [Run Java Samza PortableValidatesRunner] if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Java Samza PortableValidatesRunner' steps: - uses: actions/checkout@v4 diff --git a/.github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml b/.github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml index b94e40037bdc..ba51a8f8af50 100644 --- a/.github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml +++ b/.github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml @@ -59,7 +59,7 @@ jobs: job_phrase: [Run Java Spark v3 PortableValidatesRunner Streaming] if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Java Spark v3 PortableValidatesRunner Streaming' steps: - uses: actions/checkout@v4 diff --git a/.github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml b/.github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml index d7461e3236be..eaddb569d4d8 100644 --- a/.github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml +++ b/.github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml @@ -59,7 +59,7 @@ jobs: job_phrase: [Run Java Spark PortableValidatesRunner Batch] if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' + (github.event_name == 'schedule' && github.repository == 'apache/beam') steps: - uses: actions/checkout@v4 - name: Setup repository diff --git a/.github/workflows/beam_PostCommit_Java_Sickbay.yml b/.github/workflows/beam_PostCommit_Java_Sickbay.yml index a9e21f4c9f33..ceac541bdb8f 100644 --- a/.github/workflows/beam_PostCommit_Java_Sickbay.yml +++ b/.github/workflows/beam_PostCommit_Java_Sickbay.yml @@ -59,7 +59,7 @@ jobs: job_phrase: [Run Java Sickbay] if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Java Sickbay' steps: - uses: actions/checkout@v4 diff --git a/.github/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml b/.github/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml index ba2c72ffa2cc..227df1815deb 100644 --- a/.github/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml +++ b/.github/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml @@ -60,7 +60,7 @@ jobs: job_phrase: [Run Java SingleStoreIO_IT] if: | github.event_name == 'push' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Java SingleStoreIO_IT' steps: diff --git a/.github/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml b/.github/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml index 5fb70ef56127..75c12362c08b 100644 --- a/.github/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml @@ -75,7 +75,7 @@ jobs: beam_PostCommit_Java_Tpcds_Dataflow: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Dataflow Runner Tpcds Tests' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 240 diff --git a/.github/workflows/beam_PostCommit_Java_Tpcds_Flink.yml b/.github/workflows/beam_PostCommit_Java_Tpcds_Flink.yml index fcd87f41326d..9717bc5e1056 100644 --- a/.github/workflows/beam_PostCommit_Java_Tpcds_Flink.yml +++ b/.github/workflows/beam_PostCommit_Java_Tpcds_Flink.yml @@ -72,7 +72,7 @@ jobs: beam_PostCommit_Java_Tpcds_Flink: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Flink Runner Tpcds Tests' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 240 diff --git a/.github/workflows/beam_PostCommit_Java_Tpcds_Spark.yml b/.github/workflows/beam_PostCommit_Java_Tpcds_Spark.yml index dd1b3a18b8cd..805eab350d7c 100644 --- a/.github/workflows/beam_PostCommit_Java_Tpcds_Spark.yml +++ b/.github/workflows/beam_PostCommit_Java_Tpcds_Spark.yml @@ -71,7 +71,7 @@ jobs: beam_PostCommit_Java_Tpcds_Spark: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Spark Runner Tpcds Tests' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 240 diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml index 4d0357d9215b..f0f1de35719d 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml @@ -59,7 +59,7 @@ jobs: job_phrase: [Run Dataflow ValidatesRunner] if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Dataflow ValidatesRunner' steps: - uses: actions/checkout@v4 diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml index 62f742f5b5b9..461456487d7c 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml @@ -61,7 +61,7 @@ jobs: java_version: ['11','17'] if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || startswith(github.event.comment.body, 'Run Dataflow ValidatesRunner Java') steps: - uses: actions/checkout@v4 diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml index 175893f5b015..19be1a9e8301 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml @@ -59,7 +59,7 @@ jobs: job_phrase: [Run Dataflow Streaming ValidatesRunner] if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Dataflow Streaming ValidatesRunner' steps: - uses: actions/checkout@v4 diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml index 723d17cf094c..b4511fac3e93 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml @@ -59,7 +59,7 @@ jobs: job_phrase: [Run Java Dataflow V2 ValidatesRunner] if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Java Dataflow V2 ValidatesRunner' steps: - uses: actions/checkout@v4 diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml index d81f7cf2752d..e89a0414a3dc 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml @@ -59,7 +59,7 @@ jobs: job_phrase: [Run Java Dataflow V2 ValidatesRunner Streaming] if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Java Dataflow V2 ValidatesRunner Streaming' steps: - uses: actions/checkout@v4 diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml index 03453f0eb646..a86b45596554 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml @@ -59,7 +59,7 @@ jobs: job_phrase: [Run Direct ValidatesRunner] if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Direct ValidatesRunner' steps: - uses: actions/checkout@v4 diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml index b16a20f6c210..41f6440a20b5 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml @@ -61,7 +61,7 @@ jobs: java_version: ['11','17'] if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || startswith(github.event.comment.body, 'Run Direct ValidatesRunner Java') steps: - uses: actions/checkout@v4 diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml index 03dd89592a73..2a4e81e787b3 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml @@ -57,7 +57,7 @@ jobs: job_phrase: [Run Flink ValidatesRunner] if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Flink ValidatesRunner' steps: - uses: actions/checkout@v4 diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml index 181483597a9f..14c186047533 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml @@ -59,7 +59,7 @@ jobs: job_phrase: [Run Flink ValidatesRunner Java 11] if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || startswith(github.event.comment.body, 'Run Flink ValidatesRunner Java 11') steps: - uses: actions/checkout@v4 diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml index 986eab3595b9..a8682db34180 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml @@ -57,7 +57,7 @@ jobs: job_phrase: [Run Samza ValidatesRunner] if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Samza ValidatesRunner' steps: - uses: actions/checkout@v4 diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml index bc01dac79e96..c8c5efb4dce1 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml @@ -57,7 +57,7 @@ jobs: job_phrase: [Run Spark ValidatesRunner] if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Spark ValidatesRunner' steps: - uses: actions/checkout@v4 diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml index 404c15e65149..df4f0c273e9a 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml @@ -57,7 +57,7 @@ jobs: job_phrase: [Run Spark StructuredStreaming ValidatesRunner] if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Spark StructuredStreaming ValidatesRunner' steps: - uses: actions/checkout@v4 diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml index d8249a17df8e..00884e35f4d9 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml @@ -59,7 +59,7 @@ jobs: job_phrase: [Run Spark ValidatesRunner Java 11] if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || startswith(github.event.comment.body, 'Run Spark ValidatesRunner Java 11') steps: - uses: actions/checkout@v4 diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml index 87db69c308ad..8335fec403c6 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml @@ -57,7 +57,7 @@ jobs: job_phrase: [Run Twister2 ValidatesRunner] if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Twister2 ValidatesRunner' steps: - uses: actions/checkout@v4 diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml index 4e3b9c49d471..7d978a75ee97 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml @@ -57,7 +57,7 @@ jobs: job_phrase: [Run ULR Loopback ValidatesRunner] if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run ULR Loopback ValidatesRunner' steps: - uses: actions/checkout@v4 diff --git a/.github/workflows/beam_PostCommit_Javadoc.yml b/.github/workflows/beam_PostCommit_Javadoc.yml index fe72554de6fb..283516714326 100644 --- a/.github/workflows/beam_PostCommit_Javadoc.yml +++ b/.github/workflows/beam_PostCommit_Javadoc.yml @@ -59,7 +59,7 @@ jobs: job_phrase: [Run Javadoc PostCommit] if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Javadoc PostCommit' steps: - uses: actions/checkout@v4 diff --git a/.github/workflows/beam_PostCommit_PortableJar_Flink.yml b/.github/workflows/beam_PostCommit_PortableJar_Flink.yml index a0d5f51d04a2..8dd433537cf1 100644 --- a/.github/workflows/beam_PostCommit_PortableJar_Flink.yml +++ b/.github/workflows/beam_PostCommit_PortableJar_Flink.yml @@ -50,7 +50,7 @@ jobs: beam_PostCommit_PortableJar_Flink: name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) if: | - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run PortableJar_Flink PostCommit' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PostCommit_PortableJar_Spark.yml b/.github/workflows/beam_PostCommit_PortableJar_Spark.yml index 500c5919d213..5cb9cb611df5 100644 --- a/.github/workflows/beam_PostCommit_PortableJar_Spark.yml +++ b/.github/workflows/beam_PostCommit_PortableJar_Spark.yml @@ -50,7 +50,7 @@ jobs: beam_PostCommit_PortableJar_Spark: name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) if: | - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run PortableJar_Spark PostCommit' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PostCommit_Python.yml b/.github/workflows/beam_PostCommit_Python.yml index 09e813e4a41f..d10690d612e4 100644 --- a/.github/workflows/beam_PostCommit_Python.yml +++ b/.github/workflows/beam_PostCommit_Python.yml @@ -61,7 +61,7 @@ jobs: python_version: ['3.8', '3.9', '3.10', '3.11'] if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || startswith(github.event.comment.body, 'Run Python PostCommit 3.') steps: - uses: actions/checkout@v4 diff --git a/.github/workflows/beam_PostCommit_Python_Arm.yml b/.github/workflows/beam_PostCommit_Python_Arm.yml index f28d6b1da783..90094636cad3 100644 --- a/.github/workflows/beam_PostCommit_Python_Arm.yml +++ b/.github/workflows/beam_PostCommit_Python_Arm.yml @@ -63,7 +63,7 @@ jobs: python_version: ['3.8', '3.9', '3.10', '3.11'] if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || startsWith(github.event.comment.body, 'Run Python PostCommit Arm') steps: - uses: actions/checkout@v4 diff --git a/.github/workflows/beam_PostCommit_Python_Examples_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_Examples_Dataflow.yml index b659b2522b19..5a548650ce0d 100644 --- a/.github/workflows/beam_PostCommit_Python_Examples_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_Examples_Dataflow.yml @@ -50,7 +50,7 @@ jobs: beam_PostCommit_Python_Examples_Dataflow: name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) if: | - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Python Examples_Dataflow' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PostCommit_Python_Examples_Direct.yml b/.github/workflows/beam_PostCommit_Python_Examples_Direct.yml index b7344be286e0..a803b6702e04 100644 --- a/.github/workflows/beam_PostCommit_Python_Examples_Direct.yml +++ b/.github/workflows/beam_PostCommit_Python_Examples_Direct.yml @@ -49,7 +49,7 @@ env: jobs: beam_PostCommit_Python_Examples_Direct: if: | - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || startsWith(github.event.comment.body, 'Run Python Examples_Direct') runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PostCommit_Python_Examples_Flink.yml b/.github/workflows/beam_PostCommit_Python_Examples_Flink.yml index af349c0ef368..527b9885a8c0 100644 --- a/.github/workflows/beam_PostCommit_Python_Examples_Flink.yml +++ b/.github/workflows/beam_PostCommit_Python_Examples_Flink.yml @@ -49,7 +49,7 @@ env: jobs: beam_PostCommit_Python_Examples_Flink: if: | - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || startsWith(github.event.comment.body, 'Run Python Examples_Flink') runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PostCommit_Python_Examples_Spark.yml b/.github/workflows/beam_PostCommit_Python_Examples_Spark.yml index 66a240eec23b..2482f2b6033e 100644 --- a/.github/workflows/beam_PostCommit_Python_Examples_Spark.yml +++ b/.github/workflows/beam_PostCommit_Python_Examples_Spark.yml @@ -49,7 +49,7 @@ env: jobs: beam_PostCommit_Python_Examples_Spark: if: | - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || startsWith(github.event.comment.body, 'Run Python Examples_Spark') runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml b/.github/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml index e406a58d01a5..d658f714a7b0 100644 --- a/.github/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml +++ b/.github/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml @@ -50,7 +50,7 @@ jobs: beam_PostCommit_Python_MongoDBIO_IT: name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) if: | - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Python MongoDBIO_IT' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PostCommit_Python_Nexmark_Direct.yml b/.github/workflows/beam_PostCommit_Python_Nexmark_Direct.yml index 9f00b6f163d1..661d0e61d8df 100644 --- a/.github/workflows/beam_PostCommit_Python_Nexmark_Direct.yml +++ b/.github/workflows/beam_PostCommit_Python_Nexmark_Direct.yml @@ -102,7 +102,7 @@ jobs: query: [0, 2, 3, 5, 7, 8, 10, 11] if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Python Direct Runner Nexmark Tests' steps: - uses: actions/checkout@v4 diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml index e572c5c9198f..46668ffb007a 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml @@ -49,7 +49,7 @@ env: jobs: beam_PostCommit_Python_ValidatesContainer_Dataflow: if: | - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || startsWith(github.event.comment.body, 'Run Python Dataflow ValidatesContainer') runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml b/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml index cfc5db3ff8e7..fc7c72a9e5e2 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml @@ -49,7 +49,7 @@ env: jobs: beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC: if: | - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || startsWith(github.event.comment.body, 'Run Python RC Dataflow ValidatesContainer') runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml index 79b2269fa26c..81fe6b811353 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml @@ -49,7 +49,7 @@ env: jobs: beam_PostCommit_Python_ValidatesRunner_Dataflow: if: | - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || startsWith(github.event.comment.body, 'Run Python Dataflow ValidatesRunner') runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml index 367a5da28fd1..e5a5b1a53773 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml @@ -49,7 +49,7 @@ env: jobs: beam_PostCommit_Python_ValidatesRunner_Flink: if: | - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || startsWith(github.event.comment.body, 'Run Python Flink ValidatesRunner') runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml index 9fff3935639d..a1215b33b086 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml @@ -49,7 +49,7 @@ env: jobs: beam_PostCommit_Python_ValidatesRunner_Samza: if: | - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || startsWith(github.event.comment.body, 'Run Python Samza ValidatesRunner') runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml index 67ec4b5af84d..eeb96712b101 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml @@ -49,7 +49,7 @@ env: jobs: beam_PostCommit_Python_ValidatesRunner_Spark: if: | - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || startsWith(github.event.comment.body, 'Run Python Spark ValidatesRunner') runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml index 65d3f70d130b..93ca3e015ac6 100644 --- a/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml @@ -50,7 +50,7 @@ jobs: beam_PostCommit_Python_Xlang_Gcp_Dataflow: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Python_Xlang_Gcp_Dataflow PostCommit' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 180 diff --git a/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml b/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml index 00bcabf3b6a4..36eeed07c257 100644 --- a/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml +++ b/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml @@ -50,7 +50,7 @@ jobs: beam_PostCommit_Python_Xlang_Gcp_Direct: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Python_Xlang_Gcp_Direct PostCommit' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 diff --git a/.github/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml index 0e73b0e4bc82..c42dbc7aa068 100644 --- a/.github/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml @@ -50,7 +50,7 @@ jobs: beam_PostCommit_Python_Xlang_IO_Dataflow: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Python_Xlang_IO_Dataflow PostCommit' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 180 diff --git a/.github/workflows/beam_PostCommit_SQL.yml b/.github/workflows/beam_PostCommit_SQL.yml index 0ac8af3c6bb8..80aad5b82908 100644 --- a/.github/workflows/beam_PostCommit_SQL.yml +++ b/.github/workflows/beam_PostCommit_SQL.yml @@ -59,7 +59,7 @@ jobs: job_phrase: [Run SQL PostCommit] if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run SQL PostCommit' steps: - uses: actions/checkout@v4 diff --git a/.github/workflows/beam_PostCommit_Sickbay_Python.yml b/.github/workflows/beam_PostCommit_Sickbay_Python.yml index b01ee16ade13..5fbffed22cd4 100644 --- a/.github/workflows/beam_PostCommit_Sickbay_Python.yml +++ b/.github/workflows/beam_PostCommit_Sickbay_Python.yml @@ -62,7 +62,7 @@ jobs: python_version: ['3.8', '3.9', '3.10', '3.11'] if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || (startswith(github.event.comment.body, 'Run Python') && endswith(github.event.comment.body, 'PostCommit Sickbay')) steps: diff --git a/.github/workflows/beam_PostCommit_TransformService_Direct.yml b/.github/workflows/beam_PostCommit_TransformService_Direct.yml index a180adf810bb..e3fac46bd9c3 100644 --- a/.github/workflows/beam_PostCommit_TransformService_Direct.yml +++ b/.github/workflows/beam_PostCommit_TransformService_Direct.yml @@ -50,7 +50,7 @@ jobs: beam_PostCommit_TransformService_Direct: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run TransformService_Direct PostCommit' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 120 diff --git a/.github/workflows/beam_PostCommit_Website_Publish.yml b/.github/workflows/beam_PostCommit_Website_Publish.yml index ef857d231094..ed1729021e98 100644 --- a/.github/workflows/beam_PostCommit_Website_Publish.yml +++ b/.github/workflows/beam_PostCommit_Website_Publish.yml @@ -50,7 +50,7 @@ jobs: beam_PostCommit_Website_Publish: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' + (github.event_name == 'schedule' && github.repository == 'apache/beam') runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 30 name: beam_PostCommit_Website_Publish diff --git a/.github/workflows/beam_PostCommit_Website_Test.yml b/.github/workflows/beam_PostCommit_Website_Test.yml index 3782b9ca17dd..06f9bcd73243 100644 --- a/.github/workflows/beam_PostCommit_Website_Test.yml +++ b/.github/workflows/beam_PostCommit_Website_Test.yml @@ -50,7 +50,7 @@ jobs: beam_PostCommit_Website_Test: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Full Website Test' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 60 diff --git a/.github/workflows/beam_PostCommit_XVR_Direct.yml b/.github/workflows/beam_PostCommit_XVR_Direct.yml index c63969051a07..247aa4c23e11 100644 --- a/.github/workflows/beam_PostCommit_XVR_Direct.yml +++ b/.github/workflows/beam_PostCommit_XVR_Direct.yml @@ -50,7 +50,7 @@ jobs: beam_PostCommit_XVR_Direct: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run XVR_Direct PostCommit' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 120 diff --git a/.github/workflows/beam_PostCommit_XVR_Flink.yml b/.github/workflows/beam_PostCommit_XVR_Flink.yml index ea6e0b55c764..ad90720cd4f2 100644 --- a/.github/workflows/beam_PostCommit_XVR_Flink.yml +++ b/.github/workflows/beam_PostCommit_XVR_Flink.yml @@ -51,7 +51,7 @@ jobs: beam_PostCommit_XVR_Flink: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run XVR_Flink PostCommit' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 diff --git a/.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml b/.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml index 2d37e0522c93..0fa27b2c6dc8 100644 --- a/.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml @@ -50,7 +50,7 @@ jobs: beam_PostCommit_XVR_GoUsingJava_Dataflow: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run XVR_GoUsingJava_Dataflow PostCommit' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 diff --git a/.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml b/.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml index 494ab1e277d3..cbf666157c9b 100644 --- a/.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml @@ -50,7 +50,7 @@ jobs: beam_PostCommit_XVR_JavaUsingPython_Dataflow: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run XVR_JavaUsingPython_Dataflow PostCommit' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 diff --git a/.github/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml b/.github/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml index 52adbc2ad824..e90449742ce4 100644 --- a/.github/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml @@ -50,7 +50,7 @@ jobs: beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run XVR_PythonUsingJavaSQL_Dataflow PostCommit' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 diff --git a/.github/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml b/.github/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml index 796544539dc2..5b650fcf003b 100644 --- a/.github/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml @@ -50,7 +50,7 @@ jobs: beam_PostCommit_XVR_PythonUsingJava_Dataflow: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run XVR_PythonUsingJava_Dataflow PostCommit' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 diff --git a/.github/workflows/beam_PostCommit_XVR_Samza.yml b/.github/workflows/beam_PostCommit_XVR_Samza.yml index 0f08490e415b..34b226e93e5c 100644 --- a/.github/workflows/beam_PostCommit_XVR_Samza.yml +++ b/.github/workflows/beam_PostCommit_XVR_Samza.yml @@ -50,7 +50,7 @@ jobs: beam_PostCommit_XVR_Samza: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run XVR_Samza PostCommit' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 diff --git a/.github/workflows/beam_PostCommit_XVR_Spark3.yml b/.github/workflows/beam_PostCommit_XVR_Spark3.yml index f49765381fad..5419883911d8 100644 --- a/.github/workflows/beam_PostCommit_XVR_Spark3.yml +++ b/.github/workflows/beam_PostCommit_XVR_Spark3.yml @@ -50,7 +50,7 @@ jobs: beam_PostCommit_XVR_Spark3: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run XVR_Spark3 PostCommit' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 diff --git a/.github/workflows/beam_PreCommit_CommunityMetrics.yml b/.github/workflows/beam_PreCommit_CommunityMetrics.yml index 46afb4017882..570dd5dc3354 100644 --- a/.github/workflows/beam_PreCommit_CommunityMetrics.yml +++ b/.github/workflows/beam_PreCommit_CommunityMetrics.yml @@ -67,7 +67,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run CommunityMetrics PreCommit' steps: diff --git a/.github/workflows/beam_PreCommit_GHA.yml b/.github/workflows/beam_PreCommit_GHA.yml index 79970bcfaa00..94811bc0f507 100644 --- a/.github/workflows/beam_PreCommit_GHA.yml +++ b/.github/workflows/beam_PreCommit_GHA.yml @@ -67,7 +67,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run GHA PreCommit' steps: diff --git a/.github/workflows/beam_PreCommit_Go.yml b/.github/workflows/beam_PreCommit_Go.yml index 57239d5a399d..a59a2ed5e34d 100644 --- a/.github/workflows/beam_PreCommit_Go.yml +++ b/.github/workflows/beam_PreCommit_Go.yml @@ -67,7 +67,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Go PreCommit' steps: diff --git a/.github/workflows/beam_PreCommit_GoPortable.yml b/.github/workflows/beam_PreCommit_GoPortable.yml index 213d0b25999a..231828ee6208 100644 --- a/.github/workflows/beam_PreCommit_GoPortable.yml +++ b/.github/workflows/beam_PreCommit_GoPortable.yml @@ -67,7 +67,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run GoPortable PreCommit' steps: diff --git a/.github/workflows/beam_PreCommit_GoPrism.yml b/.github/workflows/beam_PreCommit_GoPrism.yml index 17fece4bc722..00da770600b0 100644 --- a/.github/workflows/beam_PreCommit_GoPrism.yml +++ b/.github/workflows/beam_PreCommit_GoPrism.yml @@ -67,7 +67,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run GoPrism PreCommit' steps: diff --git a/.github/workflows/beam_PreCommit_ItFramework.yml b/.github/workflows/beam_PreCommit_ItFramework.yml index 8a5179099db5..cf4cd2980176 100644 --- a/.github/workflows/beam_PreCommit_ItFramework.yml +++ b/.github/workflows/beam_PreCommit_ItFramework.yml @@ -71,7 +71,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run It_Framework PreCommit' steps: diff --git a/.github/workflows/beam_PreCommit_Java.yml b/.github/workflows/beam_PreCommit_Java.yml index ea3340beebac..fe694b4eab70 100644 --- a/.github/workflows/beam_PreCommit_Java.yml +++ b/.github/workflows/beam_PreCommit_Java.yml @@ -164,7 +164,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Java PreCommit' steps: diff --git a/.github/workflows/beam_PreCommit_Java_Amazon-Web-Services2_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Amazon-Web-Services2_IO_Direct.yml index 4591fcd3dacc..16d6562f827f 100644 --- a/.github/workflows/beam_PreCommit_Java_Amazon-Web-Services2_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Amazon-Web-Services2_IO_Direct.yml @@ -88,7 +88,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Java_Amazon-Web-Services2_IO_Direct PreCommit' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PreCommit_Java_Amazon-Web-Services_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Amazon-Web-Services_IO_Direct.yml index 960c59587085..f7c1d66e2f1e 100644 --- a/.github/workflows/beam_PreCommit_Java_Amazon-Web-Services_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Amazon-Web-Services_IO_Direct.yml @@ -88,7 +88,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Java_Amazon-Web-Services_IO_Direct PreCommit' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PreCommit_Java_Amqp_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Amqp_IO_Direct.yml index e23ccea56d8b..8df097d8428f 100644 --- a/.github/workflows/beam_PreCommit_Java_Amqp_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Amqp_IO_Direct.yml @@ -70,7 +70,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Java_Amqp_IO_Direct PreCommit' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml index 1b997178aa6a..d32361708491 100644 --- a/.github/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml @@ -88,7 +88,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Java_Azure_IO_Direct PreCommit' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml index e026a1536fb8..04e1b10c23bd 100644 --- a/.github/workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml @@ -70,7 +70,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Java_Cassandra_IO_Direct PreCommit' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml index b1c324720b3d..f23aad3ea455 100644 --- a/.github/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml @@ -74,7 +74,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Java_Cdap_IO_Direct PreCommit' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml index f20daacd89a8..f7c5ea1b8615 100644 --- a/.github/workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml @@ -70,7 +70,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Java_Clickhouse_IO_Direct PreCommit' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml index cf7f99605c29..496e0c15ea66 100644 --- a/.github/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml @@ -70,7 +70,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Java_Csv_IO_Direct PreCommit' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml index ac68cc6040fb..97264ac146a5 100644 --- a/.github/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml @@ -70,7 +70,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Java_Debezium_IO_Direct PreCommit' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml index e6176729b376..f51e50aa8200 100644 --- a/.github/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml @@ -72,7 +72,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Java_ElasticSearch_IO_Direct PreCommit' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow.yml b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow.yml index fc099666bb15..b96e459470cf 100644 --- a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow.yml +++ b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow.yml @@ -82,7 +82,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Java_Examples_Dataflow PreCommit' steps: diff --git a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml index 132665df05a0..e63b7652ce75 100644 --- a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml +++ b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml @@ -83,7 +83,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Java_Examples_Dataflow_Java11 PreCommit' steps: diff --git a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml index 6ae9eaf2572d..464800d44635 100644 --- a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml +++ b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml @@ -80,7 +80,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Java_Examples_Dataflow_Java17 PreCommit' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PreCommit_Java_File-schema-transform_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_File-schema-transform_IO_Direct.yml index 2446eb972c62..8ce77fef6416 100644 --- a/.github/workflows/beam_PreCommit_Java_File-schema-transform_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_File-schema-transform_IO_Direct.yml @@ -70,7 +70,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Java_File-schema-transform_IO_Direct PreCommit' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PreCommit_Java_Flink_Versions.yml b/.github/workflows/beam_PreCommit_Java_Flink_Versions.yml index 9850183ba4a0..d5c244b2dbc9 100644 --- a/.github/workflows/beam_PreCommit_Java_Flink_Versions.yml +++ b/.github/workflows/beam_PreCommit_Java_Flink_Versions.yml @@ -71,7 +71,7 @@ jobs: github.event_name == 'push' || github.event_name == 'workflow_dispatch' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Java_Flink_Versions PreCommit' steps: - uses: actions/checkout@v4 diff --git a/.github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml index e5ef36c00451..73dd9d63d872 100644 --- a/.github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml @@ -88,7 +88,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Java_GCP_IO_Direct PreCommit' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PreCommit_Java_Google-ads_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Google-ads_IO_Direct.yml index 559da13a430a..7e0ca73e7efd 100644 --- a/.github/workflows/beam_PreCommit_Java_Google-ads_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Google-ads_IO_Direct.yml @@ -70,7 +70,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Java_Google-ads_IO_Direct PreCommit' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml index dce33b36b9ce..805561f3475d 100644 --- a/.github/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml @@ -72,7 +72,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Java_HBase_IO_Direct PreCommit' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml index 85723824ac52..bb9e5ee31f27 100644 --- a/.github/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml @@ -72,7 +72,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Java_HCatalog_IO_Direct PreCommit' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml index ec3a0c5bd839..ea09a8df70d6 100644 --- a/.github/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml @@ -96,7 +96,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Java_Hadoop_IO_Direct PreCommit' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml index 67ee59b7c4ff..52a03ceea204 100644 --- a/.github/workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml @@ -70,7 +70,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Java_InfluxDb_IO_Direct PreCommit' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml index 4b86f8b23038..f328dd513c88 100644 --- a/.github/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml @@ -70,7 +70,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Java_JDBC_IO_Direct PreCommit' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml index 686cc759ca31..9abd63bb79c0 100644 --- a/.github/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml @@ -70,7 +70,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Java_Jms_IO_Direct PreCommit' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml index 86d59f7696ff..c73c384c7881 100644 --- a/.github/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml @@ -78,7 +78,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Java_Kafka_IO_Direct PreCommit' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml index bfe0d50ec048..2aa4d4c10fd8 100644 --- a/.github/workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml @@ -88,7 +88,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Java_Kinesis_IO_Direct PreCommit' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml index 55c81bb501a2..25f90072db5c 100644 --- a/.github/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml @@ -70,7 +70,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Java_Kudu_IO_Direct PreCommit' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml index 563c8714fbf4..34fa18a23c71 100644 --- a/.github/workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml @@ -70,7 +70,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Java_MongoDb_IO_Direct PreCommit' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml index b85697d7e39f..0b26f28e8f69 100644 --- a/.github/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml @@ -70,7 +70,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Java_Mqtt_IO_Direct PreCommit' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml index 21f746ef2cec..ed6d6baf456b 100644 --- a/.github/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml @@ -72,7 +72,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Java_Neo4j_IO_Direct PreCommit' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PreCommit_Java_PVR_Flink_Batch.yml b/.github/workflows/beam_PreCommit_Java_PVR_Flink_Batch.yml index eb2f85b5e106..0ff7785650de 100644 --- a/.github/workflows/beam_PreCommit_Java_PVR_Flink_Batch.yml +++ b/.github/workflows/beam_PreCommit_Java_PVR_Flink_Batch.yml @@ -77,7 +77,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Java_PVR_Flink_Batch PreCommit' steps: diff --git a/.github/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml b/.github/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml index b3b5bd5b344b..841c6dbdcbab 100644 --- a/.github/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml +++ b/.github/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml @@ -81,7 +81,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Java_PVR_Flink_Docker PreCommit' timeout-minutes: 240 diff --git a/.github/workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml index 9e52618edf9f..998a6bbbac04 100644 --- a/.github/workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml @@ -70,7 +70,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Java_Parquet_IO_Direct PreCommit' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml index b60f53cc31fc..00becbfd7ee6 100644 --- a/.github/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml @@ -88,7 +88,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Java_Pulsar_IO_Direct PreCommit' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml index d917c2830f51..7adbfe74bab5 100644 --- a/.github/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml @@ -70,7 +70,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Java_RabbitMq_IO_Direct PreCommit' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml index 91095a000e18..255df740a57f 100644 --- a/.github/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml @@ -70,7 +70,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Java_Redis_IO_Direct PreCommit' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PreCommit_Java_RequestResponse_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_RequestResponse_IO_Direct.yml index 27de388a6a65..cfe721c1e948 100644 --- a/.github/workflows/beam_PreCommit_Java_RequestResponse_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_RequestResponse_IO_Direct.yml @@ -70,7 +70,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Java_RequestResponse_IO_Direct PreCommit' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml index b380e96b813f..733921da7ad2 100644 --- a/.github/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml @@ -72,7 +72,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Java_SingleStore_IO_Direct PreCommit' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml index fab5fd4be694..c84f0026b726 100644 --- a/.github/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml @@ -74,7 +74,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Java_Snowflake_IO_Direct PreCommit' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml index 06ed141281d0..f3142b41dd4d 100644 --- a/.github/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml @@ -70,7 +70,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Java_Solr_IO_Direct PreCommit' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PreCommit_Java_Spark3_Versions.yml b/.github/workflows/beam_PreCommit_Java_Spark3_Versions.yml index aaab30164805..9665e9770bc0 100644 --- a/.github/workflows/beam_PreCommit_Java_Spark3_Versions.yml +++ b/.github/workflows/beam_PreCommit_Java_Spark3_Versions.yml @@ -73,7 +73,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Java_Spark3_Versions PreCommit' steps: diff --git a/.github/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml index f31ad63d1cf5..7032cc62a62a 100644 --- a/.github/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml @@ -70,7 +70,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Java_Splunk_IO_Direct PreCommit' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml index bb22fa8aaca4..da26e226e105 100644 --- a/.github/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml @@ -70,7 +70,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Java_Thrift_IO_Direct PreCommit' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml index 1dbb51bdf2a7..20eff37521e2 100644 --- a/.github/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml @@ -70,7 +70,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Java_Tika_IO_Direct PreCommit' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PreCommit_Kotlin_Examples.yml b/.github/workflows/beam_PreCommit_Kotlin_Examples.yml index 613a082a74fd..13d533442575 100644 --- a/.github/workflows/beam_PreCommit_Kotlin_Examples.yml +++ b/.github/workflows/beam_PreCommit_Kotlin_Examples.yml @@ -83,7 +83,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Kotlin_Examples PreCommit' steps: diff --git a/.github/workflows/beam_PreCommit_Portable_Python.yml b/.github/workflows/beam_PreCommit_Portable_Python.yml index 1540600b7205..169bdb74a649 100644 --- a/.github/workflows/beam_PreCommit_Portable_Python.yml +++ b/.github/workflows/beam_PreCommit_Portable_Python.yml @@ -89,7 +89,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || startsWith(github.event.comment.body, 'Run Portable_Python PreCommit') steps: - uses: actions/checkout@v4 diff --git a/.github/workflows/beam_PreCommit_Python.yml b/.github/workflows/beam_PreCommit_Python.yml index 53cc2af4a07d..670f41875ffb 100644 --- a/.github/workflows/beam_PreCommit_Python.yml +++ b/.github/workflows/beam_PreCommit_Python.yml @@ -68,7 +68,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || startsWith(github.event.comment.body, 'Run Python PreCommit') steps: diff --git a/.github/workflows/beam_PreCommit_PythonDocker.yml b/.github/workflows/beam_PreCommit_PythonDocker.yml index a84ba8fd3a26..0a0ee5de32a8 100644 --- a/.github/workflows/beam_PreCommit_PythonDocker.yml +++ b/.github/workflows/beam_PreCommit_PythonDocker.yml @@ -68,7 +68,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || startsWith(github.event.comment.body, 'Run PythonDocker PreCommit') steps: - uses: actions/checkout@v4 diff --git a/.github/workflows/beam_PreCommit_PythonDocs.yml b/.github/workflows/beam_PreCommit_PythonDocs.yml index 3f641681d91c..2ada89147321 100644 --- a/.github/workflows/beam_PreCommit_PythonDocs.yml +++ b/.github/workflows/beam_PreCommit_PythonDocs.yml @@ -67,7 +67,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run PythonDocs PreCommit' steps: diff --git a/.github/workflows/beam_PreCommit_PythonFormatter.yml b/.github/workflows/beam_PreCommit_PythonFormatter.yml index 70f13a2ee736..17d9cd855138 100644 --- a/.github/workflows/beam_PreCommit_PythonFormatter.yml +++ b/.github/workflows/beam_PreCommit_PythonFormatter.yml @@ -66,7 +66,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run PythonFormatter PreCommit' steps: diff --git a/.github/workflows/beam_PreCommit_PythonLint.yml b/.github/workflows/beam_PreCommit_PythonLint.yml index 199640d7b9ac..583c484e8e93 100644 --- a/.github/workflows/beam_PreCommit_PythonLint.yml +++ b/.github/workflows/beam_PreCommit_PythonLint.yml @@ -66,7 +66,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run PythonLint PreCommit' steps: diff --git a/.github/workflows/beam_PreCommit_Python_Coverage.yml b/.github/workflows/beam_PreCommit_Python_Coverage.yml index e04f1006c2af..f0444013df59 100644 --- a/.github/workflows/beam_PreCommit_Python_Coverage.yml +++ b/.github/workflows/beam_PreCommit_Python_Coverage.yml @@ -66,7 +66,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Python_Coverage PreCommit' steps: diff --git a/.github/workflows/beam_PreCommit_Python_Dataframes.yml b/.github/workflows/beam_PreCommit_Python_Dataframes.yml index 8024f8d53909..822c10862ac7 100644 --- a/.github/workflows/beam_PreCommit_Python_Dataframes.yml +++ b/.github/workflows/beam_PreCommit_Python_Dataframes.yml @@ -68,7 +68,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || startsWith(github.event.comment.body, 'Run Python_Dataframes PreCommit') steps: diff --git a/.github/workflows/beam_PreCommit_Python_Examples.yml b/.github/workflows/beam_PreCommit_Python_Examples.yml index 19839118dbff..36a0bde3ebdc 100644 --- a/.github/workflows/beam_PreCommit_Python_Examples.yml +++ b/.github/workflows/beam_PreCommit_Python_Examples.yml @@ -68,7 +68,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || startsWith(github.event.comment.body, 'Run Python_Examples PreCommit') steps: diff --git a/.github/workflows/beam_PreCommit_Python_Integration.yml b/.github/workflows/beam_PreCommit_Python_Integration.yml index c8f22339565d..d0af4932168e 100644 --- a/.github/workflows/beam_PreCommit_Python_Integration.yml +++ b/.github/workflows/beam_PreCommit_Python_Integration.yml @@ -68,7 +68,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || startsWith(github.event.comment.body, 'Run Python_Integration PreCommit') steps: diff --git a/.github/workflows/beam_PreCommit_Python_PVR_Flink.yml b/.github/workflows/beam_PreCommit_Python_PVR_Flink.yml index 2725cc383f2a..7e083bf84313 100644 --- a/.github/workflows/beam_PreCommit_Python_PVR_Flink.yml +++ b/.github/workflows/beam_PreCommit_Python_PVR_Flink.yml @@ -88,7 +88,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Python_PVR_Flink PreCommit' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PreCommit_Python_Runners.yml b/.github/workflows/beam_PreCommit_Python_Runners.yml index d622f9e6bbbd..f43c4eb47b6e 100644 --- a/.github/workflows/beam_PreCommit_Python_Runners.yml +++ b/.github/workflows/beam_PreCommit_Python_Runners.yml @@ -68,7 +68,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || startsWith(github.event.comment.body, 'Run Python_Runners PreCommit') steps: diff --git a/.github/workflows/beam_PreCommit_Python_Transforms.yml b/.github/workflows/beam_PreCommit_Python_Transforms.yml index d034296038ea..6d06596ed4c8 100644 --- a/.github/workflows/beam_PreCommit_Python_Transforms.yml +++ b/.github/workflows/beam_PreCommit_Python_Transforms.yml @@ -68,7 +68,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || startsWith(github.event.comment.body, 'Run Python_Transforms PreCommit') steps: diff --git a/.github/workflows/beam_PreCommit_RAT.yml b/.github/workflows/beam_PreCommit_RAT.yml index 200e69f94bbf..c45ae4fc5543 100644 --- a/.github/workflows/beam_PreCommit_RAT.yml +++ b/.github/workflows/beam_PreCommit_RAT.yml @@ -65,7 +65,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run RAT PreCommit' steps: diff --git a/.github/workflows/beam_PreCommit_SQL.yml b/.github/workflows/beam_PreCommit_SQL.yml index f4a4e30a01ff..ac72f6b752da 100644 --- a/.github/workflows/beam_PreCommit_SQL.yml +++ b/.github/workflows/beam_PreCommit_SQL.yml @@ -67,7 +67,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run SQL PreCommit' steps: diff --git a/.github/workflows/beam_PreCommit_SQL_Java11.yml b/.github/workflows/beam_PreCommit_SQL_Java11.yml index 92b8fe366494..9dadb4dcc2ff 100644 --- a/.github/workflows/beam_PreCommit_SQL_Java11.yml +++ b/.github/workflows/beam_PreCommit_SQL_Java11.yml @@ -67,7 +67,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run SQL_Java11 PreCommit' steps: diff --git a/.github/workflows/beam_PreCommit_SQL_Java17.yml b/.github/workflows/beam_PreCommit_SQL_Java17.yml index 7d6617177613..8a1e88f7c654 100644 --- a/.github/workflows/beam_PreCommit_SQL_Java17.yml +++ b/.github/workflows/beam_PreCommit_SQL_Java17.yml @@ -66,7 +66,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run SQL_Java17 PreCommit' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PreCommit_Spotless.yml b/.github/workflows/beam_PreCommit_Spotless.yml index b4bd436a19f6..c3a2f08d65f1 100644 --- a/.github/workflows/beam_PreCommit_Spotless.yml +++ b/.github/workflows/beam_PreCommit_Spotless.yml @@ -81,7 +81,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Spotless PreCommit' steps: diff --git a/.github/workflows/beam_PreCommit_Typescript.yml b/.github/workflows/beam_PreCommit_Typescript.yml index 5459ebe89522..f3184dc0f90d 100644 --- a/.github/workflows/beam_PreCommit_Typescript.yml +++ b/.github/workflows/beam_PreCommit_Typescript.yml @@ -68,7 +68,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Typescript PreCommit' steps: diff --git a/.github/workflows/beam_PreCommit_Website.yml b/.github/workflows/beam_PreCommit_Website.yml index 3eb60c2cd9ef..848b4d44deec 100644 --- a/.github/workflows/beam_PreCommit_Website.yml +++ b/.github/workflows/beam_PreCommit_Website.yml @@ -67,7 +67,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Website PreCommit' steps: diff --git a/.github/workflows/beam_PreCommit_Website_Stage_GCS.yml b/.github/workflows/beam_PreCommit_Website_Stage_GCS.yml index e89d092be16c..c7b51f137290 100644 --- a/.github/workflows/beam_PreCommit_Website_Stage_GCS.yml +++ b/.github/workflows/beam_PreCommit_Website_Stage_GCS.yml @@ -69,7 +69,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Website_Stage_GCS PreCommit' steps: diff --git a/.github/workflows/beam_PreCommit_Whitespace.yml b/.github/workflows/beam_PreCommit_Whitespace.yml index dff494c81370..da58d309f354 100644 --- a/.github/workflows/beam_PreCommit_Whitespace.yml +++ b/.github/workflows/beam_PreCommit_Whitespace.yml @@ -66,7 +66,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Whitespace PreCommit' steps: diff --git a/.github/workflows/beam_Prober_CommunityMetrics.yml b/.github/workflows/beam_Prober_CommunityMetrics.yml index ce788aca216e..3526b3ced8c0 100644 --- a/.github/workflows/beam_Prober_CommunityMetrics.yml +++ b/.github/workflows/beam_Prober_CommunityMetrics.yml @@ -52,7 +52,7 @@ jobs: beam_Prober_CommunityMetrics: if: | github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Community Metrics Prober' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 diff --git a/.github/workflows/beam_Publish_Beam_SDK_Snapshots.yml b/.github/workflows/beam_Publish_Beam_SDK_Snapshots.yml index 95f993bc33ac..f91717a86d5b 100644 --- a/.github/workflows/beam_Publish_Beam_SDK_Snapshots.yml +++ b/.github/workflows/beam_Publish_Beam_SDK_Snapshots.yml @@ -50,9 +50,8 @@ env: jobs: beam_Publish_Beam_SDK_Snapshots: if: | - (github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule') && - github.repository == 'apache/beam' + github.event_name == 'workflow_dispatch' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 name: ${{ matrix.job_name }} (${{ matrix.container_task }}) diff --git a/.github/workflows/beam_Publish_Docker_Snapshots.yml b/.github/workflows/beam_Publish_Docker_Snapshots.yml index 1abc26863067..158f51c57d94 100644 --- a/.github/workflows/beam_Publish_Docker_Snapshots.yml +++ b/.github/workflows/beam_Publish_Docker_Snapshots.yml @@ -50,10 +50,9 @@ env: jobs: beam_Publish_Docker_Snapshots: if: | - (github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule' || - github.event.comment.body == 'Publish Docker Snapshots') && - github.repository == 'apache/beam' + github.event_name == 'workflow_dispatch' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || + github.event.comment.body == 'Publish Docker Snapshots' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) diff --git a/.github/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml b/.github/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml index eaf3f2c5ca1a..fe00e921d3b0 100644 --- a/.github/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml +++ b/.github/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml @@ -61,7 +61,7 @@ jobs: python_version: ['3.8','3.9','3.10','3.11'] if: | github.event_name == 'push' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || startsWith(github.event.comment.body, 'Run Python ValidatesContainer Dataflow ARM') diff --git a/.github/workflows/beam_Release_NightlySnapshot.yml b/.github/workflows/beam_Release_NightlySnapshot.yml index 735d0e466177..8f6b85218a9a 100644 --- a/.github/workflows/beam_Release_NightlySnapshot.yml +++ b/.github/workflows/beam_Release_NightlySnapshot.yml @@ -50,9 +50,8 @@ jobs: job_name: [beam_Release_NightlySnapshot] job_phrase: [Release Nightly Snapshot] if: | - (github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule') && - github.repository == 'apache/beam' + github.event_name == 'workflow_dispatch' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') steps: - uses: actions/checkout@v4 - name: Setup repository diff --git a/.github/workflows/beam_Release_Python_NightlySnapshot.yml b/.github/workflows/beam_Release_Python_NightlySnapshot.yml index fe132356fc53..eca74ea5952a 100644 --- a/.github/workflows/beam_Release_Python_NightlySnapshot.yml +++ b/.github/workflows/beam_Release_Python_NightlySnapshot.yml @@ -49,10 +49,9 @@ jobs: matrix: job_name: [beam_Release_Python_NightlySnapshot] job_phrase: [Release Nightly Snapshot Python] - if: (github.event_name == 'workflow_dispatch' || - github.event_name == 'schedule') && - github.repository == 'apache/beam' - + if: | + github.event_name == 'workflow_dispatch' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') steps: - uses: actions/checkout@v4 - name: Setup repository From b5134202a0c4dfe8cbb635cd0002554a544f49a2 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw <robertwb@gmail.com> Date: Tue, 31 Oct 2023 16:01:56 -0700 Subject: [PATCH 323/435] [YAML] Implement basic java mapping operations. (#28657) --- .../JavaMapToFieldsTransformProvider.java | 251 +++++++++++++ .../transforms/providers/JavaRowUdf.java | 342 ++++++++++++++++++ .../transforms/providers/StringCompiler.java | 262 ++++++++++++++ .../transforms/providers/package-info.java | 27 ++ .../JavaMapToFieldsTransformProviderTest.java | 196 ++++++++++ .../transforms/providers/JavaRowUdfTest.java | 158 ++++++++ .../providers/StringCompilerTest.java | 71 ++++ ...ionServiceSchemaTransformProviderTest.java | 3 +- .../apache_beam/yaml/standard_providers.yaml | 28 ++ 9 files changed, 1337 insertions(+), 1 deletion(-) create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/JavaMapToFieldsTransformProvider.java create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/JavaRowUdf.java create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/StringCompiler.java create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/package-info.java create mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/providers/JavaMapToFieldsTransformProviderTest.java create mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/providers/JavaRowUdfTest.java create mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/providers/StringCompilerTest.java diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/JavaMapToFieldsTransformProvider.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/JavaMapToFieldsTransformProvider.java new file mode 100644 index 000000000000..ddf892f03fc2 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/JavaMapToFieldsTransformProvider.java @@ -0,0 +1,251 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.schemas.transforms.providers; + +import com.google.auto.service.AutoService; +import com.google.auto.value.AutoValue; +import java.net.MalformedURLException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import javax.annotation.Nullable; +import org.apache.beam.sdk.schemas.AutoValueSchema; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.annotations.DefaultSchema; +import org.apache.beam.sdk.schemas.annotations.SchemaFieldDescription; +import org.apache.beam.sdk.schemas.transforms.SchemaTransform; +import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider; +import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.PCollectionRowTuple; +import org.apache.beam.sdk.values.PCollectionTuple; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.TupleTagList; + +/** + * An implementation of {@link TypedSchemaTransformProvider} for MapToFields for the java language. + * + * <p><b>Internal only:</b> This class is actively being worked on, and it will likely change. We + * provide no backwards compatibility guarantees, and it should not be implemented outside the Beam + * repository. + */ +@SuppressWarnings({ + "nullness" // TODO(https://github.com/apache/beam/issues/20497) +}) +@AutoService(SchemaTransformProvider.class) +public class JavaMapToFieldsTransformProvider + extends TypedSchemaTransformProvider<JavaMapToFieldsTransformProvider.Configuration> { + protected static final String INPUT_ROWS_TAG = "input"; + protected static final String OUTPUT_ROWS_TAG = "output"; + + @Override + protected Class<Configuration> configurationClass() { + return Configuration.class; + } + + @Override + protected SchemaTransform from(Configuration configuration) { + return new JavaMapToFieldsTransform(configuration); + } + + @Override + public String identifier() { + return String.format("beam:schematransform:org.apache.beam:yaml:map_to_fields-java:v1"); + } + + @Override + public List<String> inputCollectionNames() { + return Collections.singletonList(INPUT_ROWS_TAG); + } + + @Override + public List<String> outputCollectionNames() { + return Collections.singletonList(OUTPUT_ROWS_TAG); + } + + @DefaultSchema(AutoValueSchema.class) + @AutoValue + public abstract static class Configuration { + @Nullable + public abstract String getLanguage(); + + @Nullable + public abstract Boolean getAppend(); + + @Nullable + public abstract List<String> getDrop(); + + public abstract Map<String, JavaRowUdf.Configuration> getFields(); + + @Nullable + public abstract ErrorHandling getErrorHandling(); + + public static Builder builder() { + return new AutoValue_JavaMapToFieldsTransformProvider_Configuration.Builder(); + } + + @AutoValue.Builder + public abstract static class Builder { + + public abstract Builder setLanguage(String language); + + public abstract Builder setAppend(Boolean append); + + public abstract Builder setDrop(List<String> drop); + + public abstract Builder setFields(Map<String, JavaRowUdf.Configuration> fields); + + public abstract Builder setErrorHandling(ErrorHandling errorHandling); + + public abstract Configuration build(); + } + + @AutoValue + public abstract static class ErrorHandling { + @SchemaFieldDescription("The name of the output PCollection containing failed writes.") + public abstract String getOutput(); + + public static Builder builder() { + return new AutoValue_JavaMapToFieldsTransformProvider_Configuration_ErrorHandling.Builder(); + } + + @AutoValue.Builder + public abstract static class Builder { + public abstract Builder setOutput(String output); + + public abstract ErrorHandling build(); + } + } + } + + /** A {@link SchemaTransform} for MapToFields-java. */ + protected static class JavaMapToFieldsTransform extends SchemaTransform { + + private final Configuration configuration; + + JavaMapToFieldsTransform(Configuration configuration) { + this.configuration = configuration; + } + + @Override + public PCollectionRowTuple expand(PCollectionRowTuple input) { + Schema inputSchema = input.get(INPUT_ROWS_TAG).getSchema(); + Schema.Builder outputSchemaBuilder = new Schema.Builder(); + // TODO(yaml): Consider allowing the full java schema naming syntax + // (perhaps as a different dialect/language). + boolean append = configuration.getAppend() != null && configuration.getAppend(); + List<String> toDrop = + configuration.getDrop() == null ? Collections.emptyList() : configuration.getDrop(); + List<JavaRowUdf> udfs = new ArrayList<>(); + if (append) { + for (Schema.Field field : inputSchema.getFields()) { + if (!toDrop.contains(field.getName())) { + try { + udfs.add( + new JavaRowUdf( + JavaRowUdf.Configuration.builder().setExpression(field.getName()).build(), + inputSchema)); + } catch (MalformedURLException + | ReflectiveOperationException + | StringCompiler.CompileException exn) { + throw new RuntimeException(exn); + } + outputSchemaBuilder = outputSchemaBuilder.addField(field); + } + } + } + for (Map.Entry<String, JavaRowUdf.Configuration> entry : + configuration.getFields().entrySet()) { + if (!"java".equals(configuration.getLanguage())) { + String expr = entry.getValue().getExpression(); + if (expr == null || !inputSchema.hasField(expr)) { + throw new IllegalArgumentException( + "Unknown field or missing language specification for '" + entry.getKey() + "'"); + } + } + try { + JavaRowUdf udf = new JavaRowUdf(entry.getValue(), inputSchema); + udfs.add(udf); + outputSchemaBuilder = outputSchemaBuilder.addField(entry.getKey(), udf.getOutputType()); + } catch (MalformedURLException + | ReflectiveOperationException + | StringCompiler.CompileException exn) { + throw new RuntimeException(exn); + } + } + Schema outputSchema = outputSchemaBuilder.build(); + boolean handleErrors = + configuration.getErrorHandling() != null + && configuration.getErrorHandling().getOutput() != null; + Schema errorSchema = + Schema.of( + Schema.Field.of("failed_row", Schema.FieldType.row(inputSchema)), + Schema.Field.of("error_message", Schema.FieldType.STRING)); + + PCollectionTuple pcolls = + input + .get(INPUT_ROWS_TAG) + .apply( + "MapToFields", + ParDo.of(createDoFn(udfs, outputSchema, errorSchema, handleErrors)) + .withOutputTags(mappedValues, TupleTagList.of(errorValues))); + pcolls.get(mappedValues).setRowSchema(outputSchema); + pcolls.get(errorValues).setRowSchema(errorSchema); + + PCollectionRowTuple result = + PCollectionRowTuple.of(OUTPUT_ROWS_TAG, pcolls.get(mappedValues)); + if (handleErrors) { + result = result.and(configuration.getErrorHandling().getOutput(), pcolls.get(errorValues)); + } + return result; + } + + private static final TupleTag<Row> mappedValues = new TupleTag<Row>() {}; + private static final TupleTag<Row> errorValues = new TupleTag<Row>() {}; + + private static DoFn<Row, Row> createDoFn( + List<JavaRowUdf> udfs, Schema outputSchema, Schema errorSchema, boolean handleErrors) { + return new DoFn<Row, Row>() { + @ProcessElement + public void processElement(@Element Row inputRow, MultiOutputReceiver out) { + try { + Row.Builder outputRow = Row.withSchema(outputSchema); + for (JavaRowUdf udf : udfs) { + outputRow.addValue(udf.getFunction().apply(inputRow)); + } + out.get(mappedValues).output(outputRow.build()); + } catch (Exception exn) { + if (handleErrors) { + out.get(errorValues) + .output( + Row.withSchema(errorSchema) + .withFieldValue("failed_row", inputRow) + .withFieldValue("error_message", exn.getMessage()) + .build()); + } else { + throw new RuntimeException(exn); + } + } + } + }; + } + } +} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/JavaRowUdf.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/JavaRowUdf.java new file mode 100644 index 000000000000..e18df5b29a59 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/JavaRowUdf.java @@ -0,0 +1,342 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.schemas.transforms.providers; + +import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; + +import com.google.auto.value.AutoValue; +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; +import java.io.File; +import java.io.IOException; +import java.io.Serializable; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.lang.reflect.Modifier; +import java.lang.reflect.Type; +import java.math.BigDecimal; +import java.net.MalformedURLException; +import java.net.URL; +import java.net.URLClassLoader; +import java.nio.channels.FileChannel; +import java.nio.channels.ReadableByteChannel; +import java.nio.file.StandardOpenOption; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.function.Function; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import javax.annotation.Nullable; +import org.apache.beam.sdk.io.FileSystems; +import org.apache.beam.sdk.schemas.AutoValueSchema; +import org.apache.beam.sdk.schemas.FieldValueTypeInformation; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.annotations.DefaultSchema; +import org.apache.beam.sdk.schemas.annotations.SchemaFieldDescription; +import org.apache.beam.sdk.schemas.utils.StaticSchemaInference; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.beam.sdk.values.TypeDescriptors; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Strings; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.io.ByteStreams; + +public class JavaRowUdf implements Serializable { + private final Configuration config; + private final Schema inputSchema; + private final Schema.FieldType outputType; + + // Transient so we don't have to worry about issues serializing these dynamically created classes. + // While this is lazily computed, it is always computed on class construction, so any errors + // should still be caught at construction time, and lazily re-computed before any use. + @SuppressFBWarnings("SE_TRANSIENT_FIELD_NOT_RESTORED") + private transient Function<Row, Object> function; + + // Find or implement the inverse of StaticSchemaInference.fieldFromType + @DefaultSchema(AutoValueSchema.class) + @AutoValue + public abstract static class Configuration implements Serializable { + @SchemaFieldDescription("Source code of a java expression in terms of the schema fields.") + @Nullable + public abstract String getExpression(); + + @SchemaFieldDescription( + "Source code of a public class implementing Function<Row, T> for some schema-compatible T.") + @Nullable + public abstract String getCallable(); + + @SchemaFieldDescription("Path to a jar file implementing the function referenced in name.") + @Nullable + public abstract String getPath(); + + @SchemaFieldDescription( + "Fully qualified name of either a class implementing Function<Row, T> (e.g. com.pkg.MyFunction), " + + "or a method taking a single Row argument (e.g. com.pkg.MyClass::methodName). " + + "If a method is passed, it must either be static or belong to a class with a public nullary constructor.") + @Nullable + public abstract String getName(); + + public void validate() { + checkArgument( + Strings.isNullOrEmpty(getPath()) || !Strings.isNullOrEmpty(getName()), + "Specifying a path only allows if a name is provided."); + int totalArgs = + (Strings.isNullOrEmpty(getExpression()) ? 0 : 1) + + (Strings.isNullOrEmpty(getCallable()) ? 0 : 1) + + (Strings.isNullOrEmpty(getName()) ? 0 : 1); + checkArgument( + totalArgs == 1, "Exactly one of expression, callable, or name must be provided."); + } + + public static Configuration.Builder builder() { + return new AutoValue_JavaRowUdf_Configuration.Builder(); + } + + @AutoValue.Builder + public abstract static class Builder { + public abstract Configuration.Builder setExpression(String expression); + + public abstract Configuration.Builder setCallable(String callable); + + public abstract Configuration.Builder setPath(String path); + + public abstract Configuration.Builder setName(String name); + + public abstract Configuration build(); + } + } + + public JavaRowUdf(Configuration config, Schema inputSchema) + throws MalformedURLException, ReflectiveOperationException, StringCompiler.CompileException { + this.config = config; + this.inputSchema = inputSchema; + FunctionAndType functionAndType = createFunction(config, inputSchema); + this.outputType = functionAndType.outputType; + this.function = functionAndType.function; + } + + public Schema.FieldType getOutputType() { + return outputType; + } + + public Function<Row, Object> getFunction() + throws MalformedURLException, ReflectiveOperationException, StringCompiler.CompileException { + if (function == null) { + FunctionAndType functionAndType = createFunction(config, inputSchema); + assert functionAndType.outputType.equals(outputType); + function = functionAndType.function; + } + return function; + } + + private static class FunctionAndType { + public final Schema.FieldType outputType; + public final Function<Row, Object> function; + + public FunctionAndType(Function<Row, Object> function) { + this(outputOf(function), function); + } + + public FunctionAndType(Type outputType, Function<Row, Object> function) { + this(TypeDescriptor.of(outputType), function); + } + + public FunctionAndType(TypeDescriptor<?> outputType, Function<Row, Object> function) { + this( + StaticSchemaInference.fieldFromType(outputType, new EmptyFieldValueTypeSupplier()), + function); + } + + public FunctionAndType(Schema.FieldType outputType, Function<Row, Object> function) { + this.outputType = outputType; + this.function = function; + } + + public static <InputT, OutputT> TypeDescriptor<OutputT> outputOf(Function<InputT, OutputT> fn) { + return TypeDescriptors.extractFromTypeParameters( + fn, + Function.class, + new TypeDescriptors.TypeVariableExtractor<Function<InputT, OutputT>, OutputT>() {}); + } + } + + @SuppressWarnings({ + "nullness" // TODO(https://github.com/apache/beam/issues/20497) + }) + private static FunctionAndType createFunction(Configuration config, Schema inputSchema) + throws ReflectiveOperationException, StringCompiler.CompileException, MalformedURLException { + config.validate(); + if (!Strings.isNullOrEmpty(config.getExpression())) { + return createFunctionFromExpression(config.getExpression(), inputSchema); + } else if (!Strings.isNullOrEmpty(config.getCallable())) { + return createFuctionFromCallable(config.getCallable()); + } else if (!Strings.isNullOrEmpty(config.getName())) { + return createFunctionFromName(config.getName(), config.getPath()); + } else { + throw new UnsupportedOperationException(config.toString()); + } + } + + private static FunctionAndType createFunctionFromExpression(String expression, Schema inputSchema) + throws StringCompiler.CompileException, ReflectiveOperationException { + if (inputSchema.hasField(expression)) { + final int ix = inputSchema.indexOf(expression); + return new FunctionAndType( + inputSchema.getField(expression).getType(), (Row row) -> row.getValue(ix)); + } else { + Map<String, Type> fieldTypes = new HashMap<>(); + for (Schema.Field field : inputSchema.getFields()) { + if (expression.indexOf(field.getName()) != -1) { + fieldTypes.put(field.getName(), typeFromFieldType(field.getType())); + } + } + Type type = StringCompiler.guessExpressionType(expression, fieldTypes); + StringBuilder source = new StringBuilder(); + source.append("import java.util.function.Function;\n"); + source.append("import " + Row.class.getTypeName() + ";\n"); + source.append("public class Eval implements Function<Row, Object> {\n"); + source.append(" public Object apply(Row __row__) {\n"); + for (Map.Entry<String, Type> fieldEntry : fieldTypes.entrySet()) { + source.append( + String.format( + " %s %s = (%s) __row__.getValue(%s);%n", + fieldEntry.getValue().getTypeName(), + fieldEntry.getKey(), + fieldEntry.getValue().getTypeName(), + inputSchema.indexOf(fieldEntry.getKey()))); + } + source.append(" return " + expression + ";\n"); + source.append(" }\n"); + source.append("}\n"); + return new FunctionAndType( + type, (Function<Row, Object>) StringCompiler.getInstance("Eval", source.toString())); + } + } + + @SuppressWarnings({ + "nullness" // TODO(https://github.com/apache/beam/issues/20497) + }) + private static FunctionAndType createFuctionFromCallable(String callable) + throws StringCompiler.CompileException, ReflectiveOperationException { + Matcher matcher = + Pattern.compile("\\bpublic\\s+class\\s+(\\S+)", Pattern.MULTILINE).matcher(callable); + Preconditions.checkArgument(matcher.find(), "No public class defined in callable source."); + return new FunctionAndType( + (Function<Row, Object>) StringCompiler.getInstance(matcher.group(1), callable.toString())); + } + + @SuppressWarnings({ + "nullness" // TODO(https://github.com/apache/beam/issues/20497) + }) + private static FunctionAndType createFunctionFromName(String name, String path) + throws ReflectiveOperationException, MalformedURLException { + if (path != null && !new File(path).exists()) { + try (ReadableByteChannel inChannel = + FileSystems.open(FileSystems.matchNewResource(path, false))) { + File tmpJar = File.createTempFile("map-to-fields-" + name, ".jar"); + try (FileChannel outChannel = FileChannel.open(tmpJar.toPath(), StandardOpenOption.WRITE)) { + ByteStreams.copy(inChannel, outChannel); + } + path = tmpJar.getPath(); + } catch (IOException exn) { + throw new RuntimeException(exn); + } + } + ClassLoader classLoader = + path == null + ? ClassLoader.getSystemClassLoader() + : new URLClassLoader( + new URL[] {new URL("file://" + path)}, ClassLoader.getSystemClassLoader()); + String className, methodName = null; + if (name.indexOf("::") == -1) { + className = name; + methodName = null; + } else { + String[] parts = name.split("::", 2); + className = parts[0]; + methodName = parts[1]; + } + if (methodName == null) { + return new FunctionAndType( + (Function<Row, Object>) + classLoader.loadClass(className).getDeclaredConstructor().newInstance()); + } else { + Class<?> clazz = classLoader.loadClass(className); + Method method = clazz.getMethod(methodName, Row.class); + Object base = + Modifier.isStatic(method.getModifiers()) + ? null + : clazz.getDeclaredConstructor().newInstance(); + return new FunctionAndType( + method.getGenericReturnType(), + (Row row) -> { + try { + return method.invoke(base, row); + } catch (IllegalAccessException | InvocationTargetException exn) { + throw new RuntimeException(exn); + } + }); + } + } + + private static class EmptyFieldValueTypeSupplier + implements org.apache.beam.sdk.schemas.utils.FieldValueTypeSupplier { + @Override + public List<FieldValueTypeInformation> get(Class<?> clazz) { + return Collections.<FieldValueTypeInformation>emptyList(); + } + } + + private static final Map<Schema.TypeName, Type> NULLABLE_PRIMITIVES = + ImmutableMap.<Schema.TypeName, Type>builder() + .put(Schema.TypeName.BYTE, Byte.class) + .put(Schema.TypeName.INT16, Short.class) + .put(Schema.TypeName.INT32, Integer.class) + .put(Schema.TypeName.INT64, Long.class) + .put(Schema.TypeName.FLOAT, Float.class) + .put(Schema.TypeName.DOUBLE, Double.class) + .put(Schema.TypeName.BOOLEAN, Boolean.class) + .put(Schema.TypeName.DECIMAL, BigDecimal.class) + .build(); + + private static final Map<Schema.TypeName, Type> NON_NULLABLE_PRIMITIVES = + ImmutableMap.<Schema.TypeName, Type>builder() + .put(Schema.TypeName.BYTE, byte.class) + .put(Schema.TypeName.INT16, short.class) + .put(Schema.TypeName.INT32, int.class) + .put(Schema.TypeName.INT64, long.class) + .put(Schema.TypeName.FLOAT, float.class) + .put(Schema.TypeName.DOUBLE, double.class) + .put(Schema.TypeName.BOOLEAN, boolean.class) + .put(Schema.TypeName.DECIMAL, BigDecimal.class) + .build(); + + private static Type typeFromFieldType(Schema.FieldType fieldType) { + Map<Schema.TypeName, Type> primitivesMap = + fieldType.getNullable() ? NULLABLE_PRIMITIVES : NON_NULLABLE_PRIMITIVES; + if (primitivesMap.containsKey(fieldType.getTypeName())) { + return primitivesMap.get(fieldType.getTypeName()); + } else if (fieldType.getRowSchema() != null) { + return Row.class; + } else { + throw new UnsupportedOperationException(fieldType.toString()); + } + } +} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/StringCompiler.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/StringCompiler.java new file mode 100644 index 000000000000..04730dce80c0 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/StringCompiler.java @@ -0,0 +1,262 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.schemas.transforms.providers; + +import java.io.ByteArrayOutputStream; +import java.io.File; +import java.io.IOException; +import java.io.OutputStream; +import java.lang.reflect.Method; +import java.lang.reflect.Type; +import java.net.URI; +import java.net.URL; +import java.net.URLClassLoader; +import java.security.AccessController; +import java.security.PrivilegedAction; +import java.security.SecureClassLoader; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.function.Supplier; +import java.util.jar.Attributes; +import java.util.jar.Manifest; +import java.util.zip.ZipEntry; +import java.util.zip.ZipFile; +import javax.tools.Diagnostic; +import javax.tools.DiagnosticCollector; +import javax.tools.FileObject; +import javax.tools.ForwardingJavaFileManager; +import javax.tools.JavaCompiler; +import javax.tools.JavaFileObject; +import javax.tools.SimpleJavaFileObject; +import javax.tools.StandardJavaFileManager; +import javax.tools.ToolProvider; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Suppliers; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; + +public class StringCompiler { + @SuppressWarnings({ + "nullness" // TODO(https://github.com/apache/beam/issues/20497) + }) + private static final Supplier<String> classpathSupplier = + Suppliers.memoize( + () -> { + List<String> cp = new ArrayList<>(); + cp.add(System.getProperty("java.class.path")); + // Javac doesn't properly handle manifest classpath spec. + ClassLoader cl = StringCompiler.class.getClassLoader(); + if (cl == null) { + cl = ClassLoader.getSystemClassLoader(); + } + if (cl instanceof URLClassLoader) { + for (URL url : ((URLClassLoader) cl).getURLs()) { + File file = new File(url.getFile()); + if (file.exists() && !file.isDirectory()) { + try (ZipFile zipFile = new ZipFile(new File(url.getFile()))) { + ZipEntry manifestEntry = zipFile.getEntry("META-INF/MANIFEST.MF"); + if (manifestEntry != null) { + Manifest manifest = new Manifest(zipFile.getInputStream(manifestEntry)); + cp.add(manifest.getMainAttributes().getValue(Attributes.Name.CLASS_PATH)); + } + } catch (IOException exn) { + throw new RuntimeException(exn); + } + } + } + } + return String.join(System.getProperty("path.separator"), cp); + }); + + public static class CompileException extends Exception { + private final DiagnosticCollector<?> diagnostics; + + public CompileException(DiagnosticCollector<?> diagnostics) { + super(diagnostics.getDiagnostics().toString()); + this.diagnostics = diagnostics; + } + + public DiagnosticCollector<?> getDiagnostics() { + return diagnostics; + } + } + + // TODO(XXX): swap args? + public static <T> Class<T> getClass(String name, String source) + throws CompileException, ClassNotFoundException { + JavaCompiler compiler = ToolProvider.getSystemJavaCompiler(); + InMemoryFileManager fileManager = + new InMemoryFileManager(compiler.getStandardFileManager(null, null, null)); + DiagnosticCollector<JavaFileObject> diagnostics = new DiagnosticCollector<>(); + JavaCompiler.CompilationTask task = + compiler.getTask( + null, + fileManager, + diagnostics, + ImmutableList.of("-classpath", classpathSupplier.get()), + null, + Collections.singletonList(new InMemoryFileManager.InputJavaFileObject(name, source))); + boolean result = task.call(); + if (!result) { + throw new CompileException(diagnostics); + } else { + return (Class<T>) fileManager.getClassLoader().loadClass(name); + } + } + + public static Object getInstance(String name, String source) + throws CompileException, ReflectiveOperationException { + return getClass(name, source).getDeclaredConstructor().newInstance(); + } + + public static Type guessExpressionType(String expression, Map<String, Type> inputTypes) + throws StringCompiler.CompileException, ClassNotFoundException { + + String expectedError = "cannot be converted to __TypeGuesserHelper__.BadReturnType"; + + try { + StringCompiler.getClass( + "__TypeGuesserHelper__", typeGuesserSource(expression, inputTypes, "BadReturnType")); + // Must have returned null. + return Void.class; + } catch (StringCompiler.CompileException exn) { + // Use the error message to derive the actual type. + for (Diagnostic<?> d : exn.getDiagnostics().getDiagnostics()) { + String msg = d.getMessage(Locale.ROOT); + int expectedErrorIndex = msg.indexOf(expectedError); + if (expectedErrorIndex != -1) { + String typeSource = + msg.substring( + 1 + "incompatible types: ".length() + msg.lastIndexOf('\n', expectedErrorIndex), + expectedErrorIndex); + Class<?> clazz = + StringCompiler.getClass( + "__TypeGuesserHelper__", typeGuesserSource(expression, inputTypes, typeSource)); + for (Method method : clazz.getMethods()) { + if (method.getName().equals("method")) { + return method.getGenericReturnType(); + } + } + // We should never get here. + throw new RuntimeException("Unable to locate declared method."); + } + } + // Must have been some other error. + throw exn; + } + } + + private static String typeGuesserSource( + String expression, Map<String, Type> inputTypes, String returnType) { + StringBuilder source = new StringBuilder(); + source.append("class __TypeGuesserHelper__ {\n"); + source.append(" private static class BadReturnType { private BadReturnType() {} }\n"); + source.append(" public static " + returnType + " method(\n"); + boolean first = true; + for (Map.Entry<String, Type> arg : inputTypes.entrySet()) { + if (first) { + first = false; + } else { + source.append(", "); + } + source.append(arg.getValue().getTypeName() + " " + arg.getKey()); + } + source.append(" ) {\n"); + source.append(" return " + expression + ";\n"); + source.append(" }\n"); + source.append("}\n"); + return source.toString(); + } + + private static class InMemoryFileManager + extends ForwardingJavaFileManager<StandardJavaFileManager> { + + private Map<String, OutputJavaFileObject> outputFileObjects = new HashMap<>(); + + public InMemoryFileManager(StandardJavaFileManager standardManager) { + super(standardManager); + } + + @Override + public JavaFileObject getJavaFileForOutput( + Location location, String className, JavaFileObject.Kind kind, FileObject sibling) { + + OutputJavaFileObject classAsBytes = new OutputJavaFileObject(className, kind); + outputFileObjects.put(className, classAsBytes); + return classAsBytes; + } + + public ClassLoader getClassLoader() { + return AccessController.<ClassLoader>doPrivileged( + (PrivilegedAction<ClassLoader>) + () -> + new SecureClassLoader() { + @Override + protected Class<?> findClass(String name) throws ClassNotFoundException { + OutputJavaFileObject fileObject = outputFileObjects.get(name); + if (fileObject == null) { + throw new ClassNotFoundException(name); + } else { + byte[] classBytes = fileObject.getBytes(); + return defineClass(name, classBytes, 0, classBytes.length); + } + } + }); + } + + @Override + public ClassLoader getClassLoader(Location location) { + return getClassLoader(); + } + + private static class InputJavaFileObject extends SimpleJavaFileObject { + private String source; + + public InputJavaFileObject(String name, String source) { + super( + URI.create("input:///" + name.replace('.', '/') + Kind.SOURCE.extension), Kind.SOURCE); + this.source = source; + } + + @Override + public CharSequence getCharContent(boolean ignoreEncodingErrors) { + return source; + } + } + + private static class OutputJavaFileObject extends SimpleJavaFileObject { + + private ByteArrayOutputStream content = new ByteArrayOutputStream(); + + public OutputJavaFileObject(String name, Kind kind) { + super(URI.create("output:///" + name.replace('.', '/') + kind.extension), kind); + } + + public byte[] getBytes() { + return content.toByteArray(); + } + + @Override + public OutputStream openOutputStream() { + return content; + } + } + } +} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/package-info.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/package-info.java new file mode 100644 index 000000000000..6c5d1cb7c570 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/package-info.java @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +/** + * Defines transforms that work on PCollections with schemas.. + * + * <p>For further details, see the documentation for each class in this package. + */ +@DefaultAnnotation(NonNull.class) +package org.apache.beam.sdk.schemas.transforms.providers; + +import edu.umd.cs.findbugs.annotations.DefaultAnnotation; +import org.checkerframework.checker.nullness.qual.NonNull; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/providers/JavaMapToFieldsTransformProviderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/providers/JavaMapToFieldsTransformProviderTest.java new file mode 100644 index 000000000000..64fc48564ccf --- /dev/null +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/providers/JavaMapToFieldsTransformProviderTest.java @@ -0,0 +1,196 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.schemas.transforms.providers; + +import java.util.Collections; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.testing.NeedsRunner; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionRowTuple; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Tests for {@link JavaMapToFieldsTransformProvider}. */ +@RunWith(JUnit4.class) +public class JavaMapToFieldsTransformProviderTest { + @Rule public TestPipeline pipeline = TestPipeline.create(); + + @Test + @Category(NeedsRunner.class) + public void testRenameFields() { + Schema inputSchema = + Schema.of( + Schema.Field.of("a", Schema.FieldType.STRING), + Schema.Field.of("b", Schema.FieldType.INT32), + Schema.Field.of("c", Schema.FieldType.DOUBLE)); + + PCollection<Row> input = + pipeline + .apply( + Create.of( + Row.withSchema(inputSchema).addValues("foo", 2, 0.5).build(), + Row.withSchema(inputSchema).addValues("bar", 4, 0.25).build())) + .setRowSchema(inputSchema); + + PCollection<Row> renamed = + PCollectionRowTuple.of(JavaMapToFieldsTransformProvider.INPUT_ROWS_TAG, input) + .apply( + new JavaMapToFieldsTransformProvider() + .from( + JavaMapToFieldsTransformProvider.Configuration.builder() + .setFields( + ImmutableMap.of( + "newC", + JavaRowUdf.Configuration.builder().setExpression("c").build(), + "newA", + JavaRowUdf.Configuration.builder().setExpression("a").build())) + .build())) + .get(JavaMapToFieldsTransformProvider.OUTPUT_ROWS_TAG); + + Schema outputSchema = renamed.getSchema(); + + PAssert.that(renamed) + .containsInAnyOrder( + Row.withSchema(outputSchema) + .withFieldValue("newC", 0.5) + .withFieldValue("newA", "foo") + .build(), + Row.withSchema(outputSchema) + .withFieldValue("newC", 0.25) + .withFieldValue("newA", "bar") + .build()); + + pipeline.run(); + } + + @Test + @Category(NeedsRunner.class) + public void testAppendAndDropFields() { + Schema inputSchema = + Schema.of( + Schema.Field.of("a", Schema.FieldType.INT32), + Schema.Field.of("b", Schema.FieldType.DOUBLE)); + + PCollection<Row> input = + pipeline + .apply( + Create.of( + Row.withSchema(inputSchema).addValues(2, 0.5).build(), + Row.withSchema(inputSchema).addValues(4, 0.25).build())) + .setRowSchema(inputSchema); + + PCollection<Row> renamed = + PCollectionRowTuple.of(JavaMapToFieldsTransformProvider.INPUT_ROWS_TAG, input) + .apply( + new JavaMapToFieldsTransformProvider() + .from( + JavaMapToFieldsTransformProvider.Configuration.builder() + .setLanguage("java") + .setAppend(true) + .setDrop(Collections.singletonList("b")) + .setFields( + ImmutableMap.of( + "sum", + JavaRowUdf.Configuration.builder() + .setExpression("a+b") + .build())) + .build())) + .get(JavaMapToFieldsTransformProvider.OUTPUT_ROWS_TAG); + + Schema outputSchema = renamed.getSchema(); + + PAssert.that(renamed) + .containsInAnyOrder( + Row.withSchema(outputSchema).withFieldValue("a", 2).withFieldValue("sum", 2.5).build(), + Row.withSchema(outputSchema) + .withFieldValue("a", 4) + .withFieldValue("sum", 4.25) + .build()); + + pipeline.run(); + } + + @Test + @Category(NeedsRunner.class) + public void testErrorHandling() { + Schema inputSchema = Schema.of(Schema.Field.of("x", Schema.FieldType.INT32)); + + PCollection<Row> input = + pipeline + .apply( + Create.of( + Row.withSchema(inputSchema).addValues(4).build(), + Row.withSchema(inputSchema).addValues(-1).build())) + .setRowSchema(inputSchema); + + PCollectionRowTuple result = + PCollectionRowTuple.of(JavaMapToFieldsTransformProvider.INPUT_ROWS_TAG, input) + .apply( + new JavaMapToFieldsTransformProvider() + .from( + JavaMapToFieldsTransformProvider.Configuration.builder() + .setLanguage("java") + .setFields( + ImmutableMap.of( + "sqrt", + JavaRowUdf.Configuration.builder() + .setCallable( + "import java.util.function.Function;" + + "import org.apache.beam.sdk.values.Row;" + + "public class Sqrt implements Function<Row, Double> {" + + " public Double apply(Row row) {" + + " int x = row.getInt32(\"x\");" + + " if (x < 0) {" + + " throw new ArithmeticException(\"negative value\");" + + " } else {" + + " return Math.sqrt(x);" + + " }" + + " }" + + "}") + .build())) + .setErrorHandling( + JavaMapToFieldsTransformProvider.Configuration.ErrorHandling + .builder() + .setOutput("errors") + .build()) + .build())); + + PCollection<Row> sqrts = result.get(JavaMapToFieldsTransformProvider.OUTPUT_ROWS_TAG); + Schema outputSchema = sqrts.getSchema(); + PAssert.that(sqrts) + .containsInAnyOrder(Row.withSchema(outputSchema).withFieldValue("sqrt", 2.0).build()); + + PCollection<Row> errors = result.get("errors"); + Schema errorSchema = errors.getSchema(); + PAssert.that(errors) + .containsInAnyOrder( + Row.withSchema(errorSchema) + .withFieldValue("failed_row", Row.withSchema(inputSchema).addValues(-1).build()) + .withFieldValue("error_message", "negative value") + .build()); + pipeline.run(); + } +} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/providers/JavaRowUdfTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/providers/JavaRowUdfTest.java new file mode 100644 index 000000000000..78ee36e7ca54 --- /dev/null +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/providers/JavaRowUdfTest.java @@ -0,0 +1,158 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.schemas.transforms.providers; + +import static org.junit.Assert.assertEquals; + +import java.net.MalformedURLException; +import java.util.function.Function; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.values.Row; +import org.junit.Test; + +public class JavaRowUdfTest { + + public static final Schema TEST_SCHEMA = + Schema.of( + Schema.Field.of("anInt32", Schema.FieldType.INT32).withNullable(true), + Schema.Field.of("anInt64", Schema.FieldType.INT64).withNullable(true), + Schema.Field.of("aDouble", Schema.FieldType.DOUBLE).withNullable(true)); + + @Test + public void testExpressionUdf() + throws MalformedURLException, ReflectiveOperationException, StringCompiler.CompileException { + JavaRowUdf udf = + new JavaRowUdf( + JavaRowUdf.Configuration.builder().setExpression("anInt32 + anInt64").build(), + TEST_SCHEMA); + assertEquals(Schema.FieldType.INT64, udf.getOutputType()); + assertEquals( + 5L, + udf.getFunction() + .apply( + Row.withSchema(TEST_SCHEMA) + .withFieldValue("anInt32", 2) + .withFieldValue("anInt64", 3L) + .build())); + } + + @Test + public void testFieldNameExpressionUdf() + throws MalformedURLException, ReflectiveOperationException, StringCompiler.CompileException { + JavaRowUdf udf = + new JavaRowUdf( + JavaRowUdf.Configuration.builder().setExpression("anInt32").build(), TEST_SCHEMA); + assertEquals(Schema.FieldType.INT32.withNullable(true), udf.getOutputType()); + assertEquals( + 2, + udf.getFunction() + .apply( + Row.withSchema(TEST_SCHEMA) + .withFieldValue("anInt32", 2) + .withFieldValue("anInt64", 3L) + .build())); + } + + @Test + public void testCallableUdf() + throws MalformedURLException, ReflectiveOperationException, StringCompiler.CompileException { + JavaRowUdf udf = + new JavaRowUdf( + JavaRowUdf.Configuration.builder() + .setCallable( + String.join( + "\n", + "import org.apache.beam.sdk.values.Row;", + "import java.util.function.Function;", + "public class MyFunction implements Function<Row, Double> {", + " public Double apply(Row row) { return 1.0 / row.getDouble(\"aDouble\"); }", + "}")) + .build(), + TEST_SCHEMA); + assertEquals(Schema.FieldType.DOUBLE, udf.getOutputType()); + assertEquals( + 0.25, + udf.getFunction() + .apply(Row.withSchema(TEST_SCHEMA).withFieldValue("aDouble", 4.0).build())); + } + + public static class TestFunction implements Function<Row, Double> { + @Override + public Double apply(Row row) { + return 1.0 / row.getDouble("aDouble"); + } + } + + public static double staticTestMethod(Row row) { + return 1.0 / row.getDouble("aDouble"); + } + + public static class TestClassWithMethod { + public double testMethod(Row row) { + return 1.0 / row.getDouble("aDouble"); + } + } + + @Test + public void testNamedFunctionUdf() + throws MalformedURLException, ReflectiveOperationException, StringCompiler.CompileException { + JavaRowUdf udf = + new JavaRowUdf( + JavaRowUdf.Configuration.builder() + .setName(getClass().getTypeName() + "$TestFunction") + .build(), + TEST_SCHEMA); + assertEquals(Schema.FieldType.DOUBLE, udf.getOutputType()); + assertEquals( + 0.25, + udf.getFunction() + .apply(Row.withSchema(TEST_SCHEMA).withFieldValue("aDouble", 4.0).build())); + } + + @Test + public void testClassMethodUdf() + throws MalformedURLException, ReflectiveOperationException, StringCompiler.CompileException { + JavaRowUdf udf = + new JavaRowUdf( + JavaRowUdf.Configuration.builder() + .setName(getClass().getTypeName() + "$TestClassWithMethod::testMethod") + .build(), + TEST_SCHEMA); + assertEquals(Schema.FieldType.DOUBLE, udf.getOutputType()); + assertEquals( + 0.25, + udf.getFunction() + .apply(Row.withSchema(TEST_SCHEMA).withFieldValue("aDouble", 4.0).build())); + } + + @Test + public void testStaticMethodUdf() + throws MalformedURLException, ReflectiveOperationException, StringCompiler.CompileException { + JavaRowUdf udf = + new JavaRowUdf( + JavaRowUdf.Configuration.builder() + .setName(getClass().getTypeName() + "::staticTestMethod") + .build(), + TEST_SCHEMA); + assertEquals(Schema.FieldType.DOUBLE, udf.getOutputType()); + assertEquals( + 0.25, + udf.getFunction() + .apply(Row.withSchema(TEST_SCHEMA).withFieldValue("aDouble", 4.0).build())); + } +} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/providers/StringCompilerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/providers/StringCompilerTest.java new file mode 100644 index 000000000000..0c7bb4fa0538 --- /dev/null +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/providers/StringCompilerTest.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.schemas.transforms.providers; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.util.function.Function; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; +import org.junit.Test; + +public class StringCompilerTest { + + public static final String SQUARE_SOURCE = + "import java.util.function.Function;" + + "public class Square implements Function<Integer, Integer> {" + + " public Integer apply(Integer x) { return x * x; }" + + "}"; + + @Test + public void testGetClass() throws Exception { + Class<?> clazz = StringCompiler.getClass("Square", SQUARE_SOURCE); + assertTrue(Function.class.isAssignableFrom(clazz)); + assertEquals("Square", clazz.getSimpleName()); + } + + @Test + public void testGetInstance() throws Exception { + Function<Integer, Integer> square = + (Function<Integer, Integer>) StringCompiler.getInstance("Square", SQUARE_SOURCE); + assertEquals(4, (int) square.apply(2)); + } + + @Test + public void testGuessExpressionType() throws Exception { + assertEquals( + double.class, + StringCompiler.guessExpressionType( + "a+b", ImmutableMap.of("a", int.class, "b", double.class))); + assertEquals( + double.class, + StringCompiler.guessExpressionType( + "a > 0 ? a : b", ImmutableMap.of("a", int.class, "b", double.class))); + assertEquals( + double.class, + StringCompiler.guessExpressionType("a * Math.random()", ImmutableMap.of("a", int.class))); + assertEquals( + int.class, + StringCompiler.guessExpressionType("(int) a", ImmutableMap.of("a", double.class))); + assertEquals( + long.class, + StringCompiler.guessExpressionType( + "a.getInt64(\"foo\")+b", ImmutableMap.of("a", Row.class, "b", int.class))); + } +} diff --git a/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/ExpansionServiceSchemaTransformProviderTest.java b/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/ExpansionServiceSchemaTransformProviderTest.java index d7a665eabe0f..696fed0f8ff4 100644 --- a/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/ExpansionServiceSchemaTransformProviderTest.java +++ b/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/ExpansionServiceSchemaTransformProviderTest.java @@ -20,6 +20,7 @@ import static org.apache.beam.runners.core.construction.BeamUrns.getUrn; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertTrue; import com.google.auto.service.AutoService; import java.util.ArrayList; @@ -305,7 +306,7 @@ public void testSchemaTransformDiscovery() { ExpansionApi.DiscoverSchemaTransformRequest.newBuilder().build(); ExpansionApi.DiscoverSchemaTransformResponse response = expansionService.discover(discoverRequest); - assertEquals(2, response.getSchemaTransformConfigsCount()); + assertTrue(response.getSchemaTransformConfigsCount() >= 2); } private void verifyLeafTransforms(ExpansionApi.ExpansionResponse response, int count) { diff --git a/sdks/python/apache_beam/yaml/standard_providers.yaml b/sdks/python/apache_beam/yaml/standard_providers.yaml index cdb4036f98c2..01049569edec 100644 --- a/sdks/python/apache_beam/yaml/standard_providers.yaml +++ b/sdks/python/apache_beam/yaml/standard_providers.yaml @@ -24,3 +24,31 @@ version: BEAM_VERSION transforms: Sql: 'beam:external:java:sql:v1' + MapToFields-java: "beam:schematransform:org.apache.beam:yaml:map_to_fields-java:v1" + MapToFields-generic: "beam:schematransform:org.apache.beam:yaml:map_to_fields-java:v1" + +- type: renaming + transforms: + 'MapToFields-java': 'MapToFields-java' + 'MapToFields-generic': 'MapToFields-generic' + config: + mappings: + 'MapToFields-generic': + language: 'language' + append: 'append' + drop: 'drop' + fields: 'fields' + error_handling: 'errorHandling' + 'MapToFields-java': + language: 'language' + append: 'append' + drop: 'drop' + fields: 'fields' + error_handling: 'errorHandling' + underlying_provider: + type: beamJar + transforms: + MapToFields-java: "beam:schematransform:org.apache.beam:yaml:map_to_fields-java:v1" + MapToFields-generic: "beam:schematransform:org.apache.beam:yaml:map_to_fields-java:v1" + config: + gradle_target: 'sdks:java:extensions:sql:expansion-service:shadowJar' From f5c0d16c6800f766c8ccc75219b341c76cf87da1 Mon Sep 17 00:00:00 2001 From: Pranav Bhandari <bhandari.pranav22@gmail.com> Date: Tue, 31 Oct 2023 19:04:15 -0400 Subject: [PATCH 324/435] Update `it` module. (#29232) --- .../CassandraResourceManagerUtils.java | 2 +- .../cassandra/matchers/CassandraAsserts.java | 2 +- .../CassandraResourceManagerTest.java | 3 +- .../org/apache/beam/it/gcp/LoadTestBase.java | 10 +- .../it/gcp/artifacts/utils/JsonTestUtil.java | 149 +++++++++ .../gcp/dataflow/DefaultPipelineLauncher.java | 6 +- .../it/gcp/dataflow/DirectRunnerClient.java | 6 +- .../it/gcp/datagenerator/DataGenerator.java | 54 ++-- .../datastore/matchers/DatastoreAsserts.java | 3 +- .../beam/it/gcp/dlp/DlpResourceManager.java | 5 +- .../beam/it/gcp/kms/KMSResourceManager.java | 5 +- .../it/gcp/monitoring/MonitoringClient.java | 12 +- .../gcp/spanner/matchers/SpannerAsserts.java | 45 +++ .../beam/it/gcp/bigtable/BigTableIOLT.java | 18 +- .../beam/it/gcp/spanner/SpannerIOLT.java | 285 ++++++++++++++++++ .../beam/it/gcp/storage/FileBasedIOLT.java | 2 +- .../it/jdbc/AbstractJDBCResourceManager.java | 63 ++-- .../beam/it/jdbc/JDBCResourceManager.java | 11 +- .../beam/it/jdbc/MSSQLResourceManager.java | 7 +- .../beam/it/jdbc/MySQLResourceManager.java | 4 +- .../beam/it/jdbc/OracleResourceManager.java | 8 +- .../beam/it/jdbc/PostgresResourceManager.java | 20 +- .../beam/it/kafka/KafkaResourceManager.java | 11 +- .../it/mongodb/MongoDBResourceManager.java | 10 +- .../it/mongodb/matchers/MongoDBAsserts.java | 2 +- .../beam/it/neo4j/Neo4jResourceManager.java | 11 +- .../it/neo4j/conditions/Neo4jQueryCheck.java | 15 +- .../beam/it/splunk/SplunkResourceManager.java | 10 +- .../TestContainerResourceManager.java | 15 +- .../beam/it/truthmatchers/RecordsSubject.java | 2 +- 30 files changed, 638 insertions(+), 158 deletions(-) create mode 100644 it/google-cloud-platform/src/test/java/org/apache/beam/it/gcp/spanner/SpannerIOLT.java diff --git a/it/cassandra/src/main/java/org/apache/beam/it/cassandra/CassandraResourceManagerUtils.java b/it/cassandra/src/main/java/org/apache/beam/it/cassandra/CassandraResourceManagerUtils.java index ef617de518b1..f01800763787 100644 --- a/it/cassandra/src/main/java/org/apache/beam/it/cassandra/CassandraResourceManagerUtils.java +++ b/it/cassandra/src/main/java/org/apache/beam/it/cassandra/CassandraResourceManagerUtils.java @@ -30,7 +30,7 @@ final class CassandraResourceManagerUtils { Pattern.compile("[/\\\\. \"\0$]"); // i.e. [/\. "$] private static final String REPLACE_DATABASE_NAME_CHAR = "-"; private static final DateTimeFormatter TIME_FORMAT = - DateTimeFormatter.ofPattern("yyyyMMdd-HHmmss"); + DateTimeFormatter.ofPattern("yyyyMMdd-HHmmss-SSSSSS"); private CassandraResourceManagerUtils() {} diff --git a/it/cassandra/src/main/java/org/apache/beam/it/cassandra/matchers/CassandraAsserts.java b/it/cassandra/src/main/java/org/apache/beam/it/cassandra/matchers/CassandraAsserts.java index 6aecc6609cfb..61f730bf3579 100644 --- a/it/cassandra/src/main/java/org/apache/beam/it/cassandra/matchers/CassandraAsserts.java +++ b/it/cassandra/src/main/java/org/apache/beam/it/cassandra/matchers/CassandraAsserts.java @@ -31,7 +31,7 @@ public class CassandraAsserts { /** - * Convert Cassandra {@link com.datastax.oss.driver.api.core.cql.Row} list to a list of maps. + * Convert Cassandra {@link Row} list to a list of maps. * * @param rows Rows to parse. * @return List of maps to use in {@link RecordsSubject}. diff --git a/it/cassandra/src/test/java/org/apache/beam/it/cassandra/CassandraResourceManagerTest.java b/it/cassandra/src/test/java/org/apache/beam/it/cassandra/CassandraResourceManagerTest.java index fe00457159fa..318ef6d76c68 100644 --- a/it/cassandra/src/test/java/org/apache/beam/it/cassandra/CassandraResourceManagerTest.java +++ b/it/cassandra/src/test/java/org/apache/beam/it/cassandra/CassandraResourceManagerTest.java @@ -72,7 +72,8 @@ public void testGetUriShouldReturnCorrectValue() { @Test public void testGetKeyspaceNameShouldReturnCorrectValue() { - assertThat(testManager.getKeyspaceName()).matches(TEST_ID.replace('-', '_') + "_\\d{8}_\\d{6}"); + assertThat(testManager.getKeyspaceName()) + .matches(TEST_ID.replace('-', '_') + "_\\d{8}_\\d{6}_\\d{6}"); } @Test diff --git a/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/LoadTestBase.java b/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/LoadTestBase.java index 14bb05394de2..44a439b0ce91 100644 --- a/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/LoadTestBase.java +++ b/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/LoadTestBase.java @@ -516,20 +516,20 @@ public abstract static class MetricsConfiguration { public abstract @Nullable String outputPCollectionV2(); - public static MetricsConfiguration.Builder builder() { + public static Builder builder() { return new AutoValue_LoadTestBase_MetricsConfiguration.Builder(); } @AutoValue.Builder public abstract static class Builder { - public abstract MetricsConfiguration.Builder setInputPCollection(@Nullable String value); + public abstract Builder setInputPCollection(@Nullable String value); - public abstract MetricsConfiguration.Builder setInputPCollectionV2(@Nullable String value); + public abstract Builder setInputPCollectionV2(@Nullable String value); - public abstract MetricsConfiguration.Builder setOutputPCollection(@Nullable String value); + public abstract Builder setOutputPCollection(@Nullable String value); - public abstract MetricsConfiguration.Builder setOutputPCollectionV2(@Nullable String value); + public abstract Builder setOutputPCollectionV2(@Nullable String value); public abstract MetricsConfiguration build(); } diff --git a/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/artifacts/utils/JsonTestUtil.java b/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/artifacts/utils/JsonTestUtil.java index 1ef12d33fa11..9a83558f7bfc 100644 --- a/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/artifacts/utils/JsonTestUtil.java +++ b/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/artifacts/utils/JsonTestUtil.java @@ -18,13 +18,22 @@ package org.apache.beam.it.gcp.artifacts.utils; import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.MappingIterator; +import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.json.JsonMapper; +import com.fasterxml.jackson.databind.node.ArrayNode; +import java.io.BufferedReader; +import java.io.ByteArrayInputStream; import java.io.IOException; +import java.io.InputStreamReader; import java.nio.charset.StandardCharsets; import java.util.ArrayList; +import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.TreeMap; +import java.util.stream.Collectors; /** * The {@link JsonTestUtil} class provides common utilities used for executing tests that involve @@ -56,6 +65,67 @@ public static List<Map<String, Object>> readRecords(byte[] contents) throws IOEx return records; } + /** + * Reads NDJSON (Newline Delimited JSON) data from a byte array and returns a list of parsed JSON + * objects. Each JSON object is represented as a Map of String keys to Object values. + * + * @param jsonBytes A byte array containing NDJSON data. + * @return A list of parsed JSON objects as {@code Map<String, Object>}. + * @throws IOException if there's an issue reading or parsing the data. + */ + public static List<Map<String, Object>> readNDJSON(byte[] jsonBytes) throws IOException { + try (ByteArrayInputStream inputStream = new ByteArrayInputStream(jsonBytes)) { + InputStreamReader reader = new InputStreamReader(inputStream, StandardCharsets.UTF_8); + JsonMapper mapper = new JsonMapper(); + + return new BufferedReader(reader) + .lines() + .map( + line -> { + try { + // Deserialize each line as a Map<String, Object> + return mapper.readValue(line, mapTypeRef); + } catch (IOException e) { + throw new RuntimeException(e); + } + }) + .collect(Collectors.toList()); + } + } + + /** + * Recursively sorts the keys of a nested JSON represented as a Map. + * + * @param jsonMap A {@code Map<String, Object>} representing the nested JSON. + * @return A sorted {@code Map<String, Object>} where the keys are sorted in natural order. + */ + public static Map<String, Object> sortJsonMap(Map<String, Object> jsonMap) { + return jsonMap.entrySet().stream() + .collect( + Collectors.toMap( + Map.Entry::getKey, + entry -> { + Object value = entry.getValue(); + if (value instanceof Map) { + return sortJsonMap((Map<String, Object>) value); + } else if (value instanceof List) { + return ((List<Object>) value) + .stream() + .map( + item -> + item instanceof Map + ? sortJsonMap((Map<String, Object>) item) + : item) + .collect(Collectors.toList()); + } else { + return value; + } + }, + (a, b) -> a, // Merge function (not needed for a TreeMap) + TreeMap::new // Resulting map is a TreeMap + )); + } + /** * Read JSON records to a list of Maps. * @@ -86,4 +156,83 @@ public static Map<String, Object> readRecord(byte[] contents) throws IOException public static Map<String, Object> readRecord(String contents) throws IOException { return readRecord(contents.getBytes(StandardCharsets.UTF_8)); } + + /** + * Parses a JSON string and returns either a List of Maps or a Map, depending on whether the JSON + * represents an array or an object. + * + * @param jsonString The JSON string to parse. + * @return A List of Maps if the JSON is an array, or a Map if it's an object. + * @throws IOException If there's an error while parsing the JSON string. + */ + public static Object parseJsonString(String jsonString) throws IOException { + ObjectMapper objectMapper = new ObjectMapper(); + JsonNode jsonNode = objectMapper.readTree(jsonString); + if (jsonNode.isArray()) { + return parseJsonArray((ArrayNode) jsonNode); + } else if (jsonNode.isObject()) { + return parseJsonObject(jsonNode); + } else { + throw new IllegalArgumentException("Input is not a valid JSON object or array."); + } + } + + /** + * Parses a JSON array represented by an ArrayNode and returns a List of Maps. + * + * @param arrayNode The JSON array to parse. + * @return A List of Maps containing the parsed data. + */ + private static List<Object> parseJsonArray(ArrayNode arrayNode) { + List<Object> result = new ArrayList<>(); + for (JsonNode element : arrayNode) { + if (element.isObject()) { + result.add(parseJsonObject(element)); + } else { + result.add(parseSimpleNode(element)); + } + } + return result; + } + + /** + * Parses a JSON object represented by a JsonNode and returns a Map. + * + * @param objectNode The JSON object to parse. + * @return A Map containing the parsed data. + */ + private static Map<String, Object> parseJsonObject(JsonNode objectNode) { + Map<String, Object> result = new HashMap<>(); + objectNode + .fields() + .forEachRemaining( + entry -> { + String key = entry.getKey(); + JsonNode value = entry.getValue(); + if (value.isObject()) { + result.put(key, parseJsonObject(value)); + } else if (value.isArray()) { + result.put(key, parseJsonArray((ArrayNode) value)); + } else { + result.put(key, parseSimpleNode(value)); + } + }); + return result; + } + + /** Parse following value from JSON node: text, number, boolean, null. */ + @SuppressWarnings("nullness") + private static Object parseSimpleNode(JsonNode element) { + if (element.isTextual()) { + return element.asText(); + } else if (element.isNumber()) { + return element.numberValue(); + } else if (element.isBoolean()) { + return element.asBoolean(); + } else if (element.isNull()) { + return null; + } else { + throw new IllegalArgumentException("Element is not a valid JSON object or array."); + } + } } diff --git a/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/dataflow/DefaultPipelineLauncher.java b/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/dataflow/DefaultPipelineLauncher.java index 3d43618821aa..620d24d4e117 100644 --- a/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/dataflow/DefaultPipelineLauncher.java +++ b/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/dataflow/DefaultPipelineLauncher.java @@ -99,7 +99,7 @@ public class DefaultPipelineLauncher extends AbstractPipelineLauncher { .put(PipelineResult.State.UNRECOGNIZED, JobState.UNKNOWN) .build(); - private DefaultPipelineLauncher(DefaultPipelineLauncher.Builder builder) { + private DefaultPipelineLauncher(Builder builder) { super( new Dataflow( Utils.getDefaultTransport(), @@ -109,8 +109,8 @@ private DefaultPipelineLauncher(DefaultPipelineLauncher.Builder builder) { : new HttpCredentialsAdapter(builder.getCredentials()))); } - public static DefaultPipelineLauncher.Builder builder(Credentials credentials) { - return new DefaultPipelineLauncher.Builder(credentials); + public static Builder builder(Credentials credentials) { + return new Builder(credentials); } @Override diff --git a/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/dataflow/DirectRunnerClient.java b/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/dataflow/DirectRunnerClient.java index 57f8ad40c1b6..8017009ff378 100644 --- a/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/dataflow/DirectRunnerClient.java +++ b/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/dataflow/DirectRunnerClient.java @@ -53,8 +53,8 @@ public class DirectRunnerClient implements PipelineLauncher { this.mainClass = builder.getMainClass(); } - public static DirectRunnerClient.Builder builder(Class<?> mainClass) { - return new DirectRunnerClient.Builder(mainClass); + public static Builder builder(Class<?> mainClass) { + return new Builder(mainClass); } @Override @@ -172,7 +172,7 @@ public Class<?> getMainClass() { return mainClass; } - public DirectRunnerClient.Builder setCredentials(Credentials value) { + public Builder setCredentials(Credentials value) { credentials = value; return this; } diff --git a/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/datagenerator/DataGenerator.java b/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/datagenerator/DataGenerator.java index 832a75defd95..99016b5dd3a4 100644 --- a/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/datagenerator/DataGenerator.java +++ b/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/datagenerator/DataGenerator.java @@ -61,16 +61,14 @@ private DataGenerator(Builder builder) { .build(); } - public static DataGenerator.Builder builderWithSchemaLocation( - String testName, String schemaLocation) { - return new DataGenerator.Builder(testName + "-data-generator") + public static Builder builderWithSchemaLocation(String testName, String schemaLocation) { + return new Builder(testName + "-data-generator") .setSchemaLocation(schemaLocation) .setAutoscalingAlgorithm(AutoscalingAlgorithmType.THROUGHPUT_BASED); } - public static DataGenerator.Builder builderWithSchemaTemplate( - String testName, String schemaTemplate) { - return new DataGenerator.Builder(testName + "-data-generator") + public static Builder builderWithSchemaTemplate(String testName, String schemaTemplate) { + return new Builder(testName + "-data-generator") .setSchemaTemplate(schemaTemplate) .setAutoscalingAlgorithm(AutoscalingAlgorithmType.THROUGHPUT_BASED); } @@ -131,27 +129,27 @@ public Map<String, String> getParameters() { return parameters; } - public DataGenerator.Builder setSchemaTemplate(String value) { + public Builder setSchemaTemplate(String value) { parameters.put("schemaTemplate", value); return this; } - public DataGenerator.Builder setSchemaLocation(String value) { + public Builder setSchemaLocation(String value) { parameters.put("schemaLocation", value); return this; } - public DataGenerator.Builder setMessagesLimit(String value) { + public Builder setMessagesLimit(String value) { parameters.put(MESSAGES_LIMIT, value); return this; } - public DataGenerator.Builder setQPS(String value) { + public Builder setQPS(String value) { parameters.put("qps", value); return this; } - public DataGenerator.Builder setSinkType(String value) { + public Builder setSinkType(String value) { parameters.put("sinkType", value); return this; } @@ -166,87 +164,87 @@ public Builder setNumWorkers(String value) { return this; } - public DataGenerator.Builder setMaxNumWorkers(String value) { + public Builder setMaxNumWorkers(String value) { parameters.put("maxNumWorkers", value); return this; } - public DataGenerator.Builder setAutoscalingAlgorithm(AutoscalingAlgorithmType value) { + public Builder setAutoscalingAlgorithm(AutoscalingAlgorithmType value) { parameters.put("autoscalingAlgorithm", value.toString()); return this; } - public DataGenerator.Builder setOutputDirectory(String value) { + public Builder setOutputDirectory(String value) { parameters.put("outputDirectory", value); return this; } - public DataGenerator.Builder setOutputType(String value) { + public Builder setOutputType(String value) { parameters.put("outputType", value); return this; } - public DataGenerator.Builder setNumShards(String value) { + public Builder setNumShards(String value) { parameters.put("numShards", value); return this; } - public DataGenerator.Builder setAvroSchemaLocation(String value) { + public Builder setAvroSchemaLocation(String value) { parameters.put("avroSchemaLocation", value); return this; } - public DataGenerator.Builder setTopic(String value) { + public Builder setTopic(String value) { parameters.put("topic", value); return this; } - public DataGenerator.Builder setProjectId(String value) { + public Builder setProjectId(String value) { parameters.put("projectId", value); return this; } - public DataGenerator.Builder setSpannerInstanceName(String value) { + public Builder setSpannerInstanceName(String value) { parameters.put("spannerInstanceName", value); return this; } - public DataGenerator.Builder setSpannerDatabaseName(String value) { + public Builder setSpannerDatabaseName(String value) { parameters.put("spannerDatabaseName", value); return this; } - public DataGenerator.Builder setSpannerTableName(String value) { + public Builder setSpannerTableName(String value) { parameters.put("spannerTableName", value); return this; } - public DataGenerator.Builder setDriverClassName(String value) { + public Builder setDriverClassName(String value) { parameters.put("driverClassName", value); return this; } - public DataGenerator.Builder setConnectionUrl(String value) { + public Builder setConnectionUrl(String value) { parameters.put("connectionUrl", value); return this; } - public DataGenerator.Builder setUsername(String value) { + public Builder setUsername(String value) { parameters.put("username", value); return this; } - public DataGenerator.Builder setPassword(String value) { + public Builder setPassword(String value) { parameters.put("password", value); return this; } - public DataGenerator.Builder setConnectionProperties(String value) { + public Builder setConnectionProperties(String value) { parameters.put("connectionProperties", value); return this; } - public DataGenerator.Builder setStatement(String value) { + public Builder setStatement(String value) { parameters.put("statement", value); return this; } diff --git a/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/datastore/matchers/DatastoreAsserts.java b/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/datastore/matchers/DatastoreAsserts.java index 78fa7543150f..ef67a5a5c4fb 100644 --- a/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/datastore/matchers/DatastoreAsserts.java +++ b/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/datastore/matchers/DatastoreAsserts.java @@ -61,8 +61,7 @@ public static List<Map<String, Object>> datastoreResultsToRecords(Collection<Ent /** * Creates a {@link RecordsSubject} to assert information within a list of records. * - * @param results Records in Datastore {@link com.google.cloud.datastore.Entity} format to use in - * the comparison. + * @param results Records in Datastore {@link Entity} format to use in the comparison. * @return Truth subject to chain assertions. */ public static RecordsSubject assertThatDatastoreRecords(Collection<Entity> results) { diff --git a/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/dlp/DlpResourceManager.java b/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/dlp/DlpResourceManager.java index de818a1bbff1..f59794af3e1f 100644 --- a/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/dlp/DlpResourceManager.java +++ b/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/dlp/DlpResourceManager.java @@ -113,9 +113,8 @@ public void cleanupAll() { * @param project the GCP project ID * @return a new instance of Builder */ - public static DlpResourceManager.Builder builder( - String project, CredentialsProvider credentialsProvider) { - return new DlpResourceManager.Builder(project, credentialsProvider); + public static Builder builder(String project, CredentialsProvider credentialsProvider) { + return new Builder(project, credentialsProvider); } /** A builder class for creating instances of {@link DlpResourceManager}. */ diff --git a/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/kms/KMSResourceManager.java b/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/kms/KMSResourceManager.java index 2cad6d0b9fab..7e1a403c7352 100644 --- a/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/kms/KMSResourceManager.java +++ b/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/kms/KMSResourceManager.java @@ -72,9 +72,8 @@ private KMSResourceManager(Builder builder) { this.keyRing = null; } - public static KMSResourceManager.Builder builder( - String projectId, CredentialsProvider credentialsProvider) { - return new KMSResourceManager.Builder(projectId, credentialsProvider); + public static Builder builder(String projectId, CredentialsProvider credentialsProvider) { + return new Builder(projectId, credentialsProvider); } /** diff --git a/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/monitoring/MonitoringClient.java b/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/monitoring/MonitoringClient.java index 06591ea4fe0a..0fc5614a3630 100644 --- a/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/monitoring/MonitoringClient.java +++ b/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/monitoring/MonitoringClient.java @@ -150,8 +150,8 @@ public List<Long> listTimeSeriesAsLong(ListTimeSeriesRequest request) { Aggregation aggregation = Aggregation.newBuilder() .setAlignmentPeriod(Duration.newBuilder().setSeconds(60).build()) - .setPerSeriesAligner(Aggregation.Aligner.ALIGN_MEAN) - .setCrossSeriesReducer(Aggregation.Reducer.REDUCE_MEAN) + .setPerSeriesAligner(Aligner.ALIGN_MEAN) + .setCrossSeriesReducer(Reducer.REDUCE_MEAN) .addGroupByFields("resource.instance_id") .build(); ListTimeSeriesRequest request = @@ -188,7 +188,7 @@ public List<Long> listTimeSeriesAsLong(ListTimeSeriesRequest request) { Aggregation aggregation = Aggregation.newBuilder() .setAlignmentPeriod(Duration.newBuilder().setSeconds(60).build()) - .setPerSeriesAligner(Aggregation.Aligner.ALIGN_MEAN) + .setPerSeriesAligner(Aligner.ALIGN_MEAN) .setCrossSeriesReducer(Reducer.REDUCE_MAX) .build(); ListTimeSeriesRequest request = @@ -225,7 +225,7 @@ public List<Long> listTimeSeriesAsLong(ListTimeSeriesRequest request) { Aggregation aggregation = Aggregation.newBuilder() .setAlignmentPeriod(Duration.newBuilder().setSeconds(60).build()) - .setPerSeriesAligner(Aggregation.Aligner.ALIGN_MEAN) + .setPerSeriesAligner(Aligner.ALIGN_MEAN) .setCrossSeriesReducer(Reducer.REDUCE_MAX) .build(); ListTimeSeriesRequest request = @@ -269,7 +269,7 @@ public List<Long> listTimeSeriesAsLong(ListTimeSeriesRequest request) { Aggregation aggregation = Aggregation.newBuilder() .setAlignmentPeriod(Duration.newBuilder().setSeconds(60).build()) - .setPerSeriesAligner(Aggregation.Aligner.ALIGN_RATE) + .setPerSeriesAligner(Aligner.ALIGN_RATE) .build(); ListTimeSeriesRequest request = ListTimeSeriesRequest.newBuilder() @@ -312,7 +312,7 @@ public List<Long> listTimeSeriesAsLong(ListTimeSeriesRequest request) { Aggregation aggregation = Aggregation.newBuilder() .setAlignmentPeriod(Duration.newBuilder().setSeconds(60).build()) - .setPerSeriesAligner(Aggregation.Aligner.ALIGN_RATE) + .setPerSeriesAligner(Aligner.ALIGN_RATE) .build(); ListTimeSeriesRequest request = ListTimeSeriesRequest.newBuilder() diff --git a/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/spanner/matchers/SpannerAsserts.java b/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/spanner/matchers/SpannerAsserts.java index c9964d16f3b1..5a101e08d375 100644 --- a/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/spanner/matchers/SpannerAsserts.java +++ b/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/spanner/matchers/SpannerAsserts.java @@ -17,13 +17,17 @@ */ package org.apache.beam.it.gcp.spanner.matchers; +import static org.apache.beam.it.gcp.artifacts.utils.JsonTestUtil.parseJsonString; import static org.apache.beam.it.truthmatchers.PipelineAsserts.assertThatRecords; import com.google.cloud.spanner.Mutation; import com.google.cloud.spanner.Struct; import com.google.cloud.spanner.Type; +import com.google.cloud.spanner.Type.Code; import com.google.cloud.spanner.Value; +import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -81,6 +85,47 @@ public static List<Map<String, Object>> mutationsToRecords(List<Mutation> mutati } } + /** + * Convert a list of Spanner {@link Mutation} objects into a list of maps, extracting specified + * columns. + * + * @param mutations The list of mutations to process. + * @param columns The columns to extract. + * @return List of maps to use in {@link RecordsSubject} + */ + public static List<Map<String, Object>> mutationsToRecords( + List<Mutation> mutations, List<String> columns) { + try { + List<Map<String, Object>> records = new ArrayList<>(); + mutations.forEach( + entry -> { + records.add( + entry.asMap().entrySet().stream() + .filter((e) -> columns.contains(e.getKey())) + .collect( + Collectors.toMap( + Map.Entry::getKey, + (e) -> { + if (e.getValue().getType().getCode() == Code.ARRAY) { + return e.getValue().getAsStringList(); + } + if (Arrays.asList(Code.JSON, Code.PG_JSONB) + .contains(e.getValue().getType().getCode())) { + try { + return parseJsonString(e.getValue().getJson()); + } catch (IOException ex) { + throw new RuntimeException(ex); + } + } + return e.getValue().getAsString(); + }))); + }); + return records; + } catch (Exception e) { + throw new RuntimeException("Error converting TableResult to Records", e); + } + } + /** * Creates a {@link RecordsSubject} to assert information within a list of records. * diff --git a/it/google-cloud-platform/src/test/java/org/apache/beam/it/gcp/bigtable/BigTableIOLT.java b/it/google-cloud-platform/src/test/java/org/apache/beam/it/gcp/bigtable/BigTableIOLT.java index e232ed31cb5a..a6516863b8d7 100644 --- a/it/google-cloud-platform/src/test/java/org/apache/beam/it/gcp/bigtable/BigTableIOLT.java +++ b/it/google-cloud-platform/src/test/java/org/apache/beam/it/gcp/bigtable/BigTableIOLT.java @@ -59,7 +59,7 @@ * -DfailIfNoTests=false". * * <p>Example trigger command for specific test: "mvn test -pl it/google-cloud-platform -am \ - * -Dtest="BigTableIOLT#testWriteAndRead" -Dconfiguration=local -Dproject=[gcpProject] \ + * -Dtest="BigTableIOLT#testBigtableWriteAndRead" -Dconfiguration=local -Dproject=[gcpProject] \ * -DartifactBucket=[temp bucket] -DfailIfNoTests=false". */ public class BigTableIOLT extends IOLoadTestBase { @@ -67,7 +67,7 @@ public class BigTableIOLT extends IOLoadTestBase { private static final String COLUMN_FAMILY_NAME = "cf"; private static final long TABLE_MAX_AGE_MINUTES = 100L; - private static BigtableResourceManager resourceManager; + private BigtableResourceManager resourceManager; private static final String READ_ELEMENT_METRIC_NAME = "read_count"; private Configuration configuration; private String tableId; @@ -114,7 +114,7 @@ public void teardown() { /** Run integration test with configurations specified by TestProperties. */ @Test - public void testWriteAndRead() throws IOException { + public void testBigtableWriteAndRead() throws IOException { tableId = generateTableId(testName); resourceManager.createTable( @@ -205,7 +205,7 @@ private PipelineLauncher.LaunchInfo testRead() throws IOException { return pipelineLauncher.launch(project, region, options); } - /** Options for Bigquery IO load test. */ + /** Options for BigtableIO load test. */ @AutoValue abstract static class Configuration { abstract Long getNumRows(); @@ -227,18 +227,18 @@ static Configuration of(long numRows, int pipelineTimeout, String runner, int va @AutoValue.Builder abstract static class Builder { - abstract Configuration.Builder setNumRows(long numRows); + abstract Builder setNumRows(long numRows); - abstract Configuration.Builder setPipelineTimeout(int timeOutMinutes); + abstract Builder setPipelineTimeout(int timeOutMinutes); - abstract Configuration.Builder setRunner(String runner); + abstract Builder setRunner(String runner); - abstract Configuration.Builder setValueSizeBytes(int valueSizeBytes); + abstract Builder setValueSizeBytes(int valueSizeBytes); abstract Configuration build(); } - abstract Configuration.Builder toBuilder(); + abstract Builder toBuilder(); } /** Maps long number to the BigTable format record. */ diff --git a/it/google-cloud-platform/src/test/java/org/apache/beam/it/gcp/spanner/SpannerIOLT.java b/it/google-cloud-platform/src/test/java/org/apache/beam/it/gcp/spanner/SpannerIOLT.java new file mode 100644 index 000000000000..949b863be3df --- /dev/null +++ b/it/google-cloud-platform/src/test/java/org/apache/beam/it/gcp/spanner/SpannerIOLT.java @@ -0,0 +1,285 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.it.gcp.spanner; + +import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.cloud.ByteArray; +import com.google.cloud.spanner.Mutation; +import java.io.IOException; +import java.io.Serializable; +import java.text.ParseException; +import java.time.Duration; +import java.time.ZoneId; +import java.time.format.DateTimeFormatter; +import java.util.Map; +import java.util.Objects; +import java.util.Random; +import java.util.UUID; +import org.apache.beam.it.common.PipelineLauncher; +import org.apache.beam.it.common.PipelineOperator; +import org.apache.beam.it.common.TestProperties; +import org.apache.beam.it.common.utils.ResourceManagerUtils; +import org.apache.beam.it.gcp.IOLoadTestBase; +import org.apache.beam.sdk.io.GenerateSequence; +import org.apache.beam.sdk.io.gcp.spanner.SpannerIO; +import org.apache.beam.sdk.io.synthetic.SyntheticSourceOptions; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; +import org.junit.After; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; + +/** + * SpannerIO performance tests. + * + * <p>Example trigger command for all tests: "mvn test -pl it/google-cloud-platform -am + * -Dtest=SpannerIOLT \ -Dproject=[gcpProject] -DartifactBucket=[temp bucket] + * -DfailIfNoTests=false". + * + * <p>Example trigger command for specific test: "mvn test -pl it/google-cloud-platform -am \ + * -Dtest="SpannerIOLT#testSpannerWriteAndRead" -Dconfiguration=local -Dproject=[gcpProject] \ + * -DartifactBucket=[temp bucket] -DfailIfNoTests=false". + */ +public class SpannerIOLT extends IOLoadTestBase { + @Rule public TestPipeline writePipeline = TestPipeline.create(); + @Rule public TestPipeline readPipeline = TestPipeline.create(); + private String tableName; + private SpannerResourceManager resourceManager; + private Configuration configuration; + private static final String READ_ELEMENT_METRIC_NAME = "read_count"; + + @Before + public void setup() throws IOException { + // generate a random table name + tableName = + "io_spanner_" + + DateTimeFormatter.ofPattern("MMddHHmmssSSS") + .withZone(ZoneId.of("UTC")) + .format(java.time.Instant.now()) + + UUID.randomUUID().toString().replace("-", "").substring(0, 10); + + resourceManager = SpannerResourceManager.builder(testName, project, region).build(); + + // parse configuration + String testConfig = + TestProperties.getProperty("configuration", "local", TestProperties.Type.PROPERTY); + configuration = TEST_CONFIGS_PRESET.get(testConfig); + if (configuration == null) { + try { + configuration = Configuration.fromJsonString(testConfig, Configuration.class); + } catch (IOException e) { + throw new IllegalArgumentException( + String.format( + "Unknown test configuration: [%s]. Pass to a valid configuration json, or use" + + " config presets: %s", + testConfig, TEST_CONFIGS_PRESET.keySet())); + } + } + // prepare schema + String createTable = + createTableStatement( + tableName, configuration.numColumns, (int) configuration.valueSizeBytes); + // Create table + resourceManager.executeDdlStatement(createTable); + } + + @After + public void teardown() { + ResourceManagerUtils.cleanResources(resourceManager); + } + + private static final Map<String, Configuration> TEST_CONFIGS_PRESET; + + static { + try { + TEST_CONFIGS_PRESET = + ImmutableMap.of( + "local", + Configuration.fromJsonString( + "{\"numRecords\":1000,\"valueSizeBytes\":1000,\"pipelineTimeout\":2,\"runner\":\"DirectRunner\"}", + Configuration.class), // 1 MB + "medium", + Configuration.fromJsonString( + "{\"numRecords\":10000000,\"valueSizeBytes\":1000,\"pipelineTimeout\":20,\"runner\":\"DataflowRunner\"}", + Configuration.class), // 10 GB + "large", + Configuration.fromJsonString( + "{\"numRecords\":100000000,\"valueSizeBytes\":1000,\"pipelineTimeout\":80,\"runner\":\"DataflowRunner\"}", + Configuration.class) // 100 GB + ); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + @Test + public void testSpannerWriteAndRead() throws IOException { + PipelineLauncher.LaunchInfo writeInfo = testWrite(); + PipelineOperator.Result writeResult = + pipelineOperator.waitUntilDone( + createConfig(writeInfo, Duration.ofMinutes(configuration.pipelineTimeout))); + assertNotEquals(PipelineOperator.Result.LAUNCH_FAILED, writeResult); + + PipelineLauncher.LaunchInfo readInfo = testRead(); + PipelineOperator.Result result = + pipelineOperator.waitUntilDone( + createConfig(readInfo, Duration.ofMinutes(configuration.pipelineTimeout))); + assertNotEquals(PipelineOperator.Result.LAUNCH_FAILED, result); + assertEquals( + PipelineLauncher.JobState.DONE, + pipelineLauncher.getJobStatus(project, region, readInfo.jobId())); + double numRecords = + pipelineLauncher.getMetric( + project, + region, + readInfo.jobId(), + getBeamMetricsName(PipelineMetricsType.COUNTER, READ_ELEMENT_METRIC_NAME)); + assertEquals(configuration.numRecords, numRecords, 0.5); + + // export metrics + MetricsConfiguration metricsConfig = + MetricsConfiguration.builder() + .setInputPCollection("Map records.out0") + .setInputPCollectionV2("Map records/ParMultiDo(GenerateMutations).out0") + .setOutputPCollection("Counting element.out0") + .setOutputPCollectionV2("Counting element/ParMultiDo(Counting).out0") + .build(); + try { + exportMetricsToBigQuery(writeInfo, getMetrics(writeInfo, metricsConfig)); + exportMetricsToBigQuery(readInfo, getMetrics(readInfo, metricsConfig)); + } catch (ParseException | InterruptedException e) { + throw new RuntimeException(e); + } + } + + private PipelineLauncher.LaunchInfo testWrite() throws IOException { + SpannerIO.Write writeTransform = + SpannerIO.write() + .withProjectId(project) + .withInstanceId(resourceManager.getInstanceId()) + .withDatabaseId(resourceManager.getDatabaseId()); + + writePipeline + .apply(GenerateSequence.from(0).to(configuration.numRecords)) + .apply( + "Map records", + ParDo.of( + new GenerateMutations( + tableName, configuration.numColumns, (int) configuration.valueSizeBytes))) + .apply("Write to Spanner", writeTransform); + + PipelineLauncher.LaunchConfig options = + PipelineLauncher.LaunchConfig.builder("write-spanner") + .setSdk(PipelineLauncher.Sdk.JAVA) + .setPipeline(writePipeline) + .addParameter("runner", configuration.runner) + .build(); + + return pipelineLauncher.launch(project, region, options); + } + + private PipelineLauncher.LaunchInfo testRead() throws IOException { + SpannerIO.Read readTrabsfirn = + SpannerIO.read() + .withProjectId(project) + .withInstanceId(resourceManager.getInstanceId()) + .withDatabaseId(resourceManager.getDatabaseId()) + .withQuery(String.format("SELECT * FROM %s", tableName)); + + readPipeline + .apply("Read from Spanner", readTrabsfirn) + .apply("Counting element", ParDo.of(new CountingFn<>(READ_ELEMENT_METRIC_NAME))); + + PipelineLauncher.LaunchConfig options = + PipelineLauncher.LaunchConfig.builder("read-spanner") + .setSdk(PipelineLauncher.Sdk.JAVA) + .setPipeline(readPipeline) + .addParameter("runner", configuration.runner) + .build(); + + return pipelineLauncher.launch(project, region, options); + } + + /** Options for SpannerIO load test. */ + static class Configuration extends SyntheticSourceOptions { + + /** + * Number of columns (besides the primary key) of each record. The column size is equally + * distributed as valueSizeBytes/numColumns. + */ + @JsonProperty public int numColumns = 1; + + /** Pipeline timeout in minutes. Must be a positive value. */ + @JsonProperty public int pipelineTimeout = 20; + + /** Runner specified to run the pipeline. */ + @JsonProperty public String runner = "DirectRunner"; + } + + /** + * Generate a create table sql statement with 1 integer column (Id) and additional numBytesCol + * columns. + */ + static String createTableStatement(String tableId, int numBytesCol, int valueSizeBytes) { + int sizePerCol = valueSizeBytes / numBytesCol; + StringBuilder statement = new StringBuilder(); + statement.append(String.format("CREATE TABLE %s (Id INT64", tableId)); + for (int col = 0; col < numBytesCol; ++col) { + statement.append(String.format(",\n COL%d BYTES(%d)", col + 1, sizePerCol)); + } + statement.append(") PRIMARY KEY(Id)"); + return statement.toString(); + } + + /** Maps long number to the Spanner format record. */ + private static class GenerateMutations extends DoFn<Long, Mutation> implements Serializable { + private final String table; + private final int numBytesCol; + private final int sizePerCol; + + public GenerateMutations(String tableId, int numBytesCol, int valueSizeBytes) { + checkArgument(valueSizeBytes >= numBytesCol); + this.table = tableId; + this.numBytesCol = numBytesCol; + this.sizePerCol = valueSizeBytes / numBytesCol; + } + + @ProcessElement + public void processElement(ProcessContext c) { + Mutation.WriteBuilder builder = Mutation.newInsertOrUpdateBuilder(table); + Long key = Objects.requireNonNull(c.element()); + builder.set("Id").to(key); + Random random = new Random(key); + byte[] value = new byte[sizePerCol]; + for (int col = 0; col < numBytesCol; ++col) { + String name = String.format("COL%d", col + 1); + random.nextBytes(value); + builder.set(name).to(ByteArray.copyFrom(value)); + } + Mutation mutation = builder.build(); + c.output(mutation); + } + } +} diff --git a/it/google-cloud-platform/src/test/java/org/apache/beam/it/gcp/storage/FileBasedIOLT.java b/it/google-cloud-platform/src/test/java/org/apache/beam/it/gcp/storage/FileBasedIOLT.java index 704f8337c66f..a36f3b340e83 100644 --- a/it/google-cloud-platform/src/test/java/org/apache/beam/it/gcp/storage/FileBasedIOLT.java +++ b/it/google-cloud-platform/src/test/java/org/apache/beam/it/gcp/storage/FileBasedIOLT.java @@ -90,7 +90,7 @@ public class FileBasedIOLT extends IOLoadTestBase { @Rule public TestPipeline readPipeline = TestPipeline.create(); - private static final Map<String, FileBasedIOLT.Configuration> TEST_CONFIGS_PRESET; + private static final Map<String, Configuration> TEST_CONFIGS_PRESET; static { try { diff --git a/it/jdbc/src/main/java/org/apache/beam/it/jdbc/AbstractJDBCResourceManager.java b/it/jdbc/src/main/java/org/apache/beam/it/jdbc/AbstractJDBCResourceManager.java index b57185b70ebc..6d50dddb0ccd 100644 --- a/it/jdbc/src/main/java/org/apache/beam/it/jdbc/AbstractJDBCResourceManager.java +++ b/it/jdbc/src/main/java/org/apache/beam/it/jdbc/AbstractJDBCResourceManager.java @@ -196,7 +196,8 @@ public boolean write(String tableName, List<Map<String, Object>> rows) valueList.add(null); } else if (NumberUtils.isCreatable(value.toString()) || "true".equalsIgnoreCase(value.toString()) - || "false".equalsIgnoreCase(value.toString())) { + || "false".equalsIgnoreCase(value.toString()) + || value.toString().startsWith("ARRAY[")) { valueList.add(String.valueOf(value)); } else { valueList.add("'" + value + "'"); @@ -226,34 +227,9 @@ public boolean write(String tableName, List<Map<String, Object>> rows) @SuppressWarnings("nullness") public List<Map<String, Object>> readTable(String tableName) { LOG.info("Reading all rows from {}.{}", databaseName, tableName); - - List<Map<String, Object>> resultSet = new ArrayList<>(); - - StringBuilder sql = new StringBuilder(); - try (Connection con = driver.getConnection(getUri(), username, password)) { - Statement stmt = con.createStatement(); - - sql.append("SELECT * FROM ").append(tableName); - ResultSet result = stmt.executeQuery(sql.toString()); - - while (result.next()) { - Map<String, Object> row = new HashMap<>(); - ResultSetMetaData metadata = result.getMetaData(); - // Columns list in table metadata is 1-indexed - for (int i = 1; i <= metadata.getColumnCount(); i++) { - row.put(metadata.getColumnName(i), result.getObject(i)); - } - resultSet.add(row); - } - result.close(); - stmt.close(); - } catch (Exception e) { - throw new JDBCResourceManagerException( - "Failed to fetch rows from table. SQL statement: " + sql, e); - } - + List<Map<String, Object>> result = runSQLQuery(String.format("SELECT * FROM %s", tableName)); LOG.info("Successfully loaded rows from {}.{}", databaseName, tableName); - return resultSet; + return result; } @Override @@ -290,9 +266,21 @@ protected String getFirstRow(String tableName) { } @Override - public synchronized ResultSet runSQLQuery(String sql) { + @SuppressWarnings("nullness") + public synchronized List<Map<String, Object>> runSQLQuery(String sql) { try (Statement stmt = driver.getConnection(getUri(), username, password).createStatement()) { - return stmt.executeQuery(sql); + List<Map<String, Object>> result = new ArrayList<>(); + ResultSet resultSet = stmt.executeQuery(sql); + while (resultSet.next()) { + Map<String, Object> row = new HashMap<>(); + ResultSetMetaData metadata = resultSet.getMetaData(); + // Columns list in table metadata is 1-indexed + for (int i = 1; i <= metadata.getColumnCount(); i++) { + row.put(metadata.getColumnName(i), resultSet.getObject(i)); + } + result.add(row); + } + return result; } catch (Exception e) { throw new JDBCResourceManagerException("Failed to execute SQL statement: " + sql, e); } @@ -307,6 +295,21 @@ public synchronized void runSQLUpdate(String sql) { } } + @Override + public synchronized long getRowCount(String tableName) { + try (Connection con = driver.getConnection(getUri(), username, password)) { + Statement stmt = con.createStatement(); + ResultSet resultSet = stmt.executeQuery(String.format("SELECT count(*) FROM %s", tableName)); + resultSet.next(); + long rows = resultSet.getLong(1); + resultSet.close(); + stmt.close(); + return rows; + } catch (Exception e) { + throw new JDBCResourceManagerException("Failed to get row count from " + tableName, e); + } + } + /** * Builder for {@link AbstractJDBCResourceManager}. * diff --git a/it/jdbc/src/main/java/org/apache/beam/it/jdbc/JDBCResourceManager.java b/it/jdbc/src/main/java/org/apache/beam/it/jdbc/JDBCResourceManager.java index 9292d4cb42ec..deb29ff3a5ec 100644 --- a/it/jdbc/src/main/java/org/apache/beam/it/jdbc/JDBCResourceManager.java +++ b/it/jdbc/src/main/java/org/apache/beam/it/jdbc/JDBCResourceManager.java @@ -19,7 +19,6 @@ import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; -import java.sql.ResultSet; import java.util.List; import java.util.Map; import org.apache.beam.it.common.ResourceManager; @@ -102,7 +101,7 @@ boolean write(String tableName, List<Map<String, Object>> rows) * @param sql The SQL query to run. * @return A ResultSet containing the result of the execution. */ - ResultSet runSQLQuery(String sql); + List<Map<String, Object>> runSQLQuery(String sql); /** * Run the given SQL DML statement (INSERT, UPDATE and DELETE). @@ -111,6 +110,14 @@ boolean write(String tableName, List<Map<String, Object>> rows) */ void runSQLUpdate(String sql); + /** + * Gets the number of rows in table. + * + * @param tableName The name of the table. + * @return a count of number of rows in the table. + */ + long getRowCount(String tableName); + /** Object for managing JDBC table schemas in {@link JDBCResourceManager} instances. */ class JDBCSchema { diff --git a/it/jdbc/src/main/java/org/apache/beam/it/jdbc/MSSQLResourceManager.java b/it/jdbc/src/main/java/org/apache/beam/it/jdbc/MSSQLResourceManager.java index c515b2c4844f..0bcb16c61095 100644 --- a/it/jdbc/src/main/java/org/apache/beam/it/jdbc/MSSQLResourceManager.java +++ b/it/jdbc/src/main/java/org/apache/beam/it/jdbc/MSSQLResourceManager.java @@ -61,14 +61,13 @@ private MSSQLResourceManager(Builder builder) { } @VisibleForTesting - <T extends MSSQLResourceManager.DefaultMSSQLServerContainer<T>> MSSQLResourceManager( - T container, Builder builder) { + <T extends DefaultMSSQLServerContainer<T>> MSSQLResourceManager(T container, Builder builder) { super(container, builder); initialized = true; } - public static MSSQLResourceManager.Builder builder(String testId) { - return new MSSQLResourceManager.Builder(testId); + public static Builder builder(String testId) { + return new Builder(testId); } private synchronized void createDatabase(String databaseName) { diff --git a/it/jdbc/src/main/java/org/apache/beam/it/jdbc/MySQLResourceManager.java b/it/jdbc/src/main/java/org/apache/beam/it/jdbc/MySQLResourceManager.java index 688c26dfb56d..e1bf3640b53d 100644 --- a/it/jdbc/src/main/java/org/apache/beam/it/jdbc/MySQLResourceManager.java +++ b/it/jdbc/src/main/java/org/apache/beam/it/jdbc/MySQLResourceManager.java @@ -49,8 +49,8 @@ private MySQLResourceManager(Builder builder) { super(container, builder); } - public static MySQLResourceManager.Builder builder(String testId) { - return new MySQLResourceManager.Builder(testId); + public static Builder builder(String testId) { + return new Builder(testId); } @Override diff --git a/it/jdbc/src/main/java/org/apache/beam/it/jdbc/OracleResourceManager.java b/it/jdbc/src/main/java/org/apache/beam/it/jdbc/OracleResourceManager.java index 8054d26c33f7..f44e939936d2 100644 --- a/it/jdbc/src/main/java/org/apache/beam/it/jdbc/OracleResourceManager.java +++ b/it/jdbc/src/main/java/org/apache/beam/it/jdbc/OracleResourceManager.java @@ -45,7 +45,7 @@ public class OracleResourceManager extends AbstractJDBCResourceManager<OracleCon private static final String DEFAULT_ORACLE_USERNAME = "testUser"; private static final String DEFAULT_ORACLE_PASSWORD = "testPassword"; - private OracleResourceManager(OracleResourceManager.Builder builder) { + private OracleResourceManager(Builder builder) { this( new OracleContainer( DockerImageName.parse(builder.containerImageName).withTag(builder.containerImageTag)), @@ -53,12 +53,12 @@ private OracleResourceManager(OracleResourceManager.Builder builder) { } @VisibleForTesting - OracleResourceManager(OracleContainer container, OracleResourceManager.Builder builder) { + OracleResourceManager(OracleContainer container, Builder builder) { super(container, builder); } - public static OracleResourceManager.Builder builder(String testId) { - return new OracleResourceManager.Builder(testId); + public static Builder builder(String testId) { + return new Builder(testId); } @Override diff --git a/it/jdbc/src/main/java/org/apache/beam/it/jdbc/PostgresResourceManager.java b/it/jdbc/src/main/java/org/apache/beam/it/jdbc/PostgresResourceManager.java index 7f054dfbc5db..26bdff2305b5 100644 --- a/it/jdbc/src/main/java/org/apache/beam/it/jdbc/PostgresResourceManager.java +++ b/it/jdbc/src/main/java/org/apache/beam/it/jdbc/PostgresResourceManager.java @@ -38,21 +38,13 @@ public class PostgresResourceManager extends AbstractJDBCResourceManager<Postgre // https://hub.docker.com/_/postgres/tags?tab=tags private static final String DEFAULT_POSTGRES_CONTAINER_TAG = "15.1"; - private PostgresResourceManager(PostgresResourceManager.Builder builder) { - this( - new PostgreSQLContainer<>( - DockerImageName.parse(builder.containerImageName).withTag(builder.containerImageTag)), - builder); - } - @VisibleForTesting - PostgresResourceManager( - PostgreSQLContainer<?> container, PostgresResourceManager.Builder builder) { + PostgresResourceManager(PostgreSQLContainer<?> container, Builder builder) { super(container, builder); } - public static PostgresResourceManager.Builder builder(String testId) { - return new PostgresResourceManager.Builder(testId); + public static Builder builder(String testId) { + return new Builder(testId); } @Override @@ -80,7 +72,11 @@ public Builder(String testId) { @Override public PostgresResourceManager build() { - return new PostgresResourceManager(this); + PostgreSQLContainer<?> container = + new PostgreSQLContainer<>( + DockerImageName.parse(containerImageName).withTag(containerImageTag)); + container.setCommand("postgres", "-c", "fsync=off", "-c", "max_connections=1000"); + return new PostgresResourceManager(container, this); } } } diff --git a/it/kafka/src/main/java/org/apache/beam/it/kafka/KafkaResourceManager.java b/it/kafka/src/main/java/org/apache/beam/it/kafka/KafkaResourceManager.java index 7f7fb5b69569..d9a647dbeebd 100644 --- a/it/kafka/src/main/java/org/apache/beam/it/kafka/KafkaResourceManager.java +++ b/it/kafka/src/main/java/org/apache/beam/it/kafka/KafkaResourceManager.java @@ -71,16 +71,13 @@ public class KafkaResourceManager extends TestContainerResourceManager<GenericCo private final String connectionString; private final boolean usingStaticTopic; - private KafkaResourceManager(KafkaResourceManager.Builder builder) { + private KafkaResourceManager(Builder builder) { this(null, new DefaultKafkaContainer(builder), builder); } @VisibleForTesting @SuppressWarnings("nullness") - KafkaResourceManager( - @Nullable AdminClient client, - KafkaContainer container, - KafkaResourceManager.Builder builder) { + KafkaResourceManager(@Nullable AdminClient client, KafkaContainer container, Builder builder) { super(container, builder); this.usingStaticTopic = builder.topicNames.size() > 0; @@ -105,8 +102,8 @@ private KafkaResourceManager(KafkaResourceManager.Builder builder) { : AdminClient.create(ImmutableMap.of("bootstrap.servers", this.connectionString)); } - public static KafkaResourceManager.Builder builder(String testId) { - return new KafkaResourceManager.Builder(testId); + public static Builder builder(String testId) { + return new Builder(testId); } /** Returns the kafka bootstrap server connection string. */ diff --git a/it/mongodb/src/main/java/org/apache/beam/it/mongodb/MongoDBResourceManager.java b/it/mongodb/src/main/java/org/apache/beam/it/mongodb/MongoDBResourceManager.java index 80216b14ac0e..ed0e556bf0df 100644 --- a/it/mongodb/src/main/java/org/apache/beam/it/mongodb/MongoDBResourceManager.java +++ b/it/mongodb/src/main/java/org/apache/beam/it/mongodb/MongoDBResourceManager.java @@ -69,7 +69,7 @@ public class MongoDBResourceManager extends TestContainerResourceManager<MongoDB private final String connectionString; private final boolean usingStaticDatabase; - private MongoDBResourceManager(MongoDBResourceManager.Builder builder) { + private MongoDBResourceManager(Builder builder) { this( /* mongoClient= */ null, new MongoDBContainer( @@ -80,9 +80,7 @@ private MongoDBResourceManager(MongoDBResourceManager.Builder builder) { @VisibleForTesting @SuppressWarnings("nullness") MongoDBResourceManager( - @Nullable MongoClient mongoClient, - MongoDBContainer container, - MongoDBResourceManager.Builder builder) { + @Nullable MongoClient mongoClient, MongoDBContainer container, Builder builder) { super(container, builder); this.usingStaticDatabase = builder.databaseName != null; @@ -93,8 +91,8 @@ private MongoDBResourceManager(MongoDBResourceManager.Builder builder) { this.mongoClient = mongoClient == null ? MongoClients.create(connectionString) : mongoClient; } - public static MongoDBResourceManager.Builder builder(String testId) { - return new MongoDBResourceManager.Builder(testId); + public static Builder builder(String testId) { + return new Builder(testId); } /** Returns the URI connection string to the MongoDB Database. */ diff --git a/it/mongodb/src/main/java/org/apache/beam/it/mongodb/matchers/MongoDBAsserts.java b/it/mongodb/src/main/java/org/apache/beam/it/mongodb/matchers/MongoDBAsserts.java index 1a1b86acf562..ec08854b5c61 100644 --- a/it/mongodb/src/main/java/org/apache/beam/it/mongodb/matchers/MongoDBAsserts.java +++ b/it/mongodb/src/main/java/org/apache/beam/it/mongodb/matchers/MongoDBAsserts.java @@ -31,7 +31,7 @@ public class MongoDBAsserts { /** - * Convert MongoDB {@link org.bson.Document} to a list of maps. + * Convert MongoDB {@link Document} to a list of maps. * * @param documents List of Documents to parse * @return List of maps to use in {@link RecordsSubject} diff --git a/it/neo4j/src/main/java/org/apache/beam/it/neo4j/Neo4jResourceManager.java b/it/neo4j/src/main/java/org/apache/beam/it/neo4j/Neo4jResourceManager.java index 97bcca9e84b9..835be71ce0f5 100644 --- a/it/neo4j/src/main/java/org/apache/beam/it/neo4j/Neo4jResourceManager.java +++ b/it/neo4j/src/main/java/org/apache/beam/it/neo4j/Neo4jResourceManager.java @@ -66,7 +66,7 @@ public class Neo4jResourceManager extends TestContainerResourceManager<Neo4jCont private final String adminPassword; - private Neo4jResourceManager(Neo4jResourceManager.Builder builder) { + private Neo4jResourceManager(Builder builder) { this( builder.driver, new Neo4jContainer<>( @@ -79,10 +79,7 @@ private Neo4jResourceManager(Neo4jResourceManager.Builder builder) { @VisibleForTesting @SuppressWarnings("nullness") - Neo4jResourceManager( - @Nullable Driver neo4jDriver, - Neo4jContainer<?> container, - Neo4jResourceManager.Builder builder) { + Neo4jResourceManager(@Nullable Driver neo4jDriver, Neo4jContainer<?> container, Builder builder) { super(container, builder); this.adminPassword = builder.adminPassword; @@ -101,8 +98,8 @@ private Neo4jResourceManager(Neo4jResourceManager.Builder builder) { } } - public static Neo4jResourceManager.Builder builder(String testId) { - return new Neo4jResourceManager.Builder(testId); + public static Builder builder(String testId) { + return new Builder(testId); } /** Returns the URI connection string to the Neo4j Database. */ diff --git a/it/neo4j/src/main/java/org/apache/beam/it/neo4j/conditions/Neo4jQueryCheck.java b/it/neo4j/src/main/java/org/apache/beam/it/neo4j/conditions/Neo4jQueryCheck.java index 32e283edb72c..16f251338425 100644 --- a/it/neo4j/src/main/java/org/apache/beam/it/neo4j/conditions/Neo4jQueryCheck.java +++ b/it/neo4j/src/main/java/org/apache/beam/it/neo4j/conditions/Neo4jQueryCheck.java @@ -20,6 +20,9 @@ import com.google.auto.value.AutoValue; import java.util.List; import java.util.Map; +import java.util.Set; +import java.util.TreeMap; +import java.util.stream.Collectors; import javax.annotation.Nullable; import org.apache.beam.it.conditions.ConditionCheck; import org.apache.beam.it.neo4j.Neo4jResourceManager; @@ -55,9 +58,17 @@ protected CheckResult check() { if (actualResult == null) { return new CheckResult(expectedResult == null); } + + Set<Map<String, Object>> sortedActualResult = sort(actualResult); + Set<Map<String, Object>> sortedExpectedResult = sort(expectedResult); + return new CheckResult( - actualResult.equals(expectedResult), - String.format("Expected %s to equal %s", actualResult, expectedResult)); + sortedActualResult.equals(sortedExpectedResult), + String.format("Expected %s to equal %s", sortedActualResult, sortedExpectedResult)); + } + + private static Set<Map<String, Object>> sort(List<Map<String, Object>> list) { + return list.stream().map(TreeMap::new).collect(Collectors.toSet()); } public static Builder builder(Neo4jResourceManager resourceManager) { diff --git a/it/splunk/src/main/java/org/apache/beam/it/splunk/SplunkResourceManager.java b/it/splunk/src/main/java/org/apache/beam/it/splunk/SplunkResourceManager.java index 1ef4726df43a..0115a791eefe 100644 --- a/it/splunk/src/main/java/org/apache/beam/it/splunk/SplunkResourceManager.java +++ b/it/splunk/src/main/java/org/apache/beam/it/splunk/SplunkResourceManager.java @@ -85,7 +85,7 @@ public class SplunkResourceManager extends TestContainerResourceManager<SplunkCo private final SplunkClientFactory clientFactory; @SuppressWarnings("resource") - private SplunkResourceManager(SplunkResourceManager.Builder builder) { + private SplunkResourceManager(Builder builder) { this( new SplunkClientFactory(), new SplunkContainer( @@ -98,9 +98,7 @@ private SplunkResourceManager(SplunkResourceManager.Builder builder) { @VisibleForTesting @SuppressWarnings("nullness") SplunkResourceManager( - SplunkClientFactory clientFactory, - SplunkContainer container, - SplunkResourceManager.Builder builder) { + SplunkClientFactory clientFactory, SplunkContainer container, Builder builder) { super(setup(container, builder), builder); String username = DEFAULT_SPLUNK_USERNAME; @@ -169,8 +167,8 @@ private static SplunkContainer setup(SplunkContainer container, Builder builder) .withPassword(builder.password); } - public static SplunkResourceManager.Builder builder(String testId) { - return new SplunkResourceManager.Builder(testId); + public static Builder builder(String testId) { + return new Builder(testId); } /** diff --git a/it/testcontainers/src/main/java/org/apache/beam/it/testcontainers/TestContainerResourceManager.java b/it/testcontainers/src/main/java/org/apache/beam/it/testcontainers/TestContainerResourceManager.java index 77dd6da58053..098938a291d0 100644 --- a/it/testcontainers/src/main/java/org/apache/beam/it/testcontainers/TestContainerResourceManager.java +++ b/it/testcontainers/src/main/java/org/apache/beam/it/testcontainers/TestContainerResourceManager.java @@ -32,11 +32,11 @@ * resources. * * <p>Optionally, a static resource can be specified by calling the useStaticContainer() method in - * the {@link TestContainerResourceManager.Builder} class. A static resource is a pre-configured - * database or other resource that is ready to be connected to by the resource manager. This could - * be a pre-existing TestContainer that has not been closed, a local database instance, a remote VM, - * or any other source that can be connected to. If a static container is used, the host and port - * must also be configured using the Builder's setHost() and setPort() methods, respectively. + * the {@link Builder} class. A static resource is a pre-configured database or other resource that + * is ready to be connected to by the resource manager. This could be a pre-existing TestContainer + * that has not been closed, a local database instance, a remote VM, or any other source that can be + * connected to. If a static container is used, the host and port must also be configured using the + * Builder's setHost() and setPort() methods, respectively. */ public abstract class TestContainerResourceManager<T extends GenericContainer<?>> implements ResourceManager { @@ -48,12 +48,11 @@ public abstract class TestContainerResourceManager<T extends GenericContainer<?> private final String host; protected int port; - protected <B extends TestContainerResourceManager.Builder<?>> TestContainerResourceManager( - T container, B builder) { + protected <B extends Builder<?>> TestContainerResourceManager(T container, B builder) { this(container, builder, null); } - protected <B extends TestContainerResourceManager.Builder<?>> TestContainerResourceManager( + protected <B extends Builder<?>> TestContainerResourceManager( T container, B builder, @Nullable Callable<Void> setup) { this.container = container; this.usingStaticContainer = builder.useStaticContainer; diff --git a/it/truthmatchers/src/main/java/org/apache/beam/it/truthmatchers/RecordsSubject.java b/it/truthmatchers/src/main/java/org/apache/beam/it/truthmatchers/RecordsSubject.java index 39a0c0cebedc..75d5ce3a67cd 100644 --- a/it/truthmatchers/src/main/java/org/apache/beam/it/truthmatchers/RecordsSubject.java +++ b/it/truthmatchers/src/main/java/org/apache/beam/it/truthmatchers/RecordsSubject.java @@ -81,7 +81,7 @@ public void hasRecordSubset(Map<String, Object> subset) { Map<String, Object> expected = convertMapToTreeMap(subset); for (Map<String, Object> candidate : actual) { boolean match = true; - for (Map.Entry<String, Object> entry : subset.entrySet()) { + for (Entry<String, Object> entry : subset.entrySet()) { if (!candidate.containsKey(entry.getKey()) || !candidate.get(entry.getKey()).equals(entry.getValue())) { match = false; From 33a527b1d4f183a005770f8a6c4b15fa27e45d44 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw <robertwb@gmail.com> Date: Tue, 31 Oct 2023 16:18:25 -0700 Subject: [PATCH 325/435] Declare undeclared libraries to work around BEAM-11761. --- sdks/java/extensions/schemaio-expansion-service/build.gradle | 3 +++ 1 file changed, 3 insertions(+) diff --git a/sdks/java/extensions/schemaio-expansion-service/build.gradle b/sdks/java/extensions/schemaio-expansion-service/build.gradle index ae6599dfbb99..68fb67e93e37 100644 --- a/sdks/java/extensions/schemaio-expansion-service/build.gradle +++ b/sdks/java/extensions/schemaio-expansion-service/build.gradle @@ -48,8 +48,11 @@ dependencies { implementation library.java.vendored_guava_32_1_2_jre // Stage default drivers for JdbcIO schema transforms implementation 'mysql:mysql-connector-java:8.0.22' + permitUnusedDeclared 'mysql:mysql-connector-java:8.0.22' // BEAM-11761 implementation 'com.oracle.database.jdbc:ojdbc8:23.2.0.0' + permitUnusedDeclared 'com.oracle.database.jdbc:ojdbc8:23.2.0.0' // BEAM-11761 implementation 'com.microsoft.sqlserver:mssql-jdbc:12.2.0.jre11' + permitUnusedDeclared 'com.microsoft.sqlserver:mssql-jdbc:12.2.0.jre11' // BEAM-11761 testImplementation library.java.junit testImplementation library.java.mockito_core } From d00303ab32673193a20eab37df6f57601e03125a Mon Sep 17 00:00:00 2001 From: martin trieu <martinkt@google.com> Date: Tue, 31 Oct 2023 17:39:04 -0700 Subject: [PATCH 326/435] add self to java reviewers to be able to merge and review streaming worker code (#29235) --- .github/REVIEWERS.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.github/REVIEWERS.yml b/.github/REVIEWERS.yml index 1c260fbea5eb..9fd61727f106 100644 --- a/.github/REVIEWERS.yml +++ b/.github/REVIEWERS.yml @@ -40,6 +40,7 @@ labels: - kennknowles - robertwb - bvolpato + - m-trieu exclusionList: [] - name: IO reviewers: From 13e803a640383b873925da3370b78b1429aa8c17 Mon Sep 17 00:00:00 2001 From: Jeffrey Kinard <jeff@thekinards.com> Date: Tue, 31 Oct 2023 22:32:54 -0400 Subject: [PATCH 327/435] [YAML] Remove redundant WithSchema transform Signed-off-by: Jeffrey Kinard <jeff@thekinards.com> --- sdks/python/apache_beam/yaml/yaml_provider.py | 42 ------------------- 1 file changed, 42 deletions(-) diff --git a/sdks/python/apache_beam/yaml/yaml_provider.py b/sdks/python/apache_beam/yaml/yaml_provider.py index 2d7471e506b0..bf0231ee89d4 100644 --- a/sdks/python/apache_beam/yaml/yaml_provider.py +++ b/sdks/python/apache_beam/yaml/yaml_provider.py @@ -28,7 +28,6 @@ import subprocess import sys import urllib.parse -import uuid from typing import Any from typing import Callable from typing import Dict @@ -535,46 +534,6 @@ def create(elements: Iterable[Any], reshuffle: bool = True): """ return beam.Create([element_to_rows(e) for e in elements], reshuffle) - def with_schema(**args): - # TODO: This is preliminary. - def parse_type(spec): - if spec in PRIMITIVE_NAMES_TO_ATOMIC_TYPE: - return schema_pb2.FieldType( - atomic_type=PRIMITIVE_NAMES_TO_ATOMIC_TYPE[spec]) - elif isinstance(spec, list): - if len(spec) != 1: - raise ValueError("Use single-element lists to denote list types.") - else: - return schema_pb2.FieldType( - iterable_type=schema_pb2.IterableType( - element_type=parse_type(spec[0]))) - elif isinstance(spec, dict): - return schema_pb2.FieldType( - iterable_type=schema_pb2.RowType(schema=parse_schema(spec[0]))) - else: - raise ValueError("Unknown schema type: {spec}") - - def parse_schema(spec): - return schema_pb2.Schema( - fields=[ - schema_pb2.Field(name=key, type=parse_type(value), id=ix) - for (ix, (key, value)) in enumerate(spec.items()) - ], - id=str(uuid.uuid4())) - - named_tuple = schemas.named_tuple_from_schema(parse_schema(args)) - names = list(args.keys()) - - def extract_field(x, name): - if isinstance(x, dict): - return x[name] - else: - return getattr(x, name) - - return 'WithSchema(%s)' % ', '.join(names) >> beam.Map( - lambda x: named_tuple(*[extract_field(x, name) for name in names]) - ).with_output_types(named_tuple) - # Or should this be posargs, args? # pylint: disable=dangerous-default-value def fully_qualified_named_transform( @@ -635,7 +594,6 @@ def log_and_return(x): 'Create': create, 'LogForTesting': lambda: beam.Map(log_and_return), 'PyTransform': fully_qualified_named_transform, - 'WithSchemaExperimental': with_schema, 'Flatten': Flatten, 'WindowInto': WindowInto, }, From 92a1a457efaab49c6bc7e3f2acee2b45737a0528 Mon Sep 17 00:00:00 2001 From: Danny McCormick <dannymccormick@google.com> Date: Wed, 1 Nov 2023 09:51:11 -0400 Subject: [PATCH 328/435] Remove cred rotation jobs from jenkins (#29243) --- ...job_IODatastoresCredentialsRotation.groovy | 56 ------------------- .../job_MetricsCredentialsRotation.groovy | 56 ------------------- 2 files changed, 112 deletions(-) delete mode 100644 .test-infra/jenkins/job_IODatastoresCredentialsRotation.groovy delete mode 100644 .test-infra/jenkins/job_MetricsCredentialsRotation.groovy diff --git a/.test-infra/jenkins/job_IODatastoresCredentialsRotation.groovy b/.test-infra/jenkins/job_IODatastoresCredentialsRotation.groovy deleted file mode 100644 index 58d18205b54a..000000000000 --- a/.test-infra/jenkins/job_IODatastoresCredentialsRotation.groovy +++ /dev/null @@ -1,56 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonJobProperties as commonJobProperties - -job('Rotate IO-Datastores Cluster Credentials') { - description('Rotates Certificates and performs an IP rotation for Metrics cluster') - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate) - - // Sets that this is a cron job. - commonJobProperties.setCronJob(delegate, 'H 2 1 * *')// At 00:02am every month. - def date = new Date().format('E MMM dd HH:mm:ss z yyyy') - - steps { - //Starting credential rotation - shell('''gcloud container clusters update io-datastores \ - --start-credential-rotation --zone=us-central1-a --quiet''') - - //Rebuilding the nodes - shell('''gcloud container clusters upgrade io-datastores \ - --node-pool=pool-1 --zone=us-central1-a --quiet''') - - //Completing the rotation - shell('''gcloud container clusters update io-datastores \ - --complete-credential-rotation --zone=us-central1-a --quiet''') - } - - publishers { - extendedEmail { - triggers { - failure { - subject('Credentials Rotation Failure on IO-Datastores cluster') - content("Something went wrong during the automatic credentials rotation for IO-Datastores Cluster, performed at ${date}. It may be necessary to check the state of the cluster certificates. For further details refer to the following links:\n * Failing job: https://ci-beam.apache.org/job/Rotate%20IO-Datastores%20Cluster%20Credentials/ \n * Job configuration: https://github.com/apache/beam/blob/master/.test-infra/jenkins/job_IODatastoresCredentialsRotation.groovy \n * Cluster URL: https://pantheon.corp.google.com/kubernetes/clusters/details/us-central1-a/io-datastores/details?mods=dataflow_dev&project=apache-beam-testing") - recipientList('dev@beam.apache.org') - } - } - } - } -} diff --git a/.test-infra/jenkins/job_MetricsCredentialsRotation.groovy b/.test-infra/jenkins/job_MetricsCredentialsRotation.groovy deleted file mode 100644 index 800899d0cd93..000000000000 --- a/.test-infra/jenkins/job_MetricsCredentialsRotation.groovy +++ /dev/null @@ -1,56 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonJobProperties as commonJobProperties - -job('Rotate Metrics Cluster Credentials') { - description('Rotates Certificates and performs an IP rotation for Metrics cluster') - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate) - - // Sets that this is a cron job. - commonJobProperties.setCronJob(delegate, 'H 2 1 * *')// At 00:02am every month. - def date = new Date().format('E MMM dd HH:mm:ss z yyyy') - - steps { - //Starting credential rotation - shell('''gcloud container clusters update metrics \ - --start-credential-rotation --zone=us-central1-a --quiet''') - - //Rebuilding the nodes - shell('''gcloud container clusters upgrade metrics \ - --node-pool=default-pool --zone=us-central1-a --quiet''') - - //Completing the rotation - shell('''gcloud container clusters update metrics \ - --complete-credential-rotation --zone=us-central1-a --quiet''') - } - - publishers { - extendedEmail { - triggers { - failure { - subject('Credentials Rotation Failure on Metrics cluster') - content("Something went wrong during the automatic credentials rotation for Metrics Cluster, performed at ${date}. It may be necessary to check the state of the cluster certificates. For further details refer to the following links:\n * Failing job: https://ci-beam.apache.org/job/Rotate%20Metrics%20Cluster%20Credentials/ \n * Job configuration: https://github.com/apache/beam/blob/master/.test-infra/jenkins/job_MetricsCredentialsRotation.groovy \n * Cluster URL: https://pantheon.corp.google.com/kubernetes/clusters/details/us-central1-a/metrics/details?mods=dataflow_dev&project=apache-beam-testing") - recipientList('dev@beam.apache.org') - } - } - } - } -} From 0d478daf760b4735bbb4d4718fc404a8e578d1fe Mon Sep 17 00:00:00 2001 From: Robert Bradshaw <robertwb@gmail.com> Date: Wed, 1 Nov 2023 07:30:43 -0700 Subject: [PATCH 329/435] [YAML] Java Filter and explode operations. (#29110) --- .../transforms/providers/ErrorHandling.java | 65 +++++ .../JavaExplodeTransformProvider.java | 225 ++++++++++++++++++ .../JavaFilterTransformProvider.java | 181 ++++++++++++++ .../JavaMapToFieldsTransformProvider.java | 47 +--- .../transforms/providers/JavaRowUdf.java | 4 + .../JavaExplodeTransformProviderTest.java | 104 ++++++++ .../JavaFilterTransformProviderTest.java | 119 +++++++++ .../JavaMapToFieldsTransformProviderTest.java | 6 +- .../apache_beam/yaml/standard_providers.yaml | 14 +- sdks/python/apache_beam/yaml/yaml_mapping.py | 5 +- 10 files changed, 725 insertions(+), 45 deletions(-) create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/ErrorHandling.java create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/JavaExplodeTransformProvider.java create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/JavaFilterTransformProvider.java create mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/providers/JavaExplodeTransformProviderTest.java create mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/providers/JavaFilterTransformProviderTest.java diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/ErrorHandling.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/ErrorHandling.java new file mode 100644 index 000000000000..3fab157a0e55 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/ErrorHandling.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.schemas.transforms.providers; + +import com.google.auto.value.AutoValue; +import javax.annotation.Nullable; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.annotations.SchemaFieldDescription; +import org.apache.beam.sdk.values.Row; + +@AutoValue +public abstract class ErrorHandling { + @SchemaFieldDescription("The name of the output PCollection containing failed writes.") + public abstract String getOutput(); + + public static Builder builder() { + return new AutoValue_ErrorHandling.Builder(); + } + + @AutoValue.Builder + public abstract static class Builder { + public abstract Builder setOutput(String output); + + public abstract ErrorHandling build(); + } + + public static boolean hasOutput(@Nullable ErrorHandling errorHandling) { + return getOutputOrNull(errorHandling) != null; + } + + public static @Nullable String getOutputOrNull(@Nullable ErrorHandling errorHandling) { + return errorHandling == null ? null : errorHandling.getOutput(); + } + + public static Schema errorSchema(Schema inputSchema) { + return Schema.of( + Schema.Field.of("failed_row", Schema.FieldType.row(inputSchema)), + Schema.Field.of("error_message", Schema.FieldType.STRING)); + } + + @SuppressWarnings({ + "nullness" // TODO(https://github.com/apache/beam/issues/20497) + }) + public static Row errorRecord(Schema errorSchema, Row inputRow, Throwable th) { + return Row.withSchema(errorSchema) + .withFieldValue("failed_row", inputRow) + .withFieldValue("error_message", th.getMessage()) + .build(); + } +} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/JavaExplodeTransformProvider.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/JavaExplodeTransformProvider.java new file mode 100644 index 000000000000..48ce5e33d9fa --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/JavaExplodeTransformProvider.java @@ -0,0 +1,225 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.schemas.transforms.providers; + +import com.google.auto.service.AutoService; +import com.google.auto.value.AutoValue; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import javax.annotation.Nullable; +import org.apache.beam.sdk.schemas.AutoValueSchema; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.annotations.DefaultSchema; +import org.apache.beam.sdk.schemas.transforms.SchemaTransform; +import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider; +import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionRowTuple; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.primitives.Booleans; + +/** + * An implementation of {@link TypedSchemaTransformProvider} for Explode. + * + * <p><b>Internal only:</b> This class is actively being worked on, and it will likely change. We + * provide no backwards compatibility guarantees, and it should not be implemented outside the Beam + * repository. + */ +@SuppressWarnings({ + "nullness" // TODO(https://github.com/apache/beam/issues/20497) +}) +@AutoService(SchemaTransformProvider.class) +public class JavaExplodeTransformProvider + extends TypedSchemaTransformProvider<JavaExplodeTransformProvider.Configuration> { + protected static final String INPUT_ROWS_TAG = "input"; + protected static final String OUTPUT_ROWS_TAG = "output"; + + @Override + protected Class<Configuration> configurationClass() { + return Configuration.class; + } + + @Override + protected SchemaTransform from(Configuration configuration) { + return new ExplodeTransform(configuration); + } + + @Override + public String identifier() { + return "beam:schematransform:org.apache.beam:yaml:explode:v1"; + } + + @Override + public List<String> inputCollectionNames() { + return Collections.singletonList(INPUT_ROWS_TAG); + } + + @Override + public List<String> outputCollectionNames() { + return Collections.singletonList(OUTPUT_ROWS_TAG); + } + + @DefaultSchema(AutoValueSchema.class) + @AutoValue + public abstract static class Configuration { + @Nullable + public abstract List<String> getFields(); + + @Nullable + public abstract Boolean getCrossProduct(); + + public static Builder builder() { + return new AutoValue_JavaExplodeTransformProvider_Configuration.Builder(); + } + + @AutoValue.Builder + public abstract static class Builder { + + public abstract Builder setFields(List<String> fields); + + public abstract Builder setCrossProduct(@Nullable Boolean append); + + public abstract Configuration build(); + } + } + + /** A {@link SchemaTransform} for Explode. */ + protected static class ExplodeTransform extends SchemaTransform { + + private final Configuration configuration; + + ExplodeTransform(Configuration configuration) { + this.configuration = configuration; + } + + @Override + public PCollectionRowTuple expand(PCollectionRowTuple input) { + Schema inputSchema = input.get(INPUT_ROWS_TAG).getSchema(); + Schema.Builder outputSchemaBuilder = new Schema.Builder(); + for (Schema.Field field : inputSchema.getFields()) { + if (configuration.getFields().contains(field.getName())) { + if (field.getType().getCollectionElementType() == null) { + throw new IllegalArgumentException( + String.format( + "Exploded field %s must be an iterable type, got %s.", + field.getName(), field.getType())); + } else { + outputSchemaBuilder = + outputSchemaBuilder.addField( + field.getName(), field.getType().getCollectionElementType()); + } + } else { + outputSchemaBuilder = outputSchemaBuilder.addField(field); + } + } + Schema outputSchema = outputSchemaBuilder.build(); + + PCollection<Row> result = + input + .get(INPUT_ROWS_TAG) + .apply( + "Explode", + ParDo.of( + createDoFn( + configuration.getFields(), + configuration.getCrossProduct(), + outputSchema))); + result.setRowSchema(outputSchema); + + return PCollectionRowTuple.of(OUTPUT_ROWS_TAG, result); + } + + private static DoFn<Row, Row> createDoFn( + List<String> fields, Boolean crossProductObj, Schema outputSchema) { + boolean crossProduct; + if (crossProductObj == null) { + if (fields.size() > 1) { + throw new IllegalArgumentException( + "boolean cross product parameter required to explode more than one field"); + } + crossProduct = false; + } else { + crossProduct = crossProductObj; + } + int numFields = outputSchema.getFields().size(); + boolean[] toExplode = + Booleans.toArray( + IntStream.range(0, numFields) + .mapToObj(index -> fields.contains(outputSchema.getField(index).getName())) + .collect(Collectors.toList())); + if (crossProduct) { + return new DoFn<Row, Row>() { + @ProcessElement + public void processElement(@Element Row inputRow, OutputReceiver<Row> out) { + emitCrossProduct(inputRow, 0, new Object[numFields], out); + } + + private void emitCrossProduct( + Row inputRow, int index, Object[] current, OutputReceiver<Row> out) { + if (index == numFields) { + out.output(Row.withSchema(outputSchema).attachValues(ImmutableList.copyOf(current))); + } else if (toExplode[index]) { + for (Object value : inputRow.getIterable(index)) { + current[index] = value; + emitCrossProduct(inputRow, index + 1, current, out); + } + } else { + current[index] = inputRow.getValue(index); + emitCrossProduct(inputRow, index + 1, current, out); + } + } + }; + } else { + return new DoFn<Row, Row>() { + @ProcessElement + public void processElement(@Element Row inputRow, OutputReceiver<Row> out) { + @SuppressWarnings("rawtypes") + Iterator[] iterators = new Iterator[numFields]; + for (int i = 0; i < numFields; i++) { + if (toExplode[i]) { + iterators[i] = inputRow.getIterable(i).iterator(); + } + } + while (IntStream.range(0, numFields) + .anyMatch(index -> toExplode[index] && iterators[index].hasNext())) { + Row.Builder builder = Row.withSchema(outputSchema); + for (int i = 0; i < numFields; i++) { + if (toExplode[i]) { + if (iterators[i].hasNext()) { + builder.addValue(iterators[i].next()); + } else { + builder.addValue(null); + } + } else { + builder.addValue(inputRow.getValue(i)); + } + } + out.output(builder.build()); + } + } + }; + } + } + } +} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/JavaFilterTransformProvider.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/JavaFilterTransformProvider.java new file mode 100644 index 000000000000..4ae8d2e41b30 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/JavaFilterTransformProvider.java @@ -0,0 +1,181 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.schemas.transforms.providers; + +import com.google.auto.service.AutoService; +import com.google.auto.value.AutoValue; +import java.net.MalformedURLException; +import java.util.Collections; +import java.util.List; +import javax.annotation.Nullable; +import org.apache.beam.sdk.schemas.AutoValueSchema; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.annotations.DefaultSchema; +import org.apache.beam.sdk.schemas.transforms.SchemaTransform; +import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider; +import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.PCollectionRowTuple; +import org.apache.beam.sdk.values.PCollectionTuple; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.TupleTagList; + +/** + * An implementation of {@link TypedSchemaTransformProvider} for Filter for the java language. + * + * <p><b>Internal only:</b> This class is actively being worked on, and it will likely change. We + * provide no backwards compatibility guarantees, and it should not be implemented outside the Beam + * repository. + */ +@SuppressWarnings({ + "nullness" // TODO(https://github.com/apache/beam/issues/20497) +}) +@AutoService(SchemaTransformProvider.class) +public class JavaFilterTransformProvider + extends TypedSchemaTransformProvider<JavaFilterTransformProvider.Configuration> { + protected static final String INPUT_ROWS_TAG = "input"; + protected static final String OUTPUT_ROWS_TAG = "output"; + + @Override + protected Class<Configuration> configurationClass() { + return Configuration.class; + } + + @Override + protected SchemaTransform from(Configuration configuration) { + return new JavaFilterTransform(configuration); + } + + @Override + public String identifier() { + return "beam:schematransform:org.apache.beam:yaml:filter-java:v1"; + } + + @Override + public List<String> inputCollectionNames() { + return Collections.singletonList(INPUT_ROWS_TAG); + } + + @Override + public List<String> outputCollectionNames() { + return Collections.singletonList(OUTPUT_ROWS_TAG); + } + + @DefaultSchema(AutoValueSchema.class) + @AutoValue + public abstract static class Configuration { + @Nullable + public abstract String getLanguage(); + + public abstract JavaRowUdf.Configuration getKeep(); + + @Nullable + public abstract ErrorHandling getErrorHandling(); + + public static Builder builder() { + return new AutoValue_JavaFilterTransformProvider_Configuration.Builder(); + } + + @AutoValue.Builder + public abstract static class Builder { + + public abstract Builder setLanguage(String language); + + public abstract Builder setKeep(JavaRowUdf.Configuration keep); + + public abstract Builder setErrorHandling(ErrorHandling errorHandling); + + public abstract Configuration build(); + } + } + + /** A {@link SchemaTransform} for Filter-java. */ + protected static class JavaFilterTransform extends SchemaTransform { + + private final Configuration configuration; + + JavaFilterTransform(Configuration configuration) { + this.configuration = configuration; + } + + @Override + public PCollectionRowTuple expand(PCollectionRowTuple input) { + Schema inputSchema = input.get(INPUT_ROWS_TAG).getSchema(); + JavaRowUdf keepFn; + try { + keepFn = new JavaRowUdf(this.configuration.getKeep(), inputSchema); + } catch (MalformedURLException + | ReflectiveOperationException + | StringCompiler.CompileException exn) { + throw new RuntimeException(exn); + } + if (!keepFn.getOutputType().withNullable(false).equals(Schema.FieldType.BOOLEAN)) { + throw new RuntimeException( + String.format( + "KeepFn %s must return a boolean, but returns %s instead.", + this.configuration.getKeep(), keepFn.getOutputType())); + } + boolean handleErrors = ErrorHandling.hasOutput(configuration.getErrorHandling()); + Schema errorSchema = ErrorHandling.errorSchema(inputSchema); + + PCollectionTuple pcolls = + input + .get(INPUT_ROWS_TAG) + .apply( + "Filter", + ParDo.of(createDoFn(keepFn, errorSchema, handleErrors)) + .withOutputTags(filteredValues, TupleTagList.of(errorValues))); + pcolls.get(filteredValues).setRowSchema(inputSchema); + pcolls.get(errorValues).setRowSchema(errorSchema); + + PCollectionRowTuple result = + PCollectionRowTuple.of(OUTPUT_ROWS_TAG, pcolls.get(filteredValues)); + if (handleErrors) { + result = result.and(configuration.getErrorHandling().getOutput(), pcolls.get(errorValues)); + } + return result; + } + + private static final TupleTag<Row> filteredValues = new TupleTag<Row>() {}; + private static final TupleTag<Row> errorValues = new TupleTag<Row>() {}; + + private static DoFn<Row, Row> createDoFn( + JavaRowUdf keepFn, Schema errorSchema, boolean handleErrors) { + return new DoFn<Row, Row>() { + @ProcessElement + public void processElement(@Element Row inputRow, MultiOutputReceiver out) { + boolean keep = false; + try { + keep = (boolean) keepFn.getFunction().apply(inputRow); + } catch (Exception exn) { + if (handleErrors) { + out.get(errorValues).output(ErrorHandling.errorRecord(errorSchema, inputRow, exn)); + } else { + throw new RuntimeException(exn); + } + } + if (keep) { + out.get(filteredValues).output(inputRow); + } + } + }; + } + } +} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/JavaMapToFieldsTransformProvider.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/JavaMapToFieldsTransformProvider.java index ddf892f03fc2..2e2042aef05d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/JavaMapToFieldsTransformProvider.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/JavaMapToFieldsTransformProvider.java @@ -28,7 +28,6 @@ import org.apache.beam.sdk.schemas.AutoValueSchema; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.annotations.DefaultSchema; -import org.apache.beam.sdk.schemas.annotations.SchemaFieldDescription; import org.apache.beam.sdk.schemas.transforms.SchemaTransform; import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider; import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider; @@ -68,7 +67,7 @@ protected SchemaTransform from(Configuration configuration) { @Override public String identifier() { - return String.format("beam:schematransform:org.apache.beam:yaml:map_to_fields-java:v1"); + return "beam:schematransform:org.apache.beam:yaml:map_to_fields-java:v1"; } @Override @@ -117,23 +116,6 @@ public abstract static class Builder { public abstract Configuration build(); } - - @AutoValue - public abstract static class ErrorHandling { - @SchemaFieldDescription("The name of the output PCollection containing failed writes.") - public abstract String getOutput(); - - public static Builder builder() { - return new AutoValue_JavaMapToFieldsTransformProvider_Configuration_ErrorHandling.Builder(); - } - - @AutoValue.Builder - public abstract static class Builder { - public abstract Builder setOutput(String output); - - public abstract ErrorHandling build(); - } - } } /** A {@link SchemaTransform} for MapToFields-java. */ @@ -192,13 +174,8 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) { } } Schema outputSchema = outputSchemaBuilder.build(); - boolean handleErrors = - configuration.getErrorHandling() != null - && configuration.getErrorHandling().getOutput() != null; - Schema errorSchema = - Schema.of( - Schema.Field.of("failed_row", Schema.FieldType.row(inputSchema)), - Schema.Field.of("error_message", Schema.FieldType.STRING)); + boolean handleErrors = ErrorHandling.hasOutput(configuration.getErrorHandling()); + Schema errorSchema = ErrorHandling.errorSchema(inputSchema); PCollectionTuple pcolls = input @@ -226,24 +203,24 @@ private static DoFn<Row, Row> createDoFn( return new DoFn<Row, Row>() { @ProcessElement public void processElement(@Element Row inputRow, MultiOutputReceiver out) { + Row outputRow; try { - Row.Builder outputRow = Row.withSchema(outputSchema); + Row.Builder builder = Row.withSchema(outputSchema); for (JavaRowUdf udf : udfs) { - outputRow.addValue(udf.getFunction().apply(inputRow)); + builder.addValue(udf.getFunction().apply(inputRow)); } - out.get(mappedValues).output(outputRow.build()); + outputRow = builder.build(); } catch (Exception exn) { if (handleErrors) { - out.get(errorValues) - .output( - Row.withSchema(errorSchema) - .withFieldValue("failed_row", inputRow) - .withFieldValue("error_message", exn.getMessage()) - .build()); + out.get(errorValues).output(ErrorHandling.errorRecord(errorSchema, inputRow, exn)); + outputRow = null; } else { throw new RuntimeException(exn); } } + if (outputRow != null) { + out.get(mappedValues).output(outputRow); + } } }; } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/JavaRowUdf.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/JavaRowUdf.java index e18df5b29a59..2ec0a9a60cd6 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/JavaRowUdf.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/JavaRowUdf.java @@ -313,6 +313,8 @@ public List<FieldValueTypeInformation> get(Class<?> clazz) { .put(Schema.TypeName.FLOAT, Float.class) .put(Schema.TypeName.DOUBLE, Double.class) .put(Schema.TypeName.BOOLEAN, Boolean.class) + .put(Schema.TypeName.BYTES, byte[].class) + .put(Schema.TypeName.STRING, String.class) .put(Schema.TypeName.DECIMAL, BigDecimal.class) .build(); @@ -325,6 +327,8 @@ public List<FieldValueTypeInformation> get(Class<?> clazz) { .put(Schema.TypeName.FLOAT, float.class) .put(Schema.TypeName.DOUBLE, double.class) .put(Schema.TypeName.BOOLEAN, boolean.class) + .put(Schema.TypeName.BYTES, byte[].class) + .put(Schema.TypeName.STRING, String.class) .put(Schema.TypeName.DECIMAL, BigDecimal.class) .build(); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/providers/JavaExplodeTransformProviderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/providers/JavaExplodeTransformProviderTest.java new file mode 100644 index 000000000000..7c8166d92b9a --- /dev/null +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/providers/JavaExplodeTransformProviderTest.java @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.schemas.transforms.providers; + +import java.util.List; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.testing.NeedsRunner; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionRowTuple; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +public class JavaExplodeTransformProviderTest { + @Rule public TestPipeline pipeline = TestPipeline.create(); + + private static final Schema INPUT_SCHEMA = + Schema.of( + Schema.Field.of("a", Schema.FieldType.iterable(Schema.FieldType.INT32)), + Schema.Field.of("b", Schema.FieldType.DOUBLE), + Schema.Field.of("c", Schema.FieldType.array(Schema.FieldType.STRING))); + + private static final Schema OUTPUT_SCHEMA = + Schema.of( + Schema.Field.of("a", Schema.FieldType.INT32), + Schema.Field.of("b", Schema.FieldType.DOUBLE), + Schema.Field.of("c", Schema.FieldType.STRING)); + + private static final List<Row> INPUT_ROWS = + ImmutableList.of( + Row.withSchema(INPUT_SCHEMA) + .addValues(ImmutableList.of(1, 2), 1.5, ImmutableList.of("x", "y")) + .build()); + + @Test + @Category(NeedsRunner.class) + public void testCrossProduct() { + PCollection<Row> input = pipeline.apply(Create.of(INPUT_ROWS)).setRowSchema(INPUT_SCHEMA); + + PCollection<Row> exploded = + PCollectionRowTuple.of(JavaExplodeTransformProvider.INPUT_ROWS_TAG, input) + .apply( + new JavaExplodeTransformProvider() + .from( + JavaExplodeTransformProvider.Configuration.builder() + .setFields(ImmutableList.of("a", "c")) + .setCrossProduct(true) + .build())) + .get(JavaExplodeTransformProvider.OUTPUT_ROWS_TAG); + + PAssert.that(exploded) + .containsInAnyOrder( + Row.withSchema(OUTPUT_SCHEMA).addValues(1, 1.5, "x").build(), + Row.withSchema(OUTPUT_SCHEMA).addValues(2, 1.5, "x").build(), + Row.withSchema(OUTPUT_SCHEMA).addValues(1, 1.5, "y").build(), + Row.withSchema(OUTPUT_SCHEMA).addValues(2, 1.5, "y").build()); + + pipeline.run(); + } + + @Test + @Category(NeedsRunner.class) + public void testZipProduct() { + PCollection<Row> input = pipeline.apply(Create.of(INPUT_ROWS)).setRowSchema(INPUT_SCHEMA); + + PCollection<Row> exploded = + PCollectionRowTuple.of(JavaExplodeTransformProvider.INPUT_ROWS_TAG, input) + .apply( + new JavaExplodeTransformProvider() + .from( + JavaExplodeTransformProvider.Configuration.builder() + .setFields(ImmutableList.of("a", "c")) + .setCrossProduct(false) + .build())) + .get(JavaExplodeTransformProvider.OUTPUT_ROWS_TAG); + + PAssert.that(exploded) + .containsInAnyOrder( + Row.withSchema(OUTPUT_SCHEMA).addValues(1, 1.5, "x").build(), + Row.withSchema(OUTPUT_SCHEMA).addValues(2, 1.5, "y").build()); + + pipeline.run(); + } +} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/providers/JavaFilterTransformProviderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/providers/JavaFilterTransformProviderTest.java new file mode 100644 index 000000000000..b269b12f1554 --- /dev/null +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/providers/JavaFilterTransformProviderTest.java @@ -0,0 +1,119 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.schemas.transforms.providers; + +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.testing.NeedsRunner; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionRowTuple; +import org.apache.beam.sdk.values.Row; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +public class JavaFilterTransformProviderTest { + @Rule public TestPipeline pipeline = TestPipeline.create(); + + @Test + @Category(NeedsRunner.class) + public void testFilter() { + Schema inputSchema = + Schema.of( + Schema.Field.of("a", Schema.FieldType.STRING), + Schema.Field.of("b", Schema.FieldType.INT32), + Schema.Field.of("c", Schema.FieldType.DOUBLE)); + + PCollection<Row> input = + pipeline + .apply( + Create.of( + Row.withSchema(inputSchema).addValues("foo", 2, 0.5).build(), + Row.withSchema(inputSchema).addValues("bar", 4, 0.25).build())) + .setRowSchema(inputSchema); + + PCollection<Row> renamed = + PCollectionRowTuple.of(JavaFilterTransformProvider.INPUT_ROWS_TAG, input) + .apply( + new JavaFilterTransformProvider() + .from( + JavaFilterTransformProvider.Configuration.builder() + .setKeep( + JavaRowUdf.Configuration.builder() + .setExpression("b + c > 3") + .build()) + .build())) + .get(JavaFilterTransformProvider.OUTPUT_ROWS_TAG); + + PAssert.that(renamed) + .containsInAnyOrder( + Row.withSchema(inputSchema) + .withFieldValue("a", "bar") + .withFieldValue("b", 4) + .withFieldValue("c", 0.25) + .build()); + + pipeline.run(); + } + + @Test + @Category(NeedsRunner.class) + public void testErrorHandling() { + Schema inputSchema = Schema.of(Schema.Field.of("s", Schema.FieldType.STRING)); + + PCollection<Row> input = + pipeline + .apply( + Create.of( + Row.withSchema(inputSchema).addValues("short").build(), + Row.withSchema(inputSchema).addValues("looooooooooooong").build())) + .setRowSchema(inputSchema); + + PCollectionRowTuple result = + PCollectionRowTuple.of(JavaFilterTransformProvider.INPUT_ROWS_TAG, input) + .apply( + new JavaFilterTransformProvider() + .from( + JavaFilterTransformProvider.Configuration.builder() + .setLanguage("java") + .setKeep( + JavaRowUdf.Configuration.builder() + .setExpression("s.charAt(7) == 'o'") + .build()) + .setErrorHandling(ErrorHandling.builder().setOutput("errors").build()) + .build())); + + PCollection<Row> good = result.get(JavaFilterTransformProvider.OUTPUT_ROWS_TAG); + PAssert.that(good) + .containsInAnyOrder( + Row.withSchema(inputSchema).withFieldValue("s", "looooooooooooong").build()); + + PCollection<Row> errors = result.get("errors"); + Schema errorSchema = errors.getSchema(); + PAssert.that(errors) + .containsInAnyOrder( + Row.withSchema(errorSchema) + .withFieldValue( + "failed_row", Row.withSchema(inputSchema).addValues("short").build()) + .withFieldValue("error_message", "String index out of range: 7") + .build()); + pipeline.run(); + } +} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/providers/JavaMapToFieldsTransformProviderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/providers/JavaMapToFieldsTransformProviderTest.java index 64fc48564ccf..6ad6f353a4dd 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/providers/JavaMapToFieldsTransformProviderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/providers/JavaMapToFieldsTransformProviderTest.java @@ -171,11 +171,7 @@ public void testErrorHandling() { + " }" + "}") .build())) - .setErrorHandling( - JavaMapToFieldsTransformProvider.Configuration.ErrorHandling - .builder() - .setOutput("errors") - .build()) + .setErrorHandling(ErrorHandling.builder().setOutput("errors").build()) .build())); PCollection<Row> sqrts = result.get(JavaMapToFieldsTransformProvider.OUTPUT_ROWS_TAG); diff --git a/sdks/python/apache_beam/yaml/standard_providers.yaml b/sdks/python/apache_beam/yaml/standard_providers.yaml index 01049569edec..c612d4412081 100644 --- a/sdks/python/apache_beam/yaml/standard_providers.yaml +++ b/sdks/python/apache_beam/yaml/standard_providers.yaml @@ -30,7 +30,9 @@ - type: renaming transforms: 'MapToFields-java': 'MapToFields-java' - 'MapToFields-generic': 'MapToFields-generic' + 'MapToFields-generic': 'MapToFields-java' + 'Filter-java': 'Filter-java' + 'Explode': 'Explode' config: mappings: 'MapToFields-generic': @@ -45,10 +47,18 @@ drop: 'drop' fields: 'fields' error_handling: 'errorHandling' + 'Filter-java': + language: 'language' + keep: 'keep' + error_handling: 'errorHandling' + 'Explode': + fields: 'fields' + cross_product: 'crossProduct' underlying_provider: type: beamJar transforms: MapToFields-java: "beam:schematransform:org.apache.beam:yaml:map_to_fields-java:v1" - MapToFields-generic: "beam:schematransform:org.apache.beam:yaml:map_to_fields-java:v1" + Filter-java: "beam:schematransform:org.apache.beam:yaml:filter-java:v1" + Explode: "beam:schematransform:org.apache.beam:yaml:explode:v1" config: gradle_target: 'sdks:java:extensions:sql:expansion-service:shadowJar' diff --git a/sdks/python/apache_beam/yaml/yaml_mapping.py b/sdks/python/apache_beam/yaml/yaml_mapping.py index 501c7a5c57be..3497660aedc3 100644 --- a/sdks/python/apache_beam/yaml/yaml_mapping.py +++ b/sdks/python/apache_beam/yaml/yaml_mapping.py @@ -239,8 +239,6 @@ def expand(pcoll, error_handling=None, **kwargs): return expand -# TODO(yaml): This should be available in all environments, in which case -# we choose the one that matches best. class _Explode(beam.PTransform): def __init__( self, @@ -289,11 +287,12 @@ def explode_zip(base, fields): copy[field] = values[ix] yield beam.Row(**copy) + cross_product = self._cross_product return ( pcoll | beam.FlatMap( lambda row: - (explode_cross_product if self._cross_product else explode_zip) + (explode_cross_product if cross_product else explode_zip) ({name: getattr(row, name) for name in all_fields}, to_explode))) From b10c7cef7c8bf1d41008d8cfcdcdfc3e0d40ff59 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw <robertwb@gmail.com> Date: Wed, 1 Nov 2023 07:36:23 -0700 Subject: [PATCH 330/435] Less frequent reporting of potentially stuck records. (#29222) Once the threshold has been reached, don't report on every sample, but instead in a capped exponentially decaying manner. --- .../control/ExecutionStateSampler.java | 60 +++++++++++-------- 1 file changed, 36 insertions(+), 24 deletions(-) diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ExecutionStateSampler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ExecutionStateSampler.java index a82ce9276820..5509d6380ef6 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ExecutionStateSampler.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ExecutionStateSampler.java @@ -250,6 +250,8 @@ public class ExecutionStateTracker implements BundleProgressReporter { private final AtomicReference<@Nullable Thread> trackedThread; // Read by multiple threads, read and written by the ExecutionStateSampler thread lazily. private final AtomicLong lastTransitionTime; + // Used to throttle lull logging. + private long lastLullReport; // Read and written by the bundle processing thread frequently. private long numTransitions; // Read by the ExecutionStateSampler, written by the bundle processing thread lazily and @@ -333,31 +335,41 @@ private void takeSample(long currentTimeMillis, long millisSinceLastSample) { transitionsAtLastSample = transitionsAtThisSample; } else { long lullTimeMs = currentTimeMillis - lastTransitionTime.get(); - Thread thread = trackedThread.get(); if (lullTimeMs > MAX_LULL_TIME_MS) { - if (thread == null) { - LOG.warn( - String.format( - "Operation ongoing in bundle %s for at least %s without outputting or completing (stack trace unable to be generated).", - processBundleId.get(), - DURATION_FORMATTER.print(Duration.millis(lullTimeMs).toPeriod()))); - } else if (currentExecutionState == null) { - LOG.warn( - String.format( - "Operation ongoing in bundle %s for at least %s without outputting or completing:%n at %s", - processBundleId.get(), - DURATION_FORMATTER.print(Duration.millis(lullTimeMs).toPeriod()), - Joiner.on("\n at ").join(thread.getStackTrace()))); - } else { - LOG.warn( - String.format( - "Operation ongoing in bundle %s for PTransform{id=%s, name=%s, state=%s} for at least %s without outputting or completing:%n at %s", - processBundleId.get(), - currentExecutionState.ptransformId, - currentExecutionState.ptransformUniqueName, - currentExecutionState.stateName, - DURATION_FORMATTER.print(Duration.millis(lullTimeMs).toPeriod()), - Joiner.on("\n at ").join(thread.getStackTrace()))); + if (lullTimeMs < lastLullReport // This must be a new report. + || lullTimeMs > 1.2 * lastLullReport // Exponential backoff. + || lullTimeMs + > MAX_LULL_TIME_MS + lastLullReport // At least once every MAX_LULL_TIME_MS. + ) { + lastLullReport = lullTimeMs; + Thread thread = trackedThread.get(); + if (thread == null) { + LOG.warn( + String.format( + "Operation ongoing in bundle %s for at least %s without outputting " + + "or completing (stack trace unable to be generated).", + processBundleId.get(), + DURATION_FORMATTER.print(Duration.millis(lullTimeMs).toPeriod()))); + } else if (currentExecutionState == null) { + LOG.warn( + String.format( + "Operation ongoing in bundle %s for at least %s without outputting " + + "or completing:%n at %s", + processBundleId.get(), + DURATION_FORMATTER.print(Duration.millis(lullTimeMs).toPeriod()), + Joiner.on("\n at ").join(thread.getStackTrace()))); + } else { + LOG.warn( + String.format( + "Operation ongoing in bundle %s for PTransform{id=%s, name=%s, state=%s} " + + "for at least %s without outputting or completing:%n at %s", + processBundleId.get(), + currentExecutionState.ptransformId, + currentExecutionState.ptransformUniqueName, + currentExecutionState.stateName, + DURATION_FORMATTER.print(Duration.millis(lullTimeMs).toPeriod()), + Joiner.on("\n at ").join(thread.getStackTrace()))); + } } } } From f53ebfef62d5bb3765a97bb8a2e5e83551495533 Mon Sep 17 00:00:00 2001 From: Jeff Kinard <35542536+Polber@users.noreply.github.com> Date: Wed, 1 Nov 2023 10:46:45 -0400 Subject: [PATCH 331/435] [YAML] Add yaml label to yaml transform (#29227) Signed-off-by: Jeffrey Kinard <jeff@thekinards.com> --- sdks/python/apache_beam/yaml/yaml_transform.py | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/sdks/python/apache_beam/yaml/yaml_transform.py b/sdks/python/apache_beam/yaml/yaml_transform.py index ca63834e283f..3fde097f5c22 100644 --- a/sdks/python/apache_beam/yaml/yaml_transform.py +++ b/sdks/python/apache_beam/yaml/yaml_transform.py @@ -32,6 +32,7 @@ from yaml.loader import SafeLoader import apache_beam as beam +from apache_beam.options.pipeline_options import GoogleCloudOptions from apache_beam.transforms.fully_qualified_named_transform import FullyQualifiedNamedTransform from apache_beam.yaml import yaml_provider from apache_beam.yaml.yaml_combine import normalize_combine @@ -940,9 +941,11 @@ def __init__(self, spec, providers={}): # pylint: disable=dangerous-default-val def expand(self, pcolls): if isinstance(pcolls, beam.pvalue.PBegin): root = pcolls + pipeline = root.pipeline pcolls = {} elif isinstance(pcolls, beam.PCollection): root = pcolls.pipeline + pipeline = root pcolls = {'input': pcolls} if not self._spec['input']: self._spec['input'] = {'input': 'input'} @@ -951,9 +954,14 @@ def expand(self, pcolls): self._spec['transforms'][0]['input'] = self._spec['input'] else: root = next(iter(pcolls.values())).pipeline + pipeline = root if not self._spec['input']: self._spec['input'] = {name: name for name in pcolls.keys()} python_provider = yaml_provider.InlineProvider({}) + + options = pipeline.options.view_as(GoogleCloudOptions) + options.labels = ["yaml=true"] + result = expand_transform( self._spec, Scope( From 3d3a7afe0df985f3b1ff7632edbf435bcf86c80c Mon Sep 17 00:00:00 2001 From: johnjcasey <95318300+johnjcasey@users.noreply.github.com> Date: Wed, 1 Nov 2023 09:51:13 -0500 Subject: [PATCH 332/435] Bugfix/kafka nullable header coders (#29244) * Update 2.50 release notes to include new Kafka topicPattern feature * Create groovy class for io performance tests Create gradle task and github actions config for GCS using this. * delete unnecessary class * fix env call * fix call to gradle * run on hosted runner for testing * add additional checkout * add destination for triggered tests * move env variables to correct location * try uploading against separate dataset * try without a user * update branch checkout, try to view the failure log * run on failure * update to use correct BigQuery instance * convert to matrix * add result reporting * add failure clause * remove failure clause, update to run on self-hosted * address comments, clean up build * clarify branching * update kafka coders to support nullable header values --- .../main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java | 4 ++-- .../org/apache/beam/sdk/io/kafka/KafkaRecordCoder.java | 3 ++- .../org/apache/beam/sdk/io/kafka/KafkaRecordCoderTest.java | 7 +++++++ 3 files changed, 11 insertions(+), 3 deletions(-) diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java index 7275986de8b5..ea0c34e576b9 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java @@ -1759,10 +1759,10 @@ public void populateDisplayData(DisplayData.Builder builder) { static class KafkaHeader { String key; - byte[] value; + byte @Nullable [] value; @SchemaCreate - public KafkaHeader(String key, byte[] value) { + public KafkaHeader(String key, byte @Nullable [] value) { this.key = key; this.value = value; } diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaRecordCoder.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaRecordCoder.java index 2cb1efe65704..dbb3a053099c 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaRecordCoder.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaRecordCoder.java @@ -27,6 +27,7 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.IterableCoder; import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.coders.NullableCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.StructuredCoder; import org.apache.beam.sdk.coders.VarIntCoder; @@ -44,7 +45,7 @@ public class KafkaRecordCoder<K, V> extends StructuredCoder<KafkaRecord<K, V>> { private static final Coder<Long> longCoder = VarLongCoder.of(); private static final Coder<Integer> intCoder = VarIntCoder.of(); private static final Coder<Iterable<KV<String, byte[]>>> headerCoder = - IterableCoder.of(KvCoder.of(stringCoder, ByteArrayCoder.of())); + IterableCoder.of(KvCoder.of(stringCoder, NullableCoder.of(ByteArrayCoder.of()))); private final KvCoder<K, V> kvCoder; diff --git a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaRecordCoderTest.java b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaRecordCoderTest.java index 6720d67821ae..84d8cedb895a 100644 --- a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaRecordCoderTest.java +++ b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaRecordCoderTest.java @@ -55,6 +55,13 @@ public void testKafkaRecordSerializableWithoutHeaders() throws IOException { verifySerialization(consumerRecord.headers()); } + @Test + public void testKafkaRecordSerializableWithNullValueHeader() throws IOException { + RecordHeaders headers = new RecordHeaders(); + headers.add("headerKey", null); + verifySerialization(headers); + } + private void verifySerialization(Headers headers) throws IOException { KafkaRecord<String, String> kafkaRecord = new KafkaRecord<>( From b9725e352a053ad62fb1e314f526bfb024dfa085 Mon Sep 17 00:00:00 2001 From: Jeffrey Kinard <jeff@thekinards.com> Date: Tue, 31 Oct 2023 17:06:23 -0400 Subject: [PATCH 333/435] unpollute namespace Signed-off-by: Jeffrey Kinard <jeff@thekinards.com> --- sdks/python/apache_beam/yaml/yaml_mapping.py | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/sdks/python/apache_beam/yaml/yaml_mapping.py b/sdks/python/apache_beam/yaml/yaml_mapping.py index 7e6acc424d5b..42af9a803fbb 100644 --- a/sdks/python/apache_beam/yaml/yaml_mapping.py +++ b/sdks/python/apache_beam/yaml/yaml_mapping.py @@ -17,6 +17,7 @@ """This module defines the basic MapToFields operation.""" import itertools +from collections import abc from typing import Any from typing import Callable from typing import Collection @@ -79,13 +80,14 @@ def __setstate__(self, state): self.__dict__.update(state) +# TODO(yaml) Improve type inferencing for JS UDF's def py_value_to_js_dict(py_value): if ((isinstance(py_value, tuple) and hasattr(py_value, '_asdict')) or isinstance(py_value, beam.Row)): py_value = py_value._asdict() if isinstance(py_value, dict): return {key: py_value_to_js_dict(value) for key, value in py_value.items()} - elif not isinstance(py_value, str) and isinstance(py_value, Iterable): + elif not isinstance(py_value, str) and isinstance(py_value, abc.Iterable): return [py_value_to_js_dict(value) for value in list(py_value)] else: return py_value @@ -131,8 +133,8 @@ def _js_object_to_py_object(obj): return obj if expression: - source = '\n'.join(['function(row) {'] + [ - f' {name} = row.{name}' + source = '\n'.join(['function(__row__) {'] + [ + f' {name} = __row__.{name}' for name in original_fields if name in expression ] + [' return (' + expression + ')'] + ['}']) js_func = _CustomJsObjectWrapper(js2py.eval_js(source)) From 0113befc701992c10bfab7a77a45e98faf898984 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 1 Nov 2023 11:25:45 -0400 Subject: [PATCH 334/435] Bump github.com/aws/aws-sdk-go-v2/config from 1.19.1 to 1.20.0 in /sdks (#29240) Bumps [github.com/aws/aws-sdk-go-v2/config](https://github.com/aws/aws-sdk-go-v2) from 1.19.1 to 1.20.0. - [Release notes](https://github.com/aws/aws-sdk-go-v2/releases) - [Changelog](https://github.com/aws/aws-sdk-go-v2/blob/v1.20.0/CHANGELOG.md) - [Commits](https://github.com/aws/aws-sdk-go-v2/compare/v1.19.1...v1.20.0) --- updated-dependencies: - dependency-name: github.com/aws/aws-sdk-go-v2/config dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] <support@github.com> Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 24 ++++++++++++------------ sdks/go.sum | 37 ++++++++++++++++++++++++------------- 2 files changed, 36 insertions(+), 25 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 64e319040dbf..3ab88acd0049 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -30,12 +30,12 @@ require ( cloud.google.com/go/pubsub v1.33.0 cloud.google.com/go/spanner v1.51.0 cloud.google.com/go/storage v1.33.0 - github.com/aws/aws-sdk-go-v2 v1.21.2 - github.com/aws/aws-sdk-go-v2/config v1.19.1 - github.com/aws/aws-sdk-go-v2/credentials v1.13.43 + github.com/aws/aws-sdk-go-v2 v1.22.0 + github.com/aws/aws-sdk-go-v2/config v1.20.0 + github.com/aws/aws-sdk-go-v2/credentials v1.14.0 github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.91 github.com/aws/aws-sdk-go-v2/service/s3 v1.40.2 - github.com/aws/smithy-go v1.15.0 + github.com/aws/smithy-go v1.16.0 github.com/docker/go-connections v0.4.0 github.com/dustin/go-humanize v1.0.1 github.com/go-sql-driver/mysql v1.7.1 @@ -98,18 +98,18 @@ require ( github.com/apache/thrift v0.16.0 // indirect github.com/aws/aws-sdk-go v1.34.0 // indirect github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.4.14 // indirect - github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.13.13 // indirect - github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.43 // indirect - github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.37 // indirect - github.com/aws/aws-sdk-go-v2/internal/ini v1.3.45 // indirect + github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.0 // indirect + github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.0 // indirect + github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.0 // indirect + github.com/aws/aws-sdk-go-v2/internal/ini v1.4.0 // indirect github.com/aws/aws-sdk-go-v2/internal/v4a v1.1.6 // indirect github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.9.15 // indirect github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.1.38 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.37 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.0 // indirect github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.15.6 // indirect - github.com/aws/aws-sdk-go-v2/service/sso v1.15.2 // indirect - github.com/aws/aws-sdk-go-v2/service/ssooidc v1.17.3 // indirect - github.com/aws/aws-sdk-go-v2/service/sts v1.23.2 // indirect + github.com/aws/aws-sdk-go-v2/service/sso v1.16.0 // indirect + github.com/aws/aws-sdk-go-v2/service/ssooidc v1.18.0 // indirect + github.com/aws/aws-sdk-go-v2/service/sts v1.24.0 // indirect github.com/cenkalti/backoff/v4 v4.2.1 // indirect github.com/census-instrumentation/opencensus-proto v0.4.1 // indirect github.com/cespare/xxhash/v2 v2.2.0 // indirect diff --git a/sdks/go.sum b/sdks/go.sum index 3e2cd5dff4d1..2622786d8e70 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -81,30 +81,36 @@ github.com/aws/aws-sdk-go v1.30.19/go.mod h1:5zCpMtNQVjRREroY7sYe8lOMRSxkhG6MZve github.com/aws/aws-sdk-go v1.34.0 h1:brux2dRrlwCF5JhTL7MUT3WUwo9zfDHZZp3+g3Mvlmo= github.com/aws/aws-sdk-go v1.34.0/go.mod h1:5zCpMtNQVjRREroY7sYe8lOMRSxkhG6MZveU8YkpAk0= github.com/aws/aws-sdk-go-v2 v1.7.1/go.mod h1:L5LuPC1ZgDr2xQS7AmIec/Jlc7O/Y1u2KxJyNVab250= -github.com/aws/aws-sdk-go-v2 v1.21.2 h1:+LXZ0sgo8quN9UOKXXzAWRT3FWd4NxeXWOZom9pE7GA= github.com/aws/aws-sdk-go-v2 v1.21.2/go.mod h1:ErQhvNuEMhJjweavOYhxVkn2RUx7kQXVATHrjKtxIpM= +github.com/aws/aws-sdk-go-v2 v1.22.0 h1:CpTS3XO3MWNel8ohoazkLZC6scvkYL2k+m0yzFJ17Hg= +github.com/aws/aws-sdk-go-v2 v1.22.0/go.mod h1:Kd0OJtkW3Q0M0lUWGszapWjEvrXDzRW+D21JNsroB+c= github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.4.14 h1:Sc82v7tDQ/vdU1WtuSyzZ1I7y/68j//HJ6uozND1IDs= github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.4.14/go.mod h1:9NCTOURS8OpxvoAVHq79LK81/zC78hfRWFn+aL0SPcY= github.com/aws/aws-sdk-go-v2/config v1.5.0/go.mod h1:RWlPOAW3E3tbtNAqTwvSW54Of/yP3oiZXMI0xfUdjyA= github.com/aws/aws-sdk-go-v2/config v1.19.0/go.mod h1:ZwDUgFnQgsazQTnWfeLWk5GjeqTQTL8lMkoE1UXzxdE= -github.com/aws/aws-sdk-go-v2/config v1.19.1 h1:oe3vqcGftyk40icfLymhhhNysAwk0NfiwkDi2GTPMXs= -github.com/aws/aws-sdk-go-v2/config v1.19.1/go.mod h1:ZwDUgFnQgsazQTnWfeLWk5GjeqTQTL8lMkoE1UXzxdE= +github.com/aws/aws-sdk-go-v2/config v1.20.0 h1:q2+/mqFhY0J9m3Tb5RGFE3R4sdaUkIe4k2EuDfE3c08= +github.com/aws/aws-sdk-go-v2/config v1.20.0/go.mod h1:7+1riCZXyT+sAGvneR5j+Zl1GyfbBUNQurpQTE6FP6k= github.com/aws/aws-sdk-go-v2/credentials v1.3.1/go.mod h1:r0n73xwsIVagq8RsxmZbGSRQFj9As3je72C2WzUIToc= -github.com/aws/aws-sdk-go-v2/credentials v1.13.43 h1:LU8vo40zBlo3R7bAvBVy/ku4nxGEyZe9N8MqAeFTzF8= github.com/aws/aws-sdk-go-v2/credentials v1.13.43/go.mod h1:zWJBz1Yf1ZtX5NGax9ZdNjhhI4rgjfgsyk6vTY1yfVg= +github.com/aws/aws-sdk-go-v2/credentials v1.14.0 h1:LQquqPE7cL55RQmA/UBoBKehDlEtMnQKm3B0Q672ePE= +github.com/aws/aws-sdk-go-v2/credentials v1.14.0/go.mod h1:q/3oaTPlamrQWHPwJe56Mjq9g1TYDgddvgTgWJtHTmE= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.3.0/go.mod h1:2LAuqPx1I6jNfaGDucWfA2zqQCYCOMCDHiCOciALyNw= -github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.13.13 h1:PIktER+hwIG286DqXyvVENjgLTAwGgoeriLDD5C+YlQ= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.13.13/go.mod h1:f/Ib/qYjhV2/qdsf79H3QP/eRE4AkVyEf6sk7XfZ1tg= +github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.0 h1:lF/cVllNAPKgjDwN2RsQUX9g/f6hXer9f10ubLFSoug= +github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.0/go.mod h1:c28nJNzMVVb9TQpZ5q4tzZvwEJwf/7So7Ie2s90l1Fw= github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.3.2/go.mod h1:qaqQiHSrOUVOfKe6fhgQ6UzhxjwqVW8aHNegd6Ws4w4= github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.91 h1:haAyxKHwoE+y/TJt+qHcPQf1dCViyyGbWcKjjYUllTE= github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.91/go.mod h1:ACQ6ta5YFlfSOz2c9A+EVYawLxFMZ0rI3Q0A0tGieKo= -github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.43 h1:nFBQlGtkbPzp/NjZLuFxRqmT91rLJkgvsEQs68h962Y= github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.43/go.mod h1:auo+PiyLl0n1l8A0e8RIeR8tOzYPfZZH/JNlrJ8igTQ= -github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.37 h1:JRVhO25+r3ar2mKGP7E0LDl8K9/G36gjlqca5iQbaqc= +github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.0 h1:tN6dNNE4SzMuyMnVtQJXGVKX177/d5Zy4MuA1HA4KUc= +github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.0/go.mod h1:F6MXWETIeetAHwFHyoHEqrcB3NpijFv9nLP5h9CXtT0= github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.37/go.mod h1:Qe+2KtKml+FEsQF/DHmDV+xjtche/hwoF75EG4UlHW8= +github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.0 h1:bfdsbTARDjaC/dSYGMO+E0psxFU4hTvCLnqYAfZ3D38= +github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.0/go.mod h1:Jg8XVv5M2V2wiAMvBFx+O59jg6Yr8vhP0bgNF/IuquM= github.com/aws/aws-sdk-go-v2/internal/ini v1.1.1/go.mod h1:Zy8smImhTdOETZqfyn01iNOe0CNggVbPjCajyaz6Gvg= -github.com/aws/aws-sdk-go-v2/internal/ini v1.3.45 h1:hze8YsjSh8Wl1rYa1CJpRmXP21BvOBuc76YhW0HsuQ4= github.com/aws/aws-sdk-go-v2/internal/ini v1.3.45/go.mod h1:lD5M20o09/LCuQ2mE62Mb/iSdSlCNuj6H5ci7tW7OsE= +github.com/aws/aws-sdk-go-v2/internal/ini v1.4.0 h1:21tlTXq3ev10yLMAjXZzpkZbrl49h3ElSjmxD57tD/E= +github.com/aws/aws-sdk-go-v2/internal/ini v1.4.0/go.mod h1:d9YrBHJhyzDCv5UsEVRizHlFV6Q0sLemFq6uxuqWfUw= github.com/aws/aws-sdk-go-v2/internal/v4a v1.1.6 h1:wmGLw2i8ZTlHLw7a9ULGfQbuccw8uIiNr6sol5bFzc8= github.com/aws/aws-sdk-go-v2/internal/v4a v1.1.6/go.mod h1:Q0Hq2X/NuL7z8b1Dww8rmOFl+jzusKEcyvkKspwdpyc= github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.2.1/go.mod h1:v33JQ57i2nekYTA70Mb+O18KeH4KqhdqxTJZNK1zdRE= @@ -113,8 +119,9 @@ github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.9.15/go.mod h1: github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.1.38 h1:skaFGzv+3kA+v2BPKhuekeb1Hbb105+44r8ASC+q5SE= github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.1.38/go.mod h1:epIZoRSSbRIwLPJU5F+OldHhwZPBdpDeQkRdCeY3+00= github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.2.1/go.mod h1:zceowr5Z1Nh2WVP8bf/3ikB41IZW59E4yIYbg+pC6mw= -github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.37 h1:WWZA/I2K4ptBS1kg0kV1JbBtG/umed0vwHRrmcr9z7k= github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.37/go.mod h1:vBmDnwWXWxNPFRMmG2m/3MKOe+xEcMDo1tanpaWCcck= +github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.0 h1:dJnwy5Awv+uvfk73aRENVbv1cSQQ60ydCkPaun097KM= +github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.0/go.mod h1:RsPWWy7u/hwmFX57sQ7MLvrvJeYyNkiMm5BaavpoU18= github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.5.1/go.mod h1:6EQZIwNNvHpq/2/QSJnp4+ECvqIy55w95Ofs0ze+nGQ= github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.15.6 h1:9ulSU5ClouoPIYhDQdg9tpl83d5Yb91PXTKK+17q+ow= github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.15.6/go.mod h1:lnc2taBsR9nTlz9meD+lhFZZ9EWY712QHrRflWpTcOA= @@ -122,16 +129,20 @@ github.com/aws/aws-sdk-go-v2/service/s3 v1.11.1/go.mod h1:XLAGFrEjbvMCLvAtWLLP32 github.com/aws/aws-sdk-go-v2/service/s3 v1.40.2 h1:Ll5/YVCOzRB+gxPqs2uD0R7/MyATC0w85626glSKmp4= github.com/aws/aws-sdk-go-v2/service/s3 v1.40.2/go.mod h1:Zjfqt7KhQK+PO1bbOsFNzKgaq7TcxzmEoDWN8lM0qzQ= github.com/aws/aws-sdk-go-v2/service/sso v1.3.1/go.mod h1:J3A3RGUvuCZjvSuZEcOpHDnzZP/sKbhDWV2T1EOzFIM= -github.com/aws/aws-sdk-go-v2/service/sso v1.15.2 h1:JuPGc7IkOP4AaqcZSIcyqLpFSqBWK32rM9+a1g6u73k= github.com/aws/aws-sdk-go-v2/service/sso v1.15.2/go.mod h1:gsL4keucRCgW+xA85ALBpRFfdSLH4kHOVSnLMSuBECo= -github.com/aws/aws-sdk-go-v2/service/ssooidc v1.17.3 h1:HFiiRkf1SdaAmV3/BHOFZ9DjFynPHj8G/UIO1lQS+fk= +github.com/aws/aws-sdk-go-v2/service/sso v1.16.0 h1:ZIlR6Wr/EgYwBdEz1NWBqdUsTh0mV7A68pId3YZl6H0= +github.com/aws/aws-sdk-go-v2/service/sso v1.16.0/go.mod h1:O7B5cpuhhJKefAKkM7onb0McmpHyKnsH4RrHJhOyq7M= github.com/aws/aws-sdk-go-v2/service/ssooidc v1.17.3/go.mod h1:a7bHA82fyUXOm+ZSWKU6PIoBxrjSprdLoM8xPYvzYVg= +github.com/aws/aws-sdk-go-v2/service/ssooidc v1.18.0 h1:3BZyJei4k1SHdSAFhg9Qg15NnG3v5zosZyFWPm7df/A= +github.com/aws/aws-sdk-go-v2/service/ssooidc v1.18.0/go.mod h1:Td8EvzggonY02wLaqSpwybI3GbmA0PWoprKGil2uwJg= github.com/aws/aws-sdk-go-v2/service/sts v1.6.0/go.mod h1:q7o0j7d7HrJk/vr9uUt3BVRASvcU7gYZB9PUgPiByXg= -github.com/aws/aws-sdk-go-v2/service/sts v1.23.2 h1:0BkLfgeDjfZnZ+MhB3ONb01u9pwFYTCZVhlsSSBvlbU= github.com/aws/aws-sdk-go-v2/service/sts v1.23.2/go.mod h1:Eows6e1uQEsc4ZaHANmsPRzAKcVDrcmjjWiih2+HUUQ= +github.com/aws/aws-sdk-go-v2/service/sts v1.24.0 h1:f/V5Y9OaHuNRrA9MntNQNAtMFXqhKj8HTEPnH81eXMI= +github.com/aws/aws-sdk-go-v2/service/sts v1.24.0/go.mod h1:HnCUMNz2XqwnEEk5X6oeDYB2HgOLFpJ/LyfilN8WErs= github.com/aws/smithy-go v1.6.0/go.mod h1:SObp3lf9smib00L/v3U2eAKG8FyQ7iLrJnQiAmR5n+E= -github.com/aws/smithy-go v1.15.0 h1:PS/durmlzvAFpQHDs4wi4sNNP9ExsqZh6IlfdHXgKK8= github.com/aws/smithy-go v1.15.0/go.mod h1:Tg+OJXh4MB2R/uN61Ko2f6hTZwB/ZYGOtib8J3gBHzA= +github.com/aws/smithy-go v1.16.0 h1:gJZEH/Fqh+RsvlJ1Zt4tVAtV6bKkp3cC+R6FCZMNzik= +github.com/aws/smithy-go v1.16.0/go.mod h1:NukqUGpCZIILqqiV0NIjeFh24kd/FAa4beRb6nbIUPE= github.com/cenkalti/backoff/v4 v4.2.1 h1:y4OZtCnogmCPw98Zjyt5a6+QwPLGkiQsYW5oUqylYbM= github.com/cenkalti/backoff/v4 v4.2.1/go.mod h1:Y3VNntkOUPxTVeUxJ/G5vcM//AlwfmyYozVcomhLiZE= github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= From e20c5365cc9037f01334293c8fb7044b8851a0a6 Mon Sep 17 00:00:00 2001 From: Jeff Kinard <35542536+Polber@users.noreply.github.com> Date: Wed, 1 Nov 2023 11:41:29 -0400 Subject: [PATCH 335/435] add self to java reviewers to be able to merge and review streaming worker code (#29235) (#29246) Co-authored-by: martin trieu <martinkt@google.com> --- CHANGES.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGES.md b/CHANGES.md index 366f3236df9b..fa6e51cde2e4 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -75,6 +75,7 @@ should handle this. ([#25252](https://github.com/apache/beam/issues/25252)). using the DataSet API. * `upload_graph` as one of the Experiments options for DataflowRunner is no longer required when the graph is larger than 10MB for Java SDK ([PR#28621](https://github.com/apache/beam/pull/28621). * state amd side input cache has been enabled to a default of 100 MB. Use `--max_cache_memory_usage_mb=X` to provide cache size for the user state API and side inputs. (Python) ([#28770](https://github.com/apache/beam/issues/28770)). +* Beam YAML stable release. Beam pipelines can now be written using YAML and leverage the Beam YAML framework which includes a preliminary set of IO's and turnkey transforms. More information can be found in the YAML root folder and in the [README](https://github.com/apache/beam/blob/master/sdks/python/apache_beam/yaml/README.md). ## Breaking Changes From 5d6c18265837cb4eed3f449667f515ca6e898e0e Mon Sep 17 00:00:00 2001 From: Jack McCluskey <34928439+jrmccluskey@users.noreply.github.com> Date: Wed, 1 Nov 2023 11:41:58 -0400 Subject: [PATCH 336/435] Implement Cross-Bundle BatchElements (#29175) * Initial StatefulBatchElements transform w/ test * add todo for streaming case * Stateful dynamic batch sizing * Formatting * Imporve stateful handling, incorporate existing tests * Remove logging statement * line too long * Remove extra key for test data * Reorder test methods * whitespace * Move to dedicated route through BatchElements * Remove standalone StatefulBatchElements * Remove reference to StatefulBatchElements, fix test name * Streamline state read logic --- sdks/python/apache_beam/transforms/util.py | 118 ++++++++++++- .../apache_beam/transforms/util_test.py | 160 ++++++++++++++++++ 2 files changed, 277 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/transforms/util.py b/sdks/python/apache_beam/transforms/util.py index fb0e8e9789d8..cacfdb37d7b4 100644 --- a/sdks/python/apache_beam/transforms/util.py +++ b/sdks/python/apache_beam/transforms/util.py @@ -61,6 +61,7 @@ from apache_beam.transforms.trigger import Always from apache_beam.transforms.userstate import BagStateSpec from apache_beam.transforms.userstate import CombiningValueStateSpec +from apache_beam.transforms.userstate import ReadModifyWriteStateSpec from apache_beam.transforms.userstate import TimerSpec from apache_beam.transforms.userstate import on_timer from apache_beam.transforms.window import NonMergingWindowFn @@ -392,7 +393,7 @@ def ignore_next_timing(self): def record_time(self, batch_size): start = self._clock() yield - elapsed = self._clock() - start + elapsed = float(self._clock() - start) elapsed_msec = 1e3 * elapsed + self._remainder_msecs if self._record_metrics: self._size_distribution.update(batch_size) @@ -646,6 +647,107 @@ def finish_bundle(self): self._target_batch_size = self._batch_size_estimator.next_batch_size() +def _pardo_stateful_batch_elements( + input_coder: coders.Coder, + batch_size_estimator: _BatchSizeEstimator, + max_buffering_duration_secs: int, + clock=time.time): + ELEMENT_STATE = BagStateSpec('values', input_coder) + COUNT_STATE = CombiningValueStateSpec('count', input_coder, CountCombineFn()) + BATCH_SIZE_STATE = ReadModifyWriteStateSpec('batch_size', input_coder) + WINDOW_TIMER = TimerSpec('window_end', TimeDomain.WATERMARK) + BUFFERING_TIMER = TimerSpec('buffering_end', TimeDomain.REAL_TIME) + BATCH_ESTIMATOR_STATE = ReadModifyWriteStateSpec( + 'batch_estimator', coders.PickleCoder()) + + class _StatefulBatchElementsDoFn(DoFn): + def process( + self, + element, + window=DoFn.WindowParam, + element_state=DoFn.StateParam(ELEMENT_STATE), + count_state=DoFn.StateParam(COUNT_STATE), + batch_size_state=DoFn.StateParam(BATCH_SIZE_STATE), + batch_estimator_state=DoFn.StateParam(BATCH_ESTIMATOR_STATE), + window_timer=DoFn.TimerParam(WINDOW_TIMER), + buffering_timer=DoFn.TimerParam(BUFFERING_TIMER)): + window_timer.set(window.end) + # Drop the fixed key since we don't care about it + element_state.add(element[1]) + count_state.add(1) + count = count_state.read() + target_size = batch_size_state.read() + # Should only happen on the first element + if target_size is None: + batch_estimator = batch_size_estimator + target_size = batch_estimator.next_batch_size() + batch_size_state.write(target_size) + batch_estimator_state.write(batch_estimator) + + if count == 1 and max_buffering_duration_secs > 0: + # First element in batch, start buffering timer + buffering_timer.set(clock() + max_buffering_duration_secs) + + if count >= target_size: + return self.flush_batch( + element_state, + count_state, + batch_size_state, + batch_estimator_state, + buffering_timer) + + @on_timer(WINDOW_TIMER) + def on_window_timer( + self, + element_state=DoFn.StateParam(ELEMENT_STATE), + count_state=DoFn.StateParam(COUNT_STATE), + batch_size_state=DoFn.StateParam(BATCH_SIZE_STATE), + batch_estimator_state=DoFn.StateParam(BATCH_ESTIMATOR_STATE), + buffering_timer=DoFn.TimerParam(BUFFERING_TIMER)): + return self.flush_batch( + element_state, + count_state, + batch_size_state, + batch_estimator_state, + buffering_timer) + + @on_timer(BUFFERING_TIMER) + def on_buffering_timer( + self, + element_state=DoFn.StateParam(ELEMENT_STATE), + count_state=DoFn.StateParam(COUNT_STATE), + batch_size_state=DoFn.StateParam(BATCH_SIZE_STATE), + batch_estimator_state=DoFn.StateParam(BATCH_ESTIMATOR_STATE), + buffering_timer=DoFn.TimerParam(BUFFERING_TIMER)): + return self.flush_batch( + element_state, + count_state, + batch_size_state, + batch_estimator_state, + buffering_timer) + + def flush_batch( + self, + element_state, + count_state, + batch_size_state, + batch_estimator_state, + buffering_timer): + batch = [element for element in element_state.read()] + if not batch: + return + element_state.clear() + count_state.clear() + batch_estimator = batch_estimator_state.read() + with batch_estimator.record_time(len(batch)): + yield batch + batch_size_state.write(batch_estimator.next_batch_size()) + batch_estimator_state.write(batch_estimator) + buffering_timer.clear() + + return _StatefulBatchElementsDoFn() + + @typehints.with_input_types(T) @typehints.with_output_types(List[T]) class BatchElements(PTransform): @@ -677,6 +779,9 @@ class BatchElements(PTransform): in seconds, excluding fixed cost target_batch_duration_secs_including_fixed_cost: (optional) a target for total time per bundle, in seconds, including fixed cost + max_batch_duration_secs: (optional) the maximum amount of time to buffer + a batch before emitting. Setting this argument to be non-none uses the + stateful implementation of BatchElements. element_size_fn: (optional) A mapping of an element to its contribution to batch size, defaulting to every element having size 1. When provided, attempts to provide batches of optimal total size which may consist of @@ -696,6 +801,7 @@ def __init__( target_batch_overhead=.05, target_batch_duration_secs=10, target_batch_duration_secs_including_fixed_cost=None, + max_batch_duration_secs=None, *, element_size_fn=lambda x: 1, variance=0.25, @@ -712,10 +818,20 @@ def __init__( clock=clock, record_metrics=record_metrics) self._element_size_fn = element_size_fn + self._max_batch_dur = max_batch_duration_secs + self._clock = clock def expand(self, pcoll): if getattr(pcoll.pipeline.runner, 'is_streaming', False): raise NotImplementedError("Requires stateful processing (BEAM-2687)") + elif self._max_batch_dur is not None: + coder = coders.registry.get_coder(pcoll) + return pcoll | WithKeys(0) | ParDo( + _pardo_stateful_batch_elements( + coder, + self._batch_size_estimator, + self._max_batch_dur, + self._clock)) elif pcoll.windowing.is_default(): # This is the same logic as _GlobalWindowsBatchingDoFn, but optimized # for that simpler case. diff --git a/sdks/python/apache_beam/transforms/util_test.py b/sdks/python/apache_beam/transforms/util_test.py index d8a8bacb96cd..5dfe166d3c31 100644 --- a/sdks/python/apache_beam/transforms/util_test.py +++ b/sdks/python/apache_beam/transforms/util_test.py @@ -190,6 +190,30 @@ def sleep(self, duration): class BatchElementsTest(unittest.TestCase): + NUM_ELEMENTS = 10 + BATCH_SIZE = 5 + + @staticmethod + def _create_test_data(): + scientists = [ + "Einstein", + "Darwin", + "Copernicus", + "Pasteur", + "Curie", + "Faraday", + "Newton", + "Bohr", + "Galilei", + "Maxwell" + ] + + data = [] + for i in range(BatchElementsTest.NUM_ELEMENTS): + index = i % len(scientists) + data.append(scientists[index]) + return data + def test_constant_batch(self): # Assumes a single bundle... p = TestPipeline() @@ -461,6 +485,142 @@ def test_numpy_regression(self): self._run_regression_test( util._BatchSizeEstimator.linear_regression_numpy, True) + def test_stateful_constant_batch(self): + # Assumes a single bundle... + p = TestPipeline() + output = ( + p + | beam.Create(range(35)) + | util.BatchElements( + min_batch_size=10, max_batch_size=10, max_batch_duration_secs=100) + | beam.Map(len)) + assert_that(output, equal_to([10, 10, 10, 5])) + res = p.run() + res.wait_until_finish() + + def test_stateful_in_global_window(self): + with TestPipeline() as pipeline: + collection = pipeline \ + | beam.Create( + BatchElementsTest._create_test_data()) \ + | util.BatchElements( + min_batch_size=BatchElementsTest.BATCH_SIZE, + max_batch_size=BatchElementsTest.BATCH_SIZE, + max_batch_duration_secs=100) + num_batches = collection | beam.combiners.Count.Globally() + assert_that( + num_batches, + equal_to([ + int( + math.ceil( + BatchElementsTest.NUM_ELEMENTS / + BatchElementsTest.BATCH_SIZE)) + ])) + + def test_stateful_buffering_timer_in_fixed_window_streaming(self): + window_duration = 6 + max_buffering_duration_secs = 100 + + start_time = timestamp.Timestamp(0) + test_stream = ( + TestStream().add_elements([ + TimestampedValue(value, start_time + i) for i, + value in enumerate(BatchElementsTest._create_test_data()) + ]).advance_processing_time(150).advance_watermark_to( + start_time + window_duration).advance_watermark_to( + start_time + window_duration + + 1).advance_watermark_to_infinity()) + + with TestPipeline(options=StandardOptions(streaming=True)) as pipeline: + # To trigger the processing time timer, use a fake clock with start time + # being Timestamp(0). + fake_clock = FakeClock(now=start_time) + + num_elements_per_batch = ( + pipeline | test_stream + | "fixed window" >> WindowInto(FixedWindows(window_duration)) + | util.BatchElements( + min_batch_size=BatchElementsTest.BATCH_SIZE, + max_batch_size=BatchElementsTest.BATCH_SIZE, + max_batch_duration_secs=max_buffering_duration_secs, + clock=fake_clock) + | "count elements in batch" >> Map(lambda x: (None, len(x))) + | GroupByKey() + | "global window" >> WindowInto(GlobalWindows()) + | FlatMapTuple(lambda k, vs: vs)) + + # Window duration is 6 and batch size is 5, so output batch size + # should be 5 (flush because of batch size reached). + expected_0 = 5 + # There is only one element left in the window so batch size + # should be 1 (flush because of max buffering duration reached). + expected_1 = 1 + # Collection has 10 elements, there are only 4 left, so batch size should + # be 4 (flush because of end of window reached). + expected_2 = 4 + assert_that( + num_elements_per_batch, + equal_to([expected_0, expected_1, expected_2]), + "assert2") + + def test_stateful_buffering_timer_in_global_window_streaming(self): + max_buffering_duration_secs = 42 + + start_time = timestamp.Timestamp(0) + test_stream = TestStream().advance_watermark_to(start_time) + for i, value in enumerate(BatchElementsTest._create_test_data()): + test_stream.add_elements( + [TimestampedValue(value, start_time + i)]) \ + .advance_processing_time(5) + test_stream.advance_watermark_to( + start_time + BatchElementsTest.NUM_ELEMENTS + 1) \ + .advance_watermark_to_infinity() + + with TestPipeline(options=StandardOptions(streaming=True)) as pipeline: + # Set a batch size larger than the total number of elements. + # Since we're in a global window, we would have been waiting + # for all the elements to arrive without the buffering time limit. + batch_size = BatchElementsTest.NUM_ELEMENTS * 2 + + # To trigger the processing time timer, use a fake clock with start time + # being Timestamp(0). Since the fake clock never really advances during + # the pipeline execution, meaning that the timer is always set to the same + # value, the timer will be fired on every element after the first firing. + fake_clock = FakeClock(now=start_time) + + num_elements_per_batch = ( + pipeline | test_stream + | WindowInto( + GlobalWindows(), + trigger=Repeatedly(AfterCount(1)), + accumulation_mode=trigger.AccumulationMode.DISCARDING) + | util.BatchElements( + min_batch_size=batch_size, + max_batch_size=batch_size, + max_batch_duration_secs=max_buffering_duration_secs, + clock=fake_clock) + | 'count elements in batch' >> Map(lambda x: (None, len(x))) + | GroupByKey() + | FlatMapTuple(lambda k, vs: vs)) + + # We will flush twice when the max buffering duration is reached and when + # the global window ends. + assert_that(num_elements_per_batch, equal_to([9, 1])) + + def test_stateful_grows_to_max_batch(self): + # Assumes a single bundle... + with TestPipeline() as p: + res = ( + p + | beam.Create(range(164)) + | util.BatchElements( + min_batch_size=1, + max_batch_size=50, + max_batch_duration_secs=100, + clock=FakeClock()) + | beam.Map(len)) + assert_that(res, equal_to([1, 1, 2, 4, 8, 16, 32, 50, 50])) + class IdentityWindowTest(unittest.TestCase): def test_window_preserved(self): From 4dd147dbe2062393edd6bbdda236e243d532902b Mon Sep 17 00:00:00 2001 From: Yi Hu <yathu@google.com> Date: Wed, 1 Nov 2023 12:11:37 -0400 Subject: [PATCH 337/435] Shutdown LoadTest, PerformanceTest, and Others Category Jenkins jobs (#29247) --- .../job_CancelStaleDataflowJobs.groovy | 44 --- .../job_CleanUpDataprocResources.groovy | 37 -- .../jenkins/job_CleanUpGCPResources.groovy | 44 --- .../job_CleanUpPrebuiltSDKImages.groovy | 44 --- .../job_CloudMLBenchmarkTests_Python.groovy | 67 ---- .../job_InferenceBenchmarkTests_Python.groovy | 206 ---------- .../jenkins/job_LoadTests_CoGBK_Java.groovy | 215 ---------- .../job_LoadTests_Combine_Flink_Go.groovy | 139 ------- .../job_LoadTests_Combine_Flink_Python.groovy | 186 --------- .../job_LoadTests_Combine_Python.groovy | 154 -------- .../jenkins/job_LoadTests_GBK_Flink_Go.groovy | 231 ----------- .../job_LoadTests_ParDo_Flink_Python.groovy | 374 ------------------ .../job_LoadTests_SideInput_Flink_Go.groovy | 105 ----- .../job_LoadTests_coGBK_Flink_Go.groovy | 184 --------- .../jenkins/job_LoadTests_coGBK_Python.groovy | 210 ---------- .../job_PerformanceTests_KafkaIO_IT.groovy | 130 ------ .../job_Publish_Docker_Snapshots.groovy | 50 --- .../job_Publish_SDK_Image_Snapshots.groovy | 56 --- .../job_ReleaseCandidate_Python.groovy | 38 -- .../job_Release_NightlySnapshot.groovy | 78 ---- 20 files changed, 2592 deletions(-) delete mode 100644 .test-infra/jenkins/job_CancelStaleDataflowJobs.groovy delete mode 100644 .test-infra/jenkins/job_CleanUpDataprocResources.groovy delete mode 100644 .test-infra/jenkins/job_CleanUpGCPResources.groovy delete mode 100644 .test-infra/jenkins/job_CleanUpPrebuiltSDKImages.groovy delete mode 100644 .test-infra/jenkins/job_CloudMLBenchmarkTests_Python.groovy delete mode 100644 .test-infra/jenkins/job_InferenceBenchmarkTests_Python.groovy delete mode 100644 .test-infra/jenkins/job_LoadTests_CoGBK_Java.groovy delete mode 100644 .test-infra/jenkins/job_LoadTests_Combine_Flink_Go.groovy delete mode 100644 .test-infra/jenkins/job_LoadTests_Combine_Flink_Python.groovy delete mode 100644 .test-infra/jenkins/job_LoadTests_Combine_Python.groovy delete mode 100644 .test-infra/jenkins/job_LoadTests_GBK_Flink_Go.groovy delete mode 100644 .test-infra/jenkins/job_LoadTests_ParDo_Flink_Python.groovy delete mode 100644 .test-infra/jenkins/job_LoadTests_SideInput_Flink_Go.groovy delete mode 100644 .test-infra/jenkins/job_LoadTests_coGBK_Flink_Go.groovy delete mode 100644 .test-infra/jenkins/job_LoadTests_coGBK_Python.groovy delete mode 100644 .test-infra/jenkins/job_PerformanceTests_KafkaIO_IT.groovy delete mode 100644 .test-infra/jenkins/job_Publish_Docker_Snapshots.groovy delete mode 100644 .test-infra/jenkins/job_Publish_SDK_Image_Snapshots.groovy delete mode 100644 .test-infra/jenkins/job_ReleaseCandidate_Python.groovy delete mode 100644 .test-infra/jenkins/job_Release_NightlySnapshot.groovy diff --git a/.test-infra/jenkins/job_CancelStaleDataflowJobs.groovy b/.test-infra/jenkins/job_CancelStaleDataflowJobs.groovy deleted file mode 100644 index a0ecd1eacfdd..000000000000 --- a/.test-infra/jenkins/job_CancelStaleDataflowJobs.groovy +++ /dev/null @@ -1,44 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonJobProperties as commonJobProperties - -job("beam_CancelStaleDataflowJobs") { - description("Cancel stale dataflow jobs") - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate) - - // Sets that this is a cron job, run once randomly per day. - commonJobProperties.setCronJob(delegate, '0 */4 * * *') - - // Allows triggering this build against pull requests. - commonJobProperties.enablePhraseTriggeringFromPullRequest( - delegate, - 'Cancel Stale Dataflow Jobs', - 'Run Cancel Stale Dataflow Jobs') - - // Gradle goals for this job. - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':beam-test-tools:cancelStaleDataflowJobs') - commonJobProperties.setGradleSwitches(delegate) - } - } -} diff --git a/.test-infra/jenkins/job_CleanUpDataprocResources.groovy b/.test-infra/jenkins/job_CleanUpDataprocResources.groovy deleted file mode 100644 index 142b4c732c6e..000000000000 --- a/.test-infra/jenkins/job_CleanUpDataprocResources.groovy +++ /dev/null @@ -1,37 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonJobProperties as commonJobProperties - - -job('Cleanup Dataproc Resources') { - description('Deletes leaked resources for all the jobs that generates flink clusters.') - - def CLEANUP_DIR = '"$WORKSPACE/src/.test-infra/dataproc"' - def CLEANUP_SCRIPT = 'cleanup.sh' - - commonJobProperties.setTopLevelMainJobProperties(delegate) - - // Sets that this is a cron job. - commonJobProperties.setCronJob(delegate, 'H */6 * * *') - - steps { - shell("cd ${CLEANUP_DIR}; ./${CLEANUP_SCRIPT} -xe") - } - -} diff --git a/.test-infra/jenkins/job_CleanUpGCPResources.groovy b/.test-infra/jenkins/job_CleanUpGCPResources.groovy deleted file mode 100644 index 59b78358e769..000000000000 --- a/.test-infra/jenkins/job_CleanUpGCPResources.groovy +++ /dev/null @@ -1,44 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonJobProperties as commonJobProperties - -job("beam_CleanUpGCPResources") { - description("Clean up stale resources on Beam's GCP testing project (BQ datasets, )") - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate) - - // Sets that this is a cron job, run once randomly per day. - commonJobProperties.setCronJob(delegate, 'H H * * *') - - // Allows triggering this build against pull requests. - commonJobProperties.enablePhraseTriggeringFromPullRequest( - delegate, - 'Clean Up GCP Resources', - 'Run Clean GCP Resources') - - // Gradle goals for this job. - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':beam-test-tools:cleanupOtherStaleResources') - commonJobProperties.setGradleSwitches(delegate) - } - } -} diff --git a/.test-infra/jenkins/job_CleanUpPrebuiltSDKImages.groovy b/.test-infra/jenkins/job_CleanUpPrebuiltSDKImages.groovy deleted file mode 100644 index 224c1bbeac52..000000000000 --- a/.test-infra/jenkins/job_CleanUpPrebuiltSDKImages.groovy +++ /dev/null @@ -1,44 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonJobProperties as commonJobProperties - -job("beam_CleanUpPrebuiltSDKImages") { - description("Clean up stale dataflow prebuilt sdk container images") - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate) - - // Sets that this is a cron job, run once randomly per day. - commonJobProperties.setCronJob(delegate, '0 H * * *') - - // Allows triggering this build against pull requests. - commonJobProperties.enablePhraseTriggeringFromPullRequest( - delegate, - 'Clean Up Prebuilt SDK Images', - 'Run Clean Prebuilt Images') - - // Gradle goals for this job. - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':beam-test-tools:removeStaleSDKContainerImages') - commonJobProperties.setGradleSwitches(delegate) - } - } -} diff --git a/.test-infra/jenkins/job_CloudMLBenchmarkTests_Python.groovy b/.test-infra/jenkins/job_CloudMLBenchmarkTests_Python.groovy deleted file mode 100644 index 1867cccf7754..000000000000 --- a/.test-infra/jenkins/job_CloudMLBenchmarkTests_Python.groovy +++ /dev/null @@ -1,67 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonJobProperties as commonJobProperties -import PhraseTriggeringPostCommitBuilder -import CronJobBuilder - -def cloudMLJob = { scope -> - scope.description('Runs the TFT Criteo Examples on the Dataflow runner.') - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(scope, 'master', 360) - - Map pipelineOptions = [ - influx_db_name : InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influx_hostname : InfluxDBCredentialsHelper.InfluxDBHostUrl, - metrics_dataset : 'beam_cloudml', - publish_to_big_query: true, - project : 'apache-beam-testing', - region : 'us-central1', - staging_location : 'gs://temp-storage-for-perf-tests/loadtests', - temp_location : 'gs://temp-storage-for-perf-tests/loadtests', - runner : 'DataflowRunner', - requirements_file : "apache_beam/testing/benchmarks/cloudml/requirements.txt" - ] - // Gradle goals for this job. - scope.steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - commonJobProperties.setGradleSwitches(delegate) - switches("-Popts=\'${commonJobProperties.mapToArgString(pipelineOptions)}\'") - tasks(':sdks:python:test-suites:dataflow:tftTests') - } - } -} - -PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_CloudML_Benchmarks_Dataflow', - 'Run TFT Criteo Benchmarks', - 'TFT Criteo benchmarks on Dataflow(\"Run TFT Criteo Benchmarks"\"")', - this - ) { - cloudMLJob(delegate) - } - -CronJobBuilder.cronJob( - 'beam_CloudML_Benchmarks_Dataflow', - 'H H * * *', - this - ) { - cloudMLJob(delegate) - } diff --git a/.test-infra/jenkins/job_InferenceBenchmarkTests_Python.groovy b/.test-infra/jenkins/job_InferenceBenchmarkTests_Python.groovy deleted file mode 100644 index a98b8d170437..000000000000 --- a/.test-infra/jenkins/job_InferenceBenchmarkTests_Python.groovy +++ /dev/null @@ -1,206 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonJobProperties as commonJobProperties -import LoadTestsBuilder as loadTestsBuilder -import PhraseTriggeringPostCommitBuilder -import CronJobBuilder -import static PythonTestProperties.RUN_INFERENCE_TEST_PYTHON_VERSION - - -def now = new Date().format("MMddHHmmss", TimeZone.getTimeZone('UTC')) - -def loadTestConfigurations = { - -> - [ - // Benchmark test config. Add multiple configs for multiple models. - [ - title : 'Pytorch Vision Classification with Resnet 101', - test : 'apache_beam.testing.benchmarks.inference.pytorch_image_classification_benchmarks', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - job_name : 'benchmark-tests-pytorch-imagenet-python' + now, - project : 'apache-beam-testing', - region : 'us-central1', - machine_type : 'n1-standard-2', - num_workers : 75, - disk_size_gb : 50, - autoscaling_algorithm : 'NONE', - staging_location : 'gs://temp-storage-for-perf-tests/loadtests', - temp_location : 'gs://temp-storage-for-perf-tests/loadtests', - requirements_file : 'apache_beam/ml/inference/torch_tests_requirements.txt', - publish_to_big_query : true, - metrics_dataset : 'beam_run_inference', - metrics_table : 'torch_inference_imagenet_results_resnet101', - input_options : '{}', // this option is not required for RunInference tests. - influx_measurement : 'torch_inference_imagenet_resnet101', - influx_db_name : InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influx_hostname : InfluxDBCredentialsHelper.InfluxDBHostUrl, - pretrained_model_name : 'resnet101', - device : 'CPU', - input_file : 'gs://apache-beam-ml/testing/inputs/openimage_50k_benchmark.txt', - model_state_dict_path : 'gs://apache-beam-ml/models/torchvision.models.resnet101.pth', - output : 'gs://temp-storage-for-end-to-end-tests/torch/result_101' + now + '.txt' - ] - ], - [ - title : 'Pytorch Imagenet Classification with Resnet 152', - test : 'apache_beam.testing.benchmarks.inference.pytorch_image_classification_benchmarks', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - job_name : 'benchmark-tests-pytorch-imagenet-python' + now, - project : 'apache-beam-testing', - region : 'us-central1', - machine_type : 'n1-standard-2', - num_workers : 75, - disk_size_gb : 50, - autoscaling_algorithm : 'NONE', - staging_location : 'gs://temp-storage-for-perf-tests/loadtests', - temp_location : 'gs://temp-storage-for-perf-tests/loadtests', - requirements_file : 'apache_beam/ml/inference/torch_tests_requirements.txt', - publish_to_big_query : true, - metrics_dataset : 'beam_run_inference', - metrics_table : 'torch_inference_imagenet_results_resnet152', - input_options : '{}', // this option is not required for RunInference tests. - influx_measurement : 'torch_inference_imagenet_resnet152', - influx_db_name : InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influx_hostname : InfluxDBCredentialsHelper.InfluxDBHostUrl, - pretrained_model_name : 'resnet152', - device : 'CPU', - input_file : 'gs://apache-beam-ml/testing/inputs/openimage_50k_benchmark.txt', - model_state_dict_path : 'gs://apache-beam-ml/models/torchvision.models.resnet152.pth', - output : 'gs://temp-storage-for-end-to-end-tests/torch/result_resnet152' + now + '.txt' - ] - ], - // Pytorch language modeling test using HuggingFace BERT models - [ - title : 'Pytorch Lanugaue Modeling using Hugging face bert-base-uncased model', - test : 'apache_beam.testing.benchmarks.inference.pytorch_language_modeling_benchmarks', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - job_name : 'benchmark-tests-pytorch-language-modeling-bert-base-uncased' + now, - project : 'apache-beam-testing', - region : 'us-central1', - machine_type : 'n1-standard-2', - num_workers : 250, - disk_size_gb : 50, - autoscaling_algorithm : 'NONE', - staging_location : 'gs://temp-storage-for-perf-tests/loadtests', - temp_location : 'gs://temp-storage-for-perf-tests/loadtests', - requirements_file : 'apache_beam/ml/inference/torch_tests_requirements.txt', - publish_to_big_query : true, - metrics_dataset : 'beam_run_inference', - metrics_table : 'torch_language_modeling_bert_base_uncased', - input_options : '{}', // this option is not required for RunInference tests. - influx_measurement : 'torch_language_modeling_bert_base_uncased', - influx_db_name : InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influx_hostname : InfluxDBCredentialsHelper.InfluxDBHostUrl, - device : 'CPU', - input_file : 'gs://apache-beam-ml/testing/inputs/sentences_50k.txt', - bert_tokenizer : 'bert-base-uncased', - model_state_dict_path : 'gs://apache-beam-ml/models/huggingface.BertForMaskedLM.bert-base-uncased.pth', - output : 'gs://temp-storage-for-end-to-end-tests/torch/result_bert_base_uncased' + now + '.txt', - ] - ], - [ - title : 'Pytorch Langauge Modeling using Hugging Face bert-large-uncased model', - test : 'apache_beam.testing.benchmarks.inference.pytorch_language_modeling_benchmarks', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - job_name : 'benchmark-tests-pytorch-language-modeling-bert-large-cased' + now, - project : 'apache-beam-testing', - region : 'us-central1', - machine_type : 'n1-standard-2', - num_workers : 250, - disk_size_gb : 50, - autoscaling_algorithm : 'NONE', - staging_location : 'gs://temp-storage-for-perf-tests/loadtests', - temp_location : 'gs://temp-storage-for-perf-tests/loadtests', - requirements_file : 'apache_beam/ml/inference/torch_tests_requirements.txt', - publish_to_big_query : true, - metrics_dataset : 'beam_run_inference', - metrics_table : 'torch_language_modeling_bert_large_uncased', - input_options : '{}', // this option is not required for RunInference tests. - influx_measurement : 'torch_language_modeling_bert_large_uncased', - influx_db_name : InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influx_hostname : InfluxDBCredentialsHelper.InfluxDBHostUrl, - device : 'CPU', - input_file : 'gs://apache-beam-ml/testing/inputs/sentences_50k.txt', - bert_tokenizer : 'bert-large-uncased', - model_state_dict_path : 'gs://apache-beam-ml/models/huggingface.BertForMaskedLM.bert-large-uncased.pth', - output : 'gs://temp-storage-for-end-to-end-tests/torch/result_bert_large_uncased' + now + '.txt' - ] - ], - [ - title : 'Pytorch Imagenet Classification with Resnet 152 with Tesla T4 GPU', - test : 'apache_beam.testing.benchmarks.inference.pytorch_image_classification_benchmarks', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - job_name : 'benchmark-tests-pytorch-imagenet-python-gpu' + now, - project : 'apache-beam-testing', - region : 'us-central1', - machine_type : 'n1-standard-2', - num_workers : 75, // this could be lower as the quota for the apache-beam-testing project is 32 T4 GPUs as of November 28th, 2022. - disk_size_gb : 50, - autoscaling_algorithm : 'NONE', - staging_location : 'gs://temp-storage-for-perf-tests/loadtests', - temp_location : 'gs://temp-storage-for-perf-tests/loadtests', - requirements_file : 'apache_beam/ml/inference/torch_tests_requirements.txt', - publish_to_big_query : true, - metrics_dataset : 'beam_run_inference', - metrics_table : 'torch_inference_imagenet_results_resnet152_tesla_t4', - input_options : '{}', // this option is not required for RunInference tests. - influx_measurement : 'torch_inference_imagenet_resnet152_tesla_t4', - influx_db_name : InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influx_hostname : InfluxDBCredentialsHelper.InfluxDBHostUrl, - pretrained_model_name : 'resnet152', - device : 'GPU', - experiments : 'worker_accelerator=type:nvidia-tesla-t4;count:1;install-nvidia-driver', - sdk_container_image : 'us.gcr.io/apache-beam-testing/python-postcommit-it/tensor_rt:latest', - input_file : 'gs://apache-beam-ml/testing/inputs/openimage_50k_benchmark.txt', - model_state_dict_path : 'gs://apache-beam-ml/models/torchvision.models.resnet152.pth', - output : 'gs://temp-storage-for-end-to-end-tests/torch/result_resnet152_gpu' + now + '.txt' - ] - ], - ] -} - -def loadTestJob = { scope -> - List<Map> testScenarios = loadTestConfigurations() - for (Map testConfig: testScenarios){ - commonJobProperties.setTopLevelMainJobProperties(scope, 'master', 180) - loadTestsBuilder.loadTest(scope, testConfig.title, testConfig.runner, CommonTestProperties.SDK.PYTHON, testConfig.pipelineOptions, testConfig.test, null, - testConfig.pipelineOptions.requirements_file, RUN_INFERENCE_TEST_PYTHON_VERSION) - } -} - -PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_Inference_Python_Benchmarks_Dataflow', - 'Run Inference Benchmarks', - 'RunInference benchmarks on Dataflow(\"Run Inference Benchmarks"\"")', - this - ) { - loadTestJob(delegate) - } - -CronJobBuilder.cronJob( - 'beam_Inference_Python_Benchmarks_Dataflow', 'H H * * *', - this - ) { - loadTestJob(delegate) - } diff --git a/.test-infra/jenkins/job_LoadTests_CoGBK_Java.groovy b/.test-infra/jenkins/job_LoadTests_CoGBK_Java.groovy deleted file mode 100644 index 373e11e5a7af..000000000000 --- a/.test-infra/jenkins/job_LoadTests_CoGBK_Java.groovy +++ /dev/null @@ -1,215 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonJobProperties as commonJobProperties -import CommonTestProperties -import LoadTestsBuilder as loadTestsBuilder -import PhraseTriggeringPostCommitBuilder -import CronJobBuilder -import InfluxDBCredentialsHelper - -def loadTestConfigurations = { mode, isStreaming -> - [ - [ - title : 'Load test: CoGBK 2GB 100 byte records - single key', - test : 'org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - project : 'apache-beam-testing', - region : 'us-central1', - appName : "load_tests_Java_Dataflow_${mode}_CoGBK_1", - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${mode}_cogbk_1", - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 20000000, - "keySizeBytes": 10, - "valueSizeBytes": 90, - "numHotKeys": 1 - } - """.trim().replaceAll("\\s", ""), - coSourceOptions : """ - { - "numRecords": 2000000, - "keySizeBytes": 10, - "valueSizeBytes": 90, - "numHotKeys": 1000 - } - """.trim().replaceAll("\\s", ""), - iterations : 1, - numWorkers : 5, - autoscalingAlgorithm : "NONE", - streaming : isStreaming - ] - ], - [ - title : 'Load test: CoGBK 2GB 100 byte records - multiple keys', - test : 'org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - project : 'apache-beam-testing', - region : 'us-central1', - appName : "load_tests_Java_Dataflow_${mode}_CoGBK_2", - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${mode}_cogbk_2", - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 20000000, - "keySizeBytes": 10, - "valueSizeBytes": 90, - "numHotKeys": 5 - } - """.trim().replaceAll("\\s", ""), - coSourceOptions : """ - { - "numRecords": 2000000, - "keySizeBytes": 10, - "valueSizeBytes": 90, - "numHotKeys": 1000 - } - """.trim().replaceAll("\\s", ""), - iterations : 1, - numWorkers : 5, - autoscalingAlgorithm : "NONE", - streaming : isStreaming - ] - ], - [ - - title : 'Load test: CoGBK 2GB reiteration 10kB value', - test : 'org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - project : 'apache-beam-testing', - region : 'us-central1', - appName : "load_tests_Java_Dataflow_${mode}_CoGBK_3", - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${mode}_cogbk_3", - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 2000000, - "keySizeBytes": 10, - "valueSizeBytes": 90, - "numHotKeys": 200000 - } - """.trim().replaceAll("\\s", ""), - coSourceOptions : """ - { - "numRecords": 2000000, - "keySizeBytes": 10, - "valueSizeBytes": 90, - "numHotKeys": 1000 - } - """.trim().replaceAll("\\s", ""), - iterations : 4, - numWorkers : 5, - autoscalingAlgorithm : "NONE", - streaming : isStreaming - ] - - ], - [ - title : 'Load test: CoGBK 2GB reiteration 2MB value', - test : 'org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - project : 'apache-beam-testing', - region : 'us-central1', - appName : "load_tests_Java_Dataflow_${mode}_CoGBK_4", - tempLocation : 'gs://temp-storage-for-perf-tests/loadtests', - influxMeasurement : "java_${mode}_cogbk_4", - publishToInfluxDB : true, - sourceOptions : """ - { - "numRecords": 2000000, - "keySizeBytes": 10, - "valueSizeBytes": 90, - "numHotKeys": 1000 - } - """.trim().replaceAll("\\s", ""), - coSourceOptions : """ - { - "numRecords": 2000000, - "keySizeBytes": 10, - "valueSizeBytes": 90, - "numHotKeys": 1000 - } - """.trim().replaceAll("\\s", ""), - iterations : 4, - numWorkers : 5, - autoscalingAlgorithm : "NONE", - streaming : isStreaming - ] - ] - ].each { test -> test.pipelineOptions.putAll(additionalPipelineArgs) } -} - -def streamingLoadTestJob = { scope, triggeringContext -> - scope.description('Runs Java CoGBK load tests on Dataflow runner in streaming mode') - commonJobProperties.setTopLevelMainJobProperties(scope, 'master', 240) - - for (testConfiguration in loadTestConfigurations('streaming', true)) { - testConfiguration.pipelineOptions << [inputWindowDurationSec: 1200, coInputWindowDurationSec: 1200] - loadTestsBuilder.loadTest(scope, testConfiguration.title, testConfiguration.runner, CommonTestProperties.SDK.JAVA, testConfiguration.pipelineOptions, testConfiguration.test) - } -} - -CronJobBuilder.cronJob('beam_LoadTests_Java_CoGBK_Dataflow_Streaming', 'H H * * *', this) { - additionalPipelineArgs = [ - influxDatabase: InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influxHost: InfluxDBCredentialsHelper.InfluxDBHostUrl, - ] - streamingLoadTestJob(delegate, CommonTestProperties.TriggeringContext.POST_COMMIT) -} - -PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_LoadTests_Java_CoGBK_Dataflow_Streaming', - 'Run Load Tests Java CoGBK Dataflow Streaming', - 'Load Tests Java CoGBK Dataflow Streaming suite', - this - ) { - additionalPipelineArgs = [:] - streamingLoadTestJob(delegate, CommonTestProperties.TriggeringContext.PR) - } - - -def batchLoadTestJob = { scope, triggeringContext -> - - loadTestsBuilder.loadTests(scope, CommonTestProperties.SDK.JAVA, loadTestConfigurations('batch', false), "CoGBK", "batch") -} - -CronJobBuilder.cronJob('beam_LoadTests_Java_CoGBK_Dataflow_Batch', 'H H * * *', this) { - additionalPipelineArgs = [ - influxDatabase: InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influxHost: InfluxDBCredentialsHelper.InfluxDBHostUrl, - ] - batchLoadTestJob(delegate, CommonTestProperties.TriggeringContext.POST_COMMIT) -} - -PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_LoadTests_Java_CoGBK_Dataflow_Batch', - 'Run Load Tests Java CoGBK Dataflow Batch', - 'Load Tests Java CoGBK Dataflow Batch suite', - this - ) { - additionalPipelineArgs = [:] - batchLoadTestJob(delegate, CommonTestProperties.TriggeringContext.PR) - } diff --git a/.test-infra/jenkins/job_LoadTests_Combine_Flink_Go.groovy b/.test-infra/jenkins/job_LoadTests_Combine_Flink_Go.groovy deleted file mode 100644 index 9b8adc732f98..000000000000 --- a/.test-infra/jenkins/job_LoadTests_Combine_Flink_Go.groovy +++ /dev/null @@ -1,139 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonJobProperties as commonJobProperties -import CommonTestProperties -import LoadTestsBuilder as loadTestsBuilder -import PhraseTriggeringPostCommitBuilder -import Flink -import InfluxDBCredentialsHelper - -import static LoadTestsBuilder.DOCKER_BEAM_JOBSERVER -import static LoadTestsBuilder.GO_SDK_CONTAINER - - -String now = new Date().format('MMddHHmmss', TimeZone.getTimeZone('UTC')) - -def batchScenarios = { - [ - [ - title : 'Combine Go Load test: 2GB of 10B records', - test : 'combine', - runner : CommonTestProperties.Runner.FLINK, - pipelineOptions: [ - job_name : "load-tests-go-flink-batch-combine-1-${now}", - influx_namespace : 'flink', - influx_measurement : 'go_batch_combine_1', - input_options : '\'{' + - '"num_records": 200000000,' + - '"key_size": 1,' + - '"value_size": 9}\'', - fanout : 1, - top_count : 20, - parallelism : 5, - endpoint : 'localhost:8099', - environment_type : 'DOCKER', - environment_config : GO_SDK_CONTAINER, - ] - ], - [ - title : 'Combine Go Load test: fanout 4 times with 2GB 10-byte records total', - test : 'combine', - runner : CommonTestProperties.Runner.FLINK, - pipelineOptions: [ - job_name : "load-tests-go-flink-batch-combine-4-${now}", - influx_namespace : 'flink', - influx_measurement : 'go_batch_combine_4', - input_options : '\'{' + - '"num_records": 5000000,' + - '"key_size": 10,' + - '"value_size": 90}\'', - fanout : 4, - top_count : 20, - parallelism : 16, - endpoint : 'localhost:8099', - environment_type : 'DOCKER', - environment_config : GO_SDK_CONTAINER, - ] - ], - [ - title : 'Combine Go Load test: fanout 8 times with 2GB 10-byte records total', - test : 'combine', - runner : CommonTestProperties.Runner.FLINK, - pipelineOptions: [ - job_name : "load-tests-go-flink-batch-combine-5-${now}", - influx_namespace : 'flink', - influx_measurement : 'go_batch_combine_5', - fanout : 8, - top_count : 20, - parallelism : 16, - input_options : '\'{' + - '"num_records": 2500000,' + - '"key_size": 10,' + - '"value_size": 90}\'', - endpoint : 'localhost:8099', - environment_type : 'DOCKER', - environment_config : GO_SDK_CONTAINER, - ] - ], - ].each { test -> test.pipelineOptions.putAll(additionalPipelineArgs) } -} - -def loadTestJob = { scope, triggeringContext, mode -> - Map<Integer, List> testScenariosByParallelism = batchScenarios().groupBy { test -> - test.pipelineOptions.parallelism - } - Integer initialParallelism = testScenariosByParallelism.keySet().iterator().next() - List initialScenarios = testScenariosByParallelism.remove(initialParallelism) - - def flink = new Flink(scope, "beam_LoadTests_Go_Combine_Flink_${mode.capitalize()}") - flink.setUp( - [ - GO_SDK_CONTAINER - ], - initialParallelism, - "${DOCKER_BEAM_JOBSERVER}/beam_flink${CommonTestProperties.getFlinkVersion()}_job_server:latest") - - // Execute all scenarios connected with initial parallelism. - loadTestsBuilder.loadTests(scope, CommonTestProperties.SDK.GO, initialScenarios, 'combine', mode) - - // Execute the rest of scenarios. - testScenariosByParallelism.each { parallelism, scenarios -> - flink.scaleCluster(parallelism) - loadTestsBuilder.loadTests(scope, CommonTestProperties.SDK.GO, scenarios, 'combine', mode) - } -} - -PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_LoadTests_Go_Combine_Flink_Batch', - 'Run Load Tests Go Combine Flink Batch', - 'Load Tests Go Combine Flink Batch suite', - this - ) { - additionalPipelineArgs = [:] - loadTestJob(delegate, CommonTestProperties.TriggeringContext.PR, 'batch') - } - -CronJobBuilder.cronJob('beam_LoadTests_Go_Combine_Flink_Batch', 'H H * * *', this) { - additionalPipelineArgs = [ - influx_db_name: InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influx_hostname: InfluxDBCredentialsHelper.InfluxDBHostUrl, - ] - // TODO(BEAM): Fix this test. - loadTestJob(delegate, CommonTestProperties.TriggeringContext.POST_COMMIT, 'batch') -} diff --git a/.test-infra/jenkins/job_LoadTests_Combine_Flink_Python.groovy b/.test-infra/jenkins/job_LoadTests_Combine_Flink_Python.groovy deleted file mode 100644 index 54b92fdade26..000000000000 --- a/.test-infra/jenkins/job_LoadTests_Combine_Flink_Python.groovy +++ /dev/null @@ -1,186 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonJobProperties as commonJobProperties -import CommonTestProperties -import LoadTestsBuilder as loadTestsBuilder -import PhraseTriggeringPostCommitBuilder -import Flink -import InfluxDBCredentialsHelper - -import static LoadTestsBuilder.DOCKER_CONTAINER_REGISTRY -import static LoadTestsBuilder.DOCKER_BEAM_JOBSERVER -import static LoadTestsBuilder.DOCKER_BEAM_SDK_IMAGE - -String now = new Date().format("MMddHHmmss", TimeZone.getTimeZone('UTC')) - -// TODO(https://github.com/apache/beam/issues/20402): Skipping some cases because they are too slow. -def TESTS_TO_SKIP = [ - 'load-tests-python-flink-streaming-combine-1', -] - -def loadTestConfigurations = { mode, datasetName -> - [ - [ - title : 'Combine Python Load test: 2GB 10 byte records', - test : 'apache_beam.testing.load_tests.combine_test', - runner : CommonTestProperties.Runner.PORTABLE, - pipelineOptions: [ - job_name : "load-tests-python-flink-${mode}-combine-1-${now}", - project : 'apache-beam-testing', - publish_to_big_query: true, - metrics_dataset : datasetName, - metrics_table : "python_flink_${mode}_combine_1", - influx_measurement : "python_${mode}_combine_1", - input_options : '\'{' + - '"num_records": 200000000,' + - '"key_size": 1,' + - '"value_size": 9,' + - '"algorithm": "lcg"}\'', - parallelism : 5, - job_endpoint : 'localhost:8099', - environment_type : 'DOCKER', - environment_config : "${DOCKER_CONTAINER_REGISTRY}/${DOCKER_BEAM_SDK_IMAGE}", - top_count : 20, - ] - ], - [ - title : 'Combine Python Load test: 2GB Fanout 4', - test : 'apache_beam.testing.load_tests.combine_test', - runner : CommonTestProperties.Runner.PORTABLE, - pipelineOptions: [ - job_name : "load-tests-python-flink-${mode}-combine-4-${now}", - project : 'apache-beam-testing', - publish_to_big_query: true, - metrics_dataset : datasetName, - metrics_table : "python_flink_${mode}_combine_4", - influx_measurement : "python_${mode}_combine_4", - input_options : '\'{' + - '"num_records": 5000000,' + - '"key_size": 10,' + - '"value_size": 90,' + - '"algorithm": "lcg"}\'', - parallelism : 16, - job_endpoint : 'localhost:8099', - environment_type : 'DOCKER', - environment_config : "${DOCKER_CONTAINER_REGISTRY}/${DOCKER_BEAM_SDK_IMAGE}", - fanout : 4, - top_count : 20, - ] - ], - [ - title : 'Combine Python Load test: 2GB Fanout 8', - test : 'apache_beam.testing.load_tests.combine_test', - runner : CommonTestProperties.Runner.PORTABLE, - pipelineOptions: [ - job_name : "load-tests-python-flink-${mode}-combine-5-${now}", - project : 'apache-beam-testing', - publish_to_big_query: true, - metrics_dataset : datasetName, - metrics_table : "python_flink_${mode}_combine_5", - influx_measurement : "python_${mode}_combine_5", - input_options : '\'{' + - '"num_records": 2500000,' + - '"key_size": 10,' + - '"value_size": 90,' + - '"algorithm": "lcg"}\'', - parallelism : 16, - job_endpoint : 'localhost:8099', - environment_type : 'DOCKER', - environment_config : "${DOCKER_CONTAINER_REGISTRY}/${DOCKER_BEAM_SDK_IMAGE}", - fanout : 8, - top_count : 20, - ] - ] - ] - .each { test -> test.pipelineOptions.putAll(additionalPipelineArgs) } - .each { test -> (mode != 'streaming') ?: addStreamingOptions(test) } - .collectMany { test -> - TESTS_TO_SKIP.any { element -> test.pipelineOptions.job_name.startsWith(element) } ? []: [test] - } -} - -def addStreamingOptions(test) { - test.pipelineOptions << [streaming: null, - use_stateful_load_generator: null - ] -} - -def loadTestJob = { scope, triggeringContext, mode -> - def datasetName = loadTestsBuilder.getBigQueryDataset('load_test', triggeringContext) - List<Map> testScenarios = loadTestConfigurations(mode, datasetName) - Map<Integer, List> testScenariosByParallelism = testScenarios.groupBy { test -> - test.pipelineOptions.parallelism - } - Integer initialParallelism = testScenariosByParallelism.keySet().iterator().next() - List initialScenarios = testScenariosByParallelism.remove(initialParallelism) - - def flink = new Flink(scope, "beam_LoadTests_Python_Combine_Flink_${mode.capitalize()}") - flink.setUp( - [ - "${DOCKER_CONTAINER_REGISTRY}/${DOCKER_BEAM_SDK_IMAGE}" - ], - initialParallelism, - "${DOCKER_BEAM_JOBSERVER}/beam_flink${CommonTestProperties.getFlinkVersion()}_job_server:latest") - - // Execute all scenarios connected with initial parallelism. - loadTestsBuilder.loadTests(scope, CommonTestProperties.SDK.PYTHON, initialScenarios, 'Combine', mode) - - // Execute the rest of scenarios. - testScenariosByParallelism.each { parallelism, scenarios -> - flink.scaleCluster(parallelism) - loadTestsBuilder.loadTests(scope, CommonTestProperties.SDK.PYTHON, scenarios, 'Combine', mode) - } -} - -PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_LoadTests_Python_Combine_Flink_Batch', - 'Run Load Tests Python Combine Flink Batch', - 'Load Tests Python Combine Flink Batch suite', - this - ) { - additionalPipelineArgs = [:] - loadTestJob(delegate, CommonTestProperties.TriggeringContext.PR, 'batch') - } - -CronJobBuilder.cronJob('beam_LoadTests_Python_Combine_Flink_Batch', 'H H * * *', this) { - additionalPipelineArgs = [ - influx_db_name: InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influx_hostname: InfluxDBCredentialsHelper.InfluxDBHostUrl, - ] - loadTestJob(delegate, CommonTestProperties.TriggeringContext.POST_COMMIT, 'batch') -} - -PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_LoadTests_Python_Combine_Flink_Streaming', - 'Run Load Tests Python Combine Flink Streaming', - 'Load Tests Python Combine Flink Streaming suite', - this - ) { - additionalPipelineArgs = [:] - loadTestJob(delegate, CommonTestProperties.TriggeringContext.PR, 'streaming') - } - -CronJobBuilder.cronJob('beam_LoadTests_Python_Combine_Flink_Streaming', 'H H * * *', this) { - additionalPipelineArgs = [ - influx_db_name: InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influx_hostname: InfluxDBCredentialsHelper.InfluxDBHostUrl, - ] - // TODO(BEAM): Fix this test. - loadTestJob(delegate, CommonTestProperties.TriggeringContext.POST_COMMIT, 'streaming') -} diff --git a/.test-infra/jenkins/job_LoadTests_Combine_Python.groovy b/.test-infra/jenkins/job_LoadTests_Combine_Python.groovy deleted file mode 100644 index b1adca8e8f19..000000000000 --- a/.test-infra/jenkins/job_LoadTests_Combine_Python.groovy +++ /dev/null @@ -1,154 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonJobProperties as commonJobProperties -import LoadTestsBuilder as loadTestsBuilder -import PhraseTriggeringPostCommitBuilder -import InfluxDBCredentialsHelper - -def now = new Date().format("MMddHHmmss", TimeZone.getTimeZone('UTC')) - -def loadTestConfigurations = { datasetName, mode -> - [ - [ - title : 'Combine Python Load test: 2GB 10 byte records', - test : 'apache_beam.testing.load_tests.combine_test', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - job_name : "load-tests-python-dataflow-${mode}-combine-1-${now}", - project : 'apache-beam-testing', - region : 'us-central1', - temp_location : 'gs://temp-storage-for-perf-tests/smoketests', - publish_to_big_query : true, - metrics_dataset : datasetName, - metrics_table : "python_dataflow_${mode}_combine_1", - influx_measurement : "python_${mode}_combine_1", - input_options : '\'{' + - '"num_records": 200000000,' + - '"key_size": 1,' + - '"value_size": 9,' + - '"algorithm": "lcg"}\'', - num_workers : 5, - autoscaling_algorithm: "NONE", - top_count : 20, - ] - ], - [ - title : 'Combine Python Load test: 2GB Fanout 4', - test : 'apache_beam.testing.load_tests.combine_test', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - job_name : "load-tests-python-dataflow-${mode}-combine-4-${now}", - project : 'apache-beam-testing', - region : 'us-central1', - temp_location : 'gs://temp-storage-for-perf-tests/smoketests', - publish_to_big_query : true, - metrics_dataset : datasetName, - metrics_table : "python_dataflow_${mode}_combine_4", - influx_measurement : "python_${mode}_combine_4", - input_options : '\'{' + - '"num_records": 5000000,' + - '"key_size": 10,' + - '"value_size": 90,' + - '"algorithm": "lcg"}\'', - num_workers : 16, - autoscaling_algorithm: "NONE", - fanout : 4, - top_count : 20, - ] - ], - [ - title : 'Combine Python Load test: 2GB Fanout 8', - test : 'apache_beam.testing.load_tests.combine_test', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - job_name : "load-tests-python-dataflow-${mode}-combine-5-${now}", - project : 'apache-beam-testing', - region : 'us-central1', - temp_location : 'gs://temp-storage-for-perf-tests/smoketests', - publish_to_big_query : true, - metrics_dataset : datasetName, - metrics_table : "python_dataflow_${mode}_combine_5", - influx_measurement : "python_${mode}_combine_5", - input_options : '\'{' + - '"num_records": 2500000,' + - '"key_size": 10,' + - '"value_size": 90,' + - '"algorithm": "lcg"}\'', - num_workers : 16, - autoscaling_algorithm: "NONE", - fanout : 8, - top_count : 20, - ] - ], - ] - .each { test -> test.pipelineOptions.putAll(additionalPipelineArgs) } - .each{ test -> (mode != 'streaming') ?: addStreamingOptions(test) } -} - -def addStreamingOptions(test){ - test.pipelineOptions << [streaming: null, - experiments: "use_runner_v2" - ] -} - -def loadTestJob = { scope, triggeringContext, jobType -> - scope.description("Runs Python Combine load tests on Dataflow runner in ${jobType} mode") - commonJobProperties.setTopLevelMainJobProperties(scope, 'master', 720) - - def datasetName = loadTestsBuilder.getBigQueryDataset('load_test', triggeringContext) - for (testConfiguration in loadTestConfigurations(datasetName, jobType)) { - loadTestsBuilder.loadTest(scope, testConfiguration.title, testConfiguration.runner, CommonTestProperties.SDK.PYTHON, testConfiguration.pipelineOptions, testConfiguration.test) - } -} - -PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_LoadTests_Python_Combine_Dataflow_Batch', - 'Run Load Tests Python Combine Dataflow Batch', - 'Load Tests Python Combine Dataflow Batch suite', - this - ) { - additionalPipelineArgs = [:] - loadTestJob(delegate, CommonTestProperties.TriggeringContext.PR, "batch") - } - -CronJobBuilder.cronJob('beam_LoadTests_Python_Combine_Dataflow_Batch', 'H H * * *', this) { - additionalPipelineArgs = [ - influx_db_name: InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influx_hostname: InfluxDBCredentialsHelper.InfluxDBHostUrl, - ] - loadTestJob(delegate, CommonTestProperties.TriggeringContext.POST_COMMIT, "batch") -} - -PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_LoadTests_Python_Combine_Dataflow_Streaming', - 'Run Load Tests Python Combine Dataflow Streaming', - 'Load Tests Python Combine Dataflow Streaming suite', - this - ) { - additionalPipelineArgs = [:] - loadTestJob(delegate, CommonTestProperties.TriggeringContext.PR, "streaming") - } - -CronJobBuilder.cronJob('beam_LoadTests_Python_Combine_Dataflow_Streaming', 'H H * * *', this) { - additionalPipelineArgs = [ - influx_db_name: InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influx_hostname: InfluxDBCredentialsHelper.InfluxDBHostUrl, - ] - loadTestJob(delegate, CommonTestProperties.TriggeringContext.POST_COMMIT, "streaming") -} diff --git a/.test-infra/jenkins/job_LoadTests_GBK_Flink_Go.groovy b/.test-infra/jenkins/job_LoadTests_GBK_Flink_Go.groovy deleted file mode 100644 index d5a6910b2a0d..000000000000 --- a/.test-infra/jenkins/job_LoadTests_GBK_Flink_Go.groovy +++ /dev/null @@ -1,231 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonJobProperties as commonJobProperties -import CommonTestProperties -import LoadTestsBuilder as loadTestsBuilder -import PhraseTriggeringPostCommitBuilder -import Flink -import InfluxDBCredentialsHelper - -import static LoadTestsBuilder.DOCKER_BEAM_JOBSERVER -import static LoadTestsBuilder.GO_SDK_CONTAINER - -String now = new Date().format('MMddHHmmss', TimeZone.getTimeZone('UTC')) - -// TODO(https://github.com/apache/beam/issues/20146): Skipping some cases because they are too slow or have memory errors. -def TESTS_TO_SKIP = [ - 'load-tests-go-flink-batch-gbk-7', -] - -def batchScenarios = { - [ - [ - title : 'Group By Key Go Load test: 2GB of 10B records', - test : 'group_by_key', - runner : CommonTestProperties.Runner.FLINK, - pipelineOptions: [ - job_name : "load-tests-go-flink-batch-gbk-1-${now}", - influx_namespace : 'flink', - influx_measurement : 'go_batch_gbk_1', - input_options : '\'{' + - '"num_records": 200000000,' + - '"key_size": 1,' + - '"value_size": 9}\'', - iterations : 1, - fanout : 1, - parallelism : 5, - endpoint : 'localhost:8099', - environment_type : 'DOCKER', - environment_config : GO_SDK_CONTAINER, - ] - ], - [ - title : 'Group By Key Go Load test: 2GB of 100B records', - test : 'group_by_key', - runner : CommonTestProperties.Runner.FLINK, - pipelineOptions: [ - job_name : "load-tests-go-flink-batch-gbk-2-${now}", - influx_namespace : 'flink', - influx_measurement : 'go_batch_gbk_2', - input_options : '\'{' + - '"num_records": 20000000,' + - '"key_size": 10,' + - '"value_size": 90}\'', - iterations : 1, - fanout : 1, - parallelism : 5, - endpoint : 'localhost:8099', - environment_type : 'DOCKER', - environment_config : GO_SDK_CONTAINER, - ] - ], - [ - title : 'Group By Key Go Load test: 2GB of 100kB records', - test : 'group_by_key', - runner : CommonTestProperties.Runner.FLINK, - pipelineOptions: [ - job_name : "load-tests-go-flink-batch-gbk-3-${now}", - influx_namespace : 'flink', - influx_measurement : 'go_batch_gbk_3', - iterations : 1, - fanout : 1, - parallelism : 5, - input_options : '\'{' + - '"num_records": 20000,' + - '"key_size": 10000,' + - '"value_size": 90000}\'', - endpoint : 'localhost:8099', - environment_type : 'DOCKER', - environment_config : GO_SDK_CONTAINER, - ] - ], - [ - title : 'Group By Key Go Load test: fanout 4 times with 2GB 10-byte records total', - test : 'group_by_key', - runner : CommonTestProperties.Runner.FLINK, - pipelineOptions: [ - job_name : "load-tests-go-flink-batch-gbk-4-${now}", - influx_namespace : 'flink', - influx_measurement : 'go_batch_gbk_4', - iterations : 1, - fanout : 4, - parallelism : 16, - input_options : '\'{' + - '"num_records": 5000000,' + - '"key_size": 10,' + - '"value_size": 90}\'', - endpoint : 'localhost:8099', - environment_type : 'DOCKER', - environment_config : GO_SDK_CONTAINER, - ] - ], - [ - title : 'Group By Key Go Load test: fanout 8 times with 2GB 10-byte records total', - test : 'group_by_key', - runner : CommonTestProperties.Runner.FLINK, - pipelineOptions: [ - job_name : "load-tests-go-flink-batch-gbk-5-${now}", - influx_namespace : 'flink', - influx_measurement : 'go_batch_gbk_5', - iterations : 1, - fanout : 8, - parallelism : 16, - input_options : '\'{' + - '"num_records": 2500000,' + - '"key_size": 10,' + - '"value_size": 90}\'', - endpoint : 'localhost:8099', - environment_type : 'DOCKER', - environment_config : GO_SDK_CONTAINER, - ] - ], - [ - title : 'Group By Key Go Load test: reiterate 4 times 10kB values', - test : 'group_by_key', - runner : CommonTestProperties.Runner.FLINK, - pipelineOptions: [ - job_name : "load-tests-go-flink-batch-gbk-6-${now}", - influx_namespace : 'flink', - influx_measurement : 'go_batch_gbk_6', - iterations : 4, - fanout : 1, - parallelism : 5, - input_options : '\'{' + - '"num_records": 20000000,' + - '"key_size": 10,' + - '"value_size": 90,' + - '"num_hot_keys": 200,' + - '"hot_key_fraction": 1}\'', - endpoint : 'localhost:8099', - environment_type : 'DOCKER', - environment_config : GO_SDK_CONTAINER, - ] - ], - [ - title : 'Group By Key Go Load test: reiterate 4 times 2MB values', - test : 'group_by_key', - runner : CommonTestProperties.Runner.FLINK, - pipelineOptions: [ - job_name : "load-tests-go-flink-batch-gbk-7-${now}", - influx_namespace : 'flink', - influx_measurement : 'go_batch_gbk_7', - iterations : 4, - fanout : 1, - parallelism : 5, - input_options : '\'{' + - '"num_records": 20000000,' + - '"key_size": 10,' + - '"value_size": 90,' + - '"num_hot_keys": 10,' + - '"hot_key_fraction": 1}\'', - endpoint : 'localhost:8099', - environment_type : 'DOCKER', - environment_config : GO_SDK_CONTAINER, - ] - ], - ] - .each { test -> test.pipelineOptions.putAll(additionalPipelineArgs) } - .collectMany { test -> - TESTS_TO_SKIP.any { element -> test.pipelineOptions.job_name.startsWith(element) } ? []: [test] - } -} - -def loadTestJob = { scope, triggeringContext, mode -> - Map<Integer, List> testScenariosByParallelism = batchScenarios().groupBy { test -> - test.pipelineOptions.parallelism - } - Integer initialParallelism = testScenariosByParallelism.keySet().iterator().next() - List initialScenarios = testScenariosByParallelism.remove(initialParallelism) - - def flink = new Flink(scope, "beam_LoadTests_Go_GBK_Flink_${mode.capitalize()}") - flink.setUp( - [ - GO_SDK_CONTAINER - ], - initialParallelism, - "${DOCKER_BEAM_JOBSERVER}/beam_flink${CommonTestProperties.getFlinkVersion()}_job_server:latest") - - // Execute all scenarios connected with initial parallelism. - loadTestsBuilder.loadTests(scope, CommonTestProperties.SDK.GO, initialScenarios, 'group_by_key', mode) - - // Execute the rest of scenarios. - testScenariosByParallelism.each { parallelism, scenarios -> - flink.scaleCluster(parallelism) - loadTestsBuilder.loadTests(scope, CommonTestProperties.SDK.GO, scenarios, 'group_by_key', mode) - } -} - -PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_LoadTests_Go_GBK_Flink_Batch', - 'Run Load Tests Go GBK Flink Batch', - 'Load Tests Go GBK Flink Batch suite', - this - ) { - additionalPipelineArgs = [:] - loadTestJob(delegate, CommonTestProperties.TriggeringContext.PR, 'batch') - } - -CronJobBuilder.cronJob('beam_LoadTests_Go_GBK_Flink_Batch', 'H H * * *', this) { - additionalPipelineArgs = [ - influx_db_name: InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influx_hostname: InfluxDBCredentialsHelper.InfluxDBHostUrl, - ] - // TODO(BEAM): Fix this test. - loadTestJob(delegate, CommonTestProperties.TriggeringContext.POST_COMMIT, 'batch') -} diff --git a/.test-infra/jenkins/job_LoadTests_ParDo_Flink_Python.groovy b/.test-infra/jenkins/job_LoadTests_ParDo_Flink_Python.groovy deleted file mode 100644 index 4af2efd1be6a..000000000000 --- a/.test-infra/jenkins/job_LoadTests_ParDo_Flink_Python.groovy +++ /dev/null @@ -1,374 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonJobProperties as commonJobProperties -import CommonTestProperties -import LoadTestsBuilder as loadTestsBuilder -import PhraseTriggeringPostCommitBuilder -import Flink -import InfluxDBCredentialsHelper - -import static LoadTestsBuilder.DOCKER_CONTAINER_REGISTRY -import static LoadTestsBuilder.DOCKER_BEAM_JOBSERVER -import static LoadTestsBuilder.DOCKER_BEAM_SDK_IMAGE - -String now = new Date().format("MMddHHmmss", TimeZone.getTimeZone('UTC')) - -/** - * The test results for these load tests reside in BigQuery in the load_test/load_test_PRs table of the - * apache-beam-testing project. A dashboard is available here: - * https://apache-beam-testing.appspot.com/explore?dashboard=5751884853805056 - * - * For example: - * SELECT - * timestamp, value - * FROM - * apache-beam-testing.load_test_PRs.python_flink_batch_pardo_1 - * ORDER BY - * timestamp - * - * The following query has been been used to visualize the checkpoint results of python_flink_streaming_pardo_6: - * Select timestamp, min, sum/count as avg, max - * FROM ( - * SELECT - * timestamp, - * MAX(IF(metric LIKE "%\\_min\\_%", value, null)) min, - * MAX(IF(metric LIKE "%\\_sum\\_%", value, null)) sum, - * MAX(IF(metric LIKE "%\\_count\\_%", value, null)) count, - * MAX(IF(metric LIKE "%\\_max\\_%", value, null)) max - * FROM apache-beam-testing.load_test_PRs.python_flink_streaming_pardo_6 - * WHERE metric like "%loadgenerator/impulse%" - * GROUP BY test_id, timestamp - * ORDER BY timestamp - * ); - * - * Subsumed by the new Grafana dashboard: - * http://metrics.beam.apache.org/d/MOi-kf3Zk/pardo-load-tests?orgId=1&var-processingType=streaming&var-sdk=python - */ - -def batchScenarios = { datasetName -> - [ - [ - title : 'ParDo Python Load test: 20M 100 byte records 10 iterations', - test : 'apache_beam.testing.load_tests.pardo_test', - runner : CommonTestProperties.Runner.PORTABLE, - pipelineOptions: [ - job_name : 'load-tests-python-flink-batch-pardo-1-' + now, - project : 'apache-beam-testing', - publish_to_big_query : true, - metrics_dataset : datasetName, - metrics_table : 'python_flink_batch_pardo_1', - influx_measurement : 'python_batch_pardo_1', - input_options : '\'{' + - '"num_records": 20000000,' + - '"key_size": 10,' + - '"value_size": 90,' + - '"algorithm": "lcg"}\'', - iterations : 10, - number_of_counter_operations: 0, - number_of_counters : 0, - parallelism : 5, - job_endpoint : 'localhost:8099', - environment_type : 'DOCKER', - environment_config : "${DOCKER_CONTAINER_REGISTRY}/${DOCKER_BEAM_SDK_IMAGE}", - ] - ], - // TODO(BEAM-10270): Takes too long time to execute (currently more than 3 hours). Re-enable - // the test after its overhead is reduced. - // [ - // title : 'ParDo Python Load test: 20M 100 byte records 200 times', - // test : 'apache_beam.testing.load_tests.pardo_test', - // runner : CommonTestProperties.Runner.PORTABLE, - // pipelineOptions: [ - // job_name : 'load-tests-python-flink-batch-pardo-2-' + now, - // project : 'apache-beam-testing', - // publish_to_big_query : true, - // metrics_dataset : datasetName, - // metrics_table : 'python_flink_batch_pardo_2', - // influx_measurement : 'python_batch_pardo_2', - // input_options : '\'{' + - // '"num_records": 20000000,' + - // '"key_size": 10,' + - // '"value_size": 90,' + - // '"algorithm": "lcg"}\'', - // iterations : 200, - // number_of_counter_operations: 0, - // number_of_counters : 0, - // parallelism : 5, - // job_endpoint : 'localhost:8099', - // environment_type : 'DOCKER', - // environment_config : "${DOCKER_CONTAINER_REGISTRY}/${DOCKER_BEAM_SDK_IMAGE}", - // ] - // ], - [ - title : 'ParDo Python Load test: 20M 100 byte records 10 counters', - test : 'apache_beam.testing.load_tests.pardo_test', - runner : CommonTestProperties.Runner.PORTABLE, - pipelineOptions: [ - job_name : 'load-tests-python-flink-batch-pardo-3-' + now, - project : 'apache-beam-testing', - publish_to_big_query : true, - metrics_dataset : datasetName, - metrics_table : 'python_flink_batch_pardo_3', - influx_measurement : 'python_batch_pardo_3', - input_options : '\'{' + - '"num_records": 20000000,' + - '"key_size": 10,' + - '"value_size": 90,' + - '"algorithm": "lcg"}\'', - iterations : 1, - number_of_counter_operations: 10, - number_of_counters : 1, - parallelism : 5, - job_endpoint : 'localhost:8099', - environment_type : 'DOCKER', - environment_config : "${DOCKER_CONTAINER_REGISTRY}/${DOCKER_BEAM_SDK_IMAGE}", - ] - ], - [ - title : 'ParDo Python Load test: 20M 100 byte records 100 counters', - test : 'apache_beam.testing.load_tests.pardo_test', - runner : CommonTestProperties.Runner.PORTABLE, - pipelineOptions: [ - job_name : 'load-tests-python-flink-batch-pardo-4-' + now, - project : 'apache-beam-testing', - publish_to_big_query : true, - metrics_dataset : datasetName, - metrics_table : 'python_flink_batch_pardo_4', - influx_measurement : 'python_batch_pardo_4', - input_options : '\'{' + - '"num_records": 20000000,' + - '"key_size": 10,' + - '"value_size": 90,' + - '"algorithm": "lcg"}\'', - iterations : 1, - number_of_counter_operations: 100, - number_of_counters : 1, - parallelism : 5, - job_endpoint : 'localhost:8099', - environment_type : 'DOCKER', - environment_config : "${DOCKER_CONTAINER_REGISTRY}/${DOCKER_BEAM_SDK_IMAGE}", - ] - ], - ].each { test -> test.pipelineOptions.putAll(additionalPipelineArgs) } -} - -def streamingScenarios = { datasetName -> - [ - [ - title : 'ParDo Python Stateful Streaming Load test: 2M 100 byte records', - test : 'apache_beam.testing.load_tests.pardo_test', - runner : CommonTestProperties.Runner.PORTABLE, - pipelineOptions: [ - job_name : 'load-tests-python-flink-streaming-pardo-1-' + now, - project : 'apache-beam-testing', - publish_to_big_query : true, - metrics_dataset : datasetName, - // Keep the old name to not break the legacy dashboard - metrics_table : 'python_flink_streaming_pardo_5', - influx_measurement : 'python_streaming_pardo_1', - input_options : '\'{' + - '"num_records": 2000000,' + - '"key_size": 10,' + - '"value_size": 90,' + - '"algorithm": "lcg"}\'', - iterations : 10, - number_of_counter_operations: 0, - number_of_counters : 0, - parallelism : 5, - // Turn on streaming mode (flags are indicated with null values) - streaming : null, - stateful : null, - job_endpoint : 'localhost:8099', - environment_type : 'DOCKER', - environment_config : "${DOCKER_CONTAINER_REGISTRY}/${DOCKER_BEAM_SDK_IMAGE}", - use_stateful_load_generator: null, - ] - ], - [ - title : 'ParDo Python Load test: 20M 100 byte records 200 times', - test : 'apache_beam.testing.load_tests.pardo_test', - runner : CommonTestProperties.Runner.PORTABLE, - pipelineOptions: [ - job_name : 'load-tests-python-flink-streaming-pardo-2-' + now, - project : 'apache-beam-testing', - publish_to_big_query : true, - metrics_dataset : datasetName, - metrics_table : 'python_flink_streaming_pardo_2', - influx_measurement : 'python_streaming_pardo_2', - input_options : '\'{' + - '"num_records": 20000000,' + - '"key_size": 10,' + - '"value_size": 90,' + - '"algorithm": "lcg"}\'', - iterations : 200, - number_of_counter_operations: 0, - number_of_counters : 0, - parallelism : 5, - streaming : null, - job_endpoint : 'localhost:8099', - environment_type : 'DOCKER', - environment_config : "${DOCKER_CONTAINER_REGISTRY}/${DOCKER_BEAM_SDK_IMAGE}", - use_stateful_load_generator: null, - ] - ], - [ - title : 'ParDo Python Load test: 20M 100 byte records 10 counters', - test : 'apache_beam.testing.load_tests.pardo_test', - runner : CommonTestProperties.Runner.PORTABLE, - pipelineOptions: [ - job_name : 'load-tests-python-flink-streaming-pardo-3-' + now, - project : 'apache-beam-testing', - publish_to_big_query : true, - metrics_dataset : datasetName, - metrics_table : 'python_flink_streaming_pardo_3', - influx_measurement : 'python_streaming_pardo_3', - input_options : '\'{' + - '"num_records": 20000000,' + - '"key_size": 10,' + - '"value_size": 90,' + - '"algorithm": "lcg"}\'', - iterations : 1, - number_of_counter_operations: 10, - number_of_counters : 1, - parallelism : 5, - streaming : null, - job_endpoint : 'localhost:8099', - environment_type : 'DOCKER', - environment_config : "${DOCKER_CONTAINER_REGISTRY}/${DOCKER_BEAM_SDK_IMAGE}", - use_stateful_load_generator: null, - ] - ], - [ - title : 'ParDo Python Load test: 20M 100 byte records 100 counters', - test : 'apache_beam.testing.load_tests.pardo_test', - runner : CommonTestProperties.Runner.PORTABLE, - pipelineOptions: [ - job_name : 'load-tests-python-flink-streaming-pardo-4-' + now, - project : 'apache-beam-testing', - publish_to_big_query : true, - metrics_dataset : datasetName, - metrics_table : 'python_flink_streaming_pardo_4', - influx_measurement : 'python_streaming_pardo_4', - input_options : '\'{' + - '"num_records": 20000000,' + - '"key_size": 10,' + - '"value_size": 90,' + - '"algorithm": "lcg"}\'', - iterations : 1, - number_of_counter_operations: 100, - number_of_counters : 1, - parallelism : 5, - streaming : null, - job_endpoint : 'localhost:8099', - environment_type : 'DOCKER', - environment_config : "${DOCKER_CONTAINER_REGISTRY}/${DOCKER_BEAM_SDK_IMAGE}", - use_stateful_load_generator: null, - ] - ], - [ - title : 'ParDo Python Stateful Streaming with Checkpointing test: 2M 100 byte records', - test : 'apache_beam.testing.load_tests.pardo_test', - runner : CommonTestProperties.Runner.PORTABLE, - pipelineOptions: [ - job_name : 'load-tests-python-flink-streaming-pardo-6-' + now, - project : 'apache-beam-testing', - publish_to_big_query : true, - metrics_dataset : datasetName, - metrics_table : 'python_flink_streaming_pardo_6', - influx_measurement : 'python_streaming_pardo_6', - input_options : '\'{' + - '"num_records": 2000000,' + - '"key_size": 10,' + - '"value_size": 90,' + - '"algorithm": "lcg"}\'', - iterations : 5, - number_of_counter_operations: 10, - number_of_counters : 3, - parallelism : 5, - // Turn on streaming mode (flags are indicated with null values) - streaming : null, - stateful : null, - // Enable checkpointing every 10 seconds - checkpointing_interval : 10000, - // Report checkpointing stats to this namespace - report_checkpoint_duration : 'python_flink_streaming_pardo_6', - // Ensure that we can checkpoint the pipeline for at least 5 minutes to gather checkpointing stats - shutdown_sources_after_idle_ms: 300000, - job_endpoint : 'localhost:8099', - environment_type : 'DOCKER', - environment_config : "${DOCKER_CONTAINER_REGISTRY}/${DOCKER_BEAM_SDK_IMAGE}", - use_stateful_load_generator: null, - ] - ], - ].each { test -> test.pipelineOptions.putAll(additionalPipelineArgs) } -} - -def loadTestJob = { scope, triggeringContext, mode -> - def datasetName = loadTestsBuilder.getBigQueryDataset('load_test', triggeringContext) - def numberOfWorkers = 5 - List<Map> testScenarios = mode == 'batch' ? batchScenarios(datasetName) : streamingScenarios(datasetName) - - Flink flink = new Flink(scope, "beam_LoadTests_Python_ParDo_Flink_${mode.capitalize()}") - flink.setUp( - [ - "${DOCKER_CONTAINER_REGISTRY}/${DOCKER_BEAM_SDK_IMAGE}" - ], - numberOfWorkers, - "${DOCKER_BEAM_JOBSERVER}/beam_flink${CommonTestProperties.getFlinkVersion()}_job_server:latest") - - loadTestsBuilder.loadTests(scope, CommonTestProperties.SDK.PYTHON, testScenarios, 'ParDo', mode) -} - -PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_LoadTests_Python_ParDo_Flink_Batch', - 'Run Load Tests Python ParDo Flink Batch', - 'Load Tests Python ParDo Flink Batch suite', - this - ) { - additionalPipelineArgs = [:] - loadTestJob(delegate, CommonTestProperties.TriggeringContext.PR, 'batch') - } - -PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_LoadTests_Python_ParDo_Flink_Streaming', - 'Run Load Tests Python ParDo Flink Streaming', - 'Load Tests Python ParDo Flink Streaming suite', - this - ) { - additionalPipelineArgs = [:] - loadTestJob(delegate, CommonTestProperties.TriggeringContext.PR, 'streaming') - } - -CronJobBuilder.cronJob('beam_LoadTests_Python_ParDo_Flink_Batch', 'H H * * *', this) { - additionalPipelineArgs = [ - influx_db_name: InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influx_hostname: InfluxDBCredentialsHelper.InfluxDBHostUrl, - ] - // TODO(BEAM): Fix this test. - loadTestJob(delegate, CommonTestProperties.TriggeringContext.POST_COMMIT, 'batch') -} - -CronJobBuilder.cronJob('beam_LoadTests_Python_ParDo_Flink_Streaming', 'H H * * *', this) { - additionalPipelineArgs = [ - influx_db_name: InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influx_hostname: InfluxDBCredentialsHelper.InfluxDBHostUrl, - ] - // TODO(BEAM): Fix this test. - loadTestJob(delegate, CommonTestProperties.TriggeringContext.POST_COMMIT, 'streaming') -} diff --git a/.test-infra/jenkins/job_LoadTests_SideInput_Flink_Go.groovy b/.test-infra/jenkins/job_LoadTests_SideInput_Flink_Go.groovy deleted file mode 100644 index bd0eaa4f23e6..000000000000 --- a/.test-infra/jenkins/job_LoadTests_SideInput_Flink_Go.groovy +++ /dev/null @@ -1,105 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonTestProperties -import CommonJobProperties as commonJobProperties -import LoadTestsBuilder as loadTestsBuilder -import PhraseTriggeringPostCommitBuilder -import InfluxDBCredentialsHelper - -import static LoadTestsBuilder.DOCKER_BEAM_JOBSERVER -import static LoadTestsBuilder.GO_SDK_CONTAINER - -def now = new Date().format("MMddHHmmss", TimeZone.getTimeZone('UTC')) - -def batchScenarios = { - [ - [ - title : 'SideInput Go Load test: 400mb-1kb-10workers-1window-first-iterable', - test : 'sideinput', - runner : CommonTestProperties.Runner.FLINK, - pipelineOptions: [ - job_name : "load-tests-go-flink-batch-sideinput-3-${now}", - influx_namespace : 'flink', - influx_measurement : 'go_batch_sideinput_3', - input_options : '\'{' + - '"num_records": 400000,' + - '"key_size": 100,' + - '"value_size": 900}\'', - access_percentage : 1, - parallelism : 10, - endpoint : 'localhost:8099', - environment_type : 'DOCKER', - environment_config : GO_SDK_CONTAINER, - ] - ], - [ - title : 'SideInput Go Load test: 400mb-1kb-10workers-1window-iterable', - test : 'sideinput', - runner : CommonTestProperties.Runner.FLINK, - pipelineOptions: [ - job_name : "load-tests-go-flink-batch-sideinput-4-${now}", - influx_namespace : 'flink', - influx_measurement : 'go_batch_sideinput_4', - input_options : '\'{' + - '"num_records": 400000,' + - '"key_size": 100,' + - '"value_size": 900}\'', - parallelism : 10, - endpoint : 'localhost:8099', - environment_type : 'DOCKER', - environment_config : GO_SDK_CONTAINER, - ] - ], - ] - .each { test -> test.pipelineOptions.putAll(additionalPipelineArgs) } -} - -def loadTestJob = { scope, triggeringContext, mode -> - def numberOfWorkers = 10 - - Flink flink = new Flink(scope, "beam_LoadTests_Go_SideInput_Flink_${mode.capitalize()}") - flink.setUp( - [ - GO_SDK_CONTAINER - ], - numberOfWorkers, - "${DOCKER_BEAM_JOBSERVER}/beam_flink${CommonTestProperties.getFlinkVersion()}_job_server:latest") - - loadTestsBuilder.loadTests(scope, CommonTestProperties.SDK.GO, - batchScenarios(), 'SideInput', mode) -} - -PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_LoadTests_Go_SideInput_Flink_Batch', - 'Run Load Tests Go SideInput Flink Batch', - 'Load Tests Go SideInput Flink Batch suite', - this - ) { - additionalPipelineArgs = [:] - loadTestJob(delegate, CommonTestProperties.TriggeringContext.PR, 'batch') - } - -CronJobBuilder.cronJob('beam_LoadTests_Go_SideInput_Flink_Batch', 'H H * * *', this) { - additionalPipelineArgs = [ - influx_db_name: InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influx_hostname: InfluxDBCredentialsHelper.InfluxDBHostUrl, - ] - // TODO(BEAM): Fix this test. - loadTestJob(delegate, CommonTestProperties.TriggeringContext.POST_COMMIT, 'batch') -} diff --git a/.test-infra/jenkins/job_LoadTests_coGBK_Flink_Go.groovy b/.test-infra/jenkins/job_LoadTests_coGBK_Flink_Go.groovy deleted file mode 100644 index 8c7a60e724f9..000000000000 --- a/.test-infra/jenkins/job_LoadTests_coGBK_Flink_Go.groovy +++ /dev/null @@ -1,184 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonJobProperties as commonJobProperties -import CommonTestProperties -import LoadTestsBuilder as loadTestsBuilder -import PhraseTriggeringPostCommitBuilder -import Flink -import InfluxDBCredentialsHelper - -import static LoadTestsBuilder.DOCKER_BEAM_JOBSERVER -import static LoadTestsBuilder.GO_SDK_CONTAINER - -String now = new Date().format("MMddHHmmss", TimeZone.getTimeZone('UTC')) - -// TODO(BEAM-11398): Skipping the first test because it is too slow. -def TESTS_TO_SKIP = [ - 'load-tests-go-flink-batch-cogbk-1-', -] - -def batchScenarios = { - [ - [ - title : 'CoGroupByKey Go Load test: 2GB of 100B records with a single key', - test : 'cogbk', - runner : CommonTestProperties.Runner.FLINK, - pipelineOptions: [ - job_name : "load-tests-go-flink-batch-cogbk-1-${now}", - influx_measurement : 'go_batch_cogbk_1', - influx_namespace : 'flink', - input_options : '\'{' + - '"num_records": 20000000,' + - '"key_size": 10,' + - '"value_size": 90,' + - '"num_hot_keys": 1,' + - '"hot_key_fraction": 1}\'', - co_input_options : '\'{' + - '"num_records": 2000000,' + - '"key_size": 10,' + - '"value_size": 90,' + - '"num_hot_keys": 1000,' + - '"hot_key_fraction": 1}\'', - iterations : 1, - parallelism : 5, - endpoint : 'localhost:8099', - environment_type : 'DOCKER', - environment_config : GO_SDK_CONTAINER, - ] - ], - [ - title : 'CoGroupByKey Go Load test: 2GB of 100B records with multiple keys', - test : 'cogbk', - runner : CommonTestProperties.Runner.FLINK, - pipelineOptions: [ - job_name : "load-tests-go-flink-batch-cogbk-2-${now}", - influx_measurement : 'go_batch_cogbk_2', - influx_namespace : 'flink', - input_options : '\'{' + - '"num_records": 20000000,' + - '"key_size": 10,' + - '"value_size": 90,' + - '"num_hot_keys": 5,' + - '"hot_key_fraction": 1}\'', - co_input_options : '\'{' + - '"num_records": 2000000,' + - '"key_size": 10,' + - '"value_size": 90,' + - '"num_hot_keys": 1000,' + - '"hot_key_fraction": 1}\'', - iterations : 1, - parallelism : 5, - endpoint : 'localhost:8099', - environment_type : 'DOCKER', - environment_config : GO_SDK_CONTAINER, - ] - ], - [ - title : 'CoGroupByKey Go Load test: reiterate 4 times 10kB values', - test : 'cogbk', - runner : CommonTestProperties.Runner.FLINK, - pipelineOptions: [ - job_name : "load-tests-go-flink-batch-cogbk-3-${now}", - influx_measurement : 'go_batch_cogbk_3', - influx_namespace : 'flink', - input_options : '\'{' + - '"num_records": 20000000,' + - '"key_size": 10,' + - '"value_size": 90,' + - '"num_hot_keys": 200000,' + - '"hot_key_fraction": 1}\'', - co_input_options : '\'{' + - '"num_records": 2000000,' + - '"key_size": 10,' + - '"value_size": 90,' + - '"num_hot_keys": 1000,' + - '"hot_key_fraction": 1}\'', - iterations : 4, - parallelism : 5, - endpoint : 'localhost:8099', - environment_type : 'DOCKER', - environment_config : GO_SDK_CONTAINER, - ] - ], - [ - title : 'CoGroupByKey Go Load test: reiterate 4 times 2MB values', - test : 'cogbk', - runner : CommonTestProperties.Runner.FLINK, - pipelineOptions: [ - job_name : "load-tests-go-flink-batch-cogbk-4-${now}", - influx_measurement : 'go_batch_cogbk_4', - influx_namespace : 'flink', - input_options : '\'{' + - '"num_records": 20000000,' + - '"key_size": 10,' + - '"value_size": 90,' + - '"num_hot_keys": 1000,' + - '"hot_key_fraction": 1}\'', - co_input_options : '\'{' + - '"num_records": 2000000,' + - '"key_size": 10,' + - '"value_size": 90,' + - '"num_hot_keys": 1000,' + - '"hot_key_fraction": 1}\'', - iterations : 4, - parallelism : 5, - endpoint : 'localhost:8099', - environment_type : 'DOCKER', - environment_config : GO_SDK_CONTAINER, - ] - ], - ] - .each { test -> test.pipelineOptions.putAll(additionalPipelineArgs) } - .collectMany { test -> - TESTS_TO_SKIP.any { element -> test.pipelineOptions.job_name.startsWith(element) } ? []: [test] - } -} - -def loadTestJob = { scope, triggeringContext, mode -> - def numberOfWorkers = 5 - - def flink = new Flink(scope, "beam_LoadTests_Go_CoGBK_Flink_${mode.capitalize()}") - flink.setUp( - [ - GO_SDK_CONTAINER - ], - numberOfWorkers, - "${DOCKER_BEAM_JOBSERVER}/beam_flink${CommonTestProperties.getFlinkVersion()}_job_server:latest") - - loadTestsBuilder.loadTests(scope, CommonTestProperties.SDK.GO, batchScenarios(), 'CoGBK', mode) -} - -PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_LoadTests_Go_CoGBK_Flink_Batch', - 'Run Load Tests Go CoGBK Flink Batch', - 'Load Tests Go CoGBK Flink Batch suite', - this - ) { - additionalPipelineArgs = [:] - loadTestJob(delegate, CommonTestProperties.TriggeringContext.PR, 'batch') - } - -CronJobBuilder.cronJob('beam_LoadTests_Go_CoGBK_Flink_batch', 'H H * * *', this) { - additionalPipelineArgs = [ - influx_db_name: InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influx_hostname: InfluxDBCredentialsHelper.InfluxDBHostUrl, - ] - // TODO(BEAM): Fixe this test. - loadTestJob(delegate, CommonTestProperties.TriggeringContext.POST_COMMIT, 'batch') -} diff --git a/.test-infra/jenkins/job_LoadTests_coGBK_Python.groovy b/.test-infra/jenkins/job_LoadTests_coGBK_Python.groovy deleted file mode 100644 index 2e20a1369efd..000000000000 --- a/.test-infra/jenkins/job_LoadTests_coGBK_Python.groovy +++ /dev/null @@ -1,210 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonJobProperties as commonJobProperties -import CommonTestProperties -import LoadTestsBuilder as loadTestsBuilder -import PhraseTriggeringPostCommitBuilder -import CronJobBuilder -import InfluxDBCredentialsHelper - -def now = new Date().format("MMddHHmmss", TimeZone.getTimeZone('UTC')) - -def loadTestConfigurations = { mode, datasetName -> - [ - [ - title : 'CoGroupByKey Python Load test: 2GB of 100B records with a single key', - test : 'apache_beam.testing.load_tests.co_group_by_key_test', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - project : 'apache-beam-testing', - region : 'us-central1', - job_name : "load-tests-python-dataflow-${mode}-cogbk-1-${now}", - temp_location : 'gs://temp-storage-for-perf-tests/loadtests', - publish_to_big_query : true, - metrics_dataset : datasetName, - metrics_table : "python_dataflow_${mode}_cogbk_1", - influx_measurement : "python_${mode}_cogbk_1", - input_options : '\'{' + - '"num_records": 20000000,' + - '"key_size": 10,' + - '"value_size": 90,' + - '"num_hot_keys": 1,' + - '"hot_key_fraction": 1,' + - '"algorithm": "lcg"}\'', - co_input_options : '\'{' + - '"num_records": 2000000,' + - '"key_size": 10,' + - '"value_size": 90,' + - '"num_hot_keys": 1000,' + - '"hot_key_fraction": 1,' + - '"algorithm": "lcg"}\'', - iterations : 1, - num_workers : 5, - autoscaling_algorithm: 'NONE' - ] - ], - [ - title : 'CoGroupByKey Python Load test: 2GB of 100B records with multiple keys', - test : 'apache_beam.testing.load_tests.co_group_by_key_test', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - project : 'apache-beam-testing', - region : 'us-central1', - job_name : "load-tests-python-dataflow-${mode}-cogbk-2-${now}", - temp_location : 'gs://temp-storage-for-perf-tests/loadtests', - publish_to_big_query : true, - metrics_dataset : datasetName, - metrics_table : "python_dataflow_${mode}_cogbk_2", - influx_measurement : "python_${mode}_cogbk_2", - input_options : '\'{' + - '"num_records": 20000000,' + - '"key_size": 10,' + - '"value_size": 90,' + - '"num_hot_keys": 5,' + - '"hot_key_fraction": 1,' + - '"algorithm": "lcg"}\'', - co_input_options : '\'{' + - '"num_records": 2000000,' + - '"key_size": 10,' + - '"value_size": 90,' + - '"num_hot_keys": 1000,' + - '"hot_key_fraction": 1,' + - '"algorithm": "lcg"}\'', - iterations : 1, - num_workers : 5, - autoscaling_algorithm: 'NONE' - ] - ], - [ - title : 'CoGroupByKey Python Load test: reiterate 4 times 10kB values', - test : 'apache_beam.testing.load_tests.co_group_by_key_test', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - project : 'apache-beam-testing', - region : 'us-central1', - job_name : "load-tests-python-dataflow-${mode}-cogbk-3-${now}", - temp_location : 'gs://temp-storage-for-perf-tests/loadtests', - publish_to_big_query : true, - metrics_dataset : datasetName, - metrics_table : "python_dataflow_${mode}_cogbk_3", - influx_measurement : "python_${mode}_cogbk_3", - input_options : '\'{' + - '"num_records": 20000000,' + - '"key_size": 10,' + - '"value_size": 90,' + - '"num_hot_keys": 200000,' + - '"hot_key_fraction": 1,' + - '"algorithm": "lcg"}\'', - co_input_options : '\'{' + - '"num_records": 2000000,' + - '"key_size": 10,' + - '"value_size": 90,' + - '"num_hot_keys": 1000,' + - '"hot_key_fraction": 1,' + - '"algorithm": "lcg"}\'', - iterations : 4, - num_workers : 5, - autoscaling_algorithm: 'NONE' - ] - ], - [ - title : 'CoGroupByKey Python Load test: reiterate 4 times 2MB values', - test : 'apache_beam.testing.load_tests.co_group_by_key_test', - runner : CommonTestProperties.Runner.DATAFLOW, - pipelineOptions: [ - project : 'apache-beam-testing', - region : 'us-central1', - job_name : "load-tests-python-dataflow-${mode}-cogbk-4-${now}", - temp_location : 'gs://temp-storage-for-perf-tests/loadtests', - publish_to_big_query : true, - metrics_dataset : datasetName, - metrics_table : "python_dataflow_${mode}_cogbk_4", - influx_measurement : "python_${mode}_cogbk_4", - input_options : '\'{' + - '"num_records": 20000000,' + - '"key_size": 10,' + - '"value_size": 90,' + - '"num_hot_keys": 1000,' + - '"hot_key_fraction": 1,' + - '"algorithm": "lcg"}\'', - co_input_options : '\'{' + - '"num_records": 2000000,' + - '"key_size": 10,' + - '"value_size": 90,' + - '"num_hot_keys": 1000,' + - '"hot_key_fraction": 1,' + - '"algorithm": "lcg"}\'', - iterations : 4, - num_workers : 5, - autoscaling_algorithm: 'NONE' - ] - ], - ] - .each { test -> test.pipelineOptions.putAll(additionalPipelineArgs) } - .each { test -> (mode != 'streaming') ?: addStreamingOptions(test) } -} - -def addStreamingOptions(test) { - // Use highmem workers to prevent out of memory issues. - test.pipelineOptions << [streaming: null, - worker_machine_type: 'n1-highmem-4' - ] -} - -def loadTestJob = { scope, triggeringContext, mode -> - def datasetName = loadTestsBuilder.getBigQueryDataset('load_test', triggeringContext) - loadTestsBuilder.loadTests(scope, CommonTestProperties.SDK.PYTHON, - loadTestConfigurations(mode, datasetName), 'CoGBK', mode) -} - -CronJobBuilder.cronJob('beam_LoadTests_Python_CoGBK_Dataflow_Batch', 'H H * * *', this) { - additionalPipelineArgs = [ - influx_db_name: InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influx_hostname: InfluxDBCredentialsHelper.InfluxDBHostUrl, - ] - loadTestJob(delegate, CommonTestProperties.TriggeringContext.POST_COMMIT, 'batch') -} - -PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_LoadTests_Python_CoGBK_Dataflow_Batch', - 'Run Load Tests Python CoGBK Dataflow Batch', - 'Load Tests Python CoGBK Dataflow Batch suite', - this - ) { - additionalPipelineArgs = [:] - loadTestJob(delegate, CommonTestProperties.TriggeringContext.PR, 'batch') - } - -CronJobBuilder.cronJob('beam_LoadTests_Python_CoGBK_Dataflow_Streaming', 'H H * * *', this) { - additionalPipelineArgs = [ - influx_db_name: InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influx_hostname: InfluxDBCredentialsHelper.InfluxDBHostUrl, - ] - loadTestJob(delegate, CommonTestProperties.TriggeringContext.POST_COMMIT, 'streaming') -} - -PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_LoadTests_Python_CoGBK_Dataflow_Streaming', - 'Run Load Tests Python CoGBK Dataflow Streaming', - 'Load Tests Python CoGBK Dataflow Streaming suite', - this - ) { - additionalPipelineArgs = [:] - loadTestJob(delegate, CommonTestProperties.TriggeringContext.PR, 'streaming') - } diff --git a/.test-infra/jenkins/job_PerformanceTests_KafkaIO_IT.groovy b/.test-infra/jenkins/job_PerformanceTests_KafkaIO_IT.groovy deleted file mode 100644 index d513dd96a7e2..000000000000 --- a/.test-infra/jenkins/job_PerformanceTests_KafkaIO_IT.groovy +++ /dev/null @@ -1,130 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonJobProperties as common -import Kubernetes -import InfluxDBCredentialsHelper - -String jobName = "beam_PerformanceTests_Kafka_IO" -String HIGH_RANGE_PORT = "32767" - -/** - * This job runs the Kafka IO performance tests. - It runs on a kafka cluster that is build by applying the folder .test-infra/kubernetes/kafka-cluster, - in an existing kubernetes cluster (DEFAULT_CLUSTER in Kubernetes.groovy). - The services created to run this test are: - Pods: 3 kafka pods, 3 zookeeper pods, 1 kafka-config pod which run a job that creates topics. - Services: 1 bootstrap, 1 broker, 3 outside, 1 zookeeper - Job: job.batch/kafka-config-eff079ec - When the performance tests finish all resources are cleaned up by a postBuild step in Kubernetes.groovy - */ -job(jobName) { - common.setTopLevelMainJobProperties(delegate, 'master', 120) - common.setAutoJob(delegate, 'H H/12 * * *') - common.enablePhraseTriggeringFromPullRequest( - delegate, - 'Java KafkaIO Performance Test', - 'Run Java KafkaIO Performance Test') - InfluxDBCredentialsHelper.useCredentials(delegate) - - String namespace = common.getKubernetesNamespace(jobName) - String kubeconfig = common.getKubeconfigLocationForNamespace(namespace) - Kubernetes k8s = Kubernetes.create(delegate, kubeconfig, namespace) - - String kafkaDir = common.makePathAbsolute("src/.test-infra/kubernetes/kafka-cluster") - String kafkaTopicJob = "job.batch/kafka-config-eff079ec" - - /** - * Specifies steps to avoid port collisions when the Kafka outside services (1,2,3) are created. - Function k8s.availablePort finds unused ports in the Kubernetes cluster in a range from 32400 - to 32767 by querying used ports, those ports are stored in env vars like KAFKA_SERVICE_PORT_${service}, - which are used to replace default ports for outside-${service}.yml files, before the apply command. - */ - steps { - String[] configuredPorts = ["32400", "32401", "32402"] - (0..2).each { service -> - k8s.availablePort(service == 0 ? configuredPorts[service] : "\$KAFKA_SERVICE_PORT_${service-1}", - HIGH_RANGE_PORT, "KAFKA_SERVICE_PORT_$service") - shell("sed -i -e s/${configuredPorts[service]}/\$KAFKA_SERVICE_PORT_$service/ \ - ${kafkaDir}/04-outside-services/outside-${service}.yml") - } - } - k8s.apply(kafkaDir) - (0..2).each { k8s.loadBalancerIP("outside-$it", "KAFKA_BROKER_$it") } - k8s.waitForJob(kafkaTopicJob,"40m") - - Map pipelineOptions = [ - tempRoot : 'gs://temp-storage-for-perf-tests', - project : 'apache-beam-testing', - runner : 'DataflowRunner', - sourceOptions : """ - { - "numRecords": "100000000", - "keySizeBytes": "10", - "valueSizeBytes": "90" - } - """.trim().replaceAll("\\s", ""), - bigQueryDataset : 'beam_performance', - bigQueryTable : 'kafkaioit_results', - influxMeasurement : 'kafkaioit_results', - influxDatabase : InfluxDBCredentialsHelper.InfluxDBDatabaseName, - influxHost : InfluxDBCredentialsHelper.InfluxDBHostUrl, - kafkaBootstrapServerAddresses: "\$KAFKA_BROKER_0:\$KAFKA_SERVICE_PORT_0,\$KAFKA_BROKER_1:\$KAFKA_SERVICE_PORT_1," + - "\$KAFKA_BROKER_2:\$KAFKA_SERVICE_PORT_2", //KAFKA_BROKER_ represents IP and KAFKA_SERVICE_ port of outside services - kafkaTopic : 'beam-batch', - readTimeout : '1800', - numWorkers : '5', - autoscalingAlgorithm : 'NONE' - ] - - // We are using a smaller number of records for streaming test since streaming read is much slower - // than batch read. - Map dataflowRunnerV2SdfPipelineOptions = pipelineOptions + [ - sourceOptions : """ - { - "numRecords": "100000000", - "keySizeBytes": "10", - "valueSizeBytes": "90" - } - """.trim().replaceAll("\\s", ""), - kafkaTopic : 'beam-sdf', - readTimeout : '1500', - bigQueryTable : 'kafkaioit_results_runner_v2', - influxMeasurement : 'kafkaioit_results_runner_v2', - experiments : 'use_runner_v2,use_unified_worker', - ] - - steps { - gradle { - rootBuildScriptDir(common.checkoutDir) - common.setGradleSwitches(delegate) - switches("--info") - switches("-DintegrationTestPipelineOptions=\'${common.joinOptionsWithNestedJsonValues(dataflowRunnerV2SdfPipelineOptions)}\'") - switches("-DintegrationTestRunner=dataflow") - tasks(":sdks:java:io:kafka:integrationTest --tests org.apache.beam.sdk.io.kafka.KafkaIOIT.testKafkaIOReadsAndWritesCorrectlyInStreaming") - } - gradle { - rootBuildScriptDir(common.checkoutDir) - common.setGradleSwitches(delegate) - switches("--info") - switches("-DintegrationTestPipelineOptions=\'${common.joinOptionsWithNestedJsonValues(pipelineOptions)}\'") - switches("-DintegrationTestRunner=dataflow") - tasks(":sdks:java:io:kafka:integrationTest --tests org.apache.beam.sdk.io.kafka.KafkaIOIT.testKafkaIOReadsAndWritesCorrectlyInBatch") - } - } -} diff --git a/.test-infra/jenkins/job_Publish_Docker_Snapshots.groovy b/.test-infra/jenkins/job_Publish_Docker_Snapshots.groovy deleted file mode 100644 index 510acd8f37d8..000000000000 --- a/.test-infra/jenkins/job_Publish_Docker_Snapshots.groovy +++ /dev/null @@ -1,50 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonJobProperties as commonJobProperties -import CommonTestProperties -import static PythonTestProperties.SUPPORTED_CONTAINER_TASKS - -job('beam_Publish_Docker_Snapshots') { - description('Builds SDK harness images and job server images for testing purposes.') - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate) - - // Allows triggering this build against pull requests. - commonJobProperties.enablePhraseTriggeringFromPullRequest( - delegate, - 'Beam Publish Docker Snapshots', - 'Publish Docker Snapshots', - false - ) - - // Runs once per day. - commonJobProperties.setAutoJob(delegate, '@daily') - - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - commonJobProperties.setGradleSwitches(delegate) - tasks(":runners:spark:${CommonTestProperties.getSparkVersion()}:job-server:container:dockerPush") - tasks(":runners:flink:${CommonTestProperties.getFlinkVersion()}:job-server-container:dockerPush") - switches("-Pdocker-repository-root=gcr.io/apache-beam-testing/beam_portability") - switches("-Pdocker-tag=latest") - } - } -} diff --git a/.test-infra/jenkins/job_Publish_SDK_Image_Snapshots.groovy b/.test-infra/jenkins/job_Publish_SDK_Image_Snapshots.groovy deleted file mode 100644 index 77b593ab3085..000000000000 --- a/.test-infra/jenkins/job_Publish_SDK_Image_Snapshots.groovy +++ /dev/null @@ -1,56 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonJobProperties as commonJobProperties -import static JavaTestProperties.SUPPORTED_CONTAINER_TASKS as SUPPORTED_JAVA_CONTAINER_TASKS -import static PythonTestProperties.SUPPORTED_CONTAINER_TASKS as SUPPORTED_PYTHON_CONTAINER_TASKS - -// This job publishes regular snapshots of the SDK harness containers for -// testing purposes. It builds and pushes the SDK container to the -// specified GCR repo, tagged at the current Git commit. -job('beam_Publish_Beam_SDK_Snapshots') { - description('Builds SDK harness images snapshots regularly for testing purposes.') - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate) - - // Runs once every four hours. - commonJobProperties.setAutoJob(delegate, 'H H/4 * * *') - - // Use jenkins env var interpolation - leave in single quotes - def imageRepo = 'gcr.io/apache-beam-testing/beam-sdk' - def imageTag = '${GIT_COMMIT}' - - steps { - shell("echo 'Pushing SDK snapshots to ${imageRepo} at tag: ${imageTag}'") - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - commonJobProperties.setGradleSwitches(delegate) - tasks(':sdks:go:container:dockerTagPush') - SUPPORTED_JAVA_CONTAINER_TASKS.each { taskVer -> - tasks(":sdks:java:container:${taskVer}:dockerTagPush") - } - SUPPORTED_PYTHON_CONTAINER_TASKS.each { taskVer -> - tasks(":sdks:python:container:${taskVer}:dockerTagPush") - } - switches("-Pdocker-repository-root=${imageRepo}") - switches("-Pdocker-tag-list=${imageTag},latest") - switches("-Pjava11Home=${commonJobProperties.JAVA_11_HOME}") - } - } -} diff --git a/.test-infra/jenkins/job_ReleaseCandidate_Python.groovy b/.test-infra/jenkins/job_ReleaseCandidate_Python.groovy deleted file mode 100644 index b337d6c03837..000000000000 --- a/.test-infra/jenkins/job_ReleaseCandidate_Python.groovy +++ /dev/null @@ -1,38 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonJobProperties as commonJobProperties - -job('beam_PostRelease_Python_Candidate') { - description('Runs verification of the Python release candidate.') - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 360) - - // Allows triggering this build against pull requests. - commonJobProperties.enablePhraseTriggeringFromPullRequest( - delegate, - 'Python SDK Release Candidates Validation', - 'Run Python ReleaseCandidate') - - // Execute shell command to test Python SDK. - steps { - shell('cd ' + commonJobProperties.checkoutDir + - ' && bash release/src/main/python-release/python_release_automation.sh') - } -} diff --git a/.test-infra/jenkins/job_Release_NightlySnapshot.groovy b/.test-infra/jenkins/job_Release_NightlySnapshot.groovy deleted file mode 100644 index 2833c263b057..000000000000 --- a/.test-infra/jenkins/job_Release_NightlySnapshot.groovy +++ /dev/null @@ -1,78 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonJobProperties as commonJobProperties - -// This creates the nightly snapshot build. -// Into https://repository.apache.org/content/groups/snapshots/org/apache/beam. -job('beam_Release_NightlySnapshot') { - description('Publish a nightly snapshot.') - previousNames(/beam_Release_Gradle_NightlySnapshot/) - - // Execute concurrent builds if necessary. - concurrentBuild() - - // Set common parameters. Timeout is longer, to avoid [BEAM-5774]. - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 200, true, 'ubuntu') - - // This is a post-commit job that runs once per day, not for every push. - commonJobProperties.setAutoJob( - delegate, - '@daily', - 'builds@beam.apache.org') - - - // Allows triggering this build against pull requests. - commonJobProperties.enablePhraseTriggeringFromPullRequest( - delegate, - './gradlew publish', - 'Run Gradle Publish') - - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks('clean') - } - /* - * Skipping verification on 'ubuntu' labelled nodes since they don't have access to the - * some required GCP resources. - * TODO: Uncomment this after we publishing snapshots on 'beam' nodes. - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks('build') - commonJobProperties.setGradleSwitches(delegate) - switches('--no-parallel') - switches('--continue') - } - */ - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks('publish') - commonJobProperties.setGradleSwitches(delegate) - // Publish a snapshot build. - switches("-Ppublishing") - // No need to run checker framework for snapshot publishing - switches("-PskipCheckerFramework") - // Don't run tasks in parallel, currently the maven-publish/signing plugins - // cause build failures when run in parallel with messages like 'error snapshotting' - switches('--no-parallel') - switches('--continue') - } - } -} - From ea8596f2df0e3e4b9da9f215ae6745c2ddfb6612 Mon Sep 17 00:00:00 2001 From: Pranav Bhandari <bhandari.pranav22@gmail.com> Date: Wed, 1 Nov 2023 12:50:01 -0400 Subject: [PATCH 338/435] Add ability to run performance regression checks on Beam IO Load tests. (#29226) --- .../testing/analyzers/io_tests_config.yaml | 256 ++++++++++++++++++ .../analyzers/load_test_perf_analysis.py | 98 +++++++ .../testing/analyzers/perf_analysis_utils.py | 11 +- 3 files changed, 364 insertions(+), 1 deletion(-) create mode 100644 sdks/python/apache_beam/testing/analyzers/io_tests_config.yaml create mode 100644 sdks/python/apache_beam/testing/analyzers/load_test_perf_analysis.py diff --git a/sdks/python/apache_beam/testing/analyzers/io_tests_config.yaml b/sdks/python/apache_beam/testing/analyzers/io_tests_config.yaml new file mode 100644 index 000000000000..2a33ae31797d --- /dev/null +++ b/sdks/python/apache_beam/testing/analyzers/io_tests_config.yaml @@ -0,0 +1,256 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +spanner_io_read: + test_description: | + SpannerIO Read test 100 GB. + project: apache-beam-testing + metrics_dataset: performance_tests + metrics_table: io_performance_metrics + # test_name is in the format testName,pipelineName + test_name: testSpannerWriteAndRead,read-spanner + metric_name: + - RunTime + - EstimatedCost + +spanner_io_read_runnerV2: + test_description: | + SpannerIO RunnerV2 Read test 100 GB. + project: apache-beam-testing + metrics_dataset: performance_tests + metrics_table: io_performance_metrics + # test_name is in the format testName,pipelineName + test_name: testSpannerWriteAndRead,read_spanner_v2 + metric_name: + - RunTime + - EstimatedCost + +spanner_io_write: + test_description: | + SpannerIO write test 100 GB. + project: apache-beam-testing + metrics_dataset: performance_tests + metrics_table: io_performance_metrics + # test_name is in the format testName,pipelineName + test_name: testSpannerWriteAndRead,write-spanner + metric_name: + - RunTime + - EstimatedCost + +spanner_io_write_runnerV2: + test_description: | + SpannerIO RunnerV2 write test 100 GB. + project: apache-beam-testing + metrics_dataset: performance_tests + metrics_table: io_performance_metrics + # test_name is in the format testName,pipelineName + test_name: testSpannerWriteAndRead,write_spanner_v2 + metric_name: + - RunTime + - EstimatedCost + +bigquery_io_storage_api_read: + test_description: | + BigQueryIO Storage write API read test 100 GB. + project: apache-beam-testing + metrics_dataset: performance_tests + metrics_table: io_performance_metrics + # test_name is in the format testName,pipelineName + test_name: testStorageAPIWriteThenRead,read-bigquery + metric_name: + - RunTime + - EstimatedCost + +bigquery_io_storage_api_read_runnerV2: + test_description: | + BigQueryIO RunnerV2 Storage write API read test 100 GB. + project: apache-beam-testing + metrics_dataset: performance_tests + metrics_table: io_performance_metrics + # test_name is in the format testName,pipelineName + test_name: testStorageAPIWriteThenRead,read_bigquery_v2 + metric_name: + - RunTime + - EstimatedCost + +bigquery_io_storage_api_write: + test_description: | + BigQueryIO Storage write API write test 100 GB. + project: apache-beam-testing + metrics_dataset: performance_tests + metrics_table: io_performance_metrics + # test_name is in the format testName,pipelineName + test_name: testStorageAPIWriteThenRead,write-bigquery + metric_name: + - RunTime + - EstimatedCost + +bigquery_io_storage_api_write_runnerV2: + test_description: | + BigQueryIO Storage write API write test 100 GB. + project: apache-beam-testing + metrics_dataset: performance_tests + metrics_table: io_performance_metrics + # test_name is in the format testName,pipelineName + test_name: testStorageAPIWriteThenRead,write_bigquery_v2 + metric_name: + - RunTime + - EstimatedCost + +bigquery_io_avro_file_loads_read: + test_description: | + BigQueryIO Avro file loads read test 100 GB. + project: apache-beam-testing + metrics_dataset: performance_tests + metrics_table: io_performance_metrics + # test_name is in the format testName,pipelineName + test_name: testAvroFileLoadsWriteThenRead,read-bigquery + metric_name: + - RunTime + - EstimatedCost + +bigquery_io_avro_file_loads_read_runnerV2: + test_description: | + BigQueryIO RunnerV2 Avro file loads read test 100 GB. + project: apache-beam-testing + metrics_dataset: performance_tests + metrics_table: io_performance_metrics + # test_name is in the format testName,pipelineName + test_name: testAvroFileLoadsWriteThenRead,read_bigquery_v2 + metric_name: + - RunTime + - EstimatedCost + +bigquery_io_avro_file_loads_write: + test_description: | + BigQueryIO Avro file loads write test 100 GB. + project: apache-beam-testing + metrics_dataset: performance_tests + metrics_table: io_performance_metrics + # test_name is in the format testName,pipelineName + test_name: testAvroFileLoadsWriteThenRead,write-bigquery + metric_name: + - RunTime + - EstimatedCost + +bigquery_io_avro_file_loads_write_runnerV2: + test_description: | + BigQueryIO RunnerV2 Avro file loads write test 100 GB. + project: apache-beam-testing + metrics_dataset: performance_tests + metrics_table: io_performance_metrics + # test_name is in the format testName,pipelineName + test_name: testAvroFileLoadsWriteThenRead,write_bigquery_v2 + metric_name: + - RunTime + - EstimatedCost + +bigquery_io_json_file_loads_read: + test_description: | + BigQueryIO Json file loads read test 100 GB. + project: apache-beam-testing + metrics_dataset: performance_tests + metrics_table: io_performance_metrics + # test_name is in the format testName,pipelineName + test_name: testJsonFileLoadsWriteThenRead,read-bigquery + metric_name: + - RunTime + - EstimatedCost + +bigquery_io_json_file_loads_write: + test_description: | + BigQueryIO Json file loads write test 100 GB. + project: apache-beam-testing + metrics_dataset: performance_tests + metrics_table: io_performance_metrics + # test_name is in the format testName,pipelineName + test_name: testJsonFileLoadsWriteThenRead,write-bigquery + metric_name: + - RunTime + - EstimatedCost + +bigtable_io_read: + test_description: | + BigTableIO read test 100 GB. + project: apache-beam-testing + metrics_dataset: performance_tests + metrics_table: io_performance_metrics + # test_name is in the format testName,pipelineName + test_name: testBigtableWriteAndRead,read-bigtable + metric_name: + - RunTime + - EstimatedCost + +bigtable_io_write: + test_description: | + BigTableIO write test 100 GB. + project: apache-beam-testing + metrics_dataset: performance_tests + metrics_table: io_performance_metrics + # test_name is in the format testName,pipelineName + test_name: testBigtableWriteAndRead,write-bigtable + metric_name: + - RunTime + - EstimatedCost + +text_io_read: + test_description: | + TextIO read test 100 GB. + project: apache-beam-testing + metrics_dataset: performance_tests + metrics_table: io_performance_metrics + # test_name is in the format testName,pipelineName + test_name: testTextIOWriteThenRead,read-textio + metric_name: + - RunTime + - EstimatedCost + +text_io_read_runnerV2: + test_description: | + TextIO RunnerV2 read test 100 GB. + project: apache-beam-testing + metrics_dataset: performance_tests + metrics_table: io_performance_metrics + # test_name is in the format testName,pipelineName + test_name: testTextIOWriteThenRead,read_textio_v2 + metric_name: + - RunTime + - EstimatedCost + +text_io_write: + test_description: | + TextIO write test 100 GB. + project: apache-beam-testing + metrics_dataset: performance_tests + metrics_table: io_performance_metrics + # test_name is in the format testName,pipelineName + test_name: testTextIOWriteThenRead,write-textio + metric_name: + - RunTime + - EstimatedCost + +text_io_write_runnerV2: + test_description: | + TextIO RunnerV2 write test 100 GB. + project: apache-beam-testing + metrics_dataset: performance_tests + metrics_table: io_performance_metrics + # test_name is in the format testName,pipelineName + test_name: testTextIOWriteThenRead,write_textio_v2 + metric_name: + - RunTime + - EstimatedCost diff --git a/sdks/python/apache_beam/testing/analyzers/load_test_perf_analysis.py b/sdks/python/apache_beam/testing/analyzers/load_test_perf_analysis.py new file mode 100644 index 000000000000..ee9d04e6260f --- /dev/null +++ b/sdks/python/apache_beam/testing/analyzers/load_test_perf_analysis.py @@ -0,0 +1,98 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +import argparse +import logging + +from apache_beam.testing.analyzers import constants +from apache_beam.testing.analyzers import perf_analysis +from apache_beam.testing.analyzers import perf_analysis_utils +from apache_beam.testing.analyzers.perf_analysis_utils import MetricContainer +from apache_beam.testing.analyzers.perf_analysis_utils import TestConfigContainer + +try: + from google.cloud import bigquery +except ImportError: + bigquery = None # type: ignore + + +class LoadTestMetricsFetcher(perf_analysis_utils.MetricsFetcher): + """ + Metrics fetcher used to get metric data from a BigQuery table. The metrics + are fetched and returned as a dataclass containing lists of timestamps and + metric_values. + """ + def fetch_metric_data( + self, *, test_config: TestConfigContainer) -> MetricContainer: + if test_config.test_name: + test_name, pipeline_name = test_config.test_name.split(',') + else: + raise Exception("test_name not provided in config.") + + query = f""" + SELECT timestamp, metric.value + FROM {test_config.project}.{test_config.metrics_dataset}.{test_config.metrics_table} + CROSS JOIN UNNEST(metrics) AS metric + WHERE test_name = "{test_name}" AND pipeline_name = "{pipeline_name}" AND metric.name = "{test_config.metric_name}" + ORDER BY timestamp DESC + LIMIT {constants._NUM_DATA_POINTS_TO_RUN_CHANGE_POINT_ANALYSIS} + """ + logging.debug("Running query: %s" % query) + if bigquery is None: + raise ImportError('Bigquery dependencies are not installed.') + client = bigquery.Client() + query_job = client.query(query=query) + metric_data = query_job.result().to_dataframe() + if metric_data.empty: + logging.error( + "No results returned from BigQuery. Please check the query.") + return MetricContainer( + values=metric_data['value'].tolist(), + timestamps=metric_data['timestamp'].tolist(), + ) + + +if __name__ == '__main__': + logging.basicConfig(level=logging.INFO) + load_test_metrics_fetcher = LoadTestMetricsFetcher() + + parser = argparse.ArgumentParser() + parser.add_argument( + '--config_file_path', + required=True, + type=str, + help='Path to the config file that contains data to run the Change Point ' + 'Analysis.The default file will used will be ' + 'apache_beam/testing/analyzers/tests.config.yml. ' + 'If you would like to use the Change Point Analysis for finding ' + 'performance regression in the tests, ' + 'please provide an .yml file in the same structure as the above ' + 'mentioned file. ') + parser.add_argument( + '--save_alert_metadata', + action='store_true', + default=False, + help='Save perf alert/ GH Issue metadata to BigQuery table.') + known_args, unknown_args = parser.parse_known_args() + + if unknown_args: + logging.warning('Discarding unknown arguments : %s ' % unknown_args) + + perf_analysis.run( + big_query_metrics_fetcher=load_test_metrics_fetcher, + config_file_path=known_args.config_file_path, + # Set this to true while running in production. + save_alert_metadata=known_args.save_alert_metadata) diff --git a/sdks/python/apache_beam/testing/analyzers/perf_analysis_utils.py b/sdks/python/apache_beam/testing/analyzers/perf_analysis_utils.py index 11b1cc18ca56..a9015d715e90 100644 --- a/sdks/python/apache_beam/testing/analyzers/perf_analysis_utils.py +++ b/sdks/python/apache_beam/testing/analyzers/perf_analysis_utils.py @@ -28,13 +28,18 @@ import pandas as pd import yaml from google.api_core import exceptions -from google.cloud import bigquery from apache_beam.testing.analyzers import constants from apache_beam.testing.load_tests import load_test_metrics_utils from apache_beam.testing.load_tests.load_test_metrics_utils import BigQueryMetricsPublisher from signal_processing_algorithms.energy_statistics.energy_statistics import e_divisive +# pylint: disable=ungrouped-imports +try: + from google.cloud import bigquery +except ImportError: + bigquery = None # type: ignore + @dataclass(frozen=True) class GitHubIssueMetaData: @@ -118,6 +123,8 @@ def get_existing_issues_data(table_name: str) -> Optional[pd.DataFrame]: LIMIT 10 """ try: + if bigquery is None: + raise ImportError('Bigquery dependencies are not installed.') client = bigquery.Client() query_job = client.query(query=query) existing_issue_data = query_job.result().to_dataframe() @@ -354,6 +361,8 @@ def fetch_metric_data( ORDER BY {load_test_metrics_utils.SUBMIT_TIMESTAMP_LABEL} DESC LIMIT {constants._NUM_DATA_POINTS_TO_RUN_CHANGE_POINT_ANALYSIS} """ + if bigquery is None: + raise ImportError('Bigquery dependencies are not installed.') client = bigquery.Client() query_job = client.query(query=query) metric_data = query_job.result().to_dataframe() From 27ccdc9acfc743ed89f38fd3db277cd9fe3a96f0 Mon Sep 17 00:00:00 2001 From: damccorm <actions@GitHub Actions 106.local> Date: Wed, 1 Nov 2023 17:02:13 +0000 Subject: [PATCH 339/435] Moving to 2.53.0-SNAPSHOT on master branch. --- .asf.yaml | 1 + gradle.properties | 4 +-- release/src/main/scripts/jenkins_jobs.txt | 36 ----------------------- sdks/go/pkg/beam/core/core.go | 2 +- sdks/python/apache_beam/version.py | 2 +- sdks/typescript/package.json | 2 +- 6 files changed, 6 insertions(+), 41 deletions(-) diff --git a/.asf.yaml b/.asf.yaml index 0657d888cb2c..ef141f3bb3fa 100644 --- a/.asf.yaml +++ b/.asf.yaml @@ -49,6 +49,7 @@ github: protected_branches: master: {} + release-2.52.0: {} release-2.51.0: {} release-2.50.0: {} release-2.49.0: {} diff --git a/gradle.properties b/gradle.properties index 6bad220e641b..ef84ea9a5cc9 100644 --- a/gradle.properties +++ b/gradle.properties @@ -30,8 +30,8 @@ signing.gnupg.useLegacyGpg=true # buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy. # To build a custom Beam version make sure you change it in both places, see # https://github.com/apache/beam/issues/21302. -version=2.52.0-SNAPSHOT -sdk_version=2.52.0.dev +version=2.53.0-SNAPSHOT +sdk_version=2.53.0.dev javaVersion=1.8 diff --git a/release/src/main/scripts/jenkins_jobs.txt b/release/src/main/scripts/jenkins_jobs.txt index 65ccc4d9dec2..ae4c13b24d4c 100644 --- a/release/src/main/scripts/jenkins_jobs.txt +++ b/release/src/main/scripts/jenkins_jobs.txt @@ -42,45 +42,13 @@ Run Java examples on Dataflow Java 11,beam_PostCommit_Java_Examples_Dataflow_Jav Run Java examples on Dataflow Java 17,beam_PostCommit_Java_Examples_Dataflow_Java17_PR Run Java_Amazon-Web-Services2_IO_Direct PreCommit,beam_PreCommit_Java_Amazon-Web-Services2_IO_Direct_Phrase Run Java_Amazon-Web-Services_IO_Direct PreCommit,beam_PreCommit_Java_Amazon-Web-Services_IO_Direct_Phrase -Run Java_Amqp_IO_Direct PreCommit,beam_PreCommit_Java_Amqp_IO_Direct_Phrase Run Java_Azure_IO_Direct PreCommit,beam_PreCommit_Java_Azure_IO_Direct_Phrase -Run Java_Cassandra_IO_Direct PreCommit,beam_PreCommit_Java_Cassandra_IO_Direct_Phrase -Run Java_Cdap_IO_Direct PreCommit,beam_PreCommit_Java_Cdap_IO_Direct_Phrase -Run Java_Clickhouse_IO_Direct PreCommit,beam_PreCommit_Java_Clickhouse_IO_Direct_Phrase -Run Java_Csv_IO_Direct PreCommit,beam_PreCommit_Java_Csv_IO_Direct_Phrase -Run Java_Debezium_IO_Direct PreCommit,beam_PreCommit_Java_Debezium_IO_Direct_Phrase -Run Java_Elasticsearch_IO_Direct PreCommit,beam_PreCommit_Java_ElasticSearch_IO_Direct_Phrase -Run Java_Examples_Dataflow PreCommit,beam_PreCommit_Java_Examples_Dataflow_Phrase -Run Java_Examples_Dataflow_Java11 PreCommit,beam_PreCommit_Java_Examples_Dataflow_Java11_Phrase -Run Java_Examples_Dataflow_Java17 PreCommit,beam_PreCommit_Java_Examples_Dataflow_Java17_Phrase -Run Java_File-schema-transform_IO_Direct PreCommit,beam_PreCommit_Java_File-schema-transform_IO_Direct_Phrase Run Java_GCP_IO_Direct PreCommit,beam_PreCommit_Java_GCP_IO_Direct_Phrase Run Java_Google-ads_IO_Direct PreCommit,beam_PreCommit_Java_Google-ads_IO_Direct_Phrase -Run Java_Hbase_IO_Direct PreCommit,beam_PreCommit_Java_HBase_IO_Direct_Phrase -Run Java_Hcatalog_IO_Direct PreCommit,beam_PreCommit_Java_HCatalog_IO_Direct_Phrase Run Java_IOs_Direct PreCommit,beam_PreCommit_Java_IOs_Direct_Phrase -Run Java_Influxdb_IO_Direct PreCommit,beam_PreCommit_Java_InfluxDb_IO_Direct_Phrase -Run Java_JDBC_IO_Direct PreCommit,beam_PreCommit_Java_JDBC_IO_Direct_Phrase -Run Java_Jms_IO_Direct PreCommit,beam_PreCommit_Java_Jms_IO_Direct_Phrase -Run Java_Kafka_IO_Direct PreCommit,beam_PreCommit_Java_Kafka_IO_Direct_Phrase Run Java_Kinesis_IO_Direct PreCommit,beam_PreCommit_Java_Kinesis_IO_Direct_Phrase -Run Java_Kudu_IO_Direct PreCommit,beam_PreCommit_Java_Kudu_IO_Direct_Phrase -Run Java_MongoDb_IO_Direct PreCommit,beam_PreCommit_Java_MongoDb_IO_Direct_Phrase -Run Java_Mqtt_IO_Direct PreCommit,beam_PreCommit_Java_Mqtt_IO_Direct_Phrase -Run Java_Neo4j_IO_Direct PreCommit,beam_PreCommit_Java_Neo4j_IO_Direct_Phrase Run Java_PVR_Flink_Batch PreCommit,beam_PreCommit_Java_PVR_Flink_Batch_Phrase -Run Java_PVR_Flink_Docker PreCommit,beam_PreCommit_Java_PVR_Flink_Docker_Phrase -Run Java_Parquet_IO_Direct PreCommit,beam_PreCommit_Java_Parquet_IO_Direct_Phrase Run Java_Pulsar_IO_Direct PreCommit,beam_PreCommit_Java_Pulsar_IO_Direct_Phrase -Run Java_Rabbitmq_IO_Direct PreCommit,beam_PreCommit_Java_RabbitMq_IO_Direct_Phrase -Run Java_Redis_IO_Direct PreCommit,beam_PreCommit_Java_Redis_IO_Direct_Phrase -Run Java_Singlestore_IO_Direct PreCommit,beam_PreCommit_Java_SingleStore_IO_Direct_Phrase -Run Java_Snowflake_IO_Direct PreCommit,beam_PreCommit_Java_Snowflake_IO_Direct_Phrase -Run Java_Solr_IO_Direct PreCommit,beam_PreCommit_Java_Solr_IO_Direct_Phrase -Run Java_Spark3_Versions PreCommit,beam_PreCommit_Java_Spark3_Versions_Phrase -Run Java_Splunk_IO_Direct PreCommit,beam_PreCommit_Java_Splunk_IO_Direct_Phrase -Run Java_Thrift_IO_Direct PreCommit,beam_PreCommit_Java_Thrift_IO_Direct_Phrase -Run Java_Tika_IO_Direct PreCommit,beam_PreCommit_Java_Tika_IO_Direct_Phrase Run Java_hadoop_IO_Direct PreCommit,beam_PreCommit_Java_Hadoop_IO_Direct_Phrase Run Javadoc PostCommit,beam_PostCommit_Javadoc_PR Run Jpms Dataflow Java 11 PostCommit,beam_PostCommit_Java_Jpms_Dataflow_Java11_PR @@ -91,7 +59,6 @@ Run Jpms Flink Java 11 PostCommit,beam_PostCommit_Java_Jpms_Flink_Java11_PR Run Jpms Spark Java 11 PostCommit,beam_PostCommit_Java_Jpms_Spark_Java11_PR Run PortableJar_Flink PostCommit,beam_PostCommit_PortableJar_Flink_PR Run PortableJar_Spark PostCommit,beam_PostCommit_PortableJar_Spark_PR -Run Portable_Python PreCommit,beam_PreCommit_Portable_Python_Phrase Run PostCommit_Java_Avro_Versions,beam_PostCommit_Java_Avro_Versions_PR Run PostCommit_Java_Dataflow,beam_PostCommit_Java_DataflowV1_PR Run PostCommit_Java_DataflowV2,beam_PostCommit_Java_DataflowV2_PR @@ -129,9 +96,6 @@ Run Python_Xlang_Gcp_Dataflow PostCommit,beam_PostCommit_Python_Xlang_Gcp_Datafl Run Python_Xlang_Gcp_Direct PostCommit,beam_PostCommit_Python_Xlang_Gcp_Direct_PR Run Python_Xlang_IO_Dataflow PostCommit,beam_PostCommit_Python_Xlang_IO_Dataflow_PR Run SQL PostCommit,beam_PostCommit_SQL_PR -Run SQL PreCommit,beam_PreCommit_SQL_Phrase -Run SQL_Java11 PreCommit,beam_PreCommit_SQL_Java11_Phrase -Run SQL_Java17 PreCommit,beam_PreCommit_SQL_Java17_Phrase Run Samza ValidatesRunner,beam_PostCommit_Java_ValidatesRunner_Samza_PR Run Spark Runner Nexmark Tests,beam_PostCommit_Java_Nexmark_Spark_PR Run Spark Runner Tpcds Tests,beam_PostCommit_Java_Tpcds_Spark_PR diff --git a/sdks/go/pkg/beam/core/core.go b/sdks/go/pkg/beam/core/core.go index ed62a2e9eac0..07326d96528d 100644 --- a/sdks/go/pkg/beam/core/core.go +++ b/sdks/go/pkg/beam/core/core.go @@ -27,7 +27,7 @@ const ( // SdkName is the human readable name of the SDK for UserAgents. SdkName = "Apache Beam SDK for Go" // SdkVersion is the current version of the SDK. - SdkVersion = "2.52.0.dev" + SdkVersion = "2.53.0.dev" // DefaultDockerImage represents the associated image for this release. DefaultDockerImage = "apache/beam_go_sdk:" + SdkVersion diff --git a/sdks/python/apache_beam/version.py b/sdks/python/apache_beam/version.py index a69e3839fff3..fa890eab5005 100644 --- a/sdks/python/apache_beam/version.py +++ b/sdks/python/apache_beam/version.py @@ -17,4 +17,4 @@ """Apache Beam SDK version information and utilities.""" -__version__ = '2.52.0.dev' +__version__ = '2.53.0.dev' diff --git a/sdks/typescript/package.json b/sdks/typescript/package.json index b582b3d5c07f..918846a79add 100644 --- a/sdks/typescript/package.json +++ b/sdks/typescript/package.json @@ -1,6 +1,6 @@ { "name": "apache-beam", - "version": "2.52.0-SNAPSHOT", + "version": "2.53.0-SNAPSHOT", "devDependencies": { "@google-cloud/bigquery": "^5.12.0", "@types/mocha": "^9.0.0", From 00a55272eaac451c3f364c4071e4af1ab56b88bd Mon Sep 17 00:00:00 2001 From: Damon <damondouglas@users.noreply.github.com> Date: Wed, 1 Nov 2023 10:09:14 -0700 Subject: [PATCH 340/435] [RRIO] [Testing] Mock API integration tests (#29236) * Update README with integration test instructions * Replace refresher with human readable ids * Impl integration tests; refactor code errors * Add missing code comment * Fix whitespace --- .test-infra/mock-apis/README.md | 56 +++-- .test-infra/mock-apis/go.mod | 6 +- .test-infra/mock-apis/go.sum | 12 +- .../kustomization.yaml | 34 --- .../configmap.yaml | 24 -- .../deployment.yaml | 27 --- .../configmap.yaml | 8 +- .../deployment.yaml | 6 +- .../kustomization.yaml | 4 +- .../configmap.yaml | 7 +- .../deployment.yaml | 6 +- .../kustomization.yaml | 4 +- .../src/main/go/internal/service/echo/echo.go | 47 ++-- .../go/test/integration/echo/echo_test.go | 220 ++++++++++++++++++ .../main/go/test/integration/integration.go | 32 +++ .../src/main/go/test/integration/vars.go | 49 ++++ 16 files changed, 402 insertions(+), 140 deletions(-) delete mode 100644 .test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/123079b5-1e58-4b28-a185-66702e2b10c3/kustomization.yaml delete mode 100644 .test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/e1064224-3671-46fe-971d-47887fac3d4c/configmap.yaml delete mode 100644 .test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/e1064224-3671-46fe-971d-47887fac3d4c/deployment.yaml rename .test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/{f588787b-28f8-4e5f-8335-f862379daf59 => echo-should-exceed-quota}/configmap.yaml (87%) rename .test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/{f588787b-28f8-4e5f-8335-f862379daf59 => echo-should-exceed-quota}/deployment.yaml (88%) rename .test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/{f588787b-28f8-4e5f-8335-f862379daf59 => echo-should-exceed-quota}/kustomization.yaml (92%) rename .test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/{123079b5-1e58-4b28-a185-66702e2b10c3 => echo-should-never-exceed-quota}/configmap.yaml (87%) rename .test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/{123079b5-1e58-4b28-a185-66702e2b10c3 => echo-should-never-exceed-quota}/deployment.yaml (88%) rename .test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/{e1064224-3671-46fe-971d-47887fac3d4c => echo-should-never-exceed-quota}/kustomization.yaml (92%) create mode 100644 .test-infra/mock-apis/src/main/go/test/integration/echo/echo_test.go create mode 100644 .test-infra/mock-apis/src/main/go/test/integration/integration.go create mode 100644 .test-infra/mock-apis/src/main/go/test/integration/vars.go diff --git a/.test-infra/mock-apis/README.md b/.test-infra/mock-apis/README.md index df34757b770d..9c4911a0d636 100644 --- a/.test-infra/mock-apis/README.md +++ b/.test-infra/mock-apis/README.md @@ -75,7 +75,45 @@ go test ./src/main/go/internal/... ## Integration -TODO: See https://github.com/apache/beam/issues/28859 +Integration tests require the following values. + +### Quota ID + +Each allocated quota corresponds to a unique ID known as the Quota ID. +There exists a one-to-one relationship between the allocated quota and +the +[infrastructure/kubernetes/refresher/overlays](infrastructure/kubernetes/refresher/overlays). + +To query the Kubernetes cluster for allocated Quota IDs: +``` +kubectl get deploy --selector=app.kubernetes.io/name=refresher -o custom-columns='QUOTA_ID:.metadata.labels.quota-id' +``` + +### Service Endpoint + +To list available endpoints, run: + +``` +kubectl get svc -o=custom-columns='NAME:.metadata.name,HOST:.status.loadBalancer.ingress[*].ip,PORT_NAME:.spec.ports[*].name,PORT:.spec.ports[*].port' +``` + +You should see something similar to: + +``` +NAME HOST PORT_NAME PORT +echo 10.n.n.n grpc,http 50051,8080 +``` + +When running tests locally, you will need to first run: +``` +kubectl port-forward service/echo 50051:50051 8080:8080 +``` + +which allows you to access the gRPC via `localhost:50051` and the HTTP via +`http://localhost:8080/v1/echo`. + +When running tests on Dataflow, you supply `10.n.n.n:50051` for gRPC and +`http://10.n.n.n:8080/v1/echo` for HTTP. # Local Usage @@ -186,24 +224,14 @@ The Refresher service relies on [kustomize](https://kustomize.io) overlays which are located at [infrastructure/kubernetes/refresher/overlays](infrastructure/kubernetes/refresher/overlays). Each folder contained in [infrastructure/kubernetes/refresher/overlays](infrastructure/kubernetes/refresher/overlays) -corresponds to an individual Refresher instance that is identified by the UUID. -You will need to deploy each one individually. +corresponds to an individual Refresher instance that is identified by a unique +string id. You will need to deploy each one individually. For example: ``` -kubectl kustomize infrastructure/kubernetes/refresher/overlays/f588787b-28f8-4e5f-8335-f862379daf59 | ko resolve -f - | kubectl apply -f - +kubectl kustomize infrastructure/kubernetes/refresher/overlays/echo-should-never-exceed-quota | ko resolve -f - | kubectl apply -f - ``` Like previously, you may see "Does not have minimum availability" message showing on the status. It may take some time for GKE autopilot to scale the node pool. - -## Additional note for creating a new Refresher service instance - -Each Refresher service instance relies on a unique UUID, where -the [kustomize](https://kustomize.io) overlay replaces in the -[infrastructure/kubernetes/refresher/base](infrastructure/kubernetes/refresher/base) -template. - -You can copy the entire folder and paste into a new one with a unique UUID -and then perform a find-replace of the old UUID with the new one. diff --git a/.test-infra/mock-apis/go.mod b/.test-infra/mock-apis/go.mod index ef2953f27c57..cc65cfbaac76 100644 --- a/.test-infra/mock-apis/go.mod +++ b/.test-infra/mock-apis/go.mod @@ -45,11 +45,11 @@ require ( github.com/googleapis/enterprise-certificate-proxy v0.2.4 // indirect github.com/googleapis/gax-go/v2 v2.12.0 // indirect go.opencensus.io v0.24.0 // indirect - golang.org/x/crypto v0.13.0 // indirect - golang.org/x/net v0.15.0 // indirect + golang.org/x/crypto v0.14.0 // indirect + golang.org/x/net v0.17.0 // indirect golang.org/x/oauth2 v0.12.0 // indirect golang.org/x/sync v0.3.0 // indirect - golang.org/x/sys v0.12.0 // indirect + golang.org/x/sys v0.13.0 // indirect golang.org/x/text v0.13.0 // indirect google.golang.org/api v0.128.0 // indirect google.golang.org/appengine v1.6.7 // indirect diff --git a/.test-infra/mock-apis/go.sum b/.test-infra/mock-apis/go.sum index 74b587a72449..a928e3dae2f1 100644 --- a/.test-infra/mock-apis/go.sum +++ b/.test-infra/mock-apis/go.sum @@ -101,8 +101,8 @@ golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACk golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/crypto v0.0.0-20220314234659-1baeb1ce4c0b/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= -golang.org/x/crypto v0.13.0 h1:mvySKfSWJ+UKUii46M40LOvyWfN0s2U+46/jDd0e6Ck= -golang.org/x/crypto v0.13.0/go.mod h1:y6Z2r+Rw4iayiXXAIxJIDAJ1zMW4yaTpebo8fPOliYc= +golang.org/x/crypto v0.14.0 h1:wBqGXzWJW6m1XrIKlAH0Hs1JJ7+9KBwnIO8v66Q9cHc= +golang.org/x/crypto v0.14.0/go.mod h1:MVFd36DqK4CsrnJYDkBA3VC4m2GkXAM0PvzMCn4JQf4= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= @@ -121,8 +121,8 @@ golang.org/x/net v0.0.0-20201110031124-69a78807bb2b/go.mod h1:sp8m0HH+o8qH0wwXwY golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= golang.org/x/net v0.0.0-20211112202133-69e39bad7dc2/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= -golang.org/x/net v0.15.0 h1:ugBLEUaxABaB5AJqW9enI0ACdci2RUd4eP51NTBvuJ8= -golang.org/x/net v0.15.0/go.mod h1:idbUs1IY1+zTqbi8yxTbhexhEEk5ur9LInksu6HrEpk= +golang.org/x/net v0.17.0 h1:pVaXccu2ozPjCXewfr1S7xza/zcXTity9cCdXQYSjIM= +golang.org/x/net v0.17.0/go.mod h1:NxSsAGuq816PNPmqtQdLE42eU2Fs7NoRIZrHJAlaCOE= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20200107190931-bf48bf16ab8d/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.12.0 h1:smVPGxink+n1ZI5pkQa8y6fZT0RW0MgCO5bFpepy4B4= @@ -144,8 +144,8 @@ golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.12.0 h1:CM0HF96J0hcLAwsHPJZjfdNzs0gftsLfgKt57wWHJ0o= -golang.org/x/sys v0.12.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.13.0 h1:Af8nKPmuFypiUBjVoU9V20FiaFXOcuZI21p0ycVYYGE= +golang.org/x/sys v0.13.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= diff --git a/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/123079b5-1e58-4b28-a185-66702e2b10c3/kustomization.yaml b/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/123079b5-1e58-4b28-a185-66702e2b10c3/kustomization.yaml deleted file mode 100644 index 496e53544630..000000000000 --- a/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/123079b5-1e58-4b28-a185-66702e2b10c3/kustomization.yaml +++ /dev/null @@ -1,34 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -# Configures the overlay for .test-infra/mock-apis/infrastructure/kubernetes/refresher/base -# Using the Quota Id: -# 123079b5-1e58-4b28-a185-66702e2b10c3 - -resources: -- ../../base - -nameSuffix: -123079b5-1e58-4b28-a185-66702e2b10c3 - -patches: -- path: configmap.yaml - target: - kind: ConfigMap - name: refresher - -- path: deployment.yaml - target: - kind: Deployment - name: refresher diff --git a/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/e1064224-3671-46fe-971d-47887fac3d4c/configmap.yaml b/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/e1064224-3671-46fe-971d-47887fac3d4c/configmap.yaml deleted file mode 100644 index 71b19b257f2b..000000000000 --- a/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/e1064224-3671-46fe-971d-47887fac3d4c/configmap.yaml +++ /dev/null @@ -1,24 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -# Configures patch for ../base/configmap.yaml -# See https://kubectl.docs.kubernetes.io/references/kustomize/kustomization/patches/ - -- op: replace - path: /metadata/labels/quota-id - value: e1064224-3671-46fe-971d-47887fac3d4c -- op: replace - path: /data/QUOTA_ID - value: e1064224-3671-46fe-971d-47887fac3d4c diff --git a/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/e1064224-3671-46fe-971d-47887fac3d4c/deployment.yaml b/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/e1064224-3671-46fe-971d-47887fac3d4c/deployment.yaml deleted file mode 100644 index 9f13ec4b7844..000000000000 --- a/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/e1064224-3671-46fe-971d-47887fac3d4c/deployment.yaml +++ /dev/null @@ -1,27 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -# Configures patch for ../base/deployment.yaml -# See https://kubectl.docs.kubernetes.io/references/kustomize/kustomization/patches/ - -- op: replace - path: /metadata/labels/quota-id - value: e1064224-3671-46fe-971d-47887fac3d4c -- op: replace - path: /spec/selector/matchLabels/quota-id - value: e1064224-3671-46fe-971d-47887fac3d4c -- op: replace - path: /spec/template/metadata/labels/quota-id - value: e1064224-3671-46fe-971d-47887fac3d4c diff --git a/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/f588787b-28f8-4e5f-8335-f862379daf59/configmap.yaml b/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/echo-should-exceed-quota/configmap.yaml similarity index 87% rename from .test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/f588787b-28f8-4e5f-8335-f862379daf59/configmap.yaml rename to .test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/echo-should-exceed-quota/configmap.yaml index b6a12f7f133d..274ae43ebb89 100644 --- a/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/f588787b-28f8-4e5f-8335-f862379daf59/configmap.yaml +++ b/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/echo-should-exceed-quota/configmap.yaml @@ -18,7 +18,11 @@ - op: replace path: /metadata/labels/quota-id - value: f588787b-28f8-4e5f-8335-f862379daf59 + value: echo-should-exceed-quota - op: replace path: /data/QUOTA_ID - value: f588787b-28f8-4e5f-8335-f862379daf59 + value: echo-should-exceed-quota +- op: replace + path: /data/QUOTA_SIZE + # We need at least 1 + value: "1" diff --git a/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/f588787b-28f8-4e5f-8335-f862379daf59/deployment.yaml b/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/echo-should-exceed-quota/deployment.yaml similarity index 88% rename from .test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/f588787b-28f8-4e5f-8335-f862379daf59/deployment.yaml rename to .test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/echo-should-exceed-quota/deployment.yaml index 214ed9634a82..e903a6c7c29c 100644 --- a/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/f588787b-28f8-4e5f-8335-f862379daf59/deployment.yaml +++ b/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/echo-should-exceed-quota/deployment.yaml @@ -18,10 +18,10 @@ - op: replace path: /metadata/labels/quota-id - value: f588787b-28f8-4e5f-8335-f862379daf59 + value: echo-should-exceed-quota - op: replace path: /spec/selector/matchLabels/quota-id - value: f588787b-28f8-4e5f-8335-f862379daf59 + value: echo-should-exceed-quota - op: replace path: /spec/template/metadata/labels/quota-id - value: f588787b-28f8-4e5f-8335-f862379daf59 + value: echo-should-exceed-quota diff --git a/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/f588787b-28f8-4e5f-8335-f862379daf59/kustomization.yaml b/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/echo-should-exceed-quota/kustomization.yaml similarity index 92% rename from .test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/f588787b-28f8-4e5f-8335-f862379daf59/kustomization.yaml rename to .test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/echo-should-exceed-quota/kustomization.yaml index 5198e4238198..9330ea4c6c78 100644 --- a/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/f588787b-28f8-4e5f-8335-f862379daf59/kustomization.yaml +++ b/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/echo-should-exceed-quota/kustomization.yaml @@ -15,12 +15,12 @@ # Configures the overlay for .test-infra/mock-apis/infrastructure/kubernetes/refresher/base # Using the Quota Id: -# f588787b-28f8-4e5f-8335-f862379daf59 +# echo-should-exceed-quota resources: - ../../base -nameSuffix: -f588787b-28f8-4e5f-8335-f862379daf59 +nameSuffix: -echo-should-exceed-quota patches: - path: configmap.yaml diff --git a/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/123079b5-1e58-4b28-a185-66702e2b10c3/configmap.yaml b/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/echo-should-never-exceed-quota/configmap.yaml similarity index 87% rename from .test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/123079b5-1e58-4b28-a185-66702e2b10c3/configmap.yaml rename to .test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/echo-should-never-exceed-quota/configmap.yaml index bf87b0646ea6..409d83a81269 100644 --- a/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/123079b5-1e58-4b28-a185-66702e2b10c3/configmap.yaml +++ b/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/echo-should-never-exceed-quota/configmap.yaml @@ -18,7 +18,10 @@ - op: replace path: /metadata/labels/quota-id - value: 123079b5-1e58-4b28-a185-66702e2b10c3 + value: echo-should-never-exceed-quota - op: replace path: /data/QUOTA_ID - value: 123079b5-1e58-4b28-a185-66702e2b10c3 + value: echo-should-never-exceed-quota +- op: replace + path: /data/QUOTA_SIZE + value: "1000000000" diff --git a/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/123079b5-1e58-4b28-a185-66702e2b10c3/deployment.yaml b/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/echo-should-never-exceed-quota/deployment.yaml similarity index 88% rename from .test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/123079b5-1e58-4b28-a185-66702e2b10c3/deployment.yaml rename to .test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/echo-should-never-exceed-quota/deployment.yaml index 7ad531ce7d8b..d550adf02048 100644 --- a/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/123079b5-1e58-4b28-a185-66702e2b10c3/deployment.yaml +++ b/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/echo-should-never-exceed-quota/deployment.yaml @@ -18,10 +18,10 @@ - op: replace path: /metadata/labels/quota-id - value: 123079b5-1e58-4b28-a185-66702e2b10c3 + value: echo-should-never-exceed-quota - op: replace path: /spec/selector/matchLabels/quota-id - value: 123079b5-1e58-4b28-a185-66702e2b10c3 + value: echo-should-never-exceed-quota - op: replace path: /spec/template/metadata/labels/quota-id - value: 123079b5-1e58-4b28-a185-66702e2b10c3 + value: echo-should-never-exceed-quota diff --git a/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/e1064224-3671-46fe-971d-47887fac3d4c/kustomization.yaml b/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/echo-should-never-exceed-quota/kustomization.yaml similarity index 92% rename from .test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/e1064224-3671-46fe-971d-47887fac3d4c/kustomization.yaml rename to .test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/echo-should-never-exceed-quota/kustomization.yaml index 3dd6ff160abc..1f8d23ba01bd 100644 --- a/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/e1064224-3671-46fe-971d-47887fac3d4c/kustomization.yaml +++ b/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/echo-should-never-exceed-quota/kustomization.yaml @@ -15,12 +15,12 @@ # Configures the overlay for .test-infra/mock-apis/infrastructure/kubernetes/refresher/base # Using the Quota Id: -# e1064224-3671-46fe-971d-47887fac3d4c +# echo-should-never-exceed-quota resources: - ../../base -nameSuffix: -e1064224-3671-46fe-971d-47887fac3d4c +nameSuffix: -echo-should-never-exceed-quota patches: - path: configmap.yaml diff --git a/.test-infra/mock-apis/src/main/go/internal/service/echo/echo.go b/.test-infra/mock-apis/src/main/go/internal/service/echo/echo.go index d958d18576e5..d0682551775f 100644 --- a/.test-infra/mock-apis/src/main/go/internal/service/echo/echo.go +++ b/.test-infra/mock-apis/src/main/go/internal/service/echo/echo.go @@ -39,7 +39,7 @@ import ( const ( metricsNamePrefix = "echo" echoPath = "/proto.echo.v1.EchoService/Echo" - echoPathAlias = "/v1/echo" + PathAlias = "/v1/echo" healthPath = "/grpc.health.v1.Health/Check" healthPathAlias = "/v1/healthz" ) @@ -58,6 +58,11 @@ func Register(s *grpc.Server, opts *Options) (http.Handler, error) { Name: reflect.TypeOf((*echo)(nil)).PkgPath(), }) } + var attrs []any + for _, attr := range opts.LoggingAttrs { + attrs = append(attrs, attr) + } + opts.Logger = opts.Logger.With(attrs...) srv := &echo{ opts: opts, } @@ -77,7 +82,7 @@ type echo struct { // ServeHTTP implements http.Handler, allowing echo to support HTTP clients in addition to gRPC. func (srv *echo) ServeHTTP(w http.ResponseWriter, r *http.Request) { switch r.URL.Path { - case echoPath, echoPathAlias: + case echoPath, PathAlias: srv.httpHandler(w, r) case healthPath, healthPathAlias: srv.checkHandler(w, r) @@ -104,7 +109,7 @@ func (srv *echo) checkHandler(w http.ResponseWriter, r *http.Request) { return } if err := json.NewEncoder(w).Encode(resp); err != nil { - srv.opts.Logger.LogAttrs(context.Background(), slog.LevelError, err.Error(), srv.opts.LoggingAttrs...) + srv.opts.Logger.Log(r.Context(), slog.LevelError, err.Error()) http.Error(w, err.Error(), http.StatusInternalServerError) } } @@ -113,7 +118,7 @@ func (srv *echo) checkHandler(w http.ResponseWriter, r *http.Request) { func (srv *echo) Watch(request *grpc_health_v1.HealthCheckRequest, server grpc_health_v1.Health_WatchServer) error { resp, err := srv.Check(server.Context(), request) if err != nil { - srv.opts.Logger.LogAttrs(context.Background(), slog.LevelError, err.Error(), srv.opts.LoggingAttrs...) + srv.opts.Logger.Log(server.Context(), slog.LevelError, err.Error()) return err } return server.Send(resp) @@ -128,7 +133,7 @@ func (srv *echo) Echo(ctx context.Context, request *echov1.EchoRequest) (*echov1 return nil, status.Errorf(codes.NotFound, "error: source not found: %s, err %v", request.Id, err) } if err != nil { - srv.opts.Logger.LogAttrs(context.Background(), slog.LevelError, err.Error(), srv.opts.LoggingAttrs...) + srv.opts.Logger.Log(ctx, slog.LevelError, err.Error()) return nil, status.Errorf(codes.Internal, "error: encountered from cache for resource: %srv, err %v", request.Id, err) } @@ -137,7 +142,7 @@ func (srv *echo) Echo(ctx context.Context, request *echov1.EchoRequest) (*echov1 } if v < 0 { - return nil, status.Errorf(codes.ResourceExhausted, "error: resource exhausted for: %srv", request.Id) + return nil, status.Errorf(codes.ResourceExhausted, "error: resource exhausted for: %s", request.Id) } return &echov1.EchoResponse{ @@ -154,7 +159,7 @@ func (srv *echo) writeMetric(ctx context.Context, id string, value int64) error Timestamp: time.Now(), Value: value + 1, }); err != nil { - srv.opts.Logger.LogAttrs(context.Background(), slog.LevelError, err.Error(), srv.opts.LoggingAttrs...) + srv.opts.Logger.Log(ctx, slog.LevelError, err.Error()) } return nil } @@ -163,23 +168,29 @@ func (srv *echo) httpHandler(w http.ResponseWriter, r *http.Request) { var body *echov1.EchoRequest if err := json.NewDecoder(r.Body).Decode(&body); err != nil { err = fmt.Errorf("error decoding request body, payload field of %T needs to be base64 encoded, error: %w", body, err) - srv.opts.Logger.LogAttrs(context.Background(), slog.LevelError, err.Error(), srv.opts.LoggingAttrs...) + srv.opts.Logger.Log(r.Context(), slog.LevelError, err.Error()) http.Error(w, err.Error(), http.StatusBadRequest) return } resp, err := srv.Echo(r.Context(), body) - if status.Code(err) == http.StatusNotFound { - http.Error(w, err.Error(), http.StatusNotFound) - return - } - if err != nil { - http.Error(w, err.Error(), http.StatusInternalServerError) - return - } - - if err := json.NewEncoder(w).Encode(resp); err != nil { + switch status.Code(err) { + case codes.OK: + if err := json.NewEncoder(w).Encode(resp); err != nil { + srv.opts.Logger.Log(r.Context(), slog.LevelError, err.Error()) + http.Error(w, err.Error(), http.StatusInternalServerError) + } + case codes.InvalidArgument: + http.Error(w, err.Error(), http.StatusBadRequest) + case codes.DeadlineExceeded: + http.Error(w, err.Error(), http.StatusRequestTimeout) + case codes.NotFound: + http.Error(w, err.Error(), http.StatusNotFound) + case codes.ResourceExhausted: + http.Error(w, err.Error(), http.StatusTooManyRequests) + default: + srv.opts.Logger.Log(r.Context(), slog.LevelError, err.Error()) http.Error(w, err.Error(), http.StatusInternalServerError) } } diff --git a/.test-infra/mock-apis/src/main/go/test/integration/echo/echo_test.go b/.test-infra/mock-apis/src/main/go/test/integration/echo/echo_test.go new file mode 100644 index 000000000000..73ad597c7d34 --- /dev/null +++ b/.test-infra/mock-apis/src/main/go/test/integration/echo/echo_test.go @@ -0,0 +1,220 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// Tests for the src/main/go/cmd/service/echo service. +package echo + +import ( + "bytes" + "context" + "encoding/json" + "errors" + "fmt" + "net/http" + "regexp" + "testing" + "time" + + echov1 "github.com/apache/beam/test-infra/mock-apis/src/main/go/internal/proto/echo/v1" + "github.com/apache/beam/test-infra/mock-apis/src/main/go/internal/service/echo" + "github.com/apache/beam/test-infra/mock-apis/src/main/go/test/integration" + "github.com/google/go-cmp/cmp" + "github.com/google/go-cmp/cmp/cmpopts" + "google.golang.org/grpc" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/credentials/insecure" + "google.golang.org/grpc/status" +) + +const ( + // QuotaIds below correspond to: + // kubectl get deploy --selector=app.kubernetes.io/tag=refresher -o custom-columns='QUOTA_ID:.metadata.labels.quota-id' + // See https://github.com/apache/beam/tree/master/.test-infra/mock-apis#writing-integration-tests + shouldExceedQuotaId = "echo-should-exceed-quota" + shouldNeverExceedQuotaId = "echo-should-never-exceed-quota" + shouldNotExistId = "should-not-exist" +) + +var ( + grpcOpts = []grpc.DialOption{ + grpc.WithTransportCredentials(insecure.NewCredentials()), + grpc.WithBlock(), + } + + timeout = time.Second * 3 +) + +func TestEcho(t *testing.T) { + payload := []byte("payload") + + for _, tt := range []struct { + tag string + quotaId string + client echov1.EchoServiceClient + want *echov1.EchoResponse + wantErr error + }{ + { + tag: "http", + quotaId: shouldExceedQuotaId, + client: withHttp(t), + wantErr: errors.New("429 Too Many Requests"), + }, + { + tag: "grpc", + quotaId: shouldExceedQuotaId, + client: withGrpc(t), + wantErr: status.Error(codes.ResourceExhausted, "error: resource exhausted for: echo-should-exceed-quota"), + }, + { + tag: "http", + quotaId: shouldNotExistId, + client: withHttp(t), + wantErr: errors.New("404 Not Found"), + }, + { + tag: "grpc", + quotaId: shouldNotExistId, + client: withGrpc(t), + wantErr: status.Error(codes.NotFound, "error: source not found: should-not-exist, err resource does not exist"), + }, + { + tag: "http", + quotaId: shouldNeverExceedQuotaId, + client: withHttp(t), + want: &echov1.EchoResponse{ + Id: shouldNeverExceedQuotaId, + Payload: payload, + }, + }, + { + tag: "grpc", + quotaId: shouldNeverExceedQuotaId, + client: withGrpc(t), + want: &echov1.EchoResponse{ + Id: shouldNeverExceedQuotaId, + Payload: payload, + }, + }, + } { + t.Run(fmt.Sprintf("%s/%s", tt.quotaId, tt.tag), func(t *testing.T) { + ctx, cancel := withTimeout() + defer cancel() + + req := &echov1.EchoRequest{ + Id: tt.quotaId, + Payload: payload, + } + + var resps []*echov1.EchoResponse + var errs []error + + for i := 0; i < 3; i++ { + resp, err := tt.client.Echo(ctx, req) + if err != nil { + errs = append(errs, err) + } + if resp != nil { + resps = append(resps, resp) + } + } + + if tt.wantErr != nil && len(errs) == 0 { + t.Errorf("Echo(%+v) err = nil, wantErr = %v", req, tt.wantErr) + return + } + + for _, err := range errs { + if diff := cmp.Diff(tt.wantErr.Error(), err.Error()); diff != "" { + t.Errorf("Echo(%+v) err mismatch (-want +got)\n%s", req, diff) + } + } + + if tt.want != nil { + for _, resp := range resps { + if diff := cmp.Diff(tt.want, resp, cmpopts.IgnoreUnexported(echov1.EchoResponse{})); diff != "" { + t.Errorf("Echo(%+v) mismatch (-want +got)\n%s", req, diff) + } + } + } + + }) + } +} + +func TestMain(m *testing.M) { + integration.Run(m) +} + +func withGrpc(t *testing.T) echov1.EchoServiceClient { + t.Helper() + ctx, cancel := withTimeout() + defer cancel() + + conn, err := grpc.DialContext(ctx, *integration.GRPCServiceEndpoint, grpcOpts...) + if err != nil { + t.Fatalf("DialContext(%s) err %v", *integration.GRPCServiceEndpoint, err) + } + t.Cleanup(func() { + if err := conn.Close(); err != nil { + t.Fatal(err) + } + }) + + return echov1.NewEchoServiceClient(conn) +} + +type httpCaller struct { + rawUrl string +} + +func (h *httpCaller) Echo(ctx context.Context, in *echov1.EchoRequest, _ ...grpc.CallOption) (*echov1.EchoResponse, error) { + ctx, cancel := withTimeout() + defer cancel() + buf := bytes.Buffer{} + if err := json.NewEncoder(&buf).Encode(in); err != nil { + return nil, err + } + + resp, err := http.Post(h.rawUrl, "application/json", &buf) + if err != nil { + return nil, err + } + + if resp.StatusCode > 299 { + return nil, errors.New(resp.Status) + } + + var result *echov1.EchoResponse + if err := json.NewDecoder(resp.Body).Decode(&result); err != nil { + return nil, err + } + return result, nil +} + +func withHttp(t *testing.T) echov1.EchoServiceClient { + p := regexp.MustCompile(`^http://`) + rawUrl := fmt.Sprint(*integration.HTTPServiceEndpoint, echo.PathAlias) + if !p.MatchString(rawUrl) { + t.Fatalf("missing 'http(s)' scheme from %s", *integration.HTTPServiceEndpoint) + } + return &httpCaller{ + rawUrl: rawUrl, + } +} + +func withTimeout() (context.Context, context.CancelFunc) { + return context.WithTimeout(context.Background(), timeout) +} diff --git a/.test-infra/mock-apis/src/main/go/test/integration/integration.go b/.test-infra/mock-apis/src/main/go/test/integration/integration.go new file mode 100644 index 000000000000..777225061eac --- /dev/null +++ b/.test-infra/mock-apis/src/main/go/test/integration/integration.go @@ -0,0 +1,32 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// Package integration provides functionality that needs to be shared between all +// integration tests. +package integration + +import ( + "flag" + "os" + "testing" +) + +// Run a testing.M, first calling flag.Parse if not flag.Parsed. +func Run(m *testing.M) { + if !flag.Parsed() { + flag.Parse() + } + os.Exit(m.Run()) +} diff --git a/.test-infra/mock-apis/src/main/go/test/integration/vars.go b/.test-infra/mock-apis/src/main/go/test/integration/vars.go new file mode 100644 index 000000000000..92ba445fdfa3 --- /dev/null +++ b/.test-infra/mock-apis/src/main/go/test/integration/vars.go @@ -0,0 +1,49 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package integration + +import ( + "flag" + "fmt" +) + +const ( + grpcServiceEndpointFlag = "grpc_service_endpoint" + httpServiceEndpointFlag = "http_service_endpoint" + + moreInfoUrl = "https://github.com/apache/beam/tree/master/.test-infra/mock-apis#writing-integration-tests" +) + +var ( + moreInfo = fmt.Sprintf("See %s for more information on how to get the relevant value for your test.", moreInfoUrl) + + requiredFlags = []string{ + grpcServiceEndpointFlag, + httpServiceEndpointFlag, + } +) + +// The following flags apply to one or more integration tests and used via +// go test ./src/main/go/test/integration/... +var ( + // GRPCServiceEndpoint is the address of the deployed service. + GRPCServiceEndpoint = flag.String(grpcServiceEndpointFlag, "", + "The endpoint to target gRPC calls to a service. "+moreInfo) + + // HTTPServiceEndpoint is the address of the deployed service. + HTTPServiceEndpoint = flag.String(httpServiceEndpointFlag, "", + "The endpoint to target HTTP calls to a service. "+moreInfo) +) From e03988e5e494bfeec88c7c33e7460f7f6d3fde81 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 1 Nov 2023 13:21:36 -0400 Subject: [PATCH 341/435] Bump github.com/aws/aws-sdk-go-v2/feature/s3/manager in /sdks (#29241) Bumps [github.com/aws/aws-sdk-go-v2/feature/s3/manager](https://github.com/aws/aws-sdk-go-v2) from 1.11.91 to 1.12.0. - [Release notes](https://github.com/aws/aws-sdk-go-v2/releases) - [Changelog](https://github.com/aws/aws-sdk-go-v2/blob/v1.12.0/CHANGELOG.md) - [Commits](https://github.com/aws/aws-sdk-go-v2/compare/feature/s3/manager/v1.11.91...v1.12.0) --- updated-dependencies: - dependency-name: github.com/aws/aws-sdk-go-v2/feature/s3/manager dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] <support@github.com> Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 14 +++++++------- sdks/go.sum | 41 ++++++++++++++--------------------------- 2 files changed, 21 insertions(+), 34 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 3ab88acd0049..9afad643c1d4 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -33,8 +33,8 @@ require ( github.com/aws/aws-sdk-go-v2 v1.22.0 github.com/aws/aws-sdk-go-v2/config v1.20.0 github.com/aws/aws-sdk-go-v2/credentials v1.14.0 - github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.91 - github.com/aws/aws-sdk-go-v2/service/s3 v1.40.2 + github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.12.0 + github.com/aws/aws-sdk-go-v2/service/s3 v1.41.0 github.com/aws/smithy-go v1.16.0 github.com/docker/go-connections v0.4.0 github.com/dustin/go-humanize v1.0.1 @@ -97,16 +97,16 @@ require ( github.com/apache/arrow/go/v12 v12.0.0 // indirect github.com/apache/thrift v0.16.0 // indirect github.com/aws/aws-sdk-go v1.34.0 // indirect - github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.4.14 // indirect + github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.5.0 // indirect github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.0 // indirect github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.0 // indirect github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.0 // indirect github.com/aws/aws-sdk-go-v2/internal/ini v1.4.0 // indirect - github.com/aws/aws-sdk-go-v2/internal/v4a v1.1.6 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.9.15 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.1.38 // indirect + github.com/aws/aws-sdk-go-v2/internal/v4a v1.2.0 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.10.0 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.2.0 // indirect github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.0 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.15.6 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.16.0 // indirect github.com/aws/aws-sdk-go-v2/service/sso v1.16.0 // indirect github.com/aws/aws-sdk-go-v2/service/ssooidc v1.18.0 // indirect github.com/aws/aws-sdk-go-v2/service/sts v1.24.0 // indirect diff --git a/sdks/go.sum b/sdks/go.sum index 2622786d8e70..59db69123004 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -81,66 +81,54 @@ github.com/aws/aws-sdk-go v1.30.19/go.mod h1:5zCpMtNQVjRREroY7sYe8lOMRSxkhG6MZve github.com/aws/aws-sdk-go v1.34.0 h1:brux2dRrlwCF5JhTL7MUT3WUwo9zfDHZZp3+g3Mvlmo= github.com/aws/aws-sdk-go v1.34.0/go.mod h1:5zCpMtNQVjRREroY7sYe8lOMRSxkhG6MZveU8YkpAk0= github.com/aws/aws-sdk-go-v2 v1.7.1/go.mod h1:L5LuPC1ZgDr2xQS7AmIec/Jlc7O/Y1u2KxJyNVab250= -github.com/aws/aws-sdk-go-v2 v1.21.2/go.mod h1:ErQhvNuEMhJjweavOYhxVkn2RUx7kQXVATHrjKtxIpM= github.com/aws/aws-sdk-go-v2 v1.22.0 h1:CpTS3XO3MWNel8ohoazkLZC6scvkYL2k+m0yzFJ17Hg= github.com/aws/aws-sdk-go-v2 v1.22.0/go.mod h1:Kd0OJtkW3Q0M0lUWGszapWjEvrXDzRW+D21JNsroB+c= -github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.4.14 h1:Sc82v7tDQ/vdU1WtuSyzZ1I7y/68j//HJ6uozND1IDs= -github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.4.14/go.mod h1:9NCTOURS8OpxvoAVHq79LK81/zC78hfRWFn+aL0SPcY= +github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.5.0 h1:hHgLiIrTRtddC0AKcJr5s7i/hLgcpTt+q/FKxf1Zayk= +github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.5.0/go.mod h1:w4I/v3NOWgD+qvs1NPEwhd++1h3XPHFaVxasfY6HlYQ= github.com/aws/aws-sdk-go-v2/config v1.5.0/go.mod h1:RWlPOAW3E3tbtNAqTwvSW54Of/yP3oiZXMI0xfUdjyA= -github.com/aws/aws-sdk-go-v2/config v1.19.0/go.mod h1:ZwDUgFnQgsazQTnWfeLWk5GjeqTQTL8lMkoE1UXzxdE= github.com/aws/aws-sdk-go-v2/config v1.20.0 h1:q2+/mqFhY0J9m3Tb5RGFE3R4sdaUkIe4k2EuDfE3c08= github.com/aws/aws-sdk-go-v2/config v1.20.0/go.mod h1:7+1riCZXyT+sAGvneR5j+Zl1GyfbBUNQurpQTE6FP6k= github.com/aws/aws-sdk-go-v2/credentials v1.3.1/go.mod h1:r0n73xwsIVagq8RsxmZbGSRQFj9As3je72C2WzUIToc= -github.com/aws/aws-sdk-go-v2/credentials v1.13.43/go.mod h1:zWJBz1Yf1ZtX5NGax9ZdNjhhI4rgjfgsyk6vTY1yfVg= github.com/aws/aws-sdk-go-v2/credentials v1.14.0 h1:LQquqPE7cL55RQmA/UBoBKehDlEtMnQKm3B0Q672ePE= github.com/aws/aws-sdk-go-v2/credentials v1.14.0/go.mod h1:q/3oaTPlamrQWHPwJe56Mjq9g1TYDgddvgTgWJtHTmE= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.3.0/go.mod h1:2LAuqPx1I6jNfaGDucWfA2zqQCYCOMCDHiCOciALyNw= -github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.13.13/go.mod h1:f/Ib/qYjhV2/qdsf79H3QP/eRE4AkVyEf6sk7XfZ1tg= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.0 h1:lF/cVllNAPKgjDwN2RsQUX9g/f6hXer9f10ubLFSoug= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.0/go.mod h1:c28nJNzMVVb9TQpZ5q4tzZvwEJwf/7So7Ie2s90l1Fw= github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.3.2/go.mod h1:qaqQiHSrOUVOfKe6fhgQ6UzhxjwqVW8aHNegd6Ws4w4= -github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.91 h1:haAyxKHwoE+y/TJt+qHcPQf1dCViyyGbWcKjjYUllTE= -github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.91/go.mod h1:ACQ6ta5YFlfSOz2c9A+EVYawLxFMZ0rI3Q0A0tGieKo= -github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.43/go.mod h1:auo+PiyLl0n1l8A0e8RIeR8tOzYPfZZH/JNlrJ8igTQ= +github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.12.0 h1:IJ1PN7TqW79Dmc0hSSd+clznCToXmN8gtWQO1hOxFbE= +github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.12.0/go.mod h1:Y0Ksox8/7Bl2UTwxdxWQ0JfRQi4sMv48F8Iy4ueS/Dc= github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.0 h1:tN6dNNE4SzMuyMnVtQJXGVKX177/d5Zy4MuA1HA4KUc= github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.0/go.mod h1:F6MXWETIeetAHwFHyoHEqrcB3NpijFv9nLP5h9CXtT0= -github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.37/go.mod h1:Qe+2KtKml+FEsQF/DHmDV+xjtche/hwoF75EG4UlHW8= github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.0 h1:bfdsbTARDjaC/dSYGMO+E0psxFU4hTvCLnqYAfZ3D38= github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.0/go.mod h1:Jg8XVv5M2V2wiAMvBFx+O59jg6Yr8vhP0bgNF/IuquM= github.com/aws/aws-sdk-go-v2/internal/ini v1.1.1/go.mod h1:Zy8smImhTdOETZqfyn01iNOe0CNggVbPjCajyaz6Gvg= -github.com/aws/aws-sdk-go-v2/internal/ini v1.3.45/go.mod h1:lD5M20o09/LCuQ2mE62Mb/iSdSlCNuj6H5ci7tW7OsE= github.com/aws/aws-sdk-go-v2/internal/ini v1.4.0 h1:21tlTXq3ev10yLMAjXZzpkZbrl49h3ElSjmxD57tD/E= github.com/aws/aws-sdk-go-v2/internal/ini v1.4.0/go.mod h1:d9YrBHJhyzDCv5UsEVRizHlFV6Q0sLemFq6uxuqWfUw= -github.com/aws/aws-sdk-go-v2/internal/v4a v1.1.6 h1:wmGLw2i8ZTlHLw7a9ULGfQbuccw8uIiNr6sol5bFzc8= -github.com/aws/aws-sdk-go-v2/internal/v4a v1.1.6/go.mod h1:Q0Hq2X/NuL7z8b1Dww8rmOFl+jzusKEcyvkKspwdpyc= +github.com/aws/aws-sdk-go-v2/internal/v4a v1.2.0 h1:NpsAO1LaZyc72xMoQB/qgcOwI9Ag1d5FvaEp+omzFqg= +github.com/aws/aws-sdk-go-v2/internal/v4a v1.2.0/go.mod h1:6WVV80e6jigvvX0QqFDx3tjUME7qtNV9AJqGAZyc/R8= github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.2.1/go.mod h1:v33JQ57i2nekYTA70Mb+O18KeH4KqhdqxTJZNK1zdRE= -github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.9.15 h1:7R8uRYyXzdD71KWVCL78lJZltah6VVznXBazvKjfH58= -github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.9.15/go.mod h1:26SQUPcTNgV1Tapwdt4a1rOsYRsnBsJHLMPoxK2b0d8= -github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.1.38 h1:skaFGzv+3kA+v2BPKhuekeb1Hbb105+44r8ASC+q5SE= -github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.1.38/go.mod h1:epIZoRSSbRIwLPJU5F+OldHhwZPBdpDeQkRdCeY3+00= +github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.10.0 h1:CJxo7ZBbaIzmXfV3hjcx36n9V87gJsIUPJflwqEHl3Q= +github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.10.0/go.mod h1:yjVfjuY4nD1EW9i387Kau+I6V5cBA5YnC/mWNopjZrI= +github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.2.0 h1:Wnw0IZKxx/PlKWbrUssl3Z2FP7cJS30QAeN1MuDVh1Q= +github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.2.0/go.mod h1:3ZHaPNnLwe0E+gAvdlA3Tl7M3SHohQloXL4hiDk1nec= github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.2.1/go.mod h1:zceowr5Z1Nh2WVP8bf/3ikB41IZW59E4yIYbg+pC6mw= -github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.37/go.mod h1:vBmDnwWXWxNPFRMmG2m/3MKOe+xEcMDo1tanpaWCcck= github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.0 h1:dJnwy5Awv+uvfk73aRENVbv1cSQQ60ydCkPaun097KM= github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.0/go.mod h1:RsPWWy7u/hwmFX57sQ7MLvrvJeYyNkiMm5BaavpoU18= github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.5.1/go.mod h1:6EQZIwNNvHpq/2/QSJnp4+ECvqIy55w95Ofs0ze+nGQ= -github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.15.6 h1:9ulSU5ClouoPIYhDQdg9tpl83d5Yb91PXTKK+17q+ow= -github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.15.6/go.mod h1:lnc2taBsR9nTlz9meD+lhFZZ9EWY712QHrRflWpTcOA= +github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.16.0 h1:Kcts/mLwm4LxbF8YULGzilm+IeI4cBoesFUJpSsCcx0= +github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.16.0/go.mod h1:ca7+eyqTRByXctslWXXqhSwItEk+4y30azwLYJBBlsU= github.com/aws/aws-sdk-go-v2/service/s3 v1.11.1/go.mod h1:XLAGFrEjbvMCLvAtWLLP32yTv8GpBquCApZEycDLunI= -github.com/aws/aws-sdk-go-v2/service/s3 v1.40.2 h1:Ll5/YVCOzRB+gxPqs2uD0R7/MyATC0w85626glSKmp4= -github.com/aws/aws-sdk-go-v2/service/s3 v1.40.2/go.mod h1:Zjfqt7KhQK+PO1bbOsFNzKgaq7TcxzmEoDWN8lM0qzQ= +github.com/aws/aws-sdk-go-v2/service/s3 v1.41.0 h1:XJfQwEGLnoqdVQtf+faXr7DMm/Q65SkgnGZJKWxIkPc= +github.com/aws/aws-sdk-go-v2/service/s3 v1.41.0/go.mod h1:DDsTwoAqmg5h2Up70/2XeCA4woeYdaD71PRucUwltFM= github.com/aws/aws-sdk-go-v2/service/sso v1.3.1/go.mod h1:J3A3RGUvuCZjvSuZEcOpHDnzZP/sKbhDWV2T1EOzFIM= -github.com/aws/aws-sdk-go-v2/service/sso v1.15.2/go.mod h1:gsL4keucRCgW+xA85ALBpRFfdSLH4kHOVSnLMSuBECo= github.com/aws/aws-sdk-go-v2/service/sso v1.16.0 h1:ZIlR6Wr/EgYwBdEz1NWBqdUsTh0mV7A68pId3YZl6H0= github.com/aws/aws-sdk-go-v2/service/sso v1.16.0/go.mod h1:O7B5cpuhhJKefAKkM7onb0McmpHyKnsH4RrHJhOyq7M= -github.com/aws/aws-sdk-go-v2/service/ssooidc v1.17.3/go.mod h1:a7bHA82fyUXOm+ZSWKU6PIoBxrjSprdLoM8xPYvzYVg= github.com/aws/aws-sdk-go-v2/service/ssooidc v1.18.0 h1:3BZyJei4k1SHdSAFhg9Qg15NnG3v5zosZyFWPm7df/A= github.com/aws/aws-sdk-go-v2/service/ssooidc v1.18.0/go.mod h1:Td8EvzggonY02wLaqSpwybI3GbmA0PWoprKGil2uwJg= github.com/aws/aws-sdk-go-v2/service/sts v1.6.0/go.mod h1:q7o0j7d7HrJk/vr9uUt3BVRASvcU7gYZB9PUgPiByXg= -github.com/aws/aws-sdk-go-v2/service/sts v1.23.2/go.mod h1:Eows6e1uQEsc4ZaHANmsPRzAKcVDrcmjjWiih2+HUUQ= github.com/aws/aws-sdk-go-v2/service/sts v1.24.0 h1:f/V5Y9OaHuNRrA9MntNQNAtMFXqhKj8HTEPnH81eXMI= github.com/aws/aws-sdk-go-v2/service/sts v1.24.0/go.mod h1:HnCUMNz2XqwnEEk5X6oeDYB2HgOLFpJ/LyfilN8WErs= github.com/aws/smithy-go v1.6.0/go.mod h1:SObp3lf9smib00L/v3U2eAKG8FyQ7iLrJnQiAmR5n+E= -github.com/aws/smithy-go v1.15.0/go.mod h1:Tg+OJXh4MB2R/uN61Ko2f6hTZwB/ZYGOtib8J3gBHzA= github.com/aws/smithy-go v1.16.0 h1:gJZEH/Fqh+RsvlJ1Zt4tVAtV6bKkp3cC+R6FCZMNzik= github.com/aws/smithy-go v1.16.0/go.mod h1:NukqUGpCZIILqqiV0NIjeFh24kd/FAa4beRb6nbIUPE= github.com/cenkalti/backoff/v4 v4.2.1 h1:y4OZtCnogmCPw98Zjyt5a6+QwPLGkiQsYW5oUqylYbM= @@ -270,7 +258,6 @@ github.com/google/go-cmp v0.5.3/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/ github.com/google/go-cmp v0.5.4/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.6/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.5.8/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= github.com/google/go-cmp v0.5.9/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= From dc242135da26d491ca3c2790e6c8cf6f5dbf664a Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 1 Nov 2023 13:37:34 -0400 Subject: [PATCH 342/435] Bump google.golang.org/grpc from 1.54.0 to 1.56.3 in /playground/backend (#29145) Bumps [google.golang.org/grpc](https://github.com/grpc/grpc-go) from 1.54.0 to 1.56.3. - [Release notes](https://github.com/grpc/grpc-go/releases) - [Commits](https://github.com/grpc/grpc-go/compare/v1.54.0...v1.56.3) --- updated-dependencies: - dependency-name: google.golang.org/grpc dependency-type: direct:production ... Signed-off-by: dependabot[bot] <support@github.com> Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- playground/backend/go.mod | 18 +++++++++--------- playground/backend/go.sum | 27 ++++++++++++++++++--------- 2 files changed, 27 insertions(+), 18 deletions(-) diff --git a/playground/backend/go.mod b/playground/backend/go.mod index 9f5fb433ab7e..bcb3a78fbd4e 100644 --- a/playground/backend/go.mod +++ b/playground/backend/go.mod @@ -18,7 +18,7 @@ module beam.apache.org/playground/backend go 1.20 require ( - cloud.google.com/go/datastore v1.10.0 + cloud.google.com/go/datastore v1.11.0 cloud.google.com/go/logging v1.7.0 github.com/GoogleCloudPlatform/functions-framework-go v1.6.1 github.com/confluentinc/confluent-kafka-go/v2 v2.1.1 @@ -33,16 +33,16 @@ require ( github.com/spf13/viper v1.14.0 github.com/stretchr/testify v1.8.2 go.uber.org/goleak v1.2.0 - google.golang.org/grpc v1.54.0 + google.golang.org/grpc v1.56.3 google.golang.org/protobuf v1.30.0 gopkg.in/yaml.v3 v3.0.1 ) require ( cloud.google.com/go v0.110.0 // indirect - cloud.google.com/go/compute v1.19.0 // indirect + cloud.google.com/go/compute v1.19.1 // indirect cloud.google.com/go/compute/metadata v0.2.3 // indirect - cloud.google.com/go/functions v1.12.0 // indirect + cloud.google.com/go/functions v1.13.0 // indirect cloud.google.com/go/longrunning v0.4.1 // indirect github.com/cenkalti/backoff/v4 v4.1.3 // indirect github.com/cespare/xxhash/v2 v2.2.0 // indirect @@ -75,15 +75,15 @@ require ( go.uber.org/atomic v1.9.0 // indirect go.uber.org/multierr v1.8.0 // indirect go.uber.org/zap v1.21.0 // indirect - golang.org/x/net v0.8.0 // indirect - golang.org/x/oauth2 v0.6.0 // indirect + golang.org/x/net v0.9.0 // indirect + golang.org/x/oauth2 v0.7.0 // indirect golang.org/x/sync v0.1.0 // indirect - golang.org/x/sys v0.6.0 // indirect - golang.org/x/text v0.8.0 // indirect + golang.org/x/sys v0.7.0 // indirect + golang.org/x/text v0.9.0 // indirect golang.org/x/xerrors v0.0.0-20220907171357-04be3eba64a2 // indirect google.golang.org/api v0.114.0 // indirect google.golang.org/appengine v1.6.7 // indirect - google.golang.org/genproto v0.0.0-20230331144136-dcfb400f0633 // indirect + google.golang.org/genproto v0.0.0-20230410155749-daa745c078e1 // indirect gopkg.in/ini.v1 v1.67.0 // indirect gopkg.in/yaml.v2 v2.4.0 // indirect nhooyr.io/websocket v1.8.6 // indirect diff --git a/playground/backend/go.sum b/playground/backend/go.sum index 779dc5a38c1f..516388aeb276 100644 --- a/playground/backend/go.sum +++ b/playground/backend/go.sum @@ -168,8 +168,9 @@ cloud.google.com/go/compute v1.13.0/go.mod h1:5aPTS0cUNMIc1CE546K+Th6weJUNQErARy cloud.google.com/go/compute v1.14.0/go.mod h1:YfLtxrj9sU4Yxv+sXzZkyPjEyPBZfXHUvjxega5vAdo= cloud.google.com/go/compute v1.15.1/go.mod h1:bjjoF/NtFUrkD/urWfdHaKuOPDR5nWIs63rR+SXhcpA= cloud.google.com/go/compute v1.18.0/go.mod h1:1X7yHxec2Ga+Ss6jPyjxRxpu2uu7PLgsOVXvgU0yacs= -cloud.google.com/go/compute v1.19.0 h1:+9zda3WGgW1ZSTlVppLCYFIr48Pa35q1uG2N1itbCEQ= cloud.google.com/go/compute v1.19.0/go.mod h1:rikpw2y+UMidAe9tISo04EHNOIf42RLYF/q8Bs93scU= +cloud.google.com/go/compute v1.19.1 h1:am86mquDUgjGNWxiGn+5PGLbmgiWXlE/yNWpIpNvuXY= +cloud.google.com/go/compute v1.19.1/go.mod h1:6ylj3a05WF8leseCdIf77NK0g1ey+nj5IKd5/kvShxE= cloud.google.com/go/compute/metadata v0.1.0/go.mod h1:Z1VN+bulIf6bt4P/C37K4DyZYZEXYonfTBHHFPO/4UU= cloud.google.com/go/compute/metadata v0.2.0/go.mod h1:zFmK7XCadkQkj6TtorcaGlCW1hT1fIilQDwofLpJ20k= cloud.google.com/go/compute/metadata v0.2.1/go.mod h1:jgHgmJd2RKBGzXqF5LR2EZMGxBkeanZ9wwa75XHJgOM= @@ -220,8 +221,9 @@ cloud.google.com/go/dataqna v0.6.0/go.mod h1:1lqNpM7rqNLVgWBJyk5NF6Uen2PHym0jtVJ cloud.google.com/go/dataqna v0.7.0/go.mod h1:Lx9OcIIeqCrw1a6KdO3/5KMP1wAmTc0slZWwP12Qq3c= cloud.google.com/go/datastore v1.0.0/go.mod h1:LXYbyblFSglQ5pkeyhO+Qmw7ukd3C+pD7TKLgZqpHYE= cloud.google.com/go/datastore v1.1.0/go.mod h1:umbIZjpQpHh4hmRpGhH4tLFup+FVzqBi1b3c64qFpCk= -cloud.google.com/go/datastore v1.10.0 h1:4siQRf4zTiAVt/oeH4GureGkApgb2vtPQAtOmhpqQwE= cloud.google.com/go/datastore v1.10.0/go.mod h1:PC5UzAmDEkAmkfaknstTYbNpgE49HAgW2J1gcgUfmdM= +cloud.google.com/go/datastore v1.11.0 h1:iF6I/HaLs3Ado8uRKMvZRvF/ZLkWaWE9i8AiHzbC774= +cloud.google.com/go/datastore v1.11.0/go.mod h1:TvGxBIHCS50u8jzG+AW/ppf87v1of8nwzFNgEZU1D3c= cloud.google.com/go/datastream v1.2.0/go.mod h1:i/uTP8/fZwgATHS/XFu0TcNUhuA0twZxxQ3EyCUQMwo= cloud.google.com/go/datastream v1.3.0/go.mod h1:cqlOX8xlyYF/uxhiKn6Hbv6WjwPPuI9W2M9SAXwaLLQ= cloud.google.com/go/datastream v1.4.0/go.mod h1:h9dpzScPhDTs5noEMQVWP8Wx8AFBRyS0s8KWPx/9r0g= @@ -276,8 +278,9 @@ cloud.google.com/go/functions v1.7.0/go.mod h1:+d+QBcWM+RsrgZfV9xo6KfA1GlzJfxcfZ cloud.google.com/go/functions v1.8.0/go.mod h1:RTZ4/HsQjIqIYP9a9YPbU+QFoQsAlYgrwOXJWHn1POY= cloud.google.com/go/functions v1.9.0/go.mod h1:Y+Dz8yGguzO3PpIjhLTbnqV1CWmgQ5UwtlpzoyquQ08= cloud.google.com/go/functions v1.10.0/go.mod h1:0D3hEOe3DbEvCXtYOZHQZmD+SzYsi1YbI7dGvHfldXw= -cloud.google.com/go/functions v1.12.0 h1:TtRl25/oNsZyH3e4WfMRSMmFvmHC3YyQZuWaOpKI9+0= cloud.google.com/go/functions v1.12.0/go.mod h1:AXWGrF3e2C/5ehvwYo/GH6O5s09tOPksiKhz+hH8WkA= +cloud.google.com/go/functions v1.13.0 h1:pPDqtsXG2g9HeOQLoquLbmvmb82Y4Ezdo1GXuotFoWg= +cloud.google.com/go/functions v1.13.0/go.mod h1:EU4O007sQm6Ef/PwRsI8N2umygGqPBS/IZQKBQBcJ3c= cloud.google.com/go/gaming v1.5.0/go.mod h1:ol7rGcxP/qHTRQE/RO4bxkXq+Fix0j6D4LFPzYTIrDM= cloud.google.com/go/gaming v1.6.0/go.mod h1:YMU1GEvA39Qt3zWGyAVA9bpYz/yAhTvaQ1t2sK4KPUA= cloud.google.com/go/gaming v1.7.0/go.mod h1:LrB8U7MHdGgFG851iHAfqUdLcKBdQ55hzXy9xBJz0+w= @@ -1941,8 +1944,9 @@ golang.org/x/net v0.2.0/go.mod h1:KqCZLdyyvdV855qA2rE3GC2aiw5xGR5TEjj8smXukLY= golang.org/x/net v0.5.0/go.mod h1:DivGGAXEgPSlEBzxGzZI+ZLohi+xUj054jfeKui00ws= golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.7.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= -golang.org/x/net v0.8.0 h1:Zrh2ngAOFYneWTAIAPethzeaQLuHwhuBkuV6ZiRnUaQ= golang.org/x/net v0.8.0/go.mod h1:QVkue5JL9kW//ek3r6jTKnTFis1tRmNAW2P1shuFdJc= +golang.org/x/net v0.9.0 h1:aWJ/m6xSmxWBx+V0XRHTlrYrPG56jKsLdTFmsSsCzOM= +golang.org/x/net v0.9.0/go.mod h1:d48xBJpPfHeWQsugry2m+kC02ZBRGRgulfHnEXEuWns= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= @@ -1970,8 +1974,9 @@ golang.org/x/oauth2 v0.0.0-20221006150949-b44042a4b9c1/go.mod h1:h4gKUeWbJ4rQPri golang.org/x/oauth2 v0.0.0-20221014153046-6fdb5e3db783/go.mod h1:h4gKUeWbJ4rQPri7E0u6Gs4e9Ri2zaLxzw5DI5XGrYg= golang.org/x/oauth2 v0.4.0/go.mod h1:RznEsdpjGAINPTOF0UH/t+xJ75L18YO3Ho6Pyn+uRec= golang.org/x/oauth2 v0.5.0/go.mod h1:9/XBHVqLaWO3/BRHs5jbpYCnOZVjj5V0ndyaAM7KB4I= -golang.org/x/oauth2 v0.6.0 h1:Lh8GPgSKBfWSwFvtuWOfeI3aAAnbXTSutYxJiOJFgIw= golang.org/x/oauth2 v0.6.0/go.mod h1:ycmewcwgD4Rpr3eZJLSB4Kyyljb3qDh40vJ8STE5HKw= +golang.org/x/oauth2 v0.7.0 h1:qe6s0zUXlPX80/dITx3440hWZ7GwMwgDDyrSGTPJG/g= +golang.org/x/oauth2 v0.7.0/go.mod h1:hPLQkd9LyjfXTiRohC/41GhcFqxisoUQ99sCUOHO9x4= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -2124,8 +2129,9 @@ golang.org/x/sys v0.0.0-20220908164124-27713097b956/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.2.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.4.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.6.0 h1:MVltZSvRTcU2ljQOhs94SXPftV6DCNnZViHeQps87pQ= golang.org/x/sys v0.6.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.7.0 h1:3jlCCIQZPdOYu1h8BkNvLz8Kgwtae2cagcG/VamtZRU= +golang.org/x/sys v0.7.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/term v0.0.0-20201117132131-f5c789dd3221/go.mod h1:Nr5EML6q2oocZ2LXRh80K7BxOlk5/8JxuGnuhpl+muw= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210220032956-6a3ed077a48d/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= @@ -2150,8 +2156,9 @@ golang.org/x/text v0.4.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= golang.org/x/text v0.5.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= golang.org/x/text v0.6.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= golang.org/x/text v0.7.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= -golang.org/x/text v0.8.0 h1:57P1ETyNKtuIjB4SRd15iJxuhj8Gc416Y78H3qgMh68= golang.org/x/text v0.8.0/go.mod h1:e1OnstbJyHTd6l/uOt8jFFHp6TRDWZR/bV3emEE/zU8= +golang.org/x/text v0.9.0 h1:2sjJmO8cDvYveuX97RDLsxlyUxLl+GHoLxBiRdHllBE= +golang.org/x/text v0.9.0/go.mod h1:e1OnstbJyHTd6l/uOt8jFFHp6TRDWZR/bV3emEE/zU8= golang.org/x/time v0.0.0-20180412165947-fbb02b2291d2/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= @@ -2468,8 +2475,9 @@ google.golang.org/genproto v0.0.0-20230223222841-637eb2293923/go.mod h1:3Dl5ZL0q google.golang.org/genproto v0.0.0-20230303212802-e74f57abe488/go.mod h1:TvhZT5f700eVlTNwND1xoEZQeWTB2RY/65kplwl/bFA= google.golang.org/genproto v0.0.0-20230306155012-7f2fa6fef1f4/go.mod h1:NWraEVixdDnqcqQ30jipen1STv2r/n24Wb7twVTGR4s= google.golang.org/genproto v0.0.0-20230320184635-7606e756e683/go.mod h1:NWraEVixdDnqcqQ30jipen1STv2r/n24Wb7twVTGR4s= -google.golang.org/genproto v0.0.0-20230331144136-dcfb400f0633 h1:0BOZf6qNozI3pkN3fJLwNubheHJYHhMh91GRFOWWK08= google.golang.org/genproto v0.0.0-20230331144136-dcfb400f0633/go.mod h1:UUQDJDOlWu4KYeJZffbWgBkS1YFobzKbLVfK69pe0Ak= +google.golang.org/genproto v0.0.0-20230410155749-daa745c078e1 h1:KpwkzHKEF7B9Zxg18WzOa7djJ+Ha5DzthMyZYQfEn2A= +google.golang.org/genproto v0.0.0-20230410155749-daa745c078e1/go.mod h1:nKE/iIaLqn2bQwXBg8f1g2Ylh6r5MN5CmZvuzZCgsCU= google.golang.org/grpc v0.0.0-20160317175043-d3ddb4469d5a/go.mod h1:yo6s7OP7yaDglbqo1J04qKzAhqBH6lvTonzMVmEdcZw= google.golang.org/grpc v1.17.0/go.mod h1:6QZJwpn2B+Zp71q/5VxRsJ6NXXVCE5NRUHRo+f3cWCs= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= @@ -2517,8 +2525,9 @@ google.golang.org/grpc v1.50.0/go.mod h1:ZgQEeidpAuNRZ8iRrlBKXZQP1ghovWIVhdJRyCD google.golang.org/grpc v1.50.1/go.mod h1:ZgQEeidpAuNRZ8iRrlBKXZQP1ghovWIVhdJRyCDK+GI= google.golang.org/grpc v1.51.0/go.mod h1:wgNDFcnuBGmxLKI/qn4T+m5BtEBYXJPvibbUPsAIPww= google.golang.org/grpc v1.53.0/go.mod h1:OnIrk0ipVdj4N5d9IUoFUx72/VlD7+jUsHwZgwSMQpw= -google.golang.org/grpc v1.54.0 h1:EhTqbhiYeixwWQtAEZAxmV9MGqcjEU2mFx52xCzNyag= google.golang.org/grpc v1.54.0/go.mod h1:PUSEXI6iWghWaB6lXM4knEgpJNu2qUcKfDtNci3EC2g= +google.golang.org/grpc v1.56.3 h1:8I4C0Yq1EjstUzUJzpcRVbuYA2mODtEmpWiQoN/b2nc= +google.golang.org/grpc v1.56.3/go.mod h1:I9bI3vqKfayGqPUAwGdOSu7kt6oIJLixfffKrpXqQ9s= google.golang.org/grpc/cmd/protoc-gen-go-grpc v1.1.0/go.mod h1:6Kw0yEErY5E/yWrBtf03jp27GLLJujG4z/JK95pnjjw= google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= From 59e6e0017ba3c3d641a58d2be9d5ab2270a4fc08 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw <robertwb@gmail.com> Date: Wed, 1 Nov 2023 10:45:44 -0700 Subject: [PATCH 343/435] Better error on late use of DoFnContext. --- .../main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java index 1800e997b2d8..ddf52125b2e4 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java @@ -2503,6 +2503,10 @@ private class NonWindowObservingProcessBundleContext @Override public void output(OutputT output) { // Don't need to check timestamp since we can always output using the input timestamp. + if (currentElement == null) { + throw new IllegalStateException( + "Attempting to emit an element outside of a @ProcessElement context."); + } outputTo(mainOutputConsumer, currentElement.withValue(output)); } From 1f5bddcd2b72a645565bf7a7a2500ba7386b8f3e Mon Sep 17 00:00:00 2001 From: Arun Pandian <arunpandianp@gmail.com> Date: Wed, 1 Nov 2023 11:11:41 -0700 Subject: [PATCH 344/435] Improve Coder encode/decode documentation. (#28596) * Improve Coder encode/decode documentation. * fix formatting and typo --------- Co-authored-by: Arun Pandian <pandiana@google.com> --- .../src/main/java/org/apache/beam/sdk/coders/Coder.java | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java index 04bf8af4d187..08e25c6b77e7 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java @@ -112,7 +112,10 @@ public String toString() { } /** - * Encodes the given value of type {@code T} onto the given output stream. + * Encodes the given value of type {@code T} onto the given output stream. Multiple elements can + * be encoded next to each other on the output stream, each coder should encode information to + * know how many bytes to read when decoding. A common approach is to prefix the encoding with the + * element's encoded length. * * @throws IOException if writing to the {@code OutputStream} fails for some reason * @throws CoderException if the value could not be encoded for some reason @@ -134,7 +137,9 @@ public void encode(T value, OutputStream outStream, Context context) /** * Decodes a value of type {@code T} from the given input stream in the given context. Returns the - * decoded value. + * decoded value. Multiple elements can be encoded next to each other on the input stream, each + * coder should encode information to know how many bytes to read when decoding. A common approach + * is to prefix the encoding with the element's encoded length. * * @throws IOException if reading from the {@code InputStream} fails for some reason * @throws CoderException if the value could not be decoded for some reason From decd2b72c2fa144f23f88bc21654d6bd177d332a Mon Sep 17 00:00:00 2001 From: Danny McCormick <dannymccormick@google.com> Date: Wed, 1 Nov 2023 14:28:45 -0400 Subject: [PATCH 345/435] Add trigger_all_tests trigger to postcommits (#29253) --- .github/workflows/beam_Java_JMH.yml | 3 +++ .github/workflows/beam_PostCommit_Go.yml | 3 +++ .github/workflows/beam_PostCommit_Go_Dataflow_ARM.yml | 3 +++ .github/workflows/beam_PostCommit_Go_VR_Flink.yml | 3 +++ .github/workflows/beam_PostCommit_Go_VR_Samza.yml | 3 +++ .github/workflows/beam_PostCommit_Go_VR_Spark.yml | 3 +++ .github/workflows/beam_PostCommit_Java.yml | 3 +++ .github/workflows/beam_PostCommit_Java_Avro_Versions.yml | 3 +++ .../workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml | 3 +++ .github/workflows/beam_PostCommit_Java_DataflowV1.yml | 3 +++ .github/workflows/beam_PostCommit_Java_DataflowV2.yml | 3 +++ .github/workflows/beam_PostCommit_Java_Examples_Dataflow.yml | 3 +++ .../workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml | 5 ++++- .../beam_PostCommit_Java_Examples_Dataflow_Java.yml | 3 +++ .../workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml | 3 +++ .../beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml | 3 +++ .github/workflows/beam_PostCommit_Java_Examples_Direct.yml | 3 +++ .github/workflows/beam_PostCommit_Java_Examples_Flink.yml | 3 +++ .github/workflows/beam_PostCommit_Java_Examples_Spark.yml | 3 +++ .github/workflows/beam_PostCommit_Java_Hadoop_Versions.yml | 3 +++ .../workflows/beam_PostCommit_Java_IO_Performance_Tests.yml | 3 +++ .github/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml | 3 +++ .../workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml | 3 +++ .../workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml | 3 +++ .../workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml | 3 +++ .../workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml | 3 +++ .../workflows/beam_PostCommit_Java_Jpms_Direct_Java21.yml | 3 +++ .github/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml | 3 +++ .github/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml | 3 +++ .github/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml | 3 +++ .../workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml | 3 +++ .../beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml | 3 +++ .github/workflows/beam_PostCommit_Java_Nexmark_Direct.yml | 3 +++ .github/workflows/beam_PostCommit_Java_Nexmark_Flink.yml | 3 +++ .github/workflows/beam_PostCommit_Java_Nexmark_Spark.yml | 3 +++ .../workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml | 3 +++ .github/workflows/beam_PostCommit_Java_PVR_Samza.yml | 3 +++ .../workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml | 3 +++ .github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml | 3 +++ .github/workflows/beam_PostCommit_Java_Sickbay.yml | 3 +++ .github/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml | 3 +++ .github/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml | 3 +++ .github/workflows/beam_PostCommit_Java_Tpcds_Flink.yml | 3 +++ .github/workflows/beam_PostCommit_Java_Tpcds_Spark.yml | 3 +++ .../beam_PostCommit_Java_ValidatesRunner_Dataflow.yml | 3 +++ ...PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml | 3 +++ ...am_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml | 3 +++ .../beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml | 3 +++ ...PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml | 3 +++ .../beam_PostCommit_Java_ValidatesRunner_Direct.yml | 3 +++ ...m_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml | 3 +++ .../workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml | 3 +++ .../beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml | 3 +++ .../workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml | 3 +++ .../workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml | 3 +++ ...tCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml | 3 +++ .../beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml | 3 +++ .../beam_PostCommit_Java_ValidatesRunner_Twister2.yml | 3 +++ .../workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml | 3 +++ .github/workflows/beam_PostCommit_Javadoc.yml | 3 +++ .github/workflows/beam_PostCommit_PortableJar_Flink.yml | 3 +++ .github/workflows/beam_PostCommit_PortableJar_Spark.yml | 3 +++ .github/workflows/beam_PostCommit_Python.yml | 3 +++ .github/workflows/beam_PostCommit_Python_Arm.yml | 3 +++ .../workflows/beam_PostCommit_Python_Examples_Dataflow.yml | 3 +++ .github/workflows/beam_PostCommit_Python_Examples_Direct.yml | 3 +++ .github/workflows/beam_PostCommit_Python_Examples_Flink.yml | 3 +++ .github/workflows/beam_PostCommit_Python_Examples_Spark.yml | 3 +++ .github/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml | 3 +++ .github/workflows/beam_PostCommit_Python_Nexmark_Direct.yml | 3 +++ .../beam_PostCommit_Python_ValidatesContainer_Dataflow.yml | 3 +++ ...PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml | 3 +++ .../beam_PostCommit_Python_ValidatesRunner_Dataflow.yml | 3 +++ .../beam_PostCommit_Python_ValidatesRunner_Flink.yml | 3 +++ .../beam_PostCommit_Python_ValidatesRunner_Samza.yml | 3 +++ .../beam_PostCommit_Python_ValidatesRunner_Spark.yml | 3 +++ .../workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml | 3 +++ .../workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml | 3 +++ .../workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml | 3 +++ .github/workflows/beam_PostCommit_SQL.yml | 3 +++ .github/workflows/beam_PostCommit_Sickbay_Python.yml | 3 +++ .../workflows/beam_PostCommit_TransformService_Direct.yml | 3 +++ .github/workflows/beam_PostCommit_Website_Test.yml | 3 +++ .github/workflows/beam_PostCommit_XVR_Direct.yml | 3 +++ .github/workflows/beam_PostCommit_XVR_Flink.yml | 3 +++ .../workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml | 3 +++ .../beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml | 3 +++ .../beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml | 3 +++ .../beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml | 3 +++ .github/workflows/beam_PostCommit_XVR_Samza.yml | 3 +++ .github/workflows/beam_PostCommit_XVR_Spark3.yml | 3 +++ 91 files changed, 274 insertions(+), 1 deletion(-) diff --git a/.github/workflows/beam_Java_JMH.yml b/.github/workflows/beam_Java_JMH.yml index f7cc8838f911..d05d69cf8ddc 100644 --- a/.github/workflows/beam_Java_JMH.yml +++ b/.github/workflows/beam_Java_JMH.yml @@ -18,6 +18,8 @@ name: Java JMH on: schedule: - cron: '0 0 * * 0' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event @@ -52,6 +54,7 @@ jobs: beam_Java_JMH: if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 900 diff --git a/.github/workflows/beam_PostCommit_Go.yml b/.github/workflows/beam_PostCommit_Go.yml index 8664b386cc37..f9da27ff2ce9 100644 --- a/.github/workflows/beam_PostCommit_Go.yml +++ b/.github/workflows/beam_PostCommit_Go.yml @@ -18,6 +18,8 @@ name: PostCommit Go on: schedule: - cron: '30 3/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event @@ -50,6 +52,7 @@ jobs: beam_PostCommit_Go: if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Go PostCommit' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PostCommit_Go_Dataflow_ARM.yml b/.github/workflows/beam_PostCommit_Go_Dataflow_ARM.yml index dcc1b018f914..d8fd9753f287 100644 --- a/.github/workflows/beam_PostCommit_Go_Dataflow_ARM.yml +++ b/.github/workflows/beam_PostCommit_Go_Dataflow_ARM.yml @@ -24,6 +24,8 @@ on: types: [created] schedule: - cron: '30 3/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event @@ -57,6 +59,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Go PostCommit Dataflow ARM' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PostCommit_Go_VR_Flink.yml b/.github/workflows/beam_PostCommit_Go_VR_Flink.yml index 1d12aed4e798..ce1e82d22e2a 100644 --- a/.github/workflows/beam_PostCommit_Go_VR_Flink.yml +++ b/.github/workflows/beam_PostCommit_Go_VR_Flink.yml @@ -18,6 +18,8 @@ name: PostCommit Go VR Flink on: schedule: - cron: '30 3/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event @@ -50,6 +52,7 @@ jobs: beam_PostCommit_Go_VR_Flink: if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Go Flink ValidatesRunner' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PostCommit_Go_VR_Samza.yml b/.github/workflows/beam_PostCommit_Go_VR_Samza.yml index 6d1f0bb3ef90..5d1ab6a8d13e 100644 --- a/.github/workflows/beam_PostCommit_Go_VR_Samza.yml +++ b/.github/workflows/beam_PostCommit_Go_VR_Samza.yml @@ -18,6 +18,8 @@ name: PostCommit Go VR Samza on: schedule: - cron: '30 3/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event @@ -50,6 +52,7 @@ jobs: beam_PostCommit_Go_VR_Samza: if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Go Samza ValidatesRunner' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PostCommit_Go_VR_Spark.yml b/.github/workflows/beam_PostCommit_Go_VR_Spark.yml index 2008db6a1bbf..ed9f50583133 100644 --- a/.github/workflows/beam_PostCommit_Go_VR_Spark.yml +++ b/.github/workflows/beam_PostCommit_Go_VR_Spark.yml @@ -18,6 +18,8 @@ name: PostCommit Go VR Spark on: schedule: - cron: '30 3/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event @@ -50,6 +52,7 @@ jobs: beam_PostCommit_Go_VR_Spark: if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Go Spark ValidatesRunner' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PostCommit_Java.yml b/.github/workflows/beam_PostCommit_Java.yml index a60e3c7f24f1..d9fa4d52f965 100644 --- a/.github/workflows/beam_PostCommit_Java.yml +++ b/.github/workflows/beam_PostCommit_Java.yml @@ -20,6 +20,8 @@ name: PostCommit Java on: schedule: - cron: '0 5/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs @@ -59,6 +61,7 @@ jobs: job_phrase: [Run Java PostCommit] if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Java PostCommit' steps: diff --git a/.github/workflows/beam_PostCommit_Java_Avro_Versions.yml b/.github/workflows/beam_PostCommit_Java_Avro_Versions.yml index 31107c24f989..36bdec265dd5 100644 --- a/.github/workflows/beam_PostCommit_Java_Avro_Versions.yml +++ b/.github/workflows/beam_PostCommit_Java_Avro_Versions.yml @@ -20,6 +20,8 @@ name: PostCommit Java Avro Versions on: schedule: - cron: '30 3/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs @@ -59,6 +61,7 @@ jobs: job_phrase: [Run Java Avro Versions PostCommit] if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Java Avro Versions PostCommit' steps: diff --git a/.github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml b/.github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml index 20f3ecdcde8b..f21f301cc265 100644 --- a/.github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml +++ b/.github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml @@ -20,6 +20,8 @@ name: PostCommit Java BigQueryEarlyRollout on: schedule: - cron: '30 3/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs @@ -59,6 +61,7 @@ jobs: job_phrase: [Run Java BigQueryEarlyRollout PostCommit] if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Java BigQueryEarlyRollout PostCommit' steps: diff --git a/.github/workflows/beam_PostCommit_Java_DataflowV1.yml b/.github/workflows/beam_PostCommit_Java_DataflowV1.yml index 90ee56c475f5..68c901c2e527 100644 --- a/.github/workflows/beam_PostCommit_Java_DataflowV1.yml +++ b/.github/workflows/beam_PostCommit_Java_DataflowV1.yml @@ -20,6 +20,8 @@ name: PostCommit Java Dataflow V1 on: schedule: - cron: '30 3/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs @@ -59,6 +61,7 @@ jobs: job_phrase: [Run PostCommit_Java_Dataflow] if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run PostCommit_Java_Dataflow' steps: diff --git a/.github/workflows/beam_PostCommit_Java_DataflowV2.yml b/.github/workflows/beam_PostCommit_Java_DataflowV2.yml index 066c5933ec72..90a97296df77 100644 --- a/.github/workflows/beam_PostCommit_Java_DataflowV2.yml +++ b/.github/workflows/beam_PostCommit_Java_DataflowV2.yml @@ -20,6 +20,8 @@ name: PostCommit Java Dataflow V2 on: schedule: - cron: '30 3/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs @@ -59,6 +61,7 @@ jobs: job_phrase: [Run PostCommit_Java_DataflowV2] if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run PostCommit_Java_DataflowV2' steps: diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow.yml index d050b9b1a8c4..54874493e283 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow.yml @@ -22,6 +22,8 @@ on: types: [created] schedule: - cron: '45 3/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs @@ -61,6 +63,7 @@ jobs: job_phrase: [Run Java examples on Dataflow] if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Java Examples on Dataflow' steps: diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml index f8f8ffe617f3..478707b850d4 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml @@ -33,6 +33,8 @@ on: types: [created] schedule: - cron: '45 3/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs @@ -75,7 +77,8 @@ jobs: if: | github.event_name == 'push' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || - github. event_name == 'workflow_dispatch' || + github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || startswith(github.event.comment.body, 'Run Java_Examples_Dataflow_ARM PostCommit') steps: - uses: actions/checkout@v4 diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml index 685b3aa01e4c..54644e27160a 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml @@ -20,6 +20,8 @@ name: PostCommit Java Examples Dataflow Java on: schedule: - cron: '45 3/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs @@ -61,6 +63,7 @@ jobs: java_version: ['11','17'] if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || startswith(github.event.comment.body, 'Run Java examples on Dataflow Java') steps: diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml index 9e2b37ab4570..ae50ec506dff 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml @@ -20,6 +20,8 @@ name: PostCommit Java Examples Dataflow V2 on: schedule: - cron: '45 3/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs @@ -59,6 +61,7 @@ jobs: job_phrase: [Run Java Examples on Dataflow Runner V2] if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Java Examples on Dataflow Runner V2' steps: diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml index a752c939b994..1ed3cf5eb981 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml @@ -20,6 +20,8 @@ name: PostCommit Java Examples Dataflow V2 Java on: schedule: - cron: '45 3/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs @@ -62,6 +64,7 @@ jobs: java_version: ['11', '17', '21'] if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || (contains(github.event.comment.body, 'Run Java') && contains(github.event.comment.body, 'Examples on Dataflow Runner V2')) diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Direct.yml b/.github/workflows/beam_PostCommit_Java_Examples_Direct.yml index 897660cb97cf..fa7e76942808 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Direct.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Direct.yml @@ -20,6 +20,8 @@ name: PostCommit Java Examples Direct on: schedule: - cron: '45 3/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs @@ -59,6 +61,7 @@ jobs: job_phrase: [Run Java Examples_Direct] if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Java Examples_Direct' steps: diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Flink.yml b/.github/workflows/beam_PostCommit_Java_Examples_Flink.yml index b03575950cc4..8e7a99f12ac8 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Flink.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Flink.yml @@ -20,6 +20,8 @@ name: PostCommit Java Examples Flink on: schedule: - cron: '45 3/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs @@ -59,6 +61,7 @@ jobs: job_phrase: [Run Java Examples_Flink] if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Java Examples_Flink' steps: diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Spark.yml b/.github/workflows/beam_PostCommit_Java_Examples_Spark.yml index 42e265e8ae35..f927fa8b3b46 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Spark.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Spark.yml @@ -20,6 +20,8 @@ name: PostCommit Java Examples Spark on: schedule: - cron: '45 3/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs @@ -59,6 +61,7 @@ jobs: job_phrase: [Run Java Examples_Spark] if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Java Examples_Spark' steps: diff --git a/.github/workflows/beam_PostCommit_Java_Hadoop_Versions.yml b/.github/workflows/beam_PostCommit_Java_Hadoop_Versions.yml index f031f87c159e..c0c85fb0cd31 100644 --- a/.github/workflows/beam_PostCommit_Java_Hadoop_Versions.yml +++ b/.github/workflows/beam_PostCommit_Java_Hadoop_Versions.yml @@ -18,6 +18,8 @@ name: PostCommit Java Hadoop Versions on: schedule: - cron: '45 3/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event @@ -57,6 +59,7 @@ jobs: job_phrase: [Run PostCommit_Java_Hadoop_Versions] if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run PostCommit_Java_Hadoop_Versions' steps: diff --git a/.github/workflows/beam_PostCommit_Java_IO_Performance_Tests.yml b/.github/workflows/beam_PostCommit_Java_IO_Performance_Tests.yml index bbb3ced69b70..62fb402df7d0 100644 --- a/.github/workflows/beam_PostCommit_Java_IO_Performance_Tests.yml +++ b/.github/workflows/beam_PostCommit_Java_IO_Performance_Tests.yml @@ -24,6 +24,8 @@ on: types: [created] schedule: - cron: '0 4/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event @@ -57,6 +59,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Java PostCommit IO Performance Tests' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml b/.github/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml index 331a5bdaa305..3e0022ba1bea 100644 --- a/.github/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml +++ b/.github/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml @@ -20,6 +20,8 @@ name: PostCommit Java InfluxDbIO Integration Test on: schedule: - cron: '0 4/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs @@ -60,6 +62,7 @@ jobs: if: | (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || github.event.comment.body == 'Run Java InfluxDbIO_IT' steps: - uses: actions/checkout@v3 diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml index 13e67f0d1123..2449054a1073 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml @@ -18,6 +18,8 @@ name: PostCommit Java Jpms Dataflow Java11 on: schedule: - cron: '0 4/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event @@ -50,6 +52,7 @@ jobs: beam_PostCommit_Java_Jpms_Dataflow_Java11: if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Jpms Dataflow Java 11 PostCommit' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml index 72a5af15f357..611c8a9d31de 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml @@ -18,6 +18,8 @@ name: PostCommit Java Jpms Dataflow Java17 on: schedule: - cron: '0 4/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event @@ -50,6 +52,7 @@ jobs: beam_PostCommit_Java_Jpms_Dataflow_Java17: if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Jpms Dataflow Java 17 PostCommit' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml index 09190a51cb62..6fdff9c451eb 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml @@ -18,6 +18,8 @@ name: PostCommit Java Jpms Direct Java11 on: schedule: - cron: '0 4/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event @@ -50,6 +52,7 @@ jobs: beam_PostCommit_Java_Jpms_Direct_Java11: if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Jpms Direct Java 11 PostCommit' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml index ed1bf0898020..3f62861eb025 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml @@ -18,6 +18,8 @@ name: PostCommit Java Jpms Direct Java17 on: schedule: - cron: '0 4/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event @@ -50,6 +52,7 @@ jobs: beam_PostCommit_Java_Jpms_Direct_Java17: if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Jpms Direct Java 17 PostCommit' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java21.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java21.yml index d300e5ed50b2..e1926a430069 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java21.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java21.yml @@ -18,6 +18,8 @@ name: PostCommit Java Jpms Direct Java21 on: schedule: - cron: '0 */6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event @@ -50,6 +52,7 @@ jobs: beam_PostCommit_Java_Jpms_Direct_Java21: if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Jpms Direct Java 21 PostCommit' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml index b83d229f02f7..fbc0d16e4994 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml @@ -18,6 +18,8 @@ name: PostCommit Java Jpms Flink Java11 on: schedule: - cron: '0 4/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event @@ -50,6 +52,7 @@ jobs: beam_PostCommit_Java_Jpms_Flink_Java11: if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Jpms Flink Java 11 PostCommit' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml index 830a2c0b2f1e..2c2e2acc6f19 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml @@ -18,6 +18,8 @@ name: PostCommit Java Jpms Spark Java11 on: schedule: - cron: '0 4/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event @@ -50,6 +52,7 @@ jobs: beam_PostCommit_Java_Jpms_Spark_Java11: if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Jpms Spark Java 11 PostCommit' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml index 5bf6676e4268..f4e96961061e 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml @@ -20,6 +20,8 @@ name: PostCommit Java Nexmark Dataflow on: schedule: - cron: '15 4/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs @@ -85,6 +87,7 @@ jobs: queryLanguage: [sql, zetasql, none] if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Dataflow Runner Nexmark Tests' steps: diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml index c8de83d4c4cc..875f54ea7632 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml @@ -20,6 +20,8 @@ name: PostCommit Java Nexmark Dataflow V2 on: schedule: - cron: '15 4/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs @@ -85,6 +87,7 @@ jobs: streaming: [false, true] if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Dataflow Runner V2 Nexmark Tests' steps: diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml index 4000f1ce5bd1..ba3f8bb61005 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml @@ -20,6 +20,8 @@ name: PostCommit Java Nexmark Dataflow V2 Java on: schedule: - cron: '0 4/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs @@ -86,6 +88,7 @@ jobs: java_version: ['11','17'] if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || (contains(github.event.comment.body, 'Run Dataflow Runner V2 Java') && contains(github.event.comment.body, 'Nexmark Tests')) diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Direct.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Direct.yml index d6376cfb6ed7..23d766c89823 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Direct.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Direct.yml @@ -20,6 +20,8 @@ name: PostCommit Java Nexmark Direct on: schedule: - cron: '15 4/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs @@ -80,6 +82,7 @@ jobs: queryLanguage: [sql, zetasql, none] if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Direct Runner Nexmark Tests' steps: diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Flink.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Flink.yml index 57a8562c534a..dd87fec145a4 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Flink.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Flink.yml @@ -20,6 +20,8 @@ name: PostCommit Java Nexmark Flink on: schedule: - cron: '15 4/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs @@ -79,6 +81,7 @@ jobs: queryLanguage: [sql, zetasql, none] if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Flink Runner Nexmark Tests' steps: diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Spark.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Spark.yml index fad07f56ce83..05229fb4c89b 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Spark.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Spark.yml @@ -20,6 +20,8 @@ name: PostCommit Java Nexmark Spark on: schedule: - cron: '15 4/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs @@ -79,6 +81,7 @@ jobs: queryLanguage: [sql, zetasql, none] if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Spark Runner Nexmark Tests' steps: diff --git a/.github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml b/.github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml index 979d3937ad59..cffe074f58eb 100644 --- a/.github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml +++ b/.github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml @@ -20,6 +20,8 @@ name: PostCommit Java PVR Flink Streaming on: schedule: - cron: '15 4/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs @@ -59,6 +61,7 @@ jobs: job_phrase: [Run Java Flink PortableValidatesRunner Streaming] if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Java Flink PortableValidatesRunner Streaming' steps: diff --git a/.github/workflows/beam_PostCommit_Java_PVR_Samza.yml b/.github/workflows/beam_PostCommit_Java_PVR_Samza.yml index cebadc256695..729e95fe5219 100644 --- a/.github/workflows/beam_PostCommit_Java_PVR_Samza.yml +++ b/.github/workflows/beam_PostCommit_Java_PVR_Samza.yml @@ -20,6 +20,8 @@ name: PostCommit Java PVR Samza on: schedule: - cron: '15 4/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs @@ -59,6 +61,7 @@ jobs: job_phrase: [Run Java Samza PortableValidatesRunner] if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Java Samza PortableValidatesRunner' steps: diff --git a/.github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml b/.github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml index ba51a8f8af50..d3508e968fe0 100644 --- a/.github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml +++ b/.github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml @@ -20,6 +20,8 @@ name: PostCommit Java PVR Spark3 Streaming on: schedule: - cron: '15 4/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs @@ -59,6 +61,7 @@ jobs: job_phrase: [Run Java Spark v3 PortableValidatesRunner Streaming] if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Java Spark v3 PortableValidatesRunner Streaming' steps: diff --git a/.github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml b/.github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml index eaddb569d4d8..8cc977ddea82 100644 --- a/.github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml +++ b/.github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml @@ -20,6 +20,8 @@ name: PostCommit Java PVR Spark Batch on: schedule: - cron: '15 4/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs @@ -59,6 +61,7 @@ jobs: job_phrase: [Run Java Spark PortableValidatesRunner Batch] if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') steps: - uses: actions/checkout@v4 diff --git a/.github/workflows/beam_PostCommit_Java_Sickbay.yml b/.github/workflows/beam_PostCommit_Java_Sickbay.yml index ceac541bdb8f..e6de4b2538b5 100644 --- a/.github/workflows/beam_PostCommit_Java_Sickbay.yml +++ b/.github/workflows/beam_PostCommit_Java_Sickbay.yml @@ -20,6 +20,8 @@ name: PostCommit Java Sickbay on: schedule: - cron: '30 4/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs @@ -59,6 +61,7 @@ jobs: job_phrase: [Run Java Sickbay] if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Java Sickbay' steps: diff --git a/.github/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml b/.github/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml index 227df1815deb..d718928e2482 100644 --- a/.github/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml +++ b/.github/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml @@ -20,6 +20,8 @@ name: PostCommit Java SingleStoreIO IT on: schedule: - cron: '30 4/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event permissions: @@ -62,6 +64,7 @@ jobs: github.event_name == 'push' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || github.event.comment.body == 'Run Java SingleStoreIO_IT' steps: - uses: actions/checkout@v3 diff --git a/.github/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml b/.github/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml index 75c12362c08b..a7cc537a6998 100644 --- a/.github/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml @@ -18,6 +18,8 @@ name: PostCommit Java Tpcds Dataflow on: schedule: - cron: '30 4/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event @@ -75,6 +77,7 @@ jobs: beam_PostCommit_Java_Tpcds_Dataflow: if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Dataflow Runner Tpcds Tests' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PostCommit_Java_Tpcds_Flink.yml b/.github/workflows/beam_PostCommit_Java_Tpcds_Flink.yml index 9717bc5e1056..a7b9daa56995 100644 --- a/.github/workflows/beam_PostCommit_Java_Tpcds_Flink.yml +++ b/.github/workflows/beam_PostCommit_Java_Tpcds_Flink.yml @@ -18,6 +18,8 @@ name: PostCommit Java Tpcds Flink on: schedule: - cron: '30 4/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event @@ -72,6 +74,7 @@ jobs: beam_PostCommit_Java_Tpcds_Flink: if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Flink Runner Tpcds Tests' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PostCommit_Java_Tpcds_Spark.yml b/.github/workflows/beam_PostCommit_Java_Tpcds_Spark.yml index 805eab350d7c..82dfef660053 100644 --- a/.github/workflows/beam_PostCommit_Java_Tpcds_Spark.yml +++ b/.github/workflows/beam_PostCommit_Java_Tpcds_Spark.yml @@ -18,6 +18,8 @@ name: PostCommit Java Tpcds Spark on: schedule: - cron: '30 4/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event @@ -71,6 +73,7 @@ jobs: beam_PostCommit_Java_Tpcds_Spark: if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Spark Runner Tpcds Tests' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml index f0f1de35719d..338b148ff3fb 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml @@ -20,6 +20,8 @@ name: PostCommit Java ValidatesRunner Dataflow on: schedule: - cron: '30 4/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs @@ -59,6 +61,7 @@ jobs: job_phrase: [Run Dataflow ValidatesRunner] if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Dataflow ValidatesRunner' steps: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml index 461456487d7c..86b6b32a4abb 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml @@ -20,6 +20,8 @@ name: PostCommit Java ValidatesRunner Dataflow JavaVersions on: schedule: - cron: '30 4/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs @@ -61,6 +63,7 @@ jobs: java_version: ['11','17'] if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || startswith(github.event.comment.body, 'Run Dataflow ValidatesRunner Java') steps: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml index 19be1a9e8301..abe21ac3f783 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml @@ -20,6 +20,8 @@ name: PostCommit Java ValidatesRunner Dataflow Streaming on: schedule: - cron: '30 4/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs @@ -59,6 +61,7 @@ jobs: job_phrase: [Run Dataflow Streaming ValidatesRunner] if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Dataflow Streaming ValidatesRunner' steps: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml index b4511fac3e93..63625b48ea2f 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml @@ -20,6 +20,8 @@ name: PostCommit Java ValidatesRunner Dataflow V2 on: schedule: - cron: '30 6/8 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs @@ -59,6 +61,7 @@ jobs: job_phrase: [Run Java Dataflow V2 ValidatesRunner] if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Java Dataflow V2 ValidatesRunner' steps: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml index e89a0414a3dc..b372c4c2acda 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml @@ -20,6 +20,8 @@ name: PostCommit Java ValidatesRunner Dataflow V2 Streaming on: schedule: - cron: '30 6/8 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs @@ -59,6 +61,7 @@ jobs: job_phrase: [Run Java Dataflow V2 ValidatesRunner Streaming] if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Java Dataflow V2 ValidatesRunner Streaming' steps: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml index a86b45596554..16e21c9faa26 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml @@ -20,6 +20,8 @@ name: PostCommit Java ValidatesRunner Direct on: schedule: - cron: '45 4/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs @@ -59,6 +61,7 @@ jobs: job_phrase: [Run Direct ValidatesRunner] if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Direct ValidatesRunner' steps: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml index 41f6440a20b5..fafc821a2b20 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml @@ -20,6 +20,8 @@ name: PostCommit Java ValidatesRunner Direct JavaVersions on: schedule: - cron: '30 4/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs @@ -61,6 +63,7 @@ jobs: java_version: ['11','17'] if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || startswith(github.event.comment.body, 'Run Direct ValidatesRunner Java') steps: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml index 2a4e81e787b3..8171760f5528 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml @@ -18,6 +18,8 @@ name: PostCommit Java ValidatesRunner Flink on: schedule: - cron: '45 4/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event @@ -57,6 +59,7 @@ jobs: job_phrase: [Run Flink ValidatesRunner] if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Flink ValidatesRunner' steps: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml index 14c186047533..6bc1b0ffa03c 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml @@ -20,6 +20,8 @@ name: PostCommit Java ValidatesRunner Flink Java11 on: schedule: - cron: '45 4/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs @@ -59,6 +61,7 @@ jobs: job_phrase: [Run Flink ValidatesRunner Java 11] if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || startswith(github.event.comment.body, 'Run Flink ValidatesRunner Java 11') steps: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml index a8682db34180..8f8993f3108e 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml @@ -18,6 +18,8 @@ name: PostCommit Java ValidatesRunner Samza on: schedule: - cron: '45 4/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event @@ -57,6 +59,7 @@ jobs: job_phrase: [Run Samza ValidatesRunner] if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Samza ValidatesRunner' steps: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml index c8c5efb4dce1..ce67510b1e50 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml @@ -18,6 +18,8 @@ name: PostCommit Java ValidatesRunner Spark on: schedule: - cron: '45 4/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event @@ -57,6 +59,7 @@ jobs: job_phrase: [Run Spark ValidatesRunner] if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Spark ValidatesRunner' steps: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml index df4f0c273e9a..7883218d262a 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml @@ -18,6 +18,8 @@ name: PostCommit Java ValidatesRunner SparkStructuredStreaming on: schedule: - cron: '45 4/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event @@ -57,6 +59,7 @@ jobs: job_phrase: [Run Spark StructuredStreaming ValidatesRunner] if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Spark StructuredStreaming ValidatesRunner' steps: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml index 00884e35f4d9..099f9ab1e724 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml @@ -20,6 +20,8 @@ name: PostCommit Java ValidatesRunner Spark Java11 on: schedule: - cron: '45 4/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs @@ -59,6 +61,7 @@ jobs: job_phrase: [Run Spark ValidatesRunner Java 11] if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || startswith(github.event.comment.body, 'Run Spark ValidatesRunner Java 11') steps: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml index 8335fec403c6..de3d4914052e 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml @@ -18,6 +18,8 @@ name: PostCommit Java ValidatesRunner Twister2 on: schedule: - cron: '45 4/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event @@ -57,6 +59,7 @@ jobs: job_phrase: [Run Twister2 ValidatesRunner] if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Twister2 ValidatesRunner' steps: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml index 7d978a75ee97..eb2139c562e9 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml @@ -18,6 +18,8 @@ name: PostCommit Java ValidatesRunner ULR on: schedule: - cron: '45 4/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event @@ -57,6 +59,7 @@ jobs: job_phrase: [Run ULR Loopback ValidatesRunner] if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run ULR Loopback ValidatesRunner' steps: diff --git a/.github/workflows/beam_PostCommit_Javadoc.yml b/.github/workflows/beam_PostCommit_Javadoc.yml index 283516714326..8207cbe9ca9a 100644 --- a/.github/workflows/beam_PostCommit_Javadoc.yml +++ b/.github/workflows/beam_PostCommit_Javadoc.yml @@ -20,6 +20,8 @@ name: PostCommit Javadoc on: schedule: - cron: '0 5/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs @@ -59,6 +61,7 @@ jobs: job_phrase: [Run Javadoc PostCommit] if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Javadoc PostCommit' steps: diff --git a/.github/workflows/beam_PostCommit_PortableJar_Flink.yml b/.github/workflows/beam_PostCommit_PortableJar_Flink.yml index 8dd433537cf1..2f066979681e 100644 --- a/.github/workflows/beam_PostCommit_PortableJar_Flink.yml +++ b/.github/workflows/beam_PostCommit_PortableJar_Flink.yml @@ -18,6 +18,8 @@ name: PostCommit PortableJar Flink on: schedule: - cron: '0 5/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event @@ -52,6 +54,7 @@ jobs: if: | (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || github.event.comment.body == 'Run PortableJar_Flink PostCommit' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 120 diff --git a/.github/workflows/beam_PostCommit_PortableJar_Spark.yml b/.github/workflows/beam_PostCommit_PortableJar_Spark.yml index 5cb9cb611df5..8b5bc031f7fa 100644 --- a/.github/workflows/beam_PostCommit_PortableJar_Spark.yml +++ b/.github/workflows/beam_PostCommit_PortableJar_Spark.yml @@ -18,6 +18,8 @@ name: PostCommit PortableJar Spark on: schedule: - cron: '0 5/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event @@ -52,6 +54,7 @@ jobs: if: | (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || github.event.comment.body == 'Run PortableJar_Spark PostCommit' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 diff --git a/.github/workflows/beam_PostCommit_Python.yml b/.github/workflows/beam_PostCommit_Python.yml index d10690d612e4..0b22466f517b 100644 --- a/.github/workflows/beam_PostCommit_Python.yml +++ b/.github/workflows/beam_PostCommit_Python.yml @@ -20,6 +20,8 @@ name: PostCommit Python on: schedule: - cron: '30 5/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs @@ -61,6 +63,7 @@ jobs: python_version: ['3.8', '3.9', '3.10', '3.11'] if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || startswith(github.event.comment.body, 'Run Python PostCommit 3.') steps: diff --git a/.github/workflows/beam_PostCommit_Python_Arm.yml b/.github/workflows/beam_PostCommit_Python_Arm.yml index 90094636cad3..1a88c468a67c 100644 --- a/.github/workflows/beam_PostCommit_Python_Arm.yml +++ b/.github/workflows/beam_PostCommit_Python_Arm.yml @@ -22,6 +22,8 @@ on: types: [created] schedule: - cron: '0 5/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs @@ -63,6 +65,7 @@ jobs: python_version: ['3.8', '3.9', '3.10', '3.11'] if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || startsWith(github.event.comment.body, 'Run Python PostCommit Arm') steps: diff --git a/.github/workflows/beam_PostCommit_Python_Examples_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_Examples_Dataflow.yml index 5a548650ce0d..ae07cb0273e0 100644 --- a/.github/workflows/beam_PostCommit_Python_Examples_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_Examples_Dataflow.yml @@ -18,6 +18,8 @@ name: PostCommit Python Examples Dataflow on: schedule: - cron: '0 5/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event @@ -52,6 +54,7 @@ jobs: if: | (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || github.event.comment.body == 'Run Python Examples_Dataflow' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 180 diff --git a/.github/workflows/beam_PostCommit_Python_Examples_Direct.yml b/.github/workflows/beam_PostCommit_Python_Examples_Direct.yml index a803b6702e04..b4b620e5dd1f 100644 --- a/.github/workflows/beam_PostCommit_Python_Examples_Direct.yml +++ b/.github/workflows/beam_PostCommit_Python_Examples_Direct.yml @@ -18,6 +18,8 @@ name: PostCommit Python Examples Direct on: schedule: - cron: '0 5/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event @@ -51,6 +53,7 @@ jobs: if: | (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || startsWith(github.event.comment.body, 'Run Python Examples_Direct') runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 120 diff --git a/.github/workflows/beam_PostCommit_Python_Examples_Flink.yml b/.github/workflows/beam_PostCommit_Python_Examples_Flink.yml index 527b9885a8c0..1f334f6a9963 100644 --- a/.github/workflows/beam_PostCommit_Python_Examples_Flink.yml +++ b/.github/workflows/beam_PostCommit_Python_Examples_Flink.yml @@ -18,6 +18,8 @@ name: PostCommit Python Examples Flink on: schedule: - cron: '0 5/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event @@ -51,6 +53,7 @@ jobs: if: | (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || startsWith(github.event.comment.body, 'Run Python Examples_Flink') runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 240 diff --git a/.github/workflows/beam_PostCommit_Python_Examples_Spark.yml b/.github/workflows/beam_PostCommit_Python_Examples_Spark.yml index 2482f2b6033e..6a33c63f24a0 100644 --- a/.github/workflows/beam_PostCommit_Python_Examples_Spark.yml +++ b/.github/workflows/beam_PostCommit_Python_Examples_Spark.yml @@ -18,6 +18,8 @@ name: PostCommit Python Examples Spark on: schedule: - cron: '0 5/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event @@ -51,6 +53,7 @@ jobs: if: | (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || startsWith(github.event.comment.body, 'Run Python Examples_Spark') runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 120 diff --git a/.github/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml b/.github/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml index d658f714a7b0..1c8daf4cffa3 100644 --- a/.github/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml +++ b/.github/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml @@ -18,6 +18,8 @@ name: PostCommit Python MongoDBIO IT on: schedule: - cron: '15 5/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event @@ -52,6 +54,7 @@ jobs: if: | (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || github.event.comment.body == 'Run Python MongoDBIO_IT' runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 diff --git a/.github/workflows/beam_PostCommit_Python_Nexmark_Direct.yml b/.github/workflows/beam_PostCommit_Python_Nexmark_Direct.yml index 661d0e61d8df..f63461369be0 100644 --- a/.github/workflows/beam_PostCommit_Python_Nexmark_Direct.yml +++ b/.github/workflows/beam_PostCommit_Python_Nexmark_Direct.yml @@ -20,6 +20,8 @@ name: PostCommit Python Nexmark Direct on: schedule: - cron: '15 5/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs @@ -102,6 +104,7 @@ jobs: query: [0, 2, 3, 5, 7, 8, 10, 11] if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Python Direct Runner Nexmark Tests' steps: diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml index 46668ffb007a..335e9791d960 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml @@ -18,6 +18,8 @@ name: PostCommit Python ValidatesContainer Dataflow on: schedule: - cron: '15 5/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event @@ -51,6 +53,7 @@ jobs: if: | (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || startsWith(github.event.comment.body, 'Run Python Dataflow ValidatesContainer') runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml b/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml index fc7c72a9e5e2..ac2179dd2521 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml @@ -18,6 +18,8 @@ name: PostCommit Python ValidatesContainer Dataflow With RC on: schedule: - cron: '15 5/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event @@ -51,6 +53,7 @@ jobs: if: | (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || startsWith(github.event.comment.body, 'Run Python RC Dataflow ValidatesContainer') runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml index 81fe6b811353..79bc303f1117 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml @@ -18,6 +18,8 @@ name: PostCommit Python ValidatesRunner Dataflow on: schedule: - cron: '15 5/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event @@ -51,6 +53,7 @@ jobs: if: | (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || startsWith(github.event.comment.body, 'Run Python Dataflow ValidatesRunner') runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 200 diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml index e5a5b1a53773..b403f76b9f9e 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml @@ -18,6 +18,8 @@ name: PostCommit Python ValidatesRunner Flink on: schedule: - cron: '15 5/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event @@ -51,6 +53,7 @@ jobs: if: | (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || startsWith(github.event.comment.body, 'Run Python Flink ValidatesRunner') runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml index a1215b33b086..4229304278c0 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml @@ -18,6 +18,8 @@ name: PostCommit Python ValidatesRunner Samza on: schedule: - cron: '15 5/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event @@ -51,6 +53,7 @@ jobs: if: | (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || startsWith(github.event.comment.body, 'Run Python Samza ValidatesRunner') runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml index eeb96712b101..06db87f8fb76 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml @@ -18,6 +18,8 @@ name: PostCommit Python ValidatesRunner Spark on: schedule: - cron: '15 5/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event @@ -51,6 +53,7 @@ jobs: if: | (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || startsWith(github.event.comment.body, 'Run Python Spark ValidatesRunner') runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 100 diff --git a/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml index 93ca3e015ac6..dd899a538e9a 100644 --- a/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml @@ -18,6 +18,8 @@ name: PostCommit Python Xlang Gcp Dataflow on: schedule: - cron: '15 5/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event @@ -50,6 +52,7 @@ jobs: beam_PostCommit_Python_Xlang_Gcp_Dataflow: if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Python_Xlang_Gcp_Dataflow PostCommit' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml b/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml index 36eeed07c257..33eb748a2f84 100644 --- a/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml +++ b/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml @@ -18,6 +18,8 @@ name: PostCommit Python Xlang Gcp Direct on: schedule: - cron: '30 5/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event @@ -50,6 +52,7 @@ jobs: beam_PostCommit_Python_Xlang_Gcp_Direct: if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Python_Xlang_Gcp_Direct PostCommit' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml index c42dbc7aa068..4d71e507fe32 100644 --- a/.github/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml @@ -18,6 +18,8 @@ name: PostCommit Python Xlang IO Dataflow on: schedule: - cron: '30 5/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event @@ -50,6 +52,7 @@ jobs: beam_PostCommit_Python_Xlang_IO_Dataflow: if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Python_Xlang_IO_Dataflow PostCommit' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PostCommit_SQL.yml b/.github/workflows/beam_PostCommit_SQL.yml index 80aad5b82908..eae7d4374d5b 100644 --- a/.github/workflows/beam_PostCommit_SQL.yml +++ b/.github/workflows/beam_PostCommit_SQL.yml @@ -20,6 +20,8 @@ name: PostCommit SQL on: schedule: - cron: '30 5/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs @@ -59,6 +61,7 @@ jobs: job_phrase: [Run SQL PostCommit] if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run SQL PostCommit' steps: diff --git a/.github/workflows/beam_PostCommit_Sickbay_Python.yml b/.github/workflows/beam_PostCommit_Sickbay_Python.yml index 5fbffed22cd4..b4d054f07a3d 100644 --- a/.github/workflows/beam_PostCommit_Sickbay_Python.yml +++ b/.github/workflows/beam_PostCommit_Sickbay_Python.yml @@ -20,6 +20,8 @@ name: PostCommit Sickbay Python on: schedule: - cron: '0 8 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs @@ -62,6 +64,7 @@ jobs: python_version: ['3.8', '3.9', '3.10', '3.11'] if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || (startswith(github.event.comment.body, 'Run Python') && endswith(github.event.comment.body, 'PostCommit Sickbay')) diff --git a/.github/workflows/beam_PostCommit_TransformService_Direct.yml b/.github/workflows/beam_PostCommit_TransformService_Direct.yml index e3fac46bd9c3..e40112f0c5b1 100644 --- a/.github/workflows/beam_PostCommit_TransformService_Direct.yml +++ b/.github/workflows/beam_PostCommit_TransformService_Direct.yml @@ -18,6 +18,8 @@ name: PostCommit TransformService Direct on: schedule: - cron: '30 5/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event @@ -50,6 +52,7 @@ jobs: beam_PostCommit_TransformService_Direct: if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run TransformService_Direct PostCommit' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PostCommit_Website_Test.yml b/.github/workflows/beam_PostCommit_Website_Test.yml index 06f9bcd73243..ba372d223281 100644 --- a/.github/workflows/beam_PostCommit_Website_Test.yml +++ b/.github/workflows/beam_PostCommit_Website_Test.yml @@ -18,6 +18,8 @@ name: PostCommit Website Test on: schedule: - cron: '30 5/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event @@ -50,6 +52,7 @@ jobs: beam_PostCommit_Website_Test: if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Full Website Test' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PostCommit_XVR_Direct.yml b/.github/workflows/beam_PostCommit_XVR_Direct.yml index 247aa4c23e11..dcb6d3b3b0b8 100644 --- a/.github/workflows/beam_PostCommit_XVR_Direct.yml +++ b/.github/workflows/beam_PostCommit_XVR_Direct.yml @@ -18,6 +18,8 @@ name: PostCommit XVR Direct on: schedule: - cron: '30 5/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event @@ -50,6 +52,7 @@ jobs: beam_PostCommit_XVR_Direct: if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run XVR_Direct PostCommit' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PostCommit_XVR_Flink.yml b/.github/workflows/beam_PostCommit_XVR_Flink.yml index ad90720cd4f2..abb77eeee970 100644 --- a/.github/workflows/beam_PostCommit_XVR_Flink.yml +++ b/.github/workflows/beam_PostCommit_XVR_Flink.yml @@ -18,6 +18,8 @@ name: PostCommit XVR Flink on: schedule: - cron: '30 5/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event @@ -51,6 +53,7 @@ jobs: beam_PostCommit_XVR_Flink: if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run XVR_Flink PostCommit' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml b/.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml index 0fa27b2c6dc8..aab8a0e0a84f 100644 --- a/.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml @@ -18,6 +18,8 @@ name: PostCommit XVR GoUsingJava Dataflow on: schedule: - cron: '45 5/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event @@ -50,6 +52,7 @@ jobs: beam_PostCommit_XVR_GoUsingJava_Dataflow: if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run XVR_GoUsingJava_Dataflow PostCommit' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml b/.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml index cbf666157c9b..0b81a444d58b 100644 --- a/.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml @@ -18,6 +18,8 @@ name: PostCommit XVR JavaUsingPython Dataflow on: schedule: - cron: '45 5/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event @@ -50,6 +52,7 @@ jobs: beam_PostCommit_XVR_JavaUsingPython_Dataflow: if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run XVR_JavaUsingPython_Dataflow PostCommit' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml b/.github/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml index e90449742ce4..6759930d5de8 100644 --- a/.github/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml @@ -18,6 +18,8 @@ name: PostCommit XVR PythonUsingJavaSQL Dataflow on: schedule: - cron: '45 5/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event @@ -50,6 +52,7 @@ jobs: beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow: if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run XVR_PythonUsingJavaSQL_Dataflow PostCommit' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml b/.github/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml index 5b650fcf003b..dda068049a17 100644 --- a/.github/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml @@ -18,6 +18,8 @@ name: PostCommit XVR PythonUsingJava Dataflow on: schedule: - cron: '45 5/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event @@ -50,6 +52,7 @@ jobs: beam_PostCommit_XVR_PythonUsingJava_Dataflow: if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run XVR_PythonUsingJava_Dataflow PostCommit' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PostCommit_XVR_Samza.yml b/.github/workflows/beam_PostCommit_XVR_Samza.yml index 34b226e93e5c..239d3c849b27 100644 --- a/.github/workflows/beam_PostCommit_XVR_Samza.yml +++ b/.github/workflows/beam_PostCommit_XVR_Samza.yml @@ -18,6 +18,8 @@ name: PostCommit XVR Samza on: schedule: - cron: '45 5/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event @@ -50,6 +52,7 @@ jobs: beam_PostCommit_XVR_Samza: if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run XVR_Samza PostCommit' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_PostCommit_XVR_Spark3.yml b/.github/workflows/beam_PostCommit_XVR_Spark3.yml index 5419883911d8..720900b7a081 100644 --- a/.github/workflows/beam_PostCommit_XVR_Spark3.yml +++ b/.github/workflows/beam_PostCommit_XVR_Spark3.yml @@ -18,6 +18,8 @@ name: PostCommit XVR Spark3 on: schedule: - cron: '45 5/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event @@ -50,6 +52,7 @@ jobs: beam_PostCommit_XVR_Spark3: if: | github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run XVR_Spark3 PostCommit' runs-on: [self-hosted, ubuntu-20.04, main] From 83906290ab0a4b5fd6cf2754397f35135bed72c1 Mon Sep 17 00:00:00 2001 From: Ritesh Ghorse <riteshghorse@gmail.com> Date: Wed, 1 Nov 2023 14:59:09 -0400 Subject: [PATCH 346/435] [Python] Add device param to HuggingFacePipelineModelHandler (#29223) * add device param * add spare line * change validations * rm note * valueError, fix default * valueError * Make param optional to make linter happy --------- Co-authored-by: Danny McCormick <dannymccormick@google.com> --- .../ml/inference/huggingface_inference.py | 38 +++++++++++++++++-- 1 file changed, 34 insertions(+), 4 deletions(-) diff --git a/sdks/python/apache_beam/ml/inference/huggingface_inference.py b/sdks/python/apache_beam/ml/inference/huggingface_inference.py index 3ec063808ae3..878d7bfc9cf2 100644 --- a/sdks/python/apache_beam/ml/inference/huggingface_inference.py +++ b/sdks/python/apache_beam/ml/inference/huggingface_inference.py @@ -573,6 +573,7 @@ def __init__( task: Union[str, PipelineTask] = "", model: str = "", *, + device: Optional[str] = None, inference_fn: PipelineInferenceFn = _default_pipeline_inference_fn, load_pipeline_args: Optional[Dict[str, Any]] = None, inference_args: Optional[Dict[str, Any]] = None, @@ -583,10 +584,6 @@ def __init__( """ Implementation of the ModelHandler interface for Hugging Face Pipelines. - **Note:** To specify which device to use (CPU/GPU), - use the load_pipeline_args with key-value as you would do in the usual - Hugging Face pipeline. Ex: load_pipeline_args={'device':0}) - Example Usage model:: pcoll | RunInference(HuggingFacePipelineModelHandler( task="fill-mask")) @@ -606,6 +603,11 @@ def __init__( task="text-generation", model="meta-llama/Llama-2-7b-hf", load_pipeline_args={'model_kwargs':{'quantization_map':config}}) + device (str): the device (`"CPU"` or `"GPU"`) on which you wish to run + the pipeline. Defaults to GPU. If GPU is not available then it falls + back to CPU. You can also use advanced option like `device_map` with + key-value pair as you would do in the usual Hugging Face pipeline using + `load_pipeline_args`. Ex: load_pipeline_args={'device_map':auto}). inference_fn: the inference function to use during RunInference. Default is _default_pipeline_inference_fn. load_pipeline_args (Dict[str, Any]): keyword arguments to provide load @@ -638,8 +640,36 @@ def __init__( if max_batch_size is not None: self._batching_kwargs['max_batch_size'] = max_batch_size self._large_model = large_model + + # Check if the device is specified twice. If true then the device parameter + # of model handler is overridden. + self._deduplicate_device_value(device) _validate_constructor_args_hf_pipeline(self._task, self._model) + def _deduplicate_device_value(self, device: Optional[str]): + current_device = device.upper() if device else None + if (current_device and current_device != 'CPU' and current_device != 'GPU'): + raise ValueError( + f"Invalid device value: {device}. Please specify " + "either CPU or GPU. Defaults to GPU if no value " + "is provided.") + if 'device' not in self._load_pipeline_args: + if current_device == 'CPU': + self._load_pipeline_args['device'] = 'cpu' + else: + if is_gpu_available_torch(): + self._load_pipeline_args['device'] = 'cuda:1' + else: + _LOGGER.warning( + "HuggingFaceModelHandler specified a 'GPU' device, " + "but GPUs are not available. Switching to CPU.") + self._load_pipeline_args['device'] = 'cpu' + else: + if current_device: + raise ValueError( + '`device` specified in `load_pipeline_args`. `device` ' + 'parameter for HuggingFacePipelineModelHandler will be ignored.') + def load_model(self): """Loads and initializes the pipeline for processing.""" return pipeline( From 2e52a6559a8fa15bcc9823d4d41628b004e71553 Mon Sep 17 00:00:00 2001 From: Yi Hu <yathu@google.com> Date: Wed, 1 Nov 2023 15:27:59 -0400 Subject: [PATCH 347/435] Fix bigtable_it_test instance naming (#29249) --- sdks/python/apache_beam/io/gcp/bigtableio_it_test.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/io/gcp/bigtableio_it_test.py b/sdks/python/apache_beam/io/gcp/bigtableio_it_test.py index 6cdd0bbeecf6..fdf049bba944 100644 --- a/sdks/python/apache_beam/io/gcp/bigtableio_it_test.py +++ b/sdks/python/apache_beam/io/gcp/bigtableio_it_test.py @@ -160,8 +160,9 @@ def setUpClass(cls): cls.args = cls.test_pipeline.get_full_options_as_args() cls.expansion_service = ('localhost:%s' % os.environ.get('EXPANSION_PORT')) - timestr = "".join(filter(str.isdigit, str(datetime.datetime.utcnow()))) - instance_id = '%s-%s-%s' % (cls.INSTANCE, timestr, secrets.token_hex(3)) + timestr = "".join(filter(str.isdigit, str(datetime.utcnow().date()))) + # instance id length needs to be within [6, 33] + instance_id = '%s-%s-%s' % (cls.INSTANCE, timestr, secrets.token_hex(4)) cls.client = client.Client(admin=True, project=cls.project) # create cluster and instance From 73043940ffef75f1873a7b1124f851b2f3123cb5 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 1 Nov 2023 14:45:53 -0700 Subject: [PATCH 348/435] Bump cloud.google.com/go/bigquery from 1.56.0 to 1.57.0 in /sdks (#29208) Bumps [cloud.google.com/go/bigquery](https://github.com/googleapis/google-cloud-go) from 1.56.0 to 1.57.0. - [Release notes](https://github.com/googleapis/google-cloud-go/releases) - [Changelog](https://github.com/googleapis/google-cloud-go/blob/main/CHANGES.md) - [Commits](https://github.com/googleapis/google-cloud-go/compare/bigquery/v1.56.0...bigquery/v1.57.0) --- updated-dependencies: - dependency-name: cloud.google.com/go/bigquery dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] <support@github.com> Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 2 +- sdks/go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 9afad643c1d4..852f21d0c372 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -23,7 +23,7 @@ module github.com/apache/beam/sdks/v2 go 1.20 require ( - cloud.google.com/go/bigquery v1.56.0 + cloud.google.com/go/bigquery v1.57.0 cloud.google.com/go/bigtable v1.20.0 cloud.google.com/go/datastore v1.15.0 cloud.google.com/go/profiler v0.4.0 diff --git a/sdks/go.sum b/sdks/go.sum index 59db69123004..9dcec3a1dc26 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -13,8 +13,8 @@ cloud.google.com/go v0.110.8/go.mod h1:Iz8AkXJf1qmxC3Oxoep8R1T36w8B92yU29PcBhHO5 cloud.google.com/go/bigquery v1.0.1/go.mod h1:i/xbL2UlR5RvWAURpBYZTtm/cXjCha9lbfbpx4poX+o= cloud.google.com/go/bigquery v1.3.0/go.mod h1:PjpwJnslEMmckchkHFfq+HTD2DmtT67aNFKH1/VBDHE= cloud.google.com/go/bigquery v1.4.0/go.mod h1:S8dzgnTigyfTmLBfrtrhyYhwRxG72rYxvftPBK2Dvzc= -cloud.google.com/go/bigquery v1.56.0 h1:LHIc9E7Kw+ftFpQFKzZYBB88IAFz7qONawXXx0F3QBo= -cloud.google.com/go/bigquery v1.56.0/go.mod h1:KDcsploXTEY7XT3fDQzMUZlpQLHzE4itubHrnmhUrZA= +cloud.google.com/go/bigquery v1.57.0 h1:1iVepOxF9XmHSmKePQG1FMMfv0Gw3f9+5zqRhTvxglw= +cloud.google.com/go/bigquery v1.57.0/go.mod h1:SmaaA61ZgwNhNDB+yKFyxHGgSMuxDJeB0T52SnGxnok= cloud.google.com/go/bigtable v1.20.0 h1:NqZC/WcesSn4O8L0I2JmuNsUigSyBQifVLYgM9LMQeQ= cloud.google.com/go/bigtable v1.20.0/go.mod h1:upJDn8frsjzpRMfybiWkD1PG6WCCL7CRl26MgVeoXY4= cloud.google.com/go/compute v1.23.0 h1:tP41Zoavr8ptEqaW6j+LQOnyBBhO7OkOMAGrgLopTwY= From f1a6cb2eed049584dd16dcb9a86bd6423cc569a2 Mon Sep 17 00:00:00 2001 From: Svetak Sundhar <svetaksundhar@google.com> Date: Wed, 1 Nov 2023 17:46:31 -0400 Subject: [PATCH 349/435] [Go] Update website with supported IOs (#29189) * Create HealthcareUtils file with shared resources * revert * Updates supported Go IOs on Beam website * native IO typo --- .../content/en/documentation/io/connectors.md | 28 +++++++++++++++---- 1 file changed, 23 insertions(+), 5 deletions(-) diff --git a/website/www/site/content/en/documentation/io/connectors.md b/website/www/site/content/en/documentation/io/connectors.md index 0a17954d9028..ab8ccf935cb7 100644 --- a/website/www/site/content/en/documentation/io/connectors.md +++ b/website/www/site/content/en/documentation/io/connectors.md @@ -50,7 +50,10 @@ This table provides a consolidated, at-a-glance overview of the available built- ✔ <a href="https://beam.apache.org/releases/pydoc/current/apache_beam.io.fileio.html">native</a> </td> - <td>Not available</td> + <td class="present"> + ✔ + <a href="https://pkg.go.dev/github.com/apache/beam/sdks/v2/go/pkg/beam/io/fileio">native</a> + </td> <td>Not available</td> <td class="present">✔</td> <td class="present">✔</td> @@ -158,7 +161,10 @@ This table provides a consolidated, at-a-glance overview of the available built- ✔ <a href="https://beam.apache.org/releases/pydoc/current/apache_beam.io.parquetio.html">native</a> </td> - <td>Not available</td> + <td class="present"> + ✔ + <a href="https://pkg.go.dev/github.com/apache/beam/sdks/v2/go/pkg/beam/io/parquetio">native</a> + </td> <td class="present"> ✔ <a href="https://github.com/apache/beam/blob/master/sdks/typescript/src/apache_beam/io/parquetio.ts">via X-language</a> @@ -536,6 +542,9 @@ This table provides a consolidated, at-a-glance overview of the available built- <td class="present"> ✔ <a href="https://pkg.go.dev/github.com/apache/beam/sdks/v2/go/pkg/beam/io/bigqueryio">native</a> + <br> + ✔ + <a href="https://pkg.go.dev/github.com/apache/beam/sdks/v2/go/pkg/beam/io/xlang/bigqueryio">via X-language</a> </td> <td class="present"> ✔ @@ -619,7 +628,10 @@ This table provides a consolidated, at-a-glance overview of the available built- ✔ <a href="https://beam.apache.org/releases/pydoc/current/apache_beam.io.gcp.spanner.html">via X-language</a> </td> - <td>Not available</td> + <td class="present"> + ✔ + <a href="https://pkg.go.dev/github.com/apache/beam/sdks/v2/go/pkg/beam/io/spannerio">native</a> + </td> <td>Not available</td> <td class="present">✔</td> <td class="present">✔</td> @@ -676,7 +688,10 @@ This table provides a consolidated, at-a-glance overview of the available built- ✔ <a href="https://beam.apache.org/releases/pydoc/current/apache_beam.io.mongodbio.html">native</a> </td> - <td>Not available</td> + <td class="present"> + ✔ + <a href="https://pkg.go.dev/github.com/apache/beam/sdks/v2/go/pkg/beam/io/mongodbio">native</a> + </td> <td>Not available</td> <td class="present">✔</td> <td class="absent">✘</td> @@ -787,7 +802,10 @@ This table provides a consolidated, at-a-glance overview of the available built- ✔ <a href="https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.html">native</a> </td> - <td>Not available</td> + <td class="present"> + ✔ + <a href="https://pkg.go.dev/github.com/apache/beam/sdks/v2/go/pkg/beam/io/fhirio">native</a> + </td> <td>Not available</td> <td>Not available</td> <td class="present">✔</td> From a6b7321ec1c0135793e2c5a86cf7f6128bdb06cc Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 1 Nov 2023 14:56:07 -0700 Subject: [PATCH 350/435] Bump cloud.google.com/go/storage from 1.33.0 to 1.34.0 in /sdks (#29239) Bumps [cloud.google.com/go/storage](https://github.com/googleapis/google-cloud-go) from 1.33.0 to 1.34.0. - [Release notes](https://github.com/googleapis/google-cloud-go/releases) - [Changelog](https://github.com/googleapis/google-cloud-go/blob/main/CHANGES.md) - [Commits](https://github.com/googleapis/google-cloud-go/compare/pubsub/v1.33.0...spanner/v1.34.0) --- updated-dependencies: - dependency-name: cloud.google.com/go/storage dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] <support@github.com> Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 2 +- sdks/go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 852f21d0c372..e55a8f35e342 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -29,7 +29,7 @@ require ( cloud.google.com/go/profiler v0.4.0 cloud.google.com/go/pubsub v1.33.0 cloud.google.com/go/spanner v1.51.0 - cloud.google.com/go/storage v1.33.0 + cloud.google.com/go/storage v1.34.0 github.com/aws/aws-sdk-go-v2 v1.22.0 github.com/aws/aws-sdk-go-v2/config v1.20.0 github.com/aws/aws-sdk-go-v2/credentials v1.14.0 diff --git a/sdks/go.sum b/sdks/go.sum index 9dcec3a1dc26..c4e2ee22a304 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -43,8 +43,8 @@ cloud.google.com/go/spanner v1.51.0/go.mod h1:c5KNo5LQ1X5tJwma9rSQZsXNBDNvj4/n8B cloud.google.com/go/storage v1.0.0/go.mod h1:IhtSnM/ZTZV8YYJWCY8RULGVqBDmpoyjwiyrjsg+URw= cloud.google.com/go/storage v1.5.0/go.mod h1:tpKbwo567HUNpVclU5sGELwQWBDZ8gh0ZeosJ0Rtdos= cloud.google.com/go/storage v1.6.0/go.mod h1:N7U0C8pVQ/+NIKOBQyamJIeKQKkZ+mxpohlUTyfDhBk= -cloud.google.com/go/storage v1.33.0 h1:PVrDOkIC8qQVa1P3SXGpQvfuJhN2LHOoyZvWs8D2X5M= -cloud.google.com/go/storage v1.33.0/go.mod h1:Hhh/dogNRGca7IWv1RC2YqEn0c0G77ctA/OxflYkiD8= +cloud.google.com/go/storage v1.34.0 h1:9KHBBTbaHPsNxO043SFmH3pMojjZiW+BFl9H41L7xjk= +cloud.google.com/go/storage v1.34.0/go.mod h1:Eji+S0CCQebjsiXxyIvPItC3BN3zWsdJjWfHfoLblgY= dario.cat/mergo v1.0.0 h1:AGCNq9Evsj31mOgNPcLyXc+4PNABt905YmuqPYYpBWk= dario.cat/mergo v1.0.0/go.mod h1:uNxQE+84aUszobStD9th8a29P2fMDhsBdgRYvZOxGmk= dmitri.shuralyov.com/gpu/mtl v0.0.0-20190408044501-666a987793e9/go.mod h1:H6x//7gZCb22OMCxBHrMx7a5I7Hp++hsVxbQ4BYO7hU= From 3934117096392e5c8276635222ddf520dc0504a0 Mon Sep 17 00:00:00 2001 From: Svetak Sundhar <svetaksundhar@google.com> Date: Wed, 1 Nov 2023 18:08:57 -0400 Subject: [PATCH 351/435] [Dataframes] Fix Broken Link on Website (#29258) * Create HealthcareUtils file with shared resources * revert * Fix Broken Colab Link (and remove duplicate) --- .../site/content/en/documentation/dsls/dataframes/overview.md | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/website/www/site/content/en/documentation/dsls/dataframes/overview.md b/website/www/site/content/en/documentation/dsls/dataframes/overview.md index fa1ab0169f67..bc04c3cc6f7a 100644 --- a/website/www/site/content/en/documentation/dsls/dataframes/overview.md +++ b/website/www/site/content/en/documentation/dsls/dataframes/overview.md @@ -18,7 +18,7 @@ limitations under the License. # Beam DataFrames overview -{{< button-colab url="https://colab.research.google.com/github/apache/beam/blob/master/examples/notebooks/tour-of-beam/dataframes.ipynb" >}} +{{< button-colab url="https://colab.research.google.com/github/apache/beam/blob/master/examples/notebooks/interactive-overview/dataframes.ipynb" >}} The Apache Beam Python SDK provides a DataFrame API for working with pandas-like [DataFrame](https://pandas.pydata.org/pandas-docs/stable/reference/api/pandas.DataFrame.html) objects. The feature lets you convert a PCollection to a DataFrame and then interact with the DataFrame using the standard methods available on the pandas DataFrame API. The DataFrame API is built on top of the pandas implementation, and pandas DataFrame methods are invoked on subsets of the datasets in parallel. The big difference between Beam DataFrames and pandas DataFrames is that operations are deferred by the Beam API, to support the Beam parallel processing model. (To learn more about differences between the DataFrame implementations, see [Differences from pandas](/documentation/dsls/dataframes/differences-from-pandas/).) @@ -107,4 +107,3 @@ pc1, pc2 = {'a': pc} | DataframeTransform(lambda a: expr1, expr2) [pydoc_to_dataframe]: https://beam.apache.org/releases/pydoc/current/apache_beam.dataframe.convert.html#apache_beam.dataframe.convert.to_dataframe [pydoc_to_pcollection]: https://beam.apache.org/releases/pydoc/current/apache_beam.dataframe.convert.html#apache_beam.dataframe.convert.to_pcollection -{{< button-colab url="https://colab.research.google.com/github/apache/beam/blob/master/examples/notebooks/tour-of-beam/dataframes.ipynb" >}} From cf48cafe48de1c5e9ba422c110214b4e6d7f69d1 Mon Sep 17 00:00:00 2001 From: Yi Hu <yathu@google.com> Date: Wed, 1 Nov 2023 23:46:29 -0400 Subject: [PATCH 352/435] Fix ./gradlew spotlessApply fail groovy check (#29264) --- .../main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy | 3 +++ 1 file changed, 3 insertions(+) diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index 2d2ac32d815d..a16b21f4d633 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -2276,6 +2276,9 @@ class BeamModulePlugin implements Plugin<Project> { } groovyGradle { greclipse().configFile(grEclipseConfig) } } + // Workaround to fix spotless groovy and groovyGradle tasks use the same intermediate dir, + // until Beam no longer build on Java8 and can upgrade spotless plugin. + project.tasks.spotlessGroovy.mustRunAfter project.tasks.spotlessGroovyGradle } // containerImageName returns a configurable container image name, by default a From f803451ac90b9007c0778ecffd319715d434cb3d Mon Sep 17 00:00:00 2001 From: tvalentyn <tvalentyn@users.noreply.github.com> Date: Thu, 2 Nov 2023 06:16:40 -0700 Subject: [PATCH 353/435] Upgrade protobuf version to remediate a leak. (#29255) --- sdks/python/container/py310/base_image_requirements.txt | 2 +- sdks/python/container/py311/base_image_requirements.txt | 2 +- sdks/python/container/py38/base_image_requirements.txt | 2 +- sdks/python/container/py39/base_image_requirements.txt | 2 +- sdks/python/setup.py | 2 +- 5 files changed, 5 insertions(+), 5 deletions(-) diff --git a/sdks/python/container/py310/base_image_requirements.txt b/sdks/python/container/py310/base_image_requirements.txt index c1d4f2c551dc..035d5697b0d7 100644 --- a/sdks/python/container/py310/base_image_requirements.txt +++ b/sdks/python/container/py310/base_image_requirements.txt @@ -98,7 +98,7 @@ pandas==1.5.3 parameterized==0.9.0 pluggy==1.3.0 proto-plus==1.22.3 -protobuf==4.24.4 +protobuf==4.25.0 psycopg2-binary==2.9.9 pyarrow==11.0.0 pyasn1==0.5.0 diff --git a/sdks/python/container/py311/base_image_requirements.txt b/sdks/python/container/py311/base_image_requirements.txt index 0126b5506d63..f6fea89d0188 100644 --- a/sdks/python/container/py311/base_image_requirements.txt +++ b/sdks/python/container/py311/base_image_requirements.txt @@ -95,7 +95,7 @@ pandas==1.5.3 parameterized==0.9.0 pluggy==1.3.0 proto-plus==1.22.3 -protobuf==4.24.4 +protobuf==4.25.0 psycopg2-binary==2.9.9 pyarrow==11.0.0 pyasn1==0.5.0 diff --git a/sdks/python/container/py38/base_image_requirements.txt b/sdks/python/container/py38/base_image_requirements.txt index 979600b03897..d2d48f19d351 100644 --- a/sdks/python/container/py38/base_image_requirements.txt +++ b/sdks/python/container/py38/base_image_requirements.txt @@ -102,7 +102,7 @@ parameterized==0.9.0 pkgutil_resolve_name==1.3.10 pluggy==1.3.0 proto-plus==1.22.3 -protobuf==4.24.4 +protobuf==4.25.0 psycopg2-binary==2.9.9 pyarrow==11.0.0 pyasn1==0.5.0 diff --git a/sdks/python/container/py39/base_image_requirements.txt b/sdks/python/container/py39/base_image_requirements.txt index 0dda688963e8..4fb867387475 100644 --- a/sdks/python/container/py39/base_image_requirements.txt +++ b/sdks/python/container/py39/base_image_requirements.txt @@ -99,7 +99,7 @@ pandas==1.5.3 parameterized==0.9.0 pluggy==1.3.0 proto-plus==1.22.3 -protobuf==4.24.4 +protobuf==4.25.0 psycopg2-binary==2.9.9 pyarrow==11.0.0 pyasn1==0.5.0 diff --git a/sdks/python/setup.py b/sdks/python/setup.py index e0bc6197e93b..b9fd89f0707e 100644 --- a/sdks/python/setup.py +++ b/sdks/python/setup.py @@ -297,7 +297,7 @@ def get_portability_package_data(): # # 3. Exclude protobuf 4 versions that leak memory, see: # https://github.com/apache/beam/issues/28246 - 'protobuf>=3.20.3,<4.25.0,!=4.0.*,!=4.21.*,!=4.22.0,!=4.23.*,!=4.24.0,!=4.24.1,!=4.24.2', # pylint: disable=line-too-long + 'protobuf>=3.20.3,<4.26.0,!=4.0.*,!=4.21.*,!=4.22.0,!=4.23.*,!=4.24.*', # pylint: disable=line-too-long 'pydot>=1.2.0,<2', 'python-dateutil>=2.8.0,<3', 'pytz>=2018.3', From 93336481231c7fa2dd4fed3031d1c579e6a0ea55 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 2 Nov 2023 10:23:25 -0400 Subject: [PATCH 354/435] Bump cloud.google.com/go/storage from 1.34.0 to 1.34.1 in /sdks (#29268) Bumps [cloud.google.com/go/storage](https://github.com/googleapis/google-cloud-go) from 1.34.0 to 1.34.1. - [Release notes](https://github.com/googleapis/google-cloud-go/releases) - [Changelog](https://github.com/googleapis/google-cloud-go/blob/main/CHANGES.md) - [Commits](https://github.com/googleapis/google-cloud-go/compare/spanner/v1.34.0...spanner/v1.34.1) --- updated-dependencies: - dependency-name: cloud.google.com/go/storage dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] <support@github.com> Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 16 ++++++++-------- sdks/go.sum | 36 ++++++++++++++++++------------------ 2 files changed, 26 insertions(+), 26 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index e55a8f35e342..4d35f99aae46 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -29,7 +29,7 @@ require ( cloud.google.com/go/profiler v0.4.0 cloud.google.com/go/pubsub v1.33.0 cloud.google.com/go/spanner v1.51.0 - cloud.google.com/go/storage v1.34.0 + cloud.google.com/go/storage v1.34.1 github.com/aws/aws-sdk-go-v2 v1.22.0 github.com/aws/aws-sdk-go-v2/config v1.20.0 github.com/aws/aws-sdk-go-v2/credentials v1.14.0 @@ -57,8 +57,8 @@ require ( golang.org/x/sync v0.4.0 golang.org/x/sys v0.13.0 golang.org/x/text v0.13.0 - google.golang.org/api v0.148.0 - google.golang.org/genproto v0.0.0-20231002182017-d307bd883b97 + google.golang.org/api v0.149.0 + google.golang.org/genproto v0.0.0-20231016165738-49dd2c1f3d0b google.golang.org/grpc v1.59.0 google.golang.org/protobuf v1.31.0 gopkg.in/retry.v1 v1.0.3 @@ -86,10 +86,10 @@ require ( require ( cloud.google.com/go v0.110.8 // indirect - cloud.google.com/go/compute v1.23.0 // indirect + cloud.google.com/go/compute v1.23.1 // indirect cloud.google.com/go/compute/metadata v0.2.3 // indirect cloud.google.com/go/iam v1.1.3 // indirect - cloud.google.com/go/longrunning v0.5.1 // indirect + cloud.google.com/go/longrunning v0.5.2 // indirect github.com/Azure/go-ansiterm v0.0.0-20210617225240-d185dfc1b5a1 // indirect github.com/Microsoft/go-winio v0.6.1 // indirect github.com/andybalholm/brotli v1.0.4 // indirect @@ -131,7 +131,7 @@ require ( github.com/google/pprof v0.0.0-20230602150820-91b7bce49751 // indirect github.com/google/renameio/v2 v2.0.0 // indirect github.com/google/s2a-go v0.1.7 // indirect - github.com/googleapis/enterprise-certificate-proxy v0.3.1 // indirect + github.com/googleapis/enterprise-certificate-proxy v0.3.2 // indirect github.com/googleapis/gax-go/v2 v2.12.0 // indirect github.com/gorilla/handlers v1.5.1 // indirect github.com/gorilla/mux v1.8.0 // indirect @@ -170,6 +170,6 @@ require ( golang.org/x/tools v0.10.0 // indirect golang.org/x/xerrors v0.0.0-20220907171357-04be3eba64a2 // indirect google.golang.org/appengine v1.6.7 // indirect - google.golang.org/genproto/googleapis/api v0.0.0-20231002182017-d307bd883b97 // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20231012201019-e917dd12ba7a // indirect + google.golang.org/genproto/googleapis/api v0.0.0-20231016165738-49dd2c1f3d0b // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20231016165738-49dd2c1f3d0b // indirect ) diff --git a/sdks/go.sum b/sdks/go.sum index c4e2ee22a304..64a21d35a35f 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -17,20 +17,20 @@ cloud.google.com/go/bigquery v1.57.0 h1:1iVepOxF9XmHSmKePQG1FMMfv0Gw3f9+5zqRhTvx cloud.google.com/go/bigquery v1.57.0/go.mod h1:SmaaA61ZgwNhNDB+yKFyxHGgSMuxDJeB0T52SnGxnok= cloud.google.com/go/bigtable v1.20.0 h1:NqZC/WcesSn4O8L0I2JmuNsUigSyBQifVLYgM9LMQeQ= cloud.google.com/go/bigtable v1.20.0/go.mod h1:upJDn8frsjzpRMfybiWkD1PG6WCCL7CRl26MgVeoXY4= -cloud.google.com/go/compute v1.23.0 h1:tP41Zoavr8ptEqaW6j+LQOnyBBhO7OkOMAGrgLopTwY= -cloud.google.com/go/compute v1.23.0/go.mod h1:4tCnrn48xsqlwSAiLf1HXMQk8CONslYbdiEZc9FEIbM= +cloud.google.com/go/compute v1.23.1 h1:V97tBoDaZHb6leicZ1G6DLK2BAaZLJ/7+9BB/En3hR0= +cloud.google.com/go/compute v1.23.1/go.mod h1:CqB3xpmPKKt3OJpW2ndFIXnA9A4xAy/F3Xp1ixncW78= cloud.google.com/go/compute/metadata v0.2.3 h1:mg4jlk7mCAj6xXp9UJ4fjI9VUI5rubuGBW5aJ7UnBMY= cloud.google.com/go/compute/metadata v0.2.3/go.mod h1:VAV5nSsACxMJvgaAuX6Pk2AawlZn8kiOGuCv6gTkwuA= -cloud.google.com/go/datacatalog v1.17.1 h1:qGWrlYvWtK+8jD1jhwq5BsGoSr7S4/LOroV7LwXi00g= +cloud.google.com/go/datacatalog v1.18.1 h1:xJp9mZrc2HPaoxIz3sP9pCmf/impifweQ/yGG9VBfio= cloud.google.com/go/datastore v1.0.0/go.mod h1:LXYbyblFSglQ5pkeyhO+Qmw7ukd3C+pD7TKLgZqpHYE= cloud.google.com/go/datastore v1.1.0/go.mod h1:umbIZjpQpHh4hmRpGhH4tLFup+FVzqBi1b3c64qFpCk= cloud.google.com/go/datastore v1.15.0 h1:0P9WcsQeTWjuD1H14JIY7XQscIPQ4Laje8ti96IC5vg= cloud.google.com/go/datastore v1.15.0/go.mod h1:GAeStMBIt9bPS7jMJA85kgkpsMkvseWWXiaHya9Jes8= cloud.google.com/go/iam v1.1.3 h1:18tKG7DzydKWUnLjonWcJO6wjSCAtzh4GcRKlH/Hrzc= cloud.google.com/go/iam v1.1.3/go.mod h1:3khUlaBXfPKKe7huYgEpDn6FtgRyMEqbkvBxrQyY5SE= -cloud.google.com/go/kms v1.15.2 h1:lh6qra6oC4AyWe5fUUUBe/S27k12OHAleOOOw6KakdE= -cloud.google.com/go/longrunning v0.5.1 h1:Fr7TXftcqTudoyRJa113hyaqlGdiBQkp0Gq7tErFDWI= -cloud.google.com/go/longrunning v0.5.1/go.mod h1:spvimkwdz6SPWKEt/XBij79E9fiTkHSQl/fRUUQJYJc= +cloud.google.com/go/kms v1.15.3 h1:RYsbxTRmk91ydKCzekI2YjryO4c5Y2M80Zwcs9/D/cI= +cloud.google.com/go/longrunning v0.5.2 h1:u+oFqfEwwU7F9dIELigxbe0XVnBAo9wqMuQLA50CZ5k= +cloud.google.com/go/longrunning v0.5.2/go.mod h1:nqo6DQbNV2pXhGDbDMoN2bWz68MjZUzqv2YttZiveCs= cloud.google.com/go/profiler v0.4.0 h1:ZeRDZbsOBDyRG0OiK0Op1/XWZ3xeLwJc9zjkzczUxyY= cloud.google.com/go/profiler v0.4.0/go.mod h1:RvPlm4dilIr3oJtAOeFQU9Lrt5RoySHSDj4pTd6TWeU= cloud.google.com/go/pubsub v1.0.1/go.mod h1:R0Gpsv3s54REJCy4fxDixWD93lHJMoZTyQ2kNxGRt3I= @@ -43,8 +43,8 @@ cloud.google.com/go/spanner v1.51.0/go.mod h1:c5KNo5LQ1X5tJwma9rSQZsXNBDNvj4/n8B cloud.google.com/go/storage v1.0.0/go.mod h1:IhtSnM/ZTZV8YYJWCY8RULGVqBDmpoyjwiyrjsg+URw= cloud.google.com/go/storage v1.5.0/go.mod h1:tpKbwo567HUNpVclU5sGELwQWBDZ8gh0ZeosJ0Rtdos= cloud.google.com/go/storage v1.6.0/go.mod h1:N7U0C8pVQ/+NIKOBQyamJIeKQKkZ+mxpohlUTyfDhBk= -cloud.google.com/go/storage v1.34.0 h1:9KHBBTbaHPsNxO043SFmH3pMojjZiW+BFl9H41L7xjk= -cloud.google.com/go/storage v1.34.0/go.mod h1:Eji+S0CCQebjsiXxyIvPItC3BN3zWsdJjWfHfoLblgY= +cloud.google.com/go/storage v1.34.1 h1:H2Af2dU5J0PF7A5B+ECFIce+RqxVnrVilO+cu0TS3MI= +cloud.google.com/go/storage v1.34.1/go.mod h1:VN1ElqqvR9adg1k9xlkUJ55cMOP1/QjnNNuT5xQL6dY= dario.cat/mergo v1.0.0 h1:AGCNq9Evsj31mOgNPcLyXc+4PNABt905YmuqPYYpBWk= dario.cat/mergo v1.0.0/go.mod h1:uNxQE+84aUszobStD9th8a29P2fMDhsBdgRYvZOxGmk= dmitri.shuralyov.com/gpu/mtl v0.0.0-20190408044501-666a987793e9/go.mod h1:H6x//7gZCb22OMCxBHrMx7a5I7Hp++hsVxbQ4BYO7hU= @@ -279,8 +279,8 @@ github.com/google/uuid v1.1.2/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+ github.com/google/uuid v1.2.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.4.0 h1:MtMxsa51/r9yyhkyLsVeVt0B+BGQZzpQiTQ4eHZ8bc4= github.com/google/uuid v1.4.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= -github.com/googleapis/enterprise-certificate-proxy v0.3.1 h1:SBWmZhjUDRorQxrN0nwzf+AHBxnbFjViHQS4P0yVpmQ= -github.com/googleapis/enterprise-certificate-proxy v0.3.1/go.mod h1:VLSiSSBs/ksPL8kq3OBOQ6WRI2QnaFynd1DCjZ62+V0= +github.com/googleapis/enterprise-certificate-proxy v0.3.2 h1:Vie5ybvEvT75RniqhfFxPRy3Bf7vr3h0cechB90XaQs= +github.com/googleapis/enterprise-certificate-proxy v0.3.2/go.mod h1:VLSiSSBs/ksPL8kq3OBOQ6WRI2QnaFynd1DCjZ62+V0= github.com/googleapis/gax-go/v2 v2.0.4/go.mod h1:0Wqv26UfaUD9n4G6kQubkQ+KchISgw+vpHVxEJEs9eg= github.com/googleapis/gax-go/v2 v2.0.5/go.mod h1:DWXyrwAJ9X0FpwwEdw+IPEYBICEFu5mhpdKc/us6bOk= github.com/googleapis/gax-go/v2 v2.12.0 h1:A+gCJKdRfqXkr+BIRGtZLibNXf0m1f9E4HG56etFpas= @@ -670,8 +670,8 @@ google.golang.org/api v0.14.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsb google.golang.org/api v0.15.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI= google.golang.org/api v0.17.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= google.golang.org/api v0.18.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= -google.golang.org/api v0.148.0 h1:HBq4TZlN4/1pNcu0geJZ/Q50vIwIXT532UIMYoo0vOs= -google.golang.org/api v0.148.0/go.mod h1:8/TBgwaKjfqTdacOJrOv2+2Q6fBDU1uHKK06oGSkxzU= +google.golang.org/api v0.149.0 h1:b2CqT6kG+zqJIVKRQ3ELJVLN1PwHZ6DJ3dW8yl82rgY= +google.golang.org/api v0.149.0/go.mod h1:Mwn1B7JTXrzXtnvmzQE2BD6bYZQ8DShKZDZbeN9I7qI= google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/appengine v1.5.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= @@ -697,12 +697,12 @@ google.golang.org/genproto v0.0.0-20200204135345-fa8e72b47b90/go.mod h1:GmwEX6Z4 google.golang.org/genproto v0.0.0-20200212174721-66ed5ce911ce/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= google.golang.org/genproto v0.0.0-20200224152610-e50cd9704f63/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= google.golang.org/genproto v0.0.0-20200526211855-cb27e3aa2013/go.mod h1:NbSheEEYHJ7i3ixzK3sjbqSGDJWnxyFXZblF3eUsNvo= -google.golang.org/genproto v0.0.0-20231002182017-d307bd883b97 h1:SeZZZx0cP0fqUyA+oRzP9k7cSwJlvDFiROO72uwD6i0= -google.golang.org/genproto v0.0.0-20231002182017-d307bd883b97/go.mod h1:t1VqOqqvce95G3hIDCT5FeO3YUc6Q4Oe24L/+rNMxRk= -google.golang.org/genproto/googleapis/api v0.0.0-20231002182017-d307bd883b97 h1:W18sezcAYs+3tDZX4F80yctqa12jcP1PUS2gQu1zTPU= -google.golang.org/genproto/googleapis/api v0.0.0-20231002182017-d307bd883b97/go.mod h1:iargEX0SFPm3xcfMI0d1domjg0ZF4Aa0p2awqyxhvF0= -google.golang.org/genproto/googleapis/rpc v0.0.0-20231012201019-e917dd12ba7a h1:a2MQQVoTo96JC9PMGtGBymLp7+/RzpFc2yX/9WfFg1c= -google.golang.org/genproto/googleapis/rpc v0.0.0-20231012201019-e917dd12ba7a/go.mod h1:4cYg8o5yUbm77w8ZX00LhMVNl/YVBFJRYWDc0uYWMs0= +google.golang.org/genproto v0.0.0-20231016165738-49dd2c1f3d0b h1:+YaDE2r2OG8t/z5qmsh7Y+XXwCbvadxxZ0YY6mTdrVA= +google.golang.org/genproto v0.0.0-20231016165738-49dd2c1f3d0b/go.mod h1:CgAqfJo+Xmu0GwA0411Ht3OU3OntXwsGmrmjI8ioGXI= +google.golang.org/genproto/googleapis/api v0.0.0-20231016165738-49dd2c1f3d0b h1:CIC2YMXmIhYw6evmhPxBKJ4fmLbOFtXQN/GV3XOZR8k= +google.golang.org/genproto/googleapis/api v0.0.0-20231016165738-49dd2c1f3d0b/go.mod h1:IBQ646DjkDkvUIsVq/cc03FUFQ9wbZu7yE396YcL870= +google.golang.org/genproto/googleapis/rpc v0.0.0-20231016165738-49dd2c1f3d0b h1:ZlWIi1wSK56/8hn4QcBp/j9M7Gt3U/3hZw3mC7vDICo= +google.golang.org/genproto/googleapis/rpc v0.0.0-20231016165738-49dd2c1f3d0b/go.mod h1:swOH3j0KzcDDgGUWr+SNpyTen5YrXjS3eyPzFYKc6lc= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.20.1/go.mod h1:10oTOabMzJvdu6/UiuZezV6QK5dSlG84ov/aaiqXj38= google.golang.org/grpc v1.21.1/go.mod h1:oYelfM1adQP15Ek0mdvEgi9Df8B9CZIaU1084ijfRaM= From edbeda78b7fab7a55198afdae1efb1513e4d0e1a Mon Sep 17 00:00:00 2001 From: Robert Burke <lostluck@users.noreply.github.com> Date: Thu, 2 Nov 2023 07:49:38 -0700 Subject: [PATCH 355/435] [#23893] Support composite scope transform metadata (#29204) * [#23893] Do Annotation plumbing to the graph. * fmt * Groundwork for DisplayData and Deps support * rm env change * Add plubming and simpler handling and test! * Docs and details. * Missed a rename. * pipeline doc --------- Co-authored-by: lostluck <13907733+lostluck@users.noreply.github.com> --- sdks/go/pkg/beam/core/graph/graph.go | 2 +- sdks/go/pkg/beam/core/graph/scope.go | 4 + .../core/runtime/contextreg/contextreg.go | 120 ++++++++++++++++++ .../runtime/contextreg/contextreg_test.go | 108 ++++++++++++++++ .../pkg/beam/core/runtime/graphx/translate.go | 17 +++ .../core/runtime/graphx/translate_test.go | 78 +++++++++++- sdks/go/pkg/beam/pipeline.go | 15 +++ 7 files changed, 341 insertions(+), 3 deletions(-) create mode 100644 sdks/go/pkg/beam/core/runtime/contextreg/contextreg.go create mode 100644 sdks/go/pkg/beam/core/runtime/contextreg/contextreg_test.go diff --git a/sdks/go/pkg/beam/core/graph/graph.go b/sdks/go/pkg/beam/core/graph/graph.go index 474ab1cb37da..f7826ccbef69 100644 --- a/sdks/go/pkg/beam/core/graph/graph.go +++ b/sdks/go/pkg/beam/core/graph/graph.go @@ -37,7 +37,7 @@ type Graph struct { // New returns an empty graph with the scope set to the root. func New() *Graph { - root := &Scope{0, "root", nil} + root := &Scope{id: 0, Label: "root", Parent: nil} return &Graph{root: root} } diff --git a/sdks/go/pkg/beam/core/graph/scope.go b/sdks/go/pkg/beam/core/graph/scope.go index 2fe836897c3d..8c8c3a041a5f 100644 --- a/sdks/go/pkg/beam/core/graph/scope.go +++ b/sdks/go/pkg/beam/core/graph/scope.go @@ -15,6 +15,8 @@ package graph +import "context" + // Scope is a syntactic Scope, such as arising from a composite Transform. It // has no semantic meaning at execution time. Used by monitoring. type Scope struct { @@ -24,6 +26,8 @@ type Scope struct { Label string // Parent is the parent scope, if nested. Parent *Scope + // Context contains optional metadata associated with this scope. + Context context.Context } // ID returns the graph-local identifier for the scope. diff --git a/sdks/go/pkg/beam/core/runtime/contextreg/contextreg.go b/sdks/go/pkg/beam/core/runtime/contextreg/contextreg.go new file mode 100644 index 000000000000..d91141477576 --- /dev/null +++ b/sdks/go/pkg/beam/core/runtime/contextreg/contextreg.go @@ -0,0 +1,120 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// Package contextreg contains the global registrations of functions for extracting +// ptransform annotations or environment resource hints from context.Context attached to +// scopes. +// +// For beam internal use only. API subject to change. +package contextreg + +import ( + "context" + "maps" + "sync" +) + +var defaultReg = &Registry{} + +// Default is the default registry for context extractors. +func Default() *Registry { + return defaultReg +} + +// Registry contains a set of registrations for extracting annotations and hints from a context.Context. +// +// This type is exported to allow simpler testing of new extractors, and their interaction with the registry. +type Registry struct { + mu sync.Mutex + transforms []func(context.Context) TransformMetadata + envs []func(context.Context) EnvironmentMetadata +} + +// TransformMetadata represents additional information on transforms to be added to the Pipeline proto graph. +type TransformMetadata struct { + Annotations map[string][]byte + // DisplayData []*pipepb.DisplayData +} + +// EnvironmentMetadata represent additional information on environmental requirements to be added to the Pipeline +// proto graph. +type EnvironmentMetadata struct { + ResourceHints map[string][]byte + // DisplayData []*pipepb.DisplayData + // Dependencies []*pipepb.ArtifactInformation +} + +// TransformExtractor registers a transform metadata extractor to this registry. +// These will be set on the current composite transform scope. +// They are accessible to runners via the transform hypergraph. +func (r *Registry) TransformExtractor(ext func(context.Context) TransformMetadata) { + r.mu.Lock() + r.transforms = append(r.transforms, ext) + r.mu.Unlock() +} + +// EnvExtrator registers an environment metadata extractor to this registry. +// When non-empty extraction occurs, a new environment will be derived from the parent scopes environment. +func (r *Registry) EnvExtrator(ext func(context.Context) EnvironmentMetadata) { + r.mu.Lock() + r.envs = append(r.envs, ext) + r.mu.Unlock() +} + +// ExtractTransformMetadata runs all registered transform extractors on the provided context, +// and returns the resulting metadata. +// +// A metadata field will be nil if there's no data. A nil context bypasses extractor execution. +func (r *Registry) ExtractTransformMetadata(ctx context.Context) TransformMetadata { + r.mu.Lock() + defer r.mu.Unlock() + if ctx == nil { + return TransformMetadata{} + } + ret := TransformMetadata{ + Annotations: map[string][]byte{}, + } + for _, ext := range r.transforms { + k := ext(ctx) + maps.Copy(ret.Annotations, k.Annotations) + } + if len(ret.Annotations) == 0 { + ret.Annotations = nil + } + return ret +} + +// ExtractEnvironmentMetadata runs all registered environment extractors on the provided context, +// and returns the resulting metadata. +// +// A metadata field will be nil if there's no data. A nil context bypasses extractor execution. +func (r *Registry) ExtractEnvironmentMetadata(ctx context.Context) EnvironmentMetadata { + r.mu.Lock() + defer r.mu.Unlock() + if ctx == nil { + return EnvironmentMetadata{} + } + ret := EnvironmentMetadata{ + ResourceHints: map[string][]byte{}, + } + for _, ext := range r.envs { + k := ext(ctx) + maps.Copy(ret.ResourceHints, k.ResourceHints) + } + if len(ret.ResourceHints) == 0 { + ret.ResourceHints = nil + } + return ret +} diff --git a/sdks/go/pkg/beam/core/runtime/contextreg/contextreg_test.go b/sdks/go/pkg/beam/core/runtime/contextreg/contextreg_test.go new file mode 100644 index 000000000000..dd0c5fb92c57 --- /dev/null +++ b/sdks/go/pkg/beam/core/runtime/contextreg/contextreg_test.go @@ -0,0 +1,108 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package contextreg + +import ( + "context" + "testing" +) + +func TestPTransformExtractor(t *testing.T) { + reg := &Registry{} + + type keyType string + key1 := keyType("annotation1") + key2 := keyType("annotation2") + key3 := keyType("annotation3") + + reg.TransformExtractor(func(ctx context.Context) TransformMetadata { + v := ctx.Value(key1).(string) + return TransformMetadata{ + Annotations: map[string][]byte{ + "beam:test:annotation": []byte(v), + }, + } + }) + reg.TransformExtractor(func(ctx context.Context) TransformMetadata { + v := ctx.Value(key2).(string) + return TransformMetadata{ + Annotations: map[string][]byte{ + "beam:test:annotation2": []byte(v), + }, + } + }) + // Override the extaction for result annotation to use the last set version. + reg.TransformExtractor(func(ctx context.Context) TransformMetadata { + v := ctx.Value(key3).(string) + return TransformMetadata{ + Annotations: map[string][]byte{ + "beam:test:annotation": []byte(v), + }, + } + }) + + ctx := context.Background() + // Set all 3 distinct context values. + ctx = context.WithValue(ctx, key1, "never seen") + want2 := "want_value2" + ctx = context.WithValue(ctx, key2, want2) + want3 := "want_value3" + ctx = context.WithValue(ctx, key3, want3) + + ptrans := reg.ExtractTransformMetadata(ctx) + + key := "beam:test:annotation" + if got, want := string(ptrans.Annotations[key]), want3; got != want { + t.Errorf("extracted annotation %q = %q, want %q", key, got, want) + } + key = "beam:test:annotation2" + if got, want := string(ptrans.Annotations[key]), want2; got != want { + t.Errorf("extracted annotation %q = %q, want %q", key, got, want) + } + if got, want := len(ptrans.Annotations), 2; got != want { + t.Errorf("extracted annotation %q = %q, want %q - have %v", key, got, want, ptrans) + } +} + +func TestHintExtractor(t *testing.T) { + reg := &Registry{} + + type keyType string + hintKey := keyType("hint") + + reg.EnvExtrator(func(ctx context.Context) EnvironmentMetadata { + v := ctx.Value(hintKey).(string) + return EnvironmentMetadata{ + ResourceHints: map[string][]byte{ + "beam:test:hint": []byte(v), + }, + } + }) + + ctx := context.Background() + wantedHint := "hint" + ctx = context.WithValue(ctx, hintKey, wantedHint) + + env := reg.ExtractEnvironmentMetadata(ctx) + + key := "beam:test:hint" + if got, want := string(env.ResourceHints[key]), wantedHint; got != want { + t.Errorf("extracted annotation %q = %q, want %q", key, got, want) + } + if got, want := len(env.ResourceHints), 1; got != want { + t.Errorf("extracted annotation %q = %q, want %q - have %v", key, got, want, env) + } +} diff --git a/sdks/go/pkg/beam/core/runtime/graphx/translate.go b/sdks/go/pkg/beam/core/runtime/graphx/translate.go index ad76703e3001..9ef28eb7809b 100644 --- a/sdks/go/pkg/beam/core/runtime/graphx/translate.go +++ b/sdks/go/pkg/beam/core/runtime/graphx/translate.go @@ -26,6 +26,7 @@ import ( "github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph/coder" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph/window" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph/window/trigger" + "github.com/apache/beam/sdks/v2/go/pkg/beam/core/runtime/contextreg" v1pb "github.com/apache/beam/sdks/v2/go/pkg/beam/core/runtime/graphx/v1" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/runtime/pipelinex" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/state" @@ -154,6 +155,18 @@ type Options struct { // PipelineResourceHints for setting defaults across the whole pipeline. PipelineResourceHints resource.Hints + + // ContextReg is an override for the context extractor registry for testing. + ContextReg *contextreg.Registry +} + +// GetContextReg returns the default context registry if the option is +// unset, and the field version otherwise. +func (opts *Options) GetContextReg() *contextreg.Registry { + if opts.ContextReg == nil { + return contextreg.Default() + } + return opts.ContextReg } // Marshal converts a graph to a model pipeline. @@ -273,10 +286,14 @@ func (m *marshaller) addScopeTree(s *ScopeTree) (string, error) { subtransforms = append(subtransforms, id) } + metadata := m.opt.GetContextReg().ExtractTransformMetadata(s.Scope.Scope.Context) + transform := &pipepb.PTransform{ UniqueName: s.Scope.Name, Subtransforms: subtransforms, EnvironmentId: m.addDefaultEnv(), + Annotations: metadata.Annotations, + // DisplayData: metadata.DisplayData, } if err := m.updateIfCombineComposite(s, transform); err != nil { diff --git a/sdks/go/pkg/beam/core/runtime/graphx/translate_test.go b/sdks/go/pkg/beam/core/runtime/graphx/translate_test.go index 2836351f2666..a331aedd585d 100644 --- a/sdks/go/pkg/beam/core/runtime/graphx/translate_test.go +++ b/sdks/go/pkg/beam/core/runtime/graphx/translate_test.go @@ -28,6 +28,7 @@ import ( "github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph/coder" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph/window" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/runtime" + "github.com/apache/beam/sdks/v2/go/pkg/beam/core/runtime/contextreg" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/runtime/graphx" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/typex" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/util/protox" @@ -165,8 +166,8 @@ func TestMarshal(t *testing.T) { if err != nil { t.Fatal(err) } - if len(edges) != test.edges { - t.Fatal("expected a single edge") + if got, want := len(edges), test.edges; got != want { + t.Fatalf("got %v edges, want %v", got, want) } payload, err := proto.Marshal(&pipepb.DockerPayload{ContainerImage: "foo"}) @@ -192,6 +193,79 @@ func TestMarshal(t *testing.T) { } } +func TestMarshal_PTransformAnnotations(t *testing.T) { + var creg contextreg.Registry + + const annotationKey = "myAnnotation" + + // A misused ptransform extractor that, if a context is attached to a scope will add an annotation to those transforms. + creg.TransformExtractor(func(ctx context.Context) contextreg.TransformMetadata { + return contextreg.TransformMetadata{ + Annotations: map[string][]byte{ + annotationKey: {42, 42, 42}, + }, + } + }) + + tests := []struct { + name string + makeGraph func(t *testing.T, g *graph.Graph) + + transforms int + }{ + { + name: "AnnotationSetOnComposite", + makeGraph: func(t *testing.T, g *graph.Graph) { + in := newIntInput(g) + side := newIntInput(g) + s := g.NewScope(g.Root(), "sub") + s.Context = context.Background() // Allow the default annotation to trigger. + addDoFn(t, g, pickSideFn, s, []*graph.Node{in, side}, []*coder.Coder{intCoder(), intCoder()}, nil) + }, + transforms: 2, + }, + } + for _, test := range tests { + test := test + t.Run(test.name, func(t *testing.T) { + g := graph.New() + test.makeGraph(t, g) + + edges, _, err := g.Build() + if err != nil { + t.Fatal(err) + } + + payload, err := proto.Marshal(&pipepb.DockerPayload{ContainerImage: "foo"}) + if err != nil { + t.Fatal(err) + } + p, err := graphx.Marshal(edges, + &graphx.Options{Environment: &pipepb.Environment{Urn: "beam:env:docker:v1", Payload: payload}, ContextReg: &creg}) + if err != nil { + t.Fatal(err) + } + + pts := p.GetComponents().GetTransforms() + if got, want := len(pts), test.transforms; got != want { + t.Errorf("got %d transforms, want %d : %v", got, want, proto.MarshalTextString(p)) + } + for _, pt := range pts { + // Context annotations only apply to composites, and are not duplicated to leaves. + if len(pt.GetSubtransforms()) == 0 { + if _, ok := pt.GetAnnotations()[annotationKey]; ok { + t.Errorf("unexpected annotation %v on leaf transform: %v", annotationKey, pt.GetAnnotations()) + } + continue + } + if _, ok := pt.GetAnnotations()[annotationKey]; !ok { + t.Errorf("expected %q annotation, but wasn't present: %v", annotationKey, pt.GetAnnotations()) + } + } + }) + } +} + // testRT's methods can all be no-ops, we just need it to implement sdf.RTracker. type testRT struct { } diff --git a/sdks/go/pkg/beam/pipeline.go b/sdks/go/pkg/beam/pipeline.go index b3a2a10dc1ba..c591eeb33722 100644 --- a/sdks/go/pkg/beam/pipeline.go +++ b/sdks/go/pkg/beam/pipeline.go @@ -16,6 +16,8 @@ package beam import ( + "context" + "github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/metrics" ) @@ -47,6 +49,19 @@ func (s Scope) Scope(name string) Scope { return Scope{scope: scope, real: s.real} } +// WithContext creates a named subscope with an attached context for the +// represented composite transform. Values from that context may be +// extracted and added to the composite PTransform or generate a new +// environment for scoped transforms. +// +// If you're not sure whether these apply to your transform, use Scope +// instead, and do not set a context. +func (s Scope) WithContext(ctx context.Context, name string) Scope { + newS := s.Scope(name) + newS.scope.Context = ctx + return newS +} + func (s Scope) String() string { if !s.IsValid() { return "<invalid>" From f98063c7e478ca61bba8800dadbca656a4ffff5a Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 2 Nov 2023 10:49:46 -0400 Subject: [PATCH 356/435] Bump github.com/aws/aws-sdk-go-v2/feature/s3/manager in /sdks (#29265) Bumps [github.com/aws/aws-sdk-go-v2/feature/s3/manager](https://github.com/aws/aws-sdk-go-v2) from 1.12.0 to 1.13.0. - [Release notes](https://github.com/aws/aws-sdk-go-v2/releases) - [Changelog](https://github.com/aws/aws-sdk-go-v2/blob/v1.13.0/CHANGELOG.md) - [Commits](https://github.com/aws/aws-sdk-go-v2/compare/v1.12.0...v1.13.0) --- updated-dependencies: - dependency-name: github.com/aws/aws-sdk-go-v2/feature/s3/manager dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] <support@github.com> Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 32 +++++++++++++-------------- sdks/go.sum | 64 ++++++++++++++++++++++++++--------------------------- 2 files changed, 48 insertions(+), 48 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 4d35f99aae46..5d2744cf574c 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -30,11 +30,11 @@ require ( cloud.google.com/go/pubsub v1.33.0 cloud.google.com/go/spanner v1.51.0 cloud.google.com/go/storage v1.34.1 - github.com/aws/aws-sdk-go-v2 v1.22.0 - github.com/aws/aws-sdk-go-v2/config v1.20.0 - github.com/aws/aws-sdk-go-v2/credentials v1.14.0 - github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.12.0 - github.com/aws/aws-sdk-go-v2/service/s3 v1.41.0 + github.com/aws/aws-sdk-go-v2 v1.22.1 + github.com/aws/aws-sdk-go-v2/config v1.21.0 + github.com/aws/aws-sdk-go-v2/credentials v1.15.0 + github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.13.0 + github.com/aws/aws-sdk-go-v2/service/s3 v1.42.0 github.com/aws/smithy-go v1.16.0 github.com/docker/go-connections v0.4.0 github.com/dustin/go-humanize v1.0.1 @@ -98,18 +98,18 @@ require ( github.com/apache/thrift v0.16.0 // indirect github.com/aws/aws-sdk-go v1.34.0 // indirect github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.5.0 // indirect - github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.0 // indirect - github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.0 // indirect - github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.0 // indirect - github.com/aws/aws-sdk-go-v2/internal/ini v1.4.0 // indirect - github.com/aws/aws-sdk-go-v2/internal/v4a v1.2.0 // indirect + github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.1 // indirect + github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.1 // indirect + github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.1 // indirect + github.com/aws/aws-sdk-go-v2/internal/ini v1.5.0 // indirect + github.com/aws/aws-sdk-go-v2/internal/v4a v1.2.1 // indirect github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.10.0 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.2.0 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.0 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.16.0 // indirect - github.com/aws/aws-sdk-go-v2/service/sso v1.16.0 // indirect - github.com/aws/aws-sdk-go-v2/service/ssooidc v1.18.0 // indirect - github.com/aws/aws-sdk-go-v2/service/sts v1.24.0 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.2.1 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.1 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.16.1 // indirect + github.com/aws/aws-sdk-go-v2/service/sso v1.17.0 // indirect + github.com/aws/aws-sdk-go-v2/service/ssooidc v1.19.0 // indirect + github.com/aws/aws-sdk-go-v2/service/sts v1.25.0 // indirect github.com/cenkalti/backoff/v4 v4.2.1 // indirect github.com/census-instrumentation/opencensus-proto v0.4.1 // indirect github.com/cespare/xxhash/v2 v2.2.0 // indirect diff --git a/sdks/go.sum b/sdks/go.sum index 64a21d35a35f..5844d2dcbe35 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -81,53 +81,53 @@ github.com/aws/aws-sdk-go v1.30.19/go.mod h1:5zCpMtNQVjRREroY7sYe8lOMRSxkhG6MZve github.com/aws/aws-sdk-go v1.34.0 h1:brux2dRrlwCF5JhTL7MUT3WUwo9zfDHZZp3+g3Mvlmo= github.com/aws/aws-sdk-go v1.34.0/go.mod h1:5zCpMtNQVjRREroY7sYe8lOMRSxkhG6MZveU8YkpAk0= github.com/aws/aws-sdk-go-v2 v1.7.1/go.mod h1:L5LuPC1ZgDr2xQS7AmIec/Jlc7O/Y1u2KxJyNVab250= -github.com/aws/aws-sdk-go-v2 v1.22.0 h1:CpTS3XO3MWNel8ohoazkLZC6scvkYL2k+m0yzFJ17Hg= -github.com/aws/aws-sdk-go-v2 v1.22.0/go.mod h1:Kd0OJtkW3Q0M0lUWGszapWjEvrXDzRW+D21JNsroB+c= +github.com/aws/aws-sdk-go-v2 v1.22.1 h1:sjnni/AuoTXxHitsIdT0FwmqUuNUuHtufcVDErVFT9U= +github.com/aws/aws-sdk-go-v2 v1.22.1/go.mod h1:Kd0OJtkW3Q0M0lUWGszapWjEvrXDzRW+D21JNsroB+c= github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.5.0 h1:hHgLiIrTRtddC0AKcJr5s7i/hLgcpTt+q/FKxf1Zayk= github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.5.0/go.mod h1:w4I/v3NOWgD+qvs1NPEwhd++1h3XPHFaVxasfY6HlYQ= github.com/aws/aws-sdk-go-v2/config v1.5.0/go.mod h1:RWlPOAW3E3tbtNAqTwvSW54Of/yP3oiZXMI0xfUdjyA= -github.com/aws/aws-sdk-go-v2/config v1.20.0 h1:q2+/mqFhY0J9m3Tb5RGFE3R4sdaUkIe4k2EuDfE3c08= -github.com/aws/aws-sdk-go-v2/config v1.20.0/go.mod h1:7+1riCZXyT+sAGvneR5j+Zl1GyfbBUNQurpQTE6FP6k= +github.com/aws/aws-sdk-go-v2/config v1.21.0 h1:BZEX5S6hkrAJ9rJn2hEMWIMkg+l1RC7gBQ84qkb0Cp0= +github.com/aws/aws-sdk-go-v2/config v1.21.0/go.mod h1:DMs4GYoTUo21V0cuDvwbdRb6coTcJ/EJxD7/WYZOinA= github.com/aws/aws-sdk-go-v2/credentials v1.3.1/go.mod h1:r0n73xwsIVagq8RsxmZbGSRQFj9As3je72C2WzUIToc= -github.com/aws/aws-sdk-go-v2/credentials v1.14.0 h1:LQquqPE7cL55RQmA/UBoBKehDlEtMnQKm3B0Q672ePE= -github.com/aws/aws-sdk-go-v2/credentials v1.14.0/go.mod h1:q/3oaTPlamrQWHPwJe56Mjq9g1TYDgddvgTgWJtHTmE= +github.com/aws/aws-sdk-go-v2/credentials v1.15.0 h1:gSRUMOU/wxxf89+4XYg0hYwOmcgdA0bohb7A/5nO+oE= +github.com/aws/aws-sdk-go-v2/credentials v1.15.0/go.mod h1:2zRQYW9jm3t18Ku+qP/107djyjAL7Ght6eBTWiNF/5c= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.3.0/go.mod h1:2LAuqPx1I6jNfaGDucWfA2zqQCYCOMCDHiCOciALyNw= -github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.0 h1:lF/cVllNAPKgjDwN2RsQUX9g/f6hXer9f10ubLFSoug= -github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.0/go.mod h1:c28nJNzMVVb9TQpZ5q4tzZvwEJwf/7So7Ie2s90l1Fw= +github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.1 h1:PYlUVUJRnXf2QU7IS+WLNSoU42f/oYRAmpwvXRH0Zq4= +github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.1/go.mod h1:wLyMIo/zPOhQhPXTddpfdkSleyigtFi8iMnC+2m/SK4= github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.3.2/go.mod h1:qaqQiHSrOUVOfKe6fhgQ6UzhxjwqVW8aHNegd6Ws4w4= -github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.12.0 h1:IJ1PN7TqW79Dmc0hSSd+clznCToXmN8gtWQO1hOxFbE= -github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.12.0/go.mod h1:Y0Ksox8/7Bl2UTwxdxWQ0JfRQi4sMv48F8Iy4ueS/Dc= -github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.0 h1:tN6dNNE4SzMuyMnVtQJXGVKX177/d5Zy4MuA1HA4KUc= -github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.0/go.mod h1:F6MXWETIeetAHwFHyoHEqrcB3NpijFv9nLP5h9CXtT0= -github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.0 h1:bfdsbTARDjaC/dSYGMO+E0psxFU4hTvCLnqYAfZ3D38= -github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.0/go.mod h1:Jg8XVv5M2V2wiAMvBFx+O59jg6Yr8vhP0bgNF/IuquM= +github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.13.0 h1:LPTcGCwO6bup9pTuOmYFXlcocpMX8PPO+TIB7CtSVnI= +github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.13.0/go.mod h1:QJXFEiHtNvgiOuBx9cyK6hC28xTB7mQ/iya6Xr1Fu7Y= +github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.1 h1:fi1ga6WysOyYb5PAf3Exd6B5GiSNpnZim4h1rhlBqx0= +github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.1/go.mod h1:V5CY8wNurvPUibTi9mwqUqpiFZ5LnioKWIFUDtIzdI8= +github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.1 h1:ZpaV/j48RlPc4AmOZuPv22pJliXjXq8/reL63YzyFnw= +github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.1/go.mod h1:R8aXraabD2e3qv1csxM14/X9WF4wFMIY0kH4YEtYD5M= github.com/aws/aws-sdk-go-v2/internal/ini v1.1.1/go.mod h1:Zy8smImhTdOETZqfyn01iNOe0CNggVbPjCajyaz6Gvg= -github.com/aws/aws-sdk-go-v2/internal/ini v1.4.0 h1:21tlTXq3ev10yLMAjXZzpkZbrl49h3ElSjmxD57tD/E= -github.com/aws/aws-sdk-go-v2/internal/ini v1.4.0/go.mod h1:d9YrBHJhyzDCv5UsEVRizHlFV6Q0sLemFq6uxuqWfUw= -github.com/aws/aws-sdk-go-v2/internal/v4a v1.2.0 h1:NpsAO1LaZyc72xMoQB/qgcOwI9Ag1d5FvaEp+omzFqg= -github.com/aws/aws-sdk-go-v2/internal/v4a v1.2.0/go.mod h1:6WVV80e6jigvvX0QqFDx3tjUME7qtNV9AJqGAZyc/R8= +github.com/aws/aws-sdk-go-v2/internal/ini v1.5.0 h1:DqOQvIfmGkXZUVJnl9VRk0AnxyS59tCtX9k1Pyss4Ak= +github.com/aws/aws-sdk-go-v2/internal/ini v1.5.0/go.mod h1:VV/Kbw9Mg1GWJOT9WK+oTL3cWZiXtapnNvDSRqTZLsg= +github.com/aws/aws-sdk-go-v2/internal/v4a v1.2.1 h1:vzYLDkwTw4CY0vUk84MeSufRf8XIsC/GsoIFXD60sTg= +github.com/aws/aws-sdk-go-v2/internal/v4a v1.2.1/go.mod h1:ToBFBnjeGR2ruMx8IWp/y7vSK3Irj5/oPwifruiqoOM= github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.2.1/go.mod h1:v33JQ57i2nekYTA70Mb+O18KeH4KqhdqxTJZNK1zdRE= github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.10.0 h1:CJxo7ZBbaIzmXfV3hjcx36n9V87gJsIUPJflwqEHl3Q= github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.10.0/go.mod h1:yjVfjuY4nD1EW9i387Kau+I6V5cBA5YnC/mWNopjZrI= -github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.2.0 h1:Wnw0IZKxx/PlKWbrUssl3Z2FP7cJS30QAeN1MuDVh1Q= -github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.2.0/go.mod h1:3ZHaPNnLwe0E+gAvdlA3Tl7M3SHohQloXL4hiDk1nec= +github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.2.1 h1:15FUCJzAP9Y25nioTqTrGlZmhOtthaXBWlt4pS+d3Xo= +github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.2.1/go.mod h1:5655NW53Un6l7JzkI6AA3rZvf0m532cSnLThA1fVXcA= github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.2.1/go.mod h1:zceowr5Z1Nh2WVP8bf/3ikB41IZW59E4yIYbg+pC6mw= -github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.0 h1:dJnwy5Awv+uvfk73aRENVbv1cSQQ60ydCkPaun097KM= -github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.0/go.mod h1:RsPWWy7u/hwmFX57sQ7MLvrvJeYyNkiMm5BaavpoU18= +github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.1 h1:2OXw3ppu1XsB6rqKEMV4tnecTjIY3PRV2U6IP6KPJQo= +github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.1/go.mod h1:FZB4AdakIqW/yERVdGJA6Z9jraax1beXfhBBnK2wwR8= github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.5.1/go.mod h1:6EQZIwNNvHpq/2/QSJnp4+ECvqIy55w95Ofs0ze+nGQ= -github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.16.0 h1:Kcts/mLwm4LxbF8YULGzilm+IeI4cBoesFUJpSsCcx0= -github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.16.0/go.mod h1:ca7+eyqTRByXctslWXXqhSwItEk+4y30azwLYJBBlsU= +github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.16.1 h1:dnl0klXYX9EKpzZbWlH5LJL+YTcEZcJEMPFFr/rAHUQ= +github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.16.1/go.mod h1:Mfk/9Joso4tCQYzM4q4HRUIqwln8lnIIMB/OE8Zebdc= github.com/aws/aws-sdk-go-v2/service/s3 v1.11.1/go.mod h1:XLAGFrEjbvMCLvAtWLLP32yTv8GpBquCApZEycDLunI= -github.com/aws/aws-sdk-go-v2/service/s3 v1.41.0 h1:XJfQwEGLnoqdVQtf+faXr7DMm/Q65SkgnGZJKWxIkPc= -github.com/aws/aws-sdk-go-v2/service/s3 v1.41.0/go.mod h1:DDsTwoAqmg5h2Up70/2XeCA4woeYdaD71PRucUwltFM= +github.com/aws/aws-sdk-go-v2/service/s3 v1.42.0 h1:u0YoSrxjr3Lm+IqIlRAV+4YTFwkXjyB9db9CfUFge2w= +github.com/aws/aws-sdk-go-v2/service/s3 v1.42.0/go.mod h1:98EIdRu+BNsdqITsXfy+57TZfwlUQC9aDn9a9qoo90U= github.com/aws/aws-sdk-go-v2/service/sso v1.3.1/go.mod h1:J3A3RGUvuCZjvSuZEcOpHDnzZP/sKbhDWV2T1EOzFIM= -github.com/aws/aws-sdk-go-v2/service/sso v1.16.0 h1:ZIlR6Wr/EgYwBdEz1NWBqdUsTh0mV7A68pId3YZl6H0= -github.com/aws/aws-sdk-go-v2/service/sso v1.16.0/go.mod h1:O7B5cpuhhJKefAKkM7onb0McmpHyKnsH4RrHJhOyq7M= -github.com/aws/aws-sdk-go-v2/service/ssooidc v1.18.0 h1:3BZyJei4k1SHdSAFhg9Qg15NnG3v5zosZyFWPm7df/A= -github.com/aws/aws-sdk-go-v2/service/ssooidc v1.18.0/go.mod h1:Td8EvzggonY02wLaqSpwybI3GbmA0PWoprKGil2uwJg= +github.com/aws/aws-sdk-go-v2/service/sso v1.17.0 h1:I/Oh3IxGPfHXiGnwM54TD6hNr/8TlUrBXAtTyGhR+zw= +github.com/aws/aws-sdk-go-v2/service/sso v1.17.0/go.mod h1:H6NCMvDBqA+CvIaXzaSqM6LWtzv9BzZrqBOqz+PzRF8= +github.com/aws/aws-sdk-go-v2/service/ssooidc v1.19.0 h1:irbXQkfVYIRaewYSXcu4yVk0m2T+JzZd0dkop7FjmO0= +github.com/aws/aws-sdk-go-v2/service/ssooidc v1.19.0/go.mod h1:4wPNCkM22+oRe71oydP66K50ojDUC33XutSMi2pEF/M= github.com/aws/aws-sdk-go-v2/service/sts v1.6.0/go.mod h1:q7o0j7d7HrJk/vr9uUt3BVRASvcU7gYZB9PUgPiByXg= -github.com/aws/aws-sdk-go-v2/service/sts v1.24.0 h1:f/V5Y9OaHuNRrA9MntNQNAtMFXqhKj8HTEPnH81eXMI= -github.com/aws/aws-sdk-go-v2/service/sts v1.24.0/go.mod h1:HnCUMNz2XqwnEEk5X6oeDYB2HgOLFpJ/LyfilN8WErs= +github.com/aws/aws-sdk-go-v2/service/sts v1.25.0 h1:sYIFy8tm1xQwRvVQ4CRuBGXKIg9sHNuG6+3UAQuoujk= +github.com/aws/aws-sdk-go-v2/service/sts v1.25.0/go.mod h1:S/LOQUeYDfJeJpFCIJDMjy7dwL4aA33HUdVi+i7uH8k= github.com/aws/smithy-go v1.6.0/go.mod h1:SObp3lf9smib00L/v3U2eAKG8FyQ7iLrJnQiAmR5n+E= github.com/aws/smithy-go v1.16.0 h1:gJZEH/Fqh+RsvlJ1Zt4tVAtV6bKkp3cC+R6FCZMNzik= github.com/aws/smithy-go v1.16.0/go.mod h1:NukqUGpCZIILqqiV0NIjeFh24kd/FAa4beRb6nbIUPE= From caf6339b369835add9b02b2059a96872b0398190 Mon Sep 17 00:00:00 2001 From: Jeffrey Kinard <jeff@thekinards.com> Date: Thu, 2 Nov 2023 11:40:32 -0400 Subject: [PATCH 357/435] [YAML] fix renaming provider caching and YAML schema validation Signed-off-by: Jeffrey Kinard <jeff@thekinards.com> --- sdks/python/apache_beam/yaml/pipeline.schema.yaml | 5 ++++- sdks/python/apache_beam/yaml/yaml_provider.py | 3 +++ 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/yaml/pipeline.schema.yaml b/sdks/python/apache_beam/yaml/pipeline.schema.yaml index e784531d9be1..e6b10f06e9a6 100644 --- a/sdks/python/apache_beam/yaml/pipeline.schema.yaml +++ b/sdks/python/apache_beam/yaml/pipeline.schema.yaml @@ -159,7 +159,10 @@ properties: # These are the only top-level properties defined in pipeline. - type: object properties: - type: { const: chain } + type: + oneOf: + - { const: composite } + - { const: chain } windowing: $ref: '#/$defs/windowing' transforms: {} diff --git a/sdks/python/apache_beam/yaml/yaml_provider.py b/sdks/python/apache_beam/yaml/yaml_provider.py index bf0231ee89d4..01e39b770c9b 100644 --- a/sdks/python/apache_beam/yaml/yaml_provider.py +++ b/sdks/python/apache_beam/yaml/yaml_provider.py @@ -765,6 +765,9 @@ def _affinity(self, other): def underlying_provider(self): return self._underlying_provider.underlying_provider() + def cache_artifacts(self): + self._underlying_provider.cache_artifacts() + def parse_providers(provider_specs): providers = collections.defaultdict(list) From 07c8bc4b4fa0528632f64d703cb921a7cefcb794 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw <robertwb@gmail.com> Date: Thu, 2 Nov 2023 10:10:04 -0700 Subject: [PATCH 358/435] [YAML] Note about error output schemas. --- sdks/python/apache_beam/yaml/yaml_errors.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/sdks/python/apache_beam/yaml/yaml_errors.md b/sdks/python/apache_beam/yaml/yaml_errors.md index e7a60f750a10..aec602393674 100644 --- a/sdks/python/apache_beam/yaml/yaml_errors.md +++ b/sdks/python/apache_beam/yaml/yaml_errors.md @@ -67,6 +67,10 @@ Note that with `error_handling` declared, `MapToFields.my_error_output` logging the bad records to stdout would be sufficient (though not recommended for a robust pipeline). +Note also that the exact format of the error outputs is still being finalized. +They can be safely printed and written to outputs, but their precise schema +may change in a future version of Beam and should not yet be depended on. + Some transforms allow for extra arguments in their error_handling config, e.g. for Python functions one can give a `threshold` which limits the relative number of records that can be bad before considering the entire pipeline a failure From bc29b966402825559dc2a7d34e20f679c4ca3c93 Mon Sep 17 00:00:00 2001 From: Yi Hu <yathu@google.com> Date: Thu, 2 Nov 2023 13:40:39 -0400 Subject: [PATCH 359/435] Fix XVR test fail python docker (#29256) --- sdks/java/container/common.gradle | 8 ++++---- sdks/python/container/common.gradle | 6 +++++- 2 files changed, 9 insertions(+), 5 deletions(-) diff --git a/sdks/java/container/common.gradle b/sdks/java/container/common.gradle index e4488d20f4c7..c242f714543e 100644 --- a/sdks/java/container/common.gradle +++ b/sdks/java/container/common.gradle @@ -137,8 +137,8 @@ docker { dockerfile project.file("../Dockerfile") files "./build/" buildArgs([ - 'pull_licenses': project.rootProject.hasProperty(["docker-pull-licenses"]) || - project.rootProject.hasProperty(["isRelease"]), + 'pull_licenses': project.rootProject.hasProperty("docker-pull-licenses") || + project.rootProject.hasProperty("isRelease"), 'base_image': javaBaseImage, 'java_version': imageJavaVersion, ]) @@ -148,8 +148,8 @@ docker { push pushContainers } -if (project.rootProject.hasProperty(["docker-pull-licenses"]) || - project.rootProject.hasProperty(["isRelease"])) { +if (project.rootProject.hasProperty("docker-pull-licenses") || + project.rootProject.hasProperty("isRelease")) { project.logger.lifecycle('docker-pull-licenses set, creating go-licenses') dockerPrepare.dependsOn copyJavaThirdPartyLicenses dockerPrepare.dependsOn copyGolangLicenses diff --git a/sdks/python/container/common.gradle b/sdks/python/container/common.gradle index efee0fbd2112..4996dae52db5 100644 --- a/sdks/python/container/common.gradle +++ b/sdks/python/container/common.gradle @@ -61,6 +61,10 @@ def copyLicenseScripts = tasks.register("copyLicenseScripts", Copy){ def copyLauncherDependencies = tasks.register("copyLauncherDependencies", Copy) { from configurations.pythonHarnessLauncher into "build/target/launcher" + + // Avoid seemingly gradle bug stated in https://github.com/apache/beam/issues/29220 + mustRunAfter "copyLicenses" + if(configurations.pythonHarnessLauncher.isEmpty()) { throw new StopExecutionException(); } @@ -93,7 +97,7 @@ dockerPrepare.dependsOn copyLauncherDependencies dockerPrepare.dependsOn copyDockerfileDependencies dockerPrepare.dependsOn copyLicenseScripts -if (project.rootProject.hasProperty(["docker-pull-licenses"])) { +if (project.rootProject.hasProperty("docker-pull-licenses")) { def copyGolangLicenses = tasks.register("copyGolangLicenses", Copy) { from "${project(':release:go-licenses:py').buildDir}/output" into "build/target/go-licenses" From 8afdda64983d92e78370ee2bd08176519e63150a Mon Sep 17 00:00:00 2001 From: reuvenlax <relax@google.com> Date: Thu, 2 Nov 2023 13:13:56 -0700 Subject: [PATCH 360/435] Merge pull request #28050: Don't invalidate streams on quota errors --- .../main/java/org/apache/beam/sdk/io/FileIO.java | 2 +- .../StorageApiWriteUnshardedRecords.java | 16 +++++++++++++--- .../bigquery/StorageApiWritesShardedRecords.java | 12 +++++++++--- 3 files changed, 23 insertions(+), 7 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileIO.java index 2d28279f90b6..76fc1a70b78c 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileIO.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileIO.java @@ -227,7 +227,7 @@ * {@link Sink}, e.g. write different elements to Avro files in different directories with different * schemas. * - * <p>This feature is supported by {@link #writeDynamic}. Use {@link Write#by} to specify how to + * <p>This feature is supported by {@link #writeDynamic}. Use {@link Write#by} to specify how too * partition the elements into groups ("destinations"). Then elements will be grouped by * destination, and {@link Write#withNaming(Contextful)} and {@link Write#via(Contextful)} will be * applied separately within each group, i.e. different groups will be written using the file naming diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWriteUnshardedRecords.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWriteUnshardedRecords.java index df79ece207f0..21c2a485c279 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWriteUnshardedRecords.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWriteUnshardedRecords.java @@ -711,7 +711,19 @@ long flush( retrieveErrorDetails(contexts)); failedContext.failureCount += 1; - invalidateWriteStream(); + boolean quotaError = false; + Throwable error = failedContext.getError(); + Status.Code statusCode = Status.Code.OK; + if (error != null) { + statusCode = Status.fromThrowable(error).getCode(); + quotaError = statusCode.equals(Status.Code.RESOURCE_EXHAUSTED); + } + + if (!quotaError) { + // This forces us to close and reopen all gRPC connections to Storage API on error, + // which empirically fixes random stuckness issues. + invalidateWriteStream(); + } // Maximum number of times we retry before we fail the work item. if (failedContext.failureCount > 5) { @@ -720,8 +732,6 @@ long flush( // The following errors are known to be persistent, so always fail the work item in // this case. - Throwable error = Preconditions.checkStateNotNull(failedContext.getError()); - Status.Code statusCode = Status.fromThrowable(error).getCode(); if (statusCode.equals(Status.Code.OUT_OF_RANGE) || statusCode.equals(Status.Code.ALREADY_EXISTS)) { throw new RuntimeException( diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWritesShardedRecords.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWritesShardedRecords.java index 77d9e8023a05..f4982396e9d5 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWritesShardedRecords.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWritesShardedRecords.java @@ -689,7 +689,7 @@ public void process( boolean offsetMismatch = statusCode.equals(Code.OUT_OF_RANGE) || statusCode.equals(Code.ALREADY_EXISTS); - // Invalidate the StreamWriter and force a new one to be created. + boolean quotaError = statusCode.equals(Code.RESOURCE_EXHAUSTED); if (!offsetMismatch) { // Don't log errors for expected offset mismatch. These will be logged as warnings // below. @@ -697,13 +697,19 @@ public void process( "Got error " + failedContext.getError() + " closing " + failedContext.streamName); } - // TODO: Only do this on explicit NOT_FOUND errors once BigQuery reliably produces them. try { + // TODO: Only do this on explicit NOT_FOUND errors once BigQuery reliably produces + // them. tryCreateTable.call(); } catch (Exception e) { throw new RuntimeException(e); } - clearClients.accept(failedContexts); + + if (!quotaError) { + // This forces us to close and reopen all gRPC connections to Storage API on error, + // which empirically fixes random stuckness issues. + clearClients.accept(failedContexts); + } appendFailures.inc(); boolean explicitStreamFinalized = From e11a82b02dff35bb6776478b2d8b742c6b6eaf11 Mon Sep 17 00:00:00 2001 From: Danny McCormick <dannymccormick@google.com> Date: Thu, 2 Nov 2023 17:11:57 -0400 Subject: [PATCH 361/435] Organize dashboard and make it more config driven (#29270) * Organize dashboard and make it more config driven * Rearrange * Fall back to rules --- .../GA-Post-Commits_status_dashboard.json | 2871 +---------------- .../metrics/sync/github/sync_workflows.py | 391 ++- 2 files changed, 392 insertions(+), 2870 deletions(-) diff --git a/.test-infra/metrics/grafana/dashboards/GA-Post-Commits_status_dashboard.json b/.test-infra/metrics/grafana/dashboards/GA-Post-Commits_status_dashboard.json index 52f4747688d4..3d0ed21734fe 100644 --- a/.test-infra/metrics/grafana/dashboards/GA-Post-Commits_status_dashboard.json +++ b/.test-infra/metrics/grafana/dashboards/GA-Post-Commits_status_dashboard.json @@ -935,7 +935,7 @@ "hide": false, "metricColumn": "none", "rawQuery": true, - "rawSql": "with workflows as (\n select\n *,\n case when run1 like 'success' then 1 when run1 like 'in_progress' then 2 when run1 like 'queued' then 2 when run1 like 'waiting' then 2 when run1 like 'cancelled' then 3 when run1 like 'None' then 4 else 0 end as run_1,\n case when run2 like 'success' then 1 when run2 like 'in_progress' then 2 when run2 like 'queued' then 2 when run2 like 'waiting' then 2 when run2 like 'cancelled' then 3 when run2 like 'None' then 4 else 0 end as run_2,\n case when run3 like 'success' then 1 when run3 like 'in_progress' then 2 when run3 like 'queued' then 2 when run3 like 'waiting' then 2 when run3 like 'cancelled' then 3 when run3 like 'None' then 4 else 0 end as run_3,\n case when run4 like 'success' then 1 when run4 like 'in_progress' then 2 when run4 like 'queued' then 2 when run4 like 'waiting' then 2 when run4 like 'cancelled' then 3 when run4 like 'None' then 4 else 0 end as run_4,\n case when run5 like 'success' then 1 when run5 like 'in_progress' then 2 when run5 like 'queued' then 2 when run5 like 'waiting' then 2 when run5 like 'cancelled' then 3 when run5 like 'None' then 4 else 0 end as run_5,\n case when run6 like 'success' then 1 when run6 like 'in_progress' then 2 when run6 like 'queued' then 2 when run6 like 'waiting' then 2 when run6 like 'cancelled' then 3 when run6 like 'None' then 4 else 0 end as run_6,\n case when run7 like 'success' then 1 when run7 like 'in_progress' then 2 when run7 like 'queued' then 2 when run7 like 'waiting' then 2 when run7 like 'cancelled' then 3 when run7 like 'None' then 4 else 0 end as run_7,\n case when run8 like 'success' then 1 when run8 like 'in_progress' then 2 when run8 like 'queued' then 2 when run8 like 'waiting' then 2 when run8 like 'cancelled' then 3 when run8 like 'None' then 4 else 0 end as run_8,\n case when run9 like 'success' then 1 when run9 like 'in_progress' then 2 when run9 like 'queued' then 2 when run9 like 'waiting' then 2 when run9 like 'cancelled' then 3 when run9 like 'None' then 4 else 0 end as run_9,\n case when run10 like 'success' then 1 when run10 like 'in_progress' then 2 when run10 like 'queued' then 2 when run10 like 'waiting' then 2 when run10 like 'cancelled' then 3 when run10 like 'None' then 4 else 0 end as run_10\n from\n github_workflows\n where\n dashboard_category = 'core_java'\n)\nselect\n job_name,\n job_yml_filename,\n run_1,\n run1Id,\n run_2,\n run2Id,\n run_3,\n run3Id,\n run_4,\n run4Id,\n run_5,\n run5Id,\n run_6,\n run6Id,\n run_7,\n run7Id,\n run_8,\n run8Id,\n run_9,\n run9Id,\n run_10,\n run10Id\nfrom\n workflows;", + "rawSql": "with workflows as (\n select\n *,\n case when run1 like 'success' then 1 when run1 like 'in_progress' then 2 when run1 like 'queued' then 2 when run1 like 'waiting' then 2 when run1 like 'cancelled' then 3 when run1 like 'None' then 4 else 0 end as run_1,\n case when run2 like 'success' then 1 when run2 like 'in_progress' then 2 when run2 like 'queued' then 2 when run2 like 'waiting' then 2 when run2 like 'cancelled' then 3 when run2 like 'None' then 4 else 0 end as run_2,\n case when run3 like 'success' then 1 when run3 like 'in_progress' then 2 when run3 like 'queued' then 2 when run3 like 'waiting' then 2 when run3 like 'cancelled' then 3 when run3 like 'None' then 4 else 0 end as run_3,\n case when run4 like 'success' then 1 when run4 like 'in_progress' then 2 when run4 like 'queued' then 2 when run4 like 'waiting' then 2 when run4 like 'cancelled' then 3 when run4 like 'None' then 4 else 0 end as run_4,\n case when run5 like 'success' then 1 when run5 like 'in_progress' then 2 when run5 like 'queued' then 2 when run5 like 'waiting' then 2 when run5 like 'cancelled' then 3 when run5 like 'None' then 4 else 0 end as run_5,\n case when run6 like 'success' then 1 when run6 like 'in_progress' then 2 when run6 like 'queued' then 2 when run6 like 'waiting' then 2 when run6 like 'cancelled' then 3 when run6 like 'None' then 4 else 0 end as run_6,\n case when run7 like 'success' then 1 when run7 like 'in_progress' then 2 when run7 like 'queued' then 2 when run7 like 'waiting' then 2 when run7 like 'cancelled' then 3 when run7 like 'None' then 4 else 0 end as run_7,\n case when run8 like 'success' then 1 when run8 like 'in_progress' then 2 when run8 like 'queued' then 2 when run8 like 'waiting' then 2 when run8 like 'cancelled' then 3 when run8 like 'None' then 4 else 0 end as run_8,\n case when run9 like 'success' then 1 when run9 like 'in_progress' then 2 when run9 like 'queued' then 2 when run9 like 'waiting' then 2 when run9 like 'cancelled' then 3 when run9 like 'None' then 4 else 0 end as run_9,\n case when run10 like 'success' then 1 when run10 like 'in_progress' then 2 when run10 like 'queued' then 2 when run10 like 'waiting' then 2 when run10 like 'cancelled' then 3 when run10 like 'None' then 4 else 0 end as run_10\n from\n github_workflows\n where\n dashboard_category = 'core_infra'\n)\nselect\n job_name,\n job_yml_filename,\n run_1,\n run1Id,\n run_2,\n run2Id,\n run_3,\n run3Id,\n run_4,\n run4Id,\n run_5,\n run5Id,\n run_6,\n run6Id,\n run_7,\n run7Id,\n run_8,\n run8Id,\n run_9,\n run9Id,\n run_10,\n run10Id\nfrom\n workflows;", "refId": "A", "select": [ [ @@ -959,7 +959,7 @@ ] } ], - "title": "Core Java Tests", + "title": "Core Infrastructure", "transformations": [ { "id": "merge", @@ -1880,7 +1880,7 @@ "hide": false, "metricColumn": "none", "rawQuery": true, - "rawSql": "with workflows as (\n select\n *,\n case when run1 like 'success' then 1 when run1 like 'in_progress' then 2 when run1 like 'queued' then 2 when run1 like 'waiting' then 2 when run1 like 'cancelled' then 3 when run1 like 'None' then 4 else 0 end as run_1,\n case when run2 like 'success' then 1 when run2 like 'in_progress' then 2 when run2 like 'queued' then 2 when run2 like 'waiting' then 2 when run2 like 'cancelled' then 3 when run2 like 'None' then 4 else 0 end as run_2,\n case when run3 like 'success' then 1 when run3 like 'in_progress' then 2 when run3 like 'queued' then 2 when run3 like 'waiting' then 2 when run3 like 'cancelled' then 3 when run3 like 'None' then 4 else 0 end as run_3,\n case when run4 like 'success' then 1 when run4 like 'in_progress' then 2 when run4 like 'queued' then 2 when run4 like 'waiting' then 2 when run4 like 'cancelled' then 3 when run4 like 'None' then 4 else 0 end as run_4,\n case when run5 like 'success' then 1 when run5 like 'in_progress' then 2 when run5 like 'queued' then 2 when run5 like 'waiting' then 2 when run5 like 'cancelled' then 3 when run5 like 'None' then 4 else 0 end as run_5,\n case when run6 like 'success' then 1 when run6 like 'in_progress' then 2 when run6 like 'queued' then 2 when run6 like 'waiting' then 2 when run6 like 'cancelled' then 3 when run6 like 'None' then 4 else 0 end as run_6,\n case when run7 like 'success' then 1 when run7 like 'in_progress' then 2 when run7 like 'queued' then 2 when run7 like 'waiting' then 2 when run7 like 'cancelled' then 3 when run7 like 'None' then 4 else 0 end as run_7,\n case when run8 like 'success' then 1 when run8 like 'in_progress' then 2 when run8 like 'queued' then 2 when run8 like 'waiting' then 2 when run8 like 'cancelled' then 3 when run8 like 'None' then 4 else 0 end as run_8,\n case when run9 like 'success' then 1 when run9 like 'in_progress' then 2 when run9 like 'queued' then 2 when run9 like 'waiting' then 2 when run9 like 'cancelled' then 3 when run9 like 'None' then 4 else 0 end as run_9,\n case when run10 like 'success' then 1 when run10 like 'in_progress' then 2 when run10 like 'queued' then 2 when run10 like 'waiting' then 2 when run10 like 'cancelled' then 3 when run10 like 'None' then 4 else 0 end as run_10\n from\n github_workflows\n where\n dashboard_category = 'dataflow_java'\n)\nselect\n job_name,\n job_yml_filename,\n run_1,\n run1Id,\n run_2,\n run2Id,\n run_3,\n run3Id,\n run_4,\n run4Id,\n run_5,\n run5Id,\n run_6,\n run6Id,\n run_7,\n run7Id,\n run_8,\n run8Id,\n run_9,\n run9Id,\n run_10,\n run10Id\nfrom\n workflows;", + "rawSql": "with workflows as (\n select\n *,\n case when run1 like 'success' then 1 when run1 like 'in_progress' then 2 when run1 like 'queued' then 2 when run1 like 'waiting' then 2 when run1 like 'cancelled' then 3 when run1 like 'None' then 4 else 0 end as run_1,\n case when run2 like 'success' then 1 when run2 like 'in_progress' then 2 when run2 like 'queued' then 2 when run2 like 'waiting' then 2 when run2 like 'cancelled' then 3 when run2 like 'None' then 4 else 0 end as run_2,\n case when run3 like 'success' then 1 when run3 like 'in_progress' then 2 when run3 like 'queued' then 2 when run3 like 'waiting' then 2 when run3 like 'cancelled' then 3 when run3 like 'None' then 4 else 0 end as run_3,\n case when run4 like 'success' then 1 when run4 like 'in_progress' then 2 when run4 like 'queued' then 2 when run4 like 'waiting' then 2 when run4 like 'cancelled' then 3 when run4 like 'None' then 4 else 0 end as run_4,\n case when run5 like 'success' then 1 when run5 like 'in_progress' then 2 when run5 like 'queued' then 2 when run5 like 'waiting' then 2 when run5 like 'cancelled' then 3 when run5 like 'None' then 4 else 0 end as run_5,\n case when run6 like 'success' then 1 when run6 like 'in_progress' then 2 when run6 like 'queued' then 2 when run6 like 'waiting' then 2 when run6 like 'cancelled' then 3 when run6 like 'None' then 4 else 0 end as run_6,\n case when run7 like 'success' then 1 when run7 like 'in_progress' then 2 when run7 like 'queued' then 2 when run7 like 'waiting' then 2 when run7 like 'cancelled' then 3 when run7 like 'None' then 4 else 0 end as run_7,\n case when run8 like 'success' then 1 when run8 like 'in_progress' then 2 when run8 like 'queued' then 2 when run8 like 'waiting' then 2 when run8 like 'cancelled' then 3 when run8 like 'None' then 4 else 0 end as run_8,\n case when run9 like 'success' then 1 when run9 like 'in_progress' then 2 when run9 like 'queued' then 2 when run9 like 'waiting' then 2 when run9 like 'cancelled' then 3 when run9 like 'None' then 4 else 0 end as run_9,\n case when run10 like 'success' then 1 when run10 like 'in_progress' then 2 when run10 like 'queued' then 2 when run10 like 'waiting' then 2 when run10 like 'cancelled' then 3 when run10 like 'None' then 4 else 0 end as run_10\n from\n github_workflows\n where\n dashboard_category = 'core_java'\n)\nselect\n job_name,\n job_yml_filename,\n run_1,\n run1Id,\n run_2,\n run2Id,\n run_3,\n run3Id,\n run_4,\n run4Id,\n run_5,\n run5Id,\n run_6,\n run6Id,\n run_7,\n run7Id,\n run_8,\n run8Id,\n run_9,\n run9Id,\n run_10,\n run10Id\nfrom\n workflows;", "refId": "A", "select": [ [ @@ -1904,7 +1904,7 @@ ] } ], - "title": "Dataflow Java Tests", + "title": "Core Java Tests", "transformations": [ { "id": "merge", @@ -2825,7 +2825,7 @@ "hide": false, "metricColumn": "none", "rawQuery": true, - "rawSql": "with workflows as (\n select\n *,\n case when run1 like 'success' then 1 when run1 like 'in_progress' then 2 when run1 like 'queued' then 2 when run1 like 'waiting' then 2 when run1 like 'cancelled' then 3 when run1 like 'None' then 4 else 0 end as run_1,\n case when run2 like 'success' then 1 when run2 like 'in_progress' then 2 when run2 like 'queued' then 2 when run2 like 'waiting' then 2 when run2 like 'cancelled' then 3 when run2 like 'None' then 4 else 0 end as run_2,\n case when run3 like 'success' then 1 when run3 like 'in_progress' then 2 when run3 like 'queued' then 2 when run3 like 'waiting' then 2 when run3 like 'cancelled' then 3 when run3 like 'None' then 4 else 0 end as run_3,\n case when run4 like 'success' then 1 when run4 like 'in_progress' then 2 when run4 like 'queued' then 2 when run4 like 'waiting' then 2 when run4 like 'cancelled' then 3 when run4 like 'None' then 4 else 0 end as run_4,\n case when run5 like 'success' then 1 when run5 like 'in_progress' then 2 when run5 like 'queued' then 2 when run5 like 'waiting' then 2 when run5 like 'cancelled' then 3 when run5 like 'None' then 4 else 0 end as run_5,\n case when run6 like 'success' then 1 when run6 like 'in_progress' then 2 when run6 like 'queued' then 2 when run6 like 'waiting' then 2 when run6 like 'cancelled' then 3 when run6 like 'None' then 4 else 0 end as run_6,\n case when run7 like 'success' then 1 when run7 like 'in_progress' then 2 when run7 like 'queued' then 2 when run7 like 'waiting' then 2 when run7 like 'cancelled' then 3 when run7 like 'None' then 4 else 0 end as run_7,\n case when run8 like 'success' then 1 when run8 like 'in_progress' then 2 when run8 like 'queued' then 2 when run8 like 'waiting' then 2 when run8 like 'cancelled' then 3 when run8 like 'None' then 4 else 0 end as run_8,\n case when run9 like 'success' then 1 when run9 like 'in_progress' then 2 when run9 like 'queued' then 2 when run9 like 'waiting' then 2 when run9 like 'cancelled' then 3 when run9 like 'None' then 4 else 0 end as run_9,\n case when run10 like 'success' then 1 when run10 like 'in_progress' then 2 when run10 like 'queued' then 2 when run10 like 'waiting' then 2 when run10 like 'cancelled' then 3 when run10 like 'None' then 4 else 0 end as run_10\n from\n github_workflows\n where\n dashboard_category = 'spark_java'\n)\nselect\n job_name,\n job_yml_filename,\n run_1,\n run1Id,\n run_2,\n run2Id,\n run_3,\n run3Id,\n run_4,\n run4Id,\n run_5,\n run5Id,\n run_6,\n run6Id,\n run_7,\n run7Id,\n run_8,\n run8Id,\n run_9,\n run9Id,\n run_10,\n run10Id\nfrom\n workflows;", + "rawSql": "with workflows as (\n select\n *,\n case when run1 like 'success' then 1 when run1 like 'in_progress' then 2 when run1 like 'queued' then 2 when run1 like 'waiting' then 2 when run1 like 'cancelled' then 3 when run1 like 'None' then 4 else 0 end as run_1,\n case when run2 like 'success' then 1 when run2 like 'in_progress' then 2 when run2 like 'queued' then 2 when run2 like 'waiting' then 2 when run2 like 'cancelled' then 3 when run2 like 'None' then 4 else 0 end as run_2,\n case when run3 like 'success' then 1 when run3 like 'in_progress' then 2 when run3 like 'queued' then 2 when run3 like 'waiting' then 2 when run3 like 'cancelled' then 3 when run3 like 'None' then 4 else 0 end as run_3,\n case when run4 like 'success' then 1 when run4 like 'in_progress' then 2 when run4 like 'queued' then 2 when run4 like 'waiting' then 2 when run4 like 'cancelled' then 3 when run4 like 'None' then 4 else 0 end as run_4,\n case when run5 like 'success' then 1 when run5 like 'in_progress' then 2 when run5 like 'queued' then 2 when run5 like 'waiting' then 2 when run5 like 'cancelled' then 3 when run5 like 'None' then 4 else 0 end as run_5,\n case when run6 like 'success' then 1 when run6 like 'in_progress' then 2 when run6 like 'queued' then 2 when run6 like 'waiting' then 2 when run6 like 'cancelled' then 3 when run6 like 'None' then 4 else 0 end as run_6,\n case when run7 like 'success' then 1 when run7 like 'in_progress' then 2 when run7 like 'queued' then 2 when run7 like 'waiting' then 2 when run7 like 'cancelled' then 3 when run7 like 'None' then 4 else 0 end as run_7,\n case when run8 like 'success' then 1 when run8 like 'in_progress' then 2 when run8 like 'queued' then 2 when run8 like 'waiting' then 2 when run8 like 'cancelled' then 3 when run8 like 'None' then 4 else 0 end as run_8,\n case when run9 like 'success' then 1 when run9 like 'in_progress' then 2 when run9 like 'queued' then 2 when run9 like 'waiting' then 2 when run9 like 'cancelled' then 3 when run9 like 'None' then 4 else 0 end as run_9,\n case when run10 like 'success' then 1 when run10 like 'in_progress' then 2 when run10 like 'queued' then 2 when run10 like 'waiting' then 2 when run10 like 'cancelled' then 3 when run10 like 'None' then 4 else 0 end as run_10\n from\n github_workflows\n where\n dashboard_category = 'dataflow_java'\n)\nselect\n job_name,\n job_yml_filename,\n run_1,\n run1Id,\n run_2,\n run2Id,\n run_3,\n run3Id,\n run_4,\n run4Id,\n run_5,\n run5Id,\n run_6,\n run6Id,\n run_7,\n run7Id,\n run_8,\n run8Id,\n run_9,\n run9Id,\n run_10,\n run10Id\nfrom\n workflows;", "refId": "A", "select": [ [ @@ -2849,7 +2849,7 @@ ] } ], - "title": "Spark Java Tests", + "title": "Dataflow Java Tests", "transformations": [ { "id": "merge", @@ -3770,7 +3770,7 @@ "hide": false, "metricColumn": "none", "rawQuery": true, - "rawSql": "with workflows as (\n select\n *,\n case when run1 like 'success' then 1 when run1 like 'in_progress' then 2 when run1 like 'queued' then 2 when run1 like 'waiting' then 2 when run1 like 'cancelled' then 3 when run1 like 'None' then 4 else 0 end as run_1,\n case when run2 like 'success' then 1 when run2 like 'in_progress' then 2 when run2 like 'queued' then 2 when run2 like 'waiting' then 2 when run2 like 'cancelled' then 3 when run2 like 'None' then 4 else 0 end as run_2,\n case when run3 like 'success' then 1 when run3 like 'in_progress' then 2 when run3 like 'queued' then 2 when run3 like 'waiting' then 2 when run3 like 'cancelled' then 3 when run3 like 'None' then 4 else 0 end as run_3,\n case when run4 like 'success' then 1 when run4 like 'in_progress' then 2 when run4 like 'queued' then 2 when run4 like 'waiting' then 2 when run4 like 'cancelled' then 3 when run4 like 'None' then 4 else 0 end as run_4,\n case when run5 like 'success' then 1 when run5 like 'in_progress' then 2 when run5 like 'queued' then 2 when run5 like 'waiting' then 2 when run5 like 'cancelled' then 3 when run5 like 'None' then 4 else 0 end as run_5,\n case when run6 like 'success' then 1 when run6 like 'in_progress' then 2 when run6 like 'queued' then 2 when run6 like 'waiting' then 2 when run6 like 'cancelled' then 3 when run6 like 'None' then 4 else 0 end as run_6,\n case when run7 like 'success' then 1 when run7 like 'in_progress' then 2 when run7 like 'queued' then 2 when run7 like 'waiting' then 2 when run7 like 'cancelled' then 3 when run7 like 'None' then 4 else 0 end as run_7,\n case when run8 like 'success' then 1 when run8 like 'in_progress' then 2 when run8 like 'queued' then 2 when run8 like 'waiting' then 2 when run8 like 'cancelled' then 3 when run8 like 'None' then 4 else 0 end as run_8,\n case when run9 like 'success' then 1 when run9 like 'in_progress' then 2 when run9 like 'queued' then 2 when run9 like 'waiting' then 2 when run9 like 'cancelled' then 3 when run9 like 'None' then 4 else 0 end as run_9,\n case when run10 like 'success' then 1 when run10 like 'in_progress' then 2 when run10 like 'queued' then 2 when run10 like 'waiting' then 2 when run10 like 'cancelled' then 3 when run10 like 'None' then 4 else 0 end as run_10\n from\n github_workflows\n where\n dashboard_category = 'flink_java'\n)\nselect\n job_name,\n job_yml_filename,\n run_1,\n run1Id,\n run_2,\n run2Id,\n run_3,\n run3Id,\n run_4,\n run4Id,\n run_5,\n run5Id,\n run_6,\n run6Id,\n run_7,\n run7Id,\n run_8,\n run8Id,\n run_9,\n run9Id,\n run_10,\n run10Id\nfrom\n workflows;", + "rawSql": "with workflows as (\n select\n *,\n case when run1 like 'success' then 1 when run1 like 'in_progress' then 2 when run1 like 'queued' then 2 when run1 like 'waiting' then 2 when run1 like 'cancelled' then 3 when run1 like 'None' then 4 else 0 end as run_1,\n case when run2 like 'success' then 1 when run2 like 'in_progress' then 2 when run2 like 'queued' then 2 when run2 like 'waiting' then 2 when run2 like 'cancelled' then 3 when run2 like 'None' then 4 else 0 end as run_2,\n case when run3 like 'success' then 1 when run3 like 'in_progress' then 2 when run3 like 'queued' then 2 when run3 like 'waiting' then 2 when run3 like 'cancelled' then 3 when run3 like 'None' then 4 else 0 end as run_3,\n case when run4 like 'success' then 1 when run4 like 'in_progress' then 2 when run4 like 'queued' then 2 when run4 like 'waiting' then 2 when run4 like 'cancelled' then 3 when run4 like 'None' then 4 else 0 end as run_4,\n case when run5 like 'success' then 1 when run5 like 'in_progress' then 2 when run5 like 'queued' then 2 when run5 like 'waiting' then 2 when run5 like 'cancelled' then 3 when run5 like 'None' then 4 else 0 end as run_5,\n case when run6 like 'success' then 1 when run6 like 'in_progress' then 2 when run6 like 'queued' then 2 when run6 like 'waiting' then 2 when run6 like 'cancelled' then 3 when run6 like 'None' then 4 else 0 end as run_6,\n case when run7 like 'success' then 1 when run7 like 'in_progress' then 2 when run7 like 'queued' then 2 when run7 like 'waiting' then 2 when run7 like 'cancelled' then 3 when run7 like 'None' then 4 else 0 end as run_7,\n case when run8 like 'success' then 1 when run8 like 'in_progress' then 2 when run8 like 'queued' then 2 when run8 like 'waiting' then 2 when run8 like 'cancelled' then 3 when run8 like 'None' then 4 else 0 end as run_8,\n case when run9 like 'success' then 1 when run9 like 'in_progress' then 2 when run9 like 'queued' then 2 when run9 like 'waiting' then 2 when run9 like 'cancelled' then 3 when run9 like 'None' then 4 else 0 end as run_9,\n case when run10 like 'success' then 1 when run10 like 'in_progress' then 2 when run10 like 'queued' then 2 when run10 like 'waiting' then 2 when run10 like 'cancelled' then 3 when run10 like 'None' then 4 else 0 end as run_10\n from\n github_workflows\n where\n dashboard_category = 'runners_java'\n)\nselect\n job_name,\n job_yml_filename,\n run_1,\n run1Id,\n run_2,\n run2Id,\n run_3,\n run3Id,\n run_4,\n run4Id,\n run_5,\n run5Id,\n run_6,\n run6Id,\n run_7,\n run7Id,\n run_8,\n run8Id,\n run_9,\n run9Id,\n run_10,\n run10Id\nfrom\n workflows;", "refId": "A", "select": [ [ @@ -3794,7 +3794,7 @@ ] } ], - "title": "Flink Java Tests", + "title": "Java Runners Tests", "transformations": [ { "id": "merge", @@ -4715,7 +4715,7 @@ "hide": false, "metricColumn": "none", "rawQuery": true, - "rawSql": "with workflows as (\n select\n *,\n case when run1 like 'success' then 1 when run1 like 'in_progress' then 2 when run1 like 'queued' then 2 when run1 like 'waiting' then 2 when run1 like 'cancelled' then 3 when run1 like 'None' then 4 else 0 end as run_1,\n case when run2 like 'success' then 1 when run2 like 'in_progress' then 2 when run2 like 'queued' then 2 when run2 like 'waiting' then 2 when run2 like 'cancelled' then 3 when run2 like 'None' then 4 else 0 end as run_2,\n case when run3 like 'success' then 1 when run3 like 'in_progress' then 2 when run3 like 'queued' then 2 when run3 like 'waiting' then 2 when run3 like 'cancelled' then 3 when run3 like 'None' then 4 else 0 end as run_3,\n case when run4 like 'success' then 1 when run4 like 'in_progress' then 2 when run4 like 'queued' then 2 when run4 like 'waiting' then 2 when run4 like 'cancelled' then 3 when run4 like 'None' then 4 else 0 end as run_4,\n case when run5 like 'success' then 1 when run5 like 'in_progress' then 2 when run5 like 'queued' then 2 when run5 like 'waiting' then 2 when run5 like 'cancelled' then 3 when run5 like 'None' then 4 else 0 end as run_5,\n case when run6 like 'success' then 1 when run6 like 'in_progress' then 2 when run6 like 'queued' then 2 when run6 like 'waiting' then 2 when run6 like 'cancelled' then 3 when run6 like 'None' then 4 else 0 end as run_6,\n case when run7 like 'success' then 1 when run7 like 'in_progress' then 2 when run7 like 'queued' then 2 when run7 like 'waiting' then 2 when run7 like 'cancelled' then 3 when run7 like 'None' then 4 else 0 end as run_7,\n case when run8 like 'success' then 1 when run8 like 'in_progress' then 2 when run8 like 'queued' then 2 when run8 like 'waiting' then 2 when run8 like 'cancelled' then 3 when run8 like 'None' then 4 else 0 end as run_8,\n case when run9 like 'success' then 1 when run9 like 'in_progress' then 2 when run9 like 'queued' then 2 when run9 like 'waiting' then 2 when run9 like 'cancelled' then 3 when run9 like 'None' then 4 else 0 end as run_9,\n case when run10 like 'success' then 1 when run10 like 'in_progress' then 2 when run10 like 'queued' then 2 when run10 like 'waiting' then 2 when run10 like 'cancelled' then 3 when run10 like 'None' then 4 else 0 end as run_10\n from\n github_workflows\n where\n dashboard_category = 'core_python'\n)\nselect\n job_name,\n job_yml_filename,\n run_1,\n run1Id,\n run_2,\n run2Id,\n run_3,\n run3Id,\n run_4,\n run4Id,\n run_5,\n run5Id,\n run_6,\n run6Id,\n run_7,\n run7Id,\n run_8,\n run8Id,\n run_9,\n run9Id,\n run_10,\n run10Id\nfrom\n workflows;", + "rawSql": "with workflows as (\n select\n *,\n case when run1 like 'success' then 1 when run1 like 'in_progress' then 2 when run1 like 'queued' then 2 when run1 like 'waiting' then 2 when run1 like 'cancelled' then 3 when run1 like 'None' then 4 else 0 end as run_1,\n case when run2 like 'success' then 1 when run2 like 'in_progress' then 2 when run2 like 'queued' then 2 when run2 like 'waiting' then 2 when run2 like 'cancelled' then 3 when run2 like 'None' then 4 else 0 end as run_2,\n case when run3 like 'success' then 1 when run3 like 'in_progress' then 2 when run3 like 'queued' then 2 when run3 like 'waiting' then 2 when run3 like 'cancelled' then 3 when run3 like 'None' then 4 else 0 end as run_3,\n case when run4 like 'success' then 1 when run4 like 'in_progress' then 2 when run4 like 'queued' then 2 when run4 like 'waiting' then 2 when run4 like 'cancelled' then 3 when run4 like 'None' then 4 else 0 end as run_4,\n case when run5 like 'success' then 1 when run5 like 'in_progress' then 2 when run5 like 'queued' then 2 when run5 like 'waiting' then 2 when run5 like 'cancelled' then 3 when run5 like 'None' then 4 else 0 end as run_5,\n case when run6 like 'success' then 1 when run6 like 'in_progress' then 2 when run6 like 'queued' then 2 when run6 like 'waiting' then 2 when run6 like 'cancelled' then 3 when run6 like 'None' then 4 else 0 end as run_6,\n case when run7 like 'success' then 1 when run7 like 'in_progress' then 2 when run7 like 'queued' then 2 when run7 like 'waiting' then 2 when run7 like 'cancelled' then 3 when run7 like 'None' then 4 else 0 end as run_7,\n case when run8 like 'success' then 1 when run8 like 'in_progress' then 2 when run8 like 'queued' then 2 when run8 like 'waiting' then 2 when run8 like 'cancelled' then 3 when run8 like 'None' then 4 else 0 end as run_8,\n case when run9 like 'success' then 1 when run9 like 'in_progress' then 2 when run9 like 'queued' then 2 when run9 like 'waiting' then 2 when run9 like 'cancelled' then 3 when run9 like 'None' then 4 else 0 end as run_9,\n case when run10 like 'success' then 1 when run10 like 'in_progress' then 2 when run10 like 'queued' then 2 when run10 like 'waiting' then 2 when run10 like 'cancelled' then 3 when run10 like 'None' then 4 else 0 end as run_10\n from\n github_workflows\n where\n dashboard_category = 'load_perf_java'\n)\nselect\n job_name,\n job_yml_filename,\n run_1,\n run1Id,\n run_2,\n run2Id,\n run_3,\n run3Id,\n run_4,\n run4Id,\n run_5,\n run5Id,\n run_6,\n run6Id,\n run_7,\n run7Id,\n run_8,\n run8Id,\n run_9,\n run9Id,\n run_10,\n run10Id\nfrom\n workflows;", "refId": "A", "select": [ [ @@ -4739,7 +4739,7 @@ ] } ], - "title": "Core Python Tests", + "title": "Java Load/Perf Tests", "transformations": [ { "id": "merge", @@ -5660,7 +5660,7 @@ "hide": false, "metricColumn": "none", "rawQuery": true, - "rawSql": "with workflows as (\n select\n *,\n case when run1 like 'success' then 1 when run1 like 'in_progress' then 2 when run1 like 'queued' then 2 when run1 like 'waiting' then 2 when run1 like 'cancelled' then 3 when run1 like 'None' then 4 else 0 end as run_1,\n case when run2 like 'success' then 1 when run2 like 'in_progress' then 2 when run2 like 'queued' then 2 when run2 like 'waiting' then 2 when run2 like 'cancelled' then 3 when run2 like 'None' then 4 else 0 end as run_2,\n case when run3 like 'success' then 1 when run3 like 'in_progress' then 2 when run3 like 'queued' then 2 when run3 like 'waiting' then 2 when run3 like 'cancelled' then 3 when run3 like 'None' then 4 else 0 end as run_3,\n case when run4 like 'success' then 1 when run4 like 'in_progress' then 2 when run4 like 'queued' then 2 when run4 like 'waiting' then 2 when run4 like 'cancelled' then 3 when run4 like 'None' then 4 else 0 end as run_4,\n case when run5 like 'success' then 1 when run5 like 'in_progress' then 2 when run5 like 'queued' then 2 when run5 like 'waiting' then 2 when run5 like 'cancelled' then 3 when run5 like 'None' then 4 else 0 end as run_5,\n case when run6 like 'success' then 1 when run6 like 'in_progress' then 2 when run6 like 'queued' then 2 when run6 like 'waiting' then 2 when run6 like 'cancelled' then 3 when run6 like 'None' then 4 else 0 end as run_6,\n case when run7 like 'success' then 1 when run7 like 'in_progress' then 2 when run7 like 'queued' then 2 when run7 like 'waiting' then 2 when run7 like 'cancelled' then 3 when run7 like 'None' then 4 else 0 end as run_7,\n case when run8 like 'success' then 1 when run8 like 'in_progress' then 2 when run8 like 'queued' then 2 when run8 like 'waiting' then 2 when run8 like 'cancelled' then 3 when run8 like 'None' then 4 else 0 end as run_8,\n case when run9 like 'success' then 1 when run9 like 'in_progress' then 2 when run9 like 'queued' then 2 when run9 like 'waiting' then 2 when run9 like 'cancelled' then 3 when run9 like 'None' then 4 else 0 end as run_9,\n case when run10 like 'success' then 1 when run10 like 'in_progress' then 2 when run10 like 'queued' then 2 when run10 like 'waiting' then 2 when run10 like 'cancelled' then 3 when run10 like 'None' then 4 else 0 end as run_10\n from\n github_workflows\n where\n dashboard_category = 'dataflow_python'\n)\nselect\n job_name,\n job_yml_filename,\n run_1,\n run1Id,\n run_2,\n run2Id,\n run_3,\n run3Id,\n run_4,\n run4Id,\n run_5,\n run5Id,\n run_6,\n run6Id,\n run_7,\n run7Id,\n run_8,\n run8Id,\n run_9,\n run9Id,\n run_10,\n run10Id\nfrom\n workflows;", + "rawSql": "with workflows as (\n select\n *,\n case when run1 like 'success' then 1 when run1 like 'in_progress' then 2 when run1 like 'queued' then 2 when run1 like 'waiting' then 2 when run1 like 'cancelled' then 3 when run1 like 'None' then 4 else 0 end as run_1,\n case when run2 like 'success' then 1 when run2 like 'in_progress' then 2 when run2 like 'queued' then 2 when run2 like 'waiting' then 2 when run2 like 'cancelled' then 3 when run2 like 'None' then 4 else 0 end as run_2,\n case when run3 like 'success' then 1 when run3 like 'in_progress' then 2 when run3 like 'queued' then 2 when run3 like 'waiting' then 2 when run3 like 'cancelled' then 3 when run3 like 'None' then 4 else 0 end as run_3,\n case when run4 like 'success' then 1 when run4 like 'in_progress' then 2 when run4 like 'queued' then 2 when run4 like 'waiting' then 2 when run4 like 'cancelled' then 3 when run4 like 'None' then 4 else 0 end as run_4,\n case when run5 like 'success' then 1 when run5 like 'in_progress' then 2 when run5 like 'queued' then 2 when run5 like 'waiting' then 2 when run5 like 'cancelled' then 3 when run5 like 'None' then 4 else 0 end as run_5,\n case when run6 like 'success' then 1 when run6 like 'in_progress' then 2 when run6 like 'queued' then 2 when run6 like 'waiting' then 2 when run6 like 'cancelled' then 3 when run6 like 'None' then 4 else 0 end as run_6,\n case when run7 like 'success' then 1 when run7 like 'in_progress' then 2 when run7 like 'queued' then 2 when run7 like 'waiting' then 2 when run7 like 'cancelled' then 3 when run7 like 'None' then 4 else 0 end as run_7,\n case when run8 like 'success' then 1 when run8 like 'in_progress' then 2 when run8 like 'queued' then 2 when run8 like 'waiting' then 2 when run8 like 'cancelled' then 3 when run8 like 'None' then 4 else 0 end as run_8,\n case when run9 like 'success' then 1 when run9 like 'in_progress' then 2 when run9 like 'queued' then 2 when run9 like 'waiting' then 2 when run9 like 'cancelled' then 3 when run9 like 'None' then 4 else 0 end as run_9,\n case when run10 like 'success' then 1 when run10 like 'in_progress' then 2 when run10 like 'queued' then 2 when run10 like 'waiting' then 2 when run10 like 'cancelled' then 3 when run10 like 'None' then 4 else 0 end as run_10\n from\n github_workflows\n where\n dashboard_category = 'core_python'\n)\nselect\n job_name,\n job_yml_filename,\n run_1,\n run1Id,\n run_2,\n run2Id,\n run_3,\n run3Id,\n run_4,\n run4Id,\n run_5,\n run5Id,\n run_6,\n run6Id,\n run_7,\n run7Id,\n run_8,\n run8Id,\n run_9,\n run9Id,\n run_10,\n run10Id\nfrom\n workflows;", "refId": "A", "select": [ [ @@ -5684,7 +5684,7 @@ ] } ], - "title": "Dataflow Python Tests", + "title": "Core Python Tests", "transformations": [ { "id": "merge", @@ -6605,7 +6605,7 @@ "hide": false, "metricColumn": "none", "rawQuery": true, - "rawSql": "with workflows as (\n select\n *,\n case when run1 like 'success' then 1 when run1 like 'in_progress' then 2 when run1 like 'queued' then 2 when run1 like 'waiting' then 2 when run1 like 'cancelled' then 3 when run1 like 'None' then 4 else 0 end as run_1,\n case when run2 like 'success' then 1 when run2 like 'in_progress' then 2 when run2 like 'queued' then 2 when run2 like 'waiting' then 2 when run2 like 'cancelled' then 3 when run2 like 'None' then 4 else 0 end as run_2,\n case when run3 like 'success' then 1 when run3 like 'in_progress' then 2 when run3 like 'queued' then 2 when run3 like 'waiting' then 2 when run3 like 'cancelled' then 3 when run3 like 'None' then 4 else 0 end as run_3,\n case when run4 like 'success' then 1 when run4 like 'in_progress' then 2 when run4 like 'queued' then 2 when run4 like 'waiting' then 2 when run4 like 'cancelled' then 3 when run4 like 'None' then 4 else 0 end as run_4,\n case when run5 like 'success' then 1 when run5 like 'in_progress' then 2 when run5 like 'queued' then 2 when run5 like 'waiting' then 2 when run5 like 'cancelled' then 3 when run5 like 'None' then 4 else 0 end as run_5,\n case when run6 like 'success' then 1 when run6 like 'in_progress' then 2 when run6 like 'queued' then 2 when run6 like 'waiting' then 2 when run6 like 'cancelled' then 3 when run6 like 'None' then 4 else 0 end as run_6,\n case when run7 like 'success' then 1 when run7 like 'in_progress' then 2 when run7 like 'queued' then 2 when run7 like 'waiting' then 2 when run7 like 'cancelled' then 3 when run7 like 'None' then 4 else 0 end as run_7,\n case when run8 like 'success' then 1 when run8 like 'in_progress' then 2 when run8 like 'queued' then 2 when run8 like 'waiting' then 2 when run8 like 'cancelled' then 3 when run8 like 'None' then 4 else 0 end as run_8,\n case when run9 like 'success' then 1 when run9 like 'in_progress' then 2 when run9 like 'queued' then 2 when run9 like 'waiting' then 2 when run9 like 'cancelled' then 3 when run9 like 'None' then 4 else 0 end as run_9,\n case when run10 like 'success' then 1 when run10 like 'in_progress' then 2 when run10 like 'queued' then 2 when run10 like 'waiting' then 2 when run10 like 'cancelled' then 3 when run10 like 'None' then 4 else 0 end as run_10\n from\n github_workflows\n where\n dashboard_category = 'spark_python'\n)\nselect\n job_name,\n job_yml_filename,\n run_1,\n run1Id,\n run_2,\n run2Id,\n run_3,\n run3Id,\n run_4,\n run4Id,\n run_5,\n run5Id,\n run_6,\n run6Id,\n run_7,\n run7Id,\n run_8,\n run8Id,\n run_9,\n run9Id,\n run_10,\n run10Id\nfrom\n workflows;", + "rawSql": "with workflows as (\n select\n *,\n case when run1 like 'success' then 1 when run1 like 'in_progress' then 2 when run1 like 'queued' then 2 when run1 like 'waiting' then 2 when run1 like 'cancelled' then 3 when run1 like 'None' then 4 else 0 end as run_1,\n case when run2 like 'success' then 1 when run2 like 'in_progress' then 2 when run2 like 'queued' then 2 when run2 like 'waiting' then 2 when run2 like 'cancelled' then 3 when run2 like 'None' then 4 else 0 end as run_2,\n case when run3 like 'success' then 1 when run3 like 'in_progress' then 2 when run3 like 'queued' then 2 when run3 like 'waiting' then 2 when run3 like 'cancelled' then 3 when run3 like 'None' then 4 else 0 end as run_3,\n case when run4 like 'success' then 1 when run4 like 'in_progress' then 2 when run4 like 'queued' then 2 when run4 like 'waiting' then 2 when run4 like 'cancelled' then 3 when run4 like 'None' then 4 else 0 end as run_4,\n case when run5 like 'success' then 1 when run5 like 'in_progress' then 2 when run5 like 'queued' then 2 when run5 like 'waiting' then 2 when run5 like 'cancelled' then 3 when run5 like 'None' then 4 else 0 end as run_5,\n case when run6 like 'success' then 1 when run6 like 'in_progress' then 2 when run6 like 'queued' then 2 when run6 like 'waiting' then 2 when run6 like 'cancelled' then 3 when run6 like 'None' then 4 else 0 end as run_6,\n case when run7 like 'success' then 1 when run7 like 'in_progress' then 2 when run7 like 'queued' then 2 when run7 like 'waiting' then 2 when run7 like 'cancelled' then 3 when run7 like 'None' then 4 else 0 end as run_7,\n case when run8 like 'success' then 1 when run8 like 'in_progress' then 2 when run8 like 'queued' then 2 when run8 like 'waiting' then 2 when run8 like 'cancelled' then 3 when run8 like 'None' then 4 else 0 end as run_8,\n case when run9 like 'success' then 1 when run9 like 'in_progress' then 2 when run9 like 'queued' then 2 when run9 like 'waiting' then 2 when run9 like 'cancelled' then 3 when run9 like 'None' then 4 else 0 end as run_9,\n case when run10 like 'success' then 1 when run10 like 'in_progress' then 2 when run10 like 'queued' then 2 when run10 like 'waiting' then 2 when run10 like 'cancelled' then 3 when run10 like 'None' then 4 else 0 end as run_10\n from\n github_workflows\n where\n dashboard_category = 'runners_python'\n)\nselect\n job_name,\n job_yml_filename,\n run_1,\n run1Id,\n run_2,\n run2Id,\n run_3,\n run3Id,\n run_4,\n run4Id,\n run_5,\n run5Id,\n run_6,\n run6Id,\n run_7,\n run7Id,\n run_8,\n run8Id,\n run_9,\n run9Id,\n run_10,\n run10Id\nfrom\n workflows;", "refId": "A", "select": [ [ @@ -6629,7 +6629,7 @@ ] } ], - "title": "Spark Python Tests", + "title": "Python Runners Tests", "transformations": [ { "id": "merge", @@ -7550,7 +7550,7 @@ "hide": false, "metricColumn": "none", "rawQuery": true, - "rawSql": "with workflows as (\n select\n *,\n case when run1 like 'success' then 1 when run1 like 'in_progress' then 2 when run1 like 'queued' then 2 when run1 like 'waiting' then 2 when run1 like 'cancelled' then 3 when run1 like 'None' then 4 else 0 end as run_1,\n case when run2 like 'success' then 1 when run2 like 'in_progress' then 2 when run2 like 'queued' then 2 when run2 like 'waiting' then 2 when run2 like 'cancelled' then 3 when run2 like 'None' then 4 else 0 end as run_2,\n case when run3 like 'success' then 1 when run3 like 'in_progress' then 2 when run3 like 'queued' then 2 when run3 like 'waiting' then 2 when run3 like 'cancelled' then 3 when run3 like 'None' then 4 else 0 end as run_3,\n case when run4 like 'success' then 1 when run4 like 'in_progress' then 2 when run4 like 'queued' then 2 when run4 like 'waiting' then 2 when run4 like 'cancelled' then 3 when run4 like 'None' then 4 else 0 end as run_4,\n case when run5 like 'success' then 1 when run5 like 'in_progress' then 2 when run5 like 'queued' then 2 when run5 like 'waiting' then 2 when run5 like 'cancelled' then 3 when run5 like 'None' then 4 else 0 end as run_5,\n case when run6 like 'success' then 1 when run6 like 'in_progress' then 2 when run6 like 'queued' then 2 when run6 like 'waiting' then 2 when run6 like 'cancelled' then 3 when run6 like 'None' then 4 else 0 end as run_6,\n case when run7 like 'success' then 1 when run7 like 'in_progress' then 2 when run7 like 'queued' then 2 when run7 like 'waiting' then 2 when run7 like 'cancelled' then 3 when run7 like 'None' then 4 else 0 end as run_7,\n case when run8 like 'success' then 1 when run8 like 'in_progress' then 2 when run8 like 'queued' then 2 when run8 like 'waiting' then 2 when run8 like 'cancelled' then 3 when run8 like 'None' then 4 else 0 end as run_8,\n case when run9 like 'success' then 1 when run9 like 'in_progress' then 2 when run9 like 'queued' then 2 when run9 like 'waiting' then 2 when run9 like 'cancelled' then 3 when run9 like 'None' then 4 else 0 end as run_9,\n case when run10 like 'success' then 1 when run10 like 'in_progress' then 2 when run10 like 'queued' then 2 when run10 like 'waiting' then 2 when run10 like 'cancelled' then 3 when run10 like 'None' then 4 else 0 end as run_10\n from\n github_workflows\n where\n dashboard_category = 'flink_python'\n)\nselect\n job_name,\n job_yml_filename,\n run_1,\n run1Id,\n run_2,\n run2Id,\n run_3,\n run3Id,\n run_4,\n run4Id,\n run_5,\n run5Id,\n run_6,\n run6Id,\n run_7,\n run7Id,\n run_8,\n run8Id,\n run_9,\n run9Id,\n run_10,\n run10Id\nfrom\n workflows;", + "rawSql": "with workflows as (\n select\n *,\n case when run1 like 'success' then 1 when run1 like 'in_progress' then 2 when run1 like 'queued' then 2 when run1 like 'waiting' then 2 when run1 like 'cancelled' then 3 when run1 like 'None' then 4 else 0 end as run_1,\n case when run2 like 'success' then 1 when run2 like 'in_progress' then 2 when run2 like 'queued' then 2 when run2 like 'waiting' then 2 when run2 like 'cancelled' then 3 when run2 like 'None' then 4 else 0 end as run_2,\n case when run3 like 'success' then 1 when run3 like 'in_progress' then 2 when run3 like 'queued' then 2 when run3 like 'waiting' then 2 when run3 like 'cancelled' then 3 when run3 like 'None' then 4 else 0 end as run_3,\n case when run4 like 'success' then 1 when run4 like 'in_progress' then 2 when run4 like 'queued' then 2 when run4 like 'waiting' then 2 when run4 like 'cancelled' then 3 when run4 like 'None' then 4 else 0 end as run_4,\n case when run5 like 'success' then 1 when run5 like 'in_progress' then 2 when run5 like 'queued' then 2 when run5 like 'waiting' then 2 when run5 like 'cancelled' then 3 when run5 like 'None' then 4 else 0 end as run_5,\n case when run6 like 'success' then 1 when run6 like 'in_progress' then 2 when run6 like 'queued' then 2 when run6 like 'waiting' then 2 when run6 like 'cancelled' then 3 when run6 like 'None' then 4 else 0 end as run_6,\n case when run7 like 'success' then 1 when run7 like 'in_progress' then 2 when run7 like 'queued' then 2 when run7 like 'waiting' then 2 when run7 like 'cancelled' then 3 when run7 like 'None' then 4 else 0 end as run_7,\n case when run8 like 'success' then 1 when run8 like 'in_progress' then 2 when run8 like 'queued' then 2 when run8 like 'waiting' then 2 when run8 like 'cancelled' then 3 when run8 like 'None' then 4 else 0 end as run_8,\n case when run9 like 'success' then 1 when run9 like 'in_progress' then 2 when run9 like 'queued' then 2 when run9 like 'waiting' then 2 when run9 like 'cancelled' then 3 when run9 like 'None' then 4 else 0 end as run_9,\n case when run10 like 'success' then 1 when run10 like 'in_progress' then 2 when run10 like 'queued' then 2 when run10 like 'waiting' then 2 when run10 like 'cancelled' then 3 when run10 like 'None' then 4 else 0 end as run_10\n from\n github_workflows\n where\n dashboard_category = 'load_perf_python'\n)\nselect\n job_name,\n job_yml_filename,\n run_1,\n run1Id,\n run_2,\n run2Id,\n run_3,\n run3Id,\n run_4,\n run4Id,\n run_5,\n run5Id,\n run_6,\n run6Id,\n run_7,\n run7Id,\n run_8,\n run8Id,\n run_9,\n run9Id,\n run_10,\n run10Id\nfrom\n workflows;", "refId": "A", "select": [ [ @@ -7574,7 +7574,7 @@ ] } ], - "title": "Flink Python Tests", + "title": "Python Load/Perf Tests", "transformations": [ { "id": "merge", @@ -8495,7 +8495,7 @@ "hide": false, "metricColumn": "none", "rawQuery": true, - "rawSql": "with workflows as (\n select\n *,\n case when run1 like 'success' then 1 when run1 like 'in_progress' then 2 when run1 like 'queued' then 2 when run1 like 'waiting' then 2 when run1 like 'cancelled' then 3 when run1 like 'None' then 4 else 0 end as run_1,\n case when run2 like 'success' then 1 when run2 like 'in_progress' then 2 when run2 like 'queued' then 2 when run2 like 'waiting' then 2 when run2 like 'cancelled' then 3 when run2 like 'None' then 4 else 0 end as run_2,\n case when run3 like 'success' then 1 when run3 like 'in_progress' then 2 when run3 like 'queued' then 2 when run3 like 'waiting' then 2 when run3 like 'cancelled' then 3 when run3 like 'None' then 4 else 0 end as run_3,\n case when run4 like 'success' then 1 when run4 like 'in_progress' then 2 when run4 like 'queued' then 2 when run4 like 'waiting' then 2 when run4 like 'cancelled' then 3 when run4 like 'None' then 4 else 0 end as run_4,\n case when run5 like 'success' then 1 when run5 like 'in_progress' then 2 when run5 like 'queued' then 2 when run5 like 'waiting' then 2 when run5 like 'cancelled' then 3 when run5 like 'None' then 4 else 0 end as run_5,\n case when run6 like 'success' then 1 when run6 like 'in_progress' then 2 when run6 like 'queued' then 2 when run6 like 'waiting' then 2 when run6 like 'cancelled' then 3 when run6 like 'None' then 4 else 0 end as run_6,\n case when run7 like 'success' then 1 when run7 like 'in_progress' then 2 when run7 like 'queued' then 2 when run7 like 'waiting' then 2 when run7 like 'cancelled' then 3 when run7 like 'None' then 4 else 0 end as run_7,\n case when run8 like 'success' then 1 when run8 like 'in_progress' then 2 when run8 like 'queued' then 2 when run8 like 'waiting' then 2 when run8 like 'cancelled' then 3 when run8 like 'None' then 4 else 0 end as run_8,\n case when run9 like 'success' then 1 when run9 like 'in_progress' then 2 when run9 like 'queued' then 2 when run9 like 'waiting' then 2 when run9 like 'cancelled' then 3 when run9 like 'None' then 4 else 0 end as run_9,\n case when run10 like 'success' then 1 when run10 like 'in_progress' then 2 when run10 like 'queued' then 2 when run10 like 'waiting' then 2 when run10 like 'cancelled' then 3 when run10 like 'None' then 4 else 0 end as run_10\n from\n github_workflows\n where\n dashboard_category = 'core_go'\n)\nselect\n job_name,\n job_yml_filename,\n run_1,\n run1Id,\n run_2,\n run2Id,\n run_3,\n run3Id,\n run_4,\n run4Id,\n run_5,\n run5Id,\n run_6,\n run6Id,\n run_7,\n run7Id,\n run_8,\n run8Id,\n run_9,\n run9Id,\n run_10,\n run10Id\nfrom\n workflows;", + "rawSql": "with workflows as (\n select\n *,\n case when run1 like 'success' then 1 when run1 like 'in_progress' then 2 when run1 like 'queued' then 2 when run1 like 'waiting' then 2 when run1 like 'cancelled' then 3 when run1 like 'None' then 4 else 0 end as run_1,\n case when run2 like 'success' then 1 when run2 like 'in_progress' then 2 when run2 like 'queued' then 2 when run2 like 'waiting' then 2 when run2 like 'cancelled' then 3 when run2 like 'None' then 4 else 0 end as run_2,\n case when run3 like 'success' then 1 when run3 like 'in_progress' then 2 when run3 like 'queued' then 2 when run3 like 'waiting' then 2 when run3 like 'cancelled' then 3 when run3 like 'None' then 4 else 0 end as run_3,\n case when run4 like 'success' then 1 when run4 like 'in_progress' then 2 when run4 like 'queued' then 2 when run4 like 'waiting' then 2 when run4 like 'cancelled' then 3 when run4 like 'None' then 4 else 0 end as run_4,\n case when run5 like 'success' then 1 when run5 like 'in_progress' then 2 when run5 like 'queued' then 2 when run5 like 'waiting' then 2 when run5 like 'cancelled' then 3 when run5 like 'None' then 4 else 0 end as run_5,\n case when run6 like 'success' then 1 when run6 like 'in_progress' then 2 when run6 like 'queued' then 2 when run6 like 'waiting' then 2 when run6 like 'cancelled' then 3 when run6 like 'None' then 4 else 0 end as run_6,\n case when run7 like 'success' then 1 when run7 like 'in_progress' then 2 when run7 like 'queued' then 2 when run7 like 'waiting' then 2 when run7 like 'cancelled' then 3 when run7 like 'None' then 4 else 0 end as run_7,\n case when run8 like 'success' then 1 when run8 like 'in_progress' then 2 when run8 like 'queued' then 2 when run8 like 'waiting' then 2 when run8 like 'cancelled' then 3 when run8 like 'None' then 4 else 0 end as run_8,\n case when run9 like 'success' then 1 when run9 like 'in_progress' then 2 when run9 like 'queued' then 2 when run9 like 'waiting' then 2 when run9 like 'cancelled' then 3 when run9 like 'None' then 4 else 0 end as run_9,\n case when run10 like 'success' then 1 when run10 like 'in_progress' then 2 when run10 like 'queued' then 2 when run10 like 'waiting' then 2 when run10 like 'cancelled' then 3 when run10 like 'None' then 4 else 0 end as run_10\n from\n github_workflows\n where\n dashboard_category = 'go'\n)\nselect\n job_name,\n job_yml_filename,\n run_1,\n run1Id,\n run_2,\n run2Id,\n run_3,\n run3Id,\n run_4,\n run4Id,\n run_5,\n run5Id,\n run_6,\n run6Id,\n run_7,\n run7Id,\n run_8,\n run8Id,\n run_9,\n run9Id,\n run_10,\n run10Id\nfrom\n workflows;", "refId": "A", "select": [ [ @@ -8519,7 +8519,7 @@ ] } ], - "title": "Core Go Tests", + "title": "Go Tests", "transformations": [ { "id": "merge", @@ -9426,2841 +9426,6 @@ "showHeader": true }, "pluginVersion": "10.0.3", - "targets": [ - { - "aggregation": "Last", - "alias": "job", - "decimals": 2, - "displayAliasType": "Warning / Critical", - "displayType": "Regular", - "displayValueWithAlias": "Never", - "editorMode": "code", - "format": "table", - "group": [], - "hide": false, - "metricColumn": "none", - "rawQuery": true, - "rawSql": "with workflows as (\n select\n *,\n case when run1 like 'success' then 1 when run1 like 'in_progress' then 2 when run1 like 'queued' then 2 when run1 like 'waiting' then 2 when run1 like 'cancelled' then 3 when run1 like 'None' then 4 else 0 end as run_1,\n case when run2 like 'success' then 1 when run2 like 'in_progress' then 2 when run2 like 'queued' then 2 when run2 like 'waiting' then 2 when run2 like 'cancelled' then 3 when run2 like 'None' then 4 else 0 end as run_2,\n case when run3 like 'success' then 1 when run3 like 'in_progress' then 2 when run3 like 'queued' then 2 when run3 like 'waiting' then 2 when run3 like 'cancelled' then 3 when run3 like 'None' then 4 else 0 end as run_3,\n case when run4 like 'success' then 1 when run4 like 'in_progress' then 2 when run4 like 'queued' then 2 when run4 like 'waiting' then 2 when run4 like 'cancelled' then 3 when run4 like 'None' then 4 else 0 end as run_4,\n case when run5 like 'success' then 1 when run5 like 'in_progress' then 2 when run5 like 'queued' then 2 when run5 like 'waiting' then 2 when run5 like 'cancelled' then 3 when run5 like 'None' then 4 else 0 end as run_5,\n case when run6 like 'success' then 1 when run6 like 'in_progress' then 2 when run6 like 'queued' then 2 when run6 like 'waiting' then 2 when run6 like 'cancelled' then 3 when run6 like 'None' then 4 else 0 end as run_6,\n case when run7 like 'success' then 1 when run7 like 'in_progress' then 2 when run7 like 'queued' then 2 when run7 like 'waiting' then 2 when run7 like 'cancelled' then 3 when run7 like 'None' then 4 else 0 end as run_7,\n case when run8 like 'success' then 1 when run8 like 'in_progress' then 2 when run8 like 'queued' then 2 when run8 like 'waiting' then 2 when run8 like 'cancelled' then 3 when run8 like 'None' then 4 else 0 end as run_8,\n case when run9 like 'success' then 1 when run9 like 'in_progress' then 2 when run9 like 'queued' then 2 when run9 like 'waiting' then 2 when run9 like 'cancelled' then 3 when run9 like 'None' then 4 else 0 end as run_9,\n case when run10 like 'success' then 1 when run10 like 'in_progress' then 2 when run10 like 'queued' then 2 when run10 like 'waiting' then 2 when run10 like 'cancelled' then 3 when run10 like 'None' then 4 else 0 end as run_10\n from\n github_workflows\n where\n dashboard_category = 'dataflow_go'\n)\nselect\n job_name,\n job_yml_filename,\n run_1,\n run1Id,\n run_2,\n run2Id,\n run_3,\n run3Id,\n run_4,\n run4Id,\n run_5,\n run5Id,\n run_6,\n run6Id,\n run_7,\n run7Id,\n run_8,\n run8Id,\n run_9,\n run9Id,\n run_10,\n run10Id\nfrom\n workflows;", - "refId": "A", - "select": [ - [ - { - "params": [ - "value" - ], - "type": "column" - } - ] - ], - "timeColumn": "time", - "units": "none", - "valueHandler": "Number Threshold", - "where": [ - { - "name": "$__timeFilter", - "params": [], - "type": "macro" - } - ] - } - ], - "title": "Dataflow Go Tests", - "transformations": [ - { - "id": "merge", - "options": { - "reducers": [] - } - } - ], - "type": "table" - }, - { - "datasource": "BeamPSQL", - "fieldConfig": { - "defaults": { - "color": { - "mode": "thresholds" - }, - "custom": { - "align": "auto", - "cellOptions": { - "type": "auto" - }, - "inspect": false - }, - "mappings": [ - { - "options": { - "0": { - "color": "semi-dark-red", - "index": 0, - "text": "Fail" - }, - "1": { - "color": "semi-dark-green", - "index": 1, - "text": "Success" - }, - "2": { - "color": "semi-dark-yellow", - "index": 2, - "text": "Pending" - }, - "3": { - "color": "semi-dark-purple", - "index": 3, - "text": "Cancelled" - }, - "4": { - "color": "light-blue", - "index": 4, - "text": "None" - } - }, - "type": "value" - } - ], - "thresholds": { - "mode": "absolute", - "steps": [ - { - "color": "semi-dark-red", - "value": null - } - ] - } - }, - "overrides": [ - { - "matcher": { - "id": "byName", - "options": "job_name" - }, - "properties": [ - { - "id": "unit", - "value": "short" - }, - { - "id": "decimals", - "value": 2 - }, - { - "id": "links", - "value": [ - { - "targetBlank": true, - "title": "", - "url": "https://github.com/apache/beam/actions/${__data.fields.job_yml_filename}" - } - ] - }, - { - "id": "custom.align", - "value": "auto" - }, - { - "id": "mappings", - "value": [] - }, - { - "id": "displayName", - "value": "Job Name" - } - ] - }, - { - "matcher": { - "id": "byName", - "options": "job_yml_filename" - }, - "properties": [ - { - "id": "custom.hidden", - "value": true - } - ] - }, - { - "matcher": { - "id": "byRegexp", - "options": "/run\\d+id/" - }, - "properties": [ - { - "id": "custom.hidden", - "value": true - }, - { - "id": "custom.align", - "value": "center" - } - ] - }, - { - "matcher": { - "id": "byName", - "options": "run_1" - }, - "properties": [ - { - "id": "color", - "value": { - "fixedColor": "dark-green", - "mode": "fixed" - } - }, - { - "id": "custom.hidden", - "value": false - }, - { - "id": "thresholds", - "value": { - "mode": "percentage", - "steps": [ - { - "color": "semi-dark-red", - "value": null - }, - { - "color": "semi-dark-red", - "value": 0 - }, - { - "color": "semi-dark-green", - "value": 1 - }, - { - "color": "semi-dark-yellow", - "value": 2 - }, - { - "color": "light-blue", - "value": 3 - }, - { - "color": "purple", - "value": 4 - } - ] - } - }, - { - "id": "custom.cellOptions", - "value": { - "mode": "basic", - "type": "color-background" - } - }, - { - "id": "custom.align", - "value": "center" - }, - { - "id": "links", - "value": [ - { - "title": "", - "url": "${__data.fields.run1id}" - } - ] - }, - { - "id": "displayName", - "value": "Run" - } - ] - }, - { - "matcher": { - "id": "byName", - "options": "run_2" - }, - "properties": [ - { - "id": "color", - "value": { - "fixedColor": "dark-green", - "mode": "fixed" - } - }, - { - "id": "custom.hidden", - "value": false - }, - { - "id": "thresholds", - "value": { - "mode": "percentage", - "steps": [ - { - "color": "semi-dark-red", - "value": null - }, - { - "color": "semi-dark-red", - "value": 0 - }, - { - "color": "semi-dark-green", - "value": 1 - }, - { - "color": "semi-dark-yellow", - "value": 2 - }, - { - "color": "light-blue", - "value": 3 - }, - { - "color": "purple", - "value": 4 - } - ] - } - }, - { - "id": "custom.cellOptions", - "value": { - "mode": "basic", - "type": "color-background" - } - }, - { - "id": "custom.align", - "value": "center" - }, - { - "id": "links", - "value": [ - { - "title": "", - "url": "${__data.fields.run2id}" - } - ] - }, - { - "id": "displayName", - "value": "Run 2" - } - ] - }, - { - "matcher": { - "id": "byName", - "options": "run_3" - }, - "properties": [ - { - "id": "color", - "value": { - "fixedColor": "dark-green", - "mode": "fixed" - } - }, - { - "id": "custom.hidden", - "value": false - }, - { - "id": "thresholds", - "value": { - "mode": "percentage", - "steps": [ - { - "color": "semi-dark-red", - "value": null - }, - { - "color": "semi-dark-red", - "value": 0 - }, - { - "color": "semi-dark-green", - "value": 1 - }, - { - "color": "semi-dark-yellow", - "value": 2 - }, - { - "color": "light-blue", - "value": 3 - }, - { - "color": "purple", - "value": 4 - } - ] - } - }, - { - "id": "custom.cellOptions", - "value": { - "mode": "basic", - "type": "color-background" - } - }, - { - "id": "custom.align", - "value": "center" - }, - { - "id": "links", - "value": [ - { - "title": "", - "url": "${__data.fields.run3id}" - } - ] - }, - { - "id": "displayName", - "value": "Run 3" - } - ] - }, - { - "matcher": { - "id": "byName", - "options": "run_4" - }, - "properties": [ - { - "id": "color", - "value": { - "fixedColor": "dark-green", - "mode": "fixed" - } - }, - { - "id": "custom.hidden", - "value": false - }, - { - "id": "thresholds", - "value": { - "mode": "percentage", - "steps": [ - { - "color": "semi-dark-red", - "value": null - }, - { - "color": "semi-dark-red", - "value": 0 - }, - { - "color": "semi-dark-green", - "value": 1 - }, - { - "color": "semi-dark-yellow", - "value": 2 - }, - { - "color": "light-blue", - "value": 3 - }, - { - "color": "purple", - "value": 4 - } - ] - } - }, - { - "id": "custom.cellOptions", - "value": { - "mode": "basic", - "type": "color-background" - } - }, - { - "id": "custom.align", - "value": "center" - }, - { - "id": "links", - "value": [ - { - "title": "", - "url": "${__data.fields.run4id}" - } - ] - }, - { - "id": "displayName", - "value": "Run 4" - } - ] - }, - { - "matcher": { - "id": "byName", - "options": "run_5" - }, - "properties": [ - { - "id": "color", - "value": { - "fixedColor": "dark-green", - "mode": "fixed" - } - }, - { - "id": "custom.hidden", - "value": false - }, - { - "id": "thresholds", - "value": { - "mode": "percentage", - "steps": [ - { - "color": "semi-dark-red", - "value": null - }, - { - "color": "semi-dark-red", - "value": 0 - }, - { - "color": "semi-dark-green", - "value": 1 - }, - { - "color": "semi-dark-yellow", - "value": 2 - }, - { - "color": "light-blue", - "value": 3 - }, - { - "color": "purple", - "value": 4 - } - ] - } - }, - { - "id": "custom.cellOptions", - "value": { - "mode": "basic", - "type": "color-background" - } - }, - { - "id": "custom.align", - "value": "center" - }, - { - "id": "links", - "value": [ - { - "title": "", - "url": "${__data.fields.run5id}" - } - ] - }, - { - "id": "displayName", - "value": "Run 5" - } - ] - }, - { - "matcher": { - "id": "byName", - "options": "run_6" - }, - "properties": [ - { - "id": "color", - "value": { - "fixedColor": "dark-green", - "mode": "fixed" - } - }, - { - "id": "custom.hidden", - "value": false - }, - { - "id": "thresholds", - "value": { - "mode": "percentage", - "steps": [ - { - "color": "semi-dark-red", - "value": null - }, - { - "color": "semi-dark-red", - "value": 0 - }, - { - "color": "semi-dark-green", - "value": 1 - }, - { - "color": "semi-dark-yellow", - "value": 2 - }, - { - "color": "light-blue", - "value": 3 - }, - { - "color": "purple", - "value": 4 - } - ] - } - }, - { - "id": "custom.cellOptions", - "value": { - "mode": "basic", - "type": "color-background" - } - }, - { - "id": "custom.align", - "value": "center" - }, - { - "id": "links", - "value": [ - { - "title": "", - "url": "${__data.fields.run6id}" - } - ] - }, - { - "id": "displayName", - "value": "Run 6" - } - ] - }, - { - "matcher": { - "id": "byName", - "options": "run_7" - }, - "properties": [ - { - "id": "color", - "value": { - "fixedColor": "dark-green", - "mode": "fixed" - } - }, - { - "id": "custom.hidden", - "value": false - }, - { - "id": "thresholds", - "value": { - "mode": "percentage", - "steps": [ - { - "color": "semi-dark-red", - "value": null - }, - { - "color": "semi-dark-red", - "value": 0 - }, - { - "color": "semi-dark-green", - "value": 1 - }, - { - "color": "semi-dark-yellow", - "value": 2 - }, - { - "color": "light-blue", - "value": 3 - }, - { - "color": "purple", - "value": 4 - } - ] - } - }, - { - "id": "custom.cellOptions", - "value": { - "mode": "basic", - "type": "color-background" - } - }, - { - "id": "custom.align", - "value": "center" - }, - { - "id": "links", - "value": [ - { - "title": "", - "url": "${__data.fields.run7id}" - } - ] - }, - { - "id": "displayName", - "value": "Run 7" - } - ] - }, - { - "matcher": { - "id": "byName", - "options": "run_8" - }, - "properties": [ - { - "id": "color", - "value": { - "fixedColor": "dark-green", - "mode": "fixed" - } - }, - { - "id": "custom.hidden", - "value": false - }, - { - "id": "thresholds", - "value": { - "mode": "percentage", - "steps": [ - { - "color": "semi-dark-red", - "value": null - }, - { - "color": "semi-dark-red", - "value": 0 - }, - { - "color": "semi-dark-green", - "value": 1 - }, - { - "color": "semi-dark-yellow", - "value": 2 - }, - { - "color": "light-blue", - "value": 3 - }, - { - "color": "purple", - "value": 4 - } - ] - } - }, - { - "id": "custom.cellOptions", - "value": { - "mode": "basic", - "type": "color-background" - } - }, - { - "id": "custom.align", - "value": "center" - }, - { - "id": "links", - "value": [ - { - "title": "", - "url": "${__data.fields.run8id}" - } - ] - }, - { - "id": "displayName", - "value": "Run 8" - } - ] - }, - { - "matcher": { - "id": "byName", - "options": "run_9" - }, - "properties": [ - { - "id": "color", - "value": { - "fixedColor": "dark-green", - "mode": "fixed" - } - }, - { - "id": "custom.hidden", - "value": false - }, - { - "id": "thresholds", - "value": { - "mode": "percentage", - "steps": [ - { - "color": "semi-dark-red", - "value": null - }, - { - "color": "semi-dark-red", - "value": 0 - }, - { - "color": "semi-dark-green", - "value": 1 - }, - { - "color": "semi-dark-yellow", - "value": 2 - }, - { - "color": "light-blue", - "value": 3 - }, - { - "color": "purple", - "value": 4 - } - ] - } - }, - { - "id": "custom.cellOptions", - "value": { - "mode": "basic", - "type": "color-background" - } - }, - { - "id": "custom.align", - "value": "center" - }, - { - "id": "links", - "value": [ - { - "title": "", - "url": "${__data.fields.run9id}" - } - ] - }, - { - "id": "displayName", - "value": "Run 9" - } - ] - }, - { - "matcher": { - "id": "byName", - "options": "run_10" - }, - "properties": [ - { - "id": "color", - "value": { - "fixedColor": "dark-green", - "mode": "fixed" - } - }, - { - "id": "custom.hidden", - "value": false - }, - { - "id": "thresholds", - "value": { - "mode": "percentage", - "steps": [ - { - "color": "semi-dark-red", - "value": null - }, - { - "color": "semi-dark-red", - "value": 0 - }, - { - "color": "semi-dark-green", - "value": 1 - }, - { - "color": "semi-dark-yellow", - "value": 2 - }, - { - "color": "light-blue", - "value": 3 - }, - { - "color": "purple", - "value": 4 - } - ] - } - }, - { - "id": "custom.cellOptions", - "value": { - "mode": "basic", - "type": "color-background" - } - }, - { - "id": "custom.align", - "value": "center" - }, - { - "id": "links", - "value": [ - { - "title": "", - "url": "${__data.fields.run10id}" - } - ] - }, - { - "id": "displayName", - "value": "Run 10" - } - ] - } - ] - }, - "gridPos": { - "h": 20, - "w": 24, - "x": 0, - "y": 200 - }, - "id": 5, - "links": [], - "options": { - "cellHeight": "sm", - "footer": { - "countRows": false, - "fields": "", - "reducer": [ - "sum" - ], - "show": false - }, - "showHeader": true - }, - "pluginVersion": "10.0.3", - "targets": [ - { - "aggregation": "Last", - "alias": "job", - "decimals": 2, - "displayAliasType": "Warning / Critical", - "displayType": "Regular", - "displayValueWithAlias": "Never", - "editorMode": "code", - "format": "table", - "group": [], - "hide": false, - "metricColumn": "none", - "rawQuery": true, - "rawSql": "with workflows as (\n select\n *,\n case when run1 like 'success' then 1 when run1 like 'in_progress' then 2 when run1 like 'queued' then 2 when run1 like 'waiting' then 2 when run1 like 'cancelled' then 3 when run1 like 'None' then 4 else 0 end as run_1,\n case when run2 like 'success' then 1 when run2 like 'in_progress' then 2 when run2 like 'queued' then 2 when run2 like 'waiting' then 2 when run2 like 'cancelled' then 3 when run2 like 'None' then 4 else 0 end as run_2,\n case when run3 like 'success' then 1 when run3 like 'in_progress' then 2 when run3 like 'queued' then 2 when run3 like 'waiting' then 2 when run3 like 'cancelled' then 3 when run3 like 'None' then 4 else 0 end as run_3,\n case when run4 like 'success' then 1 when run4 like 'in_progress' then 2 when run4 like 'queued' then 2 when run4 like 'waiting' then 2 when run4 like 'cancelled' then 3 when run4 like 'None' then 4 else 0 end as run_4,\n case when run5 like 'success' then 1 when run5 like 'in_progress' then 2 when run5 like 'queued' then 2 when run5 like 'waiting' then 2 when run5 like 'cancelled' then 3 when run5 like 'None' then 4 else 0 end as run_5,\n case when run6 like 'success' then 1 when run6 like 'in_progress' then 2 when run6 like 'queued' then 2 when run6 like 'waiting' then 2 when run6 like 'cancelled' then 3 when run6 like 'None' then 4 else 0 end as run_6,\n case when run7 like 'success' then 1 when run7 like 'in_progress' then 2 when run7 like 'queued' then 2 when run7 like 'waiting' then 2 when run7 like 'cancelled' then 3 when run7 like 'None' then 4 else 0 end as run_7,\n case when run8 like 'success' then 1 when run8 like 'in_progress' then 2 when run8 like 'queued' then 2 when run8 like 'waiting' then 2 when run8 like 'cancelled' then 3 when run8 like 'None' then 4 else 0 end as run_8,\n case when run9 like 'success' then 1 when run9 like 'in_progress' then 2 when run9 like 'queued' then 2 when run9 like 'waiting' then 2 when run9 like 'cancelled' then 3 when run9 like 'None' then 4 else 0 end as run_9,\n case when run10 like 'success' then 1 when run10 like 'in_progress' then 2 when run10 like 'queued' then 2 when run10 like 'waiting' then 2 when run10 like 'cancelled' then 3 when run10 like 'None' then 4 else 0 end as run_10\n from\n github_workflows\n where\n dashboard_category = 'spark_go'\n)\nselect\n job_name,\n job_yml_filename,\n run_1,\n run1Id,\n run_2,\n run2Id,\n run_3,\n run3Id,\n run_4,\n run4Id,\n run_5,\n run5Id,\n run_6,\n run6Id,\n run_7,\n run7Id,\n run_8,\n run8Id,\n run_9,\n run9Id,\n run_10,\n run10Id\nfrom\n workflows;", - "refId": "A", - "select": [ - [ - { - "params": [ - "value" - ], - "type": "column" - } - ] - ], - "timeColumn": "time", - "units": "none", - "valueHandler": "Number Threshold", - "where": [ - { - "name": "$__timeFilter", - "params": [], - "type": "macro" - } - ] - } - ], - "title": "Spark Go Tests", - "transformations": [ - { - "id": "merge", - "options": { - "reducers": [] - } - } - ], - "type": "table" - }, - { - "datasource": "BeamPSQL", - "fieldConfig": { - "defaults": { - "color": { - "mode": "thresholds" - }, - "custom": { - "align": "auto", - "cellOptions": { - "type": "auto" - }, - "inspect": false - }, - "mappings": [ - { - "options": { - "0": { - "color": "semi-dark-red", - "index": 0, - "text": "Fail" - }, - "1": { - "color": "semi-dark-green", - "index": 1, - "text": "Success" - }, - "2": { - "color": "semi-dark-yellow", - "index": 2, - "text": "Pending" - }, - "3": { - "color": "semi-dark-purple", - "index": 3, - "text": "Cancelled" - }, - "4": { - "color": "light-blue", - "index": 4, - "text": "None" - } - }, - "type": "value" - } - ], - "thresholds": { - "mode": "absolute", - "steps": [ - { - "color": "semi-dark-red", - "value": null - } - ] - } - }, - "overrides": [ - { - "matcher": { - "id": "byName", - "options": "job_name" - }, - "properties": [ - { - "id": "unit", - "value": "short" - }, - { - "id": "decimals", - "value": 2 - }, - { - "id": "links", - "value": [ - { - "targetBlank": true, - "title": "", - "url": "https://github.com/apache/beam/actions/${__data.fields.job_yml_filename}" - } - ] - }, - { - "id": "custom.align", - "value": "auto" - }, - { - "id": "mappings", - "value": [] - }, - { - "id": "displayName", - "value": "Job Name" - } - ] - }, - { - "matcher": { - "id": "byName", - "options": "job_yml_filename" - }, - "properties": [ - { - "id": "custom.hidden", - "value": true - } - ] - }, - { - "matcher": { - "id": "byRegexp", - "options": "/run\\d+id/" - }, - "properties": [ - { - "id": "custom.hidden", - "value": true - }, - { - "id": "custom.align", - "value": "center" - } - ] - }, - { - "matcher": { - "id": "byName", - "options": "run_1" - }, - "properties": [ - { - "id": "color", - "value": { - "fixedColor": "dark-green", - "mode": "fixed" - } - }, - { - "id": "custom.hidden", - "value": false - }, - { - "id": "thresholds", - "value": { - "mode": "percentage", - "steps": [ - { - "color": "semi-dark-red", - "value": null - }, - { - "color": "semi-dark-red", - "value": 0 - }, - { - "color": "semi-dark-green", - "value": 1 - }, - { - "color": "semi-dark-yellow", - "value": 2 - }, - { - "color": "light-blue", - "value": 3 - }, - { - "color": "purple", - "value": 4 - } - ] - } - }, - { - "id": "custom.cellOptions", - "value": { - "mode": "basic", - "type": "color-background" - } - }, - { - "id": "custom.align", - "value": "center" - }, - { - "id": "links", - "value": [ - { - "title": "", - "url": "${__data.fields.run1id}" - } - ] - }, - { - "id": "displayName", - "value": "Run" - } - ] - }, - { - "matcher": { - "id": "byName", - "options": "run_2" - }, - "properties": [ - { - "id": "color", - "value": { - "fixedColor": "dark-green", - "mode": "fixed" - } - }, - { - "id": "custom.hidden", - "value": false - }, - { - "id": "thresholds", - "value": { - "mode": "percentage", - "steps": [ - { - "color": "semi-dark-red", - "value": null - }, - { - "color": "semi-dark-red", - "value": 0 - }, - { - "color": "semi-dark-green", - "value": 1 - }, - { - "color": "semi-dark-yellow", - "value": 2 - }, - { - "color": "light-blue", - "value": 3 - }, - { - "color": "purple", - "value": 4 - } - ] - } - }, - { - "id": "custom.cellOptions", - "value": { - "mode": "basic", - "type": "color-background" - } - }, - { - "id": "custom.align", - "value": "center" - }, - { - "id": "links", - "value": [ - { - "title": "", - "url": "${__data.fields.run2id}" - } - ] - }, - { - "id": "displayName", - "value": "Run 2" - } - ] - }, - { - "matcher": { - "id": "byName", - "options": "run_3" - }, - "properties": [ - { - "id": "color", - "value": { - "fixedColor": "dark-green", - "mode": "fixed" - } - }, - { - "id": "custom.hidden", - "value": false - }, - { - "id": "thresholds", - "value": { - "mode": "percentage", - "steps": [ - { - "color": "semi-dark-red", - "value": null - }, - { - "color": "semi-dark-red", - "value": 0 - }, - { - "color": "semi-dark-green", - "value": 1 - }, - { - "color": "semi-dark-yellow", - "value": 2 - }, - { - "color": "light-blue", - "value": 3 - }, - { - "color": "purple", - "value": 4 - } - ] - } - }, - { - "id": "custom.cellOptions", - "value": { - "mode": "basic", - "type": "color-background" - } - }, - { - "id": "custom.align", - "value": "center" - }, - { - "id": "links", - "value": [ - { - "title": "", - "url": "${__data.fields.run3id}" - } - ] - }, - { - "id": "displayName", - "value": "Run 3" - } - ] - }, - { - "matcher": { - "id": "byName", - "options": "run_4" - }, - "properties": [ - { - "id": "color", - "value": { - "fixedColor": "dark-green", - "mode": "fixed" - } - }, - { - "id": "custom.hidden", - "value": false - }, - { - "id": "thresholds", - "value": { - "mode": "percentage", - "steps": [ - { - "color": "semi-dark-red", - "value": null - }, - { - "color": "semi-dark-red", - "value": 0 - }, - { - "color": "semi-dark-green", - "value": 1 - }, - { - "color": "semi-dark-yellow", - "value": 2 - }, - { - "color": "light-blue", - "value": 3 - }, - { - "color": "purple", - "value": 4 - } - ] - } - }, - { - "id": "custom.cellOptions", - "value": { - "mode": "basic", - "type": "color-background" - } - }, - { - "id": "custom.align", - "value": "center" - }, - { - "id": "links", - "value": [ - { - "title": "", - "url": "${__data.fields.run4id}" - } - ] - }, - { - "id": "displayName", - "value": "Run 4" - } - ] - }, - { - "matcher": { - "id": "byName", - "options": "run_5" - }, - "properties": [ - { - "id": "color", - "value": { - "fixedColor": "dark-green", - "mode": "fixed" - } - }, - { - "id": "custom.hidden", - "value": false - }, - { - "id": "thresholds", - "value": { - "mode": "percentage", - "steps": [ - { - "color": "semi-dark-red", - "value": null - }, - { - "color": "semi-dark-red", - "value": 0 - }, - { - "color": "semi-dark-green", - "value": 1 - }, - { - "color": "semi-dark-yellow", - "value": 2 - }, - { - "color": "light-blue", - "value": 3 - }, - { - "color": "purple", - "value": 4 - } - ] - } - }, - { - "id": "custom.cellOptions", - "value": { - "mode": "basic", - "type": "color-background" - } - }, - { - "id": "custom.align", - "value": "center" - }, - { - "id": "links", - "value": [ - { - "title": "", - "url": "${__data.fields.run5id}" - } - ] - }, - { - "id": "displayName", - "value": "Run 5" - } - ] - }, - { - "matcher": { - "id": "byName", - "options": "run_6" - }, - "properties": [ - { - "id": "color", - "value": { - "fixedColor": "dark-green", - "mode": "fixed" - } - }, - { - "id": "custom.hidden", - "value": false - }, - { - "id": "thresholds", - "value": { - "mode": "percentage", - "steps": [ - { - "color": "semi-dark-red", - "value": null - }, - { - "color": "semi-dark-red", - "value": 0 - }, - { - "color": "semi-dark-green", - "value": 1 - }, - { - "color": "semi-dark-yellow", - "value": 2 - }, - { - "color": "light-blue", - "value": 3 - }, - { - "color": "purple", - "value": 4 - } - ] - } - }, - { - "id": "custom.cellOptions", - "value": { - "mode": "basic", - "type": "color-background" - } - }, - { - "id": "custom.align", - "value": "center" - }, - { - "id": "links", - "value": [ - { - "title": "", - "url": "${__data.fields.run6id}" - } - ] - }, - { - "id": "displayName", - "value": "Run 6" - } - ] - }, - { - "matcher": { - "id": "byName", - "options": "run_7" - }, - "properties": [ - { - "id": "color", - "value": { - "fixedColor": "dark-green", - "mode": "fixed" - } - }, - { - "id": "custom.hidden", - "value": false - }, - { - "id": "thresholds", - "value": { - "mode": "percentage", - "steps": [ - { - "color": "semi-dark-red", - "value": null - }, - { - "color": "semi-dark-red", - "value": 0 - }, - { - "color": "semi-dark-green", - "value": 1 - }, - { - "color": "semi-dark-yellow", - "value": 2 - }, - { - "color": "light-blue", - "value": 3 - }, - { - "color": "purple", - "value": 4 - } - ] - } - }, - { - "id": "custom.cellOptions", - "value": { - "mode": "basic", - "type": "color-background" - } - }, - { - "id": "custom.align", - "value": "center" - }, - { - "id": "links", - "value": [ - { - "title": "", - "url": "${__data.fields.run7id}" - } - ] - }, - { - "id": "displayName", - "value": "Run 7" - } - ] - }, - { - "matcher": { - "id": "byName", - "options": "run_8" - }, - "properties": [ - { - "id": "color", - "value": { - "fixedColor": "dark-green", - "mode": "fixed" - } - }, - { - "id": "custom.hidden", - "value": false - }, - { - "id": "thresholds", - "value": { - "mode": "percentage", - "steps": [ - { - "color": "semi-dark-red", - "value": null - }, - { - "color": "semi-dark-red", - "value": 0 - }, - { - "color": "semi-dark-green", - "value": 1 - }, - { - "color": "semi-dark-yellow", - "value": 2 - }, - { - "color": "light-blue", - "value": 3 - }, - { - "color": "purple", - "value": 4 - } - ] - } - }, - { - "id": "custom.cellOptions", - "value": { - "mode": "basic", - "type": "color-background" - } - }, - { - "id": "custom.align", - "value": "center" - }, - { - "id": "links", - "value": [ - { - "title": "", - "url": "${__data.fields.run8id}" - } - ] - }, - { - "id": "displayName", - "value": "Run 8" - } - ] - }, - { - "matcher": { - "id": "byName", - "options": "run_9" - }, - "properties": [ - { - "id": "color", - "value": { - "fixedColor": "dark-green", - "mode": "fixed" - } - }, - { - "id": "custom.hidden", - "value": false - }, - { - "id": "thresholds", - "value": { - "mode": "percentage", - "steps": [ - { - "color": "semi-dark-red", - "value": null - }, - { - "color": "semi-dark-red", - "value": 0 - }, - { - "color": "semi-dark-green", - "value": 1 - }, - { - "color": "semi-dark-yellow", - "value": 2 - }, - { - "color": "light-blue", - "value": 3 - }, - { - "color": "purple", - "value": 4 - } - ] - } - }, - { - "id": "custom.cellOptions", - "value": { - "mode": "basic", - "type": "color-background" - } - }, - { - "id": "custom.align", - "value": "center" - }, - { - "id": "links", - "value": [ - { - "title": "", - "url": "${__data.fields.run9id}" - } - ] - }, - { - "id": "displayName", - "value": "Run 9" - } - ] - }, - { - "matcher": { - "id": "byName", - "options": "run_10" - }, - "properties": [ - { - "id": "color", - "value": { - "fixedColor": "dark-green", - "mode": "fixed" - } - }, - { - "id": "custom.hidden", - "value": false - }, - { - "id": "thresholds", - "value": { - "mode": "percentage", - "steps": [ - { - "color": "semi-dark-red", - "value": null - }, - { - "color": "semi-dark-red", - "value": 0 - }, - { - "color": "semi-dark-green", - "value": 1 - }, - { - "color": "semi-dark-yellow", - "value": 2 - }, - { - "color": "light-blue", - "value": 3 - }, - { - "color": "purple", - "value": 4 - } - ] - } - }, - { - "id": "custom.cellOptions", - "value": { - "mode": "basic", - "type": "color-background" - } - }, - { - "id": "custom.align", - "value": "center" - }, - { - "id": "links", - "value": [ - { - "title": "", - "url": "${__data.fields.run10id}" - } - ] - }, - { - "id": "displayName", - "value": "Run 10" - } - ] - } - ] - }, - "gridPos": { - "h": 20, - "w": 24, - "x": 0, - "y": 220 - }, - "id": 4, - "links": [], - "options": { - "cellHeight": "sm", - "footer": { - "countRows": false, - "fields": "", - "reducer": [ - "sum" - ], - "show": false - }, - "showHeader": true - }, - "pluginVersion": "10.0.3", - "targets": [ - { - "aggregation": "Last", - "alias": "job", - "decimals": 2, - "displayAliasType": "Warning / Critical", - "displayType": "Regular", - "displayValueWithAlias": "Never", - "editorMode": "code", - "format": "table", - "group": [], - "hide": false, - "metricColumn": "none", - "rawQuery": true, - "rawSql": "with workflows as (\n select\n *,\n case when run1 like 'success' then 1 when run1 like 'in_progress' then 2 when run1 like 'queued' then 2 when run1 like 'waiting' then 2 when run1 like 'cancelled' then 3 when run1 like 'None' then 4 else 0 end as run_1,\n case when run2 like 'success' then 1 when run2 like 'in_progress' then 2 when run2 like 'queued' then 2 when run2 like 'waiting' then 2 when run2 like 'cancelled' then 3 when run2 like 'None' then 4 else 0 end as run_2,\n case when run3 like 'success' then 1 when run3 like 'in_progress' then 2 when run3 like 'queued' then 2 when run3 like 'waiting' then 2 when run3 like 'cancelled' then 3 when run3 like 'None' then 4 else 0 end as run_3,\n case when run4 like 'success' then 1 when run4 like 'in_progress' then 2 when run4 like 'queued' then 2 when run4 like 'waiting' then 2 when run4 like 'cancelled' then 3 when run4 like 'None' then 4 else 0 end as run_4,\n case when run5 like 'success' then 1 when run5 like 'in_progress' then 2 when run5 like 'queued' then 2 when run5 like 'waiting' then 2 when run5 like 'cancelled' then 3 when run5 like 'None' then 4 else 0 end as run_5,\n case when run6 like 'success' then 1 when run6 like 'in_progress' then 2 when run6 like 'queued' then 2 when run6 like 'waiting' then 2 when run6 like 'cancelled' then 3 when run6 like 'None' then 4 else 0 end as run_6,\n case when run7 like 'success' then 1 when run7 like 'in_progress' then 2 when run7 like 'queued' then 2 when run7 like 'waiting' then 2 when run7 like 'cancelled' then 3 when run7 like 'None' then 4 else 0 end as run_7,\n case when run8 like 'success' then 1 when run8 like 'in_progress' then 2 when run8 like 'queued' then 2 when run8 like 'waiting' then 2 when run8 like 'cancelled' then 3 when run8 like 'None' then 4 else 0 end as run_8,\n case when run9 like 'success' then 1 when run9 like 'in_progress' then 2 when run9 like 'queued' then 2 when run9 like 'waiting' then 2 when run9 like 'cancelled' then 3 when run9 like 'None' then 4 else 0 end as run_9,\n case when run10 like 'success' then 1 when run10 like 'in_progress' then 2 when run10 like 'queued' then 2 when run10 like 'waiting' then 2 when run10 like 'cancelled' then 3 when run10 like 'None' then 4 else 0 end as run_10\n from\n github_workflows\n where\n dashboard_category = 'flink_go'\n)\nselect\n job_name,\n job_yml_filename,\n run_1,\n run1Id,\n run_2,\n run2Id,\n run_3,\n run3Id,\n run_4,\n run4Id,\n run_5,\n run5Id,\n run_6,\n run6Id,\n run_7,\n run7Id,\n run_8,\n run8Id,\n run_9,\n run9Id,\n run_10,\n run10Id\nfrom\n workflows;", - "refId": "A", - "select": [ - [ - { - "params": [ - "value" - ], - "type": "column" - } - ] - ], - "timeColumn": "time", - "units": "none", - "valueHandler": "Number Threshold", - "where": [ - { - "name": "$__timeFilter", - "params": [], - "type": "macro" - } - ] - } - ], - "title": "Flink Go Tests", - "transformations": [ - { - "id": "merge", - "options": { - "reducers": [] - } - } - ], - "type": "table" - }, - { - "datasource": "BeamPSQL", - "fieldConfig": { - "defaults": { - "color": { - "mode": "thresholds" - }, - "custom": { - "align": "auto", - "cellOptions": { - "type": "auto" - }, - "inspect": false - }, - "mappings": [ - { - "options": { - "0": { - "color": "semi-dark-red", - "index": 0, - "text": "Fail" - }, - "1": { - "color": "semi-dark-green", - "index": 1, - "text": "Success" - }, - "2": { - "color": "semi-dark-yellow", - "index": 2, - "text": "Pending" - }, - "3": { - "color": "semi-dark-purple", - "index": 3, - "text": "Cancelled" - }, - "4": { - "color": "light-blue", - "index": 4, - "text": "None" - } - }, - "type": "value" - } - ], - "thresholds": { - "mode": "absolute", - "steps": [ - { - "color": "semi-dark-red", - "value": null - } - ] - } - }, - "overrides": [ - { - "matcher": { - "id": "byName", - "options": "job_name" - }, - "properties": [ - { - "id": "unit", - "value": "short" - }, - { - "id": "decimals", - "value": 2 - }, - { - "id": "links", - "value": [ - { - "targetBlank": true, - "title": "", - "url": "https://github.com/apache/beam/actions/${__data.fields.job_yml_filename}" - } - ] - }, - { - "id": "custom.align", - "value": "auto" - }, - { - "id": "mappings", - "value": [] - }, - { - "id": "displayName", - "value": "Job Name" - } - ] - }, - { - "matcher": { - "id": "byName", - "options": "job_yml_filename" - }, - "properties": [ - { - "id": "custom.hidden", - "value": true - } - ] - }, - { - "matcher": { - "id": "byRegexp", - "options": "/run\\d+id/" - }, - "properties": [ - { - "id": "custom.hidden", - "value": true - }, - { - "id": "custom.align", - "value": "center" - } - ] - }, - { - "matcher": { - "id": "byName", - "options": "run_1" - }, - "properties": [ - { - "id": "color", - "value": { - "fixedColor": "dark-green", - "mode": "fixed" - } - }, - { - "id": "custom.hidden", - "value": false - }, - { - "id": "thresholds", - "value": { - "mode": "percentage", - "steps": [ - { - "color": "semi-dark-red", - "value": null - }, - { - "color": "semi-dark-red", - "value": 0 - }, - { - "color": "semi-dark-green", - "value": 1 - }, - { - "color": "semi-dark-yellow", - "value": 2 - }, - { - "color": "light-blue", - "value": 3 - }, - { - "color": "purple", - "value": 4 - } - ] - } - }, - { - "id": "custom.cellOptions", - "value": { - "mode": "basic", - "type": "color-background" - } - }, - { - "id": "custom.align", - "value": "center" - }, - { - "id": "links", - "value": [ - { - "title": "", - "url": "${__data.fields.run1id}" - } - ] - }, - { - "id": "displayName", - "value": "Run" - } - ] - }, - { - "matcher": { - "id": "byName", - "options": "run_2" - }, - "properties": [ - { - "id": "color", - "value": { - "fixedColor": "dark-green", - "mode": "fixed" - } - }, - { - "id": "custom.hidden", - "value": false - }, - { - "id": "thresholds", - "value": { - "mode": "percentage", - "steps": [ - { - "color": "semi-dark-red", - "value": null - }, - { - "color": "semi-dark-red", - "value": 0 - }, - { - "color": "semi-dark-green", - "value": 1 - }, - { - "color": "semi-dark-yellow", - "value": 2 - }, - { - "color": "light-blue", - "value": 3 - }, - { - "color": "purple", - "value": 4 - } - ] - } - }, - { - "id": "custom.cellOptions", - "value": { - "mode": "basic", - "type": "color-background" - } - }, - { - "id": "custom.align", - "value": "center" - }, - { - "id": "links", - "value": [ - { - "title": "", - "url": "${__data.fields.run2id}" - } - ] - }, - { - "id": "displayName", - "value": "Run 2" - } - ] - }, - { - "matcher": { - "id": "byName", - "options": "run_3" - }, - "properties": [ - { - "id": "color", - "value": { - "fixedColor": "dark-green", - "mode": "fixed" - } - }, - { - "id": "custom.hidden", - "value": false - }, - { - "id": "thresholds", - "value": { - "mode": "percentage", - "steps": [ - { - "color": "semi-dark-red", - "value": null - }, - { - "color": "semi-dark-red", - "value": 0 - }, - { - "color": "semi-dark-green", - "value": 1 - }, - { - "color": "semi-dark-yellow", - "value": 2 - }, - { - "color": "light-blue", - "value": 3 - }, - { - "color": "purple", - "value": 4 - } - ] - } - }, - { - "id": "custom.cellOptions", - "value": { - "mode": "basic", - "type": "color-background" - } - }, - { - "id": "custom.align", - "value": "center" - }, - { - "id": "links", - "value": [ - { - "title": "", - "url": "${__data.fields.run3id}" - } - ] - }, - { - "id": "displayName", - "value": "Run 3" - } - ] - }, - { - "matcher": { - "id": "byName", - "options": "run_4" - }, - "properties": [ - { - "id": "color", - "value": { - "fixedColor": "dark-green", - "mode": "fixed" - } - }, - { - "id": "custom.hidden", - "value": false - }, - { - "id": "thresholds", - "value": { - "mode": "percentage", - "steps": [ - { - "color": "semi-dark-red", - "value": null - }, - { - "color": "semi-dark-red", - "value": 0 - }, - { - "color": "semi-dark-green", - "value": 1 - }, - { - "color": "semi-dark-yellow", - "value": 2 - }, - { - "color": "light-blue", - "value": 3 - }, - { - "color": "purple", - "value": 4 - } - ] - } - }, - { - "id": "custom.cellOptions", - "value": { - "mode": "basic", - "type": "color-background" - } - }, - { - "id": "custom.align", - "value": "center" - }, - { - "id": "links", - "value": [ - { - "title": "", - "url": "${__data.fields.run4id}" - } - ] - }, - { - "id": "displayName", - "value": "Run 4" - } - ] - }, - { - "matcher": { - "id": "byName", - "options": "run_5" - }, - "properties": [ - { - "id": "color", - "value": { - "fixedColor": "dark-green", - "mode": "fixed" - } - }, - { - "id": "custom.hidden", - "value": false - }, - { - "id": "thresholds", - "value": { - "mode": "percentage", - "steps": [ - { - "color": "semi-dark-red", - "value": null - }, - { - "color": "semi-dark-red", - "value": 0 - }, - { - "color": "semi-dark-green", - "value": 1 - }, - { - "color": "semi-dark-yellow", - "value": 2 - }, - { - "color": "light-blue", - "value": 3 - }, - { - "color": "purple", - "value": 4 - } - ] - } - }, - { - "id": "custom.cellOptions", - "value": { - "mode": "basic", - "type": "color-background" - } - }, - { - "id": "custom.align", - "value": "center" - }, - { - "id": "links", - "value": [ - { - "title": "", - "url": "${__data.fields.run5id}" - } - ] - }, - { - "id": "displayName", - "value": "Run 5" - } - ] - }, - { - "matcher": { - "id": "byName", - "options": "run_6" - }, - "properties": [ - { - "id": "color", - "value": { - "fixedColor": "dark-green", - "mode": "fixed" - } - }, - { - "id": "custom.hidden", - "value": false - }, - { - "id": "thresholds", - "value": { - "mode": "percentage", - "steps": [ - { - "color": "semi-dark-red", - "value": null - }, - { - "color": "semi-dark-red", - "value": 0 - }, - { - "color": "semi-dark-green", - "value": 1 - }, - { - "color": "semi-dark-yellow", - "value": 2 - }, - { - "color": "light-blue", - "value": 3 - }, - { - "color": "purple", - "value": 4 - } - ] - } - }, - { - "id": "custom.cellOptions", - "value": { - "mode": "basic", - "type": "color-background" - } - }, - { - "id": "custom.align", - "value": "center" - }, - { - "id": "links", - "value": [ - { - "title": "", - "url": "${__data.fields.run6id}" - } - ] - }, - { - "id": "displayName", - "value": "Run 6" - } - ] - }, - { - "matcher": { - "id": "byName", - "options": "run_7" - }, - "properties": [ - { - "id": "color", - "value": { - "fixedColor": "dark-green", - "mode": "fixed" - } - }, - { - "id": "custom.hidden", - "value": false - }, - { - "id": "thresholds", - "value": { - "mode": "percentage", - "steps": [ - { - "color": "semi-dark-red", - "value": null - }, - { - "color": "semi-dark-red", - "value": 0 - }, - { - "color": "semi-dark-green", - "value": 1 - }, - { - "color": "semi-dark-yellow", - "value": 2 - }, - { - "color": "light-blue", - "value": 3 - }, - { - "color": "purple", - "value": 4 - } - ] - } - }, - { - "id": "custom.cellOptions", - "value": { - "mode": "basic", - "type": "color-background" - } - }, - { - "id": "custom.align", - "value": "center" - }, - { - "id": "links", - "value": [ - { - "title": "", - "url": "${__data.fields.run7id}" - } - ] - }, - { - "id": "displayName", - "value": "Run 7" - } - ] - }, - { - "matcher": { - "id": "byName", - "options": "run_8" - }, - "properties": [ - { - "id": "color", - "value": { - "fixedColor": "dark-green", - "mode": "fixed" - } - }, - { - "id": "custom.hidden", - "value": false - }, - { - "id": "thresholds", - "value": { - "mode": "percentage", - "steps": [ - { - "color": "semi-dark-red", - "value": null - }, - { - "color": "semi-dark-red", - "value": 0 - }, - { - "color": "semi-dark-green", - "value": 1 - }, - { - "color": "semi-dark-yellow", - "value": 2 - }, - { - "color": "light-blue", - "value": 3 - }, - { - "color": "purple", - "value": 4 - } - ] - } - }, - { - "id": "custom.cellOptions", - "value": { - "mode": "basic", - "type": "color-background" - } - }, - { - "id": "custom.align", - "value": "center" - }, - { - "id": "links", - "value": [ - { - "title": "", - "url": "${__data.fields.run8id}" - } - ] - }, - { - "id": "displayName", - "value": "Run 8" - } - ] - }, - { - "matcher": { - "id": "byName", - "options": "run_9" - }, - "properties": [ - { - "id": "color", - "value": { - "fixedColor": "dark-green", - "mode": "fixed" - } - }, - { - "id": "custom.hidden", - "value": false - }, - { - "id": "thresholds", - "value": { - "mode": "percentage", - "steps": [ - { - "color": "semi-dark-red", - "value": null - }, - { - "color": "semi-dark-red", - "value": 0 - }, - { - "color": "semi-dark-green", - "value": 1 - }, - { - "color": "semi-dark-yellow", - "value": 2 - }, - { - "color": "light-blue", - "value": 3 - }, - { - "color": "purple", - "value": 4 - } - ] - } - }, - { - "id": "custom.cellOptions", - "value": { - "mode": "basic", - "type": "color-background" - } - }, - { - "id": "custom.align", - "value": "center" - }, - { - "id": "links", - "value": [ - { - "title": "", - "url": "${__data.fields.run9id}" - } - ] - }, - { - "id": "displayName", - "value": "Run 9" - } - ] - }, - { - "matcher": { - "id": "byName", - "options": "run_10" - }, - "properties": [ - { - "id": "color", - "value": { - "fixedColor": "dark-green", - "mode": "fixed" - } - }, - { - "id": "custom.hidden", - "value": false - }, - { - "id": "thresholds", - "value": { - "mode": "percentage", - "steps": [ - { - "color": "semi-dark-red", - "value": null - }, - { - "color": "semi-dark-red", - "value": 0 - }, - { - "color": "semi-dark-green", - "value": 1 - }, - { - "color": "semi-dark-yellow", - "value": 2 - }, - { - "color": "light-blue", - "value": 3 - }, - { - "color": "purple", - "value": 4 - } - ] - } - }, - { - "id": "custom.cellOptions", - "value": { - "mode": "basic", - "type": "color-background" - } - }, - { - "id": "custom.align", - "value": "center" - }, - { - "id": "links", - "value": [ - { - "title": "", - "url": "${__data.fields.run10id}" - } - ] - }, - { - "id": "displayName", - "value": "Run 10" - } - ] - } - ] - }, - "gridPos": { - "h": 20, - "w": 24, - "x": 0, - "y": 240 - }, - "id": 14, - "links": [], - "options": { - "cellHeight": "sm", - "footer": { - "countRows": false, - "fields": "", - "reducer": [ - "sum" - ], - "show": false - }, - "showHeader": true - }, - "pluginVersion": "10.0.3", "targets": [ { "aggregation": "Last", diff --git a/.test-infra/metrics/sync/github/sync_workflows.py b/.test-infra/metrics/sync/github/sync_workflows.py index 1e1b3b004f62..a2c062b175ea 100644 --- a/.test-infra/metrics/sync/github/sync_workflows.py +++ b/.test-infra/metrics/sync/github/sync_workflows.py @@ -47,6 +47,330 @@ GH_NUMBER_OF_WORKFLOW_RUNS_TO_FETCH =\ os.environ['GH_NUMBER_OF_WORKFLOW_RUNS_TO_FETCH'] +# Maps workflows to dashboard category. Any workflows not in one of these lists +# will get auto-mapped to misc. +CORE_JAVA_TESTS = [ + 'PreCommit SQL Java17', + 'PreCommit SQL Java11', + 'LoadTests Java GBK Smoke', + 'PreCommit Java Amazon-Web-Services IO Direct', + 'PreCommit Java Amqp IO Direct', + 'PreCommit Java Amazon-Web-Services2 IO Direct', + 'PreCommit Java', + 'PreCommit Java Cassandra IO Direct', + 'PreCommit Java Azure IO Direct', + 'PreCommit Java Cdap IO Direct', + 'PreCommit Java Clickhouse IO Direct', + 'PreCommit Java Csv IO Direct', + 'Java Tests', + 'PostCommit Java Avro Versions', + 'PreCommit Java Debezium IO Direct', + 'PreCommit Java File-schema-transform IO Direct', + 'PostCommit Java', + 'PreCommit Java GCP IO Direct', + 'PostCommit Java BigQueryEarlyRollout', + 'PreCommit Java Google-ads IO Direct', + 'PreCommit Java HBase IO Direct', + 'PreCommit Java ElasticSearch IO Direct', + 'PreCommit Java HCatalog IO Direct', + 'PreCommit Java Hadoop IO Direct', + 'PreCommit Java IOs Direct', + 'PostCommit Java Hadoop Versions', + 'PreCommit Java Jms IO Direct', + 'PostCommit Java ValidatesRunner Direct JavaVersions', + 'PreCommit Java Kafka IO Direct', + 'PostCommit Java Examples Direct', + 'PreCommit Java JDBC IO Direct', + 'PostCommit Java ValidatesRunner Samza', + 'PreCommit Java Mqtt IO Direct', + 'PreCommit Java Kinesis IO Direct', + 'PreCommit Java MongoDb IO Direct', + 'PostCommit Java IO Performance Tests', + 'PreCommit Java Kudu IO Direct', + 'PostCommit Java InfluxDbIO Integration Test', + 'PostCommit Java Jpms Direct Java21', + 'PostCommit Java ValidatesRunner Twister2', + 'PreCommit Java Neo4j IO Direct', + 'PostCommit Java Jpms Direct Java11', + 'PostCommit Javadoc', + 'PostCommit Java Jpms Direct Java17', + 'PreCommit Java Pulsar IO Direct', + 'PostCommit Java ValidatesRunner ULR', + 'PreCommit Java Parquet IO Direct', + 'PreCommit Java Redis IO Direct', + 'Java JMH', + 'PreCommit Java RabbitMq IO Direct', + 'PreCommit Java RequestResponse IO Direct', + 'PostCommit Java Nexmark Direct', + 'PreCommit Java Splunk IO Direct', + 'PreCommit Java Thrift IO Direct', + 'PreCommit Java Snowflake IO Direct', + 'PreCommit Java Solr IO Direct', + 'PostCommit Java PVR Samza', + 'PreCommit Java Tika IO Direct', + 'PostCommit Java SingleStoreIO IT', + 'PostCommit Java Sickbay', + 'PostCommit Java ValidatesRunner Direct', + 'PreCommit Java SingleStore IO Direct', + 'PreCommit Java InfluxDb IO Direct', + 'PreCommit Spotless', + 'PreCommit Kotlin Examples' +] + +DATAFLOW_JAVA_TESTS = [ + 'PostCommit XVR GoUsingJava Dataflow', + 'PostCommit XVR PythonUsingJavaSQL Dataflow', + 'PostCommit XVR JavaUsingPython Dataflow', + 'PostCommit XVR PythonUsingJava Dataflow', + 'PreCommit Java Examples Dataflow Java11', + 'PreCommit Java Examples Dataflow Java17', + 'PreCommit Java Examples Dataflow Java21', + 'PreCommit Java Examples Dataflow', + 'PostCommit Java ValidatesRunner Dataflow', + 'PostCommit Java Dataflow V1', + 'PostCommit Java ValidatesRunner Dataflow Streaming', + 'PostCommit Java Dataflow V2', + 'PostCommit Java ValidatesRunner Dataflow V2', + 'PostCommit Java Examples Dataflow', + 'PostCommit Java Examples Dataflow ARM', + 'PostCommit Java ValidatesRunner Dataflow V2 Streaming', + 'PostCommit Java ValidatesRunner Dataflow JavaVersions', + 'PostCommit Java Examples Dataflow Java', + 'PostCommit Java Examples Dataflow V2 Java', + 'PostCommit Java Jpms Dataflow Java11', + 'PostCommit Java Jpms Dataflow Java17', + 'PostCommit Java Nexmark Dataflow', + 'PostCommit Java Nexmark Dataflow V2', + 'PostCommit Java Nexmark Dataflow V2 Java', + 'PostCommit Java Tpcds Dataflow', + 'PostCommit Java Examples Dataflow V2' +] + +RUNNERS_JAVA_TESTS = [ + 'PostCommit Java PVR Spark3 Streaming', + 'PostCommit Java ValidatesRunner Spark', + 'PostCommit Java Examples Spark', + 'PostCommit Java ValidatesRunner SparkStructuredStreaming', + 'PostCommit Java ValidatesRunner Spark Java11', + 'PostCommit Java PVR Spark Batch', + 'PreCommit Java Spark3 Versions', + 'PostCommit Java Tpcds Spark', + 'PostCommit Java Jpms Spark Java11', + 'PostCommit Java Nexmark Spark', + 'PostCommit Java Examples Flink', + 'PostCommit Java Tpcds Flink', + 'PostCommit Java PVR Flink Streaming', + 'PostCommit Java Jpms Flink Java11', + 'PreCommit Java PVR Flink Batch', + 'PostCommit Java Nexmark Flink', + 'PreCommit Java PVR Flink Docker', + 'PreCommit Java Flink Versions', + 'PostCommit Java ValidatesRunner Flink Java11', + 'PostCommit Java ValidatesRunner Flink' +] + +LOAD_PERF_JAVA_TESTS = [ + 'LoadTests Java CoGBK Dataflow Batch', + 'LoadTests Java CoGBK Dataflow V2 Streaming JavaVersions', + 'LoadTests Java CoGBK Dataflow Streaming', + 'LoadTests Java Combine Dataflow Batch', + 'LoadTests Java Combine Dataflow Streaming', + 'LoadTests Java CoGBK Dataflow V2 Batch JavaVersions', + 'LoadTests Java GBK Dataflow Batch', + 'LoadTests Java GBK Dataflow Streaming', + 'LoadTests Java GBK Dataflow V2 Batch Java11', + 'LoadTests Java GBK Dataflow V2 Streaming Java11', + 'LoadTests Java GBK Dataflow V2 Batch Java17', + 'LoadTests Java GBK Dataflow V2 Streaming Java17', + 'LoadTests Java ParDo Dataflow Streaming', + 'LoadTests Java ParDo Dataflow V2 Streaming JavaVersions', + 'LoadTests Java ParDo Dataflow V2 Batch JavaVersions', + 'LoadTests Java ParDo Dataflow Batch', + 'LoadTests Java ParDo SparkStructuredStreaming Batch', + 'LoadTests Java CoGBK SparkStructuredStreaming Batch', + 'LoadTests Java Combine SparkStructuredStreaming Batch', + 'LoadTests Java GBK SparkStructuredStreaming Batch', + 'PerformanceTests BigQueryIO Batch Java Avro', + 'PerformanceTests BigQueryIO Streaming Java', + 'PerformanceTests BigQueryIO Batch Java Json', + 'PerformanceTests SQLBigQueryIO Batch Java', + 'PerformanceTests XmlIOIT', + 'PostCommit XVR Samza', + 'PerformanceTests ManyFiles TextIOIT', + 'PerformanceTests XmlIOIT HDFS', + 'PerformanceTests ParquetIOIT', + 'PerformanceTests ParquetIOIT HDFS', + 'PerformanceTests AvroIOIT', + 'PerformanceTests ManyFiles TextIOIT HDFS', + 'PerformanceTests TFRecordIOIT', + 'PerformanceTests Cdap', + 'PerformanceTests TextIOIT', + 'PerformanceTests AvroIOIT HDFS', + 'PerformanceTests SingleStoreIO', + 'PerformanceTests SparkReceiver IO', + 'PerformanceTests Compressed TextIOIT', + 'PerformanceTests TextIOIT HDFS', + 'PerformanceTests Compressed TextIOIT HDFS', + 'PerformanceTests HadoopFormat', + 'PerformanceTests JDBC', + 'PerformanceTests Kafka IO' +] + +CORE_PYTHON_TESTS = [ + 'Python Dependency Tests', + 'PreCommit Python Dataframes', + 'PreCommit Python Examples', + 'PreCommit Python Integration', + 'PostCommit Python ValidatesRunner Samza', + 'LoadTests Python Smoke', + 'Update Python Depedencies', + 'PreCommit Python Runners', + 'PreCommit Python Transforms', + 'PostCommit Python Xlang Gcp Direct', + 'Build python source distribution and wheels', + 'Python tests', + 'PostCommit Sickbay Python', + 'PostCommit Python', + 'PostCommit Python Arm', + 'PostCommit Python Examples Direct', + 'PreCommit Portable Python', + 'PreCommit Python Coverage', + 'PreCommit Python Docker', + 'PreCommit Python', + 'PostCommit Python MongoDBIO IT', + 'PreCommit Python Docs', + 'PreCommit Python Formatter', + 'PostCommit Python Nexmark Direct', + 'PreCommit Python Lint' +] + +RUNNERS_PYTHON_TESTS = [ + 'PostCommit Python ValidatesRunner Dataflow', + 'Python ValidatesContainer Dataflow ARM', + 'PostCommit Python Xlang Gcp Dataflow', + 'PostCommit Python Xlang IO Dataflow', + 'PostCommit Python Examples Dataflow', + 'PostCommit Python ValidatesContainer Dataflow', + 'PostCommit Python ValidatesContainer Dataflow With RC', + 'PostCommit Python ValidatesRunner Spark', + 'PostCommit Python Examples Spark', + 'PostCommit Python ValidatesRunner Flink', + 'PreCommit Python PVR Flink', + 'PostCommit Python Examples Flink' +] + +LOAD_PERF_PYTHON_TESTS = [ + 'PerformanceTests xlang KafkaIO Python', + 'LoadTests Python FnApiRunner Microbenchmark', + 'PerformanceTests SpannerIO Write 2GB Python Batch', + 'PerformanceTests SpannerIO Read 2GB Python', + 'PerformanceTests BiqQueryIO Read Python', + 'PerformanceTests BiqQueryIO Write Python Batch', + 'PerformanceTests TextIOIT Python', + 'PerformanceTests WordCountIT PythonVersions', + 'Performance alerting tool on Python load/performance/benchmark tests.', + 'LoadTests Python SideInput Dataflow Batch', + 'LoadTests Python CoGBK Dataflow Batch', + 'LoadTests Python CoGBK Dataflow Streaming', + 'LoadTests Python Combine Dataflow Batch', + 'Inference Python Benchmarks Dataflow', + 'LoadTests Python Combine Dataflow Streaming', + 'LoadTests Python GBK Dataflow Batch', + 'LoadTests Python GBK Dataflow Streaming', + 'LoadTests Python GBK reiterate Dataflow Batch', + 'LoadTests Python GBK reiterate Dataflow Streaming', + 'LoadTests Python ParDo Dataflow Streaming', + 'CloudML Benchmarks Dataflow', + 'LoadTests Python ParDo Dataflow Batch', + 'LoadTests Python CoGBK Flink Batch', + 'LoadTests Python Combine Flink Batch', + 'LoadTests Python Combine Flink Streaming', + 'PerformanceTests PubsubIOIT Python Streaming', + 'LoadTests Python ParDo Flink Batch', + 'LoadTests Python ParDo Flink Streaming' +] + +GO_TESTS = [ + 'PerformanceTests MongoDBIO IT', + 'PreCommit Go', + 'PreCommit GoPortable', + 'PreCommit GoPrism', + 'PostCommit Go VR Samza', + 'Go tests', + 'PostCommit Go', + 'PostCommit Go Dataflow ARM', + 'LoadTests Go CoGBK Dataflow Batch', + 'LoadTests Go Combine Dataflow Batch', + 'LoadTests Go GBK Dataflow Batch', + 'LoadTests Go ParDo Dataflow Batch', + 'LoadTests Go SideInput Dataflow Batch', + 'PostCommit Go VR Spark', + 'PostCommit Go VR Flink', + 'LoadTests Go CoGBK Flink Batch', + 'LoadTests Go Combine Flink Batch', + 'LoadTests Go GBK Flink Batch', + 'LoadTests Go ParDo Flink Batch', + 'LoadTests Go SideInput Flink Batch' +] + +CORE_INFRA_TESTS = [ + 'Release Nightly Snapshot Python', + 'Rotate Metrics Cluster Credentials', + 'Community Metrics Prober', + 'Publish Docker Snapshots', + 'Clean Up GCP Resources', + 'Clean Up Prebuilt SDK Images', + 'Rotate IO-Datastores Cluster Credentials', + 'Release Nightly Snapshot', + 'Mark issue as triaged when assigned', + 'PostCommit BeamMetrics Publish', + 'PreCommit Community Metrics', + 'Beam Metrics Report', + 'Build and Version Runner Docker Image', + 'PreCommit GHA', + 'pr-bot-prs-needing-attention', + 'PreCommit RAT', + 'Assign or close an issue', + 'PostCommit Website Test', + 'PostCommit Website Publish', + 'PreCommit Website', + 'PreCommit Website Stage GCS', + 'Cleanup Dataproc Resources', + 'PreCommit Whitespace', + 'Publish Beam SDK Snapshots', + 'Cancel Stale Dataflow Jobs', + 'pr-bot-pr-updates', + 'pr-bot-new-prs' +] + +MISC_TESTS = [ + 'Tour of Beam Go integration tests', + 'Tour of Beam Go unittests', + 'Tour Of Beam Frontend Test', + 'PostCommit XVR Spark3', + 'TypeScript Tests', + 'Playground Frontend Test', + 'PostCommit PortableJar Flink', + 'PostCommit SQL', + 'Cancel', + 'PostCommit PortableJar Spark', + 'PreCommit Integration and Load Test Framework', + 'pr-bot-update-reviewers', + 'PostCommit TransformService Direct', + 'Cut Release Branch', + 'Generate issue report', + 'Dask Runner Tests', + 'PreCommit Typescript', + 'PostCommit XVR Direct', + 'Mark and close stale pull requests', + 'PostCommit XVR Flink', + 'IssueTagger', + 'Assign Milestone on issue close', + 'Local environment tests', + 'PreCommit SQL', + 'LabelPrs', + 'build_release_candidate' +] class Workflow: def __init__(self, id, name, filename): @@ -56,26 +380,59 @@ def __init__(self, id, name, filename): self.runs = [] def get_dashboard_category(workflow_name): + # If you add or remove categories in this function, make sure to add or + # remove the corresponding panels here: + # https://github.com/apache/beam/blob/master/.test-infra/metrics/grafana/dashboards/GA-Post-Commits_status_dashboard.json + + if workflow_name in CORE_INFRA_TESTS: + return 'core_infra' + if workflow_name in CORE_JAVA_TESTS: + return 'core_java' + if workflow_name in DATAFLOW_JAVA_TESTS: + return 'dataflow_java' + if workflow_name in RUNNERS_JAVA_TESTS: + return 'runners_java' + if workflow_name in LOAD_PERF_JAVA_TESTS: + return 'load_perf_java' + if workflow_name in CORE_PYTHON_TESTS: + return 'core_python' + if workflow_name in RUNNERS_PYTHON_TESTS: + return 'runners_python' + if workflow_name in LOAD_PERF_PYTHON_TESTS: + return 'load_perf_python' + if workflow_name in GO_TESTS: + return 'go' + if workflow_name in MISC_TESTS: + return 'misc' + + print(f'No category found for workflow: {workflow_name}') + print('Falling back to rules based assignment') + workflow_name = workflow_name.lower() - lang = '' if 'java' in workflow_name: - lang = 'java' + if 'dataflow' in workflow_name: + return 'dataflow_java' + if 'spark' in workflow_name or 'flink' in workflow_name: + return 'runners_java' + if 'performancetest' in workflow_name or 'loadtest' in workflow_name: + return 'load_perf_java' + return 'core_java' elif 'python' in workflow_name: - lang = 'python' + if 'dataflow' in workflow_name or 'spark' in workflow_name or 'flink' in workflow_name: + return 'runners_python' + if 'performancetest' in workflow_name or 'loadtest' in workflow_name: + return 'load_perf_python' + return 'core_python' elif 'go' in workflow_name: - lang = 'go' - else: - # If no language found, toss in miscellaneous bucket - return 'misc' - if 'dataflow' in workflow_name: - return f'dataflow_{lang}' - if 'spark' in workflow_name: - return f'spark_{lang}' - if 'flink' in workflow_name: - return f'flink_{lang}' - return f'core_{lang}' - -async def github_workflows_dashboard_sync(): + return 'go' + + return 'misc' + +def github_workflows_dashboard_sync(data, context): + # Entry point for cloud function, don't change signature + return asyncio.run(sync_workflow_runs()) + +async def sync_workflow_runs(): print('Started') print('Updating table with recent workflow runs') @@ -291,4 +648,4 @@ def database_operations(connection, workflows): connection.close() if __name__ == '__main__': - asyncio.run(github_workflows_dashboard_sync()) + asyncio.run(github_workflows_dashboard_sync(None, None)) From 07d5d534aa8e0f0066e5182a3f17fd9b8a29287c Mon Sep 17 00:00:00 2001 From: Jeff Kinard <35542536+Polber@users.noreply.github.com> Date: Thu, 2 Nov 2023 20:04:39 -0400 Subject: [PATCH 362/435] [YAML] Fix error handling for KafkaSchemaTransforms (#29261) --- .../transforms/providers/ErrorHandling.java | 16 +++ ...KafkaReadSchemaTransformConfiguration.java | 7 + .../KafkaReadSchemaTransformProvider.java | 127 +++++++++--------- .../KafkaWriteSchemaTransformProvider.java | 54 ++++++-- .../beam/sdk/io/kafka/KafkaDlqTest.java | 12 +- .../KafkaReadSchemaTransformProviderTest.java | 3 +- ...KafkaWriteSchemaTransformProviderTest.java | 15 ++- sdks/python/apache_beam/yaml/standard_io.yaml | 2 + .../python/apache_beam/yaml/yaml_transform.py | 2 + 9 files changed, 148 insertions(+), 90 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/ErrorHandling.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/ErrorHandling.java index 3fab157a0e55..7fa29708c9ff 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/ErrorHandling.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/ErrorHandling.java @@ -53,6 +53,12 @@ public static Schema errorSchema(Schema inputSchema) { Schema.Field.of("error_message", Schema.FieldType.STRING)); } + public static Schema errorSchemaBytes() { + return Schema.of( + Schema.Field.of("failed_row", Schema.FieldType.BYTES), + Schema.Field.of("error_message", Schema.FieldType.STRING)); + } + @SuppressWarnings({ "nullness" // TODO(https://github.com/apache/beam/issues/20497) }) @@ -62,4 +68,14 @@ public static Row errorRecord(Schema errorSchema, Row inputRow, Throwable th) { .withFieldValue("error_message", th.getMessage()) .build(); } + + @SuppressWarnings({ + "nullness" // TODO(https://github.com/apache/beam/issues/20497) + }) + public static Row errorRecord(Schema errorSchema, byte[] inputBytes, Throwable th) { + return Row.withSchema(errorSchema) + .withFieldValue("failed_row", inputBytes) + .withFieldValue("error_message", th.getMessage()) + .build(); + } } diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaReadSchemaTransformConfiguration.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaReadSchemaTransformConfiguration.java index 299a4f222e3a..ae03c49b9b04 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaReadSchemaTransformConfiguration.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaReadSchemaTransformConfiguration.java @@ -24,6 +24,7 @@ import org.apache.beam.sdk.schemas.AutoValueSchema; import org.apache.beam.sdk.schemas.annotations.DefaultSchema; import org.apache.beam.sdk.schemas.annotations.SchemaFieldDescription; +import org.apache.beam.sdk.schemas.transforms.providers.ErrorHandling; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Sets; /** @@ -105,6 +106,10 @@ public static Builder builder() { /** Sets the topic from which to read. */ public abstract String getTopic(); + @SchemaFieldDescription("This option specifies whether and where to output unwritable rows.") + @Nullable + public abstract ErrorHandling getErrorHandling(); + /** Builder for the {@link KafkaReadSchemaTransformConfiguration}. */ @AutoValue.Builder public abstract static class Builder { @@ -127,6 +132,8 @@ public abstract static class Builder { /** Sets the topic from which to read. */ public abstract Builder setTopic(String value); + public abstract Builder setErrorHandling(ErrorHandling errorHandling); + /** Builds a {@link KafkaReadSchemaTransformConfiguration} instance. */ public abstract KafkaReadSchemaTransformConfiguration build(); } diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaReadSchemaTransformProvider.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaReadSchemaTransformProvider.java index fcba5c355df1..2f5278cd7129 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaReadSchemaTransformProvider.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaReadSchemaTransformProvider.java @@ -43,10 +43,9 @@ import org.apache.beam.sdk.schemas.transforms.SchemaTransform; import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider; import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider; +import org.apache.beam.sdk.schemas.transforms.providers.ErrorHandling; import org.apache.beam.sdk.schemas.utils.JsonUtils; import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.DoFn.FinishBundle; -import org.apache.beam.sdk.transforms.DoFn.ProcessElement; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.SimpleFunction; @@ -77,8 +76,6 @@ public class KafkaReadSchemaTransformProvider public static final TupleTag<Row> OUTPUT_TAG = new TupleTag<Row>() {}; public static final TupleTag<Row> ERROR_TAG = new TupleTag<Row>() {}; - public static final Schema ERROR_SCHEMA = - Schema.builder().addStringField("error").addNullableByteArrayField("row").build(); final Boolean isTest; final Integer testTimeoutSecs; @@ -98,6 +95,9 @@ protected Class<KafkaReadSchemaTransformConfiguration> configurationClass() { return KafkaReadSchemaTransformConfiguration.class; } + @SuppressWarnings({ + "nullness" // TODO(https://github.com/apache/beam/issues/20497) + }) @Override protected SchemaTransform from(KafkaReadSchemaTransformConfiguration configuration) { final String inputSchema = configuration.getSchema(); @@ -114,14 +114,32 @@ protected SchemaTransform from(KafkaReadSchemaTransformConfiguration configurati consumerConfigs.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, autoOffsetReset); String format = configuration.getFormat(); - - if (format != null && format.equals("RAW")) { - if (inputSchema != null) { - throw new IllegalArgumentException( - "To read from Kafka in RAW format, you can't provide a schema."); + boolean handleErrors = ErrorHandling.hasOutput(configuration.getErrorHandling()); + + if ((format != null && format.equals("RAW")) || (!Strings.isNullOrEmpty(inputSchema))) { + SerializableFunction<byte[], Row> valueMapper; + Schema beamSchema; + if (format != null && format.equals("RAW")) { + if (inputSchema != null) { + throw new IllegalArgumentException( + "To read from Kafka in RAW format, you can't provide a schema."); + } + beamSchema = Schema.builder().addField("payload", Schema.FieldType.BYTES).build(); + valueMapper = getRawBytesToRowFunction(beamSchema); + } else { + assert Strings.isNullOrEmpty(configuration.getConfluentSchemaRegistryUrl()) + : "To read from Kafka, a schema must be provided directly or though Confluent " + + "Schema Registry, but not both."; + + beamSchema = + Objects.equals(format, "JSON") + ? JsonUtils.beamSchemaFromJsonSchema(inputSchema) + : AvroUtils.toBeamSchema(new org.apache.avro.Schema.Parser().parse(inputSchema)); + valueMapper = + Objects.equals(format, "JSON") + ? JsonUtils.getJsonBytesToRowFunction(beamSchema) + : AvroUtils.getAvroBytesToRowFunction(beamSchema); } - Schema rawSchema = Schema.builder().addField("payload", Schema.FieldType.BYTES).build(); - SerializableFunction<byte[], Row> valueMapper = getRawBytesToRowFunction(rawSchema); return new SchemaTransform() { @Override public PCollectionRowTuple expand(PCollectionRowTuple input) { @@ -138,59 +156,23 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) { PCollection<byte[]> kafkaValues = input.getPipeline().apply(kafkaRead.withoutMetadata()).apply(Values.create()); + Schema errorSchema = ErrorHandling.errorSchemaBytes(); PCollectionTuple outputTuple = kafkaValues.apply( - ParDo.of(new ErrorFn("Kafka-read-error-counter", valueMapper)) + ParDo.of( + new ErrorFn( + "Kafka-read-error-counter", valueMapper, errorSchema, handleErrors)) .withOutputTags(OUTPUT_TAG, TupleTagList.of(ERROR_TAG))); - return PCollectionRowTuple.of( - "output", - outputTuple.get(OUTPUT_TAG).setRowSchema(rawSchema), - "errors", - outputTuple.get(ERROR_TAG).setRowSchema(ERROR_SCHEMA)); - } - }; - } + PCollectionRowTuple outputRows = + PCollectionRowTuple.of( + "output", outputTuple.get(OUTPUT_TAG).setRowSchema(beamSchema)); - if (inputSchema != null && !inputSchema.isEmpty()) { - assert Strings.isNullOrEmpty(configuration.getConfluentSchemaRegistryUrl()) - : "To read from Kafka, a schema must be provided directly or though Confluent " - + "Schema Registry, but not both."; - - final Schema beamSchema = - Objects.equals(format, "JSON") - ? JsonUtils.beamSchemaFromJsonSchema(inputSchema) - : AvroUtils.toBeamSchema(new org.apache.avro.Schema.Parser().parse(inputSchema)); - SerializableFunction<byte[], Row> valueMapper = - Objects.equals(format, "JSON") - ? JsonUtils.getJsonBytesToRowFunction(beamSchema) - : AvroUtils.getAvroBytesToRowFunction(beamSchema); - return new SchemaTransform() { - @Override - public PCollectionRowTuple expand(PCollectionRowTuple input) { - KafkaIO.Read<byte[], byte[]> kafkaRead = - KafkaIO.readBytes() - .withConsumerConfigUpdates(consumerConfigs) - .withConsumerFactoryFn(new ConsumerFactoryWithGcsTrustStores()) - .withTopic(configuration.getTopic()) - .withBootstrapServers(configuration.getBootstrapServers()); - if (isTest) { - kafkaRead = kafkaRead.withMaxReadTime(Duration.standardSeconds(testTimeoutSecs)); + PCollection<Row> errorOutput = outputTuple.get(ERROR_TAG).setRowSchema(errorSchema); + if (handleErrors) { + outputRows = outputRows.and(configuration.getErrorHandling().getOutput(), errorOutput); } - - PCollection<byte[]> kafkaValues = - input.getPipeline().apply(kafkaRead.withoutMetadata()).apply(Values.create()); - - PCollectionTuple outputTuple = - kafkaValues.apply( - ParDo.of(new ErrorFn("Kafka-read-error-counter", valueMapper)) - .withOutputTags(OUTPUT_TAG, TupleTagList.of(ERROR_TAG))); - - return PCollectionRowTuple.of( - "output", - outputTuple.get(OUTPUT_TAG).setRowSchema(beamSchema), - "errors", - outputTuple.get(ERROR_TAG).setRowSchema(ERROR_SCHEMA)); + return outputRows; } }; } else { @@ -259,25 +241,38 @@ public List<String> outputCollectionNames() { } public static class ErrorFn extends DoFn<byte[], Row> { - private SerializableFunction<byte[], Row> valueMapper; - private Counter errorCounter; + private final SerializableFunction<byte[], Row> valueMapper; + private final Counter errorCounter; private Long errorsInBundle = 0L; - - public ErrorFn(String name, SerializableFunction<byte[], Row> valueMapper) { + private final boolean handleErrors; + private final Schema errorSchema; + + public ErrorFn( + String name, + SerializableFunction<byte[], Row> valueMapper, + Schema errorSchema, + boolean handleErrors) { this.errorCounter = Metrics.counter(KafkaReadSchemaTransformProvider.class, name); this.valueMapper = valueMapper; + this.handleErrors = handleErrors; + this.errorSchema = errorSchema; } @ProcessElement public void process(@DoFn.Element byte[] msg, MultiOutputReceiver receiver) { + Row mappedRow = null; try { - receiver.get(OUTPUT_TAG).output(valueMapper.apply(msg)); + mappedRow = valueMapper.apply(msg); } catch (Exception e) { + if (!handleErrors) { + throw new RuntimeException(e); + } errorsInBundle += 1; LOG.warn("Error while parsing the element", e); - receiver - .get(ERROR_TAG) - .output(Row.withSchema(ERROR_SCHEMA).addValues(e.toString(), msg).build()); + receiver.get(ERROR_TAG).output(ErrorHandling.errorRecord(errorSchema, msg, e)); + } + if (mappedRow != null) { + receiver.get(OUTPUT_TAG).output(mappedRow); } } diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaWriteSchemaTransformProvider.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaWriteSchemaTransformProvider.java index 1dbe37791c05..93d6d73493d2 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaWriteSchemaTransformProvider.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaWriteSchemaTransformProvider.java @@ -36,13 +36,14 @@ import org.apache.beam.sdk.schemas.transforms.SchemaTransform; import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider; import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider; +import org.apache.beam.sdk.schemas.transforms.providers.ErrorHandling; import org.apache.beam.sdk.schemas.utils.JsonUtils; import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.DoFn.ProcessElement; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.SimpleFunction; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionRowTuple; import org.apache.beam.sdk.values.PCollectionTuple; import org.apache.beam.sdk.values.Row; @@ -67,8 +68,6 @@ public class KafkaWriteSchemaTransformProvider public static final TupleTag<Row> ERROR_TAG = new TupleTag<Row>() {}; public static final TupleTag<KV<byte[], byte[]>> OUTPUT_TAG = new TupleTag<KV<byte[], byte[]>>() {}; - public static final Schema ERROR_SCHEMA = - Schema.builder().addStringField("error").addNullableByteArrayField("row").build(); private static final Logger LOG = LoggerFactory.getLogger(KafkaWriteSchemaTransformProvider.class); @@ -100,25 +99,38 @@ static final class KafkaWriteSchemaTransform extends SchemaTransform implements } public static class ErrorCounterFn extends DoFn<Row, KV<byte[], byte[]>> { - private SerializableFunction<Row, byte[]> toBytesFn; - private Counter errorCounter; + private final SerializableFunction<Row, byte[]> toBytesFn; + private final Counter errorCounter; private Long errorsInBundle = 0L; + private final boolean handleErrors; + private final Schema errorSchema; - public ErrorCounterFn(String name, SerializableFunction<Row, byte[]> toBytesFn) { + public ErrorCounterFn( + String name, + SerializableFunction<Row, byte[]> toBytesFn, + Schema errorSchema, + boolean handleErrors) { this.toBytesFn = toBytesFn; - errorCounter = Metrics.counter(KafkaWriteSchemaTransformProvider.class, name); + this.errorCounter = Metrics.counter(KafkaWriteSchemaTransformProvider.class, name); + this.handleErrors = handleErrors; + this.errorSchema = errorSchema; } @ProcessElement public void process(@DoFn.Element Row row, MultiOutputReceiver receiver) { + KV<byte[], byte[]> output = null; try { - receiver.get(OUTPUT_TAG).output(KV.of(new byte[1], toBytesFn.apply(row))); + output = KV.of(new byte[1], toBytesFn.apply(row)); } catch (Exception e) { + if (!handleErrors) { + throw new RuntimeException(e); + } errorsInBundle += 1; LOG.warn("Error while processing the element", e); - receiver - .get(ERROR_TAG) - .output(Row.withSchema(ERROR_SCHEMA).addValues(e.toString(), row.toString()).build()); + receiver.get(ERROR_TAG).output(ErrorHandling.errorRecord(errorSchema, row, e)); + } + if (output != null) { + receiver.get(OUTPUT_TAG).output(output); } } @@ -129,6 +141,9 @@ public void finish() { } } + @SuppressWarnings({ + "nullness" // TODO(https://github.com/apache/beam/issues/20497) + }) @Override public PCollectionRowTuple expand(PCollectionRowTuple input) { Schema inputSchema = input.get("input").getSchema(); @@ -145,13 +160,17 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) { toBytesFn = AvroUtils.getRowToAvroBytesFunction(inputSchema); } + boolean handleErrors = ErrorHandling.hasOutput(configuration.getErrorHandling()); final Map<String, String> configOverrides = configuration.getProducerConfigUpdates(); + Schema errorSchema = ErrorHandling.errorSchema(inputSchema); PCollectionTuple outputTuple = input .get("input") .apply( "Map rows to Kafka messages", - ParDo.of(new ErrorCounterFn("Kafka-write-error-counter", toBytesFn)) + ParDo.of( + new ErrorCounterFn( + "Kafka-write-error-counter", toBytesFn, errorSchema, handleErrors)) .withOutputTags(OUTPUT_TAG, TupleTagList.of(ERROR_TAG))); outputTuple @@ -167,8 +186,11 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) { .withKeySerializer(ByteArraySerializer.class) .withValueSerializer(ByteArraySerializer.class)); + // TODO: include output from KafkaIO Write once updated from PDone + PCollection<Row> errorOutput = + outputTuple.get(ERROR_TAG).setRowSchema(ErrorHandling.errorSchema(errorSchema)); return PCollectionRowTuple.of( - "errors", outputTuple.get(ERROR_TAG).setRowSchema(ERROR_SCHEMA)); + handleErrors ? configuration.getErrorHandling().getOutput() : "errors", errorOutput); } } @@ -227,6 +249,10 @@ public abstract static class KafkaWriteSchemaTransformConfiguration implements S @Nullable public abstract Map<String, String> getProducerConfigUpdates(); + @SchemaFieldDescription("This option specifies whether and where to output unwritable rows.") + @Nullable + public abstract ErrorHandling getErrorHandling(); + public static Builder builder() { return new AutoValue_KafkaWriteSchemaTransformProvider_KafkaWriteSchemaTransformConfiguration .Builder(); @@ -242,6 +268,8 @@ public abstract static class Builder { public abstract Builder setProducerConfigUpdates(Map<String, String> producerConfigUpdates); + public abstract Builder setErrorHandling(ErrorHandling errorHandling); + public abstract KafkaWriteSchemaTransformConfiguration build(); } } diff --git a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaDlqTest.java b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaDlqTest.java index 48fe969bc9f3..e65d9591a0be 100644 --- a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaDlqTest.java +++ b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaDlqTest.java @@ -22,6 +22,7 @@ import java.util.List; import org.apache.beam.sdk.io.kafka.KafkaReadSchemaTransformProvider.ErrorFn; import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.transforms.providers.ErrorHandling; import org.apache.beam.sdk.schemas.utils.JsonUtils; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; @@ -47,7 +48,6 @@ public class KafkaDlqTest { private static final Schema BEAMSCHEMA = Schema.of(Schema.Field.of("name", Schema.FieldType.STRING)); - private static final Schema ERRORSCHEMA = KafkaReadSchemaTransformProvider.ERROR_SCHEMA; private static final List<Row> ROWS = Arrays.asList( @@ -75,13 +75,14 @@ public void testKafkaErrorFnSuccess() throws Exception { } catch (Exception e) { } PCollection<byte[]> input = p.apply(Create.of(messages)); + Schema errorSchema = ErrorHandling.errorSchemaBytes(); PCollectionTuple output = input.apply( - ParDo.of(new ErrorFn("Kafka-read-error-counter", valueMapper)) + ParDo.of(new ErrorFn("Kafka-read-error-counter", valueMapper, errorSchema, true)) .withOutputTags(OUTPUTTAG, TupleTagList.of(ERRORTAG))); output.get(OUTPUTTAG).setRowSchema(BEAMSCHEMA); - output.get(ERRORTAG).setRowSchema(ERRORSCHEMA); + output.get(ERRORTAG).setRowSchema(errorSchema); PAssert.that(output.get(OUTPUTTAG)).containsInAnyOrder(ROWS); p.run().waitUntilFinish(); @@ -98,13 +99,14 @@ public void testKafkaErrorFnFailure() throws Exception { } catch (Exception e) { } PCollection<byte[]> input = p.apply(Create.of(messagesWithError)); + Schema errorSchema = ErrorHandling.errorSchemaBytes(); PCollectionTuple output = input.apply( - ParDo.of(new ErrorFn("Read-Error-Counter", valueMapper)) + ParDo.of(new ErrorFn("Read-Error-Counter", valueMapper, errorSchema, true)) .withOutputTags(OUTPUTTAG, TupleTagList.of(ERRORTAG))); output.get(OUTPUTTAG).setRowSchema(BEAMSCHEMA); - output.get(ERRORTAG).setRowSchema(ERRORSCHEMA); + output.get(ERRORTAG).setRowSchema(errorSchema); PCollection<Long> count = output.get(ERRORTAG).apply("error_count", Count.globally()); diff --git a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaReadSchemaTransformProviderTest.java b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaReadSchemaTransformProviderTest.java index 367f0f7a535e..1570a33c2580 100644 --- a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaReadSchemaTransformProviderTest.java +++ b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaReadSchemaTransformProviderTest.java @@ -100,7 +100,8 @@ public void testFindTransformAndMakeItWork() { "consumerConfigUpdates", "format", "confluentSchemaRegistrySubject", - "confluentSchemaRegistryUrl"), + "confluentSchemaRegistryUrl", + "errorHandling"), kafkaProvider.configurationSchema().getFields().stream() .map(field -> field.getName()) .collect(Collectors.toSet())); diff --git a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaWriteSchemaTransformProviderTest.java b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaWriteSchemaTransformProviderTest.java index 3a7769f42d72..b8649727f36d 100644 --- a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaWriteSchemaTransformProviderTest.java +++ b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaWriteSchemaTransformProviderTest.java @@ -24,6 +24,7 @@ import java.util.List; import org.apache.beam.sdk.io.kafka.KafkaWriteSchemaTransformProvider.KafkaWriteSchemaTransform.ErrorCounterFn; import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.transforms.providers.ErrorHandling; import org.apache.beam.sdk.schemas.utils.JsonUtils; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; @@ -53,7 +54,6 @@ public class KafkaWriteSchemaTransformProviderTest { private static final Schema BEAMRAWSCHEMA = Schema.of(Schema.Field.of("payload", Schema.FieldType.BYTES)); - private static final Schema ERRORSCHEMA = KafkaWriteSchemaTransformProvider.ERROR_SCHEMA; private static final List<Row> ROWS = Arrays.asList( @@ -93,12 +93,14 @@ public void testKafkaErrorFnSuccess() throws Exception { KV.of(new byte[1], "{\"name\":\"c\"}".getBytes("UTF8"))); PCollection<Row> input = p.apply(Create.of(ROWS)); + Schema errorSchema = ErrorHandling.errorSchema(BEAMSCHEMA); PCollectionTuple output = input.apply( - ParDo.of(new ErrorCounterFn("Kafka-write-error-counter", valueMapper)) + ParDo.of( + new ErrorCounterFn("Kafka-write-error-counter", valueMapper, errorSchema, true)) .withOutputTags(OUTPUT_TAG, TupleTagList.of(ERROR_TAG))); - output.get(ERROR_TAG).setRowSchema(ERRORSCHEMA); + output.get(ERROR_TAG).setRowSchema(errorSchema); PAssert.that(output.get(OUTPUT_TAG)).containsInAnyOrder(msg); p.run().waitUntilFinish(); @@ -113,12 +115,15 @@ public void testKafkaErrorFnRawSuccess() throws Exception { KV.of(new byte[1], "c".getBytes("UTF8"))); PCollection<Row> input = p.apply(Create.of(RAW_ROWS)); + Schema errorSchema = ErrorHandling.errorSchema(BEAMRAWSCHEMA); PCollectionTuple output = input.apply( - ParDo.of(new ErrorCounterFn("Kafka-write-error-counter", valueRawMapper)) + ParDo.of( + new ErrorCounterFn( + "Kafka-write-error-counter", valueRawMapper, errorSchema, true)) .withOutputTags(OUTPUT_TAG, TupleTagList.of(ERROR_TAG))); - output.get(ERROR_TAG).setRowSchema(ERRORSCHEMA); + output.get(ERROR_TAG).setRowSchema(errorSchema); PAssert.that(output.get(OUTPUT_TAG)).containsInAnyOrder(msg); p.run().waitUntilFinish(); diff --git a/sdks/python/apache_beam/yaml/standard_io.yaml b/sdks/python/apache_beam/yaml/standard_io.yaml index aa243313d4e7..116d200a86a7 100644 --- a/sdks/python/apache_beam/yaml/standard_io.yaml +++ b/sdks/python/apache_beam/yaml/standard_io.yaml @@ -61,11 +61,13 @@ 'confluent_schema_registry_url': 'confluentSchemaRegistryUrl' 'confluent_schema_registry_subject': 'confluentSchemaRegistrySubject' 'auto_offset_reset_config': 'autoOffsetResetConfig' + 'error_handling': 'errorHandling' 'WriteToKafka': 'format': 'format' 'topic': 'topic' 'bootstrap_servers': 'bootstrapServers' 'producer_config_updates': 'ProducerConfigUpdates' + 'error_handling': 'errorHandling' underlying_provider: type: beamJar transforms: diff --git a/sdks/python/apache_beam/yaml/yaml_transform.py b/sdks/python/apache_beam/yaml/yaml_transform.py index 3fde097f5c22..ff5547db034c 100644 --- a/sdks/python/apache_beam/yaml/yaml_transform.py +++ b/sdks/python/apache_beam/yaml/yaml_transform.py @@ -227,6 +227,8 @@ def get_pcollection(self, name): outputs = self.get_outputs(transform) if output in outputs: return outputs[output] + elif len(outputs) == 1 and outputs[next(iter(outputs))].tag == output: + return outputs[next(iter(outputs))] else: raise ValueError( f'Unknown output {repr(output)} ' From 2aaf09c0eb6928390d861ba228447338b8ca92d3 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 3 Nov 2023 08:31:24 -0400 Subject: [PATCH 363/435] Bump github.com/aws/aws-sdk-go-v2/config from 1.21.0 to 1.22.0 in /sdks (#29285) Bumps [github.com/aws/aws-sdk-go-v2/config](https://github.com/aws/aws-sdk-go-v2) from 1.21.0 to 1.22.0. - [Release notes](https://github.com/aws/aws-sdk-go-v2/releases) - [Commits](https://github.com/aws/aws-sdk-go-v2/compare/v1.21.0...v1.22.0) --- updated-dependencies: - dependency-name: github.com/aws/aws-sdk-go-v2/config dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] <support@github.com> Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 6 +++--- sdks/go.sum | 12 ++++++------ 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 5d2744cf574c..34cb2ef2cd1f 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -31,8 +31,8 @@ require ( cloud.google.com/go/spanner v1.51.0 cloud.google.com/go/storage v1.34.1 github.com/aws/aws-sdk-go-v2 v1.22.1 - github.com/aws/aws-sdk-go-v2/config v1.21.0 - github.com/aws/aws-sdk-go-v2/credentials v1.15.0 + github.com/aws/aws-sdk-go-v2/config v1.22.0 + github.com/aws/aws-sdk-go-v2/credentials v1.15.1 github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.13.0 github.com/aws/aws-sdk-go-v2/service/s3 v1.42.0 github.com/aws/smithy-go v1.16.0 @@ -98,7 +98,7 @@ require ( github.com/apache/thrift v0.16.0 // indirect github.com/aws/aws-sdk-go v1.34.0 // indirect github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.5.0 // indirect - github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.1 // indirect + github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.2 // indirect github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.1 // indirect github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.1 // indirect github.com/aws/aws-sdk-go-v2/internal/ini v1.5.0 // indirect diff --git a/sdks/go.sum b/sdks/go.sum index 5844d2dcbe35..317eead7ba87 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -86,14 +86,14 @@ github.com/aws/aws-sdk-go-v2 v1.22.1/go.mod h1:Kd0OJtkW3Q0M0lUWGszapWjEvrXDzRW+D github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.5.0 h1:hHgLiIrTRtddC0AKcJr5s7i/hLgcpTt+q/FKxf1Zayk= github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.5.0/go.mod h1:w4I/v3NOWgD+qvs1NPEwhd++1h3XPHFaVxasfY6HlYQ= github.com/aws/aws-sdk-go-v2/config v1.5.0/go.mod h1:RWlPOAW3E3tbtNAqTwvSW54Of/yP3oiZXMI0xfUdjyA= -github.com/aws/aws-sdk-go-v2/config v1.21.0 h1:BZEX5S6hkrAJ9rJn2hEMWIMkg+l1RC7gBQ84qkb0Cp0= -github.com/aws/aws-sdk-go-v2/config v1.21.0/go.mod h1:DMs4GYoTUo21V0cuDvwbdRb6coTcJ/EJxD7/WYZOinA= +github.com/aws/aws-sdk-go-v2/config v1.22.0 h1:9Mm99OalzZRz0ab5fpodMoHBApHS6pqRNp3M9NmzvDg= +github.com/aws/aws-sdk-go-v2/config v1.22.0/go.mod h1:2eWgw5lps8fKI7LZVTrRTYP6HE6k/uEFUuTSHfXwqP0= github.com/aws/aws-sdk-go-v2/credentials v1.3.1/go.mod h1:r0n73xwsIVagq8RsxmZbGSRQFj9As3je72C2WzUIToc= -github.com/aws/aws-sdk-go-v2/credentials v1.15.0 h1:gSRUMOU/wxxf89+4XYg0hYwOmcgdA0bohb7A/5nO+oE= -github.com/aws/aws-sdk-go-v2/credentials v1.15.0/go.mod h1:2zRQYW9jm3t18Ku+qP/107djyjAL7Ght6eBTWiNF/5c= +github.com/aws/aws-sdk-go-v2/credentials v1.15.1 h1:hmf6lAm9hk7uLCfapZn/jL05lm6Uwdbn1B0fgjyuf4M= +github.com/aws/aws-sdk-go-v2/credentials v1.15.1/go.mod h1:QTcHga3ZbQOneJuxmGBOCxiClxmp+TlvmjFexAnJ790= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.3.0/go.mod h1:2LAuqPx1I6jNfaGDucWfA2zqQCYCOMCDHiCOciALyNw= -github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.1 h1:PYlUVUJRnXf2QU7IS+WLNSoU42f/oYRAmpwvXRH0Zq4= -github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.1/go.mod h1:wLyMIo/zPOhQhPXTddpfdkSleyigtFi8iMnC+2m/SK4= +github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.2 h1:gIeH4+o1MN/caGBWjoGQTUTIu94xD6fI5B2+TcwBf70= +github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.2/go.mod h1:wLyMIo/zPOhQhPXTddpfdkSleyigtFi8iMnC+2m/SK4= github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.3.2/go.mod h1:qaqQiHSrOUVOfKe6fhgQ6UzhxjwqVW8aHNegd6Ws4w4= github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.13.0 h1:LPTcGCwO6bup9pTuOmYFXlcocpMX8PPO+TIB7CtSVnI= github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.13.0/go.mod h1:QJXFEiHtNvgiOuBx9cyK6hC28xTB7mQ/iya6Xr1Fu7Y= From 41c495d00bd329fd9c7fec6182b3c8201166c5a8 Mon Sep 17 00:00:00 2001 From: Vlado Djerek <rage.bubblegum@gmail.com> Date: Fri, 3 Nov 2023 14:07:15 +0100 Subject: [PATCH 364/435] adjust node limits to accomodate more runners (#29281) Co-authored-by: Vlado Djerek <djerek.vlado6@gmail.com> --- .../arc/environments/beam.env | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/.github/gh-actions-self-hosted-runners/arc/environments/beam.env b/.github/gh-actions-self-hosted-runners/arc/environments/beam.env index 95c09e1cfad3..7f58e016a85c 100644 --- a/.github/gh-actions-self-hosted-runners/arc/environments/beam.env +++ b/.github/gh-actions-self-hosted-runners/arc/environments/beam.env @@ -38,9 +38,9 @@ main_runner = { runner_image = "us-central1-docker.pkg.dev/apache-beam-testing/beam-github-actions/beam-arc-runner:2b20e26bb3b99d8e4f41a3d1d9d2e7080043de5c" machine_type = "e2-standard-16" min_node_count = "1" - max_node_count = "24" + max_node_count = "30" min_replicas = "1" - max_replicas = "200" + max_replicas = "240" webhook_scaling = false disk_size_gb = 200 requests = { @@ -53,9 +53,9 @@ additional_runner_pools = [{ machine_type = "e2-standard-2" runner_image = "us-central1-docker.pkg.dev/apache-beam-testing/beam-github-actions/beam-arc-runner:2b20e26bb3b99d8e4f41a3d1d9d2e7080043de5c" min_node_count = "1" - max_node_count = "10" + max_node_count = "15" min_replicas = "1" - max_replicas = "10" + max_replicas = "15" webhook_scaling = false requests = { cpu = "1500m" @@ -70,9 +70,9 @@ additional_runner_pools = [{ machine_type = "c3-highmem-8" runner_image = "us-central1-docker.pkg.dev/apache-beam-testing/beam-github-actions/beam-arc-runner:2b20e26bb3b99d8e4f41a3d1d9d2e7080043de5c" min_node_count = "1" - max_node_count = "10" + max_node_count = "15" min_replicas = "1" - max_replicas = "10" + max_replicas = "15" webhook_scaling = false requests = { cpu = "7.5" From a3fe3fa1d037fc0f47f7f70bab493632874d8594 Mon Sep 17 00:00:00 2001 From: Jeff Kinard <35542536+Polber@users.noreply.github.com> Date: Fri, 3 Nov 2023 10:54:21 -0400 Subject: [PATCH 365/435] [YAML] Fix PyPi caching for non-dev beam (#29273) Signed-off-by: Jeffrey Kinard <jeff@thekinards.com> --- sdks/python/apache_beam/yaml/cache_provider_artifacts.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/yaml/cache_provider_artifacts.py b/sdks/python/apache_beam/yaml/cache_provider_artifacts.py index 6c96dd3b0fd9..5efc6d04355b 100644 --- a/sdks/python/apache_beam/yaml/cache_provider_artifacts.py +++ b/sdks/python/apache_beam/yaml/cache_provider_artifacts.py @@ -16,6 +16,7 @@ # import logging +import sys import time from apache_beam.version import __version__ as beam_version @@ -37,7 +38,8 @@ def cache_provider_artifacts(): if '.dev' not in beam_version: # Also cache a base python venv for fast cloning. t = time.time() - artifacts = yaml_provider.PypiExpansionService._create_venv_to_clone() + artifacts = yaml_provider.PypiExpansionService._create_venv_to_clone( + sys.executable) logging.info('Cached %s in %0.03f seconds.', artifacts, time.time() - t) From 04df11c8fc1f9caaef293e0af65b8495ac9e6ac7 Mon Sep 17 00:00:00 2001 From: Anand Inguva <34158215+AnandInguva@users.noreply.github.com> Date: Fri, 3 Nov 2023 15:16:17 +0000 Subject: [PATCH 366/435] Pass empty_match_treatment flag (#29251) --- sdks/python/apache_beam/io/fileio.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/io/fileio.py b/sdks/python/apache_beam/io/fileio.py index 23e979b44cae..e671cfea0797 100644 --- a/sdks/python/apache_beam/io/fileio.py +++ b/sdks/python/apache_beam/io/fileio.py @@ -195,7 +195,8 @@ def __init__( self._empty_match_treatment = empty_match_treatment def expand(self, pcoll) -> beam.PCollection[filesystem.FileMetadata]: - return pcoll.pipeline | beam.Create([self._file_pattern]) | MatchAll() + return pcoll.pipeline | beam.Create([self._file_pattern]) | MatchAll( + empty_match_treatment=self._empty_match_treatment) class MatchAll(beam.PTransform): From f25c441111f9c9212131074c59c034a1faa1ce68 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 3 Nov 2023 11:24:20 -0400 Subject: [PATCH 367/435] Bump cloud.google.com/go/bigquery from 1.57.0 to 1.57.1 in /sdks (#29286) Bumps [cloud.google.com/go/bigquery](https://github.com/googleapis/google-cloud-go) from 1.57.0 to 1.57.1. - [Release notes](https://github.com/googleapis/google-cloud-go/releases) - [Changelog](https://github.com/googleapis/google-cloud-go/blob/main/CHANGES.md) - [Commits](https://github.com/googleapis/google-cloud-go/compare/bigquery/v1.57.0...bigquery/v1.57.1) --- updated-dependencies: - dependency-name: cloud.google.com/go/bigquery dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] <support@github.com> Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 2 +- sdks/go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 34cb2ef2cd1f..5618da7ec14c 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -23,7 +23,7 @@ module github.com/apache/beam/sdks/v2 go 1.20 require ( - cloud.google.com/go/bigquery v1.57.0 + cloud.google.com/go/bigquery v1.57.1 cloud.google.com/go/bigtable v1.20.0 cloud.google.com/go/datastore v1.15.0 cloud.google.com/go/profiler v0.4.0 diff --git a/sdks/go.sum b/sdks/go.sum index 317eead7ba87..dd53a66dfa62 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -13,8 +13,8 @@ cloud.google.com/go v0.110.8/go.mod h1:Iz8AkXJf1qmxC3Oxoep8R1T36w8B92yU29PcBhHO5 cloud.google.com/go/bigquery v1.0.1/go.mod h1:i/xbL2UlR5RvWAURpBYZTtm/cXjCha9lbfbpx4poX+o= cloud.google.com/go/bigquery v1.3.0/go.mod h1:PjpwJnslEMmckchkHFfq+HTD2DmtT67aNFKH1/VBDHE= cloud.google.com/go/bigquery v1.4.0/go.mod h1:S8dzgnTigyfTmLBfrtrhyYhwRxG72rYxvftPBK2Dvzc= -cloud.google.com/go/bigquery v1.57.0 h1:1iVepOxF9XmHSmKePQG1FMMfv0Gw3f9+5zqRhTvxglw= -cloud.google.com/go/bigquery v1.57.0/go.mod h1:SmaaA61ZgwNhNDB+yKFyxHGgSMuxDJeB0T52SnGxnok= +cloud.google.com/go/bigquery v1.57.1 h1:FiULdbbzUxWD0Y4ZGPSVCDLvqRSyCIO6zKV7E2nf5uA= +cloud.google.com/go/bigquery v1.57.1/go.mod h1:iYzC0tGVWt1jqSzBHqCr3lrRn0u13E8e+AqowBsDgug= cloud.google.com/go/bigtable v1.20.0 h1:NqZC/WcesSn4O8L0I2JmuNsUigSyBQifVLYgM9LMQeQ= cloud.google.com/go/bigtable v1.20.0/go.mod h1:upJDn8frsjzpRMfybiWkD1PG6WCCL7CRl26MgVeoXY4= cloud.google.com/go/compute v1.23.1 h1:V97tBoDaZHb6leicZ1G6DLK2BAaZLJ/7+9BB/En3hR0= From f3905caeea448f06be2ab0a625d241edaf275f8a Mon Sep 17 00:00:00 2001 From: Yi Hu <yathu@google.com> Date: Fri, 3 Nov 2023 14:31:30 -0400 Subject: [PATCH 368/435] Setup Bigtable instance cleaner (#29279) * Setup Bigtable instance cleaner * Add task --- .test-infra/tools/build.gradle | 5 ++ .../tools/stale_cbt_instances_cleaner.sh | 58 +++++++++++++++++++ .../apache_beam/io/gcp/bigtableio_it_test.py | 14 +++-- 3 files changed, 72 insertions(+), 5 deletions(-) create mode 100755 .test-infra/tools/stale_cbt_instances_cleaner.sh diff --git a/.test-infra/tools/build.gradle b/.test-infra/tools/build.gradle index 5ed6ebfc18f6..048fc0a8d023 100644 --- a/.test-infra/tools/build.gradle +++ b/.test-infra/tools/build.gradle @@ -28,6 +28,10 @@ task removeStaleBqDatasets(type: Exec) { commandLine './stale_bq_datasets_cleaner.sh' } +task removeStaleCbtInstances(type: Exec) { + commandLine './stale_cbt_instances_cleaner.sh' +} + task removeStaleK8sWorkload(type: Exec) { commandLine './stale_k8s_workload_cleaner.sh' } @@ -35,5 +39,6 @@ task removeStaleK8sWorkload(type: Exec) { task cleanupOtherStaleResources { // declared as finalizedBy dependency so that other task continue even if one dep task fails finalizedBy tasks.removeStaleBqDatasets + finalizedBy tasks.removeStaleCbtInstances finalizedBy tasks.removeStaleK8sWorkload } diff --git a/.test-infra/tools/stale_cbt_instances_cleaner.sh b/.test-infra/tools/stale_cbt_instances_cleaner.sh new file mode 100755 index 000000000000..6996c9b05453 --- /dev/null +++ b/.test-infra/tools/stale_cbt_instances_cleaner.sh @@ -0,0 +1,58 @@ +#!/usr/bin/env bash +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +# Deletes stale and old BQ datasets that are left after tests. +# + +set -euo pipefail + +PROJECT=apache-beam-testing + +# get first 50 instances +CBT_INSTANCES=`cbt -project=$PROJECT listinstances | awk 'NR>2 {print $1} NR==52{exit}'` + +CLEANUP_INSTANCE_TEMPLATES=(bt-read-tests bt-write-xlang test[a-z]+) + +# A grace period of 5 days +GRACE_PERIOD=$((`date +%s` - 24 * 3600 * 5)) +# count number of failed api calls +declare -i failed_calls=0 + +for instance in ${CBT_INSTANCES[@]}; do + for template in ${CLEANUP_INSTANCE_TEMPLATES[@]}; do + pattern=$template-"([0-9]{8})"- + if [[ $instance =~ $pattern ]]; then + CREATE_DATE=${BASH_REMATCH[1]} + # skip if not a valid date + CREATED=`date -ju -f "%Y%m%d-%H%M%S" ${CREATE_DATE}-000000 +%s` || continue + if [[ $GRACE_PERIOD -gt $CREATED ]]; then + if cbt -project=$PROJECT deleteinstance $instance; then + echo "Deleted $instance (created $CREATE_DATE)" + else + failed_calls+=1 + fi + fi + break + fi + done +done + +# fail the script if failed_calls is nonzero +if [[ failed_calls -ne 0 ]]; then + echo "Failed delete $failed_calls instances" + exit 1 +fi diff --git a/sdks/python/apache_beam/io/gcp/bigtableio_it_test.py b/sdks/python/apache_beam/io/gcp/bigtableio_it_test.py index fdf049bba944..867dca9a5e7e 100644 --- a/sdks/python/apache_beam/io/gcp/bigtableio_it_test.py +++ b/sdks/python/apache_beam/io/gcp/bigtableio_it_test.py @@ -49,6 +49,13 @@ HttpError = None +def instance_prefix(instance): + datestr = "".join(filter(str.isdigit, str(datetime.utcnow().date()))) + instance_id = '%s-%s-%s' % (instance, datestr, secrets.token_hex(4)) + assert len(instance_id) < 34, "instance id length needs to be within [6, 33]" + return instance_id + + @pytest.mark.uses_gcp_java_expansion_service @pytest.mark.uses_transform_service @unittest.skipUnless( @@ -65,8 +72,7 @@ def setUp(self): self.project = self.test_pipeline.get_option('project') self.expansion_service = ('localhost:%s' % os.environ.get('EXPANSION_PORT')) - instance_id = '%s-%s-%s' % ( - self.INSTANCE, str(int(time.time())), secrets.token_hex(3)) + instance_id = instance_prefix(self.INSTANCE) self.client = client.Client(admin=True, project=self.project) # create cluster and instance @@ -160,9 +166,7 @@ def setUpClass(cls): cls.args = cls.test_pipeline.get_full_options_as_args() cls.expansion_service = ('localhost:%s' % os.environ.get('EXPANSION_PORT')) - timestr = "".join(filter(str.isdigit, str(datetime.utcnow().date()))) - # instance id length needs to be within [6, 33] - instance_id = '%s-%s-%s' % (cls.INSTANCE, timestr, secrets.token_hex(4)) + instance_id = instance_prefix(cls.INSTANCE) cls.client = client.Client(admin=True, project=cls.project) # create cluster and instance From 42bd0d089500ef57ac2c3a1aaa541bbf2924c8b4 Mon Sep 17 00:00:00 2001 From: Damon <damondouglas@users.noreply.github.com> Date: Fri, 3 Nov 2023 11:33:57 -0700 Subject: [PATCH 369/435] Add 10/1s refresher (#29295) --- .../echo-10-per-1s-quota/configmap.yaml | 30 ++++++++++ .../echo-10-per-1s-quota/deployment.yaml | 27 +++++++++ .../echo-10-per-1s-quota/kustomization.yaml | 34 +++++++++++ .../go/test/integration/echo/echo_test.go | 56 ++++++++++++++----- .../src/main/go/test/integration/vars.go | 12 ++-- 5 files changed, 140 insertions(+), 19 deletions(-) create mode 100644 .test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/echo-10-per-1s-quota/configmap.yaml create mode 100644 .test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/echo-10-per-1s-quota/deployment.yaml create mode 100644 .test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/echo-10-per-1s-quota/kustomization.yaml diff --git a/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/echo-10-per-1s-quota/configmap.yaml b/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/echo-10-per-1s-quota/configmap.yaml new file mode 100644 index 000000000000..d78c862c2afd --- /dev/null +++ b/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/echo-10-per-1s-quota/configmap.yaml @@ -0,0 +1,30 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# Configures patch for ../base/configmap.yaml +# See https://kubectl.docs.kubernetes.io/references/kustomize/kustomization/patches/ + +- op: replace + path: /metadata/labels/quota-id + value: echo-10-per-1s-quota +- op: replace + path: /data/QUOTA_ID + value: echo-10-per-1s-quota +- op: replace + path: /data/QUOTA_SIZE + value: "10" +- op: replace + path: /data/QUOTA_REFRESH_INTERVAL + value: 1s diff --git a/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/echo-10-per-1s-quota/deployment.yaml b/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/echo-10-per-1s-quota/deployment.yaml new file mode 100644 index 000000000000..37af48ec97d4 --- /dev/null +++ b/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/echo-10-per-1s-quota/deployment.yaml @@ -0,0 +1,27 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# Configures patch for ../base/deployment.yaml +# See https://kubectl.docs.kubernetes.io/references/kustomize/kustomization/patches/ + +- op: replace + path: /metadata/labels/quota-id + value: echo-10-per-1s-quota +- op: replace + path: /spec/selector/matchLabels/quota-id + value: echo-10-per-1s-quota +- op: replace + path: /spec/template/metadata/labels/quota-id + value: echo-10-per-1s-quota diff --git a/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/echo-10-per-1s-quota/kustomization.yaml b/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/echo-10-per-1s-quota/kustomization.yaml new file mode 100644 index 000000000000..d50698354435 --- /dev/null +++ b/.test-infra/mock-apis/infrastructure/kubernetes/refresher/overlays/echo-10-per-1s-quota/kustomization.yaml @@ -0,0 +1,34 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# Configures the overlay for .test-infra/mock-apis/infrastructure/kubernetes/refresher/base +# Using the Quota Id: +# echo-10-per-1s-quota + +resources: +- ../../base + +nameSuffix: -echo-10-per-1s-quota + +patches: +- path: configmap.yaml + target: + kind: ConfigMap + name: refresher + +- path: deployment.yaml + target: + kind: Deployment + name: refresher diff --git a/.test-infra/mock-apis/src/main/go/test/integration/echo/echo_test.go b/.test-infra/mock-apis/src/main/go/test/integration/echo/echo_test.go index 73ad597c7d34..102d8508d481 100644 --- a/.test-infra/mock-apis/src/main/go/test/integration/echo/echo_test.go +++ b/.test-infra/mock-apis/src/main/go/test/integration/echo/echo_test.go @@ -24,6 +24,7 @@ import ( "fmt" "net/http" "regexp" + "sync" "testing" "time" @@ -45,6 +46,8 @@ const ( shouldExceedQuotaId = "echo-should-exceed-quota" shouldNeverExceedQuotaId = "echo-should-never-exceed-quota" shouldNotExistId = "should-not-exist" + refresh10Per1s = "echo-10-per-1s-quota" + defaultNumCalls = 3 ) var ( @@ -60,11 +63,12 @@ func TestEcho(t *testing.T) { payload := []byte("payload") for _, tt := range []struct { - tag string - quotaId string - client echov1.EchoServiceClient - want *echov1.EchoResponse - wantErr error + tag string + quotaId string + client echov1.EchoServiceClient + want *echov1.EchoResponse + numCalls int + wantErr error }{ { tag: "http", @@ -108,11 +112,25 @@ func TestEcho(t *testing.T) { Payload: payload, }, }, + { + numCalls: 20, + tag: "grpc", + quotaId: refresh10Per1s, + client: withGrpc(t), + wantErr: status.Error(codes.ResourceExhausted, "error: resource exhausted for: echo-10-per-1s-quota"), + }, } { t.Run(fmt.Sprintf("%s/%s", tt.quotaId, tt.tag), func(t *testing.T) { ctx, cancel := withTimeout() defer cancel() + if tt.numCalls == 0 { + tt.numCalls = defaultNumCalls + } + + wg := sync.WaitGroup{} + wg.Add(tt.numCalls) + req := &echov1.EchoRequest{ Id: tt.quotaId, Payload: payload, @@ -121,16 +139,21 @@ func TestEcho(t *testing.T) { var resps []*echov1.EchoResponse var errs []error - for i := 0; i < 3; i++ { - resp, err := tt.client.Echo(ctx, req) - if err != nil { - errs = append(errs, err) - } - if resp != nil { - resps = append(resps, resp) - } + for i := 0; i < tt.numCalls; i++ { + go func() { + resp, err := tt.client.Echo(ctx, req) + if err != nil { + errs = append(errs, err) + } + if resp != nil { + resps = append(resps, resp) + } + wg.Done() + }() } + wg.Wait() + if tt.wantErr != nil && len(errs) == 0 { t.Errorf("Echo(%+v) err = nil, wantErr = %v", req, tt.wantErr) return @@ -163,6 +186,10 @@ func withGrpc(t *testing.T) echov1.EchoServiceClient { ctx, cancel := withTimeout() defer cancel() + if *integration.GRPCServiceEndpoint == "" { + t.Fatalf("missing flag: -%s", integration.GrpcServiceEndpointFlag) + } + conn, err := grpc.DialContext(ctx, *integration.GRPCServiceEndpoint, grpcOpts...) if err != nil { t.Fatalf("DialContext(%s) err %v", *integration.GRPCServiceEndpoint, err) @@ -205,6 +232,9 @@ func (h *httpCaller) Echo(ctx context.Context, in *echov1.EchoRequest, _ ...grpc } func withHttp(t *testing.T) echov1.EchoServiceClient { + if *integration.HTTPServiceEndpoint == "" { + t.Fatalf("missing flag: -%s", integration.HttpServiceEndpointFlag) + } p := regexp.MustCompile(`^http://`) rawUrl := fmt.Sprint(*integration.HTTPServiceEndpoint, echo.PathAlias) if !p.MatchString(rawUrl) { diff --git a/.test-infra/mock-apis/src/main/go/test/integration/vars.go b/.test-infra/mock-apis/src/main/go/test/integration/vars.go index 92ba445fdfa3..a4bd9bb60e2e 100644 --- a/.test-infra/mock-apis/src/main/go/test/integration/vars.go +++ b/.test-infra/mock-apis/src/main/go/test/integration/vars.go @@ -21,8 +21,8 @@ import ( ) const ( - grpcServiceEndpointFlag = "grpc_service_endpoint" - httpServiceEndpointFlag = "http_service_endpoint" + GrpcServiceEndpointFlag = "grpc_service_endpoint" + HttpServiceEndpointFlag = "http_service_endpoint" moreInfoUrl = "https://github.com/apache/beam/tree/master/.test-infra/mock-apis#writing-integration-tests" ) @@ -31,8 +31,8 @@ var ( moreInfo = fmt.Sprintf("See %s for more information on how to get the relevant value for your test.", moreInfoUrl) requiredFlags = []string{ - grpcServiceEndpointFlag, - httpServiceEndpointFlag, + GrpcServiceEndpointFlag, + HttpServiceEndpointFlag, } ) @@ -40,10 +40,10 @@ var ( // go test ./src/main/go/test/integration/... var ( // GRPCServiceEndpoint is the address of the deployed service. - GRPCServiceEndpoint = flag.String(grpcServiceEndpointFlag, "", + GRPCServiceEndpoint = flag.String(GrpcServiceEndpointFlag, "", "The endpoint to target gRPC calls to a service. "+moreInfo) // HTTPServiceEndpoint is the address of the deployed service. - HTTPServiceEndpoint = flag.String(httpServiceEndpointFlag, "", + HTTPServiceEndpoint = flag.String(HttpServiceEndpointFlag, "", "The endpoint to target HTTP calls to a service. "+moreInfo) ) From 42b04bdfef911dce48a2e2208881774236ebdf61 Mon Sep 17 00:00:00 2001 From: Yi Hu <yathu@google.com> Date: Fri, 3 Nov 2023 14:36:15 -0400 Subject: [PATCH 370/435] Fix check permission for GHA XVR workflows (#29291) * Fix check permission for GHA XVR workflows * Add missing PostCommit_Singlestore_IOIT to change.md * Fix readme copy-paste leftover --- .github/workflows/README.md | 1 + .github/workflows/beam_CloudML_Benchmarks_Dataflow.yml | 2 +- .github/workflows/beam_PostCommit_XVR_Direct.yml | 4 ++-- .github/workflows/beam_PostCommit_XVR_Flink.yml | 4 ++-- .../beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml | 4 ++-- .github/workflows/beam_PostCommit_XVR_Samza.yml | 4 ++-- .github/workflows/beam_PostCommit_XVR_Spark3.yml | 4 ++-- 7 files changed, 12 insertions(+), 11 deletions(-) diff --git a/.github/workflows/README.md b/.github/workflows/README.md index e8ec3ce18323..5afbae8677d3 100644 --- a/.github/workflows/README.md +++ b/.github/workflows/README.md @@ -294,6 +294,7 @@ Please note that jobs with matrix need to have matrix element in the comment. Ex | [ PostCommit Java Nexmark Spark ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Spark.yml) | N/A |`Run Spark Runner Nexmark Tests`| [![.github/workflows/beam_PostCommit_Java_Nexmark_Spark.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Spark.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Spark.yml?query=event%3Aschedule) | | [ PostCommit Java PVR Flink Streaming ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml) | N/A |`Run Java Flink PortableValidatesRunner Streaming`| [![.github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml?query=event%3Aschedule) | | [ PostCommit Java PVR Samza ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Samza.yml) | N/A |`Run Java Samza PortableValidatesRunner`| [![.github/workflows/beam_PostCommit_Java_PVR_Samza.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Samza.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Samza.yml?query=event%3Aschedule) | +| [ PostCommit Java SingleStoreIO IT ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml) | N/A |`Run Java SingleStoreIO_IT`| [![.github/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml?query=event%3Aschedule) | | [ PostCommit Java PVR Spark3 Streaming ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml) | N/A |`Run Java Spark v3 PortableValidatesRunner Streaming`| [![.github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml?query=event%3Aschedule) | | [ PostCommit Java PVR Spark Batch ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml) | N/A |`Run Java Spark PortableValidatesRunner Batch`| [![.github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml?query=event%3Aschedule) | | [ PostCommit Java Sickbay ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Sickbay.yml) | N/A |`Run Java Sickbay`| [![.github/workflows/beam_PostCommit_Java_Sickbay.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Sickbay.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Sickbay.yml?query=event%3Aschedule) | diff --git a/.github/workflows/beam_CloudML_Benchmarks_Dataflow.yml b/.github/workflows/beam_CloudML_Benchmarks_Dataflow.yml index 65e388c8a729..4902062d3928 100644 --- a/.github/workflows/beam_CloudML_Benchmarks_Dataflow.yml +++ b/.github/workflows/beam_CloudML_Benchmarks_Dataflow.yml @@ -40,7 +40,7 @@ permissions: # This allows a subsequently queued workflow run to interrupt previous runs concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' cancel-in-progress: true env: diff --git a/.github/workflows/beam_PostCommit_XVR_Direct.yml b/.github/workflows/beam_PostCommit_XVR_Direct.yml index dcb6d3b3b0b8..ee90f9176f6d 100644 --- a/.github/workflows/beam_PostCommit_XVR_Direct.yml +++ b/.github/workflows/beam_PostCommit_XVR_Direct.yml @@ -25,8 +25,8 @@ on: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event permissions: actions: write - pull-requests: read - checks: read + pull-requests: write + checks: write contents: read deployments: read id-token: none diff --git a/.github/workflows/beam_PostCommit_XVR_Flink.yml b/.github/workflows/beam_PostCommit_XVR_Flink.yml index abb77eeee970..75d3353e8a15 100644 --- a/.github/workflows/beam_PostCommit_XVR_Flink.yml +++ b/.github/workflows/beam_PostCommit_XVR_Flink.yml @@ -25,8 +25,8 @@ on: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event permissions: actions: write - pull-requests: read - checks: read + pull-requests: write + checks: write contents: read deployments: read id-token: none diff --git a/.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml b/.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml index 0b81a444d58b..113c51626078 100644 --- a/.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml @@ -25,8 +25,8 @@ on: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event permissions: actions: write - pull-requests: read - checks: read + pull-requests: write + checks: write contents: read deployments: read id-token: none diff --git a/.github/workflows/beam_PostCommit_XVR_Samza.yml b/.github/workflows/beam_PostCommit_XVR_Samza.yml index 239d3c849b27..b05b588e0cf8 100644 --- a/.github/workflows/beam_PostCommit_XVR_Samza.yml +++ b/.github/workflows/beam_PostCommit_XVR_Samza.yml @@ -25,8 +25,8 @@ on: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event permissions: actions: write - pull-requests: read - checks: read + pull-requests: write + checks: write contents: read deployments: read id-token: none diff --git a/.github/workflows/beam_PostCommit_XVR_Spark3.yml b/.github/workflows/beam_PostCommit_XVR_Spark3.yml index 720900b7a081..0742196a6960 100644 --- a/.github/workflows/beam_PostCommit_XVR_Spark3.yml +++ b/.github/workflows/beam_PostCommit_XVR_Spark3.yml @@ -25,8 +25,8 @@ on: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event permissions: actions: write - pull-requests: read - checks: read + pull-requests: write + checks: write contents: read deployments: read id-token: none From cc2e63ea75a6ff31fc6c97c5feb63191e81ebf53 Mon Sep 17 00:00:00 2001 From: Ritesh Ghorse <riteshghorse@gmail.com> Date: Fri, 3 Nov 2023 14:54:54 -0400 Subject: [PATCH 371/435] [RRIO][Python] Add python grpc binding for mock API (#29129) * add python grpc binding for mock api * add licenses * update python version for buf plugin * add poetry for python dependencies * add license * del author,desc * update package for pyproject * update python path --- .test-infra/mock-apis/README.md | 1 + .test-infra/mock-apis/buf.gen.yaml | 6 +- .test-infra/mock-apis/poetry.lock | 236 ++++++++++++++++++ .test-infra/mock-apis/pyproject.toml | 38 +++ .../src/main/python/proto/echo/v1/echo_pb2.py | 47 ++++ .../python/proto/echo/v1/echo_pb2_grpc.py | 87 +++++++ 6 files changed, 414 insertions(+), 1 deletion(-) create mode 100644 .test-infra/mock-apis/poetry.lock create mode 100644 .test-infra/mock-apis/pyproject.toml create mode 100644 .test-infra/mock-apis/src/main/python/proto/echo/v1/echo_pb2.py create mode 100644 .test-infra/mock-apis/src/main/python/proto/echo/v1/echo_pb2_grpc.py diff --git a/.test-infra/mock-apis/README.md b/.test-infra/mock-apis/README.md index 9c4911a0d636..ec94eb45a199 100644 --- a/.test-infra/mock-apis/README.md +++ b/.test-infra/mock-apis/README.md @@ -62,6 +62,7 @@ TODO: See https://github.com/apache/beam/issues/28859 | [go](https://go.dev) | For making code changes in this directory. See [go.mod](go.mod) for required version. | | [buf](https://github.com/bufbuild/buf#installation) | Optional for when making changes to proto. | | [ko](https://ko.build/install/) | To easily build Go container images. | +| [poetry](https://python-poetry.org/) | To manage python dependencies. | # Testing diff --git a/.test-infra/mock-apis/buf.gen.yaml b/.test-infra/mock-apis/buf.gen.yaml index 31e57ff2da1e..e5f6e51c14d8 100644 --- a/.test-infra/mock-apis/buf.gen.yaml +++ b/.test-infra/mock-apis/buf.gen.yaml @@ -37,4 +37,8 @@ plugins: - name: java out: src/main/java - name: grpc-java - out: src/main/java \ No newline at end of file + out: src/main/java +- plugin: buf.build/protocolbuffers/python:v24.4 + out: src/main/python +- plugin: buf.build/grpc/python:v1.59.1 + out: src/main/python \ No newline at end of file diff --git a/.test-infra/mock-apis/poetry.lock b/.test-infra/mock-apis/poetry.lock new file mode 100644 index 000000000000..e9bcdbb4750d --- /dev/null +++ b/.test-infra/mock-apis/poetry.lock @@ -0,0 +1,236 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# This file is automatically @generated by Poetry 1.7.0 and should not be changed by hand. + +[[package]] +name = "beautifulsoup4" +version = "4.12.2" +description = "Screen-scraping library" +optional = false +python-versions = ">=3.6.0" +files = [ + {file = "beautifulsoup4-4.12.2-py3-none-any.whl", hash = "sha256:bd2520ca0d9d7d12694a53d44ac482d181b4ec1888909b035a3dbf40d0f57d4a"}, + {file = "beautifulsoup4-4.12.2.tar.gz", hash = "sha256:492bbc69dca35d12daac71c4db1bfff0c876c00ef4a2ffacce226d4638eb72da"}, +] + +[package.dependencies] +soupsieve = ">1.2" + +[package.extras] +html5lib = ["html5lib"] +lxml = ["lxml"] + +[[package]] +name = "google" +version = "3.0.0" +description = "Python bindings to the Google search engine." +optional = false +python-versions = "*" +files = [ + {file = "google-3.0.0-py2.py3-none-any.whl", hash = "sha256:889cf695f84e4ae2c55fbc0cfdaf4c1e729417fa52ab1db0485202ba173e4935"}, + {file = "google-3.0.0.tar.gz", hash = "sha256:143530122ee5130509ad5e989f0512f7cb218b2d4eddbafbad40fd10e8d8ccbe"}, +] + +[package.dependencies] +beautifulsoup4 = "*" + +[[package]] +name = "grpcio" +version = "1.59.2" +description = "HTTP/2-based RPC framework" +optional = false +python-versions = ">=3.7" +files = [ + {file = "grpcio-1.59.2-cp310-cp310-linux_armv7l.whl", hash = "sha256:d2fa68a96a30dd240be80bbad838a0ac81a61770611ff7952b889485970c4c71"}, + {file = "grpcio-1.59.2-cp310-cp310-macosx_12_0_universal2.whl", hash = "sha256:cf0dead5a2c5a3347af2cfec7131d4f2a2e03c934af28989c9078f8241a491fa"}, + {file = "grpcio-1.59.2-cp310-cp310-manylinux_2_17_aarch64.whl", hash = "sha256:e420ced29b5904cdf9ee5545e23f9406189d8acb6750916c2db4793dada065c6"}, + {file = "grpcio-1.59.2-cp310-cp310-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:2b230028a008ae1d0f430acb227d323ff8a619017415cf334c38b457f814119f"}, + {file = "grpcio-1.59.2-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:0a4a3833c0e067f3558538727235cd8a49709bff1003200bbdefa2f09334e4b1"}, + {file = "grpcio-1.59.2-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:6b25ed37c27e652db01be341af93fbcea03d296c024d8a0e680017a268eb85dd"}, + {file = "grpcio-1.59.2-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:73abb8584b0cf74d37f5ef61c10722adc7275502ab71789a8fe3cb7ef04cf6e2"}, + {file = "grpcio-1.59.2-cp310-cp310-win32.whl", hash = "sha256:d6f70406695e3220f09cd7a2f879333279d91aa4a8a1d34303b56d61a8180137"}, + {file = "grpcio-1.59.2-cp310-cp310-win_amd64.whl", hash = "sha256:3c61d641d4f409c5ae46bfdd89ea42ce5ea233dcf69e74ce9ba32b503c727e29"}, + {file = "grpcio-1.59.2-cp311-cp311-linux_armv7l.whl", hash = "sha256:3059668df17627f0e0fa680e9ef8c995c946c792612e9518f5cc1503be14e90b"}, + {file = "grpcio-1.59.2-cp311-cp311-macosx_10_10_universal2.whl", hash = "sha256:72ca2399097c0b758198f2ff30f7178d680de8a5cfcf3d9b73a63cf87455532e"}, + {file = "grpcio-1.59.2-cp311-cp311-manylinux_2_17_aarch64.whl", hash = "sha256:c978f864b35f2261e0819f5cd88b9830b04dc51bcf055aac3c601e525a10d2ba"}, + {file = "grpcio-1.59.2-cp311-cp311-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:9411e24328a2302e279e70cae6e479f1fddde79629fcb14e03e6d94b3956eabf"}, + {file = "grpcio-1.59.2-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:bb7e0fe6ad73b7f06d7e2b689c19a71cf5cc48f0c2bf8608469e51ffe0bd2867"}, + {file = "grpcio-1.59.2-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:c2504eed520958a5b77cc99458297cb7906308cb92327f35fb7fbbad4e9b2188"}, + {file = "grpcio-1.59.2-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:2171c39f355ba5b551c5d5928d65aa6c69807fae195b86ef4a7d125bcdb860a9"}, + {file = "grpcio-1.59.2-cp311-cp311-win32.whl", hash = "sha256:d2794f0e68b3085d99b4f6ff9c089f6fdd02b32b9d3efdfbb55beac1bf22d516"}, + {file = "grpcio-1.59.2-cp311-cp311-win_amd64.whl", hash = "sha256:2067274c88bc6de89c278a672a652b4247d088811ece781a4858b09bdf8448e3"}, + {file = "grpcio-1.59.2-cp312-cp312-linux_armv7l.whl", hash = "sha256:535561990e075fa6bd4b16c4c3c1096b9581b7bb35d96fac4650f1181e428268"}, + {file = "grpcio-1.59.2-cp312-cp312-macosx_10_10_universal2.whl", hash = "sha256:a213acfbf186b9f35803b52e4ca9addb153fc0b67f82a48f961be7000ecf6721"}, + {file = "grpcio-1.59.2-cp312-cp312-manylinux_2_17_aarch64.whl", hash = "sha256:6959fb07e8351e20501ffb8cc4074c39a0b7ef123e1c850a7f8f3afdc3a3da01"}, + {file = "grpcio-1.59.2-cp312-cp312-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:e82c5cf1495244adf5252f925ac5932e5fd288b3e5ab6b70bec5593074b7236c"}, + {file = "grpcio-1.59.2-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:023088764012411affe7db183d1ada3ad9daf2e23ddc719ff46d7061de661340"}, + {file = "grpcio-1.59.2-cp312-cp312-musllinux_1_1_i686.whl", hash = "sha256:da2d94c15f88cd40d7e67f7919d4f60110d2b9d5b1e08cf354c2be773ab13479"}, + {file = "grpcio-1.59.2-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:6009386a2df66159f64ac9f20425ae25229b29b9dd0e1d3dd60043f037e2ad7e"}, + {file = "grpcio-1.59.2-cp312-cp312-win32.whl", hash = "sha256:75c6ecb70e809cf1504465174343113f51f24bc61e22a80ae1c859f3f7034c6d"}, + {file = "grpcio-1.59.2-cp312-cp312-win_amd64.whl", hash = "sha256:cbe946b3e6e60a7b4618f091e62a029cb082b109a9d6b53962dd305087c6e4fd"}, + {file = "grpcio-1.59.2-cp37-cp37m-linux_armv7l.whl", hash = "sha256:f8753a6c88d1d0ba64302309eecf20f70d2770f65ca02d83c2452279085bfcd3"}, + {file = "grpcio-1.59.2-cp37-cp37m-macosx_10_10_universal2.whl", hash = "sha256:f1ef0d39bc1feb420caf549b3c657c871cad4ebbcf0580c4d03816b0590de0cf"}, + {file = "grpcio-1.59.2-cp37-cp37m-manylinux_2_17_aarch64.whl", hash = "sha256:4c93f4abbb54321ee6471e04a00139c80c754eda51064187963ddf98f5cf36a4"}, + {file = "grpcio-1.59.2-cp37-cp37m-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:08d77e682f2bf730a4961eea330e56d2f423c6a9b91ca222e5b1eb24a357b19f"}, + {file = "grpcio-1.59.2-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:1ff16d68bf453275466a9a46739061a63584d92f18a0f5b33d19fc97eb69867c"}, + {file = "grpcio-1.59.2-cp37-cp37m-musllinux_1_1_i686.whl", hash = "sha256:4abb717e320e74959517dc8e84a9f48fbe90e9abe19c248541e9418b1ce60acd"}, + {file = "grpcio-1.59.2-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:36f53c2b3449c015880e7d55a89c992c357f176327b0d2873cdaaf9628a37c69"}, + {file = "grpcio-1.59.2-cp37-cp37m-win_amd64.whl", hash = "sha256:cc3e4cd087f07758b16bef8f31d88dbb1b5da5671d2f03685ab52dece3d7a16e"}, + {file = "grpcio-1.59.2-cp38-cp38-linux_armv7l.whl", hash = "sha256:27f879ae604a7fcf371e59fba6f3ff4635a4c2a64768bd83ff0cac503142fef4"}, + {file = "grpcio-1.59.2-cp38-cp38-macosx_10_10_universal2.whl", hash = "sha256:7cf05053242f61ba94014dd3a986e11a083400a32664058f80bf4cf817c0b3a1"}, + {file = "grpcio-1.59.2-cp38-cp38-manylinux_2_17_aarch64.whl", hash = "sha256:e1727c1c0e394096bb9af185c6923e8ea55a5095b8af44f06903bcc0e06800a2"}, + {file = "grpcio-1.59.2-cp38-cp38-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:5d573e70a6fe77555fb6143c12d3a7d3fa306632a3034b4e7c59ca09721546f8"}, + {file = "grpcio-1.59.2-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:31176aa88f36020055ace9adff2405a33c8bdbfa72a9c4980e25d91b2f196873"}, + {file = "grpcio-1.59.2-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:11168ef43e4a43ff1b1a65859f3e0ef1a173e277349e7fb16923ff108160a8cd"}, + {file = "grpcio-1.59.2-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:53c9aa5ddd6857c0a1cd0287225a2a25873a8e09727c2e95c4aebb1be83a766a"}, + {file = "grpcio-1.59.2-cp38-cp38-win32.whl", hash = "sha256:3b4368b33908f683a363f376dfb747d40af3463a6e5044afee07cf9436addf96"}, + {file = "grpcio-1.59.2-cp38-cp38-win_amd64.whl", hash = "sha256:0a754aff9e3af63bdc4c75c234b86b9d14e14a28a30c4e324aed1a9b873d755f"}, + {file = "grpcio-1.59.2-cp39-cp39-linux_armv7l.whl", hash = "sha256:1f9524d1d701e399462d2c90ba7c193e49d1711cf429c0d3d97c966856e03d00"}, + {file = "grpcio-1.59.2-cp39-cp39-macosx_10_10_universal2.whl", hash = "sha256:f93dbf58f03146164048be5426ffde298b237a5e059144847e4940f5b80172c3"}, + {file = "grpcio-1.59.2-cp39-cp39-manylinux_2_17_aarch64.whl", hash = "sha256:6da6dea3a1bacf99b3c2187e296db9a83029ed9c38fd4c52b7c9b7326d13c828"}, + {file = "grpcio-1.59.2-cp39-cp39-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:c5f09cffa619adfb44799fa4a81c2a1ad77c887187613fb0a8f201ab38d89ba1"}, + {file = "grpcio-1.59.2-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:c35aa9657f5d5116d23b934568e0956bd50c615127810fffe3ac356a914c176a"}, + {file = "grpcio-1.59.2-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:74100fecaec8a535e380cf5f2fb556ff84957d481c13e54051c52e5baac70541"}, + {file = "grpcio-1.59.2-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:128e20f57c5f27cb0157e73756d1586b83c1b513ebecc83ea0ac37e4b0e4e758"}, + {file = "grpcio-1.59.2-cp39-cp39-win32.whl", hash = "sha256:686e975a5d16602dc0982c7c703948d17184bd1397e16c8ee03511ecb8c4cdda"}, + {file = "grpcio-1.59.2-cp39-cp39-win_amd64.whl", hash = "sha256:242adc47725b9a499ee77c6a2e36688fa6c96484611f33b1be4c57ab075a92dd"}, + {file = "grpcio-1.59.2.tar.gz", hash = "sha256:d8f9cd4ad1be90b0cf350a2f04a38a36e44a026cac1e036ac593dc48efe91d52"}, +] + +[package.extras] +protobuf = ["grpcio-tools (>=1.59.2)"] + +[[package]] +name = "grpcio-tools" +version = "1.59.2" +description = "Protobuf code generator for gRPC" +optional = false +python-versions = ">=3.7" +files = [ + {file = "grpcio-tools-1.59.2.tar.gz", hash = "sha256:75905266cf90f1866b322575c2edcd4b36532c33fc512bb1b380dc58d84b1030"}, + {file = "grpcio_tools-1.59.2-cp310-cp310-linux_armv7l.whl", hash = "sha256:9b2885c0e2c9a97bde33497a919032afbd8b5c6dc2f8d4dd4198e77226e0de05"}, + {file = "grpcio_tools-1.59.2-cp310-cp310-macosx_12_0_universal2.whl", hash = "sha256:2f410375830a9bb7140a07da4d75bf380e0958377bed50d77d1dae302de4314e"}, + {file = "grpcio_tools-1.59.2-cp310-cp310-manylinux_2_17_aarch64.whl", hash = "sha256:e21fc172522d2dda815223a359b2aca9bc317a1b5e5dea5a58cd5079333af133"}, + {file = "grpcio_tools-1.59.2-cp310-cp310-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:072a7ce979ea4f7579c3c99fcbde3d1882c3d1942a3b51d159f67af83b714cd8"}, + {file = "grpcio_tools-1.59.2-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:b38f8edb2909702c2478b52f6213982c21e4f66f739ac953b91f97863ba2c06a"}, + {file = "grpcio_tools-1.59.2-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:12fdee2de80d83eadb1294e0f8a0cb6cefcd2e4988ed680038ab09cd04361ee4"}, + {file = "grpcio_tools-1.59.2-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:a3cb707da722a0b6c4021fc2cc1c005a8d4037d8ad0252f93df318b9b8a6b4f3"}, + {file = "grpcio_tools-1.59.2-cp310-cp310-win32.whl", hash = "sha256:ec2fbb02ebb9f2ae1b1c69cccf913dee8c41f5acad94014d3ce11b53720376e3"}, + {file = "grpcio_tools-1.59.2-cp310-cp310-win_amd64.whl", hash = "sha256:b0dc271a200dbab6547b2c73fcbdb7efe94c31cb633aa20d073f7cf4493493e1"}, + {file = "grpcio_tools-1.59.2-cp311-cp311-linux_armv7l.whl", hash = "sha256:d634b65cc8ee769edccf1647d8a16861a27e0d8cbd787c711168d2c5e9bddbd1"}, + {file = "grpcio_tools-1.59.2-cp311-cp311-macosx_10_10_universal2.whl", hash = "sha256:b0b712acec00a9cbc2204c271d638062a2cb8ce74f25d158b023ff6e93182659"}, + {file = "grpcio_tools-1.59.2-cp311-cp311-manylinux_2_17_aarch64.whl", hash = "sha256:dd5c78f8e7c6e721b9009c92481a0e3b30a9926ef721120723a03b8a34a34fb9"}, + {file = "grpcio_tools-1.59.2-cp311-cp311-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:724f4f0eecc17fa66216eebfff145631070f04ed7fb4ddf7a7d1c4f954ecc2a1"}, + {file = "grpcio_tools-1.59.2-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:77ec33ddee691e60511e2a7c793aad4cf172ae20e08d95c786cbba395f6203a7"}, + {file = "grpcio_tools-1.59.2-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:fa1b9dee7811fad081816e884d063c4dd4946dba61aa54243b4c76c311090c48"}, + {file = "grpcio_tools-1.59.2-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:ba8dba19e7b2b6f7369004533866f222ba483b9e14d2d152ecf9339c0df1283a"}, + {file = "grpcio_tools-1.59.2-cp311-cp311-win32.whl", hash = "sha256:df35d145bc2f6e5f57b74cb69f66526675a5f2dcf7d54617ce0deff0c82cca0a"}, + {file = "grpcio_tools-1.59.2-cp311-cp311-win_amd64.whl", hash = "sha256:99ddc0f5304071a355c261ae49ea5d29b9e9b6dcf422dfc55ada70a243e27e8f"}, + {file = "grpcio_tools-1.59.2-cp312-cp312-linux_armv7l.whl", hash = "sha256:670f5889853215999eb3511a623dd7dff01b1ce1a64610d13366e0fd337f8c79"}, + {file = "grpcio_tools-1.59.2-cp312-cp312-macosx_10_10_universal2.whl", hash = "sha256:1e949e66d4555ce319fd7acef90df625138078d8729c4dc6f6a9f05925034433"}, + {file = "grpcio_tools-1.59.2-cp312-cp312-manylinux_2_17_aarch64.whl", hash = "sha256:09d809ca88999b2578119683f9f0f6a9b42de95ea21550852114a1540b6a642c"}, + {file = "grpcio_tools-1.59.2-cp312-cp312-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:db0925545180223fabd6da9b34513efac83aa16673ef8b1cb0cc678e8cf0923c"}, + {file = "grpcio_tools-1.59.2-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:a2ccb59dfbf2ebd668a5a7c4b7bb2b859859641d2b199114b557cd045aac6102"}, + {file = "grpcio_tools-1.59.2-cp312-cp312-musllinux_1_1_i686.whl", hash = "sha256:12cc7698fad48866f68fdef831685cb31ef5814ac605d248c4e5fc964a6fb3f6"}, + {file = "grpcio_tools-1.59.2-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:55c401599d5093c4cfa83b8f0ee9757b4d6d3029b10bd67be2cffeada7a44961"}, + {file = "grpcio_tools-1.59.2-cp312-cp312-win32.whl", hash = "sha256:896f5cdf58f658025a4f7e4ea96c81183b4b6a4b1b4d92ae66d112ac91f062f1"}, + {file = "grpcio_tools-1.59.2-cp312-cp312-win_amd64.whl", hash = "sha256:b53db1523015a3acda75722357df6c94afae37f6023800c608e09a5c05393804"}, + {file = "grpcio_tools-1.59.2-cp37-cp37m-linux_armv7l.whl", hash = "sha256:d08b398509ea4d544bcecddd9a21f59dc556396916c3915904cac206af2db72b"}, + {file = "grpcio_tools-1.59.2-cp37-cp37m-macosx_10_10_universal2.whl", hash = "sha256:09749e832e06493841000275248b031f7154665900d1e1b0e42fc17a64bf904d"}, + {file = "grpcio_tools-1.59.2-cp37-cp37m-manylinux_2_17_aarch64.whl", hash = "sha256:e972746000aa192521715f776fab617a3437bed29e90fe0e0fd0d0d6f498d7d4"}, + {file = "grpcio_tools-1.59.2-cp37-cp37m-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:cbeeb3d8ec4cb25c92e17bfbdcef3c3669e85c5ee787a6e581cb942bc0ae2b88"}, + {file = "grpcio_tools-1.59.2-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:ed8e6632d8d839456332d97b96db10bd2dbf3078e728d063394ac2d54597ad80"}, + {file = "grpcio_tools-1.59.2-cp37-cp37m-musllinux_1_1_i686.whl", hash = "sha256:531f87c8e884c6a2e58f040039dfbfe997a4e33baa58f7c7d9993db37b1f5ad0"}, + {file = "grpcio_tools-1.59.2-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:feca316e17cfead823af6eae0fc20c0d5299a94d71cfb7531a0e92d050a5fb2f"}, + {file = "grpcio_tools-1.59.2-cp37-cp37m-win_amd64.whl", hash = "sha256:41b5dd6a06c2563ac3b3adda6d875b15e63eb7b1629e85fc9af608c3a76c4c82"}, + {file = "grpcio_tools-1.59.2-cp38-cp38-linux_armv7l.whl", hash = "sha256:7ec536cdae870a74080c665cfb1dca8d0784a931aa3c26376ef971a3a51b59d4"}, + {file = "grpcio_tools-1.59.2-cp38-cp38-macosx_10_10_universal2.whl", hash = "sha256:9c106ebbed0db446f59f0efe5c3fce33a0a21bf75b392966585e4b5934891b92"}, + {file = "grpcio_tools-1.59.2-cp38-cp38-manylinux_2_17_aarch64.whl", hash = "sha256:32141ef309543a446337e934f0b7a2565a6fca890ff4e543630a09ef72c8d00b"}, + {file = "grpcio_tools-1.59.2-cp38-cp38-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:5f2ce5ecd63c492949b03af73b1dd6d502c567cc2f9c2057137e518b0c702a01"}, + {file = "grpcio_tools-1.59.2-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:2a9ce2a209871ed1c5ae2229e6f4f5a3ea96d83b7871df5d9773d72a72545683"}, + {file = "grpcio_tools-1.59.2-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:7f0e26af7c07bfa906c91ca9f5932514928a7f032f5f20aecad6b5541037de7e"}, + {file = "grpcio_tools-1.59.2-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:48782727c5cff8b8c96e028a8a58614ff6a37eadc0db85866516210c7aafe9ae"}, + {file = "grpcio_tools-1.59.2-cp38-cp38-win32.whl", hash = "sha256:4a1810bc5de51cc162a19ed3c11da8ddc64d8cfcba049ef337c20fcb397f048b"}, + {file = "grpcio_tools-1.59.2-cp38-cp38-win_amd64.whl", hash = "sha256:3cf9949a2aadcece3c1e0dd59249aea53dbfc8cc94f7d707797acd67cf6cf931"}, + {file = "grpcio_tools-1.59.2-cp39-cp39-linux_armv7l.whl", hash = "sha256:f52e0ce8f2dcf1f160c847304016c446075a83ab925d98933d4681bfa8af2962"}, + {file = "grpcio_tools-1.59.2-cp39-cp39-macosx_10_10_universal2.whl", hash = "sha256:eb597d6bf9f5bfa54d00546e828f0d4e2c69250d1bc17c27903c0c7b66372135"}, + {file = "grpcio_tools-1.59.2-cp39-cp39-manylinux_2_17_aarch64.whl", hash = "sha256:17ef468836d7cf0b2419f4d5c7ac84ec2d598a1ae410773585313edacf7c393e"}, + {file = "grpcio_tools-1.59.2-cp39-cp39-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:dee5f7e7a56177234e61a483c70ca2ae34e73128372c801bb7039993870889f1"}, + {file = "grpcio_tools-1.59.2-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:f50ff312b88918c5a6461e45c5e03869749a066b1c24a7327e8e13e117efe4fc"}, + {file = "grpcio_tools-1.59.2-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:a85da4200295ee17e3c1ae068189a43844420ed7e9d531a042440f52de486dfb"}, + {file = "grpcio_tools-1.59.2-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:f518f22a3082de00f0d7a216e96366a87e6973111085ba1603c3bfa7dba2e728"}, + {file = "grpcio_tools-1.59.2-cp39-cp39-win32.whl", hash = "sha256:6e735a26e8ea8bb89dc69343d1d00ea607449c6d81e21f339ee118562f3d1931"}, + {file = "grpcio_tools-1.59.2-cp39-cp39-win_amd64.whl", hash = "sha256:3491cb69c909d586c23d7e6d0ac87844ca22f496f505ce429c0d3301234f2cf3"}, +] + +[package.dependencies] +grpcio = ">=1.59.2" +protobuf = ">=4.21.6,<5.0dev" +setuptools = "*" + +[[package]] +name = "protobuf" +version = "4.25.0" +description = "" +optional = false +python-versions = ">=3.8" +files = [ + {file = "protobuf-4.25.0-cp310-abi3-win32.whl", hash = "sha256:5c1203ac9f50e4853b0a0bfffd32c67118ef552a33942982eeab543f5c634395"}, + {file = "protobuf-4.25.0-cp310-abi3-win_amd64.whl", hash = "sha256:c40ff8f00aa737938c5378d461637d15c442a12275a81019cc2fef06d81c9419"}, + {file = "protobuf-4.25.0-cp37-abi3-macosx_10_9_universal2.whl", hash = "sha256:cf21faba64cd2c9a3ed92b7a67f226296b10159dbb8fbc5e854fc90657d908e4"}, + {file = "protobuf-4.25.0-cp37-abi3-manylinux2014_aarch64.whl", hash = "sha256:32ac2100b0e23412413d948c03060184d34a7c50b3e5d7524ee96ac2b10acf51"}, + {file = "protobuf-4.25.0-cp37-abi3-manylinux2014_x86_64.whl", hash = "sha256:683dc44c61f2620b32ce4927de2108f3ebe8ccf2fd716e1e684e5a50da154054"}, + {file = "protobuf-4.25.0-cp38-cp38-win32.whl", hash = "sha256:1a3ba712877e6d37013cdc3476040ea1e313a6c2e1580836a94f76b3c176d575"}, + {file = "protobuf-4.25.0-cp38-cp38-win_amd64.whl", hash = "sha256:b2cf8b5d381f9378afe84618288b239e75665fe58d0f3fd5db400959274296e9"}, + {file = "protobuf-4.25.0-cp39-cp39-win32.whl", hash = "sha256:63714e79b761a37048c9701a37438aa29945cd2417a97076048232c1df07b701"}, + {file = "protobuf-4.25.0-cp39-cp39-win_amd64.whl", hash = "sha256:d94a33db8b7ddbd0af7c467475fb9fde0c705fb315a8433c0e2020942b863a1f"}, + {file = "protobuf-4.25.0-py3-none-any.whl", hash = "sha256:1a53d6f64b00eecf53b65ff4a8c23dc95df1fa1e97bb06b8122e5a64f49fc90a"}, + {file = "protobuf-4.25.0.tar.gz", hash = "sha256:68f7caf0d4f012fd194a301420cf6aa258366144d814f358c5b32558228afa7c"}, +] + +[[package]] +name = "setuptools" +version = "68.2.2" +description = "Easily download, build, install, upgrade, and uninstall Python packages" +optional = false +python-versions = ">=3.8" +files = [ + {file = "setuptools-68.2.2-py3-none-any.whl", hash = "sha256:b454a35605876da60632df1a60f736524eb73cc47bbc9f3f1ef1b644de74fd2a"}, + {file = "setuptools-68.2.2.tar.gz", hash = "sha256:4ac1475276d2f1c48684874089fefcd83bd7162ddaafb81fac866ba0db282a87"}, +] + +[package.extras] +docs = ["furo", "jaraco.packaging (>=9.3)", "jaraco.tidelift (>=1.4)", "pygments-github-lexers (==0.0.5)", "rst.linker (>=1.9)", "sphinx (>=3.5)", "sphinx-favicon", "sphinx-hoverxref (<2)", "sphinx-inline-tabs", "sphinx-lint", "sphinx-notfound-page (>=1,<2)", "sphinx-reredirects", "sphinxcontrib-towncrier"] +testing = ["build[virtualenv]", "filelock (>=3.4.0)", "flake8-2020", "ini2toml[lite] (>=0.9)", "jaraco.develop (>=7.21)", "jaraco.envs (>=2.2)", "jaraco.path (>=3.2.0)", "pip (>=19.1)", "pytest (>=6)", "pytest-black (>=0.3.7)", "pytest-checkdocs (>=2.4)", "pytest-cov", "pytest-enabler (>=2.2)", "pytest-mypy (>=0.9.1)", "pytest-perf", "pytest-ruff", "pytest-timeout", "pytest-xdist", "tomli-w (>=1.0.0)", "virtualenv (>=13.0.0)", "wheel"] +testing-integration = ["build[virtualenv] (>=1.0.3)", "filelock (>=3.4.0)", "jaraco.envs (>=2.2)", "jaraco.path (>=3.2.0)", "packaging (>=23.1)", "pytest", "pytest-enabler", "pytest-xdist", "tomli", "virtualenv (>=13.0.0)", "wheel"] + +[[package]] +name = "soupsieve" +version = "2.5" +description = "A modern CSS selector implementation for Beautiful Soup." +optional = false +python-versions = ">=3.8" +files = [ + {file = "soupsieve-2.5-py3-none-any.whl", hash = "sha256:eaa337ff55a1579b6549dc679565eac1e3d000563bcb1c8ab0d0fefbc0c2cdc7"}, + {file = "soupsieve-2.5.tar.gz", hash = "sha256:5663d5a7b3bfaeee0bc4372e7fc48f9cff4940b3eec54a6451cc5299f1097690"}, +] + +[metadata] +lock-version = "2.0" +python-versions = "^3.8" +content-hash = "1f2dadd2821a62cdfb7562f211be17c8bd12b762551bdb923954f9e7404087ec" diff --git a/.test-infra/mock-apis/pyproject.toml b/.test-infra/mock-apis/pyproject.toml new file mode 100644 index 000000000000..ed3f035cbf40 --- /dev/null +++ b/.test-infra/mock-apis/pyproject.toml @@ -0,0 +1,38 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +[tool.poetry] +name = "mock-apis" +version = "0.1.0" +authors = ["Ritesh Ghorse <riteshghorse@gmail.com>, Damon Douglas <>"] +license = "Apache-2.0" +description = "" +readme = "README.md" +packages = [ + { include = "src/main/python/"} +] + +[tool.poetry.dependencies] +python = "^3.8" +google = "^3.0.0" +grpcio = "^1.59.2" +grpcio-tools = "^1.59.2" + + +[build-system] +requires = ["poetry-core"] +build-backend = "poetry.core.masonry.api" diff --git a/.test-infra/mock-apis/src/main/python/proto/echo/v1/echo_pb2.py b/.test-infra/mock-apis/src/main/python/proto/echo/v1/echo_pb2.py new file mode 100644 index 000000000000..0a1bd2aff771 --- /dev/null +++ b/.test-infra/mock-apis/src/main/python/proto/echo/v1/echo_pb2.py @@ -0,0 +1,47 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# -*- coding: utf-8 -*- +# Generated by the protocol buffer compiler. DO NOT EDIT! +# source: proto/echo/v1/echo.proto +"""Generated protocol buffer code.""" +from google.protobuf import descriptor as _descriptor +from google.protobuf import descriptor_pool as _descriptor_pool +from google.protobuf import symbol_database as _symbol_database +from google.protobuf.internal import builder as _builder +# @@protoc_insertion_point(imports) + +_sym_db = _symbol_database.Default() + + + + +DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n\x18proto/echo/v1/echo.proto\x12\rproto.echo.v1\"7\n\x0b\x45\x63hoRequest\x12\x0e\n\x02id\x18\x01 \x01(\tR\x02id\x12\x18\n\x07payload\x18\x02 \x01(\x0cR\x07payload\"8\n\x0c\x45\x63hoResponse\x12\x0e\n\x02id\x18\x01 \x01(\tR\x02id\x12\x18\n\x07payload\x18\x02 \x01(\x0cR\x07payload2P\n\x0b\x45\x63hoService\x12\x41\n\x04\x45\x63ho\x12\x1a.proto.echo.v1.EchoRequest\x1a\x1b.proto.echo.v1.EchoResponse\"\x00\x42;\n*org.apache.beam.testinfra.mockapis.echo.v1Z\rproto/echo/v1b\x06proto3') + +_globals = globals() +_builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, _globals) +_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, 'proto.echo.v1.echo_pb2', _globals) +if _descriptor._USE_C_DESCRIPTORS == False: + _globals['DESCRIPTOR']._options = None + _globals['DESCRIPTOR']._serialized_options = b'\n*org.apache.beam.testinfra.mockapis.echo.v1Z\rproto/echo/v1' + _globals['_ECHOREQUEST']._serialized_start=43 + _globals['_ECHOREQUEST']._serialized_end=98 + _globals['_ECHORESPONSE']._serialized_start=100 + _globals['_ECHORESPONSE']._serialized_end=156 + _globals['_ECHOSERVICE']._serialized_start=158 + _globals['_ECHOSERVICE']._serialized_end=238 +# @@protoc_insertion_point(module_scope) diff --git a/.test-infra/mock-apis/src/main/python/proto/echo/v1/echo_pb2_grpc.py b/.test-infra/mock-apis/src/main/python/proto/echo/v1/echo_pb2_grpc.py new file mode 100644 index 000000000000..0a92ee4af6c4 --- /dev/null +++ b/.test-infra/mock-apis/src/main/python/proto/echo/v1/echo_pb2_grpc.py @@ -0,0 +1,87 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Generated by the gRPC Python protocol compiler plugin. DO NOT EDIT! +"""Client and server classes corresponding to protobuf-defined services.""" +import grpc + +from proto.echo.v1 import echo_pb2 as proto_dot_echo_dot_v1_dot_echo__pb2 + + +class EchoServiceStub(object): + """EchoService simulates a mock API that echos a request. + """ + + def __init__(self, channel): + """Constructor. + + Args: + channel: A grpc.Channel. + """ + self.Echo = channel.unary_unary( + '/proto.echo.v1.EchoService/Echo', + request_serializer=proto_dot_echo_dot_v1_dot_echo__pb2.EchoRequest.SerializeToString, + response_deserializer=proto_dot_echo_dot_v1_dot_echo__pb2.EchoResponse.FromString, + ) + + +class EchoServiceServicer(object): + """EchoService simulates a mock API that echos a request. + """ + + def Echo(self, request, context): + """Echo an EchoRequest payload in an EchoResponse. + """ + context.set_code(grpc.StatusCode.UNIMPLEMENTED) + context.set_details('Method not implemented!') + raise NotImplementedError('Method not implemented!') + + +def add_EchoServiceServicer_to_server(servicer, server): + rpc_method_handlers = { + 'Echo': grpc.unary_unary_rpc_method_handler( + servicer.Echo, + request_deserializer=proto_dot_echo_dot_v1_dot_echo__pb2.EchoRequest.FromString, + response_serializer=proto_dot_echo_dot_v1_dot_echo__pb2.EchoResponse.SerializeToString, + ), + } + generic_handler = grpc.method_handlers_generic_handler( + 'proto.echo.v1.EchoService', rpc_method_handlers) + server.add_generic_rpc_handlers((generic_handler,)) + + + # This class is part of an EXPERIMENTAL API. +class EchoService(object): + """EchoService simulates a mock API that echos a request. + """ + + @staticmethod + def Echo(request, + target, + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None): + return grpc.experimental.unary_unary(request, target, '/proto.echo.v1.EchoService/Echo', + proto_dot_echo_dot_v1_dot_echo__pb2.EchoRequest.SerializeToString, + proto_dot_echo_dot_v1_dot_echo__pb2.EchoResponse.FromString, + options, channel_credentials, + insecure, call_credentials, compression, wait_for_ready, timeout, metadata) From 03c811e296e96ae682f53b877e9bc4c2820b36e7 Mon Sep 17 00:00:00 2001 From: Danny McCormick <dannymccormick@google.com> Date: Fri, 3 Nov 2023 15:10:15 -0400 Subject: [PATCH 372/435] Declare signMavenJavaPublication dependency on copyPom (#29296) --- sdks/java/bom/build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/java/bom/build.gradle b/sdks/java/bom/build.gradle index 1559d6b6bea5..da2434684205 100644 --- a/sdks/java/bom/build.gradle +++ b/sdks/java/bom/build.gradle @@ -76,7 +76,7 @@ tasks.whenTaskAdded { task -> expand(version: project.version, modules: bomModuleNames) } } - } else if (task.name.startsWith('publishMavenJavaPublication')) { + } else if (task.name.startsWith('publishMavenJavaPublication') || task.name.startsWith('signMavenJavaPublication')) { task.dependsOn copyPom } } From 55bf335bfa9f208a426dc1fdc86d221364c7ed6a Mon Sep 17 00:00:00 2001 From: Danny McCormick <dannymccormick@google.com> Date: Fri, 3 Nov 2023 16:00:39 -0400 Subject: [PATCH 373/435] fix typo in workflow (#29300) --- .github/workflows/build_wheels.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/build_wheels.yml b/.github/workflows/build_wheels.yml index f4ccf368bacb..58d14c6fcd68 100644 --- a/.github/workflows/build_wheels.yml +++ b/.github/workflows/build_wheels.yml @@ -150,7 +150,7 @@ jobs: - name: Build RC source if: steps.is_rc.outputs.is_rc == 1 working-directory: ./sdks/python - run: pip install -U build && pythom -m build --sdist + run: pip install -U build && python -m build --sdist - name: Add RC checksums if: steps.is_rc.outputs.is_rc == 1 working-directory: ./sdks/python/dist From 35f19d55d9997e217b3ecb5bce289a4f29942889 Mon Sep 17 00:00:00 2001 From: Svetak Sundhar <svetaksundhar@google.com> Date: Fri, 3 Nov 2023 20:54:31 +0000 Subject: [PATCH 374/435] Redirect committer guide link (#29229) * Create HealthcareUtils file with shared resources * revert * Redirect committer guide --- website/www/site/static/.htaccess | 2 ++ 1 file changed, 2 insertions(+) diff --git a/website/www/site/static/.htaccess b/website/www/site/static/.htaccess index ff5e56fc37dd..216b415cab10 100644 --- a/website/www/site/static/.htaccess +++ b/website/www/site/static/.htaccess @@ -24,3 +24,5 @@ RedirectMatch permanent "/documentation/sdks/(javadoc|pydoc)(.*)" "https://beam. RedirectMatch "/contribute/design-documents" "https://cwiki.apache.org/confluence/display/BEAM/Design+Documents" RedirectMatch "/contribute/release-guide" "https://github.com/apache/beam/blob/master/contributor-docs/release-guide.md" + +RedirectMatch "/contribute/committer-guide" "https://github.com/apache/beam/blob/master/contributor-docs/committer-guide.md" \ No newline at end of file From 87ca6140c3b023daada11d60ea5a1142a6289b5d Mon Sep 17 00:00:00 2001 From: Calvin Swenson Jr <142366140+niv-lac@users.noreply.github.com> Date: Fri, 3 Nov 2023 17:57:46 -0700 Subject: [PATCH 375/435] Enable ssl use for cluster in CassandraIO (#29302) * Add properties / methods to facilitate optional ssl use Add properties / methods to facilitate optional ssl use by surfacing datastax driver functionality to programmatically configure ssl options when building Cluster. * Add comments with link to driver documentation for programmatic config Add comments with link to driver documentation for programmatic config of ssl * Apply suggested change to address spotless check failure. --- .../beam/sdk/io/cassandra/CassandraIO.java | 55 ++++++++++++++++++- .../sdk/io/cassandra/ConnectionManager.java | 3 +- 2 files changed, 54 insertions(+), 4 deletions(-) diff --git a/sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/CassandraIO.java b/sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/CassandraIO.java index d33642b9c3ab..1429253d1948 100644 --- a/sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/CassandraIO.java +++ b/sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/CassandraIO.java @@ -24,6 +24,7 @@ import com.datastax.driver.core.ConsistencyLevel; import com.datastax.driver.core.PlainTextAuthProvider; import com.datastax.driver.core.QueryOptions; +import com.datastax.driver.core.SSLOptions; import com.datastax.driver.core.Session; import com.datastax.driver.core.SocketOptions; import com.datastax.driver.core.policies.DCAwareRoundRobinPolicy; @@ -192,6 +193,9 @@ public abstract static class Read<T> extends PTransform<PBegin, PCollection<T>> @Nullable abstract ValueProvider<Set<RingRange>> ringRanges(); + @Nullable + abstract ValueProvider<SSLOptions> sslOptions(); + abstract Builder<T> builder(); /** Specify the hosts of the Apache Cassandra instances. */ @@ -385,6 +389,22 @@ public Read<T> withRingRanges(ValueProvider<Set<RingRange>> ringRange) { return builder().setRingRanges(ringRange).build(); } + /** + * Optionally, specify {@link SSLOptions} configuration to utilize SSL. See + * https://docs.datastax.com/en/developer/java-driver/3.11/manual/ssl/#jsse-programmatic + */ + public Read<T> withSsl(SSLOptions sslOptions) { + return withSsl(ValueProvider.StaticValueProvider.of(sslOptions)); + } + + /** + * Optionally, specify {@link SSLOptions} configuration to utilize SSL. See + * https://docs.datastax.com/en/developer/java-driver/3.11/manual/ssl/#jsse-programmatic + */ + public Read<T> withSsl(ValueProvider<SSLOptions> sslOptions) { + return builder().setSslOptions(sslOptions).build(); + } + @Override public PCollection<T> expand(PBegin input) { checkArgument((hosts() != null && port() != null), "WithHosts() and withPort() are required"); @@ -422,7 +442,8 @@ private static <T> Set<RingRange> getRingRanges(Read<T> read) { read.localDc(), read.consistencyLevel(), read.connectTimeout(), - read.readTimeout())) { + read.readTimeout(), + read.sslOptions())) { if (isMurmur3Partitioner(cluster)) { LOG.info("Murmur3Partitioner detected, splitting"); Integer splitCount; @@ -495,6 +516,8 @@ abstract static class Builder<T> { abstract Builder<T> setRingRanges(ValueProvider<Set<RingRange>> ringRange); + abstract Builder<T> setSslOptions(ValueProvider<SSLOptions> sslOptions); + abstract Read<T> autoBuild(); public Read<T> build() { @@ -543,6 +566,8 @@ public abstract static class Write<T> extends PTransform<PCollection<T>, PDone> abstract @Nullable ValueProvider<Integer> readTimeout(); + abstract @Nullable ValueProvider<SSLOptions> sslOptions(); + abstract @Nullable SerializableFunction<Session, Mapper> mapperFactoryFn(); abstract Builder<T> builder(); @@ -725,6 +750,22 @@ public Write<T> withMapperFactoryFn(SerializableFunction<Session, Mapper> mapper return builder().setMapperFactoryFn(mapperFactoryFn).build(); } + /** + * Optionally, specify {@link SSLOptions} configuration to utilize SSL. See + * https://docs.datastax.com/en/developer/java-driver/3.11/manual/ssl/#jsse-programmatic + */ + public Write<T> withSsl(SSLOptions sslOptions) { + return withSsl(ValueProvider.StaticValueProvider.of(sslOptions)); + } + + /** + * Optionally, specify {@link SSLOptions} configuration to utilize SSL. See + * https://docs.datastax.com/en/developer/java-driver/3.11/manual/ssl/#jsse-programmatic + */ + public Write<T> withSsl(ValueProvider<SSLOptions> sslOptions) { + return builder().setSslOptions(sslOptions).build(); + } + @Override public void validate(PipelineOptions pipelineOptions) { checkState( @@ -799,6 +840,8 @@ abstract static class Builder<T> { abstract Optional<SerializableFunction<Session, Mapper>> mapperFactoryFn(); + abstract Builder<T> setSslOptions(ValueProvider<SSLOptions> sslOptions); + abstract Write<T> autoBuild(); // not public public Write<T> build() { @@ -880,7 +923,8 @@ static Cluster getCluster( ValueProvider<String> localDc, ValueProvider<String> consistencyLevel, ValueProvider<Integer> connectTimeout, - ValueProvider<Integer> readTimeout) { + ValueProvider<Integer> readTimeout, + ValueProvider<SSLOptions> sslOptions) { Cluster.Builder builder = Cluster.builder().addContactPoints(hosts.get().toArray(new String[0])).withPort(port.get()); @@ -913,6 +957,10 @@ static Cluster getCluster( socketOptions.setReadTimeoutMillis(readTimeout.get()); } + if (sslOptions != null) { + builder.withSSL(sslOptions.get()); + } + return builder.build(); } @@ -941,7 +989,8 @@ private static class Mutator<T> { spec.localDc(), spec.consistencyLevel(), spec.connectTimeout(), - spec.readTimeout()); + spec.readTimeout(), + spec.sslOptions()); this.session = cluster.connect(spec.keyspace().get()); this.mapperFactoryFn = spec.mapperFactoryFn(); this.mutateFutures = new ArrayList<>(); diff --git a/sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/ConnectionManager.java b/sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/ConnectionManager.java index 21e7d257dcaa..962e8ad8ec00 100644 --- a/sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/ConnectionManager.java +++ b/sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/ConnectionManager.java @@ -71,7 +71,8 @@ static Session getSession(Read<?> read) { read.localDc(), read.consistencyLevel(), read.connectTimeout(), - read.readTimeout())); + read.readTimeout(), + read.sslOptions())); return sessionMap.computeIfAbsent( readToSessionHash(read), k -> cluster.connect(Objects.requireNonNull(read.keyspace()).get())); From e612060217120329d415b44f9ecdd7d0b347a21d Mon Sep 17 00:00:00 2001 From: Danny McCormick <dannymccormick@google.com> Date: Sun, 5 Nov 2023 13:05:59 -0500 Subject: [PATCH 376/435] Fix staging java source/python source (#29299) * Add missing checkout and revert java artifact type changes * Fix variables * Install dateutil * Fix yaml * Install requests * Add option to skip confirmation * Skip confirmation * Checkout workflow branch * Fetch tags * Fetch tags * Fix arg * Fix repo url * Skip another prompt * Fix variable + debug * Fix svn auth --- .github/workflows/build_release_candidate.yml | 29 +++++++++++++------ .../download_github_actions_artifacts.py | 13 +++++---- 2 files changed, 28 insertions(+), 14 deletions(-) diff --git a/.github/workflows/build_release_candidate.yml b/.github/workflows/build_release_candidate.yml index 5c35aa744198..eac2655492ce 100644 --- a/.github/workflows/build_release_candidate.yml +++ b/.github/workflows/build_release_candidate.yml @@ -122,18 +122,18 @@ jobs: mkdir -p beam/${{ github.event.inputs.RELEASE }} cd beam/${{ github.event.inputs.RELEASE }} RC_DIR="beam-${{ github.event.inputs.RELEASE }}-RC${{ github.event.inputs.RC }}" - RC_ZIP="${RC_DIR}.tar.gz" + RC_ZIP="${RC_DIR}.zip" RELEASE_DIR="beam-${{ github.event.inputs.RELEASE }}" RC_TAG="v${{ github.event.inputs.RELEASE }}-RC${{ github.event.inputs.RC }}" - SOURCE_RELEASE_ZIP="apache-beam-${{ github.event.inputs.RELEASE }}-source-release.tar.gz" + SOURCE_RELEASE_ZIP="apache-beam-${{ github.event.inputs.RELEASE }}-source-release.zip" # Check whether there is an existing dist dir if (svn ls "${SOURCE_RELEASE_ZIP}"); then echo "Removing existing ${SOURCE_RELEASE_ZIP}." svn delete "${SOURCE_RELEASE_ZIP}" fi - echo "Downloading: https://github.com/apache/beam/archive/${RC_TAG}.tar.gz" - wget https://github.com/apache/beam/archive/${RC_TAG}.tar.gz -O "${RC_ZIP}" + echo "Downloading: https://github.com/apache/beam/archive/${RC_TAG}.zip" + wget https://github.com/apache/beam/archive/${RC_TAG}.zip -O "${RC_ZIP}" unzip "$RC_ZIP" rm "$RC_ZIP" @@ -155,6 +155,8 @@ jobs: if: ${{github.event.inputs.STAGE_PYTHON_ARTIFACTS == 'yes'}} runs-on: ubuntu-latest steps: + - name: Checkout + uses: actions/checkout@v4 - name: Validate and mask apache id/password run: | echo "::add-mask::${{ github.event.inputs.APACHE_PASSWORD }}" @@ -175,28 +177,37 @@ jobs: uses: crazy-max/ghaction-import-gpg@111c56156bcc6918c056dbef52164cfa583dc549 with: gpg_private_key: ${{ secrets.GPG_PRIVATE_KEY }} + - name: Install dependencies + run: | + pip install python-dateutil + pip install requests - name: stage python artifacts env: RC_TAG: "v${{ github.event.inputs.RELEASE }}-RC${{ github.event.inputs.RC }}" GIT_REPO_BASE_URL: https://github.com/apache/beam RELEASE_DIR: "beam/${{ github.event.inputs.RELEASE }}" + RELEASE: "${{ github.event.inputs.RELEASE }}" SCRIPT_DIR: release/src/main/scripts GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} + SVN_ARTIFACTS_DIR: "beam/${{ github.event.inputs.RELEASE }}/python" run: | svn co https://dist.apache.org/repos/dist/dev/beam mkdir -p "${SVN_ARTIFACTS_DIR}" - + + git fetch --all --tags --prune RELEASE_COMMIT=$(git rev-list -n 1 "tags/${RC_TAG}") python "${SCRIPT_DIR}/download_github_actions_artifacts.py" \ --github-token-var GITHUB_TOKEN \ - --repo-url "${GIT_REPO_BASE_URL}" \ + --repo-url "apache/beam" \ --rc-tag "${RC_TAG}" \ --release-commit "${RELEASE_COMMIT}" \ - --artifacts_dir "${RELEASE_DIR}/python" + --artifacts_dir "${RELEASE_DIR}/python" \ + --yes True cd "${RELEASE_DIR}"/python - + ls + echo "------Checking Hash Value for apache-beam-${RELEASE}.tar.gz-----" sha512sum -c "apache-beam-${RELEASE}.tar.gz.sha512" @@ -216,7 +227,7 @@ jobs: cd .. svn add --force python svn status - svn commit --no-auth-cache -m "Staging Python artifacts for Apache Beam ${RELEASE} RC${RC_NUM}" + svn commit -m "Staging Python artifacts for Apache Beam ${RELEASE} RC${RC_NUM}" --non-interactive --username ${{ github.event.inputs.APACHE_ID }} --password ${{ github.event.inputs.APACHE_PASSWORD }} stage_docker: diff --git a/release/src/main/scripts/download_github_actions_artifacts.py b/release/src/main/scripts/download_github_actions_artifacts.py index 5fbeb51a10cd..99526f1ac7d1 100644 --- a/release/src/main/scripts/download_github_actions_artifacts.py +++ b/release/src/main/scripts/download_github_actions_artifacts.py @@ -50,6 +50,7 @@ def parse_arguments(): parser.add_argument("--release-commit", required=True) parser.add_argument("--artifacts_dir", required=True) parser.add_argument("--rc_number", required=False, default="") + parser.add_argument("--yes", required=False, default=False) args = parser.parse_args() github_token = get_github_token(args.github_token_var) @@ -57,7 +58,7 @@ def parse_arguments(): print("You passed following arguments:") pprint.pprint({**vars(args), **{"github_token": github_token}}) - if not get_yes_or_no_answer("Do you want to continue?"): + if not args.yes and not get_yes_or_no_answer("Do you want to continue?"): print("You said NO. Quitting ...") sys.exit(1) @@ -67,8 +68,9 @@ def parse_arguments(): artifacts_dir = args.artifacts_dir if os.path.isabs(args.artifacts_dir) \ else os.path.abspath(args.artifacts_dir) rc_number = args.rc_number + skip_prompts = args.yes - return github_token, repo_url, rc_tag, release_commit, artifacts_dir, rc_number + return github_token, repo_url, rc_tag, release_commit, artifacts_dir, rc_number, skip_prompts def get_github_token(github_token_var): @@ -241,7 +243,7 @@ def wait_for_workflow_run_to_finish( ) -def prepare_directory(artifacts_dir): +def prepare_directory(artifacts_dir, skip_prompts): """Creates given directory and asks for confirmation if directory exists before clearing it.""" print(f"Preparing Artifacts directory: {artifacts_dir}") if os.path.isdir(artifacts_dir): @@ -249,7 +251,7 @@ def prepare_directory(artifacts_dir): f"Found that directory already exists.\n" f"Any existing content in it will be erased. Proceed?\n" f"Your answer") - if get_yes_or_no_answer(question): + if skip_prompts or get_yes_or_no_answer(question): print(f"Clearing directory: {artifacts_dir}") shutil.rmtree(artifacts_dir, ignore_errors=True) else: @@ -328,6 +330,7 @@ def extract_single_artifact(file_path, output_dir): release_commit, artifacts_dir, rc_number, + skip_prompts, ) = parse_arguments() try: @@ -335,7 +338,7 @@ def extract_single_artifact(file_path, output_dir): run_id = get_last_run_id( workflow_id, repo_url, rc_tag, release_commit, github_token) validate_run(run_id, repo_url, github_token) - prepare_directory(artifacts_dir) + prepare_directory(artifacts_dir, skip_prompts) fetch_github_artifacts(run_id, repo_url, artifacts_dir, github_token, rc_number) print("Script finished successfully!") print(f"Artifacts available in directory: {artifacts_dir}") From 34b93c0d10ddd67d081b401a59a2f77be0ad5a89 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 6 Nov 2023 09:19:34 -0500 Subject: [PATCH 377/435] Bump github.com/spf13/cobra from 1.7.0 to 1.8.0 in /sdks (#29308) Bumps [github.com/spf13/cobra](https://github.com/spf13/cobra) from 1.7.0 to 1.8.0. - [Release notes](https://github.com/spf13/cobra/releases) - [Commits](https://github.com/spf13/cobra/compare/v1.7.0...v1.8.0) --- updated-dependencies: - dependency-name: github.com/spf13/cobra dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] <support@github.com> Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 2 +- sdks/go.sum | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 5618da7ec14c..0ee2c233dab4 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -46,7 +46,7 @@ require ( github.com/lib/pq v1.10.9 github.com/linkedin/goavro/v2 v2.12.0 github.com/proullon/ramsql v0.1.3 - github.com/spf13/cobra v1.7.0 + github.com/spf13/cobra v1.8.0 github.com/testcontainers/testcontainers-go v0.25.0 github.com/tetratelabs/wazero v1.5.0 github.com/xitongsys/parquet-go v1.6.2 diff --git a/sdks/go.sum b/sdks/go.sum index dd53a66dfa62..6343b5689263 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -158,7 +158,7 @@ github.com/coreos/go-systemd/v22 v22.3.2/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSV github.com/cpuguy83/dockercfg v0.3.1 h1:/FpZ+JaygUR/lZP2NlFI2DVfrOEMAIKP5wWEJdoYe9E= github.com/cpuguy83/dockercfg v0.3.1/go.mod h1:sugsbF4//dDlL/i+S+rtpIWp+5h0BHJHfjj5/jFyUJc= github.com/cpuguy83/go-md2man/v2 v2.0.0-20190314233015-f79a8a8ca69d/go.mod h1:maD7wRr/U5Z6m/iR4s+kqSMx2CaBsrgA7czyZG/E6dU= -github.com/cpuguy83/go-md2man/v2 v2.0.2/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46tRHOmNcaadrF8o= +github.com/cpuguy83/go-md2man/v2 v2.0.3/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46tRHOmNcaadrF8o= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/creack/pty v1.1.18 h1:n56/Zwd5o6whRC5PMGretI4IdRLlmBXYNjScPaBgsbY= github.com/cyphar/filepath-securejoin v0.2.3/go.mod h1:aPGpWjXOXUn2NCNjFvBE6aRxGGx79pTxQpKOJNYHHl4= @@ -408,8 +408,8 @@ github.com/sirupsen/logrus v1.9.3 h1:dueUQJ1C2q9oE3F7wvmSGAaVtTmUizReu6fjN8uqzbQ github.com/sirupsen/logrus v1.9.3/go.mod h1:naHLuLoDiP4jHNo9R0sCBMtWGeIprob74mVsIT4qYEQ= github.com/spf13/afero v1.2.1/go.mod h1:9ZxEEn6pIJ8Rxe320qSDBk6AsU0r9pR7Q4OcevTdifk= github.com/spf13/afero v1.2.2/go.mod h1:9ZxEEn6pIJ8Rxe320qSDBk6AsU0r9pR7Q4OcevTdifk= -github.com/spf13/cobra v1.7.0 h1:hyqWnYt1ZQShIddO5kBpj3vu05/++x6tJ6dg8EC572I= -github.com/spf13/cobra v1.7.0/go.mod h1:uLxZILRyS/50WlhOIKD7W6V5bgeIt+4sICxh6uRMrb0= +github.com/spf13/cobra v1.8.0 h1:7aJaZx1B85qltLMc546zn58BxxfZdR/W22ej9CFoEf0= +github.com/spf13/cobra v1.8.0/go.mod h1:WXLWApfZ71AjXPya3WOlMsY9yMs7YeiHhFVlvLyhcho= github.com/spf13/pflag v1.0.5 h1:iy+VFUOCP1a+8yFto/drg2CJ5u0yRoB7fZw3DKv/JXA= github.com/spf13/pflag v1.0.5/go.mod h1:McXfInJRrz4CZXVZOBLb0bTZqETkiAhM9Iw0y3An2Bg= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= From 7fe906497db776dd87eb3f8b1af87767a365d893 Mon Sep 17 00:00:00 2001 From: Israel Herraiz <ihr@google.com> Date: Mon, 6 Nov 2023 16:21:07 +0100 Subject: [PATCH 378/435] [BigQueryIO] Upserts work with CREATE_IF_NEEDED -- doc only PR (#29276) * Upserts work with CREATE_IF_NEEDED * Switch order of points --- .../beam/sdk/io/gcp/bigquery/BigQueryIO.java | 25 +++++++++++-------- 1 file changed, 15 insertions(+), 10 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java index cf62ffed3b9d..2ea5d1c29202 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java @@ -486,8 +486,11 @@ * <h3>Upserts and deletes</h3> * * The connector also supports streaming row updates to BigQuery, with the following qualifications: - * - The CREATE_IF_NEEDED CreateDisposition is not supported. Tables must be precreated with primary - * keys. - Only the STORAGE_WRITE_API_AT_LEAST_ONCE method is supported. + * + * <p>- Only the STORAGE_WRITE_API_AT_LEAST_ONCE method is supported. + * + * <p>- If the table is not previously created and CREATE_IF_NEEDED is used, a primary key must be + * specified using {@link Write#withPrimaryKey}. * * <p>Two types of updates are supported. UPSERT replaces the row with the matching primary key or * inserts the row if non exists. DELETE removes the row with the matching primary key. Row inserts @@ -535,8 +538,8 @@ * }</pre> * * <p>Note that in order to use inserts or deletes, the table must bet set up with a primary key. If - * the table is not previously created and CREATE_IF_NEEDED is used, a primary key must be - * specified. + * the table is not previously created and CREATE_IF_NEEDED is used, a primary key must be specified + * using {@link Write#withPrimaryKey}. */ @SuppressWarnings({ "nullness" // TODO(https://github.com/apache/beam/issues/20506) @@ -2167,9 +2170,10 @@ public static Write<TableRow> writeTableRows() { * apply row updates; directly calling {@link Write#withRowMutationInformationFn} is preferred * when writing non TableRows types (e.g. {@link #writeGenericRecords} or a custom user type). * - * <p>This is only supported when using the {@link Write.Method#STORAGE_API_AT_LEAST_ONCE} insert - * method and {@link Write.CreateDisposition#CREATE_NEVER}. The tables must be precreated with a - * primary key. + * <p>This is supported when using the {@link Write.Method#STORAGE_API_AT_LEAST_ONCE} insert + * method, and with either {@link Write.CreateDisposition#CREATE_NEVER} or {@link + * Write.CreateDisposition#CREATE_IF_NEEDED}. For CREATE_IF_NEEDED, a primary key must be + * specified using {@link Write#withPrimaryKey}. */ public static Write<RowMutation> applyRowMutations() { return BigQueryIO.<RowMutation>write() @@ -2826,9 +2830,10 @@ public Write<T> withMethod(Write.Method method) { * function that determines how a row is applied to BigQuery (upsert, or delete) along with a * sequence number for ordering operations. * - * <p>This is only supported when using the {@link Write.Method#STORAGE_API_AT_LEAST_ONCE} - * insert method and {@link Write.CreateDisposition#CREATE_NEVER}. The tables must be precreated - * with a primary key. + * <p>This is supported when using the {@link Write.Method#STORAGE_API_AT_LEAST_ONCE} insert + * method, and with either {@link Write.CreateDisposition#CREATE_NEVER} or {@link + * Write.CreateDisposition#CREATE_IF_NEEDED}. For CREATE_IF_NEEDED, a primary key must be + * specified using {@link Write#withPrimaryKey}. */ public Write<T> withRowMutationInformationFn( SerializableFunction<T, RowMutationInformation> updateFn) { From cbd756770d2f9b96a562c0f47a6d13b1f05c44ed Mon Sep 17 00:00:00 2001 From: tvalentyn <tvalentyn@users.noreply.github.com> Date: Mon, 6 Nov 2023 07:37:31 -0800 Subject: [PATCH 379/435] Update dev container tags. (#29311) --- sdks/python/apache_beam/runners/dataflow/internal/names.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/runners/dataflow/internal/names.py b/sdks/python/apache_beam/runners/dataflow/internal/names.py index b39c4a2b7576..9a96baeb2a3c 100644 --- a/sdks/python/apache_beam/runners/dataflow/internal/names.py +++ b/sdks/python/apache_beam/runners/dataflow/internal/names.py @@ -34,6 +34,6 @@ # Unreleased sdks use container image tag specified below. # Update this tag whenever there is a change that # requires changes to SDK harness container or SDK harness launcher. -BEAM_DEV_SDK_CONTAINER_TAG = 'beam-master-20231023' +BEAM_DEV_SDK_CONTAINER_TAG = 'beam-master-20231102' DATAFLOW_CONTAINER_IMAGE_REPOSITORY = 'gcr.io/cloud-dataflow/v1beta3' From aa890ea562103c8a6040ce8dd238355666724a02 Mon Sep 17 00:00:00 2001 From: Damon <damondouglas@users.noreply.github.com> Date: Mon, 6 Nov 2023 10:50:49 -0800 Subject: [PATCH 380/435] [RRIO] [Call] Implement PTransform without adaptive throttling (#29144) * Implement Call PTransform * Replace use of Coder with JSON * Remove public modifiers from ApiIOError Builder * Reference #29248 --- sdks/java/io/rrio/build.gradle | 2 + .../beam/io/requestresponse/ApiIOError.java | 36 +- .../apache/beam/io/requestresponse/Call.java | 337 +++++++++++- .../beam/io/requestresponse/CallTest.java | 493 ++++++++++++++++++ 4 files changed, 845 insertions(+), 23 deletions(-) create mode 100644 sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponse/CallTest.java diff --git a/sdks/java/io/rrio/build.gradle b/sdks/java/io/rrio/build.gradle index 6963fcb23ddf..52119c91b47e 100644 --- a/sdks/java/io/rrio/build.gradle +++ b/sdks/java/io/rrio/build.gradle @@ -30,6 +30,8 @@ dependencies { implementation project(path: ":sdks:java:core", configuration: "shadow") implementation library.java.joda_time implementation library.java.vendored_guava_32_1_2_jre + implementation library.java.jackson_core + implementation library.java.jackson_databind implementation "redis.clients:jedis:$jedisVersion" testImplementation project(path: ":sdks:java:core", configuration: "shadowTest") diff --git a/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/ApiIOError.java b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/ApiIOError.java index 5936c5dd84b0..cfff3bd89414 100644 --- a/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/ApiIOError.java +++ b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/ApiIOError.java @@ -17,11 +17,16 @@ */ package org.apache.beam.io.requestresponse; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.auto.value.AutoValue; +import java.util.Optional; import org.apache.beam.sdk.schemas.AutoValueSchema; import org.apache.beam.sdk.schemas.annotations.DefaultSchema; import org.apache.beam.sdk.schemas.annotations.SchemaCaseFormat; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.CaseFormat; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Throwables; +import org.checkerframework.checker.nullness.qual.NonNull; import org.joda.time.Instant; /** {@link ApiIOError} is a data class for storing details about an error. */ @@ -30,12 +35,31 @@ @AutoValue public abstract class ApiIOError { + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + /** + * Instantiate an {@link ApiIOError} from an {@link ErrorT} {@link T} element. The {@link T} + * element is converted to a JSON string. + */ + static <T, ErrorT extends Exception> ApiIOError of(@NonNull ErrorT e, @NonNull T element) + throws JsonProcessingException { + + String json = OBJECT_MAPPER.writeValueAsString(element); + + return ApiIOError.builder() + .setRequestAsJsonString(json) + .setMessage(Optional.ofNullable(e.getMessage()).orElse("")) + .setObservedTimestamp(Instant.now()) + .setStackTrace(Throwables.getStackTraceAsString(e)) + .build(); + } + static Builder builder() { return new AutoValue_ApiIOError.Builder(); } - /** The encoded UTF-8 string representation of the related processed element. */ - public abstract String getEncodedElementAsUtfString(); + /** The JSON string representation of the request associated with the error. */ + public abstract String getRequestAsJsonString(); /** The observed timestamp of the error. */ public abstract Instant getObservedTimestamp(); @@ -49,13 +73,13 @@ static Builder builder() { @AutoValue.Builder abstract static class Builder { - public abstract Builder setEncodedElementAsUtfString(String value); + abstract Builder setRequestAsJsonString(String value); - public abstract Builder setObservedTimestamp(Instant value); + abstract Builder setObservedTimestamp(Instant value); - public abstract Builder setMessage(String value); + abstract Builder setMessage(String value); - public abstract Builder setStackTrace(String value); + abstract Builder setStackTrace(String value); abstract ApiIOError build(); } diff --git a/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/Call.java b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/Call.java index 4f854ea69c7e..52181af534ed 100644 --- a/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/Call.java +++ b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/Call.java @@ -17,55 +17,250 @@ */ package org.apache.beam.io.requestresponse; +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; + +import com.fasterxml.jackson.core.JsonProcessingException; import com.google.auto.value.AutoValue; +import java.io.Serializable; import java.util.Map; +import java.util.Optional; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import org.apache.beam.io.requestresponse.Call.Result; import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionTuple; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.TupleTagList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; +import org.checkerframework.checker.nullness.qual.MonotonicNonNull; +import org.checkerframework.checker.nullness.qual.NonNull; +import org.joda.time.Duration; /** * {@link Call} transforms a {@link RequestT} {@link PCollection} into a {@link ResponseT} {@link * PCollection} and {@link ApiIOError} {@link PCollection}, both wrapped in a {@link Result}. */ -class Call<RequestT, ResponseT> extends PTransform<PCollection<RequestT>, Result<ResponseT>> { +class Call<RequestT, ResponseT> + extends PTransform<@NonNull PCollection<RequestT>, @NonNull Result<ResponseT>> { + + /** + * The default {@link Duration} to wait until completion of user code. A {@link + * UserCodeTimeoutException} is thrown when {@link Caller#call}, {@link SetupTeardown#setup}, or + * {@link SetupTeardown#teardown} exceed this timeout. + */ + static final Duration DEFAULT_TIMEOUT = Duration.standardSeconds(30L); + + /** + * Instantiates a {@link Call} {@link PTransform} with the required {@link Caller} and {@link + * ResponseT} {@link Coder}. Checks for the {@link Caller}'s {@link + * SerializableUtils#ensureSerializable} serializable errors. + */ + static <RequestT, ResponseT> Call<RequestT, ResponseT> of( + Caller<RequestT, ResponseT> caller, Coder<ResponseT> responseTCoder) { + caller = SerializableUtils.ensureSerializable(caller); + return new Call<>( + Configuration.<RequestT, ResponseT>builder() + .setCaller(caller) + .setResponseCoder(responseTCoder) + .build()); + } + + /** + * Instantiates a {@link Call} {@link PTransform} with an implementation of both the {@link + * Caller} and {@link SetupTeardown} in one class and the required {@link ResponseT} {@link + * Coder}. Checks for {@link SerializableUtils#ensureSerializable} to report serializable errors. + */ + static < + RequestT, + ResponseT, + CallerSetupTeardownT extends Caller<RequestT, ResponseT> & SetupTeardown> + Call<RequestT, ResponseT> ofCallerAndSetupTeardown( + CallerSetupTeardownT implementsCallerAndSetupTeardown, Coder<ResponseT> responseTCoder) { + implementsCallerAndSetupTeardown = + SerializableUtils.ensureSerializable(implementsCallerAndSetupTeardown); + return new Call<>( + Configuration.<RequestT, ResponseT>builder() + .setCaller(implementsCallerAndSetupTeardown) + .setResponseCoder(responseTCoder) + .setSetupTeardown(implementsCallerAndSetupTeardown) + .build()); + } private static final TupleTag<ApiIOError> FAILURE_TAG = new TupleTag<ApiIOError>() {}; - // TODO(damondouglas): remove suppress warnings when configuration utilized in future PR. - @SuppressWarnings({"unused"}) private final Configuration<RequestT, ResponseT> configuration; private Call(Configuration<RequestT, ResponseT> configuration) { this.configuration = configuration; } + /** + * Sets the {@link SetupTeardown} to the {@link Call} {@link PTransform} instance. Checks for + * {@link SerializableUtils#ensureSerializable} serializable errors. + */ + Call<RequestT, ResponseT> withSetupTeardown(SetupTeardown setupTeardown) { + setupTeardown = SerializableUtils.ensureSerializable(setupTeardown); + return new Call<>(configuration.toBuilder().setSetupTeardown(setupTeardown).build()); + } + + /** + * Overrides the default {@link #DEFAULT_TIMEOUT}. A {@link UserCodeTimeoutException} is thrown + * when {@link Caller#call}, {@link SetupTeardown#setup}, or {@link SetupTeardown#teardown} exceed + * the timeout. + */ + Call<RequestT, ResponseT> withTimeout(Duration timeout) { + return new Call<>(configuration.toBuilder().setTimeout(timeout).build()); + } + + @Override + public @NonNull Result<ResponseT> expand(PCollection<RequestT> input) { + TupleTag<ResponseT> responseTag = new TupleTag<ResponseT>() {}; + + PCollectionTuple pct = + input.apply( + CallFn.class.getSimpleName(), + ParDo.of(new CallFn<>(responseTag, configuration)) + .withOutputTags(responseTag, TupleTagList.of(FAILURE_TAG))); + + return Result.of(configuration.getResponseCoder(), responseTag, pct); + } + + private static class CallFn<RequestT, ResponseT> extends DoFn<RequestT, ResponseT> { + private final TupleTag<ResponseT> responseTag; + private final CallerWithTimeout<RequestT, ResponseT> caller; + private final SetupTeardownWithTimeout setupTeardown; + + private transient @MonotonicNonNull ExecutorService executor; + + private CallFn( + TupleTag<ResponseT> responseTag, Configuration<RequestT, ResponseT> configuration) { + this.responseTag = responseTag; + this.caller = new CallerWithTimeout<>(configuration.getTimeout(), configuration.getCaller()); + this.setupTeardown = + new SetupTeardownWithTimeout( + configuration.getTimeout(), configuration.getSetupTeardown()); + } + + /** + * Invokes {@link SetupTeardown#setup} forwarding its {@link UserCodeExecutionException}, if + * thrown. + */ + @Setup + public void setup() throws UserCodeExecutionException { + this.executor = Executors.newSingleThreadExecutor(); + this.caller.setExecutor(executor); + this.setupTeardown.setExecutor(executor); + + // TODO(damondouglas): Incorporate repeater when https://github.com/apache/beam/issues/28926 + // resolves. + this.setupTeardown.setup(); + } + + /** + * Invokes {@link SetupTeardown#teardown} forwarding its {@link UserCodeExecutionException}, if + * thrown. + */ + @Teardown + public void teardown() throws UserCodeExecutionException { + // TODO(damondouglas): Incorporate repeater when https://github.com/apache/beam/issues/28926 + // resolves. + this.setupTeardown.teardown(); + checkStateNotNull(executor).shutdown(); + try { + boolean ignored = executor.awaitTermination(3L, TimeUnit.SECONDS); + } catch (InterruptedException ignored) { + } + } + + @ProcessElement + public void process(@Element @NonNull RequestT request, MultiOutputReceiver receiver) + throws JsonProcessingException { + try { + // TODO(damondouglas): https://github.com/apache/beam/issues/29248 + ResponseT response = this.caller.call(request); + receiver.get(responseTag).output(response); + } catch (UserCodeExecutionException e) { + receiver.get(FAILURE_TAG).output(ApiIOError.of(e, request)); + } + } + } + /** Configuration details for {@link Call}. */ @AutoValue - abstract static class Configuration<RequestT, ResponseT> { + abstract static class Configuration<RequestT, ResponseT> implements Serializable { static <RequestT, ResponseT> Builder<RequestT, ResponseT> builder() { return new AutoValue_Call_Configuration.Builder<>(); } + /** The user custom code that converts a {@link RequestT} into a {@link ResponseT}. */ + abstract Caller<RequestT, ResponseT> getCaller(); + + /** The user custom code that implements setup and teardown methods. */ + abstract SetupTeardown getSetupTeardown(); + + /** + * The expected timeout of all user custom code. If user custom code exceeds this timeout, then + * a {@link UserCodeTimeoutException} is thrown. User custom code may throw this exception prior + * to the configured timeout value on their own. + */ + abstract Duration getTimeout(); + + /** + * The {@link Coder} for the {@link ResponseT}. Note that the {@link RequestT}'s {@link Coder} + * is derived from the input {@link PCollection} but can't be determined for the {@link + * ResponseT} and therefore requires explicit setting in the {@link Configuration}. + */ + abstract Coder<ResponseT> getResponseCoder(); + abstract Builder<RequestT, ResponseT> toBuilder(); @AutoValue.Builder abstract static class Builder<RequestT, ResponseT> { - abstract Configuration<RequestT, ResponseT> build(); - } - } + /** See {@link #getCaller()}. */ + abstract Builder<RequestT, ResponseT> setCaller(Caller<RequestT, ResponseT> value); - @Override - public Result<ResponseT> expand(PCollection<RequestT> input) { - return Result.of(new TupleTag<ResponseT>() {}, PCollectionTuple.empty(input.getPipeline())); + /** See {@link #getSetupTeardown()}. */ + abstract Builder<RequestT, ResponseT> setSetupTeardown(SetupTeardown value); + + abstract Optional<SetupTeardown> getSetupTeardown(); + + /** See {@link #getTimeout()}. */ + abstract Builder<RequestT, ResponseT> setTimeout(Duration value); + + abstract Optional<Duration> getTimeout(); + + abstract Builder<RequestT, ResponseT> setResponseCoder(Coder<ResponseT> value); + + abstract Configuration<RequestT, ResponseT> autoBuild(); + + final Configuration<RequestT, ResponseT> build() { + if (!getSetupTeardown().isPresent()) { + setSetupTeardown(new NoopSetupTeardown()); + } + + if (!getTimeout().isPresent()) { + setTimeout(DEFAULT_TIMEOUT); + } + + return autoBuild(); + } + } } /** @@ -73,8 +268,9 @@ public Result<ResponseT> expand(PCollection<RequestT> input) { */ static class Result<ResponseT> implements POutput { - static <ResponseT> Result<ResponseT> of(TupleTag<ResponseT> responseTag, PCollectionTuple pct) { - return new Result<>(responseTag, pct); + static <ResponseT> Result<ResponseT> of( + Coder<ResponseT> responseTCoder, TupleTag<ResponseT> responseTag, PCollectionTuple pct) { + return new Result<>(responseTCoder, responseTag, pct); } private final Pipeline pipeline; @@ -82,10 +278,11 @@ static <ResponseT> Result<ResponseT> of(TupleTag<ResponseT> responseTag, PCollec private final PCollection<ResponseT> responses; private final PCollection<ApiIOError> failures; - private Result(TupleTag<ResponseT> responseTag, PCollectionTuple pct) { + private Result( + Coder<ResponseT> responseTCoder, TupleTag<ResponseT> responseTag, PCollectionTuple pct) { this.pipeline = pct.getPipeline(); this.responseTag = responseTag; - this.responses = pct.get(responseTag); + this.responses = pct.get(responseTag).setCoder(responseTCoder); this.failures = pct.get(FAILURE_TAG); } @@ -98,12 +295,12 @@ public PCollection<ApiIOError> getFailures() { } @Override - public Pipeline getPipeline() { + public @NonNull Pipeline getPipeline() { return this.pipeline; } @Override - public Map<TupleTag<?>, PValue> expand() { + public @NonNull Map<TupleTag<?>, PValue> expand() { return ImmutableMap.of( responseTag, responses, FAILURE_TAG, failures); @@ -111,6 +308,112 @@ public Map<TupleTag<?>, PValue> expand() { @Override public void finishSpecifyingOutput( - String transformName, PInput input, PTransform<?, ?> transform) {} + @NonNull String transformName, + @NonNull PInput input, + @NonNull PTransform<?, ?> transform) {} + } + + private static class NoopSetupTeardown implements SetupTeardown { + + @Override + public void setup() throws UserCodeExecutionException { + // Noop + } + + @Override + public void teardown() throws UserCodeExecutionException { + // Noop + } + } + + private static class CallerWithTimeout<RequestT, ResponseT> + implements Caller<RequestT, ResponseT> { + private final Duration timeout; + private final Caller<RequestT, ResponseT> caller; + private @MonotonicNonNull ExecutorService executor; + + private CallerWithTimeout(Duration timeout, Caller<RequestT, ResponseT> caller) { + this.timeout = timeout; + this.caller = caller; + } + + private void setExecutor(ExecutorService executor) { + this.executor = executor; + } + + @Override + public ResponseT call(RequestT request) throws UserCodeExecutionException { + Future<ResponseT> future = checkStateNotNull(executor).submit(() -> caller.call(request)); + try { + return future.get(timeout.getMillis(), TimeUnit.MILLISECONDS); + } catch (TimeoutException | InterruptedException e) { + throw new UserCodeTimeoutException(e); + } catch (ExecutionException e) { + parseAndThrow(future, e); + } + throw new UserCodeExecutionException("could not complete request"); + } + } + + private static class SetupTeardownWithTimeout implements SetupTeardown { + private final Duration timeout; + private final SetupTeardown setupTeardown; + private @MonotonicNonNull ExecutorService executor; + + SetupTeardownWithTimeout(Duration timeout, SetupTeardown setupTeardown) { + this.timeout = timeout; + this.setupTeardown = setupTeardown; + } + + private void setExecutor(ExecutorService executor) { + this.executor = executor; + } + + @Override + public void setup() throws UserCodeExecutionException { + Callable<Void> callable = + () -> { + setupTeardown.setup(); + return null; + }; + + executeAsync(callable); + } + + @Override + public void teardown() throws UserCodeExecutionException { + Callable<Void> callable = + () -> { + setupTeardown.teardown(); + return null; + }; + + executeAsync(callable); + } + + private void executeAsync(Callable<Void> callable) throws UserCodeExecutionException { + Future<Void> future = checkStateNotNull(executor).submit(callable); + try { + future.get(timeout.getMillis(), TimeUnit.MILLISECONDS); + } catch (TimeoutException | InterruptedException e) { + future.cancel(true); + throw new UserCodeTimeoutException(e); + } catch (ExecutionException e) { + parseAndThrow(future, e); + } + } + } + + private static <T> void parseAndThrow(Future<T> future, ExecutionException e) + throws UserCodeExecutionException { + future.cancel(true); + if (e.getCause() == null) { + throw new UserCodeExecutionException(e); + } + Throwable cause = checkStateNotNull(e.getCause()); + if (cause instanceof UserCodeQuotaException) { + throw new UserCodeQuotaException(cause); + } + throw new UserCodeExecutionException(cause); } } diff --git a/sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponse/CallTest.java b/sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponse/CallTest.java new file mode 100644 index 000000000000..18574b00978d --- /dev/null +++ b/sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponse/CallTest.java @@ -0,0 +1,493 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.io.requestresponse; + +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; +import static org.apache.beam.sdk.values.TypeDescriptors.strings; +import static org.junit.Assert.assertThrows; +import static org.junit.Assert.assertTrue; + +import java.io.Serializable; +import org.apache.beam.io.requestresponse.Call.Result; +import org.apache.beam.sdk.coders.SerializableCoder; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Count; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.Filter; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Objects; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Throwables; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.UncheckedExecutionException; +import org.checkerframework.checker.nullness.qual.NonNull; +import org.joda.time.Duration; +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Tests for {@link Call}. */ +@RunWith(JUnit4.class) +public class CallTest { + @Rule public TestPipeline pipeline = TestPipeline.create(); + + private static final SerializableCoder<@NonNull Response> RESPONSE_CODER = + SerializableCoder.of(Response.class); + + @Test + public void givenCallerNotSerializable_throwsError() { + assertThrows( + IllegalArgumentException.class, () -> Call.of(new UnSerializableCaller(), RESPONSE_CODER)); + } + + @Test + public void givenSetupTeardownNotSerializable_throwsError() { + assertThrows( + IllegalArgumentException.class, + () -> + Call.ofCallerAndSetupTeardown( + new UnSerializableCallerWithSetupTeardown(), RESPONSE_CODER)); + } + + @Test + public void givenCallerThrowsUserCodeExecutionException_emitsIntoFailurePCollection() { + Result<Response> result = + pipeline + .apply(Create.of(new Request("a"))) + .apply(Call.of(new CallerThrowsUserCodeExecutionException(), RESPONSE_CODER)); + + PCollection<ApiIOError> failures = result.getFailures(); + PAssert.thatSingleton(countStackTracesOf(failures, UserCodeExecutionException.class)) + .isEqualTo(1L); + PAssert.thatSingleton(countStackTracesOf(failures, UserCodeQuotaException.class)).isEqualTo(0L); + PAssert.thatSingleton(countStackTracesOf(failures, UserCodeTimeoutException.class)) + .isEqualTo(0L); + + pipeline.run(); + } + + @Test + public void givenCallerThrowsQuotaException_emitsIntoFailurePCollection() { + Result<Response> result = + pipeline + .apply(Create.of(new Request("a"))) + .apply(Call.of(new CallerInvokesQuotaException(), RESPONSE_CODER)); + + PCollection<ApiIOError> failures = result.getFailures(); + PAssert.thatSingleton(countStackTracesOf(failures, UserCodeExecutionException.class)) + .isEqualTo(0L); + PAssert.thatSingleton(countStackTracesOf(failures, UserCodeQuotaException.class)).isEqualTo(1L); + PAssert.thatSingleton(countStackTracesOf(failures, UserCodeTimeoutException.class)) + .isEqualTo(0L); + + pipeline.run(); + } + + @Test + public void givenCallerTimeout_emitsFailurePCollection() { + Duration timeout = Duration.standardSeconds(1L); + Result<Response> result = + pipeline + .apply(Create.of(new Request("a"))) + .apply(Call.of(new CallerExceedsTimeout(timeout), RESPONSE_CODER).withTimeout(timeout)); + + PCollection<ApiIOError> failures = result.getFailures(); + PAssert.thatSingleton(countStackTracesOf(failures, UserCodeExecutionException.class)) + .isEqualTo(0L); + PAssert.thatSingleton(countStackTracesOf(failures, UserCodeQuotaException.class)).isEqualTo(0L); + PAssert.thatSingleton(countStackTracesOf(failures, UserCodeTimeoutException.class)) + .isEqualTo(1L); + + pipeline.run(); + } + + @Test + public void givenCallerThrowsTimeoutException_emitsFailurePCollection() { + Result<Response> result = + pipeline + .apply(Create.of(new Request("a"))) + .apply(Call.of(new CallerThrowsTimeout(), RESPONSE_CODER)); + + PCollection<ApiIOError> failures = result.getFailures(); + PAssert.thatSingleton(countStackTracesOf(failures, UserCodeExecutionException.class)) + .isEqualTo(1L); + PAssert.thatSingleton(countStackTracesOf(failures, UserCodeQuotaException.class)).isEqualTo(0L); + PAssert.thatSingleton(countStackTracesOf(failures, UserCodeTimeoutException.class)) + .isEqualTo(1L); + + pipeline.run(); + } + + @Test + public void givenSetupThrowsUserCodeExecutionException_throwsError() { + pipeline + .apply(Create.of(new Request(""))) + .apply( + Call.of(new ValidCaller(), RESPONSE_CODER) + .withSetupTeardown(new SetupThrowsUserCodeExecutionException())); + + assertPipelineThrows(UserCodeExecutionException.class, pipeline); + } + + @Test + public void givenSetupThrowsQuotaException_throwsError() { + pipeline + .apply(Create.of(new Request(""))) + .apply( + Call.of(new ValidCaller(), RESPONSE_CODER) + .withSetupTeardown(new SetupThrowsUserCodeQuotaException())); + + assertPipelineThrows(UserCodeQuotaException.class, pipeline); + } + + @Test + public void givenSetupTimeout_throwsError() { + Duration timeout = Duration.standardSeconds(1L); + + pipeline + .apply(Create.of(new Request(""))) + .apply( + Call.of(new ValidCaller(), RESPONSE_CODER) + .withSetupTeardown(new SetupExceedsTimeout(timeout)) + .withTimeout(timeout)); + + assertPipelineThrows(UserCodeTimeoutException.class, pipeline); + } + + @Test + public void givenSetupThrowsTimeoutException_throwsError() { + pipeline + .apply(Create.of(new Request(""))) + .apply( + Call.of(new ValidCaller(), RESPONSE_CODER) + .withSetupTeardown(new SetupThrowsUserCodeTimeoutException())); + + assertPipelineThrows(UserCodeTimeoutException.class, pipeline); + } + + @Test + public void givenTeardownThrowsUserCodeExecutionException_throwsError() { + pipeline + .apply(Create.of(new Request(""))) + .apply( + Call.of(new ValidCaller(), RESPONSE_CODER) + .withSetupTeardown(new TeardownThrowsUserCodeExecutionException())); + + // Exceptions thrown during teardown do not populate with the cause + assertThrows(IllegalStateException.class, () -> pipeline.run()); + } + + @Test + public void givenTeardownThrowsQuotaException_throwsError() { + pipeline + .apply(Create.of(new Request(""))) + .apply( + Call.of(new ValidCaller(), RESPONSE_CODER) + .withSetupTeardown(new TeardownThrowsUserCodeQuotaException())); + + // Exceptions thrown during teardown do not populate with the cause + assertThrows(IllegalStateException.class, () -> pipeline.run()); + } + + @Test + public void givenTeardownTimeout_throwsError() { + Duration timeout = Duration.standardSeconds(1L); + pipeline + .apply(Create.of(new Request(""))) + .apply( + Call.of(new ValidCaller(), RESPONSE_CODER) + .withTimeout(timeout) + .withSetupTeardown(new TeardownExceedsTimeout(timeout))); + + // Exceptions thrown during teardown do not populate with the cause + assertThrows(IllegalStateException.class, () -> pipeline.run()); + } + + @Test + public void givenTeardownThrowsTimeoutException_throwsError() { + pipeline + .apply(Create.of(new Request(""))) + .apply( + Call.of(new ValidCaller(), RESPONSE_CODER) + .withSetupTeardown(new TeardownThrowsUserCodeTimeoutException())); + + // Exceptions thrown during teardown do not populate with the cause + assertThrows(IllegalStateException.class, () -> pipeline.run()); + } + + @Test + public void givenValidCaller_emitValidResponse() { + Result<Response> result = + pipeline + .apply(Create.of(new Request("a"))) + .apply(Call.of(new ValidCaller(), RESPONSE_CODER)); + + PAssert.thatSingleton(result.getFailures().apply(Count.globally())).isEqualTo(0L); + PAssert.that(result.getResponses()).containsInAnyOrder(new Response("a")); + + pipeline.run(); + } + + private static class ValidCaller implements Caller<Request, Response> { + + @Override + public Response call(Request request) throws UserCodeExecutionException { + return new Response(request.id); + } + } + + private static class UnSerializableCaller implements Caller<Request, Response> { + + @SuppressWarnings({"unused"}) + private final UnSerializable nestedThing = new UnSerializable(); + + @Override + public Response call(Request request) throws UserCodeExecutionException { + return new Response(request.id); + } + } + + private static class UnSerializableCallerWithSetupTeardown extends UnSerializableCaller + implements SetupTeardown { + + @Override + public void setup() throws UserCodeExecutionException {} + + @Override + public void teardown() throws UserCodeExecutionException {} + } + + private static class UnSerializable {} + + private static class Request implements Serializable { + + final String id; + + Request(String id) { + this.id = id; + } + + public String getId() { + return id; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + Request request = (Request) o; + return Objects.equal(id, request.id); + } + + @Override + public int hashCode() { + return Objects.hashCode(id); + } + } + + private static class Response implements Serializable { + final String id; + + Response(String id) { + this.id = id; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + Response response = (Response) o; + return Objects.equal(id, response.id); + } + + @Override + public int hashCode() { + return Objects.hashCode(id); + } + } + + private static class CallerExceedsTimeout implements Caller<Request, Response> { + private final Duration timeout; + + CallerExceedsTimeout(Duration timeout) { + this.timeout = timeout.plus(Duration.standardSeconds(1L)); + } + + @Override + public Response call(Request request) throws UserCodeExecutionException { + sleep(timeout); + return new Response(request.id); + } + } + + private static class CallerThrowsUserCodeExecutionException implements Caller<Request, Response> { + + @Override + public Response call(Request request) throws UserCodeExecutionException { + throw new UserCodeExecutionException(request.id); + } + } + + private static class CallerThrowsTimeout implements Caller<Request, Response> { + + @Override + public Response call(Request request) throws UserCodeExecutionException { + throw new UserCodeTimeoutException(""); + } + } + + private static class CallerInvokesQuotaException implements Caller<Request, Response> { + + @Override + public Response call(Request request) throws UserCodeExecutionException { + throw new UserCodeQuotaException(request.id); + } + } + + private static class SetupExceedsTimeout implements SetupTeardown { + + private final Duration timeout; + + private SetupExceedsTimeout(Duration timeout) { + this.timeout = timeout.plus(Duration.standardSeconds(1L)); + } + + @Override + public void setup() throws UserCodeExecutionException { + sleep(timeout); + } + + @Override + public void teardown() throws UserCodeExecutionException {} + } + + private static class SetupThrowsUserCodeExecutionException implements SetupTeardown { + @Override + public void setup() throws UserCodeExecutionException { + throw new UserCodeExecutionException("error message"); + } + + @Override + public void teardown() throws UserCodeExecutionException {} + } + + private static class SetupThrowsUserCodeQuotaException implements SetupTeardown { + @Override + public void setup() throws UserCodeExecutionException { + throw new UserCodeQuotaException(""); + } + + @Override + public void teardown() throws UserCodeExecutionException {} + } + + private static class SetupThrowsUserCodeTimeoutException implements SetupTeardown { + @Override + public void setup() throws UserCodeExecutionException { + throw new UserCodeTimeoutException(""); + } + + @Override + public void teardown() throws UserCodeExecutionException {} + } + + private static class TeardownExceedsTimeout implements SetupTeardown { + private final Duration timeout; + + private TeardownExceedsTimeout(Duration timeout) { + this.timeout = timeout.plus(Duration.standardSeconds(1L)); + } + + @Override + public void setup() throws UserCodeExecutionException {} + + @Override + public void teardown() throws UserCodeExecutionException { + sleep(timeout); + } + } + + private static class TeardownThrowsUserCodeExecutionException implements SetupTeardown { + @Override + public void setup() throws UserCodeExecutionException {} + + @Override + public void teardown() throws UserCodeExecutionException { + throw new UserCodeExecutionException(""); + } + } + + private static class TeardownThrowsUserCodeQuotaException implements SetupTeardown { + @Override + public void setup() throws UserCodeExecutionException {} + + @Override + public void teardown() throws UserCodeExecutionException { + throw new UserCodeQuotaException(""); + } + } + + private static class TeardownThrowsUserCodeTimeoutException implements SetupTeardown { + @Override + public void setup() throws UserCodeExecutionException {} + + @Override + public void teardown() throws UserCodeExecutionException { + throw new UserCodeExecutionException(""); + } + } + + private static <ErrorT extends UserCodeExecutionException> void assertPipelineThrows( + Class<ErrorT> clazz, TestPipeline p) { + + // Because we need to wrap in a timeout via a java Future, exceptions are thrown as + // UncheckedExecutionException + UncheckedExecutionException error = assertThrows(UncheckedExecutionException.class, p::run); + + // Iterate through the stack trace to assert ErrorT is among stack. + assertTrue( + error.toString(), Throwables.getCausalChain(error).stream().anyMatch(clazz::isInstance)); + } + + private static <ErrorT extends UserCodeExecutionException> PCollection<Long> countStackTracesOf( + PCollection<ApiIOError> failures, Class<ErrorT> clazz) { + return failures + .apply( + "stackTrace " + clazz.getSimpleName(), + MapElements.into(strings()).via(failure -> checkStateNotNull(failure).getStackTrace())) + .apply( + "filter " + clazz.getSimpleName(), Filter.by(input -> input.contains(clazz.getName()))) + .apply("count " + clazz.getSimpleName(), Count.globally()); + } + + private static void sleep(Duration timeout) { + try { + Thread.sleep(timeout.getMillis()); + } catch (InterruptedException ignored) { + } + } +} From 827afb9f3b7203af910e9de78b051472e29a100c Mon Sep 17 00:00:00 2001 From: Danny McCormick <dannymccormick@google.com> Date: Mon, 6 Nov 2023 14:00:20 -0500 Subject: [PATCH 381/435] Revert "Support DatabaseID in Datastore beam connector (#27987)" (#29317) This reverts commit 181296dec2a6d6973e7aaa17f2cb0286c9308457. --- .../sdk/io/gcp/datastore/DatastoreV1.java | 349 +++--------------- .../sdk/io/gcp/datastore/DatastoreV1Test.java | 94 +---- .../sdk/io/gcp/datastore/SplitQueryFnIT.java | 17 +- .../beam/sdk/io/gcp/datastore/V1ReadIT.java | 26 +- .../beam/sdk/io/gcp/datastore/V1TestUtil.java | 59 +-- .../beam/sdk/io/gcp/datastore/V1WriteIT.java | 11 +- 6 files changed, 98 insertions(+), 458 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java index b94186b9893d..cdd003abae22 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java @@ -126,9 +126,8 @@ * <p>To read a {@link PCollection} from a query to Cloud Datastore, use {@link DatastoreV1#read} * and its methods {@link DatastoreV1.Read#withProjectId} and {@link DatastoreV1.Read#withQuery} to * specify the project to query and the query to read from. You can optionally provide a namespace - * to query within using {@link DatastoreV1.Read#withDatabase} or {@link - * DatastoreV1.Read#withNamespace}. You could also optionally specify how many splits you want for - * the query using {@link DatastoreV1.Read#withNumQuerySplits}. + * to query within using {@link DatastoreV1.Read#withNamespace}. You could also optionally specify + * how many splits you want for the query using {@link DatastoreV1.Read#withNumQuerySplits}. * * <p>For example: * @@ -136,14 +135,12 @@ * // Read a query from Datastore * PipelineOptions options = PipelineOptionsFactory.fromArgs(args).create(); * Query query = ...; - * String databaseId = "..."; * String projectId = "..."; * * Pipeline p = Pipeline.create(options); * PCollection<Entity> entities = p.apply( * DatastoreIO.v1().read() * .withProjectId(projectId) - * .withDatabaseId(databaseId) * .withQuery(query)); * }</pre> * @@ -159,7 +156,7 @@ * * <pre>{@code * PCollection<Entity> entities = ...; - * entities.apply(DatastoreIO.v1().write().withProjectId(projectId).withDatabaseId(databaseId)); + * entities.apply(DatastoreIO.v1().write().withProjectId(projectId)); * p.run(); * }</pre> * @@ -168,7 +165,7 @@ * * <pre>{@code * PCollection<Entity> entities = ...; - * entities.apply(DatastoreIO.v1().deleteEntity().withProjectId(projectId).withDatabaseId(databaseId)); + * entities.apply(DatastoreIO.v1().deleteEntity().withProjectId(projectId)); * p.run(); * }</pre> * @@ -177,7 +174,7 @@ * * <pre>{@code * PCollection<Entity> entities = ...; - * entities.apply(DatastoreIO.v1().deleteKey().withProjectId(projectId).withDatabaseId(databaseId)); + * entities.apply(DatastoreIO.v1().deleteKey().withProjectId(projectId)); * p.run(); * }</pre> * @@ -278,9 +275,6 @@ public class DatastoreV1 { Code.PERMISSION_DENIED, Code.UNAUTHENTICATED); - /** Database ID for the default database. */ - private static final String DEFAULT_DATABASE = ""; - /** * Returns an empty {@link DatastoreV1.Read} builder. Configure the source {@code projectId}, * {@code query}, and optionally {@code namespace} and {@code numQuerySplits} using {@link @@ -319,8 +313,6 @@ public abstract static class Read extends PTransform<PBegin, PCollection<Entity> public abstract @Nullable ValueProvider<String> getProjectId(); - public abstract @Nullable ValueProvider<String> getDatabaseId(); - public abstract @Nullable Query getQuery(); public abstract @Nullable ValueProvider<String> getLiteralGqlQuery(); @@ -342,8 +334,6 @@ public abstract static class Read extends PTransform<PBegin, PCollection<Entity> abstract static class Builder { abstract Builder setProjectId(ValueProvider<String> projectId); - abstract Builder setDatabaseId(ValueProvider<String> databaseId); - abstract Builder setQuery(Query query); abstract Builder setLiteralGqlQuery(ValueProvider<String> literalGqlQuery); @@ -364,16 +354,10 @@ abstract static class Builder { * size from Cloud Datastore. */ static int getEstimatedNumSplits( - Datastore datastore, - String projectId, - String databaseId, - Query query, - @Nullable String namespace, - @Nullable Instant readTime) { + Datastore datastore, Query query, @Nullable String namespace, @Nullable Instant readTime) { int numSplits; try { - long estimatedSizeBytes = - getEstimatedSizeBytes(datastore, projectId, databaseId, query, namespace, readTime); + long estimatedSizeBytes = getEstimatedSizeBytes(datastore, query, namespace, readTime); LOG.info("Estimated size bytes for the query is: {}", estimatedSizeBytes); numSplits = (int) @@ -394,11 +378,7 @@ static int getEstimatedNumSplits( * table. */ private static long queryLatestStatisticsTimestamp( - Datastore datastore, - String projectId, - String databaseId, - @Nullable String namespace, - @Nullable Instant readTime) + Datastore datastore, @Nullable String namespace, @Nullable Instant readTime) throws DatastoreException { Query.Builder query = Query.newBuilder(); // Note: namespace either being null or empty represents the default namespace, in which @@ -410,8 +390,7 @@ private static long queryLatestStatisticsTimestamp( } query.addOrder(makeOrder("timestamp", DESCENDING)); query.setLimit(Int32Value.newBuilder().setValue(1)); - RunQueryRequest request = - makeRequest(projectId, databaseId, query.build(), namespace, readTime); + RunQueryRequest request = makeRequest(query.build(), namespace, readTime); RunQueryResponse response = datastore.runQuery(request); QueryResultBatch batch = response.getBatch(); @@ -427,15 +406,9 @@ private static long queryLatestStatisticsTimestamp( * readTime specified, the latest statistics at or before readTime is retrieved. */ private static Entity getLatestTableStats( - String projectId, - String databaseId, - String ourKind, - @Nullable String namespace, - Datastore datastore, - @Nullable Instant readTime) + String ourKind, @Nullable String namespace, Datastore datastore, @Nullable Instant readTime) throws DatastoreException { - long latestTimestamp = - queryLatestStatisticsTimestamp(datastore, projectId, databaseId, namespace, readTime); + long latestTimestamp = queryLatestStatisticsTimestamp(datastore, namespace, readTime); LOG.info("Latest stats timestamp for kind {} is {}", ourKind, latestTimestamp); Query.Builder queryBuilder = Query.newBuilder(); @@ -450,8 +423,7 @@ private static Entity getLatestTableStats( makeFilter("kind_name", EQUAL, makeValue(ourKind).build()).build(), makeFilter("timestamp", EQUAL, makeValue(latestTimestamp).build()).build())); - RunQueryRequest request = - makeRequest(projectId, databaseId, queryBuilder.build(), namespace, readTime); + RunQueryRequest request = makeRequest(queryBuilder.build(), namespace, readTime); long now = System.currentTimeMillis(); RunQueryResponse response = datastore.runQuery(request); @@ -475,16 +447,10 @@ private static Entity getLatestTableStats( * <p>See https://cloud.google.com/datastore/docs/concepts/stats. */ static long getEstimatedSizeBytes( - Datastore datastore, - String projectId, - String databaseId, - Query query, - @Nullable String namespace, - @Nullable Instant readTime) + Datastore datastore, Query query, @Nullable String namespace, @Nullable Instant readTime) throws DatastoreException { String ourKind = query.getKind(0).getName(); - Entity entity = - getLatestTableStats(projectId, databaseId, ourKind, namespace, datastore, readTime); + Entity entity = getLatestTableStats(ourKind, namespace, datastore, readTime); return entity.getPropertiesOrThrow("entity_bytes").getIntegerValue(); } @@ -504,18 +470,9 @@ private static PartitionId.Builder forNamespace(@Nullable String namespace) { * the requested {@code readTime}. */ static RunQueryRequest makeRequest( - String projectId, - String databaseId, - Query query, - @Nullable String namespace, - @Nullable Instant readTime) { + Query query, @Nullable String namespace, @Nullable Instant readTime) { RunQueryRequest.Builder request = - RunQueryRequest.newBuilder() - .setProjectId(projectId) - .setDatabaseId(databaseId) - .setQuery(query) - .setPartitionId( - forNamespace(namespace).setProjectId(projectId).setDatabaseId(databaseId)); + RunQueryRequest.newBuilder().setQuery(query).setPartitionId(forNamespace(namespace)); if (readTime != null) { Timestamp readTimeProto = Timestamps.fromMillis(readTime.getMillis()); request.setReadOptions(ReadOptions.newBuilder().setReadTime(readTimeProto).build()); @@ -529,18 +486,11 @@ static RunQueryRequest makeRequest( * at the requested {@code readTime}. */ static RunQueryRequest makeRequest( - String projectId, - String databaseId, - GqlQuery gqlQuery, - @Nullable String namespace, - @Nullable Instant readTime) { + GqlQuery gqlQuery, @Nullable String namespace, @Nullable Instant readTime) { RunQueryRequest.Builder request = RunQueryRequest.newBuilder() - .setProjectId(projectId) - .setDatabaseId(databaseId) .setGqlQuery(gqlQuery) - .setPartitionId( - forNamespace(namespace).setProjectId(projectId).setDatabaseId(databaseId)); + .setPartitionId(forNamespace(namespace)); if (readTime != null) { Timestamp readTimeProto = Timestamps.fromMillis(readTime.getMillis()); request.setReadOptions(ReadOptions.newBuilder().setReadTime(readTimeProto).build()); @@ -554,8 +504,6 @@ static RunQueryRequest makeRequest( * namespace}. */ private static List<Query> splitQuery( - String projectId, - String databaseId, Query query, @Nullable String namespace, Datastore datastore, @@ -564,8 +512,7 @@ private static List<Query> splitQuery( @Nullable Instant readTime) throws DatastoreException { // If namespace is set, include it in the split request so splits are calculated accordingly. - PartitionId partitionId = - forNamespace(namespace).setProjectId(projectId).setDatabaseId(databaseId).build(); + PartitionId partitionId = forNamespace(namespace).build(); if (readTime != null) { Timestamp readTimeProto = Timestamps.fromMillis(readTime.getMillis()); return querySplitter.getSplits(query, partitionId, numSplits, datastore, readTimeProto); @@ -588,18 +535,12 @@ private static List<Query> splitQuery( */ @VisibleForTesting static Query translateGqlQueryWithLimitCheck( - String gql, - Datastore datastore, - String projectId, - String databaseId, - String namespace, - @Nullable Instant readTime) + String gql, Datastore datastore, String namespace, @Nullable Instant readTime) throws DatastoreException { String gqlQueryWithZeroLimit = gql + " LIMIT 0"; try { Query translatedQuery = - translateGqlQuery( - gqlQueryWithZeroLimit, datastore, projectId, databaseId, namespace, readTime); + translateGqlQuery(gqlQueryWithZeroLimit, datastore, namespace, readTime); // Clear the limit that we set. return translatedQuery.toBuilder().clearLimit().build(); } catch (DatastoreException e) { @@ -610,7 +551,7 @@ static Query translateGqlQueryWithLimitCheck( LOG.warn("Failed to translate Gql query '{}': {}", gqlQueryWithZeroLimit, e.getMessage()); LOG.warn("User query might have a limit already set, so trying without zero limit"); // Retry without the zero limit. - return translateGqlQuery(gql, datastore, projectId, databaseId, namespace, readTime); + return translateGqlQuery(gql, datastore, namespace, readTime); } else { throw e; } @@ -619,27 +560,13 @@ static Query translateGqlQueryWithLimitCheck( /** Translates a gql query string to {@link Query}. */ private static Query translateGqlQuery( - String gql, - Datastore datastore, - String projectId, - String databaseId, - String namespace, - @Nullable Instant readTime) + String gql, Datastore datastore, String namespace, @Nullable Instant readTime) throws DatastoreException { GqlQuery gqlQuery = GqlQuery.newBuilder().setQueryString(gql).setAllowLiterals(true).build(); - RunQueryRequest req = makeRequest(projectId, databaseId, gqlQuery, namespace, readTime); + RunQueryRequest req = makeRequest(gqlQuery, namespace, readTime); return datastore.runQuery(req).getQuery(); } - /** - * Returns a new {@link DatastoreV1.Read} that reads from the Cloud Datastore for the specified - * database. - */ - public DatastoreV1.Read withDatabaseId(String databaseId) { - checkArgument(databaseId != null, "databaseId can not be null"); - return toBuilder().setDatabaseId(StaticValueProvider.of(databaseId)).build(); - } - /** * Returns a new {@link DatastoreV1.Read} that reads from the Cloud Datastore for the specified * project. @@ -750,24 +677,13 @@ public DatastoreV1.Read withReadTime(Instant readTime) { public long getNumEntities( PipelineOptions options, String ourKind, @Nullable String namespace) { try { - V1Options v1Options = - V1Options.from(getProjectId(), getDatabaseId(), getNamespace(), getLocalhost()); + V1Options v1Options = V1Options.from(getProjectId(), getNamespace(), getLocalhost()); V1DatastoreFactory datastoreFactory = new V1DatastoreFactory(); Datastore datastore = datastoreFactory.getDatastore( - options, - v1Options.getProjectId(), - v1Options.getDatabaseId(), - v1Options.getLocalhost()); - - Entity entity = - getLatestTableStats( - v1Options.getProjectId(), - v1Options.getDatabaseId(), - ourKind, - namespace, - datastore, - getReadTime()); + options, v1Options.getProjectId(), v1Options.getLocalhost()); + + Entity entity = getLatestTableStats(ourKind, namespace, datastore, getReadTime()); return entity.getPropertiesOrThrow("count").getIntegerValue(); } catch (Exception e) { return -1; @@ -788,8 +704,7 @@ public PCollection<Entity> expand(PBegin input) { getQuery() == null || getLiteralGqlQuery() == null, "withQuery() and withLiteralGqlQuery() are exclusive"); - V1Options v1Options = - V1Options.from(getProjectId(), getDatabaseId(), getNamespace(), getLocalhost()); + V1Options v1Options = V1Options.from(getProjectId(), getNamespace(), getLocalhost()); /* * This composite transform involves the following steps: @@ -833,7 +748,6 @@ public void populateDisplayData(DisplayData.Builder builder) { String query = getQuery() == null ? null : getQuery().toString(); builder .addIfNotNull(DisplayData.item("projectId", getProjectId()).withLabel("ProjectId")) - .addIfNotNull(DisplayData.item("databaseId", getDatabaseId()).withLabel("DatabaseId")) .addIfNotNull(DisplayData.item("namespace", getNamespace()).withLabel("Namespace")) .addIfNotNull(DisplayData.item("query", query).withLabel("Query")) .addIfNotNull(DisplayData.item("gqlQuery", getLiteralGqlQuery()).withLabel("GqlQuery")) @@ -843,46 +757,30 @@ public void populateDisplayData(DisplayData.Builder builder) { @VisibleForTesting static class V1Options implements HasDisplayData, Serializable { private final ValueProvider<String> project; - private final ValueProvider<String> database; private final @Nullable ValueProvider<String> namespace; private final @Nullable String localhost; private V1Options( - ValueProvider<String> project, - ValueProvider<String> database, - ValueProvider<String> namespace, - String localhost) { + ValueProvider<String> project, ValueProvider<String> namespace, String localhost) { this.project = project; - this.database = database; this.namespace = namespace; this.localhost = localhost; } - public static V1Options from( - String projectId, ValueProvider<String> databaseId, String namespace, String localhost) { + public static V1Options from(String projectId, String namespace, String localhost) { return from( - StaticValueProvider.of(projectId), - databaseId, - StaticValueProvider.of(namespace), - localhost); + StaticValueProvider.of(projectId), StaticValueProvider.of(namespace), localhost); } public static V1Options from( - ValueProvider<String> project, - ValueProvider<String> databaseId, - ValueProvider<String> namespace, - String localhost) { - return new V1Options(project, databaseId, namespace, localhost); + ValueProvider<String> project, ValueProvider<String> namespace, String localhost) { + return new V1Options(project, namespace, localhost); } public String getProjectId() { return project.get(); } - public String getDatabaseId() { - return database == null ? DEFAULT_DATABASE : database.get(); - } - public @Nullable String getNamespace() { return namespace == null ? null : namespace.get(); } @@ -891,10 +789,6 @@ public ValueProvider<String> getProjectValueProvider() { return project; } - public ValueProvider<String> getDatabaseValueProvider() { - return database; - } - public @Nullable ValueProvider<String> getNamespaceValueProvider() { return namespace; } @@ -908,8 +802,6 @@ public void populateDisplayData(DisplayData.Builder builder) { builder .addIfNotNull( DisplayData.item("projectId", getProjectValueProvider()).withLabel("ProjectId")) - .addIfNotNull( - DisplayData.item("databaseId", getDatabaseValueProvider()).withLabel("DatabaseId")) .addIfNotNull( DisplayData.item("namespace", getNamespaceValueProvider()).withLabel("Namespace")); } @@ -941,10 +833,7 @@ static class GqlQueryTranslateFn extends DoFn<String, Query> { public void startBundle(StartBundleContext c) throws Exception { datastore = datastoreFactory.getDatastore( - c.getPipelineOptions(), - v1Options.getProjectId(), - v1Options.getDatabaseId(), - v1Options.getLocalhost()); + c.getPipelineOptions(), v1Options.getProjectId(), v1Options.getLocalhost()); } @ProcessElement @@ -953,12 +842,7 @@ public void processElement(ProcessContext c) throws Exception { LOG.info("User query: '{}'", gqlQuery); Query query = translateGqlQueryWithLimitCheck( - gqlQuery, - datastore, - v1Options.getProjectId(), - v1Options.getDatabaseId(), - v1Options.getNamespace(), - readTime); + gqlQuery, datastore, v1Options.getNamespace(), readTime); LOG.info("User gql query translated to Query({})", query); c.output(query); } @@ -1006,10 +890,7 @@ public SplitQueryFn(V1Options options, int numSplits, @Nullable Instant readTime public void startBundle(StartBundleContext c) throws Exception { datastore = datastoreFactory.getDatastore( - c.getPipelineOptions(), - options.getProjectId(), - options.getDatabaseId(), - options.getLocalhost()); + c.getPipelineOptions(), options.getProjectId(), options.getLocalhost()); querySplitter = datastoreFactory.getQuerySplitter(); } @@ -1027,13 +908,7 @@ public void processElement(ProcessContext c) throws Exception { // Compute the estimated numSplits if numSplits is not specified by the user. if (numSplits <= 0) { estimatedNumSplits = - getEstimatedNumSplits( - datastore, - options.getProjectId(), - options.getDatabaseId(), - query, - options.getNamespace(), - readTime); + getEstimatedNumSplits(datastore, query, options.getNamespace(), readTime); } else { estimatedNumSplits = numSplits; } @@ -1043,8 +918,6 @@ public void processElement(ProcessContext c) throws Exception { try { querySplits = splitQuery( - options.getProjectId(), - options.getDatabaseId(), query, options.getNamespace(), datastore, @@ -1112,10 +985,7 @@ public ReadFn(V1Options options, @Nullable Instant readTime) { public void startBundle(StartBundleContext c) throws Exception { datastore = datastoreFactory.getDatastore( - c.getPipelineOptions(), - options.getProjectId(), - options.getDatabaseId(), - options.getLocalhost()); + c.getPipelineOptions(), options.getProjectId(), options.getLocalhost()); } private RunQueryResponse runQueryWithRetries(RunQueryRequest request) throws Exception { @@ -1175,13 +1045,7 @@ public void processElement(ProcessContext context) throws Exception { queryBuilder.setStartCursor(currentBatch.getEndCursor()); } - RunQueryRequest request = - makeRequest( - options.getProjectId(), - options.getDatabaseId(), - queryBuilder.build(), - namespace, - readTime); + RunQueryRequest request = makeRequest(queryBuilder.build(), namespace, readTime); RunQueryResponse response = runQueryWithRetries(request); currentBatch = response.getBatch(); @@ -1264,42 +1128,21 @@ public static class Write extends Mutate<Entity> { @Nullable String localhost, boolean throttleRampup, ValueProvider<Integer> hintNumWorkers) { - super(projectId, null, localhost, new UpsertFn(), throttleRampup, hintNumWorkers); - } - - Write( - @Nullable ValueProvider<String> projectId, - @Nullable ValueProvider<String> databaseId, - @Nullable String localhost, - boolean throttleRampup, - ValueProvider<Integer> hintNumWorkers) { - super(projectId, databaseId, localhost, new UpsertFn(), throttleRampup, hintNumWorkers); + super(projectId, localhost, new UpsertFn(), throttleRampup, hintNumWorkers); } - /** Returns a new {@link Write} that writes to the Cloud Datastore for the default database. */ + /** Returns a new {@link Write} that writes to the Cloud Datastore for the specified project. */ public Write withProjectId(String projectId) { checkArgument(projectId != null, "projectId can not be null"); return withProjectId(StaticValueProvider.of(projectId)); } - /** Returns a new {@link Write} that writes to the Cloud Datastore for the database id. */ - public Write withDatabaseId(String databaseId) { - checkArgument(databaseId != null, "databaseId can not be null"); - return withDatabaseId(StaticValueProvider.of(databaseId)); - } - /** Same as {@link Write#withProjectId(String)} but with a {@link ValueProvider}. */ public Write withProjectId(ValueProvider<String> projectId) { checkArgument(projectId != null, "projectId can not be null"); return new Write(projectId, localhost, throttleRampup, hintNumWorkers); } - /** Same as {@link Write#withDatabaseId(String)} but with a {@link ValueProvider}. */ - public Write withDatabaseId(ValueProvider<String> databaseId) { - checkArgument(databaseId != null, "databaseId can not be null"); - return new Write(projectId, databaseId, localhost, throttleRampup, hintNumWorkers); - } - /** * Returns a new {@link Write} that writes to the Cloud Datastore Emulator running locally on * the specified host port. @@ -1345,16 +1188,7 @@ public static class DeleteEntity extends Mutate<Entity> { @Nullable String localhost, boolean throttleRampup, ValueProvider<Integer> hintNumWorkers) { - super(projectId, null, localhost, new DeleteEntityFn(), throttleRampup, hintNumWorkers); - } - - DeleteEntity( - @Nullable ValueProvider<String> projectId, - @Nullable ValueProvider<String> databaseId, - @Nullable String localhost, - boolean throttleRampup, - ValueProvider<Integer> hintNumWorkers) { - super(projectId, databaseId, localhost, new DeleteEntityFn(), throttleRampup, hintNumWorkers); + super(projectId, localhost, new DeleteEntityFn(), throttleRampup, hintNumWorkers); } /** @@ -1366,27 +1200,12 @@ public DeleteEntity withProjectId(String projectId) { return withProjectId(StaticValueProvider.of(projectId)); } - /** - * Returns a new {@link DeleteEntity} that deletes entities from the Cloud Datastore for the - * specified database. - */ - public DeleteEntity withDatabaseId(String databaseId) { - checkArgument(databaseId != null, "databaseId can not be null"); - return withDatabaseId(StaticValueProvider.of(databaseId)); - } - /** Same as {@link DeleteEntity#withProjectId(String)} but with a {@link ValueProvider}. */ public DeleteEntity withProjectId(ValueProvider<String> projectId) { checkArgument(projectId != null, "projectId can not be null"); return new DeleteEntity(projectId, localhost, throttleRampup, hintNumWorkers); } - /** Same as {@link DeleteEntity#withDatabaseId(String)} but with a {@link ValueProvider}. */ - public DeleteEntity withDatabaseId(ValueProvider<String> databaseId) { - checkArgument(databaseId != null, "databaseId can not be null"); - return new DeleteEntity(projectId, databaseId, localhost, throttleRampup, hintNumWorkers); - } - /** * Returns a new {@link DeleteEntity} that deletes entities from the Cloud Datastore Emulator * running locally on the specified host port. @@ -1434,16 +1253,7 @@ public static class DeleteKey extends Mutate<Key> { @Nullable String localhost, boolean throttleRampup, ValueProvider<Integer> hintNumWorkers) { - super(projectId, null, localhost, new DeleteKeyFn(), throttleRampup, hintNumWorkers); - } - - DeleteKey( - @Nullable ValueProvider<String> projectId, - @Nullable ValueProvider<String> databaseId, - @Nullable String localhost, - boolean throttleRampup, - ValueProvider<Integer> hintNumWorkers) { - super(projectId, databaseId, localhost, new DeleteKeyFn(), throttleRampup, hintNumWorkers); + super(projectId, localhost, new DeleteKeyFn(), throttleRampup, hintNumWorkers); } /** @@ -1455,15 +1265,6 @@ public DeleteKey withProjectId(String projectId) { return withProjectId(StaticValueProvider.of(projectId)); } - /** - * Returns a new {@link DeleteKey} that deletes entities from the Cloud Datastore for the - * specified database. - */ - public DeleteKey withDatabaseId(String databaseId) { - checkArgument(databaseId != null, "databaseId can not be null"); - return withDatabaseId(StaticValueProvider.of(databaseId)); - } - /** * Returns a new {@link DeleteKey} that deletes entities from the Cloud Datastore Emulator * running locally on the specified host port. @@ -1479,12 +1280,6 @@ public DeleteKey withProjectId(ValueProvider<String> projectId) { return new DeleteKey(projectId, localhost, throttleRampup, hintNumWorkers); } - /** Same as {@link DeleteKey#withDatabaseId(String)} but with a {@link ValueProvider}. */ - public DeleteKey withDatabaseId(ValueProvider<String> databaseId) { - checkArgument(databaseId != null, "databaseId can not be null"); - return new DeleteKey(projectId, databaseId, localhost, throttleRampup, hintNumWorkers); - } - /** Returns a new {@link DeleteKey} that does not throttle during ramp-up. */ public DeleteKey withRampupThrottlingDisabled() { return new DeleteKey(projectId, localhost, false, hintNumWorkers); @@ -1517,7 +1312,6 @@ public DeleteKey withHintNumWorkers(ValueProvider<Integer> hintNumWorkers) { private abstract static class Mutate<T> extends PTransform<PCollection<T>, PDone> { protected ValueProvider<String> projectId; - protected ValueProvider<String> databaseId; protected @Nullable String localhost; protected boolean throttleRampup; protected ValueProvider<Integer> hintNumWorkers; @@ -1532,13 +1326,11 @@ private abstract static class Mutate<T> extends PTransform<PCollection<T>, PDone */ Mutate( @Nullable ValueProvider<String> projectId, - @Nullable ValueProvider<String> databaseId, @Nullable String localhost, SimpleFunction<T, Mutation> mutationFn, boolean throttleRampup, ValueProvider<Integer> hintNumWorkers) { this.projectId = projectId; - this.databaseId = databaseId; this.localhost = localhost; this.throttleRampup = throttleRampup; this.hintNumWorkers = hintNumWorkers; @@ -1580,14 +1372,7 @@ public PCollectionView<Instant> expand(PBegin input) { ParDo.of(rampupThrottlingFn).withSideInputs(startTimestampView)); } intermediateOutput.apply( - "Write Mutation to Datastore", - ParDo.of( - new DatastoreWriterFn( - projectId, - databaseId, - localhost, - new V1DatastoreFactory(), - new WriteBatcherImpl()))); + "Write Mutation to Datastore", ParDo.of(new DatastoreWriterFn(projectId, localhost))); return PDone.in(input.getPipeline()); } @@ -1605,7 +1390,6 @@ public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); builder .addIfNotNull(DisplayData.item("projectId", projectId).withLabel("Output Project")) - .addIfNotNull(DisplayData.item("databaseId", databaseId).withLabel("Output Database")) .include("mutationFn", mutationFn); if (rampupThrottlingFn != null) { builder.include("rampupThrottlingFn", rampupThrottlingFn); @@ -1615,10 +1399,6 @@ public void populateDisplayData(DisplayData.Builder builder) { public String getProjectId() { return projectId.get(); } - - public String getDatabaseId() { - return databaseId.get(); - } } /** Determines batch sizes for commit RPCs. */ @@ -1700,7 +1480,6 @@ static class DatastoreWriterFn extends DoFn<Mutation, Void> { private static final Logger LOG = LoggerFactory.getLogger(DatastoreWriterFn.class); private final ValueProvider<String> projectId; - private final ValueProvider<String> databaseId; private final @Nullable String localhost; private transient Datastore datastore; private final V1DatastoreFactory datastoreFactory; @@ -1732,14 +1511,13 @@ static class DatastoreWriterFn extends DoFn<Mutation, Void> { DatastoreWriterFn(String projectId, @Nullable String localhost) { this( StaticValueProvider.of(projectId), - null, localhost, new V1DatastoreFactory(), new WriteBatcherImpl()); } DatastoreWriterFn(ValueProvider<String> projectId, @Nullable String localhost) { - this(projectId, null, localhost, new V1DatastoreFactory(), new WriteBatcherImpl()); + this(projectId, localhost, new V1DatastoreFactory(), new WriteBatcherImpl()); } @VisibleForTesting @@ -1748,18 +1526,7 @@ static class DatastoreWriterFn extends DoFn<Mutation, Void> { @Nullable String localhost, V1DatastoreFactory datastoreFactory, WriteBatcher writeBatcher) { - this(projectId, null, localhost, datastoreFactory, writeBatcher); - } - - @VisibleForTesting - DatastoreWriterFn( - ValueProvider<String> projectId, - ValueProvider<String> databaseId, - @Nullable String localhost, - V1DatastoreFactory datastoreFactory, - WriteBatcher writeBatcher) { this.projectId = checkNotNull(projectId, "projectId"); - this.databaseId = databaseId; this.localhost = localhost; this.datastoreFactory = datastoreFactory; this.writeBatcher = writeBatcher; @@ -1767,10 +1534,7 @@ static class DatastoreWriterFn extends DoFn<Mutation, Void> { @StartBundle public void startBundle(StartBundleContext c) { - String databaseIdOrDefaultDatabase = databaseId == null ? DEFAULT_DATABASE : databaseId.get(); - datastore = - datastoreFactory.getDatastore( - c.getPipelineOptions(), projectId.get(), databaseIdOrDefaultDatabase, localhost); + datastore = datastoreFactory.getDatastore(c.getPipelineOptions(), projectId.get(), localhost); writeBatcher.start(); if (adaptiveThrottler == null) { // Initialize throttler at first use, because it is not serializable. @@ -1838,14 +1602,11 @@ private synchronized void flushBatch() batchSize.update(mutations.size()); - String databaseIdOrDefaultDatabase = databaseId == null ? DEFAULT_DATABASE : databaseId.get(); while (true) { // Batch upsert entities. CommitRequest.Builder commitRequest = CommitRequest.newBuilder(); commitRequest.addAllMutations(mutations); commitRequest.setMode(CommitRequest.Mode.NON_TRANSACTIONAL); - commitRequest.setProjectId(projectId.get()); - commitRequest.setDatabaseId(databaseIdOrDefaultDatabase); long startTime = System.currentTimeMillis(), endTime; if (adaptiveThrottler.throttleRequest(startTime)) { @@ -1867,7 +1628,6 @@ private synchronized void flushBatch() ServiceCallMetric serviceCallMetric = new ServiceCallMetric(MonitoringInfoConstants.Urns.API_REQUEST_COUNT, baseLabels); try { - datastore.commit(commitRequest.build()); endTime = System.currentTimeMillis(); serviceCallMetric.call("ok"); @@ -2008,9 +1768,8 @@ public void populateDisplayData(DisplayData.Builder builder) { static class V1DatastoreFactory implements Serializable { /** Builds a Cloud Datastore client for the given pipeline options and project. */ - public Datastore getDatastore( - PipelineOptions pipelineOptions, String projectId, String databaseId) { - return getDatastore(pipelineOptions, projectId, databaseId, null); + public Datastore getDatastore(PipelineOptions pipelineOptions, String projectId) { + return getDatastore(pipelineOptions, projectId, null); } /** @@ -2018,10 +1777,7 @@ public Datastore getDatastore( * locahost. */ public Datastore getDatastore( - PipelineOptions pipelineOptions, - String projectId, - String databaseId, - @Nullable String localhost) { + PipelineOptions pipelineOptions, String projectId, @Nullable String localhost) { Credentials credential = pipelineOptions.as(GcpOptions.class).getGcpCredential(); // Add Beam version to user agent header. @@ -2041,10 +1797,7 @@ public Datastore getDatastore( } DatastoreOptions.Builder builder = - new DatastoreOptions.Builder() - .projectId(projectId) - .databaseId(databaseId) - .initializer(initializer); + new DatastoreOptions.Builder().projectId(projectId).initializer(initializer); if (localhost != null) { builder.localHost(localhost); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java index ab6f5567dd9a..3280e17998e4 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java @@ -125,7 +125,6 @@ @RunWith(Enclosed.class) public class DatastoreV1Test { private static final String PROJECT_ID = "testProject"; - private static final String DATABASE_ID = ""; private static final String NAMESPACE = "testNamespace"; private static final String KIND = "testKind"; private static final Query QUERY; @@ -139,7 +138,7 @@ public class DatastoreV1Test { Query.Builder q = Query.newBuilder(); q.addKindBuilder().setName(KIND); QUERY = q.build(); - V_1_OPTIONS = V1Options.from(PROJECT_ID, StaticValueProvider.of(DATABASE_ID), NAMESPACE, null); + V_1_OPTIONS = V1Options.from(PROJECT_ID, NAMESPACE, null); } @Mock protected Datastore mockDatastore; @@ -154,9 +153,6 @@ public void setUp() { when(mockDatastoreFactory.getDatastore(any(PipelineOptions.class), any(String.class), any())) .thenReturn(mockDatastore); - when(mockDatastoreFactory.getDatastore( - any(PipelineOptions.class), any(String.class), any(String.class), any())) - .thenReturn(mockDatastore); when(mockDatastoreFactory.getQuerySplitter()).thenReturn(mockQuerySplitter); // Setup the ProcessWideContainer for testing metrics are set. MetricsContainerImpl container = new MetricsContainerImpl(null); @@ -171,12 +167,10 @@ public void testBuildRead() throws Exception { DatastoreIO.v1() .read() .withProjectId(PROJECT_ID) - .withDatabaseId(DATABASE_ID) .withQuery(QUERY) .withNamespace(NAMESPACE); assertEquals(QUERY, read.getQuery()); assertEquals(PROJECT_ID, read.getProjectId().get()); - assertEquals(DATABASE_ID, read.getDatabaseId().get()); assertEquals(NAMESPACE, read.getNamespace().get()); } @@ -186,13 +180,11 @@ public void testBuildReadWithReadTime() throws Exception { DatastoreIO.v1() .read() .withProjectId(PROJECT_ID) - .withDatabaseId(DATABASE_ID) .withQuery(QUERY) .withReadTime(TIMESTAMP); assertEquals(TIMESTAMP, read.getReadTime()); assertEquals(QUERY, read.getQuery()); assertEquals(PROJECT_ID, read.getProjectId().get()); - assertEquals(DATABASE_ID, read.getDatabaseId().get()); } @Test @@ -201,12 +193,10 @@ public void testBuildReadWithGqlQuery() throws Exception { DatastoreIO.v1() .read() .withProjectId(PROJECT_ID) - .withDatabaseId(DATABASE_ID) .withLiteralGqlQuery(GQL_QUERY) .withNamespace(NAMESPACE); assertEquals(GQL_QUERY, read.getLiteralGqlQuery().get()); assertEquals(PROJECT_ID, read.getProjectId().get()); - assertEquals(DATABASE_ID, read.getDatabaseId().get()); assertEquals(NAMESPACE, read.getNamespace().get()); } @@ -220,12 +210,10 @@ public void testBuildReadAlt() throws Exception { .withQuery(QUERY) .withNamespace(NAMESPACE) .withProjectId(PROJECT_ID) - .withDatabaseId(DATABASE_ID) .withLocalhost(LOCALHOST); assertEquals(TIMESTAMP, read.getReadTime()); assertEquals(QUERY, read.getQuery()); assertEquals(PROJECT_ID, read.getProjectId().get()); - assertEquals(DATABASE_ID, read.getDatabaseId().get()); assertEquals(NAMESPACE, read.getNamespace().get()); assertEquals(LOCALHOST, read.getLocalhost()); } @@ -236,7 +224,6 @@ public void testReadValidationFailsQueryAndGqlQuery() throws Exception { DatastoreIO.v1() .read() .withProjectId(PROJECT_ID) - .withDatabaseId(DATABASE_ID) .withLiteralGqlQuery(GQL_QUERY) .withQuery(QUERY); @@ -270,7 +257,6 @@ public void testReadDisplayData() { DatastoreIO.v1() .read() .withProjectId(PROJECT_ID) - .withDatabaseId(DATABASE_ID) .withQuery(QUERY) .withNamespace(NAMESPACE) .withReadTime(TIMESTAMP); @@ -278,7 +264,6 @@ public void testReadDisplayData() { DisplayData displayData = DisplayData.from(read); assertThat(displayData, hasDisplayItem("projectId", PROJECT_ID)); - assertThat(displayData, hasDisplayItem("databaseId", DATABASE_ID)); assertThat(displayData, hasDisplayItem("query", QUERY.toString())); assertThat(displayData, hasDisplayItem("namespace", NAMESPACE)); assertThat(displayData, hasDisplayItem("readTime", TIMESTAMP)); @@ -290,7 +275,6 @@ public void testReadDisplayDataWithGqlQuery() { DatastoreIO.v1() .read() .withProjectId(PROJECT_ID) - .withDatabaseId(DATABASE_ID) .withLiteralGqlQuery(GQL_QUERY) .withNamespace(NAMESPACE) .withReadTime(TIMESTAMP); @@ -311,7 +295,6 @@ public void testSourcePrimitiveDisplayData() { DatastoreIO.v1() .read() .withProjectId(PROJECT_ID) - .withDatabaseId(DATABASE_ID) .withQuery(Query.newBuilder().build()) .withNumQuerySplits(numSplits); @@ -330,34 +313,29 @@ public void testSourcePrimitiveDisplayData() { @Test public void testWriteDisplayData() { - Write write = DatastoreIO.v1().write().withProjectId(PROJECT_ID).withDatabaseId(DATABASE_ID); + Write write = DatastoreIO.v1().write().withProjectId(PROJECT_ID); DisplayData displayData = DisplayData.from(write); assertThat(displayData, hasDisplayItem("projectId", PROJECT_ID)); - assertThat(displayData, hasDisplayItem("databaseId", DATABASE_ID)); } @Test public void testDeleteEntityDisplayData() { - DeleteEntity deleteEntity = - DatastoreIO.v1().deleteEntity().withProjectId(PROJECT_ID).withDatabaseId(DATABASE_ID); + DeleteEntity deleteEntity = DatastoreIO.v1().deleteEntity().withProjectId(PROJECT_ID); DisplayData displayData = DisplayData.from(deleteEntity); assertThat(displayData, hasDisplayItem("projectId", PROJECT_ID)); - assertThat(displayData, hasDisplayItem("databaseId", DATABASE_ID)); } @Test public void testDeleteKeyDisplayData() { - DeleteKey deleteKey = - DatastoreIO.v1().deleteKey().withProjectId(PROJECT_ID).withDatabaseId(DATABASE_ID); + DeleteKey deleteKey = DatastoreIO.v1().deleteKey().withProjectId(PROJECT_ID); DisplayData displayData = DisplayData.from(deleteKey); assertThat(displayData, hasDisplayItem("projectId", PROJECT_ID)); - assertThat(displayData, hasDisplayItem("databaseId", DATABASE_ID)); } @Test @@ -622,8 +600,6 @@ private void datastoreWriterFnTest(int numMutations) throws Exception { while (start < numMutations) { int end = Math.min(numMutations, start + DatastoreV1.DATASTORE_BATCH_UPDATE_ENTITIES_START); CommitRequest.Builder commitRequest = CommitRequest.newBuilder(); - commitRequest.setProjectId(PROJECT_ID); - commitRequest.setDatabaseId(DATABASE_ID); commitRequest.setMode(CommitRequest.Mode.NON_TRANSACTIONAL); commitRequest.addAllMutations(mutations.subList(start, end)); // Verify all the batch requests were made with the expected mutations. @@ -668,8 +644,6 @@ public void testDatastoreWriterFnWithLargeEntities() throws Exception { while (start < mutations.size()) { int end = Math.min(mutations.size(), start + entitiesPerRpc); CommitRequest.Builder commitRequest = CommitRequest.newBuilder(); - commitRequest.setProjectId(PROJECT_ID); - commitRequest.setDatabaseId(DATABASE_ID); commitRequest.setMode(CommitRequest.Mode.NON_TRANSACTIONAL); commitRequest.addAllMutations(mutations.subList(start, end)); // Verify all the batch requests were made with the expected mutations. @@ -707,16 +681,12 @@ public void testDatastoreWriterFnWithDuplicateEntities() throws Exception { CommitRequest.Builder commitRequest = CommitRequest.newBuilder(); commitRequest.setMode(CommitRequest.Mode.NON_TRANSACTIONAL); commitRequest.addAllMutations(mutations.subList(0, 2)); - commitRequest.setProjectId(PROJECT_ID); - commitRequest.setDatabaseId(DATABASE_ID); verify(mockDatastore, times(1)).commit(commitRequest.build()); // second invocation has key [0, 2] because the second 0 triggered a flush batch commitRequest = CommitRequest.newBuilder(); commitRequest.setMode(CommitRequest.Mode.NON_TRANSACTIONAL); commitRequest.addAllMutations(mutations.subList(2, 4)); - commitRequest.setProjectId(PROJECT_ID); - commitRequest.setDatabaseId(DATABASE_ID); verify(mockDatastore, times(1)).commit(commitRequest.build()); verifyMetricWasSet("BatchDatastoreWrite", "ok", "", 2); } @@ -871,26 +841,17 @@ public void testEstimatedSizeBytes() throws Exception { long timestamp = 1234L; RunQueryRequest latestTimestampRequest = - makeRequest( - PROJECT_ID, DATABASE_ID, makeLatestTimestampQuery(NAMESPACE), NAMESPACE, readTime); + makeRequest(makeLatestTimestampQuery(NAMESPACE), NAMESPACE, readTime); RunQueryResponse latestTimestampResponse = makeLatestTimestampResponse(timestamp); // Per Kind statistics request and response RunQueryRequest statRequest = - makeRequest( - PROJECT_ID, - DATABASE_ID, - makeStatKindQuery(NAMESPACE, timestamp), - NAMESPACE, - readTime); + makeRequest(makeStatKindQuery(NAMESPACE, timestamp), NAMESPACE, readTime); RunQueryResponse statResponse = makeStatKindResponse(entityBytes); when(mockDatastore.runQuery(latestTimestampRequest)).thenReturn(latestTimestampResponse); when(mockDatastore.runQuery(statRequest)).thenReturn(statResponse); - assertEquals( - entityBytes, - getEstimatedSizeBytes( - mockDatastore, PROJECT_ID, DATABASE_ID, QUERY, NAMESPACE, readTime)); + assertEquals(entityBytes, getEstimatedSizeBytes(mockDatastore, QUERY, NAMESPACE, readTime)); verify(mockDatastore, times(1)).runQuery(latestTimestampRequest); verify(mockDatastore, times(1)).runQuery(statRequest); } @@ -956,18 +917,12 @@ public void testSplitQueryFnWithoutNumSplits() throws Exception { long timestamp = 1234L; RunQueryRequest latestTimestampRequest = - makeRequest( - PROJECT_ID, DATABASE_ID, makeLatestTimestampQuery(NAMESPACE), NAMESPACE, readTime); + makeRequest(makeLatestTimestampQuery(NAMESPACE), NAMESPACE, readTime); RunQueryResponse latestTimestampResponse = makeLatestTimestampResponse(timestamp); // Per Kind statistics request and response RunQueryRequest statRequest = - makeRequest( - PROJECT_ID, - DATABASE_ID, - makeStatKindQuery(NAMESPACE, timestamp), - NAMESPACE, - readTime); + makeRequest(makeStatKindQuery(NAMESPACE, timestamp), NAMESPACE, readTime); RunQueryResponse statResponse = makeStatKindResponse(entityBytes); when(mockDatastore.runQuery(latestTimestampRequest)).thenReturn(latestTimestampResponse); @@ -1075,11 +1030,9 @@ public void testTranslateGqlQueryWithLimit() throws Exception { GqlQuery gqlQueryWithZeroLimit = GqlQuery.newBuilder().setQueryString(gqlWithZeroLimit).setAllowLiterals(true).build(); - RunQueryRequest gqlRequest = - makeRequest(PROJECT_ID, DATABASE_ID, gqlQuery, V_1_OPTIONS.getNamespace(), readTime); + RunQueryRequest gqlRequest = makeRequest(gqlQuery, V_1_OPTIONS.getNamespace(), readTime); RunQueryRequest gqlRequestWithZeroLimit = - makeRequest( - PROJECT_ID, DATABASE_ID, gqlQueryWithZeroLimit, V_1_OPTIONS.getNamespace(), readTime); + makeRequest(gqlQueryWithZeroLimit, V_1_OPTIONS.getNamespace(), readTime); when(mockDatastore.runQuery(gqlRequestWithZeroLimit)) .thenThrow( new DatastoreException( @@ -1091,8 +1044,7 @@ public void testTranslateGqlQueryWithLimit() throws Exception { when(mockDatastore.runQuery(gqlRequest)) .thenReturn(RunQueryResponse.newBuilder().setQuery(QUERY).build()); assertEquals( - translateGqlQueryWithLimitCheck( - gql, mockDatastore, PROJECT_ID, DATABASE_ID, V_1_OPTIONS.getNamespace(), readTime), + translateGqlQueryWithLimitCheck(gql, mockDatastore, V_1_OPTIONS.getNamespace(), readTime), QUERY); verify(mockDatastore, times(1)).runQuery(gqlRequest); verify(mockDatastore, times(1)).runQuery(gqlRequestWithZeroLimit); @@ -1106,18 +1058,11 @@ public void testTranslateGqlQueryWithNoLimit() throws Exception { GqlQuery.newBuilder().setQueryString(gqlWithZeroLimit).setAllowLiterals(true).build(); RunQueryRequest gqlRequestWithZeroLimit = - makeRequest( - PROJECT_ID, DATABASE_ID, gqlQueryWithZeroLimit, V_1_OPTIONS.getNamespace(), readTime); + makeRequest(gqlQueryWithZeroLimit, V_1_OPTIONS.getNamespace(), readTime); when(mockDatastore.runQuery(gqlRequestWithZeroLimit)) .thenReturn(RunQueryResponse.newBuilder().setQuery(QUERY).build()); assertEquals( - translateGqlQueryWithLimitCheck( - gql, - mockDatastore, - V_1_OPTIONS.getProjectId(), - V_1_OPTIONS.getDatabaseId(), - V_1_OPTIONS.getNamespace(), - readTime), + translateGqlQueryWithLimitCheck(gql, mockDatastore, V_1_OPTIONS.getNamespace(), readTime), QUERY); verify(mockDatastore, times(1)).runQuery(gqlRequestWithZeroLimit); } @@ -1129,20 +1074,13 @@ public void testTranslateGqlQueryWithException() throws Exception { GqlQuery gqlQueryWithZeroLimit = GqlQuery.newBuilder().setQueryString(gqlWithZeroLimit).setAllowLiterals(true).build(); RunQueryRequest gqlRequestWithZeroLimit = - makeRequest( - PROJECT_ID, DATABASE_ID, gqlQueryWithZeroLimit, V_1_OPTIONS.getNamespace(), readTime); + makeRequest(gqlQueryWithZeroLimit, V_1_OPTIONS.getNamespace(), readTime); when(mockDatastore.runQuery(gqlRequestWithZeroLimit)) .thenThrow(new RuntimeException("TestException")); thrown.expect(RuntimeException.class); thrown.expectMessage("TestException"); - translateGqlQueryWithLimitCheck( - gql, - mockDatastore, - V_1_OPTIONS.getProjectId(), - V_1_OPTIONS.getDatabaseId(), - V_1_OPTIONS.getNamespace(), - readTime); + translateGqlQueryWithLimitCheck(gql, mockDatastore, V_1_OPTIONS.getNamespace(), readTime); } } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/SplitQueryFnIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/SplitQueryFnIT.java index 913e05c86129..ea00821f3604 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/SplitQueryFnIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/SplitQueryFnIT.java @@ -24,7 +24,6 @@ import java.util.List; import org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.Read.SplitQueryFn; import org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.Read.V1Options; -import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider; import org.apache.beam.sdk.transforms.DoFnTester; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Duration; @@ -59,32 +58,29 @@ public class SplitQueryFnIT { @Test public void testSplitQueryFnWithLargeDataset() throws Exception { String projectId = "apache-beam-testing"; - String databaseId = ""; String kind = "sort_1G"; String namespace = null; // Num splits is computed based on the entity_bytes size of the input_sort_1G kind reported by // Datastore stats. int expectedNumSplits = 32; - testSplitQueryFn(projectId, databaseId, kind, namespace, expectedNumSplits, null); - testSplitQueryFn(projectId, databaseId, kind, namespace, expectedNumSplits, readTime); + testSplitQueryFn(projectId, kind, namespace, expectedNumSplits, null); + testSplitQueryFn(projectId, kind, namespace, expectedNumSplits, readTime); } /** Tests {@link SplitQueryFn} to fallback to NUM_QUERY_SPLITS_MIN for a small dataset. */ @Test public void testSplitQueryFnWithSmallDataset() throws Exception { String projectId = "apache-beam-testing"; - String databaseId = ""; String kind = "shakespeare"; String namespace = null; int expectedNumSplits = NUM_QUERY_SPLITS_MIN; - testSplitQueryFn(projectId, databaseId, kind, namespace, expectedNumSplits, null); - testSplitQueryFn(projectId, databaseId, kind, namespace, expectedNumSplits, readTime); + testSplitQueryFn(projectId, kind, namespace, expectedNumSplits, null); + testSplitQueryFn(projectId, kind, namespace, expectedNumSplits, readTime); } /** A helper method to test {@link SplitQueryFn} to generate the expected number of splits. */ private void testSplitQueryFn( String projectId, - String databaseId, String kind, @Nullable String namespace, int expectedNumSplits, @@ -94,10 +90,7 @@ private void testSplitQueryFn( query.addKindBuilder().setName(kind); SplitQueryFn splitQueryFn = - new SplitQueryFn( - V1Options.from(projectId, StaticValueProvider.of(databaseId), namespace, null), - 0, - readTime); + new SplitQueryFn(V1Options.from(projectId, namespace, null), 0, readTime); DoFnTester<Query, Query> doFnTester = DoFnTester.of(splitQueryFn); List<Query> queries = doFnTester.processBundle(query.build()); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1ReadIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1ReadIT.java index 5918ec203d9a..249cadd48866 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1ReadIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1ReadIT.java @@ -31,7 +31,6 @@ import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; import org.apache.beam.sdk.io.gcp.datastore.V1TestUtil.UpsertMutationBuilder; import org.apache.beam.sdk.io.gcp.datastore.V1TestUtil.V1TestWriter; -import org.apache.beam.sdk.io.gcp.firestore.FirestoreOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; @@ -49,7 +48,6 @@ public class V1ReadIT { private V1TestOptions options; private String project; - private String database; private String ancestor; private final long numEntitiesBeforeReadTime = 600; private final long totalNumEntities = 1000; @@ -60,11 +58,9 @@ public void setup() throws Exception { PipelineOptionsFactory.register(V1TestOptions.class); options = TestPipeline.testingPipelineOptions().as(V1TestOptions.class); project = TestPipeline.testingPipelineOptions().as(GcpOptions.class).getProject(); - database = TestPipeline.testingPipelineOptions().as(FirestoreOptions.class).getFirestoreDb(); - ancestor = UUID.randomUUID().toString(); // Create entities and write them to datastore - writeEntitiesToDatastore(options, project, database, ancestor, 0, numEntitiesBeforeReadTime); + writeEntitiesToDatastore(options, project, ancestor, 0, numEntitiesBeforeReadTime); Thread.sleep(1000); readTime = Instant.now(); @@ -72,12 +68,12 @@ public void setup() throws Exception { long moreEntitiesToWrite = totalNumEntities - numEntitiesBeforeReadTime; writeEntitiesToDatastore( - options, project, database, ancestor, numEntitiesBeforeReadTime, moreEntitiesToWrite); + options, project, ancestor, numEntitiesBeforeReadTime, moreEntitiesToWrite); } @After public void tearDown() throws Exception { - deleteAllEntities(options, project, database, ancestor); + deleteAllEntities(options, project, ancestor); } /** @@ -97,7 +93,6 @@ public void testE2EV1Read() throws Exception { DatastoreIO.v1() .read() .withProjectId(project) - .withDatabaseId(database) .withQuery(query) .withNamespace(options.getNamespace()); @@ -113,7 +108,6 @@ public void testE2EV1Read() throws Exception { DatastoreIO.v1() .read() .withProjectId(project) - .withDatabaseId(database) .withQuery(query) .withNamespace(options.getNamespace()) .withReadTime(readTime); @@ -158,7 +152,6 @@ private void testE2EV1ReadWithGQLQuery(long limit) throws Exception { DatastoreIO.v1() .read() .withProjectId(project) - .withDatabaseId(database) .withLiteralGqlQuery(gqlQuery) .withNamespace(options.getNamespace()); @@ -174,7 +167,6 @@ private void testE2EV1ReadWithGQLQuery(long limit) throws Exception { DatastoreIO.v1() .read() .withProjectId(project) - .withDatabaseId(database) .withLiteralGqlQuery(gqlQuery) .withNamespace(options.getNamespace()) .withReadTime(readTime); @@ -189,17 +181,11 @@ private void testE2EV1ReadWithGQLQuery(long limit) throws Exception { // Creates entities and write them to datastore private static void writeEntitiesToDatastore( - V1TestOptions options, - String project, - String database, - String ancestor, - long valueOffset, - long numEntities) + V1TestOptions options, String project, String ancestor, long valueOffset, long numEntities) throws Exception { - Datastore datastore = getDatastore(options, project, database); + Datastore datastore = getDatastore(options, project); // Write test entities to datastore - V1TestWriter writer = - new V1TestWriter(datastore, project, database, new UpsertMutationBuilder()); + V1TestWriter writer = new V1TestWriter(datastore, new UpsertMutationBuilder()); Key ancestorKey = makeAncestorKey(options.getNamespace(), options.getKind(), ancestor); for (long i = 0; i < numEntities; i++) { diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestUtil.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestUtil.java index 0cb3ffac3222..e3057c4b90ca 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestUtil.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestUtil.java @@ -137,12 +137,10 @@ public void processElement(ProcessContext c) throws Exception { } /** Build a new datastore client. */ - static Datastore getDatastore( - PipelineOptions pipelineOptions, String projectId, String databaseId) { + static Datastore getDatastore(PipelineOptions pipelineOptions, String projectId) { Credentials credential = pipelineOptions.as(GcpOptions.class).getGcpCredential(); HttpRequestInitializer initializer; if (credential != null) { - initializer = new ChainingHttpRequestInitializer( new HttpCredentialsAdapter(credential), new RetryHttpRequestInitializer()); @@ -151,22 +149,14 @@ static Datastore getDatastore( } DatastoreOptions.Builder builder = - new DatastoreOptions.Builder() - .projectId(projectId) - .databaseId(databaseId) - .initializer(initializer); + new DatastoreOptions.Builder().projectId(projectId).initializer(initializer); return DatastoreFactory.get().create(builder.build()); } /** Build a datastore query request. */ - private static RunQueryRequest makeRequest( - String projectId, String databaseId, Query query, @Nullable String namespace) { - RunQueryRequest.Builder requestBuilder = - RunQueryRequest.newBuilder() - .setQuery(query) - .setProjectId(projectId) - .setDatabaseId(databaseId); + private static RunQueryRequest makeRequest(Query query, @Nullable String namespace) { + RunQueryRequest.Builder requestBuilder = RunQueryRequest.newBuilder().setQuery(query); if (namespace != null) { requestBuilder.getPartitionIdBuilder().setNamespaceId(namespace); } @@ -174,16 +164,14 @@ private static RunQueryRequest makeRequest( } /** Delete all entities with the given ancestor. */ - static void deleteAllEntities( - V1TestOptions options, String project, String database, String ancestor) throws Exception { - Datastore datastore = getDatastore(options, project, database); + static void deleteAllEntities(V1TestOptions options, String project, String ancestor) + throws Exception { + Datastore datastore = getDatastore(options, project); Query query = V1TestUtil.makeAncestorKindQuery(options.getKind(), options.getNamespace(), ancestor); - V1TestReader reader = - new V1TestReader(datastore, project, database, query, options.getNamespace()); - V1TestWriter writer = - new V1TestWriter(datastore, project, database, new DeleteMutationBuilder()); + V1TestReader reader = new V1TestReader(datastore, query, options.getNamespace()); + V1TestWriter writer = new V1TestWriter(datastore, new DeleteMutationBuilder()); long numEntities = 0; while (reader.advance()) { @@ -197,15 +185,14 @@ static void deleteAllEntities( } /** Returns the total number of entities for the given datastore. */ - static long countEntities(V1TestOptions options, String project, String database, String ancestor) + static long countEntities(V1TestOptions options, String project, String ancestor) throws Exception { // Read from datastore. - Datastore datastore = V1TestUtil.getDatastore(options, project, database); + Datastore datastore = V1TestUtil.getDatastore(options, project); Query query = V1TestUtil.makeAncestorKindQuery(options.getKind(), options.getNamespace(), ancestor); - V1TestReader reader = - new V1TestReader(datastore, project, database, query, options.getNamespace()); + V1TestReader reader = new V1TestReader(datastore, query, options.getNamespace()); long numEntitiesRead = 0; while (reader.advance()) { @@ -260,17 +247,12 @@ static boolean isValidKey(Key key) { return (lastElement.getId() != 0 || !lastElement.getName().isEmpty()); } - private final String projectId; - private final String databaseId; private final Datastore datastore; private final MutationBuilder mutationBuilder; private final List<Entity> entities = new ArrayList<>(); - V1TestWriter( - Datastore datastore, String projectId, String databaseId, MutationBuilder mutationBuilder) { + V1TestWriter(Datastore datastore, MutationBuilder mutationBuilder) { this.datastore = datastore; - this.projectId = projectId; - this.databaseId = databaseId; this.mutationBuilder = mutationBuilder; } @@ -313,8 +295,6 @@ private void flushBatch() throws DatastoreException, IOException, InterruptedExc commitRequest.addMutations(mutationBuilder.apply(entity)); } commitRequest.setMode(CommitRequest.Mode.NON_TRANSACTIONAL); - commitRequest.setProjectId(projectId); - commitRequest.setDatabaseId(databaseId); datastore.commit(commitRequest.build()); // Break if the commit threw no exception. break; @@ -338,8 +318,6 @@ private void flushBatch() throws DatastoreException, IOException, InterruptedExc static class V1TestReader { private static final int QUERY_BATCH_LIMIT = 500; private final Datastore datastore; - private final String projectId; - private final String databaseId; private final Query query; private final @Nullable String namespace; private boolean moreResults; @@ -348,15 +326,8 @@ static class V1TestReader { private QueryResultBatch currentBatch; private Entity currentEntity; - V1TestReader( - Datastore datastore, - String projectId, - String databaseId, - Query query, - @Nullable String namespace) { + V1TestReader(Datastore datastore, Query query, @Nullable String namespace) { this.datastore = datastore; - this.projectId = projectId; - this.databaseId = databaseId; this.query = query; this.namespace = namespace; } @@ -391,7 +362,7 @@ private Iterator<EntityResult> getIteratorAndMoveCursor() throws DatastoreExcept query.setStartCursor(currentBatch.getEndCursor()); } - RunQueryRequest request = makeRequest(projectId, databaseId, query.build(), namespace); + RunQueryRequest request = makeRequest(query.build(), namespace); RunQueryResponse response = datastore.runQuery(request); currentBatch = response.getBatch(); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1WriteIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1WriteIT.java index 0062208630f6..4f794a03e58d 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1WriteIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1WriteIT.java @@ -56,7 +56,6 @@ public class V1WriteIT { private V1TestOptions options; private String project; - private String database = ""; private String ancestor; private final long numEntities = 1000; @@ -84,12 +83,12 @@ public void testE2EV1Write() throws Exception { ParDo.of( new V1TestUtil.CreateEntityFn( options.getKind(), options.getNamespace(), ancestor, 0))) - .apply(DatastoreIO.v1().write().withProjectId(project).withDatabaseId(database)); + .apply(DatastoreIO.v1().write().withProjectId(project)); p.run(); // Count number of entities written to datastore. - long numEntitiesWritten = countEntities(options, project, database, ancestor); + long numEntitiesWritten = countEntities(options, project, ancestor); assertEquals(numEntities, numEntitiesWritten); } @@ -186,18 +185,18 @@ public void testE2EV1WriteWithLargeEntities() throws Exception { ParDo.of( new V1TestUtil.CreateEntityFn( options.getKind(), options.getNamespace(), ancestor, rawPropertySize))) - .apply(DatastoreIO.v1().write().withProjectId(project).withDatabaseId(database)); + .apply(DatastoreIO.v1().write().withProjectId(project)); p.run(); // Count number of entities written to datastore. - long numEntitiesWritten = countEntities(options, project, database, ancestor); + long numEntitiesWritten = countEntities(options, project, ancestor); assertEquals(numLargeEntities, numEntitiesWritten); } @After public void tearDown() throws Exception { - deleteAllEntities(options, project, database, ancestor); + deleteAllEntities(options, project, ancestor); } } From 9acf947780e3438e76245b572dcb068dead857ed Mon Sep 17 00:00:00 2001 From: Robert Burke <lostluck@users.noreply.github.com> Date: Mon, 6 Nov 2023 12:08:49 -0800 Subject: [PATCH 382/435] [prism] Add fusion. (#29306) --- .../go/pkg/beam/runners/direct/direct_test.go | 98 ++++ sdks/go/pkg/beam/runners/prism/README.md | 3 +- .../beam/runners/prism/internal/execute.go | 6 - .../runners/prism/internal/execute_test.go | 105 +++- .../runners/prism/internal/handlecombine.go | 17 +- .../runners/prism/internal/handlepardo.go | 37 +- .../runners/prism/internal/handlerunner.go | 61 +- .../beam/runners/prism/internal/preprocess.go | 526 ++++++++++++------ .../runners/prism/internal/preprocess_test.go | 78 +-- .../runners/prism/internal/separate_test.go | 3 +- .../pkg/beam/runners/prism/internal/stage.go | 44 +- .../prism/internal/unimplemented_test.go | 1 - 12 files changed, 731 insertions(+), 248 deletions(-) diff --git a/sdks/go/pkg/beam/runners/direct/direct_test.go b/sdks/go/pkg/beam/runners/direct/direct_test.go index a8108580aa2e..bcfce612da33 100644 --- a/sdks/go/pkg/beam/runners/direct/direct_test.go +++ b/sdks/go/pkg/beam/runners/direct/direct_test.go @@ -29,6 +29,7 @@ import ( "github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph/mtime" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph/window" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/metrics" + "github.com/apache/beam/sdks/v2/go/pkg/beam/register" "github.com/apache/beam/sdks/v2/go/pkg/beam/transforms/filter" "github.com/google/go-cmp/cmp" ) @@ -542,3 +543,100 @@ func TestMain(m *testing.M) { beam.Init() os.Exit(m.Run()) } + +func init() { + // Basic Registration + // beam.RegisterFunction(identity) + // beam.RegisterType(reflect.TypeOf((*source)(nil))) + // beam.RegisterType(reflect.TypeOf((*discard)(nil))) + + // Generic registration + register.Function2x0(identity) + register.DoFn2x0[[]byte, func(int)]((*source)(nil)) + register.DoFn1x0[int]((*discard)(nil)) + register.Emitter1[int]() +} + +type source struct { + Count int +} + +func (fn *source) ProcessElement(_ []byte, emit func(int)) { + for i := 0; i < fn.Count; i++ { + emit(i) + } +} + +func identity(v int, emit func(int)) { + emit(v) +} + +type discard struct { + processed int +} + +func (fn *discard) ProcessElement(int) { + fn.processed++ +} + +// BenchmarkPipe checks basic throughput and exec overhead with everything registered. +// +// Just registered: ~700-900ns per call, 330B per DoFn, across 5 allocs per DoFn +// +// goos: linux +// goarch: amd64 +// pkg: github.com/apache/beam/sdks/v2/go/pkg/beam/runners/direct +// cpu: 12th Gen Intel(R) Core(TM) i7-1260P +// BenchmarkPipe/dofns=0-16 1657698 763.0 ns/op 10.49 MB/s 763.0 ns/elm 320 B/op 6 allocs/op +// BenchmarkPipe/dofns=1-16 832784 1294 ns/op 12.37 MB/s 1294 ns/elm 656 B/op 11 allocs/op +// BenchmarkPipe/dofns=2-16 633345 1798 ns/op 13.35 MB/s 899.0 ns/elm 992 B/op 16 allocs/op +// BenchmarkPipe/dofns=3-16 471106 2446 ns/op 13.08 MB/s 815.4 ns/elm 1329 B/op 21 allocs/op +// BenchmarkPipe/dofns=5-16 340099 3634 ns/op 13.21 MB/s 726.8 ns/elm 2001 B/op 31 allocs/op +// BenchmarkPipe/dofns=10-16 183429 6957 ns/op 12.65 MB/s 695.7 ns/elm 3683 B/op 56 allocs/op +// BenchmarkPipe/dofns=100-16 17956 65986 ns/op 12.25 MB/s 659.9 ns/elm 33975 B/op 506 allocs/op +// +// Optimized w/ Generic reg: ~200-300ns per call, 150B per DoFn, across 2 allocs per DoFn +// +// goos: linux +// goarch: amd64 +// pkg: github.com/apache/beam/sdks/v2/go/pkg/beam/runners/direct +// cpu: 12th Gen Intel(R) Core(TM) i7-1260P +// BenchmarkPipe/dofns=0-16 9319206 131.5 ns/op 60.85 MB/s 131.5 ns/elm 152 B/op 2 allocs/op +// BenchmarkPipe/dofns=1-16 4465477 268.3 ns/op 59.63 MB/s 268.3 ns/elm 304 B/op 3 allocs/op +// BenchmarkPipe/dofns=2-16 2876710 431.9 ns/op 55.56 MB/s 216.0 ns/elm 456 B/op 5 allocs/op +// BenchmarkPipe/dofns=3-16 2096349 562.1 ns/op 56.93 MB/s 187.4 ns/elm 608 B/op 7 allocs/op +// BenchmarkPipe/dofns=5-16 1347927 823.8 ns/op 58.27 MB/s 164.8 ns/elm 912 B/op 11 allocs/op +// BenchmarkPipe/dofns=10-16 737594 1590 ns/op 55.36 MB/s 159.0 ns/elm 1672 B/op 21 allocs/op +// BenchmarkPipe/dofns=100-16 60728 19696 ns/op 41.02 MB/s 197.0 ns/elm 15357 B/op 201 allocs/op +func BenchmarkPipe(b *testing.B) { + makeBench := func(numDoFns int) func(b *testing.B) { + return func(b *testing.B) { + b.ReportAllocs() + b.SetBytes(8 * int64(numDoFns+1)) + + disc := &discard{} + p, s := beam.NewPipelineWithRoot() + imp := beam.Impulse(s) + src := beam.ParDo(s, &source{Count: b.N}, imp) + iden := src + for i := 0; i < numDoFns; i++ { + iden = beam.ParDo(s, identity, iden) + } + beam.ParDo0(s, disc, iden) + Execute(context.TODO(), p) + if disc.processed != b.N { + b.Fatalf("processed dodn't match bench number: got %v want %v", disc.processed, b.N) + } + d := b.Elapsed() + div := numDoFns + if div == 0 { + div = 1 + } + div = div * b.N + b.ReportMetric(float64(d)/float64(div), "ns/elm") + } + } + for _, numDoFns := range []int{0, 1, 2, 3, 5, 10, 100} { + b.Run(fmt.Sprintf("dofns=%d", numDoFns), makeBench(numDoFns)) + } +} diff --git a/sdks/go/pkg/beam/runners/prism/README.md b/sdks/go/pkg/beam/runners/prism/README.md index 7ad9dc1d4579..0be9ca5617dc 100644 --- a/sdks/go/pkg/beam/runners/prism/README.md +++ b/sdks/go/pkg/beam/runners/prism/README.md @@ -152,6 +152,8 @@ can have features selectively disabled to ensure * Progess tracking * Channel Splitting * Dynamic Splitting +* FnAPI Optimizations + * Fusion ## Next feature short list (unordered) @@ -165,7 +167,6 @@ See https://github.com/apache/beam/issues/24789 for current status. * Support SDK Containers via Testcontainers * Cross Language Transforms * FnAPI Optimizations - * Fusion * Data with ProcessBundleRequest & Response This is not a comprehensive feature set, but a set of goals to best diff --git a/sdks/go/pkg/beam/runners/prism/internal/execute.go b/sdks/go/pkg/beam/runners/prism/internal/execute.go index c1ac6ea4488c..5e07e161dd5c 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/execute.go +++ b/sdks/go/pkg/beam/runners/prism/internal/execute.go @@ -104,7 +104,6 @@ func makeWorker(env string, j *jobservices.Job) (*worker.W, error) { type transformExecuter interface { ExecuteUrns() []string - ExecuteWith(t *pipepb.PTransform) string ExecuteTransform(stageID, tid string, t *pipepb.PTransform, comps *pipepb.Components, watermark mtime.Time, data [][]byte) *worker.B } @@ -166,11 +165,6 @@ func executePipeline(ctx context.Context, wks map[string]*worker.W, j *jobservic urn := t.GetSpec().GetUrn() stage.exe = proc.transformExecuters[urn] - // Stopgap until everythinng's moved to handlers. - stage.envID = t.GetEnvironmentId() - if stage.exe != nil { - stage.envID = stage.exe.ExecuteWith(t) - } stage.ID = fmt.Sprintf("stage-%03d", i) wk := wks[stage.envID] diff --git a/sdks/go/pkg/beam/runners/prism/internal/execute_test.go b/sdks/go/pkg/beam/runners/prism/internal/execute_test.go index 1a5ae7989a06..ce821bef8985 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/execute_test.go +++ b/sdks/go/pkg/beam/runners/prism/internal/execute_test.go @@ -37,7 +37,7 @@ import ( "github.com/apache/beam/sdks/v2/go/test/integration/primitives" ) -func initRunner(t *testing.T) { +func initRunner(t testing.TB) { t.Helper() if *jobopts.Endpoint == "" { s := jobservices.NewServer(0, internal.RunPipeline) @@ -64,7 +64,7 @@ func execute(ctx context.Context, p *beam.Pipeline) (beam.PipelineResult, error) return universal.Execute(ctx, p) } -func executeWithT(ctx context.Context, t *testing.T, p *beam.Pipeline) (beam.PipelineResult, error) { +func executeWithT(ctx context.Context, t testing.TB, p *beam.Pipeline) (beam.PipelineResult, error) { t.Log("startingTest - ", t.Name()) s1 := rand.NewSource(time.Now().UnixNano()) r1 := rand.New(s1) @@ -587,3 +587,104 @@ func init() { func TestMain(m *testing.M) { ptest.MainWithDefault(m, "testlocal") } + +func init() { + // Basic Registration + // beam.RegisterFunction(identity) + // beam.RegisterType(reflect.TypeOf((*source)(nil))) + // beam.RegisterType(reflect.TypeOf((*discard)(nil))) + + // Generic registration + register.Function2x0(identity) + register.DoFn2x0[[]byte, func(int)]((*source)(nil)) + register.DoFn1x0[int]((*discard)(nil)) + register.Emitter1[int]() +} + +type source struct { + Count int +} + +func (fn *source) ProcessElement(_ []byte, emit func(int)) { + for i := 0; i < fn.Count; i++ { + emit(i) + } +} + +func identity(v int, emit func(int)) { + emit(v) +} + +type discard struct { + processed int +} + +func (fn *discard) ProcessElement(int) { + fn.processed++ +} + +// BenchmarkPipe checks basic throughput and exec overhead with everything registered. +// +// No fusion (all elements encoded) (generic registration): +// +// ~2000ns per call, 2000B per DoFn, across 22 allocs per DoFn +// (using Basic regsitration adds 3 allocs per DoFn, a ~200 bytes, and ~200-400ns/elm) +// +// goos: linux +// goarch: amd64 +// pkg: github.com/apache/beam/sdks/v2/go/pkg/beam/runners/direct +// cpu: 12th Gen Intel(R) Core(TM) i7-1260P +// BenchmarkPipe/dofns=0-16 885811 1333 ns/op 1333 ns/elm 1993 B/op 22 allocs/op +// BenchmarkPipe/dofns=1-16 457683 2636 ns/op 2636 ns/elm 3986 B/op 44 allocs/op +// BenchmarkPipe/dofns=2-16 283699 3975 ns/op 1988 ns/elm 6138 B/op 66 allocs/op +// BenchmarkPipe/dofns=3-16 212767 5689 ns/op 1896 ns/elm 8504 B/op 88 allocs/op +// BenchmarkPipe/dofns=5-16 121842 8279 ns/op 1656 ns/elm 11994 B/op 132 allocs/op +// BenchmarkPipe/dofns=10-16 22059 52877 ns/op 5288 ns/elm 30614 B/op 443 allocs/op +// BenchmarkPipe/dofns=100-16 6614 166364 ns/op 1664 ns/elm 192961 B/op 2261 allocs/op +// +// With fusion (generic registration): +// ~200ns per call, 150B per DoFn, across 2 allocs per DoFn +// AKA comparible to Direct Runner, as expected. +// +// goos: linux +// goarch: amd64 +// pkg: github.com/apache/beam/sdks/v2/go/pkg/beam/runners/direct +// cpu: 12th Gen Intel(R) Core(TM) i7-1260P +// BenchmarkPipe/dofns=0-16 7660638 145.8 ns/op 145.8 ns/elm 152 B/op 2 allocs/op +// BenchmarkPipe/dofns=1-16 3676358 313.3 ns/op 313.3 ns/elm 304 B/op 4 allocs/op +// BenchmarkPipe/dofns=2-16 2242688 507.4 ns/op 253.7 ns/elm 457 B/op 6 allocs/op +// BenchmarkPipe/dofns=3-16 1726969 662.6 ns/op 220.9 ns/elm 610 B/op 8 allocs/op +// BenchmarkPipe/dofns=5-16 1198765 1005 ns/op 201.0 ns/elm 915 B/op 12 allocs/op +// BenchmarkPipe/dofns=10-16 631459 1874 ns/op 187.4 ns/elm 1679 B/op 22 allocs/op +// BenchmarkPipe/dofns=100-16 57926 19890 ns/op 198.9 ns/elm 15660 B/op 206 allocs/op +func BenchmarkPipe(b *testing.B) { + initRunner(b) + makeBench := func(numDoFns int) func(b *testing.B) { + return func(b *testing.B) { + b.ReportAllocs() + disc := &discard{} + p, s := beam.NewPipelineWithRoot() + imp := beam.Impulse(s) + src := beam.ParDo(s, &source{Count: b.N}, imp) + iden := src + for i := 0; i < numDoFns; i++ { + iden = beam.ParDo(s, identity, iden) + } + beam.ParDo0(s, disc, iden) + _, err := execute(context.Background(), p) + if err != nil { + b.Fatal(err) + } + d := b.Elapsed() + div := numDoFns + if div == 0 { + div = 1 + } + div = div * b.N + b.ReportMetric(float64(d)/float64(div), "ns/elm") + } + } + for _, numDoFns := range []int{0, 1, 2, 3, 5, 10, 100} { + b.Run(fmt.Sprintf("dofns=%d", numDoFns), makeBench(numDoFns)) + } +} diff --git a/sdks/go/pkg/beam/runners/prism/internal/handlecombine.go b/sdks/go/pkg/beam/runners/prism/internal/handlecombine.go index ff9bd1e1c88a..3f31ad77fd53 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/handlecombine.go +++ b/sdks/go/pkg/beam/runners/prism/internal/handlecombine.go @@ -58,10 +58,16 @@ func (*combine) PrepareUrns() []string { } // PrepareTransform returns lifted combines and removes the leaves if enabled. Otherwise returns nothing. -func (h *combine) PrepareTransform(tid string, t *pipepb.PTransform, comps *pipepb.Components) (*pipepb.Components, []string) { +func (h *combine) PrepareTransform(tid string, t *pipepb.PTransform, comps *pipepb.Components) prepareResult { // If we aren't lifting, the "default impl" for combines should be sufficient. if !h.config.EnableLifting { - return nil, nil + return prepareResult{ + SubbedComps: &pipepb.Components{ + Transforms: map[string]*pipepb.PTransform{ + tid: t, + }, + }, + } } // To lift a combine, the spec should contain a CombinePayload. @@ -197,7 +203,7 @@ func (h *combine) PrepareTransform(tid string, t *pipepb.PTransform, comps *pipe liftEID: tform(liftEID, urns.TransformPreCombine, pcolInID, liftedNID, t.GetEnvironmentId()), gbkEID: tform(gbkEID, urns.TransformGBK, liftedNID, groupedNID, ""), mergeEID: tform(mergeEID, urns.TransformMerge, groupedNID, mergedNID, t.GetEnvironmentId()), - extractEID: tform(mergeEID, urns.TransformExtract, mergedNID, pcolOutID, t.GetEnvironmentId()), + extractEID: tform(extractEID, urns.TransformExtract, mergedNID, pcolOutID, t.GetEnvironmentId()), }, } @@ -205,5 +211,8 @@ func (h *combine) PrepareTransform(tid string, t *pipepb.PTransform, comps *pipe // TODO recurse through sub transforms to remove? // We don't need to remove the composite, since we don't add it in // when we return the new transforms, so it's not in the topology. - return newComps, t.GetSubtransforms() + return prepareResult{ + SubbedComps: newComps, + RemovedLeaves: t.GetSubtransforms(), + } } diff --git a/sdks/go/pkg/beam/runners/prism/internal/handlepardo.go b/sdks/go/pkg/beam/runners/prism/internal/handlepardo.go index 2ac5ca5bbf59..e9d422ca107d 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/handlepardo.go +++ b/sdks/go/pkg/beam/runners/prism/internal/handlepardo.go @@ -58,7 +58,7 @@ func (*pardo) PrepareUrns() []string { // PrepareTransform handles special processing with respect to ParDos, since their handling is dependant on supported features // and requirements. -func (h *pardo) PrepareTransform(tid string, t *pipepb.PTransform, comps *pipepb.Components) (*pipepb.Components, []string) { +func (h *pardo) PrepareTransform(tid string, t *pipepb.PTransform, comps *pipepb.Components) prepareResult { // ParDos are a pain in the butt. // Combines, by comparison, are dramatically simpler. @@ -89,21 +89,22 @@ func (h *pardo) PrepareTransform(tid string, t *pipepb.PTransform, comps *pipepb // so they're not included here. Any nearly any ParDo can have them. // At their simplest, we don't need to do anything special at pre-processing time, and simply pass through as normal. - return &pipepb.Components{ - Transforms: map[string]*pipepb.PTransform{ - tid: t, + return prepareResult{ + SubbedComps: &pipepb.Components{ + Transforms: map[string]*pipepb.PTransform{ + tid: t, + }, }, - }, nil + } } // Side inputs add to topology and make fusion harder to deal with // (side input producers can't be in the same stage as their consumers) - // But we don't have fusion yet, so no worries. // State, Timers, Stable Input, Time Sorted Input, and some parts of SDF - // Are easier to deal including a fusion break. But We can do that with a - // runner specific transform for stable input, and another for timesorted - // input. + // Are easier to deal with by including a fusion break. But we can do that with a + // runner specific transform for stable input, and another for time sorted input. + // TODO add // SplittableDoFns have 3 required phases and a 4th optional phase. // @@ -235,10 +236,16 @@ func (h *pardo) PrepareTransform(tid string, t *pipepb.PTransform, comps *pipepb EnvironmentId: t.GetEnvironmentId(), }, } - - return &pipepb.Components{ - Coders: coders, - Pcollections: pcols, - Transforms: tforms, - }, t.GetSubtransforms() + return prepareResult{ + SubbedComps: &pipepb.Components{ + Coders: coders, + Pcollections: pcols, + Transforms: tforms, + }, + RemovedLeaves: t.GetSubtransforms(), + // Force ProcessSized to be a root to ensure SDFs are able to split + // between elements or within elements. + // Also this is where a transform would be stateful anyway. + ForcedRoots: []string{eProcessID}, + } } diff --git a/sdks/go/pkg/beam/runners/prism/internal/handlerunner.go b/sdks/go/pkg/beam/runners/prism/internal/handlerunner.go index 3f699e47e675..59e926754821 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/handlerunner.go +++ b/sdks/go/pkg/beam/runners/prism/internal/handlerunner.go @@ -67,13 +67,55 @@ func (*runner) ConfigCharacteristic() reflect.Type { var _ transformPreparer = (*runner)(nil) func (*runner) PrepareUrns() []string { - return []string{urns.TransformReshuffle} + return []string{urns.TransformReshuffle, urns.TransformFlatten} } // PrepareTransform handles special processing with respect runner transforms, like reshuffle. -func (h *runner) PrepareTransform(tid string, t *pipepb.PTransform, comps *pipepb.Components) (*pipepb.Components, []string) { +func (h *runner) PrepareTransform(tid string, t *pipepb.PTransform, comps *pipepb.Components) prepareResult { + switch t.GetSpec().GetUrn() { + case urns.TransformFlatten: + return h.handleFlatten(tid, t, comps) + case urns.TransformReshuffle: + return h.handleReshuffle(tid, t, comps) + default: + panic("unknown urn to Prepare: " + t.GetSpec().GetUrn()) + } +} + +func (h *runner) handleFlatten(tid string, t *pipepb.PTransform, comps *pipepb.Components) prepareResult { + if !h.config.SDKFlatten { + t.EnvironmentId = "" // force the flatten to be a runner transform due to configuration. + forcedRoots := []string{tid} // Have runner side transforms be roots. + + // Force runner flatten consumers to be roots. + // This resolves merges between two runner transforms trying + // to execute together. + outColID := getOnlyValue(t.GetOutputs()) + for ctid, t := range comps.GetTransforms() { + for _, gi := range t.GetInputs() { + if gi == outColID { + forcedRoots = append(forcedRoots, ctid) + } + } + } + + // Return the new components which is the transforms consumer + return prepareResult{ + // We sub this flatten with itself, to not drop it. + SubbedComps: &pipepb.Components{ + Transforms: map[string]*pipepb.PTransform{ + tid: t, + }, + }, + RemovedLeaves: nil, + ForcedRoots: forcedRoots, + } + } + return prepareResult{} +} + +func (h *runner) handleReshuffle(tid string, t *pipepb.PTransform, comps *pipepb.Components) prepareResult { // TODO: Implement the windowing strategy the "backup" transforms used for Reshuffle. - // TODO: Implement a fusion break for reshuffles. if h.config.SDKReshuffle { panic("SDK side reshuffle not yet supported") @@ -106,12 +148,15 @@ func (h *runner) PrepareTransform(tid string, t *pipepb.PTransform, comps *pipep // We need to remove the consumers of the output PCollection. toRemove := []string{} + // We need to force the consumers to be stage root, + // because reshuffle should be a fusion break. + forcedRoots := []string{} - for _, t := range comps.GetTransforms() { + for tid, t := range comps.GetTransforms() { for li, gi := range t.GetInputs() { if gi == outColID { - // The whole s t.GetInputs()[li] = inColID + forcedRoots = append(forcedRoots, tid) } } } @@ -120,7 +165,11 @@ func (h *runner) PrepareTransform(tid string, t *pipepb.PTransform, comps *pipep toRemove = append(toRemove, t.GetSubtransforms()...) // Return the new components which is the transforms consumer - return nil, toRemove + return prepareResult{ + SubbedComps: nil, // Replace the reshuffle with nothing. + RemovedLeaves: toRemove, + ForcedRoots: forcedRoots, + } } var _ transformExecuter = (*runner)(nil) diff --git a/sdks/go/pkg/beam/runners/prism/internal/preprocess.go b/sdks/go/pkg/beam/runners/prism/internal/preprocess.go index bca40709626d..fb244cb4fbbb 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/preprocess.go +++ b/sdks/go/pkg/beam/runners/prism/internal/preprocess.go @@ -23,6 +23,7 @@ import ( pipepb "github.com/apache/beam/sdks/v2/go/pkg/beam/model/pipeline_v1" "github.com/apache/beam/sdks/v2/go/pkg/beam/runners/prism/internal/urns" "golang.org/x/exp/maps" + "golang.org/x/exp/slices" "golang.org/x/exp/slog" "google.golang.org/protobuf/encoding/prototext" ) @@ -35,7 +36,13 @@ type transformPreparer interface { PrepareUrns() []string // PrepareTransform takes a PTransform proto and returns a set of new Components, and a list of // transformIDs leaves to remove and ignore from graph processing. - PrepareTransform(tid string, t *pipepb.PTransform, comps *pipepb.Components) (*pipepb.Components, []string) + PrepareTransform(tid string, t *pipepb.PTransform, comps *pipepb.Components) prepareResult +} + +type prepareResult struct { + SubbedComps *pipepb.Components + RemovedLeaves []string + ForcedRoots []string } // preprocessor retains configuration for preprocessing the @@ -73,6 +80,7 @@ func (p *preprocessor) preProcessGraph(comps *pipepb.Components) []*stage { // TODO move this out of this part of the pre-processor? leaves := map[string]struct{}{} ignore := map[string]struct{}{} + forcedRoots := map[string]bool{} for tid, t := range ts { if _, ok := ignore[tid]; ok { continue @@ -106,30 +114,33 @@ func (p *preprocessor) preProcessGraph(comps *pipepb.Components) []*stage { continue } - subs, toRemove := h.PrepareTransform(tid, t, comps) + prepResult := h.PrepareTransform(tid, t, comps) // Clear out unnecessary leaves from this composite for topological sort handling. - for _, key := range toRemove { + for _, key := range prepResult.RemovedLeaves { ignore[key] = struct{}{} delete(leaves, key) } + for _, key := range prepResult.ForcedRoots { + forcedRoots[key] = true + } // ts should be a clone, so we should be able to add new transforms into the map. - for tid, t := range subs.GetTransforms() { + for tid, t := range prepResult.SubbedComps.GetTransforms() { leaves[tid] = struct{}{} ts[tid] = t } - for cid, c := range subs.GetCoders() { + for cid, c := range prepResult.SubbedComps.GetCoders() { comps.GetCoders()[cid] = c } - for nid, n := range subs.GetPcollections() { + for nid, n := range prepResult.SubbedComps.GetPcollections() { comps.GetPcollections()[nid] = n } // It's unlikely for these to change, but better to handle them now, to save a headache later. - for wid, w := range subs.GetWindowingStrategies() { + for wid, w := range prepResult.SubbedComps.GetWindowingStrategies() { comps.GetWindowingStrategies()[wid] = w } - for envid, env := range subs.GetEnvironments() { + for envid, env := range prepResult.SubbedComps.GetEnvironments() { comps.GetEnvironments()[envid] = env } } @@ -141,167 +152,217 @@ func (p *preprocessor) preProcessGraph(comps *pipepb.Components) []*stage { topological := pipelinex.TopologicalSort(ts, keptLeaves) slog.Debug("topological transform ordering", slog.Any("topological", topological)) - // Basic Fusion Behavior - // - // Fusion is the practice of executing associated DoFns in the same stage. - // This often leads to more efficient processing, since costly encode/decode or - // serialize/deserialize operations can be elided. In Beam, any PCollection can - // in principle serve as a place for serializing and deserializing elements. - // - // In particular, Fusion is a stage for optimizing pipeline execution, and was - // described in the FlumeJava paper, in section 4. - // https://static.googleusercontent.com/media/research.google.com/en//pubs/archive/35650.pdf - // - // Per the FlumeJava paper, there are two primary opportunities for Fusion, - // Producer+Consumer fusion and Sibling fusion. - // - // Producer+Consumer fusion is when the producer of a PCollection and the consumers of - // that PCollection are combined into a single stage. Sibling fusion is when two consumers - // of the same pcollection are fused into the same step. These processes can continue until - // graph structure or specific transforms dictate that fusion may not proceed futher. - // - // Examples of fusion breaks include GroupByKeys, or requiring side inputs to complete - // processing for downstream processing, since the producer and consumer of side inputs - // cannot be in the same fused stage. - // - // Additionally, at this phase, we can consider different optimizations for execution. - // For example "Flatten unzipping". In practice, there's no requirement for any stages - // to have an explicit "Flatten" present in the graph. A flatten can be "unzipped", - // duplicating the consumming transforms after the flatten, until a subsequent fusion break. - // This enables additional parallelism by allowing sources to operate in their own independant - // stages. Beam supports this naturally with the separation of work into independant - // bundles for execution. - - return defaultFusion(topological, comps) -} + facts := computeFacts(topological, comps) + facts.forcedRoots = forcedRoots -// defaultFusion is the base strategy for prism, that doesn't seek to optimize execution -// with fused stages. Input is the set of leaf nodes we're going to execute, topologically -// sorted, and the pipeline components. -// -// Default fusion behavior: Don't. Prism is intended to test all of Beam, which often -// means for testing purposes, to execute pipelines without optimization. -// -// Special Exception to unfused Go SDK pipelines. -// -// If a transform, after a GBK step, has a single input with a KV<K, Iter<X>> coder -// and a single output O with a KV<K, Iter<Y>> coder, and if then it must be fused with -// the consumers of O. -func defaultFusion(topological []string, comps *pipepb.Components) []*stage { - var stages []*stage - - // TODO figure out a better place to source the PCol Parents/Consumers analysis - // so we don't keep repeating it. - - pcolParents, pcolConsumers := computPColFacts(topological, comps) - - // Explicitly list the pcollectionID we want to fuse along. - fuseWithConsumers := map[string]string{} - for _, tid := range topological { - t := comps.GetTransforms()[tid] - - // See if this transform has a single input and output - if len(t.GetInputs()) != 1 || len(t.GetOutputs()) != 1 { - continue - } - inputID := getOnlyValue(t.GetInputs()) - outputID := getOnlyValue(t.GetOutputs()) - - parentLink := pcolParents[inputID] - - parent := comps.GetTransforms()[parentLink.transform] - - // Check if the input source is a GBK - if parent.GetSpec().GetUrn() != urns.TransformGBK { - continue - } - - // Check if the coder is a KV<K, Iter<?>> - iCID := comps.GetPcollections()[inputID].GetCoderId() - oCID := comps.GetPcollections()[outputID].GetCoderId() - - if checkForExpandCoderPattern(iCID, oCID, comps) { - fuseWithConsumers[tid] = outputID - } - } - - // Since we iterate in topological order, we're guaranteed to process producers before consumers. - consumed := map[string]bool{} // Checks if we've already handled a transform already due to fusion. - for _, tid := range topological { - if consumed[tid] { - continue - } - stg := &stage{ - transforms: []string{tid}, - } - // TODO validate that fused stages have the same environment. - stg.envID = comps.GetTransforms()[tid].EnvironmentId - - stages = append(stages, stg) - - pcolID, ok := fuseWithConsumers[tid] - if !ok { - continue - } - cs := pcolConsumers[pcolID] - - for _, c := range cs { - stg.transforms = append(stg.transforms, c.transform) - consumed[c.transform] = true - } - } + return greedyFusion(topological, comps, facts) +} - for _, stg := range stages { - prepareStage(stg, comps, pcolConsumers) - } - return stages +// TODO(lostluck): Be able to toggle this in variants. +// Most likely, re-implement in terms of simply marking all transforms as forced roots. +// Commented out to avoid the unused staticheck, but it's worth keeping until the docs +// and implementation is re-added. + +// // defaultFusion is the base strategy for prism, that doesn't seek to optimize execution +// // with fused stages. Input is the set of leaf nodes we're going to execute, topologically +// // sorted, and the pipeline components. +// // +// // Default fusion behavior: Don't. Prism is intended to test all of Beam, which often +// // means for testing purposes, to execute pipelines without optimization. +// // +// // Special Exception to unfused Go SDK pipelines. +// // +// // If a transform, after a GBK step, has a single input with a KV<K, Iter<X>> coder +// // and a single output O with a KV<K, Iter<Y>> coder, and if then it must be fused with +// // the consumers of O. +// func defaultFusion(topological []string, comps *pipepb.Components, facts fusionFacts) []*stage { +// // Basic Fusion Behavior +// // +// // Fusion is the practice of executing associated DoFns in the same stage. +// // This often leads to more efficient processing, since costly encode/decode or +// // serialize/deserialize operations can be elided. In Beam, any PCollection can +// // in principle serve as a place for serializing and deserializing elements. +// // +// // In particular, Fusion is a stage for optimizing pipeline execution, and was +// // described in the FlumeJava paper, in section 4. +// // https://static.googleusercontent.com/media/research.google.com/en//pubs/archive/35650.pdf +// // +// // Per the FlumeJava paper, there are two primary opportunities for Fusion, +// // Producer+Consumer fusion and Sibling fusion. +// // +// // Producer+Consumer fusion is when the producer of a PCollection and the consumers of +// // that PCollection are combined into a single stage. Sibling fusion is when two consumers +// // of the same pcollection are fused into the same step. These processes can continue until +// // graph structure or specific transforms dictate that fusion may not proceed futher. +// // +// // Examples of fusion breaks include GroupByKeys, or requiring side inputs to complete +// // processing for downstream processing, since the producer and consumer of side inputs +// // cannot be in the same fused stage. +// // +// // Additionally, at this phase, we can consider different optimizations for execution. +// // For example "Flatten unzipping". In practice, there's no requirement for any stages +// // to have an explicit "Flatten" present in the graph. A flatten can be "unzipped", +// // duplicating the consumming transforms after the flatten, until a subsequent fusion break. +// // This enables additional parallelism by allowing sources to operate in their own independant +// // stages. Beam supports this naturally with the separation of work into independant +// // bundles for execution. + +// // Explicitly list the pcollectionID we want to fuse along. +// fuseWithConsumers := map[string]string{} +// for _, tid := range topological { +// t := comps.GetTransforms()[tid] + +// // See if this transform has a single input and output +// if len(t.GetInputs()) != 1 || len(t.GetOutputs()) != 1 { +// continue +// } +// inputID := getOnlyValue(t.GetInputs()) +// outputID := getOnlyValue(t.GetOutputs()) + +// producerLink := facts.pcolProducers[inputID] + +// producer := comps.GetTransforms()[producerLink.transform] + +// // Check if the input source is a GBK +// if producer.GetSpec().GetUrn() != urns.TransformGBK { +// continue +// } + +// // Check if the coder is a KV<K, Iter<?>> +// iCID := comps.GetPcollections()[inputID].GetCoderId() +// oCID := comps.GetPcollections()[outputID].GetCoderId() + +// if checkForExpandCoderPattern(iCID, oCID, comps) { +// fuseWithConsumers[tid] = outputID +// } +// } + +// var stages []*stage +// // Since we iterate in topological order, we're guaranteed to process producers before consumers. +// consumed := map[string]bool{} // Checks if we've already handled a transform already due to fusion. +// for _, tid := range topological { +// if consumed[tid] { +// continue +// } +// stg := &stage{ +// transforms: []string{tid}, +// } +// // TODO validate that fused stages have the same environment. +// stg.envID = comps.GetTransforms()[tid].EnvironmentId + +// stages = append(stages, stg) + +// pcolID, ok := fuseWithConsumers[tid] +// if !ok { +// continue +// } +// cs := facts.pcolConsumers[pcolID] + +// for _, c := range cs { +// stg.transforms = append(stg.transforms, c.transform) +// consumed[c.transform] = true +// } +// } + +// for _, stg := range stages { +// prepareStage(stg, comps, facts) +// } +// return stages +// } + +// // We need to see that both coders have this pattern: KV<K, Iter<?>> +// func checkForExpandCoderPattern(in, out string, comps *pipepb.Components) bool { +// isKV := func(id string) bool { +// return comps.GetCoders()[id].GetSpec().GetUrn() == urns.CoderKV +// } +// getComp := func(id string, i int) string { +// return comps.GetCoders()[id].GetComponentCoderIds()[i] +// } +// isIter := func(id string) bool { +// return comps.GetCoders()[id].GetSpec().GetUrn() == urns.CoderIterable +// } +// if !isKV(in) || !isKV(out) { +// return false +// } +// // Are the keys identical? +// if getComp(in, 0) != getComp(out, 0) { +// return false +// } +// // Are both values iterables? +// if isIter(getComp(in, 1)) && isIter(getComp(out, 1)) { +// // If so we have the ExpandCoderPattern from the Go SDK. Hurray! +// return true +// } +// return false +// } + +type fusionFacts struct { + pcolProducers map[string]link // global pcol ID to transform link that produces it. + pcolConsumers map[string][]link // global pcol ID to all consumers of that pcollection + usedAsSideInput map[string]bool // global pcol ID and if it's used as a side input + + directSideInputs map[string]map[string]bool // global transform ID and all direct side input pcollections. + downstreamSideInputs map[string]map[string]bool // global transform ID and all transitive side input pcollections. + + forcedRoots map[string]bool // transforms forced to be roots (not computed in computeFacts) } -// computPColFacts computes a map of PCollectionIDs to their parent transforms, and a map of -// PCollectionIDs to their consuming transforms. -func computPColFacts(topological []string, comps *pipepb.Components) (map[string]link, map[string][]link) { - pcolParents := map[string]link{} - pcolConsumers := map[string][]link{} +// computeFacts computes facts about the given set of transforms and components that +// are useful for fusion. +func computeFacts(topological []string, comps *pipepb.Components) fusionFacts { + ret := fusionFacts{ + pcolProducers: map[string]link{}, + pcolConsumers: map[string][]link{}, + usedAsSideInput: map[string]bool{}, + directSideInputs: map[string]map[string]bool{}, // direct set + downstreamSideInputs: map[string]map[string]bool{}, // transitive set + } // Use the topological ids so each PCollection only has a single - // parent. We've already pruned out composites at this stage. + // producer. We've already pruned out composites at this stage. for _, tID := range topological { t := comps.GetTransforms()[tID] for local, global := range t.GetOutputs() { - pcolParents[global] = link{transform: tID, local: local, global: global} + ret.pcolProducers[global] = link{transform: tID, local: local, global: global} + } + sis, err := getSideInputs(t) + if err != nil { + panic(err) } + directSIs := map[string]bool{} + ret.directSideInputs[tID] = directSIs for local, global := range t.GetInputs() { - pcolConsumers[global] = append(pcolConsumers[global], link{transform: tID, local: local, global: global}) + ret.pcolConsumers[global] = append(ret.pcolConsumers[global], link{transform: tID, local: local, global: global}) + if _, ok := sis[local]; ok { + ret.usedAsSideInput[global] = true + directSIs[global] = true + } } } - return pcolParents, pcolConsumers + for _, tID := range topological { + computeDownstreamSideInputs(tID, comps, ret) + } + + return ret } -// We need to see that both coders have this pattern: KV<K, Iter<?>> -func checkForExpandCoderPattern(in, out string, comps *pipepb.Components) bool { - isKV := func(id string) bool { - return comps.GetCoders()[id].GetSpec().GetUrn() == urns.CoderKV - } - getComp := func(id string, i int) string { - return comps.GetCoders()[id].GetComponentCoderIds()[i] - } - isIter := func(id string) bool { - return comps.GetCoders()[id].GetSpec().GetUrn() == urns.CoderIterable - } - if !isKV(in) || !isKV(out) { - return false - } - // Are the keys identical? - if getComp(in, 0) != getComp(out, 0) { - return false +func computeDownstreamSideInputs(tID string, comps *pipepb.Components, facts fusionFacts) map[string]bool { + if dssi, ok := facts.downstreamSideInputs[tID]; ok { + return dssi } - // Are both values iterables? - if isIter(getComp(in, 1)) && isIter(getComp(out, 1)) { - // If so we have the ExpandCoderPattern from the Go SDK. Hurray! - return true + dssi := map[string]bool{} + for _, o := range comps.GetTransforms()[tID].GetOutputs() { + if facts.usedAsSideInput[o] { + dssi[o] = true + } + for _, consumer := range facts.pcolConsumers[o] { + cdssi := computeDownstreamSideInputs(consumer.global, comps, facts) + maps.Copy(dssi, cdssi) + } } - return false + facts.downstreamSideInputs[tID] = dssi + return dssi } // prepareStage does the final pre-processing step for stages: @@ -309,7 +370,7 @@ func checkForExpandCoderPattern(in, out string, comps *pipepb.Components) bool { // 1. Determining the single parallel input (may be 0 for impulse stages). // 2. Determining all outputs to the stages. // 3. Determining all side inputs. -// 4 validating that no side input is fed by an internal PCollection. +// 4 Validating that no side input is fed by an internal PCollection. // 4. Check that all transforms are in the same environment or are environment agnostic. (TODO for xlang) // 5. Validate that only the primary input consuming transform are stateful. (Might be able to relax this) // @@ -320,22 +381,22 @@ func checkForExpandCoderPattern(in, out string, comps *pipepb.Components) bool { // Finally, it takes this information and caches it in the stage for simpler descriptor construction downstream. // // Note, this is very similar to the work done WRT composites in pipelinex.Normalize. -func prepareStage(stg *stage, comps *pipepb.Components, pipelineConsumers map[string][]link) { +func prepareStage(stg *stage, comps *pipepb.Components, pipelineFacts fusionFacts) { // Collect all PCollections involved in this stage. - pcolParents, pcolConsumers := computPColFacts(stg.transforms, comps) + stageFacts := computeFacts(stg.transforms, comps) transformSet := map[string]bool{} for _, tid := range stg.transforms { transformSet[tid] = true } - // Now we can see which consumers (inputs) aren't covered by the parents (outputs). + // Now we can see which consumers (inputs) aren't covered by the producers (outputs). mainInputs := map[string]string{} var sideInputs []link inputs := map[string]bool{} - for pid, plinks := range pcolConsumers { + for pid, plinks := range stageFacts.pcolConsumers { // Check if this PCollection is generated in this bundle. - if _, ok := pcolParents[pid]; ok { + if _, ok := stageFacts.pcolProducers[pid]; ok { // It is, so we will ignore for now. continue } @@ -354,10 +415,10 @@ func prepareStage(stg *stage, comps *pipepb.Components, pipelineConsumers map[st outputs := map[string]link{} var internal []string // Look at all PCollections produced in this stage. - for pid, link := range pcolParents { + for pid, link := range stageFacts.pcolProducers { // Look at all consumers of this PCollection in the pipeline isInternal := true - for _, l := range pipelineConsumers[pid] { + for _, l := range pipelineFacts.pcolConsumers[pid] { // If the consuming transform isn't in the stage, it's an output. if !transformSet[l.transform] { isInternal = false @@ -384,10 +445,149 @@ func prepareStage(stg *stage, comps *pipepb.Components, pipelineConsumers map[st if l := len(mainInputs); l == 1 { stg.primaryInput = getOnlyValue(mainInputs) } else if l > 1 { - // Quick check that this is a lone flatten node, which is handled runner side anyway - // and only sent SDK side as part of a fused stage. - if !(len(stg.transforms) == 1 && comps.GetTransforms()[stg.transforms[0]].GetSpec().GetUrn() == urns.TransformFlatten) { - panic("expected flatten node, but wasn't") + // Quick check that this is lead by a flatten node, and that it's handled runner side. + t := comps.GetTransforms()[stg.transforms[0]] + if !(t.GetSpec().GetUrn() == urns.TransformFlatten && t.GetEnvironmentId() == "") { + panic("expected runner flatten node, but wasn't") + } + } +} + +// greedyFusion produces a pipeline as tightly fused as possible. +// +// Fusion is a critical optimization for performance of pipeline execution. +// Thus it's important for SDKs to be capable of executing transforms in a fused state. +// +// However, not all transforms can be fused into the same stage together. +// Further, some transforms must be at the root of a stage. +// +// # Fusion Restrictions +// +// Environments: Transforms that aren't in the same environment can't be +// fused together *unless* their environments can also be fused together. +// Eg. Resource hints can often be ignored for local runners. +// +// Side Inputs: A transform S consuming a PCollection as a side input can't +// be fused with the transform P that produces that PCollection. Further, +// no transform S+ descended from S, can be fused with transform P. +// +// Splittable DoFns: An expanded Splittable DoFn transform's Process Sized +// Elements and Restrictions component must be the root of a stage. +// +// State and Timers: Stateful Transforms (transforms using State and Timers) +// must be the root of transforms, since they are required to be keyed. +// A sequence of Key Preserving stateful transforms could be fused. +// +// TODO: Sink/Unzip Flattens so they vanish from the graph. +// +// This approach is largely cribed from the Python approach at +// fn_api_runner/translations.py. That implementation is very set oriented & +// eagerly adds data source/sink transforms, while prism does so later in +// stage construction. +func greedyFusion(topological []string, comps *pipepb.Components, facts fusionFacts) []*stage { + fused := map[int]int{} + stageAssignments := map[string]int{} + + stageEnvs := map[int]string{} + forcedRoots := map[int]bool{} + directSIs := map[int]map[string]bool{} + downstreamSIs := map[int]map[string]bool{} + + var index int + replacements := func(tID string) int { + sID, ok := stageAssignments[tID] + if !ok { // No stage exists yet. + sID = index + index++ + + t := comps.GetTransforms()[tID] + stageAssignments[tID] = sID + stageEnvs[sID] = t.GetEnvironmentId() + forcedRoots[sID] = facts.forcedRoots[tID] + directSIs[sID] = maps.Clone(facts.directSideInputs[tID]) + downstreamSIs[sID] = maps.Clone(facts.downstreamSideInputs[tID]) + } + + var oldIDs []int + rep, ok := fused[sID] + for ok { + oldIDs = append(oldIDs, sID) + sID = rep + rep, ok = fused[sID] } + // Update the assignment & fusions for path shortening. + stageAssignments[tID] = sID + for _, old := range oldIDs { + fused[old] = sID + } + return sID + } + + overlap := func(downstream, consumer map[string]bool) bool { + for si := range consumer { + if downstream[si] { + return true + } + } + return false + } + + // To start, every transform is in it's own stage. + // So we map a transformID to a stageID. + // We go through each PCollection, (facts.PcolProducers) and + // try to fuse the producer to each consumer of that PCollection. + // + // If we can fuse, the consumer takes on the producer's stageID, + // and the assignments are updated. + + // Use the topological sort instead? + + keys := maps.Keys(facts.pcolProducers) + slices.Sort(keys) + for _, pcol := range keys { + producer := facts.pcolProducers[pcol] + for _, consumer := range facts.pcolConsumers[pcol] { + pID := replacements(producer.transform) // Get current stage for producer + cID := replacements(consumer.transform) // Get current stage for consumer + + // See if there's anything preventing fusion: + if pID == cID { + continue // Already fused together. + } + if stageEnvs[pID] != stageEnvs[cID] { + continue // Not the same environment. + } + if forcedRoots[cID] { + continue // Forced root. + } + if overlap(downstreamSIs[pID], directSIs[cID]) { + continue // Side input conflict + } + + // In principle, we can fuse! + fused[cID] = pID // Set the consumer to be in the producer's stage. + // Copy the consumer's direct and downstream side input sets into the producer. + maps.Copy(directSIs[pID], directSIs[cID]) + maps.Copy(downstreamSIs[pID], downstreamSIs[cID]) + } + } + + var stages []*stage + fusedToStages := map[int]*stage{} + for _, tID := range topological { + sID := replacements(tID) + s := fusedToStages[sID] + if s == nil { + s = &stage{ + envID: stageEnvs[sID], + } + fusedToStages[sID] = s + stages = append(stages, s) + } + s.transforms = append(s.transforms, tID) } + for _, stg := range stages { + prepareStage(stg, comps, facts) + } + return stages } diff --git a/sdks/go/pkg/beam/runners/prism/internal/preprocess_test.go b/sdks/go/pkg/beam/runners/prism/internal/preprocess_test.go index ba39d024e716..8d0d6accdf6d 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/preprocess_test.go +++ b/sdks/go/pkg/beam/runners/prism/internal/preprocess_test.go @@ -31,6 +31,7 @@ func Test_preprocessor_preProcessGraph(t *testing.T) { wantComponents *pipepb.Components wantStages []*stage + forcedRoots []string }{ { name: "noPreparer", @@ -57,7 +58,8 @@ func Test_preprocessor_preProcessGraph(t *testing.T) { }, }, }, { - name: "preparer", + name: "preparer", + forcedRoots: []string{"e1_early", "e1_late"}, input: &pipepb.Components{ Transforms: map[string]*pipepb.PTransform{ "e1": { @@ -125,7 +127,9 @@ func Test_preprocessor_preProcessGraph(t *testing.T) { } for _, test := range tests { t.Run(test.name, func(t *testing.T) { - pre := newPreprocessor([]transformPreparer{&testPreparer{}}) + pre := newPreprocessor([]transformPreparer{&testPreparer{ + ForcedRoots: test.forcedRoots, + }}) gotStages := pre.preProcessGraph(test.input) if diff := cmp.Diff(test.wantStages, gotStages, cmp.AllowUnexported(stage{}, link{}), cmpopts.EquateEmpty()); diff != "" { @@ -139,47 +143,53 @@ func Test_preprocessor_preProcessGraph(t *testing.T) { } } -type testPreparer struct{} +type testPreparer struct { + ForcedRoots []string +} func (p *testPreparer) PrepareUrns() []string { return []string{"test_urn"} } -func (p *testPreparer) PrepareTransform(tid string, t *pipepb.PTransform, comps *pipepb.Components) (*pipepb.Components, []string) { - return &pipepb.Components{ - Transforms: map[string]*pipepb.PTransform{ - "e1_early": { - UniqueName: "e1_early", - Spec: &pipepb.FunctionSpec{ - Urn: "defaultUrn", +func (p *testPreparer) PrepareTransform(tid string, t *pipepb.PTransform, comps *pipepb.Components) prepareResult { + return prepareResult{ + ForcedRoots: p.ForcedRoots, + SubbedComps: &pipepb.Components{ + Transforms: map[string]*pipepb.PTransform{ + "e1_early": { + UniqueName: "e1_early", + Spec: &pipepb.FunctionSpec{ + Urn: "defaultUrn", + }, + Outputs: map[string]string{"i0": "pcol1"}, + EnvironmentId: "env1", + }, + "e1_late": { + UniqueName: "e1_late", + Spec: &pipepb.FunctionSpec{ + Urn: "defaultUrn", + }, + Inputs: map[string]string{"i0": "pcol1"}, + EnvironmentId: "env1", }, - Outputs: map[string]string{"i0": "pcol1"}, - EnvironmentId: "env1", }, - "e1_late": { - UniqueName: "e1_late", - Spec: &pipepb.FunctionSpec{ - Urn: "defaultUrn", + Pcollections: map[string]*pipepb.PCollection{ + "pcol1": { + UniqueName: "pcol1", + CoderId: "coder1", + WindowingStrategyId: "ws1", }, - Inputs: map[string]string{"i0": "pcol1"}, - EnvironmentId: "env1", }, - }, - Pcollections: map[string]*pipepb.PCollection{ - "pcol1": { - UniqueName: "pcol1", - CoderId: "coder1", - WindowingStrategyId: "ws1", + Coders: map[string]*pipepb.Coder{ + "coder1": {Spec: &pipepb.FunctionSpec{Urn: "coder1"}}, + }, + WindowingStrategies: map[string]*pipepb.WindowingStrategy{ + "ws1": {WindowCoderId: "global"}, + }, + Environments: map[string]*pipepb.Environment{ + "env1": {Urn: "env1"}, }, }, - Coders: map[string]*pipepb.Coder{ - "coder1": {Spec: &pipepb.FunctionSpec{Urn: "coder1"}}, - }, - WindowingStrategies: map[string]*pipepb.WindowingStrategy{ - "ws1": {WindowCoderId: "global"}, - }, - Environments: map[string]*pipepb.Environment{ - "env1": {Urn: "env1"}, - }, - }, []string{"e1"} + RemovedLeaves: []string{"e1"}, + } } diff --git a/sdks/go/pkg/beam/runners/prism/internal/separate_test.go b/sdks/go/pkg/beam/runners/prism/internal/separate_test.go index 97ae494e4abb..1be3d3e70841 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/separate_test.go +++ b/sdks/go/pkg/beam/runners/prism/internal/separate_test.go @@ -141,6 +141,7 @@ func TestSeparation(t *testing.T) { count := 10 imp := beam.Impulse(s) ints := beam.ParDo(s, emitTenFn, imp) + ints = beam.Reshuffle(s, ints) out := beam.ParDo(s, &sepHarness{ Base: sepHarnessBase{ WatcherID: ws.newWatcher(3), @@ -379,7 +380,7 @@ func (fn *sepHarnessBase) setup() error { sepWaitMap[fn.WatcherID] = c go func(id int, c chan struct{}) { for { - time.Sleep(time.Second * 1) // Check counts every second. + time.Sleep(time.Millisecond * 50) // Check counts every second. sepClientMu.Lock() var unblock bool err := sepClient.Call("Watchers.Check", &Args{WatcherID: id}, &unblock) diff --git a/sdks/go/pkg/beam/runners/prism/internal/stage.go b/sdks/go/pkg/beam/runners/prism/internal/stage.go index 4ce3ce7ffeb6..4925405bb4ef 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/stage.go +++ b/sdks/go/pkg/beam/runners/prism/internal/stage.go @@ -128,12 +128,32 @@ func (s *stage) Execute(ctx context.Context, j *jobservices.Job, wk *worker.W, d previousIndex := int64(-2) var splitsDone bool progTick := time.NewTicker(100 * time.Millisecond) + defer progTick.Stop() + var dataFinished, bundleFinished bool + // If we have no data outputs, we still need to have progress & splits + // while waiting for bundle completion. + if b.OutputCount == 0 { + dataFinished = true + } + var resp *fnpb.ProcessBundleResponse progress: for { select { + case <-ctx.Done(): + return context.Cause(ctx) + case resp = <-b.Resp: + bundleFinished = true + if b.BundleErr != nil { + return b.BundleErr + } + if dataFinished && bundleFinished { + break progress // exit progress loop on close. + } case <-dataReady: - progTick.Stop() - break progress // exit progress loop on close. + dataFinished = true + if dataFinished && bundleFinished { + break progress // exit progress loop on close. + } case <-progTick.C: resp, err := b.Progress(ctx, wk) if err != nil { @@ -145,9 +165,10 @@ progress: md := wk.MonitoringMetadata(ctx, unknownIDs) j.AddMetricShortIDs(md) } - slog.Debug("progress report", "bundle", rb, "index", index) + slog.Debug("progress report", "bundle", rb, "index", index, "prevIndex", previousIndex) // Progress for the bundle hasn't advanced. Try splitting. if previousIndex == index && !splitsDone { + slog.Debug("splitting report", "bundle", rb, "index", index) sr, err := b.Split(ctx, wk, 0.5 /* fraction of remainder */, nil /* allowed splits */) if err != nil { slog.Warn("SDK Error from split, aborting splits", "bundle", rb, "error", err.Error()) @@ -187,16 +208,6 @@ progress: // Tentative Data is ready, commit it to the main datastore. slog.Debug("Execute: commiting data", "bundle", rb, slog.Any("outputsWithData", maps.Keys(b.OutputData.Raw)), slog.Any("outputs", maps.Keys(s.OutputsToCoders))) - var resp *fnpb.ProcessBundleResponse - select { - case resp = <-b.Resp: - if b.BundleErr != nil { - return b.BundleErr - } - case <-ctx.Done(): - return context.Cause(ctx) - } - // Tally metrics immeadiately so they're available before // pipeline termination. unknownIDs := j.ContributeFinalMetrics(resp) @@ -279,6 +290,9 @@ func buildDescriptor(stg *stage, comps *pipepb.Components, wk *worker.W, ds *wor for _, tid := range stg.transforms { transforms[tid] = comps.GetTransforms()[tid] } + if len(transforms) == 0 { + return fmt.Errorf("buildDescriptor: invalid stage - no transforms at all %v", stg.ID) + } // Start with outputs, since they're simple and uniform. sink2Col := map[string]string{} @@ -287,7 +301,7 @@ func buildDescriptor(stg *stage, comps *pipepb.Components, wk *worker.W, ds *wor col := comps.GetPcollections()[o.global] wOutCid, err := makeWindowedValueCoder(o.global, comps, coders) if err != nil { - return fmt.Errorf("buildDescriptor: failed to handle coder on stage %v for output %+v, pcol %q %v:\n%w", stg.ID, o, o.global, prototext.Format(col), err) + return fmt.Errorf("buildDescriptor: failed to handle coder on stage %v for output %+v, pcol %q %v:\n%w %v", stg.ID, o, o.global, prototext.Format(col), err, stg.transforms) } sinkID := o.transform + "_" + o.local ed := collectionPullDecoder(col.GetCoderId(), coders, comps) @@ -343,7 +357,7 @@ func buildDescriptor(stg *stage, comps *pipepb.Components, wk *worker.W, ds *wor col := comps.GetPcollections()[stg.primaryInput] wInCid, err := makeWindowedValueCoder(stg.primaryInput, comps, coders) if err != nil { - return fmt.Errorf("buildDescriptor: failed to handle coder on stage %v for primary input, pcol %q %v:\n%w", stg.ID, stg.primaryInput, prototext.Format(col), err) + return fmt.Errorf("buildDescriptor: failed to handle coder on stage %v for primary input, pcol %q %v:\n%w\n%v", stg.ID, stg.primaryInput, prototext.Format(col), err, stg.transforms) } ed := collectionPullDecoder(col.GetCoderId(), coders, comps) diff --git a/sdks/go/pkg/beam/runners/prism/internal/unimplemented_test.go b/sdks/go/pkg/beam/runners/prism/internal/unimplemented_test.go index 5f8d38759998..b8a04a7306b2 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/unimplemented_test.go +++ b/sdks/go/pkg/beam/runners/prism/internal/unimplemented_test.go @@ -108,7 +108,6 @@ func TestImplemented(t *testing.T) { {pipeline: primitives.Flatten}, {pipeline: primitives.FlattenDup}, {pipeline: primitives.Checkpoints}, - {pipeline: primitives.CoGBK}, {pipeline: primitives.ReshuffleKV}, } From f619affcf704258ec11f5328a39010ba12cc2584 Mon Sep 17 00:00:00 2001 From: Danny McCormick <dannymccormick@google.com> Date: Mon, 6 Nov 2023 15:31:49 -0500 Subject: [PATCH 383/435] Add action to publish RC to PyPi (#29316) --- .../deploy_release_candidate_pypi.yaml | 89 +++++++++++++++++++ 1 file changed, 89 insertions(+) create mode 100644 .github/workflows/deploy_release_candidate_pypi.yaml diff --git a/.github/workflows/deploy_release_candidate_pypi.yaml b/.github/workflows/deploy_release_candidate_pypi.yaml new file mode 100644 index 000000000000..7f873dbdc2ab --- /dev/null +++ b/.github/workflows/deploy_release_candidate_pypi.yaml @@ -0,0 +1,89 @@ +name: deploy_release_candidate_pypi + +# Workflow added after https://github.com/apache/beam/commit/4183e747becebd18becee5fff547af365910fc9c +# If help is needed debugging issues, you can view the release guide at that commit for guidance on how to do this manually. +# (https://github.com/apache/beam/blob/4183e747becebd18becee5fff547af365910fc9c/website/www/site/content/en/contribute/release-guide.md) +on: + workflow_dispatch: + inputs: + RELEASE: + description: Beam version of current release (e.g. 2.XX.0) + required: true + default: '2.XX.0' + RC: + description: Integer RC version for the release (e.g. 3 for RC3) + required: true + PYPI_USER: + description: PyPi username to perform the upload with + required: true + PYPI_PASSWORD: + description: PyPi password to perform the upload with + required: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + deploy_release_candidate_pypi: + runs-on: [self-hosted, ubuntu-20.04, main] + steps: + - name: Mask PyPi id/password + run: | + echo "::add-mask::${{ github.event.inputs.PYPI_USER }}" + echo "::add-mask::${{ github.event.inputs.PYPI_PASSWORD }}" + - name: Checkout + uses: actions/checkout@v4 + with: + ref: "v${{ github.event.inputs.RELEASE }}-RC${{ github.event.inputs.RC }}" + repository: apache/beam + - name: Setup environment + uses: ./.github/actions/setup-environment-action + with: + python-version: 3.11 + - name: Install dependencies + run: | + pip install python-dateutil + pip install requests + pip install twine + - name: Deploy to Pypi + env: + RC_TAG: "v${{ github.event.inputs.RELEASE }}-RC${{ github.event.inputs.RC }}" + GIT_REPO_BASE_URL: https://github.com/apache/beam + RELEASE_DIR: "beam/${{ github.event.inputs.RELEASE }}" + RELEASE: "${{ github.event.inputs.RELEASE }}" + SCRIPT_DIR: release/src/main/scripts + GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} + SVN_ARTIFACTS_DIR: "beam/${{ github.event.inputs.RELEASE }}/python" + run: | + git fetch --all --tags --prune + RELEASE_COMMIT=$(git rev-list -n 1 $RC_TAG) + + PYTHON_ARTIFACTS_DIR="./python" + python "release/src/main/scripts/download_github_actions_artifacts.py" \ + --github-token-var GITHUB_TOKEN \ + --repo-url "apache/beam" \ + --rc-tag "${RC_TAG}" \ + --release-commit "${RELEASE_COMMIT}" \ + --artifacts_dir "${PYTHON_ARTIFACTS_DIR}" \ + --rc_number "${{ github.event.inputs.RC }}" \ + --yes True + + echo "------Checking Hash Value for apache-beam-${RELEASE}rc${RC_NUMBER}.tar.gz-----" + sha512sum -c "apache-beam-${RELEASE}rc${RC_NUMBER}.tar.gz.sha512" + + for artifact in *.whl; do + echo "----------Checking Hash Value for ${artifact} wheel-----------" + sha512sum -c "${artifact}.sha512" + done + + echo "===================Removing sha512 files=======================" + rm $(ls | grep -i ".*.sha512$") + + echo "====================Upload rc to pypi========================" + mkdir dist && mv $(ls | grep apache) dist && cd dist + echo "Will upload the following files to PyPI:" + ls + + twine upload * -u ${{ github.event.inputs.PYPI_USER }} -p ${{ github.event.inputs.PYPI_PASSWORD }} From 3e790032b7d5253010cc8172efe3f4466dfb2c3c Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 6 Nov 2023 15:32:42 -0500 Subject: [PATCH 384/435] Bump golang.org/x/sys from 0.13.0 to 0.14.0 in /sdks (#29309) Bumps [golang.org/x/sys](https://github.com/golang/sys) from 0.13.0 to 0.14.0. - [Commits](https://github.com/golang/sys/compare/v0.13.0...v0.14.0) --- updated-dependencies: - dependency-name: golang.org/x/sys dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] <support@github.com> Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 2 +- sdks/go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 0ee2c233dab4..5ba1ac0109ba 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -55,7 +55,7 @@ require ( golang.org/x/net v0.17.0 golang.org/x/oauth2 v0.13.0 golang.org/x/sync v0.4.0 - golang.org/x/sys v0.13.0 + golang.org/x/sys v0.14.0 golang.org/x/text v0.13.0 google.golang.org/api v0.149.0 google.golang.org/genproto v0.0.0-20231016165738-49dd2c1f3d0b diff --git a/sdks/go.sum b/sdks/go.sum index 6343b5689263..540bc6a8713b 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -600,8 +600,8 @@ golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.8.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.11.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.13.0 h1:Af8nKPmuFypiUBjVoU9V20FiaFXOcuZI21p0ycVYYGE= -golang.org/x/sys v0.13.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.14.0 h1:Vz7Qs629MkJkGyHxUlRHizWJRG2j8fbQKjELVSNhy7Q= +golang.org/x/sys v0.14.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= From c8e7bde45d9ddb976498506e29e0891830a37c35 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 6 Nov 2023 14:12:53 -0800 Subject: [PATCH 385/435] Bump github.com/aws/aws-sdk-go-v2/feature/s3/manager in /sdks (#29288) Bumps [github.com/aws/aws-sdk-go-v2/feature/s3/manager](https://github.com/aws/aws-sdk-go-v2) from 1.13.0 to 1.13.1. - [Release notes](https://github.com/aws/aws-sdk-go-v2/releases) - [Changelog](https://github.com/aws/aws-sdk-go-v2/blob/config/v1.13.1/CHANGELOG.md) - [Commits](https://github.com/aws/aws-sdk-go-v2/compare/v1.13.0...config/v1.13.1) --- updated-dependencies: - dependency-name: github.com/aws/aws-sdk-go-v2/feature/s3/manager dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] <support@github.com> Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 2 +- sdks/go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 5ba1ac0109ba..c3c1c4593f82 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -33,7 +33,7 @@ require ( github.com/aws/aws-sdk-go-v2 v1.22.1 github.com/aws/aws-sdk-go-v2/config v1.22.0 github.com/aws/aws-sdk-go-v2/credentials v1.15.1 - github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.13.0 + github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.13.1 github.com/aws/aws-sdk-go-v2/service/s3 v1.42.0 github.com/aws/smithy-go v1.16.0 github.com/docker/go-connections v0.4.0 diff --git a/sdks/go.sum b/sdks/go.sum index 540bc6a8713b..875ba55d8596 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -95,8 +95,8 @@ github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.3.0/go.mod h1:2LAuqPx1I6jNfaGDu github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.2 h1:gIeH4+o1MN/caGBWjoGQTUTIu94xD6fI5B2+TcwBf70= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.2/go.mod h1:wLyMIo/zPOhQhPXTddpfdkSleyigtFi8iMnC+2m/SK4= github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.3.2/go.mod h1:qaqQiHSrOUVOfKe6fhgQ6UzhxjwqVW8aHNegd6Ws4w4= -github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.13.0 h1:LPTcGCwO6bup9pTuOmYFXlcocpMX8PPO+TIB7CtSVnI= -github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.13.0/go.mod h1:QJXFEiHtNvgiOuBx9cyK6hC28xTB7mQ/iya6Xr1Fu7Y= +github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.13.1 h1:ULswbgGNVrW8zEhkCNwrwXrs1mUvy2JTqWaCRsD2ZZw= +github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.13.1/go.mod h1:pAXgsDPk1rRwwfkz8/9ISO75vXEHqTGIgbLhGqqQ1GY= github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.1 h1:fi1ga6WysOyYb5PAf3Exd6B5GiSNpnZim4h1rhlBqx0= github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.1/go.mod h1:V5CY8wNurvPUibTi9mwqUqpiFZ5LnioKWIFUDtIzdI8= github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.1 h1:ZpaV/j48RlPc4AmOZuPv22pJliXjXq8/reL63YzyFnw= From ad287193c1ee3b47ea17a586a68381ad5065e8ce Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Johanna=20=C3=96jeling?= <51084516+johannaojeling@users.noreply.github.com> Date: Mon, 6 Nov 2023 23:15:23 +0100 Subject: [PATCH 386/435] [#29210] Fix SDF progress fraction to not return NaN (#29307) --- sdks/go/pkg/beam/core/runtime/exec/sdf.go | 11 +- .../go/pkg/beam/core/runtime/exec/sdf_test.go | 117 +++++++++++++----- 2 files changed, 96 insertions(+), 32 deletions(-) diff --git a/sdks/go/pkg/beam/core/runtime/exec/sdf.go b/sdks/go/pkg/beam/core/runtime/exec/sdf.go index b21b47b20ae2..3977d31dda7d 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/sdf.go +++ b/sdks/go/pkg/beam/core/runtime/exec/sdf.go @@ -781,7 +781,7 @@ func (n *ProcessSizedElementsAndRestrictions) singleWindowSplit(ctx context.Cont func (n *ProcessSizedElementsAndRestrictions) multiWindowSplit(ctx context.Context, f float64, pWeState any, rWeState any) ([]*FullValue, []*FullValue, error) { // Get the split point in window range, to see what window it falls in. done, rem := n.rt.GetProgress() - cwp := done / (done + rem) // Progress in current window. + cwp := progressFraction(done, rem) // Progress in current window. p := (float64(n.currW) + cwp) / float64(n.numW) // Progress of whole element. sp := p + (f * (1.0 - p)) // Split point in range of entire element [0, 1]. wsp := sp * float64(n.numW) // Split point in window range [0, numW]. @@ -923,7 +923,7 @@ func (n *ProcessSizedElementsAndRestrictions) newSplitResult(ctx context.Context // DoFns, so 1.0 is only returned once all windows have been processed. func (n *ProcessSizedElementsAndRestrictions) GetProgress() float64 { d, r := n.rt.GetProgress() - frac := d / (d + r) + frac := progressFraction(d, r) if n.numW == 1 { return frac @@ -959,6 +959,13 @@ func (n *ProcessSizedElementsAndRestrictions) GetOutputWatermark() map[string]*t return nil } +func progressFraction(done float64, remaining float64) float64 { + if done == 0 { + return 0 + } + return done / (done + remaining) +} + // SdfFallback is an executor used when an SDF isn't expanded into steps by the // runner, indicating that the runner doesn't support splitting. It executes all // the SDF steps together in one unit. diff --git a/sdks/go/pkg/beam/core/runtime/exec/sdf_test.go b/sdks/go/pkg/beam/core/runtime/exec/sdf_test.go index a0380796e863..460a7111b110 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/sdf_test.go +++ b/sdks/go/pkg/beam/core/runtime/exec/sdf_test.go @@ -715,6 +715,14 @@ func TestAsSplittableUnit(t *testing.T) { currWindow: 0, wantProgress: 0.5, }, + { + name: "SingleWindowZeroWork", + windows: testWindows, + doneWork: 0.0, + remainingWork: 0.0, + currWindow: 0, + wantProgress: 0.0, + }, { name: "MultipleWindows", windows: multiWindows, @@ -724,6 +732,14 @@ func TestAsSplittableUnit(t *testing.T) { // Progress should be halfway through second window. wantProgress: 1.5 / 4.0, }, + { + name: "MultipleWindowsZeroWork", + windows: multiWindows, + doneWork: 0.0, + remainingWork: 0.0, + currWindow: 1, + wantProgress: 1.0 / 4.0, + }, } for _, test := range tests { test := test @@ -776,15 +792,19 @@ func TestAsSplittableUnit(t *testing.T) { name string fn *graph.DoFn frac float64 - doneRt bool // Result that RTracker will return for IsDone. + done float64 + remaining float64 + isDoneRt bool // Result that RTracker will return for IsDone. in FullValue wantPrimaries []*FullValue wantResiduals []*FullValue }{ { - name: "SingleElem", - fn: dfn, - frac: 0.5, + name: "SingleElem", + fn: dfn, + frac: 0.5, + done: 0.0, + remaining: 1.0, in: FullValue{ Elm: &FullValue{ Elm: 1, @@ -823,9 +843,11 @@ func TestAsSplittableUnit(t *testing.T) { }}, }, { - name: "SingleElemStatefulWatermarkEstimating", - fn: statefulWeFn, - frac: 0.5, + name: "SingleElemStatefulWatermarkEstimating", + fn: statefulWeFn, + frac: 0.5, + done: 0.0, + remaining: 1.0, in: FullValue{ Elm: &FullValue{ Elm: 1, @@ -864,9 +886,11 @@ func TestAsSplittableUnit(t *testing.T) { }}, }, { - name: "KvElem", - fn: kvdfn, - frac: 0.5, + name: "KvElem", + fn: kvdfn, + frac: 0.5, + done: 0.0, + remaining: 1.0, in: FullValue{ Elm: &FullValue{ Elm: &FullValue{ @@ -914,10 +938,12 @@ func TestAsSplittableUnit(t *testing.T) { }}, }, { - name: "DoneRTracker", - fn: dfn, - doneRt: true, - frac: 0.5, + name: "DoneRTracker", + fn: dfn, + frac: 0.5, + done: 0.0, + remaining: 1.0, + isDoneRt: true, in: FullValue{ Elm: &FullValue{ Elm: 1, @@ -936,9 +962,11 @@ func TestAsSplittableUnit(t *testing.T) { { // MultiWindow split where split point lands inside currently // processing restriction tracker. - name: "MultiWindow/RestrictionSplit", - fn: dfn, - frac: 0.125, // Should be in the middle of the first (current) window. + name: "MultiWindow/RestrictionSplit", + fn: dfn, + frac: 0.125, // Should be in the middle of the first (current) window. + done: 0.0, + remaining: 1.0, in: FullValue{ Elm: &FullValue{ Elm: 1, @@ -990,9 +1018,11 @@ func TestAsSplittableUnit(t *testing.T) { { // MultiWindow split where the split lands outside the current // window, and performs a window boundary split instead. - name: "MultiWindow/WindowBoundarySplit", - fn: dfn, - frac: 0.55, + name: "MultiWindow/WindowBoundarySplit", + fn: dfn, + frac: 0.55, + done: 0.0, + remaining: 1.0, in: FullValue{ Elm: &FullValue{ Elm: 1, @@ -1033,10 +1063,12 @@ func TestAsSplittableUnit(t *testing.T) { { // Tests that a MultiWindow split with a Done RTracker will // fallback to a window boundary split. - name: "MultiWindow/DoneRTrackerSplit", - fn: dfn, - frac: 0.125, - doneRt: true, + name: "MultiWindow/DoneRTrackerSplit", + fn: dfn, + frac: 0.125, + done: 0.0, + remaining: 1.0, + isDoneRt: true, in: FullValue{ Elm: &FullValue{ Elm: 1, @@ -1077,9 +1109,34 @@ func TestAsSplittableUnit(t *testing.T) { { // Test that if a window boundary split lands at the end of an // element, it results in a no-op. - name: "MultiWindow/NoResidual", - fn: dfn, - frac: 0.95, // Should round to end of element and cause a no-op. + name: "MultiWindow/NoResidual", + fn: dfn, + frac: 0.95, // Should round to end of element and cause a no-op. + done: 0.0, + remaining: 1.0, + in: FullValue{ + Elm: &FullValue{ + Elm: 1, + Elm2: &FullValue{ + Elm: &VetRestriction{ID: "Sdf"}, + Elm2: false, + }, + }, + Elm2: 1.0, + Timestamp: testTimestamp, + Windows: testMultiWindows, + }, + wantPrimaries: []*FullValue{}, + wantResiduals: []*FullValue{}, + }, + { + // Tests that an RTracker progress of 0.0 done and 0.0 remaining + // is treated as a current window progress of 0.0. + name: "MultiWindow/ZeroWork", + fn: dfn, + frac: 0.95, + done: 0.0, + remaining: 0.0, in: FullValue{ Elm: &FullValue{ Elm: 1, @@ -1104,9 +1161,9 @@ func TestAsSplittableUnit(t *testing.T) { node := &ProcessSizedElementsAndRestrictions{PDo: n} node.rt = &SplittableUnitRTracker{ VetRTracker: VetRTracker{Rest: test.in.Elm.(*FullValue).Elm2.(*FullValue).Elm.(*VetRestriction)}, - Done: 0, - Remaining: 1.0, - ThisIsDone: test.doneRt, + Done: test.done, + Remaining: test.remaining, + ThisIsDone: test.isDoneRt, } node.elm = &test.in node.numW = len(test.in.Windows) From cf560ca2f70c53d6c9c625e8a4c6fb13c192e640 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Johanna=20=C3=96jeling?= <51084516+johannaojeling@users.noreply.github.com> Date: Tue, 7 Nov 2023 00:02:13 +0100 Subject: [PATCH 387/435] [Go SDK]: Create natsio.Write transform for writing to NATS (#29184) * Create natsio.Write transform for writing to NATS * Emit representation of acknowledged message from writeFn * Use type map[string][]string for ProduceMessage headers --- sdks/go.mod | 9 +- sdks/go.sum | 18 +- sdks/go/pkg/beam/io/natsio/common.go | 58 +++++ sdks/go/pkg/beam/io/natsio/example_test.go | 55 +++++ sdks/go/pkg/beam/io/natsio/helper_test.go | 130 +++++++++++ sdks/go/pkg/beam/io/natsio/write.go | 114 ++++++++++ sdks/go/pkg/beam/io/natsio/write_option.go | 31 +++ sdks/go/pkg/beam/io/natsio/write_test.go | 252 +++++++++++++++++++++ 8 files changed, 664 insertions(+), 3 deletions(-) create mode 100644 sdks/go/pkg/beam/io/natsio/common.go create mode 100644 sdks/go/pkg/beam/io/natsio/example_test.go create mode 100644 sdks/go/pkg/beam/io/natsio/helper_test.go create mode 100644 sdks/go/pkg/beam/io/natsio/write.go create mode 100644 sdks/go/pkg/beam/io/natsio/write_option.go create mode 100644 sdks/go/pkg/beam/io/natsio/write_test.go diff --git a/sdks/go.mod b/sdks/go.mod index c3c1c4593f82..6f64472eb8a2 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -45,6 +45,8 @@ require ( github.com/johannesboyne/gofakes3 v0.0.0-20221110173912-32fb85c5aed6 github.com/lib/pq v1.10.9 github.com/linkedin/goavro/v2 v2.12.0 + github.com/nats-io/nats-server/v2 v2.10.4 + github.com/nats-io/nats.go v1.31.0 github.com/proullon/ramsql v0.1.3 github.com/spf13/cobra v1.8.0 github.com/testcontainers/testcontainers-go v0.25.0 @@ -76,12 +78,17 @@ require ( github.com/Microsoft/hcsshim v0.11.0 // indirect github.com/go-ole/go-ole v1.2.6 // indirect github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0 // indirect + github.com/minio/highwayhash v1.0.2 // indirect + github.com/nats-io/jwt/v2 v2.5.2 // indirect + github.com/nats-io/nkeys v0.4.6 // indirect + github.com/nats-io/nuid v1.0.1 // indirect github.com/power-devops/perfstat v0.0.0-20210106213030-5aafc221ea8c // indirect github.com/shirou/gopsutil/v3 v3.23.8 // indirect github.com/shoenig/go-m1cpu v0.1.6 // indirect github.com/tklauser/go-sysconf v0.3.12 // indirect github.com/tklauser/numcpus v0.6.1 // indirect github.com/yusufpapurcu/wmi v1.2.3 // indirect + golang.org/x/time v0.3.0 // indirect ) require ( @@ -138,7 +145,7 @@ require ( github.com/inconshreveable/mousetrap v1.1.0 // indirect github.com/jmespath/go-jmespath v0.4.0 // indirect github.com/klauspost/asmfmt v1.3.2 // indirect - github.com/klauspost/compress v1.16.7 // indirect + github.com/klauspost/compress v1.17.2 // indirect github.com/klauspost/cpuid/v2 v2.2.5 // indirect github.com/kr/text v0.2.0 // indirect github.com/magiconair/properties v1.8.7 // indirect diff --git a/sdks/go.sum b/sdks/go.sum index 875ba55d8596..efffec5951a8 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -319,8 +319,8 @@ github.com/klauspost/asmfmt v1.3.2/go.mod h1:AG8TuvYojzulgDAMCnYn50l/5QV3Bs/tp6j github.com/klauspost/compress v1.9.7/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= github.com/klauspost/compress v1.13.1/go.mod h1:8dP1Hq4DHOhN9w426knH3Rhby4rFm6D8eO+e+Dq5Gzg= github.com/klauspost/compress v1.13.6/go.mod h1:/3/Vjq9QcHkK5uEr5lBEmyoZ1iFhe47etQ6QUkpK6sk= -github.com/klauspost/compress v1.16.7 h1:2mk3MPGNzKyxErAw8YaohYh69+pa4sIQSC0fPGCFR9I= -github.com/klauspost/compress v1.16.7/go.mod h1:ntbaceVETuRiXiv4DpjP66DpAtAGkEQskQzEyD//IeE= +github.com/klauspost/compress v1.17.2 h1:RlWWUY/Dr4fL8qk9YG7DTZ7PDgME2V4csBXA8L/ixi4= +github.com/klauspost/compress v1.17.2/go.mod h1:ntbaceVETuRiXiv4DpjP66DpAtAGkEQskQzEyD//IeE= github.com/klauspost/cpuid/v2 v2.2.5 h1:0E5MSMDEoAulmXNFquVs//DdoomxaoTY1kUhbc/qbZg= github.com/klauspost/cpuid/v2 v2.2.5/go.mod h1:Lcz8mBdAVJIBVzewtcLocK12l3Y+JytZYpaMropDUws= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= @@ -343,6 +343,8 @@ github.com/minio/asm2plan9s v0.0.0-20200509001527-cdd76441f9d8 h1:AMFGa4R4MiIpsp github.com/minio/asm2plan9s v0.0.0-20200509001527-cdd76441f9d8/go.mod h1:mC1jAcsrzbxHt8iiaC+zU4b1ylILSosueou12R++wfY= github.com/minio/c2goasm v0.0.0-20190812172519-36a3d3bbc4f3 h1:+n/aFZefKZp7spd8DFdX7uMikMLXX4oubIzJF4kv/wI= github.com/minio/c2goasm v0.0.0-20190812172519-36a3d3bbc4f3/go.mod h1:RagcQ7I8IeTMnF8JTXieKnO4Z6JCsikNEzj0DwauVzE= +github.com/minio/highwayhash v1.0.2 h1:Aak5U0nElisjDCfPSG79Tgzkn2gl66NxOMspRrKnA/g= +github.com/minio/highwayhash v1.0.2/go.mod h1:BQskDq+xkJ12lmlUUi7U0M5Swg3EWR+dLTk+kldvVxY= github.com/minio/md5-simd v1.1.2 h1:Gdi1DZK69+ZVMoNHRXJyNcxrMA4dSxoYHZSQbirFg34= github.com/minio/minio-go/v7 v7.0.63 h1:GbZ2oCvaUdgT5640WJOpyDhhDxvknAJU2/T3yurwcbQ= github.com/minio/sha256-simd v1.0.1 h1:6kaan5IFmwTNynnKKpDHe6FWHohJOHhCPchzK49dzMM= @@ -360,6 +362,16 @@ github.com/montanaflynn/stats v0.0.0-20171201202039-1bf9dbcd8cbe/go.mod h1:wL8QJ github.com/morikuni/aec v1.0.0 h1:nP9CBfwrvYnBRgY6qfDQkygYDmYwOilePFkwzv4dU8A= github.com/morikuni/aec v1.0.0/go.mod h1:BbKIizmSmc5MMPqRYbxO4ZU0S0+P200+tUnFx7PXmsc= github.com/mrunalp/fileutils v0.5.0/go.mod h1:M1WthSahJixYnrXQl/DFQuteStB1weuxD2QJNHXfbSQ= +github.com/nats-io/jwt/v2 v2.5.2 h1:DhGH+nKt+wIkDxM6qnVSKjokq5t59AZV5HRcFW0zJwU= +github.com/nats-io/jwt/v2 v2.5.2/go.mod h1:24BeQtRwxRV8ruvC4CojXlx/WQ/VjuwlYiH+vu/+ibI= +github.com/nats-io/nats-server/v2 v2.10.4 h1:uB9xcwon3tPXWAdmTJqqqC6cie3yuPWHJjjTBgaPNus= +github.com/nats-io/nats-server/v2 v2.10.4/go.mod h1:eWm2JmHP9Lqm2oemB6/XGi0/GwsZwtWf8HIPUsh+9ns= +github.com/nats-io/nats.go v1.31.0 h1:/WFBHEc/dOKBF6qf1TZhrdEfTmOZ5JzdJ+Y3m6Y/p7E= +github.com/nats-io/nats.go v1.31.0/go.mod h1:di3Bm5MLsoB4Bx61CBTsxuarI36WbhAwOm8QrW39+i8= +github.com/nats-io/nkeys v0.4.6 h1:IzVe95ru2CT6ta874rt9saQRkWfe2nFj1NtvYSLqMzY= +github.com/nats-io/nkeys v0.4.6/go.mod h1:4DxZNzenSVd1cYQoAa8948QY3QDjrHfcfVADymtkpts= +github.com/nats-io/nuid v1.0.1 h1:5iA8DT8V7q8WK2EScv2padNa/rTESc1KdnPw4TC2paw= +github.com/nats-io/nuid v1.0.1/go.mod h1:19wcPz3Ph3q0Jbyiqsd0kePYG7A95tJPxeL+1OSON2c= github.com/ncw/swift v1.0.52/go.mod h1:23YIA4yWVnGwv2dQlN4bB7egfYX6YLn0Yo/S6zZO/ZM= github.com/opencontainers/go-digest v1.0.0 h1:apOUWs51W5PlhuyGyz9FCeeBIOUDA/6nW8Oi/yOhh5U= github.com/opencontainers/go-digest v1.0.0/go.mod h1:0JzlMkj0TRzQZfJkVvzbP0HBR3IKzErnv2BNG4W4MAM= @@ -561,6 +573,7 @@ golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.4.0 h1:zxkM55ReGkDlKSM+Fu41A+zmbZuaPVbGMzvvdUPznYQ= golang.org/x/sync v0.4.0/go.mod h1:FU7BRWz2tNW+3quACPkgCx/L+uEAv1htQ0V83Z9Rj+Y= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20190130150945-aca44879d564/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190312061237-fead79001313/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -619,6 +632,7 @@ golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxb golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20191024005414-555d28b269f0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.3.0 h1:rg5rLMjNzMS1RkNLzCG38eapWhnYLFYXDXj2gOlr8j4= +golang.org/x/time v0.3.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= diff --git a/sdks/go/pkg/beam/io/natsio/common.go b/sdks/go/pkg/beam/io/natsio/common.go new file mode 100644 index 000000000000..53f595516987 --- /dev/null +++ b/sdks/go/pkg/beam/io/natsio/common.go @@ -0,0 +1,58 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// Package natsio contains transforms for interacting with NATS. +package natsio + +import ( + "fmt" + + "github.com/nats-io/nats.go" + "github.com/nats-io/nats.go/jetstream" +) + +type natsFn struct { + URI string + CredsFile string + nc *nats.Conn + js jetstream.JetStream +} + +func (fn *natsFn) Setup() error { + var opts []nats.Option + if fn.CredsFile != "" { + opts = append(opts, nats.UserCredentials(fn.CredsFile)) + } + + conn, err := nats.Connect(fn.URI, opts...) + if err != nil { + return fmt.Errorf("error connecting to NATS: %v", err) + } + fn.nc = conn + + js, err := jetstream.New(fn.nc) + if err != nil { + return fmt.Errorf("error creating JetStream context: %v", err) + } + fn.js = js + + return nil +} + +func (fn *natsFn) Teardown() { + if fn.nc != nil { + fn.nc.Close() + } +} diff --git a/sdks/go/pkg/beam/io/natsio/example_test.go b/sdks/go/pkg/beam/io/natsio/example_test.go new file mode 100644 index 000000000000..0516b8efa921 --- /dev/null +++ b/sdks/go/pkg/beam/io/natsio/example_test.go @@ -0,0 +1,55 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package natsio_test + +import ( + "context" + "log" + + "github.com/apache/beam/sdks/v2/go/pkg/beam" + "github.com/apache/beam/sdks/v2/go/pkg/beam/io/natsio" + "github.com/apache/beam/sdks/v2/go/pkg/beam/x/beamx" + "github.com/nats-io/nats.go" +) + +func ExampleWrite() { + beam.Init() + + p, s := beam.NewPipelineWithRoot() + + uri := "nats://localhost:4222" + msgs := []natsio.ProducerMessage{ + { + Subject: "events.1", + ID: "123", + Data: []byte("hello"), + Headers: nats.Header{"key": []string{"val1"}}, + }, + { + Subject: "events.2", + ID: "124", + Data: []byte("world"), + Headers: nats.Header{"key": []string{"val2"}}, + }, + } + + input := beam.CreateList(s, msgs) + natsio.Write(s, uri, input) + + if err := beamx.Run(context.Background(), p); err != nil { + log.Fatalf("Failed to execute job: %v", err) + } +} diff --git a/sdks/go/pkg/beam/io/natsio/helper_test.go b/sdks/go/pkg/beam/io/natsio/helper_test.go new file mode 100644 index 000000000000..cd47ed331de0 --- /dev/null +++ b/sdks/go/pkg/beam/io/natsio/helper_test.go @@ -0,0 +1,130 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package natsio + +import ( + "context" + "testing" + + "github.com/nats-io/nats-server/v2/server" + "github.com/nats-io/nats-server/v2/test" + "github.com/nats-io/nats.go" + "github.com/nats-io/nats.go/jetstream" +) + +func newServer(t *testing.T) *server.Server { + t.Helper() + + opts := &test.DefaultTestOptions + opts.Port = server.RANDOM_PORT + opts.JetStream = true + + srv := test.RunServer(opts) + t.Cleanup(srv.Shutdown) + + return srv +} + +func newConn(t *testing.T, uri string) *nats.Conn { + t.Helper() + + conn, err := nats.Connect(uri) + if err != nil { + t.Fatalf("Failed to connect to NATS: %v", err) + } + t.Cleanup(conn.Close) + + return conn +} + +func newJetStream(t *testing.T, conn *nats.Conn) jetstream.JetStream { + t.Helper() + + js, err := jetstream.New(conn) + if err != nil { + t.Fatalf("Failed to create JetStream instance: %v", err) + } + + return js +} + +func createStream( + t *testing.T, + ctx context.Context, + js jetstream.JetStream, + stream string, + subjects []string, +) jetstream.Stream { + t.Helper() + + cfg := jetstream.StreamConfig{ + Name: stream, + Subjects: subjects, + } + str, err := js.CreateStream(ctx, cfg) + if err != nil { + t.Fatalf("Failed to create stream: %v", err) + } + + t.Cleanup(func() { + if err := js.DeleteStream(ctx, stream); err != nil { + t.Fatalf("Failed to delete stream: %v", err) + } + }) + + return str +} + +func createConsumer( + t *testing.T, + ctx context.Context, + js jetstream.JetStream, + stream string, + subjects []string, +) jetstream.Consumer { + t.Helper() + + cfg := jetstream.OrderedConsumerConfig{ + FilterSubjects: subjects, + } + cons, err := js.OrderedConsumer(ctx, stream, cfg) + if err != nil { + t.Fatalf("Failed to create consumer: %v", err) + } + + return cons +} + +func fetchMessages(t *testing.T, cons jetstream.Consumer, size int) []jetstream.Msg { + t.Helper() + + msgs, err := cons.FetchNoWait(size) + if err != nil { + t.Fatalf("Failed to fetch messages: %v", err) + } + + var result []jetstream.Msg + + for msg := range msgs.Messages() { + if err := msg.Ack(); err != nil { + t.Fatalf("Failed to ack message: %v", err) + } + + result = append(result, msg) + } + + return result +} diff --git a/sdks/go/pkg/beam/io/natsio/write.go b/sdks/go/pkg/beam/io/natsio/write.go new file mode 100644 index 000000000000..8991ef8cac16 --- /dev/null +++ b/sdks/go/pkg/beam/io/natsio/write.go @@ -0,0 +1,114 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package natsio + +import ( + "context" + "fmt" + "reflect" + + "github.com/apache/beam/sdks/v2/go/pkg/beam" + "github.com/apache/beam/sdks/v2/go/pkg/beam/register" + "github.com/nats-io/nats.go" + "github.com/nats-io/nats.go/jetstream" +) + +func init() { + register.DoFn3x1[context.Context, ProducerMessage, func(ack PublishAck), error](&writeFn{}) + register.Emitter1[PublishAck]() + + beam.RegisterType(reflect.TypeOf((*ProducerMessage)(nil)).Elem()) + beam.RegisterType(reflect.TypeOf((*PublishAck)(nil)).Elem()) +} + +// ProducerMessage represents a message to be published to NATS. +type ProducerMessage struct { + Subject string + ID string + Headers map[string][]string + Data []byte +} + +// PublishAck represents an acknowledgement from NATS after publishing a message. +type PublishAck struct { + Stream string + Subject string + ID string + Sequence uint64 + Duplicate bool +} + +// Write writes a PCollection<ProducerMessage> to NATS JetStream and returns a +// PCollection<PublishAck> of the acknowledged messages. The ID field can be set in the +// ProducerMessage to utilize JetStream's support for deduplication of messages. +// Write takes a variable number of WriteOptionFn to configure the write operation: +// - UserCredentials: path to the user credentials file. Defaults to empty. +func Write(s beam.Scope, uri string, col beam.PCollection, opts ...WriteOptionFn) beam.PCollection { + s = s.Scope("natsio.Write") + + option := &writeOption{} + for _, opt := range opts { + opt(option) + } + + return beam.ParDo(s, newWriteFn(uri, option), col) +} + +type writeFn struct { + natsFn +} + +func newWriteFn(uri string, option *writeOption) *writeFn { + return &writeFn{ + natsFn: natsFn{ + URI: uri, + CredsFile: option.CredsFile, + }, + } +} + +func (fn *writeFn) ProcessElement( + ctx context.Context, + elem ProducerMessage, + emit func(PublishAck), +) error { + msg := &nats.Msg{ + Subject: elem.Subject, + Data: elem.Data, + Header: elem.Headers, + } + + var opts []jetstream.PublishOpt + if elem.ID != "" { + opts = append(opts, jetstream.WithMsgID(elem.ID)) + } + + ack, err := fn.js.PublishMsg(ctx, msg, opts...) + if err != nil { + return fmt.Errorf("error publishing message: %v", err) + } + + pubAck := PublishAck{ + Stream: ack.Stream, + Subject: elem.Subject, + ID: elem.ID, + Sequence: ack.Sequence, + Duplicate: ack.Duplicate, + } + emit(pubAck) + + return nil +} diff --git a/sdks/go/pkg/beam/io/natsio/write_option.go b/sdks/go/pkg/beam/io/natsio/write_option.go new file mode 100644 index 000000000000..b1ee48cbffe4 --- /dev/null +++ b/sdks/go/pkg/beam/io/natsio/write_option.go @@ -0,0 +1,31 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package natsio + +type writeOption struct { + CredsFile string +} + +// WriteOptionFn is a function that can be passed to Write to configure options for +// writing messages. +type WriteOptionFn func(option *writeOption) + +// WriteUserCredentials sets the user credentials when connecting to NATS. +func WriteUserCredentials(credsFile string) WriteOptionFn { + return func(o *writeOption) { + o.CredsFile = credsFile + } +} diff --git a/sdks/go/pkg/beam/io/natsio/write_test.go b/sdks/go/pkg/beam/io/natsio/write_test.go new file mode 100644 index 000000000000..5e9387ece5f6 --- /dev/null +++ b/sdks/go/pkg/beam/io/natsio/write_test.go @@ -0,0 +1,252 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package natsio + +import ( + "bytes" + "context" + "testing" + + "github.com/apache/beam/sdks/v2/go/pkg/beam" + "github.com/apache/beam/sdks/v2/go/pkg/beam/testing/passert" + "github.com/apache/beam/sdks/v2/go/pkg/beam/testing/ptest" + "github.com/google/go-cmp/cmp" + "github.com/nats-io/nats.go" +) + +func TestMain(m *testing.M) { + ptest.Main(m) +} + +func TestWrite(t *testing.T) { + stream := "STREAM" + subject := "subject" + + tests := []struct { + name string + input []any + wantAcks []any + wantMsgs []jsMsg + }{ + { + name: "Write messages and deduplicate based on ID", + input: []any{ + ProducerMessage{ + Subject: subject, + ID: "1", + Data: []byte("msg1a"), + }, + ProducerMessage{ + Subject: subject, + ID: "1", + Data: []byte("msg1b"), + }, + ProducerMessage{ + Subject: subject, + ID: "2", + Data: []byte("msg2"), + }, + }, + wantAcks: []any{ + PublishAck{ + Stream: stream, + Subject: subject, + ID: "1", + Sequence: 1, + Duplicate: false, + }, + PublishAck{ + Stream: stream, + Subject: subject, + ID: "1", + Sequence: 1, + Duplicate: true, + }, + PublishAck{ + Stream: stream, + Subject: subject, + ID: "2", + Sequence: 2, + Duplicate: false, + }, + }, + wantMsgs: []jsMsg{ + testMsg{ + subject: subject, + headers: nats.Header{nats.MsgIdHdr: []string{"1"}}, + data: []byte("msg1a"), + }, + testMsg{ + subject: subject, + headers: nats.Header{nats.MsgIdHdr: []string{"2"}}, + data: []byte("msg2"), + }, + }, + }, + { + name: "Write messages without ID", + input: []any{ + ProducerMessage{ + Subject: subject, + Data: []byte("msg1a"), + }, + ProducerMessage{ + Subject: subject, + Data: []byte("msg1b"), + }, + ProducerMessage{ + Subject: subject, + Data: []byte("msg2"), + }, + }, + wantAcks: []any{ + PublishAck{ + Stream: stream, + Subject: subject, + ID: "", + Sequence: 1, + Duplicate: false, + }, + PublishAck{ + Stream: stream, + Subject: subject, + ID: "", + Sequence: 2, + Duplicate: false, + }, + PublishAck{ + Stream: stream, + Subject: subject, + ID: "", + Sequence: 3, + Duplicate: false, + }, + }, + wantMsgs: []jsMsg{ + testMsg{ + subject: subject, + data: []byte("msg1a"), + }, + testMsg{ + subject: subject, + data: []byte("msg1b"), + }, + testMsg{ + subject: subject, + data: []byte("msg2"), + }, + }, + }, + { + name: "Write message with headers", + input: []any{ + ProducerMessage{ + Subject: subject, + ID: "1", + Headers: map[string][]string{"key": {"val"}}, + Data: []byte("msg1"), + }, + }, + wantAcks: []any{ + PublishAck{ + Stream: stream, + Subject: subject, + ID: "1", + Sequence: 1, + Duplicate: false, + }, + }, + wantMsgs: []jsMsg{ + testMsg{ + subject: subject, + headers: nats.Header{nats.MsgIdHdr: []string{"1"}, "key": []string{"val"}}, + data: []byte("msg1"), + }, + }, + }, + } + for _, tc := range tests { + t.Run(tc.name, func(t *testing.T) { + ctx := context.Background() + srv := newServer(t) + uri := srv.ClientURL() + conn := newConn(t, uri) + js := newJetStream(t, conn) + + subjects := []string{subject} + createStream(t, ctx, js, stream, subjects) + cons := createConsumer(t, ctx, js, stream, subjects) + + p, s := beam.NewPipelineWithRoot() + + col := beam.Create(s, tc.input...) + gotAcks := Write(s, uri, col) + + passert.Equals(s, gotAcks, tc.wantAcks...) + ptest.RunAndValidate(t, p) + + gotMsgs := fetchMessages(t, cons, len(tc.input)+1) + + if gotLen, wantLen := len(gotMsgs), len(tc.wantMsgs); gotLen != wantLen { + t.Fatalf("Len() = %v, want %v", gotLen, wantLen) + } + + for i := range gotMsgs { + if gotSubject, wantSubject := gotMsgs[i].Subject(), tc.wantMsgs[i].Subject(); gotSubject != wantSubject { + t.Errorf("msg %d: Subject() = %v, want %v", i, gotSubject, wantSubject) + } + + if gotHeaders, wantHeaders := gotMsgs[i].Headers(), tc.wantMsgs[i].Headers(); !cmp.Equal( + gotHeaders, + wantHeaders, + ) { + t.Errorf("msg %d: Headers() = %v, want %v", i, gotHeaders, wantHeaders) + } + + if gotData, wantData := gotMsgs[i].Data(), tc.wantMsgs[i].Data(); !bytes.Equal( + gotData, + wantData, + ) { + t.Errorf("msg %d: Data() = %q, want %q", i, gotData, wantData) + } + } + }) + } +} + +type jsMsg interface { + Subject() string + Headers() nats.Header + Data() []byte +} + +type testMsg struct { + subject string + headers nats.Header + data []byte +} + +func (m testMsg) Subject() string { + return m.subject +} + +func (m testMsg) Headers() nats.Header { + return m.headers +} + +func (m testMsg) Data() []byte { + return m.data +} From 29c0b41afeca1cb0fb97362a40d869d46735593a Mon Sep 17 00:00:00 2001 From: Damon <damondouglas@users.noreply.github.com> Date: Mon, 6 Nov 2023 16:58:16 -0800 Subject: [PATCH 388/435] [RRIO] [testing] Downgrade mock-apis grpcio-tools dependencies to remove Beam SDK version conflict (#29320) * Constrain grpc* versons to >=1.53.0 * Downgrade grpcio dependencies * Bump project version * Make version constraint >= * Add poetry.lock to rat --- .test-infra/mock-apis/poetry.lock | 19 +------------------ .test-infra/mock-apis/pyproject.toml | 6 +++--- build.gradle.kts | 3 +++ 3 files changed, 7 insertions(+), 21 deletions(-) diff --git a/.test-infra/mock-apis/poetry.lock b/.test-infra/mock-apis/poetry.lock index e9bcdbb4750d..b36baff7a74b 100644 --- a/.test-infra/mock-apis/poetry.lock +++ b/.test-infra/mock-apis/poetry.lock @@ -1,20 +1,3 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - # This file is automatically @generated by Poetry 1.7.0 and should not be changed by hand. [[package]] @@ -233,4 +216,4 @@ files = [ [metadata] lock-version = "2.0" python-versions = "^3.8" -content-hash = "1f2dadd2821a62cdfb7562f211be17c8bd12b762551bdb923954f9e7404087ec" +content-hash = "35ed5a98dd3f951bbfc44b949ad9148634159976cb54ac6f257d119c12d9d924" diff --git a/.test-infra/mock-apis/pyproject.toml b/.test-infra/mock-apis/pyproject.toml index ed3f035cbf40..680bf489ba13 100644 --- a/.test-infra/mock-apis/pyproject.toml +++ b/.test-infra/mock-apis/pyproject.toml @@ -17,7 +17,7 @@ [tool.poetry] name = "mock-apis" -version = "0.1.0" +version = "0.1.1" authors = ["Ritesh Ghorse <riteshghorse@gmail.com>, Damon Douglas <>"] license = "Apache-2.0" description = "" @@ -29,8 +29,8 @@ packages = [ [tool.poetry.dependencies] python = "^3.8" google = "^3.0.0" -grpcio = "^1.59.2" -grpcio-tools = "^1.59.2" +grpcio = "^1.53.0" +grpcio-tools = "^1.53.0" [build-system] diff --git a/build.gradle.kts b/build.gradle.kts index b330bd07861e..59161809f37c 100644 --- a/build.gradle.kts +++ b/build.gradle.kts @@ -202,6 +202,9 @@ tasks.rat { // Ignore buf autogenerated files. "**/buf.lock", + + // Ignore poetry autogenerated files. + "**/poetry.lock", ) // Add .gitignore excludes to the Apache Rat exclusion list. We re-create the behavior From f3636f32b3f5a97bdd5c8421bf3b1f702ad0980f Mon Sep 17 00:00:00 2001 From: Danny McCormick <dannymccormick@google.com> Date: Mon, 6 Nov 2023 19:59:46 -0500 Subject: [PATCH 389/435] A couple small fixes to the pypi deploy action (#29321) * Update deploy_release_candidate_pypi.yaml to checkout master * get to right location for artifacts * Debug info * Use RC number from inputs --- .github/workflows/deploy_release_candidate_pypi.yaml | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/.github/workflows/deploy_release_candidate_pypi.yaml b/.github/workflows/deploy_release_candidate_pypi.yaml index 7f873dbdc2ab..fd3994f658e8 100644 --- a/.github/workflows/deploy_release_candidate_pypi.yaml +++ b/.github/workflows/deploy_release_candidate_pypi.yaml @@ -35,9 +35,6 @@ jobs: echo "::add-mask::${{ github.event.inputs.PYPI_PASSWORD }}" - name: Checkout uses: actions/checkout@v4 - with: - ref: "v${{ github.event.inputs.RELEASE }}-RC${{ github.event.inputs.RC }}" - repository: apache/beam - name: Setup environment uses: ./.github/actions/setup-environment-action with: @@ -70,8 +67,11 @@ jobs: --rc_number "${{ github.event.inputs.RC }}" \ --yes True - echo "------Checking Hash Value for apache-beam-${RELEASE}rc${RC_NUMBER}.tar.gz-----" - sha512sum -c "apache-beam-${RELEASE}rc${RC_NUMBER}.tar.gz.sha512" + cd "${PYTHON_ARTIFACTS_DIR}" + ls + + echo "------Checking Hash Value for apache-beam-${RELEASE}rc${{ github.event.inputs.RC }}.tar.gz-----" + sha512sum -c "apache-beam-${RELEASE}rc${{ github.event.inputs.RC }}.tar.gz.sha512" for artifact in *.whl; do echo "----------Checking Hash Value for ${artifact} wheel-----------" From bb864ab2b9968a59bd6f3b28df5f2f05cb382412 Mon Sep 17 00:00:00 2001 From: Chamikara Jayalath <chamikaramj@gmail.com> Date: Mon, 6 Nov 2023 20:24:47 -0800 Subject: [PATCH 390/435] Adds a list of requirements to the expansion service that should be used to interpret the components provided. --- .../beam/model/job_management/v1/beam_expansion_api.proto | 4 ++++ .../beam/runners/core/construction/TransformUpgrader.java | 1 + .../apache/beam/sdk/expansion/service/ExpansionService.java | 2 +- .../apache_beam/runners/portability/expansion_service.py | 3 ++- 4 files changed, 8 insertions(+), 2 deletions(-) diff --git a/model/job-management/src/main/proto/org/apache/beam/model/job_management/v1/beam_expansion_api.proto b/model/job-management/src/main/proto/org/apache/beam/model/job_management/v1/beam_expansion_api.proto index 568f9c877410..7a26ff6a2af3 100644 --- a/model/job-management/src/main/proto/org/apache/beam/model/job_management/v1/beam_expansion_api.proto +++ b/model/job-management/src/main/proto/org/apache/beam/model/job_management/v1/beam_expansion_api.proto @@ -53,6 +53,10 @@ message ExpansionRequest { // coders for the output PCollections. Note that the request // may not be fulfilled. map<string, string> output_coder_requests = 4; + + // A set of requirements that must be used by the expansion service to + // interpret the components provided with this request. + repeated string requirements = 5; } message ExpansionResponse { diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformUpgrader.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformUpgrader.java index 5e1609f27a39..db5dfcf6825d 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformUpgrader.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformUpgrader.java @@ -187,6 +187,7 @@ RunnerApi.Pipeline updateTransformViaTransformService( .setComponents(runnerAPIpipeline.getComponents()) .setTransform(ptransformBuilder.build()) .setNamespace(UPGRADE_NAMESPACE) + .addAllRequirements(runnerAPIpipeline.getRequirementsList()) .build(); ExpansionApi.ExpansionResponse response = diff --git a/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java b/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java index ec53e3f11e43..fe02533ed0d0 100644 --- a/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java +++ b/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java @@ -626,7 +626,7 @@ private Map<String, TransformProvider> loadRegisteredTransforms() { // Needed to find which transform was new... SdkComponents sdkComponents = rehydratedComponents - .getSdkComponents(Collections.emptyList()) + .getSdkComponents(request.getRequirementsList()) .withNewIdPrefix(request.getNamespace()); sdkComponents.registerEnvironment( Environments.createOrGetDefaultEnvironment( diff --git a/sdks/python/apache_beam/runners/portability/expansion_service.py b/sdks/python/apache_beam/runners/portability/expansion_service.py index 9670ac1ad7be..8be9d98508ed 100644 --- a/sdks/python/apache_beam/runners/portability/expansion_service.py +++ b/sdks/python/apache_beam/runners/portability/expansion_service.py @@ -54,7 +54,8 @@ def with_pipeline(component, pcoll_id=None): context = pipeline_context.PipelineContext( request.components, default_environment=self._default_environment, - namespace=request.namespace) + namespace=request.namespace, + requirements=request.requirements) producers = { pcoll_id: (context.transforms.get_by_id(t_id), pcoll_tag) for t_id, From 26acfaefc8c785f8af0e2937c6168aff95cc477a Mon Sep 17 00:00:00 2001 From: Alexey Romanenko <aromanenko.dev@gmail.com> Date: Tue, 7 Nov 2023 14:08:52 +0100 Subject: [PATCH 391/435] [Spark Runner] Add Spark 3.5.0 version for compatibility tests --- runners/spark/3/build.gradle | 1 + 1 file changed, 1 insertion(+) diff --git a/runners/spark/3/build.gradle b/runners/spark/3/build.gradle index 5380146044d5..5103805db347 100644 --- a/runners/spark/3/build.gradle +++ b/runners/spark/3/build.gradle @@ -34,6 +34,7 @@ createJavaExamplesArchetypeValidationTask(type: 'Quickstart', runner: 'Spark') // Additional supported Spark versions (used in compatibility tests) def sparkVersions = [ + "350": "3.5.0", "341": "3.4.1", "340": "3.4.0", "332": "3.3.2", From 1ed6567a5cdc4549922f25700f76c3a86b230a11 Mon Sep 17 00:00:00 2001 From: Danny McCormick <dannymccormick@google.com> Date: Tue, 7 Nov 2023 10:09:51 -0500 Subject: [PATCH 392/435] Add missing comment trigger (#29332) --- .../beam_PostCommit_Python_ValidatesContainer_Dataflow.yml | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml index 335e9791d960..196dd1eaa84e 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml @@ -21,6 +21,8 @@ on: pull_request_target: paths: ['release/trigger_all_tests.json'] workflow_dispatch: + issue_comment: + types: [created] #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event permissions: @@ -105,4 +107,4 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/pytest*.xml' \ No newline at end of file + files: '**/pytest*.xml' From 2f6ec18cbc34f8340298aaf635acd27d1b3114ef Mon Sep 17 00:00:00 2001 From: Danny McCormick <dannymccormick@google.com> Date: Tue, 7 Nov 2023 10:25:23 -0500 Subject: [PATCH 393/435] Add new CHANGES.md section (#29329) * Add new CHANGES.md section * Remove unused sections/bullets --- CHANGES.md | 48 ++++++++++++++++++++++++++++++++++-------------- 1 file changed, 34 insertions(+), 14 deletions(-) diff --git a/CHANGES.md b/CHANGES.md index fa6e51cde2e4..17ac8a1d7010 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -53,6 +53,40 @@ * ([#X](https://github.com/apache/beam/issues/X)). --> +# [2.53.0] - Unreleased + +## Highlights + +* New highly anticipated feature X added to Python SDK ([#X](https://github.com/apache/beam/issues/X)). +* New highly anticipated feature Y added to Java SDK ([#Y](https://github.com/apache/beam/issues/Y)). + +## I/Os + +* Support for X source added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). + +## New Features / Improvements + +* X feature added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). + +## Breaking Changes + +* X behavior was changed ([#X](https://github.com/apache/beam/issues/X)). + +## Deprecations + +* X behavior is deprecated and will be removed in X versions ([#X](https://github.com/apache/beam/issues/X)). + +## Bugfixes + +* Fixed X (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). + +## Security Fixes +* Fixed (CVE-YYYY-NNNN)[https://www.cve.org/CVERecord?id=CVE-YYYY-NNNN] (Java/Python/Go) ([#X](https://github.com/apache/beam/issues/X)). + +## Known Issues + +* ([#X](https://github.com/apache/beam/issues/X)). + # [2.52.0] - Unreleased ## Highlights @@ -64,10 +98,6 @@ should handle this. ([#25252](https://github.com/apache/beam/issues/25252)). * Publishing Java 21 SDK container images now supported as part of Apache Beam release process. ([#28120](https://github.com/apache/beam/issues/28120)) * Direct Runner and Dataflow Runner support running pipelines on Java21 (experimental until tests fully setup). For other runners (Flink, Spark, Samza, etc) support status depend on runner projects. -## I/Os - -* Support for X source added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). - ## New Features / Improvements * Add `UseDataStreamForBatch` pipeline option to the Flink runner. When it is set to true, Flink runner will run batch @@ -79,17 +109,12 @@ should handle this. ([#25252](https://github.com/apache/beam/issues/25252)). ## Breaking Changes -* X behavior was changed ([#X](https://github.com/apache/beam/issues/X)). * `org.apache.beam.sdk.io.CountingSource.CounterMark` uses custom `CounterMarkCoder` as a default coder since all Avro-dependent classes finally moved to `extensions/avro`. In case if it's still required to use `AvroCoder` for `CounterMark`, then, as a workaround, a copy of "old" `CountingSource` class should be placed into a project code and used directly ([#25252](https://github.com/apache/beam/issues/25252)). * Renamed `host` to `firestoreHost` in `FirestoreOptions` to avoid potential conflict of command line arguments (Java) ([#29201](https://github.com/apache/beam/pull/29201)). -## Deprecations - -* X behavior is deprecated and will be removed in X versions ([#X](https://github.com/apache/beam/issues/X)). - ## Bugfixes * Fixed "Desired bundle size 0 bytes must be greater than 0" in Java SDK's BigtableIO.BigtableSource when you have more cores than bytes to read (Java) [#28793](https://github.com/apache/beam/issues/28793). @@ -97,13 +122,8 @@ as a workaround, a copy of "old" `CountingSource` class should be placed into a * `MLTransform` doesn't output artifacts such as min, max and quantiles. Instead, `MLTransform` will add a feature to output these artifacts as human readable format - [#29017](https://github.com/apache/beam/issues/29017). For now, to use the artifacts such as min and max that were produced by the eariler `MLTransform`, use `read_artifact_location` of `MLTransform`, which reads artifacts that were produced earlier in a different `MLTransform` ([#29016](https://github.com/apache/beam/pull/29016/)) ## Security Fixes -* Fixed [CVE-YYYY-NNNN](https://www.cve.org/CVERecord?id=CVE-YYYY-NNNN) (Java/Python/Go) ([#X](https://github.com/apache/beam/issues/X)). * Fixed [CVE-2023-39325](https://www.cve.org/CVERecord?id=CVE-2023-39325) (Java/Python/Go) ([#29118](https://github.com/apache/beam/issues/29118)). -## Known Issues - -* ([#X](https://github.com/apache/beam/issues/X)). - # [2.51.0] - 2023-10-03 ## New Features / Improvements From 1c24bc94c9ae5cf93dc129e657964858d4aafc67 Mon Sep 17 00:00:00 2001 From: Jack McCluskey <34928439+jrmccluskey@users.noreply.github.com> Date: Tue, 7 Nov 2023 10:28:54 -0500 Subject: [PATCH 394/435] Add support for collections.abc.Collection type hints (#29272) * Add support for collections.abc.Collection type hints * Extra test cases * Merge subclass helper (linting) * Change comment, refine general check * Add extra test case, TODO --- .../typehints/native_type_compatibility.py | 9 +++ .../native_type_compatibility_test.py | 10 +++- .../python/apache_beam/typehints/typehints.py | 58 +++++++++++++++++++ .../apache_beam/typehints/typehints_test.py | 28 +++++++++ 4 files changed, 104 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/typehints/native_type_compatibility.py b/sdks/python/apache_beam/typehints/native_type_compatibility.py index 67cb2b8c3f03..b2960ba0c7b7 100644 --- a/sdks/python/apache_beam/typehints/native_type_compatibility.py +++ b/sdks/python/apache_beam/typehints/native_type_compatibility.py @@ -48,6 +48,7 @@ _CONVERTED_COLLECTIONS = [ collections.abc.Set, collections.abc.MutableSet, + collections.abc.Collection, ] @@ -118,6 +119,10 @@ def _match_is_exactly_iterable(user_type): return getattr(user_type, '__origin__', None) is expected_origin +def _match_is_exactly_collection(user_type): + return getattr(user_type, '__origin__', None) is collections.abc.Collection + + def match_is_named_tuple(user_type): return ( _safe_issubclass(user_type, typing.Tuple) and @@ -322,6 +327,10 @@ def convert_to_beam_type(typ): match=_match_issubclass(typing.Iterator), arity=1, beam_type=typehints.Iterator), + _TypeMapEntry( + match=_match_is_exactly_collection, + arity=1, + beam_type=typehints.Collection), ] # Find the first matching entry. diff --git a/sdks/python/apache_beam/typehints/native_type_compatibility_test.py b/sdks/python/apache_beam/typehints/native_type_compatibility_test.py index 9c2762dff710..2e6db6a7733c 100644 --- a/sdks/python/apache_beam/typehints/native_type_compatibility_test.py +++ b/sdks/python/apache_beam/typehints/native_type_compatibility_test.py @@ -189,7 +189,15 @@ def test_convert_to_beam_type_with_collections_types(self): ( 'enum mutable set', collections.abc.MutableSet[_TestEnum], - typehints.Set[_TestEnum]) + typehints.Set[_TestEnum]), + ( + 'collection enum', + collections.abc.Collection[_TestEnum], + typehints.Collection[_TestEnum]), + ( + 'collection of tuples', + collections.abc.Collection[tuple[str, int]], + typehints.Collection[typehints.Tuple[str, int]]), ] for test_case in test_cases: diff --git a/sdks/python/apache_beam/typehints/typehints.py b/sdks/python/apache_beam/typehints/typehints.py index 238bf8c321d6..5726a8a8ca92 100644 --- a/sdks/python/apache_beam/typehints/typehints.py +++ b/sdks/python/apache_beam/typehints/typehints.py @@ -82,6 +82,7 @@ 'Dict', 'Set', 'FrozenSet', + 'Collection', 'Iterable', 'Iterator', 'Generator', @@ -1017,6 +1018,62 @@ def __getitem__(self, type_param): FrozenSetTypeConstraint = FrozenSetHint.FrozenSetTypeConstraint +class CollectionHint(CompositeTypeHint): + """ A Collection type-hint. + + Collection[X] defines a type-hint for a collection of homogenous types. 'X' + may be either a built-in Python type or another nested TypeConstraint. + + This represents a collections.abc.Collection type, which implements + __contains__, __iter__, and __len__. This acts as a parent type for + sets but has fewer guarantees for mixins. + """ + class CollectionTypeConstraint(SequenceTypeConstraint): + def __init__(self, type_param): + super().__init__(type_param, abc.Collection) + + def __repr__(self): + return 'Collection[%s]' % repr(self.inner_type) + + @staticmethod + def _is_subclass_constraint(sub): + return isinstance( + sub, ( + CollectionTypeConstraint, + FrozenSetTypeConstraint, + SetTypeConstraint)) + + # TODO(https://github.com/apache/beam/issues/29135): allow for consistency + # with Mapping types + def _consistent_with_check_(self, sub): + if self._is_subclass_constraint(sub): + return is_consistent_with(sub.inner_type, self.inner_type) + elif isinstance(sub, TupleConstraint): + if not sub.tuple_types: + # The empty tuple is consistent with Iterator[T] for any T. + return True + # Each element in the hetrogenious tuple must be consistent with + # the collection type. + # E.g. Tuple[A, B] < Collection[C] if A < C and B < C. + return all( + is_consistent_with(elem, self.inner_type) + for elem in sub.tuple_types) + elif not isinstance(sub, TypeConstraint): + if getattr(sub, '__origin__', None) is not None and getattr( + sub, '__args__', None) is not None: + return issubclass(sub, abc.Collection) and is_consistent_with( + sub.__args__, self.inner_type) + return False + + def __getitem__(self, type_param): + validate_composite_type_param( + type_param, error_msg_prefix='Parameter to a Collection hint') + return self.CollectionTypeConstraint(type_param) + + +CollectionTypeConstraint = CollectionHint.CollectionTypeConstraint + + class IterableHint(CompositeTypeHint): """An Iterable type-hint. @@ -1187,6 +1244,7 @@ def __getitem__(self, type_params): Dict = DictHint() Set = SetHint() FrozenSet = FrozenSetHint() +Collection = CollectionHint() Iterable = IterableHint() Iterator = IteratorHint() Generator = GeneratorHint() diff --git a/sdks/python/apache_beam/typehints/typehints_test.py b/sdks/python/apache_beam/typehints/typehints_test.py index 7f8c322f9f40..1d938edcc24b 100644 --- a/sdks/python/apache_beam/typehints/typehints_test.py +++ b/sdks/python/apache_beam/typehints/typehints_test.py @@ -865,6 +865,33 @@ class FrozenSetHintTestCase(BaseSetHintTest.CommonTests): string_type = 'FrozenSet' +class CollectionHintTestCase(TypeHintTestCase): + def test_type_constraint_compatibility(self): + self.assertCompatible(typehints.Collection[int], typehints.Set[int]) + self.assertCompatible(typehints.Iterable[int], typehints.Collection[int]) + self.assertCompatible(typehints.Collection[int], typehints.FrozenSet[int]) + self.assertCompatible( + typehints.Collection[typehints.Any], typehints.Collection[int]) + self.assertCompatible(typehints.Collection[int], typehints.Tuple[int]) + self.assertCompatible(typehints.Any, typehints.Collection[str]) + + def test_one_way_compatibility(self): + self.assertNotCompatible(typehints.Set[int], typehints.Collection[int]) + self.assertNotCompatible( + typehints.FrozenSet[int], typehints.Collection[int]) + self.assertNotCompatible(typehints.Tuple[int], typehints.Collection[int]) + self.assertNotCompatible(typehints.Collection[int], typehints.Iterable[int]) + + def test_getitem_invalid_composite_type_param(self): + with self.assertRaises(TypeError) as e: + typehints.Collection[5] + self.assertEqual( + 'Parameter to a Collection hint must be a ' + 'non-sequence, a type, or a TypeConstraint. 5 is ' + 'an instance of int.', + e.exception.args[0]) + + class IterableHintTestCase(TypeHintTestCase): def test_getitem_invalid_composite_type_param(self): with self.assertRaises(TypeError) as e: @@ -893,6 +920,7 @@ def test_compatibility(self): self.assertCompatible( typehints.Iterable[typehints.Any], typehints.List[typehints.Tuple[int, bool]]) + self.assertCompatible(typehints.Iterable[int], typehints.Collection[int]) def test_tuple_compatibility(self): self.assertCompatible(typehints.Iterable[int], typehints.Tuple[int, ...]) From 6c95636f6e7fecebbfac8fc552741be0ce442357 Mon Sep 17 00:00:00 2001 From: Anand Inguva <34158215+AnandInguva@users.noreply.github.com> Date: Tue, 7 Nov 2023 16:30:11 +0000 Subject: [PATCH 395/435] Use RunInference to perform custom inference in custom_inference notebook (#29304) * Use RunInference to perform custom inference * Update custom_remote_inference.ipynb * Update custom_remote_inference.ipynb * Update custom_remote_inference.ipynb * Update custom_remote_inference.ipynb * Update custom_remote_inference.ipynb * Update custom_remote_inference.ipynb * Update custom_remote_inference.ipynb * Update custom_remote_inference.ipynb --- .../beam-ml/custom_remote_inference.ipynb | 1386 ++++++++--------- 1 file changed, 663 insertions(+), 723 deletions(-) diff --git a/examples/notebooks/beam-ml/custom_remote_inference.ipynb b/examples/notebooks/beam-ml/custom_remote_inference.ipynb index 2fad42bc0d9d..6657a137d6b0 100644 --- a/examples/notebooks/beam-ml/custom_remote_inference.ipynb +++ b/examples/notebooks/beam-ml/custom_remote_inference.ipynb @@ -1,725 +1,665 @@ { - "cells": [{ - "cell_type": "code", - "execution_count": null, - "metadata": { - "cellView": "form", - "id": "paYiulysGrwR" - }, - "outputs": [], - "source": [ - "# @title ###### Licensed to the Apache Software Foundation (ASF), Version 2.0 (the \"License\")\n", - "\n", - "# Licensed to the Apache Software Foundation (ASF) under one\n", - "# or more contributor license agreements. See the NOTICE file\n", - "# distributed with this work for additional information\n", - "# regarding copyright ownership. The ASF licenses this file\n", - "# to you under the Apache License, Version 2.0 (the\n", - "# \"License\"); you may not use this file except in compliance\n", - "# with the License. You may obtain a copy of the License at\n", - "#\n", - "# http://www.apache.org/licenses/LICENSE-2.0\n", - "#\n", - "# Unless required by applicable law or agreed to in writing,\n", - "# software distributed under the License is distributed on an\n", - "# \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY\n", - "# KIND, either express or implied. See the License for the\n", - "# specific language governing permissions and limitations\n", - "# under the License" - ] - }, - { - "cell_type": "markdown", - "metadata": { - "id": "0UGzzndTBPWQ" - }, - "source": [ - "# Remote inference in Apache Beam\n", - "\n", - "<table align=\"left\">\n", - " <td>\n", - " <a target=\"_blank\" href=\"https://colab.research.google.com/github/apache/beam/blob/master/examples/notebooks/beam-ml/custom_remote_inference.ipynb\"><img src=\"https://raw.githubusercontent.com/google/or-tools/main/tools/colab_32px.png\" />Run in Google Colab</a>\n", - " </td>\n", - " <td>\n", - " <a target=\"_blank\" href=\"https://github.com/apache/beam/blob/master/examples/notebooks/beam-ml/custom_remote_inference.ipynb\"><img src=\"https://raw.githubusercontent.com/google/or-tools/main/tools/github_32px.png\" />View source on GitHub</a>\n", - " </td>\n", - "</table>\n" - ] - }, - { - "cell_type": "markdown", - "metadata": { - "id": "GNbarEZsalS2" - }, - "source": [ - "This example demonstrates how to implement a custom inference call in Apache Beam using the Google Cloud Vision API.\n", - "\n", - "The prefered way to run inference in Apache Beam is by using the [RunInference API](https://beam.apache.org/documentation/sdks/python-machine-learning/).\n", - "The RunInference API enables you to run models as part of your pipeline in a way that is optimized for machine learning inference.\n", - "To reduce the number of steps that you need to take, RunInference supports features like batching. For more infomation about the RunInference API, review the [RunInference API](https://beam.apache.org/releases/pydoc/current/apache_beam.ml.inference.html#apache_beam.ml.inference.RunInference),\n", - "which demonstrates how to implement model inference in PyTorch, scikit-learn, and TensorFlow.\n", - "\n", - "Currently, the RunInference API doesn't support making remote inference calls using the Natural Language API, Cloud Vision API, and so on.\n", - "Therefore, to use these remote APIs with Apache Beam, you need to write custom inference calls.\n", - "\n", - "**Note:** all images are licensed CC-BY, creators are listed in the [LICENSE.txt](https://storage.googleapis.com/apache-beam-samples/image_captioning/LICENSE.txt) file." - ] - }, - { - "cell_type": "markdown", - "metadata": { - "id": "GNbarEZsalS1" - }, - "source": [ - "## Run the Cloud Vision API\n", - "\n", - "You can use the Cloud Vision API to retrieve labels that describe an image.\n", - "For example, the following image shows a cat with possible labels." - ] - }, - { - "cell_type": "markdown", - "metadata": { - "id": "q-jVQn3maZ81" - }, - "source": [ - "![cat-with-labels.png](data:image/png;base64,iVBORw0KGgoAAAANSUhEUgAAAxYAAAGaCAYAAACSU9UtAAAKrGlDQ1BJQ0MgUHJvZmlsZQAASImVlwdUU9kWhs+96SGhJYQiJdRQpLcAUkJoAQSkg42QBAglxEBQsSPiCI4FFRFUFB2qgmMBZCyIKBYGAXsdkEFEGQcLoqLyLrAIM/PWe2+9nbXX+bKzz3/2OeuerH0BICtyxeJUWBGANFGmJNTXkx4dE0vHDQIIEIAC8mFweRliVkhIIEBsZvy7fbyHZCN223xS699//6+mxBdk8ACAQhCO52fw0hA+jfgrnliSCQDqEBLXX54pnuQ2hKkSpECEH0xy4jQPT3L8FKPBVE54KBthKgB4EpcrSQSAREfi9CxeIqJD8kDYSsQXihAWI+yWlpbOR/gEwsZIDhIjTeoz4/+ik/g3zXiZJpebKOPpvUwZ3kuYIU7lrvw/j+N/W1qqdGYNI8RJSRK/UGRURs7sQUp6gIxF8UHBMyzkT+VPcZLUL2KGeRns2Bnmc70CZHNTgwJnOEHow5HpZHLCZ1iQ4R02w5L0UNlaCRI2a4a5ktl1pSkRsniSgCPTz04Kj5rhLGFk0AxnpIQFzOawZXGJNFRWv0Dk6zm7ro9s72kZf9mvkCObm5kU7ifbO3e2foGINauZES2rjS/w8p7NiZDlizM9ZWuJU0Nk+YJUX1k8IytMNjcTeSBn54bIzjCZ6x8yw4AN0kEq4hJAB4HINy8AMgUrMic3wk4Xr5QIE5My6SzkhgnoHBHPYi7dxsrGFoDJ+zr9OLynTd1DiHZjNpbzAQBX/sTExLnZWKABAKc3AUB8MRtjXABAXhWAawU8qSRrOjZ1lzCAiPwLUIE60Ab6wBiYAxvgAFyAB/AG/iAYhIMYsATwQBJIQypfDlaDDSAPFIAdYA8oAWXgCKgCx8FJ0AjOgUvgKrgJusBd8Bj0ggHwGoyAj2AcgiAcRIYokDqkAxlCZpANxITcIG8oEAqFYqA4KBESQVJoNbQRKoAKoRLoMFQN/QydhS5B16Fu6CHUBw1B76AvMAomwVRYCzaCLWEmzIID4HB4MZwIL4Oz4Vx4G1wMl8PH4Ab4EnwTvgv3wq/hURRAyaFoKF2UOYqJYqOCUbGoBJQEtRaVjypClaPqUM2odtRtVC9qGPUZjUVT0HS0OdoF7YeOQPPQy9Br0VvRJegqdAO6DX0b3YceQX/HkDGaGDOMM4aDicYkYpZj8jBFmArMGcwVzF3MAOYjFoulYRlYR6wfNgabjF2F3Yo9gK3HtmC7sf3YURwOp44zw7nignFcXCYuD7cPdwx3EdeDG8B9wsvhdfA2eB98LF6Ez8EX4WvwF/A9+EH8OEGRYEhwJgQT+ISVhO2Eo4Rmwi3CAGGcqERkEF2J4cRk4gZiMbGOeIX4hPheTk5OT85JboGcUG69XLHcCblrcn1yn0nKJFMSm7SIJCVtI1WSWkgPSe/JZLIR2YMcS84kbyNXky+Tn5E/yVPkLeQ58nz5dfKl8g3yPfJvFAgKhgoshSUK2QpFCqcUbikMKxIUjRTZilzFtYqlimcV7yuOKlGUrJWCldKUtirVKF1XeqmMUzZS9lbmK+cqH1G+rNxPQVH0KWwKj7KRcpRyhTJAxVIZVA41mVpAPU7tpI6oKKvYqUSqrFApVTmv0ktD0YxoHFoqbTvtJO0e7YuqlipLVaC6RbVOtUd1TG2OmoeaQC1frV7trtoXdbq6t3qK+k71RvWnGmgNU40FGss1Dmpc0RieQ53jMoc3J3/OyTmPNGFNU81QzVWaRzQ7NEe1tLV8tcRa+7Quaw1r07Q9tJO1d2tf0B7Soei46Qh1dutc1HlFV6Gz6Kn0YnobfURXU9dPV6p7WLdTd1yPoRehl6NXr/dUn6jP1E/Q363fqj9ioGMw32C1Qa3BI0OCIdMwyXCvYbvhmBHDKMpos1Gj0UuGGoPDyGbUMp4Yk43djZcZlxvfMcGaME1STA6YdJnCpvamSaalprfMYDMHM6HZAbPuuZi5TnNFc8vn3jcnmbPMs8xrzfssaBaBFjkWjRZvLA0sYy13WrZbfreyt0q1Omr12FrZ2t86x7rZ+p2NqQ3PptTmji3Z1sd2nW2T7Vs7MzuB3UG7B/YU+/n2m+1b7b85ODpIHOochhwNHOMc9zveZ1KZIcytzGtOGCdPp3VO55w+Ozs4ZzqfdP7TxdwlxaXG5eU8xjzBvKPz+l31XLmuh1173ehucW6H3Hrddd257uXuzz30PfgeFR6DLBNWMusY642nlafE84znGNuZvYbd4oXy8vXK9+r0VvaO8C7xfuaj55PoU+sz4mvvu8q3xQ/jF+C30+8+R4vD41RzRvwd/df4twWQAsICSgKeB5oGSgKb58Pz/efvmv8kyDBIFNQYDII5wbuCn4YwQpaF/LIAuyBkQemCF6HWoatD28MoYUvDasI+hnuGbw9/HGEcIY1ojVSIXBRZHTkW5RVVGNUbbRm9JvpmjEaMMKYpFhcbGVsRO7rQe+GehQOL7BflLbq3mLF4xeLrSzSWpC45v1RhKXfpqThMXFRcTdxXbjC3nDsaz4nfHz/CY/P28l7zPfi7+UMCV0GhYDDBNaEw4WWia+KuxKEk96SipGEhW1gifJvsl1yWPJYSnFKZMpEalVqfhk+LSzsrUhaliNrStdNXpHeLzcR54t5lzsv2LBuRBEgqMqCMxRlNmVSkMeqQGks3Sfuy3LJKsz4tj1x+aoXSCtGKjpWmK7esHMz2yf5pFXoVb1Xrat3VG1b3rWGtObwWWhu/tnWd/rrcdQPrfddXbSBuSNnwa45VTmHOh41RG5tztXLX5/Zv8t1UmyefJ8m7v9llc9kP6B+EP3Rusd2yb8v3fH7+jQKrgqKCr1t5W2/8aP1j8Y8T2xK2dW532H5wB3aHaMe9ne47qwqVCrML+3fN39Wwm747f/eHPUv3XC+yKyrbS9wr3dtbHFjctM9g3459X0uSSu6WepbW79fcv2X/2AH+gZ6DHgfryrTKCsq+HBIeenDY93BDuVF50RHskawjL45GHm3/iflTdYVGRUHFt0pRZW9VaFVbtWN1dY1mzfZauFZaO3Rs0bGu417Hm+rM6w7X0+oLToAT0hOvfo77+d7JgJOtp5in6k4bnt5/hnImvwFqWNkw0pjU2NsU09R91v9sa7NL85lfLH6pPKd7rvS8yvntF4gXci9MXMy+ONoibhm+lHipv3Vp6+PL0ZfvtC1o67wScOXaVZ+rl9tZ7RevuV47d935+tkbzBuNNx1uNnTYd5z51f7XM50OnQ23HG81dTl1NXfP677Q495z6bbX7at3OHdu3g26230v4t6D+4vu9z7gP3j5MPXh20dZj8Yfr3+CeZL/VPFp0TPNZ+W/mfxW3+vQe77Pq6/jedjzx/28/te/Z/z+dSD3BflF0aDOYPVLm5fnhnyGul4tfDXwWvx6fDjvD6U/9r8xfnP6T48/O0aiRwbeSt5OvNv6Xv195Qe7D62jIaPPPqZ9HB/L/6T+qeoz83P7l6gvg+PLv+K+Fn8z+db8PeD7k4m0iQkxV8KdagVQiMMJCQC8qwSAHAMApQvpHxZO99NTBk2/A0wR+E883XNPmQMAdcgw2RaxWwA4gbjRekTbA4DJlijcA8C2tjKf6X2n+vRJwyJvLIe8JunhrsXrwT9suof/S93/HMGkqh345/gvQNIG0qk2u10AAACKZVhJZk1NACoAAAAIAAQBGgAFAAAAAQAAAD4BGwAFAAAAAQAAAEYBKAADAAAAAQACAACHaQAEAAAAAQAAAE4AAAAAAAAAkAAAAAEAAACQAAAAAQADkoYABwAAABIAAAB4oAIABAAAAAEAAAMWoAMABAAAAAEAAAGaAAAAAEFTQ0lJAAAAU2NyZWVuc2hvdFxP3LAAAAAJcEhZcwAAFiUAABYlAUlSJPAAAAHWaVRYdFhNTDpjb20uYWRvYmUueG1wAAAAAAA8eDp4bXBtZXRhIHhtbG5zOng9ImFkb2JlOm5zOm1ldGEvIiB4OnhtcHRrPSJYTVAgQ29yZSA2LjAuMCI+CiAgIDxyZGY6UkRGIHhtbG5zOnJkZj0iaHR0cDovL3d3dy53My5vcmcvMTk5OS8wMi8yMi1yZGYtc3ludGF4LW5zIyI+CiAgICAgIDxyZGY6RGVzY3JpcHRpb24gcmRmOmFib3V0PSIiCiAgICAgICAgICAgIHhtbG5zOmV4aWY9Imh0dHA6Ly9ucy5hZG9iZS5jb20vZXhpZi8xLjAvIj4KICAgICAgICAgPGV4aWY6UGl4ZWxZRGltZW5zaW9uPjQxMDwvZXhpZjpQaXhlbFlEaW1lbnNpb24+CiAgICAgICAgIDxleGlmOlBpeGVsWERpbWVuc2lvbj43OTA8L2V4aWY6UGl4ZWxYRGltZW5zaW9uPgogICAgICAgICA8ZXhpZjpVc2VyQ29tbWVudD5TY3JlZW5zaG90PC9leGlmOlVzZXJDb21tZW50PgogICAgICA8L3JkZjpEZXNjcmlwdGlvbj4KICAgPC9yZGY6UkRGPgo8L3g6eG1wbWV0YT4KC94jVAAAABxpRE9UAAAAAgAAAAAAAADNAAAAKAAAAM0AAADNAALkCKE5GNoAAEAASURBVHgB7L15rGfJdd9Xvb1+ve+zcDZxNKQkUpJJyhQpkpJFiRxLNAOOQjmBYceAEwSIk8AL4MBA/kmC/JMEhgM4MBIhsQDZghE5cqzFlixRCymS4iJKXGfjzHBmODOctad7en3v9Vvy/XxPnbp17/u9190jDsMmf/e9e6vq1Nnq1P1V1blVde+ODR2XLi+VtfX1sqNc5diEsAmwBYONzfAKGnI2yoYTQwgRMEEcIVvqNl470Nj/itUwCzHkJXrgklIsDkUGmRYk+MA/0QiDJnmEPKDIzVzHAkA0jpoO+gROw0nuJDnFfjXpnuXsElauZPbIE2HTrBGvTEyRJjyc3AY3yRPFVSLgJvgsvsCSMAlm4V0LTqWbxSbJk3XiTOGZf9VwFmEyhdj5ukzvrxHjZNITjhCuPXE9rLbBnWqSqFdTJOmMn5dZZW8MMzKilJhMdxKnoK5N6bC2iYpB8kixm7Az46qIouz4wcekXK6FFoL5caNZYN/i4rdM5XX69lm/HWmQ8OxXwV1dXS1ra2tl//793zId54LmFphb4MazAO3FyspKWZy0ZztwLL76+JPl6Wef95glG5os4saGh+hugEZtkxLu9npg9oONmD7SboEiOAzZ2ZIkneFGWd9YLyhJg7ZR427k1tfk9OgUfFUnDtC6CBG1c8fOsnvnrrJ7V5y7du4UbEfZqXDXriFOmXYIN0I1psq32mKyLp7r6zSkq2VD5/rauuXTuSMjaIJn8tkpecjYuWu3Q3iTDr6iMn/so78ujlng4RDu+qckFWS484wC9eajWhP1dATOLLyBsvKZIPV1MeCiz1BHCU8pVa0Ej8Koz6rV7AIFfr0HwE9b9IwmajZtrEOXmfonPCR395gAs/inLOOjS+VZrZTZLRzBO/kgDDo0dEdClzEs+UTeZgxDqm2SsjcjWeAAS17gjesr+aLoRNlmSag25wa0u/asTFD5TXQ0xRS3shk0GGJkjXUO5CwTeYE90PjeQn8VfoAG3aAOdQ9K4oRSyS2w40obwdH0EKH5JPNJeQwWrIGjEoJZAiPVrsE7GQKuiJvwpa/QqkqmRxfQkjrvM2de1wUt0h494aBExGbhDPi9/K1/U9WGSUaZanyQlpnTjDEGKc4dY/BAnIyBCKe33YDUx3qCHh7xvnxxD01x+toceE3l0i/1R2AO+L2ct/3gm3vU1zRO35n1lmEvEBi60d9yLi8v+zxx4kSPNo/PLTC3wNwCIwvwEIL24sCBAyO4HYvPffmB8vBjT0QDrUZmaArBjbQbpC7DUNJ969rlQxntbDS2xN08K+JGDMA6DkWkVzWopwHkXLcjISdDIfAra1fKigpw5coVpYGvFw3jC47EHjkWCwt7yh4N8nEwdu0ELkdjN84GA36cgHQMYvBP2oVUsC7ea6tXyvrqSlkT/3A01q08ZaTc6TQEH8mQrF27ddqxEH/J3WFnIxwOHAucHpwK09qmwSvsRhwLpR5OAPCRJqWMw5Hx6OSyCwuouTbUxGwAIsk0gdg949GNt5R760yBZIapf2QEbeWgwDHqtOImeRYx06AYm7DxBZZaD0PBwCVPh22YmonYfCIr8jsdGhi7qNOsyCmB7IQNqNWGlXWoFhQ9Xaem72PTCyFxQi2uFZJ6NvtXQMInujT71PJmp48cdCZtzqLvWJA9Olpe6taMWfVKeFKZgAv24mgcLNOgemmoxkl9pN3AumJWQEecvMmxSk2PxE25kR4GY8BBrvKgh6+YJM9QJ/MFN0mkAyvibsugFUGYRYjGNdDx4GlxkVezkM9BELSZ0YdJbUykRGYN4OBopgFQro4nNGi7qfzB6RquTar5oCziqvqm30r/wT5CmyBVlcUHpfMIKKjo65yWL7lTwS4btGAmx7Goodzo3MtKmbNCFOjhlX9PL2VCnYFnL6unJm4sEehxkxO+g8wvBFUJVt7lnDBIKYE9ZN774+8ZEq9xbO5YvMYGnrOfW+C71ALbOhaf+rMvlq989TGbZtqI0zC2xlSJ1pCCrQbWDacuNJw1pVg0o1z7Rpu4TzkUzEoQx7HwzER1LHIalsYQp4JzRYP/ZQ367VisxqxFOBa75FCEY7GggT7OBc6GZzDsWMRMhmcXcpCvcIccC8opyXJkcCpwLuRYaEpnXfKYtZByxgEv6XEU7LzYsdgTjgUOBQ6GQxwLORmWAbw6GG22BAuF7GZnOimfMlY9bFPFAzuAiU+3mJ1U2LzmCzuOGJDURAtMA4oiA2bNVlmtg5KBYuysxkBSRuqObSpGZRDBAE0Jje0gU4R5T2SZMkwuDsHLSKMOjK2vvVZVB2wLpz6r8UWX1HHQOfmP9Upohipt8hRpUmfZbM/MT5JEIt3yIjJOKiW9QUcH8pKvi9PRNzpwgdfDNIpn8UPggNH4gN+YEAmcoS5DB9DyiHILV//NRtUY0BlmnoM80yppvtBVZqBlvPGvEeBZboOq0mmTrIDkCaOwmgmDy5R5lZbgEf+gaFfyWvmAojcPJeqxHW3iEPa2JI2OPW3KcLmaHSG0ESF51cfIvvDrKr7XoReQ+vSwjIdKnV3SHFXVKc8Aj7QQq4C2uqrMe9o+vp0+qRdh0oRK/G6qUsrrbQtuzzPpgE+PnDUJx6JqPPyoXJR2/02JR+nUJbT70PvfN8p9LRNzx+K1tO6c99wC370W2Nax+PTnv1zuf+SxaHybjYbOw81p69SHBpwusvbTpho6iqFJd6PtNlUw9UrTc51ZC89MhCNBI3hFsxMo7FkM5a1o4L98ZcWOBfkbokEWA34G+gt79vgcHIvqXCgvHAEG+Qz65VSoHO5UpDparq2HY+EZi4ljQaHATcfC8upMBTMWI4fCsxd1aRSzJE1eN2uBbP5ShxAQ+ijOE1/rBlxHdEGE/MVh+9UEZk34gCGg7DweAFVaOlr9Q9PLcR1JJw6urfZ6AYLKEsbhYhqFPZ+mTMOKSK9b1n/SZQhm8iRudUJdki2vx+/hxDMvilIt4wJREB2j8sAzwNtdk2fgBAF0aSOXbSQqc6q8yrzxqbgGw6cqUc1fsYcAuqYmkRzpNGDiBmP4hH4KFdeUZCJA7Lg5pkBn9zgdeo1GiZJ20Bl2rW47OeBHeaHpBtGpu6EwD55hg0GHZEVu2ocbYrBhx3OwzpAPK5gEewQpHcH1XpHf5Iq4jzfdJkx7HJZ4Its2GSkURD1uzwbeyR+cHi/hPT6sG5yiZ/0qo7uDmhkarpgQH/BHXDfBMasLVNGmdCO+iUPoygxAjzOlryRDWQSY4idND4cu01HizRWe+eDCI/mIkP84unsGPplB/YmgInUWMG0QN34Vq5eXhIkzdyzSIvNwboG5BW5UC2zrWHzmi18pDzzytVa21sa2dlQR/mmMheX8htTInJepbFTdYNP4klEbYeLkc+JUeMZCjkTMUuBYaHaCpU/MYmj/A3E7FuBoxmJDS6HQhOVG7KVojoUG+56tqA6H4xrg76qD/OZY1KVQaJWOhWcs5LysScb6ai6FijKnYwGfOFkKtSccDhyMdDK87AoHZuxYeIYEfTvHJiwpQ2QHJ8Nil+x4FPWBvTkyJJ6mNz4AHR6zZccnu7oTNBGMq/0rJeAmX3HqoZebdQcsKS2Dy0gTA4aLGMO3pxkorIxxk7/zUmclrgVuBttc+nIMaCqf+etiNQZdwOnlkr62I2tBvOLfZMnLuYminNl6bSdpRGxE25a69ZHhuCwqzVDLZBlNl0Sv1J3ZK2QaBF+4USbfL03UhBmkiUPUwkAOglb2dCwaObwhnh4CTuFm1RToCBJRGirbKS6zGXd0m6NZd87pRLnsFb2VRekev4+PcNAoVdwsMk2knBDYiR3xH/Hsytaz7mmbKCEYXjPRs6dpeF2kl5XgKF9KGHNI/N4G0CVWk18BU7yk72UlLHEzTHjiWk5nD9JTnKTNcJpvRbNoPeNpPAtkIZGZPA3qflTA+7xklbLnjkVaZB7OLTC3wI1qgW0di89+8f7ywKNfax3BqJC1waVzdaNIum9gG3IdfDg7ENyw9p1J7dTc6IKntB0LLT3y7IRmI3AcmKHAqeD0UiilV5ixmDgW6MN+inAsdjvcPXEq2oxFHdQzuHdPq3JsaKCD4+KlUHYqtBSqypByLmduBodu7FiwxwIHojoWOBc7OatjIXzvvZCO6Nk2eGcaJRTnSLtiKscNHS6BFWni1aSj/nCmYwGJZAwdXFC6LoOdr+T3cq1HzQ+KpAPYa1ORMlAWvHuagWKgG/QZl/da4Clqq7Avx4CztWPRyxzwrzPmcgcNtw01VIPGaLZeLXtmBC5htRBg24YAywiiwa6kXR6DgqZWBhmBXq/11hvBxongiw7wRHZVRmhjXpGsOM7tNRclwoLdQCsWYE1ZDTp0eY22RQY0Mwg4YloxW6RDnRVNlojraRJuGpe+UWdd9vh9PEiSgRhz1CAS9ZooDRiABDeeAqTMMNlgm55t0jV2XaZto4zGsyFtjjRZm/BTQsdYOODP4ptYUPU4U9xZ8mbBoOvhqfksfr1sx0U7k75HTIYKtwAPGeCqYCm71ythGSbbHmfuWKRV5uHcAnML3KgW2N6x+FI4Fq01nVFKN4rqQLJrmYEyAnnQAEQtdGtgadwB1Uae0HsscCh0omQshZIjUR0LnAsvhVpZLqvaZ+EZC9Ht4G1V0ocBf85YLOzRPgsN7Nm0nbMVu9kPIQeAWYOcMXAh6BTsWGjGRI4LMxbrOWPBcivLiMESdINjEfy8edszFZ1z4eVQ6VgQxtKrDL2ZmzdI1T0XzWD0+hiWsD+kQztqXo9BbqZHjoXgzf6iSxwbvyaM35hPIp0eWXeNxwSVZHaYxm20ne4gTZKAtjuSZ+KkHpnO/Ck88/uw6a5Ib9LEwVbNSg2ZXOVUvUdgsCvA2VuULXVLXVNehsATJ2HTsNetx5/SbSUj+Rm/Vfqg/7heBn16fkGbnAhrgbty9/YBnPrBx7zSYEnb4YCb9h/xAd7oBp5o0A7410SnDsgNZRQxclBwDSzqOU4AI32TOUzIq8zAyTI6a4Y8eHfqtzJWFg6o36qEg+RvwuQpYG+Hhp86W06jDL5JO8nrdTbiJD9hhGmTlD0tMziZl/iEs45ZtLPwev2ulfeUj/WuQC8JpSI4OpsEYPY10cFPHZr+rq6G0fKTU+pPmLSZ14dzx6K3xjw+t8DcAjeiBa7qWDyoGYuh29xcxGwkM8wGdDMmkBgOZV7gukVunRXLmWie/Yo7NcIsQcKpCMeC2Yp4E9SyQ8W1/4HN216qVJdCocsuDdKvxbFgIzX41p/eXv84Fusb6Vgsx1uhkIFjIRkcohqcCjktu7Qp3G+DklMRYXUscCq85IpZCxyRcCwiHumUz1Ao48gInYhhueFoQ4VudOJhFLon2qzOksxKTJQycPja8TIwL5VP45twEUFnuQqz3q1z4tTQeSP+yU0ckr/CWbSwgL7PI84JPOVORM5MJk1muty2mHg5oXCwYLVOlHGQ02GoGKFXhTkNdzEL5jUYDzZTfh9OdevzZsWtj8VUWZKNFlkfm2iqPgE3sqPBZ5SZqjs/yrdZ/8EeyVE8dYw4jRLkdXyc1yNU+lqvVEMvYxO6uYXAegtZdk9jhba6jBiGbK4GR9J3gm1a066jxm8bWzeczZHh7kGSGFfemzGlS/3NGKUWcoDNIJwB6vn2tkk+s/Izr8cHbyt4zyPjiZvpKa+EbxVuh9/zTrwMk1/iEGZehipIK8u0DhIn74M0aYZZYbP4W/aAWG+m0Cj5EiZt6kqY+ffd+/4e/JrG6c9Slwx7gcDQy32x+r3562Z768zjcwvMLbCVBbZ1LP7EMxaPu4FsA5bWi7vpNd9po5SN5EyhJotONVipqyXCf23EHNZ4OBWxh4LvVfAmqBUN8peZsVDIUihOCsJbm+iLvccCx0KD/FwOFTMWvC2qzlqkA8DsQd1b4YEPqmlVFI6F3wx1JRwLv3IWGTgW0tVLoeCFU+GlT+FYeI+F0t7ELWcjNnLL2bBDgVMxOBY7ahz7+a92eGlPh7aXdAoThUkFM5jC1sP1U5NCxZiZNYSABhLLJLOBOn6NqPKZwc22to41k3pL3aE3OPUY8e4USf4T2iYfPuBgmwpEBidw5/XI28STpkcRFyXjGs7FkJvyoiDAXaIhacrA8oCxlUGwShxB6Dpw3hybpdtmrIC0MqeMagtyBytNqCtuliELAS/LnoEOPI7N+jcdKkYOmJMiy59sp3qFvSI3rQpt6mL+kZEsuAUmB/cB90eAyZ7qNSEYEB1LhhNBNUmAnjmhk/aI7LhOyzVT3gQYtqqyU/mKU0XbfCmPrCzXCEYlJsFExqzkLB5TPHBSRuInjuEWeXWhySNpp7wSvlWY+L0+PW7PH5zEbzgyL3UDXuY5VNqWz1AEo/zKwDideaPE43LDu+dv0h6lVnHqhBzOXvfIAx6xuWOR1pqHcwvMLXCjWuAqjsUD5cHHHvcgqTbHtMKtL5s2kJnOhjqNkmnnqzGOo3YGalCzwc0wXzXLq2dxLFZ5lSyhTs9Y4FhomRKvmk3HwvsslI+MXTp375ATgWNRnQteP8tSKELO+N5EDPJDrxjU0OtsyNHwjIVfORuvm11jOdQVPpYXMxaWAx85JrnHAifC37LQ0qvcuD3IATf2WnhfhXTByRg+0BedFHytj4yUYdqR0LbEEdLR50e82tZ1FD1VWtsEKlyrPeTUzsy8AsF1nZ1cT5t12NBEb7oaEp/idOyjMzfFWO+ehnhfpopuvlN4S3f3Y+JvFULTy2tKmUctbVdoR1shWsTsM9V+F4Ju0r+xHMu1HtQEbzHr7NfrnXqO8qdlTf4KleUjQDWjZ6h41v00N8tiJlmvMk7KTl0m7IJjJYZ32sIspkIgboLCVo1fVy5oO7S+iqRPoxhHeoI+pxFIuw6ngeE+yqjEHa5tVtNpD7C2tsns+7un7VXs7xnEJF90THtafOqpjDQDuInf80RWLy9x4OO3gdX8hEOb+D0M2wxim1SLqiZxfKtLUpDvsnGdElLQegyxwQ6ZNytMncnr9XadVQKslXk9PtmZJj9xKtkQVH17noBS1+TRCFr5FEmkITJDThCk/Pvuvbexeq0j8xmL19rCc/5zC3x3WmB7x0IfyHsIx8IHnZUireFUvOsUAmeL69A7BUIyEq9s1Pswp14J+ehdOBW5zyL2VizhXNTZiraBu85YsAxqt87rciw0WHc/oJAn12sshcKxkIz13MchmV4KJb3pUOxQyDngDVThXOBQ6DsWcmZwLNKpcCinglkLZiniA3kRpmPBDAg880xLZsdFmJ1PKCol9Z/VYd2F48PAzAEt4FwHaEoAGNCU1eXEEqHKL/n0+VvFrWsoZT1HTkzqOSFu5ZvAt0r2+s6i7fOnPMb4FLDaDsQaBaeDBosE2CaZiCyuY74CUKfAyeRQolFV4KhuQenquue3ZXkaQ0vohI3L1Q+OgmQW4RS2uUyzqq+XBIfp7A8GyGYABXt8pwH0h5hsut8GBoMRezoLhlsczV6C247KAKVVgNLgGlbhKbO3O1nTo89vcqZISvd4ZM/CneKYTa+oyt30tPFJ6UD/apNZfBNmanD1l+ULBle5ImaGzQ2eRTrRJeWDip5RioEQPqlPq4Mhu8VSfwDw6flmPO2QRD1NwqjoQc4Qm9KCD99ZcHQe2YR0d2R5Rk9taj7mGetvbspluVGUa+5YdMacR+cWmFvghrTANTsWNLT+G9pjFXiUUPO4xTHpcBILfDfedDo682vbdig0+5AOhr+0rVkL3gTFzIQ3bXvGYqUsa4+F3xbFXgzhoGfMWAyOxR59zyJmKmIZVMxexKyFy9UthxIDF8tvhZK8dS+FqrMWbSnUhpdbsU/CS5xYEoUjwYyFZityr4VnMASPjeIxW+FZCsvL2RI2bcuy9UR+2DqtFJ0cMA7bq8bT/L3dqSPj1doglTDiPW4gDpCUYXiHSycdXJNTYswOrSNZJtKF/0GMyzebMqCNfjsk2KcdFJ9F0+dPWQ34nWK1lMaVzuB0IsYsNo2ch+yBd4X1TMIcMypioEfv5JEhuTPLI34zDxdrGITxE/RgqyJfW00Gcq9DyuqLlLAMUcniU7dqYnTIghPNbGtWccDgaGVNJPKDgfNbxSQs8Tq00FEZ+s8y+D4Et8ozXIhJnnITP4SNr6O8jnaMFakRbkVIGYk/CycUqlqpjFF30tOFSuWjGKl78utD8E1tEn7H22F3lFWEJRDvyEi27I6kr58sY4ZZxrwHkz716dj3HB1PGoRS3TZBxUp75G2QxCkv0w4lZJAz/MZGONeQmMm70rk8FkIJh1KmftCmzkESOHPH4hoMP0eZW2BugRvCAts6Fp/TjMWDjz3hhpzGkM+gZUcRpYuOzk2oGszWjNKZZUsKzSxTdPjg5skMRToU6WD0sxZ2LNpSKDkWmlFgSdQqb3BiKZT+GKjv1smH8Zi1GByL+DCe3wxVv8AdX8OOckRBQ+GNtnlb+zfkvMRSKM1gsMdCh/dYaFbEzgXLrHAs5MAQeraibtpmz4UdDs1UxP6K6lCQLweDE5vyh3zbt4b5ETPnCzY9+rFt2n6KQzo3aTuvq5ceN+uVMOsOnrGVPurQOgo21STxe1p4J0/iqR+0gddDN+PCM+VM+UJ5rcdIh0nZhzs2B1wpsXJXsjd7llPQViDrWZF6PQfcsaa9PuOcIdXzGaAMqtKKATWvXkGBowSBB3pmA+npsy4NSyTou7gIQhC0WYMdzHpWDOQmbchJWmwVcQeh4KAYvMlwpkLJT93MuuLjEPT6t4JV+RGEHOLEUhS4SQss44EUWA0X4u5wGav+gBstiWorB4NocnyA2+OnfTKfsOcPbo/Tx/1BPfD5q3LNu1e86jDl06uWtLZQzQCWemY41c11IlmZP5PnNuW1jHoXJQ/L4NIzI61j0HOcDS15WeyeV1AOOma6DxudgEnblx/cXnbS9rCky7w+PeBRqCiYVO4ONEhAhugScj/0/vnm7c5Y8+jcAnML3IAW2N6x+MqD5eGvPeFitafqo0K6m3Mz2QagfadXcbMxJ5mNsJvUrsUFzjnbsWCPRXzTgjdAecYC58JnOBexx2LVbXY6Fgsa1DfHAgej7q9gr0UsY6oD+zq4d2+FsjrZ37Gur2/zutlVlkLhXOjEsXCpVc6csTAvzVTsljw7Fsxc4Dj4TVE4FjgT6UgoXvdX5EbuXA6F3CEeVourVdrU4WFDzJ3dU4bYuT+GbhhoDtgrZ+wuKB2iIQk2JnYIroCjthXZ5mj48NOZ6TFJFdI0D/k9zmy6HmN2HM6b7JBlGmWMElF2V0DHINUUqIsOqbSdIK4IQh3Xqjt4w0AkaLe6Tnk2Omzclbiva+Chd1zhMS61tdWllk68OGrg+EAgynovRIauE2ZNpz5jQpMkvjfMSDqB0zKCu8tR1apoY/komfmmHXj0DnctUTBtQrpk5QELRxsbbCcIQPOvNCRrmVp5JziZD0XGG25lA3wKM74u1sOiHTPFLD4TtaodN/M1bavb4DncG2I/ZWSJAQ9a6VTpSacu3ChwyzzI+vyEA8t4wyGio4cPivR3MUhDDrqmVZoegQK7VpQ+D3iT08o667cA5nAkj6RNraaUiQdl4gaXEDbki4NsFummSI9aPnTv3LEIg8yvcwvMLXCjWmBbx+JP739IjsWTbsjDsaCYNOtqFN0uKk5DCUQDceI0nBWjdUB9R2AyOidYKcyDxpYzN2632QrNQrAUCocDZfn6tt8KhYORjoVgwMmHJ0/oc48FsxW8GcoORudYMGuRMwY5a2DFpSxaaWeHnAiWV4VjsbYcMxfxdW+KGk4Jr5LFccCh8KwFzoWdCjkS5DGT4dkJ5IVM77GoMDZ/N7vZfjGDASwPYk5V2yYcPQMeOgMfLJpYFScwG0bUUuC4o6u8EZs8CKMTDB6hhyUOzCexht/pP0FREh4phWSvTWAnn0hd2zU16zgHYc3objfBx1hNg55JxifiGy5wmE7Keq26gzceiEwE1eQmftILHXIAnSVBXXHsmJAzaJtLDYGiNr/CpA0iM3YULqlbcAzOhpMH1pi44Y8yxkaHynpbfyc2awE4ZRNvcpJXKOesId8Faqi9GQYL9MwG8oQmnq1SZaFHLzapRnWHPp0tNtVXEikMfh1yl0c0c/KWajp1+vR4rZzKB8X1MuXpjMhJu2bNG5pCezrDgidMG53lBIF16/KsF7J0pH3SFkmfeUbSJeGVCoiz4lqxlKjqdOWrulUUgqQBN/CDq/PSoA0UkZYEaXJMde/5j1CxSQVkeSKZOpIb1OQn36G2lV0ZzB2Lash5MLfA3AI3rAW2dSz+7P6Hy8OPP+kmMWcsKOnQMKqxrK0tsGxUM0yrJD4hZw5wMt9hbZzJZ0mTnQlCx2MTtx0LOxTxte02czFxLFg2oSG8lkCxDKq+cnY0YxFLonAI0DUdjFYWtfJs3mafxdqaZiqWhxmL6eZtz1pUxwInAqeCmYt4K1R1LATvZyx6x6I5NTKCdZE+01kLBpB04pia/lEm8lFN3xIVHOksTPZYFZqB66gSRBDXvu6AZN1Bx1I4H8ro8RJM2OP3OL3ePb4LJMCoLEaIeyVwo+zg9vx7Pr2sHk686/YH45GRGz9c9KpBp0hYBMR6VIBRqKeE14h120S02VZblaGxw1j1AHeE37IGx6LhOhID4RlqeLaN395aPe3E580Ud1ewkoxc9og4n8B08gX7XYLVF5M1E0I41MFgcSlfOZA/aGU3pSU7fPOBWxypnkolAJrAJy9Ohn3IFnJjKZyBVnUVpAYOeirZKMAJJGBp94AM14aTzEOF0XVUX6OcSMCjx2k8lZ3wVHcG+QASUqNFn1r4hIHoclQK8+zKCDhLFiHlHnQwWZYzDdgM3IiNNuuSZcm8kNHUNHikayIqtLgqayqy55tx+DT+iqTaQySYN3kwTYJOru1ViRvuBDWyq1YomsLSRuYXtmysU1ZfmIpH4KV+UuhD976vkbzWES8drjr3ZU25wLAHD/k459+xSMvMw7kF5hbYzgLbOxYPPFweefzrbn9pZGiDsg0NpgLUBjPbTcNrY5XIbqyrFtlwb3IuxJg2NxsyOxTNsYhXzQLLGQt/w0IOhfdY2LGIzdt6bZMbaS+H0iA/Zyz82lktTYrvWOAAaNCvQVI6FW5Y0ZuC6FzbwKnIpVB1nwVLoaQDioIfS5w00LJDEU5EzFzEvgo7F8rzjIVnKJAXb4bybEbOYFhudIzhwI1nLexYVJvazshHTcHCalapWphggGcdZLfrcppWF/HB5hzJJ3gGLDIDo9Fnlupr4DUMlKi/PDI/0sCtfWY32WNoZg98oDMO5e34JyZyrPeMPHBctp4dQDOMp/ChSNVCgVF1cVjBkATAMdNnVtpmlm5g93b48+CYtgrdsC1Qqda1lE2nYVVvmPEMX7UHJMzikX+FAYLu4ctaWkicMu6Sl5AvPHCZhZtlQ3+kgLNH56J+N3t3Ka4ZO2Dcr5wcBBGTxYORSDMCQubPrscOoclv9oJxsmp8kNqJSFlVn4E2yhO6BU1e06Z9iblfkhaWphPPKB/2RmYqUzkp3ddz8ifchNtnTuLJdaQrwBEgiHzPJ30nf/pb6OUPNGJKgQjEo8fp48meEHjaKcoaxL0ptpUNk1o3RKeH5bb8QRYKDjIwRj0UTV1Dn8wYh1viNFYt0vRzWZsu4lcVMKbjUSFuW6q4XoeUSZbNnCKg7flSAcEUzPlSqGrLeTC3wNwCN64FtnUsPv/AV8sjT6RjMS4kzX78RwMb1zEOqWxgW1hhvWPR8tToEs8nJP7qtgY/fL+CN0Ixi5Gbt6/wqlkcixWdWq4EnO9d0AGw6ZnBzh7NHOQeiwU7Fbv0XQscimHGAufCMwjCd8eggjCQz9fNrq4OsxXstVi3DPoGDag0yPKMxSzHQrJjn0XstfCH8uxIhPyktaORsmVQ8820jaqBTDVudlw83Yosrn3XhsUDuUFli4AGPOkkAtLo04wReNnpOZUg5yfnSLjO0LPmEWQ9Jij1dZ4kjbF72WPeQV+FO1EpJW8qg+xeTtCOr7NoUDyGiMK1qFoSBdVkg22ykL1KiDAPgkCYJWeq2ywcs3KFENtsx4CGXtwLuANYcw0HQjrxW1I0HAc5DDgOK4T6vVAItMNxYKZiWb+nS7qHOa/ghOtvn/b87NXJRyQpok9dMk5MPkRZ0G9lv34/i0osGJ8XJOi3pDQnMijvTuGHDSuTLIDN5ItYwr0/KlxBM4XJKw+yk2RADQYVxTwbMSIqAXoJ01nJo4pOHPTOOFn+/VS+Qaur5foywjWrKmta3+aVehjxapdUMORsh22dE0EyUvamskzlj1iHbUZln+JXGeAENraESWtlUotNYc839duEVAHGNV8AYYeUF6m0TRJEaL0aXc3rgyQblVsICR8i3c038CYWpYUmiYJZWsCp1EE4IyxlNjIiiQdjZwzY86VQGGV+zC0wt8CNbIFtHYsvpGOhhnHUFkYPa2A2/NHMbjYFjX7fudCEplOReaN84TMzMd1jEc5FOBbMWsTMBa+bjVkLnAxeNwsvBt5jx6Ju3N6tr3EzU+HBUIQ56zAsSaJT0ECNXRa5edtLocLBwLHID5vZOUh+npnIvRZyKnKmglDn9l/gHmYo6Hx5uuxOuBqV7ryZXPZLm0elZKdExlALadNu9Ykrp2GA607N4LiQ2fFovaPBM/AF76SP+PWDCHCaXMXzcPc7YhA5PW3imr7qlmWzrtMyiGBK3/CTmUMGYhVgHSLRq+N7qSFNylpJkdXYTHSZ6gHJbF3GOo9wouC2M+xjGVMpq0pcqbMOV+xgMEshx0Gwy7pHOXEwXAZR40Azk4FjcVFO+CWcdfFjadMR3Z8HcRh04qDIHTF/5EGvQGXc8JvWFnFC7FjkDMZu0+3fE78tOxgU1FRB3yxUy+Jso8A5j7Cizd0MCpfAGdlEJIliG1c2LmtfX1V3JGRdtGzRTHlO09DBuq/j/qbBPu0YJQIK3Yhn4qfyjbhGkofyQ99AxAb977ixgX+1DxYxTfLoeDcdXJguY0a04SqvqSm6lJlA8gyTvJYHTTNwZd7l+z6oTHs5YE7pzDUZN0Uqzz5InB7Wx7t8VOuSXaKDdvqjY+rZ2xbspi/2TvJeTyOlIn09JSzC4O8WHqZl/laosX0yde7SWvnqN5bLM6fp+zfKbcf3lB9+/b5yYLEt0E3UeTi3wHemBa48W77wsY+Vz93/VHl5ZXc58cZ3lHe/4y3ljbfs99LlsrFcXn7y/vK5P/50+fIz58rqzgPl1OvfUt71k+8q339cDw1XzpcXH/zD8m9+76Fybuex8vr3frj8/FuOvya2uqpj8SgzFhpIcHRtrtpSYENLOsSMOrpk4wzQToXCbLQJcSI4elg6F56pYBBUZyuYmcCpYDZjRV/CvuzXzWrGQnsv1jSrwaCfhh7HguVPzFiwPGmPBk08XeVJqx0LnAstTyLuJVGCZ3no1OTaSK9cCrVUVuvmbeRKUePaKRE9b3nyEijvrwhHYteucC7yWxZtzwUfymNZVN38jW3DqYkGMvdX2LoYXP8xYxHDs75zHndofQ2g3xaHdScv+I2wzCJkGi42PdfEpZ6yY+0lgUseB/mZ1/MwvOIkbsPvaM1El5RlHvCsOJlPaJgQ+hKlfi2/J1Ac0+Yxg2WTmzjXG/bye9pZ+vf5xNNuaXwP9gXEgWAmYkUhcZwInAeWNOXMBc6FHQvBWe7E742aQB87HjgWylvihQiStaD78LBebnBIv5F9+o1gFmiYzVjVzxIHhjTO7m5lLuh+3aNwp0a6zJwwS8HyqCMLu8uxvQvlkPY04aDsUj68+jpRMtLKAJ62GNVF1RVcjsSpCQdQd1ZSnHRCq/UIBJbqcQRKxazUyrPsJGnI0FaCYFOZAK73tXA9+BXHxBzpCgU8kmfVpzHqIz1ewpNppkdhlznRo913oEimtRTOoGVltIU+fRmCxkxM1GZOxS3KHrxGNJ3dWtlDlUEHmwUFNh+pf/J0KPwGn5A0S8xm5/un4fS6dXyQYZwuP+V3aC3a51mvKrvJEWbqHURjewHjtvCdNCKa77EIe42vX3nicvnVP36lLK2sl6MH2BdZytmLa+rPd5T/7P0nyl03L4wJtkl95Yml8vzZK+Wn33JoG6x51twC324WOFce+si/Lr/3yI5y+xu+r9y5/8XywJeeLCu3vbP8+Ht+pLxB/sErj/9p+eQnP1ueWLuj/NAP3l4WLzxTHrn/iXL65DvLh//qW8uRlx4vf/Jrv12efeO7y/fveqp86o9Lufe/ua/cs+Ni+fKv/25Zf/cHypuO79XKhT9/2bd1LL7IjMWTcixqgzu0u7SGtYOtOvTt4yy1sjGmDSbOYIXWlXCWY5EzFls5FqtyKpilWNLJPosrSjPo9+ZqyWAg5NfL4lSwibs6Fl66Yafi6o6FN29rmdXaCo5FXRJVnRc6XRwKfyCPgRQ87cTEEijifNeCTdt2QJS2k2FnZuxYNGcCA+vE3v6zmYFhUSD16rQStUMj162tI1z6jAZskf7pZ9ZtdISVccc/o43Y3IeBSkrq8YCRnpWHvHYvcA+AS7l1JHyrdE9rgkpj/FQghSpvyidpCEPkWG6fjy5J38OvNb4VbZZxOz4UgUGcQ12YaViR57AkRyFOpe1YaFZCjgIzF14eJVyBhYNTEY4FJcxzRbiXdV7Q7+SSQmj2avnTURwLnfu5h5GrsrOM6qL4wgcHZqcUSseCGQv16Wjp3y97L5jxuGnf3nJq755yRA5GLK1iiRS/xLQ3NLqPpSP2SVvU6idTHMd2T5ya6SAuYqJ/H9Ylfh9twEuey1JxCIzXpSt9gkeypFTCO4pRNPH7uk5YvcEwZtBUfUYMMoEBEq/BMjIr7DQTncss/qmHw8ZSuZX3qERb6JO4FN740IKrYyvHwnkpY1SZVsq07dLJbbJaZkTSHC2/FddKTbClZ9VvmpH0QT6btuF0eidsyi/TmZ/2NpyiVgSrgz0SoEi7L8Fp+iZORVRyvnm7GrEGjzyzXP7ZR06Xm4/uLn/jvcfLqSO7nfOKHIuPfOF8uf3knvLO7zswJtom9X//0Zny4NeXyv/wN27dBmueNbfAt5kFXvl8+Ve/+Ily8QfeVX7yXT9U7tx7qTz6R79Wfu+xPeXud76/vO8tC+VrH/3t8okvXiin3vvz5d43HS67V75RHvzUR8sffHZnefd/9XPl7le+VH7rnz1Y3vgP/3r5C0uPlt/4x/+2HP97/6C84+lfKb/4xdvKh3/ux8qtBzVmbe3Wq7fB9o7Fg7HHAvbR7qqrUSQb1K7PcRs6wFvLuUkzGmWcCcI8p2njaOCTG7iZrfDGbQ3qmZng+xXxPQs5F9pgjYPh71jIuVgXLp00nWLMUOBYsNdCMxY6/XE8BlD9Pguld8qaoX90Amt6YtveCoVj4W9ZMINxpTovzDTgUGhDNvxwGOxYKM4rZyXTH8fj+xXKZx+H0zgY7ZsWotUTY8utdgWPdNoSw2a+68GWHsyanWrDVxb2E5GRskMjhU2yV4u4UczfNJEEMdESMgrhmfIQxWH+kpl8+juAvFkHuMmH/KSdwvp08pniMuuTsAwTd1bY89wKH3ji9TgJyzD5g5N45PX5PbzHz3iG2C2dCpwEjenlPLO8iSVM2hshwLIyPGOh3wjOAjMWHKbVhWVOOBc4H3ukh/c+SDecinNyKl7R7+SCfic4Fvt0bx7bs+DlUPt1b+51HWofhn5rL2mZ4cs6wWffEh+d3Cs7M7txQPf9Xj3ZWBEf7gfk3Cyn4mY5F7foZBZkf85e6LfFDKL+232V960V18V5zh7bfIoHPuWUobna3rZzvcnSzmZYcYzoOzRiLTrKh1nFTGUqeh9UlAB19H1dJz64qU/+Psgzj6RFVsbJTNmCNVm1bGRzpCzAnqElFH7CzcOZ3I9QVE7JW5BZdjUmBL28VIKwh4OsA5lZxgwjJ/IgSrHT/MTLcJo/TRsPZr0eTb+I9Pok32zrjDGiTeLUNSh6uUnbbCiUgSol1NAZmTsIck1WsKE1y3UzoNmO9937nfcdi2+c2ygnDqjt4HVy13Fgsn/y6y+WF8+tln/48zeXQ/u2f5QK/rNaKvXS+dWyV09Bbjmmhxya4eDA1s++fKX8zp+dK197fqX87Z89afixg7vKPhqy+TG3wLezBZ74d+V/++dPllv+ys+Un3rb3eWE2o2lr/5u+X9+/8lSvu895T/4qdfrBn+2nD6vFQi331VO7VdhVp4vj372j8pvfXy1vOvv/Fy558z95fd+6bPl1H/xt8o71r5S/tU//li56+9/oJz9Fx8tu+/7a+X9dx/xioRvhhmuybHIjiFadDqKaA0znCrSN8x9HnC3r4Q1TmjHQoOe3sFgxsKzFhq4eH+FBkSxDCpmKngrlPdaaAAUjgXftZBjoQEVrUjMWOhVsxrwx3cscCxYssTSp3AsvAwKx6AbzNORUDyGSzgW63rd7Gp1LK7wVijPirgU4ViIFsehzVhoMBVORnUuqmPh/RhyOBymYyE6ZONcIBR75mklZDybGnjXm/Z2j5poNTPq9NCyWrxRD/g4UEFnIbJZHi0mWMoinFWvgSu8KiHxwE3a5DsNpzg9/6vRTnlN8XteDbcWPnUF3uNN9Wl0r1Gkl20RVT+2VOMqeE+EZiouXFnzeREHQ7+LK8pkGRNOhZdFeRN2LGFiBuPllVUvd6KGj+qeO8B9JnxmKl6RY2zngt+T6ogZhZsWFsoJOReH+K34cQVy1sqzS0vlhctL5aVLl60L9ykfnTyqGYljC3vKcU1bMO+BE74qufvF64ju/1OLC+Wo8o8yeyHeOBh7pcNu4TOgxf6+r/u71XCs0O4+EvWuctSXvJ8jR7igV7uZMu/jEGAasrlD2xGIjW4ikptidO+6npK88qUM/nYPKlT8vAcHHZtERaAYjk11P2S1mNV0Kjgmf3MSM/Or8hsREWVkESOWkpPjWJcRrRKJZXi1Z+grusoqy7yJNu2vjNB3TAP+tOxZrll5wDhGNCgR/4I7N+pgZOGQ3+jAQ3fREiQd1NMjadDL2CYAa3PNgks2x8CzK3NkQdrwMkJ/Nz2+Ex2LB09vlLuO7NCLH6al3T791Isr5Z/+u5fK29+4v3z4XUe3RX7uzJXyy39wxk7FgpwK9mHwsPADbz9c3v0DB8pLck7+0f/7wiYeP/sXD5e/9IMHN8HngLkFvq0s8OS/L//0l75WTn3gZ8pP41ioQ9944mPlV37zgXLhnneWD/7sW8stI4W1MuHZ+8un/+APy1d2vrv8zf/4reXAhWfLw3/4m+UT3+DNjlfKxYV7ytuOP1TuX/uZ8uEP3qMxQKwuGLF5lYmrOhaPailUHNl80mhGPMOp7GyYZ8Ezj5Bm1aHiNLI4Ei2teDoXOBY4ESiLc7HiM94EdcUzFtXZ0NPVqWPBHou9GgzxNiicinAs5GB4tqEO7OtshbsRlY0GCcdinVfO+gN5sRTqCm+Fki75lDA/eMceDTspkrW7dywkl/0U8XYoyVI8TuRW2Rpw9Uuh3Jlh32pjOqHWwVWD2vpZBx3MnSA2rTDbt6WGjg16cBPPHV1NkNfgqpOmD3Clp0cPCb16BlPs7dM9/63ura04TPF7Xo1GCqaNgM3EacjjCPzbsEKFTtqp3DHV1inzgk+PIv1IM1PBjMSSnIpLcibOy5M4L+fivO69S4KtqB4YznC1My4aHAdmLs7KuX52abmc129kQTrfsndvOaFBPjMNwNKxOCMnmb0WyLtpAZy9WhKl/RHC461O7LF4bulyeeHSJZ2XPWPCK253ywE5ubhYTslhuNkN0Yb2YbBEK/YeIZMZjcPac8G+CxwLlkaRPiD4HvFn5iNmCOOe9H0jPVCmuyuB+NZ0RJdmfwOSihLEMcQSEmHWecvPSP/YPmGQ1Ps+uURWRZDuHK536qGeyMh7Yaxncql0NZn3T+ZuFzZ+KbtaxUmpZc2kR3+MYIjus5VOm/Q0Ge9RsQUHQRU/M22g8QbqtEeGWeYMk4aw4aDowKJHsa0NQJH4DxgkUjB5jIgmiSx3iBgLmuqV/Aaa2fiYN9Su+dIPWI/d32qB3mqVZDu+Ex2Ll1aKHjgU781qBb2GyOcfu1x+5eNnyn8op+JH5Vxsdzzw5FJ5Qo7Ij33/gcIsxOXl9fJLv/9yeVozGP/tf3Rz2a9ZibMX1spvfOaV8uizy+Xvf+gmszu4X2/Di3Wd27Gf580t8P+vBS48WH7tF3+rPHnyreWnf+od5fsOXSyPffK3y7/9+Oly4sffXz547w+VU01DrTh46Yny5U99snzhhf3lh993b/mxuw6VjbXlcun018oDDz5XLm6wOuF0+cIDK+Vtb7+5PPPo01oefbC8/p3vLT9y5yHtk3Sr1jheb+S6HYtobENoNrzbCe0ba+KZptF1uobpRCQOoWcsNPjxMiiWOymesxZ+K5RhsRTK37XwBm49y9UTXb8VSoP2PRrM79WyDN4GhZORsxQ8qWVZlDdP1+VIlCfPDfUE63wkT47L6vLlcmV5Sedy7OOgl1XX4dkG8cGx8J4K8YvlUCyL0owFjkZ1LmLzdi6Lqk6G6GK2JMNBft+Lp05p51bl7sBqXVijxJBtK5I7N+sb9jYv4w5DuOgA45qdqDmJDv3yyLqDR8bJC8rAUlY7el49fkNQBF4c5M/C6cve5091gEfySn6E0yNx4JVxcHreUxrSlpclVYETv+eReISZT7w/psOJtB0hJ04FMxUsf2KGAqfigsJzOs9qKdR5lkJRn7pn98p0C6oeliFhRZZAfePycnnq4uXyipxsXiP7PQf2l9ftW/Smat4GdU6/mVf0O3lR9zIhr53FqTghB+S4nACWOLE5G/1f1kzdaTkXL2rW4oLwVnVT8Qrn43sXy01yFm6Rs7DbvxNeYct+J83wiY7FB+yxQD7Lq04sir9mMW5SeFDpRc0c8mrbnerQ+Z3mcxLfkZQNQ+jg1qBceaTtoi4SOoRb2by/D+E4xgthUY9VcA3M2bZIGcqo92vwDET49TIafytfS0C5rnLAoy9jogNL6tH9lkAQO/4GK516jGgqU9uwo0lZs3CnfIKsF57U24fJZ0uscWW3+6C/CZBaq8D5yZNwlu5TWVlPg0UDI/lswm/CxjkNXwql2uaZ94fCAYf7YzhGlmsZRDa+I98KpTG+26lW1MEU28bYQ/H7Otmg/Ybb9m6LOyvz84/KMfnEmfJffuBkufOm2OA932Mxy1Jz2Le/BS6Vp//kI+V3P/1UWd57qBw9crDsPv/18vALB8o97/mp8sGfuKfkvNvK2afKA5/5WPn04xvl9e9+v2bkbimL/Y9P49qVC0+UT/zrj5Tn3/Sj5finP1me/Z43lFMrXy+PXXxz+dDP/Ui5/fCi+/FXa5dtHYsvPfTV8uiTT9UGUk2ylIvGGy1J99rSt0WTmWGvFDByM1QkOkuFfupKujvT0cCpYAO3ZyschnPB8ifvr9DgyEuhFOJsQOfXwUo/b95meUfnWMSH8ViWwayFBvQK07lAX5dJxWJgjmOxjmOxoqUgcixW9CR4VTJixiIcC/ZYeGlVOi04E5IXjkadsSBPZz9bscMzFrH3YlgOlfIH26aNM7SOtn2NqQ6iNkjHgZ3Bd21QFtnER1SP8kjpQp5gtSbMh0xngxGMTGrEiI2v8KgQoyse9MklMxNrQh7KtLof54oX+lQc8mbdW8BdXgqTRy++gqd8+nSSESYX2yVpyUg9RnIQlPdyln3AjVyI08rE4zBr7nklGXrkK2S9n4LlT5rOv6yqYxM1jsVLmo04wz4LOR7U0sk97I/YpU3XOz3DgaPwpJyKp85fLOf1FjMG8d9z+GC5/eB+z1qwEfuceJzWLNzzly+X08K5IJqjzELIsTipgf9hHHCVk9/kK1eWyxmdLwsPHVh+tai3nR3XDMgxORgnmJ1TAdc9Y8G3M/jWzLoaJC0jFMxvsZKuB6TjcS2Luk0Ozi04GXJKDnhmD8c/HHPcV00UeuYlrDPYEhvWmyqzIsyKqtDNFo6M7o6eYIpB5eF7oa/XipmB0axIQrqw16PnMVIcJAH6/MqiZ9uyeyB4ee8Fl1S7clDQCInKEvUEoX840OzY6zxwcSxFT1Hy9zL9DSYc4j6vj5MXRUjuQOJo9VZNNLUT/JMX7XLj0Cno/M5GIkj2ozDvhSZzlg4jikj0ZQSS+sxAHbVF4DV9hWw+qVrNQJdQd6Pcd++9s1i+JjD61ixXhr0gYOifffGr/fI2D0v4bV/v8QdfOl9+98/Ol/9Em7bffNfiNZHzWtoXX9EeMjWcT2m24hP3Xyj/qRyTN1bHZO5YXJMZ50jfjhZYfrE8ev/D5cnnXynLew6XxTMPlIcunih3v+295X0/fKKw0nDtwvPlkc9/qnzmoXPl6A+/u/zEX/zecmyyt2lt+ZyclF8vv/P0PeWD710q/+J/ebp84L/7a+X7y6fK//k/P17e/nfvKz980+Fy/a78YLRrciwSfWh8NBjvG/FEUEhDNG10M0176vwamoyGq9IkLWE2ZpsdC5ZDybnQIIYP4+VG7lgqFY6Fe10Ji43aciw0wPKH8uqsBQ6FZxk0qIklSeEktDKpEWRj6waOhfZZsMciZiy0JEoDMXSjnbRDIAfBzgpLnnAgkGHHom7eZvkTcOd1Mxa9Y9HNmMiw7jnRJU/s1HQjzp8b6ugm+zxwfdR8WXyoj9ahBR2FyDppZMhtLGpM9bHlUXmQn1hQDVwq5RY8Uves+4rdgt4GAMGbdYA3ystCJLLIUlaCtgpDguyWCBkRT5crdYhKqHhCMp4wUnbmCz9YVEY9HDLly4fQciI2aettTCx7kiOxJKdiTSMplj/hUDytAf5pzUSwgZvlRLdrk/StGqQf0RI/ZiteVP4T5y6Wb1y4KB5XtPxoQY7FofI6zVrgDLAngxmLF7Skj2VOp+UoL8nROC6n4hYtb7p9v94Lz6yb9OdNUGflVJzVG9de1rKpC0qzFOoQMxa6v3FA2FNBiVg25Q/u6beiwpRFQfmYJTCcHYp7WI7EbdL3rv37vcH7qHSLj1aGg79bv4H80B7mYyCCrYlDz1mNSEaLE+VIC0dqfA0uwAZsUkkT0JqXdQvC5GBQO/Oo1UrZObgadbgY3vSeISNZj7ISGNTVCJnowo4ofwO0UXmw7KyVPXmmzol0TSHEmwn731XKT3bTdI8bOqXNFBKdpV8ntjkWEzUmSfHaBLFKHatUscm0rrPJdP+lYkE2LdfATOyEO8pvunRtQyWwOOXnvfid6FhQxrH1emttHf+SXjP7Lz96pnzwR4+U97xp+zc/LV/ZKL/CG5+eXipH9u/y9y2WVja0mXV17lhsbeJ5zg1nAT3gW75UzjzzcPnsx/+kvHTsB8vb3/PO8gPHd5f1i8+Vx77wmfKZh8+VxTf+WHnvO9g3MSng+kq59PyD5bd/+8vldff+fHnHwU+Xf/Q/PlV+Ro7Fm3d8qvzC//QtcSweKY99/amJZjSONBPRVGQDCiwbVGAJJzRcFAnPxhTGxBu80pGmY8SpmOVYsOeCfRY4FUsa/PCGKDykVc0uJK8d6oF2abDigT6ORXUqcDK8JwLnQo5FzGAMS5GaTnUp1Ib2WYRjITnazLrKBm532rFMiNkKZMSSJ5Z34Fgs1FkLbda2QxFLn3LWgtkKv6ZWYf8WqLShDGabZZowD8NqQlgJzupw2vCalfYlI0A1oyMlD7tx9Pz7MUSPbtxkI90YcFKTlUWvlXnCp9ErgiR0bDCok7iLZ7kzhFmPR3rWESWhLJE7lhRl7Pn08cav6pm8gKe+ve4MdBqOI01oY8UQb12P7chOnTIT2bn0iU3S8pwEAABAAElEQVTazCiwn2JJyDgVOL58j+JFbch+7JLewa5lScw88AXsu+Uw3LF/sRzXPcaG7OeU99gr52OztWiOaiB/+6GD5dS+fX4dLJu12cD9vGbfntWMxRk5Ihvi9Trl36FZjXsOHvAeCxz9i3Lcz2i51Gk5ITgWrwiPvUBeNiV5diqEd9m40ltOBG9vW1ApD2kgu0vOBr/Rs6JlIzhf675Jzsv3HNhXbpW8Y3Jm+KZMfLE7vvzNBm+WUOVXvFkmhZMzvFEqLF1vN5twas+06zjsKknRWkOOYH/S/T1CTYUkZSg/D+NUYoNbXDwqmvmlUgozamHgCy/uNyIRn3V/97CUf7Uw+AYWbVSmeQBifpJvNXUJVUihYy0IpF15g9P2V2h7kjErS7MegRfpkbyOfWf1sE2rhaF2+L0B7jRuHII7ScVqYiy3gUPnysT61ELk/rnGtEZ6nbFr2nYKz3SGVqUzEPYZ9FS85QHdoRmL909Fv2bpb9WMxastwPNnVsv/+usvlLtOLZS//VfiLU5b8fo3nzpb/lRLn1g29fpbYtkTr5X9pT94ee5YbGW0OfwGs8CZ8rU/e6A8/sxL5cw5zVrsvrm88W1vLW+656ZyYIceAD70ifK7v/mR8vnzp8r3vfVN6tejeDsWDpUjd/xAeScfu5BjsXT6sfLprx3Qx/VuL7tXnykf/d9/rTx55z3lptWnyqMXf7B86L63fWuWQrVGUnoS33DrHvFZNZONZYbGMV00qcAzr5+tENBwYM2x0KDEzgWOg87cZ8HyJ75f4RkLxZmx4DsW7lCZe9XhWQm9gYllSX46WsNcvhSzFgzyWQ4VnW/ttzRYlg786SN5vGrWMxZyLNbkxHiDuPjzJJCBXzgWOCw1rie6OBe7ODVYSofCb4RiBoNTuNHhD7JjAFU7etkLW/enC6VL1kd0t9WmwCsC8FpF9FxJFqF4ckQ91rwOxbm6OOzg0GSdJT0hiGO00Mp5/aXqEXoFjlXpiHv+SZplzfQsnMwjDHYDUyRNeYzwpVdvC/NHMQygMOWZB7iVOEuJpCbN+JHmY3R+ZbFoNPkQs3LGhQc66f5RyJe0mR24hFOhwTn7KS5rhoJlR+yjYC8C+xueW75SHtZMxEssxxP+ES0tulvOwO1aXsSXs3k17DNyLJ44f0EzeXrdomhvUd4pnQc1W8BD6yUNNnl17HNyGF6QY3FBv5+d0vNWzVTceUBrNQ8d0AZr3a9SDh6vyClguRQzHOfkWOCon9RMwzHdv2zS5g1UZ8GTE/KydOS3x1rOY3IODssjWFDcv2PBWV7Fx/OYHTms2ZNFfgPKdzF14bfExm6cDV6Bi4PBhvO9vMlNebt0Yq+YI6nVI7qs29GgtMKz7mRiH1VcJLh2gKzPzGz8TFsZEFBxNajQ4QZIQcYIpE5Ekxd6KWeKn3QmDcosQ8uSUOuWetQM8BKXMNtVyhjtCu1bqBqcIYwYNmww+DRhV4/0v52GXXVzupYxdTOsl1eJkNrkEql0ZGcdE++wpHNoDSzjxtmCljyOYF+licVAK1gFB+Zw7cs5KsuAsmUs8a3txDYpLsB8eft9W/L5Zmd8uzsWlJcN2A8+tVT+6nuOlh+5Z7yB+8x5tZdLa+UOOR7/5Dde1FshN8o/+HBsyob2c49eKr/6ibMjx+JXPykH5JFL5b//67dq72W760GfH3MLfJtb4MVy/0c/Vx5+9mLZddPd5c1vfoM+DnnIeyK1xqC89Nj95Suff6Bo0m58LB4tp+55e/nLb7l5DCeltvLy4x8vv/O5p/Qw8WC5+x3vLW99zTdvP/TV8pj2WHBk4x4NrCENZoTukg0pIOJJSyNK2mcfT5hCCso6bQYkMSjBsWD2ojoWGsjwJBRHIpdC5atncSzAzT0FDIRwKthwimMRS6JifwXOBIP7nLHY2rGQrLoUin0Wa3pyzFNeOjNev+nlUAzGNBgKXsxY8KpZZi0W2owFb4baIRycCr8lirjktzdCoY96F04BfWK3PLFl2rGF7lija+KazSQd5XaOhfGQgd1hHCw6+sqswkHJI+s2dQDe0FC7cRn0CRmB5WvFsx7JGD7Uf3f0MhI8xUk4YSfB4NTFcihvd/R8kJNph4nrEK5WWE+lx/oZnjaEvfDlE+j+lbMgByG+kh17DTxrIdzq87qKqet0LFj+xHIjlj2x2XEN2Tq5h3E4XmCZk97OdFEOBHrwytc75BDcrD0LbLh+UY6HHQY5vyypPKz7DMeCJUh80I43NzGrwVKq5+ScvCS8y3KSF1S2WzWLcLuWKDGbwIfyFnQvsm/inHBfllPxvBwRllqh77Hd2h+hfGTgWJzT7xLH55weAGA7Xld7VPqw/+JQdQ64z8BnCaI/UKkyUbw8eNpf70TJYOO3llnJoWBWBqeIL4Kz4ZvlXzg94EBOveWrBaJmsn743dT7qeMNTVA60i7gTuGpz3BzmxFCjQpJShsiACs0mMK4+0U4Wcsa2jT8yPK1v+/zvmzZWZ5eZ8kEL890KtDE6uoCz/Z7cBG4VEiGCIFPE3b1SP/bGWGLp49qj1E5enlCSokVdUSH9ZIVGaFbXNE+aYOoXmsBejuO8uFTyxlmHOs6xSXdl3NUllnIHcz6pUEn5aYxyHY6dZ07Fp3xFGW/xC/81kvlghrFv/D6fZ6N4CHD119YKV/42mV/Qfsnf+hgYe8E6ft+7Eh5w+v2lq/qw3q/9blzGidsjByLj375Qvn3f3quvEuvoH3zHYtlUW+/uE0f2ZsfcwvMLfDNs8C2eyy+qA/k5etmafiy8aOxjwY/QncNXa8w4EUDnuq2jk+AjPehOzXxaY6FBjfNwdDAhiVPft0soZY9EV/2zIXiwHTyFAYaePFKWfZWePO2Bih808IDGw14/GE75XtGgUG9zqa3ihVvhZJDowFTOBaxgXtNT3w3NACEv22iRo6ZCH+7wqHidizYwM0+i9hf4Y/i2akI5yJmLSTTcnEq0KHamJ5UZ3MysDawGmbcaUPrJeug0huqTo3utx2VD/wpQ+JkPrh02D6Es1UnOhMu/NStcki25tM62arDFCeRk0emp7IyvS2emFMOcBI/+RECS/oep4djBqzB4Wvay5C4gB9LwUDS+9NxKnAQ5ICyx4CN2Fqgx4vKNI6QI6F8nAkp4HsOm0DDvgnOy4qvCO+K8PUArmjXgh2OV3SvM9DHaWYmAAeA2YOjOM66b17WfQkOy44O6L7m1bEn5XywsdtOgByAlzQD8dzSimc2zsix4IvdzA68Tg7KrXJC2FjNrMKiYNx75/R74rW0L8ixWMGxkC77xZsBPgcDWF5xyx4LyshBHo7OCenHNy14xSzyOfwKXeHz7Q0cHeDIgS+/W3itCA4PnAleW3t0L6+s1ZfB9QYqXoXLZm9OFdl1ol+QqOt9rFgehmJnHxlGqt3fNdeBCMZYXWbLUKSWvcut90YPibjJmg6Q1l+iYHnvgYkduR+ANTi4sgeHfzeOOeGCN5UAiZYlPG4ra5tp9OSnEBvH7yFDwWRPjibTqfEF3td6tN8OxkcaelXiIRZ5wzX0J93oScw6sIn1qVyVDgHiLngrh7JbXHyCJhgmvIfNEpWwxEs6wxHvMibWECY+kBav9gA2Iuvsk/znjgVWGh9sxPZrYr+hBxwr8Zs4qg/fvf0N+8tfklPB93HYtP3Lf3imfF2vnOU4dXh3+YC+UcFSqP/83hPle+VscFySg/J//c7p8g19LI8DR+R6vtxtovllboG5Bba1wLaORb4VKjgMTWI0gl0nKITWiCqejWQPyzhtMvEM03FAhuHqTOlogZMmZHDvgQfOgwZPdiK0UdTfsxBsyUui9F0LDbzAUy+rkyekY8eCV8+mY5EbuD3wVwfrTpaOCj24KCpNJDv3WIRj4c3bGiyimwdFDHIYbGkQxIwF37FgRsJvhWJQZeei7rUQXsxWsBwqZiyglcH0z2CJzj7sisNhOLpwGGdsc4PdVVljow2wTE7yahkpoHNUDhVG7If6TUrzEpyycmToxIzLVjxMG9JsXHOTOEskUWX3GiSvXmaLC7/H7eGhVuhLvPEhHpkRq2VqoFcZsfnEOByH2IDNsqbz1bFgEI0jQb52KXhwjXNBOmpAswmyDd+suKB7HeeC/RXna5pvVuxUmr0Ge0WxS4/3eRMTm50XdPpVs0qzf4LlUwwVWUK0X/fXYb2NicE5jgdfyGZZE7MVT2vmgzc98Raqo3rVLDMVN2uJkp0Q7j/9UScvyqlgQ/hLciz4HTJQhD+DWAb2zGyE86DZQAA6mFFgOdOi9YsZBiqd2Q32jryk3+slftPCPSg8ZjVwhPgts3fkovTEsVh0npwKlSG+h0FcH/tTGqeDt7ohi9M3FcJd7bqXiU8O0PKuyP0QI5QuH3i7pxRvd9uEcUuK8ayBs2HKM54V2IyVOlkX2de/eyesRAUnk8iwE4FE4XNEe6n6l/MYD1bkBNKmuY3R7JPqCx1oX7y3S2E+SOH3kb8RM+su8B/0Uww1KAdHZtQkcPAdZmbiBIWgCahEghPLcoCW/HsY8JmHdQmegZ98BauiUi/TW1hwGvFPMrJElzoE5hZX848SJX7Pcyhr0sdd1Kkws9xzxyLtNTs8LweCPvLAYjjFU6wLeuMFv49D2sDNcUXt6KzvVOCIUG9X+5r3lP88PbfA3AJXt8B1OBYwixY4GtJxh9QaVf1Y3XjWTg+qlkc8T+X3zkPiJQwa4pzNscCpSMdCnahnLORUNMdCgxb2P0DD4IFOFEdiQU88/fVtxe1cqNP1xm2HMWuRjoX7C3RUIRiqrbF5W0ug2GdBaOdFOtB4xXKq6MDtVOBcVMdilwY/OC27tXSkvRXKX+GWsyG9/F0LQp2M0tKxwLac7vgxiuIRVHvXtIH1Ui0+Sg35WaKAZN2Rcg71pDPgA1WLoQ+44HWhE91lS3poGh6yajqK1XKQ0oPglzJB6uO9rB6ethokiqeYpvyBP8CENhWuOdJTwgYnwa+K1T3hNzqxX0KOBTMXDPgZ+PHtiSXls9RJgcvKEl/G4wz7wGPATT6vk+WbFedEj2PBRmkG4Lz9iW9D2JkQ3LMjomHQjk3Yo4BDwX3JQJ/9Ekc1GN+nOM4Nm7Bf0EzFMzgW0g+92UR9pzZTM8PBYAhedlKkI7MgLJ+6rHOX6PnStGcopBcHS5Zu0vcp+ADfIctl4Ips/SaEj41ZUkV5WUp1Ro7/s3JUzirObMxxzUKw8fykzkv67TJDclYnxsFp4g1VlBnHgtfVMgNzQiFf9F7wtzD0O5YsW1D8OESqIwbSjlZIBuRxNOciCAyrLCKO/vXwndnhJZww+YwHkqmFeFQ20/u559HiwuV+bfexbR4M4O+YYF4aysMW1dW6wlXdK9ku+mvgYrFbDpiXWmIfVRdOBW0Uy0LjI6HxDR/08jkYaFAH+UPKZQHXR2Yo2WAV11SZ3+ghT2DloTxi+RtOPpnuSGdHrYt4VrbJvQfA0/wGkc4GZnz077kLmHr04E3xpFdG4g96DyVNumzdXF4BB1zowQot5o5FWmwezi0wt8CNaoHrdiyyEaUhHOLj4rvBVsPL0RpwErSgNMg6GWw5b4uQAQmDd57A8bYZhwxIesdCgxA2cONYtH0WwmEJgdj6aSadqT+QpwFWe+WsOlvPWDAY4mQgpLC28O5wWAolLaVnzFis6qmtN3FrcMZeDpZD+amgaM1D9NFpM0vBOXYs7FzIsWDGojkWoulnLLAnAwDCdtpspDVAkPlsc+yoY9w5BQy4Y2F+khWQwYDXuj8ZSxICVbx7UhsSFsAxqo4MndDFOtVwmgfbMb+kEhyRvewha2YM3rNkpczMg69LI8GWMYOby1vLMyN7M8h2yfJHdpaLAf4KDgHOgJ0K7ZfQoJwBOq+RZVDPAB1n4Yxmuzj2ih/OAg4AT92BLosPG6wvKnxFuHxpmxmPg7pPWPLE4J8n+Pu4X8WXNzydkZwzCmUYL2ECDztwT7PM6aQG/Qd1L2IP3gL1Im+EknNxXvypdRwPvqLNLMNFyc6P5uEIUS5qb5/48TE+7IyDgFzPpMgpZjmWT35flEcnG6934FDwe+UputhAd1aOxTPs75DOzM5Ad5P0u1XnBf2+XtbJfg6cNDTGcaHsLIXCobh9/1696UKv2NWbsPbLDnz0kvXWHFx96pLx0Y0H0AdlagnM1o68jzIkw/fJDJwAjTKCj0EJp42r7AVKvr5PE16zCZIqQeAnjVq0cNKaI8HST9WDHngssV9GG/dpA70JXvcJyzG9f8t85Yzqvtgrey8u7nO4Rzh8v8d/MoLLWQWjX8rt1Zzql3oaPxOtwFHesO/AL9FGYWWMDiO5godetaUS717PnofpsjLhhx5VedsbZMFTPfArV7NpfJGRfJwTl9SLlNl3eYkfOIPFUpZpxDPpRryarKD70Pvnb4XqTDuPzi0wt8ANaIFtHYsvPqQ9Fk983cXKxjNCmkg1lK1RHJc8G06aytaYKt7gtcXN2Qmoycu04wxMNLDBoUinIsLYxM3eCjsT6kzDsdBSKN4MJfx1DfoRwYCNp3MLbN5WR5qORdtEysBEZzoILg9l0j+dDs6NZyy0edszFmzi1uBoXQMrBk1BF44Jeyy8HEpydnH6ySAwLYNSXrx2FqdCJzKBecYi91ngVOA8hF3RhZM15L1e5LeDQta0cVqGwNFPBaSSdJQBn4FDBnsGGn2tqyC4vis6pYjsxOHbYFLIOiVAOZK8WUiCGt5mlBEkbSJgynU+siuvsdwR9cwEdIgPc8CVj8iF07CskfNleRBsYj6rmQCcCwbSvEaWAfhFwV/QgPq0nFK+T7HIU3jdIzfpZAaCtx7hWDCQxxlZFt+Lomcztx0AwTUM9FIlP7XXQBt6XkOLw/Ki7kn2OLDRm5mDdepP517dX6f0lB8YB6+OZTkU36bg4M1Oh6QLS6dwgE5roMqsBrK5rw7qHsVROSVZi1qLxe+Sr3dTjrM4B5K/TzgsUUIvHBmWMPmNTrIXsxaebZQsvqHBm6b44jfLsKBnNhGnh/0Y/PaX9Pvli+F8wRtbYBMcbfZ8HNWsxR3aB3LngcVyh5ZuHVtkWVQsbaQs+BfcOxEqDVC0tcKUYMBoYAtJRa2CRu22ZNyYAkHiHNMaI0lqTtyxQT3cbYleuVbCLYJESh1SjNLZFjI7Qft2Rba/vHSpXL50qVy6TKgN/YqTZuaChxoLqgsebJhWduS+2L9vfzl8+HA5eux4OcBrhTVT5fZLebFPBXOl1rP1TDUxSKJGiXu68W847eq2YFI+pPRthKVSZmc4tenS2iXlINW4uphKShnmi4FBX5VNLYOmygEj8QO7AmoZjSz+qXtNm8a0IZNo4CQCACNwGY5UokE6fMHmjkUzzDwyt8DcAjeoBbZ3LLbdvE3D6+57U9FbI1xzsulkliIPcPpz6lR4HbFwRo6Fn9bFEih/eVtOBfst/NpZfSzPm7fVuQavnLFg83Z1KtTZMphhyUgsC6iDfJ4A0ykpdI+pxp9uR65N3WPBbAVORXzHYk0yeDMU+J7t0AAul0LxRigci7ZxW18qprPPt0HF7MZ4jwWdK7xwLGJmQmlgOnPA5I5Maes3GFEx8AgmPZZM3SAtEoQtOVSHkVtSvIwDoKuzoL6Oq5mktMq9D/r8Tk5SNEkCJKxDczY24hjdc80WrUTGiY5euPCbZAXC5muiETLQRQ6hxmsejDMAXtKo/KIchlfkULCBGgeDj90xA8FA/CVtmH5R36Dg9a2rKsmJfXpSr43St7KkR/WOusxqpCOi7zppcC+nQk/7WT60JL7UAzMVt2jZ0q16as/Xq1c12Lyg+9D7FoTPzAD8YMigEYeBpUYMzOHPngmWGuH0sMEah+OofhNYkP0NvLIWJ4WZEpzLE3KKT8mJeZ2edOMAsM+DGRVmPZ6/vOyZD2pmv+7xE974rQ/naSaBDdsHdOOCz4CY5VXoShku66ULF+VU4EDgRPG7Y88EMzjMNC7LduwHYbM4dMjjbsRpuVXLrm7TbMXtci5O6psYh7Sxm1kLkRoHXrH/hBmb/O1QaTp9GX4rlDmOOsCULj5qAM9MVxBChqPxBJQZ3WC1QhvtQLk5lrwqMoHvM9mBhyTsnVjR93qYmViSM7F0+WK5dPGCHAp9CPGi4nIqljRjwYMYlmLuUX3RngQPbZLHsdDrhI8cOVaO33STwqPl4AFtelU9xZ6L6lqoGPWXv1lHdAJadYwijxKVJmDJJ3+X/E4zXhEdGN4DJCC5jsA10f9usbpxVXeErS1whiA9I8nPWoJVr4sfNmT9k5lWaAI6/OSZzPJGGdF3SBk1X7POWA17hA05Ft95X96eFHienFtgboHvcAtck2NBg52NdtijtaoT+GZr0YBn0+nGvDbEhisPJyCdigzJi7XZkd87F14KxZM7DZBwLvzaWT2BXdagiRkMcHFK1HOoQ40ZiT04FOpE41sWOBbVEdCgJJcxZRkzzO9YtLdC2alIxyKWQoGbsxZ00iyFYolB71ikQ+E3R9GRe6ZCoeJtxgKHQgMBL10g5K/aHHMRb0eFk+47WXrNtLPzBgLRR8L5upA0T9mIwzWUSJZtcFyw5as9OrVDKk8zs5PWQEOJUXcvfZo0dKv0Wf6grfqSjc7Yo/EcKwqvViyyGvPQAfrknZRxv1ZE5RPjuy2ow+Ac55iNxyzX4QUlzFZcrI6FP24nRwKngJkKZhNwKp65cKlckGPBE+UDLOk5tL/cpqfuJxWnkPDkDU28Ecn7MJRmcP+KZj/OaKaD18KiAF+9vl0fxXudBtcsI8LxvazfT+yF0CyJ6oq3SOE4ewO3Btfsz6CM5OE08D0KnCFeB3uTZgFYiqQfoPdRvCw5LzDo96C+1NkKHIs95ZhOliZRZ96noQFtfDNDT8n1ezqqJTan5Fzg8IDLTAibsHESWDqFY8HeEL3dUUXBGeNDgPoNq9xUILMr5EuCSiD7opMG1CyfYraDOYhj0vmUbHYzjhXf50AnyaH+Reon75Sbb18wc4Jjg2OO7dp9pvS4zmtdCycPcOHHEe1UxC2oRoH391NFt30qimW2O9r8A2ssHxmUL2Qhm/sh9k7Ibmrbrmim6bKcCJyJy5cu6Js6l8qyHAxmLJitWNJX1Fc0C6RGxO0KDzUoQOgYztte1c+BQ0fKsVM3l2MnTpYjR49pSdReOxduh4SPdlxNRzyNUHVTMBxZYEESzxbk0tkyCcBJvgkbhVV+s5e1GVINV7xD05CbZUTsSI9aOaiSqmZ+41UjXnZbdYb3Vnjb6j9lOkknLbzDrGkPW03YEc4di4nh5sm5BeYWuOEssK1jMeutUNkoRklnNcJu4pshaFDdZBLWxjtfz0k6nYkMAz9wgfHEMx2LcCqYsRgcC5wJL4WqsHVmExiISCizCTgOOTthB0Np1oLzxijg3g+hkAbfsxYUkH+FDNzWefsUX/deueyP5PHl7XjlbLxVwp0yMuQo2LnQjIU3cKtzN++6r2InAy05Fd5jQVgdi+ZMeACYDgbdG0fY1zbXxTDCGiddTWobA5/RFRvf3JQflUFKBnLFEB8f8GlHChCAuhnlNaSwVyajDpUSm8ZplixnBoazq6woQ6OUzOSs0DgB8DUzOz077OuKWocqK9SNTc9+RawG4+yjiNer8hamOO0IyLFgfwWzFksaSLMJG+eCb0+8oCfJL8ixWNd9ul+6npJDcZs+RHdSg3DuzyUP4sO5lgh95kZvhFKEvQ68Xvaclg5tiDdP9E/oif2detKMY8GGZpwbZh/46B37LFhmxT4JnuIzsD+he4zBOjrjfPDaWGZVoDulpTC3MAPAshndCODgGL0sPmfEk5kQ3ih1SvfxHcLh9bYM2tdlH15B+5wGtE9eOO+3X1FBhxcWtf9BH+RTufh2BpvIcSzYG8CsDfy5d9hQzr4S9mBc5LersvHRSxx0v+VJ5ectVlQDe6vOSydmf3DqmLVAJz6wd5D9Aorz+0YGB/OnOBQH5FEc2hPOBbz85ijuD8nPe0bRdvCrMQvjAO4ynQd/wTow93gVaz6ZBWaP50wuVT5RcEnGEZFgpztfdb/Gq7R5UKIlTytyGvg454pnTNUOqS1a1+zsmkL2VyyrTlfA1b2CfflWzi7NIMWDCqUFY98F7RTwPfsPlkNHT5Qjx06UgwcPadnUotslt30zFM/fu8tbNW6wlq4loShDwRQd2gtoeh6VdBQYJ+tilJOmlgD9p/wJyph/VnCnzxQ/+aBX6pb12HCTjwCJkyE4yaPh18hmnKGekyWqNXn1KdF3g2NxZvmC2ktauvkxt8DcAt8sCxzYtVcrCxa/KezoV7dq265FwJ/bsUDIoEA0nl1zOWqMs7EFy2dt0IHniTPB4ad2Gnywl2FVJ84FyuaMBQ5FLoeK/RYxYwF+fGeC183iWHDWQb+MZedCcC9dUhpnIGYd4skuZaHh52SAu7ahZRl1GdTKsl45q058XYMdlkKB5AFR5Z/OhR0LHAx15MxUWH51JsKxwOlQxamzh745F23mog6A1O2kPlZIdol05gsgQzaro7RBCXHSNMRcT86q+WO0QA7EocOj97uGI/VyPQrfdY0dk3YWm5rJM0gRGBObd1SG1WIFJ+MFoa+ZeY16BpPxNVXLMDbKMkMRT85xKFY0AGZ2gqf97FBY1aN7nAuWLbHXgcEvJ9+R4OvZfIma/QSnNeNwSc7FPpWLzde3an07T9sZFDN4Z78Dg+s9PG3WvcBeh5fFg6VCfqIv52SH8Pgo3Uk9qb/70EEthdKH7ORYsLH7op7q84YnZhvYHM6maJYi8aYmljDxPP+ScL6hL20/5+VY2iMk2C1yTm7T9yvuki4MyBn44wCcVRn46B0fvOMjeyyFuktOCDyZCdGvy3rxgb2v6wn6S9KPJVxH9TFI9GLGIpY2UZ74LTHj4D0TKhubrnEQcC743S7pt8QbofgWhswm3eJ1uTgELGvizVrMXuBY6Jei30sMklnSyEwJA2iWVIm1y8ESr6OScWKBfSxsYI+3VO3QwA167lN4+H4VTRy6A8Ug7ip+JwklVAbMORLB6QqswSgrE0EVV+5PGBMIgsYm1WVo+9TOySbs56KtWdGSpysKcSZ4YQSzGMxubOhhx7rwwKU9pF0k35xtk7ALtnHb5gJJHrL0oGNx/6Fy8Mjxcvjo8bJv3wEtndJ+C+GETQblHeuMgcbg9AdlSEiUx9cexXHIkL/dAW+3GzOQqrUwnI+pHgC3og2K7a/QZjlGmF15k3+GkTWDSjo2fcVs0JU6oAADjWO+QKSlUPf+5ZH41zJBn5q6ZdjLA4a+9MmcOLGcJ06c6NGuO/5ff/z/KL/8yMeum25OMLfA3AJbW+Bv3fkT5e+94QNbI1xHzq233qq3qS5cB8UY9VU4FjQ2AxManziHhp38vqHKhpjQDZXIM56cDFfjlXCn1YnaoVADmI6FnQsNfOxYpHOhDpZ9Fl4KpcHVhjpg3rnP2uJc6pSbq73HAoeiO+NjeVEOGn06DDeqO9SgSgcci3Qu2oyF5IAXjkU4KJ6R0AAsZyyGfRaC2fmo+y3UucfrZjXUUeePY+H9FY6nPXt9UCgslfZOu7XKoB4akg3cUKyoUi5TVp7CMf6A3vABJX6XTXTEy7191Vd6QkLHWlU2JfVpOqRWHNLdrWSaLCj8x7lBzcDKNiA5kmBAXLbQuaFXoSzr4Qh9Qxp6MlBmoM3AnYHtkka8lzQAZukTA/811ZeyHMfJYOAMLidfyeap/wvMNuhc0rlb9y+vTWXG4WYN5vlQI0ulvqFOmgE6ewkOyxHdLzgD6+d1X/OkniVQvChgp072LJyUE/D6Q4fK6zTrwYfs2MPBLMMLcnZPCxfHgj0JR9Ug3CJn4Hu0D4HlQDgoLFt6+vzFchZHRba9XbMmd8pJ+V45OhiAmRb2hrAEi03ZF5Q+onv5FjkKrxeffRqs8+Sf2ZtL+r3xBic7K9Kf8h+S7rdogHpQ5bAzL1xmLHAuMDdOGs4FsyksX+Kr2vym2bB9XrywFW/SwtliqSLf1MAeClwG3oLFUjE+IHhe+umHo2VHqge5C5SZ5Wnod0x8WeL1ukU5V3Is2PPh73NUXcDxNzAYdOuGyAEgtwIOltTl5taFI3JpS/LIHPJafMgOtJoxBbffkjLg7HzxjpdU4BzIUeCV1ktyKDRbsXZFDzLUrjGDAQ560V4wI7uuNpC9F46TJt+HyiS7GLfiUx+4bDxwWdW5c/fesnffoXL42Ekvj9qrjd3eK+ZyhwxYUYwoSlzTVuRtdbTfuXhlPHGTXhISNDvE3taFWzPKFe3BkJ4Skp+407yEJw/yE5a4vUZpSfJm0UAb6tGWDZRegpsMs3634JFoA7msI773zR2LNM08nFtgboHrsMAN6Fhk40lDmiWNhpyGNc6ARz9A0xyI2YBnY02a2YiEQ0V8evopieAMPvJcVUeKY2FvSIMlz1jQGev0Bm46YQ2OmLVQT+tBfyx5ihmLdCaac4ETwJNPdbztiV0tIH1zfn07lkJ1m7clm+UpUi9kaDDDbAXLm9K5aPssNDjqnQqWI8RbouJVs+lQDBu3MZ2eqja7yo41jq040uaRGl/ptG35Vk81v5bLKRTvjqytIOkJw/nIuko89HHhk4dJQm5yZhAx2gNSceE16ozRJUWabyAmKEWkE+DRmDKRlrIaPQDzqzaAuC+rbYBmHCGBOGMy7kmeijNwZgMxm5lxKJiNYKZiWWk7FaJjMMtTcp/mFWIY9D6vwfFpDZLP6z7l2w9MTB6QXJYA7ZczwEwFsxxnlf+MZjL4dgM6npQjcFD5OBbPafDPvoTLuq/5IOMO8eSVryfklNxx8KCck0V/44FlWLxuFnxmGXBs9mpkfETLl45rgM+XtEmz5OhxORZP6cSxwOFmOdZdOBY6eSsVjhTLr3BQzmi51FnxOy4+bDC/E8dC+rPHgpkElifC02+Y4um6bMNSLU54s9SRj1LFLAz1xJKyWDrGK075TbKnYkOzgcxGMnPxiviwn4S3Zy3L0NSO5ckmyD6ofRPUER8QZIM8S3twLnZoH5Vfy6s87quTwmffyG1yLo5qyuOg9OAtVXYoLDs2i+PgMAPDITLP7Cg7HIt6bwBHj4SDTZoz7yvHQax3FfdTu/ucOeBmG5c3rveRyYHDefCrrDU7sVb3crGvYl0OhT96JxwVz06FHSqlaeNydheZIHgvkKLcQznIxSa0b1aRmjAftVdyAg8cOlr2HzpW9h08rPZL7ZKQrDvIjhNkIRCCGEuLxBbXKU5LV55RmODb8294E77AZ+FN4RMyJ3ues3i4LroiEo0ScpU1Wvmx72YJQz6EQqg42cqYovLodQEOTooGBfL77v3O37z9dz7+C+VfPvpHNs38MrfA3ALfHAv8zTt/vPzde372m8LsWzRj0Zo/+hsdmY4y0LjWttOAaEADh3jfHjstGIMEMJwHTj3T6SBNx5kfx8sZi5z6b0uhqmNBeklLBtggCy5P8hjk4DjwUTwvU3I8ZivsXOAMMOjg6WU9s2hu6NULr+kpKUsROOMDeVqWIFkbGnyhOw4By5qaY2EHg3TdYyFHouUpHns6csYinAvsN3UswqbRsWXntVUYtRBXbOrBAcm+msYVFMjdNVB7gqi15DWqR6ExQAkMxStvqAMWdX4tjsWggiht9ODRa5I4di6MNuQ2hwMk3TNWoOlT0+SZZHBGlON7TmNxz1Cw78DLnDRwvay6Zc8EzkXORPBUHseC5U+82Ymn5HzrgQEzb2LykiM7FgzMtSxFeYd1bzFTcVj3HfcYdmLBCjMCvG6VpVIsP+LJOR+A46k/5XlJMxA8uUc+Mxa7FDJoP6TZg5v1dPm4wkUNrJnpYDM2syP8blhedFjyeHvUPuUzK8CsAR+4+7r2eTzLG4QUZ7/ELd4EzkZwzaAIH/oL+i09L8cDR+V58WSD982SdYccC5wi+LHUCFycCWZVluWA8DvFiQKOc6bn6B7IM5hnsMovHdthL56ms0RxkbJqgMzvm70UF1XmM7xtSsu1zi7rtyxbMpO3T7MvR2QbPqbHXhDqgU3x4qw6leXEi43pzAAxa3FMvz/ehHWL8A8JZa9OllRJDcvFydinfJwW9FC2dcR5aHeHkJV02nS64FDxjQ5e0etyKZ+DfHDjxqu3n9MtI4DSr20SJq5y2GngbXYssWTJE46F0jiTzF6w9AknjtkImc9l5R7CacjfI/blIB84deBT8dQGGpdNqLE8NNqnRe232Gfn4qheOsHrZ7mLdVSepqtxwHkg+2pH4mQIPvx8JH0np8cLpOFKXqMVOHGxSrZPA/Y4lrg9PRgJH2NHXU5LN1W7p2l8s0wiDvqOi4qd2QNtrdMKQAY43w2OBX0q5/yYW2BugW+eBTSadF/2zeDY2rVXyey6l0LR9KZQwmygic9qgMnPJtbx2sISTz4Jp0NscXWoxD1bQQergUzbvK2ON94KVTdyqxPOGQtmL3gKuq4RHwOAdCD88brqWADj+xaEXirFAKOe1on+T6NiaeDBgB0LPUHklbMrGvysadDFjAWdGk9gzUODleZAKG4HwjCWhtTZDJwNnBydO3lCKPne/M0AyfartpUuaRvqNfOm8VFdNCuDRU4eig2JBDrMenFxOyTs7kMZDHo5gCTcHXqtbFBjcKNIJatB44j+SWtm9dKXMYlDNBy2ULrKJTflgJoqO9KTJhIw2Tn5U7drukdYv+89FAqXlLZDkTMVguFwMIvB7ASDVwa8fMCO5Tg4F+jJa1v3ijebttlEzZuavCdCT86PMyBXfUMPnE3dOA1824JZDWZC0J9vRrBkhwSzD+Aib4fC3QoZHDPbwVKjA8LFeWCgDh7fumDAz1uTcAYoJAN8lkqhIV8Df56vbcuRwQy3yqm4RcuqjmnAjly/olZw3uD0jF4j+7RmUp7WEieWQvHxvNu0t+OAynJQaTaF8zvFbujOFzI4ZVnZA0eMZWHxNF3iXUd8V2NFHhN7UKgDZgsO6v5nxoKBM44Fs48X9OT+tOQza8EHAq9s6JW4kn9C36zAwdkn83Af8bYoHBUPK8UPZy32u8i5UppX4x7XuV+/Tb4nzjN8rry0ge9u4FgwwPbvShbZpd86TrCG8LYbuL5dFO5UBpvW92nGhPIf0NIqNoibXlyjxsCv7ZyI6y0qag5xks4cOFEe/CtcxYFQm2KnYkXfoGCmwg9GWOIU7Z2XOYkUcr90V4z9WxPAuksv9plwxOxHdUKUP7SlypON4zcrJ1hl8BvydB8t6E1R+w4e0XlUm7j3eSa1/01axrgwtj/yqIceF9j0QCcftgkWrQcF0sH1ajyMN5HV2hKxpA3Oo8E7vpR9loxaWybdikfyva4wiiaSFsk7o7JB38zLMO4ZzPLdsBTquuw5R55bYG6BG84C2zoW+YG86FvqoNddaG2KlTE067PLno29m1C1nKQzrhY/iASLp2zqGBOnhtnJpmOxqgEIDsYyzoXi+cpZZixyn4UdC3UozFiw3IEZC78KVoMNBv8MbK7mWKAlf37drN5g4U7fA4GYseBJMt0aHT3OQXMqNLDzkigGLzpxJHAi2jIppXPWIjZyK58nhdiCf4cMV2rcMeKcncMBQEc+sWRgBMi2TRqHgUe0Wj4QOxww+pqkDuLw0K3h93VZxVte0ItCZIkDfd+hE+/zpvnkWX94RKauVY8aAOYYShTpADC0axTOcJr7qINTNMrHwJhXvC7LiWCpE04CA18cC5wHBuUMXn0/igN0PBlnD8WLuteYcYCG/QB8RI63IMHzkhB5cs6biU5q8HaY+0D34GXReQZE+fCGlg/RESqpGQicWy0ZEi0zETh07I9Y0Mn+gqzbXEaEE4JjsWR+6/qIXQy+cQR4GsjMBK9qRV/e4vSCZgJWBOOtSncfOaSN2/v9NetwKuL3x3IoXo/7AjMHGtzHIF0zAAzqdV/zPQye2GNR6ktqablT7G0gscyLDlQm7HpBNmI2xh/4033LwJjB/4bKyWwFXwRfVHkZfP5/7L0JsGbJVeeXr6revu+v1u7qltRCCwLtAonQAi2QkBCIZQYcNrYIGzThMTgARzhmmBmH7QiHGWbsYDxjeVBgQbCYxdKAhBZraxAgIYEktLTUe22v6u37vvj3O3nz++579bq6uofRRLferbrv3i9v3syTy808/zxLqlblN7shoOIdN+ATgPmOtidDxFXFqZO8zD0MvUnLeiOIOtNAPQMsDeFdUOgFAPieQEggYN06Fmjfcgp67NmCPQ8lIQIQ844TWkzXb0s1MSU1ggq9XblRX492G4RhwhF5BWvvB+hBmvV+T7IRZjljLHPhg/Jtb64GmNgLMJElonqC0gbCM2wmSlrQG0eVhbS5oOFiiGNCzsI2gZG3XQyAduun5CuD7YMY/yiHu3C3d3Slju7e1Nndj80FKnHUdZ32+n0m4Mn9tYcELZJzG68G2cYzsj9qR+NnSdBnxKuPW/XxpdBeDyvERD7UTxyFsCqDA/FzjPhbDy9p1x6TdNXupGO5PSJp/jTfjZx5kuulQQI3Fsu+/c1gvB2Vc/znuAaOa+AZWwO3BhbVBnkOgHkwdagsp7fe55C4OfSnOaDmB/6OIddrGYh5FOH8lolzBbP89noTsID5KMBiixXOcD0rwDgCWMicyfgJLJqb5AkyAAIwSbeSWEip9OypB14HFiG5UFVBYCHzkVWpAiQwYTtxlzPC+O0u3NpVZKCBlIJ7n+k1SpCRJRcV80CdxmTptarfyIg//i5h5T7i+sxIuTm8axzluQFV7ednVdq11sz5RrxqMqT8JT9u4j3bpBxFDamRbW5cM8qH9JbIh66NdKtw0zULk4/Xy4slrdr75VEJKvFJoZF19CUeZLsJV8VzurmPqfZUAQu4L/dSUGohQy/zv869dhCut5q2qVoSmV13p9bo2o3mTMOV9wawIK6M/i4E9sLMuqN02fVahtfVfAupJEJJg0y/7mFlcJV4lDqxJ2TjZZhwmUd+KzkpwERdesN3eCAQkbpsW9AWDLvt7C7X0qi9h7YQSgFaSWMQmi729WB/0YnbVkGtDKj9PDOzAgKNvTUKN1+Z8FAbkjmHdqmx3b13JV/bEdWESIIyAfgtp+CAOpoOkIKxeDDBxoOFJz0BijuHq7ZlHqpgKf0QkOiFbZ78Q7rBUwGb8ZTG9ECDYIT/TdBHvuYtIJyjrAsw5zRfSBbGUOEKqQzvBu3kHZLJkK/kviHzLWuu1EJj+WXGFD1y2Rdsf43PB2hjdxfXq1cf3qYEGP1c9TjVDvoTOEpT/oJzX7EvW7YYyyyXUhalFBpnq/qElEKXsbqO3SZPx7ItvHsJTneJzyWnSZ1JM2TmgnIx3QAVhNtnLL/52IbxffJbwKGcIiRC1E1ILYjjuNeuKh1100EfCHUovER1dGNngWcv0y1H6Y/l95O92g+lrRzS/XhHI1pVb7nAzdj5OX9LxEOJ+X1G2atXyshzYMwrxNg45Si3Jl2el2e1a3l2VJ0cfnbwt3VQiC7XWsLRmvm3pPzAN4GNRb30x/fHNXBcA8+8GniSwKI5kR2uisMDbnMwbcZ0WG2ENwbb/NxwJ78CLDKzcxBYNIy3YUJC9QkmIsAFk3YGFllFyngaN8oEucIrsGiHMXDzujiZXJ1gPYsqVPGKEuVwsuafE3MAi93sNz7cQKIiEsaWME9OXsZXMhISCdIvkokCIgQVcQomBBkyCuZreAAMacjgQn3xfOR0S52WOvP3TWdtYqrfVgnl+NV8FumUiTRor2LxPIKr9Et+Xkv0QotvGB5t6Q8ixLNGHrJj+bB+PEoaObSKX37UriVfgwK0lDTJreRfUvRap8P4QRM3UlAkEtpN6NlpG25Tqxh/xzMZR17IHqAMA0xwuq+ETK3AQiNk8wmmkXu9Jk3DcKsqJDOstyAZ9X4YMpnvDdJfJcz1bpngURhigYWrz4IQ6VNCIJDQ65JpuWeEEhKlA+45IWDo4ZSZVtrhirslk9HV25TMs/tleMgEytTGO8RTdUmQo8E2OjcBROb4RpQAuKt2F3momqVdRR/9T8md+1Io4bBFlYzEN2gZOOMbJB/rS+mNDLtl2JNW3u1FIqBbW1fvfaKURObcze8mUb26wvno6now56of9WMwHBJEaLYeZNC1AQnpDe8E8IIWjchtM6ol1wl1GXtqoLqV98jA7W3QA9VE8ms1z0VsM6b1UgXt7ZRvDCP3QZjobu7JhljGzGDJb9Zqsj/4RNa9GJFPA8IEZYI/+90wdAsSBRbdqERF2wDKdGurapR2G0o2sk0J7UKKps1/6KMcjEfaZbkvxQ6b23nusVixy7iiBHYDAOb4tUV/iDqHJsvlmOT45HhBZ4dCaOVqyiHl5HmWWNBW1JfS1bDHIIp0m1bYsFiXloV/jnmCii76QICLru7UgbRClSgNuiNdcvAo35w55iPfWa5cuhz6eH+D4qCXGBVNxs3jSlVBNkwuWcSJSsuRgt4S32scOfPyK9Lyh2Ur9JpXOaLOyo9bXJtjT+3lBgWZZjI4MoXybiP/Klb+DQW2XdRDPe2SVElT+o933i61cnw9roHjGnj61sCtgcX9X08PPnopSucgmQdKB8I8GDpMNobFxxl0fbkMvMZv3JcJJ0eI8DypwuTwrNyHxIIJs0guVJnYZqLehFESVMSO23hPyXtZZGARPt+ZTAUWoQ7FZNoGA5R1iwEX/Pbes2G4Df0yaq5sOoHknbcrYMHkn1WhdAWpdyi9TxWJhSuoMBamxRmuZisQ0QgrIEI3s+Stq9kMMLjyOwMLn2XgVuo5X3OdRV1DV72+Y9WyEUJFBuk5RqnnaLdGnEMTWyO9kmpOwybJk7OsWPWMNin05Md5yi7Pc8plAiVGpM27jXaOEvhqptNrIYe4zaNKiffMvWSfY9gH83sRK+KQDGFhNAy/vQPHGXr+tL/MOOYFARhk5uOk/2gzkdn8zCyHahLxBRTq73OJo7hLNUtb270plABM0/4yjHpqGgNUaAytE4AV8ligr25CnIBiAkZUw23zkzmXePdwEOBoYzFJWlP0I/ee0GN0N8/6AJ49XmHWZaZ1K9sBM+uh6pSg5gZ2CAIMV/4FH67+y9iGBIG+p42BKlQCBvNZoUxLVIQARAmDq/+WyXLLQAuYrE/zC/sB6PS35bcVtJlwEz735xB8qc4kYNeAXGDhTtv9MNgy0zaq3+jl1bX0yMpqemB5mTBcx5LnKAytIMxyKuWQXo8AWtAhqJNm1cyUnKywgm9jSFMfAOEckpZBvGcJSuYpf9h4EN+NAEt5F5HOCFR0CRze2HhXZtk0LH/socEzDd21uRBYmKcMuFKeRdrWncWvh2E9jDoV4V4eA5zWddhb8J4SJSU+RS2qg6KoJma5sHIh7ww2WpDi7COZ2EVSsb/NCaDYA1Do9ckds9cAX+5zovqTjRLjAnT6rdUXHLJ7WeiBRvtTfJnEy96cskF9SERoSyUeRUUqACJl8zDMMa8Db2Fd7DnSBBaoQ+kZKgy4c945fm5/7+0PHtHE3pQAw6C1HGXcafw2YhW3Ea/2biTYfL0xXNSCSlL56oPqfWuh3PuwkX6O2fjbzC7f+TfSL6/z3cYRD6r7CGhSke/4y/9SxnK9Od+cRnNshNKSR84p8o+wSLiKyf03wwZ5VRUcX45r4LgGnqE1cGtg8dWvpQceuxxFd+iLATQmkTwQxjhcVczNg2uzxhxAy3DduK8PtD7nLGDCa/yWQWBSdKLMbhfVP86qUEVC4a6zgopNVAq0u8geVZiAYVIKsJB5VxWqYbgIc5VBBgwhk3MBFMGoUz7LIrPKOl/8C7sOmIJQY4Dp2AXQ7MD8WKhQsSCNAAhM2iGV4CqoaJyqPIVUAvBA3uFuFpoyE5F/x8okDFDQQHoQkSc/qrHUnTXqPGTdxHPiCJxKRH7aSvHMuOWwPL5309EIrz0tt+bB0Xiz/DYTjmaLlheqMF6LNwkuT6pX4z3Dcsr5WlTTc+Ty1L4Q0SNuob9czRteMPcXoskWh8SBfhIqTVxD6sB1m4YsHp82eUkGTmbLnGS3lEyEByaYZnexVrLg+rtMooyodg7G06Wp3p5kerVtaCe8l/bSJWsHbaqB8iJxlCq4Yq704AzMsB6KLL/Mq4dxrRnd2LpfxST9dx5mUGZUaUM//UTDb5lvV8O1MdAN7SkIkamXYb8MMzrFKrcAQlCha9nMOuoFKYMSGWkzFmTJaHsW6YaG3jLx2mDo8tb9I6zbQfTu+2GgVT9SBcjyGy6wmKTfuxneMgClHTq7KLdG6QIg3dsOI03QLayMtQbP15RWACq+urgUXrbMu5/0h1gVjz07iKukw+/PmlH9SvuGbfLSLkM3vEuUUS9vAoleQMkEe24IUGxD1bukq4tnFzAu1/ZBew89VWU7Ge1dsmqawMWyaX/ijuNKhgIAQKv2GqrCCVLMfwGGX4nFNFelJo4hY7Sj0gpBicpbSibEekqGArDQNqpo+dvTvpOf0Y9Q7ToBmGjBOBsYQ0fIHp/W2LBwBeC1iqcuJRX2CceALEXNYOgk7VqkmC5kxOZ5XB17AlwInmiL2NuCMkYcyuF4aXp+QrnX5UUBx5ZW0uygDjqpz07ARacSiwOqUJW9lylQjvKdm5aH300cJYAfxvOIcSnuDv4p4SVeEGaUkli5Gka6Jl0PMrhx+KCWd9BX/W6k34icb5rRm3c+8buMo3HjLwNvzj2Pqz7lefXeTeXKicXf/Kekw1sH8qiiGFZFKePssSpUrfqOb49r4LgGnpY1cEtg8fkAFpcY+/K/GAWZRMoAXoZNS25YGTzL81IjhjsWl2s9PO59Xp2K7ANgMOEbliUVAgsmzAAZMH8Aiaw6AKCAAdmA0fdUPapMru6M3QJTKePvhB1Agkm4gIt2GIU2JtkCLKS5gIQoC4WTaqHFVqwwZmCxC4OVwYvAItYNMzCR6fAkj3J10g9vVBEmqFAqUU5BRDkzuMk7cdfARamo6mp9mKf14yGd5pWlLLZOswzVKzkeDEU5StvU246EyuOo8/hRzXklvvk2jmb0yLPxhDjex2/i1KLl8PLAeuXeJPMbOeWgkpeMlvuA/cIyNQGcaWqkmqUAAAoScdU5GGjAgYAiVJ8IU2NIhts4Mvsyj4IK69HV2w2eKwVQHUlm1PckSmZRhlWvQh5r9DsZWfdPUAoiY60kQqlEN1IoVaY2oGmeFfZ5+yB5y8ieRxVHplTQUNpMyYIMnsyuG9pdIV13zVYNSVepsSpOejLTsX8DfVSphZvKSftVjLAnYUrneE8JgwyvDHqUG/JlmGX29Rol9Zbf9wJMka7PtQ3QhkKw8ChuaFUj0sZDiYKqQxoqu6LfxdXvQ/exj6yskPdamoPZHyae5yAgwbJZVwISd+dWuqIkYIEV/8u889W5+TS1ukEeFUCAqR3gHMQrVTEKl1CBhKpU27SFbemGhKo2ueO5oKSLd/QQZb9RKjFP+vYXXdE+mz05xliFt1ymoZqa9itKl5RsCFQuoPpzJ8DkIvuACA4EP3Z7gaQgS0NzadDQXe9Z/rb+9JrlJoGCOOnI/dI6FaCqXoTKEvWrO+AOAQblbyOsm5d7aNQ+Ntns3N/BvmWbb5XRhIWRLehfWlpKywALJRYCgzJWmJ79I75r6lMi7a8ulDi2OWDrEc+r0jnjOgoZJ+LF1bHTrmxqHNASbrXpr4557dSlEotO2qBTl7NdfakdI+6wsSCO40jjsJLiMEFuaj+rB5CYA83/KR8mUXv9cFolj5K+vw/HKc8OhDfoN30yKPRXP+OdQ3Q3yl+jKcap2ruPR2s9u1idMp+IXCtczrSiJydakj423o7KOf5zXAPHNfA0roEnBBYPPpaBRYzI1ajpoN4YfKvCHx7463XiQO+w8zRmpwAAQABJREFUWq4xwPu7ihQTgXE4m8Ai/xZQhD93J1aYgJBYMLEWYKGhaAYVqBbI1LmaZ1xOV8PD/oHJ15U67Sv0CiNj2MZqcHZBWyQWMGKWqzqdgFSHCr38SnVBX/O7MB27MB9O8tJrPagKlSURXJmYlU5kRiFLMFRX8Gxx0gZMZACR42ZwIROYmYBcJ2QedDQBQdRN1JHMb1RmNEnkY9kok7TLCNbVKazisuoZ98SJq39ILy5VWPyIgEgaiiQjx6+CueS2LKF1uqMdjehRIpCFzKAMWfzj3mwNk9nNDHdm8gRI/lNNSW9NMpjq0quS4x4C6ufLmPuOq9Z6ApKhzleNrgEVvCfTzmuZBG68j3eq/KhB2lX3p9gFkIcG1KYjkxn0kJ+06OLV1V9XsmVkBSiq0XXQj1QF0i5CAOCK9woM7Aor3Xo48t1BQMV5GNkzMG+9MJtWh/UjMyo90rgOIXP0VyUmtn4PaSotoIjQQaQot3r9AFTykcEXUCyqmgetAhHVoawnmWjBkavpSktUeZLpNw37RGxwR/rZxoG9MkhHYHEZsGD8YZj2cZhzy2MaQWRc8t4bggrtFzSOvgBzfg6vUq7+20ONLxCxjPYX6bfOpnjnAYDFpdn5NA8gspQ9vNePu9tR0tBGQ+NxjdKVFvl9+67p2TfcoNDdtk3Z79b6ttyqSQk8lCYJaAQMo7rEpcybSjx4bj0pNREI2TYTAItz5HknGwKqxqRKUxvEm7dSDg3GbQtVw5ao31bKMkB6Y9SLQM32EYAIPgOkWVZOaVY10zEqXAIT2EMPHgZUjCNCGWMH8G4aotUlCtSisgrURlpmF3TVOO3fbeRhGxEhVC0Vx2nw3kK7cBNN0fi2iO8YGJJagJdqZ3m8M317Ngf1V1Sh7EaxR47tKuhDFU1QYTtoZ9ElsOgpxtu0p2NUlCxSivbMd/41NY8aU0956/EbdOaIT/lvSac+/phYCT+ccInnc//VaToct/yO0hD/8Y5Ik8d5NGnGsi+YQz64byThOFbCeVrCCTpMt78jJn+8lqjH7maraj2+HNfAcQ08bWvglsAi3M0CLGLUc8CsBs3MJsWw2Cj4gQG1EZpvyqCamcjmIFsG03juhMAZwIIJ0rg+VwXKsAwwBBasaAIgYh8LGAHdzW6EGlSWWKgmYKEKsAgGUYYKBkRpRfPMUozDjHiUw6JR1mJnsYP6wnalCrUDc9UAFnB/TmAy7ieYtGNShiHIXp9IH2YoSzDytYXfYWNBfN+kiHGVBvOVGQhGhTJoQxLlqJiFUjfZpWSejGQ+2mDuVGno5vS+DQAjyPAMyQlll1urt0/kZdNUdextHFX7ei8oKy1sfNvCs/wtz4wVzw6nVcV09d5Tpl2mMH7zgkx6NpyW3czMpKDBvGSq1fuXiTOewELvO6qiyBBadzLm2g0scGrs62rzhvXGe6oluWouGOngXosC+4F0mja7kCCtkFEENEBTbHpHejJ59jopcqXfN7ZhbtW9XyN9mW7dFNOI8VyWV9o2eLYGqNhgZd7+ab0P9nWnC6ykn4N5U6pgWpbfDeW0bVCtyXw1+LaMMtPabLhbNZEDJCllMY9w80re0rcOAxsAh7im63tKU67DROvNSEUr7RG0+7C+lFD4vtKH2MnaMvgOTO0UYEnXsndA43lOd//OjHL20CRQs37cUXwGKckCcVd472x3D3thdKcRQJNpKylwRd9D2nQ5G16aeOfq4mK6DLCYQYVrnbrsBFQMwuBP9PVCj/t7YJTO92x8vULJ0Nt2flMallPkUEcKlT/uAyBCkw2pu1f39BgEEIXERDsP3tC1r0DwCsz7FPWyDLgY5Ps4AwC5s78vjXRkCYSqU4IUVcKmOd1oUOmRIGdQUIE0ZEKmn49hnWcL2EQIHM1e6YW7i/tdOgZp+9BGWZRUjFGIs+0nAZWoqbEHh5v07dPu4f2J+lNisQF9FIHFBtqFPHgN+wvGM/JQ9VKgogctPgiuuSKsizIOZGDBrufQte2YR/qOeXYexxEQW/QjfnELWKBOHXu0Netik8Ve2k9g0Y2koqu3L1zOarxt3/Wd+tEcO6Qgt0njhqj1+NL3d3HkdCzLzakdlUeMUVXefsN1mm5OoRlyVFolrFnuZvxa6euBMZYf9SzoiDI4v+VX8lUaOQ6V7xhYHKjW4x/HNXBcA0/DGrg1sPgaxtvYWMhkNkZ4Rvr8z9LmiTXfNeOUgTnCnRjLgM/VpOIso6y/DfdkxpZxKFILw/Lus0yycFXZw0lWhcrAIksu6jYWhpfduaU7gAWTpV6hZLozsBBUFH1mJAxMwq7yO5E0JhPpliEgEY0iG8AC5moHxkDJSClXAAsnZCbvYOJJz8lc5iMmau5deVRi4eRuXjKZecM/dLBbpCUDAZmDVVZ6FxeXQwdbyUzTP731BFdDuaTTHb87ZRJYhR3o7089vTAJ/O6EQeyAWWlv78hMi4wa8Ut7NK7U78GDONaBGVSPGvMeN9V6aMyFxiqRZJDiNDFfrV6yLWUWXdGX2ZPpVx1JQJH3csjggVoLpl3G3faSPQopBHVk3BOkI7BwNV/vQlmVKAWocJfoSYyZZ2FiN7hXR1/pkRu69XEO+Q6nevXWQZYUAEqgI0spILii1zLJwKrxbjxlC0pG9LjTQr0r6ZJpW+D3imUgjrS7L8EWwGJvZYPC4+EJhr+/vyddwNj4LIy03oOsC8u+ChOtmo5MewATaIsdnQEAMtWSohKOBteq4yhR6SDPNvqMtNksJ6kL1ZVk5l09n0QacIX+MgcNsve9MPx9MNuqY7mKb70pqfB9wZv3rszLUKs29Fz2tbgDpnuI+IIkPTut07+1KYkTOpa1xQBUr3IO0sf6K8nDCMz3GIz6BBID7TLcE2SedhDkLME8L6+vpSnsLBbwDrVEWBvvDcLUngFc2NZKa66Tps/sHyeoVw2fAxRTj0pCTNdvWWBl/xR4dHMVZAqs/P7sO9aH9iN+IwKGxxaW0nXAxSK0d/ItjPOd3DHYly4C+DTGtl8t03YCsxus/l+l/qwXJUCqsI1SNr1oCRzmsYWYwWZEqZRk2Bfc20PQ4Rh1kve6sacYokNc7GkDVHakM5ztLDiE+hPvbXD6zSsFI+sYD/QOp12FHsYy+CB/7Ds8BXlxUEa9Pflb4Gr5tq0vaNeRhDYaghHtUSJd2jnGIa5FraqMPS6wFGDRTVv0ACq6e90krx/Q0ZHHK/I7fOSgHA6lUQcRh3LYjuUoY6LXMuY0nvmW5T4Uv/67xPVa0vLeOJ4lrFx9duAg/fw9Gz8/KXHL1dB6nlEa3+MwTvldj1PSKnEaJY4HVb04Nlc1Yzr5/Ux3M9w8TEUa8jWeGUjAMbAodXJ8Pa6B4xp4utbAbQGLMhnEOMjg5798NAf6oyqgDMwxWFcDrfcx/Fajq2OsYY62BVhEHCZRB/gD0gomTlfmJDr2r2BCDYkFzFrZHM/fAS6YgKVSP/z6z1e3OECFakNM5OG1BSYkM/QwKMRxEm6M9rwsbXswDuEW0o2sUIXSK5SMpHYWuoT0kLGRyY/3TcOTR9K+Dc2b0OkpYxBlJW13Bs/SF9hoym6dCnCMIwOiUWd4i6E8vhM1Xl3jN8yWIKWVVca88ohqCavA3awmyzD0ATKGBgfT4MAg4ELj8QygpLW0S6Qp/RajKq/lOXxIW4CFKp7Py7vey9zqzlXDae/L9Kp0QiZNhnptm3LRbqqTyNArkZB5lnGPVXXo0vBY2lyhDSlCpE2dkY5qKz5392nrWeZ7kTSvsBJ+dWklzS4th1GrTHd3dyfMKyfMYT/tojGtzI/kuxotY6+HJw2arUuNlGVMI33qycMy7MAoWAZfNA0lRNplXMeWYoF0BAeaHu/D5HmeoE+odtJBH+vr6UxnYdYnVDuBBlVuQocfZlvbAevAcgwAAPpgYLVTsB8IVmS2sxcm+jL3SgWy1CKDVYGI6lG+r1THVfmpBZhewJWHqkbD5H2WOnCTOIGV7WJcAZN9XFUhVYaUFJwj/hlUt7R5kA49LEnLEu2l9ynVpubIYwHAu4BdwD7OCLq0s+jrS+e7sx2JXqEwTQ61soUtpIrQ7UJAC228TZlDqsM34GaRgvw+QExpDw3iZfCVDgmdYvM56qz0WeOpEuU+IkE/AUpr3ENE0GTZ/HY0eNfNrxveudhwFVqnlFrQR8g49bFCf3qgLz0HIKXUQrsIwZXlu45UQ4DjJn2qkGkUz97gYduwBThaw15EuwjBzz75tPO+KnHWVSv9qHdvm00KTyD5QTVrAFWvblSzqJPYv4J6doFgB+maAMEuFQf5u+DguMcD6py+RJhxigQi3qcuHS/i3rFPiW2Mc947FvIdkb6LEjFSkEb5xv1SYzHDhQv6jGpXqkIFqFAdDWDR0zeQOrm2Ab7CyQQ0NY6qv/i7pOlYEOOYgZTF7lKeRZDv1I6IayRiVpd4r0Rpxs7joOGN9KtIjfStM9L3nUYcfpfnpt9Mjx83BeQEG+8anzhG84jwBv0lND8rfyXBw0vOq8SDrnj3AAURt/6nJF/SiUSqJI6BRb2mju+Pa+C4Bp6ONXBrYHF/llhYMAfBmBa4yWOgf28fWJTKceAtk0IZfl3NjfCK0fO+bPYU0gvCnTRdkSvAQsIFEU7YsadFgAtBhZOuDGtRsSnAQjWobGfhCqGnK5/hhQWGVSZGRs0JqtDVUIWCaXAzqwAWlcTCiVzaPHwve33K6gamIfMQ0gdWS5dhbBZhcNZh/EICwcMoYzAQ6kjLVFQTFWWXflUdBEiRB2FWujWf697f3JOvaleuQrYDMFQz6WTTK1Wj+pBgjA6PpPGxMVbPYR4AG3qD0SajgIvcgiXNPEnaOqQeR5msq585LBMS98aznFkiocEtDGW0XWbMA3AQQXCxwTMBRoAM2lE99mXKKeNeXHhmNSAYLcqW1ZEyUDEOeCGv7NNOygn05iRTPAmouMbK9DJXEVInzPEIzOMwTPUADLtqP7mcuRQyy2UlHjYuJBm6d5VxV1Woy74AgFBqIhhQYmI5BR+yPaoFXYMZlfmXUZc9PsH9SU/pJJ7SMKVGxRg6JAS8o3rOPMzrqswgcZVsnO6BCUXC0E/e1lN4n4LGGeKqdqNhsAy03qSAXfS1LOHopN2t6xXLg8RicXklbcIga7MwCKg4zeq8K/MaNSu1ECSZr+5nVW+6Tr+ch5m2DGMw3NooxA7XgjHosk2UmOh+1RX9OeLP0X/n6MsrMLmq8AhgzkP/CO9YPyukpSRoGYBAtQfj3kfDtVNHlsOV9mAdeSh80xBecKZxtTYT9helfBqraxOidMVeL6gwziz1IRjapi8NUb9ugDdOX+czDilMqGXxPhlRLtSbAEPT0DtD/bhxoV6Weoh/kbqxXgR8buJ3HSDvucjY4bvaYHRAqVKDZcDULpKKbc6t1ZUAvCf43lp5Xxp1M9tD3FEMLC70tgMqupGMAPa1q4HuPBY5fll+xrWqP5X7kNBSNn/HFwj9RInvPo95GUjsEqc4rwiQQtto59WUVvCbuszp0OepukiRP3mDQoCFQMiFCOrA8aCXxYc+wKHAogupRVt7JwbcefftxqggXSZmS1TXvMiQv4tYlIintedRliqQS47Jc/4ddUhnprb2NAfWAprpN2KTTy7vwWcHX+LXobTKO410IKtQ1nh26J2SZlUF/Gy8UR7FNRfdlx8nAZ9UjyKtEq1K7hhYHKjO4x/HNXBcA0/DGrg1sGDn7Qc03nbg5U/841oGxubgevslLwN3DL1ODLxqmKdMdLkv1wPAopJWCBqyylP2CrUJ0x/7WQTTAePhpEscx2pXqgUPSiyK7UETWGSAEeCCOAVYWBopEzaEKlQBFriMzBILVBpguKTRw1VqgUWAE1dMYf48lD4swdjMLSym6enZtMB1HQZNY8+YR3hdACVDEAxHVQ8ma9oBcaLepcejmprj5WiUYMJtGxlKmYJszM2qJOomw8PDaXx8PJ2ZmEgjQ0OhLqXveg3ZS1lN0ToyyQB4UfLIrMFI5F+5D3hvftID2TDDeU+GRVbxF1mpVsUpw60muJA5t81U/xIYytzK4Gs0az3LxMoUSsceXIst5+q3zL1rsDLhPlOyoTrPFhkobXCFex6Gbw5Vm22YSMvfC6N7egBQxQq8rkmVUMiYSq+toqqLm8ZJgy5B9frjJnRKLMIwuwYspFHJgvUijda/EoUZ3jV/GolN57BzIXXVmCIGv6XRtLqhx70eBACzMK4znGsABvuF4E7d+vMw/0oW1OlX/UvQMwV9U+5xwL2ep2T4rR/zdk1f8CI9MtuhckSfUhWLjhSgZoyV6LOk+6zeLiQmGI9TDzawbmU1vhYoPIY3qGnfg+EdAVgMwSgrBVClaRig0M29tiWCm2Xi6y1JFacMMpBA0CZ+TwIjpQ+qmoXkAZrcJND6dDO7MdzA6g63D0a7k/oWXNjfBaMCy1D3sj6hQ9sYv0XVlGTMlRJZ9wI82+0a9E6j9rbEKai4AGN8z1B/3jSP+nOhAHY++pk2MQI51cP08qSELO+Kju0EjPU4wMT9PCZ5rstd9wcxL93G9pP3rqACIDG/sJBasBNpoZ/t0SZ85Okk9dPG2Um8Hhp9EPB0Rx+AZbgv3THaDwika9DPN8mXaqJ3uGAhZfm7cUxrLpQoxfCL8XvO36D143PpCU9QlEW7MvuN44TP8phB3wawOhY6kOfx0zGUpKg7v608jvKb9ggbC8rdJahAstmHhLO/rx9JZ3/qUhUKFbcDu29X6UBcHOW7l87IxFA/rNpR8j0cXoty020ZRxsPIv3GryiD6XqUa2RrQTlirKyeR8ChPzelX70T5TAuSVcja5SrFKkRVsUveftKKZ91XI4m2YY1w8tzr3VacomqpxX9x8CiXlvH98c1cFwDT8caeEJgoVcoDwfV5sBaHxLr4c0qqA+gJbRMADHkMik4ccYQ7LV25gkyh5VJOCZiJtSQWHhlIi1uZzdgArIqVF7lL56hzFeGVMY/dtxm5ddVYpkXDasDYMCQxNWJl9MJ2EN6wniUJbkdvLnsuFNuSCu0sQBYMNEXhiADiyz1EFyYnmlRYWmDiX8JpmR2FmCBh5w1VkBVX1D9QSbLNFR30XWkp6uSMg88hO5c7xAlRRHuI+8r2JHrL2ZGwUWm36tqDzIQfag4jI6MpLGxUaQX42lkZAjmuyc2x3L1WyY13osyV+ACuqQ9AAfXA0f1G1JhCvNmZjc29aYjowhDFKTbtpn5l1ZVS2QqccxqBtVKtcykTDusMlmYF0UOJnyV7Jetl2rmd6VeyYNed4wTalQwiqf8QT9oQfpxijpUj1yGb5hyD8j8wWjq6UcGfCPqtfIoRHz78jB9QaZXtSLpjt0ECNc1Jz8D/MiMC57U5Zf9M84GDwU54WZVIMlv29MV92ARieuu1qpWeSwBugQWeifSQDu8SpknaQzD0LvJnJ6jXPEXWMzAxC8T1xXvcZh2DZMFKwuU1b0ZzEM1niUY/eWN7G5ZoOEO0AIQXbmqgnUHakrDqOxYPnuMdgyqNk3x3mVsHgQK9jU9JhnPlXqlG8ZXjcdvQFU1HQloYO3u3XOocelWV8Am+OsXkECfnqXspUqX9LKlJMLSm84ZGXnijbQDvq0jngtatIMIYEGZbdNsVwOllM1vKOigXkxXo/crfDtT0D2HkbySiotI5V4wOpw9QpGPkiZtejx1bzvDuKAk4ipSIg3HlZDYZUbpG9brCHTfqJ5fQRrj9xDG49hKbKD2tLG4kNYBFrhwSif49k9Awwn6S94TB8kKBRwFRZwFwF0cH0oTA71poAsDeLzIxXhFfhluWgLHSa6UzW9C6WsGCFlaEWMh7R0LJtRLSCshNoMM+iBjToAI3ovxofosHQfzqWqm5cvjhNk4RnjE2MrV77wVdacuJE29eILqr4CF0s2unoEAFnqNyuNBlUGkcOgPjw4w3eSUD8vYzC8q2wc8bs4dVdTqIm1HHSV+0E6c8juoijrMb5X3fV7uvZb49Xvf8PfhsCqlfCl/qzIeJq+RD2THKFyjxVdNO54EDYZEQj7wR/U8bmPMy3f5rzGOgUW9Ro7vj2vguAaejjVwa2Bxv6pQlxwn43BQzQN2DO+NsJhmagO7D/IAG1Eaf+oDuvcxBHPNk2oe8CPcsMYkmZnvmDxhDuMqU8OEmvWMsyrUBszYFpILJ2SBhX7fTd8JMFa7K4ZAF7PZcBsVIpiXbGvRBANRPssS/2TiYALwQ68B9y4SC/ex2FZyQX4xwZNHMPIy6aSXgUWWXsigymyGzQQ62ivqaPO+utL7lCWXNRtkaosRq/oyDtJO+aO+YQYEO5bF/DQkduVSxkSViA2YPFd/BSWu+FZN5Uwe4EIVKY27hwaHABUAjPFR7C4GQg1CfWtXzVURU9JhPfleTJjWG4yazJatndsqSxF2+LFKflMwmZPrui3dCmDhqrMTqHFlqVwhFhDIsA7B+HWTnivZsRLLMyUZeT8Ar/mUKV0mnRnaUOmHiekzJxsfw9gTpgqUTKKqLF3wap0QKBMrEJRe90lwEzbVaoIZhiF2pX5d5ox+I4W9MLoTxNP4OKQBpKcOv1IB6111Dw3OVTWSSbYnWS6lKAIe95AQmIzzvnnLzBZja9NQrcuyk2zo8YcbWhLQKLgLhlYDc1WSMvMMICLuPHkFEKJdfVFD5LMaYpOXFOi1yDhrMMmnrIfo85VNgJIJ0hVM9cEcakPgu4YJgvTKpXRjGjBxlXqQoVY16pQ0hRoUdBHPsnMJIBVSHH6f5F2lPOZ/g/fdf0MQYPtYd25OF8DBb4yXZeqVXihpsM7cbVupi6c7VRtHw+k1wIpuafOpDQoAjDyWSHuP+usiPcuh1EYFuBsw/6o2zaLapJTkHKD5HoDymOWkLSy744bfgxIpgchlJBFKJOyzFCskPePQMQLN2t9oV6Fb2mtcXSBogWa4+LS1OJ92kITto2LGR4tROXtRQIPjhfR30ui6kj03gAvbkYF0enQwdUOrNjeCgDD6t/DUH6XlG3Zs4BujLIaFUwry8nt2HMhqToxp5k1dO76Yht+6AETph+PIJt+cYabh9+q35HihgXiMWTwr4xK5E0a/BbAQDRr0IteRusO2AmDBuCCo6OtXFWoAu5GucEf7hMDChKNceVwofwn0CUceA7jEvTXweIdlP+qIcbh6cDhOpEb+HjkLC8fvKi3jl/fr9xH/cfKLxHJq+ZbkpPuW0XPMQ38pOy8dyFdSq2Ia7iG5BseNV4KttWNgYWUcH8c1cFwDT+cauA1gcTkPkpTSwbqcFtoxsgzg1TDZqIvmABrDZ4SXsPxunnwMux1g4eSqCpEEew3dZe9hwtwkT4mFkosMLLBNIH6hzwk4VKEa4EIgkfciOApYxBxl+TgzsGB1UWChncUGEgcm+boqlHUQqlAyDzCzMumuuIZBN1fpkJ4d6JMBQE+C1WhTzzkIKFR52EUvPHbV5d74McGTJokSV2YEBgRmQ/uODRihFdRZFrDdWEY1ZAWpgQbUoXZEvGAmoCtLazDgZYW3B0ZicAiDbtWiABc9qEtpxKkhrvYXqkgFOKIMoTphuShLWWkl61BZWYGxVR3o8tomG6dtoK8Pk6q0gnxzuSgZTMw2ZXdjNw17x0l7UHBBeqqsFLhimdx7QKNubQz0trRIWtdhrFYBTXvk4/4FASygK6Q71IagYQimvh8GUeBiTco8W6095CXjb1vMwqyq8jNFXS1h4G2ETt4Zxu7kjLYFpCFDGrYKxFW6kdlho7L6Dl3BIFP2sP0gfe07BshjAnByJ2pMqlRprDwNzRoCL5KGHp8EHLGCT7rUaNgunCe+xskCFFWtlGyoRqSxenF3KmjsgyZpu8DqsuWzbHouUnVngTx2qRu/gTA0dgUe1adB2pCoxIdphr4R2lUphl+aIElwFdIK0hCgmKbelfR8pF2Fxs9hD0NDu9FcL4DTcL1aqQrl6r+Mul6xBGFh+xJ1wao9dN6NwbrqRfbRUJkCJKjC5G/b2037RlGdUu3KvqD3qfDoRt3JWAt83FvjOn15g/guAmgc7h4b1sci3908NOj69iSe1HQhe05bItJTqqNLXaVa2o1oP/J12vsy7S7dfpM9tLtSitPUq6pZynHcn+O6khD6MANR2lMSubaaWlgEOEF4C3kqOTqF5FL3tm3Q2cmmd3007oUhPEyND6eLp0eRDnYEWFjnHd3J+k3HJ047h2TQBQ2Y+mzDQJuQZiye0AZeY8EESZrAwrHFccwzxgbS0j2tzhzWodNvwMNxrYy/LkTEaEWmOjtwTM2H44ajmBCDcZA+IbDoR41MiUUv9dfbPxh2FgIL99upA4v6mG16JT9uqvS9lLxqQbcIrsc6nH551siHgMeLc4CGRnlLCuUqbU1aj0yLx83Xm2Wxfku479VpKqnffPV9x1+uVR25UWGNgnie32u2n79952333psffQP+OqeWMpVrPVvDpMn+4ym49VTF9pv5mF3aSQ9MbqYX39UFX9Fs2W/mOjku+3EN1GvglsDi8/dX7marN/yEDg5AERJPS3gZjGJgred0xH0MWg5cDsKc5Z0ykMVqXDWoOQg6yYakwn0lYHwCZDD5ugrdABZMyoZr7Gia0hO6xTAW4XIW5kKmP6QVFRCQAZWhjgmV+L7D/5gu9Qq1AzMR6g1O+nqGYnDVM5R0ehT9ZW0rMlCpgIXgBUYrdtt2NTQmd9VqxAr8DstHyg/YiBVH8jHNYBJgPDKwgJuBcZWYmLL4I7OtYek6K63LSEIWWFWdx3h5bnEVrz0Yi8NIaZdgtTqpulIZQApaOljd7RZQwFyoEqEbTkFFB8yZO/LGXhgwcoKM8JgFowG6gAaYTtJapVqXWP69jj3FVZgcXb0ubWQmNVgXSG2BaWshLyxFo15V/+mHMRygDXQBq+1D9sQk8239usqcjYqVVixy774CuvaUSWshLNTLeH+X9GXGe1kFH+ts7hKtHYUMpZIPV9H1BCY9emKaXWaVG3esazKL0CLIGh0ZZMUb42ZW621r93WYpl1VRbIfqvok4NDYXJUtAZPM8S5Mgivr7p/guxoCC25sW1f1r9ImSgOUclDzNkCAvTbAhO5fL8CAqrak3YV7JkTfI57AwjIvqKbFsyHKmoELrl151y6gTcIV0r+qfQSnK9cy06cph56dVP+SdvuvqlMao9tD3Vsi76GhYTVAC/pVu7JHDRBHT0pKR/QeJcBzs7jrSAXcC0MJizuICzSsn0eVGECn6l2CmwBMxDmDp6Xn4l73NHWi9GWGVf5ZmHuB1gJ9RDWwTYyuR61zaD0LYIjNLUl3nTILrrYCXOD6lbhKxOi4YfOhpENVK58rZVFNTCN61Rl7aUvrSSClS14Biv1/kja/5jdB/mvWJ6BnDFA3QVmUnPGRUSY8R2n/xPezSL3GeMG3qNpTO/YUJ2lPvVoJzVqJ3wYc7MDeahCEYVp3nx1L58ZH0uhgf3y367yzil3GJuV2YcD6DecK9NUMKlQzEqjLsOX6N0/bzPFNSWu4jSXPPDBjA0NZta9wIWGNNlnjKvAsY6XjlmmUMdQ8PcpzUg+g4XfhI1WdujHu78doexAnB/14jevj7O7DexzAwoUEFyMcN+KFnJxJBt1xEz/yXXmce3rOvdxHGuZ76Ch99FBw/KzHbqZjuvlJ/K1HqiUS8f3zOM+NWuql8RrxmyQ6YjRfL99v4z3arRzNuxISsaofEiDFVSzFnwcOnhGllhwxMxP/tjceA4t6VX3qq6t8fym96lu668Fx/9iNrfSlyxvp9S9kk0c3i/kGHdL0R59eTD/7A6NpYlDlyePjuAaOa6BeA7cGFhhvxz4WjIsOjU6IzaPc5+vBZznWTYN48+W483mAhzKyc40wGKY6qJDZLsAiwEQFLAJkwAR5Dc9QMNvFY4rxTCuYLBiO0IuW+YCpFVQcBhYy8YWRzyuBeWoId7MwK+H+UVUrmA5P7SwOAosMTgQXDfuNABYw6OSrv3qZVdWjchyBhdVA2aA1NsSinBl85EmnSA0ysGjOlq5Iqk7lyuYGtOiadgnmaIaNyG7MLqapucXw3a8nJhb/G20XdQHjEJsFBpjogPe3TjhherVPaOVUfcqVS+nU+487AOtidJtV4nUmyRXSnIETnoGxnGN1eYtMYhKlbBGXFeEWmL19/OJbBlfuuyhbNwyLDHkHm4a5z4R7MXSCsgRa1qXqTa72z1EfegBahynclfkmPxmofd7VLkEPSTK9buimu9GyKh+bzpGGBuACKw2s3Y9hBb38Fepnm9O6FFiMAyzOwwyPwWiqjuVmcTLO5i9jrxqTwCL2GyEtJQNO/kqEFiizeSohOMcKvKvlxl9BqjUF4ydDraqQ8a1zGXv7lgbR5wB2Pdzbv6ftu6Srmk5WL8sqXual+tM4bXGXgIHVfdWZVJ26AXN5Feb+sl7GYEbd1+M84OYCjK7SBaUQSllUl9LAmv/RmVxYc+rVXesygaZlf1RaIaPtRoJtNIQSmykY80dJX3sLjaT1LqUtzBL97RrSAmmYZ/V8E4mVgLObth5m9fsegIV0qF5WvEndAFTMwrTPeeUcCY9VbFRHmgIj1aCUpLhPifVumFKV8BgGvQIbQYP9JoNHACg0LlO/GkV38szN/Qbtx9TrCu9O4Wb2Et/AnHXEN3DCeqe9z6K2lAEN7zNWzCCZuIHd0yI2FJuUR1XCE3zrrXxbnXxXpwAeYVeBhLGVcHoK4DixuV83RtpD6fwEaoVIvjqgcZP4K3yHAv0dwICjh23ud69LaFWQTp0SWPg1lCOPdyFdpDwxzpFPqDxSD9s6paBPZlCBNy7afZXT8c7x0CPGLOj2CBsL8vXIId7RB2hv69U+d4rv2IUFPcUNDw2kAVQk+waGUjdSi3a8QkmvfdZ/vts4+KaqbBpBRqnFqOXZjHIgAsFlTjCPxzuOSlM23aPxrNxQpjhILqf4+OnG+yV+fiv+1lM++HbzV9AdP3NY80lJqEFQFSDFxIrgm2PXShLxrW9Je9sbv6ck+B/8ah8q7VCu9UwNa8zH9J//GBKLX/l/pxiyW9LPvm20Tlrcf+wLy+nDf7Oc/tu3jaUxP8xv0HEMLL5BFX2czdO2Bm4fWPBxN4fHcleu8i7N+8O1USaTo8Kd7OrP6wOZk2H5HcCCCTUDC5hqGWt+qy4gmHBVLxtwFxWCClhAdTDyMBfuYxFG3DCjoYLC1WeuGjdAhfeWleI4Nci+7MJYaMC9C6OxzapkeIaCkRAMeNQlFjLA2XZDqYUSiwwsglHnt89LHLIiE7weQb/lk8GwHgs9QQdxvLrCmo9Sz7ne9rH/kPneQpKyOL+Qrt+YSpeu3UhXZhbCyHUdyYJ16KTlYfpRVhhjpSmhtgUDGYDH39ILncbJrChXfhMx7bDaKrjAqSmefzQ21mOPzCs0US73KIDLSnswejuoWCUYlT0ZKRibEzB8J2F2XdtvQxe9h5XtfphsJRjdSDjcp8L2VkVHuwhXzbU70djXFX511He54hA0fucN4gAp5OvparuHalV6qFIVZ4GrzNYejOseDP8ezKPVoF3J+PAAwKIv9PNdCRdYaOwrM27JXUEX8LjSa823c7qIrvRCb1Z6exLcqNaj1MA226KPyCjHfh3kG8wCfwU4GnzL+J6m7DLzSj/m6E/aKyg9UK1ImxbzVe2rHRA2iCTkboBFH+0TK/rQYtkEL4/AxLrXheF3AG7Oa2dAPO1P3JPhBuWZRXXK1ee8M7YqY1kqJ9urHYjqZGPWP/m2U1YuSF12QuryEPYF1vswwOIOVrZ9LrhWsjMpiIVpl9EVcJ6iDjphVi/S7hcBDnf3d4ekxz0iVDFy80Jdv06urGPcnHfAvhtQZ79WEiKtAh8lHR1cbQslTYLDUdrKtrbuBEOCTyVT1hmPCT+Z7gakjALyBKnTgJ7LuB9+EC9sK9Bog3dRN886Mxab440jKZmGnmuUYRLD7MX5+bQOGNgnzx3qRyliG3l0YUfVxpji3iQn97ZCBaobFagJyqbq07POnwFU9IZnLNUX10hzVTsq2kWA4DeeFxT8xgDsgnXqqthYuIAQY5v9i/jSSdGjf3KJQymmzNw6+2gsA350AuECglKMvDu34wXjFd+yhuumk7/13JdMjwgRHvYadGDpysCiP40MZ7VIgUVP/xDgB3ez1HUZy+2/vJ4P6Hu8ozzKcWul8AGn42hJU/o8ym/vjwqL8FIbjQwMbR4usMQj0jTvg2nmePUwQ0pezVTyXZVStEHzWSl89V7zpzUT0Uw/p5nbs/mulJUXuBKv/LJGPHyvqo5GesfAIqqm8ecYWDSq4vjmuAaeNjVwG8DiUjAAZYAu1zJoNgbGPKscWfAymJerkcqA7BAb4Q6y1X2RVgSwYLKMlTwY7wa44D42woMxy6CCK/cBLGB8Qj+Z39JmPqqFNNzNCjCYXBtSC+4LIx8AgQk6VAGYBYIe/gosdsMzFGkDKAQXSiyUGhgp+4lncpeZEEwwOWepRWbUY3ddmFCZCpmL/DyrtwgmNPYUJAVQIV/zPwXDJLMvgy/DqbG3M9OB6ck6gzYPV3v3WDFfXl5CcjGbHhNc3JhJ11m53djUPkWmI6ISmVRIU3o1IvbqxnLSp12Iv+MgfrwDPQRiY8BKO+/uAipYq4aR1mg5M9wy/i0Ai30YvG3cVu7gznIfTzOtMJytrUgvYPQ3YEQ3YPwsj2pXnTCZXa60o9KkkXEH5dV4W738xYqxzLSijBL9y7ZhAzzqRCbZFUFLLyM+CuOmq1BVl1zN182odgBtrPYLTjBMQUWFXZNhRtzAbgJmWRUiN9HLaj4aoMOIk6Y2E2F4bdtzxkow6SoB0gahhTpyFV0DccGF/UbwoXRDUCLTqARC6YGqRWVjN5ljGeBB+ocSD0GMHqC0X9D7lJIPgZbepPwmZLbdDVvvU5bR5lOioDTnCuBCY3GZ/zthnLXHsNWUCug6NVTJeEPGu5PT+jJ/6896U1qhwfM5QIFqUEpE/ALd3+ISKkQPzeFmlQwHkO5cYGV7UCkUz+cAX1PYEcwDFFaJu2mJ7fOUawxatSFxAzr3oLCu3AzwBvFv0CaXYYq7iTMK6LwTUNdD/koEs/pZ9iJlG16HSReUCCD0bmU9CxyV0AgcBRdKLcJFLTS6a/hp+tIIZZnlW7oM/V+fXUgLXG37QZ49Z2IknUeiMoBnqivQ8RiA4rHpGbw/Lca33EK69qWTXNv5rruQFrRxnuCbOsWpk4AB+uldZyfSRc5zE2PUpd2Kfg29MvuxoktbyuyrUuRp/7VH+C9/Z/n7MlRgFAsrPqON8neXmfAYH4mg1EJgsYgan65v51Hdsh8LLBwPaRLGE8Yz+o3flcbgAhL7pHmYf9hlEVdw4VjkJpoDlcRikE00+weHA1hkG4tCc0V10E8mHtUl/2j+jfGbeJFfFWxfanHBoRy1W+OXeSTereLUw6L8Zuj/enzi1pKKZ5EXoUFq9bSebiGheT1YEJJvHo3ED5anWfgcwTz9n/P0dUJq6eRb4xC/iuSbzfg8YuzX/sKjlP1t9x5LLKJCqj9PFVisbe6la7PaKO2nM8OMq13VnFZL3DiXp/l+MZob7DmZzo0iUczNUYvFfkkRbxtV6hYkn63p019fO1IVagsp/pUZbAOJ309+p1yZ4hjCfVyHK2fVsbCym67N4SGQsNNDLI7VnpU4T0Tb8jqLK+u7vN/KHM8Yu7CT7ppgHj2inCXN4+vTvAb2UEOenEwz86vMgSw+9uvpcxj+yeVaD/iOtcU0O30jTS8qgadv9Q6niTMT8BU8RY13Y2EyPXZ1IW214NTn9J24SHfJ9O/+uCWw+ML9TRuLMvCVK0NhUFMG02b40UQ60NcHe+PnSZX4tWcFVAgm4rlXTifRABZOkDBUWWLRBBahCsUkr+QigAXMl7SZT+wrILPPpKrvffXdBRZeDzLwmZGPslA8JwcnEE1xY9db3EiGxAJwscOpHYeTg8xaGGiSpsyBeSihEFy4KVdIKwg7YTjAItOgLjOsGsBAVQfpdkXYsprxCc6wc4DGwujLOIQkgWexEk/e2V0rQATmMDZ2gwlaU8VjejpdBlxcunadjuiuzEh0oNdV9zgoX4AW6M1XgRHlNz+ZEmgLQowPoyQQsc1lSg2SCSO5YKZdbd8DbJBQ2qPcuzCZO53dab93MLWyGtqGj3zfW4UZVSVJXXKlG6dgiNtZke9lFXkAlaR+mHy7lcyjzL5eh3TvqpTC9X/ry5V8jYxd3ZZhl3G3X40CXrTb0B7GnbhDlYY4PTCbFCsYrlUYMgGY6kPjMJlDlXGzqjgaDutNSPsPGXAZb3oQAAHgZ39DCrBBG1l/PTDPwwKEAENMWMQ1XIbX6YOf8dsyyAwXd7VKVkL9y/RlAomoXYZ5W44x0tQQWWNtfwuSrOzwMFXRZJnn6SdKdZSm6KZWYKEdhozjZdSAruKOdZY4O/Y38jgF89JGXD1ABciGjn4kJzLiZ6l3vWsJLPzervD+w/OL6YGp2ZiU+0j7HGo/ZyhzG3nprnYaYL1AXa5zKsEBkbIxY0caoa+78ZyG3BroewiuZkJaATOPJEHVJYHFXew1IoiyrpahVU9S5m83uw4AVdVqBVBi3AFoEHjyiOrIoE1Xte7zocqZXsAGACyCRO0vQmqhRIU0VPHSHuJb8No0zlVA9ygM+iNT0+mRa5MYaiOtCEBJ5UBHKwsIbVy7GIRbUbk8yXmK34Lf04MD6Z67LqSzeFYboP9s8d367crol3HLxRAPwYVgW3rDmNq+AS1+yy4s+M2RI+/xh/aJcPqe310GBZaW+qPPbVLfAovZubk4ta0y3PHQtP1uVWkUYDjouUBhXvkb9ju1D+fFBW0+lFgEsKA8AosBgcXAMPZX2XjbRilsdVCRSQl6guh8d+Cv32DJzzL7v6TRiFhLpz4XlOe+f1R4JBatn++MX5KK+o28rPMSmlN8/LRKjiVe9fvA63XqbSSPZoSctmXOTyj+gePQz8a7xveV8n4zzZzQMbA4UI3pqQALVZX+5LNL0TZ+jqoCvuYFPen7XtrXaME//qul9BfE87t1EUTHJxMDremnvpdvwV0vq+NvHlpPf/DnCzEWGjTShxT5dHv69NdWD9hYPIwx9299cp7FzdwntlxZqo43v7wvveZ5PSwe7qf3YZvx2QfWAkxssDLXjreJt796IH3rnZ0lerod2j7010vpM19bS/e+pC+9F/rM7TXP70lvfllfI53jm2dSDWDXeOlz6WMf+6v0yA3UYdFUOdF/d3rRK1+dXv78syxKMnZuzqcrX/lc+rNPfzE9hoOBE877fWfT87/re9N3PX8wnVi+nh745HvTH31pORa92u+8N/3k21+QeltQR748mfbHTMfFx3//erslsPhiBSzMpgza5ZqzdiKo7oKYTJGDpvFK3IhCmOF5QI0E829fr8L9yH0eoIIRocSvA4sCLsLVLIxbllQIMLIqlACjCSygg38yia1FUsCkXkBFAAsmelcLQ52Ae5l36S7DglfYdxhWVXMysBBUhMSC/GXwjS9jUFYdQyIh4yxjByPlKmFZwSzAQpAjc+n7m6gxbWofwiqktKtasU/axfahFaYpe5kCqJCP9VVWJq0z8895wrC4SgIzvA7DdOPGjXT58uX06JXraQpwsQi4kHkWCPhPyUgLTMkBYEH6YSgNbYbboqZfzmiuaDIYPNPiDJ/8pBmDOHFVf9qF0d8FUJwcGk2nhlnd7eoJRnAZpmgLhk8WvIVytbBTeBfAop/TTe3U63egl2mWaV0jUZlpgYWAxxV6vQvJOKku414Om9RXH2parupbH3PkoevZdhi1flarBUWqVelVSPbUjesGg8mETtKW8VWio2pQPypagjQlJNojhCtUmP8NGG6lY8YZgSkegVaZWRnykG7QUZQmuEpfXOgqURAcCVQEQ9qGWJ+mKxCxTrWHcOXdHqf9xQQMv6pPls19Jtw3QlY1wDHpqA6kGpDASSbZPTDuVvLDvV6WHsJIXfep86ZJWvvE3Ue9CeJFgaE2NgAIOM2eCxOoUOkmtg+VK20clLgorXhwZi599eqNkBh0YRtwGgPlu7BP0C5mkXxVXVrgdKfvjaiTk7F3iEBFsKVXKuvPNrJ+BAHXBSww9F30+9Mwts9hYzs9hNkPlda4o7YG79qj3AB8qmq1AMDo70KqhERiGOmSdieW2f6vdMl9OawH7V78zlULEzyvUd4VToGekqWzvQAL8lPqYts8jKTiUVZ+Ll25mlqQBpzku9OhbwtptvKdtwH2O09orE0vVYJBo40h4bp49nS65+470xCr/RQrrWmoTT3sUK9+3+E9jfKU8cvvzENDbsct21vvccX7GgHxnFcCTLjwYDouUhjXQw9w2m8oiZydnUszM9OhFqVL2uy22u/Y7x+wTTt67yFr5FjmIajQcH4TOgU73drtIFUaBlgM6SFuSInFCMBC6SJfCONCOSoK7azRd0t4/VrG9HItz+pv2M6lTD6vxy3h9bCSxuHrgTgVcdayt/X8ynsH4kdgbpPyvHnNiUlnOXJ6+Tdd7gj6febZqKXyqqWtnjWC4ibKamKHDvMy72OvUAcrRmAh0/8j3zlw8AG//vrh9WDS6zYW92PM/esfnUv3fntveu0Le0N68Mm/XUl/9pWV9Pe/azC96K7MwP/OffPBzN9zzoWnlvSlRzfSb35iLgDImwEgHnPLu+mX//BGOjvcln70NQPxzX/siyvpcw+uxfNivG1z/s//zw1Ax8n0n3/PcLT8h7D9+NSXV9J/we+7TuPEhDzeD5j5y/tX03/yusF0z7mOdH1+O8Ievr6VfuGHxtIAUhOP26FNYHHflzBsp5u95RX96QKSFDM+NiaPKnzm/dm+nD7+nt9Jn03PS6/77lenF/ZdT3/+xx9In996dnr5G96QXnVnR9p89C/TRz/+mfS1zpemt7/lFen02sPpcx/9UPrk5J3pR/7BvWlk+ivpY+/5VBp4xzvSq/b/Nv3Ov/yr9Nx/8s70ss2vp/f/1l+m0z/0Y+nbRrG7bQ7/T7kenwBYPJAeunT5iMTLQMpwGLf+OTjAlsmivBxDKV9gGei9Glb//UTAog4wGqpQMm9MskVi0QAWhMtUSVkwOJWkIEsTssQi1KFkZJnsg3GH+ZMJlfbGBEMCsrW6gt2qgEW2sUAVCoZE+p0UAljwfqw8whwUcHES5jZUoQQS5hPPKmkGjKI5ufooo7CB2oNqUa5Qakwq3aoMudFdO8xgGx5dQr3CegtGPq/WW8eChKw6JUMHzSDa9dVlNuWbTZevXEmXJ6fD7mJxzXyUjOT2ykAKRr2oQsG4hUoUiQi2AnyQfrRzrMJCMz9sq9hTgzrOjBTMsowh6YZdAnEFF3AsbGQwlPY4t1o70jrh4bEKBgexBSf1j7pSByoqfZSxF+ZPt6C6ZLVmlRgIMDYhVzevu+Qto+6pJymlEGswmB0VbTs8l0kVFGksPQxQsJZDLYlwQWZua1hJ6sHV5sWl1XBBO6ArVAyh9TwloFmCITP9dVRddmB2RU5dtMdpdNP7ABaqVFkXSkqUHE3RZjK6MnBKspxMOmlzJTEBeqhHXczKaIdKFe/GHuzQLmA6B63nOd0VWuNn07uEGpFAgiQpRQYlAjpdjuopSS9H98D0Csi07XgY24grqJvNQvsu/XEb5nwHD1LJk/y6AXLj0H8eG5MJmXXqyA3uVEOjRdMVgcX0XPpbVjBUN7LfjWH4fLEf9RmAww5EXKNOwtAdYLFHnhpWn4FZPQ0wUr1JSrVvsYxKLvROtUofv4SNQD9l04PVcwBnlkHgpsco20yXv9rtTFOGWXa7XkDC0cNGbmMwwOcR+bqZoVOve4LEhnow/YKReduJtALosmHgLukJdluIP0R9uoHds5EwqPK1CoB/cPJaunb1WppBYnEKpr2VzS9PAShOIq1opQ50MNOO5ydV0XRf6yaA5wFXz77zfDrNLvYaa+sVbpXyuBAAOoepx7sY6dPw0IF0gHCBlwOsfcO69/vVMYLftOOEfdRxM4+FZESck8Qpdk72IfNxPFgBxCwCzJRazMzMVBttqrqXxzf7mt9RVut0USADO1IN0LyBrZVSIdUZHU/66TPjw0MYcAMsQhVqmLbWxgKwDx3lKPflWsZrnx8V5lhQwksa9bje5zTK3FF+12KbvcPTExxH0VJeiTqlLTxuoof0fZ5LWdFRqSQ1xn3eq6cfCZU/FX0lbkknP64Tnp9UGZW345qf1OLG7X76gXvfeCDef8gfBeyax011VIVZBwUo/8cy3p5aVG79+EcdWPyrP54JICLTXw6ByT/7reu4hm5L77j38V3l/q+/P5WGAAclzgeQetz3pZX0cz8wlsYHHdfy8bt/Op+UZBRgMQN9vwwA+nuvGUzfdncGLktruwE27n1xX3r9t/agbrWX/sffvZFe/pyu9FaAQDmUdLzrQ7Ppe4n3WuI93nGYNoHFxwE5RRryeO8dhz9DauD6x9K7fu3raRB31K97yV1phIF//vPvRZo2nfpe+Ib0/a+5kGY/8yfp45+/lrpf9qPp7d8+SMEX0qUv3pc+8r7H0rn/7B3ppenz6f3/9oH0/H/04+kFGw+nD/zy76fOn/6v07P/8v9M7+3+kfTO110I9by/ixp7SsAij9eZAW8OSI6M1UDqXW1yKoTWB/u450EZvMtvBzHvy2BW7h0EDYuNpJggQz0Fxl6AUaQVqqqEe0bCnNhDmgBZsMehfiSoCGDBpC5z2YpqR/5teKUSxGQv7TFpVMWR4doBWAQA0N0s6g9ZYmEeTFAyBDDIDXBBWpEujI1AIkssuC/AApATEgwYEcGDzJlpuw+HXp42YCb1hy8j6wpiF3rmHZztrB7rvUlVCSfeqKuqciVVOuAnYCq853mobuGaFJ27q6hEPXZtCr3TJVxrkg8rmNatoMj3BFQBJKSJBOKs6DOOR8ShnMYTl4Q6VgAUQYWn7oBpJ5i9AB4wMbt4htpjJXS3byjtdvelzc6etEM59mCw9PJkQi2UU7ZWr1Sq3ihlGIQhlNlVGiAocKVeycUS+eT6hl5osgPLRDrsy6RvcNVYWgark3ZQhcY0KGwAH8siya5sb6HetA2TvA0DP6g7UpitMRhQ7UdcEdfeYAmGfJ3V9n3ao5O0XD0/B2PuBnu2LUkFWHEn6Uuq8AhEACxmOaS6Ee02Tpl0J6uxtZu8LQMeZao7eL8wkbbpGKDqLOkquVCNaoq4D5G3Rtiu6tP5AUOoR/HuSQoxitRFz1bPhfF39d/dqVWFcq8JN7Jzk7gNbCV20cvfJ/wUjOOQTDr2ARdYwR+CNgVcgjjrSq9QN3DT+jDexb6IGp29oxdaxojrJnSqiW0T/ypldG8I7Rn4MJAWAYqQQpyFHqVBgkvdzAoUVJlzTxF6W4Al68Sdvi+yj4JAyLoSXAg8jL9M+W6g9jMNqJjnVD1nFJrvhLF3Q0OBhaBLGwzjZ9UwJDgw4QLWFuqKDhH1c4pvpY9+pFtg61SXscsYVz969Uqavn49rWOL1M5iQdu24AJjbSQV7lUhGGrlRq9gnZT59GBfugM91bsunIuNJe1/bnSpxGIbAGU96cbVNrTf2ye1mVLyqJQr1KOotw6Mo7soj17JspTU707Y63jjwV9+y9zbLwzXhmOTfrCKBHJ5WbC1kKampmJPFtWwfNMxxP6u1EJbKUFGfL92Qg4Bv2DXPr8PCOpUYgGwGx8ZBlgMow6FVyjczbbxrbr4AVHxXrxdpWGZPWKcdtw4Irw8j4iH/pT4Ecdy8r+EmWbjyNnE80bY49yU90o69WiFTsMOP28CgpxZPA+SanTwXkm/ni4V3jgy2bZ+LTCelnSq8MOPiZODqni18j8TgcU2uB5N3XgAAEAASURBVFqAXnWZRv3dzo0SC7pv+onXyigdPD6DrcNfopJUgIW1+Uu/MZnG8RD1oorBL2986surMU7/wg+PlSCn17B1WAYEqMIks66NwjvfPBJxfv3/mwsbjH/89yca73hz2CtUARGvRhXp+ytVpBvzO+lfvG8qvR1Jy8ue3RXp/Kv3z6QXXWTvnVGkC9XBUJ4++Lml9FLi/HBNKvNEtBVg8UvQ1vUNdLVb6D6+fmNrYP/B96X/7TevpLNvfVN6w4svpiGy37/0yfS7f/y1tH73K9Kb33h3Wr7vA+kzX99O42/4ifT6i9IHH3D/p9MnPvDpdOrNP5feOPhg+ovf+3Ba+Y63pG8/+bX0gT+8lr7tJ06nz/32ZnrjL74l3c2Gr86vfxfHEwCLbGNhRmVwzoOtwyLDKSNFCa8TY1gZlOvP64N93JeXGFj9zXfemLgKsChXJ2d1h2VeG/tZxATOyqCTb+PMoCKAhSMSo02WWBQpAiBCUCHTykQcAADGxzAn46bEIooYg6ErhyGxIA+9L2V3s9kVaoAXogaoCKa7Ai2CCpiSrAqVVRxUgwqpBZN3gA3ylCHQziJUHvBEsw5TqLHmhuo8MIo+VyXKlePClIRnGer/QB1mcq1Bn8ifxAkbjRoPjNr0FJKLyfQQ4OIGalELgIs9mHTTKJN8lJ16CYBhXVTtWybODD4ygLJiyrtkGsyTDP4OjKHAIkAH9abMaI/63QZQ7GJzsdM/nHZxbalxt+CiBYZnD4PjfZhFXdV2wJy6I/AY0oM+GG2ZXnubtgqq4UyHiktmbFQtUv1FUKHthIBiTYAT3bNitiiH465smikJgARVi6zq6673BMyf8ECbCxnICfLWtkLG+GE2WJuFKVcNSu9A/YCKcRiyizC63dCmfQ4lDZUhN617hBXsOVzbqvtv/zpHmrqCPQsjrYHyNdpWw2E3OzsFvYMwu9qAaG+yClOs1GEEZti9LgRTc8R7GE9Dk6QpvXTyOFXPcdPAc+yfcJGdp+/BfkC7ESUEU6xuX4LpVx3KvS428BK2C7g4we9eGMnxiXE8JE2gBtUdQGGdPqZUyrocoEy6hr0EQ/8VbCy6AHrDMKHnAC/uNq0kaoV2UGIhsNAtLEgeHU2MECnLOVzO9vGOXUrvUfNKIjiV1ni6ij7ANzFC3AlWzZVG+V2rgrUFHQIN1ce0kblB3U9z9TsZZM+V86yu30E9qgq1TNrWySw0aJ+yStrbnHt+09SNu5LTs1j9z/ZUGr8P2I/4fldI9/q1q2kVSV6CWddQu11QsbmKyhNAHhpdcAhgiuSsn7a4C49SF6izs9Sd3562VbahjP02dRDSEaRvMjaqPzk+ueDgPwfYWBShUgQUXagEdqPe1QDv0FVAhA4aXDjRDuIUQMWr9khbtOkatK4ggVzG2HxmehqAsRiubcNQ22/AMY20BBXiAsc8Tz8FujxAx8URKCLNTto+gAUbnY1wDgyOsPN2P+8rrcgDh+/FQRoe9THd776M6+UaYwHxyrUev7zv1cN68X8cpk96/mzmyY/yPMe67b8l/3jdjshRaPQ+5+Sd+TVyPJhfFVzSynGJEvsOlV+NSAfTjzzJhawb+dayOVgu6cs0mqovve2N3xu334g/9stCY7nW8zXMOihz8FOVWHz22l66hyXWXkzonuzxZGwslAr8UyQTGk4f5X5WQ+c3VWpOf/voevp3n15iSN1jfJEfYFEFycNp9qUowOJX/2g6DLt//u1NMCL9h4GFYe/64GwYbr/4btRLkW78+VdWQ0rxcz84Foz/Vx7bSO/5+FzYcfTqCeLQ8RLeK9KO26HtGFgcqsBn+s8bf5be/e4vpLbvfH16/SvuSadbmYO+9vH0e+//Wtp63nemN7/pOWlNYPEgwOK7ARYXrBA2Bv76p9N9AIuW7/vF9IMXl9LUVz6Ufvf3P5eunxhKd73mtenC/R9I86/9ufT6CT1I4rq/F+2HNnjh+pj1FOr2lsDiC7GPxaUY8ZsDT8mRazV4Np/dTEEMTgQ3BmkGqhhKHXlrR6xw+6w6y2AW4TAeMTkz6Tppa+QcK4JM2l4zqMAAVsafU1ChmpJpBI1MIEoLMqDI0gkBRdhYOBkzKTsxB7BwYqVoMelURYVVJX/zUiKSPULtwNS4/8Q+9DghHAQWMgbkAyA46Yo5eWUJRZZUSEsAi5BykB90yiDEpncwEare6GnG9J13pU01ig7ARSerzm5iZ34+LPUUkpOoTyf9PPH77gnu9Ra1CriYm5tJD1+6li7dmE3XZ5cZ+FztF7DZkLzM6QqqwCJWUknA9iunz9X/lrlqYSD2lWhMXg86ABQyWSGxoEwxKVE3ggtdee5q0I2nqF0YmW1Uo5RgwEalXSQH+zDdVDLehdwlHIYbJrKba6iNUJeq4Kj2UvcWpQQhwIU0U4YwIocgmULXkfW65CqtKi15Q75TaZ0yr8IMupIs/ado7zZWp0fJbxwm+jynWhF6anoA5m0W5nYFBlKbgT6Y4TGYMl2qDsFAK1HRXmJGRhuG/Bp2AUvEVYXIOtILkeDiNAyldglXYYanOXv5asdYub8TRtwyKGnxuUbeYcNBmCBBGq4ADmZYwV8lbTb2CBDVRv9pZxX+DMDiAiDnLiQtAhlp0TbjMvRcBcBcRfKwyer2SSQpvYC4s9gInMPw+Aw2An0AGBl97UIEMX4bw9A0RzvoivWh+SUkIl3pDOpS7tUhw2/dKynQLe90VeZt6GsnjX765wXUlU5TJo2oBRbuyH5FVSyY4xbarhX6OmkXd0YfRMLhnimxyg4dusSFn48+4z4d2mRMUmY9Q3WQ9ghA9C5oUQKl9OMx2uY6bTO9zPdIe7cQ3kadDFIGjcJ7KI8G8CvQofcqjddBkmkH0L6+MJ92AWwnWCRo365OvutWPCbIYNgGglVBxThg89l3nEsTYyOpDymLo5bl2aQestvXLEn1G4rBlHCBkof9UDDuO/4JNSi+X79jP7YC1HVFqxqj/VFvT343LQzy2lz4nosOG+ur8Q0vASzm3XsDVTHdz2p/EWp39OMAFtCf1RtNzfr028zfBlXJnh54GauAxVgAC/biwA6qiwnlpMDCf/lVSCw3kVTQ5d2txvvGOH8oXoSX5KJCcpoxP0jk3+HhDFOSLFmavLmU38apRrCcMw9zWONnvol3Mn0+zymQSlU3plcvc+OlI29yOk0qjFTCuEL0MxFYXGHvyVG0Yp/KwvqTARbW5i/95mTSfuEoCYfPPRbZ5fV/+YMb6XnnO8N2ok1xCsf//u+mmWtbGsDi3R+ZTZemt9M/+fGJRr8x3mFgoZcnVaFe8qxONALkE/bTWWwe3vCi3vAI5TtXZrbTr/7xdPqBV/anVz335s3+jONxu7QdA4tcX988f6fTX/zGe9LHpibSK173nelbR5bS32A/8Ykvt6TnvenN6ftffz7NCSweAFh8Tw1YILG474MZWLz9nlJb8GRbLLD9+W+ld119ZfpPJz6Vfv3DD8EL9KbnveUd6UdeiRF3R17QLW882eutgcX9SiwAFh4MpPnz82++c0xsMJ0R6eY/PhcclCMG4drvEh6MKeHG9N54JcxrE1jADAsuYHQEFRlYZKmFwEKVIoFF8fMeNEJvrOgFuCjAAmadiTurD2RwESt8MgO1srpKFRKL4m4WqUJILGCuXE2MHXah9SSMgMzBSVYuddnaUIUK5qkCFpWdhVKLABrGlaFwTnG1EgCwUYAFDKpldCXeOAGEYAoEFbGBHcyZK5pKcbJKUrYncaLiFcrgxOnhFYYfdY/1teU0yQr25Un2upicSfMwZWvopLvBXeiB+wbMXQFYwfiU+oBBst1d0YzVVgbjRj0F/YIj2oy2UfUipBa8UQCjRrl6jNpr70p7qF3sAC62UI/abgdsmD+MKuKgYIpOAi7aUBlphSFvZUX/JEwwFRtgQWmAacq06bpUZkpGMLxoQaISDOH2GnS4Wr6CZEZvSO22CXUX3nGsV5gtgWVsmMhVNR93xlZlxn6wAoOnh6Rl2sF39shDl559MIGnuY7QrkpTBDuqTM0QbxnAogF1qFmRZwErbsI3icRhEoZ5FamDnpjcIO5ONlrT9bFGxtOAK1WllLoIWLQr0VvVNWhYRIKwxfUUz7upE71StUPnEIx/3sWaiYpiCyw2ON3gbhI7hUdQf9tGH78Nmobx/vPsO86nOwAW0qMbWsGLu2h7VfoisJgCKEySl7YWgqhz0HgBWgUVqj6ZtnWjYfkM5VnDnfE+oKqD8p4dZyfq0eGQcKzSL90c7xEkBEvsX6HK2Unob6XuVS/qBYC4at4LLUohLLOr7STDzuXUJ3WqWlm0N2HdfCtuGKjKl6BXl7Sz0LmAfYygOtTooPU8IGsYkNgDStEt7nXyn8TLFR8XHp44oR0OHQkOUqgNQJHAYneTU8P87CFGkKahuqpxAjHVoPqh1wUCFxgEpZu0oxKA+CagxzGqqD7pmczw+E54Fkf0WcYev19OgWd+zrjhAgS/7bqqRfoNxUIBdFDYuNfBg4sDywIKwMUSYHN5hd3kKaPjhB+94NCxQnupPO6SHv3I8ZJL0HSS/tuDFM26Hxth1/DhsQAWnT19lE+boUxu/C0/SNuSlsO0jzqOYrCNe1S4YTF+PM7zkn6JV34/0TXTyd86wdVLBkl5KU2Z0eLxofjNOOVBCTG2qXBCe6RHWZ74OBzHNz1KOOmTzg8+AyUWa/Q9HS1VX0Iu9m3+fbLA4l+jbjSLR5z/7kfHY5HgqGy+emkj/d8fmztgE+Gi1K/84TTj64kGsCg2Fu980wiLJlnc4jf/7o/MpYevbzZsLL6AEflvYwz+37x1NNy/HpWn0pT/CRsLPUr95HeryHL0cbu0HQOLo+vvGR269JX04T94f7rvi5fSctdd6cUXW9Pc7ki641u/I735VUPp6p/+Sfr0/ZtpDFWoN9zl+LKergMsPv7+v0rtb/mF9EPPqWpnjzn8xl+l33z319IrfuaF6VP/6NPpRf/sp9NLtz6SfvXXltMbf+ZN6bmjaCn8e1TmbQILB9EqF5dzq4klBx0YnhukVFH4nZmFGD4dgI8YhB1QCwPqffxmom6Ec99Qg2ISvQlYwCxswpRnw+0KWDD5y+A6bjuBCSyaUousNqDXoGDyy4TsRAEjUCY8yyfdMt0aQ6sOta0qlOACJlhGQomFTEA2npbJ5QzAktWg1PNWeiGQaG6S1wQWIQEwX/KAqwhDTcHFGoxVMAxQECuoMLEyH8Z3gzdXOWN/CJ6rSiW4sG5L9VKj/KYODedAkYV7VEHQ0xZcPIanqMs35gEX6Iqzk7TMUAGAjTqgLjKjksGEJFonAThg3CKegIPyl1NGRhBo3Vt30mO6tlmoRUG/NhfbuLfcGhhN233D4W9ZhorC+wYrtTBZqIu0sLJ7EiZMqY9gzX4nkHSCCtetPBNwyMYJHnSbqq2Am78twvguwMyvLqwEiFDdpBUj3lDzIp3wLkVcgYUMnoQaprtUbTL0nrUESLVtPXYqpleP0T3cCyzMS5uAsCegv8HPxWp1XilnEz3aXNWgQdKcUloBAy6geRYM67PwynSuS4WdvPfGEn3JVX6vepLSjeo8jLyM8TrnPkx2J0zmmJua8X4HdWKfdnM+dyAXHGxRBl3Yqio0hfrTQ489ljbxJOT6+ChGx99y8Y501xi+r1nZJ1pIIQQ8vme/HYDWq3iVUhVpGoZ1HGBxFib0AobbSixUzZqkX6qWpiRggfIsTU2nLYBPC8BoeGw89ne4mz0jrGe9Nj1KelNIPxaRguwBRFRHaoOB7QPoXDgznkZJW4AWbn0pt7Y0bo6o/YTG+Up/tqiTXSRNXfy2/k8AMlbpDtpnbEGD6lg9pKktyLPGh9MoEhb3InmMDSMfnZlNDyO14WOij6BCRv9qR2pxCglAC+pFnYAKd9TWE4Z2PkpN2vkzglTnHEDpznNn0jBMuP3Efq1UUaNspRXu5xIupAH8tqNjlN+snqei81PJglT7UAHagog2+oO2Ui5CRN+j33n1W/M72mM8c9PL6HmEC+Z16rCmKtTiQhhyu8P3KvYia5xKLZRu+n64nTYd3lPtLxZfrD+/XdJxv5FepGja2oyPjKaRkfE0BLjoQCKUgYVfeHXQLzz861jskRcWmmChhMfDJ/qTkyC9DCpyejm/o9IxzLOMx8a/KZ4d2cNLlX7jxt8+r2g3mof558NWq44qyLzK88i7EbeEEj/mQN9rvF0lkrPzh1nWaS1kNiIeuslxn5nAQvzt2PhUjicLLB68tpn+7Ydn2dOhPYymVXNaYc+HrxPufhIveVZX0tj6nyNhECy89eX9jNn7sYO3htTn2cuiqELNAFB+5b1TSFdPhQqVko2PfH6ZORNbsI29BrDQxuL/ANAobejtZH5GRO7eFKpjvfp57LGDCpbHh/56GYPr5fRKJBYvxjtVF2hrbmk3ffnyenrlPTne7dJ2DCyeSm96przDWLS3kR746G+nT83gVORVb0yvvrM1Xf3sB9NH//rR1PZtb08/9vJxeMrZ9OjnP5k+/EfX093veGd6w1nLv5+2lqfSF/7gXemLL/jZ9JOn70v/9F/vpZ/6x9+Xzrd+Kb3nFz+bnvXzP5xeMjGAdsFTP24NLIoqlOkzMjroHjzy7zJu+9gQrzksg4ryTo5dfuWB1/HcQVXmU8lAvjoqV+EwGiG54KoKlJN7AAsmyyyxEFQUYJENudWdd+J3ojcd6c4b1qlLmaUUru7JLIRaEhOxk3JWXWgCi0IpFECXzHIGFltM5EotYvKXgYZ2GX4Z/wxgCriA+ZMBdEVSYBGSClcnm8DC++x5CUJlTPAMpWHoOuDCldGs5gAw4v08DcIYASqUXAgsokn4E3Xrn1pdCioCdFBnSjBYVwUQsQfB/FyAi0eu3EiTGHPPAi7Ubw+VKMjwiMkc5izADHWTme/8LJ7DfAUAIE4ocZOvG2JJgoArg6LchtZPw1sUMZRcbKMGtYW9xfbwBMbcSDGQ+Oy7msxhuqdQlelgVb4TJkz7AVd4VUeyLyiVUHWmnXCZYlWeJFtgoC6/alDuYbAIg7w0PZ/bh/rqGBhAxQqXtkhEXCGP/myavCvo8XeE59oMRhMWLdcpz0wbd9FhHyGwsbRKGUIvnnDVsqRDCYTG1kofIC3crwpy1umTeqp63shAgIthdPjDsJY+tE3f17ORNgOu9KtqNAXTPA8zvkt/UI1Ir1nnsZOYwOahBw9Aq9Yz70qLeeo1a4/20JPQHOoyDz70SNrBlkBD7+7R0XThPBILDbdh5jWyFoxoi6GHJcveD7MrsAhJBMzzOQCMthjueaF6j/YYAh839VNCMwdtKwCLHYCF9h89o2OoDI3ifWU4DKZlJlStegyVrOsAnTkkB3swyO3YGg0MDabn33kW2wn2UQDQSMc1wNckV0GM9UGPCiP3LaUDSJ5aeFeVKr0vubv7Pu2vsVkr/UbbjVEkK3egHtZHn/GbvcwmeJeg7wqG2r7rRnetODRoE1iwOODvXpiATupSL1BKB0/xnWi0fYZyXDhzJt1x7ixSMRl+xh9BA220TR34bWq3YHsLsJUexvgBzY47JcxFBUGFthh6r7Nvt9GvNfYOA27ezd9YXjSgGfgOAOGUye/GdjEP7a5WkFIorVhTnUt1SdrAsUhg4ZjnEd6hoB8Cos22Ab6qbYXrW/JyoUNp0QjeoM6Mj6eR0XFsLEbD1axjFBlSgoOHdJRDeoKmWtjh5/Xfjfe4abxSZWA6h4+j3q3Huel5lYYjQEmtXoKIXytTeb9cS9qFlnI1POIE0bn8McqUqqjyzXURsUtSvhn3pbzNNJ3jrL8StR7PQIHFM88rlKVsFLkU/Tav//J908yPKf1DpAGHj0/87UoYPv88dgwj/Sov5kN7hj/6DCqDMPrl0KBbb0xKDDxUZ/oQRtPuN6H6k96aVGGaBnT8NBKKcui+9g8+tRBAohVg8e0AAg2w/y88ORWjcQGFwMKN+L7lAvZxpDOPetQXkWRow/Fzb8uuZBlC0ifwMqWnKSUYHnyS6c6x9vRjuLPt73Y0uz3aPoI72499YTnUtI7aYC8SOv7zDKwBxvOl6fTQX7w/ffAzK2n81femN772W9II38ju1c+lj37kT9Nn956X3vb270rnF7+c/vSDH0+f3X9J+i//q9elcEGwg3fBR+5L7/rkUPoHP/Wy1LH9N+nf/MNPpef/D+9M37H94fQvfm0lfd9PZ4lF08XAk6/G2wYWMTg2R0Ryag4VzQE0h5ZoDqKNcfgI2hy4fV7ARAEXBsYznhdQ4VUGrkgrJNwJ3v0s6sAiu25V/xlXmDABpiXtWRVKSYU2DwIM72X2BQGVCoEMAqfxm5MBScBsxO7bAItsvJ0NuHVrqUtbZ0z9w8sgNMCFaXOqPlM2yDsJQ2VYNtzmKjMisODdzMDCDsHsxKqoKlEwQzIy1qKgJApDXtItqAhbC9MzDfLOE6qTl4CCtKgzVaxi125+y0LrKWot7C3QH8Xzz5UpQMacG+hRNhiakPLYVjSvdZC911TAwnAbzKaHeYnVf677LElRRZG/be/qaUhzjM9hW4b9g+1JvD2ZHjzQbHf3pq2hsbSN3cVORxcJUAtVG5yEidbGQq9EPdSh7aJRtgBF5l9DZ1fr/Z3d0dpG5G1+nFswXqswyXMz86HGdJL0e2CUBzUKh/F0snIDOtWWKGgwgvvkURwbuimdm8851Jue4YKZTuKesDz0bVWPXL3vhZZ+mFO9H8lqyuRraD7DKv4CTPEuzN26diSE91Oe5wIsLiKxGITpdDXblXnf2aSNlunT1wEh7lI9BeO+AcCw5S1rL6pP51k9P0MZhgAlumkVXFi3gpJ1rkKzXRjNJVbqH33wobSHFyFd7LYjTRg9e5adOkfD9mMCdSqBkgbQWb0JaYC/US/SJkEG/lmjQ2kMZt262KYfFbep1+mX16BNW4xlpF87ALgT1GMveUygOnQXXqrOkb57a+jd6hL2EFcAFZOoTa1RNuu7B4Ps5yCxOAsQ0Ij9Emm5Y/bVxZWQXljnMudKjrQx2GUPFg38lTxE90OS0kV9aAfRS62r+iQwMi2BgHV6DVB17fqNdANj7RNIYE4IJlAJbOMb68RQuxNGQclPO23nGHCC+hdDqBZ3B3V1/sxpgNIY/Zv8SS9UlGh3xyCZdccewbi05m+Z759vOYAFdSbYaKMf+x2FjRb0y/5qVB5Anfds/5D8lbHDscAC0rdCGurVfgaYEFToFUpgoQQjVD5V/SRdxz2/d98NyQf5h1viAEEZgCrNUhWrv78/jY4Oo7p2BmAxhutZNrHk+3AcirGP7A8f0mDF5zHm4NN4RtCBMdP4h48SZDrQctRR0jrqWQk7EKeRTjPxesrGredV3i3XkmY9TgnLV9KNpHM72y5xVPn63sG0Ch31VKTIdiyv5t/xi7AcnMOeicCiXhNP9j4AMS8JmA8f1puqSdpFHXXIvLtDdS/qTUcx3767iLRhAIbe9PkUoy2Pykvw0EM65ZkubEu+bqD3ZfbB+O9/bDxASqHlywCc38Bg+8fxaFXfAM/nSjkEIIKJskN3ec/rE9FmX3LuKzTU3z2+f4bWwP5D6UP/5vfSJ74yk07d9bL0BvazeOlzT7NICM9qkfc30/zDf5Pu+9CH0se/PIl7/8F09oWvTW9923enFwzLH0ak4EN22OgrFszQxJn9s3elf/6HX8YT5mB68d/7mfQjLzuNW/jgSOONp/LnyQELcoiBtCKxmSEkxwCeQ+qDtBNpOcqdA4KDcRmQ/S2oCEbYL8azihMTOhNmARb+dmJ3UlViEfrOTPDuMSHAMFxQse0qHfFk5pwMG8BClRiYCCUXRXpR1Aec7ONkki+HZZFZVBVKl7MabyutCImFwELGFHoPA4sAGDCDsbkdeWZphSDgZnARwIJ8rB+ZGKUWrkSuykCQh2V3QPN52FxApy4pQ19bY27VKgBMSj5IIf+rAwvrQXUpzj3LsAHziErUdVZzr96YSVenWU3G3mI17C1kIJttELRV9WKdVPMpAAB6YByDQSK61OV/MLbkF+1Wa8ecZAUsiK/NwjbgaIt9LrZQi9rG3mIfuwrEO6SdGbRuAECfm+fBLMp8CRqUUHTSPr0wZwPUq3USakswoIsw8NpWyEjvs5q7ibrIHGo4u6wOt8PI9sKQq3+v/ryM5xJM9Sr1GwbcrJq30C8WScf33YRvkPZzPqPmwnbBfRB6OFthLpWSyMjLKp5hRV83se5+LaBbhwmXFt3PXsV9q7Ys9kd3C9ddra5ezyE1GKIMhqk6o6RCcOFqvfYLl2DEp2DyT0BnD+Xv1bYC2nXVeobfbqQnMJJ+wcgC5yJ9URWqLWwIltkcceqhh7B/WI1+0nn6bOqZYMBgpXoYZvw5SD00QFdt61GMpPVYpeRiBZr3SLMLul6INyS9ZcmgrvNt2bvck0OPUJdRwfn/2XuPXk22dM8rtvfe+70z89i6Vd2tKyRaaiSayQWJwQWJIUNm9BdBfAAmLYGEmMEYCZBADTO665quOidPerMzt/fe9u/3rFjxxt6ZeUxx7xVZ9Ubmu8OtWOZZK1Y8//W4l9hx7AEszgAEbfTp6Mx0MTsxDmgaKmaUDEEPwZeRtLUVeYtK1w7AwngKjiuNyCegxwCqWabZwF5iGzAocNMAmUEextuC3RueaaGNHPADFNCf47RhdmywGKecPirXxnO+r3qaOqO+69iXrL97V2y/eY0BOypxjPtW3i2cxhDPo4PxgNtXnk3OCgCM9Gc3yEIw+8XSEoEBWWlkdV+1R6UOaWPcQrOQWAh4OOYx3msZBN5vxk16f3gnaGOndkKMU8uJ+Y262d+ClCRxTTZDjm/nqAiK6eIHeenQIeY9xoTqkWG4DbAwpoVgwrlNJxXakyk9dV5U9SpoJx0cC4zDVA5l8rZoszSCtGhKe5hp1LzGJooBvLSpWmmfuPlOucVcD02rzRvlqXN3nufrx1Vak9aejTRlzvFcLqSepv4wxzl/L9fzupcsThPbnzLNWdefy3nlfPLeNPleTl8/91raSmCQ61sWApmpW5kiH3Dqdf6mG87I1T1myeoyD1YnqY1/jDYWJRH+aHf/0/+5W/weyca/+E9RMy3jXdjdBsT7f35PHAwkFh/zUvVHS5Bmw/6eKMBczgKlwXzxwsGc7cJ4CSrKEm/5PsZCE2nUUZEv7ITf+QT2jqdu8Yp4ijq8Tv87DGlgnnmO+gNb8uPA4vvkbtZ503KirFxivmjBMFpedkJuTMoNaUU185qWzUcz8+qxmxNvfHjLCTifey3/ZBgy0xqRtwURfHTDK1QJLOKDz0fXfTaCtPwAFjL6fOQ1mFVi4cc7Vv9hJrwvkxyrh6kxqWLWrS6xKIFFRN+GARNY+IHPDHhIJMybskJiIajwRwffUYcqz0PaoAcYyg6MqMoVjMIlK5In6IGfsPKuqoO0z8HdQupCnjIDSi0MzpWARVpxNHFILWBwlfIosXAV9TYYFfM2AB9MNyu665tbxXvUhbb3WRFlVfhYd678gpGyL+xTQUT0bWMQ229ZYhGqXNLMjTEfNinQxMcTzGEP95X6NH2gBQkZXITkAoPuW1SjbpFitCDNUPqgqloP7dNY2aBk9pWlsNAc7k/1LqQRsizfDkyn8R5ksIPBhMmCowqAxtsSK8e9MOPaRUhnGXLVpa5oq56G9ETVyv6Yeqvz30sfCuL01BT2C7SkizYOUJ6SBpzjFkfc36H/pzhfgGGb7wHgMVb0ZGTE7Deo9QgsNliFv6UuvrADBHwbIhL0MOWNMzGM8axeq7TT2KJOxmgwavQ26kXH9L2uUkdgoIdgpF1dn+M5pQH+7FslFtqTbFLeNvsdnt9HWrFPELj9ly9jXPUNjxRjy8tFL/suPFspmfjayNsw/9LiBQbbz2Hs36CKpPepNuoxDK1/Mz+V4nrwjlmGTKsG6QKQTYDba4yHN7HhOARgXKO+NTA0TF1ZDYcZl36OV+NenNKXh9R1l3yM4aFnLt+bPsa94KML1bRTaKmXMmOKtDD+jW3iGNB313WAWJhewQbpNPaX+Z8D9DwaHyK+Bf3HLRnoiJRe7jfX14rNt2+K7dcvsacAVPAetWFnJLDs78UQnPZHntST/zFuzHdqdLj4cmUlIlPL6Gcm3kRK4mLxgvnFiO+CDOd5wX0XY1T7J6V3MdbZB2BwLuB6qC7SBqUX4boTKYPzlJOw7XIO6ma8qyblO06VeG/Th+IUIKextuDimLYosVAyyIMBLJTq+M76ngaw4I7zpXFlVNnyJz276L8xvEHN4Dp3dmqmGBkbL/p0NUuZd95jc5bW/BpzOhe9zj8olk4+8dfn6lvk4zP+l9i1LZdTuxRpcrp6XvmaaevX4zxncKds65HKi2fv3CsfoD7388pZ3d2TV/p/97IU4fq9ZpHGslM979Y7LnuHX65bStcEFpk2n8/+zeZF8S8x6NYJysQgdnWsXKhSdYQdxl/8+UDxH3z76cB3n08rmzVtUuDnU+AXAQuzTZNzWUD5fcgfGe+lCbTx4WhMnWni9EmvCSzqk7nHMugBOEzDeQYU9X2ACz6oIa2AAfHDHNIK9meqEXkOc+XHX2CR7Ab4aMMQJhUiGDk+4FkVSs9QATJkOGXu+aUvBG0omyGw0MYiguSF4XYpsYDZciU1pCJIC1ytFChkUHEXWHg9gQsBRkTYdYWTXzKIpHzpB3VcqbymDREUC2biEKNNGTHr36u7SlZBDWalNCRLRJSEmE/+uAUtpaeAgn2SVsDQwVhds2oruDg62g9d/C0Cha1ja7Hj6voxqlIyeGW70qfRXkt9H30NXfyQBr1glt1Xfc91aW6f+axMiGn9E0asZR97yZXea55VLeoa//636HrfIr24UYrh6in3ZLZ6UCfpZVW7W+aNfhJ9B8ji2H51JfaAvt4Fze/za6Hu2lvobUimHcLzH0YSRt5nDWa3BTN9Qf91UodxpCLGCTGdXpWGSdfvMem0KVAioFGx2rlGu16AObulXoc8+548hjmeh7ldgdnvogCBiK5W38RK/Qk2LDDrMp88q6vPDtIKimSI52hXH8BWN66vBCJ4UNqTwddgm2sjGHhPwOgOYHMizWao5yJG6F8ADKSpEav1oLROu7f56UlqZ3Oz2Hn/vth98wbvUcQFwUh3fmW56MEgXmBtPj4/C2NtBPOX1PMH6vh7JA8HSA303jROXX8zh8QCVahQK6JOJ9BVr1hKaTSu3oJR38Xl6RHSiEvuC3D7WPFQRUkvXY4739cb6HNJX1zQX0Y0j7FFGW2OaerSBb0NAhnqbazmG3zyzPHPOBTAqaLWCgAzD1W9fH6Y/lQl7MHIAN6idC/Lu8ezBzyn8fohTPv2xnqx9fZNsfPqZdGiJygWBdph1AWYfdC9B9omD0oMNerrmBkbxGh7ZqpYWpjDYxW2P4yrUKuk3o7zUPOj/JhnaMMZfaRalKqIgnzboUvoAMG8j4IIn1Glow2AGy52aa9AIM9VSiddMBG29zC2pKNzReAo3t9wN4sqlwHyDvgZ60awE/mGhDOBE98vsoh30fdOoKG0SanFBajC96QXFbJJ1OFmMeafnsTbFWpQvYD5tDCR52+HlqMkbR75GrmZr1vMA3H04Z9I03g8Engtzx1eqD8fc3+tvLifJ7J4+u4f55p6/fLdKPJOPrkSqfaRZb6UH3LP7TuPVfdInNNXBMgUKB+MtDmReZUJ45J/SM8uX87JG7mUlCgf+8tm5O0g0ef254IAe9+vMh9iW+G7rsH3wxlcXysebW5NCvyJUeBnAovGClWeX4NOzJYJSHjWOHaqzBN/xXCSIn9AnG7rwCJfdwbOx/l+MCYyJ/wys+xHWIlFirrtymE61t2s0osMLEyXV8oDWPDx94OfpBcwAhxrAJyvqc5QSSxskhsNriQWMCUXAAulFeFyNoCFnqH4ZMv88FPHOgCF5bgazUpgA2gILGAYyusBMEivyoTBSTKDDkcA8w+Dj9eafVYodw92KfMyGGt96ffjGrILnehQgYJRDP1u8vDrFfSr0SuDiizBuL0RcKE6cYW6FeoVgot9jFzXNlAb2SGAipILVKJiNZYVmNwP9k3V11GOfL+ro7BDZdsFRhIsAUE/nfZnfFoDReS+iL60r0ntT3BxA9OFnk5xgzH39ehEcdrZEx6OuB2ed1zJVTXKoG26iQ17iChOk/SWUEs6oj9OYehvoZWMe+jfw0CqW+5q7AgMqGoohzDgb9D7145giL6YZeW7lfRXpFHVyhgTGgMrDdD16QbpjxhjHVR2lNVu1YikPf6IileU10f+U1x/gGqNEhGNtl8DXF7DcK/BcB+RRgP0HqUuAAPr44r+AnVbAdQM86w2Eo9h7H9AdesdhteqF3WTzzQ2CDPYLPTDsB7SrjHyWMSt6q8nRhya4VL3HWUJLAxI54r9LpKoLTwhbQAuugETEwCLh4sL0M54CcQToY0PaLNuZw0Ip6TiCcDibzTcZq90ZQLA+2u8O2m8rQTKeq2jxkSVAFWAQdpwBtg+k7kGCBgfogXmVXCrJy/7WNUlwRGDpLiljQRwKK4BXvY5X1902PCkRV0EFvN4KZoFMAm2zG8Pac0W9VLaYV/7Dh3xmJG27YsRgPQE0gXBxUQXkh/yUMpygNrSHs/v0m8725vF9upqAhaM8zbAdAcqjSG5QrrUDf3bsa/w029QvX76dBrj+CUibOuKVZXJc94R7VxsT8wPUfXk5UwbL8fbkRIE6C547cW70ijqZtqQKL24Yr44RiXvEBuSm1tAMtcHsPWRCjGP8Z6fIY3INhI9gJmwnXLBgbJ8gQLcUHeD4+3tE5uE9gkqvK6KVCyicB7vvs/QN9nGyjg1SiCNSm5wvCHoPI3RtsBiEo9QAwN4/2CMhI0Wz/0o0556zhKoW9Tuo+nTmx3J0gvOodfimfRYeTNuVMfWP2/VXJMvfGR/P339PCWnVLLMedG8sj6NckxnzfKW0vpc+kUG3iRJbrNtiQv5obyPAjwhcX7eM0VYeYvD2nkuu6xcE1hkQjX3TQo0KfC5UuBnAYtq7qOVeXKtGuzHyJOYGD1K53mSd6LOE3uerJ1W65IJH6/u8QHPzGxmQLNqjStwIbEogUXoDpegQnChxELPKxEkD6bFD3fKQ715Vq/5sAoiOmAaBRXxg9EKxl/mmGMZ5ZAcRHtsDoxySCywsQi1hAwqBBhp1VCG2TaqGhS61riRDANxmNWstiTISBILgEXUQUDAsaAiyrXsBGzk3lRbioi7+NrXWPNKJoz7MtidrAq7wt6lPhy/JPUoGXra7Cpp1s/mIACZjAaZclyqRdGW7OXqFLWo3Z2dYhPPPYKLrYMTXOdp45H0+aS5/RN9aWfRx/ZhtFlwYR9DN39uprUPox9laEzMn1hRdZ9/kTrlpQQArqu4Ib6FMS5O+4aLS0CYen+OKGnWCdPZi1RAxotOTF6XGBPmawC9ZOiLegj1HiDtCGmHypV+azYGI2q3njBWVpEIuFquncEiqjsyyjKtShtmADCjPK9kQ+9M66wQKxVQh32AMfQ1EgRVmSB8AA+D7wk4ZmBU3fbI/zkr/y9h+DdhPHUxO0X6IfK1ndpmyNA/pH4rqkXxrNKAx9TpMQbOz7d2wvC8i7rMY4/xJZ6chpAc7OAZyW2Ucr6hDjLEtmUDhn8HHclD6iuo2QeQbuNBaB01JcfMOGD0K+wfrLMAW4NqVZr0UGV717WZoH3PYOT32d/AiA7QRw8BNUZAV0r0bHu/eAfwOEKq0kIbWqB/KwBMpv9SwE0eASJom30lYO+hjd2MO8s4Zwyc0Ze67b12vPCuKFnSxSs5pUB8AKYVytME3nggm9TlEJrL+HVTV1XdtkrAoXvZYd6BMdJP0ofT0HYWuhwCLPb56ap3B+9nWxhub798UbTsbhYdMOediCb6oLvSim6eiSjbtLWdcoYZK7PYHug9a5AxoSrcGaDhjIUEpXDSMhhwxznzic4pZOqTFzeYfdrpu9gPvQeQfPQpCVCCBW2SXQTgA2lEjxId2s9LEeBA8BISCN7biMJt/tAr3hPydD5QWnqC9HIfYHGElChLZJV6+L67gODs6zsncIx30OegsSpmN9TVskeJ1D49hS0MwGIMAN+PGpSLFNkugyyqrZq3yyvkGvWynE9tlvup7cM79mzK7WPP5fLr+X0sXf3+jx2n/Bq1MK+c38fLIjfT8M9a5jRBB67YVOeTaitPIs/qXiqjniynN79cvrWSFn/5F3+Rb/+97/0+5jblfb3QXD+/of7+0Mjb9Tybx00KNCnwx0+BnwYWr98EFdLEmD8DdwkTk1JMqjH9xiScp+/7E2pM5jzuB7DaYvJmDi8nsGBIuRaggEk9T2wfAAtWDMPWAkYpPEO5+qfhM+AiG2+HjQF5ZWChznQYb8uo8stqUWFjIbPDRz2pJNFW2+TkD7AIr1Cs9uuxKeJYsCJ6xXGoQsGQmlamI/mSF0RkENOwsWioQgEqqEdIGmBEAszItPBTehGsNgyZBpnJy1XyACN9ZG4MBiag8KdExFVit1D94mORQUXdI1TQnTxDghHgAmYjQEeSXhwH00KkaVbyt/YwssUuYAeA4er+OaoUqlQEPSwod12Qh3YLLqLuZT3KvoMXiw9nBhTxkeaez6dxUBsd0pq23BiNG3uLswiehwvAdphxP7nk38qqdBdMWTtMMSKD8AYl/eGsokqyVgGgqLNMcxg9k95StI+YFJTQRxpJ69VIt7VGiX7I6j3sW7iIPSa/CRjmCa5ry2H0a6Nqv2Yl/wAg2Y1k6WuYfSNS99CHO1wTJGhMLkjRsMpAcs9I/4Kfge/Ull+GAdYTk0z5LuPW7QHXVlilHwFYqG71RGCBZOA5htv7HLcxro0l8eXUOD7YB0Lt6QDmuh0azsNU673I8SYoUmVL17famOyqooTqzB4SE/t7hHHyCIZylGfsIdXAlKw4Ri94/og0u7R7g7YeU99LGFED2U2yeq4qlvYtSjJ2WXU/4lnLa+Wa+voaJSsFUsogsHBx1vdA70wG3NOGxHgTEWAPSZiqUOfQC7FhMMygdTxKoZsMWFwAWHw5SoA7JBDWW5e2etYSLPmeGhdjiza9w/6kj/N+jP2HkR6NMRYEg3NIf5TYHPPeGINEm4Sd9XfF5otnxSX7NgB0N506YN8hrdG2w5glbeQvTSdwRzyH7cHC3FxEx/b9MOK1bl01hHb8xzjknTVKtWqIvnu+Z8a2uIaGjmuBr+qK3cRrCdUzwRW01T5D9jFAhYOSTSbV9zykqwB/qBdt17W24CFJJgQG5AEwMm7FPpHDdTcbedJey+QPdUnzapo/fS0SsNDN8S31dL4Yx7g+7Csw5B8FWPQp/exiXNon97bqfec6JURdPfgwZXow6nEvj/un5pO3nM+nnquXn59x/6n09TQfO27kV9YCmpmXZ4176clIUVU2HeSvX3riIyXYAWV+QadoYCrD1Lm9+UnLzG2JOnDjjzHydm5vc9+kQJMCfxoU+Alg8aR4CrBwvsybjEPenBRjQiZB7FkVc4sJs0xUezSu5Ik8T6hlsphgA0CUk32AinzMRzWkD3woK4mFTDcf1QAWfIRVi1JqcQ6z709g4cdZMGJZASxk9gUWqD+E5ALmpBMmQaZFSUMYI5dtsQ1hzAjQIAc+7OpJJ1Uo1aEEFyGxgFkIVSjSRywM8qmMwikrxbBIxtshGWGlNdlYpL3BvmIlVMaZsgJYRB2gIwyPLidtR1J/QK2Etsi4aL2v8XaACq5JL4jE7dRe08Wx4IFr6TyDNOmi5EJGB4ZJ+uoZgNXZQ1zRGul5CwbXwGYbezDUSAAEF35QyZZf/ImuCzrBnCapi6u4JXNDfcIVZ9SDZ6yfWzzLLp1Ve8eJcSrQFSqukVxEVG5sLS67+1jlxi4AOrUALNpkzvUeBVOXvAXRBsGF+brSa0Yy+zCuMsRKdQQIfdwTDPRzzVgP71lFdtNF6Vdc12bDYGy6Xx2knAkY1TEYT700CSx+AGxtIsUx+69GBosVGNph0sjsWrb2HIIUpSYGknvJ6v9T1Jm0tTAGxSN02xcoRwPsLcoQ3MwDeozArcRCo+3nMO4+8womfg8G/pp2DDKeFon1MEY6V/u3sYM5oR7djLtxVK/GuT6Chym9QWm8/RbphflrJC3jeXOBVylosTBMtGWAxQ1MvDElBEDGgWghDyUMqqPpOlf7GuNG3MCYh9SBunXRzlvoIJA+R4qge98bnumGOfe9kek9pexWrktHAbsgbobfIm3Uo5QSH1XDtsn/BHrcQK9435G0hISJfOYBUd9Mj4UUxwB3O7TjLbTQY5bjzNgWu+Sxurkdamm9AJ/hsdFw82t/6S3LKUi3vapL6V1tfwuVsFdPi6NXL0COBwEshqBFL2l1DuA7pjqW3rmmYbrnWMl3NV+Jo04UdJygupFjTQkFFQEcquZIjBVjrfge8p47vtPCh+6hE3iIhYPyfXfBwjQCBeOMxNxEuebn5l9/vltOyvbdmV7hKF8JrDZevrPW6YAgeQdE4D6EFrE40MjGrOJV4FLU2TgWSn9uXZCg/ycncDMbbQRYjE2EfUWnwKKsR2RQ++MiSzmtR3/FrSivwRTXkqdD2xWb9CoP3eXjfDvf4ty5pb59qj45zf3vh9Wvio1EKb9E4lRwPU9qH7Q2ab3kyCPX0wqXmdafjWfuFhYlppxyblUmZFHOfZEqw5PIpXyusWsCiwYtmkdNCjQp8HlS4EeBxd98n4BFY6qMteOqpU7uTrhp0s0TqVN22rySJ+R8LU/U9z8MrrLVgUXc9xqTsh/1rNbk3vNLV/T4yKrbHB9imQBVoUqGSheRAg+BiMvRGlZnNShVoe4ACxghJRbRFhiaYAxyuzj3++Kq7LUSC/zgJ1ChSpSG0IlJl8GPCNnkY15JYgF4gDFr185CkEE5ASpQgapLLAQioRJFHQMokJcfSukXqku0TQPOoDdpzEsf+SAfufgANjIYqm74THqQ67Q9MfSJ+kFfwIRp0y/W+EnufY1EdXOLXjvG4gesmO8AMN6hArOF3cXeERG6YQpDLUqaUlZs1LUV4+MAZTAvSimiHBkx0uXV01QPqsZ9mQifjp9/2KLeHpPHNfYW16iRXBDf4lJw0UOkAkEUIKCVVfIWVaFov4w3AyDAVyIYuVCvWxixMKSH7q0wnNpG9EGrGcDAKIy4xtPaC7gSbnTsBzD9g4wJJSsbMMDGbNAGYlF3pGQpeHgOg2sANxnyFRjgBxg/CyxUdXKF3JXvORjpfvKhRsUq+T9GdUhXricw6nPUexYmW3e3em46YdwY50EDauN06Cb3LWWvAmJUvTLeg8C1HbuFYSQbfbabuh5Rh3NWq1tIo1RgHFAxDfBQvWoPRlxJzAF1MgZKqCYBlgygN4E0YIB6C96UPmiQfU35LeTRC131WGUQPOuq+kwH4111Jceskplu6CdjuweTLT0MxxZjmjSCCcdmN2NBumn4rlG4gEYVsX6AmuW9pW6r1F3pyAXPaKNwg0TiFne8PfT7DJKjr2Ynii9ok/2hmtd7AJY2E6pTGcdknzze4cnsHDooNejDnsH6DVGOKlGD0OqWugosVKm6xLbicH212HjyuLjc3UI164o4DgQIRBIjcLqhDoY1VNVqflaDZoJt4Y41zTnMLYwlJQeCilA5ZNy1ILVSBUxA0YXjgQD6vuPklxcGpEeW1IUUFLqEdMN5JOas5FzCdL6Lbmn+0WVx8vSkGpZgX1ur05CaMLKgg2DjEHUoPUSdMQ6cE+N52u0/3wXfU/sxAuQxZgXdAgvjmMzNzMRvGJVDDbfTAgVP8tydjXrUr8X76lvrwSc26563yK3Ms56P96t0Jq8Vm6/fT5/zzHvTVWnNgnKiZP44uzSaUss8Pxz7XM/UxlTtD9PmMnykkScn5eP1+41JLFL7hzqW9Ykz/5RlxERYztfVvabEokaK5mGTAk0KfKYU+PnAghnSKbE+4Tupep6uNSbl+nRZTx804pk7k3FJuPvAwhnZdAEsZCr4SMqw3gUWSVIRAIOPf0gtAlioOgRDILDwo8rkLqMpw69P3w+ABcxCSCz8MJPOD4gf6Ggb11yxsx7aWFyVwCLFsiiNKFkFNm2AF8oIOwv2YV9BebqaFFwkUFGCC1YyQzWKsgNwsCKf1YmiXBl2SSodBAIwD5neeWU+6CiwgC5KNkwedeeP9+I69+LTVp6H/UUAC54RcZXPRJooB7qdE9kXcCHzkiQXh0guUI2CAVStQoNQ+yKoRFmCCplQCxdYyFBV4IJyow3sk7QrlUrtvBx5+De2uEDdBQ0AiSvUogyed4mB6Y1eozAybWOVuJ1VYvvpCgb6Gsb0hn6W0YvGk2cLAFMG2pXlFujcSl31+GPcA1f3XYHfg2k/ZWzYZ1Mw7YNIsdp51lV/GVrtFxYACgMwxbJtqzBw72FmBVeT9Ok8AKUfgLPJeFNdTFAnEFHdSRUl1ZI0iNaVq7YC44yBKRi7MewAdl21pwzVs1S7MvKzXq20dTBYna5dVa25gclvp45dMP/GH3CcyJQKqG5gNGWGB6n7GPYAxoY4oC7b0OSMvFURawN8Ky0RPA0IYAAWqi4d87xenSKgHu33viv2dFqAdFW3NHQ3SrRSOF36GhwwvEOxSm40bWknoAwVQsaqEkGlQkqANApnRITalXSchomX7dsAVLyAhtsAK93L2o4rpC9XqN3p8WsSo+almcniCwCG0iKrtA3QEiRcWjfy2kMV6A2OBgS9RmsfGBmJOBhGEu+n/ATAsGOBnt2gwlbG8tXedvH+h98XF1vrRTeSuWHy74HuHY5Z6NRFOwag4yxqUCPE4NCzU17ISBJA+XLeZwCvYEbwYOUcg4KNJAUFwPqOUw8lnb5PvgO+C+Gu2XnB52i377ITr4DCdzRsJJhfyDTy9LUwTYB8XU7T5hO8WqmGGapRPBORuPUYxzsarmfpD+nl3KH/cuufXM2ytw6U34W3qohfQUTxOX56hNJwu512OXe4+fz9LV+J17W8GXPP/YS1e/mZqFR5/U7e0CHy80+VWPJQ14/U4X5RMb+RNm9QLh+Sr3l42rhW3awOfLbx/MfTpnm0eoSDeo7RgpxF/Ub5QK5e2dLyWRJGWh60rfXMOf5TkFj899//78X/vfbdvZY3T5sUaFLg51LgV4PzxT+f+BUu7kd/7iO/KN0wasHyG3/o9pPA4hmqUDF3lrPk/Um/cc4UWU6yodJS1uj+xJk/SPUPSP5IBLignHoaP75Z/emOjUV8OFWHUorgL7maDQMzVt5d4ffD6gfW/JIqlMwBsQ/4ZYlFCpRXU4WCWQhGwPrzdYr20YgAFqgj6OXlEreVeoZSXSFJLFz5b4CXDDDsmJBWsJIa4AImREZEaUMdaAgwAlx4j/KzClYcl5+eAAkwH8GwQ+gGaKBgGI38kcr0TgyRtIS5iZ8MDcxcqIdJE365w8oPObCAfAAxIbkg6jM66Ud4tBFcrG3voUJDQLQj4mu4qg0j45CQPjJYgi8/k0lCkfpQpir6koQaMKeNenAuG2X5if30mLzMMDbygRG6QkXjhhXVK+0tCOJ1i+1FG7ESOmGQVL26ggm/dGUfsGNOFbigjSkbmSzqB31cVe+BAe+Due4CEKiypAco+7gfxr6L+uty1IBxujmVvlOAgFGYZFVm9mAENynLoHp9PDcC82ncggPGmRICCItUAi9PMK0aRmug/BIG/BXPqA41DPM6yrhzZX1fYMHYVIqikXgn+23yMaJ12BRwbJA6jZs7IWwHTHA7ZckcaijeTn9GQDRaJ8PbA0N7wlgUMJyGyhpjkfYY20GpgfYUMu7anNgWqb9FH+7y082tNgb2jwCsg73PIrB+AABAAElEQVSqWfNIZIwaL/MtOOgWkFDGkwjcdxIes+CUY6xKW96gkFQIKpZRFQvgB50EUPMCI3htDaqfAbQET2e+1/Yd4OucPHtRbxvBIH0GqcEyAG0S2mu3YnRx+0SDZFWhdgA2r7aIqE1E9XMG0TBAYFKg4CRInrxIAI4UTLHbfiXSfNvxQbH59LvicnOt6Lk4AVhgkE5fODpsm7ErxgjWNzk+RjwTpGGU5YKEY1v6+s76vnYonQDw+v76bjqOHT+qC5km7C7I1/sxb8RwZgTSv+bjMxRZAo70Lvtu6kRBIOEcY5a+F+YdNlbMN6pinSK1OANkqZrlHKAU5QzHDvtG4kaK47yXkHsa89Zfaa5zIAF+qR+xPlAdmwY8zc/NBbAwMJ6G29adoZLqHC/Ox/9Em8pb1uEP2SxHmkVDObS9UXgcc8KtOkjgMmlM5GPxdHWer8e9nAknMa+Uab334ZbmpQ+TQLuyrPRMKi8dpzrEce0w1feDFKld6XLUpzxk5zfFMzKhrHoJXv1TABb/9b/674r/8cn/ZXObW5MCTQr8ART45xPfFv/Vyn9U/Gpg/g94+qcfmUGqrefLP3T7WcDCzPOEmyf3tHeGzVMj+zzhMnM2DtN9/5pHzifvI+/avZzG5z0OYMFHUiNsgYUfNIGG0ojsbjaABavU8bHlA+vKnkBDLz5+YM0nAwsZq04YeYGFq61pn45dnY3Vd+pv+6yz+yyx0IA7AQvVoJLxZAAL6mPqYLDII1ShzJsPdgALGDSBhR/3xIB43WN+1CMYF+tDvZLkJK1uJpuPJEGR0Q1QIQ1kQDgPGtI2Cd9gyjkviR/se0nzoKX0lIEJOgouUh6NfGyHfcRqKu5obaPqGPuHB2HU/Z4I3RvYGmwjuThDN15/3bGV/e1pBgxeD2DhRevIPo2ENAZ80p/jJ+3jJF20nl6HJkoulFZcDY8X12NTRcvkFFKLflaGGfTmDYOl/r8RnsFNcUmbCvtN5swAeNrAhIE5Kk8COF37chuGnLEA8yljfYYdyZmSAJ6JFV6YwAGAgiv9Srm0KzjQkBhGFwsPXMfKQLaG0fUpjHILUhxdnirJGENqofcnY1yswbzrrcm4EBPcl1lWlUiDZoPfaajcDgowmrXRrzX2djwbt8RI0AKDPhhfXZfKMA+RRw/p7ccd8jFv6RyxCmhH0vBP7mwFLZNIOgQznQCbDsprh8l1nFqernDfYtOhV6Nr8tJdbBdljGtUPjlW9ANofE5JU6hrASy+22D1n2eMFyF4UOpwC5ixb0dgXGfwJPUQY3PbfwUtBQXzMOuCpx2Y4u9RDxNonTH20OtCPWm/uMB7VS+BA4fHxopJgIW2KLPQcBwwZUwKpSsCMRn0Y97vDTxevVzfCiDVi/RqGkAyBJjrhC4G79NN8CF1iv5FythxelicvH5WtO1vFgPXZ9GvGtwrBZUmI4yLGdzLjpGPkUxDMkB/U/0ABLqITu8x7WDcJWChAhXjjHaqyqhNRrzbnAej7jvh+CSTABZlGse7L4nvXhRALoJ9F0C0u0gSxca7rSMGF0+0yxBU6EEq7Migg+dKMvQSdYxUI6QglGex1/RlqELZDurZDpjuw9XtDN7B5mfnMOCexSPUYIAlQbr999HNF6V8Q9OMmFI5Z8S8ce+h9F1opLl3+94pedy78uGFuwliPrZjful2/xmalevfmIEaRyn7RJUgARdy+kwP0yTqpHT5zOuN4qxro505pWnc7p83gUWiS/NvkwJNCnyaAp85sCDy9uu35eTJBOkcySzrZJgn2zQ1ltNjmaR2k8PG5yhPzO6r45J2+VzmOabiMk0ACT66ASy4l84/BBZJciHjl4y3k8cUn0sfaZkpDXqVVoTBNoxeMP6xTzYRd4CFrfQ/z1kfSuaDzcef1XwZ7iuBBR/3pJqQmGbTZjWokFbATFTuZgUXARxKMEG5MiQR1yKYjnQsKMkMSnzwS/plCUUGFEEn2pa/YCQr+6ciaNA4GJjqEkxHSUNVp2SiM7iIfAQawfWwN96FYA1VkhPAxQHgYsvVamNd7B7iIYjrMM1G9JVOUsCPafRdLq+kfWKwvGgCUwgA/Nha6bSlqxxHncpr0p6V3itUNSJw3uRMcTu7iMX1MEvoBCHjcX1GdfKMEbJl/gUFrrKbn8z3+TFMP/eNV9CCjcNVVFbGGya9VEPSW4/t2YOxlykLFRfGiga+eg6yT72ufQWZFjiajTJuKF+VJ9XDGGgYWgMUeKYfZl7PSTswt4cwr1J1FmAw1Y2LVPpdNSztMlQ/miSegu5cNxm3r3BRq43EKeBCMKTdwxD3B2mrDK0qWWGzQFts4Ts8P2krckQ/hIvRsm09Mssw5TMw6BlY9AAQwgMSdfZd0AbkJRGzn+Da9ghm/4rzFujVRTmqVi3OTYeKkcy5dB4HYBkI7ndrG8Uq6fWQ5ftywpgw6raDYAbPU3OT48UD3OMKeA5Jb4TzJeoxAIBSrvQd6bUj0RNVi8CCmB0X6xt4+8K4nCjQE3OzxTz2IDP0lZHBrxiTeszSIJ2OCduqI4yan71fL/boM0HPDGVOIGFRunRIvbbpx3UN7WX2kQZ0AixuVl8W3Ue7xUjLZTEITQWfvrvSeIxo4Ro0D+tilt6SSXdBIoJrSlPGoAsBelXqpi/aWRBwjks2XI40DNa9Tx9nN9CxQEAa5zVtK3yXlWRWsyGPpTnPchgj8V4mcBHvts+Rs3W89D0sgYXSixP63X1IZz1XVYp3NLxIUW8Bv/UPSS/HSpC1r+hHKjMLqJifmcXWYjpibhh7IyQplPXJzYqUm/XPc3Xe53vufwnj7wzwwfaRSx+k+aUXoGXK1r9VD3AtF5Zonc9Mk7a0h3yxpfY2UpWJyp2J/HmfnKtk1UGZ7sNdmX08+acQebspsfhwDDSvNCnwSyjwWQOLv/5eYPEmpuI8CYeePDNtngz9kKTJOn1w0rTKlXI2dp/T1j9EHsePpz92PT60pJEB9TipNLlPx37UQ2oBg5YlFkopdDdr1F4/umGXAbNk/n7UZdpDDcp9CSxCYiGTzy9JC6gvFU719iCaF8xhBhZKLa74oEfEXOqhBMDNZ2QgAlSUeab8yxVPVz6jHEGE5SUwURl0+6zX2HtN9Yka8UKlxy+WNLbMDDZy2VHnqDw0DUaFNK6MRhv8k2gt05pUL1If+CGEyJE2AQ2eAUSBLKBhadANM3eIWtQOzKjxLt4hvdjGHa3B34IBs1wLin0qJzFLZT9T79hi7zXTpEv+9VAGPDZ2jhr7Tcqq4nQ7gCrUxExRLD4srkbHilsYUT0xDcNk97HX8PgYkHPCT91/6XMBU3uOdMXV8i6YepACkg3qSL5G0B4FWPTCfKpGtAZTr9cnJSXaIXQBBPu4L8hIbSIOgPnQvm6yUNVIKYmuY3VxqmqYQEGVHHXptSE4Y9XY+vdSnyXAxhwMrYH3BBYy5m4yzxo377Mq/QSVs1f89qgzrcfjEHXEHW0AC8YORYTtwxBtpsnFayQHb3QLDBhh0FBNGFcYyTHqoK3HMkbdQwAMpSS639XTkuDCFfRXMPWPt3eLf7u6VhyiVnSFRypeKFzIEm9jZLSYWphLEaphmJWSzFMPmdzvABZvsG+w/gK4A87PKV+QvrIwi43EdDGHStFbjP81vN4hkvgSqlFTPK8E6PHuHkbcp8UOebXCIF8RP+V09X3RDsM+RL9Ozc9HwMJpwIi06aLeSitWeUaJ1BVMuDEdnvLMJjE/LpA0zc/iIhYJywJRuI/oy/cAjhfU0T5Qra/9jJgl714X/Sf7xUQ7QRGph6pZN8wdGnCPY6cxPzuDcXs/8wtSIDwuudrvqr+A0j4UWAgqDHznAoFD9TgYfCVM0E2JAPY//YND7Bmb0N33mGES80L0TZzZ62w8n98131Vv+WYrjUzvDW8Dc5cLJCGpQDohmPCneqI2Fs5xIZ11z0+JhXOektqYMwUVlMNEUnQiTRpAbWyeGB1zSCsmxyYjKrsgyXnjl271Obv+fL6e9/V876TzhsVGBYMc8adek5w+JbEhd7d836sfK69KTQY+bZr6M2Wp3qkn5TjVIqWt38vHeZ8eS+lyzdO9T9anrItP3q1LyutPAVj8r2/+TfG77Vepwc2/TQo0KfCLKbDUN1H84+EltCAGf/GzP+cBv3N+v/7Q7UdVof5KYPHqTZl3Y+L0SMbPibExOab7TqsxqXrPdLWPVlxPX4nI0/NIH2fpubwi770AFe6DSU7SBxliP5xWXGDharPRtiPiNsxZRN8OqQWqUNw3vSvmrsrptSmrQakSldWglGRU6gykye3K9U/2A6raWB4fcO0skFhc+jGHOYlYFrRB24gKWJB/JbWAKQkPOgKLUv1JF5+Ch0qKUQKKClgEo5goLSGD3lArrkA0KWe7Mk1jLx2jT0paSl9+aYsn4jCDjlATg1GLNAEsTJsYm2CnlGBEm10xRRKEyowMjfEBVte3i7cbOwRh248Vexkwy47yrSXlplVf8sv1tS5RJ+sfh6T0iI1nBRaRxPMyQdy1T3BBW+DF5mZ+pbiemilahkfCc5B2C6oLme5UcAH3HUw9q/hngB69B6nK1EWMgzaYe/tWdZ8Jrhl125XrLZhkXcqqhpQlHjLig6xAO2bcQk+dY42Yx2G0lXio6rMNeNFO4chxCFNnPWRELxBnqCrVR90mkHrMU/YMQGWY+kacBRjGMyQaqhh1w+Vq7/ESScArJELvsWeRhEaIHiNmRjeryhpQm/kM7dAdrp6rXrv6D3O9CRPtyFDdq4XBasC/lfFhAuuNR/1lb7sYT4IobSVUvXoGKPk9ak2/fb1aHAMOrmCmBRYdrGr3ohY0SHRm3w/VqGZg8JcwepcRfox9wyqgxCjnhMcr9tc3MUy5CHe4v36wXCwiPTBK+Eviobygfq+o3zCAbgJ1o1na8ha1nXUYclWxOnhvr0l39G6NuqMmZlRoGN9FjLinUccah2a+mwI4Xc4KLM55n48Yiz8AiLZpNyKM4pvlueIB5c4P9eG1iqjqtO07wO+m4J/x2nlyWHS9f1MMnB4QV4O4Hv2ot9E3qtD164IVFax5jJl7BJ70y94eYAeph8BCcG+8Cm0rBA/d2IIEsGAchcSA9DozUFKhIXSACtqv9MKxk20rsn1FvHu0x3cu5jVUnSLejNfo8yRFFNQQQwM6HaL2pYG26k/G0wiJBRIK41gIOAJMxBzHQkCZr2UIQmP+dPDS1i5AzxAetFaWV4o53p/xUSKLIwm0Xh/M0T5TbvleYx7Jdxr7nCauWIfy1v1ncrrqOu1tbJ7kJ9O3wzkhrpFn3mqHVb2r/HKi2j5yKB+KXZRJ/smTRJUylcDfqqhIyP00JzVumMBfvm8WHpunx6lvq2ziUgPQWNdIlhJ7N8ZCps2fArCIRjf/NCnQpMAfLQV+HFh890PxBGBxdyIsmduYSOsTaqQKQlWTKpNnvlpN/rUvg9fyNB3Hnt/7xceXa3Ujbj+6ASpqwCLFsQBgoDagxOICxucKoBEr8wIL6hI2FTDwobYA0yS4EJUJABLISOoKESDPupeTv3vYYxhJAE12Nyu44EMvsHCV0IYkYAGAEUgEaCjBhcdxzevpXgIWHsOAcC2kFzAxIb1gH/YVJY1dNU91SuPQb2JFpwAEsORy71Hf9IGTrsFoBIVz+mDd07OkvxtIz0wFEvSBecrmcyybbDr956efK6gw4ax2v1nbLN6i624gPW0PZMrdcq8nWw5yMk//uY/8PYvqpfFRfuRlyC3Sn1sqn/Fm+zHkLvpHisvp+eJqarboGJ/AlgF7BlaGdU2qDQ0KRMUxdV/XrgHpwykr6Reo+LSRpgeD4F4Yx376YYjfJMy+nqAcn2v04zor3xoVK01opQK6aB2M1X4Yf/rPKp3RPqUW2gwY1M7Vf71IbTHeNLpWBUfbCQ1mISVeozpwL9sdNgOCigmYTaUHelfS3axgBJ4v+k13qm8wZH6DxOLt5q6DiTgRXcUwzHao8cF1KplZxlB5iHy1cXhJPIe3eEjaBiDERrvaYHCXkBgYVO/XM+PhmSm6hQboCreXNusFyXgZAov/99Xb4vDd+7B14IUpugAVAxOTxSCMuv1omRqxz+NRy177AYb9ACb3gjGhbcrmu/WihbZPc//XD5eLJYEFdNuG7kotngCSjLotcz6MFGIfWu/x2+XXyfM3qvFsboWkxWjVc4sLxQNsHabpL/tJUKEHKtWhfA+l0wmA5PHb98UuwKINWv5qZRGbEAK/IRVR/eodwOcpgOU9K/tGq27D5WwnAfL6zw6KkbbrABbdtEsbFiOzT03gghVgoWOHc97rfeJEnAI0lcRlSUVEuQdcqA7lOwpHzrvPe0F9HKcCj6wqZVuTZNL5hJ+LBKT3Hc2LJL4bSl6cP3RpG9djlDHu+ec7d0L9Nc7eA/SFiic0E1AcAzRUhzpnvPpu+YQAPUCCw4lfgApuhXc0+r0fda9x7JMerqwUU+NTxRCSFW28lF7l99UxlN45jz7cvJfnxDt36ZeY56Mu1iZtH+RVfQzuPF3OW3evxVRmrmX7vFtm30j4qfwaKVK94tlUq/T1sh21RN7Pp9VBvmC5+aL7fNy4n2Y8MjRP0qbZrZaSaxXdzKtWeM47328Cizpdm8dNCjQp8DlS4KeBxUuAhXNmTIYxc0Y743NUmyC9mCdH93em4HJizpNoJpTnkc79j/wyuBBQ5J9xLKx8qETxcb4AUESAvFJ6obtOP9iml0EIzy2uPvKR1yNMBheqvSR1JZh/7qk3LpNatSU1jHomYHGhjYVeoTKwcCUVRlLe2A+7gCADiwxa8rkAImwuYExaBRMBMriGXrsf+dDD9p758JOhjD31cp/rJNGy1CFc0ZbMTbpPz9gvmebUPz51MnEAowaduR60kQlOqhMQKu5L76ymkb7tnHuNdC3mgYqUUov3MLVv1taL91t7xW5Wi4pyHSdprOiJKuUXtSjzL/u7SuUokBmqqs2Z6d0lhqmVgGS3vQPF+chkcT0JE4iO+AzB0WTcZUAFArcw6UcwbE9RhTFy+CGM+jWMmIbePa6AlyvhU9hBjLLqL5PfRsetI9lYY7ysnWBAjdTAlWjBhS5Ip8rYE6rOKdlwxXyJa3Pk0QeDbjC8Ta6/g5F+re0BEpILGG09TBnUbg7d/0es1BvATQlJH/nYMvMRXOjpSPuQE8aykoC32K+8UmJBfxvZuo/nHV8y4RO08yvyGgakGG/jKXYGb1i530ctSIaphfStuImbn5oovpqeKP4JEgvVsxz/Z+QvnVS70lvSc4EF9hX/+g0M+uq74hxbByUW/ajJjKHONDk5Gu+XweOk0wTetPS1ZVRweNGw8XAsrL7mWdo+ifH1nz1cKlYANBP0iWkFWs+QWLyCJsauuKaPXADQFkAaq1bWoiQMAHIDDYcRvy4sLRRfjg0H3Y2uLdBTIrTLz3dThllg9vztO4DFXtFGnb9cWizmUYUap18Et/vk/Q6bg3cw5AaTu93fLTq38AiFStRg6xVG5kkFDkU33A9TFrErZohE7WuTVI0O2GvTgV2WYwcVqG4Am+DCsRRujKUrfey7FmqW8S67kOB7nRYsfP9DYsHedL57LnQ4J7nooUtsVZg0yNZGTDsZkjnoI50xLPZpwx7tNAChEoozxvMhgEo7EIGFr5lzWzgncN7w2LLMxswoW1fFY4Cn2bn5YmVxGZe6uJlF8tKYU6LQqJ+PfWyz7ml+SW32vNosx/vlhXznbpoqtc27u/m8W87AwztpnC8iBX/KecE0kb58qLqf07mPt6K6EHmUGae6pWejb3KqRkH5Srm3gFTInXZx1W+hW+7jqGNJk7jhnzLfeln1fMzBe38KxtsVTZoHTQo0KfBHSYGfBhalxMJJr5w+q++C1/KWJ0bP65On506gMS3fn7S9zi+tupXpyjTxjPdgqGL1jX2ACpjb9GEuVaFgWLStqIy3OVYd6owPtmoNqk3JFPsRbccbkMBCJjGABYyWoMK4FkltKTH06eNcY+RtA//0CnWhtySkFrqbNZaFrk4DWHA/JBaWI2NR/rLkwtXQSmqB1CQBi8SIdIR6FMBCpoT6ZcYkgE4wC3DM0h8iB80TMRNtSlBgG3P/SHO3REPpl4BBNhL1enzo2AeAkMb8PA56Q7N8nlY0U34Q0lz5j8QIYHXAyukOTM87GNR3qEWtoRZ1AlN9pc4KWxoHlAEHFuoe7GXGct+mrqZcc426mHtiivybciEfaACXjSsm4jUMjhU36oePTxbTU5MwvL24bMWVKgCRDigOGA+/Z9VfFa1DmOBbbAekZx+qOEvo4q+Mj0QEbt22DsBgd+hJCOZ0AynHGxj7J2tbxS6M/QVMWz+M+goM9hJGyQPYLBjVeo900zBqs6ziD1KmHos2ubYKiHmG9GafFX0NoTuo08LEeLE0NVY8HAMMRB0BM0gbtAdRbcrYDKr62E49RT1jlf0pjPhzwIUxJLoBLz2UewzwaaUcDcC/hukepuz3rKg/fr9RvEaNSBuFW4A2YoGiFReiw9hWLE6MFX82Nxk2FkpWzmmjNgsadasG9IK2fk9Zv0WVaZPnz5BAtdLmsfl5Vu+nAQhj8e4ofREEaUQvONnmuQFoFjYbgJQnL16zqn4UdiC//mKl+Aqjb9WmLnlmF2bYKOLPAHp6vNoCUKg2qG0DLw92DjDepLtGXekWCZMqVHOLs8Uj6j8NkBmQvtQ7RRTHSBnQKPMtGNwG0B5Q53P6dxwJyzBqPgNIcwahWRv1U6KyurdbHOxsF5fbm0X77mbRdXZU9GO8PdzTEWphqodNjCHpQO1rmtV85wtduB7jack5RlDQCb26OktgAcBwMcAXUbBmvzmvxLuN5NG9CwcuVvissV2cE+LFJXWoBlIv8xZcqV54oboW80ioOnnN+4wLA3I6vyi1UB3qCOlbBBSEBkbkPqEflFqEpDLetfS++c75s16OIUHFEDYkc4CKxYUlIm9PIbkjSCT1jLmknMLTu0hGtS3e39qNSM/9mD/KdNKgzCL2+Z31duMNLhNH2pw6XYu8qK9byp8c6pnEHXIqr1V52sb0VJST6dDIJ25+5E/KKOdXFp2K5GK+nh/M99P5BxXLySoaVCnMq7rbOKi3Pu7XC6SwpsSiQavmUZMCTQp8nhT4+cCC9qWJP02NTor5Q2PT4yoTY957rZpYy8kzAwjvueUPlExsPs/X3PvLoCIYXj+4fJR1y2jFw50i+wwqIpYFK3+qDRh5+DyABQwXTIDGkxEZW2YBRjNJLNC9lxkAWPiLj7EfZNrhz/blNlKT+Nhf4BdfYBGgAtUE3UCGKhSt9RmlHfdBRXvk3wAWSieSqkSSWiRmpFzpFJCU9hZpxTMxJ6E/b338xz7oBH2SG16YM2iTPq61lUOZHxiUayQMYewd0oaqV+yAyCekEYIK6Q19Q0WKvWUILFKnQkauUUj8jIOhROgUxmgDd6GvZHJlymEiz2C0BXTSM+hnvmX9ZMgS0IkB4J8YJ/5NsMW2WWSqGxnECnFBsLwbDLhPBRZDo0UnNhbGHZiBATUA2yhMt16adMH6N6gSvd/E2xGMJ+GJQxI0iLTioavhrKjPorYjU6KnJA3A9aKzA1P3lrr/7u1a8R5JwDHSB707fbUwU6zAaPcjfXgN87sDsyywmIO5VwKxz3NG5H4Dk/6SVXTLNAZFL5KDLxZmiy/mpoql4X5sJLDvgZEd4llpcc64FVwY5K4NGhvQ7Skr0Y/J5wl2FreOV0CI6lvHMOWtlDPDWP1qAoNsAEJE9kaV6S0elc5hshn0cLkws7gQ7cXGQtepy0gtBlHZMkaH0qkHtHsSoOJ78AwpwWPK+7eUt4sq0jlgrAVgsYAR84P5meJXqFEJgIzbsY4RtoH3lB6cMt5HqPMgdeuDBs+fvyr2AHDdlP3NFw+LL20zQMp+DrUkQNYTmP9XMMZGFlcqwguEpILgfzLe1OtGYAEA6UEqME75y4CiaVW+qPsFwFn7ivfUY5uf7mvBDcXp9nZxwrg7AQT2Ag56GQ+99LGucjVYNzje6g5qehvrxdn6WtG+v1V0nR8VRBgpBrpx2QvTrb2JalAzU0i/ABcy+qcw68fEjHCe8T3UrkLbCZnxHmI+KFV0dMZc5vvOuVKNpOLIe825adJ7o+8uaM87o8OE7FbWd9b5KwEL1Jso7wgJ4AFSCPtaqY7esFr4qX6oZELHCZe8V6FeCP2MuH2C9MJ31y3mqjiIk5iDBKb9qJfNoOa1MI/R9sxcuJjtLG0rTO5WmxHShfJvNc9wHu9xeT3mnnzsvfL4Y7v0dqc7zl31rZ5PvOf1m9Vxqp1zQrVFNo3ccj4xX9Enja3+UO1qebme1GcbZeTnUl737THMKZeSU+Zr1bn5NYqMo/xMvlzdbxRc/Gf/8X+Sb/+97x3juV/zvl5o7v/87Q2bIr6rY9gkNbcmBZoUaFLgUxT4aWChKhRPOwk70aTJ5u7HKN2PBB+U4+TpfbeYvNNhdV5dLyfXNMEnptKpOQELjviAppgWAgtX/GqqUHx8k4pFigGgZ6gELPSUklbfrXcAC6QWGnDneBZ6/wkPUTBtYddQtrEOLmyAdSEnVhQpSzUoflcCC5gtveX4VbIM1ZdkSLLUIoGWElRQRlaVCEYkAIb3UIWiTj6jj/xKhUKQE79yBdLjcgVUiX7QB7pksOUHIH8d4z6MSVK9AFgALhITkvrQukLcoG8wPn4I/Zkf9PQ5rwcHTlmCAduZJA9lXAi4c89VSVEl6iX69us7h8U+jLYxITKwiDFjWdGH1imBGMuTwU/G8ZThsb/yj3ufbQl3s6jmjE4Wp8MTxLUYKNrxcjMK02TMgyWYyVnAxSXpjZHwWwzL17D/OMbQ+BZVEu1ZhrBV+PLhcvEVkZ1nXVGnPnqLkrmU6dctrLr5f41q0GvG/D5SGCUDXy4vFsuLs0UnajYCC5ncWRj7JRh0jai1zdBI+TVqYaus3l8gdQBOFYOAkT97sFR8NT9dzGIvoKta2zPiijqkd1zqGlVXs538bPsLGHCZ/R+U/OjdinqFQTHjrB3aT8KwfoEkoZ/8VL16Srlv8eh0ubWlDg/qV6juoE6kB6B+wIUB5zqgXwd910l7v50YDo9L5qu04jHg6RWg4ZD9BYx/K7R7NDtZfIvU4Z+wd9V8C8nIU5j356i7rdOv2jyoiqX9SR/gYv316+IQMNVG+5YfPSweLc4XX6KWJG0FTtqe/EC7XsIY64nqFvoJKnhhaR99C5NMIIbilnI6ABaDqCTp4cko3MOAMMegXqF2ATRKaYzC7Zxwi4rTmcCCfuoemyi6kVj00sfj0Nfo5sbweLe1WWytvSuO368WbXtbRcfZMa6JAX1424rgeQCkmUnsK3A1OwuwMCCdwOLg6CAYf99jVaAGsUfoR9Wrl3gqIYmAfm4uVriIkN9b72lTEaCC+w7jNOYTWDfmTcSqoE16t1JCosTi6ABwt7tbbOMhaxfAd6IklHfFxY92wK/zzwn9pBTtCnDhexuLKDzv++qrDCVNFu+SF4ybM4gh/jhqXg9WHgSoGAeACYKMHJ5S84DP+A+axpyQLsXfeG8dtGz5nunq28eeq9//2PH9PHKaXIbnn0pzP+2n0jWu1+srhdw+3qb0TD1NUDQ9wjP5jvTNx+aUy7L+1XXHaPmku8adOIk7+bnyJNL/501gEeRo/mlSoEmBz5cCPw9YMFvmj5dNzVOsE2OeTN3XJ9I6SfJkm55Nd+oTclx3Iq7/yC2fByNRMqUy0dlwO+wrXPnjp8tZ/b0rrdB4231Exia9H3cndpn+dj6q2b7CfdhbZOYAhqFi5P1IxC99VP1M3MCtJwCje0fsLGDmrilHIlLZKENAkuJTJIajAhYCBxiZABYyDIKKYByM6NsV4CKvdsqgBANDXqk+1sv6JAmA9arTLKs4xSoq9ShvBji4oW6VEXt5L/dZlUfuOe/zC1AB4xNMEeeChwB15JVBiLyVbRUO6EN/E6bo3cZmsWoQPVa/93Ezan1MEfWNrBO4sA/NO8YPKWIskC4BjNQ2n/E+SKtoYaX4FvWny/GZ4mh4rLhu7w4pxgCMp5Guv8Cd6RfsXfV/zcr3v36/WawHsIDhRmIh4zc8OlJ8/QWML8DCSM0yqKoI6bXJVf0zynJl/m8xCn767FWxjWH0GIDlEcBidm6muKL/XsuEwxQ/xPvQI1b/NaJ+QXk/rG8BqtaKnVdvY/XdPh5k5f3r5QUY9aliCiBwACN9BoM8AEOu6lY3/XnE6l8GsHRwxHfQ9uEJ6lR7ABYjS3fDJCMWKnqo7xRAZpn69wCmVC3S3ew6QOYccNHOffN2NXGAevfAYDuutEeI+Abc/wo1Kr0tXVPWXyPVeYIdyi7XL1j5BonD8F+Ed6WvZ7DP4GcQuQPq+BKw87v32NOwP6IdPdafsdzJGDiFab+g71sAScMrD4tZvDpJY8GXHrSUMDw2XgbPPqGuNzDHtzLHjKUWxlWLZXNfFcVe3KGOzM+HLcwokiglD8422p9s8twa4EejdV37oh9UXAJoL6BVPzYhA+MTeJUaCfWpYeikQf8+qnr7O5vFPuDihqjbN9hatKAO1dVJNHLAhe6E51GnWyDK6Dx5aEhtsLmd3Z0KWPTqphUA29/Pjyjw8a4ynnw/7bN4P5VSeM54EtC7pbnLxRAlie6RKvouAvAFF0pTtefQy5pOBg4AwJa7BUg75Nz5zGjrqtQ5h8QCAc8YkPAKT1iXjEMlhpbj3JDswnyHqAP16katTFAxBz2Xl5fDC1Qf9c/zCqlSRa1rfv+rK+kgzzOexbt4736+fj9dPs/7eIx6lqS5k8vHruX0dxJyYn6fqkdO27ifcr5Th5zozj7nmWuS6ZLO7+fnox/k6Tx177rP5RzjZknvRPd8z+WEtJmFJTclFpkizX2TAk0KfK4U+GlgUbOxqDeynEvjY+r6XZ5s706m6Yk8VXuW77tvTNqN45iQnZT5hcpMPo6Ps6vzrPaxol4HFRlYaP/gSvk5EgvBhQyVDXSF3DpqZKm/fe0qsipU5XaWa973Iy2zYHtk+vgfx34KYBFg1GDEyD8MuAUWMF73gYWrljKz/jpgWpMOdgIVgolstB17VhATs5JUobKdRnhCsj6AiQwukr1DVEhSxpYZ/ZA0sBLqamZ5o8bQwMhzr05bTiR6ulb2StVPJdNPBgEOIhowdJQx0gDcXsy0MY8LVMM05t4mRsF7mNz3rHAbofv0jNVl8spRwVOR6ppn1S3qazUix7SPz6sVkfBu0LIVCcUNwfEupuaKo8ERgs+1w5QSM4K+esjK9jeoB32DUbPM+zMY2H+zul5s4EL1DHWZFvTl22GyRrF3+OLRg7CXGILp3oVR7YEhdGV7AmnHCXYh7wEJvwNYPHvxhmCABFPDY9DiwhyGzBPFOfVZQ1VKFaavRlGr4tfHqrAM8/cYUL/kudP3a9CfiM9INybn5lCFYvUd+4xu+vy9alSs3uuNaA6gMQFTq6tb6WiXnUOcVcDCS+pgLIsdJBLnjNs2ymiFAe2j3CmemUOVq4P8BRYCoR1WuA0y142EQzWgxYXZYgbQoxqXpi7H0GSfd2KP1X5jUYzA8N9Sh7/dwh0sZVnuuZIEGPxWgM8sdDLq9re4q1Ui4/u2wSr+7wBqL+jXXaQWMqdKXTpgkG8xim5hhd+o3X0rXxQTC/PYloyFREfaCk6eABx+QLryHdKFM5jmG+p+y+p9K6D8lrxbYKq7Yd5Hp6eKmZXlMJq3/kqTNHZXLWiH/nqN5EMJiuACww6AAipjSEKGUfMZASAMAUxaqa8G6/200fF7csy4xMbidP09Eb43iltcyXa1GJU7RVZfIJjfIsBioQQWBwDR7e2tYOxtpxIL7XMGABZ9/Lox4PZ9DgY95gvGZwksAvwHeyiA5t0KQF6CC+jseySocHEiomfTJg2ynacEGAnU7PEuYT/h3MIzqvdFlrRL97g30C0ASgALAFq8i0hOaG8GOXqlGkIVb0bAtLjIfrYYGBgCUOluOWbrks3l8cjBN/DDrZrTqcfP3XJKny3fYKqYruZ7Oa/75/l6Tl+d/4KDxjfF3NP8/eHjVc2qWz5XXc0HXIt/VUXLNLTtg802lm2O5PHs/VQZVuTrpkyZp8ebNhaZMs19kwJNCny+FPjFwCJNmqnBHgcDXu4rMtybePNk73ScptG0z9d97v6x53lVO+/DqBGm1EprVxHggg9t2FhwHgHyYFZcDTSuRQAL7rvaZz1VhRJYCCrCaJuV5ZAoCAJgDsIjlKAiGPoPgQXKQQEsLuvAgvJUccjGgzKKKSZGBhYwkCGpENCo2pCARdhhlCpQyT0l16mbH/5g2oNpScyCgMc6xcfaj5f09SfdAkgk0BB2FjIgMPN+VEsqS9wACNLUNNUqKnQxrR9P2Y2cpzKEKIs8BC4BBGB+QkUKZinl6wP2JvmRjy5oj5BcbAEu3qE242+HoHO6/RXYmaeb1VWSUUktZMA49y672NylNrKHJq0wdLfTc+Fq9nRwFHUcvAPBBHfCzD5CWvEt6kHfYpS9A5D5nhXfv8JOYhvbg0tWgFtQWevG888I6i5KH8ZZCe+AljKoSg5UU5pDremAPnyNEfJ3AIRXSB92kQYMok40jYelYcCLaj1KYVqg17fTo8U819phfB9jD/H969XiNQbQ1zD4AsmRkaFiaWm+eMSz4zClukt9St4aeEv/ZRj8FdSxZgECjkm9Qmnj8RqmXanIOyQJ2jao8tfCuIS4RS90N0r3OIbNrTC7qgXt8jumnpaLaXsxQ51+9WilWNFYnDQabO8DLLZg5FdJp0eoHupnnk+ozxrXb2D8T6HZJfr9jD4A2EQxTRlLg4AjmXvun8AMf4c9xwuAwXtAj0yzNh+t9Hnb3mbRAY3bYbiHH31djM7NF1N4stK4fQL6aifxCimAwOJ3SIH2AQMG1OMlRVoBqKBc3HcVfayuTyHteIBUqRdA4vvoNo2NSUhOYKSf8uwbwMU6jPcVfduq/Qy0mVhcKsYBFmPQPQI20ld6nNIW5ZyFgHXS7W+uFxcbgKCt9aKdaPJ9rUiPMOJeBMwsAirm2SuxOEAtaRtAqvMHR2Ubc4UqUKpCDfDrQYJhFHTrF++qIIsx6lgQbMjcu4XUj/7zXXO8+9O9bATzpJyYm6Cr74HlGJhPWwrBxQHj5BBAfET/nAIwrIuSR+1kHH9JesH7SD5ulhnqk8wzOVDfZCmtmKM/hkfGIrifCx0JWKTZIR72Dy9cvIHlnFK9iFWCDw/i/fTR/NKWSdK760kJLMr75atdpooiq+N8Vs8qZq/yQi6r9sAHh3m280a9TrlJ91oc80s9nTSwHpaVZqPIKF0u/6YdCSPTBFru5pHvmVetHuadpmsuclAe10qK9P5pGm9XpGgeNCnQpMBnSoGfBBZVgLxyMk1zfczC0WQn4lgDi/vOuWnWdMI1VX1OzTSK6/WvSL5hep8rf16OD7Mf0/hAJyY32VgkYGED0sc6gYnw9y6w4FeXWFgtmbhsTxEuZlFHqiQWMArVqh/MfUgveCg1B8aAulyrCgUDcIlk5EpVKFYbVU3wA69xppsShiytiH1ILO5JK2BElFZkiUUce436pdVPAUViUgQZMg7S9T64SOtnUDNoSR1lZGCqVLtIgCF9w1xJdbOGmZ7B2MvUQFefj36jjMiT5KFeQR28EjYRMkHQOqldlflbjgbhPC/oUJJzCKO4gXrKqpILfgeu8mtYDEBxs6q5HvGcdeCit5WqxIAxYVSZNksXGLu2mfmidX6xuBmfLi6wk7mAYb6F7hokf41U4CESiw3Ueb6jzL8FWOxvbhY3SFFk+EZglicBFvPEOhDYnJBOO5AFmPtHY8ROQLVoi2svtDt4i7QDRl3QOsHK/Sh66o4V3ddewvgNwJz/ZmEaw+We4gTafQej/fzNu2IdIKNkYZBV8xkAxZcYTq+g86+EQPuN77GbMKbD5v5hMYPE5BESj1+hbtQDMyjweANIeIbLXoHFNnWznySBBs7XqPi1cj7ASvQwYERQGiv3SBn0DkSgg2IA2s1R9m+++qJ4CMjSPuEIcK1XJSUla+R5Th72r2NK962qGGlevI3q2CV56VFpEInFMDQ1Tof11DOT8Vt+AHw8A1i8hL4aEd/S1nYkFe1720U393tYIV/41a+LWVbIJwcGApAYRFB6bcMcCwh+D33fAzD0JnaNpKIVaUMAC4670f+fmJsrlh+sUA9VA1H/YVwI+vT4pYH9U6Qzz/m9Iq9zJEo3AgvoNoFHp0mAxTT9tY9k6Jy2OdBGUXkTHqwBePdQnbo53Cu6DwB/1hmVqJ6bC0CFwGIKSdAk7zPjl3TaOhxRpzPGtGNVw22jcg+x6m/8h176XhXKLE1sI7hfdrTgexTvGH2aAH4JKqD1kcbZ1EVphHY/BtFzUcFBrxRWt7OHgKd92ngAbTSUP2OOMTCl8Vh8/0JCST+6mHFNO2NeUFrB/NVOn2lbMzo+hmevOYy253ClO1X08f4IhsL+g9Kc1OL1ciJ2i5N06LsYc0E6jXezPLxzPV8zfd7i2XKuMY+4Vbsf6UheMe75wU/s63nX61Qlp4xUiLtGvW1dfjb2ZfvqedTTV/l5cL++1DYuf+xejW7pdqMO6aHy2cgzV8I71DCqXtLO+15gawKLIEPzT5MCTQp8xhT4UWDx198/KZ5gyOqnwMlQpiRt5YToVSbEWGEvj8sEMbGbKqe8/2Se+HN69/lafZ8Z4djDPMkQZ+NtJRU2wH3EsPBDXEosLthnYKE6h+WHxIIPenILmSQIATRgXAQBFbBwFTK3q6y4LBlsNUa/loeec2m8rSrUNeUmBr2xepilFO5DFUopCb8KRCi9ANi04aYyXRPY1H8l0AhQAXSjPtI6fny8Gx9Jjmmb6kaVHrc0ganU+4y9pl2JoMS+kI5BQ9IEoxLMZgME2scBKqhLgBvL5LnE+SeAF1IMn48VV3TlvQ1zafmqeLjav47k4jWuTDdgpg84F5yEdIVKZGNt+znAEHVIqiOkiQHjn7KtMkwwRu0ExutaWC46YZZu2jpRJ8HtJqDOVX/VoeZgIjXc/oHyvkcV6khpBUxZHwzhNKpM/iaVHsCMbbNivgPgWcRb06PS7uA9jPczmP9XqPwYHE2D6mU8Og2xYm6l3sBQ6wFMt7HfYpB9CV3WiXvxGGDxDsP1Q9rbx/iYw7ZiBePnR6yeT6Oy1AdjrZvUJ6R7DGP+HLe8w4y1FSQtf744U/TCDMrgPwe4fIf04xV77T+6SKNxtFIGPSNdkIeb49XeMvL1MT+9k8mc93Hf9n354AEB5jB8pp6qbR3zOyA/pTkanhvATy9mApZQxYL2a3j0kkmdB5CMo8Y0iFSHEHARe8PyWwHUr1Bte0kbnwMwDqDPKcy9NgttMOmdvJNdMNyLX3+TVG8AGar8OC4iEB3j9ZA6vAU0GTRvE2BxBCC6pt7hEQrA1o5Uahj7kFno1wco8l28oO7jMsu0WQnBOxhsvWGtwvSfk8cVwBFOHSBCoERA3AQM9RnlGKxR4Dk02Ie+VntxSBuPGZdXuJJtOcXGAteznQCMvpODYg41ukWA4ALj44Z3RmCxicRij/xP6G/VkVSH6kctbhiPW6PYcQwhuejlmu9HSCt8b8t3zLfFcZ5VFKVBSCugewYWSiCUhAgsOvE6ZX8GsKAvj6DrEaBC+mjErdc1x+MFba4H5HMxQ8cR5u1cINjs5l0YgoZjGKJPogY1hWvZiVGM/XGT63jSDsOtmjviXfNCXI4/vpPxxpfX8lycU1TPcuH+vUhT5VVlkB/lAf/nQhuX/y6Oot7QwS36gHa43S8v1z/XPZ+nxB+v2508qnzjifhjsTk/L1R55rSxJ1FJknRQK8v7Zd2bwKJB1+ZRkwJNCnyeFPhpYPHq7Z1J02ZWE2d57HwpuMiTo5Nsnmhr02c8l6+bTz7Oe6+55XP3WXXnDrCAMZJxSaBCJjqpQQku7gALGAzvyUhbPSUCYWPhR11GH+beOAKhFiVzwC88Q1XAIrXVNggsYJ1Rw1EVC4YORuWaj34y4M7qPokO+rFvg6EJiQUr7gFkLA9Gwmvh+UmJRekVSmARkbgpN54JRkpmSiPRpAZlA6R7pv2dY+pnHwhuQocbBknDUFU71PO2zZEPaVz5VbogGJM5CbpyXS80ttFyBGDWMRgnjnWHKtDyH6XQP9q5JEDnXnbFhUrdMkrrI1a/t1lNfoHb1rcw6hswo8b7uAEMWESZOPrZOodExD2/6Hv6PcqiTGN7tPb1F10Aix6AxeD8PJG0e1hFx/iZlfhQZ4KBH0UysAnz/IwV32cwyieU305dxmA2FyZGWc0n/gM2CjLjWzCja0gHjNS8hNrOCODhNWpbej/aZEW9nXrovenrhZlwD2ubXgIK9DTWS79+iV6+3o5eIH3QNeweK9y3rEQbdO+rlaXiy0VW3lGxGnScQT8lBUbVfoYk5/cYQRuYbR5pyb+3BBMN43yEBOAHJS3cewkDj3wHJpaAfvwmyMeI3wEQoKFg4QImVa9oRqGGeEUL17p5J4ZYSVevfgEGU/sGIciJ/cE943vsKKnRdoP9RB/SCeJ4tPMeraLmZEC/bxZnA1z00/YL0qdI3TCseCbaAHitwvS+hL7r7HfYHwPeilA3u8AWpKuYXnpQzM/NhWRIj1cCA0eMLm47yMN6vGNsrAEotmCcT1H10cbgmva30M99BG0bIUJ0N6DI7YQ8ummjMUeUECCbKQ4436fOqghdokp1Q70ENgMw6MM8eyMggTYn5G0cEt31dtgeRvcpY/6E96J9H09SeInqPdgBmBIZHXC5iDSrlUWDQ6Qw63gT2zb6Nn0s495FfZRQDSKJmcD71DAgagAphu+Iqo3OHzF3wLirapTfk2gEfwQajiElDwbuNJinQD+eYa7wnfC9df7ShazRtc8AEko3QsIhkOLaRcxnAENoIMgVWPiOO/dq5D0AuBpCQjGEdK5fCRAqUJPUdQQQJLDQWUGeP6Juvot2ULnleTfVn4vcy9dymtj7jM+6810l32qrHZYZxK2P5lM99OHBnTy5/VPPx33bR1r/5vR3QEHc80/jPmd3Nhdiot5lG2le2qJd6STnnW7kBGW6evJ8iSQupqStzDif5n1ZUNN4OxOkuW9SoEmBz5UCPwos/uqOxMLJPTUzM7V58neqzNdMUp9483FOmwmVr7vPv3zPvdf8qAfjy4c0ztn7gbbSSi0ijgUf5HC9CDPtB1ujbdWg9AgVAfJIKyPtxyZiTMAMdMLMZalFSCwAFKrMhMQA5j50lv1I1SQDfqBQ2oGhlaFWzYoVUD72qkP5gVf/2S2t9pegwo+5P5jRUHtQN5vz5FIWZoTjBCpKtShUKipg4b2SuZcJSWoX1AnGxTIyzaPQkoZQLegTEhzqZfvVzc6qatEHdFBINqBbSAsACUoLDGgWBt5kGDYdrm5KAwrKZcsy5b5SGhIxNOgT6+KqtuXYX4K7fZjHVxhQvwZcvNvYwoj2NFbKKS7yjYzNnDoLLOxXf+ZPRbhN2QI9dPdb8QTVOTtfdM8uFEMEQ+sGWNhHZ3jH0auTevwDMLa7MJRvAQ2vAQGqCLWR1Tz2FytID+ZgtjUG1jPTLgzjeyQWenUaB5D0wpQ+RVrxA56SVFPpJz/jKHwBINEzkUBqVfUUGGBpP4d61CbteYkUQo9JJ3gnaofJHcXT0COAxaOFuWIR9aoBwSpNVNpgkLdVytTjUwfXlLT8OR6jBG075KUB+O/erIXE4gYmdhqJxgzqXbNIVbT/MNjcHiv7x/xO/QEONMJ2vHQBEAagVw/jqwfPPyO2izZok6GnLGNtbPD8FuUfIjU4p7wpPFsZi6KNd0OJkoz5b4icbbtlQgUGuuRtp36DALIz6rDPeNpg9VybjU36d8NVfQyvz2F8QabEzoCZHSfgHGBuj7TH0FlGegwaKvkAKRcH0PKAMWmdTgBqMu+X0JwXK2xfelQPQmJxS/+Gq2j6sZW82h0WgIsr6nJBOy8Yb8ZbuSUPOq3oYNx1U+8Wnr1iDF1Bn/CShVrXBJKYK9phmesw69eqcCGx6D/cLSYw5J4PcIFxM51lcLwN3Pfq9tV5xE1goYRiAKnFKOBlEKBrNO4WVPIqyYPSR/rbd1WgEUCePsnzmMOaoR5jKSR+jvX45+uAupt1pj2+u763l7RLkHGClCX2SDGMZXFE/xkV/BK6xsIA+Wgn1kPdhlD56xoBIBHTowMJ0BR1nUc9zujxvdQpAwvfuvtbvHf3Lnrt/rxtknztY8/kLGxbmqVss2c8l2+We/P5eB6m9Im01dPdSc/zseWk5Wns4hp/8j0TcqN+mh6u/813y4y8FZf8wy8/XyazLrkK9VxolERqXLpz7nXulzTJiXLqJrDIFGnumxRoUuBzpcDPBhbOjXlzos8fl/o+T445af0jkNPlPOr3ZEbd6tc8FljEirbHpEkMsW4bcbfIBzWBiwwsXM1NoCKMt2FuQxUKZuI+sEgSBFWhYPj5hcRCYKGkAmbNvYxBksKkGruab30CWBAk70oD7lJi8SGwMB9BQimBcOUaZihUoTj2XvxY7Qygwb18LYBF3Fd6kqQGWXIQdYp6+d1Kn+1Ms7Rv0EkbgbCJYB8fOuoe3zP+yNh4TQ9PrqZKW2kpYy+4iL4ifx/x+5jADLCBC/aHvwwqzMt+F4TxnzQEf4MxMtjXKoDiDbYHAos9VvdVTxHEkDrVPzL3EfNL4CLuxwAiM5i5FvTaWyeni7aJmaIbUDGO5KEfsKF61zHjwDgNvRyrunQoAw/TtSmzDwPcBa2+gHnXHa3uT91UB9qE6VQtx+jRwzDgndD5Mav2T1FT0jh2GtWqRbwiLcFkyxDb7xsw0/4OATODML87HK8hhVijXVdEeE7AAmPg5aVicW62mKI8A+hp/OzzO7R9DQb4JUx8L3WdB1j8GuBy6iq+nqVer4Ya1h7168A17hDAREPzSaQWXtMr0jbM5hmM+JmqSNRfJlZ3tCOknWFlXi9V5zC73dBkvLsD+5P+kGrY5nXaLBjaBXgd8nwPQKLLLoV5lXGfRo3qz798GJIMu3ELW4VN2qhheR+MtYDKSOFGGjeuxA7A1bgfu0guTlR1o22DGDYP6t0Jm5cj+kKJ0jFpBnhGD14y4RcAg0vocsp7lpwvwEiT7pb2cSEARJv9Tr8ofbgCvFhH0GpINeDoi1vKuYGG4TGJMWsMjFYWFtqoVwtlaJCu16le9iMw1WMYo9/QhkP6YYt6XwAu2k8Oi97Tw2L04qSYALXMAMK6QXzGlTAui5Gu9d7kpitcAU8f5Q7TPgGGdhbGgzAuhJ6iuny/BRSkVW1LevnyxPwFDWWzM9iA6tW7FvMcr6MudH2n9Ffre6sq5wHg5pj36ATQegYNjmnnMfU64ae7WaWRSlj1QtYLiOgBVNz2DRaXPdgkAXymAMDLjOUFxvEA9OxQmmK14l80rfpjDf3/c7b6XJ7nn/pzkVe+8Ik8fzwPR+CHD9bLuvt8WVi0rTyOx8s83Jkl24e5puvpb75bJq5okq6H1KFMYl2k5c/aSBtb/QGu5dK8l7NqAotEqubfJgWaFPh8KfDTwAJ3s8FQVnNjngKZDMuJ0isyOXmrfwDyNfc5fb6W07nPx97L5+5lyhKoSKoyfnRDapGBhat8gAe9QKlKkDxDCTCINcE1VQV8poWvQgSvgxnQ5WuACRgcgYU/GXpXHFUDSpILP8K0jP9+Amx+BhahCoUBdwUsYMw0mg468Ewl/SjzVaUpgRk9RCmdyOUJLFD1Ka+FEaj1oI4NI+4ShASzIuhRbiAto2oV3aQVbIzki2vWtdL1hiFM6kZKBARxZWeakumzNQAAQABJREFUhvMbvCuFBELawqzk9nI7ClFiIbgwrZKZ8EpjOvJN+SVJTerfm/AEpT75Orrqq0R1foc3oC0Md12dvlIdKrJNthwCEuseKloyicGEkQDpzS2gomUc3XcMt1uQWnSh426k7UGYYmm0qwSBOhqMzbLPqM8xDJd66heUZXTl3yzNFV+y8q8hsmBUBvsNjNkrPDx1QdMBxkA7ZT5FDek1Add6ufYAw+uH06y896IDz7kr4Fvk+4oV/3UYaMfjMQDjkLz20fO/2Ua96RRVKJj76aXFYoogbzJyo/yMp+BK+CYrzG959jXlDtPfiwCLL0YGik3BBmpQTwnMJxPeisrQIMbEnTDCXTyrNOaAspVYbMB8n1HmGQDhAqbTgdAPgJljRf6riTEC+nUj1YDhpL6jMPBfA44EbIeAGkGRgMo81jk+5bqr3hpAj9Djy6jR/PlXj8INrs/revc5alkCNd8HpXwyzI4OGWRVrN6bJ4bjh6Qzcra0Mn5GP8zsBeNaKYcRzNuhkQH+BBa3MOY3tB1kwzBkLMBQ3/COXiL1uCa/FurfRpt0EdwJwLhhxf6Ke5fQBue7dDZqUjDNN7SPl9XBji0N0hvq2aLkQ6LEy4F6EGNUG5ZexswtYOCC8XDEfHENuNBbWBdukofOABdXZ8UoylLdZHfNgoHSLo2mlSA4Httos0E1jQyuOlQ/9e9TxYr2dHUSrwMJmvdi4QBG3/cz/lC3eEdKYAHh4j2yenGdfF2wuGQsGZvCydi5I2yVKH8HJwKCC20stGcSsJ/RRsGFaV346IZWfYCKXtSebvuHirMOVOewQWpj/ExhyK207oHqfkhyHO++L74rUUfrWW6+8/7/1OaYcItnbUBtS/ecS2oX82GZ1BLjdi1Rmi/sQp+9m3+eVyRkvldlWZbv8/fv5TSNtuRKsbcu7OJKvsylqEftPB/aO265ubksq5qulRlGqh/5c6/NOX+fyHmmOtw241j8CBmbt5oUaFLg86DAjwOL7zDeFljUtvtTaXw3mGXdVx+KWvpPHdY/CnlyNa3H+ed5AhUwnsEAp1V1gUKSWLgqb8A8AUX6JYCR4lgILrKNhR9UP8QypFkvOsWycLU7SS0CUPjxjXQwATKsftj8RwP9IFyXqlDGbriQ+eBDrzeZW+rhJtMlAxZ2FgIIQUUJJLLEIrmc9V4CFjJcWT0qSSksGxABMy/ASNIMV0E5DuaEWlGf+OWvpYX7AZN+HruZgC1JJRIgCPURmMoACdJUpscfx2HPAsOYwYXPCyjsK7OKfiGtfRJqSzBuPBxlpkIpmRVXwZx++TdhjFY3DZq3zTE6+TDR2gaQKGqd8k0r+gEgYTJjLNBOlmGLFoxPjV9xNT5Z3LAK28HKsJKAAZg8W7am+1f2qjhJJ42gj2FOr41xALDQfuHrh8vFl6xYz6Pnb71lrN/AmD/jWVV99HvVDs30ZGX8gElAywPUpxZg1vthXKXlMfV6x3NveU4gcAajLqml5RXGwNfra0U7KjRKUxZWVopx3JfaT4ICvSuNoXL0FubwKYbZj7dZKYeYqitNwaBuInnYQGKxCwM+jCrVFKo7swCbVtqUA/gpIdCz0gskA5u7hyElOBdY0F5VtRap729wbzuJ2ssB7ZaW2pN8AaDyPTkEgG0DKlZtuz+OBSvaONxQ7igM9Tyr23+2tIAkpScMrvcAHd+tbxevAYT71D2PZ22B+ugDBjUSACREPH9Eu65R0zFIXnheA1zcYDR/S5/Y3/0A/B46ysB6Z9D3Appc8TzFMlwcg9fRb1fUDTSC8XZ3MQIYGEPi0gqNtafYASwewHyfSXfeqRaeh8hpXMJwgwT4Idm4LMcjNFb6qO1BO/W5RVXuijpo98GLi10KAeh4l4euz4sRgMXwFZG/eadvyOMCkKjEwHHsgoGdHe0C1BgwrwdQpKSoi3NVonqxDUnxLagXbfZdinkL2rcwxtxixDMmkg0XLXcAOXjZ7C/nMY2zHXBOypbveNS2IlSjuKbU4gSQpSqUuWqz1Uef6065fXAYFbGe4rQVQOccR3unkFQsI63TuYEqf33OM9AFqqV3mjzifSM3q5O3PI/n8/p8fP9eThMV5yTnY7pallFelEUC/6UZABKYLj9UZuY16fexLZef9x97NpVjjVINSjJbu0aWTOi1s8Z1E5c30vNlHly/X1bjoQ/v5frV0zTqka5GzrnttFniNSUWdYr9cR0fnt4Uv31+Uvyj5Z5iqI9vXHNrUuCPlAL/n4GFdHHCdCLNk+lHJ+x7BDStE/X9yTpfy9eDiZWZ9WMt01v+VKOw8p57LLAQVGhn4V49/7Cz4Lrp4HcrYBFG2zAmCViwhxnIEgsZQn+hdlS2KT4wzvv8ZLq1sTgPYJGMtz8AFjCFYT9R5htAhvLcRzkBNBKoEFxooJzTt7BSH2VbB4FFMCMyzqUkI18D9IStRaZ7fKBoZPxXTE9l/dk73JN5C5sK6JUDdyU1jZKZjzRJHSOYIjOKx/3DFn3lzr6AIn742cevzNt8YZOCKVBatIf+/erGZniHWsMw+siVdpjcvGXmwlVbAY7lhqUGTFEr3mzaiF1RTM6yyo3kgmsCrG5o6pTsCu8WTDmsXMRmcIVW+wilFbpBZT08VFbmF+bx/jRYTMMI+pySAz0LvaQuruRfkI/A4gwmu536L6FW8gBPQVMwtbZNo+ltmPUXSCsEFdo5GB9CprgdVbIWArDdEiOhC3Cp4fTSg5VijBgClzyrytMI9TIgnpKSFwEs9uGBVWO5SfEZWOk/I98u+tNylydHiwnqa9/JBPZjP6FhvYy+kbZl9HXZuo2k4Io6a8exhErVr5cBFkgdDkhn1w0BLGQobcMpbbCtr0n/krKe81Ot6pg63VB+H/mPwXzPj40Ws3hS0huVdiHPkaSsYni/RXmOJw2oe6HPOMxqF/Q8Je/31GlXaQKr6DL3LTwnCOlCMtEFM6uaGXHTQxWqC0BxBN38nfCsrF38490+hiZXjg1+/TDM45QzhyvgIfJytV73usbd2BNcOFbI49aX2k3mX6kh/d+CRCxGrO8L75z2Ed20rQWQoUG6BvjJixsSFEofxNfb0M1lMXKDZyoAxS2/y5MjjKONT5OBRQqwmcAFqnO0oxMaOX+oAtVNfT32HW6hvlEB2uecFRKCspr2aVYr9D2SphlohBQWsJekF9paCC6wowAABYBlzClF0d5CI27L6RTQAGa1p7jtRiLTCnBjfvCXgcVSKbGYhJ6CUNsgsHBsxUY9/Of/tFnHfJz2eU6OM9tw93Z1FvmkRIkE1Z0YzuV0URX0yXzyY/kbkM9jTuMk771+P01OW79+t76Wz5WqyY365GdTB6Yz26RTirxBLrbGeZzdOU0n9TrmZ+v1qB5JGZLEu01gkWn1sf2/+t0Rixw3xT/9CvXGQf3W/WHb+eVt8T/8HzvFt4vdxT/7Fs9x/0Dbk9Xz4l/+b9vFf/kfjha/Wk6quf9ARTeLaVLgH5QCPwEsfih+CHeztck8PkSNOjqBOiXGRFoeO2nWJ/ZG6lo+n0jjc/lZ97GKDuMTzC77+8AiXM/yEQ6DZZjZcDsrwEClQdWBsLPgvl81P6jq5t8HFiFREATItPLhDUmBH+BSxaZiYGjoNcxkKguJBWoauvt0pVWJRdDB5wM4kJd5CihgOsJ2QmABw1MdCyg0+iS9THOoP7mnjkn9SPCQwAU3qVv6yYwkdal0z3ItPEe4jlP64pZfbEFTmHboF2ACcCHIiHOYn6C5NCZdABDT8y+YN/dB/+hVzsqVPtJEOe5h/GSA/JGYy/YhKjhhZ7FZvNI7FFKLRkyL9KzPe5TKTeXcQo9bPO50ov7UObtQdGJf0cGqsNIembLwcsSqtsHDjgEWel3qVb8dHXnddJ4BZgbo/2GNWWG0+zEmHkdtRcmBzLtG1G9xE7sKsNjFjuBIZpg6aAuhh6lvMJaWGRtFwmEcA2NAvCPt422kCjDPpldiJlPaTkyT9iO8SCEhGeD5xQcPiyVsLIbxHKSLVwPxCSwekKd2DqtISZ7pnhaQostb23AOEOqGGZ9FUvGPludCBUtJi9IUGow9CeOFvGWK12jvC9SOXsDov0LCcsyzOMMN4PTNwiwr/INhdG2XqIqljYgendw04lb96znlCixU7VK6cQ24aOc9gfXGLWl3MQ6g6geU+K4oSdnGhmKf8uwpjdxHASt62eoFNBxCh1cAC5l+mXC49vBQ5Sd/kLoMYvuhjUMPdOhizNoO42fsUJ4qVkohhaICdmNP3FxwBnAQkE1A/wUkLkqaOnlWw/HX1H2T5zRm1zvWpeMO2sWY49qtoIp6KzXQ1kKJwgAqQAbKa6Us3RPvueLP+HAecST3txAfBMZxBCPuNqRstwDFq+ODcL97Q/+r+tcKQdt4XvlVO2nDVS91lEahXkmpzhW+vzL8vvfZxsKxJV9avi4xTzpX+n74nqe5QfWnpEJ4LsCiLdEuyvNB0+qONqSy0OESANwK0O7AiLwd9cCiC0fHHUpkcE7AI1ijhJeucfp/AYnGCn02g3RM8Gg/NDy8UTnLqO2CflQ6z8Hp5t2/6YnyOW/lecbj+mXP3SI/D5xVahsn0sfn3f9Ymfmp+wx79UzO+F7d6+lz2lSWOaaH7tYqahT9lMrMGbtP96IdZYe6S80v79lf/KuXm/L58G+uj3fiaTJqupv9kE5e2Tq4Kv7b/2Ujbv77X/UVf/lPhz6e8Gdc3T++Lv6b/3mj+McPeor/4p8N/4wn/m6SNIHF3w0dm7n8/58CPwosfvvdDxHHIn0Iy0m4nFBtmtfzLzc1T5D5/Mf2dyZW8srnsXLtBM0vJBV8dJM6T1pRzx6hrHzypMLqHh/ecDcb4EJgAciAkYh4DqQjs5IRSF5bYrURxiNsLQQCMAOCjrrEQlUoZ/z8kRBgBLCAobxUYoHahB5tBBe3MERu/469Nw+uK8vv+w52kiBBggBIACTAFWSzl+npWXq0WDPSWLI0ki3Llix5kWXHjisul+1KUpWkYjuJEjupuBz7j1TiyC4rKcmOFkeWbVnLjGRJM6ORRhppNEv3NLu5giBA7ACJfUc+n9+55+GBze6eHlkaa4RLPtz37j33bPfcc7/f89skAkX6sGe8LZmoIxcSiZBSSDQKsbBsCYagRO8yfOgTCQZ/QjUqr3YW4AKp8Dzl+SFxvJzK/Yg6x82gUnHPuH/2qSQBMFZPKvgBsBHYSzj4Hv+y+pmeeUI1qgK6luPmaIj7ZZ6Arx2kOLv0eRiURgoWsFn1ncBu4R5xHnQ9+xBQrGRAe4rIoKpz3OeoF2UCmDTYPjx4IR0/M0DAttPhsegI98p2znFPjTkhODdCsaomSicEjtuLjwIcDkAMzuJyU1ez6+ib61JYsOrqv25Th3EtO8K1rrSvQi6s8wlUb4ym/XwP6kzopmuvYGA4A8x5zXU8Ro1Mz4XKlIHy6CiMntdSEy5LTwBOezG4vvaOd2Bf0RuRse8jnVCVqatSSVpjDIbROHndp8wHgPwpiIG2Eu303yDE4t0XB4JYtHDvJwHrxrNQzauburjXLa32IQaJM5K1/dnKuDsLgL/adwoXtUcB7sRiAKTr0nYAUN7NR0CvJOMebb4PmRinrxdY2V9GirHD8Qbuk1KbI4DTk8chAoxNvYkZOX2B++Wzpfeq44D0s9RT9RoDSz6gjkYLnyEPgTrsIOxVjmDwfhxC0QWh0yNRN6v7kjrVkIwY/oA23LfvuUbJjg+ZDhAaaUsrP3URayDAU7TrHPX3vmk4rtRHUqIB+SLP3SNW7xepH0w/E3ufQerYAAFqlgQxZnogFT3tjBCuV1rxkGsW2CvFkdBKGNp4rtslDBurYSi+recopRW2ifsseT2MncxhHjPtNkD+kCDshcgngmTy3fwlSD4fLhxI/lVnjKfSNvrhXKg20ncuJJQYN3RNpf6kdHXP3sLjXBLPtVIKF1l2PUj/NiKt0A1zasVehfu2wzyyzVyxAc+yT5VwGo9E1brzuNN1TGvgr7vk8A7lM2rX5z+5nv6s28p8XA7V5kGuqZrDKepTbRzdn89eoqr9JqwaFGXvXRnf6o5ZVinffZTCsVIH09efLznt3+8BfKtSv5VrPWYf1//2WEle+j/Prp7xHGerDEu6fCb3hYT18S2OcM2+M3WVKu06UIV6vOfy71/47GL6+EtLeMprSTOPttLf/tPZo96TU7/1Ud9Bzmm/l9sBsfi97O2Dsr6cPfBFEYtSwccnXyfD8ilp3JdJsv7YG30veZZr/P04sSikIqtDsXrHy13dbUmFDXDlMqtCEceiIhahFgVYqRELJhKnEYmDhpiqdYQLWECAcR6CBNQRiwAFTDy+WPJbLb9sgNmUKwBA1Up97Ipc7ABqlBiEakPJT7JCOVmCIZCQXGSCIbEItSgAgVKMYkcRkgvqEeQi3mqAE/uZegusJRfxPfa5/61k9eqNtCHZANiExKXuVWb9XP00SngQC895zH/0T5ZMIMkQpHPcFWX7V9DoOQmOneHLMUsZJCmSCkAP4NQAY5bhiq797H2ZxM5CYnHr/lh4htL4WA84+QVbvYKtQ5RPHXQve7wztZ+7lLoHiOIMsegEJOrBSfKnvYPgdARiMY5+v65XtwX6rDQ3YeeAUkh66vxgujw4kLrD7Sn3i8JaqPtxANkMwPKeMSsA5XqPkpwweLBP6EgXTnWl9/QTTA9AdhRJg3d8iz7QzaxRuW9NYuANuXgEMJa47hJ9vYFozj2Qh/PEtngeYtGJasoG90bJhGX2oDJ0BaKzClDVzmGSuo6uZnWsccjFKlKWFso/Tbpn8GA1iG2HHq7GtKdAQtFI/wqOXXnfof0z5DNOHsMA8wXIi3XvYQwPQGyOAcYXqPMjyIPerXoA1icA9brdXWSlfprjc/TzIuNwlf0GecdY4B7rXcs+7sULlZGujTsy8YiI1YDmXQE1Zeie9xR1OQ/g11PSGKB+lI+qYo7/FvLTWL2bejgGjCouuDf98Vby5LykYhhCdZt7N087NLbfFeTTN9ZTtaITrLLr6SrUsyjPPL0X1l8pyRrP/yPbg3RqHNfCu1yrChYPX2oASDcAul04MGq3bn37UUVzvlhljC6xCBDl0o/mo5pZA/VSDa4BKecO55XiSCyyah9EB+mPLoiVHrUR66IRQrGDutQaHqvWGXN6mfIZ0B5D8L/3nPJ8+i8eG/bOL+TTTBtznJvsRAKLgniOJBU+x5J8+yo/9zSdTWcEoVrFvWmCIEgqmpDsIapDJ6+FtjcHodAFsyR2lzmlHUJ6ErI5cPJEzTvUCcouUgsKY/OPtdy/lXm5HC3zs7+dL/K1/IjGlVR1e57p129VX3Bi/9m6/F5/EfNRmd04+Vh5uZ71uZWW5GP+ylfXp8mFRBVJkK+gVVWSsveExZXeMV0+Zw+8Pr+9lDn/+lZG6lrG1Znqd+lb0/zJb/lQufh3fR9Su6o/Sx3qC/VYeR87/68z7v10EYTx93r7Bz8xlfq7W9KVM23pX/3qw/R9HzwZqkz19dCGYWl1O8jHBupOw1MbzFPEDOK6FqOB1m2jM5tIMhuJR5NtHeqvVVXq7sR6XHuhD7Xc6tLxOeZwJCd9XdjNHXu9KtbKOvGKplXDRgJ6FM9/PSyG1RV7QCzqbsDB17ffA2uj6Td/4efTr31+GByDzeKpp9KLH3h/evGZs2lPM5DndHY4vfIbH02/+Nvjqen0s+mDf+qPp+cRzO2sE6PppQ+nH/2Zl9LDxq409M3fm/78i91vvx5fxBVfNLF4/EVj3k48ZUIq+3L8iyi7lqQ+b7+XT4BZUrn3mC9WJ8MsscikokgssgrU48Riz+UsF8cLRMCrXnTRj86G1RAAgJJgKVSMaFfWfWZWcOKtJgcn/hqxCHezAExWTjcEGgAX3zqC+ZBYAMRCCuKeT5ALys1qUB6TcGRSIbEoxtux2vkEYhFgJYiFfV5JKJz47R/+4NgpPp7TaFWPQln6krvZJsSHPrSedCpwJvd1vOY4FH0cfQ1ZII0gKYgFoKt2L6I8znEf4l4AqGLlFtC2DdkC2kXfqrrkNVNEax4en8AJwEgmFqp6AAKtc4Ao68JHYhH3HbWnphMn07Hzl1PvwLnUTxThTsCQfvjbBFX0zQxg975qRRCDaUDqMuo6W3hmOgwo7AFMvfPa1fQUhsinUYWaQu1Jl672lAbT2iqMAspvQCzuzxAQD1uFbSQIfRCLq3096b2DqBQBSJUQOKZb6EvbMcE1t/XexDUPNFimvWG8jz3Hacq8SPTmd1+9il3HkQDtt8nfeBXdEILLeOVZAagaP2OO9k9Qh3G+j0EwNIxW55+153QOUNyP+oqAcIy2TUBOjBGhbvxR6mSgt1XGgLEgVGVapg92AOeHGXsnMSrWQ9AGaTVilzgepf56vNpBMmHANZSV0hbkCkOJII5bnqOOjNxYldd4/Dz1P6GEgDoNP1wK0qARdushYiVw/BjlnwXYquY1SZvC7oF2bPLp5B71AuLPdx2POmxyn5EfpGtEApeQ+G6fhNgNI3F5BYnLA4jRQ/p1G6mHXp18Xg6xst7V05V68L50iracov+UWIAPQi1NVTjrPAugH8fj2N2RUVgTBMthLdDGTqQJNTifr8v05XnidQxAUpZ5Rlchgy48KLVQtUr7mRX6TpLoPdBOw73SE42pHY8+d9poHKP+J1zxRzrVjKQKY560/nAOcoG3M8ihhMSgdUYwdyzXNu5BqDY6dnn+GyUVtCsCYtLesHngflqWz0EDfbjLM0xzqrknP+MbELhQn2I8tnKPmlWDIiJ9A9IKgwuq/qXtjxIcVdy8Z22otik50n3yecitKlGn8bZl4EPvH1WrbaprlS3qUn5U+/r5nSfVhzdvZGI25Wd1NNpS+173JVJzQeTh8epCd3XVqbvCrPLcHQcp7/X142r7r75BVQ776/Z4KbkW9eWajdu+7GxjddwKW36pf1xbKzf3hdd7qL6e5fKoTynEhHWb6f/kh7617sjv7tffL8RiBILwj392JmwTLvW3pr/7Y5Pp2sCh9L3f0Lmvgz7y2wvpU6+tpO/5us70Ix+fB+DHU5S6jjanv/zNXZDsTCKUVvztHx5PX3OtPX37+7JKVblW1agf+dg80tV8x85AIv7yH+lK/+5Tj9JnboPm2LyHH3pPR3r/s87aefvp31xIn7zOAgP3UAnxJi/k3hMt6T/9FiPf875mOyAWua8O/n4pPYDWxM/88/SRu4fTleefB6s0pOHf+pX08uKp9K73fzB93dWTZIot5t3PpF//lV9Lt7YH03PPX03nsPfs6T1JvK+ttDR1K/3GT3wkzb33m9PzTbfTL/7scvqGv/Xd6anGpfTpH/uptPPB70zPd7eBBb6U+u2/5s2JxSvaWIzsv6Luly+b8sIp+7rTta9vdK5+4jVx+e2+fC+kwtVzV1f9nQ2oK2kF4EA7ipoqVBhW85uXfLicFSjE6iMggTIEpiGtAPhKLgTyEorsvckXveC1Wu2v9lE368c/1WO2MPbMcSwAaxILQJGqI1Q6+kOPPhqw1pOI/J3yqjJz1G3Jh9IK0ruXbFQko37VM4y4ARvRj/Y59WKJMlQvJBTq46/zxZgDvoKDWJCnuukCl2ZQWXa1S/u5PnwdUVeJmv3sZBgvSFvI9/jQzyGVEMhVfW6/+5HYZYLHPaBvDea1Bdhy1Vb7FHXb7VuvnwK4j0xALACA84BtPdpsU1dXZa171MF6W659jOvOFiQWHecvpd7Bc+nU6V5APhIm2nOYvhFkr1P+DPncQdog4J9jv8rq9SEkJL0AwPdcPJ8uA/S7VQ0inYbIvuj10qRqk0bYdyQKSCE0hnblv5t8L6My8h7sHHQbq9TA1eyjkJFD9KNqQ6OUo02BalSzjKtHtNt4GR1UvB9g/w4kDj2QA0HdHdJpgNtO/w+Qn4RTYL9un0AuxiU4rIzP81nldxNt6gUIngaYu+I/C1ieclWfsloA1vYnAyRtAf43qdsqDdrk2l2AZAMEo418lbztQkDCaBoCYeyIFSQ7qp9pf9BB3kcBmq2AS8mEdVFysUL/uP52iv4ZAogbA0OVuFuQm3H6aoN0Ed+FPtzlpp1EuuIYnOPeh50K+euFaxBScAFScRUjdL1OLXJeUqdBvBIZXfuqxjRG37+MitxdJEAzRmWHGDIQCIaIChOSl+6e7lDhOUN/RHA92q/0Z4k8V6nXBuN2HDe540jC7o+MpF3qyeW0/VgiGl5qxHC8hTZe5P5fxBj9osQCIrjCJ6QW7B+GKhVxSSAZRuletx9dIJAcMEZ2aN8u465RCQOE4DD3sJ37YlyTw8SxaUFKsYW0am1xAXewEgvuBe1tpH3u/Tgn7LpIwPPQDGFupD0N5MUDCTphjuEeGh29gXHbyFhz/HOzQwKxy77B+0kanyPJu5IVrDHCFXELUhk/Dbi71auc98Nna55xM4+ka9dnkPKMvdHJM6HXrwtILi5Atk4w1nWCkCWaFprBUt7vB+4+l3HcB6jafFI9vgejOcH5vRQ5Ybk20tZdX/J5W/uqHrk2T7py70x+3+Tf1WV1F1h3q2ttSxraUurHoXy07pL4WtqX224fuEWbo+2ZAOY+yPmV9puu5BmlPlapSBd5pAOvUHbWY9u/+XVA/a2V9N/9md54l/3wL82lGxhC/53vOY2Ecw8FSQ4+9tISc1VD+rb3dqR34H3ptdH19OO/Mp9ewJ7ie96ficgbEQuv9V357V91PD0NcfnYy0uRn8Tg2OGm9Ce/5njsLX/y4Vb6b//U6XRU/Ui2H4PIWN7Vs7pCb0gvD6+lf/HRufR1kI9vg4S4HRCL6IaDP19KD+wOp4/8Hz+cXu3+xvRNH3wxXTtFTK7P/1z6yU/OpWPPvj992x+6kI6s3Euf+cSn0mfH29ML3/w1uNk/Rmwm5vmYlLDrHPls+tkfvJGu/a0/l55bu51+5h/+m3Tsb/zX6avu/VD6gVevpe/7jvegNswid57EvpRa1q55C2LxWs14u3ZF9aVMxLW9x8vk/Fjikuaxw/GzTL5l78Hy3b0v1bALAEyEHYCgFnBbvEK5z8SisrOQWASpyNG3s2codf/zZB8SCwCDQC2IBS/YLLXIBEPioRqHwD5evFWbfDH4MgFmA6wlFthWhBqUdhau1gMiqKttNRaButRBWigrSAUgIUgF5dYkFZCaMPCOfSEVgBmu9bh5FbWnsLXwd3XMvtb/TZAKFmYkFuu0UYDlOYGYuuGSizCcrUgGu1j9jbFj3/qPvVt+NZY+d0+fA+TC9kJSYd+HGlhWP9PIVFUzV+5Vg9oFILnqXIIBqk41TbCxEYLk3YJYPAQAqgqV7TlikZnS8/12stdgVVUoicUJiEUPEosTp07T53nyNuDdGUCq8SnUuTeuhEBfz0RLgFX14M8AIt830IfqBy8BAJRRogW3lqO3IdV/HgnsuWd3Ae16WtJl6hHKPw2wfaqvOx0HQDomJDDt3EtjW7hiacToGa7V9ew0Y8zYEnpWamXfCVC8oMQBg2M9Jul5agGAp5pWF50dsVIApc2swC9yf2Zp6wPyUm1pFdLSzO/T1Fd3un0QAFfV9XplUDvHUwBH0mxyAyUWu9TP+A+qASXSNfLdVejDqHF1Uo+jR3jB0f+zkDmjbWuH0M+qtcbZxwDJq9zbZewsjFa+RPn23SDHJRbahYi+DOYnsZAMOJ426cc5fguIvVdG516hfPX/uanp8qmudPV0V3qG/RzHtPfQaL2PdqkupnG8rnPHMeq/DqFQPW5majptQz4F4A1IHBo6ufc9GNwDhCWJJ2irge7aKNOI4M5665Q1TIyU++PjaRxJ2K5SJ56BHVbxd7uNPH0itUIuzjJWLjAeLtKfGj8rrVhmrNq3c0g8JG+6J16nDZLjiIdBW2PMI6XYBZjjViruveSiDZDeTjvaMfQ+rGtapFXrEAxdK0skKCRU18wHkQ/HmBOof4PqfdpEaD8kWeA+OR5VG2ziuWpEvUrVtdgkHnz0gsYk4G2gbTwXknbStjAQJb26XpasNFE/vco5Lxjj4hEkbY77tk1ftVLfI0hvDHB4mn4YpE9Vy1O17qjzEOMy5hPrkktnv0csyrzgKdOVbf+csXd8XxorzlbmxHLtl7wv+b1pBrnMXA/ntTdNXJ3cS1da8kaXlfPxJohEJSVn9k7Wf31iBfKl+dpaH1f9e6AKtb/LfC/8z0gongLof/fXZUPrzw+vph/56Hz6zq89kd47hI1RtUksfvnzS+kvoCZ1DY9PZfvHPzPDAsJ2+m++63QceiNi4bV/8Q+fjLJMuMU79ft/ZAI11Mb0N769B+cV+T30WSQXPwZZqU9byqrfq751sqMpJB4ePyAW9b1z8P3t9cBi+u0f/cfpF5eeTu//wx9I7x5M6e7HfjL97PWmdPlrvil9y7t608a9T6aP/SrEYvoI79xFbEOb0vH+q+ndX//B9O6+3fRw7Avpl37ok6nzr/yl9FU7L6Uf+0efSJf/y29MD/7ZJ1P39/2Z9P4zx+I9//bq9eTUb0osfvsViMXdkX0vlfqXR/13sy+/3ZcJ0/m2HC/TcElrmpKu7D1XvruPVXNerPGdvS/ZDHAriQUv9HDNyEs9Sy30DKXhtoAnG3FLPIp6gsRCGwuNT5VcZPsKVZOUHlQSC8BbllpkclHq7Qs1PNlovB02FgSqAqQotdAlpoDZt0p4bSIv1Z8sq0YsXLnkd7avqKQUpNNLVUgr0JNWFSK7ls1gMghOvbTClU6BCX3siv+aNrMSC75vSC7oM1fa6bAgFu3kfwT9dl3q5jaxIEodQ3JRexlCiOI+Wf18T0q/hw0FUhr1zf0ugdL7U5A5+jm8b9F+QW6DwAdQFISNvvW+TFXE4s59bCwAuYI47yX/Q+VDFQ7L8r6qHy6xaEYV6vi5i6mz/2xEE16iOWsAUlfVLwOQekNVCeNhDZnJc4TPPABRADWI2tH7eruIx9COwXAjRt7LQSRUy1FXnqwCmD4U4AJ6Bc7TALEdrpdcnNK+gjIkAOu0s4V6+Yo6Sj0F8+sA23n6QVUkA/StompFgI50CBB5Cs9UZ/CY1MGqsNGyJ2aJAQHQO0T/KHFRVemobmwBrSsAzAmuMcaErnOBsamf4356IQXLjN05xrBxN7hx9A+2E5S1RNkb3PsGAK5G8+EJiT5tpjwJkDr1p1mpOEr9VROcJO7FQ4iF+v9nAZWqfHW2H8JWBZJgHZf1nrZFBPLmUFk6zwr/SaQ0bvbvFP0jOTAI4TJE4g5ugx9xTEnNuvedfB09quNc7T+VrmFE/iz9PwVh0oOT6l992mZQ3y7av0ybxyGBNyAWd4bRFUXqsCOxQKVnV2NkiMUhdLhPAIL15nWIMaU05jAfjfKPQdos8VWuuzs2libu3SM4H+5hubE7AO1d3RTjlasFcnGKvtTO4oJqZFyjm+gl6jyDtEP7jBnGpu6PlboppWhgz4Mc43OboHO7EBPEA0gjsFnB45ISlSOUf4SB2I7UchdJxQakQqNw74XkoJFx1sK9a+DZUMKwwzxjDI0G6wZ5UlLhUyZpl4w1s1cKoecpn0GlJEo1lFhwiCSSex5yfrRKKsgi3GOTRlLRxJziPOMcq1rmEiT5ER9GDapSEAukSId4Lk7S9waXfBqJkHYnuiPWriaee2vEd7dCLCg6ynTO81xO6UEO+8/K5R9715qOT36+c5qSb07M3+qyKLh2sHwpJ60Pc1xVRtmbqtTS77Xj+aBHPFyrQ/ldroqTtT9elNPX8olze3XISWuZx8/8q7qupH/8knxh7e/jfVCSR7mlH6v+PyAWtW6LL9dH1tIPISH4S9/Yla4gDXBTzejv/ugE7qhb0n/2oT0d8UIs/nskG4UEmP5HUW26fn8t/U/f2+fPmEufpAolsXj82v/1/5tECo0qFepQZRue3Eg/8HMz4VHq3Zf3iA2PanqAHcbiyjYYZDdZH204/tq35ToeEIvSgwf7t98Du2npwafSz//0J9PwPFiL9aeVdRxzvPBV6QNf/XQ628Ei6ks/nz7y4V9MN5qeTS++52rqTnPp/ms30njLM+lD3/MNaXAbW9eP/1T6hessNLKq1ND9fHpX22+l60e/J/2pbxrAtXv9LP/2a1h/xZdMLB6fLM3UY48fr5+WnVDrz5cJ3X35XipXjsWLtTofAJSXbCYWglxAQRCKTCoEsmvYPqzxYi/EQgBsGvPxRQJcz8SCF3j4o+elHMDfPcAsVKFqxML2cAWNKK8SvUJtscK4GTYWkooqlgVlKFnxzefqspIPX/iufEscwu1sjVio9lRJKzTe9jy/G41h4R7QGTYeFYkwP2959F30MTrYVAgbMVadibVAscxl4d9/CaC3CEhaAVTqiacLgNINiFCdp/S9q8/+bmMlXLGt//gaIjBLsrWZYCitkNRVH8kF7ZRcbEneUH9ao/2r6O/722BnhwCIbQAe27TKfXiA7cPwg/E08mASqQJ9JXmIPIUn2rIAq2gT1U/4WEobR46l7c7u1NZ3JqIJt6JHvsaLxFV9DXTPALr7AP4dgEb15MeQWowtrLAKDTEAxGmo+zyqOOrVHybvu9gJjAG0HgKG9Q7VAshtAkTv0LcP6UTdn85Sr01ANA0KW47DAK9mQK112gFANwEcfaUZw6GF/AX2j2iHakzrSBTU8W9hLOpadZCI3ScA7wLoB9gRqJrSxneJXAvlaj/RDsBTVWWc+i8KzLmHHdxjV9bPH8XlK3VUEqUqkSpfK/TXEnWbggBpAL7MfVflKUYliFppBdr2XNcWHpsE8Ir05wGauqidp+3GR+ik7E7adZL2LXPNHHV/QP+0MVbPQCheCEKGjYXEgmGgZ6pZ+nV5fQuy2BRemF4am0rjePh6hFH+riv1zayua9sAEL8CqVBicQ0D9FHK9b7M0P4h8j4HoerHm5T5jerVCne5I8P30jxqchKL7SAWqDJBLFqwkzjE/TsCIG+EhDTRjlb66dpgfwQulBh9YexBujFyPw3fHQ7Db+0StgHSiWBxjdhnNLNSfxTw3UN7+ztQB1Lqw/hdUhI0P5emUcWawfDbuSTmFfpUQ3kGe5B264EeEQC/OduA2CGQ3laeJWNbHSW4HkYuEUzPQIOqaYbzBka1RIgkXIGUh/u6Rf+qWuVDG04TuJ8+Azx6lYTPOYL6mYa0YWNBWucu6+d1Wb0Rj3bcB+cK/oSBdswrjGvnLRhIWsPeQ3KhUXwT5PIQkhujhB/n3utd7CrkbJC+tV+cB5Q+ueW/fKEfyuYcXJ2OQw3h9sqvPrvVX76UeSXyqC6on889RFZ7m9895hEvii/8ifIiF/LcIxYmqxJVyfPFpYyYSfJlOampKqOIOFwVFLsqRbm2+lnLv5QTdao1vmSec8i/6nPLubw+zyr3Wj65uR716ielPyAWVZ9Vu/8XycRLSCjedQmJn5y82lRx0lBbKcQJDKXd3ohY/MtPPEyfv7ua/t6ff/vE4u//xGTqxjK2nljcw+bj/8Lm47uwx3hPRSys40/9BhJMAnRquK3R+CTeq/o6Ww6IRXXPDna/kx4gBtUn/nX68OdW0snBizh5SWn0+qtpuvVCet/XfW1617nWdO/XP5x+/hOvpt1n/lj6zg9cSccb59PdT380/cLHxlLfd/7V9B1DuCN/OJruDs+CKVCPX7mTPn69Lb3/Xa3p5c8Np6Ud7Ew/8K3pqy/ilVMD0d/B9raIRXmB+LZ5UrGer6WpKlWfzsm0/nyZWB/fe6nH/BQQmlfQXdnOxMJVPCsf/t3Z68rS+BKqO+jmdA0w6Iu5fGKFkAoInA3a5aq6Absy8K/IBS/n7NEo6x9nOwfbtNeY8JTEaqUSi6IKFdG3AY9BLGwjM2DYTfDy308sirSiEAv3Sk04Hp8cYEuwHepQvFzNK+oRL8vSv6pAufLOKjagdIkPcYOCWBhATONm3XIe4dpeQOJpAIauc1dIrxqJxmVKMo4BUiQYrTTQY5IM1SOcpt2HW0pf0KICSIPEYRdSpRef6Hf6eZVYHsuog4QqCfemHXDb5gotffkIA9fRKQLkPZhIk4C4VeJP7EASVDURMHl/bZ+RlLfohzWCfK13dKZNiEUDUosWAGsLwFWQvQmxcDVYNZ0ugLErx9pLGGdillV3g+NB4vFC1JzOo44k+VAipV3EBBKNR4DodgDYccC7kaN3afcC+c7TX3NIAtYB3xrvujLfQn9pYEu3pm3AtaotwvgTSCKOAlJVZZFYzGGwvALw3QbISSy6kBacxwC8G/C2DrGbwPuUoJ7155AS+dRs83I8BgBuJf8F7oVqWZvczA7qM4RHpnPUz0jd2sbQ00Euop0QivsQFd3YzgIad0jjbdHdrq5iXZ0/S/lXMJQ26rbE0PgSeqHSq5VxMPSMFO55yds4ENqNPIKwnCJK9iVsI94JKTgpCQXkbpGn0hQBqml1BjAPKP/86ES6c1tJwwS6YQQHBMC3ICU42n8mPYXE4iJRzg0IeGvmURo2wB7xS57uxu7idHcaOnMa+xaMtyESX8C+Yuz+/fRoZpbYEXjmOgQpQGLRCEE5hBpbK+2RjK1xfls7E+px7RK2M2d703nye5Ux9SpqUDfuDGMwzcIB9xpREa6Kj2HLkFWOdNsbsTkYC12MC+KBh8re9PR0mpuaSvNTM9FOVQolt7HRN7sQEgJ2UCdUmDi4xT0W8DfRT230jxoRbUg/EqpQmwB5I2KLFPNzz/NEHs3Vs6t9iioVRkDfoQ2Cf34yhlSZZH5gDjIieQv9K3kwn/Auxdy1zr1z3jNfJayq00mOnZC8P34aPc712jX5LtDI2/gmSxJljh1iTByj3noVc1xcQk3uHMd6eYZUKbSeVMXZJW+UX9vIv3aiOl7SlWTxDFd9F+f442Vlezx9HOd8JCkn86+4MOfh33JyL7PHjzAdcWXd+ZKgXMvvOFTKK5Vib71fv3ms/njJsOz3zvl81W/x60l5lnFVl9jcLH9/DjnBVyKxuDO/y3yMu2ZFzm9j0/j67/34JKQYl9vHM3kol68hERDgf8u7O9LXP5eNqL9cxMK4GH//X01il3E41LVaVQlg+99/ajoWeA4kFuWuHey/5B5Yvp5++v/5cJoa/Or0/j/0ArGxUnr46kfTT390PLVe++r0R77+Ylr61M+lX/rseDr64nen73pBeyKC+975dPr4z/37NPOuv5b+yldXzg5YFNtYvJ1+6cd/Oa28+GJq+civpsV3vZDObNxMn31wMf2x73oxnTuBmu2XXFm1gok1BW5qBx/Ubyy+7e7Wq0IVQuC+TMo+PuW4L7va97qcypQch+om2ZJHSVp+1+/97id08mOfDbdjlZEXtSvn4WoWUBe2FhILwKHEQvUESYXqURFwChBHZvuJBWAkVrErAqDEIiQNvMiVGNgeV/RKu6w+8ACJheWhBgSo3ioSi4pYUASAgXzMi4/EIq8qsudFn9WgMpEpBtsRy6KeWET5XO9Ln+8Ciyw5yXUBSwbYXgX5zrFivcTi6SoMVAnGLG0eA5CohnKUNpwFUGg4u0bFJgE2cwBMcotzgq120ijZEMQ5H0ow/O1q/2HOFbIR+uB4xHHV10jAfrSv0EB+yZVbXK9yQ7BjgFjQZl+aU3NzaRSVldHxqfQIkK2OftzPilzEvaYcV6s3246klY4Tae14V9pkr4cfpR7aDajGpmcj2EjYSRicrRlgvUx+y4DyVVbUtwF3rtxrK9CNOlQn4EnQNgNp0LhbkHaS9vSycn4CAL9Nex9BVGYlCIJ7CEoD+R0BVLsqrNqKXnY05m2kXe3ciy5Ihd6ZBKFBSCAWi4DOLYhFM2OxB+JxsbcHd4fH6WPiLlB2ViMiT/rUcTvDuDzEuDtE3YQ9cxCGVeqgLcElVvYHWCU/Dlg8hpG2Y1M7FT0XaSsyjI3KTdzdjtKXxoCQoPnwq+oyCJm5CBgeQiqgxEHgMwfZmqVti+S/wfifwEPXLJ6mVpEmaCOx7bjifl2AUDx1qis9130i7HEk4dpdBAD2meHTxljRNuEV7uUXXkW8OnI/7XB/EzFHjvb2pb5Ll9NzZ09HLJAj9PNrE7PpJlKqu6Pj6QrqYU8P9qUXLp8LsnMbNajfHhtPE6MP0iKqYmGUDCFo43MYEHwU8OuqvCpKM6RZo96NtOPipQu4Ej6brlHOHex2XqMOL9+6G21pwFi7CUlHC9czaOOZ95Fvon1HkBD1cE87uAeHeIIfTo6nufGJ9AjJi1Ip+0FXvn5UWyKQBm5r8RbDb9WjjE6uytER+vcQK/3eW10N76JOtQl53kRiZ9rw9MS9CwmoUkfGjIRiE/Km8fU291DboFJOI/fNCN6HeEZ9ZvRA1sA42ib9Jml1CqGr30bHC4Rd8iHhlHiEVJbneVfCRJ3aGTeq2+nNaw1SsQC5cAzrQeoY/WrQww7KU5I3yLPRz/44bQr7K5575zjnaj8F8Qb09UEuG9/jPL+L8MJnuDY/mo4E8Vznrx6JrWQT+/LDMyVDv5uX+9j4Vvvh8eoH9Sz5h+ppSc6eU9VW+xJ1I9tqq32p5VHO5L3n99LUVWB/svhVn646vVdQLvcJV+VLM6l4vO/M8SuRWNx4iBrj0bdPLH7z5kq4ln2SLYNkPUgHLmP/i+84Fd365SIWRV3rT+ON6p2XXIJiTuGd9Y9+choJdeOBxCJ65ODP76gH5n4t/dA/+Vxq/4ZvSh94z2VczJPb9KfTv/3Jz6SZ/nemb/rWd6e2l38+/ftP3Uy7174j/dk/dJb3FO/PW59OH/3ZX0yP3vfX01+uPKBtrxFg99d+Iv37hy+k7/ja+fSD/2Ayffv3f3e6kj6VfvB/uZle+M+/Iz1/igW+30GFvyRiUcor07f7WpRnvseLpkyyvrD2ZvzahF5eDubl+fLbffm+J63IL6swYgyJBS9fXtjx4SUbEWkB+kosJBK+dFXDyV6hOCY4CGLBS5rKtlUSC+0fskShIgC83PdsK/YkBV5jE/xss4JfVKEE06EKBeDKAbVAMrwAa8TCFUEBAx/3em/JJCPrRYdHqDiHxKJRo20kFgIUAEHYadQRixzXQoKRvcToAWoFYjEPsF4OUgGY4py6//dZJZ8FVJ4AiFxEbaQDAKFNwT2Ayjgg3NVrdc57ALftAHHbp7qNOtdHAdZdpNclpdKMUOHhvDCYU7HiqjH3Liph7iV3axjErtEXO6iHqed/iDw15J6cnkmjgMvxiWkMXAFXlOuwkLy1AHLCUw4EYh3PNksAukcE+1ohivA2q9dtgCXjJmjb4H1eRCqhsbDgyVVj4wFgbothNn6bkYTsshLeBNHR0LcbgNzBiqyruVOc0yZAsHaCc70AqpMQC8erfTINgJ6jT5TG6Eq1F49IAjwB4YyG0/Sz4Ksd5XYJgfFO0HBMs1yr5EAbiw3Lph+OA/4uo4Z1AalBHyvuSgP8uOrczn02JsZd1JlUf3Ilu5nhsqikBJBvZOTzEKJTEBcNlY/RxghmRp2BsaFaNUnaV1jpN56GEiDJkCowx1G9uoAa0gVUkAZpm15/rLPSKW0cjAJueWOSEsD02P3RsC2QVBzGnuHS2X6kDT3pGbxicasxKCdOBlIegbHxG3QV2w7RUX3qFqTgc6+8mu4jKVifnACAn0ydZ86kwcuX0/PU4QzEzbKHITC3qef1+w/CVuIKhOu9Vy9E3neRQvwmtlsTkIsl+5jxFgbbSDt0jXoCMOz2UOkGXulmkXps8v0ssUkuDZ7B1WQfKnZz6eboWHr59t0gnq1Hj6UOYokY8VvQqzH/Q+79Jv2jrcEJBu9x6tXRSF9OQmjGcCYAaVElbod5YJt7t4OB9Q4rLLuQE4PsSXi0n8HCO+xjOk51p3ZIj1JLXcxuQiy2IBa7PP/amPCgYx9RVJRQteSeh50F/Snx3SAvbaA2lYjQRvu/GcLj/KArZQl9K/OX9j6OKaORKx2TgBxmcSBswkjjfVf9MLx9cX8OQZzaIbXmoa2Ix43PopNln7NDtP9wEIs21MLaURf0QwBC7usxSQv94qPt3BIAnjq6lbmYw7HlXf5bYHXM1XFdnh/L3C1AL2SgpC25vI6w5BP8LcSCfRRaFVw7b4q8VVWMH7VU8aWqX5Ugv1tMxpUeqyXO+ZS/Obm5lxLKmXyZx/faU+pJmtI5ft27JPrOn4/nVtKUfqtdUtX3K9HdLGYH4We/WsivNfmtvvzTD88mY0f8nTcIhvcTxLP4LcjH3/xjPakfe4svF7EwYN8//NdTafBUa/r2F4+DD3bTz39mMd0ZX08DxLI4kFi81Z0+OP+WPbB6I/27f/bT6UH/+9IH3v9uFhDBfi//Qvq3H5tIh577uvShr7+Wjkx8Jn38lz+Rrm8MpW/80Nenofa59Nonfzn98mdW07N//i+kD/bzjtpBlXr85fTvPnwrXf02CMThX0//6Pvvpm/8/j+Xnk2/ln7gH4ykr/qbfzw9d6oj1L/fsl5vkOBNicVnXtnzCpVfOlUu1WRaJsn6ydUU8VKom9gfP19eWLW0fCkvpHKufi8wM7s9YlGRCsCr/uYlE0EuAJaSC423JRZKLvRYo+SiqEIJlFWF8mWeiUV+sQf4B7QqZVBKUPv4OqGh0WT2O4VYKLEIYoHLWV7k4b8egBhpJQYAGe0o6olFrGQGOPX4nlqU3xuRWDRALhp4yQu8a+VbFwkG9SovW19WeoEqxGIFJIVGdazCC3YfsAr+iLZ3ATrOA1QPUQ+9/9zjuK5O9ZKke8segLuvPmMDaGsgsdBwV+9AJwQslJlfqBCyeLGyB1S5Gu5LQnUp75P9vYa9xQZlGsW4jeONGLjP4rlnagqXoqjErKtXRDtsq3YGrYC4Jgxad3Avuw7Q0ph5jjYsybOxNVE9pJNPBGujftOs0s9gC+HLXT13jas3KGcVYqVkoYH6a+jbAbjqA9gfBzSpWvIAQqMtwSIEoI16Crq1z5DALdKHC9yzVap2HELRS77nUeNRHUr1nzHBGS8J1eNURVH1ZJ30umf1OoP1uXrcRpoj1EUvVN2UK0k4x0cyux7jExUxASdpjBpu1OkppAZLfNapn9KhS0gNJCUSC8e0nn8CbMZ9R7RIXtrPvDY1m15DDejOyAPIGv3BWGnHHuACoPo8koFBgONJ+sby9GpmFG4/3uMxwPgIYP7urTtcC6FpIaJ3X1+6PNCPX+xT6RkImfdzCruVL0zOck8JIsU40A3vabxFqZpl4L+XXn013bt9J62MjqZG1NY6zwyk80OX0zuxLxkAeBt5e5r87+HK9+UxyAf/ejn+3GA/Y49+pR6fwzZicmQMgIyXKYD8OeKOXDzTSx+cDPKolGacVfeb2OeMQ0QWIWQGPDyDX25J1CzSkrEHD9Kdu/d4JiHBqEGd6OtNfZArJT3LAO8HjBnv8TbE5QgA/hgDtgOatonEYvXBWHqE8fcWYy8TiqNph5V9DbZ3lX4wHhshhQ2oe+0EsTicjuHCuAOJkMRiEUKhJGUbaV0DJB5xAszbD+PEa/mtqqWB95Sk7XCvtONZgwhvcM92KIMEQX59tr3XRoc/IpnR2xxjZJXytxgzqmYdYVz4TEist5F+rFInY5PsODYlDhILzitZVEKiJ7CIF+MzRz1Ul2qHnJzBmFuvYzpAOM2nC3UvyYXPs5LMmOecYHzyyStvZZ/Bcx2sZkTkjcvj2nyNRz3y+PbYMfIv15eUzi1ukTIqU844E+1tpWr7cqxLv1f3cg1X1y7KVz0xTUle7fMldXXieKlHfR3r6xHnq7JKWrN7XZqqjNhV6b8SiYXhJFpgrvXtr2/6k74bbO7v/uhEeu+VI7h5zd6gHk93d2Ij/ZMPz6QPPn8s/ZEXjqVfAMz/0ucW0//wZ3FRXueGVgLyEjYW/+P3ZhsLpksG9J8AAEAASURBVNL0t//5g/S1146mP/pidgX7Rtf+b/9qCjWs5vQXv9E4AXkzCN7/iaepegnFrxLD4iOfXgCL+L5oSC9S702+T0M6/uq3ZuNticY//cjs67xWlXwP9gc98MY9sJzu/+ZH0ydemkzrzcRTOrSb1tCU2OkcSu988d3p2cHjqXnzYRp99dPpN37rdprZPca7G1VtDDIPX3hv+qY//EzqFs7tsOD+6H767L3D6d3v7MPT4Hj69R/8yfTSUWyVtqbSZPOL6U/80Xeguog6+BtX5i3PvAWxyHEsnBAyqK3Lj0k8Joq6ybycLcSidl2VpjbJMomG7nDdtWWSd18+RWLhdR7LxKJ4hZJcZGIRkgpe3hFtm70Br9YA/qpDSTgijoUTN/99eRq3IusruzotsciShQD+/N4P6nM7S1V3WPGMyN96hYJYGHlbdahtALs2FtFmyYD5VPlqvB1qDACOKCMIB8TCFzo2BEEsSNMAoN5HLCQUdSQn+pWK7IBQJRYabmtjsRrEQsFXCgIxyYqmwb+6AXiDgAeB8UOBGiBWguFqvOotnQAOXYnqenWCFdIWyusFdA8BUowf4EqmKjRh4+BszDW+IFxZdQVeXXJVxTa4DzniOWv5SDL0vdSMutjSwqO0iD79IkB1F686zazQHoJQdLBi2oghrHYV6xwPGwvKfki9VNvRdkSgZdwD40AIiscxMtatrOpUrQBng8WZzlV5V4hU/lHicJKV9QHAtXYKqnGNAlzHAJjTRJFe5UFs5QapdmJ8AmAjhrWoKbE67TUXTmBrgESjjRfDKgT1PsRCiYjqMt4X3cvO0lcGIAvXvrTd6NLasNhfAjSJq0Sh31V+ySVlrNHnoc9OWXpj0nD5tam5dG98OlRkOumLpy+cSU9jn3EKicMK5ajmoSqcUgvb4SaxuY3R9GtIgV65PUK/Loa6UhsekM6wmn4OUH4BNSyNdJX2WDeJjYbkGoKPGgUdYnHn5i1sQ7gnkLojZwfShf7edAUbiKcgED5nk7T7cxhpz9Bfh6jzM5CeSxh2KwEytsUrr72W7t++nVaIIZE6e1Ln2cF0cehyehf1Pw/wPkbZeq96QBnXkTbMMu4k7gOck+DPAshvYR8xo9SD+9OMwfVTTw2laxcG07PkYdsf0s8jGnlDQqz3DCpT7dS3EwLRg2ej5dnpND85maaROuxCkNpQXTp+GnKFZKQD71w+83fxYKV3qlX6UaN9tCYA7pupeWoibUyMpwXsRDYZj9vHkHJAznYlFvyGRcbYbqEOEot12tEMAD/a0xXSIWavUP9bgdxsIbloIp3G+JILx7JSkgbyUFrQSn828XwoTduCDKxzb/Vwts2zoA1EJiS6aIZwc81xiHkjREXbjRWlHJDXLcZTK89xO2SknWt0PbvO86rKk/OoRtqtPFPG2lBVCtFgSIKUeOgNzZglTbT/KOfPIBGSvBoRvpffpyFSjhcXFRy7MdScKsl3/1Z+57G4/xy/OOwZHT3kb6Z4ctryfjCF9S9bSS25qCcv5XzsqeO+rVxed/iN6r6/bvvbWLKty6ZGIMzvScetR3kvlvOlOs6Vb2crdf5KJBY8znlcvZ0OIa3AvBnGW/r2SZebxrgVjlu7PN4F/K7fLN85RcBfNlWpnFfLfX+ja11YsgY+VvWb5T4ezds8H+FF5QTeHczbcr2vZf72+iddV5/vwfeDHnjDHticT8PXr6c7o7Oh/t4CGbhwZSid78eTYjU+d9Yepsl7N9Nrtyew4WxJx7rPpaGnh/Aa9QYGTozP9dHfSh9/icW2bVSx3/m+9HR/xkBvWI8v4sSbE4vrN9LN4fv5ia0y41mpTaYecmItk2uVpPbbxzjOVye8tmzx3ae52nwAyy+/10iF33lZeawQi3B7ygtTFZmIvA2IKJG3dX8qsfC3Egv3TyIWupkNqQVgyX3+nSUWWWJAu2LiyZOPk4u/VUvZ0viFctY3sirUJgBAiQUVjNYEGQAMKP0oRMJI1OW7+2xrwTHBKABbicWeKlRFKABE9dKLYmeh3r2ekpRYLLrACbHYcLWf/lAVSmLhCrc2FGcBDRKLReo2DflRx9v59RD3zdXPGYDObYDkfVRfZKg9gJQrgI8ugIer5hIRSYk6/RpzH+eYUo2TABzduToxW5YTsCu53jd7CV2oIFzGt2gAnEdsC8CUuuqCrwWOLUAinLY1FhZ4a0uySdt8OQguHD9+n6SOYxgtj0MQdFebSRsAzrK4xpXabsCSEo5O2qykRiNuY3hMAPpcZb83jX0BIDdch1JGuPWFUBwWWGGfcBlAfhFi0aekg4LXAGcTgF4J2hp1EJyN00/jGk9DcGzlESQcp1mJv4j6zwASimO0TWmQYF7ydZw6KPXZoJ2qV7XSTu147gDYX0EicH0EVSCAq0blVwb70zMYJveRnxIYAaEvJNWrjHgtgZOg3NHOYnI6vYqK0DztMS5II0bPJ7u7Uz/k4sKpk8TyOBKxCjroB58pY3dIIG+R/jZAfPjWbdRpiL/BdV2XL0cwQSMzGyPEtqsKdRdXubchPxLzftpnVHK9ad3EgPzGzZtp4g55jNzDor07dRFv5Oq1a+m954gfQj6CW2OASMLuIXW4BalzHGXjZIgwhHMWacHynTtpi1X3Zuw0zmKjcQFVpyGCDBrUR8I4A8C+AaG4Dzmd5B5K0lRHO8KY3pieTusA+1VcxqZ2bDLI4wjk4izt1+uVtjYjqH5NA77nuR+NtK2NRYE27KJap3FxOzWVVvBYtoEq3jZxU3a6utMupKUR8K5qluSwGQDfAIBf5b7vQBZaWe0/BgG1j7bIZwVJiupQ+N7lgUICwXXGj5AI+F0jcgm8d6HZsVpJLZTSmN8OaYJcmDfflSgchxg3M2/pQncZcrHGM6v6VCNtdqweoX7NSk15JlSXUgVTcqK6VxvtlpArh9Q+Q6nUKnuJjOpU7TwjuhvuIi+N9I1noVvmPvvG9jrfca2DJvZ89cjezOzJ/VuZ+0sa5+l8bckhgyuvKmnrczD9vnPV75JfDGBrUWVnHvkKUuQv9dntz4tfOX8S+t8LatUin8iA+pa8H8upPnuTlLqW47WsHrvu7f4s+XrdVyKxsL/+Q/XV2+3bg/QHPXDQA7/3PfCmxOKzEIsbEgs3JuHahFpmYg4HcXjsd6Qv56ofMTF7rPodeT32ZvCYk2xWfdojF4VYCFrLJ4xPWQkMo21ergKgNT5ZaiGxUD3HOAuoR3G8eGxSYqFeepFYuM/fJRgVseAlqxpJuGKsXrbxUoyXWkUskIisK7Hgo1codfjDlz2NKFKGMOCWtPiBwGT1p0pqgXpD/i2xgGgAtjOxYA8wCzuLal+z+4BouMEpsioUX3Q1u46FJ/AyDLMfQCrGABS29zTA8hxgQqnMIkDP4G72v8a1rqarYmUch1uA5XsAN/3wdwOGLrvqDPDQReYD2jWPHYJg+Rj94+r8Gew2tMMQiEgoNiEUOVCfRr+uksJ2uI+y6A7K0Rj5CO2lkUGCJAoT1PMh+R4iD4mAalmq3QS4cTw53visUqcxwPMd4jEMo1qzCAGSWIZhO3U02Fs3UoZ+pSz8tiwN0pV26PJvjjJusHKtMfEIth4bqJgwIMJVZxtGrJ0Y5J4/RWA3iMV5jLNVlRK6qLJjhG4lKEZ7lgLptvY+eU1g32Bcg+OUOxgr/R142snuTR/SNu0q7INQ5QL0ZbKltKcRI9wWIk+vppvYKrx0fyJNk1cjdTyLGtFViQXA3PG/ChCnWwGardiFoIZD30gQtNG4iTrU9ZHRNINK1CrSoCZAZU/v6dSPOpTEogfgaF9K2MpzJ+W9N0/sCIjFrRs30zrAVRWaU5eHKLcfT1qdGPnjBYkLtKWYpI536G/bo6vZswBSicEIfTB862aaHb6bdsfuE9Cuklg89VR6B/U/CzHSxkI1nCWA+TT3+RZ1ts1KIVxt3yRa9TpSg637I2FP0HAUaQNSj27UsvpOkR/tPUz9edrTbYjXCGRqDCkPegYxRlXb2cVlrOpFYcDbgaEZUocOJA66tO0AoDfiaGCMfpqGFASpgWyZY9syHrymp9IupGIdUrCBFGQLYrHb3RNSi0a9XDG2dVxgjAmNt/WyhMZdqC7pctjyG3z2lVhILCAvBtMz9sQ2z4jEYhcpVBiUc5nPr+p1zbjn3aQ9irC3efZU0dKFLQ9+qCsdJo/jh5GW0Ee7SiwoV6cDjiVJiC6Q2yhfSZRSiS2ec9Uwfa6aIBwtnG+F/Nn/DYyfNYjHsupXkjqeW90cd0OMNOL2+ThF+gGemwGegyPUPdzPUt+yOXYqqsHojxm7nKrtC1nwfD1ALtdxp3yUYytpaxfzxVOWU9t87uNodSaufZxY5AxLvrVrqy+lnFp9qoQ53/rSzCfnVX+0Pr98Ntexll911RtdU3/9W32vz9O0X4nE4q364OD8QQ8c9MBXVg988cSCdpdJsEzcZW+X+P3x33HcP9XEXn8+H64m9YqY+KueVAjILFNiUX9c0FwMt/UKJXHIEopMKPwuoVhl5W8fsSAvX3ih/gSQL4HyMrngxfo4sQAMRJ2r+vldiYUubze2lVjwYgdgRIA8ANgu4Nu2hk0EYEIg5gp1SCt4kYeUgr2Ewki+e5KLTCoijoUEA6DhSrz7ICd+B4gE0bEOlKEq1KofSYWqURCLKUDwCCB8GECh96YzAIiLABGJzQJ10/5CUCRB0BZB485xgMkt1IwE7U30qxGSBzAiVi97iRtyn/MawW4jYRD8nweEGETN1XjvjWBXKYPgewEA4yr/CuX4vjaewgDEph8iojG46kMzgMNXAZragRhvw3r0U+YgaY1RcYy+UcUqMqCuGh1LQm7MLYR9wdQkgBCVEjfdkvajvjMIOVBqIJDXq5MgTgmIUgINpW+w+v7yxAz6+tN4ISJmgmCM/jzSeSL1QCquAOqvdOKGU6kD9XQT2GsA/xDDbr1P0cUEVltMY9iLjIxNhgpVB31xAReqT6HXb58I2HR/G0HlGA/WR2KgmpYjR4nPYfpwinN3IUifJ59xvCytLa9hUH4sXcDbkTYCBkB7RP8ouThMX1zGFaxGtwK/EYDiTerw8siDNAVJWAE4GxH6DORgAHev5+mLWG2nrIBo9J8qLkcYA/O4BR5Ffej69VfTIiv+ujjuunAxPX3pYrqI3YKqMfa7K9xz1HEEMqC72iWep6O0g4GH297l9OD2rbRw/15qZOW/uRvdTEhB/8XL6Rx1P00fdtCHgi6lFo8AwCOQn/vc8/sQhHXtApBYND6cSQ2QC1YGsG0gUvbpM+kw4L6de9ILOVBd5yR2L0o87iGxuIf0YZOxReAUiBh7bRu4h7g8Sk2Mgw6IhcENjbwe9x2HAg+Q0Bj8UDWwJvJrQz2vjbKbUKPafkRwPEjmpsQCacfuyR4GLMH1sPdoYjy2kbdqc0Q8CUNwx4PewJohX/o8a+K5b5BUUL8gOEjjdnymSLPN/dshXUQSNx/6Q3e9LbR9EwnLKqpd2wwoDcZD9crnHGCvil8HbW5FXauBe7Vq/RgH4f2OPBsgHpKL8Irm3KiqJ+PBsaxqYSP1VuVLVb82yT7PyTJ5eD9bIFvt9NUJnjOlGqo5GsviPITxHGSwg7pk1cYMor1/PsN5/vMHPxlLZfN4/e8M2vfSOOLjWjOJLXKMb3Ed17vtHd27NgqO68rZktY8S0mmjyxe92d/3UhUJYzZP7Iq+ZZzexmVNuW679VpX55V3WtpX1eD1x+opX2s3+pTWsaf+JYP1R/6Xf3uO7S0s+zrCyxtLot5uo/008VzcrAd9MBBDxz0wBv1wJsSi89UqlBlgnlSJp4rn3I+z92+WtjeYPZ/Up5O707AAudCKuJ3jWDwUqnOBbGAVLjPhttZYhFuGCEVEc8ibCz0RqR71Az6NW4salDaPhSJhWo/NVUlXrpKDQK4VO2rGsObUFUdwPQ20hANliuJxRbSkYiEDPiSBAjmXKms5QlwCMkEAEP1pxw0D4JBHUoMixwYb49YqAZVU4Wq1UNiAR7jj0B9daeRwHIQCwjGPOB+DGA7ykqu7TTi8SCg07YIsDXQBpvmGBaAD3oEYoE0AHCkoWsL12iboHHzBteY3wQgfBmPTI1c28+q6QWAvACS6gSp0L7CdXHtIIydMfJoJUCxfdEFqRkA8J8F7An4VStSTesOgGcWoKZRn8d7ITHnjkiCMCYFoLlarJqTREr9Wdapw95BV6s3hsfSHAB1jfwPYVtwrqcrDbFKr8RBQ2+Jk1IPbRNcuV2jDfcA3zc0JEb9aAqJwwq2CdxAVn/bkVigPmMegNnzldSjlTIdg7pcNdDbNGDWTYN3bRNu4eloljxdde5BYnEJFaRL9Iurv0orHtC++4ByjXE1gu+WYLDXgFjyqvrZMID3cxCLB5AdV5U76CuNlwcgOkZFnqacKcpYxuhcb1F6mjqLqpbk7R4r3p/BjesYNhpLgO3D3ONzqCpdwMZCguP4kOgt0t+qYTVxQG9fqlQtLTxM1197LU3ductq+EZq78O245mn09ULxIiAwPh8SSSmGRfD1EFSMA7xiQBuglVI68N7d9MaHqGaAPfaV3QPDKS+M2cZyxgf0+9dAGRjJqi6KNHUfmcSgDvFfVucQ32IT8v8DKB8IVy0Nvb0ppPnLqR2iEULbTnC9doAnAEET0JkRrDJuDExlZYhdlv8bmDlX+KsdKG551Q6DMHrghCcgdQMQi5ceBhdWEgjk9NILFApYly20fbW9RXKJW4G4F5vbpLzbdSodo5iY9FODAzya+A+NDIOmwD+EjnVCI0wvkEeelyCJYQDg1ae/SbAv4bbEXWecetD4by1yX4boK/LZNX2JAy6BG51roBUrKJCFRJXrtkmjdIIVZW0yfA+tSGxaLR+tHWLe6SDgE3qohF3A3kd4XlSlVGj8DXaxy0LmyEjwzcwt+h5TqlkA6RwgzwcC62QiGNKJqi/83Ij81g3fa1NzEWImepRh+L5cT4vgN853NTOzG8Asstxk5Xv8YUynGxq2953c/OX7wA3x5yfsnk4//T4Xro4X5cuMsl/yqXVfn99y8lSnr+lJ6VGtbLJu9SinNt3TSm7qnfJt7Yv52sH9vqsHHrSe89z5fiBxKL01MH+oAcOeuD3aw98UcTizRpXJl738akSl+PVG2J/FmVirpvIY0KvJubHiYUTfxANchE0+L0Qiy0ApoG/QmIBoXC/DggwAne4m+WY53WNKuIScCtF0NWrkoQ9YpFVlJRmCJDCniHIRW5Xfgvx4uGFrMRii1XFsLFgZXSTj8RCAEBBAbQlF4KS4hkq9hWpcHUyiIWSC8BAEwbcRRUq7AcKoaAuQSzIq9a/9J3AUUNe1TPWcDWrjQVUIjzgKBEQuOrF6SQg4hRARNeqehQypoKyAF3CKkFQu0O7i3sA21GALg0LCcIhQCGQKU3Tb7MrEChAYRNguZvr+lkR7wNIt+B+Vdxgf2pPsEafqGp1B+nHDPnp7eg47TS2gx6BTmJQax+q6z8KWFKCYtwNDcaNZNwF8BsC1J0FAAnGtXEg6wAnkj5XnSeo4w3A+CgSCCM4awDbQ97nAPXPAqqN36C0wjHSTtnt5Om4mhIkA9JfgVyMQiymWQE3NoGESH39k6gBCcovko8ecyQ7She075AkGIFanX8lAbuMp/sYTxubY55+aQPQDqKCdAUD6KeRfCi9UYXqBhIJvSop/5BY6Oa2h7yVYISHJq79DAbSI0hSlsj/BKB2CImDqkzqv8+Rh0bUE9gntCMdUhJwAeDvmDL2xedQ6xrm2kdc24EajNde6sHdLKBYg/uHlD0OKTH4n8+KKmldSI62VpfT7Vu30+SNG2kFoN7a2ZUuXns6DV2+mK4htdB+QDsR46DcgWxKgO5TB6GYMRZ2Abxr4xhMs/LPGn0auHgpDZwbRCXsVFqgTJ9dx5aesRwf45CTSe7VJGNiGkI3j23D2vRUap6ZoqAVVtnxWtZ3Ng1cHoKkQRK4H+tIx1xR74NcSFwnuO46xGIKQ+4V7l0DhEZ1pUPct/azZ0O6187z3Mk9P8t4E9DOQa7HISRKmRYhAM0QgWaITDPEYktbB/poHYnNNoEYdzTeJsK7HrbCUxPPjIBaQh5GmNzzbfpDyaQG7HpG0/6iifojTojnqxFpAs1lfEiYmSPIe8dj1ElicZhPG4sRCSnHOupvEauC8jYhIDsAfNWmmnlG2slEVagw4Kae2m4ZM0bpoipWDbRbIiWBa6Uea4wj1cH0crbNtdpu7PK86KGskbpsS4poaxt1Ocrz5fPh/Ol96ua3xOIS6ndKiJRqqa7nfbMt/s37+MHz6NPEUZ6N8j2fcXTkc+4KaK+9A0qi+tzIo+RtXiU/Dte2fNzyPFRORAE5zb7jtcv4Yg3q0lWnSt5VTfcusF1V28q5Ulokql1YznK0HNvLJX+r8imHS7vK77faHxCLt+qhg/MHPXDQA/+x98AXRSzqJ8fHXxblt3sn47J34t03OXOublqOfin5erx8d4Ivv4NM+LvuE2pRAOBMLLLbWSUWQShin6UV6xWxUE1KYqEHKQEnvCDAuuQhJBa++HmhCl6VZISEwRe4L9jHiEUWwwMaNEwOYpElFjViweqw4NyWh8SCfJoAAJaViYWriflTTywaIRgab2dykclESCu4vj5+RX7h81Knb1lsraQWmVjocnYZgLEI2xC8SyAEeK6Uu+Jv9GaBn4afAgiBtxILpRjaWdzFKHeDNAK6Q4CMRfp8ImJAIPFhhbwRgKYrzC7Vh3C7eRJ7BL3/HAGJKmFwVVRC8xrEYhJQvQwoVgJyHBDUo2EwkotO0gtcp6wL4NWgfUoljG+hNOA8oPIyK+4aUAsuNWB2NAiiSIax+la6i669hsWj2IU8Ih/7sweAdA1ioPtM3awahV3DZYmGxtMPJSUA7Fus7mrIPQbgXGPleBfS0EQ/avQ7iI3CeaQPg7SrA0KiNyRV2Vy5nwIY23+qszRCoGYBuMPEYRjXqxPXnxk4k66cH0zvxrPTIdIY6foG/amh9xKA9DD93Y/UYQC1E/OXdEqMPvtgKt3BQHqBNN0QlGsQk4uQB1WAjHg9w30xGJ75SXjPIbHognhtUodXp+fTLcjF/MJKRFV+ZuB0GjqNO1b6TtKpKtNdrr3P+WWuP4I7r5OcM7Dbg+HhNHfzRlrF5qIRQN03dDVdunwpPXuO+lOOQNSggdpzDNOGEe5nrNhTzwaMlhPgXANjSdnVp66mIYjFUE9XmoI8WG/H6SAAX5WkKSQ3DwT5qAFNsXo+hQrWMuSiaWqSG7oKmMZmoP9seurqlTTQdzp1UkeN9He4/zoJOEZfLACOX5kmJsrUDEUDypFGtAHEj2GPcRyDb+M6GNW6gefvFONMEL/FA7BIuQvYKjwCzO9S54SkpOHhPIQRl6886xvES9kMo22C4R1GWuGzS+UlD7vco9joS7w/BKHapg0Sf+lqE8fCzSzpdyUikAPbvUs9NlGfY0SjGsW4hfzqGtk4Ekew+2hcWUrbjOEt+mqDNq5TD71SNSBpaYH4tVNeC/Zb5m2wO6UhSv8kuXow01D7MGnbyc8xatRv50mBt3MiAYkhINgYMU58/nZ5FpQchbtZ1J1Ud9SzWhALyJ+qUBILietRCGzx9EZTIs+8jx97c7RneCBrc3Y+HX/3jkF2Is3eyXgj1GfoQ23/+pS7Y/OQbcnH8vG9Y3HUBLE5l0fS/Kc66i5n5txR6mOOT6xzqUNVgVqWZpMLJjtz2r+Z15tupUEkKnV40/TVyQNi8cX00kGagx446IH/mHvgzYnFK9ndrA0oU6svByfV8qmfNOMYacve6/zu5sRc0sYk7cRbNzmXc5HWibz6+AKUEFi+x4q0oniGyhKLbLwdBtu8iLPrU6QXgB+jQ0sufOnuAhzIJIy3BfHZKxTEAiBRJBcRtA7wIKlw5bN4hvLdZXWtRyYWSEnwCrUREgtUHAAUO4BPQUAjID9WO3nBh0SC/IOwsNoYe8BBIRYSjgZIhWpQ1ilsKii/GdezRQ1KkpM9QpX+rKQWgI01mmTU7RW6U7ezy9pdCNSpayYQ2bZhAUnGIkBF2wPBt2DXmAy6UJ1A3eXVyZnoI111noQ8SCwMtDe3BMCBWBA6O/z6dwFCzuK55yruR3sA8kcB4azzAvJxZ0r6lzGyHp9dSEtc00JfnARQ9wI6NQjvxYZCsvOI+qmapY2IBtazlsMH2QrRp3H5Ctix7rqH1NOUZES1MonQBCvgD6ivKkrjgLNFQCVJWCFvx1Urqk0AU++ShtxGHHf1X5UmycEoJEHpzH3qNo9a1TbA2VV4icWZHmIwQE60szgBwDrMCr83XPUuQfaSpJHfqsE9ZAV9YuR+miFGwzr5GrjtaVb833NxIMiWq+ETkBbVyyZRIwpSovQI4PgUbeuMlfjt9Aqk4ubMQ4LYQSwgR08jcVDtyT4S/Bp/YoZzBsZTT/44ALUXqYeQ9xbSl9uoQ82wb+XeXyUi9RXsPc4Rb8JV5xXS20e6zJ3mHitJUdS1xqr9HG5it0bvIbVZYHmcmAZ4ZLp4+XJ6x6VzQSx85pRwjQNavd5gd0ZyXgGk79D2RuwTWGdPR1FFewpicfXc2TSEJEvjbG0R3E7Rj3qHmqfuYwDyyIf+ngLkL/NpXJhPDdTPMd+KZ6krl8+ncxBWyaFG8pLcDp6TAWxfhM33IBMjup3lswwhontSG16cjmPwrbRwmbyWULVSLewQ46AVghGu7Dm3icH2MjEvNqen0s5DbDUwvN8A8K8fIjbHSfzTA+yNti2o9mP/bTCodAxh3iiW400MT03MJ0orVEFSVUn7EL066U0KJB9SA+2stiGxu4xtbTIMtKfhdBskqY0b18x80UgfbtGvG4xd1Rm39UZFYL/D9Fk7+TczZ+0qBeX52aEOfjYZfxILDcT1DmVMC1W1XFRRaugzuMU4D/fQTFbaYUjsPM7jE3YXqjpp9K0xOKMhxtw5xpvP22nJSkUsWFaJa0jyxM05vX6+LomcGyPj6gDJnG5rW8k10tWlyTU0rWSkJHe+L9/dU2btJ+nK96rIvevKiVyVnEe+Ml9T5eKJ6iKP19pTX+m6TGvnSft4+2t1qYqOEuorX1eOSerzqi6p7Q6IRa0rDr4c9MBBD/w+7YG3IBZ7AfJsX54feT3wpRAGj5eJshxzon38u2mccMuk676k4auZ7L04/F59dK0qwPR3iPBBFIVcZM9QqkLtEYs9lahMLDK5qIgFL/F4CatqJHjXxoIXdRg7Btj3GCRAYsGKZSYWgvqq7eytdxCLHYgFq781YiH44KUteWkA4AuEXfGuSSrMlzJy5G1JjWpQ2l2g88y5QiyCUFC+9cueoTLJsRL2VxhwWweAlSvTa+BdCcUK5EID7lXaqNqO98DVZ9u2zMroAgBJj00ncGd52HKp2yOAiIB+ilXpG6j2CFh7AabnUe1ZAbQMA4pHIAorrL7vIrHA92XqBoScQ2XmnayQ9wHgjqoSxXVKD8YAVJ+HVGhoO4fKURP35ThE4dQpiIU2CACnboC+qiKlnrqElViMATp143qKVXDBjl6e8gqqfaknL8EermddARe0s9o9RXka5y5w/WEAajfSkeMAPNWYtBXpg8i4ci5AcwV/AoIwAhEwvsIsKjVb5NVMR3agTnUWwjSIrvlZgLuSDu0zNEzXxmSScmyf0dBd/Z0BYE+PjhEHgT6DrFy8eD49PXQxPY9XpA6uFZjPclxgP0YfanwuODbfZyFY/RU5uA0puIVRutGxTwA+n4HYnHfFHSYl6dO4PYgFeUkUlMIYc0Cp3S1Wve9xz2bJQ93/C2f60pCG5F2AbcphxAZplJRMcd/mSPOQtszT7nmCw6UH9yEIGB/TV70XLqXLV66kF4glIel07GibssCqvIRMT1CTjJFZyOUSAH+H61TD0QvTU0NcS9nW+xHtMLCdsRc03I84KHyf59lQKjYMORnlM4/kYgf1pF1IXhMd04jk4Dx59GK7owvUB9rQKGXhnp/BBkaPag8Zv3OA+QVUf5apwyLAHAMNSCGgnnGuB6QFjNpdCAjXq6hJHTtKHfjdACFYICje2sw09hVILHi2N7BXkFjsQip2UUPTOxPKehCL7Nxhk04wPs421+7Sf4VYNEgMIFxKFHZVnTLuBXVo4Flo5FnzwdyiLjsQgYC/qCAZM0VvTbqdbUIlqwGihR/YCGS3ybO5jbSqCVe22k60Kwmhv7Yoh8c6z4PkaaDPbcrTFe4RiKkSLG5djFGfP++a40ApnyWr+qRdjM+N85njuZULXJBxPnV+C+cKPB8ab0sEvfchsYjRQ7+xfztbbQaPST7PmTwKb7rZBseqm3N8/h2/+B2Hqz97xGLvW7w2aon2rq0dijys115bqkzN/EkXeGl9wXVprJ+b83D5Hgc8Vn3ZV+WSD+nLeZNFmnKuuq7sDohF6YmD/UEPHPTA79ceeFNi8duvQCxQ+XAijY+tLN/Zu5UJNr8U9qZPv8WnShdp/cNmWidX86xt1URbn5/fg1hIJrymkIrYZzWoEP8HsdBQtfIOpStYVv3WeDlnOwteygCzXQCttgeNrFhmYsGLVGJRfTSsVYqxRywkFbRdcBKNyfV2FX0LYhHG24Cd8AxFWVltAVBP4vDoJLkwvyAt+4lFJjCZWAhIJBhNSikAyFE+14YxN/sssfDlZD1yz9KUUI8wxoKkYgXjbaNv6/PeVUtThb0IYOQRIOcRwMygb9pcuHKpepRgb5rV/xlA4zBqNRp8niP68rODfWmTOt0DFF9H5WYOYqE6FJ0Zrl1VGXrXYG+Af71DScCMRq1rUYnFLVSExiZnAI+rsQrb2QVg7sFzE4ThDCvSza4YU2/r6apqGDxDEF7FfkLpRC/qUEOU0Q/YUfKgFxvvgYB1HDIwBrGw7hGzA+JjIDdB1QnA2QlAknEhtCk4xbVDqBa5AhuG46iojAESHwDqZtVNB6ypGS8hOUeZgwC7PlRIBOYGU1qiz+4hddCAWZUxVdoch0aEnoVYrE7TZ6xMXx1CjQhi8TSqSJIhx622DaPUU6nPKGUacVnpztP0wTnIj16HwnUr0gBtJvRG5fWSKleZde+rXYl9OoW9im53VWuTXKjqdRNiMQmQfki/qdrSi6eWi1z/LP3cG164ANv0lxKpGfLQSH8UID8BaJ+bnkoNEw9Sg+pB1O304Pk0NDSU3vPs00GMJHJ6dFJCZB20k7hLmyV/05CLVchJM/19HPB6ceBsuOs1kvMiRFXyqlewiHBOPxr13fs8b19S/l36fYpxYXwFAi1kF7Lcg9NIDjohJwZxs4w17lELfXsS4NsGmdpmnDDMA+yv8szNQl5d8fdZaWa1fZuxvYILX12w4tIsNSN1Ow6JE4QzFHBTTPR3PjsYrzsPGN19E6mFxEBXsaoYNdGP9r1ExngULGWgSoT0gLru+HyTdyMRtyNuBf2iR6ddYoHAYFBjIgaIzzBzg17LtugfyaSqUA3k18TYaWbO0KNVA/mokqSE03lpEzuTFvI4wnhvo5xd+kX1Sq/z2XdTnVMio4H3YerqPLbD/Y1nnTlBKYu/Jb7OT8bF0B5HYtHKmDHmjBIOF2XM0edOkuozpl2KcWAMeqnNk+edZpxvyuaY3jdfVyc8VubsDJn3X1Ouf/0+4HVdnjmfmN5IXB/MLpdRCoyT+7KjarHla/fK92CuW5UgUlXfvagUFsfr/pQMy6HH0pnnXptzoiiVdKX0KMV0nLb+5bzX1dJU583BY577zm/9Nn/+nmy+O8s9Lfv6gnO/7y3o/YfyCjWxMs/7SLutg+2gB/7j7YF11N2/HNvJ1mMEN8ZWkLn4y7Vpf/ykOeGLrc/bJxbm7ERZN9nGZFn3u0yi7uNTnXOSrU3Idek5WDvu+ZImyAS/3ZumSCw0no4XKBOjwe8iloUifj7FK9QGgEBiEb8BNZlY8Konq5Am8JIW3Aep4CVc1KFqxIIXcVGDUkpQquurIogFdhYSC73LbKJ3vskqo8GqVIVSbSmMsHl5B7Hwhc3H8rLrWYFrkVgIOgDnfJoaPa6EovrE90wsVK2yN+13/0kswBEBOCUWS3iH0jRbwlCtlQLSuYb089RLdRxVIPoBE6p9SUhuAcjvYxMxwX4ewCmgvtB9PL0T70Susk5iY/G5qfk0zqr4Mmn06X+c68+yivzcmVPELCDCtcCfegqABL8aLX9hdDK9en+cAGLYMVBWu1IO4zSc6iK6dQdgJ6/Ia8Rs/VyNnwOM3YRYrAC6jEA8NNAXQesGWN3Xb79gV5IpeBWw3wPsjgDepyhvDsAr+OsBSJ7pPhk2Gm0MPFVPugGqEilB8qzXArDHALauxNuJqlydJo0Rty8AsvoBWHpPcqBo+zAKqRihLMlI2LbQxjmA9RTqRMvo/R/hXj11ZSg9d/Vyemd/T9TVVeF56qbRslIcXQCr+uWq8RX6bAAQ6urxHUjFPSQ7j8hfWxXJVB9t8E47ziRfksJRyNQCfX8E4KrHKMf/Hfp2ASmShKodYGxMjdOA0/OQFqUuSk58Rh6x8q8q1xRgX1I1TdsfYfxsHIhNPmtIH05qwH12IL1w5XIYl9vXq9gkORZVbFLadYs6agB/H4JhwDhJ5mHGYK82N6zYG5TO+6nEYplznfTrKUB9H4RNRUYdB9yDVIxw7+z7ZZ4XSc0u5KCRcXaUfj8ksKb+ixBdo10LwFsZb7p4beDTBKg2ArrtWkYlahOigjiKyN3Hose2UDHaxn5mFxJnbI82JCrNAn6kIGvMBy4ANGC8DkpH7c/FCgYJz74G0aoY2abQ2+e3we0aZCTUvRF7LUlQA5IqRCVZPYkFgJ2jRurOAfWM3u4iRYxt0rrIYSwPyYDPBiYuIcFrQgKCmDPmAsefcWA2uK6J9h2izBYkObqv1b7C6N1NfJRoukX8HZ4FnSVwYcx9qlIxYcRCifNmniOzg4o8zzWEPYaLJkryNPSWYPhM6b3L4Hj99PlJ2hFey0gT8x3lOc+UrczJ9fN+OVf2JU35Xb8n29pW0rmP+az+ZJWqpKld9LovtLtWPzP39/6N7NnK8bLfn+aJv7jwSalL2+3jJ23WoqTxfGnDvmN1F5bzdYfSd37oW+t//q5+/3IRi7/+Kz+Q/sXNj/2utu0g84Me+P3aA//JuQ+kv3L+gyz0sPD1Zdr6UDF2cepL3b4oYmHm8QKoSomJ8rGXQZk8y/vjdemra51Ma5O2eeTZP86WiTbSmI5PIRfxnQk9jLfZOymq4vI4sVBSUfvwAl8DwOglSgNhV/TcBNy+dIuNRRsdKMGQXBRiIamQMeZ99ZKluoKkIrHYhFgUchFB8gATEgsuCrWZMPQEnIV0IvLWtqIiF65u8lEdKkgFQCXbWCgBQKIiuaiIRTEiN9/crxWxoH+MwL1EkUuQC6Nv6z3G/5lUZHUpV9t1zypM6gXAtFLWCn34BdRwbmIIPQap2ARU9aA6dB7S8OwpYinQJw8Bta9wfgQphN51NARVGtAFCBw6fTJcybrC7+q0Rst0WEgSXsGF6sv3HqQHEAWlDK0AxgHUp57GJerVUycjD9W4tAWxHrMA33EA/D2kJrpzbWFcDA32h0H2BYCpgd7cJBbeQdsj0H8Vb0WjtGEa0iOxkPBcZtV+CPLSQh8IwDWiFyDbb0vURXB9H0mMqlSOhyMc7wVgDSEpUKLiar+ru463RdKqiqR05D6AeJk+VNVuGfA6PXwX9ZqpGDcDFy6moUsX0rMQMgMLCszW6DulJAJqXewapM1xr6ejLkCo4OQGUocJVt7XAMjadVzEs5RRwNe5zjxcbVZlTZWqCeqsFMMx6tif4lo9/ng/jLh93PtAOzR4Pg2Q7iA/+0v7EvvLoIR649LAfpXnYcdgbRCMRe7rUWI59CMxeMdAfwRI5DaG6pjrNdpmKPkYpaxx+sJ8jOqstKeZNh7juWlHrcg4F6qqlUCJBlDUpsQo4D6n84DlEQjgDP0Xhvvcm13ySpCuBvrX+2XsFOM0rFLWDuMhYlV47+gvjZubcM3aQDnc0Ag0pwRBY+lGVtwDY2pcjZqVpO0QxKIZwtXItXp6UhltU4cOLAIo1ciqTdBwnsdd6hrkgrziJvF8NHCfQmLA2FM60MS1zeS9DbGSOG7R5h1sI+I6ylOG4L1ROqmjiPhwH00rgQhvdCxGNHLMQJS2NeZMzq1xvV6dVMMyfoV2HRIOy5fg64pWCasE3o/qXY3kK6lUZU5bDhJE1ZXGOEfSa3muIK0R4pX6SSzMV+mGak8GUtR7lx/VonSP7JjzaSNHc+Bv3nweyhb1Lj+qff35x07lzDhYcjOrnD7nGXNaOUm6uqJel1U+sFeXvVz3J815lHRlvz/NG/1607ZUFz0pTWlCrX+oRK1k71Hd9qTrPX1ALOo66eDrQQ/8AeyBPzDEwkkwJn9uckyadZNkmUTjfHXcKbR+Gi1pHCNvNKGWCTjOOyHzkVQ8mVgA7n258wLXvkJ2FFILfksisoeoPRsLj2XPUMBS8gxiwYtYVaFCKjKxAPhrjMxxgV0B9LlttglAz5JmtrEQZKonjZGl/ukBQ0oswuVsda3EQFCbPUNJWsw/k4tm1TWCXGSJhaQi7CwAOqpEZakFL/kgGXvSk1wL6kFfwilqxGIR/KKNxY6khvINsmbfaW8xB2BylVf9aVWhXL1cAWB8buZRuo7x8D2IRTNpNZw9h7HsVQC2kXgF/TdRBRqeWwwvTAKZFtp/DNDaR5pBVFcGWSU3sJrSDmMFLAA2b2KU/HkkFq9BLly9bqHMXgxzn8Ul6jMYfWtnESu71EG3qPcA7beRnIwSZ0JvUvL0obP96RnIyyU8IVlvjaZV+xBou4ouYP8CalC3qf84+x3KMcbG0/3d6XmMoAVRK4JYSIub0gbd245DNEcAsqo3bXBNO3kaR2OIa3Q3e5K6ahugka4qPRpta5+hhEQj6AUA8hbGw3N3bqdFYjlICjv7z6b+wXNR5+MAYFVPJKDZWiEDcz1yCdCVWgjeVBm6AamYJYjbLmNHFbFLg2dSZ2dnGHMrM9GYVgN5DcBH6Zcx7oWSJsf3EupCbex1YfsUHqkMfFZWnJVqKImh1UTRzvFFxmizMU5iLAh2+biq/oiyBcXathjoT4mNZMz4Ew+530paNEafsy+5BgoXREEbggZISwPntVOJtkp8GVs+P10A827qdKq9LS1DRFQ9G0FCswbQ3WBy0DVqBJTknEbc2hUIvgX6sQAAaTf6tM9rA8/KLnYIDRo5QxgS7VPNTmNqo3Gj78S8RHN5FlsZJwYq7CStkjlVkJQa7OCRSUK1APlYQuqgC2UlA2EnwViXsER96N/YgmwgxeBeNmFHZBTuJgE/dQrbLp7XMNqmDGNqNGxAYvmudMXYKw2kZ9Uj1KeKZyelLTGvUFmD8Lm40Uhx2vKEpJM+3obwCf4NshdSSsaAdhW619VeSg9s2p5YH+vrtc4D2xyzE8Iejfvh/Ml/2mdQRhdOIC7cH6UoEoujtOsU90dSoc2Pkj3HjNIMuzJ//Ju3Mm/Xz+X156L/OWCZtW3v8rpDWeUp55cTl2trifiS88kZ7J3P6UsZe8dJ78VKoNj03ueW0+XvdQfiawyYkpFH6jIrbc0Jv4i/5GMp9c2NfqryLzUo+305louqkwfEYl/vHPw46IE/cD3wB49YMPnWv1ge/15+O1eW7/UTuMfLpF32jhrT1ibdapL2fNhUsA/xvr/97guVfRALQQ8v7xq5kFjwco6PBANwEtILvutlxmtc0YtVbMC70oM2XrCH+EgsQppQAaNCKsKAu2p31J8/yD4AW5KYLLHYQh1KYiGAEBT5albSEaoIlJHtLHJ5uQxJRV7drMWwMB1gJdSi6ohFgA/JAmCg1qfkL7EI422+LIMTF8BXEgvdZApcBSESC9WTHgKIlbLontJo0BIqjaf14HQdiYQqOQaWcwXTgHZXUC1Rl9819lsA8LtIBvTqI4FztVW1jhOquaB+cx67hvOAaV3JhrEuZep16VVIwssjE+Hi1rgdPXgNErw/haqV0biN56AO+DL9dRvA/BK2HDfuT0aU43bq+dS5/vQc6lOXITrqfS/Tt4JxV9AlJeK/UcDlLcjRPVS1vOeXUYN6HhWt5yjDVVe9Ks0CypWMSLAMczcOuFYVytVzdeEPM376aesF1LT6MfgVnBtHQ0lAiPO4TlIwhlThHipJk6xYq6u/OnwnrU+MszK8m9q7eiAXA+nMufPpMNISoqGFhKhDgE1ePfwW8wgCVwHk2rtMAaZvQSwe4T61ld9DGG4/g1emHuxR9KSkVyhXlc9yP1QvmqKu96izalkT3AtV146SXz/34Nr5ASI2432IflHaoZqLeviubmtjoQH3MJKJGWw1dAcrMPY+Woa2FI6rTlb2dWcbXqkAsxIygxgqaVEVSkPyZa+hYyWpW6gb7dIWHoJw86rb3kbK9FnqYGxohG0U6XZW/B8ilZhhtX8YMmQshh3ubzzvAHIeTAYwtha0TYNmdBZZqQfom4YVfvPfVYKALUMLpKsJCZbnDVq3Aynh4UeCkVWFaBhjI6VOJBWnUM+SKEsyJHPa7qxARGYgFKOosM2iSrWkFM65h/yVgjCZhLqfkrkIjsl1rAQEcYn4FfSv5EWCrUpRMqCe1zEmJThRb4hPVjtinkFHt1HVJuqpzcW2ZMW2VZKI8ERHOyUg2lXozlayG25ltc2guB3KlPwYJ8O+PcT4PEa9qFXYaYQtDMUrtXCu8sGQcMRiC/dJ97MtXHeYj26bqXg8P44PyV8/qlBnIRfZVTNzIHlQo9icx8o8vTf3VCerneet5962N5dnuF1/hu/Up+Tpj7iWY3Eikpbrc6b78/baSFT3pxwwff5ef00pq1Rx7/pyXb6qvn3lmrpCqCc5VBe/7nwpsJznQst7qzzr8y/f/yAYb/9Xv/Z/p39551dLkw/2Bz1w0AN1PfDnBr42fd+59xPf68unCtXb2xuL4HXVeltfAzuxWNiuY5S6jcXD3d1ivO1E6iRZmyyr7/mtkCdRv5c05hOTat2EWyZ2z5WJueQbx/zjRllO+Z6rEQu+Sy489iRiES9RXs4atfopZELD7fxBYsHLtkYsaIngWwlCIRWuMgcgAPhkO4cM5mPVsNZeKkZDJBauoK8hsdAr1B6xQOEiiIVqz6pbZXWmiGPBSz2Tiqz+pORinyqUKlEAHN3dhuTCa8kjPhKLqn+jb6iGxMKVyhX+LEIqFsRgQAKDfCmt0COUaSQWAkPBoCpGJwA3rvob8+ILSAluoEp0D512X+fqXPcD3K5CLFRvsqxbAlrIhZ6YWO7E6FjXlTmKcw+rnGcFJuw1PhbwW55elyQrN7HP0EvNEco8DWg9h8qKRtK9ABqNnDUgNfr2A4Cv3pFeujcGYF4MVY+ncZ/6TB8RrSUW9MMywE0jb0mCAFqgKCkxCvUwBEm3uH3YLzwFeXkOdSvVOjZIq03BPCDZiNULpJdYqEY1RZvWWblup2/OobJzGU9Y/RATx6nemyRWrjpbfyNRBzi3HwCjjyAWa6MjaVNiQVyCY9gonDwzmE4PDWHIeyytcy9XAbkG/TtDWy8g1TGat0RwibpMU/4Yed1GnUnvWa2M2csYXT9/AXLS1ZnBP/dH25BeQJ+2MRKEce7BbQD4Xe6XnpOa6RNJyxXc/woS9bC1Sl4MhZDqSCDmIRNzPOATgHtJjappPmGukDsmJNqSYNWZDJJ28aTBDAG1HJdILJCfHplm6EcJkbFHlBgZZ0Iw7aeFgXKIe2z8iKOsxBt1W/uKHKBwNyQwBsiT3EAFon6xuEwZAeKpW4KwSSyUgujVKWwruA8blGsMB/y1psOQ0zbGWxP9alDBVdIL2l31d0xL1pUKtDN2e8jjIiRPGyBVjNw2IS1LzAn3pmbSBNGvVSczSnYQHZ4XSUwQFfqmBgq9lLZJMMLmgj7YZTyEdEM1K8amJC1+S3Aw6I4o2jwjzUhJmll0UGXLQHdb5CGBaeAZ36U854BY0OBJbaReuxCL9YpY7GpYTr01WKc2oa4VRthcL7Ew8rZSRtuk+pn3PAzE2TfyWxXRNfrS+DJ0TtwX1fQkJD53qkadoP6q/vkMO86OUh/HkE2OHnMCqLZaf5QD7Ms8TlX2bYwoz8b8kU/kjCpBQhyi6vRxVYA/qq1KyS/fJ+Xo3r4u6b4rco338vHakrZQlZJLOV5fx/r2lXaV9FGNugwfP1+r6F7Gcemb5Vnyfnz/B4FYPN7mg98HPXDQA19ZPfAlEYsyYZa9XeL3+t9OtnHMk49NuPnQ3ksgfvuHzUk7Ju5qX9Sh6omFq4Wh58xLVf1t1ZyUXGRised6NttXrLMoWhELyIDuYNU/DmLBC15iodGln/3EYg/Y2xbflfFuYS+52fr/2XvT2Mq29DxvcSaLM4tVnIpVrLnu1Pd2t4ZOq9WK1JIsq+VIsiU4gw1DMWwEkoLEyZ8gQPTDAfInQfwjEBAnkGDDCiIFhluyLHmQ225JLcnq9HTnGm5VsYpksTjUwHkm8zzfOuvwkLeu+t5uydDtnF11uM/Ze+017bXXft/1TXwkFtpZaMC9p9QC0KEqlO0u6laqhfjCVxIRxIJyirSiuJsNbzKcL8Qi1KIkFhVJRSEX0T/2ER+xlsHZBNdP+PJ0l9Vq0hvkT5UGyYU69upgC4xdnVZS42q8q8Er/H4DYnEHUD5DEDVfvpKJEQK4SSwGAB0CTonFfVb95wH/LVyjZyMNg1VPMrL3IGDyFIBFaYVERmNj1W6mXFknX6UNJ4mNcQZd+2FVY0graA+3lnYom6vhkpGv351Oc3j2AfGyCj+GWhPRpPHq5Mq79gUSC1WgHE+qOkkANZCeQuIxDSGxzRqifgRJh0BbIvWY8wa507vSHNfOArQF2UvU8QC3pZ3ozl9BKvDc+CixLAZQl8GOxFVh6qj0QrWUdY7pMlVbByOURywG1KC2cWGaHi3iiaovDY6dSUNXr6VtgOUabVZta4h+GqfNFyBUxsVQTWUVz1S6sLV/NJ5foN4sL2PU3p1ewV3tOWwsLNNx7xhVxcz7uEb7JRd3ICR3AOEGrjMwXCdlnQVAn+nOBtsSBwMCPubcI+7dslIK8jeSukDVHnf8aHMkcW8ibyVYqr51cy/HIWdD2Np0c0zwr/qWxOIe0gbJQZBU6mWcBphKBGHrBHj3I7nqBqQaNLGP8Sfh0TZmk2f0EdIFSdEk0gLrovTINnkfvdcNkK0GVuwlKYLiTtSeOpHE6CVplWs2Xe137NLONsaPD+IS6Veozzb92ATJUUrTSfo99m0QO+/fNfrS8alUSVJnu239DKpsU0iKpomAvc7xUCPiGT1gXBg523o18tvV/QbqL1BWpeiAMR7EwzEouaAfzNTgjvblAeU24OGpAYKm4TXDJ2wztCUxcrcEJm/k5XWkN3heexskg7Fh/Iod0tEb4ZVKEoIcJIiVThCsk8C/0/tFfVzPyvNeVldrhqzrhUsbIYnMOuVKZO27Lp6HXsainp/MQ1e0PRJ+pJSqvxkHRhVHn0tHSX4yK9Vld3R+d6En96YpyJ72lN+HV5Yez7nYi+Vc2XsNn8q1JQd/hsSokndOE3+jHjk/LytXeMQ8a3+XVJ7Jxy2/1MkjR6+3TO/L0TxKTWvP1abxeNlqj5dj5fyzzpU0tfs6sajtjfr3eg/Ue+DD2APvi1iUhjlJllejbxN/l4kzptfKJOvU7O84Xy5+xr52si3TucfKcfdBLHhpF2IRx/it9CHUoSQWgJVaYqFajGSi6m4WYHQoscgAuxCLIBW84CPyNi/dAP28eI8Gp6M1to3/B6yyWSdVi7Z2AQFBLDAIBTyF0SV1se1FYhHSCsB+NX5FEBh/Z4lPTW+lAABAAElEQVRFJhV+5xjpqqSiIrEIVShe9pKMADiCTvIXmEksVvmzQLTtefabuHHVc1A/oMb6m2YD0LVKX4TEAjAi0LItS/Tbq09Y/YZcGB1ZzKP7yXEkCtcA2v2u0tKQe4CtyZXs4rWV38aH0A5BYN8HiNNYuAvAYz013n0IOJwkP73/bABq+kkzClC9CPDVoNfgbx0AI6UO/OdeA165n7pcfWPqQbpPbIZ1AOMENhkvEQn6GrEZeiE5EiOJhfEhVClSgqOe+BpAUPeldwHoK5wXJGsrIJGRRKkqIsh+SJr7SDUecL0kYx2iILHooW+eQ8rxChKSM6dOhvtdV4EFaGfx0CRIk0Rqq7DIPZ6jXaoiPUJqsUwk57WFeQBZexrE3evY+QvpgLruca8cX0MQKleEBerNHLMNG5IbSI39cx1CZF6OTSNmv2i96X9x0Dblec9U4/HmAGUJLKjROsbkkAVjU2ySTxPAfBBCeKGfeCHkodRonvZJgu4iBVKFTAmDq+jq2TuO1LPf4j4preggb42+uzkvsHSFX7W2XtJKRCUquoq9gyes+0gcFqi36kMy20ZIkqpGpxB3jlJ3VW0MwOj9VWLRhpSmSHsMtjcLoZKkSCTt4whYKcHht6pUWXUQaRHtGEAKZpTxJVbxtZURx/YBgI0l4j1/zPFl2qitRAvj3/r3Q86aAMiqZLl4IKHzWfDJsR2OZ+/LYzxDzUKUpiCxxifZ4ZjqSQeMV9WpfM60gYi5DnWmEr1acqItiN6s4kOdQlrhcfqY8NaQCspHQtVIX2qns8/4yobiSDpMx/zADQ+jc5h7SENasHFop88l5w30j/cH2UmQi5A++czT50pjJM+6hVZioUxR9cst6u2iivYgutftZAxKdjap5ypjRNsij/dBvFRBVBKm1ELy5X0aRbqjtMrnyfEmALc9zt9lLrZ5Za53gPrcHt1qD0RvHJ7mp8SrXC9pyPlyDX1U3aK86q/ql8itZFlbMOmpSs3G75KuetQyjiTKl3Ds6NF8wfH2Rnb+eVbiahlHrz3eb7XJYgxY7/fY6sTiPTqmfrjeA/Ue+ND0wPsmFnmCrbxwbB6ToxNoeVnEIf4E6PAHmy/m2vNxsOZP7SRe5u1yLPZM/oLjQipq94KuIBa8UG1EkVZIMiQVqkNliQUuZwEDSjZUDfEF56tTgJGjbSupUJrAyzb2WbIQUgJe4u59WdlySYUbcCGIhS5twyvUVg2xADS4CXwFSkFUJA2uTgJeaqUV+VgmGB4PGwsJheVCElRrKsSirOAJBoU+LhirArECoZBUPJRYAKdckZQUBNAnkSvduip1rc7Aa64k84qPgGmvQyo0Yo6o1wBSYxEY6Owqq8WCcu+lXpTuQiymIAuuuAs8lTx0018Six4+LMyGKoYG4bOAHFfVtUcQdOkF6SJSkEuAmiAg5Gkv5nsg0EPSwnVKFt4kkvUk5GLp8WOiaHfhGYpo0kR1HgIgCohVg1L6oLoPTQhg5DjRBsFI1wu4dpWkaPCsescI9VTqYNRq1YFmqJPxHOb5KLHYIiZBO9c+jxesj4yPYJDeG5IPwbherl4cOhlg2QoLbvUQpeRGtTCB7RKGvGsAVElWN+BsdGgIQ1kkJY4t2qUEqB87A6UOPj+73A/rLrExSvkt8jFGhCorA/TnJYDxKMDY61chSRtx3+gj6iKw1FZmkf7VqDqAJ+RDcCkJuMx9M9K45Qjgde06yf0NMExddFvqpqqewdbEc0qCVHsagFBaV2OSSPxUI9PzkKRKw+1F+k4PVpPEM3lIeyMIJPfMMXkK1a9RCNgliIWkwr7wMZHkqP6lfYqB8bRrWQQ0r9N+jaiNY7HLRwkftBkDf0gRdWhi7Cpt0tvYWe7jY0imkcmNp3HCMUkbVni2dU+sxGKHfTPkuZt+70dC0Ym0TY9LtlO7ESVsrsK7Zdsqxj7qiw+p0zRkSc9frugLyhvoT1XAtDPo4hkQfO9D5tatN+U7x+hymRvGJ7srjgqZOdcpgdDNrSpbzdxzicWWXp4gVZKLPa4/YC85OeAe2ZYDDNMbueetjNc22m0cDNXUlMgqSQk1J797Z+1f6qkaUxALbyJ57TCetyUk3M92+s1YIG20aZs6rtE/2lN53GCVen9SuiOBUH1xkDKHcDXr/VLapG2WpMwuic1KVrYyl5c5uhw/3JfE5eLDMyJzHtVKvuV89ACnqie4QDLjdfEnMojUtRXyVG0Wkco/HCzH41glj5zh4ZHjv+PM4Z8j1Tk8fDTrmuPP+mrJpZ9qq1Q9Vm3P0avrxOJof9R/1Xug3gMfvh74YMSC9sXLhUnRybL63Uny2GRdzpumdmI93kVloi2vkfLb/PwusZAMHJdcFGIhuQgDbl7avvgPo3BnV7ObSBSCWHDOawTmVWKBhCDiSvCidqU7iAarnRnkZ1JRiEWpt+tc/tPOYnsPScg2PutRg9I71B7AT8NO632UWBT1pyypyGQDQmNZRXJRkVZk+w5e7RViUSQf5aWepSVIK2iHcQ50NbsAzp5nT/guwF1z6gXgaMi5AZB1dVj7Ao2zu1F/GAC8KVmYZ7X5bQiAKiqrgCsBmGD8AivmFwE6EgtvnFGj7wDIJwHAgmKNvzW+HoZgaJisgXGsxAPu1MdXYnGXlXjtMgg7ls5DKq4BPCdYldUNqCvHK6QTN6iSoVtU1W00Tn7rwVy6T3yIpbk5AtxhuDx+Jp2fOBd2B+30k4aoBnsTnLrOKQgO/XnaI3nQbe0TV+Kpt3YcRvrW3amwMgygBbdcLzlRSvMEYtCIepxRwV/AePs07b+P5EMwrJHsK9h4uMJLdni3wpiZfjLat6pejwFygt0N7rlA0JXkfvrN/umm7sh7on2t9LlEQbCdx3MKYjDP9fepi4RoFQlEK2NzjOtP0089tHUZwBiG1oBHXf/aXo2gH5FWOwdV+FRncfW6j/F7mX4+TdmqqNwHcOvFaoq9IFTS7Dha4/o16r7Jvp20/YDNUVSfBgH13m9XsAWckgOBskRBEvuIuryFofltPFg9RH1on3ozQFMD15/GJuQsn+dx9avBu/YOEkCNt+0TVZ/egpTcx3j7kYSGf0r2Nun7TdTKJAAtuLtthdTpalavUGMQnQnG2BXA/SMI3BwfVeUsc5v+UAryhN/r3E9VjJoY5x1Ezu5FVWoAcqFDhnDzSp+5gNDGsyVZ0qOS42WO9sxBvGawsTFiN7NCuI21zdotIHNIpyEHShEODlDlIs3Tyv1QehG2GJTLZdSJe0sZSl1cCGhBFaoHlTolBJKEtSXsOHRTC7kQ/O8D9vUC1kh+FJX2eVYbcKPbRFubGH8tkLFGpTmMK59xyYgSLe+/5Mf4Gs4btkPVxGbnHfpnG6KChUpqgSR0Mo68F/aN0j/dJLdAeHoxaB/k+TZOTHiXIo0LBapDnea6Hr5LOlyUcMz7yfOx3/JiiW32HubGe0vyQoHnozsOT+UMPBEbV3mOLc9lFSlPXFXNzbORJgLklfS01S3KKpnEEf5U0rgrbxrrV1urSpKa9pSL373PtSpXHD2fa3H02Hv9Mofcd6VFOWU55q8yp/vd4/6uEwt7o77Ve6DeAx/mHnhfxKJ2ArSx5bf72u/lzVEmyegY01SuEWzXbmWSdR8TcTnpy8NjfIJQlO8SDD6Sidj7HWAomciSiuwhalt7C44VUqFKlOeVWphnI8hTwKE9QthW8DIV7IfLWfZKLnIMidy+I9Xmh0DENcSdisRiOyQWrEgCHDSatO62WjuLqpRCiQX5hutZgEGOxu2xyvEKscgqWFlaoatZQUuQm+hF8V4mWzleACCOrlrCwnNpH2NlyEgr1wgI7U+jYavqsQboF4goZXD/FKAyRVAyA7c9YW/fDOBS0xXiC4BxjbEFmnpt0sWqxMIAabqG1XD0DKDpAmmVCGioS5WCvAjaZ8jvHuo9D0lPdIF0Rfe12EmoEmXfr3IP5gDHkiKJxSh5KGV4ABF5Y/pBWoBY7CzMIQ1pSaPj42n83EQ6DVjUK88W6Er7Ae1GLNMVcdujColt1hB8lba5si/pGKGOlwdwO8o926IfVLdSgrNKex9R11lAlyBVjzjq47tKrh2B6lb2s3EtVA8LWwGuNzjfI/JW2mBMBwMPSqp0kyoU815JRKx7N2VKxCQHGp4L1gW3GhZru4JFQXrI9UpQrNcu/aYESFW109SjKaRj+V5rp6PNxjy2LrqelYjpeUkgqPpSF+2/CmBUOtTIdTewm3lHA2/A5knqr/tZy5+BHMzhQWsZoCwIHyO43SUIgeAykwI8GdEGmZkkvJMxocqa9+dVVNRuIFGamX4IOKb2jOWDnl5ANCpgELPnII+DECCJhbYgSkF0l0ux6TUIyS2M+SWxrR7Do9ouxGKD+ih668A1bOfpwVjtlzxql2LAwueo4xrSCUnnHdSwdPurtGKd53uVMbYlKeA+BvmGHHUgPRmAoBhTQ9lGVrkKSI6kDnU12tlP+XoTm1aKQp46XqDI8NZEwwHruFO2Dqi29UPyDJb4AMnWPOBdo3snTIP6haG5fYXR+wHloRcZ5LGJwJd9OAHoQgWug9PrK8shGZNcGP/Decm5Sylftr/iOYMUNXHfDQBI56AHx1wiEeE5CakOv22F8TaMEi65UMUy4nXQH4glIligCwkGEmzXRoWxpJRkg3MSPd30dtM3p+nTLgiTUhylhT7nqkONQ0aUXtl3IXXixjlb+5y5xRxemYPsJ58P722c40sk45jp48rKuZyAv5Vz8Zs/1fdGOWCSmu85fT6SyymFlUT+5nz1okodOByufiNZfq+UK/Kh6gVHDh/9YSZWuVJ+5eQzryRNaUvJI9LZXg+YR+moyu+Sjy2obqYh7f8fIm9X21z/Uu+Beg98W/bAt0Qs7JEj5OJ4F1UmVCfQ+NROsJW0ZfJ2si0TbkzGTsx8nkUsCrkoUgsbUchESCyqxAKpBRILA24JniUkWWKhPjcvZsC8q3/ZqFqjz8PfRQUpu5utVJb6294gFgCALQx/t4lhEcbbkheATyEWRWKhelMQCvIuJEP3q1lSkklFGHEXYgGIzIQikwvVTWpfXOCeUG+RWKg7vQEI3ACmoBCSdiEW0S4BDx2uRxiNqQXTGjKrimR7FgG6k4CW+wCdJQARHQO4aE3nAHTnAdkaZQvMJRZKACYBdrcgCgLGTsD8KODD1eQxALBgxMIEcer2TwKSp1gNfgr4HqK8qwDOi4B7db6ts3XRTsD0XqfhqC5kBfRvTs2k5ZmZ1IDtwgB1mTh3Nl08fz6iO+vWU6Nfr5NYuJrr+BCgqcuvKo/1BVtBLFT9Qq+cNuijX9LRSFoBvqBcj00GjVPysMBeadV5JBujtMXVYYmFRq+u6qqjr5eodgAp2mYQkV0C3mGnwfnHfN+E2DpeyZZ7of0OxrX0fz9lOj6Vrjymr5W6SSyGWZVWl904Dg/I4yF5GRnbeBDeH8sTBPcB6k8gkQgyzr0zOrmqXPdw/atqWz8AcgjwKrHzDpyjncPYcggoJRW3kLwYH2RAMA0Q7eX4FC6Ap+aROuCJ6gRqMUOsyl88TaRyyKQA0z6y/+xfwai2FuG9C3D9NYIdvnl/Jk3evZ8OAKuqye1DLNoB4EMA6cuoQglQ5UN60NLmQULTz+cmtgzvoEZ1mzo1oxrWZJ8B1jfnFtUPSycwfj9BAMVmXMqqljRGfS8AdDWqF55pW3MfIqAEZgEisAKBXydI4U5xUeuqP0bzTRiQ9/b2pm76RkmOpFFipzRrBGA9Rh+N0WcGR9Ql8iQuZ7WNospZYsE9MSbHCe7lOdz+jkIs+4nDMYtkY5b6PmRcb0Iu9mifNhb7kIrESv8BeSuxaOQ6Y9D0AN67uRfWYQd7jm0CEW7SZ7rIleCqnmb8ExcXWslDiZTfG/Hj3EwHamexR1vXIT47lBdxMMgrAvnRN8a1ML3SpTbGVwN5b6uWx7O+7znqo+QBcWra4voIjqkkA4nFKVTFHH/OB84lElPVoLRHUYUw3EAzfoMo0y9l83mqbtTfPvMhy/Qjn6kcrSQ7ckUcK3O9P8p1JVVRM41MTWAZfI5slcS182G1Il5Sznsw6njsejM7nueRAo7+8OpSP88cye1YPqVOkaZy7l31r2Rfm09t/p6uE4tKJ9V39R6o98CHtgf+VIiFrXdirZ0ky0Rb2zOeL8ePTMCVRB6LPHwpVD7vl1gUiUUmGKgoASo2AP5KK9Tn9ndIOgB/gjWNaZVYhFoBwKSFF7GqUMXOIlYTfcECxKuSFipn/X2B7UEsJC3Z3Sy63pSzCwCyHF9eh8RC97VIKsg7SyskGofEIo7z2zTF1awrsKpVxMfy7B/+uPedJTCRVLBWyYcoxJCKnQbUP7iGCocuuQBXIG60aQEH2BugQ57sF/ktAbiHWo8g3L7WwHoC0HXWFU/arP69uuYrXH8fIHYLiYIrxp0cGwHcaIehxELPMt60VaQi6v/fBnQ9ABRtsFp6ibxUgzKQnh5qXHUPtSe+P4HQ6Jr2JADU9ujO9MaDh2nt4cPUjFH1SUDgtfPn0nMXJtI48SWaAdnaYugyVaCklygNs1X9Uc1LQOQKuWpH66QR0Es+BOfZw1OWmKiKJUhXbWqacfEAMrQGmj4FiJYwufqvKpRShCVAWTf5nQIwCzKVckmClHgYPVppkFILgXiQHvpAEAukQy0JQ1qIgivNi0FUtlGzAbDSbyfpa+ONaMAdkbC5XluQE9w/pUqSAL1naRgfboMpU7Wr+xCL6wQEXKSPlUQYzDAMtinDfhyEHEr09Dh1F9D9DippGuqq/ibof7j4JE0+XEx3kDw0AGh7AZlnhgYj2rrGu5JGV71t/xPqPEI9jeKt0bTE4rV7U+nmrTtExV4OFab9jq7UdvpUOsXnPBIHbSOU4Fh+tt8wLkpn2DNM4h73BuQiYjhoEA1Y33k4j83Cbmrr7kkdRB1vRXIh6R5iTI1DWiV7ksMtVIM0vr8N0H4AiH4CQN/AZewu5EI3tfus+CcD6EEqTpCHNgYSLGN6aEOxYluw2TlLey/g9vgx19+nLjfnFiIKd9iF0UdueqVq43MWYjGONGcESdAipGKWMflgaSltICnZZnwrKdlHhWkfAnTAXnW4kITyHHfRX7rdldDu69yBoHfaWmxx35RibTEOGnzmaWcb5aqWJilWOuSz18zCSKK9S3ivCiJD3qEGRXptUELaKbEg/w76Zk/SQj8Yt2WH8nUg4NgzJohSVKWDTSHRQSpBP9mnRjr3WbCfTvHcG61e0isRdJxrjxL/nDAqm5I5qh6TkH12eKakiJOVH4dn45o46rye59CcUUwdfK2kqF7CnGdCj1dOVb9U6hPzcLmscqwkdcY8cn0lX3f5eM6tWlwuyZPV95PVNb93pYkTpSR/HG65Tvlc/I02HJ732/Era/P3fJ1Y2Av1rd4D9R74MPfAN0UsSoOdSP24xffKCUF17QRa0jhxl+3wW3m5VM5U0vgCqJIKwIrf45jf+UgSisTC70VqYZRtjbclGBuoEGhfIchVkhDEgmutcahCAeRaXDH0RcunSiwkAgBrpRa+7Es73ftCdXVNreeqxAJVqB3Ayo4rjQCl8oIKAkMZxd1slVwAKixLgNBE+SHRQIUiIm5Tbom6nQ22c1/a9mg/dQ9iQSsITQah0LBXUuEqOKua0fdG3gZg8B2sG6vqtlkwYZ2WOPgQYH0HoCOw1b2p6idKLEYAp0p1XNG0jwT/ruzfAVAJkFWZcdX3MkBNqYabEbpVu9KWQ2nEI9IbEOxFQNw1wJmejjRG9n5ZJ6NHK7nw431d5pwB4yZZ2V7GcNvVbD0VXRsfTS8SVfoyqjbaFLhJFvR0JUDTbsKVdesviAtDXfpPr1FLSCeUbihd0l+/Rqq95LHDavkSbZ8B1E9ST71ePSJtB3n30d5ByIluRG2Tdhi69RS0Gzgu+o90uvNV4qDalWpR8xIlytoClAus4TLpFPf2DORI4KaNwgyShg3AqOpKAwBPy7ijcTXptYs5jdcsJSSqoahOpMqOxEKgJ3lSvcv4FW/OLBCkby0C4j03coqVdVboISUa69oHGuA+ljDZNvrU/pKcGKPkEWB+iqjo79x/EPr+rYzDXuwjJBdnkV6MU1/tNzTmn8aj1EWA/Vnu4WliW7z2YCG9irTijRvvpD08Yhm34QBi0Ts6koZHhtPF0VOQOiKvU8/bqFsp+dB97Rnq57P5lOfjHqvqyqk0Zt5FKrUPsNeWoKENw+XhoXQCMN+tgTHgWWI1AnkyerfAU89fk1yv+tqCxPXBLCpzC4inKAtgj25XahoYwAgaQ2jK9ZnaZR7Qc9YG6mMnGYe2UemZxvkz2H3chMhuoeJktHqCcADEWXSgv3SRPIIUZlT1OPpgFWKwCJmaB+hvQG7Mc5sxuEtZ+4DxhF1FXMt1Lg4orTkhYUDqpNc4Vcd2KUd7DsergRp5+OPZN3idxEI1OiemTsaNcppW5q7HT5+G/ZP3xAB5Sjxd+FAa4Ufy0ObcR77rEAsJF6bh4fbWsdBkgEH6XsPvIDG0sQfJhWSTYc18mAmtfXypQiwMfFlcQZOEKw8JRHXOpj6q8+WNPb9jix1/PMQBRnJNEo9Egppd/h05Hf7JWUWepWxHQOVaM69utd+rBytfKldY15KsUs/IKY6XE9UqxRxbPcqF1e/kWmpQbW9NkeUd5xxdtvK9nPN4zOMlAfvoXwqJdwvX1m0sajqn/rXeA/Ue+FD2wJ9ILL721o10AzBRttoJ0mP+jk/+UZ2Eq5NxZUY/nNhN6MSbUxxOwR6vbEyuHo8JmO+FUPhbQlH7CXIRBCO7nNUTlDrM4XqWvcRCUqEq1B5EY4+XuvlZugBbIFRiWJQ4Fr64s/RAUsHLGbBX2uheaYX1C2JRcTe7vZklFjuUtUe5FJHLACw0F2IhUQEMhFtZQEVWhcrkoqoKhRpFuJuVXAh2yAWcU+0D6+6H9UwIBW5DUX3aa4QINAFMICYCJKBEXGPdA6yQh0Qk2sw5a6/3JoHwNCo6kgFd554BXAwDyHQna4yDIF7UwxV37QoeqN8PgNGmwdXvc6jPqKOtBEIAb6A+Va9UMVoG2Ort5wqg9ALSCr1NCfZtjxIgQad1cvVfr0OujmuTYbC7BbwOrQLCNJm+hEH1CwR/uwaxEMzbI9bHFVjLlVgofTBv2yeAtn47tEfpiTYfruJ2ca12BNpcCLgsW+B9O8gFhryUz8BJnZCOQYBeGwBsj2skBPpZOsl9GwdgKklQXakHIG897DvVxGa43v5UfUQi2sank3GjzYpkRtWlO4DtBaQ+JxgTXYJYxtIcxKIX1aBhpBjnALz2h21SfeckZEhpwQgkTkPWZdpitPDXiGYu6HfsvnB2OF3Cc5USFVXClDDpzSoie1MnyZN3XiCpqtgcoHgSu4ZbU7NpEyDqQFEffwxQP0E+2qPoqvkR9bwPAbkMoTiPK95R1KVen83E4rXrt9IugDeez86edPrMaDo3NpyeQ5XJZ+oJUp4bRFH3nurVqB8pQSs3bou+NRaIUitX8HchkPtz87BeXAR3dKaesdHUM3gy4lc0cG87yUvnABPcew3gH5HvPVbmZxmDjyAm6zMP0tb8fNpHrUtbj5aTXIuL4g5JBvdOu5BtCOoObd5lbPWcOpXGkKpcwLB6k3lh9tGj9A42PZIE1a+0XTDwnfewHbJ/GrI1BLFQPWgDdcenSCwecY0qWFu0zTG409WN1AKSaDA7xrQLAi4USLg7eY7a+EgsjHFjZO1NVKmUsm34bHN/lRq5qBHG+daBOktEtWNCuSot8Swsc61jXNsJyVLYRpDWse5Y7uCeNyGZ0J5iib5ZhSQrEemwfPpfMuyc4bwjgVGyp7cr6+v8aV0lFhPcJ22NdO3see+lz5Sf8je++of8HL+1W/zisBnHLg4cTROFxsmadHyl5UfyM/tnbiU7JmHHn9uxangkjquTF0n4U03D97jKEyZ7RjnlUORSvTBneeRvZF45Upuu5viRvCpJPVbq7qHad6jH68Si0lH1Xb0H6j3woe2Bb0Asbqabk+hU10yWToRutRNi5UDNS6hMqSVdpHBGrXxxl/OJvGsmZn9HKveVjy9Gv9eSivI9pBW8ICUTEgtVokItSolFqEGpClUhFqRzdduSBe6C5zDYZpWwEItMKnL07VBP4AUebbXG1DMTiyyxCLUrvELtADz0DLXLy13JiCv+JI6Xs16eQlJR2Rc7i2K0rdSiWWlFSEmypCQkFtTPmlpfV/oLqYj4FeS9i6RiT6lEIwAFl5UCGlfBPY9b/yifVop2+JF722/2rgBWEvAUlSHtDUxuLARXyHsAK0abFljo1cf6R9wAAKC68xwOT056XbIfNwA1K5EPYJ96qqpkeu/PeBfAuNhh0CUhAULdJuxcBER8rMsT8p5FOqCqz4wr0gCkHfIZg7xcYaX5eQCv5MDy7Fq9WqkWpQqSEgjJyQ5sRZUoVTgkDtpBzKDqtbCG1yCuUwKgh6pR8tQrzgL36h3I0p0gTKjKUW4z924AoKWr0SbA1ip1UEFGInGKa1xBNxig3rW8H6pC3QW8h80I4E/SYVTyWPGlnpK1PspVjekdAPAURMJgb6rgac+wQZkTgNfLQwPpIpG3NSjXle0sdRqm384i6VEdSFN8CYfk69W7D9Jt7CSMQ/Hc2VGibp+KmB1UM+6NdkJ6YlJtTWNp759gVTuRGYjJbQjDdYjFKqpABxCYVlbbzxAr5MLwqXQNt7tGfn6MLcR97B+s2wR1O0OgwjceLqTXJqfSq9dvpl3yUZrWNHAyTYyPpMuoMT0PEHczKN9NPEfdIw/d4qrzr/G7xujeK3whhRvUfdSK9ucXiEGxh5QCNZzxM6kP6YkqO48hmHsQw14A7nOU3UU/LkMG7gLMDS6oh6bVWQMUzqc9Ayr2D6ROiEPfyBBB+k5EIDrtMtYgAttIGVSXOnF6CLJwOp2lntpEzS8upslpCBbppLxKEIziHcSQvj81MpIGGXsGldtiHllZXkpPuWadiN2qNGnFE8QC2w7cUNEqKAl938hzqMqhqlBtSP+ytzjsOCAWOxAd1bO2WEA4oIwG1Zq4ptlnl/GgJyufOd3AOsY0vF4JsoDSo8SCcz4FDaTz+XTRQhsYAzp2MPY3SCu5WKbvWqiTjgLCgD7UPxmXpOujTNUFeQhDYiGxkJiOIykaRJqhlEmCav5u+e/Rb9a1vAciUeUPt5h+4I+fuLBcnfde5xZZV77n9CW5hCGSPPtPyZM0OV0Naai9ohRLoiN1rfyOpJX2HS+wtvjjbSzZen1pS25MTeGVBhzJp/Z07bV8j3cLdTFv86wTi5rOqn+t90C9Bz6UPfBNEYsywcbLp0zQNL92Ii5pao/ZQ4fJD1edYhKumZD97SR7/FPIxPH9LuA24lhUiUX2dLTuah+kQjuLXSUWgmgAqdN4vJgBR0ooitQipBWAQ1/YWT0pSxxMa71tS5VY8GLO7mZZvaxILMLeArBmvW1/lCFhIM8sqcjfC6kox7LEIpML7StyDIsMXqvEgnoLqgVBu6TZh4zoA/+gCY8yQSwwYKV+Ghi7mir4lmDoZcfKCxbaaK+YwnT8hFBkYK9RtWBYY1NBkeTEzTaYVhCv3YLGsPwMoNgrICITgYH5bALsBbSCcW0g7AG98LjKr4qP5Mj+CNernBXA2VYcWoWdh/EhZpCgzAH0NC5/CsjuBFwpRXlB70yAJFdgBenLnLcs6+bKrUastsF+J+uoo+pJ98lzGmIhUeohzRWIxfOswiu5eEwedwHw2oRodL4GqUE/KIyueyEfHdgXmKekxI9KX3qw8tMPQLOHdOOpVye9ZdnnkojoD+vCeY3bjfPxFMB6BxuDCEYImdgF+DUBultpwwtE235+fCidR01H1aXrAPI7fMKjFe1+8SSGwPS1bVMa9LW7Mxi5P0yzSEAunRlJVyAFZ7G12IbYqPblSrkETlUxY2Q4YFWbUY/+MeD2LnYWX4ecPAb875BfJ+o8VyAHV8dOpytIJ3Y59hjicA9Dbz1P6ZHrHG5334LMvHZvOr1x83bapQ1tgNDB0bH04lmIxdBJgGmugxILo5O/w2r7NJIPY4Y00U966jlg7O3xLAig96lLE2l6uX/Dp5AkIPXoQr1ONaF3kI4sk4ek48Vzo+kkUi+eqnSDPtTwWrfGSiu2Afl7qDQ1owLVwadncBBiQRA97o8kdw3pwiaqUjsQiyY9WGGrM0g/72PvsILh9jzEJtxZ07fh0pp0SixUpxo8O556UYfq5H6vOY8sP00bSiywAdpm3ITEENuQA4jFPqv8uJFjoAD7WSgwQKDuZtu5dpe5Zx9SpK3FHvlvQ4p8nva1CQmpBepNgH2fNuc1nRyoGhf3kflsg/5cYyzreUpPUCoERswKVLEY1sT2gDRCOlRhUnKxgrTvKfddct0BSeuhP3yuXVTQzkp7I72oScCzhCSryenqWIcA3ZIdxo/POf+PbflIzG8mqGyO9dh8/vKXyt7dYbp8sEIGIml+TuK498DUlXyzGmjliiP5ml9tPXKa8rfMvdVivZY8vcJzfmKrqX+5Nvbl/JGDh/XycDUPvx9LFz9LHu9VRrmmpCu/2ddtLGo6o/613gP1HvhQ9sA3IBZHVaFsoRN/eVXkebP8qpyr/DTV8fP5+vjrn9iqEzOTbEz8HPVYmbzjWOWcL95MDip2Fvwux4r6kw0qHqLWkSQcEgtUflQZ4BrrJoBQXegoscgr9EottLEIguFLWRBsY3KDAsAJ43Q3u4OBeJFYqAqlnQWFRNuUiqhKdcQzlMQF4BBSDPWsBejsGy2T+hSDbcuy7UEsQBCqD5mrnz1JBaucDS24nmyGWPDdMrxGQCHIVd9flSftGax7O2BBwOPrW1IheHAF1Hw1PBb4a1dRQIc3oRCMars55vXeYlUsBCcSBOsdUhDArYa/IcEgP93JxuqtbSZ9BjestHKd6hZKhKxvjqi9HSvcjwGdTwCUuqTVTsOVWA3KjV7tCqzuapcoQyNppSuqOLnqugXQCnJB3TogJK6WK0mQXGzTSI2yL2BD8Dwg/Axgq6gx3SUa9zz5rXHfDJA2ACjtB6R1AbBsq5vEZJl8JCcGGBxGiiB1ingKkATHk25JNV5WWqN0QTU01af0tqTNie5WbwGCVWPaYOW9kTK7AaEfmRhLz40PpzOoorwDoXgTycYt0hibQte/Hzndl8Y4103eSplenZxJr92dTtfZj6Dacx5APjHMKrxjhTq54t0DuZD0qVKlBMn7pFG3akgPKOPrXDsNUdDmowfpzPOQm2tIBpRY+Hw8RepwHyNx02tcPAjBuQ+ov41NwvV37rKyT1/gwekMMUY+CvC/gEcovU8Z+VyjdwP03cUOZApyoG2JQdwOIPbxZJd+BXC3QrhOQb7OowZ1EXLSBtA21sXbGGYvUE90GYO0nIJYtJH/bSQZRvB+xL3aRXqwDzk4QNXpxOnTqRtS0YP7XEF6I31Hk2PFf5W+XuGzxzPWSv4nANsEwAgvSutIBE4g4WglvaB8nfyMB+EM0Y8EpQvVrHZW8jdQ3dpGFWrHSOvEWFGqs4v6k16x9pH4HJA33R/No9iYUyQVbYyzPdrJpICaFfMCBEWXsxuQWhhLGFlLRIyK7jOqCtg+55yTTiDtwMSf67ETU9rBfdWw3boaB0SJlWPe9vZRll7UurjnG5SxDIFZ5d6qYtXDGBrg2fH50/haIq5qnGTc51FJ4IBjmrpq16Q0L9zNOqptTGzOzX7JBzzu70ICPBPNz4lIFYk9fDRNHC7nyj5fXJ41y8jlVguPfMr7oDbPSnE5g0jlnzw/5apW8jhMGHNqJK00rpRSrU1N2mqWz/hS0v+J1x92YO6wSj7l2vhJebX9WCcWz+js+qF6D9R74EPVA3/KxCJeC9UXw7NeELWTaOmpmGiZYN1XiUTlZPU35wuJiD2gx305ZkOyGlSOYaFa1BrEQhuLIxILgKkF+SJvMu4DoLRILEItgRdv2EVU9oLfQiyi7rxJAtzzt0gsDJK348ufsnzpR2Ri6m/68LEP6A8yAfiMMo4Ri0aIQo62zaseMlJ5f9O2CrGgnRIApRXaURy0oIbUimpEK/EO+LQosaC+rv6brhCLNYCKRtUChdDHJo32CNbfuhk4zdewqkSqFal8pW98CYgvzLKaq4cfV1EFqxoDhzcmX4ikEbTaRyUfYz08Bhxqc6Bxs/YGgheBvSumlis4yl6nIBbUZ4O0rrArZQn7D66bBqA+WQfAkVcveQRwAiBpnGpaVYnOAL6GIR1KBfR8ZRvc+tuaw/haw/QgFtRZQKVq0iXUizSglgAoITBCuETGFXGJVT91VbdfdRRbZY6mnYIY7EAiJF6jAH37USNgCY7SAO0k9Ky0CDB9DKjzvp8DEAu4lVi8DWF4GzWkSSQBa4B0Izj3s8r/kYkz6SrAXl3+m6R5kzTvoN7TQpnDAM4XkCI8f2YojbLSLiN8/d5MevXmZPr69dupl2NjEIuL42MR50OStkO9NPjWjaiez1YZj5JTV6q9308gOa/en013UW3SbqOXleoXIBXPIZXQZSxFhAH2NMDe4H3r5Ocz8pTxPUc8inuoRvJApT69LJ2fgFiMpfPYQVie+anOpQvde0g1ZiAXM+SzD5DXSPsAVSjVhhS9NbCC3864G8EF6rULE+kieUiypynzbSQNs3i0UuVojH45CUHr56PhtpKteYjN/mI23G7gWM/oaOqFZHX3IY2g3+17iat2VQZ/fELdN7YxcqYtgnevaUCKINAe4Dq9SLWix7OEMfgqZW9QhxOD2GsgsWgj9oPqTHqxOnj6OK0g5XD87+gJCi9UCaIS3tgY97qc1Vg6FiXIswVQb0wKVh9SE3Ux8J3RsNfpo7DpgJw0cF8kFj5LjdRtj3MSCImJHs0aKHsXYoTsKdJpmK772S3G4ibEQtezXUiP+hmDStIkIqsYuS8zrnXRrIeqYca8RML2+rzm55lnkJvtuNBr2CD1Dc9q3GvTWJ+Y73gGuIzNPx7Nxz0Wv0hLwjxvm6aS9vD6uCT+5Hz8aqJIyD5f69HjZeRjHI0LD9NbZL7OfTle9nEmzluv2CoFR4pSCc5VzuY0/D1yvnK05FqbthwzybuOl/zj5NHyPVR7baVjPRx9bTt/6rM/Fr//ffxxkS7f41z+8TI9Z53Ke3aLseXnJPZM9a3eA/UeqPfAe/XANyAW2cai9uI8VQoj+VYmbhJICzzmeSek/CkTVmWCrc3IayqTsJNtmcQ8Fp9y3omtZnIr3w2MVyY8J0gNs5VIaLytAeqmagS8lDXU9KN0wSi2pnN2t6a+QPWME3YWvKD1DOUqvivw2R6gsg+wkttkE1Q5CYkFgb6yxALjbQ1Sg1gAMKiPm20Kg07z5CWffddDLpROAKKaKNt9IzrX2SuMa4iBu2gzIAOgsSvYAIAJ6INUKJkgUnFTu95vOsE1mVhYjvUK0kP7DCbnCrcqIQLmMOSkHaoqrbo6yzltMiQLrQBQAWXpe20FJBhBGtirStNJPflaVYmyPmUTzEaPkolkRGnAAgDpATYOAu8m0nYBkuxfDdhdURb0Gn9BuYCSByNw6yLXehkfYgZCoRqNKlHWK/TL6UdL1WB1EPLwPGD+FMbPSmM0IA/VJSpp9GsNuJcB1bMAUfPEuj3sDIyboYG65GRyaTVNA8D0VKU3JUFWGHpzvfVx5dX+XKUud1jBNyr3U9pk3AtXnh1Pev+5Cii/hIGzBGMOUCr4td3jkg3IkPfvbaQQbyAluPVgPq1CIFQPGsIL0kvnz4R9g7rwt1FTuv5wMd0C9Ktu1EO9JliFf+XyWVSFhkJ68NbkdHrt5t305s07odpz5sxoeuHSBDr9uHqlzXOqwZC3ZNIAehJopVNBLOl743NcRyKgutES/duP2o/E4hI2HsazcDzYNw9pw10IkLYv0bcA43VW7Z9g8GzUaMfdwNBQeuncmXQFick53ApLtp6Qv+pKt5Au2F8P2O9DKvaVWJBveBPivjSgjtRB3UaQCrxw9XK6Sv/5LExCRq5zX3RZrCcxV+G9Lz2SSq5fot8XiAtxsLCYGjEiJ5R4asN+oh2JRTuqTgdKALg/klejaO/zjG4yH7iCrw3CDiB9nzo1kJflaZdhfIdWyliGPKwTKVv1KZ/RFqQy7bTLUdfI872PN6wV1K+0S9qj/ft4ooo4FpQneTBwXiN11taggb5vMr4FkbsbSa/am64VnCM2HTssBmjAfYCrXIM/ajehKh4PeqhItUD4OhEzRLlIZdYZg5IRo3M3MJ5UY9vmWdb2ooNjugQe4BmTyKzTt09R11I9q5Pzusz1/mvn4pKBDg50Vas0UecCkl8DGvZQH59zToc0I+Zkxj+NiGeQ2xVb3vmXT+WYz2XMCKR1i7OVc/6uHPYrm2mcTz1eiAtH/sT0cWHMqflb/pvzJT+/kEFNFrXJct08UqmIbXMr7534wR/rU91Ic+R3OVGTpuQTV9Ucj6SV+hzPo6T1eLm+7OsSi9LJ9X29B+o98GHtgW+KWAi48sbkXJmgmZI5lF8WTu9lonSfk7x7yq9OsHFlzsHJNj4cq36vHCtEQnIRZEJy4Use4OjvsLUIYoGbSV7SEgrdz26qrgTRsLHZHaw1dbVdg8ZsuN0GscjSBAAJL+QsaSjfJQiH9dfdrCB+R2KhZyjUDw6JBZF1BQmcVxWqEItGXvqZTEgyANGSCz6CINWflFRIzYK0AMQlTnpn8mPQNledD0jbiKSi6QSuT/l04O5TgKcqldcLgu1TP9pXbNFPqjnpPlPbBomC9gCqCa3QF1SuqiLltQImyYheabopSzLgSr/Gv+EXnzS6iNXGwXRx/zhme+0ey1AVR4mJEbLvo2ZksDlVZNxcMde9Zhv5eQ+5aamPss6xGq16kuNGKYfqMA8pQ4ISBIiS3HtO3/saWI8DND+CRGAQQiBQcvVWVSDbqLRBYqZrTz0yGXnc+AyuNEuY1FNXuqVKlXUdREXK1VoDh0l4etgrBaFqMVBctZ8G7F7H29HtxaW0BHjfQM1HMNkPAHzp3GhIFYbxguUq/wy2BUuATN23GqBPEGfQujeQRryJ29ZVDJfbqNsYxspKLC4M4dGI8mbJcxLS8Q7kYgl1FsF4NwBUO4MJXMt2sKr8JmpMb925l2buT7NyOJAunD2TXr56IdRdvCGqXS1QB1XSoLgAbI17s7qL+3XqrGTkjsSCduht6wLE4CzSiiGIhfdZL2Eaf09SnymAvpGvBcRbAO9tVvUbAMaS4hOsXF4+M5au4BHqKmpUjlFtX7x/BkmcQeowhR2FblaV5MHEos8Ey40YQ7eQx0kkA+cuXwrvVqpdWe6UbaAMiaHOELRvUX2HJwBQvok34uXUhMSiAcDNA572BgZTAwbczX39YRDdxP1znLUw9qXbzleSC70yraOOto9UIjxAaWiN0fcJyFsHKk8+c9vUbYP7s8O49floRhrQRuC6RupysLqc1iAljpldSQHEQ7BPQyAErP7SRw1IrBhcTC4816RhQDEmSUKfGvxuz4UO8t6H6KpCBaOxGCQoEhNUxriuEbLXgjQL32SpCenDvn3I+T3yM4K2xML4FM4LDdTBYIe9fPqYRlQ720BCJGk0rw4I5xDEIxvQUy16w4j1Eo0u5h+N+nVFnIkF/cF4yMQiz9vxjB/+oQAfiThw5JvHD4/6q3bLZ3zky1beC3mOz0cP3xOHZCOfObywvFdKPrGvZGyqw1n6MEVcXVu46XJhh4kq346kraSxju+1lfKOXPdeiWuO16avrUudWNR0Uv1rvQfqPfCh7IFvglgIf8tWIRAecPJlIvZf3uc05QVS+VUujH2Zrp24IwuOVslE7XfOl+Oh+sSLNQiFL1jASqzoSxoAgFliUUMsIBQR16ICUmxwjr7tC99VdIApIKRNIMKLvqyMN4Iqs+RCkpHbmd8zwhvqCUjYDWJB/hhv70oueKkLoIJYRJsogxe1dhYSCL+7Wh/fJRYQiiKpEP5IR2xLtIe27ABQwn0ue1Wc9DjTjGvOtm6AUGcvQBNyAbFoIK99Kpf70302ypZUaCgagF+QRDtcUVZVaQVQorqIq5YaV2+QbpFyjLTdDfA4SV2N6RAeauijIBbcJMG9we1UdYo6cUzJg4G9BC85YF6OFG0AOKNFzwOYjHptP+gOVVuJDeog+TtFGc8ZnRs9fqUzy6zCCkyNLQGUi/5poj4G8ltA5167hj7yGEOd6SqA6SSSC+ufx5mr8w1RV9XItBFQgiIp8XqB7hPA9Kp69PRv2A9ADC6zWm+9NXAneYDYCCJGucVzlQTpdYjB6xCDO0odAKiNtKsfMP7S+fH0/PmxdAZXpnp00i3sY/bniE2hZycNa+9BNrSfeO0hqlCQhg7qdo42v3x2FGkH6jj0wxPye8C1t0k3x171o136egLQP4aHonba+/a9B+kuUoNVVuyNtXAZYvExiMUQHriUSmnfcQ8SM0V5utS1X4yrIHlz5d/AgTfI/z7E4inAvIXfw7hpHaKM06g3aRMCjg6SOQMhkFhMIXnYoM57EIt9CIHgWTLc3NufJpCYXJRY0Pbw+kRfG/tEA+t56j/F6v8T2rLB9wOONSJtaAKgN3GcBoXK0eD5iTR26mR4f1IdTvKqkb7uafVytMV4ELYGteA5O5CUYO/QQJ4UlbbwTrXb0wfQR7rAfWyivS2swishaSE/I7O7FLADSNer0/4jXN0yHvZ9BiEjGn6fgOR1oFrHwIBcIKUBzGtL4dMuEW40qB+A3UjWxovRxsL4FRpVM4lQPZ5g6hnEQnJBxQ7Iv4E8m1BRyjZJEAvmox1I5348PDz/jF0lGge0VRWzPYhMEypWrcSVaIOEYMWOpGQJQo19GM+/we9Mv09ZzncJO49ODOx7Sd/HfTtASqs3Ke2WLL89SAOG5BIWetHxrDqfARl1CqB0b7BCLCL+hvMVY8a0zncxp+SJhSNu/ijztbO22yERqAXJ5VykOJJHBvaRvw8cm7vDa/N8X37XAvtyLC6q/UMGpYhSq6Ony9l89L3yiVSVOkUHVDKprUNtvpFPtQ2HZZj+mWXkRkcWx9P4u64KVdu79e/1Hqj3wIexB94XscgvmMqkya5CH2rn3Wh7TKQkVv3CVG4182jldz7+XhO1x/3E6nvluxdWj/PC9oWaCUbe2wjVPzyubUVIKVwZjA+/fZnzInb1UyNv32K8zmN1WmKRyYXBpzSO1B4CElAjtQjDShpi+xogFOXfdlUVSgBo5O1svL1PfUgW6Yvko0gsNNIWTASpAKCoGrKH5x5JhepFgmvb4oq69d4GKEowJA7NqE20d0ImevFW092P/WdPSCwkKqxdRp/l9VkMd8nDaNiutqtTLYiQHCzRBwaPE7xJGgx4J/EI9SPAiIHzjCFwCuAxzGqmQFMXlRoaczikHbMAoPuoBKl6pBrSSKSThBDlGZUK77D5C24XyFMbBtVpaGoQGVdJn3D9I0hHFyu6lyEWE6z2a2hsOqNae70ZWXdBkBKSp9TbqNfqihsQ7gTl9wLIVeXoY+VYdSJXYm2n40Oj6yBr1Md4FjchBjem59IDPCKZxziA/gr2DS8hDdDQ3H73HjQDRC03DFwlG5X2CMjfgFi8fX8mPcXeYA+g2s44uTQOOYBcGM/BYIL3AfYr7C8jUblMgLhh9vfpCyNPv4k3J4lNB+VPoCr1UbwxXdb7EPfCiNBKO+4AxDWAXoDMaJ8xCJgUADo2p1CTmoNU7AM4R1hpvzQ2EsTiDMRB9S3r+oB+vQeJuol7W68XSNrHGqUrUXgVlawHD+bwjIRHJcjdSQLc9aOS1It0ZYD7rXqMqmHGL5mGWNwm9sUT+syV/IMNjLGpp9K2lv6T6TzE5jyuasexOzGwngbjSo66kCgpAZsEFM8Y+BASs88z0ox3pSbq3kjfHfQNplbUmLrGz4S9icbb1l/je5/lJcp5iDrWsqSE3w2A62aIRcsqqkpILRBLAcQ70y7G23tdPeEprZlnqZH6N9NWAfwB10iUBNT7SCM0wD4ghsaeUgPdVJ3oSi3YZrQj/emG6LUzljR0XoUAafS9CrHRo1Ujc0moJTEGjXFiuSQmhgXEgrHic9xIuib6dx8Slm1KIBfcY+vDLYh8bIfjjMEZ9h5NjNNm8ja9apq7xMZo4r62U5dW6nuAutc2JKdILEO8wTNsnBhVIw0M2IWtTR9OCfodq/TZCn1roEImsrDV0Djb9jPk2FdIBeNNtb9+7rPE4iSEpocxJqE+JBZe4x0pW+33cizPzfmXYJo6xeY8nuf/eJA5VvJyHvGY+9rsPRwlcNB/zwTmXsoWeZGuNk2pXWRvolzQ0UI8/h6b15tv9fpKusi3klecq630e+VVkybX61jtKueP1J9jdWJxtEPfuo+XuUcuLORR1NHWmC4OE5NoQLlzfav3QL0H/jz2wPsiFnmmZWKsTIbA4SMTemlYTJJMwP5zq500S5qyPzbNlsMxsTu5ez4+fq98qqpQvIzjOy9y96EyBICwMa7wF7sKV8QF5/7WReMO50wjqLQNzagjRDRbQHNWiZJc5E/YWAAaNcS0HZlcUKnKW0dVm6wKBfjHeFtioZGnXqGMSmxf2Q+ZWFQkFuYVn7LC7ks4u4iNWAwQgYjBIZmAAEko/G10Zu0AWtpxg9rtCuVg6uzpxzCTVWxWTRshRPaVi6ASMqdhV/znIQlKJ7oocwjAZAwGQbvndLca0Z1pezdAfBmwMwMQmwWQqPs+wLEhASZAQ1ArGXALN66sxk9BDCxTe4zTpBOQ9iE96AekKCVQzcg4ChICiZM2D97HMJCmHk+QPswColVfMzL3GQCYOvQaIIMXo73lJa8UQumLkocIfEYb9Gi0DIB3RbqP/MZZ0R9RNQqVKqUn3ibVrbLUQtsQVvIB9W9io/AOcRxUFxkfxhPSxHh6Bc9MAiqlHBIx77dEw7ZLuszLVfg7GHC/DbB/8+5UWsQ7kG5MtdG5iJ3BxfNn0wTxHBYgBK7wL/I5R7uuDPYRZG4g7A7eASS/xXHBtypZRjt/5fTJdBHQLyEyCJxxLO7xeUD7jMVhtPN2+gMFmNTMavoT6rCmKhAkdpx8r+By9mVIzRBAWpJonxlvRAnJdcjFAvfKsq70d4Wq0yb9/VXaMH3/QagF9dK4kXPjqQ9ioUSpl/GgrYhEQTUyJShvzy2mqdn59Bh3q0oLVLdpggS0D2E4jlvW88SGGIPYPKRMVcDsd+/FHuz6DqD4LtcvoH60D1hvXnqSmtmrNrQ/OJw6hkfSAF6hBpGWSJy00wh7HsbLOs/TJHYnj+hnFwaUGrTwrLWto4ZGX+3T3m2eh4Q61QHPgYPfvupkDHaiGrSK/cUafai9lQsCDZCShA0FkefC+5KxYA6wV1KdSNWjE6qCOX7oywMWDXT1uoSUxkjhiTIlQ0woSA4IItmNq1k/lKXEUPopeFcy0UB9NVjftS/IE5EHaThB25wVDkgHUwgS0koft9gWnjudDwSxQPrQzvPQgq2E5NWPY1JJq3PLPnMMs1hIMBLSjS4MzPsYawOoQunGV69QGtI3UW4nEjmDI3q9z5akQRKrsX0X7ejnuVUV6iRjUWJh5G/HfKEHeRbmAahsZFGz5R+OubzROvJ/9/asY7WpuL5kEYf9wTX5f23CfJbyIrnlPqO8XFp5C5n1kcyr+VXzqB7J1aitbb6ypiKlrWXPtbVtPuyLSpNKundnGqXWXuuBuipUdEv1z9/79fm0uIzK7AlVNA/SygbvE56DVy50pJ/+FA4ffJY+wPaF11cjr1cuMl/Ut3oP1Hvgz6QH3iexyODel6fTtZNh3jN5VufsfDy/DGoe9pqJ36PV5DXNKanLEB69OAAAQABJREFURO/k7Pfa34KVWmJhGn8XtSEbkm0sKhILgIuAopAM1RokFqpOWelMLJBa+CLlI0DMHqIAkwDx8AzFC1ZiEfYVtjnanetlDSUW2+FyFpemxrKAwFSJBXWzj8KjlC9q8wG4hTtZgS952ZuCaFf+1ZdWOqHaR1Z/yiTI9tn2A0hOK6ukHa5Q9p/C4BRigUrUCdRJ1Hcnw5h4JRYoUEASdtI9AOYDQEsf5U8AmM6xIr1KWRKIBwD7XoDZCGBC0uF1C5IB+knD337OneSjhyi9DEkunNjD4w+2EwsALOd0SYvkQsmBIEXAsg5BekI+6vlrsyCQMc/SFt2hhpEvAHqRFWHtG3Q7qjqJ0gbdx+qFKhODLEXQXkI1nk3qoAHyE0Ds9MPFcA+qOtclAsVdRfJwFoIRxqn0tWRQI2qJmdcuUp7Gz2/fuZ8WMWAeZJX6IsTiI5cmUj/lC7qeUm+lHBILXdlad8mQbnvvo19/i9X+N7FxWCCewRYqJx2A0vPnzqYrYStxKgy9H0J4pgHEgwDUs5CGS8Mnw1vSPQDfTaUZ1MX7P8hq90vU9wIgWHUu1a3muF+z3J9pALHkQqmFgewaIYJN1G0LwKq6Syvj+BySAiUuL0IuTtKHRvWW8BpE7h736DrShnmNzSnrCsBzvPdEeI36GtKKe5AjXaiOImG6dPkiwekGAPLbIaXRu9Vz2F3oreshebxNm2/ianZWV6uPFoNYNAPkO0bPhCrWhdMSi47w5GTEbcnBBO1yvE/SZsnYA9SP9pBWtADsmwTqgOj908Ope3g4jY2OcD0qfoxxvVg55gKH00/29yzSjhX6o3EPF6p4k2pFahLSR+xPtgH3jagz6R1Jd8EavA+gMnYKQmc8h8dIHeZV4bLPIRYh6aB8hkTaxV0z4aiROiBlox/0ztahNyZIUkc7UgnmizWC2m0gNdjn0wghaeCe7FHWLs/hfj/G4sSxMDCec1gTkhYDJDZ5jyh3C2KjahOTC3qMPOv0i1IEPWO5uKEaWhvtbeEZkIQECcfbVBNkpJV6NFtPP9xv5yEaHXYYu9ynXcjMrsH5aL8BHcMrVCMSHNKvcM9WyK+F8dcNYdJmyE2XxPardhSqxUmcffaU+A3SfqNu++w5/8VCCtcY9T1vtjDPy3lirhz2GPnm7VnEwuuOgu/Ip1zCuXx9frdUSojy48Jn/CnlxZ77fXwrR5x/3UqetenKu6X2WE6bj8T8HO0q9S/15Hy1vYdXl/Q1zYqT1jFyoJ7uoy4kMl3OOfdNub5OLA771G//6+fmYyz81z9xKk5sbu+nf/KHSwTr3Eg/9l296VPPIzn8ANv/9GtzaWywJf2Nzwx8gKvqSes98OegB3bx1HjvXnqwsMS7gldJx0AaGT+Thk8i3Y6VIBZcn86nB1PTaX5ZOz4wQf9QGp84y/wO3tzDscfiJPhnMW014Enw3HPp+ZE/G4L9DYjFjRx5O6bBPEE6AeZ/gWXp7Tw9VqbP+G2a6ubEWvkd+9pzlUTxgiBddbKvSROg2nN8CrFQTaL8Fqz624bssKIZNha8rJVQbEIuNgD7kotQhwIohBoCINEa1hpvK7GImBYAW1+sfsLGAnCkHURpQ1SZi63rLh5fJBbbgB2lFga/ysQCEEMZvoCUWHht7AFOrlyaH6eCVKh6E8H9AI4SC6Usxa5iH/AVCe0P6iOxaJdYnJRYYPCr1AJwoQpX5EmrlGwosZgBfN9GJeYu4KaLY5cgFVcAsOuUO8kq9juAj0YIwDCgYgIgM4Au+Dp9aUwHJR8GLjuJCpJB9XohAgIRjZ8N0rYIaYGeZSNW6ihFcrVcoGJblVZIKlx51pWlAN0V+Wir4Ir0j6ifq/N6AfKIEoNeytOI208v4E1JhWBIac4jiJCeiiRKTylf0PhkbiFiDzTRZ+fw7nMN16cX8HCkL35tFkIliuuVWujp6DHXXZ9+mN4mgvQ0blNPQCaGUQO6evlCrJi7Uqvhs9INv59DPUv//qoGKSkJtSBUgq7fvZ8WKFvbnb5Tp9NFgP3l0SHUnoijwHXaj8xBQhxrGm4PAvy8XtWicG9L2yVK3ttr6Mef7+2MQIIauUtsNK6fAqCrVjXHfos67aHiIqiNeAi0twWweRrSchb7iGvjuHxFHeYk6kfmOUX628vr6Q0kFrp/VWJxEbB8FvCvq9LXsdGYujcTeWq/8crVi2kAdaAF7U8A8N3crxchaQyDUKW6R1veQfVren6BPn/ICvpaRIvuGKLdTGwanytxmhfIA5LXqZ9RuzWgVm3uDa8lRsTWylKoQTVxzEEmMO8bxHAcydEFPDNJKCQQ2orYbxpN30XKNEN5C7R/i+d4D3KgncYu/WIEkT3cLTdAJJTaOQ77sUMYAkyPnx6IwHKLrPZPUfY6e13GhrSE+yKZ2EV9qpFrCXARhs77RJeUDLXwLBjgTi9OAvUtDMX3kdY0EMVbl7JKLHZQSTzgWUTEke0eGLvaT2mn08C4QNyRdrl/B4xrV1WdbxzL2kiQeTyvGnS3QxYauV87jOt15gANvhshO61I/7S5OKAvJSTm3YTEqin6gDmC9ury1rYbm0Ni39OAVJBr1lGhMq8W1QORWPhs+Vx6P7XvDhsL8pNwKM1Q4iex8LsB8kJiUalzzD9c4+b1sdGOyC8OVI9GGw9/lcRlTx/YCWwx3+dvld9xNL6XP+Zf3SivbLncw9/leO3e/I9cz8nDMnPK2hxqSjpsYzXDZ9S7pj7VZJUvtfmWc7X1sazjdbGupV11YlF6Le+PEwuPrm3up//xVx+iEtWW/taPHLq+9bbMP8Uj4ZJONZrT8EB+9rxmi0ixj5Z30y/9q0fpdF9z+kuQEreh/sM0caD+p94Dfy57APxy8w/Tv/ni63h+ZIGItaLNTWxhr3xH+uQnXsZdO14Tn06l61/94/Tlt9FGQBrfyLtlr2kgnfvOz6TPfOdIal2ZTdc//xvpX0/zCmpCO6bvk+mv/ScfS/0NeODEy2QavwRWy4t632oXvE9iYTF5ynRiLKtZvknyKzNXI8P1yjEny3y4+jcm+/LCqJn8nWjjQ8pyTZmgg1gAyjwfxKLsOVaIRujGF2LBXlKhv+3iGco4FtleIUfkNi/gfhCLNsBvqEHxkg01KFYCw1VlqEFlW4gisYiGWP9KJfckFpCZLYkFZUgwNODWSFXWYLL4xDUSFK/NH3mHK+naAWwBVkJSASDd5hPqT+Sh6oNbeJeS+OARph1jzY4BiAX66V1BLHA7CxgI4kJphWAskOddgM0kHzXvzwEcLgMuDyAKet0xwrMuKU8COi4AbM+juqIdxyOuW4Fw6OO+B6DXAcjrAjQJTqyvUgsJwh5p9TD1BOAq/9FAVpsEAbPuWXVBa56jAB9XRAU0XhsqX1wwD5DS7sCgcMuARKUVpyE/VwHIVwDMkhGlBvag8TkEyDMA1xuA3FmiaS/yfR2Qv4m9gQHPhllxv4BK0lnsHDq5ViJkvTPZyVIhJQU3ZhfTm/em0+TkPQzJW9Iw+vnPXblIrAR04SlL1Z95ALnkbpRV/gusvBujwnunatck9gI3We1/wgq82+j5c0gkTqULkArVgbS5sJ+WAihCwPguSYUShMG8Ub6VTDwhryeUM4ra0RhqQAPeG/JzfRjNsVBhUzqkO1yNn9dJu8/YgkGHS9EmVtKNMj2EZ6Yr2Dm8iDvaMfTyHTIzkMm7oQpl3AfGI/0+0tkWHrQEnndnZtMCfactwSX666OocfUBQKdQ1XpI/1rfFyAWXRBK1Wes7yQqTfexTZiem09bpLGy7aghXTwzms5heK1L04dIJ7wva7R9GHBvUDvgPDYj3DPObVDnBkhCAySBgRCgvBc1njNIPM5BeiWxTyEmrqq3c/+6WbVXPewxdX7MM/1EwKyKEZKiPUE74Nk+MwJ9I/dSNcMeJARDqBKNQJh0v6sB/DR2Hsan2EONqwly4dYASW/Ek1QHKl8NPD9GqF+HvDrfNALSmwH3LdyNxi3czM7PpwOIxb4qVIyRPYPjSSiwz9ArlEHu6DTKz1KyMPTmGWAQ8ezyEuBZMG6JxGIfO6mD7u7wBqUtVzttbaI/dmj3Jm21DZIb741SFj3eOf71dBUezSAXqk3tME9t01bJg1KSE6TpVLqAJEsVSp9Rg+e10Q6lhaGSyRjWI5TETWNunw0ljDouMLhjvwQFcqFnt5iqoqf8Yy/XbNWf1S+czPP94REqFJtze/7m1He4HaZ0fFaLqKbJX6o/qwlyDnENX48TCM/GudrCyP+wtHz9N/u3Wu6xDGrzL3WuPVbthGPX+bO2Dd+OxMI5yfH0zWzPIhbm8wu/MpuG+1vSz352MLJ9srqX/u/ffZLuL2wzdzTy7t9nTm9Of/37+yEYLelLN9eRdKAGeWz7Oa4fP4WdVH2r98Cf6x5YTH/wy/9n+uLuR9IP/NCn00fHU7r3e59Lv/XWQbrwiR9KP/Id/Wn2S/8qff6P7qSDFz+bfvx7JlLHyu305c//m/T7M6Ppx3/+R9PY4pvp8//w36WTf/tn0if230y/9vf+KF3+H34+fcfqq+mffu7tdOkv/+X0AgRFm8RvdfsmiAUvECZt54nQW45vuRoHIqI4ntPko5W/XnNssudAXO1kHZNweQNVLvGYL3oJxAGf6vdyzD0v7UNioQQhx7GIYD4VaYWSi7Cz8OXOy9r8GqmrdhTqdec4FkVigSoAx/T9HxIL9sUrlPWNOkfjBcqoLyGx2AiJRbax0IBbYiEpiN6wTVxnW5QmeNTWqicqqdgGiGdioeQDIEodc0wO20xqrlGNqpmXv8SiFdWHNm0sIBbdPRicAnB0lWuMjCBL1heAtkzeM66QAwoFtxpiT7BSqhqE6jY3APT3HhO1GBDjivlLeEayL55CKua4TpeaKnh08kZwhVMw0gIo0YhbQKIUQOCrF6RNRRy2kGueurIMsJGYvMJK9lkAo8RCiYFgh/+R1lgPd4nKrEH0HIB2CwA4TNrnUcF5Hp19JQVKHFRPkiyFvj9g/HXS3wY0q8+/A9jdhlg0ABb7e3rSGVbPRwHK7bTVOms7YtyKWLHlYVHV5BYuY9+aeZjeQepwArCmN6IXLp0P9RkrNwOIv6deP+BcycclAKpqPV20YRHgeY/yb6EKpctYvYlduoqr1NMn0zjAWlUwx43j2ajIJfCfbddGw4CFq3wMUmebtVuQvJ2GuAyRxwlWqyNmCP27BUh8wlhQojP7aCk9QoVqi74VbGo83YCdhSp8gwDz8AyFnchZyI3PiGRCj1D3uTcP+WiwHRIAQK7xMRbnFtIafamB9eXx0fQyEhfJmN6oJonX4bN25RReorgfAnz7zYjX9yEIdyFWqwB1A7m1AG7PDxO8jz46BTma5n48ov9WybsHUtAGWOXhChW7xzwXa+Rh3IkGiSREoYFlly5cq57GVmSEPpB8PqG9tsG+7eWYVglKyjTan+faJUjFtqpGlKEHpvCm5MgPcI9thTYFkIVByGkD/bjGfLAAmVlDlWmXejdoI8IYbIIUtEHKevEGJWjfpI1KwSLIIuNGg+swxkYS2Tw/h4taPEmRzx7BKY227QfUHqpNjk8s/vPCgeCeMrWn0HuTdjHaw6iG5TO1h01H6oKQoELXRP+0AuybkHDsQyx2aBeTWWrkGgP5BYl3zuC51NNVzFWMa13l7jFuJDgugjRRZguEAspCT7BxjfYyGtgbvNLH03FpWh0hhESI3z5f4XaWftI26iTzg4beOi6IxaM8gZljdWMIVbZCGPIcRaHx7xDCx8WRtgrGDw9xPGdUzuWfHKPueSt7cy7bsWs4XN4p5lP7/TAfSuLcB90sM66qubY2l8M65ZyjDOeqmoJq078XsSh1Lpd9OxKLL83sp2uDeAxkOvig27OIxRTk4Rd/azF999XO9JP/QW/cp1/8zQXewwfpZ35wIJ3sbU63ZrbSr/8Rzil4Z/2dn8C5A8/eyvo+1y0gpWhJP/VJpI3crL4untsPWql6+noP/PvugYOp9Pm//w/Smz3fnz7zA9+dnhtCm+Drv50+98dLqfelT6fPfk9fmv3yl3BOgyv+T/5AevE0FdzFg+VXfi/9i99ZTR/7+Z9OV1ZeTb/9S9fT1f/ur6WPbN1Kv/m//Ebq/dm/nUa/8A/Svx3+6fQ3P3UG/PGnwCosmve1GLyT93vthgrwwcHX3rqRbty9F5N9efic9/0ev+N7fIt583CSZIKtviBqsj12LF9p9rw8mcBrJ3K/xzGOF0JRJBTV4wAgv6sKlVf5WfEvxIKX7yaAv5AK93qL0s5C4GTZQPBMKnhxC2SKOpSAyRexn1jp4+UcEoFovEDX9tF52FhsFWKBxELj7T0+emyxjNKmqD/luRotsPZ3iVGRiYUqXEpaAAy0RbJk9G47xL5xFbMF0JmJBTEs+lCDglj0oA7VLUBSfYI2CAZyvfGiREHzrOhOQxL0ECWAHQM8DgJQjEMhYL35aDmkGcZb+BiqI92Aik1AiqpSM4BT7SRaIWCunrrSqfG3nozGkEJos2AQPEHsEnU32vQTgN6aK9KAnD7q89HxYUB5T+olXyNSr9MvbqcBnV4/DTl4ffZRmpojbgOgVJWji5CK51h9vwjYVOrgaisK6SEJUZXoLcDhHa5zJX+HNmyjZtPASvgAq8BnkViModrkJa7aqlk+gdRkAADdBqASPE4D6O/MP05vTz0MAjIOKH5pfAxVMDznkF7j4/uCfoiXUodB1MTOIAk4wyq47ZwCWL99627aAqBav0tXL6fLkKFRQbjjhcK9z9zi2AtUdZtq3ZewAVBNbRlC9ogy7qOS1MXvUdSArlw4G0bHSn1cmdbFqVIO2zmJN6kZiIVkRymcMQx0taodkPUzjsQrkIMx6iiJ1K5EMqBr4IeAeI3ANeJfhWysIjlYM5YDefRxHy5jo3ENNa4WxsV1iMV9bBqM4aAx+BBEpZv8HX9P4n6jngWwX6bu2wD7RlS6RpEUnULtSePhB0gFHmlcDrmIQJNIyZrpl03apI2KsWQOANCEng7ja8Fzuyo4SKl6uU9uS9zTLeotmTb6tKv6dGqA7CXarNvaHcpGvEAekBTBuI8K6RtYcW9GxaldAo0ECNaCLUJTkAvjzOguNzF+lCS0Qgo6kVichFjqmnaTZ2SesauDANUO/TSwcKDqUeNjpGLYo+xKaJB0aFcRthWSGe45wy2iXMczb3qIn2LoZp6lRvpNCc0+xGKH+7UvMZGQQMQaqV8z7W9B6qnqlBHBNfpmoop5QPKgS9x9n22eoZCicJ9YGamoabLgQONVvfJ++qwobTAQZSvpnLOcQ5Q6OS/4DLgwIJnIUos8xznm+sh/CBJ+Sqko54PQ5xk57kuei/JX/zq3le2Zc31+Q5Qkx/Zem4kJs1t8N0Hk6U+Pxd5d5UuUl8ssZT+73JJPJWeui6u83onBrdS9/M5Hn/235vpSnvlVcor5udTneAa1x3PNrUKlRWVfuaik/XYkFlMM6dMMeRw6feBNYuHt+ps/fJKFuANstXbSP//KMvZg++nv/Pjp1N/dlG5Ob6Vf/teP0l//DwfSCxMUVNl+52sr6fOvrqSf/VHULU9nqUTdxqL0Tn3/4eqBvXTv938l/dYbB2nsygvp0khDuvfV/zfd3hlL3/np70ufuJhV+w7bxPt26V56/Q++kL5wZyT9+H/xmTT85E760uf+ZVp44fvS80130+99fim98uPd6au/3Zb+4n/1F9L5NhaUDjP4lr59A2JxPd1AF92tOiH6PY7kL9WJP36WMxx91qTNseoEW5OPacvEatZ+N13s+S4Q93shFiHBKL95eWcbC+M+ZGKRyUXFcBtQlwlGNuo2jde78hZxLHhxC+JUJ1JyUbxC+SLO0bfzPl7SFRBhB9g8JRaqWK0rsdgCOFSIRUT4BqAoceBvXnmkvq5AunISH8BTkCHAqvvsASqTimifAIdN942uiLbw4m9FOtEKgG6DWJzg0wWx6CKWhcSiibrn9qjnjToO1z4FyM4BojTKBpvh6SnHf1ByYGyJW7gjte8G0Of+GED+FEDONms0fAdpwm3A9TpgRyDbxiqqqjZXWVm/wCqvgNlV5JVt41pkMG5wt3UIQiP9K1C5dnYkTQBOBwB6Skm0vaDodBEJSSvlaIfxdVST7rByP4ckoY17abyGC6jhXEXyMABQ1KhZWwuVXrz+bUD2bcDnA8iPRqw7i0gskB6chBlfuzgR6lCqeazRBiUn/ejL65+/FXUR3dlqA/CAdt2ce5Q6IAZ6k3oZIuOKrVIWV5W1swjJxRI6+dSzn345C2g3YJ/6+m/fupM2KVflpXMT59LVs0gL6JcudfXp9xivlUGyTb89oq+N5yHAV3KxAbHTc8/C7FwYyNufr1y7GOOQgQ6xJEgh9dZgVzUkbTtm+KiKZV8bkXqX/rSfT3LvvScvYbx+GhJghGkJksEI22m3tiwac6v+Nss9XaDcVdp1AJDtoW/PUvYE7dc9613IywNidKzRn908Jz24PT0B6WkCAG/RN6uU95SxsA5xkNQdMDYkdD1ICDqREDzm3ks6Nvg44PQc1YIkCSMAXLQ2hXrcPm0wHgRiDaJfE0uFe3CC69tMQ703qOcO7bP7jEmhpE7JAw9BOGPYpQ8DfFP/IBa6vyWfCDJJ/zOA87MAYEd0kfZRF9rlHh7sIEmEFMFcQuJjUEntlfogsoJ7CZmG3pKyICnUlxmIOgL6ibidkBAdULd9DaZ5DuNDWc4DjdyzRtp6wHygBOUAb05KKZrpwwb6i4c7njPVpvYpFytvbC8AOvS5kbO1tWiFkDTTL3vcnz3GOeYeERBvl2f2wHIYCxIL6+qY9PmhV0jEM0UZ+4xNuH8sjoTEjn51MKpK5bPjYonj3SCQSiqUYJmH18QYZ6wMc58NpqfULNSnzJ8tZmPysq1xY8z4XZsnK8fdRVoTVb8cnvcbfeYW74+4NP+u5BDnjv7hXhw9UEqrHq09X/L3ZPU7DYiiqlcc/VJNd/RwtdzaupWyynvuWdc+61hkXamHz/qR/iHTb0discaD3cHj9M2AFonF/JJvs8NtfLA1/Uff3VNVYdLT07+AbPzAy7hibyt3Bmck2FuoAvVXv7c/fbTiBapOLA77sf7tw9QDOKCZfy39zq9/Ho2LZVRhWR9rGEkvfs/3ph/4xFUWa+NtUGkQGHOLxdHX/zh98cs3UsMrP53+s08M8w5cSQs3vpD+6T97Lc039qbxj3009V3/vbTzvT+TPtrD+7SB98PgENobYslvrW/eF7HwhVIm0CiuMsOWF02pQzUNq9zV77X144IyOXtN7XW1k7DfTRd7vn9gYgHY3wIQZGkFUgtVogArEShPYsHLVmLhWmMrL9xiYxEqUQD0EiRPNYJMLlRzEERkApTbzc2DNkhidIspsfCzy+rkLseUSOyxwmKaUHuiHYIX41SEwbnnAQPegPBfH+lJTZqwrfClYydQmLrbzQCXNsBzu6pQfQT0UmpBPIsTEItW1Cv0bhWAAwBim6zrGuUbn0IPQ67WC5wnApg0RCC62wCpNc53cPxlVp01mnYV/il1uQPIfIv4BVMPF1Ejwb0pfTqOp53nxkfTZUC/tEcVC4G47lPfYaX7OipGTwHerqh2QybOsoo+AWEZBLBIZIxpoY3FCxCUU4Ak7/LrcxhDE6xthut1QaqtwRjXXDuDC1LyUJ3JFVZVO3QFa5A5o1hr+E1HpW1sHRpZhT4NSHv5yoX0/OXzeLBqC89R2m5I7LyH9ofubK2r5OL2wlOAFmJD6vIyBt+63BRsWc4qwN8I2rcgMcWQexjQLhGaZlX+5u3JtDH7ILVx30eQkDx/+SJuV8+kAdoZ99p7yNhSemF7lVhMYiMxTZ5KDkKiAEhdxWZhhP6+DDH4LoiFKndbjM81+qmX9pzgnkrgtC+RmM3wech3yYbSB2MfGLn8Miv+z5FHH2TkMdKcdt7gxvUYQVqk1ELJksEKjW3xgM8j7ruxFjroDyVYQ0g8mrn3RrtemJlLG3h+aiFNK6TAmAoJw27VcXap345kGpUhXaAeQDAMwNbOuGyjDkoTNjm3i2QAB0Wxat4OMWmk73YBq0qsdun7fQgJuljhbtVnTLC8z7ilq+KZECgH6OL+NKJyZGBIPS+p8njAsywpOkACIbFo3ISgmJ7tANWqmEfIU7uJfSQSB0gGIgI2bVWFDJEaH6QtAnPUkU5gH2P5ej9ao16qJ9pWlv159mgEz/IB0opEexP9Hm5tWdUPYkF/RIwLntVGxqqaoBpb65pWz1B+jFLugkJE2UYFSpIU2fLwHDDeDpCW+Fy2Uk4r7bLvdnkmtyFSe/SrhuLZPS3jmLHSRF1V4XP9tQXS6sKGc4gSTslAO33WwT3Vq5oSlFi8IK1zWwfPhV7OJA6O9TA055zXqQLlGB9hDKs6qDc3qkWD+LijDUfn9MqJ6iye08V9M71PAPkezvIlfc6rkvqZeTr1VQsuCSv7yDMnKFWrpijvEw/EOKjZeyyujTodno/jlXSHNfRo3mrz9EhJU47X9kl5X5nO46UOtd89V90q7bCPSj5/5bOfrZ7+s/7iolIpt+xryyz19p3kJ9SLmctOokL4QTbfE98sUJFY7JLBT3+qn8UhjFV7WDDJLnCqVfjNLy2lP3hrDbfXLEQcK8jb/aMf7wk7Cy+oE4tqt9W/fJh6YO9R+nf/1y+nP9i6kj71me9LHxtvSLe/8Ln0L683pkuf/KH0wx8fCQ0Nm7S/s5oeXv/j9HtffDMtj39/+quffSnpVv5wA1NvPU33fvfX0q8ufyL9ZMcfpH/8FWwnd5rTmU//p+mnPjWBBse3piL4PojFvaiPk8wzJ5/KiyXOkzJPRnwpM3DlmJnEocrEbjuPtNUEla1M0HnlN78EPFYmOL+H69LKhOcE6UdphB/Vk3zhhjcoXvTZkLt4hsoGkb4A7TpX8rKbWaNvY8fACzmIBXsjU/u9qEGVJvnStDUahxpvYpPV0C3czW6rRw3Y1BA7q2ZRL9KEsTMAfBcgYL0lHao7FUJh0CzJznFSYSmhdmU91KfWnoIV4vYgFhie4mazvaMHaYvEQi9KFW86FXLBayOtUd4M4MbVdkGv6lB6fVHX/zYASwNs12afB9AbV0GgqRG3HphuP15ON4j5MP9wnhXqlXSK1f1LuFUVRDcBTAq50B/PDOo1N3BjOotb0m0ArxKgEYjFOCvCg4BL3dMa40F7gyuUJYlx5fRtyngHcjEFYaATUy91GCGo3HmMqjUeVr2oF6DTQ70dMG9ACEJlBwDrfdlDDaplbSUN8123sS/xceXVtuqWVjKg2tWWQI52GrRNGwfVodpAeBpcv3xmKFb7BWwCXQ3QdfPqCv4yeUgO9Fi0xFjSNmJ6eiZtTk+lVsDxKF6ZPvLCtXTl4nleejnI3yaSEZ8DjaCts2RGw/MpwPoMkhaNyHXtqjRogPt2jtX+V86OUDvwKy9uvTNpxxIBywB8Rk9XOmTskQfcy1BvWgagA1xPcB9H6bMJbCK0A9Eg3CBtSowujgyGXY/32vgjs7RJgjKPK9pNAPYB39tJ28v9aWFlHwWriEytalknK+ENHN8FDK+hXrRDW/Zoi6vyu4ybfVf/IRZhaExZLRDeRl/q1LWBa8M4GCCrUfgm93mFfvf+b1F/3eW6st9Kv6ou1MyzuI3HrjBUhu7rAakBgGxUa42TlVpI/nUIEOqMFdUiOjA1GRPDD8892cTYVSoQZIIYDwc8MxHEjrKUHGAhjeoc6amnNhLNg8TvoJwm1Br3JAHcN6UcB4wFR4ySDsRLDBzIDP2PiCUbbJOGAQhJsL0sVLDSzwNIB6E+xX1ppA8bkXJEhG+lFJCRFkhOPDeUvcW4sqwGytH4ug31QYmFsVHCsx3Afh9iF8bh9gf3SbDfzLwk0W6lvqpbOZ7zem5WWdQhwQmJRSvX0GYXL2yq/SexMPJ9SCucU6h/zIPUQzuoUe6V5EJCG56huJ38d6qLjWTRx/lX5aA//BoJ85mjf31veF1OX/am8ZDn/BzfSjr3z3rvmP74udpsyvVH8q0pqPa815XfpVW1eR3J4xv8KPmYrNS79lhcXumL6AA7jnrFe5OO/HYMkPcnDo9v0J/PsrE4fsnvvrGa/vmXl9N/8xOnw+PT8fO1v+vEorY36t8/ND3w5A/TP/rfv5LaP/0j6fu+63I6zSslLXwl/cY/eTU9Hns5/eBnP57gGrx+UEm//eX0+7/7x2my82PpJ37q+9K5rAVYberB3mZanv7D9Kv/z2z65H9+Mf3bX/hq+vjf/Vvp49ufT7/495+mH/y5z6bnTrFgXb3ig3/5UyAWzIs1k2OuAi9kXxoeZyv7/KMyZVcm1zL5xjn+OAk5EVc/Nb8DeHOuEIxMLiQVgnXViTKg16BaYiEI0UVltq+o2FiQzustCFgQ4E+jyIhl4Qu1QixUQxEYqlKSiYX1qqkfsFpiEWVRhoa1W0gsNgEumwDJcBkLQJXwRH252HoGCfKF73F+l0+2ycj1sofsBzf7JyJ1A2haUcNow81lGzEsTuCqswOC0dJBpF6IRROecayroEFQ1047GlGJUqd/kfpILDQo7gdYuDKuLvk7rB4/BuAYGO4idghnAajGtFB33/QPAL43UVWawdh3mZX6bsDTOPr45yEMqhf5QnQ1Vm8+j0l7H3IxjXqPUZZB9mkY0H2W9MOo2xi3Yg7A/BRQP9bVnkax1egEMN5CPcfAdQ/RzW8AcA5ShzMQi8tcGypd9FMrHTLAcVXWbpD2OmpTd1VpARzt07YW7vUpSNvzSFJehPichwCpa+7qs6pDGl0/pXxVQgTpeh66jfqV3qQGAXXPY2dgkDe9CQmUtUF5DAiXXElKJClKH8xjHuA3j8H42uws8QdWqetAevnapXQVX9F9rsyT1nJjRbwy1DV0V8pg4DqNqR8xLu2PxwBtwd1JVqEvR50bQ1pxH5LVRLs0nO/inEEcVZ/ZoW6Pyd/7OYdnrC3ylER0gg+NA+K93yTfdsblEP11hfgZrl4LYA2MqMRIyZH1MDK4AeC2+d0F4PTTg5SkHSDdxnOkQfAy5S4xFp7wpBxQT1fkHZlb9N8O91KphzJISWQnXqC6ANveoxYImyvmglgB7SJjaVHCBHA2NkSs6lOXFvpFNSDW7TmuIhTgGALtqv4B1+2x74VYdNIGV9FXuHaFfFbJJwzZ2RsNuxHSy8MXq/ZKBvZpi/YW4dqV50Y7BbIO2x8jUzcggVDKYXMaIETNkBjrIXhXcrKFBHCfMg3MrZclDc6VDjHJMGlUpBkQgtgY/0ogGgDlkhG6OrUwB0h29UC1zTN3wD1sgCS0cY9cKBDsr3OfosX2kc8s47eVuu1KLKjbJnnus4gQrnRpj/WzxGaOY6UR9x3f1sSsQU3QeyNR4LuLJB0Q8k4+pm+AUBvbpJk+aAsSo0QT1UquifaTn3xIiZ3SikIsVD/Mdha5mTEjkbZ2cz4sM1XkZePjiPtjiStn8jU5FbWLo/nPu9MfB+TH3xU1Fx9+PSyA7Cv5c6zkXvI4nndM7pVcStr4ybXVWpb8DkurfqteU1t+9ewzvhxJxxxP3tby25FYPKP17/vQ+yEWGmr/0u88Sj/6nT3p0y9kW633KuB//sfzqIg2pP/yL+W4GO+Vrn683gN/rnrgyZfSr/wfX00t3/P9QSyGmNz3Zr+UfuM3Xk9L4x9LP/wXP57GcBu7ev/r6Ytf/Eq63fhC+uxPfm86/64wFUgel2bTVz73K+nuR38u/cenfz/93f9tK/2NX/ixNNH8WvqH//3X0pX/9q+kjw31pXdd+gE65E8mFm9jY6HxNluZkMu+lOGkK3GI4068/I9Jm5k2jnugdiuTc83EWpunE3QhFUos3Mpv94VUlP2eUgBX7QBOhVgoRcjEQvsKCUUmFxF9W/AAOPNFYv0kD9ngUVLhR3WY/FFa4XnVY6LacRl14AUQkgjK3YHEZIkFQA1AsQE4jlgUgD/tJpRMlLrGHtAZK7O87FV5klCU+ti+2t6yfuojCFJ1p9nCyrFSi/YebjrEoh2C0YgqVAPEohFiIYjTfa4rmq5cCsyt+CrgZoX+cXXTqNRu65R9G3Ap2BT8nEWNxs8oAENSIgBXbecWQH4KlagF1ILMe0wvSaiPnBIskber+QIQVaoWAXqTGFM/xpXpNmBsGKB+gWjUZ4hToJG6wdZ0R9rTTqRuSEorgOnuCjYPSAaMrqyXn2Hcr55H5eolCEnkS/1UD+qhXm0AQFWKbkosICKu1OppCxiaegGWV1m1fwF7gauQBFW6fFFLBrTHeIiRudIbVcJWAMaTqG2pRtXF6vpVrnnp4rmoq8RiiWt0o7tC+1VfUzLjWJQMaISu+9c11Jgaaa8uaV9QikM7uwDYVCfumfrrZdx6/RpkZYm2KAmZI2/tWB7wWyNrA/qN/X/svdmPZtt5n7drnufquc/AQx6JoShPiCIhiGEYCAQhsGgjgWQHSJAb5ypBglwkf0SuchFAUmRYkSEkRiLHohxLimQzkmUSsixRNkVSJA/Nc/qcnquqq7rmufI8v7XXV7vqVDePCBNhU9/u3rX3t/ca37X2Wu9vvcOiflPU7wgG+x70PoZWo9B0DsA3C0M6wakdxC4gbou2WZXJZpVfEHvMOUnG45wa2S/RTrqxfQuAJlgY4rf1tq1XLQNgQkmV+2ZsQxcN+5fJ/84sroFhTAUGO6T/yPrSbs+pe5hM0veL3IR+e7TjMRIYDfsnqfcsQHEB6YZ7YGgXo7oNXHT2RvmAvqCkRJWwMOekrYqQTmKsu/YvZAGoQ1oIINAeww0hBTM3qPc8bTlJvdaI/wyar5O/O2prizHItx0DbtMm3TPtXDSOpm/ZGGcDpEN5vFeS4o7dZ9iQGEdwMRjg5rcCI015jsn3ADBybP/xu7fG5KXBt2W3z9vnohppoT35faqaE2pbSlrGsLly1+whpDoHvkelaRCVMiWiqlwprdinrDQjKl4w+bRPgAV1EFi4madg9BRVrQHiDvOtjEHoMdJS/WmIfmHZTxnnjqFxPEQ5TlEOQag2FgFjlJ8emX4ooAm48Bmn6k+WXeykI5BlQOFN8rluX6Pu0kMwbXcODbim8b1yWK0c3hAo4fKntKXv7P8eZlWO8rs+L898WRNrg7WXi+Ha0OeJXQzsr16h2lc1LM9rDnWuuZD25XgmdSm9VKGlh+9q/PSHNrsP5V+fd64JX/PLlZT9T9qm2QcWHWJx+1GAhTF+5tdWm4fPjpof/0szONPAmQn99xG/v/L+fvOf/pWFXh/8BQDINx8dNp/5sTkWx7Axw6h7esKvoH/0KfC9TIGV5gt/7xeb39v7RPMj/8GPNp++jQfLf/EbzT/52kDzsX//P2x+4t+72zQP/7D5f3/jnzZf3P5481f/459ofhgt5hzMEUM4DVGKjbpBs3nvC80vfmGh+dv/+V9kEfGPm1/473+nef2/+y+aHzn67ebnf/m0+Y/+9o83P7A83VOt+k6o8m2BxTda4+2auANg9/C3T/K3feXAO1CWN8vzNkI3bh2YfXX5eQZunhumhqv3Mucy472V/xZYxPiZyfqIFUaBRVSUmKizlwXX7DXBZKyKUtIkbctYwYMqBlVaEVUomJpcYXKKxILKEUcG01X6U1QezEupSIzDW1BR7C30PoX9BGcFFjHkllloz4AK0nEKy0VCcLj6mWv+yMBwSmMYh2E6xyjMiwbcgotRgcUUbjV1Yek7GJcJGKsJQIXgYBQmSeZAD1HaJ+j5RnUKGV2NiN+DYXoCc7oD83gHhuh1mNA7nO6+q1//dcr/Djr7ekjSY46Ow17D0PbjgIvbMCDSQcAST0SktwH930VasYLnpGMMeZeh3dtsXPf2nVtR69mCoVqDMXSlVi9WMkTvwvQ/hYl+pjEwDKPA4m0Awo++VuIIGPU2pWcbd93WW9V7gB29O+2R3ilMEiRqpgjz8YUZdpK81nwKJn+S+gg6VWUSVLjTuEbsMvgCi7WnbHAHOJg4PmBl/0bzI0gdPvWxuwFk2gK4wZiG10orrJ+uY8vGf7iOpQ9FpYfrFJ523O37Oqu9GvG6GZpgVCNZ2zpSNb4CXaYKvtwA7yHl+YDyvA+42Gz74yxt5Y7kMuqbAL5TGPdJ8lfN6Toek9yzQt1/V6h3yec5ZdTe4ilA5RlpjZPXEkzqXaQmevm6AcM8S9m0I3FWtf3dbO8Z+Sut2qLvqvKldOUmzHw2JiSeDKXG4W5MZ3jB2CBgQbVA+5LM8COA1fOWMZ8AjAss3LNiFmZeEKDqmuH3YKIDoGgbvWJZ/zO+TQHAMO2iFzAlX25G52Z3+6R/IKMcOuoOdbi5RT2mSeuM8q6Rp6Big/RsQ79lD+0cZPrdoVwwwweLhAdVIO7doE5JAh9ipBpxd6tnKBj/Acqkt6Uh6q9htPFk1D17EhrTRjIgI6/UQtUnP1jVjIahaX4T72SaPUSUMAAwxtiozg0MAywg2AnfyhCG5KrZKaFUVdI+5acuTeMJjPqNCCxodwHTge8FSDD7buI3Sd+IJJIBIpIV0jihXErIgBmUmY4DvYapq8BCt7LTfH+qGwqPMoYQRo9PMfw2vHTmorqhEi+9Qi1Cb9tUiQVUIKZn92h/MXaGXacSuh33MEn+9mLUsfv8XQnXfe67q44XhalzhdcLYWyX9ji/szTnR43r+/O4/iJUJ36NUdPpptF7d0X4+u6l15buJW1pezH1PrC4SL3/6bMrGd//m8+8XMKgl6h/hOvNP77HXKFRB4c2GT/0xniMt2uquqr9xX/yjDFOqXLT/Lc/eT2b5NX3/WufAt+zFNh9r/ndz/5K87k/eAetAxb0bv255q/8xI83f/kvfqyZH9lu3v38bza//tnfbL6ESeCFY+Z289Zf/lvN//CZH7jwOD+Y3zZ//+80/+Pf/1fN2vFy86P/2X/V/Cd/4Trz8sVx6cMRX/7kIwOLC9mUGSQp+7w3OCZQ/VWudXDuhWrj1oG9plsH/Z6UgnQNU8PV+wALnsdGgclX5lHm0/0fNKS2Ql49i/E2jH+ABSucTMQy/AITZr8M6Rr2BlwwKVdgUZ+ppqK0QmBRj9hIwFAcoeMsY6O0wg349lGD2hPIoM5wlEkfFRrK1bOdsMwyC7VOXk20EghCVFpUWoUmPodmrpDqyUZVkWFUokZhZIbZoGyIq5t9sWkAhrQYXsIkTcGQCCwsv4zLEcyHjIqG1qpAyAG4kvoewOIBzOmzg+PmGsuWd2EMX4PJkzG0WFswQd+CCb4Ps/OYc5j6vAbj+vbSbPMGK9QkEZUpJSECFZnde6zSKt3YQ1VpgQBv487zk6goCUiSJrSRwURxJVKT90j/wQYekgAkJzCL17LSPt/8GMDiFnnpFnMdMOIKuxIUGXI9Nj1BcrErncl3iHzH0bP/BLYKP4S9xKdfvxNVEGm9iwRAdSi9I60RftX6UpenMJd7qMSMw9B9Ynmx+UtIHT6J5EL9cstpuhpOxx0w6WvELeCwru6ibR00wj6l/7nKq/GrK95KcIyvkSwsPav/eETh3rnukPRUrXpKWdxZ+5vYOlgm0x3j3QxNM0O7yWzPcr8IE3gLeizA8KnGonRD18G6vVXq8AHSHoGF0ohrxLnNJnhvQLMF7t0JXVUY1aOOGDxWEx4AR/1tf3eAt3/YTwSJGr8LCmTa3STw6+zlYblODSOwoA5+B4LCNYCPjhAUlSoZU5LltyMgdbd1pQvW1b0r3kdaofG5diWWYwg6jJH/NGlpRyIIdi8HJWhKYuxztpvqbzOEmeVUDUg6a3y/y3e+CZO/RZrWW8Nmv40AB2ipvZINoJcojaNltkE5ASZw9LGViLQCesCBoHaF+hLAaBCaRUpC2bRNSiK0pS5n3XMj7m0FU5QldhWU0VB+X8bL3hZIFwQCSo1GiKOaFq71mxP6huBgiPoJSOwjR8SzHYw/DmM+Qt30JOU3cEI5HceO+NbdiE+3ubaBqmXjpC2AFVwIpCKlJT3zUX1Me5ERgUXaQ/eyLEgAlCBTxhHbW3shwYyHkgslTNcBFKohavSfscPwENI6+qHX8dmfvpHG/M9huPamXI3JI6tYQ/m7Hj63jV92vOi9Sday9FLoplUyNYMk798Ug+e58ttnSb8bL6G//Z9engb9DuKHKMSr6dQy+cCnP/XXfvLbF+LfUggX5yot67WbtM+kU10M+06Nt7tp/mnvXdjxcMHmoxwG32CzPIPPTrmw9uFYhnmOq6oJBpnLhuAfDt1/0qdAnwJ/Wgq8HFh89WvNN+693xuQLyReB3Aelm+3Hbj54V195rBQhoby/EIa7cDcHdRq+DqxdK9dxrwOdkUNSu9KrV0DjKIMv+ozBVgotSjqUJmEYcrKHhFMlmRWvT6pAqXUQvsCBzHBRt5xX8tnWfRME0kF+SUPGCwlFgdILMrAC1NA3qcwtDHKZhRLWVNXgVKlQL1pR77QTWJWyrX0ckLkUXbftkyoiwy6wgq40KvNEDYXA6pMsJ/FJNKLaa7TSjVSF1W4XGWm3JwytwILN/7TEPcDmDM30HvE1Z17b8JY3oXBdDduGQ71vB/ABN9HqqAXJldSb6Iu9TaelDxNy83XtqGpBsJ6k9Jz0RqgYgvmdJLn7lz9KbwOffI6xuYwkK6QK0WQ0dxAKvAt0n2fjfoewuifENc9L+4CEP7dO9eRQLgD91hUdmSMNT6+T1n12LTKqQTkGAZOFapxQMIPAHg+DSD5NHtCzMAoeQgQBABKZbQbeQRT7N4Qum/dArBoOOwO4Z++xs7deMaagBFzZdg+IHO8TxyNsZVS+FuAtgchldQobdgi/1noJZOs6llAE+8mia9HqznqvAwwsB2cJGUq3d38PmX+ysZO1KJUuRrk2TxluQlj+BYSG9WS3O9jeULGHekH79yBWhUtdy3/AJWZR0iTtnH36yZ27tdxF0N0vW3Be4b5lWlVAqHx+D0kPO9C4w3SsG/MCCRIf5G+Yj7ubyDDeg9bGt0GfwtwaMcbRgoi8++Kvi6NXclX8iUzOQWdtKVwQzWlYyPmZ1+F+TafpzDibpqne133R9F+QRCims48INa4BeiiKgZtQ1PoAPLHvTFAi/JolK1bXZKmLLpRFnycQHckLvRPwYPSPBl2Ge3YbxAvy5GkrzpT8apEuqQ1QBjEnTG4dmCyPoIPvU4Zh5x4yAvSC1gB0Gj7447hZ9hMSKMcFog6J2/yz34T1Euj8ahVmRdjQtzG0jcGeDdMfyFWbF5O2vxU2VANaphymc8pCxMatus+9lCVLIDFKUbfGrELwLTHUH3M3bwHiXNI+Bh707eOASJKYHRNLZidQfQ9S3ilR6pJ2R+kufYqXtM/8mwo/UZphUb/tmf6EASKRKbUOH24vYVE1qQcl4FFl0Q1TH3mb0nnUcf28uvi3xe9S1QS8JqWuCrhmhTv2tZK2btxLqffC9fG9XvthbHAplXzaiuQ9OqzmmdbLt/Vw3h1Dknl2zjmWcPVtPvAolKtf+1ToE+BV5UCLwUWX/wqwOK9e726ZRBsB9X6sDdg8sD3OQlThn4fOEDXSaDGOh9QfdIdeB1sc3YGbN/XswcumPi9l3F03wjBRFWBOuQ+TD9MX/UIJfMvsFC6kVVNJmKBhauw2lRE7ckJ1d9O3Jxh5rnWwzIcA1ycyPcDKAAVrcTiAKYgNhxM9hqiKlGhcKXc1IjsclwiX+hT05eWlZ4GDxULQQtjQNk0yB6A0XSPgCGYwhFAxaAAAyPUCXbinp5GdxRw4QpyTc9rUeFCPEwdPa23qkHvw6y/D7MtsygTch0m9Y4rnryXyVqBwVGdRXUiV7NnVbeBUf4hXJkuwizJsGwRxpV3PT+tw/CtIxHYhOk9hnG/g2Tj38GI+C+wz4JGypZlJ+EwhEb68DXCvYfR90MYWfc40Ph0Ge9KApFPsRv4G9zbxqohraK25a7gGlU/xXjZ/SG0a1FtaJy9Bn6QMv3w67eyk7bMq2uyqiWZp3yQ6kDuJ/EeeeoRS0mGne025XoLNSDVjrRBKYxX6QsCITds8xQ0CKRcyT8gnmpCpmFXlUmzDwlAjmn/cR7egnm/MzORDfZ8bz0EeKoYvUuZv7yO5AW6as8wAXP5OrR/C3r94I3FeKlaYJXaTc00uo99Bqv/jzgFRe8D3g4BONpl3MF70x0kFXO0m9+E4EX1F71qqY6nNOhreOx6AGjYgmbuHTGP/csCBte6Atb7lNKVPZhUvXM9BgyofqYBt1KY6O7L3BJ2FF186VmZUsHTJPWeoL/ETSrfnN/Bc9J6TlnXAXyn1O+E9yfEHdbAm/wmAMh+Y6bF/wJ+Kbff7Sk0HSQNGXGlBUovpgG046z6H5LOc/rYJnU/ciWe+tvGNsIxz2Mc7rdN4/vppN/TyEopZZKtT6QZMNuq4qkuJaDwNGykme13K8AJY01dBpAk6CK3N1YR/ow6CC4gRsYsK6L0JHYQjAfa4ByZpnlAp0gauOrKtgILaTvBmMLKBF6+SJ+8Ilmhjx3znety9gQVq0G8SpXdtwG+5OceGUOE3aNMWTChDx4CPjDGievgKcaHWYEFdYwhNuXSa5eez6bIX4mGeQvstMlapq8t0Tbz9NkAC+lkrSi/h+10fvis8zth+O0j69sG7MZpkzHAhbRC7zZ8vdT49Xf3mjTPE+u+spD5nWK0b2yvq47uc9Osv0sKV8Wwap26XR3kwtNa/5q2L7t5dQPXMH1g0aVK/75PgT4FXkUKvBRY/BHA4usdYHG5gg6S3YG4+9s3vYGViBne8+diHNOsg6oTg0F6v33J4W/PHqho7wMsYBar1OII40wn2QosYv/AhF1UobSvgMGBWVFioRFlBRYy2ZFQyKgwacs01Su3hC1lCGPYSkYCLEw70hBABcyQKlASVOPsTJgVTVArptPMe126SLyQpFSzR6/2J5dSDpkVmQCBTmgKszAouGCFchSGYxhQMYLNxbj7WsywqRkG3Xq5CgPVNlDqQIqqv8g4yIy5QvwYJv8+jO4KDJwVddO012QyiK/KhFKIR9TtoWFgNl3hX4KZ+xSMqWpRggulAY8Jcx+gsA5TvUXYHRjnHRjUeRjjN5E8/PnXbsAAI01B7UQ3sBpCPwWwfBMG/x4uZwUWZ6wK2xazMPifAFj88I2lSC1gywJeVOMxH6UOT5GiPON6CP01yB2GKVPy8EltLHCHqwGqq66qJLnKK920c3gfYOHO3eYrCJLxm6euSgquwYi7p4cqPaokTVN29czhSQFZqJPRrqojKQHZBgi4k/ZjGHUlNqqbyWCrgjTH6c7jyxipK3Fx52/beR+AsE3/1JD5A8uBOtcGdZLhFNS8AbiRpu7irXvdSE3oT+66/QggsQpAcPftNRjKZ6TlrtVzgDz1482boqeOStVUu5mwzqx+ryPd+ABD9U3o6/ehFGKKuNMAhXmY1knS0CZBycaK9hOE0+vSMHHdv0S1m6jyEHZEOwIY2Kzwk/4U/UXVJr0VaUQuY39KHOPvIbHYJ28QRwyeT6DxAH12gJX4SD9sF9pbhldgIxDXbiBqR+TtrtXD0Fkp0gz0GSa+EqP1XdKXqSetERhhJY0CgAAb+ocbOmqToWSCpmNPCECdgJy8lKroZU1QoQvdCix8Zxt5Gsczv0lXKUc2vqOtMxiYBumpAhWi85teUsYT7oapwxBtPIAqlP0i9hrWlfyUjoD6AzZsX2mnOp4eynSd7AaAZ/QvxzvBxym0Vh1qAKbf3bRVSdOIO+0CcNGrl+MdKTQH0AODFTzHTUYiZd9XqjQCfat6p5IMQYUSC/NX0qTdhe5mlawptSjAgnqTTz0o5qWj86ATDoL2wp2P4/XZeXqdQOW2E6/3zpvLabe/a9p1junFIUE3YR4AAEAASURBVB1zS06E7bVhJ/0at9eW9V0nrxqmm77PujWoterlfemmxq1pXXrd+9l93wcWPbL0b/oU6FPgFaXAtwUWSiy6g6n1dCB00KwDZx1gvZZhvazMGNYwNX4Zt2sI355PAL17wzuAt3nkefv7MrCoHqFiyM1kXKQJboTXgguYThl/V2IPYQ4KsGDShlnJYE66RSpRpBS5b+uVuqRiRdoQyQgMQIALjEMxCi/A4ggmKHYVTvAwMzVty16PSq9cpRL5eGTi61Gohi5Xw8hoe8oYlDjEs4wwDHqKGmeFcgxbCz1EjeB+dgxgMR5gAZNBGHXkKVASlFFQ4qF+tcxDPD/BFOv9x70uBAgCh3iHYsVzAebcVf4VGHDBxSNWbdVzH6cIH4cpfRu7iddhhE1HUKGXKV2L7suwc7+zsRGj4mUY609j+6AbWDe9U/VHm4kVJBDvEe799a3sD3EqsKCk0zDjHwdU/FALLARVGhortVBiIbBYAVjswLypAscf9NIPmuswRm+SxydvXI8+usDCVfxpgJCAyvqpBqXE4j3AjzuTCxR8F51/GG4NWRcIv0Tdb7hKzm/VwjSiV61Fbz2WRUN1vSwJygRJGhy7s/nrgAMlOuqsa9htfMuvRMg9NZ5Cw1XiqNqlhyhtcGbJS3ezt4krEJEhlOlz5/DHgIl3nqw27+Npa537XZjJA/rvMe0zh+rWDO2gTnzUtMgjRr+UTXe16vkLeHfp/5swrqeuokMvWGLAlrYT5AUoHQXI6KJ1H256C5oKCtxrwj0iBonnxoXo4cC0YqCO21Q9IGlYLMOsIbAseSQC0HOQso1aPsp5zGq6G+kNyDgTxr54jLvW7D5NeifUWxsH93ZwFV71nlOBIuAJRAQ4ElTAHFPOaexH/P73AGLPAa2n4xg1y2zTn1wU8BNy0cCyH3GeUP4z6KDS1iEb552Qn2DIthZEnPIdHPId+C3le+JZpA+U5ZRwR5aVuHTtUjfanI+7fEvEUQphn/DI904BHOlU29JWYlAaSoMKLKgH3hdE9uRTvuVIC7ifsA8DjA9RtVLdMPvemAfl0G5C9SoNzJU0aTMzAp00JrfPK6mNATflPCIPvVyNs9gwyzdnv/BbNx/VoZRUFlfUgonSx3zn/SIA7Rr9dklgQlgoasUDlkIjfkqq86P8cDR3kaZ3dAIVutQ3hqeQ7dF9V8fC+u5DV8evmq73HDX+VXF9Vt8rhapHwvLbJ76v1bE/9A7fX47TvqzPz1OkWL2IH76pZavxaoj6+6r3fWBRqdS/9inQp8CrSoFvCyx0N1uG4otVdEDNwNgO+PltkHbU9V0dOOvwm1f86Q7MvVTbdOqgW5/7u3vKXFSAETUHJtkTGJciKYCxgunPHhKCC+4PUFXaY7LOxloyGkziMnMyQqYr0245lVKwpljuk3kppWH04OLmXDKxgpO4r2ViVxIi43ZMPupEa2BqmZSE1Bknrin92ebhJHZOF2lh/dracpN7iGkYTwFFBRc8yDOnxKQjQwaDNTk710wirRhmw7wRVKGGUYUaxTWtjEwMUklfRkFG2/QEFTIZqsw8Z+V7FSDwBEZUlSbB2i2Y8Vusjspcy0yr/74OA6ONg6o/+9wv8061nTc5KUwY7HeQAjwELKhuJvMzCPM8jpH7LIbhb+FJ6g2kHNdYTXXVXvsKd4R+DybxAV6n1liV1yuUrjDV/f8ELm11l3qL9LOaTfk1Ptddqntv6F1JkOdO5koQ3B/DHYyzJwR5qUKkCsgc9XA1Vg852VGbcG4wJwDaUoJAfCUOh5xKsjQUvkX+b86hkoQdiRIIPTXZFjGCp17xbkXbK3ERkFVpxRswZpE4cF2AsRNQ2FcPYLiV9qjGdM9N9yj3EWW2bwmGbqFmpMqYuvNZJSY/y/OM1f53Hz9p7j1dbZ4i0XEDxlNoT7IwmjDbqECNASxk1Hegjd6ilKy4mdsATP0wKlOF6Vc1j1V3ng3gZGCI7yVuXqHNBPHduG0QoOB+CIJC+7OuWQdIJ7tVw6SH2QUUnBDuBGcB7kA9xL3uUu2+fh+CDFWi3FfjkM3hDlGD2se1qwyqHdu+HfZbIAAzeyojDMPvBngap1OtMMtuvifjLKBbBDxNAXwmoecONiI70GSXNnRzuwHOIQED8eJ1DdqkTwBOjgUWlEmW9xhgcYqUpICYAjC0iRA8lEKpIkXhZPxh4jXE3qN8B4APqIB00D7Ad0hGSpdgtyOBqe6iaWRiWzlP3tK+epBSpcvnsfNQwtIelnfYRQHqr9emMVxWnwraAWGH9BWlaKp8nUknwgkoRjnz3ZoG7aMdl99ZNvQjPJAngEj7iklBLd+wIMoyj1B2nSCM820JdANOSNv3fm9KL5bo8wEWgDXDGa9ILCyt/Z+TKzX88GG9Oa58lzfnjP6HQiXdkkeCtn8+/OT8rd/NRz0uAItOpKTQppPxmAqmfjyr6ed5N84V9azlNGyNV6PU+Jef19/1veHrsz6wqNTrX/sU6FPgVaXAy4HFV/4k+1hk0uzWsB2EMxQzspYhuQToDrSZjWq8uCdsp5VLE4ODfI13ecpwwL189oAFE7oSAoFFVYfKpnhMvDIYgoB4h+J3ARZMyDDQkXDARGaVlboIKhzkBQSWgykiBZWFcGJyAo8NB2kKVgJYYF7UB3eCd8OrABXK45E06p+aNjN1VWXq1jW0Nbu2nkmAFKKKxeQfYGEaMAE5uDcH1Tl02TqGxGIKicU0oGIUdaih6dlmaGIatRF04dnbQj1wyW1sgYXMnwyKK5WwtkgkzrJ5mt54VPWx6gIBXXyqFmRcV31ZU0043ZW68i6zraHwTVY6R8nDTds0BNcWwzpNUc5F8sBJaiQcS3g3UtVCl6QyTtopKIHQXmINQOKGbW74ZlyNke+gPnULhnKZ9Mdh8GSA9NCltEAApNRDF7B6a9JL0hr5HtIXZIr0QKXtQSQzlEEJhKpN9kclMNswb3pXCmDi3t2sVbNyfwFY5uYt8v3BxZnmLVSrpmCcBWW2rHltk6/1F9hoDL5NOtJTT07XBTGculHVUN0+Zd/RNuVd7EjeBRx8oG1ECw5003qTsi4DttRzt1/bnwWpOgPYQkLxZHW12YA534XZ1lbB1X/7p3ZBY9BfvXuZZKBEPA2d2j1YLR8k/hDM6ggAK/uwkLaM/uke+yvg6WiSuFNIm6YBFWMwooOUWTerAppTyjzMx3Dcrv5HIsfzwyP7CM/pVzFSZmXcPug3osRuQoaWdN0BfB9p1e7zjWaHOlA0O3iAW+6hS9SCaCvthQQWQzDZfkPmpV2FbLg7RS+xUeKk4JKwm+yREpsC6CfTLUihOMTjSyWu5XYsEHx5Jk/SiX0H4QcAGKoQqoolsImNBOWSnhYx6k28EzztoVak8fQJ4VVBGiS+AD8fBOV3wUBAEeab/JVYCdSVaMWQ3G+JtjRhw0ZySDi/Q9NRpWmcvqUq1BDjlOpbSisEqX5rfnOnjhmU07BKbkYNC3BR3Uxg6nd0SptpP2T53cPCvTIEaUooCuOKxynCCCgmeSbw07hcegos7LtKyAQVShMXkIyoOmg5pYsHrZX0M1blSfljniVE+7dcOiHKLcVM/IQ2EofPXnRkLH7RS55XJvwlQXoZvCSbF0bvpl9oWIImLQreTdPq1DDdeFclXuK3tAjxSkrd9PrA4irK9Z/1KdCnwKtEgZcCiy8CLL7xrfcycfVG03ZiKIN/mXDKrMGLdrbIQMskeD7g8ipUYaq6NKOc/ywTvMH0OFKHXMMnjte8K7/DSDCJvwhYRGoRCQNMGiuCAQRO3jBA6mIXcMH0TTkziXK1/DWvcl+MwyuoMK5gRbe1ppe9KmQEZSDaikiepFluimRBBoFJ3OctxUpetZZUrNbTa2w8mPBV8ehJKySMZTQNT9+zmjmGeorqLDNzbMCF5GIINahBVpSHRifDQMk8yXxZHOs51qpCCSx86H4VSgKKvxoYF555Sl+lF1ZLBkRGxZVa7TKew9gIQlwF1Z2lK56qFK1FCoEKC+Wbh7G+CZPppm26GNUFZwzHYaYEWK7xqr7jLtcaRetGVMNijaQJgI0B+zDA0LnRnQbCMj/qgntoKyHAUAIhc6/0QzeySh5k8NyATy9HMlKwkDFK14WuhqsCNZlv87YullnPVLptVc1rGboqqXh7fprN2QBN0omyanStp6N1wgkqdBvrKr4M2hwM8AwrxDJvtHLaR1UmY8qorwAMvs7u5d9CnUlvVqcwv0pX3PhuCQNqmfyotFAvP8hT8jkGAByhHrOPOpJqRac4H9DWIcwq8V2NT98g/7g7ba9Rz+E9nRzpxCEL8DCZtJHlUnpwLOCgTWYBTeVkJ3faz3hxSQzQstGl9R75qm62SzqHAi/Ofep/yvc5CNM96sZvhJOxF4TMsFfDLEBpBiCwyR4hW0gYtil/PC+RZsoM7ck+h9+CGxLajylA+psSQOsogzyD2tUCkqsxwag0B6wcAegF1fZPv79DwGBW7m0P2lOpoQDFOmg/YV4y3pqzm4fqToIoAYaqhH6CFsdTAKAthhKOI4D5MZKLAYD7KH1RNaqMQNaDcKYrTctiwWBAhe5v7b8BdwSApGkjEycbcQjPrDPAnu+A3Pk2eGOb02ZKRv3G3JPCqwkICgXVcenLIxc/Mh5Bc1oqYUJHbpXS5Rsjfd0J5yAt29u+OcXpYkHZVdtvuniJ0u4pLpMBhm5EGVBC5DaFXM3DhZZ61HfWq9TO64fHd5+m8v6pkV4ULoFLMMe4ejgmXv5d3111Tfj2RSnfVaFe/Mz49biQb/vwwnue1TDd59173yfFy+ny+zynkngfWFTK9699CvQp8KpS4CMBCytXJlVuMt47zXjLv3b87w6QGWh90b5z9Axs6AYy0fYogzAh2oHXYN7mZ/nDfRmE884JmN/FmNpJ2ZVKmDIm59hXwKQWqQLMkNKKAIxW2sCKaICFDACMQClZmRxkSAJYSMv8Tl0d5JlgwjhKO5zUXTFULcr8fF/F7aEJf8I8yGRCA1dBZdpkAnv0SP2oD3HrBGQ1vTcNwwoqAixkEvjte19KWxkTjV81Zp3AI9QUe1nMAComkFyoBgXaAFTIDMFWAywOiayrU9MQDMioaLxdbAdkvmQ2NPIsZZbJ15BXhl9GXUZEQ251sWXOTG9PtS/StLzqx3sn8+0pUzoDs6KNgaBCmmaTOWjuiq7vXaF3tdQWcKU3NhQw1L4XCAl4ZJxk1edIRyPhSC0oa2mbskqs7YK2E4IE90KIShDMsUy9cW5qQA2o0CWsq75jnNJX9al16iigeB9mf5vf1v9jMMWvsXp7nXh6I5I2aEwhJcErFfYGK0gBVAez3QVn7l8hqFBq4w7mqnkpjfHQ3mCfOA9RZ3qA8fQqqkmWV9uBMThEvU+NwzS7Kn3Es0Hj0V91OToAGB46U2WJfsizMxjqE8p6JhPKKYNd+4vSC9Vl6ilTKc017M3KNO8FOhoG8yISoVl2DBcAKLWQQbZf637XOthvJ5A6bKOmtsW5iQqX0hztlA5RdfObsf+NwnTL+AoYdBV8DenC8vXlZmFpoVl5utKs44Vqm/pnRV1a0jcEEnT8eHByZV/GfAQQ6qHTg2weR7+bQCVn+fq1Zgr1Ob/tTUAZGSFogMlGiqXamoPXEeXaJe99QI+gJOqIpCttRgAP2m9YRr175dvmuTuRW36vdBmrUw7e2R9PKNMJG066P8ww7pytJwWNFNQ8/ZDsQ0kDycYgp31WiZk7avtR5D3pC16qtNEwvAgYM5AG5icAW/uJ+2OkX0N7pRsal/ud+426M7bfre3sgoKSCtMyOfGDfd3vDNIERPgsNljUO2MFcf12BBca9Jum37SSiVm+06IuOBGpm9+IfYYkcpoJwcs9z8o8YM7lqHeGMZR1uHzkVeehIa4KV4Mkb8oqnT0MmznF+/KgXLmvafu+ppnwhuNIEjVQeXQhvfZRLjV+fVbzrL+7NathU9ZS+aTbe94+M259VgrDg8677jPD/fRPfqZm912/+l3VOtZrN9NK03w3fFP/f+xj0S1P/75PgT4FXg0KvBxYfPmrzdeRWLzseNGAVEb8MqKXycBJoTvCt5OGiXubGYAbrt7KrHsmRp6dh3cAdrCrwKJIH2D8GSitkOAidhCACL039YBFJBaoQwEK6uZ1Gh6SHPmQJsyHaVXmJHmYpquJpCu46Klb8NwVfcuYklHQCrTCVMgIcGbVlEk8g3StflufqHCYeY5yBTYUZlDmJ8wFEyzpJJh5wPTILKk+MgpTqqGmwGI6wAJVKNSgBgEWgzJGrLy6mq36j5u6HcGNqE7hqmYx6iwMpyua3X0vBBZuLOdmcDL6FL+oU5CvwEK2WcBhddy3QINQmZV4maHcpp9VT8Pw3rILUtzUTUNnVTHc90H1JHiYMHdpS/KShqavx6ZnxFFdaYb09C6lt6WAIuhR87McllEDcncKf0Y8N2TT+88iedxFBWtJyYcgx4oQV0nDFn1D9SfBhcDENlMqchtGexGgoLRDuwFVrtw7YhVA4a7humHdJK6yk1kYYl29zrCibVsZ9pnvOY9dWVeVCBWmdVbvN9Whp9+p8jMOYBiTSab/uFOyNJVxN44qMXTOgI8RQMU45RZYHJPnEcCCThcgrdRMBtDd1t25WslBmGj7CtV0NX9bGwvqHFUa+oqesiahhTr441x17Wsc+7wODp6htnQA/eyJuis2vsBim/oc0R/8Bvxu7MvWQ9exYXah/xlte5P9Sm6w0/qNm9cDLJ6trwdYCNDDnJCPdLZDmI51SZ+B3n7Tlnkf4CIYn4GuN9hBnZTxMMbGfkh8lLwogZmwvpSRRCkTkiuA4T6gokoi863yXNoIWgQWe5TP9P3mVBka4Z20zzeY715IQdGgXaQFAMUB7JT8llQ3kybSXLVHgiRNpTa6fB7CzsnvTDWsI9uPdvUQRA2pekXf8EMIiCFcUcNC6kKijjlD0g+blqhvES9G25TvjLguLuRbNS3j0p5RlzMyNJN+vXHH7k0+pZsDTNo+EYCSdMrigDYWSkAEEe4n4rkAQHZzSMGG40Hom1pQdK/5Y+GkhGf9m9uX/rkQtRvyRS/I//xVyas7z9hXytPzYnWT9X3i98Lxq03Qd/Xopmm9zt8Y/DxODd+NezFsG6KTRk27F+fKCDXlcjXsnwVg8XNf+fXmtx9++WLl+7/6FPg+p8DS6HTz4zf/fPNjC5/4nq/p4iI2u8wR3+nhHOxCxJSbxnYOFhfPzv7wj78SYFEHyfo+Y6SDKKfv6vv623D1mfdlTC0DdYlbnhI9R66mZ0iuYdaZOL1mwG+vJbSPzoGFTFEkClwrwNC+QsPq7IQNs7YPQxebCxgDJ369x3hWcOLVNLPy2qZTGSFBRhiq+hxGoEg2ilTDKkTf2uq1tBBQ1DNG1hacd6W6hW4yNMkzD/nTxpd5kFlzMzBXVLPa2TJjpqH0wwaXMarAYhK7Cg24J6Y7wEI9eD3wEMfN6FyRFyiQYlY/BQNVAiAjMQIjVYAQmhmUzdOVUQFJBXiGg63Mqq6MvTv2atysrngYfRjkGI1ytVoy5j6XPmswfrpYvQ+D7TFHHeZhbNxJ2HTtwkoylGLIFuhxyV22n8Bwj8Ll3IChvAGDp/vM2E6Q97BLsxwUNVIUjcHdG0MQY7n0zX8LJnoG5tI4xzC42SyP92uUw70hVF9x9+hpzlnymCcPw9o2SinccfwhDPYjmNtNmO9tjIIPYbTdLVl7irt6ZopK0HBUpLSn2CJtVZk0oD7T2w+ARBBq200gCRkHVIwikTjDsYBqQNJqF9rssrrvHhAe2rBEfYyynwBYYucAc6wqDIVLX59CsqLUYQ6bEHe1jjSNdrbvHlC3TQzFPdS5n8WW4vrSfLMIwz6NTYfSAyV9MqnG26GMa6gu7QJeVN+yP27hDneHuhwCNqJySB0ipaJ9BwUWAKrYRxDWfSduIrG4CbC4CSB4trZG/thZUKfkQR1k/ksfI0L6fwGtw9aRdjgiHyUPk7i0XVhabK4BVDawS1lHWvFsdS27TytJESC5oSWppDxKLFxM8JuyLtbdvAIsKKfAQumLEhfHmGIM7feDRI84ghxtpQSdfmMBFqbOt1jAuSCgSEgoaPINoKLtBBaqHsbWRPow7pRN+gpQGRSgkD+Nz3fAaR480+A+9+QnsBA8nghGqVNc8pLuKYsDQ9RVAOi3KmhwIcRNIQNCKLtALWMEaYsoBJIZh3guaLRcUZt0zKC82q1M5sShAWPIgqACoObGeEoklcJVYGFZpFdvLKfPlMOnH/1IOr24/uJoL7nvvfN5eXExTkL1yuG36eHfbjJ56HPfcyaNPGxDcalxfdyrlz9qmkQtRSBwG628Pq+zj0sJSrxOsN5z0+7m1XthYt0I/u4cfxZUof7r3/3Z5pfe+Z1Orfu3fQp8/1Pg7sRi81+++Vebv3H7R77nK3vr1q3Mn99pQV8KLP7gSwKLd69M24FVBjsrkFeEuDhoE8DB1ME2Yflb/mfwdUyXYXEgLhKE9p5AqjxUxr+OyIYL40/+ARbuXwFDYWWiqwyToErFLpN8dt+GgYshNxNyXXUMo2R4JmrzTh5O8DAHpinTVc+oO/mb96WMlMt760KdUtf2KvMoIzPERC8T1c5Sba2Jx11RgeKa+soPFIbA8FFdgCEQPAzCsPeYhqRAcoRxxVWmaQzmYxymdgJgoZvZYewrhseRWIywiqoaFGFdf9Y1qp6IBBgjFCAG3DBNMuayZ2Hs+W1bak8hgy+D4brlnquwMl+U9RBeSVUngcYkZZMJn5fJIx29M6luEkmFtKC8pqH0wlVgd+92/4h7rC6rMiJAWISh1POUIEDvOLpd1ce+zNEmzJ5h3bzPPO/A+LymcXDKVlZUbTte5b1SB8HLDqo88zCfC5RtlhXuUZk5/h3Srtu8V9rwHOZ5hbQFTgI4ffdry2He1l2mW3uLpzDbDwj3BOZ6g3tdmZ5yHQAsjMMIztC8CzDpkwILmDalPAIId55WKoE1elSbCMaRlofWtCH9EaQAQ6sEw9X6ptmGid/C49EewEWplDr4SpEEBao/qYanapO0sZ94Xr+21CzjAWsWz07PMMx+xu7am6RhP3V1fg+G2xXxMfKYx5bjtds3muvLiwEjMqcHpOf3YR/Vte0G+45sILXQrsLvaZMy1femF1sG2s7+bn2HoNuY7kltE+q9xCrHMoBgiXLtUJ4D9nFwJV4nB3u04yYATUlI2eUeWwbS8tuyf8QdtPSj/a7fvNnMU85p7EDuv3e/WUVaoSH7JJ6LtEeZBJyFsSdexgHqKBEtl4BCr1FbtJvfTxhvwvnM/PL9QNN4WRIcEYcPkm+eRqBPCsRNLmOA/Z6z2G4wNpBHNrkjSr5ZVa1QgxqGDsZ1rDqhXU9l/Nuxwi/BcplvkVwovaBPGp7T56pC2cb2h2yACWATrJwiMRmG8Y+dDO2uutQpYQ4Ba0q2bNuEF7xA1wGuggvpKSiKxCLPuKe9RvguprDJEljOIm1SXVFVvgAL6qCtlN9vxiGSMBnP3mFH5fB7cuyqR8Y/n7fP6u/6Ph28/WGs+t60axxf+7z722eXj8thUgryrWkmvuWwXS8dl9OucRKsU5/zml1M4MLzTviaU9K/lG/Ns5tXfWbqNW4tbx9YXKR5/1efAt8vFOgDC8Y7Br+zP/jSl5s/+ea3rmxXB0fPDJh1MG0HW5/VATPh+OETn3cH55qGk3cYhHYyLgyj6cN0MHk6oRuWYLnmnrBRXeJaVJW0e1BVCUaYyVcvUEoscrISKKCoUotIKzKRAxaSLmmZXq8cLaiwPMmnAyqkhvX0tN7WtZ4yDzJ+7epiMews5Kv1LvVoJ+Y8lFEscbR7UMWkrDQWRh/OgQQKNWWSZKhGYWbGYD4mUIMaRQ1qBEAxwHk6gi74KCuow6iKwFCokHHA6rgr7+4Y7a7Q7lOhJxqZ/qI2AZPI72GkG5GUkL4qQW64pl63m7optYgqlfRpq63Ew/0hDGvbHgBAdKuaFqa4MmxKAVS5kNnRW5SbyemiVQNXJRVuTLeMlMC8lHpEikKe0lPpiu5tDS/zdQ2gcMPV6jA+qpVg+M27Ldp5l5Xu7N8Aw6atwxIrsNVlplITd7beJNxzwqvmpR78LuWVnkorVJWSqaoqXDKgO/SdVdx/PkPqoI2Bm73pUQkOtRnBANpN0EahrR6sJtDBlwF0BXqSMmoAOzeBihN5eMqsuSJ+aH8EmKjas8/1hDpYN1fr92GEfScDL0Po7tTuMzFNGwv8lKZpTxCjX95DLoDFctyxDsPgPmFFf+XZOqpLSEoIbz/M90baukpVXWp5cT4AQ+9Ne4ClfdpD+kwADjwEJY9X8EIFEy/QUH3Ib1PJ2S4r/gKRMOe0clShYPQnBLa0oXubLC4sNPM4EZgBxOztbGUFfhh1LtPZkdmnbNPkpRqWYEfAoYcsT9/bt0Yxmr5151YziUqa3+m9b91r1tfWQ78ZnimxyE7biG7sd9Yzv4nrQoCupVXdUj2KrzfjhQkf0Q7pu9Db70spgG1n2/gtC4xjBA/tIq2jv0SSoTSDcjg2GM5v1X7jNxM7Buqh1CbjCPkXD3FF5SpE7Y4TMPs0SmkXy859fhPGsS7Ahf47RF8aZNHgjO9cd7qCIG0isiBBv9OVrlIs1TIdG1QdE9gMUK9IQSjfGd9zNtGkfOUdxuqk60KEGyMqZVNK4bcisF6kXabIR5BofyFFUj4/zu8pK48dx+qReoTW9YnVamN0wtUYV707j3lV2uWt8Wq+9dqN5/2Lnl8I15YtJbyqnN0yc1/Lm/J33l1I84of3bLUNCjg+Rxo2lKZ/0mbsvz0X/vJK1L67jwSPNdy1Ws3J59Zhzo3/9uysfj8468272w86mbVv+9T4PueAlPwZZ+cvd18bPL693xdJ/X4yDzynR4vlVj8/r/+4+ar7/ybkvYVA2oGzjowE8rfDtblUYbtTJYm4LM6eDmI+tYkIzFgUq+rl2WlrzL6LWPv4MZZAEg7CQcIlHAOkIkPo+KqnwBCo+0qsdhrgUV8v/M8dhIwC8YrA6fpl4k9QML8WEVMWWAoAnBaxiITgOUnTFYhrViYOFdHneTLtTLtLTHKxEEc8zMNLoVWxInhtCvUMDuqhchYlrRp2BCzMDPqW6vOMIZO9zgrjxNsVKZx6dn4VHM0PNHsDI6yFwE++2FOzN+V/nhQgqlyBfqUPSVGeRZvNJaCMP6zAw0PFf/+Skw0rNauQeNnPUbp8ckz06DVpfyCJ5kyJRw+P2bF1zaQjh4yQhqGRrWJONoybLcgRYmFth5TMKyqQ0XNI7EKbdLWlHMHoCKgGYdx1vi6ZydB2D3a+REr4EoensO4CuJkjnSbOUe+HofUW0CxxntPXcXKfJ4BCMzD8ltX3cOWfQqgBuVSrUZAuh01JlaHd9lfgM3LjgEaqja5AdpQjKth2kljnHxlzkdxUzqPatICqknzGEcPwfxKHQGL0o5tGOs13M56usmb7VOlYcV+B4aUZ7a/am6TrCzPz80FnElzmVYBwvgooJM6LMy7y/okbYvLXFb1VzGWVsIwCqMveJhw5Zt4ZZUdz0AtQ28/ksl3k0caP3lohyHYuHf/YfMUQ/N9JRT2efsG9Nle3wQsqNYEGIJGkZoJLKivqjUTAMXlhcVmjvIKePcFFqzeC4RUQ1KlynMRY+wZ6iWzHC9TfI8CjC1Uwah4M00a164vRbLzHNuU+x88QJqzHXA1A031FiUb5rerTG0UqZSG3vb3bAZI/ZW4bAvSoIuLD37TOahL/WZjn0Kd7QOQHHBDv+C9EgvbQHUqpUQOkLErIQFBhRIuJUpEiWQgEkboaXk8za93GIiQpufXXlQm7WP8SgIJcB6c9FVdUgqiQbhSCEGdtM4gz7cl2HLjP10S5z5JkDZxbSvBjkBDVassMEgv68jvBtWqEUDFJOOG4GKRdlikjyix0OXxnBINxhclmi4MkGIZqCy9Ze6V1MflW/VRb1xPPUug+izETSD+UFaLm3edsCVG+Vto1X1ynr7xku+L4hqt8+4iddvyW5H26JXR39147X23LDXvGvejXmsavbxq2r082/J4oWx/FoDFR6VdP1yfAn0KvJoUeCmw+L0/+lLz5W+8k5rJXGfwbUdrB8oyRpeBUbYzq3pcyxjJJOBk2v6+TB7D1Eki+tVMmJE8OEH3QAMTdRj8lgGD0YlUoQUBZSUFhsDnxoM5OPcGVaQVWWUBWERK4YQMQ1pVOnrAgsG+AJrzybL+DqOcOvPHazsxpP5M5jIZARZMxpFQyBy0tOlNJm1dKw1MxncyC6oduOIqg6S7yCGBBb8LcaES6avuoPpI1J9gXsdgDMb0VsM5ODbV7A+PN1s4Vl3Fz+UmzaQhc1kBLUyV3neGYUTd/G2Sq/dDgAzrYvm1xZDp0M2mm4LBXmUfi1kYEvM/pMAHpKtqjm1uu2mvIGNsGyslcMV/nGsAWltRpSLmodcnjbCts0ytaWhcDZSBVtA+YbDnoA8oVdEVrHYOqmoITpSKCFTsM0kLhngdRv199oQQWOj6dR5G+nVcnd5BP38aptXQ+wCLNZhMVZp0ERsPV5Tb3acPkDzQaWJ8PiXzZZ3oQ0oFNMKVBjLLo6cYEyOh2IbJ3cFdq/soyNgNtHHV95+DuXY/iHEY3MLQw/yzGlyMZGF8aVNtJzSGlvl/vLrOhoBsHoekxef2X0Gl/UDaKEWQQTS9xQV2UofJlJbSfZn9PaZh6M9gyAUgRM1q/yOAxXPAj/r8czDvs4CbKeIrMVL9SlBSvZv5nWzD5MvQy4wqaZiEwRR4fwNnDY+erKYd8iXz3nz2Ye5VxfK7SDtiuDzGTuFusOfK/xhlu4kXJ1WhZpFY7Gw+BzggNYDe67TTFnVXHeomoEGbENV7wrzTdqa5rdE2fXBmaSlg5Ii4GysrUc9SsiIjPgV40z5CIh3Qrn5nUzDF1lWgoDqVqlBKLLYAQdqKCCrsC9I10gboKmCyz/nPNDL+EDbqQ7zX6NpxQ3usY9rIfhcDeNrUdnJ8cgdw+5MvpYfjT8aHjAmMe+ZBuDwkmHnZVnFVy1PT9Kn1iupSysj3QP6CCU/D5rfAgudGstwDpg3dbFOBTKlLSTRSFL5DPcK5X4eAwl3Gs5+NYIUFiVH66hTfybL2OXw37i2j1GKZ/qIEQ7XESDSTJAVsD+tZym3Jrz4qI13f1vD1d/dqWNOsx+W4Pu++74Yz/17MNo3ET5v4ppTQn/W4nFb9ndAETNBOBOlaj15e9UHnWtO5qvy9YG1ahq2p1vLW3777s2C83aNJ/6ZPgT4Fvi8p8FJg8YU//KPmS1//RhgaB8E6ENZBtjegQhrfFSY87IgzAk9rSONKvzp4l3tHWOM4UStlUCXClXUZq6wACiq4j91DO1E7WWeC9spZpQlOsDKGMmqCi3iDkjngXgYhK9UwjMVAu+R5IX478FtQi5r6ttc6SdUZITWjflE3gfEqaif1GjamVL9X/1LfUKOlyyDcnqBCRltPLDIsggvVNBIZhr4yFrrNzMZXMH9jTP4jU2yCh+rTIPYUR7iV3Tobbtbgix7BFLtZnAbKcDaoMxTpxAQZT8IoI9fILr+Dx7othaGVyR4gT6QVg0hBjmHsDimPtB/lHexrJvZjwniOwqhIC+0IZPCBdLxnMz3UV2YAPJMwLj7XpeoBYaSlbmazczn348SfctUcpikrojyzDZRgCCjc02JLiRPPlDrcgZG8yVmNqQ1XNqfDuxTAQg9MWzCSur/Vq407Zt/gmv0kqLOr6xpTG3abvqCO+gD9gaXtqJPYJ3ym1MW+mjbn2QiW00pRdAkLW0ZCqNWhHrSBcfMm1+rG1boswdQqoZhixVdD26xKUx77WYzCZdxQ4fFbcc+FdWwPHgACVpBauD+ENLIP6KlJ+wP7ukyxKv8a6CsBUKIlA8zL5jb7OixgezBA+q7KCxB2UBlbixoUe15IOxj7eYy0vU6ZLnXRpsKPXYmGnp7cOFLGSZU5pSIa9So1+dZ776NW9Sz2J36P0tA20lWtjKztL9Adhc6TlGMSYCEgGoSpv7681FxDPWsZd7PbbpCHpCe2G9BMqYV7YFzj3TSAxJV/mW+vk/TpfbZ4d5VdT1Pu3bELMNnEq5TxI22jHEpr/Bb8Pneptypi03j9mhPU0RZ+X6pCbSoZAswEVBDG7yzSBsopjfeghbT3NE4S5BLX0IR3kUS1NcckvwVBxThtYR72FI3Zd+lTjkMeNo3tGy9Q+X74NognPesYaSa2aR1XwB/tYeT8L2mkrIIbxgEXKWg77a0EGv728G/GxdikADBoH8dRR14lMgEmfquCCtoXS57Qlg6FJT92G4CLcWhm/1DVbpq2nOMUWLifxQKSC8FFdoJv8zuvhyV4+WEd61FKXH+9+GqcmkeNX38bK88kdH70/kj48oy/NV7vweWbTrnyirg1tiX+UPzL4T+UXnnQKUIenNf+cgR+916ezzM1lPXtA4tKjf61T4E+BV5VCrwUWPzzf/kHzb/66tcy0TroOlFn8O0MuFltY7DMe0fN9p0rcTkYuX3kMJow7X2eBCy0jCqMjiuMRe+ae0CAK6sVWAgcygRqOUyrBRa5hynxPZN5dx8LGYPiWlaPMTCIlr+tQ1b5UifTa+vVacVSH0vp0VaGi5NIJjxuIplgIhdY9PTR2lkm8KK9NwXXqYxXTif/c7sKGe2oQXFVnUqayiSoG61xr6BCtZZxDDpHYQoGARUngIrj4clmF9WnZ2z68Bgm6amr9zBEGrBrQKxUwA3qZshrduCkmT7FDevRHl5r9PQDuICJHRpgpdwVcdMEpByRZzzk0B6DtgF1PkWiccYK6BCMiXQ5wiDXtrAu6vfPUqZZyjaJXrhMvjYNnsekH1DB1T4wTR3mYGSmYGJlln0mSNFrl/s/bMB4l03uTvHohLE2zLpgwZ2DlZC418UaDJ1AYZP6amAtCLUvTFPueZihWaUMkE71H+mwDTO7Q3htbqSJG83puYeXOe1jAtIYFAvuYOTGkTCMosIjOFDaowqZ9gUaNz/H05JMrbYUCzBmd66xAg9wULVHz0w6DXAl3hX1MGysDC9h2+B+FdZ3G4nHA/a1eIpNxC5pygRqT6GkQ4Bjn9U7k66BbfsZbGjs75bT9nrtzs3mGukJGp9ie/CMMu1St23sI0x7B1Aioz6P9OYaq/9KU+xb8SREAXQru0q8sKHUVdW6Gco4ylWvTI8ePW6eAwRUCxK0xEsU5Yw+P/WDtAE8E7TNDNKTKdJXjeyYvqdq1jJG13qH2lxHwoPqmOpW2jtoPG3eGg4rrfB7kWGfpX7XoOHRWVG7O6BvbWDnIajYVvWMPkb1840GJBHHb1fDdPuf0psZ1N+m6FtK9HYBW+uU/+nac95rEwFjThzBm3Yd7mOiROM5dFIdzf5cpIas8hNWiWjsPugjSlDNK/YupG1c6VDcWTOmUC4PipH0YxtFPr6Xlg6u1tP30s1voYw79M8WJJhGGRNKQgIXy5N4XunTgq9hAIIgw8RMzzGrqHS2izD0VeUjATf0qWw6SJwTFgQO+cZPdD2tFIPvdUBwgY3FJBsQCgztL9NVaiE45xRcRFJIZoHdZnrVUSrwoTfS9QUxrgzrQ+lQD+Onoj7wvh6pfP3RCcOjxKmvrrp20/H95fwux3lR+PZ5qWNpD0veKeWF+wvJ9gIx77ShajbWvw8sLlCr/6NPgT4FXkEKvBRY/M7v/Yvmj778lYj8XRFz5bKs8svYl2HxfDDvjZisAnNf3zt8eu+j3JcJwGcyp54WwjOemwAXkTbAoMlkyUyZp4auEfvLZZgOz5Iuv+ONxTBMrjWtXGUMiG/YFMAGsiDt0YvfptWdCWuZk1cbxXmoMAEVIHCFQcizMBAEsK6mn0mrBR/tT8PJMMRrFFc3+4q0AkbAVV+Npy1DVr2NL4MAQ6Pu/iQGwuOoPqkCdTI23ewAKrYGRpt1qqb9wAqM0h6M2Air65MDGuwqsSAJkpnhBhjSTAEs8H0K08OGYq7IQi+NtsfwPjPOxnpnqkMpnWBVX9eXusDcY1VUYKGaimoWMkfSVqDk6q+7Ik9g4zEJuFDXXZULpRUaSKfduI90g7xcFZ+mHjNs6qd6j8ySDG6MqclLMJKdi8ljBtro2Sn2D4ST6XPTvOeU243csmJM2trURDUJBnYcDkhvVdo21L0f9mGO1eW3zNJdsKCNhcBB9TilWXuADyUOqtQIBqxbvIvxXrAbRhGGUremggptDWapww2Y6Ndxr2parqo/Z6W82POgYgQw0BPTMqpM11APGofxte0PMLx9iorPGnYMphfVFehY+par4eyFgXenY9DlCPRSYhFbBNrYdrmDZ6elxTk8CZ0WYAETbTtq+K1XqY21jQAU3cveoWwLqEWNw8gLFmU8dSv75OlavmmZ0BForC2HAOcEYLdHHQTjghpVmJ6jArUDA65OPx9SGGJtKGYALkvX9DA1Db0Hm11UnewLc0hKbgksVB2jPxb1RqRY1EuAIZ38pgQV0/SZZcDP3bt32PF6IEBmbZWN9ZS+AJg0cvdb8rsQXM/TPgI4Aat19nuK8TzPBG6G2STPNei3Ah389mXgs0M9YSYEcIQT0CkVek6ZBRLSQTU/D9tbNSfBZqShfM9KDFS1UurhP8tvuTKUlD8BAzaiz/eho+p0jjvx7mbj8kabD8cxo9gXPBxn/OjLuJLPP/e+z5lxwbGC8YFnjjc5iJd/Lqhgu+R36W/VMaWXp2px2Umc7/dE2yvOM77v01ZyMZK9b9j5PeCCfgB9VIu6CVC7RZ+4BtidoG9Huki2pe4l+95fymHZPbwWuviXI3Vrn7f35cX531L/8rumQ0KFLqnjeVjvklObX9Kv97yraXmtjy1zSmNaed7SL4mV+xrPR92jE7L7OPeJQ3qhynlmvTJcjlDz6NWRAOVZS6tSsz6wuEy4/u8+BfoUeOUoIL/1wn0sPvfPP9/8y3/9pUyGriyrixypgcy8A3WZRqInnD91JG4H8SoVqNeMw74jvkDFyVsmQUDgpKt6RiQOMHtxPwkzosTCvE0joIYJ1CP5O0m3wCJp8i7MbMopIIE5sIzEtWj1TAKm0UuHUClc+yZRCgPgkwtDP4k4OciwOIl7zWSRCb9MiE7xmXDIMWHb8DIGARVM+DIqgonYTsA4+1tGIhIg0+Iebi/AYkTGH0Z2bHKWlcbZZpeNVtYBFSsnA9kFegt7AXX/R/Z3Ah5mhmBoYCNJLVKaUUDGhLs4cw6yIizTLIOnwbUM+wgrmRqEm98JZXVn8ewuzlXVklNWPDUmHYPplp5huqGx5ZUhHYZREZyMCj5gGK0z/1nZhgmkjZQsuGI+wqqpkhdByCT5yYgaUOmUYFJm0TaznSZRC8u+FuThbwgSZlu7Ceu7DWN4CJOurcQRbk21m9CWQSZaMOqGcoIjOk900m0rV64j5aBewyIu2ty+dwCwyKotDJXSIxnK9Hf6k16a7JueAgGlEUqqri0uNDcADEotBBsbqDhtYONg+W3jMehlGEHFAuBglJViGiPAYpNwqlQpYRBIWSYlKtLNb8y6KR1SD/46dgsaWitFMP9FQIWelU6oX1zDIgWgQPHwtGu4Z8+T1wKM/2uv3UaaohE5e5nAcGuQrhvWdZj2InGCJrTRKCpGQ9Bb5KFqGCxpmOg10oqEhvIIyPwSVH0b164BAOFO2wsAJ/vwHuBDVTkBmmBGw2t3B1cS4herZygB0zOkMX7bMvLLy8vse3GzuQ2wsHzuffHk0SNUptxgkD4IXZQC2I8m6Huqiyl9kWG2PfzKAhzoI2H8+Z42AEPJh/0vHGOMr71L+h39zf5h33e/DHcF3wMEBFz4zdEfjmDSNeQ2j0hFvbfe/DOvKVbzdQEs4+63Yf8QRNhHHUJ0MBHbDvK2PVVvdCxwXFKaZft6+C6R2vsCJMiFNMq7sghhGH/XscN65n1eEJlMi7S2pJt0CF/GEvKFNsd815E6AjBOkQzlN201iLRodAZX1dhsjVMvJT4CNe0t7tBv7iKNWoJ2eljzc7GOHiX/cp/f57e5s11CDOvI4d86vtZrXnT+mGbe1Tjt7xeF70TtlSv5tnmVZGy187y7cep9jVN/9642Jketc33eC9++L4Euh6qhX3x9Ub36EosX06z/pk+BPgVeDQq8FFj8P7/9283vffGLYUJk+gQCToyRXmRgbQdfLqodlF9eBQH8ZXL1GlBAeH4yyhfm34k2wMI0YcaczKOCAIMZbywwf2EueB91KOPnJJFkVH6bPhk4e7TvzafkyZ0ZZnaI2hK/nGrqNJC3xEuxuPaONi3TNHrnzbmEgkRUXXECz0TrROg/Aseos80rAIQwbnynhELGoJzoMMOcyGCppuE5oJWtMyIMTKQWPJPBHJJpR1IxPDXXNBPzzSb7VKycDjYPULdQd/0InfRBXKHOHO02M2hUCywEE9I6DCGqPPrKF2yoQoMH0F5ZqRD5FnUoKpPn8dsP8yVj5+qtEMVVXzdkc9XW9neHZts18a2fIAjgINAQMKk2MoH+u152ZJzVz7eOuspVAjPOKSNjfQMG6Q/hbLnal5SGyOtZ0qgBpcyE5Ulcl7I6v7UNsw1DfwiDbt3c52B8YjS2DG4o594AGlDL1KsGMwqo0Y2sK9amLantU67mj0FnGUalbf6rfVY9+QosNCK2yjLON7AnmIfxN/xTbBKewkirhjMKuNFo2r0lrsM4L8F4T7HCb1UEa/sYJe8jMVLiImBw5XwTZl8mV2bW3nmAIfM0zN4iedy+exvVpWfNCnm4Gj9BHe13+8Qt7mABMjLLpk06W6hqTUBX1a9ev3srHphcOV93ozkAxQYSCIFLVuTJK8bBlFlgESacuBruDvENPcMTlFIQ1cD8lj0EFtJJl7ICi2WMsZUCHALOtEGI5IDybGHoTsOiyjWV8UO1KlW3Hj1eiVqU7XH39btspnerWQI8PX74qFlBRWz16dMw0X4jAhZVtcqGdthfpP8oYRuNBMTBK33QFvObp48LLKyjxuI2smpMgh33v7B81QGB9LIfbfL9qH5pOjL1Qgj7rN7ZpNuxwJoxyPZTamDba0TuvRIP2/EIwJCxKWUoTgCMEG9v0DV9jHFTUOr46e90PhPlcAyxvpFI8DsjlO/8Nghr+CwAUH9pYrjeuEOw1D1lJCZxIkXluS2WhQLiaWsRd7p8y/SYgAs+0GYIcDHsaZ2gqxIhVRVv0W5vICESXLjD/Di0TJFI0+MqcEGt/KCgYTlqGOOFPj627peOGq5HF977LHGuCH8pen4mj/ZF6EV8j97zTjq9tNvwvTK1cXq/2/jnwT5c9vrOtrp8pD48rPWr7+vz+rt77QOLLjX6930K9CnwKlLgpcDiV3/rt5rf/f3fD2NQ1A9cnVPKILhwkHUaLoN32Gqe+S+TIWEKg29Yme0CMpyso96k+N60OF091c+8K4muZsaDU8uMuhpnnBymb77lfyapTGP8LgxhLxg3PMxBGRn0y8kknhKXt4YoJW5DJn2TJY75+L7NLz+obNJxkiXNMsGX+6TFH8MnDhEMIwMQ1QSZJM8wB0VaIZNTTxly9amJFKlF0oOJg4NhJ17ABaBiYHK+OQVYrA2ONU8AYk9gak5YyR7BSHYcacXkyX4zAbCYGsSrElFlw2W2pCfcURinshkejFPLnMDNqmsS4KBajuSxHWTmPV2RtQ2klKokGgKXMLZbaWvBRakjuvPek4bqE6Mw+dn5l3oJOnShqj2HIEv1DfMRVIzAfMmsmocdUlWcAkiLhMwyeMjUSidBjVKC7OwMsDiBodcuwtVkGdEwzsSRUXSPhejWQ0NVfrRHcOWfRur1PVfoA+wos+BWgOUh4JCpLn0S+vF+Gt30N9+8G9sA01/FVkJgsYkbWtV7tG1QijEDuFiYZ1duAIJlVuXKvSt2AIGR0PBbI2MZ29gyCHYEFuRxeshmc67ms1Gcezo8fPiweYDtwwbAQqZRRnePzeuyUk5ZxwRw0O4IxnV3Yyer+zLAd25di2qS+e8AOp5qNE5Zn8N8p439MMlPBty8lV7IiNs3aNyoQEV6IE04an+2vaZgPJcBPoILgRONFkmKtHYssK5eZer11GRd9Yil96p9dkcfByx/7O23ABXLabMP3ruX3bUPkHJoMyEQiHSC8jnK+D1k00DSNw8N0E3X8cM+JFBQ9CqgiGSJ37oB1qbD3cmrdNBrgBn5qOa1QXjTkeG3DyvVUg1MOseTFn3FMUKVJiU17lw+hv2B33jdTFAJhf2pnt5GwgAdlV45HgrWDWc+jnem2Y5qCVuAFN+6dJbYHOZhWBcqpH1UD2kjFyTsr4JCn2dMhQ7mm4NnpmEuyplOLAPlP9POgu+dpYXYXpzQBrqqPhNg0IbupK5qnDYwy7TBbej2cSVTfMfTfMPKXur4ab71qONd/V2v5yHKk1o854V6mMxVaaVebaCavrEup1nT+UjXHoEKbb9dnJSrjdMtw8vivaguNU5Nx9/ed8P7rA8spEL/6FOgT4FXmQIvBRa//Gu/1nzu85/PxH2MXnoBAqrQFDDgxMXwGAZOIoRJZ7DMPybQTHj8dmJ1gnSlu0yC3PM+6ZCWnmeqfUSYTePKMHCtk7VZlUmF1M/npeRq3r6/8mgnrkxgptBOujV4QFAb2YHehdky+JcQGfxJ2Kk6c2mZCcN4KAWJKhTvDZ2zLVwJX1aEZQKGYcaKhOIcYAgqVL2RGVYiECYchiYGq6TtVZsFlhGbs6nZ5nR8rjkanW1WmeKfAMBWYWbHYKwn9lmlxnZiPMCCVXqkFW44p/xBYGEjS38LKGMQDzkwPVn1lImEYRfUVKAkYAgDZBsYjys3IW/0zGVuCVPrLEPmimpoS1qu1MoouWGXTJrejaawrRBYDCIZ0ATYlV6lLe7vIKOnrckpfax69bKRJbWHZXECLivAABCBCeXZZ5VfD0ICC/NT7cY6CToqGHE1vTBtRSVGnqb0xbMwoBon+9usZHi0BZDRNj+BleVzfwRBjd6HrmlX8cbd9MFNGPQH9x/FbkFpk96QblxbapYAFAEz5C1t7NO7lFXD6j3UtpTgqEbzHGlAVa8SrEd6QPl1J3rzBqDi1i3AxXWAxYPmgwcPm1U8LekZSyb1CEmSbaGEZAxAFcDB82OZdpjyaQzK5/DapOvXajj+GGnBExj7pytrMOHYMFEO+7+H7aer40iKuIcoaQslNvlmCWN/V6ohwx+VJ6Qx1ymfUgmlYDMwp4JPbU4EFtrxlI0qATzQ0X0sVrF9OMMt8vziUnMDe5ERmNgd6PAE4CSN3A1ed7Tm4fcfYJv2wQEBjK7SAiUP7gWixIak0n8Ef6pTpU8B2uzDo4TTKD6SNsIJKuwj9gelT3GDS5n08iQdlELYV/V4le+G50q+pLfAS0ARkMI361gmuIl6E+ULkLCPcnrY7uYnXR3bLJ+Sr4yhhI+Ujjy9GsN+W8CtY005Tae0TrlK+zGlb+RvWY3jR1LH1Ixb5l/+J66pH0PTSCyIp3TyDNVGPvgYdJ+ivniCV7cBVBMHAcODqptBtzn61A1A2ZsA5dvcL9FO7j3jooFjm0dlist4mUfl+fnthbteXdo+50tTqun4+3JaPvOoccuvj/DXPNq2SOiaZ3v9U6d3KctaTtMp1CgBunW5FOXKutV0DGvcPrC4TLX+7z4F+hR41SjwUmDxv332V5rf/Gf/LBNX8fBTmFSNrKvdQ0AEg/WFgdrfMAWZ8DKJCi7OgYUMQyZWrlkZ9+oJE+xzz6xqOQm0CTOF5ugO4r0HnczrwO6Anfs2gvf193mYMpEZtlePFlj0asS7qFGRTpLqpVPSq+8sQiku+XIvIypsvXj8AABAAElEQVRTIUMis6a0ws3iirG24OJcWuFKsd6VXFmObjeMc3bQJa8znrkD78nETHMwOsOeFVPNY8q4CqO2gSrQ9PZmM3Ow20wf7zdjp2xIxr4LI3iAGieea5PS0lV46SvzJFOQMsOUyph41tVV94rw3roY/qiVcmgoTEKRKklz46S+/uBwNbfSVJedMkB1528Bk4bLM3iMsp4gDQxJ8TjESnA2aYOxUzpkiurkh5FHHS6SFRjUGFyTvWWu4MHdmbPbMeXbRw3qEIBln7Ls0l5m1I4t+FG1Q/BX0gPwUFaNxmXEVZlRH193rVVtT4ab7DhIiT7rKriMoTr6N1ldv4m71+Vry1kZfwyT/uCDh6n7HAzYm2/cae7cvJFN4CyH/Uo1G3e0VkVHxldG2/ZwlTx7LcBwKoEgkbS/Ug8B1M3rN8jrRly4rjx9gsTiUfPo6WroJiCQ/jLYE6yi64Y1TKnFRhIkY+gp0+5GaPE4Rb0FFo8ePc1Vg3VX+a2b/VZ6BFzKDNt3/V79DqFrvg/bCPoKkJUuyeDPYvy7DE00FJeJnoYxVdIwSb4CKFWv3BFcl7EBt6SltGIUJvY6oEmJlkbnT1GD0pWvtjvWR+bd9tbzk4brlk9m+uaN5YA7wZRetQRngjGlLIK/ddKo0of6rcuAC1JMxDIaVkAmKNnYQJWOeKp52t4BTQEWo6GLkj73sVCFSXpbd2npNy1tSt8Q9Bb1MNX/XDyg6O035Rr/QOhcpStlcYZxjvK4oOLvjHXEsc5Gtu942vfLwgXAgPCOJ6mDfTjAwlB+moxWnvyz3mm/vOEdv4+p7ynlciyhEqTD6TfAt6i9xQnups/4ps4A/3QmXP7iIpn+7oaTr81MNq/hHviGAI3vMYbcpF1yLmWVNvUoz+uvi9cazv5Uj5S3/uDafdd5HPp3f7/svqaRskjTNj+f1/JZgu59L73ui97Dq29M73LwOg5+KEYb9kPPedArL2XtA4urKNR/1qdAnwKvEgVeCiz+3v/1y82vfe5zTFpMgK58M/EbQRURJ0RXmB0UYT3aOtehmp9MmszOYfgCKpj4FOk7IUeFpmVaZKpMv6i+MFA7AOc0yTZdLqbsoB2m2Mkiv8ufTMw+IJ4Tcz3qIJ8rj73CA5BqCXOeV83Ta8nVdzV/2VXm5hLfxE2n/Z28+JHgbYzkwzMZkKhXMKELKMZlbLgW3fGyOizAkPmW4S0r/zAAhCMy/5n8YXRYKm2OWk9Qm4PjzQOYx1UY8u2tzWZud6uZP95r5gEVo5zuUYEfyriZla1R7Uz7lXhmopCl7m35YVRSLpl/yyEjY8UIJ/NaVTxkyq2/jJDG5dbPNjXMEc9kdoqUwrqwmsv7I9r7mLCuALtiPsK96Qg8eIgqBkakMkOS2T4Bw73H3gVKIexf2qNIp6h8mCfBzNc9FMZdGccAfIi8tlRncSUfBlXvRDKR1kkpRPooeURiRN3MTGZN2w4Z41XsDh7iIck9JexqMs5V/ctV6hSNeppmdaOqepON74q5K/8rMPuTpHULBvsH3/4Y12uxr5Bhl5lUBejxU6QE2EmoehM1P8ph+QQYlXE3s1FWxMdg4LQ/mSOfuG9dmIMm7MuAYfM9pCOuspuue2ZMIZWY8STOBGW0Dey39rsCUNljBGBhGwgUHz5+2jwBXDyj7LrD1TZAMJ82SNsIhAGGxBek2S6Wz29WxlYmV3sHDaAFi7bBPFIL1b2UVEwBHt0wcBRL320kSat4d7pPno4PfkPDqOHMzM03c+yuvQg4W8el7ApeoCKtYPXfsWBMCQzgxCM2CeRte7kvx+u3b0JbmF/GnTXUqgQsqtbFKB+aPAfIKJGw39iWu9iz6Awi0k+lDvQPgRtVje2QtBT4OR5lEYA4fofuXF6/f8clpRv2JTuJ8f1GCjNfHBgocVQNT4CnfYt91THpBOcIsR+jb5tGWTyRGo6DpQ8UhwX8hr45qa/5mIf1cEyQdlWqZr383iroIGjKWtqw9GG/k3wxXjnx8RAwH3ABmBhw3xoXMii3Rt1KM45x4HCqAwe+LdWidBCwoNSC612AxW2uemhzbwsXAVQDLPmYlflxUIjes/z0CyqHz89/lTL7xudt7JbmbYTuxbipYIlXw9d86++kT7huPvVdN7nL9ylbJ/36vqZXf9e69epiXsa7VAfzLGFL3QzTK1WnLjVdrwlJuJ/6zGe6j7+r945R3Tpdziz1pEz5/umXAmnPJTy59Y8+BfoU6FPgRRR4KbD4hf/j7zf/9z/9rXZCLNKKAAsYIvW7C8PBROng2j38KRPKYFRBgzYWUbFhQq2MSgnj4GwaJtAbfr3Nb4fcHA7e3FZgcT54O9m0MY3TBu8O9g6Q5awDfkk+Az6R69WJvaRlmilAydNkSSN5W5heHu0PC8ZRsicvJ18YAM+oRMAgVGCh+lMXWITpFVRwauMggxNbByb7ISb7IZjMQfeHGJ1qtgYnmmfsV30fJmmdFeHDna3m2sFOs3By0Mw3MJtILc4wDD7GW9IQ5XcvBxogzP8R9wF2beGtjypIrrJOwgxNtAyZz22f2s7SwXpHqiHDKsLgt2odrvpqCC3RYiAN4xEmT6Yapg8ChKmV2YNrzGpngAK/YfNiTKpnKGkis7LPiv6R6dG3pJEgUKIWaULpZ6p2jEETd6XWHesODHJcouJpyd2IBUfTMNplrwS9JZVN6DJxk1zqTdoaeiuteACwePT0WZhxy2nd3X1b1RWZvyEAjnr6t3DdKpNum+rVSTsOz11sFwQpN2GUPwmwWMYTlN6LNNSWNusYMQsIHpOPOvmq2KRvpK1xl0qeSgCUWA3YD/gdFR4Yd9N1jww+tuzp4P4XT1aexSbDFedZpCSL8xhRc9p+Mnt6ftJeSToJvDRYl5CqXMnoa8TtnhcHGC5n40i/UfsJ4aVNUQUqm8EN8A3bth6RktgXoK+r/tVeRtezkzKi1Flm1BXto33oAtBzJ3BBm2BEO4x5JBxL11CdYv8EaX3//gfYXDwhzHrsGZRc+Z0U5p+VduotOy/jrm3LdVTNZNxPkZi6X4vQL5IryilIsj7Z6wI62meUDgkQ/QZiK4LrZEcYv3Pp8Rw7lZ7htn2VeKo7RSpBPaSL/UGA7fdg2yntqsDL9375frO2oXRJW/LbfivjpvQsZ+I7Bpazpq0KXJ5Rh6ISSnvwz88siw7Q2+/PciQvwTM0CrDwOWWgeqUN+S34s3/Znn6nLgQIVWLEDfB0Pws6BsACWxpA2SB9hM7He8CFklEli3pto8/PcS7Rrrext7jDuYSEyc0n9RBl/Up5/HvxsB+96Ci9ibeU+09z1NCV5jXu5bx875G/7f3lMNKlBKqp1p8Xf/u0puf95XRqHkmNNL3W8PlVyUBA29TjwzkUOuYv5e0Di5Cp/6dPgT4FXmEKvBRY/J3//ZeaX/3NXy/AgonPidIzTKf3rnYy4dXBtB3Ny0TncxizAAtmvqzQMXAWaYUjbTl9f+FoB+g8M4yTQE4vhflxvM59J6IDdspRB3PD+tDo/uN34YlLgISnwFl5JB/jdoFFykfQTNGJayqmV+JniigPzp+ZJxN/ndwFFTIAGjzLDIzBlIRxYiKPygvvZR4MJ3PCTbvaLMhgBZqJfggVhSFWEg9HJhulFc/ORpBYnDTPYd5O97abG2x4t8D+FLMYbQ9iZ3HC8yNWat13oNDWusFcUFaZtELtQgtXPmVsBRZes0pNOFeOc9Ju0iXAwrCqkVAXaXcIkyZTr+GxEgbBgbr9Mp7uS7EHo6ckRgZSMKHajRKLqIrwm8Vc6DYYXX319fX0oxewAfqUkhb19G0P+5p2F67SywTqDlQQpjvXMRhZpR6qxMgs7wO2XJ3W+Nc9JFxB3osOvSo/ZXVZBs0e4aq7Xn2e4KlIcOFK9RhMeJg46KvhrhIFpRXu1XATewAZQI2DNdiu6kh2ARnq64uLzVsfey2eoqwLHSu2FzLz37x3v1FtSgAkgyGjHHUlVuf1HuWpC159D3naf1ThUgowA3iCfYzK0GMMr92DQimIkoUFXM+6k/XNpfmo2dkOrsJnk0TKpfTjDBrLGFtXVc92kAipbnawU4BF+X6liN14EEdBAAVWqAU1qsBJD8Fi9j2xT9EmtrMMvV6GBljxtr4zhNfW4gxp2fPV1QAzQY4MvHYeS7jd1Rh9cWk5ddNF8LsYbD968rR5RtvFvsHvhO8gjDH5ymIf2/94Zj8SnAiGI/WxFxLeb+dQ8EhZ7V8CC/uxIMD28vC53q2iWpU+hM0LqmDb20g0aGPrpITHPqH0SRBDFNJgzKMv2A8z9hHWtKWH5fHDijSWzux3ETW79E2+aWlDGl1gEaBBelG9Im3TNQ/BRVGLKvnZR4ybRQeBAj/yuVAX7/3WrLf9JGqlKUeqmmeVfkmExyk/8bJHDePKGapQg4ILJYme9LVj2v6ItjxCmjjInjYTtKXtuQA9bwFu7/JNXVcqI/Ajrerc4vI4XEphX7JHXX1QXI78uTrAFU9r6LTVpfeX8zKMZz0uvO+WqxOmF95nl8LUlLrp9MK3mVhb39fn7WzBQ15YnjZcvbY/89p7wxu3DywqZfrXPgX6FHhVKfBSYPG//NL/2nz2N/5xJq+I8QUTLcBwNVH1KCdGJ66MnO3VFbQ8z7vCnLqCKDMZtaWMru0Q6zPjcTgw905/52l7w4Tukfe9Ybr8NqQpOHy3qSbueXyHbf85MZvIeXEtu/n3TtNpE3F+cSJPzv5o4xHk/OiU2QBO6p4yGjm5jzcbGIEeqIAxKJ5dVD8qoCKqK8SRYVRNwYl/GMPKnsQCYLGFN6gNgMUKbbCNPcIJnqAWABPTGG3rEWr4CG9HARcYL8PURc2MMsnkuPIuM1IqUSook6TfeoGF9zJiZYWX96k7F+sP0VSzEVhYN6ksQJTRV11FptP4YaiJl70iYKC0S3BvA3XaZTAtj4fqTAGn5Of+BLpWFVy4aj9OWTylmeFddZYpXEfdSaZZMOM6saBFCcIYzKYG4TLVbqrmXg7uHeDq9iKekTQ+1ihYVRTjKIkwb2khWNkgbaUAGs5PwFTqdlY7AdvOVXfrK4M+Q1ruWP3kyWqzgRqOdHF1O1IFgMcie1XcxLBbV7MajKv+tYn9x6OV1eYb/+ZebAI0LE56pD8G6HGFW4Nk1Xzm2bWazymSIEGYtHNXaY1o9yi/q/+r2BCsI2WRJsZ1jwyBxRISCz64GIc/AXzoIljJ0jRG0EqV9DyljUfdeFJ7BN3DZjdu4qWNqY80maWe09RnCk9Tw3SDMPQADTe3k4bu9mxbKVk5CKhDMkXcAIvZqUgT1rAHOYp0oLT1baQ9gorrnALCE2xoNtdWmvsPHzdPcNO7Tpup0hWXsDCuMt6q7nnaVjLPftiR6thP04t44IdKWHdHd/dv+3KRdBXwaycuDP8QHsQ2I2Fy9++yOzp9Fzo6npFK+rugRENvJVQxtCZv81dK4fjgfYAWfUVmPMC9HTt8nkUF+o3tP0k69g3j7ZOnbW9bCIL9bnzuGOl4WMZRf7fPeGeFs0BBuiXtMoL5LfUkGdCiCyysSfleGUeIZ7tYMZ9LsyNodAJ40HhbpxAD9PUR6qz7ZzJD5omrYzfQw9ZimG9rcoYNHqGFLmdfx9biFn3b3bhHKEOkoUm9lO985OVhPaRJvW+v1vtPe9Q4NeblNE3PetfD8DXOheeXwhEoUfL3ReXqPr+UR82vXs9LUJ+k25Z+2olb31rG0qrpfinzT/31v15ff9evAt1Kn3rtZuozy1jn8r4qVJc6/fs+BfoUeBEFnCsdL6ZYpOseLEidnf3cL/7d5lf+8a9mwou4Hs6nGhzmKrBw4GVCdIJ0FctrlU6UidNpjTAGa69m1BuEM6qXBw5kMTzkZR1wE9DATIo1ThmjS8QilRBYFHBh2vVIvvlhuiVP5kRuWiBimSyzdSCFcrkQqze5G9/1z+SaOKZjUmUSt+wBFEzQFVSEIYdZc18ApQNRgYJxcmUzdgk8k+nnTzHW5p0MrgzUsGpQXM+BBczdAMzq2XCzgfhhK0w9+yagCuU5zjmFStToiSpRGKTSqK6qq4ImwyioGZWZaKloLWVSJmD0plyp517DZY1pbTfrM4QLGCUrxs1qLGDH+gdUmjbhpJ+McPoB9BBglIO0SVcmSPq6Wh7PR+QzQjnKRohIWfgtQJiFiSkbdBUbA/ejMJ7MrLtUr8GAPkONZwsJRBgzMrHTKmVQNcoN7tyF+5B6az+gZyGZdm0EZOzMz/ZxBVk1PqUGgh0NqHdhxFXJkhHUvsEduGWo3fRuD7UybVRcnVf6sIZXIzcjNC0NjfUAJYhZwhbC+KrquKu4IEXQch/1pa+9824xLCYf+4ZlHoehExyoNlU8OM3HbmEIxk9piSAu71TZcuM47DQ0hFZCJK2l2Y1ri8lXNaQAK6QS7hWhhEK6zwE49mGelVboZlamVjoItgwfoEeDCm5llANmAAcCh3FA0zj9T6nJNExy3LvSFtJTV7N6/NoGpG1tI/mAPvZn1ctO6HsbSCz8XqWxO8a/dvdOc+PGDcDTAvEZcLAZ2dx41jzA/kI1qA3oqQrZDABGwK3RtwDykLw0qDYdQZzfmuOO4NDRQJUcT8cde52SCfunKn8y6wRNb5fJFlwKqIzvIocA2n4kLfOdko/AxrxsW0GAg6N9UFo6ziih9RAYK7Xwe7afqE7oYVj+51vRNW1U4ohjeQsA19Wx6oOAYwKGYUt5iOT40X6bxSatXbAxP99xZnzkt+p+tpmLBX7PphU1U65lDC1jkbWHPD1goVTCnbiVCZ4CIE74Dofoq0P0RyWk7myPOTy2FjxjQ84pJE3XAL2voXL3MfqFu3HrdjZSpTISJgeKdvXRvqivLWc9vLes3+64EKcN/O1imctVYWru3Xc1bN5Rpnq9qly1vKG5Idv6JI4RUqcSs31VfnTqaRrdOhkgc1hLz5/6TB9YFKL1//Yp0KfAq0qBlwKLn/27P9/8w3/02Uy+Wc2GkTwX2yuxcJJmWA2wUEzBwMyzqDvxvK7AFeI4/J4P6fCTmZR6hHPw5SwTo7dOtDVKeX4e1rjtcJ5kTbd9Yjoclitly0/jG8J0/eN7byxyWQ1tpxRelLi+SxkoqEbBxhFYyIgk3TZfw8hMy4yoppEJn4lfpj0MCxO5qlABEkzKMvmqfSRc4mFwS9gYbPtc3WdXEmFYvA7D8A6gDnUwPN5sI7HYAljA1jbbMEY7SC2O2cNiaHezGdvbbGZh6iYbmFJYh2NsLWT4Xf21HIIKPevoFtTD6gt4sroqY0EYmfgdAIArWdbXcipRkMlVd1z2TVUkGTKZdelgvPQN4sh4CRqsu4DJVXuvMnLulk33CKOoelH2sWilB6qf6FlIaYH1dgfvCZhZC6E6zQoSBQ20NzFWV5UnTCf1V0Lixnxel1rbBrKOhyGZwRjhWi7CCoYsb2UWzdNN6QQqpqE60wxAZRwdc1V3BFQBEhgYP4PR3drFixCr/tpVqM9Pk4YRvYl72ddQk1JyYF1tZ5lNddTXkDB8wKr81wEWeoaSAQ9NAUEynUotlKIIajSA1mBbt7yDECpG6NRrbBzVt/sPm/fvP0BtiD0gAAr2Ib1Q3UBC4g7bhpV51Z2re1RY1xFAwTzuZvcBW+7VoKTF1XrDCShkaiEIFNZmBSbYNgBQuWIvYBLkSRNBp+DCFfdILGCELbcG+lsw6gILPUy594h1O2Vn9x1sSQROUzCn80hy7t69iyteNtIjrT3acBdPZltcH2JILo22AbO68RXACA6eYBRvu6gWpr3DJNIMDdUdTwQHu6gwjdIAAt5xpFMyZlHNoT56MhMIqo53jNTEOI4jR0hJbH/bSCZd4KFr4TwjLaUdqp/5W4mGwML+nB3b/U6JUw/7vOCi7gY+AegxH+meb4eAVV3KfEzTcUXpiFIybW8cdwQ3AjzflfGDuhC3Agu/Xb8dv4MACwtAAO+ltcDGccTDtFULdIArY2iFKSYP8OB5PEQJztx9G6p4njLexEEEiwaGcQ/xU78rpYjYwlzDOP8NVO7eAqTeEVjQ1wQWfNUpq3mb31WHzylRDkOEDp2rL3pxTUM6XTpqHOvl4e9ubjVG0u/E7YapcU3/Qvg2v245i/poJ6FaJvKvZfWbkaoelqemWctfq1Kf13iGP4/pr87R5vPTfWDRIUr/tk+BPgVeRQq8FFj8zM//XPMPPvsPwxAeM8G5+i2YKOpQAgiARc4CIphdGVsNx29G1YvAopDHgdWB12kvkyU/wuP70GGXqzsvl3B1kiSs0X146XDwLuyRr0t8nwUA8CaDO4+TH89lPH2a5/xxPM+96Zh+HpBijcON5Sx7S+jNs9Q1ExzPBRVRtwhT2QIIGJVsFEdmTvyeMpxFkgFj4z3vYldheeRSeTagpALmOrtTM7nH441qCmxktTcEqBiAweM8Iq89aL0Ns+Q+DgM7m83IzkYzjdvZyTM2BlPxQcYKZk+phXWxnAKJMGHWDQZDRkpmXsbU+sh8ueJfReTaMqgWojRAcKEqWxgjVpPdBdq0ZYLSByQiGVlPaUgWYbBlggv4gMGjngVYKQURgAyykrsfJlim03LURlZqo7nqIUBmCzUcwx1xunN1VWeR8Ts+0g6APSZgsK8tL4VB39WwXYkN9RFcWTTr56qxTLXtMYfKjy5vPTVYXgDYLLADsUbhMtr2nxUkBI+QFDx+utqsIC2RIVQaYH2tnyvct/EG9fE3Xmtu4Qo1/Y941lFgsYKU5X1AwTuoQrlBnd+LRs9KUpRSTEwCGgmr9y/bfG5mtpkF3EwKKnluJtoYaIvw/gcPsru3NA8QQmpwjd21Z1FREbie0nkD8KivNggyM3pX2qbMSk6UtMQTlOWnzUoble9LtR0NwZdgIKeIMymYgA6CPdtFZn8XQCWYVH3IvTSUJqgKtX9QNivc1jDdvoBEQonLHBIcN/lTUuGZvS4I//wZqmTQZQOjdgGE6mKOF6pS2Xdktm1vPYq5G7R7dNhPfXcCw39Ie7nD+NgoNks8V22uAFnBPMwyNLaNBAd6sSPpfL8Cj3wD9Dv7gqBCYKmEVQ9kbqRoWD1NKVXgK0884xjeqypRMvT5bml/8xeA2q8FbEqDBI8BbbzPN04jKg2K/RFpeFT1J4Ga6naOpwO0n+NMCVDAhd+hfTz9Le94z/8KLM6lJba9Y649ncOxhX+WO6fPiH/Kc4HDEXkdch4BME6UXCAli3coynfK2EInbQbpXxMCCwDzm5wfpz11PTtLXcaor6pQbWlNPf0tN+0f873AUJO/zzzq1fsapi35hTR9f+EgQ5NIfG9Is1uGy2nUtLtpdPPu3tewPus+r/e+r2Eu51nDpHBtZnlGnHqc39Un1OP8tr07a/7mZ/7Gh55+tx7Ucd70a926efnMekSyxnfVV4XqUud78/4r77FpLj3rz73pwtz37+G384U/2WnuLo00b1zHprF/fE9R4KXA4n/+2Z9p/s9f+QdMWKyIMdE6eRUVA8T0cJkFWBRGu4AKAjoQOThTzVyJk38+4HCsdTpwinWVLoOXo64Dd6YJB3B/+8jJtlyzLudzjjYp7hj48sRb0ywBkndyPQ+r1nXyJEgvjmnxo04M3ec1rGlajqz2EdM6Sw+PTPBMssXVagEPUVOgXtatMCP1eQEW2lJEfcrJ34pyKrGAa4lthcaUowAJ1Xu8HxhFT3tkvHkOoNgaHG120I+WEdXL0x6M8x7AYnBvq5ngnDzcbcZQhxpGcnGKxEKjZ6ULZpNN6GDOVDWRSYs6lnnaBvyLsSuMjFep+v+xdyZgfl9lvT+TZJLMmpnJLEkm26QrLVootEKhtaBwFbBSyy3qBVzQ595HXO/jch99XO4VvSqCyxW9+igKLXCLLJatUmQRWnYpu9AmbfZtMpPZZzKZ5X4+7/md//wznYQGXGqbX/Kf33bW95zfOe/3vMvJ6h7ZmNVVdJlt1UhcsVffX2Ah42YdCrCSoTY9V2FltCR0BhIyyVCUevq+qPkoaVAtx7LIpBlWMBoSBphjPUtNq57FT5K7z4ar1BoIz1J3N5zL+ahO5YZw2AfAEGmwLKAQjKi2lIFw9mJlPzW/Fpj7UVbhXRlXSrIZUOKvHWZKxlHanUAFS1BxBDuJw5wn8KQkw2oZpZFMbTcrurvYMG8L+1voGcsOJfOp1ONEDVjsDRsFgZlSnC4YeO0/WlD9UbnGNWlVoNazeqyNRidlcKVeQKYU4oE9D6a9+w/EztW2gXRS7UqmT6mEq+2mIcMqaJD+Mrgy7KpBaeytobI0kbbR33ln2/lrhRbu1N3T3RmegAQ3GqwHqIg2ycbjsckfjLBpzigNoB/OUacZyjiGSpODifS1O/eyB8eW/i1p2/btSFXYNR56TSj9AayNjI6E5GkESZCSBft/IwytAJFioaJDX4Fhb5DZt0+S7iJ9QfsOQYteoVytlw4y70qnAsBTF2kcHuhkyvlJQ9vXvmbfESRl4JFdwEoLv1nBS7bp0ZsX3w/1kJYkF2WKMAI+8opvhnQyyODbIZT7bQTQFiRUZbJPy/wLPgTpebzI5RagKX1TxUx7lAARxPWjgXwR1rIJKgKwSZj4n8fEkFiQtmDcNoyx1sJyeM+fKHg92KDgGViQA3g8zaJSOVcBjDne5U30oDnAokE1wwpYDGDLc/HGDWkH6lAFWKwmnuWMQsW5yre6PtepjLeGibJyrooe0XK6cUmA6lxO0kBarHRY53IsC1PLp3puGtbAuWn5UUvf5Ja9LunUx6mFL3laRiPWl4cIIVWLiCTM//qkozyEf/EFiUVQaGh8Lv3T7ul6Mj/sWrfWN35L68OePxYevP1jI4wnAM0bOs+rOn/2nhPM+YvpZ1/Yc17xvpnA32hZv5k85UV/5Q1H0nVPaEk3fRsu4C8cjyoKnBNY/NGf/km64+1vDabFCSp0l52wnezil0FFjKf+qX4OmPlSRsch1nOut2Ot47WDuoN0/JiI46n38Zy7eMaZZ14KLIwbaZuUNxyh0sSld8Yth5N9LSwPA1jwWsCQp25D5jClcFURI5UAFqYYCcs8sHpMeDt0CRfAgQk5mBqYnAAYMC8ywDIfMj6uJpf3wZBYGRLNtiT52s3w1FlfJRMC45ntBnD7iArUwpomNsVbn4ZSIxKLNWmaFUZVmmyPU67IT7P7NvYVbacnY/ftNRhvLyK5mGVl3xWmWZgwFSxkxNyrQmbYXaFlsqRttCsMjYBCRsqpVnWRUOUgrPsV+HNFXfWlCRh1bQ9UbTK8dSsMlMyOxFGqER54YMIlX2yYRx2V+phneN4JKQhp0p9cgY+VX8LItLsBY17NnQkmz1VkW07mryPsHwAXgJoJwI2qMfaxUNlitV2JwFqAkDTWy5R7fcRu3qqgwADbb20Hd1cegeEeY/VbQLANA+OdMMI9qDbZV2WiTwIM3H9CYLHv8DHqPRl2CqZhvWRo2zEe39rPLtl9PaFKFeDHcsKou9eFO2bfv/uhACX2O2npRm+9MPGCIVe59UwkmFlkIlGCtBEPU12sEMsQj0Hrr96/Oz24bz/pjQawt/wy/QKBbhg+JSDSPdRhSMt2V7qiypieqLTLyLYltLPMa3Tg+BP5CiK6utoxPt+I6gugByCgQbnfoO0jE6yUSEZ4ApoNnUSdiY3jFimrEhe9TI0ifZDedm+Z+P7tW9mhfHsaGNiJuhIb2AFIThw7GntXuFGgtisCQw39BYKxsSJpaRTeIJOPfY9qTTOAmFklGDDuSjKDOSeT3HezOpE2DoJlgUmw5jSgfcJ2su8VI/tQBaRfCLAc+KRXfOekJ00FytIpjLupi+2iBMW84lumXAKZvJDA4gF5wl+HlESa+33YL/x2BJCqu9kvjSvNA3Bw7bfo92H7qAY2C3D2m5HOfo9C/disknw9fFYbb30f5WFcIR2/G8tXxjRj+N762MLGDabVhuHZgvUMMAGwANXGD6LNEWcBdagFv3MWNVYBLNYiQRO8DtDPLsXzWAALwF6oQpGGtIsKk4dliFvLwv25jvr3EauKG7GquGeklzts1W85rZB+CW++9enUlyPoVMVdKY0StryrD1/eea7Py/sSnoulMlqKql6GiXecKirlR/kmXxPXdC8AiyBHeuDQqfSWe9hfqDpOgYRnGR9b1rlolwknsPiFW3pLkEft+cNfnEgdzavTky565FKE17zjOGrMi+mXXtR3XvX69wAW32hZz6tiywJfABbLCPIouz0nsHjNn/xxetNb31KboAQWsfoboCKrPMU47aBIxVSziYP7eM6Ng27+5xsYggiQ/ziQxtgbZ98xpXrNwFHOjtTGKcDCa7PxaZwjgTxgO9EtvSdMXWZ56q3SrZ5HuUrZI6bxcyUsTS06efhUEaMTvNeWTwY8SyUyiHCVXyZiDUxR2FvIoMe97xkQYXz4ExO8E3+ACxkAGSuYBH3KNwYjjwoIuv5pbXOoQJ3EaPvIApvBQR9VF5pd0aXcp5FMNAAq2uenU5eeoXA9yxJ9Oo3dhTsfawvgimfWRYfZgWlTWqAdhMyHOtxlpdR2CrULwILAQ9UNw8r061pUJteKm6ZeoNT193AFV+bJenpIH9OUYZaplakXLKguEnQjXDBcpK1qie+luum4qitblRkp1Jxg7EwnvOm4isxAqySgDTUjXYPOnYaZ4z0ZBuiIBoOWjTBG2kqYpi5op2FgJzEY1n7EtLQTWYVu/jiMnQy4q8pb8Vi0c2t/2ratP+xQ9MQ0chKj8RHc0QIsHth7EFuLiVCTsY72DdteWnWjkiQD5k+7BIGTrlE1QtbzkcAiPDVZd0DA1i2bY0M9N7abROI0CXNJYrHqLS3aWtsJ0xduaCdhwr96/560G3UobU3U43ditT3Mt693I7t990S/k5Z6jVJCMUadx5FSqGoTDKuAmDa2LZRsRGPSB+yH7nQtkFGda2tfb+z23QmwMD2N+QWTqg1p7O5+EO7/MAkzPEv/ts/qKGAcYCHDLthyn4qBXQNp+/ZtacvmTXCtSFBIZwpJ14TtQJtM4RY5QA9ub6foUy3Yl+jitBH7DV0EW289OZ2iDnMw3RQc0CjQ5UfbK6mSGbcvChiUKDBwxKJH2DoICGwjaC4gjf7M2bamytF2YfvEdyddBBQBNsjH/jcGoLHNBBfSKH7mwfsamOa7cKxSimt8ohFO4KBqm7Yz2ibhSQ06ewQI4Ow3qVpe2KjRFtoRKZkSAGof4mgjeFlH2fM4QR60ewAfzjE2RdtlaWGUjbpnQMG3RDkDzJCXE7D1YbCJtGrqUICJWX4IS0J6oXrUAuVeABAt4PoYkV4Ai176xUB3R7oEYLEdUN9Ge681LdL251Hq55joIT3rz3Gzwh9Dl7Dxuopfgpb0yv0ZYZfFjfmiLr5hoxScSzqeSxp+w+Uo770v78u7cl4pjGGXZglyK3kaibwyNbjOBclJRRny5VIAghuGd7fedGGDvIo6Z5zuvm88ffDz4+nnb+5N3Ruys4QzAjyKb377jmOpv7sx/dB3dD3iUsYQTWjZhfM5/j2AxTda1vOp1/KwF4DFcoo8uu7PCSxe9cd/mG5/yx2Mi05O1STlxMtPkBFSAZ47KJZJpjZZOFByxADPgFk/iOaBNsbRPDA7CNf/mMjzPQkQNSYx3uc8cllM3hXwKhvH5MjCMDGk+8CjnKr3MvP5sFY5tkHyr6RXTRj1EwVhnIoqWBH5FWDhKma2m4ARYFIvUgrVO8okL7CQCWF251z9KEuAC9/J7Mv4uOLOvhWN61vSXCMb4qECdXQBA14YgAlUFygSm1TBWFKSNYvosgMoOhb4zU2ltUgrGpBUzMG4zbAirGoHKeOpKOvsywAEwCF/q+4qqwxLMJu2p6ofNLSqNU14ZVLXvlHmCEZdKYqH+vuCC5khV3YFJzJPSkKyjUQECyZNxlEPODKctqc00lhZYBESE4K6gZ35S1BXl8OWBSbHtpGuMu+WUSbvFKvkMtQCHvOTtgFXYOxCRcad4EmzgZXXFo1PYe7tm9plTMEcT/IbxRZAHf5ZV6dJ1/yaACG9qHts6e1N27f1k25mlkdR2TmKh6P9R46mPQcOp5PUR0PlebmxqnwyqqoRaUjdv6k39lEQuCkZGIGRPorqz979h0N3X1uODsINoDq1FVWhVurhbtHF+5Qr2AL3pnVZCtIJiHK/hz3sg/HQ/kPhGUpaCCy0nxDI9GHj0Q8gcIXc71KJghv/aRTtzuLj1Fkm1qO0c6hMcW+flykVCFmHTYAUd7feArjowhj+FKBLI2tVwgQ/AVAoX6hWTQM6+CCUWGjPMk49ZZj1htXT25d2XbIrXMwq0VlEsuaeGK7UT+kRijSHAWwj9g8BCq24HnBjWnOquCGlmeBdOBJQMkK/tJyCA3fmNh37ntIE1Z60HfLDUE1MZj3GJ+lk/fjGAlzAECtZKz/BcAsgxvRsKwGnaQqKTUdvYfk+M6B0ExOLOiodkd4CB/ubErY5QK/ftaDCn9IEyyyVMwOaQr3N/O3T5hMfMyH8llQxFAxOCnzpX6rNWE8BhukIDsKmiHgBFLj3iG+a97HAQXjjRP58G4bTJsw+FQMW5ReWKP9TagG+rn4+453AQokFRvfox6W1bGi4SWABqLgE9b1+JGz1wCLGTAhTzlGgZX/OyqhX4WJ+qItTz8BHmcu7qr7l1nN92rV4VXlq6RKvvPNc4tTTsLxfnmZ9Xl5bT4+gfK08S0+9yq0SwfKfWrjq3QqBSgpmcOsFG4s64i1dngtYSOLjI3NpcPQ0dnJr0qau7ECkxHbl//DQ6bStGxvFmfn00NHZ1NdJuM5G5oCzvyvxRybm0+FhXICvXZU2k/Z6zsuPCEMetv+OPhxfrGehCJWkobG59Fd3D6XejjXpe67NKjvm7dw2OKor64bU0bo67Tns+LOYLt2GZzqenSCe9eqpA1GmfYQ8TqAmtg5pjeXf0JIX9Ep5HimwmDq1kA4MsjBDnp3kv7XHBZCSigsOS3Tx6cETSPyJs41wLdSt/lhe1vq4zuP7j8MvsIpx0WbGzMacyejkfNpH/t1ta6ApUuq6vE3b8fTA4Ok0NsW+VE2rAGZqWiwFelwCi+kD6RN3vTd99HMPpkHWIpv6rkjXPftZ6Wnfuj1FN2EuPDU+nA7u+VL6whc/l46c6k5PeO5L0rN25NZaODWajn7u3en1d34+jazqTpc9/0fSjz79X0dl7pzA4nf+8A/SbXe8OSYoeT8ZEydtmWsH5jJmyrzZ5KXZ/QBqL72Ml3UrOPGePzyPOMFgc+2kENec6eVxbzACxWqc8TjM2+nefCKvqlf6YWRmAia2eubJPHLa3njHQUQ7fbzjtnqa04sAVfqlkrzRQLZKjMnePJzQOTOxW94AGkzoMh7xk8HgPjPzldEng4YMCFxUlMm4Mj6qQakGsgb7ijUAi9VNrWlq9fp0jH0r9s+tSocQBU/CSMFNsWKI4TBE37BqIW1exWAKsGgFWKxmFRguEvsKJAowKDJ65q9nHxmPmGyritqWrj7LFMl4+E66+lovQBomt2A4qzGzuzfLFEscwYpG4TJirrC64m/53QOglbABoKCxjFO4ciXMDOWh1pURq/sz5I3O7Etl47VgfiBL9DPyceCQCbMs0liVmRmZTFrYuuj1KAy+YaSoaNhSuKK7ALgwDTeca4NBMozARgmOwGKYPSjKfhgkFMzhBlzdategy9N+pAmqI8nAj4+NBqjYizrT3sNHYKgBbDDUbsgm47eKtnQVeiOSA1WhBrZuSc2AMD6SYIqHATG6fz2AGpUGz+6H0QP4uHhgRzDwGkqPEyaXSa9X7gTtpnwYhRNuAzrtcwCzQ0eOh2vWY9h5+MHqZnQ9DHy3wAI1lT5AkTSVoZSOpncMMHD4+GC46FXSEG1PmbMdBgSyRWhs+5800lC7r6czbe/fDEDqS90YXqtmpj3EYYDVFODMNjW85RzD3mQSPRq/w1n6wDj2E/aDdtSotmzLalAdSCFW2wcJIxPfxb02QSOAiuODbBiItGKa8s7Tlm7gqHH4OLYa0xibnwLIhBSAFi/gW2lFqDyRnvYMgkLzFIZattP0jxgbyNP62pfpOvGdGi8kF/Qpga0eqNphlO1j9kMBo31R8CSg0J4mSyGkFV2M9EzQs3QWmOg5y3ulJ9qJKN1TghUqgcQxXSUeC3LwHHq2sp1CPQo68j8O6x22S4DCkXGlORlclLHD/u5hXktqqLmOvvG9353lEtz7ky7SQpBkObiMcVibGNWewiNUHbiInbmLKhTfziJqUOtoy03YEA2gbncxAMMduANYWHby9SeNC2NezjyqHUG3cheklI7GyscZ73l0RholnIXnMGz9+/q4Ps+hbO8lAFEfJ8KQhuF8XuKXuaLk4flshyUv8XKYpbpYglKGeFeVmxhLz60TgQxnzFLqSJf7W7/35oj6b/HHb6bQs5zr8y30sv/4cyHK30Y8vP1bH2cDFidh+t/8jyfTfphUGf4ZGNiNMKsvfRYbh8KweqhW9VfvH0r/+Zkd6V2fwq6LMP0Y/f7U9/Sc8502Dnd+cjR95oGpWtoy9LeQTr1xtKpOd983lpoaXQTjG6Vxf+amnvTQsdmk/cHy4xXP7w4GXRDQhooUwvN034Nwihw/jFTj8m3r03KAcPTk6XT7B08GqFAFzLLJgzzvmvb0DOwMyrE8Xnlef373p8fSxzF8dnxQ5VIgsKmjMf3Yd20MQGTYQrMfenZXuuufxtJxQJCHef+XGzvTZVvzQqPPludZ4r6MuO/9DAtTgCSPZtrnv353d3rgyKl0F8+d4z2uoL4vIaxjtcfnocXffWI0wIj5FRW4lxJmJ6DN4/EHLKbSl+78m/S+/e3piddck67sT2nPxz+Q7ju5MT35hu9M3/6EjWlq32fSvfd8NH12BGDbwJ5fI23pihe/Ir3gIinGnHZ8d/rYHXeniRtuSk9ZdX9679uH0g2/+oPpilXj6dO3YUv93O9PV/ew6JunGyN9w8c5gcVv/8Fr0t+8+c0MpAwsdEJ/MiZxrgbNqi/kkbIqRhk04zZGzxwqzxMlxlIonzuIyajHIJ8fxLN4TkJ2uhjouM5lyVkGr094U/Wv4fxppJnTLNdQK9IlYFX2XILcuV3Z9/Au/5x48hPPcRnvSJcMTLuACj/wfJ/tKWRiVMOJ1cO4zipSchIBKjibngyTTIGr+Kob6VNeYNGAtGJhHcbFuJc9srA67WXB+QiMj8w83B6SisXUwWDUu3Yx7eTXsXgqNZ6eCmnFAgP/PMxRXv3GAxLpu2pcVkqDBuSuTYXAIut2u+KqjQW0EojABKpupIGwEhQKXa18qjqVVZ3CDoK8ZMYEMHqOcgdpGSupZceaYTVaJklmzQHMFWf3ntCYN5jgarKSqXPV1v0lZBhnYRBVtVIaIMCRUZI1UwFJdSlBWTBx0KwREGDbyZydDhe7bnoG20T7yKC7+R4Rwh5lFKAQEgsYuCmYZiUkrtb3sDldJ4yUjG8vEoCwJ6Fs46yc7z14KD148GA6wA7RenYSsJ2ivHTCYLIbWfHuxS5jGwz5xQPboXVLeMwRlLhZ3cEjg2nvgSNphjIJ0PoAILvwIrUVmw7ppeQkAA97VQwDQiZJew3hugArMqAzlFXbCu01hvnN4851lV6MiNuJaoq/jUg2BEe6rLU+YwIaJAj7UcM6jItagVQwBzSw368NLX0ECfZdJTYbABY9PR1pS083kove8ORkG57Ee9MB6h/tTDzbYgiJyEnAxTjta3vpbWqKPJto2y4AST/1szw6ClgAhKlmFdIV6KS72THAyhBAZJIJTanHKfrUScrrbwQ6KH2S4ZK2fi+2uxJS86a42NfAKNOqIQEDGChFyz/sdQhvHw97IcLmMQCbGvqSILUFcBIulgEXAmfBhmE0ds9qX0gOqNukwIJ0fZclPNLNTO1X2lBU3tQsG+VZQzoCDfu1oNPvKexnALVKVkxDaY7esQRxkbdhKWMAKtrZdjqB/YoSNXeuJ+l4H21F1tWA5BWXFob3nE2jLFAIdC2D37EvaotB5G+ckFbwPbuPxUJ8165Oshkh14sCPL6ZOWy7+HAw3kY9DvCqV6gBbHC2AoyK8bZzjz8PRj+LGof08ijl89q3tfsS0Bdf76jSInItZLRH3X3tBRcRKuhx7kwMZ3lKmUyzPkZ5Xp92/bU0z5lxImIpXSluhM2ZVNEysJMS0Vi1xGi7qi7VKb345u+rvf3XvviPDiwk8WvfNci3sph+5DvZ04dlW5nav/v4SEgCfu6FvUHuwug2wqQ+9eLm9LTLWoJplUk917v3wIB/4quT6SWAFBlpmXufPYjE4xe+rzckDa6o/++3HEs3fmtr+k9Xt0dX/drBmQAuTdiDjE8tpNe+ZxAJSWN60XUd0QWUUNjfZMgPk6aA5NbrO/huGwIUtbJCv5xZ/8q+mbQX8PT0yxn3iT+N9OD1H2B+QYLxy7f2saCVv8bl8VbqQ//vIycDGF22VXfwDelLeJK6/cPD6fontqbnP7U9ohS6+P5plzenG65sDenB6/9hGN6mIf0ith+OxR7L86yP+4wrWtL1/PYjfTAPbWQYbtIt0GLX5rUBPARuL3tWV7piRwYrH0Dlzfo8eRfeKOFylez83/eeiHr/JGDQ43EHLBYfTHf9n9vTA73PTc959lPT5T24e//SXelt9w6lliuvTy+4flfsZTaOlsT04ng69MWPp499bDwN/EABFlNpaP996T1/uSdd+as/kJ44/WB676vfnlpe8UvpaQ/9ZfrT3U9OL3/hk9kQlTkpKPzN/TknsHjlq18NsHhTDMCZmadHMEGdASzKoOxXTolikK0fYXlWG7YVbZRiV5e+c2CPyTPSIoI91uu49zanEJMKr4LJjIkh51cfn0XkiJ4lCjCixI34TJxcxpEHcSaWfBd/A1jwwKc+N4/alJEDxnNXSUvZYkWxYs4ykwZIQHUhSyuySoITveFkgnL5LWAGFuF5CSZAg23tAnQvu0qDbWwrZtawD0JDYzqM380DM3PpOMzODCvmoAFUP1LaiFhwEwPQTvj+zlUwrahELcJcz6MC5UqzIETwYP1dRQ+mrGK64HL470ZjOZwAQDUodcfVX2+HqVDvPlSJiCOzqycgjXczAEEnHYbYNBzOrJ+r3qpO5UNCCl5cvZF4eZXXMqhW1YFrU/uBK71hEIwKSLj4FMCSpp3SzdyM78AmIBGoCXBsF9ei9YzVDGBqgZF3N2fLomRCcKF0QtsLc3ZfENvMlVsNhke0QeDjcyVZVZYN1LMfZrq7Uw9LG+CnWP0hvMyiKj6Hjh4NcKHxtipAM9AiVqgpmwBTQ10lFv3YRFx20a6QHjTBxIa049CR9BAqVPsOHg3vR9oI9JLX9q2b46ynrWAAqdUcgHEIYDGKHccp6tIE86mbUlfux0bwwjWGahAMpy50/YQ0SA4mFWZvAwyrEhftIjoBRxqen0BqcYAN6PRmpWqU6lAy36r9yDi5x4dMtXR1k0TVrjb1YjROGt0Yj/cALiSghtZHoIF2FuZt9x+mPMOUZ0SgBY3sb6cBEG3QsLO3J/UBslxXWmP/gKnehPRlE897uzfWXAZP0ld1V5vd4Y6nA3v3p0HKO0pZbXOBcEuLbn+ZRChvSLAoM9mjLgWQ4Fn0LApkn7Efut+McT18ZhrWT/VEv0/7n7vMC5wFrHre8r0MlvXQuF8pnHT2LCDwsAuXBRXBqKCimTSiT0Ze9FPSVkoQwMYIlE4664EqPHRBd8vjJoKtALAA7fQTxym/Q/N2E0n3HNGhgPQxbq4b6VPPPI7l+lkuj3JnuxRwEeUAfIZNly+klXXgT+xoTqyAboALVaVmAXi6WVgApC+ysDEvsNATHH3QvrADqdgAfXwbErQO6w4NWNdYyjtKku9jjDOzr3MYbqUjYpb4JUy5XyHCuUDA8jxqparSq49bwn69Z0HIWjlyio5HDz94x/8CHHxfy78ucI5ZpUO5XnzzLXVv/3Uv/6MDi/sPnkqv+4eh9NIbu9KVO8vck9L7sceQOf2J5zHW4oq0MLqujL9smZ3D2d4p1XglthHXXtp8htehB1lt/4v3DaXvAkQIJo6gIvVH7xxM3/u0DcH0r9RiZ7OxkCE/MDRLOXvSVmww6o/lzHr9u3J9H16z7rjnZK2ePn8k8Ur8+vOr3no8dbWvTi9/bpZGFbo8E0DxAqQi5fh7pBdKaP4HwEKA5LE8zxK3HqgY7s/vOpH2Hp9Nr3j+Un0FZr8NMFse1vD1x1vvHUmfQ5Lxmy/dHF/b4w5YsHvZ5+54bXr/6GXp+u+4MT11BxKLD70t3fW1xnTpdc8F1PbFmJxpdiLd//H3p/cDPHfUgAVu5w99JX3oDfem9h99eboufSG9+TUfS5f83I1p/198OvX/2A+mZ2zCxrGaP+tp/41cnxtY/P7vp9e96Y0xIhaXhw685WeGDsh5cGRVqgCHMiFEgFysCLP0J4+y3BvbNAqwiOmJykW6PjePSCc/8+YMYMGYXAMRxAtgwTkmWeKvOCETx6E81j2rScY8/Hnrc9/7869lLIcrVHJXpp+BRWYmogzBWGRg4T4N/gQWMo8yocYLxpiz1666azOwTnUjGO41rBw3ACpO4QlqAjWo42yGd5gB7iDM7ElWMmdh8uDAWQkGWIDotzStTgOIUjeugRGFVdC2YlaGTWYPZkVmyKzWsRIZZeAm6iWwgJlS1UPJgDtuW36ZJVfRZbBlWvVgJIhUF16VnmE2qVNNRGZHYCEQ0X7DlWCZLdWT7FD2D8lkmkoYfL4OplBGzNVUVVFk3LW/cOM4XW6GNCaIDBMZjF4ul9cCC1eDZZhinwCbgOtW9MD1XuT+D40ACPvoLAz6hKgdWuitSOZQ9Q91y637GKvCelo6heG75ZaR3gEo2NTTk92i0q6uVFsP2137goOoAgkQXFHODKd1tBDWT9eyraEKdenFu8KzVAe0E1jsfmgfRtf7UYU6HnSzL3SjtrQZRls1IYGlQK4ZkOBvJOwiWM0HXDTA5GnQr+vYqVDBoi7Q7DTtJU3cFd0dssPNLioqG5EQuAt4F6orenAaQtJwEFCh1MRyT7kCT1xV2WL1nDjrYXIFNy0Aiy7Ankbg7aQZqmFdG6kekiCkEe56rvqS4MJVeDcKFFgMY8StLc+cxvAw9Rt6ulM79Wu3buS1FiK1IHXQgLsX+nYBPPye7FPaQij10r7kKLuT737gwXQMYDEBUy3gaWaPD6Vm2hU4iRhHMCThtfHRPkZ6yBjbz/zWBEhKCbmN8Ep8pLkqawQLpl8pStCbPuheHY47fivWTeZeSYVSNtvZfkB2Qa/CcNoPTVeAIIhwpBAAK1kJSWb0mpyX31z0P8pqHTKwNr7fSwY10tg2Fvj5DRRPUbpBdo8VpTd2XHWxM3DKCyR+X3EQP675k8dM3pPv0qIHI6h9lcOyzkJ3hpTYZXuxAhanKRuLvrFZ3qLurRmHBBaN9gX601badAA1wR24Nu5mjGilTVm0zPkSr5TFs2UotMrjQAkYRYg/hjnbURWVwtauImhJsz5ueXa2tHx+Rngf1KUbOVT3JVx9muVZjkZowuY4JpzHuKUkl9Wz5MO5vra19ONhfpP/kjZhLwALKfrwYyVVKBlcGd1nX9XGWL5E5UHsLT51/1R68fWd6cl4YiqM7nIAYi5ne6f9wWth/K8aaML+YInp51OOPJ96SXN60TM64rv+Y4DF0Ph8ejoqSdchDelsywx3qcW5gMXZXMMuZ9ZLWjLiruBPIOo9gLTini9PpB99zsZ0aT8rjBxni1filzPDQNiNjJOeZXjfZ8dCLesnUNPyOBtdPgld34F610++YAkcLM/zbHHf8tGR9PmHptNvvWxzKUacf/n1DFx7pAAAI1BJREFUh2mn5lBVq39hPU9CV2077ntwKn0NadQrARaOhY8/YLGYpo59Nt1950fS/cfYbJguOTNPu1/7jPSs6y5Pm1uW+mhKKwEL5s3JY2nPPe9Md312OJ2G91y37dp0dbo3fbXnh9OLb9yU2hjUl76i+pY4/+tzAovffNWr0l+/8Y0x4DmgOvCVH3cxyNYPvoziUYL8LIbgPBDXlSvelcmwem51chzi819GwYm6gI0MLnwneCFnf5RFZoFTxHWlMlw0cnbC97eG8CUNsypDfI5r+XMdfEeSzhVxxPvqnY/inW+sn/+rPGSsi3GlQCNWRmFsNN6V0dEIWkNg62O8WHmt0tW9pky2qkYaRq9lMm/EtmJhLWLa1dlo++D8qnSYDciOyuywkjsP8wP3gQ7cYlJ9dDNGVJe0rkVysTq1rUatZEIGMm9kJgNZiizTJ1ud60U5eCeocOVd5kmGXGNomWxdmHaiGqQEIrw0QWyZrGEY1cETJ2JVVXUgGcwACa78wihZX+tpe8j8K9mw2gGcSFsVnVghh2YeMocCC1dpZeRCCkBc7STKBmeGCTUTlFBNR3rLYNruSoBkDLUz6IEJbm/LjLr10fOQak96FXLFPVZ+oUAjK7EhJUHP/yQen9ayqruROu/ahl0BRsvWW0ZQV6Dh4Yhya0A8CLjYC7A4iuejk6j8yNBGv6OClkMm05XdS7CduAR1KPe2ELzcv/vB9MCDe9OR40OhDiYI2aAtBwy4Ll5pElwA03Yw+b143pGpPIG6075DRwE+MKPQyS7nDtLmt0rbD6QGcwIEfs1sWObGgF3sitwLGOwCXLgaPk79BwGBAothAGnsOwGtpWcAP2aVZvJ3L49mQFA7fbAbOuhytw0pkJIgf2QdYESm+wQqSieRgqhKNhK/vLquy2MdD7hLcwcSCe1IFugbSiw2UK++nq7US/tI22ZsXpTK+f0ITk/hwWxocDDt27c/7dl7ADqfDAY7PD9Fv1JiEfA/CGH/CkkSdbf/OrlAlgC1YeND3QUjSgbse+W7M0yMFc6m/AQfStdUyzMFpRUTAFzBl96ZCnOvmqDfit+N35DA2O/C8tmXzUMJkP3dfhCLGYwDNqx5218tr43oeBYAGaBj/u5b4iEdBDbWx77rz+9BdSjBhd+ZlYzxDMDgOCfAMO8y9nqOcc68qzEqxiMC+dz4jpmUBIkFEwz1cR+LedLxe5KOYcgdNhYsbjRhSE/7r0Ya2IQkq5f+vJ3v7GL6WV8LEkfqrsF8jMuka/rl8LYc0s2xPTpSeci5lNtHEcYyVkckRbxy+N6jPk55J+1Wel7eLz8vpZrfLOW6FLLk55Pl78u7nE5VLspXSzfqUcWKh7U39Rnka4JZtZJHGZ8vAIslUtVfrQQs3vWp0XTvVybTACpNMpv1h03xvKfgkIKJ8myMruHP9k7Vozd8aDhsD9qa7elnHk+BES7uY2V83/fZ8fTZPUr0F9NTULe6+enYl/mRcvxLAAuZ/ztQYfpn1Kw2sJioAbWG1+73cb7A4ot7p9M7P8niGoOAalUaix+Did+MutbXAxaf2T2V3oob4GInYv0eKbD4W+IpdVgOLNyP4irUnm7FdsVDQ/w3YTdzHGP8Lozx16OdMYKx98QMUqTHLbCYSQ9++G/T3395Lm266NI00NuQ9n/x8+lgw/Z07fXXp2sGlqRKKwMLxpsFFrDGj6WDB7FjlLkY+Uq6+6tt6TlXLaRPfWYPzoE2pCu+43vTMy9hrq4zlI9GOc8/5wQW/+v3fm8JWFQD6BmDq98Nz2PyMGPuvfZ7ckgNAOBzjjLElonAqCWe12VC9GyYMlEuSRyqMJwysHAo5trBmfD1wCK7emUSZuI0Pv9z2Cq8E5ITam1KMAzv4qgKWt56W95F2QmbmQYYDcCBBtveB2PhxA/ToD63tgb+YlWTRPReo1/+Ai5WsVooYy4TFMCCyXwtE/ni+rY0GcBiXdo/3xDA4piMLqvW8zAaDTBBa+dnUycSij4kFQN8eL3rWTUHbc6hJlWkFQKLUk+JZJtEPSi/jLEMi7YPMj4yegKLTphUV9Rlkl3Jtr7SSZsHV8FH0Y+X0daFaZEwuCKszrr1JHjUPxgVmC2prpSheHKSLoa3MUIHnfw1LDbtbDzLqi31y25EM7NlHTSGNR1tUlzBluauXrfBGGpw3YsxYffGbtSSlFrkXZVHYeyH0eM/DhhSjUjJzBr0x2XwZBb1+NRAWm2swO7a0Z92YHBsvS2Xq/Qye1ZIgKENxFHAgTYLR3H5KuNnPtJH+xh3J9f168C2/rSL/Rs2VlIDN7YTWBw8Ohh1sn/FjtYw8W0wbOthstez+qtqjipZruILAvYDLARzAiJGA2ha7W8Cs6ekaZEyzc+eglHHNoI2036hm/Q0Bpe2QwCqE/yGaC/dz44rWaDMfuyqQ6k2tg6gF6CC+NpobIKOupptQwqkxyP7tOpk0sH2kZYahQ/jaUoJyCj9QOmC9NJFcgdgSXDhVzlJGHeyVjVLI/Q+AEfnBsAqacvwmv880ooZ3CIPHj/OzuL7w9hcht7vLcYYL7izTwXDzrflWemAAMmzjPv6kDxktaagrZI2JFH2d78FQYNtT3NF/3elXWlBEzSX0bcvKC1wkz+Bbrh9RTVvlJ99XEbfw29VughgsgpZ1pO2f/o9hYE5mfodFIbCemR1KsYH+rD5mU5+Dx1473cq2JWOpuW347cZZYEeGvMLXix8WSQxfoyL5OUz+2E5RyV9RhjBUP4ug22tgAXjELQ47eZ4BLYfS0uzmAMgLmjj1dzKPhb8GI/WQM9OAGsvXsN2IbHQ5WwfQDrUocwnqMMfCcxd7d5Htp9P6h+W8I/gXOYag8bYWxfnjHfV8+gydWHqL+vD+zyX9sx0l4fJdapLxXoQsTZvRIbxIAey09VXFprkIPyto0G5DJIZIy5yX7/1hY89VajPHlpIV/c/nDnPRHtkf1cCFv/4pYkwAv7v2FLodelsx9nAg+HP9u7gidPpT959bhWn5fmpPvUPn5tI93xlIn0XoKZs3vcvASzegd2IGwa+HOnEwKZswPzP+2fS6z84fF7AQm9Mv/u2YxhMw8hj16FxtIdSF8HZowFYvPrtx+PT+3GMyYvXq3djdC9dH7fAYuLL6Z2vuzsN73pGuuGZT0o7wRHjD3wkveuDB9Lqy56WnvusJ6SNZWBZUWJR11vxJnpq7IF095v+MS3ecE2av/PedPqZT0/bT/9z+vgD/emm778u7ejAXX9dlPO9/LrA4q9uv52RlAGyGgUdKp0u4vBUDa5OIN4y18Qk52N/envxiGD5MiaJCMt9bSqqwhE5nsVkWU3SZUKNxE2DsJahyjqYRSdavxEnUoGFK2peyyCZtKt1zJ9Rj2C4mcSjJmVyLGfTrwrryVBRNN8zUceETZpZWpGZnZwP1zAPMsHq7of7VZgI6Sbz7qqmwMLfHM+c9LXFUKrhnhXrYJLXwng1NG1Ik9hXuHfFAYDFUVarT6CqNAeDNA+jMw/zs3Z6Im2Ym0m9OOrsb2pI3ahFtatTbZ2COVEigTTCH4yRzIt1iHKSp55lQmLBe6kjs+8qrLYVqkG5uiwYsN0NW+ogk6nqi0ausZpaMV3huUnakImMk7Q2npnaLgIB6SXoCjechA3aRjlgMKmXwEJm2v0SPAs4ZLjcKVpJkP3CLmiHlbau8su4ysQLCHq6s9G1kiCNUd0ZXDuDI6jZDGIUrEGzTJv2LJZDpnMOe4z1lGvn1k24gN0edgCWS5Uh7Stk9jQClo6u1O89eASm/xhMO1ILymUf1W5CmxGZ551IPfpZ9VeVSNC2B1WoPapCoeIj02wcV71VYRJYdEHnFlaAQ7JF/TRSFrQcA7y4u7mSExlfPXNpc9EK464UrkEmlD08VMkRkAgsNsAA2p+mWXU/hmelE8SfBHyMwuTrulV3vdnGgjpBd50FNGOfocH6pm7UswAGm1FXsv1V95lDVUz7DsugcbEqS+6ULbDQZkNgoV2Mjb6evtsFKNHexD0nxlGdaqaOGqD3sz/IZuwrBBbrAUYa1kc5ABaTSNgGkVjs3XeAXeRpa9rH70t1JPuEdLe/+Au1QmhnP5aRt58pPdAWSEZfexQN+d0s0A3z4JahIWp2gEKZdL9d3UC7SaTx/EYbkYTZn/zZ1+wTAgulaNZP26LYuZ5cwz5CEEM8y2PbW5ZgzOmYjhICH8ceGX8lLUqaHGuoFH1OtcclqZ59OcYh2tJvMcZX0nSs89sp4GKGcoXEzQj8eJ3tSarxrSxsWJY4DOD/GPvyGGiZ7NexuEEaSisEF7qcFf5bxAAXAgvBtwAwgIXggr5Hu6neJ6i4CICxE4DRTl3Ux63lm3PnfulJGV+rV7VxtS5IfNMRYylaBK/NN1Hn/LKAi/KupJvjOz5YyxUOni9/433JUpKtGHXFh1aD2LUEaxckaIpQoEovl4fQBIlXVdFyvlV5qzyczQzzWAQWe0YX00Ubcq1XaJ1H9GglYFFAgZ6RNC4+21HCnY8qlCDht7Cx0EXqD2MYfj7Hr91+JD1xR2bcjaf9wnpUtfRCVX8sX+k/1zsZ/1k+2p+/Bdu36ijSg/ORWBQw8v2oiRWJi+Pua94+yKLJqn93YKFk5tffeCRdg22LBt7lUI1KidDjFlgMfyy9/s8/n1qe9Zz07ddcnHpU/Bj8p3Tn2+9LJzZflZ7z/GvS9poG3kqqUIWSLHbNjKa9H3lL+sD009Mt33Ys/cWrT6Sbf/1F6eKGT6fXvfJr6Uk/e3O6qm9DWrJaKnEf+fmcwOJ//u7vJoGFQ0Jt0ObGobT8ivpQDIwOqryLCYBRMsJEeK/yUa6MF6tsPPYcEUsg7muqUEyKXudJxem7Sr+K4POQGHAuK3lZYuEqN8y2afHLDIDGwZnRd1K3Thko5HAmmetTlZ33tfKWfJywSTeY9HiWr11R97cGZkWGT2mFzLTMqYyDDIrElulVauEsokqE4WLVVS8s2AqsamH/ANShRlfhahaPUEPM/GPEMR1XjmdgvBdHh1PrzHjqwcVsj8bb8FHtlKkFpnkt+VvfsrOvTJUMmrOb9FE1y3paBGlg/nqo0aOQxqwaVwsqZJKsfDA/BC4r9NJDw+eTMK66INX1bCvASOmNbeNB9lGGrOdtn3ClOUuJfClzLXOf8xC85JVdV451UStDKDAI2wsY26IOY3tpuKtb243uu4CajYyw6kVtMPPahMjAqfMv8FFtSTUmpRYnsDUYxYZB24xGV45hEOdR41lNmpuR0ly8czuenTaFtMmyqveeXfHCCMMMC6iUPOzFEHvvgWOsguu1B4aTdDb34hUKULFrez87g2PvgUqJ+zXsO3AIw21UqGC03dPDPTuss9KgDQCLfoycN7JHgEzrBIy7m6PNwIxOwOCOAl6UlNh31ldApBMbBUGVAHoRxn89zF0rdWlX+sDP72QCydYx1KCGod8UccdQrZogLaUwgjTrFn0B+mucLyjpx1OVwKIblaUupD/ukTEL6JoUnMD8a7Niu4xRHjfHC2BBG+mSVU9XTfSddgCe9j2z5CMoU02rC8AnsNiGC1/BlmuWp6BjSJ1oh1HU0QYH8YwC+FsNHe1juq41fYGloErQII39pgRg9k37zobW9tiMUGmZUgm/1DVIdloAnGpFCBKkoUBNSZDhlGQUEMenkPspNLNPh0oTbe6eJ7GnBHS0fjoR8EMIr1JIOfIIl5lFxxb7ZHwbpGO/FnRI3wJY3NU9lzkbifvtaKeU86zGIvqXY0pIOqmjfUTHC6GSJdj2G+AbEeSatpHNW4ms+Rme2ygLheWaG545RoXEkLT9OP3O9MClnUW4naVFXHIQ0Ma4iHrWIlK0VbTnqpa2UIVqoH31Uia43Qxocz+LSzvbkZIyxplv1ITEqyPy9nrpUbypzR/xipflvfUpN9Wz+rARue6P6dfeR9zqpXU+21GFM565eZR2rG7iSTyvS7+WT+1tpn3EiTKbXknRQPbwcthevvPMq1K8uM43liHCVOGs260vfOx5hRpm/aErmwAU4pz3eSVgYSJ/hrcg95h47tVt6ZJNWdKqQfWXWc3/gW/vDLp/I8DCtFVv+tAXxvGK1JKuRlWnGfWj4bH59OUD0+FZyv0XPo1HI6UbT724iXlpdeT7blS0XogqlN6nPP4aV7e7j8ymmzDwbmMRUIPylTw/ReDqz3LQoScn1YhM95It69L92BvoylW3s+cDLE6g8vTqdxyPMtzEvhraqElbjdLdo+LRILFQwuNYpY2MBuKf+NpkuMf1a3rcAouZ3endf3lnOth7bbrhxqemy7pQg/vC3enOjx5PLd96Q/ruGy9PnWWMOZfEgq0JJg5/If3d+w6kq256Qbpy3afSH/zaA+nG33hpuirdk/7s1YfS03/qpvQtvbgbr++Q53l9TmDxGwCL191+2xlJ2rgenst1ARc+dwB1gIxBk+sSpryrDda8cHJ0KJa5LzSJuMb3FxNmBhWqP5l4nEydC8NEGsSPiZZzTLbEk6kvgMNITv6uBCo98CeDZVmclAOQRF6RqKk7HUTZDeN95EeYsKPwbJkj/zyB+9xJPqtMaHOQDaYlcKyGCixgGqMMpGxcw7tngm4oXfVdz4S+prUznV6HOhQG3MMYb09g6896azBHMpwj2AdMDZ9ITRMjaePpyXA5q+vZFgjZAuPkyq6F1duSjInAQiZZumUd8cxghZtWy0BdXLmVKQ9QAeMq3TXwlbHLdLL1Mr01mp12JRxmMxg6mB1XM5u1ESFvnwlkXBXXMNlntoVlyiAL1Q8BVsXk+i6ABsDE95bZzdjCqBzGWGbb58Goca99gYx0Ny5M3ZSuWe81lFnjd1VhZOIsg951Rlyxpv5KQJRYDA0OAbimA6isgwF1ZdtN6nqRfAgMNrPqrnta28a6n6aeSmk0CNejlDYWSix27zsMXbEtoA/YdrqO3YGnpx3s/9BCGewv2rocA9C4Qd4gkpNjx4cCXJi2EgGBxdatW8KblNKXScqnGtSYjCTMrEbtGhPbX1SVU8KhAbS2GQK2KWxe1mPFH8AC4GH/t20mqbuSmmHiu9naJGlOwySr6qPkKvozYQUoSlq2oqq0hQ32egAUG3Av2oYEQxoKADR+nwYgaaeiJy3BkaBvCKnFCAbmgre1gBOBj4ynLmeVrNnZNODeBD13bt8RG+61AlpV3zrFz29RcHASwKXdzmEkOu4+L60nbTNATKiB0efyBo0ZWLiZXbQ1ZVe6pmoSVQkQLTCQibbPRX8jD/MpthGCE783vwMBpVITAY7fv0fu69lFrBKa7AUNiY0e02gDyysAzGOBqQTLGN+0iwahdhR9WbsKvvPqJy2jfwPoo/87HtCPLZvMfAF6Spsso3m4IGLqqgTqHcq+5piVxyLGJeJZDkNZ/hiL6IsEsFjRfzXgDlAR0kLHM/o07zOwAPgQbl7j7Xgu6CC6iw4Ai9WACiUW2lg0SGN+awASfQEsOtLlgIvNGNc3mbcNUHfkO8fG/LCM/5a9HGfGKE/PDH/m06U7vx+P+vSW3i67iiyrfMn/4SUoo7zxLPNSyZanH69MYCmRKrO6NGrAgmcRrorgDQnUUmcyK1kthcvJvfgxCCym6WxNdK1v5nDXbRngX7ylD1sy5pPq0L7hXew18UVsIrRv8FC950pcl8qYehRPTu7J8ITtZ67Dnusdn3368Jcm0kf4KcHwcLjY2buOtDtCTUdQ4T4a2jp4qE4kEHk+UpTS3hqC66Z1grIa/2e+B/VQVgP1kuQK/U+z58XyY/k7jbZv/1Der8OwPe1r0vNwDasq1I/jyekiwIbH8njxcNmfe9nD4n0Yvbs/hOXV85UARWPp/4YnLY+z0eW+PXii+ujJ9NNIX7awF4jH8jzPFvdtGH2HZ6eXbI545c+v3nYkPQngdgvG8B5uYCiQGqXOfieXIDW6DI9eAjaBRUiEaY5fue0we3jgteraevuCkupj7TyVDt/30XTP5w6w0Ow8yFg+NZsa+y5PT7n26vSEfhaCalU+h8QCO4vZ8cPpy4fWp2+5opd9po6nz7zhb9OnG5j7T0Pz9memm7/7iSwi6cb/Gz++DrD4nZrEomSRh8o8SOfprXrjCzqBg7MfVBmk86cer/IfHkRszgEKDE8E176cGyNePPN5ARVMjDnRyCzCEjji8yQARHUfdg9O8NWkZ9rGDYaekcLV7LI7ryXJ0o0KHBAnyk8ky12mDGtkXgGAzKeEs57mwy+kFeQrYy9zIENtuWSyQ2KBnUIwG5RB9TDzMUzsWg3TtV4GGQZtbVtHWmhqTzNr29IYm+PNsoo4r3oPM9AITNsgwGL4xGBaO34ydZ8aT32Ni6mdjfKaQHfryN9VSlgQVnxlirJRtEyUZdQrjV6fsrtLjDNJNx8yZag2WG6ADpWKssaKvVadHNZfGio1UV3J9PPKcwOrx6xYU3bdvrrS7Oq+eZpfGKparoqxkllSEpF3csbIFxroJcpVfFcpVNmx3LaZfvYXyDfUyKCj0goSDiDTg5pNN8yrdLex7Csy4DJy06w6l70WZuCYVPeQmXOfhGEkGO7CvRbVrw2odnQr+SAtVZl6WGGXIbasMnyCGOsyQ10n8TKlnYWb3e3edyiYdogaak3bt21BnWoLO7t2B1iRYRTAnEQKcQIpicDiEPFUI6I6QWM9SW1hJX8HcbVtcS+IExgvax8xhgqO0golOPZDgYgSmT5cwHYBpmScR1AhclJo1tYEiZN9SwbX/Tk0tNeTlwbAusc1baUJwcRWbWk/UAVqYPuW2HVcyU+TxroAXJnoUBUjPc9KpkaQgrinhe56T7CvhruQm34jwGoNbSczPMOzBdsd2rrBXn9/f7poYADjbdSg6Atu9hduiqmV340G4ceOD4bEQluCaYCKGwaGCpzMM2GUnmgPoYF8duNsn9JrFJIJyhn1BkBZN7+3/E3pqcx9KtiMjr7ltZJEiRn7r9A2UT+kbX6HoT5Iv/KwT/vNqvoWKnkVUx/fK3XIan1889DesUQ7DPs8PTC+Mb8TAYTlEuyoHqn3L+ui5K0sbMjkCxBkYCk2fSKPG34z0pJH0e9tU3ecV40puNUqXvR5C8wRwIIIjlPxn5M2X9IrnEdwbz+S73KjRsGEm+TNM7aUHbijPNw3VBILwUUAC+4X+KaUKPXQZy9hszyBxVbo3yx4IZ3Il/QdL8nKjzGf45kPLLol8CJe52v+Gr688mEVKr/wwbLDdihHLc3yoP5cS4iLKoPao1I6aRkFMuJSuvXJlDeRrUn5gD+lGKUM8bwEMkitUkbyLTQhi8glB843UQYTzcdjEVgoHatU+Us1z/ssyQQOGhqvdPApJXe/9ltqZzdqz/WHjLP7RKx0nOtdCS9jbzh1/h13lx++l1HvOMt7y6d9QxO725adu53vPGSUlx9ne6cBs+OGG+t5uLldPU3OFm95+tJSxt3yBqNOUeyz9WU5G12WP18pz+VhzN/qOjcup59lCR6ujgx+MkEvJDxlx23D1cf13vKWb3F5HR9z96fZU2r3/WnfYfaAYrpqbO1J2wd2sTiI2tIZKADe48g+nKJMp/bLn5x2LWmUPZwkEPrU4c+nT371GHtKNactV16dLuuDD12hjz888tmffH1gcdttueHseKQTDL7n6lfXF2K0deCPH+99V001ca2RmvHi4KXTj99UBghLzHstjCGYkC2A+bofgWkbPowX41zFJ6F4ziQeKgDEK2Ux1zltG5joZVIFFjITli+YFcKG2o4TOtemk+sXhYyKWFaPnGY+W664Nz4MsmkFsHCVkmtekx+r8zBNngPQVDQIIEIc1R9ceZUp1qPOunZ6QcuGNKutxeJaNq4SWKzBHSSbiME0HoXhHBwSWIykbtSh+hoXUite6NexcZolDN1u6unKsKov1tOJzfoJYjpY+c4bqbGRG+X2CFUjwsQKJ+VxjwhBhGpOpmG61l9JgKvX6qAbRyZI9ZK1rCS7Am76WdKR9dazpCHT29ViD8GBwGKsMo6VXqpiqaqidEK9dsGFm8BZjkXKOAsD56Ang9bcBIOD2o3Mqh6M3CzN/RXCfoHyGGeW+yGYeZn6aaRE4bqV9pii7ENuGMdKuQNcJ6BiM6v1O5E2KP3QXqMVFRvVqewASixMW2mFK/YHMKreu/9w2sv+FKrY2EeaAFQ7tvWH4fYOGH8HOttbZtzVfaUH7iVxHFAyzr0Ja+fRSjt0Y3uwhVV9Xb363E3wlAaobiTjK3NqJ5LOgsE+wm4EAAmeRoeHQpJiT22G/vY1PWqFdAb6Bo05n6Kcp+k3RYpkQNu5DxWo7bjZvRQvVtopuEmeHpuK1yalXGWFPdzCql6F7YaqZQILJUIT7EJOx4oyQlDUs7QhQcWK9DahAuUu5tsxiu9UogFNNTw3f79H+9dh9sc4fORIAAuZc70yTQNag8mFuPY5AYQbynXikSi+S9o72hoQ7LcX/ZRvO/cPgAV1M45ASbU0d7q2b5JUTd1J0Cd9bVvLotqRQDh/P1mNSa9eSk7sjwIIy+03rdvkWDgAqAiSNWAfn5iO/O0PfmsxxkAL45m+z4kef3gdYX0TL8hV18ExHpBmjEOA5DLBB8BlAvWc45Aefdd0/GOy/snnPAY6STsGWt48DlU0J7z2XQHY+Zbm+VY04pY58XuYJzyd8wxVKDpEbJy3inp3009VhbpCYIGUrYW2DmARlbMgS4flsYi1Iwpc3fmiFJxQReIdVYogPCtpcl5i0o0XEaPuVWpnnM4Ia/AoxFJJzritMi5z1BkJGTcXMh5LwTOPXMP6/HI6VTjTri7rwxirqkG+KGGqxB+LwMIq1up8JhEv3F2gwAUKPAYp8AiAxRtiUHCgd4CQuc9zRJ4wpUkMwDFicsV7GYAzBpJq8GxwQqyFy4ONIZ1EI47xTD/+mbKMfvUsnhOHswyx4Y0nsJI99j7SiHe852xYC+LALqgIRhfmRmbEPAgRuwNr1ClD4mQcaZNuqaulIOE4+bAAq5y26efyOIHLJMtACRZMy7wLsAhVHuqfc5YfY0URhnEtE7eMiiv26quvRR0ltaKv3tSRpnDaObuqMZ2GCZii/EMwiYMAi/GTQ2kdwKJreix1NbAfA16iGudQQaGA1i109WGcBBXSSCmOKiItpC+wUH1IN7eu7hZVDdsUfibqX6QLMugaMgcxSF3DVr0tqaZjuuHPHzDkCqvMr/UvpJIRKqu56qmrKqY9h+FsC5/5XobOcgjMlCrIyGkUbFk0xJa+1gfScc9GfJRbANCDF6ONqEPZbtZTBjq3GSuzMtjQSQPcMZg+pT3zNJyqNmOsumtr4eq4khJtDAawjdgGk93XoxvYNsqi4bogSOZa41kZyClsK9iFG7eoGnHHHgPUwzbUYHsABvoSdpwWXJm/qmKCm2OoTx3CNkPDZ+1e/HjcTE3JQAe2HX3sHeFeHDJo0lUPVtZf5jm8AVEQwZvG2ZavmXiuvlu2mVN4jlIKwE/9e/t30FWGl7zcQVppxRw6/YIUv4P4bqCZ6k8Cqksv2kE7Kg0QnNCHKb+r3QJH7Q00QjfdoCe0G8RWRWnQGKBFFR1thmxP1azsSxqEd0DTbiVAqFdFO0HTNlSl7B/2fw2z3bNjL25mj+IV6iSugQMQU2dBkH1YyYDfhAbgtncnbe3ChEBXGwXBn9+goEB1P9MUTFsGjeM17BeM2V+sWwEP0s1vMuhBaaTHwqIgIC8AxHfKtc4ElKppZyGAkJkvdhD2YQGU5bQsAuGQqhFPo2u9qNl+WX0p4EAeP6q/Mb7ZWTlq4xkfTqhZCir4VgJgOIYRxnYr6psZdJFCxJeVrcamSCyPgxlUkAbxvTZULMxwFdFoXzJBFcoFi7zqKXZagE7aWIAMMdpGJZBxAkIzWCFFoQ9uBFhcjMTiCQCL7Uo+BULQzzpEOS1DOXIBy13tXHtMhLqS18aYpYscoCJTLX65KMx6jMPlIefyvO5RzoeMH5aWz6IQZ5QkouYnFXVrheZVXficngs35lvlyPs8e1Q18YX51BXIJJaX29eGeSwCC+t24bhAgQsUePxQ4GzA4v8DAAD//zOYwgsAAEAASURBVOy9yZNlW3antb3ve/foI15EvCZTZZRhZTBiSg2KEQiVQKoBAwbUhD+gRlhJ1VBKNTUFMySVUCqLAWbADEZlGBNkmIEZBlIqM1//onP38PC+b/i+3z773hsvX76UyZCs3tM97ueec/bZ7drNWb+91tp75IbjH/7Gf1l+9w//sIwUjpH8lhuuNz7zc8PfiH/1VRkZHY3fERxG9cPR/Obmhic9+8/VYKNcx0aNo54tTMIZN+9qmHpN3LiNkVbC4mxa3hvfyAh5MB88xwE3ilIurq7K5dV1ueLqs+8MMzk6VibGRjnHypin8XJ63BihB/7Mj+Wt1+pW86X/kTJO2KnxsTI9PlEmx8cTR9K9vCwXnKZ7dX1Tro2DePU/OTFRpiYmy9TUZJnwnJ4pYwuLpcwulfOZpXJUpsrp6EQ5p0z7xLF7elp2jo7Lzf6bMn24WxaO98rC1WmZujwr45fnZOo65Ty9uCynp2cWvEyQpynSmZ2eKvOzM2Vxfr5MT02VCdKdwW12ZobrTOgsWW5urssoebu4vChnZ2fljDQttOU+ODgsu/v75eDoKDSr4aetpaQzMTEWmkpXj7Pz83J8clZOzy5Cb/3PkYdx6AMpyjn5PDk5KVeXVzxTR1xPyPfp2Xm5vpZSlAiP55T9nHdXOJjvlaXFsrG2Wu7d3ihz5N86OyPc5TXxEO6SvJ+enZaj42Pye1iOTs/L6bn1YPyn5Zj8H+4fJNziwnx5+s6DnA/u3i7z0GecOrRury4uyBfthvRPoMUnnz8rH378afnws+fl4PC4XPDekpqXdx7cK7/w/rtlifik1pu9vfJya7u82Nwqz15slUPq7bwrl+1zElosLC2VdcqwvLKY9ndyfAI9ztNOQzDiuaHNTk2MU2+z5c7GepmcnKCd0uY4L0n/5OSo7O/sQmvq6pw6043rEfQwj2cnp+WSZ9uifUJajdMmpN3jh3fL+0/eKVOTk3G3YU54z3ur8PDgoBxDw9CfurQMr3felJ3d/bJ3eNSr20v7FJSYmZ8rC8uLZWVtpSwvLZTl+YWyNDdXNlZ4Xlws89TVOfnb3d0rL569KJ9+/kV5/eYN8Z+X0fHan20YtoUJ2sjsLGW+tVHWiG9xcSHlvaTNXFKP5jP9lPZqPR+Qn33q1XxbngXSnYbG3k+OTZQb2sV12of9/5o6ShXj/wZ6X5XztNXTtPlz6GWftd0e015OaMNXhG9jg/19lD5p5V+l3V4T3jZ3nnZiPZjHtGHe2w97Y4n3ST05qOOWmfafq+PXOHVrH7HvjpOWf8ZlHtopjYyHYL3DZ/ue4SeoZ+lT+2KNP2ngNkL/9rwaoZ8T6IyOdUkGr233jAU3swvlZmaujExNk++xQoMr41MTZY26fXdtufzC6lJ5Z4G6niSPybM55OA+Y6tl5vip/HXutbDx8lb+K2FqWP3kritgK2fc+HmLiqQ7eCQPgw7c6+PL7jUufrvw0r/50X9K1cvzQIT6y6O/5CSX6uJzy01zqR58o/faD73NN8qb7jDt//gXf6k9/qVf/Sa1PLTrYKKNHrY9z3wPGAfX1tYGvQ3vhxQYUmBIgbcoIM/keDHHd3jwGGGQ64DFf4s7Q2UbTLtBuHnO4JvX3PHOQVW2nO91jjou14fcG75GF/9+oP0A1mhrHPFtRN0QbayJ248sZxgrAox58uyInTQTrwETOAO5HyA/wpd+lAUWMBe+Nz7DBlSQ/jgAQ2bfU4Dh0T5e5qed7YMegNXlx7gEJlNj4wALPrZcdZMJkMmVcQmjkRiJSf/4C6iA+ZmE0R/nOspHvfBRv55eKBdTC2UfYHHMh/2E9A9hVo5glk9Pjsvk8X6ZOdrLOX1xWiavABZXl4ldxvIYplJGSbrJRE7CmE4DXGY4BRWegokVmDWBxuzMbMCEDFeAFzHJNJ/CVHn6wZPB3z88hIE7hKE8DfMigLIcvp+EyZCJa5SyWmTQpUPqQ4YpeZkIUzg5OQWdxynPSTmCITwWYFhHXT0JAmQia3lOyyFpyuiNEqdMpsz8w7t3y/LCQpkmXYFjzecV8QEmBA+exHsIg314DNAAYKQuBDS8u6Lxz0KTdx8/KB+8+7g8vHenTE1Pl3HAiwzedeKseRJoffH8Rfnwk8/Ljz78rOzCdAuCZFgFJ/fu3Cq/8MG75S7Mv/QVWLwAWDzbrODicP+IssKgWk80AZn3GRhngcUKjJr1c3ZaGVLpYj7GoK3g4Yq6nIWxu3/vLgBxkrY6SrpX5HM88e3v7pZd0tsH+AVQAAAsr/UkmBG4SRuZ1YBZwMntdeh371Z5/Og+DPgMgA3AQh/IGeAyUvaI9wAgKbiQFT+D/gLL7Te7lO+AtnCC2znM52iZoizzK8tlFgA0TfwLs9OAitmyNL9Ybm9slFXAhcz+3t5+2Xq1Vb747AvyewAYOi3XozJbgm3igfb2P8smiNyAngvUseB7HzB4QT9wzFhYFCBTV/h1ELP97Nk2yesF9JLI1oPAs4LF8bQ3AZlgQvpdU6fn9ina+BH96pCwgqcjaCfozUQE9WuatmuBhf3A9AQe0kQ20voUfAmkbRMXF04iVGbMl20cSSEJkwAUOKOUAwtH+olunJZfUCGItI9Z34Zx7DFe++YF10vykrgSA/eEFUhkooMwue/iNF7dBbWjxCmwuIaO5xTinHigarmZmimjs3PlZo4xaBKAQV+7FhERZpw+trqyVJ6uLZXv0F6fLs2XJYEF71KO5CE9nSx1JeZa+399qbuHv829H1ZmfbA87Y0UNkx9NlyNp0dVo0zZ+yEkS42rpVM98TuQh55bzVEeDWcY40p8Nct9r9w1kNh7lTjJT5emafxUul0MLX4fv+zHd0Ng0RFqeBlSYEiBbywF/hzA4g8zaNdBtH74WmkZf3NkEPajlb/urQEYKPuDL484ZTAlgADB+zqr1n2cjJD/hEnkPvhJqX4FAjIGMiDc9j6cpshjAib+Ll1TDxDg2Y/wNYxzBnZ8D36AG6BoUgufEyEZMQ7DVCaCGfQubjJfbmomElcDFlN8tMf4YOMtTMAljKTMemYvzSIZteyTME1h8gUWzrpzjsDgXUzNlZPx2XI4Nlv2ykQ5ICPHMBHOgpaLszJ6flLmTgAWgouTgzKO28T1RRmDUQqDBWNzBEPZZqMy6wmT0psBpWwy4gswghvOLCMhcWZ4dFSAILCQoXJWGObp4jyMo8yUhyApM+MgUcuTirKchGsz4XU2tzIxlYmtwEnG1TxY39IQIkDCyrw723vOjLtxCvxqoyRt3E9Jy/IcHMEow9iOE88is+GrMLHOhC/BBM3AhNsuzL9hlYLIaMosKl3Zg+HePzjiHgY5zBj1CMiUsVQa8PjBnfLB08flEVKHSWgzAmNvAxA0WPeel9eX5eWrzfIxDPGfffhJ2YGxlnGX6VTqc2t9rXz3/Sflnfv3AjRMd5uZ+Zev3yC12C47XA9kjGEKTds24Kz7xu31skrYeWb7q3ThKqBieXWVyePpck59HjCrD9IvD8jfCrPGMptnlM9+IIN8fAxoOaW8MMYy19s7ewAAGH8YZPMXYEFZpmC05+dmkB4sIP1Yg+FfQyKgFARga93Qpqu0j9JzfwTjf0R8R8QvMJdpfkOZXiOxsPz7gCWlSGOUY4Y4Bci2tzHqYZkZ7VUkS2vU0Z3bd8oq5ZHBlwaCi13AieD3nPZ7xqlkyH43Sx9QVmUbsa7mBFi0E8tgmQTolmODvC+RpnVvXpUGnlOf1r1g1VOGPOACQFPbn20QqRp0o1KpByRhAAulPoJQQbNgZ4+2Iriw/bXJCMcXaWObEHT4TjBruv7pz7xV6WQFFa3t+N7xImOUHWngLp3I13Gu41zGOOpDYKG0cZI8e9g/BDumoXTuCuInbsIbj3TwDIDwimueiT39kjanNFJQAXopV/i4oBsrtbhUMgGoUGI6Mr9UriemkGjwnvQEjrbV1dXl8t76MsACyQXAYpG8TTgGmk5+ux9omywNurX77p35+rlH82tk+m9BuvIa/men08XewnSP1vvPDNP8DAb9Kc/VwYmlHF6Nk9MrDaReefnnKWPzk/CEGQKLStbh75ACQwp8cylQebifKbH4Z1GF6r56tZQMpA6pdUCU6a/3Gfcz2NYZpTrQ1iCOt+1IOOPgg+QZiYXPCcuY7DhtrEbsD2G9lelRUqFqQAUWhqlpJ+5eGg7u/DPAVx5SBgWGNe+rJ2GM6QlUZIgzQ8rHW0mDzI3p4INoOwbHD0eeOmDR8torQ1WpkuFTauFrkgywCKhIXmraltn4ZcKmYYqmYUoDLGCoRpAcHI3NlL2R6bJdJstOGSt7xHMEAwRHWiZhwOaQUCye7pe504Mye3YEoKigYgQmR7UlmXBP6R+aUS7LJLMShoNyzMAkLMPIqlqjisoM6TpLbhiBhcxeZkVhYGT6a1zwIoICGJq8A3Q4M9sYqV65BBAdDSMpoVxKRJwVdxZaujvLfZYZYUTx/Bm/DJv0runLIKqWggoTwMLZ9/0jwALPAoslGNbV5aWAImfBlb6YlsyoeZNhPVEVCuZy7/AAZnifGfb9zHjLCNrmxinvKDPXSiwewty/+/hReYBEwDqhYZIX25D1KPMI2IEu29uvy+dILX788Weo8BAfQEWaCxLXV1ci9Xj66AHM9DKqVxdlj/dbpP0MQGLYPZhy/Vc1GVo5bWEdxn6Dcxmm7QzwdAVN5+bmkWSgloUk5Bz/Wy9elguY+g2kG7c5ZwEcp6j+wBkHWFygCifdVP/aQ6IgkNlCPWoPicIFdL7uJBZzhBOY3CW9W0gszPMy6ljSwnqxz6TxpuwwsNBeBv0YgHYKTQVmO6pC7aMSB1jbOzimY9JfbMu0JzvcCPUoyF1ndluwdffO7QCLZQCGEppj6lEJgQC1gsBj6mWPoNdh+meQQghiImHkOkldXFOGE9J+A9N/SUe2jqTDGm1ggXRtu2H6ybfSCkWwJ+TZWhTc6l8JUZXI0Lehu0cYddSwrJNjpFwCC+mnNGYPEBNwIf3IRx0BUBcin9af9atal/m8yUBTgZAShbRnGw9H7gndxrc4ZmzJ2/poeA/qQH+ZbKEPTSGlquqSlI8/064qnbbHpmpFuASvaVi0flp1FNMt4xz9d0SQQtyCBdWfLgl7xVU1qBFU18YWlwEXS+VqfKpckqaSQ5noqELRXt4HWPwNJBaPAY7z9LmoallW855C1DJ3t195kSYe/Xx+pbfQLm8SvQXr/HXhfaoxde7tMuhoGPzrlPS6++b1664tubcSMbwvKG8Or4mTFNIO6EM8t+PnlbFfJstyU37lF/9uC/qXfm2TTyb0VfnUzbqy/XsOVaH+0qtkmMCQAt8KCvxcYPHfaGMx8NFwHO09+8DhbwamDESOqwyR3eDartVf/XDiOYyAH/g2u9bi5PM4MI538eufUyDgLOTEWBdPl3j15QOHaTPAB1jw2GY+HbT9l9mo0goAhXHKBMt8czaJhTOkFsrPQwZWB1fj4vTIZ4P3ybNxcE7AYGljIbjQ3dnWqPU4KHe0yAee9GSOp2DyVHeZAliMo4IwDnM8Oj1bDkany6ubifLF1XjZ5gu2y4zhEQzMGAzQDAz/AhKLpZO9snx+VBaxr5hUjUTmF4bqGObLWXSZHj8EoZnpka+om0lv8j+HlGQ1s9brqEKhShTGS+ZFZgtVIhlJmFRVX1RHkpbmfQJGQnr4QVLC4KxtVEZgPsbRkVedx9l7VaL06yz4FLPhUzCLzoorDTAfggolCqqtOBPtYTswn878Rl89zKe2Eer2AxAsG6pMhhcMqQq1sb4eHT7TVZVNetugzwE9hhNYCCi2X+8wi/8m6kIX2FpYgeZlfh51MKQf9zdWysM7d8rtW7dRs1qEV6ac5Emwo9TmHHUdJQLaBrwCIHz27GV5hQRCBvSIfDkbvra8XN5/+qi88/B+2YABg3sLEHpN+p8BDHZUK4JxVSoh81sB0CXSCsqBGtU6tgRX0HuERrbArLHPAgvpL7A4JB7byx3sQJYBVjeU8+LshPeoItEyL6gP6+yYfD5/uVlebu/AiCNpsP5gxq13VbY20JHXlsQ8CirmkCIEMIbuqHbZCGzvSGjsR7WuzzOTbxm2KL8g74ByRLXM/kp/GZX+MPQjxDOFrc0dGP+H9++iavUQ6cKtlGkMOsnEW3aB0CHMvJKLLexQaKwBpFWdjnvaj33E9mSFCXD2ABa2uVFm1wWWS6hIzQNWHEds76qNtfFGN9X0VMezLXpYn0qpTLv2zdruLOs5eVfyo8Ril3p9/WYv4OKAsp7T3sPIUzb7csLiJpOv9CmAjAbTxookZnr2e8rlkYt+OvfqmleRRnV3Dk7ptwKlqEJBM9UMBcNpj/TPgHnKYj6ayimhqOMKLlKFNUJTI6xjZ51AEVjccC+kJ+sBFdpakEgZpd2NL61w7UssIpGxbyI9uQVY/EAbC64PkHzNEhdDcfJrcknXMn/pcBzyaHXTrs1be9+uze+gP8vn/6BbC6/bl8PGvx5qpuI1t12gmsvBvPr27SNxtvLgtecbr/FtuXifb4tB9dudb+WzVT5eWjnivffTbkYAFkMbC6kxPIYUGFLgm0uBnwssfvf7AIs6jNaPI2V1wHVgbTN/dbB0kM1/Bv+Mrw65uDWooL+ENTwfNz+ifvSaW00mQ3ZHUe6TUPUnAx+JRffx9V1vsCdEhvjMGhGMhAMquGpgqb9ExbXZZgggGrDQ8NiZW4GLKjoehjFk4uG+AQvfeZjv+tE2XxpMVt1o3Uwt+SGSfGT06wcaZkemVnWOCRj8CZjuUc4x7RwCLGbK1rXAYrS8hCHcgZE6hOmZgCGauURagZRCYLGGXcXKyBXAAh1pGUqY1lNn+fF/BtNpnpM/0pSpyCwoeVBVZYXZxtuopmgYuwCwMC/OuptvZ+b3mEGWwVK1xgo0rFIiGTUpIpN1RpoatUannPSMV2PZmRkMwmGAe2peAIoJ1Costzrzzi6btxhRY6wsCOh9hHF3FlhgoX2D4EwmStWUHSUOzNIr7dEuZG15hZn+qrsvA66URP8ylzLYiR8m+wCGVJuALRjt11uvsbc4inqQ9T4Hc7TKDP5DGPtHMMH3mF1fgrGapD5GqEtBVkAFQOv4CNUiaLJDXM+RCGiQvbn9BhqdBFgoCdAI3Hg2oK1ldTb4EBq92tkJUyw9rwEVzthrbLyNVEF1pxX8331wHxsE7F0AmDPUxzzAT2mWnP4BDL0qSYKwJRhqbUy0Y7iknJcAiisY4wMkM/sHGlUfkq/XGFkDZGCKBWrSTJWhNRhCAcVtVOBWMttPPJaV5iqAuCC+dJLUcgWZAXkdUx81KACaIC82K0iQVJWR+aRZwLRfph+obqVa2WOkN+8+flxWVtZqWUjnEEmHxvNKQax7DdZVjzJhZ+dVy5vQtoA+ImNtuxGIKEnQBoImEsmHkiolN6oKeTRg4X3aK+3RsLY361q3jAm0b+s0Ug2AqrYzDoKqRVWpTzV6F4gKHG1z2k84c69Njsy8aWXiIG20L6Egazl67dmnDAWOfBySiQI0f21c1EEf9lcrw4s/AiJBq9I4+6ZhVYGy/6l+KPBLuCTD+NKLOUn1+pXxpf9LA/sJDkCSvrQCYDFKm1NSMU6/GsEu5pp+e4165DX9YIT2Y13cW15ADWq5fJfFBjbo69O4CyxyEGe7fav8yVv10sr9U+/N4Fccg/5Cm85P4hmgo85fjsGwPTfj57kd/RpoLj2fAzTr6kMvptUFb2HNT0K1F81freQWcf/aJWG4fk6M2/++y6/8B99+YHHOeGU7Hh5/tRT441d/9leb4DC1UGBxYqY8nr1V5pACf1OONgn8F83v1wKLX/sNVKEAFvns9cfe7uPnwOutA6wvObtBUsYx463X5KwO8vGpfz9wXnNfr4nMWHDrH0kgr3T3I9akFvHFT42/MvGmm4S5OJPnx0U3PsG5r2mKZ6qUQTAg0yFTOg6wmOBD6nPLg3EbNuXJfUsvpSWemt8KVFTTkgFX5UIbkFpOy2JejVP95gYsoo/uLCGnq6+MAixGpufLMapQr7GteH45Ur5gdn0bFZh9VIEmrs7LPGpQCwCLlVOAxc1FWWGiUWmFjJpgwNlpmY6oZ5gwR8ragQvLpurPLZiD+7dvlVtr65WRgwmWqTOP2lHsoNP/BmZWcNEAhcBLiY40kQmVcZdRPiZvghHLVSUW0zGYdfWnWXS2BS3j49otQBeBG/7kQ2XslFpU2wpjdXZcaQXMHsAi9IKO0t6VnV7D0KueIjiT8V4QXLA6yTqGwStIC2RGjVhpgGowkYjAKB/DlO5ocAyjuP2K2XYAimo1pjVFGPX0H92/U94VFMAML8NYTbMizhh5Vu8/khl08I9g3A8OsQ1AarG5tROpxYtX2zEqF5AazzsP76JOdSfAQimNkg9V8PahlUyxdhHXMNPq8wsqPn/+Ku1yHnB378GDch/7jPVVmHCZaqUwqqdQJ9IjQI447PDSVhUgV626hElWdW13l/JRb1uAGAHAPjRzVSiZYGmzhCTmzq016ny1rCGlme9AjO+cCZdpVuoRCSJ05yZ1oaRAycIhgEX6q1bmilDavGhQr6qVtDQOG7ptYBGJyrvvPARUvFOePn4MgFtI2w9AsF0JEogzqnFpT4SFTgJXgYVl9J2ASImAki2lJYJmjYiXAHEC10noY19zlBGIGr/Mtky0TLlSFOlnxgIuuJel1mD8hDiVhAlwBKNmwH5s/3ElqB0kFqp7ZWUt3ltOB8sGLExH4CnAqLZbRGFOKIetuXeYPEfcupeDDLNvkkMmM+yrISIX810lqYILaJF3plklFXWsS9QJx4jKtT4bfyZTzItpmgbh62QO9OI+EgveRA2qAxbjAIsJ2j+ottwALK6g742TAdYLalmPaDcabn/AdYl2M0k8SbNLt9K6pVnz0ohh9htd3i5/LWvnu3f5sh9f9OLvfA366bLQC++N7+Nu4tWBPEgUHnDqO/dDtzhDr76Hrl5bCRI8UQ6ETLw1fN9fPPljXEmzAsSee+c1+cLxrwOw+M//t/+qfP/H/2uPBMObIQW+zRT4t5eflH/wnX+/vD9/5xtTzLssjON3+C96/BxgwXKz3/9+BkU/yTm6AdL7wcE37xnI/Y6FEe8+aAPjeD6ASgO+Cljko9GSqJH3PiTdY579wMr8tI+o7/x8ODD7sc/hxfSTB9937t3AbloBKOSlBwY6pr/N7lvcGo2MAydRXuvWi6r7WONe4wOkwNC8FR/PjUaZMYRRmohqRl0ZSdWQET7QAouRmXmAxUI5nZgt+yMTZQuJxWfMdr+AOdyBkZtAFWrh4qQsXxyVNdSglgEWs+TqBPUNmT4ZV2GQR81irTHpJSDQAFRGepnZ/XvM9N9ndl5phTPr1on5U6VDYCGocJUh451y5RcYi546FfHJ3LjqksBCGwiJYtmNS4Z3ntnkOVYEmpudR32HFYdgUmTyUmn4VQoRA2tmq1U18mhMoKDFUyZLEOLXWEb8DUyts8embX2a3hzARaNgVxyS2Y6qCLPoJ0gqEj/A4ghmeRcgIZO4v4chMtIPgdgRzzeX18zkY2MBGPiFD56Ud1l6daUDFuPkGcLAcMNUEocGvnt7u9gY7JRXqO6oDvVi83WkAtaxM+gPACgP7t2O7YL1PKmKG/HcQH9n57WXOKe+XkNfVZW+eLkV0CCoc1b/6dOn5SEAYw2pQq0PSSujDM25CrpuoI0rmWUZN5fCjQTkKPW1/ZoVqF6+wmi7Lr+6t3sQCZAARRuORw/uxr5ilTYwhRTJdAUWAhfjuQCgTEGPMfzDz0c1qIKKg0hZVPsSXOxjW6EUScAXexHajY1OMDCLtGIFGxNtVp6+84glbR+R/yp1sD3tUnaNwS2Ly8jaBlXFy2ptxAEpu/Zby66KklImDb5HaA/aaWhjYz8TqHtKmyyPDAi3HESRfqZal21JQMKIEaAygcre5ZVATQBKfVC3SitGKYPhBBaq0alSeAh4UiJ1gt9INWx7pKW0IAw+fr0KLuxzvMoYVO87N940ZlV/7chbA3DYU23vARYSQDcutoExaGo/sLy+UepSY+7TyXCxH+nCSo8GgAR9rc/UCCpoVGqhcXZsLKDNKO20AYvRJrGgP98wNgWAI4l8IrBAWqF9xSx0HyeOzL+TMftAMm3mu3J5WyvTcpp7H/kbeN/Gx7zsfgbfe5+4Bz0M+uvKHD8D8X6F9y7dfh20eN8K1s9qTbd7KQ1ztPxYXB360eVhMO8t/oTrfmyTLYjhB/374q+DKtQQWAy2iOH9t50CQ2DRr2G+Xzc3v47E4vf+qAGL+rKuhlFH1bCuGV1552jJoOs47GBZB0wfDFf9qyfsYKsKVa4+ByT4vouzfSgM1n2MXB4yT4blzKxq9d58JF1//Ktp9q/mM9H6k6TqR7wHBPyA+6HkzMwmzx5JlTgjtfCZsPm84OYHNSBJj+Yp+arAQkZcZt74BFG9dzII3amKwyhMw4jgAgZUYFFmFsr5xFw5HMFwm8Q+gbF5BvO2DVM1zizrHMBi6fKkrF0eAzLcu4JZf2Z+Vf1RhSh56mjbaGfak+RjGoCg8a4rKd1lll/D7RhUw0RWJhZmBCZaBljGz1li94YQUDhzLM09ZFSUNqhepFRAhnCOWWqZD2eRlSTMwfypxmP8MtcTk6h6MXsvnfxAy8yFuYfBNy7pKFMqMHGmVZUUZ6qTLmFkArMUaIAITDAMnYeMseDCFZVM0xls8ykTmCVEKcMu9NkBRAhMDqDlKeovGg672pL7O8iw3cHO4W8ALFxydhWJQZU2uJdFba+W8QywsreH4fLO67K1VZeQfbWFlIAZfGezlTLcvrMRVajb68RBWVRzG+ecJG/OqCuxOCE/m0hPXmH3scX1EmNtmUIN6B/cv18ewYg/wi5BekoX+1EF4pVJvkElx5pQ794VoayzM2wtZNg3tzbLsy++KCfWIeUTWKiqZvtfgBmMuhe2D+uADLuUfcl2KCAXVArMVHmx3YexBghod6AhfCRMg8BCZt9VoQiTPCau8dTFGnr4SiweP3yYMtlxBJOq6u0CzpQoSbNVAJR1JwjzvTQiE2ljPkcaYhlpDwIYgbh0UhXMejZd3d0Hw3IKJgVvqoxJOwll/urSsY4HlUm9wL9txLDjiEG1jZqZBPiStkvGthWholZI2zN8ZSxDNJtewEyTEFYmvjLzda8cykBa5k9amm4mJwJA4qJrPYySu4yHnVMdG3Wj36nSZd+gD7sCU5Vc9ILWcNA+4yJXY+8BC0EFNI0tSPLiW1Kjzd/Q5i45hVNYUAHakBJhszKBKuAEkgs6b6QVN1yXAO23WEbYlaAeLcyW26qgmTeiIsl6mIfu9qcuKXJHi596+Rd3kL4tA4NpS4scvudIyl0e6nOcu5/BkIPuXdSESz0aVxefIVoSgyFqarh0/vD1lj/zET+DgfEbN67G+9fBeHsILAZbzfD+206BIbDo13CAxT/63m+U3xdYcGT45SfzevmIxCUvMsDjJ9du/G0GjQmcAbZ+/PLR5IuUDyEfuDwbXzfY5tpFnYE5EXRDdvzhyjXgwnz1BnEz4H/1myjyTqa/hiGg/z7UOPg4VpuNt4FFy0uSTrQ1VkGFd6ZQo8lvfcDFPMmoasAtwyYzkFlH3LxGpQEGW3AhgNGQcgSmYZQlZiOxmF0MsDganSxvEI98jKrGF+h5byE9GIOxnTs/DrBYBmDMoBIFlwqjzKw/zKW0JwlmcGtZkikyKqvhDLcqUItIEW6jPhR1GGbmNaw2L+bNmV1nclUnEVyoUhTD39AwEaXUMmQycS+3twMspMgyTIcb8Hk6kz6jATcMtTPRE6yHL7BwbwhzI4OmBEDgUsHLKXUBbWBotW24gJFUCuIGf5GCQCPbkiDGGXM3IVP1SuZJGxmNfQU0qtAoKcnSs5THGXElKjswsi77+ppzFxDgbPQ5zHxTnZFR22AG/LvvvcOSs+9gt3ELwDJFnioo1NbCw/j295Hk7KJqtMOqSxhvv8LG4jlSCxlR28waDLsb5bmnhbYS7rkhuFgAzFVgwSw46mWbSD22lAoBdKrqjnuNTJT1tXUkHvfKu0/fjcRHhpuIUz9c4rf2KxlknqHDFUyyjLKrNb16+bJ8+snH5RT3M043JzwDSMnouIfJPdTf3GdjHcbf+pUBr+2y2jJIb2fIrXdV65QSCCxi90K7kHHXsFn1oH33BqF9CrqTGcLa7hcBCrcxhlda8QigdAsQa70LSI3nADsQ1atsby4/axukN6ZO7VnmQb9KUVRJEjzxOpIJ26oqhNa1YbQtieobwKfa1VjeuodLlaQIUkmXs+5JAYNP+xN0ClZsu/MwyUu03xUYZlls2532LwKMqv4EsCGMh5MggkD7uWWudheqJgnwKgPvRo4uTpB3lFEhhXUWmwweVJtKXF/+TT+LIz91nMqkhP3ZclMvU9xn3wgr34Orf/57kBQlML0KcgSKAnRtRLSDsc/Er4CC84qxyA3wKrBgXEI9bnJpuUyxMlT21IHWdK6yQZ0+QFrxPsDiFpIL969QWsHonfhMvi2/2mWFrFXgx6sc7RvRnv//uCbOjhYtXeN9a/ymzDVtqdM/3n4aDN33Y9QJi+f+96wCgEr7vt+348M99VnjrfHol7zo1OU5LvHnqxrDr/7it39VqF/7P/5l+Z8+/WOLPzyGFPjWU+BvLjwsf//p3y7vzK5/Y8p6Cz5IHvYvesgnOhn5lRvk/ePfbMCiP/C2gTEfNFN14M6lDuCD93nBx8eBPn9e+SjLzOjmbHqb0R8sQD4MXxqAM/Di3yOvjCf33UDvi4zNdYBuH5caDekY1v9c67Ppy1hGcgGjXyUM9R2/xth9BAbK2Ny7d9IjKXI1vgYsZASiQsQHPNIQGGQrKhIRGGKIUE/cx2CUxgEVY7PLARYH7La9I7BgudBnzLZvM9s+cXpc5s88j9jHgufjg3LDVf3wzKYyBa1Odgw+YdItrB9FmQtnOudQcVmG8V5HTcW9K2QAsxoUzIU0kVm3xNxmJjcAAwBg47CRyJTUjeOqREBdfoGBDPM8zIYqRe6sLaMoo6+0YhZGZRqD9EgsABbS1Dypm574idv4bQ+ulOWMtBvavcmSn3tR33KDs0kYKldU2tU+QqaUOHJwFUi4ctAsDLwrOnm6elIYSGfIYZC33sDIIyHYxHjbPS0ERjKaljsMvTYBj2DosbNQt3AaKYuqWY1Zygww5VTt7Ag7i33sGdwx+iXxffp8M0a+gpUFgJurQqla5Q7cQL1ILFw6VhUm9frfkB/3gXDpVBlzaW67sAwL2CJoY/GQmX5XbBKgab9hm9Gju3CLkq0j+8MVANBVoWSwX79+XZ49e1Z+/KMflWOAxjX+VEETWHIb+yGXZ12jrCtsLqfESAZfBlj1tWYPY3+0Tt5As9fUsYbVgjHpKbMtqKj7PCB9gY5kzp3tApInaNduiueu3u8/fQzAupMVvOSmKg0BiEhslFgRIflSImPbA+DQX7QrmSW80q0AWCRU+hc82G4yblAv6TuQoE4qVImF7TfL1MLUCw6ODUv9n2GnZHsZof3JdMv4u+u6S+lKmyUkORvsy3AHsOVqVqZlW3NTxYBY2n4krNBFUGFd1YkB8m4MxC04FIxlh3Lid7nnuuM84ISyCUwEONJQOhimHRlnBt3aK+rMuvC0XwdY2L+ht+OMh+1XcO3oxKKwSafGb/Oo+TLNbKZH+rXfkGfCXAsqtCNK26og2nFIacUMkw5ZTIJ+PEqfvg2NHmO4/T7gYpV+ntWgTJ8//0nMzOTWfNUGmrvez2CZm6P5b+5efc5hfNKBx5BjwL35aWFbeMO1d7n3pzsaGK+PNQ1pVvPZpdk8c21x1viSg+pmvjgSoruPQ/dT39aHAWp0Ibq3DPvNX9JpD7ry/6v/4bcfWAzSbHg/pMCQAt8+CvwcYPG9SCzqIMlQ3BuD2+AMQTIw1tkwbx0s65hbB0qHYQfogAg+aH4j6j3ufDS992xMXAZb/GRQ13N3JJnuXlff+5qQGeh1E2h4xL2F1V/+ujByWd07w2RlKD/eMK91r4eax4TpwhpnylYLm/LlmQiqOx6I13L0gEU3y+jeGDL70WfHLbYGeJcBkPm7Qed7EluEiQUMkOdW2Rxvhh23J8pruKBPtbFgln4HRmcaycQsm+LNHe2VKZjbMYEFYCPLgjILKvPjykyqsjj7bzoBTLg706nEYgGVhhUY77qijqpK2HbwJ2OntMLSWAb3p1BdSB107RVkilKHxCnpnH3VXT+uBCSA0mhXYCEzrGGwezHMoT8/BZMeQJKwlWDSTHDhLG+dOZZXrIzaIcz7LrPa7vAsLZ1pF/y567cgoTG5MkzuKh11KNJcJD1tLZYx5HYJXytGyYurJW1rF4H60jPsDzTKdUba1Y5sPaowraE3/i4rGL335FFsHFTjqhsGmtOurpKeRr8YLiO52NzaIj42zPvCpWS1ZbiMxOThQ+wYYKwXmAl3AzNXxLpzl433aAMy1Ko/7apGBMiQebW9u6v0cpZPXaJ+YP5XVrEjWM6MvgZUkW7JQJKHMLnQw6uSH1eFil0MQEc1qD/94Z+yahO2KLxfxMjZDQCz1wDtaQkJwSKz86quBTCSZys0qyvBRFbpFSoyAUEYmG+zsR/xCsasfxlngUU23wMUqaImk6rkbYw25/4oGoY/vHurPEFiobRinrZtP8vMucw30gU7kAy6/d+xIqpteGobwzlD75HBCXorvbC+rQ0lETL8AlPbkFKGG0B1xgPK7GHdKq1yA7+D2EiwjK4rHFEGAaXG+wJyva/CMN9G0nSX/RlmZ1hMgVSkjW1EcJIyKmUgrGddZQqwRx6zmhpltg1aN6pvqT6lBOuQsAJHVyUTgJlf+1GkCPQfG2h++Wljpi5RS+NtSkIG7RfWv/16mnSsT/uo2ZF20kXA8GXAYh/NO94HcPlsWgRUDcpdtW9om95TEcTH+MdkwJhSJNWhAHiq8Y3RVgQW7wgsOFfp4w1YSOt2mF/roB2W7c97ZMzHc8JbqK88iDtl7gOQFq55b8kbRT8n3ltw481d6J5nA7RALRL81RzU70pzJoIaRYuleuonZBJ9z6TUcuC1i9Hbzjl5bwGSNx9GhsBigIbD2yEFhhT4ZlLg64HFbwks/qgbC9soWMdon+LiQMyNw3Hc4qifeuMA64ew2Ve0j2I+Irh71c0BPuHzS+Rxa0TtRmMfc1uH7epaZ2RlEmB7a7AWlmv9q/GYVku3RcUnlY+3ZwUWDew0vwErxsfRyieLk480bt7rnplF4nAVJW0aVAvxdDlMmdfstcCzH3CZgKhPGCnAYgpjyUmAxfj8avax2LthDwskFs9goDZR29iFSZk+PSrTByxPusuaUVzHUYMaxc7CHERhnoxOAmKij0062kbI8MzkxL4ChkC1D5dqnUNPXVAh0yrz4Uy0S3CaJ2f3BBaqI1Vg4WZ7TkpXUFErQKYFMAKzZAOS6DLOSg80xnUFokUYYyUWk6pBqdIjsOBPv1LT9tFOySB1VY/JbtnOGDNTLTAStJiBfQCCYOOEfGV2GiZVnX1prOH2KukJLJaY6a/LxapS4+w1y80CLF5sbpbPn72ICtMe6jyWzfS1hXDJ2fefPMTG4kl5DEM8DX0CdKzk5Iy2iV8NzU+xDzg4YEftV68S30efPw+wUG1mFonQ3Xu32F+DXbOh/zXG4dOogj3EbsI9M6T1JgBnD2bTU8ZXRtNletc21gIqlpQmwYwvAjDm0XmP9ATap49Ao7RL6OISoIIKZ9iVWuwKLJBY/Mmf/knUrJzBXlvHhkF9eGkIUyuAULI0SXgZXRuuOyrbDgSl1ovG0B7WhVKQLU7pZ7uVcZZZdunfQ5hzme5rZhsMM068Smlccewh+2So0rXGClRzrLBljRufQFBDaQGMRtjucC6FbUOubCUguETaElBsnJTRla+UimV5VTxHRYr+4GpOARcYsDsp4KputnfbjEBFo2uXixUEKWG5AORdUp+u0qVBOJWRPrIOqFRacVdazaK6Nw5UgVbGbxupalM1/drnoT1pCDCa+t08TLjtxTIKegQkDYApnXJjQd0DKhq4oD1lciFXssPV99k3gvscVIjx2vcmWZVpmrYqsLCMHranJhHpBaF/2VZ8F0Ah7YyXKIGlgCvC2h6QViixsF9mUgf3LCJB25ukD03QlmMjhNrTLdTEXBHqfWi1ThuaMw925S6byQw/aZtcv+TcXue9D/altw7LjkPGZl/kva3Ro/NLmfL05bBx7f/obdBLQuFQvy79b4x+EmX96SLA0f9EMAAsBrJibhKLNx55Vz0MlquOctWLv827s189fzq2zHbXX/2lX+4H+ku+s702mrfrYJK6mVfbkqeA29PV+IbHkAJDCgwp8LMo8PXA4rd/s/z+D34QlYOqdlA/AI6BDjhNJSXP3dBZ39XkHG4dYP3QhXH3yql7pB/ed2cd5esH1tA4+1sH7navk0d9mXiIiTiVPFS1psTP4N2LF18J3oWLeyKpZRBYxE1wAYMi09DCJi6zkPRqLAEFhLfs7bsgLSyfDE4DFZEeyGzDtM0yG+zeDjL9RBbmIQaVxGOa0zCTSizK3HLZvZkqb27GyubNaHkFU+U+FkcM5uPHLM8JqBjbflmmABYzbJQ3raSCL7y8glkMjSmP+XaWcxpmRLuHBdLWAHMRZkF1HWfkm32FTJjMkyv2nGIE7Kx6DGJJ29lXmb3YjsCMqIYhLQQfNhyZafXmZVwEMpbT/RFcWckZ92mYxxhvMzMaSY0Z9ZB2NhQOGdZ6CLbUcXeGl3xQ5qx0Q5oCHiUPu0gKXJXIGWBtHlQ/sYxLlGuN1aFUg5LZU9og82petT9QpUdg8ekXzwIstBNwmVw/rEpFVA3ScPu7779bnjx5HPuQ5Itymu/saUF+K7CAaUSi8vLVywCLjz9/liVYVblxQ7f1jVXUmNjBmHoYo5G7idt7772bvJmfHRj1XZjWN8yIv97Zow2xkzTh/FgrTVoCZLppYYAFV3cVNy+GzUk7rdfKdMuIm689bDaeAyyUWGj/co7dzTKAyTgFF0otbJOuhmZfFpBA/Uh3srQtdNaeofYf/MABKSF6jQrZK6QzSl8EFqoXyawLLGSWI82CRlO0sw1m/e8AkFzKeANpxcryKkbjC6nrgAPaimURzAjexrjaDmxzGp9bhn1sYXRTyjeHdKWBBelkPch8Jx9IMlSLcxneMfqA/pWYyYRHKkGb3sOGRRDkLukHh9phaPjPVADx21enAUO3AyrIN3kXfAtmSCRpBMAiIcmVtqLamOCk4k2lXdgUoUa2SBmVXnic0w40Tred7rCk8CtW/9IeRVp5BFwIysmHYIlLBRiUS1oKBmTi0j/oIgEW1JsSHsG7p3XkkbGoK4/+bRe+M0yAhXnmTJ71rzwG+lRQgZoccSpxyjjmFQnFCIshZHUo1aIok8tMb1APD5FWfIDK2IbjCHkQWDh2Dh6m3452n3J0jj/lVocA3nY3A+Gbk0GNtUnLB+Pz3VcfdWz2nWGbHV7iIam8zYv81CiSBd5Axxxeea2P+txuvnSlP3UhiJi7LnzK2qIaCNLK0ZyStwSrnofAolFmeB1SYEiBbyoFvh5Y/M5vlX/xL39QB8tM1zn4OfjqVIGFHzdH1jaUx0c0B2RO+Ph0TJHMqYOtHyO/H031KQM3DnHjXRvYfXZYb1cH5AzW3bWO33VWyY9cD1h06URlJOETUfJojDj1vhYjRJpHE/G/AxVk2pBJD6fcdxlBl9kPOsCiRwcjrSpEmVnko5sduGUCmKmXWdFOYJ4PsnYEpuHsoUy0cTrbO6W6CKtCXWHE/fpmsmwBLF5cjZRNZ4hlDmDIRgEW12+2y/XWyzIDsJhnRahZpgxd1cZPG+xOnUHs6qMBizlmYVdYAnQZSYXnPIyDM+SuwETAAAd3qN7DQPwIKYEgI3UbRqoaofosPa1LZ0tlXtTrV5VJRsj3AgslM6rayOAvAywWUalwTwh1/cdg9i2rTI8Jq/KR9f+5mnd+wiipvqS6TGbUJS1pKRmpS8gyiwzgEOzEoJj7LL0qY84KQ6pfKSGpwSozbuyWS5uIZ+xg/WKTje0wvHafB4GRjLZL8Gq4LbB4zN4L2oeYT2dklQzY7sJYK7GAPoKcTeJRavHsBepVMMOq3FivMsMz5McdtJUUrCFF+c4H7wE41mNofkxYd63eBciZD9vSBOomi4DLBZi5RYDF+vo6Up+qCtXLC4VK3yAv5kfGODOJMo6AC5nylxhvf/jRh+U5+bJOrfvssE35psOQhpxEhBoZ4ExQ6f4i7g1h/9QOxCM2A7Q9jeUPoJ37mVQQWYFkVuhCcqHKjyo3tms3xbsFk34bac0t80/9Cyq03fBoeZVB1uZGO5hR6KO7wMLN/TRsPwJwmS/LJx2V+FHctIfWZgwjANVgW3WwSAlVxRIYEKd0UuJW9z9h35HXGO4jHTo5AYTRZomOdlIlXfdvr5V7G2waCLBw00HIQLu23VfAbFz1lDG3NSlxtC4YcwA6gjIldbU/0U6I3/K4RLLG7w1YCDgsEyED3NOOaX+WtcZX09EGJACGPJiuvdtw9mfHF+/Th8gnWej9mB8Lpn/bh7QSkAtgIg2h79IBARWACTZqurb/86wEA+Fo7gUWhbFolHqbyGpuqnmNRWLRgMVtxjH3sFCuZb2YnkfG5tzVn96z9NOpy1/PC8+OW/mvUVR/PQ/1pntVExt8l0gN3t0MvmtuIYrpDBwtPzrVAnQ0HIirC9eieSu8Ybqjpjz4++W09Mh7vXThGrBodEtDMk82AqIeAouOuMPLkAJDCnxjKfC1wOKf/PPf7gGLDHwwJA6EjrtZ5YQR02sdOdtvvHQAon7kGqhwMPWDk6F5cIDGoQ3XxubR/FT3fILC3CY8X/8AGvz5qRZUZJUSPJtWmELTSVpea56Mt5dQbru8NGc9espd6Ma98SVQdarAAheZSI/2DUoeVGuCEVJdISpQ3LvEq1KCBWcCZZJgWNX1NlziR6ViDMa7wGyds0Pj9vV4eX49Vj4HWGzDdJxmShOGT13zN6/L5farMnuErQXM5Dzc68QozAfqIDcyIjL5nH7IBACqvSyigrPGjPwKzII69oIcmSAlJZFWwIA5++wGZO6KLMMmCZzN9/0lBQ1TmAzDBMPcqH5kjUiCMF3caUuirUCYY5YDVSVpdXUd5rACC0GF6iNKAAQoVQdcNQ1OmSvynTRh6AQVNszmpnRCwKEtQTJHuqqryOSP0CZlfjJrDCBI2Yi/Vo/ATbUfVoiC0VOt5/mrTcDFdjaoc5M/wcMyOuTvs4fFd957mqVeXdJUplE6WEeWUVqYL1cdUk1LNaGc2GwILHaYoXZmum5qBsNGv5ihXl2B6zvfea/cYu8QVdBkhN0Z/RA6b7/eFVLBDCPVgmGbgunTgPvWbYyeYczdOK/ae9DKyaf0sP2aJxlNHa1vVb4OkC5sb28hRfkC24+XzJjvp2+4tKxG24sw/s6OKm3yPKPskfoQ3rqxsGFGB8opcx4/5LeukuSMv5sjnkXVyJ29adABqsu0sQ129nZ3bzcsFOTV5YbR4ydd6eFVtatJ6KK6ms+xSyAvp6jiabNTpWSoDfEnI62kT/BX1e5qW7mhvcdGhD6gNEFwqWRDwOzVsNb5DmpIgoqXLAu8yx4mGlQrjbMP1r7B8qn32NPlFmAIOjk7T6aiWiZwbfSG0GmzAca04czy489DCYx1UetDAFrrQ2mKO7W7tLBqUQILpSrWmcBBVS3LKmBLX8DdMS2rSnWSwqRPiEiRSCdjG1dBvvVlO+CxHmQnOfKVvRN6J17rGx9Z5Yz2BWLIMrLX3FdQQV80PstlnbjsNeOEYNOJEZep1sbiEWpQ30FicRuwutiAReKt+ehy0buYtxyUyfSt67h07k2KUDNX857892Lo33Qxpcx9VyP1/6tCJcUu0vpemvgf/12Q5FE3nqtT90IHj+6ih155uvy3V12Mneea0/qr00A8caReuDZfXqWL/obAAmIMjyEFhhT4VlDga4HFP/2d3y5/8N/9IIOeA2CYko5B8CPYNxrsxvz2MYE0DuR8AuvHD/d8BH1fR9XuUh8c7NvAnYG2i6e+7QIYp/E4k8yHMP7iVoGFzMeg1KIHLrpwvY9EL31z6EN/8HfQzxPX5KfLt950z4mbzGBmAQnQhQ7zEyNmmAdnPp29n+UMqBBYoK4zOlaBBZHXtLnCLaCewMcd5vJybKps30wAKkbLRxcAC2h9xjkm4w0jesk+CleAiznsK+bYiXsWmDPORnkj6KffwASqvw43Qe6gCcyOOuOrMHy3YfaWARbqgrcZYCUNbSOwA5hud7Y+ggGS8TUGmXyZPqNTuhLpAu4akKpeNSNzIUPCIS1cvUgmcJlVp1YAFRpRLzLrLnMpoKr1IRdE2Rs9CSfzlNlw0s0qRTB0pi3gkJGXsY1uL1fTyY7egCOZHhlRGU7tPWRC3elaYFHVoaqkwXq03ToD7SpWm6j2vEDa8OzFZtRVDO/KQBpuv//0MbYB99H/BwQGPFHn5lFaJF+VwT7SkFwpBYyjKkc7qFZtARJevEJliLQsoDPFGp6vI0l5/73HMch2ZlvGcxTaSeN9wIjqPc7iZ3la3FRRW13fYHdp1YjYYRr6VZ16wTShaHCWScDkYdnMn0blbuC3hRrUi81XgBY270MKtUT9Cy5uIUmwMjW2t51I00iHKFeNp7Zv24SAK0vC4tfyy9g6UNgePJvtjYy69jPuIbLqamOk5WpjruykClfbObyCxNpTdIvUgjKr7naC1EPQp26N/UDaHZG++bMdCIrmBcNIxIxHIJH9VrQJ4tnDWfy2QILg1mapX5cEfrnJRoSvdgIs3MPENuSSugL+FVTWnty/zYaR7OBOHxljYsDy2iYisSD9BoiqdKJK+pTyDB4Zi9JHGBMFFrRH6eeCA7YLgbvG/Y5PHrYRVaO0UbHtXxAmwJVwgk5ByAWnfcO4Y8tAWEMPjksVYCTK3jiEj+rQ/RI87+wbLn/snjkBFRqzM/Z4BlTw/gpgcQ349bR/uammiz7cY5nZJ7Sh96HRrVlstQAbo+ar5Wfg+lbi7aFlYiBrubVJdDTRq9+BtJK4t8BvX1N+nAwvbar//MZj4uWne+PgNECSvK0RdvT0wTbRspG6NL8eg0GrS9ILOOgCxH/37qcuLZ4vvUgukoR3Na2UBf9/7+/+R1/y/Zf3aP/q0bMRYCA535kv26FnxmH65dDGYoBIw9shBYYU+CkK+P11vPjK5Wb/6W//VvkDVaEcph1gOOtMmIxmBRYOi54OQjkd8vPf/xA6ZvmUQSzjaB1M+yO+nwEH2aRkbL1XNZa8CqiI9ENmXF+dN90aqFAtSkZIZsz0nB2v0Zl+jacXecsLV2+d2/OjEf94Ttp5rgAiwAOPMoWWX2NIl8s0XteXV1JRV0dCPYIP8jwfapkipQQylc54Rnc/4Voc0JL838AQ30xMl4Ox6fIMqYXA4vnFTTnEANj9CuDCyjUz5UxNlxlWg5pDFWkOYDFxjZoUy9BeATZu8FPVWdQhB1jAPDkbfwtde5ln1bFm0JuWXoIFVVkEFerMq4t+xAy0jaExOV47Xjb1L/Uso2UTOKmCIhMnnd1zwRljN8hbjirUEkztEm6qiQBCYFxC/9Daj1WVAmisaiMMiEC1RZCQyiAt3Z01dw8L703HmX2XZnWfjLQn68y6gDnz4yfwVP0muvkwRvpxVr82dFS+mMl3ZSZXdNoGZDizLPh7ylKzT995h+Vm78Aos0oXeU6bIF1XzGoAJqsSwfjKDLd8uTPzPgbCLj+rmlAM33Gz/biU61NAyxoXFXI4AABAAElEQVRAa4Y8yziOUQcypxomz7Acr0BItR8qJQBtniU/F7FTmW/AIu0GgpghjrRMaF8/+jC/1JNMuntO7AI+NbZ2RaetrU359YAxDcpddlg4Ij0C2Ggv1nf6tTTkNP9ZphV369Z+pJQq4JK6MU1n88+wV7Cc1qsrgS0jpdI4vC0KIECyvVsnMjCGU1olsBC0CcJd8lbm27bnilOCFukbA33uPe6z6eAGO3Qbv3FEwgWQViKRtgI1TMf4tD+w/0XCAeB7uckSvC+3y2fPt6ELy+ZSJ1JvmjYkCLqFwfY7rGB1K/GjjkhZtTPRdieglTgECjZcy2m+qxoShLEdk5/uJmnavnWxDNJR8LmJnYWSHRdE0J6nqjJp+1MBhJIgJRenngE0AnrywCntkgaRpq0nPRLgyDMJJk1/ONpr3zkG+rIaZuOPSY1RpBUj1MsV9ZCVoSwv9VFXiGK3daSmARe0T4HFDPRcpE4fIq14CrB4FzuLlWnc0+9rmvW3y08yIQ3aaJ5sQfF6tPz51Ny6V3Xc5aH6kYoe+Pqyx+qat/mJ1+Z/0Llz8/JTcbR424s6htTaw7+ZyEnQjrbGnBhxD7D4GXnrUtV74qhhBrOgiz3Y9/546YcaqkJVmgx/hxQYUuCbS4HKb/0sYPFbv1n+4Ad/lFHRcTYMiIwCzGAY626Q9QPmnx/d+kHrD6QOoA7OXuvnhoj642ifcp1z75UfRcPqowufWW+ZI9LJi557/ZA6j9iTXCQv/VlrvHb5qPG2D3aGdRIKqCDFpN/5NRHD+RPAwXPKjVO95qUeYldRZ/PrsqvZLI4P9Bwzt+7rkJ2UwyCSd9ODjjLulzAyaD0xi81MKzOFZ+y8/apMls8AFp+cXZfXLv0KowNnUkZUUwJUTMGAz6IOMs9U5iSSi1FWjCqoR7G1MypRbJZHvDIw5mEJVSg3gNOOQAZaUCCwkHHZR599H/WnXWbeXTnHJTLdA0DmN3kLQ0jxOEIXroIJpR6qQ3kvvZUWyIgLNmSQXXlKcOHqUJbd2eRIFLr6UloS5pRZ2YAX4pWSopiAQuKVUbSd6U8GXuBgvgUUGso6M12XYq1G2jZkGULrVf198xOpA2nKgEZvHkba1X40SlYd6iUShj3KroTJje2ePHpU7rL3whz5V11H5kL1K1dd8tpUS8J4wlRrOCzgsF2fUU/uRi5Tr2qUalEy75b/EXGrEiWAOKXMFI5JYvJHebSpWKBuBJ7a/AgItUsRVLhcr0bOoR1+0x5JK/2JZ9tQY9zNn+pNrljl5nav2SF8E8mFUhpb/BIAx7p3FtqYZGxlagVWSkIaE9uAhXkXVMisp+1aP0m7SrAEC87CS2PLaNlsczE0B74oUbHX268sk/5U+XKZVuO1OG9Qv9t2Yz8kC4IZJSHuTm68tjjbzaMHd8t9mP+7t9ZT/0po3BRQ9S/zbJ4a02+7VMXJ4cE6++KlqmEY7XO+YUlgDewdO5bpDxvYVTwAtNwj3hWelYQRFU2wAl13M3cFKssZFSjC1UkBmVBpoV/Tp/YJl1W1HCfIn0DBfuWmjFsY6B8h2dFGR1AuPbVHMl79nfDumD7nZoAnSFSqsbVgTPBN/I6xnNZ5HbNaTyRpK9KfXL33qO0jEyoSWWKYd+wqBBUjtLtrN8Uz04IKaHZDv3XpWUHFDeDimjoyr0pc17EZery6VN5DWvGQ1aEWkFYIUNuYnhSNqzuSx+6+ubZrct5lv7nptZWojrE8NQfK4u1A9DVmHHtuPb/ctHsi7zHrA241cJdyLwJdK40TgZWbMPVqOdvRyubrQfe873vrZaPWVw3df53QvSAtTh2M81eG+1hUgg1/hxQYUuAbS4GvBRb/5De/V/6Fy80yKjoc8s2tDHX3scug2A28ARV8wGQMO6cQpfs05N44/FAa2VsDqh+Q6syr+Kr+E1f3EfOe6deaDvf5q4OxA3K+oeQzkgvyIfPiHhXV6LjGkYxZFpPo8mA+wlhxrX8mXYFKLQsBiP9tYFFzWYFSjTsz+XyI52R6YW6ypwPPrhzj7tAySZm1J1+mrUqHzN0FV7PjLLb7WdxMz5c3o1PlGepQPzq5Ki9OL8s+DNEIjB6cDkvMchLGFaFmsK+YYTZ96pQVo9jfYhSphZINwQVZDsO8oFEtTME6zJPLgcpYytQ4K6wBs+BCQ9MjmZswOTB3MHYCj4AemBtpJM0stXQPjWEcAwIoj4xvZUAtt+pfc5GQuPSs6++7ao9WGZMTdcb27AxQwyyuM7ky/Eo6NP5VGhF6wQDHaJlZYmljI5XJsi1JQxk9mVXvBRBezZ9+PHvAk7Ayf8m/V+imDYkGybG3UGoBY2s579+5XR6zl4W7RWt0LLCwXal2U4GFeah0EFReUh8u0auhucbIZCq01L7B/TKUihxTPlXE7rv8KpKQFSRHZ9LVQtE2zbsSHu1vnOmvoKletU1xSVbViupMeZW+pN0T1jJGDYpyqXal6s4ZZXMDP1et0hjaVbR2tpGiAJ4oBHYo/Z3JbfOCNg20bQ/0gpRXt0hcYPJpQgGRAlJtR6R7GCrz3x1VQlHBY1QlaEdKBoxHtls1uFVW7JpH4uCeJr2VnaDd8xcvIjn64tV2VKJUHzoFCCk1sP+6atPDB3fYcNDla9kJNOlTp9oUWZ/JA20LRliJWd2FnY0YAdwCo89YXviTzz21OUHND9pbH7dRfTLOJ4/uxyZEUJFdzonPfm57sV4jpRJY2P99F4ok0TzVu27soT+YJyUpAetIAd9g47ELYBfQRJLGRpKCMPuK6k4uS7sP/bW3EGAo+RJoKx1SXcq2LNBp/c/0vOcnSdsfvUudePXPjt+5meZboALwYDtVihE7L9r4NWPTtXYX1g2SpxHaoWGcOFigXz5ASvEuEgulFevmv+v3AS5JJ8n1fpK/3pN5aod3Nd81pzx2tNZHezN4V0NbJovc+fAj1B21qF0K7X33rlKmhgtN4g16DKSUiOO/0rjeUssDcfXS7eJtl0bnt94nQzV//vbe8dD8J3yXZe/jx/Q6P0OJRSg0/BlSYEiBbzAFvhZY/OMAi+93HwdnHztg0X3s2lfDD1gYea7OKtcBvz9otnE6g60jKP8yZ4lwkHh5xQ8R8JurA3IzVjRuGarMeOZ1/ZD63jSdI80MOn406BznI2qYmiHjrJ8bB3PTbx/tNhNd0zQe06hptY+S76rqU43D7MW/NzB6Agtn7N18zBleGeowOny8ZSacYZcB0q9pV1BRjWFVi3HlJIHFCMDiCGDxCgPuH51el88BFluonVwxuzkCs4b+TJkg/ATFmgRYTKIWMstu3LPswj3NdUSphTYXMCiqXqgasqYKA4ydalFTpCUzrxTAGdVsBAZTE6NR4hdUOGMcCYBMMPGEYQ+jY4kruGgAUgZDAOcsrDPGs+hguxGfzOgsjEhd4tT6GKkz5vg5PUdVhBV6ZKSsH2frF5gZzQ7QzNCHWWX2VPCQtiCjBbNpfTnj7cyxObEcl0gKzF9qTeBJGNU4nDG3raT92DgIcIP6mCou7ubsxm8vNlkhCsZbdbAl6HOXZVKfsOu1+2GoG2h9ySxeqQpFWtUwF5p4D41kIrOKFqnbNmQIVevZ1Egc0CJwM6/OuD+6dxdgsUT6MN32H/I0BUMXQMZVg/GlTv1JUKPRdsAGtNFuRPqmLLbyrk/Q6sOYWEfaSwiajgARGnIfADCOAJrub7FHPR/h1tql5XLpY9uh+a/2FM7M17ZZBwWABemoOqf0yU0XKzA21Y6eENW4Uh+0D5csPgSoClZU63E37XmM0O+xp4U2N32wRhsjrx9/8kn55NMvyidsMuju0PYJAdsNp93WpYQfYAPxMOet0Mo23doejYMc0P6gjRKugAvCWNlKkz7+9Fn5+LPnnC+j9kdxKc80O4KvIwm5U94BWGhormqdNhS2KQgKXcgLeahSEds7btSZUgQaBE/8GVkNQTZol+RLNycMdpBcbW2/iY2H0grdBZFzgGcnHMyr9h7aXgjss19GwFidaLCdpc9BE6+REHdjbtK1PScXZrg7bBO5rVfrjg4QlTt2umNpWVTTVLfsgIXXK+o2wEJgjGRViek4tLCtOY6t0Y8fr2CHgo3FA/un7YaKqe2wS8c0a8KSjqM/Pvr05aOXxxZGDwnXfHZ0JbIAuZS1lwR+G927CLrwSbVmoEVUr/HGT7vi2pILjeKr1acvvxpY1Pommpaf7mrw9i5RxZ0U8l+vBMqr/JjHt/Kinzqm+n4ILPqkGt4NKTCkwDeTApWH+BmqUP/oe7+BxEJgUQdGhkA+up4O4z7VgVYGzlnrpkPsOOpg61mZUu8z1iZc7n2fj2Wi6VKo920pWgdkQUv9SNYPWgMOlbmobhVYeN/phBMmS76Sp8w28sI8D56m3c9bfdfK09uN248ocXm4JKOLcZp3S6JBpbd5hHmUWXAzulkYG5lFgYV6/qrtaNQs8+zsrZmUcTiHAcnsJCypeufu9zDJh30MYHEKsNhhyVmBxScAi89PMSaV4YKZJXBUEQROYzI07Gcxi43FElKL+XMkF9zfONuKfzOrupIGvOqUL8MczJBPZ2Jl/rZQ21FlR+kF/EsYGNWUnAFX3UdGT+Y9jE6ABUyV9cYZ+lN+60YgNggs5gEUGo43WgguplChUDVLsHWCxOIIsKT6kOGkTbWdkE4y1NACuskoyjSahupM1pcqRH6ZnWVXIqCqj0awMrcyvzLPAjkZXkFIrrYD6lGjZcFAXbEHAIAdgjtoq8IkGJHxf4LUYoPlUt1HQlUrkg64sKN4amyrBENJhvcxdCZ98yPzeM51FzUYN+PbBWSY9/uAivt3bkWSo12Bfpw5nqW+bUf2HdWf1m/dDrjIPg/QQJWubF5HvJWBpz36R1lbm4cYqR+BhapQRyxnu/eGekUl6hhgcYTdxYErVlHXgjPLI4CYpC6kifV8woy5ZQoQo81bv9q8yCxrN7EaUFD3YUl/sO3jJ/2HOrGdKK3YxVj8gLIf07bOSWuCsCsbtwAHD2LcbX/QzuIawHqBSt/HH39UPvr4s/ITwMW57a9rW9fEldW+MKZ+cB9QlvN26tbVx6qdQm0XsmRNgmUftN8JBt3I8CfE/eEnzwEur8jfRaQVSm2UVkQScv9ulsW1zRmHnTllSjupK0hVJp+2R58QOFLwnJS+0kCggQTFeuFN2oFgVXubza0daKNaIvWrJA8JlNJM2/QpgEObJsFF9mXp+lvGBNqH6eWeNM2DfbL1PSsxYyLp0RszptgOPGxPueoHWtNwOGnHAAs6SNxu7AvWQ4CF4ALgIagATDsGuTDDKv33NqpPSiruA4g2yLcbK9rvEz/lrVd+ve3STX76j3oZOKqnmvdaBsvUwtZo9KNbiwnX+l/j0X/v4EUe65jUeei9zU2X3xpJraNBDzUvA+GJ33yYfi7Ji4/9dBvt27VmoXsv3RO8y7/xdXl425+x176W9HzC31AVSroMjyEFhhT4JlPga4HFrwMsfv/7f5gBz+HSgTGgohtDHUEdDGVAZHzC7Prx6ZgTP9IyXHX1qDo4O9wKTvKRDLDwIUMssXcfGwfi7jT+gItupsy4m6F2Y670o7tXwY2b1LnzsQytM+pmV4Y5KgZcI6HwyqDfPtZeUxb8W57KvFiWGj7Agmw5e+g3pi7dmaxn9lk1DfcKmOOjLLCQyZSRcoZS421nRZ15bnmRGYNVSDnDQMpMY8g7Nj1XLkYnyx4Si5+cXpUPTy7KRwCLQ2Y4ZQJHYDCmWBZzAqbLMl/BoM0AKpY9zw9Qi0IditlgmUxBgXR0plTj7RXOWfJ0BROppCL7OcgIwuSE3mnJ9WPnevzJI1fpFlDGVfWblN8qwj8sbmhkWZ1h9hRQzAEulrHvUGqhepTlD8MPY3gCk3dwDGNFmQicvQDCxEMO25GGv9pqNKNm3UJp3ssAWn8y9c72HqBqol2I9aVUSAmN4EWVl6wUxL0z5QIYZ6BlmAVpzu6/YTZ/G1uEbRhBmTeXg31w/06ARXbwJqxAxnYRYGFYaOfKWbbrBixUJfPeMrpMp4a4Xzx7HqN4mcrHjx5Gr9+2tAODr19BprPlCJ3S3qZnkSrBhC/iluVnYfKUWBje8nuG8a+dLtVlvjxtu8Ypc68B9xsA05udbWizC0OqkTBgijyqAiYQUv1OECaNDJ/dralnVZDsU0ZunNqeWCZ3A9fWQ8DT0qvld38RNlcE3LnXRfZCEaTY7ijrFFKrZfbvuHPrFtKyxdTLJH1AqG4bfPH8i/LZZ5+XH330WdmjPZwhEcjeFtB5hDwqQ3hwf728A7B458Ft+pM7hEML8i0tyGzyChXSLgLQaHvWs9KTH/744/IRwOJzjLdtv/ZF9xV58uhe6vnO7Q1WVu1W3gpgJV/kXZCljUVT7QqQpO3b4qsdiS1fEGIbsE1UqYT9xOeXAIpNNsZzYz5Vq2zbLqTg4gn2E2ms5Mp2qwqU0gtVx6yXOmZWUBHJoeMnbbMCHPNgkYm0G+/aM80oEzXpnD0/0AdaaT+hfYVLWisdvcHGIuDC8Uh32tmI4w9qebOcK/TZ20xCPFicLe8wZqzSf+dtf6RryVsbMG2HMJtM//A74V//qGNr/zn5T0wEdzDx4NKiMXTG1/qmlre7rxH3Y293b6XR4iRMTasGFjJUldYWWT/F5mJGGjjzvuXPKJvvvt+OFp1DzUPfVwvbdxkoL2HMe3vX8jkEFoPUHd4PKTCkwDeRAl8LLH4NYPF73/9+Pii9wjHCtsGcUTsfeJl5mfhqzFslDA6qMoDOuqla0Ay+404EXjPwGpmjtgejbAZYP5zdmdlZP2gdwyM/4QdOwJAPHV4DAjpAYB6yQlPyU1WhyEY+zOrSO/NnfiKxSB55Nm3uAyiYybM8MikNXJgv/WhSGga7y7vZNue+M4xqRrMwYzFu9gpDEVsLZgZlEmWILJdxCLYsk+olqsFMcU4yaz2O5OJibJKN8UbKxwCLnwAsBBfb6q7DsN6wSpTAZRxgQVHLFQzQ5PFBmcPGYuUUdRcAhsDiHB15bQpk/GeZfVTVaBFGz5VeZCSd6Vdt5+CAVYBkBqFLPuj4tw7qzK1SAoFgH4RR1N4Rhk76U3alEYtz7PANI6K0wnOJNN2kTYmEzL3MoGUXyOzsYtzKSj3adsiEKl3wCECDdi6NK+O9jBTB/JNM8kLWklcNXTWM3Sf/zvjKgBNN6kEj5UUNTUlfMCPIi6oLZQlDHGmDNGBGHwb0CFrIuJm2y6WaZuwsVMciX7bBrEYE0yko0ZbBTdA0vnWzPplE6ZVdxsm7s9Rbm1uJVwDx6OED6p96he6v2LxO9S0lWRsw3dq8CDhn5hbKAkvMzrEilIyo9hXTpO8Mv+0w5eMahjp1lO6SupJu5j/2EQImVLHevN4KuHDncYGF6k4y/geU+Yx8GI9n+ht5h3S9dFp6tf0K8AA5giZocUM6PbACqIjhvxIRJCXV2JleYtumXJOEWYCWMvMCC0HKlMwtdXhFng5YwWprWwPrl+XFFjtvs0rZOR3Kfj2On0nO+3cwsr67yrkRqZLSvbrHiIBfJraOGQLHqJVBf1fIUtXtT/7so/IJalAvt/cCSDTavovk6CkqUHexqXG3dsvmDtMux6qkQ/Cu7cwpgP2EdiHdlHSlb3RAJmMXfuVAAwRsF4TLRpPQ4gWrUb2hbx3RByPJBFRENZA+4rhiXLabtuSsO4IHoJgOtEuc5oV21CQYqmHlnbS1wiWi4IJbaeBPmzDJuKobaYFgMZTBMH0K6RjtKTYu0o2ALnFNo+Ud5ad9zqguKY0W58p9+s8Dzg0nRhgzptJWbHNJ3djrQbq6NNfkhQevlrMd7X3fZ3sTj/VH781jP2iKWl/gGPf6spUzTy1ci7bRiQgllUcNxU1ziGv96WIccOluiSfptMA8ezQg0PnKJW+Iu2Wl5c/+0Bzb2xZdL3wX71AVqkeR4c2QAkMKfEMp8HOBxe92wMLBMgOmA21uHLDrrLkfS2fQZcwqsy/z7IdOQNEZI3JfmVQHat75k39/OD3aoNwYHp8Hzrwn7SadqAADJgSGS3sKGaIGCpRayPASPGlFvcdZ5gCd+rEQXGgkmeTx18JOEFdsA4yDOI1EP5cCJG4aMPDDYc4tq2mZpqpGUceBOezNoMPoOtsahsg8cRqu7ruAVANQoSqM4EJgccXSkCc3o6hAXUUV6kPUoV7CjO+ictNUKyzzKNPdVxoPsyLU1MGbsniMSs/JIatEHbPvBTryMCqYHQTczMOoxkAaxks1Exkbl/oUWCi9UM9bpjv5kmgcKZ9lxDH1hbP1EbpTBq+qpkkjQYUb8bnR2ALgQoNYbS2UPGicPeq6+UQkA+YKQNn3AQbsJev8n5FPPEAPaQjQIo+uYrTOSkouk+pO3qkH8m0rPIcxP4AJfbZZ9yc4BmSYY2eq3SxwHjuPu7dXyx3CmnZmuGUK8dVUWvQrwy0TKlMX6Q6ltw3PIy1wZagpmK0AHsrZGEhnr09Qs4mhMWFl/qIiA2sugypQNO597Bq05xDU3GKlKetaGr/C9kKCuvfD+sZadP412p6eY7YYUDENwNCA3SVcVQdTLS/2ItBGcNHrD5QltdTVifkzL4KcffazUGKxs/Wq5hWwIZPsKkxKLdyrwjYdcE685sd6dDY9tkFcBTux8SAvgkIlHPoLSKFtaaNzIDglTo2V3Rk9kwv0nfRPxwPyPytQwxh+gfLNwcQHYJJnwa0z/e6K/gZVtC9evi5bb5B60M7tH8al9O/+7UV2xnZH7yXqsdp5uIu7h21VNluApH8lLLYTlwJ+CZ3/5Ecfl89fsNTsQV1S+DYrQD0G5LlKV1N3C+CnrLa/qB+ZL4CFoPOY3e5PyF/aFWlFjStjWh3XrBcz4XK/ggSlNu4H8wqJxR400c0xQFAhYFVyJwDyqEbySi048Wd/jIQQGqfD4ScSVoBcVjVjkLGO2xhqHG32vfZT2z/5in8pwyEoZhxyQQgGGO4FdTXPjlxXgA6XuRZcjAL+5wEV7nnyFGPtd7CruIvEcaEDFY4jjZluzHHSIKm0Qx68mnJS56GOjvHV89P3Xd17v5TbotMs+0ci0q2fYmXWu5gN03wbLmFNuHPlash+nBWIJshgOjgMxNRizDXpGd1AnEb4peDVbxey0anmtQvbBUjI7r5FWQPXPA+BRUfE4eWnKPDF9kX5ZPO8/DvfZU8sP2fDY0iBf00p8LXA4h8qscDGQtUj23HGQ8a/tja6rTs2D3x11H1WFSozqvph1AwDLrDgwxt1Ghl5P5C8y6CqP+IdHNTzESHeDM4O4Dn5qHNNDrhUP7wzfdxN11OAI8DoAQ3eGbdpOvOnOlSAhR/o5K9jmJMZvsOEl0GRufXqWRmBCiwsQ8CFcRKmqkXV3DdgMU3YqAVFWqHUwhnzqu8vbWQ8VecZ4aPfgMX0ZLUrkJGd5LxhJvF8ZLxsso/Fc5TPn2Hs/AUMyCbnDrObYa9hBOHdYqw9iqHuKMBi9rACiwkY35Hz4zIOg6uRt+pB2nmoirEMg6ORucyfDJjr+7+BCd5BbcPVgCxjSN1rsKl1SkxaARHQBVpH3aujkYBq2dlNdua9C7OsIbRgyZl3z6rKVJlzmScZKVVFPnv+snzMqj176JlfQk/1/pU2zMKQLpHXNTZdc4O5VZgdGTQZO2l4htRm7+iU8NpHKHVR/QbmDkB1w74ea8vupH0flZe7MHPoz8NMVVYAQNe1TcupSpPA4hyJhypOMnDW4zygIjtHw9xH957QtuFqo6H6Sp3NVlXGdN3LRH/j5NvDTehc3cjOpfsC9hqX+FGyIuCQAXY1qCXKZ/6s9ymlAhjvT6EKpzQjwKKrp9puatm9T/vvSmR6tkD7m+3RfMqsCy5eb2FAvoe9BfdulucytEpnlFik79AGbZsyxh6226igkR/T9xRgSD/38pBWqhjtMSu/i2G6zLOnAGoEkCsDrbG3fcYxQgPxaWfrAVGqA04DmgVqNKPQ2nZWV1K6YCEBdqlml+znMOXHMNkCMjd3fHQHULGGmhjty92jLb/twHpsh+1LQGTb0dD+NcvXfvzJ5+XHHz9HqrWf9rIOyHznwf3ywbuPy+2NjexsPut+JeTXPkkBkqeoQlGeI0CFK2wdcl4CNHgZkKf9tmOJxuama+9wqV+BlZI/7SZcEcxdv91xu+7tgmF+wDYAkjyOo8pYAXa1s4g6FH3CfuG4ZP063hl3q9sAR9K8pO3brjKedgQQbGRs6oCF4MJOPELduSEe6I7KxUBblSjjBKAThAkMQLDgQmABHRYBFnfY0+O7LMX7ZBlgwWSBq0AF3FpZ5Kbmyuh97g7iGnjClwc57xzjN471zdu+uzgMYdl9bHF33p2G6dU3NzrnVV9nKVH2/BiF8XDopeXVa/PDvA1HTaAHAHTi+PKz3np5473xtPi/0r+OHC2eRgfdDJnidXHqNngMgcUgNUr54eeM8zDUGwvj5W+9R1v+muP/+gl97+CyPL41WT64X8fir/H+jXv1v/yf++Vf/d+H5b/41Tt8I9OAv3FlGGb4rwcFvhZY/Nr3vhdgkZlpRlIZAgdZmRJHR6/eyow52x91DUdNB03/GMXzAeSDWMFFe67gQhLjlbM/TDtoh3kisfwlKYFN88PVPqU/mAFnKCc7UBFwYT4EF1zNSsANH1oBRT6+fpw5G7jJh7xGB8PsikIwKZw9YIGbEdV4ACid1EJGOMAiJajSDiUdMtmGb2vqR9+f5zCeARjMqsI0kXHo50emztI7kyvjNQMTMAZjd81KLnvXo4UxFYBxXV4w078JqPA8hJnNMrWscjTC7DHWsmUERmga+4opmOspVocaQyVqghnhyZvLMOWqIwVYsOSsu+kKLI5hfDQw3mETr503e9WIGzo1+kMW8lfpbl4FXgEp6GBrQzHV0cqZWBmnVZZTXWNpW42SLY+Ml4xklTb4Ue8YIBgjd8D+FAPnP/vJp6hEAWpwq8CCuMnrEnYhbormHgNrgAvtRJxdNl6Z9P3j8/IpwOLl1pswpecwZe4+fXN9zi7Kc+W77z4s7z1+EMlA1KDSmmpdWq7kBZqq2uTSogILOwMRxJZA1aVstCfTRbllvmWuBSPZa0FQYlhOw9k+3FlduhJ5wEVmuqnfKaRR57RBbUsMq/RokVn87JcBQ+dyta4Kpr8p7Gy0uWnAQvJXNa8K2nvAgheNrzJte1Fto9cwuEihMN7exX7kgCVnVTnaQzXIJWgFk6eUVcbffisDrz679Wx9SWPrz1WxBIX2QZeyFUwdACrc9M19OgQWBxppQ0P7+gTMshKxtAvCGXdWaxKsBDDW9m2/si154CXp+qSa3/4hjIFAEeacF1Flu3drGYDBCmO0Wf01KSl3+dNRtcDY09C/DrG5efbiVfnRh5+ikrQbmkvfhxhqa5jvucxKVbNIUFz+VlW1ABXito4Fjxrma4NzivH7CaeG4Lpn2KGP2++licy+KnGurHUk2AQ4KoHQ9kcD/kP6l4DYXb5V7bOfRC2P8crxUJDtsrMuPtDCqnbYGwNDWTKW8dSxzKZVx1IlsAKIC8CwY5uSTM+qMhWisIQ1qnS0JxBrNsHDgEWyER1A2ZN2rXTUDfKYASkL7ER+FxD/b9xaZd8K7qlP1UqpMaujd7TbNjbUF50rbcm/3hHnFqK5du+7cr0dT/PDNd7IbXdtb3zMN6N1AD3GT/NRr7aQdgymUcPb5uvbAID0oe65BRq46qd699f7fuxJOuXuAgzEFReC9LPXD+c44ZH0uZrHIbAISXo///x/3Cybu3VJ9v/0b6+VDx58NWD4+OV5+a//5+3U0ZM7U+U/+ztrvTi+LTdDYPFtqclvfzl+LrD4/R/8UQCFDFOTXFTmBsadj47MQWbXGBR1j2SB8dIh0wHTs4ry64dQ5j5gI+86f92Q7Ujv4NqYJ/kPmZA2e5evovHq3vkTQERPnY9klVQILAQVZIw0VHWKtMIPcZe2oCAf6S6PRBWmIYBAZhlmI8DCOJ19xZ9MWz7mXj07cGI8BE+a5kVwIsAJsOBjLbBQz1oGe5xZVY17x6J6AVghZnXWpauqUurUz2FMmmVn0YlmF4Syx34Wr5FcbOFvEybE8w3M7AGzx0oY4Ho4sadQJQomZ4ZVomaQVkyy/Ozk+VGZRMfeFZmcOVU9STsLyyczeQwDtIfEYtf19mEUZZKsL4FArQMKxiH9o6ZEuAXtKJB6eFXyMUW+Y7+hG6BFFR8Z0gooVEeTkezDQuOTIXoNo+s+A//vjz5CB36H5WfdD6LapMiImc9FbCSWYMBdrcmVrYzfWWYZpCNWlvr0Bfr5nK9Y2lOGLsDi6gyJxUz57nuPygdPHia8eSATpExeuDYmQ0bSGeqACmbxsxkeDKMqcjLzShKq5AVwQbgrgJwral0EiHCFQa/6+AITECCtSGY6/YCnAA7jgkYXXGX8LmGgBRWLgKZsfsc7jbWVXgVUwNQLLGTww/TTuMxLAxfmPW3fRmvD5LBt5o+rs9eueuXeFfsAikOAxb6n4AKpxSHSjGw4SDj7tO20SupqG3UJXKUMSmxMRwbY+NyxXFDxmhWnXmPbspfVxFBBo26VTEzSxmbYlTkbQ9LWAs4F0uQ9tg8AKKUZkd7hZqdJu3JcIQ6L4iphh9kN3k37MB4nnnUY3AXyY9+2PxuO/9RHaIGz7dVT901WY/qU5Wt//NHnMPvnoe0KkqF3Hz9idal7qMjdjv2Kth7WS20bFfRqvO5YFRBJ3db6pZ8BGi7pR7ysxs9crW+XtHXDRVfBUi1MFb/sBcP1EJq58Z1lD6inDIIMxxPbYN2DhPqCvkoqctKGG7BQEtyOVDNhPHSNhEIwwan06RRw5zXAIoMjaQBUxxhDtJ2gY0EwbEiME1q7ItQV8V1yvaHduSGeUgv72D02w/s3USN8j9Wg7gAslMnY1NrRbnPtvWiu+LKObI1edOaUlU4ZeKwHT51D8/ZWIj1v1ZNjVe8gvs7VVHrONcH+Y81R/U0+8uNzDdPCmrPG2L8VnVF1wXXXfz/Oem+5PBIjefTai4v7Ftybmqq++7GY50H/0uDv/dIv6+mv5LBv24c82nUwYd3MX1XBY+NHvjeea2t/dUz77/wPm/T9kfLm8Ko8uT1Z/pN/d3Uwi7377/+rnfJyp07wLM2Nlb//76333n1bbobA4ttSk9/+cnwtsPh19rH4PTbIkwEJsOAayQQfJT/IumVA6o2gDpvVrQ2YXutHtA8s2gpNlSEy8MCHh8cwZnwAAygY3PweDg58DtI+Z8YVhiJSApgVGaQwGBkslTJU6USkFR2wMM0sR08cbbBPDvhxLwpn4d1BO6pdMAHGp7+q+mR8FaxE7YT7Wj511CtzI31kgqJWAqMs8y2jrY56llDlXoZGUOHArs6+OXG501mYAIHFFHr24zDQFyw7u4/UYodduHfI/64qQDCnWwCAbZiXHWdSmd29gTGBwy1jMD/TSClmXYL2lBWizg6QWqCOQbmUNDhzLNAxPWdKjwAWRzCIMkau7iPDL60DhChH/QbyEYWelmmGsEuopKygf639hOpVAgkNPl3BaRJgZNjYBEgH7htDbF12EYYxklF1r4f/589+Uj5jhnkHgCPTNQOD4xKnC+p3Ay4EEzI8q0tLARjaW4yjOnbKLnOfv9ouH7H52bOX23zwMMZmNv768pT8TUViIbBwqVQNfiuggKGF+U9b6j7qquJIDz+ezlTLLDoDrQRE9sGZbTf6m0WaIDi5xoZDf7ERoB5cPShGvnxwjUNA1ZbJPYOxVHdeXk/g4KGBs7tqu5ztNFIB7QWcMTd/2thEHQ4GvKmQSTbp6NkAt/Gkn3nD0dqgbdtTqYiSCfezEFC4OpSb5u0BMNx1XNU3w9hfbBNZDpk0tfWwvgR43itxkZHInieo92wjZfJ8Q13JNNNcyoQAiJWYrDuXP3aZZ5cbtn24Qphp+E7gLGNdQafltQTQhR/LGGabZ+M8V3LEYdi6khJlx5NMv2OJZ2+M6Pqd71xlSVDx2bNX5QtsK6z3VUDFw/v3i8DC1akEdePkxfowTSlJhKFHjbuOV7YL61M3++gl/eqcU/Dhimm2gag/0VaOA4YA6YDzXdTE3PBOw34nNKIOBm1VBWsSi9QjI0CkFvRDQYVSC+0uTDeTKfY/8kZuzF7aovSSWF7Nm9IKJT3uB3NieuTrmjKNuLysxtr0kywzCx0hGCDCd+y8TdmVVqAQF2BRVaYm2euG/SpQZ/xbSCweL82VDfqhcC3pJjdmoB4RFvjCyuMwj+bS/5pbf/nL+xaD7wgivb3prl0UvtH1S0c/TgPZbuxPHrbht47u2dftTfp6y2Mvft92PgzT4tGpn5lE3fmqflr2cGy3ehr00+7zvnnCsbkn0u7H12+VgbSHwGKQQqUILKYnWXZ6faL87z88Kv/gl2+XxVnH8f6xf3xV/tl//6r8nX9rsfzxD495P/pTwOLNwVV5/trJn8JmsWySuVLH4xbLwQn78LAh7d3VCUD6Tfn4JfZxAJond51Uqr5e7KBmuX9Z7q5NlDVUswaPrT1W+sP/yjwbpZKfz7CFWMXPPfx62MQ+eoHKLczHww2kmNN1omAwjp+XxyGwGKTW8P5fZwp8PbBAFUqJRZVMyFxWYOGsvMBCZrqNsBmfGT7D8Dj4Z7Cuw6n3bd+IntSAD2N2l3XQ7b4U7YOR9IijMg/EyP3AGB16+s7NmqbIR0+6QL4EO34c/HQFwMCp+IGPlEQgwB8Xfj27WA3CcxhwGLg2iyuT1T5kFkegEuNt804kTWph+Wp+BBU1P8YRJllmXqaNqwxbVC/4sJu+xHcm1PAy5LOoZszCxGrIK7C4Gp8uhyw7y3iGSlQfWOzA6LyG4duBkcrmZhi8wpmwgd5ZmUIfXKnFHMBi+uyQDfSO0JMeCViagtFzFvySwU1m5gRAcQpTJAOsWoaHQEqApTG+s6bXQWECpxEkCDMx7ryFHrbSD1f5USdfO4GoPsGwNQY4zFPIa0nla2p7aQxk9tGASf3hTz4sH372rLzcfB2mOIAMWqn6pPpINs7juoKahuDCjdZmoA8a7uUFkooPsdH47MUmdHAnY1ShrlSFmi6/oMTi6SNsPzD6hbbA0D6Tk8qnJdguuae55F51F6U22ihoS6BKjnr9ShJWWbFJKYMAyrLJyMpwRjUKqVFbKSrx2SYBkqeAP0GK7U/phDQynCtAqfLms3mQSZPRnQFoNH8ypK39OcufMphRjtYb6pPtuJbDq8BCmxGlN4fZ0+I1kgaMiZVYBFygxgSwsD9o1F6lWBrbu4CANhDUE+6qf7lksTPyWZp42z0/dsoOdVZVoOpeHK5elT0xaDfXMNvK92aQXihxUqqVTRKzPHJte7b/KtWhb6U0tRfWVmK/1ZVfyup4U6WHFVTp3s5ab3VskPmXmX+NPcVnz9lNHXUqJWC2lwfsIfLe48fl3t27AOCl9EWjSd1zI5jWaFu6DdJYIKF0Ksb61i/tQomWkplIWKBfdjgHnO+jfrXXXd1sUYDTDLGtO1cmU5IjgJu0D6Y9AqIAAtUYHmCRtOqqUCELZEhpLSj/AY2WnufWXi/I96DEgkekEjA8SJ1uBBau+oSqEw222ygPOtoGSf8SAJIlD6xrgaRLzNLHHq8tlb+5waaG9L9l3GsdJUctWzVjPn3pZc1bdazgwRbJMxnW1fJ4RAJBOepzz/Wt+FrUeZsPQ/VfR/YaT4uh9QeI8+UsJQ3bUj3Mi/no0uxoa7je0fOLSy+PNWs9X4P+ewG5acnErT0Qqv7/f+y9CZRm6Vnf91bv1Ut1dXX3TE8vs2pGGxJCSIDAQiBiiIGAQYBYRIyd45PjJSQn5+TEcRbHsY8NBoHICYEkHBxbYiSBRRCrQMBhkSyBLCEBkQ6LpBlNz0x3dVV3VVd19VZLfr//c9+vvmqNWmgUfDyjut237v3ufdfnXe7zf5/lNcHN0E91S97ftS2x2KQRdx1YfNMrDrcf+fkL7au/4FB79ecf2hLmnX+w1H77j5fbP/y2O9uP/uLcFmBxA5Dwr37zYvsoTL3ziU3nt/vF90627/iKI6Mmk2n//T9Zad/6V6bbw799icWNaqBTAIP/7KuPtl/4/cX2Bx9FO4DDlv1rL5tqX/55SM+H48d+aY7FlJ0Ak13t1ylPb94vfu7+9uoXH2r/mjI8PgCbvbi8+xtIXu4HtHh8JmXctrEYCP65eLm52B77sz9pjzxxqV2BXdt98Hi7+/772xnsEAunwpOy0Hn54nmclzzZltb2tzseeEm770gRa4NF1+Un/6R94CNPslfagXb0oZe2l93NgulfwnFbYPFPkFj8y4cf5rtUTL4DUzsCVyBrV+tC3Q4iPyrjR5+fM73yyo9/VoUZ1F09Kav+xPOjmTSSAJ8J8+PDJwOXvE1sSLDnsovfSgd0g9hXRS2XcTyUCHT3smus9MvQqULgYW5+gixyvlmkRZa4i1ViUTtId6bOFbeAC+NYzqG8AgyBRYELvt+Ei1EzH21BRZiJMBSl1iKwcAVVhk3gZJ1dsXTl01LJbKgK5Qq5wGI3jOfG7n1tBWChKtQsRtwXURVRYnER1YcFGJHLnKpDXYeJWuXcwL5iNx1rH5vkHcxO3EgtkFjshXPVzaxGwzs4BRbXWSm9CuMs83gTkCFoclVYpk+vTPvCDKIy4SooHIv0cUXzBMbZd3Lq7UdgEa86lF3G2NX3qEDRR2y3ormrzF2iYzhXm/DjT55uqPbRRx/FyPYTrDKfi5657beHlW1dzMr0Ci7iuparRuGuQLvavwtVlkswdB9/fBZgcQHmzpVjjNY3bmJAPtVe8OA9GHCfIS42K5Sr2lAVFHsA7U7beW/Z7G/pnwEW7IgMA27Z5lilX8HV78QOVqhmjsZF6bFjx6i34KLiRH0KptOdpGXmNco1TRloy1OufNdiwK3dgv2tbCk0Koe+hqcs0m0Sg+5JjLgFova/3ge7+pC0MW0PmSmSypH+zHO7t/d9nw4lOAt6h5rHyB0PUQvYXWgnoaRKtT7Tl76qm7nruWpflkN6OFaMr+ewS9Dj3OwcaeBGFXscGWfHbspMXw2woN+s0hd1cbybIThF2+luWLe/Si7CkNP/0z/oK44B24T/NkLGQNXGuUZpZJ2O7U4Logy1pq7k3+cTwZvG0k+eZ/8IQIUqVUpGztx1Z7xA3X/PPTgAwAOX9SM9x13U4JgTIn3wN20QQ3XeS0Xtbm4wnlQD09vTCu2rREH1rNSFcl9GOtEN2Rfoi7qbVXogIBG4WTXbTimhkstJ7ZK4xs6EjhCJBeNfaZtAJGONem22p300jRp610JMMc81hwIsiHeDMWp7MkpjL7HOvIGvXyYlQAZjUSIrqdBDlKpRqj8FWNiVHLc8FwSeQVrxwLHp9ny8Qh3DHmSSNt3sZUPrVPfbbK/eCX3C/00m3mb1WR7mOXd1+GwIn2t/ns5QP0bJGhJajXpIRCU+G6WWCFVOaLP1sbnUOEmCQ6pcKjpvDTCeVi/D8KwnZ8x+Px5+9MxSbCa/+cPnpDUq71hCo2eGMV/+f9drvs1f/14O+19vr34dz9hnltGx4fl0VaHe/8R6e+jYjnZIYflneHRg8Xe/7lj7337hAvs5rbf/9lvuDLlMynno+376fHvOyb3ttV9+pP3A22a3AIvFK2vtV95/uf2VFxxsp5B68Jlrv/zvLrd3fXi5fQ/M/fPOMFY4BBa//UdsMAuP8w1fcri9gOeCFZ8dhGs7NLmzffOX4oacqyDhPHYf/x3Sk4OTxW8ILM4CHKYBF6/5ssPtjund7c2/w8IXgMb4DwAivualU5GK/OSvX2xHUNf6L7/xePL+TMq4DSxCss/BP9jX/tFvtd/4vY+2pQ0XqVpbXkQb5fTnt1e84gvbc+/Enf3i4+1jf/qR9pGzF9r8kx9rc8sz7UWv+Tvtax+QXIxf3n/4V3++vXvpUDu881pbbC9qr/0br2jHJ260xz/04bbxwOdhV8f3YJjHPhsi3xZY/NMf+IH2r96sKpSr14OqDx9e1WJKFYoPGbk7+YxPkk5I8AVhVMNADO+z4s9E4CphMf18EJ20nLxIKGkxu5pf8jQdEuoSjPFJXCZYw8I9XC1PGBA+4sZz1vFDFMafyTMqAlyVmnj4Lnf8WScs/1PWPXyEVavyw590yJt5iPcqBCQijGHFFVioimAeGkwG5BBXBkIbAVcnVelRv1qQoRqUq/YeARXWmbgylSau+ozAQsZSYLGXcwcMwlV24L4IsDiHy1ntK+YASQKLK9RHg2BXw1dggpZhatdgAndw3YPB6YEVvA9dvdz2YWexF4897q+xH2ZBJkIGxLjX8aykC1TVf2jA0FpJxQHrQFhVwkqqUuBAFSilFXccOxrDY/XwNTK2vYtRtI9UG9j+HVhYz7Qn9Z+I6g/63TBEeig6+8QT7c8feRSAcTYbikkbV473qa8vHWFQi561m/cRVp1n2BdBz03aLTwxf7k9dg63tbOgeBjhXdT1vjN3tBc+eG/2K1AqUO1nn/AjQMFod4FE9dt6pNqPIEHPR3oDWsCNqrsnzy+yAg2oO4Tk4ww6+qcxAj7m/gcwqYIHGdR1jbqh4VUM6G8I1AQL0MQVbYEFlW5HjsygAnUobW/buufFBOVfFdTRj2S4lVbsQwKgFCCr+gO4Lsa6GHH7j30xNK9emXrUGCoGwD7hSrgG14sYcF84/2R7/OwnWNGfj5GxBIjaG2PF/ulK/iGlJdzvoE4eMseXUVe7xDnHeV5a4G1pCSN/93qgmVJmN8ETWOiQwL0pNqAhHDLgtJhVd3zX/e/kpGpQAmv7hwxvjXHrkQZJrvVHUOH4duxnv5bMOQJXaWDbFShXzUxXw/OAnVncFs+yX4WqBtLvBK5lH2BjQiUWd+AFaj9SIg3klchFQmG7OXY5008ZD6O5g76hy9lreoa6spQ9MVRxWrYtKVOpSe6INErJ2yKgYgGQrlG8aUsc+791c8FAOwslL7qhdp5y0SR9j/ZPu1kOylMexiyPz+tZzRH0E9LkMWOX+YfwaWPG0HVARepgX5CuzBnrgG6QNwVgvnE+9KQ93MeCTgawoN/yTF9gG4xJpZhHMZC//+h0e0iPUIDBQy4smGTahxsOWiRX/wwzqEMpbZJnlFEGv9q02ikdxZemU8lUXCNyl0e59zfHEGbsJnnUy/53PEJ/xpX8c3jtQZJtJWq5RuXu741A+PGflUg97/eJ29P34VDOqkHFzt8hoVE1hqDj38ee5viz0IxI3/ksBBaPsmfrCXDu03FkNA4s3vsnV9rPvWex/c3/aKY993QBgj985Gp7+Lcutb/ztWykiTeoW4FFp/X4VUb+n/8MqlMw+l/x4pI6dDWjcbDhAtz//PC5dghg8F98A3PIUIEPIrl4y+9e2gJMBBaqQ30v4aZRh/JQ9en//NX5gJTvfvXMaJy87d8utPf96Ur7X153FyrQ4z1ls5S3K+O2V6hNOn3u3M223/mJn2jvbS9tr/6rf6V9wamJ9ti7f6794h+ttnu+6Kvaf/xFp9v6hT9rH/nwR9rHrzNPLzzaHvnT1h78jr/Xvj7A4lq7dPaP26//3/+u3fl3v6d90dqH28/88O+2+/7h97aXL763ve2Xn2gv/uavb8+bYQH8qbvkZ0Tq2wKLf/b6H2xvfPPD8EWuHso0FqjwA+zpR6QmUybmPukyeWfl2/CJQykJ5HtX2QpUDB9SP6Z8iPOcMH44PcKEkk6Mxck7ecEQVH2Hv8N71aFi98H7vrJZHzZAgfnxEdaoMQygZeQ0m2SVe9Lz/yg90xljesyfd6MPJvemMAIWlp+CC3JkxA8AIGqVXc866pljtClDAXAwe8sjU2DZ/G3eltvVUleq1eXf56o1K+0CixswIcuIZQMsMFg+H3AB41h4hLJtYMh8vV0AUMgICi7cMG//0sU2CbDYv4oPf5j0aRhAXc3uRk1CO5GrxLmMrry7Rmt0KkQso3NVstigLqobqkK4ei28arW6DaN4GDsHmVCZ0ujlwzwV7VHZgXEMyEiMUDl30rdOmcPyinMNZtyNzB49e7Z97NHH2CjtPIzitfQDmdDYhcCQJq/BRkQGWJsOV9n3AH6W2JV8lv0PHsNDlHWxDs+7/wzSitPtNJuhuXO2koHiBGSM7D+2YTGAaQN+BVTITMKMa/gcYIFR+RzA4ur1dUDBdDsFk3qKNGfIP4bktJUqSzakkielFq50C9Rk7JdIx9V9GVpBxT5Uquw3qrvtBZBpuL8Bc+jYsH8IPvcCplRJSp+HTtJMoBVbH8KFhtaA5zkgsfH91xlS+5fGxa7kL2JsPTcLsPjEozDeswEW2sDQ/UijJGXTtKd7d9iWOwDVHgKLS0ptAFizqE4pvVkAVKxgR+DKunlaxt2Udw/9QKcEDO54w1oHeLjytw+1IUHhgQPaGCiBKqld6jPUyTQEEB1I5DrUzd6ThQV+xw6F3wJC6yddr6DG5waJF3FVq9raNcaGgPMOpEruVXHm5MmAQGlfoIK2Ii2Z/w4sIVvGYuYHjPPLKxRSAEBCtaX9Qbe6SCe4XqVd+0KFUobu0UnVKyUajm8pWPVhwQN6Cirsy/Zp50RpF3DDOPTeLpk29J4z73lnnzQPN1R0zDoKbWfn0ORNfi4QCE4ErwETjG89yk0wDjd45jlB/u6sDbprawEdqkcB8ukHUcFkHJ1k0eB+DLcfmDmUTfHiZpb8bIrR3DcCFpaT/0MXNAw/R38M7/88tGy5qWcJV6F5zK+xyJE2+NsYdcl9/bn1QaU0CpCf/Nly5S3pJWbyqTjSsMJtpplnJjYWLuXrGYw/T7j6k3KmjkOSJmE2wzNDfSbHd37Ls09igd110yxiE5b+xSkyDiy0ffinbz3XHmT1vxtx//gvzzHuN9p/9ddr9f9TAQvjnkOneAk7ihXuf/bdC+0rXnQwdhmWpgOLW5n27wOAHMcmQ3Wofjxy/kb78V+Zi9rUFz4Hr2scAgvz6OXwmTYT3/+2rQDG57/5oaX2a6hL/QMkLx2E+PzpltG428eznAIbT7Tf+omfbB+afGV79au/pL3wBOrxH/il9vb3LbejL/mK9rWvuLvtq+mNCWiu/el73tne+RsX2z0jYHG1XXzsQ+1XfvLD7Tn/zevaS67/aXv763+xzfy972lH3vnm9oHnfFv77pffBQ/7dEbpJ9P+tsDi+37o9e2Nb3lz5towjnwUZXBqdXqYlHtlmEi9leHxo1pGzKpNVUFlerr4PqtzMAcyCAEWfIxluAiSNDqg8OqHOMyH+Tq555BJrY93X90MiEn5akXQsJbIfEuqUCuURk+Rh/KaZ3156pLyw+jEdztX61p5VDmoXD56xlNiIagwDxnaAhZ4NIKB76fMmuoQMkYBVazmyhQUMwCtBGswWQEWggsYgOjZw4TugElYIx7SX2wq1tjHYq2dg3m6iNRCk+/dMA17Kc8iTNA5mOpzMIHX0KvfySrrAYDFIYy3D6MadJQVyKOoEU3DPO6EyZbpXUZacBHGUdeYSi0EFntZpdwPg6i04kCABSu81L8Dy9hVsBqsCpQr7Bqcx31mmMSyA8hmdIlje0lT26z6RNpPmvJPYCUdtIs4d362PfrY2ahEzev2FqbR/qbKiaojewBGSoJKV13Dbj1G6YUK1SLCraBK8iRuZzUy1j7kwXvPoAZzR/bB2IEaU5hSGa982ixUAYswADy1R9SO2iuxr9DoWQNnV8EDLPhgHABYnEZP35Xwg4CdKRhYjboPYIgtLWQk1sKMlr2FBvHaasjgd6NsvT/Z3VR3kxnfoP+7+ixdNCjuBv72GZ/1c6TiN/RV6ecYC1Nq6e3LGT8FpEeMJypaSwu440UN6smzn8BGYhZRaWLUxgAAQABJREFUPK6IKW/imDdpahyfzesANtnTgVI6Mei5axZAcZ4zG+EBKq5i7C8T3vN2H4icgCfRimDJ99LUcbOHPqXRsrYWqrR1BlsmW6bWFfzaRR4mnPD2t7SP9KR8jhP+k9xmn5GZd1frZcCEe5gotVBdRnXDu+68g3Y6EYPtY3iv0VBewB4pCcmYpiCw18H6pl/Ctrth300Atwb5qn0ILgRYVwTtgAp3GVdqYf9UWlK7sevmFbVCQJyud53b+rxl3ayjRusayUdiQx0zJwnOKIt1c3xVWw/r3zwTzJuHGx6WIbizmf8AfQOw0Mjde2MJIAIs6GPZUZvfSmM15oYwbQJp2AbnGu8FGjsydnexgIDRNu1yL9637gVcnGSuOMgY2k1c5wT7Wv7zm4LarHWk4I5tm6uu1SeqDrZhhSVCGnCIZtCx3xWuEr09sEhOlQh/k/x4OibBby+O7hz5QYb5b3nq/Wb+FqaO4U1PeHg4pOOv1JHf/VGiSnfejZfDsIYxr+F5p4+vPuVhHNJ6NgILzRWe7iroOLCQdj/9roX2wY+uxIh7hQWfN7z9QvtGVJdewaZxHk8FLN6BKtS7P4Ikn8UODb9tyicAGX8RYPH9GIUfm9oKLB7FMPvHADTfgj3Gy24DLBZAVDEqH5OMWMbf/MPl9muoXqnSpbG3x2dTxiSw/edZToG19vjv/Uz7pQ9caUfOPNDuBkefRToxu/u+9sWv/LL20nvG7Y6eCljAa8090t7/C+9oZ0+/tN236/H2/vettpd81Y72oXcfb1/3n7+qndkL7/z/ExVvDyx++PXtTQCLWlWUwS7mOh9BClCT6lASBqtzrIcfo3xUwzTUwHGSLWABmOCjWqCCqx96GHNPw/jP9M2zPtDDvcwqz/skXSuC/OajnHs+eonHtYAJJCK8acqcyGzJzPYPg/n4nGzzyN/5GKQelad2HNbZugiowmSzAqnbRj+fquKE8eFXVKGUWMAAy8C7B4Mr61nRh3nx6IAqe1CQnSuGqtMILGTIVbGQwXT1V9/7O7jX+NI10CWMt+fYKG8WgHERES07WFAemCnKKLB4DGDxCLv+arC7ExWOQ0grZtautjt3brST2CUcQWKxn7z0zy8zvwygWCCOeuHqeFsjjdeVVExp38BVWxFpqmqYTH3f+E4jdI/QG/qkxQmXsEN9BAYx1LZNqWuM/XkmkOLncLD6DOPmjtBPPHmu/clHPxZbizlUW4yT1V7AheBFw/NIMCiHbku756iDgIzdMG56x5EZE8SdOH4sEg116mPXMoCLAAvKScFGDL19FjKGqXSDvWVBBQz1RWjpTtAXL6NnD7CYmjqKGtRJVsNn2k72BpmkPIdgWo/foftSABuMmkDhBkyp6VyFthp02+dUe4rkhPYV5AgebXMHn/0zIMpVf+qpSpz1tVyWU5p2YMGv0M2/PjeuR8YN9/bFrKZDB/uYAPIK6ljaWcyee7zNz12gvdnzhDbI+IMRXof+IzfB0NV+6GHZLqA6dX5uHtA2F6ClEbcG6aoU9jwFKapBWfZevhSr/lBQxgb9XKZaVTylFlETtH/RtoJH3UXbBhlr6U+MFesiA00ZvboxXBh4GP7lFT1VlYtW30svvT1p/3M3UooTgItjR7GFQf1J6Y+SIAoS2iiFUMVJV7LWIV64oiIFTVY1egcQogqX/T4AZuapHYUSi4BwxoqSCZ8LDCLpgNYFNpRmsN8FdSgVqALESvbsr9bVdrNPRGIRGlW9u2RG2ktfAZAAKm5oaUcNtROP9zVfluRGMUbmMOa9qDg5X9CGTBFRdVplIWEDGkwgXdOge9U+SFtog+KYPnZosp0CWD6EfcUJ7Cym6dcFKqSY/cwSUSau1b7DA5752/qM90VC2iETNxHzp+aAzd+bd0MPzpwwAhajvHjL/UCmzUj9bvxFJVR5OzaGMF6tg6UeD57X/QGB8i0zkMcocvVzE0gdRy8q2OjvEN4wPZM+PhLG+KPAW28M51F1B1h862u3BvhL/OX3KGU2f8t+y5E6Uz7nVc+na2OhpO3pMiy3AosuLdCIe3FlHYPqlfbfv/YE36oq/63A4kMfuxpbh6/F2PqVL9SFdkkG/tFPPfkfDLD4bMt4S7Nt/3xWUmCtXT77vvbOX3ovfN5N5nC2L7vBYtAXfEn7ii99QTu1xYDpqYAFU9PN5Tb/sfe0X/v1j7T5Cexln/tA2/GRD7R9X/ZN7d5daLs0FplO39fuwqvmIA942pS8PbBAYvGmtwIsGI0yPwEYTpLDJJTJmKz7lOTz/OM6snsgnocTqB/L2FYALDqTHWCRd8MkbnpJp64yG3UWgOjvOtjZwSqE93nuHE2hRqCjl5MPgquEMl7920Bu+e1HuYODulaQ6HeTrnYXGqqXehS1g0kiw6wGuslUuVOEOaCe+2CQ3MhLYHEERkeVoWzCBfOl5CMTtBM1p79TbuJZ9gJH1oW8CC+oiFtMdaVhBvBgx/4V7P/AucC9jvN6nedggB6BkX2MjeZuoMpzgJXXY6hAHVm/3qZhgqdg6JRs7MRV6k2YXT0ZrcKs3JD5RM1C40/1zFXnOgDToeGtXmyUEsjUuuKrLYXqMjKIAgaZHNs6DBGfjdSBusjIyzQLIKxLSO6fsaOHtV0NoOehixcvog71KPYWZcitC0+PnfTwrPoCGFQpkxkSYOwbVsGVXqhytlsGFQYzOx0jEXAvhj2AM8uzE2ZrJ+CCAkE0ymq78c+PXdFdYKHxN959UB0SVMxzzi1ouAzwwgPD9JFjBSywMYF4KbfSlKPHjrcjGAYfRHohrdwXQzsNVapkYlehsaBiAlqoguQK+W7aVD17mdM8o9zdVa/lHQELymcXlp6RFFLu8SPMSx4M/ZvxJYAO40m+SktWUIuzThfOP9EuCixgqk00q928VxXLXcD3p21LQuVKvKpGs/NzARbn8QS1SN9SQqAbXvtxGCL7sf13qJt0HRUxTe4fwlBXy++O0wKykj7RJ2Vw1eXn3Wb/rzo6RqyL/VMQJGMtiLjOKbPNz/QLvXQdw+bm+NGZduexo7GncAM8JTDSPcCdseRYs9wxbL+KLYxtwzjo0ldZYTe9ixcswKXukJVO6FZX6Z4SEhl9+00OaKh3I1WnBMeq8ClhENg597m/S1z50g8dSzHIT/8b5h3bitN5KeOYRK155kjSiLSCdAUx1n9zbnJzO+fSmk8D8oi5IYCgX3mu09dDP56vkecEYxfDrbYOwF31d/pbqaedYoftezHWfoh9K45q12RbDGXp5bK+m8DCXxx2TC8pdW75M9AGunhUiNxmvOSpfWaI6+/xPpw4eUeqWyIbsKdeaSfVsdt6e0tpkoZ/KqB5VbIVLn149JZQPU+C53aoh3lZZtvelHq2PbgPRve+H4tn3F6ZTxlmLPyzEViEBk/zz63AwmRe/7Oz8dp0lW/h598/2V7zpdOj1G8FFr/4+2Worceo7qa2qyj9hyKx+GzLOKr89s2zlwKr59u73/TG9vsbz2+vfPUr20tOtvbIu3++veOP17GxeHX7amwsNreOfGpgsUkcvj1X59uf/cbb2i+vf0n7qpvvar/6cbYt4Fs79eK/3l7zlc9px/fDs25G+Izvbgssvv+HfjASi1IHkjkoJnj0YWBCrA+OE6sfgzoDBPhAGT7MI8XyQ+eHvZgEvbGgNuBvngsu/MCOT8hb0pIphjlxVTNsob/5QMq8huEnr3wqnKBdHuEwvuEzqXP1lR8AL/7NTz/OlsuTJ1kR5F5FDhmdvpN2bDj8IIcv5Q/3qhhEh9lMuC9mAp1ymN4jMGmHYTSnON3BWeZGWlRLGb4AhHT1WVfLsFzmGwNnGOldqEJpbL0DZuE6hF4CTWjIvUgBUSsNuLkO03EepkZgcYEVZfeyOIyu+DF2oD4As7yHzfJ2uEM3zz13eAIKsHEuZou62gbaoQgB9QQT+4ow8DDxMEWu/E8CNgIqKLAgRObTOsn0K40ooORvysvzeIni6qfcdjVOqX7whHrLYCsJkuFTl/wKtiGP4yLtzz/+CODisXYBZlbmyiMr3oTNRmvQQ9UST70NHWCVtfa7wMhbQMSpXUv2DIF+Mum7dlqmYu7NO32Lckn+NB/0ULqwzEr1pQALNoLDy5CG25evwMRCmaNH72inULFxd/E1gFukEfSfKQyf7zpRBsIHWCE3bfu23rZUqZFZpfo5lK5FggITaH+137narypR37ci0iuBCDFIKunZx0NjHwyHNE0fTuLFZLoKXjsv81tmnLzdOdqdt+dmzwEsZvEKxcoE462YYA2tmUwAwLXTdrm/dTVfZwCz2JjMzmP8rdE2qkBu/iZY6cAi41UiVkFTZou3WcpeWp5JaM7sbk/frhV9pXTOEQNAtb5D7IxJ6lbSzWLCi5l2noBPhmaHcY9657Fj2L2ciF1F9xg2iRqhdHROycH4TP5EtN/eVNWJ9nZn7ag8qW4kE0+7aZei6pNASheySwBc7ZF8JxOfzTgZI5kLaYHQmfiRWJCGYayT5Sv7Evsj7Uud66j5xjaoxRUlEdQpcyDzoWqOAij6/g3azzBKem3stDeJqOKkRKL6gKBCaYUb3aFex/hz/shCimEFCoxfd99eZ1xosxVgwTjSJfC9SCoe0raCfSsOC85tA8pjP/ZIe3A7UDLP6sXY+zwYC2EDcVSIuvb+auP1tA1Tz6svVxxi5X9dfTaeefpcHt7y55Z0Ey0F8E+Vx7h5xBPrVcUc8ubFqAbDTdziDtlYZkP6ahQu6Qz1HB7mUgkPMQ1Uufa8+4stdRnibAOLTp26PhWw0FvTr+DZyeN7/5Pjo70i/H0rsPj9P1uJPcUXPbS/fTkSCw2s3/57LBxgwP2qv4CNxb8PVajPtozWe/t4llPg0r9tb/xxpAuv+pr2qpc/2O7Qx8qF97e3/+wH2/zJz29/9ete1s7wKa7j9sBiY5WF3I/9Tvs371huX/6dp9qv/uM/bF/yj/9We8nN32o/9r9faF/597++vQDPmqWX0tP8zK5/IWAh8+fpx9SVVmdnJ0U/iE6zTtJOvD3cFmNq4jkdG7avxmXDOhjZGG3zQe3qUFsmWlLuaQoiopZkHkM+liWroHzkXfXNB5A4ZZhZs3ziU1avKfSQJpfhg1YrhPkI8/GWwc5qIFej7KKuJbGw/iaUBExwABVypsVYycjozlUVoiOslh5WB5+V81K1UPogY1t0dAXXe0sZsAUDUeAChp93AUyktwsVDZnOHay8s/1dW16bwM3sBv6JcdcKA3ODci7A8DyOOtNjMEFKIfZTj5mJdbxBYQR9ZbHdWLqE96fLbSd7W+wBYByAVvuh2T5XxskDIgaL3YCB2UF6qkFMUla9+rjiquFrbX4H00lbyXSWGgl5UDa9I2XTM9Jx5VoGOJ5/YHS8DyGHelpX+0tsHgwr0yMdTJfV40vYAzzy2Nn2UbxEPXL2iWw4pjqPpLdPWV5VnUqNQ/Uoy1j6+32/C1WjLK8G35ZP25WSWig1KslRgA35q9M/gQ7COnVS/WVxcQFbAlShFpFUcM5fxv4EFbRduydRrzkZN7uHkI5onO0u1roiVUpyz5kz7Z6772nHjt9J2fRC5bYi6uq7io2XKJhEwYbdp6QR0qXaWSPuGD9Tt6gVQUf7UgAmCVHt9Ik+FkgiRx9/XjO+oOG4QXABC1R0YJ6XcTN7CVChrYX7WcQewDLR5hP0F/cG0ejZVXUlEtqqXIYec0ptsK+Yw9Ykm77xTkY/rnnJNwxxL5CV5lk/rWtv+zQgtbe/O47Sx6lYtT+/Hb22xZCWID+B+Z05YfgpDWxPpVHH8WB0EpWnM4C9E6ijTR8+EtsVx1vZiSiFQmWJtrUPWVcTNX/vBRYr2CKphqfnqwWAU4yvebdOW+nN6zJ2Mp4BFcQWCCvRU8pif4y6FvTok6jz2wQLALXbNsACkCu4EJA7X/VKSTclWdJSKYyg3mukEwILyuAc6XgxTa9RueptrUyB/PXoxGAbXMwyPnUzq2co3hvH5lDCujOqUNhXILG4kfAl3ZthrDzAvhXP47x3Crsw60UjBFiQCnebbTj8tA1zjG42g3zS/E0BepuOBR89M53efytR/1aM/O2R86xSELiN+hXve7qqEvXgKQd9xXc19w+hJMgoFLf9cfSQ+O0xPKsfW/8WrNgaJOO0p5P2qTi9LP6q1z0QDz75ZUXi77PRxmJUuadx84afu4CUc2LLhnfLbGan7cJJNrPTDe34oTRjCleuf/trytja7vI2DLU/8OcraYf9bLb3114+1d71/y63F94zmX0xjO9eGBpV/6PvPMF3nPE1HD/4ttlsqPc9eKLqx2MXbrQfxVj72195pL3kAaSBHP8HxtwaX+sVqh9ulPfPcIXrnhev+rzNPS86MNJdrbuEf7Zl7PltX5/FFFh4f3v4//q91l7+qvaqL35eO8lUf+Ox97S3/+JH2sq9L2tf/TVf0E76iclxG2CB1sr1hbPtPW//2Tb38r/dvvnYu9s/ecNy+67/6Rvb/Ts/2P7l//iH7QX/9WvaF9zJBr49uadx7d9EFyzHD74tGxvfF4nFw2EEipG/FViEBWCeHGP4+eC5wuzHvTOOTq1+6AQQqn8otSiVjfpo+oF2NbKAyjARM/lupqvEYjiTPh9AV7xhkCsvWBPeO0DzEfaGPDsTUx+XTcaFoPnoFtipshkvQMeyDJ8C89RAWmBjGhGVG5dnYZZyrXuZV1fQD8KcHQFQHAZclKcd9NVg0qPeRBqCCv5UeuQVugAsbAgZHstsWJlnN2bbA+O+E4bgGhvCCSwEFVfWJzDoxrMTZb4I03QeRvCCrjBhTnbCKKrytHYNoMGOyzcX5nV4zP4W+DxeZ9dnaHYA5lwQFAZWQMApxfxIui+IBtz7qYfqT26cpsqPzLhqIjLMuldVP17G3TPqJqqdCFY6I889Caeedqx87L3hGKeB9fVQpUT1oQuskJ99AskFXqLOnb8Qz0S6NzWU7WA/jPtOyt+lFqqaHIThn4JRKmDRN+4rr1WqlNkfC9jRZ6BBdse2LVnBvgatFmUwOd0BfAFAsbgsY4k6FsqM+9kN/dRdJ7HbYONC8l9GnegSqlt6C5IxP3P6dHvgvvvaPffez34VU6GBQMLVbxlYzwAL8ivg0YEFNJS+sbkQkBXYso4jYDH0lerD0q7oJT3D2OfKOHT82Ic5A5Spl+OsbAYw0meTPIHFJewmFlH1MZyHgF2JhWDMtnM3afe5cDO9eeihvYtnrciXXUHyNj/q7r19J4f3nB6WMiXlT1cX9JXl9+z140eF5eq9hy6djZ2xwHiR5tonHBSwU9aZmekY5uv29w5sKQ4jNdrPOz2e2Rf1AmYKSsLKpghpC4b0bn4oTVTzEjwplXGfjqg8DQBWlkLIr0ODJcbUEn0gKmsUTVsj3TarumUdVMvKfNbHLnFVm4wtkCpQ2CppT2J/6+02VDHA4iai5wALyhQ1L8Ceafa9KWyjSPls04HWTm2OhmyEpxSONlun3huc6/wWsErnLgUNiKZt1+lj6+xtcRNaUqA4Cjh+cLI9iLTiBexdcS/uZjXaztadEq8f1ST9V66j19yk2dKCPrX9ufAw0fjRo6df9MobJCnVn+pD1Y+MueVdWrI/IfEh/cTsj/mRvuVDbgwyOoY8B5gxelwFreTGHt76oCpoAPLqfbvqWLHSn71N0TZzTi0SkLYYrkOgok8ve+INaXHZBhZFi/5XvkBaMvVvOXQFW9/3LY/DRzxVeJn+FbygTB9iLBLFdGsOqvg2kZ7X3G17/PhU+d/EIn33mEW64TyUvo4fur/WQ974U0Na/lvzerplHM9v+/7ZSoH59r63Ptzes3CyvfALP789dAdezj70O+29n9jbHnzFV7ZXv/TUmIThNsBijQ1vH3tfe+v7ptvrvvVFbKD84fbw//CbbeZ7vqm9+OZ721vfua993d98Nd4B+Z5+FqT8NMDiB9qb3lzG2yNgwQh3YDjJ+rHzyEBmAi/GrxiBMK2EdfAathtuuzoXbyqobTjhBgiQYD6i/k6KpM9IdDB2JsR0IrXgw5iVaz+OfLBLnYYSJB8nh/oIm4xxLFOkLEm3BjiPq/xMBhqGjhiy1Mky+HFi4iGOKlhOFqYPP89T/jh5ONPxzA+3K/W6HdXYWWAxzYd8CkZHpjwqLryTacxHyDie/JOOqghlVZXVVRloV/CzIg9TH+9Bk7idBVisbAAsNnA9q7SCuHqKWoEhmWNVXFAxj/67+vLS8TppqoazCqDYuHyp7eG6F+Z5EtWogyxoCiy0BXH1vO8Gbp2ccndzPYDKxAFBBWf2NqC8dhT1zl2F38Buw5VYdeRl0HcIhGBw4i5VKYsnjI50iXSG+KGVFSYP73PkVnrwyD4CYNFl7BxM+6NnH8f97JPtSTxGXcYbkeohtlP6QYABee4VICG1UCUKJk5pggBDta3sgTGUX5WyAj+CCsFPrdjCtUXP/jKMtm5v59lczY3OBBVXMBC+hrRiH25/XQ0/cccd1JG9GmBWZbo18NYtravbujV9zgP3t+c97/ltBtWcSQxlIRKAwp23MeRGR19Gz3aPS9eAHGhDW+sBrFbZARP87jZD1tPTOB792u9r/NUY9H4TWGyCdoGqe1msYNB/BZCpjcU8dhOXLs2n79mv99B2riqosmcfVQqjm9wFaFISC4AFnro08Jcpz8G42Ry7NGqYXt9QjrRxWjl/LL2P8tiX1onfqc9wn2fW09+e9kXKIkOu4bN9TYnUUSQrR7Gn0NuTdiF65fIUAMedrBIpkjEby+e4iioYEhulUe6mrrMCpS8CCyU3epTS1kjVor7jfIAF0pkrqH7ZDwS98hvON2406Zwiza8AUFRZyoIAdfK9gD3G2oIKyqXTgUjlCG//NZ7UiLRCSQX9xzko6k/keY154JrgAlorgaIiRT3jE0/LphhkAyI2UH2aGEDFhgCDucr50zwEshLC8beDeUhpxRrjco3+P0EdBOV3Mlaee5TNJAEWd+MN6gDlD19lGc3Ww8a65cir/p53abNE2Nr+41FT7zzYfNrvfFf/Krv0jRRhLJPh1kslw9+eADdJ33Lf5ki6/X3y7D+43i5u+mbR1RijoBbBd1se1s/0cvMYzno69td+TtRRWr7i97Nxg7yxWm/fblNgmwJPlwJXH2vvfcevtHd98GPtAuvIk3c+v73i1V/ZvuRFdzc8Io8dF9vHP/Cu9rtI6k59w3/avuqesVe33vLtXvrAT7Uf/bk/xinQTHvpa/5W+/oX4flSndjP4rgtsPjnrwdYvAWJBR/SDixUb3H6dsIUWJi9k2hn4iNFgFlRkmA833bGXU81UXsaQEWlUZNvqSPVvTOuwML3pu0EXB6iCrSUfjYqCebDxzD605aJU9BiWt4XKKDsJmBBOepSHyLL5QdeZrznX6tLxiYsgUfMXS+TCXAfplnGmToGWFAOgYW7Vh+CGToEUyFjHh1/ywijFGoQXrpk1Z6kLIPMuoah12GEXPGcDCOF3jteh3bLpKIKtQSwWFrfgbQCmwTyVxVqBeZhnpXV86hrzOIGdIVV16vQ9jp0W0U6sQM1nN2owkwCLPatXG6TuJ/dDyhw86uDWVF1ZR/mB4aj2hTGnbsDMCGHYEYORM3JTfJgUGGQVf+xrLg/ihRD2lffKAlL1I9UQYJJ1WhdcKFalDSSjn5EPY1TFJbKRWMecaj7X6o4F7F1eBxPUZ84+2Q7y/USexW434YJZCWb/iVAkBmUmRMo7cfjkABDu5CsFlM3y6Q6mXVUtct3qvwIfFexldAl7AU8H81dvBxphaDCPQmu44FrB7YZhwEVR2Bmp1EJixQChtQVf5lvmXYZP20vBBYv+rwXxkvUQSQcjhONuK8LLAhnOwtwY6QNGIrNBzTZrT0IZZMmdTKWhhWu6vtDx5U8HGFkQssaf6br6ZHxxH2togswWJ0nb3cEv4oRt9KKS3iIElyQREB5+uzQTx0dSqTcnXse+mu0fQFpxTy0jzoXaY+YPtrBfM0zq+peOW89LH2NRq91+KyPqwBPgIQFqr40tGvU3PBahFTi2FGMs7lOs+N67RJ+iPmoDLPdpV1QoQTTtNzo0HZyYhPU2b7L9H/35NBlbndnHPUoQKJjP8w3Y1eg7EKFJXUsXQNMOS4nCBNgQRtJL/uftNVLlGGcc/TopqRCl7oHkQQIKuxvjhHrnb5DWlHPGuYc9wOJ5yuukVQw/gUVql6pCqXrXiMXvbiFRuv0K13GZmdtVPToQMOGdzy3TQjvHCZIz/zpHKW0gn62BvhYpfzSS5XNU3iEehAbC1WhTh0ECGWO6q1UbbXZav13XcebegtzPfbCNvYwxd4H8oAa1RtfbL5JeYcXhhi9ubVIlUj6TL/1mv43lr/PRmXz/ShT3wzhuaY0Y/G29OOhDvbPzecWaFTQLIpUelvzI4f6f2vaiUqupmmY4fD3NrDo1Ni+blNgmwLPVAp8WmDxRoBFGHQ+OlHRkANkYnSu7BOtfEGkFbzLSuMQtk/qfrxjNMnHsjPxTqfG72dWQPObFz390aRbwKX07GHgWXVTHUaA4Q7FAoukR1TzqTRV81DaILCQkTdd/9QhU2RZAiwoVxi0sfyJEQZC+w0ZntTF+D0d0tzgeSQYPLcMMdqkXBpwH3SllQ+6zIb7E6iK05kpyxFAQlEsd7mVlJllMzXCyZRMASomWY3diTqUutMBFuzCvQKwULda+wqBxZOAirPYAzyBsakebAQc0b3Gm5Lrl3vXMOK+CqBYWQBcLLYDq9faNMDiCKv6uo/dTxkFP4I+GVJVg/YBZA4hcXEzOlc2pY0M5zJMmp6TVPHRE45MnOV3hVa1r4PsK+G9oKLc7GpXQvsMdKfIuVdqEKAFDbW36Ax1dPcthyvNqORoc3H+wlzAhZKLC+j6631Hps5D8OrpSvFe+oKqaKWGohpXAQnLr1RGKZCMnpIVmT2Pa2Gir8QL1CLqT6q91Aq2a8MwkdDmIAbZSp7Mw3JdA1i42i1osP6CBXciv+fM6fbC5z7UTp0+02aOHg+AkJl0R24ZXPuZhyBCMBGPX9BFt8LVP6pMkkrGNX0kMcL21N1AR+lpHxfYb/bbGksC5K6ek8ENHV21VxVI+wo9RF26NBfvSHZl915xp2rpYp8sYLEU4/knob3AQimOm7SZX5hFy0Fb9rErsLAc+c0LSzwUNcMlP1Iln9e40gOZ4Et1yT6vKPWznWL0rH0C5TqGBEgJhZIKAWE2nIR+ASTJhPwcn/yzf6nuFKBOvVeQFK1Q9+xDAVhyLwptJkIz+jkNONCPW2jRVRzFuNo5dJXNnSRML479QaQV1FWAIACV1h66QC6pWXkpK2kebUoZ7a8CGTf0U6VMKYfxNtVBAQKkuQp9AzSGvLu9mO3kqYH2BmNnHbexG3swyEa1SXfUqkUJOqJSCv1NS/CfucnxB7026HcdWKi+OA0Auh+PULqZfYA9LI4xdlSDzEwqIUfH+D0PQ3OuozBp2PqdoBXe8o4fI6a+x/clQXs4+04eDJFsT48w3pVkvTH+KG8eDekZutKoYJ/0t0fz2l8ON+NG2r7ako4JW5Ye359DCnmV3/Uo8SjPlue+Gitv6pM4lWaPniR59Lpvefa5mx3VcftmmwLbFPicoMCnBRaRWMA8dSZAJrBP5k6YMgoy1THYllmQUfBjJnM0TLEy+35EdTXbJQOduqbh/D4CFtxnIuah7F3mZPMgOb00yUC6aqh3lpKiUB7yLQmHcYvBMaLlqHLJvJqjk/mmBMUPfp2ADO4tg2Wx5L2+fVW+mF8TMQ3LxoecX7VKWF+oUi3BhgFm9oDMNgBBlRxtFFwhD0NIHDMJM5R64fpT9QcYVYGFEosOLPbC3O90tREG4oo2Fkgtrq5j7EUYpRJLMHuPx83sUjuLF6NrqHi4YjkB87FTJhpwsQdD7oPYV0wKLK6wG/f15TZNGx2DsZgZgIVl68BCZkrAprRC43OBQl+hVXVEd6Wu1us15wpMmh1II9VpPPTopWeS8sYjE6uqto8MYAjmX/sHZXfVXm9IWXWmrAUuZAxhUGHW48KT1WRX+i+T17nZC+2Jc7PtHIzuJfOFsReMySB62Dc0xO4eh7rtRUAFzKoMqypboxVpItjeV1GDESQsKe1B7UV1H13vymDqqtSdsmVmBRX2jKiswRQKqszbLm66R9ivxI3ZnvvA/e2+e+9rJ7DHmIR29kHDKjWQEbVfu1pcUgulFNoOFPD0Ph2DvwEW1suO1g9u/W0f8jCtTwksyKuDC9WxClhcRqVsISpBiwC265RpA1rrFjXG+ZTL9hE0XkbV6/yF2r9iDmN2d7Vew6jYPC1b2nRUDp5DC8e37edR4AHQSBjPnSz3O34KQNBO0FNpY0keudoXqL9X2+wQmx8KKuxXR44c5Zyhb02P0rBtPFQvWwXs2ZbdGFqbiqh/CYBpW/ts7EYAARlj0CY0tK/R1sZVOmC732DwWIOMU+rqP2muQwOBRXoBdZW25Q5Wt8PMFfRzXR7rncyNG5WWOW/YUjL5gooCsUhQ4mUKyYXAgjwDyKwMeZiv+TtH1pxZ82VmJccR9d5gLljXdSzAwh22s4jAO0tKC2Q+qlqQkB1UY27adp12Vl1KiYXSCu0rng+gEFjcjUcoba+Upqd3UffeF7nZcuQ95cwcnTc84TcPhijGHdLJXf0pYGHYsYeGq+CU3PjDD5+PBdzMyxdDAj3s8NunW8Lxe8tBgBRxPP8hu08PLEgpdbSKlrNSTlLJuH6P558+NhRg/Hni97r1smxWG73nbWAxkG37sk2BbQo8QylwW2AxbrztyqyrdQEMTrLDrBrmgeel+sTH1zB5X7NmPpzM6F5djXOS7fPoaF4lfAcWYe4ND0ET1sBmR5g9fMBlSPRoFNsKGORby9NnfT8WARaEqY98EkkzxUhbZmRgLLJySNl63n7UTF8GSHUb7wvEyCzUx18PUhpZ3pSpAjBZ1hh7E0dgcQimyBX/skUoYGGYAlYyjyUJ8ZpGgOFxtVggpPpO9oyAudqhxAIJwHWMiFfWd2FbAaiACVtCKfwSahOPwzCdY0V5Fgb8Goy+yEKmfS9xVcuahKmbZOMvpRW7lufbniuX2hSrmTOUMZvmyTjD3Nq+0sXdxC2nqkOqQ8UAnXslLjJv2esBVaDzSBDm8RgkuFAF6QigojYELDWjnTA0BSyKYbKeGge7t4SblrlS76q9zyLNgQmyj2jDIAMfhlH6cq+UQvsO7SB0fzqL+9N5GF6ZNHcmtlwyKDK99j/bezdSGRnXSJyGvus7+5L109ZHmxYZfvdKkMkLc2x3g77dEN1+Z3eWaUv7QSOvxTADdKHdQRjK46yo33/3mfbgcx5sZ87cjRrb4aIpZZOJdyVdBnwXoKoDCwFdVHloL/uq6dp/bYtxxsQy5+B1ja2qg/TKSTzjeloHmdao3KhWg8TkKvs2LGNjIbBYQiVIYHgNxltgEQ9GqrzRDwQWuv1113ElReehs8bs7uFgPhKv99st5SBPGfSUH2JJ84wdxo3zgupq5lPADrDLfRn+1yZt0jDjK2BDF8Ll1cs+qAvfA0ju3EHbwzIKwszLPiJoi7pXAAQSJ34LLLQVsU07U28/MY7l95BGMa6HRqWGBK2kZ59vCEMrZDzutT/Rt+TTlTaYlsDC1X3bMCpQ2ICU3QfjDsbdcWx+LhoUINcZgFIUAU4Biz7/2Lyh6zDPWD7jCvYFJpmXBAiqPTEfrO450G4CKm76jL4SycQQx76i8fsOykyinLiepX9lN27ir0P7w0jz7sKm4kXHDrcH2b/iLkCG+4tJGedNT8lgWvUnv+yaGQvOgcMTA+Sh5c3DAVz253WtMFQqQXrf9trj5U1PlHyTdf+dRIZnw/1TXZLWU70YnjmGUwF/m7Zl5jCvp4zr2E+I+mOYxBji5elYmFvT6PU0XOL2eMTpR7/rcV/3bd/eX/2lXx2zvYz9Op5pb58+z/h98jyKBHH72KbANgW2KfCpKHBbYPH9P/z69lNvfUs+sE4ymQ+98oWt335si9mQaZOxq9NP8gAWhg9jGGqBRf45mdeHuk/0ARaUMlcm4Ij1nciHb4H57JFJ4UOuKlSXJIw+qk75zvrM1CkTt13NQiZnJEEhHfPISiVMRX3cBQd+wGUaJZUgQbAEkOFDLIMaIBOmREkFH33CF7CAIeVextZVc4GP9gtTMOV6EVJioeGwh3n5AZF2xVCXGpBSAhmVGzBK0tFVclf+d8OA7+KcgJlY37m3XdOAG+PtRXi8ebxczFP++Rur7RKT/QLM3xLMlB8BmZ0Z84dh0yCTXfHaBDtxbyxdbDsW59r+m1fbFHQ6zMrqFPkoXbF+qZcMIv9k/A5puAxDl704SFP6yIwtsaI9e2G2zc7Ohdmv8Kx8U1ddawrGshotw5h+URIg1YDcE0OjdI2Ws7s4Hnx6e1h2P7S6gNUYOf/I0+cyiK48q/+/wE7IC+4zgFH3AkzyEsbJMm8awPaPZfqnnGCO6qc+87AN8rEEnHX7APunbW+eUbOh3DL7ieKf3ueTQq1SW8eoXkG/abwVncbW4qEHHojr2anpmQAn09fO4gZlt2y1q7rASgCG5E3JEnRS2jJ+jMqa8gz1SN8kFD87o1JA1XpQJ2gVYAEzW8ACux0AjTtNB1hgwGzbacSsatQa/c18u6tZx8hlANwcthVP0rYabS8izbkKI2z6EqOPdxl8j01asrpOWWWQHZsBtYDXSU69du2HmT0wWZvi2bdUldNVsRIKvaBZXwHDLlR7BJ72YecI87Tf7AdcOGYqX8Yf40WQezmG2UixUHu6phpUmP4CpgJ9KWfZpYf0r5P+BLAQJFwlfIAF48h+6Mq6ZRlJYC0Xaej9S/oGtBFX0JJ6Un6lK4fZvXqKU6mFZR/1WcalwEz7oBh7IxXTA5Rj3jDSz26a+pOXTWw/FMYJ9LPA4nPoooRiDbXIm8wFNwAMWFHV+3QIC0lZkVLGbbFzDvOl0rc12lhgMeF45PkM4/4MUooXYbR9DwDjGG1jHVPPlIAfHJZF+o0OC3brQZ6jQ/r1MHlsxSqdUZjhZtS/k+SQbuJWm1XDGbdKkeR62rcmxm/T63n3663BkksVKa9GyZk4P4ZSDHU2YHLNuzRKgvVQm6n3uvhkPO9P9TzhKrB/txzbwGILObZ/bFNgmwLPQArcFlj8wBt+qD3802+lWsMEawW5DXjwI+bKKl9FfzuJepWR7AyZE7dMt67YipGrD2mSSfgCI/52QpallX+JByl+a+htGs7vMglKLLoqlIxwPiYUqJgrPsEE9lnK4JUPqzYYBSyGvFzhM21AjpXv7h4tp0xgmIshHW00ZB6N74qqTJOfFcMILOJNxjic5iszYvlUgTqMSsS0wALJheVw4zJXL/18aWhauv7aGsAcuMrM6qTAQxqW2g6SBBl+mK+dnBMwFTewsVhG/2oetZQL7Do6h9eiReIscV6GWVkAYKgb7irxCcpwHAZuGib/mhu6Ybx7Y3mhTVyabXuxuTiAcfc0DN+0khXyUY1LGtxkrwvrqO3FFKvFM9NHAgZkgmWstC3QIFYD4POz2D3MzUedSJpaT9WONKbOztgwj7u09SBty6R+vMCiPCEBLPitZENmWEZL+ktnGbOs4EJP49rP6BlRfRF86QpURnIRgKFB7kUY5qhmAaxcDbZ9PKvvVR8KUPF52osacp+K+scKD0fd0uHyv/q1bWs76TUp5eK37SeoEEypwqNbUXeAfuDee9u9d9/djrKnxV6emdCNbMiG2hjtrzF7eakCWBBPJnov7WCbJx9iSEvvPewvA6uV3/2Pz9PvqY9V6fWScXZjPIFYqW5dj/G2XqGWkTS5CaCet64Axtw7RGBRXqFgLslTyZDA4glc/c5htK1NggbKvUyWSxqkfNxnzISmYYHTfkomDgIiBNh66fLUY9cUp6v5UZcDLMdFLMyubVy1ZLw6NgASgjvHp5IewcQegTY050WeK5UQUOpQwDLH7kWGn76UzRghipIGx6SUDKDgfcY840UQqvqTwOKaoJ6z160vSEgPV++VbOm6VlChdEy6e1hP7SqmDh6MxG4KGyOlLR6q6i1DuyUMvJWsxf6J9neM97aSph4uSFR/r7nM+dKFFW0u3AwvtlwAAxcXBBU3J3axrw07txPXsKZjShP2Lei5S7DGlc5GXOLbb12M4dl+9n25A+nKaYDF848cbieQNh6iHtKon31AJM085aU5DPn4q3qndxyUMZ2Q216nCmC7+t4/Y0clPDzwZUIljVFfHyU5BCbvT0qGZ+btc9uu592vYznmdpTtkJDRcySv/sMnpFmJDkWDvmlzwowHq9ijMvgzr3vCQ9nyvD/rcQz7FM+2gcUYgbZvtymwTYFnJAVuCyx+8Ed+uL15ABaZMIcJuTNXMvA1AVfdC1S40i9DWOtf+UDC7Kij7ce9T6YCEu0JelphBmEe64NaH9UCFuYMSCCjDiw0OO3AIkx+mAk+wkzUhrMcWXEMkwpzgVpMBwalclKAR2ZeZt/Vy87Y5qPvhM9/P1ZluC5TKYiyTvXxMFzACGGtU8LyXrUPPUK5QZ52BzKc1rFLJayNjKiMqbriMqseqT9pSlDLL6MpsNgNA+ap3QSmwqhCsUketJwHVMwDLi5T9wVAywIMy6yruKShW97TMHUnABZHyeuy3o8wYr2Kd5wJJBZ72DjvAB6jjuzf06YJN4XhpvrqMu3u9mtdtLE4MlXuPffjmWon5bX93PBNL0NLuiNVLQmVmYswd65qW0cBisz2FOoVqlLp81/GUZsYGe14oIJptO5KLGSsA0RpY+kYYEFeunY1jIym5SEJDmkt4wgAot3Up1eCIcC4BLhwTwIlF7aLbSrzGJUn7t0zIG0NI6khMgklP9tjC7MwMBk+N1/7nu1nuXICeiyTdgnaoBwCPEZ9hETc2+OeU6cCLE6ePI1R9IEACb1PqQ4lIx8G2n7PCrRSi0ldvUIDJT3mEzZp6E8p2qe6JyXruU7by+hKO+nigC7mucCFbofdYfoqTPgKoOIKm8JJs2V+q0rUpS7ZxA0iq9p2EUmFwOICdjsabmt/0vt4tYWMF7Th7Plu0PY+UwVN5vUwO6Iftg/kZDUfoO0+I9oPBVjSp9ynRXU4x5XpuGau1MIVd9sk9imocgn+006kbx+7yjPb+goASQNt1TNk+jd9yQNE6G+eto1G9tIG7jD5lCcmwTwSOCUWnDeQWAQYJ559zjIRhnHVXdMaTmBgr7AvaLAtaFIyOYNKkS5x9Tom/ZeR9FxCjczzCipQ7t6d9ko9e9+ixtTJuaxLRq2n5bPvXrPf8lubKnfUXsdL2RrAgpKyyEC4Xjev9lOlYAAwpZwQlpP6M/5cyBDsH6Js2lacYDO8U56U1922NdruR9q1lzEPra1H9bG6T9NvHTfDi2rHql+3exsPmHb0z5Cs9R89465gQiWW57nlrv8YypbHSYM/9T9hej+tFIZ0xuOMvTBskiX+eJDq20MdhhcC6M1jrJS+pw5PfVShDd3pYrjx+1G8oSzf/drvGD36y75xPk9dyahfx/P0mWV17Hhuq0KNU2f7fpsC2xT4VBS4LbB4/Y+8oT38M2/NZF/Gd5nDwwD5QZBh7IdTdBgwPlIaDfvh9bcLPfmQw8yZWT7wROthBQDeZwJjEou0grTillZGIPO5zLbAgtXM4cMvePGQITf9koqMAQveFyNYwMKPd+U5SDqGeNcDLGo1Uua6JlEmU977WaAEmXRleEpfvz4qqj6NvnbQwkk4gIB8XKnVvuIwK5iu0vtOhkhmmB/50Gf/CJlm6p6T53p2kaaqgwRYuOIPsyBTn03sXN2H4bkGX7zEeRmAsQhjOQdtn8Q96uMAiyUILlN0ypVJmIkZmLQr5Lsco9altgOpxSQSiynUo6b2UFYYwUmwjbtPu8otPayXq8qqQR3DePYgaiiqQ0mTmzLJgxtP93KYZ88JbR4uw+QJLuwSGodqY6J9SYFMpTAyT9SLUzDlqrQM5gGYc1WBdli3fMCK7jLvMpihgwAj/aTo7Ip0VpBh1GUoBQwyf2EAudbK8rBHAO9lCPXGo0pK3KbSFmFoaEP7jeHzsad+NEHayDYzjNfc0/6+tNnXGAyCBj13HWFvBXuJtiHaLJy843i75/TpdveZe6K+IzgKIINmeogq5pe0qZ+SCzfIK2AB0Bz6KNlsYbDI4JMOqVR9vsCF5Zd+8URFfW5S5y6xEFxcB1yoEqUKlAy5Bt3S0HGVNrLepCHzsIj3pHMXLrZzc2yqF8YYGwveeVQ7uKCQQma8ZDWXgeo7wdVBVGumcV86Aha46p3i1FWsHqg0it+tBAJgGXfEJFXgSBLXXOBccU0AQVlsNyVR1lmpWUkWrbfjnjE7SAFsHN9btgBa0pX2jruuOmmbRloB4x5VKPqFYFpJR1fhc05yHgh4VfKAUwQlG+kn9BfnEeehAtA1zqeUTtLfBWiOI8udzQXxZKZzgJKk1cKB80jamrpKf+nm/KR0z5aXFgEXlAHzcCQUSCdQGVzdwQIA5xpjZZU+6CJM5izbnjT0/qQXuR0jYFG2LVPMBTO0yXFA0EkAxR1IVWaQVExR1r22v21/S4/LIx+PHRkjEtnD9u/H0Df6Kx+nf3BNaINyk5E9RCe7HIwus67j1nSG55uPK3I97pGGuD1zg4yXzWAkUGn0+BXX+tQT/o7FqWebddiMb9pDfj0QKdSj1KSXwkyHgJu3Rb/Nx+N3/d02sBinyvb9NgW2KfBMpMDtgcX/+ob25p/56WHOHaZPJuDOcMlEOn06Kcp4hHHnoxnmn5V7mXEjF1NdK8YBFkSSgc4KoQw7H0VTMo2uAuC1Vqedy4vhVm1Dd6xe/SA7o8tchaEKU6rKVH2swyzlY70psfADHsaIMB7G7cBC5jQfaRgMy1HAxiuMy5C/EpZIQkjHFKSDZU/5KUsYBH6r7nEQBspTw1RpEAYHZskyu4LYGWw/wBoLB7RYPxjvPTCcnupDh/GCpnHPCuBQ3WGdjfJuwFisUP5lgMUs0ovHr6+2RzjnKb9mpSdgGo7BmE/DALnyeRVJwxWY2wlUoiavX2lTuJ09SPNM7sCewhoCLNS5lw6WUcmChttHD8+EgVZtRSnT6irgA5CiC1X18QUX5/AgdJF7GTCSK716GJlurCtdsiJLn/CqtEaj5b0wltl1mt8Ci95XXBWW/tG5pw6TMEwBJdxLa3pKMbQwjUW/spu5ARMoM6paiyor0tw21c5gBYZZPXfTllGwTJAvDJy6+YYjUVqVfmy/gebmlfQlCKfMnnY1V/AmtYuya1R89OhMwqxBt5sw8Cf4ffepk9hZ3Mv7qQBD+/wN6KWdhWPBwzbfySq09VfNR4N9VcAKXNjjzfNTH7JEI2BBRWTura8DWlUomWJX/G8ABJWU3NAGBe9g13TBCsDQJavet7RnqZV8GXUBCeAU9Z1Z1KEeP4+7WaQXeoXq6j9Klxy30sXFBpl8x7/JOO73ZYdsgcV+gKmSOzwl4TVryj0oOAVRUYFT55+0BBIe3XNc1OKsA31RyYRugMPY07YCSrINE+4YEwgYX6mEjel8IUiQFravv+0H0sKwgtUOLGT0swO3Ei7oJriPFzHaPmFGYBSpGP36OhINVZOcX6yntiN6rVIS4wKCXqHs185rprvAeLgwdwlwIf2QqpAepUw+OwEQSoqcTyxnzSM1r0oLq2Pb6sj2JhVW7ek6wEIVKEHFWuySClgEZBnH+uLkYQKwDpGj/qTq0wFAxQkBBZKK05TzBDYVqkAeoJ00SGcKGHo3N8OxpedZGInO1fJzU6F8Nhxhig3WH3C1f4yOis5PAg3Re2D7uf2ontf7LUGGuD33SvOpRsd42pV30knalWL6qQnwM/13rE5J0bwqg7TBcGsJ+Vdp9sso4OYbXtXCQ8UbUuoX6XibIzTk/TawuA2Rtl9tU2CbAs8ICnxaYPGWfyOwcMrkH3Nr7offTsMy4Tn5EEa9iQ/zHlbBZQRllGTQdOeoKopMvB9xmQA/ptFl5oPo1cSdXAMU+ICbpkefjw3fP8RRnRFYcLhCGb1qAhpWfXw/1ubbJRaqZ5hHQID58kGvuAKLQR2KsgVYDIxSygwjqc1CfVbMX6bCOhVDkFXGsfKbvoBHMCHjo3GqK5jSzHq7UmqpXT2VNjIzsrKxU5GZhWYHUBFRWqBxa/SrB5rXCr7epUp3msoQV9Wo1i4CLM5dX2t/fu1mexLGWfWoaZioadLUtaxM21UYtSuAixusVO+9cZX9LK63A7AuB1Ci2D/BhncymZwyKjLWAhyNyDXgnjp4OAa+0tCVedWh9Dbk6vcCalDuMTF/CVUkGCjbVnoLIKSHNJKJUj1GoCHz5iZigoZ1OIpVgJFtDhHiLUlaqX6i/rzSK+vtfiAayJbr2+ovaVvTJ23Lqv69jKerxZ6l/lJgIcwlzOqIqScv62h/8ZkqVQKDuLolT8srk2jaNpjh7GcyjJfZQO8CO3Rfp9w7aSP3WLCvy1IokbgLYHHm5F0x4N7vRnm0lwyuqlBKehxw9vUwxDCJSqT0EqUExDpkzMB49jFnP+1Mh/ceFCfPLFNW+qGf4EXQFGkNdVEaEZChVAdwIU1UfYqXKIDFGgAxYIpeZJtLr0iBqIOqUrqb/cST2NCwj4USA8eYh3TPZnTpf7adsLskhdoXuVHhAVXsABRHWB0/gkHzkRk2GMRd7EFU6/Q+ZdtnnBO3AxbbyDKuwMSXcXO1Vzx/MW7sIpbR8aUKknuwqHZU8xIlSP2RzAGc7D/Sj4ehi1Kc0JN296qkooMKVec8sn8Gfcx5w7QEoQKaJaQ7ARaUQbqrUlQSOexHBE0AC93jpm+StjTXhuUiamTZAwQHA7U3CvMIRXIBwTGhWqcLJFEHJV1paMvajpLa+W8NOt0cnew1A6BA2XCQVhCGGGl/60p7bNCPPN0Mb522cA46DPh5zvSBdi/tcMb2wPZlH+90oVt9bGCZ7eQ5mJuH0tjx0/cMO3rbw/mgP+XWsFufVBsYjKPe9nvjVTrVTnVfdgzDmyHPhBzSzhvrOuQbFcT8HtLtdeCZ+Y2X1BCJOfawjyuvKUcCEdf8elqjVIg9ykuamZoHCY6l2dPZjM/r8fSNMaRtWO/76e/PBWDx+g/+P+0dj/1BqLf9Z5sCz3YKPP/Qyfbdd7+yndn/zPGm5v5R4X+eZuN8emARiUX/CDm3jk2wTIpRX/EDzn2Yf5jmYsxU54Fx4rcMUNSh+ODLzOQDQjruZhummo+i076fgnwovRsm69EkTIislhJWhjWTPIECarhmciaNSEJ4H8mCH+6BiR+pQsCY1ITuqiDAQWDhCi8MgXYgltW0BBaRtHC1TMYpT1HDaqP1hOkNcwCzIEjoUhgZzX0wjDIbm8DCvJAGkH5WxKGNcaL+BBPuJmluTLefVcfYX1B2V0itp4yY0gtBhafSi5wyaEgvNOgWXHwCI1slF+ehs8zBftI/ZLlCf1Q/qKeelSYABnsAFuwl3Q7vYJ8LgMUejLlZro7UYie0t+0ESKoqqfJzCENuGSJ39BVYuAqua1KNZ91nQoDh/hLSLjSBPtzmkHbSRMAp6HAfEusF+dMGMoLeqx4m42oay9hxCPpMYx/PVS2TgdNgOntLkIb9IBKegdZmp7ceV+zd6M/+lM8/iZBd6F0MfTGYvgsjR9vH4NdCwK4VUCnjapKgjDC5tJ1eiC7BMD6BmtACm+mtYu9y6PDhlEcD3DUY+C6xuPceJRaHo8Zmn98EFkqE6IO03QSMoupASiy65MaVdQe0dXM8eaRvd2LmCc/4Z7+0/5e0ouwrXJ23/oIEGWrL7TONoCPFoY1v0n4CRIGUdJJh98LaJ4YAAEAASURBVHfaFmChJOrJ8xfao0+chzleRNrT1YBKimTfEGRmJ2j6lqV09VvbpwCLSfT5UbeZhpnV9kB1sUNIK/bThxwPAZXQOuOLNnbH+bhxNR+Y/oxB6qZsx74BwUITAZqLBgJUbTUsh4e0EjwLTJS2WF/7gnSzXaWD7azqlPOQdhVKtQxn2a1Lxhd9j4aBJusBKKowubu25ZPW5nNQKQV1m2Ll3zrKvFse5xINtgVGMdqO4TYetcjHxQlwqKseSSPqT7Rt5hPno5SCAJaXE1hEL+RkfllV7SlX1KC4riO1WKeMgvJShaJepLVO2dwEb51zg3aYAHxNUdY7kVI8d+YQwAJpBZKL/Yy/GIunOJWzNLA/hWj8qac+tNDDka44CjkK219XXKNUnHwnRi+H4Elj7KG39nHj8L9Sr/ejsevPel0v/NvTGeJsvqi7SEAIlDoNLz+pTr1uXHPb0xzKQ9JVLq95xx/eVbC8Tbl8fduDCJXVJ8ep75D51+nvzwVg8fd/98fbm/7st29Ltu2X2xR4tlDgZdP3tX/w3G9sDx488Yyp0l133ZXFtqdb4E8LLKIKRepOeplWnV/556TthFib3hUz7oe3r1DLSHZvSs7VhIBxcXWUeJ5JM3N10jZVJ2A/rHVw9X9+17Mw2OThql/eEVCGIROzkchIZqzKURKLXh6ZtUgthrjGi1QCpkPmJT7jKV8YTeslsAjTJdiwIJW2aQhSZACVXmTlESZRsBCGgfeufk7ygVe9xXuqn/RUyYnaBWVMeYi/g3OXRs0wApOqBkE3GQ4pJLOUOqGnX56EBBaqQw35wWhg1R21KMHFOUDFE6hsPIG9hRvoCdf2SfyhPjFWh6liab3thJE8xOZ5hwAVAozd6xjooiqzqroU9BAcCCwOAiwOAyymUPvZQ94b7IER5pW6aAB8CUnF7NwFdMrZG8HVX/IKfaiX7UJT59o/ovxK/fKb8sksSn9X223ftC1lllbuL6GBqqu62mu4X4SMne5K7V+2s2BE6c4kjGbSJMOisQyn0qlqL+ltfaIOQ3rq7lSfTiH4E1JLbvotfZewu9yAjH/rgilUqZZx1epeGk8goblwaQm7FVRsBumSEhqU/duJY0cGiYXAYpp2G3YuR2KhdyilBmlTyjChxAIgqWrQbupkvnFBG+Z7E1hYtvRxaDE6HFPUtZ/WuaRixZyrsubgLkmEUrmSUPh7PaBD4FFAUhBiHX1nPQtYzLZHHz/fZpVYIKlRiiDz7PgKsGAM1DiU4S7HCt3F7P59AgvUoJRWHMbAnb4jQN2LBE8mXPAIEWh3VPQiBcJzEldBoW2qAwZpZG2VbPQ8C7g79mrRwueCAMen41V1Ixl7AZRtG9o4jlPfkubEKBpAkT5CvCqP443dtu1vzAHWU5B8hbSUmAi+LJd9Tm9ven/y1I1uABbpLKMupnRDo229llmW6r+UzxlAAEEavc+VwX8eZ34YIDAggXZV1Yn+oT3FOuqPq/TdNa66nN3gmS5k7QrV9rQ7KawJOuiz7llBJ8KAG2kaZb0bYPcCNsE7AxDSziL7VUjetABlyn3Nx0l0eM6lDjPyMAJ/6hd/h8ejm+F3zdeGtQmd04cXlVG9GP/rc8PwP1nwzhiWqP/Og+F5ovYXPek8rD95NLwfXqcceZuE+dPLNFxz6WmOp9XD+YxyWp86TGO45RLq8Xvz/fBuCN/fJ9JYvBF9eCaY9vc2sNik6/bdNgWeDRTYBhabrch3b2Oje4VywvOfx9i8yIfNjzCrzUyKfkBK/xoxP0xHJAUy2zLBflRNg3MDBj1pED4T7pDn6DlPzckwphkQkgme9JNeAQYf+b7nnSDELObDMJvAIuCCD67XviLeV0sjmRBcyJilPtSJxKyXQMhnKUtKNaQvsODswEKjctNWjUKjURlfGV1XVZU+mIBSkSt6BnI1lfSz0i6gIMw+1J808hZUyHD0VWTDUWDUaSg7QEVVGe8FMdpe1AnzASNyHcnFAozRHJ6iLgAuNOK+aT04r1EPbQM8J6jnThivvQAEFCfavg3u1/EOtLrS1rG/WF25TGHZT4O2sjy60pxhtfmI+vEAHxmsgAHCLMYzFHr4c3MxCNZgmMLT9jLx0ISrEimKAuiA6YuBNYwav0MvaChtwtDCuNoGvqOjhMtwRdyfxlcCpaqNaigyePYv21ia6G1KEGcfTZ8hlvSVaTSs7ZE2IYxG45ar90nrkz7Ds/RVmDbfl1oSTBpHGHZW+ZfwPHXpop6wZtk8bgGPSYAF+rOqTALDvVjB3jkz3U6yC/fp026SNx2JROIjCVBVSobfcaBNiWpSqlMJGndGUiHDTJkHYFFk6MxM9fcUaPhTjKV9VIAxSCzoZ0onovoUwKA0yJV61YlgcqGxkopVw6RMBXYEFq7uy4SrCjWLKtQjZ59s5zHgdmO3DizMWvpkfNsGFNKxEBfDlHsSD2OqvekBqp+Caz1gabczkjbRfwWTqs9diBode63QFtOoOOlZTZBu/Rwv60gSk28kXra9thUy1tSLfiM4UeLRjbALiMCY896+lTQYAwJYpWDa7xi/20rYZ7S5uITBtR6wsgu36ZG34MwVfuunhzc9QOn9STfSjnfrIABxI0GlJYIKy5Sxm3kPeO+VuhWwoGvTn6W15Us4fqevE4ZOgRqTgEIvUCWl2OD3Gn1lnT7jmcHhoDC+bQYIWaOfr0LnDWjjYsV+QPgpQN0DAIvnAyxO4KVLt7JCqPQoolOsHI6yjLv88n29qPmagB48Gu7qh7/yoOIOSSVo/8ObhDGYaVWyfgd6CF6bcdIZXg+v6tHwYriMYg3xU+bNgo8S6MG9mm/yNnLy4qn/ZeRLtFHZD2mO1zll65mOp5OkNnPp1Ul+A+1yP1TU+1FalKGHH+VFAN8b/HWfA16hPjT/8fbY0lyn7PZ1mwLPagpM79nfHjp4Vzu4S/fzz4xDfkre6Oket5VYuI+FG+TVRNinQ7Iabp0Mw6xwdeVQNZMw13zYOgiIzQMf4DBwmZxdtbO4m0wg0fu3haeVuB8lVyL7x9enpjliSPnd33sVgHiYz602FsbZZEZV55HRqhXMSCZgDnIVSPDBKWABE0adciZlyyhDWBKTrMqTrqu3AowADX5Hz5xGEVToptV8LZ8rxqr3dGAhg6VHIDeL0+2mdZMIbsQVdpo4yYu09UFfgEIwUWpQARswHrot5U9c0S6v4nYWlahLnBp2K7W4QXoCjovkfxlmaYI67uHZft7t5LoTYLFzDWBx40rbxSZ6OwAWO2F+d/DettTTzVFAxTF05A+rygIjbzurbnMZYDEPoz0/fyG7ObvbsfWTUZepV8oQiQ00c0V3AX1zPQ5Ja9tZ5tGPfbVjrTqHSYYQUREjnh9f36f9uY+rYZ/LrPE7kiIYWxlyglVa3Lh/ibT3tCwCC8tju2ivofRDg3TV1ZQSFFgr4KlXLpnXbNxG268jiYhNCRIad69eQGpxAW9YFxdx24rUQoCg7YCGy0dxMXzHsZl2112nMFo+krYNgwzoCrDoEgvacQfxNALXyFbPX1Gto7xKw2x769fHw9AF00eqp1tXgYIgWAa6JBYBaeRlf9MAOkBi6McFlkvtSdUs3dC6A7fucMvmAqab/iEA1KPRY4+fC7Bwk7xIPyQwR8Yh9LV/CD4nKX+3oZlEWnGAlfHDqN6oKmT/KdUpJTSexdpS3NRRJl5bhov0DRn/pKc9DumGcR/qJQ0d1xnf5FvgomyX3ChRA+krg71EhWE8wXTTeNBQKZo0rfzlqK2KTLn9VXuLsqeozQAth+PeQHbR2FVQD+ujpMJFAPuXzgKy8R3lX4ZmghuBRp+LbD/Bb/oW9zRoNSPp1rxJu9k2loUxRaIBFXp/uqn3J/rhmv2avrEBoNBoW5WqMKGUfQdxnXfXqJsqUKucEDu2Wkco573TB9tD01OACxYHaJf9zh1K6jKaRtP4qEy5GdrYsns4Dyc8P/NqqAKPh4MQo/v+bPM6/qpXf/Mtdz1yDzgKZM51JEt+5HfPv0ft8Q3a35FGZ+QzhnqahjURrkk9CfaIlV9PwqcePR3vQ5Mhrc34PZEes/9OjM0y9cfj5TVIP4b3r3vt9s7bnSTb120KbFPgmUmB2wKLf/HDr29vesubRzXrc6Nzaz42PHCCdU0/HzuedtWTqLUMzJ/PcvKR9Ro992HyN83xybvPxGE2YZz88JJ8vsky7zIIXbfaj7+rj17zMWfSDlNBHlnRJqygwo+7q8B1DxMugIBh6aCi30d6QVrjwMKVrdG3gDL3lcduhGl5wrzyQZfpneR3mFcZWICFG8bJFOl5yE3dZMg9ItEAVOzHdkDG1MNyiBFluMtGRFAhmFD9SWZ6OGFAZIa1t9jpuzAdO9oKXIcuaJcAFlfJE+FFwMUczNN5mKB5GM6dPIf9aPiNSX6q70wALHbhKWr3ymL2uNi9dj12GbazDN4Mqix3sFHecU7BknnLpC5jvL24eAl1qIvZ18Idna2j72xn1Ub2Y6gtjVQpcVV3gVPJiQyRHFXaDgLzi/IU7QMkpMHQdv62D0gbV5n9hNtnqo94zeu0U38GUdLuBSrLj79Mrntr6Br0CLYR09OHWX12Z3EZRWlKW8IkF20LXJS6jfr77GANM34VqY57eMzrapeV9ktILYBnxBe84NYT9Z/jR4+0EydOtKnpo7TzAUrLyraMvuCCdDJ+aDPbdSfMYDY1Q2rR7Svs3wLyDi5IIEzNaJxYX07HnfQL3eg7sSfg6kq7oMI87VMVRuABcAA03kTd6Wpcz7LHAntb+FtJhYw2yUaNx/0XzmJjocRikb0swmwzFhy7BeaQFtHG2hgI2HQvrA2N4GoSYDENY6unJGkiYJIps21W7ZcAXVWSlNqYn2BSqUg8J1GfvUg94iSBtg5o4pn1tf2dd2wTvYo5Bq2bdi+LSAuUHOQIgekl9kPu3Sgv0kPi6CCBjEMXPYgtAUq0iRD4dvUl0xQcqw5of8smeIIKwJKg1LnFDS91KZsN8Ihbm2VSSPJTIuFhMbLAQn62ZQ7e9Tmz99U1Aip11FhbUHFDYDGBBAK6rTrWOJkAaB9xknMt7YDEMcACwiilWAWgarRNIwQInVRaceRQe+4RdoR3LxFcS2uwTbVyZP4e7tMIEnh0jN8PD4k7CkK8qmOFGz0fxd+8GU+pZ1dveTP+skdJIKHgU7w0yngiCeLDIXJ/J+HzGFDEfT3272bYzbFk5LH8qEwARC+PsYYKbj4nfG9k35mfh0l55Nlwa48dsh69q1db/w5xt4HFVrJs/9qmwDYFnnkUuC2w+L4f+sH2xjc/TK2Y9eq/c3AmXljc+jc2p2YSD0NYjEQnR1RN+LB2xr7AxRCxUu9BMyn7wZXZ0MahJvVa3Y7LUpk/mAOfyzQJBmQEtgALGAKZRFcVo7LBb1VjvJcpMe3RKi+MeAELVqWH9GR+tqgqULrUzY+UXwmvMgvkIaNiuTQw3u8qvdeACiQW6N/LsJq+BqMrg7tRmQx1zvXprxpPGAzy9IMkU5mVa8sPY+GpTUV9UMm75+tqd06BBWofMHwCC0HFZc7rpMcl6lAXYeQvkP88K6qExu4CdRbycn8LO8AEajA7b6y0ncuXOC/iNeoawEI9dcpDfocp5zE8+pw8dhxVkMMwjri0JA33PHA/hOzmjMGvqlHaIKifrsGsK7yee1mB1rD1ssawMGHq1rtSHO8/qqZAHyVFYZqkNTxYBwQyd5LGo6vd2PaCQ+O5K3SMcUnTPsCrfN8Fh8ajmKSlfnx5yRntMRL9f8AFAEO3unoYEmAoxVBdKqpqXCNJIo0YQUOX69ihuGu1+3fMcc6zO/UKqmca02rDcZzdjO88djTA4rBAbP+hMOLSwzqv0g45bEcYa0Fl7GcENJ72KercAfKWfkfE1M+6WVf+damFY8AzfRmQEANu8wNMxIibZ90rlG5vBRTajCh1cp8LwzhOBcJRUYJRfyzA4lIkTbaVdLffj8YW4ZX4uEncNEz3Hui8O65Ud46kFQILJUDWw/Gs21a9w9lmN5GwCVSyUMB7y64Kl/lkTFBfx2nPV8sMx69zjzYwAgvfa2AtoL2GGpIEsm/0ecF2P8h4FOTuA/A4ZgVJbliX3dsX3TTQHduxMRLdDnmoylfqTzoOEFRoa+SmfsBy+lVUpwA0lwFdguZAMstGmdIBKYdXFwnSlnZq3lsXHmc8d/UnVZ/cq+IGBhbX1t2zgt/Q7Cb940bARjk0mKDNBRMldYB2pGSf175idfe+tsZ8sxt6H2S83YsK1ENHpwAWuETm9z48ZegsY4A3GReWxyP0zZ3jZWCyh3IOj3Ox3P3YTKfCp0795RDQ5CVDDm4qt+GnF58NZainRZe86g9uuVrWpzpM27L3YxyAjJ4Std+Pp2Nf67+Tzi1lSppj2ea2Awtf9vDj6feC9Pfj5e73/UqYpMmf7/727Q3yxkm3fb9NgW0KPPMo8GmBxb9+80/lA+DE5zzoZ7F/3DUy7iopYYBgOvzd3bmOJmsm3gIXpT6RFU+ejX9UKv2a4CNR4AMqI+JE7+qtoEAmwRV0GRFXxWUsXf0sqYIpyHcXgClD2AIWXXXDq3laLhmzYsRgxmA0Ythsepw3SdtnASyENY7pWm8/IpEcwEC4Sh0GlHLt45xEZaG84iitUA0EPWzKajoyU/rl915pjsy570g0z0xXEKGqUYGgYcXcfAkfQoTeAhreBSjJtMlgCSxgSgAWy6CJRWji/XUYrBvUc4k6XYZxvEKd9pKGm2LJpC0IDGTwKNvGTZhLNs9bX7rY9l1dbrvwEkWqMfY8xIr0UaQWdx093mZ0G8q9gEkayqy6p4WuZ5dgVgUWGv+qFmV77YeZc1XbOrjTsaovrrDLyMWDjlIOGMJsIEbfEnRKF6VToTfltJ0jxaGePg/jaPuQTpcudaAhUxzJE+8sn/3HjcdKXa2YRUGgYOeAABDQVGpRZROjBOkA4OIQtiV6w9I2wP6X1X7qKhMuUy6ocK+HWVzPLrLifR2As4twd8wcbnfdcaydxqvCDEDsEEBM42xVkmTe9b7k6rNtNsHKdMYCTGTak3LZp0ar3KTnuPHM+KKNR32Xfln3VU/7lafMuaAieXEvsFAaoaREd7dRgUL9yTpcyWZ5y7SfdhZ6UtqIxAaSZd+PJ85dyEZ589RRYCEDVnYtlpe2oA62rfQL080zmVfbRaCmW2Hdsdqf7cC2vapDURkijFIL203Ji2BKxrDKDwiDVr7rY1fJQRYkSCn15qq0UpsJ+5J2DbzIO/tzdoKn/aXdXmgqSKjxBthO3tqiEI5+mn1PyE/gmv5HWZS8qNIlaLIOXicZB85tyZf21mtU+i5jCKUrJ5/MfxAKhp9akY9lsg9at9wYzlvy2AFdJmhz96a4mrG7wRWjdvrFGuMaaJ8N8sBfob3qiZ7Osb1PpP8Tf5V5YwLpmNKjmQN724MAiudw3qfxvG1lnGRd86TFGB0+GhXP0lYY6ey/XnDrkN+WY4iQZwlnrM20TWV0bD6uR9BZumw57Nt5MJ7KEGIsrGVyrvQY5TGkl98mYvJDmAQc/oze8zvp5DkR/D+Wh4/Hv00J5p8Kmny3lJK8elnMflS3XgZf5sXYu14Prr4yf1PZBhYQY/vYpsA2BZ7RFPiLAQuq6MTXV3b8QMV2IkyvzI8f1LqGAefe8DVZO20yZTLJFtOvisoAMDLx1pRsWFcfZY5GKkncy1z6sZcxdEVYZlWQ4apjrXyzksq/pJKwqtCUupSMRKQUggA/5JbTPClSL19XIzE91RkEFl5lUC0LAUdxq57WhXRIs6vMZN8O8pqEUSg//pOskpbahEyVFJDZc5XWOvqJF0RYbhsAyobB1GBYQ2CZWVdkd8twymD3ciffUrsx791jEgsCUW6kEJR7GWAhwLgKY6Ph9lXyXEEyc53rJOEEFtL1IkBnWaNqyrWK+9nrVy63VaQW+1D32YcUY49uaCHXAWg+BRN+dIqdprG3KPUhpDHUQbrLuLppnhuvLWKDoORiGTURmUSySx2UMsVmQZUOaCozqL//BVaMs+qrlIPy2WZRg6G9/F2AofYvEFTYjraZYINk0p6dCTC87XgNxtJ9UwyjMbzMcKRlFEbphTS1jwSownC66i6DrHraPhizI9TzKHsvHD92jOfYv0BrjxgCw6Bbz/mLlwIszmGLMMcmctZhHeb2KCvFAgs3yTtx54k2M3OUHbinEl+GW8NpVddieE97R+WJOklLXZ5KI0GYfT7vwkQWwLCe0i6gmOX1fi89vJfe/bRfBXzLoKP+pIvg6wAKd9++BqBYvoIaFPT3dLM8VaeMqzoWRAUE4LaYDd5mh03eVK+S3mW7Au2g254w34Iz+ruSKfffIJAuY01H4HYEmxNdKVt2FwGu0t9sF4iQ8Rt3z+Rt/zCMAMx9IzT2dx4QnKjCpgcmmWkHk+WMShVljDtX+q9jldxrnCEd08tUQAvlcXHZdyRfY26QjhmnXO46ztEiom+4aKHxuOpYAosD1EsPV/YP3xEl6ca7FPX0GumIaVs+T4BFdgOnL1pWGiK0c7AUKKAPCiI5J6DNdZbXl65jC3VtFdsoJRbMDUosSEcQozRP2lv+gG37hvOZY0F601/W6UuTSEFnMCw/ycaEAot7D+9vJ1RRo15x7U0v7Ox7OiRpjg7S9pC5NTP/efRrvTEQz6Wp4YbDkJvh6uHo7XgePYJxe4BesaRR6diX+5hOFMN4mLV/clR79vvhZSVL2hVqLJ8eqyfV06xE8zavRs8pogQfOyyXh3XfLMfWcD3MKJppmIxRe9r9yrsePnUm4DawGFFu+2abAtsUeIZS4LbA4vtVhXrLw5lIo5rgB5PV/HxYmBRl8sZPGeDOBNeEyfTrJKpc2v9DHBlEmfGEHybXMJB8iGU6cvrRJ65x4voUxiM744bxkkmpVU3BhR9bz6TPh90yydDLCHivekIv1/jHwqJl1XtgVGWmrsNQawMgs6CagemGKU1d4ZLNh3q6B4HMqUyq5av9GdwgzJVadJr5yHtqpG0cV45N1zxWYfKjCiRDj+Gs6ekVKC5HcT+qetRerwINmFqZbcttOCrDb58BPIZ3kRDxTmbLjdtWoOMS9FmCkRNoFMCQScG+AlrsGxiNizD3lzVchRG7Sb2VPKwCKvZfu9IObNxsB9hGey/tto9svaqr7mZ1BzHi7ipRqg0pMYqaDcyraWhrsYSazSIuaGVcVW8RiMmMxuUrbWT99cAjQ64XIvXcZaJsJ+1LZKptA2lmf7AdYuwtHYbBFpr4nLMzpoKLa9TdVWj7iL9lA3pY0w/QkI7pK4INaQr9aQPLePzIdDuh1OHEnZHOuM+ENJc3kFHU3mJhYREDbnYdn73Avhbzub+Mvr42HHdiY3Hv6bvamVOnMOJGcnH0zqRvfBlnVaAidYJ2qrFVmYpRFFTEjsF6Dqfl9fS348mzO00IyKDNwyRTV8sXxjv9TNoVsFBicQ0wocG2kotl2ibthFqXoMLTdvKQWZXml2gbN3pTYlEqSoBE6OP47V6SVBESQMRtqw0zlEFg5O7Uupx1LFpHxzNDjXpAb55JDyegSLdI13qpXndh/hLG2KjMUYbjR6cBa1ORiFi2SMmI41hwzlD6sYLRtuVTahF7CaQi2XuD91l8UFWNcWA/tfNkOoKW6VP8jood9LVOSrP2aePBfTf6V+rmOLfnCVz5D40oi2nlnyXjXlBhGzHO0gb0ccuk9DM2RYQROAoq9giU6HemsYwx1MLV1XbxKp7jBmChu9ksOkAD6WLb0wk4WbThqkqX9lUIONoqYwWCAuKmcC97sN0PuL0PUHGccusJynFjdP4PthFVbhK22MPh2+HY8nwsjIl48KiHtmzDo1zzMpQa8hh7mTjGHegUWhG/5xBa+pszfX9I0XijfMbyTqWGMLkMZck46c+J3NP30ebsMfZ07Faa9LxC854O1/Hn4/c9iM9Gz6VQKpxMR4Xo742Tcg5l7oXcBhadmtvXbQpsU+CZSoFPAyxej1eoN48mwDAGw4fOCocJ9EPJhy4MPB97J0tXFvsEmqsTN899J4MkY+KqYK3G18zvaqbqQnHxSB4ywcZNHqQfVRY/kn5cSceCK1VwVdbVyKjK8DEO00j6kVb4AfakPKbjmQ9L5vz62MbeQmBBetc43UROZta6esggVd0Mz0edvP1iWJcuGVEf2/rsh2EQWMhoqbfvpnLuoi1TLdPc7SyWMQK+gl74NXXbYTxkimU09BKlr/99rJLv2+upa9XSu0/ZLX/qKO1ghGEsZFQEFjJa6vnrYvYG9bkGDVeg6bLMFnRSYqFEI0DJDzuHqlCXARXLMGfS8cZNXGUCKvYBEA5iHHoIYLGHvS52ATJ2wKC60Z5SmUOU0b0tNOQWAEkjGV3VfGRQVRe6is77YuwuLgMcsEPICrK7TrsybPu4Kj9IL3inJyKZRFo96enCVwawGOhiKAUJBRi0IaCN6CNpJ5oqkgjStf1hS9I3AuKofxhK6mt6ve1gxQYD+QIWBXZRlYIZVg1qBmb2GADhSFS/BIgYxtLmfQzIlF8CXJy/MNeemJ1r51jZvwSwsC3dGO4kwOS+M6faPWfOtJMnTyOJcp+N2shtB4x11OhI06t91D4WSZh5UC/7V8YSV4Gx9ZLOjgnPbpNiWJnW0IJr3ZdExz5doIF9KAQWSiuUWgAuVmgfVaEu0zZKmyK1gDmXrqW2h6oPzL1G0XrzEoDbx9Tht+2sp7QSaCpNkAnfxbK4XUvG2vYIABFos4quq1n7MK/ShwXF1s+FCse8wChjkD45jxRoifIJHNxgLy5ekRz43j7gmHXs2ZcsVza4pJ1NR4Nq7XguI/UIgCdO3L8Sj+yg4UBXrtIzXqisB0y4UokYm1M/0w6tycsFCtsg5fWF80DGXKWVtrJrWXkyceaIJHQsbx9mgcPxyiktbUPVwy6tYP/kCbC4ug6goH/EoJv4EMaUA0QnmGMEEAKL/4+99wDXLKvOM3fleyvH7qqmI90NAoEAIyQhopBBAUU0CggFNGM945FtPZqxZ+axNQ5j2ZKFQCjYM2ONwyOJJlvREiAskEBCCREUAHVD5+6q6gq3qm7lOO/7rbP//9xb1UXQwEzj/9x7/nPOPjusvXY469tr7b2No5afpl5wL4jbTZ29AQ3RTZvXt2vWw3eBMjTZ5s07v4nLH9sZGfSmjuFVHvlR8zQ5JsEmN6OYBj7peRxmErhu5KXpJ4YQw90QXfisN569zyEN5cRNXkxILQ+T4P1xGnbiQnxJg5/6rziSRLWxeDXN+r9iHizf5UfoNJ7xq057PNeL8Ln7MQ35MMTXr+M8z4DFck7PnmccmHHgscYBv9UOEm5ADh4ffFcuXfoJVoV6/ZtZbpaPoB2vH0JH30o1r5CKUMZHTUFDcBHhl06zd5h+aNKR9l5d/4PAkV2YCRtBkLgzoXOYiFuAASEJioyzBPgapZMWO+tuqqQQo5rfUW41B6FFoTX3Ct0CigIWhpVm6bNv94OrQKYAo0BVwAKGIER7ZERTAWIQKqYfKb8Urt9f5jSOdLrkZk4EqQIXmlCw6hPCt6OL8szJ2wpyC5gK1aRZJ4yy9KvCGYJTlp4VWBBmHkF0jjMbtSFEZSQbOiJkQpOmM/IlJjTkNTwnb2SHtNROMJpLvk4N4CKaC8CF9toxn+D9ESdUO+Kr8I05VlYMAlSsQgidZ/L25mgsWJ6WVaNWcjrJe460BFCbmIdg3qRTASdmKsSiuZD2/QqymjodVXPBrtxHyLMbiJ1idFmhXs3FfIBTzUHRht8ysGydu1P7YAgM8c2zQqKgojYtY56DtvqAGN0V9hUYNW1Sw2O5KIAan2EUXq1Llpm1yjK3blo3ep2JORvhFSznEfjdp8DR+B2sHOUE72zAB9+Nw9FodzBfRIh1dah97MS9j9WTDjCRW/FJQXsX+1ncduP17dabbmw33ngzoJF6ABBMeyCeaJzIvxvoWYbWcw/fS1+vo16tyx1Y6CcgjrI13/q1DRhucpL3jJhDZ2kiBBZokwZTKHcQd9K92iRBn9fTADvLzEnN7gtxAm3ScXjcl2FVJBQAuRpTtBbwynKfg9eCaldE03xIMCFfbedePTcC1FwdLHu6QL91WQ2Q7cvyti27IaIdkacrh7kAgIBBEysFfuuLWi7zbBt29S75Qqbjz3IWmLrsbIAF14AQ4lZLBPnpI5yrEQ0V9NlXuDeFWib3poi5UzSppVFz3o8mWd18yfyvoKxWkOfaQ6by2sFKOpV0DdSzlA1lRDlpimU5powtZ/x0bYsmWwfZxV1gcRitBVtXMl8D0GFdkG7iMd5V1JUV8JsOAVBDvaXPyRLK8HwToGg7/L0OUHEd+djF/ZJ5FaRn+aWPsAJxyPfEzSUHfnLrj2lymJXJkYepy/RO74ltEm4SZnSjH2nwCB3cpj8lqUrNFxVX/AyOhuh0T9LRA0fFVvf++n5yDPcZCKpI4n/qZUqP+U3I6ctJNP2m09CfE2aUXLmPHapuT/wPN51G4+txdjfj/O7v+M7lQT5rz/YRnYZ+HSemm7R1bWhvnzt27Bh7m93PODDjwIwDSzhwVWDx6tf+ZHvDW9/EiJgjyw6eKXyWqZKjwHY8NXJeAn0fcZ10UoSxY8Jb/Eaw96PIR10hsEyVahT2NEKBE4kVajTDcFTS3r6EKj7g0IBEwcfIVZ0cofUsP6txM851fPhryU4FsdE5AAs/7B1Y5B534/Ajb5qn1VYgGDvir26jgAX0IciYrh9Czy5oKMSWuQQCFh98Rz8jZOG+HuF0I8Kkqz85Ki3Ni5iiHGGTtUNHDiN0s58Dgq/0bNZsignRG7Ht1+zG0e05ruvWltAuj0Ovgg15iMBm/jCpMZ+VFwWWMDoFDGfCQyeqnkb4PkW5lYkUWgx4fRQh7RBATo2G80ooYXgAuIpJzOm2hvv1aC22AC7WI+bMN+zmEdCdc5FNAMlvBHDodMdwzV0cObVe1Kg59QR+KoC7Q/dhlqR1wrOTu91EzDpSmqsCAgp5Cs/WIfMXPjLBWvM3BTqBh/b5Tvg+cgw+croakFoJy1AeCPBqHg5zIqh3AYsIqn0Ogn4sB8vaUzCauTuavgz1MSPv5ENTGOdebGPDtq2YmIi8LQfFmcwpIV5NcE4gFLrB237Md/aitXAysYBvB5Nmb7vp+nb7zTe3mznnWR1qFUvKagYkn9y3QlCRUWfyZ7zW6f4h73XMOli8sbyrrZjfnicFWMN4evR748qiBC4vS5kGWLhcrvMrRhO3rYdqX04KLsiPk6stnxPwOntDIOC7z0O0KvKF8lBroWCrSc8K0rH9WffdEM/5EJoTCSykHQk7q2058dnFDCz3SdsEYCjSWRauzKTWxEnkrtakeZTzcKzvBIqgXitX1QaULgFreZjtRfyehF6BQDeNEniGJ9CnJ8GEYDUghfYYEMrzpmgWnaxv+Ra4E4jKA0GVplW2/eSfPGZuBOWWFeGI0zpk3OnzzC/laNmVQEteDTuclpBx2+6dXL4IeDtC/Tl4nKWY0VYcO8s7eh561uTZupowpLGSPmElfHWzPEG8+6YI1q5hKdnHsav2DazKdQ3830pdXg9f+mRtMp/8y0NpDJ1DPQnzuI+XpOQDh4l6DHWqyKm81QtzWcfU63DXw3SPXE2z18+JMxFUCPM6xEXYHm8WXZLeZeEvi2eIcIk78UyeicO8G9FyOip2Ihj7D/2mKn1+u+p+HDYuk+ziq7wkzPRHx/I0oWX6Mnc97jwM6c40FsuYNHuccWDGgcccB64OLNgg741veVMEGztn+z4DaGfsR88+t4/4KiRmAywc/bMz9awRxhqtK40BH9t8jBEg+SD7kdSfwEKthWcXmuSmgrcCp0KhNsyhAeFV4cq47bsVbEpjAWjxo8q5CiFBAS8aDMJPBPAeH1fjNA7Tc8K2GouYz/DsZ6EEAoQNBQf84kCeEX41+Uge1FgIJhSAa7S2C8sBFpg0OdLtCK2jQwKLo4s1eq/QZNouLevov/MWNmxiAzoBhSeChOYjtVdFjdwbj+lqQhRBZ3jWzY+UNFom9RWNTJc0NCs7hxnUKQDGUQSaRxBo9nHuRXg6iv25wMIlaN2cT8HHiearARXudbGJbLs79xomd69GSF3FvhfoGCIwadKkQLkJejcCjlxhSYARGsibwm32fxjAxVF3rj7CBnNoMAQI7htggTqKLCAQRBg4dYdcKLAp4Ds6rZsmLS5bK+/qRGCG9ghgqY9lnqZwa3zyxZFwlyHNHgnE4afeU967ao/xrqQ8Nc0pjYcj8WouENAou1oRiKVUEeR6HZIu60nRcg4N1CITuRfaA/segS5W0+K9O0hff832duOea5lrcT17WjARnEnca9TwUK4CCucZZL4F9cr6nHxDjxT28rQOGp9txvx0P46EEwSf+peNFZ6byhtX61y0RwEX8qGARV9q1lW8jjHXQo2S4EI+aXgTsz14fZA5JM4bEVioVRFM1Cm48B4e4z9COzzTZGkzGoaNmvThV9qj4YQO6c/keEfeya97etjuLG/z56FmIRtJQkuWj0261Onku/oes2t/4LwH+SFwFgA4p8I5NRlwkAcyR+5Qn+VVTCVJx7K1nqktC8igDHTLPCVo4T/NJxoQ4hQAqLG0vVleWbVNYDWAi7RF2w4Bu1gckyfLjDMDGWaOtwKoMjMDUBwHIAMojrBU7VE2WTwh+KeNnkdKdYU3eyDrqOXhHheXGKhYYduinkqPWrGtgKFbtnpubDcCLDavo96aR8JHi5J0q24Mt6HTOpJawzWZzcu45M7+Wz/jI0KwvMHZutaP6V1epd9cHnbit4cLk4eUh3v99LpsGiblkXTrNr/jtEfO01vSkKbur8ploJK0koX+OITqfn3Mve/xGN+dkMFvv/TVZtUeVqUpWid50GMQRyXW0wg9nQ94mUYfymYai87g2XXGgRkHHrMcuDqwQGPRgUVG4egj/TgKLPyA2xVGcEcwVKCOgKuowYsIbnSgXhENIhz0kTs7126eJOfsdBXUHGE2bp/5T/zltwRn+2nj05SnAwu7beOP4ICgIR0K69KVe4GGy2AihBdAUShXOC9Bza+Qgqm2zs6vCB0IIjViVsAmArtCAmfuDZs41NQM4AJBy1F2QU3ZbWsSxWg+AoF8UWA/jo37oiPEjA6X2U8Bs/WYP2UX6PXY8meORWkrYu4hSAqQKHrH4EL3AhPwNzwtPsm4fMBkLg/yUnChJuYwgOJhRknvY4TY8xAjvacQyJycvYYTmQ8es1wo1/U8bELoXo0Z1CVMaVYw6Xclk81XsRTtauZerCVf8/BZYLEFjctmTF6ci6CgaXn4wVW4VYh03oVL0roU7cKRhcy/OI7GQcFfr33VLzVVEdoBmKkD8DkgwazwrmvKYnpF/KVtKjt6tWke2Y8CUKBZk8JqjcSzzwFCovVHuhT4rMsK3ropBEbopAwFFQqeAgtNopwn40i9fgSrLqVqPRVYOMJ/jFHtw6xu9RBzLZyQbhIu+3kNK/PsYRfuG5jAvWPXnrZp205MogCPxC240BxIoOphnY+QUuSl/Kw3OQOCq4wjoOCnBOhBEIJ/5S5/6jRPZTLFKLgaC8rtDCP7zn05BcBdXGT1LoBeB3uadaldcmUqBwhcpnjf/kcCBDUHklZ54GIEMYfCn1qJ1H9AhIJu9ntQAwBQVmi3vQksTmiWBK8FboaX9hOYY/nO8M5Hsq3aDjWVE3QKSO2c1FTKA1u5wr7xyOD5eYFjTabWZKtrK+RlB6TWq2gIqN+dL17VGNm25tGI2VekrloGtBWFdtuVYESA5SkNhjHeLAoBqBBUCbZsfx5UKUNDZvUt1hXD6UaJJM+CikVoPQI/Dh5jDhLt8CiaClduY1GodtZypZNzkW1rcvV1XInzAlpBgYWT/lcSrytVXYOp3pPZq+I2wMV1aC7mpV0SQg83w1F1ox7KVYrGh65Tl+QDl7jCLw/7k/QpPCc+3HwziUkGcJTAzZtpdBP/8RBPPQUDVLh6V+EMOnY1XY+ebh6u8nMlf3Ejmp7yOI2xf80K+5F0x4T0F1zjjNe0WZ/8h84JP+IXx0Q3jXMURfKTeAa/hv2emSnUmEWz+xkHZhx4DHLgqsDCORZveJMai0GwJ4MRxvjY2gHbkU6EC4QGn9O50o/aWavIt0u1M7cDzfuBSd7jkCcFDT/gCg010lieEiY99rTTzggecXdwoSBqNAEtfICzQpMCAB/fTPh2DfdBKMgovx/8CA98hLmXMuMwXzEDGsCNH3XzEDOILA9aQp7rz+ckLQUeAYwCk5Mn+zyLrCyD4DiHMKBQbDp+Ydx9WyHXHX9Nz/jlgysrrXUlKEDF/JybtTHPgrAxuzC9pCV4QsiKkFnCj+lfBiyQporPxVvvLYMI7AgwCwiJDzFaei87Rt/F+RAjpocQkN3724meggUFto3wbTt820LZX2TE+zSTus8hkK48ewozqbOAjkvxswlhZyPC4kboV0uj0DnZiwPBy3xHSEOoOku+a9UoViQiLgGWE4g1fwlQhVYFRM1YFBY1K4pgZz7JRK3+xNK3bFTm6LgAIpPOEThdFcgzG7nBowADAME8IOEi2hfnDiwwB8KraQn+5Iv31jvv1XopEG/aMJclVBWCFSQFS5pC1cZ5pVnR3t/5Hk7gPo7Avki5HnEDQFZSOg24ULjbiTmUwOLG63a3a3dfz74WuwEWG1PWqwMs5HodY+Em5QcvLAcBcC9PWxPO1JsSOruWQx6nrvY6y3v96uZyqppunYWv8j/7WKC5WFxk4zvM0w4fOhgN0jHNociHiwgILpSvHt67LxOpT6GFUZB2tbJ5gFZ2U6deutKSo/9qqraSV+u9ZnEdkMk7I3LpYeu9tCiIqT1yJTDzpQmSGwp6fwYNxQIAjVymjdquxoMAfQ8MAch6gIXtSuAg8NF0STNK86yWy7kZTibXLC/1Alr1dwI63HMk5U3866kf9hMCnIABaBZEVBnAe9IwH4IEgYXAPv7w430/9B93gQr9i/2RXVf6ETxZj63TB9EA7aUeHqINLqKpOIOmwv0q1FR4pYlyJZxuPNjPXaA+X6IuUggFLsjfDsyerkdT8YxdW9vjmay9C36sodLZ6qu3HSgjfDLro/fDkcH0/rDkSuiRv/Er82gM47pqPUulxN33SWMIn9T4Sd0k/3mPvwKK5X8c1zitK93r1zh6POOw43vDdj/e+66/X+7u+/HR/ekWPspQ783b+DCPPW/dfeyF+yXl0P2MrmO6dPb5e1/+ipGPz+6t/V7PV7+OU9RNmuwjPWdzLMbcmd3PODDjwKNx4KrA4lU/yeTtN7IqVITV9KMRUHvnG8HaDy8fUgXs6pyqd02nSc+bj8vyLrYc8y4dl/4QQBLGDnt09M49ndwQn17ywRUQKAHZiXP6MV/Nx14TnQIWCgWlRcioI+718VdImAIL01VY6RoTR8wDdBCCnOwc8ycFbgUG8hkbeQQLQUxG2hE8HKGeF0xwLw0RRPAjQHAuhunWXhZlxnMWYd3O2rgdwY75E6DCjfPmEGTXCSwUXggXTUkABcJLrgr/ngU6LJ+YXOgGD1JeoZt7Pw7wUyFUjcWRAVjczYjpXyEE34swvB/NBVJMBOtoeuDZTkZzr0PIvAY79BWYQgmKzqJxWcHKUWucxM0eF/PwfB7hTfOLOfjiJoFuRtbNwwQY5tsyVMC7QDhHz884kRjh1knEJwAYCp1nB/MmNx1TA+DIv8vGlqkTOaCMFNYt19pbQD4pmFN25MtaBxtyWJaXkJzULqwnH27mpkBr3I56ewj4PAIs8F9lwU7SxFlaitIoCCY04dmEmVrMXyJMAhKh2ZW9jrjyFVqA2PkjsGo2pMmWpOxgMu0ehOYb9uxue667oW3beS2COPtiRFClnsCraltVTiHIn5Sd9U53y9nyrXL0ZcqT/NhmFFxzcPXWt7nmnTxnMr3gQl4KLNBKyPsTx5lMj1neMbRHC8x7cQUvV2KSHuNU+D7ABoDulq6Au4a6IMByY0G1EwrZmpLN8SzocqlTFy3QLFCTIcG8oLdoKS2DPLEeOVl8gfR8ltc72TPE+mx+nIzv/CLBgbt4u9KY4NF3i/C7wMXZpKsAb7nJ88wLAehJq3VObQQsy3v31Uj7hgddsyGfrOvuxJ55IYCmMqEsgFGrr1Vbj8kacVoOZCq0CjgC9NMOC+CXFlR/VSQpDQGDNNK+HgFU7GP53ocX0FxCk3OfslcFnLiAP0iH93WmT0LwcyDlAnXuEvVVcLFCQErenE9xy7aN7W/s2NJuQFuxDTOoVaDv1Bk5O1QLeZyH4bmehgcu3Vle5X5wqHAJPP0Z+enApPvr8ei5tBbeWRd5479XjqrTPVScrvqT/ouw4/BXCtDf+y5hxp6uEn4crgfpeVlOpfFODuPkIeETYAg1+FnidxJoyoeebr/GC9F/73fMgMWIXbPbGQdmHHgMcuCqwOLHX/Pq9ro33kG2qkOddKQ8K4Bk4iIfZ4UMBXX9VR9bnS69bj3LGF5U11s33tupeqrZmHgsTz3JxGnw3qc7mum/I/CCiyxLOwT2w+/uxyXwF6gorQUfXeicgoqi2ecIbAMdCh9qEjRziUkUz7UAagl+CviOXroqjsJzn2Pg6LdClRO4Na0It2CHQCRaC4UtVgQ6z/wEVzKKZoSreVegynwKhKk5gEX2sABU1PyKKbAIkFCo6YLNINDoXhqNEkD9oOWEj5N73BQWnWchsHiY0dJ70FbcCbC4j+vDCPKOSst6gdNGBMfHIXDdiLnFbkZ0EcvhB2Y/CKXnEQwvYVazGrCxznkYCLjryKub7q0DZLhPxjqESpeLHe/DobZBxmjK5Mh1116cBFicCsjANAR7f1cEUphVANQ8zVNwIa/kq/XMMpD/Ai/dViDJ6J6RZ8rIchRIKZQrBCv8KjRqiqIgakaNgygHv5hEkYYE6l9hV22FwqVgQkFXMCOQSxiEXScXa+JzDI3LEQTemOMASAVQlEjq4A6E7d0Ai+vYKO+aPde3rdt3Ep/7F1S51hyLoUyhP4RBk3n16OVnve73vvMUKHWNRXwPYbzvgCOaGITy8MN6DXhzgnRpLdBQABRPsGrXEcCFGxVqpudKTGoOnLS8qPYFAVitXQRpeLga8KhmQoDnyLYaKpdX3sKcko0xP9MEiknV0iGdlINaDQG+gFtTJ82gLGvp0l2Nh23ReS4C6k1oowR+zjQQiNpJmReXkj2eid2AWwCvbTcaC0Co2iNXHTvDKmj2OsaXhQQIF+Bp/YOe4l0J4PpxQEBgIXiUFt0CGMir+ahBhAL3FIKFkjZrGcav/jlTPiknyjOF5w9cQAI/Qx0+Ck/vP3SkPbjAfJxjrMoFimAPy5g9ZWftAVRIX6GL6osy34K0qIQFLLiqVbqOCdu3obF4OqZQ17G07GbAs+2AkgotKQAosH2Ejlynt/Ko+7GsBvaUL2mIW4Uux2Slbg06vKqLv/K2+xylq9/xC7zIq0/36HE8Wtj+3nj10+lLNnv6XFNOvh/c+nU5PWZlQmXPFw49/R6uX5P5Ub66v3G83a/Xfj99TyKEnwGLKUdmdzMOzDjw2OTAVYHFj73mVe0XX39HfX+GzjUdLgKbo3MZ3R6Ahc/pMOHDpOMcdd7e5nHorbsfR+TGnWxGu4ZwdrS9d+8fBB38aCgod3DhKJ/+FDRqsyuFFIXCMlPqQmgJDdCNICDtXnu8nR6FEEc9BRZnEXzOKcARuf4MP8eIoYKSgs9EY5FR+hKcIuwQxsNJ3vrPyD3mVGcZsa+zNCLG6ai6S7Y6kl0jtZoTlQlOmWeUABPNhWBiGbiQpoAL4grAkGXeDzT3/EmPIOwYwGKfcywAFPdwfQBtxV7MoRYQ+Nyh2zh2IbzcyIozNzuaPIcAT9izzKs4imB6DH/HARhIqW0d+ZlHcF3PfIs5hKp5PAoynKvhuRZBUg2GgqhAKSYl8pyyyzKoZwAQzNtQc+Hk4UOY5ggs1CxYDgF60CS4sPL0vDiXQvBn+ZNVaLbcS5sRszfSDQhEOIwGIoJjpa9/yxqyI/SmDil8I4Aa/0bAlIKuwCJiWgTIMpUJTdQL521oFuCqUDGDQqgNsMDEx2VrBZmbiWPn1q3shbGj7dp1Tdu2g832Nm+bCqvGG+G1AOGk/UhbKrQlRp2mPHq+M2KOm/RLryBJgJGmOUhSvR4bh5oKBchyoy5rLgawOAvdLi3rvBf3tnDpY+e+uBSyk6YXXXWLSfaZ7yQN0KmZkW3C+lFaQNoXZWkbsj1o/rQFrU4mbpN/tX9ZQY5y2oTpmitsCRw0IRNcC/LOSAf3AsdVbPi4geWLt8MzzaPUVpSJXOXTPDgnR/rUWtjeBQ7OkRGsCDhcJczVrGJCZ/3gneHwSv1z3kxpW7wP0Em7rfsMGNAnVJkMbY5n61HmUtDuArCsa5Sd9WyikQMUDqUAUdX++mpQtkP3itmPtuKuRw61h46irTkJL/tEbd6fh0bBoCcEi3apnNZvQJJgFnC7Ak3mJeoxajr2DlnXbgBY3M7ciqcCLK7FDGo9NKw0rIeVfDhy193jVu+mPqo+DSEr/R52FE/4PbhP/HZ/g4O87kfir8bWnVIWPlif+zEO0928fip+HtU/L8bAIjzVM/RdKd7UkRFNPd7QNs1ShR1IvyLd4/zqbxR2HKdhl6eZoHiaAYvOqaXXYycvtDsfPtMeOuQeRpfa47avaV90C321u7d+jo8/uvNk+9AnTrbv+ort/5+k/znO7iy5GQc+bQ5cFVj86Ktf1X7+9a+bfGzslO0rHbnNKkwR3vkw8/FV6PGdHWYEHoSefCxxrI6Ub+XQ0/rcj+mdgRMBFwPxT+ecfpx07Xj9SNdNjTgpvmeuheHw0IGFQlpMoRiBdEK1wKKPRiqodTOogKPQXp2TZEl7AQtGy/mwa6ogHeW34iqb7BqpVIDwnVf5E3ohRzMN03Tydp+ke54RfsdTpVs6HPUUWDiK7ch4ToGIoEUtB4JP2XYzOqt/BWavgos8Gwduphv30bXTw9X3Hsgy7SQj0IcY4d3HijQPACweRkjbi9B4AIFsEWFMUXQX6d+IECOwcPlKC+MUI9lHEEwPMVK/wKlgug4NzAbeCSzWw8INmJzPgwwVyVerReBcA10uA+x+AfJCYUw+KfRmCVSBBSPlxxg5P+QqRIxka9oi8HPk2HJV8LT0zbdXBUuXQlWwz1wVXDuwqHCWP1oNBMj1CJMbyIeb3m1mVF2eCxCcIKxWxCV2rQ8dbEYTJf8JD5NJTsHeKwLgICwLjBzBVw48jYB8HH66VLJCvtqa9UO6W1k+eCsb7G3asp1zG/MTNiaumAoNGhfbTcqP+KudWJnrKCHIemXyXqscexsTXET4nAaB8wWYorWgLmuXUv6py5S95XYeWs+hgdL8zF24TwAoXBLYVaIW4b8g4yhL6Lqho2koUMtnQZhLL/c0IT0CuyDMUX/3Y3HPCvf9cE6G1AowFOhtM7aTAA7i0jzoIu8EFq7w5QaZ1vfNLALg8r5qEATNtj1pEIzUDu2YQ1H2mhUa3lWfYl5HHLX3hoBPc0NNDWvHdudqWP9KO0Fbg5Y+p8Kr5ZHr4J42ZbuiXqRdpd3xnDzY7mtgRRBiO9RPPywj62n1CfLqUjsEvQ8sHG13PnIYgHECcI5JoG1N/GB5wSkHStIvymPLDdrd+8YduC/RJ6xgQjxMaavh8w7qsztrPxGNxW2sBrU9q0FRP4Z+tVeHqi2dsn7Fn7f+dI+mrdsQ3tslxyiiHqTeD3EZ2n/C9zq6JPyjPOj/0w2zJKoxMctpnDwPGQ2J0DcOM0R2JRo6bT09c5q8EZ3vPqVjQgO+hyDRmA6Bl/NpuoY1AABAAElEQVTKUpgBi8s5+xf3nmpvfd9R+qOLbSsfGbvBIycuIHesaP/di3e0m66dzlW7PPT/+y6/8gdH25/cdbL9T9+0q+3Y7LDb7JhxYMaBMQeuCiz+5U/8+ARYVKea7jUfUwWx2lTMj3IJvH6YoklwtFDBIR/J+nhwS7fpn0ceIqwt6Xt1zsemQ5AitTpgPgp+DelVfPbpIl+JxGtPU2/5oJeJVkYU+fALLPreBArpAR9dMCAP5iOj/sZJ+tIfYIEw45yECLVErwCREVo+9KUlkI5K1SviBXHXs8KioEKtRh+pVUg1VzXqieAhqBA4YCIVswqFlgjDHWCUcKvwsnqYPN5tusvufgQuiLsDDXkTYcdrhOKic+B6zDIWMXsSXOwHWOw/xWgqgthDCGJHtEEHfThp+waW87wBIWYTJicKOO53sYCgeQBh/gDAwtWc1lDGmyiDDchVm8AALk0bYAGAcknaAAver8XdTfUydwRgIZc0h3KlorNoP9ysTcFWO39t5RUks3keAnqAgIIWYRKOAnfk2n0snBPgRGCFXmuEZamWKmANujXFcm+LEng3tK1bt0WAVNjLXgkBJux8bj3Bn+VrDZW/7hGiwCmw8LDcpMs6rdDoqk7OlbCO1FKn51IvBVHONVGYFsxsYIlZ97CYZznetQArjw4ce33swK8LM1XfSZo0U9epk/3e8PqTlqJn2lbStoY6XKC+WlxG9q3LCOdqAZy34IRuQcbZ7JLuTtw138WlZ48xId3VotQEdP5G2wSfKJBMrlfjoAAsgE89B1wIMDLfAq2DV8Mo3DsIEeAN7ekXoF9aCxSgGYQOTa7ku+1lM5oPV0oTnFjyakpOQWfAJGXuylaadAks0CMgLDIYwL1xHEf75uR/QacA0v4gG1gSr2Zuxh9tpX1BQL2DDgUseh4FEGlP9BO5DnuPdGARMAhdgssqF8HFtL31eVC+O81yzvswM7ubDRQ/cfBIO7jonhvyv1Yjy+wgeGo+U/5yJsxRI4tGxnfwoQFKV6MFc/+O65isfRuT5Z+whdWgNrDYAHnJxG346mE8HqZ/+VFueVXeyv8Q5nL/U5fB+8QhMfWIeGm6V05zEmRy02n8dMJMAvebKxIk+3wx5L3Th0vSHMKMObOchjFtPSlaYqI0f77vfpbnt9LWa6UQ/6My9fvSj+VhdZ+tCtW5U9e7HjrT/v07D7Vrt66OhmDXlhLkjwIs3vmhxXb9zjXty564dHffpTF8dp7O8510cYjZMePAjAOXc+CTA4s7XsfHon+kStCJYMXHzJHAMilwdLlMoRS8YqYSwQehh+felXrtT/avdqw2Tc/eWdvxeqbjTrvNjx7SWSeMnTvnRV5F5S2BeVuaBQVET+nzdA6EtuEdQCjIKTD4PAEWCBEe0qUWRLvomEMhAFTaFXdGzXu6Q84MuQYBwLgiRIU3ZQIUYIFgoBBToKKnrfDiBFeEWWjxvYKsQqdmQ04aFnhkZBV/0iqPCzT0kVTDKQQXsIsQhB95NAUVllmyxo8TRGsi6QkEnsMIYAcAGO5r8RDg4tCZ8+04w6gbSetaBOM9ml2QF03CjiFIunrUQQS2g4wOn+J5FQKdS9JuoYPdhunKVq7r0F6sdq8LwIVmUWt5XgO4WAddawdAF/BGeIHFeYV7BVwmdGte5Iie+dC0JkvXIuT1+sCrmLc4cu2O0QqRJfzW0qTy3wndmuZkBJrn1AN4mSVo3awQN8vYdM4BbjRryryNlL+rUrGhHnVW7ZDLzOo/AgBMzDwDykl+W04CQVfwUWthnbc6uDGZE/hdJUsw4Y7btekhAi3uVthMTOa+x92Lp7eBXseTLvEvF2J8noAK7hNuKGLz5nNAve/842pDF0xkMvdwH2AB/wV3MUljrstRlwIGXCwC8gQbrhSlhmjHju1tIwK/QHdhmOsg6HCU3DpfK6E5H6XmKwgsUvdpdxAQEJL5J/DZfNmaNYvSrec/wjm8dUdpy09NQJaBxt8pgIQrellH1LR4kknSro0WyXbiO0mdkN6TmnwBhk276oDtqtpXtFrEL3AR7AboDe8ysZ4yt2yqb6Cdch9toVfrge2feL0xL7Y13/f2WW2UtoYX59/cz8aJd+0/kPkV7ltxhnZWwIIojBNa0p9ZhsZJvm2ranXOcX9+NfNy4Keat21M1L6dZYxdYtbVoDbTPp3fpIwTPsqI0GZkHBJ7hYMmWf68+N4wHqZfd0PdGR58Nb294p08WV5nl6Tf07hiaEmYhn8UL0v9jPKQdEYEWu/rGDmao55NXqYMuXaahwBx7+/G0Yz9Td73QEM85W4iVTcmr4eEO1W663ccp24zYCEX6pBXP/OrB9qBY+fb//rfXMsO8/WN7u+XX/W/F1Opg4vM/1u9ou3etqZtUY0+Oh44cLbtRMuwmvd/9cAZFlxZ0W69jhXXOPq7eT5cB46eb48cOd92b2ez001LtRLHT7O6ImncsGttvhUPHjyXOJebZZ3A30FovxF/vRn6bTlAvOZJ7Ys0roGW8WHagpatG1e1T2D+dYZ1qJ9wA5pWGzmHcyXN5+LJiyxpzkIrgK2lMYxjm93POPC558BVgcWPYQr1C8yx8INvxbUTrNFchXLNTfj48lFUMHMUL6CAhhOzCUYLXWa0d8Cp+L11EVc62HEvi4vCUMWRsX3Clr98JLjvcUiDLdWPsY7SVR98zZI8ESgV7qEtwgRCnCsD9Q++goBCQIQH/Hjt7ywCaXDTOEdlYw7i6CIf+RxDHkxbuiJkktZ6hVfSUVOiIKLwkvQRkjISikBaAkqBgeQB4jUtuQjNBSzQcGRjPCZvR7gSWBS4GAsuEBt6pdmzTDMqDx1QdH7IJ/nWP2DmTYFF4HQSwesY2osjCGlqLg5zfxyTFETpjIJuQSh0xS/X2T/CaPABwMc+zgP4Pw5PVhPXJt7vIq874O9mOr6VAIkVzMdYxUj1ehLWBNaVoxR+nNK7AncngisYagp1CeFUsw9qT+hMuVinPIdyUWhXQ5KVf5gTosAroLCeOSKuaY729FYGhVwFRSdse66LmVnxUf5bp6hm8Stv5JOdePiDewACby0P96vIBF7qk349LTdvi/e4k5428n4wyB4CNmZBTvZGCJxj1H0tSwevsSwDJKgDBO4j5OY1mQ49JWhImUe1NUvu8sM2ZX3spz6MwrzVWe/1lz9oC48AUkuBBSPnAXQKu7UM8sLCoXaYHdIXDh2C306ip5xoBzt37mxbMHGyXh4GeKgxcrlgl5w1HwI6gUS0g/DOjfTUZAg0qn+grpLZaGkoB/moOZD57KZJ8lR+qIUwTSfIO29C0yZN1iqPDlaUhsLM2t7LbNBaW8DQcCfRapwlDnkgfQ4sqMnq5kvRBFImtRkl2hbKOoA+Joj2Cb1tVRnTCUIY9SWppCrkLvXAOkEd8V5QYJoCopPwbh8mUPezeeLdB9h5Hg3hSdqZSyILaO1D3JdCMHOJ9Jyo7ZwKKl/O1Cvjw8+cc3aYV3E9560Aixs3zbdrmbTt3CblkpjnXV5V4tLbfhg48iOdk2N03/0Xv8tPd0sFmwSqmx5P9+Oz5+RZb+Stm2qNg/ew3a2H6c/jeMZu/b7yJI0yoVwrDtzqP+9IPkdyM+R1edrlY/id+Onhpvkx/iuFNYnEvyQiHwZCBhr664pjqeNsudnOnRL0/81vHGzPesL69i1fvnX64gp3+xbOtde9ayGgYi0NwnkYyiRf+6zN7TlPKo2GffQP/8Le9pJnbGofvudU24+A7/HPX7GH9t7y7mu+eHO7H/Dxl/cxj2s4Xvz0Te0rOfvxjg8ca+/+s+Ptn7x8d6rcv3jT/vbE69e173nR9u4l19f/zkL76IOn409QcN8jZ9sbfpeFMtC2WB+tYhsAMd/63K3tC24obbYB/0/yvGm9g2KtffBu5jNyvPIrt8fP3XvPtDe+5wjzHB0QW0E/A+h43Lr2Hc/fhtkvfdTsmHHg/wccuCqw+Fev+QlWhXp9PhB2pv2MYM6HUIFBodyPsN2jwMAR8dhPD6AiI5KG5X0JDtOOtHfOXvtp41f47c+BGOmXq3Oujwbx8cHNx4q4Y5rgNffQxbsI9goOnI4Q++zovodxCAiiYQiwGAQJ3AVRpESDZQUnhGlHSd0NuJvBqCWJ2YJ+OV39yP0rXBHHq+lJWwRk0utCU5k+lcCscCVHjIfssk49NEHHWoTg9UxgnRvARQcVJQxNR0SNPwIN8Xs1L3WlHBByOqgoXlV+e77lq/xVK+OuxQKMEwhgrhYlyFhU6IEmd+/NnhbQKYg4BC/cqftBtBvOyVgkjOMkW8jLdfDXZWnd8wLkwDKkTPJlYjfr2DDvYmXbgJ3GPHxdhRYDaRLeIqwiyDpivgKQoSkLfWh4Frt20pbffZ6Advmu+qMg60ZuLvWqjb31LnUQHlSpRfZLnXQ/CoFBND+UUQd1FE54JKDrYM063HllmjxEAFWIttxwCICRd1YOXk8P0jZO41+1ikm0aCg2uEEc9SFpq3mCP72MLKeAJsrbcvJIvKOrbtLTafJ5fAgoUo7D1XvIKvmFm95+ur/4hcfZ0wK+Ci6iwaAMLwDszmqSRlkI3g4fPtgOHHik7du7NyZnumtKtWP7dkzJMHFCwE05oDVyZSeFZPuDeczm5Ln1vbNHAFWaJ837MGuTr/AzGgOuslp+dKBlOAFQJsejRVFr4rK0ghvEOtq19VxfvY+gBRFHQAPCt2mXiVcBV9uw815sa+mjSN9Fi31WS2C5rCM/LpoQUASokJY+0GBavZ+pQQzCJXTVAfOdE7r0px/7QLV5Lj+8wCIEe5mr8uDCsexdcZz25TK+8mwFZUdCgAo0X/Qbl2hHboR3Ub6gobzEO+uofYVmdVsBqq4EdQNaihu57mTC9hbaHLLFElDReW916If5zWH9hcbhgXIdbr2MHyZ+dJbzHN1t8Bf3wc378lJx9+c48pO3+u1x9Re+wy3vfTfE1689nv7sNf6H9Iym+/F+7M+0PLwk2iHuOMa9vjH9+YpXAvekpHIc/zjdykCl0/13lkPBMhqnKZXfgWeD8ytfPltutnPog5841d703oX2MkDFlwAurnZ8BCBwL4Dg2V/A0teM9J9ix8mf/+3D7UFG9v/Rt10bobsDCzUE1+9Y077+S7YgoF9styOY224FHQ5U3MKcjZcCSBTUXw8QEGj8/W++JloJaRgDC/2Yzl1oFv7xd+zm+1/lKbD552/c154MYHj5C7bRH1xsP/Gf9rftaD++/XlbMe1aw6DDOUDCQjt8/EL7B8SvhsJDYPEw7+b5bn4bfqVXrYk12jhu3bMucayi8X8I4PHLf3CkPev29e2bn3118JXIZz8zDnwOOHBVYPEqNsi7401viEBgp5om45WzwAVXPoaevguoQFh1JLl/ADKSFv/lN/Hw7BE/w1VhKCOwXnPW+7qvzrk680o/8Zg2cZWWou6LrhI4C1QwcZQJyAEW0on/ot9wjAogSCjodYAkIPC9o51OyO3AwrkWyR+tO4AAP6uxvTYNzV4CLBBEndNheA9/FWhMx/hNy9F0JwjrR8HK+QuOxTpCvpalNjdswCYfQccJ3ZmDAW0lAFccCt7heb8qfMiH8ALaB/q70NppCT0DXXZQ8rLzViCoOc8pzKNOobHwufzAB+6PIgw5uft+gMU9mHLcj5AvEFlLfNsQzpyLsQcebyfvK9k4LxuhARicg7GBzs/N9jYCOjSdUmNxAY3GJYBBnUyyRQAUjFj+CpYCOfc6OE16ajaspO5roVmOAMMNz6TfHbKzPCzzQdYhZKld0ZbfdxHepQe+Oprss8Kjgv9agJv7BuAt5eC7Xo/NuPcKq67OpVilNuQ0gm5WGaLQLEsDK7RaJywnR743bN7K5OUtlOHGQUA1/aHcFFKhzbh7nfBeWvtpGXnor5/lMv3VbwcM/Vrhux/rVfnp77NyVNymoCJAmTrtnItsoAfPNUcTWOwFVNx3333cL0Rr4AfMfSq2Mvl9M0vBWrfUKLi8q8sUW+fcPVsNhXXR9I3fewGH76z35Cq8622hj/L7bB4EPO4zohZK0Ol+G64Wpmajg2sYn/w7ABB/0EHQ8NlydMnVvqCCZRMecPVQsDAuuqgAi7XQtp6ycj6HdUM6evtPuyGfNuIOKlJ+lqE58epJ3fLkIQs9uDjCAcDQflY3249W5yCTtRfQsrkam6ZZ52hjFwRj5ENTupWYYq0CfF6C9gvw63yuggtoWbuqbQGsXcM8isexb8XjMIPaw7LH2633tCc3tDQeD/kXmrlP71POS+7HwrH3Q9CE72FTl+IyxDPcexmiNLEl6S1JW56NDtOZhhtejGj1neE9loaMU7kvj3PwP0536pt48N/j1H0afJqC77ufcTzhw0Dw2M+YuMvi7/6nOR0SrRc9naJx8By6oJOIdel+vu/l31XePge/ttFe7v06Trbn0zbk+bneIM85FL/N6QRthf9P9/jgxwEmv7fQfuBrd7YbrymzJcGDJkg/9I270C7XQKPxdtBxE2ZL3//VO6IN0L3P8VCr8MzbCtwsBxZ/xuRytRNjP93t+/7mdrQZc+033n+svfcvjrcf+oZdmFdhEjsc96PF+D9+82B73lM2tpeiLfEQWDxw6Cx078ocksFre9ufVhw//O27l6xG9W/fhn/MsX7ku/aM6noPNbt+3nDg5L3tPb/yy+1d77+r7T/JtLs9X9Re8NVf1Z7/N25p4FQOLFDu/XB77zt+o73zww+106s2t8c95fntG77l69vTdiK3nl5oD7//l9r/9ab3t8Orrmlf+LIfaH/n+dd+VthzVWDx6p9+LTtvv7H6VPpku2U/sl7ro+qHtYRZO0c7qr5MaDrK4QOgXzupXPU/7enToU4EIfw7cpAO3bQ4FYq6AJzRZBwNn/iIK4KaH1jvTYOrwoFAQrMMhR2vzgUJrfgJbVydZKm7KwDpX8GkRjZZzYW8nFGgRIitlWwQCBRMoFHbac0WBDGCBCcmb0AIzeo3PIcO3heN0qR2RLMsQAVChEKWTDRf7m2h9sD5FgKL9e7MjOC7huU3FTwUqBR4ki+v/d789ntpSf6nwMIyGvN5fM+r+gTK74HHTlgXPBWAEvCQT/KrRmOBuRX7WSLzXkw5PoGQdDfC/SEESsxT2zb4dzOj84/D7GiXfHYAn1idmCuQYCw2k7cdyEFcitZCEyo388r4DH4uwYNLpO1I9Xk0HY5QL7IEak5Gxd1XweVONYVyeVcrLVnOspuu9OPym3OujEP6aqcsFz1Yzu5rINCVP04KdgK3gn8BC2sYAFEe814eqfFxlFjeO6JtHAIKl0bN8qfEZdoRKOW/dYjyci6FwEKNk3NDqr4VqChTQuOuMjGtcV1MfbRQOHo5ja/j9957TkDDIDRP/SjYA3goz5jb6DdhFA5snwrWw1V+kzeBhSsrqZ04cnShHXhkf3vg/gfaYZaclecK65o6bWTUfCvAwnlLaiKdIG37kIfOi1BjYF2Vh6ZpHnWbaC2sz4C/aAEIE7CAXzUiEzqob4qiqQvQlHkYxkW4izS8LuQwTJ9y9b1uMrc0kAOApx12Pl0gTv1Ynz1tN+58Pke99UybhM4c1gFvvNKv5AqN3lvWk3bm+/i1tjPPBj4ehVeH0FI8jPnTfjRr3p+g7TiB21WvpMN8aj4mD9RUrKburEBrQSVEW8Emmpzn5QnXbfD7elZ/ejyTtPewG7ybVm6mTmpauJq0bT9cJkevAzp1Z9hUbX3iq26s+b4ZcluOceFnCFxhl0bQw10xUoOOCepEJHYeKEcP6ez+4jK41/ueeLx+yj/jOJcEShmVS+dPf9+fHy1s3heBQxBo49/+3Xe+StjhbfzHS/lbnk49Fw+8lwc9ekvCY6axCBvy864/W2y/9YHF9t0s6/qFN01NhaY+Lr9zWVrnKBxHQ/AA2orf+8vj7b8FmGguZNsXWHw5plHf8KVblgR+tHcHievVv/xI+1qE/ucj/HssBxbOefgXb9jXbkbT8X2k5fGL7z7c7t13tgkCaOrt595+qBnXP/r2ywW5f/K6ve3xu9e1VwJCPAQWZ85daj/EqlPj4z8wif2BA+fai55edPR3H7v/TPvEvjPtH37rtZfNKel+ZtfHOgdOtA+9+efa2/fubl/6wue1Z9zU2l+969faex/e3J7xwq9uX/mUne3C3g+13/3t32t/fObx7ate+px244mPt/e/+3fan154Zvu+//45beO+O9t73/DOduHrXtG+fOVftLf8h/vas3/kle2LVh5r7/u5O3B/ZfvS3cxtneLtz5hpfucciHCu5fhA0XDp0k/+zE+1N7z5Tf07M/ng9A9SzBL8kHAqSCi01IffqKoDtbu0Aw2o4Op9Pipc48sO2rB8+Ceggue84ydCRt4rVODOfeIchLMIhMuARRcwFDRj541QVEJPje8lHmIuwadMKDJfBIGpm6xE6I+wjeClAJYTAYj0Q0ayXQDGnbU1g3KOhUCjRqSnQEfBVfCiaY1CjDbhHhdcfpZR2nPkfQXaD93nsMkXYKzmvmssNB+R1oyoEle/z3XgQwk9JfxMeJxUiv/D7RUv6idSbuYtAlgBi9JksBoUq9jsR1NxH8BCcHEvAtQRhCXGmdFErGo3INjvhsc74J+b5KG5zUiqJk6eSKHEK9BwXoZ7XtQStOvUYlg3iCefWN5pS6+AewJAcRQh9xirE2kv7xKjMYfi3rJQS7WRFXLUFKm1UGMhsHMSsUIoWcl8CYXfMwh31qONAIotm7ewStN6eG91RjgD9AWMyEcFRwQ6T4GFGiYrW5nm1IRhhXDrvRoly7IEVMxpAITzGzZFy2EdsA4GEEYotaUO9Xao/5ZdP6z/nmkXOPar7733XT+6X8up34/fS7D1OwAfXnY/AeiUQbXPQZDn/USgR0B3CVpX51o4fCimUM6jOI3JmnFQrPB2ddtMR6H5mTzp81ykLSZE5Mm67gnhod18usKXcy0EXJVvQYGanhLmbQOCxywhS74MK3AR0NkOeUx520fUYhDV1wj4u/ZKPkmXcztsRyk72NbzW3wgIvwp0Ft2bkbpBG7bl/3IlI/Srld+cJfmSfsa7nW3I7DtukP8YerlATYU3Htkse1FW7FAfV1ES+HqMdnEE38CCvkmTZlboZZEYAEfLkHDBerMeeJVc2F/9DhXftq+qT2ZORWCCjV/aiksC4V+D0nsx5R+3H2XOm7NW3pMnnu9Gkei1yHOAIvuB+dJOO+mD0sjH57CuxFtiXSISzrzvsepe+LrkRJwSdgrJrHEsee9xzt52fNGGuN0fd/DLHfvYft7PA5ORdcEWEj25N3gZ6C70zF9P4m134QHPbi9g0Fncywm7Gl91P/rMFl67pOXCihTX3WnIP4mzIqc07CF+QlOpD7NpOdDTLL+6wCLBcyUfvyt+9tXP3Nze+FTrwwspOAtv3ckZkk/DHBwQYkfwQzqizHf+sYBwLzmlx6hn1kRTcly2n+UORqaQf3ASxlW5ng0YPHTTGR3YvrNaF+WH5pLaXLVJ3gvfz97foxz4NLH22/89B3t7uu+tr3kRc9sTwC/nvzoO9pb33ugzT3pOe3rnn9LO/aBd7T/8v6728qnfHP79i9j3tClg+3uD/xOe+fbD7cv+Fvf1Z567oPt1//ve9rT/9nL25NO3tPe/po3t3V/+x+2Z9/1s+1nHnxh+3vf9JS2ZV4z/b/+cVVg8dqf/an2pre8eWkqSXXoRO1YeRtQQQ9ZH/HqInt/bgebE3/93gjT8fLOjtezh+3P+vFQILTzdQR9CjwIT+P1g18aBtLg3jh9ziowfKwFFlmtRmCBsKd/43fEVrp9FhREWzEIgmW2hNAgbabPzxRYADBwMA7fJz8DDcZTZ7chV0tRWhAFzVrpaRBkIlApAApYEI4QkIoewcUAPhRa0VoofDmymyvCRwRW3Yg7+Sd9r9PR1KJL3vXD9x7heXccXcN/84Vb7hWCeOhCk3MvDjLp9CFAxcOYJO1DYDqOH0ea18GHbdCzFUFoM0AhoEHBm9MYz5PHaH4YfW6YQAks1kPPBnjifg9zhHOJ1rVcBQuWuMKXy6C6K/eJ40djDrPIhOGjLNuZHZsFFqTlJnSOhjuXwvLNniXEY37Nh5XbpUedmOx9NFP40wTH+SVOrndVIEfiDSN/AiwoS4GFZRbhl7ic1+GkYJds1W3euTBoKdwtfQ5he47nNQJC82G5c1oXe7mYL4/UGdLp96lLPHh9tPKJ58HP2L/3tpvuVv4KWOjuCHn8wPMyhxLg1ZmN6QATBSyc1F335vM4+1kILk6QXxcwSJnAP0fJ5zE7yw7e8gNhOnmEdwG9XuFNQEbqZ9VF+aip0Tz8Mo8OQGhqaFuzDgkojKtrJ/Rjn+KzbVVuVX0vIOi9gMcREbVblp18L1Bv+TOW76g/cXgYX+iz/VD+gsFMpodW/fa+Q3+mtQIzOe8pPN4VWE854tbnXfjeif6LpO8+FQ+ipXgYUPEI54Jme5hEWcdQ8aboNQnV5C/8N2pM51ZxagpFpQuYEFC4CpR1cCMg7onbN7cv3LkFYLExAN55TytL2k++8iOdw1EtmAeyPQEevhv85H2xpEIM/Bk7mS9J9shk6yUvy73z1ae8HvxMKTHJ8dPl4fp745rEZzw97SuEr1iu/DuJY9nrno7O8n78vMzrZY/GOc5FZ4W1pKdnuxofU7INOYSQz6N89bAVrqdQ1xmwmHJz/8L59tpffaRpnvQ/DEL39O3SO+cZ/CmmT5pN3bK7BO+P3n+6/fy7Dn9OgMXHmWPx737rUOaDrAVAvJG5IX8Hml05ykONxSNHzrX/jXkY48Pv7D/+xb3tqTfPZQK27x4NWPzHdzJn5ODZ9o+ZND47/mvjwOn252/92fb2g7e2537lV7QvvmVFu/Odb2lv//hc+8LnfVV7ydM3tHvf+7b2hx871XZ9xXe2F9+mPpvNjz/2B+3d//kDbf4bf7C9aP1H27tf97ttw/f8rfbc9uF2x2v/uD3p7z+7fexf/1l78g+9oj1rO/sk9e7or8neqwKLn/rZn25veuuVgYVdZibX0mk6kugIpB23bnbeXeAYmwWF1mnPm0fjKWHHD4xxDJ9HO2OOpJMr7n6ncc8bP/rE1QVChQPTcqRPYKGGojQWCpw1+hyBk9CO5nr43M03IjA4Yp04jbcEVOM1zcyFIH/RrECAk7g9pEE/axBaFFwVbhRYjVeTEQXMcq9R0i7QlFDlyC3zGrBtV0hRHRvBDMEnQAShw5FVN5WLUEz8CrUFMqCPuGNWI93mf+CJH77xB3TCI2gdf9S6n/B04Kv3CnUBi9CjKdQiQpLCkuZQjwAwNIM6hXAo2HP0dCM0ueyl2otUTOsDYd21/DxnAArCo3MnLiLAboTWLYC9bRmFxVSG8OuGciMLfrZjGnU+k4oRIJnf4CZ6dborN6ZQ+MlSxwrvxEfWquy4cWSoRhVbBGMny1rRs5oQZa+t+0m0IoRMHdnCfhMuURuBGF7HDApQp8maWiP5bB1112qFXeuGE34FE+uwz1+nlmkyIm+9UQCuutnNnqq21G/nu9dxeYz99Hvfd/+9nHyXtmY52eZ4HqqjFbLapXWVvFZ5Dm0z/tWSAWjJx3h+Rfa4gK/uzi0Qc+UtAa9aKxNw3gw/OU86/4UycaM98yqvpbGEd8E0p21uqKsdIFtvi+5q174PsHA+BQK6GggcwmPrSpaWhU7TN+6YJlny1CnzYN4s+zK1sp2wbwZaQMs7WippIg21Epaj82ZsR7pZxk7grnZDmalZwj+/k/zIS93i7lUu8y+/rdfuQv8IWp0HDxeoOHScuRS0EzUYZ1m2OWDOroYACukx9yP8RfizirqzgnqzkvatWeV5zgue9COCil3Mq3jKji3tCwAVNzG3Yo5wMX0iujqI1H/CSJOHdPlLUkkv4MJ3+smbXMq7YfXowdU8To7R7RDp5JU3Qw88dRuimTrU3SRO3xNnT2/ijnMF7XQsj6HCjP1f7uNyl56ObyZhSSLu47xdHnTi0uPo3gcK89448944L2NQ+SzWdmAyhB7xeBJ/3HoqaixmcywmhcCNE6M/+sDpJfMX+vuFxQtZHUnh/Wd+7QD9+qX2D77lmv66vf/jJ9tb0SR8LjQWlvePvnl/u4alX53A7VK1Y1r+858ci1nW3/6anTGZ6kR2rczLnsMEdSZgezwasOgmWH/365bOvehxza6fzxxAFlv4y/bOt769feieQ+00Y8XnVt/UnvnCF7YXfdntaLRPtU+85zfbH991rl37N1/RXnSTvOD79LE/au952x+1FV/7P7dvvOFgu/cPf6X90rvva8dXrW/bngpAOfZb7c7b/177jmezqhhy07Qn+uvx8lMDFqPUcsuPDUkhIaAC4VPbfJeXtau141UQ6KO2zmXQLR1yfNSPnbICrJ1sQArx5dnuOn1xftJ157keTSGjagoaPd4ScNRAKDwUsHAEW+EzWgnMCBQiCJC0ZJsmMwKCcq882c/77JyILE9JPhKGVMlhBC0BACJW0V1ZKf+kHe0EAo5mUQIaeWD6mmcIEDqwMG55ZQEcd1lPhCsFuRXShNZCM43pClEKTTUqHwFJwYh4PUswGoBFeMz9wGsSmPCn86l/0Po7rx4pA8shZTuAC8EBZXoS4c7dgt1U7xDaiiOsw+8StGpvzMN66FgPz+eg3frgZnquGnUq9UGBH0EMofQkQMG9D+ahzwmouxj93orQt4kymid8wAW8j1BOGAqKOoYgjImOo+QK9k7m9llBt2jWtKRAg3VIegwfQZR6p2CcXY7hrWYzmafBHJFFBGfDW0bb2SF740ZWckIDUlqHEkQtswih5E/+nSdfCrQCSc16nLDtHhUuJxvTGtxKQIUMDstGYCHI7odpGpeHV5+vdix/73M/bX+eRGQB5hq2EaEAvYR4r3U/MYeCd4KKAlyuVARfqX9qDqyPAjev1RaHdks68t2dujVPO61JGn40h5vkR5BBfj0Fxpr+qU3SzA/iOIsW6RVk2FagNOm7AlvSJB1Bi+UsuBBsmDXBnPmOtgM6FZqNw7kdm5j3oUbEtm4eChixTCNhXHXJSfirB3DRBxIsVyf2R0OlP84MKkBn2g95mOQLev2TfksrJoKks4+NBO9ngvu9BxYwuWDVMgD3aedSwG/bgfSmShJGeintLCt7iXyvEFhgBrUS+sghJ1oa6usaAO4ulpF1OdknYwZ1M6Bi1xz9CH5YFyFUEF3okDEBFjpwSFv/7QlLtQ2w3vlQR3LTiSOentf+fmlCE1fiISYjS8SD+/LIB+fE2d/hX354jNOKS4+Lh8nt4NcwY/9D1Fe99HSWpEXcce8JXC0G0ixKr+xJenoa/Tr1WSHLve6XJ7k87spfxfnK7/zuaVSf5bv0ZfYdHFficc9n72c+08nbHz1wsT1p17QP/HSy5XyJf8vk5uOs8vS0W9hxHm2E/buTnl0RyWVgNVFydSWfv+nZW9rt7EtxJxvr/SYTpl2d6XMBLMyT4OF9HzlOX7WifQU0jZeoXWTux6swh3J/ja9mvsa1ABBXrHo7NLrXxg994zWEK848GrBwX4yfII45gIvL6O5hDwzz5zK2R4j/azDXmh2frxw41e58xx3tbXetbbc+9Wntide1ds+fvK997NSe9swXvLA957bVBSw+PgCLG+UDwOJOgMVvAiy+5n9p3/IELAVOsiTzwePZTuHcQ3/Q3vqx69o3P/VQ+63f/at29MLW9tSve3l78ZO2Y1GyvNf69Pjqt/xR51hMNBajNOyHeofjiKhCmx9/hRQFEb86vlegXu3KJXzASwAu4Sw++OnCjh1wP3VzFHzyZ3wGMFbuhy6Q+NEi8OBpWp4RJmmZ0RSQZo2WlvDdhfBoURDyYupA4/dZOhMfCRX9fodZajVgwFFXTWfo9HGTrgiqCA3noNV7hTVpVtiMsETazreYR2BwB+bQo3DjiXtGTAUtxGt8joAfP7nIRFg2KUOQMh3nV8wzEr6JXZvXY2rj3hYKrgq6E2DhqKfCEOmOTwWgCEhDLyVvfO/h/aMd9SHUE2VjGXB6VUA6w+jrSeZUHAVUaBZ1lHtBg5oIPMb8Sdtvoz9GPXgYe/mHMEE6gZAl2NgMT1yAIwAFAZbCz3yM6xA6XTZTYKG2Yz3XmEVRJgEXhAnN8FhtgxPCNZNSk1G7Rjuhmk31EHh93wXp1AfLFf6kThkGQVAzHzfXO8L8DU8/rGqVXEp1+7ZtbTPzL5zjEj5TRu6KLtAzPgXD8Eh+Erej4O6+rd+cPFu+ARawUd7FfCb1Yvhi4D7h86MVxDL3sX/vJ+covzK++wu/fObPvBdg9774cwEQlmVnabMCJTUVpzE78xR4ZUUvwiXPDggodMMLosS/czAwScNM6vgiH23qLi0x7VL/nf96drnd9QA1J2854OC7LsgYn7xbB7iEcDogd3KvidrOqzilNsS2wLskTAqmIn36i/kTtDl/yhWdBIXuWWJbPk9diD+ACSoJ5i+gocCfwEIwkQED8qTfgAvcMsfC3bWhu2su+qBFisPM55D/pQU7hpnYJ9j07p5HDrb72FH7BO3CdnKxkDn5LfPN0ijYBmnX1AVXfxJYgGKZtA3o4tl2JLBws7xtrP50E0vKPoEN8G5xWdm51VmyeSUJdyoGYkJLd+vln3c42h5gWtozifcgk2vCUc4eS8LynDqUN0t/uj+v3Y/X7j72nfdEbz280tHD+76DI2mKloXr8jiXP18pzu4WmoaHzh8fjePR4un0JJiFrH/P4d7n7ifxj9zLTzKrp4ScuBlOl5H/8lNuy+N85Xd+D74/N0dvj6bW6Rin3PPZ+47PFFh84tilduvmcUmMU/nk907E/rU/Oto0Nzp5lnrN4cpOLrH6AgR493Nw0vbr3l1Lw/p+F5vgOeFaU6jvf8mObIJnk/jhX3yYfS02tq/7kqVC+KO9c4fvH3vL/iw/+7wvrDkW7/zgYnvXhxfbP/3O3Wi8p21r7+Fz7Wd//UAK3OVpl2+s5/u3/v6R9jCAwrrlvAiXo/2mL2Pe32jGrKs8OWfkB1lBavmhedgvve8Im27aP9bbTdjFy4dPNg9leVyz58cQBxY/3H7p3727HX/y89sLvvyp7caNK9rJu3+v/fpv3dPO3/qs9pIX39wW34vG4s4BWNxs3k63/WgsfgdgsfqlAIsnDvm9xLfqyMfar//C77dNL31WW+A6940vabed+UB7x/u3t6//7ue3x29bz2DWZ35cFVi8lsnbb3wLk7f5KFb36G99KuwoFfI04ckSjtxHINUn3jOREmFRYXsySsiH08Pv2Xk+wnZsXRhJfLgvBRaDZ9MkvXRNRM5nuj5GtmmeFaQ7sBDIKCxO7bwVjurD3gGG6+F3UKHQbYwT2iMAlL126Ca+dLAhBWEamtUsaCJUKyiRD+hTKNDUYg5Bc4MCD4KTpljGUcKnZlKDEIqbAox5j305o8AZndXcBHoUWjWv2bB+EyOyggo3HpsCi6xalDwUsOjmNoKbgAoFQvli3uRXTllVnXvvkMjSkmPop1LC8jtlQmEJoM4iHLqh3knAwnGAxRmedTMPCj0eLk17kJHne1g56hPYnR9h0vQctLjRnvtdaBp1UsGRcwfP1yEM7mFVJ9fjd4dvl9B0h24FMM2Zsh+JZk0pIcsfmgAXjrZfAFC4Y/cFAINL1RatpVGwxxUsBhBattBVdZW0ETiddHwqE7pr0vEmhF9XippnroVltQohs0zhir9EoRTADzfSN5Sp2qcy+VEjVWcJpl1oKP6PNRbyyUN6r3ToPBRTXku7R/ffr7p735/7Vb+Ws7EHVOiHMhRkaHbnnIXSAgksSpg/dapMm8pUrdqxcawEWAlk5Yk0uVP6CVbrOsKStCeYh+HSsHA376yDAo3SDrE5IqBCfgowdOv0hk8K+2lXYkzaEeEmNFE3BDmaZNk/qFW0LClMypqyx5Qu/smXwFpN3sYNtBFG+n1OPJY35wroz07aA7AIKBcwWr84U8a02yx7S/y6pa2Qq8uABQx1EME27x4VB48dbx/f9wgf+MNtP/MrTp+FNnh8aQAWGYiQbts6adj3WHecpH1RfqJFKVOsVe0M8eIJ7dnadtPWje0WVoG6jdWgds6zlLWDM5bFpLpMbixqCnl8GfpInQd3y80CmoTqL3SeOE7rl1Xc/rAfeulPvY557X1J6ppxjuLqAQLGExF86REmviQydekJ6DL2OPjo6U4DFL0JRt68LgmWTJfvJVFDp3GN6e/hxv56Okk3HvypfjTv8DyhaZL48ph87m7eVroJP7zpPOxuXj8fgQVTC4blMMc5/czuHfkXpC/f5brH5r4U9nduMOfhik3jCc0upqCsMKoiPSh9y5XfGUc2UR18Wt1t6+N4eyTG4aH/RzvUMrivhfttXIkONZ4e0vloh+k4sVxg88l2JH+0OGbujyEOHHpf+/mf+7O24UUvbi/44lvbLqv3gT9tv/pLH2wH9zytvfilT2/t/W9rv/vhfW3Dl3xbe9nT3dPkaHvgz9/b3vmrd7frvucH21ffaH6RA04daR//7Tvae1a9pH3b0+9t//qnjrRv/acva7eu+NP2H//3j7Sn/I8va8+4lkFWvX+Gx1WBxU+y3GyAhbXfk/qufbOdqh9uR7P92HuN4DAQYVcfwZ2PooJ12TbzQVeYwI9C/HkE1Mwr4ENtfDZWYg7o4Mmkyt1GNqSZ6KEjphU2OiPjuQMLBVFNWzLPAaE+gjVu+rMT72Ciz4FQQEznbn6kgegMo1Yh8yN4r7Di4XuFsoAKBCE3wVJTk1WiSGAlwrOrC61HYNjIOS+wUPAkLtPx6vuYZSnM8CfPLABX41GIUhjpo+EuOSu40CxKwS5mVIkLYUlBBbq8KnjlCt0ZIQ9/iMdnTvPH/3CVYSnG4SePS37kQR3FEwtGsGdnlw31ABOCKkdnFbIcbXVE2o72NO6PMHJ7F7slf4wVch7G5Ejb8k3QuR1eSIdC1AmBBfy6EY3FDazstBkzNTUbDtrwn3kbrqyxVjfCriEfPqeftS4gIGdkHb5lQqzAInUQd8rID0vPt6JWr19eQ7PlBs0KlgEJAYDF1wASBEJ55+h2mcpAFWEFbgqhARRDmVq3EwdlY1033PgoOiyHsavRWeeL26mDvL7Sc3cb+zcm647H2L37zQt+BBNWautuaW4KWFiPBWTnAQZqfk5mh22EedzSjqUNvhUgVhPjGvK0WYT9kwCKxWPsb+EcDJ7ldzY4pKwU+HU7DWjL0sq0A7U4qdvEaT6NU35ZfwU61n+1SelXhjI0jsz1ELhQ6J2HcktgWeCo5lfYntwrw7ZlOwhLKT+BvmZQaiqiXeJeDVTaCnSkfXBNmep/KPNeFl77SLpxuoCDdce5OUcAzc6tuGf/wbbv8JG2QF0/FxMoy6O4bz2hQnCixSK/0uOGd2osLtqOBTvSSH1xg0z7ra3k49atm9pNAIvrN64L2HZJZ0JRGtNjfG/5etRl6k864q//cI2fTiBhLgMW+pUnQ5zTeCuuXr+8Tvhk+zLmURjDeRjdJExc6qeHzVPoG71cdtvDj527WwWt/m1J8uShH/3OMD2c105DD9f99XDJUng19RE//HSX+NUx/oaQ/WU89wf9+H5KQ7gTP0M43xLP973ie6cOn+U722XnQ7+Ok9RNmuwTPD9TjYVKhtHA/jiJ2f2MAzMOfCocOHNPe9u//0/t7q1f3J7/FV+CaeGlduADb2+/+r5DbcvTX9i+9gW3tzX3/EH7L7/z/nb3pi9t3/LSZ7SdJz6R5Wbft+9x7WXf/w3t9nkSusDiLA99oL2FhQae+y1f1W5d8yftp3/4I+25/+yV7RmX3tP+zU/ta8/7u1/fvvCaze3T3zlmmpFPCize8Gb2sUgHUx1fBxF2NF1g6B1170bp6vmAK5g5ulsj+X1E177UsGcZ9Y5gTucWk42hczYORabIRLgpJPJDByfs4PDD50dbKVMhjudsiMXVDdIytwEhw70r9JcO0zCcKx0F52Ounxo5LWVP7zyN3w+9gk8m7uJXAcT3WRkKIch1+91MTrOlACPCKDCsZsRUO++N2VeBFXAQHBR63J9CYcZTwOPEcgGQX90aSTZvfOhwcwlKzaBKoGPXZ54FY5pqSJOnwvCqUZzRXkinaRgHESev3Hute7le96abg5FVXtftwPt6mv6G334McerCqQBCoVwtRoR07gUcnm6udxBzKYHFR9gY7C7W8le74b4VW6Qfmi5Rbmcpy93sO3ELvLqZ0eYNAFCLM6PN1AdBwhqkmw3kybkbmkgJMmIehbuiu8JsTKMoiwueCKcgDt6UcGXezLsjyOag88S6ViDSOlplI/81wbExWL8DHqC3zM8KHFoPdK/R5yqLVSNQEWF5KINKt7gnN3327IfpedoOlr/r/n3v0a/9vj+Pr95f/jyEJZpJ+cFb22zmUMAzhXc3xXO1qzMuICBA888w/CT/gApBbSZMU+fPAEJcsStzMowDDZAgz70uLD+1GG5kqIbIo4CZ+a/6GMEeXlpXNWsSRHRNiQkbj+VqPC4vnAnoxKNJWo00lrAjO+X5eNU3yzNllHZD+6M9egZYdFBBGYXnxpk2UX3IpBwsK94JKpxUbV2xnbtZpnuoHEXb5YZ3h6njHVScYO7RuWhgFcCsf/Q1tlWA1UX6AQil3gO84AENOu/sg6zTa6HHfGwHYF+3eUO7lSVmr2GOxRbahwshWHtr2eZp/THvHqbTy13e8ViHBT4+hgA6W479sK/wSBz68d+rfW53z11FbRI5uj8eJukPr6Rj5M3HyUGwRDRymrwbXk2ex37GaUjfkueeaSLvaSUr/nCM/V7pOZ4SlhT5T/574ualR4rHvOM6fIkqaNIZAnjpYanHo4f49WccXyc9L02Lm89HYGF1sn+fHTMOzDjwmXLgdDuAWdN7//BjbR9LDrOWLDLL6rbt8U9rz3rWF7Xbrt3QVp5icvZf/En7/T+9qx04zwA5fcqK+d3tCV/6gva8p15TGgiW/D934kC7+5G17bbHb28rLx5uf8Em2L+9gHx6nl33bnxJe9mLv4BNWZFdP1NSCfcpAQs7wy5YRkuB0KBQ5Ok7O1y1BtzQV/LHrYKYwEJBwkmVtb48wj6JKtxkF1pMa/pqUulVh86ngIUChsLKkHb/IOpnABamq0AusBBUKLRniVk+2gILhZoyo6iIQ5PmNoPQ7wfdw3zUCjoKlWospLk0DQIk6ZBRLpEpsDgD3Wou/MBkxBNBplYFUlNReymoFRHEZCQU+tQsRFMSYAA90h76NflBwCW9VWgnBBZZlQhBqJtqBDAYjvi6YOu9ZyaYQ2/eQftyUyjzl3R6emGF6fvGw49nyOHe0vPjpzsHYfrHseqAMocajBKgBBfWB8tQDcYp+HKIVaM+wcZgH8NU5COcB9j/wom58wj4AZtoJ9Zj+nQjZjK3ce5mjsVa6NbM5AzgQL7KazfXYynyzN+Yg5fueRFBzPKTl4RxQisSLDt4DxoM5g/4bAbCH+sJeajyLdCROmvWeBf+Du8Vjh0JL17AR8HcAPI6aIiQjHsBP0FgAdReFpaTvO5HiQrF1e4uH8dnd+/X3p68Lj96uO7en82TR3+e3ldZxh1edT4ERAkKAApOiFdjIbBQgyC/pIWaXVoy2wF5Ng7BggBOYJJTbWW0Rmg6cNO0yonXJ1jBy0nygk0YGcF+NeVewBcnCDQFd1c/wSTwE2gA5JpzstxN3cM6YBwud6s2QBBbi0MYZfmbR+O1Xm1F2qrlWSDcnazV+M1hjrWWCfUCi7QT2+NQRsXvoQ2Semo+RFhmWjOoiVQz57yg48wZWoQWTze8O865iNsxAIagwt3Hrbdn5QdlsYY8px1D33loy/4UpO2md2vh5XrctpDPLfRRmxmQ2MC5DTBxDeBiJwsauF+FdbxKAWZAS+RUGTPUr7RM3XUbHalzy6tOyrPyZlweQ45zP/HeI9MBf5a5R3ee3E0d6v0Q/5Iw0mkc8VE/k2BDHvqr7ifL2+Loc+rgkH6nY4n/TluPK+npY6B5eF5KQcUbX8vCp43glnqRKAwp34vqTmMxgzc6+MPrSb6mySesj5Nj8DQkS5hJjPHik15e+YrvnQT5bN98rjQWPW+f7fzM4p9x4POaA+ePt33339f2HjjGHFe+l/Pb2u7rH9d272D/rKAABniPH277HnqwPXz4BJpwBrp37Gk33XQd35pH48yldvbAne3P7z7Uzlyca7tufVK7eQffzKuY8j1aTGP3qwKL1/z0T7bXg2ZKuC/hu4BFgYou0ESYzQe7BFv7dIUIBUmFLoX0tQiTmhjZ8SuIKqR3Uyh31J0chPWpgIyC6wAuekds7+vIH4lk7gRX0wqwAMDMk5ajmJoWmLaChH6NUzrVoARY8F6BMQKTAjI0mR+8xE8BixLYdS9GFaiIpgU3gYoaBVdyWaMAoyA6AIoaXbe062Plc8ygSL9MIGqksoRTBB/CemrXntVqEE78yElzRntNC7cIsaRRo+3DyDn5FMRF8E0YhBLS0zTKw3imZz3jENriIdwpP53N5a6fOvKZhYl17WCvQIb2ngpVp9BOHEZjcR/Lbt6FechH2CjsIUDGYcAFKo4SCBGmdiBA3c4k1ds2rGd1KPJMrOcQ6t29+ATATTMzbelX8RV2UvgG8ruBsvWcR+jKlfy6V4bgokZ0gXkIt54xzUHILGDpvgkFVi6h0ehCikJo6gbZU2AuEyriIMua2kUzRH0tsyeFa0FdjTZbzpre9PIO2KN8LKvia9VhBQ//4Gz4Lye7ANOvEWRw99pp61f99/fed/fx1brp8/iM3+En7kNY/UZjAZ81P4rGAmBxijk+7guhm/QHjJOf0urZjqqdJN1BMkq8xBdAJtgISDkZEymBxQkBJUDTXSjKFAjeABadj4OIndH44/oDVHjahtdStk7ClgbL7CwCvKDU+TAnWepYEysBqCBhIztRO7ci8ytoyw5epC2yTLA7oa910QOAhat3ZeK2ZaewruaEepP2UCUz3FefU0sjUw+pg25wp4biGHnx/qQTtMmT9fMUdJ1xrpFXgI9azIAf8mc/50phqzjPQ9c50naJWbUU7py9gzq/A0C9DeBh/d9EnjcDNDznrdeWAWXmmSPdY37Cm3Kr54kfHFM+FWLpr3WLPw/9VJhKY4h1mpgOozJOIH56mClRwxteGFPiIVynIfXW5x7B1Ps0D4NbDzv2+mjtYRLfQOOkfZDHTvc4noGwchr4av0KHwwzHD5P3Ia4O/XJnwlPvA/5HPwZn68uz0enlrdD2ApS7j267ktSPh81FuZrdsw4MOPAfz0cuCqwePVPvaa97o2vj8AdoQRBwknX/b53xB1YdBt/nwtUlFlRNBZ+/BEQDeNIdy0vqVmGpi/Dx46eNpMe8aO/CEEKQz73ntmysSPPWWAhI+EDgOmb4qk5CLDAn2HtvLMyC/4UXgIcBoHJ/Gjq5NXO3vh8HyGee90FE5pnaMIluPJLYb40mcoeCIySKmD2j5YUmwdP5JgSnPAb/46ocj/npGyXK0VQVSiq8KXliHAXquuDpRAsPeM5GwpYfTQ9o8G8F+z08vBzJ59086h7iPEf9+FTmPt6LidIHg79TA/z4uEvtaDKCDeBp+v2CyyOInztRRC7F0DxETYLuxdzqL2M7J6Gd0h0jEgzYRuB8InsHSGw2AzgNMYzCJKu/38UYHFYO3aulolAeyvC2E6EtB2Um2v8Z6I35SJKF3gIMLwi5eZcQfnUJGoFiBKmBQ4KpREBBp54b54s3wAL3/NO3oePCIjdHC1gEH5bToKNmLVx7SPhnedhbvhTZU8CPHV+Qx5pTfjIuwnf4wtO4Nbfj8MNr5dcim7ikvTlNAAAQABJREFUG+IUjI/DEvnkWfcpqCiNQ+ZEIDirZVBjoXlT0pdL1KXpSmSWUdWF4o88qmf56zyX84Q/fRJNBXMvTmX5ZM0FEbwRut0gUXMn01hJOVhW69eyyR1pa7dtu5K/gn7bnu0rfMKvNLrJ4TFM644uusgB2iz8bmHFpM2A040sx+rO6y5ra5tyoYO162pvkTlWXlJj0VeEKm3FCFhY1vmr3DngIbAVTBxEm3KQensQkCyosH7LG0F0BxcXcLOOmkcHStIvCJCo46sxwVpBOz9LfbnIqdncLsDEdYDq66n/Lim7lfq8ibo9z/u1aGusx1kIgSrTR+9T4DxTslX2oXmoBtAzPpaU/fCil1P3Nw2RSMt53MynHvJuHOckrvgfAvX7XvcGmvQ7DtvT9/po78b+J2nhP+5DfQsnBhrHZI/jXxKP4YeX+k+8Q1xjf1NfQ3o9wh7Y+j9JsLczXvp+SXwVYOKV1+Vl6jJJd1R+3c8MWHTGz64zDsw48FjlwFWBxat+8tXtdW+4Ix/5aCr44NdHv4Qxevx01AphdtgToZb7Ligo+GbkcxhRVAjyY6yAkNVi+Jjb0drtVpes0FrxKwhFc8H78tHZjG+EStO0T09aCiakoTbClZkEGJ0uQxm3I/5Oynayp6ZSCij27QoxMsJUFBBLwCmNgeGkU2ChqY7AyshqhFVNjCeCBPkUQRhfJscaJ6c0Sp90ZQlOgMQ85zrMnnRz2U3pNCM1Wj6MqiLcyhfTF2wF5ETbIZgYBF4E3dKQqMEobcakDHqZELb41K/GX99C3fO99Do+THRy9HclKIQeOSVtnArrfWK3Wovj7HFxAPOQBwEXdyMM3o0tuhqMBUab2XsbzdLqdi3mHrciXN2KKdQ2ywl397lw8vchBLv9mJns1USHZ6Ba2wmfrqfMXJ52MwKp+2asQ1W3FvsQl1vuAGM1zyuhxzBq8kL5QGfuh3wW7ZZjZfSyuS7ykvJ0zotXR+zlc5UB/O98H9wmPHc0nDQsNWO27pqG6fWj860/9+tyP929X4v05GISn5qWmrQ+pJNyIYQgaziSHu6Cvw4syozJpWYByvDZfR9yIlBbdwWAmubFjCigV46WCG7+pu3KdMgf9VzNxVkBSo8rk8IBBADLBSY6L6KdEMCsobDn+NnA6Pw5AEXSg3zjtB3WKmms+EV7CyfhnZoBtRrHqEf2HbanLSzJumXThraZcwN1wzbtnhbuQ5KFDwAX0SLaNgfgrvlbwIUC/FBW5sBysl9zwzt30d6vCR+gwk0hFwHKVXzknwZqvyVIVmtRq1S5Kh59GeGda7VCsAkdgoqL3J+njqwBDG8EbNzK3hQ3Qe+N0C6gmK6CJj2ecNnErC6VKDfeDz/4SZsdv4uH+ul1KH5G7v22v/e5amh/U6U7fRrd9bQNM61WBKj6oM/ywi906drjNr1PRsuS90ZRsYWA8buxe14mJZNM6uX0Gf5WHBUP2RqxPgQNFJmzfpTfoqlcJ1QM9BjPKKLwy9DlXCGX0D7w6vu+65V6+5wc9gedx/06Tlg3aewDGJ/p5O1xnLP7GQdmHPj858BVgcWPv+Yn2i+OgIUdTAcW6RTtDBEI/CjaCXkqIGQyNR//PlfBkX1HeRXMEgcCox/oAAfirA7YD1J9KApMqM1AYCEN0/KTNem87Z7toRFSAgQ0EVLoJh0nmmoG5eZ4XQCq0F3AX4Mggv21o4p89JMmNCiwGKVmEtJZnWqBBGnNalCafOCXHGc+R1agQkDQ9EEzC+OqFZTK1lqB2zTcoGv9ACic4D3PKOo6TTYUQghrmuQ8eS0+IqDAx66p8d54PEvQnQq8EXIRdEvohQ8pj2HE3TKJEGXZlNBb1/rOdZMQEk9+vVa5etcPafMYCwxSO5QLZSNPnNStaZvL0R5hhPcRgMUDCGd3o7lw+VlXizqNH03CXCHqekDCDQCM7dYN6DSvpykDgcVegMX92rDDd9PZA/9upsxuIMwmzGUc8V41gAjBhBO956kLmpHM8U7Q4cRXAYt1S56GL/CmPpR+LIkbulO3oCtX/Ko1WpmRc8p0ABYxVxv4nzLATy1ZPGiIwvMSVlOJSHZSb4kzPJUWjp5OPU1/u5/lV2lffvQ4LsFv21PyEW/lN/WJcPoT+AU4CQDhb1ZxQxAWXAgsziFMq6nwdIdz24ES7moEY0FvhHLyCxPDx2glyW8EdDKbOmSmScvlaPuEcCdlO4n7GKDi0MICAAO7UJa1BT5QNmXiVmCe5IyP+H0+6VwGwKhCjHHKb3nZJ3OrVdDsT43Fls0b21YmO2/E5Mm5Fi7JXHS7/CyawNBvG9PEkDbd82F6MNU0S9tWJniPsLeJoGLv0ePRvLkYgVaaaTMWAl2Vcz5KEwP/5Odw8ooRDtoyaWbSNvfnTY/6vRHNxC40FU9hKdlbWfHpOnfSJg/RUBCs92wTUGFc6ezy41OO1AVoli9XOnrduazO4N125DENOb3T8bIwPYGJN2Iw6cG9+xdsJW7jyNt67rT0aLp/n8fvxu4VeafU+EjQgjJMvcx9OdXvOK68/HR+4GPlp+dKPkhfj4T3o/uBouFlD0sAAsXb4DnlKHnTwBUxTuVcYZfT7rsZsBjYO7vMODDjwGOWA1cFFv8KYPELd7wuwkmEfYSYJcCCbNdIWwkdfiQUEiLs81F15L9s0RWKEYYBAIb3Y6zpQEZce2dMl+ut6VzAFr77s/P1o91X9wmn7f057YhNs2tHHGVVWFcr4TKlEwGLbl+aanI35geuse+KMYxmGlXSgK4en4KEdDgKqbYiE7aZpOocAAURgZMTsbPyU8AB2goC9zBObjYvCl9zgAg38drATrvhBXyQL4KRmDJB59LVbohoELQQ8YwE0KDWRyBUgEJbf4GE8wACOHwe7sN/wndBWvMvz/wFXCjEFd94DP/GH/flH7vwOz/hTj2SN/35MfW3AKdmItqaX2SU92xbABioebiXUeZ7TrBRS4AF2gQS3wB9Ljfr6WpRCljGdRp+LzAavA+h8kEmgS84Ekw6exztRbshsJgjLJIy9UOhGg5RV4SHG1jofysjw55OiHVkXKi3kjjllXVE3lh+0lsCeQnm3ltgvf4qgBbAoIwsX3hfWrcy5zOu8B8G9vpu2DEfk4Y84hwfSWtw0H9/H36O/Hb3cVjv42685kNgAc9j4gUPIWZankPcphfhF14lDP7dB8SG72Rrd7h2boVLybpClGWohizLodI+NAUzjxGuuQosknd51O+pe7Y1J3Z3LYgTxF316SSTs48fX2Tfi0WABTt+umcLAOYcmg2BqG0m4IQ0BT2nqDMLR44FlBiXgEHe6+cUGgQBkdqozQKLLZva9i3Drun6oy6tBESsVmux1l3UARo8235sixBshc9ZdZe9V6DzGGDmAKDiAZaO3Yf53mHqrNoH8+5AhXQ5gTwDDPDReRju2SKvMvih1pX4LwVUzGdJWUGFp4MdOwEVN0Lv03ZuZtO7+WwKKZiGEk7L0vqXwrWIc1R77U/T6/I609+M69647iQV6wse/e1aB/2bfo6lVXRSJ/NOfg1HD4uH9Cc65+0Q3vvKRlJaEo91qB9j+nTrlCQ8cedIxPnhseivF0Oace0ul18nIYe866O7dd/S0bPes+m1k6C/+IlD9Q+dz6OQRDzEXJmv6Icweejv680SvgxOkzAzYLGEI7OHGQdmHHgMcuCTA4vX30FHqOYAwZkP4BKBic6zAws713ysvPIBd0TXD30mbSMYdEHY8H6QTdhVXvKR6f3ykEZARQTH+kCZdvrsgcHe14fRUWI1DDWi34GFmhJXEMpHgPeXHN3ms9JXjSqNRdlkR/A2XuOB7vpwuMwk5jnQqTBxitFcl5t0iVnpX4XQsQ5hZZ6R9JgzITz40S1tRWlZjEeBZt2godBMQx4UANAkS80KNuEDb0qTICM4+a+cy0uF2BJuS1tRgEINyRhYdIARgZiPeAGLAVRAi4JVzD/gRIAGbknKi/fLjss+/vETwuLT72bKhBuFQ/mihkaTJkd6jwMkNH/ah+ZiLwLhIZ7Pwk8PJ6dugH6Xk52nrKAu4FGB7QR+FuD5Q4Rxs72TPKvhuBnB8QZOzUcsFyfXHkYo9upqRpsBFtfOrWnXMDrsnIx1lKVmUnOkMaeWQ6FTrkKj9Tkn6SmYK6xGuIO28AleCS6yOhdCqgKqWifLKnMrBmG7BO6qg/LQP/89upZA3nRe9mv5WPrru/H7fr/8mrjxawUtU6ghP5aP/ymniruXS7VZ8myYAAoBgJvjMY9FcyQmytse5YO8JcbsCF1axgFYUA8Dorh2rU0muFPHwwd4JtBLHJRJCePOq6CM2PvixOLRdvzYEYAMWhHakmDGjQqz43bSrSWoBRa2OWm2bWuyuA6w6GCBQKOv3ua8ig2Y023etJHN+DAtFARJi8DCvV/QWHi1PcgYN3Q8Q96sg9VOHcBw5ada9WkB7coR6upJtG3n2OxOTQVBoolzp++z1GkHGi6QT4HXBU7rf/VVaGvgCxUFbcVcu0Bdu6g/0t4A7dcx0fx2NCtP3r6p7WZuxUbM+VgtEKqMgcPy9MRlqD6592HynPd6rqPXi/7cy91YLD//J2H1FCfe9PqxJO6lfpfEnUi6Zx8q7qLXiEdU8zphh7Tqbf12+nzq8XtdluNpkKSb1KZuFXjp8/ip523sNuLbQD1vwyEJMTccUJH0hkTjVj9Fq748p9T2kEPAKaEV4eC/HrrTlXhQqRCz9Ql6ZnMsOkdm1xkHZhx4rHLgqsDix5lj8QuYQtnhOdJbVwWUAgR2mI5D9Q4zV4VzPqoTYIEwlhVb+Mj6XmHHEdQAiyHOzrwef32sFdBJh5e6+wnyl9u4CRSMzzMCOx9zRwczUVyNgB96vxOcnc4AC+hxmcrYY3NfI/oI4oAFl7vMyBr3piMd5xByTjC6ejKna/bXqHuABaBBkyq1CTKC7CCQFG+MRwEsk0kRemKa5SjoAILUZKxDaBVcxJ4f/+FfiDauynuWRQ2wgFYEpwIMNaIe8y/cStDj6nv4XAJgAYjKn24KwKZR6SQteAP7hnR7KUyv8t2j6Mrd8AG1HGRuPaZ88BtwwVUTEZfqdA8LzaLcgfsoAmx26ua9Av4a6NCkSbrkG5uRIrjV7t3HCf+I5lQIoIsIdPPwZg+AQc3FJgVMBFjjewhhcL9LgOJvA+BxD2v/7wFcbMbPesLMUQ+2wPvNuK0HXLjKFAVYIAL+ZkK3gibCpQVePKn6G4AG37XTX7tW0xrBhUCQ8qZcO58N41njscWT8IX4Up9Nx3Tl1XDt91O+1rvl7ztfJ+49HiOw7YzaoSUxERpNm/epQ9wH3PgcwZ+2B7+62ZLai77UbjQbQ72jYFIvFdYFw32lsklbw700Z9YzNRa2GeqswvfQvrOcLeZRJzGDOnZ0oS0cOlAb6yGo68cdts849wb+n6auOHfBlZ/UzKlR3LRxQ03MZj7GWsrSPiPaFa7uieKgRW2OV22wNHq0EwEFAOMSdAkQqi6iPSGvp9SShDeWT7XXrEbmO2i4gMZNTdAZrq705FKypwFHLp0b/kKbm9wJPH0mqgAUgcUl+wFAqO8F/tK8Ddpv3FjA4hbmVvisqZ7mMjGZqcpQDdF7jqpF07oUAFJVKO8n9SFPBiC+4Z5cpb+rnx6XtcPDd4NPLj2MLyf3+pIx/Zi8sJ77cohr5KenPqS8NPwQz/K6rrPp9LCTiHteSGtExRBLAo3uh9tOYwicvs7dEjqNs/58N3015G0JFyqeKS+Kmp7UhLaBJ5VWhZlSXr4mfnnd+TCNt8LortsMWHQezq4zDsw48FjlwFWBxate+5r2i29gVSg6PAUGBUc7v3SKudYHqfetaioUSLwqgDhR16VfS/BF9OK1H/USPBDwhrjsePt9hFPTUkjx/fB58RMUOvCMcz5IjmialkJgH81fgwApgPDMd4I09S9p7uDsvAjBQDQFCgjEobCueUvmgnA1Pt0VGrJSDKYbJxBiXXPfkdiJKRTCzzoFTvLoBl4KGqGYBEu41xSsBNGYz5BOaXHWIjCtz6pQfUUoBTO/3NMPjxoAJG3dFO4UVlgGVWAgP2Oi4/X/Ye9NoDzPqjrPl2tkRGRERq61Q0GBFiDajAgCAoJiK2q3DWMjM664TJ8Z7dPtzOk5M3Om2zO2ChTgcs6ctvuM2mCxlK2o0G2piMgqhUAhyA42FLXnGpmRERlLZs73873vvt/7/zMyqqAptfD/In7/t9173333vd/vd+/vbbqs/FUFsI1UUA/R5Eol2KNLlINg+NeFo8waJOZ/0pF/8kPczvLXT1WlSQMufbWa24k29k5RUgI53+KcDARGJKiTlarAsOK3JkFzXbiwrayL1jkpcSel/B6Xosk6C/oMayv2y2BgNILyMDg+p/b4vC6Mix1Smo9o/tMVgtkrA3Gv5ME5AYf1NZuzAThwTMtppQXqrIY6QsGXer6wX5CP0WVjQgoigomRIsnXIxYyLtTWtKUvy3yQb8pPtk1zBOnD9Fn6dLqUVcbTJ73PIwxejyuABDcsxoPTxO9wnygJPMkQQ4L7JmnTJh6dwLCoxgXToTydSjjedAA+VIr7HP1H/aj1LynVNixqf/QIQe1j9F/6kMsSLUYXGK3AeFiSYXHyxLFy3733lJMnT5QV7biUI0UuU3xZgVdbs1vcnHYMO3zoULnumqvLgYUFfwjYLuGuYoiwJkRGPvWgLGTvZ43bZo8WncugkEF4Xu1Iv2Na3qJGIhY9uiX6sjR8Hwhvp9qcbap5OLDCyrs7ybhYO8dWs6s6EI8RDBlBkiPPte16dujG8wLtbeoD1JlD9GwUa/viixgd4oN7ckr9dU4jE9dohOIRGrF4tKZCHdKuUKwDYn3QNtquurz3hpR6C7YEBVpYSIkwBJrsIQlogou9ETd0oe7eNkJijICPRPJZgNwv5y6Xl7jgjcM4bprD848KJVyPG5WL8slveVS01pW0xDVMMtto5r0WCI2G4SoR80lCX9cIB30D158KYy/CeT8CAMUKUeGrR51rAxlG8b8PU6Hef/Qz2tDj/lFZTGITCXyZJIC+8HfBXTdzsDxx/rq/C6x80Tyw0RC645fqtjQsbvqlV2m7Wc6xCAWpfSHVA5AHdj60m0KFcqGXNC9cr6/QixZFHoUdBYUnLAoP6xY8YpF0xH3SQ/kJAwIFNRQy5wmZqQt0GnlWDnjAMwIQyk8o0HkwHQs8KY8uxmXDQjxM8dKXwsm8bRQSHuypcHv/efEayrqUeP3Bi40KbaG5JCUW3oHna+QeDAumyUiZ8B75NAQF6Z+vvFZ4JAOMiZyXz1dVjJoZ9toX7i59WTWulBxw8yXHgABy16tT9KjLwGdsA8ri2linQTl8NQ5DQnDiI6apRDptQRq0fZmmSbZ+o1Lk4jc9p1QcwrQDwkSesqL4jTSHIkwecLSdR3zU3j6Z24orym70GxoR2bKGgpEMDthbk9K3qvwV9YElKYWLUvLOoKAqf6dkM6M23Ud/Ul1WRe8uKX5/rV187tS1LCV2RizNCWaPDIt5yXy/2uURWk9zlRS7I1Lq9mpF9y4ZF9u1cPu8FOsNfU3nCzjnW+ys081oH77EeyoUCrTKY56+T2+mbLcfxtylMm1foJFBk0PIIvt3tm8VWfPc1pIFcMBkHB+XsldWONqB/iF42sv9JFrGMgbPU6WURthGBdOdVF+U/nXJa8OH22G48RWftkljXkXU/hj3luqtvur1CpIrdQ+jIgxw7vfWv1QePLGGylvMyghY0vqFYxqtuPOuu8rx48e83uIixpz6CLcLxrAIeJ0CleGhtrBvX7ni8OGyf+GA48jEdYBXGYPL3tpWO03pvoQf2g0eMSjOSSRLuk9PaTodh9mtqM7b1L4e/VP+8AzSc0iwMjXlY1iEYbOq0TKvq8LIUFn5PMNowbBQYR6V8AcM3Vs8C5DXBWirzzBta06G7H6NlDEN6ioZtmw1y/Q/bypAm3Gprtmc0aj8kl4dYAQNG2l9/wkKFbZ6yL5RqJ0FszNdCykQ2cGD+1ECPYD/YGDh87Jwla+sV8KZk04gmd6z0+ov/vt8p1fcvmzDUFcRYUpsuPSJOaf6yISRt8gPn/4cWJwKnzw6rSNDsJczz7/Ga6D7N2lnEjDmoBby98Gw+Ml3/mq5+dNvTxFM/IkEviIl8MKrn1L+9eNe8LCs21VXXWW99ktlfkvD4hW//Ivltbe8YVTp0AMwFH8pPHqa5rOahyPKQRoW/jIvRawZFigO+uOLP/O48aETtEIZIYzywMMXlRqfB7Z9BcKwiDgPZBR8DIu4YoEuCnwo8SjjIRZoMOWGr5MYHjmdBYUeIN5zfLGOXaWGefQoFJTNXHCMiyUpMRgWwKNUMFrB3G4MC6YloYxayYInKRmxLSnKaBgZKPsYLRyUx85Q8DG6ODaMAhegMjC805CS2ucXFUoUU6jYCtT4ivdGBS/GZmypPJRkH55H24jxvJAM9UgXwZrQvIBvcpQscOEJiPaJJMspadFqbkNlwr/ntOtF67pUwwIY2pvdpMKwYLtZTT/RhWHBVrNnMCz0BZt1FsCzyHtOsmNHHdTtY5qmwojFf9XOU8ds9GmKjOCoL4fqHZR8H6V5+I+e1xaf2omHQ8lmpAPukCGxIYWXL98o2EzhoR12aQpNjATJ6FTfJY2RojA6wjik/WIqFPKsMqUPiZ8qNosBsYzKIZR25L+Zs1KPYl8zLb9U9BFyzeixbVjUjPFpUBh1HolQ3binmPLEKAUjFBgW+P7qz+iNYOIeg2M5ilOZ0MSw8KiN+htrFtLojj5XDQv6VfYv0GlXGTFxdoW2bNWIxdFjR8sXdCLo8eM64VMLuGnL3aLt8xt0f2Doe02H20FT2WRczM3N6ZyKefd3DBfzZPYuVsPirA0L7m2MBp4/a+LdRoUMipPLumfls9sY2zt79zaNLGxotCsPuVvBoFVnwrhgS+l19TmmZCEvr+eQwN0XuMd1YVSoU2iqFdPsdvksigUZDkylRF6svWDaI+etLGjaEwYFo2yMoo0aFdlf+hYlTXKvbS2Wos41ob93a0vVngekHO1mfAXEhx1tkwSV1EgDEhABmzCZdjlfcI2GYGiToKPfSjCKdk26AoENoskaMfDTjd8bfV4PE3db4CZMwzUbWTayDPqw5lDlscaSbPVFWcwFCrgkdzRMO0BrliLA1zKgXelzPzeeAqXBDfBRnrMrjYlhUYU18SYSeJhLYGJYrGod5OxIK+pddPHiK3/ll2xY8CD0F035hAdFRM9RxXnQ8xD1hYKBEq8XMLsgeSqUFD2UMCuSfDXVi56Xtg2FRlN09TC2QtSn6cGOEomSHadwDwoaygZKJPOtueJLqpR4hf0FFYWDF4Ue9izMxuBgehYjBnzFRwHPNwFwHmWRMgrPOQohVjSdQ4u3pYSeZTcb8Y7jq7mnxQg+lCKUjfqlW35uA4scMEKCnzB6gPeibxRX5e2QskN+TMeJr7fIkLItJ5XJV2WUNxTdqTbaEYpv4OZIRdTdU6cwLDBmqjyCB7Pf2itiKQXFeEdKFnbyaNt0aqEMmjei8EifGHfuMwLIfL7gOU0JCU+b0icY0fDJ3VL4MCiYB4/Ch1HBdZY+I1pMQeMMCxZkY1SuCI9RCw7j+9xpTbeRAciaGAnKi/f3S86P1hfjJxzYVx53YL5cqznuezVFBeNjXW2KgsvUGmTrL/NuD7UXI0n1QpH2RTupv9C3aDPLUjzQb9Jp/fgQV3LIAOOwU9w7WSWu4cZgSMsrjfcsJ/wqR+SvhDqARKGBJ3oYwUzjQUnGiOJAOnaBYu1DGBXqy8rPJu75uag+B1223N09M+vF0MgkRyqi/t1IRScH+OMe50wKto09rYXb98uwuOvuu8ri4inxeKHs03SnvTog0c8HdXnuY48G6d6zoVBlSxpl2shXnXgW0L5eZF77lMtTns/AULlL6jOs62G3p7Mqn+1hL1zk+YNBus3npbDBwLL61VlGKTRShoGLvOHbu20pz0WprXcyLKyPCJxRwWF39JUpPT+uk7H6CBmtnE2BoYRTS/sDxpTqw8jZDPcrzx5lp4KfvhEQfu0T3GuIkajb3i0QUPlLG5GXLtvM6dxQwjFu3x4J36fB7kAmu0CStT9STpfToXWpEWxFEDA7AW2eKmIPQzqs2LWMiPblVwjfX1nnoBn1JY2/5mqdlTvUkxGHWgbpKRYAIr3DN6GgnTQzt+fL3CchATT61CvLUngEp8KTnzDJzMSwSGlP/IkEHt4SmBgWWxgWr/utW+JFVR+OPCBRBvOhHoYFHUAPSSvH+KGk56hFKvAol1Z2eHlXJYFnrKj54W/apPOSxxe8jQrlpxIa5VJcjFiEsTAYFiic5sPK9DACAA8o2Fb09bIPIwTDAgdcGhYYQ0xvYkpMHKDn+eJSxpge4TUCKoNRCIyDWDAqZaNO94rpFlW5l1JPOT7zAH7gQWXbsJBBAm6MsIQhYgNDMH7ZiB/K4ssvF/Lw12PhsZDY289qegc0UfK8wNZlRJmh/IYC3PIsm1pjwrpc+/SJ1Jc/8k3nkH5oq2ioAIt4pic0cbeoYd2++ok+Q5tGAfxGHgaj6qmLkQsOyeNifjyLv5ekFGNobKgv4Fj0zVadtDNfp09IibxXCuQdS0s+4IyFuhgfC1qgz+nG12qNxSN0QvN12ubzoLb9RMljCo6/2vMF31OhRFt06bdSpa3Ixg5j0Ua0dU4nC8MiDOVRxYBRLzjUj33CYThlv46+G/WP3PjNdNZMGLbWNdOjTVBhOsJCjfsGHIVrlkcxlHARwx2jQn2HMyU4nG5Fozor8jkB2yM1TPMRLPcE/TKMBdFSW5yX3G3I6l7YM7uv7NK0P0Yscl0FHwoMrz4X/Sh4h5moA0o6/VcjTjq/YvH0oqdBLWvbWcqclVExq2lqHu2j39Iu7sf68i++sx7uhqYZtFgTEyMJ4HDfaYqT7gNGC+hbjFCcUj2Pa/rVkqY0Lasf0Z881U7znlj3w+5Q9JPYJUpGpvDy+cJzycaFeGdXpx3qR9vF5zZGK2TgqFDvMsYi7Bv3z5dHa7en67R+wgcywqf+eKrQF+inGMPkqVZubBuJgrvE1aSob+TSpshyK5f3sGEESvnuECJkzMSnf5CmRIKtKxknok43ofrjvApQ+TekAOmLw9QiMAUn1/qoklw0P/xHJMDAd0UjasT6k+kgmyJ4FTbhEqaXDeFWdgK6TEXMbyRWqk7LbHW2Doi6pDMHEWnAmTfKe2J1GAOgQuY1adT6gDMO/5Lv/+ERvIcyQj9PWabfl0cafPtdrfviy3VA3is/9Lvlj+68vS9qEp5I4CtOAt986PHlJdd/88OyXoe0xhGd4Et16B08LzYdsXjFL/9SeT2GBdT1gGl/PGx4SPI2luNlyUOoXXqzWolGqWbaQGhcNhj40uiFpXqYV2zT8I8S+gdZGBbdS988BDgvkVREdunLoNc0VCUHBTHz8IcpU/CjUQsJDMU7FEmUgICPEYs6FUoKCzRxVnyl0Hi0QjxAk1EPlCKmbzAFSsASQl68t5AHtCtvynfZ4GEcyOdrrI0LwszxR1Yg+WLhOIohW4HGQlUUKb4aY1h4mo7Kdf3Ejw0Y8JUWRgWKl8LUX37KwS9ZF1HbiwqqPL9YaBDVz2HSe9e9BWkj/VPJ2jcC0O3pn2hHUt1nlBY40YMinTTyIw9jEuOBkQsrffWLMgYG8XXl2eAUAuKhcMaOWOh9RnI6IYWS3aGYzoLxsE8L9A9IKWTh9hFdB7Sr1F4pg/r2rQJFi36IAuu6RF+kOxMPw5S+IngZjO4vjHBZluorqVT3coMj84VfA0qjftwrHrXQyxn6OJcjuCjfBRsu44aDtwA2Dn0qXaXiaOBQlv4oQ5eVAcmG0QkWUa+w+YBOv17SwumzUvTZ9pWzJmQ6e9oRU/MweKFi2QiPBdy7dAbL9N4F747FGgMbFn39JZOob9YRP0b14BE+mBJ1TqNDlL+mUZNtkr9H+9SXbdBQrSpLgvDFOgou+v55rQXxKAthrY/x6N02jTthUOzYXTa0qYHGnDwawY5hjFKc1rqbkzqY8bTWWTAqgXHBlrFrdd0EGwm4vX3PyihR+cQZFcV44ZR4CaRsV/23y6i6oH7A2RZMpzyovnWtdnp6/EHOpZi1AauWtMEURKl5tJAauLUaMJkM+RFXwUnLVm7GYu0zI/A1MtLXEg6/9kH3IxMVVf07y3H9JIsKukzQyOuce6DhKrDz4nkRKfyCFbERfoClQF14hqngwZ4lEulk20UvD3D/ivw4VwmbftAfoCo/4+jQgRfQ6jCg+YqfrIGJXlJkVCALTNEFbEt1DbvYEMz7uaWogMplSyLw98GwGKnwJDKRwEQCX3ES2NKwYPH2a2+5JRSH+tTmsZy70fCwrM9nw/ilIjj8VO5RlIkDi4Llr41SLOKFF0T5jRcSLxro89LnChxFPW2qvp38onEZohsKNYo1CnwoOX2evzAr3SMWVelGwff6CnhT2WF4xFdpFJ0YRQgFPebRxwsDnoHFAPDaCiv4UjhE/7wI8d1LIKHcUQ/VQcxaqWfUAoPC+/LbsKhbZEq5isPvVC4GgOhLGMalcVhsm4aF1wGwSxHlioccmUmjAt5tVKTBURVAGxYKWy4xX8dht02VfUhCkRqP0PAb7RVxwvWdbV4TyjA1w3LgR24ct8GTVykBSpuj8HFwWk6P4gAzp/H1G0VZ+TGFDiriWu2OMYUsOKNgHYVUX9tRkadkcO6V8Ten0adZXUxfCVMRvtR3aX9dqfjj0/egy4Ux5v4gBRO55jknaaQlPr6d8DOcPiUlXcuNMuoFTsom02yAKF8ZJpkwxIKrUMaAdxkkKmxDQrx7m1n5YViwZkCGhRRtRioWNV1sUdOSuDA06Ft8Td83N1/mNReSLV5FwAr9eRkDPmhuZm+ZnltQGCMgpiWloUr5wYOIdDykYeH2EQz3gUdP2MlJbYSMYTtcyCPpUC8MC3iDh3MygnzJMIE3L6DlQ4WMibUL27X+Zls5JWNhWaMR7OzE9rPcGxihdx8/VY7q5O+T2hWK0TDfT+ob+HQ7+g6L8hmdYi0ShkWMnGnUTG2/obyLGBbkKa5Ny7zd8SNlVNy4f2+5cWGuHNGomM9iEd8q1HXLtss2pZ7Ul7jD2V8c634ie0ioQjJe5tW0pJM0iScI5VuepPVhgKojP/p64DWyjYgAW6LCTs9MMrgyrmC6Hoe0WufqDSRTsTfIQAe++OOeScf9lvVMn7zRogb4kRwlR98KP/HzuQOdkEPg91RcRm0ry5EEuaThiPl1yD+J38OTMYJTwd0GXT3F6MSwqLKZeBMJTCTw8JXAlobFy1/1Sq+x8MPe75JQavyy0AORh6hHK6g/D+D6pMeoAMeKa33wAu1FvHrBM2KRD9pAAd5ETKNNCZGqzsuelwxfEtOZnzYqgQEQRgUvoGAjymfeNoaF11+gVNqgCAOC9GQ4ecXYIB06aYhgROR6DJTXmE7FaEUs9CUNVdQKLwqdlJ9WN+pUlXvosWjbxoUNDKZbMfrAFpkxWpFTpvQasmLor7X1y7q3P2WEg6/GMiBQ7ijbRoWNIRkVaViQV+Uf06BidMZyM09iq+bzEnd6Crf6pI27rFe2hH3A+pejPrPS1iQlvKKbOsORQ1/SRRMzOpRTo2KkCENCl+bB+2sysEKxsg4qSoj8dSWekRLNtqKLmhqFEsouUldoKtRhTbs5IAWQHcGYkoIiQXkhD1VA/2bRPzUsmtEPqrHm/hayCoOsyk1ySln1cmRUAwe31AvlOo0Gys7LMDVOOGEEYBjSHK7t5LiT6uJQlZ+0jEsfVJ9hcwR/eceo0KYDyxqpOKn1DSc1Jek0p2DL0DivL/no6AcW9pX983OemiTkpvyztmJqZq7s0QLqGK3AuJJB2wzWHK0IrvwxgPJ1hRQlG4SLfKmPWio/SgCDAWIDQkYE4Zy+xRQNXxqdWFmWYeFrKe5JpqVp9GBDIxZnZUycXFkv9+tk91OrjGxdaGfUsHaD07KXNZLFdrPHF0+XM5oidY7tY8WJ7x0Zm9yD3H+MziALP2/E57ribB27fc+010hx/7JWCkPikZr69Jh9s+VK7zQmmfheUZ2ptwhQV1z4ND7/4TsDeP2TtKXr6PR9bAStg0laCZtx80E/zYTqNz7FSMsV8YE1cx3QpDcCCpiJ4CR+K9EGE/GIRh9thAjkFykFowpBJfgAPqBhhvvNUSUmzybR1anjtCKmJ4oD406EdoN3QVkY2TUnGai40GhlZ14W0fsV/nKwrSTR6PkCnvhLfuBHemoPafhvayrUQ1qpCfGJBCYS+FuXwJaGxcte+QpvN2tFSm9dPwjrg5Mnvx+Gwysp3kjKt2GhqjXDQi8HHtcxzQAlEcWOq9YfXzD61088wKGN8oiKYiVSMPkyQGnziIi/IodyjUEAf7CXil8aFTYw0rCQ76ktFR4eKMsswIMIWKGUj1HB1pGst8hD/sD1NCjmmyvMKxnFl20pvaBUygVKCzSsXFa5wW/gMg1KC8RlUHgRtnfb0ZQq1YV99cFz3VG6RJMwjq+mLA63z8gG9MFBBhgc5BPWZfm4HtSlxpMPhJz1FIz/7LuYkR+3d5eSvARHQcfhymOAijjyVEa2l4UbrdtAIhAQlr8VsmjrGK2qYckz5EF/kDIqulykYajmrlKsw1iUYcGUqEVdrKNgJ54rZVgckWGxXwrhHrUl8/sxLFBss794VEryofOQBu8IKef9R1tGXsOp9Yn4kBeY0Y+QLTWM/quQCFv5h4fuAqePE4aJ5lei4ihkav70I17D1TKQoeplo0LKOtOHzmlkYlkK9ZJOvz5+8mQ5IePitNajsIibUQTulcP7F8oBbe86p7UoKMfIiH4zNTuna2+Zmt7bGbMxquY+XvsrPJh/uBNuGBZIEIdcHVCuXM1nmhP8bWia1rrWfORuVYywsFUtbS0mREv3FXCaQrVdwyv0cXVy7fxUbFQcXVop9y9xToUOspNhwXSuaRkEs5oGhyFPoYx6nTh1upxSvTEutA+YplDpftHI307BpkHOlsZcjOJcEC67QE1p4TY7jE2r78xop6crtBnANdpp7Gr1p1n1L06Rl9BVLfGrutE5+fM/8YhFnuuvn9ZuBkvxhJxASWd8UAcl1P0T4naUTVkusaYoTf+9cz9SAri9y3Sw6at2IkcoKbqk+jOCn7z1vEBgrAzT5GeERxNsfCYfAQsneQ9WENMER1zVcoH1vQesEStNh7ufyk/WL9ChA0zijPqjeQITDcudjMjsChgL1vIan5vAR9FZ5ij+j/7gS0YTHsLYxLB4CIU7IT2RwN9jCWxpWLz0lTcVDshDUW1Klh6c+WDk4ZkvG15GftCjwHLxp0TSUF54iGNYMJ3F6yys3NUXheiMP2ahnYYFeXFFeaYpxQfDwVNUUKYpU+U4T2VTphdOGkZw/tKJEo5hgSIeinzwpLUMTNGglMovuHyhxIjYo/n5KCzgMG2JaVBcKDnwSJ04PThHYgwrZYQpGdA07/pCx7Qar6/QV1KMiil9DcVP2mxZaUlJSUkF1PyoHO84pTr3X4v5ehyjKflVHYMLWWDUVFk4jNGV7RO9PWRIeQHXvzDJG3ektZclmRWmb7mRfFWaett1YVMeIw+e2xvdTAg9ncxzvuVSDVT1JUaJOHiPBbos8j4jn12lZI7pdOPt2ppWh+rJKGS9BesrpqQY0nttWKjNcNSLr9HIrBkYbgX0WtKQj8Ipp4pj5BpOeQVsVM59vgoAJdn1EHy2a1+vPmy6VR6kC7EtzBa3IRvTHe492gBYGzCM7DB9TMo4O0HxxX+Jhe2aBnX0xHEbF6e1sBmjgmoyAmfDQqMW+zQdinEW+jUnju+RYbF7mh2hmF7EqFf0P+qZ8sp+1urgFjRpV8U8w59ihsHwwQhXe7ErF2s9cutfFphjBHEQJa3DgvG9czJsdO9td/10nwmfHcPuk0Fx3+nlco+u0zrQbkUnZa9ZVtz37Aq2o8zKcJhV22MUrLHDmOgeOyNZiMaq8i/oXvb0QZ4J6gOc1D6ji75D+rRw52RYsFZnn54BXrej9Tr71JdmBbdD7cp6EX8kEc2soxJcdzwbHFUSVTTRpjXNvYUO1lzgNlilIzd/pCAcpVRo4WVZSuFetuvJKQH8dC1LZQ7pQ/5IuSC1LPpblq903X/pIhTPEQzOdKbf+BvgI388rlQnBYetLkmsMhL1j0Tfd5UMXt6HwWXSh1LWmgpEun+bXIgl/OahqHwHU3Eps8lR4caD8hs0sAgPHimp4qbvjPozMSx6aUzCEwlMJPBwlMCWhsXP3/RyGRav9UsN5SsUMB6NPKDjAcnzkhR+PYrAw1VKvlPqgzbnpnuBrhR4FHFGLZjqktNSeMjyvOVxPOorLYoLmtWAyHneNiz0MrOiozwbFIqjXBCOuL5ESrlAseTU7SldHrUQzBoLO6V0rPgLbigHgYOygWGBMSGjQooGhoWnMgl/j9c5sPRV87JRZqmLGOfFArxHDsQH7192T3G6Rz44+yKmQOV5FExv8siD5TXIk5eRjTTqp7LDaJDhIFlT3zD4FEcx5sIw6QyLhKHsdum9T+s4jiwdyzItYudFaPgFHjfyMoykSHdm/CATC4Z2c4SEwRmtxwVGgNHuFYd4RSGEQt6mSNW+I/3Scl+v24WidLLAnkXgfHXnyzPbfKIwMloxLfmjSNM946t8yjHkY3n3siLMn/2QGSylLDKc8d53uArCfVvA0cepZ9Rns3hUWoj8WwYhBcK49AmnCMmi5/peUodDcffaCinuizrp+oQMiqPHjpcTGrE4palQZzn5WjLifuZ+OHL4UDm0/0BZmNeUJ8mH9RS7tbZgqm4z640C3K/og/QxXZKJ+1uVTatLMC4+4TD5rUaPynQ7YvhopIK1E2urOhtG9x7T/lZlCGFYLLP+Q/XYLQV+fmG/tqXVdCwZN2ySwAnt7Px0x7ET5QvHTso/qVOydQ+rT2iPK91rNK54U9l8HKC9WWvDKAb3MGslFnXPnxb8imD26N6b1nNhr/rHwrTKw2hQmBFKcOZ0n8/rfiaNtRT0pSl1LPqVKuMrRl+jv7p9uOmri/aK+LZcja28gB76kiFSaCO4Ecm+hXjBvsRR7871/aRLbn0389OHJn2d/6SO7xlLrUy4rrkdr8YzbuAnrz1t81DpBGrQ4Tfhowbxa0Yq45SZqQN3CkmeDUeByhkZlXEIVMzMTL/Lis5a69a1kYvvpmw5Xn+ybvCeYZOs7bBZGv0SbsjzVWlVDp02MSyqUCbeRAITCTxsJbClYfFzL39Zec3rbrYSa8VXL/adeqnmi8DP71p1p/nBqXwrr/gBy/SCxMkRi3VphSiK/oJrPx62MW2ElwRx+flewIe+LivRUhBScUYZIhzKfHyx9JaktVzyd8lA4AvkrBZ3zuirIyMR4CxrS8qzmpN/WodpedRChRpeow3UOddX2IBRfIoRi2ZYMCKhdQHMCZfSxEsMmmmEsL7B7zH4kBLHQtHYDYcdeFi0HQu324iF4MJYiPr4C7Ho2WCrX4uh71ELXlHIouUP8gija6CFzPRf5QeXBJUGXwqTEr4jzovQ5vH+pdkQaSuD11+8SpR2xDmqH5dGROm0s116pDmd/Mjl133FRkMaGaGsosNxgYNxR1vYuGAqmeLwimHLF+wZyX8KQ1HhGM1CfuLGwgk2QlaRBouOw3GFyfyArvld3ki6ImIrmKvhVCpy5AI/L8tVCBaN607FQEcCDkCt0ctg5tuXMDwigOKu06lPHr23HLv/3nLffffpDIlFjV6c9foK6HFy+6zOkzh85Ipy4MDBMq9F3Kyl8M5jGBUaUcPI8E5Q6mdpyFoW9J9oSbMUvIgqDSHacR8Hz+Ze6VFPGX4areAckXMrS/JjETlrLNgCmDUQGBjgcx/tnd9XZjwda9bTl1hHcVow95/U2Ria3nTfohZna/enxbr7E3chi7iTBzoGbT2n6XBze2dlpMz6dPdzkvuK7tkZjV7O634+oPUSXPM6LZvD7TjvhoPuphm5UH/h3BqMCWgxIQ36YcjRxzDraJZaf5hwSqQRS4fMcMgEhyxDZo6O/PTp2f8CLXBHgEWndz1un550Gq8tE5r08+Q8/KZXUzcXLjgXP/CQdTIy9ETEdBpIDYgGrueNlOQpapD1SJ/itjIsKpwKNH8dn0nXxaogIMIlTo02HMXzhUNWYyHxKjxgtS6UkWGjUPEuv09r8qn54LUiatrEsEBiEzeRwEQCD2cJbGlY/NuX/UJ59WsHwwIlmz39UWb5qh8P1OHRmI9Jnq08cD19Sr5HO1CAFUZ5Z9cftn4cNyxS2eKZzbsZ+u05H0SlDKtsvs6jhFc+MARQ4m38KJ3RChSAUMiDV9ZKYFDMa879nOZIszsTtM5oYeeiDlg7eeasdxRSoTZQMCCyrow45CiGp0HpKyajDhgb8AffVmIpUxenezPdCuUkRhP0pVdpzOfGsPAJzzvZujNGQhg9QbZ8Xd2p6U02kMBV2jACEfLzdJ18HSFn4TRZVzmkcRLpiRdt0lCFa6PEvVeyRL6bOGiMu/5Fetk80CpuwicleLZTO2eeQiAg/noRj/xYW1EVU/pESx9wkD99i52hVjUC5TUvKO2Cp24YExgWjFrQ9iiKGBxwAkXzIXLAus72Q2ZKcVrKAn+zMPz26cTHHeVwpTHR+5mHnJKfTKtCQUQWUKY3XzwB412hJIONtZWydnaxnLpPp13fe2e5/567y1lNieIUebbkFaOeasSIwMKRq8rc/kNlWiMDO3bJ6FXfTqPCaxDUF11nyyH6U/AxWrvgxWMn5j83YQCWPNqRaVpeTyHDgqlQG+ta8cC0LDmmEvJAYl2FwN0/d2kXtO1qN83HkkGwrSxrBygOvqO9vWZLhtRdJ07qOlXulrHBIn6eHezg5M0ilM+UuT0ykha0QP2wpnzNcg/q3lJjl2n1gwMaqbhSh93laAUjHNyLPFe87on2hqG8ose4PtSL9NYOKs9pVMhZ+sHRbtURrKlOSfi+X5lehaL/pSO9xaDfIgkx6idtUgHlnsdRPn3PjjqQy7+u5K2mKoE6AZk5gNZwBWrZJhO0DNVQCATvLckkiSW8QxH3nQkAzm+DCIrBVidVPu+3KCv7HuunKjhmYDCvBMqvGWTXYORXrhBoRU4agAIz0CHlUjfwEvCXQiilIwq9jhvTnxgWm0ptkjiRwEQCDyMJbGlY/OxLf6H8x5tf45cRyi7KWUwLQrGXgpGPRZ6OfvDy6ObZyQM//FDIwxAgnRELbycq46IZFuDqJceDlulSKINWDuo7x/KEphQBGxZ+6Y8aFh5d8HSNMCr8ZREewREuigQK5V4dkrZXxgVTQFD6l2RYnOaLpwyLDSk4AaspUzIsoEkVvYuMmMB48TazKKbKJw8WbVSIZ3BRRjxdSsqLd5wRDU9jks/uM94BioWhXueBQaF1FzZCwmjbKYMjt/X0TjUYUdpaE9r6l+Onhp0WcqBct4nSesMCvEgXFgTi37RslAS1yIP8mDPOWNpm0c1euuNKj9kXMjT9GueHtq8EHVIk8yKbFzpg+EBkPIwGdRf3lzQs2KYWxZnRi1ScMGwZsWCuPkafDV0Zn+oavqw0wgq8wYv4Q1QRDHlmOOWBn1dARr02CycufquH6tIbFYy0ZB5FE4ahTFNA/6EIOk99FXzvsgQ8lcFX/RmxOK+RgPXlxbJ8/K5y5v47y6mj92iKkXZF0iJnFrxvZ7RMOz7NLBzSdbjs0ZayU1pP4REK5XkXKPqrjArufcuEfqcy4hf24AmpwSdXKHYRr3lkC6lCRZ3VPjFdSwvItRbEO0L5/lebiX/HwVAbcXbEqoJn9SHi1IpO09YI4xI7O6kO3Cd7OGNCha+whsIjj5r+hS8DakVT5Fh3saG2Yve1Od37BzVycVD3/wF9XDjARwaNYM1rhAKjgilOnKAdoxK0e1zUJw0L6uE2cd2jD1J5/sRI5CGM6lI+7vNd+ng+8exbhE2PQLqBZExPUrphakfty0mU5vfluptEX4HvdA45OdMqDPXiv4ON9k848ioVoSTWCN2M4Bs2EYJurUJAOZJU5AvUsRw6UX6UN9AAEZ6yXXRTNEacHpRNJ+U69GEVob6Xrm8DpyUrKi5l0EquFb8EJ4nJT5yglaW6WiPkE2WyK1RKYuJPJDCRwMNVAg/KsODBiXKKoo1hgZLur+oo7qp5vhjaCwZpKIPHaOCGjzqJUpPGRSpWfiEIGZ80n1MgEjzA2zcmvfCtMOvlDy8xShEGi8NSvkO5riMWhg9Y2LFyWddYYFhgZFCHZSkpZ5dXvbATvuAX5dP5MiRQjlFcTEPl+mA7jX6wkwwOVYp86kr5jFJwgF1MeWJUQl97pdjaIBIOuzqBi+9dpxSObWjZilay1dfiNC6Y0mTlX8qVX171JadaOZ5GU9QbRZlyQk7A55UGh9uJdlFAYEFTdYB303eNRn8ulz4KFbGRlyhJld9xWMrDWapVttHWIeea1N6+xJO2+0qVuUcy1J+Qv6dBKewv2SinKBfCg39GJtIw9ogEMhIPHrFADuLlEladaMlEPsLzPz8hL2ilo5zLyWqz9Ozr+J5KJ8PChoLiUHU9XfEaRvlBcVeaDQzF2VmpKUUqH6XJh9uhuK/rq/7K6bJ66p6ycuKesnTyPm07e07riTSaodpvn9JUp9mFMi2jYmruQNktI2OXNhKwYSGF3TuQuf/FGh7qGdKIGpv/KuNU1pB3C3d1aPUXj2EMxeGEvv+pt+oRV5eOXESD8yfOyBg6rgPv7j+tE7w15WlFRsaG8rm32ABhhvsZQ12C45TxszIqTmpq4ymtxVgW/prK5UPCnBZh79d1WEbFEU2LukKH283rOeC1E0zxVH2QIkZEq6twJfCsdMjftaTfhWGHLFxzfP4TvsoAZGTQp5OGy7QmI9IgMu76JIczIfpgj2Na4mEcApI29vUTWEMh8REgMTIdKKXx3+rU4wa8syrBS+gmKfwR8pVuTUTMzWUkmApeM99yTMgkWH17qjcMNX4iQBahgIz2jXCtW61f3w5ZSvqGF1z6mX45nEFmCVn7wRB1KKtGZGJYjAlnEp1IYCKBh50EtjQsmAr1G7/5GlcqRh5Yc8A0nzp1h6k6etBLn/XXvXzA8kAN9YiHuf7qk5N0DItU/vqvtMZRvvOEzTckaPgVIHyXz9f7NCq6MHk2OsRI8BmLNgknT9ukODDdiK/W0yj+GAcyLGLx9pqmiGjbS42iwDGGU67BUNS8UA+mWHnKkqZKgMsLxpdgmpHAFCqVkdtY5ghLjLZ0oxdS2sDhbAWmVU0xvUpfVXdzAJ6NC03Vgn8pgQgwv0gjSmSQoxIRFlRNM06td4bJSyNEWW6PoC1icpRCPB1tkbiZ9qB9v3UFDbkkqTRoNlfTKdevafIj1EAsWMeAkas0oGODQj4KNkaFR7nUr4Aj7q/4KoMq0WYOmFa0F8Xb2LARRt+J+pNuPgngSPdfRFt9SHMeIBEOcMIOGSHrjPw3c+S7PjISYgqQ6qNwjt4FEfHU0jAsdGdotILpQxfrgXPABR3SpaSzQ9mGvuqvanrfmaNl9fSxsqxrRQr6qs5+WLsoY3dKOz7NHrBhsXvvvtj9Sf3OC7XVJ23UUpl6IQccPpKGD+7fdL0ckHLfZuTZyXebYRTpyhEz6seaC6ZInVed2LGK6UvratNljUSwhuIerae459QZLdTW1rMijmGhs8E1mrFNO0fNlYPaPeoKbZd7QGuodqnNz8m4ukdnV5wSXY2JeIeoee3stD/XUzBqwVayTIvC8KcfYahRJ4XTwXu2Y7YX9ePiWeX2Ar7DSXho9JaAupEAAEAASURBVOGk6fQaqZJpWSPwmWnywVO0Q4TNRspWFBoueJUfWqLJX+GKqfoCZMDos7WsoFEjzq+siR64maMCasYQGmQVedFXEiwLIC5aKS/8RrTntSUGgRbNAH7SyfIC1HXv5JIFDLWvcA1c+JWfQVYkwVuUF6U1hIF/JV2CM0p+JDbIaCS5RSZToZooJoGJBCYSeJhKYGvD4uUv9VSofIdYgZeSZMPCX97HFX29SuqDmIcyz+Xh2RwvaL4sMzIQXytDUQIWQwKlMA0L4yoNeijR3mO+GylhehFf6v2FXr550xsKFY4wiiO4frnprQCcDQamT+grJQYS6zQ8LUu7xKzqYr97lFTSOfsilavkgfI8jUZ5jN4wEmGFXmHoMTXDdMnD6II/8R7rQeralMq36yR81mJMy6DYo114uKamZmTUxN76bfvTqhybD2qETOCFenO5DMqqdZaPI6+lIQ/+Oj3bCnWlZ4SxH3Av58bzmqKwCQIvdMoeddE/Mi1e+hUGjw5QHdjuDxgPCkR/GcKZBobRjA9yKCqk2eCQD99uD8HQV0I+aawpEWDwcdS/i0Z6ZA4gNV5lFfQiDb5STukH4eE3eUdJ9z2BslrDMc1J9VSaFidI6eVaU2U4wZrweqQbh6/+UqFZs6A8K+gKr64slnM6w2JpSestljQyt6Ytes/vLLP7tFj7wBVl/+GrfF6F11TIII6F//QrTX9ShemD4/UWR9EGMgDSZb2znYHBcT+1bmR5xr3Pfd7jsCsUO0WtcSl8Tvfjii6mNR3XGigMi6OnlzQNSkaTGNqpkYeLuudWVcZF39PaDlbToq6Z31sOyWhga9h8lkhSMjC1BbHuAbaJnWUjB+Hu0RXPsrg3MSpyxMfMj/3QVvQPfI9WYNgie9Jx6bueVQKZFhCWJTIyxlie6SdclV8VfkulZwUf1VdZ2RcRdMq90VIZsJMu8xm6cHLmpW/6QHcJFTn7MDmNDuGRelRZQAF+HO1oKW5cMvQPbwYxPCEluICKYy9go8yaHkCUrv+koCgu463iPa0EGcOJ5AGXeMU3thkNnNH6JmL4m+W1FhKhzfLBJB15TQyLUXlOYhMJTCTw8JPAlobFz9300vLqm2+OB399WIfSHlOhQskmHIo2fiizoVDk1+N8IfFYZoEthoVHKzpFkZeGF1uqHPx87IdRgQLOGoQYKfGC6FSmpSy4TL1orCiqDCX5tRMvBr3cFAcmDItQJuAVKPOjkQovKGcKSVWWgG+Kew1zMjZl+8A9GQhsSYkhEQdsMeUpjAkYYISBrS/hCloYImkM2UCBH6V5BGXPTJnWLjx7dBDZHp9rgWERyo6QZQwoDC3TC3+Ev5SF8nGGS/hxHz2xyqenZ8TuJ9usSxoJbpZ/2ZemWrO9XDsqPXyvNMDfaB4JkUY3JI+/UOgUcmchg/9auexD8tOoAAy+o38ILqAtkJQv7Fk+ZFeuHYyo0wjiWjphkMb8rEOfl2FgM9/1gU+UVF2sP4i1BjHVRsMP5eKGDApNb7q4oVPFdV3UiARpFzxqoX7LKIWUco9U2LBgkbSUc23nuqTtZU9IKT92+pwMCxW8a6YcvvKacviKq8tBLdyekkHLFCj6Z8ohDIpqmI7UFM6RHHKXwWOZh/LqO7DKwRDkReMYhyyPWFBXPQeyPSiLdRU+20KH5a1oCtNZjTSc1rSnY5oCdUyGxVGdP3FCPou32fVpRlOZdmqU76LuR0YuuKdmZBhdrZGLa3QyNqMXnGHBtEbuFVlnuhN1gJ448e5O8ukH5GFEMQKJw7hggbkYNe/ZXlkP+Mfgu3ARQzBkEP2L2nDhas+oNCKt9hFlQdNi8U/FAjYBFaoUasoQY6nBwMuAYT6zWGElDAW1JiHdFCEClxUBbygiYAzYJQKS4PLZ6rVRq/Uw6VqCqRuO1IpoANVTcotUpbesSIs4iD1atEWg1wzwjFLxevhOrtl+LqiW1WRjgqM/5BksK6vsFlfeVrhQCpYqT5W05fwAZbtclTkxLKrQJt5EAhMJPGwlsKVh8fM3vcy7QlE7Hnz50I2XcUwHSuOCaT2hcIcyEtNT4mseD/d8mYZhoZc3isWYYcH7xsaFHs/5aGYqy24p2V54yxQjfW1ECUia6atUv/jq8zv4zWZR+eB4jYh8eDY/yvcZFBg7Ni4wLKQ46C8MKIyLKI8pUNCwUQAtxfd4gXaMLqClYEjAN6MvfMvlQmly2RggjPJIkdnBIlHwGa2QQjcjg2JaC2e5WPDNVCpw2BUqviBT30HJ8ygNdK0UpTEXMlaRrpuVqRpOGeHztZ43JXJSFKkZHrx0wF3ehZz7/HzZWqb5UucFXem4NbNBhdjSKyy0MjtLHnrAkAdQKjSgeh1FQwgKSRIfGmF8gIfCMPCePPdlE8ZF2yMrjM9BVo7kTxTnWNaHSIbx+zKI59VIJLM1AXgMi41qWMQXdPVHjIf1lXJxdUmLCJbKxbWzulY8OsEIxXmNVJzXVq0XZGiwGJqbmi13uTjz4cSZlXLXsaVy/+I5feHfVRYOHS7XX39Dufra68r+A4c19a6OkNX6Bjvwq75V+Q4pJOfpS8Li2X+uS20M4SiDBrALj98Kj3KufN9rwFGG/qj7muqwpPURizrP4jhToLSugvUVp1iQrbpwwvqaUPaxdaymMnF4pdZoW7Ys0D+kLWOPaJH2lTIs9it/r0Y2eHbwgcLcySCwrPiAoPJI9Igf95vudfj24nGeTwpne1KRbB+modkIVL4NDfu1bq50lYPS0zVaVZ7E7cCtQIQqplKUmhldatQhURtA8JnFkl2zBuqBM2RUAOEge9rADq/yliyS7uwszqAV3uCZAWR1CdMTqVmMwIVc6WMCNIw4BaeRqvTtKS9heEkkr4YNhL6dkoX0k+QI+czEh3al2dqpz8/wJnUJ9FqCaFSuoxoVPuQbRHr6rQ9U+sR/7Id+NEt7yH3ewSm39PtCSYMn93W12So7uek6ePBgDzYJTyQwkcBEAiMS2NKw+AUdkPfq12nEQg8XPwTl87DxhVKrsJVtFO16oTSjkPmBpOet8WqRQtcaizAqUOR5gfurX81HCeS9wcvQB13pMY0RwFfHaSkH7MjEKEEaBsnLSI0qr+ZZGcmrlXvxiDLOg54yeGD6zINqVGD0ePqEaFAHRigwanKkxKMTpENH6SgsNgLEHy9Ftrik3Bh5CcOCRdrAcNgX89cdVnxauz/tkb9HvtdWaDqU/VyjgSGiC3wUHit5MgrSwMDggkfql37KI33kkuHmm0a8R3kJ+qVX4YAfd+Dhxn3S+rYljtssLXKGvKRl+My078ZvKYrV1AwpWoMSs9sQvzlFHHUeqZHJr/nKfOcIFjjS5MfUFuQRC/1pe/oKCZZAiCHCgmkFOb9mKjnrht/LgnheoOPG84MXFG4ptdKW84WO8XABQ0KGxcWVU+XiudO+zivtvL7wM42Ir+w+cZspRZ7WJwVcZz6cPLvmE6rvuP+0dknSgXja/ekR1z+qPOIR15cjV1yhMyLm6kLt6EvBmH8rv6SHgRWp+YuskZ2laxk6WLMznWjKxPWrOISpn+83+TwHCDNVkg0VTuuQvBMyMO7Vbm1H5Z/Q6AUlQYv+Pl+NhlkZDjimSu7Rfcmp2OzwNKfRCqY6cf9y37KpAhfPDhXm6WKs6YAHnA0o7qd6T2HUxYePMIKASZ496gIN1YEwjEX/UcBphubHPPODrFr7qw7AhYOGQq0L1XTjDMl5nyZWEIZCptC/kmL4kTvQ61MJG7yVG4FtGhIxP5SfPCbMUFQlNZqR9TCvZCW8oSOSNLM+ZroWCERLryUgt8FV3iqc85LHAUhEkq8uMeHG8zJdoFlS9tcOezS4CQ4A4GX90t+Ek0YrYRIXfzJi0cQzCUwkMJHAw1QCWxsWr3h5eY3OsWhD/TxQ9fDMFyQveK72VV8v7VibEAoZMolncLwQePly4nZOP0KBiq+W8VD3g10/emW3clDiUeCthDfDIkcsmMoQj27KMZ9SUNpLXryiMMT0pRhR4esksKHEMFKhr7udYeHdhVw86yxUtpQSn3sgn9GSNKAYsWCBt40NwZ0XG/CNURHGkhJUNl+Dp6T8sC2mDx6T4bAbY0JGxm4pr1Z2MDgEt1vKkBdyV+MCQyQNC+oZX5BRfqhX/Zpc01ubdO2D/DMdXgiruWqafADkeJmTt5lr+MCNwfQvxsTdLC3z8Mnv6UjUAyM0jBNIDBfRIdGhGoVWxJP3QfEA27zULMMCX3EyjiEbBm3k0V98hoH6cvvKXXlBYJBDXlr96tQQiVMSqtWPMvq69rJswDVAXuMplVb6ssKe5qRpUBe0GPv88qlyYflEOX9Wlw6Y2+Dkas7tUD/OU+Q9WiGjgoXax5bOaTelczpI7lyZ0iLtQ5r69KhHP7pcccVVZd/CgnaCmhKPnVERtRNX1C/6xaWGBXIPeQEH38C63zvs1EpDOapbSIh7E0yyMCTYIU7rm8Q/i7RZU7Emg2pdzwXqsiKDCoPiFIaGpkBNYczr2qP7cEb3COeSzGq9FG3AgZhTajPyOB2baYbcMxjgGIjcp3yg4CMB28l6OhMGmUeHGJ3gK7rkYLwYufDIhJ5Xo6MSYQRFWhoXIYOY4qMwnQonvkIc5CsqfpwMw9VZGiG+lhICClxD6id7WKUcsI5kioEoMtENY/qV8kiG0jAiBtdh0kYjtA2clTGZvl9DI8D1W+uW7R30g3b0kyGFWoWMej5IAp7iSK95ncwikxzlBaNOaj9jsFmuqfZ5He4oB2ZLP8FHo0tgC5yEy/IyvgmVWreEoKiAmhgWg0wmoYkEJhJ4eEpgS8PipTIsXv3a1/oh2JR11ZOHYH4lb8aFXsjtUCm92K1M8KwULA9aXrYYEmFUVOOiGyEYHuyC11/gM2VIij1f+HV5xALFQApE8rBdIyQ4FBaMljBUeDEGDWAxBqxQSMEALxSa4IWvuz73AONC/LAbDTvOAMdXa07fZbH3Lk7jFQ/Q8XoPaDJ6ojhfOTfEBlMyMCx45aGUotxMz8xoqpMu+VPa8QmjYocOwbPB41eoEFWHHaLFzlAzLORmdyjOEsCwEP2QsRQl5K4rFMFoh/zCmvJqBofgcJmugMMYFpEuo6y+wBtMZLXfPp3w+AszAcnDuZ1d/8jJNiAW+ISGliYGX6RZauojQSnhAzb7A+CGowz+a1kBVTEjAmjLd0Q/9GErtsJDYTe/Th9gY3pZlbO4yTIUcB2oR7SByov/WgeiXeGim3LJ8vFJy/RePuRlWY031xFFVpeUYA6T2zh7qqwvHS8bi/fLsFBYp2uva5rQsnZ8OqutVpdW4pwKrRLQ1/9Sji7q8MfljXLu4o5y9XWPLNde94hy9TXXllmNVHBoHCN58MOIm/lRLZBk8Bj1oVUi1VwKDhM67rHkmXwbaGpDp3VKa9YXuuSZqsrkXsWoWNL2sEfPLNuIWFzd0H0lo1337bTuC3Z3wthfE122hZ3VfTKrEQmfqSPed9OhRRhjwRdGt41tSnOW60JefhRgwwQ+hsAXRhtrOxi9gA73E8a864vcGbmg3yhs3sfCkVZlIR4FZDiXLfrIhDRoq0CSq2yVrDD4Ndl5hq3pIA1o9cYFKsgYF/x0Tc6UK35xTgPebIzDZjx94Ab+B9rio5Y5BKAersOOStX0wXAJ5Eav8gxv4LZ0hUnLovr0LNc4FZ/8BqOw61rLTs+8deUlTI/b00w8fKd3CckXScavMAnSeBkSHMoyiVwCU2ETZmJYVIFU76OfP6fDL7VJhRx9cG56e7lqYVd55BU6R+rvgLvtU8vlQ59dLt//nANldk93j/4d4G3CwkQCf1sSeEDD4jUYFuKOebFpXOQXTn8550Vcr9jCs04VkmLg0QQ9DfiSyYsuRyrsVyMA5QJlfHg58bKJlwT4PhcCo0KXz3mQMuH1B8qLL5FxM/NOP6+XPsYFfOoJbr68QFqGAWdFAE85fA1l55k1fQWNqVBSXoRnvoTHlri83YBnKgVrKRi1YESBbWr58snFV1B2fxKgFDPxIXhkgzwwPuB5RguzWZDNiAW7RpGPogZ/8XJSTHUhz4aFlL1pwTMtaqe+ysaISBoXYRjx6vWLSGTw86WEPxgekS4Q85NwsInjIU0etHDwjAMuX349XWfqZzwv03s/YTKtp0lan59lbJbewwWb0TdaumRovFqTGrHn9JofMCFv+mFvfAYwwghJwA+ywQGbX+FJC6ND7SshJt8VNIRJywZLQQ8kAMxH0k3fRTQ6EYPHkE8aQCRQ3ziVWgrwubNlTQffrS2d1Dayx8vK6ZPl7KlTWpx9ppyScr6or/soxhiqnOy+rFGLC9t14vjcfhkU12nB9hVl3759Ut45AFL913VRmdRX5UR96EekwUxyFj78xHMgFNfWFsLseQ4hVuSKioc8ud+ZEsl9xm5sGEV3LJ4pd4n/uzVCwbawh6enynU6DXs39zltIkY4Ld33lbZ7Rv42omFUDojoHY7UNJoBfIxoXcZRHPkw1c31VxXpE17wjhEhGQDvGyXuBRsX4pf+MBgZaWhE+4BnWdh38f5p8hFNcyriApG7VDaRqvSWH7WCH+OYgMlGWQRJMwptpz+AcUm+wxkSA+TSuFIoKBhMoOpDezMMpVUeDJjlNgZGyaQ8kq0Eb3wn+CY8ANvDQSvp9TwnTMsT41len9fyO7p9WrJyid8aJPjJOlyCW+uQZUKHPpSuTyeN+MSwSOmE/6rfvV+bNpwv+2d36PlSNEWSjVYulhuv3VN+8FsO+Dk1ivE3G/u9P18sf/Hp5fLT33O4HJyPs63+ZjmYlDaRwN89CTywYfG61/qhbOOgfq3jDcNDMA0KwlwoXjYupPzzdZDRBF7mNh6kuDPlKIwK1lmEgmejghcEshEszrspKQhNz41Gke9GC6wUqCymXVEOaODbsLAyGC99RlAwBPZoQTSGAfzyYD+nL7ur7Omvr6WMVti4wLBQHvzwAIMjGwiiP60vpNMyKjj/widviyZfPb24uhoWnJJthUU+xoBHWjAWXDYLsnXSt5Qg6oYCZoWVUhS27FxHTe3Q6MY0xogMDA7Z28UIh/LCmBoMi3xTWmKi6VdnlRn0UvlFNqGAVVxpWKThwqttWRPBzRck4XR9unG7vITp/aSRsH28D/dlZPpmaaYtdtxPsr8oseOwL77pGQJ1W4IIffpIGAz9C77KhjqZdijPKPOMboFH29mwkMJJn0aIWXbyG3FzGPnAJFC0kNs6ZNKzm3CUFXxaSa8sUj4KLYstGbVY16naa8tnysqZU2V58UQ5ffJ4OX7ilAwLtmNdk0GqL/oYwUxz2qEzW7Q5wL79B8tBLdqe37fgfjX0DxgUff8OPGEgSQr8OC/84MMjFjBaXfAnOPhUmnqQf8kmJfIjj5PRmba1LINiVWGPEMq/Vwu171paKXfo2i/D4hotvn7swlyZ00ghJ2LLfLfceb7oNq5OvFu+USLlWG61TNqFC9w0LgB3WjUs2oiN0r3VbDUgEte1kdzJ89qXzsBo9XL7IKha18odYohaUyj/ZhYwOf8kZPOhGQ4fDOpg0pGshhnBdweLNNOnzkAGeuBsWlaWU0FAybLTb1nBNzJp9RGMKbj8BlirdSnthGhlKAGqjc1L6IzSyBg8pDOt5KPj2XyOxRMn8cEd4aXS7dMS5xIf2Eo/ZCII6I0DAiPY5HizMhPHMIL9sR98yTiVhyz+cFi8jWGxUzfvP/9Hhy0Hnsdvum2x3PbJ5fK937RQvv4xMw+ZfB4s4Q1NVYDHiZtIYCKBkMDWhsUrbyq/mYaFHpIoZBgJOB6ovWFBGsoXaTHlQEYGoxaCy9EARgqsHEFHygT08kHuxdqC9VvUfpQRtFjfIGW9GixOk2KQhgXl4sKwiAXhPOiB8/oMGwQyBKQUohxwGN457TDDFfPRq8EjnDB04ustdKExK0VnViMOe3X4Fif45oJtGxbKZ7oShgPGhs+y0Jdi8FBabGzI+GCxN3Xz1z3xytQw3ju8jpAl06NQCGc4SZipU9ohihEMzrSAphdyS7bIM1+O1Hl4MdV0izDCUpcNm+0UPgD8gxlyc1si8+qyTUjPsvAzPeE28xOevMvB9+k9fE8v03tYwn59J6+qQgajOn0dglriWNaSeY660fey/sgl6EQd2VTA0/YEIxQqUo2KKo8qdHCUYhnBL3+tRcgjrbvgiDYxGJHORX1dO8st+wfA1CFHCbwtK9ODNHXn3NmlcpYzKk6dLKdPL3pb2TX1afqajVL1JbYwnta0p71z8z4nhb4bZUXhhJuMFYZ/ywxZIzTHLYKaHl/1SU9c8+f+HIIxGWOCF88NG9MKs5aCnaoWZQCd5UOD4Bj9Y6rTCRn7d2pL2X2616/S6dhftV+Ghe43DAsRir6f/OEL1zwgI/1RBooHsiJMHheGRW4xSw1xpHMvxf0pw133GO2HAcFieLaTRfZMjaJsGxaSexofHrlQOTBB2bmuImXnQvQTXLpAUXOCZUJw3IW8x1ODV8rBBb14BjtBdbBTPqNPOMvFIdIUh087IkmjEsz0oQCljOdZEnVkgjxxIZng2nO7xis3lm+DqW1hBHAqrOOJB6O6+ryk1XOTNJrf0epxW75o0tbpMgRsD58wfVribOlD+3I81HKhbbqblJnluXzBTQyLUWmPGxbkHl3cKK+UwfGMx8+W737KvhGEdSn59xxfL2eWL5Qj+3eWQ/uY1jjq6E/AHDvDuq1t5cr9u8o+jYj0jjIwFvbv3VHOrFwoX7h/rSwofNUBnp8D5NI5bZAhOtcdHp2aRZe4/9SGeF0XjZ3lygOMknaIA4lJaCKBByeB9RPlsx/5SPn0HUfLGc0O3D1/dbnhxhvLDdfqoFt3X300P3lX+euPf6x88u7FsrF9T1m48obyhK97Qrl6Vo+pDe22+IW/LO+5/QtledtcueJrn1GeecPcgyv7i4Ta0rB4+ateWW5+w+v84EyjwtNI9GLhJcxLl6+4PBTzsnGhuLdnVR4vb4yImHKklzZKW2dU5IPVnyG5Y33vQV3Ohkoo5SjWKOueWy26rH8gvrOegk355pGHN0qjfHiDj9hNCsOC0ZPzHqlY07zqVU2FwvfCVysUYTihnOAwjBh5wKCY0xaW+CzERjnznv+ijVEBbzFvm4cHLZz1wNgKQ4spHDs0d9x4FYatU3kAAc8Cc2hMa7ebaaZD6WI6lHeK0siFjQvkqcsvIWEFl+CHs9QoWvXk8kgFfsWjbZh2ZdkKJekIhFhLD2p9fmIg1qHUAX/IT1zguBJmM7yETT9hM977Sa+Kq/I68Gz6URG3fcel+Wj48GXFyGAqgr6s34pLH4q+LgPVdY0yDANc1kth+LVs6ygGkNC2EwI4wMR0I/yglfkBGHD8ug5JQWSyDvCTowQxNQvDXCNtjLxpfcK5lbO6NIqhrSBRinHwhbK8W9PqdtOX6lQ80nsHT61O+eJTHS0HKaQWgRDgJeC4t0JRHdJUa/MoHwRkpD9uI+IesZTPaOUZGQ8sxj6qKU9LilP+nEYT58UjvC3L2NilkZI5GRdsG8s0RO55s+b7WjyZl/BVgv6QOx8D4oOAjRhSqrypsfZti+lQCuNIa23D/VE/AritlMdzgot6pYwuyLDw7lucbq68aAtzQ5Xl9MN/REiIcO0LkdDli78GU2Ejib6ScD1y9JWgX2XtOgIT/awShFoNhh/xKI9w0Oj6Y8IHln4DdoAgPtBqdQyGI0f1Rlabuk4m5JvSWNplcRMenzIggKtlZZy8xldANJgWrYGsiaNjfJDmNidQ87IPRNJQz74ePQxwbhMHRut7CY+A1vtyYlhUgVVvM8Pic/etlV+99Vj59q+fL9/8xL0N4a/vWS1veMcpjdqet1GwpilTX3XNVPm+Z+3X2sV47t17cr3c/KcnbVTsllHBtCrei8//hvnyjMdJ+6ru3/2XY3rna13awV3lLR/ULnw1HXo/+NwDpk/SHynvbR9eKv/6xVe2Mk5q6tbr336y3HF0Tc+v7ZohcaEcnNtZfuA5+2VgcJLOxE0k8MVKYK3c/f4/Ln/ywbvLhemFosH8cureo2Xj0OPLNz79KeVrZDlcPHNn+dgHbit//smTZffBQ2XvxpmyuKg++FXfXL7zOdeX3SfvLB/5wzeXD5arypEdZ8rdpx5VXvDjzy5XbTtXPvfe28vFxz+5XDenj+L5QP1iWezgtzQsXvFLryqvveUNfrjykg2FC2UjbjNewigDPFC5/DVdN6m/BMp3ul4DORXKU430oraiVBUReIEaD1beE1bwlOB3BjSlhOdUoFyIvVPlptHgUQsp9pQZtPBFACVEDv6YmuT1EIJBIWi7QEmJWWWtBdMydOHnqeAwAG2mYM3pJF8Mi1n5rLPwugfKZEQGhQT6nq+dO07lV1MdyiVec73FLilQObKB7Kg388xRiOAzDYsZGxeaEiXf6zNYb8GoBQpWlSu44y7krVTLTbw1v7ZTbRvezCEtQGmnpBSBjJKXjnC2e59GuIcjnnD4mZdpGQdu3G2Vl/j2xdag8ASVSK/8jigKKJ0oHMEXYRzwBsN3fsBZoa59s5dxwMeXcMKUD7+0GzsPpah6HLjxPaJMK6yVPXs13NfEvMGneWwAMiJilCB4UK7K91d5G8MyMOriY0Yx8nA9Kgd/XqfDqBcjFb7HgteQdZSRdF0HfqDPfWrDIuUVfIUBEc8AchI3cYg7rDqg4HvTBvk8A9jt6YwWZ5/SaMUxRg2Vxn3M9rBHZnUuhe4tDHN2Y9qpKT8s1o5NF+j3ul+Uzn0d5eZzKKQVaXrWwBNly8dRQ1SKHeKhrm5yGum0IfwiC2TjtSm6l7nPlGzDghEKnA07lc1ibxsXOlskDQvKptT0LIOImT5BehfO3FZQc6eymwyV6xaBH8MaZfjJdJiDEr7TBI2vfmg+nK/QmC+Q5iIvEuI35AhAhoILuEyIyCWWtCnXTmXBVfSrSBr5rbxkWsPPBPmXxVWeawyNSmcE1jIIni6h2/HXisq0TBjjLZNdt648aF9CvwEH//C5GY89Xh82OvxXniaGRSdQBTEsEM2PfttBi/X46Y3ypvedtrL+U999uC2YZlThpt+5r9xw1VR50TMXdA9vKx/665Xyu39+qnzDY2fKP3naggl/TIvBPyeF/2k3zno0YmX1Qnn1W0+UOzWC8X/+U22/XQ0QDAvSWCz+AuFee3hX+ZMPnSnv+fjZ8sJnLJgmBMcNC9r//33z0bKiw3Z+5FsPlIMaMfn0Xavl98QHIyD/8nuOuD5mZvIzkcCDlsC95c/+w6+V9+98SvmW5z2jfO3VGnV775vKmz60Wq558nPLdzz12rLyqXeUt77zw+XeI88q//3zvqbsX/tC+ci7/qT82Ufnyrf+s39crlv8SPnj37i9XPPPf6g8ef3j5Xd+8W3luv/9X5SnHH9becOfni1P+yf/sNywX7pCfaQ/aNY2AdzSsHjVr/xSed1vvcEPUx6GVmZQhPWC9ZNeBK04VWXXhoWY4iEJb/ky8PQEFALTqA/n+sA2jOnwcBWOf0gIOrHIUsq7vyiGgpGGSzM0ah6wKPsmUfnkdQc/PvwKnqzI51dpbW0pY+KcFLMVTdFg7QVGx0WNqKDAY0BMsSe+DApGK2Z0YRzskJJmJR/DQgoJYeSAIoVhsobCJyUIZQTFiakeU1LuWMAd6z10yrF4Bd47UiFP4aMEsnvUrK69Ool7Vlcu/vZaC5XjXbBUH7/AVJ/ehfzjJQVfzbBwGLnUFxiyrXIBH3nFj3wRBg+XLztHNvlp9GreJS/MDoe8pJc+2T1On96hGibzxuGjG0WfSpge12VQDpeBQ3KE2yXx+yt39m1BA2VlVnD0d3+9BqfSoSz+gMEhsXHenC64HDka+AseQhEc5Jz46YNPmPLx2wV/Xfw8U4r4mi5l14fr1Sk7qpT7mUfJ6KP0Axvq0V9pZpExLf1SXHP0XRsGXTmhXAc/NjoqHxAJOpVPUbE8Ky5hDAs+LKxoq6qzGilc1D2Hyr5L99lejU4s6N6a1/3BrmguW/fPBY3KcP+4L9vgqMYEdOG30q8xpcR9gVFBbdyt5dNC2wTrPq+M6N0AwXetN+2pe5BpZIz0MEUKQLc9zwSMLP1xuF4YF5yBEfgmkav2zVal2cLE83Kx7V6gEGQZDh7TKVQj0HfQPKoWLjDKjj4FbL23Wx50gAmf36H/wYMzSA7WalmOdnn08XBIL3DoN5Vwzdvc68sDPkvsywYm4z18wiZlw0BDVw9HvXFmaQDO0OX9imeArr7JC+njvPV5PeHkFfiUVs/nON5IvOMfmpMD8nrJhmFxv6YljbtnPmFv+bYnzenDQ0j81g+cLu/8q6Xyf73oymZsgPPvNbLxhWPr5We//6rsKuOkyu2fWSm3vOtk+Z+ff6g84khMacKwYDrUT2ltB9OhcIyA/Mzr7ilPevSM13eQNm5YfOrO1fLrf3K8/MA3HyhPuH4PIHZvuf1MeetfnhkpI/Mm/kQCDyyB+8o7f+3/K+/f9bTy3G95enniVaXc9xdvLm/64Eo58qTnlOd/45Fy73tvLe/88LEy//TvK//oaxjJO13u/Kt3lz958yfL4Rf/T+VpOz5S/ujXP1yu/+kfKE9a+2T5vVf9YTnyUy8ue/7zG8tnvu5F5UVPOqKpgaHPPDA/W0M8gGHxi+X1/+kWP2BT2eTt4GkCns6kCA9TXdIPpADgc6PXaUlWyFJBUXL3cpGa6xudh3IYE/F4jm0vg2mm7bB4m9EAK/JSMvzwpjzK0pXGRWwDy8iElO/65Q4jga+9KDb5yOeh3l8sJl2VsnBW875XNJVkfV1KhPiOcyowLFhQLYWHsyi05oHddjzNy0p+fK2GD0pAcVpVeWuUyxxtlQXf3kVKi7gxKnxaN1OpUJT0x5oOT70Sz0yrmmYthwyKeebFY1hovQWjFsyZ90gJCqJgvVbDskbaIfc0JqxAuk1CKUuZIdUIO2T5VxJSOuMBDa0IBSyyMk4g8zviyEs38sLMxDE/eUnY9BOsp5dpvd/D97Dj6RmPXhUUkHdLb/1AeQJCgURRHJS8qBfwVi5R/sAxqdqfBELcNE0vyokWkRyVnzy63mQP4nI/dXqXFiCRAN3LXVb6a31YZI4SHjsbxXQdvrRjILt8MUKfoX/QNxhhwydPRchZMgQGl2XXMlIOF+vuSTZ2gDar/FReHQplMWWTdUA1Zwtc1lOsSd6UyvOCLWP36L7aI8MaA5x7mjvKa11UB2hzv7gU8ZWjCMpwmmofcjIkqYMDxyyKmbhLleA64wfPDTrl5JHImO7IfRF9g/6BsaN7W88Ln3Iu3jA4cNTxEr+VE3U1AD9mKGMhKzNVaahhnJlgJEeSUgiYLj9kAMpP5tV0ecjXsIapoIDLGUqEfbfX/MxJNiKeeD0QdKEQbghF3H0uM8f87AsJE31wnMIYkqKGUJngJ66hqqxGMJI3y+pS2qSM1CbhKaeG+zLGy0wYyjT/BKrr6WZaD59p9ivvPc7EsBiRkEcsaJJ/qlEI/GWNMHxSIwDv/cTZcr22nP2J7zjktvz1txwvXzi6Xp77D4apUVD6xB2r5bP3rpb/43u1G163juL08nkbDkusn9DIxLs+ulRe8ryDnjoFHobF6vrF8i+021Pvfu6W+8o1mh71wxqNwI0bFn/2kaXyhzJynvt1c9rsZWjZo1pv8T5tTfuiZ+4vT7phuic5CU8k8CAkoA1Obv/98ge3HSu7D1xVrjy4rdz32TvK2fmvKk975lPLE68p5bPv+AP1sfVyxbf+j+W510NS51d94rby9j+4rez4zv+1PP/w58uHbr21fHrvY8vVO4+WT3x6qjzpG5bLhz/6mPLdP/zUcvVu6c4PgpMHA7KlYfHKX/7F8gYZFijOrGew0aB7ha/y/krql348XKUb+OtivCiA0VfUevFA4Mp3AHSaEqywX7sodeKYhzBKE45y07BAqbZiJHjoUA4XX/DJi92f4sA6GxaiwdoJGxdS9HH5gO996sKIxbJGK5ZZzK0RDN5ibDM7pR1pWKy9R0aFd4PSaEUuym7GReUD44B1JOy7v4bypfJ5rDCdijUZLMLGMIltc7VDlA0EKRUCko4VL04pf7tVBjtD9YYFU6LiVO5hfQeKIYLgz/KsiiMySzkhh5STBaCfjOPj7FmeNc8UnWVYZAVkwAMYefmbdFKmmX45P+EzfzO8cZiExe/he7jx9IxHTzJmU6YaHWWifKEbplGReAC3fikA0k3L8gg5kAaMlXy1v131EGyKSmCWG/FKxaA5kpH3FYluUXADuNU3+bJCT7nJk8LmQ33PxkVVfi+oz4cCngzVvqB+k30vCqk8waS/uifXIWvTdhmqKcaXyvF0KzqtQD3qYn5R2yuOMjLsilJr06cs8rjnw1EaDzP6rNdQ6V72hwT6tyDXNfWIcuE172t4QI4pK7dTlUdQHf3NGsWgQpVHMNjk22PQrzgoj53eUlYhB557Wl/BdCiMC93nflYhH/56mi5GP/53hmEop++3xAMUQIdIcn2Tb1IjnCkAjMImRGVitF7uTA0CZGO7TRQe+IGHEcqG5We0fICG8odQBUd+DXMIZHnpD+UOMFuFEm8EptatpSVfmZ7xCpC89vz1dPtw8pdpLV5pjdAYS0ucxtdmgbF2oQ/9+A/92GaQD0na39SuUGd1RMzs6NrmB12fzdZYgPw77zlV/kKK+j+TYYGB8cu/f9TrJq6vIw59AYxqvPjZ+6VLbLOxcMs7TpaP33mu7NMaCs6eOKdpS8e1APvBGBYv/U/3lSMLOw1LGeOGxZvft1je/bGz5VHiaXynKJr7+VoXMlln0bfOJPzgJLBRTnz23eUtb/lwuX9VH+Kmt2nDlu3l8OO+vjzr6V9brt+/HobFZ6ph8Qio6nDcT91W3iHDYtt3/KvyghuWy6nPv7+8/V2fKSfLTDl43aFy9uOfKgef9tyysHZC57ftKYdveFy5/qBm5uhe+W9xWxoWN/3iq8obfusWfeHkpa8vnXzJ092BMr6hT48YDjhumDzDglcQSrWn+CgfGD/b9UQHDnzo8CWfaQ7A8xD2OgwUF8IVAdjcbpb1BzHNCaUrKo3vUQ197WTK0rS+6jM6QDqKImsmvJ2sfMrIh33vw9+qlIUVjAsMCz6pCpZD8bimdcXaCIwWeM5F4+GjEOEwYKCDkbKmxZ1pWOyWcsJBehgmeXGmxk7FfRCX8P1FXHVn+hRy2aOD9PYyHWp2r6dFsZCbXaJyzUWut4ivzqEwonTltBFkMm5UZJ2RDTwn32ZeGhcGCVK1TFMtIKG+icklL+DD2+y3L2ezfNKgk3DjMK2MmtHDjeeN4/aw5BF3WbUSxFOxMKwiSnJfoaKEDaOADQa1ScYNXwtEFsSBYScy50FYaai7dvKMi8EM4eqcZjjuhbifwrBI+YeBbgMReQ+oQU/lYVSkgdHoce9wr+Hnl3UMapcVSriE4Zuwl2OQN/OmP8BoVAxc0WvTwKir7n3TVxjYbYwkqD+FnKOSwVOyHrSRShSv+8XwQZ/6AR/5PBu4r2KEkjDTu1CAuIhb9m4XpB99CcJRpnhKgZFZHbTTtbDZos3jyvz0MfgxmqgfUx89oifYmGrGdCjd4xgX5oV2rwUmPbU7tCM58hPG8kcY1WWo5wWYTM/6GbyrFwA9TCsreYBGRwfUhDdvlQfCmU4ZznNhw4/pCD7z0h8gFKr0RtJIrgmJkz7JlkXNx+vzoJe4CTKe36ePwCYvtEGGO/o9bE+zD/e8kd7iHT0xnCzYT3zDZ04Pn2njvmDoHz/xFWhY3H7PhfKkq+I9OV7tB4pfzrB4n86OeOO7T5UXfZNGAB4zXX7jLVoncWyt/N9aRL2VY83FBzT16Uc1OvGoK8Pa+fgd58qr//TEl8WweLumY936/tM61+KIDZCteJnkTSTwoCWwfld5+2++vnx41xPLs57z9PKEK7eVO2/7L+XW28+Vq5/8nPK8bzxQ7mHE4tPVsHgklGPE4h233la2P/9flRd+dZYmXXP5vvLRP/z98s6Zp5annHh3ec/ijrJrfblse8S3lRd8643lyF7p2wn+JfhbGhbsCvU6Ld62wioLxgqrHvUoSv7Cq5cqD1uUoJ0oprp4zrIgGYUeZSAMi3j48pqAlpUHT1nS450HqrJzDrbPt/DDOqZJsPA5jAvmPetFz8WDWjh4GBs2AjQaMMuOTYLHeYoT51RgXOhKBYA8HvpwhNLEKEMaBOcEx8gD9DEo8mK+dSzURnmvhoWUHMrytCTRw6A4py+Z+EyJgj4KI4ZOjFhwyJ/OwpBB4Z1uWKgquhKalZQ1KVDr+gJ6UfLBcOD07VmNVOzVotZ5GRjze+d9WvKMpkexABxclJ40ElhATBje+8tvZpipjrzEUbA61Zn2A9d/yJa2yXyCXWRIdsiwY2lESc8X7SbZlyRdjk5PI2HGaftFPlRopFy3tmTQ2l2t09Os3S3yBUdfcR9HkXZm4CbDpqN0WtnbwtLanYxDTeMeCToBU8sHT1fQ7WUa8lYz+B5hJyTfd7RJgrmY4Anl3nTNX9yP9OeL6s/mT2HuP6eJOdoPORgcZhvLybh82suVjAIxGKBlOtQWvq0w1zCw8Cc4+jEuyuA3YspwKpQJWVm3wq6+Sg40YUpX5udmDYxEkmc5qi52lKe08zLe4YV+wH0AcpDhw0B8HAiERIN6DWeAYiWndOajRrIGEPYHAD1nPH3R/MjQ0fONM0U8goHcByIuP2nBa8oiDUzznLJWJm0TDlm4KvUnMVt2DQBfAUHAiV5zmebkbNPIrZiWa8J3mE5K3h2pdC8HQ+ktb4wH52Va5WmEdmUg72miI/mblL1Zvsvp6pyySL6qhCBeSxz1RmiOZl0+VnkD1+UQVzhppQ+BrB9+n57EnWZ6Mix++Mcz+SH3ua973sYLTH59/6mPr2qqMNfBgwfHQbeM37VcyjVf4nETlzMsbnnnyXL7Z1fK//Kdh7zVa44c/OR3HS7XHrr8zku/8qaj+uB4sfxvLzzSeH7/Z5bLb7/r1JfFsGCh9q9pWha7TD1L60AmbiKBL4sETrynvObff7BMP/sflmc/5bHliFTHcvQD5fff+KFy/OqvK8/7zieUjT+/tbznY6fLwWe9uHz7V08J4Gy5+2PvKX/6nz9S5l740+Uf3xCcXNg4W4594q3ld9+1vTz3hfvLm//tx8ozf+aHy9eef0f51V+5tzzzJ7+7POHIfIHCl+q2NCxeqnMsXvv614fCasWTlwgvqvqy1HMaowIFiMPoeMLy8kSpx6Dg8lfU+kDnQZWKOcaFv9SKcx73aViglFvxUlqMWGgqkZRon5yNMUI5aF/OlxImOijr05w1obUQTKfgQc0Cag7A43Ttjc6wyAc7ZXAxXYodobw7FNNHRBceZ+qIxR4pN1bYqb/rGl9UMSpQfOCEr9Ys/l7RrjwYKTjXVbx4VxvxD+wUBoPozohfdpcKw0IjQDJIPLLC1AqVgSKzSzAzWm8xx3qLvXNlYX6fjIx9ZUZGxi6t1chRCy/GFb+0AWVywZSVlRCT+akpzh8MC+DJ1suR+oHvGgX/kZeYlUznpSxdptLTH0AgvvnLfIAZQpfiR16WQ2wrmPG8xOv97LuZlqXTRUlDJ6Uv5qhA5qdvmNp3TMuI5EJAXgpN6YxauZ9ZGRe04lxBP+SCgR7pA7rvJ/Uvb61c7xPqFioieKFsD0q+UjCCdN9ZUa7lpELQrwNwWapfuGyf4AXexY6oR5y+5bQqD6dX2vIMBySqFfw5TEaE/CsClW/dtkBiVNiwID06KDzJerAcXE/6s+8v9owPGJ4jyFJATvM0JNeDvhujJpTluiKLVsehpuTX/2ARTjs48xHg5sVxyuTe0sV9aeNCMJxzsb4mw4KduHTPU2t4g3DDI42cyAzPaWKj1guUqGH8mkYNVsQBz3QgEPUI2i0xKEPXfGQ02iVi9Vf5PVYrrmV3uaI3ng9YygpI59f6mEQtP6kkfuIYZuxnkEdiVYCu/M3wEzrXzjdmK95mOGNFt7qMp/dx+Bun1cruARUGLmGzXmMgLZpw0EJOP/EjX3mGxVl9E+iWN7S6P5gAhgWLpp+nhdo4wp+9Z6381edXyuOu21N+SKdv487qPImb3ni/1mnF1rFX6WwKtpL9vM6fOKX1FN+hKUi4N2gaFLtFfc/T9pXHXj1VPiVD4A80wgDsl2MqFGX8uz84Vu4+sV6+7b+bK4+9ko1atpV7FP+oRkaYktXfKsBP3EQCDyiB0x8qt/yHd5W1f/Cs8uynPb5cq2U65z73rvL7t366rD76KeXbv+2JZcfH3lre+t5PlpXHfIfWJD2qTG3cUz75vj8rb3v3Rnnyj/9Q+cZDKuWiZtWc+Hx555tuLeee/iPlOw+8u/z8K06V7/03LyiP2f6B8us/87HyNf/yBeVJV+wrw9YDD8jdJQBbGha/cNPLy2ted7MVa+6G/iHJQzAU7fiKz1QoXle51oCvIVagUmEAXjQ8WqGXNMq7H9ZKN54exuCmYcFTlnyUq91cTB+qSj6GBTcnhgeGRJ6KPTPNTRxfWVlA7ZO1GYVQGF78EK8PfZQU0hilwBhYQ7lHq4KmeLOxonMnKBs+rHir3J0cZKe0MBjiSylTmFijwSJwpkSFYhgjFRyiJ6aovWlhqOSoBVPMSEdWPt8DhQgFRjDsTOOF3FpvsY8Ri7l9dcRir4wSGVAatcC4yPnoNn4QCg1jVwPNi/ajLsgtjYuQMyw2xKivMlzvJCc/X4KkZxifOM7wNeyEL+InaTwgCu1XgXqcno+eBumNV3qakKEwDp8Kfo7EhUGQCnjUzbREHIU9whBLbpRBMOuvdNPAr8ZD0rTCX/Ppg/R7w5gANGLqIX1/j6bh7NLXcvoUbRRqYjUsRMN80G/hSRcLuYMj8hRPXqsPv6T1slOJUQ01I7JJPrlXPWrhfHKAq2XiV1gFjV8hgIoEhVQZiwSx0EvcR7JvOdGFDiMtwg0lXsaFDpo0D5IHBeTICTRsaFAn3Xs5cueRC/MX6VEpUEXTpZudDFU2Q1pwCly6kKvoQI/6qEz42lFHC4mva8RiQ8bFhu5947psQZsOtAZ60I10QsgkOAo80gJayZ0L/FEqZCdQljWkOFdEGk5Xp45wy2+ULgPX95OB/47SKMORUWmZh8vQBRB6PX3Cm5bRFTeST9mb0O9pjsAnncrzA9W9yVB4wF6OVtJJ8vibwm6RnrhfiYYFs4u/1M1mfu2Pj5dP372a4tGzcFs5MLfDOzN90xNm9f4bpH/fyY3yRq29uENTorJbzOnksGfrrItvenycUcGi7ZvfFmdMQPTw/M7y/CfPeyrUj2tL2xtkbODYTYrF23nitxP1c9Nv3++D99KgYbenP9VuT//mf7hSHw15n8cC8zfrdPCPaGtbTuXGcWbGEx65x4u3nTD5mUjgi5KADOI3/Xb583vnyyMf99XlUQcvlqOf+ED5q2Pz5auf9qzyzCdeWXYe/US57V3vKe9T2td8w43l0PKd5ZMf/a/l2MFvLN/3gicVm+DnV8rS3R8qb/rLfeWF33Vj2bXxqfLGn/3jsvO7n1e+ev12TeM7UL7r+59VHr1/Rmc/feluS8Pi517+svLq196sF+rwIuQJyyvRyqle+F53geKshzUKCbshMVLBlKY2LUP8gQMdH2zHC1rKO47bji/+gQt+fVmKnkcsWJ/gKUQo0TFiIcrG9boCKV2xnSuHgWleGEoINKV0eNQEo6IaOZ46oTwe+i5PMDYsZBAwcsFkC15KKDPsUoNCt0t8onOnIu7F5HxN5eul4FA6MApi6hecXRRejFAwN/u8cDdUP5THHaI9JaOB0RAWcXtqV+WXqVaUu13pfBllOgj1jkXjOoFbxsS01lnMysiYndmrBeVa0K2RC0/TEg1k5Z2iqLwc8va/0jNO3Xqjgni0LfWuOOCqwkxUIb9/QRIm7XLO9Lr8HncrvM3o9fDj5fZ0e9xRHHIGxSvhaB96z0CDOhJHCR+UavLzAhfaiZeGQdBAaQ3ZOS444g1WdKFPf6P/ud+l0a20wbCIPpKjHPQ3+t+0+oOn06lvxBqfaEP4Df7CF+Ggj7JNWR57Ex+6D21cMHUKRnDOj6AZJehKMKYAj4Hjlqbete7ABm1BmYZga51MWniqaaUPcBCmiyM/9yn3q8hi5EIZANVnRawJgRlGBmxg1/sM2Kwz/FEHTweTLKERoyBRQhCUHGx4BKzTgHMgfpo8FHX7UidXBPIKg688ytUvUGWb7v3tGDy6P5ErW89iXHjUoso+YF152Ky40CGMg19F+B8SSR4csF28h+v7eUdU8CMIjdYYKadDr6fT02+IsNDRvBxMD09ZdpX+A+H0fGQbgN/oBLX47WWllB4+wbbit/FS65TS6nnYtFyId2WPy7MvM/kIlFFqrfweaCz8lWhYcPfEW3mssg9RFGWeQ+pQ9DmHYjPHidk8jzkED8eJ3f2CVd7XuPHTsknnKYLKg6NbMPW7x40cnvk6xEx8oD/Ma8gGf+ImEviSJXDu7vKBt721vO+v7ijHz5Wy59Bjy5O/6Rnl6x93TdH5i3IbMho+VT74rneU935aI307F8pVNz61PPdbnloedblZeXq3LX3kjeU3/vAT5dTGQnnid724PO9xB8qsRv7+W9yWhsXPvuyl5T/e/JuhfKoUHqA8k6XGNgU1lONQPkJZr1OgRl7qoQyxPoIpUxgVrAng1jWOjBFu2FSqqJC/Dgp2ylOSNC1IayhiTUMsDueGpup8xbWx4qkTjC5IuVAG+dD2yz8Ni4pDpvPkr3ikIQ7Hg6CVavHJSAWGDPxSBvzCd+4G5REX5YEzKGLUUwqhRxxkeOgzzbrksKopTlw8kKCLsoiBwQgM8sNnO1kMBRZ1sxsN9bCapLIvUr6MDM61WJhf8OgFi7ltWIgnZAVtlMJ00VZISCzSbvLhjbCNJPGBkTFuWCQsvpFE0nQraaeb6qU/xgWvuv5FuhVewvd+wrtYGlMu0wj3tImnSxjyA60yXgGIZX2CdtCKvgJOGLlJP+iEEuaw8K2oizg4UEO61Jr8Vj5wivsPX5e6gvtKnjwfRgUGRTUq5HtjBOgIn1EK9xO1lXcp0wjVbo2i0SfVlBRNobU+8nUPWRmuac5phkUqx4HnvIavNPqFLtJDFjJEsojqO6HmhyyoU8ATl3WAcOSHQk4Hoh7IpF2Kk+4M8ukv+vc0rsQ3jvo/RoXqjAEf07KoLmVSBnXV9ChdQRE60bcpFbqxmD3WmRjIdAVXnXmuYeAzjj9cqgv1oX44wbFjFLzhI+8wLnSuBVOiDJeSM5tCCtwkYTr+QWZDLGQCzpBo+VSQTHcNxEeQGGAd3yQdiKHWiaZU6hJR/yb9LslyzPhm+ZmXfuNGdehlmvnjPjSzjoYHQGmNjqLJ4wOWv0V9LsHdRE6t/HEmK0+ZvJk8sw4JEyh9LUbbtYfrw1+JhkVfv0l4IoGJBL7yJbClYfH/vPQXym/85mtC+ZQseMBbKZVWE1MzUHCUWh/SNg70oveUBV6O9eKhCx5f5BMPJQYFBmPC52J0hgXwwGFI8MV+ui7MZktZFG4rYoyIQB+eKn3KCEU5lCQ4dhl6+aP8+yWmNOpBGDociucpUxrZIMO8anjVRoWU/uTZxpDou/7Vp95+oQiP0QgbHlL0w7CIKVCs3fCOUxsxPLtLuEyH8g5RMjDYMYqF2hy+x2F4GBB8IUaWG4ykyCBZl8KFwbGwb6EcOXDExsXMHhZxS+nCsEg+LI34MV8KUltlV188WilCUYu6OM/1HmAUrUQyIF+iTpqkZjjbINMynTgKYMbTJ/2Ldfl6hq+eTl/2pTRRDkkFKykMIVQX8IHxNCQChGvN7l73AABAAElEQVQ/SYw0IqCU5TWYQHEJ5De5KdyMiooHrprUMoHm/8/em0f7llR1npFvuPO9byTJZFRAEXBAHOh2akVLq5UqFVZNqL20RECge1X1qlr9R7VWIaNW29Wrl7W6lzaQibRlVWtBOaAWIqIiYKmAA6NAQiJDTi/fe3d8U/b38917nxO/3/vdm5lglj7yF/eeX8SJ2FPsiHPO3ieG400KiCcO9Un161r87/qqnagzzjNrc9hQYEnbIHMe7RqMLRuyi55lN78wwsvZKPmqHsQDPGREMPQy6gFqUZfUHTjQVmXK4B7jcCygiTag50NnBNL6GdNOJV/RFFE7CzSKXxDQV9Ox8DqKxLXcVVccC3AV+IUH1wTXInJ4BytGNRIGOMsBfJdHfoXIj3oGDckFPwG4LOtx+KimXkpGaF/c25WDIecCXg6BnyeAdKE7UUGdoZlKG2GGruBf7WiCCdMzGGgkxx6eukPDMClU6aMTcCIZ+pjImjxBhqTZ8wJoGrfOp3nW+SB7ygaNkpk0oWiQHvCA7/WV+MD28OAcFIpewQzyVEbFnXyVdX9j5Or5wes53//s+0vmM4bnOV38K+6JkYeM3K84PtPF2z3NeXqugbkGPvc1cKBj8aKXv0yOxc2YCLpppwFfToVuOrwp982c54p05beX+TDXHcllPGjKOPebR/B4AOiYNqqML0oY6Rj28W0KvcnXFCe+AUEeuBg6HFhq0Ld40CSJwSwZgTMf0Su6yISccbOUESdnAsOfaUyXcCyMH04NIxNxaEpUjizUKAs0IGQecqww0mN0QziSsbbLxMkZHQttTSn+0PDULRmJfCMDp2ldW8uuaGrTsraZvSK5L8pYYr3GBU2xwJxh+sW61lgcP3aindzQMJVGLnBGWGMxOgjUv+qMgGNATkZdDl3HvPVy7nAuAh7VlUNWOhuxM0V9SaJXRz5zYf/gDnxnDz993vTDtIB6GpVXMfountO0gOnzOJ+mVecVG0Z6hS55YSQrjbGceaQoK5xZafdBY6BHZAzDnCzazTiix0OZIHJO4zT6YS1jlWsARyIcjXB23V8FzBRBPia3hxMt2qtyJPlC9Tofa5RDyjC929DaEXEYcMAr+SNjGPrOpsBlwFk+rlfSWWxcnZMXR5RP5BsnRw0wuAUrJnmQhlr0D5x+wtBGM/qPAYwmGqINPetTzgGOhb8pgSPFH/giaecOWF0rtFvwDL705XK4yUfX6GBCD2Y6/rgO46llwJmwY5IyQcs6ExycPGVRbcK0KD4cyIsApkWVowW8hQXeuDAAs9NH0nRm/0MlMwy603nJ6VLBBDUXFLjjaZyRWoCZTs9jAruDUbJ4ToP0PCgbZClA6v8ZhMIqmeFTMlR8INmq17Ruyb8Xmfo6lRzgkC55ivdBstBXIwxUgkbJVkS6GMi5Y9EpZJ6ca2CugWtSA/fiWLy0veo1N+d3szBMw+gOgzSmAQ0PTlWfG21vyADPjRpje3hTzo1VB7AYThhVTP/w14O5gevuWqMVfrMv45uPxrGLEny5+cLDCUXMCKl7ODTLsYBGPSR4LNRDAFTz5c2wHAo7FUyV0jnBdWR6kmRmNAHnhq9v4yxgsLA4uwxBHh7+fofg+c4Gi22rrtSXt6572kaWj+axFS1yk+cdojwSw5ayMhS1reyKHIUFGSnAMEqBY8HOMxgtrKfAsVhb1ZazrK/QlCk+uOfF7OgWA0wyQHtQhmujylrdTCuRbIJDPowu6olTgo6iSaKtSA+h0iLTh8IZeKHUDGNKpdBWfrUDIG6jCSaRl+gzI2iWKME7zmbTgkQvRdB3TspJuvqEs/Rj21Q/LtN5lZNRafgVHacTLuoXdQ3ugjNN4jCCg45oiQeGbkz760Ys1P/C0Y5riD7GzmZb6m9b2mCABZBcD6fVV45pk4JVXRPemUx90n0dxlJL6Sn4hZGOLEOgPjgDyhjexpM2TMk7yl8OUMDH5grIiWHP1ehDaRGDiI6Bk2Xp28slM/rEgIH+cRRSXnA9YqF6e2G2KHIPoN8CE04F946p6U6C4VoADo2Eg5CORdIeeE4linfhTDoWoz4h43uNrlk2UuB6Qg62oGWXNzsxyatTyUTddGLuxXNClKlrBF30cG5n8oLChN6LTvUFYCpNmXFcgS43ZYm+TFMeIFsxgG4nZ2B0hUmjy3Fyui59eU+jpCv4kqmHv7f0BA6ydjJVWdWhYmgOcpQelFfyFM/Cr/OrYhEpnFr/1vOYhofn3LGY1sr8fK6BuQauNQ3cu2Nx802+GXNDZCSg1lRwy/TNsn+w6CbsP8WYWWW8ei2B3rrb2AaPP8FgXOFU8C0JDHvfqHV3hQdrEdgRh9EKf6BOhhQ3XmAwuqDPTZv3mITgjGMQ07TCsB9u635S9PjwHJ0KtsXtHYvYDWqBL2/jAMiYw8nQU7RtaerULtvYyllg/SXf7/DOVZK1dyw8RULwjD5gIHLAwXpU3RbkHOBMrOjtM9/fiN1/Dttgq7Um1NsLtlfW7Vzw/YrYCYo59+Knc+qJ88Hhed+dPqwYaQZZWOTtReU4FelQVAwcBhshHnxoipP8ydPwMAPGcMoHpIqBHh62KqAtCMAEXZ9O/Azwyu3TBTSN159XmhjcWfjOL2LuJXFS8MZRR65zOxjQ4o+TDMP0J58nLxWHs2hFub8HvyinTP9uU2KCR0V0Auky2j1qof7BtRBTA3VNqN/gkJ7TdskcF7S4cEFtd1KO6PF0LJaYKqh1PEyvq7akGX1lhEipk2JOW4VsyBn8Vaa0IZyHbHEt9iMrwFAfXzf0ZaWx2/WuXs69rkedD1t+RlWHX9qnjsqstqtzx6LhkYVyUlSTWGfBegamQQad4drGQZMT7rUNSlMn0x34SUCl6YC1FS8OQx+m5Rj6Ac6TnRxi0Tb9cKisL/GCtkctdC0yVZHANrgXtNc/X+eG1r4hy0rvM+GQXWGoVwJl05p/thwNO4QJeHSCAhwGaOOSW2h2hHQ+Sx+Bu88vuu7pdHUurmCWLqbp91QNk3XucXv8okPeBC3hDWXUudNd8SCvYIgrXddOwVV+nROTVzSn8/c7n7gekIf/lKvHIW2eKps7FtOamZ/PNTDXwLWmgQMdix97mUYs5FjUTZUbMG9HHfJG6bulMuKRlQ8ulfFg4K0h8INjIcOIABQ0Jx0L5aUhxxZyGOnsnIRj4RELGdEQ1SPejkUYaBCLR2OIowc9/ITLl8L9wDBHeAIaPMo4Iq6do2puNM4Q+F4sK57LGP4rbO+qj+6o7LyMhk0dO7sXPFpyVJbcooweFpnXaAUjG3qdacdL22NrtykZZeKPPBjbngNuGRkVgZfwbSTGtKtFTY9alDODU3H06JKcgtzPP59ULN5lxAK4BY1m4KTgZOBAjEaEGOaDbHAscqtc64U2QmmAJVydR6Z/B3rx4BsfroYtqyTIBELlmW7woHikzVm0fyCMv0bN9qzcEa9yRlrTZZZxBHPKbU4q6YZ44u8s/fLvI2QqZwCI6o+BjlHZ4WUdQi9onb8I5ilY+mrxLx4DPyViyhPGMSN32RedZrSCUa7L7aycirt3L3rkAhNxXaNn6zJiV+Xori8cbqv0G49WRd/neguHnvNJuVI6y2RjWUa2Y1cQPcS1FVPDKMOZCBhfedRJ5xdSRvRxVJ3nKA6GKo9zEQqNPqVft/vQTvQ5QVhP2fesD/34tNpBRrwbRfl2lnV9OYaHrx/VUX0dGJyK2JFpcqQjCIqfr0XFA+0RjmoPsiG55Q+H76oRCzsWtCcODAd1hT6yyLFXm3iHNvFj+1lvQ6tRx8Hbgr8x9KNAun6hQ4jfUY6ow3g+yFqAYKTMJtD/ZD44gMdp9F/AyK/6TqKN1zj59wbT06n6Rd06qvvJKJCqE3wq3WEO/KvsIHnM9wBes+iSZw0NOu2hJtOzeE9CzDgLoVQgBjN4TNdrvnh7hg7nWXMNzDVwTWngXhyLl7RXesQiHgCehqAHuwMPLFsDccojy/dQ8vhXHIa0DHXeqMsQYMoOgRs0f16grAe2jXu9kS1DDqeArTaZGoTBHo4F6wn0MBS+pyIJD8OHt72i6IdDrc2AH47BOLoiY0E8bSjBTwYbTgXGG2kfOsf4YGoTDgLrKhitYERhZUUjFpJDlWrndnfb+R3tBbyzZ2OKt8jMd0dW8Nj5Cmn4Z70EOzoxjQNZWOOA0XePh3rELPVEfRdFA17r2k52XR/EW9eXttlilvfB1PcSb2b9teEr4XRJtiUt9l72egs5F3IyMGwwvIZ2cRvQDlrsK9mu+uYFjaEAvP4Dj0SGLomKLS9FE/k+H3EMB5CC6VbcIdUDOuLgDTws/BuJkMd5kz9Vv4qrtOhOn/f52UvNyzYIfVH8gCGut/U+hxCyUKa/cDpIBTz5OA+qQcgKgSrL2M6FM/lJPMfRJz2tqHMswunVBxdlMJ/VyNidcmDvlGOxeRH3NL7kvqy+sqaRilNyuo/LMV1WX8dJp4HoZ4xgsE0y/YqWqfZVV3T/4yoKx1zXkIxkAujUz9ei+hv1d6tiRPsaC2Of+rC9ovuk4iNqV5xrnItDlkGE9B/NHXpxO2X7Wx4Ycm5Q/fTBggR/ALh2MNzZejbqERRY/0R9eSFwWQZ8P4UKckPfEB+Mf5/DL+vT94mCjT4g3UiGaccCB0xMsg+UfGYk2nqBwvXPyIViT4mSc8EH9KCjH0SaCAN/K9k/oe+EqvIes+ScILTfCType19OvfIcWsWjQFzW4Q3nBZAxeNOyDHQFU2mDQ++AMC1HT7eXr/L7PMjW+TSdA1i6qPA4GbQ0oayrKfQ4V5cekJM6nQUxXa+5YzFLS/O8uQbmGriWNHCgY/Gil73EIxY8KbgB1sHTipvxcK4a88hiNyOXJWw5FjgUOBacV+AmjSGDYcU0Ifae5hxedixkOOAcYLD7kGFfoyWDAQReGkEeacCgwqkoQ1/4YchDO94Mw++ijDSmQV1k2sKEY6HF1aJhx0I02LmJr1+v4lhIDvwBnIrz2zgWuzaqGGnAAUFG6iiGHp24wkNKaaYoMbrA1CX0dY/qidLZCQtjBUMNvCVNb+EL29efOtVOHDupfa83/F2NPX3Zd0s7zly8pOkVkldIMcKhhd5sObu6uq5RlRWPXngxd+k5HYyYvhb6dztgZHHQVuhbodqRLJ04L/KjjDZxyKIOJLK7jHr4QjPM2pF+UrExUHCVR7/pQz3sLWdfkOlZ+T3NWWnnJR+qlNVPI5s+gnrLaEwjLPskZ/Q7aNSBKLRfyToQTBkNB59BgclTfOBvp0L8iNmhhb5sh9cjFRfap3cutFu3duxYbKnfcP3grC6oX23Iabhx6Wg7LediRek94V+UDUv/XaI/6bCjq/pSZZzyBY0ELqicLarJQy7rJNM4+r4emAKkPD5q5VcB1kHtrBR6kjh2MGgHRitwLjwdypipE5VR3rfVkIbmrGBeKnO5Yvox9w9dX3X/wNHhA5LQQn47Fro2qi6QHfjEiWlEnuSXrmJq06SBXPjEPnyNxjTNmhZVbVkwUQXVEX3nlCgcDUZSWG9R37dwi7tOFm6QFS30stIuBNOPpH97GMoOCqYh3RAKzxjgFS7llRbcUB5I/DqYV8JBaxbvmflTsFWvotvHRbNkLXr75Rduldd5HxetPo904Tju1dgJOAu38Kbp7XdedZhZXnqXjsw22wIez/nHz5mJ8kBkzneFeiC0Oqc518BcAwc6FpNToeIh5Ztu3hBJx01YD5y4QwpI/xgZhpFxqbicihEexesBL0MGo8pvSXVT5X7LrRbjiDevZeA7xkkQXSB42+ovVYOrA7rg1PSpcETiOxGUYUTU17djdCRGKS5oUSxOTU2Jwu2xYyFngClK/jCfRgLYCpY1FtDaZLRie9uOBTs8MV2LL3/j0GCw41zhVJA+LJlZ+I1TwWiCP6Snt6sX9CaamzqBei7JcVnfONZOnjjZTm2c0IJujUCIHvC7cizYEveCtrL0g0A4LMLmGxY4FGtyLFb0de7aJQo+6Ps6jY5UO9QuVZFfbWb2/ol2Cd2TJjimHvrjP/Iijt/uqSycBBkAwC/HInBHiP0e0lBM9sHX5yPeQHyfBDyhPU2/zh2LXElOf+OM/qT/jDEdg8YEnoANY/qy4I3paPiZlhR8HI9gE3HNu4Y3TkytqSBdo2hMgfqEHIqPbu60D5xTX9M5hjzrjnaRUxxX5SQ8Zvlou1GOxbLyz6kv7wqIETfOmS4HbSTlGsTwX1N/3FC/XtfBOZfToJOUB97IgWuBc8KHn/gMpKzxOHjjr9bhuMJkQNHwNS4ezqViwOsMfVgnlOlwqJgTw0a2f+ucWIdPoc+oha6HmOonMSQD0wPtaGDE6xrBkK/RF2iZGwSSH85JXQ92HqVv4pQq9JCioBMfqjPXXL0E8OiD6xZ6s+6AhZ/4xMhKrH2CL6MobEMba0DCWUsWY9TrY8wdUsVj0J9K4FeB8r7M+VnvIT95FK3CdXnKbzpZMNCvuhXC/YzN737iAI5cM+ulslk0K6/qWzG0ZtEp+IqB60OPTz5ws+j0OKR7vKJdcV8mYkYtPY99sLXn/uDcsZjW6/x8roG5Bq4tDRzoWLz4FS/ziAUOQJgSMhBkjfhPN3/fLOuhRZx3SGAwKinnwCDA+Kg3jqEiGT0YdHq416gDN1rgvcsRhrmM7nBKlActLCGYCM9f63Ys2QQb6w40yiBDH8cCRyAMaT0YgMNA0JQTjA87FzKg7FgwciEDABlkeuRoSTgl0GFLWEYtSEsEORWMVmy3LY1YYPAwOsKWsciMfBhyTH/ydCyNsrBGgx2jqD87Q+3t8d0MTZGQTN7xSbjrx461E8dPaKRCToXWTOAgXSc5GVG5CI5jRjnibTp6XNQe+qsasVjT1Cm+f+GP5aUDEzqTztG7jcPQBXjRZtEC/EqrzgvN6lf/1U6GBYaKE4YojClnZV6kxxNweseiyutBy/l0GB60IoNchIH3NPC9nPd8Kk1sHilm5dMPpe7BgLAzkMZEwRhXeTGqNta/xBhrXjlJTxzh6UP4o2OhfJ3HAunoz/TLXbX1ltZVfPjcVvvg+a32PjkWOAj0fxyGTcm5p/Ml2fSPX15oj5BjgZNwp3C21L+ZBrVMXxT8jvoLBzIzynFSDsX1gr9BeCu56Dsk4zeuxx2uCx18LZ4pV4usd0JoyYahzLUn4j5ihDL6FE62aYlXGU7oxEfCoxm3J+cEwZZ+IyN/kwbXCAQYAXBfxnnXOThce54mpfraqZDeGFVwCLQJkh6lo//r4EVDORfIDLv4CdogDlOhRLdGlsqxsMwpey+/eXC98X0LHCEJ669y62UCH9GDzVWhdHFVQWQUr/466OkUf2s0lDNS0nnhGSdlRonT8EWn9FBEyC8alXdf44HmfUWYAdfz3o9e5RdsxZCbKHNDKy/5VNkMtnX3gYJpgDPobBYC5dmWPd0+XeXouGSY1vdzf/C5+1D/q8/2i6qUeZCtY0Me8nOtcPxVfcfiX/2Xn2u/dMsfdJzmybkGPvc08G0P/dL2wsd+2zVZseuvv17PMU3//wzDgY7FS37i5d5u9ooMDW6API58A9INB+OTB+dwgyStOy83X2D6w28cZdj4RlWCcnPlpiUDxg95ESqc3rFQZhAVbATf3o0LPrnQZ+tN3vKjDJwK5mDbkFa5+aRTQRonwuss5FTseStPOR2S44gMGOamM/pwRDTYFYovYq/KsYAu8m1u6Q2yRiy2tNYiHAs+WKYyDBaJZltIDhFrQ5aFx2gHIy6EHTkU8UG+C9blotZQrK9vtNOnT7djbCerkQe7JtzIJZudIW05e0kye2QHuXUgB1vOruNYaMeoxUU5I3IqWN+BTDgsNd+bRd3UxzotXTqmNVFttpUljDYoBwQ+s4Kz00Kehqlz6A79Q/JXPvRoA0Kfxzm5xbEv69OG2w+/yy8ePTx9rXoR+Q7KkEolE2f5ENWJ/4iTJvE4FQoHL9Ctwxl6Ap6/oBp185k6CFcNaYkj3kGXa4CvtLO24u7dvfa+s1vt/ec3219o5EL7kclZOKQpT4fbWY1KbIv2ghg/YWWxPVx9lClIt8mx2FQfXxTMomBFup3T+R2ityOngN2bHibYRy0ttMeuLbUT9E85F9epHakL8lC/PV3rHMiyJEdkSX1qARgcCw7BuD2i4kpjrPPiAMeCALFQqEGyf1XL+sWE9IVTYh2JT9GsdubcFxIx9ATL9eUF3IotcObhRHtak5wKT3ECRSEWkkfa54LXxWE6nAfvqI8dGPEyX6MoLbnCoGLEAhnzSHmNb/mCltGQSXpgmhZrLRgpxNnBqYj1FiFcimgUfqrepEcZOMsgPgVDeaUptaNLYiqfLAEa1vx6WTNfSMGvBOphjE4L3rcwLXcvY8+f/B62T/ecevxpnILbL3/fcuon/gT4Fu+eF7JW3+Eeho4I/EYq2msCJ/VmwBk/xYeikaJOkKHgk8bznv28ynnA478ux+KFv/t/t9d+8C0PeP3mDOYa+OvUwDMf9tXtR5/wjL9OET5j3jfeeKNt38+UwIGOxUt/4hX+8nbMSe5uxDyg9QBlapJv0/oZbpA8tCSNDQGlYxckHujxkLOgAvaNXVjxZjBpAy/aGOJMSYIGQY/4eAiMTJzPDzd4pv/gSOBY4BB4pEL8PMqRkBgPPIQ9YiHjIBwLto2NtQ6QxrGIaVdyUHAqZBzUiAWOCzCMWDBasa2Rh4LFicCp8LQW8WAKlBd+a0rTikYkPE1Kcvor2jIyiDH+2fVpY5XvU6yZl0SWQRPrKLB20SMPQkZAxm9+XLaOlrRb1KpGKhZ4O4qeABacR0rksECbxd2LGgHBwIm2ijZAZ7SLIuuPtgSflsMZKMcC9qbrRJSPeRjIUV4P2YojG16GNlAl+4esCQxtKvqJUHFhm16W9fjTcH1Z4VYebEjz54ZMANeCLPcNlUrv4UBEnyvxjFswNjKDAPqaDiPPwMbIr+Ay0SGIVfDNvsk0pHOa9vaprd324c3t9jE5Fbdr2py+xa4RihjJu1POwpbgF9VkT2DEQn2PNQ53yUHeVhl9nmvngljcqY9ffEr07lafYzvlh8iReLRwvnhtuT2MNT3q4xqQsE5KZhyKi6LPAm1GQrzjExAykGvLVvRu3VN196O8VlMXjLahY4qtH6oLjsHjXtA7FhjwA03BSCnWCzQIQVb9UveFuCdgnIqHCmLDAqGIBoaSGSfvSgeRIBR86O9UXPcD4cC/eEY7Z9snTd+jyrGg7ZAPTpVWXDKSQMa4J8ULB3TPV7mRz04Q+gTHgoVclayYsqx+wAaDEScBq90KbyJOHNNBVgW3WwK5FiOTzM165VnBF586L+CetuuUPCkfYJVn/JShcCsu2nU+HUNnoDVVOAu38nqcg/J6kgOcMqPHZlv3QJR1MoFTeD1Y8Z9VNp1X5z88dyx6Fc7Tcw1csxqYOxZ7WgO8OtF+emlzzz0v/fFXtJte+5owttL44QYYbynDCPVzxD/jg7BuumWk1jQh32j1JOJW7WeMH9Zh8POA4mGKUVtv2HlAe0RDcMR1q/cNXzyhDwwGfhzsHoPDE4ZyORbFF4PRb/71gI+F24wKpNMi7nYU9IYW/uxExfoIHAScC4w18HEsdrTdLCMdfBcCWJwZ6oRjwYgCTg5Tk/j4Hd+qYHoW07qogadwqaLse89Iw+ICoxTSiZwNPqzFXHZJb34x8kKd5NRIQR5pUTn1P6KdnhZyG1rVIPQjQ3CRxeZaBH5s/URb0fqLJfFgFCPaBENP+rExKCrorw7hDk6gBIIHwbpzwqeZb3PCGUIPmIoDLOBMp0OWnPwPQeXUq4JOB3kq76B4kC2B6uHc45BXLJzuGDo/FgfZMCiHwlOjsm8CU3QDnzfZ9N+g6r7WM+zhnR/8exkiGxo4F9H/6ZfnNepwm5zWj5zf9qLtO+S80uZLUgx9nfUVd2rEYkeIa2qvJ6wstEdrahM7RG3KidgVLAGXFOfjNq3mvlV99XY5F4yEHBPOoxaPtCfJsXicDqZFsfuaLnBhISF9OBwe+rKvX41W4FJ7tEL0MYzJd8i+ow4mAaOPkW/Hglh/9OUI0aO4Xt1ukqXaw4Z9QRVt8UcicyJBGsci+y4fjzR95dEG0GKhNHUYcYSYuEEAIsKCBnLotByLGJEQT9XPI1u0C2kcKveFqLtlNjH9IGMeIfaoF5wgrwPR/YEArUtaCxKjPtK38IZQdR4yot4FAQ+Cr9UOhqRLelpT5QNM5lvC5Fd0B5SOTpXVNVbnwFYe6Z4/MH2Z0z2vjj64fejp9/mki2bFffk03vQ5OLPyoDGdP50XPTb6Yc+TNHRLHujMojWNU+ezYCvvweBYfOjcJ3WfO1vqmMdzDXxOauD0wnp71Mrpa7JuzNSp+9tnUoGDRyxe8fL26te+xg9BjC49a30D5VnBTXe4uebzVBmDDJTxlhxjNeb8h0Hhm7AeRzxjfDPNmzIPKAwWDHg7FmlEYIhjdPGwL55h8AR/jAros6C5RipECAHj7a1oloIw4vgI2YUcNWCRKnIQoAkNOxVyFBiFsGOB8cU0I/0xB54pUBdkqF2SIcf6itpKF/lGxwJ8zWOXY7Eqx8K7RomeF3dSLzkKR5WPMcYc+62tzbardRss9MQQ8yJ0OSYrK1qUzRoNTWciWBeSwWtFFGOskPbiUun0sKaurGqb2hMnTrVTJx6i3azW5SAtqv6imnpAX+FchI6ga71LLhwQysdWjAcoMKVD0hFCcdXOzoOk+BAqdto57kaZygjQagBrWOfQ4C/pAFkPXdKVXzF5fehhya9zpCVtM0Enlj6qYHSX6Zx2DAcjDMsBf+inoiC9l9jW1wxZk1PQ7uUoRPIwXnVOH6Av3r69pwXb2+09Zzfbp7SdMSMY6xpSYLQCg/829bm76MPCPSVn4gkrR9tjNfJwg6Y3McKwR38QvT3BnhPCX+5dbh/VtKpPyrk4qz6/pLLrNQTxRZpC9cUbK+2RfGhPfSamfoSxi3j61+HG0a9SyqTO6nDWISUc/LgdtFGALj5OyDV8pADhLwKx4dXHfI1CtzuAmm7XnHGnEsmh/hmOiZx0GfyVx3VPg/CxvKFhqIfyHBQPdCXE4FjoumAkxg6E2h14OxLUU0c4FhHfI0cm2j1pBuUkP+aRglfxYDcrO0TSwkXtEuWF3GrrQU4odP3HBKd+hnoA2pf1eFXXvjzTE/jCGaUNgAmayurhi1yfZ10mkmmp780M6IEC5Ez5oMNR7UHc0y4603kFX+UVT8NV/qy450t5j1v0+7xZNCqv4Hs6PX1r5IA2KToVF98Hg2NRdZ7Hcw3MNfC5qYEDHYsXv1yLt3/2Zt2BVXkd4+NDDwZlcXOtN5AYUgR+eY7wYLURi+GPMYsxkQ8RvxHMB4wsq8ARRWDYapYpTTYWBM/6gnBq4k0pMCwUNQ/hIEO8aQ/8kIk0+TqQR39BI0YUvHZBhgNGWMgbsPH9i5gC5RELTRXxInDxxIBnsfe2DLWLijEyGJlATmSCDu99ZZJ45AOnZEVTodb40J2mQy3pYBcnvk3BaAWG4u4FfRNje7OdP39ejsWO32hSFxyRVTkVx7T+gh2foIXMe3JoduXY7GqtxkUWhDJ6IXimXjFqsryy1jY2jsuxOK3pVRserTjCh/MkH8G6CcVZJyjR+lQdGK0JvdKyYwCHUHGVRHboP+iiAQW3/RSNKIlC/VZfqewRGnopJ4Qy8NA1TjRW1KMKM+7lq4d0gRQ+6C5TQiSHMJQ7X5yyzzmfNIYmMnDwp1hRyhoOM/wjG8LA+Nexq0IGJRElTDgVrGnY3LvYPqzF2h/Q8V4t3L5bU6D4LsQpte2GDpyJj+5damfot+J1WtfIl6wutMevLspBkJMqWPozB9OgzunLjJ/QCMgHt3fbxzXycVZG94LKjqsrPEzOxZPWV9pjV5faQ7RlbazrQeCSkVqif/1mnsvS+I6WCVjrnR3IJKO8hWg1wfUBeK5B40UD27EIOrBNnYJU/ArehOClvqp7g50L9fnBsVCa0QHCMKrS1aPoDf1D/En7nOtCsL43MDJBOxOn0+4pTG77eLHh6Ve9rOYaP6GNSLu+1FP0Y9F57ArH6MhljVowclFyGQPYCiU755lPqfsceSqvulQMb8oHKuAVnYxBvSoU3ymYogt88a2YPJcbN/p58erxgNsvWNbiLaCeNjjT5+QV7dJFr++CLxjgp0PBTOffl/PCPYg+dArONLMNJvJUMItGDzN3LKy9+c9cA3MNXMMaONCxeNHL48vbUb80DHh88RBTxE1ywrHIhwU3/yqz4aoHbKyxYDpRGGmDscaL0oRnuhBv62PEInC8a474cfOFFgdGsHnot27UnBPMV3J4Ryk5F+Tz3GTUwwe06g2bCpE/jOsYKfGIBdObfGBwh9PAmgwWXvPFbUYuIAoPHAHLoNizapg6IiOLr2LjWKzLQVjTgYPA1rAs2GZqE9Opzm+eb3efO9M2Nze94wbOC/zZhWpNH77bWFvTWgm+oaE30nIkdvZ22rYWjuNYIAM64cN90FwWfUYrNtaPa0H4cTkVWnCu0QrmetsgsyZQkJVkmb3QW7q0U8FoBe2ZbSgohzqvmExA4nyENxr5BqifePwbNi0B0moBACaC8YQ9QVsQwNJ+7iXEAghY8sYAXdMO4LFAKfQUf5GmsGzfeKgHj4ALGIxNnyuuN9emMdDHkINntH/wDrbAITSgxim9ID/5AWaDlhG0bTmqjFb8+dnz7b06PqA1FkxdYgoUi7NPqY0x1T+o71rcIfhLqutpOZJftrbYnri63D5fDsIhEbXMgsNp3VR/v12OyIc18vFJOSmsv7hOb92Piv+KqD1OOI8R/iM1euEP26EQ4Xl0AjGRUbCjrpWD7OQRU67fqL90gGMhXbiglGsYaLrZrC8TjEZ2KXojlK7hQZjVyl5jgXMhR8Y7VAErfK7fanu+9VLXNzQHeuYZlMEBnnsSnIBzG3NN8RFKxTgV5HHP8OgF9y1aQbCmC6kuhNSR4RolD3TCeqrDNeqo6/iSXij0siGD9QI6MhPxAw3iDIVTOquYfOB7WFBmyZmkIhJ9h8Qn7ZwZfAdaVQbulC5KniAavwNeZt4bzDR80So85APGcJKhhy+Ywqm4h6m8+xMX/n70oVUwA92STbJWmInf6144z3sQ7ApV+pjHcw3MNfC5qYGDHQt9IM9f3lbdecRxU+fmyE2U2M5CPtStHvJSTy7XeTkDtXgb3DDUFHNT9Y1VeMCytaUMYU8xwnlgCggjGgnH6ACjHx4lEDyh5MGYwABAUkYrbCyzMlXn4HtvfhkMGAqEooMDwTcrwpFQWvwpg3eNhCAvH9RjtGJXRhpvHqlp1BEW4oOBKRySGBKMMizrTTLOwcb6uuJj+pr2mvgcleF3pZ3bPNfOnTvbzp69u217tEIGjWiynoO1GWwlu6yF18x1wzFghGJHU6V2djVlihETsUVmYFbkhOBUMEoRi7Y1KoIxw+40ir3GQrJZFxYXXeNQpAOltJ0K8Xco3SYsedS1gqtrg9olY5lAOjAVxkN1Ejf0RptMB2QgoNshKNmDdiVu16Jd8YDXJaKPqX8or/i6v6QILqePZXnAM/MnnGAv3q101cn0qUsYtZYZ4UTEZKFnmoqVP9RpyA9HAMfinPrUrZoC9a4z59ufaRrURzQqxQjVhvTxGC22vlF9AsW+Xw7Hp+Tg7onNcTkWT9Y6CdZLMPLAeqJBUUpe0PVwVh+C/JQc4TPqL2xRS99nbRF9mI/rPVKjHZ+vgzUU1+FsCyarMNSBapIXFcs6Ca4CercO1Ee5BtwBKC8YaAJjBP1GwnB9m1W7AFYgCDGkyaefMh1KsUcWzEP0ybMccqpyNLGnZ5pFCD1xQM843B8QV/rBKZHew7HIKVLS2bDOQgIVXeJKQ2fQiPKrXo5VFqMWXMeHde/Qui69oAiaMOY/sEtECxcCDslKwPMquCzsZSj4fePUQZUPdakMxeZD3ahr5lcep8ZJPZQurQvlVXnRdYZ+Sjd1XnD9+X7pwrU8yaOHrfI+j3TB9+Xk1TnS7qfTotXDO68UAv3UzjQM5xWK1ySuyqfaYe5YlMbm8VwDcw1cqxo40LHgA3mvfM1N3JmzfmUgxAOCm6XfhpdxlXdnoClzuYz0eqMIEW625VD4bWDelgsWh4DFzh6VEG4Yd3HzrnUUHilQWQXT9BtGGQXKxECtt/DQxZW4pAc6b/m9M4tkwHlgdABDfpF1FOaZRrjg/VBQfagxab4nseNvUMixkLFhTVBf0Q+e1FO5MrA8YiGHgO1mj29stOPHjys+rhEETWnSVIudXX0LQ6MVm1pbsbW15W9boBP0VNOmmA5lmeQcQA9WNgzl4FAHAqMRjGisaArUCjtL6WvcODW0CQvmWRsS2+9i1MQH/BihwTHzdrQ4FkoPTkXW18T7tDKqPSsGRlXnlx+Xk6S8gjUzAWPIxCuoiE3XHQeKI41JqPGMNul5UTJ9Tp7bkdhHGoM6IV//ke90nvvNdJWTN46wuS8Kw7JCXHLWiFXPG7pQNg+SCs5yQvkui0wWZ2/KafyUdoDCqWAa1C2avnRB7byu/vQ4jSgw1YkvZn9Mju2n+V6FnAC2m/0iORVfoBGLz0vHAtVHb4wpUXu6JjblXGzLGWEq1UWd76rvbIvfMTnTpzQacr0cjEOqo6c8SfDSfOgtFYTcBPSkq6nqQjtVvelXdq4tRFwzVrDQoBlwFhBKoZBiFjnxC34o0DDGzXI7EOrbOBcURhtKdvEOB0/OmupHO+km01OdTJto9DLwCNSM6wrDn+21azpUfBxP5647+gi6fdtWNVxSsotm1dnySebDcgbRBk4RI5DhHAGHAEjQhaRT+h1Kev0MmZEofLed8Cfk6mCvoqmywgWs8DqUmcnQf2DOpJl1AG5WuXmpPpZ3JofZmfvBT/PgvJdxNrWu7ilvwfX0ruJZCktl+bSrZ/GGVo9rmoVLofBL32TPHQuUMg9zDcw1cC1r4GDHQlOhXn3zTcONkRtfb/TZIPWDvh7s3V0SWN3YhxELHogK3GTLsXDcPdKAxdBlm1cbvDr3DjzGxJDmLXtOhSp6pslbRdHFoBA9SWNYjzwIzo4FxpWMBi/YFhy84LOixdE4Fws6eINfDwSmJfGop77EKGpXjoW/mo0hRr6qBAQx6xPK4WG9A2sq1lZX9OG74+2YvqrNNCiMlF1NZzovh2JHoxSsl9hjzYZoY1247nIo+FJ37QiF88N0p5JNDC0PBhFvQJc0YsHBdChoICvtRN0XNP2CxduLOBxK43R4lEL1PnpE58L3zlmpS7duNFNonLQq53YUTI3iRGH0BtKUc1RaKuHfP5mbGQVr0OEnYEUDkgpFK86u/qUPBbugXhDTeNaE9RU6qwc8sZrQenIe54KjPxqSc6XsSNixiGkx9DHKq76O6W1d/UuWCV7G4gd8c+DHp4xesUj7jPrBR/Sl7Y/qOyl/qWlRfExxSXQfqRGLh6mP8nE8tpTl4EN47LxEPtvG3qDjCE5t1MgcqAvy40ywsBvRLyovnItL2q5WH9zTAvA1rl8rIwWyhJBIBdW5BUZsrqYK1e7Eajf1OZwLn6QOqWToSTjRaIGMKiabbywXjoMiQAps0rEIWTxKKZ7w4H5UIwHOTzJBbOoXecmyTMHhsqZBeR2ErscatcCZZwcq6mFyxClfnU9RHsqh7T9iHYd0XXPNksax8AiJ71lQ6ISt+idh4IdAeqp8KFMCKtWnyQezo0yWwwTNzCu4jluWzI7gY5yUp6dZOpqNOeYWzjT8LFnMbwavkRrNebX007R7eNLFa1qvPa0JGoP+xUv/vQ6gBx5H4VRcZSNDgHUMAsixePbn/gfy0MM8zDUw18DnrgYOdCxYvP3q19wsGyPe2qKGugdyP/QNVMYEb8gx1JXhvFIX5eRjwPpGC74f1PFwzsdSgMuA5Y0/sP4ehd7SgxtOiJl5Wg9rMPjKNjdkm/71cEvayIWB4WlMipEJmXEoqCxGPMYCMLzNZ9oRX8dmFyemDYGCAWkHRLShR2CNBYuncSy8TkSAV1RofSiNTDbU9WaSD+vhVDAF6pR2aFrVVCVGCDa3znux9qYWbDOd6RKH5IEfRjtGPzRwMBxLHuTCSbGzpXPrUzxiSgjb4sqoVP5h4TOq4jesktuOhRwKOx1aNM7XgPloF2svcDiQx+0n2fmripIXoVoHXWY7qo7AUud6WA40TAd1Bz5RpU08yZJPOxOgYXylDVudq84ByuAiwTtAuwQugMqDtMBKPsBIgxmOZyCELZ39kPKEKT4FDw7GahmsYNMGlps6p1GLPL1MQS/4Op2yk65Q+WxQwBevcS7u0JqIOzV9iTUWOA8nNLJwWk7vstqY6U187I6vadOH13SNrMuJXVe7MusvPnYX12qNtJif6F/R9WVnQ5qgv9EEHIeRJx0LYJUVwfkhK20WdCgKXQUQ9VdhHtZJ6sbwATToBW0MOjJacDNfaGQYcMXepJ0fvDwdStcHATjaBZrVDlFv6UD6KToDT3D0VyE4Igh8NO1Suh1HLWJkMNqdO43Bop+Ib9GGFmnrIU4ChrSCdZJ1Y/tZdoli9BCngg/ncVwVkv5AswMgb6yBCgQrJsEfPGfpl3xClkcy83RiudRWQyh4ZK20Cques2QZqCV8wQ40u4TlLrjMR//TdIsGfZXy6cD1WDg9zcqbhue8aE6X7YdT8H155U3T62UApnAqLp69rlyv7BOla6tGBfPF26WxeTzXwFwD16oGDnQsXlKOhe563DTjsaUbddY2HgwysmShxHC/cnjI6a+M0XIsQLGTIKMGw9w31Lq5qsxYOscgLsOaNReUQANjlLf37MTElChosP7ikgwLeHEjJxa4adUj03ILzh/+krHGFBM/nETTjoWmK+FYMAUJ2tCxAeY7vcn5/ILeMO7K8Ct86ollFm9pkVEHoyn6vsTaynI7oVGKUydPysFYM8yuFmuf2zqndRrbck72ZPhImzrqBbDrKIOpppZBG+diUU4A06KYVkW61kyUYwEM9fHXffkIl/SBbllfwXcylrRWYyFHNBi5WBIdRiuGEZDUV7VqtTHt5baU4UybWD7qPBFC7+iMoijNdMLGAzby3DbKL8eiSBVeyUC+MIqgzwo2+k2cGSYL4DP0S/fVKHBenrsHZ7sSVRvYURQ45cjiMsW9U0G6jAt0MdaLukk/4j/IIyJBQxSVqCMkoqwkjTQ7OTElakfTlnbUz7bkcF6QkSsBPbKwpDY+Kp70SybBeTczpY+IJ/lH0amVqGsLd1tlyAt+ORiMPFE/guuppDUmWAFNyMS59WDgNPCMyk/VR0nqnAfpiXNwhzBQG3OA3y8gkwP0OyDz4hrhOrXYEhWNhO65D1k+1Z2Rh17PA5UiPWQYXfQY7Qg8RhPYFrafBoVCTD111dMO1aSeoKsM54EwBPq9+omuS5x6iF2+zEJufdtiqG8CJ48BNRPR56ZzdY5eMhu+ptfRBK94VAyO75fAgyv4afrAGj7pD+UdPVAdOn7QIsSvk6ZdvCuOEsQv6SOn6jCZO8oJ1DROT2ua/vS5kAd9FV4fXwXfFc4qKzmRe1qugq98j6RBr68z+srzuWPRKXuenGtgroFrUgMHOxavqBEL3+r9oCBFAuOE4Fs0N2qMLT3YfeicEQMMUgxfHqgE7/Ckhzc3Ww7upYFPzL01Hr6sDwBnmFokGhj9CzgVOqDtj8V5mkc4FsDikAw3eRntdhAEw9tdFsnihAwjEaLPdKNVGd04Fct6K8xahpLVAucPeHsaqdjR1CVGGjBAQj7xE187F6oiBg/OCk7F6eMn2sljJ1z/PW0vySjF9s6WtqzV3GoZfNcxxUh/wrbOoGfe0mGZKNBl9IY1JzgWS3IMGG2IdRSsmdA2pJq+cUFOy64XdSdt6WeBUQ6+vK01GAtyMNjqdkVOBs5FOCdhHA/6sjFAq6pdaU+M5ZTJoz9KI2+EaDsJEG0GPH9Z7HjIi8zoF2Mb9/olXZQrf+A1FJBwxwsQnQ4w4IufeyT9KiDLvok6OS/6XZHBseDtvY3vwDZtVEH/dP9R2/PWuta1wKcOS+Dz1AOcYa5QshBXfyeuUOkqt3PhPsr0JabthVPAnmQYgGkzD1LSTtQfduVEw9VOkviMb+7TwC7WNd8Mp3ZKnpIpqxCiAtPBkXZ9LAm6iLpbEMlJJygHf1DGWOlIoTNSxQjZQIVVQAy/6NoBGcDTjlA1nQgdlKEGmGURMHV3eyGrz4OqKU0xCAhww/j2qIWuUY8m4Jxwv0rvv2Bc/04nA0nlpbQhsn8nfxitOCLH/pBGoqB9GV4avSjdB6Kk6uj3FAZ99Jmk0Y2iaJss7GgMslOU+aaF4qbzDBK1GuSAfvIwuJG6n44X9AedFAh8EmagWWUZW0bSid/r0tlZZrlJ348wwbOrxywSE7AdQJ9f6ZLFslLHLhRMlzXUjTzXV3XtaTzv2c+bAH8gT+jrA+8p2Xv57HCrr+7pOcNx6tSpB1KsOe25BuYauMY1cC+OhT6Qp8XbvkHWQ0E3IM45rRtnOBQyb7hh6/DbexmiOAg29vOmhWPhOcu6SRG4DfM4lG3i2G99ZUGVIYuzgPHPSAU7RR2V4e+veIsexlcYYJeVx25SsTajbtaskWDak2H0YTFujvCWqWDeODysXVjBqdCIAFOO7ASJp+XIugC8p+kKjDjsMGIhB4PKe6oRRj7OjEZWLssq8lQjOSrXnzotp0LbvmoK1K52ctra3rJjwVvUwnUdO+O9HDL4ISFGLfpk7QZ1Y6SBaUzjNrJH9RJaX1je0QJwnBat22DnKIwrFm0vaYvbBUYnmPokx2JVC7w57JiobaBtA9AMw4i2OaBGwXBjYTijL5aL1rE+AI7gdiYPh4N6GEZlNlwDnjo6QFNtBA4Bh8X0hDPDBLECBtmEEi1mVOuP3mI6SS9LJiLTTUT5cA7Vb2mD6L+Qi74xyCEcwwmjHqiDYyE6rjf6swyll9Bl1Qlm0Cs6wXeiFi4b4JSI/hntXnzBE2BRM05WBdRBL/WtiYGn6yeDWA6RDWPVMegYzbL5XKShV5KZX4AkYJZYhiww7ZAJ7GjTiK1n9xMUVfBjEtji5eKCATQL+ixTmGjj4BOORWwDTftRF8AGJycdi9J7X6/wq0qKvM5glHy8xkLX0SVd817I3esO+dHFoANLOPFDedWB63PgDZ6CnXVNiTqiFxnWB/cxRi1oq4QBrk9zXiH0XWcdnHgVX0rNraNHHri+rxhgNGg53S/0cph38iF/1KKwp3iZnmAJwBo3YXqaBtjnp+o6Ez5pD3yLf+YXbvHvY9J+GUSCULhOTtTKxfXTy1Fp13AGT3AKpvBn5ZWc0IHzfMSi19Y8PdfAXAPXogbuk2PBW11C3XLjbSjGaFVZDzUMRxlc5RzUaAVx3TyhY+Mex0K43I8p4xB6xIKvUQ4b/3IoeGs/bkErg0K4dhj8NlkfqhNOOR/coHl4sn7hguatM3WJSpLHjR6RebizwJpvTZRTgfFOHShDZmICOHsyNPiGBVOhMN7Jw2FikbUdJ6ZsySFZ1HqNdX3U7tSJk1oUvuwHPU7FnnaBwnhANwN9jHHz4g2s0qJXxjSyxjSMUDD5OD7eAWp5rS3KYRCydcC6DRwLPrCHEclohB0QfdeCD/Fdh/4WcucojVjUYm3LkrpHZxhBdhIkC29Ua72HckMPaM7ijDIxHYs6hHMBZJWl46A6iqwbOhyLsY2TKJxd7vPuJ7gmS+VXH0L3JVPpq0NLCUKOyEeqOBeq245TG1h5jiEPDLQTlAlF1mdNh7FTmPpynV03V86yuQ+nruBb9ETRwbQzTVTnxPxxLdmh0Ln7KjK5DNhMI5/CoJs8t9AuglLAh3MiR9bXmq43MQhsQxgFWiWH0/woUBfnJ/0grV/9Rxs7RzCSxDoxUsiVOipnrmStNjODWT+Q8l8nk2hNBzux6nf0OWvON6GQJ+4jwpHc48iNoKiHQSqGauqq0zN1sd6YBsX6B3aI0j1mOrhdUjfTEvb6hN5wLnjSIb/uV7r3MPJIb76o+0N8f0PtlHUe8MS8+v60HJz3cLPK72teT6f4kTedP5QV4dRDnU7Efft1cD3NgievaFfcl12VnqYN/cybxi9cx4Mc7nDuFn0blmwVT+DOOCm44llx5fco5PX5Bdvz/+Ef+uEe5QFNz0csHlD1zonPNfCg1cC9Oxb68nZMF+HJ7KdzPHx1k7RxZnMibtI87DFcbTDLQGV0gVEHbvjcUMuxiBtsPAcwDrnB6l94rLGohcsVx0gFi5cx+GGnwQhPawpDDMeCrVWZMhTbOZLPaAVOBfElzV0nz9IL304IhrrWVrAwmvOgHQ+3cCz44FcYe0yDYuH2HrtCyWFRZVxHb4vL9Ck5BR790Hcqjh3b0ILwFdfpgnB2dvjuRIxyeLtcG+JRT5wSDHg+mFfOjFja+WJrXBZjoyve/OMEsQicrWUZGfEXli/uaXrVtuTS9qTige5Zh+HvX2ik4pD0cZ3qdkTTn/jIHrzukfJY5AtNRoDKgbBjo3J/+8IjMbH17nhl1EMRU4i2C8cIOuFYWF2ut/Pcpulg4LBxrjjaWnES9oPWJ9EHih9Z0dsqZzKGAn0GrgMtg0xi1ZnUGKVKxEvoqA/504Y7oO6vmr/vN9j5Ntk80/mk/lFjdIEsquMgCbLzF3Rg7HqSyFDnxD6UT8w1ZceinAny1Xcjz8IWieBABQiOI811ZsO63oIL3+WCSal0TlbiBgXLSz2CnAEi7Qz9CH5wLCimvq67ylIHjoGvRnGyO6FsRii+QA5yFc0ub3QsgiZ1dYMKptrB5CUrO26ZVtWT6lYaTSg9oVvxI497BV/HvqI1EKSLHnGQ4HcMVbtB7rHI8CPPYI+cbD2NY+HpjFpzdVn3KUYtCJPUo14dyYnkNM/p89JrIfW0S27KpvEG+EFfnRySv69Twd7XeJpXnZesFVd+0Z0+p68NciAn5/uEojnAuyf32hgRr+IzFk2kjJ36GehPQIwnRbNiSoyTIiA51+bzn/P8EekBTs0diwdYwXPycw08SDVwoGPx0h9nKtRrwijXDZSbIn88bAcDKC0I7o84FBzeIUlxTC3qd3DRVKSkIzJ+DnBDrWkyTGnCsaitVnnwQsNbzCoNHAFbokY+MAox0Jn+g6GP8UcZRnkY55ofL2eAvLqpe6clORWrWmTNVrO85SdQt5CHEYSYasGUKhyEvb2Lji9qFAQ6PBQW2AZWuDgmq6urGq1Yb2s6MHQuyDBhS1li4O0wqW5+yy4mdm4kL3JTRxwwnAvoIjfODF/YxkCMBeH6CJ4Wgi/LuUDXF7VQG4eCBuStOusFWN+xpF2iWI/hdRiifQi9aQTCOpEsLBzHOORDgP6Gh+AXvGMU382QLthlSjEGj1rI8qATntnIQohzyqK9nScAYHAq3A9wOJyX51k37HEb5dIJodoE5Gxe04lSg+z7Q1sT/KufwolcCuINtIH40Wt0+MHajmbKUGna30FR9HP1Hem2yqu+1M9GQfIHB1n6ERQoBTX4xQHcUN9M17nhgVP+xNt24LLvIoclC8KW0Vw6+lQOmiHz6FBUPtdLpUXMIepiSpHR0yNHerSc8FfZoF9SKnOGYtOpOCj5d4QHvD/rgPqkeARi0uzKwPfGBYoBc580PDgBD4x1oLpabsucfJV2HtojrTpV+3rqkqigIxZVD46F6ZMv+E4WJ7Oszx5gOvi+1tBhqiHrLbj24efREZyLJARM6arqQ1HlFT/gKlS6YvKvgi/grgzZepwC6fMsf9d2fVnBE1/FL+Xr83vcPt3DQKsv4/zeQsFDp9KFM02b/GmYgq2Y8h5vJrx4Tbdt4c+Kexqm3befbpH9wwAAQABJREFUWv8Fz3nBLLQHJG/uWDwgap0TnWvgQa+BAx2Ll/3EK9qrfzYcCz98hzehvBGNB1qYOtKjToepRDlaUY6GSnwT5wHqN6+Ac9NWzG0Zgwynob68jdGNwW7jmHycFIw5LM98uJm/aPDHSADwjGpAD9p8swKngJsnxyXWWaShwVexV2R8r+njYnzZ2o4FDwjIi08ZiBjjHvWQkc+IAA4GU6x4S4qReUSOBd+/WJKTsr627qlKGPcXZbyzoBp4ePIAwYAg9iEe7PSEI1Q7PeE8UU9GWLaFu6XpETuKCThCJ7QQfENf117SYmx/7E9OQrxVVQtAXzpgitQCayoE72lK8FQ+9dgVPaZMsfhObERT39mQk8LoBgu6mZoRW9LqK88CKCPaBpD0Xnp1Q1sqfrI+rtfoWNq5KAdDtMoQHA1ycOk/aj13I+ige3pEhehfdQYvgn97sMrL8hHIKfcFWAWfiOkfVx2Wh3KAI+b7BfQdG6/w6ZylasvgQlnoKOQLAc3DdQx+wJp+xpWeyM+yuE5CcKfV53TVpdzGSFrKV/ualyvJOfzSgQDUFUrewKZMArLcBskfcQTQ9Pp8SJDvwyc6d6NFH6g0OOiiD3U2nd/DzEr38JUmph3gV3UOPVpyXbtRBs/YKjbyB127birMuvieJJ24jS1orEO4rOuL6VCMWJnCDJ1U/izZyRt4IjPnBShavr78QiFemHB/ZV1HtB2QY9tQ56JVeoAUeZVf58SEHi5y9vkVbQf0MRV62lEDASR4DzrAuV4CSJpDPmjFR+k+v0/3NCtNeY9b+X1MOXDTtGbhFq0etk9Dt4epNPn7wkUhvw4F1+NW2ex4lH3uWMzW0Dx3roG5Bq4dDdyrY3HTz/6sDVMe0sObPRsuuhmqnnYs6plkJwAjV8akHpr19hpAQLjhGsvnemAo5uaLIc9zp3Z2qm824FjwIAPmqgMdU6a/GtHAQYAnfDAUkNfHZTkaGmlgdyeMtEUWbcsZYFtYfxgPvJS5piQxaoAzwRQopjJd0BePOfeuUKKBPEflVLDr0oroYKBjhKMRG/xyQPyRLcSUnNBHB96SVnBeNC5jHgOfKUnogHKciU05AZtMcRINdsFa1dqI0/oexppGRVjXwWiFP7BVU6VE76joLAvOUyzsYGFgYVzdE06F1mKwHoNREBaE4wzhqLCgm1EQvnfBSAVvUS2n5MUZYkQlHINsP9pQBwEdmIl+7RgmDvDwsE6VjjjOjQIyPSHI+Aw62IRBTwVV5jbObLd2pYEdA/1gCJksOeGDnU1wH+Q8M0aY6JtRrrT0Rh/yaJBhow8OhqsqEvUvPbgGIUVWcj/aoxxVyZDLAqaMNv9TQbSh/yQHco91iIr5HIdB8P5TWic+Rg4mnLhZXgy7uJe5y441+ZCANiF1XEa+z9EJia4pOtBBX8a/Dz+lX9NInZJXPKlbOVVKmSJlhef1EanD0Flqw7oBWdqSPod7WzAyHa5hpid57YPhAjcpuIqlKyPM+BnKrZeUMOUBvJx/7nPAetpd7RCl86oHcdHq05YfQiWUW5+MsU/G2f3/LX495tCs2RaU7QdXIlU5ck/Ivg9uz89pCA2MryqdyCheE5k6qXz4zwpVXmUFR37fn3u4ggGnz58+7+GK/mRMHzSWs+eOxaR25mdzDcw1cO1p4EDH4uU/8ePtpte+Rm/Iw0Dn7S030YlDdea+6FswDw87FzkNSobmcGMVEEaPgYHPBye2ZDkWGKCe1pRTjHBOHPQ8gAePBWBs/ItXOC7xxWqMYDsW+dbbIxqC9xtAGWa1gxQOw6I+YIdhv7oSb/dZswBujJTEyAIOxS4fKtN0I6YlxYhF7DIFbRwEFmsva/clDkljA+WSjH5ciMMSWJRCV8ihP4xF6sd6iTWtw1hhlyeNLng6hMpoDJyKLW1ruy2+1JmF2MdYEH78pGHZAegiRo9HZOJtKk7FoujhWLBGgo+IMe2H9tq7sNvOnbu7bW2eb9taSM7ULPiZrrbFXV/XV8E1fWtlZd06iOkgMXWL712EY2GLX3VRP8j2l2hj26phGDFx28gxqWlfOIo2ntRmlNEXoj9QswjjQzn6jitdhcSiTbvbJjB+X9ilq4MYFr1TFg9tpy23cpJ1ORaGclkUlPM84Ziq3gTqEE4WEkUY65RyWlqVSdaqm6+XrJjTya/Ki1bF1jGy84dTAbz+kKkM6aLjCqVT4XJg89xlVlxSVpl1UuU6c3s4nxL0E3pwnOmobdY5aZii20P58DCfhMkImC6pNHDGvOon+kVklwx93pA2T5zmIBR6GB2pcjqAH0YsJHPRJLYcsCJfLxs8KsW9TVngQdojnkynZHoSdeYAxT+Rdkb+lHzFpy8zin8m8ewEqU9x/zFTtW+MWlCfaQrj+cDDMqVCSZduUzdgIFfBQ3IEyXtT1mukPpkqXHKN29GehJxxJob0W0Lph3RPk/N7C7GTV0BBLyjed5rFr5fh3ni6vNevKj+LDnkc0J7Q9ZReZ/MOXHgBjmrnjsV9apk50FwDcw38DdbAfXAsGLHg4RvTivz2lJspf45VOx4g3BW5ucqwx/DHePU3HniDiALyRls3Z3DsUKgobrosKA5cHrQxDSoN2nyyRDlrLmKxc4yMhDOCY+AP3Ik/f8FUbPWMxlhk0XVNj1pa5LsQcizkGLDomrUFrJeoBeLUka1lt2Xgs07Ci7cZtdCBkyXrUl/Q1YfwtFibbWDZYYopRkydwPD3TlM4K4LDaGFdxkV2mFEZu1utyhHZ0OiDP8onQ5x86DINim9eEDN9yd/Z0GjChox/RkRUEU2x2jVN6ofjxSgDX99e0HQmztE/phNrWbY06nH23Nl27u4z2jVq26MclDNVylPB5Fgc2zjeNrQ17pK2pMVo5qENDOs6+JCXF5frHIUOhitQtLdz44GKo0WbQwOcI3xlmLx0OPzgRehAyzanW/BQTkpJc+gj0AdFPy7KNNAHh2RCvxR96sSP03lSPPoYyKF/q+PQb2rEAhkO8Q0FtSnT5QjmAr2Uz/3OJWYXKfEnxPWiJqRDco5cWeaM7sdlyAJWwjmWPINzRxocYudzBjy0yctz0019gGF6GK6UTwXKlDXIZWLOiDr24EWShqmjL890gXFa+gkuARDXPiRGyIE/euXPLOjXSSVHJbzzlOsaOqBOg2MhPKaylf6CW/JUZB7SEc4H16gXTxtf/NRv0SlGvg199JKHKShdoeSuuOCG8wIsnIoln/9VF+4lXDfqYP6uhadfiX/RKBIVW3adEI9aq1LFnS6hMcAnSOFUfofpZJ9vHklvljx9eU+np9Hn75cu+GkeyMrtvzRO36k0tHr4orGfTD3ve4Uxk+BU/IpX8enpVRqYKq+YssItuKgElOlbmavKzh2LQUPzxFwDcw1coxo40LF4mUYsbtYaC3ZX8cMX5yLfoHI39J9uiuFUxM3TO0NheGNkpqHrB5nvoXETBY9/PzSkuLjpYhTEtCg7FTZSMeBAVBAwjgSLjpnK5B2WBHP4iBwLOxW8KedNZoGHIYLRflmGPVvG1ttHFi0zFYrF20yFYg0DMdOMIADc1o5GDeRU4Fjsajco8DkwC1kQzXoGFmzDG+OELWUx8hYk37och9ht6rCdBEYJcE6oN7woX5dTwhoLVdqOByMpTNcqwxMDlulJK4JluhK6YfoTayV4YKFfFlp7jYh4Am9NCQ4ZdwV7ViMVd911Z9vc3AynQvk4UMtyqNhhakNOxbqcC+jjDJTu/A0Lnds58DSt0Gu8SUdkKdl6TsNPvMuxwJEALxalj46FWAceiQzQQSdhPGbDkUP34EdZMZolCBldZsvPfQleqA1g0HNKNOthPyvGVK8Ri5gKxfS/ME4xWGvEIhzikKPkR1bqERwdUYlMRC+Gpw/LNMoyACnRywVt/4GnvmX961ocZIR+lgWe4Ep35AsWqSakqHwA+5DnlWt6nJCfZW53cKhmtUM0CrkzQ2gkitAP9ZkORbfiqIsV6j5DPnTCjKYf6OBah1o6Flx74AUNRsfCiSMv6IXY5u0qKR+d6rpjulMs0g98OxbCY1SQKVHluFny1EVfh5KbvOJFmvwJnKtwo45exK37CPfLWsSNXD1d6FUoHhWjm4lA5TNYhuTbax6Iwi9Y4gl5swC4kqXiwgF+5Ba5s+hSUvlX0ej00pcV3YMcC+gWTtHv5aV8OhQc+YU7ATMoalZvnYC86qToFY+KK98IoeTAzbQjVXjuWFyl0nnGXANzDVxjGjjYsdCuUDexeFs3fkYs+No1MU8fboTcNP3WkCeAHmbxNjcWEmP4xsgFBkCEmNaReDxM8oHiB4jwk4yNBmhBoyZ3c2PmWxMY47zprwXeR/QFW4x7YIFBVtNFQAVkZrSCqU04RwTwmQa1tqovUyvt3ZHkWISToO9WyAnY2taUJDkV24pZl+DpUHI4VEkvdMapYAoShs0FGfsYIEdVtuqpS1rILccFB4n1GYxCwJ/Ark2MVqyKNyMvjEywlgLZEN3Ok/AYrVjS9KZFjSQwIsEuVywGx0lhChijOvDHIcDwuaytMR0kg0ySdk5Tn86cuUuOxR2WAd1QPzsV4r+h6VXHNk7oa9wr0uWCDWgLIAO+tqaNkQe2xBwdC+rvdhY9FUjnOlea6Vd2djTSQduwiN1tmCMWIdyMX8jwBz0F9yn1LmLyoF38APGIjNs2GzhJFn4Qqa4lwy772EA7z2fl946FRyvcJhjnNHs6Fq4v9R7ldccVDPUouiUdUMCST54dxyFNjg7+95NTVpVxy6HgLTv4anNTnEhDKuAFNEFTjPI8DHCdkBUh00SmqdySGTo9rOvthjAQgElk/2hfiOQ70BxIJIZoU+byLIvzTv92/FQn9CF6ASt84VmnOB5RsaCgIp+jH+HYqVA7095uG/jR3xWucixS3iA0+9e0s6iX21mJ32nefQZ+vKxg5BN8r7XQfYPQ03NG99OXwas/L7BpGQYYwQuhwIZ4KB9yIkF+0ap4CsSnA35H31w6XtP4fTllHOZn7UzqgP45kMpuAuPCmZap5wXNQb4OsIdx9qAWuN3/sJ8sA6WsQA7EJ0vxUr1fMN9udlDT34TEnecutQ9+cq895TH64OzRrsP9TRBuLsNcA39DNXCgY/GSV7xM283e7JsxBjDTRBix4G7LLdc3Xb9J5mGgh73SfnMtY9JOBed6aNblyP0Uo2icbgKhPHgQKUDTxiTz8zl40MBD+RisrElgtIFRC4xXG7HKBwbRarqT3zgrgwXbfCjPuznJyIAejoVHLLQrFIY2zoV3lFIZsjF1qZwKpkPtyphnd6gLooXRvSjngalJscA7doBiTcWyZNqQob6+rilSonlYb/tZ2+H1GekQkM+IBaMlGOLmJxj0gGwY/3Gw49SyF2PzJnOPnaZEA55ew4AhLz3jkODUsKDbC6/VDhfk/N2hkYpzZ+9uO1pXQVtQvyVGKuRUrGvRNh/yYzcoeEoI/UsGJXGGFrX9LPVkdOiQRh/QrRaOKNAWGNgxzS2mBuH80e4hjx0+4THq4TUWGOLC2TeIdDzYlZAO/AY6epb7FLKXY4F8vREQxkdSph4E+pMA+U0ypBymDYsyMiqfGOPSjrT04SkppleORTivOHbIAUfz0e/IjzqoAFkdCU6wZBnKPEglpvpb8RfIZDphKl+SqZ1GIzocdRz9oOf6iGzQw6A2ZsgJjDMMMFImj7LkDQbBMleZ4jKCrH90TZ8onQfKvr/ooYI5iZnpKHMoK1oVqyRgiMEOSPLqsNx2HMp5oBYEwQJOvagDNwZO89fVSj3S723Ic66pUzCjjwNL/kEjFqFPE01ZS/ezjfBZ8IEtdXL9cB9T3/KLAr1IAL6OQV+Sz/WgEgo9zaJVeQMOBaHECXjgCqZwBQDwcBqJ6B+V2eOULJVXvM0vZSy8oUwZBU9Z1GTkOpRVHYf2C8gawfAZTZ19YpoP+T3PShMXj6GmM/QzlEkfkxoITtP0p/nXefGq89Cx5O71g6wCmH/HYtCSE+/52G77+J3hZNMey4uH2mNvWGg3nmR2wQMf3vrerfbL7zjb/sl3PqTdcOK/Ds8HvlZzDteaBq5clB2qDYRsf08Jf0j2GjNhNHlHzyxtMKQZNhdlA96jqdtMlV9clB0Hju43Vy7tad0wM2/0clr25fLRA2yzKT735/RAx+LFL3tpe9XNN0mIcAbKKTADrnLdDMPoS+NPxoadAT2Y/aE8lXNTrRs0N86igUFNRbnJ+/CNNdJgQPeQNGXDMunwBW7WR6yyrSrTl3AuMMT1UBaI1xWwa5OnFGEUiAeGN0rGICNgSC8xHUjOybJGLVhrwPazGPM82rjX40TtsICa6VAajfDXuzE4hcsuTnyZGuOaaUn+loTeLjIFalU0j2mK0/raqp0VjAXWS1yQkcDIBYb2spwSRizY6tVvKMWVRiCUY+GP1GlaVq2bQIF8f4J1GshmZwD1qX4XtDib0QzyZZm0PZyi7e12l0YrkO86yb2szuUpVTgVTH1aXfc0KxauXhFv1oZ42ozqh7PDCAlTmWKdRExpc7umw0DdqZuncNjJUDodANZm4FDZsSAvjXBXsP/JThH9I04w6sOxCGOh31nKcJJP/zRT9J2kl6TSQMkzOptCGRN92kZClld+9UNkQJcxNSanQbk/Rh2Ro+pkMeCRnM2r6KYYRL4CLDjyqB/R5zsc+lzgKjevh5EqEnIWMMaXfG4vYNUHQmcjLnnKNK1AVhoK5GVcNMlzuoOnjpXnWPycI1CXlVORcAbWzyRethX5CeA6FnDGVRYNCzDaUq5pR2nRBYU0h68B1cWOlnSK8xB1AwjICOaJPhRcTgys6uRRCq6BbG/rVLj0bzAYzWD6YbWJaZW+TDDokuwDcCVz1GUsNQ2dVvl4HiO0jFwQ7PDoWnbbwtN6CTxzLTkyNlL3MylDFohG8SvQHs55pjcqsFKpXYOU7JyUBgrOefvINF1WdKDR41c+fdPyCmBabmi5RbM/xPn4W/RKvln4BTNijfUhL8pnOxU9Tp8e+Yy1oj6V77qhn2kdpTDPn3/Holdn+zevv63dce5yO76il2jS2fkd7s33tCc/Zrn9va87LvtgVitOkPisTj5Tx2L3wpX2m+/ebF/2eUvtkQ/RFu7zMNfAZ6GBO99+U7vp19/dPnRbfILApPQijM8oPPy/+e729O/49vaU6/fabe97e/vtN/xae+st5+WF39ge99Rvbc98+le2hy9pJsvOne3Wt/9C+5nX/0k7d/h0+4Jv/6H2gm95hDYZkp2sWTp6iy27/bMQskM90LF40Utf0l5586t9U+TZXCMNfsjpZskWoxiiw1tlnccbbQxKP1I7VpHk5lB0fLOFLnlUToeNIt3SwbeBaoM1znEsGBXgA3A4AyyExiHwzUXw0MWpKOeC0QqMRDsxYs9NHbp8oXtRDgpbrnpaFd6e6MQDjQdJ82LsXS3IZsSC6VRC9JesF+RU3KP5XxeYLrW1JV76VoXKGRFY1yjEcY1WrOtDdniQ4NixkHGCc4EThCOD87Gh7170MNCQaGGQsxsTIwY60CJl7DbFWpdwzHRzhaanRmnROA8p6emy5Dqr9RR3yKnY1jcr8EUZ4VnXKMoGDg+LtY+dbAuSgboymnFJDssVHAuaR/xpP6+xkKzxpXA+WIiTI2dDIxw4PW5z2l2OxiGVeScq0eMPByNGK1hrgcM3vt2HxUSAX+LRF2zMqycMdHBgoIfzApwPKFja+I1kks2rgijz0V2FerhzThp6lebcoxWCL8cCeYDoZUBJk44FJleEnldmGd86UAbTBuFTBzCkK678yMt8l45y6moxjnl1Mo+4qi80QR94BU5kkh20TdpgWYcZ+YYxPXCiDZiygeqsv9QhcL0+gR3yBrojX8ugU6CCjsE5STrqvTNoA1uH66f2svHtWASTl6doQtJZY76zdB2VY8GLBxvxwqf9YRkfh5SmeTkhxxvYauVIpw6TF/L0OnV9xEiShgAw7erCaQ9POuoUUxy5BgnDdzTgk/imrfMe38DdTy9PyVLwfVmhVBnnRd9lZjsa1jp1AMZ08rx0Pp4WZOXMjq/iOw2W+gXOFBVbvh6u02vRc58CRmWVVxJFmxSBzK1CZQ/9xugd/hTvohv6CnrVR4p6H0/CF1/FCJuB3Bc+94V1+oDH9P3SZ8U9U/KQm3sNBxuUcJw6daoHe0DT//vrbvMz+59810PMB4P9P/7+2fYnt+y0p3/1sfZ1T1x9QPl/po7FHZpC9b/9x9vad3zVRvv6J609oDLOiT84NXDPx3+/ve6NH2mXHv2U9m1Pe0I7/KG3tjf+599rH1z5qvaMZ3xlW7nl7e0Nv/q2tvmEf9he8J2f33Y++b72lte+qR39R89p/92hd7ab/4/3tK96xXPaU667u73lp25u9zzjOe1rbtR3zf4KBjEOdCz+lRyL/+emV/nmovvLEHPDDmNr3OqVt/Hk15Qo39y5MfnJDi6Nj/EYD8W60UauHqKUCajKeQCEkYphySGHQEYtow3+uByjFXYsYioOdHTv81QhnAt2ecFAhA+suUly4Fjw3QimUrG2AueC0Q+M5zKCwWDqlNc1iI6nYvG2XsY2uzexY9Tm1rYNe4wPynF6mF51TGsncCxqahWLqGNE4bLz2GY2RjXWzRdcRjOgi6y8sbRTgREvw9wGkGQI5yuMXkZJ+AAfb1N5MIFzSLrZklx3ageoO+68Xc4C2+ri7Gh0xNOf1rVYO0YrMAwZAdnb2bKDguLQEb9+my450BWOASM0K0tsZau1GBpyi5EK1rVoRAOHTHnhWNACkiUdDsppMxvh+fSMloh+hOBi4TYGj7pfZiqKArwPM/Khdu+NevIDwJ0pO1XmJY8AiF+3vejOCn0ZcnGOA+oRC7VpjVggZMkQznL2ceUXZ+OjO/ft0ZFB3vorYKSZ4J3yVR6xICZE7s9cjqxlHCuNYe2Hv2KdmL4ARj7AUjYjFN/pouBDbtKTEK6v2tQNl3WrelV7DnQsNLCQiBoIxWHg6fzQo9vW5ToHUA6p+SRBk1HaOlU5bUGg3q5bXevJywZips0/ldjrz1Og1NbhWMToAAJj2MMH2nEfoUwEirb49ucWZOKHOpChH3BC1AkI8F1P5ZoWfIXEPcbXmNL+joacWzszRQ8qJQc0kqqrJxzOi24WJf06ixj4wiUnZIiy4dey91BRMk2/4IGE7kxaBTQVHwgr/q5XhzNIEwoedVEw1knooSoYNISZyJQG3yrJ55RoTPMbye5XUkhZ7jYIRoP06DHDmBrEMYGAvW7uWJSiMp52LMje2r3SXvzzn9KUqMX2Q387nJzbz+qbT3rdenztcPvQJ1gTeU/7wkfqZVi+gqUJbrv7Urv97MV2Yu1Iu+FkvpCc4re9d6XdevtFvXy8rj3i9NH2jg9sz5wKFXDakl58TojnIzQqwa2RcG77cvv4HRfba37rLjk+a+0pj9U3oiTH9cdjNBIY+sFd5y+1T991Sc/Y6zzNalVvlftwb3XqYefpB5sGdtoHfuPn2u/eppGHr/229g2P0Vqgt/xae9t7d9oN3/ys9re+QH1t89b23t97Y3vdOxfbf/8/fVd71B1/3H7ppz/Wvuol/7B94dYt7Y0/+XPt0A/9i/Y17/vX7Sdv/7vtf/muL2xri7Kn/wrCgY7Fv3zJi9OxiIeFb4r6wRguY4stXm0AyrGIB45u2yr3NaafwuHC5mbOgfGqhMUHB5MHo7YcCx7oIGI8lHPBR/CYbhRfq9ZXtnEGxBv+PF65MeOU2MDKqU/ml0qyUYihKprlWCwuybFgfYVo2RHAoEAWZExDhXMWSvMWk3fpm5pmdE6jAuc1WoFhQsABQD4cHqZDrWr9RdGLbWbDqPPaDn1QD2N/RTHOjJTmUQ0WxSMvvDDWWZhtWRh1kWNBwNhlBGRnLxaLY3CwToLdoVg8fVYLtu+SY3G3DvTn9Sis58h1FWsaJcFpYb0GIxoXtDid6VAY/zDDEfPuVJoCgp4YxVlRXZblWLAdbTlgsXCcr3WzXa94Szfg05Yx2sGoRty46+1+tb3bX3URqPky6kUIh4Y3xtHmLBz3lCvJ5rZTPmUcrpyxog+ZubXlzOEHXhVmpS0TilKoNrcxWY6FjPGS5zBbzQ4yhEwDbdEo/N6Ar7oHHAqKlPmmbCVX5dV50R7ixEVccQuHM/tpvXEvGirUf1xrpie4Ua5OJxCnLOOBl7MTDn7oQXm+qtG/2yxyqk7gumlIEOrVL2CiEWVBk6oP8ugk6I70cfCNkAQhMQTljf1AdHydhj6iLskDBJIwI5FpaotO4j4RTkWNWoRMSV/3AmhTxnVHukKQgiap/QJ0ADkIhmqWkQsd9X3up7qW6fvcX/yRTfhHcdRD6Zl00Q1UYNyFadheooKchjE6sk/RIn+afscq0Pi9l3oXzky+WThdVrL2MvUwpAMm9QAdZTh3QC7ixKUJCvM6yOKJSGAHlKqw+ArL+ipm9DUoFZ8+1am2oz8fsZjQfJvlWADxo6/9pI3x53/HaSP8X796h56rGl3Xo+idH9YOjQrf/80n2xc9cqmd2bzc/t1bzrSP3X5Bz8RDcgautFPrR9r3fdMJORjjuol3fmin/eLv361nYLTX6Y0j7bE3LrZ3vH9rYo3Fr/yXczLg9PxX4x7RvfCi4G84frQ9W07OmpyDn/+dM+1dKYMF0Q8Ozou/70af4nj8/O/c3T78qT13F/oIL+G++clr7Zu+dD37cGsH1anozuMHqQbu+sP2n/6/P27nHvkV7Zv+1le0Rxz9VHvXr/9q+/33LbUvesb3tKc9Snq550z76Lvf0t7wulvbo3/gue1rFzVi8bO/1Ra/54faN1z3rvaaf/NH7Uv+2Ve0d/7k+9vX/Mj3ti9Z04viunV9lmq9V8fiZ179Kj/ExhtrGH7lWPhbBX4QhnGKPPHgD8m4RJk6xB2WG78NSIqUHh5Quhnz2LZjISPGD1QZRtTRC5X1kIcPb9AZGVhQ7O9PKI1Bz9NDpoWnLHGRQrvohyw4PaOjwkgHU6EW5QQwHWnR045qKlQYmTwL4Ml0K9Y6IM2upj8xInD2/DmvY4ANevB0LMF61yrJhEFPHusSYEzsaVxyYlhjwVQujHSXU+90KpAVp4IFNxjv6AGjEeMGQxEnJbau3VP1ruimdkhrNeQsyLhnFOJubS977vxZOT9bqr/klwyMOCxrBGVVBw4C88a3tjbbpuAY9cCV8jQw0WIXHBaaY0ihd+tGMnsExaNFjPAse7eqJTlHi0ojL21qZ1POgLeZpe7CP5JOATpw2+P4YfSqzvWlbox1zjGWCRjj4ajFrlI2zpUf7Rh9z5VTXuBgRNBTOPYP8K8DqIk0lFRuYxNjEp3rQFbLQ/+TY+H+KnnJKycDWsDZIREOdAklr2P1f+emiAUDXKV7ecivENTiTKwdAjZ5luGLXuFNPXEqMLiJzTnzQ4qASQbFn1PSvRaHMvIRRAKUDkhP6LxHNDF+COBkSkQQUZzMq9Jkmi7czUO9UjqPvMDtf61TwSEf14EdAoj53AxMh3M1zkQ+OL4H0b52HDQamGnOCUFfjgWwKuO6GHSRggTv4DVdNkvu+wJj3vQ17qe6lpDfsullQrQjmpOOqNeskIpG3RzTUD4v3A4WUr18yN+fFx1oTofIo38nVEcXGj2dq3Cn+Azl0JjAHfvQAKNE0a540LtwS1bXBaSUq/ALJ4q6+oaSDBb3lcCIu1ZhT8XJzzw7PsqeCJwOelI65A2gkIcRixdM4DyQJ/+1pkLdcuaetrHU2snlapX7XqtZjsWtchD+rRyJpz5+tX33f3vMxDDCP3Hmohej/v2vP67n0HV2HlaXD7V/+8u3tx2NLPzAt5xsp44daR/8y732+rfd7VGEf/pd17tr3HX+sqYufbo9/NRCA593J7/1J5vtj/5i2/T7xds4Dl/6ecvt8Y/QaL0A/+yW3fba376rff0Xr7Xv+MoNOy6f0ILzn/6NO+UorLWv/gKtp5TDg+NDa/+U5IHf39cakS94+GLb1LoRnJU/++hOe9Y3njBtmO5XpzXVaR4ezBq43G5582vaf75lvX3hU7+lfcMT1V/bXe29v/WG9rb33NMe9bef1b7lcYw83CXH4k3tV37hlnbD9/3z9l2P0hqLP3x9+7nX/Wm74/Dx9shveHr7yo+/rn3wK/9Fe9aX6+UxKH9F4UDH4kdf/GPNjkUx01XBfZOHPgY1xqMdC9IynuOeGrfjmlYEqo023WVtgOmcGy63mH53Dy44w+EiYGinUWTHQrRrWgzGNA4Gow5euK3zenjgmMCdc0/DsYwh53V6Y0AedBhNwLBnnUbsLMWIg7QqPGRgWhLyUcb0JgxrRgoYqThz9pymQW3p2xZa2yAZzUu4jJxw4FzwPQyP5CiNXjDc4bksfssy9BklKcfDsluvGM04TJoCJVh07HUVfmuJESlDiLSmCyEjxi3TjZbkLKBr5Nvc0te1Nb2Jr3azLgTjhO9trOg7FRj83NV2VL4jx4KYKVi0BG1JhePt7z2qQ2wXaxwVwQ9cnKNVjXr4o3r6Eji0D2nKkoosi7e/FR/q4fU3kov60eBuWwxwQdswF0/agsDDlgdr6BJnMQ50B6z7iuhQbnrG0o+tXQhM5Vd5F0O/P5AJuZ2nFPKRLmPTb6mVH7LGqJy4uI8QIwfOBQHD1utf1EYi4aNkDYj4dUWMMf7AUxjGqdzIq7OMqXtACjb1qTa24Wu+MooHIzqvH8GZFrF5IJzKzHOkC3OkcOjKgHO+Ynhb924D+ou14HjEDyoTbZRkRz2IZvIwfaeDvmmqT0iz5sU1QCj44JlyZD4jblFO3TiMMvyYBw6Dyiiqc9q5nAZfWwMdUFU38QY2ytBnOB1F2HQse+RwXmFW/Qt+VtmQJxIxGsx9IxxZy5DXvVn4Z+xPs/haEsENdFOwq2CzT1GMPvpy8sAfajWDHjB9MH5Hk7KiSTwtT4/bpwunz5vGnQVT8KmdOBVf10FyEYpOOZEBRItHiKssT4RbBdKOM8maDnUbShbTxToP6qCmNAPMNM7n4ncsPnC3pgstXdceIufi/gYcC3T+g996yiMDGOy/9kfn9Iy70v7pd17fTqyHNYQRfuudF9rzv/0hnsJUfD7w8b32qt+8s33fN55sT9JC6gpvfOf59qZ3nxf86fao6xfaG/7wXPudP9s0zYeeGKcs/fvfPdMYyegdi6LRx//6F25rJzcOW07y91tj8b5bd9tNb7qr/Z2nHmtf+4RxfQijHi/9959up1WfF/6dWE+yX516vvP0g1AD2+9tv/r/vrnd8dCntK/75q9uj13lOXmlnXnPb7c3vvEd7S8f+g3tu5/+5HbsU+9qb/5Pr29/sPno9rQffGH71ofLTrmwLTtRGwFpXe3OB3+tveq9j2/f+6S/aP/hDX/ezlw+2Z7yzH/cnv4lp9uapud9NuFAx+JHXvyi9jOvepVuhiMTboQYohizxDFiEUYiN20fgvd0qDQKbazp7uAP7ekmwQ3W5qLTGBpxw7Vxx6033+Dz8IC++ZhnGHMYpIxUOMaIRSgd3LRDJox8jNOYVoCRj5EKPIedCTsXsRAZWlGn+GI104EIOANMBcLIYNHa2XPn1SibWtCtOZy82UdOwQVN3tZzhJMCPRvFoo0s9f0NHBUcC2CBQW7U6zf8GPQy1lmzQB7rIJgKgYKuiBdGjhIAi6acopyKRBlf5GYx9o52ieK7GZhBh3FiJD+OEWtGtjWNa1vTpfgK90XRZuqT20v0EAM9YtSzwHxBOMi3p0Xj0KOj4HCwTe2JE6d8LIg/BhAOzvj176jX8EaftpX8GN/WlvjAA4cRGI9g0AFUYd7UeiqVymk7t4lkKFplFCCrFWTt+8T1IHe/EDIgRxzAYSwOfQ4dcyjv8hXeYochObRjOknRv8VdQvgv62Aj1cbppDyDzDQoISPkiBDxcKpM53QZpgE/5VEGbsjKyAr9It6o2/gFJp0H8/B5V+8sN+9QJAR9Gr9mMJwXjUEGai0H3dOVsjLTdak6m0j+pFbyDHlIjnJZFOroI+4vPR3zoExYle96qv7EDhAdKlFZoz4o8wsL+qLaKqZAsRYrRi+irtm24gUp4MrhHuuJykL24KJfzoeT+5eo+hRW9HvuWWE0uZ/auVA9rThpIfsROL1cUk6QMVyvq0npiidx1aViCLh8H1rBYPJ3kCFxSryraE6iDWcD/pAzmSh5yb03WGDg7xpPtUvRmUUjZI4+ZuxU2T2MtJkvlKdCtkepqkqr/nFeDtrsPtLjfi46Fnfp/dWyuvJn8qIdx+I2rZ/owyNPL7S/+9SNid2WMML3Lt7TapF3wf/2n262X5cj8rQv07elFsdWuV3rLf5A6yf+wdefaF+uNRA3/eZdWltxof3IP7qhUB3vt3ibx8MnVLHzmtYE39/443MekaipWfs5FjgzODX//BnXt1OaatWHV7/xrvaRT++1H/vemDK1X516nHn6waeBT7/j37X/8MeXNFrxtPaNX/7wVt36yt6Z9pF3vqW9+c1vbx88v9RO3HBKfWyx7f7lUnvq//g/tK/eSF3dow8tn/mz9ouvfEd72D/4qnbrT/1eO/0Dz2xP3Htb+8U3LbXv+P6ntcee0nfaPgvVHuxY/NiL2k+/6pW6qeYtVxE3Zg4bXH4zLWMd4xnj3GUYBQmTMUYab9A5GFWAHrB55++MOz3IKRdcLITFURhHGpBi4K2TevM9GJ6C9WiGnQamI+WoBsa+DjsWGMIYtjgeyqMe5bzwSBFnrzMQIxvSGNgYbyzYPi+jHAfD37SQoU2MTQN/HAdPm9LUKfjwaKMuYmD6OClsk7uqb0mwvoJzeBOYKYZRjQOwIEeANA8+dpzCsaDeoZMYtbhO8mPULy6vyvg4KqdBi8i0ZoLtZf2tCwnFbk1MVWJtBCMw58+dbXfq2xbbcowuMtqixd02YDSSE22pBejiz1QttsLF8eFhuskICHS1AxVyra8fa6dOntLuUsftTDC6siI5Fhf1ASE5MmP7ULcygP1Y1plaXnWOrWzlgEhvl7TvMi/awWPHLY9WpNNl+ZRf8gETIeLxPPCz8KqojAjiPk2/5Lycm3IsrmhUKIzV6Md2ou1YQDr6Lrz9pxj8ME7BC5hp2Upy+lUvQ1UJvMLp5TQfeAFo+dFqOEDhBOFc0A+jLsQphPkUrYrpsK5bMUbcEJrUEAxfZyGczkofKY/OgYtQcSFFHHJXnuqQiiAK3LFNKIv6xj1kABas6SSZURfogesjeJveIE8wQleuC/mMgvpAZ+FM+EvXpNELuiOgb65NoaCrcC46OQUy0Bz4GTMlybTKerkjt/vNNkiVjAXw18ELCfoedWCdVbWxSq0aYCyHBR3R+1TB9Hmkp+Ua6gO3bMppGK5D4PowfS7CLuZ3pBkYLsnyAY/zpDnkCbyHDeyQuad5X+o2DV90ya/6TfIFgidUyA9vakwb9FUvXMoDglhYCTRZPmT7Xm8ZAO+C1SJGL/gcnAolu9vbWMbTrqv0fUjiWLDm4e993Qk9HzS9SaMCrJOYDvsZ4b/8B2fbW9+z1T7/oTxbJjWPzr/9Kza8zoLpSSzE/mfPvH6C9CzH4k+1I9UvveOcHIorXrjN+olPy/m5Ud+5uDfH4nWagvWO92+3fykHhm9y9IEpVu/W2gzWYiDrfnXqcebpB5kGLn20vfm1v9I+vP7k9jXf+NT2hFO9c3qlXdzdaufPnm1bfI9559Ptw3/+rvb2ux7fnv0DX9dOubvJrtw+0z7w6ze1tx57ZnvWE9/bfvL/3Grf86Pf2T7/8LvaTT/yJ+2J//Mz25c/9Fgbx/fuv44PdCz+13IsuLPqIuRmWUa8jT49aGz02bEIw51yHi6GExI43LhxFC7pIY1zgQFHPpc5B8EwwOlgJACjCZyYUoNxGfDF3/jFx3IELwziYZoTO0fJeWAKTzkVyOu35TKS4w05hn/IHsKERDENB5NF27LykTwZ4/X1bm8hqwf9roxtRguQCcdiCWNeB/WnHhdVBx5I8GRdx1ou3PYXv2XEw1eghq+3/oclM8G70diYCMeijGDKcCqWZMwviB5b6+JQ7MoBuCR5zE90WQOxoJERRkLOnz9vp+LOO+/QFCgt/BZdAYazhcNlXWjxueC9A5ToIxhw/lYHIxZycHiDylqN9Y0N7TSlna8Ev6A1Fqv64B4ODI4R7YKugQ2jLNoShjbQ///2zjXWtiyr67vqvs85911F29BEiNghDdqgpkmIIiIhhhhiggngR/3Co2mQLyY8pPlkNJEYiZG2KRITAh8MBjERUBDRkIgYoiISg0EF6eruqrrv96Ou/9//P8Zcc+977q2iOl3dVbXmOXuvueYcrznm3HONseZLThEnfcf5YtqZZDEO08XiWESeGrFQntuarvpXSD07lgSi5uvIFKdNzUYDOiR0el+pK8uqK0Yk90qAFdxMu9sbKaMNAqCwOBYx7gPbDw1+PMjXxnL4O9Ffyac+qoDmXanhbRmCEdH4naDXHrHASI6hfJhjMXarAtmMtvUCX0oy68qQhqfEFazvfnFAWn7byW2Jkx4+ycl3qISTUnzbOH0lMfpuTHRJsE7Jy43rizrj4yBZTaW+gEv9IkmV105F9Ma0wl6YHd3JKQwly8bvhgCNrPlK3W7pCP2UjgZu0yjclh8pEpby+H5uwwXR6W73+i1BI+utJANtGFIhM2O8ofhcnhBAVwuplr+v5IAz3zutUVofTUS0UmcNsGhiFKN10DgNunUvKlXmXZn7PmSAG63Msja513M1C/g2byXQftR0ljIjeIeSqW/Hld973Vg+SLriSPRN6SVQ+X60+Y5v/c5B4tMdebPWWKDKruI/bJkOW2NxGI0nGeG/oulNP6dpTt+jtRTzrky7NH7837yixd33Nz/41/7IqDdgdh2Lqzcfbv7uT39i877PP+W1GDg7hH/4sy/ZGXgtx+KX/otGLPQ5TJ5/8vOvbF6Wg/K93/Qu03xSmZy5fr0jNXDtt35m81O/fGXz+R/46s2f+9NfsKmZgFu6ePXhvc2tV/7P5n/8+q9ufu13Hmze8zXfsvkGrfXxE+3h7c313//1zU/90q3NX/qmv7h5z7P/efMjf+u/bb7ih/765s88/HebH/lHL23+wnf85c37nj+z+VROX3ltx+KFFyI0nbV+QzGqcCi44aGXN/IZtZAxCBCw/Bmn3qqpd+HNPyMWNtxElXzML650vp4qJUOdN+x2LpSmLB+21+sjwr/0SKb4IAvpMeCZliMDX0avT+eWU8EoBmsz7CyMUZYY1D2S0cYrV87PAAdZcSbYCYo39sjEM4G3nTgVt/zmXzsoSQYcChaBx7HQtrd6EvlAO9FAvuFYaIemMyyklnx5I8kUIBwPDvzT1CLxxnG5q52fHsppwIB0Kat8OCCn5FQwDYptZm8y/UlOxV05DLI6REcOjhyOPS3qxrjnjeyVK5d0YJ52i9JJ3He1AN1vaZGrjF3gjh/XGSGiuScHgelIGOGMJtBAbNSo3EelV7ad3VMZ0C0nkJ/QGo+Dg7NeQN4Omh036RE+/UaYN8DIzhQurlQdRiEPN/iT3iMW0KEe+tPtiCuh7zvuxJ30TqNdteExO2edrmK5rDHU4wTFYI3x1Lz6Ct3WG23cRod0BW3aBSGwtGyoJyz4KUPLlFzBAVrlW4yaqbxuBaEHbqZAiS9rbsx/cSz6zTa0KcvjjgXp27IFdkkbsYazbMiz/K7BIYTWwFAKcPM9UIQqD1dHd2FCO7D5Rm8E609IuRN1lVk9ySjHMNqKpGvGdY8kaQM9WuH2PByLZfG2ywE//eNYwJO+qh0LMTO/obvWjeTbLYnvld/yLxBLeapgvkC7Q+Pwm0m/lFGLlnvwL4SGb/zXc92lseBQxuWOGPRnHo07pwHXZe78JjRqrSpvhzwMQE+Ymc/xUfOpzwbn2vwGmV2jfgZ+arwkg+8QMhF/N4ORV8Qm8Qf5XVjRnCh5WqMbWuFyIf+Db0fHQuU6TEVDV0+JfKqOBQu1X5DT8PU6T+KrvuTJ50n0Gotec4FIjJT8uKYnsXtTr7H4bZ0E/k+1jew3awrVl2kKFYG1lj/8z1/SbIRnx4gFDsjf+Wef8BkWnGXRofG/7stPe3pWpwP/9376k5s/oXUg3/xV5528OhatnfVqDTx6cfNrP/mTm19/8OWbP/s1X7n5k9rxrF9ftoau/O/f2Pynf/+Lm//68snNuXd/4eaL3//lmy/9Y+/RGqeCkL3w4M4VbXRwbPOez9UOZK9e2/zOz/zE5md+V5sU3H+4Of3+v7L5xq/+os3zexotb6Jv4Pp0x+KHPuypUDwY+uHCNQ87scW5qPvZQO8HRfBUdBk3dP78ZSoUDwPQyyHQFTMBY4kzKPzRgx/jHH5s6Xasplvxpps0v2WmK9Y/D2Cf9K0rhjUjFie1lSxvwBmx4ATnTHfCmcjD2kaw8pwvQ9blEhz47IaEc8FUJ7aVvc70IRn58IIPjs8drVG4IWOe9GeVbsdCeExxovyU5Z4Ma0ZpcCy88FlToThT4qwci2U6VKY8sG0shjq4d7Re4roWYt9nBEKKQjamD7GLlbd+lWMha1zt4P7mmqY49UJspqPta6SC6Uo4Hxhf5OFYMI3rtkY22uGh4pHdb+KkN0YrcHZOyMBvQxVDmfK2bjhYD6eFKVbPat4xoxR7Wsx9av+MnQMaIuswOFiP+H0dH/9Ai4SgdwTHCUdEnzguvDHW9A7B2algi107HZmq1m3MvCUfgfj8cWJ9PSnd7Y62N31AQSYCbRLDm/tMj2E6U+BnGZo+OE7nN6E/YAeu6jpwhnKeG41uGx9HvAO4HdBXhyU1KeAS4GeJhUf7itOA4xCnAvltcDtvLrNgK810RGaLd9EnrwMyGAYZS+9IYCkOgW+5wCd7pk+aA3P+TGO5JAOcpW5n3KXsBQO+ZOpF29SfyYoQeHAgdN0YFhjl2cGSjuLwSmf0MTjO6E91B6wJSJY4FtvTkKCLHkUOioEnkVA6QV54Kdf5LT+3cxjp4MFXwfJPdICJc6F3TYqzo9uYElXEBh3dO960Kn/3Agx85tA0kh5dJT/aJL9hkHXGdp6U1qkue9Ovqxc3i0aHhvV9p8NDCUO2ptFIAR584DtgnTdJVb+xpjfDjXJMONusUr9kV+kLUvRbVlg1koCac7KDZaSiEAIFFaEgEMSmWfcf/LYPBfVN+H6zRiw+laJ8qo4FvP/xv9KOUVoP8XV/6vTmj+vsC57HL+r+t+QkfMufP+9qZU3ED+uU7/P7Rzdfr52dGIlgZOGyzpq4rnMz2rFgROHva3oWC76/QQf08Tz911oz8bsv3vWajx6xoJv4oZ98US/qnvVC7fsPHm3er9PCaQU/otGNj1+5v/naLzu9+SLJw/azv/Ab1zdX5FywRoStcC33E9aNOHP9egdq4Nbmld/72ObGUa2deO7coQus711/ZfPSJz62uXzvxObg3MXNc89f2BzoTJYnB80aufJ7m//1B9c29x8d35z7vC/YvPusXnovZsqTUZ+S8xqORdZYZJQgHT+/Qj9MMPboFPXxwmgZ554Cg7PhAHzH6UTTser35h6bPAw03v5ivOrRboP9fj3s7Vzw61Qeow1sMXtCb9UZSYCsHQto4nyQIDgewt5KljfrXoAMfC0mVp5HVWpa1FjMLWO8p+7gUJyU4XtK6yB4GOE8XJVjceOGRgRk5DPtCZMZA+62Fm9f9yLo+/L6HmW0Qrs+QQP9IN893oqWY8Hhfhju+0yHknOxr/MlcEI8WqF1EoywsNAaBwsH4IYcAXZtghYjG6dYLyFc7wKlexaPM1rBeRSsxWBhL7IzRelAU5MwjO6KzvXrV7S1rNaGsFhb8rPw1pXOiIumeDEIwxkYHj2hijCy0KngcBaRG53jiJ0Qf5yDYyonT0WmYzEN6pScGT+wVV9HJS+OFsYai89l9trpAJbyeVRGdYFxxzQo5Dx2jO13tXWfp0mpzlS+MSpAvT7h0wbDnE/zInRex7nvtI77qnJwpU4zBQ/Hwq1UfJc3xvwGaGYEZBs3wh2jHXYsgIjMTZ+nCTJ2ewcCmNnAAvawAN5jeehXvNqx4JppPUpTm2sngnTzcNmBT1l3aXJPmPlswVQ+vzP/2vreV+QO3RTh8XKEbnRi3bQizVVfpVd4WpaZRKkd3i2nBI1j4dK5hE3JZShyFAjRnNZlp1267cmh8HRDtcF2zAYRycEoGv0JYRjz7VSYbglpZiXkIGC25j9knvKQ6bB0StLp4yoZaDduL+KPc8Hv02WDJvyrPrh1QD8d37k2XfCBQa+mMeCUXsjoDPjGcVFh3dSLr9OF3+0HdOMOmsWHdKcVA8WhvdwVggWA6nbO9l3BclGGZSpBXCYDl6QVTzkGkBG7rJChOP7tD0amRJb75eWOlAFUsZa36Aui9RaFLvADl3osHUKRsDoW0UN//4N/8ZL6283mQ9+QnZI6fff6kZ/L4u3D4NhB6l/+2tXNb/5f7ZaoUQgCjsOX/NGTXrzdtNix6ad/9YodCbar/XI5Au//wlObj2rb2HnqEtOjfkELwu/JWWAtxAfeu6eXaI90+N6Dzbdql6kOHK73s//xqp4tjzandb7F931zFoZzwB9rLX7797OzJOV7jxakf+NXntvMO1I9rUzNY72uGvhs1MBrOhYf/fEX3EGOTlIdoeO+6oGnXwUPvh4JSD85dcGjP60IAPrYoQCPB6fu7Vjowck0oDFigdWrYMdCxqanNvmBXyMWGE4YwUUTuiygjoOQqVBeqD2NUmC0Mq0KHD7wxoBmpOKUDGccEt7eY2h6GpScB07ZvnNbW3R51IJ1AZoKpTfxjFhw7gPWuadB4ZhopIQREpl3HrGgU6GMJyTXSRnPrLM4c6BRBTkAOBMY7MgI/2PCx9hmTQdnTXiaELjCYyTilJyRYyzGlmwsqGZ7Wc6iwJCEB/KzkBrnA2fjppyTazrbgnUSLNYmAOfRFMnPVC46WrRM7bzK21uVTSqxXOjymJyeHgXiPAzOD2EKFiM6bGO7r9EKFsLzBvlVHCHh95tVFhVzHgcOxZ5wT57UFC6VBaPDb42FwzkYOBukowuPeIh2r0lwPWH+dH1JfuK7ofPn9GHoiN9unHubHSo4Iw5MKUIvnm5S1oYdAfNLGzVtmu+wxOIcoP829CNb5B3OrzjZSWlHpQiV2eO7lm+Wnzj05jzLXfK1YwFv2goGcrZqxolYHAmbRaUD8Hdp7vJsvlz755syxwhMGSObIaDNX8m1S6/TXafW3VR/Fd2i2UxFyNmtc9qAhYKXyoe5VzwnFIlUd1z515WP61k6cttzP0N7z65QDePaFR+mJMaB1JQI/VYWZ80ETdflbAG5cdypQy4nz+21ZRNwUIOE/jq0Loxb7d1plIO2po/LDUrzbB5Ff6FWVEkvmEZpmIUfemopdN3F6bwmAOgOX5KsSyIKC+gSW5QXmHwrH5AhgG8GQLMeCR1Rxqy7TmaUZMFRm4c4JDsoPlgpjWJYp41l5EYIh6qxojBTb6JcGydpuUOYgre+6p4kAzSOzrH4trffGoto4o198/wk8Mx+Wng9cJC6osPyIHVmn9/34RSZlsRZEc2TrWD7BO/G4Ll5VSMN50QHOGjTfhpnF+6szrDYXTyun/Hm8s0Hm8PywH89ZWo+63XVwGeTBl7Dsfiwt5vlwdMPH//M6RzL4OLNdjsW/E4bLv1zOoWlwILgXwYWBm62jM18eq+v0EMfgXL6c6ZCQYcfK6du29CVEctoA0YbnTU/ZvOklwCOKUM4CZo2xIgAi7ljvPPD5s08RmsbhL2wGuMX412HhMi4x/nASGONBM7FzVu3NTKg7Vy1M3dJfocAAEAASURBVBQGHM7PHX04L+K+1ixgzMEHh+YUjoXewuvxnxELHAvJiyyMWJwWj/MHBzoLQgueZXD77Ty6LH0ySuLF2HIu7DCY7gk5I4KX08A6B7aU9daxmuaUXaPkHKlsJ7X+gTf/vG1llOLGDZ25oc89ZJSecKLQO7XClLRldyuNLsgRoWxoBh15lKIOhYMmTs3Baa3bkLNB/XG/L8fipHaD8htgyX2f8zPkxHgrW+mHetpXWZmaxSgK6zGOeKpVGXOSgzQcCw4FpB2Zr/TndTPWCY/zOBPUGzCjjQk/xgDNMW+XO6/TuXacdtj38zUGJ85BnAtBCS40wy+OBXo7LIQWxl6mQsVaiEOASUI+Riq0TG8Qye+BW2QgANtl4Mp9h/BZypP7ZQqXDU7V4TCC1fbCP7I1rV26TX++tgxJ8492yta9Qr6RLzK1pEk3iOUP39CAbmhTrwU5IYy0QT90hLToxfxUpuJbELpTaH1x9Sc6RYbUs+pYbXMetfCIhequS4R89Bn0bQT6pKHTojtP70GOEXbqbKTPEdHocsJrYE96ADx6csRx7gHB+U3dD8yiV5JQ9KDpO4F7cMyi6CzYDXXIlfJWcJmJQ0Q0CObTEd8HfsEicyqL4ujy8SB6/m+6uxQi/4yHNrZ0P2dOcpNszYW0oYKXBIqSNkrW43ydquRRH9Au+kzDa1aun9bLIFOwC9ATeCDLM5vvXB0L1L2GVQOrBt7CGniqY/EDtStUHmjphN1f0nnqoeu3sGUsYbS6T6UflULccRs42kl/qweBUDGiMXA5TK5HOujYeWPMQx/HAiOXHaQIGGU4Iawz4E05uOGiiwjbWEMODHgMcRmpOAo4GD1CgcEKP4xmyyd+9PXQY00Gb/sZUYAPBqJLIACMkTt3tZ2rPrf1hp/pSSyA5hTumzKiiTNFCvp2amT4QwPHh5EFytU89ve0oEZv7i+c0SFzOg8Cx4LpP5SBNx6Um12b7shxYH0FAQMHw/40hrlkZHSAcyjuyojHGWDExqM/lFuOBVOLGG3hZG22lsVJwRHiwUrZGf2AJ/UAPmdyMCUKYwW94rx5BEn3jMYwh5Q1FXYOTu+bP47LaS3YZl0IbePWbR24p8MDb2sE5a6cMBwLjDVGZNiW9sL5C5szOlCPkRTO6HhwX6MswiNuZ0n1RZx6RIY2wH1V3SN7P7Rd18IlkN6GRec7Y/oaDqjhVauqj90PdezRiloI3W2r27ev4tnNGfzd0CMEEQ35BN+8hIl8w7lQnD/qncuAK7pNH5wOuzDI6LUhlpnRirzF9pXfDR+JGbnKsShi0DJ/3bf+nFXsLJ14L6UkQ6mTPEXKFMTEsJHbTA3bMgMLLdNV/Y66UiM0S5PXlxkGEhySQxneup/4+zdqvkYCPKFkQaZ8uKTO0Q3t0n2MRiHsYNSZJdabKMCTb/qlTIXSaKpgySe4jKbt26Tp27JRhOI1y7pATrFZ7DAdmb5VWUMz7Yh4fyhXyiJHtuiAM0jO9IRnmYs6MHP2YHpIZMA1Ycqt4D686sIplR7CA2uLb+tjyIKeRGuBhnKVmehSGqdzD3zjwx62bqUVB2sE5QFPGDxK5jmnk0zX5QArbV/STMgTwS6vqYeasURslJMO3QG8jis6hFGmC9AJXJ/ZfGh1LKK29XvVwKqBt6wGXrdjkRJWJ0hvXJ2oDT2MQacJSp0l3ai/qz91l0l+ddQ8sHEC7FjU22k6dh6WOBT+EK+HOZ0106EwqjG0/dAv/uRBC2OXPNYEMHKQU7WzkFioxjcNwdCf98MxW9PKCZEjwhQmHik9HQiRebj4zb6mQd2Vc2GFyeDGybim3aJwOB480FQkOz9yAmSYM8JC0SkPpWYqEY7Owb6mQWl9xTm9xT+jt/8nNe3qiKYaYbLgVGR0ROs5RJM3qiCzgxILns+cwZA/4TQOwuM8DWAweplGhJPCSdiU7Z4Md07XvosDIgcFOByOLGpnlCS7MuE4sFD8tj69bS46Qm5Px2JxuuIY/6fFn7Un6MmOxv5ZKln4tzdXr17e3MSRkXNxj3UbcmQIjACdO39uc/HiczpQ7znLSHvB8XiWcllmTY9itEJ1m7YUA7x3lvJIjqmRro/w++FN3aS1SS3VHgJqk0C6IFd/VIa/nOt0yug2p3Tink4ko1OeWoBEe3EsiFOTCTZC+kbXvucasMBy749kIKXp2cFA3nojDswybWqht7AoOqaXVPi03J53D40qk98IY9iYbpwKyhU18E0zKhmdGpokJZ3yCr0gk0tacAp6uizyGUt8O7QOTNllXhyLlsGwTXvCJV+UK5s7C+j7vPUmd+GVDGQhpi+uumkZwPELC/0ecBb4LXO4oPWI7kwASTNadZhjYXrANTDxRaximcwn62sHv2iEZjl9fnETfTcr6LmfFXzLbIgGaDpcCaVTZ5deiYeqIZ76NcOhw+0yKxeAEA+dRtgBPYyJ6SmjURpmrmPSkp92YFZiGJa5a7xcO03wHW0a3b62CzH4Rx6Q+LRUfV2ItdzwW3KVX/TTZotM85p0vyAtNIMrakr60Levi7fR7RpWDawaeOtq4KmOxff/UBZv9wPSfae+0vmrIySuByCGEtd+yM+dL6opDIE3ThwBG/oyFPXvN+/gsYCWN+55Gxsjj4ep32SX82BHQkik8fDH0bDDwVt7GfF2LORckOYpNbCVHDZMkUF3OCJMXzopY/m4FiNj/GN0eI69DA7wvNOU4Hlrj+HPGgucHQxyjPErN3SStd74k04ZcShwlvrhz6ODxczIs6fRiv29U/4cyOA+I+fCjoUMbNaTmaaMcnag8qJn8VHhfFbEntZXMHWKERvWQbD+4q5GNXAGcCpYv8CIAo4Dax3uydjnJO57clBYa4HMjNzgoHj6lWQkUF6Odr/H21uVgbKTxta6yHFNC9dZmcFBfAfa/Yn1FaflEJ0RL9Zy3JG8l9nK9tJLciyu6ZC+Wz7hmwckozb7Wkty7tx5ORYXNxcuPGcHgjwO9EOWU/s4V0zvYsQii2VjPFGfmnJF/bq+qg0pjm5J63QXRF99v5uXtQ/UhEJdiLZhxpVpeMB5dEP3oSVexb9pdrtuXiapNtvp3HfAlkh6HJimgYPXjgX0gcGk5dp0MmIWSkknjvANR3lDn3wcx/69GJ62o4+3V7VDERmaPtRorwmTUia9Ir+ZGKhho+e5/ItMLb8LXrQXGbv8OFfQ5TqoUhjCpANuZz7GLwxxMqx5Wy9AJ7iMyC5aTZ+0Ud9q36yZyMhopkQximG9LURU95kyiay9xgKaI5h4c5CssCSTMgygJbJbFuTpsFBRipBbV/DuQiBfp3PtMg1+zRSU1icMCrbhtvJKgOYPzScGylXZhp94LD1/sJvOTK1lB4L8hmnepGd6mWNAEVHxXSDd5Z5r4zaMAbe+dhyLSdYtHJPMLyE0SeATqaKr4ovMxYPYLLeTxYO01lGBLhiN7JStG6cEW1OhVsdiW3Xr3aqBVQNvOQ081bH4PrabfeGFFIr+na5THShGtw0DOlM+vufxkuBOuuGD5Qx31MbPiAWOQbYzVbaQbeTpgRtjkKS8vcNgx0nw/Htdn8GREC4OBiMFGa3AGGXEIIuNcRTi9NRjooUTKww64BhF2JORDywBA8JvMWV8UFY7F4L1tCYMbhwLXb3GQlOgWLzNAmjWMAjIZRlv2nnKKLBNLs7OnkYTsl2seHoBdzkWymNcg12e2A2Kw+zsWOgJxaLufY1wnNF0IraphQfToG5yrgbToASDwc9oBo4FunwkIxNn4aHymbaFrAQMdcrcxjJasWErHaN3wwqHkZCbXrB+Q+d33N48kk5ZB3FK/BkROSte586csxHA+o1Ll1/ZXNOIxR3pAt3xZBVlT7tixIKRGeRjW1r4+ckrQ47tcA8OzvkgvzgWcSSA4eMpYqLUwWkYo9XmOp1rt6vOW7BgF0PE1a94B9LjwGZkbIwYID9t13wiC3G36UaerqSb/05a83UrlnE/04tjEafYcLT0ki14i9EJ2aaFw7HsCMSvo9QpfVKHmfYErTgWwHaaInCJlJSvddsqcaFJJ8wapARJI3W3rA065DYfAU7lMbpJVH9RuhWxytJ1wMNjYDR5J45kC8RXPpEvONajk/U1aKIHtXP9dvmwQQK/Yxyyh5z8Tp+D/ootsH55gRMkCexYCL7pIRSyMJ0vsUIkraOiAZ3Wc8pUiMApz8HXhirkVgDkt0L0R79EgMb8MZUJp+uqeXHNKEwRbRnqtmlOt6O+TaMLVzwGq4kOJUYLhjdAoNL+EyfP+cUoqaVJ8QibRXet40f2PIRrvBamdYluyOBr+/faejBay5YbMzOeyTXtSNRyjfI0zriWDF1fna7kRTolLoQQ1gmgNIyzfbM6Fq3C9bpqYNXAW1cDT3csPvzhzUd+vBwLlZEOcBmhKCOvnIq586bvdF+bL2vHnae++gHjkQYMSGgKjgcNxh0jFnTv6X4zyuD1EXqjnQPWsqiyDVBfcTBkAONs8GYeeJyN7tDTcVc3Ll6sw/Biba1JYNQAGhgbD2SIMzKB8UEgnXUJEPKbfPK1u1JPjWI61G1N62HqEtMseLBzhgN41ocYH5N+MOgZteCNP+sO2Hnq9L54a7SEaUCP4C+6rJtoxwL+nFnBCAEnXeP8cLI26yZuYsRLXnTI+oWzp3E8Djxa4d2ZMJRUBpdJxj66td4lnx/q0i6OITQ9MiA5KftdTaFiFALH5ZZ43JGj8YxGSY5qdIEzLDxyorUeB1q0fV+jItd1hgbToG5pZANeBNeHdEAdUGbvtMWictHwA1pGMM7gvkYrDuQMnRQtdsNimptxXY/UMXt5qx3QmBSyu1fe9jtBMqeNxKDYcnQNoHzhNn4ljXsbkzLAvbYCo3IHFn2lLBiXh4emTb27qQvM0orWcFQYj3AZgCmaKn/aSGi7tQuR6yJHyg1n0hYnIUawEi0U38mHT7157/L4WvyLXIwzWrR/Faax/UNxaSqdi6XTNfLPWA3UkrickssYXEvGhkNJ4Luu1HaH0oYs4AxpjGZ+rVxwSTVDvvLxLekKLUOySgYRdX3wu1D7W5yKGrFQn5NRi5QWesOxEO+H+m0wOqgfmHn4qxVh2fomEjm/yt96TjnIUUz/QzfWUfDzbWzB1N12ojNpO60y2jFKa3rb4OFuFqmVpzoWTaMkePzSip6YsM11h8TUx1S9JJ120xAjMuQlZ0ktuIVk5W3T7DYM9CIzfJpS+rmitkU/snVOrsES0/wrMSl8j7KUTM0ht5UImeZdSQsfYTQScF3fpJEFfOGQ9MFv/y59vzmB30HrrK8zZ9LQbzvevHTiwwj0pxJ++WO/ufmfl//fp0JixV018Fmpgf2jJzZfcuY9my/az9bCn5VCvg6hDjSjxs+Z1wF7GMhTHYvv/fAPbj7ywgujX6SjyehEGUaTU8EjzP1nf9HRVmdbl/B3Mg8BOQPQU+rSgaUT606ZdJwFDrHDOGeqDwZmRkygoU85FZkeFXho02tnoVx6bjpw4FEWuzMxasAbdbY3xejoXZH6LTYdKrCci8G0CDpXpggx7QnH4j6OgAwO7+IkxwJDntJkJCXTKCgbZTyi00biFGnUQM4FJ1wf2LFgq1WtL5AM8GOK0PXrN/yGlLfaLPDGscCgZxSCcyluyoj3mgjRZhrU81q/wI5RpxTnpGwJqpwEysLDg0rOJ84GvHBK2FaXQwTRKVvDogMeHCwgvyscdo56VvI9w7oR7ebEAvIDOTsn5ATc1GgF6ypual0F062kXDtzLMJuJ4+2guOE4wAPjDfWA3AIH47FPgvYNXJBOY560XgcQ/SBfEzlYfSKesMJYipY6raexUrv9uPpeNzXhwc45eQzh06jzj39ycZmG/8898sJEx3qf9CbiMw0iQ/egjF9WQqzY0HbgY7MVdP0iIVozz9c2nzTBb7jsDVN8cFxcJ7zU8+SVunAtHFchiYwRQfaTwyWC9mgkd9I63SRwRmCiH4NbPhtuoGP7I5DcA7Uje6tU5VfEe7qOgE2mvCjtymPKPlNWz/sUb6Rjq504w+X3KMP17na4HAu1M7RqdsmIzz1Rxu2AyT5+O34fBDRkbYtgL8tf8oDK8KQF55JSvoUJzrnLWWRLjoD1VSwvsRri4d+W4SMTqV8AZeMTUMJ4ILnJKvaX66H1FdhNfHCIXXON1TDVLmHQM7sLzQ0CaDkBlesgXRtKLWJAZ5I3wLtFiO+Dd3XJrXISNuEfLA6vTlyJc34umm9gFHacQx+zd8ydpmVCH3oGKZpKb0XZFtWUzHmoDPaBHkLcRMbZS/67wTH4oP/4Uc3P/E7v1KaWi+rBt4+Gvjck+c3f+MLvnrzVz/vK97ShXr3u99t2/SNFuKpjsX3ybH40Rd+zJ2pu1P1rDa2yigaoxf15KDTpff1LcZ9EiKb4r51R40xJlqGWR58MY4wTKADqSzaZkF01k7IAJVRC18eEuTboZDj0UZgr2+AgB8jYaq7clJkMOBQ9JkVXtCJAS5HwQ8dHhj6YBhCHwMD2qS1cc4hfjgWdirkbDBy8aoWcCM4sIyWxAGTfE5DTtHRxwf4ydjeZwRAIxaMnDAVCF44J0wpeihavPFnRyXOr6DM11gcrUXbTJMSsM6zyNqKs3rrzy5YlJvRCmRIYPQnaygYUWENh50iOQHZrUqHBDGCIlyuBBteMrI4WRweHJx3hMPwdH1Gxj67OnFQ4bOCuanzMTgnAycEXTHi4EXh1TaoPyShbBzcd1J8XmWqmeTINDTVAdvVaoqVt5vFIRHP0KmdssoQxFlh4Tq7Y1GvBJdS8GlHtLn+yBkgXzLx2Q1dt53fzkXfA9+0uk0dRqPTDCuOuS58eavaNDFaJR6U1T5oT/qo7Y8pLUUMeELj9ZXSJo4BXA6ErlFF9OE1FkrD+RCQrvr4Ktyi/9glQikZGqEDDGXxbcnje0MEpqS0TMBb7sGEyOG6B7YUUTziXLQOnU+9IfGgV/SVSX0QpG19AxMg66YQIkt+iwY2WJzIOBaKy7Hgt+ZdoYhLT/5MIxL0K8gFC+AyYtE8i28Z9+bTX8nSXemzLp192NUyH5LhcjaZqisXWWmjjuCDzupDr0eYDVanAKb06LCIzjozFnQrogtulEPRtjImAHgaYvkayNHdUEbkVa7RoQekr6lNI+7QdlrBjbgjhV/xzrNOfJN6G+mKIKJ1JM6LVIGY2EZOAQBjpy0g/m6dllacFpoFJEKtCtITtqCV3zXU+YJStKG+8zu+u/A+/ZfP1IjF6lh8+ut25fCZ0cDqWETvr9uxoNPmDZ4NZxvPipdjQMfsLpXe0XCOuLecus/qPJVCRwqO8Ksrdqffb5G7U4YXC7wzlSjGJUY2xiSwEGThdpyNcmR4WBVTLvABDhyMdWgxWsHbeg5v81aSTIWQUcFoQz+cMDTAt+GqdOKkZXE5B/lpa1g5FBmx0A5NcgYst/jEKIGW8C2ERJARYsdC8iPDKRnce17jod2cZHh78bJ4sOAaOrzhZ/0ExjpG49VrWseg6UcYRIxyMIpxoDf+x3XAHAWmHFzhSZHRDyMWrJ3AqXigkRavDZHMnM/BgmUb+DLWObsDXbuOhfsMU5kYIZCO2K0JpwIng3SmY93Rad8s1maBuB9OysvakiwKNx2cPwlD2VjHwogKU0rA8SgUfDVFak8jFzgMnr6m9oDjRZmtbww90fD5HDoYkNEPAnmU0m1R+dRR2meuANh88NXA+up2kSv5PSKCvvtjykUT+lZok6ir61nxbivRW1qy6Zj7QjMGClIjczkWoo3cKUsTLrl16/an8jevyLc4FjgrBHNVhatpCracCdpux13uLS7G2y4XLYZPlSnR3Pu7+FRc6lKgfJGv7wOlTOG33IWyXFRmQnSneNVdcAM2487xUceWVfzRHrJQf44QlSL0e3EG2aNu+Y3LqZBu+A35YDyPVsjBkC6jb/AS+K26IKIbxwKnPfSAeAbHscoSDH0v6JV0iCIH8HZkLmfnpEXpbiLTeYvekhJ8GoHu9TEK8jZC0xj6nvEaKP0Hd8OpyM3j9WnCgzoKaSIlLvepleTw22wQpRdqU3DWArDwa8BGHVQ7oSlQZlEJoUEfKJL4uC2UTKRP2pFsRgx6oN1eApdvQ4hdIJMWmgUlGsnr1tj35LecQHT+ktw0V8cCXa1h1cBbUwOrY5F6ex2OxUfVWZYRZ+Mz6xls1I03doKgU6V3dBqRdJUME3c36hQe9MrledEGIe8h6aBtDJNPpgL57Qyc1JtzT4VidEIGaDsW5LO7E/KAlznz3YmLBga9PjggwNmYtlORt+IYDY/YlklSmU69qYQOhifpY/pNycUUIdZEMCXpDs6FnAHuLZMwmMbTIxb9lgt14Dy0DJzEzYJuRk96ATl6wFBHkV5sLaMcpyLrHjRFSqMN6HlPBjlz4Pa1rgJ4z33VNCn0ix48oiM8Riao4DhExDUiIpmZyoVj4TUQGP7SRztV4GPocyYGi7V9ArEcCxH11K8rVy5vLl9+2SMr6IYRHRyBVCrqyrSfHqHxQXqSc0987JTcueW3v0fkuLBDlXeb0pa7LNA3bzkPOBAy/0ySbWlPaw3JccGiEz/IpQP0gDPjqWd1TzslnRbWbUgSRbSqO2WQNNJmgxIcoZtGO9FuswVvxEO+0vbhLrqi4U/JALs2wKFNGZ/V+hFfpVNkdeBSzMBvuWwoD7rlWGAYl2NhXMrcfGengngRb5lgYnmbGfkI5qBrR4FbUhXLHXTETPeUE8civ5MhvhVYiGAN2mDMATlyHxhuSocFZt2YFwklt5CAtxyWoeUBJrrXD9EyzjCtT14gZKRCOizHYmvUypzETW0rcpVjIX1nly34UC6+HK2CVem2CtkAE6yigKSk2+m6q3JBX7iDJO1yogUgQWktI7duK64fZdX8HJOYUR+jsyUwVCBVAcTWdZIGqVm2oYjGAw1IARWc5ZyQnbxkG3Euo9vrqPuFbqgW0SYOu1AoQPoA9BH3CLrO133+EKtpFIouwLQMjb/kNg9dJ9SmZzilb6t3kWqbX9ItRdGKhI827wTH4iO/9fObX3nxv8+qXeOrBt4WGrhw7GDzte/60s1XXnjvW7o8Fy5c8Av7N1qI13QsPsJUKPWWMYh4qz05Ft2Lqp/06IOMQ3fMSs/zTd3p9HBYulO9VKxHAcY0n+7qMdptBKjjB54390yF4q06U3Z4o81DHyMeXhi2LMbGCSEN3Cx0JV+0DZM34TgVJ0QjU3YiI4Y5OAhs49qGuSTSfTp+yhDJoQdPcsC5o61mPWrhE7nLiJMOYBwM4crIAR48Tz2Sg0RZkIPpWO1YwBuDh8XjlOmYRiIopw+x0xQodqZBJhypA73lx2Bn3cG1a9cyRUqLrlVYO13UFfyRCJ1gcDCVg+lerBNh0hcwTMXak8HO6AnBepCcpzQasifH5ZScC5dFsKjoihZrv6JdoFiwTbBRj9ySGWMOntRF61EFMS2cB6ZCcX7FbZ0WjgxHVT7KAA9P2dEoCgcUspieNRnIwhQoHIozZy/WdLEY4l3vR4aBLn1LDuqa4Jqj/ii7U4h2rBJ0gcfj6WlX1CA6GqFoj/spAl9kctuwvtF7aEeM1AHSRXa1V+kGPEulL8jPErZsMd7Br9+F365XmxW9yN/lFlyVaVwlB5SB40+c8qffMbJQacjokGynOa+gK9cCmgoI5qMy9lVJoYceQIxMdWMSTdOEkgJ7yyVtO6W/IlK+SQM3+EWXROuY8lWgjMjGv6+5jw7LKZO8/s3jXPPb7wXvgnf7Fyl+7vkNhZdHLHAsTD+8lrIsvFGkJfYVPSsgt68Fp4uyE0K+ylUydxb6K7iiOuCSEeRZDjsWLrzYVp/l+i2ayNLB+tFNJ7VIEY67hu0rKcSVN4Cb2nIdWQI1ZiWYTzEbMFArZfS1y9P3oRyMIatv+Uo6JJCtcblLziKDJTdgaxPshgqXBT/3oz7rti/DseiKdEF5nqQPBG6m1XzMTYVo/MjUVLm+MxyLucRrfNXAqoG3nwZe07Fgu1kMID9oZWhhwGH40nH6g04q7kXVwOiePpeONm/xuQHOoIrS8bs39jfrEIgAQtSjBRjDuvGIRY00nDh+1MY003NsTogPRqxHGsxT/JjWJOMBQsjx7BGNemikoNcS4FxQFvOSMYxxkdGJuDoZbUgZhO4yGFgCcm9jU1d42KnQaIV3hRr8MJDykMHoMm2loBsO7WMK1ikOmdPVC8i1xgDjHnnvyvBGZzaoZWQzjYkRBk7PFoB3nGLUBsfC07hE/9KlS17rcE8jJzzA+k37Vl2Qg2NRH/JYLM0idox5nLUOGLwHLAaX0c8BdpSXF8B3NVry8isvy5G5Knlu2TmLYfzIcgIkrbk+kNUna0vXbIPLdrPUEdOgbmsaFdvzsvaCbWyZxsZIzKv6sKgdveBYUHfZ5la7R+mUb+rcFaB0j5LI4WQEA/l6vYLrjoYEf+lxfJSSNpk24bIqH8eL/8MCdW1nWREomvdhgEo73LGY+FuWGPhQQ8eUAblN3QxCvMWx7KrfdlDiaNC2eXPebXxH/OITZzL8Y3wTX5woeFpX4k/hhDaYU+7omaxJMCUjE8FX6FmOyONGonbVLxgMGCUqSjopMz16gNxDteNAESRxIro6L4KJTtNYyheERbbIt8hp3ZUuiXt9BU5FOWnWtXUXmnYsSjeUq6dKoShUsMgQEZvfIrWALI5k9X+Xv+GraFWUpmc6nUU5TcaEKhXelCu34G3jgtDw2xp16pbuoNG6DT3DDPykbX+LZpM39nbuuOtykdDwSkPWvnVW8ZrLTfp2mQyprxmTNELqI3FUbSa5dmLABBp8YQBYwXeOB7fTO7txpvRGn/VUeu1ycNvabwq5io4yW4eIYTItj27eCSMWkzbX6KqBVQNvQw081bH4/g/nHAsMUQy4Nrodp4OsDpUrnxmOTpZOM1fiGHX0q/oSvdG701EDSHoFP/hlAPBAx3HA+MQwbcfCRrWJMa1ERrKNy+BjQGM8wJwpQTZCjx0ZOyBBx0EsgcOxYJSAt5IEl4Xy2vATTfiUeF1G4BhZuaM375xsfV+GMsalp+bIiRF4aGuEgDJwjxGJY4Exf+pU3szvaWcopjSdPK4pT5KBhdvomPIxqnLj5nVPg2LBM1O5WE/BKAfrKzD2b+m8icuXXvHOTEyHojyWX7JzQJ1PKpc8BByuTPnSKBBTsuRMHBcP4j3CAF8M+4P9M762HPckG1vQvvTyJ3SydqZcMVpCuTgLgEXhFkgJvudtsAxPRj7OX7i4Oa9D8jCh72qNyB2NWODgePcoOTWM6HCg3wM5VazDYEte+B5HDjkUbEnL4m70gsFHfTJV6qh2lsp6G9YspB24XWI1uM5watUWaFsKbneOVVzp+dM9OBUCTVrgrE/Ra3zSgVmMCLWVygcGdtClLhom8bRJ2LRjQXsxfXHzC+ZibrmQTzR6mk7ToJ0MJ2EWFnjjUG5TgFVgkWWSxyWWzswbvQqegB5QncNUZtMtGPJyXzSRx86OaOh30zS5Qqyvpq77cMKgJRSzLkelIH1CX0MraYXTMMhl2VJ+YFreUWb02B8Wa9MuuK9rw5FmfH25HOIBaf+G1U6Hnqpcg8+km+CX3LqETq4mPhjUHXpSaNqVOi676YA3O9Mu/NCI/MSpy1z5Rh7dC5ZYaJZTQTYh4InTuSjQJpsXNGaKDd/5I2+i42hImXfjmHYjwoe4/8VDMvJZYCCwrXtnBmKSj4Tg0dcm6N60I4SoVHpf61Z4/v3p1hRCpnAbhmvRGejRIfJC2+VAfuiQVnDmO2gqb+Ar0elcg/+hb3/7L96eNbrGVw2sGnj7aeB1ORZ0ksOxwGBto0TpBC7DqVCnno5WGepA6WzdsRpSsG18VOdPJ5sHXQHo0mdJYPTDtxdoY3R621HesMsoM2Px9iF5EcWGKj06NFlEjfHJ23mfI4HBDZ5Cy8W6g+zypKlGDnGQ/EZZvF0u9f55K557CobhfFcOBQuibSgLltEV1lF4hx7JjnLttMhg4VkS50hToFhbIafi9GmdmC3Hgi1fKStTq45IVhb4Ejgn4q6mOGFM8nb+JFOntF6BtQlMkbqmBdTXrl71zkx2PpABh0GGOUYrhp4/0oWnhzFiUcYTdWSnS/KiIy8I17Qk1nswiuFzJcTzmIz6uw/ES+dnXNHailfl5BxT3ZGPsU94VWtUmN7E2Rc4IHeZtiXe58+f13a4z+tAvbPOx7FgOhROAc4BC8LZ6eoOu12pnOjPzo30c+7cOe2KdV5TxU7bUH7AlBXxOqFF3DhW6ODIsxl9StusOur2R5mFMRwL4VJmfxC6AjTTjNOA5rZIOmX0yEKAjLUNQ35oA0cefN2+FPc99a/2Ejym3MkZcluUDs2khOC3sIW7GMTgt3GMIayGDpLl4XvgYQD7T2nFX4gTf2AjL6MLI0iOoRvLFAOp85tWl8HlEV07OtBHpqYhnaG3oW/oIevghwQKvjSfSivZKzNwfIuGiuPgUvsrZXRGl7V1ztXOgORCTkYn/NIh08jidKJHfhupmy5j2IUBaZ0/BCgp0DMBmJiWJZRTl6+kdjkrvUFdroXOgrUdswqdJK4GDwHaPsEyKIlrQvEDrNOURW7LrRoyqL/HF3qeoAJiLNehMRIzJ3914gB23aN/B+hV1gxOHuVq8SKd5B5plLUxcvXvJVSnPOgsvNX5NLs8D0ZW02oCdQV6huk4vPU/brnJfyF2nSpRwXqVHMDz7dSp7KSONGMsX03hu77jby6Jn+YYz5zWW19nlqSh/+53vJZPL7A+1XMsZh5rfNXAqoG3nwZeh2PxY+oP6ewxqtVhl2PhuPRB5xNnQd35ZEywLoFOiX517p2BwVj3lAl3weq4Cs4ds+KsBaDTo0ODNoY2b9a9xqKci2eewbEwZcQzJTh1pw5jvxkux8JGK7T0F6wAY2izvSo7JSEH+MjoURDKirOgsuNM8enpPxgmLIIGF6P9GHD1wehiFyZ2Y/KOTBj0SgOfMjAVCofCp1LrnAqMesrK1Cfe5KO3+zLUmXaEY4EuMLj3tT0rO0GhC6ZIXZdjwXavOBUiMPSELOiXt3CUCdp2oNjuVbr1w1pKw9E5zvQsOzr7Xu/B1rDWleQ4hgOg+1v37myuXr+yuS4n5pHKfFz14ZEFnAvRuXfvgRyKm3J0tFuUHIsHKitTnZ5//vnNeW2Zuydn4KamQN2Xg0Id751kC13tyCW5LmvNxk1OG9cDC71zaOBprcn4nOee3+x7V6zjmnKmMuqcDdoBU6v47MnhYI1FO6redlhtwm8rqUQFeOWBTbugHdHuiBUADUchl8StG6cm/amOhVDcooou5gz8oDHehivO/exYUE6PbpX8CNAyWWIRCX6NeElP1GGM+dDSTUuZa/Mx/zKUSxbaBrjWSKGhhxQ85bYMleYUyxTSfCNXDHURgBfuU8ul9mnHAkB+I/rMjkXKp98d9EeATpe6ExHOnDphuYJbsi+JwCpU2VPOyOk+xeWObDgWvADI9KfSZePpmj5oYWD9mHR+P24XgptDtxVSu2TbEAs0+YFpyEpQuQadHfoLduGCiswjQ/rb0ikZS+4AK7pcwA5KdD9JI3DdlZ5NZXm1PkjNkTnbPec2MYNGb4U1IzgJfpVXMnZxcmuBC6BpNHzK0pmLHiBY7SoFjkYe4y2wWVUtRxOcMht11JNhgpDvrhMIKsX/+qoypYwNCfLj5WoKb6ZjQZ/SYdFfp6gYqgzKDByf1bFYdLPGVg2sGniyBl7TsWDxNqEdCxtwGA/qdGxA6EoHFAeg4rrnZVUMEZDBT0fVeP0WmA6VhzpTizBO6MhwLHgTSjq0cRB8SB5OhQxWRi1CEAMM6YQrWPjItPR5COYjwxdcdjZiuhVde2//Cj4yEzDiPEpSb9jA9VQi4drI1pv7nm5FOWBJR/tAn15DMRwL6Qa5MfbvaESD6VLehUnlQV+MJtix0GjF2TOnN2e1ngFng3JCCz0ztYrTrK/rnAjWV7C1racoydA+LccCI5G3/7dlzCOHy4VgBIQTf9RhvapMTFXyqdoy3nEsCOiFnbYYoTiQLOfOnvU0Kw6ho05OaNrVntZAsCvTTU1fuqIF2zc1aqGK0VkWGj2Rw+FpZeLLSMU1OQd82CHrGembNRrPP/85Xj+C7q/qHI6HGnXAoTijqVbwZ03Jix9/cXNDC9BxpHCYzqh8F7Ujwbuef5edKRroNU0JY5EtDtjF8xoBOXdRTodoSFduB6rHXm8xHFapwyopvai2rf9ud2S63UoXXAlRnb9973zautu7eCkL3dDWgELx0KO9AGt5Kp/2ZDjVT+CNoTgOYGh6VEn4C67ZGp56ZQoOU8u8wLjpULHw9zXwiN/3cLEDUjBJB76NCCBaOcHve+uhdIFGfA8IN7BVW+K3ImJKqN8q5eQj+RzQBeVT/YK/0Ew5wSTggo0FxknSN3oFoqFglfiQZcAueaOMki99SGRDhzgSlJ3+JC8r8sLCcEWr8fm9EIbc3CjNv3HynG9lk2O6vlJOpxTIJH9nWJ8udQH2pfTtW9g3IUW77JBbwKjLyEniopdGJK/yTaO/Qg/9dvmMMUBLQtHsaUHCBHn6Do/gpT+l5INjsg3fX8ja8iYbCaqYFOoQHJIaBzrWbsupzM4LJSDQTxMqnbi+lNG6IlplAR6qvhfdBbXSArCI5gK2ANDpAK9Fnk7l2tI4LqDGgWfH22EBpmX5rg++eSMW/D5aPkemr9Ynus7vaHUsJvWs0VUDqwaeooGnOhYckPeRH2vHohyJYWjtGGoy7jIdKl0qNjqdEv16P8jiVCx4LRfdW7/htUElI8DTFoQMLiMBrAnINCjm4DMVKp31Q4wJGQ7wopP323TBM+XJC36BUwaGhZiIU2jauGOqFAaQUjHi4G15MVSbrwzh7DqFYcQagsAxAuEy1mPCoxXSDaMSEGF6FVOlvAZDceAxtpiSxW5MZzRicZZRCzkLTIWCNtvD3tfb/1s3b3m3Jwx68DFsDjSywSgHxv5djSCwDz8a4BRsdET5cUCYkgTO1qF4Ms5xLhiVYfQFPeKocDjf/p62fMWx0DoI7zSl8vp0bE2HwnmANw4O28xywjYOFIusMfKtN5XN2+5qZIWtdz26ItxTWpR+7twFy8jakUtyTJg2hgPALlHIzDa6f/DixzQV6hYV4NGKi5o+9ZxOEz+r6VPQuqWF4qwlYTocU6A+57l3aTH4OTlB+2k+ooNej0qHjGj0m3I7EKpH+BDaMfY0JNWRHRBlYbTgYLqtCo42YAzyoA2sPnYapGPgbIAW3eFUGCa8GMFa2lNwqCu3PrWDposskSNGdxhLiOKBUfwAx0J6iIHsH5XzKVMHygB1oWUaVv0mYIi8srZzFYLFlphObwIwroxFX7PuQAhOfmvQC00bHZJvHM4oOta16qSdC7Tsj/JEpoJ+h8uNcIApuZDZQdck526kV7Yu8DdR5cW5iSFkZ0J5eUHRjoV0SVp9WgewaI5N2e2IdNEFnqvhG9Cy5IZv6mAJDVQpZDlbX8qirszQugJmxg1Oy5Y7fQ+SRMaNUaM72jDQyUua4vwnQ9HkAeV8lyncmwaiABWJFrlIW9KVr5vOJb3xiHaAZvOOIJ1jjNxApOq+CtBAvprHBE5i79KHELtlCk+whGS8RhZk6SGEBdNZS8RZj30VnGUJZYOYnBI7vfF27zvdDluXlcRZnpL1Q2/iVCj/diRG6r6lzLXT0Fn/XtYRi20drXerBlYNHK6BpzoW3/uDcizGiEU6oBhZMrj8ppY3r3qktvFlQw5G2fq1++0YXxhnGGoxWMDpANzcgbXxjoGG0WfHgmk7erPPlCDm9/Mw4qHS29PSSQvUb9NxBI7JgGUOP7Qx6qHZb1Xp+L3+QvlsZYux3fwtswAwKOF7XGcsZJEwZWJKUaYTYXD7gQisZAEWnXAeA3JjEOJY3JNB78Wf4nlE8mPMHzDdR0byGa2XOC0DnFEYHCRGN25oOtF1vfnnw3QnRkXQ2VmNKHh9hUYZ7iod7VFOtq2l7Dgyt7VG49btO1q3cEtTpO7EuZAxj6wYw5SbUQbvTKX1Ggda57F/oIP25LCcliHPlCjKyhtoRi6I39aIyY0bmuKk0Qr0bZ4y4jGK0Qdv1Gcj+llNRzouOidPaS2IynlX+FeuXNm8dOll6wNjkylO6PvWrdubT770SZ9vgcPC1LDzrK3QqAUOyL372p5WzseDh/e9LuXsGUYznpNjpqlUkgF5cCZoD8fl7DCNDPrbzkPap9tr5dmgp81KH36AElGg7sfDlgTp3fQNm/aG3HxAxpgEv52Lfhij634YxwA3uDkgMwHYxbEInYZCErfHarceteDNu9pIpDSJfEUUU4W2/wS3NW2pZBbLYhEZKIbDkmG5uhy7V5cd2pKrP7R1nAocHyUWuRqxkN74HYUpCkOHHXYcC9eGS15FXCDbA7HeG11XywPPkgn+SVuchzhl3LMWqkctuA+siz7RJEq5u+zQoy4Hb4ulr7r6Ak7JH1KdWuUx0eREF4o3iNuev8KzdNjZxho3RMZNEzTJRd5OjkQqpVD4PgRvSkIPYMyjFUuZlF5yQd28ChfJm8yMGzi+Kx/8BlwiQx2BfPzbmjHqjB99ofK5XK43yrAwLYIF1WXoSl8AH2fcKYbpm+U69EFbWZK34lPyUmKA4d+yADR4PNqsjsWstTW+amDVwFtRA6/hWPztzY9qxKIfWhQQQ597G9MypLn3238ZeHYa6NjnfrNg8+Y3uKZH5zoHdbR01jaCZaRwxSHg7SejD3YWhmORaRb0zRhxwBEw6pmmwyfTdJLPSECcldGDW35gOKiOUQvKAy0gkB/HB77AkE+ABoume8E2DwivvfDIR7Y+5THTOyMxesDUIxv1ku0Yu0IxSoBRr4XSOBcHciyOavTgrhyQazLgX9H2sdc5m0JToZhOg95YB3HuHOsKsj0rp2wzNeuknCxkw7BntOCGDPUbGu24cQPH4rYNeWTmaYeB7Olkwsmaij0Z8Eyt0kF7cixOaf0GoxBeR6KyAo8urmltxW05Kqz/eEZb92IMewG8dGXjWbo/elS6klGPMwKNEyf2JPMp1KMpVFc2n3z5k5tPyIG4e1c0hH9KzgwNDwfo2pWrfsiilwtyKg7kbEGD0QymSgHHtDScivPn5XRotAKHh0JRWzggOGaczs3ZHxy8N863wKjVh7KgJ2A5F4X7dmxphR6x0JUQx4KSqyaNG+MYN4T2KZN0GFl2VgRDnUOHAgNjOOkFx4u43+4agPaYtgoGvxcJKLT8hiDhIPaUjb+0W0YtMIYz2tFggQWqg+LF345FGcRKUpAcJYPf9BYPypmPS2xZUhiKUwh9hVDRj7Gt8omH42qDOBsE8HzAnPRsx8Jk9NV0zDG6MkLdAxZR/V1ZxKOLSogM3Fgcyiy+o6zROeWPw5sRRp+0XWssstai6sayQaPkRvb6RJdTm7BY/grvSCtE/k0oQi2Cim5ogzD0OWBNRtDBJh94woAF3yl8KeabJcWkSl5DjKyhzehnpFdEOBPhuWqc3OXhOlDFwKXcwlW5Ipkzt2CBI3DBkSNO+dzBBnImNegANwfrsCkXTYg6WnQKPvIqo/SYZPF2ErI2HiCFW6QtS7OZ+RMHtsuDPJ2vNMTgM/KJT/zNxwC7MIUHvAJwH1qnQkUZ6/eqgVUDb1kNvA7HQidvd4eqYrprVieJIR7Dqh0LGWA4GsCml3VHy70dDhtp6oSLVnfM3f9WsjvXdioYjcCx8KJnGYSMLmDos/PSTMeGu/lomg9GpA2aPKRxOnjrH+OHB4L+TFeGvoxNLwjH4JR8TJkiPLKzlFEIjFiZR35Q4DDck1EfxwKDPYY2dDCqma5DGlOOPCVJ6w1YL2HHR/R7ChJbzu7JkGa0gjUFR/W2/ZYM6ZcuX95ceuWVzW2cCjkllPEEIxwy/M+eZS0G04+kOcmPvHaeFMfZYYTi2o2bXjx961ZGK3qeP8YdhjVrGMDpg/nOyKmA9p7k8JkVkh8zmRO3gUdn1zSFiQXkXoNShiP1jLHfD2kWcp/Q9Cc+ma52wo4GBtylK5c2n/jkxzcvy2FiqpbP6FCZONPipuS9LWeI+j3F9DDJg+FPfXkRuJwipqRxiCBOBx+mkWHwU+d8qB/4npSTxonhR2o0xc4D5RE9RjS8EF11hAOUEQvqOm01zTV1T5lp5dSjp/SIV9o1aocnDkOMKfRqWtEEWLSuwNHmBOu2Rn5YBUb4QFK/+dTvBnnIypdxl9EgDHfyLLWv0N4OzVtX8c+ogmAAS/F84bZxm0Jlu9zQbNmIN4zTxTN6yDVbzYoXjoWdizBjkwc7F/QJOFDVN7jNQGgK5j0zmfLERXfRYydb9qnsi3ODHKV3GbKMori+SFNbom5Ji7M202zmXR+5ws9ltS4Ds0zfkmRDaZHMt3zNsllf6DBtJpBUB4BltNMOyGi8ujcpI0QLlgAYw3EnCOmWuloC/V7fJZI+Q2nKcIrAkWYLTdmmspsIwqAXHKQdSaZUdBt0S5zc8N1iJ6UpCJfohNM5Xa65vgMqYP2D1+Iu9VL6MFHRVWjpcm3uQjajQdGwkxiFCx8DDhmBqRSJsYsBmnIbwFSmrwZvknXf4G/2iEXreJLQ0U7v9s9vh6lQPNc4lXcNqwZWDawaeJIGXqdjAXo/TNIT0qFj9NEBtQHGlCKMPvLcMXU+xpk/gvVb2tCgM3Vnrwg43UfLJtDbRr1xlEFBx43haSfABqIMRYyWoo1kfkjiDIi/37iTp3Q6Q4xURiyY+uC3xXpI8DYTfIxnphLZWZEBahlsBImWePjtPMapLAg6WBYfYyxDD7qkoQOmXZ1g9EDyUc77mnbBNCichbt3tEZCRg1lZW0IhjfODHxP6+08W7EelUF8XW/vP/7SS94+9r6M7mckI/AY/my9yvoKFc+LwjlQkHIy/QpDioXivN3Pdq86hO72XS+iDl/e6veOVegxU8pYP7Gn9RX7ciq8tkLyeK2C8lkgjm4YGbnBzlSMHGCYyVBGs+RZZ65TyUG9uPwY7Yxe5IwJHMTLWpvBiMXVq9dsiDGFielsrLu4qWlfPKjAYe0JIynolgXg6A794kye02jFBe0uxagFenPbon5VH4xC4FCw/Sxb2CJLOxOOy5E4pvwTx1gXEoenHYtuoyI1gtsId6pXt1kpHSeaECM17cgjdnJgTSvZgal2RttrgwadmUbBUa5YHlyVK/p8nKz7DsDhHPT6Cu677ACDXUiFAjwGta5un7RRAyW/SSut0yNBofel5GleJlF5LMTnPnzkhCEf7Rt+qrsetbAjoXp1f8C1+4apfJBskVKYYjJdVBrdpVydDG+HuoY3euqP2mqNTlh/grOTUWkNFzpz6aC61EcxMd3BE4hmP4QPJN/0ki1eUnOP1hbwxGgXJoW+AZ4QF1glb90IQ3D8OdApOH8GShbfSQVHN0U/9Ap+B63rfFAwXt9FTrdVJ5UMLQssFtCO6Ro85zWAZekbQEeJFjzpZQSXOXctY9cJ1CdIsdP9xCxcQr8heTmwSNupra/BtdRWcm4xWWAai+wuRbeRGcrxAC3Jup+TPhsci9YvQqLj/r3kZdnd1bFYam+NrRpYNXCIBl7DsfgBTYX6qNDo+vikK6a/d4o7/hhNcRzydpc4nVPeVJaBprSMXCi98LY6fGgqnTTZRf60kQI93tBn7USMZNOHh/ksjkaklJxlZMSxYIcdGR0YxthCXIWHgewRC95i620+xl/LjPMCXztL9VzhoYF5gMnIlqoY7vghLBjn7Aev1xAdeLIj1A0Zz4wkMMJBgUxP+fBiMTrTfs5pag/Td65qGtTHP/EJL2R+pBEPjG0MbXZrunDxotcYMGLCmQ84M8gKvTboPJrCTlRyZG55rUUWUlNnwFGuLo+vKp8XcGta1qk91idgoGPcn/JCbvhz5sRtycUDhTfn1H6cCjlHktnrSgTH6IAUap0KxA4WzgMPJaZCvXL5kh0JOyAarcHYvKORipui7a1yhYuDgFxM6brJNC6NwnDPehA7FZq2hRNkHSudesZYRAYWv+M0ZMcy9CtHR2Vhy9ujOoeDaVknNT2Lso0RC+SVTmgHBKo49ZvKJp31Ij29DxgesN2OaBfwZnTMbQb8eggzQoXzipBQt1ENv7AynA0QGzdQnoMxnAA92mo+xCfHIhBKC9+QDozbPriSl3yH4l1oLqsLTXrBCCOglnXSzcQXCOSAsOXjd4VDYecijsWQs50J6cgLuV1vC10z4ytEfRsJ8p38TtF1FKZkMC7l7LLym5zXUsThgYHrDX1Y3jggLkcXPszqWzLSNlx7May6L1rASnGzqJUZuqn7hn8MrBpDeAiqy9bXWa6qj9BKeYkrJiZyLAgSB1RXs77crpspGY4DELk7C1TiwBuXhAqkzeIYAtr6g/cg6Vju2wkyf+gUo3AtwoXodlJJXFQ7y53lMbOkKas4pmxKbT0DkB4hoC6jBDfPUYASpeSPYxGac9l3HYKWyFcIEiaavnVi6zwY0GkdkDJktVBBcFU0rnW60RqLN++APPqzlrHE8GVOQ24+wLZjwflEa1g1sGpg1cCTNPAajkWtsTC2ukd6SHpDd7B5uPCQoSOKoYshJkOzjHLSnUcaRgUPawxcPQzzBpf8pQOGbpwKdWR0ZvrAClgbcaKLYYvRG9qhC+3IlA58MTRkUHiEQW/1/cZd9ESTIDFMp89jYLSh5YWWDUrkFzhx3kxj/CIHD08ObMNh4LHOmo49OQG8dWe6Ebxu6627HQu9eb8nJ4O3unlrW2WRY8EuR5zZgHHAVq0vvfzS5r7gUTSG+2kZ04xWsL4CHTyQsc20JMqP48PVdSLjE+OOdRCcm3FHaxnYFYqy+8FgivrSw2EYWMiNI6GpRSfkXOAIsN6CERRkYk6611aInx9A0gGjFHy8TqHqOPUqnbi+8NpYIJ7RG/Rw5doVOxfUCaMkGPeM/LDz1S0tCGcNCjJVtWh0RNO65FzgWOAE7Em2s5KHQwV95gY6Fm/aBUYk9QE/HDXVltsB8rGAnOldx/U5ocXeJ/VhJylo4gihz4yuQSkhLUNxyChfrcvttfPRgz8qpKeDiY5HqQQLjtssMK4PHtok6w9e3DhebVQ00JkVpmuXv+Hg2fmuQ8MsjgVwcz7waQv8bspwliwjDPiRUqxHqV0GciPDAofkLoyvkSuii5fquKdBOW5HKDSh4zavOsqaC373/FZbJyI4VBCclAlGFYal1/pKuuGMS/riMODs27nQb5N0Oz0Slnobeqy4UqJDdGuy9DeutMitODk9xWuWLfBCEu6uEUkaAb05pvKGbOiRVyCGMU/DIw+5CiJK3Lrnlt86QYmzHOgyKI0YsG5HlgFCztZXEwwp33eZjWkBAmRcJ1JfRPRVV0lhmsiySAB9A+iSK2AVKzmhE9yiqDvdK9D3E4pFx5w2iOhuq/xATbwMDIGiNUWM17CUGToGE3zLiBANY1p8KRO4QZI062k7rWn4ugU8wYEneQ1juUsGkkX2zVxjsatHikXYLX/3ezxfWPvGVFpeeq1h1cCqgVUDh2ngNRwLtptlxILQPWW6z34zRXdICp0RRiYGuKcRYUzQgVb6MNRloDoN58LwXEVBcHDAFGIalI2j6rwxI4FhOlQb1KEdGonT14Onjlr4eVubN5UetZDBwRQRuCAx9OBvR8WjCHpjjrxDPhfKxWZROAZkDNg6c0JGC2spoMUC8H2MWBnnlN0dMNN5ZCD7HAs5FigauSgnZfDOTFo7gLHLuo1r165vLl16xWsrkIPdoy7ozRC7JJ3ReRcY+jgWnPeAQ2GDHiNNgUcTRhRGFQfzsa7jrh4COBZ5KOBgZW45285tFnl0AAASN0lEQVS2k4VTdVwjAqzj4GGBE3NB284y3YgD6zD8HwreulfZfHq59ADvlCPOFvnQhzb1ZgdMBjxpN3QGxU2tGcEhOYWONLLgaVtK49TtIY+NvY11kUMFc3YHC9b3tcUuDhsOASNMYietU+ZM84qzo1GTai/A4lTgxOA04VSc1IF6LFBnKhbOiNdgUN/6QI1/vnzra/KWtoXhUwaseB/RaIadGjkxgclIVvTd08aaJrTiDJPiGlNTUGstmWmv9fuKAIJBoKS1UQxm5M01hkH0QJ7LL5QhJ/rgM2hyG5qGN8r2fdPvfK5WjSPRk2mYtEqgOo5jgW74jWHAt0OT3xmHWWZKVPUJaj/mY3L6Klqw2JXPVjsZAprzIgOI+hfP1nvauUYt/HtLPbT+Gofr+BRdpZgHpU191m8L2HJEzKygkEhEfAF16CgpW/fuK7fqoIDM0xwD3/TIRi0T3bleXI4mUYxLkk711ThkOLMgDoFXibfwDrvp+qL37ND67Pspa7TTKuIAiZ51K3EmSgsYYo6MEZnS0EuVBdBJr4NIoxkssInqW/Bk840syVUCaQFyXlLy3c+6GbbrfisNGvXZIqy0kC7oltnXSqvLm3mOxazHlBT1tPI6JfrmN8QzjL6cXQKv6rDUlzR1t39bC/QaWzWwamDVwMY7mX7gAx/YUoVe7D96xDkW7AqVTjQ9X77T+RDvjogUDE4+GFwx3toAloFBh45FWDjgMY0EOAxB8qDHhBuvr8BYwEBRIrBM/+FNNZ/Maw+95g/d4VTowUOnyacXb/vQPR5I+njERPxaXo+wILN42OEQD8srcUEBDscCx4GdmLjnbScOC0Y+6ys424FdlygDowW8cb+Dka/OmA/GMjIAgDPDguSTMnqZ/39T5zRc1Sna19RZPxAu075Y//D8cxe9YJn1Ffc0LYlzJFj34OlZkpXF5l5wrqv1IGFfFQ+vuRA/HgQesVEazo7PsvDic3SbNRxHj2t9iJwLH0yHI6O1DLzlZbSCA/gwEnGocHRwhigvZRRLG+nHlUetQh+H4Z4MTRZoMzKAjthu9r7K5BEd4x/RwmwtMsexULnHdrzUt4xrdMVWvRjivcictRNmqi/aCzf8UY/oHMeC+mqDFv3yRs2OBVdGK/ZxLE5b370tLfXogmBKUCAFrvngzupP7aSy3B5ttEsBtEHa79FxenZEjIGLcU1LDu7iVISHGVECGlddE09Oy8Jdt+POn/PC0UBFizg4qV9kea3QdBuu77f4iGbrQEUSffhE/uFMiJf5+pp2nvKjA+lSus4IEddKa52LaPNtOXKNfhz3G3x4wtpfAyQ6lxOhtsdoha+0/cnRsS4sePCbH1f/1RWiqS/qv2DVLvnTXV2doa8KyjIst6Lj9qMo1wUjdR86BhvI4VQ0SkYh8w9Lh7k+WvZkDJAAPvYtOk2TvNJ5yOYbNoLawnQOeAXvbIEMjAC4LhoX0ORPpDoB8hVvHZDUuHNaY/dmGrkHmiDIuTxKsZ6ntFlX1EeLAF7nofOkL/kkJLWlGiLD1dx3v5w6eAtPYi6SWlwR2cHtOmhAE5Vs8mA+Wx0LdNdvIem72Q6dXf34Xe3Wx66O1vtVA6sG3nkaYH3we9/73q2CD8eCcyzccahv3OketxC4GYa6jAhPEdEVQ5DO3B+AuuNVGkYq01byNppdmTJikSklNfrAw8C0syZie8QidCFL340TwYtfP0CUwK5STLPx9B/erAIEXxs6wUVmZOPKyARXb58qo9HGqjv/yIoBe1JTcWzIaovVyJUysiiaBAx5RgsYzcDxgDcjBBjMD+9rOpT+jksn7ILEgmd4M7JxvTrrhxptwDDmjIl3aW3FOe0GxTSgGzLCOb8Cp4EHkPWp7V9Z+M0uTqYnvEfif4/REhn0TCtiGhbrFkizc6N8y6WrCijnQG/3ZfBfPH9B27me9/a3N3UgHqMJ97RWw6Mr4mGnQg4XOsRcRUYcLe/ipLLCA2eK2sKgZ0E4MrK7FSMt4KMj8K/rjRdOBTjoxfWidKZFRcaHhm/nazyqcTRF3w6ndH1MZ4zQvqg3t07REEu3J+9SBU85JXEstKWunAuvvZDTw+5RaaMZcZjbttux11ekbUBTbO2MISuw7RQzckE5ySfHo2WS006O0smDXgIGdEV94UY40KwMrq5bMxXPJzy4EalDO1QmXV9NM7f+bnBfm9+c+FjatrCjGsAhK2/yU1bHMcCr7KZltiq/2k12iEJP1XZ9jX63CEeVZgB6yuaYeSolvCVAeCCI2iROhH5ncS7iXD/UJgo5aFPtBjD+0GcVIMVzKpyczu8z+k9bbx4NW6+1oRD+ifi+o1QddBKg23FDhH/lmhf5MAiTykFVC94cn6mkzha4sOLepSy6E60CdS70J56LzBEhkhcCoEOyJdL8gbK4FMPZCzSxUNG3/mnT1rNxkhMUQfo2cKEw5Qso/Io2sk86Mo2ZpvKBt+4EF9wB1RGT2Mor8rNuBrBKh+wNUq0pehQP1ydXITTNScVFJth8L6ULzmfSsdhtY0uZUxbqDeeCtRa8SGJ0kLQOXd6+X6+rBlYNvHM1wFT73R3kFsdCIxY8YB7vHB9XWHeqGYXIegRP26Hzr07XtKpb9ltfGYYYp5yRAAydLR2UHQRYFGNGEqBlerpmREGdsY1KjD5GO+JIxLjD2OCeaxl6ykfG4LbRGBqkYaBiSNvZkVy8EVeyZfJ0IKWdkEHKAmIM8rzZ5OFQsiODDCuvGRAWZYF3nAscDB12J4Jsicv0KWE5/546aw63Y+SCqUeMyrC24KKmJjFygQHMGRd3BPOqYDFUWVNwQudbsFUsniFGOA4PhhTTmHIatqYzaU4sb5jsXMjJ8AiGYJCJBSJHRAfH5eKFi15MjpPHKdvs2sToByMCLrOmIFkZ1InKwEJ66oJwV6MgTOfC4D+phdKsDWFhOsYea0KQByeE+qY+GbFopwInEg1TZxyGx6gKijutsy5wRig7jlDXWxvp6B4dAAsfDi7ECEW/1A8jFjhMjJyc0FQqzrlg2hmnsfPBwTiucy/YwaofqLwhJZ61NDgcqX/n9xtzCgwMoxVqtzgWDqrMjGakPAGLU0E9g5Om7DujuNxKHI5BACzDkMn5wEA3YaZACnn8OSxgS/qEG6DCMeyCMPMIraJa+CrCKIdh4TscCf3GVIeWhbZlHGjz+2JEspwI0aBs0S03SvcvwaBmO76KPnriY9ol7jBogBE/PnYs1BaYBkV7yEJuXjZEHkQyDeiZjnATCX19WzZ+GBS2+FOulAcAIyrSJJb7JouaHguVaL2RaWWGn28hg4AVuv77nutIm2jZrBPaFs/5xoU2drP0C5xtjIlvUVr0EtxtBqQlpDypwYXtNnbDtgDGgaUQwDH2VHbDKSNShWq+URGpJa8ujnVmMbKeBOc8XX1f+h76h3MAtlUBjU73tYg27emW7HwWQHiNekKGJWsHM7iL6NHhZ8qxGDJPUs7R1hvX2aHgvvNm+DW+amDVwDtbA/gBOBdzGI4FU6HceVSuO8K5k+6ekw5VMHRQ7VjwVptPOlt1pMrzaIRxNK1FBodhy7EosqMz5t0QIXQZEcEpyPQqOwc4GHYscEjYqSnTk7K2QCl2KnBSMCz8CA4NUXRHilFjmhiAvMCP7BjovJHGiE2Hy45OeruvNPIwaD3KIniKgn4oVz9DPLVKuDgpODcYQcjmczmUbtqSp6ctgceUIEYYMIqQA6eBw/PYMQpnhaHne1qULUY29jnPgSlSnG/BidU+PE+6xsC6j6EvWnZWNFx9lWFrOQqkebE0jgX6UJmOyfjmJGvWV1BX4F7WLk5MX2IKBQ7EqEfJjk7YdYk0Cs90E5wKnDq2x72oNRrnz56Xo7MvnpnyxII/4O2ICIf90HudB44D9Y5DgZzIh8F+1mda5LwK3pLRThh9iINSbUBp6ItpYuy+xRtq606NeV8OGVOpnsWx0DS1E3J4WN/hOlK5j2u3KEZ6Tsi5wPClkWHs+uRu8Tmi0ZDUv5ug8Lp2u41ntKYdHXShGnU7AAM5yDMN3S/YtKnQJDVOr6gLv0N+LwWkdP/+uApg4C7gSp/xB3Hhpc0jW8Kc13QLppnXtXmCS3zwVSTkRJN/8cinZFD7tKFfPMFDrxDIb05X6X8pY//GAJyEgC+3LkNot462ZAMGx6LensaZyBoL2oaNIMsIDeSFqilP38t95MJhppwgqE5FHx4OQ0YiweNi/QDeckPBiUHju0pkGBwtwoApUvzmttJ9t3wZXqwLrGiWUJ24gFMMya6PrtK6JTaHcY/Q5HVvq9siZzIRx2kta+gEL9nBJX0JXVqlQF9ZhlUkdRBIY4SIAAIIn1kGfDnDbeGmPpvfxE3tK6NNzptojnsiYmAc0X5UziI8+GyJU2m6jGD56k4tz3KDY3wKykd853I2smkXA+A7OC68z1bHouV0+57a2WFlbNj1umpg1cA7VwP0k8M+KjU80bGg96QTpG90/1KdzHjoQEwfG+cY4TISTVzPUUAzXSmPgZ5OkqkSMbjS2fY3nX8HjHsMtjIqZZywABeDU1HDYbxzCNt9GRnDuVAahk5CDCQegD3P2yMgllm0xcAjFpKbdCvGRlHm85NGvq+VHz1gXPE8iWOCU+CFzjK0MUp65OQZyeo3txIGY9iGPm/ohWfnQ/GHGmImZGvW47IJZIgrjV2UMpWKXZeOeM3CgYz3M2cO/HYfx+KkRhXQBzXEGQ+s9WDrVk890vauPs3aQ9jSB7KKzknh7WmKEMb3HU21uqZzKzj1m5ETnBjMn9ZJ73qF8b2EOF3HT2gHJ03feu78RRn1B3YaL+lwPUZLcCQYDUI/1Cx1g65xKhgt4eEEzCWtMcEQxOHkEEHrRE4Lb4nZUpbREBaaE6ectCKPzrB1rRwoDgqEFqd4s8MVjhceM04Fp3JL+XJg5FzJGWLR+ilGMlRuzr1AHjsVckSYXuZ1GNSx5E0brJaoBNrd0SOZSmWjWRA4r8iLTG4HgjFN04BKQozsxPsh3Yb5gFEEGtRRw3Qe9x1MFTilDTh0TJoD6Yrkq9KSB/wywiJWBhPwQj40aYHgm0aRqEtkCU7z95Xfm3EEKMJdlo5znzRaV+RpvViOpu/fLQKlfCQ3H8spHtYdeteHkSs7Ev79Z+TiVaZDkQcsTr7LIZpha9rQXUJkgytlMF2cG/BIHHhbUSUnw/IVMad0XUCr0qnXmQ7JTXa5dixI0B1ARJpu0+QKjK/1tQMz+Bs/oIFvx2BJW2CThqHfZZwlW+ASGzAAVaYvuncZIDfJRZ1Mt2Hmgi7IkBptSHHTqbI27VH2KlthF73lYlrcCgCYfBboaUBqIJl363akVsTlmhJ1jw6m2kbgwashkQPISOAbR7/7O7+nbj79l64Pl+8NsmsabxB9RVs1sGrgbayB3b7FjsUv/vK/3bz88iuj2Ol+8xBauuKR7YgfLOov7QDoiZEpEGSlE6V/7k4XpoHXN9m+L1jdk0RIn94PIGCXD05MkEVVtD0iYkMiiPBqfBPTV3iG/iLDct+jIMkzglEHXssJ7yV7yIXBzBt6DEuX1mVGJ5GVzhgDiLf9nmahdNIwkpjK5AAPfSiPYQUPDDJAJw6OHCs5Bx5R0NVOFgYyzpAC8DbOtQYCGh4l0IOcAG3KiQPD1B9SvQBbBj5vepEL/IbFmeq1MOAmp3QmOpEhC8FxDMjHUWG0ARmGplCB8uIUZtoZOkI21lfYOBcyMInj1Ale+vQUNcuLg1bOmOjDg4XxnoImmdENIxs4FR4pUfloh9Cz4Sk9e7QMndXia9d1Oa1ps9Eh5UcY1z1RlV1frluMYW6jprmdpe66vhvXhTLB5ct6LD0vqYfFWuPNr2DEf1RGo1neuim0XJpGkFruRtuloxKZtq8Gavxt2o3f7aWvnQ43B0ci3NBJ50XJVtHgsqOXlscoyjNcwYRnHIcxYgKM4bocwiz4ZruULSmRNd+dt1uexg1U3/3hrqOME9ofnt4uxmFUJwZb0U8FtwmJxhYZ+D9es0A/WbI5Z4tYISXfv7ti65SdeqysN3RpCXa4vyFaT0JqHk/K7/T3ffH7OrpeVw2sGlg18LbSgB2Lt1WJ1sKsGlg1sGpg1cCqgVUDqwZWDawaWDXwpmtgdSzedJWvDFcNrBpYNbBqYNXAqoFVA6sGVg28/TSwOhZvvzpdS7RqYNXAqoFVA6sGVg2sGlg1sGrgTdfA/weIxL95qPODhgAAAABJRU5ErkJggg==)" - ] - }, - { - "cell_type": "markdown", - "metadata": { - "id": "4io1vzkzF683" - }, - "source": [ - "We want to run the Google Cloud Vision API on a large set of images, and Apache Beam is the ideal tool to handle this workflow.\n", - "This example demonstates how to retrieve image labels with this API on a small set of images.\n", - "\n", - "The example follows these steps to implement this workflow:\n", - "* Read the images.\n", - "* Batch the images together to optimize the model call.\n", - "* Send the images to an external API to run inference.\n", - "* Postprocess the results of your API.\n", - "\n", - "**Caution:** Be aware of API quotas and the heavy load you might incur on your external API. Verify that your pipeline and API are configured correctly for your use case.\n", - "\n", - "To optimize the calls to the external API, limit the parallel calls to the external remote API by configuring [PipelineOptions](https://beam.apache.org/documentation/programming-guide/#configuring-pipeline-options).\n", - "In Apache Beam, different runners provide options to handle the parallelism, for example:\n", - "* With the [Direct Runner](https://beam.apache.org/documentation/runners/direct/), use the `direct_num_workers` pipeline option.\n", - "* With the [Google Cloud Dataflow Runner](https://beam.apache.org/documentation/runners/dataflow/), use the `max_num_workers` pipeline option.\n", - "\n", - "For information about other runners, see the [Beam capability matrix](https://beam.apache.org/documentation/runners/capability-matrix/)" - ] - }, - { - "cell_type": "markdown", - "metadata": { - "id": "FAawWOaiIYaS" - }, - "source": [ - "## Before you begin\n", - "\n", - "This section provides installation steps." - ] - }, - { - "cell_type": "markdown", - "metadata": { - "id": "XhpKOxINrIqz" - }, - "source": [ - "First, download and install the dependencies." - ] - }, - { - "cell_type": "code", - "execution_count": 1, - "metadata": { - "id": "bA7MLR8OptJw", - "outputId": "7b06e838-5e81-4094-c345-b129e889ad03", - "colab": { - "base_uri": "https://localhost:8080/", - "height": 1000 - } - }, - "outputs": [{ - "output_type": "stream", - "name": "stdout", - "text": [ - "...\n" - ] - } - ], - "source": [ - "!pip install --upgrade pip\n", - "!pip install protobuf==3.19.4\n", - "!pip install apache-beam[interactive,gcp]>=2.40.0\n", - "!pip install google-cloud-vision==3.1.1\n", - "!pip install requests\n", - "\n", - "# To use the newly installed version, restart the runtime.\n", - "exit()" - ] - }, - { - "cell_type": "markdown", - "metadata": { - "id": "C-RVR2eprc0r" - }, - "source": [ - "To use the Cloud Vision API, authenticate with Google Cloud." - ] - }, - { - "cell_type": "code", - "execution_count": 2, - "metadata": { - "id": "qGDJCbxgTprh", - "outputId": "3d127ab7-abb9-41cd-e3e2-a85236408e9a", - "colab": { - "base_uri": "https://localhost:8080/" - } - }, - "outputs": [{ - "output_type": "stream", - "name": "stdout", - "text": [ - "...\n" - ] - }], - "source": [ - "# Follow the steps to configure your Google Cloup setup.\n", - "!gcloud init --console-only" - ] - }, - { - "cell_type": "code", - "execution_count": 3, - "metadata": { - "id": "74acX7AlT91N", - "outputId": "db6ec03e-745e-4b22-9d8c-97b7dbd73633", - "colab": { - "base_uri": "https://localhost:8080/" - } - }, - "outputs": [{ - "output_type": "stream", - "name": "stdout", - "text": [ - "...\n" - ] - }], - "source": [ - "\n", - "!gcloud auth application-default login" - ] - }, - { - "cell_type": "markdown", - "metadata": { - "id": "mL4MaHm_XOVd" - }, - "source": [ - "## Run remote inference on Cloud Vision API\n", - "\n", - "This section demonstates the steps to run remote inference on the Cloud Vision API.\n", - "\n", - "Download and install Apache Beam and the required modules." - ] - }, - { - "cell_type": "code", - "execution_count": 4, - "metadata": { - "id": "gE0go8CpnTy3" - }, - "outputs": [{ - "output_type": "stream", - "name": "stdout", - "text": [ - "\n" - ] - }], - "source": [ - "from typing import List\n", - "import io\n", - "import os\n", - "import requests\n", - "\n", - "from google.cloud import vision\n", - "from google.cloud.vision_v1.types import Feature\n", - "import apache_beam as beam" - ] - }, - { - "cell_type": "markdown", - "metadata": { - "id": "09k08IYlLmON" - }, - "source": [ - "This example uses images from the [MSCoco dataset](https://cocodataset.org/#explore) as a list of image URLs.\n", - "This data is used as the pipeline input." - ] - }, - { - "cell_type": "code", - "execution_count": 5, - "metadata": { - "id": "_89eN_1QeYEd" - }, - "outputs": [{ - "output_type": "stream", - "name": "stdout", - "text": [ - "\n" - ] - }], - "source": [ - "image_urls = [\n", - " \"http://farm3.staticflickr.com/2824/10213933686_6936eb402b_z.jpg\",\n", - " \"http://farm8.staticflickr.com/7026/6388965173_92664a0d78_z.jpg\",\n", - " \"http://farm8.staticflickr.com/7003/6528937031_10e1ce0960_z.jpg\",\n", - " \"http://farm6.staticflickr.com/5207/5304302785_7b5f763190_z.jpg\",\n", - " \"http://farm6.staticflickr.com/5207/5304302785_7b5f763190_z.jpg\",\n", - " \"http://farm8.staticflickr.com/7026/6388965173_92664a0d78_z.jpg\",\n", - " \"http://farm8.staticflickr.com/7026/6388965173_92664a0d78_z.jpg\",\n", - "]\n", - "\n", - "def read_image(image_url):\n", - " \"\"\"Read image from url and return image_url, image bytes\"\"\"\n", - " response = requests.get(image_url)\n", - " image_bytes = io.BytesIO(response.content).read()\n", - " return image_url, image_bytes" - ] - }, - { - "cell_type": "markdown", - "metadata": { - "id": "HLy7VKJhLrmT" - }, - "source": [ - "### Create a custom DoFn\n", - "\n", - "In order to implement remote inference, create a DoFn class. This class sends a batch of images to the Cloud vision API.\n", - "\n", - "The custom DoFn makes it possible to initialize the API. In case of a custom model, a model can also be loaded in the `setup` function.\n", - "\n", - "The `process` function is the most interesting part. In this function, we implement the model call and return its results.\n", - "\n", - "When running remote inference, prepare to encounter, identify, and handle failure as gracefully as possible. We recommend using the following techniques:\n", - "\n", - "* **Exponential backoff:** Retry failed remote calls with exponentially growing pauses between retries. Using exponential backoff ensures that failures don't lead to an overwhelming number of retries in quick succession.\n", - "\n", - "* **Dead-letter queues:** Route failed inferences to a separate `PCollection` without failing the whole transform. You can continue execution without failing the job (batch jobs' default behavior) or retrying indefinitely (streaming jobs' default behavior).\n", - "You can then run custom pipeline logic on the dead-letter queue (unprocessed messages queue) to log the failure, alert, and push the failed message to temporary storage so that it can eventually be reprocessed." - ] - }, - { - "cell_type": "code", - "execution_count": 6, - "metadata": { - "id": "LnaisJ_JiY_Q" - }, - "outputs": [{ - "output_type": "stream", - "name": "stdout", - "text": [ - "\n" - ] - }], - "source": [ - "class RemoteBatchInference(beam.DoFn):\n", - " \"\"\"DoFn that accepts a batch of images as bytearray\n", - " and sends that batch to the Cloud vision API for remote inference.\"\"\"\n", - " def setup(self):\n", - " \"\"\"Init the Google Vision API client.\"\"\"\n", - " self._client = vision.ImageAnnotatorClient()\n", - "\n", - " def process(self, images_batch):\n", - " feature = Feature()\n", - " feature.type_ = Feature.Type.LABEL_DETECTION\n", - "\n", - " # The list of image_urls\n", - " image_urls = [image_url for (image_url, image_bytes) in images_batch]\n", - "\n", - " # Create a batch request for all images in the batch.\n", - " images = [vision.Image(content=image_bytes) for (image_url, image_bytes) in images_batch]\n", - " image_requests = [vision.AnnotateImageRequest(image=image, features=[feature]) for image in images]\n", - " batch_image_request = vision.BatchAnnotateImagesRequest(requests=image_requests)\n", - "\n", - " # Send the batch request to the remote endpoint.\n", - " responses = self._client.batch_annotate_images(request=batch_image_request).responses\n", - "\n", - " return list(zip(image_urls, responses))\n" - ] - }, - { - "cell_type": "markdown", - "metadata": { - "id": "lHJuyHhvL0-a" - }, - "source": [ - "### Manage batching\n", - "\n", - "Before we can chain together the pipeline steps, we need to understand batching.\n", - "When running inference with your model, either in Apache Beam or in an external API, you can batch your input to increase the efficiency of the model execution.\n", - "When using a custom DoFn, as in this example, you need to manage the batching.\n", - "\n", - "To manage the batching in this pipeline, include a `BatchElements` transform to group elements together and form a batch of the desired size.\n", - "\n", - "* If you have a streaming pipeline, consider using [GroupIntoBatches](https://beam.apache.org/documentation/transforms/python/aggregation/groupintobatches/),\n", - "because `BatchElements` doesn't batch items across bundles. `GroupIntoBatches` requires choosing a key within which items are batched.\n", - "\n", - "* When batching, make sure that the input batch matches the maximum payload of the external API.\n", - "\n", - "* If you are designing your own API endpoint, make sure that it can handle batches.\n", - "\n", - "" - ] - }, - { - "cell_type": "markdown", - "metadata": { - "id": "4sXHwZk9Url2" - }, - "source": [ - "### Create the pipeline\n", - "\n", - "This section demonstrates how to chain the steps together to do the following:\n", - "\n", - "* Read data.\n", - "\n", - "* Transform the data to fit the model input.\n", - "\n", - "* Run remote inference.\n", - "\n", - "* Process and display the results." - ] - }, - { - "cell_type": "code", - "execution_count": null, - "metadata": { - "colab": { - "base_uri": "https://localhost:8080/" - }, - "id": "LLg0OTvNkqo4", - "outputId": "7250b11d-a805-436a-990b-0a864404a536" - }, - "outputs": [{ - "name": "stdout", - "output_type": "stream", - "text": [ - "('http://farm3.staticflickr.com/2824/10213933686_6936eb402b_z.jpg', label_annotations {\n", - " mid: \"/m/083wq\"\n", - " description: \"Wheel\"\n", - " score: 0.9790800213813782\n", - " topicality: 0.9790800213813782\n", - "}\n", - "label_annotations {\n", - " mid: \"/m/0h9mv\"\n", - " description: \"Tire\"\n", - " score: 0.9781236052513123\n", - " topicality: 0.9781236052513123\n", - "}\n", - "label_annotations {\n", - " mid: \"/m/043g5f\"\n", - " description: \"Fuel tank\"\n", - " score: 0.9584090113639832\n", - " topicality: 0.9584090113639832\n", - "}\n", - "label_annotations {\n", - " mid: \"/m/05s2s\"\n", - " description: \"Plant\"\n", - " score: 0.956047534942627\n", - " topicality: 0.956047534942627\n", - "}\n", - "label_annotations {\n", - " mid: \"/m/0h8lk_j\"\n", - " description: \"Automotive fuel system\"\n", - " score: 0.9403533339500427\n", - " topicality: 0.9403533339500427\n", - "}\n", - "label_annotations {\n", - " mid: \"/m/07yv9\"\n", - " description: \"Vehicle\"\n", - " score: 0.9362041354179382\n", - " topicality: 0.9362041354179382\n", - "}\n", - "label_annotations {\n", - " mid: \"/m/02qwkrn\"\n", - " description: \"Vehicle brake\"\n", - " score: 0.9050074815750122\n", - " topicality: 0.9050074815750122\n", - "}\n", - "label_annotations {\n", - " mid: \"/m/0h8pb3l\"\n", - " description: \"Automotive tire\"\n", - " score: 0.8968825936317444\n", - " topicality: 0.8968825936317444\n", - "}\n", - "label_annotations {\n", - " mid: \"/m/0768fx\"\n", - " description: \"Automotive lighting\"\n", - " score: 0.8944322466850281\n", - " topicality: 0.8944322466850281\n", - "}\n", - "label_annotations {\n", - " mid: \"/m/04tkfx\"\n", - " description: \"Tread\"\n", - " score: 0.878828227519989\n", - " topicality: 0.878828227519989\n", - "}\n", - ")\n", - "('http://farm8.staticflickr.com/7026/6388965173_92664a0d78_z.jpg', label_annotations {\n", - " mid: \"/m/054_l\"\n", - " description: \"Mirror\"\n", - " score: 0.9682560563087463\n", - " topicality: 0.9682560563087463\n", - "}\n", - "label_annotations {\n", - " mid: \"/m/02jz0l\"\n", - " description: \"Tap\"\n", - " score: 0.9611372947692871\n", - " topicality: 0.9611372947692871\n", - "}\n", - "label_annotations {\n", - " mid: \"/m/0130jx\"\n", - " description: \"Sink\"\n", - " score: 0.9328749775886536\n", - " topicality: 0.9328749775886536\n", - "}\n", - "label_annotations {\n", - " mid: \"/m/0h8lr5r\"\n", - " description: \"Bathroom sink\"\n", - " score: 0.9324912428855896\n", - " topicality: 0.9324912428855896\n", - "}\n", - "label_annotations {\n", - " mid: \"/m/02pkr5\"\n", - " description: \"Plumbing fixture\"\n", - " score: 0.9191171526908875\n", - " topicality: 0.9191171526908875\n", - "}\n", - "label_annotations {\n", - " mid: \"/m/02dgv\"\n", - " description: \"Door\"\n", - " score: 0.8910166621208191\n", - " topicality: 0.8910166621208191\n", - "}\n", - "label_annotations {\n", - " mid: \"/m/09ggk\"\n", - " description: \"Purple\"\n", - " score: 0.8799519538879395\n", - " topicality: 0.8799519538879395\n", - "}\n", - "label_annotations {\n", - " mid: \"/m/01j2bj\"\n", - " description: \"Bathroom\"\n", - " score: 0.8725592494010925\n", - " topicality: 0.8725592494010925\n", - "}\n", - "label_annotations {\n", - " mid: \"/m/04wnmd\"\n", - " description: \"Fixture\"\n", - " score: 0.8603869080543518\n", - " topicality: 0.8603869080543518\n", - "}\n", - "label_annotations {\n", - " mid: \"/m/04y4h8h\"\n", - " description: \"Bathroom cabinet\"\n", - " score: 0.80011385679245\n", - " topicality: 0.80011385679245\n", - "}\n", - ")\n", - "('http://farm8.staticflickr.com/7003/6528937031_10e1ce0960_z.jpg', error {\n", - " code: 3\n", - " message: \"Bad image data.\"\n", - "}\n", - ")\n", - "('http://farm6.staticflickr.com/5207/5304302785_7b5f763190_z.jpg', error {\n", - " code: 3\n", - " message: \"Bad image data.\"\n", - "}\n", - ")\n", - "('http://farm6.staticflickr.com/5207/5304302785_7b5f763190_z.jpg', error {\n", - " code: 3\n", - " message: \"Bad image data.\"\n", - "}\n", - ")\n", - "('http://farm8.staticflickr.com/7026/6388965173_92664a0d78_z.jpg', label_annotations {\n", - " mid: \"/m/054_l\"\n", - " description: \"Mirror\"\n", - " score: 0.9682560563087463\n", - " topicality: 0.9682560563087463\n", - "}\n", - "label_annotations {\n", - " mid: \"/m/02jz0l\"\n", - " description: \"Tap\"\n", - " score: 0.9611372947692871\n", - " topicality: 0.9611372947692871\n", - "}\n", - "label_annotations {\n", - " mid: \"/m/0130jx\"\n", - " description: \"Sink\"\n", - " score: 0.9328749775886536\n", - " topicality: 0.9328749775886536\n", - "}\n", - "label_annotations {\n", - " mid: \"/m/0h8lr5r\"\n", - " description: \"Bathroom sink\"\n", - " score: 0.9324912428855896\n", - " topicality: 0.9324912428855896\n", - "}\n", - "label_annotations {\n", - " mid: \"/m/02pkr5\"\n", - " description: \"Plumbing fixture\"\n", - " score: 0.9191171526908875\n", - " topicality: 0.9191171526908875\n", - "}\n", - "label_annotations {\n", - " mid: \"/m/02dgv\"\n", - " description: \"Door\"\n", - " score: 0.8910166621208191\n", - " topicality: 0.8910166621208191\n", - "}\n", - "label_annotations {\n", - " mid: \"/m/09ggk\"\n", - " description: \"Purple\"\n", - " score: 0.8799519538879395\n", - " topicality: 0.8799519538879395\n", - "}\n", - "label_annotations {\n", - " mid: \"/m/01j2bj\"\n", - " description: \"Bathroom\"\n", - " score: 0.8725592494010925\n", - " topicality: 0.8725592494010925\n", - "}\n", - "label_annotations {\n", - " mid: \"/m/04wnmd\"\n", - " description: \"Fixture\"\n", - " score: 0.8603869080543518\n", - " topicality: 0.8603869080543518\n", - "}\n", - "label_annotations {\n", - " mid: \"/m/04y4h8h\"\n", - " description: \"Bathroom cabinet\"\n", - " score: 0.80011385679245\n", - " topicality: 0.80011385679245\n", - "}\n", - ")\n", - "('http://farm8.staticflickr.com/7026/6388965173_92664a0d78_z.jpg', label_annotations {\n", - " mid: \"/m/054_l\"\n", - " description: \"Mirror\"\n", - " score: 0.9682560563087463\n", - " topicality: 0.9682560563087463\n", - "}\n", - "label_annotations {\n", - " mid: \"/m/02jz0l\"\n", - " description: \"Tap\"\n", - " score: 0.9611372947692871\n", - " topicality: 0.9611372947692871\n", - "}\n", - "label_annotations {\n", - " mid: \"/m/0130jx\"\n", - " description: \"Sink\"\n", - " score: 0.9328749775886536\n", - " topicality: 0.9328749775886536\n", - "}\n", - "label_annotations {\n", - " mid: \"/m/0h8lr5r\"\n", - " description: \"Bathroom sink\"\n", - " score: 0.9324912428855896\n", - " topicality: 0.9324912428855896\n", - "}\n", - "label_annotations {\n", - " mid: \"/m/02pkr5\"\n", - " description: \"Plumbing fixture\"\n", - " score: 0.9191171526908875\n", - " topicality: 0.9191171526908875\n", - "}\n", - "label_annotations {\n", - " mid: \"/m/02dgv\"\n", - " description: \"Door\"\n", - " score: 0.8910166621208191\n", - " topicality: 0.8910166621208191\n", - "}\n", - "label_annotations {\n", - " mid: \"/m/09ggk\"\n", - " description: \"Purple\"\n", - " score: 0.8799519538879395\n", - " topicality: 0.8799519538879395\n", - "}\n", - "label_annotations {\n", - " mid: \"/m/01j2bj\"\n", - " description: \"Bathroom\"\n", - " score: 0.8725592494010925\n", - " topicality: 0.8725592494010925\n", - "}\n", - "label_annotations {\n", - " mid: \"/m/04wnmd\"\n", - " description: \"Fixture\"\n", - " score: 0.8603869080543518\n", - " topicality: 0.8603869080543518\n", - "}\n", - "label_annotations {\n", - " mid: \"/m/04y4h8h\"\n", - " description: \"Bathroom cabinet\"\n", - " score: 0.80011385679245\n", - " topicality: 0.80011385679245\n", - "}\n", - ")\n" - ] - }], - "source": [ - "with beam.Pipeline() as pipeline:\n", - " _ = (pipeline | \"Create inputs\" >> beam.Create(image_urls)\n", - " | \"Read images\" >> beam.Map(read_image)\n", - " | \"Batch images\" >> beam.BatchElements(min_batch_size=2, max_batch_size=4)\n", - " | \"Inference\" >> beam.ParDo(RemoteBatchInference())\n", - " | \"Print image_url and annotation\" >> beam.Map(print)\n", - " )" - ] - }, - { - "cell_type": "markdown", - "metadata": { - "id": "7gwn5bF1XaDm" - }, - "source": [ - "## Monitor the pipeline\n", - "\n", - "Because monitoring can provide insight into the status and health of the application, consider monitoring and measuring pipeline performance.\n", - "For information about the available tracking metrics, see [RunInference Metrics](https://beam.apache.org/documentation/ml/runinference-metrics/)." - ] - }, - { - "cell_type": "markdown", - "metadata": { - "id": "TcBFS0rluusJ" - }, - "source": [] - } - ], - "metadata": { - "colab": { - "provenance": [] - }, - "kernelspec": { - "display_name": "Python 3", - "language": "python", - "name": "python3" - }, - "language_info": { - "name": "python", - "version": "3.10.7 (main, Dec 7 2022, 13:34:16) [Clang 14.0.0 (clang-1400.0.29.102)]" - }, - "vscode": { - "interpreter": { - "hash": "40c55305dca37c951f6b497e2e996ca59c449c4502b9f8a4515c118ec923845d" - } - } - }, - "nbformat": 4, - "nbformat_minor": 0 + "cells": [ + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "cellView": "form", + "id": "paYiulysGrwR" + }, + "outputs": [], + "source": [ + "# @title ###### Licensed to the Apache Software Foundation (ASF), Version 2.0 (the \"License\")\n", + "\n", + "# Licensed to the Apache Software Foundation (ASF) under one\n", + "# or more contributor license agreements. See the NOTICE file\n", + "# distributed with this work for additional information\n", + "# regarding copyright ownership. The ASF licenses this file\n", + "# to you under the Apache License, Version 2.0 (the\n", + "# \"License\"); you may not use this file except in compliance\n", + "# with the License. You may obtain a copy of the License at\n", + "#\n", + "# http://www.apache.org/licenses/LICENSE-2.0\n", + "#\n", + "# Unless required by applicable law or agreed to in writing,\n", + "# software distributed under the License is distributed on an\n", + "# \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY\n", + "# KIND, either express or implied. See the License for the\n", + "# specific language governing permissions and limitations\n", + "# under the License" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "0UGzzndTBPWQ" + }, + "source": [ + "# Remote inference in Apache Beam\n", + "\n", + "<table align=\"left\">\n", + " <td>\n", + " <a target=\"_blank\" href=\"https://colab.research.google.com/github/apache/beam/blob/master/examples/notebooks/beam-ml/custom_remote_inference.ipynb\"><img src=\"https://raw.githubusercontent.com/google/or-tools/main/tools/colab_32px.png\" />Run in Google Colab</a>\n", + " </td>\n", + " <td>\n", + " <a target=\"_blank\" href=\"https://github.com/apache/beam/blob/master/examples/notebooks/beam-ml/custom_remote_inference.ipynb\"><img src=\"https://raw.githubusercontent.com/google/or-tools/main/tools/github_32px.png\" />View source on GitHub</a>\n", + " </td>\n", + "</table>\n" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "GNbarEZsalS2" + }, + "source": [ + "This example demonstrates how to implement a custom inference call in Apache Beam using the Google Cloud Vision API.\n", + "\n", + "The prefered way to run inference in Apache Beam is by using the [RunInference API](https://beam.apache.org/documentation/sdks/python-machine-learning/).\n", + "The RunInference API enables you to run models as part of your pipeline in a way that is optimized for machine learning inference.\n", + "To reduce the number of steps that you need to take, RunInference supports features like batching. For more infomation about the RunInference API, review the [RunInference API](https://beam.apache.org/releases/pydoc/current/apache_beam.ml.inference.html#apache_beam.ml.inference.RunInference),\n", + "which demonstrates how to implement model inference in PyTorch, scikit-learn, and TensorFlow.\n", + "\n", + "There is [VertexAIModelHandlerJson](https://github.com/apache/beam/blob/master/sdks/python/apache_beam/ml/inference/vertex_ai_inference.py) which is used to make remote inference calls to VertexAI. In this notebook, we will make custom `ModelHandler` to do remote inference calls using CloudVision API.\n", + "\n", + "**Note:** all images are licensed CC-BY, creators are listed in the [LICENSE.txt](https://storage.googleapis.com/apache-beam-samples/image_captioning/LICENSE.txt) file." + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "GNbarEZsalS1" + }, + "source": [ + "## Run the Cloud Vision API\n", + "\n", + "You can use the Cloud Vision API to retrieve labels that describe an image.\n", + "For example, the following image shows a cat with possible labels." + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "q-jVQn3maZ81" + }, + "source": [ + "![cat-with-labels.png](data:image/png;base64,iVBORw0KGgoAAAANSUhEUgAAAxYAAAGaCAYAAACSU9UtAAAKrGlDQ1BJQ0MgUHJvZmlsZQAASImVlwdUU9kWhs+96SGhJYQiJdRQpLcAUkJoAQSkg42QBAglxEBQsSPiCI4FFRFUFB2qgmMBZCyIKBYGAXsdkEFEGQcLoqLyLrAIM/PWe2+9nbXX+bKzz3/2OeuerH0BICtyxeJUWBGANFGmJNTXkx4dE0vHDQIIEIAC8mFweRliVkhIIEBsZvy7fbyHZCN223xS699//6+mxBdk8ACAQhCO52fw0hA+jfgrnliSCQDqEBLXX54pnuQ2hKkSpECEH0xy4jQPT3L8FKPBVE54KBthKgB4EpcrSQSAREfi9CxeIqJD8kDYSsQXihAWI+yWlpbOR/gEwsZIDhIjTeoz4/+ik/g3zXiZJpebKOPpvUwZ3kuYIU7lrvw/j+N/W1qqdGYNI8RJSRK/UGRURs7sQUp6gIxF8UHBMyzkT+VPcZLUL2KGeRns2Bnmc70CZHNTgwJnOEHow5HpZHLCZ1iQ4R02w5L0UNlaCRI2a4a5ktl1pSkRsniSgCPTz04Kj5rhLGFk0AxnpIQFzOawZXGJNFRWv0Dk6zm7ro9s72kZf9mvkCObm5kU7ifbO3e2foGINauZES2rjS/w8p7NiZDlizM9ZWuJU0Nk+YJUX1k8IytMNjcTeSBn54bIzjCZ6x8yw4AN0kEq4hJAB4HINy8AMgUrMic3wk4Xr5QIE5My6SzkhgnoHBHPYi7dxsrGFoDJ+zr9OLynTd1DiHZjNpbzAQBX/sTExLnZWKABAKc3AUB8MRtjXABAXhWAawU8qSRrOjZ1lzCAiPwLUIE60Ab6wBiYAxvgAFyAB/AG/iAYhIMYsATwQBJIQypfDlaDDSAPFIAdYA8oAWXgCKgCx8FJ0AjOgUvgKrgJusBd8Bj0ggHwGoyAj2AcgiAcRIYokDqkAxlCZpANxITcIG8oEAqFYqA4KBESQVJoNbQRKoAKoRLoMFQN/QydhS5B16Fu6CHUBw1B76AvMAomwVRYCzaCLWEmzIID4HB4MZwIL4Oz4Vx4G1wMl8PH4Ab4EnwTvgv3wq/hURRAyaFoKF2UOYqJYqOCUbGoBJQEtRaVjypClaPqUM2odtRtVC9qGPUZjUVT0HS0OdoF7YeOQPPQy9Br0VvRJegqdAO6DX0b3YceQX/HkDGaGDOMM4aDicYkYpZj8jBFmArMGcwVzF3MAOYjFoulYRlYR6wfNgabjF2F3Yo9gK3HtmC7sf3YURwOp44zw7nignFcXCYuD7cPdwx3EdeDG8B9wsvhdfA2eB98LF6Ez8EX4WvwF/A9+EH8OEGRYEhwJgQT+ISVhO2Eo4Rmwi3CAGGcqERkEF2J4cRk4gZiMbGOeIX4hPheTk5OT85JboGcUG69XLHcCblrcn1yn0nKJFMSm7SIJCVtI1WSWkgPSe/JZLIR2YMcS84kbyNXky+Tn5E/yVPkLeQ58nz5dfKl8g3yPfJvFAgKhgoshSUK2QpFCqcUbikMKxIUjRTZilzFtYqlimcV7yuOKlGUrJWCldKUtirVKF1XeqmMUzZS9lbmK+cqH1G+rNxPQVH0KWwKj7KRcpRyhTJAxVIZVA41mVpAPU7tpI6oKKvYqUSqrFApVTmv0ktD0YxoHFoqbTvtJO0e7YuqlipLVaC6RbVOtUd1TG2OmoeaQC1frV7trtoXdbq6t3qK+k71RvWnGmgNU40FGss1Dmpc0RieQ53jMoc3J3/OyTmPNGFNU81QzVWaRzQ7NEe1tLV8tcRa+7Quaw1r07Q9tJO1d2tf0B7Soei46Qh1dutc1HlFV6Gz6Kn0YnobfURXU9dPV6p7WLdTd1yPoRehl6NXr/dUn6jP1E/Q363fqj9ioGMw32C1Qa3BI0OCIdMwyXCvYbvhmBHDKMpos1Gj0UuGGoPDyGbUMp4Yk43djZcZlxvfMcGaME1STA6YdJnCpvamSaalprfMYDMHM6HZAbPuuZi5TnNFc8vn3jcnmbPMs8xrzfssaBaBFjkWjRZvLA0sYy13WrZbfreyt0q1Omr12FrZ2t86x7rZ+p2NqQ3PptTmji3Z1sd2nW2T7Vs7MzuB3UG7B/YU+/n2m+1b7b85ODpIHOochhwNHOMc9zveZ1KZIcytzGtOGCdPp3VO55w+Ozs4ZzqfdP7TxdwlxaXG5eU8xjzBvKPz+l31XLmuh1173ehucW6H3Hrddd257uXuzz30PfgeFR6DLBNWMusY642nlafE84znGNuZvYbd4oXy8vXK9+r0VvaO8C7xfuaj55PoU+sz4mvvu8q3xQ/jF+C30+8+R4vD41RzRvwd/df4twWQAsICSgKeB5oGSgKb58Pz/efvmv8kyDBIFNQYDII5wbuCn4YwQpaF/LIAuyBkQemCF6HWoatD28MoYUvDasI+hnuGbw9/HGEcIY1ojVSIXBRZHTkW5RVVGNUbbRm9JvpmjEaMMKYpFhcbGVsRO7rQe+GehQOL7BflLbq3mLF4xeLrSzSWpC45v1RhKXfpqThMXFRcTdxXbjC3nDsaz4nfHz/CY/P28l7zPfi7+UMCV0GhYDDBNaEw4WWia+KuxKEk96SipGEhW1gifJvsl1yWPJYSnFKZMpEalVqfhk+LSzsrUhaliNrStdNXpHeLzcR54t5lzsv2LBuRBEgqMqCMxRlNmVSkMeqQGks3Sfuy3LJKsz4tj1x+aoXSCtGKjpWmK7esHMz2yf5pFXoVb1Xrat3VG1b3rWGtObwWWhu/tnWd/rrcdQPrfddXbSBuSNnwa45VTmHOh41RG5tztXLX5/Zv8t1UmyefJ8m7v9llc9kP6B+EP3Rusd2yb8v3fH7+jQKrgqKCr1t5W2/8aP1j8Y8T2xK2dW532H5wB3aHaMe9ne47qwqVCrML+3fN39Wwm747f/eHPUv3XC+yKyrbS9wr3dtbHFjctM9g3459X0uSSu6WepbW79fcv2X/2AH+gZ6DHgfryrTKCsq+HBIeenDY93BDuVF50RHskawjL45GHm3/iflTdYVGRUHFt0pRZW9VaFVbtWN1dY1mzfZauFZaO3Rs0bGu417Hm+rM6w7X0+oLToAT0hOvfo77+d7JgJOtp5in6k4bnt5/hnImvwFqWNkw0pjU2NsU09R91v9sa7NL85lfLH6pPKd7rvS8yvntF4gXci9MXMy+ONoibhm+lHipv3Vp6+PL0ZfvtC1o67wScOXaVZ+rl9tZ7RevuV47d935+tkbzBuNNx1uNnTYd5z51f7XM50OnQ23HG81dTl1NXfP677Q495z6bbX7at3OHdu3g26230v4t6D+4vu9z7gP3j5MPXh20dZj8Yfr3+CeZL/VPFp0TPNZ+W/mfxW3+vQe77Pq6/jedjzx/28/te/Z/z+dSD3BflF0aDOYPVLm5fnhnyGul4tfDXwWvx6fDjvD6U/9r8xfnP6T48/O0aiRwbeSt5OvNv6Xv195Qe7D62jIaPPPqZ9HB/L/6T+qeoz83P7l6gvg+PLv+K+Fn8z+db8PeD7k4m0iQkxV8KdagVQiMMJCQC8qwSAHAMApQvpHxZO99NTBk2/A0wR+E883XNPmQMAdcgw2RaxWwA4gbjRekTbA4DJlijcA8C2tjKf6X2n+vRJwyJvLIe8JunhrsXrwT9suof/S93/HMGkqh345/gvQNIG0qk2u10AAACKZVhJZk1NACoAAAAIAAQBGgAFAAAAAQAAAD4BGwAFAAAAAQAAAEYBKAADAAAAAQACAACHaQAEAAAAAQAAAE4AAAAAAAAAkAAAAAEAAACQAAAAAQADkoYABwAAABIAAAB4oAIABAAAAAEAAAMWoAMABAAAAAEAAAGaAAAAAEFTQ0lJAAAAU2NyZWVuc2hvdFxP3LAAAAAJcEhZcwAAFiUAABYlAUlSJPAAAAHWaVRYdFhNTDpjb20uYWRvYmUueG1wAAAAAAA8eDp4bXBtZXRhIHhtbG5zOng9ImFkb2JlOm5zOm1ldGEvIiB4OnhtcHRrPSJYTVAgQ29yZSA2LjAuMCI+CiAgIDxyZGY6UkRGIHhtbG5zOnJkZj0iaHR0cDovL3d3dy53My5vcmcvMTk5OS8wMi8yMi1yZGYtc3ludGF4LW5zIyI+CiAgICAgIDxyZGY6RGVzY3JpcHRpb24gcmRmOmFib3V0PSIiCiAgICAgICAgICAgIHhtbG5zOmV4aWY9Imh0dHA6Ly9ucy5hZG9iZS5jb20vZXhpZi8xLjAvIj4KICAgICAgICAgPGV4aWY6UGl4ZWxZRGltZW5zaW9uPjQxMDwvZXhpZjpQaXhlbFlEaW1lbnNpb24+CiAgICAgICAgIDxleGlmOlBpeGVsWERpbWVuc2lvbj43OTA8L2V4aWY6UGl4ZWxYRGltZW5zaW9uPgogICAgICAgICA8ZXhpZjpVc2VyQ29tbWVudD5TY3JlZW5zaG90PC9leGlmOlVzZXJDb21tZW50PgogICAgICA8L3JkZjpEZXNjcmlwdGlvbj4KICAgPC9yZGY6UkRGPgo8L3g6eG1wbWV0YT4KC94jVAAAABxpRE9UAAAAAgAAAAAAAADNAAAAKAAAAM0AAADNAALkCKE5GNoAAEAASURBVHgB7L15rGfJdd9Xvb1+ve+zcDZxNKQkUpJJyhQpkpJFiRxLNAOOQjmBYceAEwSIk8AL4MBA/kmC/JMEhgM4MBIhsQDZghE5cqzFlixRCymS4iJKXGfjzHBmODOctad7en3v9Vvy/XxPnbp17/u9190jDsMmf/e9e6vq1Nnq1P1V1blVde+ODR2XLi+VtfX1sqNc5diEsAmwBYONzfAKGnI2yoYTQwgRMEEcIVvqNl470Nj/itUwCzHkJXrgklIsDkUGmRYk+MA/0QiDJnmEPKDIzVzHAkA0jpoO+gROw0nuJDnFfjXpnuXsElauZPbIE2HTrBGvTEyRJjyc3AY3yRPFVSLgJvgsvsCSMAlm4V0LTqWbxSbJk3XiTOGZf9VwFmEyhdj5ukzvrxHjZNITjhCuPXE9rLbBnWqSqFdTJOmMn5dZZW8MMzKilJhMdxKnoK5N6bC2iYpB8kixm7Az46qIouz4wcekXK6FFoL5caNZYN/i4rdM5XX69lm/HWmQ8OxXwV1dXS1ra2tl//793zId54LmFphb4MazAO3FyspKWZy0ZztwLL76+JPl6Wef95glG5os4saGh+hugEZtkxLu9npg9oONmD7SboEiOAzZ2ZIkneFGWd9YLyhJg7ZR427k1tfk9OgUfFUnDtC6CBG1c8fOsnvnrrJ7V5y7du4UbEfZqXDXriFOmXYIN0I1psq32mKyLp7r6zSkq2VD5/rauuXTuSMjaIJn8tkpecjYuWu3Q3iTDr6iMn/so78ujlng4RDu+qckFWS484wC9eajWhP1dATOLLyBsvKZIPV1MeCiz1BHCU8pVa0Ej8Koz6rV7AIFfr0HwE9b9IwmajZtrEOXmfonPCR395gAs/inLOOjS+VZrZTZLRzBO/kgDDo0dEdClzEs+UTeZgxDqm2SsjcjWeAAS17gjesr+aLoRNlmSag25wa0u/asTFD5TXQ0xRS3shk0GGJkjXUO5CwTeYE90PjeQn8VfoAG3aAOdQ9K4oRSyS2w40obwdH0EKH5JPNJeQwWrIGjEoJZAiPVrsE7GQKuiJvwpa/QqkqmRxfQkjrvM2de1wUt0h494aBExGbhDPi9/K1/U9WGSUaZanyQlpnTjDEGKc4dY/BAnIyBCKe33YDUx3qCHh7xvnxxD01x+toceE3l0i/1R2AO+L2ct/3gm3vU1zRO35n1lmEvEBi60d9yLi8v+zxx4kSPNo/PLTC3wNwCIwvwEIL24sCBAyO4HYvPffmB8vBjT0QDrUZmaArBjbQbpC7DUNJ969rlQxntbDS2xN08K+JGDMA6DkWkVzWopwHkXLcjISdDIfAra1fKigpw5coVpYGvFw3jC47EHjkWCwt7yh4N8nEwdu0ELkdjN84GA36cgHQMYvBP2oVUsC7ea6tXyvrqSlkT/3A01q08ZaTc6TQEH8mQrF27ddqxEH/J3WFnIxwOHAucHpwK09qmwSvsRhwLpR5OAPCRJqWMw5Hx6OSyCwuouTbUxGwAIsk0gdg949GNt5R760yBZIapf2QEbeWgwDHqtOImeRYx06AYm7DxBZZaD0PBwCVPh22YmonYfCIr8jsdGhi7qNOsyCmB7IQNqNWGlXWoFhQ9Xaem72PTCyFxQi2uFZJ6NvtXQMInujT71PJmp48cdCZtzqLvWJA9Olpe6taMWfVKeFKZgAv24mgcLNOgemmoxkl9pN3AumJWQEecvMmxSk2PxE25kR4GY8BBrvKgh6+YJM9QJ/MFN0mkAyvibsugFUGYRYjGNdDx4GlxkVezkM9BELSZ0YdJbUykRGYN4OBopgFQro4nNGi7qfzB6RquTar5oCziqvqm30r/wT5CmyBVlcUHpfMIKKjo65yWL7lTwS4btGAmx7Goodzo3MtKmbNCFOjhlX9PL2VCnYFnL6unJm4sEehxkxO+g8wvBFUJVt7lnDBIKYE9ZN774+8ZEq9xbO5YvMYGnrOfW+C71ALbOhaf+rMvlq989TGbZtqI0zC2xlSJ1pCCrQbWDacuNJw1pVg0o1z7Rpu4TzkUzEoQx7HwzER1LHIalsYQp4JzRYP/ZQ367VisxqxFOBa75FCEY7GggT7OBc6GZzDsWMRMhmcXcpCvcIccC8opyXJkcCpwLuRYaEpnXfKYtZByxgEv6XEU7LzYsdgTjgUOBQ6GQxwLORmWAbw6GG22BAuF7GZnOimfMlY9bFPFAzuAiU+3mJ1U2LzmCzuOGJDURAtMA4oiA2bNVlmtg5KBYuysxkBSRuqObSpGZRDBAE0Jje0gU4R5T2SZMkwuDsHLSKMOjK2vvVZVB2wLpz6r8UWX1HHQOfmP9Upohipt8hRpUmfZbM/MT5JEIt3yIjJOKiW9QUcH8pKvi9PRNzpwgdfDNIpn8UPggNH4gN+YEAmcoS5DB9DyiHILV//NRtUY0BlmnoM80yppvtBVZqBlvPGvEeBZboOq0mmTrIDkCaOwmgmDy5R5lZbgEf+gaFfyWvmAojcPJeqxHW3iEPa2JI2OPW3KcLmaHSG0ESF51cfIvvDrKr7XoReQ+vSwjIdKnV3SHFXVKc8Aj7QQq4C2uqrMe9o+vp0+qRdh0oRK/G6qUsrrbQtuzzPpgE+PnDUJx6JqPPyoXJR2/02JR+nUJbT70PvfN8p9LRNzx+K1tO6c99wC370W2Nax+PTnv1zuf+SxaHybjYbOw81p69SHBpwusvbTpho6iqFJd6PtNlUw9UrTc51ZC89MhCNBI3hFsxMo7FkM5a1o4L98ZcWOBfkbokEWA34G+gt79vgcHIvqXCgvHAEG+Qz65VSoHO5UpDparq2HY+EZi4ljQaHATcfC8upMBTMWI4fCsxd1aRSzJE1eN2uBbP5ShxAQ+ijOE1/rBlxHdEGE/MVh+9UEZk34gCGg7DweAFVaOlr9Q9PLcR1JJw6urfZ6AYLKEsbhYhqFPZ+mTMOKSK9b1n/SZQhm8iRudUJdki2vx+/hxDMvilIt4wJREB2j8sAzwNtdk2fgBAF0aSOXbSQqc6q8yrzxqbgGw6cqUc1fsYcAuqYmkRzpNGDiBmP4hH4KFdeUZCJA7Lg5pkBn9zgdeo1GiZJ20Bl2rW47OeBHeaHpBtGpu6EwD55hg0GHZEVu2ocbYrBhx3OwzpAPK5gEewQpHcH1XpHf5Iq4jzfdJkx7HJZ4Its2GSkURD1uzwbeyR+cHi/hPT6sG5yiZ/0qo7uDmhkarpgQH/BHXDfBMasLVNGmdCO+iUPoygxAjzOlryRDWQSY4idND4cu01HizRWe+eDCI/mIkP84unsGPplB/YmgInUWMG0QN34Vq5eXhIkzdyzSIvNwboG5BW5UC2zrWHzmi18pDzzytVa21sa2dlQR/mmMheX8htTInJepbFTdYNP4klEbYeLkc+JUeMZCjkTMUuBYaHaCpU/MYmj/A3E7FuBoxmJDS6HQhOVG7KVojoUG+56tqA6H4xrg76qD/OZY1KVQaJWOhWcs5LysScb6ai6FijKnYwGfOFkKtSccDhyMdDK87AoHZuxYeIYEfTvHJiwpQ2QHJ8Nil+x4FPWBvTkyJJ6mNz4AHR6zZccnu7oTNBGMq/0rJeAmX3HqoZebdQcsKS2Dy0gTA4aLGMO3pxkorIxxk7/zUmclrgVuBttc+nIMaCqf+etiNQZdwOnlkr62I2tBvOLfZMnLuYminNl6bSdpRGxE25a69ZHhuCwqzVDLZBlNl0Sv1J3ZK2QaBF+4USbfL03UhBmkiUPUwkAOglb2dCwaObwhnh4CTuFm1RToCBJRGirbKS6zGXd0m6NZd87pRLnsFb2VRekev4+PcNAoVdwsMk2knBDYiR3xH/Hsytaz7mmbKCEYXjPRs6dpeF2kl5XgKF9KGHNI/N4G0CVWk18BU7yk72UlLHEzTHjiWk5nD9JTnKTNcJpvRbNoPeNpPAtkIZGZPA3qflTA+7xklbLnjkVaZB7OLTC3wI1qgW0di89+8f7ywKNfax3BqJC1waVzdaNIum9gG3IdfDg7ENyw9p1J7dTc6IKntB0LLT3y7IRmI3AcmKHAqeD0UiilV5ixmDgW6MN+inAsdjvcPXEq2oxFHdQzuHdPq3JsaKCD4+KlUHYqtBSqypByLmduBodu7FiwxwIHojoWOBc7OatjIXzvvZCO6Nk2eGcaJRTnSLtiKscNHS6BFWni1aSj/nCmYwGJZAwdXFC6LoOdr+T3cq1HzQ+KpAPYa1ORMlAWvHuagWKgG/QZl/da4Clqq7Avx4CztWPRyxzwrzPmcgcNtw01VIPGaLZeLXtmBC5htRBg24YAywiiwa6kXR6DgqZWBhmBXq/11hvBxongiw7wRHZVRmhjXpGsOM7tNRclwoLdQCsWYE1ZDTp0eY22RQY0Mwg4YloxW6RDnRVNlojraRJuGpe+UWdd9vh9PEiSgRhz1CAS9ZooDRiABDeeAqTMMNlgm55t0jV2XaZto4zGsyFtjjRZm/BTQsdYOODP4ptYUPU4U9xZ8mbBoOvhqfksfr1sx0U7k75HTIYKtwAPGeCqYCm71ythGSbbHmfuWKRV5uHcAnML3KgW2N6x+FI4Fq01nVFKN4rqQLJrmYEyAnnQAEQtdGtgadwB1Uae0HsscCh0omQshZIjUR0LnAsvhVpZLqvaZ+EZC9Ht4G1V0ocBf85YLOzRPgsN7Nm0nbMVu9kPIQeAWYOcMXAh6BTsWGjGRI4LMxbrOWPBcivLiMESdINjEfy8edszFZ1z4eVQ6VgQxtKrDL2ZmzdI1T0XzWD0+hiWsD+kQztqXo9BbqZHjoXgzf6iSxwbvyaM35hPIp0eWXeNxwSVZHaYxm20ne4gTZKAtjuSZ+KkHpnO/Ck88/uw6a5Ib9LEwVbNSg2ZXOVUvUdgsCvA2VuULXVLXVNehsATJ2HTsNetx5/SbSUj+Rm/Vfqg/7heBn16fkGbnAhrgbty9/YBnPrBx7zSYEnb4YCb9h/xAd7oBp5o0A7410SnDsgNZRQxclBwDSzqOU4AI32TOUzIq8zAyTI6a4Y8eHfqtzJWFg6o36qEg+RvwuQpYG+Hhp86W06jDL5JO8nrdTbiJD9hhGmTlD0tMziZl/iEs45ZtLPwev2ulfeUj/WuQC8JpSI4OpsEYPY10cFPHZr+rq6G0fKTU+pPmLSZ14dzx6K3xjw+t8DcAjeiBa7qWDyoGYuh29xcxGwkM8wGdDMmkBgOZV7gukVunRXLmWie/Yo7NcIsQcKpCMeC2Yp4E9SyQ8W1/4HN216qVJdCocsuDdKvxbFgIzX41p/eXv84Fusb6Vgsx1uhkIFjIRkcohqcCjktu7Qp3G+DklMRYXUscCq85IpZCxyRcCwiHumUz1Ao48gInYhhueFoQ4VudOJhFLon2qzOksxKTJQycPja8TIwL5VP45twEUFnuQqz3q1z4tTQeSP+yU0ckr/CWbSwgL7PI84JPOVORM5MJk1muty2mHg5oXCwYLVOlHGQ02GoGKFXhTkNdzEL5jUYDzZTfh9OdevzZsWtj8VUWZKNFlkfm2iqPgE3sqPBZ5SZqjs/yrdZ/8EeyVE8dYw4jRLkdXyc1yNU+lqvVEMvYxO6uYXAegtZdk9jhba6jBiGbK4GR9J3gm1a066jxm8bWzeczZHh7kGSGFfemzGlS/3NGKUWcoDNIJwB6vn2tkk+s/Izr8cHbyt4zyPjiZvpKa+EbxVuh9/zTrwMk1/iEGZehipIK8u0DhIn74M0aYZZYbP4W/aAWG+m0Cj5EiZt6kqY+ffd+/4e/JrG6c9Slwx7gcDQy32x+r3562Z768zjcwvMLbCVBbZ1LP7EMxaPu4FsA5bWi7vpNd9po5SN5EyhJotONVipqyXCf23EHNZ4OBWxh4LvVfAmqBUN8peZsVDIUihOCsJbm+iLvccCx0KD/FwOFTMWvC2qzlqkA8DsQd1b4YEPqmlVFI6F3wx1JRwLv3IWGTgW0tVLoeCFU+GlT+FYeI+F0t7ELWcjNnLL2bBDgVMxOBY7ahz7+a92eGlPh7aXdAoThUkFM5jC1sP1U5NCxZiZNYSABhLLJLOBOn6NqPKZwc22to41k3pL3aE3OPUY8e4USf4T2iYfPuBgmwpEBidw5/XI28STpkcRFyXjGs7FkJvyoiDAXaIhacrA8oCxlUGwShxB6Dpw3hybpdtmrIC0MqeMagtyBytNqCtuliELAS/LnoEOPI7N+jcdKkYOmJMiy59sp3qFvSI3rQpt6mL+kZEsuAUmB/cB90eAyZ7qNSEYEB1LhhNBNUmAnjmhk/aI7LhOyzVT3gQYtqqyU/mKU0XbfCmPrCzXCEYlJsFExqzkLB5TPHBSRuInjuEWeXWhySNpp7wSvlWY+L0+PW7PH5zEbzgyL3UDXuY5VNqWz1AEo/zKwDideaPE43LDu+dv0h6lVnHqhBzOXvfIAx6xuWOR1pqHcwvMLXCjWuAqjsUD5cHHHvcgqTbHtMKtL5s2kJnOhjqNkmnnqzGOo3YGalCzwc0wXzXLq2dxLFZ5lSyhTs9Y4FhomRKvmk3HwvsslI+MXTp375ATgWNRnQteP8tSKELO+N5EDPJDrxjU0OtsyNHwjIVfORuvm11jOdQVPpYXMxaWAx85JrnHAifC37LQ0qvcuD3IATf2WnhfhXTByRg+0BedFHytj4yUYdqR0LbEEdLR50e82tZ1FD1VWtsEKlyrPeTUzsy8AsF1nZ1cT5t12NBEb7oaEp/idOyjMzfFWO+ehnhfpopuvlN4S3f3Y+JvFULTy2tKmUctbVdoR1shWsTsM9V+F4Ju0r+xHMu1HtQEbzHr7NfrnXqO8qdlTf4KleUjQDWjZ6h41v00N8tiJlmvMk7KTl0m7IJjJYZ32sIspkIgboLCVo1fVy5oO7S+iqRPoxhHeoI+pxFIuw6ngeE+yqjEHa5tVtNpD7C2tsns+7un7VXs7xnEJF90THtafOqpjDQDuInf80RWLy9x4OO3gdX8hEOb+D0M2wxim1SLqiZxfKtLUpDvsnGdElLQegyxwQ6ZNytMncnr9XadVQKslXk9PtmZJj9xKtkQVH17noBS1+TRCFr5FEmkITJDThCk/Pvuvbexeq0j8xmL19rCc/5zC3x3WmB7x0IfyHsIx8IHnZUireFUvOsUAmeL69A7BUIyEq9s1Pswp14J+ehdOBW5zyL2VizhXNTZiraBu85YsAxqt87rciw0WHc/oJAn12sshcKxkIz13MchmV4KJb3pUOxQyDngDVThXOBQ6DsWcmZwLNKpcCinglkLZiniA3kRpmPBDAg880xLZsdFmJ1PKCol9Z/VYd2F48PAzAEt4FwHaEoAGNCU1eXEEqHKL/n0+VvFrWsoZT1HTkzqOSFu5ZvAt0r2+s6i7fOnPMb4FLDaDsQaBaeDBosE2CaZiCyuY74CUKfAyeRQolFV4KhuQenquue3ZXkaQ0vohI3L1Q+OgmQW4RS2uUyzqq+XBIfp7A8GyGYABXt8pwH0h5hsut8GBoMRezoLhlsczV6C247KAKVVgNLgGlbhKbO3O1nTo89vcqZISvd4ZM/CneKYTa+oyt30tPFJ6UD/apNZfBNmanD1l+ULBle5ImaGzQ2eRTrRJeWDip5RioEQPqlPq4Mhu8VSfwDw6flmPO2QRD1NwqjoQc4Qm9KCD99ZcHQe2YR0d2R5Rk9taj7mGetvbspluVGUa+5YdMacR+cWmFvghrTANTsWNLT+G9pjFXiUUPO4xTHpcBILfDfedDo682vbdig0+5AOhr+0rVkL3gTFzIQ3bXvGYqUsa4+F3xbFXgzhoGfMWAyOxR59zyJmKmIZVMxexKyFy9UthxIDF8tvhZK8dS+FqrMWbSnUhpdbsU/CS5xYEoUjwYyFZityr4VnMASPjeIxW+FZCsvL2RI2bcuy9UR+2DqtFJ0cMA7bq8bT/L3dqSPj1doglTDiPW4gDpCUYXiHSycdXJNTYswOrSNZJtKF/0GMyzebMqCNfjsk2KcdFJ9F0+dPWQ34nWK1lMaVzuB0IsYsNo2ch+yBd4X1TMIcMypioEfv5JEhuTPLI34zDxdrGITxE/RgqyJfW00Gcq9DyuqLlLAMUcniU7dqYnTIghPNbGtWccDgaGVNJPKDgfNbxSQs8Tq00FEZ+s8y+D4Et8ozXIhJnnITP4SNr6O8jnaMFakRbkVIGYk/CycUqlqpjFF30tOFSuWjGKl78utD8E1tEn7H22F3lFWEJRDvyEi27I6kr58sY4ZZxrwHkz716dj3HB1PGoRS3TZBxUp75G2QxCkv0w4lZJAz/MZGONeQmMm70rk8FkIJh1KmftCmzkESOHPH4hoMP0eZW2BugRvCAts6Fp/TjMWDjz3hhpzGkM+gZUcRpYuOzk2oGszWjNKZZUsKzSxTdPjg5skMRToU6WD0sxZ2LNpSKDkWmlFgSdQqb3BiKZT+GKjv1smH8Zi1GByL+DCe3wxVv8AdX8OOckRBQ+GNtnlb+zfkvMRSKM1gsMdCh/dYaFbEzgXLrHAs5MAQeraibtpmz4UdDs1UxP6K6lCQLweDE5vyh3zbt4b5ETPnCzY9+rFt2n6KQzo3aTuvq5ceN+uVMOsOnrGVPurQOgo21STxe1p4J0/iqR+0gddDN+PCM+VM+UJ5rcdIh0nZhzs2B1wpsXJXsjd7llPQViDrWZF6PQfcsaa9PuOcIdXzGaAMqtKKATWvXkGBowSBB3pmA+npsy4NSyTou7gIQhC0WYMdzHpWDOQmbchJWmwVcQeh4KAYvMlwpkLJT93MuuLjEPT6t4JV+RGEHOLEUhS4SQss44EUWA0X4u5wGav+gBstiWorB4NocnyA2+OnfTKfsOcPbo/Tx/1BPfD5q3LNu1e86jDl06uWtLZQzQCWemY41c11IlmZP5PnNuW1jHoXJQ/L4NIzI61j0HOcDS15WeyeV1AOOma6DxudgEnblx/cXnbS9rCky7w+PeBRqCiYVO4ONEhAhugScj/0/vnm7c5Y8+jcAnML3IAW2N6x+MqD5eGvPeFitafqo0K6m3Mz2QagfadXcbMxJ5mNsJvUrsUFzjnbsWCPRXzTgjdAecYC58JnOBexx2LVbXY6Fgsa1DfHAgej7q9gr0UsY6oD+zq4d2+FsjrZ37Gur2/zutlVlkLhXOjEsXCpVc6csTAvzVTsljw7Fsxc4Dj4TVE4FjgT6UgoXvdX5EbuXA6F3CEeVourVdrU4WFDzJ3dU4bYuT+GbhhoDtgrZ+wuKB2iIQk2JnYIroCjthXZ5mj48NOZ6TFJFdI0D/k9zmy6HmN2HM6b7JBlGmWMElF2V0DHINUUqIsOqbSdIK4IQh3Xqjt4w0AkaLe6Tnk2Omzclbiva+Chd1zhMS61tdWllk68OGrg+EAgynovRIauE2ZNpz5jQpMkvjfMSDqB0zKCu8tR1apoY/komfmmHXj0DnctUTBtQrpk5QELRxsbbCcIQPOvNCRrmVp5JziZD0XGG25lA3wKM74u1sOiHTPFLD4TtaodN/M1bavb4DncG2I/ZWSJAQ9a6VTpSacu3ChwyzzI+vyEA8t4wyGio4cPivR3MUhDDrqmVZoegQK7VpQ+D3iT08o667cA5nAkj6RNraaUiQdl4gaXEDbki4NsFummSI9aPnTv3LEIg8yvcwvMLXCjWmBbx+JP739IjsWTbsjDsaCYNOtqFN0uKk5DCUQDceI0nBWjdUB9R2AyOidYKcyDxpYzN2632QrNQrAUCocDZfn6tt8KhYORjoVgwMmHJ0/oc48FsxW8GcoORudYMGuRMwY5a2DFpSxaaWeHnAiWV4VjsbYcMxfxdW+KGk4Jr5LFccCh8KwFzoWdCjkS5DGT4dkJ5IVM77GoMDZ/N7vZfjGDASwPYk5V2yYcPQMeOgMfLJpYFScwG0bUUuC4o6u8EZs8CKMTDB6hhyUOzCexht/pP0FREh4phWSvTWAnn0hd2zU16zgHYc3objfBx1hNg55JxifiGy5wmE7Keq26gzceiEwE1eQmftILHXIAnSVBXXHsmJAzaJtLDYGiNr/CpA0iM3YULqlbcAzOhpMH1pi44Y8yxkaHynpbfyc2awE4ZRNvcpJXKOesId8Faqi9GQYL9MwG8oQmnq1SZaFHLzapRnWHPp0tNtVXEikMfh1yl0c0c/KWajp1+vR4rZzKB8X1MuXpjMhJu2bNG5pCezrDgidMG53lBIF16/KsF7J0pH3SFkmfeUbSJeGVCoiz4lqxlKjqdOWrulUUgqQBN/CDq/PSoA0UkZYEaXJMde/5j1CxSQVkeSKZOpIb1OQn36G2lV0ZzB2Lash5MLfA3AI3rAW2dSz+7P6Hy8OPP+kmMWcsKOnQMKqxrK0tsGxUM0yrJD4hZw5wMt9hbZzJZ0mTnQlCx2MTtx0LOxTxte02czFxLFg2oSG8lkCxDKq+cnY0YxFLonAI0DUdjFYWtfJs3mafxdqaZiqWhxmL6eZtz1pUxwInAqeCmYt4K1R1LATvZyx6x6I5NTKCdZE+01kLBpB04pia/lEm8lFN3xIVHOksTPZYFZqB66gSRBDXvu6AZN1Bx1I4H8ro8RJM2OP3OL3ePb4LJMCoLEaIeyVwo+zg9vx7Pr2sHk686/YH45GRGz9c9KpBp0hYBMR6VIBRqKeE14h120S02VZblaGxw1j1AHeE37IGx6LhOhID4RlqeLaN395aPe3E580Ud1ewkoxc9og4n8B08gX7XYLVF5M1E0I41MFgcSlfOZA/aGU3pSU7fPOBWxypnkolAJrAJy9Ohn3IFnJjKZyBVnUVpAYOeirZKMAJJGBp94AM14aTzEOF0XVUX6OcSMCjx2k8lZ3wVHcG+QASUqNFn1r4hIHoclQK8+zKCDhLFiHlHnQwWZYzDdgM3IiNNuuSZcm8kNHUNHikayIqtLgqayqy55tx+DT+iqTaQySYN3kwTYJOru1ViRvuBDWyq1YomsLSRuYXtmysU1ZfmIpH4KV+UuhD976vkbzWES8drjr3ZU25wLAHD/k459+xSMvMw7kF5hbYzgLbOxYPPFweefzrbn9pZGiDsg0NpgLUBjPbTcNrY5XIbqyrFtlwb3IuxJg2NxsyOxTNsYhXzQLLGQt/w0IOhfdY2LGIzdt6bZMbaS+H0iA/Zyz82lktTYrvWOAAaNCvQVI6FW5Y0ZuC6FzbwKnIpVB1nwVLoaQDioIfS5w00LJDEU5EzFzEvgo7F8rzjIVnKJAXb4bybEbOYFhudIzhwI1nLexYVJvazshHTcHCalapWphggGcdZLfrcppWF/HB5hzJJ3gGLDIDo9Fnlupr4DUMlKi/PDI/0sCtfWY32WNoZg98oDMO5e34JyZyrPeMPHBctp4dQDOMp/ChSNVCgVF1cVjBkATAMdNnVtpmlm5g93b48+CYtgrdsC1Qqda1lE2nYVVvmPEMX7UHJMzikX+FAYLu4ctaWkicMu6Sl5AvPHCZhZtlQ3+kgLNH56J+N3t3Ka4ZO2Dcr5wcBBGTxYORSDMCQubPrscOoclv9oJxsmp8kNqJSFlVn4E2yhO6BU1e06Z9iblfkhaWphPPKB/2RmYqUzkp3ddz8ifchNtnTuLJdaQrwBEgiHzPJ30nf/pb6OUPNGJKgQjEo8fp48meEHjaKcoaxL0ptpUNk1o3RKeH5bb8QRYKDjIwRj0UTV1Dn8wYh1viNFYt0vRzWZsu4lcVMKbjUSFuW6q4XoeUSZbNnCKg7flSAcEUzPlSqGrLeTC3wNwCN64FtnUsPv/AV8sjT6RjMS4kzX78RwMb1zEOqWxgW1hhvWPR8tToEs8nJP7qtgY/fL+CN0Ixi5Gbt6/wqlkcixWdWq4EnO9d0AGw6ZnBzh7NHOQeiwU7Fbv0XQscimHGAufCMwjCd8eggjCQz9fNrq4OsxXstVi3DPoGDag0yPKMxSzHQrJjn0XstfCH8uxIhPyktaORsmVQ8820jaqBTDVudlw83Yosrn3XhsUDuUFli4AGPOkkAtLo04wReNnpOZUg5yfnSLjO0LPmEWQ9Jij1dZ4kjbF72WPeQV+FO1EpJW8qg+xeTtCOr7NoUDyGiMK1qFoSBdVkg22ykL1KiDAPgkCYJWeq2ywcs3KFENtsx4CGXtwLuANYcw0HQjrxW1I0HAc5DDgOK4T6vVAItMNxYKZiWb+nS7qHOa/ghOtvn/b87NXJRyQpok9dMk5MPkRZ0G9lv34/i0osGJ8XJOi3pDQnMijvTuGHDSuTLIDN5ItYwr0/KlxBM4XJKw+yk2RADQYVxTwbMSIqAXoJ01nJo4pOHPTOOFn+/VS+Qaur5foywjWrKmta3+aVehjxapdUMORsh22dE0EyUvamskzlj1iHbUZln+JXGeAENraESWtlUotNYc839duEVAHGNV8AYYeUF6m0TRJEaL0aXc3rgyQblVsICR8i3c038CYWpYUmiYJZWsCp1EE4IyxlNjIiiQdjZwzY86VQGGV+zC0wt8CNbIFtHYsvpGOhhnHUFkYPa2A2/NHMbjYFjX7fudCEplOReaN84TMzMd1jEc5FOBbMWsTMBa+bjVkLnAxeNwsvBt5jx6Ju3N6tr3EzU+HBUIQ56zAsSaJT0ECNXRa5edtLocLBwLHID5vZOUh+npnIvRZyKnKmglDn9l/gHmYo6Hx5uuxOuBqV7ryZXPZLm0elZKdExlALadNu9Ykrp2GA607N4LiQ2fFovaPBM/AF76SP+PWDCHCaXMXzcPc7YhA5PW3imr7qlmWzrtMyiGBK3/CTmUMGYhVgHSLRq+N7qSFNylpJkdXYTHSZ6gHJbF3GOo9wouC2M+xjGVMpq0pcqbMOV+xgMEshx0Gwy7pHOXEwXAZR40Azk4FjcVFO+CWcdfFjadMR3Z8HcRh04qDIHTF/5EGvQGXc8JvWFnFC7FjkDMZu0+3fE78tOxgU1FRB3yxUy+Jso8A5j7Cizd0MCpfAGdlEJIliG1c2LmtfX1V3JGRdtGzRTHlO09DBuq/j/qbBPu0YJQIK3Yhn4qfyjbhGkofyQ99AxAb977ixgX+1DxYxTfLoeDcdXJguY0a04SqvqSm6lJlA8gyTvJYHTTNwZd7l+z6oTHs5YE7pzDUZN0Uqzz5InB7Wx7t8VOuSXaKDdvqjY+rZ2xbspi/2TvJeTyOlIn09JSzC4O8WHqZl/laosX0yde7SWvnqN5bLM6fp+zfKbcf3lB9+/b5yYLEt0E3UeTi3wHemBa48W77wsY+Vz93/VHl5ZXc58cZ3lHe/4y3ljbfs99LlsrFcXn7y/vK5P/50+fIz58rqzgPl1OvfUt71k+8q339cDw1XzpcXH/zD8m9+76Fybuex8vr3frj8/FuOvya2uqpj8SgzFhpIcHRtrtpSYENLOsSMOrpk4wzQToXCbLQJcSI4elg6F56pYBBUZyuYmcCpYDZjRV/CvuzXzWrGQnsv1jSrwaCfhh7HguVPzFiwPGmPBk08XeVJqx0LnAstTyLuJVGCZ3no1OTaSK9cCrVUVuvmbeRKUePaKRE9b3nyEijvrwhHYteucC7yWxZtzwUfymNZVN38jW3DqYkGMvdX2LoYXP8xYxHDs75zHndofQ2g3xaHdScv+I2wzCJkGi42PdfEpZ6yY+0lgUseB/mZ1/MwvOIkbsPvaM1El5RlHvCsOJlPaJgQ+hKlfi2/J1Ac0+Yxg2WTmzjXG/bye9pZ+vf5xNNuaXwP9gXEgWAmYkUhcZwInAeWNOXMBc6FHQvBWe7E742aQB87HjgWylvihQiStaD78LBebnBIv5F9+o1gFmiYzVjVzxIHhjTO7m5lLuh+3aNwp0a6zJwwS8HyqCMLu8uxvQvlkPY04aDsUj68+jpRMtLKAJ62GNVF1RVcjsSpCQdQd1ZSnHRCq/UIBJbqcQRKxazUyrPsJGnI0FaCYFOZAK73tXA9+BXHxBzpCgU8kmfVpzHqIz1ewpNppkdhlznRo913oEimtRTOoGVltIU+fRmCxkxM1GZOxS3KHrxGNJ3dWtlDlUEHmwUFNh+pf/J0KPwGn5A0S8xm5/un4fS6dXyQYZwuP+V3aC3a51mvKrvJEWbqHURjewHjtvCdNCKa77EIe42vX3nicvnVP36lLK2sl6MH2BdZytmLa+rPd5T/7P0nyl03L4wJtkl95Yml8vzZK+Wn33JoG6x51twC324WOFce+si/Lr/3yI5y+xu+r9y5/8XywJeeLCu3vbP8+Ht+pLxB/sErj/9p+eQnP1ueWLuj/NAP3l4WLzxTHrn/iXL65DvLh//qW8uRlx4vf/Jrv12efeO7y/fveqp86o9Lufe/ua/cs+Ni+fKv/25Zf/cHypuO79XKhT9/2bd1LL7IjMWTcixqgzu0u7SGtYOtOvTt4yy1sjGmDSbOYIXWlXCWY5EzFls5FqtyKpilWNLJPosrSjPo9+ZqyWAg5NfL4lSwibs6Fl66Yafi6o6FN29rmdXaCo5FXRJVnRc6XRwKfyCPgRQ87cTEEijifNeCTdt2QJS2k2FnZuxYNGcCA+vE3v6zmYFhUSD16rQStUMj162tI1z6jAZskf7pZ9ZtdISVccc/o43Y3IeBSkrq8YCRnpWHvHYvcA+AS7l1JHyrdE9rgkpj/FQghSpvyidpCEPkWG6fjy5J38OvNb4VbZZxOz4UgUGcQ12YaViR57AkRyFOpe1YaFZCjgIzF14eJVyBhYNTEY4FJcxzRbiXdV7Q7+SSQmj2avnTURwLnfu5h5GrsrOM6qL4wgcHZqcUSseCGQv16Wjp3y97L5jxuGnf3nJq755yRA5GLK1iiRS/xLQ3NLqPpSP2SVvU6idTHMd2T5ya6SAuYqJ/H9Ylfh9twEuey1JxCIzXpSt9gkeypFTCO4pRNPH7uk5YvcEwZtBUfUYMMoEBEq/BMjIr7DQTncss/qmHw8ZSuZX3qERb6JO4FN740IKrYyvHwnkpY1SZVsq07dLJbbJaZkTSHC2/FddKTbClZ9VvmpH0QT6btuF0eidsyi/TmZ/2NpyiVgSrgz0SoEi7L8Fp+iZORVRyvnm7GrEGjzyzXP7ZR06Xm4/uLn/jvcfLqSO7nfOKHIuPfOF8uf3knvLO7zswJtom9X//0Zny4NeXyv/wN27dBmueNbfAt5kFXvl8+Ve/+Ily8QfeVX7yXT9U7tx7qTz6R79Wfu+xPeXud76/vO8tC+VrH/3t8okvXiin3vvz5d43HS67V75RHvzUR8sffHZnefd/9XPl7le+VH7rnz1Y3vgP/3r5C0uPlt/4x/+2HP97/6C84+lfKb/4xdvKh3/ux8qtBzVmbe3Wq7fB9o7Fg7HHAvbR7qqrUSQb1K7PcRs6wFvLuUkzGmWcCcI8p2njaOCTG7iZrfDGbQ3qmZng+xXxPQs5F9pgjYPh71jIuVgXLp00nWLMUOBYsNdCMxY6/XE8BlD9Pguld8qaoX90Amt6YtveCoVj4W9ZMINxpTovzDTgUGhDNvxwGOxYKM4rZyXTH8fj+xXKZx+H0zgY7ZsWotUTY8utdgWPdNoSw2a+68GWHsyanWrDVxb2E5GRskMjhU2yV4u4UczfNJEEMdESMgrhmfIQxWH+kpl8+juAvFkHuMmH/KSdwvp08pniMuuTsAwTd1bY89wKH3ji9TgJyzD5g5N45PX5PbzHz3iG2C2dCpwEjenlPLO8iSVM2hshwLIyPGOh3wjOAjMWHKbVhWVOOBc4H3ukh/c+SDecinNyKl7R7+SCfic4Fvt0bx7bs+DlUPt1b+51HWofhn5rL2mZ4cs6wWffEh+d3Cs7M7txQPf9Xj3ZWBEf7gfk3Cyn4mY5F7foZBZkf85e6LfFDKL+232V960V18V5zh7bfIoHPuWUobna3rZzvcnSzmZYcYzoOzRiLTrKh1nFTGUqeh9UlAB19H1dJz64qU/+Psgzj6RFVsbJTNmCNVm1bGRzpCzAnqElFH7CzcOZ3I9QVE7JW5BZdjUmBL28VIKwh4OsA5lZxgwjJ/IgSrHT/MTLcJo/TRsPZr0eTb+I9Pok32zrjDGiTeLUNSh6uUnbbCiUgSol1NAZmTsIck1WsKE1y3UzoNmO9937nfcdi2+c2ygnDqjt4HVy13Fgsn/y6y+WF8+tln/48zeXQ/u2f5QK/rNaKvXS+dWyV09Bbjmmhxya4eDA1s++fKX8zp+dK197fqX87Z89afixg7vKPhqy+TG3wLezBZ74d+V/++dPllv+ys+Un3rb3eWE2o2lr/5u+X9+/8lSvu895T/4qdfrBn+2nD6vFQi331VO7VdhVp4vj372j8pvfXy1vOvv/Fy558z95fd+6bPl1H/xt8o71r5S/tU//li56+9/oJz9Fx8tu+/7a+X9dx/xioRvhhmuybHIjiFadDqKaA0znCrSN8x9HnC3r4Q1TmjHQoOe3sFgxsKzFhq4eH+FBkSxDCpmKngrlPdaaAAUjgXftZBjoQEVrUjMWOhVsxrwx3cscCxYssTSp3AsvAwKx6AbzNORUDyGSzgW63rd7Gp1LK7wVijPirgU4ViIFsehzVhoMBVORnUuqmPh/RhyOBymYyE6ZONcIBR75mklZDybGnjXm/Z2j5poNTPq9NCyWrxRD/g4UEFnIbJZHi0mWMoinFWvgSu8KiHxwE3a5DsNpzg9/6vRTnlN8XteDbcWPnUF3uNN9Wl0r1Gkl20RVT+2VOMqeE+EZiouXFnzeREHQ7+LK8pkGRNOhZdFeRN2LGFiBuPllVUvd6KGj+qeO8B9JnxmKl6RY2zngt+T6ogZhZsWFsoJOReH+K34cQVy1sqzS0vlhctL5aVLl60L9ykfnTyqGYljC3vKcU1bMO+BE74qufvF64ju/1OLC+Wo8o8yeyHeOBh7pcNu4TOgxf6+r/u71XCs0O4+EvWuctSXvJ8jR7igV7uZMu/jEGAasrlD2xGIjW4ikptidO+6npK88qUM/nYPKlT8vAcHHZtERaAYjk11P2S1mNV0Kjgmf3MSM/Or8hsREWVkESOWkpPjWJcRrRKJZXi1Z+grusoqy7yJNu2vjNB3TAP+tOxZrll5wDhGNCgR/4I7N+pgZOGQ3+jAQ3fREiQd1NMjadDL2CYAa3PNgks2x8CzK3NkQdrwMkJ/Nz2+Ex2LB09vlLuO7NCLH6al3T791Isr5Z/+u5fK29+4v3z4XUe3RX7uzJXyy39wxk7FgpwK9mHwsPADbz9c3v0DB8pLck7+0f/7wiYeP/sXD5e/9IMHN8HngLkFvq0s8OS/L//0l75WTn3gZ8pP41ioQ9944mPlV37zgXLhnneWD/7sW8stI4W1MuHZ+8un/+APy1d2vrv8zf/4reXAhWfLw3/4m+UT3+DNjlfKxYV7ytuOP1TuX/uZ8uEP3qMxQKwuGLF5lYmrOhaPailUHNl80mhGPMOp7GyYZ8Ezj5Bm1aHiNLI4Ei2teDoXOBY4ESiLc7HiM94EdcUzFtXZ0NPVqWPBHou9GgzxNiicinAs5GB4tqEO7OtshbsRlY0GCcdinVfO+gN5sRTqCm+Fki75lDA/eMceDTspkrW7dywkl/0U8XYoyVI8TuRW2Rpw9Uuh3Jlh32pjOqHWwVWD2vpZBx3MnSA2rTDbt6WGjg16cBPPHV1NkNfgqpOmD3Clp0cPCb16BlPs7dM9/63ura04TPF7Xo1GCqaNgM3EacjjCPzbsEKFTtqp3DHV1inzgk+PIv1IM1PBjMSSnIpLcibOy5M4L+fivO69S4KtqB4YznC1My4aHAdmLs7KuX52abmc129kQTrfsndvOaFBPjMNwNKxOCMnmb0WyLtpAZy9WhKl/RHC461O7LF4bulyeeHSJZ2XPWPCK253ywE5ubhYTslhuNkN0Yb2YbBEK/YeIZMZjcPac8G+CxwLlkaRPiD4HvFn5iNmCOOe9H0jPVCmuyuB+NZ0RJdmfwOSihLEMcQSEmHWecvPSP/YPmGQ1Ps+uURWRZDuHK536qGeyMh7Yaxncql0NZn3T+ZuFzZ+KbtaxUmpZc2kR3+MYIjus5VOm/Q0Ge9RsQUHQRU/M22g8QbqtEeGWeYMk4aw4aDowKJHsa0NQJH4DxgkUjB5jIgmiSx3iBgLmuqV/Aaa2fiYN9Su+dIPWI/d32qB3mqVZDu+Ex2Ll1aKHjgU781qBb2GyOcfu1x+5eNnyn8op+JH5Vxsdzzw5FJ5Qo7Ij33/gcIsxOXl9fJLv/9yeVozGP/tf3Rz2a9ZibMX1spvfOaV8uizy+Xvf+gmszu4X2/Di3Wd27Gf580t8P+vBS48WH7tF3+rPHnyreWnf+od5fsOXSyPffK3y7/9+Oly4sffXz547w+VU01DrTh46Yny5U99snzhhf3lh993b/mxuw6VjbXlcun018oDDz5XLm6wOuF0+cIDK+Vtb7+5PPPo01oefbC8/p3vLT9y5yHtk3Sr1jheb+S6HYtobENoNrzbCe0ba+KZptF1uobpRCQOoWcsNPjxMiiWOymesxZ+K5RhsRTK37XwBm49y9UTXb8VSoP2PRrM79WyDN4GhZORsxQ8qWVZlDdP1+VIlCfPDfUE63wkT47L6vLlcmV5Sedy7OOgl1XX4dkG8cGx8J4K8YvlUCyL0owFjkZ1LmLzdi6Lqk6G6GK2JMNBft+Lp05p51bl7sBqXVijxJBtK5I7N+sb9jYv4w5DuOgA45qdqDmJDv3yyLqDR8bJC8rAUlY7el49fkNQBF4c5M/C6cve5091gEfySn6E0yNx4JVxcHreUxrSlpclVYETv+eReISZT7w/psOJtB0hJ04FMxUsf2KGAqfigsJzOs9qKdR5lkJRn7pn98p0C6oeliFhRZZAfePycnnq4uXyipxsXiP7PQf2l9ftW/Smat4GdU6/mVf0O3lR9zIhr53FqTghB+S4nACWOLE5G/1f1kzdaTkXL2rW4oLwVnVT8Qrn43sXy01yFm6Rs7DbvxNeYct+J83wiY7FB+yxQD7Lq04sir9mMW5SeFDpRc0c8mrbnerQ+Z3mcxLfkZQNQ+jg1qBceaTtoi4SOoRb2by/D+E4xgthUY9VcA3M2bZIGcqo92vwDET49TIafytfS0C5rnLAoy9jogNL6tH9lkAQO/4GK516jGgqU9uwo0lZs3CnfIKsF57U24fJZ0uscWW3+6C/CZBaq8D5yZNwlu5TWVlPg0UDI/lswm/CxjkNXwql2uaZ94fCAYf7YzhGlmsZRDa+I98KpTG+26lW1MEU28bYQ/H7Otmg/Ybb9m6LOyvz84/KMfnEmfJffuBkufOm2OA932Mxy1Jz2Le/BS6Vp//kI+V3P/1UWd57qBw9crDsPv/18vALB8o97/mp8sGfuKfkvNvK2afKA5/5WPn04xvl9e9+v2bkbimL/Y9P49qVC0+UT/zrj5Tn3/Sj5finP1me/Z43lFMrXy+PXXxz+dDP/Ui5/fCi+/FXa5dtHYsvPfTV8uiTT9UGUk2ylIvGGy1J99rSt0WTmWGvFDByM1QkOkuFfupKujvT0cCpYAO3ZyschnPB8ifvr9DgyEuhFOJsQOfXwUo/b95meUfnWMSH8ViWwayFBvQK07lAX5dJxWJgjmOxjmOxoqUgcixW9CR4VTJixiIcC/ZYeGlVOi04E5IXjkadsSBPZz9bscMzFrH3YlgOlfIH26aNM7SOtn2NqQ6iNkjHgZ3Bd21QFtnER1SP8kjpQp5gtSbMh0xngxGMTGrEiI2v8KgQoyse9MklMxNrQh7KtLof54oX+lQc8mbdW8BdXgqTRy++gqd8+nSSESYX2yVpyUg9RnIQlPdyln3AjVyI08rE4zBr7nklGXrkK2S9n4LlT5rOv6yqYxM1jsVLmo04wz4LOR7U0sk97I/YpU3XOz3DgaPwpJyKp85fLOf1FjMG8d9z+GC5/eB+z1qwEfuceJzWLNzzly+X08K5IJqjzELIsTipgf9hHHCVk9/kK1eWyxmdLwsPHVh+tai3nR3XDMgxORgnmJ1TAdc9Y8G3M/jWzLoaJC0jFMxvsZKuB6TjcS2Luk0Ozi04GXJKDnhmD8c/HHPcV00UeuYlrDPYEhvWmyqzIsyKqtDNFo6M7o6eYIpB5eF7oa/XipmB0axIQrqw16PnMVIcJAH6/MqiZ9uyeyB4ee8Fl1S7clDQCInKEvUEoX840OzY6zxwcSxFT1Hy9zL9DSYc4j6vj5MXRUjuQOJo9VZNNLUT/JMX7XLj0Cno/M5GIkj2ozDvhSZzlg4jikj0ZQSS+sxAHbVF4DV9hWw+qVrNQJdQd6Pcd++9s1i+JjD61ixXhr0gYOifffGr/fI2D0v4bV/v8QdfOl9+98/Ol/9Em7bffNfiNZHzWtoXX9EeMjWcT2m24hP3Xyj/qRyTN1bHZO5YXJMZ50jfjhZYfrE8ev/D5cnnXynLew6XxTMPlIcunih3v+295X0/fKKw0nDtwvPlkc9/qnzmoXPl6A+/u/zEX/zecmyyt2lt+ZyclF8vv/P0PeWD710q/+J/ebp84L/7a+X7y6fK//k/P17e/nfvKz980+Fy/a78YLRrciwSfWh8NBjvG/FEUEhDNG10M0176vwamoyGq9IkLWE2ZpsdC5ZDybnQIIYP4+VG7lgqFY6Fe10Ji43aciw0wPKH8uqsBQ6FZxk0qIklSeEktDKpEWRj6waOhfZZsMciZiy0JEoDMXSjnbRDIAfBzgpLnnAgkGHHom7eZvkTcOd1Mxa9Y9HNmMiw7jnRJU/s1HQjzp8b6ugm+zxwfdR8WXyoj9ahBR2FyDppZMhtLGpM9bHlUXmQn1hQDVwq5RY8Uves+4rdgt4GAMGbdYA3ystCJLLIUlaCtgpDguyWCBkRT5crdYhKqHhCMp4wUnbmCz9YVEY9HDLly4fQciI2aettTCx7kiOxJKdiTSMplj/hUDytAf5pzUSwgZvlRLdrk/StGqQf0RI/ZiteVP4T5y6Wb1y4KB5XtPxoQY7FofI6zVrgDLAngxmLF7Skj2VOp+UoL8nROC6n4hYtb7p9v94Lz6yb9OdNUGflVJzVG9de1rKpC0qzFOoQMxa6v3FA2FNBiVg25Q/u6beiwpRFQfmYJTCcHYp7WI7EbdL3rv37vcH7qHSLj1aGg79bv4H80B7mYyCCrYlDz1mNSEaLE+VIC0dqfA0uwAZsUkkT0JqXdQvC5GBQO/Oo1UrZObgadbgY3vSeISNZj7ISGNTVCJnowo4ofwO0UXmw7KyVPXmmzol0TSHEmwn731XKT3bTdI8bOqXNFBKdpV8ntjkWEzUmSfHaBLFKHatUscm0rrPJdP+lYkE2LdfATOyEO8pvunRtQyWwOOXnvfid6FhQxrH1emttHf+SXjP7Lz96pnzwR4+U97xp+zc/LV/ZKL/CG5+eXipH9u/y9y2WVja0mXV17lhsbeJ5zg1nAT3gW75UzjzzcPnsx/+kvHTsB8vb3/PO8gPHd5f1i8+Vx77wmfKZh8+VxTf+WHnvO9g3MSng+kq59PyD5bd/+8vldff+fHnHwU+Xf/Q/PlV+Ro7Fm3d8qvzC//QtcSweKY99/amJZjSONBPRVGQDCiwbVGAJJzRcFAnPxhTGxBu80pGmY8SpmOVYsOeCfRY4FUsa/PCGKDykVc0uJK8d6oF2abDigT6ORXUqcDK8JwLnQo5FzGAMS5GaTnUp1Ib2WYRjITnazLrKBm532rFMiNkKZMSSJ5Z34Fgs1FkLbda2QxFLn3LWgtkKv6ZWYf8WqLShDGabZZowD8NqQlgJzupw2vCalfYlI0A1oyMlD7tx9Pz7MUSPbtxkI90YcFKTlUWvlXnCp9ErgiR0bDCok7iLZ7kzhFmPR3rWESWhLJE7lhRl7Pn08cav6pm8gKe+ve4MdBqOI01oY8UQb12P7chOnTIT2bn0iU3S8pwEAABAAElEQVTazCiwn2JJyDgVOL58j+JFbch+7JLewa5lScw88AXsu+Uw3LF/sRzXPcaG7OeU99gr52OztWiOaiB/+6GD5dS+fX4dLJu12cD9vGbfntWMxRk5Ihvi9Trl36FZjXsOHvAeCxz9i3Lcz2i51Gk5ITgWrwiPvUBeNiV5diqEd9m40ltOBG9vW1ApD2kgu0vOBr/Rs6JlIzhf675Jzsv3HNhXbpW8Y3Jm+KZMfLE7vvzNBm+WUOVXvFkmhZMzvFEqLF1vN5twas+06zjsKknRWkOOYH/S/T1CTYUkZSg/D+NUYoNbXDwqmvmlUgozamHgCy/uNyIRn3V/97CUf7Uw+AYWbVSmeQBifpJvNXUJVUihYy0IpF15g9P2V2h7kjErS7MegRfpkbyOfWf1sE2rhaF2+L0B7jRuHII7ScVqYiy3gUPnysT61ELk/rnGtEZ6nbFr2nYKz3SGVqUzEPYZ9FS85QHdoRmL909Fv2bpb9WMxastwPNnVsv/+usvlLtOLZS//VfiLU5b8fo3nzpb/lRLn1g29fpbYtkTr5X9pT94ee5YbGW0OfwGs8CZ8rU/e6A8/sxL5cw5zVrsvrm88W1vLW+656ZyYIceAD70ifK7v/mR8vnzp8r3vfVN6tejeDsWDpUjd/xAeScfu5BjsXT6sfLprx3Qx/VuL7tXnykf/d9/rTx55z3lptWnyqMXf7B86L63fWuWQrVGUnoS33DrHvFZNZONZYbGMV00qcAzr5+tENBwYM2x0KDEzgWOg87cZ8HyJ75f4RkLxZmx4DsW7lCZe9XhWQm9gYllSX46WsNcvhSzFgzyWQ4VnW/ttzRYlg786SN5vGrWMxZyLNbkxHiDuPjzJJCBXzgWOCw1rie6OBe7ODVYSofCb4RiBoNTuNHhD7JjAFU7etkLW/enC6VL1kd0t9WmwCsC8FpF9FxJFqF4ckQ91rwOxbm6OOzg0GSdJT0hiGO00Mp5/aXqEXoFjlXpiHv+SZplzfQsnMwjDHYDUyRNeYzwpVdvC/NHMQygMOWZB7iVOEuJpCbN+JHmY3R+ZbFoNPkQs3LGhQc66f5RyJe0mR24hFOhwTn7KS5rhoJlR+yjYC8C+xueW75SHtZMxEssxxP+ES0tulvOwO1aXsSXs3k17DNyLJ44f0EzeXrdomhvUd4pnQc1W8BD6yUNNnl17HNyGF6QY3FBv5+d0vNWzVTceUBrNQ8d0AZr3a9SDh6vyClguRQzHOfkWOCon9RMwzHdv2zS5g1UZ8GTE/KydOS3x1rOY3IODssjWFDcv2PBWV7Fx/OYHTms2ZNFfgPKdzF14bfExm6cDV6Bi4PBhvO9vMlNebt0Yq+YI6nVI7qs29GgtMKz7mRiH1VcJLh2gKzPzGz8TFsZEFBxNajQ4QZIQcYIpE5Ekxd6KWeKn3QmDcosQ8uSUOuWetQM8BKXMNtVyhjtCu1bqBqcIYwYNmww+DRhV4/0v52GXXVzupYxdTOsl1eJkNrkEql0ZGcdE++wpHNoDSzjxtmCljyOYF+licVAK1gFB+Zw7cs5KsuAsmUs8a3txDYpLsB8eft9W/L5Zmd8uzsWlJcN2A8+tVT+6nuOlh+5Z7yB+8x5tZdLa+UOOR7/5Dde1FshN8o/+HBsyob2c49eKr/6ibMjx+JXPykH5JFL5b//67dq72W760GfH3MLfJtb4MVy/0c/Vx5+9mLZddPd5c1vfoM+DnnIeyK1xqC89Nj95Suff6Bo0m58LB4tp+55e/nLb7l5DCeltvLy4x8vv/O5p/Qw8WC5+x3vLW99zTdvP/TV8pj2WHBk4x4NrCENZoTukg0pIOJJSyNK2mcfT5hCCso6bQYkMSjBsWD2ojoWGsjwJBRHIpdC5atncSzAzT0FDIRwKthwimMRS6JifwXOBIP7nLHY2rGQrLoUin0Wa3pyzFNeOjNev+nlUAzGNBgKXsxY8KpZZi0W2owFb4baIRycCr8lirjktzdCoY96F04BfWK3PLFl2rGF7lija+KazSQd5XaOhfGQgd1hHCw6+sqswkHJI+s2dQDe0FC7cRn0CRmB5WvFsx7JGD7Uf3f0MhI8xUk4YSfB4NTFcihvd/R8kJNph4nrEK5WWE+lx/oZnjaEvfDlE+j+lbMgByG+kh17DTxrIdzq87qKqet0LFj+xHIjlj2x2XEN2Tq5h3E4XmCZk97OdFEOBHrwytc75BDcrD0LbLh+UY6HHQY5vyypPKz7DMeCJUh80I43NzGrwVKq5+ScvCS8y3KSF1S2WzWLcLuWKDGbwIfyFnQvsm/inHBfllPxvBwRllqh77Hd2h+hfGTgWJzT7xLH55weAGA7Xld7VPqw/+JQdQ64z8BnCaI/UKkyUbw8eNpf70TJYOO3llnJoWBWBqeIL4Kz4ZvlXzg94EBOveWrBaJmsn743dT7qeMNTVA60i7gTuGpz3BzmxFCjQpJShsiACs0mMK4+0U4Wcsa2jT8yPK1v+/zvmzZWZ5eZ8kEL890KtDE6uoCz/Z7cBG4VEiGCIFPE3b1SP/bGWGLp49qj1E5enlCSokVdUSH9ZIVGaFbXNE+aYOoXmsBejuO8uFTyxlmHOs6xSXdl3NUllnIHcz6pUEn5aYxyHY6dZ07Fp3xFGW/xC/81kvlghrFv/D6fZ6N4CHD119YKV/42mV/Qfsnf+hgYe8E6ft+7Eh5w+v2lq/qw3q/9blzGidsjByLj375Qvn3f3quvEuvoH3zHYtlUW+/uE0f2ZsfcwvMLfDNs8C2eyy+qA/k5etmafiy8aOxjwY/QncNXa8w4EUDnuq2jk+AjPehOzXxaY6FBjfNwdDAhiVPft0soZY9EV/2zIXiwHTyFAYaePFKWfZWePO2Bih808IDGw14/GE75XtGgUG9zqa3ihVvhZJDowFTOBaxgXtNT3w3NACEv22iRo6ZCH+7wqHidizYwM0+i9hf4Y/i2akI5yJmLSTTcnEq0KHamJ5UZ3MysDawGmbcaUPrJeug0huqTo3utx2VD/wpQ+JkPrh02D6Es1UnOhMu/NStcki25tM62arDFCeRk0emp7IyvS2emFMOcBI/+RECS/oep4djBqzB4Wvay5C4gB9LwUDS+9NxKnAQ5ICyx4CN2Fqgx4vKNI6QI6F8nAkp4HsOm0DDvgnOy4qvCO+K8PUArmjXgh2OV3SvM9DHaWYmAAeA2YOjOM66b17WfQkOy44O6L7m1bEn5XywsdtOgByAlzQD8dzSimc2zsix4IvdzA68Tg7KrXJC2FjNrMKiYNx75/R74rW0L8ixWMGxkC77xZsBPgcDWF5xyx4LyshBHo7OCenHNy14xSzyOfwKXeHz7Q0cHeDIgS+/W3itCA4PnAleW3t0L6+s1ZfB9QYqXoXLZm9OFdl1ol+QqOt9rFgehmJnHxlGqt3fNdeBCMZYXWbLUKSWvcut90YPibjJmg6Q1l+iYHnvgYkduR+ANTi4sgeHfzeOOeGCN5UAiZYlPG4ra5tp9OSnEBvH7yFDwWRPjibTqfEF3td6tN8OxkcaelXiIRZ5wzX0J93oScw6sIn1qVyVDgHiLngrh7JbXHyCJhgmvIfNEpWwxEs6wxHvMibWECY+kBav9gA2Iuvsk/znjgVWGh9sxPZrYr+hBxwr8Zs4qg/fvf0N+8tfklPB93HYtP3Lf3imfF2vnOU4dXh3+YC+UcFSqP/83hPle+VscFySg/J//c7p8g19LI8DR+R6vtxtovllboG5Bba1wLaORb4VKjgMTWI0gl0nKITWiCqejWQPyzhtMvEM03FAhuHqTOlogZMmZHDvgQfOgwZPdiK0UdTfsxBsyUui9F0LDbzAUy+rkyekY8eCV8+mY5EbuD3wVwfrTpaOCj24KCpNJDv3WIRj4c3bGiyimwdFDHIYbGkQxIwF37FgRsJvhWJQZeei7rUQXsxWsBwqZiyglcH0z2CJzj7sisNhOLpwGGdsc4PdVVljow2wTE7yahkpoHNUDhVG7If6TUrzEpyycmToxIzLVjxMG9JsXHOTOEskUWX3GiSvXmaLC7/H7eGhVuhLvPEhHpkRq2VqoFcZsfnEOByH2IDNsqbz1bFgEI0jQb52KXhwjXNBOmpAswmyDd+suKB7HeeC/RXna5pvVuxUmr0Ge0WxS4/3eRMTm50XdPpVs0qzf4LlUwwVWUK0X/fXYb2NicE5jgdfyGZZE7MVT2vmgzc98Raqo3rVLDMVN2uJkp0Q7j/9UScvyqlgQ/hLciz4HTJQhD+DWAb2zGyE86DZQAA6mFFgOdOi9YsZBiqd2Q32jryk3+slftPCPSg8ZjVwhPgts3fkovTEsVh0npwKlSG+h0FcH/tTGqeDt7ohi9M3FcJd7bqXiU8O0PKuyP0QI5QuH3i7pxRvd9uEcUuK8ayBs2HKM54V2IyVOlkX2de/eyesRAUnk8iwE4FE4XNEe6n6l/MYD1bkBNKmuY3R7JPqCx1oX7y3S2E+SOH3kb8RM+su8B/0Uww1KAdHZtQkcPAdZmbiBIWgCahEghPLcoCW/HsY8JmHdQmegZ98BauiUi/TW1hwGvFPMrJElzoE5hZX848SJX7Pcyhr0sdd1Kkws9xzxyLtNTs8LweCPvLAYjjFU6wLeuMFv49D2sDNcUXt6KzvVOCIUG9X+5r3lP88PbfA3AJXt8B1OBYwixY4GtJxh9QaVf1Y3XjWTg+qlkc8T+X3zkPiJQwa4pzNscCpSMdCnahnLORUNMdCgxb2P0DD4IFOFEdiQU88/fVtxe1cqNP1xm2HMWuRjoX7C3RUIRiqrbF5W0ug2GdBaOdFOtB4xXKq6MDtVOBcVMdilwY/OC27tXSkvRXKX+GWsyG9/F0LQp2M0tKxwLac7vgxiuIRVHvXtIH1Ui0+Sg35WaKAZN2Rcg71pDPgA1WLoQ+44HWhE91lS3poGh6yajqK1XKQ0oPglzJB6uO9rB6ethokiqeYpvyBP8CENhWuOdJTwgYnwa+K1T3hNzqxX0KOBTMXDPgZ+PHtiSXls9RJgcvKEl/G4wz7wGPATT6vk+WbFedEj2PBRmkG4Lz9iW9D2JkQ3LMjomHQjk3Yo4BDwX3JQJ/9Ekc1GN+nOM4Nm7Bf0EzFMzgW0g+92UR9pzZTM8PBYAhedlKkI7MgLJ+6rHOX6PnStGcopBcHS5Zu0vcp+ADfIctl4Ips/SaEj41ZUkV5WUp1Ro7/s3JUzirObMxxzUKw8fykzkv67TJDclYnxsFp4g1VlBnHgtfVMgNzQiFf9F7wtzD0O5YsW1D8OESqIwbSjlZIBuRxNOciCAyrLCKO/vXwndnhJZww+YwHkqmFeFQ20/u559HiwuV+bfexbR4M4O+YYF4aysMW1dW6wlXdK9ku+mvgYrFbDpiXWmIfVRdOBW0Uy0LjI6HxDR/08jkYaFAH+UPKZQHXR2Yo2WAV11SZ3+ghT2DloTxi+RtOPpnuSGdHrYt4VrbJvQfA0/wGkc4GZnz077kLmHr04E3xpFdG4g96DyVNumzdXF4BB1zowQot5o5FWmwezi0wt8CNaoHrdiyyEaUhHOLj4rvBVsPL0RpwErSgNMg6GWw5b4uQAQmDd57A8bYZhwxIesdCgxA2cONYtH0WwmEJgdj6aSadqT+QpwFWe+WsOlvPWDAY4mQgpLC28O5wWAolLaVnzFis6qmtN3FrcMZeDpZD+amgaM1D9NFpM0vBOXYs7FzIsWDGojkWoulnLLAnAwDCdtpspDVAkPlsc+yoY9w5BQy4Y2F+khWQwYDXuj8ZSxICVbx7UhsSFsAxqo4MndDFOtVwmgfbMb+kEhyRvewha2YM3rNkpczMg69LI8GWMYOby1vLMyN7M8h2yfJHdpaLAf4KDgHOgJ0K7ZfQoJwBOq+RZVDPAB1n4Yxmuzj2ih/OAg4AT92BLosPG6wvKnxFuHxpmxmPg7pPWPLE4J8n+Pu4X8WXNzydkZwzCmUYL2ECDztwT7PM6aQG/Qd1L2IP3gL1Im+EknNxXvypdRwPvqLNLMNFyc6P5uEIUS5qb5/48TE+7IyDgFzPpMgpZjmWT35flEcnG6934FDwe+UputhAd1aOxTPs75DOzM5Ad5P0u1XnBf2+XtbJfg6cNDTGcaHsLIXCobh9/1696UKv2NWbsPbLDnz0kvXWHFx96pLx0Y0H0AdlagnM1o68jzIkw/fJDJwAjTKCj0EJp42r7AVKvr5PE16zCZIqQeAnjVq0cNKaI8HST9WDHngssV9GG/dpA70JXvcJyzG9f8t85Yzqvtgrey8u7nO4Rzh8v8d/MoLLWQWjX8rt1Zzql3oaPxOtwFHesO/AL9FGYWWMDiO5godetaUS717PnofpsjLhhx5VedsbZMFTPfArV7NpfJGRfJwTl9SLlNl3eYkfOIPFUpZpxDPpRryarKD70Pvnb4XqTDuPzi0wt8ANaIFtHYsvPqQ9Fk983cXKxjNCmkg1lK1RHJc8G06aytaYKt7gtcXN2Qmoycu04wxMNLDBoUinIsLYxM3eCjsT6kzDsdBSKN4MJfx1DfoRwYCNp3MLbN5WR5qORdtEysBEZzoILg9l0j+dDs6NZyy0edszFmzi1uBoXQMrBk1BF44Jeyy8HEpydnH6ySAwLYNSXrx2FqdCJzKBecYi91ngVOA8hF3RhZM15L1e5LeDQta0cVqGwNFPBaSSdJQBn4FDBnsGGn2tqyC4vis6pYjsxOHbYFLIOiVAOZK8WUiCGt5mlBEkbSJgynU+siuvsdwR9cwEdIgPc8CVj8iF07CskfNleRBsYj6rmQCcCwbSvEaWAfhFwV/QgPq0nFK+T7HIU3jdIzfpZAaCtx7hWDCQxxlZFt+Lomcztx0AwTUM9FIlP7XXQBt6XkOLw/Ki7kn2OLDRm5mDdepP517dX6f0lB8YB6+OZTkU36bg4M1Oh6QLS6dwgE5roMqsBrK5rw7qHsVROSVZi1qLxe+Sr3dTjrM4B5K/TzgsUUIvHBmWMPmNTrIXsxaebZQsvqHBm6b44jfLsKBnNhGnh/0Y/PaX9Pvli+F8wRtbYBMcbfZ8HNWsxR3aB3LngcVyh5ZuHVtkWVQsbaQs+BfcOxEqDVC0tcKUYMBoYAtJRa2CRu22ZNyYAkHiHNMaI0lqTtyxQT3cbYleuVbCLYJESh1SjNLZFjI7Qft2Rba/vHSpXL50qVy6TKgN/YqTZuaChxoLqgsebJhWduS+2L9vfzl8+HA5eux4OcBrhTVT5fZLebFPBXOl1rP1TDUxSKJGiXu68W847eq2YFI+pPRthKVSZmc4tenS2iXlINW4uphKShnmi4FBX5VNLYOmygEj8QO7AmoZjSz+qXtNm8a0IZNo4CQCACNwGY5UokE6fMHmjkUzzDwyt8DcAjeoBbZ3LLbdvE3D6+57U9FbI1xzsulkliIPcPpz6lR4HbFwRo6Fn9bFEih/eVtOBfst/NpZfSzPm7fVuQavnLFg83Z1KtTZMphhyUgsC6iDfJ4A0ykpdI+pxp9uR65N3WPBbAVORXzHYk0yeDMU+J7t0AAul0LxRigci7ZxW18qprPPt0HF7MZ4jwWdK7xwLGJmQmlgOnPA5I5Maes3GFEx8AgmPZZM3SAtEoQtOVSHkVtSvIwDoKuzoL6Oq5mktMq9D/r8Tk5SNEkCJKxDczY24hjdc80WrUTGiY5euPCbZAXC5muiETLQRQ6hxmsejDMAXtKo/KIchlfkULCBGgeDj90xA8FA/CVtmH5R36Dg9a2rKsmJfXpSr43St7KkR/WOusxqpCOi7zppcC+nQk/7WT60JL7UAzMVt2jZ0q16as/Xq1c12Lyg+9D7FoTPzAD8YMigEYeBpUYMzOHPngmWGuH0sMEah+OofhNYkP0NvLIWJ4WZEpzLE3KKT8mJeZ2edOMAsM+DGRVmPZ6/vOyZD2pmv+7xE974rQ/naSaBDdsHdOOCz4CY5VXoShku66ULF+VU4EDgRPG7Y88EMzjMNC7LduwHYbM4dMjjbsRpuVXLrm7TbMXtci5O6psYh7Sxm1kLkRoHXrH/hBmb/O1QaTp9GX4rlDmOOsCULj5qAM9MVxBChqPxBJQZ3WC1QhvtQLk5lrwqMoHvM9mBhyTsnVjR93qYmViSM7F0+WK5dPGCHAp9CPGi4nIqljRjwYMYlmLuUX3RngQPbZLHsdDrhI8cOVaO33STwqPl4AFtelU9xZ6L6lqoGPWXv1lHdAJadYwijxKVJmDJJ3+X/E4zXhEdGN4DJCC5jsA10f9usbpxVXeErS1whiA9I8nPWoJVr4sfNmT9k5lWaAI6/OSZzPJGGdF3SBk1X7POWA17hA05Ft95X96eFHienFtgboHvcAtck2NBg52NdtijtaoT+GZr0YBn0+nGvDbEhisPJyCdigzJi7XZkd87F14KxZM7DZBwLvzaWT2BXdagiRkMcHFK1HOoQ40ZiT04FOpE41sWOBbVEdCgJJcxZRkzzO9YtLdC2alIxyKWQoGbsxZ00iyFYolB71ikQ+E3R9GRe6ZCoeJtxgKHQgMBL10g5K/aHHMRb0eFk+47WXrNtLPzBgLRR8L5upA0T9mIwzWUSJZtcFyw5as9OrVDKk8zs5PWQEOJUXcvfZo0dKv0Wf6grfqSjc7Yo/EcKwqvViyyGvPQAfrknZRxv1ZE5RPjuy2ow+Ac55iNxyzX4QUlzFZcrI6FP24nRwKngJkKZhNwKp65cKlckGPBE+UDLOk5tL/cpqfuJxWnkPDkDU28Ecn7MJRmcP+KZj/OaKaD18KiAF+9vl0fxXudBtcsI8LxvazfT+yF0CyJ6oq3SOE4ewO3Btfsz6CM5OE08D0KnCFeB3uTZgFYiqQfoPdRvCw5LzDo96C+1NkKHIs95ZhOliZRZ96noQFtfDNDT8n1ezqqJTan5Fzg8IDLTAibsHESWDqFY8HeEL3dUUXBGeNDgPoNq9xUILMr5EuCSiD7opMG1CyfYraDOYhj0vmUbHYzjhXf50AnyaH+Reon75Sbb18wc4Jjg2OO7dp9pvS4zmtdCycPcOHHEe1UxC2oRoH391NFt30qimW2O9r8A2ssHxmUL2Qhm/sh9k7Ibmrbrmim6bKcCJyJy5cu6Js6l8qyHAxmLJitWNJX1Fc0C6RGxO0KDzUoQOgYztte1c+BQ0fKsVM3l2MnTpYjR49pSdReOxduh4SPdlxNRzyNUHVTMBxZYEESzxbk0tkyCcBJvgkbhVV+s5e1GVINV7xD05CbZUTsSI9aOaiSqmZ+41UjXnZbdYb3Vnjb6j9lOkknLbzDrGkPW03YEc4di4nh5sm5BeYWuOEssK1jMeutUNkoRklnNcJu4pshaFDdZBLWxjtfz0k6nYkMAz9wgfHEMx2LcCqYsRgcC5wJL4WqsHVmExiISCizCTgOOTthB0Np1oLzxijg3g+hkAbfsxYUkH+FDNzWefsUX/deueyP5PHl7XjlbLxVwp0yMuQo2LnQjIU3cKtzN++6r2InAy05Fd5jQVgdi+ZMeACYDgbdG0fY1zbXxTDCGiddTWobA5/RFRvf3JQflUFKBnLFEB8f8GlHChCAuhnlNaSwVyajDpUSm8ZplixnBoazq6woQ6OUzOSs0DgB8DUzOz077OuKWocqK9SNTc9+RawG4+yjiNer8hamOO0IyLFgfwWzFksaSLMJG+eCb0+8oCfJL8ixWNd9ul+6npJDcZs+RHdSg3DuzyUP4sO5lgh95kZvhFKEvQ68Xvaclg5tiDdP9E/oif2detKMY8GGZpwbZh/46B37LFhmxT4JnuIzsD+he4zBOjrjfPDaWGZVoDulpTC3MAPAshndCODgGL0sPmfEk5kQ3ih1SvfxHcLh9bYM2tdlH15B+5wGtE9eOO+3X1FBhxcWtf9BH+RTufh2BpvIcSzYG8CsDfy5d9hQzr4S9mBc5LersvHRSxx0v+VJ5ectVlQDe6vOSydmf3DqmLVAJz6wd5D9Aorz+0YGB/OnOBQH5FEc2hPOBbz85ijuD8nPe0bRdvCrMQvjAO4ynQd/wTow93gVaz6ZBWaP50wuVT5RcEnGEZFgpztfdb/Gq7R5UKIlTytyGvg454pnTNUOqS1a1+zsmkL2VyyrTlfA1b2CfflWzi7NIMWDCqUFY98F7RTwPfsPlkNHT5Qjx06UgwcPadnUotslt30zFM/fu8tbNW6wlq4loShDwRQd2gtoeh6VdBQYJ+tilJOmlgD9p/wJyph/VnCnzxQ/+aBX6pb12HCTjwCJkyE4yaPh18hmnKGekyWqNXn1KdF3g2NxZvmC2ktauvkxt8DcAt8sCxzYtVcrCxa/KezoV7dq265FwJ/bsUDIoEA0nl1zOWqMs7EFy2dt0IHniTPB4ad2Gnywl2FVJ84FyuaMBQ5FLoeK/RYxYwF+fGeC183iWHDWQb+MZedCcC9dUhpnIGYd4skuZaHh52SAu7ahZRl1GdTKsl45q058XYMdlkKB5AFR5Z/OhR0LHAx15MxUWH51JsKxwOlQxamzh745F23mog6A1O2kPlZIdol05gsgQzaro7RBCXHSNMRcT86q+WO0QA7EocOj97uGI/VyPQrfdY0dk3YWm5rJM0gRGBObd1SG1WIFJ+MFoa+ZeY16BpPxNVXLMDbKMkMRT85xKFY0AGZ2gqf97FBY1aN7nAuWLbHXgcEvJ9+R4OvZfIma/QSnNeNwSc7FPpWLzde3an07T9sZFDN4Z78Dg+s9PG3WvcBeh5fFg6VCfqIv52SH8Pgo3Uk9qb/70EEthdKH7ORYsLH7op7q84YnZhvYHM6maJYi8aYmljDxPP+ScL6hL20/5+VY2iMk2C1yTm7T9yvuki4MyBn44wCcVRn46B0fvOMjeyyFuktOCDyZCdGvy3rxgb2v6wn6S9KPJVxH9TFI9GLGIpY2UZ74LTHj4D0TKhubrnEQcC743S7pt8QbofgWhswm3eJ1uTgELGvizVrMXuBY6Jei30sMklnSyEwJA2iWVIm1y8ESr6OScWKBfSxsYI+3VO3QwA167lN4+H4VTRy6A8Ug7ip+JwklVAbMORLB6QqswSgrE0EVV+5PGBMIgsYm1WVo+9TOySbs56KtWdGSpysKcSZ4YQSzGMxubOhhx7rwwKU9pF0k35xtk7ALtnHb5gJJHrL0oGNx/6Fy8Mjxcvjo8bJv3wEtndJ+C+GETQblHeuMgcbg9AdlSEiUx9cexXHIkL/dAW+3GzOQqrUwnI+pHgC3og2K7a/QZjlGmF15k3+GkTWDSjo2fcVs0JU6oAADjWO+QKSlUPf+5ZH41zJBn5q6ZdjLA4a+9MmcOLGcJ06c6NGuO/5ff/z/KL/8yMeum25OMLfA3AJbW+Bv3fkT5e+94QNbI1xHzq233qq3qS5cB8UY9VU4FjQ2AxManziHhp38vqHKhpjQDZXIM56cDFfjlXCn1YnaoVADmI6FnQsNfOxYpHOhDpZ9Fl4KpcHVhjpg3rnP2uJc6pSbq73HAoeiO+NjeVEOGn06DDeqO9SgSgcci3Qu2oyF5IAXjkU4KJ6R0AAsZyyGfRaC2fmo+y3UucfrZjXUUeePY+H9FY6nPXt9UCgslfZOu7XKoB4akg3cUKyoUi5TVp7CMf6A3vABJX6XTXTEy7191Vd6QkLHWlU2JfVpOqRWHNLdrWSaLCj8x7lBzcDKNiA5kmBAXLbQuaFXoSzr4Qh9Qxp6MlBmoM3AnYHtkka8lzQAZukTA/811ZeyHMfJYOAMLidfyeap/wvMNuhc0rlb9y+vTWXG4WYN5vlQI0ulvqFOmgE6ewkOyxHdLzgD6+d1X/OkniVQvChgp072LJyUE/D6Q4fK6zTrwYfs2MPBLMMLcnZPCxfHgj0JR9Ug3CJn4Hu0D4HlQDgoLFt6+vzFchZHRba9XbMmd8pJ+V45OhiAmRb2hrAEi03ZF5Q+onv5FjkKrxeffRqs8+Sf2ZtL+r3xBic7K9Kf8h+S7rdogHpQ5bAzL1xmLHAuMDdOGs4FsyksX+Kr2vym2bB9XrywFW/SwtliqSLf1MAeClwG3oLFUjE+IHhe+umHo2VHqge5C5SZ5Wnod0x8WeL1ukU5V3Is2PPh73NUXcDxNzAYdOuGyAEgtwIOltTl5taFI3JpS/LIHPJafMgOtJoxBbffkjLg7HzxjpdU4BzIUeCV1ktyKDRbsXZFDzLUrjGDAQ560V4wI7uuNpC9F46TJt+HyiS7GLfiUx+4bDxwWdW5c/fesnffoXL42Ekvj9qrjd3eK+ZyhwxYUYwoSlzTVuRtdbTfuXhlPHGTXhISNDvE3taFWzPKFe3BkJ4Skp+407yEJw/yE5a4vUZpSfJm0UAb6tGWDZRegpsMs3634JFoA7msI773zR2LNM08nFtgboHrsMAN6Fhk40lDmiWNhpyGNc6ARz9A0xyI2YBnY02a2YiEQ0V8evopieAMPvJcVUeKY2FvSIMlz1jQGev0Bm46YQ2OmLVQT+tBfyx5ihmLdCaac4ETwJNPdbztiV0tIH1zfn07lkJ1m7clm+UpUi9kaDDDbAXLm9K5aPssNDjqnQqWI8RbouJVs+lQDBu3MZ2eqja7yo41jq040uaRGl/ptG35Vk81v5bLKRTvjqytIOkJw/nIuko89HHhk4dJQm5yZhAx2gNSceE16ozRJUWabyAmKEWkE+DRmDKRlrIaPQDzqzaAuC+rbYBmHCGBOGMy7kmeijNwZgMxm5lxKJiNYKZiWWk7FaJjMMtTcp/mFWIY9D6vwfFpDZLP6z7l2w9MTB6QXJYA7ZczwEwFsxxnlf+MZjL4dgM6npQjcFD5OBbPafDPvoTLuq/5IOMO8eSVryfklNxx8KCck0V/44FlWLxuFnxmGXBs9mpkfETLl45rgM+XtEmz5OhxORZP6cSxwOFmOdZdOBY6eSsVjhTLr3BQzmi51FnxOy4+bDC/E8dC+rPHgpkElifC02+Y4um6bMNSLU54s9SRj1LFLAz1xJKyWDrGK075TbKnYkOzgcxGMnPxiviwn4S3Zy3L0NSO5ckmyD6ofRPUER8QZIM8S3twLnZoH5Vfy6s87quTwmffyG1yLo5qyuOg9OAtVXYoLDs2i+PgMAPDITLP7Cg7HIt6bwBHj4SDTZoz7yvHQax3FfdTu/ucOeBmG5c3rveRyYHDefCrrDU7sVb3crGvYl0OhT96JxwVz06FHSqlaeNydheZIHgvkKLcQznIxSa0b1aRmjAftVdyAg8cOlr2HzpW9h08rPZL7ZKQrDvIjhNkIRCCGEuLxBbXKU5LV55RmODb8294E77AZ+FN4RMyJ3ues3i4LroiEo0ScpU1Wvmx72YJQz6EQqg42cqYovLodQEOTooGBfL77v3O37z9dz7+C+VfPvpHNs38MrfA3ALfHAv8zTt/vPzde372m8LsWzRj0Zo/+hsdmY4y0LjWttOAaEADh3jfHjstGIMEMJwHTj3T6SBNx5kfx8sZi5z6b0uhqmNBeklLBtggCy5P8hjk4DjwUTwvU3I8ZivsXOAMMOjg6WU9s2hu6NULr+kpKUsROOMDeVqWIFkbGnyhOw4By5qaY2EHg3TdYyFHouUpHns6csYinAvsN3UswqbRsWXntVUYtRBXbOrBAcm+msYVFMjdNVB7gqi15DWqR6ExQAkMxStvqAMWdX4tjsWggiht9ODRa5I4di6MNuQ2hwMk3TNWoOlT0+SZZHBGlON7TmNxz1Cw78DLnDRwvay6Zc8EzkXORPBUHseC5U+82Ymn5HzrgQEzb2LykiM7FgzMtSxFeYd1bzFTcVj3HfcYdmLBCjMCvG6VpVIsP+LJOR+A46k/5XlJMxA8uUc+Mxa7FDJoP6TZg5v1dPm4wkUNrJnpYDM2syP8blhedFjyeHvUPuUzK8CsAR+4+7r2eTzLG4QUZ7/ELd4EzkZwzaAIH/oL+i09L8cDR+V58WSD982SdYccC5wi+LHUCFycCWZVluWA8DvFiQKOc6bn6B7IM5hnsMovHdthL56ms0RxkbJqgMzvm70UF1XmM7xtSsu1zi7rtyxbMpO3T7MvR2QbPqbHXhDqgU3x4qw6leXEi43pzAAxa3FMvz/ehHWL8A8JZa9OllRJDcvFydinfJwW9FC2dcR5aHeHkJV02nS64FDxjQ5e0etyKZ+DfHDjxqu3n9MtI4DSr20SJq5y2GngbXYssWTJE46F0jiTzF6w9AknjtkImc9l5R7CacjfI/blIB84deBT8dQGGpdNqLE8NNqnRe232Gfn4qheOsHrZ7mLdVSepqtxwHkg+2pH4mQIPvx8JH0np8cLpOFKXqMVOHGxSrZPA/Y4lrg9PRgJH2NHXU5LN1W7p2l8s0wiDvqOi4qd2QNtrdMKQAY43w2OBX0q5/yYW2BugW+eBTSadF/2zeDY2rVXyey6l0LR9KZQwmygic9qgMnPJtbx2sISTz4Jp0NscXWoxD1bQQergUzbvK2ON94KVTdyqxPOGQtmL3gKuq4RHwOAdCD88brqWADj+xaEXirFAKOe1on+T6NiaeDBgB0LPUHklbMrGvysadDFjAWdGk9gzUODleZAKG4HwjCWhtTZDJwNnBydO3lCKPne/M0AyfartpUuaRvqNfOm8VFdNCuDRU4eig2JBDrMenFxOyTs7kMZDHo5gCTcHXqtbFBjcKNIJatB44j+SWtm9dKXMYlDNBy2ULrKJTflgJoqO9KTJhIw2Tn5U7drukdYv+89FAqXlLZDkTMVguFwMIvB7ASDVwa8fMCO5Tg4F+jJa1v3ijebttlEzZuavCdCT86PMyBXfUMPnE3dOA1824JZDWZC0J9vRrBkhwSzD+Aib4fC3QoZHDPbwVKjA8LFeWCgDh7fumDAz1uTcAYoJAN8lkqhIV8Df56vbcuRwQy3yqm4RcuqjmnAjly/olZw3uD0jF4j+7RmUp7WEieWQvHxvNu0t+OAynJQaTaF8zvFbujOFzI4ZVnZA0eMZWHxNF3iXUd8V2NFHhN7UKgDZgsO6v5nxoKBM44Fs48X9OT+tOQza8EHAq9s6JW4kn9C36zAwdkn83Af8bYoHBUPK8UPZy32u8i5UppX4x7XuV+/Tb4nzjN8rry0ge9u4FgwwPbvShbZpd86TrCG8LYbuL5dFO5UBpvW92nGhPIf0NIqNoibXlyjxsCv7ZyI6y0qag5xks4cOFEe/CtcxYFQm2KnYkXfoGCmwg9GWOIU7Z2XOYkUcr90V4z9WxPAuksv9plwxOxHdUKUP7SlypON4zcrJ1hl8BvydB8t6E1R+w4e0XlUm7j3eSa1/01axrgwtj/yqIceF9j0QCcftgkWrQcF0sH1ajyMN5HV2hKxpA3Oo8E7vpR9loxaWybdikfyva4wiiaSFsk7o7JB38zLMO4ZzPLdsBTquuw5R55bYG6BG84C2zoW+YG86FvqoNddaG2KlTE067PLno29m1C1nKQzrhY/iASLp2zqGBOnhtnJpmOxqgEIDsYyzoXi+cpZZixyn4UdC3UozFiw3IEZC78KVoMNBv8MbK7mWKAlf37drN5g4U7fA4GYseBJMt0aHT3OQXMqNLDzkigGLzpxJHAi2jIppXPWIjZyK58nhdiCf4cMV2rcMeKcncMBQEc+sWRgBMi2TRqHgUe0Wj4QOxww+pqkDuLw0K3h93VZxVte0ItCZIkDfd+hE+/zpvnkWX94RKauVY8aAOYYShTpADC0axTOcJr7qINTNMrHwJhXvC7LiWCpE04CA18cC5wHBuUMXn0/igN0PBlnD8WLuteYcYCG/QB8RI63IMHzkhB5cs6biU5q8HaY+0D34GXReQZE+fCGlg/RESqpGQicWy0ZEi0zETh07I9Y0Mn+gqzbXEaEE4JjsWR+6/qIXQy+cQR4GsjMBK9qRV/e4vSCZgJWBOOtSncfOaSN2/v9NetwKuL3x3IoXo/7AjMHGtzHIF0zAAzqdV/zPQye2GNR6ktqablT7G0gscyLDlQm7HpBNmI2xh/4033LwJjB/4bKyWwFXwRfVHkZfP5/7L0JsGbJVeeXr6revu+v1u7qltRCCwLtAonQAi2QkBCIZQYcNrYIGzThMTgARzhmmBmH7QiHGWbsYDxjeVBgQbCYxdKAhBZraxAgIYEktLTUe22v6u37vvj3O3nz++579bq6uofRRLferbrv3i9v3syTy808/zxLqlblN7shoOIdN+ATgPmOtidDxFXFqZO8zD0MvUnLeiOIOtNAPQMsDeFdUOgFAPieQEggYN06Fmjfcgp67NmCPQ8lIQIQ844TWkzXb0s1MSU1ggq9XblRX492G4RhwhF5BWvvB+hBmvV+T7IRZjljLHPhg/Jtb64GmNgLMJElonqC0gbCM2wmSlrQG0eVhbS5oOFiiGNCzsI2gZG3XQyAduun5CuD7YMY/yiHu3C3d3Slju7e1Nndj80FKnHUdZ32+n0m4Mn9tYcELZJzG68G2cYzsj9qR+NnSdBnxKuPW/XxpdBeDyvERD7UTxyFsCqDA/FzjPhbDy9p1x6TdNXupGO5PSJp/jTfjZx5kuulQQI3Fsu+/c1gvB2Vc/znuAaOa+AZWwO3BhbVBnkOgHkwdagsp7fe55C4OfSnOaDmB/6OIddrGYh5FOH8lolzBbP89noTsID5KMBiixXOcD0rwDgCWMicyfgJLJqb5AkyAAIwSbeSWEip9OypB14HFiG5UFVBYCHzkVWpAiQwYTtxlzPC+O0u3NpVZKCBlIJ7n+k1SpCRJRcV80CdxmTptarfyIg//i5h5T7i+sxIuTm8axzluQFV7ednVdq11sz5RrxqMqT8JT9u4j3bpBxFDamRbW5cM8qH9JbIh66NdKtw0zULk4/Xy4slrdr75VEJKvFJoZF19CUeZLsJV8VzurmPqfZUAQu4L/dSUGohQy/zv869dhCut5q2qVoSmV13p9bo2o3mTMOV9wawIK6M/i4E9sLMuqN02fVahtfVfAupJEJJg0y/7mFlcJV4lDqxJ2TjZZhwmUd+KzkpwERdesN3eCAQkbpsW9AWDLvt7C7X0qi9h7YQSgFaSWMQmi729WB/0YnbVkGtDKj9PDOzAgKNvTUKN1+Z8FAbkjmHdqmx3b13JV/bEdWESIIyAfgtp+CAOpoOkIKxeDDBxoOFJz0BijuHq7ZlHqpgKf0QkOiFbZ78Q7rBUwGb8ZTG9ECDYIT/TdBHvuYtIJyjrAsw5zRfSBbGUOEKqQzvBu3kHZLJkK/kviHzLWuu1EJj+WXGFD1y2Rdsf43PB2hjdxfXq1cf3qYEGP1c9TjVDvoTOEpT/oJzX7EvW7YYyyyXUhalFBpnq/qElEKXsbqO3SZPx7ItvHsJTneJzyWnSZ1JM2TmgnIx3QAVhNtnLL/52IbxffJbwKGcIiRC1E1ILYjjuNeuKh1100EfCHUovER1dGNngWcv0y1H6Y/l95O92g+lrRzS/XhHI1pVb7nAzdj5OX9LxEOJ+X1G2atXyshzYMwrxNg45Si3Jl2el2e1a3l2VJ0cfnbwt3VQiC7XWsLRmvm3pPzAN4GNRb30x/fHNXBcA8+8GniSwKI5kR2uisMDbnMwbcZ0WG2ENwbb/NxwJ78CLDKzcxBYNIy3YUJC9QkmIsAFk3YGFllFyngaN8oEucIrsGiHMXDzujiZXJ1gPYsqVPGKEuVwsuafE3MAi93sNz7cQKIiEsaWME9OXsZXMhISCdIvkokCIgQVcQomBBkyCuZreAAMacjgQn3xfOR0S52WOvP3TWdtYqrfVgnl+NV8FumUiTRor2LxPIKr9Et+Xkv0QotvGB5t6Q8ixLNGHrJj+bB+PEoaObSKX37UriVfgwK0lDTJreRfUvRap8P4QRM3UlAkEtpN6NlpG25Tqxh/xzMZR17IHqAMA0xwuq+ETK3AQiNk8wmmkXu9Jk3DcKsqJDOstyAZ9X4YMpnvDdJfJcz1bpngURhigYWrz4IQ6VNCIJDQ65JpuWeEEhKlA+45IWDo4ZSZVtrhirslk9HV25TMs/tleMgEytTGO8RTdUmQo8E2OjcBROb4RpQAuKt2F3momqVdRR/9T8md+1Io4bBFlYzEN2gZOOMbJB/rS+mNDLtl2JNW3u1FIqBbW1fvfaKURObcze8mUb26wvno6now56of9WMwHBJEaLYeZNC1AQnpDe8E8IIWjchtM6ol1wl1GXtqoLqV98jA7W3QA9VE8ms1z0VsM6b1UgXt7ZRvDCP3QZjobu7JhljGzGDJb9Zqsj/4RNa9GJFPA8IEZYI/+90wdAsSBRbdqERF2wDKdGurapR2G0o2sk0J7UKKps1/6KMcjEfaZbkvxQ6b23nusVixy7iiBHYDAOb4tUV/iDqHJsvlmOT45HhBZ4dCaOVqyiHl5HmWWNBW1JfS1bDHIIp0m1bYsFiXloV/jnmCii76QICLru7UgbRClSgNuiNdcvAo35w55iPfWa5cuhz6eH+D4qCXGBVNxs3jSlVBNkwuWcSJSsuRgt4S32scOfPyK9Lyh2Ur9JpXOaLOyo9bXJtjT+3lBgWZZjI4MoXybiP/Klb+DQW2XdRDPe2SVElT+o933i61cnw9roHjGnj61sCtgcX9X08PPnopSucgmQdKB8I8GDpMNobFxxl0fbkMvMZv3JcJJ0eI8DypwuTwrNyHxIIJs0guVJnYZqLehFESVMSO23hPyXtZZGARPt+ZTAUWoQ7FZNoGA5R1iwEX/Pbes2G4Df0yaq5sOoHknbcrYMHkn1WhdAWpdyi9TxWJhSuoMBamxRmuZisQ0QgrIEI3s+Stq9kMMLjyOwMLn2XgVuo5X3OdRV1DV72+Y9WyEUJFBuk5RqnnaLdGnEMTWyO9kmpOwybJk7OsWPWMNin05Md5yi7Pc8plAiVGpM27jXaOEvhqptNrIYe4zaNKiffMvWSfY9gH83sRK+KQDGFhNAy/vQPHGXr+tL/MOOYFARhk5uOk/2gzkdn8zCyHahLxBRTq73OJo7hLNUtb270plABM0/4yjHpqGgNUaAytE4AV8ligr25CnIBiAkZUw23zkzmXePdwEOBoYzFJWlP0I/ee0GN0N8/6AJ49XmHWZaZ1K9sBM+uh6pSg5gZ2CAIMV/4FH67+y9iGBIG+p42BKlQCBvNZoUxLVIQARAmDq/+WyXLLQAuYrE/zC/sB6PS35bcVtJlwEz735xB8qc4kYNeAXGDhTtv9MNgy0zaq3+jl1bX0yMpqemB5mTBcx5LnKAytIMxyKuWQXo8AWtAhqJNm1cyUnKywgm9jSFMfAOEckpZBvGcJSuYpf9h4EN+NAEt5F5HOCFR0CRze2HhXZtk0LH/socEzDd21uRBYmKcMuFKeRdrWncWvh2E9jDoV4V4eA5zWddhb8J4SJSU+RS2qg6KoJma5sHIh7ww2WpDi7COZ2EVSsb/NCaDYA1Do9ckds9cAX+5zovqTjRLjAnT6rdUXHLJ7WeiBRvtTfJnEy96cskF9SERoSyUeRUUqACJl8zDMMa8Db2Fd7DnSBBaoQ+kZKgy4c945fm5/7+0PHtHE3pQAw6C1HGXcafw2YhW3Ea/2biTYfL0xXNSCSlL56oPqfWuh3PuwkX6O2fjbzC7f+TfSL6/z3cYRD6r7CGhSke/4y/9SxnK9Od+cRnNshNKSR84p8o+wSLiKyf03wwZ5VRUcX45r4LgGnqE1cGtg8dWvpQceuxxFd+iLATQmkTwQxjhcVczNg2uzxhxAy3DduK8PtD7nLGDCa/yWQWBSdKLMbhfVP86qUEVC4a6zgopNVAq0u8geVZiAYVIKsJB5VxWqYbgIc5VBBgwhk3MBFMGoUz7LIrPKOl/8C7sOmIJQY4Dp2AXQ7MD8WKhQsSCNAAhM2iGV4CqoaJyqPIVUAvBA3uFuFpoyE5F/x8okDFDQQHoQkSc/qrHUnTXqPGTdxHPiCJxKRH7aSvHMuOWwPL5309EIrz0tt+bB0Xiz/DYTjmaLlheqMF6LNwkuT6pX4z3Dcsr5WlTTc+Ty1L4Q0SNuob9czRteMPcXoskWh8SBfhIqTVxD6sB1m4YsHp82eUkGTmbLnGS3lEyEByaYZnexVrLg+rtMooyodg7G06Wp3p5kerVtaCe8l/bSJWsHbaqB8iJxlCq4Yq704AzMsB6KLL/Mq4dxrRnd2LpfxST9dx5mUGZUaUM//UTDb5lvV8O1MdAN7SkIkamXYb8MMzrFKrcAQlCha9nMOuoFKYMSGWkzFmTJaHsW6YaG3jLx2mDo8tb9I6zbQfTu+2GgVT9SBcjyGy6wmKTfuxneMgClHTq7KLdG6QIg3dsOI03QLayMtQbP15RWACq+urgUXrbMu5/0h1gVjz07iKukw+/PmlH9SvuGbfLSLkM3vEuUUS9vAoleQMkEe24IUGxD1bukq4tnFzAu1/ZBew89VWU7Ge1dsmqawMWyaX/ijuNKhgIAQKv2GqrCCVLMfwGGX4nFNFelJo4hY7Sj0gpBicpbSibEekqGArDQNqpo+dvTvpOf0Y9Q7ToBmGjBOBsYQ0fIHp/W2LBwBeC1iqcuJRX2CceALEXNYOgk7VqkmC5kxOZ5XB17AlwInmiL2NuCMkYcyuF4aXp+QrnX5UUBx5ZW0uygDjqpz07ARacSiwOqUJW9lylQjvKdm5aH300cJYAfxvOIcSnuDv4p4SVeEGaUkli5Gka6Jl0PMrhx+KCWd9BX/W6k34icb5rRm3c+8buMo3HjLwNvzj2Pqz7lefXeTeXKicXf/Kekw1sH8qiiGFZFKePssSpUrfqOb49r4LgGnpY1cEtg8fkAFpcY+/K/GAWZRMoAXoZNS25YGTzL81IjhjsWl2s9PO59Xp2K7ANgMOEbliUVAgsmzAAZMH8Aiaw6AKCAAdmA0fdUPapMru6M3QJTKePvhB1Agkm4gIt2GIU2JtkCLKS5gIQoC4WTaqHFVqwwZmCxC4OVwYvAItYNMzCR6fAkj3J10g9vVBEmqFAqUU5BRDkzuMk7cdfARamo6mp9mKf14yGd5pWlLLZOswzVKzkeDEU5StvU246EyuOo8/hRzXklvvk2jmb0yLPxhDjex2/i1KLl8PLAeuXeJPMbOeWgkpeMlvuA/cIyNQGcaWqkmqUAAAoScdU5GGjAgYAiVJ8IU2NIhts4Mvsyj4IK69HV2w2eKwVQHUlm1PckSmZRhlWvQh5r9DsZWfdPUAoiY60kQqlEN1IoVaY2oGmeFfZ5+yB5y8ieRxVHplTQUNpMyYIMnsyuG9pdIV13zVYNSVepsSpOejLTsX8DfVSphZvKSftVjLAnYUrneE8JgwyvDHqUG/JlmGX29Rol9Zbf9wJMka7PtQ3QhkKw8ChuaFUj0sZDiYKqQxoqu6LfxdXvQ/exj6yskPdamoPZHyae5yAgwbJZVwISd+dWuqIkYIEV/8u889W5+TS1ukEeFUCAqR3gHMQrVTEKl1CBhKpU27SFbemGhKo2ueO5oKSLd/QQZb9RKjFP+vYXXdE+mz05xliFt1ymoZqa9itKl5RsCFQuoPpzJ8DkIvuACA4EP3Z7gaQgS0NzadDQXe9Z/rb+9JrlJoGCOOnI/dI6FaCqXoTKEvWrO+AOAQblbyOsm5d7aNQ+Ntns3N/BvmWbb5XRhIWRLehfWlpKywALJRYCgzJWmJ79I75r6lMi7a8ulDi2OWDrEc+r0jnjOgoZJ+LF1bHTrmxqHNASbrXpr4557dSlEotO2qBTl7NdfakdI+6wsSCO40jjsJLiMEFuaj+rB5CYA83/KR8mUXv9cFolj5K+vw/HKc8OhDfoN30yKPRXP+OdQ3Q3yl+jKcap2ruPR2s9u1idMp+IXCtczrSiJydakj423o7KOf5zXAPHNfA0roEnBBYPPpaBRYzI1ajpoN4YfKvCHx7463XiQO+w8zRmpwAAQABJREFUWq4xwPu7ihQTgXE4m8Ai/xZQhD93J1aYgJBYMLEWYKGhaAYVqBbI1LmaZ1xOV8PD/oHJ15U67Sv0CiNj2MZqcHZBWyQWMGKWqzqdgFSHCr38SnVBX/O7MB27MB9O8tJrPagKlSURXJmYlU5kRiFLMFRX8Gxx0gZMZACR42ZwIROYmYBcJ2QedDQBQdRN1JHMb1RmNEnkY9kok7TLCNbVKazisuoZ98SJq39ILy5VWPyIgEgaiiQjx6+CueS2LKF1uqMdjehRIpCFzKAMWfzj3mwNk9nNDHdm8gRI/lNNSW9NMpjq0quS4x4C6ufLmPuOq9Z6ApKhzleNrgEVvCfTzmuZBG68j3eq/KhB2lX3p9gFkIcG1KYjkxn0kJ+06OLV1V9XsmVkBSiq0XXQj1QF0i5CAOCK9woM7Aor3Xo48t1BQMV5GNkzMG+9MJtWh/UjMyo90rgOIXP0VyUmtn4PaSotoIjQQaQot3r9AFTykcEXUCyqmgetAhHVoawnmWjBkavpSktUeZLpNw37RGxwR/rZxoG9MkhHYHEZsGD8YZj2cZhzy2MaQWRc8t4bggrtFzSOvgBzfg6vUq7+20ONLxCxjPYX6bfOpnjnAYDFpdn5NA8gspQ9vNePu9tR0tBGQ+NxjdKVFvl9+67p2TfcoNDdtk3Z79b6ttyqSQk8lCYJaAQMo7rEpcybSjx4bj0pNREI2TYTAItz5HknGwKqxqRKUxvEm7dSDg3GbQtVw5ao31bKMkB6Y9SLQM32EYAIPgOkWVZOaVY10zEqXAIT2EMPHgZUjCNCGWMH8G4aotUlCtSisgrURlpmF3TVOO3fbeRhGxEhVC0Vx2nw3kK7cBNN0fi2iO8YGJJagJdqZ3m8M317Ngf1V1Sh7EaxR47tKuhDFU1QYTtoZ9ElsOgpxtu0p2NUlCxSivbMd/41NY8aU0956/EbdOaIT/lvSac+/phYCT+ccInnc//VaToct/yO0hD/8Y5Ik8d5NGnGsi+YQz64byThOFbCeVrCCTpMt78jJn+8lqjH7maraj2+HNfAcQ08bWvglsAi3M0CLGLUc8CsBs3MJsWw2Cj4gQG1EZpvyqCamcjmIFsG03juhMAZwIIJ0rg+VwXKsAwwBBasaAIgYh8LGAHdzW6EGlSWWKgmYKEKsAgGUYYKBkRpRfPMUozDjHiUw6JR1mJnsYP6wnalCrUDc9UAFnB/TmAy7ieYtGNShiHIXp9IH2YoSzDytYXfYWNBfN+kiHGVBvOVGQhGhTJoQxLlqJiFUjfZpWSejGQ+2mDuVGno5vS+DQAjyPAMyQlll1urt0/kZdNUdextHFX7ei8oKy1sfNvCs/wtz4wVzw6nVcV09d5Tpl2mMH7zgkx6NpyW3czMpKDBvGSq1fuXiTOewELvO6qiyBBadzLm2g0scGrs62rzhvXGe6oluWouGOngXosC+4F0mja7kCCtkFEENEBTbHpHejJ59jopcqXfN7ZhbtW9XyN9mW7dFNOI8VyWV9o2eLYGqNhgZd7+ab0P9nWnC6ykn4N5U6pgWpbfDeW0bVCtyXw1+LaMMtPabLhbNZEDJCllMY9w80re0rcOAxsAh7im63tKU67DROvNSEUr7RG0+7C+lFD4vtKH2MnaMvgOTO0UYEnXsndA43lOd//OjHL20CRQs37cUXwGKckCcVd472x3D3thdKcRQJNpKylwRd9D2nQ5G16aeOfq4mK6DLCYQYVrnbrsBFQMwuBP9PVCj/t7YJTO92x8vULJ0Nt2flMallPkUEcKlT/uAyBCkw2pu1f39BgEEIXERDsP3tC1r0DwCsz7FPWyDLgY5Ps4AwC5s78vjXRkCYSqU4IUVcKmOd1oUOmRIGdQUIE0ZEKmn49hnWcL2EQIHM1e6YW7i/tdOgZp+9BGWZRUjFGIs+0nAZWoqbEHh5v07dPu4f2J+lNisQF9FIHFBtqFPHgN+wvGM/JQ9VKgogctPgiuuSKsizIOZGDBrufQte2YR/qOeXYexxEQW/QjfnELWKBOHXu0Netik8Ve2k9g0Y2koqu3L1zOarxt3/Wd+tEcO6Qgt0njhqj1+NL3d3HkdCzLzakdlUeMUVXefsN1mm5OoRlyVFolrFnuZvxa6euBMZYf9SzoiDI4v+VX8lUaOQ6V7xhYHKjW4x/HNXBcA0/DGrg1sPgaxtvYWMhkNkZ4Rvr8z9LmiTXfNeOUgTnCnRjLgM/VpOIso6y/DfdkxpZxKFILw/Lus0yycFXZw0lWhcrAIksu6jYWhpfduaU7gAWTpV6hZLozsBBUFH1mJAxMwq7yO5E0JhPpliEgEY0iG8AC5moHxkDJSClXAAsnZCbvYOJJz8lc5iMmau5deVRi4eRuXjKZecM/dLBbpCUDAZmDVVZ6FxeXQwdbyUzTP731BFdDuaTTHb87ZRJYhR3o7089vTAJ/O6EQeyAWWlv78hMi4wa8Ut7NK7U78GDONaBGVSPGvMeN9V6aMyFxiqRZJDiNDFfrV6yLWUWXdGX2ZPpVx1JQJH3csjggVoLpl3G3faSPQopBHVk3BOkI7BwNV/vQlmVKAWocJfoSYyZZ2FiN7hXR1/pkRu69XEO+Q6nevXWQZYUAEqgI0spILii1zLJwKrxbjxlC0pG9LjTQr0r6ZJpW+D3imUgjrS7L8EWwGJvZYPC4+EJhr+/vyddwNj4LIy03oOsC8u+ChOtmo5MewATaIsdnQEAMtWSohKOBteq4yhR6SDPNvqMtNksJ6kL1ZVk5l09n0QacIX+MgcNsve9MPx9MNuqY7mKb70pqfB9wZv3rszLUKs29Fz2tbgDpnuI+IIkPTut07+1KYkTOpa1xQBUr3IO0sf6K8nDCMz3GIz6BBID7TLcE2SedhDkLME8L6+vpSnsLBbwDrVEWBvvDcLUngFc2NZKa66Tps/sHyeoVw2fAxRTj0pCTNdvWWBl/xR4dHMVZAqs/P7sO9aH9iN+IwKGxxaW0nXAxSK0d/ItjPOd3DHYly4C+DTGtl8t03YCsxus/l+l/qwXJUCqsI1SNr1oCRzmsYWYwWZEqZRk2Bfc20PQ4Rh1kve6sacYokNc7GkDVHakM5ztLDiE+hPvbXD6zSsFI+sYD/QOp12FHsYy+CB/7Ds8BXlxUEa9Pflb4Gr5tq0vaNeRhDYaghHtUSJd2jnGIa5FraqMPS6wFGDRTVv0ACq6e90krx/Q0ZHHK/I7fOSgHA6lUQcRh3LYjuUoY6LXMuY0nvmW5T4Uv/67xPVa0vLeOJ4lrFx9duAg/fw9Gz8/KXHL1dB6nlEa3+MwTvldj1PSKnEaJY4HVb04Nlc1Yzr5/Ux3M9w8TEUa8jWeGUjAMbAodXJ8Pa6B4xp4utbAbQGLMhnEOMjg5798NAf6oyqgDMwxWFcDrfcx/Fajq2OsYY62BVhEHCZRB/gD0gomTlfmJDr2r2BCDYkFzFrZHM/fAS6YgKVSP/z6z1e3OECFakNM5OG1BSYkM/QwKMRxEm6M9rwsbXswDuEW0o2sUIXSK5SMpHYWuoT0kLGRyY/3TcOTR9K+Dc2b0OkpYxBlJW13Bs/SF9hoym6dCnCMIwOiUWd4i6E8vhM1Xl3jN8yWIKWVVca88ohqCavA3awmyzD0ATKGBgfT4MAg4ELj8QygpLW0S6Qp/RajKq/lOXxIW4CFKp7Py7vey9zqzlXDae/L9Kp0QiZNhnptm3LRbqqTyNArkZB5lnGPVXXo0vBY2lyhDSlCpE2dkY5qKz5392nrWeZ7kTSvsBJ+dWklzS4th1GrTHd3dyfMKyfMYT/tojGtzI/kuxotY6+HJw2arUuNlGVMI33qycMy7MAoWAZfNA0lRNplXMeWYoF0BAeaHu/D5HmeoE+odtJBH+vr6UxnYdYnVDuBBlVuQocfZlvbAevAcgwAAPpgYLVTsB8IVmS2sxcm+jL3SgWy1CKDVYGI6lG+r1THVfmpBZhewJWHqkbD5H2WOnCTOIGV7WJcAZN9XFUhVYaUFJwj/hlUt7R5kA49LEnLEu2l9ynVpubIYwHAu4BdwD7OCLq0s+jrS+e7sx2JXqEwTQ61soUtpIrQ7UJAC228TZlDqsM34GaRgvw+QExpDw3iZfCVDgmdYvM56qz0WeOpEuU+IkE/AUpr3ENE0GTZ/HY0eNfNrxveudhwFVqnlFrQR8g49bFCf3qgLz0HIKXUQrsIwZXlu45UQ4DjJn2qkGkUz97gYduwBThaw15EuwjBzz75tPO+KnHWVSv9qHdvm00KTyD5QTVrAFWvblSzqJPYv4J6doFgB+maAMEuFQf5u+DguMcD6py+RJhxigQi3qcuHS/i3rFPiW2Mc947FvIdkb6LEjFSkEb5xv1SYzHDhQv6jGpXqkIFqFAdDWDR0zeQOrm2Ab7CyQQ0NY6qv/i7pOlYEOOYgZTF7lKeRZDv1I6IayRiVpd4r0Rpxs7joOGN9KtIjfStM9L3nUYcfpfnpt9Mjx83BeQEG+8anzhG84jwBv0lND8rfyXBw0vOq8SDrnj3AAURt/6nJF/SiUSqJI6BRb2mju+Pa+C4Bp6ONXBrYHF/llhYMAfBmBa4yWOgf28fWJTKceAtk0IZfl3NjfCK0fO+bPYU0gvCnTRdkSvAQsIFEU7YsadFgAtBhZOuDGtRsSnAQjWobGfhCqGnK5/hhQWGVSZGRs0JqtDVUIWCaXAzqwAWlcTCiVzaPHwve33K6gamIfMQ0gdWS5dhbBZhcNZh/EICwcMoYzAQ6kjLVFQTFWWXflUdBEiRB2FWujWf697f3JOvaleuQrYDMFQz6WTTK1Wj+pBgjA6PpPGxMVbPYR4AG3qD0SajgIvcgiXNPEnaOqQeR5msq585LBMS98aznFkiocEtDGW0XWbMA3AQQXCxwTMBRoAM2lE99mXKKeNeXHhmNSAYLcqW1ZEyUDEOeCGv7NNOygn05iRTPAmouMbK9DJXEVInzPEIzOMwTPUADLtqP7mcuRQyy2UlHjYuJBm6d5VxV1Woy74AgFBqIhhQYmI5BR+yPaoFXYMZlfmXUZc9PsH9SU/pJJ7SMKVGxRg6JAS8o3rOPMzrqswgcZVsnO6BCUXC0E/e1lN4n4LGGeKqdqNhsAy03qSAXfS1LOHopN2t6xXLg8RicXklbcIga7MwCKg4zeq8K/MaNSu1ECSZr+5nVW+6Tr+ch5m2DGMw3NooxA7XgjHosk2UmOh+1RX9OeLP0X/n6MsrMLmq8AhgzkP/CO9YPyukpSRoGYBAtQfj3kfDtVNHlsOV9mAdeSh80xBecKZxtTYT9helfBqraxOidMVeL6gwziz1IRjapi8NUb9ugDdOX+czDilMqGXxPhlRLtSbAEPT0DtD/bhxoV6Weoh/kbqxXgR8buJ3HSDvucjY4bvaYHRAqVKDZcDULpKKbc6t1ZUAvCf43lp5Xxp1M9tD3FEMLC70tgMqupGMAPa1q4HuPBY5fll+xrWqP5X7kNBSNn/HFwj9RInvPo95GUjsEqc4rwiQQtto59WUVvCbuszp0OepukiRP3mDQoCFQMiFCOrA8aCXxYc+wKHAogupRVt7JwbcefftxqggXSZmS1TXvMiQv4tYlIintedRliqQS47Jc/4ddUhnprb2NAfWAprpN2KTTy7vwWcHX+LXobTKO410IKtQ1nh26J2SZlUF/Gy8UR7FNRfdlx8nAZ9UjyKtEq1K7hhYHKjO4x/HNXBcA0/DGrg1sGDn7Qc03nbg5U/841oGxubgevslLwN3DL1ODLxqmKdMdLkv1wPAopJWCBqyylP2CrUJ0x/7WQTTAePhpEscx2pXqgUPSiyK7UETWGSAEeCCOAVYWBopEzaEKlQBFriMzBILVBpguKTRw1VqgUWAE1dMYf48lD4swdjMLSym6enZtMB1HQZNY8+YR3hdACVDEAxHVQ8ma9oBcaLepcejmprj5WiUYMJtGxlKmYJszM2qJOomw8PDaXx8PJ2ZmEgjQ0OhLqXveg3ZS1lN0ToyyQB4UfLIrMFI5F+5D3hvftID2TDDeU+GRVbxF1mpVsUpw60muJA5t81U/xIYytzK4Gs0az3LxMoUSsceXIst5+q3zL1rsDLhPlOyoTrPFhkobXCFex6Gbw5Vm22YSMvfC6N7egBQxQq8rkmVUMiYSq+toqqLm8ZJgy5B9frjJnRKLMIwuwYspFHJgvUijda/EoUZ3jV/GolN57BzIXXVmCIGv6XRtLqhx70eBACzMK4znGsABvuF4E7d+vMw/0oW1OlX/UvQMwV9U+5xwL2ep2T4rR/zdk1f8CI9MtuhckSfUhWLjhSgZoyV6LOk+6zeLiQmGI9TDzawbmU1vhYoPIY3qGnfg+EdAVgMwSgrBVClaRig0M29tiWCm2Xi6y1JFacMMpBA0CZ+TwIjpQ+qmoXkAZrcJND6dDO7MdzA6g63D0a7k/oWXNjfBaMCy1D3sj6hQ9sYv0XVlGTMlRJZ9wI82+0a9E6j9rbEKai4AGN8z1B/3jSP+nOhAHY++pk2MQI51cP08qSELO+Kju0EjPU4wMT9PCZ5rstd9wcxL93G9pP3rqACIDG/sJBasBNpoZ/t0SZ85Okk9dPG2Um8Hhp9EPB0Rx+AZbgv3THaDwika9DPN8mXaqJ3uGAhZfm7cUxrLpQoxfCL8XvO36D143PpCU9QlEW7MvuN44TP8phB3wawOhY6kOfx0zGUpKg7v608jvKb9ggbC8rdJahAstmHhLO/rx9JZ3/qUhUKFbcDu29X6UBcHOW7l87IxFA/rNpR8j0cXoty020ZRxsPIv3GryiD6XqUa2RrQTlirKyeR8ChPzelX70T5TAuSVcja5SrFKkRVsUveftKKZ91XI4m2YY1w8tzr3VacomqpxX9x8CiXlvH98c1cFwDT8caeEJgoVcoDwfV5sBaHxLr4c0qqA+gJbRMADHkMik4ccYQ7LV25gkyh5VJOCZiJtSQWHhlIi1uZzdgArIqVF7lL56hzFeGVMY/dtxm5ddVYpkXDasDYMCQxNWJl9MJ2EN6wniUJbkdvLnsuFNuSCu0sQBYMNEXhiADiyz1EFyYnmlRYWmDiX8JpmR2FmCBh5w1VkBVX1D9QSbLNFR30XWkp6uSMg88hO5c7xAlRRHuI+8r2JHrL2ZGwUWm36tqDzIQfag4jI6MpLGxUaQX42lkZAjmuyc2x3L1WyY13osyV+ACuqQ9AAfXA0f1G1JhCvNmZjc29aYjowhDFKTbtpn5l1ZVS2QqccxqBtVKtcykTDusMlmYF0UOJnyV7Jetl2rmd6VeyYNed4wTalQwiqf8QT9oQfpxijpUj1yGb5hyD8j8wWjq6UcGfCPqtfIoRHz78jB9QaZXtSLpjt0ECNc1Jz8D/MiMC57U5Zf9M84GDwU54WZVIMlv29MV92ARieuu1qpWeSwBugQWeifSQDu8SpknaQzD0LvJnJ6jXPEXWMzAxC8T1xXvcZh2DZMFKwuU1b0ZzEM1niUY/eWN7G5ZoOEO0AIQXbmqgnUHakrDqOxYPnuMdgyqNk3x3mVsHgQK9jU9JhnPlXqlG8ZXjcdvQFU1HQloYO3u3XOocelWV8Am+OsXkECfnqXspUqX9LKlJMLSm84ZGXnijbQDvq0jngtatIMIYEGZbdNsVwOllM1vKOigXkxXo/crfDtT0D2HkbySiotI5V4wOpw9QpGPkiZtejx1bzvDuKAk4ipSIg3HlZDYZUbpG9brCHTfqJ5fQRrj9xDG49hKbKD2tLG4kNYBFrhwSif49k9Awwn6S94TB8kKBRwFRZwFwF0cH0oTA71poAsDeLzIxXhFfhluWgLHSa6UzW9C6WsGCFlaEWMh7R0LJtRLSCshNoMM+iBjToAI3ovxofosHQfzqWqm5cvjhNk4RnjE2MrV77wVdacuJE29eILqr4CF0s2unoEAFnqNyuNBlUGkcOgPjw4w3eSUD8vYzC8q2wc8bs4dVdTqIm1HHSV+0E6c8juoijrMb5X3fV7uvZb49Xvf8PfhsCqlfCl/qzIeJq+RD2THKFyjxVdNO54EDYZEQj7wR/U8bmPMy3f5rzGOgUW9Ro7vj2vguAaejjVwa2Bxv6pQlxwn43BQzQN2DO+NsJhmagO7D/IAG1Eaf+oDuvcxBHPNk2oe8CPcsMYkmZnvmDxhDuMqU8OEmvWMsyrUBszYFpILJ2SBhX7fTd8JMFa7K4ZAF7PZcBsVIpiXbGvRBANRPssS/2TiYALwQ68B9y4SC/ex2FZyQX4xwZNHMPIy6aSXgUWWXsigymyGzQQ62ivqaPO+utL7lCWXNRtkaosRq/oyDtJO+aO+YQYEO5bF/DQkduVSxkSViA2YPFd/BSWu+FZN5Uwe4EIVKY27hwaHABUAjPFR7C4GQg1CfWtXzVURU9JhPfleTJjWG4yazJatndsqSxF2+LFKflMwmZPrui3dCmDhqrMTqHFlqVwhFhDIsA7B+HWTnivZsRLLMyUZeT8Ar/mUKV0mnRnaUOmHiekzJxsfw9gTpgqUTKKqLF3wap0QKBMrEJRe90lwEzbVaoIZhiF2pX5d5ox+I4W9MLoTxNP4OKQBpKcOv1IB6111Dw3OVTWSSbYnWS6lKAIe95AQmIzzvnnLzBZja9NQrcuyk2zo8YcbWhLQKLgLhlYDc1WSMvMMICLuPHkFEKJdfVFD5LMaYpOXFOi1yDhrMMmnrIfo85VNgJIJ0hVM9cEcakPgu4YJgvTKpXRjGjBxlXqQoVY16pQ0hRoUdBHPsnMJIBVSHH6f5F2lPOZ/g/fdf0MQYPtYd25OF8DBb4yXZeqVXihpsM7cbVupi6c7VRtHw+k1wIpuafOpDQoAjDyWSHuP+usiPcuh1EYFuBsw/6o2zaLapJTkHKD5HoDymOWkLSy744bfgxIpgchlJBFKJOyzFCskPePQMQLN2t9oV6Fb2mtcXSBogWa4+LS1OJ92kITto2LGR4tROXtRQIPjhfR30ui6kj03gAvbkYF0enQwdUOrNjeCgDD6t/DUH6XlG3Zs4BujLIaFUwry8nt2HMhqToxp5k1dO76Yht+6AETph+PIJt+cYabh9+q35HihgXiMWTwr4xK5E0a/BbAQDRr0IteRusO2AmDBuCCo6OtXFWoAu5GucEf7hMDChKNceVwofwn0CUceA7jEvTXweIdlP+qIcbh6cDhOpEb+HjkLC8fvKi3jl/fr9xH/cfKLxHJq+ZbkpPuW0XPMQ38pOy8dyFdSq2Ia7iG5BseNV4KttWNgYWUcH8c1cFwDT+cauA1gcTkPkpTSwbqcFtoxsgzg1TDZqIvmABrDZ4SXsPxunnwMux1g4eSqCpEEew3dZe9hwtwkT4mFkosMLLBNIH6hzwk4VKEa4EIgkfciOApYxBxl+TgzsGB1UWChncUGEgcm+boqlHUQqlAyDzCzMumuuIZBN1fpkJ4d6JMBQE+C1WhTzzkIKFR52EUvPHbV5d74McGTJokSV2YEBgRmQ/uODRihFdRZFrDdWEY1ZAWpgQbUoXZEvGAmoCtLazDgZYW3B0ZicAiDbtWiABc9qEtpxKkhrvYXqkgFOKIMoTphuShLWWkl61BZWYGxVR3o8tomG6dtoK8Pk6q0gnxzuSgZTMw2ZXdjNw17x0l7UHBBeqqsFLhimdx7QKNubQz0trRIWtdhrFYBTXvk4/4FASygK6Q71IagYQimvh8GUeBiTco8W6095CXjb1vMwqyq8jNFXS1h4G2ETt4Zxu7kjLYFpCFDGrYKxFW6kdlho7L6Dl3BIFP2sP0gfe07BshjAnByJ2pMqlRprDwNzRoCL5KGHp8EHLGCT7rUaNgunCe+xskCFFWtlGyoRqSxenF3KmjsgyZpu8DqsuWzbHouUnVngTx2qRu/gTA0dgUe1adB2pCoxIdphr4R2lUphl+aIElwFdIK0hCgmKbelfR8pF2Fxs9hD0NDu9FcL4DTcL1aqQrl6r+Mul6xBGFh+xJ1wao9dN6NwbrqRfbRUJkCJKjC5G/b2037RlGdUu3KvqD3qfDoRt3JWAt83FvjOn15g/guAmgc7h4b1sci3908NOj69iSe1HQhe05bItJTqqNLXaVa2o1oP/J12vsy7S7dfpM9tLtSitPUq6pZynHcn+O6khD6MANR2lMSubaaWlgEOEF4C3kqOTqF5FL3tm3Q2cmmd3007oUhPEyND6eLp0eRDnYEWFjnHd3J+k3HJ047h2TQBQ2Y+mzDQJuQZiye0AZeY8EESZrAwrHFccwzxgbS0j2tzhzWodNvwMNxrYy/LkTEaEWmOjtwTM2H44ajmBCDcZA+IbDoR41MiUUv9dfbPxh2FgIL99upA4v6mG16JT9uqvS9lLxqQbcIrsc6nH551siHgMeLc4CGRnlLCuUqbU1aj0yLx83Xm2Wxfku479VpKqnffPV9x1+uVR25UWGNgnie32u2n79952333psffQP+OqeWMpVrPVvDpMn+4ym49VTF9pv5mF3aSQ9MbqYX39UFX9Fs2W/mOjku+3EN1GvglsDi8/dX7marN/yEDg5AERJPS3gZjGJgred0xH0MWg5cDsKc5Z0ykMVqXDWoOQg6yYakwn0lYHwCZDD5ugrdABZMyoZr7Gia0hO6xTAW4XIW5kKmP6QVFRCQAZWhjgmV+L7D/5gu9Qq1AzMR6g1O+nqGYnDVM5R0ehT9ZW0rMlCpgIXgBUYrdtt2NTQmd9VqxAr8DstHyg/YiBVH8jHNYBJgPDKwgJuBcZWYmLL4I7OtYek6K63LSEIWWFWdx3h5bnEVrz0Yi8NIaZdgtTqpulIZQApaOljd7RZQwFyoEqEbTkFFB8yZO/LGXhgwcoKM8JgFowG6gAaYTtJapVqXWP69jj3FVZgcXb0ubWQmNVgXSG2BaWshLyxFo15V/+mHMRygDXQBq+1D9sQk8239usqcjYqVVixy774CuvaUSWshLNTLeH+X9GXGe1kFH+ts7hKtHYUMpZIPV9H1BCY9emKaXWaVG3esazKL0CLIGh0ZZMUb42ZW621r93WYpl1VRbIfqvok4NDYXJUtAZPM8S5Mgivr7p/guxoCC25sW1f1r9ImSgOUclDzNkCAvTbAhO5fL8CAqrak3YV7JkTfI57AwjIvqKbFsyHKmoELrl151y6gTcIV0r+qfQSnK9cy06cph56dVP+SdvuvqlMao9tD3Vsi76GhYTVAC/pVu7JHDRBHT0pKR/QeJcBzs7jrSAXcC0MJizuICzSsn0eVGECn6l2CmwBMxDmDp6Xn4l73NHWi9GWGVf5ZmHuB1gJ9RDWwTYyuR61zaD0LYIjNLUl3nTILrrYCXOD6lbhKxOi4YfOhpENVK58rZVFNTCN61Rl7aUvrSSClS14Biv1/kja/5jdB/mvWJ6BnDFA3QVmUnPGRUSY8R2n/xPezSL3GeMG3qNpTO/YUJ2lPvVoJzVqJ3wYc7MDeahCEYVp3nx1L58ZH0uhgf3y367yzil3GJuV2YcD6DecK9NUMKlQzEqjLsOX6N0/bzPFNSWu4jSXPPDBjA0NZta9wIWGNNlnjKvAsY6XjlmmUMdQ8PcpzUg+g4XfhI1WdujHu78doexAnB/14jevj7O7DexzAwoUEFyMcN+KFnJxJBt1xEz/yXXmce3rOvdxHGuZ76Ch99FBw/KzHbqZjuvlJ/K1HqiUS8f3zOM+NWuql8RrxmyQ6YjRfL99v4z3arRzNuxISsaofEiDFVSzFnwcOnhGllhwxMxP/tjceA4t6VX3qq6t8fym96lu668Fx/9iNrfSlyxvp9S9kk0c3i/kGHdL0R59eTD/7A6NpYlDlyePjuAaOa6BeA7cGFhhvxz4WjIsOjU6IzaPc5+vBZznWTYN48+W483mAhzKyc40wGKY6qJDZLsAiwEQFLAJkwAR5Dc9QMNvFY4rxTCuYLBiO0IuW+YCpFVQcBhYy8YWRzyuBeWoId7MwK+H+UVUrmA5P7SwOAosMTgQXDfuNABYw6OSrv3qZVdWjchyBhdVA2aA1NsSinBl85EmnSA0ysGjOlq5Iqk7lyuYGtOiadgnmaIaNyG7MLqapucXw3a8nJhb/G20XdQHjEJsFBpjogPe3TjhherVPaOVUfcqVS+nU+487AOtidJtV4nUmyRXSnIETnoGxnGN1eYtMYhKlbBGXFeEWmL19/OJbBlfuuyhbNwyLDHkHm4a5z4R7MXSCsgRa1qXqTa72z1EfegBahynclfkmPxmofd7VLkEPSTK9buimu9GyKh+bzpGGBuACKw2s3Y9hBb38Fepnm9O6FFiMAyzOwwyPwWiqjuVmcTLO5i9jrxqTwCL2GyEtJQNO/kqEFiizeSohOMcKvKvlxl9BqjUF4ydDraqQ8a1zGXv7lgbR5wB2Pdzbv6ftu6Srmk5WL8sqXual+tM4bXGXgIHVfdWZVJ26AXN5Feb+sl7GYEbd1+M84OYCjK7SBaUQSllUl9LAmv/RmVxYc+rVXesygaZlf1RaIaPtRoJtNIQSmykY80dJX3sLjaT1LqUtzBL97RrSAmmYZ/V8E4mVgLObth5m9fsegIV0qF5WvEndAFTMwrTPeeUcCY9VbFRHmgIj1aCUpLhPifVumFKV8BgGvQIbQYP9JoNHACg0LlO/GkV38szN/Qbtx9TrCu9O4Wb2Et/AnHXEN3DCeqe9z6K2lAEN7zNWzCCZuIHd0yI2FJuUR1XCE3zrrXxbnXxXpwAeYVeBhLGVcHoK4DixuV83RtpD6fwEaoVIvjqgcZP4K3yHAv0dwICjh23ud69LaFWQTp0SWPg1lCOPdyFdpDwxzpFPqDxSD9s6paBPZlCBNy7afZXT8c7x0CPGLOj2CBsL8vXIId7RB2hv69U+d4rv2IUFPcUNDw2kAVQk+waGUjdSi3a8QkmvfdZ/vts4+KaqbBpBRqnFqOXZjHIgAsFlTjCPxzuOSlM23aPxrNxQpjhILqf4+OnG+yV+fiv+1lM++HbzV9AdP3NY80lJqEFQFSDFxIrgm2PXShLxrW9Je9sbv6ck+B/8ah8q7VCu9UwNa8zH9J//GBKLX/l/pxiyW9LPvm20Tlrcf+wLy+nDf7Oc/tu3jaUxP8xv0HEMLL5BFX2czdO2Bm4fWPBxN4fHcleu8i7N+8O1USaTo8Kd7OrP6wOZk2H5HcCCCTUDC5hqGWt+qy4gmHBVLxtwFxWCClhAdTDyMBfuYxFG3DCjoYLC1WeuGjdAhfeWleI4Nci+7MJYaMC9C6OxzapkeIaCkRAMeNQlFjLA2XZDqYUSiwwsglHnt89LHLIiE7weQb/lk8GwHgs9QQdxvLrCmo9Sz7ne9rH/kPneQpKyOL+Qrt+YSpeu3UhXZhbCyHUdyYJ16KTlYfpRVhhjpSmhtgUDGYDH39ILncbJrChXfhMx7bDaKrjAqSmefzQ21mOPzCs0US73KIDLSnswejuoWCUYlT0ZKRibEzB8J2F2XdtvQxe9h5XtfphsJRjdSDjcp8L2VkVHuwhXzbU70djXFX511He54hA0fucN4gAp5OvparuHalV6qFIVZ4GrzNYejOseDP8ezKPVoF3J+PAAwKIv9PNdCRdYaOwrM27JXUEX8LjSa823c7qIrvRCb1Z6exLcqNaj1MA226KPyCjHfh3kG8wCfwU4GnzL+J6m7DLzSj/m6E/aKyg9UK1ImxbzVe2rHRA2iCTkboBFH+0TK/rQYtkEL4/AxLrXheF3AG7Oa2dAPO1P3JPhBuWZRXXK1ee8M7YqY1kqJ9urHYjqZGPWP/m2U1YuSF12QuryEPYF1vswwOIOVrZ9LrhWsjMpiIVpl9EVcJ6iDjphVi/S7hcBDnf3d4ekxz0iVDFy80Jdv06urGPcnHfAvhtQZ79WEiKtAh8lHR1cbQslTYLDUdrKtrbuBEOCTyVT1hmPCT+Z7gakjALyBKnTgJ7LuB9+EC9sK9Bog3dRN886Mxab440jKZmGnmuUYRLD7MX5+bQOGNgnzx3qRyliG3l0YUfVxpji3iQn97ZCBaobFagJyqbq07POnwFU9IZnLNUX10hzVTsq2kWA4DeeFxT8xgDsgnXqqthYuIAQY5v9i/jSSdGjf3KJQymmzNw6+2gsA350AuECglKMvDu34wXjFd+yhuumk7/13JdMjwgRHvYadGDpysCiP40MZ7VIgUVP/xDgB3ez1HUZy+2/vJ4P6Hu8ozzKcWul8AGn42hJU/o8ym/vjwqL8FIbjQwMbR4usMQj0jTvg2nmePUwQ0pezVTyXZVStEHzWSl89V7zpzUT0Uw/p5nbs/mulJUXuBKv/LJGPHyvqo5GesfAIqqm8ecYWDSq4vjmuAaeNjVwG8DiUjAAZYAu1zJoNgbGPKscWfAymJerkcqA7BAb4Q6y1X2RVgSwYLKMlTwY7wa44D42woMxy6CCK/cBLGB8Qj+Z39JmPqqFNNzNCjCYXBtSC+4LIx8AgQk6VAGYBYIe/gosdsMzFGkDKAQXSiyUGhgp+4lncpeZEEwwOWepRWbUY3ddmFCZCpmL/DyrtwgmNPYUJAVQIV/zPwXDJLMvgy/DqbG3M9OB6ck6gzYPV3v3WDFfXl5CcjGbHhNc3JhJ11m53djUPkWmI6ISmVRIU3o1IvbqxnLSp12Iv+MgfrwDPQRiY8BKO+/uAipYq4aR1mg5M9wy/i0Ai30YvG3cVu7gznIfTzOtMJytrUgvYPQ3YEQ3YPwsj2pXnTCZXa60o9KkkXEH5dV4W738xYqxzLSijBL9y7ZhAzzqRCbZFUFLLyM+CuOmq1BVl1zN182odgBtrPYLTjBMQUWFXZNhRtzAbgJmWRUiN9HLaj4aoMOIk6Y2E2F4bdtzxkow6SoB0gahhTpyFV0DccGF/UbwoXRDUCLTqARC6YGqRWVjN5ljGeBB+ocSD0GMHqC0X9D7lJIPgZbepPwmZLbdDVvvU5bR5lOioDTnCuBCY3GZ/zthnLXHsNWUCug6NVTJeEPGu5PT+jJ/6896U1qhwfM5QIFqUEpE/ALd3+ISKkQPzeFmlQwHkO5cYGV7UCkUz+cAX1PYEcwDFFaJu2mJ7fOUawxatSFxAzr3oLCu3AzwBvFv0CaXYYq7iTMK6LwTUNdD/koEs/pZ9iJlG16HSReUCCD0bmU9CxyV0AgcBRdKLcJFLTS6a/hp+tIIZZnlW7oM/V+fXUgLXG37QZ49Z2IknUeiMoBnqivQ8RiA4rHpGbw/Lca33EK69qWTXNv5rruQFrRxnuCbOsWpk4AB+uldZyfSRc5zE2PUpd2Kfg29MvuxoktbyuyrUuRp/7VH+C9/Z/n7MlRgFAsrPqON8neXmfAYH4mg1EJgsYgan65v51Hdsh8LLBwPaRLGE8Yz+o3flcbgAhL7pHmYf9hlEVdw4VjkJpoDlcRikE00+weHA1hkG4tCc0V10E8mHtUl/2j+jfGbeJFfFWxfanHBoRy1W+OXeSTereLUw6L8Zuj/enzi1pKKZ5EXoUFq9bSebiGheT1YEJJvHo3ED5anWfgcwTz9n/P0dUJq6eRb4xC/iuSbzfg8YuzX/sKjlP1t9x5LLKJCqj9PFVisbe6la7PaKO2nM8OMq13VnFZL3DiXp/l+MZob7DmZzo0iUczNUYvFfkkRbxtV6hYkn63p019fO1IVagsp/pUZbAOJ309+p1yZ4hjCfVyHK2fVsbCym67N4SGQsNNDLI7VnpU4T0Tb8jqLK+u7vN/KHM8Yu7CT7ppgHj2inCXN4+vTvAb2UEOenEwz86vMgSw+9uvpcxj+yeVaD/iOtcU0O30jTS8qgadv9Q6niTMT8BU8RY13Y2EyPXZ1IW214NTn9J24SHfJ9O/+uCWw+ML9TRuLMvCVK0NhUFMG02b40UQ60NcHe+PnSZX4tWcFVAgm4rlXTifRABZOkDBUWWLRBBahCsUkr+QigAXMl7SZT+wrILPPpKrvffXdBRZeDzLwmZGPslA8JwcnEE1xY9db3EiGxAJwscOpHYeTg8xaGGiSpsyBeSihEFy4KVdIKwg7YTjAItOgLjOsGsBAVQfpdkXYsprxCc6wc4DGwujLOIQkgWexEk/e2V0rQATmMDZ2gwlaU8VjejpdBlxcunadjuiuzEh0oNdV9zgoX4AW6M1XgRHlNz+ZEmgLQowPoyQQsc1lSg2SCSO5YKZdbd8DbJBQ2qPcuzCZO53dab93MLWyGtqGj3zfW4UZVSVJXXKlG6dgiNtZke9lFXkAlaR+mHy7lcyjzL5eh3TvqpTC9X/ry5V8jYxd3ZZhl3G3X40CXrTb0B7GnbhDlYY4PTCbFCsYrlUYMgGY6kPjMJlDlXGzqjgaDutNSPsPGXAZb3oQAAHgZ39DCrBBG1l/PTDPwwKEAENMWMQ1XIbX6YOf8dsyyAwXd7VKVkL9y/RlAomoXYZ5W44x0tQQWWNtfwuSrOzwMFXRZJnn6SdKdZSm6KZWYKEdhozjZdSAruKOdZY4O/Y38jgF89JGXD1ABciGjn4kJzLiZ6l3vWsJLPzervD+w/OL6YGp2ZiU+0j7HGo/ZyhzG3nprnYaYL1AXa5zKsEBkbIxY0caoa+78ZyG3BroewiuZkJaATOPJEHVJYHFXew1IoiyrpahVU9S5m83uw4AVdVqBVBi3AFoEHjyiOrIoE1Xte7zocqZXsAGACyCRO0vQmqhRIU0VPHSHuJb8No0zlVA9ygM+iNT0+mRa5MYaiOtCEBJ5UBHKwsIbVy7GIRbUbk8yXmK34Lf04MD6Z67LqSzeFYboP9s8d367crol3HLxRAPwYVgW3rDmNq+AS1+yy4s+M2RI+/xh/aJcPqe310GBZaW+qPPbVLfAovZubk4ta0y3PHQtP1uVWkUYDjouUBhXvkb9ju1D+fFBW0+lFgEsKA8AosBgcXAMPZX2XjbRilsdVCRSQl6guh8d+Cv32DJzzL7v6TRiFhLpz4XlOe+f1R4JBatn++MX5KK+o28rPMSmlN8/LRKjiVe9fvA63XqbSSPZoSctmXOTyj+gePQz8a7xveV8n4zzZzQMbA4UI3pqQALVZX+5LNL0TZ+jqoCvuYFPen7XtrXaME//qul9BfE87t1EUTHJxMDremnvpdvwV0vq+NvHlpPf/DnCzEWGjTShxT5dHv69NdWD9hYPIwx9299cp7FzdwntlxZqo43v7wvveZ5PSwe7qf3YZvx2QfWAkxssDLXjreJt796IH3rnZ0lerod2j7010vpM19bS/e+pC+9F/rM7TXP70lvfllfI53jm2dSDWDXeOlz6WMf+6v0yA3UYdFUOdF/d3rRK1+dXv78syxKMnZuzqcrX/lc+rNPfzE9hoOBE877fWfT87/re9N3PX8wnVi+nh745HvTH31pORa92u+8N/3k21+QeltQR748mfbHTMfFx3//erslsPhiBSzMpgza5ZqzdiKo7oKYTJGDpvFK3IhCmOF5QI0E829fr8L9yH0eoIIRocSvA4sCLsLVLIxbllQIMLIqlACjCSygg38yia1FUsCkXkBFAAsmelcLQ52Ae5l36S7DglfYdxhWVXMysBBUhMSC/GXwjS9jUFYdQyIh4yxjByPlKmFZwSzAQpAjc+n7m6gxbWofwiqktKtasU/axfahFaYpe5kCqJCP9VVWJq0z8895wrC4SgIzvA7DdOPGjXT58uX06JXraQpwsQi4kHkWCPhPyUgLTMkBYEH6YSgNbYbboqZfzmiuaDIYPNPiDJ/8pBmDOHFVf9qF0d8FUJwcGk2nhlnd7eoJRnAZpmgLhk8WvIVytbBTeBfAop/TTe3U63egl2mWaV0jUZlpgYWAxxV6vQvJOKku414Om9RXH2parupbH3PkoevZdhi1flarBUWqVelVSPbUjesGg8mETtKW8VWio2pQPypagjQlJNojhCtUmP8NGG6lY8YZgSkegVaZWRnykG7QUZQmuEpfXOgqURAcCVQEQ9qGWJ+mKxCxTrWHcOXdHqf9xQQMv6pPls19Jtw3QlY1wDHpqA6kGpDASSbZPTDuVvLDvV6WHsJIXfep86ZJWvvE3Ue9CeJFgaE2NgAIOM2eCxOoUOkmtg+VK20clLgorXhwZi599eqNkBh0YRtwGgPlu7BP0C5mkXxVXVrgdKfvjaiTk7F3iEBFsKVXKuvPNrJ+BAHXBSww9F30+9Mwts9hYzs9hNkPlda4o7YG79qj3AB8qmq1AMDo70KqhERiGOmSdieW2f6vdMl9OawH7V78zlULEzyvUd4VToGekqWzvQAL8lPqYts8jKTiUVZ+Ll25mlqQBpzku9OhbwtptvKdtwH2O09orE0vVYJBo40h4bp49nS65+470xCr/RQrrWmoTT3sUK9+3+E9jfKU8cvvzENDbsct21vvccX7GgHxnFcCTLjwYDouUhjXQw9w2m8oiZydnUszM9OhFqVL2uy22u/Y7x+wTTt67yFr5FjmIajQcH4TOgU73drtIFUaBlgM6SFuSInFCMBC6SJfCONCOSoK7azRd0t4/VrG9HItz+pv2M6lTD6vxy3h9bCSxuHrgTgVcdayt/X8ynsH4kdgbpPyvHnNiUlnOXJ6+Tdd7gj6febZqKXyqqWtnjWC4ibKamKHDvMy72OvUAcrRmAh0/8j3zlw8AG//vrh9WDS6zYW92PM/esfnUv3fntveu0Le0N68Mm/XUl/9pWV9Pe/azC96K7MwP/OffPBzN9zzoWnlvSlRzfSb35iLgDImwEgHnPLu+mX//BGOjvcln70NQPxzX/siyvpcw+uxfNivG1z/s//zw1Ax8n0n3/PcLT8h7D9+NSXV9J/we+7TuPEhDzeD5j5y/tX03/yusF0z7mOdH1+O8Ievr6VfuGHxtIAUhOP26FNYHHflzBsp5u95RX96QKSFDM+NiaPKnzm/dm+nD7+nt9Jn03PS6/77lenF/ZdT3/+xx9In996dnr5G96QXnVnR9p89C/TRz/+mfS1zpemt7/lFen02sPpcx/9UPrk5J3pR/7BvWlk+ivpY+/5VBp4xzvSq/b/Nv3Ov/yr9Nx/8s70ss2vp/f/1l+m0z/0Y+nbRrG7bQ7/T7kenwBYPJAeunT5iMTLQMpwGLf+OTjAlsmivBxDKV9gGei9Glb//UTAog4wGqpQMm9MskVi0QAWhMtUSVkwOJWkIEsTssQi1KFkZJnsg3GH+ZMJlfbGBEMCsrW6gt2qgEW2sUAVCoZE+p0UAljwfqw8whwUcHES5jZUoQQS5hPPKmkGjKI5ufooo7CB2oNqUa5Qakwq3aoMudFdO8xgGx5dQr3CegtGPq/WW8eChKw6JUMHzSDa9dVlNuWbTZevXEmXJ6fD7mJxzXyUjOT2ykAKRr2oQsG4hUoUiQi2AnyQfrRzrMJCMz9sq9hTgzrOjBTMsowh6YZdAnEFF3AsbGQwlPY4t1o70jrh4bEKBgexBSf1j7pSByoqfZSxF+ZPt6C6ZLVmlRgIMDYhVzevu+Qto+6pJymlEGswmB0VbTs8l0kVFGksPQxQsJZDLYlwQWZua1hJ6sHV5sWl1XBBO6ArVAyh9TwloFmCITP9dVRddmB2RU5dtMdpdNP7ABaqVFkXSkqUHE3RZjK6MnBKspxMOmlzJTEBeqhHXczKaIdKFe/GHuzQLmA6B63nOd0VWuNn07uEGpFAgiQpRQYlAjpdjuopSS9H98D0Csi07XgY24grqJvNQvsu/XEb5nwHD1LJk/y6AXLj0H8eG5MJmXXqyA3uVEOjRdMVgcX0XPpbVjBUN7LfjWH4fLEf9RmAww5EXKNOwtAdYLFHnhpWn4FZPQ0wUr1JSrVvsYxKLvROtUofv4SNQD9l04PVcwBnlkHgpsco20yXv9rtTFOGWXa7XkDC0cNGbmMwwOcR+bqZoVOve4LEhnow/YKReduJtALosmHgLukJdluIP0R9uoHds5EwqPK1CoB/cPJaunb1WppBYnEKpr2VzS9PAShOIq1opQ50MNOO5ydV0XRf6yaA5wFXz77zfDrNLvYaa+sVbpXyuBAAOoepx7sY6dPw0IF0gHCBlwOsfcO69/vVMYLftOOEfdRxM4+FZESck8Qpdk72IfNxPFgBxCwCzJRazMzMVBttqrqXxzf7mt9RVut0USADO1IN0LyBrZVSIdUZHU/66TPjw0MYcAMsQhVqmLbWxgKwDx3lKPflWsZrnx8V5lhQwksa9bje5zTK3FF+12KbvcPTExxH0VJeiTqlLTxuoof0fZ5LWdFRqSQ1xn3eq6cfCZU/FX0lbkknP64Tnp9UGZW345qf1OLG7X76gXvfeCDef8gfBeyax011VIVZBwUo/8cy3p5aVG79+EcdWPyrP54JICLTXw6ByT/7reu4hm5L77j38V3l/q+/P5WGAAclzgeQetz3pZX0cz8wlsYHHdfy8bt/Op+UZBRgMQN9vwwA+nuvGUzfdncGLktruwE27n1xX3r9t/agbrWX/sffvZFe/pyu9FaAQDmUdLzrQ7Ppe4n3WuI93nGYNoHFxwE5RRryeO8dhz9DauD6x9K7fu3raRB31K97yV1phIF//vPvRZo2nfpe+Ib0/a+5kGY/8yfp45+/lrpf9qPp7d8+SMEX0qUv3pc+8r7H0rn/7B3ppenz6f3/9oH0/H/04+kFGw+nD/zy76fOn/6v07P/8v9M7+3+kfTO110I9by/ixp7SsAij9eZAW8OSI6M1UDqXW1yKoTWB/u450EZvMtvBzHvy2BW7h0EDYuNpJggQz0Fxl6AUaQVqqqEe0bCnNhDmgBZsMehfiSoCGDBpC5z2YpqR/5teKUSxGQv7TFpVMWR4doBWAQA0N0s6g9ZYmEeTFAyBDDIDXBBWpEujI1AIkssuC/AApATEgwYEcGDzJlpuw+HXp42YCb1hy8j6wpiF3rmHZztrB7rvUlVCSfeqKuqciVVOuAnYCq853mobuGaFJ27q6hEPXZtCr3TJVxrkg8rmNatoMj3BFQBJKSJBOKs6DOOR8ShnMYTl4Q6VgAUQYWn7oBpJ5i9AB4wMbt4htpjJXS3byjtdvelzc6etEM59mCw9PJkQi2UU7ZWr1Sq3ihlGIQhlNlVGiAocKVeycUS+eT6hl5osgPLRDrsy6RvcNVYWgark3ZQhcY0KGwAH8siya5sb6HetA2TvA0DP6g7UpitMRhQ7UdcEdfeYAmGfJ3V9n3ao5O0XD0/B2PuBnu2LUkFWHEn6Uuq8AhEACxmOaS6Ee02Tpl0J6uxtZu8LQMeZao7eL8wkbbpGKDqLOkquVCNaoq4D5G3Rtiu6tP5AUOoR/HuSQoxitRFz1bPhfF39d/dqVWFcq8JN7Jzk7gNbCV20cvfJ/wUjOOQTDr2ARdYwR+CNgVcgjjrSq9QN3DT+jDexb6IGp29oxdaxojrJnSqiW0T/ypldG8I7Rn4MJAWAYqQQpyFHqVBgkvdzAoUVJlzTxF6W4Al68Sdvi+yj4JAyLoSXAg8jL9M+W6g9jMNqJjnVD1nFJrvhLF3Q0OBhaBLGwzjZ9UwJDgw4QLWFuqKDhH1c4pvpY9+pFtg61SXscsYVz969Uqavn49rWOL1M5iQdu24AJjbSQV7lUhGGrlRq9gnZT59GBfugM91bsunIuNJe1/bnSpxGIbAGU96cbVNrTf2ye1mVLyqJQr1KOotw6Mo7soj17JspTU707Y63jjwV9+y9zbLwzXhmOTfrCKBHJ5WbC1kKampmJPFtWwfNMxxP6u1EJbKUFGfL92Qg4Bv2DXPr8PCOpUYgGwGx8ZBlgMow6FVyjczbbxrbr4AVHxXrxdpWGZPWKcdtw4Irw8j4iH/pT4Ecdy8r+EmWbjyNnE80bY49yU90o69WiFTsMOP28CgpxZPA+SanTwXkm/ni4V3jgy2bZ+LTCelnSq8MOPiZODqni18j8TgcU2uB5N3XgAAEAASURBVFqAXnWZRv3dzo0SC7pv+onXyigdPD6DrcNfopJUgIW1+Uu/MZnG8RD1oorBL2986surMU7/wg+PlSCn17B1WAYEqMIks66NwjvfPBJxfv3/mwsbjH/89yca73hz2CtUARGvRhXp+ytVpBvzO+lfvG8qvR1Jy8ue3RXp/Kv3z6QXXWTvnVGkC9XBUJ4++Lml9FLi/HBNKvNEtBVg8UvQ1vUNdLVb6D6+fmNrYP/B96X/7TevpLNvfVN6w4svpiGy37/0yfS7f/y1tH73K9Kb33h3Wr7vA+kzX99O42/4ifT6i9IHH3D/p9MnPvDpdOrNP5feOPhg+ovf+3Ba+Y63pG8/+bX0gT+8lr7tJ06nz/32ZnrjL74l3c2Gr86vfxfHEwCLbGNhRmVwzoOtwyLDKSNFCa8TY1gZlOvP64N93JeXGFj9zXfemLgKsChXJ2d1h2VeG/tZxATOyqCTb+PMoCKAhSMSo02WWBQpAiBCUCHTykQcAADGxzAn46bEIooYg6ErhyGxIA+9L2V3s9kVaoAXogaoCKa7Ai2CCpiSrAqVVRxUgwqpBZN3gA3ylCHQziJUHvBEsw5TqLHmhuo8MIo+VyXKlePClIRnGer/QB1mcq1Bn8ifxAkbjRoPjNr0FJKLyfQQ4OIGalELgIs9mHTTKJN8lJ16CYBhXVTtWybODD4ygLJiyrtkGsyTDP4OjKHAIkAH9abMaI/63QZQ7GJzsdM/nHZxbalxt+CiBYZnD4PjfZhFXdV2wJy6I/AY0oM+GG2ZXnubtgqq4UyHiktmbFQtUv1FUKHthIBiTYAT3bNitiiH465smikJgARVi6zq6673BMyf8ECbCxnICfLWtkLG+GE2WJuFKVcNSu9A/YCKcRiyizC63dCmfQ4lDZUhN617hBXsOVzbqvtv/zpHmrqCPQsjrYHyNdpWw2E3OzsFvYMwu9qAaG+yClOs1GEEZti9LgRTc8R7GE9Dk6QpvXTyOFXPcdPAc+yfcJGdp+/BfkC7ESUEU6xuX4LpVx3KvS428BK2C7g4we9eGMnxiXE8JE2gBtUdQGGdPqZUyrocoEy6hr0EQ/8VbCy6AHrDMKHnAC/uNq0kaoV2UGIhsNAtLEgeHU2MECnLOVzO9vGOXUrvUfNKIjiV1ni6ij7ANzFC3AlWzZVG+V2rgrUFHQIN1ce0kblB3U9z9TsZZM+V86yu30E9qgq1TNrWySw0aJ+yStrbnHt+09SNu5LTs1j9z/ZUGr8P2I/4fldI9/q1q2kVSV6CWddQu11QsbmKyhNAHhpdcAhgiuSsn7a4C49SF6izs9Sd3562VbahjP02dRDSEaRvMjaqPzk+ueDgPwfYWBShUgQUXagEdqPe1QDv0FVAhA4aXDjRDuIUQMWr9khbtOkatK4ggVzG2HxmehqAsRiubcNQ22/AMY20BBXiAsc8Tz8FujxAx8URKCLNTto+gAUbnY1wDgyOsPN2P+8rrcgDh+/FQRoe9THd776M6+UaYwHxyrUev7zv1cN68X8cpk96/mzmyY/yPMe67b8l/3jdjshRaPQ+5+Sd+TVyPJhfFVzSynGJEvsOlV+NSAfTjzzJhawb+dayOVgu6cs0mqovve2N3xu334g/9stCY7nW8zXMOihz8FOVWHz22l66hyXWXkzonuzxZGwslAr8UyQTGk4f5X5WQ+c3VWpOf/voevp3n15iSN1jfJEfYFEFycNp9qUowOJX/2g6DLt//u1NMCL9h4GFYe/64GwYbr/4btRLkW78+VdWQ0rxcz84Foz/Vx7bSO/5+FzYcfTqCeLQ8RLeK9KO26HtGFgcqsBn+s8bf5be/e4vpLbvfH16/SvuSadbmYO+9vH0e+//Wtp63nemN7/pOWlNYPEgwOK7ARYXrBA2Bv76p9N9AIuW7/vF9IMXl9LUVz6Ufvf3P5eunxhKd73mtenC/R9I86/9ufT6CT1I4rq/F+2HNnjh+pj1FOr2lsDiC7GPxaUY8ZsDT8mRazV4Np/dTEEMTgQ3BmkGqhhKHXlrR6xw+6w6y2AW4TAeMTkz6Tppa+QcK4JM2l4zqMAAVsafU1ChmpJpBI1MIEoLMqDI0gkBRdhYOBkzKTsxB7BwYqVoMelURYVVJX/zUiKSPULtwNS4/8Q+9DghHAQWMgbkAyA46Yo5eWUJRZZUSEsAi5BykB90yiDEpncwEare6GnG9J13pU01ig7ARSerzm5iZ34+LPUUkpOoTyf9PPH77gnu9Ra1CriYm5tJD1+6li7dmE3XZ5cZ+FztF7DZkLzM6QqqwCJWUknA9iunz9X/lrlqYSD2lWhMXg86ABQyWSGxoEwxKVE3ggtdee5q0I2nqF0YmW1Uo5RgwEalXSQH+zDdVDLehdwlHIYbJrKba6iNUJeq4Kj2UvcWpQQhwIU0U4YwIocgmULXkfW65CqtKi15Q75TaZ0yr8IMupIs/ado7zZWp0fJbxwm+jynWhF6anoA5m0W5nYFBlKbgT6Y4TGYMl2qDsFAK1HRXmJGRhuG/Bp2AUvEVYXIOtILkeDiNAyldglXYYanOXv5asdYub8TRtwyKGnxuUbeYcNBmCBBGq4ADmZYwV8lbTb2CBDVRv9pZxX+DMDiAiDnLiQtAhlp0TbjMvRcBcBcRfKwyer2SSQpvYC4s9gInMPw+Aw2An0AGBl97UIEMX4bw9A0RzvoivWh+SUkIl3pDOpS7tUhw2/dKynQLe90VeZt6GsnjX765wXUlU5TJo2oBRbuyH5FVSyY4xbarhX6OmkXd0YfRMLhnimxyg4dusSFn48+4z4d2mRMUmY9Q3WQ9ghA9C5oUQKl9OMx2uY6bTO9zPdIe7cQ3kadDFIGjcJ7KI8G8CvQofcqjddBkmkH0L6+MJ92AWwnWCRo365OvutWPCbIYNgGglVBxThg89l3nEsTYyOpDymLo5bl2aQestvXLEn1G4rBlHCBkof9UDDuO/4JNSi+X79jP7YC1HVFqxqj/VFvT343LQzy2lz4nosOG+ur8Q0vASzm3XsDVTHdz2p/EWp39OMAFtCf1RtNzfr028zfBlXJnh54GauAxVgAC/biwA6qiwnlpMDCf/lVSCw3kVTQ5d2txvvGOH8oXoSX5KJCcpoxP0jk3+HhDFOSLFmavLmU38apRrCcMw9zWONnvol3Mn0+zymQSlU3plcvc+OlI29yOk0qjFTCuEL0MxFYXGHvyVG0Yp/KwvqTARbW5i/95mTSfuEoCYfPPRbZ5fV/+YMb6XnnO8N2ok1xCsf//u+mmWtbGsDi3R+ZTZemt9M/+fGJRr8x3mFgoZcnVaFe8qxONALkE/bTWWwe3vCi3vAI5TtXZrbTr/7xdPqBV/anVz335s3+jONxu7QdA4tcX988f6fTX/zGe9LHpibSK173nelbR5bS32A/8Ykvt6TnvenN6ftffz7NCSweAFh8Tw1YILG474MZWLz9nlJb8GRbLLD9+W+ld119ZfpPJz6Vfv3DD8EL9KbnveUd6UdeiRF3R17QLW882eutgcX9SiwAFh4MpPnz82++c0xsMJ0R6eY/PhcclCMG4drvEh6MKeHG9N54JcxrE1jADAsuYHQEFRlYZKmFwEKVIoFF8fMeNEJvrOgFuCjAAmadiTurD2RwESt8MgO1srpKFRKL4m4WqUJILGCuXE2MHXah9SSMgMzBSVYuddnaUIUK5qkCFpWdhVKLABrGlaFwTnG1EgCwUYAFDKpldCXeOAGEYAoEFbGBHcyZK5pKcbJKUrYncaLiFcrgxOnhFYYfdY/1teU0yQr25Un2upicSfMwZWvopLvBXeiB+wbMXQFYwfiU+oBBst1d0YzVVgbjRj0F/YIj2oy2UfUipBa8UQCjRrl6jNpr70p7qF3sAC62UI/abgdsmD+MKuKgYIpOAi7aUBlphSFvZUX/JEwwFRtgQWmAacq06bpUZkpGMLxoQaISDOH2GnS4Wr6CZEZvSO22CXUX3nGsV5gtgWVsmMhVNR93xlZlxn6wAoOnh6Rl2sF39shDl559MIGnuY7QrkpTBDuqTM0QbxnAogF1qFmRZwErbsI3icRhEoZ5FamDnpjcIO5ONlrT9bFGxtOAK1WllLoIWLQr0VvVNWhYRIKwxfUUz7upE71StUPnEIx/3sWaiYpiCyw2ON3gbhI7hUdQf9tGH78Nmobx/vPsO86nOwAW0qMbWsGLu2h7VfoisJgCKEySl7YWgqhz0HgBWgUVqj6ZtnWjYfkM5VnDnfE+oKqD8p4dZyfq0eGQcKzSL90c7xEkBEvsX6HK2Unob6XuVS/qBYC4at4LLUohLLOr7STDzuXUJ3WqWlm0N2HdfCtuGKjKl6BXl7Sz0LmAfYygOtTooPU8IGsYkNgDStEt7nXyn8TLFR8XHp44oR0OHQkOUqgNQJHAYneTU8P87CFGkKahuqpxAjHVoPqh1wUCFxgEpZu0oxKA+CagxzGqqD7pmczw+E54Fkf0WcYev19OgWd+zrjhAgS/7bqqRfoNxUIBdFDYuNfBg4sDywIKwMUSYHN5hd3kKaPjhB+94NCxQnupPO6SHv3I8ZJL0HSS/tuDFM26Hxth1/DhsQAWnT19lE+boUxu/C0/SNuSlsO0jzqOYrCNe1S4YTF+PM7zkn6JV34/0TXTyd86wdVLBkl5KU2Z0eLxofjNOOVBCTG2qXBCe6RHWZ74OBzHNz1KOOmTzg8+AyUWa/Q9HS1VX0Iu9m3+fbLA4l+jbjSLR5z/7kfHY5HgqGy+emkj/d8fmztgE+Gi1K/84TTj64kGsCg2Fu980wiLJlnc4jf/7o/MpYevbzZsLL6AEflvYwz+37x1NNy/HpWn0pT/CRsLPUr95HeryHL0cbu0HQOLo+vvGR269JX04T94f7rvi5fSctdd6cUXW9Pc7ki641u/I735VUPp6p/+Sfr0/ZtpDFWoN9zl+LKergMsPv7+v0rtb/mF9EPPqWpnjzn8xl+l33z319IrfuaF6VP/6NPpRf/sp9NLtz6SfvXXltMbf+ZN6bmjaCn8e1TmbQILB9EqF5dzq4klBx0YnhukVFH4nZmFGD4dgI8YhB1QCwPqffxmom6Ec99Qg2ISvQlYwCxswpRnw+0KWDD5y+A6bjuBCSyaUousNqDXoGDyy4TsRAEjUCY8yyfdMt0aQ6sOta0qlOACJlhGQomFTEA2npbJ5QzAktWg1PNWeiGQaG6S1wQWIQEwX/KAqwhDTcHFGoxVMAxQECuoMLEyH8Z3gzdXOWN/CJ6rSiW4sG5L9VKj/KYODedAkYV7VEHQ0xZcPIanqMs35gEX6Iqzk7TMUAGAjTqgLjKjksGEJFonAThg3CKegIPyl1NGRhBo3Vt30mO6tlmoRUG/NhfbuLfcGhhN233D4W9ZhorC+wYrtTBZqIu0sLJ7EiZMqY9gzX4nkHSCCtetPBNwyMYJHnSbqq2Am78twvguwMyvLqwEiFDdpBUj3lDzIp3wLkVcgYUMnoQaprtUbTL0nrUESLVtPXYqpleP0T3cCyzMS5uAsCegv8HPxWp1XilnEz3aXNWgQdKcUloBAy6geRYM67PwynSuS4WdvPfGEn3JVX6vepLSjeo8jLyM8TrnPkx2J0zmmJua8X4HdWKfdnM+dyAXHGxRBl3Yqio0hfrTQ489ljbxJOT6+ChGx99y8Y501xi+r1nZJ1pIIQQ8vme/HYDWq3iVUhVpGoZ1HGBxFib0AobbSixUzZqkX6qWpiRggfIsTU2nLYBPC8BoeGw89ne4mz0jrGe9Nj1KelNIPxaRguwBRFRHaoOB7QPoXDgznkZJW4AWbn0pt7Y0bo6o/YTG+Up/tqiTXSRNXfy2/k8AMlbpDtpnbEGD6lg9pKktyLPGh9MoEhb3InmMDSMfnZlNDyO14WOij6BCRv9qR2pxCglAC+pFnYAKd9TWE4Z2PkpN2vkzglTnHEDpznNn0jBMuP3Efq1UUaNspRXu5xIupAH8tqNjlN+snqei81PJglT7UAHagog2+oO2Ui5CRN+j33n1W/M72mM8c9PL6HmEC+Z16rCmKtTiQhhyu8P3KvYia5xKLZRu+n64nTYd3lPtLxZfrD+/XdJxv5FepGja2oyPjKaRkfE0BLjoQCKUgYVfeHXQLzz861jskRcWmmChhMfDJ/qTkyC9DCpyejm/o9IxzLOMx8a/KZ4d2cNLlX7jxt8+r2g3mof558NWq44qyLzK88i7EbeEEj/mQN9rvF0lkrPzh1nWaS1kNiIeuslxn5nAQvzt2PhUjicLLB68tpn+7Ydn2dOhPYymVXNaYc+HrxPufhIveVZX0tj6nyNhECy89eX9jNn7sYO3htTn2cuiqELNAFB+5b1TSFdPhQqVko2PfH6ZORNbsI29BrDQxuL/ANAobejtZH5GRO7eFKpjvfp57LGDCpbHh/56GYPr5fRKJBYvxjtVF2hrbmk3ffnyenrlPTne7dJ2DCyeSm96przDWLS3kR746G+nT83gVORVb0yvvrM1Xf3sB9NH//rR1PZtb08/9vJxeMrZ9OjnP5k+/EfX093veGd6w1nLv5+2lqfSF/7gXemLL/jZ9JOn70v/9F/vpZ/6x9+Xzrd+Kb3nFz+bnvXzP5xeMjGAdsFTP24NLIoqlOkzMjroHjzy7zJu+9gQrzksg4ryTo5dfuWB1/HcQVXmU8lAvjoqV+EwGiG54KoKlJN7AAsmyyyxEFQUYJENudWdd+J3ojcd6c4b1qlLmaUUru7JLIRaEhOxk3JWXWgCi0IpFECXzHIGFltM5EotYvKXgYZ2GX4Z/wxgCriA+ZMBdEVSYBGSClcnm8DC++x5CUJlTPAMpWHoOuDCldGs5gAw4v08DcIYASqUXAgsokn4E3Xrn1pdCioCdFBnSjBYVwUQsQfB/FyAi0eu3EiTGHPPAi7Ubw+VKMjwiMkc5izADHWTme/8LJ7DfAUAIE4ocZOvG2JJgoArg6LchtZPw1sUMZRcbKMGtYW9xfbwBMbcSDGQ+Oy7msxhuqdQlelgVb4TJkz7AVd4VUeyLyiVUHWmnXCZYlWeJFtgoC6/alDuYbAIg7w0PZ/bh/rqGBhAxQqXtkhEXCGP/myavCvo8XeE59oMRhMWLdcpz0wbd9FhHyGwsbRKGUIvnnDVsqRDCYTG1kofIC3crwpy1umTeqp63shAgIthdPjDsJY+tE3f17ORNgOu9KtqNAXTPA8zvkt/UI1Ir1nnsZOYwOahBw9Aq9Yz70qLeeo1a4/20JPQHOoyDz70SNrBlkBD7+7R0XThPBILDbdh5jWyFoxoi6GHJcveD7MrsAhJBMzzOQCMthjueaF6j/YYAh839VNCMwdtKwCLHYCF9h89o2OoDI3ifWU4DKZlJlStegyVrOsAnTkkB3swyO3YGg0MDabn33kW2wn2UQDQSMc1wNckV0GM9UGPCiP3LaUDSJ5aeFeVKr0vubv7Pu2vsVkr/UbbjVEkK3egHtZHn/GbvcwmeJeg7wqG2r7rRnetODRoE1iwOODvXpiATupSL1BKB0/xnWi0fYZyXDhzJt1x7ixSMRl+xh9BA220TR34bWq3YHsLsJUexvgBzY47JcxFBUGFthh6r7Nvt9GvNfYOA27ezd9YXjSgGfgOAOGUye/GdjEP7a5WkFIorVhTnUt1SdrAsUhg4ZjnEd6hoB8Cos22Ab6qbYXrW/JyoUNp0QjeoM6Mj6eR0XFsLEbD1axjFBlSgoOHdJRDeoKmWtjh5/Xfjfe4abxSZWA6h4+j3q3Huel5lYYjQEmtXoKIXytTeb9cS9qFlnI1POIE0bn8McqUqqjyzXURsUtSvhn3pbzNNJ3jrL8StR7PQIHFM88rlKVsFLkU/Tav//J908yPKf1DpAGHj0/87UoYPv88dgwj/Sov5kN7hj/6DCqDMPrl0KBbb0xKDDxUZ/oQRtPuN6H6k96aVGGaBnT8NBKKcui+9g8+tRBAohVg8e0AAg2w/y88ORWjcQGFwMKN+L7lAvZxpDOPetQXkWRow/Fzb8uuZBlC0ifwMqWnKSUYHnyS6c6x9vRjuLPt73Y0uz3aPoI72499YTnUtI7aYC8SOv7zDKwBxvOl6fTQX7w/ffAzK2n81femN772W9II38ju1c+lj37kT9Nn956X3vb270rnF7+c/vSDH0+f3X9J+i//q9elcEGwg3fBR+5L7/rkUPoHP/Wy1LH9N+nf/MNPpef/D+9M37H94fQvfm0lfd9PZ4lF08XAk6/G2wYWMTg2R0Ryag4VzQE0h5ZoDqKNcfgI2hy4fV7ARAEXBsYznhdQ4VUGrkgrJNwJ3v0s6sAiu25V/xlXmDABpiXtWRVKSYU2DwIM72X2BQGVCoEMAqfxm5MBScBsxO7bAItsvJ0NuHVrqUtbZ0z9w8sgNMCFaXOqPlM2yDsJQ2VYNtzmKjMisODdzMDCDsHsxKqoKlEwQzIy1qKgJApDXtItqAhbC9MzDfLOE6qTl4CCtKgzVaxi125+y0LrKWot7C3QH8Xzz5UpQMacG+hRNhiakPLYVjSvdZC911TAwnAbzKaHeYnVf677LElRRZG/be/qaUhzjM9hW4b9g+1JvD2ZHjzQbHf3pq2hsbSN3cVORxcJUAtVG5yEidbGQq9EPdSh7aJRtgBF5l9DZ1fr/Z3d0dpG5G1+nFswXqswyXMz86HGdJL0e2CUBzUKh/F0snIDOtWWKGgwgvvkURwbuimdm8851Jue4YKZTuKesDz0bVWPXL3vhZZ+mFO9H8lqyuRraD7DKv4CTPEuzN26diSE91Oe5wIsLiKxGITpdDXblXnf2aSNlunT1wEh7lI9BeO+AcCw5S1rL6pP51k9P0MZhgAlumkVXFi3gpJ1rkKzXRjNJVbqH33wobSHFyFd7LYjTRg9e5adOkfD9mMCdSqBkgbQWb0JaYC/US/SJkEG/lmjQ2kMZt262KYfFbep1+mX16BNW4xlpF87ALgT1GMveUygOnQXXqrOkb57a+jd6hL2EFcAFZOoTa1RNuu7B4Ps5yCxOAsQ0Ij9Emm5Y/bVxZWQXljnMudKjrQx2GUPFg38lTxE90OS0kV9aAfRS62r+iQwMi2BgHV6DVB17fqNdANj7RNIYE4IJlAJbOMb68RQuxNGQclPO23nGHCC+hdDqBZ3B3V1/sxpgNIY/Zv8SS9UlGh3xyCZdccewbi05m+Z759vOYAFdSbYaKMf+x2FjRb0y/5qVB5Anfds/5D8lbHDscAC0rdCGurVfgaYEFToFUpgoQQjVD5V/SRdxz2/d98NyQf5h1viAEEZgCrNUhWrv78/jY4Oo7p2BmAxhutZNrHk+3AcirGP7A8f0mDF5zHm4NN4RtCBMdP4h48SZDrQctRR0jrqWQk7EKeRTjPxesrGredV3i3XkmY9TgnLV9KNpHM72y5xVPn63sG0Ch31VKTIdiyv5t/xi7AcnMOeicCiXhNP9j4AMS8JmA8f1puqSdpFHXXIvLtDdS/qTUcx3767iLRhAIbe9PkUoy2Pykvw0EM65ZkubEu+bqD3ZfbB+O9/bDxASqHlywCc38Bg+8fxaFXfAM/nSjkEIIKJskN3ec/rE9FmX3LuKzTU3z2+f4bWwP5D6UP/5vfSJ74yk07d9bL0BvazeOlzT7NICM9qkfc30/zDf5Pu+9CH0se/PIl7/8F09oWvTW9923enFwzLH0ak4EN22OgrFszQxJn9s3elf/6HX8YT5mB68d/7mfQjLzuNW/jgSOONp/LnyQELcoiBtCKxmSEkxwCeQ+qDtBNpOcqdA4KDcRmQ/S2oCEbYL8azihMTOhNmARb+dmJ3UlViEfrOTPDuMSHAMFxQse0qHfFk5pwMG8BClRiYCCUXRXpR1Aec7ONkki+HZZFZVBVKl7MabyutCImFwELGFHoPA4sAGDCDsbkdeWZphSDgZnARwIJ8rB+ZGKUWrkSuykCQh2V3QPN52FxApy4pQ19bY27VKgBMSj5IIf+rAwvrQXUpzj3LsAHziErUdVZzr96YSVenWU3G3mI17C1kIJttELRV9WKdVPMpAAB6YByDQSK61OV/MLbkF+1Wa8ecZAUsiK/NwjbgaIt9LrZQi9rG3mIfuwrEO6SdGbRuAECfm+fBLMp8CRqUUHTSPr0wZwPUq3USakswoIsw8NpWyEjvs5q7ibrIHGo4u6wOt8PI9sKQq3+v/ryM5xJM9Sr1GwbcrJq30C8WScf33YRvkPZzPqPmwnbBfRB6OFthLpWSyMjLKp5hRV83se5+LaBbhwmXFt3PXsV9q7Ys9kd3C9ddra5ezyE1GKIMhqk6o6RCcOFqvfYLl2DEp2DyT0BnD+Xv1bYC2nXVeobfbqQnMJJ+wcgC5yJ9URWqLWwIltkcceqhh7B/WI1+0nn6bOqZYMBgpXoYZvw5SD00QFdt61GMpPVYpeRiBZr3SLMLul6INyS9ZcmgrvNt2bvck0OPUJdRwfn/2XuPXk22dM8rtvfe+70z89i6Vd2tKyRaaiSayQWJwQWJIUNm9BdBfAAmLYGEmMEYCZBADTO665quOidPerMzt/fe9u/3rFjxxt6ZeUxx7xVZ9Ubmu8OtWOZZK1Y8//W4l9hx7AEszgAEbfTp6Mx0MTsxDmgaKmaUDEEPwZeRtLUVeYtK1w7AwngKjiuNyCegxwCqWabZwF5iGzAocNMAmUEextuC3RueaaGNHPADFNCf47RhdmywGKecPirXxnO+r3qaOqO+69iXrL97V2y/eY0BOypxjPtW3i2cxhDPo4PxgNtXnk3OCgCM9Gc3yEIw+8XSEoEBWWlkdV+1R6UOaWPcQrOQWAh4OOYx3msZBN5vxk16f3gnaGOndkKMU8uJ+Y262d+ClCRxTTZDjm/nqAiK6eIHeenQIeY9xoTqkWG4DbAwpoVgwrlNJxXakyk9dV5U9SpoJx0cC4zDVA5l8rZoszSCtGhKe5hp1LzGJooBvLSpWmmfuPlOucVcD02rzRvlqXN3nufrx1Vak9aejTRlzvFcLqSepv4wxzl/L9fzupcsThPbnzLNWdefy3nlfPLeNPleTl8/91raSmCQ61sWApmpW5kiH3Dqdf6mG87I1T1myeoyD1YnqY1/jDYWJRH+aHf/0/+5W/weyca/+E9RMy3jXdjdBsT7f35PHAwkFh/zUvVHS5Bmw/6eKMBczgKlwXzxwsGc7cJ4CSrKEm/5PsZCE2nUUZEv7ITf+QT2jqdu8Yp4ijq8Tv87DGlgnnmO+gNb8uPA4vvkbtZ503KirFxivmjBMFpedkJuTMoNaUU185qWzUcz8+qxmxNvfHjLCTifey3/ZBgy0xqRtwURfHTDK1QJLOKDz0fXfTaCtPwAFjL6fOQ1mFVi4cc7Vv9hJrwvkxyrh6kxqWLWrS6xKIFFRN+GARNY+IHPDHhIJMybskJiIajwRwffUYcqz0PaoAcYyg6MqMoVjMIlK5In6IGfsPKuqoO0z8HdQupCnjIDSi0MzpWARVpxNHFILWBwlfIosXAV9TYYFfM2AB9MNyu665tbxXvUhbb3WRFlVfhYd678gpGyL+xTQUT0bWMQ229ZYhGqXNLMjTEfNinQxMcTzGEP95X6NH2gBQkZXITkAoPuW1SjbpFitCDNUPqgqloP7dNY2aBk9pWlsNAc7k/1LqQRsizfDkyn8R5ksIPBhMmCowqAxtsSK8e9MOPaRUhnGXLVpa5oq56G9ETVyv6Yeqvz30sfCuL01BT2C7SkizYOUJ6SBpzjFkfc36H/pzhfgGGb7wHgMVb0ZGTE7Deo9QgsNliFv6UuvrADBHwbIhL0MOWNMzGM8axeq7TT2KJOxmgwavQ26kXH9L2uUkdgoIdgpF1dn+M5pQH+7FslFtqTbFLeNvsdnt9HWrFPELj9ly9jXPUNjxRjy8tFL/suPFspmfjayNsw/9LiBQbbz2Hs36CKpPepNuoxDK1/Mz+V4nrwjlmGTKsG6QKQTYDba4yHN7HhOARgXKO+NTA0TF1ZDYcZl36OV+NenNKXh9R1l3yM4aFnLt+bPsa94KML1bRTaKmXMmOKtDD+jW3iGNB313WAWJhewQbpNPaX+Z8D9DwaHyK+Bf3HLRnoiJRe7jfX14rNt2+K7dcvsacAVPAetWFnJLDs78UQnPZHntST/zFuzHdqdLj4cmUlIlPL6Gcm3kRK4mLxgvnFiO+CDOd5wX0XY1T7J6V3MdbZB2BwLuB6qC7SBqUX4boTKYPzlJOw7XIO6ma8qyblO06VeG/Th+IUIKextuDimLYosVAyyIMBLJTq+M76ngaw4I7zpXFlVNnyJz276L8xvEHN4Dp3dmqmGBkbL/p0NUuZd95jc5bW/BpzOhe9zj8olk4+8dfn6lvk4zP+l9i1LZdTuxRpcrp6XvmaaevX4zxncKds65HKi2fv3CsfoD7388pZ3d2TV/p/97IU4fq9ZpHGslM979Y7LnuHX65bStcEFpk2n8/+zeZF8S8x6NYJysQgdnWsXKhSdYQdxl/8+UDxH3z76cB3n08rmzVtUuDnU+AXAQuzTZNzWUD5fcgfGe+lCbTx4WhMnWni9EmvCSzqk7nHMugBOEzDeQYU9X2ACz6oIa2AAfHDHNIK9meqEXkOc+XHX2CR7Ab4aMMQJhUiGDk+4FkVSs9QATJkOGXu+aUvBG0omyGw0MYiguSF4XYpsYDZciU1pCJIC1ytFChkUHEXWHg9gQsBRkTYdYWTXzKIpHzpB3VcqbymDREUC2biEKNNGTHr36u7SlZBDWalNCRLRJSEmE/+uAUtpaeAgn2SVsDQwVhds2oruDg62g9d/C0Cha1ja7Hj6voxqlIyeGW70qfRXkt9H30NXfyQBr1glt1Xfc91aW6f+axMiGn9E0asZR97yZXea55VLeoa//636HrfIr24UYrh6in3ZLZ6UCfpZVW7W+aNfhJ9B8ji2H51JfaAvt4Fze/za6Hu2lvobUimHcLzH0YSRt5nDWa3BTN9Qf91UodxpCLGCTGdXpWGSdfvMem0KVAioFGx2rlGu16AObulXoc8+548hjmeh7ldgdnvogCBiK5W38RK/Qk2LDDrMp88q6vPDtIKimSI52hXH8BWN66vBCJ4UNqTwddgm2sjGHhPwOgOYHMizWao5yJG6F8ADKSpEav1oLROu7f56UlqZ3Oz2Hn/vth98wbvUcQFwUh3fmW56MEgXmBtPj4/C2NtBPOX1PMH6vh7JA8HSA303jROXX8zh8QCVahQK6JOJ9BVr1hKaTSu3oJR38Xl6RHSiEvuC3D7WPFQRUkvXY4739cb6HNJX1zQX0Y0j7FFGW2OaerSBb0NAhnqbazmG3zyzPHPOBTAqaLWCgAzD1W9fH6Y/lQl7MHIAN6idC/Lu8ezBzyn8fohTPv2xnqx9fZNsfPqZdGiJygWBdph1AWYfdC9B9omD0oMNerrmBkbxGh7ZqpYWpjDYxW2P4yrUKuk3o7zUPOj/JhnaMMZfaRalKqIgnzboUvoAMG8j4IIn1Glow2AGy52aa9AIM9VSiddMBG29zC2pKNzReAo3t9wN4sqlwHyDvgZ60awE/mGhDOBE98vsoh30fdOoKG0SanFBajC96QXFbJJ1OFmMeafnsTbFWpQvYD5tDCR52+HlqMkbR75GrmZr1vMA3H04Z9I03g8Engtzx1eqD8fc3+tvLifJ7J4+u4f55p6/fLdKPJOPrkSqfaRZb6UH3LP7TuPVfdInNNXBMgUKB+MtDmReZUJ45J/SM8uX87JG7mUlCgf+8tm5O0g0ef254IAe9+vMh9iW+G7rsH3wxlcXysebW5NCvyJUeBnAovGClWeX4NOzJYJSHjWOHaqzBN/xXCSIn9AnG7rwCJfdwbOx/l+MCYyJ/wys+xHWIlFirrtymE61t2s0osMLEyXV8oDWPDx94OfpBcwAhxrAJyvqc5QSSxskhsNriQWMCUXAAulFeFyNoCFnqH4ZMv88FPHOgCF5bgazUpgA2gILGAYyusBMEivyoTBSTKDDkcA8w+Dj9eafVYodw92KfMyGGt96ffjGrILnehQgYJRDP1u8vDrFfSr0SuDiizBuL0RcKE6cYW6FeoVgot9jFzXNlAb2SGAipILVKJiNZYVmNwP9k3V11GOfL+ro7BDZdsFRhIsAUE/nfZnfFoDReS+iL60r0ntT3BxA9OFnk5xgzH39ehEcdrZEx6OuB2ed1zJVTXKoG26iQ17iChOk/SWUEs6oj9OYehvoZWMe+jfw0CqW+5q7AgMqGoohzDgb9D7145giL6YZeW7lfRXpFHVyhgTGgMrDdD16QbpjxhjHVR2lNVu1YikPf6IileU10f+U1x/gGqNEhGNtl8DXF7DcK/BcB+RRgP0HqUuAAPr44r+AnVbAdQM86w2Eo9h7H9AdesdhteqF3WTzzQ2CDPYLPTDsB7SrjHyWMSt6q8nRhya4VL3HWUJLAxI54r9LpKoLTwhbQAuugETEwCLh4sL0M54CcQToY0PaLNuZw0Ip6TiCcDibzTcZq90ZQLA+2u8O2m8rQTKeq2jxkSVAFWAQdpwBtg+k7kGCBgfogXmVXCrJy/7WNUlwRGDpLiljQRwKK4BXvY5X1902PCkRV0EFvN4KZoFMAm2zG8Pac0W9VLaYV/7Dh3xmJG27YsRgPQE0gXBxUQXkh/yUMpygNrSHs/v0m8725vF9upqAhaM8zbAdAcqjSG5QrrUDf3bsa/w029QvX76dBrj+CUibOuKVZXJc94R7VxsT8wPUfXk5UwbL8fbkRIE6C547cW70ijqZtqQKL24Yr44RiXvEBuSm1tAMtcHsPWRCjGP8Z6fIY3INhI9gJmwnXLBgbJ8gQLcUHeD4+3tE5uE9gkqvK6KVCyicB7vvs/QN9nGyjg1SiCNSm5wvCHoPI3RtsBiEo9QAwN4/2CMhI0Wz/0o0556zhKoW9Tuo+nTmx3J0gvOodfimfRYeTNuVMfWP2/VXJMvfGR/P339PCWnVLLMedG8sj6NckxnzfKW0vpc+kUG3iRJbrNtiQv5obyPAjwhcX7eM0VYeYvD2nkuu6xcE1hkQjX3TQo0KfC5UuBnAYtq7qOVeXKtGuzHyJOYGD1K53mSd6LOE3uerJ1W65IJH6/u8QHPzGxmQLNqjStwIbEogUXoDpegQnChxELPKxEkD6bFD3fKQ715Vq/5sAoiOmAaBRXxg9EKxl/mmGMZ5ZAcRHtsDoxySCywsQi1hAwqBBhp1VCG2TaqGhS61riRDANxmNWstiTISBILgEXUQUDAsaAiyrXsBGzk3lRbioi7+NrXWPNKJoz7MtidrAq7wt6lPhy/JPUoGXra7Cpp1s/mIACZjAaZclyqRdGW7OXqFLWo3Z2dYhPPPYKLrYMTXOdp45H0+aS5/RN9aWfRx/ZhtFlwYR9DN39uprUPox9laEzMn1hRdZ9/kTrlpQQArqu4Ib6FMS5O+4aLS0CYen+OKGnWCdPZi1RAxotOTF6XGBPmawC9ZOiLegj1HiDtCGmHypV+azYGI2q3njBWVpEIuFquncEiqjsyyjKtShtmADCjPK9kQ+9M66wQKxVQh32AMfQ1EgRVmSB8AA+D7wk4ZmBU3fbI/zkr/y9h+DdhPHUxO0X6IfK1ndpmyNA/pH4rqkXxrNKAx9TpMQbOz7d2wvC8i7rMY4/xJZ6chpAc7OAZyW2Ucr6hDjLEtmUDhn8HHclD6iuo2QeQbuNBaB01JcfMOGD0K+wfrLMAW4NqVZr0UGV717WZoH3PYOT32d/AiA7QRw8BNUZAV0r0bHu/eAfwOEKq0kIbWqB/KwBMpv9SwE0eASJom30lYO+hjd2MO8s4Zwyc0Ze67b12vPCuKFnSxSs5pUB8AKYVytME3nggm9TlEJrL+HVTV1XdtkrAoXvZYd6BMdJP0ofT0HYWuhwCLPb56ap3B+9nWxhub798UbTsbhYdMOediCb6oLvSim6eiSjbtLWdcoYZK7PYHug9a5AxoSrcGaDhjIUEpXDSMhhwxznzic4pZOqTFzeYfdrpu9gPvQeQfPQpCVCCBW2SXQTgA2lEjxId2s9LEeBA8BISCN7biMJt/tAr3hPydD5QWnqC9HIfYHGElChLZJV6+L67gODs6zsncIx30OegsSpmN9TVskeJ1D49hS0MwGIMAN+PGpSLFNkugyyqrZq3yyvkGvWynE9tlvup7cM79mzK7WPP5fLr+X0sXf3+jx2n/Bq1MK+c38fLIjfT8M9a5jRBB67YVOeTaitPIs/qXiqjniynN79cvrWSFn/5F3+Rb/+97/0+5jblfb3QXD+/of7+0Mjb9Tybx00KNCnwx0+BnwYWr98EFdLEmD8DdwkTk1JMqjH9xiScp+/7E2pM5jzuB7DaYvJmDi8nsGBIuRaggEk9T2wfAAtWDMPWAkYpPEO5+qfhM+AiG2+HjQF5ZWChznQYb8uo8stqUWFjIbPDRz2pJNFW2+TkD7AIr1Cs9uuxKeJYsCJ6xXGoQsGQmlamI/mSF0RkENOwsWioQgEqqEdIGmBEAszItPBTehGsNgyZBpnJy1XyACN9ZG4MBiag8KdExFVit1D94mORQUXdI1TQnTxDghHgAmYjQEeSXhwH00KkaVbyt/YwssUuYAeA4er+OaoUqlQEPSwod12Qh3YLLqLuZT3KvoMXiw9nBhTxkeaez6dxUBsd0pq23BiNG3uLswiehwvAdphxP7nk38qqdBdMWTtMMSKD8AYl/eGsokqyVgGgqLNMcxg9k95StI+YFJTQRxpJ69VIt7VGiX7I6j3sW7iIPSa/CRjmCa5ry2H0a6Nqv2Yl/wAg2Y1k6WuYfSNS99CHO1wTJGhMLkjRsMpAcs9I/4Kfge/Ull+GAdYTk0z5LuPW7QHXVlilHwFYqG71RGCBZOA5htv7HLcxro0l8eXUOD7YB0Lt6QDmuh0azsNU673I8SYoUmVL17famOyqooTqzB4SE/t7hHHyCIZylGfsIdXAlKw4Ri94/og0u7R7g7YeU99LGFED2U2yeq4qlvYtSjJ2WXU/4lnLa+Wa+voaJSsFUsogsHBx1vdA70wG3NOGxHgTEWAPSZiqUOfQC7FhMMygdTxKoZsMWFwAWHw5SoA7JBDWW5e2etYSLPmeGhdjiza9w/6kj/N+jP2HkR6NMRYEg3NIf5TYHPPeGINEm4Sd9XfF5otnxSX7NgB0N506YN8hrdG2w5glbeQvTSdwRzyH7cHC3FxEx/b9MOK1bl01hHb8xzjknTVKtWqIvnu+Z8a2uIaGjmuBr+qK3cRrCdUzwRW01T5D9jFAhYOSTSbV9zykqwB/qBdt17W24CFJJgQG5AEwMm7FPpHDdTcbedJey+QPdUnzapo/fS0SsNDN8S31dL4Yx7g+7Csw5B8FWPQp/exiXNon97bqfec6JURdPfgwZXow6nEvj/un5pO3nM+nnquXn59x/6n09TQfO27kV9YCmpmXZ4176clIUVU2HeSvX3riIyXYAWV+QadoYCrD1Lm9+UnLzG2JOnDjjzHydm5vc9+kQJMCfxoU+Alg8aR4CrBwvsybjEPenBRjQiZB7FkVc4sJs0xUezSu5Ik8T6hlsphgA0CUk32AinzMRzWkD3woK4mFTDcf1QAWfIRVi1JqcQ6z709g4cdZMGJZASxk9gUWqD+E5ALmpBMmQaZFSUMYI5dtsQ1hzAjQIAc+7OpJJ1Uo1aEEFyGxgFkIVSjSRywM8qmMwikrxbBIxtshGWGlNdlYpL3BvmIlVMaZsgJYRB2gIwyPLidtR1J/QK2Etsi4aL2v8XaACq5JL4jE7dRe08Wx4IFr6TyDNOmi5EJGB4ZJ+uoZgNXZQ1zRGul5CwbXwGYbezDUSAAEF35QyZZf/ImuCzrBnCapi6u4JXNDfcIVZ9SDZ6yfWzzLLp1Ve8eJcSrQFSqukVxEVG5sLS67+1jlxi4AOrUALNpkzvUeBVOXvAXRBsGF+brSa0Yy+zCuMsRKdQQIfdwTDPRzzVgP71lFdtNF6Vdc12bDYGy6Xx2knAkY1TEYT700CSx+AGxtIsUx+69GBosVGNph0sjsWrb2HIIUpSYGknvJ6v9T1Jm0tTAGxSN02xcoRwPsLcoQ3MwDeozArcRCo+3nMO4+8womfg8G/pp2DDKeFon1MEY6V/u3sYM5oR7djLtxVK/GuT6Chym9QWm8/RbphflrJC3jeXOBVylosTBMtGWAxQ1MvDElBEDGgWghDyUMqqPpOlf7GuNG3MCYh9SBunXRzlvoIJA+R4qge98bnumGOfe9kek9pexWrktHAbsgbobfIm3Uo5QSH1XDtsn/BHrcQK9435G0hISJfOYBUd9Mj4UUxwB3O7TjLbTQY5bjzNgWu+Sxurkdamm9AJ/hsdFw82t/6S3LKUi3vapL6V1tfwuVsFdPi6NXL0COBwEshqBFL2l1DuA7pjqW3rmmYbrnWMl3NV+Jo04UdJygupFjTQkFFQEcquZIjBVjrfge8p47vtPCh+6hE3iIhYPyfXfBwjQCBeOMxNxEuebn5l9/vltOyvbdmV7hKF8JrDZevrPW6YAgeQdE4D6EFrE40MjGrOJV4FLU2TgWSn9uXZCg/ycncDMbbQRYjE2EfUWnwKKsR2RQ++MiSzmtR3/FrSivwRTXkqdD2xWb9CoP3eXjfDvf4ty5pb59qj45zf3vh9Wvio1EKb9E4lRwPU9qH7Q2ab3kyCPX0wqXmdafjWfuFhYlppxyblUmZFHOfZEqw5PIpXyusWsCiwYtmkdNCjQp8HlS4EeBxd98n4BFY6qMteOqpU7uTrhp0s0TqVN22rySJ+R8LU/U9z8MrrLVgUXc9xqTsh/1rNbk3vNLV/T4yKrbHB9imQBVoUqGSheRAg+BiMvRGlZnNShVoe4ACxghJRbRFhiaYAxyuzj3++Kq7LUSC/zgJ1ChSpSG0IlJl8GPCNnkY15JYgF4gDFr185CkEE5ASpQgapLLAQioRJFHQMokJcfSukXqku0TQPOoDdpzEsf+SAfufgANjIYqm74THqQ67Q9MfSJ+kFfwIRp0y/W+EnufY1EdXOLXjvG4gesmO8AMN6hArOF3cXeERG6YQpDLUqaUlZs1LUV4+MAZTAvSimiHBkx0uXV01QPqsZ9mQifjp9/2KLeHpPHNfYW16iRXBDf4lJw0UOkAkEUIKCVVfIWVaFov4w3AyDAVyIYuVCvWxixMKSH7q0wnNpG9EGrGcDAKIy4xtPaC7gSbnTsBzD9g4wJJSsbMMDGbNAGYlF3pGQpeHgOg2sANxnyFRjgBxg/CyxUdXKF3JXvORjpfvKhRsUq+T9GdUhXricw6nPUexYmW3e3em46YdwY50EDauN06Cb3LWWvAmJUvTLeg8C1HbuFYSQbfbabuh5Rh3NWq1tIo1RgHFAxDfBQvWoPRlxJzAF1MgZKqCYBlgygN4E0YIB6C96UPmiQfU35LeTRC131WGUQPOuq+kwH4111Jceskplu6CdjuweTLT0MxxZjmjSCCcdmN2NBumn4rlG4gEYVsX6AmuW9pW6r1F3pyAXPaKNwg0TiFne8PfT7DJKjr2Ynii9ok/2hmtd7AJY2E6pTGcdknzze4cnsHDooNejDnsH6DVGOKlGD0OqWugosVKm6xLbicH212HjyuLjc3UI164o4DgQIRBIjcLqhDoY1VNVqflaDZoJt4Y41zTnMLYwlJQeCilA5ZNy1ILVSBUxA0YXjgQD6vuPklxcGpEeW1IUUFLqEdMN5JOas5FzCdL6Lbmn+0WVx8vSkGpZgX1ur05CaMLKgg2DjEHUoPUSdMQ6cE+N52u0/3wXfU/sxAuQxZgXdAgvjmMzNzMRvGJVDDbfTAgVP8tydjXrUr8X76lvrwSc26563yK3Ms56P96t0Jq8Vm6/fT5/zzHvTVWnNgnKiZP44uzSaUss8Pxz7XM/UxlTtD9PmMnykkScn5eP1+41JLFL7hzqW9Ykz/5RlxERYztfVvabEokaK5mGTAk0KfKYU+PnAghnSKbE+4Tupep6uNSbl+nRZTx804pk7k3FJuPvAwhnZdAEsZCr4SMqw3gUWSVIRAIOPf0gtAlioOgRDILDwo8rkLqMpw69P3w+ABcxCSCz8MJPOD4gf6Ggb11yxsx7aWFyVwCLFsiiNKFkFNm2AF8oIOwv2YV9BebqaFFwkUFGCC1YyQzWKsgNwsCKf1YmiXBl2SSodBAIwD5neeWU+6CiwgC5KNkwedeeP9+I69+LTVp6H/UUAC54RcZXPRJooB7qdE9kXcCHzkiQXh0guUI2CAVStQoNQ+yKoRFmCCplQCxdYyFBV4IJyow3sk7QrlUrtvBx5+De2uEDdBQ0AiSvUogyed4mB6Y1eozAybWOVuJ1VYvvpCgb6Gsb0hn6W0YvGk2cLAFMG2pXlFujcSl31+GPcA1f3XYHfg2k/ZWzYZ1Mw7YNIsdp51lV/GVrtFxYACgMwxbJtqzBw72FmBVeT9Ok8AKUfgLPJeFNdTFAnEFHdSRUl1ZI0iNaVq7YC44yBKRi7MewAdl21pwzVs1S7MvKzXq20dTBYna5dVa25gclvp45dMP/GH3CcyJQKqG5gNGWGB6n7GPYAxoY4oC7b0OSMvFURawN8Ky0RPA0IYAAWqi4d87xenSKgHu33viv2dFqAdFW3NHQ3SrRSOF36GhwwvEOxSm40bWknoAwVQsaqEkGlQkqANApnRITalXSchomX7dsAVLyAhtsAK93L2o4rpC9XqN3p8WsSo+almcniCwCG0iKrtA3QEiRcWjfy2kMV6A2OBgS9RmsfGBmJOBhGEu+n/ATAsGOBnt2gwlbG8tXedvH+h98XF1vrRTeSuWHy74HuHY5Z6NRFOwag4yxqUCPE4NCzU17ISBJA+XLeZwCvYEbwYOUcg4KNJAUFwPqOUw8lnb5PvgO+C+Gu2XnB52i377ITr4DCdzRsJJhfyDTy9LUwTYB8XU7T5hO8WqmGGapRPBORuPUYxzsarmfpD+nl3KH/cuufXM2ytw6U34W3qohfQUTxOX56hNJwu512OXe4+fz9LV+J17W8GXPP/YS1e/mZqFR5/U7e0CHy80+VWPJQ14/U4X5RMb+RNm9QLh+Sr3l42rhW3awOfLbx/MfTpnm0eoSDeo7RgpxF/Ub5QK5e2dLyWRJGWh60rfXMOf5TkFj899//78X/vfbdvZY3T5sUaFLg51LgV4PzxT+f+BUu7kd/7iO/KN0wasHyG3/o9pPA4hmqUDF3lrPk/Um/cc4UWU6yodJS1uj+xJk/SPUPSP5IBLignHoaP75Z/emOjUV8OFWHUorgL7maDQMzVt5d4ffD6gfW/JIqlMwBsQ/4ZYlFCpRXU4WCWQhGwPrzdYr20YgAFqgj6OXlEreVeoZSXSFJLFz5b4CXDDDsmJBWsJIa4AImREZEaUMdaAgwAlx4j/KzClYcl5+eAAkwH8GwQ+gGaKBgGI38kcr0TgyRtIS5iZ8MDcxcqIdJE365w8oPObCAfAAxIbkg6jM66Ud4tBFcrG3voUJDQLQj4mu4qg0j45CQPjJYgi8/k0lCkfpQpir6koQaMKeNenAuG2X5if30mLzMMDbygRG6QkXjhhXVK+0tCOJ1i+1FG7ESOmGQVL26ggm/dGUfsGNOFbigjSkbmSzqB31cVe+BAe+Due4CEKiypAco+7gfxr6L+uty1IBxujmVvlOAgFGYZFVm9mAENynLoHp9PDcC82ncggPGmRICCItUAi9PMK0aRmug/BIG/BXPqA41DPM6yrhzZX1fYMHYVIqikXgn+23yMaJ12BRwbJA6jZs7IWwHTHA7ZckcaijeTn9GQDRaJ8PbA0N7wlgUMJyGyhpjkfYY20GpgfYUMu7anNgWqb9FH+7y082tNgb2jwCsg73PIrB+AABAAElEQVSqWfNIZIwaL/MtOOgWkFDGkwjcdxIes+CUY6xKW96gkFQIKpZRFQvgB50EUPMCI3htDaqfAbQET2e+1/Yd4OucPHtRbxvBIH0GqcEyAG0S2mu3YnRx+0SDZFWhdgA2r7aIqE1E9XMG0TBAYFKg4CRInrxIAI4UTLHbfiXSfNvxQbH59LvicnOt6Lk4AVhgkE5fODpsm7ErxgjWNzk+RjwTpGGU5YKEY1v6+s76vnYonQDw+v76bjqOHT+qC5km7C7I1/sxb8RwZgTSv+bjMxRZAo70Lvtu6kRBIOEcY5a+F+YdNlbMN6pinSK1OANkqZrlHKAU5QzHDvtG4kaK47yXkHsa89Zfaa5zIAF+qR+xPlAdmwY8zc/NBbAwMJ6G29adoZLqHC/Ox/9Em8pb1uEP2SxHmkVDObS9UXgcc8KtOkjgMmlM5GPxdHWer8e9nAknMa+Uab334ZbmpQ+TQLuyrPRMKi8dpzrEce0w1feDFKld6XLUpzxk5zfFMzKhrHoJXv1TABb/9b/674r/8cn/ZXObW5MCTQr8ART45xPfFv/Vyn9U/Gpg/g94+qcfmUGqrefLP3T7WcDCzPOEmyf3tHeGzVMj+zzhMnM2DtN9/5pHzifvI+/avZzG5z0OYMFHUiNsgYUfNIGG0ojsbjaABavU8bHlA+vKnkBDLz5+YM0nAwsZq04YeYGFq61pn45dnY3Vd+pv+6yz+yyx0IA7AQvVoJLxZAAL6mPqYLDII1ShzJsPdgALGDSBhR/3xIB43WN+1CMYF+tDvZLkJK1uJpuPJEGR0Q1QIQ1kQDgPGtI2Cd9gyjkviR/se0nzoKX0lIEJOgouUh6NfGyHfcRqKu5obaPqGPuHB2HU/Z4I3RvYGmwjuThDN15/3bGV/e1pBgxeD2DhRevIPo2ENAZ80p/jJ+3jJF20nl6HJkoulFZcDY8X12NTRcvkFFKLflaGGfTmDYOl/r8RnsFNcUmbCvtN5swAeNrAhIE5Kk8COF37chuGnLEA8yljfYYdyZmSAJ6JFV6YwAGAgiv9Srm0KzjQkBhGFwsPXMfKQLaG0fUpjHILUhxdnirJGENqofcnY1yswbzrrcm4EBPcl1lWlUiDZoPfaajcDgowmrXRrzX2djwbt8RI0AKDPhhfXZfKMA+RRw/p7ccd8jFv6RyxCmhH0vBP7mwFLZNIOgQznQCbDsprh8l1nFqernDfYtOhV6Nr8tJdbBdljGtUPjlW9ANofE5JU6hrASy+22D1n2eMFyF4UOpwC5ixb0dgXGfwJPUQY3PbfwUtBQXzMOuCpx2Y4u9RDxNonTH20OtCPWm/uMB7VS+BA4fHxopJgIW2KLPQcBwwZUwKpSsCMRn0Y97vDTxevVzfCiDVi/RqGkAyBJjrhC4G79NN8CF1iv5FythxelicvH5WtO1vFgPXZ9GvGtwrBZUmI4yLGdzLjpGPkUxDMkB/U/0ABLqITu8x7WDcJWChAhXjjHaqyqhNRrzbnAej7jvh+CSTABZlGse7L4nvXhRALoJ9F0C0u0gSxca7rSMGF0+0yxBU6EEq7Migg+dKMvQSdYxUI6QglGex1/RlqELZDurZDpjuw9XtDN7B5mfnMOCexSPUYIAlQbr999HNF6V8Q9OMmFI5Z8S8ce+h9F1opLl3+94pedy78uGFuwliPrZjful2/xmalevfmIEaRyn7RJUgARdy+kwP0yTqpHT5zOuN4qxro505pWnc7p83gUWiS/NvkwJNCnyaAp85sCDy9uu35eTJBOkcySzrZJgn2zQ1ltNjmaR2k8PG5yhPzO6r45J2+VzmOabiMk0ACT66ASy4l84/BBZJciHjl4y3k8cUn0sfaZkpDXqVVoTBNoxeMP6xTzYRd4CFrfQ/z1kfSuaDzcef1XwZ7iuBBR/3pJqQmGbTZjWokFbATFTuZgUXARxKMEG5MiQR1yKYjnQsKMkMSnzwS/plCUUGFEEn2pa/YCQr+6ciaNA4GJjqEkxHSUNVp2SiM7iIfAQawfWwN96FYA1VkhPAxQHgYsvVamNd7B7iIYjrMM1G9JVOUsCPafRdLq+kfWKwvGgCUwgA/Nha6bSlqxxHncpr0p6V3itUNSJw3uRMcTu7iMX1MEvoBCHjcX1GdfKMEbJl/gUFrrKbn8z3+TFMP/eNV9CCjcNVVFbGGya9VEPSW4/t2YOxlykLFRfGiga+eg6yT72ufQWZFjiajTJuKF+VJ9XDGGgYWgMUeKYfZl7PSTswt4cwr1J1FmAw1Y2LVPpdNSztMlQ/miSegu5cNxm3r3BRq43EKeBCMKTdwxD3B2mrDK0qWWGzQFts4Ts8P2krckQ/hIvRsm09Mssw5TMw6BlY9AAQwgMSdfZd0AbkJRGzn+Da9ghm/4rzFujVRTmqVi3OTYeKkcy5dB4HYBkI7ndrG8Uq6fWQ5ftywpgw6raDYAbPU3OT48UD3OMKeA5Jb4TzJeoxAIBSrvQd6bUj0RNVi8CCmB0X6xt4+8K4nCjQE3OzxTz2IDP0lZHBrxiTeszSIJ2OCduqI4yan71fL/boM0HPDGVOIGFRunRIvbbpx3UN7WX2kQZ0AixuVl8W3Ue7xUjLZTEITQWfvrvSeIxo4Ro0D+tilt6SSXdBIoJrSlPGoAsBelXqpi/aWRBwjks2XI40DNa9Tx9nN9CxQEAa5zVtK3yXlWRWsyGPpTnPchgj8V4mcBHvts+Rs3W89D0sgYXSixP63X1IZz1XVYp3NLxIUW8Bv/UPSS/HSpC1r+hHKjMLqJifmcXWYjpibhh7IyQplPXJzYqUm/XPc3Xe53vufwnj7wzwwfaRSx+k+aUXoGXK1r9VD3AtF5Zonc9Mk7a0h3yxpfY2UpWJyp2J/HmfnKtk1UGZ7sNdmX08+acQebspsfhwDDSvNCnwSyjwWQOLv/5eYPEmpuI8CYeePDNtngz9kKTJOn1w0rTKlXI2dp/T1j9EHsePpz92PT60pJEB9TipNLlPx37UQ2oBg5YlFkopdDdr1F4/umGXAbNk/n7UZdpDDcp9CSxCYiGTzy9JC6gvFU719iCaF8xhBhZKLa74oEfEXOqhBMDNZ2QgAlSUeab8yxVPVz6jHEGE5SUwURl0+6zX2HtN9Yka8UKlxy+WNLbMDDZy2VHnqDw0DUaFNK6MRhv8k2gt05pUL1If+CGEyJE2AQ2eAUSBLKBhadANM3eIWtQOzKjxLt4hvdjGHa3B34IBs1wLin0qJzFLZT9T79hi7zXTpEv+9VAGPDZ2jhr7Tcqq4nQ7gCrUxExRLD4srkbHilsYUT0xDcNk97HX8PgYkHPCT91/6XMBU3uOdMXV8i6YepACkg3qSL5G0B4FWPTCfKpGtAZTr9cnJSXaIXQBBPu4L8hIbSIOgPnQvm6yUNVIKYmuY3VxqmqYQEGVHHXptSE4Y9XY+vdSnyXAxhwMrYH3BBYy5m4yzxo377Mq/QSVs1f89qgzrcfjEHXEHW0AC8YORYTtwxBtpsnFayQHb3QLDBhh0FBNGFcYyTHqoK3HMkbdQwAMpSS639XTkuDCFfRXMPWPt3eLf7u6VhyiVnSFRypeKFzIEm9jZLSYWphLEaphmJWSzFMPmdzvABZvsG+w/gK4A87PKV+QvrIwi43EdDGHStFbjP81vN4hkvgSqlFTPK8E6PHuHkbcp8UOebXCIF8RP+V09X3RDsM+RL9Ozc9HwMJpwIi06aLeSitWeUaJ1BVMuDEdnvLMJjE/LpA0zc/iIhYJywJRuI/oy/cAjhfU0T5Qra/9jJgl714X/Sf7xUQ7QRGph6pZN8wdGnCPY6cxPzuDcXs/8wtSIDwuudrvqr+A0j4UWAgqDHznAoFD9TgYfCVM0E2JAPY//YND7Bmb0N33mGES80L0TZzZ62w8n98131Vv+WYrjUzvDW8Dc5cLJCGpQDohmPCneqI2Fs5xIZ11z0+JhXOektqYMwUVlMNEUnQiTRpAbWyeGB1zSCsmxyYjKrsgyXnjl271Obv+fL6e9/V876TzhsVGBYMc8adek5w+JbEhd7d836sfK69KTQY+bZr6M2Wp3qkn5TjVIqWt38vHeZ8eS+lyzdO9T9anrItP3q1LyutPAVj8r2/+TfG77Vepwc2/TQo0KfCLKbDUN1H84+EltCAGf/GzP+cBv3N+v/7Q7UdVof5KYPHqTZl3Y+L0SMbPibExOab7TqsxqXrPdLWPVlxPX4nI0/NIH2fpubwi770AFe6DSU7SBxliP5xWXGDharPRtiPiNsxZRN8OqQWqUNw3vSvmrsrptSmrQakSldWglGRU6gykye3K9U/2A6raWB4fcO0skFhc+jGHOYlYFrRB24gKWJB/JbWAKQkPOgKLUv1JF5+Ch0qKUQKKClgEo5goLSGD3lArrkA0KWe7Mk1jLx2jT0paSl9+aYsn4jCDjlATg1GLNAEsTJsYm2CnlGBEm10xRRKEyowMjfEBVte3i7cbOwRh248Vexkwy47yrSXlplVf8sv1tS5RJ+sfh6T0iI1nBRaRxPMyQdy1T3BBW+DF5mZ+pbiemilahkfCc5B2C6oLme5UcAH3HUw9q/hngB69B6nK1EWMgzaYe/tWdZ8Jrhl125XrLZhkXcqqhpQlHjLig6xAO2bcQk+dY42Yx2G0lXio6rMNeNFO4chxCFNnPWRELxBnqCrVR90mkHrMU/YMQGWY+kacBRjGMyQaqhh1w+Vq7/ESScArJELvsWeRhEaIHiNmRjeryhpQm/kM7dAdrp6rXrv6D3O9CRPtyFDdq4XBasC/lfFhAuuNR/1lb7sYT4IobSVUvXoGKPk9ak2/fb1aHAMOrmCmBRYdrGr3ohY0SHRm3w/VqGZg8JcwepcRfox9wyqgxCjnhMcr9tc3MUy5CHe4v36wXCwiPTBK+Eviobygfq+o3zCAbgJ1o1na8ha1nXUYclWxOnhvr0l39G6NuqMmZlRoGN9FjLinUccah2a+mwI4Xc4KLM55n48Yiz8AiLZpNyKM4pvlueIB5c4P9eG1iqjqtO07wO+m4J/x2nlyWHS9f1MMnB4QV4O4Hv2ot9E3qtD164IVFax5jJl7BJ70y94eYAeph8BCcG+8Cm0rBA/d2IIEsGAchcSA9DozUFKhIXSACtqv9MKxk20rsn1FvHu0x3cu5jVUnSLejNfo8yRFFNQQQwM6HaL2pYG26k/G0wiJBRIK41gIOAJMxBzHQkCZr2UIQmP+dPDS1i5AzxAetFaWV4o53p/xUSKLIwm0Xh/M0T5TbvleYx7Jdxr7nCauWIfy1v1ncrrqOu1tbJ7kJ9O3wzkhrpFn3mqHVb2r/HKi2j5yKB+KXZRJ/smTRJUylcDfqqhIyP00JzVumMBfvm8WHpunx6lvq2ziUgPQWNdIlhJ7N8ZCps2fArCIRjf/NCnQpMAfLQV+HFh890PxBGBxdyIsmduYSOsTaqQKQlWTKpNnvlpN/rUvg9fyNB3Hnt/7xceXa3Ujbj+6ASpqwCLFsQBgoDagxOICxucKoBEr8wIL6hI2FTDwobYA0yS4EJUJABLISOoKESDPupeTv3vYYxhJAE12Nyu44EMvsHCV0IYkYAGAEUgEaCjBhcdxzevpXgIWHsOAcC2kFzAxIb1gH/YVJY1dNU91SuPQb2JFpwAEsORy71Hf9IGTrsFoBIVz+mDd07OkvxtIz0wFEvSBecrmcyybbDr956efK6gw4ax2v1nbLN6i624gPW0PZMrdcq8nWw5yMk//uY/8PYvqpfFRfuRlyC3Sn1sqn/Fm+zHkLvpHisvp+eJqarboGJ/AlgF7BlaGdU2qDQ0KRMUxdV/XrgHpwykr6Reo+LSRpgeD4F4Yx376YYjfJMy+nqAcn2v04zor3xoVK01opQK6aB2M1X4Yf/rPKp3RPqUW2gwY1M7Vf71IbTHeNLpWBUfbCQ1mISVeozpwL9sdNgOCigmYTaUHelfS3axgBJ4v+k13qm8wZH6DxOLt5q6DiTgRXcUwzHao8cF1KplZxlB5iHy1cXhJPIe3eEjaBiDERrvaYHCXkBgYVO/XM+PhmSm6hQboCreXNusFyXgZAov/99Xb4vDd+7B14IUpugAVAxOTxSCMuv1omRqxz+NRy177AYb9ACb3gjGhbcrmu/WihbZPc//XD5eLJYEFdNuG7kotngCSjLotcz6MFGIfWu/x2+XXyfM3qvFsboWkxWjVc4sLxQNsHabpL/tJUKEHKtWhfA+l0wmA5PHb98UuwKINWv5qZRGbEAK/IRVR/eodwOcpgOU9K/tGq27D5WwnAfL6zw6KkbbrABbdtEsbFiOzT03gghVgoWOHc97rfeJEnAI0lcRlSUVEuQdcqA7lOwpHzrvPe0F9HKcCj6wqZVuTZNL5hJ+LBKT3Hc2LJL4bSl6cP3RpG9djlDHu+ec7d0L9Nc7eA/SFiic0E1AcAzRUhzpnvPpu+YQAPUCCw4lfgApuhXc0+r0fda9x7JMerqwUU+NTxRCSFW28lF7l99UxlN45jz7cvJfnxDt36ZeY56Mu1iZtH+RVfQzuPF3OW3evxVRmrmX7vFtm30j4qfwaKVK94tlUq/T1sh21RN7Pp9VBvmC5+aL7fNy4n2Y8MjRP0qbZrZaSaxXdzKtWeM47328Cizpdm8dNCjQp8DlS4KeBxUuAhXNmTIYxc0Y743NUmyC9mCdH93em4HJizpNoJpTnkc79j/wyuBBQ5J9xLKx8qETxcb4AUESAvFJ6obtOP9iml0EIzy2uPvKR1yNMBheqvSR1JZh/7qk3LpNatSU1jHomYHGhjYVeoTKwcCUVRlLe2A+7gCADiwxa8rkAImwuYExaBRMBMriGXrsf+dDD9p758JOhjD31cp/rJNGy1CFc0ZbMTbpPz9gvmebUPz51MnEAowaduR60kQlOqhMQKu5L76ymkb7tnHuNdC3mgYqUUov3MLVv1taL91t7xW5Wi4pyHSdprOiJKuUXtSjzL/u7SuUokBmqqs2Z6d0lhqmVgGS3vQPF+chkcT0JE4iO+AzB0WTcZUAFArcw6UcwbE9RhTFy+CGM+jWMmIbePa6AlyvhU9hBjLLqL5PfRsetI9lYY7ysnWBAjdTAlWjBhS5Ip8rYE6rOKdlwxXyJa3Pk0QeDbjC8Ta6/g5F+re0BEpILGG09TBnUbg7d/0es1BvATQlJH/nYMvMRXOjpSPuQE8aykoC32K+8UmJBfxvZuo/nHV8y4RO08yvyGgakGG/jKXYGb1i530ctSIaphfStuImbn5oovpqeKP4JEgvVsxz/Z+QvnVS70lvSc4EF9hX/+g0M+uq74hxbByUW/ajJjKHONDk5Gu+XweOk0wTetPS1ZVRweNGw8XAsrL7mWdo+ifH1nz1cKlYANBP0iWkFWs+QWLyCJsauuKaPXADQFkAaq1bWoiQMAHIDDYcRvy4sLRRfjg0H3Y2uLdBTIrTLz3dThllg9vztO4DFXtFGnb9cWizmUYUap18Et/vk/Q6bg3cw5AaTu93fLTq38AiFStRg6xVG5kkFDkU33A9TFrErZohE7WuTVI0O2GvTgV2WYwcVqG4Am+DCsRRujKUrfey7FmqW8S67kOB7nRYsfP9DYsHedL57LnQ4J7nooUtsVZg0yNZGTDsZkjnoI50xLPZpwx7tNAChEoozxvMhgEo7EIGFr5lzWzgncN7w2LLMxswoW1fFY4Cn2bn5YmVxGZe6uJlF8tKYU6LQqJ+PfWyz7ml+SW32vNosx/vlhXznbpoqtc27u/m8W87AwztpnC8iBX/KecE0kb58qLqf07mPt6K6EHmUGae6pWejb3KqRkH5Srm3gFTInXZx1W+hW+7jqGNJk7jhnzLfeln1fMzBe38KxtsVTZoHTQo0KfBHSYGfBhalxMJJr5w+q++C1/KWJ0bP65On506gMS3fn7S9zi+tupXpyjTxjPdgqGL1jX2ACpjb9GEuVaFgWLStqIy3OVYd6owPtmoNqk3JFPsRbccbkMBCJjGABYyWoMK4FkltKTH06eNcY+RtA//0CnWhtySkFrqbNZaFrk4DWHA/JBaWI2NR/rLkwtXQSmqB1CQBi8SIdIR6FMBCpoT6ZcYkgE4wC3DM0h8iB80TMRNtSlBgG3P/SHO3REPpl4BBNhL1enzo2AeAkMb8PA56Q7N8nlY0U34Q0lz5j8QIYHXAyukOTM87GNR3qEWtoRZ1AlN9pc4KWxoHlAEHFuoe7GXGct+mrqZcc426mHtiivybciEfaACXjSsm4jUMjhU36oePTxbTU5MwvL24bMWVKgCRDigOGA+/Z9VfFa1DmOBbbAekZx+qOEvo4q+Mj0QEbt22DsBgd+hJCOZ0AynHGxj7J2tbxS6M/QVMWz+M+goM9hJGyQPYLBjVeo900zBqs6ziD1KmHos2ubYKiHmG9GafFX0NoTuo08LEeLE0NVY8HAMMRB0BM0gbtAdRbcrYDKr62E49RT1jlf0pjPhzwIUxJLoBLz2UewzwaaUcDcC/hukepuz3rKg/fr9RvEaNSBuFW4A2YoGiFReiw9hWLE6MFX82Nxk2FkpWzmmjNgsadasG9IK2fk9Zv0WVaZPnz5BAtdLmsfl5Vu+nAQhj8e4ofREEaUQvONnmuQFoFjYbgJQnL16zqn4UdiC//mKl+Aqjb9WmLnlmF2bYKOLPAHp6vNoCUKg2qG0DLw92DjDepLtGXekWCZMqVHOLs8Uj6j8NkBmQvtQ7RRTHSBnQKPMtGNwG0B5Q53P6dxwJyzBqPgNIcwahWRv1U6KyurdbHOxsF5fbm0X77mbRdXZU9GO8PdzTEWphqodNjCHpQO1rmtV85wtduB7jack5RlDQCb26OktgAcBwMcAXUbBmvzmvxLuN5NG9CwcuVvissV2cE+LFJXWoBlIv8xZcqV54oboW80ioOnnN+4wLA3I6vyi1UB3qCOlbBBSEBkbkPqEflFqEpDLetfS++c75s16OIUHFEDYkc4CKxYUlIm9PIbkjSCT1jLmknMLTu0hGtS3e39qNSM/9mD/KdNKgzCL2+Z31duMNLhNH2pw6XYu8qK9byp8c6pnEHXIqr1V52sb0VJST6dDIJ25+5E/KKOdXFp2K5GK+nh/M99P5BxXLySoaVCnMq7rbOKi3Pu7XC6SwpsSiQavmUZMCTQp8nhT4+cCC9qWJP02NTor5Q2PT4yoTY957rZpYy8kzAwjvueUPlExsPs/X3PvLoCIYXj+4fJR1y2jFw50i+wwqIpYFK3+qDRh5+DyABQwXTIDGkxEZW2YBRjNJLNC9lxkAWPiLj7EfZNrhz/blNlKT+Nhf4BdfYBGgAtUE3UCGKhSt9RmlHfdBRXvk3wAWSieSqkSSWiRmpFzpFJCU9hZpxTMxJ6E/b338xz7oBH2SG16YM2iTPq61lUOZHxiUayQMYewd0oaqV+yAyCekEYIK6Q19Q0WKvWUILFKnQkauUUj8jIOhROgUxmgDd6GvZHJlymEiz2C0BXTSM+hnvmX9ZMgS0IkB4J8YJ/5NsMW2WWSqGxnECnFBsLwbDLhPBRZDo0UnNhbGHZiBATUA2yhMt16adMH6N6gSvd/E2xGMJ+GJQxI0iLTioavhrKjPorYjU6KnJA3A9aKzA1P3lrr/7u1a8R5JwDHSB707fbUwU6zAaPcjfXgN87sDsyywmIO5VwKxz3NG5H4Dk/6SVXTLNAZFL5KDLxZmiy/mpoql4X5sJLDvgZEd4llpcc64FVwY5K4NGhvQ7Skr0Y/J5wl2FreOV0CI6lvHMOWtlDPDWP1qAoNsAEJE9kaV6S0elc5hshn0cLkws7gQ7cXGQtepy0gtBlHZMkaH0qkHtHsSoOJ78AwpwWPK+7eUt4sq0jlgrAVgsYAR84P5meJXqFEJgIzbsY4RtoH3lB6cMt5HqPMgdeuDBs+fvyr2AHDdlP3NFw+LL20zQMp+DrUkQNYTmP9XMMZGFlcqwguEpILgfzLe1OtGYAEA6UEqME75y4CiaVW+qPsFwFn7ivfUY5uf7mvBDcXp9nZxwrg7AQT2Ag56GQ+99LGucjVYNzje6g5qehvrxdn6WtG+v1V0nR8VRBgpBrpx2QvTrb2JalAzU0i/ABcy+qcw68fEjHCe8T3UrkLbCZnxHmI+KFV0dMZc5vvOuVKNpOLIe825adJ7o+8uaM87o8OE7FbWd9b5KwEL1Jso7wgJ4AFSCPtaqY7esFr4qX6oZELHCZe8V6FeCP2MuH2C9MJ31y3mqjiIk5iDBKb9qJfNoOa1MI/R9sxcuJjtLG0rTO5WmxHShfJvNc9wHu9xeT3mnnzsvfL4Y7v0dqc7zl31rZ5PvOf1m9Vxqp1zQrVFNo3ccj4xX9Enja3+UO1qebme1GcbZeTnUl737THMKZeSU+Zr1bn5NYqMo/xMvlzdbxRc/Gf/8X+Sb/+97x3juV/zvl5o7v/87Q2bIr6rY9gkNbcmBZoUaFLgUxT4aWChKhRPOwk70aTJ5u7HKN2PBB+U4+TpfbeYvNNhdV5dLyfXNMEnptKpOQELjviAppgWAgtX/GqqUHx8k4pFigGgZ6gELPSUklbfrXcAC6QWGnDneBZ6/wkPUTBtYddQtrEOLmyAdSEnVhQpSzUoflcCC5gtveX4VbIM1ZdkSLLUIoGWElRQRlaVCEYkAIb3UIWiTj6jj/xKhUKQE79yBdLjcgVUiX7QB7pksOUHIH8d4z6MSVK9AFgALhITkvrQukLcoG8wPn4I/Zkf9PQ5rwcHTlmCAduZJA9lXAi4c89VSVEl6iX69us7h8U+jLYxITKwiDFjWdGH1imBGMuTwU/G8ZThsb/yj3ufbQl3s6jmjE4Wp8MTxLUYKNrxcjMK02TMgyWYyVnAxSXpjZHwWwzL17D/OMbQ+BZVEu1ZhrBV+PLhcvEVkZ1nXVGnPnqLkrmU6dctrLr5f41q0GvG/D5SGCUDXy4vFsuLs0UnajYCC5ncWRj7JRh0jai1zdBI+TVqYaus3l8gdQBOFYOAkT97sFR8NT9dzGIvoKta2zPiijqkd1zqGlVXs538bPsLGHCZ/R+U/OjdinqFQTHjrB3aT8KwfoEkoZ/8VL16Srlv8eh0ubWlDg/qV6juoE6kB6B+wIUB5zqgXwd910l7v50YDo9L5qu04jHg6RWg4ZD9BYx/K7R7NDtZfIvU4Z+wd9V8C8nIU5j356i7rdOv2jyoiqX9SR/gYv316+IQMNVG+5YfPSweLc4XX6KWJG0FTtqe/EC7XsIY64nqFvoJKnhhaR99C5NMIIbilnI6ABaDqCTp4cko3MOAMMegXqF2ATRKaYzC7Zxwi4rTmcCCfuoemyi6kVj00sfj0Nfo5sbweLe1WWytvSuO368WbXtbRcfZMa6JAX1424rgeQCkmUnsK3A1OwuwMCCdwOLg6CAYf99jVaAGsUfoR9Wrl3gqIYmAfm4uVriIkN9b72lTEaCC+w7jNOYTWDfmTcSqoE16t1JCosTi6ABwt7tbbOMhaxfAd6IklHfFxY92wK/zzwn9pBTtCnDhexuLKDzv++qrDCVNFu+SF4ybM4gh/jhqXg9WHgSoGAeACYKMHJ5S84DP+A+axpyQLsXfeG8dtGz5nunq28eeq9//2PH9PHKaXIbnn0pzP+2n0jWu1+srhdw+3qb0TD1NUDQ9wjP5jvTNx+aUy7L+1XXHaPmku8adOIk7+bnyJNL/501gEeRo/mlSoEmBz5cCPw9YMFvmj5dNzVOsE2OeTN3XJ9I6SfJkm55Nd+oTclx3Iq7/yC2fByNRMqUy0dlwO+wrXPnjp8tZ/b0rrdB4231Exia9H3cndpn+dj6q2b7CfdhbZOYAhqFi5P1IxC99VP1M3MCtJwCje0fsLGDmrilHIlLZKENAkuJTJIajAhYCBxiZABYyDIKKYByM6NsV4CKvdsqgBANDXqk+1sv6JAmA9arTLKs4xSoq9ShvBji4oW6VEXt5L/dZlUfuOe/zC1AB4xNMEeeChwB15JVBiLyVbRUO6EN/E6bo3cZmsWoQPVa/93Ezan1MEfWNrBO4sA/NO8YPKWIskC4BjNQ2n/E+SKtoYaX4FvWny/GZ4mh4rLhu7w4pxgCMp5Guv8Cd6RfsXfV/zcr3v36/WawHsIDhRmIh4zc8OlJ8/QWML8DCSM0yqKoI6bXJVf0zynJl/m8xCn767FWxjWH0GIDlEcBidm6muKL/XsuEwxQ/xPvQI1b/NaJ+QXk/rG8BqtaKnVdvY/XdPh5k5f3r5QUY9aliCiBwACN9BoM8AEOu6lY3/XnE6l8GsHRwxHfQ9uEJ6lR7ABYjS3fDJCMWKnqo7xRAZpn69wCmVC3S3ew6QOYccNHOffN2NXGAevfAYDuutEeI+Abc/wo1Kr0tXVPWXyPVeYIdyi7XL1j5BonD8F+Ed6WvZ7DP4GcQuQPq+BKw87v32NOwP6IdPdafsdzJGDiFab+g71sAScMrD4tZvDpJY8GXHrSUMDw2XgbPPqGuNzDHtzLHjKUWxlWLZXNfFcVe3KGOzM+HLcwokiglD8422p9s8twa4EejdV37oh9UXAJoL6BVPzYhA+MTeJUaCfWpYeikQf8+qnr7O5vFPuDihqjbN9hatKAO1dVJNHLAhe6E51GnWyDK6Dx5aEhtsLmd3Z0KWPTqphUA29/Pjyjw8a4ynnw/7bN4P5VSeM54EtC7pbnLxRAlie6RKvouAvAFF0pTtefQy5pOBg4AwJa7BUg75Nz5zGjrqtQ5h8QCAc8YkPAKT1iXjEMlhpbj3JDswnyHqAP16katTFAxBz2Xl5fDC1Qf9c/zCqlSRa1rfv+rK+kgzzOexbt4736+fj9dPs/7eIx6lqS5k8vHruX0dxJyYn6fqkdO27ifcr5Th5zozj7nmWuS6ZLO7+fnox/k6Tx177rP5RzjZknvRPd8z+WEtJmFJTclFpkizX2TAk0KfK4U+GlgUbOxqDeynEvjY+r6XZ5s706m6Yk8VXuW77tvTNqN45iQnZT5hcpMPo6Ps6vzrPaxol4HFRlYaP/gSvk5EgvBhQyVDXSF3DpqZKm/fe0qsipU5XaWa973Iy2zYHtk+vgfx34KYBFg1GDEyD8MuAUWMF73gYWrljKz/jpgWpMOdgIVgolstB17VhATs5JUobKdRnhCsj6AiQwukr1DVEhSxpYZ/ZA0sBLqamZ5o8bQwMhzr05bTiR6ulb2StVPJdNPBgEOIhowdJQx0gDcXsy0MY8LVMM05t4mRsF7mNz3rHAbofv0jNVl8spRwVOR6ppn1S3qazUix7SPz6sVkfBu0LIVCcUNwfEupuaKo8ERgs+1w5QSM4K+esjK9jeoB32DUbPM+zMY2H+zul5s4EL1DHWZFvTl22GyRrF3+OLRg7CXGILp3oVR7YEhdGV7AmnHCXYh7wEJvwNYPHvxhmCABFPDY9DiwhyGzBPFOfVZQ1VKFaavRlGr4tfHqrAM8/cYUL/kudP3a9CfiM9INybn5lCFYvUd+4xu+vy9alSs3uuNaA6gMQFTq6tb6WiXnUOcVcDCS+pgLIsdJBLnjNs2ymiFAe2j3CmemUOVq4P8BRYCoR1WuA0y142EQzWgxYXZYgbQoxqXpi7H0GSfd2KP1X5jUYzA8N9Sh7/dwh0sZVnuuZIEGPxWgM8sdDLq9re4q1Ui4/u2wSr+7wBqL+jXXaQWMqdKXTpgkG8xim5hhd+o3X0rXxQTC/PYloyFREfaCk6eABx+QLryHdKFM5jmG+p+y+p9K6D8lrxbYKq7Yd5Hp6eKmZXlMJq3/kqTNHZXLWiH/nqN5EMJiuACww6AAipjSEKGUfMZASAMAUxaqa8G6/200fF7csy4xMbidP09Eb43iltcyXa1GJU7RVZfIJjfIsBioQQWBwDR7e2tYOxtpxIL7XMGABZ9/Lox4PZ9DgY95gvGZwksAvwHeyiA5t0KQF6CC+jseySocHEiomfTJg2ynacEGAnU7PEuYT/h3MIzqvdFlrRL97g30C0ASgALAFq8i0hOaG8GOXqlGkIVb0bAtLjIfrYYGBgCUOluOWbrks3l8cjBN/DDrZrTqcfP3XJKny3fYKqYruZ7Oa/75/l6Tl+d/4KDxjfF3NP8/eHjVc2qWz5XXc0HXIt/VUXLNLTtg802lm2O5PHs/VQZVuTrpkyZp8ebNhaZMs19kwJNCny+FPjFwCJNmqnBHgcDXu4rMtybePNk73ScptG0z9d97v6x53lVO+/DqBGm1EprVxHggg9t2FhwHgHyYFZcDTSuRQAL7rvaZz1VhRJYCCrCaJuV5ZAoCAJgDsIjlKAiGPoPgQXKQQEsLuvAgvJUccjGgzKKKSZGBhYwkCGpENCo2pCARdhhlCpQyT0l16mbH/5g2oNpScyCgMc6xcfaj5f09SfdAkgk0BB2FjIgMPN+VEsqS9wACNLUNNUqKnQxrR9P2Y2cpzKEKIs8BC4BBGB+QkUKZinl6wP2JvmRjy5oj5BcbAEu3qE242+HoHO6/RXYmaeb1VWSUUktZMA49y672NylNrKHJq0wdLfTc+Fq9nRwFHUcvAPBBHfCzD5CWvEt6kHfYpS9A5D5nhXfv8JOYhvbg0tWgFtQWevG888I6i5KH8ZZCe+AljKoSg5UU5pDremAPnyNEfJ3AIRXSB92kQYMok40jYelYcCLaj1KYVqg17fTo8U819phfB9jD/H969XiNQbQ1zD4AsmRkaFiaWm+eMSz4zClukt9St4aeEv/ZRj8FdSxZgECjkm9Qmnj8RqmXanIOyQJ2jao8tfCuIS4RS90N0r3OIbNrTC7qgXt8jumnpaLaXsxQ51+9WilWNFYnDQabO8DLLZg5FdJp0eoHupnnk+ozxrXb2D8T6HZJfr9jD4A2EQxTRlLg4AjmXvun8AMf4c9xwuAwXtAj0yzNh+t9Hnb3mbRAY3bYbiHH31djM7NF1N4stK4fQL6aifxCimAwOJ3SIH2AQMG1OMlRVoBqKBc3HcVfayuTyHteIBUqRdA4vvoNo2NSUhOYKSf8uwbwMU6jPcVfduq/Qy0mVhcKsYBFmPQPQI20ld6nNIW5ZyFgHXS7W+uFxcbgKCt9aKdaPJ9rUiPMOJeBMwsAirm2SuxOEAtaRtAqvMHR2Ubc4UqUKpCDfDrQYJhFHTrF++qIIsx6lgQbMjcu4XUj/7zXXO8+9O9bATzpJyYm6Cr74HlGJhPWwrBxQHj5BBAfET/nAIwrIuSR+1kHH9JesH7SD5ulhnqk8wzOVDfZCmtmKM/hkfGIrifCx0JWKTZIR72Dy9cvIHlnFK9iFWCDw/i/fTR/NKWSdK760kJLMr75atdpooiq+N8Vs8qZq/yQi6r9sAHh3m280a9TrlJ91oc80s9nTSwHpaVZqPIKF0u/6YdCSPTBFru5pHvmVetHuadpmsuclAe10qK9P5pGm9XpGgeNCnQpMBnSoGfBBZVgLxyMk1zfczC0WQn4lgDi/vOuWnWdMI1VX1OzTSK6/WvSL5hep8rf16OD7Mf0/hAJyY32VgkYGED0sc6gYnw9y6w4FeXWFgtmbhsTxEuZlFHqiQWMArVqh/MfUgveCg1B8aAulyrCgUDcIlk5EpVKFYbVU3wA69xppsShiytiH1ILO5JK2BElFZkiUUce436pdVPAUViUgQZMg7S9T64SOtnUDNoSR1lZGCqVLtIgCF9w1xJdbOGmZ7B2MvUQFefj36jjMiT5KFeQR28EjYRMkHQOqldlflbjgbhPC/oUJJzCKO4gXrKqpILfgeu8mtYDEBxs6q5HvGcdeCit5WqxIAxYVSZNksXGLu2mfmidX6xuBmfLi6wk7mAYb6F7hokf41U4CESiw3Ueb6jzL8FWOxvbhY3SFFk+EZglicBFvPEOhDYnJBOO5AFmPtHY8ROQLVoi2svtDt4i7QDRl3QOsHK/Sh66o4V3ddewvgNwJz/ZmEaw+We4gTafQej/fzNu2IdIKNkYZBV8xkAxZcYTq+g86+EQPuN77GbMKbD5v5hMYPE5BESj1+hbtQDMyjweANIeIbLXoHFNnWznySBBs7XqPi1cj7ASvQwYERQGiv3SBn0DkSgg2IA2s1R9m+++qJ4CMjSPuEIcK1XJSUla+R5Th72r2NK962qGGlevI3q2CV56VFpEInFMDQ1Tof11DOT8Vt+AHw8A1i8hL4aEd/S1nYkFe1720U393tYIV/41a+LWVbIJwcGApAYRFB6bcMcCwh+D33fAzD0JnaNpKIVaUMAC4670f+fmJsrlh+sUA9VA1H/YVwI+vT4pYH9U6Qzz/m9Iq9zJEo3AgvoNoFHp0mAxTT9tY9k6Jy2OdBGUXkTHqwBePdQnbo53Cu6DwB/1hmVqJ6bC0CFwGIKSdAk7zPjl3TaOhxRpzPGtGNVw22jcg+x6m/8h176XhXKLE1sI7hfdrTgexTvGH2aAH4JKqD1kcbZ1EVphHY/BtFzUcFBrxRWt7OHgKd92ngAbTSUP2OOMTCl8Vh8/0JCST+6mHFNO2NeUFrB/NVOn2lbMzo+hmevOYy253ClO1X08f4IhsL+g9Kc1OL1ciJ2i5N06LsYc0E6jXezPLxzPV8zfd7i2XKuMY+4Vbsf6UheMe75wU/s63nX61Qlp4xUiLtGvW1dfjb2ZfvqedTTV/l5cL++1DYuf+xejW7pdqMO6aHy2cgzV8I71DCqXtLO+15gawKLIEPzT5MCTQp8xhT4UWDx198/KZ5gyOqnwMlQpiRt5YToVSbEWGEvj8sEMbGbKqe8/2Se+HN69/lafZ8Z4djDPMkQZ+NtJRU2wH3EsPBDXEosLthnYKE6h+WHxIIPenILmSQIATRgXAQBFbBwFTK3q6y4LBlsNUa/loeec2m8rSrUNeUmBr2xepilFO5DFUopCb8KRCi9ANi04aYyXRPY1H8l0AhQAXSjPtI6fny8Gx9Jjmmb6kaVHrc0ganU+4y9pl2JoMS+kI5BQ9IEoxLMZgME2scBKqhLgBvL5LnE+SeAF1IMn48VV3TlvQ1zafmqeLjav47k4jWuTDdgpg84F5yEdIVKZGNt+znAEHVIqiOkiQHjn7KtMkwwRu0ExutaWC46YZZu2jpRJ8HtJqDOVX/VoeZgIjXc/oHyvkcV6khpBUxZHwzhNKpM/iaVHsCMbbNivgPgWcRb06PS7uA9jPczmP9XqPwYHE2D6mU8Og2xYm6l3sBQ6wFMt7HfYpB9CV3WiXvxGGDxDsP1Q9rbx/iYw7ZiBePnR6yeT6Oy1AdjrZvUJ6R7DGP+HLe8w4y1FSQtf744U/TCDMrgPwe4fIf04xV77T+6SKNxtFIGPSNdkIeb49XeMvL1MT+9k8mc93Hf9n354AEB5jB8pp6qbR3zOyA/pTkanhvATy9mApZQxYL2a3j0kkmdB5CMo8Y0iFSHEHARe8PyWwHUr1Bte0kbnwMwDqDPKcy9NgttMOmdvJNdMNyLX3+TVG8AGar8OC4iEB3j9ZA6vAU0GTRvE2BxBCC6pt7hEQrA1o5Uahj7kFno1wco8l28oO7jMsu0WQnBOxhsvWGtwvSfk8cVwBFOHSBCoERA3AQM9RnlGKxR4Dk02Ie+VntxSBuPGZdXuJJtOcXGAteznQCMvpODYg41ukWA4ALj44Z3RmCxicRij/xP6G/VkVSH6kctbhiPW6PYcQwhuejlmu9HSCt8b8t3zLfFcZ5VFKVBSCugewYWSiCUhAgsOvE6ZX8GsKAvj6DrEaBC+mjErdc1x+MFba4H5HMxQ8cR5u1cINjs5l0YgoZjGKJPogY1hWvZiVGM/XGT63jSDsOtmjviXfNCXI4/vpPxxpfX8lycU1TPcuH+vUhT5VVlkB/lAf/nQhuX/y6Oot7QwS36gHa43S8v1z/XPZ+nxB+v2508qnzjifhjsTk/L1R55rSxJ1FJknRQK8v7Zd2bwKJB1+ZRkwJNCnyeFPhpYPHq7Z1J02ZWE2d57HwpuMiTo5Nsnmhr02c8l6+bTz7Oe6+55XP3WXXnDrCAMZJxSaBCJjqpQQku7gALGAzvyUhbPSUCYWPhR11GH+beOAKhFiVzwC88Q1XAIrXVNggsYJ1Rw1EVC4YORuWaj34y4M7qPokO+rFvg6EJiQUr7gFkLA9Gwmvh+UmJRekVSmARkbgpN54JRkpmSiPRpAZlA6R7pv2dY+pnHwhuQocbBknDUFU71PO2zZEPaVz5VbogGJM5CbpyXS80ttFyBGDWMRgnjnWHKtDyH6XQP9q5JEDnXnbFhUrdMkrrI1a/t1lNfoHb1rcw6hswo8b7uAEMWESZOPrZOodExD2/6Hv6PcqiTGN7tPb1F10Aix6AxeD8PJG0e1hFx/iZlfhQZ4KBH0UysAnz/IwV32cwyieU305dxmA2FyZGWc0n/gM2CjLjWzCja0gHjNS8hNrOCODhNWpbej/aZEW9nXrovenrhZlwD2ubXgIK9DTWS79+iV6+3o5eIH3QNeweK9y3rEQbdO+rlaXiy0VW3lGxGnScQT8lBUbVfoYk5/cYQRuYbR5pyb+3BBMN43yEBOAHJS3cewkDj3wHJpaAfvwmyMeI3wEQoKFg4QImVa9oRqGGeEUL17p5J4ZYSVevfgEGU/sGIciJ/cE943vsKKnRdoP9RB/SCeJ4tPMeraLmZEC/bxZnA1z00/YL0qdI3TCseCbaAHitwvS+hL7r7HfYHwPeilA3u8AWpKuYXnpQzM/NhWRIj1cCA0eMLm47yMN6vGNsrAEotmCcT1H10cbgmva30M99BG0bIUJ0N6DI7YQ8ummjMUeUECCbKQ4436fOqghdokp1Q70ENgMw6MM8eyMggTYn5G0cEt31dtgeRvcpY/6E96J9H09SeInqPdgBmBIZHXC5iDSrlUWDQ6Qw63gT2zb6Nn0s495FfZRQDSKJmcD71DAgagAphu+Iqo3OHzF3wLirapTfk2gEfwQajiElDwbuNJinQD+eYa7wnfC9df7ShazRtc8AEko3QsIhkOLaRcxnAENoIMgVWPiOO/dq5D0AuBpCQjGEdK5fCRAqUJPUdQQQJLDQWUGeP6Juvot2ULnleTfVn4vcy9dymtj7jM+6810l32qrHZYZxK2P5lM99OHBnTy5/VPPx33bR1r/5vR3QEHc80/jPmd3Nhdiot5lG2le2qJd6STnnW7kBGW6evJ8iSQupqStzDif5n1ZUNN4OxOkuW9SoEmBz5UCPwos/uqOxMLJPTUzM7V58neqzNdMUp9483FOmwmVr7vPv3zPvdf8qAfjy4c0ztn7gbbSSi0ijgUf5HC9CDPtB1ujbdWg9AgVAfJIKyPtxyZiTMAMdMLMZalFSCwAFKrMhMQA5j50lv1I1SQDfqBQ2oGhlaFWzYoVUD72qkP5gVf/2S2t9pegwo+5P5jRUHtQN5vz5FIWZoTjBCpKtShUKipg4b2SuZcJSWoX1AnGxTIyzaPQkoZQLegTEhzqZfvVzc6qatEHdFBINqBbSAsACUoLDGgWBt5kGDYdrm5KAwrKZcsy5b5SGhIxNOgT6+KqtuXYX4K7fZjHVxhQvwZcvNvYwoj2NFbKKS7yjYzNnDoLLOxXf+ZPRbhN2QI9dPdb8QTVOTtfdM8uFEMEQ+sGWNhHZ3jH0auTevwDMLa7MJRvAQ2vAQGqCLWR1Tz2FytID+ZgtjUG1jPTLgzjeyQWenUaB5D0wpQ+RVrxA56SVFPpJz/jKHwBINEzkUBqVfUUGGBpP4d61CbteYkUQo9JJ3gnaofJHcXT0COAxaOFuWIR9aoBwSpNVNpgkLdVytTjUwfXlLT8OR6jBG075KUB+O/erIXE4gYmdhqJxgzqXbNIVbT/MNjcHiv7x/xO/QEONMJ2vHQBEAagVw/jqwfPPyO2izZok6GnLGNtbPD8FuUfIjU4p7wpPFsZi6KNd0OJkoz5b4icbbtlQgUGuuRtp36DALIz6rDPeNpg9VybjU36d8NVfQyvz2F8QabEzoCZHSfgHGBuj7TH0FlGegwaKvkAKRcH0PKAMWmdTgBqMu+X0JwXK2xfelQPQmJxS/+Gq2j6sZW82h0WgIsr6nJBOy8Yb8ZbuSUPOq3oYNx1U+8Wnr1iDF1Bn/CShVrXBJKYK9phmesw69eqcCGx6D/cLSYw5J4PcIFxM51lcLwN3Pfq9tV5xE1goYRiAKnFKOBlEKBrNO4WVPIqyYPSR/rbd1WgEUCePsnzmMOaoR5jKSR+jvX45+uAupt1pj2+u763l7RLkHGClCX2SDGMZXFE/xkV/BK6xsIA+Wgn1kPdhlD56xoBIBHTowMJ0BR1nUc9zujxvdQpAwvfuvtbvHf3Lnrt/rxtknztY8/kLGxbmqVss2c8l2+We/P5eB6m9Im01dPdSc/zseWk5Wns4hp/8j0TcqN+mh6u/813y4y8FZf8wy8/XyazLrkK9VxolERqXLpz7nXulzTJiXLqJrDIFGnumxRoUuBzpcDPBhbOjXlzos8fl/o+T445af0jkNPlPOr3ZEbd6tc8FljEirbHpEkMsW4bcbfIBzWBiwwsXM1NoCKMt2FuQxUKZuI+sEgSBFWhYPj5hcRCYKGkAmbNvYxBksKkGruab30CWBAk70oD7lJi8SGwMB9BQimBcOUaZihUoTj2XvxY7Qygwb18LYBF3Fd6kqQGWXIQdYp6+d1Kn+1Ms7Rv0EkbgbCJYB8fOuoe3zP+yNh4TQ9PrqZKW2kpYy+4iL4ifx/x+5jADLCBC/aHvwwqzMt+F4TxnzQEf4MxMtjXKoDiDbYHAos9VvdVTxHEkDrVPzL3EfNL4CLuxwAiM5i5FvTaWyeni7aJmaIbUDGO5KEfsKF61zHjwDgNvRyrunQoAw/TtSmzDwPcBa2+gHnXHa3uT91UB9qE6VQtx+jRwzDgndD5Mav2T1FT0jh2GtWqRbwiLcFkyxDb7xsw0/4OATODML87HK8hhVijXVdEeE7AAmPg5aVicW62mKI8A+hp/OzzO7R9DQb4JUx8L3WdB1j8GuBy6iq+nqVer4Ya1h7168A17hDAREPzSaQWXtMr0jbM5hmM+JmqSNRfJlZ3tCOknWFlXi9V5zC73dBkvLsD+5P+kGrY5nXaLBjaBXgd8nwPQKLLLoV5lXGfRo3qz798GJIMu3ELW4VN2qhheR+MtYDKSOFGGjeuxA7A1bgfu0guTlR1o22DGDYP6t0Jm5cj+kKJ0jFpBnhGD14y4RcAg0vocsp7lpwvwEiT7pb2cSEARJv9Tr8ofbgCvFhH0GpINeDoi1vKuYGG4TGJMWsMjFYWFtqoVwtlaJCu16le9iMw1WMYo9/QhkP6YYt6XwAu2k8Oi97Tw2L04qSYALXMAMK6QXzGlTAui5Gu9d7kpitcAU8f5Q7TPgGGdhbGgzAuhJ6iuny/BRSkVW1LevnyxPwFDWWzM9iA6tW7FvMcr6MudH2n9Ffre6sq5wHg5pj36ATQegYNjmnnMfU64ae7WaWRSlj1QtYLiOgBVNz2DRaXPdgkAXymAMDLjOUFxvEA9OxQmmK14l80rfpjDf3/c7b6XJ7nn/pzkVe+8Ik8fzwPR+CHD9bLuvt8WVi0rTyOx8s83Jkl24e5puvpb75bJq5okq6H1KFMYl2k5c/aSBtb/QGu5dK8l7NqAotEqubfJgWaFPh8KfDTwAJ3s8FQVnNjngKZDMuJ0isyOXmrfwDyNfc5fb6W07nPx97L5+5lyhKoSKoyfnRDapGBhat8gAe9QKlKkDxDCTCINcE1VQV8poWvQgSvgxnQ5WuACRgcgYU/GXpXHFUDSpILP8K0jP9+Amx+BhahCoUBdwUsYMw0mg468Ewl/SjzVaUpgRk9RCmdyOUJLFD1Ka+FEaj1oI4NI+4ShASzIuhRbiAto2oV3aQVbIzki2vWtdL1hiFM6kZKBARxZWeakumzNQAAQABJREFUhvMbvCuFBELawqzk9nI7ClFiIbgwrZKZ8EpjOvJN+SVJTerfm/AEpT75Orrqq0R1foc3oC0Md12dvlIdKrJNthwCEuseKloyicGEkQDpzS2gomUc3XcMt1uQWnSh426k7UGYYmm0qwSBOhqMzbLPqM8xDJd66heUZXTl3yzNFV+y8q8hsmBUBvsNjNkrPDx1QdMBxkA7ZT5FDek1Add6ufYAw+uH06y896IDz7kr4Fvk+4oV/3UYaMfjMQDjkLz20fO/2Ua96RRVKJj76aXFYoogbzJyo/yMp+BK+CYrzG959jXlDtPfiwCLL0YGik3BBmpQTwnMJxPeisrQIMbEnTDCXTyrNOaAspVYbMB8n1HmGQDhAqbTgdAPgJljRf6riTEC+nUj1YDhpL6jMPBfA44EbIeAGkGRgMo81jk+5bqr3hpAj9Djy6jR/PlXj8INrs/revc5alkCNd8HpXwyzI4OGWRVrN6bJ4bjh6Qzcra0Mn5GP8zsBeNaKYcRzNuhkQH+BBa3MOY3tB1kwzBkLMBQ3/COXiL1uCa/FurfRpt0EdwJwLhhxf6Ke5fQBue7dDZqUjDNN7SPl9XBji0N0hvq2aLkQ6LEy4F6EGNUG5ZexswtYOCC8XDEfHENuNBbWBdukofOABdXZ8UoylLdZHfNgoHSLo2mlSA4Httos0E1jQyuOlQ/9e9TxYr2dHUSrwMJmvdi4QBG3/cz/lC3eEdKYAHh4j2yenGdfF2wuGQsGZvCydi5I2yVKH8HJwKCC20stGcSsJ/RRsGFaV346IZWfYCKXtSebvuHirMOVOewQWpj/ExhyK207oHqfkhyHO++L74rUUfrWW6+8/7/1OaYcItnbUBtS/ecS2oX82GZ1BLjdi1Rmi/sQp+9m3+eVyRkvldlWZbv8/fv5TSNtuRKsbcu7OJKvsylqEftPB/aO265ubksq5qulRlGqh/5c6/NOX+fyHmmOtw241j8CBmbt5oUaFLg86DAjwOL7zDeFljUtvtTaXw3mGXdVx+KWvpPHdY/CnlyNa3H+ed5AhUwnsEAp1V1gUKSWLgqb8A8AUX6JYCR4lgILrKNhR9UP8QypFkvOsWycLU7SS0CUPjxjXQwATKsftj8RwP9IFyXqlDGbriQ+eBDrzeZW+rhJtMlAxZ2FgIIQUUJJLLEIrmc9V4CFjJcWT0qSSksGxABMy/ASNIMV0E5DuaEWlGf+OWvpYX7AZN+HruZgC1JJRIgCPURmMoACdJUpscfx2HPAsOYwYXPCyjsK7OKfiGtfRJqSzBuPBxlpkIpmRVXwZx++TdhjFY3DZq3zTE6+TDR2gaQKGqd8k0r+gEgYTJjLNBOlmGLFoxPjV9xNT5Z3LAK28HKsJKAAZg8W7am+1f2qjhJJ42gj2FOr41xALDQfuHrh8vFl6xYz6Pnb71lrN/AmD/jWVV99HvVDs30ZGX8gElAywPUpxZg1vthXKXlMfV6x3NveU4gcAajLqml5RXGwNfra0U7KjRKUxZWVopx3JfaT4ICvSuNoXL0FubwKYbZj7dZKYeYqitNwaBuInnYQGKxCwM+jCrVFKo7swCbVtqUA/gpIdCz0gskA5u7hyElOBdY0F5VtRap729wbzuJ2ssB7ZaW2pN8AaDyPTkEgG0DKlZtuz+OBSvaONxQ7igM9Tyr23+2tIAkpScMrvcAHd+tbxevAYT71D2PZ22B+ugDBjUSACREPH9Eu65R0zFIXnheA1zcYDR/S5/Y3/0A/B46ysB6Z9D3Appc8TzFMlwcg9fRb1fUDTSC8XZ3MQIYGEPi0gqNtafYASwewHyfSXfeqRaeh8hpXMJwgwT4Idm4LMcjNFb6qO1BO/W5RVXuijpo98GLi10KAeh4l4euz4sRgMXwFZG/eadvyOMCkKjEwHHsgoGdHe0C1BgwrwdQpKSoi3NVonqxDUnxLagXbfZdinkL2rcwxtxixDMmkg0XLXcAOXjZ7C/nMY2zHXBOypbveNS2IlSjuKbU4gSQpSqUuWqz1Uef6065fXAYFbGe4rQVQOccR3unkFQsI63TuYEqf33OM9AFqqV3mjzifSM3q5O3PI/n8/p8fP9eThMV5yTnY7pallFelEUC/6UZABKYLj9UZuY16fexLZef9x97NpVjjVINSjJbu0aWTOi1s8Z1E5c30vNlHly/X1bjoQ/v5frV0zTqka5GzrnttFniNSUWdYr9cR0fnt4Uv31+Uvyj5Z5iqI9vXHNrUuCPlAL/n4GFdHHCdCLNk+lHJ+x7BDStE/X9yTpfy9eDiZWZ9WMt01v+VKOw8p57LLAQVGhn4V49/7Cz4Lrp4HcrYBFG2zAmCViwhxnIEgsZQn+hdlS2KT4wzvv8ZLq1sTgPYJGMtz8AFjCFYT9R5htAhvLcRzkBNBKoEFxooJzTt7BSH2VbB4FFMCMyzqUkI18D9IStRaZ7fKBoZPxXTE9l/dk73JN5C5sK6JUDdyU1jZKZjzRJHSOYIjOKx/3DFn3lzr6AIn742cevzNt8YZOCKVBatIf+/erGZniHWsMw+siVdpjcvGXmwlVbAY7lhqUGTFEr3mzaiF1RTM6yyo3kgmsCrG5o6pTsCu8WTDmsXMRmcIVW+wilFbpBZT08VFbmF+bx/jRYTMMI+pySAz0LvaQuruRfkI/A4gwmu536L6FW8gBPQVMwtbZNo+ltmPUXSCsEFdo5GB9CprgdVbIWArDdEiOhC3Cp4fTSg5VijBgClzyrytMI9TIgnpKSFwEs9uGBVWO5SfEZWOk/I98u+tNylydHiwnqa9/JBPZjP6FhvYy+kbZl9HXZuo2k4Io6a8exhErVr5cBFkgdDkhn1w0BLGQobcMpbbCtr0n/krKe81Ot6pg63VB+H/mPwXzPj40Ws3hS0huVdiHPkaSsYni/RXmOJw2oe6HPOMxqF/Q8Je/31GlXaQKr6DL3LTwnCOlCMtEFM6uaGXHTQxWqC0BxBN38nfCsrF38490+hiZXjg1+/TDM45QzhyvgIfJytV73usbd2BNcOFbI49aX2k3mX6kh/d+CRCxGrO8L75z2Ed20rQWQoUG6BvjJixsSFEofxNfb0M1lMXKDZyoAxS2/y5MjjKONT5OBRQqwmcAFqnO0oxMaOX+oAtVNfT32HW6hvlEB2uecFRKCspr2aVYr9D2SphlohBQWsJekF9paCC6wowAABYBlzClF0d5CI27L6RTQAGa1p7jtRiLTCnBjfvCXgcVSKbGYhJ6CUNsgsHBsxUY9/Of/tFnHfJz2eU6OM9tw93Z1FvmkRIkE1Z0YzuV0URX0yXzyY/kbkM9jTuMk771+P01OW79+t76Wz5WqyY365GdTB6Yz26RTirxBLrbGeZzdOU0n9TrmZ+v1qB5JGZLEu01gkWn1sf2/+t0Rixw3xT/9CvXGQf3W/WHb+eVt8T/8HzvFt4vdxT/7Fs9x/0Dbk9Xz4l/+b9vFf/kfjha/Wk6quf9ARTeLaVLgH5QCPwEsfih+CHeztck8PkSNOjqBOiXGRFoeO2nWJ/ZG6lo+n0jjc/lZ97GKDuMTzC77+8AiXM/yEQ6DZZjZcDsrwEClQdWBsLPgvl81P6jq5t8HFiFREATItPLhDUmBH+BSxaZiYGjoNcxkKguJBWoauvt0pVWJRdDB5wM4kJd5CihgOsJ2QmABw1MdCyg0+iS9THOoP7mnjkn9SPCQwAU3qVv6yYwkdal0z3ItPEe4jlP64pZfbEFTmHboF2ACcCHIiHOYn6C5NCZdABDT8y+YN/dB/+hVzsqVPtJEOe5h/GSA/JGYy/YhKjhhZ7FZvNI7FFKLRkyL9KzPe5TKTeXcQo9bPO50ov7UObtQdGJf0cGqsNIembLwcsSqtsHDjgEWel3qVb8dHXnddJ4BZgbo/2GNWWG0+zEmHkdtRcmBzLtG1G9xE7sKsNjFjuBIZpg6aAuhh6lvMJaWGRtFwmEcA2NAvCPt422kCjDPpldiJlPaTkyT9iO8SCEhGeD5xQcPiyVsLIbxHKSLVwPxCSwekKd2DqtISZ7pnhaQostb23AOEOqGGZ9FUvGPludCBUtJi9IUGow9CeOFvGWK12jvC9SOXsDov0LCcsyzOMMN4PTNwiwr/INhdG2XqIqljYgendw04lb96znlCixU7VK6cQ24aOc9gfXGLWl3MQ6g6geU+K4oSdnGhmKf8uwpjdxHASt62eoFNBxCh1cAC5l+mXC49vBQ5Sd/kLoMYvuhjUMPdOhizNoO42fsUJ4qVkohhaICdmNP3FxwBnAQkE1A/wUkLkqaOnlWw/HX1H2T5zRm1zvWpeMO2sWY49qtoIp6KzXQ1kKJwgAqQAbKa6Us3RPvueLP+HAecST3txAfBMZxBCPuNqRstwDFq+ODcL97Q/+r+tcKQdt4XvlVO2nDVS91lEahXkmpzhW+vzL8vvfZxsKxJV9avi4xTzpX+n74nqe5QfWnpEJ4LsCiLdEuyvNB0+qONqSy0OESANwK0O7AiLwd9cCiC0fHHUpkcE7AI1ijhJeucfp/AYnGCn02g3RM8Gg/NDy8UTnLqO2CflQ6z8Hp5t2/6YnyOW/lecbj+mXP3SI/D5xVahsn0sfn3f9Ymfmp+wx79UzO+F7d6+lz2lSWOaaH7tYqahT9lMrMGbtP96IdZYe6S80v79lf/KuXm/L58G+uj3fiaTJqupv9kE5e2Tq4Kv7b/2Ujbv77X/UVf/lPhz6e8Gdc3T++Lv6b/3mj+McPeor/4p8N/4wn/m6SNIHF3w0dm7n8/58CPwosfvvdDxHHIn0Iy0m4nFBtmtfzLzc1T5D5/Mf2dyZW8srnsXLtBM0vJBV8dJM6T1pRzx6hrHzypMLqHh/ecDcb4EJgAciAkYh4DqQjs5IRSF5bYrURxiNsLQQCMAOCjrrEQlUoZ/z8kRBgBLCAobxUYoHahB5tBBe3MERu/469Nw+uK8vv+w52kiBBggBIACTAFWSzl+npWXq0WDPSWLI0ki3Llix5kWXHjisul+1KUpWkYjuJEjupuBz7j1TiyC4rKcmOFkeWbVnLjGRJM6ORRhppNEv3NLu5giBA7ACJfUc+n9+55+GBze6eHlkaa4RLPtz37j33bPfcc7/f89skAkX6sGe8LZmoIxcSiZBSSDQKsbBsCYagRO8yfOgTCQZ/QjUqr3YW4AKp8Dzl+SFxvJzK/Yg6x82gUnHPuH/2qSQBMFZPKvgBsBHYSzj4Hv+y+pmeeUI1qgK6luPmaIj7ZZ6Arx2kOLv0eRiURgoWsFn1ncBu4R5xHnQ9+xBQrGRAe4rIoKpz3OeoF2UCmDTYPjx4IR0/M0DAttPhsegI98p2znFPjTkhODdCsaomSicEjtuLjwIcDkAMzuJyU1ez6+ib61JYsOrqv25Th3EtO8K1rrSvQi6s8wlUb4ym/XwP6kzopmuvYGA4A8x5zXU8Ro1Mz4XKlIHy6CiMntdSEy5LTwBOezG4vvaOd2Bf0RuRse8jnVCVqatSSVpjDIbROHndp8wHgPwpiIG2Eu303yDE4t0XB4JYtHDvJwHrxrNQzauburjXLa32IQaJM5K1/dnKuDsLgL/adwoXtUcB7sRiAKTr0nYAUN7NR0CvJOMebb4PmRinrxdY2V9GirHD8Qbuk1KbI4DTk8chAoxNvYkZOX2B++Wzpfeq44D0s9RT9RoDSz6gjkYLnyEPgTrsIOxVjmDwfhxC0QWh0yNRN6v7kjrVkIwY/oA23LfvuUbJjg+ZDhAaaUsrP3URayDAU7TrHPX3vmk4rtRHUqIB+SLP3SNW7xepH0w/E3ufQerYAAFqlgQxZnogFT3tjBCuV1rxkGsW2CvFkdBKGNp4rtslDBurYSi+recopRW2ifsseT2MncxhHjPtNkD+kCDshcgngmTy3fwlSD4fLhxI/lVnjKfSNvrhXKg20ncuJJQYN3RNpf6kdHXP3sLjXBLPtVIKF1l2PUj/NiKt0A1zasVehfu2wzyyzVyxAc+yT5VwGo9E1brzuNN1TGvgr7vk8A7lM2rX5z+5nv6s28p8XA7V5kGuqZrDKepTbRzdn89eoqr9JqwaFGXvXRnf6o5ZVinffZTCsVIH09efLznt3+8BfKtSv5VrPWYf1//2WEle+j/Prp7xHGerDEu6fCb3hYT18S2OcM2+M3WVKu06UIV6vOfy71/47GL6+EtLeMprSTOPttLf/tPZo96TU7/1Ud9Bzmm/l9sBsfi97O2Dsr6cPfBFEYtSwccnXyfD8ilp3JdJsv7YG30veZZr/P04sSikIqtDsXrHy13dbUmFDXDlMqtCEceiIhahFgVYqRELJhKnEYmDhpiqdYQLWECAcR6CBNQRiwAFTDy+WPJbLb9sgNmUKwBA1Up97Ipc7ABqlBiEakPJT7JCOVmCIZCQXGSCIbEItSgAgVKMYkcRkgvqEeQi3mqAE/uZegusJRfxPfa5/61k9eqNtCHZANiExKXuVWb9XP00SngQC895zH/0T5ZMIMkQpHPcFWX7V9DoOQmOneHLMUsZJCmSCkAP4NQAY5bhiq797H2ZxM5CYnHr/lh4htL4WA84+QVbvYKtQ5RPHXQve7wztZ+7lLoHiOIMsegEJOrBSfKnvYPgdARiMY5+v65XtwX6rDQ3YeeAUkh66vxgujw4kLrD7Sn3i8JaqPtxANkMwPKeMSsA5XqPkpwweLBP6EgXTnWl9/QTTA9AdhRJg3d8iz7QzaxRuW9NYuANuXgEMJa47hJ9vYFozj2Qh/PEtngeYtGJasoG90bJhGX2oDJ0BaKzClDVzmGSuo6uZnWsccjFKlKWFso/Tbpn8GA1iG2HHq7GtKdAQtFI/wqOXXnfof0z5DNOHsMA8wXIi3XvYQwPQGyOAcYXqPMjyIPerXoA1icA9brdXWSlfprjc/TzIuNwlf0GecdY4B7rXcs+7sULlZGujTsy8YiI1YDmXQE1Zeie9xR1OQ/g11PSGKB+lI+qYo7/FvLTWL2bejgGjCouuDf98Vby5LykYhhCdZt7N087NLbfFeTTN9ZTtaITrLLr6SrUsyjPPL0X1l8pyRrP/yPbg3RqHNfCu1yrChYPX2oASDcAul04MGq3bn37UUVzvlhljC6xCBDl0o/mo5pZA/VSDa4BKecO55XiSCyyah9EB+mPLoiVHrUR66IRQrGDutQaHqvWGXN6mfIZ0B5D8L/3nPJ8+i8eG/bOL+TTTBtznJvsRAKLgniOJBU+x5J8+yo/9zSdTWcEoVrFvWmCIEgqmpDsIapDJ6+FtjcHodAFsyR2lzmlHUJ6ErI5cPJEzTvUCcouUgsKY/OPtdy/lXm5HC3zs7+dL/K1/IjGlVR1e57p129VX3Bi/9m6/F5/EfNRmd04+Vh5uZ71uZWW5GP+ylfXp8mFRBVJkK+gVVWSsveExZXeMV0+Zw+8Pr+9lDn/+lZG6lrG1Znqd+lb0/zJb/lQufh3fR9Su6o/Sx3qC/VYeR87/68z7v10EYTx93r7Bz8xlfq7W9KVM23pX/3qw/R9HzwZqkz19dCGYWl1O8jHBupOw1MbzFPEDOK6FqOB1m2jM5tIMhuJR5NtHeqvVVXq7sR6XHuhD7Xc6tLxOeZwJCd9XdjNHXu9KtbKOvGKplXDRgJ6FM9/PSyG1RV7QCzqbsDB17ffA2uj6Td/4efTr31+GByDzeKpp9KLH3h/evGZs2lPM5DndHY4vfIbH02/+Nvjqen0s+mDf+qPp+cRzO2sE6PppQ+nH/2Zl9LDxq409M3fm/78i91vvx5fxBVfNLF4/EVj3k48ZUIq+3L8iyi7lqQ+b7+XT4BZUrn3mC9WJ8MsscikokgssgrU48Riz+UsF8cLRMCrXnTRj86G1RAAgJJgKVSMaFfWfWZWcOKtJgcn/hqxCHezAExWTjcEGgAX3zqC+ZBYAMRCCuKeT5ALys1qUB6TcGRSIbEoxtux2vkEYhFgJYiFfV5JKJz47R/+4NgpPp7TaFWPQln6krvZJsSHPrSedCpwJvd1vOY4FH0cfQ1ZII0gKYgFoKt2L6I8znEf4l4AqGLlFtC2DdkC2kXfqrrkNVNEax4en8AJwEgmFqp6AAKtc4Ao68JHYhH3HbWnphMn07Hzl1PvwLnUTxThTsCQfvjbBFX0zQxg975qRRCDaUDqMuo6W3hmOgwo7AFMvfPa1fQUhsinUYWaQu1Jl672lAbT2iqMAspvQCzuzxAQD1uFbSQIfRCLq3096b2DqBQBSJUQOKZb6EvbMcE1t/XexDUPNFimvWG8jz3Hacq8SPTmd1+9il3HkQDtt8nfeBXdEILLeOVZAagaP2OO9k9Qh3G+j0EwNIxW55+153QOUNyP+oqAcIy2TUBOjBGhbvxR6mSgt1XGgLEgVGVapg92AOeHGXsnMSrWQ9AGaTVilzgepf56vNpBMmHANZSV0hbkCkOJII5bnqOOjNxYldd4/Dz1P6GEgDoNP1wK0qARdushYiVw/BjlnwXYquY1SZvC7oF2bPLp5B71AuLPdx2POmxyn5EfpGtEApeQ+G6fhNgNI3F5BYnLA4jRQ/p1G6mHXp18Xg6xst7V05V68L50iracov+UWIAPQi1NVTjrPAugH8fj2N2RUVgTBMthLdDGTqQJNTifr8v05XnidQxAUpZ5Rlchgy48KLVQtUr7mRX6TpLoPdBOw73SE42pHY8+d9poHKP+J1zxRzrVjKQKY560/nAOcoG3M8ihhMSgdUYwdyzXNu5BqDY6dnn+GyUVtCsCYtLesHngflqWz0EDfbjLM0xzqrknP+MbELhQn2I8tnKPmlWDIiJ9A9IKgwuq/qXtjxIcVdy8Z22otik50n3yecitKlGn8bZl4EPvH1WrbaprlS3qUn5U+/r5nSfVhzdvZGI25Wd1NNpS+173JVJzQeTh8epCd3XVqbvCrPLcHQcp7/X142r7r75BVQ776/Z4KbkW9eWajdu+7GxjddwKW36pf1xbKzf3hdd7qL6e5fKoTynEhHWb6f/kh7617sjv7tffL8RiBILwj392JmwTLvW3pr/7Y5Pp2sCh9L3f0Lmvgz7y2wvpU6+tpO/5us70Ix+fB+DHU5S6jjanv/zNXZDsTCKUVvztHx5PX3OtPX37+7JKVblW1agf+dg80tV8x85AIv7yH+lK/+5Tj9JnboPm2LyHH3pPR3r/s87aefvp31xIn7zOAgP3UAnxJi/k3hMt6T/9FiPf875mOyAWua8O/n4pPYDWxM/88/SRu4fTleefB6s0pOHf+pX08uKp9K73fzB93dWTZIot5t3PpF//lV9Lt7YH03PPX03nsPfs6T1JvK+ttDR1K/3GT3wkzb33m9PzTbfTL/7scvqGv/Xd6anGpfTpH/uptPPB70zPd7eBBb6U+u2/5s2JxSvaWIzsv6Luly+b8sIp+7rTta9vdK5+4jVx+e2+fC+kwtVzV1f9nQ2oK2kF4EA7ipoqVBhW85uXfLicFSjE6iMggTIEpiGtAPhKLgTyEorsvckXveC1Wu2v9lE368c/1WO2MPbMcSwAaxILQJGqI1Q6+kOPPhqw1pOI/J3yqjJz1G3Jh9IK0ruXbFQko37VM4y4ARvRj/Y59WKJMlQvJBTq46/zxZgDvoKDWJCnuukCl2ZQWXa1S/u5PnwdUVeJmv3sZBgvSFvI9/jQzyGVEMhVfW6/+5HYZYLHPaBvDea1Bdhy1Vb7FHXb7VuvnwK4j0xALACA84BtPdpsU1dXZa171MF6W659jOvOFiQWHecvpd7Bc+nU6V5APhIm2nOYvhFkr1P+DPncQdog4J9jv8rq9SEkJL0AwPdcPJ8uA/S7VQ0inYbIvuj10qRqk0bYdyQKSCE0hnblv5t8L6My8h7sHHQbq9TA1eyjkJFD9KNqQ6OUo02BalSzjKtHtNt4GR1UvB9g/w4kDj2QA0HdHdJpgNtO/w+Qn4RTYL9un0AuxiU4rIzP81nldxNt6gUIngaYu+I/C1ieclWfsloA1vYnAyRtAf43qdsqDdrk2l2AZAMEo418lbztQkDCaBoCYeyIFSQ7qp9pf9BB3kcBmq2AS8mEdVFysUL/uP52iv4ZAogbA0OVuFuQm3H6aoN0Ed+FPtzlpp1EuuIYnOPeh50K+euFaxBScAFScRUjdL1OLXJeUqdBvBIZXfuqxjRG37+MitxdJEAzRmWHGDIQCIaIChOSl+6e7lDhOUN/RHA92q/0Z4k8V6nXBuN2HDe540jC7o+MpF3qyeW0/VgiGl5qxHC8hTZe5P5fxBj9osQCIrjCJ6QW7B+GKhVxSSAZRuletx9dIJAcMEZ2aN8u465RCQOE4DD3sJ37YlyTw8SxaUFKsYW0am1xAXewEgvuBe1tpH3u/Tgn7LpIwPPQDGFupD0N5MUDCTphjuEeGh29gXHbyFhz/HOzQwKxy77B+0kanyPJu5IVrDHCFXELUhk/Dbi71auc98Nna55xM4+ka9dnkPKMvdHJM6HXrwtILi5Atk4w1nWCkCWaFprBUt7vB+4+l3HcB6jafFI9vgejOcH5vRQ5Ybk20tZdX/J5W/uqHrk2T7py70x+3+Tf1WV1F1h3q2ttSxraUurHoXy07pL4WtqX224fuEWbo+2ZAOY+yPmV9puu5BmlPlapSBd5pAOvUHbWY9u/+XVA/a2V9N/9md54l/3wL82lGxhC/53vOY2Ecw8FSQ4+9tISc1VD+rb3dqR34H3ptdH19OO/Mp9ewJ7ie96ficgbEQuv9V357V91PD0NcfnYy0uRn8Tg2OGm9Ce/5njsLX/y4Vb6b//U6XRU/Ui2H4PIWN7Vs7pCb0gvD6+lf/HRufR1kI9vg4S4HRCL6IaDP19KD+wOp4/8Hz+cXu3+xvRNH3wxXTtFTK7P/1z6yU/OpWPPvj992x+6kI6s3Euf+cSn0mfH29ML3/w1uNk/Rmwm5vmYlLDrHPls+tkfvJGu/a0/l55bu51+5h/+m3Tsb/zX6avu/VD6gVevpe/7jvegNswid57EvpRa1q55C2LxWs14u3ZF9aVMxLW9x8vk/Fjikuaxw/GzTL5l78Hy3b0v1bALAEyEHYCgFnBbvEK5z8SisrOQWASpyNG3s2codf/zZB8SCwCDQC2IBS/YLLXIBEPioRqHwD5evFWbfDH4MgFmA6wlFthWhBqUdhau1gMiqKttNRaButRBWigrSAUgIUgF5dYkFZCaMPCOfSEVgBmu9bh5FbWnsLXwd3XMvtb/TZAKFmYkFuu0UYDlOYGYuuGSizCcrUgGu1j9jbFj3/qPvVt+NZY+d0+fA+TC9kJSYd+HGlhWP9PIVFUzV+5Vg9oFILnqXIIBqk41TbCxEYLk3YJYPAQAqgqV7TlikZnS8/12stdgVVUoicUJiEUPEosTp07T53nyNuDdGUCq8SnUuTeuhEBfz0RLgFX14M8AIt830IfqBy8BAJRRogW3lqO3IdV/HgnsuWd3Ae16WtJl6hHKPw2wfaqvOx0HQDomJDDt3EtjW7hiacToGa7V9ew0Y8zYEnpWamXfCVC8oMQBg2M9Jul5agGAp5pWF50dsVIApc2swC9yf2Zp6wPyUm1pFdLSzO/T1Fd3un0QAFfV9XplUDvHUwBH0mxyAyUWu9TP+A+qASXSNfLdVejDqHF1Uo+jR3jB0f+zkDmjbWuH0M+qtcbZxwDJq9zbZewsjFa+RPn23SDHJRbahYi+DOYnsZAMOJ426cc5fguIvVdG516hfPX/uanp8qmudPV0V3qG/RzHtPfQaL2PdqkupnG8rnPHMeq/DqFQPW5majptQz4F4A1IHBo6ufc9GNwDhCWJJ2irge7aKNOI4M5665Q1TIyU++PjaRxJ2K5SJ56BHVbxd7uNPH0itUIuzjJWLjAeLtKfGj8rrVhmrNq3c0g8JG+6J16nDZLjiIdBW2PMI6XYBZjjViruveSiDZDeTjvaMfQ+rGtapFXrEAxdK0skKCRU18wHkQ/HmBOof4PqfdpEaD8kWeA+OR5VG2ziuWpEvUrVtdgkHnz0gsYk4G2gbTwXknbStjAQJb26XpasNFE/vco5Lxjj4hEkbY77tk1ftVLfI0hvDHB4mn4YpE9Vy1O17qjzEOMy5hPrkktnv0csyrzgKdOVbf+csXd8XxorzlbmxHLtl7wv+b1pBrnMXA/ntTdNXJ3cS1da8kaXlfPxJohEJSVn9k7Wf31iBfKl+dpaH1f9e6AKtb/LfC/8z0gongLof/fXZUPrzw+vph/56Hz6zq89kd47hI1RtUksfvnzS+kvoCZ1DY9PZfvHPzPDAsJ2+m++63QceiNi4bV/8Q+fjLJMuMU79ft/ZAI11Mb0N769B+cV+T30WSQXPwZZqU9byqrfq751sqMpJB4ePyAW9b1z8P3t9cBi+u0f/cfpF5eeTu//wx9I7x5M6e7HfjL97PWmdPlrvil9y7t608a9T6aP/SrEYvoI79xFbEOb0vH+q+ndX//B9O6+3fRw7Avpl37ok6nzr/yl9FU7L6Uf+0efSJf/y29MD/7ZJ1P39/2Z9P4zx+I9//bq9eTUb0osfvsViMXdkX0vlfqXR/13sy+/3ZcJ0/m2HC/TcElrmpKu7D1XvruPVXNerPGdvS/ZDHAriQUv9HDNyEs9Sy30DKXhtoAnG3FLPIp6gsRCGwuNT5VcZPsKVZOUHlQSC8BbllpkclHq7Qs1PNlovB02FgSqAqQotdAlpoDZt0p4bSIv1Z8sq0YsXLnkd7avqKQUpNNLVUgr0JNWFSK7ls1gMghOvbTClU6BCX3siv+aNrMSC75vSC7oM1fa6bAgFu3kfwT9dl3q5jaxIEodQ3JRexlCiOI+Wf18T0q/hw0FUhr1zf0ugdL7U5A5+jm8b9F+QW6DwAdQFISNvvW+TFXE4s59bCwAuYI47yX/Q+VDFQ7L8r6qHy6xaEYV6vi5i6mz/2xEE16iOWsAUlfVLwOQekNVCeNhDZnJc4TPPABRADWI2tH7eruIx9COwXAjRt7LQSRUy1FXnqwCmD4U4AJ6Bc7TALEdrpdcnNK+gjIkAOu0s4V6+Yo6Sj0F8+sA23n6QVUkA/StompFgI50CBB5Cs9UZ/CY1MGqsNGyJ2aJAQHQO0T/KHFRVemobmwBrSsAzAmuMcaErnOBsamf4356IQXLjN05xrBxN7hx9A+2E5S1RNkb3PsGAK5G8+EJiT5tpjwJkDr1p1mpOEr9VROcJO7FQ4iF+v9nAZWqfHW2H8JWBZJgHZf1nrZFBPLmUFk6zwr/SaQ0bvbvFP0jOTAI4TJE4g5ugx9xTEnNuvedfB09quNc7T+VrmFE/iz9PwVh0oOT6l992mZQ3y7av0ybxyGBNyAWd4bRFUXqsCOxQKVnV2NkiMUhdLhPAIL15nWIMaU05jAfjfKPQdos8VWuuzs2libu3SM4H+5hubE7AO1d3RTjlasFcnGKvtTO4oJqZFyjm+gl6jyDtEP7jBnGpu6PlboppWhgz4Mc43OboHO7EBPEA0gjsFnB45ISlSOUf4SB2I7UchdJxQakQqNw74XkoJFx1sK9a+DZUMKwwzxjDI0G6wZ5UlLhUyZpl4w1s1cKoecpn0GlJEo1lFhwiCSSex5yfrRKKsgi3GOTRlLRxJziPOMcq1rmEiT5ER9GDapSEAukSId4Lk7S9waXfBqJkHYnuiPWriaee2vEd7dCLCg6ynTO81xO6UEO+8/K5R9715qOT36+c5qSb07M3+qyKLh2sHwpJ60Pc1xVRtmbqtTS77Xj+aBHPFyrQ/ldroqTtT9elNPX8olze3XISWuZx8/8q7qupH/8knxh7e/jfVCSR7mlH6v+PyAWtW6LL9dH1tIPISH4S9/Yla4gDXBTzejv/ugE7qhb0n/2oT0d8UIs/nskG4UEmP5HUW26fn8t/U/f2+fPmEufpAolsXj82v/1/5tECo0qFepQZRue3Eg/8HMz4VHq3Zf3iA2PanqAHcbiyjYYZDdZH204/tq35ToeEIvSgwf7t98Du2npwafSz//0J9PwPFiL9aeVdRxzvPBV6QNf/XQ628Ei6ks/nz7y4V9MN5qeTS++52rqTnPp/ms30njLM+lD3/MNaXAbW9eP/1T6hessNLKq1ND9fHpX22+l60e/J/2pbxrAtXv9LP/2a1h/xZdMLB6fLM3UY48fr5+WnVDrz5cJ3X35XipXjsWLtTofAJSXbCYWglxAQRCKTCoEsmvYPqzxYi/EQgBsGvPxRQJcz8SCF3j4o+elHMDfPcAsVKFqxML2cAWNKK8SvUJtscK4GTYWkooqlgVlKFnxzefqspIPX/iufEscwu1sjVio9lRJKzTe9jy/G41h4R7QGTYeFYkwP2959F30MTrYVAgbMVadibVAscxl4d9/CaC3CEhaAVTqiacLgNINiFCdp/S9q8/+bmMlXLGt//gaIjBLsrWZYCitkNRVH8kF7ZRcbEneUH9ao/2r6O/722BnhwCIbQAe27TKfXiA7cPwg/E08mASqQJ9JXmIPIUn2rIAq2gT1U/4WEobR46l7c7u1NZ3JqIJt6JHvsaLxFV9DXTPALr7AP4dgEb15MeQWowtrLAKDTEAxGmo+zyqOOrVHybvu9gJjAG0HgKG9Q7VAshtAkTv0LcP6UTdn85Sr01ANA0KW47DAK9mQK112gFANwEcfaUZw6GF/AX2j2iHakzrSBTU8W9hLOpadZCI3ScA7wLoB9gRqJrSxneJXAvlaj/RDsBTVWWc+i8KzLmHHdxjV9bPH8XlK3VUEqUqkSpfK/TXEnWbggBpAL7MfVflKUYliFppBdr2XNcWHpsE8Ir05wGauqidp+3GR+ik7E7adZL2LXPNHHV/QP+0MVbPQCheCEKGjYXEgmGgZ6pZ+nV5fQuy2BRemF4am0rjePh6hFH+riv1zayua9sAEL8CqVBicQ0D9FHK9b7M0P4h8j4HoerHm5T5jerVCne5I8P30jxqchKL7SAWqDJBLFqwkzjE/TsCIG+EhDTRjlb66dpgfwQulBh9YexBujFyPw3fHQ7Db+0StgHSiWBxjdhnNLNSfxTw3UN7+ztQB1Lqw/hdUhI0P5emUcWawfDbuSTmFfpUQ3kGe5B264EeEQC/OduA2CGQ3laeJWNbHSW4HkYuEUzPQIOqaYbzBka1RIgkXIGUh/u6Rf+qWuVDG04TuJ8+Azx6lYTPOYL6mYa0YWNBWucu6+d1Wb0Rj3bcB+cK/oSBdswrjGvnLRhIWsPeQ3KhUXwT5PIQkhujhB/n3utd7CrkbJC+tV+cB5Q+ueW/fKEfyuYcXJ2OQw3h9sqvPrvVX76UeSXyqC6on889RFZ7m9895hEvii/8ifIiF/LcIxYmqxJVyfPFpYyYSfJlOampKqOIOFwVFLsqRbm2+lnLv5QTdao1vmSec8i/6nPLubw+zyr3Wj65uR716ielPyAWVZ9Vu/8XycRLSCjedQmJn5y82lRx0lBbKcQJDKXd3ohY/MtPPEyfv7ua/t6ff/vE4u//xGTqxjK2nljcw+bj/8Lm47uwx3hPRSys40/9BhJMAnRquK3R+CTeq/o6Ww6IRXXPDna/kx4gBtUn/nX68OdW0snBizh5SWn0+qtpuvVCet/XfW1617nWdO/XP5x+/hOvpt1n/lj6zg9cSccb59PdT380/cLHxlLfd/7V9B1DuCN/OJruDs+CKVCPX7mTPn69Lb3/Xa3p5c8Np6Ud7Ew/8K3pqy/ilVMD0d/B9raIRXmB+LZ5UrGer6WpKlWfzsm0/nyZWB/fe6nH/BQQmlfQXdnOxMJVPCsf/t3Z68rS+BKqO+jmdA0w6Iu5fGKFkAoInA3a5aq6Absy8K/IBS/n7NEo6x9nOwfbtNeY8JTEaqUSi6IKFdG3AY9BLGwjM2DYTfDy308sirSiEAv3Sk04Hp8cYEuwHepQvFzNK+oRL8vSv6pAufLOKjagdIkPcYOCWBhATONm3XIe4dpeQOJpAIauc1dIrxqJxmVKMo4BUiQYrTTQY5IM1SOcpt2HW0pf0KICSIPEYRdSpRef6Hf6eZVYHsuog4QqCfemHXDb5gotffkIA9fRKQLkPZhIk4C4VeJP7EASVDURMHl/bZ+RlLfohzWCfK13dKZNiEUDUosWAGsLwFWQvQmxcDVYNZ0ugLErx9pLGGdillV3g+NB4vFC1JzOo44k+VAipV3EBBKNR4DodgDYccC7kaN3afcC+c7TX3NIAtYB3xrvujLfQn9pYEu3pm3AtaotwvgTSCKOAlJVZZFYzGGwvALw3QbISSy6kBacxwC8G/C2DrGbwPuUoJ7155AS+dRs83I8BgBuJf8F7oVqWZvczA7qM4RHpnPUz0jd2sbQ00Euop0QivsQFd3YzgIad0jjbdHdrq5iXZ0/S/lXMJQ26rbE0PgSeqHSq5VxMPSMFO55yds4ENqNPIKwnCJK9iVsI94JKTgpCQXkbpGn0hQBqml1BjAPKP/86ES6c1tJwwS6YQQHBMC3ICU42n8mPYXE4iJRzg0IeGvmURo2wB7xS57uxu7idHcaOnMa+xaMtyESX8C+Yuz+/fRoZpbYEXjmOgQpQGLRCEE5hBpbK+2RjK1xfls7E+px7RK2M2d703nye5Ux9SpqUDfuDGMwzcIB9xpREa6Kj2HLkFWOdNsbsTkYC12MC+KBh8re9PR0mpuaSvNTM9FOVQolt7HRN7sQEgJ2UCdUmDi4xT0W8DfRT230jxoRbUg/EqpQmwB5I2KLFPNzz/NEHs3Vs6t9iioVRkDfoQ2Cf34yhlSZZH5gDjIieQv9K3kwn/Auxdy1zr1z3jNfJayq00mOnZC8P34aPc712jX5LtDI2/gmSxJljh1iTByj3noVc1xcQk3uHMd6eYZUKbSeVMXZJW+UX9vIv3aiOl7SlWTxDFd9F+f442Vlezx9HOd8JCkn86+4MOfh33JyL7PHjzAdcWXd+ZKgXMvvOFTKK5Vib71fv3ms/njJsOz3zvl81W/x60l5lnFVl9jcLH9/DjnBVyKxuDO/y3yMu2ZFzm9j0/j67/34JKQYl9vHM3kol68hERDgf8u7O9LXP5eNqL9cxMK4GH//X01il3E41LVaVQlg+99/ajoWeA4kFuWuHey/5B5Yvp5++v/5cJoa/Or0/j/0ArGxUnr46kfTT390PLVe++r0R77+Ylr61M+lX/rseDr64nen73pBeyKC+975dPr4z/37NPOuv5b+yldXzg5YFNtYvJ1+6cd/Oa28+GJq+civpsV3vZDObNxMn31wMf2x73oxnTuBmu2XXFm1gok1BW5qBx/Ubyy+7e7Wq0IVQuC+TMo+PuW4L7va97qcypQch+om2ZJHSVp+1+/97id08mOfDbdjlZEXtSvn4WoWUBe2FhILwKHEQvUESYXqURFwChBHZvuJBWAkVrErAqDEIiQNvMiVGNgeV/RKu6w+8ACJheWhBgSo3ioSi4pYUASAgXzMi4/EIq8qsudFn9WgMpEpBtsRy6KeWET5XO9Ln+8Ciyw5yXUBSwbYXgX5zrFivcTi6SoMVAnGLG0eA5CohnKUNpwFUGg4u0bFJgE2cwBMcotzgq120ijZEMQ5H0ow/O1q/2HOFbIR+uB4xHHV10jAfrSv0EB+yZVbXK9yQ7BjgFjQZl+aU3NzaRSVldHxqfQIkK2OftzPilzEvaYcV6s3246klY4Tae14V9pkr4cfpR7aDajGpmcj2EjYSRicrRlgvUx+y4DyVVbUtwF3rtxrK9CNOlQn4EnQNgNp0LhbkHaS9vSycn4CAL9Nex9BVGYlCIJ7CEoD+R0BVLsqrNqKXnY05m2kXe3ciy5Ihd6ZBKFBSCAWi4DOLYhFM2OxB+JxsbcHd4fH6WPiLlB2ViMiT/rUcTvDuDzEuDtE3YQ9cxCGVeqgLcElVvYHWCU/Dlg8hpG2Y1M7FT0XaSsyjI3KTdzdjtKXxoCQoPnwq+oyCJm5CBgeQiqgxEHgMwfZmqVti+S/wfifwEPXLJ6mVpEmaCOx7bjifl2AUDx1qis9130i7HEk4dpdBAD2meHTxljRNuEV7uUXXkW8OnI/7XB/EzFHjvb2pb5Ll9NzZ09HLJAj9PNrE7PpJlKqu6Pj6QrqYU8P9qUXLp8LsnMbNajfHhtPE6MP0iKqYmGUDCFo43MYEHwU8OuqvCpKM6RZo96NtOPipQu4Ej6brlHOHex2XqMOL9+6G21pwFi7CUlHC9czaOOZ95Fvon1HkBD1cE87uAeHeIIfTo6nufGJ9AjJi1Ip+0FXvn5UWyKQBm5r8RbDb9WjjE6uytER+vcQK/3eW10N76JOtQl53kRiZ9rw9MS9CwmoUkfGjIRiE/Km8fU291DboFJOI/fNCN6HeEZ9ZvRA1sA42ib9Jml1CqGr30bHC4Rd8iHhlHiEVJbneVfCRJ3aGTeq2+nNaw1SsQC5cAzrQeoY/WrQww7KU5I3yLPRz/44bQr7K5575zjnaj8F8Qb09UEuG9/jPL+L8MJnuDY/mo4E8Vznrx6JrWQT+/LDMyVDv5uX+9j4Vvvh8eoH9Sz5h+ppSc6eU9VW+xJ1I9tqq32p5VHO5L3n99LUVWB/svhVn646vVdQLvcJV+VLM6l4vO/M8SuRWNx4iBrj0bdPLH7z5kq4ln2SLYNkPUgHLmP/i+84Fd365SIWRV3rT+ON6p2XXIJiTuGd9Y9+choJdeOBxCJ65ODP76gH5n4t/dA/+Vxq/4ZvSh94z2VczJPb9KfTv/3Jz6SZ/nemb/rWd6e2l38+/ftP3Uy7174j/dk/dJb3FO/PW59OH/3ZX0yP3vfX01+uPKBtrxFg99d+Iv37hy+k7/ja+fSD/2Ayffv3f3e6kj6VfvB/uZle+M+/Iz1/igW+30GFvyRiUcor07f7WpRnvseLpkyyvrD2ZvzahF5eDubl+fLbffm+J63IL6swYgyJBS9fXtjx4SUbEWkB+kosJBK+dFXDyV6hOCY4CGLBS5rKtlUSC+0fskShIgC83PdsK/YkBV5jE/xss4JfVKEE06EKBeDKAbVAMrwAa8TCFUEBAx/3em/JJCPrRYdHqDiHxKJRo20kFgIUAEHYadQRixzXQoKRvcToAWoFYjEPsF4OUgGY4py6//dZJZ8FVJ4AiFxEbaQDAKFNwT2Ayjgg3NVrdc57ALftAHHbp7qNOtdHAdZdpNclpdKMUOHhvDCYU7HiqjH3Liph7iV3axjErtEXO6iHqed/iDw15J6cnkmjgMvxiWkMXAFXlOuwkLy1AHLCUw4EYh3PNksAukcE+1ohivA2q9dtgCXjJmjb4H1eRCqhsbDgyVVj4wFgbothNn6bkYTsshLeBNHR0LcbgNzBiqyruVOc0yZAsHaCc70AqpMQC8erfTINgJ6jT5TG6Eq1F49IAjwB4YyG0/Sz4Ksd5XYJgfFO0HBMs1yr5EAbiw3Lph+OA/4uo4Z1AalBHyvuSgP8uOrczn02JsZd1JlUf3Ilu5nhsqikBJBvZOTzEKJTEBcNlY/RxghmRp2BsaFaNUnaV1jpN56GEiDJkCowx1G9uoAa0gVUkAZpm15/rLPSKW0cjAJueWOSEsD02P3RsC2QVBzGnuHS2X6kDT3pGbxicasxKCdOBlIegbHxG3QV2w7RUX3qFqTgc6+8mu4jKVifnACAn0ydZ86kwcuX0/PU4QzEzbKHITC3qef1+w/CVuIKhOu9Vy9E3neRQvwmtlsTkIsl+5jxFgbbSDt0jXoCMOz2UOkGXulmkXps8v0ssUkuDZ7B1WQfKnZz6eboWHr59t0gnq1Hj6UOYokY8VvQqzH/Q+79Jv2jrcEJBu9x6tXRSF9OQmjGcCYAaVElbod5YJt7t4OB9Q4rLLuQE4PsSXi0n8HCO+xjOk51p3ZIj1JLXcxuQiy2IBa7PP/amPCgYx9RVJRQteSeh50F/Snx3SAvbaA2lYjQRvu/GcLj/KArZQl9K/OX9j6OKaORKx2TgBxmcSBswkjjfVf9MLx9cX8OQZzaIbXmoa2Ix43PopNln7NDtP9wEIs21MLaURf0QwBC7usxSQv94qPt3BIAnjq6lbmYw7HlXf5bYHXM1XFdnh/L3C1AL2SgpC25vI6w5BP8LcSCfRRaFVw7b4q8VVWMH7VU8aWqX5Ugv1tMxpUeqyXO+ZS/Obm5lxLKmXyZx/faU+pJmtI5ft27JPrOn4/nVtKUfqtdUtX3K9HdLGYH4We/WsivNfmtvvzTD88mY0f8nTcIhvcTxLP4LcjH3/xjPakfe4svF7EwYN8//NdTafBUa/r2F4+DD3bTz39mMd0ZX08DxLI4kFi81Z0+OP+WPbB6I/27f/bT6UH/+9IH3v9uFhDBfi//Qvq3H5tIh577uvShr7+Wjkx8Jn38lz+Rrm8MpW/80Nenofa59Nonfzn98mdW07N//i+kD/bzjtpBlXr85fTvPnwrXf02CMThX0//6Pvvpm/8/j+Xnk2/ln7gH4ykr/qbfzw9d6oj1L/fsl5vkOBNicVnXtnzCpVfOlUu1WRaJsn6ydUU8VKom9gfP19eWLW0fCkvpHKufi8wM7s9YlGRCsCr/uYlE0EuAJaSC423JRZKLvRYo+SiqEIJlFWF8mWeiUV+sQf4B7QqZVBKUPv4OqGh0WT2O4VYKLEIYoHLWV7k4b8egBhpJQYAGe0o6olFrGQGOPX4nlqU3xuRWDRALhp4yQu8a+VbFwkG9SovW19WeoEqxGIFJIVGdazCC3YfsAr+iLZ3ATrOA1QPUQ+9/9zjuK5O9ZKke8segLuvPmMDaGsgsdBwV+9AJwQslJlfqBCyeLGyB1S5Gu5LQnUp75P9vYa9xQZlGsW4jeONGLjP4rlnagqXoqjErKtXRDtsq3YGrYC4Jgxad3Avuw7Q0ph5jjYsybOxNVE9pJNPBGujftOs0s9gC+HLXT13jas3KGcVYqVkoYH6a+jbAbjqA9gfBzSpWvIAQqMtwSIEoI16Crq1z5DALdKHC9yzVap2HELRS77nUeNRHUr1nzHBGS8J1eNURVH1ZJ30umf1OoP1uXrcRpoj1EUvVN2UK0k4x0cyux7jExUxASdpjBpu1OkppAZLfNapn9KhS0gNJCUSC8e0nn8CbMZ9R7RIXtrPvDY1m15DDejOyAPIGv3BWGnHHuACoPo8koFBgONJ+sby9GpmFG4/3uMxwPgIYP7urTtcC6FpIaJ3X1+6PNCPX+xT6RkImfdzCruVL0zOck8JIsU40A3vabxFqZpl4L+XXn013bt9J62MjqZG1NY6zwyk80OX0zuxLxkAeBt5e5r87+HK9+UxyAf/ejn+3GA/Y49+pR6fwzZicmQMgIyXKYD8OeKOXDzTSx+cDPKolGacVfeb2OeMQ0QWIWQGPDyDX25J1CzSkrEHD9Kdu/d4JiHBqEGd6OtNfZArJT3LAO8HjBnv8TbE5QgA/hgDtgOatonEYvXBWHqE8fcWYy8TiqNph5V9DbZ3lX4wHhshhQ2oe+0EsTicjuHCuAOJkMRiEUKhJGUbaV0DJB5xAszbD+PEa/mtqqWB95Sk7XCvtONZgwhvcM92KIMEQX59tr3XRoc/IpnR2xxjZJXytxgzqmYdYVz4TEist5F+rFInY5PsODYlDhILzitZVEKiJ7CIF+MzRz1Ul2qHnJzBmFuvYzpAOM2nC3UvyYXPs5LMmOecYHzyyStvZZ/Bcx2sZkTkjcvj2nyNRz3y+PbYMfIv15eUzi1ukTIqU844E+1tpWr7cqxLv1f3cg1X1y7KVz0xTUle7fMldXXieKlHfR3r6xHnq7JKWrN7XZqqjNhV6b8SiYXhJFpgrvXtr2/6k74bbO7v/uhEeu+VI7h5zd6gHk93d2Ij/ZMPz6QPPn8s/ZEXjqVfAMz/0ucW0//wZ3FRXueGVgLyEjYW/+P3ZhsLpksG9J8AAEAASURBVNL0t//5g/S1146mP/pidgX7Rtf+b/9qCjWs5vQXv9E4AXkzCN7/iaepegnFrxLD4iOfXgCL+L5oSC9S702+T0M6/uq3ZuNticY//cjs67xWlXwP9gc98MY9sJzu/+ZH0ydemkzrzcRTOrSb1tCU2OkcSu988d3p2cHjqXnzYRp99dPpN37rdprZPca7G1VtDDIPX3hv+qY//EzqFs7tsOD+6H767L3D6d3v7MPT4Hj69R/8yfTSUWyVtqbSZPOL6U/80Xeguog6+BtX5i3PvAWxyHEsnBAyqK3Lj0k8Joq6ybycLcSidl2VpjbJMomG7nDdtWWSd18+RWLhdR7LxKJ4hZJcZGIRkgpe3hFtm70Br9YA/qpDSTgijoUTN/99eRq3IusruzotsciShQD+/N4P6nM7S1V3WPGMyN96hYJYGHlbdahtALs2FtFmyYD5VPlqvB1qDACOKCMIB8TCFzo2BEEsSNMAoN5HLCQUdSQn+pWK7IBQJRYabmtjsRrEQsFXCgIxyYqmwb+6AXiDgAeB8UOBGiBWguFqvOotnQAOXYnqenWCFdIWyusFdA8BUowf4EqmKjRh4+BszDW+IFxZdQVeXXJVxTa4DzniOWv5SDL0vdSMutjSwqO0iD79IkB1F686zazQHoJQdLBi2oghrHYV6xwPGwvKfki9VNvRdkSgZdwD40AIiscxMtatrOpUrQBng8WZzlV5V4hU/lHicJKV9QHAtXYKqnGNAlzHAJjTRJFe5UFs5QapdmJ8AmAjhrWoKbE67TUXTmBrgESjjRfDKgT1PsRCiYjqMt4X3cvO0lcGIAvXvrTd6NLasNhfAjSJq0Sh31V+ySVlrNHnoc9OWXpj0nD5tam5dG98OlRkOumLpy+cSU9jn3EKicMK5ajmoSqcUgvb4SaxuY3R9GtIgV65PUK/Loa6UhsekM6wmn4OUH4BNSyNdJX2WDeJjYbkGoKPGgUdYnHn5i1sQ7gnkLojZwfShf7edAUbiKcgED5nk7T7cxhpz9Bfh6jzM5CeSxh2KwEytsUrr72W7t++nVaIIZE6e1Ln2cF0cehyehf1Pw/wPkbZeq96QBnXkTbMMu4k7gOck+DPAshvYR8xo9SD+9OMwfVTTw2laxcG07PkYdsf0s8jGnlDQqz3DCpT7dS3EwLRg2ej5dnpND85maaROuxCkNpQXTp+GnKFZKQD71w+83fxYKV3qlX6UaN9tCYA7pupeWoibUyMpwXsRDYZj9vHkHJAznYlFvyGRcbYbqEOEot12tEMAD/a0xXSIWavUP9bgdxsIbloIp3G+JILx7JSkgbyUFrQSn828XwoTduCDKxzb/Vwts2zoA1EJiS6aIZwc81xiHkjREXbjRWlHJDXLcZTK89xO2SknWt0PbvO86rKk/OoRtqtPFPG2lBVCtFgSIKUeOgNzZglTbT/KOfPIBGSvBoRvpffpyFSjhcXFRy7MdScKsl3/1Z+57G4/xy/OOwZHT3kb6Z4ctryfjCF9S9bSS25qCcv5XzsqeO+rVxed/iN6r6/bvvbWLKty6ZGIMzvScetR3kvlvOlOs6Vb2crdf5KJBY8znlcvZ0OIa3AvBnGW/r2SZebxrgVjlu7PN4F/K7fLN85RcBfNlWpnFfLfX+ja11YsgY+VvWb5T4ezds8H+FF5QTeHczbcr2vZf72+iddV5/vwfeDHnjDHticT8PXr6c7o7Oh/t4CGbhwZSid78eTYjU+d9Yepsl7N9Nrtyew4WxJx7rPpaGnh/Aa9QYGTozP9dHfSh9/icW2bVSx3/m+9HR/xkBvWI8v4sSbE4vrN9LN4fv5ia0y41mpTaYecmItk2uVpPbbxzjOVye8tmzx3ae52nwAyy+/10iF33lZeawQi3B7ygtTFZmIvA2IKJG3dX8qsfC3Egv3TyIWupkNqQVgyX3+nSUWWWJAu2LiyZOPk4u/VUvZ0viFctY3sirUJgBAiQUVjNYEGQAMKP0oRMJI1OW7+2xrwTHBKABbicWeKlRFKABE9dKLYmeh3r2ekpRYLLrACbHYcLWf/lAVSmLhCrc2FGcBDRKLReo2DflRx9v59RD3zdXPGYDObYDkfVRfZKg9gJQrgI8ugIer5hIRSYk6/RpzH+eYUo2TABzduToxW5YTsCu53jd7CV2oIFzGt2gAnEdsC8CUuuqCrwWOLUAinLY1FhZ4a0uySdt8OQguHD9+n6SOYxgtj0MQdFebSRsAzrK4xpXabsCSEo5O2qykRiNuY3hMAPpcZb83jX0BIDdch1JGuPWFUBwWWGGfcBlAfhFi0aekg4LXAGcTgF4J2hp1EJyN00/jGk9DcGzlESQcp1mJv4j6zwASimO0TWmQYF7ydZw6KPXZoJ2qV7XSTu147gDYX0EicH0EVSCAq0blVwb70zMYJveRnxIYAaEvJNWrjHgtgZOg3NHOYnI6vYqK0DztMS5II0bPJ7u7Uz/k4sKpk8TyOBKxCjroB58pY3dIIG+R/jZAfPjWbdRpiL/BdV2XL0cwQSMzGyPEtqsKdRdXubchPxLzftpnVHK9ad3EgPzGzZtp4g55jNzDor07dRFv5Oq1a+m954gfQj6CW2OASMLuIXW4BalzHGXjZIgwhHMWacHynTtpi1X3Zuw0zmKjcQFVpyGCDBrUR8I4A8C+AaG4Dzmd5B5K0lRHO8KY3pieTusA+1VcxqZ2bDLI4wjk4izt1+uVtjYjqH5NA77nuR+NtK2NRYE27KJap3FxOzWVVvBYtoEq3jZxU3a6utMupKUR8K5qluSwGQDfAIBf5b7vQBZaWe0/BgG1j7bIZwVJiupQ+N7lgUICwXXGj5AI+F0jcgm8d6HZsVpJLZTSmN8OaYJcmDfflSgchxg3M2/pQncZcrHGM6v6VCNtdqweoX7NSk15JlSXUgVTcqK6VxvtlpArh9Q+Q6nUKnuJjOpU7TwjuhvuIi+N9I1noVvmPvvG9jrfca2DJvZ89cjezOzJ/VuZ+0sa5+l8bckhgyuvKmnrczD9vnPV75JfDGBrUWVnHvkKUuQv9dntz4tfOX8S+t8LatUin8iA+pa8H8upPnuTlLqW47WsHrvu7f4s+XrdVyKxsL/+Q/XV2+3bg/QHPXDQA7/3PfCmxOKzEIsbEgs3JuHahFpmYg4HcXjsd6Qv56ofMTF7rPodeT32ZvCYk2xWfdojF4VYCFrLJ4xPWQkMo21ergKgNT5ZaiGxUD3HOAuoR3G8eGxSYqFeepFYuM/fJRgVseAlqxpJuGKsXrbxUoyXWkUskIisK7Hgo1codfjDlz2NKFKGMOCWtPiBwGT1p0pqgXpD/i2xgGgAtjOxYA8wCzuLal+z+4BouMEpsioUX3Q1u46FJ/AyDLMfQCrGABS29zTA8hxgQqnMIkDP4G72v8a1rqarYmUch1uA5XsAN/3wdwOGLrvqDPDQReYD2jWPHYJg+Rj94+r8Gew2tMMQiEgoNiEUOVCfRr+uksJ2uI+y6A7K0Rj5CO2lkUGCJAoT1PMh+R4iD4mAalmq3QS4cTw53visUqcxwPMd4jEMo1qzCAGSWIZhO3U02Fs3UoZ+pSz8tiwN0pV26PJvjjJusHKtMfEIth4bqJgwIMJVZxtGrJ0Y5J4/RWA3iMV5jLNVlRK6qLJjhG4lKEZ7lgLptvY+eU1g32Bcg+OUOxgr/R142snuTR/SNu0q7INQ5QL0ZbKltKcRI9wWIk+vppvYKrx0fyJNk1cjdTyLGtFViQXA3PG/ChCnWwGardiFoIZD30gQtNG4iTrU9ZHRNINK1CrSoCZAZU/v6dSPOpTEogfgaF9K2MpzJ+W9N0/sCIjFrRs30zrAVRWaU5eHKLcfT1qdGPnjBYkLtKWYpI536G/bo6vZswBSicEIfTB862aaHb6bdsfuE9Cuklg89VR6B/U/CzHSxkI1nCWA+TT3+RZ1ts1KIVxt3yRa9TpSg637I2FP0HAUaQNSj27UsvpOkR/tPUz9edrTbYjXCGRqDCkPegYxRlXb2cVlrOpFYcDbgaEZUocOJA66tO0AoDfiaGCMfpqGFASpgWyZY9syHrymp9IupGIdUrCBFGQLYrHb3RNSi0a9XDG2dVxgjAmNt/WyhMZdqC7pctjyG3z2lVhILCAvBtMz9sQ2z4jEYhcpVBiUc5nPr+p1zbjn3aQ9irC3efZU0dKFLQ9+qCsdJo/jh5GW0Ee7SiwoV6cDjiVJiC6Q2yhfSZRSiS2ec9Uwfa6aIBwtnG+F/Nn/DYyfNYjHsupXkjqeW90cd0OMNOL2+ThF+gGemwGegyPUPdzPUt+yOXYqqsHojxm7nKrtC1nwfD1ALtdxp3yUYytpaxfzxVOWU9t87uNodSaufZxY5AxLvrVrqy+lnFp9qoQ53/rSzCfnVX+0Pr98Ntexll911RtdU3/9W32vz9O0X4nE4q364OD8QQ8c9MBXVg988cSCdpdJsEzcZW+X+P3x33HcP9XEXn8+H64m9YqY+KueVAjILFNiUX9c0FwMt/UKJXHIEopMKPwuoVhl5W8fsSAvX3ih/gSQL4HyMrngxfo4sQAMRJ2r+vldiYUubze2lVjwYgdgRIA8ANgu4Nu2hk0EYEIg5gp1SCt4kYeUgr2Ewki+e5KLTCoijoUEA6DhSrz7ICd+B4gE0bEOlKEq1KofSYWqURCLKUDwCCB8GECh96YzAIiLABGJzQJ10/5CUCRB0BZB485xgMkt1IwE7U30qxGSBzAiVi97iRtyn/MawW4jYRD8nweEGETN1XjvjWBXKYPgewEA4yr/CuX4vjaewgDEph8iojG46kMzgMNXAZragRhvw3r0U+YgaY1RcYy+UcUqMqCuGh1LQm7MLYR9wdQkgBCVEjfdkvajvjMIOVBqIJDXq5MgTgmIUgINpW+w+v7yxAz6+tN4ISJmgmCM/jzSeSL1QCquAOqvdOKGU6kD9XQT2GsA/xDDbr1P0cUEVltMY9iLjIxNhgpVB31xAReqT6HXb58I2HR/G0HlGA/WR2KgmpYjR4nPYfpwinN3IUifJ59xvCytLa9hUH4sXcDbkTYCBkB7RP8ouThMX1zGFaxGtwK/EYDiTerw8siDNAVJWAE4GxH6DORgAHev5+mLWG2nrIBo9J8qLkcYA/O4BR5Ffej69VfTIiv+ujjuunAxPX3pYrqI3YKqMfa7K9xz1HEEMqC72iWep6O0g4GH297l9OD2rbRw/15qZOW/uRvdTEhB/8XL6Rx1P00fdtCHgi6lFo8AwCOQn/vc8/sQhHXtApBYND6cSQ2QC1YGsG0gUvbpM+kw4L6de9ILOVBd5yR2L0o87iGxuIf0YZOxReAUiBh7bRu4h7g8Sk2Mgw6IhcENjbwe9x2HAg+Q0Bj8UDWwJvJrQz2vjbKbUKPafkRwPEjmpsQCacfuyR4GLMH1sPdoYjy2kbdqc0Q8CUNwx4PewJohX/o8a+K5b5BUUL8gOEjjdnymSLPN/dshXUQSNx/6Q3e9LbR9EwnLKqpd2wwoDcZD9crnHGCvil8HbW5FXauBe7Vq/RgH4f2OPBsgHpKL8Irm3KiqJ+PBsaxqYSP1VuVLVb82yT7PyTJ5eD9bIFvt9NUJnjOlGqo5GsviPITxHGSwg7pk1cYMor1/PsN5/vMHPxlLZfN4/e8M2vfSOOLjWjOJLXKMb3Ed17vtHd27NgqO68rZktY8S0mmjyxe92d/3UhUJYzZP7Iq+ZZzexmVNuW679VpX55V3WtpX1eD1x+opX2s3+pTWsaf+JYP1R/6Xf3uO7S0s+zrCyxtLot5uo/008VzcrAd9MBBDxz0wBv1wJsSi89UqlBlgnlSJp4rn3I+z92+WtjeYPZ/Up5O707AAudCKuJ3jWDwUqnOBbGAVLjPhttZYhFuGCEVEc8ibCz0RqR71Az6NW4salDaPhSJhWo/NVUlXrpKDQK4VO2rGsObUFUdwPQ20hANliuJxRbSkYiEDPiSBAjmXKms5QlwCMkEAEP1pxw0D4JBHUoMixwYb49YqAZVU4Wq1UNiAR7jj0B9daeRwHIQCwjGPOB+DGA7ykqu7TTi8SCg07YIsDXQBpvmGBaAD3oEYoE0AHCkoWsL12iboHHzBteY3wQgfBmPTI1c28+q6QWAvACS6gSp0L7CdXHtIIydMfJoJUCxfdEFqRkA8J8F7An4VStSTesOgGcWoKZRn8d7ITHnjkiCMCYFoLlarJqTREr9Wdapw95BV6s3hsfSHAB1jfwPYVtwrqcrDbFKr8RBQ2+Jk1IPbRNcuV2jDfcA3zc0JEb9aAqJwwq2CdxAVn/bkVigPmMegNnzldSjlTIdg7pcNdDbNGDWTYN3bRNu4eloljxdde5BYnEJFaRL9Iurv0orHtC++4ByjXE1gu+WYLDXgFjyqvrZMID3cxCLB5AdV5U76CuNlwcgOkZFnqacKcpYxuhcb1F6mjqLqpbk7R4r3p/BjesYNhpLgO3D3ONzqCpdwMZCguP4kOgt0t+qYTVxQG9fqlQtLTxM1197LU3ductq+EZq78O245mn09ULxIiAwPh8SSSmGRfD1EFSMA7xiQBuglVI68N7d9MaHqGaAPfaV3QPDKS+M2cZyxgf0+9dAGRjJqi6KNHUfmcSgDvFfVucQ32IT8v8DKB8IVy0Nvb0ppPnLqR2iEULbTnC9doAnAEET0JkRrDJuDExlZYhdlv8bmDlX+KsdKG551Q6DMHrghCcgdQMQi5ceBhdWEgjk9NILFApYly20fbW9RXKJW4G4F5vbpLzbdSodo5iY9FODAzya+A+NDIOmwD+EjnVCI0wvkEeelyCJYQDg1ae/SbAv4bbEXWecetD4by1yX4boK/LZNX2JAy6BG51roBUrKJCFRJXrtkmjdIIVZW0yfA+tSGxaLR+tHWLe6SDgE3qohF3A3kd4XlSlVGj8DXaxy0LmyEjwzcwt+h5TqlkA6RwgzwcC62QiGNKJqi/83Ij81g3fa1NzEWImepRh+L5cT4vgN853NTOzG8Asstxk5Xv8YUynGxq2953c/OX7wA3x5yfsnk4//T4Xro4X5cuMsl/yqXVfn99y8lSnr+lJ6VGtbLJu9SinNt3TSm7qnfJt7Yv52sH9vqsHHrSe89z5fiBxKL01MH+oAcOeuD3aw98UcTizRpXJl738akSl+PVG2J/FmVirpvIY0KvJubHiYUTfxANchE0+L0Qiy0ApoG/QmIBoXC/DggwAne4m+WY53WNKuIScCtF0NWrkoQ9YpFVlJRmCJDCniHIRW5Xfgvx4uGFrMRii1XFsLFgZXSTj8RCAEBBAbQlF4KS4hkq9hWpcHUyiIWSC8BAEwbcRRUq7AcKoaAuQSzIq9a/9J3AUUNe1TPWcDWrjQVUIjzgKBEQuOrF6SQg4hRARNeqehQypoKyAF3CKkFQu0O7i3sA21GALg0LCcIhQCGQKU3Tb7MrEChAYRNguZvr+lkR7wNIt+B+Vdxgf2pPsEafqGp1B+nHDPnp7eg47TS2gx6BTmJQax+q6z8KWFKCYtwNDcaNZNwF8BsC1J0FAAnGtXEg6wAnkj5XnSeo4w3A+CgSCCM4awDbQ97nAPXPAqqN36C0wjHSTtnt5Om4mhIkA9JfgVyMQiymWQE3NoGESH39k6gBCcovko8ecyQ7She075AkGIFanX8lAbuMp/sYTxubY55+aQPQDqKCdAUD6KeRfCi9UYXqBhIJvSop/5BY6Oa2h7yVYISHJq79DAbSI0hSlsj/BKB2CImDqkzqv8+Rh0bUE9gntCMdUhJwAeDvmDL2xedQ6xrm2kdc24EajNde6sHdLKBYg/uHlD0OKTH4n8+KKmldSI62VpfT7Vu30+SNG2kFoN7a2ZUuXns6DV2+mK4htdB+QDsR46DcgWxKgO5TB6GYMRZ2Abxr4xhMs/LPGn0auHgpDZwbRCXsVFqgTJ9dx5aesRwf45CTSe7VJGNiGkI3j23D2vRUap6ZoqAVVtnxWtZ3Ng1cHoKkQRK4H+tIx1xR74NcSFwnuO46xGIKQ+4V7l0DhEZ1pUPct/azZ0O6187z3Mk9P8t4E9DOQa7HISRKmRYhAM0QgWaITDPEYktbB/poHYnNNoEYdzTeJsK7HrbCUxPPjIBaQh5GmNzzbfpDyaQG7HpG0/6iifojTojnqxFpAs1lfEiYmSPIe8dj1ElicZhPG4sRCSnHOupvEauC8jYhIDsAfNWmmnlG2slEVagw4Kae2m4ZM0bpoipWDbRbIiWBa6Uea4wj1cH0crbNtdpu7PK86KGskbpsS4poaxt1Ocrz5fPh/Ol96ua3xOIS6ndKiJRqqa7nfbMt/s37+MHz6NPEUZ6N8j2fcXTkc+4KaK+9A0qi+tzIo+RtXiU/Dte2fNzyPFRORAE5zb7jtcv4Yg3q0lWnSt5VTfcusF1V28q5Ulokql1YznK0HNvLJX+r8imHS7vK77faHxCLt+qhg/MHPXDQA/+x98AXRSzqJ8fHXxblt3sn47J34t03OXOublqOfin5erx8d4Ivv4NM+LvuE2pRAOBMLLLbWSUWQShin6UV6xWxUE1KYqEHKQEnvCDAuuQhJBa++HmhCl6VZISEwRe4L9jHiEUWwwMaNEwOYpElFjViweqw4NyWh8SCfJoAAJaViYWriflTTywaIRgab2dykclESCu4vj5+RX7h81Knb1lsraQWmVjocnYZgLEI2xC8SyAEeK6Uu+Jv9GaBn4afAgiBtxILpRjaWdzFKHeDNAK6Q4CMRfp8ImJAIPFhhbwRgKYrzC7Vh3C7eRJ7BL3/HAGJKmFwVVRC8xrEYhJQvQwoVgJyHBDUo2EwkotO0gtcp6wL4NWgfUoljG+hNOA8oPIyK+4aUAsuNWB2NAiiSIax+la6i669hsWj2IU8Ih/7sweAdA1ioPtM3awahV3DZYmGxtMPJSUA7Fus7mrIPQbgXGPleBfS0EQ/avQ7iI3CeaQPg7SrA0KiNyRV2Vy5nwIY23+qszRCoGYBuMPEYRjXqxPXnxk4k66cH0zvxrPTIdIY6foG/amh9xKA9DD93Y/UYQC1E/OXdEqMPvtgKt3BQHqBNN0QlGsQk4uQB1WAjHg9w30xGJ75SXjPIbHognhtUodXp+fTLcjF/MJKRFV+ZuB0GjqNO1b6TtKpKtNdrr3P+WWuP4I7r5OcM7Dbg+HhNHfzRlrF5qIRQN03dDVdunwpPXuO+lOOQNSggdpzDNOGEe5nrNhTzwaMlhPgXANjSdnVp66mIYjFUE9XmoI8WG/H6SAAX5WkKSQ3DwT5qAFNsXo+hQrWMuSiaWqSG7oKmMZmoP9seurqlTTQdzp1UkeN9He4/zoJOEZfLACOX5kmJsrUDEUDypFGtAHEj2GPcRyDb+M6GNW6gefvFONMEL/FA7BIuQvYKjwCzO9S54SkpOHhPIQRl6886xvES9kMo22C4R1GWuGzS+UlD7vco9joS7w/BKHapg0Sf+lqE8fCzSzpdyUikAPbvUs9NlGfY0SjGsW4hfzqGtk4Ekew+2hcWUrbjOEt+mqDNq5TD71SNSBpaYH4tVNeC/Zb5m2wO6UhSv8kuXow01D7MGnbyc8xatRv50mBt3MiAYkhINgYMU58/nZ5FpQchbtZ1J1Ud9SzWhALyJ+qUBILietRCGzx9EZTIs+8jx97c7RneCBrc3Y+HX/3jkF2Is3eyXgj1GfoQ23/+pS7Y/OQbcnH8vG9Y3HUBLE5l0fS/Kc66i5n5txR6mOOT6xzqUNVgVqWZpMLJjtz2r+Z15tupUEkKnV40/TVyQNi8cX00kGagx446IH/mHvgzYnFK9ndrA0oU6svByfV8qmfNOMYacve6/zu5sRc0sYk7cRbNzmXc5HWibz6+AKUEFi+x4q0oniGyhKLbLwdBtu8iLPrU6QXgB+jQ0sufOnuAhzIJIy3BfHZKxTEAiBRJBcRtA7wIKlw5bN4hvLdZXWtRyYWSEnwCrUREgtUHAAUO4BPQUAjID9WO3nBh0SC/IOwsNoYe8BBIRYSjgZIhWpQ1ilsKii/GdezRQ1KkpM9QpX+rKQWgI01mmTU7RW6U7ezy9pdCNSpayYQ2bZhAUnGIkBF2wPBt2DXmAy6UJ1A3eXVyZnoI111noQ8SCwMtDe3BMCBWBA6O/z6dwFCzuK55yruR3sA8kcB4azzAvJxZ0r6lzGyHp9dSEtc00JfnARQ9wI6NQjvxYZCsvOI+qmapY2IBtazlsMH2QrRp3H5Ctix7rqH1NOUZES1MonQBCvgD6ivKkrjgLNFQCVJWCFvx1Urqk0AU++ShtxGHHf1X5UmycEoJEHpzH3qNo9a1TbA2VV4icWZHmIwQE60szgBwDrMCr83XPUuQfaSpJHfqsE9ZAV9YuR+miFGwzr5GrjtaVb833NxIMiWq+ETkBbVyyZRIwpSovQI4PgUbeuMlfjt9Aqk4ubMQ4LYQSwgR08jcVDtyT4S/Bp/YoZzBsZTT/44ALUXqYeQ9xbSl9uoQ82wb+XeXyUi9RXsPc4Rb8JV5xXS20e6zJ3mHitJUdS1xqr9HG5it0bvIbVZYHmcmAZ4ZLp4+XJ6x6VzQSx85pRwjQNavd5gd0ZyXgGk79D2RuwTWGdPR1FFewpicfXc2TSEJEvjbG0R3E7Rj3qHmqfuYwDyyIf+ngLkL/NpXJhPDdTPMd+KZ6krl8+ncxBWyaFG8pLcDp6TAWxfhM33IBMjup3lswwhontSG16cjmPwrbRwmbyWULVSLewQ46AVghGu7Dm3icH2MjEvNqen0s5DbDUwvN8A8K8fIjbHSfzTA+yNti2o9mP/bTCodAxh3iiW400MT03MJ0orVEFSVUn7EL066U0KJB9SA+2stiGxu4xtbTIMtKfhdBskqY0b18x80UgfbtGvG4xd1Rm39UZFYL/D9Fk7+TczZ+0qBeX52aEOfjYZfxILDcT1DmVMC1W1XFRRaugzuMU4D/fQTFbaYUjsPM7jE3YXqjpp9K0xOKMhxtw5xpvP22nJSkUsWFaJa0jyxM05vX6+LomcGyPj6gDJnG5rW8k10tWlyTU0rWSkJHe+L9/dU2btJ+nK96rIvevKiVyVnEe+Ml9T5eKJ6iKP19pTX+m6TGvnSft4+2t1qYqOEuorX1eOSerzqi6p7Q6IRa0rDr4c9MBBD/w+7YG3IBZ7AfJsX54feT3wpRAGj5eJshxzon38u2mccMuk676k4auZ7L04/F59dK0qwPR3iPBBFIVcZM9QqkLtEYs9lahMLDK5qIgFL/F4CatqJHjXxoIXdRg7Btj3GCRAYsGKZSYWgvqq7eytdxCLHYgFq781YiH44KUteWkA4AuEXfGuSSrMlzJy5G1JjWpQ2l2g88y5QiyCUFC+9cueoTLJsRL2VxhwWweAlSvTa+BdCcUK5EID7lXaqNqO98DVZ9u2zMroAgBJj00ncGd52HKp2yOAiIB+ilXpG6j2CFh7AabnUe1ZAbQMA4pHIAorrL7vIrHA92XqBoScQ2XmnayQ9wHgjqoSxXVKD8YAVJ+HVGhoO4fKURP35ThE4dQpiIU2CACnboC+qiKlnrqElViMATp143qKVXDBjl6e8gqqfaknL8EermddARe0s9o9RXka5y5w/WEAajfSkeMAPNWYtBXpg8i4ci5AcwV/AoIwAhEwvsIsKjVb5NVMR3agTnUWwjSIrvlZgLuSDu0zNEzXxmSScmyf0dBd/Z0BYE+PjhEHgT6DrFy8eD49PXQxPY9XpA6uFZjPclxgP0YfanwuODbfZyFY/RU5uA0puIVRutGxTwA+n4HYnHfFHSYl6dO4PYgFeUkUlMIYc0Cp3S1Wve9xz2bJQ93/C2f60pCG5F2AbcphxAZplJRMcd/mSPOQtszT7nmCw6UH9yEIGB/TV70XLqXLV66kF4glIel07GibssCqvIRMT1CTjJFZyOUSAH+H61TD0QvTU0NcS9nW+xHtMLCdsRc03I84KHyf59lQKjYMORnlM4/kYgf1pF1IXhMd04jk4Dx59GK7owvUB9rQKGXhnp/BBkaPag8Zv3OA+QVUf5apwyLAHAMNSCGgnnGuB6QFjNpdCAjXq6hJHTtKHfjdACFYICje2sw09hVILHi2N7BXkFjsQip2UUPTOxPKehCL7Nxhk04wPs421+7Sf4VYNEgMIFxKFHZVnTLuBXVo4Flo5FnzwdyiLjsQgYC/qCAZM0VvTbqdbUIlqwGihR/YCGS3ybO5jbSqCVe22k60Kwmhv7Yoh8c6z4PkaaDPbcrTFe4RiKkSLG5djFGfP++a40ApnyWr+qRdjM+N85njuZULXJBxPnV+C+cKPB8ab0sEvfchsYjRQ7+xfztbbQaPST7PmTwKb7rZBseqm3N8/h2/+B2Hqz97xGLvW7w2aon2rq0dijys115bqkzN/EkXeGl9wXVprJ+b83D5Hgc8Vn3ZV+WSD+nLeZNFmnKuuq7sDohF6YmD/UEPHPTA79ceeFNi8duvQCxQ+XAijY+tLN/Zu5UJNr8U9qZPv8WnShdp/cNmWidX86xt1URbn5/fg1hIJrymkIrYZzWoEP8HsdBQtfIOpStYVv3WeDlnOwteygCzXQCttgeNrFhmYsGLVGJRfTSsVYqxRywkFbRdcBKNyfV2FX0LYhHG24Cd8AxFWVltAVBP4vDoJLkwvyAt+4lFJjCZWAhIJBhNSikAyFE+14YxN/sssfDlZD1yz9KUUI8wxoKkYgXjbaNv6/PeVUtThb0IYOQRIOcRwMygb9pcuHKpepRgb5rV/xlA4zBqNRp8niP68rODfWmTOt0DFF9H5WYOYqE6FJ0Zrl1VGXrXYG+Af71DScCMRq1rUYnFLVSExiZnAI+rsQrb2QVg7sFzE4ThDCvSza4YU2/r6apqGDxDEF7FfkLpRC/qUEOU0Q/YUfKgFxvvgYB1HDIwBrGw7hGzA+JjIDdB1QnA2QlAknEhtCk4xbVDqBa5AhuG46iojAESHwDqZtVNB6ypGS8hOUeZgwC7PlRIBOYGU1qiz+4hddCAWZUxVdoch0aEnoVYrE7TZ6xMXx1CjQhi8TSqSJIhx622DaPUU6nPKGUacVnpztP0wTnIj16HwnUr0gBtJvRG5fWSKleZde+rXYl9OoW9im53VWuTXKjqdRNiMQmQfki/qdrSi6eWi1z/LP3cG164ANv0lxKpGfLQSH8UID8BaJ+bnkoNEw9Sg+pB1O304Pk0NDSU3vPs00GMJHJ6dFJCZB20k7hLmyV/05CLVchJM/19HPB6ceBsuOs1kvMiRFXyqlewiHBOPxr13fs8b19S/l36fYpxYXwFAi1kF7Lcg9NIDjohJwZxs4w17lELfXsS4NsGmdpmnDDMA+yv8szNQl5d8fdZaWa1fZuxvYILX12w4tIsNSN1Ow6JE4QzFHBTTPR3PjsYrzsPGN19E6mFxEBXsaoYNdGP9r1ExngULGWgSoT0gLru+HyTdyMRtyNuBf2iR6ddYoHAYFBjIgaIzzBzg17LtugfyaSqUA3k18TYaWbO0KNVA/mokqSE03lpEzuTFvI4wnhvo5xd+kX1Sq/z2XdTnVMio4H3YerqPLbD/Y1nnTlBKYu/Jb7OT8bF0B5HYtHKmDHmjBIOF2XM0edOkuozpl2KcWAMeqnNk+edZpxvyuaY3jdfVyc8VubsDJn3X1Ouf/0+4HVdnjmfmN5IXB/MLpdRCoyT+7KjarHla/fK92CuW5UgUlXfvagUFsfr/pQMy6HH0pnnXptzoiiVdKX0KMV0nLb+5bzX1dJU583BY577zm/9Nn/+nmy+O8s9Lfv6gnO/7y3o/YfyCjWxMs/7SLutg+2gB/7j7YF11N2/HNvJ1mMEN8ZWkLn4y7Vpf/ykOeGLrc/bJxbm7ERZN9nGZFn3u0yi7uNTnXOSrU3Idek5WDvu+ZImyAS/3ZumSCw0no4XKBOjwe8iloUifj7FK9QGgEBiEb8BNZlY8Konq5Am8JIW3Aep4CVc1KFqxIIXcVGDUkpQquurIogFdhYSC73LbKJ3vskqo8GqVIVSbSmMsHl5B7Hwhc3H8rLrWYFrkVgIOgDnfJoaPa6EovrE90wsVK2yN+13/0kswBEBOCUWS3iH0jRbwlCtlQLSuYb089RLdRxVIPoBE6p9SUhuAcjvYxMxwX4ewCmgvtB9PL0T70Susk5iY/G5qfk0zqr4Mmn06X+c68+yivzcmVPELCDCtcCfegqABL8aLX9hdDK9en+cAGLYMVBWu1IO4zSc6iK6dQdgJ6/Ia8Rs/VyNnwOM3YRYrAC6jEA8NNAXQesGWN3Xb79gV5IpeBWw3wPsjgDepyhvDsAr+OsBSJ7pPhk2Gm0MPFVPugGqEilB8qzXArDHALauxNuJqlydJo0Rty8AsvoBWHpPcqBo+zAKqRihLMlI2LbQxjmA9RTqRMvo/R/hXj11ZSg9d/Vyemd/T9TVVeF56qbRslIcXQCr+uWq8RX6bAAQ6urxHUjFPSQ7j8hfWxXJVB9t8E47ziRfksJRyNQCfX8E4KrHKMf/Hfp2ASmShKodYGxMjdOA0/OQFqUuSk58Rh6x8q8q1xRgX1I1TdsfYfxsHIhNPmtIH05qwH12IL1w5XIYl9vXq9gkORZVbFLadYs6agB/H4JhwDhJ5mHGYK82N6zYG5TO+6nEYplznfTrKUB9H4RNRUYdB9yDVIxw7+z7ZZ4XSc0u5KCRcXaUfj8ksKb+ixBdo10LwFsZb7p4beDTBKg2ArrtWkYlahOigjiKyN3Hose2UDHaxn5mFxJnbI82JCrNAn6kIGvMBy4ANGC8DkpH7c/FCgYJz74G0aoY2abQ2+e3we0aZCTUvRF7LUlQA5IqRCVZPYkFgJ2jRurOAfWM3u4iRYxt0rrIYSwPyYDPBiYuIcFrQgKCmDPmAsefcWA2uK6J9h2izBYkObqv1b7C6N1NfJRoukX8HZ4FnSVwYcx9qlIxYcRCifNmniOzg4o8zzWEPYaLJkryNPSWYPhM6b3L4Hj99PlJ2hFey0gT8x3lOc+UrczJ9fN+OVf2JU35Xb8n29pW0rmP+az+ZJWqpKld9LovtLtWPzP39/6N7NnK8bLfn+aJv7jwSalL2+3jJ23WoqTxfGnDvmN1F5bzdYfSd37oW+t//q5+/3IRi7/+Kz+Q/sXNj/2utu0g84Me+P3aA//JuQ+kv3L+gyz0sPD1Zdr6UDF2cepL3b4oYmHm8QKoSomJ8rGXQZk8y/vjdemra51Ma5O2eeTZP86WiTbSmI5PIRfxnQk9jLfZOymq4vI4sVBSUfvwAl8DwOglSgNhV/TcBNy+dIuNRRsdKMGQXBRiIamQMeZ99ZKluoKkIrHYhFgUchFB8gATEgsuCrWZMPQEnIV0IvLWtqIiF65u8lEdKkgFQCXbWCgBQKIiuaiIRTEiN9/crxWxoH+MwL1EkUuQC6Nv6z3G/5lUZHUpV9t1zypM6gXAtFLWCn34BdRwbmIIPQap2ARU9aA6dB7S8OwpYinQJw8Bta9wfgQphN51NARVGtAFCBw6fTJcybrC7+q0Rst0WEgSXsGF6sv3HqQHEAWlDK0AxgHUp57GJerVUycjD9W4tAWxHrMA33EA/D2kJrpzbWFcDA32h0H2BYCpgd7cJBbeQdsj0H8Vb0WjtGEa0iOxkPBcZtV+CPLSQh8IwDWiFyDbb0vURXB9H0mMqlSOhyMc7wVgDSEpUKLiar+ru463RdKqiqR05D6AeJk+VNVuGfA6PXwX9ZqpGDcDFy6moUsX0rMQMgMLCszW6DulJAJqXewapM1xr6ejLkCo4OQGUocJVt7XAMjadVzEs5RRwNe5zjxcbVZlTZWqCeqsFMMx6tif4lo9/ng/jLh93PtAOzR4Pg2Q7iA/+0v7EvvLoIR649LAfpXnYcdgbRCMRe7rUWI59CMxeMdAfwRI5DaG6pjrNdpmKPkYpaxx+sJ8jOqstKeZNh7juWlHrcg4F6qqlUCJBlDUpsQo4D6n84DlEQjgDP0Xhvvcm13ySpCuBvrX+2XsFOM0rFLWDuMhYlV47+gvjZubcM3aQDnc0Ag0pwRBY+lGVtwDY2pcjZqVpO0QxKIZwtXItXp6UhltU4cOLAIo1ciqTdBwnsdd6hrkgrziJvF8NHCfQmLA2FM60MS1zeS9DbGSOG7R5h1sI+I6ylOG4L1ROqmjiPhwH00rgQhvdCxGNHLMQJS2NeZMzq1xvV6dVMMyfoV2HRIOy5fg64pWCasE3o/qXY3kK6lUZU5bDhJE1ZXGOEfSa3muIK0R4pX6SSzMV+mGak8GUtR7lx/VonSP7JjzaSNHc+Bv3nweyhb1Lj+qff35x07lzDhYcjOrnD7nGXNaOUm6uqJel1U+sFeXvVz3J815lHRlvz/NG/1607ZUFz0pTWlCrX+oRK1k71Hd9qTrPX1ALOo66eDrQQ/8AeyBPzDEwkkwJn9uckyadZNkmUTjfHXcKbR+Gi1pHCNvNKGWCTjOOyHzkVQ8mVgA7n258wLXvkJ2FFILfksisoeoPRsLj2XPUMBS8gxiwYtYVaFCKjKxAPhrjMxxgV0B9LlttglAz5JmtrEQZKonjZGl/ukBQ0oswuVsda3EQFCbPUNJWsw/k4tm1TWCXGSJhaQi7CwAOqpEZakFL/kgGXvSk1wL6kFfwilqxGIR/KKNxY6khvINsmbfaW8xB2BylVf9aVWhXL1cAWB8buZRuo7x8D2IRTNpNZw9h7HsVQC2kXgF/TdRBRqeWwwvTAKZFtp/DNDaR5pBVFcGWSU3sJrSDmMFLAA2b2KU/HkkFq9BLly9bqHMXgxzn8Ul6jMYfWtnESu71EG3qPcA7beRnIwSZ0JvUvL0obP96RnIyyU8IVlvjaZV+xBou4ouYP8CalC3qf84+x3KMcbG0/3d6XmMoAVRK4JYSIub0gbd245DNEcAsqo3bXBNO3kaR2OIa3Q3e5K6ahugka4qPRpta5+hhEQj6AUA8hbGw3N3bqdFYjlICjv7z6b+wXNR5+MAYFVPJKDZWiEDcz1yCdCVWgjeVBm6AamYJYjbLmNHFbFLg2dSZ2dnGHMrM9GYVgN5DcBH6Zcx7oWSJsf3EupCbex1YfsUHqkMfFZWnJVqKImh1UTRzvFFxmizMU5iLAh2+biq/oiyBcXathjoT4mNZMz4Ew+530paNEafsy+5BgoXREEbggZISwPntVOJtkp8GVs+P10A827qdKq9LS1DRFQ9G0FCswbQ3WBy0DVqBJTknEbc2hUIvgX6sQAAaTf6tM9rA8/KLnYIDRo5QxgS7VPNTmNqo3Gj78S8RHN5FlsZJwYq7CStkjlVkJQa7OCRSUK1APlYQuqgC2UlA2EnwViXsER96N/YgmwgxeBeNmFHZBTuJgE/dQrbLp7XMNqmDGNqNGxAYvmudMXYKw2kZ9Uj1KeKZyelLTGvUFmD8Lm40Uhx2vKEpJM+3obwCf4NshdSSsaAdhW619VeSg9s2p5YH+vrtc4D2xyzE8Iejfvh/Ml/2mdQRhdOIC7cH6UoEoujtOsU90dSoc2Pkj3HjNIMuzJ//Ju3Mm/Xz+X156L/OWCZtW3v8rpDWeUp55cTl2trifiS88kZ7J3P6UsZe8dJ78VKoNj03ueW0+XvdQfiawyYkpFH6jIrbc0Jv4i/5GMp9c2NfqryLzUo+305louqkwfEYl/vHPw46IE/cD3wB49YMPnWv1ge/15+O1eW7/UTuMfLpF32jhrT1ibdapL2fNhUsA/xvr/97guVfRALQQ8v7xq5kFjwco6PBANwEtILvutlxmtc0YtVbMC70oM2XrCH+EgsQppQAaNCKsKAu2p31J8/yD4AW5KYLLHYQh1KYiGAEBT5albSEaoIlJHtLHJ5uQxJRV7drMWwMB1gJdSi6ohFgA/JAmCg1qfkL7EI422+LIMTF8BXEgvdZApcBSESC9WTHgKIlbLontJo0BIqjaf14HQdiYQqOQaWcwXTgHZXUC1Rl9819lsA8LtIBvTqI4FztVW1jhOquaB+cx67hvOAaV3JhrEuZep16VVIwssjE+Hi1rgdPXgNErw/haqV0biN56AO+DL9dRvA/BK2HDfuT0aU43bq+dS5/vQc6lOXITrqfS/Tt4JxV9AlJeK/UcDlLcjRPVS1vOeXUYN6HhWt5yjDVVe9Ks0CypWMSLAMczcOuFYVytVzdeEPM376aesF1LT6MfgVnBtHQ0lAiPO4TlIwhlThHipJk6xYq6u/OnwnrU+MszK8m9q7eiAXA+nMufPpMNISoqGFhKhDgE1ePfwW8wgCVwHk2rtMAaZvQSwe4T61ld9DGG4/g1emHuxR9KSkVyhXlc9yP1QvmqKu96izalkT3AtV146SXz/34Nr5ASI2432IflHaoZqLeviubmtjoQH3MJKJGWw1dAcrMPY+Woa2FI6rTlb2dWcbXqkAsxIygxgqaVEVSkPyZa+hYyWpW6gb7dIWHoJw86rb3kbK9FnqYGxohG0U6XZW/B8ilZhhtX8YMmQshh3ubzzvAHIeTAYwtha0TYNmdBZZqQfom4YVfvPfVYKALUMLpKsJCZbnDVq3Aynh4UeCkVWFaBhjI6VOJBWnUM+SKEsyJHPa7qxARGYgFKOosM2iSrWkFM65h/yVgjCZhLqfkrkIjsl1rAQEcYn4FfSv5EWCrUpRMqCe1zEmJThRb4hPVjtinkFHt1HVJuqpzcW2ZMW2VZKI8ERHOyUg2lXozlayG25ltc2guB3KlPwYJ8O+PcT4PEa9qFXYaYQtDMUrtXCu8sGQcMRiC/dJ97MtXHeYj26bqXg8P44PyV8/qlBnIRfZVTNzIHlQo9icx8o8vTf3VCerneet5962N5dnuF1/hu/Up+Tpj7iWY3Eikpbrc6b78/baSFT3pxwwff5ef00pq1Rx7/pyXb6qvn3lmrpCqCc5VBe/7nwpsJznQst7qzzr8y/f/yAYb/9Xv/Z/p39551dLkw/2Bz1w0AN1PfDnBr42fd+59xPf68unCtXb2xuL4HXVeltfAzuxWNiuY5S6jcXD3d1ivO1E6iRZmyyr7/mtkCdRv5c05hOTat2EWyZ2z5WJueQbx/zjRllO+Z6rEQu+Sy489iRiES9RXs4atfopZELD7fxBYsHLtkYsaIngWwlCIRWuMgcgAPhkO4cM5mPVsNZeKkZDJBauoK8hsdAr1B6xQOEiiIVqz6pbZXWmiGPBSz2Tiqz+pORinyqUKlEAHN3dhuTCa8kjPhKLqn+jb6iGxMKVyhX+LEIqFsRgQAKDfCmt0COUaSQWAkPBoCpGJwA3rvob8+ILSAluoEp0D512X+fqXPcD3K5CLFRvsqxbAlrIhZ6YWO7E6FjXlTmKcw+rnGcFJuw1PhbwW55elyQrN7HP0EvNEco8DWg9h8qKRtK9ABqNnDUgNfr2A4Cv3pFeujcGYF4MVY+ncZ/6TB8RrSUW9MMywE0jb0mCAFqgKCkxCvUwBEm3uH3YLzwFeXkOdSvVOjZIq03BPCDZiNULpJdYqEY1RZvWWblup2/OobJzGU9Y/RATx6nemyRWrjpbfyNRBzi3HwCjjyAWa6MjaVNiQVyCY9gonDwzmE4PDWHIeyytcy9XAbkG/TtDWy8g1TGat0RwibpMU/4Yed1GnUnvWa2M2csYXT9/AXLS1ZnBP/dH25BeQJ+2MRKEce7BbQD4Xe6XnpOa6RNJyxXc/woS9bC1Sl4MhZDqSCDmIRNzPOATgHtJjappPmGukDsmJNqSYNWZDJJ28aTBDAG1HJdILJCfHplm6EcJkbFHlBgZZ0Iw7aeFgXKIe2z8iKOsxBt1W/uKHKBwNyQwBsiT3EAFon6xuEwZAeKpW4KwSSyUgujVKWwruA8blGsMB/y1psOQ0zbGWxP9alDBVdIL2l31d0xL1pUKtDN2e8jjIiRPGyBVjNw2IS1LzAn3pmbSBNGvVSczSnYQHZ4XSUwQFfqmBgq9lLZJMMLmgj7YZTyEdEM1K8amJC1+S3Aw6I4o2jwjzUhJmll0UGXLQHdb5CGBaeAZ36U854BY0OBJbaReuxCL9YpY7GpYTr01WKc2oa4VRthcL7Ew8rZSRtuk+pn3PAzE2TfyWxXRNfrS+DJ0TtwX1fQkJD53qkadoP6q/vkMO86OUh/HkE2OHnMCqLZaf5QD7Ms8TlX2bYwoz8b8kU/kjCpBQhyi6vRxVYA/qq1KyS/fJ+Xo3r4u6b4rco338vHakrZQlZJLOV5fx/r2lXaV9FGNugwfP1+r6F7Gcemb5Vnyfnz/B4FYPN7mg98HPXDQA19ZPfAlEYsyYZa9XeL3+t9OtnHMk49NuPnQ3ksgfvuHzUk7Ju5qX9Sh6omFq4Wh58xLVf1t1ZyUXGRised6NttXrLMoWhELyIDuYNU/DmLBC15iodGln/3EYg/Y2xbflfFuYS+52fr/2XvT2Mq29DxvcSaLM4tVnIpVrLnu1Pd2t4ZOq9WK1JIsq+VIsiU4gw1DMWwEkoLEyZ8gQPTDAfInQfwjEBAnkGDDCiIFhluyLHmQ225JLcnq9HTnGm5VsYpksTjUwHkm8zzfOuvwkLeu+t5uydDtnF11uM/Ze+017bXXft/1TXwkFtpZaMC9p9QC0KEqlO0u6laqhfjCVxIRxIJyirSiuJsNbzKcL8Qi1KIkFhVJRSEX0T/2ER+xlsHZBNdP+PJ0l9Vq0hvkT5UGyYU69upgC4xdnVZS42q8q8Er/H4DYnEHUD5DEDVfvpKJEQK4SSwGAB0CTonFfVb95wH/LVyjZyMNg1VPMrL3IGDyFIBFaYVERmNj1W6mXFknX6UNJ4mNcQZd+2FVY0graA+3lnYom6vhkpGv351Oc3j2AfGyCj+GWhPRpPHq5Mq79gUSC1WgHE+qOkkANZCeQuIxDSGxzRqifgRJh0BbIvWY8wa507vSHNfOArQF2UvU8QC3pZ3ozl9BKvDc+CixLAZQl8GOxFVh6qj0QrWUdY7pMlVbByOURywG1KC2cWGaHi3iiaovDY6dSUNXr6VtgOUabVZta4h+GqfNFyBUxsVQTWUVz1S6sLV/NJ5foN4sL2PU3p1ewV3tOWwsLNNx7xhVxcz7uEb7JRd3ICR3AOEGrjMwXCdlnQVAn+nOBtsSBwMCPubcI+7dslIK8jeSukDVHnf8aHMkcW8ibyVYqr51cy/HIWdD2Np0c0zwr/qWxOIe0gbJQZBU6mWcBphKBGHrBHj3I7nqBqQaNLGP8Sfh0TZmk2f0EdIFSdEk0gLrovTINnkfvdcNkK0GVuwlKYLiTtSeOpHE6CVplWs2Xe137NLONsaPD+IS6Veozzb92ATJUUrTSfo99m0QO+/fNfrS8alUSVJnu239DKpsU0iKpomAvc7xUCPiGT1gXBg523o18tvV/QbqL1BWpeiAMR7EwzEouaAfzNTgjvblAeU24OGpAYKm4TXDJ2wztCUxcrcEJm/k5XWkN3heexskg7Fh/Iod0tEb4ZVKEoIcJIiVThCsk8C/0/tFfVzPyvNeVldrhqzrhUsbIYnMOuVKZO27Lp6HXsainp/MQ1e0PRJ+pJSqvxkHRhVHn0tHSX4yK9Vld3R+d6En96YpyJ72lN+HV5Yez7nYi+Vc2XsNn8q1JQd/hsSokndOE3+jHjk/LytXeMQ8a3+XVJ7Jxy2/1MkjR6+3TO/L0TxKTWvP1abxeNlqj5dj5fyzzpU0tfs6sajtjfr3eg/Ue+DD2APvi1iUhjlJllejbxN/l4kzptfKJOvU7O84Xy5+xr52si3TucfKcfdBLHhpF2IRx/it9CHUoSQWgJVaYqFajGSi6m4WYHQoscgAuxCLIBW84CPyNi/dAP28eI8Gp6M1to3/B6yyWSdVi7Z2AQFBLDAIBTyF0SV1se1FYhHSCsB+NX5FEBh/Z4lPTW+lAABAAElEQVRFJhV+5xjpqqSiIrEIVShe9pKMADiCTvIXmEksVvmzQLTtefabuHHVc1A/oMb6m2YD0LVKX4TEAjAi0LItS/Tbq09Y/YZcGB1ZzKP7yXEkCtcA2v2u0tKQe4CtyZXs4rWV38aH0A5BYN8HiNNYuAvAYz013n0IOJwkP73/bABq+kkzClC9CPDVoNfgbx0AI6UO/OdeA165n7pcfWPqQbpPbIZ1AOMENhkvEQn6GrEZeiE5EiOJhfEhVClSgqOe+BpAUPeldwHoK5wXJGsrIJGRRKkqIsh+SJr7SDUecL0kYx2iILHooW+eQ8rxChKSM6dOhvtdV4EFaGfx0CRIk0Rqq7DIPZ6jXaoiPUJqsUwk57WFeQBZexrE3evY+QvpgLruca8cX0MQKleEBerNHLMNG5IbSI39cx1CZF6OTSNmv2i96X9x0Dblec9U4/HmAGUJLKjROsbkkAVjU2ySTxPAfBBCeKGfeCHkodRonvZJgu4iBVKFTAmDq+jq2TuO1LPf4j4preggb42+uzkvsHSFX7W2XtJKRCUquoq9gyes+0gcFqi36kMy20ZIkqpGpxB3jlJ3VW0MwOj9VWLRhpSmSHsMtjcLoZKkSCTt4whYKcHht6pUWXUQaRHtGEAKZpTxJVbxtZURx/YBgI0l4j1/zPFl2qitRAvj3/r3Q86aAMiqZLl4IKHzWfDJsR2OZ+/LYzxDzUKUpiCxxifZ4ZjqSQeMV9WpfM60gYi5DnWmEr1acqItiN6s4kOdQlrhcfqY8NaQCspHQtVIX2qns8/4yobiSDpMx/zADQ+jc5h7SENasHFop88l5w30j/cH2UmQi5A++czT50pjJM+6hVZioUxR9cst6u2iivYgutftZAxKdjap5ypjRNsij/dBvFRBVBKm1ELy5X0aRbqjtMrnyfEmALc9zt9lLrZ5Za53gPrcHt1qD0RvHJ7mp8SrXC9pyPlyDX1U3aK86q/ql8itZFlbMOmpSs3G75KuetQyjiTKl3Ds6NF8wfH2Rnb+eVbiahlHrz3eb7XJYgxY7/fY6sTiPTqmfrjeA/Ue+ND0wPsmFnmCrbxwbB6ToxNoeVnEIf4E6PAHmy/m2vNxsOZP7SRe5u1yLPZM/oLjQipq94KuIBa8UG1EkVZIMiQVqkNliQUuZwEDSjZUDfEF56tTgJGjbSupUJrAyzb2WbIQUgJe4u59WdlySYUbcCGIhS5twyvUVg2xADS4CXwFSkFUJA2uTgJeaqUV+VgmGB4PGwsJheVCElRrKsSirOAJBoU+LhirArECoZBUPJRYAKdckZQUBNAnkSvduip1rc7Aa64k84qPgGmvQyo0Yo6o1wBSYxEY6Owqq8WCcu+lXpTuQiymIAuuuAs8lTx0018Six4+LMyGKoYG4bOAHFfVtUcQdOkF6SJSkEuAmiAg5Gkv5nsg0EPSwnVKFt4kkvUk5GLp8WOiaHfhGYpo0kR1HgIgCohVg1L6oLoPTQhg5DjRBsFI1wu4dpWkaPCsescI9VTqYNRq1YFmqJPxHOb5KLHYIiZBO9c+jxesj4yPYJDeG5IPwbherl4cOhlg2QoLbvUQpeRGtTCB7RKGvGsAVElWN+BsdGgIQ1kkJY4t2qUEqB87A6UOPj+73A/rLrExSvkt8jFGhCorA/TnJYDxKMDY61chSRtx3+gj6iKw1FZmkf7VqDqAJ+RDcCkJuMx9M9K45Qjgde06yf0NMExddFvqpqqewdbEc0qCVHsagFBaV2OSSPxUI9PzkKRKw+1F+k4PVpPEM3lIeyMIJPfMMXkK1a9RCNgliIWkwr7wMZHkqP6lfYqB8bRrWQQ0r9N+jaiNY7HLRwkftBkDf0gRdWhi7Cpt0tvYWe7jY0imkcmNp3HCMUkbVni2dU+sxGKHfTPkuZt+70dC0Ym0TY9LtlO7ESVsrsK7Zdsqxj7qiw+p0zRkSc9frugLyhvoT1XAtDPo4hkQfO9D5tatN+U7x+hymRvGJ7srjgqZOdcpgdDNrSpbzdxzicWWXp4gVZKLPa4/YC85OeAe2ZYDDNMbueetjNc22m0cDNXUlMgqSQk1J797Z+1f6qkaUxALbyJ57TCetyUk3M92+s1YIG20aZs6rtE/2lN53GCVen9SuiOBUH1xkDKHcDXr/VLapG2WpMwuic1KVrYyl5c5uhw/3JfE5eLDMyJzHtVKvuV89ACnqie4QDLjdfEnMojUtRXyVG0Wkco/HCzH41glj5zh4ZHjv+PM4Z8j1Tk8fDTrmuPP+mrJpZ9qq1Q9Vm3P0avrxOJof9R/1Xug3gMfvh74YMSC9sXLhUnRybL63Uny2GRdzpumdmI93kVloi2vkfLb/PwusZAMHJdcFGIhuQgDbl7avvgPo3BnV7ObSBSCWHDOawTmVWKBhCDiSvCidqU7iAarnRnkZ1JRiEWpt+tc/tPOYnsPScg2PutRg9I71B7AT8NO632UWBT1pyypyGQDQmNZRXJRkVZk+w5e7RViUSQf5aWepSVIK2iHcQ50NbsAzp5nT/guwF1z6gXgaMi5AZB1dVj7Ao2zu1F/GAC8KVmYZ7X5bQiAKiqrgCsBmGD8AivmFwE6EgtvnFGj7wDIJwHAgmKNvzW+HoZgaJisgXGsxAPu1MdXYnGXlXjtMgg7ls5DKq4BPCdYldUNqCvHK6QTN6iSoVtU1W00Tn7rwVy6T3yIpbk5AtxhuDx+Jp2fOBd2B+30k4aoBnsTnLrOKQgO/XnaI3nQbe0TV+Kpt3YcRvrW3amwMgygBbdcLzlRSvMEYtCIepxRwV/AePs07b+P5EMwrJHsK9h4uMJLdni3wpiZfjLat6pejwFygt0N7rlA0JXkfvrN/umm7sh7on2t9LlEQbCdx3MKYjDP9fepi4RoFQlEK2NzjOtP0089tHUZwBiG1oBHXf/aXo2gH5FWOwdV+FRncfW6j/F7mX4+TdmqqNwHcOvFaoq9IFTS7Dha4/o16r7Jvp20/YDNUVSfBgH13m9XsAWckgOBskRBEvuIuryFofltPFg9RH1on3ozQFMD15/GJuQsn+dx9avBu/YOEkCNt+0TVZ/egpTcx3j7kYSGf0r2Nun7TdTKJAAtuLtthdTpalavUGMQnQnG2BXA/SMI3BwfVeUsc5v+UAryhN/r3E9VjJoY5x1Ezu5FVWoAcqFDhnDzSp+5gNDGsyVZ0qOS42WO9sxBvGawsTFiN7NCuI21zdotIHNIpyEHShEODlDlIs3Tyv1QehG2GJTLZdSJe0sZSl1cCGhBFaoHlTolBJKEtSXsOHRTC7kQ/O8D9vUC1kh+FJX2eVYbcKPbRFubGH8tkLFGpTmMK59xyYgSLe+/5Mf4Gs4btkPVxGbnHfpnG6KChUpqgSR0Mo68F/aN0j/dJLdAeHoxaB/k+TZOTHiXIo0LBapDnea6Hr5LOlyUcMz7yfOx3/JiiW32HubGe0vyQoHnozsOT+UMPBEbV3mOLc9lFSlPXFXNzbORJgLklfS01S3KKpnEEf5U0rgrbxrrV1urSpKa9pSL373PtSpXHD2fa3H02Hv9Mofcd6VFOWU55q8yp/vd4/6uEwt7o77Ve6DeAx/mHnhfxKJ2ArSx5bf72u/lzVEmyegY01SuEWzXbmWSdR8TcTnpy8NjfIJQlO8SDD6Sidj7HWAomciSiuwhalt7C44VUqFKlOeVWphnI8hTwKE9QthW8DIV7IfLWfZKLnIMidy+I9Xmh0DENcSdisRiOyQWrEgCHDSatO62WjuLqpRCiQX5hutZgEGOxu2xyvEKscgqWFlaoatZQUuQm+hF8V4mWzleACCOrlrCwnNpH2NlyEgr1wgI7U+jYavqsQboF4goZXD/FKAyRVAyA7c9YW/fDOBS0xXiC4BxjbEFmnpt0sWqxMIAabqG1XD0DKDpAmmVCGioS5WCvAjaZ8jvHuo9D0lPdIF0Rfe12EmoEmXfr3IP5gDHkiKJxSh5KGV4ABF5Y/pBWoBY7CzMIQ1pSaPj42n83EQ6DVjUK88W6Er7Ae1GLNMVcdujColt1hB8lba5si/pGKGOlwdwO8o926IfVLdSgrNKex9R11lAlyBVjzjq47tKrh2B6lb2s3EtVA8LWwGuNzjfI/JW2mBMBwMPSqp0kyoU815JRKx7N2VKxCQHGp4L1gW3GhZru4JFQXrI9UpQrNcu/aYESFW109SjKaRj+V5rp6PNxjy2LrqelYjpeUkgqPpSF+2/CmBUOtTIdTewm3lHA2/A5knqr/tZy5+BHMzhQWsZoCwIHyO43SUIgeAykwI8GdEGmZkkvJMxocqa9+dVVNRuIFGamX4IOKb2jOWDnl5ANCpgELPnII+DECCJhbYgSkF0l0ux6TUIyS2M+SWxrR7Do9ouxGKD+ih668A1bOfpwVjtlzxql2LAwueo4xrSCUnnHdSwdPurtGKd53uVMbYlKeA+BvmGHHUgPRmAoBhTQ9lGVrkKSI6kDnU12tlP+XoTm1aKQp46XqDI8NZEwwHruFO2Dqi29UPyDJb4AMnWPOBdo3snTIP6haG5fYXR+wHloRcZ5LGJwJd9OAHoQgWug9PrK8shGZNcGP/Decm5Sylftr/iOYMUNXHfDQBI56AHx1wiEeE5CakOv22F8TaMEi65UMUy4nXQH4glIligCwkGEmzXRoWxpJRkg3MSPd30dtM3p+nTLgiTUhylhT7nqkONQ0aUXtl3IXXixjlb+5y5xRxemYPsJ58P722c40sk45jp48rKuZyAv5Vz8Zs/1fdGOWCSmu85fT6SyymFlUT+5nz1okodOByufiNZfq+UK/Kh6gVHDh/9YSZWuVJ+5eQzryRNaUvJI9LZXg+YR+moyu+Sjy2obqYh7f8fIm9X21z/Uu+Beg98W/bAt0Qs7JEj5OJ4F1UmVCfQ+NROsJW0ZfJ2si0TbkzGTsx8nkUsCrkoUgsbUchESCyqxAKpBRILA24JniUkWWKhPjcvZsC8q3/ZqFqjz8PfRQUpu5utVJb6294gFgCALQx/t4lhEcbbkheATyEWRWKhelMQCvIuJEP3q1lSkklFGHEXYgGIzIQikwvVTWpfXOCeUG+RWKg7vQEI3ACmoBCSdiEW0S4BDx2uRxiNqQXTGjKrimR7FgG6k4CW+wCdJQARHQO4aE3nAHTnAdkaZQvMJRZKACYBdrcgCgLGTsD8KODD1eQxALBgxMIEcer2TwKSp1gNfgr4HqK8qwDOi4B7db6ts3XRTsD0XqfhqC5kBfRvTs2k5ZmZ1IDtwgB1mTh3Nl08fz6iO+vWU6Nfr5NYuJrr+BCgqcuvKo/1BVtBLFT9Qq+cNuijX9LRSFoBvqBcj00GjVPysMBeadV5JBujtMXVYYmFRq+u6qqjr5eodgAp2mYQkV0C3mGnwfnHfN+E2DpeyZZ7of0OxrX0fz9lOj6Vrjymr5W6SSyGWZVWl904Dg/I4yF5GRnbeBDeH8sTBPcB6k8gkQgyzr0zOrmqXPdw/atqWz8AcgjwKrHzDpyjncPYcggoJRW3kLwYH2RAMA0Q7eX4FC6Ap+aROuCJ6gRqMUOsyl88TaRyyKQA0z6y/+xfwai2FuG9C3D9NYIdvnl/Jk3evZ8OAKuqye1DLNoB4EMA6cuoQglQ5UN60NLmQULTz+cmtgzvoEZ1mzo1oxrWZJ8B1jfnFtUPSycwfj9BAMVmXMqqljRGfS8AdDWqF55pW3MfIqAEZgEisAKBXydI4U5xUeuqP0bzTRiQ9/b2pm76RkmOpFFipzRrBGA9Rh+N0WcGR9Ql8iQuZ7WNospZYsE9MSbHCe7lOdz+jkIs+4nDMYtkY5b6PmRcb0Iu9mifNhb7kIrESv8BeSuxaOQ6Y9D0AN67uRfWYQd7jm0CEW7SZ7rIleCqnmb8ExcXWslDiZTfG/Hj3EwHamexR1vXIT47lBdxMMgrAvnRN8a1ML3SpTbGVwN5b6uWx7O+7znqo+QBcWra4voIjqkkA4nFKVTFHH/OB84lElPVoLRHUYUw3EAzfoMo0y9l83mqbtTfPvMhy/Qjn6kcrSQ7ckUcK3O9P8p1JVVRM41MTWAZfI5slcS182G1Il5Sznsw6njsejM7nueRAo7+8OpSP88cye1YPqVOkaZy7l31r2Rfm09t/p6uE4tKJ9V39R6o98CHtgf+VIiFrXdirZ0ky0Rb2zOeL8ePTMCVRB6LPHwpVD7vl1gUiUUmGKgoASo2AP5KK9Tn9ndIOgB/gjWNaZVYhFoBwKSFF7GqUMXOIlYTfcECxKuSFipn/X2B7UEsJC3Z3Sy63pSzCwCyHF9eh8RC97VIKsg7SyskGofEIo7z2zTF1awrsKpVxMfy7B/+uPedJTCRVLBWyYcoxJCKnQbUP7iGCocuuQBXIG60aQEH2BugQ57sF/ktAbiHWo8g3L7WwHoC0HXWFU/arP69uuYrXH8fIHYLiYIrxp0cGwHcaIehxELPMt60VaQi6v/fBnQ9ABRtsFp6ibxUgzKQnh5qXHUPtSe+P4HQ6Jr2JADU9ujO9MaDh2nt4cPUjFH1SUDgtfPn0nMXJtI48SWaAdnaYugyVaCklygNs1X9Uc1LQOQKuWpH66QR0Es+BOfZw1OWmKiKJUhXbWqacfEAMrQGmj4FiJYwufqvKpRShCVAWTf5nQIwCzKVckmClHgYPVppkFILgXiQHvpAEAukQy0JQ1qIgivNi0FUtlGzAbDSbyfpa+ONaMAdkbC5XluQE9w/pUqSAL1naRgfboMpU7Wr+xCL6wQEXKSPlUQYzDAMtinDfhyEHEr09Dh1F9D9DippGuqq/ibof7j4JE0+XEx3kDw0AGh7AZlnhgYj2rrGu5JGV71t/xPqPEI9jeKt0bTE4rV7U+nmrTtExV4OFab9jq7UdvpUOsXnPBIHbSOU4Fh+tt8wLkpn2DNM4h73BuQiYjhoEA1Y33k4j83Cbmrr7kkdRB1vRXIh6R5iTI1DWiV7ksMtVIM0vr8N0H4AiH4CQN/AZewu5EI3tfus+CcD6EEqTpCHNgYSLGN6aEOxYluw2TlLey/g9vgx19+nLjfnFiIKd9iF0UdueqVq43MWYjGONGcESdAipGKWMflgaSltICnZZnwrKdlHhWkfAnTAXnW4kITyHHfRX7rdldDu69yBoHfaWmxx35RibTEOGnzmaWcb5aqWJilWOuSz18zCSKK9S3ivCiJD3qEGRXptUELaKbEg/w76Zk/SQj8Yt2WH8nUg4NgzJohSVKWDTSHRQSpBP9mnRjr3WbCfTvHcG61e0isRdJxrjxL/nDAqm5I5qh6TkH12eKakiJOVH4dn45o46rye59CcUUwdfK2kqF7CnGdCj1dOVb9U6hPzcLmscqwkdcY8cn0lX3f5eM6tWlwuyZPV95PVNb93pYkTpSR/HG65Tvlc/I02HJ732/Era/P3fJ1Y2Av1rd4D9R74MPfAN0UsSoOdSP24xffKCUF17QRa0jhxl+3wW3m5VM5U0vgCqJIKwIrf45jf+UgSisTC70VqYZRtjbclGBuoEGhfIchVkhDEgmutcahCAeRaXDH0RcunSiwkAgBrpRa+7Es73ftCdXVNreeqxAJVqB3Ayo4rjQCl8oIKAkMZxd1slVwAKixLgNBE+SHRQIUiIm5Tbom6nQ22c1/a9mg/dQ9iQSsITQah0LBXUuEqOKua0fdG3gZg8B2sG6vqtlkwYZ2WOPgQYH0HoCOw1b2p6idKLEYAp0p1XNG0jwT/ruzfAVAJkFWZcdX3MkBNqYabEbpVu9KWQ2nEI9IbEOxFQNw1wJmejjRG9n5ZJ6NHK7nw431d5pwB4yZZ2V7GcNvVbD0VXRsfTS8SVfoyqjbaFLhJFvR0JUDTbsKVdesviAtDXfpPr1FLSCeUbihd0l+/Rqq95LHDavkSbZ8B1E9ST71ePSJtB3n30d5ByIluRG2Tdhi69RS0Gzgu+o90uvNV4qDalWpR8xIlytoClAus4TLpFPf2DORI4KaNwgyShg3AqOpKAwBPy7ijcTXptYs5jdcsJSSqoahOpMqOxEKgJ3lSvcv4FW/OLBCkby0C4j03coqVdVboISUa69oHGuA+ljDZNvrU/pKcGKPkEWB+iqjo79x/EPr+rYzDXuwjJBdnkV6MU1/tNzTmn8aj1EWA/Vnu4WliW7z2YCG9irTijRvvpD08Yhm34QBi0Ts6koZHhtPF0VOQOiKvU8/bqFsp+dB97Rnq57P5lOfjHqvqyqk0Zt5FKrUPsNeWoKENw+XhoXQCMN+tgTHgWWI1AnkyerfAU89fk1yv+tqCxPXBLCpzC4inKAtgj25XahoYwAgaQ2jK9ZnaZR7Qc9YG6mMnGYe2UemZxvkz2H3chMhuoeJktHqCcADEWXSgv3SRPIIUZlT1OPpgFWKwCJmaB+hvQG7Mc5sxuEtZ+4DxhF1FXMt1Lg4orTkhYUDqpNc4Vcd2KUd7DsergRp5+OPZN3idxEI1OiemTsaNcppW5q7HT5+G/ZP3xAB5Sjxd+FAa4Ufy0ObcR77rEAsJF6bh4fbWsdBkgEH6XsPvIDG0sQfJhWSTYc18mAmtfXypQiwMfFlcQZOEKw8JRHXOpj6q8+WNPb9jix1/PMQBRnJNEo9Egppd/h05Hf7JWUWepWxHQOVaM69utd+rBytfKldY15KsUs/IKY6XE9UqxRxbPcqF1e/kWmpQbW9NkeUd5xxdtvK9nPN4zOMlAfvoXwqJdwvX1m0sajqn/rXeA/Ue+FD2wJ9ILL721o10AzBRttoJ0mP+jk/+UZ2Eq5NxZUY/nNhN6MSbUxxOwR6vbEyuHo8JmO+FUPhbQlH7CXIRBCO7nNUTlDrM4XqWvcRCUqEq1B5EY4+XuvlZugBbIFRiWJQ4Fr64s/RAUsHLGbBX2uheaYX1C2JRcTe7vZklFjuUtUe5FJHLACw0F2IhUQEMhFtZQEVWhcrkoqoKhRpFuJuVXAh2yAWcU+0D6+6H9UwIBW5DUX3aa4QINAFMICYCJKBEXGPdA6yQh0Qk2sw5a6/3JoHwNCo6kgFd554BXAwDyHQna4yDIF7UwxV37QoeqN8PgNGmwdXvc6jPqKOtBEIAb6A+Va9UMVoG2Ort5wqg9ALSCr1NCfZtjxIgQad1cvVfr0OujmuTYbC7BbwOrQLCNJm+hEH1CwR/uwaxEMzbI9bHFVjLlVgofTBv2yeAtn47tEfpiTYfruJ2ca12BNpcCLgsW+B9O8gFhryUz8BJnZCOQYBeGwBsj2skBPpZOsl9GwdgKklQXakHIG897DvVxGa43v5UfUQi2sank3GjzYpkRtWlO4DtBaQ+JxgTXYJYxtIcxKIX1aBhpBjnALz2h21SfeckZEhpwQgkTkPWZdpitPDXiGYu6HfsvnB2OF3Cc5USFVXClDDpzSoie1MnyZN3XiCpqtgcoHgSu4ZbU7NpEyDqQFEffwxQP0E+2qPoqvkR9bwPAbkMoTiPK95R1KVen83E4rXrt9IugDeez86edPrMaDo3NpyeQ5XJZ+oJUp4bRFH3nurVqB8pQSs3bou+NRaIUitX8HchkPtz87BeXAR3dKaesdHUM3gy4lc0cG87yUvnABPcew3gH5HvPVbmZxmDjyAm6zMP0tb8fNpHrUtbj5aTXIuL4g5JBvdOu5BtCOoObd5lbPWcOpXGkKpcwLB6k3lh9tGj9A42PZIE1a+0XTDwnfewHbJ/GrI1BLFQPWgDdcenSCwecY0qWFu0zTG409WN1AKSaDA7xrQLAi4USLg7eY7a+EgsjHFjZO1NVKmUsm34bHN/lRq5qBHG+daBOktEtWNCuSot8Swsc61jXNsJyVLYRpDWse5Y7uCeNyGZ0J5iib5ZhSQrEemwfPpfMuyc4bwjgVGyp7cr6+v8aV0lFhPcJ22NdO3see+lz5Sf8je++of8HL+1W/zisBnHLg4cTROFxsmadHyl5UfyM/tnbiU7JmHHn9uxangkjquTF0n4U03D97jKEyZ7RjnlUORSvTBneeRvZF45Upuu5viRvCpJPVbq7qHad6jH68Si0lH1Xb0H6j3woe2Bb0Asbqabk+hU10yWToRutRNi5UDNS6hMqSVdpHBGrXxxl/OJvGsmZn9HKveVjy9Gv9eSivI9pBW8ICUTEgtVokItSolFqEGpClUhFqRzdduSBe6C5zDYZpWwEItMKnL07VBP4AUebbXG1DMTiyyxCLUrvELtADz0DLXLy13JiCv+JI6Xs16eQlJR2Rc7i2K0rdSiWWlFSEmypCQkFtTPmlpfV/oLqYj4FeS9i6RiT6lEIwAFl5UCGlfBPY9b/yifVop2+JF722/2rgBWEvAUlSHtDUxuLARXyHsAK0abFljo1cf6R9wAAKC68xwOT056XbIfNwA1K5EPYJ96qqpkeu/PeBfAuNhh0CUhAULdJuxcBER8rMsT8p5FOqCqz4wr0gCkHfIZg7xcYaX5eQCv5MDy7Fq9WqkWpQqSEgjJyQ5sRZUoVTgkDtpBzKDqtbCG1yCuUwKgh6pR8tQrzgL36h3I0p0gTKjKUW4z924AoKWr0SbA1ip1UEFGInGKa1xBNxig3rW8H6pC3QW8h80I4E/SYVTyWPGlnpK1PspVjekdAPAURMJgb6rgac+wQZkTgNfLQwPpIpG3NSjXle0sdRqm384i6VEdSFN8CYfk69W7D9Jt7CSMQ/Hc2VGibp+KmB1UM+6NdkJ6YlJtTWNp759gVTuRGYjJbQjDdYjFKqpABxCYVlbbzxAr5MLwqXQNt7tGfn6MLcR97B+s2wR1O0OgwjceLqTXJqfSq9dvpl3yUZrWNHAyTYyPpMuoMT0PEHczKN9NPEfdIw/d4qrzr/G7xujeK3whhRvUfdSK9ucXiEGxh5QCNZzxM6kP6YkqO48hmHsQw14A7nOU3UU/LkMG7gLMDS6oh6bVWQMUzqc9Ayr2D6ROiEPfyBBB+k5EIDrtMtYgAttIGVSXOnF6CLJwOp2lntpEzS8upslpCBbppLxKEIziHcSQvj81MpIGGXsGldtiHllZXkpPuWadiN2qNGnFE8QC2w7cUNEqKAl938hzqMqhqlBtSP+ytzjsOCAWOxAd1bO2WEA4oIwG1Zq4ptlnl/GgJyufOd3AOsY0vF4JsoDSo8SCcz4FDaTz+XTRQhsYAzp2MPY3SCu5WKbvWqiTjgLCgD7UPxmXpOujTNUFeQhDYiGxkJiOIykaRJqhlEmCav5u+e/Rb9a1vAciUeUPt5h+4I+fuLBcnfde5xZZV77n9CW5hCGSPPtPyZM0OV0Naai9ohRLoiN1rfyOpJX2HS+wtvjjbSzZen1pS25MTeGVBhzJp/Z07bV8j3cLdTFv86wTi5rOqn+t90C9Bz6UPfBNEYsywcbLp0zQNL92Ii5pao/ZQ4fJD1edYhKumZD97SR7/FPIxPH9LuA24lhUiUX2dLTuah+kQjuLXSUWgmgAqdN4vJgBR0ooitQipBWAQ1/YWT0pSxxMa71tS5VY8GLO7mZZvaxILMLeArBmvW1/lCFhIM8sqcjfC6kox7LEIpML7StyDIsMXqvEgnoLqgVBu6TZh4zoA/+gCY8yQSwwYKV+Ghi7mir4lmDoZcfKCxbaaK+YwnT8hFBkYK9RtWBYY1NBkeTEzTaYVhCv3YLGsPwMoNgrICITgYH5bALsBbSCcW0g7AG98LjKr4qP5Mj+CNernBXA2VYcWoWdh/EhZpCgzAH0NC5/CsjuBFwpRXlB70yAJFdgBenLnLcs6+bKrUastsF+J+uoo+pJ98lzGmIhUeohzRWIxfOswiu5eEwedwHw2oRodL4GqUE/KIyueyEfHdgXmKekxI9KX3qw8tMPQLOHdOOpVye9ZdnnkojoD+vCeY3bjfPxFMB6BxuDCEYImdgF+DUBultpwwtE235+fCidR01H1aXrAPI7fMKjFe1+8SSGwPS1bVMa9LW7Mxi5P0yzSEAunRlJVyAFZ7G12IbYqPblSrkETlUxY2Q4YFWbUY/+MeD2LnYWX4ecPAb875BfJ+o8VyAHV8dOpytIJ3Y59hjicA9Dbz1P6ZHrHG5334LMvHZvOr1x83bapQ1tgNDB0bH04lmIxdBJgGmugxILo5O/w2r7NJIPY4Y00U966jlg7O3xLAig96lLE2l6uX/Dp5AkIPXoQr1ONaF3kI4sk4ek48Vzo+kkUi+eqnSDPtTwWrfGSiu2Afl7qDQ1owLVwadncBBiQRA97o8kdw3pwiaqUjsQiyY9WGGrM0g/72PvsILh9jzEJtxZ07fh0pp0SixUpxo8O556UYfq5H6vOY8sP00bSiywAdpm3ITEENuQA4jFPqv8uJFjoAD7WSgwQKDuZtu5dpe5Zx9SpK3FHvlvQ4p8nva1CQmpBepNgH2fNuc1nRyoGhf3kflsg/5cYyzreUpPUCoERswKVLEY1sT2gDRCOlRhUnKxgrTvKfddct0BSeuhP3yuXVTQzkp7I72oScCzhCSryenqWIcA3ZIdxo/POf+PbflIzG8mqGyO9dh8/vKXyt7dYbp8sEIGIml+TuK498DUlXyzGmjliiP5ml9tPXKa8rfMvdVivZY8vcJzfmKrqX+5Nvbl/JGDh/XycDUPvx9LFz9LHu9VRrmmpCu/2ddtLGo6o/613gP1HvhQ9sA3IBZHVaFsoRN/eVXkebP8qpyr/DTV8fP5+vjrn9iqEzOTbEz8HPVYmbzjWOWcL95MDip2Fvwux4r6kw0qHqLWkSQcEgtUflQZ4BrrJoBQXegoscgr9EottLEIguFLWRBsY3KDAsAJ43Q3u4OBeJFYqAqlnQWFRNuUiqhKdcQzlMQF4BBSDPWsBejsGy2T+hSDbcuy7UEsQBCqD5mrnz1JBaucDS24nmyGWPDdMrxGQCHIVd9flSftGax7O2BBwOPrW1IheHAF1Hw1PBb4a1dRQIc3oRCMars55vXeYlUsBCcSBOsdUhDArYa/IcEgP93JxuqtbSZ9BjestHKd6hZKhKxvjqi9HSvcjwGdTwCUuqTVTsOVWA3KjV7tCqzuapcoQyNppSuqOLnqugXQCnJB3TogJK6WK0mQXGzTSI2yL2BD8Dwg/Axgq6gx3SUa9zz5rXHfDJA2ACjtB6R1AbBsq5vEZJl8JCcGGBxGiiB1ingKkATHk25JNV5WWqN0QTU01af0tqTNie5WbwGCVWPaYOW9kTK7AaEfmRhLz40PpzOoorwDoXgTycYt0hibQte/Hzndl8Y4103eSplenZxJr92dTtfZj6Dacx5APjHMKrxjhTq54t0DuZD0qVKlBMn7pFG3akgPKOPrXDsNUdDmowfpzPOQm2tIBpRY+Hw8RepwHyNx02tcPAjBuQ+ov41NwvV37rKyT1/gwekMMUY+CvC/gEcovU8Z+VyjdwP03cUOZApyoG2JQdwOIPbxZJd+BXC3QrhOQb7OowZ1EXLSBtA21sXbGGYvUE90GYO0nIJYtJH/bSQZRvB+xL3aRXqwDzk4QNXpxOnTqRtS0YP7XEF6I31Hk2PFf5W+XuGzxzPWSv4nANsEwAgvSutIBE4g4WglvaB8nfyMB+EM0Y8EpQvVrHZW8jdQ3dpGFWrHSOvEWFGqs4v6k16x9pH4HJA33R/No9iYUyQVbYyzPdrJpICaFfMCBEWXsxuQWhhLGFlLRIyK7jOqCtg+55yTTiDtwMSf67ETU9rBfdWw3boaB0SJlWPe9vZRll7UurjnG5SxDIFZ5d6qYtXDGBrg2fH50/haIq5qnGTc51FJ4IBjmrpq16Q0L9zNOqptTGzOzX7JBzzu70ICPBPNz4lIFYk9fDRNHC7nyj5fXJ41y8jlVguPfMr7oDbPSnE5g0jlnzw/5apW8jhMGHNqJK00rpRSrU1N2mqWz/hS0v+J1x92YO6wSj7l2vhJebX9WCcWz+js+qF6D9R74EPVA3/KxCJeC9UXw7NeELWTaOmpmGiZYN1XiUTlZPU35wuJiD2gx305ZkOyGlSOYaFa1BrEQhuLIxILgKkF+SJvMu4DoLRILEItgRdv2EVU9oLfQiyi7rxJAtzzt0gsDJK348ufsnzpR2Ri6m/68LEP6A8yAfiMMo4Ri0aIQo62zaseMlJ5f9O2CrGgnRIApRXaURy0oIbUimpEK/EO+LQosaC+rv6brhCLNYCKRtUChdDHJo32CNbfuhk4zdewqkSqFal8pW98CYgvzLKaq4cfV1EFqxoDhzcmX4ikEbTaRyUfYz08Bhxqc6Bxs/YGgheBvSumlis4yl6nIBbUZ4O0rrArZQn7D66bBqA+WQfAkVcveQRwAiBpnGpaVYnOAL6GIR1KBfR8ZRvc+tuaw/haw/QgFtRZQKVq0iXUizSglgAoITBCuETGFXGJVT91VbdfdRRbZY6mnYIY7EAiJF6jAH37USNgCY7SAO0k9Ky0CDB9DKjzvp8DEAu4lVi8DWF4GzWkSSQBa4B0Izj3s8r/kYkz6SrAXl3+m6R5kzTvoN7TQpnDAM4XkCI8f2YojbLSLiN8/d5MevXmZPr69dupl2NjEIuL42MR50OStkO9NPjWjaiez1YZj5JTV6q9308gOa/en013UW3SbqOXleoXIBXPIZXQZSxFhAH2NMDe4H3r5Ocz8pTxPUc8inuoRvJApT69LJ2fgFiMpfPYQVie+anOpQvde0g1ZiAXM+SzD5DXSPsAVSjVhhS9NbCC3864G8EF6rULE+kieUiypynzbSQNs3i0UuVojH45CUHr56PhtpKteYjN/mI23G7gWM/oaOqFZHX3IY2g3+17iat2VQZ/fELdN7YxcqYtgnevaUCKINAe4Dq9SLWix7OEMfgqZW9QhxOD2GsgsWgj9oPqTHqxOnj6OK0g5XD87+gJCi9UCaIS3tgY97qc1Vg6FiXIswVQb0wKVh9SE3Ux8J3RsNfpo7DpgJw0cF8kFj5LjdRtj3MSCImJHs0aKHsXYoTsKdJpmK772S3G4ibEQtezXUiP+hmDStIkIqsYuS8zrnXRrIeqYca8RML2+rzm55lnkJvtuNBr2CD1Dc9q3GvTWJ+Y73gGuIzNPx7Nxz0Wv0hLwjxvm6aS9vD6uCT+5Hz8aqJIyD5f69HjZeRjHI0LD9NbZL7OfTle9nEmzluv2CoFR4pSCc5VzuY0/D1yvnK05FqbthwzybuOl/zj5NHyPVR7baVjPRx9bTt/6rM/Fr//ffxxkS7f41z+8TI9Z53Ke3aLseXnJPZM9a3eA/UeqPfAe/XANyAW2cai9uI8VQoj+VYmbhJICzzmeSek/CkTVmWCrc3IayqTsJNtmcQ8Fp9y3omtZnIr3w2MVyY8J0gNs5VIaLytAeqmagS8lDXU9KN0wSi2pnN2t6a+QPWME3YWvKD1DOUqvivw2R6gsg+wkttkE1Q5CYkFgb6yxALjbQ1Sg1gAMKiPm20Kg07z5CWffddDLpROAKKaKNt9IzrX2SuMa4iBu2gzIAOgsSvYAIAJ6INUKJkgUnFTu95vOsE1mVhYjvUK0kP7DCbnCrcqIQLmMOSkHaoqrbo6yzltMiQLrQBQAWXpe20FJBhBGtirStNJPflaVYmyPmUTzEaPkolkRGnAAgDpATYOAu8m0nYBkuxfDdhdURb0Gn9BuYCSByNw6yLXehkfYgZCoRqNKlHWK/TL6UdL1WB1EPLwPGD+FMbPSmM0IA/VJSpp9GsNuJcB1bMAUfPEuj3sDIyboYG65GRyaTVNA8D0VKU3JUFWGHpzvfVx5dX+XKUud1jBNyr3U9pk3AtXnh1Pev+5Cii/hIGzBGMOUCr4td3jkg3IkPfvbaQQbyAluPVgPq1CIFQPGsIL0kvnz4R9g7rwt1FTuv5wMd0C9Ktu1EO9JliFf+XyWVSFhkJ68NbkdHrt5t305s07odpz5sxoeuHSBDr9uHqlzXOqwZC3ZNIAehJopVNBLOl743NcRyKgutES/duP2o/E4hI2HsazcDzYNw9pw10IkLYv0bcA43VW7Z9g8GzUaMfdwNBQeuncmXQFick53ApLtp6Qv+pKt5Au2F8P2O9DKvaVWJBveBPivjSgjtRB3UaQCrxw9XK6Sv/5LExCRq5zX3RZrCcxV+G9Lz2SSq5fot8XiAtxsLCYGjEiJ5R4asN+oh2JRTuqTgdKALg/klejaO/zjG4yH7iCrw3CDiB9nzo1kJflaZdhfIdWyliGPKwTKVv1KZ/RFqQy7bTLUdfI872PN6wV1K+0S9qj/ft4ooo4FpQneTBwXiN11taggb5vMr4FkbsbSa/am64VnCM2HTssBmjAfYCrXIM/ajehKh4PeqhItUD4OhEzRLlIZdYZg5IRo3M3MJ5UY9vmWdb2ooNjugQe4BmTyKzTt09R11I9q5Pzusz1/mvn4pKBDg50Vas0UecCkl8DGvZQH59zToc0I+Zkxj+NiGeQ2xVb3vmXT+WYz2XMCKR1i7OVc/6uHPYrm2mcTz1eiAtH/sT0cWHMqflb/pvzJT+/kEFNFrXJct08UqmIbXMr7534wR/rU91Ic+R3OVGTpuQTV9Ucj6SV+hzPo6T1eLm+7OsSi9LJ9X29B+o98GHtgW+KWAi48sbkXJmgmZI5lF8WTu9lonSfk7x7yq9OsHFlzsHJNj4cq36vHCtEQnIRZEJy4Use4OjvsLUIYoGbSV7SEgrdz26qrgTRsLHZHaw1dbVdg8ZsuN0GscjSBAAJL+QsaSjfJQiH9dfdrCB+R2KhZyjUDw6JBZF1BQmcVxWqEItGXvqZTEgyANGSCz6CINWflFRIzYK0AMQlTnpn8mPQNledD0jbiKSi6QSuT/l04O5TgKcqldcLgu1TP9pXbNFPqjnpPlPbBomC9gCqCa3QF1SuqiLltQImyYheabopSzLgSr/Gv+EXnzS6iNXGwXRx/zhme+0ey1AVR4mJEbLvo2ZksDlVZNxcMde9Zhv5eQ+5aamPss6xGq16kuNGKYfqMA8pQ4ISBIiS3HtO3/saWI8DND+CRGAQQiBQcvVWVSDbqLRBYqZrTz0yGXnc+AyuNEuY1FNXuqVKlXUdREXK1VoDh0l4etgrBaFqMVBctZ8G7F7H29HtxaW0BHjfQM1HMNkPAHzp3GhIFYbxguUq/wy2BUuATN23GqBPEGfQujeQRryJ29ZVDJfbqNsYxspKLC4M4dGI8mbJcxLS8Q7kYgl1FsF4NwBUO4MJXMt2sKr8JmpMb925l2buT7NyOJAunD2TXr56IdRdvCGqXS1QB1XSoLgAbI17s7qL+3XqrGTkjsSCduht6wLE4CzSiiGIhfdZL2Eaf09SnymAvpGvBcRbAO9tVvUbAMaS4hOsXF4+M5au4BHqKmpUjlFtX7x/BkmcQeowhR2FblaV5MHEos8Ey40YQ7eQx0kkA+cuXwrvVqpdWe6UbaAMiaHOELRvUX2HJwBQvok34uXUhMSiAcDNA572BgZTAwbczX39YRDdxP1znLUw9qXbzleSC70yraOOto9UIjxAaWiN0fcJyFsHKk8+c9vUbYP7s8O49floRhrQRuC6RupysLqc1iAljpldSQHEQ7BPQyAErP7SRw1IrBhcTC4816RhQDEmSUKfGvxuz4UO8t6H6KpCBaOxGCQoEhNUxriuEbLXgjQL32SpCenDvn3I+T3yM4K2xML4FM4LDdTBYIe9fPqYRlQ720BCJGk0rw4I5xDEIxvQUy16w4j1Eo0u5h+N+nVFnIkF/cF4yMQiz9vxjB/+oQAfiThw5JvHD4/6q3bLZ3zky1beC3mOz0cP3xOHZCOfObywvFdKPrGvZGyqw1n6MEVcXVu46XJhh4kq346kraSxju+1lfKOXPdeiWuO16avrUudWNR0Uv1rvQfqPfCh7IFvglgIf8tWIRAecPJlIvZf3uc05QVS+VUujH2Zrp24IwuOVslE7XfOl+Oh+sSLNQiFL1jASqzoSxoAgFliUUMsIBQR16ICUmxwjr7tC99VdIApIKRNIMKLvqyMN4Iqs+RCkpHbmd8zwhvqCUjYDWJB/hhv70oueKkLoIJYRJsogxe1dhYSCL+7Wh/fJRYQiiKpEP5IR2xLtIe27ABQwn0ue1Wc9DjTjGvOtm6AUGcvQBNyAbFoIK99Kpf70302ypZUaCgagF+QRDtcUVZVaQVQorqIq5YaV2+QbpFyjLTdDfA4SV2N6RAeauijIBbcJMG9we1UdYo6cUzJg4G9BC85YF6OFG0AOKNFzwOYjHptP+gOVVuJDeog+TtFGc8ZnRs9fqUzy6zCCkyNLQGUi/5poj4G8ltA5167hj7yGEOd6SqA6SSSC+ufx5mr8w1RV9XItBFQgiIp8XqB7hPA9Kp69PRv2A9ADC6zWm+9NXAneYDYCCJGucVzlQTpdYjB6xCDO0odAKiNtKsfMP7S+fH0/PmxdAZXpnp00i3sY/bniE2hZycNa+9BNrSfeO0hqlCQhg7qdo42v3x2FGkH6jj0wxPye8C1t0k3x171o136egLQP4aHonba+/a9B+kuUoNVVuyNtXAZYvExiMUQHriUSmnfcQ8SM0V5utS1X4yrIHlz5d/AgTfI/z7E4inAvIXfw7hpHaKM06g3aRMCjg6SOQMhkFhMIXnYoM57EIt9CIHgWTLc3NufJpCYXJRY0Pbw+kRfG/tEA+t56j/F6v8T2rLB9wOONSJtaAKgN3GcBoXK0eD5iTR26mR4f1IdTvKqkb7uafVytMV4ELYGteA5O5CUYO/QQJ4UlbbwTrXb0wfQR7rAfWyivS2swishaSE/I7O7FLADSNer0/4jXN0yHvZ9BiEjGn6fgOR1oFrHwIBcIKUBzGtL4dMuEW40qB+A3UjWxovRxsL4FRpVM4lQPZ5g6hnEQnJBxQ7Iv4E8m1BRyjZJEAvmox1I5348PDz/jF0lGge0VRWzPYhMEypWrcSVaIOEYMWOpGQJQo19GM+/we9Mv09ZzncJO49ODOx7Sd/HfTtASqs3Ke2WLL89SAOG5BIWetHxrDqfARl1CqB0b7BCLCL+hvMVY8a0zncxp+SJhSNu/ijztbO22yERqAXJ5VykOJJHBvaRvw8cm7vDa/N8X37XAvtyLC6q/UMGpYhSq6Ony9l89L3yiVSVOkUHVDKprUNtvpFPtQ2HZZj+mWXkRkcWx9P4u64KVdu79e/1Hqj3wIexB94XscgvmMqkya5CH2rn3Wh7TKQkVv3CVG4182jldz7+XhO1x/3E6nvluxdWj/PC9oWaCUbe2wjVPzyubUVIKVwZjA+/fZnzInb1UyNv32K8zmN1WmKRyYXBpzSO1B4CElAjtQjDShpi+xogFOXfdlUVSgBo5O1svL1PfUgW6Yvko0gsNNIWTASpAKCoGrKH5x5JhepFgmvb4oq69d4GKEowJA7NqE20d0ImevFW092P/WdPSCwkKqxdRp/l9VkMd8nDaNiutqtTLYiQHCzRBwaPE7xJGgx4J/EI9SPAiIHzjCFwCuAxzGqmQFMXlRoaczikHbMAoPuoBKl6pBrSSKSThBDlGZUK77D5C24XyFMbBtVpaGoQGVdJn3D9I0hHFyu6lyEWE6z2a2hsOqNae70ZWXdBkBKSp9TbqNfqihsQ7gTl9wLIVeXoY+VYdSJXYm2n40Oj6yBr1Md4FjchBjem59IDPCKZxziA/gr2DS8hDdDQ3H73HjQDRC03DFwlG5X2CMjfgFi8fX8mPcXeYA+g2s44uTQOOYBcGM/BYIL3AfYr7C8jUblMgLhh9vfpCyNPv4k3J4lNB+VPoCr1UbwxXdb7EPfCiNBKO+4AxDWAXoDMaJ8xCJgUADo2p1CTmoNU7AM4R1hpvzQ2EsTiDMRB9S3r+oB+vQeJuol7W68XSNrHGqUrUXgVlawHD+bwjIRHJcjdSQLc9aOS1It0ZYD7rXqMqmHGL5mGWNwm9sUT+syV/IMNjLGpp9K2lv6T6TzE5jyuasexOzGwngbjSo66kCgpAZsEFM8Y+BASs88z0ox3pSbq3kjfHfQNplbUmLrGz4S9icbb1l/je5/lJcp5iDrWsqSE3w2A62aIRcsqqkpILRBLAcQ70y7G23tdPeEprZlnqZH6N9NWAfwB10iUBNT7SCM0wD4ghsaeUgPdVJ3oSi3YZrQj/emG6LUzljR0XoUAafS9CrHRo1Ujc0moJTEGjXFiuSQmhgXEgrHic9xIuib6dx8Slm1KIBfcY+vDLYh8bIfjjMEZ9h5NjNNm8ja9apq7xMZo4r62U5dW6nuAutc2JKdILEO8wTNsnBhVIw0M2IWtTR9OCfodq/TZCn1roEImsrDV0Djb9jPk2FdIBeNNtb9+7rPE4iSEpocxJqE+JBZe4x0pW+33cizPzfmXYJo6xeY8nuf/eJA5VvJyHvGY+9rsPRwlcNB/zwTmXsoWeZGuNk2pXWRvolzQ0UI8/h6b15tv9fpKusi3klecq630e+VVkybX61jtKueP1J9jdWJxtEPfuo+XuUcuLORR1NHWmC4OE5NoQLlzfav3QL0H/jz2wPsiFnmmZWKsTIbA4SMTemlYTJJMwP5zq500S5qyPzbNlsMxsTu5ez4+fq98qqpQvIzjOy9y96EyBICwMa7wF7sKV8QF5/7WReMO50wjqLQNzagjRDRbQHNWiZJc5E/YWAAaNcS0HZlcUKnKW0dVm6wKBfjHeFtioZGnXqGMSmxf2Q+ZWFQkFuYVn7LC7ks4u4iNWAwQgYjBIZmAAEko/G10Zu0AWtpxg9rtCuVg6uzpxzCTVWxWTRshRPaVi6ASMqdhV/znIQlKJ7oocwjAZAwGQbvndLca0Z1pezdAfBmwMwMQmwWQqPs+wLEhASZAQ1ArGXALN66sxk9BDCxTe4zTpBOQ9iE96AekKCVQzcg4ChICiZM2D97HMJCmHk+QPswColVfMzL3GQCYOvQaIIMXo73lJa8UQumLkocIfEYb9Gi0DIB3RbqP/MZZ0R9RNQqVKqUn3ibVrbLUQtsQVvIB9W9io/AOcRxUFxkfxhPSxHh6Bc9MAiqlHBIx77dEw7ZLuszLVfg7GHC/DbB/8+5UWsQ7kG5MtdG5iJ3BxfNn0wTxHBYgBK7wL/I5R7uuDPYRZG4g7A7eASS/xXHBtypZRjt/5fTJdBHQLyEyCJxxLO7xeUD7jMVhtPN2+gMFmNTMavoT6rCmKhAkdpx8r+By9mVIzRBAWpJonxlvRAnJdcjFAvfKsq70d4Wq0yb9/VXaMH3/QagF9dK4kXPjqQ9ioUSpl/GgrYhEQTUyJShvzy2mqdn59Bh3q0oLVLdpggS0D2E4jlvW88SGGIPYPKRMVcDsd+/FHuz6DqD4LtcvoH60D1hvXnqSmtmrNrQ/OJw6hkfSAF6hBpGWSJy00wh7HsbLOs/TJHYnj+hnFwaUGrTwrLWto4ZGX+3T3m2eh4Q61QHPgYPfvupkDHaiGrSK/cUafai9lQsCDZCShA0FkefC+5KxYA6wV1KdSNWjE6qCOX7oywMWDXT1uoSUxkjhiTIlQ0woSA4IItmNq1k/lKXEUPopeFcy0UB9NVjftS/IE5EHaThB25wVDkgHUwgS0koft9gWnjudDwSxQPrQzvPQgq2E5NWPY1JJq3PLPnMMs1hIMBLSjS4MzPsYawOoQunGV69QGtI3UW4nEjmDI3q9z5akQRKrsX0X7ejnuVUV6iRjUWJh5G/HfKEHeRbmAahsZFGz5R+OubzROvJ/9/asY7WpuL5kEYf9wTX5f23CfJbyIrnlPqO8XFp5C5n1kcyr+VXzqB7J1aitbb6ypiKlrWXPtbVtPuyLSpNKundnGqXWXuuBuipUdEv1z9/79fm0uIzK7AlVNA/SygbvE56DVy50pJ/+FA4ffJY+wPaF11cjr1cuMl/Ut3oP1Hvgz6QH3iexyODel6fTtZNh3jN5VufsfDy/DGoe9pqJ36PV5DXNKanLEB69OAAAQABJREFURO/k7Pfa34KVWmJhGn8XtSEbkm0sKhILgIuAopAM1RokFqpOWelMLJBa+CLlI0DMHqIAkwDx8AzFC1ZiEfYVtjnanetlDSUW2+FyFpemxrKAwFSJBXWzj8KjlC9q8wG4hTtZgS952ZuCaFf+1ZdWOqHaR1Z/yiTI9tn2A0hOK6ukHa5Q9p/C4BRigUrUCdRJ1Hcnw5h4JRYoUEASdtI9AOYDQEsf5U8AmM6xIr1KWRKIBwD7XoDZCGBC0uF1C5IB+knD337OneSjhyi9DEkunNjD4w+2EwsALOd0SYvkQsmBIEXAsg5BekI+6vlrsyCQMc/SFt2hhpEvAHqRFWHtG3Q7qjqJ0gbdx+qFKhODLEXQXkI1nk3qoAHyE0Ds9MPFcA+qOtclAsVdRfJwFoIRxqn0tWRQI2qJmdcuUp7Gz2/fuZ8WMWAeZJX6IsTiI5cmUj/lC7qeUm+lHBILXdlad8mQbnvvo19/i9X+N7FxWCCewRYqJx2A0vPnzqYrYStxKgy9H0J4pgHEgwDUs5CGS8Mnw1vSPQDfTaUZ1MX7P8hq90vU9wIgWHUu1a3muF+z3J9pALHkQqmFgewaIYJN1G0LwKq6Syvj+BySAiUuL0IuTtKHRvWW8BpE7h736DrShnmNzSnrCsBzvPdEeI36GtKKe5AjXaiOImG6dPkiwekGAPLbIaXRu9Vz2F3oreshebxNm2/ianZWV6uPFoNYNAPkO0bPhCrWhdMSi47w5GTEbcnBBO1yvE/SZsnYA9SP9pBWtADsmwTqgOj908Ope3g4jY2OcD0qfoxxvVg55gKH00/29yzSjhX6o3EPF6p4k2pFahLSR+xPtgH3jagz6R1Jd8EavA+gMnYKQmc8h8dIHeZV4bLPIRYh6aB8hkTaxV0z4aiROiBlox/0ztahNyZIUkc7UgnmizWC2m0gNdjn0wghaeCe7FHWLs/hfj/G4sSxMDCec1gTkhYDJDZ5jyh3C2KjahOTC3qMPOv0i1IEPWO5uKEaWhvtbeEZkIQECcfbVBNkpJV6NFtPP9xv5yEaHXYYu9ynXcjMrsH5aL8BHcMrVCMSHNKvcM9WyK+F8dcNYdJmyE2XxPardhSqxUmcffaU+A3SfqNu++w5/8VCCtcY9T1vtjDPy3lirhz2GPnm7VnEwuuOgu/Ip1zCuXx9frdUSojy48Jn/CnlxZ77fXwrR5x/3UqetenKu6X2WE6bj8T8HO0q9S/15Hy1vYdXl/Q1zYqT1jFyoJ7uoy4kMl3OOfdNub5OLA771G//6+fmYyz81z9xKk5sbu+nf/KHSwTr3Eg/9l296VPPIzn8ANv/9GtzaWywJf2Nzwx8gKvqSes98OegB3bx1HjvXnqwsMS7gldJx0AaGT+Thk8i3Y6VIBZcn86nB1PTaX5ZOz4wQf9QGp84y/wO3tzDscfiJPhnMW014Enw3HPp+ZE/G4L9DYjFjRx5O6bBPEE6AeZ/gWXp7Tw9VqbP+G2a6ubEWvkd+9pzlUTxgiBddbKvSROg2nN8CrFQTaL8Fqz624bssKIZNha8rJVQbEIuNgD7kotQhwIohBoCINEa1hpvK7GImBYAW1+sfsLGAnCkHURpQ1SZi63rLh5fJBbbgB2lFga/ysQCEEMZvoCUWHht7AFOrlyaH6eCVKh6E8H9AI4SC6Usxa5iH/AVCe0P6iOxaJdYnJRYYPCr1AJwoQpX5EmrlGwosZgBfN9GJeYu4KaLY5cgFVcAsOuUO8kq9juAj0YIwDCgYgIgM4Au+Dp9aUwHJR8GLjuJCpJB9XohAgIRjZ8N0rYIaYGeZSNW6ihFcrVcoGJblVZIKlx51pWlAN0V+Wir4Ir0j6ifq/N6AfKIEoNeytOI208v4E1JhWBIac4jiJCeiiRKTylf0PhkbiFiDzTRZ+fw7nMN16cX8HCkL35tFkIliuuVWujp6DHXXZ9+mN4mgvQ0blNPQCaGUQO6evlCrJi7Uqvhs9INv59DPUv//qoGKSkJtSBUgq7fvZ8WKFvbnb5Tp9NFgP3l0SHUnoijwHXaj8xBQhxrGm4PAvy8XtWicG9L2yVK3ttr6Mef7+2MQIIauUtsNK6fAqCrVjXHfos67aHiIqiNeAi0twWweRrSchb7iGvjuHxFHeYk6kfmOUX628vr6Q0kFrp/VWJxEbB8FvCvq9LXsdGYujcTeWq/8crVi2kAdaAF7U8A8N3crxchaQyDUKW6R1veQfVren6BPn/ICvpaRIvuGKLdTGwanytxmhfIA5LXqZ9RuzWgVm3uDa8lRsTWylKoQTVxzEEmMO8bxHAcydEFPDNJKCQQ2orYbxpN30XKNEN5C7R/i+d4D3KgncYu/WIEkT3cLTdAJJTaOQ77sUMYAkyPnx6IwHKLrPZPUfY6e13GhrSE+yKZ2EV9qpFrCXARhs77RJeUDLXwLBjgTi9OAvUtDMX3kdY0EMVbl7JKLHZQSTzgWUTEke0eGLvaT2mn08C4QNyRdrl/B4xrV1WdbxzL2kiQeTyvGnS3QxYauV87jOt15gANvhshO61I/7S5OKAvJSTm3YTEqin6gDmC9ury1rYbm0Ni39OAVJBr1lGhMq8W1QORWPhs+Vx6P7XvDhsL8pNwKM1Q4iex8LsB8kJiUalzzD9c4+b1sdGOyC8OVI9GGw9/lcRlTx/YCWwx3+dvld9xNL6XP+Zf3SivbLncw9/leO3e/I9cz8nDMnPK2hxqSjpsYzXDZ9S7pj7VZJUvtfmWc7X1sazjdbGupV11YlF6Le+PEwuPrm3up//xVx+iEtWW/taPHLq+9bbMP8Uj4ZJONZrT8EB+9rxmi0ixj5Z30y/9q0fpdF9z+kuQEreh/sM0caD+p94Dfy57APxy8w/Tv/ni63h+ZIGItaLNTWxhr3xH+uQnXsZdO14Tn06l61/94/Tlt9FGQBrfyLtlr2kgnfvOz6TPfOdIal2ZTdc//xvpX0/zCmpCO6bvk+mv/ScfS/0NeODEy2QavwRWy4t632oXvE9iYTF5ynRiLKtZvknyKzNXI8P1yjEny3y4+jcm+/LCqJn8nWjjQ8pyTZmgg1gAyjwfxKLsOVaIRujGF2LBXlKhv+3iGco4FtleIUfkNi/gfhCLNsBvqEHxkg01KFYCw1VlqEFlW4gisYiGWP9KJfckFpCZLYkFZUgwNODWSFXWYLL4xDUSFK/NH3mHK+naAWwBVkJSASDd5hPqT+Sh6oNbeJeS+OARph1jzY4BiAX66V1BLHA7CxgI4kJphWAskOddgM0kHzXvzwEcLgMuDyAKet0xwrMuKU8COi4AbM+juqIdxyOuW4Fw6OO+B6DXAcjrAjQJTqyvUgsJwh5p9TD1BOAq/9FAVpsEAbPuWXVBa56jAB9XRAU0XhsqX1wwD5DS7sCgcMuARKUVpyE/VwHIVwDMkhGlBvag8TkEyDMA1xuA3FmiaS/yfR2Qv4m9gQHPhllxv4BK0lnsHDq5ViJkvTPZyVIhJQU3ZhfTm/em0+TkPQzJW9Iw+vnPXblIrAR04SlL1Z95ALnkbpRV/gusvBujwnunatck9gI3We1/wgq82+j5c0gkTqULkArVgbS5sJ+WAihCwPguSYUShMG8Ub6VTDwhryeUM4ra0RhqQAPeG/JzfRjNsVBhUzqkO1yNn9dJu8/YgkGHS9EmVtKNMj2EZ6Yr2Dm8iDvaMfTyHTIzkMm7oQpl3AfGI/0+0tkWHrQEnndnZtMCfactwSX666OocfUBQKdQ1XpI/1rfFyAWXRBK1Wes7yQqTfexTZiem09bpLGy7aghXTwzms5heK1L04dIJ7wva7R9GHBvUDvgPDYj3DPObVDnBkhCAySBgRCgvBc1njNIPM5BeiWxTyEmrqq3c/+6WbVXPewxdX7MM/1EwKyKEZKiPUE74Nk+MwJ9I/dSNcMeJARDqBKNQJh0v6sB/DR2Hsan2EONqwly4dYASW/Ek1QHKl8NPD9GqF+HvDrfNALSmwH3LdyNxi3czM7PpwOIxb4qVIyRPYPjSSiwz9ArlEHu6DTKz1KyMPTmGWAQ8ezyEuBZMG6JxGIfO6mD7u7wBqUtVzttbaI/dmj3Jm21DZIb741SFj3eOf71dBUezSAXqk3tME9t01bJg1KSE6TpVLqAJEsVSp9Rg+e10Q6lhaGSyRjWI5TETWNunw0ljDouMLhjvwQFcqFnt5iqoqf8Yy/XbNWf1S+czPP94REqFJtze/7m1He4HaZ0fFaLqKbJX6o/qwlyDnENX48TCM/GudrCyP+wtHz9N/u3Wu6xDGrzL3WuPVbthGPX+bO2Dd+OxMI5yfH0zWzPIhbm8wu/MpuG+1vSz352MLJ9srqX/u/ffZLuL2wzdzTy7t9nTm9Of/37+yEYLelLN9eRdKAGeWz7Oa4fP4WdVH2r98Cf6x5YTH/wy/9n+uLuR9IP/NCn00fHU7r3e59Lv/XWQbrwiR9KP/Id/Wn2S/8qff6P7qSDFz+bfvx7JlLHyu305c//m/T7M6Ppx3/+R9PY4pvp8//w36WTf/tn0if230y/9vf+KF3+H34+fcfqq+mffu7tdOkv/+X0AgRFm8RvdfsmiAUvECZt54nQW45vuRoHIqI4ntPko5W/XnNssudAXO1kHZNweQNVLvGYL3oJxAGf6vdyzD0v7UNioQQhx7GIYD4VaYWSi7Cz8OXOy9r8GqmrdhTqdec4FkVigSoAx/T9HxIL9sUrlPWNOkfjBcqoLyGx2AiJRbax0IBbYiEpiN6wTVxnW5QmeNTWqicqqdgGiGdioeQDIEodc0wO20xqrlGNqpmXv8SiFdWHNm0sIBbdPRicAnB0lWuMjCBL1heAtkzeM66QAwoFtxpiT7BSqhqE6jY3APT3HhO1GBDjivlLeEayL55CKua4TpeaKnh08kZwhVMw0gIo0YhbQKIUQOCrF6RNRRy2kGueurIMsJGYvMJK9lkAo8RCiYFgh/+R1lgPd4nKrEH0HIB2CwA4TNrnUcF5Hp19JQVKHFRPkiyFvj9g/HXS3wY0q8+/A9jdhlg0ABb7e3rSGVbPRwHK7bTVOms7YtyKWLHlYVHV5BYuY9+aeZjeQepwArCmN6IXLp0P9RkrNwOIv6deP+BcycclAKpqPV20YRHgeY/yb6EKpctYvYlduoqr1NMn0zjAWlUwx43j2ajIJfCfbddGw4CFq3wMUmebtVuQvJ2GuAyRxwlWqyNmCP27BUh8wlhQojP7aCk9QoVqi74VbGo83YCdhSp8gwDz8AyFnchZyI3PiGRCj1D3uTcP+WiwHRIAQK7xMRbnFtIafamB9eXx0fQyEhfJmN6oJonX4bN25RReorgfAnz7zYjX9yEIdyFWqwB1A7m1AG7PDxO8jz46BTma5n48ov9WybsHUtAGWOXhChW7xzwXa+Rh3IkGiSREoYFlly5cq57GVmSEPpB8PqG9tsG+7eWYVglKyjTan+faJUjFtqpGlKEHpvCm5MgPcI9thTYFkIVByGkD/bjGfLAAmVlDlWmXejdoI8IYbIIUtEHKevEGJWjfpI1KwSLIIuNGg+swxkYS2Tw/h4taPEmRzx7BKY227QfUHqpNjk8s/vPCgeCeMrWn0HuTdjHaw6iG5TO1h01H6oKQoELXRP+0AuybkHDsQyx2aBeTWWrkGgP5BYl3zuC51NNVzFWMa13l7jFuJDgugjRRZguEAspCT7BxjfYyGtgbvNLH03FpWh0hhESI3z5f4XaWftI26iTzg4beOi6IxaM8gZljdWMIVbZCGPIcRaHx7xDCx8WRtgrGDw9xPGdUzuWfHKPueSt7cy7bsWs4XN4p5lP7/TAfSuLcB90sM66qubY2l8M65ZyjDOeqmoJq078XsSh1Lpd9OxKLL83sp2uDeAxkOvig27OIxRTk4Rd/azF999XO9JP/QW/cp1/8zQXewwfpZ35wIJ3sbU63ZrbSr/8Rzil4Z/2dn8C5A8/eyvo+1y0gpWhJP/VJpI3crL4untsPWql6+noP/PvugYOp9Pm//w/Smz3fnz7zA9+dnhtCm+Drv50+98dLqfelT6fPfk9fmv3yl3BOgyv+T/5AevE0FdzFg+VXfi/9i99ZTR/7+Z9OV1ZeTb/9S9fT1f/ur6WPbN1Kv/m//Ebq/dm/nUa/8A/Svx3+6fQ3P3UG/PGnwCosmve1GLyT93vthgrwwcHX3rqRbty9F5N9efic9/0ev+N7fIt583CSZIKtviBqsj12LF9p9rw8mcBrJ3K/xzGOF0JRJBTV4wAgv6sKlVf5WfEvxIKX7yaAv5AK93qL0s5C4GTZQPBMKnhxC2SKOpSAyRexn1jp4+UcEoFovEDX9tF52FhsFWKBxELj7T0+emyxjNKmqD/luRotsPZ3iVGRiYUqXEpaAAy0RbJk9G47xL5xFbMF0JmJBTEs+lCDglj0oA7VLUBSfYI2CAZyvfGiREHzrOhOQxL0ECWAHQM8DgJQjEMhYL35aDmkGcZb+BiqI92Aik1AiqpSM4BT7SRaIWCunrrSqfG3nozGkEJos2AQPEHsEnU32vQTgN6aK9KAnD7q89HxYUB5T+olXyNSr9MvbqcBnV4/DTl4ffZRmpojbgOgVJWji5CK51h9vwjYVOrgaisK6SEJUZXoLcDhHa5zJX+HNmyjZtPASvgAq8BnkViModrkJa7aqlk+gdRkAADdBqASPE4D6O/MP05vTz0MAjIOKH5pfAxVMDznkF7j4/uCfoiXUodB1MTOIAk4wyq47ZwCWL99627aAqBav0tXL6fLkKFRQbjjhcK9z9zi2AtUdZtq3ZewAVBNbRlC9ogy7qOS1MXvUdSArlw4G0bHSn1cmdbFqVIO2zmJN6kZiIVkRymcMQx0taodkPUzjsQrkIMx6iiJ1K5EMqBr4IeAeI3ANeJfhWysIjlYM5YDefRxHy5jo3ENNa4WxsV1iMV9bBqM4aAx+BBEpZv8HX9P4n6jngWwX6bu2wD7RlS6RpEUnULtSePhB0gFHmlcDrmIQJNIyZrpl03apI2KsWQOANCEng7ja8Fzuyo4SKl6uU9uS9zTLeotmTb6tKv6dGqA7CXarNvaHcpGvEAekBTBuI8K6RtYcW9GxaldAo0ECNaCLUJTkAvjzOguNzF+lCS0Qgo6kVichFjqmnaTZ2SesauDANUO/TSwcKDqUeNjpGLYo+xKaJB0aFcRthWSGe45wy2iXMczb3qIn2LoZp6lRvpNCc0+xGKH+7UvMZGQQMQaqV8z7W9B6qnqlBHBNfpmoop5QPKgS9x9n22eoZCicJ9YGamoabLgQONVvfJ++qwobTAQZSvpnLOcQ5Q6OS/4DLgwIJnIUos8xznm+sh/CBJ+Sqko54PQ5xk57kuei/JX/zq3le2Zc31+Q5Qkx/Zem4kJs1t8N0Hk6U+Pxd5d5UuUl8ssZT+73JJPJWeui6u83onBrdS9/M5Hn/235vpSnvlVcor5udTneAa1x3PNrUKlRWVfuaik/XYkFlMM6dMMeRw6feBNYuHt+ps/fJKFuANstXbSP//KMvZg++nv/Pjp1N/dlG5Ob6Vf/teP0l//DwfSCxMUVNl+52sr6fOvrqSf/VHULU9nqUTdxqL0Tn3/4eqBvXTv938l/dYbB2nsygvp0khDuvfV/zfd3hlL3/np70ufuJhV+w7bxPt26V56/Q++kL5wZyT9+H/xmTT85E760uf+ZVp44fvS80130+99fim98uPd6au/3Zb+4n/1F9L5NhaUDjP4lr59A2JxPd1AF92tOiH6PY7kL9WJP36WMxx91qTNseoEW5OPacvEatZ+N13s+S4Q93shFiHBKL95eWcbC+M+ZGKRyUXFcBtQlwlGNuo2jde78hZxLHhxC+JUJ1JyUbxC+SLO0bfzPl7SFRBhB9g8JRaqWK0rsdgCOFSIRUT4BqAoceBvXnmkvq5AunISH8BTkCHAqvvsASqTimifAIdN942uiLbw4m9FOtEKgG6DWJzg0wWx6CKWhcSiibrn9qjnjToO1z4FyM4BojTKBpvh6SnHf1ByYGyJW7gjte8G0Of+GED+FEDONms0fAdpwm3A9TpgRyDbxiqqqjZXWVm/wCqvgNlV5JVt41pkMG5wt3UIQiP9K1C5dnYkTQBOBwB6Skm0vaDodBEJSSvlaIfxdVST7rByP4ckoY17abyGC6jhXEXyMABQ1KhZWwuVXrz+bUD2bcDnA8iPRqw7i0gskB6chBlfuzgR6lCqeazRBiUn/ejL65+/FXUR3dlqA/CAdt2ce5Q6IAZ6k3oZIuOKrVIWV5W1swjJxRI6+dSzn345C2g3YJ/6+m/fupM2KVflpXMT59LVs0gL6JcudfXp9xivlUGyTb89oq+N5yHAV3KxAbHTc8/C7FwYyNufr1y7GOOQgQ6xJEgh9dZgVzUkbTtm+KiKZV8bkXqX/rSfT3LvvScvYbx+GhJghGkJksEI22m3tiwac6v+Nss9XaDcVdp1AJDtoW/PUvYE7dc9613IywNidKzRn908Jz24PT0B6WkCAG/RN6uU95SxsA5xkNQdMDYkdD1ICDqREDzm3ks6Nvg44PQc1YIkCSMAXLQ2hXrcPm0wHgRiDaJfE0uFe3CC69tMQ703qOcO7bP7jEmhpE7JAw9BOGPYpQ8DfFP/IBa6vyWfCDJJ/zOA87MAYEd0kfZRF9rlHh7sIEmEFMFcQuJjUEntlfogsoJ7CZmG3pKyICnUlxmIOgL6ibidkBAdULd9DaZ5DuNDWc4DjdyzRtp6wHygBOUAb05KKZrpwwb6i4c7njPVpvYpFytvbC8AOvS5kbO1tWiFkDTTL3vcnz3GOeYeERBvl2f2wHIYCxIL6+qY9PmhV0jEM0UZ+4xNuH8sjoTEjn51MKpK5bPjYonj3SCQSiqUYJmH18QYZ6wMc58NpqfULNSnzJ8tZmPysq1xY8z4XZsnK8fdRVoTVb8cnvcbfeYW74+4NP+u5BDnjv7hXhw9UEqrHq09X/L3ZPU7DYiiqlcc/VJNd/RwtdzaupWyynvuWdc+61hkXamHz/qR/iHTb0discaD3cHj9M2AFonF/JJvs8NtfLA1/Uff3VNVYdLT07+AbPzAy7hibyt3Bmck2FuoAvVXv7c/fbTiBapOLA77sf7tw9QDOKCZfy39zq9/Ho2LZVRhWR9rGEkvfs/3ph/4xFUWa+NtUGkQGHOLxdHX/zh98cs3UsMrP53+s08M8w5cSQs3vpD+6T97Lc039qbxj3009V3/vbTzvT+TPtrD+7SB98PgENobYslvrW/eF7HwhVIm0CiuMsOWF02pQzUNq9zV77X144IyOXtN7XW1k7DfTRd7vn9gYgHY3wIQZGkFUgtVogArEShPYsHLVmLhWmMrL9xiYxEqUQD0EiRPNYJMLlRzEERkApTbzc2DNkhidIspsfCzy+rkLseUSOyxwmKaUHuiHYIX41SEwbnnAQPegPBfH+lJTZqwrfClYydQmLrbzQCXNsBzu6pQfQT0UmpBPIsTEItW1Cv0bhWAAwBim6zrGuUbn0IPQ67WC5wnApg0RCC62wCpNc53cPxlVp01mnYV/il1uQPIfIv4BVMPF1Ejwb0pfTqOp53nxkfTZUC/tEcVC4G47lPfYaX7OipGTwHerqh2QybOsoo+AWEZBLBIZIxpoY3FCxCUU4Ak7/LrcxhDE6xthut1QaqtwRjXXDuDC1LyUJ3JFVZVO3QFa5A5o1hr+E1HpW1sHRpZhT4NSHv5yoX0/OXzeLBqC89R2m5I7LyH9ofubK2r5OL2wlOAFmJD6vIyBt+63BRsWc4qwN8I2rcgMcWQexjQLhGaZlX+5u3JtDH7ILVx30eQkDx/+SJuV8+kAdoZ99p7yNhSemF7lVhMYiMxTZ5KDkKiAEhdxWZhhP6+DDH4LoiFKndbjM81+qmX9pzgnkrgtC+RmM3wech3yYbSB2MfGLn8Miv+z5FHH2TkMdKcdt7gxvUYQVqk1ELJksEKjW3xgM8j7ruxFjroDyVYQ0g8mrn3RrtemJlLG3h+aiFNK6TAmAoJw27VcXap345kGpUhXaAeQDAMwNbOuGyjDkoTNjm3i2QAB0Wxat4OMWmk73YBq0qsdun7fQgJuljhbtVnTLC8z7ilq+KZECgH6OL+NKJyZGBIPS+p8njAsywpOkACIbFo3ISgmJ7tANWqmEfIU7uJfSQSB0gGIgI2bVWFDJEaH6QtAnPUkU5gH2P5ej9ao16qJ9pWlv159mgEz/IB0opEexP9Hm5tWdUPYkF/RIwLntVGxqqaoBpb65pWz1B+jFLugkJE2UYFSpIU2fLwHDDeDpCW+Fy2Uk4r7bLvdnkmtyFSe/SrhuLZPS3jmLHSRF1V4XP9tQXS6sKGc4gSTslAO33WwT3Vq5oSlFi8IK1zWwfPhV7OJA6O9TA055zXqQLlGB9hDKs6qDc3qkWD+LijDUfn9MqJ6iye08V9M71PAPkezvIlfc6rkvqZeTr1VQsuCSv7yDMnKFWrpijvEw/EOKjZeyyujTodno/jlXSHNfRo3mrz9EhJU47X9kl5X5nO46UOtd89V90q7bCPSj5/5bOfrZ7+s/7iolIpt+xryyz19p3kJ9SLmctOokL4QTbfE98sUJFY7JLBT3+qn8UhjFV7WDDJLnCqVfjNLy2lP3hrDbfXLEQcK8jb/aMf7wk7Cy+oE4tqt9W/fJh6YO9R+nf/1y+nP9i6kj71me9LHxtvSLe/8Ln0L683pkuf/KH0wx8fCQ0Nm7S/s5oeXv/j9HtffDMtj39/+quffSnpVv5wA1NvPU33fvfX0q8ufyL9ZMcfpH/8FWwnd5rTmU//p+mnPjWBBse3piL4PojFvaiPk8wzJ5/KiyXOkzJPRnwpM3DlmJnEocrEbjuPtNUEla1M0HnlN78EPFYmOL+H69LKhOcE6UdphB/Vk3zhhjcoXvTZkLt4hsoGkb4A7TpX8rKbWaNvY8fACzmIBXsjU/u9qEGVJvnStDUahxpvYpPV0C3czW6rRw3Y1BA7q2ZRL9KEsTMAfBcgYL0lHao7FUJh0CzJznFSYSmhdmU91KfWnoIV4vYgFhie4mazvaMHaYvEQi9KFW86FXLBayOtUd4M4MbVdkGv6lB6fVHX/zYASwNs12afB9AbV0GgqRG3HphuP15ON4j5MP9wnhXqlXSK1f1LuFUVRDcBTAq50B/PDOo1N3BjOotb0m0ArxKgEYjFOCvCg4BL3dMa40F7gyuUJYlx5fRtyngHcjEFYaATUy91GCGo3HmMqjUeVr2oF6DTQ70dMG9ACEJlBwDrfdlDDaplbSUN8123sS/xceXVtuqWVjKg2tWWQI52GrRNGwfVodpAeBpcv3xmKFb7BWwCXQ3QdfPqCv4yeUgO9Fi0xFjSNmJ6eiZtTk+lVsDxKF6ZPvLCtXTl4nleejnI3yaSEZ8DjaCts2RGw/MpwPoMkhaNyHXtqjRogPt2jtX+V86OUDvwKy9uvTNpxxIBywB8Rk9XOmTskQfcy1BvWgagA1xPcB9H6bMJbCK0A9Eg3CBtSowujgyGXY/32vgjs7RJgjKPK9pNAPYB39tJ28v9aWFlHwWriEytalknK+ENHN8FDK+hXrRDW/Zoi6vyu4ybfVf/IRZhaExZLRDeRl/q1LWBa8M4GCCrUfgm93mFfvf+b1F/3eW6st9Kv6ou1MyzuI3HrjBUhu7rAakBgGxUa42TlVpI/nUIEOqMFdUiOjA1GRPDD8892cTYVSoQZIIYDwc8MxHEjrKUHGAhjeoc6amnNhLNg8TvoJwm1Br3JAHcN6UcB4wFR4ySDsRLDBzIDP2PiCUbbJOGAQhJsL0sVLDSzwNIB6E+xX1ppA8bkXJEhG+lFJCRFkhOPDeUvcW4sqwGytH4ug31QYmFsVHCsx3Afh9iF8bh9gf3SbDfzLwk0W6lvqpbOZ7zem5WWdQhwQmJRSvX0GYXL2yq/SexMPJ9SCucU6h/zIPUQzuoUe6V5EJCG56huJ38d6qLjWTRx/lX5aA//BoJ85mjf31veF1OX/am8ZDn/BzfSjr3z3rvmP74udpsyvVH8q0pqPa815XfpVW1eR3J4xv8KPmYrNS79lhcXumL6AA7jnrFe5OO/HYMkPcnDo9v0J/PsrE4fsnvvrGa/vmXl9N/8xOnw+PT8fO1v+vEorY36t8/ND3w5A/TP/rfv5LaP/0j6fu+63I6zSslLXwl/cY/eTU9Hns5/eBnP57gGrx+UEm//eX0+7/7x2my82PpJ37q+9K5rAVYberB3mZanv7D9Kv/z2z65H9+Mf3bX/hq+vjf/Vvp49ufT7/495+mH/y5z6bnTrFgXb3ig3/5UyAWzIs1k2OuAi9kXxoeZyv7/KMyZVcm1zL5xjn+OAk5EVc/Nb8DeHOuEIxMLiQVgnXViTKg16BaYiEI0UVltq+o2FiQzustCFgQ4E+jyIhl4Qu1QixUQxEYqlKSiYX1qqkfsFpiEWVRhoa1W0gsNgEumwDJcBkLQJXwRH252HoGCfKF73F+l0+2ycj1sofsBzf7JyJ1A2haUcNow81lGzEsTuCqswOC0dJBpF6IRROecayroEFQ1047GlGJUqd/kfpILDQo7gdYuDKuLvk7rB4/BuAYGO4idghnAajGtFB33/QPAL43UVWawdh3mZX6bsDTOPr45yEMqhf5QnQ1Vm8+j0l7H3IxjXqPUZZB9mkY0H2W9MOo2xi3Yg7A/BRQP9bVnkax1egEMN5CPcfAdQ/RzW8AcA5ShzMQi8tcGypd9FMrHTLAcVXWbpD2OmpTd1VpARzt07YW7vUpSNvzSFJehPichwCpa+7qs6pDGl0/pXxVQgTpeh66jfqV3qQGAXXPY2dgkDe9CQmUtUF5DAiXXElKJClKH8xjHuA3j8H42uws8QdWqetAevnapXQVX9F9rsyT1nJjRbwy1DV0V8pg4DqNqR8xLu2PxwBtwd1JVqEvR50bQ1pxH5LVRLs0nO/inEEcVZ/ZoW6Pyd/7OYdnrC3ylER0gg+NA+K93yTfdsblEP11hfgZrl4LYA2MqMRIyZH1MDK4AeC2+d0F4PTTg5SkHSDdxnOkQfAy5S4xFp7wpBxQT1fkHZlb9N8O91KphzJISWQnXqC6ANveoxYImyvmglgB7SJjaVHCBHA2NkSs6lOXFvpFNSDW7TmuIhTgGALtqv4B1+2x74VYdNIGV9FXuHaFfFbJJwzZ2RsNuxHSy8MXq/ZKBvZpi/YW4dqV50Y7BbIO2x8jUzcggVDKYXMaIETNkBjrIXhXcrKFBHCfMg3MrZclDc6VDjHJMGlUpBkQgtgY/0ogGgDlkhG6OrUwB0h29UC1zTN3wD1sgCS0cY9cKBDsr3OfosX2kc8s47eVuu1KLKjbJnnus4gQrnRpj/WzxGaOY6UR9x3f1sSsQU3QeyNR4LuLJB0Q8k4+pm+AUBvbpJk+aAsSo0QT1UquifaTn3xIiZ3SikIsVD/Mdha5mTEjkbZ2cz4sM1XkZePjiPtjiStn8jU5FbWLo/nPu9MfB+TH3xU1Fx9+PSyA7Cv5c6zkXvI4nndM7pVcStr4ybXVWpb8DkurfqteU1t+9ewzvhxJxxxP3tby25FYPKP17/vQ+yEWGmr/0u88Sj/6nT3p0y9kW633KuB//sfzqIg2pP/yL+W4GO+Vrn683gN/rnrgyZfSr/wfX00t3/P9QSyGmNz3Zr+UfuM3Xk9L4x9LP/wXP57GcBu7ev/r6Ytf/Eq63fhC+uxPfm86/64wFUgel2bTVz73K+nuR38u/cenfz/93f9tK/2NX/ixNNH8WvqH//3X0pX/9q+kjw31pXdd+gE65E8mFm9jY6HxNluZkMu+lOGkK3GI4068/I9Jm5k2jnugdiuTc83EWpunE3QhFUos3Mpv94VUlP2eUgBX7QBOhVgoRcjEQvsKCUUmFxF9W/AAOPNFYv0kD9ngUVLhR3WY/FFa4XnVY6LacRl14AUQkgjK3YHEZIkFQA1AsQE4jlgUgD/tJpRMlLrGHtAZK7O87FV5klCU+ti+2t6yfuojCFJ1p9nCyrFSi/YebjrEoh2C0YgqVAPEohFiIYjTfa4rmq5cCsyt+CrgZoX+cXXTqNRu65R9G3Ap2BT8nEWNxs8oAENSIgBXbecWQH4KlagF1ILMe0wvSaiPnBIskber+QIQVaoWAXqTGFM/xpXpNmBsGKB+gWjUZ4hToJG6wdZ0R9rTTqRuSEorgOnuCjYPSAaMrqyXn2Hcr55H5eolCEnkS/1UD+qhXm0AQFWKbkosICKu1OppCxiaegGWV1m1fwF7gauQBFW6fFFLBrTHeIiRudIbVcJWAMaTqG2pRtXF6vpVrnnp4rmoq8RiiWt0o7tC+1VfUzLjWJQMaISu+9c11Jgaaa8uaV9QikM7uwDYVCfumfrrZdx6/RpkZYm2KAmZI2/tWB7wWyNrA/qN/X/svdmPZtt5n7drnufquc/AQx6JoShPiCIhiGEYCAQhsGgjgWQHSJAb5ypBglwkf0SuchFAUmRYkSEkRiLHohxLimQzkmUSsixRNkVSJA/Nc/qcnquqq7rmufI8v7XXV7vqVDePCBNhU9/u3rX3t/ca37X2Wu9vvcOiflPU7wgG+x70PoZWo9B0DsA3C0M6wakdxC4gbou2WZXJZpVfEHvMOUnG45wa2S/RTrqxfQuAJlgY4rf1tq1XLQNgQkmV+2ZsQxcN+5fJ/84sroFhTAUGO6T/yPrSbs+pe5hM0veL3IR+e7TjMRIYDfsnqfcsQHEB6YZ7YGgXo7oNXHT2RvmAvqCkRJWwMOekrYqQTmKsu/YvZAGoQ1oIINAeww0hBTM3qPc8bTlJvdaI/wyar5O/O2prizHItx0DbtMm3TPtXDSOpm/ZGGcDpEN5vFeS4o7dZ9iQGEdwMRjg5rcCI015jsn3ADBybP/xu7fG5KXBt2W3z9vnohppoT35faqaE2pbSlrGsLly1+whpDoHvkelaRCVMiWiqlwprdinrDQjKl4w+bRPgAV1EFi4madg9BRVrQHiDvOtjEHoMdJS/WmIfmHZTxnnjqFxPEQ5TlEOQag2FgFjlJ8emX4ooAm48Bmn6k+WXeykI5BlQOFN8rluX6Pu0kMwbXcODbim8b1yWK0c3hAo4fKntKXv7P8eZlWO8rs+L898WRNrg7WXi+Ha0OeJXQzsr16h2lc1LM9rDnWuuZD25XgmdSm9VKGlh+9q/PSHNrsP5V+fd64JX/PLlZT9T9qm2QcWHWJx+1GAhTF+5tdWm4fPjpof/0szONPAmQn99xG/v/L+fvOf/pWFXh/8BQDINx8dNp/5sTkWx7Axw6h7esKvoH/0KfC9TIGV5gt/7xeb39v7RPMj/8GPNp++jQfLf/EbzT/52kDzsX//P2x+4t+72zQP/7D5f3/jnzZf3P5481f/459ofhgt5hzMEUM4DVGKjbpBs3nvC80vfmGh+dv/+V9kEfGPm1/473+nef2/+y+aHzn67ebnf/m0+Y/+9o83P7A83VOt+k6o8m2BxTda4+2auANg9/C3T/K3feXAO1CWN8vzNkI3bh2YfXX5eQZunhumhqv3Mucy472V/xZYxPiZyfqIFUaBRVSUmKizlwXX7DXBZKyKUtIkbctYwYMqBlVaEVUomJpcYXKKxILKEUcG01X6U1QezEupSIzDW1BR7C30PoX9BGcFFjHkllloz4AK0nEKy0VCcLj6mWv+yMBwSmMYh2E6xyjMiwbcgotRgcUUbjV1Yek7GJcJGKsJQIXgYBQmSeZAD1HaJ+j5RnUKGV2NiN+DYXoCc7oD83gHhuh1mNA7nO6+q1//dcr/Djr7ekjSY46Ow17D0PbjgIvbMCDSQcAST0SktwH930VasYLnpGMMeZeh3dtsXPf2nVtR69mCoVqDMXSlVi9WMkTvwvQ/hYl+pjEwDKPA4m0Awo++VuIIGPU2pWcbd93WW9V7gB29O+2R3ilMEiRqpgjz8YUZdpK81nwKJn+S+gg6VWUSVLjTuEbsMvgCi7WnbHAHOJg4PmBl/0bzI0gdPvWxuwFk2gK4wZiG10orrJ+uY8vGf7iOpQ9FpYfrFJ523O37Oqu9GvG6GZpgVCNZ2zpSNb4CXaYKvtwA7yHl+YDyvA+42Gz74yxt5Y7kMuqbAL5TGPdJ8lfN6Toek9yzQt1/V6h3yec5ZdTe4ilA5RlpjZPXEkzqXaQmevm6AcM8S9m0I3FWtf3dbO8Z+Sut2qLvqvKldOUmzHw2JiSeDKXG4W5MZ3jB2CBgQbVA+5LM8COA1fOWMZ8AjAss3LNiFmZeEKDqmuH3YKIDoGgbvWJZ/zO+TQHAMO2iFzAlX25G52Z3+6R/IKMcOuoOdbi5RT2mSeuM8q6Rp6Big/RsQ79lD+0cZPrdoVwwwweLhAdVIO7doE5JAh9ipBpxd6tnKBj/Acqkt6Uh6q9htPFk1D17EhrTRjIgI6/UQtUnP1jVjIahaX4T72SaPUSUMAAwxtiozg0MAywg2AnfyhCG5KrZKaFUVdI+5acuTeMJjPqNCCxodwHTge8FSDD7buI3Sd+IJJIBIpIV0jihXErIgBmUmY4DvYapq8BCt7LTfH+qGwqPMoYQRo9PMfw2vHTmorqhEi+9Qi1Cb9tUiQVUIKZn92h/MXaGXacSuh33MEn+9mLUsfv8XQnXfe67q44XhalzhdcLYWyX9ji/szTnR43r+/O4/iJUJ36NUdPpptF7d0X4+u6l15buJW1pezH1PrC4SL3/6bMrGd//m8+8XMKgl6h/hOvNP77HXKFRB4c2GT/0xniMt2uquqr9xX/yjDFOqXLT/Lc/eT2b5NX3/WufAt+zFNh9r/ndz/5K87k/eAetAxb0bv255q/8xI83f/kvfqyZH9lu3v38bza//tnfbL6ESeCFY+Z289Zf/lvN//CZH7jwOD+Y3zZ//+80/+Pf/1fN2vFy86P/2X/V/Cd/4Trz8sVx6cMRX/7kIwOLC9mUGSQp+7w3OCZQ/VWudXDuhWrj1oG9plsH/Z6UgnQNU8PV+wALnsdGgclX5lHm0/0fNKS2Ql49i/E2jH+ABSucTMQy/AITZr8M6Rr2BlwwKVdgUZ+ppqK0QmBRj9hIwFAcoeMsY6O0wg349lGD2hPIoM5wlEkfFRrK1bOdsMwyC7VOXk20EghCVFpUWoUmPodmrpDqyUZVkWFUokZhZIbZoGyIq5t9sWkAhrQYXsIkTcGQCCwsv4zLEcyHjIqG1qpAyAG4kvoewOIBzOmzg+PmGsuWd2EMX4PJkzG0WFswQd+CCb4Ps/OYc5j6vAbj+vbSbPMGK9QkEZUpJSECFZnde6zSKt3YQ1VpgQBv487zk6goCUiSJrSRwURxJVKT90j/wQYekgAkJzCL17LSPt/8GMDiFnnpFnMdMOIKuxIUGXI9Nj1BcrErncl3iHzH0bP/BLYKP4S9xKdfvxNVEGm9iwRAdSi9I60RftX6UpenMJd7qMSMw9B9Ynmx+UtIHT6J5EL9cstpuhpOxx0w6WvELeCwru6ibR00wj6l/7nKq/GrK95KcIyvkSwsPav/eETh3rnukPRUrXpKWdxZ+5vYOlgm0x3j3QxNM0O7yWzPcr8IE3gLeizA8KnGonRD18G6vVXq8AHSHoGF0ohrxLnNJnhvQLMF7t0JXVUY1aOOGDxWEx4AR/1tf3eAt3/YTwSJGr8LCmTa3STw6+zlYblODSOwoA5+B4LCNYCPjhAUlSoZU5LltyMgdbd1pQvW1b0r3kdaofG5diWWYwg6jJH/NGlpRyIIdi8HJWhKYuxztpvqbzOEmeVUDUg6a3y/y3e+CZO/RZrWW8Nmv40AB2ipvZINoJcojaNltkE5ASZw9LGViLQCesCBoHaF+hLAaBCaRUpC2bRNSiK0pS5n3XMj7m0FU5QldhWU0VB+X8bL3hZIFwQCSo1GiKOaFq71mxP6huBgiPoJSOwjR8SzHYw/DmM+Qt30JOU3cEI5HceO+NbdiE+3ubaBqmXjpC2AFVwIpCKlJT3zUX1Me5ERgUXaQ/eyLEgAlCBTxhHbW3shwYyHkgslTNcBFKohavSfscPwENI6+qHX8dmfvpHG/M9huPamXI3JI6tYQ/m7Hj63jV92vOi9Sday9FLoplUyNYMk798Ug+e58ttnSb8bL6G//Z9engb9DuKHKMSr6dQy+cCnP/XXfvLbF+LfUggX5yot67WbtM+kU10M+06Nt7tp/mnvXdjxcMHmoxwG32CzPIPPTrmw9uFYhnmOq6oJBpnLhuAfDt1/0qdAnwJ/Wgq8HFh89WvNN+693xuQLyReB3Aelm+3Hbj54V195rBQhoby/EIa7cDcHdRq+DqxdK9dxrwOdkUNSu9KrV0DjKIMv+ozBVgotSjqUJmEYcrKHhFMlmRWvT6pAqXUQvsCBzHBRt5xX8tnWfRME0kF+SUPGCwlFgdILMrAC1NA3qcwtDHKZhRLWVNXgVKlQL1pR77QTWJWyrX0ckLkUXbftkyoiwy6wgq40KvNEDYXA6pMsJ/FJNKLaa7TSjVSF1W4XGWm3JwytwILN/7TEPcDmDM30HvE1Z17b8JY3oXBdDduGQ71vB/ABN9HqqAXJldSb6Iu9TaelDxNy83XtqGpBsJ6k9Jz0RqgYgvmdJLn7lz9KbwOffI6xuYwkK6QK0WQ0dxAKvAt0n2fjfoewuifENc9L+4CEP7dO9eRQLgD91hUdmSMNT6+T1n12LTKqQTkGAZOFapxQMIPAHg+DSD5NHtCzMAoeQgQBABKZbQbeQRT7N4Qum/dArBoOOwO4Z++xs7deMaagBFzZdg+IHO8TxyNsZVS+FuAtgchldQobdgi/1noJZOs6llAE+8mia9HqznqvAwwsB2cJGUq3d38PmX+ysZO1KJUuRrk2TxluQlj+BYSG9WS3O9jeULGHekH79yBWhUtdy3/AJWZR0iTtnH36yZ27tdxF0N0vW3Be4b5lWlVAqHx+D0kPO9C4w3SsG/MCCRIf5G+Yj7ubyDDeg9bGt0GfwtwaMcbRgoi8++Kvi6NXclX8iUzOQWdtKVwQzWlYyPmZ1+F+TafpzDibpqne133R9F+QRCims48INa4BeiiKgZtQ1PoAPLHvTFAi/JolK1bXZKmLLpRFnycQHckLvRPwYPSPBl2Ge3YbxAvy5GkrzpT8apEuqQ1QBjEnTG4dmCyPoIPvU4Zh5x4yAvSC1gB0Gj7447hZ9hMSKMcFog6J2/yz34T1Euj8ahVmRdjQtzG0jcGeDdMfyFWbF5O2vxU2VANaphymc8pCxMatus+9lCVLIDFKUbfGrELwLTHUH3M3bwHiXNI+Bh707eOASJKYHRNLZidQfQ9S3ilR6pJ2R+kufYqXtM/8mwo/UZphUb/tmf6EASKRKbUOH24vYVE1qQcl4FFl0Q1TH3mb0nnUcf28uvi3xe9S1QS8JqWuCrhmhTv2tZK2btxLqffC9fG9XvthbHAplXzaiuQ9OqzmmdbLt/Vw3h1Dknl2zjmWcPVtPvAolKtf+1ToE+BV5UCLwUWX/wqwOK9e726ZRBsB9X6sDdg8sD3OQlThn4fOEDXSaDGOh9QfdIdeB1sc3YGbN/XswcumPi9l3F03wjBRFWBOuQ+TD9MX/UIJfMvsFC6kVVNJmKBhauw2lRE7ckJ1d9O3Jxh5rnWwzIcA1ycyPcDKAAVrcTiAKYgNhxM9hqiKlGhcKXc1IjsclwiX+hT05eWlZ4GDxULQQtjQNk0yB6A0XSPgCGYwhFAxaAAAyPUCXbinp5GdxRw4QpyTc9rUeFCPEwdPa23qkHvw6y/D7MtsygTch0m9Y4rnryXyVqBwVGdRXUiV7NnVbeBUf4hXJkuwizJsGwRxpV3PT+tw/CtIxHYhOk9hnG/g2Tj38GI+C+wz4JGypZlJ+EwhEb68DXCvYfR90MYWfc40Ph0Ge9KApFPsRv4G9zbxqohraK25a7gGlU/xXjZ/SG0a1FtaJy9Bn6QMv3w67eyk7bMq2uyqiWZp3yQ6kDuJ/EeeeoRS0mGne025XoLNSDVjrRBKYxX6QsCITds8xQ0CKRcyT8gnmpCpmFXlUmzDwlAjmn/cR7egnm/MzORDfZ8bz0EeKoYvUuZv7yO5AW6as8wAXP5OrR/C3r94I3FeKlaYJXaTc00uo99Bqv/jzgFRe8D3g4BONpl3MF70x0kFXO0m9+E4EX1F71qqY6nNOhreOx6AGjYgmbuHTGP/csCBte6Atb7lNKVPZhUvXM9BgyofqYBt1KY6O7L3BJ2FF186VmZUsHTJPWeoL/ETSrfnN/Bc9J6TlnXAXyn1O+E9yfEHdbAm/wmAMh+Y6bF/wJ+Kbff7Sk0HSQNGXGlBUovpgG046z6H5LOc/rYJnU/ciWe+tvGNsIxz2Mc7rdN4/vppN/TyEopZZKtT6QZMNuq4qkuJaDwNGykme13K8AJY01dBpAk6CK3N1YR/ow6CC4gRsYsK6L0JHYQjAfa4ByZpnlAp0gauOrKtgILaTvBmMLKBF6+SJ+8Ilmhjx3znety9gQVq0G8SpXdtwG+5OceGUOE3aNMWTChDx4CPjDGievgKcaHWYEFdYwhNuXSa5eez6bIX4mGeQvstMlapq8t0Tbz9NkAC+lkrSi/h+10fvis8zth+O0j69sG7MZpkzHAhbRC7zZ8vdT49Xf3mjTPE+u+spD5nWK0b2yvq47uc9Osv0sKV8Wwap26XR3kwtNa/5q2L7t5dQPXMH1g0aVK/75PgT4FXkUKvBRY/BHA4usdYHG5gg6S3YG4+9s3vYGViBne8+diHNOsg6oTg0F6v33J4W/PHqho7wMsYBar1OII40wn2QosYv/AhF1UobSvgMGBWVFioRFlBRYy2ZFQyKgwacs01Su3hC1lCGPYSkYCLEw70hBABcyQKlASVOPsTJgVTVArptPMe126SLyQpFSzR6/2J5dSDpkVmQCBTmgKszAouGCFchSGYxhQMYLNxbj7WsywqRkG3Xq5CgPVNlDqQIqqv8g4yIy5QvwYJv8+jO4KDJwVddO012QyiK/KhFKIR9TtoWFgNl3hX4KZ+xSMqWpRggulAY8Jcx+gsA5TvUXYHRjnHRjUeRjjN5E8/PnXbsAAI01B7UQ3sBpCPwWwfBMG/x4uZwUWZ6wK2xazMPifAFj88I2lSC1gywJeVOMxH6UOT5GiPON6CP01yB2GKVPy8EltLHCHqwGqq66qJLnKK920c3gfYOHO3eYrCJLxm6euSgquwYi7p4cqPaokTVN29czhSQFZqJPRrqojKQHZBgi4k/ZjGHUlNqqbyWCrgjTH6c7jyxipK3Fx52/beR+AsE3/1JD5A8uBOtcGdZLhFNS8AbiRpu7irXvdSE3oT+66/QggsQpAcPftNRjKZ6TlrtVzgDz1482boqeOStVUu5mwzqx+ryPd+ABD9U3o6/ehFGKKuNMAhXmY1knS0CZBycaK9hOE0+vSMHHdv0S1m6jyEHZEOwIY2Kzwk/4U/UXVJr0VaUQuY39KHOPvIbHYJ28QRwyeT6DxAH12gJX4SD9sF9pbhldgIxDXbiBqR+TtrtXD0Fkp0gz0GSa+EqP1XdKXqSetERhhJY0CgAAb+ocbOmqToWSCpmNPCECdgJy8lKroZU1QoQvdCix8Zxt5Gsczv0lXKUc2vqOtMxiYBumpAhWi85teUsYT7oapwxBtPIAqlP0i9hrWlfyUjoD6AzZsX2mnOp4eynSd7AaAZ/QvxzvBxym0Vh1qAKbf3bRVSdOIO+0CcNGrl+MdKTQH0AODFTzHTUYiZd9XqjQCfat6p5IMQYUSC/NX0qTdhe5mlawptSjAgnqTTz0o5qWj86ATDoL2wp2P4/XZeXqdQOW2E6/3zpvLabe/a9p1junFIUE3YR4AAEAASURBVB1zS06E7bVhJ/0at9eW9V0nrxqmm77PujWoterlfemmxq1pXXrd+9l93wcWPbL0b/oU6FPgFaXAtwUWSiy6g6n1dCB00KwDZx1gvZZhvazMGNYwNX4Zt2sI355PAL17wzuAt3nkefv7MrCoHqFiyM1kXKQJboTXgguYThl/V2IPYQ4KsGDShlnJYE66RSpRpBS5b+uVuqRiRdoQyQgMQIALjEMxCi/A4ggmKHYVTvAwMzVty16PSq9cpRL5eGTi61Gohi5Xw8hoe8oYlDjEs4wwDHqKGmeFcgxbCz1EjeB+dgxgMR5gAZNBGHXkKVASlFFQ4qF+tcxDPD/BFOv9x70uBAgCh3iHYsVzAebcVf4VGHDBxSNWbdVzH6cIH4cpfRu7iddhhE1HUKGXKV2L7suwc7+zsRGj4mUY609j+6AbWDe9U/VHm4kVJBDvEe799a3sD3EqsKCk0zDjHwdU/FALLARVGhortVBiIbBYAVjswLypAscf9NIPmuswRm+SxydvXI8+usDCVfxpgJCAyvqpBqXE4j3AjzuTCxR8F51/GG4NWRcIv0Tdb7hKzm/VwjSiV61Fbz2WRUN1vSwJygRJGhy7s/nrgAMlOuqsa9htfMuvRMg9NZ5Cw1XiqNqlhyhtcGbJS3ezt4krEJEhlOlz5/DHgIl3nqw27+Npa537XZjJA/rvMe0zh+rWDO2gTnzUtMgjRr+UTXe16vkLeHfp/5swrqeuokMvWGLAlrYT5AUoHQXI6KJ1H256C5oKCtxrwj0iBonnxoXo4cC0YqCO21Q9IGlYLMOsIbAseSQC0HOQso1aPsp5zGq6G+kNyDgTxr54jLvW7D5NeifUWxsH93ZwFV71nlOBIuAJRAQ4ElTAHFPOaexH/P73AGLPAa2n4xg1y2zTn1wU8BNy0cCyH3GeUP4z6KDS1iEb552Qn2DIthZEnPIdHPId+C3le+JZpA+U5ZRwR5aVuHTtUjfanI+7fEvEUQphn/DI904BHOlU29JWYlAaSoMKLKgH3hdE9uRTvuVIC7ifsA8DjA9RtVLdMPvemAfl0G5C9SoNzJU0aTMzAp00JrfPK6mNATflPCIPvVyNs9gwyzdnv/BbNx/VoZRUFlfUgonSx3zn/SIA7Rr9dklgQlgoasUDlkIjfkqq86P8cDR3kaZ3dAIVutQ3hqeQ7dF9V8fC+u5DV8evmq73HDX+VXF9Vt8rhapHwvLbJ76v1bE/9A7fX47TvqzPz1OkWL2IH76pZavxaoj6+6r3fWBRqdS/9inQp8CrSoFvCyx0N1uG4otVdEDNwNgO+PltkHbU9V0dOOvwm1f86Q7MvVTbdOqgW5/7u3vKXFSAETUHJtkTGJciKYCxgunPHhKCC+4PUFXaY7LOxloyGkziMnMyQqYr0245lVKwpljuk3kppWH04OLmXDKxgpO4r2ViVxIi43ZMPupEa2BqmZSE1Bknrin92ebhJHZOF2lh/dracpN7iGkYTwFFBRc8yDOnxKQjQwaDNTk710wirRhmw7wRVKGGUYUaxTWtjEwMUklfRkFG2/QEFTIZqsw8Z+V7FSDwBEZUlSbB2i2Y8Vusjspcy0yr/74OA6ONg6o/+9wv8061nTc5KUwY7HeQAjwELKhuJvMzCPM8jpH7LIbhb+FJ6g2kHNdYTXXVXvsKd4R+DybxAV6n1liV1yuUrjDV/f8ELm11l3qL9LOaTfk1Ptddqntv6F1JkOdO5koQ3B/DHYyzJwR5qUKkCsgc9XA1Vg852VGbcG4wJwDaUoJAfCUOh5xKsjQUvkX+b86hkoQdiRIIPTXZFjGCp17xbkXbK3ERkFVpxRswZpE4cF2AsRNQ2FcPYLiV9qjGdM9N9yj3EWW2bwmGbqFmpMqYuvNZJSY/y/OM1f53Hz9p7j1dbZ4i0XEDxlNoT7IwmjDbqECNASxk1Hegjd6ilKy4mdsATP0wKlOF6Vc1j1V3ng3gZGCI7yVuXqHNBPHduG0QoOB+CIJC+7OuWQdIJ7tVw6SH2QUUnBDuBGcB7kA9xL3uUu2+fh+CDFWi3FfjkM3hDlGD2se1qwyqHdu+HfZbIAAzeyojDMPvBngap1OtMMtuvifjLKBbBDxNAXwmoecONiI70GSXNnRzuwHOIQED8eJ1DdqkTwBOjgUWlEmW9xhgcYqUpICYAjC0iRA8lEKpIkXhZPxh4jXE3qN8B4APqIB00D7Ad0hGSpdgtyOBqe6iaWRiWzlP3tK+epBSpcvnsfNQwtIelnfYRQHqr9emMVxWnwraAWGH9BWlaKp8nUknwgkoRjnz3ZoG7aMdl99ZNvQjPJAngEj7iklBLd+wIMoyj1B2nSCM820JdANOSNv3fm9KL5bo8wEWgDXDGa9ILCyt/Z+TKzX88GG9Oa58lzfnjP6HQiXdkkeCtn8+/OT8rd/NRz0uAItOpKTQppPxmAqmfjyr6ed5N84V9azlNGyNV6PU+Jef19/1veHrsz6wqNTrX/sU6FPgVaXAy4HFV/4k+1hk0uzWsB2EMxQzspYhuQToDrSZjWq8uCdsp5VLE4ODfI13ecpwwL189oAFE7oSAoFFVYfKpnhMvDIYgoB4h+J3ARZMyDDQkXDARGaVlboIKhzkBQSWgykiBZWFcGJyAo8NB2kKVgJYYF7UB3eCd8OrABXK45E06p+aNjN1VWXq1jW0Nbu2nkmAFKKKxeQfYGEaMAE5uDcH1Tl02TqGxGIKicU0oGIUdaih6dlmaGIatRF04dnbQj1wyW1sgYXMnwyKK5WwtkgkzrJ5mt54VPWx6gIBXXyqFmRcV31ZU0043ZW68i6zraHwTVY6R8nDTds0BNcWwzpNUc5F8sBJaiQcS3g3UtVCl6QyTtopKIHQXmINQOKGbW74ZlyNke+gPnULhnKZ9Mdh8GSA9NCltEAApNRDF7B6a9JL0hr5HtIXZIr0QKXtQSQzlEEJhKpN9kclMNswb3pXCmDi3t2sVbNyfwFY5uYt8v3BxZnmLVSrpmCcBWW2rHltk6/1F9hoDL5NOtJTT07XBTGculHVUN0+Zd/RNuVd7EjeBRx8oG1ECw5003qTsi4DttRzt1/bnwWpOgPYQkLxZHW12YA534XZ1lbB1X/7p3ZBY9BfvXuZZKBEPA2d2j1YLR8k/hDM6ggAK/uwkLaM/uke+yvg6WiSuFNIm6YBFWMwooOUWTerAppTyjzMx3Dcrv5HIsfzwyP7CM/pVzFSZmXcPug3osRuQoaWdN0BfB9p1e7zjWaHOlA0O3iAW+6hS9SCaCvthQQWQzDZfkPmpV2FbLg7RS+xUeKk4JKwm+yREpsC6CfTLUihOMTjSyWu5XYsEHx5Jk/SiX0H4QcAGKoQqoolsImNBOWSnhYx6k28EzztoVak8fQJ4VVBGiS+AD8fBOV3wUBAEeab/JVYCdSVaMWQ3G+JtjRhw0ZySDi/Q9NRpWmcvqUq1BDjlOpbSisEqX5rfnOnjhmU07BKbkYNC3BR3Uxg6nd0SptpP2T53cPCvTIEaUooCuOKxynCCCgmeSbw07hcegos7LtKyAQVShMXkIyoOmg5pYsHrZX0M1blSfljniVE+7dcOiHKLcVM/IQ2EofPXnRkLH7RS55XJvwlQXoZvCSbF0bvpl9oWIImLQreTdPq1DDdeFclXuK3tAjxSkrd9PrA4irK9Z/1KdCnwKtEgZcCiy8CLL7xrfcycfVG03ZiKIN/mXDKrMGLdrbIQMskeD7g8ipUYaq6NKOc/ywTvMH0OFKHXMMnjte8K7/DSDCJvwhYRGoRCQNMGiuCAQRO3jBA6mIXcMH0TTkziXK1/DWvcl+MwyuoMK5gRbe1ppe9KmQEZSDaikiepFluimRBBoFJ3OctxUpetZZUrNbTa2w8mPBV8ehJKySMZTQNT9+zmjmGeorqLDNzbMCF5GIINahBVpSHRifDQMk8yXxZHOs51qpCCSx86H4VSgKKvxoYF555Sl+lF1ZLBkRGxZVa7TKew9gIQlwF1Z2lK56qFK1FCoEKC+Wbh7G+CZPppm26GNUFZwzHYaYEWK7xqr7jLtcaRetGVMNijaQJgI0B+zDA0LnRnQbCMj/qgntoKyHAUAIhc6/0QzeySh5k8NyATy9HMlKwkDFK14WuhqsCNZlv87YullnPVLptVc1rGboqqXh7fprN2QBN0omyanStp6N1wgkqdBvrKr4M2hwM8AwrxDJvtHLaR1UmY8qorwAMvs7u5d9CnUlvVqcwv0pX3PhuCQNqmfyotFAvP8hT8jkGAByhHrOPOpJqRac4H9DWIcwq8V2NT98g/7g7ba9Rz+E9nRzpxCEL8DCZtJHlUnpwLOCgTWYBTeVkJ3faz3hxSQzQstGl9R75qm62SzqHAi/Ofep/yvc5CNM96sZvhJOxF4TMsFfDLEBpBiCwyR4hW0gYtil/PC+RZsoM7ck+h9+CGxLajylA+psSQOsogzyD2tUCkqsxwag0B6wcAegF1fZPv79DwGBW7m0P2lOpoQDFOmg/YV4y3pqzm4fqToIoAYaqhH6CFsdTAKAthhKOI4D5MZKLAYD7KH1RNaqMQNaDcKYrTctiwWBAhe5v7b8BdwSApGkjEycbcQjPrDPAnu+A3Pk2eGOb02ZKRv3G3JPCqwkICgXVcenLIxc/Mh5Bc1oqYUJHbpXS5Rsjfd0J5yAt29u+OcXpYkHZVdtvuniJ0u4pLpMBhm5EGVBC5DaFXM3DhZZ61HfWq9TO64fHd5+m8v6pkV4ULoFLMMe4ejgmXv5d3111Tfj2RSnfVaFe/Mz49biQb/vwwnue1TDd59173yfFy+ny+zynkngfWFTK9699CvQp8KpS4CMBCytXJlVuMt47zXjLv3b87w6QGWh90b5z9Axs6AYy0fYogzAh2oHXYN7mZ/nDfRmE884JmN/FmNpJ2ZVKmDIm59hXwKQWqQLMkNKKAIxW2sCKaICFDACMQClZmRxkSAJYSMv8Tl0d5JlgwjhKO5zUXTFULcr8fF/F7aEJf8I8yGRCA1dBZdpkAnv0SP2oD3HrBGQ1vTcNwwoqAixkEvjte19KWxkTjV81Zp3AI9QUe1nMAComkFyoBgXaAFTIDMFWAywOiayrU9MQDMioaLxdbAdkvmQ2NPIsZZbJ15BXhl9GXUZEQ251sWXOTG9PtS/StLzqx3sn8+0pUzoDs6KNgaBCmmaTOWjuiq7vXaF3tdQWcKU3NhQw1L4XCAl4ZJxk1edIRyPhSC0oa2mbskqs7YK2E4IE90KIShDMsUy9cW5qQA2o0CWsq75jnNJX9al16iigeB9mf5vf1v9jMMWvsXp7nXh6I5I2aEwhJcErFfYGK0gBVAez3QVn7l8hqFBq4w7mqnkpjfHQ3mCfOA9RZ3qA8fQqqkmWV9uBMThEvU+NwzS7Kn3Es0Hj0V91OToAGB46U2WJfsizMxjqE8p6JhPKKYNd+4vSC9Vl6ilTKc017M3KNO8FOhoG8yISoVl2DBcAKLWQQbZf637XOthvJ5A6bKOmtsW5iQqX0hztlA5RdfObsf+NwnTL+AoYdBV8DenC8vXlZmFpoVl5utKs44Vqm/pnRV1a0jcEEnT8eHByZV/GfAQQ6qHTg2weR7+bQCVn+fq1Zgr1Ob/tTUAZGSFogMlGiqXamoPXEeXaJe99QI+gJOqIpCttRgAP2m9YRr175dvmuTuRW36vdBmrUw7e2R9PKNMJG066P8ww7pytJwWNFNQ8/ZDsQ0kDycYgp31WiZk7avtR5D3pC16qtNEwvAgYM5AG5icAW/uJ+2OkX0N7pRsal/ud+426M7bfre3sgoKSCtMyOfGDfd3vDNIERPgsNljUO2MFcf12BBca9Jum37SSiVm+06IuOBGpm9+IfYYkcpoJwcs9z8o8YM7lqHeGMZR1uHzkVeehIa4KV4Mkb8oqnT0MmznF+/KgXLmvafu+ppnwhuNIEjVQeXQhvfZRLjV+fVbzrL+7NathU9ZS+aTbe94+M259VgrDg8677jPD/fRPfqZm912/+l3VOtZrN9NK03w3fFP/f+xj0S1P/75PgT4FXg0KvBxYfPmrzdeRWLzseNGAVEb8MqKXycBJoTvCt5OGiXubGYAbrt7KrHsmRp6dh3cAdrCrwKJIH2D8GSitkOAidhCACL039YBFJBaoQwEK6uZ1Gh6SHPmQJsyHaVXmJHmYpquJpCu46Klb8NwVfcuYklHQCrTCVMgIcGbVlEk8g3StflufqHCYeY5yBTYUZlDmJ8wFEyzpJJh5wPTILKk+MgpTqqGmwGI6wAJVKNSgBgEWgzJGrLy6mq36j5u6HcGNqE7hqmYx6iwMpyua3X0vBBZuLOdmcDL6FL+oU5CvwEK2WcBhddy3QINQmZV4maHcpp9VT8Pw3rILUtzUTUNnVTHc90H1JHiYMHdpS/KShqavx6ZnxFFdaYb09C6lt6WAIuhR87McllEDcncKf0Y8N2TT+88iedxFBWtJyYcgx4oQV0nDFn1D9SfBhcDENlMqchtGexGgoLRDuwFVrtw7YhVA4a7humHdJK6yk1kYYl29zrCibVsZ9pnvOY9dWVeVCBWmdVbvN9Whp9+p8jMOYBiTSab/uFOyNJVxN44qMXTOgI8RQMU45RZYHJPnEcCCThcgrdRMBtDd1t25WslBmGj7CtV0NX9bGwvqHFUa+oqesiahhTr441x17Wsc+7wODp6htnQA/eyJuis2vsBim/oc0R/8Bvxu7MvWQ9exYXah/xlte5P9Sm6w0/qNm9cDLJ6trwdYCNDDnJCPdLZDmI51SZ+B3n7Tlnkf4CIYn4GuN9hBnZTxMMbGfkh8lLwogZmwvpSRRCkTkiuA4T6gokoi863yXNoIWgQWe5TP9P3mVBka4Z20zzeY715IQdGgXaQFAMUB7JT8llQ3kybSXLVHgiRNpTa6fB7CzsnvTDWsI9uPdvUQRA2pekXf8EMIiCFcUcNC6kKijjlD0g+blqhvES9G25TvjLguLuRbNS3j0p5RlzMyNJN+vXHH7k0+pZsDTNo+EYCSdMrigDYWSkAEEe4n4rkAQHZzSMGG40Hom1pQdK/5Y+GkhGf9m9uX/rkQtRvyRS/I//xVyas7z9hXytPzYnWT9X3i98Lxq03Qd/Xopmm9zt8Y/DxODd+NezFsG6KTRk27F+fKCDXlcjXsnwVg8XNf+fXmtx9++WLl+7/6FPg+p8DS6HTz4zf/fPNjC5/4nq/p4iI2u8wR3+nhHOxCxJSbxnYOFhfPzv7wj78SYFEHyfo+Y6SDKKfv6vv623D1mfdlTC0DdYlbnhI9R66mZ0iuYdaZOL1mwG+vJbSPzoGFTFEkClwrwNC+QsPq7IQNs7YPQxebCxgDJ369x3hWcOLVNLPy2qZTGSFBRhiq+hxGoEg2ilTDKkTf2uq1tBBQ1DNG1hacd6W6hW4yNMkzD/nTxpd5kFlzMzBXVLPa2TJjpqH0wwaXMarAYhK7Cg24J6Y7wEI9eD3wEMfN6FyRFyiQYlY/BQNVAiAjMQIjVYAQmhmUzdOVUQFJBXiGg63Mqq6MvTv2atysrngYfRjkGI1ytVoy5j6XPmswfrpYvQ+D7TFHHeZhbNxJ2HTtwkoylGLIFuhxyV22n8Bwj8Ll3IChvAGDp/vM2E6Q97BLsxwUNVIUjcHdG0MQY7n0zX8LJnoG5tI4xzC42SyP92uUw70hVF9x9+hpzlnymCcPw9o2SinccfwhDPYjmNtNmO9tjIIPYbTdLVl7irt6ZopK0HBUpLSn2CJtVZk0oD7T2w+ARBBq200gCRkHVIwikTjDsYBqQNJqF9rssrrvHhAe2rBEfYyynwBYYucAc6wqDIVLX59CsqLUYQ6bEHe1jjSNdrbvHlC3TQzFPdS5n8WW4vrSfLMIwz6NTYfSAyV9MqnG26GMa6gu7QJeVN+yP27hDneHuhwCNqJySB0ipaJ9BwUWAKrYRxDWfSduIrG4CbC4CSB4trZG/thZUKfkQR1k/ksfI0L6fwGtw9aRdjgiHyUPk7i0XVhabK4BVDawS1lHWvFsdS27TytJESC5oSWppDxKLFxM8JuyLtbdvAIsKKfAQumLEhfHmGIM7feDRI84ghxtpQSdfmMBFqbOt1jAuSCgSEgoaPINoKLtBBaqHsbWRPow7pRN+gpQGRSgkD+Nz3fAaR480+A+9+QnsBA8nghGqVNc8pLuKYsDQ9RVAOi3KmhwIcRNIQNCKLtALWMEaYsoBJIZh3guaLRcUZt0zKC82q1M5sShAWPIgqACoObGeEoklcJVYGFZpFdvLKfPlMOnH/1IOr24/uJoL7nvvfN5eXExTkL1yuG36eHfbjJ56HPfcyaNPGxDcalxfdyrlz9qmkQtRSBwG628Pq+zj0sJSrxOsN5z0+7m1XthYt0I/u4cfxZUof7r3/3Z5pfe+Z1Orfu3fQp8/1Pg7sRi81+++Vebv3H7R77nK3vr1q3Mn99pQV8KLP7gSwKLd69M24FVBjsrkFeEuDhoE8DB1ME2Yflb/mfwdUyXYXEgLhKE9p5AqjxUxr+OyIYL40/+ARbuXwFDYWWiqwyToErFLpN8dt+GgYshNxNyXXUMo2R4JmrzTh5O8DAHpinTVc+oO/mb96WMlMt760KdUtf2KvMoIzPERC8T1c5Sba2Jx11RgeKa+soPFIbA8FFdgCEQPAzCsPeYhqRAcoRxxVWmaQzmYxymdgJgoZvZYewrhseRWIywiqoaFGFdf9Y1qp6IBBgjFCAG3DBNMuayZ2Hs+W1bak8hgy+D4brlnquwMl+U9RBeSVUngcYkZZMJn5fJIx29M6luEkmFtKC8pqH0wlVgd+92/4h7rC6rMiJAWISh1POUIEDvOLpd1ce+zNEmzJ5h3bzPPO/A+LymcXDKVlZUbTte5b1SB8HLDqo88zCfC5RtlhXuUZk5/h3Srtu8V9rwHOZ5hbQFTgI4ffdry2He1l2mW3uLpzDbDwj3BOZ6g3tdmZ5yHQAsjMMIztC8CzDpkwILmDalPAIId55WKoE1elSbCMaRlofWtCH9EaQAQ6sEw9X6ptmGid/C49EewEWplDr4SpEEBao/qYanapO0sZ94Xr+21CzjAWsWz07PMMx+xu7am6RhP3V1fg+G2xXxMfKYx5bjtds3muvLiwEjMqcHpOf3YR/Vte0G+45sILXQrsLvaZMy1femF1sG2s7+bn2HoNuY7kltE+q9xCrHMoBgiXLtUJ4D9nFwJV4nB3u04yYATUlI2eUeWwbS8tuyf8QdtPSj/a7fvNnMU85p7EDuv3e/WUVaoSH7JJ6LtEeZBJyFsSdexgHqKBEtl4BCr1FbtJvfTxhvwvnM/PL9QNN4WRIcEYcPkm+eRqBPCsRNLmOA/Z6z2G4wNpBHNrkjSr5ZVa1QgxqGDsZ1rDqhXU9l/Nuxwi/BcplvkVwovaBPGp7T56pC2cb2h2yACWATrJwiMRmG8Y+dDO2uutQpYQ4Ba0q2bNuEF7xA1wGuggvpKSiKxCLPuKe9RvguprDJEljOIm1SXVFVvgAL6qCtlN9vxiGSMBnP3mFH5fB7cuyqR8Y/n7fP6u/6Ph28/WGs+t60axxf+7z722eXj8thUgryrWkmvuWwXS8dl9OucRKsU5/zml1M4MLzTviaU9K/lG/Ns5tXfWbqNW4tbx9YXKR5/1efAt8vFOgDC8Y7Br+zP/jSl5s/+ea3rmxXB0fPDJh1MG0HW5/VATPh+OETn3cH55qGk3cYhHYyLgyj6cN0MHk6oRuWYLnmnrBRXeJaVJW0e1BVCUaYyVcvUEoscrISKKCoUotIKzKRAxaSLmmZXq8cLaiwPMmnAyqkhvX0tN7WtZ4yDzJ+7epiMews5Kv1LvVoJ+Y8lFEscbR7UMWkrDQWRh/OgQQKNWWSZKhGYWbGYD4mUIMaRQ1qBEAxwHk6gi74KCuow6iKwFCokHHA6rgr7+4Y7a7Q7lOhJxqZ/qI2AZPI72GkG5GUkL4qQW64pl63m7optYgqlfRpq63Ew/0hDGvbHgBAdKuaFqa4MmxKAVS5kNnRW5SbyemiVQNXJRVuTLeMlMC8lHpEikKe0lPpiu5tDS/zdQ2gcMPV6jA+qpVg+M27Ldp5l5Xu7N8Aw6atwxIrsNVlplITd7beJNxzwqvmpR78LuWVnkorVJWSqaoqXDKgO/SdVdx/PkPqoI2Bm73pUQkOtRnBANpN0EahrR6sJtDBlwF0BXqSMmoAOzeBihN5eMqsuSJ+aH8EmKjas8/1hDpYN1fr92GEfScDL0Po7tTuMzFNGwv8lKZpTxCjX95DLoDFctyxDsPgPmFFf+XZOqpLSEoIbz/M90baukpVXWp5cT4AQ+9Ne4ClfdpD+kwADjwEJY9X8EIFEy/QUH3Ib1PJ2S4r/gKRMOe0clShYPQnBLa0oXubLC4sNPM4EZgBxOztbGUFfhh1LtPZkdmnbNPkpRqWYEfAoYcsT9/bt0Yxmr5151YziUqa3+m9b91r1tfWQ78ZnimxyE7biG7sd9Yzv4nrQoCupVXdUj2KrzfjhQkf0Q7pu9Db70spgG1n2/gtC4xjBA/tIq2jv0SSoTSDcjg2GM5v1X7jNxM7Buqh1CbjCPkXD3FF5SpE7Y4TMPs0SmkXy859fhPGsS7Ahf47RF8aZNHgjO9cd7qCIG0isiBBv9OVrlIs1TIdG1QdE9gMUK9IQSjfGd9zNtGkfOUdxuqk60KEGyMqZVNK4bcisF6kXabIR5BofyFFUj4/zu8pK48dx+qReoTW9YnVamN0wtUYV707j3lV2uWt8Wq+9dqN5/2Lnl8I15YtJbyqnN0yc1/Lm/J33l1I84of3bLUNCjg+Rxo2lKZ/0mbsvz0X/vJK1L67jwSPNdy1Ws3J59Zhzo3/9uysfj8468272w86mbVv+9T4PueAlPwZZ+cvd18bPL693xdJ/X4yDzynR4vlVj8/r/+4+ar7/ybkvYVA2oGzjowE8rfDtblUYbtTJYm4LM6eDmI+tYkIzFgUq+rl2WlrzL6LWPv4MZZAEg7CQcIlHAOkIkPo+KqnwBCo+0qsdhrgUV8v/M8dhIwC8YrA6fpl4k9QML8WEVMWWAoAnBaxiITgOUnTFYhrViYOFdHneTLtTLtLTHKxEEc8zMNLoVWxInhtCvUMDuqhchYlrRp2BCzMDPqW6vOMIZO9zgrjxNsVKZx6dn4VHM0PNHsDI6yFwE++2FOzN+V/nhQgqlyBfqUPSVGeRZvNJaCMP6zAw0PFf/+Skw0rNauQeNnPUbp8ckz06DVpfyCJ5kyJRw+P2bF1zaQjh4yQhqGRrWJONoybLcgRYmFth5TMKyqQ0XNI7EKbdLWlHMHoCKgGYdx1vi6ZydB2D3a+REr4EoensO4CuJkjnSbOUe+HofUW0CxxntPXcXKfJ4BCMzD8ltX3cOWfQqgBuVSrUZAuh01JlaHd9lfgM3LjgEaqja5AdpQjKth2kljnHxlzkdxUzqPatICqknzGEcPwfxKHQGL0o5tGOs13M56usmb7VOlYcV+B4aUZ7a/am6TrCzPz80FnElzmVYBwvgooJM6LMy7y/okbYvLXFb1VzGWVsIwCqMveJhw5Zt4ZZUdz0AtQ28/ksl3k0caP3lohyHYuHf/YfMUQ/N9JRT2efsG9Nle3wQsqNYEGIJGkZoJLKivqjUTAMXlhcVmjvIKePcFFqzeC4RUQ1KlynMRY+wZ6iWzHC9TfI8CjC1Uwah4M00a164vRbLzHNuU+x88QJqzHXA1A031FiUb5rerTG0UqZSG3vb3bAZI/ZW4bAvSoIuLD37TOahL/WZjn0Kd7QOQHHBDv+C9EgvbQHUqpUQOkLErIQFBhRIuJUpEiWQgEkboaXk8za93GIiQpufXXlQm7WP8SgIJcB6c9FVdUgqiQbhSCEGdtM4gz7cl2HLjP10S5z5JkDZxbSvBjkBDVassMEgv68jvBtWqEUDFJOOG4GKRdlikjyix0OXxnBINxhclmi4MkGIZqCy9Ze6V1MflW/VRb1xPPUug+izETSD+UFaLm3edsCVG+Vto1X1ynr7xku+L4hqt8+4iddvyW5H26JXR39147X23LDXvGvejXmsavbxq2r082/J4oWx/FoDFR6VdP1yfAn0KvJoUeCmw+L0/+lLz5W+8k5rJXGfwbUdrB8oyRpeBUbYzq3pcyxjJJOBk2v6+TB7D1Eki+tVMmJE8OEH3QAMTdRj8lgGD0YlUoQUBZSUFhsDnxoM5OPcGVaQVWWUBWERK4YQMQ1pVOnrAgsG+AJrzybL+DqOcOvPHazsxpP5M5jIZARZMxpFQyBy0tOlNJm1dKw1MxncyC6oduOIqg6S7yCGBBb8LcaES6avuoPpI1J9gXsdgDMb0VsM5ODbV7A+PN1s4Vl3Fz+UmzaQhc1kBLUyV3neGYUTd/G2Sq/dDgAzrYvm1xZDp0M2mm4LBXmUfi1kYEvM/pMAHpKtqjm1uu2mvIGNsGyslcMV/nGsAWltRpSLmodcnjbCts0ytaWhcDZSBVtA+YbDnoA8oVdEVrHYOqmoITpSKCFTsM0kLhngdRv199oQQWOj6dR5G+nVcnd5BP38aptXQ+wCLNZhMVZp0ERsPV5Tb3acPkDzQaWJ8PiXzZZ3oQ0oFNMKVBjLLo6cYEyOh2IbJ3cFdq/soyNgNtHHV95+DuXY/iHEY3MLQw/yzGlyMZGF8aVNtJzSGlvl/vLrOhoBsHoekxef2X0Gl/UDaKEWQQTS9xQV2UofJlJbSfZn9PaZh6M9gyAUgRM1q/yOAxXPAj/r8czDvs4CbKeIrMVL9SlBSvZv5nWzD5MvQy4wqaZiEwRR4fwNnDY+erKYd8iXz3nz2Ye5VxfK7SDtiuDzGTuFusOfK/xhlu4kXJ1WhZpFY7Gw+BzggNYDe67TTFnVXHeomoEGbENV7wrzTdqa5rdE2fXBmaSlg5Ii4GysrUc9SsiIjPgV40z5CIh3Qrn5nUzDF1lWgoDqVqlBKLLYAQdqKCCrsC9I10gboKmCyz/nPNDL+EDbqQ7zX6NpxQ3usY9rIfhcDeNrUdnJ8cgdw+5MvpYfjT8aHjAmMe+ZBuDwkmHnZVnFVy1PT9Kn1iupSysj3QP6CCU/D5rfAgudGstwDpg3dbFOBTKlLSTRSFL5DPcK5X4eAwl3Gs5+NYIUFiVH66hTfybL2OXw37i2j1GKZ/qIEQ7XESDSTJAVsD+tZym3Jrz4qI13f1vD1d/dqWNOsx+W4Pu++74Yz/17MNo3ET5v4ppTQn/W4nFb9ndAETNBOBOlaj15e9UHnWtO5qvy9YG1ahq2p1vLW3777s2C83aNJ/6ZPgT4Fvi8p8FJg8YU//KPmS1//RhgaB8E6ENZBtjegQhrfFSY87IgzAk9rSONKvzp4l3tHWOM4UStlUCXClXUZq6wACiq4j91DO1E7WWeC9spZpQlOsDKGMmqCi3iDkjngXgYhK9UwjMVAu+R5IX478FtQi5r6ttc6SdUZITWjflE3gfEqaif1GjamVL9X/1LfUKOlyyDcnqBCRltPLDIsggvVNBIZhr4yFrrNzMZXMH9jTP4jU2yCh+rTIPYUR7iV3Tobbtbgix7BFLtZnAbKcDaoMxTpxAQZT8IoI9fILr+Dx7othaGVyR4gT6QVg0hBjmHsDimPtB/lHexrJvZjwniOwqhIC+0IZPCBdLxnMz3UV2YAPJMwLj7XpeoBYaSlbmazczn348SfctUcpikrojyzDZRgCCjc02JLiRPPlDrcgZG8yVmNqQ1XNqfDuxTAQg9MWzCSur/Vq407Zt/gmv0kqLOr6xpTG3abvqCO+gD9gaXtqJPYJ3ym1MW+mjbn2QiW00pRdAkLW0ZCqNWhHrSBcfMm1+rG1boswdQqoZhixVdD26xKUx77WYzCZdxQ4fFbcc+FdWwPHgACVpBauD+ENLIP6KlJ+wP7ukyxKv8a6CsBUKIlA8zL5jb7OixgezBA+q7KCxB2UBlbixoUe15IOxj7eYy0vU6ZLnXRpsKPXYmGnp7cOFLGSZU5pSIa9So1+dZ776NW9Sz2J36P0tA20lWtjKztL9Adhc6TlGMSYCEgGoSpv7681FxDPWsZd7PbbpCHpCe2G9BMqYV7YFzj3TSAxJV/mW+vk/TpfbZ4d5VdT1Pu3bELMNnEq5TxI22jHEpr/Bb8Pneptypi03j9mhPU0RZ+X6pCbSoZAswEVBDG7yzSBsopjfeghbT3NE4S5BLX0IR3kUS1NcckvwVBxThtYR72FI3Zd+lTjkMeNo3tGy9Q+X74NognPesYaSa2aR1XwB/tYeT8L2mkrIIbxgEXKWg77a0EGv728G/GxdikADBoH8dRR14lMgEmfquCCtoXS57Qlg6FJT92G4CLcWhm/1DVbpq2nOMUWLifxQKSC8FFdoJv8zuvhyV4+WEd61FKXH+9+GqcmkeNX38bK88kdH70/kj48oy/NV7vweWbTrnyirg1tiX+UPzL4T+UXnnQKUIenNf+cgR+916ezzM1lPXtA4tKjf61T4E+BV5VCrwUWPzzf/kHzb/66tcy0TroOlFn8O0MuFltY7DMe0fN9p0rcTkYuX3kMJow7X2eBCy0jCqMjiuMRe+ae0CAK6sVWAgcygRqOUyrBRa5hynxPZN5dx8LGYPiWlaPMTCIlr+tQ1b5UifTa+vVacVSH0vp0VaGi5NIJjxuIplgIhdY9PTR2lkm8KK9NwXXqYxXTif/c7sKGe2oQXFVnUqayiSoG61xr6BCtZZxDDpHYQoGARUngIrj4clmF9WnZ2z68Bgm6amr9zBEGrBrQKxUwA3qZshrduCkmT7FDevRHl5r9PQDuICJHRpgpdwVcdMEpByRZzzk0B6DtgF1PkWiccYK6BCMiXQ5wiDXtrAu6vfPUqZZyjaJXrhMvjYNnsekH1DB1T4wTR3mYGSmYGJlln0mSNFrl/s/bMB4l03uTvHohLE2zLpgwZ2DlZC418UaDJ1AYZP6amAtCLUvTFPueZihWaUMkE71H+mwDTO7Q3htbqSJG83puYeXOe1jAtIYFAvuYOTGkTCMosIjOFDaowqZ9gUaNz/H05JMrbYUCzBmd66xAg9wULVHz0w6DXAl3hX1MGysDC9h2+B+FdZ3G4nHA/a1eIpNxC5pygRqT6GkQ4Bjn9U7k66BbfsZbGjs75bT9nrtzs3mGukJGp9ie/CMMu1St23sI0x7B1Aioz6P9OYaq/9KU+xb8SREAXQru0q8sKHUVdW6Gco4ylWvTI8ePW6eAwRUCxK0xEsU5Yw+P/WDtAE8E7TNDNKTKdJXjeyYvqdq1jJG13qH2lxHwoPqmOpW2jtoPG3eGg4rrfB7kWGfpX7XoOHRWVG7O6BvbWDnIajYVvWMPkb1840GJBHHb1fDdPuf0psZ1N+m6FtK9HYBW+uU/+nac95rEwFjThzBm3Yd7mOiROM5dFIdzf5cpIas8hNWiWjsPugjSlDNK/YupG1c6VDcWTOmUC4PipH0YxtFPr6Xlg6u1tP30s1voYw79M8WJJhGGRNKQgIXy5N4XunTgq9hAIIgw8RMzzGrqHS2izD0VeUjATf0qWw6SJwTFgQO+cZPdD2tFIPvdUBwgY3FJBsQCgztL9NVaiE45xRcRFJIZoHdZnrVUSrwoTfS9QUxrgzrQ+lQD+Onoj7wvh6pfP3RCcOjxKmvrrp20/H95fwux3lR+PZ5qWNpD0veKeWF+wvJ9gIx77ShajbWvw8sLlCr/6NPgT4FXkEKvBRY/M7v/Yvmj778lYj8XRFz5bKs8svYl2HxfDDvjZisAnNf3zt8eu+j3JcJwGcyp54WwjOemwAXkTbAoMlkyUyZp4auEfvLZZgOz5Iuv+ONxTBMrjWtXGUMiG/YFMAGsiDt0YvfptWdCWuZk1cbxXmoMAEVIHCFQcizMBAEsK6mn0mrBR/tT8PJMMRrFFc3+4q0AkbAVV+Npy1DVr2NL4MAQ6Pu/iQGwuOoPqkCdTI23ewAKrYGRpt1qqb9wAqM0h6M2Air65MDGuwqsSAJkpnhBhjSTAEs8H0K08OGYq7IQi+NtsfwPjPOxnpnqkMpnWBVX9eXusDcY1VUYKGaimoWMkfSVqDk6q+7Ik9g4zEJuFDXXZULpRUaSKfduI90g7xcFZ+mHjNs6qd6j8ySDG6MqclLMJKdi8ljBtro2Sn2D4ST6XPTvOeU243csmJM2trURDUJBnYcDkhvVdo21L0f9mGO1eW3zNJdsKCNhcBB9TilWXuADyUOqtQIBqxbvIvxXrAbRhGGUremggptDWapww2Y6Ndxr2parqo/Z6W82POgYgQw0BPTMqpM11APGofxte0PMLx9iorPGnYMphfVFehY+par4eyFgXenY9DlCPRSYhFbBNrYdrmDZ6elxTk8CZ0WYAETbTtq+K1XqY21jQAU3cveoWwLqEWNw8gLFmU8dSv75OlavmmZ0BForC2HAOcEYLdHHQTjghpVmJ6jArUDA65OPx9SGGJtKGYALkvX9DA1Db0Hm11UnewLc0hKbgksVB2jPxb1RqRY1EuAIZ38pgQV0/SZZcDP3bt32PF6IEBmbZWN9ZS+AJg0cvdb8rsQXM/TPgI4Aat19nuK8TzPBG6G2STPNei3Ah389mXgs0M9YSYEcIQT0CkVek6ZBRLSQTU/D9tbNSfBZqShfM9KDFS1UurhP8tvuTKUlD8BAzaiz/eho+p0jjvx7mbj8kabD8cxo9gXPBxn/OjLuJLPP/e+z5lxwbGC8YFnjjc5iJd/Lqhgu+R36W/VMaWXp2px2Umc7/dE2yvOM77v01ZyMZK9b9j5PeCCfgB9VIu6CVC7RZ+4BtidoG9Huki2pe4l+95fymHZPbwWuviXI3Vrn7f35cX531L/8rumQ0KFLqnjeVjvklObX9Kv97yraXmtjy1zSmNaed7SL4mV+xrPR92jE7L7OPeJQ3qhynlmvTJcjlDz6NWRAOVZS6tSsz6wuEy4/u8+BfoUeOUoIL/1wn0sPvfPP9/8y3/9pUyGriyrixypgcy8A3WZRqInnD91JG4H8SoVqNeMw74jvkDFyVsmQUDgpKt6RiQOMHtxPwkzosTCvE0joIYJ1CP5O0m3wCJp8i7MbMopIIE5sIzEtWj1TAKm0UuHUClc+yZRCgPgkwtDP4k4OciwOIl7zWSRCb9MiE7xmXDIMWHb8DIGARVM+DIqgonYTsA4+1tGIhIg0+Iebi/AYkTGH0Z2bHKWlcbZZpeNVtYBFSsnA9kFegt7AXX/R/Z3Ah5mhmBoYCNJLVKaUUDGhLs4cw6yIizTLIOnwbUM+wgrmRqEm98JZXVn8ewuzlXVklNWPDUmHYPplp5huqGx5ZUhHYZREZyMCj5gGK0z/1nZhgmkjZQsuGI+wqqpkhdByCT5yYgaUOmUYFJm0TaznSZRC8u+FuThbwgSZlu7Ceu7DWN4CJOurcQRbk21m9CWQSZaMOqGcoIjOk900m0rV64j5aBewyIu2ty+dwCwyKotDJXSIxnK9Hf6k16a7JueAgGlEUqqri0uNDcADEotBBsbqDhtYONg+W3jMehlGEHFAuBglJViGiPAYpNwqlQpYRBIWSYlKtLNb8y6KR1SD/46dgsaWitFMP9FQIWelU6oX1zDIgWgQPHwtGu4Z8+T1wKM/2uv3UaaohE5e5nAcGuQrhvWdZj2InGCJrTRKCpGQ9Bb5KFqGCxpmOg10oqEhvIIyPwSVH0b164BAOFO2wsAJ/vwHuBDVTkBmmBGw2t3B1cS4herZygB0zOkMX7bMvLLy8vse3GzuQ2wsHzuffHk0SNUptxgkD4IXZQC2I8m6Huqiyl9kWG2PfzKAhzoI2H8+Z42AEPJh/0vHGOMr71L+h39zf5h33e/DHcF3wMEBFz4zdEfjmDSNeQ2j0hFvbfe/DOvKVbzdQEs4+63Yf8QRNhHHUJ0MBHbDvK2PVVvdCxwXFKaZft6+C6R2vsCJMiFNMq7sghhGH/XscN65n1eEJlMi7S2pJt0CF/GEvKFNsd815E6AjBOkQzlN201iLRodAZX1dhsjVMvJT4CNe0t7tBv7iKNWoJ2eljzc7GOHiX/cp/f57e5s11CDOvI4d86vtZrXnT+mGbe1Tjt7xeF70TtlSv5tnmVZGy187y7cep9jVN/9642Jketc33eC9++L4Euh6qhX3x9Ub36EosX06z/pk+BPgVeDQq8FFj8P7/9283vffGLYUJk+gQCToyRXmRgbQdfLqodlF9eBQH8ZXL1GlBAeH4yyhfm34k2wMI0YcaczKOCAIMZbywwf2EueB91KOPnJJFkVH6bPhk4e7TvzafkyZ0ZZnaI2hK/nGrqNJC3xEuxuPaONi3TNHrnzbmEgkRUXXECz0TrROg/Aseos80rAIQwbnynhELGoJzoMMOcyGCppuE5oJWtMyIMTKQWPJPBHJJpR1IxPDXXNBPzzSb7VKycDjYPULdQd/0InfRBXKHOHO02M2hUCywEE9I6DCGqPPrKF2yoQoMH0F5ZqRD5FnUoKpPn8dsP8yVj5+qtEMVVXzdkc9XW9neHZts18a2fIAjgINAQMKk2MoH+u152ZJzVz7eOuspVAjPOKSNjfQMG6Q/hbLnal5SGyOtZ0qgBpcyE5Ulcl7I6v7UNsw1DfwiDbt3c52B8YjS2DG4o594AGlDL1KsGMwqo0Y2sK9amLantU67mj0FnGUalbf6rfVY9+QosNCK2yjLON7AnmIfxN/xTbBKewkirhjMKuNFo2r0lrsM4L8F4T7HCb1UEa/sYJe8jMVLiImBw5XwTZl8mV2bW3nmAIfM0zN4iedy+exvVpWfNCnm4Gj9BHe13+8Qt7mABMjLLpk06W6hqTUBX1a9ev3srHphcOV93ozkAxQYSCIFLVuTJK8bBlFlgESacuBruDvENPcMTlFIQ1cD8lj0EFtJJl7ICi2WMsZUCHALOtEGI5IDybGHoTsOiyjWV8UO1KlW3Hj1eiVqU7XH39btspnerWQI8PX74qFlBRWz16dMw0X4jAhZVtcqGdthfpP8oYRuNBMTBK33QFvObp48LLKyjxuI2smpMgh33v7B81QGB9LIfbfL9qH5pOjL1Qgj7rN7ZpNuxwJoxyPZTamDba0TuvRIP2/EIwJCxKWUoTgCMEG9v0DV9jHFTUOr46e90PhPlcAyxvpFI8DsjlO/8Nghr+CwAUH9pYrjeuEOw1D1lJCZxIkXluS2WhQLiaWsRd7p8y/SYgAs+0GYIcDHsaZ2gqxIhVRVv0W5vICESXLjD/Di0TJFI0+MqcEGt/KCgYTlqGOOFPj627peOGq5HF977LHGuCH8pen4mj/ZF6EV8j97zTjq9tNvwvTK1cXq/2/jnwT5c9vrOtrp8pD48rPWr7+vz+rt77QOLLjX6930K9CnwKlLgpcDiV3/rt5rf/f3fD2NQ1A9cnVPKILhwkHUaLoN32Gqe+S+TIWEKg29Yme0CMpyso96k+N60OF091c+8K4muZsaDU8uMuhpnnBymb77lfyapTGP8LgxhLxg3PMxBGRn0y8kknhKXt4YoJW5DJn2TJY75+L7NLz+obNJxkiXNMsGX+6TFH8MnDhEMIwMQ1QSZJM8wB0VaIZNTTxly9amJFKlF0oOJg4NhJ17ABaBiYHK+OQVYrA2ONU8AYk9gak5YyR7BSHYcacXkyX4zAbCYGsSrElFlw2W2pCfcURinshkejFPLnMDNqmsS4KBajuSxHWTmPV2RtQ2klKokGgKXMLZbaWvBRakjuvPek4bqE6Mw+dn5l3oJOnShqj2HIEv1DfMRVIzAfMmsmocdUlWcAkiLhMwyeMjUSidBjVKC7OwMsDiBodcuwtVkGdEwzsSRUXSPhejWQ0NVfrRHcOWfRur1PVfoA+wos+BWgOUh4JCpLn0S+vF+Gt30N9+8G9sA01/FVkJgsYkbWtV7tG1QijEDuFiYZ1duAIJlVuXKvSt2AIGR0PBbI2MZ29gyCHYEFuRxeshmc67ms1Gcezo8fPiweYDtwwbAQqZRRnePzeuyUk5ZxwRw0O4IxnV3Yyer+zLAd25di2qS+e8AOp5qNE5Zn8N8p439MMlPBty8lV7IiNs3aNyoQEV6IE04an+2vaZgPJcBPoILgRONFkmKtHYssK5eZer11GRd9Yil96p9dkcfByx/7O23ABXLabMP3ruX3bUPkHJoMyEQiHSC8jnK+D1k00DSNw8N0E3X8cM+JFBQ9CqgiGSJ37oB1qbD3cmrdNBrgBn5qOa1QXjTkeG3DyvVUg1MOseTFn3FMUKVJiU17lw+hv2B33jdTFAJhf2pnt5GwgAdlV45HgrWDWc+jnem2Y5qCVuAFN+6dJbYHOZhWBcqpH1UD2kjFyTsr4JCn2dMhQ7mm4NnpmEuyplOLAPlP9POgu+dpYXYXpzQBrqqPhNg0IbupK5qnDYwy7TBbej2cSVTfMfTfMPKXur4ab71qONd/V2v5yHKk1o854V6mMxVaaVebaCavrEup1nT+UjXHoEKbb9dnJSrjdMtw8vivaguNU5Nx9/ed8P7rA8spEL/6FOgT4FXmQIvBRa//Gu/1nzu85/PxH2MXnoBAqrQFDDgxMXwGAZOIoRJZ7DMPybQTHj8dmJ1gnSlu0yC3PM+6ZCWnmeqfUSYTePKMHCtk7VZlUmF1M/npeRq3r6/8mgnrkxgptBOujV4QFAb2YHehdky+JcQGfxJ2Kk6c2mZCcN4KAWJKhTvDZ2zLVwJX1aEZQKGYcaKhOIcYAgqVL2RGVYiECYchiYGq6TtVZsFlhGbs6nZ5nR8rjkanW1WmeKfAMBWYWbHYKwn9lmlxnZiPMCCVXqkFW44p/xBYGEjS38LKGMQDzkwPVn1lImEYRfUVKAkYAgDZBsYjys3IW/0zGVuCVPrLEPmimpoS1qu1MoouWGXTJrejaawrRBYDCIZ0ATYlV6lLe7vIKOnrckpfax69bKRJbWHZXECLivAABCBCeXZZ5VfD0ICC/NT7cY6CToqGHE1vTBtRSVGnqb0xbMwoBon+9usZHi0BZDRNj+BleVzfwRBjd6HrmlX8cbd9MFNGPQH9x/FbkFpk96QblxbapYAFAEz5C1t7NO7lFXD6j3UtpTgqEbzHGlAVa8SrEd6QPl1J3rzBqDi1i3AxXWAxYPmgwcPm1U8LekZSyb1CEmSbaGEZAxAFcDB82OZdpjyaQzK5/DapOvXajj+GGnBExj7pytrMOHYMFEO+7+H7aer40iKuIcoaQslNvlmCWN/V6ohwx+VJ6Qx1ymfUgmlYDMwp4JPbU4EFtrxlI0qATzQ0X0sVrF9OMMt8vziUnMDe5ERmNgd6PAE4CSN3A1ed7Tm4fcfYJv2wQEBjK7SAiUP7gWixIak0n8Ef6pTpU8B2uzDo4TTKD6SNsIJKuwj9gelT3GDS5n08iQdlELYV/V4le+G50q+pLfAS0ARkMI361gmuIl6E+ULkLCPcnrY7uYnXR3bLJ+Sr4yhhI+Ujjy9GsN+W8CtY005Tae0TrlK+zGlb+RvWY3jR1LH1Ixb5l/+J66pH0PTSCyIp3TyDNVGPvgYdJ+ivniCV7cBVBMHAcODqptBtzn61A1A2ZsA5dvcL9FO7j3jooFjm0dlist4mUfl+fnthbteXdo+50tTqun4+3JaPvOoccuvj/DXPNq2SOiaZ3v9U6d3KctaTtMp1CgBunW5FOXKutV0DGvcPrC4TLX+7z4F+hR41SjwUmDxv332V5rf/Gf/LBNX8fBTmFSNrKvdQ0AEg/WFgdrfMAWZ8DKJCi7OgYUMQyZWrlkZ9+oJE+xzz6xqOQm0CTOF5ugO4r0HnczrwO6Anfs2gvf193mYMpEZtlePFlj0asS7qFGRTpLqpVPSq+8sQiku+XIvIypsvXj8AABAAElEQVRTIUMis6a0ws3iirG24OJcWuFKsd6VXFmObjeMc3bQJa8znrkD78nETHMwOsOeFVPNY8q4CqO2gSrQ9PZmM3Ow20wf7zdjp2xIxr4LI3iAGieea5PS0lV46SvzJFOQMsOUyph41tVV94rw3roY/qiVcmgoTEKRKklz46S+/uBwNbfSVJedMkB1528Bk4bLM3iMsp4gDQxJ8TjESnA2aYOxUzpkiurkh5FHHS6SFRjUGFyTvWWu4MHdmbPbMeXbRw3qEIBln7Ls0l5m1I4t+FG1Q/BX0gPwUFaNxmXEVZlRH193rVVtT4ab7DhIiT7rKriMoTr6N1ldv4m71+Vry1kZfwyT/uCDh6n7HAzYm2/cae7cvJFN4CyH/Uo1G3e0VkVHxldG2/ZwlTx7LcBwKoEgkbS/Ug8B1M3rN8jrRly4rjx9gsTiUfPo6WroJiCQ/jLYE6yi64Y1TKnFRhIkY+gp0+5GaPE4Rb0FFo8ePc1Vg3VX+a2b/VZ6BFzKDNt3/V79DqFrvg/bCPoKkJUuyeDPYvy7DE00FJeJnoYxVdIwSb4CKFWv3BFcl7EBt6SltGIUJvY6oEmJlkbnT1GD0pWvtjvWR+bd9tbzk4brlk9m+uaN5YA7wZRetQRngjGlLIK/ddKo0of6rcuAC1JMxDIaVkAmKNnYQJWOeKp52t4BTQEWo6GLkj73sVCFSXpbd2npNy1tSt8Q9Bb1MNX/XDyg6O035Rr/QOhcpStlcYZxjvK4oOLvjHXEsc5Gtu942vfLwgXAgPCOJ6mDfTjAwlB+moxWnvyz3mm/vOEdv4+p7ynlciyhEqTD6TfAt6i9xQnups/4ps4A/3QmXP7iIpn+7oaTr81MNq/hHviGAI3vMYbcpF1yLmWVNvUoz+uvi9cazv5Uj5S3/uDafdd5HPp3f7/svqaRskjTNj+f1/JZgu59L73ui97Dq29M73LwOg5+KEYb9kPPedArL2XtA4urKNR/1qdAnwKvEgVeCiz+3v/1y82vfe5zTFpMgK58M/EbQRURJ0RXmB0UYT3aOtehmp9MmszOYfgCKpj4FOk7IUeFpmVaZKpMv6i+MFA7AOc0yTZdLqbsoB2m2Mkiv8ufTMw+IJ4Tcz3qIJ8rj73CA5BqCXOeV83Ta8nVdzV/2VXm5hLfxE2n/Z28+JHgbYzkwzMZkKhXMKELKMZlbLgW3fGyOizAkPmW4S0r/zAAhCMy/5n8YXRYKm2OWk9Qm4PjzQOYx1UY8u2tzWZud6uZP95r5gEVo5zuUYEfyriZla1R7Uz7lXhmopCl7m35YVRSLpl/yyEjY8UIJ/NaVTxkyq2/jJDG5dbPNjXMEc9kdoqUwrqwmsv7I9r7mLCuALtiPsK96Qg8eIgqBkakMkOS2T4Bw73H3gVKIexf2qNIp6h8mCfBzNc9FMZdGccAfIi8tlRncSUfBlXvRDKR1kkpRPooeURiRN3MTGZN2w4Z41XsDh7iIck9JexqMs5V/ctV6hSNeppmdaOqepON74q5K/8rMPuTpHULBvsH3/4Y12uxr5Bhl5lUBejxU6QE2EmoehM1P8ph+QQYlXE3s1FWxMdg4LQ/mSOfuG9dmIMm7MuAYfM9pCOuspuue2ZMIZWY8STOBGW0Dey39rsCUNljBGBhGwgUHz5+2jwBXDyj7LrD1TZAMJ82SNsIhAGGxBek2S6Wz29WxlYmV3sHDaAFi7bBPFIL1b2UVEwBHt0wcBRL320kSat4d7pPno4PfkPDqOHMzM03c+yuvQg4W8el7ApeoCKtYPXfsWBMCQzgxCM2CeRte7kvx+u3b0JbmF/GnTXUqgQsqtbFKB+aPAfIKJGw39iWu9iz6Awi0k+lDvQPgRtVje2QtBT4OR5lEYA4fofuXF6/f8clpRv2JTuJ8f1GCjNfHBgocVQNT4CnfYt91THpBOcIsR+jb5tGWTyRGo6DpQ8UhwX8hr45qa/5mIf1cEyQdlWqZr383iroIGjKWtqw9GG/k3wxXjnx8RAwH3ABmBhw3xoXMii3Rt1KM45x4HCqAwe+LdWidBCwoNSC612AxW2uemhzbwsXAVQDLPmYlflxUIjes/z0CyqHz89/lTL7xudt7JbmbYTuxbipYIlXw9d86++kT7huPvVdN7nL9ylbJ/36vqZXf9e69epiXsa7VAfzLGFL3QzTK1WnLjVdrwlJuJ/6zGe6j7+r945R3Tpdziz1pEz5/umXAmnPJTy59Y8+BfoU6FPgRRR4KbD4hf/j7zf/9z/9rXZCLNKKAAsYIvW7C8PBROng2j38KRPKYFRBgzYWUbFhQq2MSgnj4GwaJtAbfr3Nb4fcHA7e3FZgcT54O9m0MY3TBu8O9g6Q5awDfkk+Az6R69WJvaRlmilAydNkSSN5W5heHu0PC8ZRsicvJ18YAM+oRMAgVGCh+lMXWITpFVRwauMggxNbByb7ISb7IZjMQfeHGJ1qtgYnmmfsV30fJmmdFeHDna3m2sFOs3By0Mw3MJtILc4wDD7GW9IQ5XcvBxogzP8R9wF2beGtjypIrrJOwgxNtAyZz22f2s7SwXpHqiHDKsLgt2odrvpqCC3RYiAN4xEmT6Yapg8ChKmV2YNrzGpngAK/YfNiTKpnKGkis7LPiv6R6dG3pJEgUKIWaULpZ6p2jEETd6XWHesODHJcouJpyd2IBUfTMNplrwS9JZVN6DJxk1zqTdoaeiuteACwePT0WZhxy2nd3X1b1RWZvyEAjnr6t3DdKpNum+rVSTsOz11sFwQpN2GUPwmwWMYTlN6LNNSWNusYMQsIHpOPOvmq2KRvpK1xl0qeSgCUWA3YD/gdFR4Yd9N1jww+tuzp4P4XT1aexSbDFedZpCSL8xhRc9p+Mnt6ftJeSToJvDRYl5CqXMnoa8TtnhcHGC5n40i/UfsJ4aVNUQUqm8EN8A3bth6RktgXoK+r/tVeRtezkzKi1Flm1BXto33oAtBzJ3BBm2BEO4x5JBxL11CdYv8EaX3//gfYXDwhzHrsGZRc+Z0U5p+VduotOy/jrm3LdVTNZNxPkZi6X4vQL5IryilIsj7Z6wI62meUDgkQ/QZiK4LrZEcYv3Pp8Rw7lZ7htn2VeKo7RSpBPaSL/UGA7fdg2yntqsDL9375frO2oXRJW/LbfivjpvQsZ+I7Bpazpq0KXJ5Rh6ISSnvwz88siw7Q2+/PciQvwTM0CrDwOWWgeqUN+S34s3/Znn6nLgQIVWLEDfB0Pws6BsACWxpA2SB9hM7He8CFklEli3pto8/PcS7Rrrext7jDuYSEyc0n9RBl/Up5/HvxsB+96Ci9ibeU+09z1NCV5jXu5bx875G/7f3lMNKlBKqp1p8Xf/u0puf95XRqHkmNNL3W8PlVyUBA29TjwzkUOuYv5e0Di5Cp/6dPgT4FXmEKvBRY/J3//ZeaX/3NXy/AgonPidIzTKf3rnYy4dXBtB3Ny0TncxizAAtmvqzQMXAWaYUjbTl9f+FoB+g8M4yTQE4vhflxvM59J6IDdspRB3PD+tDo/uN34YlLgISnwFl5JB/jdoFFykfQTNGJayqmV+JniigPzp+ZJxN/ndwFFTIAGjzLDIzBlIRxYiKPygvvZR4MJ3PCTbvaLMhgBZqJfggVhSFWEg9HJhulFc/ORpBYnDTPYd5O97abG2x4t8D+FLMYbQ9iZ3HC8yNWat13oNDWusFcUFaZtELtQgtXPmVsBRZes0pNOFeOc9Ju0iXAwrCqkVAXaXcIkyZTr+GxEgbBgbr9Mp7uS7EHo6ckRgZSMKHajRKLqIrwm8Vc6DYYXX319fX0oxewAfqUkhb19G0P+5p2F67SywTqDlQQpjvXMRhZpR6qxMgs7wO2XJ3W+Nc9JFxB3osOvSo/ZXVZBs0e4aq7Xn2e4KlIcOFK9RhMeJg46KvhrhIFpRXu1XATewAZQI2DNdiu6kh2ARnq64uLzVsfey2eoqwLHSu2FzLz37x3v1FtSgAkgyGjHHUlVuf1HuWpC159D3naf1ThUgowA3iCfYzK0GMMr92DQimIkoUFXM+6k/XNpfmo2dkOrsJnk0TKpfTjDBrLGFtXVc92kAipbnawU4BF+X6liN14EEdBAAVWqAU1qsBJD8Fi9j2xT9EmtrMMvV6GBljxtr4zhNfW4gxp2fPV1QAzQY4MvHYeS7jd1Rh9cWk5ddNF8LsYbD968rR5RtvFvsHvhO8gjDH5ymIf2/94Zj8SnAiGI/WxFxLeb+dQ8EhZ7V8CC/uxIMD28vC53q2iWpU+hM0LqmDb20g0aGPrpITHPqH0SRBDFNJgzKMv2A8z9hHWtKWH5fHDijSWzux3ETW79E2+aWlDGl1gEaBBelG9Im3TNQ/BRVGLKvnZR4ybRQeBAj/yuVAX7/3WrLf9JGqlKUeqmmeVfkmExyk/8bJHDePKGapQg4ILJYme9LVj2v6ItjxCmjjInjYTtKXtuQA9bwFu7/JNXVcqI/Ajrerc4vI4XEphX7JHXX1QXI78uTrAFU9r6LTVpfeX8zKMZz0uvO+WqxOmF95nl8LUlLrp9MK3mVhb39fn7WzBQ15YnjZcvbY/89p7wxu3DywqZfrXPgX6FHhVKfBSYPG//NL/2nz2N/5xJq+I8QUTLcBwNVH1KCdGJ66MnO3VFbQ8z7vCnLqCKDMZtaWMru0Q6zPjcTgw905/52l7w4Tukfe9Ybr8NqQpOHy3qSbueXyHbf85MZvIeXEtu/n3TtNpE3F+cSJPzv5o4xHk/OiU2QBO6p4yGjm5jzcbGIEeqIAxKJ5dVD8qoCKqK8SRYVRNwYl/GMPKnsQCYLGFN6gNgMUKbbCNPcIJnqAWABPTGG3rEWr4CG9HARcYL8PURc2MMsnkuPIuM1IqUSook6TfeoGF9zJiZYWX96k7F+sP0VSzEVhYN6ksQJTRV11FptP4YaiJl70iYKC0S3BvA3XaZTAtj4fqTAGn5Of+BLpWFVy4aj9OWTylmeFddZYpXEfdSaZZMOM6saBFCcIYzKYG4TLVbqrmXg7uHeDq9iKekTQ+1ihYVRTjKIkwb2khWNkgbaUAGs5PwFTqdlY7AdvOVXfrK4M+Q1ruWP3kyWqzgRqOdHF1O1IFgMcie1XcxLBbV7MajKv+tYn9x6OV1eYb/+ZebAI0LE56pD8G6HGFW4Nk1Xzm2bWazymSIEGYtHNXaY1o9yi/q/+r2BCsI2WRJsZ1jwyBxRISCz64GIc/AXzoIljJ0jRG0EqV9DyljUfdeFJ7BN3DZjdu4qWNqY80maWe09RnCk9Tw3SDMPQADTe3k4bu9mxbKVk5CKhDMkXcAIvZqUgT1rAHOYp0oLT1baQ9gorrnALCE2xoNtdWmvsPHzdPcNO7Tpup0hWXsDCuMt6q7nnaVjLPftiR6thP04t44IdKWHdHd/dv+3KRdBXwaycuDP8QHsQ2I2Fy9++yOzp9Fzo6npFK+rugRENvJVQxtCZv81dK4fjgfYAWfUVmPMC9HTt8nkUF+o3tP0k69g3j7ZOnbW9bCIL9bnzuGOl4WMZRf7fPeGeFs0BBuiXtMoL5LfUkGdCiCyysSfleGUeIZ7tYMZ9LsyNodAJ40HhbpxAD9PUR6qz7ZzJD5omrYzfQw9ZimG9rcoYNHqGFLmdfx9biFn3b3bhHKEOkoUm9lO985OVhPaRJvW+v1vtPe9Q4NeblNE3PetfD8DXOheeXwhEoUfL3ReXqPr+UR82vXs9LUJ+k25Z+2olb31rG0qrpfinzT/31v15ff9evAt1Kn3rtZuozy1jn8r4qVJc6/fs+BfoUeBEFnCsdL6ZYpOseLEidnf3cL/7d5lf+8a9mwou4Hs6nGhzmKrBw4GVCdIJ0FctrlU6UidNpjTAGa69m1BuEM6qXBw5kMTzkZR1wE9DATIo1ThmjS8QilRBYFHBh2vVIvvlhuiVP5kRuWiBimSyzdSCFcrkQqze5G9/1z+SaOKZjUmUSt+wBFEzQFVSEIYdZc18ApQNRgYJxcmUzdgk8k+nnTzHW5p0MrgzUsGpQXM+BBczdAMzq2XCzgfhhK0w9+yagCuU5zjmFStToiSpRGKTSqK6qq4ImwyioGZWZaKloLWVSJmD0plyp517DZY1pbTfrM4QLGCUrxs1qLGDH+gdUmjbhpJ+McPoB9BBglIO0SVcmSPq6Wh7PR+QzQjnKRohIWfgtQJiFiSkbdBUbA/ejMJ7MrLtUr8GAPkONZwsJRBgzMrHTKmVQNcoN7tyF+5B6az+gZyGZdm0EZOzMz/ZxBVk1PqUGgh0NqHdhxFXJkhHUvsEduGWo3fRuD7UybVRcnVf6sIZXIzcjNC0NjfUAJYhZwhbC+KrquKu4IEXQch/1pa+9824xLCYf+4ZlHoehExyoNlU8OM3HbmEIxk9piSAu71TZcuM47DQ0hFZCJK2l2Y1ri8lXNaQAK6QS7hWhhEK6zwE49mGelVboZlamVjoItgwfoEeDCm5llANmAAcCh3FA0zj9T6nJNExy3LvSFtJTV7N6/NoGpG1tI/mAPvZn1ctO6HsbSCz8XqWxO8a/dvdOc+PGDcDTAvEZcLAZ2dx41jzA/kI1qA3oqQrZDABGwK3RtwDykLw0qDYdQZzfmuOO4NDRQJUcT8cde52SCfunKn8y6wRNb5fJFlwKqIzvIocA2n4kLfOdko/AxrxsW0GAg6N9UFo6ziih9RAYK7Xwe7afqE7oYVj+51vRNW1U4ohjeQsA19Wx6oOAYwKGYUt5iOT40X6bxSatXbAxP99xZnzkt+p+tpmLBX7PphU1U65lDC1jkbWHPD1goVTCnbiVCZ4CIE74Dofoq0P0RyWk7myPOTy2FjxjQ84pJE3XAL2voXL3MfqFu3HrdjZSpTISJgeKdvXRvqivLWc9vLes3+64EKcN/O1imctVYWru3Xc1bN5Rpnq9qly1vKG5Idv6JI4RUqcSs31VfnTqaRrdOhkgc1hLz5/6TB9YFKL1//Yp0KfAq0qBlwKLn/27P9/8w3/02Uy+Wc2GkTwX2yuxcJJmWA2wUEzBwMyzqDvxvK7AFeI4/J4P6fCTmZR6hHPw5SwTo7dOtDVKeX4e1rjtcJ5kTbd9Yjoclitly0/jG8J0/eN7byxyWQ1tpxRelLi+SxkoqEbBxhFYyIgk3TZfw8hMy4yoppEJn4lfpj0MCxO5qlABEkzKMvmqfSRc4mFwS9gYbPtc3WdXEmFYvA7D8A6gDnUwPN5sI7HYAljA1jbbMEY7SC2O2cNiaHezGdvbbGZh6iYbmFJYh2NsLWT4Xf21HIIKPevoFtTD6gt4sroqY0EYmfgdAIArWdbXcipRkMlVd1z2TVUkGTKZdelgvPQN4sh4CRqsu4DJVXuvMnLulk33CKOoelH2sWilB6qf6FlIaYH1dgfvCZhZC6E6zQoSBQ20NzFWV5UnTCf1V0Lixnxel1rbBrKOhyGZwRjhWi7CCoYsb2UWzdNN6QQqpqE60wxAZRwdc1V3BFQBEhgYP4PR3drFixCr/tpVqM9Pk4YRvYl72ddQk1JyYF1tZ5lNddTXkDB8wKr81wEWeoaSAQ9NAUEynUotlKIIajSA1mBbt7yDECpG6NRrbBzVt/sPm/fvP0BtiD0gAAr2Ib1Q3UBC4g7bhpV51Z2re1RY1xFAwTzuZvcBW+7VoKTF1XrDCShkaiEIFNZmBSbYNgBQuWIvYBLkSRNBp+DCFfdILGCELbcG+lsw6gILPUy594h1O2Vn9x1sSQROUzCn80hy7t69iyteNtIjrT3acBdPZltcH2JILo22AbO68RXACA6eYBRvu6gWpr3DJNIMDdUdTwQHu6gwjdIAAt5xpFMyZlHNoT56MhMIqo53jNTEOI4jR0hJbH/bSCZd4KFr4TwjLaUdqp/5W4mGwML+nB3b/U6JUw/7vOCi7gY+AegxH+meb4eAVV3KfEzTcUXpiFIybW8cdwQ3AjzflfGDuhC3Agu/Xb8dv4MACwtAAO+ltcDGccTDtFULdIArY2iFKSYP8OB5PEQJztx9G6p4njLexEEEiwaGcQ/xU78rpYjYwlzDOP8NVO7eAqTeEVjQ1wQWfNUpq3mb31WHzylRDkOEDp2rL3pxTUM6XTpqHOvl4e9ubjVG0u/E7YapcU3/Qvg2v245i/poJ6FaJvKvZfWbkaoelqemWctfq1Kf13iGP4/pr87R5vPTfWDRIUr/tk+BPgVeRQq8FFj8zM//XPMPPvsPwxAeM8G5+i2YKOpQAgiARc4CIphdGVsNx29G1YvAopDHgdWB12kvkyU/wuP70GGXqzsvl3B1kiSs0X146XDwLuyRr0t8nwUA8CaDO4+TH89lPH2a5/xxPM+96Zh+HpBijcON5Sx7S+jNs9Q1ExzPBRVRtwhT2QIIGJVsFEdmTvyeMpxFkgFj4z3vYldheeRSeTagpALmOrtTM7nH441qCmxktTcEqBiAweM8Iq89aL0Ns+Q+DgM7m83IzkYzjdvZyTM2BlPxQcYKZk+phXWxnAKJMGHWDQZDRkpmXsbU+sh8ueJfReTaMqgWojRAcKEqWxgjVpPdBdq0ZYLSByQiGVlPaUgWYbBlggv4gMGjngVYKQURgAyykrsfJlim03LURlZqo7nqIUBmCzUcwx1xunN1VWeR8Ts+0g6APSZgsK8tL4VB39WwXYkN9RFcWTTr56qxTLXtMYfKjy5vPTVYXgDYLLADsUbhMtr2nxUkBI+QFDx+utqsIC2RIVQaYH2tnyvct/EG9fE3Xmtu4Qo1/Y941lFgsYKU5X1AwTuoQrlBnd+LRs9KUpRSTEwCGgmr9y/bfG5mtpkF3EwKKnluJtoYaIvw/gcPsru3NA8QQmpwjd21Z1FREbie0nkD8KivNggyM3pX2qbMSk6UtMQTlOWnzUoble9LtR0NwZdgIKeIMymYgA6CPdtFZn8XQCWYVH3IvTSUJqgKtX9QNivc1jDdvoBEQonLHBIcN/lTUuGZvS4I//wZqmTQZQOjdgGE6mKOF6pS2Xdktm1vPYq5G7R7dNhPfXcCw39Ie7nD+NgoNks8V22uAFnBPMwyNLaNBAd6sSPpfL8Cj3wD9Dv7gqBCYKmEVQ9kbqRoWD1NKVXgK0884xjeqypRMvT5bml/8xeA2q8FbEqDBI8BbbzPN04jKg2K/RFpeFT1J4Ga6naOpwO0n+NMCVDAhd+hfTz9Le94z/8KLM6lJba9Y649ncOxhX+WO6fPiH/Kc4HDEXkdch4BME6UXCAli3coynfK2EInbQbpXxMCCwDzm5wfpz11PTtLXcaor6pQbWlNPf0tN+0f873AUJO/zzzq1fsapi35hTR9f+EgQ5NIfG9Is1uGy2nUtLtpdPPu3tewPus+r/e+r2Eu51nDpHBtZnlGnHqc39Un1OP8tr07a/7mZ/7Gh55+tx7Ucd70a926efnMekSyxnfVV4XqUud78/4r77FpLj3rz73pwtz37+G384U/2WnuLo00b1zHprF/fE9R4KXA4n/+2Z9p/s9f+QdMWKyIMdE6eRUVA8T0cJkFWBRGu4AKAjoQOThTzVyJk38+4HCsdTpwinWVLoOXo64Dd6YJB3B/+8jJtlyzLudzjjYp7hj48sRb0ywBkndyPQ+r1nXyJEgvjmnxo04M3ec1rGlajqz2EdM6Sw+PTPBMssXVagEPUVOgXtatMCP1eQEW2lJEfcrJ34pyKrGAa4lthcaUowAJ1Xu8HxhFT3tkvHkOoNgaHG120I+WEdXL0x6M8x7AYnBvq5ngnDzcbcZQhxpGcnGKxEKjZ6ULZpNN6GDOVDWRSYs6lnnaBvyLsSuMjFep+v+xdyZgfl9lvT+TZJLMmpnJLEkm26QrLVootEKhtaBwFbBSyy3qBVzQ595HXO/jch99XO4VvSqCyxW9+igKLXCLLJatUmQRWnYpu9AmbfZtMpPZZzKZ5X4+7/md//wznYQGXGqbX/Kf33bW95zfOe/3vMvJ6h7ZmNVVdJlt1UhcsVffX2Ah42YdCrCSoTY9V2FltCR0BhIyyVCUevq+qPkoaVAtx7LIpBlWMBoSBphjPUtNq57FT5K7z4ar1BoIz1J3N5zL+ahO5YZw2AfAEGmwLKAQjKi2lIFw9mJlPzW/Fpj7UVbhXRlXSrIZUOKvHWZKxlHanUAFS1BxBDuJw5wn8KQkw2oZpZFMbTcrurvYMG8L+1voGcsOJfOp1ONEDVjsDRsFgZlSnC4YeO0/WlD9UbnGNWlVoNazeqyNRidlcKVeQKYU4oE9D6a9+w/EztW2gXRS7UqmT6mEq+2mIcMqaJD+Mrgy7KpBaeytobI0kbbR33ln2/lrhRbu1N3T3RmegAQ3GqwHqIg2ycbjsckfjLBpzigNoB/OUacZyjiGSpODifS1O/eyB8eW/i1p2/btSFXYNR56TSj9AayNjI6E5GkESZCSBft/IwytAJFioaJDX4Fhb5DZt0+S7iJ9QfsOQYteoVytlw4y70qnAsBTF2kcHuhkyvlJQ9vXvmbfESRl4JFdwEoLv1nBS7bp0ZsX3w/1kJYkF2WKMAI+8opvhnQyyODbIZT7bQTQFiRUZbJPy/wLPgTpebzI5RagKX1TxUx7lAARxPWjgXwR1rIJKgKwSZj4n8fEkFiQtmDcNoyx1sJyeM+fKHg92KDgGViQA3g8zaJSOVcBjDne5U30oDnAokE1wwpYDGDLc/HGDWkH6lAFWKwmnuWMQsW5yre6PtepjLeGibJyrooe0XK6cUmA6lxO0kBarHRY53IsC1PLp3puGtbAuWn5UUvf5Ja9LunUx6mFL3laRiPWl4cIIVWLiCTM//qkozyEf/EFiUVQaGh8Lv3T7ul6Mj/sWrfWN35L68OePxYevP1jI4wnAM0bOs+rOn/2nhPM+YvpZ1/Yc17xvpnA32hZv5k85UV/5Q1H0nVPaEk3fRsu4C8cjyoKnBNY/NGf/km64+1vDabFCSp0l52wnezil0FFjKf+qX4OmPlSRsch1nOut2Ot47WDuoN0/JiI46n38Zy7eMaZZ14KLIwbaZuUNxyh0sSld8Yth5N9LSwPA1jwWsCQp25D5jClcFURI5UAFqYYCcs8sHpMeDt0CRfAgQk5mBqYnAAYMC8ywDIfMj6uJpf3wZBYGRLNtiT52s3w1FlfJRMC45ntBnD7iArUwpomNsVbn4ZSIxKLNWmaFUZVmmyPU67IT7P7NvYVbacnY/ftNRhvLyK5mGVl3xWmWZgwFSxkxNyrQmbYXaFlsqRttCsMjYBCRsqpVnWRUOUgrPsV+HNFXfWlCRh1bQ9UbTK8dSsMlMyOxFGqER54YMIlX2yYRx2V+phneN4JKQhp0p9cgY+VX8LItLsBY17NnQkmz1VkW07mryPsHwAXgJoJwI2qMfaxUNlitV2JwFqAkDTWy5R7fcRu3qqgwADbb20Hd1cegeEeY/VbQLANA+OdMMI9qDbZV2WiTwIM3H9CYLHv8DHqPRl2CqZhvWRo2zEe39rPLtl9PaFKFeDHcsKou9eFO2bfv/uhACX2O2npRm+9MPGCIVe59UwkmFlkIlGCtBEPU12sEMsQj0Hrr96/Oz24bz/pjQawt/wy/QKBbhg+JSDSPdRhSMt2V7qiypieqLTLyLYltLPMa3Tg+BP5CiK6utoxPt+I6gugByCgQbnfoO0jE6yUSEZ4ApoNnUSdiY3jFimrEhe9TI0ifZDedm+Z+P7tW9mhfHsaGNiJuhIb2AFIThw7GntXuFGgtisCQw39BYKxsSJpaRTeIJOPfY9qTTOAmFklGDDuSjKDOSeT3HezOpE2DoJlgUmw5jSgfcJ2su8VI/tQBaRfCLAc+KRXfOekJ00FytIpjLupi+2iBMW84lumXAKZvJDA4gF5wl+HlESa+33YL/x2BJCqu9kvjSvNA3Bw7bfo92H7qAY2C3D2m5HOfo9C/disknw9fFYbb30f5WFcIR2/G8tXxjRj+N762MLGDabVhuHZgvUMMAGwANXGD6LNEWcBdagFv3MWNVYBLNYiQRO8DtDPLsXzWAALwF6oQpGGtIsKk4dliFvLwv25jvr3EauKG7GquGeklzts1W85rZB+CW++9enUlyPoVMVdKY0StryrD1/eea7Py/sSnoulMlqKql6GiXecKirlR/kmXxPXdC8AiyBHeuDQqfSWe9hfqDpOgYRnGR9b1rlolwknsPiFW3pLkEft+cNfnEgdzavTky565FKE17zjOGrMi+mXXtR3XvX69wAW32hZz6tiywJfABbLCPIouz0nsHjNn/xxetNb31KboAQWsfoboCKrPMU47aBIxVSziYP7eM6Ng27+5xsYggiQ/ziQxtgbZ98xpXrNwFHOjtTGKcDCa7PxaZwjgTxgO9EtvSdMXWZ56q3SrZ5HuUrZI6bxcyUsTS06efhUEaMTvNeWTwY8SyUyiHCVXyZiDUxR2FvIoMe97xkQYXz4ExO8E3+ACxkAGSuYBH3KNwYjjwoIuv5pbXOoQJ3EaPvIApvBQR9VF5pd0aXcp5FMNAAq2uenU5eeoXA9yxJ9Oo3dhTsfawvgimfWRYfZgWlTWqAdhMyHOtxlpdR2CrULwILAQ9UNw8r061pUJteKm6ZeoNT193AFV+bJenpIH9OUYZaplakXLKguEnQjXDBcpK1qie+luum4qitblRkp1Jxg7EwnvOm4isxAqySgDTUjXYPOnYaZ4z0ZBuiIBoOWjTBG2kqYpi5op2FgJzEY1n7EtLQTWYVu/jiMnQy4q8pb8Vi0c2t/2ratP+xQ9MQ0chKj8RHc0QIsHth7EFuLiVCTsY72DdteWnWjkiQD5k+7BIGTrlE1QtbzkcAiPDVZd0DA1i2bY0M9N7abROI0CXNJYrHqLS3aWtsJ0xduaCdhwr96/560G3UobU3U43ditT3Mt693I7t990S/k5Z6jVJCMUadx5FSqGoTDKuAmDa2LZRsRGPSB+yH7nQtkFGda2tfb+z23QmwMD2N+QWTqg1p7O5+EO7/MAkzPEv/ts/qKGAcYCHDLthyn4qBXQNp+/ZtacvmTXCtSFBIZwpJ14TtQJtM4RY5QA9ub6foUy3Yl+jitBH7DV0EW289OZ2iDnMw3RQc0CjQ5UfbK6mSGbcvChiUKDBwxKJH2DoICGwjaC4gjf7M2bamytF2YfvEdyddBBQBNsjH/jcGoLHNBBfSKH7mwfsamOa7cKxSimt8ohFO4KBqm7Yz2ibhSQ06ewQI4Ow3qVpe2KjRFtoRKZkSAGof4mgjeFlH2fM4QR60ewAfzjE2RdtlaWGUjbpnQMG3RDkDzJCXE7D1YbCJtGrqUICJWX4IS0J6oXrUAuVeABAt4PoYkV4Ai176xUB3R7oEYLEdUN9Ge681LdL251Hq55joIT3rz3Gzwh9Dl7Dxuopfgpb0yv0ZYZfFjfmiLr5hoxScSzqeSxp+w+Uo770v78u7cl4pjGGXZglyK3kaibwyNbjOBclJRRny5VIAghuGd7fedGGDvIo6Z5zuvm88ffDz4+nnb+5N3Ruys4QzAjyKb377jmOpv7sx/dB3dD3iUsYQTWjZhfM5/j2AxTda1vOp1/KwF4DFcoo8uu7PCSxe9cd/mG5/yx2Mi05O1STlxMtPkBFSAZ47KJZJpjZZOFByxADPgFk/iOaBNsbRPDA7CNf/mMjzPQkQNSYx3uc8cllM3hXwKhvH5MjCMDGk+8CjnKr3MvP5sFY5tkHyr6RXTRj1EwVhnIoqWBH5FWDhKma2m4ARYFIvUgrVO8okL7CQCWF251z9KEuAC9/J7Mv4uOLOvhWN61vSXCMb4qECdXQBA14YgAlUFygSm1TBWFKSNYvosgMoOhb4zU2ltUgrGpBUzMG4zbAirGoHKeOpKOvsywAEwCF/q+4qqwxLMJu2p6ofNLSqNU14ZVLXvlHmCEZdKYqH+vuCC5khV3YFJzJPSkKyjUQECyZNxlEPODKctqc00lhZYBESE4K6gZ35S1BXl8OWBSbHtpGuMu+WUSbvFKvkMtQCHvOTtgFXYOxCRcad4EmzgZXXFo1PYe7tm9plTMEcT/IbxRZAHf5ZV6dJ1/yaACG9qHts6e1N27f1k25mlkdR2TmKh6P9R46mPQcOp5PUR0PlebmxqnwyqqoRaUjdv6k39lEQuCkZGIGRPorqz979h0N3X1uODsINoDq1FVWhVurhbtHF+5Qr2AL3pnVZCtIJiHK/hz3sg/HQ/kPhGUpaCCy0nxDI9GHj0Q8gcIXc71KJghv/aRTtzuLj1Fkm1qO0c6hMcW+flykVCFmHTYAUd7feArjowhj+FKBLI2tVwgQ/AVAoX6hWTQM6+CCUWGjPMk49ZZj1htXT25d2XbIrXMwq0VlEsuaeGK7UT+kRijSHAWwj9g8BCq24HnBjWnOquCGlmeBdOBJQMkK/tJyCA3fmNh37ntIE1Z60HfLDUE1MZj3GJ+lk/fjGAlzAECtZKz/BcAsgxvRsKwGnaQqKTUdvYfk+M6B0ExOLOiodkd4CB/ubErY5QK/ftaDCn9IEyyyVMwOaQr3N/O3T5hMfMyH8llQxFAxOCnzpX6rNWE8BhukIDsKmiHgBFLj3iG+a97HAQXjjRP58G4bTJsw+FQMW5ReWKP9TagG+rn4+453AQokFRvfox6W1bGi4SWABqLgE9b1+JGz1wCLGTAhTzlGgZX/OyqhX4WJ+qItTz8BHmcu7qr7l1nN92rV4VXlq6RKvvPNc4tTTsLxfnmZ9Xl5bT4+gfK08S0+9yq0SwfKfWrjq3QqBSgpmcOsFG4s64i1dngtYSOLjI3NpcPQ0dnJr0qau7ECkxHbl//DQ6bStGxvFmfn00NHZ1NdJuM5G5oCzvyvxRybm0+FhXICvXZU2k/Z6zsuPCEMetv+OPhxfrGehCJWkobG59Fd3D6XejjXpe67NKjvm7dw2OKor64bU0bo67Tns+LOYLt2GZzqenSCe9eqpA1GmfYQ8TqAmtg5pjeXf0JIX9Ep5HimwmDq1kA4MsjBDnp3kv7XHBZCSigsOS3Tx6cETSPyJs41wLdSt/lhe1vq4zuP7j8MvsIpx0WbGzMacyejkfNpH/t1ta6ApUuq6vE3b8fTA4Ok0NsW+VE2rAGZqWiwFelwCi+kD6RN3vTd99HMPpkHWIpv6rkjXPftZ6Wnfuj1FN2EuPDU+nA7u+VL6whc/l46c6k5PeO5L0rN25NZaODWajn7u3en1d34+jazqTpc9/0fSjz79X0dl7pzA4nf+8A/SbXe8OSYoeT8ZEydtmWsH5jJmyrzZ5KXZ/QBqL72Ml3UrOPGePzyPOMFgc+2kENec6eVxbzACxWqc8TjM2+nefCKvqlf6YWRmAia2eubJPHLa3njHQUQ7fbzjtnqa04sAVfqlkrzRQLZKjMnePJzQOTOxW94AGkzoMh7xk8HgPjPzldEng4YMCFxUlMm4Mj6qQakGsgb7ijUAi9VNrWlq9fp0jH0r9s+tSocQBU/CSMFNsWKI4TBE37BqIW1exWAKsGgFWKxmFRguEvsKJAowKDJ65q9nHxmPmGyritqWrj7LFMl4+E66+lovQBomt2A4qzGzuzfLFEscwYpG4TJirrC64m/53QOglbABoKCxjFO4ciXMDOWh1pURq/sz5I3O7Etl47VgfiBL9DPyceCQCbMs0liVmRmZTFrYuuj1KAy+YaSoaNhSuKK7ALgwDTeca4NBMozARgmOwGKYPSjKfhgkFMzhBlzdategy9N+pAmqI8nAj4+NBqjYizrT3sNHYKgBbDDUbsgm47eKtnQVeiOSA1WhBrZuSc2AMD6SYIqHATG6fz2AGpUGz+6H0QP4uHhgRzDwGkqPEyaXSa9X7gTtpnwYhRNuAzrtcwCzQ0eOh2vWY9h5+MHqZnQ9DHy3wAI1lT5AkTSVoZSOpncMMHD4+GC46FXSEG1PmbMdBgSyRWhs+5800lC7r6czbe/fDEDqS90YXqtmpj3EYYDVFODMNjW85RzD3mQSPRq/w1n6wDj2E/aDdtSotmzLalAdSCFW2wcJIxPfxb02QSOAiuODbBiItGKa8s7Tlm7gqHH4OLYa0xibnwLIhBSAFi/gW2lFqDyRnvYMgkLzFIZattP0jxgbyNP62pfpOvGdGi8kF/Qpga0eqNphlO1j9kMBo31R8CSg0J4mSyGkFV2M9EzQs3QWmOg5y3ulJ9qJKN1TghUqgcQxXSUeC3LwHHq2sp1CPQo68j8O6x22S4DCkXGlORlclLHD/u5hXktqqLmOvvG9353lEtz7ky7SQpBkObiMcVibGNWewiNUHbiInbmLKhTfziJqUOtoy03YEA2gbncxAMMduANYWHby9SeNC2NezjyqHUG3cheklI7GyscZ73l0RholnIXnMGz9+/q4Ps+hbO8lAFEfJ8KQhuF8XuKXuaLk4flshyUv8XKYpbpYglKGeFeVmxhLz60TgQxnzFLqSJf7W7/35oj6b/HHb6bQs5zr8y30sv/4cyHK30Y8vP1bH2cDFidh+t/8jyfTfphUGf4ZGNiNMKsvfRYbh8KweqhW9VfvH0r/+Zkd6V2fwq6LMP0Y/f7U9/Sc8502Dnd+cjR95oGpWtoy9LeQTr1xtKpOd983lpoaXQTjG6Vxf+amnvTQsdmk/cHy4xXP7w4GXRDQhooUwvN034Nwihw/jFTj8m3r03KAcPTk6XT7B08GqFAFzLLJgzzvmvb0DOwMyrE8Xnlef373p8fSxzF8dnxQ5VIgsKmjMf3Yd20MQGTYQrMfenZXuuufxtJxQJCHef+XGzvTZVvzQqPPludZ4r6MuO/9DAtTgCSPZtrnv353d3rgyKl0F8+d4z2uoL4vIaxjtcfnocXffWI0wIj5FRW4lxJmJ6DN4/EHLKbSl+78m/S+/e3piddck67sT2nPxz+Q7ju5MT35hu9M3/6EjWlq32fSvfd8NH12BGDbwJ5fI23pihe/Ir3gIinGnHZ8d/rYHXeniRtuSk9ZdX9679uH0g2/+oPpilXj6dO3YUv93O9PV/ew6JunGyN9w8c5gcVv/8Fr0t+8+c0MpAwsdEJ/MiZxrgbNqi/kkbIqRhk04zZGzxwqzxMlxlIonzuIyajHIJ8fxLN4TkJ2uhjouM5lyVkGr094U/Wv4fxppJnTLNdQK9IlYFX2XILcuV3Z9/Au/5x48hPPcRnvSJcMTLuACj/wfJ/tKWRiVMOJ1cO4zipSchIBKjibngyTTIGr+Kob6VNeYNGAtGJhHcbFuJc9srA67WXB+QiMj8w83B6SisXUwWDUu3Yx7eTXsXgqNZ6eCmnFAgP/PMxRXv3GAxLpu2pcVkqDBuSuTYXAIut2u+KqjQW0EojABKpupIGwEhQKXa18qjqVVZ3CDoK8ZMYEMHqOcgdpGSupZceaYTVaJklmzQHMFWf3ntCYN5jgarKSqXPV1v0lZBhnYRBVtVIaIMCRUZI1UwFJdSlBWTBx0KwREGDbyZydDhe7bnoG20T7yKC7+R4Rwh5lFKAQEgsYuCmYZiUkrtb3sDldJ4yUjG8vEoCwJ6Fs46yc7z14KD148GA6wA7RenYSsJ2ivHTCYLIbWfHuxS5jGwz5xQPboXVLeMwRlLhZ3cEjg2nvgSNphjIJ0PoAILvwIrUVmw7ppeQkAA97VQwDQiZJew3hugArMqAzlFXbCu01hvnN4851lV6MiNuJaoq/jUg2BEe6rLU+YwIaJAj7UcM6jItagVQwBzSw368NLX0ECfZdJTYbABY9PR1pS083kove8ORkG57Ee9MB6h/tTDzbYgiJyEnAxTjta3vpbWqKPJto2y4AST/1szw6ClgAhKlmFdIV6KS72THAyhBAZJIJTanHKfrUScrrbwQ6KH2S4ZK2fi+2uxJS86a42NfAKNOqIQEDGChFyz/sdQhvHw97IcLmMQCbGvqSILUFcBIulgEXAmfBhmE0ds9qX0gOqNukwIJ0fZclPNLNTO1X2lBU3tQsG+VZQzoCDfu1oNPvKexnALVKVkxDaY7esQRxkbdhKWMAKtrZdjqB/YoSNXeuJ+l4H21F1tWA5BWXFob3nE2jLFAIdC2D37EvaotB5G+ckFbwPbuPxUJ8165Oshkh14sCPL6ZOWy7+HAw3kY9DvCqV6gBbHC2AoyK8bZzjz8PRj+LGof08ijl89q3tfsS0Bdf76jSInItZLRH3X3tBRcRKuhx7kwMZ3lKmUyzPkZ5Xp92/bU0z5lxImIpXSluhM2ZVNEysJMS0Vi1xGi7qi7VKb345u+rvf3XvviPDiwk8WvfNci3sph+5DvZ04dlW5nav/v4SEgCfu6FvUHuwug2wqQ+9eLm9LTLWoJplUk917v3wIB/4quT6SWAFBlpmXufPYjE4xe+rzckDa6o/++3HEs3fmtr+k9Xt0dX/drBmQAuTdiDjE8tpNe+ZxAJSWN60XUd0QWUUNjfZMgPk6aA5NbrO/huGwIUtbJCv5xZ/8q+mbQX8PT0yxn3iT+N9OD1H2B+QYLxy7f2saCVv8bl8VbqQ//vIycDGF22VXfwDelLeJK6/cPD6fontqbnP7U9ohS6+P5plzenG65sDenB6/9hGN6mIf0ith+OxR7L86yP+4wrWtL1/PYjfTAPbWQYbtIt0GLX5rUBPARuL3tWV7piRwYrH0Dlzfo8eRfeKOFylez83/eeiHr/JGDQ43EHLBYfTHf9n9vTA73PTc959lPT5T24e//SXelt9w6lliuvTy+4flfsZTaOlsT04ng69MWPp499bDwN/EABFlNpaP996T1/uSdd+as/kJ44/WB676vfnlpe8UvpaQ/9ZfrT3U9OL3/hk9kQlTkpKPzN/TknsHjlq18NsHhTDMCZmadHMEGdASzKoOxXTolikK0fYXlWG7YVbZRiV5e+c2CPyTPSIoI91uu49zanEJMKr4LJjIkh51cfn0XkiJ4lCjCixI34TJxcxpEHcSaWfBd/A1jwwKc+N4/alJEDxnNXSUvZYkWxYs4ykwZIQHUhSyuySoITveFkgnL5LWAGFuF5CSZAg23tAnQvu0qDbWwrZtawD0JDYzqM380DM3PpOMzODCvmoAFUP1LaiFhwEwPQTvj+zlUwrahELcJcz6MC5UqzIETwYP1dRQ+mrGK64HL470ZjOZwAQDUodcfVX2+HqVDvPlSJiCOzqycgjXczAEEnHYbYNBzOrJ+r3qpO5UNCCl5cvZF4eZXXMqhW1YFrU/uBK71hEIwKSLj4FMCSpp3SzdyM78AmIBGoCXBsF9ei9YzVDGBqgZF3N2fLomRCcKF0QtsLc3ZfENvMlVsNhke0QeDjcyVZVZYN1LMfZrq7Uw9LG+CnWP0hvMyiKj6Hjh4NcKHxtipAM9AiVqgpmwBTQ10lFv3YRFx20a6QHjTBxIa049CR9BAqVPsOHg3vR9oI9JLX9q2b46ynrWAAqdUcgHEIYDGKHccp6tIE86mbUlfux0bwwjWGahAMpy50/YQ0SA4mFWZvAwyrEhftIjoBRxqen0BqcYAN6PRmpWqU6lAy36r9yDi5x4dMtXR1k0TVrjb1YjROGt0Yj/cALiSghtZHoIF2FuZt9x+mPMOUZ0SgBY3sb6cBEG3QsLO3J/UBslxXWmP/gKnehPRlE897uzfWXAZP0ld1V5vd4Y6nA3v3p0HKO0pZbXOBcEuLbn+ZRChvSLAoM9mjLgWQ4Fn0LApkn7Efut+McT18ZhrWT/VEv0/7n7vMC5wFrHre8r0MlvXQuF8pnHT2LCDwsAuXBRXBqKCimTSiT0Ze9FPSVkoQwMYIlE4664EqPHRBd8vjJoKtALAA7fQTxym/Q/N2E0n3HNGhgPQxbq4b6VPPPI7l+lkuj3JnuxRwEeUAfIZNly+klXXgT+xoTqyAboALVaVmAXi6WVgApC+ysDEvsNATHH3QvrADqdgAfXwbErQO6w4NWNdYyjtKku9jjDOzr3MYbqUjYpb4JUy5XyHCuUDA8jxqparSq49bwn69Z0HIWjlyio5HDz94x/8CHHxfy78ucI5ZpUO5XnzzLXVv/3Uv/6MDi/sPnkqv+4eh9NIbu9KVO8vck9L7sceQOf2J5zHW4oq0MLqujL9smZ3D2d4p1XglthHXXtp8htehB1lt/4v3DaXvAkQIJo6gIvVH7xxM3/u0DcH0r9RiZ7OxkCE/MDRLOXvSVmww6o/lzHr9u3J9H16z7rjnZK2ePn8k8Ur8+vOr3no8dbWvTi9/bpZGFbo8E0DxAqQi5fh7pBdKaP4HwEKA5LE8zxK3HqgY7s/vOpH2Hp9Nr3j+Un0FZr8NMFse1vD1x1vvHUmfQ5Lxmy/dHF/b4w5YsHvZ5+54bXr/6GXp+u+4MT11BxKLD70t3fW1xnTpdc8F1PbFmJxpdiLd//H3p/cDPHfUgAVu5w99JX3oDfem9h99eboufSG9+TUfS5f83I1p/198OvX/2A+mZ2zCxrGaP+tp/41cnxtY/P7vp9e96Y0xIhaXhw685WeGDsh5cGRVqgCHMiFEgFysCLP0J4+y3BvbNAqwiOmJykW6PjePSCc/8+YMYMGYXAMRxAtgwTkmWeKvOCETx6E81j2rScY8/Hnrc9/7869lLIcrVHJXpp+BRWYmogzBWGRg4T4N/gQWMo8yocYLxpiz1666azOwTnUjGO41rBw3ACpO4QlqAjWo42yGd5gB7iDM7ElWMmdh8uDAWQkGWIDotzStTgOIUjeugRGFVdC2YlaGTWYPZkVmyKzWsRIZZeAm6iWwgJlS1UPJgDtuW36ZJVfRZbBlWvVgJIhUF16VnmE2qVNNRGZHYCEQ0X7DlWCZLdWT7FD2D8lkmkoYfL4OplBGzNVUVVFk3LW/cOM4XW6GNCaIDBMZjF4ul9cCC1eDZZhinwCbgOtW9MD1XuT+D40ACPvoLAz6hKgdWuitSOZQ9Q91y637GKvCelo6heG75ZaR3gEo2NTTk92i0q6uVFsP2137goOoAgkQXFHODKd1tBDWT9eyraEKdenFu8KzVAe0E1jsfmgfRtf7UYU6HnSzL3SjtrQZRls1IYGlQK4ZkOBvJOwiWM0HXDTA5GnQr+vYqVDBoi7Q7DTtJU3cFd0dssPNLioqG5EQuAt4F6orenAaQtJwEFCh1MRyT7kCT1xV2WL1nDjrYXIFNy0Aiy7Ankbg7aQZqmFdG6kekiCkEe56rvqS4MJVeDcKFFgMY8StLc+cxvAw9Rt6ulM79Wu3buS1FiK1IHXQgLsX+nYBPPye7FPaQij10r7kKLuT737gwXQMYDEBUy3gaWaPD6Vm2hU4iRhHMCThtfHRPkZ6yBjbz/zWBEhKCbmN8Ep8pLkqawQLpl8pStCbPuheHY47fivWTeZeSYVSNtvZfkB2Qa/CcNoPTVeAIIhwpBAAK1kJSWb0mpyX31z0P8pqHTKwNr7fSwY10tg2Fvj5DRRPUbpBdo8VpTd2XHWxM3DKCyR+X3EQP675k8dM3pPv0qIHI6h9lcOyzkJ3hpTYZXuxAhanKRuLvrFZ3qLurRmHBBaN9gX601badAA1wR24Nu5mjGilTVm0zPkSr5TFs2UotMrjQAkYRYg/hjnbURWVwtauImhJsz5ueXa2tHx+Rngf1KUbOVT3JVx9muVZjkZowuY4JpzHuKUkl9Wz5MO5vra19ONhfpP/kjZhLwALKfrwYyVVKBlcGd1nX9XGWL5E5UHsLT51/1R68fWd6cl4YiqM7nIAYi5ne6f9wWth/K8aaML+YInp51OOPJ96SXN60TM64rv+Y4DF0Ph8ejoqSdchDelsywx3qcW5gMXZXMMuZ9ZLWjLiruBPIOo9gLTini9PpB99zsZ0aT8rjBxni1filzPDQNiNjJOeZXjfZ8dCLesnUNPyOBtdPgld34F610++YAkcLM/zbHHf8tGR9PmHptNvvWxzKUacf/n1DFx7pAAAI1BJREFUh2mn5lBVq39hPU9CV2077ntwKn0NadQrARaOhY8/YLGYpo59Nt1950fS/cfYbJguOTNPu1/7jPSs6y5Pm1uW+mhKKwEL5s3JY2nPPe9Md312OJ2G91y37dp0dbo3fbXnh9OLb9yU2hjUl76i+pY4/+tzAovffNWr0l+/8Y0x4DmgOvCVH3cxyNYPvoziUYL8LIbgPBDXlSvelcmwem51chzi819GwYm6gI0MLnwneCFnf5RFZoFTxHWlMlw0cnbC97eG8CUNsypDfI5r+XMdfEeSzhVxxPvqnY/inW+sn/+rPGSsi3GlQCNWRmFsNN6V0dEIWkNg62O8WHmt0tW9pky2qkYaRq9lMm/EtmJhLWLa1dlo++D8qnSYDciOyuywkjsP8wP3gQ7cYlJ9dDNGVJe0rkVysTq1rUatZEIGMm9kJgNZiizTJ1ud60U5eCeocOVd5kmGXGNomWxdmHaiGqQEIrw0QWyZrGEY1cETJ2JVVXUgGcwACa78wihZX+tpe8j8K9mw2gGcSFsVnVghh2YeMocCC1dpZeRCCkBc7STKBmeGCTUTlFBNR3rLYNruSoBkDLUz6IEJbm/LjLr10fOQak96FXLFPVZ+oUAjK7EhJUHP/yQen9ayqruROu/ahl0BRsvWW0ZQV6Dh4Yhya0A8CLjYC7A4iuejk6j8yNBGv6OClkMm05XdS7CduAR1KPe2ELzcv/vB9MCDe9OR40OhDiYI2aAtBwy4Ll5pElwA03Yw+b143pGpPIG6075DRwE+MKPQyS7nDtLmt0rbD6QGcwIEfs1sWObGgF3sitwLGOwCXLgaPk79BwGBAothAGnsOwGtpWcAP2aVZvJ3L49mQFA7fbAbOuhytw0pkJIgf2QdYESm+wQqSieRgqhKNhK/vLquy2MdD7hLcwcSCe1IFugbSiw2UK++nq7US/tI22ZsXpTK+f0ITk/hwWxocDDt27c/7dl7ADqfDAY7PD9Fv1JiEfA/CGH/CkkSdbf/OrlAlgC1YeND3QUjSgbse+W7M0yMFc6m/AQfStdUyzMFpRUTAFzBl96ZCnOvmqDfit+N35DA2O/C8tmXzUMJkP3dfhCLGYwDNqx5218tr43oeBYAGaBj/u5b4iEdBDbWx77rz+9BdSjBhd+ZlYzxDMDgOCfAMO8y9nqOcc68qzEqxiMC+dz4jpmUBIkFEwz1cR+LedLxe5KOYcgdNhYsbjRhSE/7r0Ya2IQkq5f+vJ3v7GL6WV8LEkfqrsF8jMuka/rl8LYc0s2xPTpSeci5lNtHEcYyVkckRbxy+N6jPk55J+1Wel7eLz8vpZrfLOW6FLLk55Pl78u7nE5VLspXSzfqUcWKh7U39Rnka4JZtZJHGZ8vAIslUtVfrQQs3vWp0XTvVybTACpNMpv1h03xvKfgkIKJ8myMruHP9k7Vozd8aDhsD9qa7elnHk+BES7uY2V83/fZ8fTZPUr0F9NTULe6+enYl/mRcvxLAAuZ/ztQYfpn1Kw2sJioAbWG1+73cb7A4ot7p9M7P8niGoOAalUaix+Did+MutbXAxaf2T2V3oob4GInYv0eKbD4W+IpdVgOLNyP4irUnm7FdsVDQ/w3YTdzHGP8Lozx16OdMYKx98QMUqTHLbCYSQ9++G/T3395Lm266NI00NuQ9n/x8+lgw/Z07fXXp2sGlqRKKwMLxpsFFrDGj6WDB7FjlLkY+Uq6+6tt6TlXLaRPfWYPzoE2pCu+43vTMy9hrq4zlI9GOc8/5wQW/+v3fm8JWFQD6BmDq98Nz2PyMGPuvfZ7ckgNAOBzjjLElonAqCWe12VC9GyYMlEuSRyqMJwysHAo5trBmfD1wCK7emUSZuI0Pv9z2Cq8E5ITam1KMAzv4qgKWt56W95F2QmbmQYYDcCBBtveB2PhxA/ToD63tgb+YlWTRPReo1/+Ai5WsVooYy4TFMCCyXwtE/ni+rY0GcBiXdo/3xDA4piMLqvW8zAaDTBBa+dnUycSij4kFQN8eL3rWTUHbc6hJlWkFQKLUk+JZJtEPSi/jLEMi7YPMj4yegKLTphUV9Rlkl3Jtr7SSZsHV8FH0Y+X0daFaZEwuCKszrr1JHjUPxgVmC2prpSheHKSLoa3MUIHnfw1LDbtbDzLqi31y25EM7NlHTSGNR1tUlzBluauXrfBGGpw3YsxYffGbtSSlFrkXZVHYeyH0eM/DhhSjUjJzBr0x2XwZBb1+NRAWm2swO7a0Z92YHBsvS2Xq/Qye1ZIgKENxFHAgTYLR3H5KuNnPtJH+xh3J9f168C2/rSL/Rs2VlIDN7YTWBw8Ohh1sn/FjtYw8W0wbOthstez+qtqjipZruILAvYDLARzAiJGA2ha7W8Cs6ekaZEyzc+eglHHNoI2036hm/Q0Bpe2QwCqE/yGaC/dz44rWaDMfuyqQ6k2tg6gF6CC+NpobIKOupptQwqkxyP7tOpk0sH2kZYahQ/jaUoJyCj9QOmC9NJFcgdgSXDhVzlJGHeyVjVLI/Q+AEfnBsAqacvwmv880ooZ3CIPHj/OzuL7w9hcht7vLcYYL7izTwXDzrflWemAAMmzjPv6kDxktaagrZI2JFH2d78FQYNtT3NF/3elXWlBEzSX0bcvKC1wkz+Bbrh9RTVvlJ99XEbfw29VughgsgpZ1pO2f/o9hYE5mfodFIbCemR1KsYH+rD5mU5+Dx1473cq2JWOpuW347cZZYEeGvMLXix8WSQxfoyL5OUz+2E5RyV9RhjBUP4ug22tgAXjELQ47eZ4BLYfS0uzmAMgLmjj1dzKPhb8GI/WQM9OAGsvXsN2IbHQ5WwfQDrUocwnqMMfCcxd7d5Htp9P6h+W8I/gXOYag8bYWxfnjHfV8+gydWHqL+vD+zyX9sx0l4fJdapLxXoQsTZvRIbxIAey09VXFprkIPyto0G5DJIZIy5yX7/1hY89VajPHlpIV/c/nDnPRHtkf1cCFv/4pYkwAv7v2FLodelsx9nAg+HP9u7gidPpT959bhWn5fmpPvUPn5tI93xlIn0XoKZs3vcvASzegd2IGwa+HOnEwKZswPzP+2fS6z84fF7AQm9Mv/u2YxhMw8hj16FxtIdSF8HZowFYvPrtx+PT+3GMyYvXq3djdC9dH7fAYuLL6Z2vuzsN73pGuuGZT0o7wRHjD3wkveuDB9Lqy56WnvusJ6SNZWBZUWJR11vxJnpq7IF095v+MS3ecE2av/PedPqZT0/bT/9z+vgD/emm778u7ejAXX9dlPO9/LrA4q9uv52RlAGyGgUdKp0u4vBUDa5OIN4y18Qk52N/envxiGD5MiaJCMt9bSqqwhE5nsVkWU3SZUKNxE2DsJahyjqYRSdavxEnUoGFK2peyyCZtKt1zJ9Rj2C4mcSjJmVyLGfTrwrryVBRNN8zUceETZpZWpGZnZwP1zAPMsHq7of7VZgI6Sbz7qqmwMLfHM+c9LXFUKrhnhXrYJLXwng1NG1Ik9hXuHfFAYDFUVarT6CqNAeDNA+jMw/zs3Z6Im2Ym0m9OOrsb2pI3ahFtatTbZ2COVEigTTCH4yRzIt1iHKSp55lQmLBe6kjs+8qrLYVqkG5uiwYsN0NW+ogk6nqi0ausZpaMV3huUnakImMk7Q2npnaLgIB6SXoCjechA3aRjlgMKmXwEJm2v0SPAs4ZLjcKVpJkP3CLmiHlbau8su4ysQLCHq6s9G1kiCNUd0ZXDuDI6jZDGIUrEGzTJv2LJZDpnMOe4z1lGvn1k24gN0edgCWS5Uh7Stk9jQClo6u1O89eASm/xhMO1ILymUf1W5CmxGZ551IPfpZ9VeVSNC2B1WoPapCoeIj02wcV71VYRJYdEHnFlaAQ7JF/TRSFrQcA7y4u7mSExlfPXNpc9EK464UrkEmlD08VMkRkAgsNsAA2p+mWXU/hmelE8SfBHyMwuTrulV3vdnGgjpBd50FNGOfocH6pm7UswAGm1FXsv1V95lDVUz7DsugcbEqS+6ULbDQZkNgoV2Mjb6evtsFKNHexD0nxlGdaqaOGqD3sz/IZuwrBBbrAUYa1kc5ABaTSNgGkVjs3XeAXeRpa9rH70t1JPuEdLe/+Au1QmhnP5aRt58pPdAWSEZfexQN+d0s0A3z4JahIWp2gEKZdL9d3UC7SaTx/EYbkYTZn/zZ1+wTAgulaNZP26LYuZ5cwz5CEEM8y2PbW5ZgzOmYjhICH8ceGX8lLUqaHGuoFH1OtcclqZ59OcYh2tJvMcZX0nSs89sp4GKGcoXEzQj8eJ3tSarxrSxsWJY4DOD/GPvyGGiZ7NexuEEaSisEF7qcFf5bxAAXAgvBtwAwgIXggr5Hu6neJ6i4CICxE4DRTl3Ux63lm3PnfulJGV+rV7VxtS5IfNMRYylaBK/NN1Hn/LKAi/KupJvjOz5YyxUOni9/433JUpKtGHXFh1aD2LUEaxckaIpQoEovl4fQBIlXVdFyvlV5qzyczQzzWAQWe0YX00Ubcq1XaJ1H9GglYFFAgZ6RNC4+21HCnY8qlCDht7Cx0EXqD2MYfj7Hr91+JD1xR2bcjaf9wnpUtfRCVX8sX+k/1zsZ/1k+2p+/Bdu36ijSg/ORWBQw8v2oiRWJi+Pua94+yKLJqn93YKFk5tffeCRdg22LBt7lUI1KidDjFlgMfyy9/s8/n1qe9Zz07ddcnHpU/Bj8p3Tn2+9LJzZflZ7z/GvS9poG3kqqUIWSLHbNjKa9H3lL+sD009Mt33Ys/cWrT6Sbf/1F6eKGT6fXvfJr6Uk/e3O6qm9DWrJaKnEf+fmcwOJ//u7vJoGFQ0Jt0ObGobT8ivpQDIwOqryLCYBRMsJEeK/yUa6MF6tsPPYcEUsg7muqUEyKXudJxem7Sr+K4POQGHAuK3lZYuEqN8y2afHLDIDGwZnRd1K3Thko5HAmmetTlZ33tfKWfJywSTeY9HiWr11R97cGZkWGT2mFzLTMqYyDDIrElulVauEsokqE4WLVVS8s2AqsamH/ANShRlfhahaPUEPM/GPEMR1XjmdgvBdHh1PrzHjqwcVsj8bb8FHtlKkFpnkt+VvfsrOvTJUMmrOb9FE1y3paBGlg/nqo0aOQxqwaVwsqZJKsfDA/BC4r9NJDw+eTMK66INX1bCvASOmNbeNB9lGGrOdtn3ClOUuJfClzLXOf8xC85JVdV451UStDKDAI2wsY26IOY3tpuKtb243uu4CajYyw6kVtMPPahMjAqfMv8FFtSTUmpRYnsDUYxYZB24xGV45hEOdR41lNmpuR0ly8czuenTaFtMmyqveeXfHCCMMMC6iUPOzFEHvvgWOsguu1B4aTdDb34hUKULFrez87g2PvgUqJ+zXsO3AIw21UqGC03dPDPTuss9KgDQCLfoycN7JHgEzrBIy7m6PNwIxOwOCOAl6UlNh31ldApBMbBUGVAHoRxn89zF0rdWlX+sDP72QCydYx1KCGod8UccdQrZogLaUwgjTrFn0B+mucLyjpx1OVwKIblaUupD/ukTEL6JoUnMD8a7Niu4xRHjfHC2BBG+mSVU9XTfSddgCe9j2z5CMoU02rC8AnsNiGC1/BlmuWp6BjSJ1oh1HU0QYH8YwC+FsNHe1juq41fYGloErQII39pgRg9k37zobW9tiMUGmZUgm/1DVIdloAnGpFCBKkoUBNSZDhlGQUEMenkPspNLNPh0oTbe6eJ7GnBHS0fjoR8EMIr1JIOfIIl5lFxxb7ZHwbpGO/FnRI3wJY3NU9lzkbifvtaKeU86zGIvqXY0pIOqmjfUTHC6GSJdj2G+AbEeSatpHNW4ms+Rme2ygLheWaG545RoXEkLT9OP3O9MClnUW4naVFXHIQ0Ma4iHrWIlK0VbTnqpa2UIVqoH31Uia43Qxocz+LSzvbkZIyxplv1ITEqyPy9nrpUbypzR/xipflvfUpN9Wz+rARue6P6dfeR9zqpXU+21GFM565eZR2rG7iSTyvS7+WT+1tpn3EiTKbXknRQPbwcthevvPMq1K8uM43liHCVOGs260vfOx5hRpm/aErmwAU4pz3eSVgYSJ/hrcg95h47tVt6ZJNWdKqQfWXWc3/gW/vDLp/I8DCtFVv+tAXxvGK1JKuRlWnGfWj4bH59OUD0+FZyv0XPo1HI6UbT724iXlpdeT7blS0XogqlN6nPP4aV7e7j8ymmzDwbmMRUIPylTw/ReDqz3LQoScn1YhM95It69L92BvoylW3s+cDLE6g8vTqdxyPMtzEvhraqElbjdLdo+LRILFQwuNYpY2MBuKf+NpkuMf1a3rcAouZ3endf3lnOth7bbrhxqemy7pQg/vC3enOjx5PLd96Q/ruGy9PnWWMOZfEgq0JJg5/If3d+w6kq256Qbpy3afSH/zaA+nG33hpuirdk/7s1YfS03/qpvQtvbgbr++Q53l9TmDxGwCL191+2xlJ2rgenst1ARc+dwB1gIxBk+sSpryrDda8cHJ0KJa5LzSJuMb3FxNmBhWqP5l4nEydC8NEGsSPiZZzTLbEk6kvgMNITv6uBCo98CeDZVmclAOQRF6RqKk7HUTZDeN95EeYsKPwbJkj/zyB+9xJPqtMaHOQDaYlcKyGCixgGqMMpGxcw7tngm4oXfVdz4S+prUznV6HOhQG3MMYb09g6896azBHMpwj2AdMDZ9ITRMjaePpyXA5q+vZFgjZAuPkyq6F1duSjInAQiZZumUd8cxghZtWy0BdXLmVKQ9QAeMq3TXwlbHLdLL1Mr01mp12JRxmMxg6mB1XM5u1ESFvnwlkXBXXMNlntoVlyiAL1Q8BVsXk+i6ABsDE95bZzdjCqBzGWGbb58Goca99gYx0Ny5M3ZSuWe81lFnjd1VhZOIsg951Rlyxpv5KQJRYDA0OAbimA6isgwF1ZdtN6nqRfAgMNrPqrnta28a6n6aeSmk0CNejlDYWSix27zsMXbEtoA/YdrqO3YGnpx3s/9BCGewv2rocA9C4Qd4gkpNjx4cCXJi2EgGBxdatW8KblNKXScqnGtSYjCTMrEbtGhPbX1SVU8KhAbS2GQK2KWxe1mPFH8AC4GH/t20mqbuSmmHiu9naJGlOwySr6qPkKvozYQUoSlq2oqq0hQ32egAUG3Av2oYEQxoKADR+nwYgaaeiJy3BkaBvCKnFCAbmgre1gBOBj4ynLmeVrNnZNODeBD13bt8RG+61AlpV3zrFz29RcHASwKXdzmEkOu4+L60nbTNATKiB0efyBo0ZWLiZXbQ1ZVe6pmoSVQkQLTCQibbPRX8jD/MpthGCE783vwMBpVITAY7fv0fu69lFrBKa7AUNiY0e02gDyysAzGOBqQTLGN+0iwahdhR9WbsKvvPqJy2jfwPoo/87HtCPLZvMfAF6Spsso3m4IGLqqgTqHcq+5piVxyLGJeJZDkNZ/hiL6IsEsFjRfzXgDlAR0kLHM/o07zOwAPgQbl7j7Xgu6CC6iw4Ai9WACiUW2lg0SGN+awASfQEsOtLlgIvNGNc3mbcNUHfkO8fG/LCM/5a9HGfGKE/PDH/m06U7vx+P+vSW3i67iiyrfMn/4SUoo7zxLPNSyZanH69MYCmRKrO6NGrAgmcRrorgDQnUUmcyK1kthcvJvfgxCCym6WxNdK1v5nDXbRngX7ylD1sy5pPq0L7hXew18UVsIrRv8FC950pcl8qYehRPTu7J8ITtZ67Dnusdn3368Jcm0kf4KcHwcLjY2buOtDtCTUdQ4T4a2jp4qE4kEHk+UpTS3hqC66Z1grIa/2e+B/VQVgP1kuQK/U+z58XyY/k7jbZv/1Der8OwPe1r0vNwDasq1I/jyekiwIbH8njxcNmfe9nD4n0Yvbs/hOXV85UARWPp/4YnLY+z0eW+PXii+ujJ9NNIX7awF4jH8jzPFvdtGH2HZ6eXbI545c+v3nYkPQngdgvG8B5uYCiQGqXOfieXIDW6DI9eAjaBRUiEaY5fue0we3jgteraevuCkupj7TyVDt/30XTP5w6w0Ow8yFg+NZsa+y5PT7n26vSEfhaCalU+h8QCO4vZ8cPpy4fWp2+5opd9po6nz7zhb9OnG5j7T0Pz9memm7/7iSwi6cb/Gz++DrD4nZrEomSRh8o8SOfprXrjCzqBg7MfVBmk86cer/IfHkRszgEKDE8E176cGyNePPN5ARVMjDnRyCzCEjji8yQARHUfdg9O8NWkZ9rGDYaekcLV7LI7ryXJ0o0KHBAnyk8ky12mDGtkXgGAzKeEs57mwy+kFeQrYy9zIENtuWSyQ2KBnUIwG5RB9TDzMUzsWg3TtV4GGQZtbVtHWmhqTzNr29IYm+PNsoo4r3oPM9AITNsgwGL4xGBaO34ydZ8aT32Ni6mdjfKaQHfryN9VSlgQVnxlirJRtEyUZdQrjV6fsrtLjDNJNx8yZag2WG6ADpWKssaKvVadHNZfGio1UV3J9PPKcwOrx6xYU3bdvrrS7Oq+eZpfGKparoqxkllSEpF3csbIFxroJcpVfFcpVNmx3LaZfvYXyDfUyKCj0goSDiDTg5pNN8yrdLex7Csy4DJy06w6l70WZuCYVPeQmXOfhGEkGO7CvRbVrw2odnQr+SAtVZl6WGGXIbasMnyCGOsyQ10n8TKlnYWb3e3edyiYdogaak3bt21BnWoLO7t2B1iRYRTAnEQKcQIpicDiEPFUI6I6QWM9SW1hJX8HcbVtcS+IExgvax8xhgqO0golOPZDgYgSmT5cwHYBpmScR1AhclJo1tYEiZN9SwbX/Tk0tNeTlwbAusc1baUJwcRWbWk/UAVqYPuW2HVcyU+TxroAXJnoUBUjPc9KpkaQgrinhe56T7CvhruQm34jwGoNbSczPMOzBdsd2rrBXn9/f7poYADjbdSg6Atu9hduiqmV340G4ceOD4bEQluCaYCKGwaGCpzMM2GUnmgPoYF8duNsn9JrFJIJyhn1BkBZN7+3/E3pqcx9KtiMjr7ltZJEiRn7r9A2UT+kbX6HoT5Iv/KwT/vNqvoWKnkVUx/fK3XIan1889DesUQ7DPs8PTC+Mb8TAYTlEuyoHqn3L+ui5K0sbMjkCxBkYCk2fSKPG34z0pJH0e9tU3ecV40puNUqXvR5C8wRwIIIjlPxn5M2X9IrnEdwbz+S73KjRsGEm+TNM7aUHbijPNw3VBILwUUAC+4X+KaUKPXQZy9hszyBxVbo3yx4IZ3Il/QdL8nKjzGf45kPLLol8CJe52v+Gr688mEVKr/wwbLDdihHLc3yoP5cS4iLKoPao1I6aRkFMuJSuvXJlDeRrUn5gD+lGKUM8bwEMkitUkbyLTQhi8glB843UQYTzcdjEVgoHatU+Us1z/ssyQQOGhqvdPApJXe/9ltqZzdqz/WHjLP7RKx0nOtdCS9jbzh1/h13lx++l1HvOMt7y6d9QxO725adu53vPGSUlx9ne6cBs+OGG+t5uLldPU3OFm95+tJSxt3yBqNOUeyz9WU5G12WP18pz+VhzN/qOjcup59lCR6ujgx+MkEvJDxlx23D1cf13vKWb3F5HR9z96fZU2r3/WnfYfaAYrpqbO1J2wd2sTiI2tIZKADe48g+nKJMp/bLn5x2LWmUPZwkEPrU4c+nT371GHtKNactV16dLuuDD12hjz888tmffH1gcdttueHseKQTDL7n6lfXF2K0deCPH+99V001ca2RmvHi4KXTj99UBghLzHstjCGYkC2A+bofgWkbPowX41zFJ6F4ziQeKgDEK2Ux1zltG5joZVIFFjITli+YFcKG2o4TOtemk+sXhYyKWFaPnGY+W664Nz4MsmkFsHCVkmtekx+r8zBNngPQVDQIIEIc1R9ceZUp1qPOunZ6QcuGNKutxeJaNq4SWKzBHSSbiME0HoXhHBwSWIykbtSh+hoXUite6NexcZolDN1u6unKsKov1tOJzfoJYjpY+c4bqbGRG+X2CFUjwsQKJ+VxjwhBhGpOpmG61l9JgKvX6qAbRyZI9ZK1rCS7Am76WdKR9dazpCHT29ViD8GBwGKsMo6VXqpiqaqidEK9dsGFm8BZjkXKOAsD56Ang9bcBIOD2o3Mqh6M3CzN/RXCfoHyGGeW+yGYeZn6aaRE4bqV9pii7ENuGMdKuQNcJ6BiM6v1O5E2KP3QXqMVFRvVqewASixMW2mFK/YHMKreu/9w2sv+FKrY2EeaAFQ7tvWH4fYOGH8HOttbZtzVfaUH7iVxHFAyzr0Ja+fRSjt0Y3uwhVV9Xb363E3wlAaobiTjK3NqJ5LOgsE+wm4EAAmeRoeHQpJiT22G/vY1PWqFdAb6Bo05n6Kcp+k3RYpkQNu5DxWo7bjZvRQvVtopuEmeHpuK1yalXGWFPdzCql6F7YaqZQILJUIT7EJOx4oyQlDUs7QhQcWK9DahAuUu5tsxiu9UogFNNTw3f79H+9dh9sc4fORIAAuZc70yTQNag8mFuPY5AYQbynXikSi+S9o72hoQ7LcX/ZRvO/cPgAV1M45ASbU0d7q2b5JUTd1J0Cd9bVvLotqRQDh/P1mNSa9eSk7sjwIIy+03rdvkWDgAqAiSNWAfn5iO/O0PfmsxxkAL45m+z4kef3gdYX0TL8hV18ExHpBmjEOA5DLBB8BlAvWc45Aefdd0/GOy/snnPAY6STsGWt48DlU0J7z2XQHY+Zbm+VY04pY58XuYJzyd8wxVKDpEbJy3inp3009VhbpCYIGUrYW2DmARlbMgS4flsYi1Iwpc3fmiFJxQReIdVYogPCtpcl5i0o0XEaPuVWpnnM4Ia/AoxFJJzritMi5z1BkJGTcXMh5LwTOPXMP6/HI6VTjTri7rwxirqkG+KGGqxB+LwMIq1up8JhEv3F2gwAUKPAYp8AiAxRtiUHCgd4CQuc9zRJ4wpUkMwDFicsV7GYAzBpJq8GxwQqyFy4ONIZ1EI47xTD/+mbKMfvUsnhOHswyx4Y0nsJI99j7SiHe852xYC+LALqgIRhfmRmbEPAgRuwNr1ClD4mQcaZNuqaulIOE4+bAAq5y26efyOIHLJMtACRZMy7wLsAhVHuqfc5YfY0URhnEtE7eMiiv26quvRR0ltaKv3tSRpnDaObuqMZ2GCZii/EMwiYMAi/GTQ2kdwKJreix1NbAfA16iGudQQaGA1i109WGcBBXSSCmOKiItpC+wUH1IN7eu7hZVDdsUfibqX6QLMugaMgcxSF3DVr0tqaZjuuHPHzDkCqvMr/UvpJIRKqu56qmrKqY9h+FsC5/5XobOcgjMlCrIyGkUbFk0xJa+1gfScc9GfJRbANCDF6ONqEPZbtZTBjq3GSuzMtjQSQPcMZg+pT3zNJyqNmOsumtr4eq4khJtDAawjdgGk93XoxvYNsqi4bogSOZa41kZyClsK9iFG7eoGnHHHgPUwzbUYHsABvoSdpwWXJm/qmKCm2OoTx3CNkPDZ+1e/HjcTE3JQAe2HX3sHeFeHDJo0lUPVtZf5jm8AVEQwZvG2ZavmXiuvlu2mVN4jlIKwE/9e/t30FWGl7zcQVppxRw6/YIUv4P4bqCZ6k8Cqksv2kE7Kg0QnNCHKb+r3QJH7Q00QjfdoCe0G8RWRWnQGKBFFR1thmxP1azsSxqEd0DTbiVAqFdFO0HTNlSl7B/2fw2z3bNjL25mj+IV6iSugQMQU2dBkH1YyYDfhAbgtncnbe3ChEBXGwXBn9+goEB1P9MUTFsGjeM17BeM2V+sWwEP0s1vMuhBaaTHwqIgIC8AxHfKtc4ElKppZyGAkJkvdhD2YQGU5bQsAuGQqhFPo2u9qNl+WX0p4EAeP6q/Mb7ZWTlq4xkfTqhZCir4VgJgOIYRxnYr6psZdJFCxJeVrcamSCyPgxlUkAbxvTZULMxwFdFoXzJBFcoFi7zqKXZagE7aWIAMMdpGJZBxAkIzWCFFoQ9uBFhcjMTiCQCL7Uo+BULQzzpEOS1DOXIBy13tXHtMhLqS18aYpYscoCJTLX65KMx6jMPlIefyvO5RzoeMH5aWz6IQZ5QkouYnFXVrheZVXficngs35lvlyPs8e1Q18YX51BXIJJaX29eGeSwCC+t24bhAgQsUePxQ4GzA4v8DAAD//zOYwgsAAEAASURBVOy9yZNlW3antb3ve/foI15EvCZTZZRhZTBiSg2KEQiVQKoBAwbUhD+gRlhJ1VBKNTUFMySVUCqLAWbADEZlGBNkmIEZBlIqM1//onP38PC+b/i+3z773hsvX76UyZCs3tM97ueec/bZ7drNWb+91tp75IbjH/7Gf1l+9w//sIwUjpH8lhuuNz7zc8PfiH/1VRkZHY3fERxG9cPR/Obmhic9+8/VYKNcx0aNo54tTMIZN+9qmHpN3LiNkVbC4mxa3hvfyAh5MB88xwE3ilIurq7K5dV1ueLqs+8MMzk6VibGRjnHypin8XJ63BihB/7Mj+Wt1+pW86X/kTJO2KnxsTI9PlEmx8cTR9K9vCwXnKZ7dX1Tro2DePU/OTFRpiYmy9TUZJnwnJ4pYwuLpcwulfOZpXJUpsrp6EQ5p0z7xLF7elp2jo7Lzf6bMn24WxaO98rC1WmZujwr45fnZOo65Ty9uCynp2cWvEyQpynSmZ2eKvOzM2Vxfr5MT02VCdKdwW12ZobrTOgsWW5urssoebu4vChnZ2fljDQttOU+ODgsu/v75eDoKDSr4aetpaQzMTEWmkpXj7Pz83J8clZOzy5Cb/3PkYdx6AMpyjn5PDk5KVeXVzxTR1xPyPfp2Xm5vpZSlAiP55T9nHdXOJjvlaXFsrG2Wu7d3ihz5N86OyPc5TXxEO6SvJ+enZaj42Pye1iOTs/L6bn1YPyn5Zj8H+4fJNziwnx5+s6DnA/u3i7z0GecOrRury4uyBfthvRPoMUnnz8rH378afnws+fl4PC4XPDekpqXdx7cK7/w/rtlifik1pu9vfJya7u82Nwqz15slUPq7bwrl+1zElosLC2VdcqwvLKY9ndyfAI9ztNOQzDiuaHNTk2MU2+z5c7GepmcnKCd0uY4L0n/5OSo7O/sQmvq6pw6043rEfQwj2cnp+WSZ9uifUJajdMmpN3jh3fL+0/eKVOTk3G3YU54z3ur8PDgoBxDw9CfurQMr3felJ3d/bJ3eNSr20v7FJSYmZ8rC8uLZWVtpSwvLZTl+YWyNDdXNlZ4Xlws89TVOfnb3d0rL569KJ9+/kV5/eYN8Z+X0fHan20YtoUJ2sjsLGW+tVHWiG9xcSHlvaTNXFKP5jP9lPZqPR+Qn33q1XxbngXSnYbG3k+OTZQb2sV12of9/5o6ShXj/wZ6X5XztNXTtPlz6GWftd0e015OaMNXhG9jg/19lD5p5V+l3V4T3jZ3nnZiPZjHtGHe2w97Y4n3ST05qOOWmfafq+PXOHVrH7HvjpOWf8ZlHtopjYyHYL3DZ/ue4SeoZ+lT+2KNP2ngNkL/9rwaoZ8T6IyOdUkGr233jAU3swvlZmaujExNk++xQoMr41MTZY26fXdtufzC6lJ5Z4G6niSPybM55OA+Y6tl5vip/HXutbDx8lb+K2FqWP3kritgK2fc+HmLiqQ7eCQPgw7c6+PL7jUufrvw0r/50X9K1cvzQIT6y6O/5CSX6uJzy01zqR58o/faD73NN8qb7jDt//gXf6k9/qVf/Sa1PLTrYKKNHrY9z3wPGAfX1tYGvQ3vhxQYUmBIgbcoIM/keDHHd3jwGGGQ64DFf4s7Q2UbTLtBuHnO4JvX3PHOQVW2nO91jjou14fcG75GF/9+oP0A1mhrHPFtRN0QbayJ248sZxgrAox58uyInTQTrwETOAO5HyA/wpd+lAUWMBe+Nz7DBlSQ/jgAQ2bfU4Dh0T5e5qed7YMegNXlx7gEJlNj4wALPrZcdZMJkMmVcQmjkRiJSf/4C6iA+ZmE0R/nOspHvfBRv55eKBdTC2UfYHHMh/2E9A9hVo5glk9Pjsvk8X6ZOdrLOX1xWiavABZXl4ldxvIYplJGSbrJRE7CmE4DXGY4BRWegokVmDWBxuzMbMCEDFeAFzHJNJ/CVHn6wZPB3z88hIE7hKE8DfMigLIcvp+EyZCJa5SyWmTQpUPqQ4YpeZkIUzg5OQWdxynPSTmCITwWYFhHXT0JAmQia3lOyyFpyuiNEqdMpsz8w7t3y/LCQpkmXYFjzecV8QEmBA+exHsIg314DNAAYKQuBDS8u6Lxz0KTdx8/KB+8+7g8vHenTE1Pl3HAiwzedeKseRJoffH8Rfnwk8/Ljz78rOzCdAuCZFgFJ/fu3Cq/8MG75S7Mv/QVWLwAWDzbrODicP+IssKgWk80AZn3GRhngcUKjJr1c3ZaGVLpYj7GoK3g4Yq6nIWxu3/vLgBxkrY6SrpX5HM88e3v7pZd0tsH+AVQAAAsr/UkmBG4SRuZ1YBZwMntdeh371Z5/Og+DPgMgA3AQh/IGeAyUvaI9wAgKbiQFT+D/gLL7Te7lO+AtnCC2znM52iZoizzK8tlFgA0TfwLs9OAitmyNL9Ybm9slFXAhcz+3t5+2Xq1Vb747AvyewAYOi3XozJbgm3igfb2P8smiNyAngvUseB7HzB4QT9wzFhYFCBTV/h1ELP97Nk2yesF9JLI1oPAs4LF8bQ3AZlgQvpdU6fn9ina+BH96pCwgqcjaCfozUQE9WuatmuBhf3A9AQe0kQ20voUfAmkbRMXF04iVGbMl20cSSEJkwAUOKOUAwtH+olunJZfUCGItI9Z34Zx7DFe++YF10vykrgSA/eEFUhkooMwue/iNF7dBbWjxCmwuIaO5xTinHigarmZmimjs3PlZo4xaBKAQV+7FhERZpw+trqyVJ6uLZXv0F6fLs2XJYEF71KO5CE9nSx1JeZa+399qbuHv829H1ZmfbA87Y0UNkx9NlyNp0dVo0zZ+yEkS42rpVM98TuQh55bzVEeDWcY40p8Nct9r9w1kNh7lTjJT5emafxUul0MLX4fv+zHd0Ng0RFqeBlSYEiBbywF/hzA4g8zaNdBtH74WmkZf3NkEPajlb/urQEYKPuDL484ZTAlgADB+zqr1n2cjJD/hEnkPvhJqX4FAjIGMiDc9j6cpshjAib+Ll1TDxDg2Y/wNYxzBnZ8D36AG6BoUgufEyEZMQ7DVCaCGfQubjJfbmomElcDFlN8tMf4YOMtTMAljKTMemYvzSIZteyTME1h8gUWzrpzjsDgXUzNlZPx2XI4Nlv2ykQ5ICPHMBHOgpaLszJ6flLmTgAWgouTgzKO28T1RRmDUQqDBWNzBEPZZqMy6wmT0psBpWwy4gswghvOLCMhcWZ4dFSAILCQoXJWGObp4jyMo8yUhyApM+MgUcuTirKchGsz4XU2tzIxlYmtwEnG1TxY39IQIkDCyrw723vOjLtxCvxqoyRt3E9Jy/IcHMEow9iOE88is+GrMLHOhC/BBM3AhNsuzL9hlYLIaMosKl3Zg+HePzjiHgY5zBj1CMiUsVQa8PjBnfLB08flEVKHSWgzAmNvAxA0WPeel9eX5eWrzfIxDPGfffhJ2YGxlnGX6VTqc2t9rXz3/Sflnfv3AjRMd5uZ+Zev3yC12C47XA9kjGEKTds24Kz7xu31skrYeWb7q3ThKqBieXWVyePpck59HjCrD9IvD8jfCrPGMptnlM9+IIN8fAxoOaW8MMYy19s7ewAAGH8YZPMXYEFZpmC05+dmkB4sIP1Yg+FfQyKgFARga93Qpqu0j9JzfwTjf0R8R8QvMJdpfkOZXiOxsPz7gCWlSGOUY4Y4Bci2tzHqYZkZ7VUkS2vU0Z3bd8oq5ZHBlwaCi13AieD3nPZ7xqlkyH43Sx9QVmUbsa7mBFi0E8tgmQTolmODvC+RpnVvXpUGnlOf1r1g1VOGPOACQFPbn20QqRp0o1KpByRhAAulPoJQQbNgZ4+2Iriw/bXJCMcXaWObEHT4TjBruv7pz7xV6WQFFa3t+N7xImOUHWngLp3I13Gu41zGOOpDYKG0cZI8e9g/BDumoXTuCuInbsIbj3TwDIDwimueiT39kjanNFJQAXopV/i4oBsrtbhUMgGoUGI6Mr9UriemkGjwnvQEjrbV1dXl8t76MsACyQXAYpG8TTgGmk5+ux9omywNurX77p35+rlH82tk+m9BuvIa/men08XewnSP1vvPDNP8DAb9Kc/VwYmlHF6Nk9MrDaReefnnKWPzk/CEGQKLStbh75ACQwp8cylQebifKbH4Z1GF6r56tZQMpA6pdUCU6a/3Gfcz2NYZpTrQ1iCOt+1IOOPgg+QZiYXPCcuY7DhtrEbsD2G9lelRUqFqQAUWhqlpJ+5eGg7u/DPAVx5SBgWGNe+rJ2GM6QlUZIgzQ8rHW0mDzI3p4INoOwbHD0eeOmDR8torQ1WpkuFTauFrkgywCKhIXmraltn4ZcKmYYqmYUoDLGCoRpAcHI3NlL2R6bJdJstOGSt7xHMEAwRHWiZhwOaQUCye7pe504Mye3YEoKigYgQmR7UlmXBP6R+aUS7LJLMShoNyzMAkLMPIqlqjisoM6TpLbhiBhcxeZkVhYGT6a1zwIoICGJq8A3Q4M9sYqV65BBAdDSMpoVxKRJwVdxZaujvLfZYZYUTx/Bm/DJv0runLIKqWggoTwMLZ9/0jwALPAoslGNbV5aWAImfBlb6YlsyoeZNhPVEVCuZy7/AAZnifGfb9zHjLCNrmxinvKDPXSiwewty/+/hReYBEwDqhYZIX25D1KPMI2IEu29uvy+dILX788Weo8BAfQEWaCxLXV1ci9Xj66AHM9DKqVxdlj/dbpP0MQGLYPZhy/Vc1GVo5bWEdxn6Dcxmm7QzwdAVN5+bmkWSgloUk5Bz/Wy9elguY+g2kG7c5ZwEcp6j+wBkHWFygCifdVP/aQ6IgkNlCPWoPicIFdL7uJBZzhBOY3CW9W0gszPMy6ljSwnqxz6TxpuwwsNBeBv0YgHYKTQVmO6pC7aMSB1jbOzimY9JfbMu0JzvcCPUoyF1ndluwdffO7QCLZQCGEppj6lEJgQC1gsBj6mWPoNdh+meQQghiImHkOkldXFOGE9J+A9N/SUe2jqTDGm1ggXRtu2H6ybfSCkWwJ+TZWhTc6l8JUZXI0Lehu0cYddSwrJNjpFwCC+mnNGYPEBNwIf3IRx0BUBcin9af9atal/m8yUBTgZAShbRnGw9H7gndxrc4ZmzJ2/poeA/qQH+ZbKEPTSGlquqSlI8/064qnbbHpmpFuASvaVi0flp1FNMt4xz9d0SQQtyCBdWfLgl7xVU1qBFU18YWlwEXS+VqfKpckqaSQ5noqELRXt4HWPwNJBaPAY7z9LmoallW855C1DJ3t195kSYe/Xx+pbfQLm8SvQXr/HXhfaoxde7tMuhoGPzrlPS6++b1664tubcSMbwvKG8Or4mTFNIO6EM8t+PnlbFfJstyU37lF/9uC/qXfm2TTyb0VfnUzbqy/XsOVaH+0qtkmMCQAt8KCvxcYPHfaGMx8NFwHO09+8DhbwamDESOqwyR3eDartVf/XDiOYyAH/g2u9bi5PM4MI538eufUyDgLOTEWBdPl3j15QOHaTPAB1jw2GY+HbT9l9mo0goAhXHKBMt8czaJhTOkFsrPQwZWB1fj4vTIZ4P3ybNxcE7AYGljIbjQ3dnWqPU4KHe0yAee9GSOp2DyVHeZAliMo4IwDnM8Oj1bDkany6ubifLF1XjZ5gu2y4zhEQzMGAzQDAz/AhKLpZO9snx+VBaxr5hUjUTmF4bqGObLWXSZHj8EoZnpka+om0lv8j+HlGQ1s9brqEKhShTGS+ZFZgtVIhlJmFRVX1RHkpbmfQJGQnr4QVLC4KxtVEZgPsbRkVedx9l7VaL06yz4FLPhUzCLzoorDTAfggolCqqtOBPtYTswn878Rl89zKe2Eer2AxAsG6pMhhcMqQq1sb4eHT7TVZVNetugzwE9hhNYCCi2X+8wi/8m6kIX2FpYgeZlfh51MKQf9zdWysM7d8rtW7dRs1qEV6ac5Emwo9TmHHUdJQLaBrwCIHz27GV5hQRCBvSIfDkbvra8XN5/+qi88/B+2YABg3sLEHpN+p8BDHZUK4JxVSoh81sB0CXSCsqBGtU6tgRX0HuERrbArLHPAgvpL7A4JB7byx3sQJYBVjeU8+LshPeoItEyL6gP6+yYfD5/uVlebu/AiCNpsP5gxq13VbY20JHXlsQ8CirmkCIEMIbuqHbZCGzvSGjsR7WuzzOTbxm2KL8g74ByRLXM/kp/GZX+MPQjxDOFrc0dGP+H9++iavUQ6cKtlGkMOsnEW3aB0CHMvJKLLexQaKwBpFWdjnvaj33E9mSFCXD2ABa2uVFm1wWWS6hIzQNWHEds76qNtfFGN9X0VMezLXpYn0qpTLv2zdruLOs5eVfyo8Ril3p9/WYv4OKAsp7T3sPIUzb7csLiJpOv9CmAjAbTxookZnr2e8rlkYt+OvfqmleRRnV3Dk7ptwKlqEJBM9UMBcNpj/TPgHnKYj6ayimhqOMKLlKFNUJTI6xjZ51AEVjccC+kJ+sBFdpakEgZpd2NL61w7UssIpGxbyI9uQVY/EAbC64PkHzNEhdDcfJrcknXMn/pcBzyaHXTrs1be9+uze+gP8vn/6BbC6/bl8PGvx5qpuI1t12gmsvBvPr27SNxtvLgtecbr/FtuXifb4tB9dudb+WzVT5eWjnivffTbkYAFkMbC6kxPIYUGFLgm0uBnwssfvf7AIs6jNaPI2V1wHVgbTN/dbB0kM1/Bv+Mrw65uDWooL+ENTwfNz+ifvSaW00mQ3ZHUe6TUPUnAx+JRffx9V1vsCdEhvjMGhGMhAMquGpgqb9ExbXZZgggGrDQ8NiZW4GLKjoehjFk4uG+AQvfeZjv+tE2XxpMVt1o3Uwt+SGSfGT06wcaZkemVnWOCRj8CZjuUc4x7RwCLGbK1rXAYrS8hCHcgZE6hOmZgCGauURagZRCYLGGXcXKyBXAAh1pGUqY1lNn+fF/BtNpnpM/0pSpyCwoeVBVZYXZxtuopmgYuwCwMC/OuptvZ+b3mEGWwVK1xgo0rFIiGTUpIpN1RpoatUannPSMV2PZmRkMwmGAe2peAIoJ1Costzrzzi6btxhRY6wsCOh9hHF3FlhgoX2D4EwmStWUHSUOzNIr7dEuZG15hZn+qrsvA66URP8ylzLYiR8m+wCGVJuALRjt11uvsbc4inqQ9T4Hc7TKDP5DGPtHMMH3mF1fgrGapD5GqEtBVkAFQOv4CNUiaLJDXM+RCGiQvbn9BhqdBFgoCdAI3Hg2oK1ldTb4EBq92tkJUyw9rwEVzthrbLyNVEF1pxX8331wHxsE7F0AmDPUxzzAT2mWnP4BDL0qSYKwJRhqbUy0Y7iknJcAiisY4wMkM/sHGlUfkq/XGFkDZGCKBWrSTJWhNRhCAcVtVOBWMttPPJaV5iqAuCC+dJLUcgWZAXkdUx81KACaIC82K0iQVJWR+aRZwLRfph+obqVa2WOkN+8+flxWVtZqWUjnEEmHxvNKQax7DdZVjzJhZ+dVy5vQtoA+ImNtuxGIKEnQBoImEsmHkiolN6oKeTRg4X3aK+3RsLY361q3jAm0b+s0Ug2AqrYzDoKqRVWpTzV6F4gKHG1z2k84c69Njsy8aWXiIG20L6Egazl67dmnDAWOfBySiQI0f21c1EEf9lcrw4s/AiJBq9I4+6ZhVYGy/6l+KPBLuCTD+NKLOUn1+pXxpf9LA/sJDkCSvrQCYDFKm1NSMU6/GsEu5pp+e4165DX9YIT2Y13cW15ADWq5fJfFBjbo69O4CyxyEGe7fav8yVv10sr9U+/N4Fccg/5Cm85P4hmgo85fjsGwPTfj57kd/RpoLj2fAzTr6kMvptUFb2HNT0K1F81freQWcf/aJWG4fk6M2/++y6/8B99+YHHOeGU7Hh5/tRT441d/9leb4DC1UGBxYqY8nr1V5pACf1OONgn8F83v1wKLX/sNVKEAFvns9cfe7uPnwOutA6wvObtBUsYx463X5KwO8vGpfz9wXnNfr4nMWHDrH0kgr3T3I9akFvHFT42/MvGmm4S5OJPnx0U3PsG5r2mKZ6qUQTAg0yFTOg6wmOBD6nPLg3EbNuXJfUsvpSWemt8KVFTTkgFX5UIbkFpOy2JejVP95gYsoo/uLCGnq6+MAixGpufLMapQr7GteH45Ur5gdn0bFZh9VIEmrs7LPGpQCwCLlVOAxc1FWWGiUWmFjJpgwNlpmY6oZ5gwR8ragQvLpurPLZiD+7dvlVtr65WRgwmWqTOP2lHsoNP/BmZWcNEAhcBLiY40kQmVcZdRPiZvghHLVSUW0zGYdfWnWXS2BS3j49otQBeBG/7kQ2XslFpU2wpjdXZcaQXMHsAi9IKO0t6VnV7D0KueIjiT8V4QXLA6yTqGwStIC2RGjVhpgGowkYjAKB/DlO5ocAyjuP2K2XYAimo1pjVFGPX0H92/U94VFMAML8NYTbMizhh5Vu8/khl08I9g3A8OsQ1AarG5tROpxYtX2zEqF5AazzsP76JOdSfAQimNkg9V8PahlUyxdhHXMNPq8wsqPn/+Ku1yHnB378GDch/7jPVVmHCZaqUwqqdQJ9IjQI447PDSVhUgV626hElWdW13l/JRb1uAGAHAPjRzVSiZYGmzhCTmzq016ny1rCGlme9AjO+cCZdpVuoRCSJ05yZ1oaRAycIhgEX6q1bmilDavGhQr6qVtDQOG7ptYBGJyrvvPARUvFOePn4MgFtI2w9AsF0JEogzqnFpT4SFTgJXgYVl9J2ASImAki2lJYJmjYiXAHEC10noY19zlBGIGr/Mtky0TLlSFOlnxgIuuJel1mD8hDiVhAlwBKNmwH5s/3ElqB0kFqp7ZWUt3ltOB8sGLExH4CnAqLZbRGFOKIetuXeYPEfcupeDDLNvkkMmM+yrISIX810lqYILaJF3plklFXWsS9QJx4jKtT4bfyZTzItpmgbh62QO9OI+EgveRA2qAxbjAIsJ2j+ottwALK6g742TAdYLalmPaDcabn/AdYl2M0k8SbNLt9K6pVnz0ohh9htd3i5/LWvnu3f5sh9f9OLvfA366bLQC++N7+Nu4tWBPEgUHnDqO/dDtzhDr76Hrl5bCRI8UQ6ETLw1fN9fPPljXEmzAsSee+c1+cLxrwOw+M//t/+qfP/H/2uPBMObIQW+zRT4t5eflH/wnX+/vD9/5xtTzLssjON3+C96/BxgwXKz3/9+BkU/yTm6AdL7wcE37xnI/Y6FEe8+aAPjeD6ASgO+Cljko9GSqJH3PiTdY579wMr8tI+o7/x8ODD7sc/hxfSTB9937t3AbloBKOSlBwY6pr/N7lvcGo2MAydRXuvWi6r7WONe4wOkwNC8FR/PjUaZMYRRmohqRl0ZSdWQET7QAouRmXmAxUI5nZgt+yMTZQuJxWfMdr+AOdyBkZtAFWrh4qQsXxyVNdSglgEWs+TqBPUNmT4ZV2GQR81irTHpJSDQAFRGepnZ/XvM9N9ndl5phTPr1on5U6VDYCGocJUh451y5RcYi546FfHJ3LjqksBCGwiJYtmNS4Z3ntnkOVYEmpudR32HFYdgUmTyUmn4VQoRA2tmq1U18mhMoKDFUyZLEOLXWEb8DUyts8embX2a3hzARaNgVxyS2Y6qCLPoJ0gqEj/A4ghmeRcgIZO4v4chMtIPgdgRzzeX18zkY2MBGPiFD56Ud1l6daUDFuPkGcLAcMNUEocGvnt7u9gY7JRXqO6oDvVi83WkAtaxM+gPACgP7t2O7YL1PKmKG/HcQH9n57WXOKe+XkNfVZW+eLkV0CCoc1b/6dOn5SEAYw2pQq0PSSujDM25CrpuoI0rmWUZN5fCjQTkKPW1/ZoVqF6+wmi7Lr+6t3sQCZAARRuORw/uxr5ilTYwhRTJdAUWAhfjuQCgTEGPMfzDz0c1qIKKg0hZVPsSXOxjW6EUScAXexHajY1OMDCLtGIFGxNtVp6+84glbR+R/yp1sD3tUnaNwS2Ly8jaBlXFy2ptxAEpu/Zby66KklImDb5HaA/aaWhjYz8TqHtKmyyPDAi3HESRfqZal21JQMKIEaAygcre5ZVATQBKfVC3SitGKYPhBBaq0alSeAh4UiJ1gt9INWx7pKW0IAw+fr0KLuxzvMoYVO87N940ZlV/7chbA3DYU23vARYSQDcutoExaGo/sLy+UepSY+7TyXCxH+nCSo8GgAR9rc/UCCpoVGqhcXZsLKDNKO20AYvRJrGgP98wNgWAI4l8IrBAWqF9xSx0HyeOzL+TMftAMm3mu3J5WyvTcpp7H/kbeN/Gx7zsfgbfe5+4Bz0M+uvKHD8D8X6F9y7dfh20eN8K1s9qTbd7KQ1ztPxYXB360eVhMO8t/oTrfmyTLYjhB/374q+DKtQQWAy2iOH9t50CQ2DRr2G+Xzc3v47E4vf+qAGL+rKuhlFH1bCuGV1552jJoOs47GBZB0wfDFf9qyfsYKsKVa4+ByT4vouzfSgM1n2MXB4yT4blzKxq9d58JF1//Ktp9q/mM9H6k6TqR7wHBPyA+6HkzMwmzx5JlTgjtfCZsPm84OYHNSBJj+Yp+arAQkZcZt74BFG9dzII3amKwyhMw4jgAgZUYFFmFsr5xFw5HMFwm8Q+gbF5BvO2DVM1zizrHMBi6fKkrF0eAzLcu4JZf2Z+Vf1RhSh56mjbaGfak+RjGoCg8a4rKd1lll/D7RhUw0RWJhZmBCZaBljGz1li94YQUDhzLM09ZFSUNqhepFRAhnCOWWqZD2eRlSTMwfypxmP8MtcTk6h6MXsvnfxAy8yFuYfBNy7pKFMqMHGmVZUUZ6qTLmFkArMUaIAITDAMnYeMseDCFZVM0xls8ykTmCVEKcMu9NkBRAhMDqDlKeovGg672pL7O8iw3cHO4W8ALFxydhWJQZU2uJdFba+W8QywsreH4fLO67K1VZeQfbWFlIAZfGezlTLcvrMRVajb68RBWVRzG+ecJG/OqCuxOCE/m0hPXmH3scX1EmNtmUIN6B/cv18ewYg/wi5BekoX+1EF4pVJvkElx5pQ794VoayzM2wtZNg3tzbLsy++KCfWIeUTWKiqZvtfgBmMuhe2D+uADLuUfcl2KCAXVArMVHmx3YexBghod6AhfCRMg8BCZt9VoQiTPCau8dTFGnr4SiweP3yYMtlxBJOq6u0CzpQoSbNVAJR1JwjzvTQiE2ljPkcaYhlpDwIYgbh0UhXMejZd3d0Hw3IKJgVvqoxJOwll/urSsY4HlUm9wL9txLDjiEG1jZqZBPiStkvGthWholZI2zN8ZSxDNJtewEyTEFYmvjLzda8cykBa5k9amm4mJwJA4qJrPYySu4yHnVMdG3Wj36nSZd+gD7sCU5Vc9ILWcNA+4yJXY+8BC0EFNI0tSPLiW1Kjzd/Q5i45hVNYUAHakBJhszKBKuAEkgs6b6QVN1yXAO23WEbYlaAeLcyW26qgmTeiIsl6mIfu9qcuKXJHi596+Rd3kL4tA4NpS4scvudIyl0e6nOcu5/BkIPuXdSESz0aVxefIVoSgyFqarh0/vD1lj/zET+DgfEbN67G+9fBeHsILAZbzfD+206BIbDo13CAxT/63m+U3xdYcGT45SfzevmIxCUvMsDjJ9du/G0GjQmcAbZ+/PLR5IuUDyEfuDwbXzfY5tpFnYE5EXRDdvzhyjXgwnz1BnEz4H/1myjyTqa/hiGg/z7UOPg4VpuNt4FFy0uSTrQ1VkGFd6ZQo8lvfcDFPMmoasAtwyYzkFlH3LxGpQEGW3AhgNGQcgSmYZQlZiOxmF0MsDganSxvEI98jKrGF+h5byE9GIOxnTs/DrBYBmDMoBIFlwqjzKw/zKW0JwlmcGtZkikyKqvhDLcqUItIEW6jPhR1GGbmNaw2L+bNmV1nclUnEVyoUhTD39AwEaXUMmQycS+3twMspMgyTIcb8Hk6kz6jATcMtTPRE6yHL7BwbwhzI4OmBEDgUsHLKXUBbWBotW24gJFUCuIGf5GCQCPbkiDGGXM3IVP1SuZJGxmNfQU0qtAoKcnSs5THGXElKjswsi77+ppzFxDgbPQ5zHxTnZFR22AG/LvvvcOSs+9gt3ELwDJFnioo1NbCw/j295Hk7KJqtMOqSxhvv8LG4jlSCxlR28waDLsb5bmnhbYS7rkhuFgAzFVgwSw46mWbSD22lAoBdKrqjnuNTJT1tXUkHvfKu0/fjcRHhpuIUz9c4rf2KxlknqHDFUyyjLKrNb16+bJ8+snH5RT3M043JzwDSMnouIfJPdTf3GdjHcbf+pUBr+2y2jJIb2fIrXdV65QSCCxi90K7kHHXsFn1oH33BqF9CrqTGcLa7hcBCrcxhlda8QigdAsQa70LSI3nADsQ1atsby4/axukN6ZO7VnmQb9KUVRJEjzxOpIJ26oqhNa1YbQtieobwKfa1VjeuodLlaQIUkmXs+5JAYNP+xN0ClZsu/MwyUu03xUYZlls2532LwKMqv4EsCGMh5MggkD7uWWudheqJgnwKgPvRo4uTpB3lFEhhXUWmwweVJtKXF/+TT+LIz91nMqkhP3ZclMvU9xn3wgr34Orf/57kBQlML0KcgSKAnRtRLSDsc/Er4CC84qxyA3wKrBgXEI9bnJpuUyxMlT21IHWdK6yQZ0+QFrxPsDiFpIL969QWsHonfhMvi2/2mWFrFXgx6sc7RvRnv//uCbOjhYtXeN9a/ymzDVtqdM/3n4aDN33Y9QJi+f+96wCgEr7vt+348M99VnjrfHol7zo1OU5LvHnqxrDr/7it39VqF/7P/5l+Z8+/WOLPzyGFPjWU+BvLjwsf//p3y7vzK5/Y8p6Cz5IHvYvesgnOhn5lRvk/ePfbMCiP/C2gTEfNFN14M6lDuCD93nBx8eBPn9e+SjLzOjmbHqb0R8sQD4MXxqAM/Di3yOvjCf33UDvi4zNdYBuH5caDekY1v9c67Ppy1hGcgGjXyUM9R2/xth9BAbK2Ny7d9IjKXI1vgYsZASiQsQHPNIQGGQrKhIRGGKIUE/cx2CUxgEVY7PLARYH7La9I7BgudBnzLZvM9s+cXpc5s88j9jHgufjg3LDVf3wzKYyBa1Odgw+YdItrB9FmQtnOudQcVmG8V5HTcW9K2QAsxoUzIU0kVm3xNxmJjcAAwBg47CRyJTUjeOqREBdfoGBDPM8zIYqRe6sLaMoo6+0YhZGZRqD9EgsABbS1Dypm574idv4bQ+ulOWMtBvavcmSn3tR33KDs0kYKldU2tU+QqaUOHJwFUi4ctAsDLwrOnm6elIYSGfIYZC33sDIIyHYxHjbPS0ERjKaljsMvTYBj2DosbNQt3AaKYuqWY1Zygww5VTt7Ag7i33sGdwx+iXxffp8M0a+gpUFgJurQqla5Q7cQL1ILFw6VhUm9frfkB/3gXDpVBlzaW67sAwL2CJoY/GQmX5XbBKgab9hm9Gju3CLkq0j+8MVANBVoWSwX79+XZ49e1Z+/KMflWOAxjX+VEETWHIb+yGXZ12jrCtsLqfESAZfBlj1tWYPY3+0Tt5As9fUsYbVgjHpKbMtqKj7PCB9gY5kzp3tApInaNduiueu3u8/fQzAupMVvOSmKg0BiEhslFgRIflSImPbA+DQX7QrmSW80q0AWCRU+hc82G4yblAv6TuQoE4qVImF7TfL1MLUCw6ODUv9n2GnZHsZof3JdMv4u+u6S+lKmyUkORvsy3AHsOVqVqZlW3NTxYBY2n4krNBFUGFd1YkB8m4MxC04FIxlh3Lid7nnuuM84ISyCUwEONJQOhimHRlnBt3aK+rMuvC0XwdY2L+ht+OMh+1XcO3oxKKwSafGb/Oo+TLNbKZH+rXfkGfCXAsqtCNK26og2nFIacUMkw5ZTIJ+PEqfvg2NHmO4/T7gYpV+ntWgTJ8//0nMzOTWfNUGmrvez2CZm6P5b+5efc5hfNKBx5BjwL35aWFbeMO1d7n3pzsaGK+PNQ1pVvPZpdk8c21x1viSg+pmvjgSoruPQ/dT39aHAWp0Ibq3DPvNX9JpD7ry/6v/4bcfWAzSbHg/pMCQAt8+CvwcYPG9SCzqIMlQ3BuD2+AMQTIw1tkwbx0s65hbB0qHYQfogAg+aH4j6j3ufDS992xMXAZb/GRQ13N3JJnuXlff+5qQGeh1E2h4xL2F1V/+ujByWd07w2RlKD/eMK91r4eax4TpwhpnylYLm/LlmQiqOx6I13L0gEU3y+jeGDL70WfHLbYGeJcBkPm7Qed7EluEiQUMkOdW2Rxvhh23J8pruKBPtbFgln4HRmcaycQsm+LNHe2VKZjbMYEFYCPLgjILKvPjykyqsjj7bzoBTLg706nEYgGVhhUY77qijqpK2HbwJ2OntMLSWAb3p1BdSB107RVkilKHxCnpnH3VXT+uBCSA0mhXYCEzrGGwezHMoT8/BZMeQJKwlWDSTHDhLG+dOZZXrIzaIcz7LrPa7vAsLZ1pF/y567cgoTG5MkzuKh11KNJcJD1tLZYx5HYJXytGyYurJW1rF4H60jPsDzTKdUba1Y5sPaowraE3/i4rGL335FFsHFTjqhsGmtOurpKeRr8YLiO52NzaIj42zPvCpWS1ZbiMxOThQ+wYYKwXmAl3AzNXxLpzl433aAMy1Ko/7apGBMiQebW9u6v0cpZPXaJ+YP5XVrEjWM6MvgZUkW7JQJKHMLnQw6uSH1eFil0MQEc1qD/94Z+yahO2KLxfxMjZDQCz1wDtaQkJwSKz86quBTCSZys0qyvBRFbpFSoyAUEYmG+zsR/xCsasfxlngUU23wMUqaImk6rkbYw25/4oGoY/vHurPEFiobRinrZtP8vMucw30gU7kAy6/d+xIqpteGobwzlD75HBCXorvbC+rQ0lETL8AlPbkFKGG0B1xgPK7GHdKq1yA7+D2EiwjK4rHFEGAaXG+wJyva/CMN9G0nSX/RlmZ1hMgVSkjW1EcJIyKmUgrGddZQqwRx6zmhpltg1aN6pvqT6lBOuQsAJHVyUTgJlf+1GkCPQfG2h++Wljpi5RS+NtSkIG7RfWv/16mnSsT/uo2ZF20kXA8GXAYh/NO94HcPlsWgRUDcpdtW9om95TEcTH+MdkwJhSJNWhAHiq8Y3RVgQW7wgsOFfp4w1YSOt2mF/roB2W7c97ZMzHc8JbqK88iDtl7gOQFq55b8kbRT8n3ltw481d6J5nA7RALRL81RzU70pzJoIaRYuleuonZBJ9z6TUcuC1i9Hbzjl5bwGSNx9GhsBigIbD2yEFhhT4ZlLg64HFbwks/qgbC9soWMdon+LiQMyNw3Hc4qifeuMA64ew2Ve0j2I+Irh71c0BPuHzS+Rxa0TtRmMfc1uH7epaZ2RlEmB7a7AWlmv9q/GYVku3RcUnlY+3ZwUWDew0vwErxsfRyieLk480bt7rnplF4nAVJW0aVAvxdDlMmdfstcCzH3CZgKhPGCnAYgpjyUmAxfj8avax2LthDwskFs9goDZR29iFSZk+PSrTByxPusuaUVzHUYMaxc7CHERhnoxOAmKij0062kbI8MzkxL4ChkC1D5dqnUNPXVAh0yrz4Uy0S3CaJ2f3BBaqI1Vg4WZ7TkpXUFErQKYFMAKzZAOS6DLOSg80xnUFokUYYyUWk6pBqdIjsOBPv1LT9tFOySB1VY/JbtnOGDNTLTAStJiBfQCCYOOEfGV2GiZVnX1prOH2KukJLJaY6a/LxapS4+w1y80CLF5sbpbPn72ICtMe6jyWzfS1hXDJ2fefPMTG4kl5DEM8DX0CdKzk5Iy2iV8NzU+xDzg4YEftV68S30efPw+wUG1mFonQ3Xu32F+DXbOh/zXG4dOogj3EbsI9M6T1JgBnD2bTU8ZXRtNletc21gIqlpQmwYwvAjDm0XmP9ATap49Ao7RL6OISoIIKZ9iVWuwKLJBY/Mmf/knUrJzBXlvHhkF9eGkIUyuAULI0SXgZXRuuOyrbDgSl1ovG0B7WhVKQLU7pZ7uVcZZZdunfQ5hzme5rZhsMM068Smlccewh+2So0rXGClRzrLBljRufQFBDaQGMRtjucC6FbUOubCUguETaElBsnJTRla+UimV5VTxHRYr+4GpOARcYsDsp4KputnfbjEBFo2uXixUEKWG5AORdUp+u0qVBOJWRPrIOqFRacVdazaK6Nw5UgVbGbxupalM1/drnoT1pCDCa+t08TLjtxTIKegQkDYApnXJjQd0DKhq4oD1lciFXssPV99k3gvscVIjx2vcmWZVpmrYqsLCMHranJhHpBaF/2VZ8F0Ah7YyXKIGlgCvC2h6QViixsF9mUgf3LCJB25ukD03QlmMjhNrTLdTEXBHqfWi1ThuaMw925S6byQw/aZtcv+TcXue9D/altw7LjkPGZl/kva3Ro/NLmfL05bBx7f/obdBLQuFQvy79b4x+EmX96SLA0f9EMAAsBrJibhKLNx55Vz0MlquOctWLv827s189fzq2zHbXX/2lX+4H+ku+s702mrfrYJK6mVfbkqeA29PV+IbHkAJDCgwp8LMo8PXA4rd/s/z+D34QlYOqdlA/AI6BDjhNJSXP3dBZ39XkHG4dYP3QhXH3yql7pB/ed2cd5esH1tA4+1sH7navk0d9mXiIiTiVPFS1psTP4N2LF18J3oWLeyKpZRBYxE1wAYMi09DCJi6zkPRqLAEFhLfs7bsgLSyfDE4DFZEeyGzDtM0yG+zeDjL9RBbmIQaVxGOa0zCTSizK3HLZvZkqb27GyubNaHkFU+U+FkcM5uPHLM8JqBjbflmmABYzbJQ3raSCL7y8glkMjSmP+XaWcxpmRLuHBdLWAHMRZkF1HWfkm32FTJjMkyv2nGIE7Kx6DGJJ29lXmb3YjsCMqIYhLQQfNhyZafXmZVwEMpbT/RFcWckZ92mYxxhvMzMaSY0Z9ZB2NhQOGdZ6CLbUcXeGl3xQ5qx0Q5oCHiUPu0gKXJXIGWBtHlQ/sYxLlGuN1aFUg5LZU9og82petT9QpUdg8ekXzwIstBNwmVw/rEpFVA3ScPu7779bnjx5HPuQ5Itymu/saUF+K7CAaUSi8vLVywCLjz9/liVYVblxQ7f1jVXUmNjBmHoYo5G7idt7772bvJmfHRj1XZjWN8yIv97Zow2xkzTh/FgrTVoCZLppYYAFV3cVNy+GzUk7rdfKdMuIm689bDaeAyyUWGj/co7dzTKAyTgFF0otbJOuhmZfFpBA/Uh3srQtdNaeofYf/MABKSF6jQrZK6QzSl8EFqoXyawLLGSWI82CRlO0sw1m/e8AkFzKeANpxcryKkbjC6nrgAPaimURzAjexrjaDmxzGp9bhn1sYXRTyjeHdKWBBelkPch8Jx9IMlSLcxneMfqA/pWYyYRHKkGb3sOGRRDkLukHh9phaPjPVADx21enAUO3AyrIN3kXfAtmSCRpBMAiIcmVtqLamOCk4k2lXdgUoUa2SBmVXnic0w40Tred7rCk8CtW/9IeRVp5BFwIysmHYIlLBRiUS1oKBmTi0j/oIgEW1JsSHsG7p3XkkbGoK4/+bRe+M0yAhXnmTJ71rzwG+lRQgZoccSpxyjjmFQnFCIshZHUo1aIok8tMb1APD5FWfIDK2IbjCHkQWDh2Dh6m3452n3J0jj/lVocA3nY3A+Gbk0GNtUnLB+Pz3VcfdWz2nWGbHV7iIam8zYv81CiSBd5Axxxeea2P+txuvnSlP3UhiJi7LnzK2qIaCNLK0ZyStwSrnofAolFmeB1SYEiBbyoFvh5Y/M5vlX/xL39QB8tM1zn4OfjqVIGFHzdH1jaUx0c0B2RO+Ph0TJHMqYOtHyO/H031KQM3DnHjXRvYfXZYb1cH5AzW3bWO33VWyY9cD1h06URlJOETUfJojDj1vhYjRJpHE/G/AxVk2pBJD6fcdxlBl9kPOsCiRwcjrSpEmVnko5sduGUCmKmXWdFOYJ4PsnYEpuHsoUy0cTrbO6W6CKtCXWHE/fpmsmwBLF5cjZRNZ4hlDmDIRgEW12+2y/XWyzIDsJhnRahZpgxd1cZPG+xOnUHs6qMBizlmYVdYAnQZSYXnPIyDM+SuwETAAAd3qN7DQPwIKYEgI3UbRqoaofosPa1LZ0tlXtTrV5VJRsj3AgslM6rayOAvAywWUalwTwh1/cdg9i2rTI8Jq/KR9f+5mnd+wiipvqS6TGbUJS1pKRmpS8gyiwzgEOzEoJj7LL0qY84KQ6pfKSGpwSozbuyWS5uIZ+xg/WKTje0wvHafB4GRjLZL8Gq4LbB4zN4L2oeYT2dklQzY7sJYK7GAPoKcTeJRavHsBepVMMOq3FivMsMz5McdtJUUrCFF+c4H7wE41mNofkxYd63eBciZD9vSBOomi4DLBZi5RYDF+vo6Up+qCtXLC4VK3yAv5kfGODOJMo6AC5nylxhvf/jRh+U5+bJOrfvssE35psOQhpxEhBoZ4ExQ6f4i7g1h/9QOxCM2A7Q9jeUPoJ37mVQQWYFkVuhCcqHKjyo3tms3xbsFk34bac0t80/9Cyq03fBoeZVB1uZGO5hR6KO7wMLN/TRsPwJwmS/LJx2V+FHctIfWZgwjANVgW3WwSAlVxRIYEKd0UuJW9z9h35HXGO4jHTo5AYTRZomOdlIlXfdvr5V7G2waCLBw00HIQLu23VfAbFz1lDG3NSlxtC4YcwA6gjIldbU/0U6I3/K4RLLG7w1YCDgsEyED3NOOaX+WtcZX09EGJACGPJiuvdtw9mfHF+/Th8gnWej9mB8Lpn/bh7QSkAtgIg2h79IBARWACTZqurb/86wEA+Fo7gUWhbFolHqbyGpuqnmNRWLRgMVtxjH3sFCuZb2YnkfG5tzVn96z9NOpy1/PC8+OW/mvUVR/PQ/1pntVExt8l0gN3t0MvmtuIYrpDBwtPzrVAnQ0HIirC9eieSu8Ybqjpjz4++W09Mh7vXThGrBodEtDMk82AqIeAouOuMPLkAJDCnxjKfC1wOKf/PPf7gGLDHwwJA6EjrtZ5YQR02sdOdtvvHQAon7kGqhwMPWDk6F5cIDGoQ3XxubR/FT3fILC3CY8X/8AGvz5qRZUZJUSPJtWmELTSVpea56Mt5dQbru8NGc9espd6Ma98SVQdarAAheZSI/2DUoeVGuCEVJdISpQ3LvEq1KCBWcCZZJgWNX1NlziR6ViDMa7wGyds0Pj9vV4eX49Vj4HWGzDdJxmShOGT13zN6/L5farMnuErQXM5Dzc68QozAfqIDcyIjL5nH7IBACqvSyigrPGjPwKzII69oIcmSAlJZFWwIA5++wGZO6KLMMmCZzN9/0lBQ1TmAzDBMPcqH5kjUiCMF3caUuirUCYY5YDVSVpdXUd5rACC0GF6iNKAAQoVQdcNQ1OmSvynTRh6AQVNszmpnRCwKEtQTJHuqqryOSP0CZlfjJrDCBI2Yi/Vo/ATbUfVoiC0VOt5/mrTcDFdjaoc5M/wcMyOuTvs4fFd957mqVeXdJUplE6WEeWUVqYL1cdUk1LNaGc2GwILHaYoXZmum5qBsNGv5ihXl2B6zvfea/cYu8QVdBkhN0Z/RA6b7/eFVLBDCPVgmGbgunTgPvWbYyeYczdOK/ae9DKyaf0sP2aJxlNHa1vVb4OkC5sb28hRfkC24+XzJjvp2+4tKxG24sw/s6OKm3yPKPskfoQ3rqxsGFGB8opcx4/5LeukuSMv5sjnkXVyJ29adABqsu0sQ129nZ3bzcsFOTV5YbR4ydd6eFVtatJ6KK6ms+xSyAvp6jiabNTpWSoDfEnI62kT/BX1e5qW7mhvcdGhD6gNEFwqWRDwOzVsNb5DmpIgoqXLAu8yx4mGlQrjbMP1r7B8qn32NPlFmAIOjk7T6aiWiZwbfSG0GmzAca04czy489DCYx1UetDAFrrQ2mKO7W7tLBqUQILpSrWmcBBVS3LKmBLX8DdMS2rSnWSwqRPiEiRSCdjG1dBvvVlO+CxHmQnOfKVvRN6J17rGx9Z5Yz2BWLIMrLX3FdQQV80PstlnbjsNeOEYNOJEZep1sbiEWpQ30FicRuwutiAReKt+ehy0buYtxyUyfSt67h07k2KUDNX857892Lo33Qxpcx9VyP1/6tCJcUu0vpemvgf/12Q5FE3nqtT90IHj+6ih155uvy3V12Mneea0/qr00A8caReuDZfXqWL/obAAmIMjyEFhhT4VlDga4HFP/2d3y5/8N/9IIOeA2CYko5B8CPYNxrsxvz2MYE0DuR8AuvHD/d8BH1fR9XuUh8c7NvAnYG2i6e+7QIYp/E4k8yHMP7iVoGFzMeg1KIHLrpwvY9EL31z6EN/8HfQzxPX5KfLt950z4mbzGBmAQnQhQ7zEyNmmAdnPp29n+UMqBBYoK4zOlaBBZHXtLnCLaCewMcd5vJybKps30wAKkbLRxcAC2h9xjkm4w0jesk+CleAiznsK+bYiXsWmDPORnkj6KffwASqvw43Qe6gCcyOOuOrMHy3YfaWARbqgrcZYCUNbSOwA5hud7Y+ggGS8TUGmXyZPqNTuhLpAu4akKpeNSNzIUPCIS1cvUgmcJlVp1YAFRpRLzLrLnMpoKr1IRdE2Rs9CSfzlNlw0s0qRTB0pi3gkJGXsY1uL1fTyY7egCOZHhlRGU7tPWRC3elaYFHVoaqkwXq03ToD7SpWm6j2vEDa8OzFZtRVDO/KQBpuv//0MbYB99H/BwQGPFHn5lFaJF+VwT7SkFwpBYyjKkc7qFZtARJevEJliLQsoDPFGp6vI0l5/73HMch2ZlvGcxTaSeN9wIjqPc7iZ3la3FRRW13fYHdp1YjYYRr6VZ16wTShaHCWScDkYdnMn0blbuC3hRrUi81XgBY270MKtUT9Cy5uIUmwMjW2t51I00iHKFeNp7Zv24SAK0vC4tfyy9g6UNgePJvtjYy69jPuIbLqamOk5WpjruykClfbObyCxNpTdIvUgjKr7naC1EPQp26N/UDaHZG++bMdCIrmBcNIxIxHIJH9VrQJ4tnDWfy2QILg1mapX5cEfrnJRoSvdgIs3MPENuSSugL+FVTWnty/zYaR7OBOHxljYsDy2iYisSD9BoiqdKJK+pTyDB4Zi9JHGBMFFrRH6eeCA7YLgbvG/Y5PHrYRVaO0UbHtXxAmwJVwgk5ByAWnfcO4Y8tAWEMPjksVYCTK3jiEj+rQ/RI87+wbLn/snjkBFRqzM/Z4BlTw/gpgcQ349bR/uammiz7cY5nZJ7Sh96HRrVlstQAbo+ar5Wfg+lbi7aFlYiBrubVJdDTRq9+BtJK4t8BvX1N+nAwvbar//MZj4uWne+PgNECSvK0RdvT0wTbRspG6NL8eg0GrS9ILOOgCxH/37qcuLZ4vvUgukoR3Na2UBf9/7+/+R1/y/Zf3aP/q0bMRYCA535kv26FnxmH65dDGYoBIw9shBYYU+CkK+P11vPjK5Wb/6W//VvkDVaEcph1gOOtMmIxmBRYOi54OQjkd8vPf/xA6ZvmUQSzjaB1M+yO+nwEH2aRkbL1XNZa8CqiI9ENmXF+dN90aqFAtSkZIZsz0nB2v0Zl+jacXecsLV2+d2/OjEf94Ttp5rgAiwAOPMoWWX2NIl8s0XteXV1JRV0dCPYIP8jwfapkipQQylc54Rnc/4Voc0JL838AQ30xMl4Ox6fIMqYXA4vnFTTnEANj9CuDCyjUz5UxNlxlWg5pDFWkOYDFxjZoUy9BeATZu8FPVWdQhB1jAPDkbfwtde5ln1bFm0JuWXoIFVVkEFerMq4t+xAy0jaExOV47Xjb1L/Uso2UTOKmCIhMnnd1zwRljN8hbjirUEkztEm6qiQBCYFxC/9Daj1WVAmisaiMMiEC1RZCQyiAt3Z01dw8L703HmX2XZnWfjLQn68y6gDnz4yfwVP0muvkwRvpxVr82dFS+mMl3ZSZXdNoGZDizLPh7ylKzT995h+Vm78Aos0oXeU6bIF1XzGoAJqsSwfjKDLd8uTPzPgbCLj+rmlAM33Gz/biU61NAyxoXFXI4AABAAElEQVRAa4Y8yziOUQcypxomz7Acr0BItR8qJQBtniU/F7FTmW/AIu0GgpghjrRMaF8/+jC/1JNMuntO7AI+NbZ2RaetrU359YAxDcpddlg4Ij0C2Ggv1nf6tTTkNP9ZphV369Z+pJQq4JK6MU1n88+wV7Cc1qsrgS0jpdI4vC0KIECyvVsnMjCGU1olsBC0CcJd8lbm27bnilOCFukbA33uPe6z6eAGO3Qbv3FEwgWQViKRtgI1TMf4tD+w/0XCAeB7uckSvC+3y2fPt6ELy+ZSJ1JvmjYkCLqFwfY7rGB1K/GjjkhZtTPRdieglTgECjZcy2m+qxoShLEdk5/uJmnavnWxDNJR8LmJnYWSHRdE0J6nqjJp+1MBhJIgJRenngE0AnrywCntkgaRpq0nPRLgyDMJJk1/ONpr3zkG+rIaZuOPSY1RpBUj1MsV9ZCVoSwv9VFXiGK3daSmARe0T4HFDPRcpE4fIq14CrB4FzuLlWnc0+9rmvW3y08yIQ3aaJ5sQfF6tPz51Ny6V3Xc5aH6kYoe+Pqyx+qat/mJ1+Z/0Llz8/JTcbR424s6htTaw7+ZyEnQjrbGnBhxD7D4GXnrUtV74qhhBrOgiz3Y9/546YcaqkJVmgx/hxQYUuCbS4HKb/0sYPFbv1n+4Ad/lFHRcTYMiIwCzGAY626Q9QPmnx/d+kHrD6QOoA7OXuvnhoj642ifcp1z75UfRcPqowufWW+ZI9LJi557/ZA6j9iTXCQv/VlrvHb5qPG2D3aGdRIKqCDFpN/5NRHD+RPAwXPKjVO95qUeYldRZ/PrsqvZLI4P9Bwzt+7rkJ2UwyCSd9ODjjLulzAyaD0xi81MKzOFZ+y8/apMls8AFp+cXZfXLv0KowNnUkZUUwJUTMGAz6IOMs9U5iSSi1FWjCqoR7G1MypRbJZHvDIw5mEJVSg3gNOOQAZaUCCwkHHZR599H/WnXWbeXTnHJTLdA0DmN3kLQ0jxOEIXroIJpR6qQ3kvvZUWyIgLNmSQXXlKcOHqUJbd2eRIFLr6UloS5pRZ2YAX4pWSopiAQuKVUbSd6U8GXuBgvgUUGso6M12XYq1G2jZkGULrVf198xOpA2nKgEZvHkba1X40SlYd6iUShj3KroTJje2ePHpU7rL3whz5V11H5kL1K1dd8tpUS8J4wlRrOCzgsF2fUU/uRi5Tr2qUalEy75b/EXGrEiWAOKXMFI5JYvJHebSpWKBuBJ7a/AgItUsRVLhcr0bOoR1+0x5JK/2JZ9tQY9zNn+pNrljl5nav2SF8E8mFUhpb/BIAx7p3FtqYZGxlagVWSkIaE9uAhXkXVMisp+1aP0m7SrAEC87CS2PLaNlsczE0B74oUbHX268sk/5U+XKZVuO1OG9Qv9t2Yz8kC4IZJSHuTm68tjjbzaMHd8t9mP+7t9ZT/0po3BRQ9S/zbJ4a02+7VMXJ4cE6++KlqmEY7XO+YUlgDewdO5bpDxvYVTwAtNwj3hWelYQRFU2wAl13M3cFKssZFSjC1UkBmVBpoV/Tp/YJl1W1HCfIn0DBfuWmjFsY6B8h2dFGR1AuPbVHMl79nfDumD7nZoAnSFSqsbVgTPBN/I6xnNZ5HbNaTyRpK9KfXL33qO0jEyoSWWKYd+wqBBUjtLtrN8Uz04IKaHZDv3XpWUHFDeDimjoyr0pc17EZery6VN5DWvGQ1aEWkFYIUNuYnhSNqzuSx+6+ubZrct5lv7nptZWojrE8NQfK4u1A9DVmHHtuPb/ctHsi7zHrA241cJdyLwJdK40TgZWbMPVqOdvRyubrQfe873vrZaPWVw3df53QvSAtTh2M81eG+1hUgg1/hxQYUuAbS4GvBRb/5De/V/6Fy80yKjoc8s2tDHX3scug2A28ARV8wGQMO6cQpfs05N44/FAa2VsDqh+Q6syr+Kr+E1f3EfOe6deaDvf5q4OxA3K+oeQzkgvyIfPiHhXV6LjGkYxZFpPo8mA+wlhxrX8mXYFKLQsBiP9tYFFzWYFSjTsz+XyI52R6YW6ypwPPrhzj7tAySZm1J1+mrUqHzN0FV7PjLLb7WdxMz5c3o1PlGepQPzq5Ki9OL8s+DNEIjB6cDkvMchLGFaFmsK+YYTZ96pQVo9jfYhSphZINwQVZDsO8oFEtTME6zJPLgcpYytQ4K6wBs+BCQ9MjmZswOTB3MHYCj4AemBtpJM0stXQPjWEcAwIoj4xvZUAtt+pfc5GQuPSs6++7ao9WGZMTdcb27AxQwyyuM7ky/Eo6NP5VGhF6wQDHaJlZYmljI5XJsi1JQxk9mVXvBRBezZ9+PHvAk7Ayf8m/V+imDYkGybG3UGoBY2s579+5XR6zl4W7RWt0LLCwXal2U4GFeah0EFReUh8u0auhucbIZCq01L7B/TKUihxTPlXE7rv8KpKQFSRHZ9LVQtE2zbsSHu1vnOmvoKletU1xSVbViupMeZW+pN0T1jJGDYpyqXal6s4ZZXMDP1et0hjaVbR2tpGiAJ4oBHYo/Z3JbfOCNg20bQ/0gpRXt0hcYPJpQgGRAlJtR6R7GCrz3x1VQlHBY1QlaEdKBoxHtls1uFVW7JpH4uCeJr2VnaDd8xcvIjn64tV2VKJUHzoFCCk1sP+6atPDB3fYcNDla9kJNOlTp9oUWZ/JA20LRliJWd2FnY0YAdwCo89YXviTzz21OUHND9pbH7dRfTLOJ4/uxyZEUJFdzonPfm57sV4jpRJY2P99F4ok0TzVu27soT+YJyUpAetIAd9g47ELYBfQRJLGRpKCMPuK6k4uS7sP/bW3EGAo+RJoKx1SXcq2LNBp/c/0vOcnSdsfvUudePXPjt+5meZboALwYDtVihE7L9r4NWPTtXYX1g2SpxHaoWGcOFigXz5ASvEuEgulFevmv+v3AS5JJ8n1fpK/3pN5aod3Nd81pzx2tNZHezN4V0NbJovc+fAj1B21qF0K7X33rlKmhgtN4g16DKSUiOO/0rjeUssDcfXS7eJtl0bnt94nQzV//vbe8dD8J3yXZe/jx/Q6P0OJRSg0/BlSYEiBbzAFvhZY/OMAi+93HwdnHztg0X3s2lfDD1gYea7OKtcBvz9otnE6g60jKP8yZ4lwkHh5xQ8R8JurA3IzVjRuGarMeOZ1/ZD63jSdI80MOn406BznI2qYmiHjrJ8bB3PTbx/tNhNd0zQe06hptY+S76rqU43D7MW/NzB6Agtn7N18zBleGeowOny8ZSacYZcB0q9pV1BRjWFVi3HlJIHFCMDiCGDxCgPuH51el88BFluonVwxuzkCs4b+TJkg/ATFmgRYTKIWMstu3LPswj3NdUSphTYXMCiqXqgasqYKA4ydalFTpCUzrxTAGdVsBAZTE6NR4hdUOGMcCYBMMPGEYQ+jY4kruGgAUgZDAOcsrDPGs+hguxGfzOgsjEhd4tT6GKkz5vg5PUdVhBV6ZKSsH2frF5gZzQ7QzNCHWWX2VPCQtiCjBbNpfTnj7cyxObEcl0gKzF9qTeBJGNU4nDG3raT92DgIcIP6mCou7ubsxm8vNlkhCsZbdbAl6HOXZVKfsOu1+2GoG2h9ySxeqQpFWtUwF5p4D41kIrOKFqnbNmQIVevZ1Egc0CJwM6/OuD+6dxdgsUT6MN32H/I0BUMXQMZVg/GlTv1JUKPRdsAGtNFuRPqmLLbyrk/Q6sOYWEfaSwiajgARGnIfADCOAJrub7FHPR/h1tql5XLpY9uh+a/2FM7M17ZZBwWABemoOqf0yU0XKzA21Y6eENW4Uh+0D5csPgSoClZU63E37XmM0O+xp4U2N32wRhsjrx9/8kn55NMvyidsMuju0PYJAdsNp93WpYQfYAPxMOet0Mo23doejYMc0P6gjRKugAvCWNlKkz7+9Fn5+LPnnC+j9kdxKc80O4KvIwm5U94BWGhormqdNhS2KQgKXcgLeahSEds7btSZUgQaBE/8GVkNQTZol+RLNycMdpBcbW2/iY2H0grdBZFzgGcnHMyr9h7aXgjss19GwFidaLCdpc9BE6+REHdjbtK1PScXZrg7bBO5rVfrjg4QlTt2umNpWVTTVLfsgIXXK+o2wEJgjGRViek4tLCtOY6t0Y8fr2CHgo3FA/un7YaKqe2wS8c0a8KSjqM/Pvr05aOXxxZGDwnXfHZ0JbIAuZS1lwR+G927CLrwSbVmoEVUr/HGT7vi2pILjeKr1acvvxpY1Pommpaf7mrw9i5RxZ0U8l+vBMqr/JjHt/Kinzqm+n4ILPqkGt4NKTCkwDeTApWH+BmqUP/oe7+BxEJgUQdGhkA+up4O4z7VgVYGzlnrpkPsOOpg61mZUu8z1iZc7n2fj2Wi6VKo920pWgdkQUv9SNYPWgMOlbmobhVYeN/phBMmS76Sp8w28sI8D56m3c9bfdfK09uN248ocXm4JKOLcZp3S6JBpbd5hHmUWXAzulkYG5lFgYV6/qrtaNQs8+zsrZmUcTiHAcnsJCypeufu9zDJh30MYHEKsNhhyVmBxScAi89PMSaV4YKZJXBUEQROYzI07Gcxi43FElKL+XMkF9zfONuKfzOrupIGvOqUL8MczJBPZ2Jl/rZQ21FlR+kF/EsYGNWUnAFX3UdGT+Y9jE6ABUyV9cYZ+lN+60YgNggs5gEUGo43WgguplChUDVLsHWCxOIIsKT6kOGkTbWdkE4y1NACuskoyjSahupM1pcqRH6ZnWVXIqCqj0awMrcyvzLPAjkZXkFIrrYD6lGjZcFAXbEHAIAdgjtoq8IkGJHxf4LUYoPlUt1HQlUrkg64sKN4amyrBENJhvcxdCZ98yPzeM51FzUYN+PbBWSY9/uAivt3bkWSo12Bfpw5nqW+bUf2HdWf1m/dDrjIPg/QQJWubF5HvJWBpz36R1lbm4cYqR+BhapQRyxnu/eGekUl6hhgcYTdxYErVlHXgjPLI4CYpC6kifV8woy5ZQoQo81bv9q8yCxrN7EaUFD3YUl/sO3jJ/2HOrGdKK3YxVj8gLIf07bOSWuCsCsbtwAHD2LcbX/QzuIawHqBSt/HH39UPvr4s/ITwMW57a9rW9fEldW+MKZ+cB9QlvN26tbVx6qdQm0XsmRNgmUftN8JBt3I8CfE/eEnzwEur8jfRaQVSm2UVkQScv9ulsW1zRmHnTllSjupK0hVJp+2R58QOFLwnJS+0kCggQTFeuFN2oFgVXubza0daKNaIvWrJA8JlNJM2/QpgEObJsFF9mXp+lvGBNqH6eWeNM2DfbL1PSsxYyLp0RszptgOPGxPueoHWtNwOGnHAAs6SNxu7AvWQ4CF4ALgIagATDsGuTDDKv33NqpPSiruA4g2yLcbK9rvEz/lrVd+ve3STX76j3oZOKqnmvdaBsvUwtZo9KNbiwnX+l/j0X/v4EUe65jUeei9zU2X3xpJraNBDzUvA+GJ33yYfi7Ji4/9dBvt27VmoXsv3RO8y7/xdXl425+x176W9HzC31AVSroMjyEFhhT4JlPga4HFrwMsfv/7f5gBz+HSgTGgohtDHUEdDGVAZHzC7Prx6ZgTP9IyXHX1qDo4O9wKTvKRDLDwIUMssXcfGwfi7jT+gItupsy4m6F2Y670o7tXwY2b1LnzsQytM+pmV4Y5KgZcI6HwyqDfPtZeUxb8W57KvFiWGj7Agmw5e+g3pi7dmaxn9lk1DfcKmOOjLLCQyZSRcoZS421nRZ15bnmRGYNVSDnDQMpMY8g7Nj1XLkYnyx4Si5+cXpUPTy7KRwCLQ2Y4ZQJHYDCmWBZzAqbLMl/BoM0AKpY9zw9Qi0IditlgmUxBgXR0plTj7RXOWfJ0BROppCL7OcgIwuSE3mnJ9WPnevzJI1fpFlDGVfWblN8qwj8sbmhkWZ1h9hRQzAEulrHvUGqhepTlD8MPY3gCk3dwDGNFmQicvQDCxEMO25GGv9pqNKNm3UJp3ssAWn8y9c72HqBqol2I9aVUSAmN4EWVl6wUxL0z5QIYZ6BlmAVpzu6/YTZ/G1uEbRhBmTeXg31w/06ARXbwJqxAxnYRYGFYaOfKWbbrBixUJfPeMrpMp4a4Xzx7HqN4mcrHjx5Gr9+2tAODr19BprPlCJ3S3qZnkSrBhC/iluVnYfKUWBje8nuG8a+dLtVlvjxtu8Ypc68B9xsA05udbWizC0OqkTBgijyqAiYQUv1OECaNDJ/dralnVZDsU0ZunNqeWCZ3A9fWQ8DT0qvld38RNlcE3LnXRfZCEaTY7ijrFFKrZfbvuHPrFtKyxdTLJH1AqG4bfPH8i/LZZ5+XH330WdmjPZwhEcjeFtB5hDwqQ3hwf728A7B458Ft+pM7hEML8i0tyGzyChXSLgLQaHvWs9KTH/744/IRwOJzjLdtv/ZF9xV58uhe6vnO7Q1WVu1W3gpgJV/kXZCljUVT7QqQpO3b4qsdiS1fEGIbsE1UqYT9xOeXAIpNNsZzYz5Vq2zbLqTg4gn2E2ms5Mp2qwqU0gtVx6yXOmZWUBHJoeMnbbMCHPNgkYm0G+/aM80oEzXpnD0/0AdaaT+hfYVLWisdvcHGIuDC8Uh32tmI4w9qebOcK/TZ20xCPFicLe8wZqzSf+dtf6RryVsbMG2HMJtM//A74V//qGNr/zn5T0wEdzDx4NKiMXTG1/qmlre7rxH3Y293b6XR4iRMTasGFjJUldYWWT/F5mJGGjjzvuXPKJvvvt+OFp1DzUPfVwvbdxkoL2HMe3vX8jkEFoPUHd4PKTCkwDeRAl8LLH4NYPF73/9+Pii9wjHCtsGcUTsfeJl5mfhqzFslDA6qMoDOuqla0Ay+404EXjPwGpmjtgejbAZYP5zdmdlZP2gdwyM/4QdOwJAPHV4DAjpAYB6yQlPyU1WhyEY+zOrSO/NnfiKxSB55Nm3uAyiYybM8MikNXJgv/WhSGga7y7vZNue+M4xqRrMwYzFu9gpDEVsLZgZlEmWILJdxCLYsk+olqsFMcU4yaz2O5OJibJKN8UbKxwCLnwAsBBfb6q7DsN6wSpTAZRxgQVHLFQzQ5PFBmcPGYuUUdRcAhsDiHB15bQpk/GeZfVTVaBFGz5VeZCSd6Vdt5+CAVYBkBqFLPuj4tw7qzK1SAoFgH4RR1N4Rhk76U3alEYtz7PANI6K0wnOJNN2kTYmEzL3MoGUXyOzsYtzKSj3adsiEKl3wCECDdi6NK+O9jBTB/JNM8kLWklcNXTWM3Sf/zvjKgBNN6kEj5UUNTUlfMCPIi6oLZQlDHGmDNGBGHwb0CFrIuJm2y6WaZuwsVMciX7bBrEYE0yko0ZbBTdA0vnWzPplE6ZVdxsm7s9Rbm1uJVwDx6OED6p96he6v2LxO9S0lWRsw3dq8CDhn5hbKAkvMzrEilIyo9hXTpO8Mv+0w5eMahjp1lO6SupJu5j/2EQImVLHevN4KuHDncYGF6k4y/geU+Yx8GI9n+ht5h3S9dFp6tf0K8AA5giZocUM6PbACqIjhvxIRJCXV2JleYtumXJOEWYCWMvMCC0HKlMwtdXhFng5YwWprWwPrl+XFFjtvs0rZOR3Kfj2On0nO+3cwsr67yrkRqZLSvbrHiIBfJraOGQLHqJVBf1fIUtXtT/7so/IJalAvt/cCSDTavovk6CkqUHexqXG3dsvmDtMux6qkQ/Cu7cwpgP2EdiHdlHSlb3RAJmMXfuVAAwRsF4TLRpPQ4gWrUb2hbx3RByPJBFRENZA+4rhiXLabtuSsO4IHoJgOtEuc5oV21CQYqmHlnbS1wiWi4IJbaeBPmzDJuKobaYFgMZTBMH0K6RjtKTYu0o2ALnFNo+Ud5ad9zqguKY0W58p9+s8Dzg0nRhgzptJWbHNJ3djrQbq6NNfkhQevlrMd7X3fZ3sTj/VH781jP2iKWl/gGPf6spUzTy1ci7bRiQgllUcNxU1ziGv96WIccOluiSfptMA8ezQg0PnKJW+Iu2Wl5c/+0Bzb2xZdL3wX71AVqkeR4c2QAkMKfEMp8HOBxe92wMLBMgOmA21uHLDrrLkfS2fQZcwqsy/z7IdOQNEZI3JfmVQHat75k39/OD3aoNwYHp8Hzrwn7SadqAADJgSGS3sKGaIGCpRayPASPGlFvcdZ5gCd+rEQXGgkmeTx18JOEFdsA4yDOI1EP5cCJG4aMPDDYc4tq2mZpqpGUceBOezNoMPoOtsahsg8cRqu7ruAVANQoSqM4EJgccXSkCc3o6hAXUUV6kPUoV7CjO+ictNUKyzzKNPdVxoPsyLU1MGbsniMSs/JIatEHbPvBTryMCqYHQTczMOoxkAaxks1Exkbl/oUWCi9UM9bpjv5kmgcKZ9lxDH1hbP1EbpTBq+qpkkjQYUb8bnR2ALgQoNYbS2UPGicPeq6+UQkA+YKQNn3AQbsJev8n5FPPEAPaQjQIo+uYrTOSkouk+pO3qkH8m0rPIcxP4AJfbZZ9yc4BmSYY2eq3SxwHjuPu7dXyx3CmnZmuGUK8dVUWvQrwy0TKlMX6Q6ltw3PIy1wZagpmK0AHsrZGEhnr09Qs4mhMWFl/qIiA2sugypQNO597Bq05xDU3GKlKetaGr/C9kKCuvfD+sZadP412p6eY7YYUDENwNCA3SVcVQdTLS/2ItBGcNHrD5QltdTVifkzL4KcffazUGKxs/Wq5hWwIZPsKkxKLdyrwjYdcE685sd6dDY9tkFcBTux8SAvgkIlHPoLSKFtaaNzIDglTo2V3Rk9kwv0nfRPxwPyPytQwxh+gfLNwcQHYJJnwa0z/e6K/gZVtC9evi5bb5B60M7tH8al9O/+7UV2xnZH7yXqsdp5uIu7h21VNluApH8lLLYTlwJ+CZ3/5Ecfl89fsNTsQV1S+DYrQD0G5LlKV1N3C+CnrLa/qB+ZL4CFoPOY3e5PyF/aFWlFjStjWh3XrBcz4XK/ggSlNu4H8wqJxR400c0xQFAhYFVyJwDyqEbySi048Wd/jIQQGqfD4ScSVoBcVjVjkLGO2xhqHG32vfZT2z/5in8pwyEoZhxyQQgGGO4FdTXPjlxXgA6XuRZcjAL+5wEV7nnyFGPtd7CruIvEcaEDFY4jjZluzHHSIKm0Qx68mnJS56GOjvHV89P3Xd17v5TbotMs+0ci0q2fYmXWu5gN03wbLmFNuHPlash+nBWIJshgOjgMxNRizDXpGd1AnEb4peDVbxey0anmtQvbBUjI7r5FWQPXPA+BRUfE4eWnKPDF9kX5ZPO8/DvfZU8sP2fDY0iBf00p8LXA4h8qscDGQtUj23HGQ8a/tja6rTs2D3x11H1WFSozqvph1AwDLrDgwxt1Ghl5P5C8y6CqP+IdHNTzESHeDM4O4Dn5qHNNDrhUP7wzfdxN11OAI8DoAQ3eGbdpOvOnOlSAhR/o5K9jmJMZvsOEl0GRufXqWRmBCiwsQ8CFcRKmqkXV3DdgMU3YqAVFWqHUwhnzqu8vbWQ8VecZ4aPfgMX0ZLUrkJGd5LxhJvF8ZLxsso/Fc5TPn2Hs/AUMyCbnDrObYa9hBOHdYqw9iqHuKMBi9rACiwkY35Hz4zIOg6uRt+pB2nmoirEMg6ORucyfDJjr+7+BCd5BbcPVgCxjSN1rsKl1SkxaARHQBVpH3aujkYBq2dlNdua9C7OsIbRgyZl3z6rKVJlzmScZKVVFPnv+snzMqj176JlfQk/1/pU2zMKQLpHXNTZdc4O5VZgdGTQZO2l4htRm7+iU8NpHKHVR/QbmDkB1w74ea8vupH0flZe7MHPoz8NMVVYAQNe1TcupSpPA4hyJhypOMnDW4zygIjtHw9xH957QtuFqo6H6Sp3NVlXGdN3LRH/j5NvDTehc3cjOpfsC9hqX+FGyIuCQAXY1qCXKZ/6s9ymlAhjvT6EKpzQjwKKrp9puatm9T/vvSmR6tkD7m+3RfMqsCy5eb2FAvoe9BfdulucytEpnlFik79AGbZsyxh6226igkR/T9xRgSD/38pBWqhjtMSu/i2G6zLOnAGoEkCsDrbG3fcYxQgPxaWfrAVGqA04DmgVqNKPQ2nZWV1K6YCEBdqlml+znMOXHMNkCMjd3fHQHULGGmhjty92jLb/twHpsh+1LQGTb0dD+NcvXfvzJ5+XHHz9HqrWf9rIOyHznwf3ywbuPy+2NjexsPut+JeTXPkkBkqeoQlGeI0CFK2wdcl4CNHgZkKf9tmOJxuama+9wqV+BlZI/7SZcEcxdv91xu+7tgmF+wDYAkjyOo8pYAXa1s4g6FH3CfuG4ZP063hl3q9sAR9K8pO3brjKedgQQbGRs6oCF4MJOPELduSEe6I7KxUBblSjjBKAThAkMQLDgQmABHRYBFnfY0+O7LMX7ZBlgwWSBq0AF3FpZ5Kbmyuh97g7iGnjClwc57xzjN471zdu+uzgMYdl9bHF33p2G6dU3NzrnVV9nKVH2/BiF8XDopeXVa/PDvA1HTaAHAHTi+PKz3np5473xtPi/0r+OHC2eRgfdDJnidXHqNngMgcUgNUr54eeM8zDUGwvj5W+9R1v+muP/+gl97+CyPL41WT64X8fir/H+jXv1v/yf++Vf/d+H5b/41Tt8I9OAv3FlGGb4rwcFvhZY/Nr3vhdgkZlpRlIZAgdZmRJHR6/eyow52x91DUdNB03/GMXzAeSDWMFFe67gQhLjlbM/TDtoh3kisfwlKYFN88PVPqU/mAFnKCc7UBFwYT4EF1zNSsANH1oBRT6+fpw5G7jJh7xGB8PsikIwKZw9YIGbEdV4ACid1EJGOMAiJajSDiUdMtmGb2vqR9+f5zCeARjMqsI0kXHo50emztI7kyvjNQMTMAZjd81KLnvXo4UxFYBxXV4w078JqPA8hJnNMrWscjTC7DHWsmUERmga+4opmOspVocaQyVqghnhyZvLMOWqIwVYsOSsu+kKLI5hfDQw3mETr503e9WIGzo1+kMW8lfpbl4FXgEp6GBrQzHV0cqZWBmnVZZTXWNpW42SLY+Ml4xklTb4Ue8YIBgjd8D+FAPnP/vJp6hEAWpwq8CCuMnrEnYhbormHgNrgAvtRJxdNl6Z9P3j8/IpwOLl1pswpecwZe4+fXN9zi7Kc+W77z4s7z1+EMlA1KDSmmpdWq7kBZqq2uTSogILOwMRxJZA1aVstCfTRbllvmWuBSPZa0FQYlhOw9k+3FlduhJ5wEVmuqnfKaRR57RBbUsMq/RokVn87JcBQ+dyta4Kpr8p7Gy0uWnAQvJXNa8K2nvAgheNrzJte1Fto9cwuEihMN7exX7kgCVnVTnaQzXIJWgFk6eUVcbffisDrz679Wx9SWPrz1WxBIX2QZeyFUwdACrc9M19OgQWBxppQ0P7+gTMshKxtAvCGXdWaxKsBDDW9m2/si154CXp+qSa3/4hjIFAEeacF1Flu3drGYDBCmO0Wf01KSl3+dNRtcDY09C/DrG5efbiVfnRh5+ikrQbmkvfhxhqa5jvucxKVbNIUFz+VlW1ABXito4Fjxrma4NzivH7CaeG4Lpn2KGP2++licy+KnGurHUk2AQ4KoHQ9kcD/kP6l4DYXb5V7bOfRC2P8crxUJDtsrMuPtDCqnbYGwNDWTKW8dSxzKZVx1IlsAKIC8CwY5uSTM+qMhWisIQ1qnS0JxBrNsHDgEWyER1A2ZN2rXTUDfKYASkL7ER+FxD/b9xaZd8K7qlP1UqpMaujd7TbNjbUF50rbcm/3hHnFqK5du+7cr0dT/PDNd7IbXdtb3zMN6N1AD3GT/NRr7aQdgymUcPb5uvbAID0oe65BRq46qd699f7fuxJOuXuAgzEFReC9LPXD+c44ZH0uZrHIbAISXo///x/3Cybu3VJ9v/0b6+VDx58NWD4+OV5+a//5+3U0ZM7U+U/+ztrvTi+LTdDYPFtqclvfzl+LrD4/R/8UQCFDFOTXFTmBsadj47MQWbXGBR1j2SB8dIh0wHTs4ry64dQ5j5gI+86f92Q7Ujv4NqYJ/kPmZA2e5evovHq3vkTQERPnY9klVQILAQVZIw0VHWKtMIPcZe2oCAf6S6PRBWmIYBAZhlmI8DCOJ19xZ9MWz7mXj07cGI8BE+a5kVwIsAJsOBjLbBQz1oGe5xZVY17x6J6AVghZnXWpauqUurUz2FMmmVn0YlmF4Syx34Wr5FcbOFvEybE8w3M7AGzx0oY4Ho4sadQJQomZ4ZVomaQVkyy/Ozk+VGZRMfeFZmcOVU9STsLyyczeQwDtIfEYtf19mEUZZKsL4FArQMKxiH9o6ZEuAXtKJB6eFXyMUW+Y7+hG6BFFR8Z0gooVEeTkezDQuOTIXoNo+s+A//vjz5CB36H5WfdD6LapMiImc9FbCSWYMBdrcmVrYzfWWYZpCNWlvr0Bfr5nK9Y2lOGLsDi6gyJxUz57nuPygdPHia8eSATpExeuDYmQ0bSGeqACmbxsxkeDKMqcjLzShKq5AVwQbgrgJwral0EiHCFQa/6+AITECCtSGY6/YCnAA7jgkYXXGX8LmGgBRWLgKZsfsc7jbWVXgVUwNQLLGTww/TTuMxLAxfmPW3fRmvD5LBt5o+rs9eueuXeFfsAikOAxb6n4AKpxSHSjGw4SDj7tO20SupqG3UJXKUMSmxMRwbY+NyxXFDxmhWnXmPbspfVxFBBo26VTEzSxmbYlTkbQ9LWAs4F0uQ9tg8AKKUZkd7hZqdJu3JcIQ6L4iphh9kN3k37MB4nnnUY3AXyY9+2PxuO/9RHaIGz7dVT901WY/qU5Wt//NHnMPvnoe0KkqF3Hz9idal7qMjdjv2Kth7WS20bFfRqvO5YFRBJ3db6pZ8BGi7pR7ysxs9crW+XtHXDRVfBUi1MFb/sBcP1EJq58Z1lD6inDIIMxxPbYN2DhPqCvkoqctKGG7BQEtyOVDNhPHSNhEIwwan06RRw5zXAIoMjaQBUxxhDtJ2gY0EwbEiME1q7ItQV8V1yvaHduSGeUgv72D02w/s3USN8j9Wg7gAslMnY1NrRbnPtvWiu+LKObI1edOaUlU4ZeKwHT51D8/ZWIj1v1ZNjVe8gvs7VVHrONcH+Y81R/U0+8uNzDdPCmrPG2L8VnVF1wXXXfz/Oem+5PBIjefTai4v7Ftybmqq++7GY50H/0uDv/dIv6+mv5LBv24c82nUwYd3MX1XBY+NHvjeea2t/dUz77/wPm/T9kfLm8Ko8uT1Z/pN/d3Uwi7377/+rnfJyp07wLM2Nlb//76333n1bbobA4ttSk9/+cnwtsPh19rH4PTbIkwEJsOAayQQfJT/IumVA6o2gDpvVrQ2YXutHtA8s2gpNlSEy8MCHh8cwZnwAAygY3PweDg58DtI+Z8YVhiJSApgVGaQwGBkslTJU6USkFR2wMM0sR08cbbBPDvhxLwpn4d1BO6pdMAHGp7+q+mR8FaxE7YT7Wj511CtzI31kgqJWAqMs8y2jrY56llDlXoZGUOHArs6+OXG501mYAIHFFHr24zDQFyw7u4/UYodduHfI/64qQDCnWwCAbZiXHWdSmd29gTGBwy1jMD/TSClmXYL2lBWizg6QWqCOQbmUNDhzLNAxPWdKjwAWRzCIMkau7iPDL60DhChH/QbyEYWelmmGsEuopKygf639hOpVAgkNPl3BaRJgZNjYBEgH7htDbF12EYYxklF1r4f/589+Uj5jhnkHgCPTNQOD4xKnC+p3Ay4EEzI8q0tLARjaW4yjOnbKLnOfv9ouH7H52bOX23zwMMZmNv768pT8TUViIbBwqVQNfiuggKGF+U9b6j7qquJIDz+ezlTLLDoDrQRE9sGZbTf6m0WaIDi5xoZDf7ERoB5cPShGvnxwjUNA1ZbJPYOxVHdeXk/g4KGBs7tqu5ztNFIB7QWcMTd/2thEHQ4GvKmQSTbp6NkAt/Gkn3nD0dqgbdtTqYiSCfezEFC4OpSb5u0BMNx1XNU3w9hfbBNZDpk0tfWwvgR43itxkZHInieo92wjZfJ8Q13JNNNcyoQAiJWYrDuXP3aZZ5cbtn24Qphp+E7gLGNdQafltQTQhR/LGGabZ+M8V3LEYdi6khJlx5NMv2OJZ2+M6Pqd71xlSVDx2bNX5QtsK6z3VUDFw/v3i8DC1akEdePkxfowTSlJhKFHjbuOV7YL61M3++gl/eqcU/Dhimm2gag/0VaOA4YA6YDzXdTE3PBOw34nNKIOBm1VBWsSi9QjI0CkFvRDQYVSC+0uTDeTKfY/8kZuzF7aovSSWF7Nm9IKJT3uB3NieuTrmjKNuLysxtr0kywzCx0hGCDCd+y8TdmVVqAQF2BRVaYm2euG/SpQZ/xbSCweL82VDfqhcC3pJjdmoB4RFvjCyuMwj+bS/5pbf/nL+xaD7wgivb3prl0UvtH1S0c/TgPZbuxPHrbht47u2dftTfp6y2Mvft92PgzT4tGpn5lE3fmqflr2cGy3ehr00+7zvnnCsbkn0u7H12+VgbSHwGKQQqUILKYnWXZ6faL87z88Kv/gl2+XxVnH8f6xf3xV/tl//6r8nX9rsfzxD495P/pTwOLNwVV5/trJn8JmsWySuVLH4xbLwQn78LAh7d3VCUD6Tfn4JfZxAJond51Uqr5e7KBmuX9Z7q5NlDVUswaPrT1W+sP/yjwbpZKfz7CFWMXPPfx62MQ+eoHKLczHww2kmNN1omAwjp+XxyGwGKTW8P5fZwp8PbBAFUqJRZVMyFxWYOGsvMBCZrqNsBmfGT7D8Dj4Z7Cuw6n3bd+IntSAD2N2l3XQ7b4U7YOR9IijMg/EyP3AGB16+s7NmqbIR0+6QL4EO34c/HQFwMCp+IGPlEQgwB8Xfj27WA3CcxhwGLg2iyuT1T5kFkegEuNt804kTWph+Wp+BBU1P8YRJllmXqaNqwxbVC/4sJu+xHcm1PAy5LOoZszCxGrIK7C4Gp8uhyw7y3iGSlQfWOzA6LyG4duBkcrmZhi8wpmwgd5ZmUIfXKnFHMBi+uyQDfSO0JMeCViagtFzFvySwU1m5gRAcQpTJAOsWoaHQEqApTG+s6bXQWECpxEkCDMx7ryFHrbSD1f5USdfO4GoPsGwNQY4zFPIa0nla2p7aQxk9tGASf3hTz4sH372rLzcfB2mOIAMWqn6pPpINs7juoKahuDCjdZmoA8a7uUFkooPsdH47MUmdHAnY1ShrlSFmi6/oMTi6SNsPzD6hbbA0D6Tk8qnJdguuae55F51F6U22ihoS6BKjnr9ShJWWbFJKYMAyrLJyMpwRjUKqVFbKSrx2SYBkqeAP0GK7U/phDQynCtAqfLms3mQSZPRnQFoNH8ypK39OcufMphRjtYb6pPtuJbDq8BCmxGlN4fZ0+I1kgaMiZVYBFygxgSwsD9o1F6lWBrbu4CANhDUE+6qf7lksTPyWZp42z0/dsoOdVZVoOpeHK5elT0xaDfXMNvK92aQXihxUqqVTRKzPHJte7b/KtWhb6U0tRfWVmK/1ZVfyup4U6WHFVTp3s5ab3VskPmXmX+NPcVnz9lNHXUqJWC2lwfsIfLe48fl3t27AOCl9EWjSd1zI5jWaFu6DdJYIKF0Ksb61i/tQomWkplIWKBfdjgHnO+jfrXXXd1sUYDTDLGtO1cmU5IjgJu0D6Y9AqIAAtUYHmCRtOqqUCELZEhpLSj/AY2WnufWXi/I96DEgkekEjA8SJ1uBBau+oSqEw222ygPOtoGSf8SAJIlD6xrgaRLzNLHHq8tlb+5waaG9L9l3GsdJUctWzVjPn3pZc1bdazgwRbJMxnW1fJ4RAJBOepzz/Wt+FrUeZsPQ/VfR/YaT4uh9QeI8+UsJQ3bUj3Mi/no0uxoa7je0fOLSy+PNWs9X4P+ewG5acnErT0Qqv7/f+y9CZRm6Vnf91bv1Ut1dXX3TE8vs2pGGxJCSIDAQiBiiIGAQYBYRIyd45PjJSQn5+TEcRbHsY8NBoHICYEkHBxbYiSBRRCrQMBhkSyBLCEBkQ6LpBlNz0x3dVV3VVd19VZLfr//c9+vvmqNWmgUfDyjut237v3ufdfnXe7zf5/lNcHN0E91S97ftS2x2KQRdx1YfNMrDrcf+fkL7au/4FB79ecf2hLmnX+w1H77j5fbP/y2O9uP/uLcFmBxA5Dwr37zYvsoTL3ziU3nt/vF90627/iKI6Mmk2n//T9Zad/6V6bbw799icWNaqBTAIP/7KuPtl/4/cX2Bx9FO4DDlv1rL5tqX/55SM+H48d+aY7FlJ0Ak13t1ylPb94vfu7+9uoXH2r/mjI8PgCbvbi8+xtIXu4HtHh8JmXctrEYCP65eLm52B77sz9pjzxxqV2BXdt98Hi7+/772xnsEAunwpOy0Hn54nmclzzZltb2tzseeEm770gRa4NF1+Un/6R94CNPslfagXb0oZe2l93NgulfwnFbYPFPkFj8y4cf5rtUTL4DUzsCVyBrV+tC3Q4iPyrjR5+fM73yyo9/VoUZ1F09Kav+xPOjmTSSAJ8J8+PDJwOXvE1sSLDnsovfSgd0g9hXRS2XcTyUCHT3smus9MvQqULgYW5+gixyvlmkRZa4i1ViUTtId6bOFbeAC+NYzqG8AgyBRYELvt+Ei1EzH21BRZiJMBSl1iKwcAVVhk3gZJ1dsXTl01LJbKgK5Qq5wGI3jOfG7n1tBWChKtQsRtwXURVRYnER1YcFGJHLnKpDXYeJWuXcwL5iNx1rH5vkHcxO3EgtkFjshXPVzaxGwzs4BRbXWSm9CuMs83gTkCFoclVYpk+vTPvCDKIy4SooHIv0cUXzBMbZd3Lq7UdgEa86lF3G2NX3qEDRR2y3ormrzF2iYzhXm/DjT55uqPbRRx/FyPYTrDKfi5657beHlW1dzMr0Ci7iuparRuGuQLvavwtVlkswdB9/fBZgcQHmzpVjjNY3bmJAPtVe8OA9GHCfIS42K5Sr2lAVFHsA7U7beW/Z7G/pnwEW7IgMA27Z5lilX8HV78QOVqhmjsZF6bFjx6i34KLiRH0KptOdpGXmNco1TRloy1OufNdiwK3dgv2tbCk0Koe+hqcs0m0Sg+5JjLgFova/3ge7+pC0MW0PmSmSypH+zHO7t/d9nw4lOAt6h5rHyB0PUQvYXWgnoaRKtT7Tl76qm7nruWpflkN6OFaMr+ewS9Dj3OwcaeBGFXscGWfHbspMXw2woN+s0hd1cbybIThF2+luWLe/Si7CkNP/0z/oK44B24T/NkLGQNXGuUZpZJ2O7U4Logy1pq7k3+cTwZvG0k+eZ/8IQIUqVUpGztx1Z7xA3X/PPTgAwAOX9SM9x13U4JgTIn3wN20QQ3XeS0Xtbm4wnlQD09vTCu2rREH1rNSFcl9GOtEN2Rfoi7qbVXogIBG4WTXbTimhkstJ7ZK4xs6EjhCJBeNfaZtAJGONem22p300jRp610JMMc81hwIsiHeDMWp7MkpjL7HOvIGvXyYlQAZjUSIrqdBDlKpRqj8FWNiVHLc8FwSeQVrxwLHp9ny8Qh3DHmSSNt3sZUPrVPfbbK/eCX3C/00m3mb1WR7mOXd1+GwIn2t/ns5QP0bJGhJajXpIRCU+G6WWCFVOaLP1sbnUOEmCQ6pcKjpvDTCeVi/D8KwnZ8x+Px5+9MxSbCa/+cPnpDUq71hCo2eGMV/+f9drvs1f/14O+19vr34dz9hnltGx4fl0VaHe/8R6e+jYjnZIYflneHRg8Xe/7lj7337hAvs5rbf/9lvuDLlMynno+376fHvOyb3ttV9+pP3A22a3AIvFK2vtV95/uf2VFxxsp5B68Jlrv/zvLrd3fXi5fQ/M/fPOMFY4BBa//UdsMAuP8w1fcri9gOeCFZ8dhGs7NLmzffOX4oacqyDhPHYf/x3Sk4OTxW8ILM4CHKYBF6/5ssPtjund7c2/w8IXgMb4DwAivualU5GK/OSvX2xHUNf6L7/xePL+TMq4DSxCss/BP9jX/tFvtd/4vY+2pQ0XqVpbXkQb5fTnt1e84gvbc+/Enf3i4+1jf/qR9pGzF9r8kx9rc8sz7UWv+Tvtax+QXIxf3n/4V3++vXvpUDu881pbbC9qr/0br2jHJ260xz/04bbxwOdhV8f3YJjHPhsi3xZY/NMf+IH2r96sKpSr14OqDx9e1WJKFYoPGbk7+YxPkk5I8AVhVMNADO+z4s9E4CphMf18EJ20nLxIKGkxu5pf8jQdEuoSjPFJXCZYw8I9XC1PGBA+4sZz1vFDFMafyTMqAlyVmnj4Lnf8WScs/1PWPXyEVavyw590yJt5iPcqBCQijGHFFVioimAeGkwG5BBXBkIbAVcnVelRv1qQoRqUq/YeARXWmbgylSau+ozAQsZSYLGXcwcMwlV24L4IsDiHy1ntK+YASQKLK9RHg2BXw1dggpZhatdgAndw3YPB6YEVvA9dvdz2YWexF4897q+xH2ZBJkIGxLjX8aykC1TVf2jA0FpJxQHrQFhVwkqqUuBAFSilFXccOxrDY/XwNTK2vYtRtI9UG9j+HVhYz7Qn9Z+I6g/63TBEeig6+8QT7c8feRSAcTYbikkbV473qa8vHWFQi561m/cRVp1n2BdBz03aLTwxf7k9dg63tbOgeBjhXdT1vjN3tBc+eG/2K1AqUO1nn/AjQMFod4FE9dt6pNqPIEHPR3oDWsCNqrsnzy+yAg2oO4Tk4ww6+qcxAj7m/gcwqYIHGdR1jbqh4VUM6G8I1AQL0MQVbYEFlW5HjsygAnUobW/buufFBOVfFdTRj2S4lVbsQwKgFCCr+gO4Lsa6GHH7j30xNK9emXrUGCoGwD7hSrgG14sYcF84/2R7/OwnWNGfj5GxBIjaG2PF/ulK/iGlJdzvoE4eMseXUVe7xDnHeV5a4G1pCSN/93qgmVJmN8ETWOiQwL0pNqAhHDLgtJhVd3zX/e/kpGpQAmv7hwxvjXHrkQZJrvVHUOH4duxnv5bMOQJXaWDbFShXzUxXw/OAnVncFs+yX4WqBtLvBK5lH2BjQiUWd+AFaj9SIg3klchFQmG7OXY5008ZD6O5g76hy9lreoa6spQ9MVRxWrYtKVOpSe6INErJ2yKgYgGQrlG8aUsc+791c8FAOwslL7qhdp5y0SR9j/ZPu1kOylMexiyPz+tZzRH0E9LkMWOX+YfwaWPG0HVARepgX5CuzBnrgG6QNwVgvnE+9KQ93MeCTgawoN/yTF9gG4xJpZhHMZC//+h0e0iPUIDBQy4smGTahxsOWiRX/wwzqEMpbZJnlFEGv9q02ikdxZemU8lUXCNyl0e59zfHEGbsJnnUy/53PEJ/xpX8c3jtQZJtJWq5RuXu741A+PGflUg97/eJ29P34VDOqkHFzt8hoVE1hqDj38ee5viz0IxI3/ksBBaPsmfrCXDu03FkNA4s3vsnV9rPvWex/c3/aKY993QBgj985Gp7+Lcutb/ztWykiTeoW4FFp/X4VUb+n/8MqlMw+l/x4pI6dDWjcbDhAtz//PC5dghg8F98A3PIUIEPIrl4y+9e2gJMBBaqQ30v4aZRh/JQ9en//NX5gJTvfvXMaJy87d8utPf96Ur7X153FyrQ4z1ls5S3K+O2V6hNOn3u3M223/mJn2jvbS9tr/6rf6V9wamJ9ti7f6794h+ttnu+6Kvaf/xFp9v6hT9rH/nwR9rHrzNPLzzaHvnT1h78jr/Xvj7A4lq7dPaP26//3/+u3fl3v6d90dqH28/88O+2+/7h97aXL763ve2Xn2gv/uavb8+bYQH8qbvkZ0Tq2wKLf/b6H2xvfPPD8EWuHso0FqjwA+zpR6QmUybmPukyeWfl2/CJQykJ5HtX2QpUDB9SP6Z8iPOcMH44PcKEkk6Mxck7ecEQVH2Hv8N71aFi98H7vrJZHzZAgfnxEdaoMQygZeQ0m2SVe9Lz/yg90xljesyfd6MPJvemMAIWlp+CC3JkxA8AIGqVXc866pljtClDAXAwe8sjU2DZ/G3eltvVUleq1eXf56o1K+0CixswIcuIZQMsMFg+H3AB41h4hLJtYMh8vV0AUMgICi7cMG//0sU2CbDYv4oPf5j0aRhAXc3uRk1CO5GrxLmMrry7Rmt0KkQso3NVstigLqobqkK4ei28arW6DaN4GDsHmVCZ0ujlwzwV7VHZgXEMyEiMUDl30rdOmcPyinMNZtyNzB49e7Z97NHH2CjtPIzitfQDmdDYhcCQJq/BRkQGWJsOV9n3AH6W2JV8lv0PHsNDlHWxDs+7/wzSitPtNJuhuXO2koHiBGSM7D+2YTGAaQN+BVTITMKMa/gcYIFR+RzA4ur1dUDBdDsFk3qKNGfIP4bktJUqSzakkielFq50C9Rk7JdIx9V9GVpBxT5Uquw3qrvtBZBpuL8Bc+jYsH8IPvcCplRJSp+HTtJMoBVbH8KFhtaA5zkgsfH91xlS+5fGxa7kL2JsPTcLsPjEozDeswEW2sDQ/UijJGXTtKd7d9iWOwDVHgKLS0ptAFizqE4pvVkAVKxgR+DKunlaxt2Udw/9QKcEDO54w1oHeLjytw+1IUHhgQPaGCiBKqld6jPUyTQEEB1I5DrUzd6ThQV+xw6F3wJC6yddr6DG5waJF3FVq9raNcaGgPMOpEruVXHm5MmAQGlfoIK2Ii2Z/w4sIVvGYuYHjPPLKxRSAEBCtaX9Qbe6SCe4XqVd+0KFUobu0UnVKyUajm8pWPVhwQN6Cirsy/Zp50RpF3DDOPTeLpk29J4z73lnnzQPN1R0zDoKbWfn0ORNfi4QCE4ErwETjG89yk0wDjd45jlB/u6sDbprawEdqkcB8ukHUcFkHJ1k0eB+DLcfmDmUTfHiZpb8bIrR3DcCFpaT/0MXNAw/R38M7/88tGy5qWcJV6F5zK+xyJE2+NsYdcl9/bn1QaU0CpCf/Nly5S3pJWbyqTjSsMJtpplnJjYWLuXrGYw/T7j6k3KmjkOSJmE2wzNDfSbHd37Ls09igd110yxiE5b+xSkyDiy0ffinbz3XHmT1vxtx//gvzzHuN9p/9ddr9f9TAQvjnkOneAk7ihXuf/bdC+0rXnQwdhmWpgOLW5n27wOAHMcmQ3Wofjxy/kb78V+Zi9rUFz4Hr2scAgvz6OXwmTYT3/+2rQDG57/5oaX2a6hL/QMkLx2E+PzpltG428eznAIbT7Tf+omfbB+afGV79au/pL3wBOrxH/il9vb3LbejL/mK9rWvuLvtq+mNCWiu/el73tne+RsX2z0jYHG1XXzsQ+1XfvLD7Tn/zevaS67/aXv763+xzfy972lH3vnm9oHnfFv77pffBQ/7dEbpJ9P+tsDi+37o9e2Nb3lz5towjnwUZXBqdXqYlHtlmEi9leHxo1pGzKpNVUFlerr4PqtzMAcyCAEWfIxluAiSNDqg8OqHOMyH+Tq555BJrY93X90MiEn5akXQsJbIfEuqUCuURk+Rh/KaZ3156pLyw+jEdztX61p5VDmoXD56xlNiIagwDxnaAhZ4NIKB76fMmuoQMkYBVazmyhQUMwCtBGswWQEWggsYgOjZw4TugElYIx7SX2wq1tjHYq2dg3m6iNRCk+/dMA17Kc8iTNA5mOpzMIHX0KvfySrrAYDFIYy3D6MadJQVyKOoEU3DPO6EyZbpXUZacBHGUdeYSi0EFntZpdwPg6i04kCABSu81L8Dy9hVsBqsCpQr7Bqcx31mmMSyA8hmdIlje0lT26z6RNpPmvJPYCUdtIs4d362PfrY2ahEzev2FqbR/qbKiaojewBGSoJKV13Dbj1G6YUK1SLCraBK8iRuZzUy1j7kwXvPoAZzR/bB2IEaU5hSGa982ixUAYswADy1R9SO2iuxr9DoWQNnV8EDLPhgHABYnEZP35Xwg4CdKRhYjboPYIgtLWQk1sKMlr2FBvHaasjgd6NsvT/Z3VR3kxnfoP+7+ixdNCjuBv72GZ/1c6TiN/RV6ecYC1Nq6e3LGT8FpEeMJypaSwu440UN6smzn8BGYhZRaWLUxgAAQABJREFUPK6IKW/imDdpahyfzesANtnTgVI6Mei5axZAcZ4zG+EBKq5i7C8T3vN2H4icgCfRimDJ99LUcbOHPqXRsrYWqrR1BlsmW6bWFfzaRR4mnPD2t7SP9KR8jhP+k9xmn5GZd1frZcCEe5gotVBdRnXDu+68g3Y6EYPtY3iv0VBewB4pCcmYpiCw18H6pl/Ctrth300Atwb5qn0ILgRYVwTtgAp3GVdqYf9UWlK7sevmFbVCQJyud53b+rxl3ayjRusayUdiQx0zJwnOKIt1c3xVWw/r3zwTzJuHGx6WIbizmf8AfQOw0Mjde2MJIAIs6GPZUZvfSmM15oYwbQJp2AbnGu8FGjsydnexgIDRNu1yL9637gVcnGSuOMgY2k1c5wT7Wv7zm4LarHWk4I5tm6uu1SeqDrZhhSVCGnCIZtCx3xWuEr09sEhOlQh/k/x4OibBby+O7hz5QYb5b3nq/Wb+FqaO4U1PeHg4pOOv1JHf/VGiSnfejZfDsIYxr+F5p4+vPuVhHNJ6NgILzRWe7iroOLCQdj/9roX2wY+uxIh7hQWfN7z9QvtGVJdewaZxHk8FLN6BKtS7P4Ikn8UODb9tyicAGX8RYPH9GIUfm9oKLB7FMPvHADTfgj3Gy24DLBZAVDEqH5OMWMbf/MPl9muoXqnSpbG3x2dTxiSw/edZToG19vjv/Uz7pQ9caUfOPNDuBkefRToxu/u+9sWv/LL20nvG7Y6eCljAa8090t7/C+9oZ0+/tN236/H2/vettpd81Y72oXcfb1/3n7+qndkL7/z/ExVvDyx++PXtTQCLWlWUwS7mOh9BClCT6lASBqtzrIcfo3xUwzTUwHGSLWABmOCjWqCCqx96GHNPw/jP9M2zPtDDvcwqz/skXSuC/OajnHs+eonHtYAJJCK8acqcyGzJzPYPg/n4nGzzyN/5GKQelad2HNbZugiowmSzAqnbRj+fquKE8eFXVKGUWMAAy8C7B4Mr61nRh3nx6IAqe1CQnSuGqtMILGTIVbGQwXT1V9/7O7jX+NI10CWMt+fYKG8WgHERES07WFAemCnKKLB4DGDxCLv+arC7ExWOQ0grZtautjt3brST2CUcQWKxn7z0zy8zvwygWCCOeuHqeFsjjdeVVExp38BVWxFpqmqYTH3f+E4jdI/QG/qkxQmXsEN9BAYx1LZNqWuM/XkmkOLncLD6DOPmjtBPPHmu/clHPxZbizlUW4yT1V7AheBFw/NIMCiHbku756iDgIzdMG56x5EZE8SdOH4sEg116mPXMoCLAAvKScFGDL19FjKGqXSDvWVBBQz1RWjpTtAXL6NnD7CYmjqKGtRJVsNn2k72BpmkPIdgWo/foftSABuMmkDhBkyp6VyFthp02+dUe4rkhPYV5AgebXMHn/0zIMpVf+qpSpz1tVyWU5p2YMGv0M2/PjeuR8YN9/bFrKZDB/uYAPIK6ljaWcyee7zNz12gvdnzhDbI+IMRXof+IzfB0NV+6GHZLqA6dX5uHtA2F6ClEbcG6aoU9jwFKapBWfZevhSr/lBQxgb9XKZaVTylFlETtH/RtoJH3UXbBhlr6U+MFesiA00ZvboxXBh4GP7lFT1VlYtW30svvT1p/3M3UooTgItjR7GFQf1J6Y+SIAoS2iiFUMVJV7LWIV64oiIFTVY1egcQogqX/T4AZuapHYUSi4BwxoqSCZ8LDCLpgNYFNpRmsN8FdSgVqALESvbsr9bVdrNPRGIRGlW9u2RG2ktfAZAAKm5oaUcNtROP9zVfluRGMUbmMOa9qDg5X9CGTBFRdVplIWEDGkwgXdOge9U+SFtog+KYPnZosp0CWD6EfcUJ7Cym6dcFKqSY/cwSUSau1b7DA5752/qM90VC2iETNxHzp+aAzd+bd0MPzpwwAhajvHjL/UCmzUj9bvxFJVR5OzaGMF6tg6UeD57X/QGB8i0zkMcocvVzE0gdRy8q2OjvEN4wPZM+PhLG+KPAW28M51F1B1h862u3BvhL/OX3KGU2f8t+y5E6Uz7nVc+na2OhpO3pMiy3AosuLdCIe3FlHYPqlfbfv/YE36oq/63A4kMfuxpbh6/F2PqVL9SFdkkG/tFPPfkfDLD4bMt4S7Nt/3xWUmCtXT77vvbOX3ovfN5N5nC2L7vBYtAXfEn7ii99QTu1xYDpqYAFU9PN5Tb/sfe0X/v1j7T5Cexln/tA2/GRD7R9X/ZN7d5daLs0FplO39fuwqvmIA942pS8PbBAYvGmtwIsGI0yPwEYTpLDJJTJmKz7lOTz/OM6snsgnocTqB/L2FYALDqTHWCRd8MkbnpJp64yG3UWgOjvOtjZwSqE93nuHE2hRqCjl5MPgquEMl7920Bu+e1HuYODulaQ6HeTrnYXGqqXehS1g0kiw6wGuslUuVOEOaCe+2CQ3MhLYHEERkeVoWzCBfOl5CMTtBM1p79TbuJZ9gJH1oW8CC+oiFtMdaVhBvBgx/4V7P/AucC9jvN6nedggB6BkX2MjeZuoMpzgJXXY6hAHVm/3qZhgqdg6JRs7MRV6k2YXT0ZrcKs3JD5RM1C40/1zFXnOgDToeGtXmyUEsjUuuKrLYXqMjKIAgaZHNs6DBGfjdSBusjIyzQLIKxLSO6fsaOHtV0NoOehixcvog71KPYWZcitC0+PnfTwrPoCGFQpkxkSYOwbVsGVXqhytlsGFQYzOx0jEXAvhj2AM8uzE2ZrJ+CCAkE0ymq78c+PXdFdYKHxN959UB0SVMxzzi1ouAzwwgPD9JFjBSywMYF4KbfSlKPHjrcjGAYfRHohrdwXQzsNVapkYlehsaBiAlqoguQK+W7aVD17mdM8o9zdVa/lHQELymcXlp6RFFLu8SPMSx4M/ZvxJYAO40m+SktWUIuzThfOP9EuCixgqk00q928VxXLXcD3p21LQuVKvKpGs/NzARbn8QS1SN9SQqAbXvtxGCL7sf13qJt0HRUxTe4fwlBXy++O0wKykj7RJ2Vw1eXn3Wb/rzo6RqyL/VMQJGMtiLjOKbPNz/QLvXQdw+bm+NGZduexo7GncAM8JTDSPcCdseRYs9wxbL+KLYxtwzjo0ldZYTe9ixcswKXukJVO6FZX6Z4SEhl9+00OaKh3I1WnBMeq8ClhENg597m/S1z50g8dSzHIT/8b5h3bitN5KeOYRK155kjSiLSCdAUx1n9zbnJzO+fSmk8D8oi5IYCgX3mu09dDP56vkecEYxfDrbYOwF31d/pbqaedYoftezHWfoh9K45q12RbDGXp5bK+m8DCXxx2TC8pdW75M9AGunhUiNxmvOSpfWaI6+/xPpw4eUeqWyIbsKdeaSfVsdt6e0tpkoZ/KqB5VbIVLn149JZQPU+C53aoh3lZZtvelHq2PbgPRve+H4tn3F6ZTxlmLPyzEViEBk/zz63AwmRe/7Oz8dp0lW/h598/2V7zpdOj1G8FFr/4+2Worceo7qa2qyj9hyKx+GzLOKr89s2zlwKr59u73/TG9vsbz2+vfPUr20tOtvbIu3++veOP17GxeHX7amwsNreOfGpgsUkcvj1X59uf/cbb2i+vf0n7qpvvar/6cbYt4Fs79eK/3l7zlc9px/fDs25G+Izvbgssvv+HfjASi1IHkjkoJnj0YWBCrA+OE6sfgzoDBPhAGT7MI8XyQ+eHvZgEvbGgNuBvngsu/MCOT8hb0pIphjlxVTNsob/5QMq8huEnr3wqnKBdHuEwvuEzqXP1lR8AL/7NTz/OlsuTJ1kR5F5FDhmdvpN2bDj8IIcv5Q/3qhhEh9lMuC9mAp1ymN4jMGmHYTSnON3BWeZGWlRLGb4AhHT1WVfLsFzmGwNnGOldqEJpbL0DZuE6hF4CTWjIvUgBUSsNuLkO03EepkZgcYEVZfeyOIyu+DF2oD4As7yHzfJ2uEM3zz13eAIKsHEuZou62gbaoQgB9QQT+4ow8DDxMEWu/E8CNgIqKLAgRObTOsn0K40ooORvysvzeIni6qfcdjVOqX7whHrLYCsJkuFTl/wKtiGP4yLtzz/+CODisXYBZlbmyiMr3oTNRmvQQ9UST70NHWCVtfa7wMhbQMSpXUv2DIF+Mum7dlqmYu7NO32Lckn+NB/0ULqwzEr1pQALNoLDy5CG25evwMRCmaNH72inULFxd/E1gFukEfSfKQyf7zpRBsIHWCE3bfu23rZUqZFZpfo5lK5FggITaH+137narypR37ci0iuBCDFIKunZx0NjHwyHNE0fTuLFZLoKXjsv81tmnLzdOdqdt+dmzwEsZvEKxcoE462YYA2tmUwAwLXTdrm/dTVfZwCz2JjMzmP8rdE2qkBu/iZY6cAi41UiVkFTZou3WcpeWp5JaM7sbk/frhV9pXTOEQNAtb5D7IxJ6lbSzWLCi5l2noBPhmaHcY9657Fj2L2ciF1F9xg2iRqhdHROycH4TP5EtN/eVNWJ9nZn7ag8qW4kE0+7aZei6pNASheySwBc7ZF8JxOfzTgZI5kLaYHQmfiRWJCGYayT5Sv7Evsj7Uud66j5xjaoxRUlEdQpcyDzoWqOAij6/g3azzBKem3stDeJqOKkRKL6gKBCaYUb3aFex/hz/shCimEFCoxfd99eZ1xosxVgwTjSJfC9SCoe0raCfSsOC85tA8pjP/ZIe3A7UDLP6sXY+zwYC2EDcVSIuvb+auP1tA1Tz6svVxxi5X9dfTaeefpcHt7y55Z0Ey0F8E+Vx7h5xBPrVcUc8ubFqAbDTdziDtlYZkP6ahQu6Qz1HB7mUgkPMQ1Uufa8+4stdRnibAOLTp26PhWw0FvTr+DZyeN7/5Pjo70i/H0rsPj9P1uJPcUXPbS/fTkSCw2s3/57LBxgwP2qv4CNxb8PVajPtozWe/t4llPg0r9tb/xxpAuv+pr2qpc/2O7Qx8qF97e3/+wH2/zJz29/9ete1s7wKa7j9sBiY5WF3I/9Tvs371huX/6dp9qv/uM/bF/yj/9We8nN32o/9r9faF/597++vQDPmqWX0tP8zK5/IWAh8+fpx9SVVmdnJ0U/iE6zTtJOvD3cFmNq4jkdG7avxmXDOhjZGG3zQe3qUFsmWlLuaQoiopZkHkM+liWroHzkXfXNB5A4ZZhZs3ziU1avKfSQJpfhg1YrhPkI8/GWwc5qIFej7KKuJbGw/iaUBExwABVypsVYycjozlUVoiOslh5WB5+V81K1UPogY1t0dAXXe0sZsAUDUeAChp93AUyktwsVDZnOHay8s/1dW16bwM3sBv6JcdcKA3ODci7A8DyOOtNjMEFKIfZTj5mJdbxBYQR9ZbHdWLqE96fLbSd7W+wBYByAVvuh2T5XxskDIgaL3YCB2UF6qkFMUla9+rjiquFrbX4H00lbyXSWGgl5UDa9I2XTM9Jx5VoGOJ5/YHS8DyGHelpX+0tsHgwr0yMdTJfV40vYAzzy2Nn2UbxEPXL2iWw4pjqPpLdPWV5VnUqNQ/Uoy1j6+32/C1WjLK8G35ZP25WSWig1KslRgA35q9M/gQ7COnVS/WVxcQFbAlShFpFUcM5fxv4EFbRduydRrzkZN7uHkI5onO0u1roiVUpyz5kz7Z6772nHjt9J2fRC5bYi6uq7io2XKJhEwYbdp6QR0qXaWSPuGD9Tt6gVQUf7UgAmCVHt9Ik+FkgiRx9/XjO+oOG4QXABC1R0YJ6XcTN7CVChrYX7WcQewDLR5hP0F/cG0ejZVXUlEtqqXIYec0ptsK+Yw9Ykm77xTkY/rnnJNwxxL5CV5lk/rWtv+zQgtbe/O47Sx6lYtT+/Hb22xZCWID+B+Z05YfgpDWxPpVHH8WB0EpWnM4C9E6ijTR8+EtsVx1vZiSiFQmWJtrUPWVcTNX/vBRYr2CKphqfnqwWAU4yvebdOW+nN6zJ2Mp4BFcQWCCvRU8pif4y6FvTok6jz2wQLALXbNsACkCu4EJA7X/VKSTclWdJSKYyg3mukEwILyuAc6XgxTa9RueptrUyB/PXoxGAbXMwyPnUzq2co3hvH5lDCujOqUNhXILG4kfAl3ZthrDzAvhXP47x3Crsw60UjBFiQCnebbTj8tA1zjG42g3zS/E0BepuOBR89M53efytR/1aM/O2R86xSELiN+hXve7qqEvXgKQd9xXc19w+hJMgoFLf9cfSQ+O0xPKsfW/8WrNgaJOO0p5P2qTi9LP6q1z0QDz75ZUXi77PRxmJUuadx84afu4CUc2LLhnfLbGan7cJJNrPTDe34oTRjCleuf/trytja7vI2DLU/8OcraYf9bLb3114+1d71/y63F94zmX0xjO9eGBpV/6PvPMF3nPE1HD/4ttlsqPc9eKLqx2MXbrQfxVj72195pL3kAaSBHP8HxtwaX+sVqh9ulPfPcIXrnhev+rzNPS86MNJdrbuEf7Zl7PltX5/FFFh4f3v4//q91l7+qvaqL35eO8lUf+Ox97S3/+JH2sq9L2tf/TVf0E76iclxG2CB1sr1hbPtPW//2Tb38r/dvvnYu9s/ecNy+67/6Rvb/Ts/2P7l//iH7QX/9WvaF9zJBr49uadx7d9EFyzHD74tGxvfF4nFw2EEipG/FViEBWCeHGP4+eC5wuzHvTOOTq1+6AQQqn8otSiVjfpo+oF2NbKAyjARM/lupqvEYjiTPh9AV7xhkCsvWBPeO0DzEfaGPDsTUx+XTcaFoPnoFtipshkvQMeyDJ8C89RAWmBjGhGVG5dnYZZyrXuZV1fQD8KcHQFQHAZclKcd9NVg0qPeRBqCCv5UeuQVugAsbAgZHstsWJlnN2bbA+O+E4bgGhvCCSwEFVfWJzDoxrMTZb4I03QeRvCCrjBhTnbCKKrytHYNoMGOyzcX5nV4zP4W+DxeZ9dnaHYA5lwQFAZWQMApxfxIui+IBtz7qYfqT26cpsqPzLhqIjLMuldVP17G3TPqJqqdCFY6I889Caeedqx87L3hGKeB9fVQpUT1oQuskJ99AskFXqLOnb8Qz0S6NzWU7WA/jPtOyt+lFqqaHIThn4JRKmDRN+4rr1WqlNkfC9jRZ6BBdse2LVnBvgatFmUwOd0BfAFAsbgsY4k6FsqM+9kN/dRdJ7HbYONC8l9GnegSqlt6C5IxP3P6dHvgvvvaPffez34VU6GBQMLVbxlYzwAL8ivg0YEFNJS+sbkQkBXYso4jYDH0lerD0q7oJT3D2OfKOHT82Ic5A5Spl+OsbAYw0meTPIHFJewmFlH1MZyHgF2JhWDMtnM3afe5cDO9eeihvYtnrciXXUHyNj/q7r19J4f3nB6WMiXlT1cX9JXl9+z140eF5eq9hy6djZ2xwHiR5tonHBSwU9aZmekY5uv29w5sKQ4jNdrPOz2e2Rf1AmYKSsLKpghpC4b0bn4oTVTzEjwplXGfjqg8DQBWlkLIr0ODJcbUEn0gKmsUTVsj3TarumUdVMvKfNbHLnFVm4wtkCpQ2CppT2J/6+02VDHA4iai5wALyhQ1L8Ceafa9KWyjSPls04HWTm2OhmyEpxSONlun3huc6/wWsErnLgUNiKZt1+lj6+xtcRNaUqA4Cjh+cLI9iLTiBexdcS/uZjXaztadEq8f1ST9V66j19yk2dKCPrX9ufAw0fjRo6df9MobJCnVn+pD1Y+MueVdWrI/IfEh/cTsj/mRvuVDbgwyOoY8B5gxelwFreTGHt76oCpoAPLqfbvqWLHSn71N0TZzTi0SkLYYrkOgok8ve+INaXHZBhZFi/5XvkBaMvVvOXQFW9/3LY/DRzxVeJn+FbygTB9iLBLFdGsOqvg2kZ7X3G17/PhU+d/EIn33mEW64TyUvo4fur/WQ974U0Na/lvzerplHM9v+/7ZSoH59r63Ptzes3CyvfALP789dAdezj70O+29n9jbHnzFV7ZXv/TUmIThNsBijQ1vH3tfe+v7ptvrvvVFbKD84fbw//CbbeZ7vqm9+OZ721vfua993d98Nd4B+Z5+FqT8NMDiB9qb3lzG2yNgwQh3YDjJ+rHzyEBmAi/GrxiBMK2EdfAathtuuzoXbyqobTjhBgiQYD6i/k6KpM9IdDB2JsR0IrXgw5iVaz+OfLBLnYYSJB8nh/oIm4xxLFOkLEm3BjiPq/xMBhqGjhiy1Mky+HFi4iGOKlhOFqYPP89T/jh5ONPxzA+3K/W6HdXYWWAxzYd8CkZHpjwqLryTacxHyDie/JOOqghlVZXVVRloV/CzIg9TH+9Bk7idBVisbAAsNnA9q7SCuHqKWoEhmWNVXFAxj/67+vLS8TppqoazCqDYuHyp7eG6F+Z5EtWogyxoCiy0BXH1vO8Gbp2ccndzPYDKxAFBBWf2NqC8dhT1zl2F38Buw5VYdeRl0HcIhGBw4i5VKYsnjI50iXSG+KGVFSYP73PkVnrwyD4CYNFl7BxM+6NnH8f97JPtSTxGXcYbkeohtlP6QYABee4VICG1UCUKJk5pggBDta3sgTGUX5WyAj+CCsFPrdjCtUXP/jKMtm5v59lczY3OBBVXMBC+hrRiH25/XQ0/cccd1JG9GmBWZbo18NYtravbujV9zgP3t+c97/ltBtWcSQxlIRKAwp23MeRGR19Gz3aPS9eAHGhDW+sBrFbZARP87jZD1tPTOB792u9r/NUY9H4TWGyCdoGqe1msYNB/BZCpjcU8dhOXLs2n79mv99B2riqosmcfVQqjm9wFaFISC4AFnro08Jcpz8G42Ry7NGqYXt9QjrRxWjl/LL2P8tiX1onfqc9wn2fW09+e9kXKIkOu4bN9TYnUUSQrR7Gn0NuTdiF65fIUAMedrBIpkjEby+e4iioYEhulUe6mrrMCpS8CCyU3epTS1kjVor7jfIAF0pkrqH7ZDwS98hvON2406Zwiza8AUFRZyoIAdfK9gD3G2oIKyqXTgUjlCG//NZ7UiLRCSQX9xzko6k/keY154JrgAlorgaIiRT3jE0/LphhkAyI2UH2aGEDFhgCDucr50zwEshLC8beDeUhpxRrjco3+P0EdBOV3Mlaee5TNJAEWd+MN6gDlD19lGc3Ww8a65cir/p53abNE2Nr+41FT7zzYfNrvfFf/Krv0jRRhLJPh1kslw9+eADdJ33Lf5ki6/X3y7D+43i5u+mbR1RijoBbBd1se1s/0cvMYzno69td+TtRRWr7i97Nxg7yxWm/fblNgmwJPlwJXH2vvfcevtHd98GPtAuvIk3c+v73i1V/ZvuRFdzc8Io8dF9vHP/Cu9rtI6k59w3/avuqesVe33vLtXvrAT7Uf/bk/xinQTHvpa/5W+/oX4flSndjP4rgtsPjnrwdYvAWJBR/SDixUb3H6dsIUWJi9k2hn4iNFgFlRkmA833bGXU81UXsaQEWlUZNvqSPVvTOuwML3pu0EXB6iCrSUfjYqCebDxzD605aJU9BiWt4XKKDsJmBBOepSHyLL5QdeZrznX6tLxiYsgUfMXS+TCXAfplnGmToGWFAOgYW7Vh+CGToEUyFjHh1/ywijFGoQXrpk1Z6kLIPMuoah12GEXPGcDCOF3jteh3bLpKIKtQSwWFrfgbQCmwTyVxVqBeZhnpXV86hrzOIGdIVV16vQ9jp0W0U6sQM1nN2owkwCLPatXG6TuJ/dDyhw86uDWVF1ZR/mB4aj2hTGnbsDMCGHYEYORM3JTfJgUGGQVf+xrLg/ihRD2lffKAlL1I9UQYJJ1WhdcKFalDSSjn5EPY1TFJbKRWMecaj7X6o4F7F1eBxPUZ84+2Q7y/USexW434YJZCWb/iVAkBmUmRMo7cfjkABDu5CsFlM3y6Q6mXVUtct3qvwIfFexldAl7AU8H81dvBxphaDCPQmu44FrB7YZhwEVR2Bmp1EJixQChtQVf5lvmXYZP20vBBYv+rwXxkvUQSQcjhONuK8LLAhnOwtwY6QNGIrNBzTZrT0IZZMmdTKWhhWu6vtDx5U8HGFkQssaf6br6ZHxxH2togswWJ0nb3cEv4oRt9KKS3iIElyQREB5+uzQTx0dSqTcnXse+mu0fQFpxTy0jzoXaY+YPtrBfM0zq+peOW89LH2NRq91+KyPqwBPgIQFqr40tGvU3PBahFTi2FGMs7lOs+N67RJ+iPmoDLPdpV1QoQTTtNzo0HZyYhPU2b7L9H/35NBlbndnHPUoQKJjP8w3Y1eg7EKFJXUsXQNMOS4nCBNgQRtJL/uftNVLlGGcc/TopqRCl7oHkQQIKuxvjhHrnb5DWlHPGuYc9wOJ5yuukVQw/gUVql6pCqXrXiMXvbiFRuv0K13GZmdtVPToQMOGdzy3TQjvHCZIz/zpHKW0gn62BvhYpfzSS5XNU3iEehAbC1WhTh0ECGWO6q1UbbXZav13XcebegtzPfbCNvYwxd4H8oAa1RtfbL5JeYcXhhi9ubVIlUj6TL/1mv43lr/PRmXz/ShT3wzhuaY0Y/G29OOhDvbPzecWaFTQLIpUelvzI4f6f2vaiUqupmmY4fD3NrDo1Ni+blNgmwLPVAp8WmDxRoBFGHQ+OlHRkANkYnSu7BOtfEGkFbzLSuMQtk/qfrxjNMnHsjPxTqfG72dWQPObFz390aRbwKX07GHgWXVTHUaA4Q7FAoukR1TzqTRV81DaILCQkTdd/9QhU2RZAiwoVxi0sfyJEQZC+w0ZntTF+D0d0tzgeSQYPLcMMdqkXBpwH3SllQ+6zIb7E6iK05kpyxFAQlEsd7mVlJllMzXCyZRMASomWY3diTqUutMBFuzCvQKwULda+wqBxZOAirPYAzyBsakebAQc0b3Gm5Lrl3vXMOK+CqBYWQBcLLYDq9faNMDiCKv6uo/dTxkFP4I+GVJVg/YBZA4hcXEzOlc2pY0M5zJMmp6TVPHRE45MnOV3hVa1r4PsK+G9oKLc7GpXQvsMdKfIuVdqEKAFDbW36Ax1dPcthyvNqORoc3H+wlzAhZKLC+j6631Hps5D8OrpSvFe+oKqaKWGohpXAQnLr1RGKZCMnpIVmT2Pa2Gir8QL1CLqT6q91Aq2a8MwkdDmIAbZSp7Mw3JdA1i42i1osP6CBXciv+fM6fbC5z7UTp0+02aOHg+AkJl0R24ZXPuZhyBCMBGPX9BFt8LVP6pMkkrGNX0kMcL21N1AR+lpHxfYb/bbGksC5K6ek8ENHV21VxVI+wo9RF26NBfvSHZl915xp2rpYp8sYLEU4/knob3AQimOm7SZX5hFy0Fb9rErsLAc+c0LSzwUNcMlP1Iln9e40gOZ4Et1yT6vKPWznWL0rH0C5TqGBEgJhZIKAWE2nIR+ASTJhPwcn/yzf6nuFKBOvVeQFK1Q9+xDAVhyLwptJkIz+jkNONCPW2jRVRzFuNo5dJXNnSRML479QaQV1FWAIACV1h66QC6pWXkpK2kebUoZ7a8CGTf0U6VMKYfxNtVBAQKkuQp9AzSGvLu9mO3kqYH2BmNnHbexG3swyEa1SXfUqkUJOqJSCv1NS/CfucnxB7026HcdWKi+OA0Auh+PULqZfYA9LI4xdlSDzEwqIUfH+D0PQ3OuozBp2PqdoBXe8o4fI6a+x/clQXs4+04eDJFsT48w3pVkvTH+KG8eDekZutKoYJ/0t0fz2l8ON+NG2r7ako4JW5Ye359DCnmV3/Uo8SjPlue+Gitv6pM4lWaPniR59Lpvefa5mx3VcftmmwLbFPicoMCnBRaRWMA8dSZAJrBP5k6YMgoy1THYllmQUfBjJnM0TLEy+35EdTXbJQOduqbh/D4CFtxnIuah7F3mZPMgOb00yUC6aqh3lpKiUB7yLQmHcYvBMaLlqHLJvJqjk/mmBMUPfp2ADO4tg2Wx5L2+fVW+mF8TMQ3LxoecX7VKWF+oUi3BhgFm9oDMNgBBlRxtFFwhD0NIHDMJM5R64fpT9QcYVYGFEosOLPbC3O90tREG4oo2Fkgtrq5j7EUYpRJLMHuPx83sUjuLF6NrqHi4YjkB87FTJhpwsQdD7oPYV0wKLK6wG/f15TZNGx2DsZgZgIVl68BCZkrAprRC43OBQl+hVXVEd6Wu1us15wpMmh1II9VpPPTopWeS8sYjE6uqto8MYAjmX/sHZXfVXm9IWXWmrAUuZAxhUGHW48KT1WRX+i+T17nZC+2Jc7PtHIzuJfOFsReMySB62Dc0xO4eh7rtRUAFzKoMqypboxVpItjeV1GDESQsKe1B7UV1H13vymDqqtSdsmVmBRX2jKiswRQKqszbLm66R9ivxI3ZnvvA/e2+e+9rJ7DHmIR29kHDKjWQEbVfu1pcUgulFNoOFPD0Ph2DvwEW1suO1g9u/W0f8jCtTwksyKuDC9WxClhcRqVsISpBiwC265RpA1rrFjXG+ZTL9hE0XkbV6/yF2r9iDmN2d7Vew6jYPC1b2nRUDp5DC8e37edR4AHQSBjPnSz3O34KQNBO0FNpY0keudoXqL9X2+wQmx8KKuxXR44c5Zyhb02P0rBtPFQvWwXs2ZbdGFqbiqh/CYBpW/ts7EYAARlj0CY0tK/R1sZVOmC732DwWIOMU+rqP2muQwOBRXoBdZW25Q5Wt8PMFfRzXR7rncyNG5WWOW/YUjL5gooCsUhQ4mUKyYXAgjwDyKwMeZiv+TtH1pxZ82VmJccR9d5gLljXdSzAwh22s4jAO0tKC2Q+qlqQkB1UY27adp12Vl1KiYXSCu0rng+gEFjcjUcoba+Upqd3UffeF7nZcuQ95cwcnTc84TcPhijGHdLJXf0pYGHYsYeGq+CU3PjDD5+PBdzMyxdDAj3s8NunW8Lxe8tBgBRxPP8hu08PLEgpdbSKlrNSTlLJuH6P558+NhRg/Hni97r1smxWG73nbWAxkG37sk2BbQo8QylwW2AxbrztyqyrdQEMTrLDrBrmgeel+sTH1zB5X7NmPpzM6F5djXOS7fPoaF4lfAcWYe4ND0ET1sBmR5g9fMBlSPRoFNsKGORby9NnfT8WARaEqY98EkkzxUhbZmRgLLJySNl63n7UTF8GSHUb7wvEyCzUx18PUhpZ3pSpAjBZ1hh7E0dgcQimyBX/skUoYGGYAlYyjyUJ8ZpGgOFxtVggpPpO9oyAudqhxAIJwHWMiFfWd2FbAaiACVtCKfwSahOPwzCdY0V5Fgb8Goy+yEKmfS9xVcuahKmbZOMvpRW7lufbniuX2hSrmTOUMZvmyTjD3Nq+0sXdxC2nqkOqQ8UAnXslLjJv2esBVaDzSBDm8RgkuFAF6QigojYELDWjnTA0BSyKYbKeGge7t4SblrlS76q9zyLNgQmyj2jDIAMfhlH6cq+UQvsO7SB0fzqL+9N5GF6ZNHcmtlwyKDK99j/bezdSGRnXSJyGvus7+5L109ZHmxYZfvdKkMkLc2x3g77dEN1+Z3eWaUv7QSOvxTADdKHdQRjK46yo33/3mfbgcx5sZ87cjRrb4aIpZZOJdyVdBnwXoKoDCwFdVHloL/uq6dp/bYtxxsQy5+B1ja2qg/TKSTzjeloHmdao3KhWg8TkKvs2LGNjIbBYQiVIYHgNxltgEQ9GqrzRDwQWuv1113ElReehs8bs7uFgPhKv99st5SBPGfSUH2JJ84wdxo3zgupq5lPADrDLfRn+1yZt0jDjK2BDF8Ll1cs+qAvfA0ju3EHbwzIKwszLPiJoi7pXAAQSJ34LLLQVsU07U28/MY7l95BGMa6HRqWGBK2kZ59vCEMrZDzutT/Rt+TTlTaYlsDC1X3bMCpQ2ICU3QfjDsbdcWx+LhoUINcZgFIUAU4Biz7/2Lyh6zDPWD7jCvYFJpmXBAiqPTEfrO450G4CKm76jL4SycQQx76i8fsOykyinLiepX9lN27ir0P7w0jz7sKm4kXHDrcH2b/iLkCG+4tJGedNT8lgWvUnv+yaGQvOgcMTA+Sh5c3DAVz253WtMFQqQXrf9trj5U1PlHyTdf+dRIZnw/1TXZLWU70YnjmGUwF/m7Zl5jCvp4zr2E+I+mOYxBji5elYmFvT6PU0XOL2eMTpR7/rcV/3bd/eX/2lXx2zvYz9Op5pb58+z/h98jyKBHH72KbANgW2KfCpKHBbYPH9P/z69lNvfUs+sE4ymQ+98oWt335si9mQaZOxq9NP8gAWhg9jGGqBRf45mdeHuk/0ARaUMlcm4Ij1nciHb4H57JFJ4UOuKlSXJIw+qk75zvrM1CkTt13NQiZnJEEhHfPISiVMRX3cBQd+wGUaJZUgQbAEkOFDLIMaIBOmREkFH33CF7CAIeVextZVc4GP9gtTMOV6EVJioeGwh3n5AZF2xVCXGpBSAhmVGzBK0tFVclf+d8OA7+KcgJlY37m3XdOAG+PtRXi8ebxczFP++Rur7RKT/QLM3xLMlB8BmZ0Z84dh0yCTXfHaBDtxbyxdbDsW59r+m1fbFHQ6zMrqFPkoXbF+qZcMIv9k/A5puAxDl704SFP6yIwtsaI9e2G2zc7Ohdmv8Kx8U1ddawrGshotw5h+URIg1YDcE0OjdI2Ws7s4Hnx6e1h2P7S6gNUYOf/I0+cyiK48q/+/wE7IC+4zgFH3AkzyEsbJMm8awPaPZfqnnGCO6qc+87AN8rEEnHX7APunbW+eUbOh3DL7ieKf3ueTQq1SW8eoXkG/abwVncbW4qEHHojr2anpmQAn09fO4gZlt2y1q7rASgCG5E3JEnRS2jJ+jMqa8gz1SN8kFD87o1JA1XpQJ2gVYAEzW8ACux0AjTtNB1hgwGzbacSsatQa/c18u6tZx8hlANwcthVP0rYabS8izbkKI2z6EqOPdxl8j01asrpOWWWQHZsBtYDXSU69du2HmT0wWZvi2bdUldNVsRIKvaBZXwHDLlR7BJ72YecI87Tf7AdcOGYqX8Yf40WQezmG2UixUHu6phpUmP4CpgJ9KWfZpYf0r5P+BLAQJFwlfIAF48h+6Mq6ZRlJYC0Xaej9S/oGtBFX0JJ6Un6lK4fZvXqKU6mFZR/1WcalwEz7oBh7IxXTA5Rj3jDSz26a+pOXTWw/FMYJ9LPA4nPoooRiDbXIm8wFNwAMWFHV+3QIC0lZkVLGbbFzDvOl0rc12lhgMeF45PkM4/4MUooXYbR9DwDjGG1jHVPPlIAfHJZF+o0OC3brQZ6jQ/r1MHlsxSqdUZjhZtS/k+SQbuJWm1XDGbdKkeR62rcmxm/T63n3663BkksVKa9GyZk4P4ZSDHU2YHLNuzRKgvVQm6n3uvhkPO9P9TzhKrB/txzbwGILObZ/bFNgmwLPQArcFlj8wBt+qD3802+lWsMEawW5DXjwI+bKKl9FfzuJepWR7AyZE7dMt67YipGrD2mSSfgCI/52QpallX+JByl+a+htGs7vMglKLLoqlIxwPiYUqJgrPsEE9lnK4JUPqzYYBSyGvFzhM21AjpXv7h4tp0xgmIshHW00ZB6N74qqTJOfFcMILOJNxjic5iszYvlUgTqMSsS0wALJheVw4zJXL/18aWhauv7aGsAcuMrM6qTAQxqW2g6SBBl+mK+dnBMwFTewsVhG/2oetZQL7Do6h9eiReIscV6GWVkAYKgb7irxCcpwHAZuGib/mhu6Ybx7Y3mhTVyabXuxuTiAcfc0DN+0khXyUY1LGtxkrwvrqO3FFKvFM9NHAgZkgmWstC3QIFYD4POz2D3MzUedSJpaT9WONKbOztgwj7u09SBty6R+vMCiPCEBLPitZENmWEZL+ktnGbOs4EJP49rP6BlRfRF86QpURnIRgKFB7kUY5qhmAaxcDbZ9PKvvVR8KUPF52osacp+K+scKD0fd0uHyv/q1bWs76TUp5eK37SeoEEypwqNbUXeAfuDee9u9d9/djrKnxV6emdCNbMiG2hjtrzF7eakCWBBPJnov7WCbJx9iSEvvPewvA6uV3/2Pz9PvqY9V6fWScXZjPIFYqW5dj/G2XqGWkTS5CaCet64Axtw7RGBRXqFgLslTyZDA4glc/c5htK1NggbKvUyWSxqkfNxnzISmYYHTfkomDgIiBNh66fLUY9cUp6v5UZcDLMdFLMyubVy1ZLw6NgASgjvHp5IewcQegTY050WeK5UQUOpQwDLH7kWGn76UzRghipIGx6SUDKDgfcY840UQqvqTwOKaoJ6z160vSEgPV++VbOm6VlChdEy6e1hP7SqmDh6MxG4KGyOlLR6q6i1DuyUMvJWsxf6J9neM97aSph4uSFR/r7nM+dKFFW0u3AwvtlwAAxcXBBU3J3axrw07txPXsKZjShP2Lei5S7DGlc5GXOLbb12M4dl+9n25A+nKaYDF848cbieQNh6iHtKon31AJM085aU5DPn4q3qndxyUMZ2Q216nCmC7+t4/Y0clPDzwZUIljVFfHyU5BCbvT0qGZ+btc9uu592vYznmdpTtkJDRcySv/sMnpFmJDkWDvmlzwowHq9ijMvgzr3vCQ9nyvD/rcQz7FM+2gcUYgbZvtymwTYFnJAVuCyx+8Ed+uL15ABaZMIcJuTNXMvA1AVfdC1S40i9DWOtf+UDC7Kij7ce9T6YCEu0JelphBmEe64NaH9UCFuYMSCCjDiw0OO3AIkx+mAk+wkzUhrMcWXEMkwpzgVpMBwalclKAR2ZeZt/Vy87Y5qPvhM9/P1ZluC5TKYiyTvXxMFzACGGtU8LyXrUPPUK5QZ52BzKc1rFLJayNjKiMqbriMqseqT9pSlDLL6MpsNgNA+ap3QSmwqhCsUketJwHVMwDLi5T9wVAywIMy6yruKShW97TMHUnABZHyeuy3o8wYr2Kd5wJJBZ72DjvAB6jjuzf06YJN4XhpvrqMu3u9mtdtLE4MlXuPffjmWon5bX93PBNL0NLuiNVLQmVmYswd65qW0cBisz2FOoVqlLp81/GUZsYGe14oIJptO5KLGSsA0RpY+kYYEFeunY1jIym5SEJDmkt4wgAot3Up1eCIcC4BLhwTwIlF7aLbSrzGJUn7t0zIG0NI6khMgklP9tjC7MwMBk+N1/7nu1nuXICeiyTdgnaoBwCPEZ9hETc2+OeU6cCLE6ePI1R9IEACb1PqQ4lIx8G2n7PCrRSi0ldvUIDJT3mEzZp6E8p2qe6JyXruU7by+hKO+nigC7mucCFbofdYfoqTPgKoOIKm8JJs2V+q0rUpS7ZxA0iq9p2EUmFwOICdjsabmt/0vt4tYWMF7Th7Plu0PY+UwVN5vUwO6Iftg/kZDUfoO0+I9oPBVjSp9ynRXU4x5XpuGau1MIVd9sk9imocgn+006kbx+7yjPb+goASQNt1TNk+jd9yQNE6G+eto1G9tIG7jD5lCcmwTwSOCUWnDeQWAQYJ559zjIRhnHVXdMaTmBgr7AvaLAtaFIyOYNKkS5x9Tom/ZeR9FxCjczzCipQ7t6d9ko9e9+ixtTJuaxLRq2n5bPvXrPf8lubKnfUXsdL2RrAgpKyyEC4Xjev9lOlYAAwpZwQlpP6M/5cyBDsH6Js2lacYDO8U56U1922NdruR9q1lzEPra1H9bG6T9NvHTfDi2rHql+3exsPmHb0z5Cs9R89465gQiWW57nlrv8YypbHSYM/9T9hej+tFIZ0xuOMvTBskiX+eJDq20MdhhcC6M1jrJS+pw5PfVShDd3pYrjx+1G8oSzf/drvGD36y75xPk9dyahfx/P0mWV17Hhuq0KNU2f7fpsC2xT4VBS4LbB4/Y+8oT38M2/NZF/Gd5nDwwD5QZBh7IdTdBgwPlIaDfvh9bcLPfmQw8yZWT7wROthBQDeZwJjEou0grTillZGIPO5zLbAgtXM4cMvePGQITf9koqMAQveFyNYwMKPd+U5SDqGeNcDLGo1Uua6JlEmU977WaAEmXRleEpfvz4qqj6NvnbQwkk4gIB8XKnVvuIwK5iu0vtOhkhmmB/50Gf/CJlm6p6T53p2kaaqgwRYuOIPsyBTn03sXN2H4bkGX7zEeRmAsQhjOQdtn8Q96uMAiyUILlN0ypVJmIkZmLQr5Lsco9altgOpxSQSiynUo6b2UFYYwUmwjbtPu8otPayXq8qqQR3DePYgaiiqQ0mTmzLJgxtP93KYZ88JbR4uw+QJLuwSGodqY6J9SYFMpTAyT9SLUzDlqrQM5gGYc1WBdli3fMCK7jLvMpihgwAj/aTo7Ip0VpBh1GUoBQwyf2EAudbK8rBHAO9lCPXGo0pK3KbSFmFoaEP7jeHzsad+NEHayDYzjNfc0/6+tNnXGAyCBj13HWFvBXuJtiHaLJy843i75/TpdveZe6K+IzgKIINmeogq5pe0qZ+SCzfIK2AB0Bz6KNlsYbDI4JMOqVR9vsCF5Zd+8URFfW5S5y6xEFxcB1yoEqUKlAy5Bt3S0HGVNrLepCHzsIj3pHMXLrZzc2yqF8YYGwveeVQ7uKCQQma8ZDWXgeo7wdVBVGumcV86Aha46p3i1FWsHqg0it+tBAJgGXfEJFXgSBLXXOBccU0AQVlsNyVR1lmpWUkWrbfjnjE7SAFsHN9btgBa0pX2jruuOmmbRloB4x5VKPqFYFpJR1fhc05yHgh4VfKAUwQlG+kn9BfnEeehAtA1zqeUTtLfBWiOI8udzQXxZKZzgJKk1cKB80jamrpKf+nm/KR0z5aXFgEXlAHzcCQUSCdQGVzdwQIA5xpjZZU+6CJM5izbnjT0/qQXuR0jYFG2LVPMBTO0yXFA0EkAxR1IVWaQVExR1r22v21/S4/LIx+PHRkjEtnD9u/H0Df6Kx+nf3BNaINyk5E9RCe7HIwus67j1nSG55uPK3I97pGGuD1zg4yXzWAkUGn0+BXX+tQT/o7FqWebddiMb9pDfj0QKdSj1KSXwkyHgJu3Rb/Nx+N3/d02sBinyvb9NgW2KfBMpMDtgcX/+ob25p/56WHOHaZPJuDOcMlEOn06Kcp4hHHnoxnmn5V7mXEjF1NdK8YBFkSSgc4KoQw7H0VTMo2uAuC1Vqedy4vhVm1Dd6xe/SA7o8tchaEKU6rKVH2swyzlY70psfADHsaIMB7G7cBC5jQfaRgMy1HAxiuMy5C/EpZIQkjHFKSDZU/5KUsYBH6r7nEQBspTw1RpEAYHZskyu4LYGWw/wBoLB7RYPxjvPTCcnupDh/GCpnHPCuBQ3WGdjfJuwFisUP5lgMUs0ovHr6+2RzjnKb9mpSdgGo7BmE/DALnyeRVJwxWY2wlUoiavX2lTuJ09SPNM7sCewhoCLNS5lw6WUcmChttHD8+EgVZtRSnT6irgA5CiC1X18QUX5/AgdJF7GTCSK716GJlurCtdsiJLn/CqtEaj5b0wltl1mt8Ci95XXBWW/tG5pw6TMEwBJdxLa3pKMbQwjUW/spu5ARMoM6paiyor0tw21c5gBYZZPXfTllGwTJAvDJy6+YYjUVqVfmy/gebmlfQlCKfMnnY1V/AmtYuya1R89OhMwqxBt5sw8Cf4ffepk9hZ3Mv7qQBD+/wN6KWdhWPBwzbfySq09VfNR4N9VcAKXNjjzfNTH7JEI2BBRWTura8DWlUomWJX/G8ABJWU3NAGBe9g13TBCsDQJavet7RnqZV8GXUBCeAU9Z1Z1KEeP4+7WaQXeoXq6j9Klxy30sXFBpl8x7/JOO73ZYdsgcV+gKmSOzwl4TVryj0oOAVRUYFT55+0BBIe3XNc1OKsA31RyYRugMPY07YCSrINE+4YEwgYX6mEjel8IUiQFravv+0H0sKwgtUOLGT0swO3Ei7oJriPFzHaPmFGYBSpGP36OhINVZOcX6yntiN6rVIS4wKCXqHs185rprvAeLgwdwlwIf2QqpAepUw+OwEQSoqcTyxnzSM1r0oLq2Pb6sj2JhVW7ek6wEIVKEHFWuySClgEZBnH+uLkYQKwDpGj/qTq0wFAxQkBBZKK05TzBDYVqkAeoJ00SGcKGHo3N8OxpedZGInO1fJzU6F8Nhxhig3WH3C1f4yOis5PAg3Re2D7uf2ontf7LUGGuD33SvOpRsd42pV30knalWL6qQnwM/13rE5J0bwqg7TBcGsJ+Vdp9sso4OYbXtXCQ8UbUuoX6XibIzTk/TawuA2Rtl9tU2CbAs8ICnxaYPGWfyOwcMrkH3Nr7offTsMy4Tn5EEa9iQ/zHlbBZQRllGTQdOeoKopMvB9xmQA/ptFl5oPo1cSdXAMU+ICbpkefjw3fP8RRnRFYcLhCGb1qAhpWfXw/1ubbJRaqZ5hHQID58kGvuAKLQR2KsgVYDIxSygwjqc1CfVbMX6bCOhVDkFXGsfKbvoBHMCHjo3GqK5jSzHq7UmqpXT2VNjIzsrKxU5GZhWYHUBFRWqBxa/SrB5rXCr7epUp3msoQV9Wo1i4CLM5dX2t/fu1mexLGWfWoaZioadLUtaxM21UYtSuAixusVO+9cZX9LK63A7AuB1Ci2D/BhncymZwyKjLWAhyNyDXgnjp4OAa+0tCVedWh9Dbk6vcCalDuMTF/CVUkGCjbVnoLIKSHNJKJUj1GoCHz5iZigoZ1OIpVgJFtDhHiLUlaqX6i/rzSK+vtfiAayJbr2+ovaVvTJ23Lqv69jKerxZ6l/lJgIcwlzOqIqScv62h/8ZkqVQKDuLolT8srk2jaNpjh7GcyjJfZQO8CO3Rfp9w7aSP3WLCvy1IokbgLYHHm5F0x4N7vRnm0lwyuqlBKehxw9vUwxDCJSqT0EqUExDpkzMB49jFnP+1Mh/ceFCfPLFNW+qGf4EXQFGkNdVEaEZChVAdwIU1UfYqXKIDFGgAxYIpeZJtLr0iBqIOqUrqb/cST2NCwj4USA8eYh3TPZnTpf7adsLskhdoXuVHhAVXsABRHWB0/gkHzkRk2GMRd7EFU6/Q+ZdtnnBO3AxbbyDKuwMSXcXO1Vzx/MW7sIpbR8aUKknuwqHZU8xIlSP2RzAGc7D/Sj4ehi1Kc0JN296qkooMKVec8sn8Gfcx5w7QEoQKaJaQ7ARaUQbqrUlQSOexHBE0AC93jpm+StjTXhuUiamTZAwQHA7U3CvMIRXIBwTGhWqcLJFEHJV1paMvajpLa+W8NOt0cnew1A6BA2XCQVhCGGGl/60p7bNCPPN0Mb522cA46DPh5zvSBdi/tcMb2wPZlH+90oVt9bGCZ7eQ5mJuH0tjx0/cMO3rbw/mgP+XWsFufVBsYjKPe9nvjVTrVTnVfdgzDmyHPhBzSzhvrOuQbFcT8HtLtdeCZ+Y2X1BCJOfawjyuvKUcCEdf8elqjVIg9ykuamZoHCY6l2dPZjM/r8fSNMaRtWO/76e/PBWDx+g/+P+0dj/1BqLf9Z5sCz3YKPP/Qyfbdd7+yndn/zPGm5v5R4X+eZuN8emARiUX/CDm3jk2wTIpRX/EDzn2Yf5jmYsxU54Fx4rcMUNSh+ODLzOQDQjruZhummo+i076fgnwovRsm69EkTIislhJWhjWTPIECarhmciaNSEJ4H8mCH+6BiR+pQsCY1ITuqiDAQWDhCi8MgXYgltW0BBaRtHC1TMYpT1HDaqP1hOkNcwCzIEjoUhgZzX0wjDIbm8DCvJAGkH5WxKGNcaL+BBPuJmluTLefVcfYX1B2V0itp4yY0gtBhafSi5wyaEgvNOgWXHwCI1slF+ehs8zBftI/ZLlCf1Q/qKeelSYABnsAFuwl3Q7vYJ8LgMUejLlZro7UYie0t+0ESKoqqfJzCENuGSJ39BVYuAqua1KNZ91nQoDh/hLSLjSBPtzmkHbSRMAp6HAfEusF+dMGMoLeqx4m42oay9hxCPpMYx/PVS2TgdNgOntLkIb9IBKegdZmp7ceV+zd6M/+lM8/iZBd6F0MfTGYvgsjR9vH4NdCwK4VUCnjapKgjDC5tJ1eiC7BMD6BmtACm+mtYu9y6PDhlEcD3DUY+C6xuPceJRaHo8Zmn98EFkqE6IO03QSMoupASiy65MaVdQe0dXM8eaRvd2LmCc/4Z7+0/5e0ouwrXJ23/oIEGWrL7TONoCPFoY1v0n4CRIGUdJJh98LaJ4YAAEAASURBVHfaFmChJOrJ8xfao0+chzleRNrT1YBKimTfEGRmJ2j6lqV09VvbpwCLSfT5UbeZhpnV9kB1sUNIK/bThxwPAZXQOuOLNnbH+bhxNR+Y/oxB6qZsx74BwUITAZqLBgJUbTUsh4e0EjwLTJS2WF/7gnSzXaWD7azqlPOQdhVKtQxn2a1Lxhd9j4aBJusBKKowubu25ZPW5nNQKQV1m2Ll3zrKvFse5xINtgVGMdqO4TYetcjHxQlwqKseSSPqT7Rt5hPno5SCAJaXE1hEL+RkfllV7SlX1KC4riO1WKeMgvJShaJepLVO2dwEb51zg3aYAHxNUdY7kVI8d+YQwAJpBZKL/Yy/GIunOJWzNLA/hWj8qac+tNDDka44CjkK219XXKNUnHwnRi+H4Elj7KG39nHj8L9Sr/ejsevPel0v/NvTGeJsvqi7SEAIlDoNLz+pTr1uXHPb0xzKQ9JVLq95xx/eVbC8Tbl8fduDCJXVJ8ep75D51+nvzwVg8fd/98fbm/7st29Ltu2X2xR4tlDgZdP3tX/w3G9sDx488Yyp0l133ZXFtqdb4E8LLKIKRepOeplWnV/556TthFib3hUz7oe3r1DLSHZvSs7VhIBxcXWUeJ5JM3N10jZVJ2A/rHVw9X9+17Mw2OThql/eEVCGIROzkchIZqzKURKLXh6ZtUgthrjGi1QCpkPmJT7jKV8YTeslsAjTJdiwIJW2aQhSZACVXmTlESZRsBCGgfeufk7ygVe9xXuqn/RUyYnaBWVMeYi/g3OXRs0wApOqBkE3GQ4pJLOUOqGnX56EBBaqQw35wWhg1R21KMHFOUDFE6hsPIG9hRvoCdf2SfyhPjFWh6liab3thJE8xOZ5hwAVAozd6xjooiqzqroU9BAcCCwOAiwOAyymUPvZQ94b7IER5pW6aAB8CUnF7NwFdMrZG8HVX/IKfaiX7UJT59o/ovxK/fKb8sksSn9X223ftC1lllbuL6GBqqu62mu4X4SMne5K7V+2s2BE6c4kjGbSJMOisQyn0qlqL+ltfaIOQ3rq7lSfTiH4E1JLbvotfZewu9yAjH/rgilUqZZx1epeGk8goblwaQm7FVRsBumSEhqU/duJY0cGiYXAYpp2G3YuR2KhdyilBmlTyjChxAIgqWrQbupkvnFBG+Z7E1hYtvRxaDE6HFPUtZ/WuaRixZyrsubgLkmEUrmSUPh7PaBD4FFAUhBiHX1nPQtYzLZHHz/fZpVYIKlRiiDz7PgKsGAM1DiU4S7HCt3F7P59AgvUoJRWHMbAnb4jQN2LBE8mXPAIEWh3VPQiBcJzEldBoW2qAwZpZG2VbPQ8C7g79mrRwueCAMen41V1Ixl7AZRtG9o4jlPfkubEKBpAkT5CvCqP443dtu1vzAHWU5B8hbSUmAi+LJd9Tm9ven/y1I1uABbpLKMupnRDo229llmW6r+UzxlAAEEavc+VwX8eZ34YIDAggXZV1Yn+oT3FOuqPq/TdNa66nN3gmS5k7QrV9rQ7KawJOuiz7llBJ8KAG2kaZb0bYPcCNsE7AxDSziL7VUjetABlyn3Nx0l0eM6lDjPyMAJ/6hd/h8ejm+F3zdeGtQmd04cXlVG9GP/rc8PwP1nwzhiWqP/Og+F5ovYXPek8rD95NLwfXqcceZuE+dPLNFxz6WmOp9XD+YxyWp86TGO45RLq8Xvz/fBuCN/fJ9JYvBF9eCaY9vc2sNik6/bdNgWeDRTYBhabrch3b2Oje4VywvOfx9i8yIfNjzCrzUyKfkBK/xoxP0xHJAUy2zLBflRNg3MDBj1pED4T7pDn6DlPzckwphkQkgme9JNeAQYf+b7nnSDELObDMJvAIuCCD67XviLeV0sjmRBcyJilPtSJxKyXQMhnKUtKNaQvsODswEKjctNWjUKjURlfGV1XVZU+mIBSkSt6BnI1lfSz0i6gIMw+1J808hZUyHD0VWTDUWDUaSg7QEVVGe8FMdpe1AnzASNyHcnFAozRHJ6iLgAuNOK+aT04r1EPbQM8J6jnThivvQAEFCfavg3u1/EOtLrS1rG/WF25TGHZT4O2sjy60pxhtfmI+vEAHxmsgAHCLMYzFHr4c3MxCNZgmMLT9jLx0ISrEimKAuiA6YuBNYwav0MvaChtwtDCuNoGvqOjhMtwRdyfxlcCpaqNaigyePYv21ia6G1KEGcfTZ8hlvSVaTSs7ZE2IYxG45ar90nrkz7Ds/RVmDbfl1oSTBpHGHZW+ZfwPHXpop6wZtk8bgGPSYAF+rOqTALDvVjB3jkz3U6yC/fp026SNx2JROIjCVBVSobfcaBNiWpSqlMJGndGUiHDTJkHYFFk6MxM9fcUaPhTjKV9VIAxSCzoZ0onovoUwKA0yJV61YlgcqGxkopVw6RMBXYEFq7uy4SrCjWLKtQjZ59s5zHgdmO3DizMWvpkfNsGFNKxEBfDlHsSD2OqvekBqp+Caz1gabczkjbRfwWTqs9diBode63QFtOoOOlZTZBu/Rwv60gSk28kXra9thUy1tSLfiM4UeLRjbALiMCY896+lTQYAwJYpWDa7xi/20rYZ7S5uITBtR6wsgu36ZG34MwVfuunhzc9QOn9STfSjnfrIABxI0GlJYIKy5Sxm3kPeO+VuhWwoGvTn6W15Us4fqevE4ZOgRqTgEIvUCWl2OD3Gn1lnT7jmcHhoDC+bQYIWaOfr0LnDWjjYsV+QPgpQN0DAIvnAyxO4KVLt7JCqPQoolOsHI6yjLv88n29qPmagB48Gu7qh7/yoOIOSSVo/8ObhDGYaVWyfgd6CF6bcdIZXg+v6tHwYriMYg3xU+bNgo8S6MG9mm/yNnLy4qn/ZeRLtFHZD2mO1zll65mOp5OkNnPp1Ul+A+1yP1TU+1FalKGHH+VFAN8b/HWfA16hPjT/8fbY0lyn7PZ1mwLPagpM79nfHjp4Vzu4S/fzz4xDfkre6Oket5VYuI+FG+TVRNinQ7Iabp0Mw6xwdeVQNZMw13zYOgiIzQMf4DBwmZxdtbO4m0wg0fu3haeVuB8lVyL7x9enpjliSPnd33sVgHiYz602FsbZZEZV55HRqhXMSCZgDnIVSPDBKWABE0adciZlyyhDWBKTrMqTrqu3AowADX5Hz5xGEVToptV8LZ8rxqr3dGAhg6VHIDeL0+2mdZMIbsQVdpo4yYu09UFfgEIwUWpQARswHrot5U9c0S6v4nYWlahLnBp2K7W4QXoCjovkfxlmaYI67uHZft7t5LoTYLFzDWBx40rbxSZ6OwAWO2F+d/DettTTzVFAxTF05A+rygIjbzurbnMZYDEPoz0/fyG7ObvbsfWTUZepV8oQiQ00c0V3AX1zPQ5Ja9tZ5tGPfbVjrTqHSYYQUREjnh9f36f9uY+rYZ/LrPE7kiIYWxlyglVa3Lh/ibT3tCwCC8tju2ivofRDg3TV1ZQSFFgr4KlXLpnXbNxG268jiYhNCRIad69eQGpxAW9YFxdx24rUQoCg7YCGy0dxMXzHsZl2112nMFo+krYNgwzoCrDoEgvacQfxNALXyFbPX1Gto7xKw2x769fHw9AF00eqp1tXgYIgWAa6JBYBaeRlf9MAOkBi6McFlkvtSdUs3dC6A7fucMvmAqab/iEA1KPRY4+fC7Bwk7xIPyQwR8Yh9LV/CD4nKX+3oZlEWnGAlfHDqN6oKmT/KdUpJTSexdpS3NRRJl5bhov0DRn/pKc9DumGcR/qJQ0d1xnf5FvgomyX3ChRA+krg71EhWE8wXTTeNBQKZo0rfzlqK2KTLn9VXuLsqeozQAth+PeQHbR2FVQD+ujpMJFAPuXzgKy8R3lX4ZmghuBRp+LbD/Bb/oW9zRoNSPp1rxJu9k2loUxRaIBFXp/uqn3J/rhmv2avrEBoNBoW5WqMKGUfQdxnXfXqJsqUKucEDu2Wkco573TB9tD01OACxYHaJf9zh1K6jKaRtP4qEy5GdrYsns4Dyc8P/NqqAKPh4MQo/v+bPM6/qpXf/Mtdz1yDzgKZM51JEt+5HfPv0ft8Q3a35FGZ+QzhnqahjURrkk9CfaIlV9PwqcePR3vQ5Mhrc34PZEes/9OjM0y9cfj5TVIP4b3r3vt9s7bnSTb120KbFPgmUmB2wKLf/HDr29vesubRzXrc6Nzaz42PHCCdU0/HzuedtWTqLUMzJ/PcvKR9Ro992HyN83xybvPxGE2YZz88JJ8vsky7zIIXbfaj7+rj17zMWfSDlNBHlnRJqygwo+7q8B1DxMugIBh6aCi30d6QVrjwMKVrdG3gDL3lcduhGl5wrzyQZfpneR3mFcZWICFG8bJFOl5yE3dZMg9ItEAVOzHdkDG1MNyiBFluMtGRFAhmFD9SWZ6OGFAZIa1t9jpuzAdO9oKXIcuaJcAFlfJE+FFwMUczNN5mKB5GM6dPIf9aPiNSX6q70wALHbhKWr3ymL2uNi9dj12GbazDN4Mqix3sFHecU7BknnLpC5jvL24eAl1qIvZ18Idna2j72xn1Ub2Y6gtjVQpcVV3gVPJiQyRHFXaDgLzi/IU7QMkpMHQdv62D0gbV5n9hNtnqo94zeu0U38GUdLuBSrLj79Mrntr6Br0CLYR09OHWX12Z3EZRWlKW8IkF20LXJS6jfr77GANM34VqY57eMzrapeV9ktILYBnxBe84NYT9Z/jR4+0EydOtKnpo7TzAUrLyraMvuCCdDJ+aDPbdSfMYDY1Q2rR7Svs3wLyDi5IIEzNaJxYX07HnfQL3eg7sSfg6kq7oMI87VMVRuABcAA03kTd6Wpcz7LHAntb+FtJhYw2yUaNx/0XzmJjocRikb0swmwzFhy7BeaQFtHG2hgI2HQvrA2N4GoSYDENY6unJGkiYJIps21W7ZcAXVWSlNqYn2BSqUg8J1GfvUg94iSBtg5o4pn1tf2dd2wTvYo5Bq2bdi+LSAuUHOQIgekl9kPu3Sgv0kPi6CCBjEMXPYgtAUq0iRD4dvUl0xQcqw5of8smeIIKwJKg1LnFDS91KZsN8Ihbm2VSSPJTIuFhMbLAQn62ZQ7e9Tmz99U1Aip11FhbUHFDYDGBBAK6rTrWOJkAaB9xknMt7YDEMcACwiilWAWgarRNIwQInVRaceRQe+4RdoR3LxFcS2uwTbVyZP4e7tMIEnh0jN8PD4k7CkK8qmOFGz0fxd+8GU+pZ1dveTP+skdJIKHgU7w0yngiCeLDIXJ/J+HzGFDEfT3272bYzbFk5LH8qEwARC+PsYYKbj4nfG9k35mfh0l55Nlwa48dsh69q1db/w5xt4HFVrJs/9qmwDYFnnkUuC2w+L4f+sH2xjc/TK2Y9eq/c3AmXljc+jc2p2YSD0NYjEQnR1RN+LB2xr7AxRCxUu9BMyn7wZXZ0MahJvVa3Y7LUpk/mAOfyzQJBmQEtgALGAKZRFcVo7LBb1VjvJcpMe3RKi+MeAELVqWH9GR+tqgqULrUzY+UXwmvMgvkIaNiuTQw3u8qvdeACiQW6N/LsJq+BqMrg7tRmQx1zvXprxpPGAzy9IMkU5mVa8sPY+GpTUV9UMm75+tqd06BBWofMHwCC0HFZc7rpMcl6lAXYeQvkP88K6qExu4CdRbycn8LO8AEajA7b6y0ncuXOC/iNeoawEI9dcpDfocp5zE8+pw8dhxVkMMwjri0JA33PHA/hOzmjMGvqlHaIKifrsGsK7yee1mB1rD1ssawMGHq1rtSHO8/qqZAHyVFYZqkNTxYBwQyd5LGo6vd2PaCQ+O5K3SMcUnTPsCrfN8Fh8ajmKSlfnx5yRntMRL9f8AFAEO3unoYEmAoxVBdKqpqXCNJIo0YQUOX69ihuGu1+3fMcc6zO/UKqmca02rDcZzdjO88djTA4rBAbP+hMOLSwzqv0g45bEcYa0Fl7GcENJ72KercAfKWfkfE1M+6WVf+damFY8AzfRmQEANu8wNMxIibZ90rlG5vBRTajCh1cp8LwzhOBcJRUYJRfyzA4lIkTbaVdLffj8YW4ZX4uEncNEz3Hui8O65Ud46kFQILJUDWw/Gs21a9w9lmN5GwCVSyUMB7y64Kl/lkTFBfx2nPV8sMx69zjzYwAgvfa2AtoL2GGpIEsm/0ecF2P8h4FOTuA/A4ZgVJbliX3dsX3TTQHduxMRLdDnmoylfqTzoOEFRoa+SmfsBy+lVUpwA0lwFdguZAMstGmdIBKYdXFwnSlnZq3lsXHmc8d/UnVZ/cq+IGBhbX1t2zgt/Q7Cb940bARjk0mKDNBRMldYB2pGSf175idfe+tsZ8sxt6H2S83YsK1ENHpwAWuETm9z48ZegsY4A3GReWxyP0zZ3jZWCyh3IOj3Ox3P3YTKfCp0795RDQ5CVDDm4qt+GnF58NZainRZe86g9uuVrWpzpM27L3YxyAjJ4Std+Pp2Nf67+Tzi1lSppj2ea2Awtf9vDj6feC9Pfj5e73/UqYpMmf7/727Q3yxkm3fb9NgW0KPPMo8GmBxb9+80/lA+DE5zzoZ7F/3DUy7iopYYBgOvzd3bmOJmsm3gIXpT6RFU+ejX9UKv2a4CNR4AMqI+JE7+qtoEAmwRV0GRFXxWUsXf0sqYIpyHcXgClD2AIWXXXDq3laLhmzYsRgxmA0Ythsepw3SdtnASyENY7pWm8/IpEcwEC4Sh0GlHLt45xEZaG84iitUA0EPWzKajoyU/rl915pjsy570g0z0xXEKGqUYGgYcXcfAkfQoTeAhreBSjJtMlgCSxgSgAWy6CJRWji/XUYrBvUc4k6XYZxvEKd9pKGm2LJpC0IDGTwKNvGTZhLNs9bX7rY9l1dbrvwEkWqMfY8xIr0UaQWdx093mZ0G8q9gEkayqy6p4WuZ5dgVgUWGv+qFmV77YeZc1XbOrjTsaovrrDLyMWDjlIOGMJsIEbfEnRKF6VToTfltJ0jxaGePg/jaPuQTpcudaAhUxzJE+8sn/3HjcdKXa2YRUGgYOeAABDQVGpRZROjBOkA4OIQtiV6w9I2wP6X1X7qKhMuUy6ocK+HWVzPLrLifR2As4twd8wcbnfdcaydxqvCDEDsEEBM42xVkmTe9b7k6rNtNsHKdMYCTGTak3LZp0ar3KTnuPHM+KKNR32Xfln3VU/7lafMuaAieXEvsFAaoaREd7dRgUL9yTpcyWZ5y7SfdhZ6UtqIxAaSZd+PJ85dyEZ589RRYCEDVnYtlpe2oA62rfQL080zmVfbRaCmW2Hdsdqf7cC2vapDURkijFIL203Ji2BKxrDKDwiDVr7rY1fJQRYkSCn15qq0UpsJ+5J2DbzIO/tzdoKn/aXdXmgqSKjxBthO3tqiEI5+mn1PyE/gmv5HWZS8qNIlaLIOXicZB85tyZf21mtU+i5jCKUrJ5/MfxAKhp9akY9lsg9at9wYzlvy2AFdJmhz96a4mrG7wRWjdvrFGuMaaJ8N8sBfob3qiZ7Osb1PpP8Tf5V5YwLpmNKjmQN724MAiudw3qfxvG1lnGRd86TFGB0+GhXP0lYY6ey/XnDrkN+WY4iQZwlnrM20TWV0bD6uR9BZumw57Nt5MJ7KEGIsrGVyrvQY5TGkl98mYvJDmAQc/oze8zvp5DkR/D+Wh4/Hv00J5p8Kmny3lJK8elnMflS3XgZf5sXYu14Prr4yf1PZBhYQY/vYpsA2BZ7RFPiLAQuq6MTXV3b8QMV2IkyvzI8f1LqGAefe8DVZO20yZTLJFtOvisoAMDLx1pRsWFcfZY5GKkncy1z6sZcxdEVYZlWQ4apjrXyzksq/pJKwqtCUupSMRKQUggA/5JbTPClSL19XIzE91RkEFl5lUC0LAUdxq57WhXRIs6vMZN8O8pqEUSg//pOskpbahEyVFJDZc5XWOvqJF0RYbhsAyobB1GBYQ2CZWVdkd8twymD3ciffUrsx791jEgsCUW6kEJR7GWAhwLgKY6Ph9lXyXEEyc53rJOEEFtL1IkBnWaNqyrWK+9nrVy63VaQW+1D32YcUY49uaCHXAWg+BRN+dIqdprG3KPUhpDHUQbrLuLppnhuvLWKDoORiGTURmUSySx2UMsVmQZUOaCozqL//BVaMs+qrlIPy2WZRg6G9/F2AofYvEFTYjraZYINk0p6dCTC87XgNxtJ9UwyjMbzMcKRlFEbphTS1jwSownC66i6DrHraPhizI9TzKHsvHD92jOfYv0BrjxgCw6Bbz/mLlwIszmGLMMcmctZhHeb2KCvFAgs3yTtx54k2M3OUHbinEl+GW8NpVddieE97R+WJOklLXZ5KI0GYfT7vwkQWwLCe0i6gmOX1fi89vJfe/bRfBXzLoKP+pIvg6wAKd9++BqBYvoIaFPT3dLM8VaeMqzoWRAUE4LaYDd5mh03eVK+S3mW7Au2g254w34Iz+ruSKfffIJAuY01H4HYEmxNdKVt2FwGu0t9sF4iQ8Rt3z+Rt/zCMAMx9IzT2dx4QnKjCpgcmmWkHk+WMShVljDtX+q9jldxrnCEd08tUQAvlcXHZdyRfY26QjhmnXO46ztEiom+4aKHxuOpYAosD1EsPV/YP3xEl6ca7FPX0GumIaVs+T4BFdgOnL1pWGiK0c7AUKKAPCiI5J6DNdZbXl65jC3VtFdsoJRbMDUosSEcQozRP2lv+gG37hvOZY0F601/W6UuTSEFnMCw/ycaEAot7D+9vJ1RRo15x7U0v7Ox7OiRpjg7S9pC5NTP/efRrvTEQz6Wp4YbDkJvh6uHo7XgePYJxe4BesaRR6diX+5hOFMN4mLV/clR79vvhZSVL2hVqLJ8eqyfV06xE8zavRs8pogQfOyyXh3XfLMfWcD3MKJppmIxRe9r9yrsePnUm4DawGFFu+2abAtsUeIZS4LbA4vtVhXrLw5lIo5rgB5PV/HxYmBRl8sZPGeDOBNeEyfTrJKpc2v9DHBlEmfGEHybXMJB8iGU6cvrRJ65x4voUxiM744bxkkmpVU3BhR9bz6TPh90yydDLCHivekIv1/jHwqJl1XtgVGWmrsNQawMgs6CagemGKU1d4ZLNh3q6B4HMqUyq5av9GdwgzJVadJr5yHtqpG0cV45N1zxWYfKjCiRDj+Gs6ekVKC5HcT+qetRerwINmFqZbcttOCrDb58BPIZ3kRDxTmbLjdtWoOMS9FmCkRNoFMCQScG+AlrsGxiNizD3lzVchRG7Sb2VPKwCKvZfu9IObNxsB9hGey/tto9svaqr7mZ1BzHi7ipRqg0pMYqaDcyraWhrsYSazSIuaGVcVW8RiMmMxuUrbWT99cAjQ64XIvXcZaJsJ+1LZKptA2lmf7AdYuwtHYbBFpr4nLMzpoKLa9TdVWj7iL9lA3pY0w/QkI7pK4INaQr9aQPLePzIdDuh1OHEnZHOuM+ENJc3kFHU3mJhYREDbnYdn73Avhbzub+Mvr42HHdiY3Hv6bvamVOnMOJGcnH0zqRvfBlnVaAidYJ2qrFVmYpRFFTEjsF6Dqfl9fS348mzO00IyKDNwyRTV8sXxjv9TNoVsFBicQ0wocG2kotl2ibthFqXoMLTdvKQWZXml2gbN3pTYlEqSoBE6OP47V6SVBESQMRtqw0zlEFg5O7Uupx1LFpHxzNDjXpAb55JDyegSLdI13qpXndh/hLG2KjMUYbjR6cBa1ORiFi2SMmI41hwzlD6sYLRtuVTahF7CaQi2XuD91l8UFWNcWA/tfNkOoKW6VP8jood9LVOSrP2aePBfTf6V+rmOLfnCVz5D40oi2nlnyXjXlBhGzHO0gb0ccuk9DM2RYQROAoq9giU6HemsYwx1MLV1XbxKp7jBmChu9ksOkAD6WLb0wk4WbThqkqX9lUIONoqYwWCAuKmcC97sN0PuL0PUHGccusJynFjdP4PthFVbhK22MPh2+HY8nwsjIl48KiHtmzDo1zzMpQa8hh7mTjGHegUWhG/5xBa+pszfX9I0XijfMbyTqWGMLkMZck46c+J3NP30ebsMfZ07Faa9LxC854O1/Hn4/c9iM9Gz6VQKpxMR4Xo742Tcg5l7oXcBhadmtvXbQpsU+CZSoFPAyxej1eoN48mwDAGw4fOCocJ9EPJhy4MPB97J0tXFvsEmqsTN899J4MkY+KqYK3G18zvaqbqQnHxSB4ywcZNHqQfVRY/kn5cSceCK1VwVdbVyKjK8DEO00j6kVb4AfakPKbjmQ9L5vz62MbeQmBBetc43UROZta6esggVd0Mz0edvP1iWJcuGVEf2/rsh2EQWMhoqbfvpnLuoi1TLdPc7SyWMQK+gl74NXXbYTxkimU09BKlr/99rJLv2+upa9XSu0/ZLX/qKO1ghGEsZFQEFjJa6vnrYvYG9bkGDVeg6bLMFnRSYqFEI0DJDzuHqlCXARXLMGfS8cZNXGUCKvYBEA5iHHoIYLGHvS52ATJ2wKC60Z5SmUOU0b0tNOQWAEkjGV3VfGRQVRe6is77YuwuLgMcsEPICrK7TrsybPu4Kj9IL3inJyKZRFo96enCVwawGOhiKAUJBRi0IaCN6CNpJ5oqkgjStf1hS9I3AuKofxhK6mt6ve1gxQYD+QIWBXZRlYIZVg1qBmb2GADhSFS/BIgYxtLmfQzIlF8CXJy/MNeemJ1r51jZvwSwsC3dGO4kwOS+M6faPWfOtJMnTyOJcp+N2shtB4x11OhI06t91D4WSZh5UC/7V8YSV4Gx9ZLOjgnPbpNiWJnW0IJr3ZdExz5doIF9KAQWSiuUWgAuVmgfVaEu0zZKmyK1gDmXrqW2h6oPzL1G0XrzEoDbx9Tht+2sp7QSaCpNkAnfxbK4XUvG2vYIABFos4quq1n7MK/ShwXF1s+FCse8wChjkD45jxRoifIJHNxgLy5ekRz43j7gmHXs2ZcsVza4pJ1NR4Nq7XguI/UIgCdO3L8Sj+yg4UBXrtIzXqisB0y4UokYm1M/0w6tycsFCtsg5fWF80DGXKWVtrJrWXkyceaIJHQsbx9mgcPxyiktbUPVwy6tYP/kCbC4ug6goH/EoJv4EMaUA0QnmGMEEAKL/4+99wDXLKvOM3fleyvH7qqmI90NAoEAIyQhopBBAUU0CggFNGM945FtPZqxZ+axNQ5j2ZKFQCjYM2ONwyOJJlvREiAskEBCCREUAHVD5+6q6gq3qm7lOO/7rbP//9xb1UXQwEzj/9x7/nPOPjusvXY469tr7b2No5afpl5wL4jbTZ29AQ3RTZvXt2vWw3eBMjTZ5s07v4nLH9sZGfSmjuFVHvlR8zQ5JsEmN6OYBj7peRxmErhu5KXpJ4YQw90QXfisN569zyEN5cRNXkxILQ+T4P1xGnbiQnxJg5/6rziSRLWxeDXN+r9iHizf5UfoNJ7xq057PNeL8Ln7MQ35MMTXr+M8z4DFck7PnmccmHHgscYBv9UOEm5ADh4ffFcuXfoJVoV6/ZtZbpaPoB2vH0JH30o1r5CKUMZHTUFDcBHhl06zd5h+aNKR9l5d/4PAkV2YCRtBkLgzoXOYiFuAASEJioyzBPgapZMWO+tuqqQQo5rfUW41B6FFoTX3Ct0CigIWhpVm6bNv94OrQKYAo0BVwAKGIER7ZERTAWIQKqYfKb8Urt9f5jSOdLrkZk4EqQIXmlCw6hPCt6OL8szJ2wpyC5gK1aRZJ4yy9KvCGYJTlp4VWBBmHkF0jjMbtSFEZSQbOiJkQpOmM/IlJjTkNTwnb2SHtNROMJpLvk4N4CKaC8CF9toxn+D9ESdUO+Kr8I05VlYMAlSsQgidZ/L25mgsWJ6WVaNWcjrJe460BFCbmIdg3qRTASdmKsSiuZD2/QqymjodVXPBrtxHyLMbiJ1idFmhXs3FfIBTzUHRht8ysGydu1P7YAgM8c2zQqKgojYtY56DtvqAGN0V9hUYNW1Sw2O5KIAan2EUXq1Llpm1yjK3blo3ep2JORvhFSznEfjdp8DR+B2sHOUE72zAB9+Nw9FodzBfRIh1dah97MS9j9WTDjCRW/FJQXsX+1ncduP17dabbmw33ngzoJF6ABBMeyCeaJzIvxvoWYbWcw/fS1+vo16tyx1Y6CcgjrI13/q1DRhucpL3jJhDZ2kiBBZokwZTKHcQd9K92iRBn9fTADvLzEnN7gtxAm3ScXjcl2FVJBQAuRpTtBbwynKfg9eCaldE03xIMCFfbedePTcC1FwdLHu6QL91WQ2Q7cvyti27IaIdkacrh7kAgIBBEysFfuuLWi7zbBt29S75Qqbjz3IWmLrsbIAF14AQ4lZLBPnpI5yrEQ0V9NlXuDeFWib3poi5UzSppVFz3o8mWd18yfyvoKxWkOfaQ6by2sFKOpV0DdSzlA1lRDlpimU5powtZ/x0bYsmWwfZxV1gcRitBVtXMl8D0GFdkG7iMd5V1JUV8JsOAVBDvaXPyRLK8HwToGg7/L0OUHEd+djF/ZJ5FaRn+aWPsAJxyPfEzSUHfnLrj2lymJXJkYepy/RO74ltEm4SZnSjH2nwCB3cpj8lqUrNFxVX/AyOhuh0T9LRA0fFVvf++n5yDPcZCKpI4n/qZUqP+U3I6ctJNP2m09CfE2aUXLmPHapuT/wPN51G4+txdjfj/O7v+M7lQT5rz/YRnYZ+HSemm7R1bWhvnzt27Bh7m93PODDjwIwDSzhwVWDx6tf+ZHvDW9/EiJgjyw6eKXyWqZKjwHY8NXJeAn0fcZ10UoSxY8Jb/Eaw96PIR10hsEyVahT2NEKBE4kVajTDcFTS3r6EKj7g0IBEwcfIVZ0cofUsP6txM851fPhryU4FsdE5AAs/7B1Y5B534/Ajb5qn1VYgGDvir26jgAX0IciYrh9Czy5oKMSWuQQCFh98Rz8jZOG+HuF0I8Kkqz85Ki3Ni5iiHGGTtUNHDiN0s58Dgq/0bNZsignRG7Ht1+zG0e05ruvWltAuj0Ovgg15iMBm/jCpMZ+VFwWWMDoFDGfCQyeqnkb4PkW5lYkUWgx4fRQh7RBATo2G80ooYXgAuIpJzOm2hvv1aC22AC7WI+bMN+zmEdCdc5FNAMlvBHDodMdwzV0cObVe1Kg59QR+KoC7Q/dhlqR1wrOTu91EzDpSmqsCAgp5Cs/WIfMXPjLBWvM3BTqBh/b5Tvg+cgw+croakFoJy1AeCPBqHg5zIqh3AYsIqn0Ogn4sB8vaUzCauTuavgz1MSPv5ENTGOdebGPDtq2YmIi8LQfFmcwpIV5NcE4gFLrB237Md/aitXAysYBvB5Nmb7vp+nb7zTe3mznnWR1qFUvKagYkn9y3QlCRUWfyZ7zW6f4h73XMOli8sbyrrZjfnicFWMN4evR748qiBC4vS5kGWLhcrvMrRhO3rYdqX04KLsiPk6stnxPwOntDIOC7z0O0KvKF8lBroWCrSc8K0rH9WffdEM/5EJoTCSykHQk7q2058dnFDCz3SdsEYCjSWRauzKTWxEnkrtakeZTzcKzvBIqgXitX1QaULgFreZjtRfyehF6BQDeNEniGJ9CnJ8GEYDUghfYYEMrzpmgWnaxv+Ra4E4jKA0GVplW2/eSfPGZuBOWWFeGI0zpk3OnzzC/laNmVQEteDTuclpBx2+6dXL4IeDtC/Tl4nKWY0VYcO8s7eh561uTZupowpLGSPmElfHWzPEG8+6YI1q5hKdnHsav2DazKdQ3830pdXg9f+mRtMp/8y0NpDJ1DPQnzuI+XpOQDh4l6DHWqyKm81QtzWcfU63DXw3SPXE2z18+JMxFUCPM6xEXYHm8WXZLeZeEvi2eIcIk78UyeicO8G9FyOip2Ihj7D/2mKn1+u+p+HDYuk+ziq7wkzPRHx/I0oWX6Mnc97jwM6c40FsuYNHuccWDGgcccB64OLNgg741veVMEGztn+z4DaGfsR88+t4/4KiRmAywc/bMz9awRxhqtK40BH9t8jBEg+SD7kdSfwEKthWcXmuSmgrcCp0KhNsyhAeFV4cq47bsVbEpjAWjxo8q5CiFBAS8aDMJPBPAeH1fjNA7Tc8K2GouYz/DsZ6EEAoQNBQf84kCeEX41+Uge1FgIJhSAa7S2C8sBFpg0OdLtCK2jQwKLo4s1eq/QZNouLevov/MWNmxiAzoBhSeChOYjtVdFjdwbj+lqQhRBZ3jWzY+UNFom9RWNTJc0NCs7hxnUKQDGUQSaRxBo9nHuRXg6iv25wMIlaN2cT8HHiearARXudbGJbLs79xomd69GSF3FvhfoGCIwadKkQLkJejcCjlxhSYARGsibwm32fxjAxVF3rj7CBnNoMAQI7htggTqKLCAQRBg4dYdcKLAp4Ds6rZsmLS5bK+/qRGCG9ghgqY9lnqZwa3zyxZFwlyHNHgnE4afeU967ao/xrqQ8Nc0pjYcj8WouENAou1oRiKVUEeR6HZIu60nRcg4N1CITuRfaA/segS5W0+K9O0hff832duOea5lrcT17WjARnEnca9TwUK4CCucZZL4F9cr6nHxDjxT28rQOGp9txvx0P46EEwSf+peNFZ6byhtX61y0RwEX8qGARV9q1lW8jjHXQo2S4EI+aXgTsz14fZA5JM4bEVioVRFM1Cm48B4e4z9COzzTZGkzGoaNmvThV9qj4YQO6c/keEfeya97etjuLG/z56FmIRtJQkuWj0261Onku/oes2t/4LwH+SFwFgA4p8I5NRlwkAcyR+5Qn+VVTCVJx7K1nqktC8igDHTLPCVo4T/NJxoQ4hQAqLG0vVleWbVNYDWAi7RF2w4Bu1gckyfLjDMDGWaOtwKoMjMDUBwHIAMojrBU7VE2WTwh+KeNnkdKdYU3eyDrqOXhHheXGKhYYduinkqPWrGtgKFbtnpubDcCLDavo96aR8JHi5J0q24Mt6HTOpJawzWZzcu45M7+Wz/jI0KwvMHZutaP6V1epd9cHnbit4cLk4eUh3v99LpsGiblkXTrNr/jtEfO01vSkKbur8ploJK0koX+OITqfn3Mve/xGN+dkMFvv/TVZtUeVqUpWid50GMQRyXW0wg9nQ94mUYfymYai87g2XXGgRkHHrMcuDqwQGPRgUVG4egj/TgKLPyA2xVGcEcwVKCOgKuowYsIbnSgXhENIhz0kTs7126eJOfsdBXUHGE2bp/5T/zltwRn+2nj05SnAwu7beOP4ICgIR0K69KVe4GGy2AihBdAUShXOC9Bza+Qgqm2zs6vCB0IIjViVsAmArtCAmfuDZs41NQM4AJBy1F2QU3ZbWsSxWg+AoF8UWA/jo37oiPEjA6X2U8Bs/WYP2UX6PXY8meORWkrYu4hSAqQKHrH4EL3AhPwNzwtPsm4fMBkLg/yUnChJuYwgOJhRknvY4TY8xAjvacQyJycvYYTmQ8es1wo1/U8bELoXo0Z1CVMaVYw6Xclk81XsRTtauZerCVf8/BZYLEFjctmTF6ci6CgaXn4wVW4VYh03oVL0roU7cKRhcy/OI7GQcFfr33VLzVVEdoBmKkD8DkgwazwrmvKYnpF/KVtKjt6tWke2Y8CUKBZk8JqjcSzzwFCovVHuhT4rMsK3ropBEbopAwFFQqeAgtNopwn40i9fgSrLqVqPRVYOMJ/jFHtw6xu9RBzLZyQbhIu+3kNK/PsYRfuG5jAvWPXnrZp205MogCPxC240BxIoOphnY+QUuSl/Kw3OQOCq4wjoOCnBOhBEIJ/5S5/6jRPZTLFKLgaC8rtDCP7zn05BcBdXGT1LoBeB3uadaldcmUqBwhcpnjf/kcCBDUHklZ54GIEMYfCn1qJ1H9AhIJu9ntQAwBQVmi3vQksTmiWBK8FboaX9hOYY/nO8M5Hsq3aDjWVE3QKSO2c1FTKA1u5wr7xyOD5eYFjTabWZKtrK+RlB6TWq2gIqN+dL17VGNm25tGI2VekrloGtBWFdtuVYESA5SkNhjHeLAoBqBBUCbZsfx5UKUNDZvUt1hXD6UaJJM+CikVoPQI/Dh5jDhLt8CiaClduY1GodtZypZNzkW1rcvV1XInzAlpBgYWT/lcSrytVXYOp3pPZq+I2wMV1aC7mpV0SQg83w1F1ox7KVYrGh65Tl+QDl7jCLw/7k/QpPCc+3HwziUkGcJTAzZtpdBP/8RBPPQUDVLh6V+EMOnY1XY+ebh6u8nMlf3Ejmp7yOI2xf80K+5F0x4T0F1zjjNe0WZ/8h84JP+IXx0Q3jXMURfKTeAa/hv2emSnUmEWz+xkHZhx4DHLgqsDCORZveJMai0GwJ4MRxvjY2gHbkU6EC4QGn9O50o/aWavIt0u1M7cDzfuBSd7jkCcFDT/gCg010lieEiY99rTTzggecXdwoSBqNAEtfICzQpMCAB/fTPh2DfdBKMgovx/8CA98hLmXMuMwXzEDGsCNH3XzEDOILA9aQp7rz+ckLQUeAYwCk5Mn+zyLrCyD4DiHMKBQbDp+Ydx9WyHXHX9Nz/jlgysrrXUlKEDF/JybtTHPgrAxuzC9pCV4QsiKkFnCj+lfBiyQporPxVvvLYMI7AgwCwiJDzFaei87Rt/F+RAjpocQkN3724meggUFto3wbTt820LZX2TE+zSTus8hkK48ewozqbOAjkvxswlhZyPC4kboV0uj0DnZiwPBy3xHSEOoOku+a9UoViQiLgGWE4g1fwlQhVYFRM1YFBY1K4pgZz7JRK3+xNK3bFTm6LgAIpPOEThdFcgzG7nBowADAME8IOEi2hfnDiwwB8KraQn+5Iv31jvv1XopEG/aMJclVBWCFSQFS5pC1cZ5pVnR3t/5Hk7gPo7Avki5HnEDQFZSOg24ULjbiTmUwOLG63a3a3dfz74WuwEWG1PWqwMs5HodY+Em5QcvLAcBcC9PWxPO1JsSOruWQx6nrvY6y3v96uZyqppunYWv8j/7WKC5WFxk4zvM0w4fOhgN0jHNociHiwgILpSvHt67LxOpT6GFUZB2tbJ5gFZ2U6deutKSo/9qqraSV+u9ZnEdkMk7I3LpYeu9tCiIqT1yJTDzpQmSGwp6fwYNxQIAjVymjdquxoMAfQ8MAch6gIXtSuAg8NF0STNK86yWy7kZTibXLC/1Alr1dwI63HMk5U3866kf9hMCnIABaBZEVBnAe9IwH4IEgYXAPv7w430/9B93gQr9i/2RXVf6ETxZj63TB9EA7aUeHqINLqKpOIOmwv0q1FR4pYlyJZxuPNjPXaA+X6IuUggFLsjfDsyerkdT8YxdW9vjmay9C36sodLZ6qu3HSgjfDLro/fDkcH0/rDkSuiRv/Er82gM47pqPUulxN33SWMIn9T4Sd0k/3mPvwKK5X8c1zitK93r1zh6POOw43vDdj/e+66/X+7u+/HR/ekWPspQ783b+DCPPW/dfeyF+yXl0P2MrmO6dPb5e1/+ipGPz+6t/V7PV7+OU9RNmuwjPWdzLMbcmd3PODDjwKNx4KrA4lU/yeTtN7IqVITV9KMRUHvnG8HaDy8fUgXs6pyqd02nSc+bj8vyLrYc8y4dl/4QQBLGDnt09M49ndwQn17ywRUQKAHZiXP6MV/Nx14TnQIWCgWlRcioI+718VdImAIL01VY6RoTR8wDdBCCnOwc8ycFbgUG8hkbeQQLQUxG2hE8HKGeF0xwLw0RRPAjQHAuhunWXhZlxnMWYd3O2rgdwY75E6DCjfPmEGTXCSwUXggXTUkABcJLrgr/ngU6LJ+YXOgGD1JeoZt7Pw7wUyFUjcWRAVjczYjpXyEE34swvB/NBVJMBOtoeuDZTkZzr0PIvAY79BWYQgmKzqJxWcHKUWucxM0eF/PwfB7hTfOLOfjiJoFuRtbNwwQY5tsyVMC7QDhHz884kRjh1knEJwAYCp1nB/MmNx1TA+DIv8vGlqkTOaCMFNYt19pbQD4pmFN25MtaBxtyWJaXkJzULqwnH27mpkBr3I56ewj4PAIs8F9lwU7SxFlaitIoCCY04dmEmVrMXyJMAhKh2ZW9jrjyFVqA2PkjsGo2pMmWpOxgMu0ehOYb9uxue667oW3beS2COPtiRFClnsCraltVTiHIn5Sd9U53y9nyrXL0ZcqT/NhmFFxzcPXWt7nmnTxnMr3gQl4KLNBKyPsTx5lMj1neMbRHC8x7cQUvV2KSHuNU+D7ABoDulq6Au4a6IMByY0G1EwrZmpLN8SzocqlTFy3QLFCTIcG8oLdoKS2DPLEeOVl8gfR8ltc72TPE+mx+nIzv/CLBgbt4u9KY4NF3i/C7wMXZpKsAb7nJ88wLAehJq3VObQQsy3v31Uj7hgddsyGfrOvuxJ55IYCmMqEsgFGrr1Vbj8kacVoOZCq0CjgC9NMOC+CXFlR/VSQpDQGDNNK+HgFU7GP53ocX0FxCk3OfslcFnLiAP0iH93WmT0LwcyDlAnXuEvVVcLFCQErenE9xy7aN7W/s2NJuQFuxDTOoVaDv1Bk5O1QLeZyH4bmehgcu3Vle5X5wqHAJPP0Z+enApPvr8ei5tBbeWRd5479XjqrTPVScrvqT/ouw4/BXCtDf+y5hxp6uEn4crgfpeVlOpfFODuPkIeETYAg1+FnidxJoyoeebr/GC9F/73fMgMWIXbPbGQdmHHgMcuCqwOLHX/Pq9ro33kG2qkOddKQ8K4Bk4iIfZ4UMBXX9VR9bnS69bj3LGF5U11s33tupeqrZmHgsTz3JxGnw3qc7mum/I/CCiyxLOwT2w+/uxyXwF6gorQUfXeicgoqi2ecIbAMdCh9qEjRziUkUz7UAagl+CviOXroqjsJzn2Pg6LdClRO4Na0It2CHQCRaC4UtVgQ6z/wEVzKKZoSreVegynwKhKk5gEX2sABU1PyKKbAIkFCo6YLNINDoXhqNEkD9oOWEj5N73BQWnWchsHiY0dJ70FbcCbC4j+vDCPKOSst6gdNGBMfHIXDdiLnFbkZ0EcvhB2Y/CKXnEQwvYVazGrCxznkYCLjryKub7q0DZLhPxjqESpeLHe/DobZBxmjK5Mh1116cBFicCsjANAR7f1cEUphVANQ8zVNwIa/kq/XMMpD/Ai/dViDJ6J6RZ8rIchRIKZQrBCv8KjRqiqIgakaNgygHv5hEkYYE6l9hV22FwqVgQkFXMCOQSxiEXScXa+JzDI3LEQTemOMASAVQlEjq4A6E7d0Ai+vYKO+aPde3rdt3Ep/7F1S51hyLoUyhP4RBk3n16OVnve73vvMUKHWNRXwPYbzvgCOaGITy8MN6DXhzgnRpLdBQABRPsGrXEcCFGxVqpudKTGoOnLS8qPYFAVitXQRpeLga8KhmQoDnyLYaKpdX3sKcko0xP9MEiknV0iGdlINaDQG+gFtTJ82gLGvp0l2Nh23ReS4C6k1oowR+zjQQiNpJmReXkj2eid2AWwCvbTcaC0Co2iNXHTvDKmj2OsaXhQQIF+Bp/YOe4l0J4PpxQEBgIXiUFt0CGMir+ahBhAL3FIKFkjZrGcav/jlTPiknyjOF5w9cQAI/Qx0+Ck/vP3SkPbjAfJxjrMoFimAPy5g9ZWftAVRIX6GL6osy34K0qIQFLLiqVbqOCdu3obF4OqZQ17G07GbAs+2AkgotKQAosH2Ejlynt/Ko+7GsBvaUL2mIW4Uux2Slbg06vKqLv/K2+xylq9/xC7zIq0/36HE8Wtj+3nj10+lLNnv6XFNOvh/c+nU5PWZlQmXPFw49/R6uX5P5Ub66v3G83a/Xfj99TyKEnwGLKUdmdzMOzDjw2OTAVYHFj73mVe0XX39HfX+GzjUdLgKbo3MZ3R6Ahc/pMOHDpOMcdd7e5nHorbsfR+TGnWxGu4ZwdrS9d+8fBB38aCgod3DhKJ/+FDRqsyuFFIXCMlPqQmgJDdCNICDtXnu8nR6FEEc9BRZnEXzOKcARuf4MP8eIoYKSgs9EY5FR+hKcIuwQxsNJ3vrPyD3mVGcZsa+zNCLG6ai6S7Y6kl0jtZoTlQlOmWeUABPNhWBiGbiQpoAL4grAkGXeDzT3/EmPIOwYwGKfcywAFPdwfQBtxV7MoRYQ+Nyh2zh2IbzcyIozNzuaPIcAT9izzKs4imB6DH/HARhIqW0d+ZlHcF3PfIs5hKp5PAoynKvhuRZBUg2GgqhAKSYl8pyyyzKoZwAQzNtQc+Hk4UOY5ggs1CxYDgF60CS4sPL0vDiXQvBn+ZNVaLbcS5sRszfSDQhEOIwGIoJjpa9/yxqyI/SmDil8I4Aa/0bAlIKuwCJiWgTIMpUJTdQL521oFuCqUDGDQqgNsMDEx2VrBZmbiWPn1q3shbGj7dp1Tdu2g832Nm+bCqvGG+G1AOGk/UhbKrQlRp2mPHq+M2KOm/RLryBJgJGmOUhSvR4bh5oKBchyoy5rLgawOAvdLi3rvBf3tnDpY+e+uBSyk6YXXXWLSfaZ7yQN0KmZkW3C+lFaQNoXZWkbsj1o/rQFrU4mbpN/tX9ZQY5y2oTpmitsCRw0IRNcC/LOSAf3AsdVbPi4geWLt8MzzaPUVpSJXOXTPDgnR/rUWtjeBQ7OkRGsCDhcJczVrGJCZ/3gneHwSv1z3kxpW7wP0Em7rfsMGNAnVJkMbY5n61HmUtDuArCsa5Sd9WyikQMUDqUAUdX++mpQtkP3itmPtuKuRw61h46irTkJL/tEbd6fh0bBoCcEi3apnNZvQJJgFnC7Ak3mJeoxajr2DlnXbgBY3M7ciqcCLK7FDGo9NKw0rIeVfDhy193jVu+mPqo+DSEr/R52FE/4PbhP/HZ/g4O87kfir8bWnVIWPlif+zEO0928fip+HtU/L8bAIjzVM/RdKd7UkRFNPd7QNs1ShR1IvyLd4/zqbxR2HKdhl6eZoHiaAYvOqaXXYycvtDsfPtMeOuQeRpfa47avaV90C321u7d+jo8/uvNk+9AnTrbv+ort/5+k/znO7iy5GQc+bQ5cFVj86Ktf1X7+9a+bfGzslO0rHbnNKkwR3vkw8/FV6PGdHWYEHoSefCxxrI6Ub+XQ0/rcj+mdgRMBFwPxT+ecfpx07Xj9SNdNjTgpvmeuheHw0IGFQlpMoRiBdEK1wKKPRiqodTOogKPQXp2TZEl7AQtGy/mwa6ogHeW34iqb7BqpVIDwnVf5E3ohRzMN03Tydp+ke54RfsdTpVs6HPUUWDiK7ch4ToGIoEUtB4JP2XYzOqt/BWavgos8Gwduphv30bXTw9X3Hsgy7SQj0IcY4d3HijQPACweRkjbi9B4AIFsEWFMUXQX6d+IECOwcPlKC+MUI9lHEEwPMVK/wKlgug4NzAbeCSzWw8INmJzPgwwVyVerReBcA10uA+x+AfJCYUw+KfRmCVSBBSPlxxg5P+QqRIxka9oi8HPk2HJV8LT0zbdXBUuXQlWwz1wVXDuwqHCWP1oNBMj1CJMbyIeb3m1mVF2eCxCcIKxWxCV2rQ8dbEYTJf8JD5NJTsHeKwLgICwLjBzBVw48jYB8HH66VLJCvtqa9UO6W1k+eCsb7G3asp1zG/MTNiaumAoNGhfbTcqP+KudWJnrKCHIemXyXqscexsTXET4nAaB8wWYorWgLmuXUv6py5S95XYeWs+hgdL8zF24TwAoXBLYVaIW4b8g4yhL6Lqho2koUMtnQZhLL/c0IT0CuyDMUX/3Y3HPCvf9cE6G1AowFOhtM7aTAA7i0jzoIu8EFq7w5QaZ1vfNLALg8r5qEATNtj1pEIzUDu2YQ1H2mhUa3lWfYl5HHLX3hoBPc0NNDWvHdudqWP9KO0Fbg5Y+p8Kr5ZHr4J42ZbuiXqRdpd3xnDzY7mtgRRBiO9RPPywj62n1CfLqUjsEvQ8sHG13PnIYgHECcI5JoG1N/GB5wSkHStIvymPLDdrd+8YduC/RJ6xgQjxMaavh8w7qsztrPxGNxW2sBrU9q0FRP4Z+tVeHqi2dsn7Fn7f+dI+mrdsQ3tslxyiiHqTeD3EZ2n/C9zq6JPyjPOj/0w2zJKoxMctpnDwPGQ2J0DcOM0R2JRo6bT09c5q8EZ3vPqVjQgO+hyDRmA6Bl/NpuoY1AABAAElEQVTKUpgBi8s5+xf3nmpvfd9R+qOLbSsfGbvBIycuIHesaP/di3e0m66dzlW7PPT/+y6/8gdH25/cdbL9T9+0q+3Y7LDb7JhxYMaBMQeuCiz+5U/8+ARYVKea7jUfUwWx2lTMj3IJvH6YoklwtFDBIR/J+nhwS7fpn0ceIqwt6Xt1zsemQ5AitTpgPgp+DelVfPbpIl+JxGtPU2/5oJeJVkYU+fALLPreBArpAR9dMCAP5iOj/sZJ+tIfYIEw45yECLVErwCREVo+9KUlkI5K1SviBXHXs8KioEKtRh+pVUg1VzXqieAhqBA4YCIVswqFlgjDHWCUcKvwsnqYPN5tusvufgQuiLsDDXkTYcdrhOKic+B6zDIWMXsSXOwHWOw/xWgqgthDCGJHtEEHfThp+waW87wBIWYTJicKOO53sYCgeQBh/gDAwtWc1lDGmyiDDchVm8AALk0bYAGAcknaAAver8XdTfUydwRgIZc0h3KlorNoP9ysTcFWO39t5RUks3keAnqAgIIWYRKOAnfk2n0snBPgRGCFXmuEZamWKmANujXFcm+LEng3tK1bt0WAVNjLXgkBJux8bj3Bn+VrDZW/7hGiwCmw8LDcpMs6rdDoqk7OlbCO1FKn51IvBVHONVGYFsxsYIlZ97CYZznetQArjw4ce33swK8LM1XfSZo0U9epk/3e8PqTlqJn2lbStoY6XKC+WlxG9q3LCOdqAZy34IRuQcbZ7JLuTtw138WlZ48xId3VotQEdP5G2wSfKJBMrlfjoAAsgE89B1wIMDLfAq2DV8Mo3DsIEeAN7ekXoF9aCxSgGYQOTa7ku+1lM5oPV0oTnFjyakpOQWfAJGXuylaadAks0CMgLDIYwL1xHEf75uR/QacA0v4gG1gSr2Zuxh9tpX1BQL2DDgUseh4FEGlP9BO5DnuPdGARMAhdgssqF8HFtL31eVC+O81yzvswM7ubDRQ/cfBIO7jonhvyv1Yjy+wgeGo+U/5yJsxRI4tGxnfwoQFKV6MFc/+O65isfRuT5Z+whdWgNrDYAHnJxG346mE8HqZ/+VFueVXeyv8Q5nL/U5fB+8QhMfWIeGm6V05zEmRy02n8dMJMAvebKxIk+3wx5L3Th0vSHMKMObOchjFtPSlaYqI0f77vfpbnt9LWa6UQ/6My9fvSj+VhdZ+tCtW5U9e7HjrT/v07D7Vrt66OhmDXlhLkjwIs3vmhxXb9zjXty564dHffpTF8dp7O8510cYjZMePAjAOXc+CTA4s7XsfHon+kStCJYMXHzJHAMilwdLlMoRS8YqYSwQehh+felXrtT/avdqw2Tc/eWdvxeqbjTrvNjx7SWSeMnTvnRV5F5S2BeVuaBQVET+nzdA6EtuEdQCjIKTD4PAEWCBEe0qUWRLvomEMhAFTaFXdGzXu6Q84MuQYBwLgiRIU3ZQIUYIFgoBBToKKnrfDiBFeEWWjxvYKsQqdmQ04aFnhkZBV/0iqPCzT0kVTDKQQXsIsQhB95NAUVllmyxo8TRGsi6QkEnsMIYAcAGO5r8RDg4tCZ8+04w6gbSetaBOM9ml2QF03CjiFIunrUQQS2g4wOn+J5FQKdS9JuoYPdhunKVq7r0F6sdq8LwIVmUWt5XgO4WAddawdAF/BGeIHFeYV7BVwmdGte5Iie+dC0JkvXIuT1+sCrmLc4cu2O0QqRJfzW0qTy3wndmuZkBJrn1AN4mSVo3awQN8vYdM4BbjRryryNlL+rUrGhHnVW7ZDLzOo/AgBMzDwDykl+W04CQVfwUWthnbc6uDGZE/hdJUsw4Y7btekhAi3uVthMTOa+x92Lp7eBXseTLvEvF2J8noAK7hNuKGLz5nNAve/842pDF0xkMvdwH2AB/wV3MUljrstRlwIGXCwC8gQbrhSlhmjHju1tIwK/QHdhmOsg6HCU3DpfK6E5H6XmKwgsUvdpdxAQEJL5J/DZfNmaNYvSrec/wjm8dUdpy09NQJaBxt8pgIQrellH1LR4kknSro0WyXbiO0mdkN6TmnwBhk276oDtqtpXtFrEL3AR7AboDe8ysZ4yt2yqb6Cdch9toVfrge2feL0xL7Y13/f2WW2UtoYX59/cz8aJd+0/kPkV7ltxhnZWwIIojBNa0p9ZhsZJvm2ranXOcX9+NfNy4Keat21M1L6dZYxdYtbVoDbTPp3fpIwTPsqI0GZkHBJ7hYMmWf68+N4wHqZfd0PdGR58Nb294p08WV5nl6Tf07hiaEmYhn8UL0v9jPKQdEYEWu/rGDmao55NXqYMuXaahwBx7+/G0Yz9Td73QEM85W4iVTcmr4eEO1W663ccp24zYCEX6pBXP/OrB9qBY+fb//rfXMsO8/WN7u+XX/W/F1Opg4vM/1u9ou3etqZtUY0+Oh44cLbtRMuwmvd/9cAZFlxZ0W69jhXXOPq7eT5cB46eb48cOd92b2ez001LtRLHT7O6ImncsGttvhUPHjyXOJebZZ3A30FovxF/vRn6bTlAvOZJ7Ys0roGW8WHagpatG1e1T2D+dYZ1qJ9wA5pWGzmHcyXN5+LJiyxpzkIrgK2lMYxjm93POPC558BVgcWPYQr1C8yx8INvxbUTrNFchXLNTfj48lFUMHMUL6CAhhOzCUYLXWa0d8Cp+L11EVc62HEvi4vCUMWRsX3Clr98JLjvcUiDLdWPsY7SVR98zZI8ESgV7qEtwgRCnCsD9Q++goBCQIQH/Hjt7ywCaXDTOEdlYw7i6CIf+RxDHkxbuiJkktZ6hVfSUVOiIKLwkvQRkjISikBaAkqBgeQB4jUtuQjNBSzQcGRjPCZvR7gSWBS4GAsuEBt6pdmzTDMqDx1QdH7IJ/nWP2DmTYFF4HQSwesY2osjCGlqLg5zfxyTFETpjIJuQSh0xS/X2T/CaPABwMc+zgP4Pw5PVhPXJt7vIq874O9mOr6VAIkVzMdYxUj1ehLWBNaVoxR+nNK7AncngisYagp1CeFUsw9qT+hMuVinPIdyUWhXQ5KVf5gTosAroLCeOSKuaY729FYGhVwFRSdse66LmVnxUf5bp6hm8Stv5JOdePiDewACby0P96vIBF7qk349LTdvi/e4k5428n4wyB4CNmZBTvZGCJxj1H0tSwevsSwDJKgDBO4j5OY1mQ49JWhImUe1NUvu8sM2ZX3spz6MwrzVWe/1lz9oC48AUkuBBSPnAXQKu7UM8sLCoXaYHdIXDh2C306ip5xoBzt37mxbMHGyXh4GeKgxcrlgl5w1HwI6gUS0g/DOjfTUZAg0qn+grpLZaGkoB/moOZD57KZJ8lR+qIUwTSfIO29C0yZN1iqPDlaUhsLM2t7LbNBaW8DQcCfRapwlDnkgfQ4sqMnq5kvRBFImtRkl2hbKOoA+Joj2Cb1tVRnTCUIY9SWppCrkLvXAOkEd8V5QYJoCopPwbh8mUPezeeLdB9h5Hg3hSdqZSyILaO1D3JdCMHOJ9Jyo7ZwKKl/O1Cvjw8+cc3aYV3E9560Aixs3zbdrmbTt3CblkpjnXV5V4tLbfhg48iOdk2N03/0Xv8tPd0sFmwSqmx5P9+Oz5+RZb+Stm2qNg/ew3a2H6c/jeMZu/b7yJI0yoVwrDtzqP+9IPkdyM+R1edrlY/id+Onhpvkx/iuFNYnEvyQiHwZCBhr664pjqeNsudnOnRL0/81vHGzPesL69i1fvnX64gp3+xbOtde9ayGgYi0NwnkYyiRf+6zN7TlPKo2GffQP/8Le9pJnbGofvudU24+A7/HPX7GH9t7y7mu+eHO7H/Dxl/cxj2s4Xvz0Te0rOfvxjg8ca+/+s+Ptn7x8d6rcv3jT/vbE69e173nR9u4l19f/zkL76IOn409QcN8jZ9sbfpeFMtC2WB+tYhsAMd/63K3tC24obbYB/0/yvGm9g2KtffBu5jNyvPIrt8fP3XvPtDe+5wjzHB0QW0E/A+h43Lr2Hc/fhtkvfdTsmHHg/wccuCqw+Fev+QlWhXp9PhB2pv2MYM6HUIFBodyPsN2jwMAR8dhPD6AiI5KG5X0JDtOOtHfOXvtp41f47c+BGOmXq3Oujwbx8cHNx4q4Y5rgNffQxbsI9goOnI4Q++zovodxCAiiYQiwGAQJ3AVRpESDZQUnhGlHSd0NuJvBqCWJ2YJ+OV39yP0rXBHHq+lJWwRk0utCU5k+lcCscCVHjIfssk49NEHHWoTg9UxgnRvARQcVJQxNR0SNPwIN8Xs1L3WlHBByOqgoXlV+e77lq/xVK+OuxQKMEwhgrhYlyFhU6IEmd+/NnhbQKYg4BC/cqftBtBvOyVgkjOMkW8jLdfDXZWnd8wLkwDKkTPJlYjfr2DDvYmXbgJ3GPHxdhRYDaRLeIqwiyDpivgKQoSkLfWh4Frt20pbffZ6Advmu+qMg60ZuLvWqjb31LnUQHlSpRfZLnXQ/CoFBND+UUQd1FE54JKDrYM063HllmjxEAFWIttxwCICRd1YOXk8P0jZO41+1ikm0aCg2uEEc9SFpq3mCP72MLKeAJsrbcvJIvKOrbtLTafJ5fAgoUo7D1XvIKvmFm95+ur/4hcfZ0wK+Ci6iwaAMLwDszmqSRlkI3g4fPtgOHHik7du7NyZnumtKtWP7dkzJMHFCwE05oDVyZSeFZPuDeczm5Ln1vbNHAFWaJ837MGuTr/AzGgOuslp+dKBlOAFQJsejRVFr4rK0ghvEOtq19VxfvY+gBRFHQAPCt2mXiVcBV9uw815sa+mjSN9Fi31WS2C5rCM/LpoQUASokJY+0GBavZ+pQQzCJXTVAfOdE7r0px/7QLV5Lj+8wCIEe5mr8uDCsexdcZz25TK+8mwFZUdCgAo0X/Qbl2hHboR3Ub6gobzEO+uofYVmdVsBqq4EdQNaihu57mTC9hbaHLLFElDReW916If5zWH9hcbhgXIdbr2MHyZ+dJbzHN1t8Bf3wc378lJx9+c48pO3+u1x9Re+wy3vfTfE1689nv7sNf6H9Iym+/F+7M+0PLwk2iHuOMa9vjH9+YpXAvekpHIc/zjdykCl0/13lkPBMhqnKZXfgWeD8ytfPltutnPog5841d703oX2MkDFlwAurnZ8BCBwL4Dg2V/A0teM9J9ix8mf/+3D7UFG9v/Rt10bobsDCzUE1+9Y077+S7YgoF9styOY224FHQ5U3MKcjZcCSBTUXw8QEGj8/W++JloJaRgDC/2Yzl1oFv7xd+zm+1/lKbD552/c154MYHj5C7bRH1xsP/Gf9rftaD++/XlbMe1aw6DDOUDCQjt8/EL7B8SvhsJDYPEw7+b5bn4bfqVXrYk12jhu3bMucayi8X8I4PHLf3CkPev29e2bn3118JXIZz8zDnwOOHBVYPEqNsi7401viEBgp5om45WzwAVXPoaevguoQFh1JLl/ADKSFv/lN/Hw7BE/w1VhKCOwXnPW+7qvzrk680o/8Zg2cZWWou6LrhI4C1QwcZQJyAEW0on/ot9wjAogSCjodYAkIPC9o51OyO3AwrkWyR+tO4AAP6uxvTYNzV4CLBBEndNheA9/FWhMx/hNy9F0JwjrR8HK+QuOxTpCvpalNjdswCYfQccJ3ZmDAW0lAFccCt7heb8qfMiH8ALaB/q70NppCT0DXXZQ8rLzViCoOc8pzKNOobHwufzAB+6PIgw5uft+gMU9mHLcj5AvEFlLfNsQzpyLsQcebyfvK9k4LxuhARicg7GBzs/N9jYCOjSdUmNxAY3GJYBBnUyyRQAUjFj+CpYCOfc6OE16ajaspO5roVmOAMMNz6TfHbKzPCzzQdYhZKld0ZbfdxHepQe+Oprss8Kjgv9agJv7BuAt5eC7Xo/NuPcKq67OpVilNuQ0gm5WGaLQLEsDK7RaJywnR743bN7K5OUtlOHGQUA1/aHcFFKhzbh7nfBeWvtpGXnor5/lMv3VbwcM/Vrhux/rVfnp77NyVNymoCJAmTrtnItsoAfPNUcTWOwFVNx3333cL0Rr4AfMfSq2Mvl9M0vBWrfUKLi8q8sUW+fcPVsNhXXR9I3fewGH76z35Cq8622hj/L7bB4EPO4zohZK0Ol+G64Wpmajg2sYn/w7ABB/0EHQ8NlydMnVvqCCZRMecPVQsDAuuqgAi7XQtp6ycj6HdUM6evtPuyGfNuIOKlJ+lqE58epJ3fLkIQs9uDjCAcDQflY3249W5yCTtRfQsrkam6ZZ52hjFwRj5ENTupWYYq0CfF6C9gvw63yuggtoWbuqbQGsXcM8isexb8XjMIPaw7LH2633tCc3tDQeD/kXmrlP71POS+7HwrH3Q9CE72FTl+IyxDPcexmiNLEl6S1JW56NDtOZhhtejGj1neE9loaMU7kvj3PwP0536pt48N/j1H0afJqC77ufcTzhw0Dw2M+YuMvi7/6nOR0SrRc9naJx8By6oJOIdel+vu/l31XePge/ttFe7v06Trbn0zbk+bneIM85FL/N6QRthf9P9/jgxwEmv7fQfuBrd7YbrymzJcGDJkg/9I270C7XQKPxdtBxE2ZL3//VO6IN0L3P8VCr8MzbCtwsBxZ/xuRytRNjP93t+/7mdrQZc+033n+svfcvjrcf+oZdmFdhEjsc96PF+D9+82B73lM2tpeiLfEQWDxw6Cx078ocksFre9ufVhw//O27l6xG9W/fhn/MsX7ku/aM6noPNbt+3nDg5L3tPb/yy+1d77+r7T/JtLs9X9Re8NVf1Z7/N25p4FQOLFDu/XB77zt+o73zww+106s2t8c95fntG77l69vTdiK3nl5oD7//l9r/9ab3t8Orrmlf+LIfaH/n+dd+VthzVWDx6p9+LTtvv7H6VPpku2U/sl7ro+qHtYRZO0c7qr5MaDrK4QOgXzupXPU/7enToU4EIfw7cpAO3bQ4FYq6AJzRZBwNn/iIK4KaH1jvTYOrwoFAQrMMhR2vzgUJrfgJbVydZKm7KwDpX8GkRjZZzYW8nFGgRIitlWwQCBRMoFHbac0WBDGCBCcmb0AIzeo3PIcO3heN0qR2RLMsQAVChEKWTDRf7m2h9sD5FgKL9e7MjOC7huU3FTwUqBR4ki+v/d789ntpSf6nwMIyGvN5fM+r+gTK74HHTlgXPBWAEvCQT/KrRmOBuRX7WSLzXkw5PoGQdDfC/SEESsxT2zb4dzOj84/D7GiXfHYAn1idmCuQYCw2k7cdyEFcitZCEyo388r4DH4uwYNLpO1I9Xk0HY5QL7IEak5Gxd1XweVONYVyeVcrLVnOspuu9OPym3OujEP6aqcsFz1Yzu5rINCVP04KdgK3gn8BC2sYAFEe814eqfFxlFjeO6JtHAIKl0bN8qfEZdoRKOW/dYjyci6FwEKNk3NDqr4VqChTQuOuMjGtcV1MfbRQOHo5ja/j9957TkDDIDRP/SjYA3goz5jb6DdhFA5snwrWw1V+kzeBhSsrqZ04cnShHXhkf3vg/gfaYZaclecK65o6bWTUfCvAwnlLaiKdIG37kIfOi1BjYF2Vh6ZpHnWbaC2sz4C/aAEIE7CAXzUiEzqob4qiqQvQlHkYxkW4izS8LuQwTJ9y9b1uMrc0kAOApx12Pl0gTv1Ynz1tN+58Pke99UybhM4c1gFvvNKv5AqN3lvWk3bm+/i1tjPPBj4ehVeH0FI8jPnTfjRr3p+g7TiB21WvpMN8aj4mD9RUrKburEBrQSVEW8Emmpzn5QnXbfD7elZ/ejyTtPewG7ybVm6mTmpauJq0bT9cJkevAzp1Z9hUbX3iq26s+b4ZcluOceFnCFxhl0bQw10xUoOOCepEJHYeKEcP6ez+4jK41/ueeLx+yj/jOJcEShmVS+dPf9+fHy1s3heBQxBo49/+3Xe+StjhbfzHS/lbnk49Fw+8lwc9ekvCY6axCBvy864/W2y/9YHF9t0s6/qFN01NhaY+Lr9zWVrnKBxHQ/AA2orf+8vj7b8FmGguZNsXWHw5plHf8KVblgR+tHcHievVv/xI+1qE/ucj/HssBxbOefgXb9jXbkbT8X2k5fGL7z7c7t13tgkCaOrt595+qBnXP/r2ywW5f/K6ve3xu9e1VwJCPAQWZ85daj/EqlPj4z8wif2BA+fai55edPR3H7v/TPvEvjPtH37rtZfNKel+ZtfHOgdOtA+9+efa2/fubl/6wue1Z9zU2l+969faex/e3J7xwq9uX/mUne3C3g+13/3t32t/fObx7ate+px244mPt/e/+3fan154Zvu+//45beO+O9t73/DOduHrXtG+fOVftLf8h/vas3/kle2LVh5r7/u5O3B/ZfvS3cxtneLtz5hpfucciHCu5fhA0XDp0k/+zE+1N7z5Tf07M/ng9A9SzBL8kHAqSCi01IffqKoDtbu0Aw2o4Op9Pipc48sO2rB8+Ceggue84ydCRt4rVODOfeIchLMIhMuARRcwFDRj541QVEJPje8lHmIuwadMKDJfBIGpm6xE6I+wjeClAJYTAYj0Q0ayXQDGnbU1g3KOhUCjRqSnQEfBVfCiaY1CjDbhHhdcfpZR2nPkfQXaD93nsMkXYKzmvmssNB+R1oyoEle/z3XgQwk9JfxMeJxUiv/D7RUv6idSbuYtAlgBi9JksBoUq9jsR1NxH8BCcHEvAtQRhCXGmdFErGo3INjvhsc74J+b5KG5zUiqJk6eSKHEK9BwXoZ7XtQStOvUYlg3iCefWN5pS6+AewJAcRQh9xirE2kv7xKjMYfi3rJQS7WRFXLUFKm1UGMhsHMSsUIoWcl8CYXfMwh31qONAIotm7ewStN6eG91RjgD9AWMyEcFRwQ6T4GFGiYrW5nm1IRhhXDrvRoly7IEVMxpAITzGzZFy2EdsA4GEEYotaUO9Xao/5ZdP6z/nmkXOPar7733XT+6X8up34/fS7D1OwAfXnY/AeiUQbXPQZDn/USgR0B3CVpX51o4fCimUM6jOI3JmnFQrPB2ddtMR6H5mTzp81ykLSZE5Mm67gnhod18usKXcy0EXJVvQYGanhLmbQOCxywhS74MK3AR0NkOeUx520fUYhDV1wj4u/ZKPkmXcztsRyk72NbzW3wgIvwp0Ft2bkbpBG7bl/3IlI/Srld+cJfmSfsa7nW3I7DtukP8YerlATYU3Htkse1FW7FAfV1ES+HqMdnEE38CCvkmTZlboZZEYAEfLkHDBerMeeJVc2F/9DhXftq+qT2ZORWCCjV/aiksC4V+D0nsx5R+3H2XOm7NW3pMnnu9Gkei1yHOAIvuB+dJOO+mD0sjH57CuxFtiXSISzrzvsepe+LrkRJwSdgrJrHEsee9xzt52fNGGuN0fd/DLHfvYft7PA5ORdcEWEj25N3gZ6C70zF9P4m134QHPbi9g0Fncywm7Gl91P/rMFl67pOXCihTX3WnIP4mzIqc07CF+QlOpD7NpOdDTLL+6wCLBcyUfvyt+9tXP3Nze+FTrwwspOAtv3ckZkk/DHBwQYkfwQzqizHf+sYBwLzmlx6hn1kRTcly2n+UORqaQf3ASxlW5ng0YPHTTGR3YvrNaF+WH5pLaXLVJ3gvfz97foxz4NLH22/89B3t7uu+tr3kRc9sTwC/nvzoO9pb33ugzT3pOe3rnn9LO/aBd7T/8v6728qnfHP79i9j3tClg+3uD/xOe+fbD7cv+Fvf1Z567oPt1//ve9rT/9nL25NO3tPe/po3t3V/+x+2Z9/1s+1nHnxh+3vf9JS2ZV4z/b/+cVVg8dqf/an2pre8eWkqSXXoRO1YeRtQQQ9ZH/HqInt/bgebE3/93gjT8fLOjtezh+3P+vFQILTzdQR9CjwIT+P1g18aBtLg3jh9ziowfKwFFlmtRmCBsKd/43fEVrp9FhREWzEIgmW2hNAgbabPzxRYADBwMA7fJz8DDcZTZ7chV0tRWhAFzVrpaRBkIlApAApYEI4QkIoewcUAPhRa0VoofDmymyvCRwRW3Yg7+Sd9r9PR1KJL3vXD9x7heXccXcN/84Vb7hWCeOhCk3MvDjLp9CFAxcOYJO1DYDqOH0ea18GHbdCzFUFoM0AhoEHBm9MYz5PHaH4YfW6YQAks1kPPBnjifg9zhHOJ1rVcBQuWuMKXy6C6K/eJ40djDrPIhOGjLNuZHZsFFqTlJnSOhjuXwvLNniXEY37Nh5XbpUedmOx9NFP40wTH+SVOrndVIEfiDSN/AiwoS4GFZRbhl7ic1+GkYJds1W3euTBoKdwtfQ5he47nNQJC82G5c1oXe7mYL4/UGdLp96lLPHh9tPKJ58HP2L/3tpvuVv4KWOjuCHn8wPMyhxLg1ZmN6QATBSyc1F335vM4+1kILk6QXxcwSJnAP0fJ5zE7yw7e8gNhOnmEdwG9XuFNQEbqZ9VF+aip0Tz8Mo8OQGhqaFuzDgkojKtrJ/Rjn+KzbVVuVX0vIOi9gMcREbVblp18L1Bv+TOW76g/cXgYX+iz/VD+gsFMpodW/fa+Q3+mtQIzOe8pPN4VWE854tbnXfjeif6LpO8+FQ+ipXgYUPEI54Jme5hEWcdQ8aboNQnV5C/8N2pM51ZxagpFpQuYEFC4CpR1cCMg7onbN7cv3LkFYLExAN55TytL2k++8iOdw1EtmAeyPQEevhv85H2xpEIM/Bk7mS9J9shk6yUvy73z1ae8HvxMKTHJ8dPl4fp745rEZzw97SuEr1iu/DuJY9nrno7O8n78vMzrZY/GOc5FZ4W1pKdnuxofU7INOYSQz6N89bAVrqdQ1xmwmHJz/8L59tpffaRpnvQ/DEL39O3SO+cZ/CmmT5pN3bK7BO+P3n+6/fy7Dn9OgMXHmWPx737rUOaDrAVAvJG5IX8Hml05ykONxSNHzrX/jXkY48Pv7D/+xb3tqTfPZQK27x4NWPzHdzJn5ODZ9o+ZND47/mvjwOn252/92fb2g7e2537lV7QvvmVFu/Odb2lv//hc+8LnfVV7ydM3tHvf+7b2hx871XZ9xXe2F9+mPpvNjz/2B+3d//kDbf4bf7C9aP1H27tf97ttw/f8rfbc9uF2x2v/uD3p7z+7fexf/1l78g+9oj1rO/sk9e7or8neqwKLn/rZn25veuuVgYVdZibX0mk6kugIpB23bnbeXeAYmwWF1mnPm0fjKWHHD4xxDJ9HO2OOpJMr7n6ncc8bP/rE1QVChQPTcqRPYKGGojQWCpw1+hyBk9CO5nr43M03IjA4Yp04jbcEVOM1zcyFIH/RrECAk7g9pEE/axBaFFwVbhRYjVeTEQXMcq9R0i7QlFDlyC3zGrBtV0hRHRvBDMEnQAShw5FVN5WLUEz8CrUFMqCPuGNWI93mf+CJH77xB3TCI2gdf9S6n/B04Kv3CnUBi9CjKdQiQpLCkuZQjwAwNIM6hXAo2HP0dCM0ueyl2otUTOsDYd21/DxnAArCo3MnLiLAboTWLYC9bRmFxVSG8OuGciMLfrZjGnU+k4oRIJnf4CZ6dborN6ZQ+MlSxwrvxEfWquy4cWSoRhVbBGMny1rRs5oQZa+t+0m0IoRMHdnCfhMuURuBGF7HDApQp8maWiP5bB1112qFXeuGE34FE+uwz1+nlmkyIm+9UQCuutnNnqq21G/nu9dxeYz99Hvfd/+9nHyXtmY52eZ4HqqjFbLapXWVvFZ5Dm0z/tWSAWjJx3h+Rfa4gK/uzi0Qc+UtAa9aKxNw3gw/OU86/4UycaM98yqvpbGEd8E0p21uqKsdIFtvi+5q174PsHA+BQK6GggcwmPrSpaWhU7TN+6YJlny1CnzYN4s+zK1sp2wbwZaQMs7WippIg21Epaj82ZsR7pZxk7grnZDmalZwj+/k/zIS93i7lUu8y+/rdfuQv8IWp0HDxeoOHScuRS0EzUYZ1m2OWDOroYACukx9yP8RfizirqzgnqzkvatWeV5zgue9COCil3Mq3jKji3tCwAVNzG3Yo5wMX0iujqI1H/CSJOHdPlLUkkv4MJ3+smbXMq7YfXowdU8To7R7RDp5JU3Qw88dRuimTrU3SRO3xNnT2/ijnMF7XQsj6HCjP1f7uNyl56ObyZhSSLu47xdHnTi0uPo3gcK89448944L2NQ+SzWdmAyhB7xeBJ/3HoqaixmcywmhcCNE6M/+sDpJfMX+vuFxQtZHUnh/Wd+7QD9+qX2D77lmv66vf/jJ9tb0SR8LjQWlvePvnl/u4alX53A7VK1Y1r+858ci1nW3/6anTGZ6kR2rczLnsMEdSZgezwasOgmWH/365bOvehxza6fzxxAFlv4y/bOt769feieQ+00Y8XnVt/UnvnCF7YXfdntaLRPtU+85zfbH991rl37N1/RXnSTvOD79LE/au952x+1FV/7P7dvvOFgu/cPf6X90rvva8dXrW/bngpAOfZb7c7b/177jmezqhhy07Qn+uvx8lMDFqPUcsuPDUkhIaAC4VPbfJeXtau141UQ6KO2zmXQLR1yfNSPnbICrJ1sQArx5dnuOn1xftJ157keTSGjagoaPd4ScNRAKDwUsHAEW+EzWgnMCBQiCJC0ZJsmMwKCcq882c/77JyILE9JPhKGVMlhBC0BACJW0V1ZKf+kHe0EAo5mUQIaeWD6mmcIEDqwMG55ZQEcd1lPhCsFuRXShNZCM43pClEKTTUqHwFJwYh4PUswGoBFeMz9wGsSmPCn86l/0Po7rx4pA8shZTuAC8EBZXoS4c7dgt1U7xDaiiOsw+8StGpvzMN66FgPz+eg3frgZnquGnUq9UGBH0EMofQkQMG9D+ahzwmouxj93orQt4kymid8wAW8j1BOGAqKOoYgjImOo+QK9k7m9llBt2jWtKRAg3VIegwfQZR6p2CcXY7hrWYzmafBHJFFBGfDW0bb2SF740ZWckIDUlqHEkQtswih5E/+nSdfCrQCSc16nLDtHhUuJxvTGtxKQIUMDstGYCHI7odpGpeHV5+vdix/73M/bX+eRGQB5hq2EaEAvYR4r3U/MYeCd4KKAlyuVARfqX9qDqyPAjev1RaHdks68t2dujVPO61JGn40h5vkR5BBfj0Fxpr+qU3SzA/iOIsW6RVk2FagNOm7AlvSJB1Bi+UsuBBsmDXBnPmOtgM6FZqNw7kdm5j3oUbEtm4eChixTCNhXHXJSfirB3DRBxIsVyf2R0OlP84MKkBn2g95mOQLev2TfksrJoKks4+NBO9ngvu9BxYwuWDVMgD3aedSwG/bgfSmShJGeintLCt7iXyvEFhgBrUS+sghJ1oa6usaAO4ulpF1OdknYwZ1M6Bi1xz9CH5YFyFUEF3okDEBFjpwSFv/7QlLtQ2w3vlQR3LTiSOentf+fmlCE1fiISYjS8SD+/LIB+fE2d/hX354jNOKS4+Lh8nt4NcwY/9D1Fe99HSWpEXcce8JXC0G0ixKr+xJenoa/Tr1WSHLve6XJ7k87spfxfnK7/zuaVSf5bv0ZfYdHFficc9n72c+08nbHz1wsT1p17QP/HSy5XyJf8vk5uOs8vS0W9hxHm2E/buTnl0RyWVgNVFydSWfv+nZW9rt7EtxJxvr/SYTpl2d6XMBLMyT4OF9HzlOX7WifQU0jZeoXWTux6swh3J/ja9mvsa1ABBXrHo7NLrXxg994zWEK848GrBwX4yfII45gIvL6O5hDwzz5zK2R4j/azDXmh2frxw41e58xx3tbXetbbc+9Wntide1ds+fvK997NSe9swXvLA957bVBSw+PgCLG+UDwOJOgMVvAiy+5n9p3/IELAVOsiTzwePZTuHcQ3/Q3vqx69o3P/VQ+63f/at29MLW9tSve3l78ZO2Y1GyvNf69Pjqt/xR51hMNBajNOyHeofjiKhCmx9/hRQFEb86vlegXu3KJXzASwAu4Sw++OnCjh1wP3VzFHzyZ3wGMFbuhy6Q+NEi8OBpWp4RJmmZ0RSQZo2WlvDdhfBoURDyYupA4/dZOhMfCRX9fodZajVgwFFXTWfo9HGTrgiqCA3noNV7hTVpVtiMsETazreYR2BwB+bQo3DjiXtGTAUtxGt8joAfP7nIRFg2KUOQMh3nV8wzEr6JXZvXY2rj3hYKrgq6E2DhqKfCEOmOTwWgCEhDLyVvfO/h/aMd9SHUE2VjGXB6VUA6w+jrSeZUHAVUaBZ1lHtBg5oIPMb8Sdtvoz9GPXgYe/mHMEE6gZAl2NgMT1yAIwAFAZbCz3yM6xA6XTZTYKG2Yz3XmEVRJgEXhAnN8FhtgxPCNZNSk1G7Rjuhmk31EHh93wXp1AfLFf6kThkGQVAzHzfXO8L8DU8/rGqVXEp1+7ZtbTPzL5zjEj5TRu6KLtAzPgXD8Eh+Erej4O6+rd+cPFu+ARawUd7FfCb1Yvhi4D7h86MVxDL3sX/vJ+covzK++wu/fObPvBdg9774cwEQlmVnabMCJTUVpzE78xR4ZUUvwiXPDggodMMLosS/czAwScNM6vgiH23qLi0x7VL/nf96drnd9QA1J2854OC7LsgYn7xbB7iEcDogd3KvidrOqzilNsS2wLskTAqmIn36i/kTtDl/yhWdBIXuWWJbPk9diD+ACSoJ5i+gocCfwEIwkQED8qTfgAvcMsfC3bWhu2su+qBFisPM55D/pQU7hpnYJ9j07p5HDrb72FH7BO3CdnKxkDn5LfPN0ijYBmnX1AVXfxJYgGKZtA3o4tl2JLBws7xtrP50E0vKPoEN8G5xWdm51VmyeSUJdyoGYkJLd+vln3c42h5gWtozifcgk2vCUc4eS8LynDqUN0t/uj+v3Y/X7j72nfdEbz280tHD+76DI2mKloXr8jiXP18pzu4WmoaHzh8fjePR4un0JJiFrH/P4d7n7ifxj9zLTzKrp4ScuBlOl5H/8lNuy+N85Xd+D74/N0dvj6bW6Rin3PPZ+47PFFh84tilduvmcUmMU/nk907E/rU/Oto0Nzp5lnrN4cpOLrH6AgR493Nw0vbr3l1Lw/p+F5vgOeFaU6jvf8mObIJnk/jhX3yYfS02tq/7kqVC+KO9c4fvH3vL/iw/+7wvrDkW7/zgYnvXhxfbP/3O3Wi8p21r7+Fz7Wd//UAK3OVpl2+s5/u3/v6R9jCAwrrlvAiXo/2mL2Pe32jGrKs8OWfkB1lBavmhedgvve8Im27aP9bbTdjFy4dPNg9leVyz58cQBxY/3H7p3727HX/y89sLvvyp7caNK9rJu3+v/fpv3dPO3/qs9pIX39wW34vG4s4BWNxs3k63/WgsfgdgsfqlAIsnDvm9xLfqyMfar//C77dNL31WW+A6940vabed+UB7x/u3t6//7ue3x29bz2DWZ35cFVi8lsnbb3wLk7f5KFb36G99KuwoFfI04ckSjtxHINUn3jOREmFRYXsySsiH08Pv2Xk+wnZsXRhJfLgvBRaDZ9MkvXRNRM5nuj5GtmmeFaQ7sBDIKCxO7bwVjurD3gGG6+F3UKHQbYwT2iMAlL126Ca+dLAhBWEamtUsaCJUKyiRD+hTKNDUYg5Bc4MCD4KTpljGUcKnZlKDEIqbAox5j305o8AZndXcBHoUWjWv2bB+EyOyggo3HpsCi6xalDwUsOjmNoKbgAoFQvli3uRXTllVnXvvkMjSkmPop1LC8jtlQmEJoM4iHLqh3knAwnGAxRmedTMPCj0eLk17kJHne1g56hPYnR9h0vQctLjRnvtdaBp1UsGRcwfP1yEM7mFVJ9fjd4dvl9B0h24FMM2Zsh+JZk0pIcsfmgAXjrZfAFC4Y/cFAINL1RatpVGwxxUsBhBattBVdZW0ETiddHwqE7pr0vEmhF9XippnroVltQohs0zhir9EoRTADzfSN5Sp2qcy+VEjVWcJpl1oKP6PNRbyyUN6r3ToPBRTXku7R/ffr7p735/7Vb+Ws7EHVOiHMhRkaHbnnIXSAgksSpg/dapMm8pUrdqxcawEWAlk5Yk0uVP6CVbrOsKStCeYh+HSsHA376yDAo3SDrE5IqBCfgowdOv0hk8K+2lXYkzaEeEmNFE3BDmaZNk/qFW0LClMypqyx5Qu/smXwFpN3sYNtBFG+n1OPJY35wroz07aA7AIKBcwWr84U8a02yx7S/y6pa2Qq8uABQx1EME27x4VB48dbx/f9wgf+MNtP/MrTp+FNnh8aQAWGYiQbts6adj3WHecpH1RfqJFKVOsVe0M8eIJ7dnadtPWje0WVoG6jdWgds6zlLWDM5bFpLpMbixqCnl8GfpInQd3y80CmoTqL3SeOE7rl1Xc/rAfeulPvY557X1J6ppxjuLqAQLGExF86REmviQydekJ6DL2OPjo6U4DFL0JRt68LgmWTJfvJVFDp3GN6e/hxv56Okk3HvypfjTv8DyhaZL48ph87m7eVroJP7zpPOxuXj8fgQVTC4blMMc5/czuHfkXpC/f5brH5r4U9nduMOfhik3jCc0upqCsMKoiPSh9y5XfGUc2UR18Wt1t6+N4eyTG4aH/RzvUMrivhfttXIkONZ4e0vloh+k4sVxg88l2JH+0OGbujyEOHHpf+/mf+7O24UUvbi/44lvbLqv3gT9tv/pLH2wH9zytvfilT2/t/W9rv/vhfW3Dl3xbe9nT3dPkaHvgz9/b3vmrd7frvucH21ffaH6RA04daR//7Tvae1a9pH3b0+9t//qnjrRv/acva7eu+NP2H//3j7Sn/I8va8+4lkFWvX+Gx1WBxU+y3GyAhbXfk/qufbOdqh9uR7P92HuN4DAQYVcfwZ2PooJ12TbzQVeYwI9C/HkE1Mwr4ENtfDZWYg7o4Mmkyt1GNqSZ6KEjphU2OiPjuQMLBVFNWzLPAaE+gjVu+rMT72Ciz4FQQEznbn6kgegMo1Yh8yN4r7Di4XuFsoAKBCE3wVJTk1WiSGAlwrOrC61HYNjIOS+wUPAkLtPx6vuYZSnM8CfPLABX41GIUhjpo+EuOSu40CxKwS5mVIkLYUlBBbq8KnjlCt0ZIQ9/iMdnTvPH/3CVYSnG4SePS37kQR3FEwtGsGdnlw31ABOCKkdnFbIcbXVE2o72NO6PMHJ7F7slf4wVch7G5Ejb8k3QuR1eSIdC1AmBBfy6EY3FDazstBkzNTUbDtrwn3kbrqyxVjfCriEfPqeftS4gIGdkHb5lQqzAInUQd8rID0vPt6JWr19eQ7PlBs0KlgEJAYDF1wASBEJ55+h2mcpAFWEFbgqhARRDmVq3EwdlY1033PgoOiyHsavRWeeL26mDvL7Sc3cb+zcm647H2L37zQt+BBNWautuaW4KWFiPBWTnAQZqfk5mh22EedzSjqUNvhUgVhPjGvK0WYT9kwCKxWPsb+EcDJ7ldzY4pKwU+HU7DWjL0sq0A7U4qdvEaT6NU35ZfwU61n+1SelXhjI0jsz1ELhQ6J2HcktgWeCo5lfYntwrw7ZlOwhLKT+BvmZQaiqiXeJeDVTaCnSkfXBNmep/KPNeFl77SLpxuoCDdce5OUcAzc6tuGf/wbbv8JG2QF0/FxMoy6O4bz2hQnCixSK/0uOGd2osLtqOBTvSSH1xg0z7ra3k49atm9pNAIvrN64L2HZJZ0JRGtNjfG/5etRl6k864q//cI2fTiBhLgMW+pUnQ5zTeCuuXr+8Tvhk+zLmURjDeRjdJExc6qeHzVPoG71cdtvDj527WwWt/m1J8uShH/3OMD2c105DD9f99XDJUng19RE//HSX+NUx/oaQ/WU89wf9+H5KQ7gTP0M43xLP973ie6cOn+U722XnQ7+Ok9RNmuwTPD9TjYVKhtHA/jiJ2f2MAzMOfCocOHNPe9u//0/t7q1f3J7/FV+CaeGlduADb2+/+r5DbcvTX9i+9gW3tzX3/EH7L7/z/nb3pi9t3/LSZ7SdJz6R5Wbft+9x7WXf/w3t9nkSusDiLA99oL2FhQae+y1f1W5d8yftp3/4I+25/+yV7RmX3tP+zU/ta8/7u1/fvvCaze3T3zlmmpFPCize8Gb2sUgHUx1fBxF2NF1g6B1170bp6vmAK5g5ulsj+X1E177UsGcZ9Y5gTucWk42hczYORabIRLgpJPJDByfs4PDD50dbKVMhjudsiMXVDdIytwEhw70r9JcO0zCcKx0F52Ounxo5LWVP7zyN3w+9gk8m7uJXAcT3WRkKIch1+91MTrOlACPCKDCsZsRUO++N2VeBFXAQHBR63J9CYcZTwOPEcgGQX90aSTZvfOhwcwlKzaBKoGPXZ54FY5pqSJOnwvCqUZzRXkinaRgHESev3Hute7le96abg5FVXtftwPt6mv6G334McerCqQBCoVwtRoR07gUcnm6udxBzKYHFR9gY7C7W8le74b4VW6Qfmi5Rbmcpy93sO3ELvLqZ0eYNAFCLM6PN1AdBwhqkmw3kybkbmkgJMmIehbuiu8JsTKMoiwueCKcgDt6UcGXezLsjyOag88S6ViDSOlplI/81wbExWL8DHqC3zM8KHFoPdK/R5yqLVSNQEWF5KINKt7gnN3327IfpedoOlr/r/n3v0a/9vj+Pr95f/jyEJZpJ+cFb22zmUMAzhXc3xXO1qzMuICBA888w/CT/gApBbSZMU+fPAEJcsStzMowDDZAgz70uLD+1GG5kqIbIo4CZ+a/6GMEeXlpXNWsSRHRNiQkbj+VqPC4vnAnoxKNJWo00lrAjO+X5eNU3yzNllHZD+6M9egZYdFBBGYXnxpk2UX3IpBwsK94JKpxUbV2xnbtZpnuoHEXb5YZ3h6njHVScYO7RuWhgFcCsf/Q1tlWA1UX6AQil3gO84AENOu/sg6zTa6HHfGwHYF+3eUO7lSVmr2GOxRbahwshWHtr2eZp/THvHqbTy13e8ViHBT4+hgA6W479sK/wSBz68d+rfW53z11FbRI5uj8eJukPr6Rj5M3HyUGwRDRymrwbXk2ex37GaUjfkueeaSLvaSUr/nCM/V7pOZ4SlhT5T/574ualR4rHvOM6fIkqaNIZAnjpYanHo4f49WccXyc9L02Lm89HYGF1sn+fHTMOzDjwmXLgdDuAWdN7//BjbR9LDrOWLDLL6rbt8U9rz3rWF7Xbrt3QVp5icvZf/En7/T+9qx04zwA5fcqK+d3tCV/6gva8p15TGgiW/D934kC7+5G17bbHb28rLx5uf8Em2L+9gHx6nl33bnxJe9mLv4BNWZFdP1NSCfcpAQs7wy5YRkuB0KBQ5Ok7O1y1BtzQV/LHrYKYwEJBwkmVtb48wj6JKtxkF1pMa/pqUulVh86ngIUChsLKkHb/IOpnABamq0AusBBUKLRniVk+2gILhZoyo6iIQ5PmNoPQ7wfdw3zUCjoKlWospLk0DQIk6ZBRLpEpsDgD3Wou/MBkxBNBplYFUlNReymoFRHEZCQU+tQsRFMSYAA90h76NflBwCW9VWgnBBZZlQhBqJtqBDAYjvi6YOu9ZyaYQ2/eQftyUyjzl3R6emGF6fvGw49nyOHe0vPjpzsHYfrHseqAMocajBKgBBfWB8tQDcYp+HKIVaM+wcZgH8NU5COcB9j/wom58wj4AZtoJ9Zj+nQjZjK3ce5mjsVa6NbM5AzgQL7KazfXYynyzN+Yg5fueRFBzPKTl4RxQisSLDt4DxoM5g/4bAbCH+sJeajyLdCROmvWeBf+Du8Vjh0JL17AR8HcAPI6aIiQjHsBP0FgAdReFpaTvO5HiQrF1e4uH8dnd+/X3p68Lj96uO7en82TR3+e3ldZxh1edT4ERAkKAApOiFdjIbBQgyC/pIWaXVoy2wF5Ng7BggBOYJJTbWW0Rmg6cNO0yonXJ1jBy0nygk0YGcF+NeVewBcnCDQFd1c/wSTwE2gA5JpzstxN3cM6YBwud6s2QBBbi0MYZfmbR+O1Xm1F2qrlWSDcnazV+M1hjrWWCfUCi7QT2+NQRsXvoQ2Semo+RFhmWjOoiVQz57yg48wZWoQWTze8O865iNsxAIagwt3Hrbdn5QdlsYY8px1D33loy/4UpO2md2vh5XrctpDPLfRRmxmQ2MC5DTBxDeBiJwsauF+FdbxKAWZAS+RUGTPUr7RM3XUbHalzy6tOyrPyZlweQ45zP/HeI9MBf5a5R3ee3E0d6v0Q/5Iw0mkc8VE/k2BDHvqr7ifL2+Loc+rgkH6nY4n/TluPK+npY6B5eF5KQcUbX8vCp43glnqRKAwp34vqTmMxgzc6+MPrSb6mySesj5Nj8DQkS5hJjPHik15e+YrvnQT5bN98rjQWPW+f7fzM4p9x4POaA+ePt33339f2HjjGHFe+l/Pb2u7rH9d272D/rKAABniPH277HnqwPXz4BJpwBrp37Gk33XQd35pH48yldvbAne3P7z7Uzlyca7tufVK7eQffzKuY8j1aTGP3qwKL1/z0T7bXg2ZKuC/hu4BFgYou0ESYzQe7BFv7dIUIBUmFLoX0tQiTmhjZ8SuIKqR3Uyh31J0chPWpgIyC6wAuekds7+vIH4lk7gRX0wqwAMDMk5ajmJoWmLaChH6NUzrVoARY8F6BMQKTAjI0mR+8xE8BixLYdS9GFaiIpgU3gYoaBVdyWaMAoyA6AIoaXbe062Plc8ygSL9MIGqksoRTBB/CemrXntVqEE78yElzRntNC7cIsaRRo+3DyDn5FMRF8E0YhBLS0zTKw3imZz3jENriIdwpP53N5a6fOvKZhYl17WCvQIb2ngpVp9BOHEZjcR/Lbt6FechH2CjsIUDGYcAFKo4SCBGmdiBA3c4k1ds2rGd1KPJMrOcQ6t29+ATATTMzbelX8RV2UvgG8ruBsvWcR+jKlfy6V4bgokZ0gXkIt54xzUHILGDpvgkFVi6h0ehCikJo6gbZU2AuEyriIMua2kUzRH0tsyeFa0FdjTZbzpre9PIO2KN8LKvia9VhBQ//4Gz4Lye7ANOvEWRw99pp61f99/fed/fx1brp8/iM3+En7kNY/UZjAZ81P4rGAmBxijk+7guhm/QHjJOf0urZjqqdJN1BMkq8xBdAJtgISDkZEymBxQkBJUDTXSjKFAjeABadj4OIndH44/oDVHjahtdStk7ClgbL7CwCvKDU+TAnWepYEysBqCBhIztRO7ci8ytoyw5epC2yTLA7oa910QOAhat3ZeK2ZaewruaEepP2UCUz3FefU0sjUw+pg25wp4biGHnx/qQTtMmT9fMUdJ1xrpFXgI9azIAf8mc/50phqzjPQ9c50naJWbUU7py9gzq/A0C9DeBh/d9EnjcDNDznrdeWAWXmmSPdY37Cm3Kr54kfHFM+FWLpr3WLPw/9VJhKY4h1mpgOozJOIH56mClRwxteGFPiIVynIfXW5x7B1Ps0D4NbDzv2+mjtYRLfQOOkfZDHTvc4noGwchr4av0KHwwzHD5P3Ia4O/XJnwlPvA/5HPwZn68uz0enlrdD2ApS7j267ktSPh81FuZrdsw4MOPAfz0cuCqwePVPvaa97o2vj8AdoQRBwknX/b53xB1YdBt/nwtUlFlRNBZ+/BEQDeNIdy0vqVmGpi/Dx46eNpMe8aO/CEEKQz73ntmysSPPWWAhI+EDgOmb4qk5CLDAn2HtvLMyC/4UXgIcBoHJ/Gjq5NXO3vh8HyGee90FE5pnaMIluPJLYb40mcoeCIySKmD2j5YUmwdP5JgSnPAb/46ocj/npGyXK0VQVSiq8KXliHAXquuDpRAsPeM5GwpYfTQ9o8G8F+z08vBzJ59086h7iPEf9+FTmPt6LidIHg79TA/z4uEvtaDKCDeBp+v2CyyOInztRRC7F0DxETYLuxdzqL2M7J6Gd0h0jEgzYRuB8InsHSGw2AzgNMYzCJKu/38UYHFYO3aulolAeyvC2E6EtB2Um2v8Z6I35SJKF3gIMLwi5eZcQfnUJGoFiBKmBQ4KpREBBp54b54s3wAL3/NO3oePCIjdHC1gEH5bToKNmLVx7SPhnedhbvhTZU8CPHV+Qx5pTfjIuwnf4wtO4Nbfj8MNr5dcim7ikvTlNAAAQABJREFUG+IUjI/DEvnkWfcpqCiNQ+ZEIDirZVBjoXlT0pdL1KXpSmSWUdWF4o88qmf56zyX84Q/fRJNBXMvTmX5ZM0FEbwRut0gUXMn01hJOVhW69eyyR1pa7dtu5K/gn7bnu0rfMKvNLrJ4TFM644uusgB2iz8bmHFpM2A040sx+rO6y5ra5tyoYO162pvkTlWXlJj0VeEKm3FCFhY1vmr3DngIbAVTBxEm3KQensQkCyosH7LG0F0BxcXcLOOmkcHStIvCJCo46sxwVpBOz9LfbnIqdncLsDEdYDq66n/Lim7lfq8ibo9z/u1aGusx1kIgSrTR+9T4DxTslX2oXmoBtAzPpaU/fCil1P3Nw2RSMt53MynHvJuHOckrvgfAvX7XvcGmvQ7DtvT9/po78b+J2nhP+5DfQsnBhrHZI/jXxKP4YeX+k+8Q1xjf1NfQ3o9wh7Y+j9JsLczXvp+SXwVYOKV1+Vl6jJJd1R+3c8MWHTGz64zDsw48FjlwFWBxat+8tXtdW+4Ix/5aCr44NdHv4Qxevx01AphdtgToZb7Ligo+GbkcxhRVAjyY6yAkNVi+Jjb0drtVpes0FrxKwhFc8H78tHZjG+EStO0T09aCiakoTbClZkEGJ0uQxm3I/5Oynayp6ZSCij27QoxMsJUFBBLwCmNgeGkU2ChqY7AyshqhFVNjCeCBPkUQRhfJscaJ6c0Sp90ZQlOgMQ85zrMnnRz2U3pNCM1Wj6MqiLcyhfTF2wF5ETbIZgYBF4E3dKQqMEobcakDHqZELb41K/GX99C3fO99Do+THRy9HclKIQeOSVtnArrfWK3Wovj7HFxAPOQBwEXdyMM3o0tuhqMBUab2XsbzdLqdi3mHrciXN2KKdQ2ywl397lw8vchBLv9mJns1USHZ6Ba2wmfrqfMXJ52MwKp+2asQ1W3FvsQl1vuAGM1zyuhxzBq8kL5QGfuh3wW7ZZjZfSyuS7ykvJ0zotXR+zlc5UB/O98H9wmPHc0nDQsNWO27pqG6fWj860/9+tyP929X4v05GISn5qWmrQ+pJNyIYQgaziSHu6Cvw4syozJpWYByvDZfR9yIlBbdwWAmubFjCigV46WCG7+pu3KdMgf9VzNxVkBSo8rk8IBBADLBSY6L6KdEMCsobDn+NnA6Pw5AEXSg3zjtB3WKmms+EV7CyfhnZoBtRrHqEf2HbanLSzJumXThraZcwN1wzbtnhbuQ5KFDwAX0SLaNgfgrvlbwIUC/FBW5sBysl9zwzt30d6vCR+gwk0hFwHKVXzknwZqvyVIVmtRq1S5Kh59GeGda7VCsAkdgoqL3J+njqwBDG8EbNzK3hQ3Qe+N0C6gmK6CJj2ecNnErC6VKDfeDz/4SZsdv4uH+ul1KH5G7v22v/e5amh/U6U7fRrd9bQNM61WBKj6oM/ywi906drjNr1PRsuS90ZRsYWA8buxe14mJZNM6uX0Gf5WHBUP2RqxPgQNFJmzfpTfoqlcJ1QM9BjPKKLwy9DlXCGX0D7w6vu+65V6+5wc9gedx/06Tlg3aewDGJ/p5O1xnLP7GQdmHPj858BVgcWPv+Yn2i+OgIUdTAcW6RTtDBEI/CjaCXkqIGQyNR//PlfBkX1HeRXMEgcCox/oAAfirA7YD1J9KApMqM1AYCEN0/KTNem87Z7toRFSAgQ0EVLoJh0nmmoG5eZ4XQCq0F3AX4Mggv21o4p89JMmNCiwGKVmEtJZnWqBBGnNalCafOCXHGc+R1agQkDQ9EEzC+OqFZTK1lqB2zTcoGv9ACic4D3PKOo6TTYUQghrmuQ8eS0+IqDAx66p8d54PEvQnQq8EXIRdEvohQ8pj2HE3TKJEGXZlNBb1/rOdZMQEk9+vVa5etcPafMYCwxSO5QLZSNPnNStaZvL0R5hhPcRgMUDCGd3o7lw+VlXizqNH03CXCHqekDCDQCM7dYN6DSvpykDgcVegMX92rDDd9PZA/9upsxuIMwmzGUc8V41gAjBhBO956kLmpHM8U7Q4cRXAYt1S56GL/CmPpR+LIkbulO3oCtX/Ko1WpmRc8p0ABYxVxv4nzLATy1ZPGiIwvMSVlOJSHZSb4kzPJUWjp5OPU1/u5/lV2lffvQ4LsFv21PyEW/lN/WJcPoT+AU4CQDhb1ZxQxAWXAgsziFMq6nwdIdz24ES7moEY0FvhHLyCxPDx2glyW8EdDKbOmSmScvlaPuEcCdlO4n7GKDi0MICAAO7UJa1BT5QNmXiVmCe5IyP+H0+6VwGwKhCjHHKb3nZJ3OrVdDsT43Fls0b21YmO2/E5Mm5Fi7JXHS7/CyawNBvG9PEkDbd82F6MNU0S9tWJniPsLeJoGLv0ePRvLkYgVaaaTMWAl2Vcz5KEwP/5Odw8ooRDtoyaWbSNvfnTY/6vRHNxC40FU9hKdlbWfHpOnfSJg/RUBCs92wTUGFc6ezy41OO1AVoli9XOnrduazO4N125DENOb3T8bIwPYGJN2Iw6cG9+xdsJW7jyNt67rT0aLp/n8fvxu4VeafU+EjQgjJMvcx9OdXvOK68/HR+4GPlp+dKPkhfj4T3o/uBouFlD0sAAsXb4DnlKHnTwBUxTuVcYZfT7rsZsBjYO7vMODDjwGOWA1cFFv8KYPELd7wuwkmEfYSYJcCCbNdIWwkdfiQUEiLs81F15L9s0RWKEYYBAIb3Y6zpQEZce2dMl+ut6VzAFr77s/P1o91X9wmn7f057YhNs2tHHGVVWFcr4TKlEwGLbl+aanI35geuse+KMYxmGlXSgK4en4KEdDgKqbYiE7aZpOocAAURgZMTsbPyU8AB2goC9zBObjYvCl9zgAg38drATrvhBXyQL4KRmDJB59LVbohoELQQ8YwE0KDWRyBUgEJbf4GE8wACOHwe7sN/wndBWvMvz/wFXCjEFd94DP/GH/flH7vwOz/hTj2SN/35MfW3AKdmItqaX2SU92xbABioebiXUeZ7TrBRS4AF2gQS3wB9Ljfr6WpRCljGdRp+LzAavA+h8kEmgS84Ekw6exztRbshsJgjLJIy9UOhGg5RV4SHG1jofysjw55OiHVkXKi3kjjllXVE3lh+0lsCeQnm3ltgvf4qgBbAoIwsX3hfWrcy5zOu8B8G9vpu2DEfk4Y84hwfSWtw0H9/H36O/Hb3cVjv42685kNgAc9j4gUPIWZankPcphfhF14lDP7dB8SG72Rrd7h2boVLybpClGWohizLodI+NAUzjxGuuQosknd51O+pe7Y1J3Z3LYgTxF316SSTs48fX2Tfi0WABTt+umcLAOYcmg2BqG0m4IQ0BT2nqDMLR44FlBiXgEHe6+cUGgQBkdqozQKLLZva9i3Drun6oy6tBESsVmux1l3UARo8235sixBshc9ZdZe9V6DzGGDmAKDiAZaO3Yf53mHqrNoH8+5AhXQ5gTwDDPDReRju2SKvMvih1pX4LwVUzGdJWUGFp4MdOwEVN0Lv03ZuZtO7+WwKKZiGEk7L0vqXwrWIc1R77U/T6/I609+M69647iQV6wse/e1aB/2bfo6lVXRSJ/NOfg1HD4uH9Cc65+0Q3vvKRlJaEo91qB9j+nTrlCQ8cedIxPnhseivF0Oace0ul18nIYe866O7dd/S0bPes+m1k6C/+IlD9Q+dz6OQRDzEXJmv6Icweejv680SvgxOkzAzYLGEI7OHGQdmHHgMcuCTA4vX30FHqOYAwZkP4BKBic6zAws713ysvPIBd0TXD30mbSMYdEHY8H6QTdhVXvKR6f3ykEZARQTH+kCZdvrsgcHe14fRUWI1DDWi34GFmhJXEMpHgPeXHN3ms9JXjSqNRdlkR/A2XuOB7vpwuMwk5jnQqTBxitFcl5t0iVnpX4XQsQ5hZZ6R9JgzITz40S1tRWlZjEeBZt2godBMQx4UANAkS80KNuEDb0qTICM4+a+cy0uF2BJuS1tRgEINyRhYdIARgZiPeAGLAVRAi4JVzD/gRIAGbknKi/fLjss+/vETwuLT72bKhBuFQ/mihkaTJkd6jwMkNH/ah+ZiLwLhIZ7Pwk8PJ6dugH6Xk52nrKAu4FGB7QR+FuD5Q4Rxs72TPKvhuBnB8QZOzUcsFyfXHkYo9upqRpsBFtfOrWnXMDrsnIx1lKVmUnOkMaeWQ6FTrkKj9Tkn6SmYK6xGuIO28AleCS6yOhdCqgKqWifLKnMrBmG7BO6qg/LQP/89upZA3nRe9mv5WPrru/H7fr/8mrjxawUtU6ghP5aP/ymniruXS7VZ8myYAAoBgJvjMY9FcyQmytse5YO8JcbsCF1axgFYUA8Dorh2rU0muFPHwwd4JtBLHJRJCePOq6CM2PvixOLRdvzYEYAMWhHakmDGjQqz43bSrSWoBRa2OWm2bWuyuA6w6GCBQKOv3ua8ig2Y023etJHN+DAtFARJi8DCvV/QWHi1PcgYN3Q8Q96sg9VOHcBw5ada9WkB7coR6upJtG3n2OxOTQVBoolzp++z1GkHGi6QT4HXBU7rf/VVaGvgCxUFbcVcu0Bdu6g/0t4A7dcx0fx2NCtP3r6p7WZuxUbM+VgtEKqMgcPy9MRlqD6592HynPd6rqPXi/7cy91YLD//J2H1FCfe9PqxJO6lfpfEnUi6Zx8q7qLXiEdU8zphh7Tqbf12+nzq8XtdluNpkKSb1KZuFXjp8/ip523sNuLbQD1vwyEJMTccUJH0hkTjVj9Fq748p9T2kEPAKaEV4eC/HrrTlXhQqRCz9Ql6ZnMsOkdm1xkHZhx4rHLgqsDix5lj8QuYQtnhOdJbVwWUAgR2mI5D9Q4zV4VzPqoTYIEwlhVb+Mj6XmHHEdQAiyHOzrwef32sFdBJh5e6+wnyl9u4CRSMzzMCOx9zRwczUVyNgB96vxOcnc4AC+hxmcrYY3NfI/oI4oAFl7vMyBr3piMd5xByTjC6ejKna/bXqHuABaBBkyq1CTKC7CCQFG+MRwEsk0kRemKa5SjoAILUZKxDaBVcxJ4f/+FfiDauynuWRQ2wgFYEpwIMNaIe8y/cStDj6nv4XAJgAYjKn24KwKZR6SQteAP7hnR7KUyv8t2j6Mrd8AG1HGRuPaZ88BtwwVUTEZfqdA8LzaLcgfsoAmx26ua9Av4a6NCkSbrkG5uRIrjV7t3HCf+I5lQIoIsIdPPwZg+AQc3FJgVMBFjjewhhcL9LgOJvA+BxD2v/7wFcbMbPesLMUQ+2wPvNuK0HXLjKFAVYIAL+ZkK3gibCpQVePKn6G4AG37XTX7tW0xrBhUCQ8qZcO58N41njscWT8IX4Up9Nx3Tl1XDt91O+1rvl7ztfJ+49HiOw7YzaoSUxERpNm/epQ9wH3PgcwZ+2B7+62ZLai77UbjQbQ72jYFIvFdYFw32lsklbw700Z9YzNRa2GeqswvfQvrOcLeZRJzGDOnZ0oS0cOlAb6yGo68cdts849wb+n6auOHfBlZ/UzKlR3LRxQ03MZj7GWsrSPiPaFa7uieKgRW2OV22wNHq0EwEFAOMSdAkQqi6iPSGvp9SShDeWT7XXrEbmO2i4gMZNTdAZrq705FKypwFHLp0b/kKbm9wJPH0mqgAUgcUl+wFAqO8F/tK8Ddpv3FjA4hbmVvisqZ7mMjGZqcpQDdF7jqpF07oUAFJVKO8n9SFPBiC+4Z5cpb+rnx6XtcPDd4NPLj2MLyf3+pIx/Zi8sJ77cohr5KenPqS8NPwQz/K6rrPp9LCTiHteSGtExRBLAo3uh9tOYwicvs7dEjqNs/58N3015G0JFyqeKS+Kmp7UhLaBJ5VWhZlSXr4mfnnd+TCNt8LortsMWHQezq4zDsw48FjlwFWBxate+5r2i29gVSg6PAUGBUc7v3SKudYHqfetaioUSLwqgDhR16VfS/BF9OK1H/USPBDwhrjsePt9hFPTUkjx/fB58RMUOvCMcz5IjmialkJgH81fgwApgPDMd4I09S9p7uDsvAjBQDQFCgjEobCueUvmgnA1Pt0VGrJSDKYbJxBiXXPfkdiJKRTCzzoFTvLoBl4KGqGYBEu41xSsBNGYz5BOaXHWIjCtz6pQfUUoBTO/3NMPjxoAJG3dFO4UVlgGVWAgP2Oi4/X/Ye9NoDzPqjrPl2tkRGRERq61Q0GBFiDajAgCAoJiK2q3DWMjM664TJ8Z7dPtzOk5M3Om2zO2ChTgcs6ctvuM2mCxlK2o0G2piMgqhUAhyA42FLXnGpmRERlLZs73873vvt/7/zMyqqAptfD/In7/t9173333vd/vd+/vbbqs/FUFsI1UUA/R5Eol2KNLlINg+NeFo8waJOZ/0pF/8kPczvLXT1WlSQMufbWa24k29k5RUgI53+KcDARGJKiTlarAsOK3JkFzXbiwrayL1jkpcSel/B6Xosk6C/oMayv2y2BgNILyMDg+p/b4vC6Mix1Smo9o/tMVgtkrA3Gv5ME5AYf1NZuzAThwTMtppQXqrIY6QsGXer6wX5CP0WVjQgoigomRIsnXIxYyLtTWtKUvy3yQb8pPtk1zBOnD9Fn6dLqUVcbTJ73PIwxejyuABDcsxoPTxO9wnygJPMkQQ4L7JmnTJh6dwLCoxgXToTydSjjedAA+VIr7HP1H/aj1LynVNixqf/QIQe1j9F/6kMsSLUYXGK3AeFiSYXHyxLFy3733lJMnT5QV7biUI0UuU3xZgVdbs1vcnHYMO3zoULnumqvLgYUFfwjYLuGuYoiwJkRGPvWgLGTvZ43bZo8WncugkEF4Xu1Iv2Na3qJGIhY9uiX6sjR8Hwhvp9qcbap5OLDCyrs7ybhYO8dWs6s6EI8RDBlBkiPPte16dujG8wLtbeoD1JlD9GwUa/viixgd4oN7ckr9dU4jE9dohOIRGrF4tKZCHdKuUKwDYn3QNtquurz3hpR6C7YEBVpYSIkwBJrsIQlogou9ETd0oe7eNkJijICPRPJZgNwv5y6Xl7jgjcM4bprD848KJVyPG5WL8slveVS01pW0xDVMMtto5r0WCI2G4SoR80lCX9cIB30D158KYy/CeT8CAMUKUeGrR51rAxlG8b8PU6Hef/Qz2tDj/lFZTGITCXyZJIC+8HfBXTdzsDxx/rq/C6x80Tyw0RC645fqtjQsbvqlV2m7Wc6xCAWpfSHVA5AHdj60m0KFcqGXNC9cr6/QixZFHoUdBYUnLAoP6xY8YpF0xH3SQ/kJAwIFNRQy5wmZqQt0GnlWDnjAMwIQyk8o0HkwHQs8KY8uxmXDQjxM8dKXwsm8bRQSHuypcHv/efEayrqUeP3Bi40KbaG5JCUW3oHna+QeDAumyUiZ8B75NAQF6Z+vvFZ4JAOMiZyXz1dVjJoZ9toX7i59WTWulBxw8yXHgABy16tT9KjLwGdsA8ri2linQTl8NQ5DQnDiI6apRDptQRq0fZmmSbZ+o1Lk4jc9p1QcwrQDwkSesqL4jTSHIkwecLSdR3zU3j6Z24orym70GxoR2bKGgpEMDthbk9K3qvwV9YElKYWLUvLOoKAqf6dkM6M23Ud/Ul1WRe8uKX5/rV187tS1LCV2RizNCWaPDIt5yXy/2uURWk9zlRS7I1Lq9mpF9y4ZF9u1cPu8FOsNfU3nCzjnW+ys081oH77EeyoUCrTKY56+T2+mbLcfxtylMm1foJFBk0PIIvt3tm8VWfPc1pIFcMBkHB+XsldWONqB/iF42sv9JFrGMgbPU6WURthGBdOdVF+U/nXJa8OH22G48RWftkljXkXU/hj3luqtvur1CpIrdQ+jIgxw7vfWv1QePLGGylvMyghY0vqFYxqtuPOuu8rx48e83uIixpz6CLcLxrAIeJ0CleGhtrBvX7ni8OGyf+GA48jEdYBXGYPL3tpWO03pvoQf2g0eMSjOSSRLuk9PaTodh9mtqM7b1L4e/VP+8AzSc0iwMjXlY1iEYbOq0TKvq8LIUFn5PMNowbBQYR6V8AcM3Vs8C5DXBWirzzBta06G7H6NlDEN6ioZtmw1y/Q/bypAm3Gprtmc0aj8kl4dYAQNG2l9/wkKFbZ6yL5RqJ0FszNdCykQ2cGD+1ECPYD/YGDh87Jwla+sV8KZk04gmd6z0+ov/vt8p1fcvmzDUFcRYUpsuPSJOaf6yISRt8gPn/4cWJwKnzw6rSNDsJczz7/Ga6D7N2lnEjDmoBby98Gw+Ml3/mq5+dNvTxFM/IkEviIl8MKrn1L+9eNe8LCs21VXXWW99ktlfkvD4hW//Ivltbe8YVTp0AMwFH8pPHqa5rOahyPKQRoW/jIvRawZFigO+uOLP/O48aETtEIZIYzywMMXlRqfB7Z9BcKwiDgPZBR8DIu4YoEuCnwo8SjjIRZoMOWGr5MYHjmdBYUeIN5zfLGOXaWGefQoFJTNXHCMiyUpMRgWwKNUMFrB3G4MC6YloYxayYInKRmxLSnKaBgZKPsYLRyUx85Q8DG6ODaMAhegMjC805CS2ucXFUoUU6jYCtT4ivdGBS/GZmypPJRkH55H24jxvJAM9UgXwZrQvIBvcpQscOEJiPaJJMspadFqbkNlwr/ntOtF67pUwwIY2pvdpMKwYLtZTT/RhWHBVrNnMCz0BZt1FsCzyHtOsmNHHdTtY5qmwojFf9XOU8ds9GmKjOCoL4fqHZR8H6V5+I+e1xaf2omHQ8lmpAPukCGxIYWXL98o2EzhoR12aQpNjATJ6FTfJY2RojA6wjik/WIqFPKsMqUPiZ8qNosBsYzKIZR25L+Zs1KPYl8zLb9U9BFyzeixbVjUjPFpUBh1HolQ3binmPLEKAUjFBgW+P7qz+iNYOIeg2M5ilOZ0MSw8KiN+htrFtLojj5XDQv6VfYv0GlXGTFxdoW2bNWIxdFjR8sXdCLo8eM64VMLuGnL3aLt8xt0f2Doe02H20FT2WRczM3N6ZyKefd3DBfzZPYuVsPirA0L7m2MBp4/a+LdRoUMipPLumfls9sY2zt79zaNLGxotCsPuVvBoFVnwrhgS+l19TmmZCEvr+eQwN0XuMd1YVSoU2iqFdPsdvksigUZDkylRF6svWDaI+etLGjaEwYFo2yMoo0aFdlf+hYlTXKvbS2Wos41ob93a0vVngekHO1mfAXEhx1tkwSV1EgDEhABmzCZdjlfcI2GYGiToKPfSjCKdk26AoENoskaMfDTjd8bfV4PE3db4CZMwzUbWTayDPqw5lDlscaSbPVFWcwFCrgkdzRMO0BrliLA1zKgXelzPzeeAqXBDfBRnrMrjYlhUYU18SYSeJhLYGJYrGod5OxIK+pddPHiK3/ll2xY8CD0F035hAdFRM9RxXnQ8xD1hYKBEq8XMLsgeSqUFD2UMCuSfDXVi56Xtg2FRlN09TC2QtSn6cGOEomSHadwDwoaygZKJPOtueJLqpR4hf0FFYWDF4Ue9izMxuBgehYjBnzFRwHPNwFwHmWRMgrPOQohVjSdQ4u3pYSeZTcb8Y7jq7mnxQg+lCKUjfqlW35uA4scMEKCnzB6gPeibxRX5e2QskN+TMeJr7fIkLItJ5XJV2WUNxTdqTbaEYpv4OZIRdTdU6cwLDBmqjyCB7Pf2itiKQXFeEdKFnbyaNt0aqEMmjei8EifGHfuMwLIfL7gOU0JCU+b0icY0fDJ3VL4MCiYB4/Ch1HBdZY+I1pMQeMMCxZkY1SuCI9RCw7j+9xpTbeRAciaGAnKi/f3S86P1hfjJxzYVx53YL5cqznuezVFBeNjXW2KgsvUGmTrL/NuD7UXI0n1QpH2RTupv9C3aDPLUjzQb9Jp/fgQV3LIAOOwU9w7WSWu4cZgSMsrjfcsJ/wqR+SvhDqARKGBJ3oYwUzjQUnGiOJAOnaBYu1DGBXqy8rPJu75uag+B1223N09M+vF0MgkRyqi/t1IRScH+OMe50wKto09rYXb98uwuOvuu8ri4inxeKHs03SnvTog0c8HdXnuY48G6d6zoVBlSxpl2shXnXgW0L5eZF77lMtTns/AULlL6jOs62G3p7Mqn+1hL1zk+YNBus3npbDBwLL61VlGKTRShoGLvOHbu20pz0WprXcyLKyPCJxRwWF39JUpPT+uk7H6CBmtnE2BoYRTS/sDxpTqw8jZDPcrzx5lp4KfvhEQfu0T3GuIkajb3i0QUPlLG5GXLtvM6dxQwjFu3x4J36fB7kAmu0CStT9STpfToXWpEWxFEDA7AW2eKmIPQzqs2LWMiPblVwjfX1nnoBn1JY2/5mqdlTvUkxGHWgbpKRYAIr3DN6GgnTQzt+fL3CchATT61CvLUngEp8KTnzDJzMSwSGlP/IkEHt4SmBgWWxgWr/utW+JFVR+OPCBRBvOhHoYFHUAPSSvH+KGk56hFKvAol1Z2eHlXJYFnrKj54W/apPOSxxe8jQrlpxIa5VJcjFiEsTAYFiic5sPK9DACAA8o2Fb09bIPIwTDAgdcGhYYQ0xvYkpMHKDn+eJSxpge4TUCKoNRCIyDWDAqZaNO94rpFlW5l1JPOT7zAH7gQWXbsJBBAm6MsIQhYgNDMH7ZiB/K4ssvF/Lw12PhsZDY289qegc0UfK8wNZlRJmh/IYC3PIsm1pjwrpc+/SJ1Jc/8k3nkH5oq2ioAIt4pic0cbeoYd2++ok+Q5tGAfxGHgaj6qmLkQsOyeNifjyLv5ekFGNobKgv4Fj0zVadtDNfp09IibxXCuQdS0s+4IyFuhgfC1qgz+nG12qNxSN0QvN12ubzoLb9RMljCo6/2vMF31OhRFt06bdSpa3Ixg5j0Ua0dU4nC8MiDOVRxYBRLzjUj33CYThlv46+G/WP3PjNdNZMGLbWNdOjTVBhOsJCjfsGHIVrlkcxlHARwx2jQn2HMyU4nG5Fozor8jkB2yM1TPMRLPcE/TKMBdFSW5yX3G3I6l7YM7uv7NK0P0Yscl0FHwoMrz4X/Sh4h5moA0o6/VcjTjq/YvH0oqdBLWvbWcqclVExq2lqHu2j39Iu7sf68i++sx7uhqYZtFgTEyMJ4HDfaYqT7gNGC+hbjFCcUj2Pa/rVkqY0Lasf0Z881U7znlj3w+5Q9JPYJUpGpvDy+cJzycaFeGdXpx3qR9vF5zZGK2TgqFDvMsYi7Bv3z5dHa7en67R+wgcywqf+eKrQF+inGMPkqVZubBuJgrvE1aSob+TSpshyK5f3sGEESvnuECJkzMSnf5CmRIKtKxknok43ofrjvApQ+TekAOmLw9QiMAUn1/qoklw0P/xHJMDAd0UjasT6k+kgmyJ4FTbhEqaXDeFWdgK6TEXMbyRWqk7LbHW2Doi6pDMHEWnAmTfKe2J1GAOgQuY1adT6gDMO/5Lv/+ERvIcyQj9PWabfl0cafPtdrfviy3VA3is/9Lvlj+68vS9qEp5I4CtOAt986PHlJdd/88OyXoe0xhGd4Et16B08LzYdsXjFL/9SeT2GBdT1gGl/PGx4SPI2luNlyUOoXXqzWolGqWbaQGhcNhj40uiFpXqYV2zT8I8S+gdZGBbdS988BDgvkVREdunLoNc0VCUHBTHz8IcpU/CjUQsJDMU7FEmUgICPEYs6FUoKCzRxVnyl0Hi0QjxAk1EPlCKmbzAFSsASQl68t5AHtCtvynfZ4GEcyOdrrI0LwszxR1Yg+WLhOIohW4HGQlUUKb4aY1h4mo7Kdf3Ejw0Y8JUWRgWKl8LUX37KwS9ZF1HbiwqqPL9YaBDVz2HSe9e9BWkj/VPJ2jcC0O3pn2hHUt1nlBY40YMinTTyIw9jEuOBkQsrffWLMgYG8XXl2eAUAuKhcMaOWOh9RnI6IYWS3aGYzoLxsE8L9A9IKWTh9hFdB7Sr1F4pg/r2rQJFi36IAuu6RF+kOxMPw5S+IngZjO4vjHBZluorqVT3coMj84VfA0qjftwrHrXQyxn6OJcjuCjfBRsu44aDtwA2Dn0qXaXiaOBQlv4oQ5eVAcmG0QkWUa+w+YBOv17SwumzUvTZ9pWzJmQ6e9oRU/MweKFi2QiPBdy7dAbL9N4F747FGgMbFn39JZOob9YRP0b14BE+mBJ1TqNDlL+mUZNtkr9H+9SXbdBQrSpLgvDFOgou+v55rQXxKAthrY/x6N02jTthUOzYXTa0qYHGnDwawY5hjFKc1rqbkzqY8bTWWTAqgXHBlrFrdd0EGwm4vX3PyihR+cQZFcV44ZR4CaRsV/23y6i6oH7A2RZMpzyovnWtdnp6/EHOpZi1AauWtMEURKl5tJAauLUaMJkM+RFXwUnLVm7GYu0zI/A1MtLXEg6/9kH3IxMVVf07y3H9JIsKukzQyOuce6DhKrDz4nkRKfyCFbERfoClQF14hqngwZ4lEulk20UvD3D/ivw4VwmbftAfoCo/4+jQgRfQ6jCg+YqfrIGJXlJkVCALTNEFbEt1DbvYEMz7uaWogMplSyLw98GwGKnwJDKRwEQCX3ES2NKwYPH2a2+5JRSH+tTmsZy70fCwrM9nw/ilIjj8VO5RlIkDi4Llr41SLOKFF0T5jRcSLxro89LnChxFPW2qvp38onEZohsKNYo1CnwoOX2evzAr3SMWVelGwff6CnhT2WF4xFdpFJ0YRQgFPebRxwsDnoHFAPDaCiv4UjhE/7wI8d1LIKHcUQ/VQcxaqWfUAoPC+/LbsKhbZEq5isPvVC4GgOhLGMalcVhsm4aF1wGwSxHlioccmUmjAt5tVKTBURVAGxYKWy4xX8dht02VfUhCkRqP0PAb7RVxwvWdbV4TyjA1w3LgR24ct8GTVykBSpuj8HFwWk6P4gAzp/H1G0VZ+TGFDiriWu2OMYUsOKNgHYVUX9tRkadkcO6V8Ten0adZXUxfCVMRvtR3aX9dqfjj0/egy4Ux5v4gBRO55jknaaQlPr6d8DOcPiUlXcuNMuoFTsom02yAKF8ZJpkwxIKrUMaAdxkkKmxDQrx7m1n5YViwZkCGhRRtRioWNV1sUdOSuDA06Ft8Td83N1/mNReSLV5FwAr9eRkDPmhuZm+ZnltQGCMgpiWloUr5wYOIdDykYeH2EQz3gUdP2MlJbYSMYTtcyCPpUC8MC3iDh3MygnzJMIE3L6DlQ4WMibUL27X+Zls5JWNhWaMR7OzE9rPcGxihdx8/VY7q5O+T2hWK0TDfT+ob+HQ7+g6L8hmdYi0ShkWMnGnUTG2/obyLGBbkKa5Ny7zd8SNlVNy4f2+5cWGuHNGomM9iEd8q1HXLtss2pZ7Ul7jD2V8c634ie0ioQjJe5tW0pJM0iScI5VuepPVhgKojP/p64DWyjYgAW6LCTs9MMrgyrmC6Hoe0WufqDSRTsTfIQAe++OOeScf9lvVMn7zRogb4kRwlR98KP/HzuQOdkEPg91RcRm0ry5EEuaThiPl1yD+J38OTMYJTwd0GXT3F6MSwqLKZeBMJTCTw8JXAlobFy1/1Sq+x8MPe75JQavyy0AORh6hHK6g/D+D6pMeoAMeKa33wAu1FvHrBM2KRD9pAAd5ETKNNCZGqzsuelwxfEtOZnzYqgQEQRgUvoGAjymfeNoaF11+gVNqgCAOC9GQ4ecXYIB06aYhgROR6DJTXmE7FaEUs9CUNVdQKLwqdlJ9WN+pUlXvosWjbxoUNDKZbMfrAFpkxWpFTpvQasmLor7X1y7q3P2WEg6/GMiBQ7ijbRoWNIRkVaViQV+Uf06BidMZyM09iq+bzEnd6Crf6pI27rFe2hH3A+pejPrPS1iQlvKKbOsORQ1/SRRMzOpRTo2KkCENCl+bB+2sysEKxsg4qSoj8dSWekRLNtqKLmhqFEsouUldoKtRhTbs5IAWQHcGYkoIiQXkhD1VA/2bRPzUsmtEPqrHm/hayCoOsyk1ySln1cmRUAwe31AvlOo0Gys7LMDVOOGEEYBjSHK7t5LiT6uJQlZ+0jEsfVJ9hcwR/eceo0KYDyxqpOKn1DSc1Jek0p2DL0DivL/no6AcW9pX983OemiTkpvyztmJqZq7s0QLqGK3AuJJB2wzWHK0IrvwxgPJ1hRQlG4SLfKmPWio/SgCDAWIDQkYE4Zy+xRQNXxqdWFmWYeFrKe5JpqVp9GBDIxZnZUycXFkv9+tk91OrjGxdaGfUsHaD07KXNZLFdrPHF0+XM5oidY7tY8WJ7x0Zm9yD3H+MziALP2/E57ribB27fc+010hx/7JWCkPikZr69Jh9s+VK7zQmmfheUZ2ptwhQV1z4ND7/4TsDeP2TtKXr6PR9bAStg0laCZtx80E/zYTqNz7FSMsV8YE1cx3QpDcCCpiJ4CR+K9EGE/GIRh9thAjkFykFowpBJfgAPqBhhvvNUSUmzybR1anjtCKmJ4oD406EdoN3QVkY2TUnGai40GhlZ14W0fsV/nKwrSTR6PkCnvhLfuBHemoPafhvayrUQ1qpCfGJBCYS+FuXwJaGxcte+QpvN2tFSm9dPwjrg5Mnvx+Gwysp3kjKt2GhqjXDQi8HHtcxzQAlEcWOq9YfXzD61088wKGN8oiKYiVSMPkyQGnziIi/IodyjUEAf7CXil8aFTYw0rCQ76ktFR4eKMsswIMIWKGUj1HB1pGst8hD/sD1NCjmmyvMKxnFl20pvaBUygVKCzSsXFa5wW/gMg1KC8RlUHgRtnfb0ZQq1YV99cFz3VG6RJMwjq+mLA63z8gG9MFBBhgc5BPWZfm4HtSlxpMPhJz1FIz/7LuYkR+3d5eSvARHQcfhymOAijjyVEa2l4UbrdtAIhAQlr8VsmjrGK2qYckz5EF/kDIqulykYajmrlKsw1iUYcGUqEVdrKNgJ54rZVgckWGxXwrhHrUl8/sxLFBss794VEryofOQBu8IKef9R1tGXsOp9Yn4kBeY0Y+QLTWM/quQCFv5h4fuAqePE4aJ5lei4ihkav70I17D1TKQoeplo0LKOtOHzmlkYlkK9ZJOvz5+8mQ5IePitNajsIibUQTulcP7F8oBbe86p7UoKMfIiH4zNTuna2+Zmt7bGbMxquY+XvsrPJh/uBNuGBZIEIdcHVCuXM1nmhP8bWia1rrWfORuVYywsFUtbS0mREv3FXCaQrVdwyv0cXVy7fxUbFQcXVop9y9xToUOspNhwXSuaRkEs5oGhyFPoYx6nTh1upxSvTEutA+YplDpftHI307BpkHOlsZcjOJcEC67QE1p4TY7jE2r78xop6crtBnANdpp7Gr1p1n1L06Rl9BVLfGrutE5+fM/8YhFnuuvn9ZuBkvxhJxASWd8UAcl1P0T4naUTVkusaYoTf+9cz9SAri9y3Sw6at2IkcoKbqk+jOCn7z1vEBgrAzT5GeERxNsfCYfAQsneQ9WENMER1zVcoH1vQesEStNh7ufyk/WL9ChA0zijPqjeQITDcudjMjsChgL1vIan5vAR9FZ5ij+j/7gS0YTHsLYxLB4CIU7IT2RwN9jCWxpWLz0lTcVDshDUW1Klh6c+WDk4ZkvG15GftCjwHLxp0TSUF54iGNYMJ3F6yys3NUXheiMP2ahnYYFeXFFeaYpxQfDwVNUUKYpU+U4T2VTphdOGkZw/tKJEo5hgSIeinzwpLUMTNGglMovuHyhxIjYo/n5KCzgMG2JaVBcKDnwSJ04PThHYgwrZYQpGdA07/pCx7Qar6/QV1KMiil9DcVP2mxZaUlJSUkF1PyoHO84pTr3X4v5ehyjKflVHYMLWWDUVFk4jNGV7RO9PWRIeQHXvzDJG3ektZclmRWmb7mRfFWaett1YVMeIw+e2xvdTAg9ncxzvuVSDVT1JUaJOHiPBbos8j4jn12lZI7pdOPt2ppWh+rJKGS9BesrpqQY0nttWKjNcNSLr9HIrBkYbgX0WtKQj8Ipp4pj5BpOeQVsVM59vgoAJdn1EHy2a1+vPmy6VR6kC7EtzBa3IRvTHe492gBYGzCM7DB9TMo4O0HxxX+Jhe2aBnX0xHEbF6e1sBmjgmoyAmfDQqMW+zQdinEW+jUnju+RYbF7mh2hmF7EqFf0P+qZ8sp+1urgFjRpV8U8w59ihsHwwQhXe7ErF2s9cutfFphjBHEQJa3DgvG9czJsdO9td/10nwmfHcPuk0Fx3+nlco+u0zrQbkUnZa9ZVtz37Aq2o8zKcJhV22MUrLHDmOgeOyNZiMaq8i/oXvb0QZ4J6gOc1D6ji75D+rRw52RYsFZnn54BXrej9Tr71JdmBbdD7cp6EX8kEc2soxJcdzwbHFUSVTTRpjXNvYUO1lzgNlilIzd/pCAcpVRo4WVZSuFetuvJKQH8dC1LZQ7pQ/5IuSC1LPpblq903X/pIhTPEQzOdKbf+BvgI388rlQnBYetLkmsMhL1j0Tfd5UMXt6HwWXSh1LWmgpEun+bXIgl/OahqHwHU3Eps8lR4caD8hs0sAgPHimp4qbvjPozMSx6aUzCEwlMJPBwlMCWhsXP3/RyGRav9UsN5SsUMB6NPKDjAcnzkhR+PYrAw1VKvlPqgzbnpnuBrhR4FHFGLZjqktNSeMjyvOVxPOorLYoLmtWAyHneNiz0MrOiozwbFIqjXBCOuL5ESrlAseTU7SldHrUQzBoLO6V0rPgLbigHgYOygWGBMSGjQooGhoWnMgl/j9c5sPRV87JRZqmLGOfFArxHDsQH7192T3G6Rz44+yKmQOV5FExv8siD5TXIk5eRjTTqp7LDaJDhIFlT3zD4FEcx5sIw6QyLhKHsdum9T+s4jiwdyzItYudFaPgFHjfyMoykSHdm/CATC4Z2c4SEwRmtxwVGgNHuFYd4RSGEQt6mSNW+I/3Scl+v24WidLLAnkXgfHXnyzPbfKIwMloxLfmjSNM946t8yjHkY3n3siLMn/2QGSylLDKc8d53uArCfVvA0cepZ9Rns3hUWoj8WwYhBcK49AmnCMmi5/peUodDcffaCinuizrp+oQMiqPHjpcTGrE4palQZzn5WjLifuZ+OHL4UDm0/0BZmNeUJ8mH9RS7tbZgqm4z640C3K/og/QxXZKJ+1uVTatLMC4+4TD5rUaPynQ7YvhopIK1E2urOhtG9x7T/lZlCGFYLLP+Q/XYLQV+fmG/tqXVdCwZN2ySwAnt7Px0x7ET5QvHTso/qVOydQ+rT2iPK91rNK54U9l8HKC9WWvDKAb3MGslFnXPnxb8imD26N6b1nNhr/rHwrTKw2hQmBFKcOZ0n8/rfiaNtRT0pSl1LPqVKuMrRl+jv7p9uOmri/aK+LZcja28gB76kiFSaCO4Ecm+hXjBvsRR7871/aRLbn0389OHJn2d/6SO7xlLrUy4rrkdr8YzbuAnrz1t81DpBGrQ4Tfhowbxa0Yq45SZqQN3CkmeDUeByhkZlXEIVMzMTL/Lis5a69a1kYvvpmw5Xn+ybvCeYZOs7bBZGv0SbsjzVWlVDp02MSyqUCbeRAITCTxsJbClYfFzL39Zec3rbrYSa8VXL/adeqnmi8DP71p1p/nBqXwrr/gBy/SCxMkRi3VphSiK/oJrPx62MW2ElwRx+flewIe+LivRUhBScUYZIhzKfHyx9JaktVzyd8lA4AvkrBZ3zuirIyMR4CxrS8qzmpN/WodpedRChRpeow3UOddX2IBRfIoRi2ZYMCKhdQHMCZfSxEsMmmmEsL7B7zH4kBLHQtHYDYcdeFi0HQu324iF4MJYiPr4C7Ho2WCrX4uh71ELXlHIouUP8gija6CFzPRf5QeXBJUGXwqTEr4jzovQ5vH+pdkQaSuD11+8SpR2xDmqH5dGROm0s116pDmd/Mjl133FRkMaGaGsosNxgYNxR1vYuGAqmeLwimHLF+wZyX8KQ1HhGM1CfuLGwgk2QlaRBouOw3GFyfyArvld3ki6ImIrmKvhVCpy5AI/L8tVCBaN607FQEcCDkCt0ctg5tuXMDwigOKu06lPHr23HLv/3nLffffpDIlFjV6c9foK6HFy+6zOkzh85Ipy4MDBMq9F3Kyl8M5jGBUaUcPI8E5Q6mdpyFoW9J9oSbMUvIgqDSHacR8Hz+Ze6VFPGX4areAckXMrS/JjETlrLNgCmDUQGBjgcx/tnd9XZjwda9bTl1hHcVow95/U2Ria3nTfohZna/enxbr7E3chi7iTBzoGbT2n6XBze2dlpMz6dPdzkvuK7tkZjV7O634+oPUSXPM6LZvD7TjvhoPuphm5UH/h3BqMCWgxIQ36YcjRxzDraJZaf5hwSqQRS4fMcMgEhyxDZo6O/PTp2f8CLXBHgEWndz1un550Gq8tE5r08+Q8/KZXUzcXLjgXP/CQdTIy9ETEdBpIDYgGrueNlOQpapD1SJ/itjIsKpwKNH8dn0nXxaogIMIlTo02HMXzhUNWYyHxKjxgtS6UkWGjUPEuv09r8qn54LUiatrEsEBiEzeRwEQCD2cJbGlY/NuX/UJ59WsHwwIlmz39UWb5qh8P1OHRmI9Jnq08cD19Sr5HO1CAFUZ5Z9cftn4cNyxS2eKZzbsZ+u05H0SlDKtsvs6jhFc+MARQ4m38KJ3RChSAUMiDV9ZKYFDMa879nOZIszsTtM5oYeeiDlg7eeasdxRSoTZQMCCyrow45CiGp0HpKyajDhgb8AffVmIpUxenezPdCuUkRhP0pVdpzOfGsPAJzzvZujNGQhg9QbZ8Xd2p6U02kMBV2jACEfLzdJ18HSFn4TRZVzmkcRLpiRdt0lCFa6PEvVeyRL6bOGiMu/5Fetk80CpuwicleLZTO2eeQiAg/noRj/xYW1EVU/pESx9wkD99i52hVjUC5TUvKO2Cp24YExgWjFrQ9iiKGBxwAkXzIXLAus72Q2ZKcVrKAn+zMPz26cTHHeVwpTHR+5mHnJKfTKtCQUQWUKY3XzwB412hJIONtZWydnaxnLpPp13fe2e5/567y1lNieIUebbkFaOeasSIwMKRq8rc/kNlWiMDO3bJ6FXfTqPCaxDUF11nyyH6U/AxWrvgxWMn5j83YQCWPNqRaVpeTyHDgqlQG+ta8cC0LDmmEvJAYl2FwN0/d2kXtO1qN83HkkGwrSxrBygOvqO9vWZLhtRdJ07qOlXulrHBIn6eHezg5M0ilM+UuT0ykha0QP2wpnzNcg/q3lJjl2n1gwMaqbhSh93laAUjHNyLPFe87on2hqG8ose4PtSL9NYOKs9pVMhZ+sHRbtURrKlOSfi+X5lehaL/pSO9xaDfIgkx6idtUgHlnsdRPn3PjjqQy7+u5K2mKoE6AZk5gNZwBWrZJhO0DNVQCATvLckkiSW8QxH3nQkAzm+DCIrBVidVPu+3KCv7HuunKjhmYDCvBMqvGWTXYORXrhBoRU4agAIz0CHlUjfwEvCXQiilIwq9jhvTnxgWm0ptkjiRwEQCDyMJbGlY/OxLf6H8x5tf45cRyi7KWUwLQrGXgpGPRZ6OfvDy6ObZyQM//FDIwxAgnRELbycq46IZFuDqJceDlulSKINWDuo7x/KEphQBGxZ+6Y8aFh5d8HSNMCr8ZREewREuigQK5V4dkrZXxgVTQFD6l2RYnOaLpwyLDSk4AaspUzIsoEkVvYuMmMB48TazKKbKJw8WbVSIZ3BRRjxdSsqLd5wRDU9jks/uM94BioWhXueBQaF1FzZCwmjbKYMjt/X0TjUYUdpaE9r6l+Onhp0WcqBct4nSesMCvEgXFgTi37RslAS1yIP8mDPOWNpm0c1euuNKj9kXMjT9GueHtq8EHVIk8yKbFzpg+EBkPIwGdRf3lzQs2KYWxZnRi1ScMGwZsWCuPkafDV0Zn+oavqw0wgq8wYv4Q1QRDHlmOOWBn1dARr02CycufquH6tIbFYy0ZB5FE4ahTFNA/6EIOk99FXzvsgQ8lcFX/RmxOK+RgPXlxbJ8/K5y5v47y6mj92iKkXZF0iJnFrxvZ7RMOz7NLBzSdbjs0ZayU1pP4REK5XkXKPqrjArufcuEfqcy4hf24AmpwSdXKHYRr3lkC6lCRZ3VPjFdSwvItRbEO0L5/lebiX/HwVAbcXbEqoJn9SHi1IpO09YI4xI7O6kO3Cd7OGNCha+whsIjj5r+hS8DakVT5Fh3saG2Yve1Od37BzVycVD3/wF9XDjARwaNYM1rhAKjgilOnKAdoxK0e1zUJw0L6uE2cd2jD1J5/sRI5CGM6lI+7vNd+ng+8exbhE2PQLqBZExPUrphakfty0mU5vfluptEX4HvdA45OdMqDPXiv4ON9k848ioVoSTWCN2M4Bs2EYJurUJAOZJU5AvUsRw6UX6UN9AAEZ6yXXRTNEacHpRNJ+U69GEVob6Xrm8DpyUrKi5l0EquFb8EJ4nJT5yglaW6WiPkE2WyK1RKYuJPJDCRwMNVAg/KsODBiXKKoo1hgZLur+oo7qp5vhjaCwZpKIPHaOCGjzqJUpPGRSpWfiEIGZ80n1MgEjzA2zcmvfCtMOvlDy8xShEGi8NSvkO5riMWhg9Y2LFyWddYYFhgZFCHZSkpZ5dXvbATvuAX5dP5MiRQjlFcTEPl+mA7jX6wkwwOVYp86kr5jFJwgF1MeWJUQl97pdjaIBIOuzqBi+9dpxSObWjZilay1dfiNC6Y0mTlX8qVX171JadaOZ5GU9QbRZlyQk7A55UGh9uJdlFAYEFTdYB303eNRn8ulz4KFbGRlyhJld9xWMrDWapVttHWIeea1N6+xJO2+0qVuUcy1J+Qv6dBKewv2SinKBfCg39GJtIw9ogEMhIPHrFADuLlEladaMlEPsLzPz8hL2ilo5zLyWqz9Ozr+J5KJ8PChoLiUHU9XfEaRvlBcVeaDQzF2VmpKUUqH6XJh9uhuK/rq/7K6bJ66p6ycuKesnTyPm07e07riTSaodpvn9JUp9mFMi2jYmruQNktI2OXNhKwYSGF3TuQuf/FGh7qGdKIGpv/KuNU1pB3C3d1aPUXj2EMxeGEvv+pt+oRV5eOXESD8yfOyBg6rgPv7j+tE7w15WlFRsaG8rm32ABhhvsZQ12C45TxszIqTmpq4ymtxVgW/prK5UPCnBZh79d1WEbFEU2LukKH283rOeC1E0zxVH2QIkZEq6twJfCsdMjftaTfhWGHLFxzfP4TvsoAZGTQp5OGy7QmI9IgMu76JIczIfpgj2Na4mEcApI29vUTWEMh8REgMTIdKKXx3+rU4wa8syrBS+gmKfwR8pVuTUTMzWUkmApeM99yTMgkWH17qjcMNX4iQBahgIz2jXCtW61f3w5ZSvqGF1z6mX45nEFmCVn7wRB1KKtGZGJYjAlnEp1IYCKBh50EtjQsmAr1G7/5GlcqRh5Yc8A0nzp1h6k6etBLn/XXvXzA8kAN9YiHuf7qk5N0DItU/vqvtMZRvvOEzTckaPgVIHyXz9f7NCq6MHk2OsRI8BmLNgknT9ukODDdiK/W0yj+GAcyLGLx9pqmiGjbS42iwDGGU67BUNS8UA+mWHnKkqZKgMsLxpdgmpHAFCqVkdtY5ghLjLZ0oxdS2sDhbAWmVU0xvUpfVXdzAJ6NC03Vgn8pgQgwv0gjSmSQoxIRFlRNM06td4bJSyNEWW6PoC1icpRCPB1tkbiZ9qB9v3UFDbkkqTRoNlfTKdevafIj1EAsWMeAkas0oGODQj4KNkaFR7nUr4Aj7q/4KoMq0WYOmFa0F8Xb2LARRt+J+pNuPgngSPdfRFt9SHMeIBEOcMIOGSHrjPw3c+S7PjISYgqQ6qNwjt4FEfHU0jAsdGdotILpQxfrgXPABR3SpaSzQ9mGvuqvanrfmaNl9fSxsqxrRQr6qs5+WLsoY3dKOz7NHrBhsXvvvtj9Sf3OC7XVJ23UUpl6IQccPpKGD+7fdL0ckHLfZuTZyXebYRTpyhEz6seaC6ZInVed2LGK6UvratNljUSwhuIerae459QZLdTW1rMijmGhs8E1mrFNO0fNlYPaPeoKbZd7QGuodqnNz8m4ukdnV5wSXY2JeIeoee3stD/XUzBqwVayTIvC8KcfYahRJ4XTwXu2Y7YX9ePiWeX2Ar7DSXho9JaAupEAAEAASURBVOGk6fQaqZJpWSPwmWnywVO0Q4TNRspWFBoueJUfWqLJX+GKqfoCZMDos7WsoFEjzq+siR64maMCasYQGmQVedFXEiwLIC5aKS/8RrTntSUGgRbNAH7SyfIC1HXv5JIFDLWvcA1c+JWfQVYkwVuUF6U1hIF/JV2CM0p+JDbIaCS5RSZToZooJoGJBCYSeJhKYGvD4uUv9VSofIdYgZeSZMPCX97HFX29SuqDmIcyz+Xh2RwvaL4sMzIQXytDUQIWQwKlMA0L4yoNeijR3mO+GylhehFf6v2FXr550xsKFY4wiiO4frnprQCcDQamT+grJQYS6zQ8LUu7xKzqYr97lFTSOfsilavkgfI8jUZ5jN4wEmGFXmHoMTXDdMnD6II/8R7rQeralMq36yR81mJMy6DYo114uKamZmTUxN76bfvTqhybD2qETOCFenO5DMqqdZaPI6+lIQ/+Oj3bCnWlZ4SxH3Av58bzmqKwCQIvdMoeddE/Mi1e+hUGjw5QHdjuDxgPCkR/GcKZBobRjA9yKCqk2eCQD99uD8HQV0I+aawpEWDwcdS/i0Z6ZA4gNV5lFfQiDb5STukH4eE3eUdJ9z2BslrDMc1J9VSaFidI6eVaU2U4wZrweqQbh6/+UqFZs6A8K+gKr64slnM6w2JpSestljQyt6Ytes/vLLP7tFj7wBVl/+GrfF6F11TIII6F//QrTX9ShemD4/UWR9EGMgDSZb2znYHBcT+1bmR5xr3Pfd7jsCsUO0WtcSl8Tvfjii6mNR3XGigMi6OnlzQNSkaTGNqpkYeLuudWVcZF39PaDlbToq6Z31sOyWhga9h8lkhSMjC1BbHuAbaJnWUjB+Hu0RXPsrg3MSpyxMfMj/3QVvQPfI9WYNgie9Jx6bueVQKZFhCWJTIyxlie6SdclV8VfkulZwUf1VdZ2RcRdMq90VIZsJMu8xm6cHLmpW/6QHcJFTn7MDmNDuGRelRZQAF+HO1oKW5cMvQPbwYxPCEluICKYy9go8yaHkCUrv+koCgu463iPa0EGcOJ5AGXeMU3thkNnNH6JmL4m+W1FhKhzfLBJB15TQyLUXlOYhMJTCTw8JPAlobFz9300vLqm2+OB399WIfSHlOhQskmHIo2fiizoVDk1+N8IfFYZoEthoVHKzpFkZeGF1uqHPx87IdRgQLOGoQYKfGC6FSmpSy4TL1orCiqDCX5tRMvBr3cFAcmDItQJuAVKPOjkQovKGcKSVWWgG+Kew1zMjZl+8A9GQhsSYkhEQdsMeUpjAkYYISBrS/hCloYImkM2UCBH6V5BGXPTJnWLjx7dBDZHp9rgWERyo6QZQwoDC3TC3+Ev5SF8nGGS/hxHz2xyqenZ8TuJ9usSxoJbpZ/2ZemWrO9XDsqPXyvNMDfaB4JkUY3JI+/UOgUcmchg/9auexD8tOoAAy+o38ILqAtkJQv7Fk+ZFeuHYyo0wjiWjphkMb8rEOfl2FgM9/1gU+UVF2sP4i1BjHVRsMP5eKGDApNb7q4oVPFdV3UiARpFzxqoX7LKIWUco9U2LBgkbSUc23nuqTtZU9IKT92+pwMCxW8a6YcvvKacviKq8tBLdyekkHLFCj6Z8ohDIpqmI7UFM6RHHKXwWOZh/LqO7DKwRDkReMYhyyPWFBXPQeyPSiLdRU+20KH5a1oCtNZjTSc1rSnY5oCdUyGxVGdP3FCPou32fVpRlOZdmqU76LuR0YuuKdmZBhdrZGLa3QyNqMXnGHBtEbuFVlnuhN1gJ448e5O8ukH5GFEMQKJw7hggbkYNe/ZXlkP+Mfgu3ARQzBkEP2L2nDhas+oNCKt9hFlQdNi8U/FAjYBFaoUasoQY6nBwMuAYT6zWGElDAW1JiHdFCEClxUBbygiYAzYJQKS4PLZ6rVRq/Uw6VqCqRuO1IpoANVTcotUpbesSIs4iD1atEWg1wzwjFLxevhOrtl+LqiW1WRjgqM/5BksK6vsFlfeVrhQCpYqT5W05fwAZbtclTkxLKrQJt5EAhMJPGwlsKVh8fM3vcy7QlE7Hnz50I2XcUwHSuOCaT2hcIcyEtNT4mseD/d8mYZhoZc3isWYYcH7xsaFHs/5aGYqy24p2V54yxQjfW1ECUia6atUv/jq8zv4zWZR+eB4jYh8eDY/yvcZFBg7Ni4wLKQ46C8MKIyLKI8pUNCwUQAtxfd4gXaMLqClYEjAN6MvfMvlQmly2RggjPJIkdnBIlHwGa2QQjcjg2JaC2e5WPDNVCpw2BUqviBT30HJ8ygNdK0UpTEXMlaRrpuVqRpOGeHztZ43JXJSFKkZHrx0wF3ehZz7/HzZWqb5UucFXem4NbNBhdjSKyy0MjtLHnrAkAdQKjSgeh1FQwgKSRIfGmF8gIfCMPCePPdlE8ZF2yMrjM9BVo7kTxTnWNaHSIbx+zKI59VIJLM1AXgMi41qWMQXdPVHjIf1lXJxdUmLCJbKxbWzulY8OsEIxXmNVJzXVq0XZGiwGJqbmi13uTjz4cSZlXLXsaVy/+I5feHfVRYOHS7XX39Dufra68r+A4c19a6OkNX6Bjvwq75V+Q4pJOfpS8Li2X+uS20M4SiDBrALj98Kj3KufN9rwFGG/qj7muqwpPURizrP4jhToLSugvUVp1iQrbpwwvqaUPaxdaymMnF4pdZoW7Ys0D+kLWOPaJH2lTIs9it/r0Y2eHbwgcLcySCwrPiAoPJI9Igf95vudfj24nGeTwpne1KRbB+modkIVL4NDfu1bq50lYPS0zVaVZ7E7cCtQIQqplKUmhldatQhURtA8JnFkl2zBuqBM2RUAOEge9rADq/yliyS7uwszqAV3uCZAWR1CdMTqVmMwIVc6WMCNIw4BaeRqvTtKS9heEkkr4YNhL6dkoX0k+QI+czEh3al2dqpz8/wJnUJ9FqCaFSuoxoVPuQbRHr6rQ9U+sR/7Id+NEt7yH3ewSm39PtCSYMn93W12So7uek6ePBgDzYJTyQwkcBEAiMS2NKw+AUdkPfq12nEQg8XPwTl87DxhVKrsJVtFO16oTSjkPmBpOet8WqRQtcaizAqUOR5gfurX81HCeS9wcvQB13pMY0RwFfHaSkH7MjEKEEaBsnLSI0qr+ZZGcmrlXvxiDLOg54yeGD6zINqVGD0ePqEaFAHRigwanKkxKMTpENH6SgsNgLEHy9Ftrik3Bh5CcOCRdrAcNgX89cdVnxauz/tkb9HvtdWaDqU/VyjgSGiC3wUHit5MgrSwMDggkfql37KI33kkuHmm0a8R3kJ+qVX4YAfd+Dhxn3S+rYljtssLXKGvKRl+My078ZvKYrV1AwpWoMSs9sQvzlFHHUeqZHJr/nKfOcIFjjS5MfUFuQRC/1pe/oKCZZAiCHCgmkFOb9mKjnrht/LgnheoOPG84MXFG4ptdKW84WO8XABQ0KGxcWVU+XiudO+zivtvL7wM42Ir+w+cZspRZ7WJwVcZz6cPLvmE6rvuP+0dknSgXja/ekR1z+qPOIR15cjV1yhMyLm6kLt6EvBmH8rv6SHgRWp+YuskZ2laxk6WLMznWjKxPWrOISpn+83+TwHCDNVkg0VTuuQvBMyMO7Vbm1H5Z/Q6AUlQYv+Pl+NhlkZDjimSu7Rfcmp2OzwNKfRCqY6cf9y37KpAhfPDhXm6WKs6YAHnA0o7qd6T2HUxYePMIKASZ496gIN1YEwjEX/UcBphubHPPODrFr7qw7AhYOGQq0L1XTjDMl5nyZWEIZCptC/kmL4kTvQ61MJG7yVG4FtGhIxP5SfPCbMUFQlNZqR9TCvZCW8oSOSNLM+ZroWCERLryUgt8FV3iqc85LHAUhEkq8uMeHG8zJdoFlS9tcOezS4CQ4A4GX90t+Ek0YrYRIXfzJi0cQzCUwkMJHAw1QCWxsWr3h5eY3OsWhD/TxQ9fDMFyQveK72VV8v7VibEAoZMolncLwQePly4nZOP0KBiq+W8VD3g10/emW3clDiUeCthDfDIkcsmMoQj27KMZ9SUNpLXryiMMT0pRhR4esksKHEMFKhr7udYeHdhVw86yxUtpQSn3sgn9GSNKAYsWCBt40NwZ0XG/CNURHGkhJUNl+Dp6T8sC2mDx6T4bAbY0JGxm4pr1Z2MDgEt1vKkBdyV+MCQyQNC+oZX5BRfqhX/Zpc01ubdO2D/DMdXgiruWqafADkeJmTt5lr+MCNwfQvxsTdLC3z8Mnv6UjUAyM0jBNIDBfRIdGhGoVWxJP3QfEA27zULMMCX3EyjiEbBm3k0V98hoH6cvvKXXlBYJBDXlr96tQQiVMSqtWPMvq69rJswDVAXuMplVb6ssKe5qRpUBe0GPv88qlyYflEOX9Wlw6Y2+Dkas7tUD/OU+Q9WiGjgoXax5bOaTelczpI7lyZ0iLtQ5r69KhHP7pcccVVZd/CgnaCmhKPnVERtRNX1C/6xaWGBXIPeQEH38C63zvs1EpDOapbSIh7E0yyMCTYIU7rm8Q/i7RZU7Emg2pdzwXqsiKDCoPiFIaGpkBNYczr2qP7cEb3COeSzGq9FG3AgZhTajPyOB2baYbcMxjgGIjcp3yg4CMB28l6OhMGmUeHGJ3gK7rkYLwYufDIhJ5Xo6MSYQRFWhoXIYOY4qMwnQonvkIc5CsqfpwMw9VZGiG+lhICClxD6id7WKUcsI5kioEoMtENY/qV8kiG0jAiBtdh0kYjtA2clTGZvl9DI8D1W+uW7R30g3b0kyGFWoWMej5IAp7iSK95ncwikxzlBaNOaj9jsFmuqfZ5He4oB2ZLP8FHo0tgC5yEy/IyvgmVWreEoKiAmhgWg0wmoYkEJhJ4eEpgS8PipTIsXv3a1/oh2JR11ZOHYH4lb8aFXsjtUCm92K1M8KwULA9aXrYYEmFUVOOiGyEYHuyC11/gM2VIij1f+HV5xALFQApE8rBdIyQ4FBaMljBUeDEGDWAxBqxQSMEALxSa4IWvuz73AONC/LAbDTvOAMdXa07fZbH3Lk7jFQ/Q8XoPaDJ6ojhfOTfEBlMyMCx45aGUotxMz8xoqpMu+VPa8QmjYocOwbPB41eoEFWHHaLFzlAzLORmdyjOEsCwEP2QsRQl5K4rFMFoh/zCmvJqBofgcJmugMMYFpEuo6y+wBtMZLXfPp3w+AszAcnDuZ1d/8jJNiAW+ISGliYGX6RZauojQSnhAzb7A+CGowz+a1kBVTEjAmjLd0Q/9GErtsJDYTe/Th9gY3pZlbO4yTIUcB2oR7SByov/WgeiXeGim3LJ8vFJy/RePuRlWY031xFFVpeUYA6T2zh7qqwvHS8bi/fLsFBYp2uva5rQsnZ8OqutVpdW4pwKrRLQ1/9Sji7q8MfljXLu4o5y9XWPLNde94hy9TXXllmNVHBoHCN58MOIm/lRLZBk8Bj1oVUi1VwKDhM67rHkmXwbaGpDp3VKa9YXuuSZqsrkXsWoWNL2sEfPLNuIWFzd0H0lo1337bTuC3Z3wthfE122hZ3VfTKrEQmfqSPed9OhRRhjwRdGt41tSnOW60JefhRgwwQ+hsAXRhtrOxi9gA73E8a864vcGbmg3yhs3sfCkVZlIR4FZDiXLfrIhDRoq0CSq2yVrDD4Ndl5hq3pIA1o9cYFKsgYF/x0Tc6UK35xTgPebIzDZjx94Ab+B9rio5Y5BKAersOOStX0wXAJ5Eav8gxv4LZ0hUnLovr0LNc4FZ/8BqOw61rLTs+8deUlTI/b00w8fKd3CckXScavMAnSeBkSHMoyiVwCU2ETZmJYVIFU76OfP6fDL7VJhRx9cG56e7lqYVd55BU6R+rvgLvtU8vlQ59dLt//nANldk93j/4d4G3CwkQCf1sSeEDD4jUYFuKOebFpXOQXTn8550Vcr9jCs04VkmLg0QQ9DfiSyYsuRyrsVyMA5QJlfHg58bKJlwT4PhcCo0KXz3mQMuH1B8qLL5FxM/NOP6+XPsYFfOoJbr68QFqGAWdFAE85fA1l55k1fQWNqVBSXoRnvoTHlri83YBnKgVrKRi1YESBbWr58snFV1B2fxKgFDPxIXhkgzwwPuB5RguzWZDNiAW7RpGPogZ/8XJSTHUhz4aFlL1pwTMtaqe+ysaISBoXYRjx6vWLSGTw86WEPxgekS4Q85NwsInjIU0etHDwjAMuX349XWfqZzwv03s/YTKtp0lan59lbJbewwWb0TdaumRovFqTGrHn9JofMCFv+mFvfAYwwghJwA+ywQGbX+FJC6ND7SshJt8VNIRJywZLQQ8kAMxH0k3fRTQ6EYPHkE8aQCRQ3ziVWgrwubNlTQffrS2d1Dayx8vK6ZPl7KlTWpx9ppyScr6or/soxhiqnOy+rFGLC9t14vjcfhkU12nB9hVl3759Ut45AFL913VRmdRX5UR96EekwUxyFj78xHMgFNfWFsLseQ4hVuSKioc8ud+ZEsl9xm5sGEV3LJ4pd4n/uzVCwbawh6enynU6DXs39zltIkY4Ld33lbZ7Rv42omFUDojoHY7UNJoBfIxoXcZRHPkw1c31VxXpE17wjhEhGQDvGyXuBRsX4pf+MBgZaWhE+4BnWdh38f5p8hFNcyriApG7VDaRqvSWH7WCH+OYgMlGWQRJMwptpz+AcUm+wxkSA+TSuFIoKBhMoOpDezMMpVUeDJjlNgZGyaQ8kq0Eb3wn+CY8ANvDQSvp9TwnTMsT41len9fyO7p9WrJyid8aJPjJOlyCW+uQZUKHPpSuTyeN+MSwSOmE/6rfvV+bNpwv+2d36PlSNEWSjVYulhuv3VN+8FsO+Dk1ivE3G/u9P18sf/Hp5fLT33O4HJyPs63+ZjmYlDaRwN89CTywYfG61/qhbOOgfq3jDcNDMA0KwlwoXjYupPzzdZDRBF7mNh6kuDPlKIwK1lmEgmejghcEshEszrspKQhNz41Gke9GC6wUqCymXVEOaODbsLAyGC99RlAwBPZoQTSGAfzyYD+nL7ur7Omvr6WMVti4wLBQHvzwAIMjGwiiP60vpNMyKjj/widviyZfPb24uhoWnJJthUU+xoBHWjAWXDYLsnXSt5Qg6oYCZoWVUhS27FxHTe3Q6MY0xogMDA7Z28UIh/LCmBoMi3xTWmKi6VdnlRn0UvlFNqGAVVxpWKThwqttWRPBzRck4XR9unG7vITp/aSRsH28D/dlZPpmaaYtdtxPsr8oseOwL77pGQJ1W4IIffpIGAz9C77KhjqZdijPKPOMboFH29mwkMJJn0aIWXbyG3FzGPnAJFC0kNs6ZNKzm3CUFXxaSa8sUj4KLYstGbVY16naa8tnysqZU2V58UQ5ffJ4OX7ilAwLtmNdk0GqL/oYwUxz2qEzW7Q5wL79B8tBLdqe37fgfjX0DxgUff8OPGEgSQr8OC/84MMjFjBaXfAnOPhUmnqQf8kmJfIjj5PRmba1LINiVWGPEMq/Vwu171paKXfo2i/D4hotvn7swlyZ00ghJ2LLfLfceb7oNq5OvFu+USLlWG61TNqFC9w0LgB3WjUs2oiN0r3VbDUgEte1kdzJ89qXzsBo9XL7IKha18odYohaUyj/ZhYwOf8kZPOhGQ4fDOpg0pGshhnBdweLNNOnzkAGeuBsWlaWU0FAybLTb1nBNzJp9RGMKbj8BlirdSnthGhlKAGqjc1L6IzSyBg8pDOt5KPj2XyOxRMn8cEd4aXS7dMS5xIf2Eo/ZCII6I0DAiPY5HizMhPHMIL9sR98yTiVhyz+cFi8jWGxUzfvP/9Hhy0Hnsdvum2x3PbJ5fK937RQvv4xMw+ZfB4s4Q1NVYDHiZtIYCKBkMDWhsUrbyq/mYaFHpIoZBgJOB6ovWFBGsoXaTHlQEYGoxaCy9EARgqsHEFHygT08kHuxdqC9VvUfpQRtFjfIGW9GixOk2KQhgXl4sKwiAXhPOiB8/oMGwQyBKQUohxwGN457TDDFfPRq8EjnDB04ustdKExK0VnViMOe3X4Fif45oJtGxbKZ7oShgPGhs+y0Jdi8FBabGzI+GCxN3Xz1z3xytQw3ju8jpAl06NQCGc4SZipU9ohihEMzrSAphdyS7bIM1+O1Hl4MdV0izDCUpcNm+0UPgD8gxlyc1si8+qyTUjPsvAzPeE28xOevMvB9+k9fE8v03tYwn59J6+qQgajOn0dglriWNaSeY660fey/sgl6EQd2VTA0/YEIxQqUo2KKo8qdHCUYhnBL3+tRcgjrbvgiDYxGJHORX1dO8st+wfA1CFHCbwtK9ODNHXn3NmlcpYzKk6dLKdPL3pb2TX1afqajVL1JbYwnta0p71z8z4nhb4bZUXhhJuMFYZ/ywxZIzTHLYKaHl/1SU9c8+f+HIIxGWOCF88NG9MKs5aCnaoWZQCd5UOD4Bj9Y6rTCRn7d2pL2X2616/S6dhftV+Ghe43DAsRir6f/OEL1zwgI/1RBooHsiJMHheGRW4xSw1xpHMvxf0pw133GO2HAcFieLaTRfZMjaJsGxaSexofHrlQOTBB2bmuImXnQvQTXLpAUXOCZUJw3IW8x1ODV8rBBb14BjtBdbBTPqNPOMvFIdIUh087IkmjEsz0oQCljOdZEnVkgjxxIZng2nO7xis3lm+DqW1hBHAqrOOJB6O6+ryk1XOTNJrf0epxW75o0tbpMgRsD58wfVribOlD+3I81HKhbbqblJnluXzBTQyLUWmPGxbkHl3cKK+UwfGMx8+W737KvhGEdSn59xxfL2eWL5Qj+3eWQ/uY1jjq6E/AHDvDuq1t5cr9u8o+jYj0jjIwFvbv3VHOrFwoX7h/rSwofNUBnp8D5NI5bZAhOtcdHp2aRZe4/9SGeF0XjZ3lygOMknaIA4lJaCKBByeB9RPlsx/5SPn0HUfLGc0O3D1/dbnhxhvLDdfqoFt3X300P3lX+euPf6x88u7FsrF9T1m48obyhK97Qrl6Vo+pDe22+IW/LO+5/QtledtcueJrn1GeecPcgyv7i4Ta0rB4+ateWW5+w+v84EyjwtNI9GLhJcxLl6+4PBTzsnGhuLdnVR4vb4yImHKklzZKW2dU5IPVnyG5Y33vQV3Ohkoo5SjWKOueWy26rH8gvrOegk355pGHN0qjfHiDj9hNCsOC0ZPzHqlY07zqVU2FwvfCVysUYTihnOAwjBh5wKCY0xaW+CzERjnznv+ijVEBbzFvm4cHLZz1wNgKQ4spHDs0d9x4FYatU3kAAc8Cc2hMa7ebaaZD6WI6lHeK0siFjQvkqcsvIWEFl+CHs9QoWvXk8kgFfsWjbZh2ZdkKJekIhFhLD2p9fmIg1qHUAX/IT1zguBJmM7yETT9hM977Sa+Kq/I68Gz6URG3fcel+Wj48GXFyGAqgr6s34pLH4q+LgPVdY0yDANc1kth+LVs6ygGkNC2EwI4wMR0I/yglfkBGHD8ug5JQWSyDvCTowQxNQvDXCNtjLxpfcK5lbO6NIqhrSBRinHwhbK8W9PqdtOX6lQ80nsHT61O+eJTHS0HKaQWgRDgJeC4t0JRHdJUa/MoHwRkpD9uI+IesZTPaOUZGQ8sxj6qKU9LilP+nEYT58UjvC3L2NilkZI5GRdsG8s0RO55s+b7WjyZl/BVgv6QOx8D4oOAjRhSqrypsfZti+lQCuNIa23D/VE/AritlMdzgot6pYwuyLDw7lucbq68aAtzQ5Xl9MN/REiIcO0LkdDli78GU2Ejib6ScD1y9JWgX2XtOgIT/awShFoNhh/xKI9w0Oj6Y8IHln4DdoAgPtBqdQyGI0f1Rlabuk4m5JvSWNplcRMenzIggKtlZZy8xldANJgWrYGsiaNjfJDmNidQ87IPRNJQz74ePQxwbhMHRut7CY+A1vtyYlhUgVVvM8Pic/etlV+99Vj59q+fL9/8xL0N4a/vWS1veMcpjdqet1GwpilTX3XNVPm+Z+3X2sV47t17cr3c/KcnbVTsllHBtCrei8//hvnyjMdJ+6ru3/2XY3rna13awV3lLR/ULnw1HXo/+NwDpk/SHynvbR9eKv/6xVe2Mk5q6tbr336y3HF0Tc+v7ZohcaEcnNtZfuA5+2VgcJLOxE0k8MVKYK3c/f4/Ln/ywbvLhemFosH8cureo2Xj0OPLNz79KeVrZDlcPHNn+dgHbit//smTZffBQ2XvxpmyuKg++FXfXL7zOdeX3SfvLB/5wzeXD5arypEdZ8rdpx5VXvDjzy5XbTtXPvfe28vFxz+5XDenj+L5QP1iWezgtzQsXvFLryqvveUNfrjykg2FC2UjbjNewigDPFC5/DVdN6m/BMp3ul4DORXKU430oraiVBUReIEaD1beE1bwlOB3BjSlhOdUoFyIvVPlptHgUQsp9pQZtPBFACVEDv6YmuT1EIJBIWi7QEmJWWWtBdMydOHnqeAwAG2mYM3pJF8Mi1n5rLPwugfKZEQGhQT6nq+dO07lV1MdyiVec73FLilQObKB7Kg388xRiOAzDYsZGxeaEiXf6zNYb8GoBQpWlSu44y7krVTLTbw1v7ZTbRvezCEtQGmnpBSBjJKXjnC2e59GuIcjnnD4mZdpGQdu3G2Vl/j2xdag8ASVSK/8jigKKJ0oHMEXYRzwBsN3fsBZoa59s5dxwMeXcMKUD7+0GzsPpah6HLjxPaJMK6yVPXs13NfEvMGneWwAMiJilCB4UK7K91d5G8MyMOriY0Yx8nA9Kgd/XqfDqBcjFb7HgteQdZSRdF0HfqDPfWrDIuUVfIUBEc8AchI3cYg7rDqg4HvTBvk8A9jt6YwWZ5/SaMUxRg2Vxn3M9rBHZnUuhe4tDHN2Y9qpKT8s1o5NF+j3ul+Uzn0d5eZzKKQVaXrWwBNly8dRQ1SKHeKhrm5yGum0IfwiC2TjtSm6l7nPlGzDghEKnA07lc1ibxsXOlskDQvKptT0LIOImT5BehfO3FZQc6eymwyV6xaBH8MaZfjJdJiDEr7TBI2vfmg+nK/QmC+Q5iIvEuI35AhAhoILuEyIyCWWtCnXTmXBVfSrSBr5rbxkWsPPBPmXxVWeawyNSmcE1jIIni6h2/HXisq0TBjjLZNdt648aF9CvwEH//C5GY89Xh82OvxXniaGRSdQBTEsEM2PfttBi/X46Y3ypvedtrL+U999uC2YZlThpt+5r9xw1VR50TMXdA9vKx/665Xyu39+qnzDY2fKP3naggl/TIvBPyeF/2k3zno0YmX1Qnn1W0+UOzWC8X/+U22/XQ0QDAvSWCz+AuFee3hX+ZMPnSnv+fjZ8sJnLJgmBMcNC9r//33z0bKiw3Z+5FsPlIMaMfn0Xavl98QHIyD/8nuOuD5mZvIzkcCDlsC95c/+w6+V9+98SvmW5z2jfO3VGnV775vKmz60Wq558nPLdzz12rLyqXeUt77zw+XeI88q//3zvqbsX/tC+ci7/qT82Ufnyrf+s39crlv8SPnj37i9XPPPf6g8ef3j5Xd+8W3luv/9X5SnHH9becOfni1P+yf/sNywX7pCfaQ/aNY2AdzSsHjVr/xSed1vvcEPUx6GVmZQhPWC9ZNeBK04VWXXhoWY4iEJb/ky8PQEFALTqA/n+sA2jOnwcBWOf0gIOrHIUsq7vyiGgpGGSzM0ah6wKPsmUfnkdQc/PvwKnqzI51dpbW0pY+KcFLMVTdFg7QVGx0WNqKDAY0BMsSe+DApGK2Z0YRzskJJmJR/DQgoJYeSAIoVhsobCJyUIZQTFiakeU1LuWMAd6z10yrF4Bd47UiFP4aMEsnvUrK69Ool7Vlcu/vZaC5XjXbBUH7/AVJ/ehfzjJQVfzbBwGLnUFxiyrXIBH3nFj3wRBg+XLztHNvlp9GreJS/MDoe8pJc+2T1On96hGibzxuGjG0WfSpge12VQDpeBQ3KE2yXx+yt39m1BA2VlVnD0d3+9BqfSoSz+gMEhsXHenC64HDka+AseQhEc5Jz46YNPmPLx2wV/Xfw8U4r4mi5l14fr1Sk7qpT7mUfJ6KP0Axvq0V9pZpExLf1SXHP0XRsGXTmhXAc/NjoqHxAJOpVPUbE8Ky5hDAs+LKxoq6qzGilc1D2Hyr5L99lejU4s6N6a1/3BrmguW/fPBY3KcP+4L9vgqMYEdOG30q8xpcR9gVFBbdyt5dNC2wTrPq+M6N0AwXetN+2pe5BpZIz0MEUKQLc9zwSMLP1xuF4YF5yBEfgmkav2zVal2cLE83Kx7V6gEGQZDh7TKVQj0HfQPKoWLjDKjj4FbL23Wx50gAmf36H/wYMzSA7WalmOdnn08XBIL3DoN5Vwzdvc68sDPkvsywYm4z18wiZlw0BDVw9HvXFmaQDO0OX9imeArr7JC+njvPV5PeHkFfiUVs/nON5IvOMfmpMD8nrJhmFxv6YljbtnPmFv+bYnzenDQ0j81g+cLu/8q6Xyf73oymZsgPPvNbLxhWPr5We//6rsKuOkyu2fWSm3vOtk+Z+ff6g84khMacKwYDrUT2ltB9OhcIyA/Mzr7ilPevSM13eQNm5YfOrO1fLrf3K8/MA3HyhPuH4PIHZvuf1MeetfnhkpI/Mm/kQCDyyB+8o7f+3/K+/f9bTy3G95enniVaXc9xdvLm/64Eo58qTnlOd/45Fy73tvLe/88LEy//TvK//oaxjJO13u/Kt3lz958yfL4Rf/T+VpOz5S/ujXP1yu/+kfKE9a+2T5vVf9YTnyUy8ue/7zG8tnvu5F5UVPOqKpgaHPPDA/W0M8gGHxi+X1/+kWP2BT2eTt4GkCns6kCA9TXdIPpADgc6PXaUlWyFJBUXL3cpGa6xudh3IYE/F4jm0vg2mm7bB4m9EAK/JSMvzwpjzK0pXGRWwDy8iElO/65Q4jga+9KDb5yOeh3l8sJl2VsnBW875XNJVkfV1KhPiOcyowLFhQLYWHsyi05oHddjzNy0p+fK2GD0pAcVpVeWuUyxxtlQXf3kVKi7gxKnxaN1OpUJT0x5oOT70Sz0yrmmYthwyKeebFY1hovQWjFsyZ90gJCqJgvVbDskbaIfc0JqxAuk1CKUuZIdUIO2T5VxJSOuMBDa0IBSyyMk4g8zviyEs38sLMxDE/eUnY9BOsp5dpvd/D97Dj6RmPXhUUkHdLb/1AeQJCgURRHJS8qBfwVi5R/sAxqdqfBELcNE0vyokWkRyVnzy63mQP4nI/dXqXFiCRAN3LXVb6a31YZI4SHjsbxXQdvrRjILt8MUKfoX/QNxhhwydPRchZMgQGl2XXMlIOF+vuSTZ2gDar/FReHQplMWWTdUA1Zwtc1lOsSd6UyvOCLWP36L7aI8MaA5x7mjvKa11UB2hzv7gU8ZWjCMpwmmofcjIkqYMDxyyKmbhLleA64wfPDTrl5JHImO7IfRF9g/6BsaN7W88Ln3Iu3jA4cNTxEr+VE3U1AD9mKGMhKzNVaahhnJlgJEeSUgiYLj9kAMpP5tV0ecjXsIapoIDLGUqEfbfX/MxJNiKeeD0QdKEQbghF3H0uM8f87AsJE31wnMIYkqKGUJngJ66hqqxGMJI3y+pS2qSM1CbhKaeG+zLGy0wYyjT/BKrr6WZaD59p9ivvPc7EsBiRkEcsaJJ/qlEI/GWNMHxSIwDv/cTZcr22nP2J7zjktvz1txwvXzi6Xp77D4apUVD6xB2r5bP3rpb/43u1G163juL08nkbDkusn9DIxLs+ulRe8ryDnjoFHobF6vrF8i+021Pvfu6W+8o1mh71wxqNwI0bFn/2kaXyhzJynvt1c9rsZWjZo1pv8T5tTfuiZ+4vT7phuic5CU8k8CAkoA1Obv/98ge3HSu7D1xVrjy4rdz32TvK2fmvKk975lPLE68p5bPv+AP1sfVyxbf+j+W510NS51d94rby9j+4rez4zv+1PP/w58uHbr21fHrvY8vVO4+WT3x6qjzpG5bLhz/6mPLdP/zUcvVu6c4PgpMHA7KlYfHKX/7F8gYZFijOrGew0aB7ha/y/krql348XKUb+OtivCiA0VfUevFA4Mp3AHSaEqywX7sodeKYhzBKE45y07BAqbZiJHjoUA4XX/DJi92f4sA6GxaiwdoJGxdS9HH5gO996sKIxbJGK5ZZzK0RDN5ibDM7pR1pWKy9R0aFd4PSaEUuym7GReUD44B1JOy7v4bypfJ5rDCdijUZLMLGMIltc7VDlA0EKRUCko4VL04pf7tVBjtD9YYFU6LiVO5hfQeKIYLgz/KsiiMySzkhh5STBaCfjOPj7FmeNc8UnWVYZAVkwAMYefmbdFKmmX45P+EzfzO8cZiExe/he7jx9IxHTzJmU6YaHWWifKEbplGReAC3fikA0k3L8gg5kAaMlXy1v131EGyKSmCWG/FKxaA5kpH3FYluUXADuNU3+bJCT7nJk8LmQ33PxkVVfi+oz4cCngzVvqB+k30vCqk8waS/uifXIWvTdhmqKcaXyvF0KzqtQD3qYn5R2yuOMjLsilJr06cs8rjnw1EaDzP6rNdQ6V72hwT6tyDXNfWIcuE172t4QI4pK7dTlUdQHf3NGsWgQpVHMNjk22PQrzgoj53eUlYhB557Wl/BdCiMC93nflYhH/56mi5GP/53hmEop++3xAMUQIdIcn2Tb1IjnCkAjMImRGVitF7uTA0CZGO7TRQe+IGHEcqG5We0fICG8odQBUd+DXMIZHnpD+UOMFuFEm8EptatpSVfmZ7xCpC89vz1dPtw8pdpLV5pjdAYS0ucxtdmgbF2oQ/9+A/92GaQD0na39SuUGd1RMzs6NrmB12fzdZYgPw77zlV/kKK+j+TYYGB8cu/f9TrJq6vIw59AYxqvPjZ+6VLbLOxcMs7TpaP33mu7NMaCs6eOKdpS8e1APvBGBYv/U/3lSMLOw1LGeOGxZvft1je/bGz5VHiaXynKJr7+VoXMlln0bfOJPzgJLBRTnz23eUtb/lwuX9VH+Kmt2nDlu3l8OO+vjzr6V9brt+/HobFZ6ph8Qio6nDcT91W3iHDYtt3/KvyghuWy6nPv7+8/V2fKSfLTDl43aFy9uOfKgef9tyysHZC57ftKYdveFy5/qBm5uhe+W9xWxoWN/3iq8obfusWfeHkpa8vnXzJ092BMr6hT48YDjhumDzDglcQSrWn+CgfGD/b9UQHDnzo8CWfaQ7A8xD2OgwUF8IVAdjcbpb1BzHNCaUrKo3vUQ197WTK0rS+6jM6QDqKImsmvJ2sfMrIh33vw9+qlIUVjAsMCz6pCpZD8bimdcXaCIwWeM5F4+GjEOEwYKCDkbKmxZ1pWOyWcsJBehgmeXGmxk7FfRCX8P1FXHVn+hRy2aOD9PYyHWp2r6dFsZCbXaJyzUWut4ivzqEwonTltBFkMm5UZJ2RDTwn32ZeGhcGCVK1TFMtIKG+icklL+DD2+y3L2ezfNKgk3DjMK2MmtHDjeeN4/aw5BF3WbUSxFOxMKwiSnJfoaKEDaOADQa1ScYNXwtEFsSBYScy50FYaai7dvKMi8EM4eqcZjjuhbifwrBI+YeBbgMReQ+oQU/lYVSkgdHoce9wr+Hnl3UMapcVSriE4Zuwl2OQN/OmP8BoVAxc0WvTwKir7n3TVxjYbYwkqD+FnKOSwVOyHrSRShSv+8XwQZ/6AR/5PBu4r2KEkjDTu1CAuIhb9m4XpB99CcJRpnhKgZFZHbTTtbDZos3jyvz0MfgxmqgfUx89oifYmGrGdCjd4xgX5oV2rwUmPbU7tCM58hPG8kcY1WWo5wWYTM/6GbyrFwA9TCsreYBGRwfUhDdvlQfCmU4ZznNhw4/pCD7z0h8gFKr0RtJIrgmJkz7JlkXNx+vzoJe4CTKe36ePwCYvtEGGO/o9bE+zD/e8kd7iHT0xnCzYT3zDZ04Pn2njvmDoHz/xFWhY3H7PhfKkq+I9OV7tB4pfzrB4n86OeOO7T5UXfZNGAB4zXX7jLVoncWyt/N9aRL2VY83FBzT16Uc1OvGoK8Pa+fgd58qr//TEl8WweLumY936/tM61+KIDZCteJnkTSTwoCWwfld5+2++vnx41xPLs57z9PKEK7eVO2/7L+XW28+Vq5/8nPK8bzxQ7mHE4tPVsHgklGPE4h233la2P/9flRd+dZYmXXP5vvLRP/z98s6Zp5annHh3ec/ijrJrfblse8S3lRd8643lyF7p2wn+JfhbGhbsCvU6Ld62wioLxgqrHvUoSv7Cq5cqD1uUoJ0oprp4zrIgGYUeZSAMi3j48pqAlpUHT1nS450HqrJzDrbPt/DDOqZJsPA5jAvmPetFz8WDWjh4GBs2AjQaMMuOTYLHeYoT51RgXOhKBYA8HvpwhNLEKEMaBOcEx8gD9DEo8mK+dSzURnmvhoWUHMrytCTRw6A4py+Z+EyJgj4KI4ZOjFhwyJ/OwpBB4Z1uWKgquhKalZQ1KVDr+gJ6UfLBcOD07VmNVOzVotZ5GRjze+d9WvKMpkexABxclJ40ElhATBje+8tvZpipjrzEUbA61Zn2A9d/yJa2yXyCXWRIdsiwY2lESc8X7SbZlyRdjk5PI2HGaftFPlRopFy3tmTQ2l2t09Os3S3yBUdfcR9HkXZm4CbDpqN0WtnbwtLanYxDTeMeCToBU8sHT1fQ7WUa8lYz+B5hJyTfd7RJgrmY4Anl3nTNX9yP9OeL6s/mT2HuP6eJOdoPORgcZhvLybh82suVjAIxGKBlOtQWvq0w1zCw8Cc4+jEuyuA3YspwKpQJWVm3wq6+Sg40YUpX5udmDYxEkmc5qi52lKe08zLe4YV+wH0AcpDhw0B8HAiERIN6DWeAYiWndOajRrIGEPYHAD1nPH3R/MjQ0fONM0U8goHcByIuP2nBa8oiDUzznLJWJm0TDlm4KvUnMVt2DQBfAUHAiV5zmebkbNPIrZiWa8J3mE5K3h2pdC8HQ+ktb4wH52Va5WmEdmUg72miI/mblL1Zvsvp6pyySL6qhCBeSxz1RmiOZl0+VnkD1+UQVzhppQ+BrB9+n57EnWZ6Mix++Mcz+SH3ua973sYLTH59/6mPr2qqMNfBgwfHQbeM37VcyjVf4nETlzMsbnnnyXL7Z1fK//Kdh7zVa44c/OR3HS7XHrr8zku/8qaj+uB4sfxvLzzSeH7/Z5bLb7/r1JfFsGCh9q9pWha7TD1L60AmbiKBL4sETrynvObff7BMP/sflmc/5bHliFTHcvQD5fff+KFy/OqvK8/7zieUjT+/tbznY6fLwWe9uHz7V08J4Gy5+2PvKX/6nz9S5l740+Uf3xCcXNg4W4594q3ld9+1vTz3hfvLm//tx8ozf+aHy9eef0f51V+5tzzzJ7+7POHIfIHCl+q2NCxeqnMsXvv614fCasWTlwgvqvqy1HMaowIFiMPoeMLy8kSpx6Dg8lfU+kDnQZWKOcaFv9SKcx73aViglFvxUlqMWGgqkZRon5yNMUI5aF/OlxImOijr05w1obUQTKfgQc0Cag7A43Ttjc6wyAc7ZXAxXYodobw7FNNHRBceZ+qIxR4pN1bYqb/rGl9UMSpQfOCEr9Ys/l7RrjwYKTjXVbx4VxvxD+wUBoPozohfdpcKw0IjQDJIPLLC1AqVgSKzSzAzWm8xx3qLvXNlYX6fjIx9ZUZGxi6t1chRCy/GFb+0AWVywZSVlRCT+akpzh8MC+DJ1suR+oHvGgX/kZeYlUznpSxdptLTH0AgvvnLfIAZQpfiR16WQ2wrmPG8xOv97LuZlqXTRUlDJ6Uv5qhA5qdvmNp3TMuI5EJAXgpN6YxauZ9ZGRe04lxBP+SCgR7pA7rvJ/Uvb61c7xPqFioieKFsD0q+UjCCdN9ZUa7lpELQrwNwWapfuGyf4AXexY6oR5y+5bQqD6dX2vIMBySqFfw5TEaE/CsClW/dtkBiVNiwID06KDzJerAcXE/6s+8v9owPGJ4jyFJATvM0JNeDvhujJpTluiKLVsehpuTX/2ARTjs48xHg5sVxyuTe0sV9aeNCMJxzsb4mw4KduHTPU2t4g3DDI42cyAzPaWKj1guUqGH8mkYNVsQBz3QgEPUI2i0xKEPXfGQ02iVi9Vf5PVYrrmV3uaI3ng9YygpI59f6mEQtP6kkfuIYZuxnkEdiVYCu/M3wEzrXzjdmK95mOGNFt7qMp/dx+Bun1cruARUGLmGzXmMgLZpw0EJOP/EjX3mGxVl9E+iWN7S6P5gAhgWLpp+nhdo4wp+9Z6381edXyuOu21N+SKdv487qPImb3ni/1mnF1rFX6WwKtpL9vM6fOKX1FN+hKUi4N2gaFLtFfc/T9pXHXj1VPiVD4A80wgDsl2MqFGX8uz84Vu4+sV6+7b+bK4+9ko1atpV7FP+oRkaYktXfKsBP3EQCDyiB0x8qt/yHd5W1f/Cs8uynPb5cq2U65z73rvL7t366rD76KeXbv+2JZcfH3lre+t5PlpXHfIfWJD2qTG3cUz75vj8rb3v3Rnnyj/9Q+cZDKuWiZtWc+Hx555tuLeee/iPlOw+8u/z8K06V7/03LyiP2f6B8us/87HyNf/yBeVJV+wrw9YDD8jdJQBbGha/cNPLy2ted7MVa+6G/iHJQzAU7fiKz1QoXle51oCvIVagUmEAXjQ8WqGXNMq7H9ZKN54exuCmYcFTlnyUq91cTB+qSj6GBTcnhgeGRJ6KPTPNTRxfWVlA7ZO1GYVQGF78EK8PfZQU0hilwBhYQ7lHq4KmeLOxonMnKBs+rHir3J0cZKe0MBjiSylTmFijwSJwpkSFYhgjFRyiJ6aovWlhqOSoBVPMSEdWPt8DhQgFRjDsTOOF3FpvsY8Ri7l9dcRir4wSGVAatcC4yPnoNn4QCg1jVwPNi/ajLsgtjYuQMyw2xKivMlzvJCc/X4KkZxifOM7wNeyEL+InaTwgCu1XgXqcno+eBumNV3qakKEwDp8Kfo7EhUGQCnjUzbREHIU9whBLbpRBMOuvdNPAr8ZD0rTCX/Ppg/R7w5gANGLqIX1/j6bh7NLXcvoUbRRqYjUsRMN80G/hSRcLuYMj8hRPXqsPv6T1slOJUQ01I7JJPrlXPWrhfHKAq2XiV1gFjV8hgIoEhVQZiwSx0EvcR7JvOdGFDiMtwg0lXsaFDpo0D5IHBeTICTRsaFAn3Xs5cueRC/MX6VEpUEXTpZudDFU2Q1pwCly6kKvoQI/6qEz42lFHC4mva8RiQ8bFhu5947psQZsOtAZ60I10QsgkOAo80gJayZ0L/FEqZCdQljWkOFdEGk5Xp45wy2+ULgPX95OB/47SKMORUWmZh8vQBRB6PX3Cm5bRFTeST9mb0O9pjsAnncrzA9W9yVB4wF6OVtJJ8vibwm6RnrhfiYYFs4u/1M1mfu2Pj5dP372a4tGzcFs5MLfDOzN90xNm9f4bpH/fyY3yRq29uENTorJbzOnksGfrrItvenycUcGi7ZvfFmdMQPTw/M7y/CfPeyrUj2tL2xtkbODYTYrF23nitxP1c9Nv3++D99KgYbenP9VuT//mf7hSHw15n8cC8zfrdPCPaGtbTuXGcWbGEx65x4u3nTD5mUjgi5KADOI3/Xb583vnyyMf99XlUQcvlqOf+ED5q2Pz5auf9qzyzCdeWXYe/US57V3vKe9T2td8w43l0PKd5ZMf/a/l2MFvLN/3gicVm+DnV8rS3R8qb/rLfeWF33Vj2bXxqfLGn/3jsvO7n1e+ev12TeM7UL7r+59VHr1/Rmc/feluS8Pi517+svLq196sF+rwIuQJyyvRyqle+F53geKshzUKCbshMVLBlKY2LUP8gQMdH2zHC1rKO47bji/+gQt+fVmKnkcsWJ/gKUQo0TFiIcrG9boCKV2xnSuHgWleGEoINKV0eNQEo6IaOZ46oTwe+i5PMDYsZBAwcsFkC15KKDPsUoNCt0t8onOnIu7F5HxN5eul4FA6MApi6hecXRRejFAwN/u8cDdUP5THHaI9JaOB0RAWcXtqV+WXqVaUu13pfBllOgj1jkXjOoFbxsS01lnMysiYndmrBeVa0K2RC0/TEg1k5Z2iqLwc8va/0jNO3Xqjgni0LfWuOOCqwkxUIb9/QRIm7XLO9Lr8HncrvM3o9fDj5fZ0e9xRHHIGxSvhaB96z0CDOhJHCR+UavLzAhfaiZeGQdBAaQ3ZOS444g1WdKFPf6P/ud+l0a20wbCIPpKjHPQ3+t+0+oOn06lvxBqfaEP4Df7CF+Ggj7JNWR57Ex+6D21cMHUKRnDOj6AZJehKMKYAj4Hjlqbete7ABm1BmYZga51MWniqaaUPcBCmiyM/9yn3q8hi5EIZANVnRawJgRlGBmxg1/sM2Kwz/FEHTweTLKERoyBRQhCUHGx4BKzTgHMgfpo8FHX7UidXBPIKg688ytUvUGWb7v3tGDy6P5ErW89iXHjUoso+YF152Ky40CGMg19F+B8SSR4csF28h+v7eUdU8CMIjdYYKadDr6fT02+IsNDRvBxMD09ZdpX+A+H0fGQbgN/oBLX47WWllB4+wbbit/FS65TS6nnYtFyId2WPy7MvM/kIlFFqrfweaCz8lWhYcPfEW3mssg9RFGWeQ+pQ9DmHYjPHidk8jzkED8eJ3f2CVd7XuPHTsknnKYLKg6NbMPW7x40cnvk6xEx8oD/Ma8gGf+ImEviSJXDu7vKBt721vO+v7ijHz5Wy59Bjy5O/6Rnl6x93TdH5i3IbMho+VT74rneU935aI307F8pVNz61PPdbnloedblZeXq3LX3kjeU3/vAT5dTGQnnid724PO9xB8qsRv7+W9yWhsXPvuyl5T/e/JuhfKoUHqA8k6XGNgU1lONQPkJZr1OgRl7qoQyxPoIpUxgVrAng1jWOjBFu2FSqqJC/Dgp2ylOSNC1IayhiTUMsDueGpup8xbWx4qkTjC5IuVAG+dD2yz8Ni4pDpvPkr3ikIQ7Hg6CVavHJSAWGDPxSBvzCd+4G5REX5YEzKGLUUwqhRxxkeOgzzbrksKopTlw8kKCLsoiBwQgM8sNnO1kMBRZ1sxsN9bCapLIvUr6MDM61WJhf8OgFi7ltWIgnZAVtlMJ00VZISCzSbvLhjbCNJPGBkTFuWCQsvpFE0nQraaeb6qU/xgWvuv5FuhVewvd+wrtYGlMu0wj3tImnSxjyA60yXgGIZX2CdtCKvgJOGLlJP+iEEuaw8K2oizg4UEO61Jr8Vj5wivsPX5e6gvtKnjwfRgUGRTUq5HtjBOgIn1EK9xO1lXcp0wjVbo2i0SfVlBRNobU+8nUPWRmuac5phkUqx4HnvIavNPqFLtJDFjJEsojqO6HmhyyoU8ATl3WAcOSHQk4Hoh7IpF2Kk+4M8ukv+vc0rsQ3jvo/RoXqjAEf07KoLmVSBnXV9ChdQRE60bcpFbqxmD3WmRjIdAVXnXmuYeAzjj9cqgv1oX44wbFjFLzhI+8wLnSuBVOiDJeSM5tCCtwkYTr+QWZDLGQCzpBo+VSQTHcNxEeQGGAd3yQdiKHWiaZU6hJR/yb9LslyzPhm+ZmXfuNGdehlmvnjPjSzjoYHQGmNjqLJ4wOWv0V9LsHdRE6t/HEmK0+ZvJk8sw4JEyh9LUbbtYfrw1+JhkVfv0l4IoGJBL7yJbClYfH/vPQXym/85mtC+ZQseMBbKZVWE1MzUHCUWh/SNg70oveUBV6O9eKhCx5f5BMPJQYFBmPC52J0hgXwwGFI8MV+ui7MZktZFG4rYoyIQB+eKn3KCEU5lCQ4dhl6+aP8+yWmNOpBGDociucpUxrZIMO8anjVRoWU/uTZxpDou/7Vp95+oQiP0QgbHlL0w7CIKVCs3fCOUxsxPLtLuEyH8g5RMjDYMYqF2hy+x2F4GBB8IUaWG4ykyCBZl8KFwbGwb6EcOXDExsXMHhZxS+nCsEg+LI34MV8KUltlV188WilCUYu6OM/1HmAUrUQyIF+iTpqkZjjbINMynTgKYMbTJ/2Ldfl6hq+eTl/2pTRRDkkFKykMIVQX8IHxNCQChGvN7l73AABAAElEQVQ/SYw0IqCU5TWYQHEJ5De5KdyMiooHrprUMoHm/8/em0f7llR1npFvuPO9byTJZFRAEXBAHOh2akVLq5UqFVZNqL20RECge1X1qlr9R7VWIaNW29Wrl7W6lzaQibRlVWtBOaAWIqIiYKmAA6NAQiJDTi/fe3d8U/b38917nxO/3/vdm5lglj7yF/eeX8SJ2FPsiHPO3ieG400KiCcO9Un161r87/qqnagzzjNrc9hQYEnbIHMe7RqMLRuyi55lN78wwsvZKPmqHsQDPGREMPQy6gFqUZfUHTjQVmXK4B7jcCygiTag50NnBNL6GdNOJV/RFFE7CzSKXxDQV9Ox8DqKxLXcVVccC3AV+IUH1wTXInJ4BytGNRIGOMsBfJdHfoXIj3oGDckFPwG4LOtx+KimXkpGaF/c25WDIecCXg6BnyeAdKE7UUGdoZlKG2GGruBf7WiCCdMzGGgkxx6eukPDMClU6aMTcCIZ+pjImjxBhqTZ8wJoGrfOp3nW+SB7ygaNkpk0oWiQHvCA7/WV+MD28OAcFIpewQzyVEbFnXyVdX9j5Or5wes53//s+0vmM4bnOV38K+6JkYeM3K84PtPF2z3NeXqugbkGPvc1cKBj8aKXv0yOxc2YCLpppwFfToVuOrwp982c54p05beX+TDXHcllPGjKOPebR/B4AOiYNqqML0oY6Rj28W0KvcnXFCe+AUEeuBg6HFhq0Ld40CSJwSwZgTMf0Su6yISccbOUESdnAsOfaUyXcCyMH04NIxNxaEpUjizUKAs0IGQecqww0mN0QziSsbbLxMkZHQttTSn+0PDULRmJfCMDp2ldW8uuaGrTsraZvSK5L8pYYr3GBU2xwJxh+sW61lgcP3aindzQMJVGLnBGWGMxOgjUv+qMgGNATkZdDl3HvPVy7nAuAh7VlUNWOhuxM0V9SaJXRz5zYf/gDnxnDz993vTDtIB6GpVXMfountO0gOnzOJ+mVecVG0Z6hS55YSQrjbGceaQoK5xZafdBY6BHZAzDnCzazTiix0OZIHJO4zT6YS1jlWsARyIcjXB23V8FzBRBPia3hxMt2qtyJPlC9Tofa5RDyjC929DaEXEYcMAr+SNjGPrOpsBlwFk+rlfSWWxcnZMXR5RP5BsnRw0wuAUrJnmQhlr0D5x+wtBGM/qPAYwmGqINPetTzgGOhb8pgSPFH/giaecOWF0rtFvwDL705XK4yUfX6GBCD2Y6/rgO46llwJmwY5IyQcs6ExycPGVRbcK0KD4cyIsApkWVowW8hQXeuDAAs9NH0nRm/0MlMwy603nJ6VLBBDUXFLjjaZyRWoCZTs9jAruDUbJ4ToP0PCgbZClA6v8ZhMIqmeFTMlR8INmq17Ruyb8Xmfo6lRzgkC55ivdBstBXIwxUgkbJVkS6GMi5Y9EpZJ6ca2CugWtSA/fiWLy0veo1N+d3szBMw+gOgzSmAQ0PTlWfG21vyADPjRpje3hTzo1VB7AYThhVTP/w14O5gevuWqMVfrMv45uPxrGLEny5+cLDCUXMCKl7ODTLsYBGPSR4LNRDAFTz5c2wHAo7FUyV0jnBdWR6kmRmNAHnhq9v4yxgsLA4uwxBHh7+fofg+c4Gi22rrtSXt6572kaWj+axFS1yk+cdojwSw5ayMhS1reyKHIUFGSnAMEqBY8HOMxgtrKfAsVhb1ZazrK/QlCk+uOfF7OgWA0wyQHtQhmujylrdTCuRbIJDPowu6olTgo6iSaKtSA+h0iLTh8IZeKHUDGNKpdBWfrUDIG6jCSaRl+gzI2iWKME7zmbTgkQvRdB3TspJuvqEs/Rj21Q/LtN5lZNRafgVHacTLuoXdQ3ugjNN4jCCg45oiQeGbkz760Ys1P/C0Y5riD7GzmZb6m9b2mCABZBcD6fVV45pk4JVXRPemUx90n0dxlJL6Sn4hZGOLEOgPjgDyhjexpM2TMk7yl8OUMDH5grIiWHP1ehDaRGDiI6Bk2Xp28slM/rEgIH+cRRSXnA9YqF6e2G2KHIPoN8CE04F946p6U6C4VoADo2Eg5CORdIeeE4linfhTDoWoz4h43uNrlk2UuB6Qg62oGWXNzsxyatTyUTddGLuxXNClKlrBF30cG5n8oLChN6LTvUFYCpNmXFcgS43ZYm+TFMeIFsxgG4nZ2B0hUmjy3Fyui59eU+jpCv4kqmHv7f0BA6ydjJVWdWhYmgOcpQelFfyFM/Cr/OrYhEpnFr/1vOYhofn3LGY1sr8fK6BuQauNQ3cu2Nx802+GXNDZCSg1lRwy/TNsn+w6CbsP8WYWWW8ei2B3rrb2AaPP8FgXOFU8C0JDHvfqHV3hQdrEdgRh9EKf6BOhhQ3XmAwuqDPTZv3mITgjGMQ07TCsB9u635S9PjwHJ0KtsXtHYvYDWqBL2/jAMiYw8nQU7RtaerULtvYyllg/SXf7/DOVZK1dyw8RULwjD5gIHLAwXpU3RbkHOBMrOjtM9/fiN1/Dttgq7Um1NsLtlfW7Vzw/YrYCYo59+Knc+qJ88Hhed+dPqwYaQZZWOTtReU4FelQVAwcBhshHnxoipP8ydPwMAPGcMoHpIqBHh62KqAtCMAEXZ9O/Azwyu3TBTSN159XmhjcWfjOL2LuJXFS8MZRR65zOxjQ4o+TDMP0J58nLxWHs2hFub8HvyinTP9uU2KCR0V0Auky2j1qof7BtRBTA3VNqN/gkJ7TdskcF7S4cEFtd1KO6PF0LJaYKqh1PEyvq7akGX1lhEipk2JOW4VsyBn8Vaa0IZyHbHEt9iMrwFAfXzf0ZaWx2/WuXs69rkedD1t+RlWHX9qnjsqstqtzx6LhkYVyUlSTWGfBegamQQad4drGQZMT7rUNSlMn0x34SUCl6YC1FS8OQx+m5Rj6Ac6TnRxi0Tb9cKisL/GCtkctdC0yVZHANrgXtNc/X+eG1r4hy0rvM+GQXWGoVwJl05p/thwNO4QJeHSCAhwGaOOSW2h2hHQ+Sx+Bu88vuu7pdHUurmCWLqbp91QNk3XucXv8okPeBC3hDWXUudNd8SCvYIgrXddOwVV+nROTVzSn8/c7n7gekIf/lKvHIW2eKps7FtOamZ/PNTDXwLWmgQMdix97mUYs5FjUTZUbMG9HHfJG6bulMuKRlQ8ulfFg4K0h8INjIcOIABQ0Jx0L5aUhxxZyGOnsnIRj4RELGdEQ1SPejkUYaBCLR2OIowc9/ITLl8L9wDBHeAIaPMo4Iq6do2puNM4Q+F4sK57LGP4rbO+qj+6o7LyMhk0dO7sXPFpyVJbcooweFpnXaAUjG3qdacdL22NrtykZZeKPPBjbngNuGRkVgZfwbSTGtKtFTY9alDODU3H06JKcgtzPP59ULN5lxAK4BY1m4KTgZOBAjEaEGOaDbHAscqtc64U2QmmAJVydR6Z/B3rx4BsfroYtqyTIBELlmW7woHikzVm0fyCMv0bN9qzcEa9yRlrTZZZxBHPKbU4q6YZ44u8s/fLvI2QqZwCI6o+BjlHZ4WUdQi9onb8I5ilY+mrxLx4DPyViyhPGMSN32RedZrSCUa7L7aycirt3L3rkAhNxXaNn6zJiV+Xori8cbqv0G49WRd/neguHnvNJuVI6y2RjWUa2Y1cQPcS1FVPDKMOZCBhfedRJ5xdSRvRxVJ3nKA6GKo9zEQqNPqVft/vQTvQ5QVhP2fesD/34tNpBRrwbRfl2lnV9OYaHrx/VUX0dGJyK2JFpcqQjCIqfr0XFA+0RjmoPsiG55Q+H76oRCzsWtCcODAd1hT6yyLFXm3iHNvFj+1lvQ6tRx8Hbgr8x9KNAun6hQ4jfUY6ow3g+yFqAYKTMJtD/ZD44gMdp9F/AyK/6TqKN1zj59wbT06n6Rd06qvvJKJCqE3wq3WEO/KvsIHnM9wBes+iSZw0NOu2hJtOzeE9CzDgLoVQgBjN4TNdrvnh7hg7nWXMNzDVwTWngXhyLl7RXesQiHgCehqAHuwMPLFsDccojy/dQ8vhXHIa0DHXeqMsQYMoOgRs0f16grAe2jXu9kS1DDqeArTaZGoTBHo4F6wn0MBS+pyIJD8OHt72i6IdDrc2AH47BOLoiY0E8bSjBTwYbTgXGG2kfOsf4YGoTDgLrKhitYERhZUUjFpJDlWrndnfb+R3tBbyzZ2OKt8jMd0dW8Nj5Cmn4Z70EOzoxjQNZWOOA0XePh3rELPVEfRdFA17r2k52XR/EW9eXttlilvfB1PcSb2b9teEr4XRJtiUt9l72egs5F3IyMGwwvIZ2cRvQDlrsK9mu+uYFjaEAvP4Dj0SGLomKLS9FE/k+H3EMB5CC6VbcIdUDOuLgDTws/BuJkMd5kz9Vv4qrtOhOn/f52UvNyzYIfVH8gCGut/U+hxCyUKa/cDpIBTz5OA+qQcgKgSrL2M6FM/lJPMfRJz2tqHMswunVBxdlMJ/VyNidcmDvlGOxeRH3NL7kvqy+sqaRilNyuo/LMV1WX8dJp4HoZ4xgsE0y/YqWqfZVV3T/4yoKx1zXkIxkAujUz9ei+hv1d6tiRPsaC2Of+rC9ovuk4iNqV5xrnItDlkGE9B/NHXpxO2X7Wx4Ycm5Q/fTBggR/ALh2MNzZejbqERRY/0R9eSFwWQZ8P4UKckPfEB+Mf5/DL+vT94mCjT4g3UiGaccCB0xMsg+UfGYk2nqBwvXPyIViT4mSc8EH9KCjH0SaCAN/K9k/oe+EqvIes+ScILTfCType19OvfIcWsWjQFzW4Q3nBZAxeNOyDHQFU2mDQ++AMC1HT7eXr/L7PMjW+TSdA1i6qPA4GbQ0oayrKfQ4V5cekJM6nQUxXa+5YzFLS/O8uQbmGriWNHCgY/Gil73EIxY8KbgB1sHTipvxcK4a88hiNyOXJWw5FjgUOBacV+AmjSGDYcU0Ifae5hxedixkOOAcYLD7kGFfoyWDAQReGkEeacCgwqkoQ1/4YchDO94Mw++ijDSmQV1k2sKEY6HF1aJhx0I02LmJr1+v4lhIDvwBnIrz2zgWuzaqGGnAAUFG6iiGHp24wkNKaaYoMbrA1CX0dY/qidLZCQtjBUMNvCVNb+EL29efOtVOHDupfa83/F2NPX3Zd0s7zly8pOkVkldIMcKhhd5sObu6uq5RlRWPXngxd+k5HYyYvhb6dztgZHHQVuhbodqRLJ04L/KjjDZxyKIOJLK7jHr4QjPM2pF+UrExUHCVR7/pQz3sLWdfkOlZ+T3NWWnnJR+qlNVPI5s+gnrLaEwjLPskZ/Q7aNSBKLRfyToQTBkNB59BgclTfOBvp0L8iNmhhb5sh9cjFRfap3cutFu3duxYbKnfcP3grC6oX23Iabhx6Wg7LediRek94V+UDUv/XaI/6bCjq/pSZZzyBY0ELqicLarJQy7rJNM4+r4emAKkPD5q5VcB1kHtrBR6kjh2MGgHRitwLjwdypipE5VR3rfVkIbmrGBeKnO5Yvox9w9dX3X/wNHhA5LQQn47Fro2qi6QHfjEiWlEnuSXrmJq06SBXPjEPnyNxjTNmhZVbVkwUQXVEX3nlCgcDUZSWG9R37dwi7tOFm6QFS30stIuBNOPpH97GMoOCqYh3RAKzxjgFS7llRbcUB5I/DqYV8JBaxbvmflTsFWvotvHRbNkLXr75Rduldd5HxetPo904Tju1dgJOAu38Kbp7XdedZhZXnqXjsw22wIez/nHz5mJ8kBkzneFeiC0Oqc518BcAwc6FpNToeIh5Ztu3hBJx01YD5y4QwpI/xgZhpFxqbicihEexesBL0MGo8pvSXVT5X7LrRbjiDevZeA7xkkQXSB42+ovVYOrA7rg1PSpcETiOxGUYUTU17djdCRGKS5oUSxOTU2Jwu2xYyFngClK/jCfRgLYCpY1FtDaZLRie9uOBTs8MV2LL3/j0GCw41zhVJA+LJlZ+I1TwWiCP6Snt6sX9CaamzqBei7JcVnfONZOnjjZTm2c0IJujUCIHvC7cizYEveCtrL0g0A4LMLmGxY4FGtyLFb0de7aJQo+6Ps6jY5UO9QuVZFfbWb2/ol2Cd2TJjimHvrjP/Iijt/uqSycBBkAwC/HInBHiP0e0lBM9sHX5yPeQHyfBDyhPU2/zh2LXElOf+OM/qT/jDEdg8YEnoANY/qy4I3paPiZlhR8HI9gE3HNu4Y3TkytqSBdo2hMgfqEHIqPbu60D5xTX9M5hjzrjnaRUxxX5SQ8Zvlou1GOxbLyz6kv7wqIETfOmS4HbSTlGsTwX1N/3FC/XtfBOZfToJOUB97IgWuBc8KHn/gMpKzxOHjjr9bhuMJkQNHwNS4ezqViwOsMfVgnlOlwqJgTw0a2f+ucWIdPoc+oha6HmOonMSQD0wPtaGDE6xrBkK/RF2iZGwSSH85JXQ92HqVv4pQq9JCioBMfqjPXXL0E8OiD6xZ6s+6AhZ/4xMhKrH2CL6MobEMba0DCWUsWY9TrY8wdUsVj0J9K4FeB8r7M+VnvIT95FK3CdXnKbzpZMNCvuhXC/YzN737iAI5cM+ulslk0K6/qWzG0ZtEp+IqB60OPTz5ws+j0OKR7vKJdcV8mYkYtPY99sLXn/uDcsZjW6/x8roG5Bq4tDRzoWLz4FS/ziAUOQJgSMhBkjfhPN3/fLOuhRZx3SGAwKinnwCDA+Kg3jqEiGT0YdHq416gDN1rgvcsRhrmM7nBKlActLCGYCM9f63Ys2QQb6w40yiBDH8cCRyAMaT0YgMNA0JQTjA87FzKg7FgwciEDABlkeuRoSTgl0GFLWEYtSEsEORWMVmy3LY1YYPAwOsKWsciMfBhyTH/ydCyNsrBGgx2jqD87Q+3t8d0MTZGQTN7xSbjrx461E8dPaKRCToXWTOAgXSc5GVG5CI5jRjnibTp6XNQe+qsasVjT1Cm+f+GP5aUDEzqTztG7jcPQBXjRZtEC/EqrzgvN6lf/1U6GBYaKE4YojClnZV6kxxNweseiyutBy/l0GB60IoNchIH3NPC9nPd8Kk1sHilm5dMPpe7BgLAzkMZEwRhXeTGqNta/xBhrXjlJTxzh6UP4o2OhfJ3HAunoz/TLXbX1ltZVfPjcVvvg+a32PjkWOAj0fxyGTcm5p/Ml2fSPX15oj5BjgZNwp3C21L+ZBrVMXxT8jvoLBzIzynFSDsX1gr9BeCu56Dsk4zeuxx2uCx18LZ4pV4usd0JoyYahzLUn4j5ihDL6FE62aYlXGU7oxEfCoxm3J+cEwZZ+IyN/kwbXCAQYAXBfxnnXOThce54mpfraqZDeGFVwCLQJkh6lo//r4EVDORfIDLv4CdogDlOhRLdGlsqxsMwpey+/eXC98X0LHCEJ669y62UCH9GDzVWhdHFVQWQUr/466OkUf2s0lDNS0nnhGSdlRonT8EWn9FBEyC8alXdf44HmfUWYAdfz3o9e5RdsxZCbKHNDKy/5VNkMtnX3gYJpgDPobBYC5dmWPd0+XeXouGSY1vdzf/C5+1D/q8/2i6qUeZCtY0Me8nOtcPxVfcfiX/2Xn2u/dMsfdJzmybkGPvc08G0P/dL2wsd+2zVZseuvv17PMU3//wzDgY7FS37i5d5u9ooMDW6API58A9INB+OTB+dwgyStOy83X2D6w28cZdj4RlWCcnPlpiUDxg95ESqc3rFQZhAVbATf3o0LPrnQZ+tN3vKjDJwK5mDbkFa5+aRTQRonwuss5FTseStPOR2S44gMGOamM/pwRDTYFYovYq/KsYAu8m1u6Q2yRiy2tNYiHAs+WKYyDBaJZltIDhFrQ5aFx2gHIy6EHTkU8UG+C9blotZQrK9vtNOnT7djbCerkQe7JtzIJZudIW05e0kye2QHuXUgB1vOruNYaMeoxUU5I3IqWN+BTDgsNd+bRd3UxzotXTqmNVFttpUljDYoBwQ+s4Kz00Kehqlz6A79Q/JXPvRoA0Kfxzm5xbEv69OG2w+/yy8ePTx9rXoR+Q7KkEolE2f5ENWJ/4iTJvE4FQoHL9Ctwxl6Ap6/oBp185k6CFcNaYkj3kGXa4CvtLO24u7dvfa+s1vt/ec3219o5EL7kclZOKQpT4fbWY1KbIv2ghg/YWWxPVx9lClIt8mx2FQfXxTMomBFup3T+R2ityOngN2bHibYRy0ttMeuLbUT9E85F9epHakL8lC/PV3rHMiyJEdkSX1qARgcCw7BuD2i4kpjrPPiAMeCALFQqEGyf1XL+sWE9IVTYh2JT9GsdubcFxIx9ATL9eUF3IotcObhRHtak5wKT3ECRSEWkkfa54LXxWE6nAfvqI8dGPEyX6MoLbnCoGLEAhnzSHmNb/mCltGQSXpgmhZrLRgpxNnBqYj1FiFcimgUfqrepEcZOMsgPgVDeaUptaNLYiqfLAEa1vx6WTNfSMGvBOphjE4L3rcwLXcvY8+f/B62T/ecevxpnILbL3/fcuon/gT4Fu+eF7JW3+Eeho4I/EYq2msCJ/VmwBk/xYeikaJOkKHgk8bznv28ynnA478ux+KFv/t/t9d+8C0PeP3mDOYa+OvUwDMf9tXtR5/wjL9OET5j3jfeeKNt38+UwIGOxUt/4hX+8nbMSe5uxDyg9QBlapJv0/oZbpA8tCSNDQGlYxckHujxkLOgAvaNXVjxZjBpAy/aGOJMSYIGQY/4eAiMTJzPDzd4pv/gSOBY4BB4pEL8PMqRkBgPPIQ9YiHjIBwLto2NtQ6QxrGIaVdyUHAqZBzUiAWOCzCMWDBasa2Rh4LFicCp8LQW8WAKlBd+a0rTikYkPE1Kcvor2jIyiDH+2fVpY5XvU6yZl0SWQRPrKLB20SMPQkZAxm9+XLaOlrRb1KpGKhZ4O4qeABacR0rksECbxd2LGgHBwIm2ijZAZ7SLIuuPtgSflsMZKMcC9qbrRJSPeRjIUV4P2YojG16GNlAl+4esCQxtKvqJUHFhm16W9fjTcH1Z4VYebEjz54ZMANeCLPcNlUrv4UBEnyvxjFswNjKDAPqaDiPPwMbIr+Ay0SGIVfDNvsk0pHOa9vaprd324c3t9jE5Fbdr2py+xa4RihjJu1POwpbgF9VkT2DEQn2PNQ53yUHeVhl9nmvngljcqY9ffEr07lafYzvlh8iReLRwvnhtuT2MNT3q4xqQsE5KZhyKi6LPAm1GQrzjExAykGvLVvRu3VN196O8VlMXjLahY4qtH6oLjsHjXtA7FhjwA03BSCnWCzQIQVb9UveFuCdgnIqHCmLDAqGIBoaSGSfvSgeRIBR86O9UXPcD4cC/eEY7Z9snTd+jyrGg7ZAPTpVWXDKSQMa4J8ULB3TPV7mRz04Q+gTHgoVclayYsqx+wAaDEScBq90KbyJOHNNBVgW3WwK5FiOTzM165VnBF586L+CetuuUPCkfYJVn/JShcCsu2nU+HUNnoDVVOAu38nqcg/J6kgOcMqPHZlv3QJR1MoFTeD1Y8Z9VNp1X5z88dyx6Fc7Tcw1csxqYOxZ7WgO8OtF+emlzzz0v/fFXtJte+5owttL44QYYbynDCPVzxD/jg7BuumWk1jQh32j1JOJW7WeMH9Zh8POA4mGKUVtv2HlAe0RDcMR1q/cNXzyhDwwGfhzsHoPDE4ZyORbFF4PRb/71gI+F24wKpNMi7nYU9IYW/uxExfoIHAScC4w18HEsdrTdLCMdfBcCWJwZ6oRjwYgCTg5Tk/j4Hd+qYHoW07qogadwqaLse89Iw+ICoxTSiZwNPqzFXHZJb34x8kKd5NRIQR5pUTn1P6KdnhZyG1rVIPQjQ3CRxeZaBH5s/URb0fqLJfFgFCPaBENP+rExKCrorw7hDk6gBIIHwbpzwqeZb3PCGUIPmIoDLOBMp0OWnPwPQeXUq4JOB3kq76B4kC2B6uHc45BXLJzuGDo/FgfZMCiHwlOjsm8CU3QDnzfZ9N+g6r7WM+zhnR/8exkiGxo4F9H/6ZfnNepwm5zWj5zf9qLtO+S80uZLUgx9nfUVd2rEYkeIa2qvJ6wstEdrahM7RG3KidgVLAGXFOfjNq3mvlV99XY5F4yEHBPOoxaPtCfJsXicDqZFsfuaLnBhISF9OBwe+rKvX41W4FJ7tEL0MYzJd8i+ow4mAaOPkW/Hglh/9OUI0aO4Xt1ukqXaw4Z9QRVt8UcicyJBGsci+y4fjzR95dEG0GKhNHUYcYSYuEEAIsKCBnLotByLGJEQT9XPI1u0C2kcKveFqLtlNjH9IGMeIfaoF5wgrwPR/YEArUtaCxKjPtK38IZQdR4yot4FAQ+Cr9UOhqRLelpT5QNM5lvC5Fd0B5SOTpXVNVbnwFYe6Z4/MH2Z0z2vjj64fejp9/mki2bFffk03vQ5OLPyoDGdP50XPTb6Yc+TNHRLHujMojWNU+ezYCvvweBYfOjcJ3WfO1vqmMdzDXxOauD0wnp71Mrpa7JuzNSp+9tnUoGDRyxe8fL26te+xg9BjC49a30D5VnBTXe4uebzVBmDDJTxlhxjNeb8h0Hhm7AeRzxjfDPNmzIPKAwWDHg7FmlEYIhjdPGwL55h8AR/jAros6C5RipECAHj7a1oloIw4vgI2YUcNWCRKnIQoAkNOxVyFBiFsGOB8cU0I/0xB54pUBdkqF2SIcf6itpKF/lGxwJ8zWOXY7Eqx8K7RomeF3dSLzkKR5WPMcYc+62tzbardRss9MQQ8yJ0OSYrK1qUzRoNTWciWBeSwWtFFGOskPbiUun0sKaurGqb2hMnTrVTJx6i3azW5SAtqv6imnpAX+FchI6ga71LLhwQysdWjAcoMKVD0hFCcdXOzoOk+BAqdto57kaZygjQagBrWOfQ4C/pAFkPXdKVXzF5fehhya9zpCVtM0Enlj6qYHSX6Zx2DAcjDMsBf+inoiC9l9jW1wxZk1PQ7uUoRPIwXnVOH6Av3r69pwXb2+09Zzfbp7SdMSMY6xpSYLQCg/829bm76MPCPSVn4gkrR9tjNfJwg6Y3McKwR38QvT3BnhPCX+5dbh/VtKpPyrk4qz6/pLLrNQTxRZpC9cUbK+2RfGhPfSamfoSxi3j61+HG0a9SyqTO6nDWISUc/LgdtFGALj5OyDV8pADhLwKx4dXHfI1CtzuAmm7XnHGnEsmh/hmOiZx0GfyVx3VPg/CxvKFhqIfyHBQPdCXE4FjoumAkxg6E2h14OxLUU0c4FhHfI0cm2j1pBuUkP+aRglfxYDcrO0TSwkXtEuWF3GrrQU4odP3HBKd+hnoA2pf1eFXXvjzTE/jCGaUNgAmayurhi1yfZ10mkmmp780M6IEC5Ez5oMNR7UHc0y4603kFX+UVT8NV/qy450t5j1v0+7xZNCqv4Hs6PX1r5IA2KToVF98Hg2NRdZ7Hcw3MNfC5qYEDHYsXv1yLt3/2Zt2BVXkd4+NDDwZlcXOtN5AYUgR+eY7wYLURi+GPMYsxkQ8RvxHMB4wsq8ARRWDYapYpTTYWBM/6gnBq4k0pMCwUNQ/hIEO8aQ/8kIk0+TqQR39BI0YUvHZBhgNGWMgbsPH9i5gC5RELTRXxInDxxIBnsfe2DLWLijEyGJlATmSCDu99ZZJ45AOnZEVTodb40J2mQy3pYBcnvk3BaAWG4u4FfRNje7OdP39ejsWO32hSFxyRVTkVx7T+gh2foIXMe3JoduXY7GqtxkUWhDJ6IXimXjFqsryy1jY2jsuxOK3pVRserTjCh/MkH8G6CcVZJyjR+lQdGK0JvdKyYwCHUHGVRHboP+iiAQW3/RSNKIlC/VZfqewRGnopJ4Qy8NA1TjRW1KMKM+7lq4d0gRQ+6C5TQiSHMJQ7X5yyzzmfNIYmMnDwp1hRyhoOM/wjG8LA+Nexq0IGJRElTDgVrGnY3LvYPqzF2h/Q8V4t3L5bU6D4LsQpte2GDpyJj+5damfot+J1WtfIl6wutMevLspBkJMqWPozB9OgzunLjJ/QCMgHt3fbxzXycVZG94LKjqsrPEzOxZPWV9pjV5faQ7RlbazrQeCSkVqif/1mnsvS+I6WCVjrnR3IJKO8hWg1wfUBeK5B40UD27EIOrBNnYJU/ArehOClvqp7g50L9fnBsVCa0QHCMKrS1aPoDf1D/En7nOtCsL43MDJBOxOn0+4pTG77eLHh6Ve9rOYaP6GNSLu+1FP0Y9F57ArH6MhljVowclFyGQPYCiU755lPqfsceSqvulQMb8oHKuAVnYxBvSoU3ymYogt88a2YPJcbN/p58erxgNsvWNbiLaCeNjjT5+QV7dJFr++CLxjgp0PBTOffl/PCPYg+dArONLMNJvJUMItGDzN3LKy9+c9cA3MNXMMaONCxeNHL48vbUb80DHh88RBTxE1ywrHIhwU3/yqz4aoHbKyxYDpRGGmDscaL0oRnuhBv62PEInC8a474cfOFFgdGsHnot27UnBPMV3J4Ryk5F+Tz3GTUwwe06g2bCpE/jOsYKfGIBdObfGBwh9PAmgwWXvPFbUYuIAoPHAHLoNizapg6IiOLr2LjWKzLQVjTgYPA1rAs2GZqE9Opzm+eb3efO9M2Nze94wbOC/zZhWpNH77bWFvTWgm+oaE30nIkdvZ22rYWjuNYIAM64cN90FwWfUYrNtaPa0H4cTkVWnCu0QrmetsgsyZQkJVkmb3QW7q0U8FoBe2ZbSgohzqvmExA4nyENxr5BqifePwbNi0B0moBACaC8YQ9QVsQwNJ+7iXEAghY8sYAXdMO4LFAKfQUf5GmsGzfeKgHj4ALGIxNnyuuN9emMdDHkINntH/wDrbAITSgxim9ID/5AWaDlhG0bTmqjFb8+dnz7b06PqA1FkxdYgoUi7NPqY0x1T+o71rcIfhLqutpOZJftrbYnri63D5fDsIhEbXMgsNp3VR/v12OyIc18vFJOSmsv7hOb92Piv+KqD1OOI8R/iM1euEP26EQ4Xl0AjGRUbCjrpWD7OQRU67fqL90gGMhXbiglGsYaLrZrC8TjEZ2KXojlK7hQZjVyl5jgXMhR8Y7VAErfK7fanu+9VLXNzQHeuYZlMEBnnsSnIBzG3NN8RFKxTgV5HHP8OgF9y1aQbCmC6kuhNSR4RolD3TCeqrDNeqo6/iSXij0siGD9QI6MhPxAw3iDIVTOquYfOB7WFBmyZmkIhJ9h8Qn7ZwZfAdaVQbulC5KniAavwNeZt4bzDR80So85APGcJKhhy+Ywqm4h6m8+xMX/n70oVUwA92STbJWmInf6144z3sQ7ApV+pjHcw3MNfC5qYGDHQt9IM9f3lbdecRxU+fmyE2U2M5CPtStHvJSTy7XeTkDtXgb3DDUFHNT9Y1VeMCytaUMYU8xwnlgCggjGgnH6ACjHx4lEDyh5MGYwABAUkYrbCyzMlXn4HtvfhkMGAqEooMDwTcrwpFQWvwpg3eNhCAvH9RjtGJXRhpvHqlp1BEW4oOBKRySGBKMMizrTTLOwcb6uuJj+pr2mvgcleF3pZ3bPNfOnTvbzp69u217tEIGjWiynoO1GWwlu6yF18x1wzFghGJHU6V2djVlihETsUVmYFbkhOBUMEoRi7Y1KoIxw+40ir3GQrJZFxYXXeNQpAOltJ0K8Xco3SYsedS1gqtrg9olY5lAOjAVxkN1Ejf0RptMB2QgoNshKNmDdiVu16Jd8YDXJaKPqX8or/i6v6QILqePZXnAM/MnnGAv3q101cn0qUsYtZYZ4UTEZKFnmoqVP9RpyA9HAMfinPrUrZoC9a4z59ufaRrURzQqxQjVhvTxGC22vlF9AsW+Xw7Hp+Tg7onNcTkWT9Y6CdZLMPLAeqJBUUpe0PVwVh+C/JQc4TPqL2xRS99nbRF9mI/rPVKjHZ+vgzUU1+FsCyarMNSBapIXFcs6Ca4CercO1Ee5BtwBKC8YaAJjBP1GwnB9m1W7AFYgCDGkyaefMh1KsUcWzEP0ybMccqpyNLGnZ5pFCD1xQM843B8QV/rBKZHew7HIKVLS2bDOQgIVXeJKQ2fQiPKrXo5VFqMWXMeHde/Qui69oAiaMOY/sEtECxcCDslKwPMquCzsZSj4fePUQZUPdakMxeZD3ahr5lcep8ZJPZQurQvlVXnRdYZ+Sjd1XnD9+X7pwrU8yaOHrfI+j3TB9+Xk1TnS7qfTotXDO68UAv3UzjQM5xWK1ySuyqfaYe5YlMbm8VwDcw1cqxo40LHgA3mvfM1N3JmzfmUgxAOCm6XfhpdxlXdnoClzuYz0eqMIEW625VD4bWDelgsWh4DFzh6VEG4Yd3HzrnUUHilQWQXT9BtGGQXKxECtt/DQxZW4pAc6b/m9M4tkwHlgdABDfpF1FOaZRrjg/VBQfagxab4nseNvUMixkLFhTVBf0Q+e1FO5MrA8YiGHgO1mj29stOPHjys+rhEETWnSVIudXX0LQ6MVm1pbsbW15W9boBP0VNOmmA5lmeQcQA9WNgzl4FAHAqMRjGisaArUCjtL6WvcODW0CQvmWRsS2+9i1MQH/BihwTHzdrQ4FkoPTkXW18T7tDKqPSsGRlXnlx+Xk6S8gjUzAWPIxCuoiE3XHQeKI41JqPGMNul5UTJ9Tp7bkdhHGoM6IV//ke90nvvNdJWTN46wuS8Kw7JCXHLWiFXPG7pQNg+SCs5yQvkui0wWZ2/KafyUdoDCqWAa1C2avnRB7byu/vQ4jSgw1YkvZn9Mju2n+V6FnAC2m/0iORVfoBGLz0vHAtVHb4wpUXu6JjblXGzLGWEq1UWd76rvbIvfMTnTpzQacr0cjEOqo6c8SfDSfOgtFYTcBPSkq6nqQjtVvelXdq4tRFwzVrDQoBlwFhBKoZBiFjnxC34o0DDGzXI7EOrbOBcURhtKdvEOB0/OmupHO+km01OdTJto9DLwCNSM6wrDn+21azpUfBxP5647+gi6fdtWNVxSsotm1dnySebDcgbRBk4RI5DhHAGHAEjQhaRT+h1Kev0MmZEofLed8Cfk6mCvoqmywgWs8DqUmcnQf2DOpJl1AG5WuXmpPpZ3JofZmfvBT/PgvJdxNrWu7ilvwfX0ruJZCktl+bSrZ/GGVo9rmoVLofBL32TPHQuUMg9zDcw1cC1r4GDHQlOhXn3zTcONkRtfb/TZIPWDvh7s3V0SWN3YhxELHogK3GTLsXDcPdKAxdBlm1cbvDr3DjzGxJDmLXtOhSp6pslbRdHFoBA9SWNYjzwIzo4FxpWMBi/YFhy84LOixdE4Fws6eINfDwSmJfGop77EKGpXjoW/mo0hRr6qBAQx6xPK4WG9A2sq1lZX9OG74+2YvqrNNCiMlF1NZzovh2JHoxSsl9hjzYZoY1247nIo+FJ37QiF88N0p5JNDC0PBhFvQJc0YsHBdChoICvtRN0XNP2CxduLOBxK43R4lEL1PnpE58L3zlmpS7duNFNonLQq53YUTI3iRGH0BtKUc1RaKuHfP5mbGQVr0OEnYEUDkgpFK86u/qUPBbugXhDTeNaE9RU6qwc8sZrQenIe54KjPxqSc6XsSNixiGkx9DHKq76O6W1d/UuWCV7G4gd8c+DHp4xesUj7jPrBR/Sl7Y/qOyl/qWlRfExxSXQfqRGLh6mP8nE8tpTl4EN47LxEPtvG3qDjCE5t1MgcqAvy40ywsBvRLyovnItL2q5WH9zTAvA1rl8rIwWyhJBIBdW5BUZsrqYK1e7Eajf1OZwLn6QOqWToSTjRaIGMKiabbywXjoMiQAps0rEIWTxKKZ7w4H5UIwHOTzJBbOoXecmyTMHhsqZBeR2ErscatcCZZwcq6mFyxClfnU9RHsqh7T9iHYd0XXPNksax8AiJ71lQ6ISt+idh4IdAeqp8KFMCKtWnyQezo0yWwwTNzCu4jluWzI7gY5yUp6dZOpqNOeYWzjT8LFnMbwavkRrNebX007R7eNLFa1qvPa0JGoP+xUv/vQ6gBx5H4VRcZSNDgHUMAsixePbn/gfy0MM8zDUw18DnrgYOdCxYvP3q19wsGyPe2qKGugdyP/QNVMYEb8gx1JXhvFIX5eRjwPpGC74f1PFwzsdSgMuA5Y0/sP4ehd7SgxtOiJl5Wg9rMPjKNjdkm/71cEvayIWB4WlMipEJmXEoqCxGPMYCMLzNZ9oRX8dmFyemDYGCAWkHRLShR2CNBYuncSy8TkSAV1RofSiNTDbU9WaSD+vhVDAF6pR2aFrVVCVGCDa3znux9qYWbDOd6RKH5IEfRjtGPzRwMBxLHuTCSbGzpXPrUzxiSgjb4sqoVP5h4TOq4jesktuOhRwKOx1aNM7XgPloF2svcDiQx+0n2fmripIXoVoHXWY7qo7AUud6WA40TAd1Bz5RpU08yZJPOxOgYXylDVudq84ByuAiwTtAuwQugMqDtMBKPsBIgxmOZyCELZ39kPKEKT4FDw7GahmsYNMGlps6p1GLPL1MQS/4Op2yk65Q+WxQwBevcS7u0JqIOzV9iTUWOA8nNLJwWk7vstqY6U187I6vadOH13SNrMuJXVe7MusvPnYX12qNtJif6F/R9WVnQ5qgv9EEHIeRJx0LYJUVwfkhK20WdCgKXQUQ9VdhHtZJ6sbwATToBW0MOjJacDNfaGQYcMXepJ0fvDwdStcHATjaBZrVDlFv6UD6KToDT3D0VyE4Igh8NO1Suh1HLWJkMNqdO43Bop+Ib9GGFmnrIU4ChrSCdZJ1Y/tZdoli9BCngg/ncVwVkv5AswMgb6yBCgQrJsEfPGfpl3xClkcy83RiudRWQyh4ZK20Cques2QZqCV8wQ40u4TlLrjMR//TdIsGfZXy6cD1WDg9zcqbhue8aE6X7YdT8H155U3T62UApnAqLp69rlyv7BOla6tGBfPF26WxeTzXwFwD16oGDnQsXlKOhe563DTjsaUbddY2HgwysmShxHC/cnjI6a+M0XIsQLGTIKMGw9w31Lq5qsxYOscgLsOaNReUQANjlLf37MTElChosP7ikgwLeHEjJxa4adUj03ILzh/+krHGFBM/nETTjoWmK+FYMAUJ2tCxAeY7vcn5/ILeMO7K8Ct86ollFm9pkVEHoyn6vsTaynI7oVGKUydPysFYM8yuFmuf2zqndRrbck72ZPhImzrqBbDrKIOpppZBG+diUU4A06KYVkW61kyUYwEM9fHXffkIl/SBbllfwXcylrRWYyFHNBi5WBIdRiuGEZDUV7VqtTHt5baU4UybWD7qPBFC7+iMoijNdMLGAzby3DbKL8eiSBVeyUC+MIqgzwo2+k2cGSYL4DP0S/fVKHBenrsHZ7sSVRvYURQ45cjiMsW9U0G6jAt0MdaLukk/4j/IIyJBQxSVqCMkoqwkjTQ7OTElakfTlnbUz7bkcF6QkSsBPbKwpDY+Kp70SybBeTczpY+IJ/lH0amVqGsLd1tlyAt+ORiMPFE/guuppDUmWAFNyMS59WDgNPCMyk/VR0nqnAfpiXNwhzBQG3OA3y8gkwP0OyDz4hrhOrXYEhWNhO65D1k+1Z2Rh17PA5UiPWQYXfQY7Qg8RhPYFrafBoVCTD111dMO1aSeoKsM54EwBPq9+omuS5x6iF2+zEJufdtiqG8CJ48BNRPR56ZzdY5eMhu+ptfRBK94VAyO75fAgyv4afrAGj7pD+UdPVAdOn7QIsSvk6ZdvCuOEsQv6SOn6jCZO8oJ1DROT2ua/vS5kAd9FV4fXwXfFc4qKzmRe1qugq98j6RBr68z+srzuWPRKXuenGtgroFrUgMHOxavqBEL3+r9oCBFAuOE4Fs0N2qMLT3YfeicEQMMUgxfHqgE7/Ckhzc3Ww7upYFPzL01Hr6sDwBnmFokGhj9CzgVOqDtj8V5mkc4FsDikAw3eRntdhAEw9tdFsnihAwjEaLPdKNVGd04Fct6K8xahpLVAucPeHsaqdjR1CVGGjBAQj7xE187F6oiBg/OCk7F6eMn2sljJ1z/PW0vySjF9s6WtqzV3GoZfNcxxUh/wrbOoGfe0mGZKNBl9IY1JzgWS3IMGG2IdRSsmdA2pJq+cUFOy64XdSdt6WeBUQ6+vK01GAtyMNjqdkVOBs5FOCdhHA/6sjFAq6pdaU+M5ZTJoz9KI2+EaDsJEG0GPH9Z7HjIi8zoF2Mb9/olXZQrf+A1FJBwxwsQnQ4w4IufeyT9KiDLvok6OS/6XZHBseDtvY3vwDZtVEH/dP9R2/PWuta1wKcOS+Dz1AOcYa5QshBXfyeuUOkqt3PhPsr0JabthVPAnmQYgGkzD1LSTtQfduVEw9VOkviMb+7TwC7WNd8Mp3ZKnpIpqxCiAtPBkXZ9LAm6iLpbEMlJJygHf1DGWOlIoTNSxQjZQIVVQAy/6NoBGcDTjlA1nQgdlKEGmGURMHV3eyGrz4OqKU0xCAhww/j2qIWuUY8m4Jxwv0rvv2Bc/04nA0nlpbQhsn8nfxitOCLH/pBGoqB9GV4avSjdB6Kk6uj3FAZ99Jmk0Y2iaJss7GgMslOU+aaF4qbzDBK1GuSAfvIwuJG6n44X9AedFAh8EmagWWUZW0bSid/r0tlZZrlJ348wwbOrxywSE7AdQJ9f6ZLFslLHLhRMlzXUjTzXV3XtaTzv2c+bAH8gT+jrA+8p2Xv57HCrr+7pOcNx6tSpB1KsOe25BuYauMY1cC+OhT6Qp8XbvkHWQ0E3IM45rRtnOBQyb7hh6/DbexmiOAg29vOmhWPhOcu6SRG4DfM4lG3i2G99ZUGVIYuzgPHPSAU7RR2V4e+veIsexlcYYJeVx25SsTajbtaskWDak2H0YTFujvCWqWDeODysXVjBqdCIAFOO7ASJp+XIugC8p+kKjDjsMGIhB4PKe6oRRj7OjEZWLssq8lQjOSrXnzotp0LbvmoK1K52ctra3rJjwVvUwnUdO+O9HDL4ISFGLfpk7QZ1Y6SBaUzjNrJH9RJaX1je0QJwnBat22DnKIwrFm0vaYvbBUYnmPokx2JVC7w57JiobaBtA9AMw4i2OaBGwXBjYTijL5aL1rE+AI7gdiYPh4N6GEZlNlwDnjo6QFNtBA4Bh8X0hDPDBLECBtmEEi1mVOuP3mI6SS9LJiLTTUT5cA7Vb2mD6L+Qi74xyCEcwwmjHqiDYyE6rjf6swyll9Bl1Qlm0Cs6wXeiFi4b4JSI/hntXnzBE2BRM05WBdRBL/WtiYGn6yeDWA6RDWPVMegYzbL5XKShV5KZX4AkYJZYhiww7ZAJ7GjTiK1n9xMUVfBjEtji5eKCATQL+ixTmGjj4BOORWwDTftRF8AGJycdi9J7X6/wq0qKvM5glHy8xkLX0SVd817I3esO+dHFoANLOPFDedWB63PgDZ6CnXVNiTqiFxnWB/cxRi1oq4QBrk9zXiH0XWcdnHgVX0rNraNHHri+rxhgNGg53S/0cph38iF/1KKwp3iZnmAJwBo3YXqaBtjnp+o6Ez5pD3yLf+YXbvHvY9J+GUSCULhOTtTKxfXTy1Fp13AGT3AKpvBn5ZWc0IHzfMSi19Y8PdfAXAPXogbuk2PBW11C3XLjbSjGaFVZDzUMRxlc5RzUaAVx3TyhY+Mex0K43I8p4xB6xIKvUQ4b/3IoeGs/bkErg0K4dhj8NlkfqhNOOR/coHl4sn7hguatM3WJSpLHjR6RebizwJpvTZRTgfFOHShDZmICOHsyNPiGBVOhMN7Jw2FikbUdJ6ZsySFZ1HqNdX3U7tSJk1oUvuwHPU7FnnaBwnhANwN9jHHz4g2s0qJXxjSyxjSMUDD5OD7eAWp5rS3KYRCydcC6DRwLPrCHEclohB0QfdeCD/Fdh/4WcucojVjUYm3LkrpHZxhBdhIkC29Ua72HckMPaM7ijDIxHYs6hHMBZJWl46A6iqwbOhyLsY2TKJxd7vPuJ7gmS+VXH0L3JVPpq0NLCUKOyEeqOBeq245TG1h5jiEPDLQTlAlF1mdNh7FTmPpynV03V86yuQ+nruBb9ETRwbQzTVTnxPxxLdmh0Ln7KjK5DNhMI5/CoJs8t9AuglLAh3MiR9bXmq43MQhsQxgFWiWH0/woUBfnJ/0grV/9Rxs7RzCSxDoxUsiVOipnrmStNjODWT+Q8l8nk2hNBzux6nf0OWvON6GQJ+4jwpHc48iNoKiHQSqGauqq0zN1sd6YBsX6B3aI0j1mOrhdUjfTEvb6hN5wLnjSIb/uV7r3MPJIb76o+0N8f0PtlHUe8MS8+v60HJz3cLPK72teT6f4kTedP5QV4dRDnU7Efft1cD3NgievaFfcl12VnqYN/cybxi9cx4Mc7nDuFn0blmwVT+DOOCm44llx5fco5PX5Bdvz/+Ef+uEe5QFNz0csHlD1zonPNfCg1cC9Oxb68nZMF+HJ7KdzPHx1k7RxZnMibtI87DFcbTDLQGV0gVEHbvjcUMuxiBtsPAcwDrnB6l94rLGohcsVx0gFi5cx+GGnwQhPawpDDMeCrVWZMhTbOZLPaAVOBfElzV0nz9IL304IhrrWVrAwmvOgHQ+3cCz44FcYe0yDYuH2HrtCyWFRZVxHb4vL9Ck5BR790Hcqjh3b0ILwFdfpgnB2dvjuRIxyeLtcG+JRT5wSDHg+mFfOjFja+WJrXBZjoyve/OMEsQicrWUZGfEXli/uaXrVtuTS9qTige5Zh+HvX2ik4pD0cZ3qdkTTn/jIHrzukfJY5AtNRoDKgbBjo3J/+8IjMbH17nhl1EMRU4i2C8cIOuFYWF2ut/Pcpulg4LBxrjjaWnES9oPWJ9EHih9Z0dsqZzKGAn0GrgMtg0xi1ZnUGKVKxEvoqA/504Y7oO6vmr/vN9j5Ntk80/mk/lFjdIEsquMgCbLzF3Rg7HqSyFDnxD6UT8w1ZceinAny1Xcjz8IWieBABQiOI811ZsO63oIL3+WCSal0TlbiBgXLSz2CnAEi7Qz9CH5wLCimvq67ylIHjoGvRnGyO6FsRii+QA5yFc0ub3QsgiZ1dYMKptrB5CUrO26ZVtWT6lYaTSg9oVvxI497BV/HvqI1EKSLHnGQ4HcMVbtB7rHI8CPPYI+cbD2NY+HpjFpzdVn3KUYtCJPUo14dyYnkNM/p89JrIfW0S27KpvEG+EFfnRySv69Twd7XeJpXnZesFVd+0Z0+p68NciAn5/uEojnAuyf32hgRr+IzFk2kjJ36GehPQIwnRbNiSoyTIiA51+bzn/P8EekBTs0diwdYwXPycw08SDVwoGPx0h9nKtRrwijXDZSbIn88bAcDKC0I7o84FBzeIUlxTC3qd3DRVKSkIzJ+DnBDrWkyTGnCsaitVnnwQsNbzCoNHAFbokY+MAox0Jn+g6GP8UcZRnkY55ofL2eAvLqpe6clORWrWmTNVrO85SdQt5CHEYSYasGUKhyEvb2Lji9qFAQ6PBQW2AZWuDgmq6urGq1Yb2s6MHQuyDBhS1li4O0wqW5+yy4mdm4kL3JTRxwwnAvoIjfODF/YxkCMBeH6CJ4Wgi/LuUDXF7VQG4eCBuStOusFWN+xpF2iWI/hdRiifQi9aQTCOpEsLBzHOORDgP6Gh+AXvGMU382QLthlSjEGj1rI8qATntnIQohzyqK9nScAYHAq3A9wOJyX51k37HEb5dIJodoE5Gxe04lSg+z7Q1sT/KufwolcCuINtIH40Wt0+MHajmbKUGna30FR9HP1Hem2yqu+1M9GQfIHB1n6ERQoBTX4xQHcUN9M17nhgVP+xNt24LLvIoclC8KW0Vw6+lQOmiHz6FBUPtdLpUXMIepiSpHR0yNHerSc8FfZoF9SKnOGYtOpOCj5d4QHvD/rgPqkeARi0uzKwPfGBYoBc580PDgBD4x1oLpabsucfJV2HtojrTpV+3rqkqigIxZVD46F6ZMv+E4WJ7Oszx5gOvi+1tBhqiHrLbj24efREZyLJARM6arqQ1HlFT/gKlS6YvKvgi/grgzZepwC6fMsf9d2fVnBE1/FL+Xr83vcPt3DQKsv4/zeQsFDp9KFM02b/GmYgq2Y8h5vJrx4Tbdt4c+Kexqm3befbpH9wwAAQABJREFUWv8Fz3nBLLQHJG/uWDwgap0TnWvgQa+BAx2Ll/3EK9qrfzYcCz98hzehvBGNB1qYOtKjToepRDlaUY6GSnwT5wHqN6+Ac9NWzG0Zgwynob68jdGNwW7jmHycFIw5LM98uJm/aPDHSADwjGpAD9p8swKngJsnxyXWWaShwVexV2R8r+njYnzZ2o4FDwjIi08ZiBjjHvWQkc+IAA4GU6x4S4qReUSOBd+/WJKTsr627qlKGPcXZbyzoBp4ePIAwYAg9iEe7PSEI1Q7PeE8UU9GWLaFu6XpETuKCThCJ7QQfENf117SYmx/7E9OQrxVVQtAXzpgitQCayoE72lK8FQ+9dgVPaZMsfhObERT39mQk8LoBgu6mZoRW9LqK88CKCPaBpD0Xnp1Q1sqfrI+rtfoWNq5KAdDtMoQHA1ycOk/aj13I+ige3pEhehfdQYvgn97sMrL8hHIKfcFWAWfiOkfVx2Wh3KAI+b7BfQdG6/w6ZylasvgQlnoKOQLAc3DdQx+wJp+xpWeyM+yuE5CcKfV53TVpdzGSFrKV/ualyvJOfzSgQDUFUrewKZMArLcBskfcQTQ9Pp8SJDvwyc6d6NFH6g0OOiiD3U2nd/DzEr38JUmph3gV3UOPVpyXbtRBs/YKjbyB127birMuvieJJ24jS1orEO4rOuL6VCMWJnCDJ1U/izZyRt4IjPnBShavr78QiFemHB/ZV1HtB2QY9tQ56JVeoAUeZVf58SEHi5y9vkVbQf0MRV62lEDASR4DzrAuV4CSJpDPmjFR+k+v0/3NCtNeY9b+X1MOXDTtGbhFq0etk9Dt4epNPn7wkUhvw4F1+NW2ex4lH3uWMzW0Dx3roG5Bq4dDdyrY3HTz/6sDVMe0sObPRsuuhmqnnYs6plkJwAjV8akHpr19hpAQLjhGsvnemAo5uaLIc9zp3Z2qm824FjwIAPmqgMdU6a/GtHAQYAnfDAUkNfHZTkaGmlgdyeMtEUWbcsZYFtYfxgPvJS5piQxaoAzwRQopjJd0BePOfeuUKKBPEflVLDr0oroYKBjhKMRG/xyQPyRLcSUnNBHB96SVnBeNC5jHgOfKUnogHKciU05AZtMcRINdsFa1dqI0/oexppGRVjXwWiFP7BVU6VE76joLAvOUyzsYGFgYVzdE06F1mKwHoNREBaE4wzhqLCgm1EQvnfBSAVvUS2n5MUZYkQlHINsP9pQBwEdmIl+7RgmDvDwsE6VjjjOjQIyPSHI+Aw62IRBTwVV5jbObLd2pYEdA/1gCJksOeGDnU1wH+Q8M0aY6JtRrrT0Rh/yaJBhow8OhqsqEvUvPbgGIUVWcj/aoxxVyZDLAqaMNv9TQbSh/yQHco91iIr5HIdB8P5TWic+Rg4mnLhZXgy7uJe5y441+ZCANiF1XEa+z9EJia4pOtBBX8a/Dz+lX9NInZJXPKlbOVVKmSJlhef1EanD0Flqw7oBWdqSPod7WzAyHa5hpid57YPhAjcpuIqlKyPM+BnKrZeUMOUBvJx/7nPAetpd7RCl86oHcdHq05YfQiWUW5+MsU/G2f3/LX495tCs2RaU7QdXIlU5ck/Ivg9uz89pCA2MryqdyCheE5k6qXz4zwpVXmUFR37fn3u4ggGnz58+7+GK/mRMHzSWs+eOxaR25mdzDcw1cO1p4EDH4uU/8ePtpte+Rm/Iw0Dn7S030YlDdea+6FswDw87FzkNSobmcGMVEEaPgYHPBye2ZDkWGKCe1pRTjHBOHPQ8gAePBWBs/ItXOC7xxWqMYDsW+dbbIxqC9xtAGWa1gxQOw6I+YIdhv7oSb/dZswBujJTEyAIOxS4fKtN0I6YlxYhF7DIFbRwEFmsva/clDkljA+WSjH5ciMMSWJRCV8ihP4xF6sd6iTWtw1hhlyeNLng6hMpoDJyKLW1ruy2+1JmF2MdYEH78pGHZAegiRo9HZOJtKk7FoujhWLBGgo+IMe2H9tq7sNvOnbu7bW2eb9taSM7ULPiZrrbFXV/XV8E1fWtlZd06iOkgMXWL712EY2GLX3VRP8j2l2hj26phGDFx28gxqWlfOIo2ntRmlNEXoj9QswjjQzn6jitdhcSiTbvbJjB+X9ilq4MYFr1TFg9tpy23cpJ1ORaGclkUlPM84Ziq3gTqEE4WEkUY65RyWlqVSdaqm6+XrJjTya/Ki1bF1jGy84dTAbz+kKkM6aLjCqVT4XJg89xlVlxSVpl1UuU6c3s4nxL0E3pwnOmobdY5aZii20P58DCfhMkImC6pNHDGvOon+kVklwx93pA2T5zmIBR6GB2pcjqAH0YsJHPRJLYcsCJfLxs8KsW9TVngQdojnkynZHoSdeYAxT+Rdkb+lHzFpy8zin8m8ewEqU9x/zFTtW+MWlCfaQrj+cDDMqVCSZduUzdgIFfBQ3IEyXtT1mukPpkqXHKN29GehJxxJob0W0Lph3RPk/N7C7GTV0BBLyjed5rFr5fh3ni6vNevKj+LDnkc0J7Q9ZReZ/MOXHgBjmrnjsV9apk50FwDcw38DdbAfXAsGLHg4RvTivz2lJspf45VOx4g3BW5ucqwx/DHePU3HniDiALyRls3Z3DsUKgobrosKA5cHrQxDSoN2nyyRDlrLmKxc4yMhDOCY+AP3Ik/f8FUbPWMxlhk0XVNj1pa5LsQcizkGLDomrUFrJeoBeLUka1lt2Xgs07Ci7cZtdCBkyXrUl/Q1YfwtFibbWDZYYopRkydwPD3TlM4K4LDaGFdxkV2mFEZu1utyhHZ0OiDP8onQ5x86DINim9eEDN9yd/Z0GjChox/RkRUEU2x2jVN6ofjxSgDX99e0HQmztE/phNrWbY06nH23Nl27u4z2jVq26MclDNVylPB5Fgc2zjeNrQ17pK2pMVo5qENDOs6+JCXF5frHIUOhitQtLdz44GKo0WbQwOcI3xlmLx0OPzgRehAyzanW/BQTkpJc+gj0AdFPy7KNNAHh2RCvxR96sSP03lSPPoYyKF/q+PQb2rEAhkO8Q0FtSnT5QjmAr2Uz/3OJWYXKfEnxPWiJqRDco5cWeaM7sdlyAJWwjmWPINzRxocYudzBjy0yctz0019gGF6GK6UTwXKlDXIZWLOiDr24EWShqmjL890gXFa+gkuARDXPiRGyIE/euXPLOjXSSVHJbzzlOsaOqBOg2MhPKaylf6CW/JUZB7SEc4H16gXTxtf/NRv0SlGvg199JKHKShdoeSuuOCG8wIsnIoln/9VF+4lXDfqYP6uhadfiX/RKBIVW3adEI9aq1LFnS6hMcAnSOFUfofpZJ9vHklvljx9eU+np9Hn75cu+GkeyMrtvzRO36k0tHr4orGfTD3ve4Uxk+BU/IpX8enpVRqYKq+YssItuKgElOlbmavKzh2LQUPzxFwDcw1coxo40LF4mUYsbtYaC3ZX8cMX5yLfoHI39J9uiuFUxM3TO0NheGNkpqHrB5nvoXETBY9/PzSkuLjpYhTEtCg7FTZSMeBAVBAwjgSLjpnK5B2WBHP4iBwLOxW8KedNZoGHIYLRflmGPVvG1ttHFi0zFYrF20yFYg0DMdOMIADc1o5GDeRU4Fjsajco8DkwC1kQzXoGFmzDG+OELWUx8hYk37och9ht6rCdBEYJcE6oN7woX5dTwhoLVdqOByMpTNcqwxMDlulJK4JluhK6YfoTayV4YKFfFlp7jYh4Am9NCQ4ZdwV7ViMVd911Z9vc3AynQvk4UMtyqNhhakNOxbqcC+jjDJTu/A0Lnds58DSt0Gu8SUdkKdl6TsNPvMuxwJEALxalj46FWAceiQzQQSdhPGbDkUP34EdZMZolCBldZsvPfQleqA1g0HNKNOthPyvGVK8Ri5gKxfS/ME4xWGvEIhzikKPkR1bqERwdUYlMRC+Gpw/LNMoyACnRywVt/4GnvmX961ocZIR+lgWe4Ep35AsWqSakqHwA+5DnlWt6nJCfZW53cKhmtUM0CrkzQ2gkitAP9ZkORbfiqIsV6j5DPnTCjKYf6OBah1o6Flx74AUNRsfCiSMv6IXY5u0qKR+d6rpjulMs0g98OxbCY1SQKVHluFny1EVfh5KbvOJFmvwJnKtwo45exK37CPfLWsSNXD1d6FUoHhWjm4lA5TNYhuTbax6Iwi9Y4gl5swC4kqXiwgF+5Ba5s+hSUvlX0ej00pcV3YMcC+gWTtHv5aV8OhQc+YU7ATMoalZvnYC86qToFY+KK98IoeTAzbQjVXjuWFyl0nnGXANzDVxjGjjYsdCuUDexeFs3fkYs+No1MU8fboTcNP3WkCeAHmbxNjcWEmP4xsgFBkCEmNaReDxM8oHiB4jwk4yNBmhBoyZ3c2PmWxMY47zprwXeR/QFW4x7YIFBVtNFQAVkZrSCqU04RwTwmQa1tqovUyvt3ZHkWISToO9WyAnY2taUJDkV24pZl+DpUHI4VEkvdMapYAoShs0FGfsYIEdVtuqpS1rILccFB4n1GYxCwJ/Ark2MVqyKNyMvjEywlgLZEN3Ok/AYrVjS9KZFjSQwIsEuVywGx0lhChijOvDHIcDwuaytMR0kg0ySdk5Tn86cuUuOxR2WAd1QPzsV4r+h6VXHNk7oa9wr0uWCDWgLIAO+tqaNkQe2xBwdC+rvdhY9FUjnOlea6Vd2djTSQduwiN1tmCMWIdyMX8jwBz0F9yn1LmLyoF38APGIjNs2GzhJFn4Qqa4lwy772EA7z2fl946FRyvcJhjnNHs6Fq4v9R7ldccVDPUouiUdUMCST54dxyFNjg7+95NTVpVxy6HgLTv4anNTnEhDKuAFNEFTjPI8DHCdkBUh00SmqdySGTo9rOvthjAQgElk/2hfiOQ70BxIJIZoU+byLIvzTv92/FQn9CF6ASt84VmnOB5RsaCgIp+jH+HYqVA7095uG/jR3xWucixS3iA0+9e0s6iX21mJ32nefQZ+vKxg5BN8r7XQfYPQ03NG99OXwas/L7BpGQYYwQuhwIZ4KB9yIkF+0ap4CsSnA35H31w6XtP4fTllHOZn7UzqgP45kMpuAuPCmZap5wXNQb4OsIdx9qAWuN3/sJ8sA6WsQA7EJ0vxUr1fMN9udlDT34TEnecutQ9+cq895TH64OzRrsP9TRBuLsNcA39DNXCgY/GSV7xM283e7JsxBjDTRBix4G7LLdc3Xb9J5mGgh73SfnMtY9JOBed6aNblyP0Uo2icbgKhPHgQKUDTxiTz8zl40MBD+RisrElgtIFRC4xXG7HKBwbRarqT3zgrgwXbfCjPuznJyIAejoVHLLQrFIY2zoV3lFIZsjF1qZwKpkPtyphnd6gLooXRvSjngalJscA7doBiTcWyZNqQob6+rilSonlYb/tZ2+H1GekQkM+IBaMlGOLmJxj0gGwY/3Gw49SyF2PzJnOPnaZEA55ew4AhLz3jkODUsKDbC6/VDhfk/N2hkYpzZ+9uO1pXQVtQvyVGKuRUrGvRNh/yYzcoeEoI/UsGJXGGFrX9LPVkdOiQRh/QrRaOKNAWGNgxzS2mBuH80e4hjx0+4THq4TUWGOLC2TeIdDzYlZAO/AY6epb7FLKXY4F8vREQxkdSph4E+pMA+U0ypBymDYsyMiqfGOPSjrT04SkppleORTivOHbIAUfz0e/IjzqoAFkdCU6wZBnKPEglpvpb8RfIZDphKl+SqZ1GIzocdRz9oOf6iGzQw6A2ZsgJjDMMMFImj7LkDQbBMleZ4jKCrH90TZ8onQfKvr/ooYI5iZnpKHMoK1oVqyRgiMEOSPLqsNx2HMp5oBYEwQJOvagDNwZO89fVSj3S723Ic66pUzCjjwNL/kEjFqFPE01ZS/ezjfBZ8IEtdXL9cB9T3/KLAr1IAL6OQV+Sz/WgEgo9zaJVeQMOBaHECXjgCqZwBQDwcBqJ6B+V2eOULJVXvM0vZSy8oUwZBU9Z1GTkOpRVHYf2C8gawfAZTZ19YpoP+T3PShMXj6GmM/QzlEkfkxoITtP0p/nXefGq89Cx5O71g6wCmH/HYtCSE+/52G77+J3hZNMey4uH2mNvWGg3nmR2wQMf3vrerfbL7zjb/sl3PqTdcOK/Ds8HvlZzDteaBq5clB2qDYRsf08Jf0j2GjNhNHlHzyxtMKQZNhdlA96jqdtMlV9clB0Hju43Vy7tad0wM2/0clr25fLRA2yzKT735/RAx+LFL3tpe9XNN0mIcAbKKTADrnLdDMPoS+NPxoadAT2Y/aE8lXNTrRs0N86igUFNRbnJ+/CNNdJgQPeQNGXDMunwBW7WR6yyrSrTl3AuMMT1UBaI1xWwa5OnFGEUiAeGN0rGICNgSC8xHUjOybJGLVhrwPazGPM82rjX40TtsICa6VAajfDXuzE4hcsuTnyZGuOaaUn+loTeLjIFalU0j2mK0/raqp0VjAXWS1yQkcDIBYb2spwSRizY6tVvKMWVRiCUY+GP1GlaVq2bQIF8f4J1GshmZwD1qX4XtDib0QzyZZm0PZyi7e12l0YrkO86yb2szuUpVTgVTH1aXfc0KxauXhFv1oZ42ozqh7PDCAlTmWKdRExpc7umw0DdqZuncNjJUDodANZm4FDZsSAvjXBXsP/JThH9I04w6sOxCGOh31nKcJJP/zRT9J2kl6TSQMkzOptCGRN92kZClld+9UNkQJcxNSanQbk/Rh2Ro+pkMeCRnM2r6KYYRL4CLDjyqB/R5zsc+lzgKjevh5EqEnIWMMaXfG4vYNUHQmcjLnnKNK1AVhoK5GVcNMlzuoOnjpXnWPycI1CXlVORcAbWzyRethX5CeA6FnDGVRYNCzDaUq5pR2nRBYU0h68B1cWOlnSK8xB1AwjICOaJPhRcTgys6uRRCq6BbG/rVLj0bzAYzWD6YbWJaZW+TDDokuwDcCVz1GUsNQ2dVvl4HiO0jFwQ7PDoWnbbwtN6CTxzLTkyNlL3MylDFohG8SvQHs55pjcqsFKpXYOU7JyUBgrOefvINF1WdKDR41c+fdPyCmBabmi5RbM/xPn4W/RKvln4BTNijfUhL8pnOxU9Tp8e+Yy1oj6V77qhn2kdpTDPn3/Holdn+zevv63dce5yO76il2jS2fkd7s33tCc/Zrn9va87LvtgVitOkPisTj5Tx2L3wpX2m+/ebF/2eUvtkQ/RFu7zMNfAZ6GBO99+U7vp19/dPnRbfILApPQijM8oPPy/+e729O/49vaU6/fabe97e/vtN/xae+st5+WF39ge99Rvbc98+le2hy9pJsvOne3Wt/9C+5nX/0k7d/h0+4Jv/6H2gm95hDYZkp2sWTp6iy27/bMQskM90LF40Utf0l5586t9U+TZXCMNfsjpZskWoxiiw1tlnccbbQxKP1I7VpHk5lB0fLOFLnlUToeNIt3SwbeBaoM1znEsGBXgA3A4AyyExiHwzUXw0MWpKOeC0QqMRDsxYs9NHbp8oXtRDgpbrnpaFd6e6MQDjQdJ82LsXS3IZsSC6VRC9JesF+RU3KP5XxeYLrW1JV76VoXKGRFY1yjEcY1WrOtDdniQ4NixkHGCc4EThCOD87Gh7170MNCQaGGQsxsTIwY60CJl7DbFWpdwzHRzhaanRmnROA8p6emy5Dqr9RR3yKnY1jcr8EUZ4VnXKMoGDg+LtY+dbAuSgboymnFJDssVHAuaR/xpP6+xkKzxpXA+WIiTI2dDIxw4PW5z2l2OxiGVeScq0eMPByNGK1hrgcM3vt2HxUSAX+LRF2zMqycMdHBgoIfzApwPKFja+I1kks2rgijz0V2FerhzThp6lebcoxWCL8cCeYDoZUBJk44FJleEnldmGd86UAbTBuFTBzCkK678yMt8l45y6moxjnl1Mo+4qi80QR94BU5kkh20TdpgWYcZ+YYxPXCiDZiygeqsv9QhcL0+gR3yBrojX8ugU6CCjsE5STrqvTNoA1uH66f2svHtWASTl6doQtJZY76zdB2VY8GLBxvxwqf9YRkfh5SmeTkhxxvYauVIpw6TF/L0OnV9xEiShgAw7erCaQ9POuoUUxy5BgnDdzTgk/imrfMe38DdTy9PyVLwfVmhVBnnRd9lZjsa1jp1AMZ08rx0Pp4WZOXMjq/iOw2W+gXOFBVbvh6u02vRc58CRmWVVxJFmxSBzK1CZQ/9xugd/hTvohv6CnrVR4p6H0/CF1/FCJuB3Bc+94V1+oDH9P3SZ8U9U/KQm3sNBxuUcJw6daoHe0DT//vrbvMz+59810PMB4P9P/7+2fYnt+y0p3/1sfZ1T1x9QPl/po7FHZpC9b/9x9vad3zVRvv6J609oDLOiT84NXDPx3+/ve6NH2mXHv2U9m1Pe0I7/KG3tjf+599rH1z5qvaMZ3xlW7nl7e0Nv/q2tvmEf9he8J2f33Y++b72lte+qR39R89p/92hd7ab/4/3tK96xXPaU667u73lp25u9zzjOe1rbtR3zf4KBjEOdCz+lRyL/+emV/nmovvLEHPDDmNr3OqVt/Hk15Qo39y5MfnJDi6Nj/EYD8W60UauHqKUCajKeQCEkYphySGHQEYtow3+uByjFXYsYioOdHTv81QhnAt2ecFAhA+suUly4Fjw3QimUrG2AueC0Q+M5zKCwWDqlNc1iI6nYvG2XsY2uzexY9Tm1rYNe4wPynF6mF51TGsncCxqahWLqGNE4bLz2GY2RjXWzRdcRjOgi6y8sbRTgREvw9wGkGQI5yuMXkZJ+AAfb1N5MIFzSLrZklx3ageoO+68Xc4C2+ri7Gh0xNOf1rVYO0YrMAwZAdnb2bKDguLQEb9+my450BWOASM0K0tsZau1GBpyi5EK1rVoRAOHTHnhWNACkiUdDsppMxvh+fSMloh+hOBi4TYGj7pfZiqKArwPM/Khdu+NevIDwJ0pO1XmJY8AiF+3vejOCn0ZcnGOA+oRC7VpjVggZMkQznL2ceUXZ+OjO/ft0ZFB3vorYKSZ4J3yVR6xICZE7s9cjqxlHCuNYe2Hv2KdmL4ARj7AUjYjFN/pouBDbtKTEK6v2tQNl3WrelV7DnQsNLCQiBoIxWHg6fzQo9vW5ToHUA6p+SRBk1HaOlU5bUGg3q5bXevJywZips0/ldjrz1Og1NbhWMToAAJj2MMH2nEfoUwEirb49ucWZOKHOpChH3BC1AkI8F1P5ZoWfIXEPcbXmNL+joacWzszRQ8qJQc0kqqrJxzOi24WJf06ixj4wiUnZIiy4dey91BRMk2/4IGE7kxaBTQVHwgr/q5XhzNIEwoedVEw1knooSoYNISZyJQG3yrJ55RoTPMbye5XUkhZ7jYIRoP06DHDmBrEMYGAvW7uWJSiMp52LMje2r3SXvzzn9KUqMX2Q387nJzbz+qbT3rdenztcPvQJ1gTeU/7wkfqZVi+gqUJbrv7Urv97MV2Yu1Iu+FkvpCc4re9d6XdevtFvXy8rj3i9NH2jg9sz5wKFXDakl58TojnIzQqwa2RcG77cvv4HRfba37rLjk+a+0pj9U3oiTH9cdjNBIY+sFd5y+1T991Sc/Y6zzNalVvlftwb3XqYefpB5sGdtoHfuPn2u/eppGHr/229g2P0Vqgt/xae9t7d9oN3/ys9re+QH1t89b23t97Y3vdOxfbf/8/fVd71B1/3H7ppz/Wvuol/7B94dYt7Y0/+XPt0A/9i/Y17/vX7Sdv/7vtf/muL2xri7Kn/wrCgY7Fv3zJi9OxiIeFb4r6wRguY4stXm0AyrGIB45u2yr3NaafwuHC5mbOgfGqhMUHB5MHo7YcCx7oIGI8lHPBR/CYbhRfq9ZXtnEGxBv+PF65MeOU2MDKqU/ml0qyUYihKprlWCwuybFgfYVo2RHAoEAWZExDhXMWSvMWk3fpm5pmdE6jAuc1WoFhQsABQD4cHqZDrWr9RdGLbWbDqPPaDn1QD2N/RTHOjJTmUQ0WxSMvvDDWWZhtWRh1kWNBwNhlBGRnLxaLY3CwToLdoVg8fVYLtu+SY3G3DvTn9Sis58h1FWsaJcFpYb0GIxoXtDid6VAY/zDDEfPuVJoCgp4YxVlRXZblWLAdbTlgsXCcr3WzXa94Szfg05Yx2sGoRty46+1+tb3bX3URqPky6kUIh4Y3xtHmLBz3lCvJ5rZTPmUcrpyxog+ZubXlzOEHXhVmpS0TilKoNrcxWY6FjPGS5zBbzQ4yhEwDbdEo/N6Ar7oHHAqKlPmmbCVX5dV50R7ixEVccQuHM/tpvXEvGirUf1xrpie4Ua5OJxCnLOOBl7MTDn7oQXm+qtG/2yxyqk7gumlIEOrVL2CiEWVBk6oP8ugk6I70cfCNkAQhMQTljf1AdHydhj6iLskDBJIwI5FpaotO4j4RTkWNWoRMSV/3AmhTxnVHukKQgiap/QJ0ADkIhmqWkQsd9X3up7qW6fvcX/yRTfhHcdRD6Zl00Q1UYNyFadheooKchjE6sk/RIn+afscq0Pi9l3oXzky+WThdVrL2MvUwpAMm9QAdZTh3QC7ixKUJCvM6yOKJSGAHlKqw+ArL+ipm9DUoFZ8+1am2oz8fsZjQfJvlWADxo6/9pI3x53/HaSP8X796h56rGl3Xo+idH9YOjQrf/80n2xc9cqmd2bzc/t1bzrSP3X5Bz8RDcgautFPrR9r3fdMJORjjuol3fmin/eLv361nYLTX6Y0j7bE3LrZ3vH9rYo3Fr/yXczLg9PxX4x7RvfCi4G84frQ9W07OmpyDn/+dM+1dKYMF0Q8Ozou/70af4nj8/O/c3T78qT13F/oIL+G++clr7Zu+dD37cGsH1anozuMHqQbu+sP2n/6/P27nHvkV7Zv+1le0Rxz9VHvXr/9q+/33LbUvesb3tKc9Snq550z76Lvf0t7wulvbo3/gue1rFzVi8bO/1Ra/54faN1z3rvaaf/NH7Uv+2Ve0d/7k+9vX/Mj3ti9Z04viunV9lmq9V8fiZ179Kj/ExhtrGH7lWPhbBX4QhnGKPPHgD8m4RJk6xB2WG78NSIqUHh5Quhnz2LZjISPGD1QZRtTRC5X1kIcPb9AZGVhQ7O9PKI1Bz9NDpoWnLHGRQrvohyw4PaOjwkgHU6EW5QQwHWnR045qKlQYmTwL4Ml0K9Y6IM2upj8xInD2/DmvY4ANevB0LMF61yrJhEFPHusSYEzsaVxyYlhjwVQujHSXU+90KpAVp4IFNxjv6AGjEeMGQxEnJbau3VP1ruimdkhrNeQsyLhnFOJubS977vxZOT9bqr/klwyMOCxrBGVVBw4C88a3tjbbpuAY9cCV8jQw0WIXHBaaY0ihd+tGMnsExaNFjPAse7eqJTlHi0ojL21qZ1POgLeZpe7CP5JOATpw2+P4YfSqzvWlbox1zjGWCRjj4ajFrlI2zpUf7Rh9z5VTXuBgRNBTOPYP8K8DqIk0lFRuYxNjEp3rQFbLQ/+TY+H+KnnJKycDWsDZIREOdAklr2P1f+emiAUDXKV7ecivENTiTKwdAjZ5luGLXuFNPXEqMLiJzTnzQ4qASQbFn1PSvRaHMvIRRAKUDkhP6LxHNDF+COBkSkQQUZzMq9Jkmi7czUO9UjqPvMDtf61TwSEf14EdAoj53AxMh3M1zkQ+OL4H0b52HDQamGnOCUFfjgWwKuO6GHSRggTv4DVdNkvu+wJj3vQ17qe6lpDfsullQrQjmpOOqNeskIpG3RzTUD4v3A4WUr18yN+fFx1oTofIo38nVEcXGj2dq3Cn+Azl0JjAHfvQAKNE0a540LtwS1bXBaSUq/ALJ4q6+oaSDBb3lcCIu1ZhT8XJzzw7PsqeCJwOelI65A2gkIcRixdM4DyQJ/+1pkLdcuaetrHU2snlapX7XqtZjsWtchD+rRyJpz5+tX33f3vMxDDCP3Hmohej/v2vP67n0HV2HlaXD7V/+8u3tx2NLPzAt5xsp44daR/8y732+rfd7VGEf/pd17tr3HX+sqYufbo9/NRCA593J7/1J5vtj/5i2/T7xds4Dl/6ecvt8Y/QaL0A/+yW3fba376rff0Xr7Xv+MoNOy6f0ILzn/6NO+UorLWv/gKtp5TDg+NDa/+U5IHf39cakS94+GLb1LoRnJU/++hOe9Y3njBtmO5XpzXVaR4ezBq43G5582vaf75lvX3hU7+lfcMT1V/bXe29v/WG9rb33NMe9bef1b7lcYw83CXH4k3tV37hlnbD9/3z9l2P0hqLP3x9+7nX/Wm74/Dx9shveHr7yo+/rn3wK/9Fe9aX6+UxKH9F4UDH4kdf/GPNjkUx01XBfZOHPgY1xqMdC9IynuOeGrfjmlYEqo023WVtgOmcGy63mH53Dy44w+EiYGinUWTHQrRrWgzGNA4Gow5euK3zenjgmMCdc0/DsYwh53V6Y0AedBhNwLBnnUbsLMWIg7QqPGRgWhLyUcb0JgxrRgoYqThz9pymQW3p2xZa2yAZzUu4jJxw4FzwPQyP5CiNXjDc4bksfssy9BklKcfDsluvGM04TJoCJVh07HUVfmuJESlDiLSmCyEjxi3TjZbkLKBr5Nvc0te1Nb2Jr3azLgTjhO9trOg7FRj83NV2VL4jx4KYKVi0BG1JhePt7z2qQ2wXaxwVwQ9cnKNVjXr4o3r6Eji0D2nKkoosi7e/FR/q4fU3kov60eBuWwxwQdswF0/agsDDlgdr6BJnMQ50B6z7iuhQbnrG0o+tXQhM5Vd5F0O/P5AJuZ2nFPKRLmPTb6mVH7LGqJy4uI8QIwfOBQHD1utf1EYi4aNkDYj4dUWMMf7AUxjGqdzIq7OMqXtACjb1qTa24Wu+MooHIzqvH8GZFrF5IJzKzHOkC3OkcOjKgHO+Ynhb924D+ou14HjEDyoTbZRkRz2IZvIwfaeDvmmqT0iz5sU1QCj44JlyZD4jblFO3TiMMvyYBw6Dyiiqc9q5nAZfWwMdUFU38QY2ytBnOB1F2HQse+RwXmFW/Qt+VtmQJxIxGsx9IxxZy5DXvVn4Z+xPs/haEsENdFOwq2CzT1GMPvpy8sAfajWDHjB9MH5Hk7KiSTwtT4/bpwunz5vGnQVT8KmdOBVf10FyEYpOOZEBRItHiKssT4RbBdKOM8maDnUbShbTxToP6qCmNAPMNM7n4ncsPnC3pgstXdceIufi/gYcC3T+g996yiMDGOy/9kfn9Iy70v7pd17fTqyHNYQRfuudF9rzv/0hnsJUfD7w8b32qt+8s33fN55sT9JC6gpvfOf59qZ3nxf86fao6xfaG/7wXPudP9s0zYeeGKcs/fvfPdMYyegdi6LRx//6F25rJzcOW07y91tj8b5bd9tNb7qr/Z2nHmtf+4RxfQijHi/9959up1WfF/6dWE+yX516vvP0g1AD2+9tv/r/vrnd8dCntK/75q9uj13lOXmlnXnPb7c3vvEd7S8f+g3tu5/+5HbsU+9qb/5Pr29/sPno9rQffGH71ofLTrmwLTtRGwFpXe3OB3+tveq9j2/f+6S/aP/hDX/ezlw+2Z7yzH/cnv4lp9uapud9NuFAx+JHXvyi9jOvepVuhiMTboQYohizxDFiEUYiN20fgvd0qDQKbazp7uAP7ekmwQ3W5qLTGBpxw7Vxx6033+Dz8IC++ZhnGHMYpIxUOMaIRSgd3LRDJox8jNOYVoCRj5EKPIedCTsXsRAZWlGn+GI104EIOANMBcLIYNHa2XPn1SibWtCtOZy82UdOwQVN3tZzhJMCPRvFoo0s9f0NHBUcC2CBQW7U6zf8GPQy1lmzQB7rIJgKgYKuiBdGjhIAi6acopyKRBlf5GYx9o52ieK7GZhBh3FiJD+OEWtGtjWNa1vTpfgK90XRZuqT20v0EAM9YtSzwHxBOMi3p0Xj0KOj4HCwTe2JE6d8LIg/BhAOzvj176jX8EaftpX8GN/WlvjAA4cRGI9g0AFUYd7UeiqVymk7t4lkKFplFCCrFWTt+8T1IHe/EDIgRxzAYSwOfQ4dcyjv8hXeYochObRjOknRv8VdQvgv62Aj1cbppDyDzDQoISPkiBDxcKpM53QZpgE/5VEGbsjKyAr9It6o2/gFJp0H8/B5V+8sN+9QJAR9Gr9mMJwXjUEGai0H3dOVsjLTdak6m0j+pFbyDHlIjnJZFOroI+4vPR3zoExYle96qv7EDhAdKlFZoz4o8wsL+qLaKqZAsRYrRi+irtm24gUp4MrhHuuJykL24KJfzoeT+5eo+hRW9HvuWWE0uZ/auVA9rThpIfsROL1cUk6QMVyvq0npiidx1aViCLh8H1rBYPJ3kCFxSryraE6iDWcD/pAzmSh5yb03WGDg7xpPtUvRmUUjZI4+ZuxU2T2MtJkvlKdCtkepqkqr/nFeDtrsPtLjfi46Fnfp/dWyuvJn8qIdx+I2rZ/owyNPL7S/+9SNid2WMML3Lt7TapF3wf/2n262X5cj8rQv07elFsdWuV3rLf5A6yf+wdefaF+uNRA3/eZdWltxof3IP7qhUB3vt3ibx8MnVLHzmtYE39/443MekaipWfs5FjgzODX//BnXt1OaatWHV7/xrvaRT++1H/vemDK1X516nHn6waeBT7/j37X/8MeXNFrxtPaNX/7wVt36yt6Z9pF3vqW9+c1vbx88v9RO3HBKfWyx7f7lUnvq//g/tK/eSF3dow8tn/mz9ouvfEd72D/4qnbrT/1eO/0Dz2xP3Htb+8U3LbXv+P6ntcee0nfaPgvVHuxY/NiL2k+/6pW6qeYtVxE3Zg4bXH4zLWMd4xnj3GUYBQmTMUYab9A5GFWAHrB55++MOz3IKRdcLITFURhHGpBi4K2TevM9GJ6C9WiGnQamI+WoBsa+DjsWGMIYtjgeyqMe5bzwSBFnrzMQIxvSGNgYbyzYPi+jHAfD37SQoU2MTQN/HAdPm9LUKfjwaKMuYmD6OClsk7uqb0mwvoJzeBOYKYZRjQOwIEeANA8+dpzCsaDeoZMYtbhO8mPULy6vyvg4KqdBi8i0ZoLtZf2tCwnFbk1MVWJtBCMw58+dbXfq2xbbcowuMtqixd02YDSSE22pBejiz1QttsLF8eFhuskICHS1AxVyra8fa6dOntLuUsftTDC6siI5Fhf1ASE5MmP7ULcygP1Y1plaXnWOrWzlgEhvl7TvMi/awWPHLY9WpNNl+ZRf8gETIeLxPPCz8KqojAjiPk2/5Lycm3IsrmhUKIzV6Md2ou1YQDr6Lrz9pxj8ME7BC5hp2Upy+lUvQ1UJvMLp5TQfeAFo+dFqOEDhBOFc0A+jLsQphPkUrYrpsK5bMUbcEJrUEAxfZyGczkofKY/OgYtQcSFFHHJXnuqQiiAK3LFNKIv6xj1kABas6SSZURfogesjeJveIE8wQleuC/mMgvpAZ+FM+EvXpNELuiOgb65NoaCrcC46OQUy0Bz4GTMlybTKerkjt/vNNkiVjAXw18ELCfoedWCdVbWxSq0aYCyHBR3R+1TB9Hmkp+Ua6gO3bMppGK5D4PowfS7CLuZ3pBkYLsnyAY/zpDnkCbyHDeyQuad5X+o2DV90ya/6TfIFgidUyA9vakwb9FUvXMoDglhYCTRZPmT7Xm8ZAO+C1SJGL/gcnAolu9vbWMbTrqv0fUjiWLDm4e993Qk9HzS9SaMCrJOYDvsZ4b/8B2fbW9+z1T7/oTxbJjWPzr/9Kza8zoLpSSzE/mfPvH6C9CzH4k+1I9UvveOcHIorXrjN+olPy/m5Ud+5uDfH4nWagvWO92+3fykHhm9y9IEpVu/W2gzWYiDrfnXqcebpB5kGLn20vfm1v9I+vP7k9jXf+NT2hFO9c3qlXdzdaufPnm1bfI9559Ptw3/+rvb2ux7fnv0DX9dOubvJrtw+0z7w6ze1tx57ZnvWE9/bfvL/3Grf86Pf2T7/8LvaTT/yJ+2J//Mz25c/9Fgbx/fuv44PdCz+13IsuLPqIuRmWUa8jT49aGz02bEIw51yHi6GExI43LhxFC7pIY1zgQFHPpc5B8EwwOlgJACjCZyYUoNxGfDF3/jFx3IELwziYZoTO0fJeWAKTzkVyOu35TKS4w05hn/IHsKERDENB5NF27LykTwZ4/X1bm8hqwf9roxtRguQCcdiCWNeB/WnHhdVBx5I8GRdx1ou3PYXv2XEw1eghq+3/oclM8G70diYCMeijGDKcCqWZMwviB5b6+JQ7MoBuCR5zE90WQOxoJERRkLOnz9vp+LOO+/QFCgt/BZdAYazhcNlXWjxueC9A5ToIxhw/lYHIxZycHiDylqN9Y0N7TSlna8Ev6A1Fqv64B4ODI4R7YKugQ2jLNoShjbQ///2zjXWtiyr67vqvs85911F29BEiNghDdqgpkmIIiIhhhhiggngR/3Co2mQLyY8pPlkNJEYiZG2KRITAh8MBjERUBDRkIgYoiISg0EF6eruqrrv96Ou/9//P8Zcc+977q2iOl3dVbXmOXuvueYcrznm3HONseZLThEnfcf5YtqZZDEO08XiWESeGrFQntuarvpXSD07lgSi5uvIFKdNzUYDOiR0el+pK8uqK0Yk90qAFdxMu9sbKaMNAqCwOBYx7gPbDw1+PMjXxnL4O9Ffyac+qoDmXanhbRmCEdH4naDXHrHASI6hfJhjMXarAtmMtvUCX0oy68qQhqfEFazvfnFAWn7byW2Jkx4+ycl3qISTUnzbOH0lMfpuTHRJsE7Jy43rizrj4yBZTaW+gEv9IkmV105F9Ma0wl6YHd3JKQwly8bvhgCNrPlK3W7pCP2UjgZu0yjclh8pEpby+H5uwwXR6W73+i1BI+utJANtGFIhM2O8ofhcnhBAVwuplr+v5IAz3zutUVofTUS0UmcNsGhiFKN10DgNunUvKlXmXZn7PmSAG63Msja513M1C/g2byXQftR0ljIjeIeSqW/Hld973Vg+SLriSPRN6SVQ+X60+Y5v/c5B4tMdebPWWKDKruI/bJkOW2NxGI0nGeG/oulNP6dpTt+jtRTzrky7NH7837yixd33Nz/41/7IqDdgdh2Lqzcfbv7uT39i877PP+W1GDg7hH/4sy/ZGXgtx+KX/otGLPQ5TJ5/8vOvbF6Wg/K93/Qu03xSmZy5fr0jNXDtt35m81O/fGXz+R/46s2f+9NfsKmZgFu6ePXhvc2tV/7P5n/8+q9ufu13Hmze8zXfsvkGrfXxE+3h7c313//1zU/90q3NX/qmv7h5z7P/efMjf+u/bb7ih/765s88/HebH/lHL23+wnf85c37nj+z+VROX3ltx+KFFyI0nbV+QzGqcCi44aGXN/IZtZAxCBCw/Bmn3qqpd+HNPyMWNtxElXzML650vp4qJUOdN+x2LpSmLB+21+sjwr/0SKb4IAvpMeCZliMDX0avT+eWU8EoBmsz7CyMUZYY1D2S0cYrV87PAAdZcSbYCYo39sjEM4G3nTgVt/zmXzsoSQYcChaBx7HQtrd6EvlAO9FAvuFYaIemMyyklnx5I8kUIBwPDvzT1CLxxnG5q52fHsppwIB0Kat8OCCn5FQwDYptZm8y/UlOxV05DLI6REcOjhyOPS3qxrjnjeyVK5d0YJ52i9JJ3He1AN1vaZGrjF3gjh/XGSGiuScHgelIGOGMJtBAbNSo3EelV7ad3VMZ0C0nkJ/QGo+Dg7NeQN4Omh036RE+/UaYN8DIzhQurlQdRiEPN/iT3iMW0KEe+tPtiCuh7zvuxJ30TqNdteExO2edrmK5rDHU4wTFYI3x1Lz6Ct3WG23cRod0BW3aBSGwtGyoJyz4KUPLlFzBAVrlW4yaqbxuBaEHbqZAiS9rbsx/cSz6zTa0KcvjjgXp27IFdkkbsYazbMiz/K7BIYTWwFAKcPM9UIQqD1dHd2FCO7D5Rm8E609IuRN1lVk9ySjHMNqKpGvGdY8kaQM9WuH2PByLZfG2ywE//eNYwJO+qh0LMTO/obvWjeTbLYnvld/yLxBLeapgvkC7Q+Pwm0m/lFGLlnvwL4SGb/zXc92lseBQxuWOGPRnHo07pwHXZe78JjRqrSpvhzwMQE+Ymc/xUfOpzwbn2vwGmV2jfgZ+arwkg+8QMhF/N4ORV8Qm8Qf5XVjRnCh5WqMbWuFyIf+Db0fHQuU6TEVDV0+JfKqOBQu1X5DT8PU6T+KrvuTJ50n0Gotec4FIjJT8uKYnsXtTr7H4bZ0E/k+1jew3awrVl2kKFYG1lj/8z1/SbIRnx4gFDsjf+Wef8BkWnGXRofG/7stPe3pWpwP/9376k5s/oXUg3/xV5528OhatnfVqDTx6cfNrP/mTm19/8OWbP/s1X7n5k9rxrF9ftoau/O/f2Pynf/+Lm//68snNuXd/4eaL3//lmy/9Y+/RGqeCkL3w4M4VbXRwbPOez9UOZK9e2/zOz/zE5md+V5sU3H+4Of3+v7L5xq/+os3zexotb6Jv4Pp0x+KHPuypUDwY+uHCNQ87scW5qPvZQO8HRfBUdBk3dP78ZSoUDwPQyyHQFTMBY4kzKPzRgx/jHH5s6Xasplvxpps0v2WmK9Y/D2Cf9K0rhjUjFie1lSxvwBmx4ATnTHfCmcjD2kaw8pwvQ9blEhz47IaEc8FUJ7aVvc70IRn58IIPjs8drVG4IWOe9GeVbsdCeExxovyU5Z4Ma0ZpcCy88FlToThT4qwci2U6VKY8sG0shjq4d7Re4roWYt9nBEKKQjamD7GLlbd+lWMha1zt4P7mmqY49UJspqPta6SC6Uo4Hxhf5OFYMI3rtkY22uGh4pHdb+KkN0YrcHZOyMBvQxVDmfK2bjhYD6eFKVbPat4xoxR7Wsx9av+MnQMaIuswOFiP+H0dH/9Ai4SgdwTHCUdEnzguvDHW9A7B2algi107HZmq1m3MvCUfgfj8cWJ9PSnd7Y62N31AQSYCbRLDm/tMj2E6U+BnGZo+OE7nN6E/YAeu6jpwhnKeG41uGx9HvAO4HdBXhyU1KeAS4GeJhUf7itOA4xCnAvltcDtvLrNgK810RGaLd9EnrwMyGAYZS+9IYCkOgW+5wCd7pk+aA3P+TGO5JAOcpW5n3KXsBQO+ZOpF29SfyYoQeHAgdN0YFhjl2cGSjuLwSmf0MTjO6E91B6wJSJY4FtvTkKCLHkUOioEnkVA6QV54Kdf5LT+3cxjp4MFXwfJPdICJc6F3TYqzo9uYElXEBh3dO960Kn/3Agx85tA0kh5dJT/aJL9hkHXGdp6U1qkue9Ovqxc3i0aHhvV9p8NDCUO2ptFIAR584DtgnTdJVb+xpjfDjXJMONusUr9kV+kLUvRbVlg1koCac7KDZaSiEAIFFaEgEMSmWfcf/LYPBfVN+H6zRiw+laJ8qo4FvP/xv9KOUVoP8XV/6vTmj+vsC57HL+r+t+QkfMufP+9qZU3ED+uU7/P7Rzdfr52dGIlgZOGyzpq4rnMz2rFgROHva3oWC76/QQf08Tz911oz8bsv3vWajx6xoJv4oZ98US/qnvVC7fsPHm3er9PCaQU/otGNj1+5v/naLzu9+SLJw/azv/Ab1zdX5FywRoStcC33E9aNOHP9egdq4Nbmld/72ObGUa2deO7coQus711/ZfPSJz62uXzvxObg3MXNc89f2BzoTJYnB80aufJ7m//1B9c29x8d35z7vC/YvPusXnovZsqTUZ+S8xqORdZYZJQgHT+/Qj9MMPboFPXxwmgZ554Cg7PhAHzH6UTTser35h6bPAw03v5ivOrRboP9fj3s7Vzw61Qeow1sMXtCb9UZSYCsHQto4nyQIDgewt5KljfrXoAMfC0mVp5HVWpa1FjMLWO8p+7gUJyU4XtK6yB4GOE8XJVjceOGRgRk5DPtCZMZA+62Fm9f9yLo+/L6HmW0Qrs+QQP9IN893oqWY8Hhfhju+0yHknOxr/MlcEI8WqF1EoywsNAaBwsH4IYcAXZtghYjG6dYLyFc7wKlexaPM1rBeRSsxWBhL7IzRelAU5MwjO6KzvXrV7S1rNaGsFhb8rPw1pXOiIumeDEIwxkYHj2hijCy0KngcBaRG53jiJ0Qf5yDYyonT0WmYzEN6pScGT+wVV9HJS+OFsYai89l9trpAJbyeVRGdYFxxzQo5Dx2jO13tXWfp0mpzlS+MSpAvT7h0wbDnE/zInRex7nvtI77qnJwpU4zBQ/Hwq1UfJc3xvwGaGYEZBs3wh2jHXYsgIjMTZ+nCTJ2ewcCmNnAAvawAN5jeehXvNqx4JppPUpTm2sngnTzcNmBT1l3aXJPmPlswVQ+vzP/2vreV+QO3RTh8XKEbnRi3bQizVVfpVd4WpaZRKkd3i2nBI1j4dK5hE3JZShyFAjRnNZlp1267cmh8HRDtcF2zAYRycEoGv0JYRjz7VSYbglpZiXkIGC25j9knvKQ6bB0StLp4yoZaDduL+KPc8Hv02WDJvyrPrh1QD8d37k2XfCBQa+mMeCUXsjoDPjGcVFh3dSLr9OF3+0HdOMOmsWHdKcVA8WhvdwVggWA6nbO9l3BclGGZSpBXCYDl6QVTzkGkBG7rJChOP7tD0amRJb75eWOlAFUsZa36Aui9RaFLvADl3osHUKRsDoW0UN//4N/8ZL6283mQ9+QnZI6fff6kZ/L4u3D4NhB6l/+2tXNb/5f7ZaoUQgCjsOX/NGTXrzdtNix6ad/9YodCbar/XI5Au//wlObj2rb2HnqEtOjfkELwu/JWWAtxAfeu6eXaI90+N6Dzbdql6kOHK73s//xqp4tjzandb7F931zFoZzwB9rLX7797OzJOV7jxakf+NXntvMO1I9rUzNY72uGvhs1MBrOhYf/fEX3EGOTlIdoeO+6oGnXwUPvh4JSD85dcGjP60IAPrYoQCPB6fu7Vjowck0oDFigdWrYMdCxqanNvmBXyMWGE4YwUUTuiygjoOQqVBeqD2NUmC0Mq0KHD7wxoBmpOKUDGccEt7eY2h6GpScB07ZvnNbW3R51IJ1AZoKpTfxjFhw7gPWuadB4ZhopIQREpl3HrGgU6GMJyTXSRnPrLM4c6BRBTkAOBMY7MgI/2PCx9hmTQdnTXiaELjCYyTilJyRYyzGlmwsqGZ7Wc6iwJCEB/KzkBrnA2fjppyTazrbgnUSLNYmAOfRFMnPVC46WrRM7bzK21uVTSqxXOjymJyeHgXiPAzOD2EKFiM6bGO7r9EKFsLzBvlVHCHh95tVFhVzHgcOxZ5wT57UFC6VBaPDb42FwzkYOBukowuPeIh2r0lwPWH+dH1JfuK7ofPn9GHoiN9unHubHSo4Iw5MKUIvnm5S1oYdAfNLGzVtmu+wxOIcoP829CNb5B3OrzjZSWlHpQiV2eO7lm+Wnzj05jzLXfK1YwFv2goGcrZqxolYHAmbRaUD8Hdp7vJsvlz755syxwhMGSObIaDNX8m1S6/TXafW3VR/Fd2i2UxFyNmtc9qAhYKXyoe5VzwnFIlUd1z515WP61k6cttzP0N7z65QDePaFR+mJMaB1JQI/VYWZ80ETdflbAG5cdypQy4nz+21ZRNwUIOE/jq0Loxb7d1plIO2po/LDUrzbB5Ff6FWVEkvmEZpmIUfemopdN3F6bwmAOgOX5KsSyIKC+gSW5QXmHwrH5AhgG8GQLMeCR1Rxqy7TmaUZMFRm4c4JDsoPlgpjWJYp41l5EYIh6qxojBTb6JcGydpuUOYgre+6p4kAzSOzrH4trffGoto4o198/wk8Mx+Wng9cJC6osPyIHVmn9/34RSZlsRZEc2TrWD7BO/G4Ll5VSMN50QHOGjTfhpnF+6szrDYXTyun/Hm8s0Hm8PywH89ZWo+63XVwGeTBl7Dsfiwt5vlwdMPH//M6RzL4OLNdjsW/E4bLv1zOoWlwILgXwYWBm62jM18eq+v0EMfgXL6c6ZCQYcfK6du29CVEctoA0YbnTU/ZvOklwCOKUM4CZo2xIgAi7ljvPPD5s08RmsbhL2wGuMX412HhMi4x/nASGONBM7FzVu3NTKg7Vy1M3dJfocAAEAASURBVBQGHM7PHX04L+K+1ixgzMEHh+YUjoXewuvxnxELHAvJiyyMWJwWj/MHBzoLQgueZXD77Ty6LH0ySuLF2HIu7DCY7gk5I4KX08A6B7aU9daxmuaUXaPkHKlsJ7X+gTf/vG1llOLGDZ25oc89ZJSecKLQO7XClLRldyuNLsgRoWxoBh15lKIOhYMmTs3Baa3bkLNB/XG/L8fipHaD8htgyX2f8zPkxHgrW+mHetpXWZmaxSgK6zGOeKpVGXOSgzQcCw4FpB2Zr/TndTPWCY/zOBPUGzCjjQk/xgDNMW+XO6/TuXacdtj38zUGJ85BnAtBCS40wy+OBXo7LIQWxl6mQsVaiEOASUI+Riq0TG8Qye+BW2QgANtl4Mp9h/BZypP7ZQqXDU7V4TCC1fbCP7I1rV26TX++tgxJ8492yta9Qr6RLzK1pEk3iOUP39CAbmhTrwU5IYy0QT90hLToxfxUpuJbELpTaH1x9Sc6RYbUs+pYbXMetfCIhequS4R89Bn0bQT6pKHTojtP70GOEXbqbKTPEdHocsJrYE96ADx6csRx7gHB+U3dD8yiV5JQ9KDpO4F7cMyi6CzYDXXIlfJWcJmJQ0Q0CObTEd8HfsEicyqL4ujy8SB6/m+6uxQi/4yHNrZ0P2dOcpNszYW0oYKXBIqSNkrW43ydquRRH9Au+kzDa1aun9bLIFOwC9ATeCDLM5vvXB0L1L2GVQOrBt7CGniqY/EDtStUHmjphN1f0nnqoeu3sGUsYbS6T6UflULccRs42kl/qweBUDGiMXA5TK5HOujYeWPMQx/HAiOXHaQIGGU4Iawz4E05uOGiiwjbWEMODHgMcRmpOAo4GD1CgcEKP4xmyyd+9PXQY00Gb/sZUYAPBqJLIACMkTt3tZ2rPrf1hp/pSSyA5hTumzKiiTNFCvp2amT4QwPHh5EFytU89ve0oEZv7i+c0SFzOg8Cx4LpP5SBNx6Um12b7shxYH0FAQMHw/40hrlkZHSAcyjuyojHGWDExqM/lFuOBVOLGG3hZG22lsVJwRHiwUrZGf2AJ/UAPmdyMCUKYwW94rx5BEn3jMYwh5Q1FXYOTu+bP47LaS3YZl0IbePWbR24p8MDb2sE5a6cMBwLjDVGZNiW9sL5C5szOlCPkRTO6HhwX6MswiNuZ0n1RZx6RIY2wH1V3SN7P7Rd18IlkN6GRec7Y/oaDqjhVauqj90PdezRiloI3W2r27ev4tnNGfzd0CMEEQ35BN+8hIl8w7lQnD/qncuAK7pNH5wOuzDI6LUhlpnRirzF9pXfDR+JGbnKsShi0DJ/3bf+nFXsLJ14L6UkQ6mTPEXKFMTEsJHbTA3bMgMLLdNV/Y66UiM0S5PXlxkGEhySQxneup/4+zdqvkYCPKFkQaZ8uKTO0Q3t0n2MRiHsYNSZJdabKMCTb/qlTIXSaKpgySe4jKbt26Tp27JRhOI1y7pATrFZ7DAdmb5VWUMz7Yh4fyhXyiJHtuiAM0jO9IRnmYs6MHP2YHpIZMA1Ycqt4D686sIplR7CA2uLb+tjyIKeRGuBhnKVmehSGqdzD3zjwx62bqUVB2sE5QFPGDxK5jmnk0zX5QArbV/STMgTwS6vqYeasURslJMO3QG8jis6hFGmC9AJXJ/ZfGh1LKK29XvVwKqBt6wGXrdjkRJWJ0hvXJ2oDT2MQacJSp0l3ai/qz91l0l+ddQ8sHEC7FjU22k6dh6WOBT+EK+HOZ0106EwqjG0/dAv/uRBC2OXPNYEMHKQU7WzkFioxjcNwdCf98MxW9PKCZEjwhQmHik9HQiRebj4zb6mQd2Vc2GFyeDGybim3aJwOB480FQkOz9yAmSYM8JC0SkPpWYqEY7Owb6mQWl9xTm9xT+jt/8nNe3qiKYaYbLgVGR0ROs5RJM3qiCzgxILns+cwZA/4TQOwuM8DWAweplGhJPCSdiU7Z4Md07XvosDIgcFOByOLGpnlCS7MuE4sFD8tj69bS46Qm5Px2JxuuIY/6fFn7Un6MmOxv5ZKln4tzdXr17e3MSRkXNxj3UbcmQIjACdO39uc/HiczpQ7znLSHvB8XiWcllmTY9itEJ1m7YUA7x3lvJIjqmRro/w++FN3aS1SS3VHgJqk0C6IFd/VIa/nOt0yug2p3Tink4ko1OeWoBEe3EsiFOTCTZC+kbXvucasMBy749kIKXp2cFA3nojDswybWqht7AoOqaXVPi03J53D40qk98IY9iYbpwKyhU18E0zKhmdGpokJZ3yCr0gk0tacAp6uizyGUt8O7QOTNllXhyLlsGwTXvCJV+UK5s7C+j7vPUmd+GVDGQhpi+uumkZwPELC/0ecBb4LXO4oPWI7kwASTNadZhjYXrANTDxRaximcwn62sHv2iEZjl9fnETfTcr6LmfFXzLbIgGaDpcCaVTZ5deiYeqIZ76NcOhw+0yKxeAEA+dRtgBPYyJ6SmjURpmrmPSkp92YFZiGJa5a7xcO03wHW0a3b62CzH4Rx6Q+LRUfV2ItdzwW3KVX/TTZotM85p0vyAtNIMrakr60Levi7fR7RpWDawaeOtq4KmOxff/UBZv9wPSfae+0vmrIySuByCGEtd+yM+dL6opDIE3ThwBG/oyFPXvN+/gsYCWN+55Gxsjj4ep32SX82BHQkik8fDH0bDDwVt7GfF2LORckOYpNbCVHDZMkUF3OCJMXzopY/m4FiNj/GN0eI69DA7wvNOU4Hlrj+HPGgucHQxyjPErN3SStd74k04ZcShwlvrhz6ODxczIs6fRiv29U/4cyOA+I+fCjoUMbNaTmaaMcnag8qJn8VHhfFbEntZXMHWKERvWQbD+4q5GNXAGcCpYv8CIAo4Dax3uydjnJO57clBYa4HMjNzgoHj6lWQkUF6Odr/H21uVgbKTxta6yHFNC9dZmcFBfAfa/Yn1FaflEJ0RL9Zy3JG8l9nK9tJLciyu6ZC+Wz7hmwckozb7Wkty7tx5ORYXNxcuPGcHgjwO9EOWU/s4V0zvYsQii2VjPFGfmnJF/bq+qg0pjm5J63QXRF99v5uXtQ/UhEJdiLZhxpVpeMB5dEP3oSVexb9pdrtuXiapNtvp3HfAlkh6HJimgYPXjgX0gcGk5dp0MmIWSkknjvANR3lDn3wcx/69GJ62o4+3V7VDERmaPtRorwmTUia9Ir+ZGKhho+e5/ItMLb8LXrQXGbv8OFfQ5TqoUhjCpANuZz7GLwxxMqx5Wy9AJ7iMyC5aTZ+0Ud9q36yZyMhopkQximG9LURU95kyiay9xgKaI5h4c5CssCSTMgygJbJbFuTpsFBRipBbV/DuQiBfp3PtMg1+zRSU1icMCrbhtvJKgOYPzScGylXZhp94LD1/sJvOTK1lB4L8hmnepGd6mWNAEVHxXSDd5Z5r4zaMAbe+dhyLSdYtHJPMLyE0SeATqaKr4ovMxYPYLLeTxYO01lGBLhiN7JStG6cEW1OhVsdiW3Xr3aqBVQNvOQ081bH4PrabfeGFFIr+na5THShGtw0DOlM+vufxkuBOuuGD5Qx31MbPiAWOQbYzVbaQbeTpgRtjkKS8vcNgx0nw/Htdn8GREC4OBiMFGa3AGGXEIIuNcRTi9NRjooUTKww64BhF2JORDywBA8JvMWV8UFY7F4L1tCYMbhwLXb3GQlOgWLzNAmjWMAjIZRlv2nnKKLBNLs7OnkYTsl2seHoBdzkWymNcg12e2A2Kw+zsWOgJxaLufY1wnNF0IraphQfToG5yrgbToASDwc9oBo4FunwkIxNn4aHymbaFrAQMdcrcxjJasWErHaN3wwqHkZCbXrB+Q+d33N48kk5ZB3FK/BkROSte586csxHA+o1Ll1/ZXNOIxR3pAt3xZBVlT7tixIKRGeRjW1r4+ckrQ47tcA8OzvkgvzgWcSSA4eMpYqLUwWkYo9XmOp1rt6vOW7BgF0PE1a94B9LjwGZkbIwYID9t13wiC3G36UaerqSb/05a83UrlnE/04tjEafYcLT0ki14i9EJ2aaFw7HsCMSvo9QpfVKHmfYErTgWwHaaInCJlJSvddsqcaFJJ8wapARJI3W3rA065DYfAU7lMbpJVH9RuhWxytJ1wMNjYDR5J45kC8RXPpEvONajk/U1aKIHtXP9dvmwQQK/Yxyyh5z8Tp+D/ootsH55gRMkCexYCL7pIRSyMJ0vsUIkraOiAZ3Wc8pUiMApz8HXhirkVgDkt0L0R79EgMb8MZUJp+uqeXHNKEwRbRnqtmlOt6O+TaMLVzwGq4kOJUYLhjdAoNL+EyfP+cUoqaVJ8QibRXet40f2PIRrvBamdYluyOBr+/faejBay5YbMzOeyTXtSNRyjfI0zriWDF1fna7kRTolLoQQ1gmgNIyzfbM6Fq3C9bpqYNXAW1cDT3csPvzhzUd+vBwLlZEOcBmhKCOvnIq586bvdF+bL2vHnae++gHjkQYMSGgKjgcNxh0jFnTv6X4zyuD1EXqjnQPWsqiyDVBfcTBkAONs8GYeeJyN7tDTcVc3Ll6sw/Biba1JYNQAGhgbD2SIMzKB8UEgnXUJEPKbfPK1u1JPjWI61G1N62HqEtMseLBzhgN41ocYH5N+MOgZteCNP+sO2Hnq9L54a7SEaUCP4C+6rJtoxwL+nFnBCAEnXeP8cLI26yZuYsRLXnTI+oWzp3E8Djxa4d2ZMJRUBpdJxj66td4lnx/q0i6OITQ9MiA5KftdTaFiFALH5ZZ43JGj8YxGSY5qdIEzLDxyorUeB1q0fV+jItd1hgbToG5pZANeBNeHdEAdUGbvtMWictHwA1pGMM7gvkYrDuQMnRQtdsNimptxXY/UMXt5qx3QmBSyu1fe9jtBMqeNxKDYcnQNoHzhNn4ljXsbkzLAvbYCo3IHFn2lLBiXh4emTb27qQvM0orWcFQYj3AZgCmaKn/aSGi7tQuR6yJHyg1n0hYnIUawEi0U38mHT7157/L4WvyLXIwzWrR/Faax/UNxaSqdi6XTNfLPWA3UkrickssYXEvGhkNJ4Luu1HaH0oYs4AxpjGZ+rVxwSTVDvvLxLekKLUOySgYRdX3wu1D7W5yKGrFQn5NRi5QWesOxEO+H+m0wOqgfmHn4qxVh2fomEjm/yt96TjnIUUz/QzfWUfDzbWzB1N12ojNpO60y2jFKa3rb4OFuFqmVpzoWTaMkePzSip6YsM11h8TUx1S9JJ120xAjMuQlZ0ktuIVk5W3T7DYM9CIzfJpS+rmitkU/snVOrsES0/wrMSl8j7KUTM0ht5UImeZdSQsfYTQScF3fpJEFfOGQ9MFv/y59vzmB30HrrK8zZ9LQbzvevHTiwwj0pxJ++WO/ufmfl//fp0JixV018Fmpgf2jJzZfcuY9my/az9bCn5VCvg6hDjSjxs+Z1wF7GMhTHYvv/fAPbj7ywgujX6SjyehEGUaTU8EjzP1nf9HRVmdbl/B3Mg8BOQPQU+rSgaUT606ZdJwFDrHDOGeqDwZmRkygoU85FZkeFXho02tnoVx6bjpw4FEWuzMxasAbdbY3xejoXZH6LTYdKrCci8G0CDpXpggx7QnH4j6OgAwO7+IkxwJDntJkJCXTKCgbZTyi00biFGnUQM4FJ1wf2LFgq1WtL5AM8GOK0PXrN/yGlLfaLPDGscCgZxSCcyluyoj3mgjRZhrU81q/wI5RpxTnpGwJqpwEysLDg0rOJ84GvHBK2FaXQwTRKVvDogMeHCwgvyscdo56VvI9w7oR7ebEAvIDOTsn5ATc1GgF6ypual0F062kXDtzLMJuJ4+2guOE4wAPjDfWA3AIH47FPgvYNXJBOY560XgcQ/SBfEzlYfSKesMJYipY6raexUrv9uPpeNzXhwc45eQzh06jzj39ycZmG/8898sJEx3qf9CbiMw0iQ/egjF9WQqzY0HbgY7MVdP0iIVozz9c2nzTBb7jsDVN8cFxcJ7zU8+SVunAtHFchiYwRQfaTwyWC9mgkd9I63SRwRmCiH4NbPhtuoGP7I5DcA7Uje6tU5VfEe7qOgE2mvCjtymPKPlNWz/sUb6Rjq504w+X3KMP17na4HAu1M7RqdsmIzz1Rxu2AyT5+O34fBDRkbYtgL8tf8oDK8KQF55JSvoUJzrnLWWRLjoD1VSwvsRri4d+W4SMTqV8AZeMTUMJ4ILnJKvaX66H1FdhNfHCIXXON1TDVLmHQM7sLzQ0CaDkBlesgXRtKLWJAZ5I3wLtFiO+Dd3XJrXISNuEfLA6vTlyJc34umm9gFHacQx+zd8ydpmVCH3oGKZpKb0XZFtWUzHmoDPaBHkLcRMbZS/67wTH4oP/4Uc3P/E7v1KaWi+rBt4+Gvjck+c3f+MLvnrzVz/vK97ShXr3u99t2/SNFuKpjsX3ybH40Rd+zJ2pu1P1rDa2yigaoxf15KDTpff1LcZ9EiKb4r51R40xJlqGWR58MY4wTKADqSzaZkF01k7IAJVRC18eEuTboZDj0UZgr2+AgB8jYaq7clJkMOBQ9JkVXtCJAS5HwQ8dHhj6YBhCHwMD2qS1cc4hfjgWdirkbDBy8aoWcCM4sIyWxAGTfE5DTtHRxwf4ydjeZwRAIxaMnDAVCF44J0wpeihavPFnRyXOr6DM11gcrUXbTJMSsM6zyNqKs3rrzy5YlJvRCmRIYPQnaygYUWENh50iOQHZrUqHBDGCIlyuBBteMrI4WRweHJx3hMPwdH1Gxj67OnFQ4bOCuanzMTgnAycEXTHi4EXh1TaoPyShbBzcd1J8XmWqmeTINDTVAdvVaoqVt5vFIRHP0KmdssoQxFlh4Tq7Y1GvBJdS8GlHtLn+yBkgXzLx2Q1dt53fzkXfA9+0uk0dRqPTDCuOuS58eavaNDFaJR6U1T5oT/qo7Y8pLUUMeELj9ZXSJo4BXA6ErlFF9OE1FkrD+RCQrvr4Ktyi/9glQikZGqEDDGXxbcnje0MEpqS0TMBb7sGEyOG6B7YUUTziXLQOnU+9IfGgV/SVSX0QpG19AxMg66YQIkt+iwY2WJzIOBaKy7Hgt+ZdoYhLT/5MIxL0K8gFC+AyYtE8i28Z9+bTX8nSXemzLp192NUyH5LhcjaZqisXWWmjjuCDzupDr0eYDVanAKb06LCIzjozFnQrogtulEPRtjImAHgaYvkayNHdUEbkVa7RoQekr6lNI+7QdlrBjbgjhV/xzrNOfJN6G+mKIKJ1JM6LVIGY2EZOAQBjpy0g/m6dllacFpoFJEKtCtITtqCV3zXU+YJStKG+8zu+u/A+/ZfP1IjF6lh8+ut25fCZ0cDqWETvr9uxoNPmDZ4NZxvPipdjQMfsLpXe0XCOuLecus/qPJVCRwqO8Ksrdqffb5G7U4YXC7wzlSjGJUY2xiSwEGThdpyNcmR4WBVTLvABDhyMdWgxWsHbeg5v81aSTIWQUcFoQz+cMDTAt+GqdOKkZXE5B/lpa1g5FBmx0A5NcgYst/jEKIGW8C2ERJARYsdC8iPDKRnce17jod2cZHh78bJ4sOAaOrzhZ/0ExjpG49VrWseg6UcYRIxyMIpxoDf+x3XAHAWmHFzhSZHRDyMWrJ3AqXigkRavDZHMnM/BgmUb+DLWObsDXbuOhfsMU5kYIZCO2K0JpwIng3SmY93Rad8s1maBuB9OysvakiwKNx2cPwlD2VjHwogKU0rA8SgUfDVFak8jFzgMnr6m9oDjRZmtbww90fD5HDoYkNEPAnmU0m1R+dRR2meuANh88NXA+up2kSv5PSKCvvtjykUT+lZok6ir61nxbivRW1qy6Zj7QjMGClIjczkWoo3cKUsTLrl16/an8jevyLc4FjgrBHNVhatpCracCdpux13uLS7G2y4XLYZPlSnR3Pu7+FRc6lKgfJGv7wOlTOG33IWyXFRmQnSneNVdcAM2487xUceWVfzRHrJQf44QlSL0e3EG2aNu+Y3LqZBu+A35YDyPVsjBkC6jb/AS+K26IKIbxwKnPfSAeAbHscoSDH0v6JV0iCIH8HZkLmfnpEXpbiLTeYvekhJ8GoHu9TEK8jZC0xj6nvEaKP0Hd8OpyM3j9WnCgzoKaSIlLvepleTw22wQpRdqU3DWArDwa8BGHVQ7oSlQZlEJoUEfKJL4uC2UTKRP2pFsRgx6oN1eApdvQ4hdIJMWmgUlGsnr1tj35LecQHT+ktw0V8cCXa1h1cBbUwOrY5F6ex2OxUfVWZYRZ+Mz6xls1I03doKgU6V3dBqRdJUME3c36hQe9MrledEGIe8h6aBtDJNPpgL57Qyc1JtzT4VidEIGaDsW5LO7E/KAlznz3YmLBga9PjggwNmYtlORt+IYDY/YlklSmU69qYQOhifpY/pNycUUIdZEMCXpDs6FnAHuLZMwmMbTIxb9lgt14Dy0DJzEzYJuRk96ATl6wFBHkV5sLaMcpyLrHjRFSqMN6HlPBjlz4Pa1rgJ4z33VNCn0ix48oiM8Riao4DhExDUiIpmZyoVj4TUQGP7SRztV4GPocyYGi7V9ArEcCxH11K8rVy5vLl9+2SMr6IYRHRyBVCrqyrSfHqHxQXqSc0987JTcueW3v0fkuLBDlXeb0pa7LNA3bzkPOBAy/0ySbWlPaw3JccGiEz/IpQP0gDPjqWd1TzslnRbWbUgSRbSqO2WQNNJmgxIcoZtGO9FuswVvxEO+0vbhLrqi4U/JALs2wKFNGZ/V+hFfpVNkdeBSzMBvuWwoD7rlWGAYl2NhXMrcfGengngRb5lgYnmbGfkI5qBrR4FbUhXLHXTETPeUE8civ5MhvhVYiGAN2mDMATlyHxhuSocFZt2YFwklt5CAtxyWoeUBJrrXD9EyzjCtT14gZKRCOizHYmvUypzETW0rcpVjIX1nly34UC6+HK2CVem2CtkAE6yigKSk2+m6q3JBX7iDJO1yogUgQWktI7duK64fZdX8HJOYUR+jsyUwVCBVAcTWdZIGqVm2oYjGAw1IARWc5ZyQnbxkG3Euo9vrqPuFbqgW0SYOu1AoQPoA9BH3CLrO133+EKtpFIouwLQMjb/kNg9dJ9SmZzilb6t3kWqbX9ItRdGKhI827wTH4iO/9fObX3nxv8+qXeOrBt4WGrhw7GDzte/60s1XXnjvW7o8Fy5c8Av7N1qI13QsPsJUKPWWMYh4qz05Ft2Lqp/06IOMQ3fMSs/zTd3p9HBYulO9VKxHAcY0n+7qMdptBKjjB54390yF4q06U3Z4o81DHyMeXhi2LMbGCSEN3Cx0JV+0DZM34TgVJ0QjU3YiI4Y5OAhs49qGuSTSfTp+yhDJoQdPcsC5o61mPWrhE7nLiJMOYBwM4crIAR48Tz2Sg0RZkIPpWO1YwBuDh8XjlOmYRiIopw+x0xQodqZBJhypA73lx2Bn3cG1a9cyRUqLrlVYO13UFfyRCJ1gcDCVg+lerBNh0hcwTMXak8HO6AnBepCcpzQasifH5ZScC5dFsKjoihZrv6JdoFiwTbBRj9ySGWMOntRF61EFMS2cB6ZCcX7FbZ0WjgxHVT7KAA9P2dEoCgcUspieNRnIwhQoHIozZy/WdLEY4l3vR4aBLn1LDuqa4Jqj/ii7U4h2rBJ0gcfj6WlX1CA6GqFoj/spAl9kctuwvtF7aEeM1AHSRXa1V+kGPEulL8jPErZsMd7Br9+F365XmxW9yN/lFlyVaVwlB5SB40+c8qffMbJQacjokGynOa+gK9cCmgoI5qMy9lVJoYceQIxMdWMSTdOEkgJ7yyVtO6W/IlK+SQM3+EWXROuY8lWgjMjGv6+5jw7LKZO8/s3jXPPb7wXvgnf7Fyl+7vkNhZdHLHAsTD+8lrIsvFGkJfYVPSsgt68Fp4uyE0K+ylUydxb6K7iiOuCSEeRZDjsWLrzYVp/l+i2ayNLB+tFNJ7VIEY67hu0rKcSVN4Cb2nIdWQI1ZiWYTzEbMFArZfS1y9P3oRyMIatv+Uo6JJCtcblLziKDJTdgaxPshgqXBT/3oz7rti/DseiKdEF5nqQPBG6m1XzMTYVo/MjUVLm+MxyLucRrfNXAqoG3nwZe07Fgu1kMID9oZWhhwGH40nH6g04q7kXVwOiePpeONm/xuQHOoIrS8bs39jfrEIgAQtSjBRjDuvGIRY00nDh+1MY003NsTogPRqxHGsxT/JjWJOMBQsjx7BGNemikoNcS4FxQFvOSMYxxkdGJuDoZbUgZhO4yGFgCcm9jU1d42KnQaIV3hRr8MJDykMHoMm2loBsO7WMK1ikOmdPVC8i1xgDjHnnvyvBGZzaoZWQzjYkRBk7PFoB3nGLUBsfC07hE/9KlS17rcE8jJzzA+k37Vl2Qg2NRH/JYLM0idox5nLUOGLwHLAaX0c8BdpSXF8B3NVry8isvy5G5Knlu2TmLYfzIcgIkrbk+kNUna0vXbIPLdrPUEdOgbmsaFdvzsvaCbWyZxsZIzKv6sKgdveBYUHfZ5la7R+mUb+rcFaB0j5LI4WQEA/l6vYLrjoYEf+lxfJSSNpk24bIqH8eL/8MCdW1nWREomvdhgEo73LGY+FuWGPhQQ8eUAblN3QxCvMWx7KrfdlDiaNC2eXPebXxH/OITZzL8Y3wTX5woeFpX4k/hhDaYU+7omaxJMCUjE8FX6FmOyONGonbVLxgMGCUqSjopMz16gNxDteNAESRxIro6L4KJTtNYyheERbbIt8hp3ZUuiXt9BU5FOWnWtXUXmnYsSjeUq6dKoShUsMgQEZvfIrWALI5k9X+Xv+GraFWUpmc6nUU5TcaEKhXelCu34G3jgtDw2xp16pbuoNG6DT3DDPykbX+LZpM39nbuuOtykdDwSkPWvnVW8ZrLTfp2mQyprxmTNELqI3FUbSa5dmLABBp8YQBYwXeOB7fTO7txpvRGn/VUeu1ycNvabwq5io4yW4eIYTItj27eCSMWkzbX6KqBVQNvQw081bH4/g/nHAsMUQy4Nrodp4OsDpUrnxmOTpZOM1fiGHX0q/oSvdG701EDSHoFP/hlAPBAx3HA+MQwbcfCRrWJMa1ERrKNy+BjQGM8wJwpQTZCjx0ZOyBBx0EsgcOxYJSAt5IEl4Xy2vATTfiUeF1G4BhZuaM375xsfV+GMsalp+bIiRF4aGuEgDJwjxGJY4Exf+pU3szvaWcopjSdPK4pT5KBhdvomPIxqnLj5nVPg2LBM1O5WE/BKAfrKzD2b+m8icuXXvHOTEyHojyWX7JzQJ1PKpc8BByuTPnSKBBTsuRMHBcP4j3CAF8M+4P9M762HPckG1vQvvTyJ3SydqZcMVpCuTgLgEXhFkgJvudtsAxPRj7OX7i4Oa9D8jCh72qNyB2NWODgePcoOTWM6HCg3wM5VazDYEte+B5HDjkUbEnL4m70gsFHfTJV6qh2lsp6G9YspB24XWI1uM5watUWaFsKbneOVVzp+dM9OBUCTVrgrE/Ra3zSgVmMCLWVygcGdtClLhom8bRJ2LRjQXsxfXHzC+ZibrmQTzR6mk7ToJ0MJ2EWFnjjUG5TgFVgkWWSxyWWzswbvQqegB5QncNUZtMtGPJyXzSRx86OaOh30zS5Qqyvpq77cMKgJRSzLkelIH1CX0MraYXTMMhl2VJ+YFreUWb02B8Wa9MuuK9rw5FmfH25HOIBaf+G1U6Hnqpcg8+km+CX3LqETq4mPhjUHXpSaNqVOi676YA3O9Mu/NCI/MSpy1z5Rh7dC5ZYaJZTQTYh4InTuSjQJpsXNGaKDd/5I2+i42hImXfjmHYjwoe4/8VDMvJZYCCwrXtnBmKSj4Tg0dcm6N60I4SoVHpf61Z4/v3p1hRCpnAbhmvRGejRIfJC2+VAfuiQVnDmO2gqb+Ar0elcg/+hb3/7L96eNbrGVw2sGnj7aeB1ORZ0ksOxwGBto0TpBC7DqVCnno5WGepA6WzdsRpSsG18VOdPJ5sHXQHo0mdJYPTDtxdoY3R621HesMsoM2Px9iF5EcWGKj06NFlEjfHJ23mfI4HBDZ5Cy8W6g+zypKlGDnGQ/EZZvF0u9f55K557CobhfFcOBQuibSgLltEV1lF4hx7JjnLttMhg4VkS50hToFhbIafi9GmdmC3Hgi1fKStTq45IVhb4Ejgn4q6mOGFM8nb+JFOntF6BtQlMkbqmBdTXrl71zkx2PpABh0GGOUYrhp4/0oWnhzFiUcYTdWSnS/KiIy8I17Qk1nswiuFzJcTzmIz6uw/ES+dnXNHailfl5BxT3ZGPsU94VWtUmN7E2Rc4IHeZtiXe58+f13a4z+tAvbPOx7FgOhROAc4BC8LZ6eoOu12pnOjPzo30c+7cOe2KdV5TxU7bUH7AlBXxOqFF3DhW6ODIsxl9StusOur2R5mFMRwL4VJmfxC6AjTTjNOA5rZIOmX0yEKAjLUNQ35oA0cefN2+FPc99a/2Ejym3MkZcluUDs2khOC3sIW7GMTgt3GMIayGDpLl4XvgYQD7T2nFX4gTf2AjL6MLI0iOoRvLFAOp85tWl8HlEV07OtBHpqYhnaG3oW/oIevghwQKvjSfSivZKzNwfIuGiuPgUvsrZXRGl7V1ztXOgORCTkYn/NIh08jidKJHfhupmy5j2IUBaZ0/BCgp0DMBmJiWJZRTl6+kdjkrvUFdroXOgrUdswqdJK4GDwHaPsEyKIlrQvEDrNOURW7LrRoyqL/HF3qeoAJiLNehMRIzJ3914gB23aN/B+hV1gxOHuVq8SKd5B5plLUxcvXvJVSnPOgsvNX5NLs8D0ZW02oCdQV6huk4vPU/brnJfyF2nSpRwXqVHMDz7dSp7KSONGMsX03hu77jby6Jn+YYz5zWW19nlqSh/+53vJZPL7A+1XMsZh5rfNXAqoG3nwZeh2PxY+oP6ewxqtVhl2PhuPRB5xNnQd35ZEywLoFOiX517p2BwVj3lAl3weq4Cs4ds+KsBaDTo0ODNoY2b9a9xqKci2eewbEwZcQzJTh1pw5jvxkux8JGK7T0F6wAY2izvSo7JSEH+MjoURDKirOgsuNM8enpPxgmLIIGF6P9GHD1wehiFyZ2Y/KOTBj0SgOfMjAVCofCp1LrnAqMesrK1Cfe5KO3+zLUmXaEY4EuMLj3tT0rO0GhC6ZIXZdjwXavOBUiMPSELOiXt3CUCdp2oNjuVbr1w1pKw9E5zvQsOzr7Xu/B1rDWleQ4hgOg+1v37myuXr+yuS4n5pHKfFz14ZEFnAvRuXfvgRyKm3J0tFuUHIsHKitTnZ5//vnNeW2Zuydn4KamQN2Xg0Id751kC13tyCW5LmvNxk1OG9cDC71zaOBprcn4nOee3+x7V6zjmnKmMuqcDdoBU6v47MnhYI1FO6redlhtwm8rqUQFeOWBTbugHdHuiBUADUchl8StG6cm/amOhVDcooou5gz8oDHehivO/exYUE6PbpX8CNAyWWIRCX6NeElP1GGM+dDSTUuZa/Mx/zKUSxbaBrjWSKGhhxQ85bYMleYUyxTSfCNXDHURgBfuU8ul9mnHAkB+I/rMjkXKp98d9EeATpe6ExHOnDphuYJbsi+JwCpU2VPOyOk+xeWObDgWvADI9KfSZePpmj5oYWD9mHR+P24XgptDtxVSu2TbEAs0+YFpyEpQuQadHfoLduGCiswjQ/rb0ikZS+4AK7pcwA5KdD9JI3DdlZ5NZXm1PkjNkTnbPec2MYNGb4U1IzgJfpVXMnZxcmuBC6BpNHzK0pmLHiBY7SoFjkYe4y2wWVUtRxOcMht11JNhgpDvrhMIKsX/+qoypYwNCfLj5WoKb6ZjQZ/SYdFfp6gYqgzKDByf1bFYdLPGVg2sGniyBl7TsWDxNqEdCxtwGA/qdGxA6EoHFAeg4rrnZVUMEZDBT0fVeP0WmA6VhzpTizBO6MhwLHgTSjq0cRB8SB5OhQxWRi1CEAMM6YQrWPjItPR5COYjwxdcdjZiuhVde2//Cj4yEzDiPEpSb9jA9VQi4drI1pv7nm5FOWBJR/tAn15DMRwL6Qa5MfbvaESD6VLehUnlQV+MJtix0GjF2TOnN2e1ngFng3JCCz0ztYrTrK/rnAjWV7C1racoydA+LccCI5G3/7dlzCOHy4VgBIQTf9RhvapMTFXyqdoy3nEsCOiFnbYYoTiQLOfOnvU0Kw6ho05OaNrVntZAsCvTTU1fuqIF2zc1aqGK0VkWGj2Rw+FpZeLLSMU1OQd82CHrGembNRrPP/85Xj+C7q/qHI6HGnXAoTijqVbwZ03Jix9/cXNDC9BxpHCYzqh8F7Ujwbuef5edKRroNU0JY5EtDtjF8xoBOXdRTodoSFduB6rHXm8xHFapwyopvai2rf9ud2S63UoXXAlRnb9973zautu7eCkL3dDWgELx0KO9AGt5Kp/2ZDjVT+CNoTgOYGh6VEn4C67ZGp56ZQoOU8u8wLjpULHw9zXwiN/3cLEDUjBJB76NCCBaOcHve+uhdIFGfA8IN7BVW+K3ImJKqN8q5eQj+RzQBeVT/YK/0Ew5wSTggo0FxknSN3oFoqFglfiQZcAueaOMki99SGRDhzgSlJ3+JC8r8sLCcEWr8fm9EIbc3CjNv3HynG9lk2O6vlJOpxTIJH9nWJ8udQH2pfTtW9g3IUW77JBbwKjLyEniopdGJK/yTaO/Qg/9dvmMMUBLQtHsaUHCBHn6Do/gpT+l5INjsg3fX8ja8iYbCaqYFOoQHJIaBzrWbsupzM4LJSDQTxMqnbi+lNG6IlplAR6qvhfdBbXSArCI5gK2ANDpAK9Fnk7l2tI4LqDGgWfH22EBpmX5rg++eSMW/D5aPkemr9Ynus7vaHUsJvWs0VUDqwaeooGnOhYckPeRH2vHohyJYWjtGGoy7jIdKl0qNjqdEv16P8jiVCx4LRfdW7/htUElI8DTFoQMLiMBrAnINCjm4DMVKp31Q4wJGQ7wopP323TBM+XJC36BUwaGhZiIU2jauGOqFAaQUjHi4G15MVSbrwzh7DqFYcQagsAxAuEy1mPCoxXSDaMSEGF6FVOlvAZDceAxtpiSxW5MZzRicZZRCzkLTIWCNtvD3tfb/1s3b3m3Jwx68DFsDjSywSgHxv5djSCwDz8a4BRsdET5cUCYkgTO1qF4Ms5xLhiVYfQFPeKocDjf/p62fMWx0DoI7zSl8vp0bE2HwnmANw4O28xywjYOFIusMfKtN5XN2+5qZIWtdz26ItxTWpR+7twFy8jakUtyTJg2hgPALlHIzDa6f/DixzQV6hYV4NGKi5o+9ZxOEz+r6VPQuqWF4qwlYTocU6A+57l3aTH4OTlB+2k+ooNej0qHjGj0m3I7EKpH+BDaMfY0JNWRHRBlYbTgYLqtCo42YAzyoA2sPnYapGPgbIAW3eFUGCa8GMFa2lNwqCu3PrWDposskSNGdxhLiOKBUfwAx0J6iIHsH5XzKVMHygB1oWUaVv0mYIi8srZzFYLFlphObwIwroxFX7PuQAhOfmvQC00bHZJvHM4oOta16qSdC7Tsj/JEpoJ+h8uNcIApuZDZQdck526kV7Yu8DdR5cW5iSFkZ0J5eUHRjoV0SVp9WgewaI5N2e2IdNEFnqvhG9Cy5IZv6mAJDVQpZDlbX8qirszQugJmxg1Oy5Y7fQ+SRMaNUaM72jDQyUua4vwnQ9HkAeV8lyncmwaiABWJFrlIW9KVr5vOJb3xiHaAZvOOIJ1jjNxApOq+CtBAvprHBE5i79KHELtlCk+whGS8RhZk6SGEBdNZS8RZj30VnGUJZYOYnBI7vfF27zvdDluXlcRZnpL1Q2/iVCj/diRG6r6lzLXT0Fn/XtYRi20drXerBlYNHK6BpzoW3/uDcizGiEU6oBhZMrj8ppY3r3qktvFlQw5G2fq1++0YXxhnGGoxWMDpANzcgbXxjoGG0WfHgmk7erPPlCDm9/Mw4qHS29PSSQvUb9NxBI7JgGUOP7Qx6qHZb1Xp+L3+QvlsZYux3fwtswAwKOF7XGcsZJEwZWJKUaYTYXD7gQisZAEWnXAeA3JjEOJY3JNB78Wf4nlE8mPMHzDdR0byGa2XOC0DnFEYHCRGN25oOtF1vfnnw3QnRkXQ2VmNKHh9hUYZ7iod7VFOtq2l7Dgyt7VG49btO1q3cEtTpO7EuZAxj6wYw5SbUQbvTKX1Ggda57F/oIP25LCcliHPlCjKyhtoRi6I39aIyY0bmuKk0Qr0bZ4y4jGK0Qdv1Gcj+llNRzouOidPaS2IynlX+FeuXNm8dOll6wNjkylO6PvWrdubT770SZ9vgcPC1LDzrK3QqAUOyL372p5WzseDh/e9LuXsGUYznpNjpqlUkgF5cCZoD8fl7DCNDPrbzkPap9tr5dmgp81KH36AElGg7sfDlgTp3fQNm/aG3HxAxpgEv52Lfhij634YxwA3uDkgMwHYxbEInYZCErfHarceteDNu9pIpDSJfEUUU4W2/wS3NW2pZBbLYhEZKIbDkmG5uhy7V5cd2pKrP7R1nAocHyUWuRqxkN74HYUpCkOHHXYcC9eGS15FXCDbA7HeG11XywPPkgn+SVuchzhl3LMWqkctuA+siz7RJEq5u+zQoy4Hb4ulr7r6Ak7JH1KdWuUx0eREF4o3iNuev8KzdNjZxho3RMZNEzTJRd5OjkQqpVD4PgRvSkIPYMyjFUuZlF5yQd28ChfJm8yMGzi+Kx/8BlwiQx2BfPzbmjHqjB99ofK5XK43yrAwLYIF1WXoSl8AH2fcKYbpm+U69EFbWZK34lPyUmKA4d+yADR4PNqsjsWstTW+amDVwFtRA6/hWPztzY9qxKIfWhQQQ597G9MypLn3238ZeHYa6NjnfrNg8+Y3uKZH5zoHdbR01jaCZaRwxSHg7SejD3YWhmORaRb0zRhxwBEw6pmmwyfTdJLPSECcldGDW35gOKiOUQvKAy0gkB/HB77AkE+ABoume8E2DwivvfDIR7Y+5THTOyMxesDUIxv1ku0Yu0IxSoBRr4XSOBcHciyOavTgrhyQazLgX9H2sdc5m0JToZhOg95YB3HuHOsKsj0rp2wzNeuknCxkw7BntOCGDPUbGu24cQPH4rYNeWTmaYeB7Olkwsmaij0Z8Eyt0kF7cixOaf0GoxBeR6KyAo8urmltxW05Kqz/eEZb92IMewG8dGXjWbo/elS6klGPMwKNEyf2JPMp1KMpVFc2n3z5k5tPyIG4e1c0hH9KzgwNDwfo2pWrfsiilwtyKg7kbEGD0QymSgHHtDScivPn5XRotAKHh0JRWzggOGaczs3ZHxy8N863wKjVh7KgJ2A5F4X7dmxphR6x0JUQx4KSqyaNG+MYN4T2KZN0GFl2VgRDnUOHAgNjOOkFx4u43+4agPaYtgoGvxcJKLT8hiDhIPaUjb+0W0YtMIYz2tFggQWqg+LF345FGcRKUpAcJYPf9BYPypmPS2xZUhiKUwh9hVDRj7Gt8omH42qDOBsE8HzAnPRsx8Jk9NV0zDG6MkLdAxZR/V1ZxKOLSogM3Fgcyiy+o6zROeWPw5sRRp+0XWssstai6sayQaPkRvb6RJdTm7BY/grvSCtE/k0oQi2Cim5ogzD0OWBNRtDBJh94woAF3yl8KeabJcWkSl5DjKyhzehnpFdEOBPhuWqc3OXhOlDFwKXcwlW5Ipkzt2CBI3DBkSNO+dzBBnImNegANwfrsCkXTYg6WnQKPvIqo/SYZPF2ErI2HiCFW6QtS7OZ+RMHtsuDPJ2vNMTgM/KJT/zNxwC7MIUHvAJwH1qnQkUZ6/eqgVUDb1kNvA7HQidvd4eqYrprVieJIR7Dqh0LGWA4GsCml3VHy70dDhtp6oSLVnfM3f9WsjvXdioYjcCx8KJnGYSMLmDos/PSTMeGu/lomg9GpA2aPKRxOnjrH+OHB4L+TFeGvoxNLwjH4JR8TJkiPLKzlFEIjFiZR35Q4DDck1EfxwKDPYY2dDCqma5DGlOOPCVJ6w1YL2HHR/R7ChJbzu7JkGa0gjUFR/W2/ZYM6ZcuX95ceuWVzW2cCjkllPEEIxwy/M+eZS0G04+kOcmPvHaeFMfZYYTi2o2bXjx961ZGK3qeP8YdhjVrGMDpg/nOyKmA9p7k8JkVkh8zmRO3gUdn1zSFiQXkXoNShiP1jLHfD2kWcp/Q9Cc+ma52wo4GBtylK5c2n/jkxzcvy2FiqpbP6FCZONPipuS9LWeI+j3F9DDJg+FPfXkRuJwipqRxiCBOBx+mkWHwU+d8qB/4npSTxonhR2o0xc4D5RE9RjS8EF11hAOUEQvqOm01zTV1T5lp5dSjp/SIV9o1aocnDkOMKfRqWtEEWLSuwNHmBOu2Rn5YBUb4QFK/+dTvBnnIypdxl9EgDHfyLLWv0N4OzVtX8c+ogmAAS/F84bZxm0Jlu9zQbNmIN4zTxTN6yDVbzYoXjoWdizBjkwc7F/QJOFDVN7jNQGgK5j0zmfLERXfRYydb9qnsi3ODHKV3GbKMori+SFNbom5Ji7M202zmXR+5ws9ltS4Ds0zfkmRDaZHMt3zNsllf6DBtJpBUB4BltNMOyGi8ujcpI0QLlgAYw3EnCOmWuloC/V7fJZI+Q2nKcIrAkWYLTdmmspsIwqAXHKQdSaZUdBt0S5zc8N1iJ6UpCJfohNM5Xa65vgMqYP2D1+Iu9VL6MFHRVWjpcm3uQjajQdGwkxiFCx8DDhmBqRSJsYsBmnIbwFSmrwZvknXf4G/2iEXreJLQ0U7v9s9vh6lQPNc4lXcNqwZWDawaeJIGXqdjAXo/TNIT0qFj9NEBtQHGlCKMPvLcMXU+xpk/gvVb2tCgM3Vnrwg43UfLJtDbRr1xlEFBx43haSfABqIMRYyWoo1kfkjiDIi/37iTp3Q6Q4xURiyY+uC3xXpI8DYTfIxnphLZWZEBahlsBImWePjtPMapLAg6WBYfYyxDD7qkoQOmXZ1g9EDyUc77mnbBNCichbt3tEZCRg1lZW0IhjfODHxP6+08W7EelUF8XW/vP/7SS94+9r6M7mckI/AY/my9yvoKFc+LwjlQkHIy/QpDioXivN3Pdq86hO72XS+iDl/e6veOVegxU8pYP7Gn9RX7ciq8tkLyeK2C8lkgjm4YGbnBzlSMHGCYyVBGs+RZZ65TyUG9uPwY7Yxe5IwJHMTLWpvBiMXVq9dsiDGFielsrLu4qWlfPKjAYe0JIynolgXg6A794kye02jFBe0uxagFenPbon5VH4xC4FCw/Sxb2CJLOxOOy5E4pvwTx1gXEoenHYtuoyI1gtsId6pXt1kpHSeaECM17cgjdnJgTSvZgal2RttrgwadmUbBUa5YHlyVK/p8nKz7DsDhHPT6Cu677ACDXUiFAjwGta5un7RRAyW/SSut0yNBofel5GleJlF5LMTnPnzkhCEf7Rt+qrsetbAjoXp1f8C1+4apfJBskVKYYjJdVBrdpVydDG+HuoY3euqP2mqNTlh/grOTUWkNFzpz6aC61EcxMd3BE4hmP4QPJN/0ki1eUnOP1hbwxGgXJoW+AZ4QF1glb90IQ3D8OdApOH8GShbfSQVHN0U/9Ap+B63rfFAwXt9FTrdVJ5UMLQssFtCO6Ro85zWAZekbQEeJFjzpZQSXOXctY9cJ1CdIsdP9xCxcQr8heTmwSNupra/BtdRWcm4xWWAai+wuRbeRGcrxAC3Jup+TPhsci9YvQqLj/r3kZdnd1bFYam+NrRpYNXCIBl7DsfgBTYX6qNDo+vikK6a/d4o7/hhNcRzydpc4nVPeVJaBprSMXCi98LY6fGgqnTTZRf60kQI93tBn7USMZNOHh/ksjkaklJxlZMSxYIcdGR0YxthCXIWHgewRC95i620+xl/LjPMCXztL9VzhoYF5gMnIlqoY7vghLBjn7Aev1xAdeLIj1A0Zz4wkMMJBgUxP+fBiMTrTfs5pag/Td65qGtTHP/EJL2R+pBEPjG0MbXZrunDxotcYMGLCmQ84M8gKvTboPJrCTlRyZG55rUUWUlNnwFGuLo+vKp8XcGta1qk91idgoGPcn/JCbvhz5sRtycUDhTfn1H6cCjlHktnrSgTH6IAUap0KxA4WzgMPJaZCvXL5kh0JOyAarcHYvKORipui7a1yhYuDgFxM6brJNC6NwnDPehA7FZq2hRNkHSudesZYRAYWv+M0ZMcy9CtHR2Vhy9ujOoeDaVknNT2Lso0RC+SVTmgHBKo49ZvKJp31Ij29DxgesN2OaBfwZnTMbQb8eggzQoXzipBQt1ENv7AynA0QGzdQnoMxnAA92mo+xCfHIhBKC9+QDozbPriSl3yH4l1oLqsLTXrBCCOglnXSzcQXCOSAsOXjd4VDYecijsWQs50J6cgLuV1vC10z4ytEfRsJ8p38TtF1FKZkMC7l7LLym5zXUsThgYHrDX1Y3jggLkcXPszqWzLSNlx7May6L1rASnGzqJUZuqn7hn8MrBpDeAiqy9bXWa6qj9BKeYkrJiZyLAgSB1RXs77crpspGY4DELk7C1TiwBuXhAqkzeIYAtr6g/cg6Vju2wkyf+gUo3AtwoXodlJJXFQ7y53lMbOkKas4pmxKbT0DkB4hoC6jBDfPUYASpeSPYxGac9l3HYKWyFcIEiaavnVi6zwY0GkdkDJktVBBcFU0rnW60RqLN++APPqzlrHE8GVOQ24+wLZjwflEa1g1sGpg1cCTNPAajkWtsTC2ukd6SHpDd7B5uPCQoSOKoYshJkOzjHLSnUcaRgUPawxcPQzzBpf8pQOGbpwKdWR0ZvrAClgbcaKLYYvRG9qhC+3IlA58MTRkUHiEQW/1/cZd9ESTIDFMp89jYLSh5YWWDUrkFzhx3kxj/CIHD08ObMNh4LHOmo49OQG8dWe6Ebxu6627HQu9eb8nJ4O3unlrW2WRY8EuR5zZgHHAVq0vvfzS5r7gUTSG+2kZ04xWsL4CHTyQsc20JMqP48PVdSLjE+OOdRCcm3FHaxnYFYqy+8FgivrSw2EYWMiNI6GpRSfkXOAIsN6CERRkYk6611aInx9A0gGjFHy8TqHqOPUqnbi+8NpYIJ7RG/Rw5doVOxfUCaMkGPeM/LDz1S0tCGcNCjJVtWh0RNO65FzgWOAE7Em2s5KHQwV95gY6Fm/aBUYk9QE/HDXVltsB8rGAnOldx/U5ocXeJ/VhJylo4gihz4yuQSkhLUNxyChfrcvttfPRgz8qpKeDiY5HqQQLjtssMK4PHtok6w9e3DhebVQ00JkVpmuXv+Hg2fmuQ8MsjgVwcz7waQv8bspwliwjDPiRUqxHqV0GciPDAofkLoyvkSuii5fquKdBOW5HKDSh4zavOsqaC373/FZbJyI4VBCclAlGFYal1/pKuuGMS/riMODs27nQb5N0Oz0Slnobeqy4UqJDdGuy9DeutMitODk9xWuWLfBCEu6uEUkaAb05pvKGbOiRVyCGMU/DIw+5CiJK3Lrnlt86QYmzHOgyKI0YsG5HlgFCztZXEwwp33eZjWkBAmRcJ1JfRPRVV0lhmsiySAB9A+iSK2AVKzmhE9yiqDvdK9D3E4pFx5w2iOhuq/xATbwMDIGiNUWM17CUGToGE3zLiBANY1p8KRO4QZI062k7rWn4ugU8wYEneQ1juUsGkkX2zVxjsatHikXYLX/3ezxfWPvGVFpeeq1h1cCqgVUDh2ngNRwLtptlxILQPWW6z34zRXdICp0RRiYGuKcRYUzQgVb6MNRloDoN58LwXEVBcHDAFGIalI2j6rwxI4FhOlQb1KEdGonT14Onjlr4eVubN5UetZDBwRQRuCAx9OBvR8WjCHpjjrxDPhfKxWZROAZkDNg6c0JGC2spoMUC8H2MWBnnlN0dMNN5ZCD7HAs5FigauSgnZfDOTFo7gLHLuo1r165vLl16xWsrkIPdoy7ozRC7JJ3ReRcY+jgWnPeAQ2GDHiNNgUcTRhRGFQfzsa7jrh4COBZ5KOBgZW45285tFnl0AAASN0lEQVS2k4VTdVwjAqzj4GGBE3NB284y3YgD6zD8HwreulfZfHq59ADvlCPOFvnQhzb1ZgdMBjxpN3QGxU2tGcEhOYWONLLgaVtK49TtIY+NvY11kUMFc3YHC9b3tcUuDhsOASNMYietU+ZM84qzo1GTai/A4lTgxOA04VSc1IF6LFBnKhbOiNdgUN/6QI1/vnzra/KWtoXhUwaseB/RaIadGjkxgclIVvTd08aaJrTiDJPiGlNTUGstmWmv9fuKAIJBoKS1UQxm5M01hkH0QJ7LL5QhJ/rgM2hyG5qGN8r2fdPvfK5WjSPRk2mYtEqgOo5jgW74jWHAt0OT3xmHWWZKVPUJaj/mY3L6Klqw2JXPVjsZAprzIgOI+hfP1nvauUYt/HtLPbT+Gofr+BRdpZgHpU191m8L2HJEzKygkEhEfAF16CgpW/fuK7fqoIDM0xwD3/TIRi0T3bleXI4mUYxLkk711ThkOLMgDoFXibfwDrvp+qL37ND67Pspa7TTKuIAiZ51K3EmSgsYYo6MEZnS0EuVBdBJr4NIoxkssInqW/Bk840syVUCaQFyXlLy3c+6GbbrfisNGvXZIqy0kC7oltnXSqvLm3mOxazHlBT1tPI6JfrmN8QzjL6cXQKv6rDUlzR1t39bC/QaWzWwamDVwMY7mX7gAx/YUoVe7D96xDkW7AqVTjQ9X77T+RDvjogUDE4+GFwx3toAloFBh45FWDjgMY0EOAxB8qDHhBuvr8BYwEBRIrBM/+FNNZ/Maw+95g/d4VTowUOnyacXb/vQPR5I+njERPxaXo+wILN42OEQD8srcUEBDscCx4GdmLjnbScOC0Y+6ys424FdlygDowW8cb+Dka/OmA/GMjIAgDPDguSTMnqZ/39T5zRc1Sna19RZPxAu075Y//D8cxe9YJn1Ffc0LYlzJFj34OlZkpXF5l5wrqv1IGFfFQ+vuRA/HgQesVEazo7PsvDic3SbNRxHj2t9iJwLH0yHI6O1DLzlZbSCA/gwEnGocHRwhigvZRRLG+nHlUetQh+H4Z4MTRZoMzKAjthu9r7K5BEd4x/RwmwtMsexULnHdrzUt4xrdMVWvRjivcictRNmqi/aCzf8UY/oHMeC+mqDFv3yRs2OBVdGK/ZxLE5b370tLfXogmBKUCAFrvngzupP7aSy3B5ttEsBtEHa79FxenZEjIGLcU1LDu7iVISHGVECGlddE09Oy8Jdt+POn/PC0UBFizg4qV9kea3QdBuu77f4iGbrQEUSffhE/uFMiJf5+pp2nvKjA+lSus4IEddKa52LaPNtOXKNfhz3G3x4wtpfAyQ6lxOhtsdoha+0/cnRsS4sePCbH1f/1RWiqS/qv2DVLvnTXV2doa8KyjIst6Lj9qMo1wUjdR86BhvI4VQ0SkYh8w9Lh7k+WvZkDJAAPvYtOk2TvNJ5yOYbNoLawnQOeAXvbIEMjAC4LhoX0ORPpDoB8hVvHZDUuHNaY/dmGrkHmiDIuTxKsZ6ntFlX1EeLAF7nofOkL/kkJLWlGiLD1dx3v5w6eAtPYi6SWlwR2cHtOmhAE5Vs8mA+Wx0LdNdvIem72Q6dXf34Xe3Wx66O1vtVA6sG3nkaYH3we9/73q2CD8eCcyzccahv3OketxC4GYa6jAhPEdEVQ5DO3B+AuuNVGkYq01byNppdmTJikSklNfrAw8C0syZie8QidCFL340TwYtfP0CUwK5STLPx9B/erAIEXxs6wUVmZOPKyARXb58qo9HGqjv/yIoBe1JTcWzIaovVyJUysiiaBAx5RgsYzcDxgDcjBBjMD+9rOpT+jksn7ILEgmd4M7JxvTrrhxptwDDmjIl3aW3FOe0GxTSgGzLCOb8Cp4EHkPWp7V9Z+M0uTqYnvEfif4/REhn0TCtiGhbrFkizc6N8y6WrCijnQG/3ZfBfPH9B27me9/a3N3UgHqMJ97RWw6Mr4mGnQg4XOsRcRUYcLe/ipLLCA2eK2sKgZ0E4MrK7FSMt4KMj8K/rjRdOBTjoxfWidKZFRcaHhm/nazyqcTRF3w6ndH1MZ4zQvqg3t07REEu3J+9SBU85JXEstKWunAuvvZDTw+5RaaMZcZjbttux11ekbUBTbO2MISuw7RQzckE5ySfHo2WS006O0smDXgIGdEV94UY40KwMrq5bMxXPJzy4EalDO1QmXV9NM7f+bnBfm9+c+FjatrCjGsAhK2/yU1bHMcCr7KZltiq/2k12iEJP1XZ9jX63CEeVZgB6yuaYeSolvCVAeCCI2iROhH5ncS7iXD/UJgo5aFPtBjD+0GcVIMVzKpyczu8z+k9bbx4NW6+1oRD+ifi+o1QddBKg23FDhH/lmhf5MAiTykFVC94cn6mkzha4sOLepSy6E60CdS70J56LzBEhkhcCoEOyJdL8gbK4FMPZCzSxUNG3/mnT1rNxkhMUQfo2cKEw5Qso/Io2sk86Mo2ZpvKBt+4EF9wB1RGT2Mor8rNuBrBKh+wNUq0pehQP1ydXITTNScVFJth8L6ULzmfSsdhtY0uZUxbqDeeCtRa8SGJ0kLQOXd6+X6+rBlYNvHM1wFT73R3kFsdCIxY8YB7vHB9XWHeqGYXIegRP26Hzr07XtKpb9ltfGYYYp5yRAAydLR2UHQRYFGNGEqBlerpmREGdsY1KjD5GO+JIxLjD2OCeaxl6ykfG4LbRGBqkYaBiSNvZkVy8EVeyZfJ0IKWdkEHKAmIM8rzZ5OFQsiODDCuvGRAWZYF3nAscDB12J4Jsicv0KWE5/546aw63Y+SCqUeMyrC24KKmJjFygQHMGRd3BPOqYDFUWVNwQudbsFUsniFGOA4PhhTTmHIatqYzaU4sb5jsXMjJ8AiGYJCJBSJHRAfH5eKFi15MjpPHKdvs2sToByMCLrOmIFkZ1InKwEJ66oJwV6MgTOfC4D+phdKsDWFhOsYea0KQByeE+qY+GbFopwInEg1TZxyGx6gKijutsy5wRig7jlDXWxvp6B4dAAsfDi7ECEW/1A8jFjhMjJyc0FQqzrlg2hmnsfPBwTiucy/YwaofqLwhJZ61NDgcqX/n9xtzCgwMoxVqtzgWDqrMjGakPAGLU0E9g5Om7DujuNxKHI5BACzDkMn5wEA3YaZACnn8OSxgS/qEG6DCMeyCMPMIraJa+CrCKIdh4TscCf3GVIeWhbZlHGjz+2JEspwI0aBs0S03SvcvwaBmO76KPnriY9ol7jBogBE/PnYs1BaYBkV7yEJuXjZEHkQyDeiZjnATCX19WzZ+GBS2+FOulAcAIyrSJJb7JouaHguVaL2RaWWGn28hg4AVuv77nutIm2jZrBPaFs/5xoU2drP0C5xtjIlvUVr0EtxtBqQlpDypwYXtNnbDtgDGgaUQwDH2VHbDKSNShWq+URGpJa8ujnVmMbKeBOc8XX1f+h76h3MAtlUBjU73tYg27emW7HwWQHiNekKGJWsHM7iL6NHhZ8qxGDJPUs7R1hvX2aHgvvNm+DW+amDVwDtbA/gBOBdzGI4FU6HceVSuO8K5k+6ekw5VMHRQ7VjwVptPOlt1pMrzaIRxNK1FBodhy7EosqMz5t0QIXQZEcEpyPQqOwc4GHYscEjYqSnTk7K2QCl2KnBSMCz8CA4NUXRHilFjmhiAvMCP7BjovJHGiE2Hy45OeruvNPIwaD3KIniKgn4oVz9DPLVKuDgpODcYQcjmczmUbtqSp6ctgceUIEYYMIqQA6eBw/PYMQpnhaHne1qULUY29jnPgSlSnG/BidU+PE+6xsC6j6EvWnZWNFx9lWFrOQqkebE0jgX6UJmOyfjmJGvWV1BX4F7WLk5MX2IKBQ7EqEfJjk7YdYk0Cs90E5wKnDq2x72oNRrnz56Xo7MvnpnyxII/4O2ICIf90HudB44D9Y5DgZzIh8F+1mda5LwK3pLRThh9iINSbUBp6ItpYuy+xRtq606NeV8OGVOpnsWx0DS1E3J4WN/hOlK5j2u3KEZ6Tsi5wPClkWHs+uRu8Tmi0ZDUv5ug8Lp2u41ntKYdHXShGnU7AAM5yDMN3S/YtKnQJDVOr6gLv0N+LwWkdP/+uApg4C7gSp/xB3Hhpc0jW8Kc13QLppnXtXmCS3zwVSTkRJN/8cinZFD7tKFfPMFDrxDIb05X6X8pY//GAJyEgC+3LkNot462ZAMGx6LensaZyBoL2oaNIMsIDeSFqilP38t95MJhppwgqE5FHx4OQ0YiweNi/QDeckPBiUHju0pkGBwtwoApUvzmttJ9t3wZXqwLrGiWUJ24gFMMya6PrtK6JTaHcY/Q5HVvq9siZzIRx2kta+gEL9nBJX0JXVqlQF9ZhlUkdRBIY4SIAAIIn1kGfDnDbeGmPpvfxE3tK6NNzptojnsiYmAc0X5UziI8+GyJU2m6jGD56k4tz3KDY3wKykd853I2smkXA+A7OC68z1bHouV0+57a2WFlbNj1umpg1cA7VwP0k8M+KjU80bGg96QTpG90/1KdzHjoQEwfG+cY4TISTVzPUUAzXSmPgZ5OkqkSMbjS2fY3nX8HjHsMtjIqZZywABeDU1HDYbxzCNt9GRnDuVAahk5CDCQegD3P2yMgllm0xcAjFpKbdCvGRlHm85NGvq+VHz1gXPE8iWOCU+CFzjK0MUp65OQZyeo3txIGY9iGPm/ohWfnQ/GHGmImZGvW47IJZIgrjV2UMpWKXZeOeM3CgYz3M2cO/HYfx+KkRhXQBzXEGQ+s9WDrVk890vauPs3aQ9jSB7KKzknh7WmKEMb3HU21uqZzKzj1m5ETnBjMn9ZJ73qF8b2EOF3HT2gHJ03feu78RRn1B3YaL+lwPUZLcCQYDUI/1Cx1g65xKhgt4eEEzCWtMcEQxOHkEEHrRE4Lb4nZUpbREBaaE6ectCKPzrB1rRwoDgqEFqd4s8MVjhceM04Fp3JL+XJg5FzJGWLR+ilGMlRuzr1AHjsVckSYXuZ1GNSx5E0brJaoBNrd0SOZSmWjWRA4r8iLTG4HgjFN04BKQozsxPsh3Yb5gFEEGtRRw3Qe9x1MFTilDTh0TJoD6Yrkq9KSB/wywiJWBhPwQj40aYHgm0aRqEtkCU7z95Xfm3EEKMJdlo5znzRaV+RpvViOpu/fLQKlfCQ3H8spHtYdeteHkSs7Ev79Z+TiVaZDkQcsTr7LIZpha9rQXUJkgytlMF2cG/BIHHhbUSUnw/IVMad0XUCr0qnXmQ7JTXa5dixI0B1ARJpu0+QKjK/1tQMz+Bs/oIFvx2BJW2CThqHfZZwlW+ASGzAAVaYvuncZIDfJRZ1Mt2Hmgi7IkBptSHHTqbI27VH2KlthF73lYlrcCgCYfBboaUBqIJl363akVsTlmhJ1jw6m2kbgwashkQPISOAbR7/7O7+nbj79l64Pl+8NsmsabxB9RVs1sGrgbayB3b7FjsUv/vK/3bz88iuj2Ol+8xBauuKR7YgfLOov7QDoiZEpEGSlE6V/7k4XpoHXN9m+L1jdk0RIn94PIGCXD05MkEVVtD0iYkMiiPBqfBPTV3iG/iLDct+jIMkzglEHXssJ7yV7yIXBzBt6DEuX1mVGJ5GVzhgDiLf9nmahdNIwkpjK5AAPfSiPYQUPDDJAJw6OHCs5Bx5R0NVOFgYyzpAC8DbOtQYCGh4l0IOcAG3KiQPD1B9SvQBbBj5vepEL/IbFmeq1MOAmp3QmOpEhC8FxDMjHUWG0ARmGplCB8uIUZtoZOkI21lfYOBcyMInj1Ale+vQUNcuLg1bOmOjDg4XxnoImmdENIxs4FR4pUfloh9Cz4Sk9e7QMndXia9d1Oa1ps9Eh5UcY1z1RlV1frluMYW6jprmdpe66vhvXhTLB5ct6LD0vqYfFWuPNr2DEf1RGo1neuim0XJpGkFruRtuloxKZtq8Gavxt2o3f7aWvnQ43B0ci3NBJ50XJVtHgsqOXlscoyjNcwYRnHIcxYgKM4bocwiz4ZruULSmRNd+dt1uexg1U3/3hrqOME9ofnt4uxmFUJwZb0U8FtwmJxhYZ+D9es0A/WbI5Z4tYISXfv7ti65SdeqysN3RpCXa4vyFaT0JqHk/K7/T3ffH7OrpeVw2sGlg18LbSgB2Lt1WJ1sKsGlg1sGpg1cCqgVUDqwZWDawaWDXwpmtgdSzedJWvDFcNrBpYNbBqYNXAqoFVA6sGVg28/TSwOhZvvzpdS7RqYNXAqoFVA6sGVg2sGlg1sGrgTdfA/weIxL95qPODhgAAAABJRU5ErkJggg==)" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "4io1vzkzF683" + }, + "source": [ + "We want to run the Google Cloud Vision API on a large set of images, and Apache Beam is the ideal tool to handle this workflow.\n", + "This example demonstates how to retrieve image labels with this API on a small set of images.\n", + "\n", + "The example follows these steps to implement this workflow:\n", + "* Read the images.\n", + "* Send the images to an external API to run inference using `RunInference` PTransform.\n", + "* Postprocess the results of your API.\n", + "\n", + "**Caution:** Be aware of API quotas and the heavy load you might incur on your external API. Verify that your pipeline and API are configured correctly for your use case.\n", + "\n", + "To optimize the calls to the external API, limit the parallel calls to the external remote API by configuring [PipelineOptions](https://beam.apache.org/documentation/programming-guide/#configuring-pipeline-options).\n", + "In Apache Beam, different runners provide options to handle the parallelism, for example:\n", + "* With the [Direct Runner](https://beam.apache.org/documentation/runners/direct/), use the `direct_num_workers` pipeline option.\n", + "* With the [Google Cloud Dataflow Runner](https://beam.apache.org/documentation/runners/dataflow/), use the `max_num_workers` pipeline option.\n", + "\n", + "For information about other runners, see the [Beam capability matrix](https://beam.apache.org/documentation/runners/capability-matrix/)" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "FAawWOaiIYaS" + }, + "source": [ + "## Before you begin\n", + "\n", + "This section provides installation steps." + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "XhpKOxINrIqz" + }, + "source": [ + "First, download and install the dependencies." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "bA7MLR8OptJw" + }, + "outputs": [], + "source": [ + "!pip install --upgrade pip\n", + "!pip install protobuf==3.19.4\n", + "!pip install apache-beam[interactive,gcp]>=2.40.0\n", + "!pip install google-cloud-vision==3.1.1\n", + "!pip install requests\n", + "\n", + "# To use the newly installed version, restart the runtime.\n", + "exit()" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "C-RVR2eprc0r" + }, + "source": [ + "To use the Cloud Vision API, authenticate with Google Cloud." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "qGDJCbxgTprh" + }, + "outputs": [], + "source": [ + "# Follow the steps to configure your Google Cloup setup.\n", + "!gcloud init" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "74acX7AlT91N" + }, + "outputs": [], + "source": [ + "!gcloud auth application-default login" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "mL4MaHm_XOVd" + }, + "source": [ + "## Run remote inference on Cloud Vision API\n", + "\n", + "This section demonstates the steps to run remote inference on the Cloud Vision API.\n", + "\n", + "Download and install Apache Beam and the required modules." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "gE0go8CpnTy3" + }, + "outputs": [], + "source": [ + "from typing import List\n", + "import io\n", + "import os\n", + "import requests\n", + "\n", + "from google.cloud import vision\n", + "from google.cloud.vision_v1.types import Feature\n", + "import apache_beam as beam\n", + "from apache_beam.ml.inference.base import ModelHandler\n", + "from apache_beam.ml.inference.base import RunInference\n" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "09k08IYlLmON" + }, + "source": [ + "This example uses images from the [MSCoco dataset](https://cocodataset.org/#explore) as a list of image URLs.\n", + "This data is used as the pipeline input." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "_89eN_1QeYEd" + }, + "outputs": [], + "source": [ + "image_urls = [\n", + " \"http://farm3.staticflickr.com/2824/10213933686_6936eb402b_z.jpg\",\n", + " \"http://farm8.staticflickr.com/7026/6388965173_92664a0d78_z.jpg\",\n", + " \"http://farm8.staticflickr.com/7003/6528937031_10e1ce0960_z.jpg\",\n", + " \"http://farm6.staticflickr.com/5207/5304302785_7b5f763190_z.jpg\",\n", + " \"http://farm6.staticflickr.com/5207/5304302785_7b5f763190_z.jpg\",\n", + " \"http://farm8.staticflickr.com/7026/6388965173_92664a0d78_z.jpg\",\n", + " \"http://farm8.staticflickr.com/7026/6388965173_92664a0d78_z.jpg\",\n", + "]\n", + "\n", + "def read_image(image_url):\n", + " \"\"\"Read image from url and return image_url, image bytes\"\"\"\n", + " response = requests.get(image_url)\n", + " image_bytes = io.BytesIO(response.content).read()\n", + " return image_url, image_bytes" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "HLy7VKJhLrmT" + }, + "source": [ + "### Create a Custom ModelHandler\n", + "\n", + "In order to implement remote inference, create a custom model handler. The `run_inference` method is the most interesting part. In this function, we implement the model call and return its results.\n", + "\n", + "When running remote inference, prepare to encounter, identify, and handle failure as gracefully as possible. We recommend using the following techniques:\n", + "\n", + "* **Exponential backoff:** Retry failed remote calls with exponentially growing pauses between retries. Using exponential backoff ensures that failures don't lead to an overwhelming number of retries in quick succession.\n", + "\n", + "* **Dead-letter queues:** Route failed inferences to a separate `PCollection` without failing the whole transform. You can continue execution without failing the job (batch jobs' default behavior) or retrying indefinitely (streaming jobs' default behavior).\n", + "You can then run custom pipeline logic on the dead-letter queue (unprocessed messages queue) to log the failure, alert, and push the failed message to temporary storage so that it can eventually be reprocessed." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "LnaisJ_JiY_Q" + }, + "outputs": [], + "source": [ + "class CloudVisionModelHandler(ModelHandler):\n", + " \"\"\"DoFn that accepts a batch of images as bytearray\n", + " and sends that batch to the Cloud vision API for remote inference.\"\"\"\n", + " def load_model(self):\n", + " \"\"\"Init the Google Vision API client.\"\"\"\n", + " client = vision.ImageAnnotatorClient()\n", + " return client\n", + "\n", + " def run_inference(self, batch, model, inference):\n", + " feature = Feature()\n", + " feature.type_ = Feature.Type.LABEL_DETECTION\n", + "\n", + " # The list of image_urls\n", + " image_urls = [image_url for (image_url, image_bytes) in batch]\n", + "\n", + " # Create a batch request for all images in the batch.\n", + " images = [vision.Image(content=image_bytes) for (image_url, image_bytes) in batch]\n", + " image_requests = [vision.AnnotateImageRequest(image=image, features=[feature]) for image in images]\n", + " batch_image_request = vision.BatchAnnotateImagesRequest(requests=image_requests)\n", + "\n", + " # Send the batch request to the remote endpoint.\n", + " responses = model.batch_annotate_images(request=batch_image_request).responses\n", + "\n", + " return list(zip(image_urls, responses))\n" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "lHJuyHhvL0-a" + }, + "source": [ + "### Manage batching\n", + "\n", + "Before we can chain together the pipeline steps, we need to understand batching.\n", + "When running inference with your model, either in Apache Beam or in an external API, you can batch your input to increase the efficiency of the model execution.\n", + "`RunInference` PTransform manages batching in this pipeline with `BatchElements` transform to group elements together and form a batch of the desired size.\n", + "\n", + "* If you are designing your own API endpoint, make sure that it can handle batches.\n", + "\n" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "4sXHwZk9Url2" + }, + "source": [ + "### Create the pipeline\n", + "\n", + "This section demonstrates how to chain the steps together to do the following:\n", + "\n", + "* Read data.\n", + "\n", + "* Transform the data to fit the model input.\n", + "\n", + "* RunInference with custom CloudVision ModelHandler.\n", + "\n", + "* Process and display the results." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "colab": { + "base_uri": "https://localhost:8080/" + }, + "id": "LLg0OTvNkqo4", + "outputId": "3eee9ae4-f4cb-49e5-e03b-9af6aaef8805" + }, + "outputs": [ + { + "output_type": "stream", + "name": "stdout", + "text": [ + "('http://farm3.staticflickr.com/2824/10213933686_6936eb402b_z.jpg', label_annotations {\n", + " mid: \"/m/083wq\"\n", + " description: \"Wheel\"\n", + " score: 0.977976143\n", + " topicality: 0.977976143\n", + "}\n", + "label_annotations {\n", + " mid: \"/m/0h9mv\"\n", + " description: \"Tire\"\n", + " score: 0.977934957\n", + " topicality: 0.977934957\n", + "}\n", + "label_annotations {\n", + " mid: \"/m/043g5f\"\n", + " description: \"Fuel tank\"\n", + " score: 0.958490431\n", + " topicality: 0.958490431\n", + "}\n", + "label_annotations {\n", + " mid: \"/m/05s2s\"\n", + " description: \"Plant\"\n", + " score: 0.95674181\n", + " topicality: 0.95674181\n", + "}\n", + "label_annotations {\n", + " mid: \"/m/0h8lk_j\"\n", + " description: \"Automotive fuel system\"\n", + " score: 0.941456497\n", + " topicality: 0.941456497\n", + "}\n", + "label_annotations {\n", + " mid: \"/m/07yv9\"\n", + " description: \"Vehicle\"\n", + " score: 0.936428607\n", + " topicality: 0.936428607\n", + "}\n", + "label_annotations {\n", + " mid: \"/m/02qwkrn\"\n", + " description: \"Vehicle brake\"\n", + " score: 0.905624092\n", + " topicality: 0.905624092\n", + "}\n", + "label_annotations {\n", + " mid: \"/m/0h8pb3l\"\n", + " description: \"Automotive tire\"\n", + " score: 0.897686064\n", + " topicality: 0.897686064\n", + "}\n", + "label_annotations {\n", + " mid: \"/m/0768fx\"\n", + " description: \"Automotive lighting\"\n", + " score: 0.897505879\n", + " topicality: 0.897505879\n", + "}\n", + "label_annotations {\n", + " mid: \"/m/0h8p7_l\"\n", + " description: \"Automotive exhaust\"\n", + " score: 0.877965152\n", + " topicality: 0.877965152\n", + "}\n", + ")\n", + "('http://farm8.staticflickr.com/7026/6388965173_92664a0d78_z.jpg', label_annotations {\n", + " mid: \"/m/054_l\"\n", + " description: \"Mirror\"\n", + " score: 0.969698846\n", + " topicality: 0.969698846\n", + "}\n", + "label_annotations {\n", + " mid: \"/m/02jz0l\"\n", + " description: \"Tap\"\n", + " score: 0.962297797\n", + " topicality: 0.962297797\n", + "}\n", + "label_annotations {\n", + " mid: \"/m/0h8lr5r\"\n", + " description: \"Bathroom sink\"\n", + " score: 0.933002412\n", + " topicality: 0.933002412\n", + "}\n", + "label_annotations {\n", + " mid: \"/m/0130jx\"\n", + " description: \"Sink\"\n", + " score: 0.930314779\n", + " topicality: 0.930314779\n", + "}\n", + "label_annotations {\n", + " mid: \"/m/02pkr5\"\n", + " description: \"Plumbing fixture\"\n", + " score: 0.920037031\n", + " topicality: 0.920037031\n", + "}\n", + "label_annotations {\n", + " mid: \"/m/02dgv\"\n", + " description: \"Door\"\n", + " score: 0.890176594\n", + " topicality: 0.890176594\n", + "}\n", + "label_annotations {\n", + " mid: \"/m/09ggk\"\n", + " description: \"Purple\"\n", + " score: 0.878831089\n", + " topicality: 0.878831089\n", + "}\n", + "label_annotations {\n", + " mid: \"/m/01j2bj\"\n", + " description: \"Bathroom\"\n", + " score: 0.866840482\n", + " topicality: 0.866840482\n", + "}\n", + "label_annotations {\n", + " mid: \"/m/04wnmd\"\n", + " description: \"Fixture\"\n", + " score: 0.862223864\n", + " topicality: 0.862223864\n", + "}\n", + "label_annotations {\n", + " mid: \"/m/09qqq\"\n", + " description: \"Wall\"\n", + " score: 0.809348285\n", + " topicality: 0.809348285\n", + "}\n", + ")\n", + "('http://farm8.staticflickr.com/7003/6528937031_10e1ce0960_z.jpg', error {\n", + " code: 3\n", + " message: \"Bad image data.\"\n", + "}\n", + ")\n", + "('http://farm6.staticflickr.com/5207/5304302785_7b5f763190_z.jpg', error {\n", + " code: 3\n", + " message: \"Bad image data.\"\n", + "}\n", + ")\n", + "('http://farm6.staticflickr.com/5207/5304302785_7b5f763190_z.jpg', error {\n", + " code: 3\n", + " message: \"Bad image data.\"\n", + "}\n", + ")\n", + "('http://farm8.staticflickr.com/7026/6388965173_92664a0d78_z.jpg', label_annotations {\n", + " mid: \"/m/054_l\"\n", + " description: \"Mirror\"\n", + " score: 0.969698846\n", + " topicality: 0.969698846\n", + "}\n", + "label_annotations {\n", + " mid: \"/m/02jz0l\"\n", + " description: \"Tap\"\n", + " score: 0.962297797\n", + " topicality: 0.962297797\n", + "}\n", + "label_annotations {\n", + " mid: \"/m/0h8lr5r\"\n", + " description: \"Bathroom sink\"\n", + " score: 0.933002412\n", + " topicality: 0.933002412\n", + "}\n", + "label_annotations {\n", + " mid: \"/m/0130jx\"\n", + " description: \"Sink\"\n", + " score: 0.930314779\n", + " topicality: 0.930314779\n", + "}\n", + "label_annotations {\n", + " mid: \"/m/02pkr5\"\n", + " description: \"Plumbing fixture\"\n", + " score: 0.920037031\n", + " topicality: 0.920037031\n", + "}\n", + "label_annotations {\n", + " mid: \"/m/02dgv\"\n", + " description: \"Door\"\n", + " score: 0.890176594\n", + " topicality: 0.890176594\n", + "}\n", + "label_annotations {\n", + " mid: \"/m/09ggk\"\n", + " description: \"Purple\"\n", + " score: 0.878831089\n", + " topicality: 0.878831089\n", + "}\n", + "label_annotations {\n", + " mid: \"/m/01j2bj\"\n", + " description: \"Bathroom\"\n", + " score: 0.866840482\n", + " topicality: 0.866840482\n", + "}\n", + "label_annotations {\n", + " mid: \"/m/04wnmd\"\n", + " description: \"Fixture\"\n", + " score: 0.862223864\n", + " topicality: 0.862223864\n", + "}\n", + "label_annotations {\n", + " mid: \"/m/09qqq\"\n", + " description: \"Wall\"\n", + " score: 0.809348285\n", + " topicality: 0.809348285\n", + "}\n", + ")\n", + "('http://farm8.staticflickr.com/7026/6388965173_92664a0d78_z.jpg', label_annotations {\n", + " mid: \"/m/054_l\"\n", + " description: \"Mirror\"\n", + " score: 0.969698846\n", + " topicality: 0.969698846\n", + "}\n", + "label_annotations {\n", + " mid: \"/m/02jz0l\"\n", + " description: \"Tap\"\n", + " score: 0.962297797\n", + " topicality: 0.962297797\n", + "}\n", + "label_annotations {\n", + " mid: \"/m/0h8lr5r\"\n", + " description: \"Bathroom sink\"\n", + " score: 0.933002412\n", + " topicality: 0.933002412\n", + "}\n", + "label_annotations {\n", + " mid: \"/m/0130jx\"\n", + " description: \"Sink\"\n", + " score: 0.930314779\n", + " topicality: 0.930314779\n", + "}\n", + "label_annotations {\n", + " mid: \"/m/02pkr5\"\n", + " description: \"Plumbing fixture\"\n", + " score: 0.920037031\n", + " topicality: 0.920037031\n", + "}\n", + "label_annotations {\n", + " mid: \"/m/02dgv\"\n", + " description: \"Door\"\n", + " score: 0.890176594\n", + " topicality: 0.890176594\n", + "}\n", + "label_annotations {\n", + " mid: \"/m/09ggk\"\n", + " description: \"Purple\"\n", + " score: 0.878831089\n", + " topicality: 0.878831089\n", + "}\n", + "label_annotations {\n", + " mid: \"/m/01j2bj\"\n", + " description: \"Bathroom\"\n", + " score: 0.866840482\n", + " topicality: 0.866840482\n", + "}\n", + "label_annotations {\n", + " mid: \"/m/04wnmd\"\n", + " description: \"Fixture\"\n", + " score: 0.862223864\n", + " topicality: 0.862223864\n", + "}\n", + "label_annotations {\n", + " mid: \"/m/09qqq\"\n", + " description: \"Wall\"\n", + " score: 0.809348285\n", + " topicality: 0.809348285\n", + "}\n", + ")\n" + ] + } + ], + "source": [ + "with beam.Pipeline() as pipeline:\n", + " _ = (pipeline | \"Create inputs\" >> beam.Create(image_urls)\n", + " | \"Read images\" >> beam.Map(read_image)\n", + " | \"Inference\" >> RunInference(model_handler=CloudVisionModelHandler())\n", + " | \"Print image_url and annotation\" >> beam.Map(print)\n", + " )" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "7gwn5bF1XaDm" + }, + "source": [ + "## Monitor the pipeline\n", + "\n", + "Because monitoring can provide insight into the status and health of the application, consider monitoring and measuring pipeline performance.\n", + "For information about the available tracking metrics, see [RunInference Metrics](https://beam.apache.org/documentation/ml/runinference-metrics/)." + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "TcBFS0rluusJ" + }, + "source": [] + } + ], + "metadata": { + "colab": { + "provenance": [] + }, + "kernelspec": { + "display_name": "Python 3", + "language": "python", + "name": "python3" + }, + "language_info": { + "name": "python", + "version": "3.10.7 (main, Dec 7 2022, 13:34:16) [Clang 14.0.0 (clang-1400.0.29.102)]" + }, + "vscode": { + "interpreter": { + "hash": "40c55305dca37c951f6b497e2e996ca59c449c4502b9f8a4515c118ec923845d" + } + } + }, + "nbformat": 4, + "nbformat_minor": 0 } From 416de1d2cbd8408bb1f97797048d85abe54f01ab Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 7 Nov 2023 10:46:41 -0800 Subject: [PATCH 396/435] Bump golang.org/x/sync from 0.4.0 to 0.5.0 in /sdks (#29310) Bumps [golang.org/x/sync](https://github.com/golang/sync) from 0.4.0 to 0.5.0. - [Commits](https://github.com/golang/sync/compare/v0.4.0...v0.5.0) --- updated-dependencies: - dependency-name: golang.org/x/sync dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] <support@github.com> Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 2 +- sdks/go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 6f64472eb8a2..f0aae11aff36 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -56,7 +56,7 @@ require ( go.mongodb.org/mongo-driver v1.12.1 golang.org/x/net v0.17.0 golang.org/x/oauth2 v0.13.0 - golang.org/x/sync v0.4.0 + golang.org/x/sync v0.5.0 golang.org/x/sys v0.14.0 golang.org/x/text v0.13.0 google.golang.org/api v0.149.0 diff --git a/sdks/go.sum b/sdks/go.sum index efffec5951a8..e42067fd981c 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -570,8 +570,8 @@ golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.4.0 h1:zxkM55ReGkDlKSM+Fu41A+zmbZuaPVbGMzvvdUPznYQ= -golang.org/x/sync v0.4.0/go.mod h1:FU7BRWz2tNW+3quACPkgCx/L+uEAv1htQ0V83Z9Rj+Y= +golang.org/x/sync v0.5.0 h1:60k92dhOjHxJkrqnwsfl8KuaHbn/5dl0lUPUklKo3qE= +golang.org/x/sync v0.5.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190130150945-aca44879d564/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= From e6f3413567b58570bdd91f7b74debfa0334d3b45 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 7 Nov 2023 14:08:31 -0500 Subject: [PATCH 397/435] Bump google.golang.org/api from 0.149.0 to 0.150.0 in /sdks (#29326) Bumps [google.golang.org/api](https://github.com/googleapis/google-api-go-client) from 0.149.0 to 0.150.0. - [Release notes](https://github.com/googleapis/google-api-go-client/releases) - [Changelog](https://github.com/googleapis/google-api-go-client/blob/main/CHANGES.md) - [Commits](https://github.com/googleapis/google-api-go-client/compare/v0.149.0...v0.150.0) --- updated-dependencies: - dependency-name: google.golang.org/api dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] <support@github.com> Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 4 ++-- sdks/go.sum | 8 ++++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index f0aae11aff36..e0f079eb1875 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -59,7 +59,7 @@ require ( golang.org/x/sync v0.5.0 golang.org/x/sys v0.14.0 golang.org/x/text v0.13.0 - google.golang.org/api v0.149.0 + google.golang.org/api v0.150.0 google.golang.org/genproto v0.0.0-20231016165738-49dd2c1f3d0b google.golang.org/grpc v1.59.0 google.golang.org/protobuf v1.31.0 @@ -178,5 +178,5 @@ require ( golang.org/x/xerrors v0.0.0-20220907171357-04be3eba64a2 // indirect google.golang.org/appengine v1.6.7 // indirect google.golang.org/genproto/googleapis/api v0.0.0-20231016165738-49dd2c1f3d0b // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20231016165738-49dd2c1f3d0b // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20231030173426-d783a09b4405 // indirect ) diff --git a/sdks/go.sum b/sdks/go.sum index e42067fd981c..94fe1436ab85 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -684,8 +684,8 @@ google.golang.org/api v0.14.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsb google.golang.org/api v0.15.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI= google.golang.org/api v0.17.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= google.golang.org/api v0.18.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= -google.golang.org/api v0.149.0 h1:b2CqT6kG+zqJIVKRQ3ELJVLN1PwHZ6DJ3dW8yl82rgY= -google.golang.org/api v0.149.0/go.mod h1:Mwn1B7JTXrzXtnvmzQE2BD6bYZQ8DShKZDZbeN9I7qI= +google.golang.org/api v0.150.0 h1:Z9k22qD289SZ8gCJrk4DrWXkNjtfvKAUo/l1ma8eBYE= +google.golang.org/api v0.150.0/go.mod h1:ccy+MJ6nrYFgE3WgRx/AMXOxOmU8Q4hSa+jjibzhxcg= google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/appengine v1.5.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= @@ -715,8 +715,8 @@ google.golang.org/genproto v0.0.0-20231016165738-49dd2c1f3d0b h1:+YaDE2r2OG8t/z5 google.golang.org/genproto v0.0.0-20231016165738-49dd2c1f3d0b/go.mod h1:CgAqfJo+Xmu0GwA0411Ht3OU3OntXwsGmrmjI8ioGXI= google.golang.org/genproto/googleapis/api v0.0.0-20231016165738-49dd2c1f3d0b h1:CIC2YMXmIhYw6evmhPxBKJ4fmLbOFtXQN/GV3XOZR8k= google.golang.org/genproto/googleapis/api v0.0.0-20231016165738-49dd2c1f3d0b/go.mod h1:IBQ646DjkDkvUIsVq/cc03FUFQ9wbZu7yE396YcL870= -google.golang.org/genproto/googleapis/rpc v0.0.0-20231016165738-49dd2c1f3d0b h1:ZlWIi1wSK56/8hn4QcBp/j9M7Gt3U/3hZw3mC7vDICo= -google.golang.org/genproto/googleapis/rpc v0.0.0-20231016165738-49dd2c1f3d0b/go.mod h1:swOH3j0KzcDDgGUWr+SNpyTen5YrXjS3eyPzFYKc6lc= +google.golang.org/genproto/googleapis/rpc v0.0.0-20231030173426-d783a09b4405 h1:AB/lmRny7e2pLhFEYIbl5qkDAUt2h0ZRO4wGPhZf+ik= +google.golang.org/genproto/googleapis/rpc v0.0.0-20231030173426-d783a09b4405/go.mod h1:67X1fPuzjcrkymZzZV1vvkFeTn2Rvc6lYF9MYFGCcwE= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.20.1/go.mod h1:10oTOabMzJvdu6/UiuZezV6QK5dSlG84ov/aaiqXj38= google.golang.org/grpc v1.21.1/go.mod h1:oYelfM1adQP15Ek0mdvEgi9Df8B9CZIaU1084ijfRaM= From 616075f6440fef00eb2b525482545a3b0e59ca3f Mon Sep 17 00:00:00 2001 From: Yi Hu <yathu@google.com> Date: Tue, 7 Nov 2023 14:17:54 -0500 Subject: [PATCH 398/435] Delete Java11/17 Examples Dataflow PreCommit (#29336) --- ...reCommit_Java_Examples_Dataflow_Java11.yml | 132 ----------------- ...reCommit_Java_Examples_Dataflow_Java17.yml | 139 ------------------ 2 files changed, 271 deletions(-) delete mode 100644 .github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml delete mode 100644 .github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml diff --git a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml deleted file mode 100644 index e63b7652ce75..000000000000 --- a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml +++ /dev/null @@ -1,132 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. - -name: PreCommit Java Examples Dataflow Java11 - -on: - push: - tags: ['v*'] - branches: ['master', 'release-*'] - paths: - - 'model/**' - - 'sdks/java/**' - - 'runners/google-cloud-dataflow-java/**' - - 'examples/java/**' - - 'examples/kotlin/**' - - 'release/**' - - '.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml' - pull_request_target: - branches: ['master', 'release-*'] - paths: - - 'model/**' - - 'sdks/java/**' - - 'runners/google-cloud-dataflow-java/**' - - 'examples/java/**' - - 'examples/kotlin/**' - - 'release/**' - - 'release/trigger_all_tests.json' - issue_comment: - types: [created] - schedule: - - cron: '30 1/6 * * *' - workflow_dispatch: - -# This allows a subsequently queued workflow run to interrupt previous runs -concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' - cancel-in-progress: true - -env: - GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} - GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} - GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} - -#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event -permissions: - actions: write - pull-requests: write - checks: write - contents: read - deployments: read - id-token: none - issues: write - discussions: read - packages: read - pages: read - repository-projects: read - security-events: read - statuses: read - -jobs: - beam_PreCommit_Java_Examples_Dataflow_Java11: - name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - runs-on: [self-hosted, ubuntu-20.04, main] - strategy: - matrix: - job_name: [beam_PreCommit_Java_Examples_Dataflow_Java11] - job_phrase: [Run Java_Examples_Dataflow_Java11 PreCommit] - timeout-minutes: 60 - if: | - github.event_name == 'push' || - github.event_name == 'pull_request_target' || - (github.event_name == 'schedule' && github.repository == 'apache/beam') || - github.event_name == 'workflow_dispatch' || - github.event.comment.body == 'Run Java_Examples_Dataflow_Java11 PreCommit' - steps: - - uses: actions/checkout@v4 - - name: Setup repository - uses: ./.github/actions/setup-action - with: - comment_phrase: ${{ matrix.job_phrase }} - github_token: ${{ secrets.GITHUB_TOKEN }} - github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - - name: Authenticate on GCP - uses: google-github-actions/setup-gcloud@v0 - with: - service_account_email: ${{ secrets.GCP_SA_EMAIL }} - service_account_key: ${{ secrets.GCP_SA_KEY }} - project_id: ${{ secrets.GCP_PROJECT_ID }} - export_default_credentials: true - # The workflow installs java 11 and as default jvm. This is different from - # PreCommit_Java_Examples_Dataflow_Java17 where the build system and sources are compiled with Java8 - - name: Setup environment - uses: ./.github/actions/setup-environment-action - with: - java-version: 11 - - name: run javaExamplesDataflowPrecommit script - uses: ./.github/actions/gradle-command-self-hosted-action - with: - gradle-command: :runners:google-cloud-dataflow-java:examples:preCommit - arguments: | - -PdisableSpotlessCheck=true \ - -PdisableCheckStyle=true \ - -PskipCheckerFramework \ - -PtestJavaVersion=11 \ - -Pjava11Home=$JAVA_HOME_11_X64 \ - - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 - if: failure() - with: - name: JUnit Test Results - path: "**/build/reports/tests/" - - name: Publish JUnit Test Results - uses: EnricoMi/publish-unit-test-result-action@v2 - if: always() - with: - commit: '${{ env.prsha || env.GITHUB_SHA }}' - comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/build/test-results/**/*.xml' \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml deleted file mode 100644 index 464800d44635..000000000000 --- a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml +++ /dev/null @@ -1,139 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -name: PreCommit Java Examples Dataflow Java17 - -on: - push: - tags: ['v*'] - branches: ['master', 'release-*'] - paths: - - 'model/**' - - 'sdks/java/**' - - 'runners/google-cloud-dataflow-java/**' - - 'examples/java/**' - - 'examples/kotlin/**' - - 'release/**' - - '.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml' - pull_request_target: - branches: ['master', 'release-*'] - paths: - - 'model/**' - - 'sdks/java/**' - - 'runners/google-cloud-dataflow-java/**' - - 'examples/java/**' - - 'examples/kotlin/**' - - 'release/**' - - 'release/trigger_all_tests.json' - issue_comment: - types: [created] - schedule: - - cron: '30 1/6 * * *' - workflow_dispatch: - -# Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event -permissions: - actions: write - pull-requests: write - checks: write - contents: read - deployments: read - id-token: none - issues: write - discussions: read - packages: read - pages: read - repository-projects: read - security-events: read - statuses: read - -# This allows a subsequently queued workflow run to interrupt previous runs -concurrency: - group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' - cancel-in-progress: true - -env: - GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} - GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} - GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} - -jobs: - beam_PreCommit_Java_Examples_Dataflow_Java17: - name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - strategy: - matrix: - job_name: ["beam_PreCommit_Java_Examples_Dataflow_Java17"] - job_phrase: ["Run Java_Examples_Dataflow_Java17 PreCommit"] - timeout-minutes: 60 - if: | - github.event_name == 'push' || - github.event_name == 'pull_request_target' || - (github.event_name == 'schedule' && github.repository == 'apache/beam') || - github.event_name == 'workflow_dispatch' || - github.event.comment.body == 'Run Java_Examples_Dataflow_Java17 PreCommit' - runs-on: [self-hosted, ubuntu-20.04, main] - steps: - - uses: actions/checkout@v4 - - name: Setup repository - uses: ./.github/actions/setup-action - with: - comment_phrase: ${{ matrix.job_phrase }} - github_token: ${{ secrets.GITHUB_TOKEN }} - github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - # The test requires Java 17 and Java 8 versions. - # Java 8 is installed second because JAVA_HOME needs to point to Java 8. - - name: Setup environment - uses: ./.github/actions/setup-environment-action - with: - java-version: | - 17 - 8 - - name: Clean - uses: ./.github/actions/gradle-command-self-hosted-action - with: - gradle-command: :clean - arguments: | - -PdisableSpotlessCheck=true \ - -PdisableCheckStyle=true \ - -PskipCheckerFramework \ - - name: Build and Test - uses: ./.github/actions/gradle-command-self-hosted-action - with: - gradle-command: :runners:google-cloud-dataflow-java:examples:preCommit - arguments: | - -PdisableSpotlessCheck=true \ - -PdisableCheckStyle=true \ - -PtestJavaVersion=17 \ - -PskipCheckerFramework \ - -Pjava17Home=$JAVA_HOME_17_X64 \ - max-workers: 12 - - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 - if: failure() - with: - name: JUnit Test Results - path: "**/build/reports/tests/" - - name: Publish JUnit Test Results - uses: EnricoMi/publish-unit-test-result-action@v2 - if: always() - with: - commit: '${{ env.prsha || env.GITHUB_SHA }}' - comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/build/test-results/**/*.xml' - - name: Archive SpotBugs Results - uses: actions/upload-artifact@v3 - with: - name: SpotBugs Results - path: '**/build/reports/spotbugs/*.html' \ No newline at end of file From c68125d8db2f1abd1510a32171b4095bace6e064 Mon Sep 17 00:00:00 2001 From: Vlado Djerek <rage.bubblegum@gmail.com> Date: Tue, 7 Nov 2023 20:25:18 +0100 Subject: [PATCH 399/435] add mtu to docker comose files in python test suite (#29334) Co-authored-by: Vlado Djerek <djerek.vlado6@gmail.com> --- .../apache_beam/io/azure/integration_test/docker-compose.yml | 5 ++++- .../apache_beam/io/hdfs_integration_test/docker-compose.yml | 3 +++ 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/io/azure/integration_test/docker-compose.yml b/sdks/python/apache_beam/io/azure/integration_test/docker-compose.yml index f22d66feb1aa..9a7d1cdf6626 100644 --- a/sdks/python/apache_beam/io/azure/integration_test/docker-compose.yml +++ b/sdks/python/apache_beam/io/azure/integration_test/docker-compose.yml @@ -48,4 +48,7 @@ services: - "azurite" networks: - azure_test_net: + test_net: + driver: bridge + driver_opts: + com.docker.network.driver.mtu: 1460 diff --git a/sdks/python/apache_beam/io/hdfs_integration_test/docker-compose.yml b/sdks/python/apache_beam/io/hdfs_integration_test/docker-compose.yml index fed64d1e8ba9..41410028ca34 100644 --- a/sdks/python/apache_beam/io/hdfs_integration_test/docker-compose.yml +++ b/sdks/python/apache_beam/io/hdfs_integration_test/docker-compose.yml @@ -59,3 +59,6 @@ services: networks: test_net: + driver: bridge + driver_opts: + com.docker.network.driver.mtu: 1460 From aa92afc1f6cf24981848e8930313232f8306c29d Mon Sep 17 00:00:00 2001 From: Damon <damondouglas@users.noreply.github.com> Date: Tue, 7 Nov 2023 13:15:34 -0800 Subject: [PATCH 400/435] [RRIO] [Test] Create test Caller and SetupTeardown (#29262) * [RRIO] [Test] Create test Caller and SetupTeardown * Fix argument checks; Implement HTTP * Revert go changes * Patch code comments * Add missing flag to documentation * Patch per PR comments --- .test-infra/mock-apis/README.md | 4 - .../beam/checkstyle/suppressions.xml | 1 + sdks/java/io/rrio/build.gradle | 19 ++- .../EchoGRPCCallerWithSetupTeardown.java | 96 +++++++++++++ ...EchoGRPCCallerWithSetupTeardownTestIT.java | 126 ++++++++++++++++++ .../io/requestresponse/EchoHTTPCaller.java | 92 +++++++++++++ .../requestresponse/EchoHTTPCallerTestIT.java | 120 +++++++++++++++++ .../io/requestresponse/EchoITOptions.java | 61 +++++++++ 8 files changed, 509 insertions(+), 10 deletions(-) create mode 100644 sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponse/EchoGRPCCallerWithSetupTeardown.java create mode 100644 sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponse/EchoGRPCCallerWithSetupTeardownTestIT.java create mode 100644 sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponse/EchoHTTPCaller.java create mode 100644 sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponse/EchoHTTPCallerTestIT.java create mode 100644 sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponse/EchoITOptions.java diff --git a/.test-infra/mock-apis/README.md b/.test-infra/mock-apis/README.md index ec94eb45a199..e2148d390a0a 100644 --- a/.test-infra/mock-apis/README.md +++ b/.test-infra/mock-apis/README.md @@ -51,10 +51,6 @@ flowchart LR end ``` -# Writing Integration Tests - -TODO: See https://github.com/apache/beam/issues/28859 - # Development Dependencies | Dependency | Reason | diff --git a/sdks/java/build-tools/src/main/resources/beam/checkstyle/suppressions.xml b/sdks/java/build-tools/src/main/resources/beam/checkstyle/suppressions.xml index 3f052508f2dd..c30c48f82445 100644 --- a/sdks/java/build-tools/src/main/resources/beam/checkstyle/suppressions.xml +++ b/sdks/java/build-tools/src/main/resources/beam/checkstyle/suppressions.xml @@ -88,6 +88,7 @@ <suppress id="ForbidNonVendoredGrpcProtobuf" files=".*it.*LT\.java" /> <suppress id="ForbidNonVendoredGrpcProtobuf" files=".*it.*ResourceManagerTest\.java" /> <suppress id="ForbidNonVendoredGrpcProtobuf" files=".*testinfra.*mockapis.*" /> + <suppress id="ForbidNonVendoredGrpcProtobuf" files=".*requestresponse.*" /> <!-- Flink --> <!-- Checkstyle does not correctly detect package files across multiple source directories. --> diff --git a/sdks/java/io/rrio/build.gradle b/sdks/java/io/rrio/build.gradle index 52119c91b47e..bfd030ce61dc 100644 --- a/sdks/java/io/rrio/build.gradle +++ b/sdks/java/io/rrio/build.gradle @@ -20,11 +20,15 @@ plugins { id 'org.apache.beam.module' } applyJavaNature( automaticModuleName: 'org.apache.beam.sdk.io.requestresponse' ) +provideIntegrationTestingDependencies() +enableJavaPerformanceTesting() description = "Apache Beam :: SDKS :: Java :: IO :: RequestResponseIO (RRIO)" ext.summary = "Support to read from and write to Web APIs" var jedisVersion = "5.0.0" +var grpcVersion = "1.59.0" +var protobufVersion = "3.21.5" dependencies { implementation project(path: ":sdks:java:core", configuration: "shadow") @@ -36,15 +40,18 @@ dependencies { testImplementation project(path: ":sdks:java:core", configuration: "shadowTest") testImplementation project(path: ":sdks:java:io:common", configuration: "testRuntimeMigration") + testImplementation project(path: ":beam-test-infra-mock-apis") + // Vendored grpc library not fully compatible with proto autogenerated code + testImplementation "io.grpc:grpc-netty-shaded:${grpcVersion}" + testImplementation "io.grpc:grpc-protobuf:${grpcVersion}" + testImplementation "io.grpc:grpc-stub:${grpcVersion}" + testImplementation "com.google.protobuf:protobuf-java-util:${protobufVersion}" + + testImplementation platform(library.java.google_cloud_platform_libraries_bom) + testImplementation library.java.google_http_client testImplementation library.java.junit testImplementation library.java.testcontainers_base testRuntimeOnly project(path: ":runners:direct-java", configuration: "shadow") testRuntimeOnly library.java.slf4j_jdk14 } - -task integrationTest(type: Test) { - group = "verification" - - include '**/*IT.class' -} \ No newline at end of file diff --git a/sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponse/EchoGRPCCallerWithSetupTeardown.java b/sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponse/EchoGRPCCallerWithSetupTeardown.java new file mode 100644 index 000000000000..22e2ff9b1a1f --- /dev/null +++ b/sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponse/EchoGRPCCallerWithSetupTeardown.java @@ -0,0 +1,96 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.io.requestresponse; + +import io.grpc.ChannelCredentials; +import io.grpc.InsecureChannelCredentials; +import io.grpc.ManagedChannel; +import io.grpc.StatusRuntimeException; +import io.grpc.netty.shaded.io.grpc.netty.NettyChannelBuilder; +import java.net.URI; +import java.util.concurrent.TimeUnit; +import org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoRequest; +import org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoResponse; +import org.apache.beam.testinfra.mockapis.echo.v1.EchoServiceGrpc; +import org.apache.beam.testinfra.mockapis.echo.v1.EchoServiceGrpc.EchoServiceBlockingStub; +import org.checkerframework.checker.nullness.qual.MonotonicNonNull; + +/** + * Implements {@link Caller} and {@link SetupTeardown} to call the {@link EchoServiceGrpc}. The + * purpose of {@link EchoGRPCCallerWithSetupTeardown} is support integration tests. + */ +class EchoGRPCCallerWithSetupTeardown implements Caller<EchoRequest, EchoResponse>, SetupTeardown { + + static EchoGRPCCallerWithSetupTeardown of(URI uri) { + return new EchoGRPCCallerWithSetupTeardown(uri); + } + + private final URI uri; + private transient @MonotonicNonNull ManagedChannel cachedManagedChannel; + private transient @MonotonicNonNull EchoServiceBlockingStub cachedBlockingStub; + private static final ChannelCredentials DEFAULT_CREDENTIALS = InsecureChannelCredentials.create(); + + private EchoGRPCCallerWithSetupTeardown(URI uri) { + this.uri = uri; + } + + /** + * Overrides {@link Caller#call} invoking the {@link EchoServiceGrpc} with a {@link EchoRequest}, + * returning either a successful {@link EchoResponse} or throwing either a {@link + * UserCodeExecutionException}, a {@link UserCodeTimeoutException}, or a {@link + * UserCodeQuotaException}. + */ + @Override + public EchoResponse call(EchoRequest request) throws UserCodeExecutionException { + try { + return cachedBlockingStub.echo(request); + } catch (StatusRuntimeException e) { + switch (e.getStatus().getCode()) { + case RESOURCE_EXHAUSTED: + throw new UserCodeQuotaException(e); + case DEADLINE_EXCEEDED: + throw new UserCodeTimeoutException(e); + default: + throw new UserCodeExecutionException(e); + } + } + } + + /** + * Overrides {@link SetupTeardown#setup} to initialize the {@link ManagedChannel} and {@link + * EchoServiceBlockingStub}. + */ + @Override + public void setup() throws UserCodeExecutionException { + cachedManagedChannel = + NettyChannelBuilder.forTarget(uri.toString(), DEFAULT_CREDENTIALS).build(); + cachedBlockingStub = EchoServiceGrpc.newBlockingStub(cachedManagedChannel); + } + + /** Overrides {@link SetupTeardown#teardown} to shut down the {@link ManagedChannel}. */ + @Override + public void teardown() throws UserCodeExecutionException { + if (cachedManagedChannel != null && cachedManagedChannel.isShutdown()) { + cachedManagedChannel.shutdown(); + try { + boolean ignored = cachedManagedChannel.awaitTermination(1L, TimeUnit.SECONDS); + } catch (InterruptedException ignored) { + } + } + } +} diff --git a/sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponse/EchoGRPCCallerWithSetupTeardownTestIT.java b/sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponse/EchoGRPCCallerWithSetupTeardownTestIT.java new file mode 100644 index 000000000000..14b6e9e6433d --- /dev/null +++ b/sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponse/EchoGRPCCallerWithSetupTeardownTestIT.java @@ -0,0 +1,126 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.io.requestresponse; + +import static org.apache.beam.sdk.io.common.IOITHelper.readIOTestPipelineOptions; +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThrows; + +import com.google.protobuf.ByteString; +import java.net.URI; +import org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoRequest; +import org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoResponse; +import org.apache.beam.testinfra.mockapis.echo.v1.EchoServiceGrpc; +import org.checkerframework.checker.nullness.qual.MonotonicNonNull; +import org.checkerframework.checker.nullness.qual.NonNull; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** + * Tests for {@link EchoGRPCCallerWithSetupTeardown} on a deployed {@link EchoServiceGrpc} instance. + * See {@link EchoITOptions} for details on the required parameters and how to provide these for + * running integration tests. + */ +@RunWith(JUnit4.class) +public class EchoGRPCCallerWithSetupTeardownTestIT { + + private static @MonotonicNonNull EchoITOptions options; + private static @MonotonicNonNull EchoGRPCCallerWithSetupTeardown client; + private static final ByteString PAYLOAD = ByteString.copyFromUtf8("payload"); + + @BeforeClass + public static void setUp() throws UserCodeExecutionException { + options = readIOTestPipelineOptions(EchoITOptions.class); + if (options.getgRPCEndpointAddress().isEmpty()) { + throw new RuntimeException( + "--gRPCEndpointAddress is missing. See " + EchoITOptions.class + "for details."); + } + client = EchoGRPCCallerWithSetupTeardown.of(URI.create(options.getgRPCEndpointAddress())); + checkStateNotNull(client).setup(); + + EchoRequest request = createShouldExceedQuotaRequest(); + + // The challenge with building and deploying a real quota aware endpoint, the integration with + // which these tests validate, is that we need a value of at least 1. The allocated quota where + // we expect to exceed will be shared among many tests and across languages. Code below in this + // setup ensures that the API is in the state where we can expect a quota exceeded error. There + // are tests in this file that detect errors in expected responses. We only throw exceptions + // that are not UserCodeQuotaException. + try { + EchoResponse ignored = client.call(request); + client.call(request); + client.call(request); + } catch (UserCodeExecutionException e) { + if (!(e instanceof UserCodeQuotaException)) { + throw e; + } + } + } + + @AfterClass + public static void tearDown() throws UserCodeExecutionException { + checkStateNotNull(client).teardown(); + } + + @Test + public void givenValidRequest_receivesResponse() throws UserCodeExecutionException { + EchoRequest request = createShouldNeverExceedQuotaRequest(); + EchoResponse response = client.call(request); + assertEquals(response.getId(), request.getId()); + assertEquals(response.getPayload(), request.getPayload()); + } + + @Test + public void givenExceededQuota_shouldThrow() { + assertThrows(UserCodeQuotaException.class, () -> client.call(createShouldExceedQuotaRequest())); + } + + @Test + public void givenNotFound_shouldThrow() { + UserCodeExecutionException error = + assertThrows( + UserCodeExecutionException.class, + () -> + client.call( + EchoRequest.newBuilder() + .setId("i-dont-exist-quota-id") + .setPayload(PAYLOAD) + .build())); + assertEquals( + "io.grpc.StatusRuntimeException: NOT_FOUND: error: source not found: i-dont-exist-quota-id, err resource does not exist", + error.getMessage()); + } + + private static @NonNull EchoRequest createShouldNeverExceedQuotaRequest() { + return EchoRequest.newBuilder() + .setPayload(PAYLOAD) + .setId(checkStateNotNull(options).getNeverExceedQuotaId()) + .build(); + } + + private static @NonNull EchoRequest createShouldExceedQuotaRequest() { + return EchoRequest.newBuilder() + .setPayload(PAYLOAD) + .setId(checkStateNotNull(options).getShouldExceedQuotaId()) + .build(); + } +} diff --git a/sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponse/EchoHTTPCaller.java b/sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponse/EchoHTTPCaller.java new file mode 100644 index 000000000000..91842f2efb27 --- /dev/null +++ b/sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponse/EchoHTTPCaller.java @@ -0,0 +1,92 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.io.requestresponse; + +import com.google.api.client.http.ByteArrayContent; +import com.google.api.client.http.GenericUrl; +import com.google.api.client.http.HttpMediaType; +import com.google.api.client.http.HttpRequest; +import com.google.api.client.http.HttpRequestFactory; +import com.google.api.client.http.HttpResponse; +import com.google.api.client.http.HttpResponseException; +import com.google.api.client.http.javanet.NetHttpTransport; +import com.google.protobuf.util.JsonFormat; +import java.io.IOException; +import java.net.URI; +import org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoRequest; +import org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoResponse; +import org.apache.beam.testinfra.mockapis.echo.v1.EchoServiceGrpc; + +/** + * Implements {@link Caller} to call the {@link EchoServiceGrpc}'s HTTP handler. The purpose of + * {@link EchoHTTPCaller} is to suppport integration tests. + */ +class EchoHTTPCaller implements Caller<EchoRequest, EchoResponse> { + + static EchoHTTPCaller of(URI uri) { + return new EchoHTTPCaller(uri); + } + + private static final String PATH = "/v1/echo"; + private static final HttpRequestFactory REQUEST_FACTORY = + new NetHttpTransport().createRequestFactory(); + private static final HttpMediaType CONTENT_TYPE = new HttpMediaType("application/json"); + private static final int STATUS_CODE_TOO_MANY_REQUESTS = 429; + + private final URI uri; + + private EchoHTTPCaller(URI uri) { + this.uri = uri; + } + + /** + * Overrides {@link Caller#call} invoking the {@link EchoServiceGrpc}'s HTTP handler with a {@link + * EchoRequest}, returning either a successful {@link EchoResponse} or throwing either a {@link + * UserCodeExecutionException}, a {@link UserCodeTimeoutException}, or a {@link + * UserCodeQuotaException}. + */ + @Override + public EchoResponse call(EchoRequest request) throws UserCodeExecutionException { + try { + String json = JsonFormat.printer().omittingInsignificantWhitespace().print(request); + ByteArrayContent body = ByteArrayContent.fromString(CONTENT_TYPE.getType(), json); + HttpRequest httpRequest = REQUEST_FACTORY.buildPostRequest(getUrl(), body); + HttpResponse httpResponse = httpRequest.execute(); + String responseJson = httpResponse.parseAsString(); + EchoResponse.Builder builder = EchoResponse.newBuilder(); + JsonFormat.parser().merge(responseJson, builder); + return builder.build(); + } catch (IOException e) { + if (e instanceof HttpResponseException) { + HttpResponseException ex = (HttpResponseException) e; + if (ex.getStatusCode() == STATUS_CODE_TOO_MANY_REQUESTS) { + throw new UserCodeQuotaException(e); + } + } + throw new UserCodeExecutionException(e); + } + } + + private GenericUrl getUrl() { + String rawUrl = uri.toString(); + if (uri.getPath().isEmpty()) { + rawUrl += PATH; + } + return new GenericUrl(rawUrl); + } +} diff --git a/sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponse/EchoHTTPCallerTestIT.java b/sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponse/EchoHTTPCallerTestIT.java new file mode 100644 index 000000000000..fa0cb9378110 --- /dev/null +++ b/sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponse/EchoHTTPCallerTestIT.java @@ -0,0 +1,120 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.io.requestresponse; + +import static org.apache.beam.sdk.io.common.IOITHelper.readIOTestPipelineOptions; +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThrows; +import static org.junit.Assert.assertTrue; + +import com.google.protobuf.ByteString; +import java.net.URI; +import org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoRequest; +import org.apache.beam.testinfra.mockapis.echo.v1.Echo.EchoResponse; +import org.apache.beam.testinfra.mockapis.echo.v1.EchoServiceGrpc; +import org.checkerframework.checker.nullness.qual.MonotonicNonNull; +import org.checkerframework.checker.nullness.qual.NonNull; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** + * Tests for {@link EchoHTTPCallerTestIT} on a deployed {@link EchoServiceGrpc} instance's HTTP + * handler. See {@link EchoITOptions} for details on the required parameters and how to provide + * these for running integration tests. + */ +@RunWith(JUnit4.class) +public class EchoHTTPCallerTestIT { + + private static @MonotonicNonNull EchoITOptions options; + private static @MonotonicNonNull EchoHTTPCaller client; + private static final ByteString PAYLOAD = ByteString.copyFromUtf8("payload"); + + @BeforeClass + public static void setUp() throws UserCodeExecutionException { + options = readIOTestPipelineOptions(EchoITOptions.class); + if (options.getHttpEndpointAddress().isEmpty()) { + throw new RuntimeException( + "--httpEndpointAddress is missing. See " + EchoITOptions.class + "for details."); + } + client = EchoHTTPCaller.of(URI.create(options.getHttpEndpointAddress())); + + EchoRequest request = createShouldExceedQuotaRequest(); + + // The challenge with building and deploying a real quota aware endpoint, the integration with + // which these tests validate, is that we need a value of at least 1. The allocated quota where + // we expect to exceed will be shared among many tests and across languages. Code below in this + // setup ensures that the API is in the state where we can expect a quota exceeded error. There + // are tests in this file that detect errors in expected responses. We only throw exceptions + // that are not UserCodeQuotaException. + try { + EchoResponse ignored = client.call(request); + client.call(request); + client.call(request); + client.call(request); + } catch (UserCodeExecutionException e) { + if (!(e instanceof UserCodeQuotaException)) { + throw e; + } + } + } + + @Test + public void givenValidRequest_receivesResponse() throws UserCodeExecutionException { + EchoRequest request = createShouldNeverExceedQuotaRequest(); + EchoResponse response = client.call(request); + assertEquals(response.getId(), request.getId()); + assertEquals(response.getPayload(), request.getPayload()); + } + + @Test + public void givenExceededQuota_shouldThrow() { + assertThrows(UserCodeQuotaException.class, () -> client.call(createShouldExceedQuotaRequest())); + } + + @Test + public void givenNotFound_shouldThrow() { + UserCodeExecutionException error = + assertThrows( + UserCodeExecutionException.class, + () -> + client.call( + EchoRequest.newBuilder() + .setId("i-dont-exist-quota-id") + .setPayload(PAYLOAD) + .build())); + + assertTrue(error.getMessage().contains("404 Not Found")); + } + + private static @NonNull EchoRequest createShouldNeverExceedQuotaRequest() { + return EchoRequest.newBuilder() + .setPayload(PAYLOAD) + .setId(checkStateNotNull(options).getNeverExceedQuotaId()) + .build(); + } + + private static @NonNull EchoRequest createShouldExceedQuotaRequest() { + return EchoRequest.newBuilder() + .setPayload(PAYLOAD) + .setId(checkStateNotNull(options).getShouldExceedQuotaId()) + .build(); + } +} diff --git a/sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponse/EchoITOptions.java b/sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponse/EchoITOptions.java new file mode 100644 index 000000000000..a32f7a78e826 --- /dev/null +++ b/sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponse/EchoITOptions.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.io.requestresponse; + +import org.apache.beam.sdk.options.Default; +import org.apache.beam.sdk.options.Description; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.testinfra.mockapis.echo.v1.EchoServiceGrpc; + +/** + * Shared options for running integration tests on a deployed {@link EchoServiceGrpc}. See <a + * href="https://github.com/apache/beam/tree/master/.test-infra/mock-apis#integration">https://github.com/apache/beam/tree/master/.test-infra/mock-apis#integration</a> + * for details on how to acquire values required by {@link EchoITOptions}. + * + * <p>To provide these values to your integration tests: + * + * <pre> + * ./gradlew :sdks:java:io:rrio:integrationTest -DintegrationTestPipelineOptions='[ + * "--grpcEndpointAddress=", + * "--httpEndpointAddress=" + * ]' + * </pre> + */ +public interface EchoITOptions extends PipelineOptions { + @Description("The gRPC address of the Echo API endpoint, typically of the form <host>:<port>.") + String getgRPCEndpointAddress(); + + void setgRPCEndpointAddress(String value); + + @Description("The HTTP address of the Echo API endpoint; must being with http(s)://") + String getHttpEndpointAddress(); + + void setHttpEndpointAddress(String value); + + @Description("The ID for an allocated quota that should never exceed.") + @Default.String("echo-should-never-exceed-quota") + String getNeverExceedQuotaId(); + + void setNeverExceedQuotaId(String value); + + @Description("The ID for an allocated quota that should exceed.") + @Default.String("echo-should-exceed-quota") + String getShouldExceedQuotaId(); + + void setShouldExceedQuotaId(String value); +} From ca6f559bb93422b5ebd9303e3f206de63f38d385 Mon Sep 17 00:00:00 2001 From: Yi Hu <yathu@google.com> Date: Tue, 7 Nov 2023 16:45:18 -0500 Subject: [PATCH 401/435] Remove push trigger for non-precommit workflows (#29339) * Remove push trigger for non-precommit workflows * fix missing schedule condition --- .../workflows/beam_CloudML_Benchmarks_Dataflow.yml | 2 -- .../beam_Inference_Python_Benchmarks_Dataflow.yml | 2 -- .../workflows/beam_PerformanceTests_SingleStoreIO.yml | 2 -- .../workflows/beam_PostCommit_BeamMetrics_Publish.yml | 3 --- .github/workflows/beam_PostCommit_Go_Dataflow_ARM.yml | 4 ---- .../beam_PostCommit_Java_Examples_Dataflow.yml | 2 -- .../beam_PostCommit_Java_Examples_Dataflow_ARM.yml | 11 ----------- .../beam_PostCommit_Java_IO_Performance_Tests.yml | 4 ---- .../beam_PreCommit_Java_Examples_Dataflow_Java21.yml | 2 +- .../beam_Python_ValidatesContainer_Dataflow_ARM.yml | 4 ---- 10 files changed, 1 insertion(+), 35 deletions(-) diff --git a/.github/workflows/beam_CloudML_Benchmarks_Dataflow.yml b/.github/workflows/beam_CloudML_Benchmarks_Dataflow.yml index 4902062d3928..ccc9822950ec 100644 --- a/.github/workflows/beam_CloudML_Benchmarks_Dataflow.yml +++ b/.github/workflows/beam_CloudML_Benchmarks_Dataflow.yml @@ -16,8 +16,6 @@ name: CloudML Benchmarks Dataflow on: - issue_comment: - types: [created] schedule: - cron: '10 21 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_Inference_Python_Benchmarks_Dataflow.yml b/.github/workflows/beam_Inference_Python_Benchmarks_Dataflow.yml index 3bf1c101646e..2ca9953ce4b8 100644 --- a/.github/workflows/beam_Inference_Python_Benchmarks_Dataflow.yml +++ b/.github/workflows/beam_Inference_Python_Benchmarks_Dataflow.yml @@ -16,8 +16,6 @@ name: Inference Python Benchmarks Dataflow on: - issue_comment: - types: [created] schedule: - cron: '50 3 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_SingleStoreIO.yml b/.github/workflows/beam_PerformanceTests_SingleStoreIO.yml index cc1a0b24a60a..0efb001a4eb6 100644 --- a/.github/workflows/beam_PerformanceTests_SingleStoreIO.yml +++ b/.github/workflows/beam_PerformanceTests_SingleStoreIO.yml @@ -18,8 +18,6 @@ name: PerformanceTests SingleStoreIO on: - issue_comment: - types: [created] schedule: - cron: '0 */12 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_BeamMetrics_Publish.yml b/.github/workflows/beam_PostCommit_BeamMetrics_Publish.yml index 7695b7f0ca30..b655eab8088c 100644 --- a/.github/workflows/beam_PostCommit_BeamMetrics_Publish.yml +++ b/.github/workflows/beam_PostCommit_BeamMetrics_Publish.yml @@ -16,9 +16,6 @@ name: PostCommit BeamMetrics Publish on: - push: - tags: ['v*'] - branches: ['master', 'release-*'] schedule: - cron: '24 2 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Go_Dataflow_ARM.yml b/.github/workflows/beam_PostCommit_Go_Dataflow_ARM.yml index d8fd9753f287..532c48e7b072 100644 --- a/.github/workflows/beam_PostCommit_Go_Dataflow_ARM.yml +++ b/.github/workflows/beam_PostCommit_Go_Dataflow_ARM.yml @@ -16,10 +16,6 @@ name: PostCommit Go Dataflow ARM on: - push: - tags: ['v*'] - branches: ['master', 'release-*'] - paths: ['model/**', 'sdks/go.**', 'release/**', '.github/workflows/beam_PostCommit_Go_Dataflow_ARM.yml'] issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow.yml index 54874493e283..488031a9244a 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow.yml @@ -18,8 +18,6 @@ name: PostCommit Java Examples Dataflow on: - issue_comment: - types: [created] schedule: - cron: '45 3/6 * * *' pull_request_target: diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml index 478707b850d4..16e7275c724a 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml @@ -18,17 +18,6 @@ name: PostCommit Java Examples Dataflow ARM on: - push: - tags: ['v*'] - branches: ['master', 'release-*'] - paths: - - 'model/**' - - 'sdks/java/**' - - 'runners/google-cloud-dataflow-java/**' - - 'examples/java/**' - - 'examples/kotlin/**' - - 'release/**' - - '.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PostCommit_Java_IO_Performance_Tests.yml b/.github/workflows/beam_PostCommit_Java_IO_Performance_Tests.yml index 62fb402df7d0..4bbc23b13de4 100644 --- a/.github/workflows/beam_PostCommit_Java_IO_Performance_Tests.yml +++ b/.github/workflows/beam_PostCommit_Java_IO_Performance_Tests.yml @@ -16,10 +16,6 @@ name: PostCommit Java IO Performance Tests on: - push: - tags: ['v*'] - branches: ['master', 'release-*'] - paths: ['it/google-cloud-platform/**','.github/workflows/beam_PostCommit_Java_IO_Performance_Tests.yml'] issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java21.yml b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java21.yml index 030c3086cc1c..f482e22ec4a2 100644 --- a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java21.yml +++ b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java21.yml @@ -80,7 +80,7 @@ jobs: if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || - github.event_name == 'schedule' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Java_Examples_Dataflow_Java21 PreCommit' runs-on: [self-hosted, ubuntu-20.04, main] diff --git a/.github/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml b/.github/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml index fe00e921d3b0..6681669f5958 100644 --- a/.github/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml +++ b/.github/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml @@ -16,10 +16,6 @@ name: Python ValidatesContainer Dataflow ARM on: - push: - tags: ['v*'] - branches: ['master', 'release-*'] - paths: ["sdks/python/**",".github/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml"] schedule: - cron: '0 */6 * * *' workflow_dispatch: From 228157f1cd2954d32b0274400fcb391d06a3e207 Mon Sep 17 00:00:00 2001 From: Vlado Djerek <rage.bubblegum@gmail.com> Date: Wed, 8 Nov 2023 00:07:33 +0100 Subject: [PATCH 402/435] revert network name (#29337) Co-authored-by: Vlado Djerek <djerek.vlado6@gmail.com> --- .../apache_beam/io/azure/integration_test/docker-compose.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/io/azure/integration_test/docker-compose.yml b/sdks/python/apache_beam/io/azure/integration_test/docker-compose.yml index 9a7d1cdf6626..8e03fe27e6bd 100644 --- a/sdks/python/apache_beam/io/azure/integration_test/docker-compose.yml +++ b/sdks/python/apache_beam/io/azure/integration_test/docker-compose.yml @@ -48,7 +48,7 @@ services: - "azurite" networks: - test_net: + azure_test_net: driver: bridge driver_opts: com.docker.network.driver.mtu: 1460 From 38f3b21c5c09db56c6ce07cc592b486a50958505 Mon Sep 17 00:00:00 2001 From: Mike Williamson <this.is.mvw@gmail.com> Date: Wed, 8 Nov 2023 02:13:42 +0100 Subject: [PATCH 403/435] Fix small typo on get_table_schema_from_string (#29340) --- sdks/python/apache_beam/io/gcp/bigquery_tools.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/io/gcp/bigquery_tools.py b/sdks/python/apache_beam/io/gcp/bigquery_tools.py index 2f9420795288..57acc320e838 100644 --- a/sdks/python/apache_beam/io/gcp/bigquery_tools.py +++ b/sdks/python/apache_beam/io/gcp/bigquery_tools.py @@ -1607,7 +1607,7 @@ def get_table_schema_from_string(schema): bigquery_v2_messages.TableSchema` instance. Args: - schema (str): The sting schema to be used if the BigQuery table to write + schema (str): The string schema to be used if the BigQuery table to write has to be created. Returns: From 70093ced2cf74dce6e639634ce1670611f447a6a Mon Sep 17 00:00:00 2001 From: Vlado Djerek <rage.bubblegum@gmail.com> Date: Wed, 8 Nov 2023 15:11:48 +0100 Subject: [PATCH 404/435] set dockerMTU to 1460 to align with GKE (#29346) Co-authored-by: Vlado Djerek <djerek.vlado6@gmail.com> --- .../gh-actions-self-hosted-runners/arc/config/arc_deployment.tpl | 1 + 1 file changed, 1 insertion(+) diff --git a/.github/gh-actions-self-hosted-runners/arc/config/arc_deployment.tpl b/.github/gh-actions-self-hosted-runners/arc/config/arc_deployment.tpl index 71f8da3d9df1..41c6700b18b9 100644 --- a/.github/gh-actions-self-hosted-runners/arc/config/arc_deployment.tpl +++ b/.github/gh-actions-self-hosted-runners/arc/config/arc_deployment.tpl @@ -26,6 +26,7 @@ spec: annotations: cluster-autoscaler.kubernetes.io/safe-to-evict: "false" spec: + dockerMTU: 1460 %{~ if selector == true ~} nodeSelector: runner-pool: ${name} From bf47289b1cd2bb0e23fb4017c329f83f410e7845 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Rados=C5=82aw=20Stankiewicz?= <radoslaws@google.com> Date: Wed, 8 Nov 2023 15:59:17 +0100 Subject: [PATCH 405/435] Add header removal for TextIO (#29202) * add header removal * add multiple header lines removal --- CHANGES.md | 2 + .../java/org/apache/beam/sdk/io/TextIO.java | 41 ++++++++-- .../beam/sdk/io/TextRowCountEstimator.java | 10 ++- .../org/apache/beam/sdk/io/TextSource.java | 64 +++++++++++++--- .../apache/beam/sdk/io/TextIOReadTest.java | 75 ++++++++++++++++--- 5 files changed, 164 insertions(+), 28 deletions(-) diff --git a/CHANGES.md b/CHANGES.md index 17ac8a1d7010..523ef3455aec 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -63,6 +63,7 @@ ## I/Os * Support for X source added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). +* TextIO now supports skipping multiple header lines (Java) ([#17990](https://github.com/apache/beam/issues/17990)). ## New Features / Improvements @@ -107,6 +108,7 @@ should handle this. ([#25252](https://github.com/apache/beam/issues/25252)). * state amd side input cache has been enabled to a default of 100 MB. Use `--max_cache_memory_usage_mb=X` to provide cache size for the user state API and side inputs. (Python) ([#28770](https://github.com/apache/beam/issues/28770)). * Beam YAML stable release. Beam pipelines can now be written using YAML and leverage the Beam YAML framework which includes a preliminary set of IO's and turnkey transforms. More information can be found in the YAML root folder and in the [README](https://github.com/apache/beam/blob/master/sdks/python/apache_beam/yaml/README.md). + ## Breaking Changes * `org.apache.beam.sdk.io.CountingSource.CounterMark` uses custom `CounterMarkCoder` as a default coder since all Avro-dependent diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java index 33beff23b311..2c7a4fc5d4f5 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java @@ -191,6 +191,7 @@ public static Read read() { return new AutoValue_TextIO_Read.Builder() .setCompression(Compression.AUTO) .setHintMatchesManyFiles(false) + .setSkipHeaderLines(0) .setMatchConfiguration(MatchConfiguration.create(EmptyMatchTreatment.DISALLOW)) .build(); } @@ -214,6 +215,7 @@ public static Read read() { public static ReadAll readAll() { return new AutoValue_TextIO_ReadAll.Builder() .setCompression(Compression.AUTO) + .setSkipHeaderLines(0) .setMatchConfiguration(MatchConfiguration.create(EmptyMatchTreatment.ALLOW_IF_WILDCARD)) .build(); } @@ -228,6 +230,7 @@ public static ReadFiles readFiles() { // but is not so large as to exhaust a typical runner's maximum amount of output per // ProcessElement call. .setDesiredBundleSizeBytes(DEFAULT_BUNDLE_SIZE_BYTES) + .setSkipHeaderLines(0) .build(); } @@ -286,6 +289,8 @@ public abstract static class Read extends PTransform<PBegin, PCollection<String> @SuppressWarnings("mutable") // this returns an array that can be mutated by the caller abstract byte @Nullable [] getDelimiter(); + abstract int getSkipHeaderLines(); + abstract Builder toBuilder(); @AutoValue.Builder @@ -300,6 +305,8 @@ abstract static class Builder { abstract Builder setDelimiter(byte @Nullable [] delimiter); + abstract Builder setSkipHeaderLines(int skipHeaderLines); + abstract Read build(); } @@ -396,6 +403,10 @@ public Read withDelimiter(byte[] delimiter) { return toBuilder().setDelimiter(delimiter).build(); } + public Read withSkipHeaderLines(int skipHeaderLines) { + return toBuilder().setSkipHeaderLines(skipHeaderLines).build(); + } + static boolean isSelfOverlapping(byte[] s) { // s self-overlaps if v exists such as s = vu = wv with u and w non empty for (int i = 1; i < s.length - 1; ++i) { @@ -422,7 +433,9 @@ public PCollection<String> expand(PBegin input) { FileIO.readMatches() .withCompression(getCompression()) .withDirectoryTreatment(DirectoryTreatment.PROHIBIT)) - .apply("Via ReadFiles", readFiles().withDelimiter(getDelimiter())); + .apply( + "Via ReadFiles", + readFiles().withDelimiter(getDelimiter()).withSkipHeaderLines(getSkipHeaderLines())); } // Helper to create a source specific to the requested compression type. @@ -431,7 +444,8 @@ protected FileBasedSource<String> getSource() { new TextSource( getFilepattern(), getMatchConfiguration().getEmptyMatchTreatment(), - getDelimiter())) + getDelimiter(), + getSkipHeaderLines())) .withCompression(getCompression()); } @@ -468,6 +482,8 @@ public abstract static class ReadAll @SuppressWarnings("mutable") // this returns an array that can be mutated by the caller abstract byte @Nullable [] getDelimiter(); + abstract int getSkipHeaderLines(); + abstract Builder toBuilder(); @AutoValue.Builder @@ -478,6 +494,8 @@ abstract static class Builder { abstract Builder setDelimiter(byte @Nullable [] delimiter); + abstract Builder setSkipHeaderLines(int skipHeaderLines); + abstract ReadAll build(); } @@ -560,6 +578,8 @@ public abstract static class ReadFiles @SuppressWarnings("mutable") // this returns an array that can be mutated by the caller abstract byte @Nullable [] getDelimiter(); + abstract int getSkipHeaderLines(); + abstract Builder toBuilder(); @AutoValue.Builder @@ -568,6 +588,8 @@ abstract static class Builder { abstract Builder setDelimiter(byte @Nullable [] delimiter); + abstract Builder setSkipHeaderLines(int skipHeaderLines); + abstract ReadFiles build(); } @@ -581,13 +603,17 @@ public ReadFiles withDelimiter(byte[] delimiter) { return toBuilder().setDelimiter(delimiter).build(); } + public ReadFiles withSkipHeaderLines(int skipHeaderLines) { + return toBuilder().setSkipHeaderLines(skipHeaderLines).build(); + } + @Override public PCollection<String> expand(PCollection<FileIO.ReadableFile> input) { return input.apply( "Read all via FileBasedSource", new ReadAllViaFileBasedSource<>( getDesiredBundleSizeBytes(), - new CreateTextSourceFn(getDelimiter()), + new CreateTextSourceFn(getDelimiter(), getSkipHeaderLines()), StringUtf8Coder.of())); } @@ -602,15 +628,20 @@ public void populateDisplayData(DisplayData.Builder builder) { private static class CreateTextSourceFn implements SerializableFunction<String, FileBasedSource<String>> { private byte[] delimiter; + private int skipHeaderLines; - private CreateTextSourceFn(byte[] delimiter) { + private CreateTextSourceFn(byte[] delimiter, int skipHeaderLines) { this.delimiter = delimiter; + this.skipHeaderLines = skipHeaderLines; } @Override public FileBasedSource<String> apply(String input) { return new TextSource( - StaticValueProvider.of(input), EmptyMatchTreatment.DISALLOW, delimiter); + StaticValueProvider.of(input), + EmptyMatchTreatment.DISALLOW, + delimiter, + skipHeaderLines); } } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextRowCountEstimator.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextRowCountEstimator.java index 32b7fb12f414..8542ce011098 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextRowCountEstimator.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextRowCountEstimator.java @@ -46,6 +46,8 @@ public abstract class TextRowCountEstimator { @SuppressWarnings("mutable") public abstract byte @Nullable [] getDelimiters(); + public abstract int getSkipHeaderLines(); + public abstract String getFilePattern(); public abstract Compression getCompression(); @@ -62,7 +64,8 @@ public static TextRowCountEstimator.Builder builder() { .setNumSampledBytesPerFile(DEFAULT_NUM_BYTES_PER_FILE) .setCompression(DEFAULT_COMPRESSION) .setDirectoryTreatment(DEFAULT_DIRECTORY_TREATMENT) - .setEmptyMatchTreatment(DEFAULT_EMPTY_MATCH_TREATMENT); + .setEmptyMatchTreatment(DEFAULT_EMPTY_MATCH_TREATMENT) + .setSkipHeaderLines(0); } /** @@ -114,7 +117,8 @@ public Double estimateRowCount(PipelineOptions pipelineOptions) new TextSource( ValueProvider.StaticValueProvider.of(file.getMetadata().resourceId().toString()), getEmptyMatchTreatment(), - getDelimiters()); + getDelimiters(), + getSkipHeaderLines()); FileBasedSource<String> source = CompressedSource.from(textSource).withCompression(file.getCompression()); try (BoundedSource.BoundedReader<String> reader = @@ -160,6 +164,8 @@ public abstract Builder setDirectoryTreatment( public abstract Builder setDelimiters(byte @Nullable [] delimiters); + public abstract Builder setSkipHeaderLines(int skipHeaderLines); + public abstract Builder setFilePattern(String filePattern); public abstract Builder setEmptyMatchTreatment(EmptyMatchTreatment emptyMatchTreatment); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextSource.java index bef30dffa8ac..3d62c677950a 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextSource.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextSource.java @@ -56,26 +56,43 @@ public class TextSource extends FileBasedSource<String> { byte[] delimiter; + int skipHeaderLines; + public TextSource( - ValueProvider<String> fileSpec, EmptyMatchTreatment emptyMatchTreatment, byte[] delimiter) { + ValueProvider<String> fileSpec, + EmptyMatchTreatment emptyMatchTreatment, + byte[] delimiter, + int skipHeaderLines) { super(fileSpec, emptyMatchTreatment, 1L); this.delimiter = delimiter; + this.skipHeaderLines = skipHeaderLines; } - public TextSource(MatchResult.Metadata metadata, long start, long end, byte[] delimiter) { + public TextSource( + ValueProvider<String> fileSpec, EmptyMatchTreatment emptyMatchTreatment, byte[] delimiter) { + this(fileSpec, emptyMatchTreatment, delimiter, 0); + } + + public TextSource( + MatchResult.Metadata metadata, long start, long end, byte[] delimiter, int skipHeaderLines) { super(metadata, 1L, start, end); this.delimiter = delimiter; + this.skipHeaderLines = skipHeaderLines; + } + + public TextSource(MatchResult.Metadata metadata, long start, long end, byte[] delimiter) { + this(metadata, start, end, delimiter, 0); } @Override protected FileBasedSource<String> createForSubrangeOfFile( MatchResult.Metadata metadata, long start, long end) { - return new TextSource(metadata, start, end, delimiter); + return new TextSource(metadata, start, end, delimiter, skipHeaderLines); } @Override protected FileBasedReader<String> createSingleFileReader(PipelineOptions options) { - return new TextBasedReader(this, delimiter); + return new TextBasedReader(this, delimiter, skipHeaderLines); } @Override @@ -98,6 +115,7 @@ static class TextBasedReader extends FileBasedReader<String> { private static final byte LF = '\n'; private final byte @Nullable [] delimiter; + private final int skipHeaderLines; private final ByteArrayOutputStream str; private final byte[] buffer; private final ByteBuffer byteBuffer; @@ -112,11 +130,16 @@ static class TextBasedReader extends FileBasedReader<String> { private boolean skipLineFeedAtStart; // skip an LF if at the start of the next buffer private TextBasedReader(TextSource source, byte[] delimiter) { + this(source, delimiter, 0); + } + + private TextBasedReader(TextSource source, byte[] delimiter, int skipHeaderLines) { super(source); this.buffer = new byte[READ_BUFFER_SIZE]; this.str = new ByteArrayOutputStream(); this.byteBuffer = ByteBuffer.wrap(buffer); this.delimiter = delimiter; + this.skipHeaderLines = skipHeaderLines; } @Override @@ -171,21 +194,42 @@ protected void startReading(ReadableByteChannel channel) throws IOException { } else { startOfNextRecord = bufferPosn = (int) requiredPosition; } + skipHeader(skipHeaderLines, true); } else { - ((SeekableByteChannel) channel).position(requiredPosition); - startOfNextRecord = requiredPosition; + skipHeader(skipHeaderLines, false); + if (requiredPosition > startOfNextRecord) { + ((SeekableByteChannel) channel).position(requiredPosition); + startOfNextRecord = requiredPosition; + bufferLength = bufferPosn = 0; + } + // Read and discard the next record ensuring that startOfNextRecord and bufferPosn point + // to the beginning of the next record. + readNextRecord(); + currentValue = null; } - // Read and discard the next record ensuring that startOfNextRecord and bufferPosn point - // to the beginning of the next record. - readNextRecord(); - currentValue = null; } else { // Check to see if we start with the UTF_BOM bytes skipping them if present. if (fileStartsWithBom()) { startOfNextRecord = bufferPosn = UTF8_BOM.size(); } + skipHeader(skipHeaderLines, false); + } + } + + private void skipHeader(int headerLines, boolean skipFirstLine) throws IOException { + if (headerLines == 1) { + readNextRecord(); + } else if (headerLines > 1) { + // this will be expensive + ((SeekableByteChannel) inChannel).position(0); + for (int line = 0; line < headerLines; ++line) { + readNextRecord(); + } + } else if (headerLines == 0 && skipFirstLine) { + readNextRecord(); } + currentValue = null; } private boolean fileStartsWithBom() throws IOException { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOReadTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOReadTest.java index 379345b1001e..84c05ee6c906 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOReadTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOReadTest.java @@ -245,15 +245,24 @@ private static File createZipFile( } private static TextSource prepareSource( - TemporaryFolder temporaryFolder, byte[] data, @Nullable byte[] delimiter) throws IOException { + TemporaryFolder temporaryFolder, byte[] data, @Nullable byte[] delimiter, int skipHeaderLines) + throws IOException { Path path = temporaryFolder.newFile().toPath(); Files.write(path, data); - return getTextSource(path.toString(), delimiter); + return getTextSource(path.toString(), delimiter, skipHeaderLines); } - public static TextSource getTextSource(String path, @Nullable byte[] delimiter) { + public static TextSource getTextSource( + String path, @Nullable byte[] delimiter, int skipHeaderLines) { return new TextSource( - ValueProvider.StaticValueProvider.of(path), EmptyMatchTreatment.DISALLOW, delimiter); + ValueProvider.StaticValueProvider.of(path), + EmptyMatchTreatment.DISALLOW, + delimiter, + skipHeaderLines); + } + + public static TextSource getTextSource(String path, @Nullable byte[] delimiter) { + return getTextSource(path, delimiter, 0); } private static String getFileSuffix(Compression compression) { @@ -384,7 +393,7 @@ public void testReadLinesWithDefaultDelimiterAndZeroAndOneLengthReturningChannel Files.write(path, line.getBytes(UTF_8)); Metadata metadata = FileSystems.matchSingleFileSpec(path.toString()); FileBasedSource source = - getTextSource(path.toString(), null) + getTextSource(path.toString(), null, 0) .createForSubrangeOfFile(metadata, 0, metadata.sizeBytes()); FileBasedReader<String> reader = source.createSingleFileReader(PipelineOptionsFactory.create()); @@ -433,7 +442,49 @@ public void testSplittingSource() throws Exception { } private TextSource prepareSource(byte[] data) throws IOException { - return TextIOReadTest.prepareSource(tempFolder, data, null); + return TextIOReadTest.prepareSource(tempFolder, data, null, 0); + } + + private void runTestReadWithData(byte[] data, List<String> expectedResults) throws Exception { + TextSource source = prepareSource(data); + List<String> actual = SourceTestUtils.readFromSource(source, PipelineOptionsFactory.create()); + assertThat( + actual, containsInAnyOrder(new ArrayList<>(expectedResults).toArray(new String[0]))); + } + } + + /** Tests for reading files with/without header. */ + @RunWith(Parameterized.class) + public static class SkippingHeaderTest { + private static final ImmutableList<String> EXPECTED = ImmutableList.of("asdf", "hjkl", "xyz"); + @Rule public TemporaryFolder tempFolder = new TemporaryFolder(); + + @Parameterized.Parameters(name = "{index}: {0}") + public static Iterable<Object[]> data() { + return ImmutableList.<Object[]>builder() + .add(new Object[] {"\n\n\n", ImmutableList.of("", ""), 1}) + .add(new Object[] {"\n", ImmutableList.of(), 1}) + .add(new Object[] {"header\nasdf\nhjkl\nxyz\n", EXPECTED, 1}) + .add(new Object[] {"header1\nheader2\nasdf\nhjkl\nxyz\n", EXPECTED, 2}) + .build(); + } + + @Parameterized.Parameter(0) + public String line; + + @Parameterized.Parameter(1) + public ImmutableList<String> expected; + + @Parameterized.Parameter(2) + public int skipHeaderLines; + + @Test + public void testReadLines() throws Exception { + runTestReadWithData(line.getBytes(UTF_8), expected); + } + + private TextSource prepareSource(byte[] data) throws IOException { + return TextIOReadTest.prepareSource(tempFolder, data, null, skipHeaderLines); } private void runTestReadWithData(byte[] data, List<String> expectedResults) throws Exception { @@ -477,7 +528,8 @@ public static Iterable<Object[]> data() { @Test public void testReadLinesWithCustomDelimiter() throws Exception { SourceTestUtils.assertSplitAtFractionExhaustive( - TextIOReadTest.prepareSource(tempFolder, testCase.getBytes(UTF_8), new byte[] {'|', '*'}), + TextIOReadTest.prepareSource( + tempFolder, testCase.getBytes(UTF_8), new byte[] {'|', '*'}, 0), PipelineOptionsFactory.create()); } @@ -489,7 +541,7 @@ public void testReadLinesWithCustomDelimiterAndZeroAndOneLengthReturningChannel( Files.write(path, testCase.getBytes(UTF_8)); Metadata metadata = FileSystems.matchSingleFileSpec(path.toString()); FileBasedSource source = - getTextSource(path.toString(), delimiter) + getTextSource(path.toString(), delimiter, 0) .createForSubrangeOfFile(metadata, 0, metadata.sizeBytes()); FileBasedReader<String> reader = source.createSingleFileReader(PipelineOptionsFactory.create()); @@ -743,7 +795,7 @@ public void testTextIOGetName() { } private TextSource prepareSource(byte[] data) throws IOException { - return TextIOReadTest.prepareSource(tempFolder, data, null); + return TextIOReadTest.prepareSource(tempFolder, data, null, 0); } @Test @@ -977,7 +1029,8 @@ public void testReadFilesWithFilename() throws IOException { new TextSource( ValueProvider.StaticValueProvider.of(input), EmptyMatchTreatment.DISALLOW, - new byte[] {'\n'}); + new byte[] {'\n'}, + 0); PCollection<KV<String, String>> lines = p.apply( @@ -1102,7 +1155,7 @@ public void processElement(ProcessContext c) { ValueProvider.StaticValueProvider.of(file.getMetadata().resourceId().getFilename()); // Create a TextSource, passing null as the delimiter to use the default // delimiters ('\n', '\r', or '\r\n'). - TextSource textSource = new TextSource(filenameProvider, null, null); + TextSource textSource = new TextSource(filenameProvider, null, null, 0); try { BoundedSource.BoundedReader<String> reader = textSource From 461c0b652faa61484c105dba6d3331447089bb7b Mon Sep 17 00:00:00 2001 From: Yi Hu <yathu@google.com> Date: Wed, 8 Nov 2023 13:18:06 -0500 Subject: [PATCH 406/435] Change BeamFnDataGrpcMultiplexer.onCompleted logging to INFO (#29259) --- .../org/apache/beam/sdk/fn/data/BeamFnDataGrpcMultiplexer.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataGrpcMultiplexer.java b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataGrpcMultiplexer.java index c05a86fdb1b7..3f83af4a8d6b 100644 --- a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataGrpcMultiplexer.java +++ b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataGrpcMultiplexer.java @@ -257,7 +257,7 @@ public void onError(Throwable t) { @Override public void onCompleted() { - LOG.warn( + LOG.info( "Hanged up for {}.", apiServiceDescriptor == null ? "unknown endpoint" : apiServiceDescriptor); } From 639111a2cc6f7ad9235589692c8e8bdb8199fd5e Mon Sep 17 00:00:00 2001 From: Yi Hu <yathu@google.com> Date: Wed, 8 Nov 2023 14:00:07 -0500 Subject: [PATCH 407/435] Install cbt cli for cleanup gcp resource (#29335) * Install cbt cli for cleanup gcp resource * Switch to setup-gcloud * Fix date command in linux * another fix of date command --- .github/workflows/beam_CleanUpGCPResources.yml | 9 ++++++--- .test-infra/tools/stale_cbt_instances_cleaner.sh | 12 ++++++++++-- 2 files changed, 16 insertions(+), 5 deletions(-) diff --git a/.github/workflows/beam_CleanUpGCPResources.yml b/.github/workflows/beam_CleanUpGCPResources.yml index 3097df8ecf7d..acebf427f6e9 100644 --- a/.github/workflows/beam_CleanUpGCPResources.yml +++ b/.github/workflows/beam_CleanUpGCPResources.yml @@ -19,7 +19,7 @@ name: Clean Up GCP Resources on: schedule: - - cron: '0 0 * * *' + - cron: '0 0,12 * * *' workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs @@ -73,10 +73,13 @@ jobs: uses: ./.github/actions/setup-environment-action - name: Authenticate on GCP id: auth - uses: google-github-actions/auth@v1 + uses: google-github-actions/setup-gcloud@v0 with: - credentials_json: ${{ secrets.GCP_SA_KEY }} + service_account_email: ${{ secrets.GCP_SA_EMAIL }} + service_account_key: ${{ secrets.GCP_SA_KEY }} project_id: ${{ secrets.GCP_PROJECT_ID }} + - name: Install gcloud bigtable cli + run: gcloud components install cbt - name: run cleanup GCP resources uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.test-infra/tools/stale_cbt_instances_cleaner.sh b/.test-infra/tools/stale_cbt_instances_cleaner.sh index 6996c9b05453..ed7f9df622e7 100755 --- a/.test-infra/tools/stale_cbt_instances_cleaner.sh +++ b/.test-infra/tools/stale_cbt_instances_cleaner.sh @@ -37,8 +37,16 @@ for instance in ${CBT_INSTANCES[@]}; do pattern=$template-"([0-9]{8})"- if [[ $instance =~ $pattern ]]; then CREATE_DATE=${BASH_REMATCH[1]} - # skip if not a valid date - CREATED=`date -ju -f "%Y%m%d-%H%M%S" ${CREATE_DATE}-000000 +%s` || continue + if [[ $OSTYPE == "linux-gnu"* ]]; then + # skip if not a valid date + CREATED=`date -d ${CREATE_DATE} +%s` || continue + elif [[ $OSTYPE == "darwin"* ]]; then + # date command usage depending on OS + CREATED=`date -ju -f "%Y%m%d-%H%M%S" ${CREATE_DATE}-000000 +%s` || continue + else + echo "Unsupported OS $OSTYPE" + exit 1 + fi if [[ $GRACE_PERIOD -gt $CREATED ]]; then if cbt -project=$PROJECT deleteinstance $instance; then echo "Deleted $instance (created $CREATE_DATE)" From f929d08417776f13836835972711a48f487edad6 Mon Sep 17 00:00:00 2001 From: Vlado Djerek <rage.bubblegum@gmail.com> Date: Wed, 8 Nov 2023 20:34:57 +0100 Subject: [PATCH 408/435] add missing gradle tasks to postcommit java singlestoreio test (#29348) Co-authored-by: Vlado Djerek <djerek.vlado6@gmail.com> --- .github/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml b/.github/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml index d718928e2482..680bcec727d3 100644 --- a/.github/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml +++ b/.github/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml @@ -104,5 +104,5 @@ jobs: - name: Run Java SingleStore IO IT uses: ./.github/actions/gradle-command-self-hosted-action with: - gradle-command: :sdks:java:io:singlestore:integrationTest --tests org.apache.beam.sdk.io.singlestore.SingleStoreIODefaultMapperIT + gradle-command: :sdks:java:io:singlestore:integrationTest --tests org.apache.beam.sdk.io.singlestore.SingleStoreIODefaultMapperIT :sdks:java:io:singlestore:integrationTest --tests org.apache.beam.sdk.io.singlestore.SingleStoreIOSchemaTransformIT :sdks:java:io:singlestore:integrationTest --tests org.apache.beam.sdk.io.singlestore.SingleStoreIOConnectionAttributesIT arguments: --info -DintegrationTestRunner=dataflow -DintegrationTestPipelineOptions='["--tempRoot=gs://temp-storage-for-perf-tests","--project=apache-beam-testing","--runner=DataflowRunner","--singleStoreUsername=admin","--singleStorePassword=secretpass","--singleStorePort=3306","--numberOfRecords=1000", "--singleStoreServerName=${{ steps.install_singlestore.outputs.lb_ip }}"]' From 6bf871ff424e106ac83cc56a6f42d393225289e6 Mon Sep 17 00:00:00 2001 From: Talat UYARER <talat@uyarer.com> Date: Wed, 8 Nov 2023 16:26:37 -0800 Subject: [PATCH 409/435] Creating a Fully Managed Beam Streaming System with Flink Runner on Kubernetes (#29305) * How to Create a Fully Managed Beam Streaming System with Flink Runner on Kubernetes * Updated Kubernetes Deployment YAML with real code * Added Palo Alto Networks on Conclusion * Update website/www/site/content/en/blog/apache-beam-flink-and-kubernetes.md Co-authored-by: Rebecca Szper <98840847+rszper@users.noreply.github.com> * Update website/www/site/content/en/blog/apache-beam-flink-and-kubernetes.md Co-authored-by: Rebecca Szper <98840847+rszper@users.noreply.github.com> * Update website/www/site/content/en/blog/apache-beam-flink-and-kubernetes.md Co-authored-by: Rebecca Szper <98840847+rszper@users.noreply.github.com> * Update website/www/site/content/en/blog/apache-beam-flink-and-kubernetes.md Co-authored-by: Rebecca Szper <98840847+rszper@users.noreply.github.com> * Update website/www/site/content/en/blog/apache-beam-flink-and-kubernetes.md Co-authored-by: Rebecca Szper <98840847+rszper@users.noreply.github.com> * Update website/www/site/content/en/blog/apache-beam-flink-and-kubernetes.md Co-authored-by: Rebecca Szper <98840847+rszper@users.noreply.github.com> * Update website/www/site/content/en/blog/apache-beam-flink-and-kubernetes.md Co-authored-by: Rebecca Szper <98840847+rszper@users.noreply.github.com> * Update website/www/site/content/en/blog/apache-beam-flink-and-kubernetes.md Co-authored-by: Rebecca Szper <98840847+rszper@users.noreply.github.com> * Update website/www/site/content/en/blog/apache-beam-flink-and-kubernetes.md Co-authored-by: Rebecca Szper <98840847+rszper@users.noreply.github.com> * Update website/www/site/content/en/blog/apache-beam-flink-and-kubernetes.md Co-authored-by: Rebecca Szper <98840847+rszper@users.noreply.github.com> * Apply suggestions from code review Co-authored-by: Rebecca Szper <98840847+rszper@users.noreply.github.com> * add my username on authors.yml and removed redundant paragraph * Updated Images with better resolutions --------- Co-authored-by: tuyarer <tuyarer@paloaltonetworks.com> Co-authored-by: Rebecca Szper <98840847+rszper@users.noreply.github.com> --- .../blog/apache-beam-flink-and-kubernetes.md | 403 ++++++++++++++++++ website/www/site/data/authors.yml | 3 + .../autoscaling-metrics.png | Bin 0 -> 79199 bytes .../backlog-graph.png | Bin 0 -> 28754 bytes .../fko-library.png | Bin 0 -> 176729 bytes .../flink-checkpoint-ui.png | Bin 0 -> 413413 bytes .../flink-deployment-yaml.png | Bin 0 -> 39477 bytes .../gcs-write-graph.png | Bin 0 -> 228927 bytes .../image1.png | Bin 0 -> 103228 bytes .../job-metrics.png | Bin 0 -> 348117 bytes .../job-start-activity-diagram.png | Bin 0 -> 177442 bytes .../latency-graph.png | Bin 0 -> 81918 bytes .../stream-service-changes.png | Bin 0 -> 1187027 bytes .../watermark-metrics.png | Bin 0 -> 46059 bytes 14 files changed, 406 insertions(+) create mode 100644 website/www/site/content/en/blog/apache-beam-flink-and-kubernetes.md create mode 100644 website/www/site/static/images/blog/apache-beam-flink-and-kubernetes/autoscaling-metrics.png create mode 100644 website/www/site/static/images/blog/apache-beam-flink-and-kubernetes/backlog-graph.png create mode 100644 website/www/site/static/images/blog/apache-beam-flink-and-kubernetes/fko-library.png create mode 100644 website/www/site/static/images/blog/apache-beam-flink-and-kubernetes/flink-checkpoint-ui.png create mode 100644 website/www/site/static/images/blog/apache-beam-flink-and-kubernetes/flink-deployment-yaml.png create mode 100644 website/www/site/static/images/blog/apache-beam-flink-and-kubernetes/gcs-write-graph.png create mode 100644 website/www/site/static/images/blog/apache-beam-flink-and-kubernetes/image1.png create mode 100644 website/www/site/static/images/blog/apache-beam-flink-and-kubernetes/job-metrics.png create mode 100644 website/www/site/static/images/blog/apache-beam-flink-and-kubernetes/job-start-activity-diagram.png create mode 100644 website/www/site/static/images/blog/apache-beam-flink-and-kubernetes/latency-graph.png create mode 100644 website/www/site/static/images/blog/apache-beam-flink-and-kubernetes/stream-service-changes.png create mode 100644 website/www/site/static/images/blog/apache-beam-flink-and-kubernetes/watermark-metrics.png diff --git a/website/www/site/content/en/blog/apache-beam-flink-and-kubernetes.md b/website/www/site/content/en/blog/apache-beam-flink-and-kubernetes.md new file mode 100644 index 000000000000..b50d475ed7bc --- /dev/null +++ b/website/www/site/content/en/blog/apache-beam-flink-and-kubernetes.md @@ -0,0 +1,403 @@ +--- +title: "Build a scalable, self-managed streaming infrastructure with Beam and Flink" +date: 2023-11-03 09:00:00 -0400 +categories: + - blog +authors: + - talat +--- +<!-- +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at +http://www.apache.org/licenses/LICENSE-2.0 +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +--> + +In this blog series, [Talat Uyarer (Architect / Senior Principal Engineer)](https://www.linkedin.com/in/talatuyarer/), [Rishabh Kedia (Principal Engineer)](https://www.linkedin.com/in/rishabhkedia/), and [David He (Engineering Director)](https://www.linkedin.com/in/davidqhe/) describe how we built a self-managed streaming platform by using Apache Beam and Flink. In this part of the series, we describe why and how we built a large-scale, self-managed streaming infrastructure and services based on Flink by migrating from a cloud managed streaming service. We also outline the learnings for operational scalability and observability, performance, and cost effectiveness. We summarize techniques that we found useful in our journey. + +<!--more--> + +# Build a scalable, self-managed streaming infrastructure with Flink - part 1 + +## Introduction + +Palo Alto Networks (PANW) is a leader in cybersecurity, providing products, services and solutions to our customers. Data is the center of our products and services. We stream and store exabytes of data in our data lake, with near real-time ingestion, data transformation, data insertion to data store, and forwarding data to our internal ML-based systems and external SIEM’s. We support multi-tenancy in each component so that we can isolate tenants and provide optimal performance and SLA. Streaming processing plays a critical role in the pipelines. + +In the second part of the series, we provide a more thorough description of the core building blocks of our streaming infrastructure, such as autoscaler. We also give more details about our customizations, which enabled us to build a high-performance, large-scale streaming system. Finally, we explain how we solved challenging problems. + + +## The importance of self-managed streaming Infrastructure + +We built a large-scale data platform on Google Cloud. We used Dataflow as a managed streaming service. With Dataflow, we used the streaming engine running our application using Apache Beam and observability tools such as Cloud Logging and Cloud Monitoring. For more details, see [1]. The system can handle 15 million of events per second and one trillion events daily, at four petabytes of data volume daily. We run about 30,000 Dataflow jobs. Each job can have one or hundreds of workers, depending on the customer’s event throughputs. + +We support various applications using different endpoints: BigQuery data store, HTTPS-based external SIEMs or internal endpoints, Syslog based SIEMs, and Google Cloud Storage endpoints. Our customers and products rely on this data platform to handle cybersecurity postures and reactions. Our streaming infrastructure is highly flexible to add, update, and delete use cases through a streaming job subscription. For example, a customer wants to ingest log events from a firewall device into the data lake buffered in Kafka topics. A streaming job is subscribed to extract and filter the data, transform the data format, and do a streaming insert to our BigQuery data warehouse endpoint in real-time. The customer can use our visualization and dashboard products to view traffic or threads captured by this firewall. The following diagram illustrates the event producer, the use case subscription workflow, and the key components of the streaming platform: + + + +<img class="center-block" +src="/images/blog/apache-beam-flink-and-kubernetes/image1.png" +alt="Streaming service design"> + + +This managed, Dataflow-based streaming infrastructure runs fine, but with some caveats: + + + +1. Cost is high, because it is a managed service. For the same resources used in a Dataflow application, such as vCPU and memory, the cost is much more expensive than using an open source streaming engine such as Flink running the same Beam application code. +2. It's not easy to achieve our latency and SLA goals, because it's difficult to extend features, such as autoscaling based on different applications, endpoints, or different parameters within one application. +3. The pipeline only runs on Google Cloud. + +The uniqueness of PANW’s streaming use cases is another reason that we use a self-managed service. We support multi-tenancy. A tenant (a customer) can ingest data at a very high rate (>100k requests per second), or at a very low rate (&lt; 100 requests per second). A Dataflow job runs on VMs instead of Kubernetes, requiring a minimal one vCPU core. With a small tenant, this wastes resources. Our streaming infrastructure supports thousands of jobs, and the CPU utilization is more efficient if we do not have to use one core for a job. It is natural for us to use a streaming engine running on Kubernetes, so that we can allocate minimal resources for a small tenant, for example, using a Google Kubernetes Engine (GKE) pod with ½ or less vCPU core. + + +## The choice of Apache Flink and Kubernetes + +In an effort to handle the problems already stated and to find the most efficient solution, we evaluated various streaming frameworks, including Apache Samza, Apache Flink, and Apache Spark, against Dataflow. + +### Performance + + + +* One notable factor was Apache Flink’s native Kubernetes support. Unlike Samza, which lacked native Kubernetes support and required Apache Zookeeper for coordination, Flink seamlessly integrated with Kubernetes. This integration eliminated unnecessary complexities. In terms of performance, both Samza and Flink were close competitors. +* Apache Spark, while popular, proved to be significantly slower in our tests. A presentation at the Beam Summit revealed that Apache Beam’s Spark Runner was approximately ten times slower than Native Apache Spark [3]. We could not afford such a drastic performance hit. Rewriting our entire Beam codebase with native Spark was not a viable option, especially given the extensive codebase we had built over the past four years with Apache Beam. + +### Community + + + +The robustness of community support played a pivotal role in our decision making. Dataflow provided excellent support, but we needed assurance in our choice of an open-source framework. Apache Flink’s vibrant community and active contributions from multiple companies offered a level of confidence that was unmatched. This collaborative environment meant that bug identification and fixes were ongoing processes. In fact, in our journey, we have patched our system using many Flink fixes from the community: + +* We fixed the Google Cloud Storage file reading exceptions by merging Flink 1.15 open source fix [FLINK-26063](https://issues.apache.org/jira/browse/FLINK-26063?page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel&focusedCommentId=17504555#comment-17504555) (we are using 1.13). +* We fixed an issue with workers restarting for stateful jobs from [FLINK-31963](https://issues.apache.org/jira/browse/FLINK-31963). + +We also contributed to the community during our journey by founding and fixing bugs in the open source code. For details, see [FLINK-32700](https://issues.apache.org/jira/browse/FLINK-32700) for Flink Kubernetes Operator. We also created a new GKE Auth support for Kubernetes clients and merged it to GitHub at [4]. + +### Integration + + + +The seamless integration of Apache Flink with Kubernetes provided us with a flexible and scalable platform for orchestration. The synergy between Apache Flink and Kubernetes not only optimized our data processing workflows but also future-proofed our system. + + +## Architecture and deployment workflow + +In the realm of real-time data processing and analytics, Apache Flink distinguishes itself as a powerful and versatile framework. When combined with Kubernetes, the industry-standard container orchestration system, Flink applications can scale horizontally and have robust management capabilities. We explore a cutting-edge design where Apache Flink and Kubernetes synergize seamlessly, thanks to the Apache Flink Kubernetes Operator. + +At its core, the Flink Kubernetes Operator serves as a control plane, mirroring the knowledge and actions of a human operator managing Flink deployments. Unlike traditional methods, the Operator automates critical activities, from starting and stopping applications to handling upgrades and errors. Its versatile feature set includes fully-automated job lifecycle management, support for different Flink versions, and multiple deployment modes, such as application clusters and session jobs. Moreover, the Operator's operational prowess extends to metrics, logging, and even dynamic scaling by using the Job Autoscaler. + + +### Build a seamless deployment workflow + +Imagine a robust system where Flink jobs are deployed effortlessly, monitored diligently, and managed proactively. Our team created this workflow by integrating Apache Flink, Apache Flink Kubernetes Operator, and Kubernetes. Central to this setup is our custom-built Apache Flink Kubernetes Operator Client Library. This library acts as a bridge, enabling atomic operations such as starting, stopping, updating, and canceling Flink jobs. + + + +<img class="center-block" +src="/images/blog/apache-beam-flink-and-kubernetes/stream-service-changes.png" +alt="Streaming service changes"> + + + +### The deployment process + +In our code, the client provides Apache Beam pipeline options, which include essential information such as the Kubernetes cluster's API endpoint, authentication details, the Google Cloud/S3 temporary location for uploading the JAR file, and worker type specifications. The Kubernetes Operator Library uses this information to orchestrate a seamless deployment process. The following sections explain the steps taken. Most of the core steps are automated in our code base. + +**Step 1:** + + + +* The client wants to start a job for a customer and a specific application. + +**Step 2:** + + + +* **Generate a unique job ID:** The library generates a unique job ID, which is set as a Kubernetes label. This identifier helps track and manage the deployed Flink job. +* **Configuration and code upload:** The library uploads all necessary configurations and user code to a designated location on Google Cloud Storage or Amazon S3. This step ensures that the Flink application's resources are available for deployment. +* **YAML payload generation:** After the upload process completes, the library constructs a YAML payload. This payload contains crucial deployment information, including resource settings based on the specified worker type. + +We used a convention for naming our worker VM instance types. Our convention is similar to the naming convention that Google Cloud uses. The name `n1-standard-1` refers to a specific, predefined VM machine type. Let’s break down what each component of the name means: + + + +* **n1** indicates the CPU type of the instance. In this case, it refers to the Intel based on instances in the N1 series. Google Cloud has multiple generations of instances with varying hardware and performance characteristics. +* **standard** signifies the machine type family. Standard machine types offer a balanced ratio of 1 virtual CPU (vCPUs) and 4 GB of memory for Task Manager, and 0.5 vCPU and 2 GB memory for Job Manager. +* **1** represents the number of vCPUs available in the instance. In the case of n1-standard-1, it means the instance has 1 vCPU. + +**Step 3:** + + + +* **Calling the Kubernetes API with Fabric8**: To initiate the deployment, the library interacts with the Kubernetes API using Fabric8. Fabric8 initially lacked support for authentication in Google Kubernetes Engine or Amazon Elastic Kubernetes Service (EKS). To address this limitation, our team implemented the necessary authentication support, which can be found in our merge request on GitHub PR [4]. + +**Step 4:** + + + +* **Flink Operator deployment**: When it receives the YAML payload, the Flink Operator takes charge of deploying the various components of the Flink job. Tasks include provisioning resources and managing the deployment of the Flink Job Manager, Task Manager, and Job Service. + +**Step 5:** + + + +* **Job submission and execution**: When the Flink Job Manager is running, it fetches the JAR file and configurations from the designated Google Cloud Storage or S3 location. With all necessary resources in place, it submits the Flink job to the standalone Flink cluster for execution. + +**Step 6** + + + +* **Continuous monitoring**: Post-deployment, our operator continuously monitors the status of the running Flink job. This real-time feedback loop enables us to promptly address any issues that arise, ensuring the overall health and optimal performance of our Flink applications. + +In summary, our deployment process leverages Apache Beam pipeline options, integrates seamlessly with Kubernetes and the Flink Operator, and employs custom logic to handle configuration uploads and authentication. This end-to-end workflow ensures a reliable and efficient deployment of Flink applications in Kubernetes clusters while maintaining vigilant monitoring for smooth operation. The following sequence diagram shows the steps. + +<img class="center-block" +src="/images/blog/apache-beam-flink-and-kubernetes/job-start-activity-diagram.png" +alt="Job Start Activity Diagram"> + + + +## Develope an autoscaler + +Having an autoscaler is critical to having a self-managed streaming service. There are not enough resources available on the internet for us to learn to build our own autoscaler, which makes this part of the workflow difficult. + +The autoscaler scales up the number of task managers to drain the lag and to keep up with the throughput. It also scales down the minimum number of resources required to process the incoming traffic to reduce costs. We need to do this frequently while keeping the processing disruption to minimum. + +We extensively tuned the autoscaler to meet the SLA for latency. This tuning involved a cost trade off. We also made the autoscaler application-specific to meet specific needs for certain applications. Every decision has a hidden cost. The second part of this blog provides more details about the autoscaler. + + +## Create a client library for steaming job development + +To deploy the job using the Flink Kubernetes Operator, you need to know about how Kubernetes works. The following steps explain how to create a single Flink job. + + + +1. Define a YAML file with proper specifications. The following image provides an example. + +```yaml +apiVersion: flink.apache.org/v1beta1 +kind: FlinkDeployment +metadata: + name: basic-reactive-example +spec: + image: flink:1.13 + flinkVersion: v1_13 + flinkConfiguration: + scheduler-mode: REACTIVE + taskmanager.numberOfTaskSlots: "2" + state.savepoints.dir: file:///flink-data/savepoints + state.checkpoints.dir: file:///flink-data/checkpoints + high-availability: org.apache.flink.kubernetes.highavailability.KubernetesHaServicesFactory + high-availability.storageDir: file:///flink-data/ha + serviceAccount: flink + jobManager: + resource: + memory: "2048m" + cpu: 1 + taskManager: + resource: + memory: "2048m" + cpu: 1 + podTemplate: + spec: + containers: + - name: flink-main-container + volumeMounts: + - mountPath: /flink-data + name: flink-volume + volumes: + - name: flink-volume + hostPath: + # directory location on host + path: /tmp/flink + # this field is optional + type: Directory + job: + jarURI: local:///opt/flink/examples/streaming/StateMachineExample.jar + parallelism: 2 + upgradeMode: savepoint + state: running + savepointTriggerNonce: 0 + mode: standalone +``` + +2. SSH into your Flink cluster and run the command following command: + +``` +kubectl create -f job1.yaml +``` + + +3. Use the following command to check the status of the job: + +``` +kubectl get flinkdeployment job1 +``` + + + +This process impacts our scalability. Because we frequently update our jobs, we can't manually follow these steps for every running job. To do so would be highly error prone and time consuming. One wrong space in the YAML can fail the deployment. This approach also acts as a barrier to innovation, because you need to know Kubernetes to interact with Flink jobs. + +We built a library to provide an interface for any teams and applications that want to to start, delete, update, or get the status of their jobs. + +<img class="center-block" +src="/images/blog/apache-beam-flink-and-kubernetes/fko-library.png" +alt="Flink Kubernetes Operator Library"> + + +This library extends the Fabric8 client and FlinkDeployment CRD. FlinkDeployment CRD is exposed by the Flink Kubernetes Operator. CRD lets you store and retrieve structured data. By extending the CRD, we get access to POJO, making it easier to manipulate the YAML file. + +The library supports the following tasks: + + + +1. Authentication to ensure that you are allowed to perform actions on the Flink cluster. +2. Validation (fetches the template from AWS/Google Cloud Storage for validation) takes user variable input and validates it against the policy, rules, YAML format. +3. Action execution converts the Java call to invoke the Kubernetes operation. + +During this process, we learned the following lessons: + + + +1. App specific operator service: At our large scale, the operator was unable to handle such a large number of jobs. Kubernetes calls started to time out and fail. To solve this problem, we created multiple operators (about 4) in high-traffic regions to handle each application. +2. Kube call caching: To prevent overloading, we cached the results of Kubernetes calls for thirty to sixty seconds. +3. Label support: Providing label support to search jobs using client-specific variables reduced the load on Kube and improved the job search speed by 5x. + +The following are some of the biggest wins we achieved by exposing the library: + + + +1. Standardized job management: Users can start, delete, and get status updates for their Flink jobs in a Kubernetes environment using a single library. +2. Abstracted Kubernetes complexity: Teams no longer need to worry about the inner workings of Kubernetes or the formatting job deployment YAML files. The library handles these details internally. +3. Simplified upgrades: With the underlying Kubernetes infrastructure, the library brings robustness and fault tolerance to Flink job management, ensuring minimal downtime and efficient recovery. + + +## Observability and alerting + +Observability is important when runing a production system at a large scale. We have about 30,000 streaming jobs in PANW. Each job serves a customer for a specific application. Each job also reads data from multiple topics in Kafka, performs transformations, and then writes the data to various sinks and endpoints. + +Constraints can occur anywhere in the pipeline or its endpoints, such as the customer API, BigQuery, and so on. We want to make sure the latency of streaming meets the SLA. Therefore, understanding if a job is healthy, meeting SLA, and alerting and intervening when needed is very challenging. + +To achieve our operational goals, we built a sophisticated observability and alerting capability. We provide three kinds of observability and debugging tools, described in the following sections. + + +### Flink job list and job insights from Prometheus and Grafana + +Each Flink job sends various metrics to our Prometheus with cardinality details, such as application name, customer Id, and regions, so that we can look at each job. Critical metrics include the input traffic rate, output throughput, backlogs in Kafka, timestamp-based latency, task CPU usage, task numbers, OOM counts, and so on. + +The following charts provide a few examples. The charts provide details about the ingestion traffic rate to Kafka for a specific customer, the streaming job’s overall throughput, each vCPU’s throughput, backlogs in Kafka, and worker autoscaling based on the observed backlog. + +<img class="center-block" +src="/images/blog/apache-beam-flink-and-kubernetes/job-metrics.png" +alt="Flink Job Metrics"> + +<img class="center-block" +src="/images/blog/apache-beam-flink-and-kubernetes/autoscaling-metrics.png" +alt="Flink Job Autoscaling Metrics"> + + +The following chart shows streaming latency based on the timestamp watermark. In addition to the numbers of events in Kafka as backlogs, it is important to know the time latency for end-to-end streaming so that we can define and monitor the SLA. The latency is defined as the time taken for the streaming processing, starting from ingestion timestamp, to the timestamp sending to the streaming endpoint. A watermark is the last processed event’s time. With the watermark, we are tracking P100 latency. We track each event’s stream latency, so that we can understand each Kafka topic and partition or Flink job pipeline issue. The following example shows each event stream and its latency: + +<img class="center-block" +src="/images/blog/apache-beam-flink-and-kubernetes/watermark-metrics.png" +alt="Apache Beam Watermark Metrics"> + + +### Flink open source UI + +We use and extend the Apache Flink dashboard UI to monitor jobs and tasks, such as the checkpoint duration, size, and failure. One important extension we used is a job history page that lets us see a job's start and update timeline and details, which helps us to debug issues. + +<img class="center-block" +src="/images/blog/apache-beam-flink-and-kubernetes/flink-checkpoint-ui.png" +alt="Flink Checkpoint UI"> + + + +### Dashboards and alerting for backlog and latency + +We have about 30,000 jobs, and we want to closely monitor the jobs and receive alerts for jobs in abnormal states so that we can intervene. We created dashboards for each application so that we can show the list of jobs with the highest latency and create thresholds for alerts. The following example shows the timestamp-based latency dashboard for one application. We can set the alerting if the latency is larger than a threshold, such as 10 minutes, for a certain time continuously: + + +<img class="center-block" +src="/images/blog/apache-beam-flink-and-kubernetes/latency-graph.png" +alt="Latency Graph"> + + +The following example shows more backlog-based dashboards: + + +<img class="center-block" +src="/images/blog/apache-beam-flink-and-kubernetes/backlog-graph.png" +alt="Backlog Graph"> + + +The alerts are based on thresholds, and we frequently check metrics. If a threshold is met and continues for a certain amount of times, we alert our internal Slack channels or PagerDuty for immediate attention. We tune the alerting so that the accuracy is high. + + +## Cost optimization strategies and tuning + +We also moved to a self-managed streaming service to improve cost efficiency. Several minor tunings have allowed us to reduce costs by half, and we have more opportunities for improvement. + +The following list includes a few tips that have helped us: + + + +- Use Google Cloud Storage as checkpointing storage. +- Reduce the write frequency to Google Cloud Storage. +- Use appropriate machine types. For example, in Google Cloud, N2D machines are 15% less expensive than N2 machines. +- Autoscale tasks to use optimal resources while maintaining the latency SLA. + +The following sections provide more details about the first two tips. + +### Google Cloud Storage and checkpointing + +We use Google Cloud Storage as our checkpoint store because it is cost-effective, scalable, and durable. When working with Google Cloud Storage, the following design considerations and best practices can help you optimize scaling and performance: + + + +* Use data partitioning methods like range partitioning, which divides data based on specific attributes, and hash partitioning, which distributes data evenly using hash functions. +* Avoid sequential key names, especially timestamps, to avoid hotspots and uneven data distribution. Instead, introduce random prefixes for object distribution. +* Use a hierarchical folder structure to improve data management and reduce the number of objects in a single directory. +* Combine small files into larger ones to improve read throughput. Minimizing the number of small files reduces inefficient storage use and metadata operations. + +### Tune the frequency of writing to Google Cloud Storage + +Scaling jobs efficiently was one of our primary challenges. Stateless jobs, which are relatively simpler, still present hurdles, especially in scenarios where Flink needed to process an overwhelming number of workers. To overcome this challenge, We increased the `state.storage.fs.memory-threshold` settings to 1 MB from 20KB (??). This configuration allowed us to combine small checkpoint files into larger ones at the Job Manager level and to reduce metadata calls. + +Optimizing the performance of Google Cloud operations was another challenge. Although Google Cloud Storage is excellent for streaming large amounts of data, it has limitations when it comes to handling high-frequency I/O requests. To mitigate this issue, we introduced random prefixes in key names, avoided sequential key names, and optimized our Google Cloud Storage sharding techniques. These methods significantly enhanced our Google Cloud Storage performance, enabling the smooth operation of our stateless jobs. + +The following chart shows the Google Cloud Storage writes reduction after changing the memory-threshold: + + + +<img class="center-block" +src="/images/blog/apache-beam-flink-and-kubernetes/gcs-write-graph.png" +alt="GCS write Graph"> + + + +# Conclusion + +Palo Alto Networks® Cortex Data Lake is fully migrated from Dataflow streaming engine to Flink self managed streaming engine infrastructure. We have achieved our goals to run the system more cost efficiently (more than half cost cut), and run the infrastructure on multiple clouds such as GCP and AWS. We have learned how to build a large scale reliable production system based on open sources. We see large potentials to customize the system based on our specific needs as we have a lot of freedom to customize the open source code and configuration. In the next Part 2 post we will give more details on autoscaling and performance tuning parts. We hope our experience will be helpful for readers who will explore similar solutions for their own organizations. + + +# Additional Resources + +We provide links here for related presentations as further reading for readers interested in implementing similar solutions. By adding this section, we hope you can find more details to build a fully managed streaming infrastructure, making it easier for readers to follow our stories and learnings. + +[1] Streaming framework at PANW published at Apache Beam: [https://beam.apache.org/case-studies/paloalto/](https://beam.apache.org/case-studies/paloalto/) + +[2] PANW presentation at Beam Summit 2023: [https://youtu.be/IsGW8IU3NfA?feature=shared](https://youtu.be/IsGW8IU3NfA?feature=shared) + +[3] Benchmark presented at Beam Summit 2021: [https://2021.beamsummit.org/sessions/tpc-ds-and-apache-beam/](https://2021.beamsummit.org/sessions/tpc-ds-and-apache-beam/) + +[4] PANW open source contribution to Flink for GKE Auth support: [https://github.com/fabric8io/kubernetes-client/pull/4185](https://github.com/fabric8io/kubernetes-client/pull/4185) + + +# Acknowledgements + +This is a large effort to build the new infrastructure and to migrate the large customer based applications from cloud provider managed streaming infrastructure to self-managed Flink based infrastructure at scale. Thanks the Palo Alto Networks CDL streaming team who helped to make this happen: Kishore Pola, Andrew Park, Hemant Kumar, Manan Mangal, Helen Jiang, Mandy Wang, Praveen Kumar Pasupuleti, JM Teo, Rishabh Kedia, Talat Uyarer, Naitk Dani, and David He. diff --git a/website/www/site/data/authors.yml b/website/www/site/data/authors.yml index 2776132cf586..13c31c4f4782 100644 --- a/website/www/site/data/authors.yml +++ b/website/www/site/data/authors.yml @@ -275,3 +275,6 @@ pabs: namitasharma: name: Namita Sharma email: namitasharma@google.com +talat: + name: Talat Uyarer + email: talat@apache.org diff --git a/website/www/site/static/images/blog/apache-beam-flink-and-kubernetes/autoscaling-metrics.png b/website/www/site/static/images/blog/apache-beam-flink-and-kubernetes/autoscaling-metrics.png new file mode 100644 index 0000000000000000000000000000000000000000..bf723d5d5a7af43cb969f960f15c91c53f150caa GIT binary patch literal 79199 zcmb@uXIN9++BK>oMGz205Rncd5<of#1QCLC1QZM{N|D|Lq$H>y7MgVF(m|R85;`a< zq4ypj^xjJ-p_~=n`#byj-u=AS^?1$?B`mVmn)9A*jB$@S-`&@`M@@N^^3<tQ)T$~< zx~EQ^4LWt|bR7j5aAnZg!1dIr7pGK}6dpV`U9Q=8yKI17+duX{&h_)AW9b_0+NdPd z5LRo7a7oPr56eCqqgx(YR>^h<n){Q>nwz&;D2sM%ao=e@6A*C6I_AO_=-h=fZJ)f9 z&#1mwUpyLpLqDk^ai$dAA%WBWL8%&d&#QmP%4!uQ&nw#DXItS56)Z!0+_N(>@+>u} z=;kjpu4#UJ>&kgD|8M{LvG=;c&wJ3as}CF_{GZn!%jTzY1|1N}*$dHSyM|dz3W(=5 zJJi%Q{i6TZ*MvVIDZ1zE5w*S>eUeByL4|lDLvOk3vR4!0mrumRB=CmWbc{kK5lkg> zYL2GK+&tOMK|*!Bb5udA2g84P)-Be+EBB!RC(v43)S=MHl3xrX4gtG)Gp}ku;o1|z zx2gE}gv>sBq1|n!{lh$`>Bfzz(Iq-+O-+6F9oZ#{I&+x`vvVy8@kYxh#twCEUg=)R z;~RTb<A<bPi*a*2JmMx=v(wG*bHsI_4|%M~mF%#W`M=bc;ksR=qe`H-UfOdFe+kZn z`L+?m#5Xxh9awDW!w34!<FxY8bEXJI#K{jm<jGfss%yxn!^>Vsr<2oFqt!iD7VU?S z2+!GkUlIF&L!949$7U-mB~tNC*x(MI&BA884_m@YHUGJ)$b9zt&HQD1CfR8!ui5Bb zi>_wtTqg?EPh6PL7A%`DhfwW7a{k-*=#Do75%I}<MYTt%vPY9LNXIFLs>8$AcfAfO zVp}WEOFy6g;wx&F5e2K}mho7P7z&DMS>Mo`*^!iw5->6^@U^Y#zK+pOj^beR<6(o# zYQz*73w*z)5fya#vS+l_e{=(&-nEcA3p&<EI#3>Q6bkQosJ;I)s76p!#^W&HaDOP) z!qY6G`e2w@#Ju0$W^Mk%V!Yx0N=kC90M@L{tyERH-_&aXRaP(!EwzX{#BXFz6LNcI zG>)aEqZ9bG<+jp-ugPgOgr1lpcGk3;lc(wVK!Py_I95;!4M9JvEs@KK+8aj03g-OD z!?W4lQq*`a#aA%0Hbt<`dm6{X$Mdi~-a+oDwM4{ma#zH17_^^qa#%F$RWf7z&5J*% zTSSECiD%m3>Ta0oamAU)CEtCglXZS3IRx_j(V^Xb4-6JD#jD-4P_n-{S0^jLhB%DV zNDx*>5PtM*3Z3BJ$|){TTGIWWQ2~S_81`seonZo$_9KIyN!}$sDmvDvNyH8B>vNLw zSn>HGa)y7pt&AKRF6Pi+@8#2Zf&b$Lzr8I1XfHzqM5q=YBUdu7fv7!Pso9+we5cpl zHN?lI92)bD%@@RzX{oDcz{}>l7w)?;a|?(NuOEElGLyJR^|Flf{ZLjJ!9H<uB)>~o z^+^}6YvHG_rr?;5iDqc`?m`*o^$!+n^6W-sm}cC`k`uT0k#f@-N07(L@Tfr0)AmFG zp}Nh_QSk$B)Tc}lM^1S0B4ZW-KQBC~baW)YJ2f@Zdcl=Hqg{vLFT;J0RQXryKY9R4 z@s%(hLs)&N-f#E0$m};qozI__H`7)}9pkq*cMs@WV;?Zddyz*+ja}yW)-?IUebS*T zC2h@VrNJPtiD=1uP_vAi^4L|e!LbsGj-Blu?Uo;F*=i>6n$gw4wT7loyJT4PpZW!+ zTx&Hasnz%ftjKm`M}9Aq38i0q1>eSEB{$(dem*TM5CKX-Y()El?aO61v6!}#do4q5 z@`6F&gLy9gMD{9EKdZO!kRGdck)k;P!l8BKAj`&PW1YMj^v8b=<0(d_qX>w?cz0fA zi-C%vcLPfrMQr?v{If^L8#o02)OD@!hdOo^eT%3~`QthHrr~*NeQvp}Njnj*#q+rZ zd9kP^$tEYZlgMaDwq`ZHpIzE{@5_)@m9LyxRJF`B7oPxsNf`X(d)Rc#6PT)tO|He7 z47;>f_DbB{u;b$}e13|_J5`ZA|5W|GanyKz_V;^XP4$`lGd~S*AK>J)O}UBJyk_@f zE&JlDn$l9sEV_8SR$@SgRZk*i@tEut9w#Xyn}hrz*^;;ap0kOyYY>uHRb}A#BJ0cb z0a7yC9&2@SkXZ9ljFr(4eP_&jjjQ634j(S`SPQY!SY-KQzIb!f&Z&tj4E%JB{J{fV zPNC|}P+$CvWJH3eVf*Y{%apb(-}gEef%Lir_OO$E>7in`vZGm~<S{-um{n5ex@MeU zNA3FO<*`V!HR$#Mvr{uLK%d5*V4VD%oIccWffY=BXWZjT9*4{6otw1X8Q#GMr_Y&s zn{<orHn0AW+ni=Qv9Rn~mfo7)5_;p_l2%6M$L(^iYNaNEtcH+$-g^zsBy5{6dz@X@ zTz81AYpU3q;|u!B@^CmgnQW=EM|R;+Xo>UWB@b*-EJlPNzQZWkSk1$~5(k0&<UyMC z*+R>oRV<Gr-C&k*Dwb?(*vhtg+O!RITcJjVleDwfYv#LL*LC#{9KGIdQ9om+XVr=> zb~8xp?Ux*!YY54)>M(iCDzAtB*`N+r^p~Tu?ujW7*&r++Ic*E_vE|Uv_?0Q?fdpK* z(bz%>(sk!{0!$wd9zyXB&HAg4qu1V+S%D|kE@iuNKQ+p?AFWkn0GZwlJ3biD=RA7k z^Ov>v@!nMu&Y%kdu|^Kt#(@~c?XxH36FnAZI{b0i!T{ERVUM55UH&uQPA2S@n6DNu z<Kr|M&$W1iC7#R1yk_<Dkw!RK+n08{9<YH$w(TH~*bZ5YwHT|9u|>SwA4RRUX1;5` zX__s-a&x&A`HGum$Zkx-D%W!VGCKS>)Zd?ugw*xs<T-r^NhoV(OZJ8$wg!7PH?m<^ zCsQiFYyUYb0{xETM$qcT@<bYWm)VOy=!S1R`s<>(p-$Dz`TybzVqaDEy#LqP=JBmc zf8<}fGns!EOBCd<3sUgvCy?1+1NMykuaoq7vUHgnf9)l=u8g{LAm7&i_3=}&%rzD| zZtCiuk4<1*TO0p+6ByDm(+`$EtVgyKIMlFq{GK;9)F~6M(VLRL#;_oBax0*PrVg=7 z14VBp8$wK7zAML$%>S!c>q5gaw<|k7(ZoU?SuG`jkHdZs(eF#Q#I^2|zivrgFV|xb z7jHq?4b058B}gFlCf=ATP;dlXm=wJcHH=VVFsbqg*`Wb0+(i_R|M<iYPkh3{8U+D< zr|`=8%GB-PP_y=31*!zx^8t4I_a8XmcYa@ZbGe!GF7YqmPasa<md5YEB1U|c3fp}K zI^ZwUPaqJ6Crs~ej9d{p)uQtZa`#DJU!SF0H5Q3&Srf3b&(6!+#rqG0rtHbEZ|`CK zPUz)hc!^ISiy=fq9)+Hva=+hEET5BraP;nXl1G7Z3|1xQ)^8WjEzzQn%pC?JeY`-w zz8`ox7IGt*xOQR9JymqoK0nZZID{#UKWP~&gT*Hp)w?-Uny%+28uRfBtqI<~|6+Q2 zR@FsyfrXUref3CnkL$){rq9j$@87@Y5n9tIyR|q!AH{VqR87;NU=*RL8IoaTF^oO~ zpaTis(uGl}=t!<PA^P|M7o#7<idu|Uut5VXvpW)_lg?FfyJ$y5#l#E^r`Ob&Y<46? z+Uk*PAKq#cH_4*mlkzjA)GX*88>2IPy7=kv(9LywUf9Okei>mBGQ0ad3&AHqRTJoQ zdZ+z${ar%5j&)j_R&sOrWVbHXLqH&Q1-G-a14o-{fByWLXKmX?&rf1>r<mAYOjpPr zFr*H?eLr@?Z&y34F(t)I>$uMEz`$#!U^gQp<3Y2KX%sL`S6`Fy7-7(m3!<Snf7IXA z)n#(_aTQBa68gir_R4Q3pT2&bT9<#s*2t2tIDVoN@Wf}p@o2P0Zp)9FIz+_r&|-*{ z8CPRs@3z?+hx8a#uA&BE%oB_Zf#oNiM-bd~?DwU*1hZvE*dr_!3<T88wI}gL)}@%m zl8CK)PVqL*3K%)GCzzv~P6)7*DtZQnA%{ZKO9JXKMWt9}!RKuX9Pf4pY0h9z5Q}1w zb7U2^V^NQpl0|PQZPcD!BTwevuH)c?5IfzmTM?4Ne0TK+x7VUjN8{S@Ua9N>TV1T= z*i~-m5MlYv1Ac>oa5#Zh#my}zl=4_UN9dL#IqF>hXrb3#s~e>57aBYcmCjm?j0^9v zpW8WyptXIU?0!FcpSj5DVXxsk!xGcapY4`!gJ7i=xd2Wuiin7Ry7l#hCp)8=zo_=Q zk(pTrs-VE9N+J1zwgSx{V%=xmcSGiqMf3JhgkRwu*RA%(o-CR~uzA)|=Xo}e)*+&D zdO8_hM1`fcjUPvm?a&U}HwO<zM0y^C4zTp9D1av%dbddK5Qo$VGs(>$!G{kTi`e{> z@!Z+NFPW__*AT`o#L4)*jEaRt@g8p@HIjr4W?Nz031#*rq*P>XvWTYc%gDzYA^^B8 zCP9oXh$0(731IVNi>E6kB!2y4@T_tuueyO!(>@hr{z%31g$MytkN8*I0s@*>nG-Zz zkm?KxCn2VFL-gDz51)HsNgwv9VjCwd`KOb<u{EPtWGbnyHb!kkW(_(hmM_aYt32&6 z;%;f(I)8Pu_y_NDvFmj?(UxQRm?=HQLwRtV5n80VHtXYx3;JU`7+x97A~}|<bC$i} zx>h{sdmR%8z~@A4x3BEuPC8DmQ!xmj_NkdR-!;kgpa<gTchD@vMRrcfrNVkhjc|>e zh9yRL>iS37^p$M;rG6)*6<FbggLn~nk=Mq1l&%icVHmCXv@MyRDi#zw>Jzc{!Hqa7 zDr9y!gVtr*=@Bb!p4_(XrE-eox)#$PT|}N9DoH=v%K<(5@+eCy#nLwT^~IVeB~ek) zL!8Uw6W=`!VfGD#bd2`k9O&lFaLp`zLyLxu>E36#xdoB%EHJdpr33^+>5FJ;mLdQU z+}2i-HuG`6&i_rQ3Ivjwk!9KFswbImS~Zpo2{5E`@1u+yh`5|L-BNU~@#2>8!@fQV z&qmH(syg_gP*yS&<nF%H-QCTJ3%(*ycN~V%Xk##63hmw2fFgaPb5-?Qnt-WGfL~JF z_<drVJa2Z#tomKEh^VL-_W=Wf=BP-0HKNz4SY~oYvhFCU`sCX^*WqaqzX?ug`SaXF z;T`&RC0M0wGvZ_5Nl}s07px&2N*TZyO)(#d=pxs3Z(evPp$1p$F&$cxte<bnU!qZE z#sZ#hX>CRq`4reNPkR=XU1TTD?Qc&<L*CG^g-aewJo;>ByV9VeQt)n!ea&yL*tnVk zx?KtoaSGABqG(1_AME*gpOtLQqWF5sN%KOc<w)J`jLJ&oS|;g+=Cn-p^y$8`Ko#gd zB<{9bX!X_@ITE249IXTZL9SvKnF98!hu`k&UCWOoa-f5KgRJsFj+FS6pjc3GY&iA) zydIOJ%V&^jrSb7$E7ipk2LcPyx;`TpU4AEgb13$^Wp@;l1e+{F%Ee@<I4&4mRIT4( zkh@2N72tJGwcm_6zW*5K#?Yvy_d?Fbluau)S;AqM1xexHKgWD{e7^}`5rNbv{j>PZ z1jSYl?WXjwV^gh!89l4EXdRqp)I+=eCS2>1hI&WTRMs9L7ItJf+YaNl(%`_42B>Cz zH3U&H2!$8j3%-~*Ml*y*H3<NK%opq%l^^^*MnsSOXw>h-he-y3ye9R0&q{743FxdF z;XdJzoHx$s)~Yo>&U>Lr^I$Bw)nz{!+k6X4%a$^(drg=WIEUM;P*h!w%wRG!G1k-Y zx^~i&nRZz{&mZ<!jyXz7i>?Suk7W$GLV81J>=8Ll^YVVs%!;S{7c=sm@0}fial(Ef zm!&Udx1o2s+D+a8oKlu>dT^SrXNBCDQINhlN7%g0<9cH?M18s<7WV~3wJ2~38c-4f zP^urjYLS^a<%89MQ)ox8<II7TH_qk8tQ|pD9hU=|JY;y?G}=VrG1R>}qO+~BRFI#e zR@<Ml;>9fEmo)|*VC0J_v8D*k$w|*<D;S)gzilbZutvFd-y^COR{8)e>`?M-h*H{R zGUNQww5VPfT;N>Q?nex7qtsTSb)=JWX!NL$?(!Rtsp|-*Ve4WI$qpH>VGZeL(;r)J zp?YIse*8<4j>T45JHBDS40@s44_FUZuw}5ZlX<-=@ZetmN^utZRWr*jfNXH6Hu2#_ z`04L#F@j1{p3C8XP>DD+TpLRI2#U@cmv(#~U|{NpMKq(I^e}i$ci{%L!>-4fQCjug zsUP3h&wr1!T6#Rnmg244KFiohq4xp4zA-m@Vm3XcEg$<$F;K<Uuo8f~Y%7*96biLj z!3Q=&<2i!TBvod|?Du=~-#os%A#O20cQNI|4ZD7ar~#a}LE3;*DRMS*U@S&G_NK;Z zeeQQY5i7jU{C0Q48j~~V<3uc*f!=9+tK+D9X{~YdVV>oDHf%WYOO5mvL-QvNwqenz zWVpkJYto4NWS!DxdOvfFX2cPJwz24BJ>U3qPN{>OS?`toHgoMJzkR>syy$z+X?)er zK?4S4y<0su7pziRpe2BUA@@|iIQ<3@&#fjAyCdKFL@O`M*P-KpI%Yy+BHSFKt~_fh zYT6;@$NSMRqrH~{Yq79-*>!tRm<jA%9h@TDhd32TxX3E|>=Bqv!Z_>W21{(N{7Exz z+0uOKx@N3PWHvVbv^k-F@cF*&^w}ZSI(0A`*f63;q}4nt-f%+3$D@b4q)l5>fS#?c zhmtoj+FvwYM-pV}Jr@W0pmnJ(<+@X~_@L=$c|^SuGtkRRP|WR*?$*2(4XE8aFxuZt z=yE;KsJ$juw4W)7i$2*q>WTRH(fi(tKjET)akWNP`$vbxsBmO^@BI{C;gQ;Yp*Mc~ zE;61(mS*qZzMz}Qy{sjM>QFLL4`G%0rk2&;FX7YC!f5KH&bl9lX~V~;LoB1ConXFl z^@uOz9QaiqwnlwDtfpkg>lA8gYI-7kZX~${2$LxRay!c{cs&t%;pZde=OSCb%BLDJ zWLjAax=R&!H?$7n0|%tsy^jJlH?!wpW!aK!QqNV!5OZ=vN#z)|=zstdyokhzfwg)o zdR05$xnzzu5OMR8MwhU|tZ{{J#Y6DXB>%og?-f-wHMC?*Yx|g0$7Ap1VtK>U0VN;S z-|3}TF+iL?(Cb;l?J4CFSzfnPf%hyv{#Y9kXwff-R`8b>dELpX_agvpq66jQ!f1RD z&ie9hl#ci0cz&(*AOmbkchzhDi<2gynj5E`@T82|X1gQouz|ta($aEsbIA$I?2zst zX+I3Y1z)wJ%HU1*QzYuH9)V%k1pJ;aLie-%jwDqFQCL$)i3LI~-c-4Xj}M8Ne$9Ew zia~B;IuidB3yLA4IiERG3pxa5=8;hKZ55AIHDoMms(X8%ilHe-7&k;%)kqOH@QAye z+SPlnJf$PTl7E>TI@f-=5xr8p*^uCGIh}J3dD)7RmRV3eR<La9V0H9eL}Vm?WL9g8 zjp%I4PxZ@qX^xNvtwheE8o9ybo#48AaaXPDe>8aza$e(Ca89p#KNe3pz+d0C#wYG0 zdwY%KEa?JTMqaC1%#M(5F2v#ny>GVLlayUl(b9}&agmuZ*DD(*DN)5~ms#;8Nk<A{ zv=c?VPBfDdt2}xo*UG1mp1n|JNr&h;n6DmUmmvQ!UQiRA;Jq+2fC~?kKT*TSU$r_| zu0TH7Vj%Fa=Pod9u6*3U+jG9n+YSa9si&?SAB-$*cAOlO?|V;s;=h0E?R}=_54R$v zUoDApJZ6+!NOsuN9*nlGd-NJuJ2-YbUNm#kE3+k>nHt?COE6<Moo$10Ys>CiHMO=P zk#ifEln-=V_pmLM$q4-AcOgM#q}_X}J<B^g<=ghnrA&A|6fisYBp>;!7vDElXw|s& z`g>NJ@F<c3_i3YSS3cbDuy1XpEJWe%1Xy{E`e<YfX5G|nIlGatvM``6yYs3Q`lUA+ zzO|><(U>J*wDtia6-1t`KV_dJNnRtQZcrH0*R?dWt9=adxX!JntdFiH9AnTghRik% zQMZQ~?EJ@=%_zV9d<xA!Jd!_57nsin9XO5vQ(i^t{DF|)ZkYaq)W!g@1Ke@giMoi0 z=t{N|jF#<5Ok3&77NFP;U)yNouJhU4cZ3G`FidQ_C%U|zq>3#n5cfieZJT2QyJmN$ z^fZjoyFV^ztGNjOVydq4BSD}ud#CK1dxtj1#%V2PKKCJ??W=hmwUw<{ZeC!5!R7AO z9@>`Y<I16AbtmDVN2NbZwE~6Z1QD^mcD5NEV7907H<-n}t6Pn}{6IWLnOR09%0~0! zw3GPsvfGesh||80OP#ySfJFpj@$)kZ4OTo7Z|)TX1k|R$5F?9wzOUq{4)P{&jipQ_ z4oTa~TSYCU3vP}rY3-eIK~GyvEEW?1f>U8IGfFFu+Euw5y7$3;Xmdtnew5=P>S+N{ z*9K%uRL#Ba>V0*Ky|-PP$M;f(TBh&&ZQI9AVYHjwgla)+G_0JYq@?&5Qa|6>%S;Zg zO7KA1Mdi?>$4ezn${~?Jas(?{>imvrnr;<S>V!MZB>l8Hx2C^28<uutwtU3H5$eo* z%J@2C4I;AUfn)!&7n&yR7w2aU2H@aZSG}*TaWb$#zpVnfbP7pufYN6P(x%b{Q4O8q zPSXT%3q48s94*-WJFgu8aBbMa^va4?f`e;Bo^<bKIbSso%HcOCH1rB6N~C>%p^-JQ zKG<$<X>oB`ByqULx9=*Gv~9juZ5%#N-^Ki1qOh24t5lVJeFh(1@AIA8sdUaloW3Hj z8L2n|wDX>}CYu)#Fg61>8t*&9B@t;yPfNri$xRvCR~1|&ZJHnO`>+?RyH`l*QNFg& zW_M%{rwn?(Y$xTr_l55u3)?cfZumY(-9B8@cH0yAY)H-M+1)Hn;ny5m-^uW`8a@%r zZEPG<>#E$f-5whemiUWei3P8h_BPzhaN2R#qoW_zyodhcZ%IDUkn%W@hPj+RyA^1A zIwmxd#bpCO+xsX~y<@iB=&|j~4G}ke>*|KM1{W6@%fwuvhL0FUdU0{dmUow=@e%OE z=zx|@bv5;s8iX^`74+riwZLU;nfB9;Bwnlbz}n{|Rx}gJUFEwM1bw~_HRN!tUlOkm zj&rlv-0dlNd=C7!-tHR@EGKvU=hXx=^sr%0Mn-G0PJn16QIPl<6gxa1X0u5Ty^JzM z0+a9vd&oM3a}V7Qq><L7swnnBCTL?3k&<OgeEE5rZt~cpbaXwB<(~8TUGDfY=VFIT z#<O)#rk?t+sZ7Tt#PTvrVpJ5qz3!yWAIL{7IbT{{%hrkwPsa=&^549v!Tb|D@P7B* zzh<cx&KH)0&R+!CpUb{axYN7s>${H1J|({)$%=J@K7SHe=i^BA=@C_}2x{(xtb;k( z;0yS$;yW9`T$;3Ew~s;Yoa@SdotztHvJ<a}T_gy5qTM=h-?6gQCck0d>O%(7$I`Z> zGReEoHEw0?<}<yT$;n&aTffSYWAdfkiu%F@*}}Qa&aJ7u8J~e$6CI;let@b;FXvZh zLNalpW4B{GAX<OU4_vC+QMcg(bAP<Irf~KoY#yNrKV&1SO{*z@+H@`B9PoKleH`&y z^m@DYTl5OpKK=UwtNx83Vu|h^yg<Gs)}5CBp9^TOf8Uxx`}<j&U+(_W-Y#^L|4XHN zl=^ENQY?`C7UN{||34I?oDJIB*T=K6!s3~~wphu>!oq@{r;JHOEY2@in==tBqmuDk z{R2v{do$?E@{f0sNUs)&p$dMT&!6)OxSM6*hG<6|2NTpMGc{G`wu}sqq);hHT)@QE zZpmJbDBWDR$oKCueyeHZJTx`dy_tZvkKS$t0dzGk!_uM|5bB1ml{}2l)wTTjxzBaI zlTI2j8I^BZqIGdML#21oT2)m&HYTQ}Eaf9HveQ<9|Be&T`X&-6T|K=>F-ggm22Ei= zwh_d7{!q6c(aO!wkN6?i6p<Xx=M$l~yw)d<a-AM@A^13$uq^Qs$F0)&e>ah{5Hg21 zTWdz8H{Mzreh^%9O#u9)thcwnd2AVu2c?-*?GECYHRaqT>h477>8>2GVOfdsd23f4 z2^o}dWL>z_$V8g09pG9^S1UESv7ww=J7Z4>93qD|Ghu+-2=k{c-s|b%YK#q}VXpXM zd6H`>@YbQSqjghDy5qvnpOLW%3S<)+=^0Va-VL$x#@^(&G%$+*g447r*L1PWE%t%F zK}&dgs%X%ncOo?4C?|a@-neFXZ0(NL+x407tJau)w1i%O=?Y4aGyO=Pxop3u;jMmT zk)(tDnToU!cymSDwmA+j=cBPz7;@CKdvka&l;wCj*cdKTyDzZ{QDdG`mnwv<!B`eD zL7NU|UOOo)h(S`c$KgMzrQm1aXJlSzOP&LAjb^U+TzRemy8%T)ozUY~?-Sv+RxcEO zW_yrXTk*r{4U}$pXb59&<A#}VX){LcSIFU!#zaxZ=ix(?uZ(x1M)GaNiypj9uc;sr zDSldM0}fv#OH9$6YD6{HE4yP*Gz@J&D%fI>AJLj`ZzI0*P`iHVa}_D2{Ex6c%WwuM z!Sv+s9qaDlS8v4A)s_lxmSU!v=NW4T?wNg({}cjWVoa65r^EL88x-VgLnT(t;oQpi zJ;~UV_b<$UN3h@=Lws0wf>i9Lj>oe>A`s-P+j+6E<}^FROfXmhnnCOOle;O~gNBkT zWtizXUyufFU5g=A-lt#Em)^K>KUOfPxw+YL>~LSiX7PO7Ez2zK%q*;vyo}3c2FMhx z=D~Aa9S-+*2#Vx&vzX|<d@=S5cC2V#0oQS#Zc&?Hma^plI!6hI4PWsh)BVbLUzEJN zhuqiRywu^=lbzEr>=k5Vb|Rcv@dK|b+8wgjOu-|@wvioj+A^HvM$pKM-zD5B_?iuJ zCASP`2Ce};m*VVCJ7+Sr^`@F6DdVamK2eJ6mq^^1GMrx5MnhON9|3oRqaX+O85R_w z8QQnTIDKCtbM9HcbkHMSnI>!Il)TSS;g4*jj)FLr8$h?+CXG_Hfmm#AJ49##KY!oP zuHDQDzY%op4sgr%IpFgL{$EAdE;gsM48L?E=-Ea=XEvyt-0MK1DGSv72C2eE{1)%8 z+i5<5TuBbj0-qbBf4%8oQ1~^n@mv(7a!-FAD2cx;Ji+2uiGIDXG4j{#<Uq+z!(RcP zr`3KH6~6rOE_C{AGpC6!|2ALHwnB2PCKC9W`ul~`s=vznAJNJL-V40dLn9UBS}!h2 z<1so<$+9Q=5~)H<EM2x!r2VgY5uZTrrTZ%+r+|=r(gp(NZ_F1oatE3b#-ev4sJGl@ zE3mipfvT!WG5rGul_xu-e1!rLla(n#Hxu|Xwv?@&e}><MW&kY%ZtN*jmH1V0V%+^@ zf0fYn6cDgZ-Wx!v3xQC70(E0T`Jy1UVkQnz0u@UZQ}4u)`+K+FNmnSKQ--;*ZX+19 zkvwgbg}U9`%=!aDz#o)3$00AgG)->S3ia$6I$?4@{@LiV$O?SRD#%Bg1}GY_&;OHl zzS<Yd&y&Zkd9|>Ai)^6zpSN8`!_O=cY2S?(rO(9l82G0Cc~`qSqnYVCJ*m*lN&W|~ zC$kKf`R83nH@!MnDb)D(4S}R6$nYP$_6M!*BN)5aLuuEUVIY%c`3FV#_JG8-a5nf| zNjIV<38}(_qlEv%9{hl!DXk<561x762dWBbGiTY~R-1^B_6VQ%$?c_1wHURXQ2$7< zq4H_wbO$CV;&jo>31np}z6n92w4u|d{<hqrfQUlvy!fEdSH?9FNZGkdDw&(72dCUp zxOwyD_vGWT;@5JP;Xq;%7&^p34Wwg7#{EH+``<OC^^*8WQTyF4HU9rwWPms1WM)G8 z^G$eDd=D~#L>booIZ66}!-qLj--iG>s82ab{%>yy0yXD>dFGbdRWa1foDP56dPG_- zcE`vlA^%Z@A)Y&Pu#T2Z`eu3^vrt0W50xhqr<&3z_&ZmDPWT@a!MZL(oCh#tVzi;p zR#L+u-=?qUgs=O&@#?sJ`*tPvtJZaP1M(cr-!DCPkgR=4|F2UPP^}w-aR!ir@Qi8t zTqo-f5cL=UEnqAFPVXvlGQtw#kL$GlHqKl<A%A&rqtN)==p{a1lMFISi|K2%vJoU} zMNZP@bBZVD-#fu3H)r&V4n&8U$VoN6d>??=vJd2~teB!iujc|O$(p4OYb%ZS-(&CD z4k~nRTV30#AAq>t;Z{BXzeUTrFO*Lkq=fUcICyBh1nuwdelJ8Y+U{}%eaX+sc{J&{ z8c{!K<`!HBBJ8q7u(+Iy_~H{rOrFw8Y-**1?*)uTgjX0WE-tY>p9~zB)uU4brrxms zn4sSPOUHZUjfXTWw*~zUTJl8P`i%iU)Z;j?TWHeZ+3}6Ra3ExvH8tOlt9_oS1jQMb zd2a7+$oqY|q13*|Ek&&J7%j2R-RWK1ZI6*xYXvWoaMBU<_BK^$uaQ^V%(Zud2Zx6} zXfbgq@|}`ErmVV|scX3ZDLJ*!LEyPyo(t%e`3`t~SmP~+pefM<_agW3i%F?yIUU4< z_6M^~GyJg*GK$E<kts=h!Le4v>OF>uM<vc(BKz2?HFaC>EWkf^BS8YA;;I&(DCMQL zxVSidxKDMl#xecJPu|A&5A%5J*1$eb@&4n#X2$<f{hLVZw&5KB=Q{qp0G5`4&*rx# z${BQvia5CJu#Z%~^I_OA`2@LnW$*R}x4riiWNUd&yMcEL_pXCsWb!W(i~aZe+RZww z=QnebT%h!!P%ujLYd~-I@%c(2#jT72;U}*L*9($_vy{BsrIe@jlcUlOsJgw<d(b7? zmmHkskrYKGrTH3J9f|x*&>>yG!?iTv6tM#2qNba!**H3e14)P`Ak&}=c>Bm{SYt-t zsp@m?plXgU(znqfsbhg9=Th$sSo6_e*X)iLEu4oT%us`e10%)x8Kn=tgpiYBw=~Vx zlOc|GQpV+oYi6}oICFKi6`f$nD-b1&E)_r!yCB_(*)~CG#KITj92qCeuJ6zM?Ghp4 zPTwv9LKxbL0m#iY;aOaWiI1Twr@D-cOm)CQV`J+WbLck5EzR@L*4D17jImeih~`>s zn=lQEDb%y3|K4Nvj!)VFMM1q^rC#l_aEkewa4YA86g9br4PQCpC_lVS>`9x{(?PTo zM$GAKSP48gEWpVjcphBqB@)Q)i`~8J;43E<II<)q|AI~`{`l57^2QD4`b!ii1jO$N z<~i+eQ1I`GEE06nm%@@x5Ry>!_??BtQ*}kf$mqM}t@cZJmWZeb!K#aq%SS=%Z$ndL z`QE8E>0+l^)9xG(wn!7`tg-h^4HRSdcQ!t%P))yU(XSG+!LtLna{FP*@Rf>#7&kkk z`MnFZvbx_f5;URTCR(_}oo!?qXi|oyu3&woYdM@{@wu=ORz>e3H>PiLLVo#y3-8{8 z7CSECg#OQTON5&Wz-;bhp+jofRq<-;A~T~9kQ|K3>FbK9kTeRtCMTFTprjtN51+~s zh=_^d5D*aXwCAcWtTY<$#gA_$0kHiFK0nqd#q13b+WGxl=e<gm;PK$#@#Vnvo$xDz zbFW0Ff%(MjlZj9A&c#~r#qXx^eoGf~i8rD{ir&;-;Q#_3IJ*0xgx4Xs{0Zhm=OGwq z8?sHMS`XZkIXr{ZRjDw%x8dj|vhg%f@;fO2$|A4OWvBoEADc|peTLjl_Y(C`)7v)~ z7b#h9Lj#1S2Fp%f&__oNR@&fR`Xiq~qU1_)($eb3U~BVTQ-BpKWcjW3htHpH`XqW^ z=tP4@sZJ9TM|U}^*><^m8;*l{8ZYgGt#PjB)`;l3=_&IPU1_nlG(jSeU3+?2&*bI7 zV73HV6y(1nV3pTPl0_}Yn;<@WDWQSNLb(S>iWZm8l3++FM#Jr}!hZ3Fam0mcr$=vE zzO0A5A=xlJ(4Pm8*8#gW?t1=jSnrRXp)Pb+NPY?0eBJ5$Bjta6o5{e;ogq^`8xXQi zUIRHl_qFV93K3o}V7U{1L2dx!1E~7flciq(^X>d4;4{0yFP5-ij-hYnybN@NKcEPd zKzsaiil9PDk+$4@p3f1K0djre^cwsR(e+K^Di=+WAia$IT%}6Ku}>h~hCiq=dk|8s zd&bNy4uH>TfA)wo&}X@si9olky?j87DA0aGzuOO>WwZc)1Q^rb^;rN!G5jrT0yx}% zCoB)jJ<|2?(NEPNEs%u+U;r@x!87KoyEq8yqM3`XuWew#{!x(lzlDJS-TUu+E#od` zho(ue?ZamPxG?^~{bww?#v;z=0zKf>AM^kKd;a``Hea^1=BYm4cKHw5>lC!KNJ`z- z$gUFO*vu*Sw{Q`l*8iQV>Yheg*F{76M$IKH!&(2J1b^6L0AL;T2N5=2s0OUM4+iwj z4}j%N;t$Gu-!D@if7wdyd!P58@w&U3MkotLE48UUG6E1Y!M`no)mK1rb9A%`*WZ#l z3{eIgKGDO&m8XZE*gpY851QYy3J^Y;-@3>Ip!oKuGEgpF?$r<%M`cOaE)^y?YkbZt zlEl&q_}u?DDj&Mjiz@^coBxzONCbnl5?jN=0%;-zgp1KC&5gtS>aj&pq(L!_Fa6&* zOzf}^)+ZxXSbo0>ShE0Qs^hu>LmxN;nUkKbXohx-?MlZqy}QJlSzN5@!LwS7>WrVK z)_q`X7R$3d#--khoXP8N1Quibl^yoK9a$+r-p2ZcoN6v{Qvsm0jwYLw^7278J1(pA zkJ$rm_y)u){NI$t#9lCZIqkfSty8I=JP4VShSZ$|EF--OqBHZwQ-E6gMDLPAWa5Y{ z(ImCpHYNnU4!9elBC}mB2%*o1qCZ-YcK-{^{x_uo@NIAmV`YfBG5es966Sj@Z<XH# z;dm0zKL)i&SOY$N+06|vc)p>*K)SMgsc}dWde%1*GCu51oJnp3`SQ=dGEeeD-q_sz zR)27&>84ujEdIk;g?3th(A|@w+}!XTW=SA^*%ot!O7E~0>YrPYQKipRH9Jb#GJxE? zDqy@$Y;HpNO+_5lq6c7=jw51SJ9n_o=Qg)(>M1*G=P#L}iihSZWo^y3$$b*b7w$2W zY@bVMs7a6%Zd%=Zic^$VPFc?jQ9K%ql_0Z*;TkUB4oEG^waqE6I70&@L4c4P4SBgx z<(zsytL>w}>(>y^Wif@+Q>ndOPwEeUtjS+~e$Dm8GfEg^t_!NhL=fh<f|89$A1!qP zwo(<%gkPJN*K8Az*aa}qHO;m*(;chQEz2HBU)#C76&C&K?YS$Lgr8hb3z>aHbsA@9 za+bWE)j<+dO;m+bg)`fZ&%+oiT-rkU=tkSKPG${KVoG0e=Q`w~%t%*!e=fhNVO_et zI?>ul2#0(EBzy_`5i?6Vr7oIF<yLg<NG-te1VDHqExUS^d1iUYS5;Lra+no)Lu!ev zLINeO3Wev=_tC38X?x8Uvh|Qu_i@^n%H7Jh=Oe8YW~3|b!)tddDMGfQNhJmwyk8nq zX)qI_L^tJNGupM*M%RRVZOx~bjF<`iVGhiMXTEs~?Q`~HM0m=X8PdYsoQX>BL+GuH z-VcWW{(TF(Wtysu$p{gmdmkWA%?xNcsyr-m-Y`sc1eFr(2<;8SC@X!u+tHk+|27uf zf6w~P*-uFTSM?r=fz%MG>|Cc3YPVA5{Ibu5cL4Kgm--&ik?ql-)Z5G%Hq3;P*$&Aw zAmQl5m~(>i5A7j-crF<81)))7PS_6Jg&ex}hbbszPGP2CR0)&b{rdpR^SocT{RXfv z?@)&z5mQf%2pJh!Ee!%jGk2ea+mBVsJ$tI-`Web*F0Y}k+VW*r=yP7)8c$HCz9n^L zgbNrlxrIJ?o;W5b<7&>`uVg%i9$VtNHR)QWX5Y>bxz=yG#3}AE{ZXQQBPT>N86D!* z)lfS<QUcriuqDwW)Y%}6-vw?EdIfiSBS~OAa0!-c*za!GsF-JL9Lh#qGv^W<c;ib~ z=Ju3%SKcFj&>)IOJT_0Wz-8A^1;W6Sokh$p;EP4)1R{#+Sj?U9Z+r5EZG0}QU%P=v z<lhNWdga656@`Su;8i2E6=o5`IMqAcFZy>@M8qt-g;HAYohg448Wx*60Xh%J<!1-x z3Ell38XNiYB*A>)$y=`v+#()E-E14(1y#<Dq7HWXgPc!NiFS(RAvfKJ4+V&Mpc(#j z3(H<Z=>$J5q6HSn;UF!BD52HQDx%foJ>qH$3LHH~#cvl)<>zX-Re!g<2T1*f#%q;= zD+9Nu4?>n|`XRQT;4&KRl9h*zOkSKgJITVhx}$l<0mlGWd?sw<)`~55aM`c6bi=)e zt=irk8|G6K_T%eDTUnw^5s)bm4g3Tf|7eeq)5h1R*i0qJZ`WzD0VDTowz*W&Rc4`^ zH<6O=t!-^6$x1{gVLZTs&dIqzlRe0nlRN0J(keU_IZr+|K71KGR`OP|+=Hayo5~vV zzPne1jirurN7|byvVl`e?C+eR*G=7kjY%~l`PGMMaz+<083zYsdW=btNF>tK$`nV7 zS59GVl9G3FUKwap4bOJk4WFVe+)=<DJgd7^TI{r&bq*!$icb%D8HSM*0=@-h$>0on zWy!uL^A_OB>A&zJQNjn-aAI=6_X8yRbC{~K3sTHe))z^!ZOmxOd^Ct8Kkcxjm%5e> znk-ATA!Y<rEcQy)_J<n|CB#fVWXB>K(<R7mB2`LGe{uLFGySK8M-F|4Jjlsl<drK$ zHj4l48vFk{l9htOl|*g?B~JWvxtQ{e;8Uz<km2q472F?X8#3yTks7K(X&~3ve*lgC z5Pafe^%yFC`~hnHBW~99!foqK0<s9CM-yeGjz#HL1tV1$|2bPAODtro<gOCaBNA+b zLe!AuA4Obj{MqPvi!P34&U=3gY5taMh>hQ2SV;%DzW?WqMWQv(#$478x(zZYG%0TR zqm-8Q*9w4ji3nw=nOcTEGhX}hr;k|xh!+#1?zEU?xadD85Lcanj_}WuD_t)xHFBB; z&eHrz+lMl<M1+b4ecoJNd+~?eCvTug#rp{|F;9+vpz`*L$RFkTt~wVQS8*;dMojxU z)EX~QhVcAhFTBq%N`_(Uy$@IiBwo2tx+^FurkA48L$@as{<)9=bPO{gMfHd?&73wi zHsOQiNxwD_*11H|C)Pb)%9-vJg93T*ekG?5^=Xsjg3saqIKc^2{25wR`N@0w{*>4y z+6vI;LI68s&51xA*oW?b$>o~e%|R_q4+1_8OQZbdk?0%LAt4hd-r9dZ&Mrd(Lh!y* zfUZuq98&r}GcXqxkn^thf-+z#G2IPj6)vJ;$MX5e`R+_gfLAC3dv5`|?WK&;YGeE= zR*27rN!I`-#H&3+v^fLZYh?hB2>xV^_~20f14s@Tq_Xu+|5k@LbQmTEXUze;SOP3~ zz)D1B%n%(N-STm*p=E5@o-mu#+!RR~6x$4JaLq&)3LfsSF&Pc&G35S_oc0w8z$JKb zCWZgj&iz7=5z4Szq=cJK`k_c`M?-_Mw2uVNG@@URVZ!RZ`HtlMSPtH6YeW%pT&l|@ z@0SRChdDiYD>EdV7hiSf{N9CmTI;E$<n5qOAzhO9*WLhVf&x_xBd|8utQ`O=bWRA6 z!LM`-Jb-BP(k;9MU*mHaNjV@rp96ZuXH?6nMn=7V-weHW3vxd)|Ggo<4`6$lh`5Y& z+0yrM_PziN+ZVghzm&NDNXtJ5&MMexYZK7bHN3vFL!Bk@J{s7aG8CD}3GA*?RaMQk zuvm=_81>00EUfkM`39Bzz$Ya`aD0ibapj3J@bU2}0_4@k5>$a^p?8GSy?fbm*SXdu zhpTtZIr)b*OH)IN0$oE7=S$tM;iV6kYrrDkS=k(SsMyXBrOxUF##BVGxf75hp}vN@ z3lN<|rw|T@D~fmhb`cuW5Nljjx1Co8nQP5xgFNi<;o=(93E0uJ?&IK40D{)4p)ouv zIxg?>v0rqErz(6JuLc=cKLEVL^={?7caig}q=wu?Vfi-;{;m$S7>@fL0NEVS85GJL ziTHjUinM>-8ZQG2y(WE~hbIecSof_L(qeJ^>Zj}Uys%8gb3mI-ZSh=9*5iJ0yo9r~ zPBvNe%nK1FJBTXLj#W9GC}##hyx}WW7JI!7)&@Ai1<C`WOzYvRm|Qy(d_1mK`}mHf z&KFT3V4n|qWsuKmtD|WVwUkc{*mHmbC1zDLHCI^9jh}QmA5n|l3YMZQC%RCs<b;c6 z=m0V{HGSd*A%M9^-?~2)88JS-tUi8`bea{=+({}-SLVO`{PiWmIrzQ#F`J1cPKnlH zI*btjn_p@p{q^kO6x-)Cx@#7huRHsOPMe!dMU$z@r+_R&GF?~54!&z9h=Qf|hr&E_ zj8=1$5Q#58Kn9gvRC9@Jqj-v!bcg-``%^u384AgEb!(IA94>M=qJIhU?p!Ia@9<&R zAR8D4GARdp*MTX~d~Dn>HW}H-*vZD~4Gkcpu+VV<TF0d98^Xc<$&*x&DY7-$lKur| z>XM0u;C-dZxzRJ-nH4k0Bc@tGT@owu(vOJ%r*a+F#P|U9>n4c><I98~YGPvIUVSOy zb}c~u(6u2K^}&@G4(I)IwJhfdW<ZE&g6XW+)@SIL(eecGtUJDD8t-~*Umo6!zsa4+ zuG4%~O%ppQcBqx$#;;YkI^}0fh8LhnKfZMVNilBEM*mKs?d-a!@6dlsjA0C*(|XJ# zm!YMuZMWZr;CR{u1n^jSG^U{`EtB8Et<{6z2k*j3P@RnwBA0UB>bJB8yPq!=8Xn3{ z4ClX{w-PzQ&-Jl0)9u(ISi>LQ8GHOvl*#wzItjbUflO`)y>Hd@H;gc;(t)M*j)5&t zo|NWiBmS9L+Em#UO*=U(Ex8&)#or6*=gUF$IIH$)Ea7shLUVHq@%RV}3rn1fe%Fy1 zl@HJyNp~3b&mNowG6=M;Gjc#y8~CpfXf(mM&!7)HGqps5dNQ(5pWlGIrPiR+lz+Q8 zi;g2HUItRo-0>i<qpM{6aYyVtn&|NOi>Rek2?CyUvpx<I1u6V(1;F%=C;#6mxv=-3 zZu?XJWkwAb47w=<i^`vTU}IghW~vO$_yft1M9a%9&VoKu#oR}6_fMDkKZH2I0>S#X zA^cl4oh)%PMx^OR7RdG8pCC)9Ua>`VhMJy_`5#wTEuNC6N=!4J?@xG8*9*Pw=jsgZ z03WIU$Fw5tG{ETO{yE*Fmf{Iv<gNml#LL82#_BLE{C(Q!FbYDa|7nz(%~Gec9sDv8 zNRBEL)?VJLZBF9{H*<EpPal6YmjZfk|HKF|(hyoiG6DM-)87_}*UUl(0Q22mHHx=a zFRly>&um8PFO>KDm}E5cwR_k=+Tr;@y?12FOn3OLlzL{K79EhCkuLKAQl&n5D_leg z>I*FzRkmZu+SKKhu6WfbK4WxxEXCYBLkZg_(KBY?BM>(IwYWX|IQXm7Tfp`ydogXx zJ3BGAJsO_5(B0iNBG(Ss*?w2}e`N9LID%UJ1_{hRl7~P>t^-FF_U$1t)Vu?Yvx)OW z(MMt~UtGFq6%YbRR;uV$Mlh9))IE}+_q_TfFDz**opYsyu~s}eXvpnD!*)#><6as| zCf)VJ=O$_5ZXJ0ASsgLK&Gka%F*Mufrk4l6<?W$-;lBAsZoat|u^H+{R(89O0I~aD z;})#-9}YTT5mpKSoiQjf314i9)*W(P;7w1PNsH=_-3CJ8lDm)=HI&HjT@cipV@>0_ zS|b>dUY@0LQ^V(tP*`t{2>gt+fdo*ZCY}EKwgeRbGK8pW&geS27TRy{&V6pVtouN3 z{-F$4r=*~GG!WceohN60*+r)wL)^Rox%1+D9Y7qL+SnYmmH!U`!(jfyO$Ut7z_;NL z{9zgxeTnsX?YJp5u%3$FT6Z6*<SP?;g55XF7TEsfT$#(uD)AUyUPK8-aFq*b0;QoD znF7qJwO#I9VtgAp@?5t#etx|OSYWuG`7be#m5OgJ-@@EZIS@@tPZymeC`<is%prv$ z0&+xxoh?gUxD){)I!M1U8!!1A?>ksKjsq8gWGSH2unge>`x*bY=lm^|{ZpUg8A~0f zS66M>|GCeLXvi%4CY*~z>f)cSMfL!O+`G$&vdh2CDF02r|KA*;j(Siik6qe83xK06 z#_E3(t=%?GPPBf5a(^J?02b4`<9>KHS#Bfqsf_$;<8ILsZlX=Qzi)H^f98MX$kox1 ztR%q4{L4wYm=3D{w4x^inCRiJgnyQKtghd(YEHYbmjakq)SmX6+SuDi0B8LGM@KyX zC`wJy3h7zMmcRXX9uLrlHINiw^R+W);~la#uRHijy*9j8dCwsjv24fe>DAqEYGnz# z(H080jHz(LR{#sK@4*A|L6X3xAzphr&$hZmncKIknGVm~TG+~!CzR6~My?HgV=65p z45YyM=a-AyOODU`UP7N*S&mdW8;`uf$D;{&gm`!*I}Ve*F>#Q=H@A{a9GuoXOVKFR zyxgu2FaTMdFXgaRKLnKi+^*r_r%R9hhW9=Lo0`-T_OY>vMI+5akq8&@-iE=G!v#hJ zN_be@dWr~MYf-{yR+f4;sJ+ZuOOEOz6IY~IPGcyI@u6Niet;`&qm>iBUVHHTDP4Q_ zhRvDRrmdnu+Y@n+)$urpBk^Bn1ccYk)zYiiuxYqTTf*^uIFMsPqbYf-(FeH{f*j>u zB}BD-8|D+&5Y`(We#II${@cx=z>GC_Wn1DB?qqwVo&nG;+sEBA=FI65#5~ALUl&;# z7;7$F;*D`VW~}gZKa&-4h&#*DEK;&@ifP?z*eIyG&mesYnFIL!Shml}3n^ADKJ-Hq z*ne4(x6d`u;{X!TNPQL9MJplx(0kStNPB6Tc`ipyMs$;dNu)nOb`v7)o_GB60LP}j zUR{{buA>+p%vOJLCLA#J5PHM9(17Deu(8K9u)jmpg7Y-+ldF`%{k)%t><fuan+vC{ ztx{;5PlG96lE*n88t-|0`kwJ4d#QoMcez8#aZmTft4~BP6})Yz%mo6R?{(rF2X#{q z8bV-S&{xUXiLg;Do`-Fcg3-m=`>*b|BLKdIsTmIjvr&Xzwl_C%WW&`DC<!XG*N$Va zoB=i(02>Y{I0S*1pQb%H`Spw8zqeMrME~9x^6mkr7Q>FafRt7r4UFN188zJt%+8?^ zvn*^6PPTlcgWx3hDfwoUojAq2I`x8}U4DocmUZgr!LQ#wu(guC-dHpz7rg9VK;k@n zSo6Yzx-Q_f1KZef`80(jJ*{hlT|Kq@_t=V0bD5lbEFEDUG=40pJE=>L){Ge{<zJ~g z58FHE@@|$&C(PU7luqsd=XmWd3l%!OVkvcho<iMmIv4K?Teh2@LIIDEBk43%FxnlJ zcf>ur;Bc%L)_11EuPRA#_dvmK)vntf_vQ<D-K}&dNgB7x4BxQ=w(;2QR~)!G=PR=A z4J9eENs~+Cm&u=Nz>2gZ*$ocQ?{tkn5AvuIrOjW96Fpu9gKw|)cyj4l=yWQ}r3gK+ z+lJ%9JAIr9v!74Zy>U{EOg_4D_Gl9`1_RNCzMB{Q=F;9*%UJ5tF?Ud$sOZ49I3>4> zT0a>en{w}vAX6Cabc1%XErY%C<W@GM2DMpG!gmk~?R#=kwL5*1#US5`C4@cZgVFx* z)JlQGQXVYufXWnR?e(1b%`TVby7BYlPqc4BF*RXD)(zvur{w)Yi+pJhxbJ{d57O3( zZ%d&^_6I}3Y<Z-v)Ca|F9P^BFi<K|NpLdn4_#D^qzw)T^NuhAO!5SYqSmzYd_1yQ1 zaY8b|blp;C{t|q}=UZ*RXxaFcOUI7{yumQP;Vb+2#Hk%w*2?7;Pm&5?LBBgibQ|^; z;MjqKI6RopdWQPfsThT#hbs@hoLuk`HTMlt(RKX{2W_eiqKx{Li(-LYZ13<jA?xDs zAp#B2nCQTDrW7;5b<d(FMgVVD^!=Ni5$b6YF?kx2QgiYmKiH5fFRfqgsXp4lK(mvs z16IzSH&7TkIXStyTa=b|7dSD2y{}Jvbd3#>%r7ORlc$Nej&=+}u~g+BrrC#$au6-D z-}b3$JYKvclQyk>rYhkO8oo8x;VI8H><4U{b{+r9kzjDg@?mtoaFJ8A&48v6S`FB4 z(I)LZN`~#1(PaB3$GMuFf@dt<iYwU%cJ#S!9qxE^wl$8}VO{o{AG*%A4y_))8!#}| zHN!q@>0bf-i}bd0NY|lSE-eX+eRO@j*^a;HW)6@x1`c?09Up&UBT2HB`u(0mJHI}p z3EX4ggLYGIv8-Av5GG*5xiHJe&kwfaRy{(s2Q7D1+dRnjZcf3U$_EE&-;zcR)*@|p z452po76+_(W?^BW&!?~E#T|}AwlN9t;))80u5Qn=ixuEH!{>3EynwS2v`QE1NVBx3 z_TJQ10-H;fig4D1JL?GZ(cBT)t=b=|5X;OO%sIrpld&&0C8H@mG!j?G<?J0DV=Q{~ zt&2V-x#XKTx@u{uqRbW7^{0w+Z5A$~t`Bb$K!iQAp0NccrmO<VIE4F;)w+0gE8ok~ zc+cq<%#`LS#&u89S32ULJ_TnfSdD;1djr@ZZ`k7(#v=bDjBdVTNy7e%Lk%zA&}vIO zgnYON-9Mf9#fZ3<Fi~-8QW={PP6C_~M(`JtkWe%9&XQIxLdh4`rTRUFjh5R4?R#|| z7u3*l@?OJ<IHBD9oA|FkO<3s=#-bC<U3XXr?6nfC80u0_;3Sn5wt0hjA*n3$lzFfG zed+xQOFDE|5m|#zd^xY`=g*1`@9OvWHX=Q6RJK{k<8pU!9is3nuXztV>kE~JtQ)#U zszp7_GD#b(uYE^<1b?s#HIWK@spfWg7YFuS^(``II3_QmJ32*h2xq#qBHXo{Co}1( z1{^;58ATRaX3a0lyO=z_XJl+_<~qj1C^!PX&P97oT_OQvmapE?Sc)MFo&v;7ZVACL z{FBluhIz(^PwR&1%JjNLkQq(EmZ#t%JmQMoJteDq9yP<mhC>x@WzwM^4cmi*tUG{% z9+t0E`2M^(WDDe*uf<~|OJTqXeqQpIruW`Oq5~V-$DHs4RMz#782>jk*jk$P1HrgT zdbo$bbWBp)|HIr{M#U9uZG*vG0tqewf&~xm79hA2TtfpPxVuZBafjgU!JURc(8euz zf;*vcpA)%v*3A3PTHnw4MN>5ARP9|=TWZ(yj5Xb(4u7G{p@0q!*SSOxbhpE=qeNGy zhUM6OeBHpd*DNf{Sk&1p%-fyn`h8krfH0=Nb#n6cDaE%8-`a)Q@g3##dm`hbII|1E zdIMF}n&oW!3IouJMZKQ5xcKD*p^&SLejrdtY}|j6FB7xaPn%{YAGOQ7(9d_q4@NjC z@}uBt5C`trm5Sq}3HWv_L)dvG6ye6h#+6lz^P;<It^tHbSD|iwX0KJ>TM{mDFJnJt zS;eMf&gj9C=i?xvQ+6dYG90nPAWu~OywP57lKrVk&VxrX;ZRQ+jY!LY`;GcEw_bW< z1wQL-&m9PpR#kEoDeth3lMu!YZC_|Y(NUK+xRs=VqAx)sf$f7e^Vo=RrGODugF{dy z|4BP={9Y6_ws7N6NQ@spvXsBw872~xj^?x|JjmF%n%4;=Z=4vW>D}8wMtx+c$FzM< zz<w@8F~*b<hoLoJy2u0r*O?ZGZplAf5jkOs_~njYL?X}`L0f>0kke@~Xt}>9I|GP0 zXlV;;-(SD-xV+C*v!%Oug7lK)x`7vE{+vC6F9#9fVlaUp)~6Q)To9vmC0@yKDZ<Ye zd$`+=1DR^N7`2?GeEmwDCSqMBbWu#GFtPGJ<tIM`xXz79Dk7mf@*d$(Q=K%u!c=C7 zfRui$^RDP+dgj?sQu%9(2Wmak&1}Ir1H=!3LQTMZ%VSL94><J$XvqW~!Z#VK%jxbb zUk<^eV2Sr_I$;5=y_m36RA$}UB?^1suH6s-pfK`!c@&HtaVK)UW<75gA=^GTaQh>d z2Dn(>y=C0?4SSLZ_XHn?`b}BMw(4100CBb1aVNsXixlgK$XXc8Fl0|G2w`7hO3CkT z%IIU8Nm#b9TR>*cT1Q_1R)Q)SCMi_h9yrAJQu)MjJc7>2m&f+wNm!6!#V)1l)|2pl zs|$-~X=z@UgQ*S)2OH0P+~$X7T))D#Xd0-SMK(kh4ZK-hHp8CXA9;~ri(U%gRb{~4 z<=vA8Q7*JB)z@7F<J=B2o{Eg6*NI6-6Wc9R_7PwM2o;PnF_gR^x~0#>QZ@3Ax3)4H z@7;C?X%tM1=m1Et_am*b3?!mfWh^2Mg=Av6Pf1DHs@wU$W17x4s$7{~Sqi$F#Ruq5 zSd4Fl6n)nNPzl=E@DF`a10I%p85`f$Ab|OmK2Y+$5R4%cWKAd@O>Ds+m~WpPK}5n= zes)hdQh;zvjBH7E8BHb#nlmwa!x8A(pA%!}E2kvO+20Y;sBCo<W|`0dqJHgq_fn_T zyO1SX<?4V3KdODI_;7<b#8FYP(&qS?IyCo*qT+CDL(yY?iSkmyN7^F*q*G9KcmH5G z0MSA{hb@{&SSdN(j3~OtT45qKG2V0$LuyfTDqGcZ?JB@kDtq2;!Q<zexg)fluyi*~ zE4erg?EsyVL&yR4{_?#+BtE^;;L9mqc4Bc`hR<<50YNB-h$N;+w=ZtakT`8^IejSm z?Z^@wmU<8|&3JE~`G)<NJes=S+)kF#QusW`n=m!=+NF<K(Dw(C%ivj*Y(nusqSe%4 z4FSKYTHt5g*tbT$=TVu>#*pCPzL(@SF%Jg5C-J&RM1n6uko~SoONU(-Immo)6ToAT zt5+{2M$A;&&6vBb+l`~m*Xb2&xK%%;8PgY=4_0_YBCQgIq464KYjx%fDoAq>w2PiO zP2pTFc;X)dxK_k$BG=Fe=4k<<ItS=C2kO~rnf{RibV?QRmldn^Qh+G|^4QSX!wc=o znV1Qb7oS3XIrQnqQkiH$ci)`Id2iQlDRB~Z0qlcynfy|2_K8qbJ^YUDh)TRV<u$%2 zv!0u%j042B<^l~!lp<@14}|zU)2cYL5yx(!cBGMpuC}{|D1o`2^MmGezva|9NG)W} ze;u&^W6!7Eh!9$!z+_SH5S{jHEURf?77LecfKuWqCBqMQaA%Ym#g<emgDH6*5m}R} znIwa8AjY+-8eSPv0}P3;j^cO|h>mSW{0HfyONo?SB`BT~E<X74ll$g%emCyhtCor8 z>|evY73b-*n$D7kHn%Ww3|>4}WUtDLzB?<As|DSI^zxN6IL5$6)`eYA*)M*%nG=0P z{WZrvuuc2Z?@?NTww(8fK5Ni<mCQ1ocTYZ2t`mOf^N38A&Egj^Tlyx1bo=ZjbDssY z=>;=8h3RMMl^x0}b^^crOpcfcp8NHUQhn}E;SZsoO%+I^c+Q1a)CX)g0v+-O4kULl zh}<|re>5jq_F+wv<7THe<vetS&0Ue3@fl{!)?oek?gi6`R!9|6++xu;y(FJDBkCWQ zQH}W-MDMK<xIQ$0=hWEu2CK$*oj~u_Jnb4YnC1=R1I~FTh$Z-Qa_arZjqhk$YMUAN z*vA~vEqwe49t%7?4Y<)hB|ja!yHNJqM!W#@y{-#$=)KCm+0yLM@I9^R`={|;VjSKZ zZnGpoG=MEr_K*SJj1nAa$0*lwl$gPX%bx42Yj@3l-7!(=FFtoy?U9LY3fj8gm^6yz z#BOKn6d7nlb9H|9cED^U@7u+bNLe&=E@yADU!c)6`rpWPC|J(vz(g9{u&RUhekf0R zpDsu#M&ZsDM?*I=a9z)D2~FoT5+S-NZOG+ou3QBGeA_2hrYuHkYT>0`A@`dzy_+AT z5`ca7z^E*k_-KLo{d)^g*%{XZ`i1onD&F@b0F46K7;`)|HK)1j#izBKFCI~Re7$|g z@o^ydM)!F+uP2i$a1JD|R>lU^ix=w}$KAqwIMqNS2F8&t&jtz6F%I7_u99uo<M!9? z?R`jCN59SbAr`ItXwQR2XHqznePgot9dcrY!t5T&2&PxavZV_Eusc9>gqtZ9Um|qL zS++-fdHg2u+G2iAi9e&}Lon@*FMREC!>nUP7r^jPZ~pX#{DSjscMC@HcM14}ji>pL z1gWAJ8c~J4T4+~*PtC+Zaca1jmT_Q+@?CUBH+esPI4kC7)=^dohv3F6OXROWp5?pv z#&wEek^_FRHd<H!Y5(ibjGyK-EYK~yKe!&<d2l{?KCmb$^6)tCHFjJQ&$WDM*J3o< z)HZ9dUB!L5!f#PjTx`)=K2o1{RK_~B*;d}Fkiu5RfPUQZe1+5l5fv>UHYGMVxIMW2 zQe;yt>#`4jFgacWo^b%ztLN0LgGu#>E3U>&M(8AXzSeij)gb0a2VJ$O|5#YKOeW^j zrBAKe+@0mKHfq}s1$gqCt%l<H?}{F9x<lh$fW7mv*)*aA6)aFcCQ=E7#Ze^GMaF`r zYjAh7=2^&f**~tfyelTpR7a(8)_755GM<MY{%F?sN&p#GuH?~pEAN4mk;|VxJ1l8@ za9Uaztb<BRBwmsb#}l6a{mR|u*;>OD7W?*#gSfGslC^txa_J9LsR}W=)Z|EIh9SSC z=y3vVmitC(kGo84-sgfrWscE0be2XHuqaitL<DDej$gGNi>#j_mB(!&23u`8gskgO zvvDMmT*yH5V$XUuf(L{O7SN$XVx(}AXy<SXB$g$0+iU}Lnr;AgvDH)bQ|xn|QRO!q zQU=KiE=uE_UvXF4&8qb~TqT>vt~zPTs=5>$!hPyvQmF!?Q5Kwt5G<!mUgy`57Oym! zo_j8<_J2eeg954CC@#qZNt&IshBERl%t!;db+K*Zm*hpyVdONuCA>W4jo4Pw4{LY9 zGH>>kwB_Z?%i8b@)P(Im{DLcbve)#bgowXDU?%F7D2qDoZlD8_ByRo`JBteTtjz+Q zp4-8Ne!?C}FF5K@E{KB9c_TnaxJZm5zsf%j`7r1d6CXMAXkbTv0jO1!5d@zZ7g3#& zEZ}dPYHIT4oQ4QVrv=!C9<ggitFDYo=PpaoW8Yi_-i*$aE%%6#3~^KyyK8E3=-~-q z*Ht=DsKn%+k`xK}>NOs#rQfFdVl5CQY!YQh)nY_Zlnv1=9DL2AHh|b_+7K%cr#jG< zjgAaM@)w0M9oJhWZ;T_pLZxm4U;!712uyU{*ODG@!ia59hMrX2YFhK|IR-Ss*R4z> zuP+3Jc0qM^Xq;PPN-u~JEZHe0=s7hhnCrE)Jn3m!45--*XFbqpZ1<yu<rafqVL89s z_o(k-5G+ZU*b?u__X`YQOiajIQ>)B2(=-SWmsGD4R;6Pfe!5Lkhi+nM?2pnXw1*(f zsg`Mn2{R=ihf#uJxrd22pO_;ZV%=2F;U}DNTh)@$#5f%83TfS79U>~^Y%<O52!(>W zTte&3VllVKhPWruSa8fxL6{&~ouG_nipsQkI)4#nR+cFj_M17RkZ-NN%bnKQ0^5u? z0<To8q<r)IqFx8Mt~-AG%!9*m73RCItC^f{#=3hmp>bQI{n3nI_iF`f;;~l6gapbO zsx<Da^`vrM%b(H2?9JcpC4PWSZ<71FMXSmCaA_sR;d(8|g&ZY#d6eUm#kwu>bKS4c zFVA<d5WgM?qnv>Wnj_!hlOUt=Kl22K=X7c3@4I8?pA*f9OGSlF=#GN$3P|N;R8l7< zR@xqOI$v;<g<;^q6Ad@oi@Xq<aPLZ6Zg&ZD6`YvpIqs8c@?uM_av>S9@OKPF=ecyv zqBv>u6_&-|WcCZx3HFwh3P(rC6w`3Gd^TsZl*h&w{@z*6UOaE2&)Iz{^J$_9*ErE~ z5URW;mOX%bMWGlb86D@!<&P_(TK4+Qw*V#0a5+lplGrir&VbqmnaupXJ>B)F7o#-E z(s^KsY4P`_^8{cfFnYe(!uWW}REGgKICV6_F))n{_dBkwyC}&UXY?1Hirbu%;L%Ny znebdyf0J;hsOYKa<y+@sk|^#fDQIjr>aWh&pjL4HQ;COs>NmtO>Y2-h&B$<*QPz9O zdvEf6rTWpRu|Ou#ueO`}RNP+Jl$;uO<@SUilbxgPVWl>^b6K@&zvy6pqRMJQkXsXq z7FK$sEgc!hgvvXO!BG5`Q@~%_6{xCB<B7Iv+Hp4{@oR5Iw(m$sG_U^Btj*od;faX1 zD<gxpE8wx!USBSNVAXg2iECXbCs#%ot|oY{XG5&tzPN0WBg2`oN*t8}{5fApawnf9 zGW?8ru$vocBA~tJxCg44s$)HCH%ZPkoG5u4BqS{y{+NMi)mNdBXp*Q0R>*Tg&n3tc z$lEMi?y{Sh%NY_4M~5mI)OcogYMH#ss|pAg2q=s!6MntyfR$)rLjdQEqF_dJ&K@~# zB}I<44!=!0M<GC-l}6mNVA)%XlDNq{(;XNM=ve8=5Pb5_ypI{r@&+;vy>LF%p`H_g zM%LO*WeNA-h{zXmLeMIQ1O?WwgjM%0#l+0|5TwI65)fP{s}QC@EPGY!mcx#=`xkJW z60vm9w{LvHxV#qPsg?tzwz-8~>ADQ%J8vT%QoxAjo&ZZRZm^J~Ihoa<gDy-tD~Tl2 zD6Q2gtd*dS%$Oh-0m%#r!G{53Ty@;fvOKr^gw%M{*dD31=e;M~9`go07yBL}v{$Un z?MR|TiFZ_ZND9b~*;=b8#jP?76Opy&?R^VH9s~a@*^zIp;?jwR<APs!k|;|Qq%9C8 zZfr@7d66s9mt(BW2m0WiY?W^oaYJ|wo)_LJ?>dlL{^*H|l~a+2`NA(&1R5M*%jdb4 zm?a*4gJ|=aLQLzLv{0a!J<^`_SGXsl4<crXPPwo`;B%~!M-C^XL%4<41^5g2i=5hX z&liLXh|Q5au{^OeKkis364oDH*%Wm*q~zaFs&c$5P@6&8G5c{NVesS>-70X)zewh~ zmu9GgLPgyLO#!<;tz^phLqTE2oWl;pNNo<Mlhg)zl+78|-P7-)^UWoUYkfhYwKl0p zhIoz%LnAAXIajg&1<nxv4PQ|JYZt$iXccf{Wk2NYjBht{Fom=aSll>u`|V@Pxcz*+ zP4TFDBJG*HO0t7(EXbNapHW37M>M9=-WX(}^9tEtmB+y*cU003=^Dcz;9(~Dux-Zf ztbSceincW~SXw$GeDysFu#v%~iYMy?W9m10hY$!ut`3J=^Q7VWz9FKV1t~*jwuaJO zNH-#@J7-4u1ax+Yez-7cRVJvp)QDK=!Xsf{W*hn$@ebtvd|GQZM<JYd`LjQJch!G2 z_$)`ITDMM#ZQ>fXxLFJ80kMQmr1q%xhtC`d-#S|?Ij6(*!b^|5@Mp(Bu|8gP-xn5E z3yL$+fG_#+g5CH-eVL4cl}t}mTw~duuE7_MwdF-SSC401+j^j<WVaBXUr8&)2&<3x zIOT})9V$HIKe}qZF~g&ZrZQ`LVbnw*qi&<oy?o_3l_>&st%Fh*V9^o0VhKlLZt3Ml z%16q3_M8$b%{2tzMo(kmkjSRGv5OkBQ$Z9-*`9<Zn-lzObLd!4U-~j%w*<FIK#mQ@ zvnZX*gX@2!0-J2*QNL;FMh&(&Myb2TY#*W47<YdoK$8@E>HIm&#XQWhz{u-IPF919 z4)>CFW5jC24I;U<dcNSp7bx{~Iqo2?jU~9!C*aHIQk}_dyH4;veqFvwC)F9ps{n!C zGj3FNQ`fB}k4%W-tH;=sD~L-K)dsT@22hzLa?+;qU;-5AzgZ6p#g|XKJKQHwh;;*W zXH9vC2z{5hZ>Lq6@?qwSnGWG3=F@;U6KGiwRoD%V3mWL{4-zS2nGO?AE$RV?9RU?N zLopvj+qTB~*tfK@dM@hqWPs8OU&5ig^aj$6JUqe@-tlf}a#Ry^r;m(pGMjV|5irhK z7O=w=!KQg&nvp9Oc2d=7GghOSH4;wr@%gVUpOoKDBfzbAi3rIb7!$y4q?r?hl#k*= zuVN*`Hp7z2T^7SVNnBF0&P(@riJFI#9~r!69!*5Nl0hk+U{cLdj=Gumq<j_xrAny5 zH=PJ!X9oi_WuMoca#B{%el5ZI<>mSc>+JAS^uxMT(|l$Y52?JVacVkG3QHa8947K0 zdpve!{aycz<KBTsCWWQI$M+cIfwj@60ZRN}mOJcJgdO%?!;T^>+5~CjGk<e}=Os$W zXS5e>+7OR&?@Aa4(_(@JA7RO=y`pI!{hlean^ar$D1kF`J&$kQZa3caARuKx2?(u( zw|2Q$(!^3;6S;eA(3OC}pBzULw2X{W(WR#e3EOE;5i$egyr)`A+VfzYIvT~NxE`U- zW{voxf<ss^;<``l`B2WfgLIzb*nUfyh3VsN7Zmj#XNsV1Dk3q8I@CkN1*=3jXYC1| z{1LGhu@$VZ%xX3$Rq8ejD+*fY4q>xY>|!T|Q#o0RZFU4E*|POB43y2Cv35INV()%< zN(h>!NzK~z99R=6?7WwZOh>*BO~PQ0EVJI*+7F6oTVVEo8gzQVv99kq%aV+2`U`K4 zbO>F6zdk873DM^L66QM_cIS|WeiE4C@p<`V=xK~KwhihYFDAP&E;=%WFN$vvPr`Id zHzs8G-J0`CM&Ij5_9r3j66qY4D4EHXL$CJG3yAxQMXIW}Q)StmILw%~96!$vkI!hk z7`I{ttVj_D>jZmtm$BJ*#?cKke0_se@O&FC6c2gq3lh>O+>j`Ya@fv9i7lzm{aGVt zq{uZ)Fu(zxD&VOT*8=2Wec!WyFa&W5Gj86a3?Uo?Yq+)Z)})Y?q2xVK@mR?FtH&|A zkYS*N%=dm#bgvIPgH5l_UgQ?0XCRx85F-8b>$|Z@m8gCg<=jv*d7Fz!9(8EB*VPW1 z@aaH(-N&bnzrr`Rwo0APKRgXBl(B2Se`jDk5&j}u_1ZKPyC_J4%Vwd~Ao5y$Tf)3Q z3&FFMr|6bNN(Rv!9iTlaolMn@Z11z2YQ@UX3<3*dtJTj3PH+(E(ko|A?Hf#J=d1N6 z$}7zG!x)_lUg#<WVxX+ztRv>5r}`ksP}J1`c8~^D0>PA+hCcNq*gG@>+=uv_??ShG z!29gm@PfYQq_s7N@jQC+>W^D?B%)h2s$*kI6M{ZsO{J5PYKVc?IbO@X+>WG>_<J%w zZ;fKoHKXl16DkyTl3JfjUtxTVVH0D+97*6vh>GRc7dPxzrho&e!frs+M{uzy-jD9p zmXlxmOg*)uo68ziE)l1Ic+>2*O4zf&V2-@(b60B5GgBfkk&d`WAP2Td)P0aWMRM7S ze+zgdi^`W{FKn}dZ*D-;`HG5qyxBlmF3ITOgsp6<*1$~yKp<&AmEspCmPs>=h!Rn~ z_yWbX?h@3vh2z8}$1S*>o_Esq?=9n^sje}3Ts*#4Od&?am=5|L_JpRUndd}Mbu3kh z)1NW=9?o9Tgn)YT$vUZe;<=$*nQ(>HIm(_m>bO&`Nv-K7$ucq!sLp*|6383E1f3h} zR9LaMqMu-&AgE)l##rN;KchKK17a>35KUft&QTv!^xzXRTvRkqa`N%__4&n-`Vx$1 zZK+btTlOLoR$sdi&JY(OureA;qNS4!C!cg6=EH{~2^4Cx2GM>^6%WMv$k7}~E3@<7 zYZdwxaUf%4(RF7Ojf;G5p|{{=<wOqYINBQ-#BB=scP12O$$ZIKKi$MW=8FXD1Sy(v zTv>QZBlq6J751~6;gS1p(7bu*8(BPK)|j?w?%X3KZOAoG(<l1abKHf8V_k|Yz}M`n zxWxf%o0=u>;<8htg5VGftj=!~<t~>dHW5KA0+Vmq7Obbu+0EHwLtPQisklsPUP%^> z*k^*0cuVHk<zaarC{?80N-hZNHe%@x)I)$6Y4AZ(rs2Wg#D5+sS1#c|CH)y4L-Imy zwQ;(d%gE1Lhv<n?8PdSb?&8G0HyNopTo0LFO*_CSE-Su;9KyptFd8VneU#1Y-&^QW zcs1pL$zI3sjQoSkLoPRm{ODvV1I-RN4>Jea(+t5SC>`b+!rcQEQ0nTeUYRW&<Y0>l zXC#8nB};YJe=Je*N1K<+)Qj{rgwYC(pK2TU>*%wCztqBkUJv1NkyUjU5?PAzhZ1{H z3F)Tb?0|?yEWK)!v0((#^cUU1Syy`*=4D2$;y6FtwL}Z{Vg9Y~bF^IHR+V6<C`Yge z;~6_0N{&KKy9B9;U7sF5zb>cEQoT+wF{j!@R=z>E*t6s|Aby?g_iwh!AwB8vEV2S6 zKlpBN3~|DUf)=%pLSSH5q@c2KvVqfzgT=BVzL#wpewxa$Dy<7=4et!f>!H^ett&Q` zmeYWzsDGAkJh`>XIK2t4i|WG5h2x8V<IYaPaTPgd_6-O0q{s60d?;c>C6sz;ndUg! z;*biFA1_Sre6wtbAes-6R3MNjDzFXQ1tvT=h;jZA1{l&Ku!f#RCk^}$OVIoGmc`?8 zoKVbXh-5vMFJAFO-NilQ>=@377`yi;(n8-#bW6xw%AVTR824gS3B$-dz<o~$$pk9j z^780O`hF#oFbhD&t8<G55xsxQc9e)T=gbHrHZGW082(CP;GY|UEdrLlqqD}>G{DxN zg@-H5^wIPc9$SVnH*o5kJSlkUmMJbm9YaDwa(<al6F%XRnr1(6e(VpIo>I-kse$b- zJc=_3eu_Q@T-e0JaHf7S6w4ZJfQkVGuVX62FBFN)yN&}>N7XjxT7yVI_f#FYxAxx^ z-$4mV@P)iQnqBT4WabcbS`~h@goa8lU(0mnRjfC(o)*KK;)G%hAxUc_<AEdcqQ!J3 z$Uh{F;ifY5@I7j3W(gMNLM2Nirn~thG6T8t5&Y-h%4^!?h;+3`XXPpNwMeIPIIi=f zXf8}wS5>B1^Tuz^R_b?nBj($tc7*K1ZK9bDo-vS!b?y`<lpRTQgtlT>Nt3*ItEv@y z+}z2|EhNRg#oyB6nXiD#lp4!$?CGR-PD-Vg0$+e%cNGgNfOA3K<E$sOL~(dH?~4w~ zin3NEc#*6^pi-pTX;LAHyLMd2KzU`m!MJBSjgRa2mN41sMRuGYQCe^)!@DsPPrhLM zA><+KA>Lpr9!zO|PL|II-&jFUgZZmS_1%&m#D#%#!_e0X_R!RxaN6jfH-R<ZS5``0 zP*zZ;p4jjTd@tGueHJ}94I!=>ps(bd_UQA+9_62mUJc^Wvlf2dtYz}?=YnKTLZCGc zCz492!Z!D$KbA{eiWbGXP>)9FY}a|HE0zG-9Ie2^fbt!ketC(^c&?OQbmNpW1&t{C z=@;q3`N@&DoX)h9pRBR0f=)u<G@Yflss~k<M*#v7E1xg{B*=_2#rkPwS}+67&Rx%V z*8y)m`M6Z8eu+@uqbg+%rm;VID;5Mn_xU-BZ-y1Z)10xOZrU*lhx@F}i|^0cSDZS; zExN8G3?hk>ckK0SQ0wWTDwfpK!$rK!jbfedLpIeE3SYT{Nqf7TrLIE;O_v9duNQJG zY~7Pzh^yz{cP$que~86fg_9)nL5K47og)Yzlve8so40&(ddHgR^cm+a$`gB0E+^^@ zf47}(gb3PXB(1&W+!>y6k{i2DtQ~v5!_i5&W3(r+=d<e=pIep3T#?X|n7Mp1mTzEJ z0LA@ceC~~oREv}!F<r4HU98py@7|ZQGt9~q6+dIOCm>!kS=#zOwaP&8Rn+V78?5V4 z;_z_^EUYY2)|t|?#zvPNClyuI(ikcq%XdPM8r@=Iy^<g2MN?qM2%%n=Vi&Ar`Z8-u zo3w@o#Iq7Z)q*PiM7s@+1=(ehvUcfJlDa6BZ5AmVY(%{@sVBtix4!%Q!FGK=`R!D9 zt(Yy6H7rG5DR%B6WO<+7N$>Ms-!9YDjArn%(^PD?pL#C<9*9o*eIm>8-8pMy=VsTc zlRVtTK+Poh_}it|cBS1-`{^{$765V1#dP03M7S2JW6ypnpElOz^w0s25(d_u{IEWL zME;y&5o?HRdHp3?^p|d1V$|*2vv`xjiBmlxHx}<N?e0hDEr}k7Y$I7*&}=^@n}@0P zv(4+DR#{_4$3o|<7_%O~eDBqMiKTD)>fOb1>Fw4{V%`Z`pd6SI=xop2-rjANUpgE^ zgw4%657>~dd{>Was|5Xa&+mmhH+>m@T^c;VzVrHY-SV%Fv4_ydP-exf^q=fxW@nv^ zHu=e?Z{4n%7Y+{9g>BX(@%UZalK5Gr+g-vBm}PG#qaPo8S&E3RHOOZgG8xQ1c8t|s z@;}kay6to=ox1JvxJWQ$sxa%kh9qthxfc^YUKaCLI_vbe`;Dn>I~jW%JYGII7BY<B z@+u~TZqEmqf8`aebcN|`IK22i`nPVa2Emo%FMe_|N&4SHH{YDU#AAfd|4$ogp2t2F zt|Hz&$c`J_?czpS;pL7Rp`rAqxRr6mf)0Ek&Da-lKTsY2^U47=1P>1O*Gz{#oUeH( znrFM1lAI@g-m@-gw0YS}e_M0L9r?p|_8om7d>pFEpTxzyTy-rN{S}pjJR?c<vVuUw zJ=1F~3k)6^*q9b}h=LA;oIPz(=cF>*^C*cs8T5*Ox0^h(eY3gKK-IFyMTM^(u>0)q zP;@>NC7um3I1SHdNUBZoa9;}5kA1(Y05gyWcXJVm;>&ISu-nOmD-5zFh#ou_G@BF^ zDtE!HSC^V9sK6#wnP4kCr-*un%JoNLrcCUdJMV8w$l5uWGlPH#D376LoJEi3#$3`J z@@!V|t@?H%8JIK?E7vDJmiHB7NIopX#sGGHV#oq!ujQ#ioR~RydYv*u4x2q<ZP7fM z4Sa=bOn^1@S4w#Bbqtl?S&H%W)t1paex|W#sEJacRAUZF7Y5;>_ZPHy$yF_p#2u>M zI|4cQq}I)Hd>`28k*zF~!|-(dPItZY`Q5N!>*n^pAg#0&HIY()Slu6d>_&Ld#HTY= zv=bl#LU*k+R`^;cZN9iE{t1HRWT|kW?=o1L_*6JTOM{n6Fflx0fGm#nTm;<QP1LH% z!Ai=CJ@j*5Qf-!t_!l%UElr2msqLGv2<tjNzSB>wF-1ls=?+N%;W^fLeT+HKb+AQ{ zmEI0>SG+EhBM2{(gL;t~<$c>E-&`X0P!(Cqp&U&m24(C}vTR}T(QcX)EmlIUfU}sE z;ZBOT@NGKi56vA}xH*9b@UVJ&<&^{n+GJ!Ikh!I^^tXyE5Ec#E@C5s<rSbG>Qv=;9 zp0+!M7lAkzi*%9L6!gZDY~SuT+VtPPG=3VROc+zX7~B_ZGoP9Fm1T7vuCZqI@H2`b zYu<~GZ$<m`mNf3mv`Jyi);Yejiu_o8L}|;W^?$`Cks-xMQ%h$mrY5)SsV=Pmp&*CQ zcX+PvTXmeQkAJTa35A%DC@Mwr1bc>o&cM`m{IHipZ%8Gl`F~f8>PQ$9m>lO;#vwQ| zdTESBuvU<#aQ{T8J@^PY=;UI^pt+$vx=7U@qnwhS2p#wRG)A7}#Mo0yKTGREu8y;p z<CUe2DbD+MrDh=ye9Dl!s}NXW(i^>$bDdAcj2V&h=n2+WAdVk+nX*x`{b7-AQ}^PF z{#&BN_BPx5^z>QTPY`ZqmpvlTQ(-La&@I;(HvQ0|hphCi*Id7agh+`QYraaJg0+3K z(QhVPE81#gyAd@ozd@zr(fUrS9Q6}vVqzW#e>UGs>#?(;3HOwhYK}@MOPeY3OHFbN z=lT1dzuN}xBFAeUCURkQflwI>b7?gCRB`f-smTdmmY^QawXD>nm2NAmch%kTTuZdQ z!Xiw|2ve^GI-3`JZh}bl@qw<qFAZq-kKR4Vh@Y)u+&$0~uy`+xtSNT-;zpnIhWlQU zb8_MEnp7V8g&AUDFRh=iTpeWjhX=9ORZLY|y@<u*=Y11ooB3?@)K7$iJyQDbUpI(< zlzVyiTyl3Fd+Gjd3cd>XJH-q&{zpUL2Lhc(`U_LnStr{+1f}+-+S<B3Z?1wsai<(Q zpg(Qy6l*4)be5o}6#f1ENng@0?y+H*C!bT$#+LP$Z2$0Mk`ay*jB~gL&;Amo0RJjB zBCsa7eo)L};iKeJo)-JVP$@)Bv9^vl!Yl!WLI;>b;w!(5Vj!~pA>>rTw8Zk4SvBiM z-xqA}K8@szq_|2DGAjxA^LkX~yWO0ebg9m~KcN4;rD8%XeSf!g55s@jSVQ@{^LeP| z{)M#ve>cQ#BY*TaP*Gt(o5Kh2J0*2>b!i2*YCjA4XqmDaO!%2mW{CZJR$JgmX{5Z| zru3R(X?Zze$}p{mhuceP7;vW!H2C93&MeEd8~TjXM0|1<m5<MTC*`TiKiRiD5h!SE zREo~zqg`}8ii5Q!#V2GGN9BRR%%!EJ3Awo@kPN<?pugHUKt=A%u~y3Dr_X3QOiY}1 zW@LUfTcq+)zdblq%F+pSYoe5Mdvj>U{Zn{E96(xT^BM{MugMT16bln81~@5)=TA)> zn`-53>57NwX!Ecs1*w`&PtD!kUmABv{M`m!5k;WC2EqYuH-Ipja6?+HwxG!3t^OUi zm5ue(ma4vYpnfYE6QLQa{5!X%jg`?9(oeFJtCZglsQ(Gn)}O%2`ZyNDBu)*e+XN7* z;^X64-z6Cf*#G)M;gXU1`OGK7=<oP@tHi>%xSIy9$5C6dQeW~W#M2(>1{wm@s9t(i zy@p9n+mh!Md2w-18Ch5oHa0YNkB(0686f}crIVCbf4k1~)M9y_^3BaH?W>!306^(O z_kZn>1wh8Wp>NsD6R9>ed5%{<+vo~_1!ZLDJbO+~V_>lCD2T%uX*<-_<$v6N74lce zSxf>)8B*GzS}GbZ`;{}}*6GIvR_>|PR8(H#<KuVQonZX4!o*N!0VgVjs#S&Lyfr+2 zmzxSQA#eTYEI!&MN5^yU9>8?WgysAe5HaI!n$Bwxid5{pzy8BEy5;k58il&PioA?L zMm!P<s-{|}vrGy28K!>*)B-uFDnM34?_KqypWtKk8|nVHTa_R0dAa1XJXyJf<kTi7 z&lUE?&j0mA=$%;14_2zWItm;3njoJDpb~+wQ1hRNN1@gfF>DBx2vMe16k|@DH!Y=7 zdpS87Qe08M_J#`+`e**wUj2Q!#NB@z#JM9YAe5jN|H!hLdD>5e2`YZ|n~qSgjyX&4 z3E%%^lFbsid&*UZ8q?es?&YoD-yDrboc)(fw%Z-{Wp>$2QKX(tku(3v6aOc&Z^Si1 zk57zy!IK=F%zeyXALEo(#~QrEP;~{cA%mBEf7Um6TQtYR8?8(SF)kSA{a+efR5=D4 zxP*a85{zgRnrXKmbv`;^9Yk>9n6bj@nBMRKOnCIqOV>SOtb{GM1R<#Xfu;o4$bV=N z5_Bs5E^}D>zv1=b{2ebW@887#zZ+oFptx$;H-Fc|7{Y(P2YU~i_>n^(hcCw<&a7C5 z<Dt#GB=qSo=`GE87xG-zNO8&yh@`n~|JT3(Jiz}Q71eUym3Hj)nP7J7XLuo0?=Lm* z1$NV7hZw?jBnp)Jjxuy2XSfi-?-u9Wv-ke;zp|#IqiawN7Jo&=-Px~*R1X%7t539I zCpVKrrM5F6iHeS9;rln<Zy4_UsFoB)<Qt?osZyxzgXpM?em`%i{?YtKVnH=Ra|2{j zNjv+Al_>y^<dxm=uHKd@UDC0`-wuIgzU2Wr#rdZ+?T=k#Uo|XKrQ8d2o-K8Ktep3{ zbbgrp$46ZAJ?9_CxY?j&hSb#5!g!9(v%CK^u3H6y{&LBKX>rn}K)~kpSEa?DkU()3 zQ#51Jvz=qN>9YU2r;JDB|26YnrMYEuNcpj~!nb^6h&}Z3TnUrt%`Ea9`<izQ=U=13 zv92nxlM8x!<X?0g%Cj^1(-CX%N9?;2$jQ>zw)}sBL!{{Gp>YB%50#;}k=g5CKAm}% z?iz*N*5lAY4xX)Kz%@(Dx>&**`Oi!v<h|Q{vFhH%9H_d{;?;K`xxUij^`E|xuUt`Z z5;xzbL*lqorIN8>J~8;~IvXs1Muh2wg1pO~zc@z*i$WR63lKk-9?O)apiBI{t|UHV zf);^od?|85mqg~T@3;Nj`WjNrzd`zMMoEbOonrsX()%B4@qaW(L#dk<Sy)&qCxHzx zM$c$^ajuv$oAzzYF!xKdS5X9SIkWZw7U9uRR$_Hg_Zq;4zBTfLC@3gEg~N_XoLiiI ze_!6|)e$Py2^K7DaC3h9?%n0*qW~o$Es_u3-fOwXdg#BF0b$;_jX>SG(!YTxS;k-6 z9FpI#+9&r(whFej7PPhLS(C^g|KyaG>EAL8Gjl&YJ3B+jmj5k)?WfLlC4~)|od%t1 zzK=tsusuD$B&He%T_I-|*P#?6A2v-*?oKOhHDf$-Q3qYMDNYLZT5L*T#_Epq&i2!X z)y+CKNRr+>zf(U#ZgsVC=go~ZBNI~wegJL)DbG=SMa7bo55aFqX9+8t5@2Ct4{aT` zZ75emXxJV%4O(z17&|&T0O8Q5RQJlsTJ0hr$jf7Kjm<P$J93MNTctd-Y{wNb?Oo%D zJm^21EiBeL9!fJZLCU$7n#n}GwI6P}WniO5kF1X>B#eqy41iUivwJD0<`{7^!fF#* zSY$C|UU=Ku+Sbt-_gj)mFH1mA5ohS--c%SiiN~v}(`uj<h<02g_tL+Vlb4UYyWxc- zG+U#^r})XFr>AePpgc`V$e{oBjn=8olU+?y%aF?L=-CI3pgDlEQQpR&15el7a~3a} z=7evyM5Ma3MJfgaXi2W>rad&YujQ#Ql<bvSwqJ3MN~hjKzCO66A_OZnH8qiYpLe)5 zO7Ep0{X1U`_}zOP(YS{PIF>8`R+?6j_njY#h*3;!6nN*1nuTWXF*WII*yvu)ar*Xz zcQQ1t#ckEc2)G|}e*2;#LSC+5&9-oqX&-TVj&~sQD?s6{X&`C#+o!65i{y@EXR`RV zhXuQjDQ~dCd+MRO;(c2U@Gzp4mXyQ-WsdR(Gkh(XDfGbg9?vJk>1mizY5>y?LQs${ z3OFW4-o_^L8NqX=sEOQj)Ou+HgP>WWsHiBlN-jx=C!0uqTbqpT;zE7&S1_HomWyJg zQhA}?k(QPfd9C%BFTd*=_@=$8r9P$ftaAB_PIRUa%O~E1&D%}Ufg7xh0=16q+vAks z3-Y$aM)?GwnsRXU;i2YbaRK1cY`+wumh<%VbRo%OHB6@2YvN%V+0Nko9g8>+02XhE z>1dBwYSZm_52p{Wq9LR4!$iSI-s*#@$lV6#^+Q+^Y@*mmUfroKEADiCNVC>)#Pj+f z(vFseMn1AQpMGozXV1sTuTB3Vgluxv>*m12<JK;FNPrd;Aun$}vw)@h#$;<UxpHxH zI;~EZsB*Pg%2&TbNvlGirlsk$K}EoOqs04cP;>+3e5u;+Qij&jl!86-MJzU%z$=@> zLp;Z;)VuqHu?BZ}*|FG7(W>}kfS5)Za-+1jN9v;MdL&RXbvYL!lES$y(V%)hDbFS= zEnO5GB2#i~$bzM)2%BBB8=h(AV_9poT`XFulY2P1e%*pgk{j{@aUF&7LDFwF%<oZO z<mO79MeGm5)JMR|7PzQ*gtVVrtZpovzab`d+V9Q>s68#$X=ymscN1*nBE6}Md~Q1? z;aBD4<XY>S+y*{PZNCT3Ghp9RjN~=Wq&hbC_Zr^lG+AXZ>vgJKFp+B*sMDA{oq!ci zN|n#qCN^JL*E*h)$Ts^*;z&ywwZG&@F)PgZqTOh-$~c^4dwn;&>BvH*y4wgR{I>Dq zbqs7dA_><2ZZ&@S7JIMolE9r3`Ff9(<lL^jtStG;dalrLH_LoDS=Q1rb=D6lL<-j3 zpKZl?&^xFuclt)oO}U!lS)_&)pYvHW7w_M^!e=FwkvwRH9_(lTV{GR`z%6A}u%t5> zU<nLcV7~b#4@va7l{_n%Ezb7Ke<K^5n3OUMxjUn!^~8I50*b)Xbvr*uhcHqFmX2IO zSMTQR3hOKMd2E)cR+)_|je?v3<VD4cgJq_uC`H(iwO#4uY9c`HJQSg6UHGNs?WfWa zSWm?Hg38S%n;$J<Quv@s_T`?@%ET^=C$jU=q4I7EVl)GPl=65kGhgeIvhptHOY@sG zd|QS%H~EKqQNL0PX{o-x;c`2l7r-o<UgZS<m8Nv{JI?k+hj2uz;x0w5wDD}0ra@DS zb`Y9()qS!W8i{wCl}%?}tVW+wRv(UV<P^AvfnAw}l*cw{ZB1>y)g|Tbz~V7i@#XI@ zSV)h+j6hudy~XyPxo+x=nXnt}MHxlAoT6gNrkwT1<r*2asn?0p^Dn8wq6ZJ#y{~P* zec<;JQNXJ*N)bADi<@LA=lz&mNBFh=<ZQ%^>8!R!QSf4j-iSFNW3be-bgpXo_WfLz zk1mbR<?^K?$5uR<h^tzL)ZiXB3N~31rJsIvy+)(=`8luS-3H@t_N^F^Te*t^(KCM? z4IQ1zDV!)2;|F7Q?BVgw#|N)@;4)fo-?!mC7t+o3vChN_y?M`AY-IdDuKUI{4X{&# z2HS5^+zRvS)cyF16exuqbVaUqC;?Spv-wg1_tI|UITSsH8xK9}dNbvA?y3R*?D|KY z-B~RsGt%wq<Ae3}Amm=d_hu@4=<&`-bV`1X$9A4|yN7x!Iq)03{yX?RlurQF^4#iG zq@nM{N?7*5<Nf1Zo$spX{S;)7{_#qGt=b^LsQoM+!0+0fXt0AA)pRj`eO+HEc=^^t zst_nc9k#aXWQtN$j4ztJU4}h}Wvo6J0_>9~ezMwc5<jliNOoTBk!BLIyWY%Lm`YEw z8Qvt_#XO#7pM~os+;nK)AlETyp?$O2YceI_<?-#cemvZUq#P_Q!oucOllFNN8g>)_ z=+x}TyY0Wu1?d-}I#f@F$5C>ROFO?g1W7*R4#$fF{_C3zb5AtFWWVLN?WGomM)J*6 zGSOM%b#W3s)Jf4E@*)psocDK88M3gO#p7Yf!y;r@*%w9$d%WM5cZ5MbbQ)~SKQ1)t z?c&D}CuAn%Td>&7H%bC**ae*sF_Fg)X?IhQhvK_^Wk0&dJIJHt%KeV~CvXe1^|S-D zF*I}$D3Ihy`cg5H2hcoJmDiilaryK0(k;K;2aFYcb^A@IciqR9tW88k<U^+iiIUI4 zgs(CUmT#J}+HhH^!eZo{<jcpIelrSxR#X|GPJIKpT{k-0Awon3owYxVhT%E(*7=fy zRJGlkS1@)mD7A)jX=EA#f^?+5t9V|=Sf4Qv_l^!%L-Ki5m>C(9I4#(A+RD3`(PO|_ zi&Mo_x#BCgx5cn+-qnPX9U|eI5ZVvfJ~aw4d*8kpr390HlAGQb9y7X0BH(p#ae=L@ zio_3;MB=bY>Opj3LVrfv=rlQQ6pMPvDmOe_tt-Eu1CHp~^0FTeL}`Vn)zI>tU|WL6 z`yFs!vc=w?(OOx4oa8J9W*Xgb`z0?wz_oG1Yz>;w6xO`0ro1z_U(w2BgShX`0oZDl zyx=9bA31XGItYwK?=Ss||HMDv{lBG3U5O?am<`?nB{>CUMOt#hOuuWvQWP_qtZeE$ z4+Va6-Aywb+KV+@-)i2*=!nnDA(#W1j56;1_w$!b8<U&b!WldDlcM}BO|)BT6_ePs z09F|5e63A&J;Nd57v70zgWHW1h?fqqKueru+deM)i1B=!&lP0@FwXE2;?Ex<vi;60 z^;<}aFZsVtHg=$3GcEIo)}bySg0$p5v`*^V;E@o}QK8_-YicGLhPv*IKP^>-pvkYi zpfJHUp_?k3*-s1|p5d$jKtb40uuc0W+tL0|`&r;=)S-T*64;7phxG}<Zbdt(AvYuC zo)BtOR0Y|GscfUUt$|@S2=(LnYW0qAv~j%m#W!BqwkYHk<ncIV^PmO8^LjRPWN~wb zcXe*eujvj&*xGV9dKr({$6=H?P+PCVx9)Dc`Z4G7RV%NTvR&f+!Xo7msa*#X;B;2s zL{okj^D4wMuD-kGzTRIIO<FBGzW8W=cuj`d`P`akK=g4#blTc5NfdHHC~OAQPwO?= z0Nm|2ys#!5k;KRA$A?IsH2eN0&AoHKo2d+dTi9;+$cM_a=P1SZ8pX!mpcRh^a=Uzd zj@Q;CdKcD9t3S8fW2kr~Y}a{T^>MLA{@C1K@g?NWGmWCM2d^{x4k*a{lN<fBwiQaY z0ZPNod}T=N>f=xI>z-`G1h!5?*7^CS2Bq)K9+B<U&g1)Xf_Ilp)w0k+CQrK@8^Oz= z#p6!4$MZH(fLC|1VC~3&-;YYcI_6y>-SyhyrX!G___*9)<a&TMQ55^o`3a5Ri%Q?n z<KE^|o2$X`HLAMRHDIe#cvs^uA8rCZt7iwnH7$ExGTonK0e+6fUYk*h4^>#h{QCe} zE3eD2f%3iEhg;?99WK)?9uSEMv97jua~8+peFhT~6L84j=X>blah)W(Kh}r&Y9B=V z$8Tc={5D*ge6x7JVineY!z9vAjFXeIMHaurKAOxw?;Qo=j+>`O*fk-7{`#%nQnqxo z`&iW{{00UFo;WnjKU>RtQNblcWv{v)-XyD}Wp)%>T)YQu?zo&kxy5|`JY`Q9{)-Ne z?RErLW2y5}a~y+}6|pBGCa3q7sD)O9{O}njJ+qFy{Md9T?qt-wq&w>gDE=iQT)}a8 z=(gU;or|z08E4rGa4hx`iJQCo8vtJSEK}*|8NvPtQ=BiD&rtqttj0P2_jbH7O^v0Q z40Th)U%jQAutnaLgcA#Qd4t~u*j>9q2XQ=f-PU{@b<S>SI>?j=F;J<rtZKIIv%|0l zb4$}vYeWZ0E)pKT-5%{PcWk%7Zxz;-33>5q0oXnJxa=WM&M?jf4%-DiL0ppvJ?L)2 zJ#Ga>h5KnM<9hB2V*}Hk2*YziyXkYJ<3i(1R(rw@CnuhC&(0|7(Sd+m9d94Ij$i-* z6P2#GKu+STi?BR8$$(Mk-xv~cnHdIUPoIE?r?ee`uLV(1*<)7M_>0vT+M!_z>4M@L z^f6;*w{?ElS>w<|wv3&6QdIYBn@riXuUtC)q8M`+oBptWb$45Iw(TJ*>hXAey?S^p zvj2FcKqUVlf>A9_Vc7nx{5ZJLXPd$YrSaE5B+drbpU@8r2>syf6n|C*)w8PHbWM`X z-cnw8#(>f%`ztT?9-G`#v0?Y^^S9X+A%&~$Cp(F^0weogce`vDb~fu7JijH&2bBK; zCkPDB;8F4;nk5%EncmR~(^z*u`*TPlM4C1IH+AeFz!jOrF~9^>`FiFo^!oUBkn8o| zT-pDK&;S3{@T&kc(UWz#byEG}_nPr={hLKaJ?R0`d3Vc;i}Z)&<gZ0Us#Zi<Dl2W& z9GcW_P>{E{)b+gD5?4FQ4Hua>IXQ8>|K`)k|C2@!1MCUA4-lEj4s_Ev9`rI=*;*x) zm5~6>p9VIC2wmI+$K=cmi;c~pBfw;SzNGpoyq$qO(_mWhcQ0zl|0AX+Q9;BKKyK<a z2R}dI6=_{)`#JVQRcU34-h7KAC!<trYimJC!5j2Q`GVXUS-^;#nfZzbY;0`;tVH=h z+V`!zJd*|CQ$V4;{~4JFYUd3|#M@{$%>dD4)3xpV(d3n)lE)cdRu4z2a%vJ6=Q>B8 zx8fh>Ra`-_ItEne(ilSVkAnI@okYe}dK>gwg1KOIF&(i70VJ8CIq^+6KE@8Ek9MCE z4U0FcKb`5$eX;A>@%ZkhsHnVu{|U(0B^r{%)^IfW+87x@FqyDId?LQoV`X#l?MzZb zZ(F^-yRk}jH3zbvVWsMIj?D`jPhN{yY>RoOflhz+o(ZsyBw%FMd9lZggiXPSe%5oL z7#b;9ljEbh3y<U#BhrKopwRm}1;MW+I&r0}=vWjGZg$CTnrLnA7X%F<z@C$H&=(St zOBWXq?tV_yZnmm6IB~YMAWJaxr&UlYr8Fkvvzo;K!CnNW8G5gzF0vfm%1=~uIpL~2 zbYe(P%v@PdPES(<aWw6tQtsK=#Zb3d@k>aK%I`S29<0#eH}OsW-PzWh<u>!Lv`)HF zwO;4}bcz2G`)-tj7E#sJCuW}8*wzDNugJm&niJFG5E02eUA#;sM=&SYm1@n5-@11E z39dcp6_OUnOzcP9PQp;85UkQS=D)nBT~b0#L}^;&--Hiv4B$dhV;w(2fgj0x@nnL? zGE~&s7@Mz@Gu(NN+C}c+YvDJky~MLU82c`y$4n-++8&x2Z|KZ5G3YGk8K+b*$G_>O z#cRsYuLOtHq7IuQ$#s;wZa`FFpDO2Vroj3bWF5KE>HDeGU3PDu^gMJ9s2ESoY`gsJ zRj>2|DJw-FbWa8hU&%IgVO`zS@%DG50xbAd+1pnmb6BIzvrmRtktILB_9EE)x)=Ms z<<2uG=}F1dubpScMEr~~mDXdj=OGr?_CGSWEu!$Ng)_HqZR*v<vPZ4ViFbWcc|6oC z36KINIMStxzPu?@hoykxbUuF|;FrCdVtyT&0Gs;+QvG5<!MbH@Nx+bh%FV^IMZ9cn zW1TQm_F?z<c+LfW)X!S4jVB&RVw}5uCoU&<jW?ck>7kZZI_%;Ss1Rp`(1{BOOH12Z zoTeet#jOv8z~L{w)M`O>{uD|6D;0i}k??db%Lz<aK)mas_Pf4ViIMQh?l<Ni#<PT6 zk%3P}sGYQ_XBe;L>js#)BB6oaF7avNe?DhHnc-J-QMVV}JU&CV4<cr;1!XsYZsSbK z)(r>&-<1S@ysbNhQjcLVeO6-y9+e?cH$ayB7SP)y^K0P1qV9!J=6lfi@6UPkypdV- z6|Uc9sk1?G9<fU4Qd_w??<s&E;^|fdX<?u^Mc{XA`2qo2Kj8NeKvJ>QO0PLBe}8^D zLT%j8ql9-2yf(-gcsR}o^(pXNoJoa0effy~|22*QJyf+~eLLX){*NAD^spCUft(q{ zXvV?W)*O)v^aAJr()Hd;PT<fJs2}P#H!gT5aHHqq+$K<HG4L=ku}YWuiUK&%oe<G% zQZ6$h)=mRX8S5-dbN-I5W(W!B8`Q`vaz9^4-Z?u86z5<`Fb@=CYyndOaz}1|Ib0da zHCUmT3{5wdiFiN#Qr5~VPGLmh+V@-%5D5X0SQqaU=!Sj)o*25YhWSaPQCjKvQEQa! z%5XH4+3n3|{$I=aQ(pEh`Z4^>28UR$Npp+~|JOcnMJfO)LIBp9*dI-ur$8E9LLA>O zwmEwD2s)bfw28;$Td%~1JCSM@8;@U>Qv!mm`>jCO3>qjD7zt-$7-d3*a7F<Go&FGh zm7KZhXARoEMUmK9*nZ60;}LhO&IBg2AcV3C6jy9Zf(d5bSAYNk?Rr!=0`rRObdF+# zI>DW*$Y}^k%=702VqGh6qJRi|rw6aJ^IA|)kY;nxA>PG3jB9l?aW?RCj*O31Z2{#1 zZT@`++&1C=vpab|oRkYI5AHZhAs&kQ1_o<d@FsGF9=bC<t}zZtvnMq&#crSvYL9(l z|GQ2Ad9d>fGj`AjPfpJ<U<sd3PA@XB+>b<uODC<|90^z*>A6EAl-`AbmB0fXPg2&W zn9<>Zxn%N5WA!dm^SRPs5<{t?Y~rp%(}Uo54jw)RewmA*x5{o^*HZskPz`NqKe0xB z_cUof0;9N)o5?`hASk+ZWgHFD)gl9!+j)h7fP4TE^uGeuB$UJNaDw@@czL-j>$^8Y z*MazsHGB%Day^@BI>9v5tuGbwA|+}2#yFVbt?>(8zDR+NkX%`TeFpCDl(KtJDi8Fc zUjuW2EIyd^BjWFU9l)M{RW;h^e;WCp`TW6+vxhWYs@p<(ruf^o^8dxxTZcuxz3an@ zBGMuyAR*EWAl;!bG*Su-jdXVop`<v{-7<i54&5Q$-5}lF{eE!o{f%?Z@BPDzO9f`F zr|<iD)>@L?OtD~Q2`%eq@TOghVGtZr`r+Qa054Np0T#!*S}4eVLyk+)`<@l1{v?4a zm-S<g8-syvsgCCETcYF_c48EDH?KX|1nT!=@%1Cp*y_HL@C^Q^u;LR#`BP~#s9C0C zAsj1CGPlKbL-SL3`z6SLeczk;a_(#|hBdGca?mJ4;M1srZ)~<9L42ETX%K@aK;ZN4 zsJM5j^oYM?b#GFkMznEY*L>cgU9(|gT+_C2CfaKsBLr8a=R`y6J0QsedkCOb@;GuH z5~Oim3z3l+B#ldARfc?;hh4vhrf1fx7fC2l+}rn`V6|gq<RXfoJMYOljuQ8iv3895 zp6fOyHBJK9z1mI@u2?<b0Y?-`fRgTmAMnM8gTEDIMr;!4UGBa?wBhuj>Opw}3GuBC zaL>3@wyBb`M(&+>lZlB*c;5C)Q%lR5yP5AC+!c~WDyK8{1o~DL92{I`I9$NxIkmO# z{Qdpw!YzdV<qJY=po;#K(Bv+(x^{n5qYp%&S{rG;+`T9635I_Kh3q3~z|5*eB@uL4 zvQ-ipbZoM2X1WAbq>goyqo9)Gus!=Pwc}WSt=+GHB+&>B*euq6sx=-=1Mq4SR&ijm zh8FsXIgpvDWfXwYrA=JY-7D;ZfZ{+3xJr8)n-@gbOlh!J|7>GxD<(Qxh1Yg7&vSNA zpEjc=)0p{f{w6!{I3zW|`2#Zm3=+X9L-WSQ|7>N8j5u;SHPi4%vCwboPcTCjt(S(Z zVM7qedqfA#z4#ADca8kHfrQk*+))%M9w4Kmh1pB#>INLCQFR6T85qrmp9W*~J2;&r z`};rX1Wq}pAZmVBOzF{nbw|I?A$))3kb5seF?EWBaBw5O8YvF+P`WF)7(@np!=Foa zcSPRadh*%}eNY-=Gmxyi8wk@6j{v3*sAodEi*1K8gRST4H3G8|tH?(uMnL>dJ7K<C zfhA5Y1i++6#f_95a5(eYQ9h{i_r&L8%(KqVZZ92BxHXr2a+F`rxC6o5`ZgHjxN~RU zo^52ZKpS^vue4EH(y%xty)2LCw?`PTwEL{)AgE{mm-Dsj6M!Xkw3vX6@*$qKhbs>) zH>SRRZnvK0`;wR0@gzt|MLRW_p8<Lr&{2$=S-M+nA6N9(mbJKfdhq~vcnR$NUm51h ziQM*sp=~@wR&m4IZ&Q1c^Cm34l&ElsFSWi##a^T%NkvCsM5f_%f#A^OD3gSIV)Rgn z-Az2usSCHc@lQ5<{0LQt4Cw9@v}q+=DFSrl;X$TO$DWR-oD~)ImUzeRciu)2v2%#D zN;4!74MMMX<r(_*k+8Iwq|NKgeKewVA6ML3q;K-?QqOvIR8<p}1CpW#=aPSTrf{Db z8K+;aVrDla(X;2|JwfvQI?L}f_qXQ(k%G9w^)|TXNjeY1hs#a#kLsobf<X}9zPnEL zHLBO_GH!0r6__F8vYEU0OI|n4=TY@dd2oH{*DEu95eI>&^A*!iuEhaa{9RJ~OC*3r zVRC&g*vpS93I0T6S7SmT1&mg@cT8a>ob<rUPn{-9^keX1BD)sSXwxf8WxXK4hz-@8 zZ1})Fos{(Z;dhKc2dSU3#1pRv?&W?^mZ#ky<+O)>P@{u`mQXg$E%fNlEBZ@qlg0Dp zz&+L)F=q|LufofBY$3Zb-_ua2wtl^;ay+ydpu8HvsX9L!PiMZ1ejc@iUVkU0|E{&@ z`R49HVAVA^)iR$hY`dlA*XZmDJ&Nbt)kvyP#jeWFNbSJL;;=Bk_M69#?|v{~t8E&u z9qJ+v!LA);-9GY)wD(T%V%z=D79r&dzCN*^vHIff$V*pOEs#}?vND0XLimO2L*{rt zEB`MA`3gM0l$hOk1|k*H)*c8@61}K=`#5|~#BXFc7Caz<qe!^&smHLhvx>>O{vtS$ z*ZzGuy@vzPa;z}a3mXhi4$I(-)H;X-M+9$PzuHU1Pog>;477`;|2?=gHY82^TNd$j zrJ2`Nz#5H*HpJLC@uQBK>$Wc{K0z=mY!_mr7+8N(d(;Ep_#uz<G#-);mKRG;6eMT3 zf4a)4g(f>MFMJ&qvpnNxlbxR%Lk{$bYWe+uNJ5j@mlzX2#RoaPQLl2`It7M+Cen@R zMn}0jI){y^n1$Y=vsv#aC5t<X3kwY^m=ckZ4K6w~v66tHCt96+l@k6RC3Le2<=gx< z7s-fF)xYP`g<wU5Zxyq#?rQJf$P>RX^*NVtZMi#m+2>{N8I1RG^0n0oe_mI+Sm{h> zOddP*#QIopm1O|+%f%wmM~Dj`WCuzwxmCS4qRvle`=nU-)YeV-G6X1e94{OtZ{gaX zU#&%6gMYkDDz$Z!xN^&&V*B8>WoBsMAWLHRzG_al89DxMsXtJsVdE3;ORS;L@jCMv z<KTDRi<+4cNE$2JH}ODV_w<z)d7>L#OukZB!>aa=RC}pm(^K%J^j*S&<^>tT{Kqb- zlBu}o6&b=X%K?kgl7nqAkz=VEEZoMrvgk?nIx)OmlD3QWaOx~p>(L4F&~~KV7lG4A zn4*@rd&o4aLzPIpr8aYDUybF8(m*WB>?6VIdK)E(xZA>{KtcWIwx<q8>OcS$laL6o zm@8N2vYZ4c_F$8P;mJw*X{VitGNAi>5Mg{g2d=C0u-oroodoVNNaVABugN^F=?y&D z5#{nNvZ?89wkjohJ1{~}3*@$eN&iX&;}kVhcT8>8(WTS=dSkqe6f}pC`s&)&BeSF( zb-nes^(9oxDjF|Ih)P-CNJ1ZKmP@VQt;)vBwhZlHd4p~HCB8!km&dZje%ZVzQ#JlE zsT}Q&o}as;(gnEgw<#=9k?aAn3IE!jiFHM!xL>D;Xq`4&MJ$A;pEx$LNxSWc!w4sM zwyQ5}Ir76zrTN}NdIsjO_1Ny`1US^ZRjzi+6H<ccB%gZD>MC7Z*h8Sn{8f5Rg~_wO zp)<j@*smwDNCbRb$~(IyS_S({p37+Pzv`8wJ)9tRJrlI55vfvGX(FB}^B{ZeQLX7> zG{O(poGD~cV9tZsy_ZC27cWNIv2InJ=GiSkd!kuYnL}5vR+`T^ot~y2&E<`pj7XY7 zi!0kSlS{Tk0dP!N7v1Ns89$0nHb9Df-^*eyDKB@*E+_~~ChON`<H16pAiiE+p{ypi z*a`&Hq`9ISzj&lIJA2tY+8s1CMfSoL@!Dzrx0{iZxU}73#=dlxtei^^WnzFXJJ%Zu zqARZMbrGFL<b7L;q+71ag32gnXMKHxwtXS&N4-Je)rX>SFXOm5$dS2$!8$98mm{v= zq^Bq;P=dIo=d!@V0Ncq8qNF=YPOR^qBqSI#>luZOVREs-L_Ru}-A@jt>fu}ln=p@u zIygB=ZI8MLlJL{>g13IOd-NfGnFV;DSZ;qWAo-bu$a`$lPkF>+PZY0w2C{qavG7X} zmS=~PcN=Y*$1z=}Ns;7O$LZD&*Q+olBHqNz{CxZbsvro61Bo!vI>Sh?VW0O7^ZQw= znMZw6aUCSRPvY%1guaHDW;g=3t%tJbu#>k)PLs50dYL$7$(*2fE+K3>FeaUA<!8@1 zND+fe)G|MOf0Bfam17++F`%vkP!gy^wI+cGNKt4)%pcdH+Qq)+W|&&t@dBaKi31ku z6-z%mKAstau2zkzj^_N`?*YQz^??e%g>%?m(0^B2%~DVT-a*jflX0WQ9(ep51N{~# zI_zNKig=i41&OB_RI&(e-ZdJGWZ;O)bn2STT@deuLq5jZ^}mehYg6w(E~n59{IGYO zlpG)VeF>UTS1W6TsmiDCA!w&RnaRdCkap|lo(y0Hpp24AQUyxGLDF~e;I|nK#oApg z5}gXr+yZElL*3*c2<g@#EaEtP6l+>*8Ttpn)BcU5?+e@BK2{imDS_)-pmd&?1rd{w zhmowEO6faQ%ceaeVuR^E!`J#2`ZL^JFoFIBKpXIT6-P2VZb@%s7ox}0pS=0mE>3MY zO+#d2j<pV?+P^`(6E}NpDg@8{l4LXe(Cj;gbLk)`{C6z)eY;t}4)bB%ea`;xs{XG; zuDoppjQX5O9%o|?X{%GQ4=3|;5b;WEvUeAXOG6Tkt<!b?5{pepXRl#lr-;3%$dkU| zH~hTJoi!CwP)o1~BKCi;Vr%Qz)24-VhXdwogaedQH86Job;O};r2hrj|14+H)oAc( zFD0;SJYW@$<>MU}L0%Da{^}lK#OyFUN2&3QH^=F+!N|tSb~5R1YkLDCh3N&i>f(Gp z72HFjiYX~hHVqtv%Uu=^qS*e0S^uta<>lqdK4{iawP*p9=>Oj$P>lwA>WH$pnPY~6 zDTzqQGaytzLE}AZ&8ejo#+0g4&+M~fS^bW~l%NcMF6<Z)VDJk+v8LvaW@<=ssP)fO zh_<#yje<?BNcpKSA*O8XC%^N9ZvG;80J7&FY&xu=#WXrz&}HHb0PIxAV}PgGrJM%< z;{PnHeIXqDubuZ2!1}lxum`>HfX}8TxWl+1K~nfWuYDGi9awlyD;u@yqP@MF6YMN^ zDyB5!I4uZHD$2Wp<b=CKLb<BS#d6O4rc`2CkXZ-WR-B-9LoX}j$s=l{P}xWulg9s- z!rXEgGcMIZz3{7n{}Y!2DAwO-@SO?4#Y1sLTZu24xwVwzxl3du#Vf4g3iuP|YfCx6 zJ7!(WJxiEhGgsUWd8KCSv}pV_$?O|{KsbbW=%jLmqvO5tHpq6vlLwPCgWr$)q}Z>i z$3|+_JP&on#pq%dmjquM#$oA_)BI1cngU#plPz1xkXARVZo{+j2dw_TD$$?{2!}L6 z5^y{0*|emo2WCVT;kyfc?c&Pv5f0T;gaUV2>%_aEm0dDQ)2jNV&pz#}wC<CJ6XMVe zS4Vsr1I2$tK0eKUZD+MaYj@jN!4NIzm^gVBl!y>wG>A*QTy@N@BqJFpDbiJNW`8RU zZ!zc++=QAu{9oZrYb8U;UQoNu4O&LDn7#!tQbhYC|He=M!9t*M4I_q?2DKP6k`f^X zfY}6O$I(j_)V*?L;;nAfjUHS=BH|FloiMLU`&l8qxT_ug_Uv<(=8>GVLTr3e_&%=r zvX$AlxgV3IkjHN3xf$DbfKUYpBk8{KFtrXEuiY}kP;phwbt_7+pX9qvhI=if%YMhW zB2x#*#J_2$zkLQBAVQlgv#XuJZI^B)1gO1~pXVk2%|ohG0bnTj!aM}vQ@>y<Onpp$ zq|q|~va5vnLfCPKmll(qP^jBJemMKk)f)cB^7rGeh&fEm`IymIn&J%BaP=q)s^^S_ z-Kjy=GWMPYGs=YPN<gn8<L8V%8z({H3lh7|<@K+ZcUfz^CzEGJF$E<Xbj+dKudjXy z@_e4^JKKNsZw|r2=p3LPDraiA8Ra&llyIG;W^O$G_<5uCe}tG^P2fk`ugfG5bQ8MJ z7canu3~i<arY|iCFe|TCoQ{|}v*cC6o=1Mo{~88A<#$_^{~>TOJFwe1P$|PuEuB%C zHX&PzFS-4*Fk{WCDmf5=XQcS1SZp(5#oA__o+YgI^_AnA6j{VQ^?3-711nGHs*SR= zz00tC@@dKQGPKZ%i`^ZFZ<+(*96x75rz&3%WkS;bUo@F_Z!P45EMi?Km<u+etUn|$ zKzHK7$zbO5T<#I(r9CUZk)44gjXybQ1jCb>nv=YUz*N6>a(skxgnFWWRYLX)a+0v` zmLvwHxysH@O$nHilWPpap$3KDPfrE7<^i8y0kgw)EoMPP8e=kJw}N(z28Cz>PZ&?7 zSNnR0Lfw>qiVI%ZzS7zGTe`Y!MjxBO+4awvmu_Yd0MB#JSPAq1PaQFz{V2sMl~mS= z2s2R`<Q~e6(S3|I1h8@crmXdQ;C+<zTgyF^=n@Iw&4lBEmA64qcDE`OeermMy-Avi zvpqM1<?N2BvX6FWJDN7zieJv~6Zxroc_-P1MzgKWttmBo<-%*6Gi9-$r_-sBQkToE z9O}LEw?rEoZWi@GD(;3E5Hx*#uU+>pfp>N#W9TE!%z6P`L`@9kiq(g#N|ehJXUc!= zrl(I<W_0CgcHQ$=xIQ<R>h?n?;)o86zlw?8BLjlop7%1+H<(rGIIrVQOv8smiH;@n zzQv?hKi(kPar<3Ej{M)G{A&Pcq0(>AMIIJBkBg1}0inX;hRL$>vQhHZaLE`$0)Zr) zZ4$HKbjJ!0KH(UF<>sG+6&_*M7Vh*}b_zQw43+0aj0Bkni5+f|r|<vL^PuGrL6{t1 zR8({&9noeK)<IM$U${cmetaS#CW#JgukGXi{Yn4zq47h&aRq`P90O`BlM>jSlJXV1 zzY@OoEh1&tK7XzM2DUps`;Xx%(Bv?KC1<1P;=yOG>WHVNo7<bwQ&W5Zzx1c^AunGD zT<P7}`1{9BD*~uk1e4|j<c-3hcCz7Sg<E~y6U$j+7)`<DsRUJ{i8Q1wmTnYW&;Gly zkfci>CmSRKWQ9oQm?2Pg2o6G$_5&)q^?rqQ^Q_gG_gY@Ffd|2_2_+qjBxF}R^E>9c zIJW3OLAKrAu2?tH(;KyW<@3W|rD!q}S>#3BRui);Dl=8)21WIAzz9NW3RSKA^P(zc zP^eg6zq2wD7viwFR+rAAy5v1AYBaaTpM5sI-WZ-j2;>b{VnkM1mdLl}{?1DCfis1H z_1wa!Z@DpLuM9`4{t-&i_2q7qNp=kxBd(|q2W)wx7k=sgB=8$>7k%o&JwgHA;p3xB zW9b5>TY>XGJR7^GnqE$=e^pYh$vyDFPQ^|O$teF@KEbQm$Fpf#;j`hdfMUl2<?ljo zXqy9JVrLd^6!zi`Z2t?l6^+zL>x&yNi(7Y2W^DlF(EvBMtGg^#iJin_l2oj(>1wMT zpPqIF{h}gLnFqjMM(YQ2^90rKUN&|?YGkOB)WX6u;Mn_u)Skc50&A#Ik8d?y9XduY zy!Fv=(eD|(K#i-YUzX=8ttA0%qbKuSQJkBOTZCj#HBHD(l;-*Gm?5P^Qkyt*g8NAI z%P7pv!*U2STfRPZ56J{g)f^G2{Vh6`YxHfUPR#UV&%U3LqD|8Saj?aAj&!O`!%(18 zKL?UlB&G$pfSerj<p6v}Dr)MFeM#6+6P4zdRlunp$+vo8x1gF&Wi3vG7k`Y%++D#g z!Ah!K0oo|u_i2M<D{6Zc!<5_%^PFEG9%xXR0wL*N3^P6lZ1pK{J_Q)7Ua7;rdzCwG zJ;-kL+?u94OrJKrC`(NxPn*`6F?ZQXyZ$A=tvcF>Jqed}#J8F#dsH(sv+%kr(e(0y z_rO>$Tw>PUm_F_?)Op!|J8Jja;b-<sQ-J^XH5$bIEIlbmgWJMPq)M74;Ro%YE~po> zts%QpXB5A3qk@jBn|Abu#${({1p~1GByUELI;?GT>gy1Vm&2x4fE!2z2J5Y5G!&?i zM0Ke0ClJ*G$b~t}OInX0gQj!2<INvBH7l!9%xQOy7J5n=YRT`^mE`@7_nDpAAv&7F zGyyRV6H#16sdfotvXje@Vp79eCuoy=E28$N<^~y-yKri&cMZBRx3P3Wu3^)eFveb= z&W}CUF(8j&4{eAcp1J}IgPs-?C{eL{D7PUF6*WDnMuQFmSB}8U&b}VN<P4`7(aI;` z#+XaKPvm=DakOiq=m=aMcmI$e&~K(%Xp>_K2JW<`m+H*67e+Hb2c};@z}=T4!{t|_ z&d5h7!ol@9HQ`>%pTFn!dH(ng#M!6OltTIl6C9DRpmKH|{(L?Y!q*vmuOUBUTpZ#6 zwiTG@h$jOEX9sWC6m)WpQR16-h(DandTuZ?5B$VCbSz#ugDe8=x|y~M{`|uds@oin z6@v{TOTeo7K+VO#yQsj3T<I^mS%m$WHb5bVCP7c7Zj+F-oLbU8l|p>`OGWtMN1j7u z50%Y<d7t^(rhB$!oah%5*@@Q8b#0qAYjzn`S5sw|^`1Vd&rY~j3iZw-OV2sN#C6i$ zsJu&#?5*rU%g=kZcV9R{HXKA%nFXvG8m$vp2+>2PSp-&^rCh&XesXXC(VetLHtpY% zrQ;jAx?eY$_ZNIA`CQ!$S@~37zb|N%`GS}<+B;f+ev2+zu_AL8yY{{N=@)bCc<N9) zoGe>H;Lz5mx}q#GEK**Ym6a7>=&nEqg*_{;_FV%F^KJYuof_sfW?va;Gk*dnLKdy) z?qb!o$Am@%b-+~gAx!5^n?(XTf!&l_Z1K)|1dW^LMQVNS?y4g_KU9`Q0wG%@fh_U_ z@|w{JJCOyC!aAaVSFaaz=rI?Z+WY0oS@cNobU>jnzf&@%mT`fNb9b`cT<V=blBKMw z<_EVFJqoY<ji_?$gYS4O0C^d!aAMiiG`nbKHgXaUoF8kgyiA_1;gW<F<b=?+o!HBV z&l%&>VXJ5?LwgDy4(b8#FBv&*l)0F8Hsr_5t0!5|F0&N%z}qvO-%ATbXR^XVu`t}I zsWUgJPXPj}>x3k*M0mLf81AA1m=e3=Wz){Oq`O88v)X=Q&T`S;&ThZfetGLVVWy96 zN<z9hw_i%W-qhY+G2Iz81=r+$v?H;W%FNYoy7I<<r2oh+msQjZnq%m0=iX%NrI$g} zhR%mu-@fzW{sImEU_dkNBK2-ve)fkiET0qz#wORdBcVnzy0`sW_AMT;Oz4s?B|Lv0 ziS9L?tfgO|$+JF%82WzMd={JZ>6aiSIR8Rv5R!8^p5N^kF(ou&JwW#*5+;Je>|;cL ziHX^LNUqXxnpCrV-l=@pX{;md>guN*lH$s!K^V+#+%F?O##Hr^By)UAUk5HK5xk|y zDPsd+hq19C3`XNm+ydjzvIMj<&jgiH#uXKrA?Ed==^@%G2WvmXE;kK<_Dm{zK>I@v zhu@AKH#h1$i;pkJDl~o@U{z~g4xYvCAYEiP`yKPW^7?qGfc1vtedhStxhPVUhq7&P z`rZ_s=#2+@FbV8xBZ(3FeaAdb;qjE<2e$wTae<Ylvx(P8j?(fnh-KgH=sEvO<aQsT z79Srw^M0;uxvJD#;UB}3!f%SQ4zECuRvudRe>yyiYrVO|aBe6uzYYG5A#=O(BD=eu zx+Ul3k6DQc&N^)0E)AJ9fdYZw#Lx+mNEUhEVnk3yJ6q{VLisxNsoJ;siIKcpFW>-8 zvGn^C^wBGJp<VlAYRQD*Q_zt3VI(3NTcp+;F`Nndjb-TC%R9tBbklg$^f<v}t!!D+ z!z;gD#+>4ZrW)yU{umwWJP6dq0WErRBT}G^Gr;a3{iL<y*H@}h>W+i!M|PBVx&uot zo?jDtgsC!;z-)qBkT$<kKGTgvU_@k(V><4DzCznyWmGwJb16HMp}QX`oqK}utG5xs zTge%E&<G7SSB3WJ4VvY(!P-nTFeJV{SoU#?OWMb+ptMxeX?KzZAl6E(?)y^Ku`&uC zqG`a91JQX>ss(ZQwxEqdHPmfY8I+8aDfc?6sHx3uqcH!2BED}{<f~DsmU_%F=f}@G zC~AW7uKbgw7JtO?J(WcVMm8vu^mkkPU#^>H#f&)4f1Ms}_#N9R((-kXJTnqS5Yh2= z!*5@LmY{j$Y$EN`k2*PNO6GMzKiS(cqR~kv0q8s8;Rr{MZ+p#Oe}C%UL!db<59)Ep zNi<<~XYG5;VX-cdMR+q3W|U<q^xKKe@tLlsmRT@|9zW?Cjt6HI0=E(Yrzm9AH8j>_ z@0*MCg;@g#*fw+W0ZEb(!3NgWpNW^2Q;?Z0rzs-K`uHu=OWBzmpT~%->Ix+qW29+X zyp822<EzC+4|%9V@&T2!p`Wfx7t}Gk=1&f>j{M}-!Z!NNZat<M!Jp5|#=1klHS>E) zK$3mP{LJaGk)o+Y*oTvuBz71u$?};ChBQC}du4Kev@mhtV0&vznKDbsi$_ewYK~N@ zzaxBc@%at$_iepSbvN;!K>JAW%?<I^B5*GJM3(A%<0mcQQ=vn2G4guhj$*aSrZQdu z%F$tnEs}{%o}mCtue3s9XxxK#x1LO^AO>w0=zVj1lP@tI6fuW=_|^UmYSU=qD&nC^ z0EL)EJU3M0eB7aW|4jg+?Q1rV&LS>t46fpOUS&KuoB)0R%;$7*--cZ36+5;d8i%of z37y`S9{6gLxaIP3(5cYA`<u5Y8!GzMyHrg)XucO%jKKUwt%;FhzhnhKi%1QNq~(ws z2R;6t)qFb5CIrX@D8qa>?(Vkh&YU%*qSr1mcj6FNEwU*>{0Id9G-8_FIMdkeXJ$=K zE_-eXV=g<R2ZQpQ0v`BVBwaY$@f>uouHwP^ZxfMbpL7TAdP4W-kCiATDI3o^&NqL( z@<Ty7Ypv1SU7P)C{8()n`@*#W?iU2nEA$2{%;+i_4AQ(+m`H6QNj80-&nvrZ9$%_I zWt__T>&VwovWe2wJ4u};5XV79%@;FhdRHs6_v3Kf=M_kXZkT2gGQq;j-dQFN#c-L2 zjg+10CZ_PXF19DgS*n69-%V^qvUC+^mZ#_7^)dY&e7t=`<!A)0C<H`WQ#^TRQl9C| zxFGeZ_{^qJp{Y|#k9{`*e1WG*^U~Zclsa_fv`41)4#=?OO@<~)oQtqt%Z$`f&jWkG zOPd*v+PVIsjt0V0N8ta_xkyk5f(D!064xVIpiD%qEEFw`a2H~e$MkAc@>5i!bSF?1 zHLCZK)kDf1LhMZm<+ReDU!W0m4#(++ohQU~A;r`8r~SBk4T-WHv6JVP!&^PVrBEz% z^CucqDx#}D<l7#hy?oU^1G-A}2BQ(Y{rBSIrxR0C%RKEaR%QhnhY+`;$)ak@K|=0% zx|!bKLq|YF=hGi!{MyDPTzKO_auX7!zxX8QLIds!b;I17j9xdvTZ3GYVOSZuiZ9c3 z!vsttKBSH8#p#6&bZos;C)-}&zbwI@upP9MVC;<|9Q>e2)*FTYRVnL=O+#H|xYd<| z#Z#ime6>yekZ%XI7kIPF_zM{2)0?E6>>nAcZc|5XT&BVyiNneqhYSJN)6+9^#~AEz z;KSTk94AaDmRK#Bne3oSMSzwiNdiuUIzvVby}|5#QpDw016DR`KI2RD7G{D)Lo%<; zeU^DKe#j49eg4=)8CPvqWa2vA)BV}NXsFjpmK8LG={p?>Y#BKL9~91l40B)r6eE?8 zPMqP{{|H8quxSEZL2@cuY{~3OmXD1+FmQwspIewQ+2*EhVF}MDH<~2`#l@TFWM-)B zlo1ejx=IUHZ7kdRaife!^dZ6Cy%j!*S&7JKh#@y-(U=(V;#niQ;CiX)^^cCG%JLPR zyDX>F5|xYA70?Li)gtLgQDFO5zLvh?`J&6HRj=nrbr!M}L~}iAuiHt#??81E?p0K# zJ4nK2%`#N!7^~^&T4Z8nA0GoZopb}tgV2Ht3G~ir<=17FHy8)71S~*O$)!>`z!lH} zMJ8npG2#s0&gsQUI5jk+Rtvb$zB=s?k~Rt<eF?qo)~1m39DG$*!7WxZ#%U&3^H!O( z48AMquzxDTO<hV(88Y9Ed$6ug&PX;+<0$MUTV6%f#}>{sKlTLE=x0Q`k)g;B<5(Of zWY}VE(x~C4mrYO#R*tTs+TUA4hy4;3ScLX)y|i!Zhxp2QNsOUnJ7+Miu2}6oQbC3p z+&j&Asws5g8A0o6fH4kN5V<zVkDwM>3<fnSoubL7MNfskV07b;vrXsdhIxZ+fj@`o za!Nr;4QpS(eH&85J-t6e<>!DY;-pjRS<3d~$2<0!pAvPSoe-1O*5fBt$tB9PCb4XM z>=Yi%_3z*Km?qQ(yA)P<zeCvkbDo^{wD=Id?xi$|?IqLaM>NMRUXJDckXTkel@u1& zAm9JywK@<ARQYoWh@~&1%JD&Nw8$_#kaT3QA9(SMb^6&}@#1Vm1&D}<RDbN%rnXfY zla~dwwUG<3tF0WsZ}MFiV0L<gn|)};gOjsdw+(-+FX#bmdqUSEKV{4}giZ}F8rWFq ze{PKVh|*E8`fYOWz8wW*sw7|OYG-S?ykM9_uW(Qgr<8@SEv}6rr~f=IXSjY;E?fG> zpF!ys<axK+Q5+H#KVoeP7l^r@4uq(jiDS%(ns%1f@$@%r5gCceG{Mi!pp#;cusv0Z z=-&OUCT!T;K$c@D&5r(Z#8cQW?m=G<Uv)Xm<>RR|Y%<1)Y#<6~ajKOT@wWd~q5(BW zjQNj0x#%>%Uovt6<|CL2KIj|ZR{y;Rp_-^EuUn<Z7DlECoca`GAXhN1A-I&5nCq&A z`ZM@h^xlPYMP~ydBVlpQZm7JnKzn>OOh1JIDpbfp{*{0YA)ZV%+&1#qx}e}66O)6* zi6;yGOftl+V7*s^cd4z~Uc|Me7J=IF>dwQxe72QXio4_)>dmE7Ju|heZx>-JMaoT- z7uD)T>z`M-jd>9!w7<Uzz2xQ<;L>)=zNr338^3Y&)Jk)72UGDt-+b^u;k1_FW|5bI z*q}peONHqj^3LK?o1YCK;I}CO@mB8geR1^Kb`EiXp}S}aBS&~O=xmV(lij&UB`U<= z0&`iG4qwl_huC|*0-8OD8ZHq_YbJD30c{iB3;eO^QqdgkMOZbc=;_r2X8y@wslGST z6<{Ul`jKX9-KMj3#2)KvP4Dk*h@lDY!;QjG!78_OijysWH+^h5us-!He=jePXj}ss z_MBqE;<1{+oHo3Eh1sss5{-aVx(Uv6pAd&=vc<FKj~d~j<D!pTSxhYTqPe+(t&|Vw z61Ese&WBSvwj6%gwcOeO`mOi<vRW!HTJuy6y`cs7@YMUE7LXX})$nv6#<ZsE^lX5% zhk(%U4GtW#E&57s+At-G=%Yv5?e^fKNq#n3q#(CdgGR6+rHV<=%LXEjMuUZx`S!pK zWBk~6?_=%#DmfZhMFpd==GN7l;6?va(C<{*8*@Q9y_N5i<akGBPx<OIPVh$!=iyD> zAT+lYC^?-Pm8$p`FghoGDcgVq)U%;LPrMZPErodz`W(ty@8>XSTMYh(Rrp%K=c3+Y z>Apir{p#GVgRX)2Agm*2Uc)#T5RAY5p-8g6;5i8CWs;-pV5CQ4?emn|3X++OFJ;>5 zGx7Q&KwiAhevOm{<&GLtfQ<tN|KBz~k!F;gnr!BE{hguUE}FsWSJl`4ys87_IOz0- zE{nhh@CgOqkzgRwe1na>G~maFGS)7&NXdUrUYZ?RT7JC)tbtxFT~`tA@DlW{amLit zP_2}fh+x5XQ4am2w%8oxodksV@sGaX;ZxM?qNgdd4O4Om66Ml68jD<Su-*Ti$bN*j zi`JBLP8H`%(_7U<on8y|5q}1(nlY^rUbgU1eRK2oclP_r2T-sCDmmwg)1Z<wE%I#5 zb+|WpEdcy}?a9drwt@x}#qsQT@bC~V#FPdRcZq`1X@uDFJj@GfajRfgia{o*dQ>xF zYL(I2>xwh73$EqAE!izln*M~FUp2zUA-KWN3@1`_f8{EF*J5-a$VkdzO1`ppFLhl6 zj_%;~D#i)3^UzOc(I&Vkuo9Gi*FoB9V(q0FE-{k8Kiw<6uL=^+z(iFz{9ZdG;PmX> z@e^FLq!<rY3V*SiUHvy9E=dlW^KSm&)jlqDGNM%s;1MABQ43|l?E4+kqz|g|lEAt2 zNBA+(%EqeP8R;3y>TT@OUZ;t&Mld0dMFk#-dr#w0D<zpoWws|_vShM(;wx);#g%QJ z9i^h#UAwU-^_Lz-y}_^i!S;QohkF-r>~^^?HxEwo9R__ZBV(NZm9VZ19o6DHSLRjz zr1;jifC3}9q&TH-l1T`tfy~$$rKmd??HYKwJO3kYQ6N4Wl$LQxtZp{o1<A|(Srsp> zo>#bfmmtPk!5Pu6msw$b%=>DP9i$EvCo6+E$S_I=Z8cxkhx)r+o5Zt~N1yp>pmfxn zNef}4Mwzto?Ae(WDeb&<(*IOSU%*e*9>97uWE?1n2zJi3Moi7pRxA-k3(Iake;JoI z_yIQ}*fie0&*n4)gpBRYQ8^|PQO=6C!>AVd^xgyDIOf7-@x(5;K%v|GjUNcS;|1UT z6(&CdRu25-HHe>$dz{rKE<Ych@zafslY70svCs8$z9_VpDQF}nUe+G7?Aqb2pmbr; zQ%kdg;`nF@e>yycBFSM_Jse*t4@XPVs3_H;@J=zQ5G4~ElUQ2a!Xq@`q3a0RPi|3M zSX&<9<k{UwuyS{e5ko85?(#tH#VC5O4*A)iPHfgp-&nVn_x|NC=zY9slD^>c<gl`b z@g>2^Hsl@&2fAUagyLaFh`IJKJq=(-fKPwW)We~{CDoceUUVcYD=j|?(0?~NJS~9? zrWW9BUhY~uJK8HeLN_&WS@z*Kn`S0-JUK2=RLKo8+uAhMf0SUksd}&ARaNn99l(sF zn<#k~-)c?--sH^`SfX9jma<%$))CI>ch+^EE=pOsSY~8K_#;xn+Kq-YQq~Cx>QXN@ z(bP$a;B_MBIJ19nhaSDUN}o)8Z0s`c2RmtoJsrHKk2gMAJ?u|#uRoJv=J|pD1*VO* zieaUQQp9~3tBr2a>ezh!(0*`z>5>upY4<mw=kK-4G3|;yo~-E+sc1QLlm`-b%$}`V zXG&L}(0iwT(m%s^$xEg+z^yA3;vGwISxE+4C46`aOKU$-Q7GI;J%4hUu)gF5+SmC7 zHr`+n@Wc)(3>y0G_-$tKY<=l~q5gA<cyxe?yJO^x5cf2iFxDBN#C&1JVXDrHw4J(& zi{<(i%I-!pY)+=Jp`XS+HxIc|T7hS<8t3+}mkk;kHD437;uG}tvkbIIWRFZw%M0Jw z`HYQ?Uj#$UE|oXex8WoeyF-iNfxv|_8$dUsHu&zcw}zvk=)8{tW*8S9uFPY2rMQXf zFY8vn^yEHl+<O%e4h`|t(os*obp|M2$LwA1s~y*SYW<`p?sR`Qy||`9I@dJB1bWWZ z`!Qy4bG-<tmibsf-g3S9mXWFoVp}SVR~AwbWWj#d&d{eaL*nA?E^I$hZ&PPE$3(wa zeujL;A8bEXx3MgJzz+yrMx>Pg7~8~t%DkME#1T|Jco+?-Wj@oqek<%4uWs5VB=tq% zcgMjJ+P2^9e%QJ`AWXfU!S8_m<>+I&f7&JU*iJ9mn$K7meH&laAIh@MZzrHlT}hZ8 zc*05yA%c^pz;2KnE$|v9rsw&OVQ(-(gbLb`_&F2OLN>CGLCy+G?=Y{xE-z7Em~H#i zJrFNG2j|%jRyI>+4tKsW0lF}r*B5+DOmXz=TgZ-Qr}ph(H;dFQX+uM3IR)9TE6(#G zM=;E+OrhKhYza)qM=adyZ97*2fp%Z}1x(q1d;nmG`-}mYbXRDRWagVCo~AyVZ~Ir5 zn3b9w5OvAm4Jp2-ogO2{5QpT;*<&8lPF9(t@{RMD6M8eKXH0Z}o9W$~Y1Kb*ScJep z5*rJ%uA8hGcgRkARY~bkKPM~iXHai1t}yMK+tDeeFtLlNeKL21V74rT7K=MMiPvsh zA2|ft?%XjKd%;t;N76So#{4e+PCS_KREyCuZpzfr*sm~7c4sFbYkEi=K{8xBxG&}1 zZ}w0Mesh-BoaAt9T0EyX?i2b@H=S>6Y7P=j;_7WhtiF}WJLs&N9ShQvS}XMk7ax1# zihHqdmxf?Us(DIOyaS%fGj?RDP#pe*=_;D++&$2@*iA@%Y<&=Xw1fszh<mzClAum^ zp~zrqVWFfhP9rD#{@?&Ol$DcHup*_TEW^()t7l6e6|Gv&PY-K-cQ@~KLw?+F)7p>` zKXs{}Bz+}Rhxg&xSiD3-LA&=>MpF!68%$H$sDp)D#cy8jqhJ_VyBvK?JAZV>`f6zc zCW4#YD_%_KgtSf4?uE^x8<M9Z@X9hZ)qm{Cr3*yscwW=tA&5`rrrL=+Yx{OQ@@Bdr zJ+c3mrR(BGr?tZUmON`6b|;?Oa-$h={(dB%YfNG~t=$_nsGMBtxOn|}ZXY`AS!A^) z{|vUto}Y(?hDM@kY0XxBXdoR;^UhDBoRq?a3<*+LehP@mKImFVfQj*PqFs7PlT`TB zW4ZzV=47b#;THXF;^y1-9Z2amn?0$z^SJzR*%4aEv%S*Nfl__SpCW=Ys}I;llEG{l zh7Vt41amg<pu02)vDPsrVPu?P2e)}0Ja62}^N7EUftn0w3R2GUcplX6bmpeHpmVj% z!CvgWe~n3Saay~l8>V=T<Xu-ihj`k6je(&w)PfhEl(Zss5b<4hb92`arPJ*oJYHiu zJFD`0j@c#;Dbq?PyV>_1P>I_5^qyDxw;Z#}1LBg1=-|XRLa<@Sn}xCv^>3?wk-DMf zVjjPP7sB-UKw0d7O4$J)@9yg>(Ay+*s@7!2!hnZt6jOY(HE!p*{oMEU<}4O)>2*yl zet!RKoryq?rSG3uu6(XU?@q2fN+)JS5J8U1b5jC`+=N=1>;3N`4MGhu{u*6bLK}qf zrunqp6S~*k>lFEN@&mhLV>Opzr-jo?tp37Zi(g!A{zx=ong9IN^3YiuKnwQi>*J*t z!{Ck$Tbt^e%|143Dwv$MunH3N;oNu&`7tT)X|^}5tE0O51;2rDtJoG>Bjo~TX(g`& z=_)}9)Phy8?^a;S3n6h|fi1<R4>WbkuR+O+fPzI?6_ub0iP<y#=DIC!<h$6hgqjuV zGxIaw7!*dJ^18?0S>{zi#Q2~~5nDenO+?&n_xv??bxfoz{zuMItFnW7vGIcWpD~3; z>j`^a@QYSzVj`jyLnGG<zvB~U;o+W{Ykr?1jGfgTs6p8zkKmQ55dq44*boiGjh?J< zPm4C~45_`{fJhjKYGo!{8fiM?`b|`iA3TZ3ybqar`^0@S--Ld)dtY)!)<GBQga9dT ziv?Iy-Hozg&(|?~Rs@9%S%{KeIBQDWuENnp22HQn_jb7)1-wL*Kl&{@TMm+3*L{D! zlG(xcE4_v8mZZ=3Tko72t<u3~u-nMxF2hNo!x39?MBl^q{zHyZpNgw%(vA<P+uhKT zLEC0du9tXU{lSZCk@|^KvO&woeDFC^FFjK%GFq~F?FEl+p^USKKpoEJ7^RC-^6X*p z8n<zV&<!@%r)-Ouj3|p_fcX0hp^endyziCybX!T7Tmy*qAwfolVCuCuB?AL+W1}HN zwnX5=_<$Ht?qucG1UwRS*9C@$S5^oWj8(CXQB9_1X5cEe7+wLHKY*9&_v^uK1iu?0 ze66t1DTHJJMwM~cm+OZoX6X+wBTQO$AO;k7ts-P(Frhr1ieBj@mXwHg2Ct1aQ)D%- zsY}NKkdnL=AMa`=(I%C&_%!0_Tis)}<fBctX3B}$O5T2X58KuvF<Fv;ezwk(l1EM) z2TCDgn<b5Ksp%Kv^Wv1cgVT?Q%P<c(MSnPi>!tt@8rc0{v{wW>|MQ11f?v*WKz4r> zj&&`R2C7bwDX{5MMkN}7AHJ&E<MtCbkAUu$=1Kf1D$(gUlJX@^*^+bH1V!Az4&7%B zM~}g4gi@9^O!Dz}FETtaQDGxb#)o{rwAmD`W4u{{|9bcedzpO9x?}0*+_Z40hgss; z{K9u4)K<dda1ZAof+Thh>lA_Fo<O!TE@}Jnz}M};Ou_r}rQ<@G#W_famR7$hJ!}ft z3^~+%0|CB_cB>cIjQUm@fcflDoYSz$!?>_g48!-1r?Nc@zD1W4{L*B${nBgr1-w+Q zxA}&9_^14>oN^=FcA5@gf7h2YpP3(d0!eZcDE#JW05A|Pbxg<>Q_w6vJVt5E3Tb^& z@CQZKwZQARUv08gC|6{NUX#sD02r931>Rv7bx)}N!TM57lvP05Ef9JFoBQUL0<eq= z<18lCCEtzAa;YMvH(9zfh6pRxr8dde2h+UpRw`(t1F-KFI?PiTI^lS87qsdLFW^3P z5!^{ZnRuY;#G<Fp+t8jQ4lB|?UZ3SWfNXfBrxVJ$_2%N!d4bCYpitJWXU6IjOYh3} zJ<v%I9EPMYBFcLh14IQ&w<Jn6YyC<PK0%<D+Bg29DVZ()-0c>i(89ME69n-o6s`08 z1KRF67T5cw0~JGVNE2lTVj=wmz?nN+02R(_2*#7~;*PuI>l~T3{RDY#SY8)M0G$eX zd8O7gua?<Hw|w#&37qot1HGC9uO7~W_gn6`OgyXSsA{xQe~hqwo>Do(t1KbKA(9TD zdN`_uev#bnJ}yrKd~x9_nv2k+nXfGbS6IsBo?W}=N}o8!PrJnBxvQ1ZOsR0^SAw>2 z|16vf4Ya9PwEpP-T)2D_ykc?U3y;%h%n(BYcZoc3?flxo02$KWx_$~>t^6SJIe_3} zaDmsE;MG-VB^C*v&^k(O&+{VOw^5Y@!ek(ih>q>{t0E8cSc00Q_igl7WgfxC{7r1i zVw`+vTM~}Y9&)&UYUU%xo$&Dh)-(Rm8*I+cm^k`{maduH(5$;g%BS1IV2%wnWkR_~ zvP2g#r-#F4sX>^K)jrfIzocA52LhN@3c%d{;;aE;<-R~eJ8^^PhG`q7Uw$lFw`-=H z-sR(D{&2+hU2S{3Z2|HAu-KF?Iwo%??;C{;^)B+m)sCqLWgxE57W1{eTG=8KwL}e2 zcXJ=(<F#&P_8<+3&K1=nNtSBqqK?|KUvZ>c{N_V?VcivSH_>G36S0I#ngF}h>D}uV z00w#lQ-m5OCWfDPKK^@gs7`<p{Zo?P_g?_T66p<UsN*#$24wqj)%L|Av01(p>Zc-4 z`2wPWAV*w}Wr&wlHcu+WJS5O-*;p8T*mr#3_Z%@loMZ0SHWqM!vr7Btx89p(FW=9s z;Sq7C3Z_einX;|<)Y}!Sc;`L^g(k`k?_TlGU{V+p2Zov$8iDD5vEizJ`<FtYxHr*X zfFTB(plQzZ>T&N>=BKF{;-NM@LHWaCg9jn}$<wi`5@b40)O_uUH8*g_&BKwdZnK=b zaoG4C!1h1nL_FpPq+gi(7&gFt63b`O^7?MWYqyaZGw*mH(LUZA`W6{Rh0X<pdL4ik z`v830UmO4+nR~p5zd0)TM)c2u)(@K~1yC^)<XCu&=}XDK18o_A*6fdPH^7yfYB!l; z?UdK5!6<q<kUKsEhV{AHZUYyOFTKiaf7`P4=sqK1lj8hCWyO~U9Yzorr-WKO#5gW= zuus7O&~^2_9fhHE4@9e*xHu{_!JqzP{(qWIxqlQyrJAMsx(NfYCV)QdMuB->y)|R! z99mq}?0N7Gpb`W+1A<(<hhi;bxs9LTvA@LQL4tXYFe0tB1|i#mSl=%Ax4-qs)iOBA zOQ{sYG1`5E2Jm$s0d47d6_D9W$U=>hR9zUB2h@dodGJBfzJF|+jOsJ>l>XLhG~lq~ zf0CAe1L>-Fb;PJf_q^X9So{!4_Yk-rVT>mAsWxye)@(QK%nD}kl42qme;G5^P>wm| zSf*HRs_c*OOe~%K+?bLKoHd9FUSOe#HNk(s4UOs49iI2$4n#!@8yqw>&G74mU?swD zXiK)gmF&zemXMquLtn#aOS>sR(dD?-0I<|4IsyVGB*evnH~MQQ!R1)$iBP=V_Xy!- zz(xLY&}6(RfC59Ej{TqAm;b}w0B+4HB!I8F8mLbr-Jn-b?di`&fD{i<Q^OtWFD|az z_auBWK0ckAmXjMPRg>tor8_zLxOf!rDjy%gCcyzP$H}x81HoPiF-0Kj1i)O_$nd^; zgFD<Cg=A$k&{2KBYRv(<*>2_E;|a_{&gBih$dC1ZRBq7>a@iYPB4<1~8s#VvihV}$ z4CvV`qdO(z6EvMY04{E{K28PBJL!h_`;ot&kmw{l1~v~Vnxrc-QEGx?8@Q<9-P9e> zWdb1k@Q>vKZIFAe<4?lE1Mvxih+n*f*LuIp$;wF`p(WyAWtAdLnvG043O_`L(K9U( z0cigv{}%%G=ZC&Cr3;Rs!<HfP2NSMw1YGjLAE$WS6NB88bXi0!f6~Mk1%<0VPT1FK z{IbrQtc~u@T}qy;0aHDH$p`8!E#>U*-hBS*fwTc?lYH=bA8YHTI_-4uNyipB)1rNG zpzWR=EpKgk>=(mYAhWs`v!oCbv{qi+y83c)nvt$%<R<@+_!GVWFO2a9E80OSvbifF z)Q2{+IkAvahU12%u_9hHzQ=Jt2eKK>ee*-$o`w7?VAy5o4qoWuOx<nOB6aojrG^jL znORw;jh;Amxm+d!SCyLD+efe=v=yA?$&r0^UedaI9_ugmYX})0+5Qz>UL}$wzf|Fi zlBb4$kR?<cl|}v7OPuzh2M;{~mi_!Lg>mJGE~zJT#S%9~1CEF2KXCbuQPAQ-ad|F* z7LEF;YVdNG_)a(s$hP&mYY<%=_tfE;caL)GYQ<!MG`>`hLkuDRp>-GR4Gu?6IKhH( zd5{2PSsJ7rXwcxWae&ZVu1ke#-gV>tti<c~jib?M%i6W2?mg=bH6E6mvkak|laH3W zBB-j+hgus<VpG?L-6Mg&ckk!M`Md`RDL|s<BL_7b+>R$Wua6~NeNfYUcNQL=U6ZML z?Jg<W^4eX1yWuM^u<Jtk6E!8fxA0i$0YaexfVlY-qPm9D?~Z~Y@nv$u-ul5QG)BH` z<_QajbE~|bk%FhnfkN_ics;abx(s99pE2Z9)hD@*XtoC14GGTQAkn`*FHuowpo^6k zBOliPlC(}DuhH&uNzV4DxRWzFOq@K#`K=Rm@(&&}QmPp11P`JckK?Ojb%pk@;z}Ge z3n5Fdu#Qj8(^=gk;zxauTj4XpZNki>6&R?9`*Q++`%4&eN_>M@*Fe$Jd7sQTzYxCH zZ%e+%XE7E=RoiJ~Xa2Yu&L~Qn(&kiB^~Bf<Vi0Yu!#nQET)S7teeuIMk{jl&dY_X2 zPJu~4f=P}nb;z+cSp>niDs~k=%8}02pM3^mKnJZ5?V2I8qmv`dcIl=b9J_2gYyZaP z0J(|tR2%6H0DCHsG#;f=xIPlHg9<LApLdd@g$ifgEgm%WV_6EhGtu=Sui<$lyU`V) zTRxmTiB%ORzf+sE;m;nQ0n7A(h1%Db*kHoX@Eo_7JbCVU!cD@f_>EC)I6BhD^VW?` zmyj7xY|h^C;q|FES2u%azbbz712X`oHW?v_HSY`7L_}`S+R|#xp-&Fck8!lECsx<> ztLZ-VXgTLA-t&kPndf)=dK2z^I`xeugs12gu?Pw$A-ijzGyjJUAi&nbsYTLA17LYa z=J31-$A#fmHba`Nna!|q;g4gjS6j+CCI!a+bKC7s#fz{0E1rb2ydDL{KGuA|JWqH& zXsWkC3AJiGVcg{fc{!<sEcW-D@DMwjsh!!S&&uO-{$rxg;?;BkOYu(5Cmpt=1m_Qp zLn%m>iiA$IIYc^$sHV289hOnr@t73+dI#HuRAwG>2jyXfyV?h`OTt6CfyUgY_kk(* z^xiY;RI!GF@n(510YlO>or5QWBPA1!Q1#AT<I;1-KHlDhd}}wZdBae#2Gv5a&m+L3 z%U}8;v|lIz2Kn}jFa;v(=ndj&AyIxqBkw*YE^EI)w*1(+9B7Cd<ju|$*Hc<q`pfr{ z@>c{W#zb?KJOdebbmUI!wI}l@q-voWX{P?IPBtsbfY$eYE9Zm(bTuUWdj)|2wfR>a z^<QcFsEN`<Uq8akjD(JTi@MCwVX&`{xoVco%*-q`rtmNeXapdfjzN|$gHM>#$$ojX zyqaPAR+^bgWLi6LD#x2mD#lPso(=$lUh5tuiigl)u`U~E=vT8@VNp_@^kz>S=$JQ+ zb%&@WET~pT7JlDkTsV1XS53OzEq;JL!s^~^;@QNNGQ{$n9+QBHXKg||zkw&8MJK)B zS2DPgLTgqqGBQ#6j9h3@j^g!R<%XiM2l14rO^8uVX4a`K?`V1#qny0CN&D{N7X0rb z<bRY#*F{gqiMx9r1ipzjwWk8{PtSVX#V2P3wTQ>RAt55zm?h_E>U#s?<B95>U^JTG zYixY_nF43((0XX!O;~v2X%j_qr5|m{$%`Ml*ip>(HRp9cck`B}nNsa9s(fk&Gqcf6 zJ#D8j3l1XNoRry_EPBA?;Xe&y;x1L8CWg>B_lM-dia2kS_o(U6woj06u4_&&=iYvO zUT5l_=dEM28Xepob~5{Ma7?|kwyfih`=#bjMkLt2;UtT;qJ%2;$?=KuB|>&;YA)C_ zp#YdndaM_cp8t{762!k8iBB6ef}!B0NQhkfR0#+j^nW|Rf7Col<OEtz{!01xeC(fi zKbTN~a*fBZjgMGz29rnMLpWhWtnS1IIvi>SzC)H)Yw()})8goq_<+x+94E<izy{UT zN_#_?tWz`c{nkZ?49?WDjS3RK(b#7jm3y15F=kt(c0KdkNy`1|k{_-fz1dk}3G|#V zw{jN<ftH$eYr|-g;ikq0ylfWVQx53dI-tNToKOT^<E6G#NO+UpSKHscVv)K!mso6U zC79vF;J+KxFOICp5*;jBiR50MoJu|2=8Lx=d$kuFAOJs~74nSNtnE%7m$1IJa@(56 zR~)=R7ouICBuJ?`YBEO=3OQ#k<Z&ADcOkG438wb9jvFDba~kwpc+dh@zI-HfryKGM zl{DL@GA5M!TFW}jlt6nPO_=vn1Qz-x=gS{NEptV}CT}j*ZR_wy%qQlQFZYn~CoViG zgxxfq!KJ);)ZEfHpM)RdrIl9P+|;$`UQt2yfkNQdTGsuT1HPK?8M7rwA*CvTOQ3JP z#g>qbiFh^l0@Z^53+drwfoor0f#lhztt8Jp<OjMTEa$>Mq_;ld?(O)1nCdq2iaItN z6=E`y5FXE|1-@X)gvchq45rkmj-msjD|-Fueg;vq!QeOZl+x2mxBylNwcfiK<~*U) zZ-nD-YU}_r8Pe50V3mZVp`n!UUaQRBe5Pu9;Xr~IS#oz89f1X#SA(v*&~Y%NuCYX1 z+hVAqM)oQhV9MysOB;A4fUD+4A`T*Ha?DKga^h*sHpQ}o%;WG3tEwVrW?AS*{y)av zIx4E}4I3W18-{L4X%M7yKt%*ZrAvmAmaYM5qy=e7Q4G3s=x&hi?v5db`Hs)?{NDFl z>-*>13uX<l*k{f@``mk<>%Ok*zRJ8@eT6PN%#CQZ)2|%ANFUJH5PR)A5$lzl_D!{1 zlSs#`pJDuB!L=mb$?j>R=LSG0p>~P!;ID+ssods6DyOpDzY?$Nir5C^@!c7(N}D^i zgt|({G*RnDTM6D^EmcQezCC6%rqrAc&>mKc-6`F#OW`TC0AJV<`i_PpR!FdaEVaaj z2IbKdG__8$+>B|??75CK<5F6D`Of`s9{jZ#8FWGa`(=TZ$tA0y080GN@axgvuIZP= zZ8PCvr%_0ft8a>vPp;n66BS?Xg7Gu4z$X^{gARD15K^<ZHu4N+?ebdi8^P%od9T`5 zcSIF3ST?;pjsU)G!TnQ$78_gy5N`MvTG{N+Q}4wR!_-u5-}!_Aw$(r=T?@W4qjn;E zp@<^JGFGR2KWM5q<H{f3Ee!c^P{-cwrc-UrT`I0(c-FL@xyLo<hWPF(cW-H3mU6fG z0$*QB#a7<B)!HpS;74eX)d6ZHTkeEW;KZ1|dzy5(yzcRn!wVOD`$49nDT`-Xh!0be z95DF;`IdmpHA;gW$!|=nMw346OC0tgP0fQl<8!`>oi>-))z&l2L33&VI_k9y_c<J9 zd!{u}65y&DjnP#~-Ps^K2xCCjGerfszB%_7g{EF|`mH?hrJV?Eq#Tly(*Kv=W6Ysf zc`%oE6Ur;%&~N-c1VpOY*k%GDj{CBP(|!%1<ZBy#daFl+|G<WJ$rcDxm7M7A``ur$ zb|We6AI)M<&2FA1l<+$Yj*l2HJKV@cXI!vMfA`dbO3t_UZI+$d0mxo~v>v7!kWMQ2 zo45mL$%V~jX<X3modZJ|e?GU|jUsN7;n-9^>{y0zO}w!=Bumpw_a|O?EF{zYwUr?0 z)hQU_N7DSzgkyWDAXs(@r=;mN*1|r*bERHdA+mTXe?tMlVqC)=q&E@Y)e&?YUv0L6 zV@}btu256nipcUNKaE(XDZ?3(>z<>94JkEKGJhjatPD=Np2c8U`A~F|O9BgobNpRi zKwy(kiRpBBJp{W%dv-3%#wj-^j#$q8Xij3be0BOih6{kpG{>jVfq&ULTBz=ZHR^o} zjmiDaK+(9q4PfiI{a0acZhPw+h7^ml{tnk_`AEw<K}nfvAiBEq0Axy>5o|rE``);q z8&On<f0W@FVU&2|%0}6L17)rm{-^4ZfwA!#()3)wp2mSvZH#WuyF{BUlz5c;vW<jP zfu@%^`VD-$G8*dCDD99Ily-C#G6vUztL_b#npX}ib7x5jJQ=MJDpQX9-t>R2a)_m< zZ1RirYE6-j1P&R8qJ;^Ss0>$`VgNK-$E&wBD@tKT7T27poJEbdumutU2xZJ~lGDUJ z@}hoF6iY+5YVclDqjp6Q<U7SI*$rL`8^0((7%O$di()ybh+)Z}{;_kX%T@L<T%O}) zpGloadXuU*c{*qQpOSLjtkp+Z^#6ClVaEI>8ZC5B{SeD%UPmu}XS}4pv3y<L!e;L# zTRW+CMqg1Gj+4CoB;7O<AKcXWzLl1)CN^HWmfl~lUVwma_E^%^xhalMAW?ud?=Pqa zY<~dDm%nWc0z7vDuzJBS?5OzHVwID_^jd1&q$k|C_s29R|FEFi)iZ!T+OWGV_LRl; zQuCH!LaPhZg45n_^u)~RIXp)g|2>M*o5>L$=Co?+o#mqD8@uV>zzP6hRcLM35b5ek z7iO+T(+q{YWv8-z7VV*L{?=cG&?2j;b#Y4|b76eAFV;}Y3#V>=NG1z_kmq?BhNF>1 z_z|Vn%xRcDYzMcyp!O%8J=va$^VRsqcJRlK%;D-WeV#+q@1^ZMR`rDY+m#F1`t$w4 z3=)6Fqqr=s-ejvI?Esl~C6`PGS1PvxZolRIIs#4%-27Lo))pVCC%A_JT;!%vcd%@3 z?B&w-Rln9(V*@1Y_tDhN#@em~S@k>jsBG?pHO2Esld%C_u<$A6XwT836jr~IOXTP! zldkEkFWVkdK)`$zB8id@f2!c`$o?@SOYRfnd!;k=URpV*_i+l|9B!4sBa!&A^4EVy zk&h_{x4+s0{bqLE;Gc<uxjP-<*C#CSXnfz|FNY0AoUJ#Vx`wPj|Jvgn!i2?aG@S`> zsIolOS<UylH=TW`_5pD4L)+rh1=%JsQR&;mOrf62O81eL<&m<X3ZU?r`vp<R&1n?e z$?oWDN{K5EyflQbzrXulqcJ4XRnti@CN>5>N6D}>H7Bn?k(lUL_6Q3d5cC!2QkR$+ z3hNh%%XJqYGVyZrf{(I4y?Ypf=cR5(KR7rHzZG~c<>PZ}>*KQ-Ol)aw9nSmf*UaMS zN8VYFx*jV}EpJPzCa*Mg%Z<;KRgBHj9O{lMx5wds{=7as?&9Lyc6V&9ZT8+LIZo-h znv@XJcCnSzcMzA8&$f92um)bedf9jD()-(V0%P~L2IA*T`GETjhkd{oT?Ku~lIhDH z#jdpzEejoj&k;UGhNjl0X&I(Dd8;q`U?=HP4fO>_%wb2Fsi}Ec&2q)R+<$%U<K^Yq z+OS*s<-Y3Yu~egcvRHQVboXG7nlUvFBQ0xDu6XV`G?X2qkTCbfcWbNK!CDpLImQ=% z6X_{!YI_}3OS7{+)`FaD?d%rw;@9(&MXz4JZf_nRnI1U$yhB1dYgkg49~ufNh68TA zj<+*y%vzoa_fcoIHt$yU;FZwTtE0pEA3q8{|9tljB45Ml-*R<*2T;ld&arB{wi8Dr za!Ixich;sVC!4FPG^+BJ`1u5KsYd05BMpp<$WbJ>PKLhJByQy)d-gSRKkC25ig#9p z$mf-KebMb%auPRO{6h_&S|OD)?nh1moU$pwteZ$VjQeHMZp#}Cdj)1pdWY49k^7bI zy6e!Shc^*BX)~S{xQljWdWDsZK@QbXFzT&xmR;(D1Mf=$Itm!$j{FHYXqzK3(f|@r zU02uZuc+fnP?J4>$0iJ(jyp#Ze@~p1mhyhCid^+`@t2Znc=2bEk`KUywE<!{?uUb8 z1No|q3?RYzxt{Mt(Ep+FT@t^;kI9V>5Eb=|vk6;U)(mT8@RXoIsVj=G6hLBIHND$6 zw<0HjY!Jx7Vy|87XkzPB1xW2&57LG~=J3J(05&hj2A!Umm>ic!^=H$n*u<8X);<Zi zl}9B2(yHl5wxd^hWHP>48mQxjC09^j@4K#wtylWS%8bUcY#`@nzvzufFmD0tcAe?? z&cpzl*UyszZm;D<9~@sJkL<(v;fu8ze^wdH>4S=Pc5Lfi_F{4Id3RFiJ!$BA%Bf>@ z0j^sBDlS_G0{J5y9|~G$4sF7{5h9zCN<N8v>T@g<iZR`MuP6s+A`_DH$?V!9vRU@< z{W=q+`iVWLnK_`oYZ-GXPHJ$VFL!g&qdMbY<HldPs*1>u@aTGD)A;>Ih0^?r#4LQ2 zf`Z4Mi3kQnAN@CP$?tCg*Jqv#%NSbo%_%Bap&S#_^UrWis_<}knLx|09p!>)5yxS; zJy|mTFZ2aXBBh*=EM7?kqh)yYzVeZWTpUq1vtKup#j6W)%R;fS!>$>`0%+)%a!s8U zzX9%aF2_<LQbVMK$7?PI*f?|LerRph!x2(-x65S6m0w3T+L+~mf;>NH=f8o%=G)YS zQY|p<Vq*nv(&nthZ<YL&MLA@j70lLSS2G@cwiUE5q?|(Zmo%GC@4i3K0{B&?XR2gw z!$X57%R%*>?p>0NUZ1;{AF3f*KH^rQ;b&RHDKG9`V(wh$1<b`;xhi0Odz43nWV<Px zzeC3^22t2Z+Z>5jxDs+LH<Z>v5YJ2$6KtHW&tGOzD7-h<2AtMHmgFTL4QBJG7e?LY z1>8&poTLzLq6KTg3#dNN%jdxV_)$AleEl8WN<C}%)K*e{1nnZjRO|VVonda>yLnOv zg*Qk8zW$`-`=nx|PjFWkSFL<cpSNOxCDp{_Vrz@W8P2(XVgKrwNb{v-TR`Spd*$o| zuHBk9=WCoNSGD@A>ICYXqm!RHr*Cl5GqcQ>)A;Qbzqq@35CnKM(a%-l@%Y^~Fc(KQ zzfDpk^Fu=_?<NvrM+NSb0a>2^mRZtFtV;|+96#3VB=YIAfOvD4G&8a+zdz-#hBcGg z!=@H<)TlcdofG@_@{E@b69BnD!w1G))16J(-*>*vBtxQPz&gPVU=2V9S!0GvW$rY< zc|WfpX3&RGt<+vK7{Xhz4KP5@$<V;83IKw<Nyw1N#i^becz=9WlEjbBjrp+1;Xv#m zI`v~p^iOl1%p;mp#fdX-SMbhb3_XA3)P|^bbxDptNj^r_27he$#>1wks!;y0m=?Xs zmxSq5)9A^(EE9#|bz2*`|NX7JOAh)i5+*O(>-Fq_t+2BL0dVUuX!K*~FvJhBzgx;s zv;sA<*hF;Sn>6QT!|D_*H>ix@xb_L!TE>QIOQ>@KoU$ITR$)y1rzFhU8pg;5dpv!m z(wM-X3Yhc<d9h?Y3)D+rc+ra|FJAzO`_(|K4-|Ksil-I^B5Buj^L1B&9~9DYUvhr` z_84RL5E*`Sry92WE0DW3w#TaN%Xta>Ho+v)tiIw`_|X7+AOyME2o!p(GclLm^>#G> zAcs1Y_DM}w9Bk`$DS~<=Z;NV$=dZ01>V08D2ypWjg!9-kOs5T4g?+Jc0zp7&{usgM zEIVYkvMfX*+oT<GK6fFx#lP!AN8~|;tfFhourtFj{vrq9zne8U*p^kntmsdnN`%BD zt^p%svqQ(EEGw@~$o#Ly-GBc6xiTO)!$OQ9woBM5!cz3-UBMAV9=m07a<o^xRShV! zDcMgd4p2{;zNhJnNr|g!cDmEbV-kQwqu2t~;#rh!`avp>R%bEDi+=wetN6{(HvnsO z$43Q{OifI<Y$o<XSMB6^_Wk*0*FW{L&FYC*vsP8roLTpt<D4H9KAh)H{gScu`<oLC zk+x@R15K(Z-wcUkVaGRPa&M@1EC8<z2MW<hgWio`qbY+Do^w`+WuQHaAH|y%<!E+t zbBq4OT=3Ea#!6Jt)R1SAQdm=yeJhnQ_+qb3P;2QpcjGhFGpaShS>8Lkql6iG&`>@M zet&39M=g(Ll^s!rAt7#MwTbX@S^`~=5S~nj-}rq>c&a&;J(`4I^oJ;F<5_bN5fiy~ zu@x%<YYdRjQrY?0nin+!NR_hU!=(m`k|dJ(pC!L|Ccg18yF@i>d~bBCn?b&L+3CGn z1t3h&Tpd4=`v%ZkkCPOkHZcY`g3$b9&8J7D+k>opbg$ANYG2Jpxk838JoV9{glf&a zDbcw=06E==@cFYT2x(Z0G}4w+@RtvQI~0Q#liQXyEr}QZP>SRui!PwxssbhXB&831 zc;)FyVrdCqpYWZIvZTj_I_uJU8o;j)b$#yEX=R_o2GZC8LeDxOtsgN6W9-fnoyLzu z@Zm2)YcLu*OJYG`W#*AVKI|4JT*a)suOE6CDTSGdHr?{Y3lnE&xSU%*Q}7VnZ#0XQ zt!fs?S9Uq{VwT*2W$QH)hdfN_D)HsxC>%7tAR4GE(&vm=nf3YeWl`M1czyH+b9VG1 zIdk~$o6sgi)>!$o=H$6HFoLZ-^l)B%m4!-fPB4fY!3!b?(#2GeH)#+E>UQbt@qxsR zWodspCc(5UuP`+eG<&^ywtr0iX~;1wGET)<mdt@by@pf>op{+G^|{sP<jd=N6x*)v z+jk#xxstKj43*w2ybEg5S(QeuNWfPLe#$msxZOI6SQD~hC|68DMO<@232|C5_;i0c z*%!=R>8Lb)7gcEUwY)Coe7<eBN9UbJk^EdX;3OTnVlu`K1TM)C<wQbqp`3QBKr}vx za!bsBTw?D)bI4h0W_p4__x3|H@u+8gHV%V*4umjg-Dv`r@OvtM?D8Hn92P<mvt9a( z$G!vsDG6zV%`s;!wH)d;)OvKc;ezlLIdK2^r}3<h_ZK`1b=e*9L?D+J48d`$1{f{5 z{LMrB%P<Tf3{vcP34rc-)jLYj>V<d7)<CGQq%&SY_`{@WXzMp;;9>Eh$szTz)D7lz zG{*<H5ghXQ_4Zz-y`basmL-7INF&rsz8@L@yE@aL%TCSfEa33TI{Vm-4rM?iwu>h= zBdBH-CDU&$?0z8(Mjyx<#_;-GTugODAlM}LbSo4B6n53GO#{UjRMFn?%c)I1iv8{! z`)``<)E!tcL@MI7|4IGXO=+}Rt-hI(n(-`Bi}kPc+OOPkf6qBR7jlq(V*=<3Z2PZ$ zQW|Xf>9Lu7$m}Pj`P#A|?R&^|eEYM1&<3+nytXWyKu9X|zCRmI{Q>2Av)OR86D5f1 zxCq2o;o;`aAC^OX0%{lTNWgD1bIEwYo)%<Avc3uGaS*dGLaMa3Ebc%Eh<XVLMDxX7 z9^3P+BN%i$2#+_DIT7{w^CLN&eSN^9S*_-9K0hHr_)rYZ&iQbZQM>U1pozD$J<pTB zy}Pn^8q4ptC*VBmVF_x&?I;Z=Dpo#S6Qg6GTWL}RQ5{PSl{M7p{%k_nBhbwfs*eZ{ z3rTPdox>Ld>{~<V95yz;+}c01*Cd<5&^PGe#1D>@eMWW}8xyBqMUB4RS}q!>1Wj(C z4i8tO1)&+3*%DtqLv#BOz!G@3vSs?1#*pf$`x?y))Yog|@f9zJNG{MUKnD|rh!j7+ z6>QFB)gGSiR{bRk-4o&?4{$|Z`qPoMN(`Otobj1_d(hy>%C~R_L>470O7uUUs~+`{ zB0G{Ef8+lWTq;HskL5tozYoGHo#%p=MpQh722tbE2eJPG<aoa1XvA$+fsYV^A|IFG z+v6)WuOGRu5K;#^jdlUmJ877f(?Na9o@wrcrhEH`mfOJ|vFN@DvY(PN+sx>ZH1?qj zJ#ro8k`p{Pl12oL%=)IFw}H$eIDSx)Q1emrD`cT*C37HuAOXnFVJuE|4P-RZ&1Q)2 zn*|i$X~MV;-0n$E`do*FcF0meT8g^{x+A+pXQ4`YQ<}|I7}RuZF~KFmb@V`X)6#r9 z`vk^ErW!`KYixxA!N<+Wx(Fti?@j^U_z*AV$Oq;Le0juSI5UK?`5S%Bk;sF#mqrM( zE_?)0mqex$-(?mxn6;f!g6dGtt4Ey|4O3XPv1jmm39MGt@^#hAln33)|L8{4|H{jr zm*S%9MaIGf>RB4F{PV5OZ2KSp!HTW1xmH-MY18q2twn#9Mv_@ZmO0s-2h1WY3`n2? zyQ>#kcD^`EPa*Nf{G6#&pI@KPFvIiUw}aMxo=l2NLFfu}_HSbb6;Yc`sMwSztP%d7 zb$dL!6PXIM%W&zSI3dbxES+D>=PkR*rIIpgjv~!&HT|aCO)$M`D?SpYd{3Kc+!0P5 zLbEc%!w<wx);ET`hX*Up5IxxIv@;<ORKRzSA__WLSVA6);=BNS92#@Q_S;p<^f*2D z4*iQg`Ow<RBR40->)p5HG$L{!R$L)3h%Z^^2y^f=wk9VYX?;LM%Kl!)(Q^!w?wNLz z7KBG!>D`=c?Nv55_IcKsHVv(5*eBqzyOKQAz<mY8m&g5h7dXx#NUPF$lKz_bv=19< zJu>arEWT@p!4HO_Tkcgs*q|5Fc0BLft@P1tyg7Ht-GiH+eC&Vtctn{I9kfE1@!YBz z97sdLB)1A8rGI8oVVJqcxYCYmWO*xaP7Ih~z29U>rd&zKurf;{QOL0L4n2TakqIhO zAhCytfEn$WWxeqLs^i$gwvT~{QeKWj2E6M$>S=4??A)d(eSqh0{Cyjv88JSUnGkFb z3B-WD+WdI``*-6<w2FOuZiIoFuEdck_FO3@)fBDzL$v5B|Gedg(-qr$P65kOj9>>+ zBy;(0X$YAESc~jH=Nk$g?0?w_$Cdv=?Z1z|0S>wS7S8j-uL<oJ$Q(^R;Q%i%<T->d zMX8BBn4lVn;_LSqF#Zad@VAsVuY@KrwUp&2x$TuMIYkCHslh-dCm|&VoyGi!1C(NT z#q_?K_y!F9X-*Lk?cZ~kpbrm(1l-@#9XW*Yu_>@rRsmbrUI)AP?~+q=EuJzai*Hd= zj@B%pI)uRLXbA^6PePiB{pl3m@&O_U^!bj{jr)7W#eULdq5~#%P)0}q%Pk=?WdAaN zX@%imvGn?0c`KXjGXMj!clICsi~K05>Xy_+CD#m$h6}tZy_PA(rS~W~Dn5M)+3y## zrH`|tt%=oY_j}}?ZfK(h&`5v{3|^Z^1yIl4gJwT1DAL5jmHu;7ByYLNPsp(d2j@`g zlUz2J8uCOK-}hE%`R}~+l|fOmX!Dtn{rWVj#ErYLXG1L``w)fDXqxHp0>hpMEiWmc z7xQ)51MwhQj>oY?_GE-CdM(DjGw;aJa5+bvRjT!;rG*|31QOi_HhP@|U9l<#A|Hrm zNKgs@U`xZQ-u)HJ4(kF%$>-$oibUw7ASbO}HI{EFoLqeN1uXN$)4XVn=kJfTwO{vZ z?zO@M8rRjQ!{CZqAIvUh&KOKSMR3H}D3=!)J0adB;$jBWT`Twn@Gc#weo0Q!6?%d` z;+1(HeT9pj+Fk`h_;3#q-J-Fcp~(pt7B+N!vubVz$;FL{)*)H)(Po4mJ$eL4Vfw3& zv((vYRVQf`-oUi+oE&4$LYO&}j#~!?j>#pVAuObw7oFcmft^H#Hmdgi7_a3;rEHXC zXv!$d8c2u``Rn;o=<zk~Pqb=T6K3nwg<4d^A=md)xko({Ah%ZUBmy{Ms|5bt_bE@K z7-b;sF|aoiD2TKR@;8NEk(5#+1p~>7U93HsISOQQ8=l5VsuN5x2vK-YfvH@@@pk1l zY{jhySwY?8s!)TxqCQwU^DetAl4pt;TO$x!$yF@B(hm~R>%$s}&IXnK&}_TyXs!AF z-MS9t&wGOl9qw^-vSCMJU=XgM;i~i6ZQtjMUG^7nhz>^jIPC3j)^{}9>ves!w4O%c z9AoCtHV3+<W@cJ3r|_KP4Of)+Kt)1R7{K9IV(l6<(|Wp7K)qn3jC^Zdu3#&Zzl|rt z@H~_j*n|S4iY460i>^UHRyI4)bV~wRN*cOkp(k(;TN1Xigm;W{`hKnXeun!`%I`1B z>+dFIlNxxX*ws6WvZ?bE*cGoD|7~HQE0;gxm+XSszn5IS`NOd}NImL@++l(Dk1RkR zooP%tPcjS#kW>IArwGCSn1yM<LmT9j$rbo~{VT=Aha61eu(ebWdWYHDG+gU;7TgTB z614sC{mu}QEeyj_sI}oVwondW*pE@}2YN^D!dOPgTMw&}>b|ll!5N&If7=%On{m)& zLO2M7X;8SOGELB6#@tK*XFpcU7{R8U4$Cej(zI4?7`+P~UOBGXc1<pL;;TJXyI9*? z8i!VBGe3Iy&OHvk=D~cDZr=(<y!)ylU|w=bS1juH*bmu`nz~^b^mNIJETsyHL6{WD ztpL(5KG9VvY7aa2a7B^#Y1Eg2O2~Id8X;x`Sq25hrj?8nY2q|(CewIuP|gScQk{&X z<vVAM5l#woQ4*hU_b?ds<^F!ymH*16$296~yplYD=>pYdC=SAZ=rZ!$Hf`+Ym#06x zP(20Vc%D;-f`LGfh@Yw`zCzB1C`(gM=gWo!ngX(_iK<J=Cg60X%`)Dk9ODF%VFu8s zFK0X^rf+wTtC!hFvEASl#U=bWPQ{vQ^XG?!H<e1yOJGi$+z~^vvM;4~8K%~1@h-dn zDkTfLduAb=6LIjxRifFGk;vQ?^WwF}pttYrj@HjcLxv}AC}k;&G)H#Faj~lNh}tCn zN)gQVl|GpMLoqVV8O#kB2~ucE$=QZ4Q*Xlo9Y|FUEap|W!GL+?J$iHW3I9zhQ~&qb ztU0(k<2Nv!A@k&G6ppm;3p)v>7djqDX+UvVV4}|D;9Ju}ar}hGL>oQ#ScYba9!N@E zi`+<b<e8iKo_j%C=S9IMGKQ^e9RIcg5_c0lKs6!xi!#kw{{DSbN)S32jb&vMr@Xf` z*9*Eyzxg_K+dQivj{Gg@Pb?iD!4<PdJrq_7n}BqWzxEmp5qZ-kslq&EfoEM79{l4U z4H`kEX+A;(|1K!lK6pad47B|IagUsjQ5<AXFnXQ3l)9O0=JN8`r$r6fFDuq4rN9Kg zWwdYHUM0cqIk<@L-m>Io^1rj_q(2@*+;F?6&-T<;z`im;!R(jRPcTO}xkkI$2-?54 zXV`H8boT$9h{fOQz>$Ro>1?!KIPiVi$k$l$E2G@i<h0p2ZP?Xo++ltr>eYGXZXN;W zL8<VKdlPv2rzMLj^SGU&YC^RN?_cr~4&&|jr5;2c{+&`kFF=l(3d)JCn2MA9)=fT? zwXoLwSFl51A3E5h7x(v%_e8Wpqa_ykn>@Fe0LMu?t{@oujgPIFK1o5i&wz1b_SmQe zGf$fbloCiVk4Ia4RKi&ime3BpCAcJ5Lx}8mYLM^i(E0P#cK{wCN&mIjQ6Quw_n9x% zcXe4U<}gzT%oAa&ynE+Zu#-k>e58xH!th8BO6lS;$?7Hi5fD6`7F+`2_d&tGZ_k?# z7~x-ixRwD|8_D$YnDI(~C2Fkd=#rR`j+<@3FYxn*gXs0;qTrtwNo}y)%fBeX(|_9z z%@<((&%x>!VY~8Jm38+T=11aun3epO8~c~Xcv^5B3cUV321%5&A8L^4fHA~PM+xUG z1K~{%^C{qTu~XhX^U{gu<eoV#_Z=l5JO8)2akx3Na5mzkB2<~LE9EX>ev7tSe!$~g z3ZPu(Q2p=u?XMR83ue?gH@4obwJ+T)<mo>e{+Iu@vCl#=sm62pc<K>;t<zhSAf-tS z88X!YO=8r(!d>~Eud>BA7JY`>^^<Pmn13Pozl0Wm_5zr&tOL=r1HelFOMw4ZcRhQT zFW*FEWJ(`#rFi)}G1F^~HGFkbVC~>|ym``;%ca3Yr$_kGM(~cf1Vu#U=CvIl<2I%X zc!XKBF^dNN3%~$V@cREAhnyV{FaO^$)oq*K*Uw0KxjPSa4dqH{ZK^r{Ij9`)rCbqp z|1J3Ejo*YkMGh*nS~0+7N|83kv2X8m41P1CirT{9x)8*s<y$#Q5BEPmDxTXp<DwT6 zV_u$m3-}`dWA^O-HD-Xr3k^WG27DlZn&^L9J^7}U=3L`BH#!(g$?1ptgr*?9$Brin z<9X>cMAE(CeCmqPt-?9Op!-g&f;?~def73wc#8N!YIcZd!e+;w<~xrcdEM&$#P)Zz zb25;BFj|g(2JwIA`YigN73qI>pYhXR2+zfld4lUz;LC>|+a|5zNP%8O&(oS8*1QTl zuN%(YH?d{8T}B7qM>O-hX`ev@SI9$np4y=JB|Epwj`EPfHNx_`|9_4YX|5d=exKR| z@n&?~%{Uof_KR;OLuvk)%@xdkykm7J-emuNe65eM-ndpHAfxQ*f~(?9ZU<!NqM@=n zvKMtb+%d6UIZpA{#Nio`p!{!g{5>yjVc;7OKm2zex^X;bk7qpZo_Dx$xCAehOcx{+ zkXqLG?98E4BVU$k6fBs~r4x*`ryJj_{sC`zc=wdKDl}D~`Dj(Cul47qreyd(|2J#? zztvyZoGd5M<xc7UGZnx1wMq~M{GP+9Ec0#Y@AGJM>ZxuT)_ih?SUOhyGo8`HYvLM( zVwY>Z757+*mk+sqa{fK*1)x1B<%Sk6lep?&a;m1UvkM8+QH})3=l_#qZn`2^n<yK- zAt;LdE8#d4B*cljOdIW}fCJpx|8Dt|HaDe*-AMV<swWT>>NjxuGQCrqeyXDl%fkv& z;pPb+S_m-z+88}Vge2T41)~GH;v%?^0V6$LBh9E)zM=~5CHi5<cna13?y8qkz|9=W zb-gmcW}$IPgo7p2E1}o&lRs4W<6u|_B%Jnv>JN)S!RRD?Xyt}#g2OPk2rpldY0i?+ zM>D6!El!v4|IOiNe?!BMfjpI7ul|v?gZ3wBkgx$tAsefN1ha(EJ2s>4m1=9srkWWV zcS71hL6&LyF^E+{e`lCV8`~WrX7D<N(|sx8wKjz6^{c-xZiaXFAWdjKVELryv`BLl zmP2;%D1GplfReq}=E7dgmo<Op_l%uNM$Ci!8T(i82+wQ=3B8b@i5Gr3Zz?CA{@csn zu4GfzI75Id`)@OT4am$P)c-9*9Qgoo;J>By7q9LA8R_1h`h5V`AOrHqi%~D<Q0V$^ ze*LnmtgCW|Y7%B1ZEkWhlVGB+U)}aXMaDbU9}Er*jSUUOo+BB9T3L#UiW1caAXR|z zKq=l9*)_7^?V*%}gr>zB1+<#4qG?*h957x4gHycFGjRcKrU8}s+V@S-LU`4Md$qPN zk-UTo^2PAbD5B#5kXpVJ1@LH{anaQ(JOk63Q};xk>9VS-!KF~C_G)r<PKr4(Q3<Z0 zc4Vt>O-&8NU!eTUY@(M@8#TZWYlH9m5sQTGw!a*sTERG0L<reuXQqD}Epu6^Fl4TL zjUe@FfmTOc9;^%oF%okP4h-O%W|2=}x${j~;Uc4q{f@f2umhN9Ckah&@eK6!Ev1*> zo~Dgcia-{1kEZ*W$%xu0PUQlEa71tvOWFs?j^`~d)7zvZT!dAJ?Ui!ekQc>%x3a30 zU2Q_}%_uDe-nipSk8X~X41SbDyEF(PX@Nh=HEtC+r}bjKA)m$#2oyZpZ*=-4f|acT zZ*17xthUxSv4Ur&rBTS0Ba7qK?h(g+Z>A=>xp%YteJw+~AlcD-=-HnZxGXgJ!(D?* z1b3a)5!iP$Q!{Ny$K~LwyzD>+1RgWKX}(6W#dk#JT5?FY;jvD`LThv6QTNpg4b2ME z9OlULHRmtI?`8AlPGB{)s!{6d^KUdbU6s1X5kU)%Tf!LPE9vcmS}8A9)<&mj4@exS z+zB#V7?;niqC&TX%Y4d<%@e9;vft!!@t5J!+jjOd{WqgJ@XrVb(Ac<?5=zXY<1z(R zoNDRomSV-7B44M_f?1#Z{85x&0>93Bw?~i2)RJJ&&p+=jX)Y-5dzVF;FIOdn5&iUO zf6fuC{)H<+6rm8)_k!_?Ey-5`izF;;t{6I)og4*J=iTxFn*0t240kJZfDOeMGyFP% z4p6!&{hpbhoqchY$b@%W{sY)raSg_3@F(4@^ZaE^JE)F4)WMC&`f5G46@d8Y<8!lg zZR-so^1oUunUE0bNOYNw&IYoH=i4U~J}10!U0s|sv(Sykysv6M&P60ljk1MLA(*u% zU_T!iEa7loGfy7n9K-p&9mK(pq?H{U`1bK9ok5Mkj7gF^*~I21b@||rHVM*N7CrxK zskl!%FL!cFeQmr{c>NT$jNh0*P)KIE{GaJQVY~oOmSAZhCsVghOwjLj1vQ#()G-0V z9)M7P{&IraykhlR*{R@j5rDKUY7<Xe;<uUr*%zeOfz(Fi4aNXBACE=Ap!c-_!`@a7 zsd)Raug~qY=`ul~D?)U?$rr^sN6f-$Xk$3Y?{I#FIv;+eWi-^B#(sCM`2OSLaJIiF zt&j3t@PRu~%wOE?ThOQX*y4cg$GmCd&_lEA2+T(!A<bwPh_e!jy2a$@R)>3H@a^40 zf1EyZ(Ph=kDBU)vac+1{DiZz3o9`w{`s&j(Y@+Emj5(V??=!zW@?@6CCC0b*K(@Ds zQXJ|WP*q!dUJQ4}l@|odVaN}WHVes!^o@v1Zxz^-Xtp1jihSXL#lAjXQ*_}+-60?O zBwZ%88Wh>+kTA=F{q+QHd0Jq#0dv+YQG0RsY{(hc;<Mf#w0)FYcP!UswJ!wgNF)az z`y1rKTOK=WARyilDvbP~yq7l@n~XjjH`B0DuJ*X9>Tp-DQ!yPxC&g&V#`xGL{b))5 zbA@vU+B{8r+dVTqb{&ul4F!;Aj=DuhaNg)$61Wh%o)PR*C%EQ)$ThF$9QslEHOO2~ zS*N1#O6)#kch(0mLjP*07cvZLwX=83^%<R6=At9Sp5DGP&yg=?ZQ<G4+`_AV_-{a_ zRmT+q;DADaWU>Xq{iH3_sxl%X$oVAoco|hx#5nO_6F)duSw^?@4&+J+C1oj8%n6q6 zsvW1bTlPsaCu>n(Y60P#$HWZY1+-9N7MuE?$-aI))zoj(%BY8l<&VT*@IqeSX4}4* z@Ha)V1gBu1T=n-eTPgLBZf4`&zf@_N`LodeLH@89Kv<mrjLr<E(cv19RW?$#_EC_D znU|9U1_s`RdI=6;Y+XFT3}t7x=pJHSE4a9DU-?z!wsz@%Z-N-SYeP4NduKP_Z*mc* zlbsXcNyjs~Uzz{J%_|Qv&X<8h;*nyaQGo2}y11sFL&BFJJ0|+G2wC9W@ty-6)0dp) zW;CBHWYFAH-!3oC%bw{p>JPSmt#6WbPcs>TvhD>Y2n;}=@kINnB+fQ2jU`?y^WAF3 z5$s!i80S!R?|_e@Wt#x>G02DoxBF2xCGv^3%a6z3VV9<v*7C4qT`fy@B5QVt)od@h z!qeKJk2JIBrLq*_7(K0ABj8a^b9gUw6D9=DiFnrEHxuxzpZ7;oepE^O+PSr#K>00u z$yB$(s>w?coVgxYT}G19j%J0;vkCfgBj+z@V)0FZFj{BYsesoAqMU=hM!%Cq+9A0F z2f-Wm33`x%i)|>vKiQmXmuzc=)PX>l4c!4Dv+JXj*}c9lW}5PC^9}LH!=LYH>z74A z2&kn>?g3iIYCU@T`FWcfjtbJM-08h)L@o9<*RphA4h=P$CqbyDM5Hj;fuPQWtFbW# zKVF8VbtC9o>VQ=rg*$#~SoB~}8+ji&63dSw9%D^-mdcFr&d<X!7N7&D&<~pD=$RO+ zUw2xHJraF*6D9J_;pj2>QMXPHX+~zck7puAno6BF)_(TN^A3%nRE%jQ<Fz|8(P9EK zwpoLwZEMiUtaJl$g{(z}PUdSG_E{;ZTI6|@B=HUI1oI7XrNsFMap4#3)FarV;-ky~ z54+x-yHvE*m^S)x_|>HYMbm$L{T+mqO%^Y-&{L85q5K_n8*opN%|M3<()wEZ!^2#g zM{_eTw3RTJo=!)`{=Y|`XVF5A?A$&lA}L(CS1MXTkqn&gu0tVj-y5-Ujt+N*SZ8&0 z>^Zy*x?VP?Td2E=&?!2(N?l(rmfj>$W@q=~a#d-Rn2cPo41?f?k*QEe`q<k#tXV3j z5>kUdyR^*CP7YBFoka57<8cQEVw!CZA)&;j*f`;mYY&u-a4@%UUqFKhrwIdt%t~r% z;!+r@Jl$Rv;YJHBDVt%H{t?_qX8`d%d-h_b;m4mxRr~T-p`8qs%Qwai-YL8EhHo#! zD{$o^0@<n1no_^ETVW1&7Hnh^filcQ4~Hy%w6}NfxS`DPt?k?*6fj#~zqwRzuJGTw zS0;PnF7g1_>d3Gb-)f8|W2<%9m^sIPpem<wmhe>^_(f*VKHWoE&6<Iu07Onx^sgTT z(d^Nj(L~TDd&&KXXV6G=#tOAD5rGhWwnE=eC(#uufO$`yqq%y<af-}n0LwHsf|qOx zt(r|41I@7>L67+>!V`)w-hM<nkMS&Z5KM*(e);U3)I$ALJg*9fXc?~s3x)RJXlE}! zy<W<sFLVHOZim~I9hp<6fdGD>4;qXp(y~6*2#op8w*129Jhj3dbkW{l3!WChOa(>O zuEf>DK0J@8bi<zLAw0)f6x960dOjTZ{>wrWK-8-hKr_yg8@5_11edH>q2JNSk4|C0 zSaiyA3R4vo07{}@Mdx4`-RN+OE;^dG7Q+<jj5_YIFC;+xvDg3-bC(EYYw!2TABzP; z3*$0x3F|kS5Eib2DOyez&z-mZTP0pZh-MC4p_*V9EG%({F-3zVgx&8jdTA!H81)>` zpdF2u<e?4#CSL~8Lf%j-b>FXdk1xAFN$D#AjK7gEUWjzNADYE8z@z6-5Y$q3P7>H} zRl{tB7H01D@;w~}?a<g$|F$*eoA=RiB3l^81k+;^V{Ba?&9l0p5!ZxwY@*WPrfby_ z^22M7b}8gOxR9+8b*1HfdKu;h$_|7a7wL7`^_Fh)17<wwrEy~|27*MYRE^^H2jBTo zn>ebrN?ae0ZpL+&LZqLW;|dI=m<Bv#*cVT%4Ex4;V~QGtQ(zd7>v(B3FWrq(yGih* zN&gIQ(;k)VX$<`L0sGkr73X>_JK=P2Lmd*R4gPT%({>pbAygN@67=r*sxuB#zfkG< zgT_kk%+5gXj{d2-J4yIry*gk4M(FeQ3x$!D9wzHZO9xZ`d!aN<vP9G8Y-9*luPzz< zqgJTCFJtNQTsCZj(sRDGgHiA6GMv0bqNe((X>;(Nw%)6@QX4O~Xvc*BZu?(f*Sj@U z6!g?H!u6r4M(wculfd6-^AJ_%o>JujX0qsca6pu63c#(-XjinQHP;gpmlFu8+j_+t z)cSS(6*g9&N1^rpm3f8aV`+TPSTa9WQo<v<irx?Jp4!slESCF|pq&rin(KvJhFTKV z24|hE{$a&^?~>epmQu&CoQqt!@$~R;4P2O?9GYRzU!x$zj{n1<Gfl$F@hI4w>@5}I z0qAu?p&(Q+=U%J(*fkI|Z;KQ0@K@nF9|q>4m!`4~fnbr60!KqEE7YMCC%!Wr`O{1y zXd-BpJWMf=arV>PhgQG~Z(LL~lT}noyfufcc|l;;n3HlOq&mn_Fc4z1XJfIAp$Z}W z*g}61cGUvz_*gqbqK6rPcf?rPiWCxgPzrrkt>sB04-%{5)0sF~9wDUyQ=e6-F|yO5 zd9c&t_kYF44)@Nkc)!Bjc&>APTjd^;*~Bd!Ob(0KBWS3S1Lx!@RX&7%TJ{p6Sv|qC zV7qyD?*CY@@-;Mf`GRe<2E&;#MT^X-RpOUQK*5104!xEWjqhkm=xq%tHMBM7sM&i9 zZOYx>g5s9qCaP~&Y82bB;HvjnQ-Mce#$k$hipP)kEOP7=zgfHBqIZBj#Gz-tjV2<4 zJk=uSAeXt70qfYv<JN*ju<PN(R<ZQ_B`8d2tF_!khyt-WFfSuJ4C~c*sZ|V<QWYn6 zykyw}J5UOVyveW#gV1qH5%EcRRcIO~d)zdfqEha8QKsVo11JgRBslp7iU&d9Bb$6N zN=(az{F4n|BgXC^v{Q9y)3b=Kl=(e#yQ*pGI?Ey;CMERT@BRpWFZg`4)QbGeVYi&= zf9OGg5=+oNe)KJEnttX?V6?@4cwi*yp&sfAj$+jODXiT}^x&)I8g@eUb5Nqp0LJ~< z(w%2L?2K3wswv7`iw_ho!5r_wZtY-V!SZB-BTnt=uUOKK$Sr88g7gL&oWI}P6#1TK z-|sC&v3yMk*2je5nv~&Sp*zg<Gtq#lz&ZeFTQi-EJKyiJGsRDWk-{9BVD>}N+LJ_3 zk)CQRIV+*}1Je+R2OB$HN$(;vM%yW9h>u*cs}@%^;mhabgs1S^yEuF4&GK5&n0*?k zHA_CoGoGx{Z2UDeCOS6u3{xJSb36g_18WQqj0$=t{s{0&5{>jxA?@t{_;TD^{|i7R z3nbNljH?=d5ySz*tp+8YhI~OVde)RdDbeztqk(08GKZ$XX@ot*A2cmA6Z(D{U}N=! zC{AiS!dh3r)PqLdpeMEc#PK*Ip&o$+T;+Q-%p@#<0}lfqLR(l|$v=n&*>?zo+<H`% zwZaiNr2y%r_IIWz3usj&gd`@?HQqrm1V#eGrAGsyVYv_kX%$v<XozVJ%Bu7gKoy?0 zp}s}2(36$XC7<QQS`u7LV(7gx?w*N1#yh94y{t_LM1@OSFKya?iMCE_Tu+oR1eb9s zL|5CBwTh$L&(In%44-Ozf*l5r#<dP@NtSr&b0xpT2*L<^AC?ypUUp|-h>mMUAU!I8 z76rZPC3V~sng<^cXf-y}#Q9UqXn~#v>IOcf5iQ-Pa5(HLy3Lb^4US;<XII(N?EiVt z5W7O1s%~}<T2is1yb*$~u(XqSpf9&sDd@MfJFdVB?V?k>h}Q6miSDpSIl;&8naurK zJV_#(`=BCGWi{dcw72}Z8f|fC@h!5Y(d6n1@Vgu7dhabcLE8Ze4{*xzrWSn#IpP_g zf7mO%N|aT0IyL>a{00heNrNMF&z<6U0<KpBa;<~o<losp;0v-aXHnV;mIlci-F~L; z7nE>}iW~sz&L1?8c$FhkQ1|lhxBibh<RqlGmtXiBJA|=S_+=rmaTrIZWlP0H9g+lm zovbE?IXM_N;eIV;*hD7A37J0e|D;&sxU|*eQ7S)Zdzn!S;6Sdw>9n2n@IfpgL4kB2 zyo=ZkgP!d!7}4V}>3O})W9~=6-@aMn#bb=D-7*J|UEDptvHD563ooxv5BB9(2la~! znd5-QUhbon3SGld$7D#i11Tp4B&x=<SX29}EjU}=>?&?r9xA^d9y2X?Yi?oTY2{)4 z*2>z$;;l7cp6@$*Sgm&8SW;+2DHzDs!&ryUyIAXs1|fqeC~#vbV&QWObbhJu&f2Zq z$s1APQIUa1P*Ccp>1$e9TFQAZUip72I&0)v9uWhBHwOvf{ZB-f=kF}HHs?mr(mA8k zn!tKGL7N~aT>YFhiP^@wxd0(z?R~th`FL0One^<|dVO@7vIUzgVNB`OpU8>lVslsf zWw}Cke#FwNI%&VRg3o-R$^sfe7^5!Yd_fXXGB5Nnw#}Yr5?!r$KyEhUme_s4@AasC z#Ydi7k)6qEkBajps8Yw-5{IsMhhVjP54U7lia~A0I+^6&z8&s-0_<94{O234S{!4& z?Z+tBV-|{=v^~cAkVA_GVu#abp~_Ee`i{FYBqAAt-V1G(UU=!ndEpIb%WxfsWpjd^ z&aapvTw;f5M6m{c*Sd^{ogAAa+-9OGrf{8xyqWGDr{v~}pMc+au3SglrY?zw)1PFv zs74e93)gCS^hRrCN!SdGKE7M85Ijy1M7`_A@r64}g`1X_eUP>GLecJ=$z4%t46APW z{8=~<Zr`BaV(=AP`RYn2mO<8GJ|!bx`i5R@V&8dj%cZ4WRm|v;DMB@mqUO(S%&}zJ z@$^@oC;7?P_;G7ao(G-YTQaSCm~j|%p<?TL^bw)y`gd8%Dz>muh6QFuJK0xtL<x)C zG6D#omc!yB^GnHY%wZRX$pKZa+XKlhfplYrAC>(xQ(}_dF)OeF+6vbr*xZr~iK>*< zjiAKt?f}V(WRxVCn!$;T^OPhf&BI}i`Rx6jnS*_`EF9b9@$=Q|bmap*mF#Z6+NMDE zcVNj9p;hL#N19Xu9+?C`PP?f`xAute_i$2s{4S;#KV(6M(0Z63dXmE@zOL(R=E2q& zq6bL~2On)BdVIE2j^hA8rO@>gb8P-+{Re>ix}TnqqWJ+u_5SJRq1PCmovo!E)1Ic# zHFRuvQMFOg{K^@uZ|tM<OgY@$F7=fwjQI6iP3WXFVfdgUSUB7w4b71~(b|zD+e&<w zzCBemMv}%=j(FS|C=R6hmaXp!!RQh}8ADPXR?-#WYshD{#re7S*KNmKi&%CVceI`J zh+PQ}-BZo1^TypTtq$e{eBs|*823&XPgB2^b!J#dXj<__N1k9_?!-533Vm8!l-slI zK-4+_B~%49beID6$V8bN2Ez0vFPF=Pj?%C}aNqmk19PK0{u3JW0fx67SMv^Q;v<hg zLXl+DOax+)h`IGGQO?WB6%DE@=_~ppS}#B6R5o9V<K`fV=XbDLe!q0)!Iz%!5}d?D z*ey~R8zx}kH9Kc_;%r);OtZD(IkYx&S$8JAcBK>?De>cwqGuALOR+ssne;ksf3$P$ z(_4ml_#sd^)O&$T{p67^Qa|C`{pR3=y%t5ob!1E{5r@Z=5gmRzJ@ny)j0~(^hDqk| zu$tyt?8&|OOmH|P-aA%54r$+=PeBiN>gryi8ZO&^)Q0TSyS`n8ZNk2LD_Uf8>MYmn zRGne@dO>`8GNTwXnp?_x*HXJVe9wecyE>NG<6N@{yk6wEbi3lO5o9tqm4uZRb-RbC zF~Lpmv%Kt2X*Pt-ml{N4;DVDzYjteSj5{@{3kWZx<Ew60!YkjegU+j6JH#56PLI3P z*&|L)Wws8T?yKk5B38@&vZv&jP<H}tvdO?)Bx}w}%WgfNe`PbMX<_cNVMfamo^cXB z5F=1hSdJcWq82MHA1jd2CE&$lt3UakjFH;O7%G2zYG-%xc5lq*agq`)Ylxn-*Mb#! zhZ@6?ENRsIozgF4TV9FZJJKjn#Uz@Ofi0WMW5UFS0-<oTJ(h6hW6l(J#>X-o^NSBM z7Wmj&y$z2mKQEfz%no8rMSXsYPj%MTc8XBah!XpdnyhJfGy7-kZq|SJOLZ1zH#LKf zvtLO@fGjT6xKe}{-gg`)O*3Xe`D!*i`CT=aP{g5flLyN5$p3ke1<(=wmj5HQ^=lWx zhevA#Cs=U!;;rNa>{e@)BloS3v0v^tNV=9Vf1IR{)3L95ivCe*O0x;mQJe){7P$73 zKl{kNu|j_xo9uU0P<HDixQ5`1EN9hKS`53}_tBjV)bsi6yE6CS-UnYKYN!-}aUf*| ze_&{_E2<xmot9Fa8Od*!v!ks9?+I6%Xqv%Rx+TU3)DTRDXmtYBH1+(EE62jAkiX+c znmc`LMTTc$S{l_omATnW&5%M0{9f+)yq#a+UbFOp`_%37#+!6ON|7kwhQ~cAE9b3< z;P*<h1M@$E266oJJ686JC;xnghjxx&JfKr0du-@GZ&v+<q-y|87i6GByH2V&&Vb0y z3S1_Q7$E!=tcfEg>rfT+;OQseOT66XuC{$9h3(&-_K1k`s<UV(<c`<sB<{W!4G!?< zd72Ha(<u`SWv`rgzgoD;@6un|?m*vcd$>?rZ8f^TpUI)gx#eGM^tZbQw)qc}PX7B? zdxQHj1Jh3%_LbEHT`8w^Fx5E0Bd2cIR}I$3Z@rlA(;zbRnaUN`*s>U|$Bn7&Pp#7s zW7=-A*`oP2eQDqC79YT<Zpq<~5$FZ(6f|Fq78Stwn`(A%!B~X(%=j;!{c}KnZv4;z zw!i28TS6@AXU5%=y2l?cO*ZL2m?&Qq={$ZT5+ti{0!@xvF{ohX5{O=HNi$EHr%}7W zkgvA#I)US16xwB!GpUGHMAeRpgrYt&w+vgDET1xR#ht)$Dz@G6{<$z6;UWJ!O0j-Y zeFtB9hN+70Fkzo#EUSPu?r5FK`c!n9Hl?Xpbb|xT?eSSw^KN@@l-#-RR*?j`^f_Vl zK>k<9HXo{!M^jY98AE&qgIhTAA9ep;z6=tRy)?ceaN0jJLk}o|{rb_h%TIfNCu21* z?=2}4zOF8}KXA%+8BOhS-To%Lu+fDa2S1E#<IdFEiM;O5TF+0K_V}alHWj}9$HG-M z1{zY8K*G?4tGc=QzSLrS+b_D<QM2b-|M-(M*URS`8v1!Wz<@>cwAy(eRmjQM+jh1r z=g=yw{GVB%^Z6sEe6^)lS%VE`nnaiOqF7{bC_6&T61Sne@p0-F<i*9~PknyViPF(F z>1tP;+HU>~x2I~(zbxonS7O8s6ErNr?sBl`G1skDrMZ<6H#jYc;aqa(0Ls9?aK-x+ zI^%l@i@4UsNO7`n3|EVrRhILI{2hxU;*y%i$xUiXcF`@um!jpP2i_mAg|d|&{%?Nr z2XzlyYdgE5<84^<Gzx!^j{B~SvcZ?ET6^s3z)&>^tGpVTUCK-s{-t~V){EE}SyAid zNv@Hl5ck!S_0(!~RR+AtRVuV)sJJ#Hk~_!<djT^SzGRN>EDDpJq~G#-De{8zJBySQ zagjq@rivr+MUA8MS%;bC1lrfE7d1P*Jl|dTSjnDpC5}!lIXyLYKXBiVi-njdlBuey zx-{>T7r%LxdpYY4g<7${Ln#_Hzl`lRUO=pm^CSAH4Npo7HvWhc5qT&0@lxUBS!n5+ zI_)*O){qIioJ2&)fBZD&n(F59ru~a5!$<ib3ND@F(+!ZLbQ!a{%*t<hx~<G%pafPs zSHLxT_T9B>rDR#35IGPvCbO=uy_0lb<^8Om^r6_w>{T;1ZZiJWu4t9;H>fP)RGdZ8 zWE*t!tfTwgV?7@hNbky=qRf8Z<&97g%N8X;mGehqk9N43EqV^VY!i<%%I|p;HQzlD zHuexqEwtwwvQjNeQ;*_O%f{T%JeHdd`jJbYc^$sd9NxRI!rra*{!qr_AW~^-P2jFS z;2s<bv39v{gLHH<i&|J8-P;(!wZntU!V}!sW(M+?P$$EMKbpdT=eHg~ZZQlV)}06M zN|sti*PdG87ksm3<(DR@^S-uXd;RA1+TMZm=*&!L()f-QZ%k~g0pGm9rTD6~>zA_d z)i1-B)AhW(zsMLV_Bv{*9pYxccoFAq+vCMDs5?&@N32#9AUr&i9EOkNQFNNVzAp2T z8?X9oqi_74iH8D{AGgJ5!@gH?-qs$IzDdBiF6g0de^Dzp!+!zPVxwpvFJUSbqONb0 z+B%&mc5ZSlEz@)?@o?Koc;dW{R=pK!xwTrsZ<rA8MK0}m)Q-FzM05tAm<|rU(9qCy z#D4MBeytNl(aUpui!3azi>>^Bs=D%UD7Qb}M#UIgjdl3hk|^64OXylAAu(ke`;xKm zJA+$fxkz-w$WqqAB_``&u4aUcU6ux;kzJP5pv3Pry7&40-ap>wIp=+!bI$iHpU>x< z=bZEX3L1G&eKY;yq&abMbM}g&i`&S+p5uMJA}I+awwLZ$l{{>+;x5yR7cb)amC-9s zE3+MjS;!cCyOl)ueVlh_-rX$+vPkj*-ZrxGG-o7;4GjKh5MIJ(tu$6O8%j7J-4v%M zI!RyX72SvOLF<HW2u_=uU?6jI=SLL{4hNSfd*Bn=(C;YH-h=CdW(yJP@Kdd4k5FgF zGW;s8OTShdi|Db;O#t>+=L1&{t5@v(c(h!xdpbp<_anL8a27Lx5SJ+XU{cKr$5aRt zzE4SpeJMWe?Cd<gKP3d4#``jpac!EPKAj(zt7-mD*`ZZbZ@w<;jNmG9y*r_HtITCw zZAdZjQI>t;pTm4Nx@T5SOg0A#4Bw(<Ve?|?55YE<tNl!q^a;Wm8vT{02kJjl^0ot! zD6{0L$ij^c$7HNFAdAzp|NO(ZZ7KnuYP5r15V9fobDJrR<sPhKRn1JfcndSLgSe)q zrp&FSt`WLe+*tw3ojyCpvRBWwfT=M9Uaj#H&q&OrevcShIx+1@s9f^=v=`)Fbh<E( z+cp~-$!j&6_Db$5q$PXs{ZuN6yUc6HGQ#{})LI#Z{)|AN-q+0VJE5baGwQr(vL}ye zh%8`Jot1k{{YfqG+gbD{k)YR|ow?%^^!(H`dP<>znUUXLpJwVP>5{!noRhW~+<Vm# z^7cV(a@d!zQbKhNVwcqV^7AwN6qo=98jV4_EQQYOB5MPjKYGf_<<4qwMDkiUL?tg~ z>x|AyDOVrUOOp`&u<JFd?qpQ_uACph^QZa<=ZX&0`a;3_E4!QJ47+rh;c_J|cRxCi z9nYQ5n(0X9akLM$@PsfO$WF`dY0l@Z=afEYhyZD6Q}7lYJ!lh#PHz`MCvQo+&Bdi6 zUZX(GMMd1Qr<uLP)Qux6mBIG7ERF>K@K$1<I&^WFZmH$oFcTaZc~xDAeQ#gN5ahlV z6cj?6*zREifgQV6f-|>2JW}e@*-CowdY8}EcB=j_jn;lINqN;z++)qn>hM(J=({U+ z_1|?Xv(E8?m0dRdb^t@`sUBi}i$Y`lJB>d4dX=d#wep-pX4EZqDBPlT7zTw#u0{}Y zK2jBuHNAGCHh#>9c1+JQ#K2{qD4y@q?UiY$uiya()N;_~K}mH|ogh}M-eK77%d6j& zlrc-!@mozjO)FOX2SeZR1^6J%vG7^gS__3O)L%`58;eqP^IZ?CBi~L~Gg{#}%=goo zK8ZIjb?Ef*FqvK=qspwb+IdVc$Iye6zs<$z8%eN9G@Z+dU5tLy%E+_i5Om@2e}$V= zFGHUsh63Ai4nJv{o4Y{dRSS=h#tHXJE(x&ltK>@>&upsKj9r0+T9#pz_@@HY1&;%> zVJ%OJSfm@zib_D6pQR!$@!K&=@<hfZhJ-6-*S#61%_fm`)SXjL374AL1b`DB3gRR? zElkZFhq{fj9|I(>v-;8bqPn%maM#v?Y;Jft(j*9SRx%^q-ZbU8i0^=v4i1gxL~Q>t zEWt=u^W5M$7MHy)nL~;pUB3qnT2q?R3-J;1QJ!q?ZH}<Hh>x3gjp7v$&_Vi3)nn?Q z4>}|=D>&fsHK9nrXSm`|nZ7LBJ*02(7-v1|6F)-w2x>T<{=%H`ivuzd1C#g3t2QH} zcEpk%i}V0BC@{du>a;r_?ijm1D{r2Eg3=v-x!^Oak=xy)XXr_(C?gV*l;jzrSOhL` zRLM2zyih2@#f<HtMDE{xQ^-E5xD7s<8`uJi7Ikqc(9-&dYQd;(cwj}U$EIa9?9nB$ zZ|3rP3vXz5dOA4`oyg6}VZn3UQIVrhOsM`yX|S!v|FoID{GUUaotUaK6LHNmoa9mc z{#R~Dhw1hc@9V~Vvis~C-IxbH$K&C!&j2=K>RqJkWALmE{A&(jcS-qZYS6)P1B3n| z+}37!Ko*-fgbf6;6e(R%UiO<sYgk!RMpqS^2H8Rddj0VS&%)oI74JON7DOwWAk9Ta zM2yP70qoMQWYa9@|B`s0hPzc8bYRxr`44iXp4bAfZ^!|X<HH6nrBZORdFpp0m6tgo zextF1)g`NHX6Q-9JaUBEoC;v8Kw-$ww??j-_8XV%4N1MCecHlkpJ;z3runn+EAww_ zBF#d#xq~#<-r(AzS>vxJPE}i0Ch5;?Y$ST0;Z`QZyaVMry5<X)?s`wBrz1fiFlvQ1 zGxHxV<9uyvYkY7w4RZxkaQ;m~ZZMgAYP7vyWO#U(9^*i^Pd}f0DJZkLH3TZl%xv)_ zjs-AEG(b+cF}UJ5qC=w5Z&QJe5N@A}+$OFeWsZcNz}b+1=fXo%pW6Ws1*3t9yJryn zv6^Rp=G4^qgTP>Hn!JL7)360a>p~%#lNzns{CjOF->0b@@XI$>hZlt`g<#6C<=<qY z={;1-d!;F>iWe{Ay%10Lca|1ZtONSU#f#%%tK&_b`X^*j78PH~USH{&#OjzW>qtUN zAxnp=;lKy_nIB-1BSjhScY(ms+Mho$J1<oK+A8aEj`rGu-4)qRWjvk6UJ&KpL#?dt ztp7|M?{lgR$L&LaJA&H@`#-AvaBGmE-0ZgTW@yZKEn6p-@rMD`!v<}!&#u^z68ZM6 z;V%ljfa|uIt!HcOKWYumPrl~1XVTu3=qZTwf?0n*@CcX0EgHQVGjZvW*_RAHagZ+e zupnW;n7h-z9PNQx7xe+*Euh9~?hh^BErD`1LXVf-9<Fq|V{d<r%UFE&?iZ<IfCq*z z+dnU<J5OnwNCK2jjmwIC?<oUyf0ntvh{fhjdH<xo?;&eR+O8HnG+NeE#N@}+?HrS3 z%qZPARE)lXG&Oxc)8kE5rKl&<GHwfSSs=|)8;PBAFjxwCRZueCYZ6aQi8}4%wKGY< z@BxScED@k{#S_xfTznnqr_j-VOD6b%VpM*?NvDBsZ{aS}jQN7#{v*wM_Tmo{Mr0qq zMUKd|l4V2%t+O6mI1szsAWw2-bpW>?%gZbBOE)R4z67>AJ<=C-0}OKN`14i5StubG z*R7$(-Q|_fZjoQl0C#GCG}%KImzIbv{{{o@BZ(y!@&!&eM(tH{cfVA@d{QZ#rS-~u z_vrKyxSwfn8TT~!n_FWb&l2B6c23UqKCJr4h#l^VgB~~6;K5!K*A1Y#Xc|QEh1|#N ze(Hp=mAdOi<^10mkQf)n&8d`}7=z}_lZ6aSUKPImSJW179w^UCY0{m9-4m@4`7<X! zX^534Jsq1$VPk{a$PnZ<!WwH0v(ha%UI?3_P^ks`t7Gt0uZ6YN@h@}ch8(D4mqeua zCRD@6>p}@wL_*qyXC)=?$@%x%wI)L{y4%_`NrHbgWOijYhUr2RYiK6c^*+icJvtlk zennN&nX?hi8peQY^tT}KwTtwS3<f2%#EhRItpr*{WX^Hxac6AIv=bBG%LL#tcw6ET z)WciagsUKAYb?UYf6330y=crm-14|e(5jE~W2SC3EdZ1M0;hM*)}(HR<v-i;lBx~d z8XOPncUoRw*XwR#H77S0HZy1SNPB;eO}O0UKi@-EjiqG-hV6clDEFvR-MXfyLxd_O zmwS46H%Xs=a-;tpB2!VeA}P=cIChm`>}u6oROORq)YB_1C-$;0Y02M0xipWI{-A2g zY5cG+g%LpEt&IV7C(0B9ym*`Xl_3fuEFB(X(Kd8jyImK9{Pl%Xfa6O((c#dgVEN=} z$S5a`19rgy^Nv?Jj@UmbnZ;Bd^<(B6m=d`L^WlKj{<g8dKBnZX#Io}ztiV@Yu3$^^ zpxM~axnI2gOzm$<j}7^?*I?${3ouQeK4>kRhJk10<$_2gm8M(K^^AGdH8=@_d6O^b zWPFot_@Zcqd~zZMcY{%wC7s)Tw{+mIQRl#uFH8g=ETPj+(Cj`UAUofU*E8GRRvV`n zND!)Ri=&b}ekp|BHxw*FlN$ev-bC*mtvO&Sk48=0nSUc%>QJy*nH}P*D@Pijw*+9W zl+?5YSaNCE$#1N&S?LU+@&V*Tv>ZpL|48%oB6CS-4%sn^=dSoiQ=b8vbB?_xh4J4I znj=+oUaNiH<W)46`aM=gs^Z>gCILj1jo4Ah9p_P^?)EYhi#V$9qxVoJ!0Z1PA>A$W zzu^o!ee_Xbe#L$@xTcV7I)Xe57N==FXSr%6RBXejw_>9zlW?F23QOC>^E@;v7Rz^J zpOF;d@k*oq(2#oK@J?t%#IC|oVo$XU1M|n0Bm)BjjjGUS*QXE&BnQI7^Q&K%WUt6A wU~Mx_B7vonM+{4!qR7E$*O=Us4~7m-p1L}wb3IAw?J?ldN8Hk_(ssE2U&zF4F#rGn literal 0 HcmV?d00001 diff --git a/website/www/site/static/images/blog/apache-beam-flink-and-kubernetes/backlog-graph.png b/website/www/site/static/images/blog/apache-beam-flink-and-kubernetes/backlog-graph.png new file mode 100644 index 0000000000000000000000000000000000000000..ab0f0c9b7f5729f2a3523b8c9fa1ed3d862da0e1 GIT binary patch literal 28754 zcmaI8by!t<*EXsm%>txZba%SwX3-6zbR)2&Lpr1u-7O^zn+9p=mIkG}kr3(TO!nT- zbG`5R&UgM<zy<RcBknQAJ;o$VO+^-j21a}K>={U2PD<n1vlk%XdpHUb@M(D5RQT+f z1F^i6gr>XULF=X4f^N$WpPykoMQu%d67TE5%)vz=H|~aEy-|;$y``o8dYjVAWyXu; zQrdVNX^n%{zb=2-U%;ao!xC-LUYPHK$ScRFtK}17<erTodj-xuTKJeDn1)ipiT?he z2={|R6Op)ju%Q^h$A%e|Lf}8YF%6y244tI>_wP(0__>h(KBXMY2>Q<&GH?Fxj|3#a z6A}_`=sSZ_Np7*7e2a>T&XohuKCiU6{r;7X_{!Ao06K7uD!}(SCH558$*uvlQ=z-F zQ^F^~c>Vaes!g8L(cPVP7eC3@RdPgiplRY8W=zoS5_#NW5VL{gXcYYKry!LZ8)lt$ zxDY-n3fgLrI#@d>))}|J-v>p2PaQlP66>5twRsnmk<JLprk8>lzmgnPPDHw(+FS&F zp!n}Hxe)Kq*aiO5|9$G;CBTxSz}8`t$1ur6B!d4w_3si^C<0<sJM;?OMX3M(8;df6 zWH)09g*0yZvytzy*(;@XjOLGy*iVVHjK0L8*Z$v*{2S}0?W`s5{F61Xf|1WDneOse zNqf%qJ8R<X)Slg|%{0b?!?o<k1q!taP0Az|E*_CU?;M9Mo>}<ctL7Q~rN&1MmSi9P z@3sGX=G%=#Z~C*=>($mtaknpzXO-<CnuQ&|e(9FC7P&q1dyIMP5w74%J#N3Ve<Vm! z!1uWi+&v_b63krNRK#;KA#k$Gg;ZfZT^|8ZuN@>PBm3zZYg0X51^yMR%u$%{(h;!8 zMM`|Q?gP8sZ|>+a*!g^YkBP+Y&OYtI-0#i=Xz)@B{C%xlWQXD9<+g?VGY#m^C3fmt zqkQ5QI(cGHpHum{E}Yda5L(!sy<S+Jdu8LhumEYehr3cyZZ0wkZF(&&;9B?sTJ$Y) z(bg|sIUF5jzdpDC&tI>yZ*2k>3n912chY2LxjNbZ{h?mIz+`#f%#&(fu=)5-8l47& zMImMziv&IsC9zs7wZ}yRRw78bsb6qtly_}zBC~nlAYdX=?O(2BCR7wVBQ{SbyHynB zHx}|GS9<33#x+OW4fDx1v&m}T;9J?@%?ru7-JBmve!8$j<!Lz0LdvOdJL|Z;iWU`O z4{Rg9U*5A+JgrQLzf?fDkI`s)QMpo2g{}QpozN1wW+@9%cB&+O!h1L)Z$tf;Nh=e# z=Bnmq&%L*R5r;er${?BP#wCeUK1jN(HWFD4DqrWlU|wup{%BL+N;DxL(x%hEyu{yl zchDGl3eqXQMr|U|1DpnMJV5oNSyrOEmM#u10ZV}!EC*!cq?Nwy5c?<<tfC==FilLJ zl0&`xzN!0-iHSeA1^!4rIERjncOlt+@;kBwE(NBpR{7c!MtTnCh-gim=mLBd1tvZo zxFP91f^Mq(CT?aFvO@i!L;V`9`F>Xmk|J5Yg}JcF4$&Zeec;?6svU+Wce4`puEzIX z+Ujh7ln?QH+(3L>BNTz3sh8Os{ebB97g?HL9ff3bvBl-;S9Dmot^VA9_tnkSPa7=y z9_2r~@>E54vr&V*7&59_<nv7~8*?%hFSIawvfx6PRD^_7J2L-ZiaPi+me-e1KOb1{ z<CRt~{zDN*l2sCWPEDJZ7kd(t=dVt`E7eEhgejcD6e=P#Gh-_vBfrZ$Mv#r<le<#_ zE{*d4UQZAsj?t!z!pL9DRf4kmJ4*N7UazHGfj_3=0G>i*q;iLOee8nc_Cxx#D}X!z z3~lIwr@Paoz>O>?ZuFo%GDr7yOz1$b^ncZo2v0}E4@|Hwe!Sb2?_P6wnPx*!;sRFe z#6D&DULD4yq#_TX6fR(u&af{#kVPnLRv{Gl3do}oK&t<MQ-@K;j!MquUR&J5S@OfT zxBnnjV>0|_$9yYE7QePo&pU-gcnik=zLBVM=wt^qprFf1hsRMEzqVM9s{(0O;}Eu! z<E#Ja&_RqK*7d@sL$U)?OSfAU?IrEr=_C63N{<8jgQMS^BpaUY8-@zq78(bz|Ks6% ziIsH2pjfOY=;a$c+&p_}FOy}e!S^!EB+-o?0!^KcfqY#_Bt*ap{iIn<(3~v-BB^G} zAYZnzWcV9c{?}3NCB$CcG#&dt9f-b(jDjLZbv{~36s|AOEfl6tI;YW`b6@D9(%Sdk z6R}kB@^=Q3Cl|9i_=xv^TjCXjN&z9FQQl5ZD4i(LEmv*CESR+ERJmGLJPGgi<|3WD zki28Q5A7YnyRixs4`oM-ZUedQiD@q0joklJ`l{n&2`Voy+5~|EKP_sVUP`2K7HjI; zo^PVL6!Y_(`CZ^;Wzm{dx?JGwltU5K<~sG};(u@$K}_Pb&{^S{kigp3Wkt^NFn>Sr zpX-&!=bgj48N*qE;a$VX`tk^58YN{(N9B-lTz*wdyZX-X{|^`2^)O_4c#Fqcz=Bs^ z0M9O(Vv+og&+Klole*>96j`!IEc|$m@8St;VFO6;?ezCpqW+^y$oN_q^Ptar$MXky z1P>oi^}iAV(+>~+4w3Ba|07C5D1@#J%<DXQ`s20aA{naJ_2_O#Hs+o83p$S+x~#Zq zzLEOtmT#zl8x5=NG*W*H+QsV?#+v&^IjrQHNDB3A>YrCGwm5?s_g_(i){lEKgJUBi z{u84~NMP==51CGPF>UK}zws;JvHAWNw4`>VrtQssNl8AWzYXs(N+3$|7L*Aru_N2Y z)pJ}lZ(`rmS>&H!DI)mfEI&w?bEoN&)J4xVd2Dmt_`A4Xn%LTsDbyihrjV4DmZli` z`z+>ju6$giZO{~bp7ar^;HN3nWE-I(-$a8i(p)t9m&gI^OK3N)8;RuqxHw-De9F!I z(YZM+Gl8c|(9(u10iks2{-yyp5yJWB=qB4_`WEwEf2UQ{Lf58TB}Q8qGu!Vua}+_G z)j{{<>gnwttjDeFwexFhTOS1I)y9tBdqaL~ly$!cqeoQ$`qln|5RrnZ0^)3M6z|mj zJs}@6NHFDOSw3)*H_$v<4!Y{A(5-81kO&w0pN`y+uF(Cbh2|jJF5yMH>=A_%5<c?H z=W{B4Sfd1Ktk2rT1*WU3YwE{OM<jb3%vKygbT)Bw!&1#^HTDE=Z@1RDGJ}kK@5#(4 zIP{j+Puso2oC*R6IrPS-gg&_lzPZKOW4kw<t7>Rp8K=jyi(hc!ei;Sryz>mmTIt|x za>CsDsR6xJR-elcEy^oqtyneQzzz|T&8TW^iwLxyZ-N^d1w>E%^rN5%Q0bH8J9!G4 zm3{O-?t-!ws>Pl2#sjf+jeji!KVu`89EsYNmTsOqKRPz2@aos&=b!7I7^Jd1AhqYL zoICd2tj&|IZ_C8`jL*Qn-a2VqJ%2ox{SdzL_-D-hVxNVc3k>8L<G(;xTWh$&J`VxM z@p15wm~37-gnRBiLN6B==TYC~iCOlScJb*P!pKnu(ZWpI1xeJu&C!Ud8KVyQq)pQN z{b1yDM#Ns$x3epPt3e}4wg<Mn_0A(7tfB@6O}a1tP`QT6Pc;bvCw_tNk+aRPJuMs( zgb+U_uwd?2)8OY&pv<U08b$6n{m7P$qnmgsZ#UxGa4Xv~D==?vhZTJxV0LRo<|TU; zdiU#gLWc?q@MA2~ArGH6(qRc9tZ!OiUcQFdku6RUn&6SOvan|&uL8n5W7>s-yj-LA zJhXKTq-C|RoKK|M)EZ9&&o>xGB(>l-ygA}A7{3;l`%Qb$TgG}YRL4ya+L$y?K(;oV zjXxtHnlIjNOUVi5cOVLpm?Qft3(b^qWM@vPJ^FZ~+_YxtwT$zN&|Sx6=Bu6N2{};h zxH?g1EDk>i)7I0PkKzUPfJ1)xR<qL5PQOeb=_M@137e%_z@MwXnBm_#b3p&CG6n~K z-n#qpb?!Vm%dN^!A^djld9-Q_zuAsC0&<ZT=(jB0{?(PmEoJ==<a?W4&oSiRNl0J_ zv8SeT%jKw<msk3A?q4h{!z!+3&I$6dZGF@BF^;eJ1ngHUHt3twVGcLtjFm~?QIb`J zicwgVhw?G<PBo6MNkx^m-{>>Fqk|0i7k8Cz19)tITX1r^D?7UV*Kik)bgDQb*}uNN z)g!=Bvg=i*7aNj%-lm4bVs2F5((;|jOHZ2g==bXJ6`pKLu~FNiuY+9l_9a%@0+z_{ zyNA&U%=*xb12P;Be!<Pik?xNt^Df^WH^1K=#V=l9MT1XPKPOmEA9|Lo#$y55LS#DL zc1v{n=C*ECo~^E8q==l2$S^UE9?G&DIH|5Qpd$$~>tn&cT~>@#dZ__^mm>8PD|JoU zNF~1?9`E9PP({ia7JRqI3C{UeANZ@!&{juIf0WH=$$v!EOFAHyzbk{fj$LvsRATQt zBr4zn&L6+YyL3GJQ7)5963<C1Aa*m7*zzT}R>rsfpTeU~iaeX0bj)r@69<AcSZ+Z5 zRu!7QX>lp==iRzLOKQ)eQ~2vLbVnIdG?p!v-Ygc_@k`Pw0cA#xB~_R<Rs^A2^ANU% zNd*?wR=uD5ixC@wH7`a`qJGXNsVuc(Cq?=|&9&Ui0%9vHx!kG=7Y^mJuL0WC;pVW+ zs!rlK(=d-E&Fc^3kOtbpYEdlqLG0;<y(;F0TSP%mHNT3UYtGnzBHY~lDbWvh%EVEs z<RNFcSaN403^kZpY;qOkVeIx<9YwRMx+Z4i96_ZxKFK>UOS{SEhuUtF6p+e*sPMOM zur=(wLi#zcPeM0&m-SMErF`s?KA<N3L}T73$<V0Stg3OP(ijmx&Al#<MTooUugGhx z9#xy8^_$vex7_4=H@I(S{C@wJoA%?}pWdTRF6RJhEs0=>g6Z5XWfsBa<~1X2WA+mW zFITU=uLO`(IVLO~phu+EynHpd`HBHPitjH)kk?1I=@E})<W7K83*Oj3SJTx{6PV1P ztfi-HpwDr$tZsFjRXb=hwS@&XwaNntmuQe?>ONGi$@Y-EbG3DBQf*Tnd{Vsp2F7Sp zhH41LR9VjX<{UYEL$&rrCix(c?^4yFDKFqbNt5}FxIL$2`Oi-CdP!!j^^j}|@ghDT z1)vAuZ2GV_k}XIsSr3L%;6yZuc$EO(5-2|7{&wU_L!=|YUE7HocZJsV=dDsY&aAeO z72}0|H34418aq@jxJfi9abf=Ad88CL_j+Vr<*_0ho#dCZS-wdy1x#C~eQdId-3nHW zV{%O3YNBe-hixR%@$jejv1n4<5C3|KBj)&ECS`p0Z1?EUlD(FbUzk||AstAAK3r0N z%|gi~RqS@6kC7lRpvuq>(52~K5XI6+Q`Tf|$%thT8uB(l-?$-~p1B|i7tyjAM=iaO zhBcP{$Mj@ePo{gR*Mq%SLEp|Giz{nkr!(bf+F;C3yu^kvPw?bEGSBkN6ekbO`$G6Y zf-aPUX=!dLBD%tGyIJWUlP~<k+FgS!3+XS1sG;tuvr!tY$%2#LPz~U{u_E~l{cHwf z#B1Q{Yk&ZI11?$npRE<pGT79l0zyx(|09r%$|9=%+hZxax^SH_>R|<}qFH6<s#i*A ze33X>pru5F8`w@#eYz>7d*;vpAqTwn7fc4co5#?D5nY+{K&s%Wf{?l9B!Kh&g6|<% z+M=yQq2NXO5|D^K3Cl_LWpu$y=Mz((wRScGCo!j_(iy*11#s>jCp}%a6|obJ1XKKF ziIe1C+LF!s67BS%b#EW?{Z+K3?d@^?uEYQFhu)tCsdvF%i&JXiX@sy<KX+;!MsCDq z*qO;P@j6B}-%AdY=*8?>gH>MS)oNA)SPK(-_X`(X*C2u^kl&cVh+MaFz>(R-#YJkw z;KY~9J-Umr`Acb_2H_jDod<V%(<<lZRPU}N+adZ*#<J6M(Ghp<J0h<)p`rccQIR8L zq20khE54TkUw{F9jz<VhFd6-?^Ia#AIbo}t`qI0%O;vIXOXiC5$$*dcj->()0+%qw z2z<DDOg%<J>1#siUk~5Af7Lb<NR+1at~)GL(;3HsX=lXn-HGftuz@Jh0mh|A*(trI zFD-#!A6=C2tAfurd1zr`-|Q^qveI1KaCxrg(t41MBYyCAB*C|$WhacFGlrD23wl82 zxBu!{rl1p_`E`4>*vj4vp#_zCHc1lvR#5oW*cPW)dypDuuvE<e@gKZbfiBTa*-DAl zQZ(jbWG(ZqZ*rvstyH_$GZ;Y>QU5|eZ@!)k|H%FL%L3?;Z2fQy#C!<ma#p`K0*Qm) z=|cZlDpoZ1$w9M6Rb9n}_?rt?ou50abZ0Sw-0&rthtH|<7vp=`W>doDmh@l)rx4-J zc8Jsz@$eWM)o>uBXHsYTAt_+YAJUVIJ{~!@dMQQnkmX#FfFxMTLT0#%@Q*mGQ$}!M zt3^dQTCH-flF~JQ^zpnZ^8H7X{xsjR32v$6yT`&!5-2=s5DF5seiC}K8;`=ibmM9@ z6kx#_K?ncBfu3+Xs#|fS(w46RG_BGZ{9UK4teofDV@P8OL@FM(y%@OVP?&SPf+Pdx zmI7-mDk6~Ojz=%7I<F{!ms|4^7I{>1MiBpTz-exnEMt-_<CXq+rH;8>=|168xcMRs z^9u7VC)s;(Imv0FCq-Qw9Z2SzZ^HI9I*$w}*qHxCNp)CijAmTv(GS;~^I#w%4o2ba zC<Irog|Yr6tO(_g3tFL&p>>*O>A?S^=`RP3CWOiLD2_+!tbJsAG8BCVkT-Czr(&^{ zGI<~cg0YV<d=9>fhude}2T)}q@2z_Ria}3;Q^&!H-9Fj$uzWL6`HZPb8Zqf$NtNHF zPa*l8Ezz+1z-cQ>p_`oM35Pf*JANNaF@6T}XX+tuIfO$PqdwD>z8r$IfVru!y!zv1 zMt2BgeD1G3wU_tTTOQ-L6bC?I6R)j}AznUV;&Z?L$6eT8U)0DedTOh0bQE(jOW@>r zVO2|up4XA>I(_i<Q`RFwI!-f?0Er|@u1Xw^k2;dXX-tGyW{Gq)dJqtg7MS4v)r9T6 z#(h!m9uMCw<QH}f`S-ZCT-9i*R}X;ZvZgxvD;d7KjZb0k)D}?<EgG_4=*tIs{y?kT zIPwIiHZeeSzt+2P6MoD+RnDIx#)yeFv0J|Nh0QK6_g(ei#rf-;U10rieMe)43($31 z76K_l8^0em#Ed_F#J4B~;U?g0`r(Ez^eLmA8uTkW-L@%)8*@*o9XA<IlJ=MiJHPj| zVJ}##9LqND<BG8f^MY}5NUv!#8e1l0Uo%Xn@IBEs82?7{h%o%C4HiH-qi#2svS8OV zT&n9vYV$ozID@1nnBXWIQVbtSZ3ua?`p@z3DQE_H*{XSin(V70*#&cC=W3sEL!>ax zJ{Ka!#1E-T1QzV!6n?%}YfTK%F{;=R{eRYfl3+wd{&aoUdLMYTMDZA;Q1D$p<c8vL zzx|7T$*QfMkf$dy5siQ!JIFHybbE^ovX%5#fch-lUj1c@rU+mQ%IQ!E%cR;D+;#_A zke~YvcDN3O6R85UzUsSNpb$6&menp_n(7R$k50ACc@XlTKt?w27WA1JXU7PIkWQ{M z4Bw49<-OVA6}>3QY0At%p=r3bI#&Wuv*+i0ZD!MkHD)ZJ(SXhbK|kTSetJie;N%1* zRwU|BnK57TL3~Yu+!+NZdG+pLfrN-@XBpkq^2dB)n7f-CW-c!^zrKOhsyTz~MS>-v zQ?8@vWzQHE`Nq%vhH+f=-Q9yuH?h0O<u?CIMFf#wx;h5I-Ftl-MPt^5%VPY)E^o#c zfJ$k-Gp{HTGJbnLDc&Nmi!=m3Jj{gi=2W##dD~^Tg5lYC1Dx0TjWVX7u6k_giVRCn zBE~dHaPZ2}O0t-!A5zpzjEhO|9@T1qd3~3DX=k>Q<cBDey5>NUnhEC-1qV=zwHIim zKV)`p?tM>An*rm&0UDneSHpG{2LWe;zGQUhYqo(zeGjtUr1JTi*e^}^f9*8c<!XPA zLrB1Z>8&N3!b712YxiPO{-i;hEF{W)uKo^qqDDk(4oI)4F*9BbnF=IzIc0o1a%z)d zl)IQZd!d>#KP#>p<dQ}(xZ_(rs&C=!6(cb4Xv`@BvgV&-nNja-$s^N0f*EgT#meQV z{S`a7ovNsCiLLV`^CgE-t4BMeth_w0JP+M$<Bj{=Ng@4<ZW=Qbr0%N&RU>hFt@ZN- z-olx+xynkSxB=`9K|l7^0$CZGFDd21w_&myX4#)DoP`8MdnpXhOt`Yl*=a_2nwwBq zABg6x6qJ6ypfeiHsd-nywgTrcbn60RWF7t(KpR3#6P}Yu<ij2=psMSwuC9;CTQVEX z*@2MS#eudqH&&(uZjq7OGY_V&W{b9i%Iv=$qw<K&-_mbiIFc$#v(jm)8?L;Mj%E6~ z`<9&zJ#47y?M%bq^Yyu~*!LO~jrp2Xx8*<a%VP8PZb+ArM<NJDEW}z0wutV~hm=&I z!s@GuR`h||+*mi(A<kBvMNqtyh1oCZE*<anX>Z5M;q75HwKJQ?BvXf;P9QBWrrml; zhLZ4J+~E@q&Yp}J@qJ;DkUs$P-v7LWlocsC)tPXVCVhmPL1;38wnV!JSKcBoGCmK+ zjBzITQ@=-fl%#Rrink&2-t6wkgW*M(X7~6D!c=d^-5oRY?zNo^c%|+w7m1#QFSU-? za0kKsg7}t6Q*7^7y1s?8+4O3F3qB+LbYs7DPX(ZzXpqaYxDN^wpJi2$ng!X<kqGwl zTPKz7?|m2cwfatC!V)`K#TDi8moU3xf2XOnFv=2je`CBq?l~r*YxO-f#645jEZu1O znN!9Yj<227@V*nCCg~TZt%arc2%YGD?aQ8NT3Q-%Ry}An0$XK(IP{m|vZD0b%*adI zMcT)tA$qA|(vWn$_dZzJtp}8{1b1?QuTz7%>Xf39*e#kjHNd`m2^}esIci&mVABFp zdS!LPf;HpJvX-Ke%9!*0=GcI3It7Flps=y<BUdb`^I`$Lrv`avOo@AO{#D8{%mMl3 z>`no&QR0l@y}>A^699vOs1jB<6-wtMC#ArXn<H$x)|JG|tG(qc1ty5&(0@$OtSDGB z`y&CnN}RAUdBB^nv77tO`>;6Kmkt?bFXFTX<#Z%Kz7g>mQ)SXt4z@yQ2jY5*sk)xT zJ7`N<7B=ai^>1VusZkT>^ic)4|NVw6c+wL_?IzJ*C&Ra_z!6zQl35sC&Wkv@3e*RR zW^<XPuUraWIlZs=x<sUf3HK!|oAfYu{>o7J=QmElT7y+Rr(Zarcq=;uKi~6kKG<=5 zl<_3arZQN!=urE6{-CuTFP>k0h9We4Q=k~tEVF2@x?xgCo8`!usX9FPO5x3Lu;Ot) zcwryaa$5fR<3N}!X~^FhOw$~Kw8ec>_GHzyo+<(bP+;kRFJ38|W9utX;SA8!^ss5l zQLnM)Mg)V0+DNG%?vm<BptMuG+ktP_r3DRIw|_Sqf_E2j&nSz1mr>ZCx`L2&5;k!^ z&+9}vXid(lv;R}28YTx#qJUYYG*>5%9Mgk*eo;;XX~i(6#SU844f?VAt1l<VueMVZ zhsV}_E$kEX^i{R_{J||~LHWBY>GV2|8b_(zfXU@j0sJl$Quc=u#G(oQYq*Z`Zw<#G z2?`ieVs}rPzL?m0vBSzR724+%nt%}FT&yG&7|dd8bG(;(Y!x6^^f?8oIh$WdC)?|Z zVjtO-iD7Q@J$)gOl}{BrU@Au_p2YyON~<?v>W+crm(pQ;fNGbVycXe2LMmZ~Q!<?d z|1`jxxe!&T|2C-aQq<O(#f6&$-=2P60W>@>fSfe4?Uogk-%**UrlV)ec+*gz<oCQ@ zcTE}W6!!7Tq8o*^j|5GWaD1=|xc#vWfnqdXP1=5?nPVp8nD{9Jv~ZppegDUKhNLE8 z)zY0hag^&+^_UK%Dg)70C~{ALJ8p0*Y>|;3@)F!H)MPI8&bknVEz-ZMm>aJMeDE$! zBOBLIg*xv))A}cCw>!%~5)`hk@u(JEB7te^$x@}r(i=(%zS~Yz5{@8cYHr#^Gihbw z9<r7T(Ta^K;}LdEvoR+_^+QBoMoA7Bga?cv#!12XU|*o5RD{>wlj~uz6p<F_r$xX? z6J7~<&z|B?WdW7dyPrjdfHk%qe(sXw?u>w#heG7a)Xh)Q`8Q1phcfTTk?5gv1odFv zn#+kYh*BzUF62jWESmEZ{LD{bE36p7<YXu1deH_7Su)}kchu$7-=U@}$En&;k{FrU z?NhW!#tZY+Nd!?vQ>spO>>NuCq<5CZlRiTn0<B;LTf@F+se*!&)6j(s2>+2|dJX+? zV~Q9obPH`)^D3@1@k4NyBJAq!oRR>?TOLY`rbSu}5t}lz-*)55jifY~x0@YUmaU!S zv{lai)`pDPFctGpj^8w~++RRTmer`8eG~;LtDwSHqDo-RQ8V01)<7+ge6@v>`Rzjy zlR~d8-hhNen9as{%+EK>_OA(DCm9W574_AI9l}Uz+bHcPvCj%kcHL-J<>K$1HXcfd zVY4gC&_%*PRX(iWbOe#2n%wWVsoi$~*d4bzKlZ#I=3esM?aj0>caq9g2OL4Fwn<qV z$6a6%&Z#;nzPBz9-gQ}cJoyTGaL?d_sd<E6_g|%vx1`Pm{D_yTcj8K4?YDXd-W-KE zA0f1Mtr!5K2tdu!MBLg`n*~?gpSq8Ozm);kl;F(4_gsg`f}PUj#9t+yATcK`6@G!H z2d@**AU<8D!lrvmC&+vfqt29ozBT)GA5s+O!uD7N#G_=^nGR&)NAQ`PMEE=Q$)GY` zt)ABGJpa{hn^os-e?+@qc>BX;_D1F75~9%^mS4c;3{zR${fWo;klYYH^Y+OEWEY>R zP7vWt2G<3SrKEjck~i0N$CNv_XvN37GC5AQAq(E+;C@&uGFpr1bB}}D)V&yz1fO<t zyLzOO3z6ytJF<s#DV#FA<6!{-tl-`xfr^Ugw;ZP@Zvyx@i!YPnW3a}rqDw3Q*%POn zLK6N^ZjztVkq${@)~lNLiB%}8(o0^=jB@NdYXs3#gIvhNhn{M#P}c1n5?AAQxRUHO z(fS9>>7}>)_XBN=B%{VRb1!ut>k#c9w*oF!+JhB;^(;@@9X6d$8rKRLLv*T0!z2r) zpSY&aBep-VyEm4NUMc)J-A7eOiKl?crG#JIQe+)7gO1b)c*Ot<6!JzKW=m8vwDq-I zEQ1hz^Pojb=WqK_(C_v)0hUODui_~^aLWW@kqf?#hok+TG=_{w&{uc9$Li#x*ujti zmjU6cMk_)@Yf{`$R%|<H3@lMnwba3zX|_Wn0+L0R)n+NrNn|z5$zV3IG~4+ueVkfS zZE0@hG4)+3%+{?jb>U((3rjWX{=d>k`uIxtxBc2-29g3_9Fp$p*x!pY^xXLQc_aA+ z$w8mku28}+A1zal>TZD0hL?;-K={45jHmsz-8=B;*e5aJwiZd+zqtdu`8n3t!j2w3 z%^Z_z1|^ir(b~>w3T6}SY+KBg-MC7sft_ee9nUO^`c_j^9pUwP2p3_7tR(rNdr3UW zJWf(y!&z+kWQxRpY=5#Ff0oTQdF-a{>biR~vgEr)HrJo1@Sa+ND}NGJR#};+&B`^z z!cq!oKg`0JWLoymhsMmAwJ_217aGCTb5%`A@B(v!N2&ZyT$B@TlehJ{bV6e|co%rQ zGfgRQ)VFwwZ8(1k%OPaCgp#u@Z@s13gR}mVVz{^mpzUm(?-|M?2z(xW4=6(Td%-z8 z_e#``8q}*v;B01Fb=C^Ews=9VanK~mH;Z+;`E2_q&J|A$STqSw^t_jwgdp(^$_R$! zsq0I_A*b#Y=`~%jD8WB)wf`3J&j&!RpT%HA<E-4RFV1?Tbq|{2k^CT0W4<tdOA}NV z=KY)Xv1Di1z*{Zs!DN6m^@xLWdAyXc2gs}AAeLk4(pfQprWF2{4JZ-0Y$(dnV52!a zrG7s{xe@CeVGv&S5l0}MR3M~~sm7Iun-||{v;Lu)CAL4|JGnOUN*6y(;E-`(zxfWv z;a4zH9rzRznIN;4$J-U_E@;S>ukS<mi#R0deNK;i?LbZx6)-S-5vx>?#ucr*uucEk zcH1Ly2;U-ac*|+{ltb*27s#RP=%b@m)pNB@^R#X^J}JTSat;M`3%8HMvrq=}AQ-=1 zY@R#wr~~Aka1L#o=MVT?FD1uhY-I(tIwRCH%q-9h=;g~rGSQ%1U&SGLdK}Mfp<Q4$ z>dd4ALqx!=htnGmZj@y$SaD+PxH9EmWU|9KGQ%Zn1TCbfAq1#J)lBP{2-jks+fTRT zkJN^gQe~&Y-~AQRf6$llvzz!cjLe0Z&AUP;$b!(Cwdo$EneZtEqsCN}0=snM@LNu8 zid&;k2=0!#ss76MPs{mScIonfTH#R%83&a&_+fvD!sKqMsjrP^c}ql2W|_>#%ll`C zqpfd#Uhn{u!P$b241J9n;3Bq$1y66Sw5ndZ6?<FTL|hjdDsF`3IycAQox4D*cfuXb zusbyg@S_Vlo3Z=(ROgM;PpORz?HM0znquKR`HJ`$ACg~1C)}EE?SzY)BWn1(7(&N4 z%=af}nn$2JI06V5@_%VDm(xo(_gtlXD*E3N<N4waj?X(gsEdN`oKZMfslO4~g^$0O zySz>ruq|Gz_Dp^u!T26b4H6vwY>>(T;NID5&)r1uRR(#Ge2y5yfL!3C!a|he2n{P) zK!r61rXEK10X<GrIr@p7b}K;G`mP6+oq8Wjz)~--R~d}fPYK!TGT*8iiTPfJj8)gx zqDQI3_+9!v$`&@{lonx$v?9nAz0Nq8ZA?V+#OKxR!3}1kjc8zq*Ri!~!JNNe=Ah`~ z1}Jo*9Ye5-v%0X=9=vx$A}i~oSJbnrHGMGJ&yVKJ_(YbqRkz^|H;Nf1nH(Jr;%&b> z(Ebv>^Q2B*3Hn$a3NT_H{>{OaonuskSW`+ME>xy(XzEF+g9K=U+y_FW7$c?lm+}5! zGv#(NI_o`Zi~Ny$`5_J5Ow1OC<OW@92gx*{>e7*AIT$cqWyl3CL>v!4bFmF{&`XXR zobuTnMJyn*vGKc`!PV~r=C_VAtgv1r`Fm7WPA~Iu2BlOkbG9#dpBx~zT|damfXT&t zmMKABBZBOzYyh$dW@R=M$>wjBjMw&)%z~zoO-ENM-Mw6mI*fdeGI`yR8k5*Cl@pPZ zMB$4X)0eLrQiXC1se&zjB(&3%hUXYvn@&`jl{k*tJ3*TDU7^tWui=^{q1qo1hyJ{M z(Yh5)>oL;U9%<@Py5{9Up%YJmA98cyncKu;B@8RGJ>n^|_LgWtbaQn}Te#^D`V{ND zu(U*jOw~1@ye(%+W!PFHy7ireu>#VbwM>`d%(Cem%GCVoqZKw5vVaBZU~G!GeV%e0 ziVU`qt_6^ml`;@uN(ht(S?Gdd1_#f09fH;EXeD!LrI(`VQY!lb%*Z2Degh@LMg~-E z6500l87Z8OR-<&d;F6=Zxh-PD%x<hw(8rBeaeFHe&oleP5cY-TYQMqO%J#P;grKzU z5(~3iIa+kDN=kL**%Z1uy=){fg=QTKKjpLNUS1}rAv-^`N&VUrN7Wk$eoY9_93a~n ziT5VrTwVG_RG&xw1T4OSJ2Oe}+6dn2r`c=@Bx;Qvt-w*=V3#9wkI%CW;Jr&d1vgSA zh<ULIZpKyMsk7R0rTA}smbAkil&*2E$RcCnf#x)aq0K$pU)UdfTc6`lP}SDDQdRK> zaf#pp3)5kEg13_gAi##Nm7a##t`}d%r^B<R#9i-FW{OU8_1m*h{fC++_8EJUGDC%U z<YXjCaiy3K=wcVPwBKYzK)Fe6-OIAvX))?F5tr<{ah(bHEVUCYiu2KAzyVpoVE>h% zHKzq-)FvMk<x9zxWvmupy1@y5u4@{9n(s30nz|9KIZM!h9GI!%oUsn=#VvlPWz4Pj zDynap10d#ZiAc#!0NNUgx3hN2#;Jvr9>+955o^wVSJhgJ5b+Z(P)WGIA63~)-$<JE z<qh%FUfh1m@~k9Z{!RTp3Oz2-jC@tbAOwTOA=r)=tyWAWI<|={i-?as$Jq++Ln-Rq z`iE{#1ijG^hVZx4*h$>Z9y2$!{!O0@Q$$Cmj$ewDB&*2OH#uv!|MiDp{rDx&O0c`} z@=<}lLVZh!!_KBL!MT$k1~E^D4oKlB#~DVyUKh82va44HaE;ucEIH`?ahGkiwXGC5 z(INd%3r@gKoRw-Io+=kuVvUUn#UA_tJ}Jm#vEk(smPo`8`F&C4)*RKEP!7H|zrG?{ ziZQtJW1jum_J&qNpzNIxh79f$m$)p^G7IQz06vfSeq(ZDN938_!oo0{NQ1v!tdJDx zD8<GTVpI9N?5$I!0lo2Ls+=^|Q{tR5o5Cd|Q$*GFR_A|#y8U2lmH8*v%@hYL1ax;Q z65W&FKd8kf#TiO-PgO%F9bWy7$BB!(m5q?BgVrQCKvt4TfgvEoZ1hPJia59fw9^k@ zCy;(dq9LWV<?OBy6pg9`Nj*;)bS+qQrU3<&kf*kUtd1kc$(7Z_!vm+t%E<B6M3oG3 zBxeP6dIwt~PHT7H=kLXk=`ILOEYFQ78S-%QM~1{?nK~()_`<G!bQVC)tdrGsckJ?P z-hsV)J~MbE!^OGL5A9n)1k{-DruR-_eT(ft;{0+0jVbY+NF2aZ04YXGB0Rr6{FlP% z%B^y~#4kA$Xd+w}rGV3jP;$`rarb3&6DdIevH3c2z8MVeo{HCQRa4JOo4QCXZ;qpb zS0hjR!}|hq%mi9{4T~iKhd<all8{D1_@kzQ=D84T7hZ1vl2`dzwN-Eugf#2O<gj=^ zuIugOtf)@+UQ0xv4>M>ZVJedU%G=jn*xBm$Zr@Nc-|!yFV|y&z@SM9guX5$Owm=7V z%*rNiszKS62N~b1hVGF{mb|_*-+}rCc(e@@cpf5$=|&R=#tRhXv|ZJ1NrHEkQtsrT zX*Nng_4nS)C=uR_tzgo|9u!Z~N|b6MiVT~fWbN*DP#)u)r7v7f{agXCfv1aS1_mfZ zZ$k%l;~e(EK<R!gB;1(|8F!}LoN}545d`fJe0ZuBP+q|tqLn%dL6kN67IYvN_x0VS z=QLO$MJ*|_-GePO`s7((@(?W|K%$Q?UTaDe4e%0;pm4~h4sqO#D0crD6e=uQdjrto zt07&5P}+1`#tZLPDg|jQQqv8!5+-WU+!S0`@e{?PpA2`Y8%T$!X$Z?hljH(>k~a;& z7piU*kz#ga6=s*_rwR+ob0TxbeVi)>^YVZdG7ZlJt2r&qiv)Zp?;e=aa*%(o?c_;B zV59Pq^avo1`G-ajeJmM-6YjOM>?FF(tbpcIM-ilFaDUPnjKO_W;Gc!Tlpfpp*XvVi z(ApmPa2$<ajK}kdEj*{|@A$v~j(|b?p8w#lU*P)s`n0XOI-YGj#2fOnhm~&-n?o{_ zA4D{h$A3)6Sw|ECohQ(fx*hBbB;HS(NWKn)U`?tuBf#W5fti4l3>Y(L;Z7cb?bias z{&1KA@@DAjsyU%hh~6wi!!WmEF=TwfZPkte3S|tw26Shq$-M=aR%$ykMspO*vIw18 zCsh8K94zH6(20@hj;GKlBZQsd9@bM<0+3M8vnR1ZY+PbZn@}*L<)rbtEZ71G`v+qv z#NAj!*F<gh0}*Q279<)IX5YEg1Dsrw_kz<*mCNA!xsYEygauS6z>w%F(lrR7-eaX^ zde8}{8?Y_R$pkSckV9*@XeXsN(f2cAM>Piz!O+}T*_<kWQ{M=}+zM)iUN+5lZt2uU z^Wq-wi*j1CPJUtbg@sm>9j<Ne1`$SMVpTUagx@20-UlMoh#b6TAequ@9!_4mXFE8s z=dLt*%xW;Pw~R^iZD=z<Irx2qPTlafrfq=7P;UOF%F2!Al?tcSi?bKfXVg-bLy`g8 z1T;<87#7;iM#d$NmgWaIn}lUANi59~*v$jTm-DG`&^RZJ6G>L<nwqxs!Dqtqfi5Vo z@^DU%Ja+3`i$fVR^4`s3hvg5}vne200!?cm8pb~0O@49H%lVi}bvPj|WD;x_@26=- zZ&e9vXM(KcTyP2d-jSY&ihK=xr=oVL_u=wwpv*3k?KsxFhs5=C&mvfcR5^#NhUaT* z%nqAOYF|cbrmOQ%BD!xAmLgc}jv1@zJT5O15iVVj0uwW`fQldX26z%svCR>~Quxi} z_hjX8t*i)NC^CO0Y4GD9Sv)~>sVkniv_*#4I;A3yV8BId*g>fuh9uX8`SudH<k}w4 z{HMjJEHkC-COVlAEB!+QqxfJb$OK<<cgU<D+D;4DtJpW5-{nzR1!*Kth2;WmUcL&9 zl<Ou{DiqiS5fWjXhBBP+3pRUxoS$FB6+Y(U0CoNV+dLQicvcjL6#Nb==>s{v;xmhV zoZm#l5t|nmzu$jF;u5Lo$P;@cjP$-2p>MlRdt>42`EqF}g|p#Xa6v&5A%?21vBwhH z{z82rXS*-@e)@LS<N6=UgYrY|U8}9+1`cd+ruZVUxC?LS4}FZiCVq`t#~y=M+sW*; zbOI#OLEr2=Mg7b8dWli_Mqba2v%d_APagwI&sY=F`+A~UUQ%w$(qUw+$qz><%_Prp z^Og9^{`#c1y^Rsl^E~AT;@lK0@^0vN_3^LE0l_V~5bq3)<6m(GkW+e)qA14?9HtCi z&<l1Uf=m|~1v?baf#>6RNl3V2lav%tDGvV`9%P0&jz;|)7V}z{$$`=o!7qgG^4B|% z{Z4^^mwX~qI?V6lqi(5K3%o=?-`BV0AnkO_R`fe)uHfajp%OSSBb>r0VOGC7LHLN3 zXnQ#BcIg~YUmIicYv!HPCXYmP#?MU@?QZmCE@FY)X=<Ave7*51QB@K%<Gy4D_h2by z&%{h56(-_Bbxn{7U6Od6W{->qe)`r&o5bm=&z0N_E@^CM=idv2d0F;nE9@Nmbp5xR zo`f`w2D}PkWz}wt^9N1lCQ(W-o06JLN{QipXfa*r2g?sbHlh0#v1&19b4QX^ygkIv zwLX!`Cj^Fy(jYyYikUoI|BiDkmCLzw{@w)1_lSpoT0t*YLPDOSxFS=z8rbQ<|61g) z5cqcNjcTA1&QLQGYvT?u3W1T$;Trs6?xL|R@+@TC9w5kHa>Xqoz4o#CM@F1x8ab7g zXyh6}_c<@`I{AMO-=vVt&vIrncI`T?_1pVlE#>3zrlsMszLwFZmBDBu^<OZR9#H1x zcKbx=U8BS5m5hBJ{5e2Hn&b}>$$Vv=^<r?NSWuE&(3`&=1QPM?i|GfQN~FT?(r6v% zK<S(33I(LR%+?_3L*CHIW5-?l1KAeK)xC!#xfgMW5y4dE5YsuE3btCR54=?J)9Il< z=hd$4yPF`|{WiPzEq)ybwPUfFKY8))7o6xp1k3rIVtmNunrKBT$d5e1kG!l+2WqNg zcl02_cwlfZ2p~C1`d8gNRx@LO-P(ZC-oi#cUtvMJsj&DdZpRYg@4_;BY&hrC60ZY* zVLE!Ws%EmQwIYt3Y>jUA28S0(NTol5j%4p0(n=t#yueJx>Km{pz5XWetlF9V$mEOg z*JQ;#f!b_I9<lI`iQ&NONN8O{?=c#pB`<yn4Imkse8`V5F)hYT=(fC^;})4jbzb3! z#cVJ$_fCBcmqFqZ<_(JGvvt3*e{^d!^z|S9R{fNHwS_QvlViC|5>tb2{_fT|gnZtE zaUN5HOwKVR|5CroU>{!bhm7Y$*|5j_`mS&fReB2EK?2b>AB%FntiUY_Oy((}{Sl6= zx*t*Ic-$mA%A^Jg?X23F6l)1)SrzV%L*=@^7Uo>l1}h*OI__Rolem7^wj=ifZ`uWs zk+Y7C@=6bE(Z^5B8ma~ze1aG1ql_V|8?+T3ufQEpdc(9BWeKa_ReZX@7?&I?hnP#l zu3iRp(M2|ZOyF{XPWhnlK?$Tecd<=egu3^<jHshI4C6{?lGJJ2k%NQsN~+2^?Ri)$ z4@HmY@af-i7oPzx)Rt-ZR7hB&ptCr(%a#)YXt-wA35XD{k7+@=0+Qb)u55UxC&ej@ zN-Aj5*{RZ-DYMIsExmq!m)_XTcbBO}jPR$kNPEy1NK!C!btUiB*t0hR!_U=KfGik@ znl!&qT||EO$8Yi^W8k6?6}k6U?ri%||5BO7zjr_0<=|9uUYHWM7xFs@kf`X3pIQNi z|GFxxXp9@ER~;d})B>i(_ASw0rXu5D3<1uH#lR?2(A{h$qckuy^;exFhXR6^9W#hz z?jv>JTg?D`P07VI=X|a|xg;kuF~&qBA&$DeU$QmhZIyKNEJARWbNH&1M#X9B9N`M4 z{+@l|dv-kJcc(k!A4Mf5tzl2?8qP{-*UlcG*Vm%Y$@E?d90`(stRGLH;+X=z_w>#0 z{6aU$`7HY<K(BmP082)oeifzYzszn}Z&AEE-r;4W<FzTvHW&y)YK{D=pA%r|8m*&J zs4-5WH2jltuP4zB;FnFayS^XQ9UK_lmkf9S?io0bQgo`K_Bjdu{2HZj!D}_s6SeX! zVx{tsP^|_2w(VyCJ#pgdJGgF0W5^%U4#pb~VP6`S#9X0V?9v@;)=$%kU#PppT?smy zt_!tkWoZ;=zVfZsm@2LJyzq!|a&H2RJGtvh5szAQIX{O{p&Pk`^{tvA<LgN^&|86S zGi8&5K@z!nL#O~#pp)(pW%i`FM1A$?`}#)svZSJ+7WGG`zOOx)gOLaudGEdo?{>J~ zq8HQ8=cMkg<Lzj&xNKBX5$^|?f2$Pe^W+klh8|t=in@luPQEz?yc6`-Nho>RA2B2$ z4oS#p(wN&=ejcHH4yE$)FV00Nu;Cqd=-9*1YwwE~RqMOGcF_=<uE>$7%)eifP<ap) z9D2`56rfOk-v}A62piIRQ)*N=Q&?y~;=*iu;~Y7(M7k4KZeu2@(Yxrhg;x7=U-EfT zzbyY>g%021E2_03jdU!!)p^3y&tEWStASae2C3-};8QrkFv-esw5;rM|M;x$o31IN zuIX!lt=W4mSNKtlX;)g>E=JNKSiw$qChvLVd8A1v@870bRM)wc5i(UPu6@VIn}Vfs z*`K{nO)|khmoq9rpuP?T)2~ocWcBKXVLnxWdVp=x*}wYffWN=@^d5x!BC*o_4O*Zo zMegOb;+e2l@A&x{Vso}3YpwL{x;2J~AG|?zL15!N;oUcnNw|@pcNn3c$ETd+cNkb8 zGh_F}H?(GXM8xD$W2>Adk(|qTOUl9x5yC|UmUyqHxSiDAKXn{~AHYzCxGsrsuYsXJ zh<3SlqD!~sBp(X2cCxCaHbe>%B5i+$_S_F6X`Q)bNz3|X_lkD;YyE2x43}b41@1uL ztq^KliZm}%Yio3~O|ce{+B#n3>rXbW$TY}b`96*M)>StR3it0_aL$xfVCjk{^p+#x z1S@ajjOiu4z&w>8Ew`{fNRHP^o^#NS>@atA7Cp47og5dLz*;%89v%|w&sgKxjx7JS zG0|>oWd>C{VMsOdgNyk*B4a3?rD_QBn*@X+wEFyHmhfA^zMF1>n~vcNZm?=%ihZr< zEmRZo1_&N5lT*{whv=oYPo|;BuvE_o{f_85)24U3!0hFiazPHi7VSzm+H0jwI&xK{ z8p3wd<Fzp&-*d28r5>@2?y~7w!)-(7-L{ufyKmUTj7X57%C67PsJ3MCZ`XSQx_*~* zcpS@;G(W`)svW}pNjrbNWp<*q_Ln@g0XKc07=IU|bSSw{Ok8=RM3hcFrqw7QUE*Tq zt{n_ZBNhzA@?3fm_zrr(0QR#@!`F4f&imNGCl3IJEjAD!*61aK5KXK3r+^s+LbP6L z>ItGZ8&iA3AwZQSA|S`=;BH<v9Vl+QQEBActv{AI>U-{Y*U}Dc(x)FIkyz=#=1#ML z+IJsyIzlBx`VTmar!5$yMMI|U55ZpPJ-+O0wQE*+0gWqxNvgD<vE{ao$xT)5&sWc% zdtl{cLvL1RFwI7}!vyvb?WWwJBqT3Re&_g|;y>IB5vFEMNECc^43IRTB1ywHc;i{a zuH4!GMim;%X>rlHs{3iF_L(fMcBbg;Qq<wO(!%|2{79F@mMl0t-qrlxqO8DlR*8X# zSHrQak7oGLgG4X9sSz0W{KBtAxdN?wimxFopuJjIB99Yn?wZOUk;>e~GW1`_GmF`3 zdqs|MWSqXTQ)r3vR;&l5sUzl_DJY#q=RNK-s68rdN)-mBk@`D@DHa7NvIOf`2C4t1 z<QEH<*fC}=RRY+emPo^$t`5lD4z?o^k(;qYS5%I?9qjkdT(`vl$0jV?NklkTSj=iA zTdj#^&8IkZWF-VQ2GH+_)8+>W*sK`;vrJVl2dn%Giee(~+k4eW|M8;;G4BV$L>ByS zwA+hQLLI3C*Ulx?;}y}So``Er5Xl)%>!>b%uBa|=l!9J<s?OLnEC6YsAAE^X1uwg8 zYSw)0yK`T|$8G)sDn9$R8-j?|_WqLrMb>!9c1DWL$WMt`#&Z!mQ!j<L2f-@cvq|sk zuadBGj=aL#g&&Z(!;n0K({R6X<2noR<Voy&B>)<H0i%ecj)DYSu!~r-I>K5zqc@G2 zr+_n<(2sdBWd=|0O7mh%36O3le#=Kl)8RQW#EqBo#C;H;Zm$jX6d7hSuuz=0mSq<V z`6-FI-^(*vY|b3X|Dyrj|HFkv)7GMZrsVsw!gu_==22$b*JhEz4~QBymH9Q|>KIWH z0>XxR77=;vh<!Pt12XUPXITVHJhd?gLtnh6;f@rs3SOVnBeFE-+>6Rs&!7B~=C4!A zKy6GDy!_x8%DD^S-L7A&R@@!f$eXel*pw;O<Q>3Q86_rDhf?!N<Xv*NqT|(lN3Xjv zO#`7NQ7j=~4hRWyR=(wG&uWaEA(3X+>Rekp-$6QgMUjXE*2H92gg&lvLUhT<MSS5Y zl`HFmr-YFZVeh*$U7%ScU`|cv4jKw_pt-?j-PmyWzKF8xqkRL){&4Y~o5Utu=V}w@ zguJuwvK5j1<EL*HW?mv#*}{FM`zth@6PPisc;rh&mcu2o*ck_1q;YVaA#+fQV+PX= zFxa*F<`HegoaN!<%z8lHe};uE<4aPueBcL-=wuTRX|e~a!J9Mth~Z|Gs0qx{UQo-= zDZF<H==Jnj_I3P!y!*mn!6ZXc3h~aOD)LH~lpjc;`n&<36oL`!k~@ngJ02H&>R)bg z(^`NA*!$z6HU(&im1t@+tby8hWIH)xTU}Wz46><?s`bREap|tdiu#3y6=6~CQ<!@a zGJap<(m3n49<r&%`sNR-A_<=$E_C~Z4zMQ6NOSy6J`<q1z@l+gl};N@r*MNJe?=h# z^b$6tC7s`${rtEAuVKZ02i$^nQKWwoe4;hbBP>WQ&M9|_ivV)Oty`sDFT@uIbBWew zv-M#9&pbCmt6$p9Mf3YoR~%^fl$vX6uvU=;ZBCYc#XSGz%aT?X$R{jOW{^u`rxoO9 z^7$tE=(dyleLbkQ);TpR&eX>T!3;M+ivEl-g(c<TQb#i)J(LX}yA`fY8TiQ)X_^Xg zyYM_Ufrw}^p?Th?ezxxi%259+gJIrpzim-&CPR^cNv6F6pWu%4_K=Uc2%bMV&+^Be zgZ;l58kgaoS_o!v8g!4y_51*4yWHz4O)@<Ke;eI>Dq&*V)L>zdvmD->eo-eCJY7L7 z7is98@qE~MfYoFe=aZpXfVVC(u}<Y?d<+eppXKYZx8KjKGONGTs-kb?6S$%_xZgAz zr~iWcSv?!&VzR+isKI<S5WT>gW1Kv<PKDT8^roxiW0y?NV&NuI?J#0#*A!5LejZVN zZ6`f|FEJe_&Xd^X$sKr(=SDR0T#noqv=<RFj)~PNj+xBZJfXhZ-v?Brj+k}v_f`y= zX0Jir7&ZbFS+NP{D(B54W-E7cw3dl@H+@v_1>>pxL4W)b1l<~CM`2+{dXS8j`I332 zy}$I5%I+%O{jeg@p!LG}lEbfCqJw!u>SwKW@89h<QxW^2zWF_d@`3)<iF*aNP=ph2 z$c7+{|Nih(rlGUHHJ~#uFE8yTm-mdGSqLJ}xs}ZWL}fTG#Mu~(7;`nsf&VpeJozXZ zQTLN9KY?W?K8iN(^?;n?jA<I_9$We%hN7oXS4#36_Oy0v3Klgk%ag)F+4(L#%@D?_ zW*>x7>6SRHWsIktP&`B)t%Mf)!T?HG0owYk%H-W2(zd}6_wnhdQFUC;e%p-k*A^03 za{`w!d*d@A!BkNs%nQ_zi)WU_HqRLctv~;yFSLXzdhvf2S8gFp+J;|vx_m&V$J=~O z4C%<>>{gt9&CWHb$r<(0%n^>~9!Pd-F7#y>{Qm_1D*)6S<kneRX2XKP2D@Aimw6Um z+Y)>nF>6|Ymxx+@yAWtLH^s0CPUSoj)<qQQP*Ez%DN4Nu<3IlirnaBqoalpNa*)A3 zJ^;tW*?qP+3BIw{a1FIVJv!Y`{Mm9@8gy^lG5!5l%mtp@Lyx#+<OSU0Z=fOhOZz(s zUCq(=@#oI;c3^mL0E)3U(2sXQJ26Zc;S|+y4^-qk)l0@<l1woerYB&Wo~$jiV+^vP z2e7w)j^IQW7Bo8eHHHhc1{9(R9RICNjr}2M$S(y@&`6PWdy!d;AQOl(#J#~Q%dGG& zp)Bb^CYoz_-`mivH0Xz)W1;H{*p&lt>Bl(7JJJq=b`-Ogi(=1}G&m=JLU8<R{L(pv z{s}WirmUPWG-1Keq!lg0Mhx~$U`lF&(Y64yBfuFZf?nr_MPY>9=z`tiX$p(c+0b@9 zZfLb;v`^UKwE3{$@pFcsAT>GQ?f(@~od(f4Kce16hGlX;GQUt;W(E2GZkZgjb0H*} z=(2B?#8PuZ0G|RnJdqB6JU1#ACYc>kH#vHX$5BaVIYmYUdHU%WaP~eXg?<25Qfw%e zu_0K-hil7hu#rNzF!2Vn!>@@nOf?jLwp^A5Lu)JaPriVy?K8M~D4%nNuj`MPAN(HL z-X9_D`kwH^$LYL8{bQGyY5x}eZy#Z%^)+lg9k35{5xP0UK_o~0whdD)9hiC3580bR z2I=bo!r6Vc$lnZL=IytzwtNnEM+eNkQW(4C7xc)Hm|w=xe&fM9UvVeWPz^64MrU!; zNE*l)Mv)orM0{o#am84TNZm3k<Rd7KzC}Vk(eS>vA*`50^qTg1e7#>|q5C`7MxT<S ziD#VS?R|pj={7i~p20o!l)*Xv7_P~)`?o<q@D&2%f5EqJM=&vMf=p`Rgc*q$(isb0 z4iR}E(l9xqgv;i~oGZX7lS4}Ar?s&0A`Kpx^k%#mwL@X?!Q@)t3?o5pvSPXK8R8}b zQ{>MJJ}1&Z2r6aJxdVtaQAnGmUK$FK7-1RtD91#i$VOA-Kp@jJiEt9BNV=iDp0Ixz zcC8bMxp}N4GFVNQI7P0Rhvw}Ma1FkInH2i-VFf)$!xO&AcFYgGgl2T2q4=}qvNRZ5 z+hKqCB`m!^!94tw@QgEjy<fsV`T)j}mrxG15MF<r!Pig>UejKOy!Qn<9zP%j{1rTd zop25jX&CJ0jJe?s*m_$bZ|@-VL)9_Npllx`4Aqv|pza|5pGcIW`&+mNh*%D(VHv!D zE~ora935~c4Rlx|7z#tHGcZ?y3@=BRA2giDok&AA7=TwHg;ze)KuXn%7%{)anK8n6 z4gd7$m1VYMhuaa54q;X{0gr5w6WlY+y;f-CXZU)4#w6Lu#(BWh@e2kAe<d6CQz(B! zG4iB#|27PdJj2}RpYexh?fCA+OZ@oqS5Elh#dG{XZa;fDg0CL@fG>Xj4!^v7f`={7 zIO9Rf6Fh$Q0;Bz-n3*9vJ(;R0rpa?>u4%uHDTxG+dz2WLX(3bVI7Oq9Avp9F^J+QV zL>$~EBZJeRhwIWZTLM!f7%|vjcFe)!oaF?E)d{!V1CN8^#=Jv^Ij0u0E*+=LJG5{+ zEpWMPO=EXh;B+>&*JHDrF)=U!yF!6Qw+n%}IZhdzw!qZ+Gi;p?p>8Jyuk8(k{A~*q ztryE|areJ~r~OCBUbkTEbr&a$yz0PcOD87YcER}aHC(^`6}r|RA@6w3DUP<U;2nMp z!)PZILw!&T4}6@Ve#i;*w&Af$Nc(#5_LqlHbi9FMv>%4S0q6z>Im0^Khq;MP7)E-a z8W@F!tV0xa|2V_>GFx;*58xU85#A|USIiCMh!lTxc;fT~r+b??*{WP+yNY*5`&XZO zddmEu!ECYNZCeM-RvQY1A~rTQQ7V=1MjDp9VaP`{kd3IXpcqvnp_@QrauQ*ZVAy)W zlF58!nJwAzJ_OAQm{fWgl}1j`%gJjg8r$m$N#DTL{w;c2zrsMrcNpm8j(&o&^#_c$ zeTU)xN0=byPZ=M6QoDZ}1_}1w|AsGLbmFg1ALEOs54qvV19JNSzkA+~KmPDFzIgl% zzI^mOr~LWBSNP(=muM&E-#a*jzTx4f=ouVD?@(j=b@UAO<HtAS=pP=%`1lB?NG1p1 z>wgOaDemf-$r{S(38*eDvt@p$2O~-q2DBCoY0R9^PuBNAwHe(CJ*H$*DCHwi$j2a; zjdO}xHpJXYWm8QfA(B5Moor~Yhd%G^vp1L?>V-}zfsV)<m$VN`VeS4Inn#~w>hVvQ zdj9B)XAdv#-vV3L6S!Kwz~qx((D(RPPUwB~9DS4rk6`@u31;7X1zpcG$Oqav#nJga zyyGvRmkdKXIRWX!WD}6@^*0(HyI^K~96cRf&=47Mknh>7kin#obB0AJgI_%ghh~bb zHyY>^ItHy=3mxURWww}Pt?){o!KYPW*5bX2)r(v*-*6&ieH?ETuopIA+dD&$)Y$P! zahO}gyTiRJ&p##NKriN`P?8#*NWkHAArgtgYIEF;G{n7L*vS`X9~--ZV|)zm!5*Z^ z&M_<RLPkAYBdHv?vdotJ$P1+8uQ5N~g}JeAPVkI$Hnd%bPty<6%P;Wi)iZRoJx3>p zw730=kvC5<^Zd`4?)(}vz28DgnCbbpcK=qi{SmH?f57vO5xnjl!Q0+pPI%KZ#N7VW zA;puH4s;I<k|I1x7~_m?A`0I>{{{oS1L%6&htAgKXn)hw(DptC$q~l4Pe;)9vIo5_ zU7RxA)rr}+&!K9625H9=!efS+_D8k*x53@^6!PIW=<aMmN9!w2XnXq#?d0}NM+=6A z$x$<rP6;`po@{%-DOz$gI^FRAqpyD<(A$qQ^6F=DwA{q=$#eRiv_RfZ#AJYoNiVl_ zwvWTn@dMPqev9!J&oJKd@{Ct6F7C7C)k|o}_v&f;2GaKbuf6krYAf6J{9ib~%&U6! zW@_fX_vYT$(Wae<#^fZjkc2=vqKE<@B#|>1<7}J_#@TJVgR|Z4{=WU)AWXWqX?x4< zTf?gQ)CmZkr5)D#?tS)=I^ef@l;E?v#nz3W$%YvbZ|1c5ND2RYjm=7#&=+IDtwFz4 z51&qtF70m7?Qm`jZF<;S#C7Zz47zMc*-c0)W<J=7v93<^+AN}<+9W*|t0Z&$A@Ki) zU9FfE1J6l^2_yc<CPoD}&R)Qq-}$X3{-naQlM(zhZpKez8yL6X)%6+t`uhEEPtf?Q zmVJpNE-YTevuDqcN)O`Xsnh7`>;E*O;lb5KTp3H^@>u^iW*rW!MC`cT+liIIkRU8s zP6yWaKZZNq7OeCduqZ}8i~S)boa+nV!Y<x#b}EGdeT@jDF8I6_1Qg=+SYa|dk!r6I zB0+;*?<zg+y7m2!A<>zOla{~1;_$)ZiNLA|iwK_FHro@>n*HzuyWtPT;ScsGB#x@q z3iw@Ncx~PA+INS?8s2v9?-33}QQH=S%M?P;8dS=N+k;bvD$F<wG3KepsHbifckTQA zk6^(ef^?t>y<*fGv(+gfYOP(}cQqi=twl1X#bB^ejP4qgG97Qkkf&AzYEF^wj)b*F zY{~Y&Z6s|wij>uW@$Mdsh6gYlR7=3sgA<~B=ED{Yc-=_49Fl(LDw*RCWzuWLDKTP? z>2+|n+7w}xbnC5{H#Z|%c?2!F1<>S{D8*Ks2Y+2D8VV~>w7(pM`zv<GzVcPJp5H*h zz6z8cD2B6Hk63F9qIH$Z8Sl_wP*;U;O$CCLRjX7~iY-|?{y>Iw6&TZegLK_7BwO`J zY7LvXB1ZpjUT&T#*lH976b}5khSzr&*LidA65c(1h~IvFKU2Z=3-O*lw6+-#i}&Kn z)ob#r6<nS!`Be*_L^RyGbOj4TBUl*R&RNlU&YA0RH4((+p@DT4)2U6zAIkMu7)!kl zoJb8|UZ^D%%=M=+pW1$YD^pYBh#Pc>izGi1GGJ68BOx)WGkY;$Y=U2mz`EA(in7e? ze+)h@crttebEgAMdOsR<YG}~;<hEKDK}A~-TB8qogI^F(N`6%-S{vQyY>&WXjKb9M z`}~pDnoVJpHpj*P#Ng;u%Y@5=<GNxT50zjvDn=Snw`3?Hwq))2LzxXY<s+TF5kF#) zpc10pe6N7q566OtkM$uno5X-(l4Ei7$NlIF`_Q{fdV*c(4Q)Tam2|{_lt}8mt`Op$ z0m0w~wjTd+C>qX$+i@o9!jKp&^@{+kilkX2^^gZ=?KKG1*27e$Q$(BG25a?LFlped zEJkg932F<<P+O>u&XOX88%xntQic2j70Ame|6Ou7u=Q^nIh<30Z}(NBxzPceu?uFM z0~VcK8GOG!iwRRM9a5bJ#C1jqf1E5XSv~$B^hQiOn=z~Z8lwg=fDnVYQD<NiH%^|z z+gJa1rs0p+yLaz!VR2EeY2fcq+`W4b*RJ2d-0>6gmc=I#4GX6)BN!b;AUd*zQOr6# znA4VG#uvb}7`;ucG2!c8-~SlKO%7c4b)w%LMV~vSgdS&P+qs>X7#f7X-GpS95kmok za?DmQhE1&q*%}d8!*6X^-~T8qHAuD{fV<O-<{gs%htkkS;oyfx7e={AxY`aMwBlIX z7EnrFWjR{wIuH`VDq{ET4qMmub2|{Vcfn;1p-hN|4t*Rp@i|m7Yj<GYREXn!LLByW zW2h%C8SIIz?|%$)BB@VzhtL=6M>L*RLL{DA-%lj_(LXkVgb>Y%sR^ZoCMVFBN})I2 zkDl1>i1X*7cR0V5bW+5NXf0Cy022No!LU-|zBn!=>^L2;BHdv`pWd*FZjH?G2QnA* zV9`~B-sUE_HSJ0XG_+$_UkzVvDeSdXaMjl+#othl>CQ5Q+v=dNu7|clvnyJ6IJcG7 z%6c5kYe7kC6nbkfEao8WmVh#JrYL+Ob{r3yG1X<mfWs+Ci+(&n){Z}fL8lW7K?6=$ zzQv@q9+QC{Oa}YbnF=MaeD0Fcrzd{TK8a{JeQ6odu@i`nZfD*XlInOmp1^GXz&bO1 zsZGZpL?SH;-G^&_69#%mknS5pI`Q$OcKKezJz+RI#i$}=6_}OdaQQIf?7$em!ZdD? zjKoaq`ya!2j|mY&IlAqg=(L5<ZtGTp!5S9Zu;{!I)LIj;c)MW}BFyH9Dy6DLgD!nD z2K`+a3Ws*ZU?{ll+)j*k2QU%!qrSZdR%<`p4t01P37l~|aW-%a3qx*9r(>8(#wC;e zF-)%Qe+&!#A)Jc&F`Vc}sz0rSWFob`KQ@xU@K8Sn2a?JOj}2pZv>&7Cehep*yJDzc z_>=to*N+b;MVZ%PvM+?G)D&h0W|T4#i{om_jg-Cxma<xOl&oT0+kPxuusGIQ3`<D` z>hkK9P@AU_TMa6YXkacq22W)Pyp478=e<=D5dC$|T!vm_J<QE*=xo&OijD^HTC&66 zHd1s<hl18POtvV(Ru`f+r!s6pI5;hSO!%#s4_Ywew@IdbR!QdgLzorJ2ih@X&cUQj zgGqn<{}0h{{mfa+_4i{gv6<t23CuWz@-~$qX4Ip{Y((!GJtpI(;}3!F5%199MwbpT zp9hgHHFSI3+s^Go%%Fj#!-cd%j07yr%CUBsF=6gN+SZ7atwEBs3fA^NhB2=WZbK8g z+gj09YeRF5T@BTCw1};~$&2#3ZnW0fpsQ9-ZAl|snikmWt+3WRc7>(hzU|yj*c<HV zZtWDGApm1r9GyCKnDsHFv`+N3RU_mM!sqUR&$UXIGrqq6F$7&vMC=hEQhMQV_9?+` z7dTe;J)R!8+`VwR)YIYZhs)aokJxuRcZbW-v+dkYc-(ymxJ_^iaqD#sin1C~O5Ewi zX;(YKMaR&&FCQKI3zT9zT#WRwL$KxKqVbDDL6HLZK82|NYd*Yr-@%r91RZ%L=qxN( z&QM)GeC4HR&8tCk?(S$hQoZfmPLv(ips2<RZ-*7bmR6-C%zC)Z9`w5GnDm)NVz;d# z1jD4CtQ~(K<6aZSOouQbMMHE~qJd98@Ru*M!6y+7C(|Q{x3nYPY}ibb9^r-t4A*=E zQ{EAn3ksCuICd15EsZdhmZ9xv3A74mEfQO@!{0U%C>5VcD@Om-M=?}WsGL|$CC2OO zFkE>ALzRbCsaV^7BolQ7=&q{9V0ke-M=IewT%`oZq1A17Nj*%3TG$U(!1-O7QXJ$! z33`hv5YDYYK={8qcK6zlcwIEN9KDq-h}RfIJ+v#MN4#IUv<W@MMF`a!1tx?voojS# z*#B_C^&N<bZJ@3LzFJin*S7(U5q|OB*}y00(r{l~hu2a}sID2lYMt2cM6k}Jl%%E} zBbpj`4j+O0@Bwt?9a2iLC=XMj--ZjmgK7Uhbnf4;4D0?JB#wNA_OHG}(>Dj8{Vqo- znuA9XFFyo#ehwPG-iNv`zfs2*-=ONh@=%%IgkXb+#jW2W-J*^$?S8m*jqqqI(4#$! zp0?aov}@asMZ9OS`Tro*@)i1ZB}nKi*6BA?;_~b;-n`uW^uwxX_$4cR645Z7nt;2> z40oex8<Cbq^wjM~f2R>ilSw)KW)rS=JCU%M5$QCe`x7#5H6Yb##dKvUCTlA(S*Mnf zx*CksG>G>u7nCVwwxJrsS|jEf%g|e|L#*DQjs^qzT8-#xvBBHW4xh%L6mOj<cWpb) zw6|fwXvFc34yBwhcHoS7-_sTg7HoE$P{>KaLPsaYI@>U5x8u0Oj`=l?+imOnABPQR ztX9m4Yt7gkN|?6VF}=1w=d@!^TxZsv1;qPKI305TxNUdL*=+K&Zau#fQ=QdFXax0o zB%3;wGA{a5sJt9zF+j2x9ftE*u2MW@g_zQpAy{@8x`X@CdLTy`#v_N3I`S<v|L-6Q zzB+=E?+z&?Ux;y6c^+DG4<Pr;931-FK4l#I>ptZC>tQq=JSK8Y9tJe~k=E=}#zezD z1e(iWtN0F{;=c&~T`_+_xa?mLEBjxFmVSXq=~wGS%f80qPz<l1Ki!&k<*!X=gHIwF zP9(<>ZZsp*VA{rLTMdTYEjZ~liJ)i@9XEr1!~Tae?J{Ap+k*vPCysY@VD1w#?(<@% zsTr}O8uV7Rp;wGxRMJzeN3^N~i5e~XYFgD&sYSfWf*C_K{OvAy+Pl<XSl#+Nd~jQ$ zXg364)O88G3h6>mR~#cI5vYw;jCI&|iEW2-+Zh*3iPxU6dqkV<QqF|MfruxJL5B<D z4ksqWRw+&wPP#o9wYo51cB;YTklP`v6Ej_IOnW^T6W1AWsAX34sexb+Lw0d&wu?I6 z4YoD5p5M&4O}xLq9r4C;1j@vyv`nXz;l>)6bMs;R?l3He^92P;u^uVJK;1F;3Ptk$ z`Uo1o%vFZ|+dT9h&VfNB;iA7CM*dfMN+~{E3G1<ZRDYL?Z~rqNU;TUD@ABmuThDLh z%fIB|KmNxN)Qis%I#!C_vT`KK%aoC>5Cf~)3iyr{!IhUUC{&7Dyl1fB0D6lGkT_a_ z-lLW4^c7d((!dDbJb$q@?PkrCi3Tc=zi0D_6hfLNgz6f%F{{nTRLlokWu08Wv(B;d z&Bq@|M@1c6^=+_JiVj;*sRYvw=XRn&B=pEPd8qlK63vGi&~jKk4M+4a<Tnc%mC}&Y zfX;myOw<>mtJMfst62%omep;S)`Eb+3un6rUcDV%dYdwQ1{<R8D4d#F#A+MSSJ$#j zns+$&_Zez!!%&MJL(OVQ=^Eud(!Mq=Qf+PGSf>`f4%6*MB!vIbrdB0H*0#M`EvBvQ znCvuSL}$Qon_el?o$VO3*)XWpV^CagS2XT$ZU;tM^*9xDVN6?zL|HZZO6rv|DH6Kw zi*M2Pbs=m;8aPUtl#{5dL8z<>9R*F$=eH`uQdke~!Cd(M`Yp^tIJAGcUn%DOxrpW; zgfk}x&42w4_5Ycpj{nGk=HK?g@z)&q_Z7mQQ-a`k3K8#@&d*2Ua3Lc5OAy(omTo~` z&QYWf79(}29Q}tX*GUSobtN&3H_tZTp7<DJF28&49?qUSkLx#<@cj96`RmmWA3ns- zPoJt04G$lFkKvI~ESx-rUw04>W3dQabro>cmT%*9M=m^8AGGBf7%OX)(_GyEYfYmN z4Rs>f*L+f%N(^xPR}plDZRj|rM`sq$ms(&gH^NqKP>R0T2zx;%#>#SGK2U}BoEr5U zXoM+W3uk_lpivoaLAa$0eytTLgA0Q~nCuSs4(I+p!yO(BwX0{?=0MyY#gwfRbB=a_ zQ8}j_otU+W<Zd%4WyWQ~f;WI!XE&zpVdcy?1l~BN#CFOWRL-eT0>>RLOtd>OZty7O zxV8?q{rT`$8diV7&>U3Ggo%sh%;+<R5i|EFBW3a;QQe5{Bee(|tb$)8FO>uY>7yb! z78JvCpd9v`N@X~5s?dF~0Ku;gA)2Q_QpC4Zxn3DPB|3B$h-6++hQUG&1_~OLk}A|; zsKkJwaxKzjVh~ZLe~<VqSB7Ts=J`MVawn0P@$2i?c=!CpW?sC+Z@>Jq>80=9y~F6( z1Y$kC@*5z}oIQ_Am#?79FZ##nGipRbxH~FEL#Nq_o!)CwTs$g9I1w>w+DKH4q-NSn z(bD8Y%<4u$bk=^wM4Vkn3r3w@jC^7SYIF#Sq!?@RA}*4cYGN&3gqyu$l;;%Kl@e-n z!d~UXNJTN+xs9+NZdQ)%NGt4ldU!+`dBkW&CH5n22)4%1XG)4-s+OcVg#p_ThMj3q z&ne}k9sMHsJ1}kO#JJUhF~v+;O_&jaWXft3`)Z*OnYFkv<Vqmz89*uvq<uq3c?ZRQ zT98so+L6Y9J&Ck6rJN~U3%V<s5N>lJ+TlagsGg+Jg24_Gx{ZGL+9S$H7#*0hSTQ1m zNuSP-c&k@ALv1FEYTHF@^B~k5R?0wwh&f*uh=GvEJxy*zo7E9+a-m%ahj3Lr67@zz z>g-DC)_4V72-dpbuXe(>2^Y?cp2Vvcn}2j_tG!%W#shx{x7*rrTe|^W2Y#HG#=8wO z6>o)z;5R_>cPE132nGg+gm_4xzNtm-D-jJ<H5$Bn^-3y{i<d5~->Tr2t2j^3%wl15 zavLW{$1&a8jd*YzGlTPj6Io$)XaV!XCqFr7EMAP7dojR7;8M?k2yzLxP=k&LdL7+r zaYhg}M=|fR;AGH)lc6r9oCtJb-q(fWK0i)~t!f5!MkMTg815OvXkuI`<NcGEPA_04 zd0a53p4cGHMC_Q339%dN#z-upl(BdS6LB#rjR%x774za`M2ysXV;D-MF_a$63d8IB z14<d_6+$pEit*&6a!!Oin2z^jK<xL6Hk3%HXCOI-;p7PV#D1)AS{Z}MQOu8xVR~Q~ z$!-yI{88m3L@Y>kdC)8NqrrqSVuFOt4AW6DKrJYRzM>k2${KZ4)u6Go5~lJZn5!#b zsIFIvMbn6orWWDSWAGOi!M6c_(NUaECh_vw_Jxoe?difPT+b`UxqtcJIQKu_bN*ld zFYb#0?k_+5u>M+#28so{(~aIlKc+?g=7d0*_^h2~n;OwjSzU+MuiwBQ2+7~!&WHy2 z*vGqfxO3+Y9(?}<4<2pj_TBrqz48e6Zf)lF@`G*XJ{NA^+?{U^E?vW|E4Oj;@@*_% zzNMU-SH=5YxrytSZ>)3W;#FL`e0|fo58;Mj`RWSpU%QGYH!k3ZB{e*{c2>OKc|5s( z0gtbV<Ez=>;gxe(xpW<OZru{?@Aj@(xpi~fxt+Ls_YUsgeTcia1S{%!xU__aH?Iia zujAGRZr;4Q>G*SEqi+vZF5}M4%UHR6L$LIxv$P`o`vbn*zO#ZmcUS9KCHI#_**y3j zcc1)>+mC<7tu=0b|07l&{j`4kG4OpizW2wH@$HWA`JRa3H<nkhC}RA%#fw>B5$7+Q z$GMA(f=j>4*^3u(M*I)ATmQD1i#U7s3}&Y$FflS9NMUj`t)AfoCP$K(92<~~i|dT@ z{Mzw{GB-Jhi_<Aw9gXACKol1@aA_ch`<Kt-?dyL$)A07uWBfEdi^qLQJleqH{xn`( zzJg!hym|jxAsV`)afF0$SXx?^zi`>*3&`1r4wF@lXyBVCg-Yj-S^W6p`}bO%Y4}Y# z@yk03h~IY2^H;C`@c(=&UVO&By%fH^mA-wR{;}-d{_?r{_U5zojdl9+_2=u~s&DVM zo&Ox5Z*M+R--z_@?dR#==j<E(diCb*F5BH|Oyp}n6)d~gYyM@Ecj9yXqX!ZGsJ#<| zeXe1+efy3)zJB9|<n`;<xFiIl647w~!2`K)f(s#4InTuypR4Wp_Tlz9`^V0rruAmW zICH0{dnq?rcAu$ppR;eY)q4Fr{rjAK%iR9{T>bl;efvG*mpEl&^~QZ}m52r!3+JD; zfc9+9_H57gY|r-W_f45-_@hRFaqIS-_3y}DT3W{U-#^+UJoqRlf0Fa|ojYr<zb%iy zf3*2^*)U%F<%ZuVzjyCG-oAaieoXasUHJ0&$rI@dpJU}Stv~$mW7d7+bF+^hKapie z`{}2j@$=75H(mF^gNL|zb4B_|zn7MmW!XJ@_AKlE{qVyNc>47Hb-w@pd$~|HbIk4i z`wwK9QMuk+S;2#c4?oE)1j~#wW4tzvzHF^qPQ%e>mffSrk8xvZNw!tqX3M7WdaSpM zFFbxI{MhIl>zVcX{P_#%JKNv#&6~1(vu$j1Y<rn@$2xoU>itjh(qEPn`!V;~4%qH) z-MYPA-q|vi%TGW5yxxYMJlX6k%ZmPT%Jr3~Us><>@5|@Wv+WzN_2|)Kym+x%XEe62 zmoMMf+1t0j;Le@9(nr=e`}DG?XV&%0EY~f|j%3P?^~N}nDZ7k47JcFQ+Gk*Vye<6A zcDrL)u#IQRifiW>W7$VH+Rs0)#z1}s`o`GIzVznJ`@W;vGW{<TQ%R=XW$fwGr&w9J zwbo~!%bddgz}IapUzN6+SGRu!v<m(-(QxY28R*+PWY9Tx{sKnF#^s7l7BU+OD}ZYy zCa0$5mrSr@d%FDcm_a<7iH{ZRaCva`>NQ#T?8vQc2GQ|XgCWn&%pS+k@F>EOm~1rs z?QtGQV?DBCWy=_N(gTBt_w>n*!?N-R!<d<wllu%l4DNismXFC?SiC3$B_E3&pO}=v zE1Qj#9kn|e!~BVpvd&naT<db}`gQ3aJN4qlORzayQaG^g`4~?)5|s%uo4(P`Us#lN zOX0vaw6wG=R|d0gSIh6Mtjl1iTLz;3WEvx*<2ZBn+_pfa()hVgo;odkV{oRi-LuWJ zKH2W*8y{^R92&;d^o$fc1B1ip=}pN0-I#<_8xur8w6pV~zF4pHk@dQ~d~>}|urArg z*!I{ChDXL^pWw6IY`596CnCuuQ)w9+C{}qbuP?A#CL*b^&9SXbPR+=&;{95}dSzQ> zJmF`^mNA}iAsL@6<8|)dy$7e$gBxOyLEl(*^k-7EySVU=!i%3}aCihK7EZ}_m@Q-S zN~H%ydzzI=m$8d&fBwXROmOrsBe;D2AQl%dVMQdpvGGYnqH($KJzJuS*JghWhP!1y zrLiq>K^|io7umDyc>P2&C4Ktw#~)>zOZ2C3Ma1uo13%Tqa_|R3GB(iHdqRK)LSbAG zKJ%Ej#n=WJ-&mfEF((#IikOm;{g{F^Tb2<G9O$U#PZbTh`9)HFEsDg(4qaSQE=MU` z8^HiwR9q^AL$^rKbF$;)795q8bot7atcnH(!lsrs9G^cS{h$yya41*KbSy0`%Y@US z)yqW4gmd}IRVga2Ub`-Z(CQ3IcKl;`4GxV+|G0lrbfSi)Rw=O9Vc5yIzNB9SQ8rXh zmk-lIP}Daz%conY!n2u}8JwBu`b4K?@MhvDI9h@Q(RuEOV9By&GOVd<kY!2naQe(y zDSQ|pvlW0cc42W*UYnzHj@Bvq*sku}y(^O*%aFmlu~{qIVtr!^oNh0sW@fV*fLJFC zK4lfv(ie`V7`%^_Rm$VDXU~h#=TDeBenJXXx7UYZk@R{+f?haz8ZJ+l^fz0^AkDU( zNz4O6xEws3CzA+AP7HvoM+WM4qe%*GgRxWAFM~P%CtJqRUM$`#lNd$u-0^w&MAwiA z%pASn6m`w|W`Cwn6O&Ui5wT90Y(~c?<j6i-#%pVJ2I&_E7^(Ci@(YV)U!|a6pQLEu zqiro(ooI*W<>PJ^n?oi*ilS^81!*+iBL^WA?Z-vTsH)Y7`W=$-j&;j+M}bvZUWMbL z{n3AlL;A*8#F^i07?U0s-nF*rW!s_<rLWDcZL;h*b4u~b{GrviBP0e!oY}Hi?XsTv zKQCTnug_96Fy6B)D7HpM$50~rEBhzQjx$-I?g)&X7AYJ!`$5sl*HeI>7P0%E(f%~i zK&z{7l7X8esEZdb%kO(&aAIXMSu+rfi4M&|r)c14IV}pG1)lAxY+n0>7%fvNQV>*B z*W$croJ@LD-I|7GIW^apNMe3|K_+n?)3?kFMYfEcxJ_@A$)rVeC?*gFX9fT&e+meW zYWO;i2HC;*bPIiB;MzFdoef)6*Td2mt*#vn&8>1KiFLbj>y``#42ml&D`*pcXU9J- z%8+%P&CznkSXK;%W1{oY4~kXR+4=K}GN?1qm@IY~3^}XA;7H-X<dkh|6Ty!oQHp_p z5Dg_FP;=$-*>e}92;<BRzjTE&4H;ob@o2W%WRS|HF+uQptY5RmE(Hw*KBv(s7TNyK zoINX(_k*>?(JVWbF=s`xX-vB5fg#b>FNm_R%fv!Kz@*H&ekRK3p77D_?UFt*Ayed2 z_@_i2XH#&|IPhRxWB+4YVsc^GEiJ9ymgS5q1y?57aS%m6IrGHVWYaEOxVS!>#=7E{ z_^@qp28N=Gqhq%arByX`au&`j1_g{Mj6dujnN{}LFb)oVVz9$DC;NnmVT{N0iAk8Z zggFDkenXMX0UU)bXIUAW>3=qjvuYGltV_;rQLK(nOs(b#A##{Q7z_CwMikF%qilEd zkDuwE(LM{&zyjy9pd1ae(is5Wh{VQ3!yr&9MqeDM3~{<n1T7|-Y)&WA_!XLLm`r35 z(cue=OJzsq`idt{evqA;eslD7{rZwroP51OjItQ?vpExz5e=N)bh<rKG&HyBWRknF zv@De_1r|q8p>R}oe$Li#Dwd;!Y)-S&GNOSAhDw_w`$#M<gA5ZJ1)tsNk@xyIE5Sg- z?U^&@vMy+78PUL`!f9>JG_Wq&_9z%ATr#48?UF&9fhu!LBb&zH$3Vx#%K(#KR4hlH zmqnjoA7Jv)iMr<~g$bIsbvSZiV5Mlxre*pcgCa*v6kD7^XJY2s56&Rbe@-9M7tS29 z>?j<$ZY7(R5e=M8px|eJ@cR6+KTyyx!7yPnu=5LRIP1pSJZzT}BFXSO^|NUi(J(SH zCa2CRxVXP0k_*=maF&aGRA=arxAqo9*>NgfFX9ZvcQ%c;Few@sH+n@f;{d43ACSq7 z10*KlK2c`XwHoyGCuR9jXmQKYb+*Mc>x%7;w+dOeoc&`gX1im*pg1sgnxzorXJNUq zjq+9jKSMT+LWZJ&{;-T{HI33AiU#%p-il>xq4=X{U|!+pV1H$ue~!i&$5>4fc}2v; zt$pK96%9;aoQ`IpjtGIkK*WM(vf*MS-uGc+=IjFRO|rAez$KD%Hb?R_78nB^1p<Wv z8wWc$M}16u^qEt%>|~s_rq7&$VuQ%`!X;jhot5_>S-CP8iE`qUHUl^BQ_^2HVou@l zwg5XhJ3OaKvpq{gF~H8kX;u11UpX@2eetEGWhwZkMSx`4GEonRV9)7M9%nl#(>M*x zK+UOa)-6>(6EzbtNAhfwavJ{GGfYp<O8*IGA+l}5;yV1U6OKMPf@7axd!rCzc`|@9 zC^0ZHShGK{u35g>7K2P)oa*Py5QQ;kJQ$RhmX_r)%ayZS?6<ss%LLAH;|v9F`()E7 z4A|y4#m@5LExA*tPs=jolr?9Pm`La|Z&9&du+K0Fahf^XeS&?IQ`Rgy_6hoNO_ZHX z%tDOv?-YNGaqK^=Um5#EzvXAjrt#Ji`ymAa%aQA&c>SAPz$f52)-UUlZGg95IZMZ~ z%dAz(rZKj%OjvgGhchq~ffNkvxBMX``pa8)^poO{g8}9V4zRLmjJK=8??-X%yYiM8 zZ<lZegtKexm#jzj0S;7H&#e3L$tihT>QmWozy5~T&tBohlV^CoffqkK$1krp*B|== z<4|`bCI>Ifg`CY~tYlmHQ$<5&B`%$$SjYtBXV0F?AHL$sR|fV>qvR-%2{GFpoW=mW zaW;daPkxC;X7M3K%f=Pa^ohrOUA7Y+-=AfXxelK`p<nbNQ>H9SCPw<nW3DVG*%n(g zF2-cy&G`3P>}MA8WuBtQEKs9Anbq3aF1L))hmGaHI^!|RF0=3}bB;b!%&>evg?}%E zAl%Wn=PzU#GWjxivd_|2wm<rp?U^?k>y!(jGUdQ>qz^n#ScmMN^pADUvSZzDd~zmR z#y&tFGG&)p6wGqT^bHC^mLJd2pVg-Z-am1XP0PfSt!0;qC#!Y)lblWBc`owfG3z(m zeU{hWsqDDr(+ZjM+?SsLjM2|zLjDx(j%Bs+I+?QEcs>(f=qt~&?6NJQX_@-lc)Z%? z*e6!&m2o%oBoF%!+h8V6e=7Uw?vGgP7sARpi8;r6=G{}cb!r)JUu}LeFf*&eTR&X; z!x<dTlu#7?siI-e_H57gY|r*=&$4S24VQ-&aXLJ|i8I|NaOcbl-oAXl0G;B7GhlpJ zfis+(spf4y&dyOh?1_dw+p|5}vpw6hJ=>M-iH1Gfvpw6hJ=?QA+m%r>{5Nd<nZ$2O R{%-&P002ovPDHLkV1gPg$4USI literal 0 HcmV?d00001 diff --git a/website/www/site/static/images/blog/apache-beam-flink-and-kubernetes/fko-library.png b/website/www/site/static/images/blog/apache-beam-flink-and-kubernetes/fko-library.png new file mode 100644 index 0000000000000000000000000000000000000000..2dac61702c6aed27f3a769861e8874cf0a71a3d4 GIT binary patch literal 176729 zcmX_obzD^K^R@!gNW;=4lF~?nba!`&Al)o2uyjkeG}7H2BHi6x(jX1*S)cFk^|OD- za`x<;x#ym_=DOx4R7pV+4Ve)6)vH%%(o$k7uU^6CzIp{Cga{A3<6nm!^Xk>NSJGm_ zYG8vuZ7%Wpb18)azP{{hTs6&}8g7b{E7Apg)53+={WuNS{lr7$apT_^D;c#vv$Ds1 zD8OO|{w2sLeV3vC^5xe1W?|An(>5W`U-wba$|j-sqzU0~_DzG~{L66%L69-%zn6qO z#n<?XH`+B{Mp8MgW{d01%*_$>+dMce7Hhu>3wZuTO%?R^P?C}gd0=E^Wu>L0q@-nM zuj##^U%I%s@Ha6v4Per1#BhH2Yje}!d2{lnOslH<c&ORs$lpI8pycyXpRTd7@pxAN zLO`?YG2wKF@5_%s6#VAnYG1G24li;Y5fPE`)i%$70=X0{4)B@LP2AdD%S*MNWm8kr zN#lFPOycQQ_sbuo0v>Q>`Yj)6+1RRktNpAA2nd8}6f;p%xgBzjwq{B+5%Tl%y&rhI z|8TyXYSdOSMctgNv<NTN+o0$)Ihu_(IqfZ*U!7N(_UmnVFwoK>?$>J@T3T8vIXl-r z44nQMTBtGCUt`v9=Gfot3qN6hIfbvT8W~DrZm}w{S*jQIzQ1x<DjR$VHAfnI8x?Dg zwCB0n>V9&!*<npixEE;y{4gcJB@qGIsOK^6hPzqLL4WMPF8X!pcVylZgeFWS8Rh&m z%=1@pYhj@M&bOjE(sQrV4_26n%hq}B<I-nOVg~QAHE^azUzO(#&J9P$)XTCW%je%1 zH-EP8k++|=*R)<nGx%1To;Smpm)X<WjXEBaK1dSnRET5dQm~=+u%PD342+Fw%E?7| z!6!7kotBRYz38bHD$LYb%`NW<nrejOeV7E+*!lL%_zn^t78dq6tK#Yv)Z``6s9T|k z_;oa$=VF1i+4<o6!?L+j<C-_jNFCsnqpO1{83l#Ex~g}^ErP;1BWYYmx93|jGBS?+ zW9PTMMZ7MD+flyz<JlJko`VAeKc}laZc(|##l;Wqua7Q5?QXcTUKe^l{2icC*v3Y( zO}j!E64ua2<oois%X}W!P(Q_ZO&h&`g!lF(6q{zc!cfTjtPA1(Vu*Q%eSL?0apgjg zYAyr+KJ5vyx2h(pvXTXeR9O;U=NLttCYx$SFNrn1CdWQ~Q$E)}_TJm<&QF@EO#3x_ zZYLH;vt<{dSZHK|%hxrEUd?0PloR@=Qw8!)HY?387opB>j|pAo?^+(dJp4UQ74mKO zf|)1bdl1b%ML-3SXGu9d^xakbagisxhlHH0b)44AZjr*&4|sEaoqXqge^6Nd!LVIV zF!pMW;V1br-?FIS#@78(%dh#}<tC?<>zOK3N$<-Mu4elU|A#2W^roW^N)NAyI}{_D z1&?RT(!5~Av1t_4^z_nO+UJ?e_nsbZS}xPSf8Xo~bL^?QWZX`@4rOUraoGCR<cZK2 zbSE@NaKrghG2DilZgn4dT_wLboWkz1KThG6ekCw>J6~nW>3#3)reC9<d79v7x^r)` z(u%M<l6G@^x9dJ?E=X~bnUU_4xWShY=;HL&Zn<%zzMmboam>p^4G2)DIn8KNfn}Fd z0lo$16SMm~V#uwPGIlOBOJEwYd|`pd6bqq1g*TW7VI#tc4gQX$s`I$EG#7(~=g6tl ztS~s)iMHsGDx$SM1$MJey>$xvx<!bg7>3ch&2r-e!T29^c>E4e|Iky@h<G<Lmd`8& zzaj{7{~j;##S)Q{p3F*i=(U0;2r8}Zri+vnXpQ(MB|#spv%GU(1}&AUiG!H?>wTU) zXc~bi-HI^(yZ3l^SzduD28s@$8AV&meQ`ToJ%)dUVUbCpfwxcemq6Kn{|g_exE9y} zJh=p~<;ZH#6v<78%XI1ny+^Mxu&^%5jF=g&XkfyRozky9JP7$bf?W<K`}9RU|Ne3K z)l6e>`&2U~=p;c+IZA~1leRR?=Sy0dZiDOGL<9l*@+UZ>bvANVk1GqVS6y`@ry<jW zoi_&!ey|9<!VqqecMcI(D3p;_5nGRkB^9;K2a|Sa7Wbh(Op1n=BvA>^I+aF#*syvX zndByE+1c4A>6E`0-UkSspP!eXPEAd%2y_bC>n9;Zg&x7O)`QPYXx=uSG@4_eBT__V zJ)kL%R(IAnSTE!NnW<@~v3$sbxn_)UjMYK$e0dYJj}R5$8RD6AgfA?%R%v_h=lSij z`v=?@%c&p(Lus&Q%EN0dn!QBRt$RXN6UY(5>Eh1)-s+|odGbCAoVxqNLJQNav?+;o zv<t90a{~PenfwPQoQe^S(GKNJ{qCcy%`QW}bvyn(+gjWbvqP*vwV1iiqEW+V(kps? zdT4n~BY2Zp@<&rVD&_PHHP4se_a;ea6>QcyvFZ5t2PBH&wvoCGv~oxNN&HpMT-0O` zCy>1&iXr&?zmS87bN2Qpaz|sfb|S0%?7OsnC3jbfp1ls@V|IzKhR_GS3nDn$GS^1s zO*~KRkuuf|VMpDEMR?0CI&+C(Y(aj{qJ|j)ngz47z$}pddXve48g&ujRfC4AEG7a3 zW=562!@)8BOMheUZxhebHEh{O5<9pc%pel)o^v52RHq1|2x?T(Ud;X@0(oL~?-3s3 zw#@k*?y*dPhq8%*8*N2x2=c?Q$DQCR9jbjVqOWNis>nG06-9rb@~96?)`l^N!y<;A zp5Wa&ar}&G=*GY&*f|uFu1g<>)+;)AsyX~6Mjy0ul6ySUBtP`OT|MmGWP#>q)zPch z!{_9cAgP5qtE3~o`#EeZuThs?L6gr@LVwsrkhCPV;Ys*Smv!If5)1qP_yyk8x#&&g zV(D^`7a~vOIbH2U-VD4v)rXDd`-0xyf3YknpZ3wkgbY){7iKBXu<_Tv+bH!@=lPVm zb=Fa8klR$t+d+St=d%k+5ik?jdjoCNz7$n#oO#s<*>}%_=|uW7S(bCF*XUjT8iwmK z5u$<J_;8CNk3#*BYpYen^R~y{z2#IvB=+ud@RhK(k$@gih+t;gqrz4)IB}xFI?4OI z^(eC;_2~4dqQiAe0bYM_3+ZN*EP$AptiSu^+_vUr)n&wcsCw1MQNJb&)tO?kl@{*r z0FjX=R_V|YS^HQsMoNfoxnWdK3s+On0Yd$dw_0oTJXtw=`BSw=&>iX68t{MzThT(U zb1pH8U`K<tv8JR0yHTFVCu83Q{HT=IlI{IYfky@ZKfK^Jf<AS<dKVv-!~qY({FkQ7 zjrA=?NrZywHzTbxuCY#EoJ`D7$ILEBS5S5k#nU_3d2IMqK^B;3E|hFYm7&HwDNKA2 z=AL_SCfrirde52dOmPA`gq44YZ84f6v>%P?ZC3zmIOa>9X@M`MD&&^p{W%!(Fr@1y z?=o;BWFmb1c{@%qJqeCNEAeSZYk>*owQwIR)%!up`vvC2zNN4|^a`>ET|91dJu*-Y zXkpdeECZAjeZ(B$9H9n>yVUz>$IdJ1bb^%H`Ic{pZQbAgZ-m*b@cvd@HxH{LrbFib zZu2je-G`m?&s6U*Dh&4X@m=rU&HL6zu2bk?QxRtyCZ2kberUZ0M-$Kr?<eeJ{;=bb zL}Y~Lk0R#TjA~+B%OAsFKt<g^>u>32-;wT)x$J$$9DSaYW#++@6B0PMgI(=JA7j2x zu}b?LOtT_OdPPTSRzKiFg*bGfubAxQ#qWe+X4Zt>)j~6`n8~wDw8}a1H_JKdK(X9P zG2~p}1p99fQ7IVb3V-RWH*4>W1mptMXZi_lRS>3M<}*AdB1ipGo+q*v!YO=a2Dtfd zfW3iHWR(cRR0)xp$sgzSQ=_*}NT@P^b9#zo_yh1A0%E9wOG-&WzAPHx4R#%c=a>d2 zUkp=F8K@K@y(wH|VL<wep#R$^(R2XU4!aZckA(<j;v)+~D8l{J{P-#7guKxGBz{wD zM!h77TUUOFBHJRGP{NBAbJ!P(P#PsNjzR3BA$ascpdz7qYaH@g3zOyZ`411{m_I@y z%^VY*Yb>NmjM|(0SbBztIHQ{G>+}2UgnK9sLwE1zDr%p9S@d<@z$0^WT`9YdRR!;f zi(*g}g~FeSz&<7QTm}^xr-AMHD=fu3e@iG~F+r;YuN`;)sLK($cjPMxu%m#x0VQzP zv#10=<Wc;wHzMX_Zr<*hEQ!LU>+ZX)N%*55VEIwU?N7A&<4(}p1IUq(ksUKV#T2w5 zkwT@x=0W)F(!P#1LW2)u9~)nO7w+k8N#F3t;`$xC2auo=Cg7QExWf~0A1yc4V0WRu z({N7fJg*WEqwsz%^5g#->`mqJ?RDh^MERk|`}cKUGa;wtcIyH}N~pL(U>`kkgQVwj zDSVQSVU<0tONjDKNbT9_mI@22CA6(jM}o(P{{z5)42qC!r67FgpGMpscl?kgR>$ZG z<z<Ub`|cR>wO|vT2qh||LAK^6S}ht0t{}cl;@RV!9Nb{j9d67{$UD<gPYGve5pn;; zBC?|lk9G7*O7fG5oXH>0YzJJ{ciLooNIk<SdtyH-Pc}D)A6oX_h=JS<{)8d3|4s7V z(=&~k$^r-P!}^+jw^=u%x=QUrs_>=v80LsVs0t{=I3<9uerd|}B>))_^ukEccyDy* zX>WLzVxto;U3EXUrt6%w>O=C|pZ+c}{fVm_7~D>)Rw&m-brGY0fmr<(YK;-xI88@@ z_|s_I`G`L(qR{L-dblq=SK6LXi0BJ27wo+*2~j$)A7Z8AZuHSYpG)!GV0<3kTNU`d z2R@*8Qgo-xbQ<yY*UT_i{c50LhY{@*LCz?9`*+Kz!~F~Er+)V@KL&kU7)1>)y-W$< zk?5VgUmtBZzgJ7a@S^+-R=^>HGa`t>JY}Dt+0OMiz{K@y|B`_UI&a<8@ABRfA|G(% zSLOZ!KL-BgxfiMt3zT&OtqOejlZ+clBa>-h8?nHkF&i56w#KtQx7sgEwkGd%VWS|b zpyFLnu(E||+I;o|;Rhg~yQ=deC3MFyj?`$7a=-6=9mL*U)acgB&W*tuf`rP`%8Yyd zBObm#B6*i)sjKMXLV?xsHo}LT=51~{hn<1=864sIgm47=;pvY5IW{c5BFkQy;I6rN zpoN1LKB+Q9z_<f60ZIajZD(Dt;yoPtoPHL@wU~q14<SrJJwu3-$?)V_U!=5RIt2(b zXXm;%5x!sC7mGCXcVBp~%O7E1dy*-1u;;uGyb1U2V5fsStOU5XmDmvd9u!Tx(YP%@ zCB@)L=4mg-kB`woECh>Wy~g+c{t4*4Mgs=d6~4l<;$}CFe_ldmTp!3Go2L;Jg-pDp zV=Jbs)mI2`OJt&ha|WpLPrI<$GLtg*2p=|txRo&ZjiZAT1}DW>$U~mQDduoU-6&xP z)5r`7t&<)CXkitt?+b~}7^w)q?eWaDah)kGc&2P^EcDjI?x~%dUawL=W_YF_wcI}8 zA-3MYjIpk2M;C2d|KFngLZGMx3l_r|oPp%Qcr2?=0#YD$@Zj$?I$2r+?ekW)Op2oo z0uRK(j>jEl1-W&!qEZS?LTj=uT&j#0(@NF5*%Fh`Y<$|>g@COsFUqqNN;tc>-hSD^ z_<>#@MY74*?+tC|-%9{k$^=J|e8UG+b;SuYfK8K0TTyy`=ocGiJm}!i3Oeq4^Lj2$ z6YdZb{!8G-YBmVP-LX&9h({2DX&~2)&YB~OQdov~l|}>i^bpfQ0oUSsd<d_@z30b2 zwr-P!p)gv752N^-CDU9aX(4#<E$v;;J!pNMpUZanb_scmqxCz4*@+CnQfVOIA<ikH zFs9NsQ+;*qd|u-nToa>xmc2EiUH!->nRC-i`1*OlIfHX^Ju@SLWf}-KcHg?X+RZw_ zuTS`t$$L^i>QStNkMGbwpSe|h)4tptS*b6{S>yRwFKmghMEmA{J&Y8~J30k_`>sZm zJmU?&^RRKMWt6SKHPwLFsKU}}iZ9Wy#@~`+ls=;B@cToE*}9_&sWDa~CXL#_n20;? zAn02pr^p6)_k80)?Bi<?E%uBq&(pR&krNDR@WEo8)t;jc0T}VB>5Dh0kC-_@l{J%R z4bkDRUzXpQOd|Xo{$7Gpgl6((WR;GWPG<2d;dOU!0e<v4n(>9KV$l98=CxES0*YU4 z5K#Y*-TPu>?vKn37^e)RKJY32{{DJ(2BShxq!#;D74&Dic5OSoAN%Dt2pwaZ3PD%M z@%zkgDFX%HTjMfw?WJ2qdLo@VnN1>RLNtGbcrJ!FuQ!$)ro73dsje+TD|e|#)>OaZ z1aHO(qlq}WM~$!Ld9Z-G1aOOlhc@jEkV1F8f4_PaAmwvY{$H_0d<Wt|#1SDzdO)fD zM2E&t3`zjMVWo#TCSXL1pyx*g+ZNRh{k<cCnKvXxz>>H<(B!9Aqv}(VeB5$4>n16I znT<3Luxx@XRiT4UaWuPBbrQx4!Qam`C?i<bd3;cZim`H^D@p46U%l#Xp+}wDB#%cN z%C|dkYkhj=sZsZH%VeD-@G|NQwt8Eolwu;S=<k^<Oq@vq?m`-&R*ScQg-<L-hP;Ml zKon_RUzS8YskL61#62w@6ujwf<FOsR1{&k*2}!ahKfD*O(c*9BZ$xnH`1%cIk!KSE zt%e!$dIj`JmbULVpT3>tWi2puY{l+0mRw|XarQcY0^hpZ-n)O5k01ofngsdA&Zuls zW!_}iBOGCbE2^2u50)S2nQwTVt2ySGU_!^WL_VK8k}I)z6w6@v<LgLeVv8gzyY&+G z>m@8*i#)Q7BW3g@=M^&qS*xRnL=h-r?WOogz-<Y4hm=|>BZA_<${4E<Bh}yU$0-1x z_&|_@*kNC(#rUYb+505oOZj{@>fp`V2;oZX&@?mL;t`Eum4AB|v90#KCdBBBnIBtf zo+&^Q;_ak=`AFXU+Tn1=OcS@1$k_kv9creROt}r5toq+N1Q=9iip>`Ba6BfC)?)s+ zz76*saVM7jPI5_gaYFD}>+ao-d)xViQyc5I-****$4<<bOz*;inqhnb65dOo0SJsL z#%OuHtf%s)OocxJJT9{bs@8k@{*d0^MQ_hyVv`Q#`~6;PCrF@yH*7<wTDc9t*6Mcn zEX5d^AQwbYF<G->gG2=8Q`D+Y;+hf7#fZ{OXx$HI;~RCRi-f52#W-~9fw<qv23<y6 zVfBsjIU1wR5+}wGsp<{+X<F#JM1$6MmmNzDemlD88Cs`pZ7*~gDY_u~p0&oIZn9Pe zYSEuM^WeTgr(Wto8Ls6dc5}Jn?C-ypD}O)L&w`;ea}$@e*wx~HSB~`B{6{XY(Gn^z zd9!=a-F4Of1-!v))E9qD^*0iHRL5Wmx`Keh%G%bv;!Xx`5iKeS1$82|zOI<wqLRV0 z>P5(2m&iK|je&E`;0;tzEEth4S-fuZu#?ZAwF6PKUN^zGy6?PumEep#sgf5yfuFn3 zx}rIgj5Cs}Q4I9aTVG!U2g4jdkdwv3PB;A}i?Z>seRHjJmBp~Dip%}Pb}neu{ICB@ zVanPp<gBDtGN3ANiF8&bdfu@-V}9h*`@bUhgfJnnEzyTU>E`fPU{@~4<;W+L)6i58 zO-x6HXIj`~bOchl|6<r|<BhDek3`zQWpR`jjW0_c2HFf~w=D-Fzr&ZgB;~dBR9o6P z&VKE<Q>4}CVNv<#JAl%-jr$+)X57~Ppk3X&a}mp~Ln02v5U9CK!lME+zX784=wjUL zzZyo5KsAi><3}@H<nll&vLBS4j95g5t=s-aC6U6X{a+B}I>yLKUdz%CvQfkINfL^j z#6mggc!br+?JzcP^vlx0-XiEbYHxe@0e-j6VuDhWU;_G`5tHZiW<rcXU;g)P0bCAz z?z!>%Tu^(qCeBv#s_hgMqW)(yszHJ>J&q2z9C|Z;QV8u-><U2E{_p1S2F2T<Y<|tq zUxK28evSdGaUsxK{<K?b|DT;~N&-pJtz#K{dpn)btAdn~QP)+>jR)iDJkI;@|Cb!h z<~w{l0K&5njp!pc`p<v&%MUQW0!UkcL2vfC8!}A&kLeh!bUQV;K3UP#ZLqCV(ua05 zVAN@ikm%^S!32g8_J0^U;SJ_uwkSf<i^-dae~)#(y}iwSx?gXz^nWXfh=w);+}-79 z_QOjw^qL+p$A2b6|IZR(+5g`&{%5m*k}?+?n@WE)8P|U(J;5(W{NUigRtJBG;2#4c zxrNJ;1LM8*EAsyy$*8?HUpDD*JX=&oUjF)J{@-nb*Ky5!P5`QQahmC$H?)(@uZEaA zrjuDsouBSqUZSBXq6dta;5-I!b*m}N9go%jw-z)C=}K$u-ZKEpo5Als`C|IdR8oKQ zct71;g5!mn!vDJxasV)Ws)oK#b%4IG{!;bNq39Im93MW6$OpTwcmCh5AjrwdasD&U znjzq+5gHo$Oh)w2DpWYSjb=JJy8l~5ey{9i%Vy}WjN?;MF>}S+HI!&~@Nww4I;42e zbu&+I9=j=VtG_`nn5F>O@pM{w85tE}VZWDt=%*%rY^;DtNy~F3qYnXo_wna_FD(rF z*p01(^Os#bI?fWQfs{ZQDQ30Dp2Y13fxfxPn(+99n@>fQI^lBZ(E3paLT2mz^{d%L z0-xD{gZW>G+{VVnW&<nuwe#ESQ~}SAzQrD_VKHJ#U-kxR1jBLCbc#aqL{?<4>L4&H zkuMc^yAK&1fi!ru^ml$rBfs6(S;dD_-$Pf=l4@XJfY<A`63|gL-(S?<{Y%ta6`|qb zx&GyX4}aK4GWe2&)XPBEU!}};@aX7re}@ivA_X>+X#UWxdecmBd5cH*@aYGo-qs{3 zSoZS#`@LN+bipk_B2G7d7IG`v?+4XqBBP^k4FA1P$e33R2@E_~aa)^-p%B%Z5K_+v zjf^CzREObYsO)1W1U9ROtutDQb_UUG(0x#G&D@Fq=LVE8&`Oh&<W_&Q+hrtHxz*~w zrXKvtBtGmCs!{w@E!6v6UMv!Um6&EoPW=Vvz{e!pwm)$M`#$d%?^*n>0^DXV6%k)( znB{)#luhJUZq429Vwpi%RrxQRI^&r_#+R4%ZdTQP^wE~&Nn!SdA`Vbb&l;*}6i`T= z-onxKMGcEWH`?@j@PLGb#34YG+g>H*Nc>ysm~%MV(>@c*X{t(-`1H}*Ka@#(1c-L~ zxrC&Xy%8ds^S`g4sYzM)_t=;lWqeZE9CZ8TW{VJ_sP95~*`gxan@mQD^<G+oK{bel z;|#KL#fq#KxfEt6^b3^Jetera-|C05_u^t=jj_g1yDg;Fk<0`qFsUSpM1$YjMorLT z=V}AWMi>l9>O-T10=>HH2faDq!Xo+Qc5huku~TWVU8!|F*2ym{Oe2Rz0U@fv;UEMF zdpb`V`$kYsR`${tA6_B?D^RS9@<ObnhvAS@bDVaj6-w(%9Q!Nf;WB6x0ootX-Nyq` z-jig&Kc}v$C@HPOe+fn>D+jW_C6L=s?0n!PB2;Q=*_u4sayaVo6P14aX+`k=ml7;t zajdv~pIbd&E2eTt11h1$_;2X*uqd(n6<+|N;LJp>BpK(6>rNjnMK#U`<Zxy>ypmiB zsg$wl|4gy*V|mvdOS@>lGl*+uW_B0+9;#4Ev<2N5$jF4Tv9mu3T@<x~%-YA2rx7kg z?@{^=EgHNckJH?sjcgGbMwIMCkV<##zhw{>R#URDD7uWv1TxIXfPTbaxHA8D94rwW z=;thNu#n?pMUlK;+Sq~1BM$vy-ibjAjLX@&@Jfg8+JzY4g9vyu)s}Q1qd*0;O5+|q z@fDM=3<>zTP3<>~?rG03RlG~bjWGE1|8;Febcm9h)O>*ca(y(XC>ce3PWJjE^zQI) zlto&J=E`&nKiVwrpKbKW$jMy^mZyWL4LKMpq}(iN4Qs)sQL1U1eb5J?10AdY46x^d z>*?xBAXSPv^zkfyV{nSCz=hqr98M3Q%?Cle7`PZ{BnI%Q`Tr{DNjqq*jQmUV%BpA8 zW=s<UaWqF$ib6BLB8l|h{Hrb&`xrc8ocaZBJg$GV&TmY2dMylmvQ^kF^pi_FxJys| ztD{A97`bXF%grt&X@>FmW$xn8P}q7chOVqVS5fT+d`Q7Ib2E+|moUvxgjmCk{a+cG zerJH>>1Yf-pj3ih-vef`kbR~-m4UY-Q@By#VNFfQ$SrDATL~NZK=HPuW+h})Q${2E zUy{$2Sl<bwnJ#c*g?dU-bjICLqC&qV=u=rbXk>U$Sxm0lR_qI(@IEtENr`@YT0w6u ze-^8e<-a92z{uGSSNXs~LDBWUB68KqCv!qO)hd_H9osFDfStR*(-bBxHPbf^ryu^C zi2@6VIR9vUWKCzUROJ6^=!~4DWie1f3;V+HTe?x9Thllmj#WlJFjK&@dGIoc*$`N& z5oTH?70aL~gT^vhw<)V$J`VwU#g=IOsiz`1T%}d$j$|p01;YI6Lbci9L~ig$>-qm* zu%u;VCiA4@rkkA1k)PESBAQkH5-cowUwt24k#p`IH)PZ>6McjaHn&>p3P%nPaoPO0 zql&RF3=nxtY$r(gTyK4?0kc5AMA_KbYQIjsr;!4L7A+oDWP+51#p)Y~f<a{IU!@f* zOdeiOD&Zyc%&Di}S`5U{XHnw=Dvfg!v+1vhi4=D8M<HlTqHa;(JcLI?IEzQvE;lN> zv!<qG<ihqdTg*)`@RDHZmaR^pWDhL*+=ji)Y%QI?eTVya-O35|jq(n(e5FswCNVXA z)ra1yhZY;UC*1Fv9r*N!TuG)*w&t8STpjDSqYRpovPJh!EEj&kHaj2JPfpfYD{w(S z0)0j3AbWh^<6BanY4d)7riIntz4072E1n`pZW+^lH4QG;1_ibn+2quUSNr+)Hmhz5 z3|z-<6wU5}%GNsn0>Czz{lkY3@;p2|A8G#~rf9!|@$YeQF85bo<+B9aE==NYYz=(; zqSD)QW^~bcggq`SYlrqqk<salI}6MEEH0lfb?SCvT#32!x2K)iUqQ_x^kZ28G=BzO z-o>`h|KgdJghLldK_6|Gi{=bG%Pvf&sTlbs_`JLA&lgv8TDI~Of_o2iz^<PJ@+e}P zc-`2A)Ba{&+n7hs+6k3farMNfrR{a_{pY<i6#*biLi_pEBD8;S{MVMWqT<|lYT2ud zQ!g5O6AnHaO-+##i>D~6)DDxh*;C$pz=w^Lq6r@?A;H&<TFp`VQR4;Pd$mS8Z_5^I z6=_D_Fi>;$7(4^tAQAbE#(Az>|8S|H;_u1I(f=@$&u23;p0Rx(C6I8lhLJ}o%1ty4 zan*SYy&lnRS7&G-o*m8Aba}LlXIhmmi=l0EWL5>-V|hA2q|<Pjw_CAXPHp9O@cZXb zSV)u`Mfbl)!6uhT(Vb2+orahEmnF`(cXpbt66sV6;1nW641C&6HshQhtt%7D*2><L zcy&9h7?dqIEQb!T&uwMU=3_C5hlo%*>FwK`OIV!FJft)eCl;V&)A4#y$PwM7Pkg_A zz^%0m#w%OR76=|Y9BrdDGt;dK85DV86|(yBrQrMb?>N}lxgjBlD{Y>dKoOh-((Vf6 zq^ql|+2gv{>ham$mb}>L08k?VAm^xFMY}1?iawOa2GnT#$6E(3_nTNMN$YkiXfT3d z_R&RLdU|?b86^CQ>ye~?F8coQPb^<5YZ|KOW<CInsdeo~vf%?uKPaqChL$Mvx-ls4 ziN-w_XP`#Sb~(A#UTRf-JP|0tO+#<0T;`GGdhoHGDi0{s_p$t<QI9ks^J$F&M35pS zFYf{qr^LS5P$-v@15Ct}IPHyEt~A%Ba(vAJvh~r^&6?YBZnU5Zz+ugl>+@Y7&d36A zbN0_dO`YuuZw9wRH<Ie|Dag#ae>Bb*bWH%i2@X>E{X5eQx+v#he(!ZFKm1(Ms`CL} zx1ECeKRg9*80sx&HzQNqKF94=bm}%MmeiBF7&mMu;@OonkT^L7GwHPrA!)aB+1=sc z-+SIpQwK`+w=~H|hWXvfNlDq~H~=|~kk7TW{r-?)4rp)o#P)%5$YD868%e~afc;+1 zp6WS{M$wfrA?s1lVN)!UOvnJ5ZbC4rIGp#D(1^K}+B|PGfHhivF&|C;2`K3F92{!E z_|EPJC}M%`I(XF4;>Jeq)*OrP#n`zqiLv?Pl~N?<p}*1i-af$X!lKi_TuMjo_-okC z6^}gBo$mAKw>fA!gwnOcvIwhUB6bKk?MBR27*<qtJnM)Cp*k3hFzGiJS<RJ?wlo24 zqs?0TN?<zcV03WY<dB@)QD7!(nBNa^Jxa^d<)$rqX<)X)5V+W^o&5S>s=o|hIjqxc z1qGkUxlaJ`!OR<Q?seefqgPT+r&FR7_CK3tf_a*l4O)wUPVwu`V1hb>1PU^Weu93t zYk(c(p!56^$jr&JwtPjnL^sVWt9@3k&HUry?OC$3$Bs<}CEhU3Q{}mfp0}k8Avm~c zMZ=}UYbaTjiZ5lLXqbs$hh8MZHjlbX=grs0(?8=LUW(OApe;_)&A=l#M7EbkB<_@w z7c(2o5gwfBPSFlB(o@<_GYN@x8k9odDSSt*I&2CIB&~(53J1V{oh{W?Xsnix_&8Jn zY=H6}r#SOfpuXubwLxnHC_lh<zR9UDy!Q{#=dJhAet)wf!r(Q3ae<PMn3+Eh!3h-q zMSrW2tdQPO5J1E9^=89q(fusKm4r9<sh|1q_W?sQ266GyO%4u8Qr`A7>dy9VIHKb0 zPT3OYOC-@-F-rgMFk%tBEIY66vcRJFj9NdSdi|LB=g)cAvFezD;ZowufL1g&(iSY4 zcxIr8I+n~T3XFC+Zzh<17#n1XXqM5^N+vp^D%$|?)BoDLbBtO94@ffy;D5AIkV|*X zkKaJQ6u)Ja(j-3k`|c<)RsF~9G;v^mJ{=>yc=kqUvx`tyT}$jhGk#ogavmeK`8I<< z%l1vpWmx=?F4xk)jm`KwIcbcP;T3kTTYK-jog^l7=e{spg-7MPM9YR5Gy_$*Gk_b= zPiCQL1<MUEh?lCI0Vp?@NyNqB;=sKM%fSTdO^JhdG&EL+(?9PYFUL4-7HbDXl_$As zF7uqTq~t~!)kHk9H&sw@x2z`U0>k8()PNa4-HU+h>Sry(a76|+6^ZOguN+PWe82ln z(?z8&A{nU@VALjq#u;c3ZMXe8lm(kUg2~PoDH9uPmUfZJS-Hq~N{BvM+i$da*#adx ziM-o%DDeZ+VSqUktWVvByA{-`Iy9Bn(%A3RZUC@@3nb2VIhs%5UOAP&k@{NB-yla` z*3HALT-dR`w(Of7aJ6XgUTV$KS{Ji4k%+x4*xknEVYU(Vr+f33lz+b%=mwT%1H)PO zNV2Q9eP-YHgzzgf^`>#z@iN2Q<%OseKWqxr!AJtF6}6O$5lbYcEihf>G*-EIHT;ie zScEy1gtu|m+}+zt(vP^%`Ed3Wm&QyZlKUOSg*jWvEevTqjg@nXUpb8Yts=cWsj3L% z5`{2`)+~xo0{<-J@BW0S)oiJQv1wzMm6IPilL#tz@eNx7>Qwh|mXIM(0+^x!fRJ0> z8J-~A)!W-kKhOguh9a8ZHp6*~VCDuB=KnF_Si7+mU+>Eoz{m~}YE5|Vt}Gzx#!(K_ zQW&6jzRSOM|CWj^9b)zJ{G@LRgk;X*E}AzE$OeF%q2KjXE~Mz3#e`#;D_;QL_MN}K z_zzcAP}sx_UUFyNb_~)>2-wl`{0LF<eXT`Al)2~``bL4DD(5KI=37Xvjx8`xq7P<L zcO=W4=S7<F1yI@_`V&eiQBhH=I&%O}=5nK!vxagpYs0^>%F#-P?$haypgi%0bY7Q1 zT&5h?;hx&o{&3|hqPy(S1M%i8Ev?+_MqC>RW}KpoU+z6lwJa@&CwnzLJ6%t2)rF+^ z3F`~cGkS7K(z)m3Ra=21^uNJa=aH~vu`%oH+Wl))g5Czy;jrXiJhq&+A5rp<U0AS& z*$y#-*D<LXlg>1Kvo-MvvwOi#=P_&_<h(KR)JFQ&jjQK@2C?~t{lLTfw`zCXH*@2^ ze-@ZN1C2B=epy4`0@&>VR$vXp8#BusGat)zp~D7T7Bmq<F&T>}rGhLO0m{DI8}lR+ z(1~EPLAmq@qM6mF&}og89K2#5R@9lhrtJ#_xffcV0JBK?l=j{jIi!e_OAU4<h8;d9 zIr-e!CQE?3KK3}>0-?=h<InQ0z_ZFc3Hlx5!VxOyU+%9Cc)jmy0m#rAP5MFNoRZB( z7PyN6)m{RR^ij22e6m$*;P6Ojmni4ry%uAD>0+ZpZpmAIpm@<J0N}=w-9P~q)<SMy zo?cu>C-ueTpZ!19pQQBDj^-*_Za%Zp=p2|#ql}@2GnXddIx#FF&PB?5W0dP^UM~TC zC8x#szq%3_Ftg`1n4ApSd;NuU65xDz&?|v_VMD{jqyUIw_EWanVSZ4%l{QF11YSTy z?ar(+*9-g8_n4VSYqY(P@%N}rOx;&B!~IR42e8b~)TS8*L*O<lYs1QOi;c;mN%(g8 zJJsf?`4<3>)c4ec<Fh2dAj#&i5OvJWnwLR?2#!$1DRf=0!>6sp?lPon;k(K7O);wU z0R7}sUbD8qjdbKPz@-eQ5z;C>{QUV-o&FDZw%X9~cy>S#D#2#)-sUb7`f7Tn8o0M7 z%={XdKO8k1T|29;ez6Xxlm@9Jr07s%9Boh<G&$xA-tSY9@PjL59)OO9O6hEws_mPf zXfgani$Du>I|jkLd8Abn6IP)l3B{ui{BBZPDMg`{X(of?OQA_d2t#ygq40CF)#T4w z$C~>YhC;N`Qvy82;Ia#l@@If=#>K@|KgkhCR2J#S18Pd*2i?Q6hSf;~GOr&%Bv?H^ z-r2%sJ<$|A0OP#n7zifStxM<RnunA5vbF-HHIzrf<m{=FrP}P~(;J5s#e%>VK*Zr! zOvx+;R_5gH-d>OsTIek>)<Z<xfZ97D?y@Px0GP?a)j|7?KHtYfye-HrUZ%^B{xf9H z_G*7Zr^-b9{7~t2Aks9jpG$35nwyZDLs~BUlg$R>#{?gS^>N}g-oIRhGq;a3RJ3l2 z(QkYnJ|3urE72%(?9g<!glUDb4tO^9NDM2d0U#1|V))3U7>@lAAO=QQLL@PBHOh2; z>bJO-$Hb!LKnW5wG-6d~?yoZ}*KgT<VU5Sh6^zBhQdSgoZVQ^@;$*OQ&jp%A`%N^z z2D>%Ey*^Y9U`*2GC<_>xzWk_>XFpLOpLTUnn8gcw0C1WL<C|ZBxR1GPR7m9*aCc8) z(l6f2^c;PP3`dBr*i#orq*4ied2gMFVJ0$`OcZ~*8MW5t*^qYTeX`u-@qD*81JF(l zUg84%hz?Aj`5WaDlrSlvMM<Ar0-a3o&&3KbC;D{kGzmq0gjirnhiU5f?*(PZ5^!?q zw8bel&^|@He^VKPnK8ui4i+BOK2n4a{2KPFSIk^!k#3{?MnfOVAPVILz&N^OH*K?Q zIJ{fNrwE<rvtR!PZFo?NvL3f-?w?M5D*zIN?>yYx2^eArYH=LV0YK-sT^4<J3*#0G z*>s?0O250N<rc$TCVRQ<_W)*jIn4%f_EO41Lf(_%M_T{@T)&8^17MP;HKe4;xpo0! z{={8>BHbMt6h%YeI*aiS7E~Otu(f?xpQKq+la)b^Lg&J^`5hr8%#m~kn^VXM_@5qL zjZwP`BMOL-t+sw5BnNRqsqjk{M=XZ#zM<n{QqL!D9Zn*Sz##1CRLHpL6F>$l{9pox zih*z`WgzS=65J$Iq&JD9GrJ`<uK9zGG_Vb+CO&!^vGdt-%u6;!S3F`L<KLQv4(}s} zTQF($vWTR5E%e6+1L0hxOk4enJv>V+UX4Sb3+nPiln|A4H^`xiIfD7Ts7%-d<f=Qy z6Y0&hrH}H<?g$s56l>UqUl9?v{f|r3OrYC%`k;h?95KFBXQdg)Fi80kGYibP=DtVr zbfD<#=#TYU0GQXO;daQcIkoF}svn}+Nxmgroj8~(<U^<3jE`C4S_SvGizsFZ9$#o@ zc^uNwfG{(`j@pUG<20qg6ER_bxVqDlFXs$>SCE#A2ZsQ7P=t#5=^|&xJ?Z;*tnt^< z|GZrS#uNqMxy#SUzA%Cm5ZXIPG@H6i+D98bElY7HKG^`hVZ+Bwd?uPYN&{sva>_)M z*T&&Du7BpMphM)>tDdJk6bDEuy=H*2!3j+H((s2~zBi1=&_?F0B32zHkGhk#oz4*t z*UO~qKUTlrxjhf7CYe<wa8M9Z`TTXc(ZSXiaGoVLjz){iQKGDk(R$QDHT^~Dzp8l; z&kNU&<zi;<|3SBY7;O;{t$a1r-|`ZklI4^ERx{<rSI&h&3ho2GiLq=Flt~HFZ<v$@ zoU-q{dKeJ@)Ti5;Z`XcT(868hCA-tV3DC`xz@dvq7?F@6ybn72&<RHq9Og<I2}AJ4 z8_mbx46~TDtcUIrhMvcqnw<mu0FyiY5eo}Tk4f<E3N^K_ukVie9}q>u`{O=&LYpVU zyVi5q{9XbU%~(4`KtvUoZayfUfEVG{Z}-w>Zokbzi55>zaO=(RHqyawtuKY8802l8 z@N$&G%*Bz)X=XBs7m?7T9p(vR*|CGFF(@6>l2<+GLe81$iV@En=)U$-OY0O&OcLh> z)tO;o%I(1ob;`LC<ssj90E$C-=?5Fnm{3>)>!#Hh9dtQwt;VU=ye=bUPkW@_W9L5W z<ABZ}l(#H09#{Jt7c>lMe2`?Sbi6UD?$Xv(jqgD`cg$^9-?3xV>cvASt?3M=ft;;b zmgqGGbjG}xJirMF8ZpeYVLKU4L;gG*N|Uf8O+)V)Trra{e6i9FkoYIxivb#@(xhuO zXCI{kX(KXiO(mt6y;%Sg7I7-oMs@Y`IPEq%2Zsrs0;`Og+wLELL<5S=&3g-7O8hr( zQL!OZs@dc)17!4e7B&t=v*CpaFoGei7rQytHpQ9%^Dwq!fMprpnOA4Cq+!=~ojBw~ znU0q&GoyzLkBKMy#;|+zmSm21O&Frm8B*K=mZgzPUVk>!ivU`*%cbkeHQbEFK>G&Z zTt!R!b|F;%lduU;Lj+WqfeTbdyQ?s0<>vTAPa_e~Q{(bi!vAds`cy(CD!`>OSIN25 zLWvU(I=w>CO&qKg-A$(#4H6R^7CC#-Er;cLn>3==MMc3l4zje2nX?&uZoGEJ?p>cu zdNX5m#ms=S5Gx#vA+iNl!u)I|ekd7Adi*VrzobGw%Of;^nq*I-H$ak)g=Zks9=`u2 zQ>EL3ZXF`|id80?0-1>-M>~2i@&oe~Rd%~|)}{=ub}-aZ=!+Aj&Mq16d(i-JyhTkR z!k{rDr;WzW;vncr0dW=rGD{0)UY<%l?ytMtSJ!XL(l#M@(a;CtoE!%Awe(xMjdDnz zMz3fX7e@9CP@OEh{NE^~C%SEuP(*}K4=15~s~&vpJqo*teY$>MXeQ*4&0dD`V*Y@` zMyZ!rxdqTdnw7B3uM7+!G(t!SfB%LnZEhaXJ!@+V6H|Rk*@}19L&B_ge*gkN{!DM@ zPRbZQ(_Lbu$Lb?Zht$`FK$)Am(q+mW{P>p@X!8Vzcl9cYlTvbTI7O9*)kBa&x+O|q zb-b8_*X6@68q`Y}M%9^P54Fs<nQgf{X>Tn7BDdLtIk%VXxvaUDcHQh<ZM>Icw>G97 zE~YA?m?Jf2E>PJ^m5Ry{{Nqj?S<fz&VPb72`Op8zqpP;2GW*xuvgL1dWL|ueS_P2k zs_%=p=G(=~W<WZd#tWgR5clgAN1Y|$l*~T8j>*u>%Zm_MxmtKfBkJV5>l#%K*`w!Y zT??J!&!VlH3jy%u$hYTDL}(zEZ*-J^Xu!OpFww={Q0s9-k<o@}Xqq`;nsj2dl^M6_ zxtZ#JYM!9=t3$mIBwmV{DH{x6g%$Y>l@$7Lxwy#^O=d1d3-aQtS=Tf8xM-PnsAlp6 zWt58?q|El$Iyz+4jed+5@qA}TaoN<U-zAY!J;e>R<YQJi-tGT+xd}!yNXko~nRMa% zIOZ-7AnS-jgnq0(MY{O=SzSAT8ea_=p1t-AZFW6sw-v)6)qOIFcYX2+D_@CHlhX3H zm6*#+pV&+n@?G80Rv9-3rulIP@8G&Ek?!&_Fh$C8RwtTY87R}Mx*kmCcZJvi<&Knp z>ZgjrZ_MmwiEBR^4yr<_Y*zP02RhX_*SsQArB(w1?0mz&Xmin?t8T}IZ6`-RpEoQy z=zfJx=J!v!Y<tvSChFV*V*?V`!Vu~j2%~m2WRV>Z7L3o|geeYbmMhK#(j*@9`Po{t z^HLiUEWDAC^CjnaN;<dWq>wlO>rI9zBg4kZ>hLeum14Y?-S`2Zv}PpJfvM8yapvB* zDZ>DGG)HwGGF%0K;a84ZCm;c2xDtNbfG;z!lsL@$GiQ1>9sAhXORCe>UrK$1xJ5b+ z@ymTry$yv9u0Mr%M6UVH^67OD;be35OO{M^!;LJa73XqQy9iMjrtk#XwgEd)Vn+Co z^eqi#DA1Py;XCLUN)SkY_7wt^mBT(X$*FVIhgJ>`!~v?Ub75kb@()LxOu=0g>V^>M zNd^^tK=`e(Esl>O;X4Fqh|wV(17+PbirSQR3NAcW;6R^_CgT9K{}So5gC7G(x9tKj zYcafL{Jyc^{z{|Qaa(GBmG1W$OV{*E$a8-u;@;~IOrK<}7%yeBjJ&@{K~_Qd`ONrz zR_C$K8E;3F#Z1~6zz}^^-_>Hk_BsR(l(#%rvMMUyLnjU)F*@|~&=V;;pV;(e#JjCi zasn$^im$NlWtpf+r|w_j)AH2ez#1c`0D`7^<37NUO1OMtg_m%5|ET%-+XR>mQ5hLK zuy4<5E`d>gJ~3z3Y4d7tLfOOe6F5c8;%nrofB?71seGt%Lq1Y!i;%qe>GtiPP?4Q~ z(rOV~<sAwNihP6y7l>iogH3%Po=$aH%0qm2B##!$$<GIv1c9DKyiA-8#&j=;Fdm%* z&fhJM$jg9j#o=0jtf+iLVf~T4L8=S0NyqD*{L|dK;)MvUHf~B4p(E__tawTmhDkkz zpNiA%ADLv1@q52+{{4L1nts4v2{wi9md>uu4^x~c^Hn{)7V_UL-hFKVRN^&-8xadj zcViA8;85JCC@-Ms*onFx(L-3V+m~z16*W0;0cLsE3|OOC(M;Ox=HUjtES2|`SYgP0 z`FPySJBO+ShaJ4i7180Y8+Fej)%1j`lXx{IoL^+CKV}U%tIjp0t1##r0p@?T@4bT* zt$JO`W9M?dhGAf291c8E2|*MdAkNtd^cMsPA?k)blCQrd;oU~1>b22l3FC(6&qBO_ z9{1cb@_+!0>U??*kj9Wg>^FG4PL_FyK9#`DW2yb3w!4f_gp~89l>1p>Uq4TSo_wSr zJ`-H6V#+zimN=j8N50wb`PuMN&EiTe;-}|Z#kvbVWOuga13{8N4h)3getuf&c+&l6 zSUB+*Kuz-s4?a4}#T#H!1-WeO8USo++EBc|5ad)sT$~5p+6K=F@ymti`L|7qf<cz- zq3xMjGxQ3eqY8(juRVO+sUz8E9~Ti?rn?^@ttiIm-3|k5Vh~FBpHWB5cqJv6?cucf zpHz;T90=QZw9`hva4XUUMqONlm%;#dk>L51rj#7-+}iQb(!tKE69?7JCY*N*bCu9> zKQqviQc+Q5w50J!E!BSA+HEnlomxzn927KdbpbN&PCL-Q&W+^iq}>3Gd`}i|#tYi? z>C{@pmY^8LN5>C~WT1{rq{IF^tYQ>Cl?$7-8`2_Z;3c|DZ0BH8$&w3qAzg|TRQ6!l z6^3T7Amxwt?b)xTMm)D7x#S{dIUt2hT9u#VQ#mGP%XC+ygvI&uz73^`NI+-3XFP&| zM!)wYfkD%9<cb+AnnL6PFgc?rGVKf^&NId8qdivYPG~(vRxuTdpOGmfTa6-VM3r-v zs@!>TzUMDIsLrs`8dO9@PF7frqv*^^OuQ?Iy*oPsL6iRukX27?qtb3_@mjw?sTIz= zeM|xZV&aIubaWD?Di<QQ{hyC?z={4@<09Teefhh0y%w?En~sB58O0XL(oKV-AJkMl zfa>ri>2|q8KicaoL+`MDj%8bx67M_;t6De3UG(m)$q96r!!kZwG?=%7iYFQ96dc`? z5xY_`yuc_TqM|Oq0ayTRjRj)g00Ijj@IQSV!^Jd+-GbIKX<CC?vmrbZZu?I0R#3hu zx0R(vcDdxv*QGu=`Ni0rx55thm&vqyF7Fg;njJ+LmgdK@Zjjspqt3SfYNr(8(y%$x zb4cD`T_mKKp0WYkR??5WPdFhXLtiB#NW2iD*GQ0Sn$+WRHMzBPN5l%y(K(O)BKmFE zngyqD%~d118`3gkgX+#A%?gI$^i>hQe7M9vmDb#<oi44s-m)^=5{ee6muRH2ng(Ip zBw$A1JPSK#3VQ2=<1@RiN)oDemKt`f26hJp2e)iWLV6vA;RyedDSikd=nshf==<!Y z`2$grcE%Tp;TOByc|n*5bl}61Ff&)Kq3=iPPDJ9nj&)&LCdy+asrvWUvKrwjRUT&6 z@z_p6r@9Yq+0m8|te0lr#!NE2pAvDT!|Kgv_NfXhqsq=Qe$|d23ZcX+_x@XJ-_^AC zEb6}f{Fus^jNnnLnOQ2Grn@uKrN04t&O5&Zc5cO1y5cl}b^XLu?|!%`PyN2|Xm*9x zX4$(M08Tbf<=DL52Z@vHmXdWE5F#4L*&-kjSRF6cjTZf6lzVbKT@~1~p(cwHkp>iU zHGp8|biepa15ZcV>lh-;^tNJx*}xO0FX_C!-ve|&{*oXf2FSW@s&z4nGiAD5n&(N~ z^56gl?C;RRn|78N5bgQpBgZ}xM9WLZ@9n+V{&WVNrO=BX4bE##5}s_xv_?|H^_pk! zQ3?g7hnr(<Gh@`;W58GB(BUVIn8gHOlbsvA!ndggR64+((marWyWqReBeSrnHO@6( zKCIr(*IQotWHf>GfuV%O<{NNIJNEvskLbi5MU4=gH-;T0)K;#E0QUNzSErAlsHbA7 z)7NcWg3O-?nWdDvIbOma53Am0y;EpJS?gnv&ZeplQ2H}hku?N8`kY^a?7llpK%;VQ zxw-N=ucPq>=k-9KSSyzazOnn&zVhBm-vf{JJWGRZx4kx3znEXa@lwOl;Y^9fkKZg2 z9&Zlt#L|cX#Gel^c(y*Vl;1Dvu0;xh$`w^gcpUDxifFk6!U0+%Ea+7}@+q05#KE<L zvWJHf1O@i5VL5&3W#WDPh--&V;<^~!M$0sGe!*h@2bq0XHZD*K)Knr2ntZrEk8RZG zG(5%3+6n-^co-*mvxa&Zn!ryY=q`O!FK`D1QI~)6iDYD?=Z-6-Y*w6Fp>c3|V#@1w zS1`$b9&zucJR;(v4M%BQM?pdHI*S>H*%~-IJ+zWH58qt{D5<vXs*W0T9vqI_QZ!na zMQS_L&LCE4q%S_uIl`$zMYpLcjC)xH`lIY`2vbBDLE#uXB2<hZ<Fw371L_UvToxzV zPt0LXUf@jcjo!86IuSk44DgaA=JGtKUZ2;_TV|16El^<I8(BsSZKbyejmYD&+gerg zGDpp&Vdq>A`OIu3<i~2>U+mLrQiCs><ofcJiMl%aQFH%pinkdHv7JeP&E({2hZ2Mw zHXBQ%9iJb0%xhMgocej#!0)SDXJA+#H#{~Mg_(il!f(vArCA9{;&2?dTpOA6Tgt+_ zvZHMn@v+Bb=NdLEi`cwr#v^TlB>U(X_#^?%8o3=)1L&e$ukZEO+%_{p%5Chaw)CwR zYkyvvV4V<3<pu@Az~DPbB{SyU@qfZ7gdkv#@&SLS0G~}x>v%8A_oN^U7+=W;s?5e> zVU|xZAo4Vawc${Tv{LE}DE2)J#Iq42JdExmpfh7bYq%!3opoPAfj__zQ8l6}2yx~* zB_J+l`KGY9(SErYFDH}c0=<Xox@(U0u#Iv8NokjnT8x{uP3Z>v!sYQxeZjGUdK!;| zhqqp>k0#oxO4Jj+pzwWnrIdFSi$P1p=cY>inz6`5)G>~nO<F+$T-={a*<83@J2|6S zJO#N3ZWjH!Gdfrtvak={`CsH$=WeTdI85(9e@Nz%wVEph_-cgoPx(xUoADuypZI_w zbEE*AeK-|_3H0_g*|3T9<<^*pVe9;SbNXwf$Q`etmT(4khC;bDBxln!2ok>i*7lv4 zhJ;3_*`sc%9YMRQ7D@@SQKu5v;`VG~luyRuLw8lNt<mgIV3(Apnp!+jS)=Q5Dhk2- zy|`m4HFQ$`B0%D7%u?&1X1wVQ#pZQ8Nk^HY=7@~ZcQkYd?DzyF7F|K}R3;GoKfsUz zZ0Z-~q~E{krZY>lE!nw7k(7NM@O}p&_6^<r8J9{bT<|pW*GLDmj385V%aLvQN&PbQ z<YF>>@UP);mkiUv=H+Ko=^(%;&$|A(xU4c}%I78eoznXuG0Bkg)EVH^<{=?U*4@Bg zP5@?ZMbfy+rZxj>IGhd<<w+hA3L+2k=z12dz!kfP)A#aqxBJxB#}i_0Eu6{?QDntV zOX$;0-VA6&(~3@CM#iI#m%LofD#Hi2nnGwf5yX4mjQH6a=?no!EYAwRo5Ty<dyf@; z0fkc9VMo8qt_B9ETvz*23yR@@`CSS9U|J0LljPnsKVIY>PO+k7J{!d_10sBNk(f8l zPp(D~!A!hdOrp5j6S*L!)gc$-%DXqwzM4;S#Dxnw&fFt`<InESdD!*OONh=^tPHd6 z7Ac#*0Ag+)%q@nI%}pvbak%_5SI}-w?f-Ffjg4`2Tey82JB@AIwr$(C-PlRn*qGQ! zgT{>;+in`0lVpN>dcVvM*t7OJYdzZG{w)hUAd&f|<MU=4JsTqZr_u+O;|ir8`VLPE zDlI-<KIi(2MmgA?3%R=_$MMO=s-37N>f@-!8ae4C4l7^9>U5D?nLNgHN>hd^Hd)d7 z9Em5I`&vjbKrJ{(c<#xW7m*tZg6RqWmnpWON(aaV{R{xtAu2djOrLW2xZmyn@fG6X zERH}!l%#-7aosrJurMM%dWg^Jz{@)nY}$uDMADf0x*bP*-o`&0`SiKxw0Bdq1z6)c z(f<7Y8|4>-+k)Xzg<QN_rUa4B>pdz&-40Mv+VV3NTJJ6P+skq2RdF>I5!96HOG$OJ zXlim+)Vvu|ixU*q#UsN=#d>a;a|x=A!5C=vk@l#?o0Z+;7b#aRc+qzf@s)Ej2&^$S z@dDR|WMg#l8Vj<}q2NGONi8mh$Y00h;MUf<N_>8huV8M^s@AFZrnvmKMV=2b+^LO? zxArw0%BODyWHOwv4ARe6BtU=#in-bQZyBW%UqNNS^%L_udoNmj(tCa?a!-bi>OXkG zs3W5q*!vfKw2Ciri}MOm?g%StFjSmv9nY)v%s3U%B9KbZS7&=oNLp-f>au&l1<xlx zMC@jNsx-S?A9ggflsgmSRy6DlU<L)({9#s6HNTQ9g;G%+USu0Xd1yI>o2wkVgI`Rg z2uf@C0TN&0Uh3hir>)pNef~3Wc(<FwQn4y?Y5O=6${&1IK~6#837#U#<9FLAjoOW2 zOSXHBxQ&xZ)fBl?YCIwX{$MO+d<rSeWh;Da1PfKn0cy>ENZ}?c6*k`^u5Sz3P;rMS zW}Q%85sr-HN%==wq`z_RkApiuB*^b~dD!q<==2PN$T@k*b>NSoou4yBS8EeUEDtd0 zs7pl;FAr#sRI@vnlvF{Zv(6VY1^4g?v{ITr`k3|O;7YIEUYsbkZdTtYQC`145yMG% zmDVIZb<PFdbuKU=g+lTCM+aNM2qN5uyEM|L7mc$WZmyYOVC1E=XeQrx(@0|ck}HZ& zSRwuu>h%d)OZReP!aSPoBOP?Nz8X}-!8$w{Jo6{JiJVRI)hA2OTKA7H7D|L7`T}h( zv%^~PoK7#SOoq6BbD5F-SnGTG`Sv%+C4oqlqu`t1z?j(CHI!Xrg>*)@{p2ilDnK76 zo%jzFXVXN;<#EX;P%LRsllbEQC8BFe=xbRyEG+dz+Sd({#5nZ~DtP@MdI>T$KVfwJ zBf%`I$YrZgmqXh#5LoH$BACTWYt>IGPIh8B)&RKVl$)_5Hqp{hmi)L&v!2p5C{dOS z6y>Rj1kzmMeWMII%nk<$cF#3K0r*?5PEMJt(iyE_^@;Xbd4wnZZy5(iOUI<pBnE}! z(dDOqr_|QEtqZj$&x=?04D8loRGQu85s8du<$O@?H0GYZ$qU_zCSd17B&9O&=?+B+ zHchsb%W+y}r&epZTr4?SoF3|)y-o{9yA5ppdk=ton0!PmZi2SAzCy;QeoUL7!#p2D zCc?Vz=uF&NM1WJ>oV<K`gIso2YT!Q;djG>O@T$IX=y1$;oo%$wcYB_8hDyzny7NTW zskkrNy-Mjy;oo}vK^AhSSdX05hC;w%63DNMjGp&=d=9@avB5df{~P|Fw67kC))m|= zL7rhDQ3m^>b8s%k$QwlW+mfw%vv%Hu>nV3a_J1x|8dpwR&Bg;T-itBDm!~gD!Eqg| z%63H@sYLCDS=Z62gdS>ayM1vTAxg7gmgC?fHlxm;5;Ukx%XLa74)?`<*&_QMN`!ac zhVPw9DnD({H-}!@Pd9_HdQOqCjDnS<X!L2{O?j+{PCyG3pS<jxQ12Zd;(7wXc8L!6 z2DQYChOiHpEX=A2{$W0OY@+J`5y*o@8ICf<%BGKFD!JPqiD(Zna5>AjKy;Azz&|ik z9ZqKLhfOj8G@wOlWw6)WmY8<GiOXGa5MY^#EPh68xZxIz!KCANWm?~l)W$rxe;FUN zeI2E76kw4Ds6a7cGk#a4@4~2Y%gdeW8=fr52wD4=Zk5p4Jv3aVehq(n*jpy<WycA) z>`~(C)CfI}zmq27Z7EeBcxf%P*vYrIiDosPrsf2y`W<Ja6}U<G)op4TTUXR?YAHV} zS(O^A`S-gGZLSyJj)XT0+J21!yA){`yp+G)9+OnP28b-*G^!j=B^{<U9b#aeT~-zk zgll@bN09p<EhI2s+6K;>*sI4E*zY!2JCBK0RkOb;1cexLQ}SjgLSi@);Z(MwgQ4Gm zbsQR34XkVlh-&iOB;+iKDgWPX`x!(5c@K9ju**#j7;Yyrlqsyba!Vvq4?zuvKc=cY zqUVo$GNomG4xDdSq<KRt$-^FY`29@jWs<&Ab<o33J*ITp-&K-)lXSEE@}J#n7Lt$% zWB(-1NpJnW(=>9TW<6FcKh~P-O>E*CY#1pyxi}zSo~zpTXbCuFG~yFdDatrtc$m^J zAl)p@%|srH)j#ekS^dV4iqWmj=eC}j(lE08KraxS!*0GrPvzc8?p(LHDxtyUb7NzF zvs8bdf%9H3{Qek_&9`I2&wc7-<#bwF93-LXX_-pSjk++wvc0siB(nTd<xMMmz4s3W zJ;w$qF~o8<<Q}n(tC+vCRP>ynR~G2TUeGRUU;a5F0RFbFgZZiCe)AU~7q5s%>kd<k zq_si_e<m)=NjEy`N@78ZM~X4EcurT#N%3MRGY_xJkC7xz0r3G)mbcN=Tx0{`c~@$s znJ^$MB5I&k<0-y1R6gG0bxox2n^mTSTsrs>oZ9j+O!+yH^lA-M@cU9<=f))_sy)7T zc3}8x(Ih+1e(S#V+&w~)GJ2C<B?F~;SQWZ6gTb$ag}FOLW9@9(d_@|mK|UQ#my?=H zud8Emx+QlIL@b}64(7}A@<KD0s`CJmd4}m=8JC_bqr}TA*rD39%9ry2&7Up_7d1qM zY)-~YFH0hGzlXQA#+QSOu_r}l!=P{T+FE-=SO`AnDs*kyW|~XI$61&+*NOe-+8g|O zx2R)s47k~mz0JJ<xM2+m%L`tHysETEf|fzccEm-&d(ovb?3Cr#*!osBS}io}JZ8Rb z;}dV?89`QRJpgobt<l)4L>&AyR5Mjp7mtI~D|;4@`4&)0Pj5}CjYMA8HPyZMq_>eR z@%6NnvyW*djIvgJR`2iczc~8aXtdU0pNqm|S9%t7$>6h0Ft+&=cUfM4?q}2v!KVY8 zTI>@#mF3Q#r}`B|t23jho~?#<_>?5qIldWomi1J+5f#j>PlxD;JZ)`P?iUoywHzCz zwt82%`y^=nZu*G7^4D+D<z5Im;4q~6+T*=&*hR41&%>IJ&3=Joh7MT!H2^Lz9+Mt0 zd3zxXmi*I&-6nq*k^foe_fME?j~YRaUaTNL1W?kiB6U8jR0tlP4~F@w;=WqVyOV<{ za`ziI;z`H%6@pB>sbyOX7%g|U9o0z&gg26-IIm-E(>c>N{GrdhHb`MuD(56JQfVv8 zam^Xh(ssl7nnPv!yv1q(u0@@@;O3x#&$YPVMsl1N@hparqFgSOr?n74TZaPesScae zAJ<a*j(%Er0{9Hh3Lcw!^O}b^UDZN+?;6$5Z<-zgAu8Ql?*iS=3-1ddZ#UbVy|=LZ ze2ww_8L0D-PoEaR12$as`#zk1^4=@8SzZ%+Y+oEEAFZ1Jw0F1HK$?gekkTvdvjc2A zPpyKhYr{JyVAMS&PiSQ^{Ha<H{9FkLN33=`{(X|5#J|VroBjHi*v0SuH|CxEJ_&F% z(GVI;$1;$0(69ZzEcxT4y=8&<heGxae&rm8!-%aEm`ep_eQ203*EcjIZ7Ng@|BME? z$S1(mDc!{`8y3U_-34$8*NQ3x7C%hF&r&_Kke@%9eZ%i!meUz<x^M*@?6yY|#o{Wd zsLVh8plR{<Cpo`(r=Xg6uWiy6tgg|T7II{HUMomzKS8#t3)Zo#&Y^eqW7srVQD3$W zuF%*sp56M`yO3o3UQ=rF!s6w(*Ly1S{sMm9i7e;r8Zv42@})T^*p?8&7XrB@zKdCG zzmuG7AB9u|?1e1Ogm!y9oaw#)$0JP?<q&_;4uryK0FY*j`4GZ3SGsW^I#4_4;&hoK z&M%MGCx8PH1(2$^e^>na?<2)1S^sr@8_;6w^yeR?Ju3q^S2~fxGDO2dKBn%bwcyS& zcoxE(!7)YiK_NbWU#0y?N|{uea-V@~+(r_|;Onu$?~xMK+4Yp-ycI#r@B3X}@m3=* z0$!4q?^HnKi^0LUD8>OVcWe_vWBj8W$b*5PiJ`HV(&%~vt3c2}kty1LWPoWCur(rd z`$;$)s4SIGi!67?hG?&CWTc_rb<N=jftrUxR8PL^q_y4=m9ju2E64R2{8n+~4cD~X z;yW4agK;8N9O^n#-YPW0Ln_!v-V=kb^4?1ef-eQ0$1g)lXA64jwS?ABZ!<#5o$Fu1 zsY712xvCf6K~sr6)yyW%vw&{UNXq?2Z<S@7q!w~zj06ww7M5yl-kksC(ee)N@(5r; zJOSg@A;LeMr9Dgnb^c_DdICPz7&4~%&!Qd=7P4(6t+0k$9HV`wzZ&W->1g^aBNzs$ zU79!ih(WzyZRCWH*5>iDTWPdtoUItBPJ1H&H`tR@oAK|3!Sm6nPkd$uP273ouFwgY zJgHv|1k2%$#Dp9DpMvY=Jcym{ioH~}r_^%zLbE!e$fZewfIqbS^!?s!!P(b&H<t3m z8OXQ$=L=xY6{*j{7&#u#2nG?n<}lbPa!@6Q$jGI<mHc$c*FH;lSMAR&PwUv>D~0JU zS%`F@^_m@#*IQU!mZr7Z)uo!cs9U$#Ui>p1Tvi@(Cvv~p`)(ivnJ1~77I3ju>r`1h zX)nKfB6(N4HFytMTeyGMI&6DSUTbx%(PGFD$VsRF8-rb4V*uX8r|{tfMrb3O;2)vD z@HlSzcOp+f2M`O-0$Gbji_)ET|C$#}J^wx+m-l?SvC<jq=?RxbO!EE8f*<3@8?wb? zc&V7QRF!h<+<_}txEl-Y9G9*KQExdnFMBYYbbo)h)lb;EjUDBK%p0Oq!2aQy26qF* zLB?q;+>pRnS20mgVvtk`Gk<Pbq7BE!0yEsU%p})|*a|DL0sZUqrH>y$an9Jlym$Gt z%P?ydR=z<@`0e3T`+r#sIyEyTu^9JT1}?z3pu+IeL}v)~tnyJf4{NzRvG_T-VrI+b z*mil_!DTvRev|H9N6sU-CbeCK&T6V4jkB}%`#}=8g)T0iUAO!>xUeVWiRSgF;7$Ae z+C@ZlB*S=)=aNf%-6o_cmgF_X8~hTGhkFCY-~J=uf0(vi`D#<k>H6!#WERIrEKcu> zIOxsHyqP)t$}(&Rz=x{=Lv@`zRuMAL=%Un!R2h$c-#Z?%-QgHw+4&L~OecD1RQ~;o zMJH4~<44)aEI0E*f?^?1yXN0Q*6DZsJYPZEEl52EKd8}T6&Ek3qr}rL`VV8dG_8WE zRZ<7Z&H8)GM(9r@EdEv+)jz@jWn5%z&U6=K)&smpp6<i;WgI6xD(c<2K=G5D_zG(k zf}@<{$fp4B3WyC!r3Xa&d0v6#1hV~_q~CK_6E^aowUJpAcPb#-X!4-u8ExFL*%AK` z@)(=m^&Yn5_1K64eh+n4dgK{Wd2jz-_O2Fd0=|l!t(4E@G6E0ZFEuSqAGN(dgP+e) z-=v&EzyXti5M(94cbv7w*KJSp8F2G{b%^%6^!B1j@cqAVNIozBTOlQ4_Wnp@V9hr{ z8ycM;#bo!_b=n%VHHw;Lbe;qBA@)F!XPr7$QRm<PyLA^~r#G*zTmi%G2$q8_ts4Db zlh5|g02=z)yqf&H^5bq@|9RS#Ax=_GmO({4*nPo6DlC|Xw!bY8mxl_`tWRQz>iCDB z8#CBFNrf^WY0@n+4()d{Mw|Oa2fj+8S~T>vv$M*>k5ZKx?rcmDHHUf2f#&LfB_EI) z!jYGG4;*PTz^~{WO2GqQyGQ>zYZXSmnqXTd<QfRP(-*e+9u@RlY#eY<{@f?>O7eOt z@&@V&+@~_in=xVAPEf5;`8xazWjBEng!!bWvg!O#n$)#yIByaphOq@{<o14Q3>$e7 zciZkNe59*Q<@IR(d^n;0SkkCJ+w?6T$oe@>7J2FE5P27<efeuu>-4Trd-f<X>2e!{ zY<Cyb@^YWN0I59WI*#nMU4nILcAw+?C%}sK#P=d3k1A8nJ0a&M!F~i7KoV0+v!|Q= zDS<KgVbUh^e>UUAPoKggQq(pB0d<S>U|SIq!dBw+{GAY;^Oj;rqt>?vgMuBxZobh; zgSa+UK$+qb^q<GzFy-q|8?w9H_25>*lt-GaPX6zijChQ@Yrxdc!K3^GTzV2L-ZR&P z=tWDp9;<Z#GsLjw9Nd+|fH_Sa5?~S7oD<z#!~Ruui-M=gsDdRj{RQ0Jb#5cf7RPEW zlBEyab^LmB=3}TjRW;j+Bx0fBNDmu3&$jgJn`4*zvHCZ;_Zx5s=>0JxWa)2jR}sUj z|MA<;qPT29!^WP1#vnkOU7>AqJRF_ZygN-&v3=V+dnGNh_*~bkx>ACJ$#Wk>6)1ES zYU6o>$z-=AIsJYW;D2i)qS~7gQaYp*Qtu-C*3SQW4Qqe1etA4^@+wIY43Tkq-W$=s zE>Gs%3t!UbJa(|z&IYO%`1b!a0o-nzcQ)*7`XRp_#bSEio=?{rEuyuS|0xNnuULVl z+-M*gbs2C(8-_+jeb$%Gbm(1-^>>rpzoo5Ihio*+W}wQ~(Y6;%nEA6Y>U>(Ftc+Lj z{JtH#jPYAXDpeUE{CI|?_*5vwuQxUgb?|Uh><)*nnKSN1V5E<xL7=;A7e2w7h)=!3 z%=+=kB?739dWGgJw;AqTaB}ajEG`r-srJ1LN#zWD^IOZ(re<F!9KI;YqT>-_OU?qZ zsA?MJifYNsc?%DFZCsBjI4uq59{#75m&eiDPuu>GLy?8IbdnSQQ)YzMrnU<t{0i3s z+{8W!e$ztpo-@#IY$4CLTJP+;35Ju$iD|C8*Mj4PukV_73&U{*8#>qXJ+qVW8l2}K zAnvL+*rfgq0$$v04_W>qB3$Y`{i09uez(1NL({tyv;86$@U~$%V&YKS9ujCWAnaIK zsQ1GtHCE|Xy_Zzc2S{wF`x6NKKBdIYKL3ZB36Q4k=F*gf1%Si4>J%81Uf$2TVKue% zdM+SSLWhZv3nDNe9QM5|xezW_DY&Srqd%`|k{@S``_A#-caJ1`Y004oC@HE6T3J@J zUlFAL6@S5HvyoyeGN017zXvbz)NU%dv?wrXM!dTu@I6>xw&<AGwjc1k9}E3TNCw*c z5MV<4nwC~tQ+!@Z_^~$i!XsOs(huE&R&OL_G#(C`N}++ZCq>UC3k%$z*h`q%R5LMm z4&D*&l%s+wPD>5p7)8e*Xt(0*^MAI|YD+!M(snl(%JjD}j&vHs!iv`~WSW+guJ+#6 zxvpfyx}|5{^p9Fd^wFn`ojuHcykILrhnH&8T*VW!rL$EkfiT_10X&>kYzF}Q7nYPj z9otpL#KA=;wqPLzfWs=O5hRp4NT;Wvp7N<1UJp3#P9Nci_3!&)&4PLNu#)^UXmhWU zhAYWHZoxOchDpg1^N((}_=MpaQ{R5`G(>|lOa?l^IJ608$?+V5vz!)$b++qk_&m^k zuP}uWmp4RZB2Kck798t)*1o$-#cwmKaCn=H4P^x!tyxdz--<A(b$Q9$25mK+R*R|c z2OH*EcivN$Esc$dv6&5jCrc)@V)0%TjRS$rk=ag}KnR6$F7K?f^&zwvnlL6NNb$Gm zC8mE_GY*;j>3)Y0YHBA#WLnRP|Gad3>19`SRX&Z~&j-!+5A(GT>nri}ZnL2t+tqVS z5$S5_u32qQC}G%!;TmgLRO|zsyLhFsiabh%O{Hm@(&wWx@;{rPkT}ljmIi5<a=kg~ zX~BT3HLX{LsWbe?vawTad+=q}=W!-AjkLriLt>mZI=O_1bWtASnzbf8XoYms{Wz>_ zI3{WJ7s)|MXwsQc%ugNOgY_X#^+!)bG;nMuNsKH6sZgEds~br9o4pdvRw0<ue!I4g znE3q^vpd9*{?W9AK15+D-$1U;XS#;@yu>;!%k!bLy5@i3l2P(@R!To%LuKx4+@Tp} z^Wo8?l#}sL{s&Xw6|s29Zz_HmtXm5|C2yg^$m2RF|B0q%DdGrU!shQoql}^mgCatr zxLF@N*7J<<^V|arjRG@Z7p%EMV_R!2WcU=+h=GBj6}1T%&#M(J*{dnx!mcg~bC@l1 z03mzzXWSMX+6ipO_V|w)EZw9{TV?mzD>2^`Txd@z?Yf?j6k0bm-10r^OI3|D19TRl ztQ6^LOKR}={a0_Bm@rE)>oy8{QV7iWUfI7dZTHU(U#1Z-ZaVU{S~ZcrwchPWNd7gE z$!Nrx^$-V9OXsk*-6U1zJk!%!GoXlQa>?I=-<|NK1TBoRO5_fM8R0}=lb}yk@<mVX zB#}^%C4Ik&M9D;vdw#_;H$rCZe=}I|fe`gPPu8Q`t`K-^<f;<l#$eS2#}JuDx_P(i zmc%aQrHm9aTN9qhpp3NtDze4p#HBeWcU^FTQ<h2?jLg*KOeBcr_%_hE;EpEP`+|*< z*DIM`fIOubKb&=ZO-sOqKa0(M^iusU{)i*=b$tLOO*)^JU?*wAxK(#{8GXEkyS^Mt zI(QlN?SMNog&hL|zQnPRCy>tgazFCk7&F@nXXFV2fwWw&0Fy{6Vm;~q&zjWg{O|#j zSS@59zO1Hwm7KZSF{qr={<O37B>nReo{+~qU2C3yMD9svjJ_{&=&FtuJLshrteJ&n zJy&X#-k7n6;9{;n%EVh@tk%CI@B^LzRY>)8mZw25=)v$SJDIwR(|guV%tc!Qk5zH8 zPM_Dg2v|rOYs1&<eY+J|zOxv94>tQ}$h|l8u9s>{QVZL5U9A=_-qFXAk3weaZ46;G z$gBgMBsXEmtW!N0jJ>Z=;T=9Q>E%3e|G`+~t+O&VBuZAvup-DHjZFRZCCubTT%;)5 zU0HFn#T7(26avkf5`<Fd3!Jb)A*+FAIIS6#QHhfxTf=vr7u#@GN-^C3Gjye=!w#TU zk^G(mizNL1spEa!h3!ef!qVr*P#AG*=JEi7!t)A6TN@L#&v9}?eP@5KfzKmfC%ppm zCU0aba{VF+hJ7rqQ@m>69KAdnKM*dFE8wFXf-Z&zCHAQ2xGYINWVY;vV}hpZGEZxt zRbjimio-<S+;8pDj+iodsI4+w$-Xsjv!#XWP=H5esNVk_K27%_@;{dnYFs<4(yKtc ziJzT;heBG+){=c*3weGoDOB2z1RXEzY~SDBdYwRv8Kri#UC_ISqSZ%P&l<n;zeVWk zDEK~{BlW%+eciQ48dFQs#WM>v!`|Fk1JN`Ik*od1ioU;qr+p|0O{Cyx;9^6LK4khe z>c_W|^EL9OD2tbu8SkDg#Z@>EiJ2F*Pj1|muq9r!+k_JzBm)u_#DDUhf;(D5BzuK5 z&EtO9K<6QbXRc#?>%e$>eTUe1LWW#et1L_`18rjOoUvohEx-I`2{6^f`IK-n3p?<9 zC;HRP1qmKiN=r+tGkd`y+4bfDFeHtcrl+S<n>W|?7v2Lt<FpxDU{tKqPWgF?5JuW8 zc;=$Va{hNT<$SEA#`pb`E~SZfmfbz{(t5Lt1xp2gq32|`*E=j0##-*A3`sxTecWfz zw`^R?L{)0y?6xOn_OZ)~&veb+%uB(SDXS$l88y34^lb8ov?<u_lRd?MxT}*z)Xi_T zShdskEv?hHw2O5^fD)aJQ>bR%OorZd+IdXU!UTku(FuA#&`xZVDR}}CTG^yuNEI}b zP$az)MAQvyS?4xjMq77jZ<R*;+g52&$NTCY#dCi%^&NLD@*1et#LlLz;AO<$kom3f zIHM>h<s^B`Vz`bv;NsU%xDC=HL0}?xRYfsT9;HmvE18O)vrYeq^3)I!f}G|*1qlSl zP6uBzcs?z<nBI<(?A$<<V=fk8F*@-qJb5G@eU776-l8%EW<Go5PNH+oK+%oHu4w2) z1H{%|-z!AES$4yAJ)8~j3lK}$+td5spHTff>yPLh<)tMh70)0=(1fo#dPx{59Fu~V z7Z<f+<`+QK-xl8hhk{IZMf=<_!qqC9g~6Tfea!z>{5hN7Y~|&fmn=ru=ujJL&lOA@ zSV+~au&u@We2c@d8zLB$8ph?ksI=cYjt`m`9+y;jX4OOESh3dG)Y8_nzP1WHe$YtP zGOQ3*X-T$iFX?P>kWLwUsqTGuS*PvS`))l_DZ>KiD)vanTetV<3~4==&E&^wEwgq# zcKVV<Of0*o?_5Y?Ka4c6R)LoA!&^Me3sd{2b>VL&)ErnGQxkV|Wky<=pkQca7Uq#G zsvK8JCnkFfY!JIY-9yr3-}Q=TE)lz2n!8;??4a`xQB8eF@fg(y0-_+a)ozmi0&>Wg zhCgXE_reA~A@jdKfLUL=FvvUc%*;g;B0MkS8;pb4tiRCBox_;NMssLCwHMMZ;CTer z>>k>ND=Ot;vKYix6@NZkQ%@`#&sf*;zIphM25%}~C?2p`6asT2x%4`_;{K#N%huLb z)dSI|e!U}IpkLVXWex=b$^W#>%+5H_Yjl5Up|VQTS4|DbQ|js(AE=oMu21u6h&$dN zljUDNryH6X&04G-Virsm#jLMi=~8htWe4L4{bBAGeEJ^8P^w$%yHp!Ju0A-&<a0Kb zBmV)M>8jTLP?owe{4Jj44OP$Dx|{k-IM?h7Q=Qeucdx{e2`3l9#~qr@2<Rd{Ogq#I zkze^8LbA$=QQ<8@{vl6Hfk=w1%QQ&wVbSJZ3$&@oK$v+ioMH=cg6FtpjhkN7*zcB= z9Mvd`+iF+#5C-J~?Y+D2`nkU=#f;49T9FJQPtdDXd93MBl5)fM%rItm_;F<z!yD@g z7?v_SiWasAJQB}LpN85RE;mq#yC=;y+j%+yZ}`sm!GpLSGK@D!BOPC8F(0AL^uCBJ zMZlPPcs5YGOEd;Jl5!(?iG`I@ZJeF6+Wlr#d_eAt5ZlT`=ObLS+W+bMCU~*cT|HcX zW+Igt{Nnmwq};o^hsP`sEr$+r!rN-dWg7qeTU1o^OH~L>WfL&+>0|3l0zPClP0a=7 zXPu7f{H_~G40oQZCxyYqX(vpZX2+@RK-ltnVZYk%G?<w=NrIZrJqbcq?38i()NJ}~ zU}1ch*LKe_Bc%oQt+aI2bPqEaV-_dA`gWyE?LY#V9AJdjHZ`?*wa^+!9Zj<BTiiY@ zSf|a!aqecgmyTSP&iHn>P^lrC^DXX!jfw6(Ym`&Q8?N(3sADfKBYdKS*<VUVRH9_b z&b~}DVdVB!1y%e8bT5r6S66GMB<qU>sAFcr@wLblM6hYd3AZEh4|pnc0#y$4Pu>ZJ zu~k=kD}UGsTb6=#H+|5Cz1Su7T^JbIyuzk4jILH$-aZipzs6FF3-lN6@xZZi%0$Va zaZ4sk7#WhID9ZiXeH`WeOcHGY-^iOq82*e$(HNkPI~P>DYi0jI#X(>&bIn3sg+uY! zJc2HYTb2kfeo+Cjp|bct0iQEXK2_g|^P&Q$Q&whQ!2K;NFaMcsZOM!^KtkdZki4}x zAq1<?oC)-MIHpe|6}e}74gp^-et3eA$9f)=2WbiRRE^l{Nb3)nr{2=k8Tfg*YhC<{ z?Y}TrbX0HHVZ=H2Sxk9+8%63;{4zLQp0S~==W8csC70Q61Wx<PBy=UJr(?&pD7HHp zA#b6*A=-A0lcu_~y_{|vb)6VOFF^qpg51HY{&iOUx9@!z6*QbTvUsWz?#pxg!1^!v z(J70s1YZ-neo-=_C`<Qjb1HJ@g(3Z*(mig4=c4d2H@PuBQ<@vJ851VV^!gAzhx%vd zEPRFEj7A=2xsbL7A#FBImn1ux?^85IQN2)EAkT-l0<YmOCLsvZEIm?3I-A!+y>Fl2 zuczOm>Vt3bYI@JZzhOJ;Vza}nk0=ey+Igso@E{`d@$qK#emJ~2*fms!nX)e3nW^FZ z8yV1FJMD9HcTD4Yyaa0}#Pn@*9RI2N{Hu*=GSpsZ<^^#~YN2ci5>e!n-SJ9$P&9i@ zbNS;-D;y0btt=2W1$=ddekD~mHGNK@R#GocEj6Lk$RdRk9~Z@8U-CQv{siV~2Ck1h zt_Nv2l3EH}afa5%R;?TR0++qos~Gyibi{h~Da_r9aGX6?WmOc<d9BRjb{Sg#mAqYN zTuo*oj-_R!&k(G2kchm3>{X7YAfVa(qSiW!L>0|k`r2BPE()?SX`bDqp=GTyV=-T> zD$jUoalPjVHlKZ~eOxYH7&2^hS@*0cva;~q#StRC9uH2o?AWKCO5L4|D8Wm|l1ygl zIwPvMPhcPu0R1f27qgR^XyD>;uR?)0Nl1>Sf#Di=Oy*2VZIHPJc|D5iV>N3ZJrJ-z zX?8uDAs-R^pz<EI{WjmblVkD<6BM$JE+l*oRqgUj-v1kq!zjwsX;AYFHzn#d#b!@e z9{c2wWpK50+^wep4M*F)9{Pz_SOlwD8g<wd(L(oiR{{6!u>9)rh2Zaf>rlV2x9-;R z$5Z6IZ;SKzh!7OXo}y24Fi4U$-H6tSIWfo%%ew&DO)e|q9?W=2Zr<E2>a%Gzo?dAw z4Tv>$>QFE%fy>Z89Ueqyk+r5!Ag?pWhvkz>P}5UkKPt^YDMhOA(^2?Y7JhPB=}JP) zuj3e)<GU*(Y<uZ4(7JZ;sq1?p(azKG90L?>HO*7A0`9xta4p8R=f}o;m4}B<E=?4; zsoC5MHY19=okTvSUp=rcT>xH*F+>ZGed~#QF4&${vKWHQFuyGYbSG|@{O+JeT34#3 zpxbXJ`AME_NS5u`q?<K_<C`djj9tWB6a)=iWnvhm$Sd3zjIASiLrL(5svH;1)YM?} zA`>ch!wj<sJwnAd(*&20O>~pDKR%DoI0d`&Ax}`f@5?GJ(>+o5I>WeT_z|gMpax1; z`mv}oJ8@17$RBn$*Be7Zx!mZzA5Cub4rI!QEWlVLNH{(rx?iHJ=-MD<x}1;urbII6 zL;NLn`);6k%e(5$_8YZi8*YHPUP0)J^`FEb?6(TGTj{!4Qp%Nf3adff8V1ACvGADj zixGGsp5w}Y3SxB7c`RT8FbDj|4J|ENt6qAAMos@_&=)~o++sAVkTqf$=|r9ay0+zd z{fd%t6NiZf1u=fBbEe$(XG!?E<xo7&WaAg&J?*V8Zv9kR9ai@j5Z>Y-X~1(bQC%gd zU_Q%4Rd+ANe44F}tNzQIc)bZe-gt?&D{C>sd9>aqLOdYi70Jj$uZe|%=@W>2nEaP5 z?}1<K2>JUYt~Ecjaksb{nr3KwK6K_uW>V#+`ET~UpY6)l8^<6>^u=|i;|<iB$jfVR z+~=3yGFp~m5@(ThcYfeeOsbo1uVQoya|{BJ=}1Jui|F0KHXk~Z!gn58NCbPYWq$ee z;jX&yvgegqc+D6(1@WyA<s=8~dAOO1W1}dI4eru;xN@3tCn*(#?{V1dUEN`7CdErt zl@s)L-b>4)a!?7jW%*m|ec(wYXt(GtLZokACMpxAJ->_trHk>p;Do}gyDsQGnjL?z zQEYV|zRPr50gRccAy06-KW+fCLRA@HR<@8zXr=j=K5OVHAug_o7uM0i@3xEutWsII zd>$~s^JY~lPN#Jp6A%778({g3W5g<-#iJ&NbWeB>p3S~W1{Gt5y_z~tM2DgqpXQpA z@1Km?Uy%r!r|s@*1aGeVhPSM%1Fv357H`bjms24%Z*N`IOF|dLaRj9bO$oJ;8Db7| zy93c|)YeFXFn$B`j7SzWhsC76z0{&$ied6kA7R*Zk4)k<mltDN<Oks9F%?@4qdcqm zYpxvz%lg_H_?}zst-^B}1dmwNVQPoa_9!LshbV^Gl^~mp=UnI@MrsEp6t`B5Z+}vg z9Frq;Pr8wyw-g!|9MH8N83&_gN@)>I66R(M9pSQ)<6;!gh*L`8=lpmD?NE{wl}Bdj zI2uFL$(K-LrMtpEg>^oNTdOcAMLh2$DB&`%NYOX?%~3-|b6)%DI?&WeES&0{-%3Md z4V#kZ(&K*#$A4fvyNrO6VX-W?rd1obB@YywNjU<}d`xtyQ~?JleB_Q(cWQE0Rx~?1 zyAyM?f&ekJh_>>wGHI$93!_6ItMoKvv&=MbKk5}V%}UKgv3p#4yh!5YHbS}v{M4Z= z1SWp{GTxS7sbj=#)T91o>*ss8|6-qy=jEoy16y&1p$?VF9eB@(y949L!o+b|Ig746 zbf~K5`Q{JUH{oE`V`t=gVT!N2YN1$UBMsz_P@iGyeyQ$ZPI%Y<LOLWz;>e3F?7?82 zqeZOrHfS7#;9^6FB%=(VKNF@qkb4q+&(k(*-Y4{Zh>Oy>v`zRi`s=^#&zLgjVyiR! zwdbOKo=o&nS{CzH@{Yu6p;h0kD)5cNve8c)Dazm$2eb=08A(q6I|Na3Vyw$dj^w-D zHu$_ao)HUR7)UD*_y2NDXlw`#In2jhAN>H~Qu9;B$i~bPS)`R_u@6ftzRz`n^3HrB zpZ|@X5=O4cZANGC=+?K^2!yogL^T5M_Ql{jxy<B+V$YS7P-2N>r=)}fPOzBHI-t87 z^hetR57yZSZl||<gYlNm{R*2FPU0(lR&9x_xYrAzbs9S^-VnW{O`u7fFf#)ej|F<v z&(u=6ae|tEvQu0p%&pUxr84(FL#~-~V$^oa3voB=V`p7^O?kn$eX{YIj3^t)`(!d| zxcbS8$@GJIhh}lF>LtAO5~*(Nh8<FU<%hh3nSaT*yrHRAnTJ*9@vgXH#F|$XDS4S? zq5}91e6>w`z|@RL6-!HPL=NQM=0<<HF{QLYwDufJsC6^?bn;P{AVz;7Ze)YxDs-s| zq81n~khcA0MxncI-NUA#BQ84zRd`B^m*QaY7j#*-8>!|ZK5+rCn4ygXCILSKS=A5U zo+>W;^W4@6yJ*>5SGXZHgOG)FfZ<~1@a(6QQ90Y{`Q+ogjjgkp3>T+1EL2`zJ}@#O zGiVzS8T_E8rWO+eJTd}8C6=9o`d6}{*N+@FQ-c6puny#<uBMDpB%@{CUyOBhRn79R zxyVyQF|{XES)ghe+A1K6&?&NP_{K>YT=ttSVWshbgGfk*9wM}O+#a%l)7+owcH6$# z#62KMQoZfV{%V*26sH<mAa1pU3W)Ry&KikcI$>XM!qqtW>Z69)TMMjWk7uX@xCriy zAX*LEqj|gbm*-*2?<-oFx@~2~`QTpqGpHr$++Q6rpzIGq5%+2kF0po40+>t5<F8ID zyv_IoARdsTE+2hVMzJ9V$ZED)mEW_HTKKjxSQcAbs*H@qlwI8rhGLcs*P?a<eQWyj z*7k$=U0cw%wM{|-BqP^;`oGY-Pfgy&7{$><<PGag2rc=c75KK{`W`-3T1>~6ad_Mp zv__pM%h}n36*uU8a#+?9fX$lXU@DIR7(x1fQtL|r<~%ka$qoyR1ul%Vi;uDK7ycBW zED{R}>V8{xLeNJ=&3h-NNZ<0kOqbxcZn3gUm#Z#V{%g6kp1yOEXQSH!5q|kmS3Dme zJ5jRTgL^IL;kq^~@DrkHE**(8o90znS0ni~1s29Yl58?4i<&ao+-^p!20<v&Yxl6> zcZkS$LLIV2FADO+@20b(pRl)oS~QGuDmI(MtP-c`*1cK>{gq0UG@6Szuu`(*Hrs0$ z*TwNdVCa7^lXGRlhSoM$Q^iY<Of6#VB0Pn0=^x%TXy7YD(6ZnLdHSNKM;?&R2jH^{ z?y-#vp=iJ&52$jm{PbdrP&w?mf|T)u^<KonZ{wS5nb4fE-wxU3%-~2KR_nk1l1&Ye zG*cRc!$d|M=HJf$&3Ts0Zh@1E>ME1oLX$o<Exq@Icut+;XYgg2J*A?__zXyK2ABmK z-Y;X*(=vcciyksZ$)Ny@bS_9VVBR91>pw%>msaBOL>M2K>bhZbQpZ_hs=%|fVOeB8 zCmjYA{10%Q$|BO6S#2&aL48R9a1%XM-{Fp3E`vDrrLtIK;nSdYtz18ZysmLEE)6>+ zy-uQ={Ylnrb7Z{QtQ(}=la426HVqn-ngFlG!E^iJ+J7&GpOXYyqjb+ZaSgmkII^0K zs_-5ESaH35|1GZjOB=;_Jwki-he?E$Fwe{T%nR)=l81zpKP}0IgkbT65!pmc;SXRB zC?;LFt)Hg)DGzxga-jM_K%Myun3jZMNQ3cWj_Ry&-&#JF1=)nv5Np$(omQ~#v5TQ^ zI^RPdR^w*~S&}3@U69Ep;wPPbLzlh1h-7>9x3CKd(8TvE{?(7cVZP`3{0J{+eurLk zsLllJCIfnu)>amC$kr>R?YVlRl3#&otV_6Kj@dCG(#&D}T7j|N{pJQk3%gWi1Cm^8 z2^wdHw$t6>LNaid(6irem160sGdIoR%fOa@b?QES09F!v$aw3v#%TJ#<8xe^nfYR; zfnl~iLbxm4BkP_W$_X~d;^~ldQ22yu{_S0uQlM5kX4i61-^wgWhDsZXWMH^YiI%`! z38Usv70n&3U2Of^iCzLcN_$MpxMbI@d5Le$Y=kY@cOzP99P&{r6^H<iMxG7DurtDc z2?M6vn<y9AAMTH>KPD>&?JgD({NX1c1C?{#Jg<O${X)0;w9z=SVbZ>*9>IP+lLM!g zhbv@D5avmMn&9y;<cj0-h<ucI)brX{(M!?l@Sjh37S12RPYpJw+#3#iSrh9Bfy&(6 z+<yQ$B(L8Mr3kQMZ<%>-uC3|&`udWxv7rNs!4!9ve?<sK(%^Hkp_WB-?A5<g*NRXJ zLhXGAHT{9M`tc&S2aE1zlGIbkm?hM5&apd_!x=uB$g0!@bz7o9kCHYT&Yu+cUYzYb z(#%?(v8l$TiU-R8#}XbiF-swX&f&;u_{?t}BZphYuRMU1vH$yiJ(6(yd<}9R(>NjN z`|c}=Bs<8M@fk}&AAd0dSuLZH`h^qwanYXKgO6H(X^?YSoDxYt3aZ>H^SG5bOW5G+ zzCohnHa$?-5q?`#vffit%}<g)AFjgGU_WKP^>BpsvWo`lcubsH#SdkFC>>|y*dT6z z`-MJEy09KmRns={1yV-d>?dG`5tUYU5yRsx`Zbi!P{3#8=a#ju!`~_=1~e{fVcBLc zsi!z`PJpHg8H><{+w{Pj>zAwo!hZK<0MJh0Y@^P&I}Y&HlkxKEwx1h!de<sVpCAGV zHK*sIRo-k(4I|(^?U`IApjy5Z8klnev(lc&HC-Gd)}qb|lU#Qlht%~z8n1<hXuvtx za^Z3$(u${lB(s`hS`2bBve2P|j-oosol*OP26k;Qzv!M`XL8dLjVk;7zS~IP%TJ5p z&VjzI-(+p@ApK5=*hv1!I~Kj6TG##*>?P~59dle){q3yj^Sq(9S)ETQwV*@GrMHFM zj#&*7Uuge~hQ2`2>%)h+&87ZnQv5O#Kmu95x%1?wvmb0QGxE_OK}c(I8l>IFTN^uk z>z)1h5_|U3Wo@dluEL6xJJXkZdj@?K=7E;oTPCk{b5ENr`@=)ue+)jD))NDx7UU+v zFu0#_iHH&br3|BSS0+g;E>^yeE906lz=;5_g~Cu#QQ1eC05GSo5bwG|8%QpNx>9@# z$k(#RAw!)I%`x!8NvhPD+5M!8ozqqy+??7JRGJY{vG@D-#ADd{#wE^1Uz(}hNc&#W zgvxYV3zXxA>J$>i+*9{Q+s+O<q8Dbwe%-#UZ+#Y3?Eu8p$cO47q+?d5P*jD*zi8G@ zzij?eiV>APp(@_Yc|sr8KP`e&>|84Awf@C#Yf&UUoG)t8tXvJ#jd<uAP8u}Vyagil z+5yEFE&Nt@)WD5{&E1=m4)043;&6~f4A(2n?pJD*(-6!87mItxl9q*wdntz6YR`Ki zD~zf&ymO)2I?P`N34-4oPhz<A+HEUDYX8+!RMQ2CP#j=(qhMx6iAEv%GeRyMDe?pG zS|^Z*5Cf4Ynx!U#MmlIvyZ1X1>1L=v1eZpNKT^ogAV}+*xf9BXQ#yH;jvuV{ys)A; zMV#tz^7B+@Tt(zEo2~;H-?C}Cr(XNcjmP_wj#gM<#mlMdOaw>d&X>xJRMOJ4yP!BV z=lFhJUI@NCs^Bk-^a^w)`WGmb0zoMyFXRoUAl5m4bi2C9-{9IlN3gr%43uTfV&LN) z9`d#@o9<^<)%F8y8Dnv=88b_NCpD(+UKHri=!TULmV=)<rs=o*PC`<dr)o7VEwZ@0 zIATFCpzX`Vt4;tl?s;1dk-5UzHWNd2NQ2MlS2bZ&|H@8%S3~`o4oyWb`fXIbibeYg z;(rd2!2Wo0AJ&@i_H114){R+_R<@P-cPEx;KkcX2ZP^<ghCUwu-g1Q+NL5?AtVu#* zuwsUgnnl0KGL|0P|EOji?TE?&`G+6&fwx2IKle2r*!uakBZS(b3i`gtU|nW5TwwdF zIBIH(1buv9HaKj=+2v}3$I#i<bdo-HV%+dYmB2QNE`RU~o)l7KX-{Mucd1y8C^m4F za}odfxqLbG>-iP&knQ10{FqQCWX1P33r=mf#g|#St$%U}hoa^?+q#nw_S3m!*@(3w z(NDugROAbukDwMEY6kk*2wEd&I?+RaC+R2yyn_H|={gmz13%|qmYwZuO&V-V{#6eU z%#x0YVaM7;d{%sJBF0(Bwg739g-^{XkL*y`9DXGW!YZw#if(j8gp}5B{>sWWa!x|# zuiq-|SNV;|ka#Op0~oXM52%hD55sQf0dd-|UOiQ65&59f1u}$@ykgZoVO(g86xWLt z_{DdGS^^_Ka%z?R+iRIbGIPyGjxO4V%kBC<GDe1msimoPHsrDZ79NcMkd}LIbw)eh z)r@obnVPc$as;bM5)vV+wBbbr4*oG0Ob{S86f%b|f6eE;uY~E?TOT`}1NXb5N|L0u zbZMT&BIY5x<GOXMP@w0ES{lnR+V5r`d|<xew&15*Za92)a|j=Ai-i@j9I9Pf%hlrg z<5ny}j&Lq-*{pKgnel3^3Vt3TfxQ4E->&GRAKgT@thK2|%{}JIBGEK2uh={({oO+N zAfR@rsnDGferB-d9RL*v&3pHC{~CQ+O}a&2WkC2O!Xy6c&`8bls0NBcs-%Z?p~a(h zcg1Blx#^>`?!q_<-sS{bTPL*k9bU}bTiuG)Np*VB*AlC{pH>Z@K@QKnMN5MQ@Y-o- zM^x8kFK1m3{##}Z_T^{&ZoiZ<dYmO1C@hiesmluM95I|tI+-mNwUMM~X%J~8pYo@N ztxJ6+6A(h+Q=Dg{GD3%d_l@a0na44K!5U5knWQOjr?o8<9s0NyR>ah6Gsx;5bXwhm z7NC>3Zmzvua^Hs5c0#Q#?+kKXE*P>sDC@-Ky^R~|InADp$SQ4PvpQF@A`99!Z@x^G z)*|uRULElzbt~>C_|*rQA`;l$J41c%QIjS52{j(nO1=#GyqeD?>#7^-3$e<q@3uj* zO`7(m<gXjlJ$=eXGx~zmw)p#&Vc?|Ue1+GNM4kd%Nb5<E4#^-}cY-ckwKR9VWNUcu z#}6fXCa=$^b#HSDjFolVqSaK<k2bAu7pY3@ZV<yLtbXPvFE8ghur227%}A``knbDS zo#j|ETd_L}9}+riC<cqB;&xQ9Gttyaa9yt|#?1Y(Cx{D5Lg(EP`Y#|zrg(6PB>pVO zq~8kr!-p>)<RpQ|EO>dbwX)iI+r40&jk}|0%(0vnRbz++ypg!LxSmbW(Lw$3tQVCX z9rzzVegs0swfcttU%kc!fT^dbr^lFz%bsO40q)e2xvCo$ysC7`G{9^WPE+3=38)*! zxP1Cga)70!XG~pzE!!YSC7`>^zW8~voVOcPQ=YQiB{EKHlz~UXufD$Z=8^1n=?3{p z9FZRGb_|P3JlN1+26sddqqp><ams@NVPn}M>6X8w)A5&Qc}w<DjjESaN*)5+usio3 z0Jz>a3cXhwQ6qHwTrGh@KHd4*4key#gj>g8I#7{4FLEZIjTO?(1rxM}XlKNfCQ#dR z^-@{(BY4iKZEZy(c1}<!p^nw{mstD<EFFIUYL#IMQ(aqz9uHWvFhf0u2&0TSfitaG zmGvVlo=ZHDnc3Z?s#ZxSTpy&WWYEeokp`iF=(S|Rs6|8^(C!4DmvPB|lBbc+W`|RG zvdYTZa`DDDy`S0WyWm(bmwqeA%8E)#LN)#Pkv6UufTsu#z@Ub8hV5a0{+MHh^x7}d zr%l9!m$jlWmDptGlm56tA+$D9^WwaZ)KlPDLN1j`#a7X`lEK>eWv+xakDMVFZ%d`5 zA7ao=QXV@qSZr{W<%1=`wJU#Z|K0aEFHS8F4WN4Bn!-`9j$$bpKW`Bx^(hw!C)%Za zN4GypUd5ho81*3H_O0nNgcwq;9hjEr(GRcxC0+J`9oC!d!(8{%-D&wqP^?@?d&ZmY zzhK_?Yf-l4x9H0kx!<fRLxefdnN}IeW}iNrUT2{Xhd4wveY{{%(!T3RT+-ARPu7D5 z4X+@MbFt|!Z6nKYHjdusx4@5x{rIqO2bOH?Bds~#i;<V`Ipcm964nSg<G39`!63oB z^ZNBDe1L1_7|!b<qsY@N5fzIummByEi{oS35CLUw4c34SalKyL3=l(J`8Sy5$pjdT zjFC6JfR|;G%m$*0nwr!%uYg|AuQKo#2{2}a_xAPzU$Y3;bE>}K@XL^2p|{%%ZL0v8 zvgKE{#%#gTFK}BWZ;V+EKlA9-_SG;Du`m14qK0;eHj;kRy&b8xRT@5H=lg8QAcI3t zPc^R264(A)yWRR&b$fi7!1o4OUjg2jLrR^uF2e|vE3vXCOUGNo(As<ejbtTp8&27s zWx{-evG}&DAE*1n(z1h5T5iQVHR^oCJwr*?HZ8jJfKJyeE2h-jL+waAVsVR0w=#A) zHB@2ezJ<sgg48d3`vnK;V4ahirkHfIBzAofMtQ~Q04ykcC*!jT!tPsfIEf}27G5#? z8{SPFEMG$02-j~GFS?NJZ($W;!-%(&phbOA!Fey<(#{gc*!4|_7R~Q^^r!n%)6e5p z^~VX27AG$+u9%pZE)e_iMlLy6ab>UTz$qeP604>!2Ru;*X20zK^i74Zkm-L(`<I{N z)ciA=aoEO22J@mt{;{sWX86fu#L(BqJh^tS0`Z8gNNo|x%CX{gV!Q#M&|$KFBTcqP z<yLjlx1%$)D4GM8W`jnbA;7+O58^|WDUj6lA^z`pX8SI861Q;Q51fSm!mj*XUxvDf zOHbv|tS=JztSuS_m{1{I2&%ieiI{thxOZtnj6LpRT}CKn`c!76)AEie)kFNr?lp}Q z(pggERgUFF!&Im_K|Dq5zAW?-$z~Q;YL<NUVxBrUn8;V^&aKb=H%{w(N?$K{z%rFZ z@$$NoIEKN>V*bV$WysvS6>SJw7wzTEI&qr%%2{_YY}GQHYyrR7WxH=&T7$4Z>N@|4 zTMbp?4X7ec&d-2D*T-v4V6P+*Vg&NM7Co-0VsNqE_%RDHOL7-!X=z0@H0H+@&w!#7 z=fFJ?WstVBqpPZ_TFYa=+w{5=d83d=6^=N}rPd0TGO6fJKFIWk2BmL)F$>&7-0n`6 z9%nU*0CNW>N2^$}>pc23n5=eIqcrwNu#UUa(!(f^m?ayYd6|46@3c&~*hL01E_Svl zFrSq1D&Pzg-3#t@Ez-hku{GRGP1#X?9u`m46R!W)W)c7Ho%*z7TDuNoX;5Zvt)iou z^ZY8smtL@rD<l6=-}xx+{xw&tiTw5QcnQ)a^bWB)ee{SQq{~rF#SJj+QQ^f;D)<G0 zMZHr=n~~q4bhNOb0IXw0`JoA9iyV%?oea(f25?e3I(TDa<CO-p@VFp*m2mq1<B6|- zKRJE9voO2#hTl}j>m6lmB@zGnkn4(WEtjGBg!6j2Wd$lt(_UV|;%NY%S!@JQifcYT zHK^ECh=lva{$QQ7V@=OW?nhr<<vrtu?aZ5RTo8l_&8soJZYR_3v@To=!IOvGFIu66 zhjpUfX}EPfyETk*GCpk>>PFk2if`NsCQi>9q9|9vMSyge^-9eUEWQQiX}b_r4IbTW zp;X}Ryh)#yL#O?@jqxH)T7~{o;q(Dz6{_&}1Qwq0$BOk<WS?fYO5ckXV_Pz-BZP68 zISbkPv#ft$v6;7#v7oZDawfY~96fl>_~a01@?Fe?NN!6Rb+U?UqB{R$>aC*Uh_<d# zBtRg9kkGgWZ`|G8-Jx-J4er6+CAbH7*AU#@-6gma;8xCe&Oh!;KlB*gRkfw|T64`c zCxq|&%JuHrjG6E6OKJI7-ERo6X1w%OdGxF6CMO(MJe@c%wrk&Sahv)$rv8D|M}xwv z?Qb%g;Q%M?7vKK!oE4>R4GwdFm_ern)*To5cB~W9aYm6rzAPU3?Tz0bOn41_+z_RE znZ%DW3-9#tq8H`+p;I^)_u>O_ie*%mCIpgfV1~yA^i?-dY5r^l3`1ve*rH$6#bb7q zGe(?!GqMBS;un0ucEF{t^E+l&O_}AwN-C<<FtdoV$!aWdXzEpC7cX~d5Kez1q8UpI zxjw+TxH&?5nIWtHqLCcBw0C5Hjf)!r7<jN{7oamz{bgwnJ+uYbXhqxE06BOO=%)FH zfufwE0qi|MS8;H7SjlB>Sr~pTJK&b(WSmE>Q01`ki7#mD*|t9PTXaXswla2^$Id!4 z4;mR&J<(0$Vx1A@{s7t6H8w2ma`EkIWu9-#@*+Q7kWFtc{Z9y=40QN?X=;9aXIddu z!5!ER6^6Ov8vBJ30y?eE{O-+`?=WeJ^)2;#-fTB)nI~826;*M$uqv;Dx50zcGF;nx zJeE%)ExEW-Pc0>_`5)5-Iv&Dtia@31dLF}E?^J=XPTw$8=D<#YyAjvY&?Aqlh@Wj> z$*@m^oir+<`<0j5{(K$sq7012gLS*lOY2o-fq@l3zB|@@BmvUXg+K=+WFa8ZDo>x= zgzMAv1(=OnoTn7hQc|b{1knJ*xeZuSFj?G_$e7pv*(E{NbA-$hdaEM_@;T@4rAWr7 zRv*ry*UY}B{q*r2=6I&?R341+I}~C6Fn*Mek$rM2j2cE5K|gsjx0Y)@>2xuP33Z{C zGe-5PC1BxwOAY6Gb~~-XkH}}6O2@*BWM<pqFvbYA6#is=xD%-*8_DFZD6K7)6f~oR zmi)q7Qxt<XkHcY*l16`QBcz8~0LpXT;k87Mj1miswOYkps>D~|IYY}Bd6Tl**gVZ! zZ?5h~^HfI}sGk~Ntj|SF!O=^{6E`Qjcyt3?>AnXC8~*x^xY*`ebw&*Ym_L9@gFm2M zpJb5ozcl@ElwKHTduM0s=;)^i*eoFEfFa{#J)cm)efKL&T~#U8cq6%l#ihyEbU~i> zqVJvOULMVR+K&*I^R<qUKo`$y4ziC&-}p)zkbC)l7?iNYbf2zQUQGQi+4A-NI1$~S zIzNZI_mb~0XiI#z{M%B<IhFh_>jS@O9dB;fZ!RM}l^Z8v3x4&amOEKPF1_NZ>EY%n z$nrNftmd4YVqT<Y4yqJLvNw2PnH)`CMf1s^E{QOFY^~<^;FqQS%<9=|^%DHIiRQ#> zY98Hda3}q--#NQ;UxH0%vjQtLCt5{~$B=af&)2I$*4~q3$w<HeDzw3BAul&q@IDE6 zd5kr?Uz1;5U8w-A<$v{>?FIkQUcY}sSaW#H$>jIt0~EOng|L5%-qRasCg9%tbcbi& zZ)Ln1e8K4$>Gvo^Q(OBT{WJg+1-_R6#BZ1SqhWNFlq3>n4`ekhxBPJ~<PdB4{Am^m zw4C{ir*d2!zCY0=DCC1DTY+3<Sa`Jr(H<*beZv<D`r->!qtg*`i@GT0v5|lITtlfw zi|Q%7X7vBb-1$ZmP`x&YO9nr#v2I0_d{f^Wa5kShwFV<u6b;a8V6Hoa8K+ip7`?d- z!|vZAS})&VQ_SA@-Rd8FcZRNfe<h%IbtlYq^(HuyuwZx^Zawg6B=N&iMB`8K%L-8k zF3@&RodX5y^|#P}BeL1oYpoGUczbu(4-l<qMae470dUZa<75PM2l#$=115ldR=emt zb2pD+(((*tH*BY7Iz8%)=j^#tZRYi%)6=-go*(B12)C9Y{b9$1EkyNrr*^4OMY*pm z^9S_(UibPci}2gk<PtF}B00mqKYd`SVSm(Cj(l~5isxQ2%y+=L(-hl|{F6OWd}zq5 zx#E=llb`D2n!_%>@r=og%|W$$&X<BcZ;0Zo8EW#aqd>SntzidDHKQ*g6NOB_KDU(! z)?XYG;a7_Bu?OzeiB~(f_%~O#+QY7$q3yRY3hB2`S>=CO$M?T=uO3#__-Ml8jAu;$ zcO-6Ma4D>OGpd1?C{QwLZ3k@4L_|fuGBPTLDPy}K225sc39C^^=qZWIn!}^v{NZ}x z$^E>h#sAH9t%_CfPof8$bkkaI)HDC)&wtTBY^n3?3j#;&8<Zo3OrPNa(X7VPP6jQU z*>|d`+Mz+T2v2VPe%IvGm6^}lkuUbs<frsZOIN(K1;xIg@Q=$@(gTsTFOm<IE5$+1 zO>v(rk$Gzr(YlwC2|{DABhtgA3gl%)VuRp_Jzn8|guKz~$USTG3_o^VkjDJ6V^Nec zcxrm!d+XZy9r+gY;74EWAWMfsA6~pZ08<4?4l|y~N&?=FO@U{vNl8h=Tv93ZK!LnW zFcpZIJOc?xM(u~fxQ-C?V%lPSy2qaeu=L!}I>^~Ni7xn(=PgFzOtv!Hj!@{<y{O1^ zzTXdDKTNePYN`gVIwaH#GD5A4y(Ff#n-9cOedaL4E-+-O^IZN~m4ZJE-^?qoWK8&| zq_{kt=KO=*Ht@u(+_FQpr1sam)`O8vHv<TRM%ZP~R3-u&!57x(yYmz5KKC1frO_)p zj`EuwkIu!$g?Z10`L7$h3jxm#zI(eG);nLK<$dbW?4(eV5R{{Ag50EEW@IDny9Wo? z9l)agvsn&r_C_I_Uch9ux4XM}wKrn;5%pc}TF=rN(Ckz48=2;SxSar?7<pcO4yJ3Z zT-D)kjHma-a)DNxp(82W#fWAbCjEY~rBA3*_w&mK3NqpiFcfz=jDj6n5ltM(pC-R{ zY!)HN8^1=&>PHk$wK|%A@f_3VD%I;cGTf$t<^1Q00EJVm$T^3w#V$L-M%(p(+0;v{ z^$nc-&a)sy-mYG}=M5vsq_6%-sBcd+<69`i=<QRZRn?E+SLRyC8=7E8o5AX2Psz{9 zd45_U>OH`#9t)5eW-_-~YgGmEhdYH>KvNbLU`MNAf5YbYczgAJ=>tm1%PfFhOCKPM zucFuN^WyPR6{OzXJgj~1+nvq@0AT?@GUmfV*S0$0qm0AYc@a#d>GW3fE)0nfrGFjL z=7G-5rfx7adqipfL9_vain8*Z@~f1h)a8I9xcstZXb^v{Ka0Z}yI|GSCGXa|UhMgx zvX!-8S7UbK^UNRKSVo^Or6be|t&Q*ND1MXp<WQJ8vk$E5Vhl1c2zk8mNw7bo)K$Fs zwcKSJRoaM;Ha@^#dbjm<HGPdAkQZ1xTYD-NH7{hi{tGb7KBJ=g-P||>huB1*W{t+C z_h>33knZ;br#ER}0nz(ENLr$R-4wE~uWz|#o!FbzR#o?^>qqr$Vw)X&4cV^0ili)0 zUJuvbMFE&u!QsJQN`q8R@K;Fi7R*8wq<7thyPDX7Jo)*f;8M5#X$_)K#~-I2NovZ9 z+yP)GeU)<X7$(2TB&G4t^dw-%@%P?;O<fy<naX_9m+<`{bB8EodMYw1ogRH;w8Jn* z@$N`0nv!5Mydij8V|jt#12PR`Qx)U>4So(Q$tW9xs1Uufhz7U5=qqu?i_=fk+@#*$ z#nkGRzX6#!u#{*d5`Q*er&H!g@9*zFjPMr_d5i}z_3z(cc|`!0sj879`_`D--<z04 z&pS^UP0tVgh$*_Lz7-WNSnj%HG<l*;M<~n(;a||T5PWC*btAv>)|=5w)m~M5J-C(I zBMJru#eB)Gk3QCM1j|{l@|xjZ%6-A;D0f&{-B&}Smw7DW?{_&4IqImW?p{TAqFG~D zTEb2Eg<xcPBQbjOQMj{)Ih;jMtAe(xcU0Z;wx8VSQ~AkxK)Vl0s_j-}+79_<&bRh^ zR(}#NJp1kM1fqaffgl8u8LF6T!r^w2Q_6jOauN|9j)a~;6QF|n{jR&k=|H7RU0b`% z(BM0Jh;5~d8G(`3C9Q^BTVynec6LxAas9ye2>Tyv-~Xn!HT&FSXYrwZrK;!C*xEcj znpxCU^UsTP*I*>Yu*witwA_oZNiWG4z41nz+?SohRZa7Ot!~^PQbD#KU_&FE1?qDx z{v@V2%U_rHk*|v<eqWX26+DC(RSpqZGhO~>JkhsiyzRIAKo4pS&LgrWO(q-dK3leF z_P#FKrFnGw%t%#_jGT2KFuJw|8SZ!=Fj=W@4pVo?=YAzZzP0`(nj~%V@FM8EMt=nI z{XjSHUsHgJw1ed(H11jmzA422?QdCX^D(*FpUxPk1v00~Nsp<dv{v^WcsW9Te;eH4 zYH~7IxT&4$UuF#9nL^b|Tu=@Bor^|iRZM40TT`5E2Wm>D?KVU6I+4?4&Fkc#5L#G= zVJ^_E){4b(M22qTAPd3w(H;`TcCW(W=r(5`wjvJquXSCp^SCWvo3dfQ;PKSBIWjet zYekTF-tU}ca=YE#{CowYYB9F+J+>QTJaN080lBmmeERB-^mL#*GYC-NEp;BN$GC6; zVxvZ6ILc?P&w2cx!rKrw-u$GSvcS|w%e?n-+TNIT)MrEur*{Yz7Fs{@ZG0X6@Tw5F zhZ;p~lwm2hVtf#?3WoK$lm65I{WdqEZol0vIGd!NM{I|vCL<*#H>p75j3}0Cgv!PR zpWn}j+=ky<qi*$F-?*_gVY88~_3n<lv!KIRb8Q2megkhZne!0zvqQ|V>Q1u)!oa5a zT(*#TmrL1r)#se=pL&2DZUVL_S?Lr`Lfy6UIDJRjhCtLij9(XtAB%?_HJoS2lEoe4 z>3X&uViviS{CsI7ltkL<`sw!ecCp>P4p^&un=}aC-Pr7X-^GBsAC*xua4c%?1Jt@^ zhn?W`y(B@!5rG=l7ob<9`4VJS$=7#OL_t{2s^gu9xLe%(e1YIFg3iaaNHOKPvlST9 zTzIRDi95c1nB~2w*@!qdqL%CYjSY!HejxAjP7RSY$37g2V398=(+lxy-q>8nf$P|h zjffF8yX$&3J#BJoA|%aM#bfrP&aKof{5;(QOW$HRF(4DB1exp^yfyx5UY;%ZelvE$ zUM>FIW_;v0Iwf+%;Au9Rofe)S@`-w+>ZXa1;%^rtG}TD1yQgOaFu_^V(@G}zQQ_fA zVY~(J+lk)Wp!2E7XK>_7wO**givvNQ+9LlE4<`Y?M<jQX?aub&HHXi``@7S{uw`1+ zlG4TRV<vlx?KP);{S$igHz_k#+Vv)-_5JE>uSK<|k57$>>N!P@%y*~D6@NzHl0qd8 zO!8I%W6A!glLZI+Hy==yW<+lY%FjuLt)1XPq7+)~U_fd1-_j#!Rhxt?3=AM3A|f`s zUFN+K12~Qi3`+X&$Ot1)i&armJNBjhSA7T|xS`bQ@L&Zr?5PY@%-s)Bex0^D?T;$D z+;b@G&?Ns>24c2(eP2M{^E*)0HNj8vkpt?tsIOc%>42*SF5rhnY^dVK&cNUfz{gmO zhVOt>sK#o6?l!Ht8u)3les+Iz%Hg3onHZVH58!%4e_c2nMsz+Ppx~hbF(P-RURNc& z1sxq-DyI{<Tow=bG9MT?&Q#8J0LTY=rZGlJ{{5<*4~Vo}5oq5zPvEoL;sNs2HEx%N zWC}oa3b-4fZ^m~rKa?@w`rm(AnUj^2l_~G(E}HY|!_)rTbMKy82Vw}zzNr6uWTJLF zK8hpBde3`X%Xh$e*cMWg2b>)$7}LLhi{Np8lYzGeZUGz1$Nu(M6+L`%cDCTH;otAx z^9x0oi;BV($!Cwt+yFOp-@8<27MGd%k)NNR2GXkXah0@D3b<VN-bjj~np!a22aFP9 zoBz8e&=Yv34;VyUTwLg*0e4BL@v0AWK%}x*e1U<1No=Dt6C?dc1|;MzmdjN7G!Mv> zf-o@tJ59f07$VL9GTYYKS@i!M;?IFYks8W_00k~>{}Vpfcc825h@Kp1=~NV4SOt`^ zf3^;nSB?6}k}m^;Q&#Ppz$fRXoyr}2+mb?~`D;-Tc<$%G`6<Af$6zW?w3J*=P7VbJ z2d9888@L75(_WzY*&iS|{A767Kh)I7@D2rU2|<~Xd1`v(E@nV51|0>Y+k}j{FoFjK z=)5YNBtaja<Z|2bZ2Zk`KYi;ldGOtA*?TK+_gwmpu+X#DGQRNghWH2f4iRubReHhv z`0i<8c4YH*V3klt;JOGA+~T9#hCF8|Ny8{nFTbchrFswWflteTqS-kn$~T~AI27=S zHv;U%G@5KPnHIX_5Ga9@Ln~K5r;QFl1B6{sA-|K8unq=hS=`(E<pWG8%>G|szqwry zMZ}i%Y@c)+6`BC5wY^mAP)*l0-8%>Fn>9{d8wYQZhBY-cpMhrC0l*7)_D$?<RdMy& z!%*xAP_X_hy7LCWJ<`XnI+*vFBiE0BIgg?a;J{S~qmG*1Yn8pQ30P4xSk9I#Fri%V z$xBHIsi`G1QMRMB^R?K@&DLb^F4D8Jt3!H)o`u+l=F7W)nfJm1JX&ME3J2>0_p{h! zmu>uy-Ru@J8PL<o#>Ynh@D^191u}0V17T1k(7t>DAeKt_B9KHC$v8(_2S5}6oV^w2 z;(gNk0FAgGX$KB~d#iF->jHLY6A1Loqu1HliN)ziYH4Ma6jaIydb}Yr)B%Lo*YsG_ z)z!%eno;B!>^G`_3J$PH%SLbOCnM%0AHa+B*6Zz>kdXfV{wh|C9b*LHFB+9GFTYQ^ zPZAccfwspags_N+4?;phvklbs^be~B`|etzzW<<igU!IyB^RJsiqWVanN^quScHti zpZn8%k$CSM>Qx#rC>sIT;5{`pby81kAhec~1-1eFCU7X`0hU<y-@d7NGt*jC!3xt; zhSJc|rse5<wB?77z(WuLjB*75(`g))8LjKi-?&vD$ZvN1UeDiN&!=jP#7!*n%8Z?w zeF0UYUOV4s?VIxrJy{UL?;x}mpfXV1)TAEORk%W$a!?=u=))xv?Y(UqrDCD-;c+n@ z|M^o<$r!Gvn|+g|FQccWl`b_dCQV)?FQ=mz>F7?Lzv~*AFfls%I~WlBWqqi%iuV3U z;qBISnf|gJLC*j<8SW{5t^ec(x!}^bQA~E<J&6HcyL`xa<<-LZKM)89Pk(br=bab* zYiF<N_K6DQKv@M|y{bbg^8uY$g`xp@X_9-CZa@oWVL1u4TDd|(AWt&jqFB~L9JIXF z<~rwLzO{Maj8>Sl$H?@_ucd_(&}*IZOBm4A(kiVjnkkIw+7b{s$P&#R0eo};CqQtM zez282C&U)|j1Uf^Nxtl>o`t4?4le(}bv{7ePMY1Kz7Il!7HPsKK%iw@ML}2v`PCCR za`W5)?lryr{qX<HiNjc$QNo>21|NAvxvaKo{;ajB+jO<CrcKaj1-oVNtOM{r|DS;o z_+2gzMMVV#C@U+gzx(W6f`IzS*4|zr&FEORKi=YuX9-Ymm9l3Ni#Z582w6!3loXIZ zel)PMD*Iv%VC?H4Z^mJ(lzU9Lq8~qgwDMEpXeP&C$lk%V?^`VGdhcv*`T<^=fUX`) zR#sLjAgbCraI@vz=SfLVFE3k^nw}nxWn*@c_DccPW0Ql6tJLA`KVy&Aiv6V(hcP%2 z76rv^cMJUf*TDOrcL?e?cR6KNR!D;8bYh%?#;acuU?bPq0WC2#lyGwt4Cm+3*ECp! zdHF(AV!U>x*{L;Gz%fzEnhp?*s|XBm267}zh2!1b0{G_o)Wa`HliR;~Xcok&3A+xA zcL6z-*-iy0M}|@F9Zut8Uq0M`z`!|=M%%U0-}8VN*sPDJhzO;tYWU&quJN9ubTh4O z2yxnoJvWe)9BcMYb4QjL<B>N>myyD4ZT18v<rFl}<D*sd{iR)yFQf0H?%HvfEKJ^x zBaU0yWW2_6>>hu6F8k7U@hFx%#$*P{U-dOjWe1Rz73;H7X$MZj-|tx!W3#d<MJN%R zt#$a{^y(#)-%rx216#D{FhWuHEhBZyd&o02FA*kzt|5bAXktR$8?Wy-U=&vPj*LR| z5F)T8H8p&D)Y}<nd)e+A9ue^w9UW*A&g==}k>i;z3nU=Op#1x<&>j->NxeUY2|jpF z*Id<{Hjo`SWZC_R{pXoc#wi;qz05_eKeAxv*Beiwrx3@eaiY*HeRA6Pao>f<;wUhj zDwa8!uaU<Zf7V`KQaiB^)d0v^KFn451VO{Xmu%Q`1NuwBX=!O+f<Wo@srLHPKzX>Y zhZdm&Wa3qbRe)9GNkmHrn6R;SzT9cEO6+q&>P(}a{;+R&<<E~mv)sP`)~(9(!YCQw z&Sn(bkO}m&PY8pc#QJ`PP$%AuzXdsat$dHWXk^0{&^G4g3csSC2W%L16NHlI#kCUE zD|LSUra@2zt@Eg%RB|-W_c<wXT(`F)qYPU}?sY1MPLIU}B&(4l4eeL9axntht?~s$ zMWxpRZ=Z0pGi9K9y?5UN4&?qj=vu2_8uzuCr1<T)1d8@1^N)r1uyCYRkad#wZ|)P5 z-2Msd?SwA(TO*qypJ@v+RzEW?`)t<5O%Ul)v<JJ5u}a5N0JNOyLxNJg_tpA5kQA+v zO<4eamz@gx*e^B}FSa#}11xR+mm6htLaED#7@#qDxK!LK%O{e$kEIXg*;MNbB19Ri zV*yv#@Et=UT(LFc+wrWVChjBenm4$B@*DWw(J!eJiazZa0^Y>JnC+N((YrTzc=%x? zse6cR%624iT$c$4!kUviUz_t$0O01M6UI;QDP_-7%YtJtH)sDFN9z7&I;<s%mz0#$ zEYXj=v}74AJ#Zs-SHDw_%M4(4ppIlj@S(4Cd@VIuhJeKE#+_!BPLvQ~md8gc4vg~1 z8xv(aN*_4Eu9fKi>bAN{-p7b=w`z8lfC3q%j7eF^ZOF{aKrAueGx>!b?m~dsSM{%I zwRzj^kTSleqZjEIA9T*yBZTQ8WCz!IP=G>v*^X&3X4cjHoT8v{!MAq+O0Q}?16bJd zZoHGKKN4`;O&>b3f$;6qCwe86?++Kd!b$k~ba&#wxEckbS>qW`4GVl;{5P&|XXmdd zZ;#n;KWS{8l?ypW2L~1CJjRSLHbf$)m4={c-uCrVyg7o%D!_QVjyY0-;#S}_sUyST zb1)Kk^(cI}=Li6r-s&M*BrOxjM?`<4g(6_$pL13SD<cL(x1<yYZxvUkZDN;9S~7=+ zrCcI`RT`_Lz|@&k+3=SZVX*#!p(yJ4*9J2q8f}SA^gMx<W(y5%@oLdHYZ~21J%)0< zqwkT=U1>Zz-syF_BGh%-z#qpi$$?(BDbM3OdrooN@!QD>@V*bdgtLwAr#!x(Ii4!e z<|-S)Bk=rP!E`7YMl`xz?R<w^BdL~13hTY?eC~vot=kRqZuHl$2EQcN^FjHbq73W2 z5k7<L0UtktuR!y<_nr%dVf{;w84|ChE_y4}=X@b6|2k5cZUM$&=nvf7|N6DIjJ_i} zavbKB<P=5+9H|D@-HSRgK8@(TUdcP8%ESs&6Rk;uEmMia2S$EVb`?_=<XF*(fn&`p zc504~U6mrQn?Snt$Ctcy=1PC3sRlc<qMq|cc1Hx6nxwxD-#S&7jf|JA(23I1rha#a z=SVF&VWe%ZZ|wj5n{rC}gla=MM=@iHxU|l6dRjPcSts&J1FKAmntIJO%{-`UkMF7p z<m(JB>Nbf_X>R6d;J037YZ5)7gmb8?g+qFMbB9b`4ZY+w?_M&|B_~6&5;opm?=2Z7 zUN#v|X!8GN90ZOil`|Ct)%*n2tYQTD$+d?Aj1D+a=qg$y!Pceem*gy5z<yl$?8Mp` z?SLcFFmZvigUrqQQfr5|`y?DDp6(7*(4OT|$aN^$?U`}nIrncsy?ajlCXnU)>~-Fd zfXKH3flq>yV;dPCjnf|Wo-y$~kmpQ>Bv?dV=5yqk{1G-ovI9n}l=76aq}dlQby`S@ zvCltmJSYb3yEEJ#lyXXN_6rlj51kMo<mlf0wrJ;sI#M8xH>+gM)EIeI1yZXYR5Es$ z9O>6!ZoOxOPoUEQ<vSk~D%kS*ZSgsAgX~6~rJdDq`%{^U5^Njl$(8h-+CYP=@C#!$ z)5pEgthBZD`fQl_c%e=LYd|ACnQ6;;tM{%Q=fi$0*qF0>`&$}E7zRnWsFt!tO95_$ z&x>fLi{HE^xy<(CKIiULeK^zEt!vkF=9e~LChYf^;|}~brh?MmV6*Jld<IS;0+d~C zfPmo@r=8y7<$$-xGceg#cck6a4gc!Z1l|(zi+G*6{mre4(E_W=iJU#puV4U{7w!<f zdvw0t%-z?CNtE5;mI>JR^ouc<Px=q=1GiPx(ZED$eZqrsu<s6`<Q78N{LMx6FIE^y zvELL^XG59gbJhc(Atf&*nyFHWGK2)|Dbd93(6f?JA8D)+i3MIQI`I1Ym<UPNREClk zthuCbTlaW=lT<Qg<P?~k=6iOOyXK!~3vY{YiEu^9X}vi)IW2RBuSLtgB8-iRX39lQ z8~((-IF3*Zw+a_OBaBH%#??|h5|Uof?RT0sNfpZ2mfK(&ewiebFsvnB`?pTxGZ^;H zDRsbJkN6NDY^PypwwR<q>#Vl@grA@TAl_;f8KJ(9JBXs|vaTPzYPatKhqmzakyR0D zDYg8`9xf&V4iK<g7xNF+bM;n_>*ZqAtou(@ns<op>TXrX?dX&|M_1!RRO>O;88qA8 zLRC=fW*G`wtsI;)t8GGrnOJ+bLoN|mHY?SijJt---c#Va+jIbFv(QNZ^8g+PgXh;e zgxZ-I`O>dILYAH@kf)F<8kbCqR6K8tU1&ZtH(Ov|0GE)=PTrdv6q77Z0Gm?W_Z?Od zNi-ZkoV)e#WgD~W;RB<Tt8TxMl7`}_Qn{kvm6I0e2xe4}0gnFO9|N0iz~$<mra~z9 z-U%Q$5u?4E%9ExhxldVgX(2enMG$dz+GAuMLd|9c#e!;hDDYyP-Y5?#87w5XeWp#~ zV4cIo<Pse_RzHp3ISGs{auX!bB`{DBr$;FI$g7KIO4Hvb9Oxpfk%2iZAbSq>*+*hv z011)g$l7)p2XY_!ip=&2H@nH{ai8#^P`C4$bkWZ}o(cDnbeG5-BoCf)tCKta8`oTW z-UELLegCBIu*#oMLr&cJ?IA)d!Dx71MfSwi?g5Ra|9~0H)8ANDz0q-%8qSzoIa|Wo zK<0<4iz!z+btUdEE+}l(9kQ=#m}u`em`czeS(AYTF||M}+}sgJnCA*t>Zh5m9q4-7 zB77ALQ-oMxA`!KwT{Yq5IBFt^XvtuSOXiB?N;=3<)6m8{zwG#>T$X~`b>t4N#H}Lf zR?4Fn04v*`jLBUeC+@$I$zobASn9(Y$~TSzquz!~IIX)%zc2CZJG?dkXDBk9$E*RN zI7urdAU9eT54Xyq4>#4~<U}<?2FC<Fi(CCfLAd~%Kk`BnvF|10R`9}Akqos|!4i*M z+nx4r=D7fWu>M^;fgA7Ddhwz0H|5SNyS3f0@v$s56<y_<Z~EC{=d7=~_q@Ksxq_cZ z`tQdmG<4y7Cm8V;zUFG!5bE01Cp*zpm8n;R07&9(k6d`&FK72{Yh1I^N7UqyQUIfX zJ?-TP-8tnClD0Z*<4`dEMbPi8n5u}!de3EWryedCPL0IfAK{2P+{_eCheWI;>k=ky zzLhUAGY7Y4RxhFysT8YtQ|o|@q=!Tz$HGJqj~MD4=DXO;14lxebb8{xzIZ=#HQlaD z(MMb{25RGqlXBJ9qO!Q$b(s1!DAuIa`f-LyIz^l9>=WGAHIqkfl-N|>lxNuwV1R(f zYa4P9_Di)X1~ihPlOs`VUm%*Ar$nLDf|A89mp6@+<K#llGg0rz^!dvKVT}AnQXH&z zWrAV+NiI>bo7rFR1h_(iKPfG3K-P-o4HH!5v4{3<i#|tU)8(G3>Z8xNRm#g!nMghz zBW8-t$Cr#rt|_wH8CeheFN!HYpn!vYKGb$U8Uqf6f#XDmy+qw56@Z{B;O_LSy)KkA z8j_h*@f4=Xq5~4?@>Z+%y-RJs#8$dF7v=@U4G1WT-YX-iY6%?q$c&J{p+l}HnI5?@ zn2SwfFnE!8fJL%eb3U&r;7sSIlYC-C2@h$aY{%*ya&h1{Z!n$)y(SZDA;2*kv4FKK zetyN<wa)~F@z90K*v1SD<Gou&fx|y>s=WjImf@lvk$-e+2hC=1Y~E3V+_NZuTOgX` z|DYSuJ^aGrz`bfFm+ZI(8rR8MDG0T*{kWhBZ+>Jv3c9aY5|HV!3LcoSnLL&<D5d`V zdt)*$=MC5U!Sf6)<)G$25{%A}+obcZqVhD>>)P7&x3M2-Dc_CbWh6|=h4m%Ob8+A2 zx>|Gf`#~Eq=!Y}*W8K@XdY*UO+YTk0Xf~3L9Cx@vj*i)qUN*A5&(Qm^jEt5v#+R%z z$&Qkpd=4_ooF)$L*Ph$f&SltF?RN~3DxGw=TQKs9LVAOyx@JRH86cxQ2uZj&(-o<E zFGZa;w#{3?xR}CFU_bbUQf5hYEBYQEX!}(WuLPklVWI9*lFS==h(w6?F!TQP*Gc`O z8cQe&AygyZ<Ze4$^MpX|d@vc2aeFG~dv@I^-uk$B`3ZVE!MOka)u|)$CYv0V)G(Za zE>c&kCoFo@hq-KQRM<HA2lm$E^}z^G%Nv!9)sn&RG22hYhD_I%IOWa5VTzbW%9o=( z@CDq^`6bqqOjkDN+M9W<+4n|8&=ObZC_I+fkS5K11fH+=-v<`iG-mo0d^1VL*7h%R z&u9rMEOr3_+zZKc&@B}f%cL=$(|b!hhRhuD;AJ!!Yz7zoE!U??#a`ix50#dq8k-sL z%`hbe=Fr6jh>WnKG8aJs%8jCH$UAul=tl-#sr0jVz@<O3kCRGlkn+Z#&|2hfn6mx% z8F76j1^XrC<f5?`d$0kRbI$-%Y{iHHHfB~f6Il!yKrkDYas56nzT;th>)G|sa?qGy z){YU9qD8U~6NLTHb&Yk)H2XUzUg_<ZrzT<Sa6s8tzmXG0L;aNkesAqzHSKpRhF+=W z3T?u)W9)^#Ex=sadJ!rrDH$$N)JqZVe5XPA-!SZqY_@&AX2*EP;$Tqk4R886Rb=uU z`Qo=20j_JJf^7hU3K1`6A;|u>K}>@D%g9ir`De5c3CKrNbS$C^T_SV=vdk2*ek?@M zr+w%kuTVM>DxA%*rw{sdRjGy<ry_^{CiT)AzKE404TXSbaTNlgQ2y_=A1v+VV{o}L zKHJ{@n*l^yYl*(Tb#FkWxJV@)C0LbbnBvG_)-CH(A1XRVo99)#qE=0R@jdo17$u;Y z`s#0(4a_o^pJ@iJzim!prz2*Bbm6}h1aHwQKuPv7@rzDq`RD&$%KTvK*I^&QXH=2G z0+cWiL!zXTg=C%RSmcdAmtH^LteMy<{7=O~NhW=y(7Z&zP<N(Z7>WM4`>_<<R~Yw= z-jCsBcaYVr>pxT4dA^xGz#6{irStaRwX~!3J{3j}&W)IYVjyTLf;Cl@q5<QBu_ddb zkiUXVkXs~L=!dySn8_J9h2cesBe-};OffPzR7H!p4ap5&!eOX*J{#RE?>kLyjDqv5 zm_PqEmu-dxgQWOT_oKD)_W$h++!P@qtvFGUrM1--CkR#t7RmHRWcsM<hPS$|=O0$- zF*1CN_q++Zc9>KWvaoy>CjU@9X6(F9mT8ra3UnCT8}n4TQo7sPg1*d{S>u006UF?I zeBV4ml$M|M<v&{*_~`!`I4z8YpzHSba{uot`6VTTTBT5Vvq9r&%;I0r(CCk`U2_uC z<szfZKMQ+eJp}}NzHhL|8HOnk;G+8lz|~zPF;T!wI9MPg7|Qf3D#xre{by_c`|cP@ z41C{<b(<SP;An7T7*Su(LTcp{wYLu+^m(#PBM}%QBT`<>q<R29Np6hT`r=YygO8U0 zIL+e3Dpg5ADzy2J|FfC}J9rkX^%Gm&|KCcU6-=kIZFSpnK3$vwW&v#9TNs6On|rg7 z+lK^h*${^ux_<0OirqdN;me7i-bmEVwP_*rqd2mkEp<c-jX!PlbO1JNXH2&YiV-XM zmzMee?3;gcQH<i|N2Acm4LsX9u#QkOK*!X@vOi3Tq>e4Tyu8}vc*%G%^T}o*Af;V5 z{hA7puBGBn^gc+6is^Jf3_iE1T=KtxO$&M<WMz{U&QWZCjh%hY=p(d_FWPUcxbIAa zHM8^Jnb}Co8vFlEgXgDNMLd_f+FBi88er^K0MqaW$y^9hL&_g)5fZ^F-n*cA`flP5 zLHBZ6SoXCvX0u`7oo`m^C@DEKME{?M`RX@*`X%O6nkwEV<l4l@y3(2}ks^e7xUlH; z@9oFV*?=%MOVN?x;bj)s-~W9z(p^6I=z{|1#9lg4Si#urjiLOzG=eW}iYZ+Mi0-3q z!0sRcTx%OJz><=ZY})oe-_;S$D#!hTj|DPSjV(uQF-yx*|3hi|4pM;kUuSf@j+#Y& zlTpvYF6CsTkRHfqfjlc+k}D>(E`>OsB%RYJ@^>z1yKFQm_{#b+*+jxyvUC<5A5C5` z!0jqp!bUDb#jH<4)wEuv@H}C-o>wWYRT{7Yml7RD`N`1<p^9&ke5|2uwNAb&kX1@< zRviAqCSd`%+4HulNyU`M#gmj3lKLbi;^~6>LQ<q1R++$j^(x_pDH2mXeM)ZjaiD&h z#9y%2h4ybe;6;9Qq`U(OpBw9;m~^toXVc&m?0DDdUq*L2pb;8vq#JLz=L3lz|CVts zBaFju7)N&)E8ikM(azMAS2HFNK=O?sW;>>Yi-dO-OfF4l*SbIFy{m?z<I*#PJ!H23 zEZRpNODo-^7D@|?Ha?VI_30^-Ld&PU>rkCOyp~X&EF0j<Iq;h}y!WgryM`kpFH|wL zmP*#tjvjen4P&ylXfIz)Arnoi{B`i(7lLiQw|(}wh^?XFc2M9{k!f5`Kr#IyvCAvQ z&$~tm6YsjjIacYP;=aefHdI^PylA&Sa`+)-G&;D(Mo=+O;_$dt1Adf!N=}lzHYeXG zH(a3>0E!8FkTXCOw^@JkGZO0}o>jDz03=pvRnLU*2gLL$+|cKd$C>ibmF~Eu=b=5z z$=fenW$yqfyk35iF5i|#E@bCB?i@59Js0v1CoY>4Pz-(k{F^1{@3g<#m%(hB!hwSX z0=ryL{$f8=FdV{mhoiuFTq93IHMMulkh$K2hL#w0Yn!6tPAl@stVSq@R(W)o*ihmk zZ(+&2A-<B@*J%Zn$0T5C{^Xqz{`j2{zSte<jfKKuQhImwXxih?y6eeCA2#9(;MQE+ zmE-qa6Mnnf9bUDPj{l)Tbka@wzEm|hrAa>&hfOc>H?f%1XSwd~Iqqme`tQxb>G{J? z`4IzeI$K9^v$UIn-?Cmp+vGe{4m*VL!e|6jdyEi*q(z5MlA|SUdz+Z|OB)VPZmS~6 z1GnDt^>iUF_#qUowJrnlp8ZRew$~-L&xfPqTt#$To$A&ZVb>dzm#$9zRlXEQeE52I zodt)T#g(VD7G?ZNHw`a0i^#E&9|OspW*E3_msUEfjndkz@(5Ne;HU8HpcbJwRKMvW zkKghnk=e#F!MO%L>{g6FvOvvW|7ctp_w82pxDvmJgehdMD%W+CIw4h089Z-83peiH z->IeI(l)h|jobQ+Rbg*%BNK?o)~WLIvP)qB|9@v=6*R7W*Xua<d4?X=9<`>X(pQxN z-L2Z`6`H<=v3R{DQcm>o_%Wq3nz#YS+s9wW<gh9+RZITUrs9eni7@$ucm@D)?V@}q zaoMV3XKXGTXoZ2qV52D^Y+L4S!dq~X7oHcrZJw5t<RbVKlgHq3g+lEX$*?Gu@th8; z)g2M=0m=y(^~YNgFMKK)ti@BpKn$&y_M1;J`nnshKw?R3a(#7MUA@ri$(<WaEX#=r z!m2YT6D_j!)AN?3-!d9XYLu^t;Q${cg7^jrBguK<ORsIhi?E%GPc3p61SRq?X9OR~ z&bM}<FS<p3!>Q}_)eN>(!%uDj_MztN0gTd)9}R`q{hnjoYszmrIn8g^3EtUnUp*zo zhVYbQ%o`HF&-uY`;2DLQ;fn9>5$kzOe$5qw8~&P;+|C!V6>p}c>d@+zHRAT+$OzG? zb@(e}0$5L66Dtlr)Lt}gEA9;4#x#G8pRaWj-fgBb!sInp)6+aXM`GS`@aOM%jz*IW zpmW&584Iw(bQ)0dvAh}_Je)A@xe?$w#n9bLAs-0G*o+RgS#xqMbB@;-`5%jG=RRKS zQvA_yqKT>Y<{j^L*t@hL9-9@*UVEFkY`26RCu?IiKFX5ObR`nMBSRnQ4|Hd8F{-6m z`C*}e6iGL8P#11h5j}hIeEenG@U&k&;B^9vF}C!?)Z{;L2XIgw1=jLn9bS#l^LH@} z^G2|T!Yt?6FImT?{hUlwTqZr*s5MUJ%t_aqGad$8T9h1F+jY^Mg+@gs0VUv6Lo7v6 z`nm3$u%e6Rlj-mv@=F`{nf{u5-9#0A<S4Z&U7MgO8@TojPyr<V7*PhwO`zXD&X=j1 zo8d1TC&Q^4fY6%<b(ef?I<x{o-MSzIuhKk+tQPp5T=#NQVjVM;2a25Rcs@hOrUENJ z!5SfWW(B#3eCm|UZd2UwtP}azESCqru74gz@M~w`<Z9qzqqX8|)AGsPwcFdq2p`05 zw);do+tuf|6@Ej1CvwVR;CpVEgXpJDMMIE=(e1hL4gm^vWDIp8pRHxV&!_}$$yHF$ zOm*q|>ONS-1OjHPce~MtUp~p?a>;pWeeo+@GU#J0kB0H?c3y1`xw*&0hx#P!Hmw79 z&FQye1)uL{eiiQu+D1P;=pr9-(o5cnXgx7H2gi06D?A4G=mgkOyZ7KadR?{|%-wlQ z>MqZaLqTmCA>|AqRkcrG1Zi&0EhWP+qrLy2;IiRJko^)*D)Wc51}P>4UBG`mp9ZNx zd}+Z#9$e2?^*rB~gUetR-<`OK7{ftCsOSN$8NIRiYz_Tj10xty^@5UtQ?OnZ`{XBR zotRa{n++j@Ya5V`=n77YP2i6%j#`{$N^3q#E!=|1sB0=1Y<z&xBut>50g0K}ew%g{ zr{yP(_*OLJze5(E2myy_COy>+`;Z?`gfHLeO{~xRd%C^7z5p_Rv?(fSLH~{S>Rs&Z z#(BB=eN!OcmL3BYXD4y&B0zI2KbGCep`K(vKxr$?(V*XsZGV7pu@<JHcf7qmC;xg% znM{7y;5c8dIo;x9=9Fn|_0azlQ+<Tb`$>ZYl88j~;05oJgpCW2jU3-9<!vN5i#)W! zDR$L_9KULlPwIJKJha3td*wW|!7F*i_G#WEcK3X8;WU$~#nVh;{Ol;c!zz5&nvj|k z_epZSaqDRn1X+*)hh~wrcos>`H%`TOxR>nO;D3cH@X{LFMFuZ2D_%84#p-zTh%dxp zvZ^DY(8LlSe|43QCB~PI>nw`(7)tGn_qG4|szWI{#aw8<fT@JXsw#y-n;id;l-#fF zY~E6Myl?v^gye$UR}+zeGyE{8^rcBfD$nfDW8h@bl>3CMUMT#wMJa2AUpV|;HhCMi z?xta+@bH2P911bLO%wGbmkhHlQ)yB)wfK2bAmWR0$Z_rBi*zrWk_UTTa5HF00l5ve zz24-3Y=FwpXhR`%ed6<4YZjV+k-)qR%2N083AMr>Y`pu#7o-*ca)HAkR6(s9WS$gK z?yFzG#rw?eWwc~ma?%NF{NlqX^eIS|f|5anmA8XZ;D|IZ8KYC7#sDO~T5>qrCIM;M zbO%GyeUt_q0nr}_bVww%nOd+;RD#Gq_HZhF?vkNOXyO1j{A{BYZGD*h0_JNLxHl<O zIsQg9bI&zFxI&ub@WJLyYFlm5W7yey@6`3w3mlbhI1q27RWQJA_Xfk+SsQbT>(;Sr zdrUueOZMc?-!_zyn#lBMsG5hYfl2|3**{cX0WOLp7%Pu$zG2cXA>>Wfvh2EE&C5Kh zrJ|B%>)#)OdGU5}ZY;cSy`0m}i^>}Cs=0zY`Cs6a!G)SnxwON54SZqli6t5;)Kgw! zmAo2mIs{M|b$k}cq;u;F8Z~hOXw?Fjbz#kkD?mx8=EQ(MLRxV+KvWG253hZ7DY&Wy z>gl}&f5v{WYkOcZUz+7XDNQ7%rCIz@7@u{)U?$Sl*4FkMmm*&ofZSgW5T*d^suf0# z1(z(UL(cH!)izf<?#6?<wFrv0&^hV&V9lh@g@iuY^|R=b$|mV7$;IgovfUd*QMozw z`a1a&tm*|4PBxV+&sIqkLTsJ-3^{hU68YjA+F7Kuw|&#oyQ!-ztng$C>R(@ntr1mC zCF+U=3a0FSrjJ4M(l@w&p_0b%Rn2<m<<eMTiZ5Q*q?5j==2FDMt|FyOsh^$XH$k22 z;a$lkJ>aOL<%Rwfvtpi|3y>`k-K>7=#esW<X*{zlN0*k!*Fu7kR+{upiXKi%oj3k+ zy~cKC9t=BcA5H5dR6MIy>`dA`db8d2ALFf68L!Y!y$H=&&3k`1a`SH0gNEoOL^l<J zc?5I$v3-q9TBFH?y<C#MMW&}OCZGK{b4rL_i#F;udbWO^N97FWNhv<)wOX;9DrDTn z^p(wh7@i?hNw7QUb)>!FXgqZLaqIToIT9K<<TRs(mKWhRgBhaNK;WsDL@<yeTVRhD z=wiav&tg$Ij4D_$>uovpcHW;O|94Jwn|>lzp@CUUnTat5Y3_t`bipaP;DAMZ;537I z1!JI^Np+!kU`o4a<!Yr7$5Lu+5^;~$?wQgeto)n?1`EIAs>F-yAjgY|Ta3I?I=-af z3;3sH^&K5WaPij?0j<i5bbX5s?N0|kEV=AF(|3~_DoE}MQDwXo>-cO}IfjOYI3F$k zm${PQ*`xBdfLCytO=VVoQSyKUKHHc(%ucesn;Hy|lA;9*EbF2x+0t1!P3Mb$Dzkex z+=)mI(5TAg4{jIq=mAlXE+FSsB=9_Cbl(o6kTEErI0Cq3V*)!r=077g!iKH`X`$oA zW{;gWumAHOpnUt0*nh|gk<W{zB?=4MfNOSsy!9Q@DZs|7@Npi`;8crPQG1b<M;Q$+ z5sbw4Zc-^cOT>Wd+YeWPOGI`mgwya#Nfzi3k_`*U2neQ_5CCz!b4v|<4+rwmf(8n9 zByhrje6c;2LMW1|3AR2q4^<=#X%rG`50W442+?#Sx^*lDy+T6KiSnRw&bw$B=q^}I zT{vYTIe&!7S{LX%q$I(>Uc8_&WQ$UIR)JbyYoZZ#i=7Xu1z{URE5$^mlY`Ibzx^hQ zEZ5bEqoCm${A}wwr3Ny0PMd6*VM0!|2ldXO>I3YIywhw)2-nplzG^_L-LvNq9p5KP zO9&?3*MG+;(DcVZwK4SM3tG?F5MX(4yl}**#_+D2q#HgI#^tru@$#q<8O7VrDF3=0 z#@opzI^fU&k%7RE4f%%l<DLheGBO)}&i#o-z&E_W&x7;cJ8mZl<D4^2M^&`Shhy>j z4Ov$BBoj-W(9>K#+`&lC{vQ2R&qdF0hq&KsX(5Csh|9Zz9S>471@;r~!mu6McF%uQ zo^V}DW|*$gd5i2YQyb4GgQREhYfeLH7USqb&^aXAcw10dVDj_T@sj%5Kw>*rCa0w$ zX>DUcQ_663hocuvpVc%p+^%N+AAo@9j~cjCB4n}h%Ut0+#?p&6UT>~3{BY=lsd&NZ z6Yfg!44neP_q^2#&AR@`$WE~RO5f&Tx>Y6P#qMoM_$&pqcGHGWo6HZkF_2OY{Gp-u zC<l~5L8Ji&2wG4npsM{!%8Fu#(m<AtKo`@U_h&`ifIru!!+N{Bl8%mbj5wb_J4Sa@ zvqTo+#gwwka3Z+z9GA_S<4>pWg1+zl`361EWs?YiR|`J;dUPo!lDK2-6sEHfrKD=9 z^pS)@#Vv+IFnU+hxPJ)|k=Chzm~1jEKavAwID9@5d%c=PIjy~ljDX?>EGis&q~Efb zeC{vxXyCoM{HzV*GOeOsK$O1aXKwRTQ1b&+A*#iSl7!`z+Tjxk(J}LZUZRSpZf-M~ z#z9FHlA>S`ft9dQ7O`wAZ})`ahSsS|w`qRBw;V3Lzg(55-KGjUMmvU?vPc6_NP69@ z3YaPvJG>>)P}D0zpKVl7Mg4D53XeL`Lj*y43WBP72yO_&=(SeiWF?ym<=Nc2I|ARN zuxiVYYoQx+&az>z(~9(Az7@G$VT(h`tDK42s8$3P{7+3`9hAY;N~8*Dv_3&*u!;og z4{U`^RAVYD<jb4(a*Mjc6)f|++?!?Rc8Y_xh=Jw;TWK~me9E3AD?&r_mm#GbW@+2Z zgLm9iY(&E%?-)uCs)qE}(|F#yOOYyI(TlbX3<>J9Sk4a-_Ra+J&yr$`HmEVTJ}Pt; zE}!x15~uZfWR_e;$O|?+7Nm8w2FKzlx`}?wYytQAf&4PNc-J!ApjXj)qn((~4k*`Y z%~Vz<P6DH3@3O8hyWGD9o@fAAw*TQ{s=m`dY`s|{;Dq(wbM*t$=g}8*;t%&ZJm1Lg z18o1AAY^~#<Mz@!J}CO5pEyB4E{uhqDUTXOllpZEsfZVcdz(m93UgnSGO@VLJ*@i> z-*wfX>sm|6RZ&=xhTHj2AEg0sh-y&L5{O@V1c+4{e@I<^mZW!+uuF9xqO0o|u>SDE zf|As?0qh3gLpj!K>2f|l{!KH>iYu$?)Z%hEHr~gxhAmm^^lpp9V*I6|jT}~OX4=*H zrzk!`O$0p)Ffwp2t0L}5y>znndq_-7yf&l}iU5sTqeQn0m!gct71bJjDF>^d-eT25 zF(jvJi&B}j)<2Z~B3d?=RwhBpRmV}h@%x-pk-MsovPm+=z6-8(!3y<|w!xQ0Y7*T1 zNvthczqpBlT9`cP7d)%@vX+DD@)7kGwmL`-rzC*vnA)~mh9PJ3Nzrjcxv++7MV~!h z%@S3Vsu~=^iDW$|H%wDX8Ey{o2S;=>9d%J=QI_{C6I;yQv(#*~$`|1s2mR_VFuF^r zsnDT-%s1wD|3pP$c)H>A$P5O>eiV_MAPY*`!3wtC{NaEVN@a{YSZ9ZqU!526B;zX& zTFg1ag_>|+GWo1Ax3g$Yluq{AD)oa0is}!~(vy<4w*dXc*tH<9UhXoCw7ThbH#tIj z*E>aJPxOK=m>aHS41Ov`qVVOi16HWeC;Z!_S<}0Zw>*QGo^ISOCPnS=;V9A$H_n_K z$CY)#Q^!w!<?GqB9cF!lW==+9tW7z2>-w>K#?_<>N_8MLR|0S%q}4iXBqfM@#>CUh zzlV~pt)2Y~luh9P{ayp#&)Z`Mu0Sp%-E2i2z_Lo$v+Muz4NO-Nq5wpWcgSZIlw`e< z8J;rVQO|@SL@qndJ-dKINU4bg1_)G-9~sXky@H`*FAqO(-LP3NJu!WkT>0Tpk)6}$ zuiMc0_-nt1A~8(9>zYf-9Df#;4l>lZIHC6}NCwIUV3uD2gxTR4d43yyJ-015_njzq zWQ-P#9D6Z+5UNO~@r%F#byYhtz$$BZKAD$QE7vekkk-05_qpeG-;EPDetEi$Pfj+y z1(xJfDbMq1T?Ie?-wWX~gBP~K1aFgUcfy~oW?F4e5Evqp@&ab3a>)&wlPAS)KQ+n_ zu?@1MkbQ9Z9#!Z(&fnZYR_ec+;ZzqPPw4Q9P^hf&Tw>C+s!;Dle(=EUPmXZ?Z74(u zr9!SDnD(lORv;91C<q44)eoIJ94or7A7jpFE1S@IvzCX(6Ie=RLs#Xa!1R{b`IQvz zA3tkLP)DWiK*j39jXJ`y1^xDGTHcIF)D@UL2BQy3q?z!Sc#5(jdq|b9&p@qiPMRrT z6T2+TF0cSWCsocIPI;PPDHLAm5VJ&KBU(wrtWM5neWNY~@QJhyug)kQuz}v=avZxY zCWFg8$^&l=Pz%Y?P10r|cTpZXO8(SJvYKL_Sy3CDci)G?gdmk6g?QQG3zRoG!E`sm zcq>mrPjSIb9m6~$vmE<yJYxjw!p--W%B-oi3egiS=X_!gb~MwLA(}<!J{nVtDP8$V z?ZU@BhrLZLm)H(L8&h5KpJMJhoO>2>Bq~(lOLq_;c=JkV=cm<TBOGRyAp6@x9yn8C z)VE*h$x<d14ni$EhPlH@{$4JnS<O7A)rB`s$*v&^P1eEQkR!HlBcBBQVaGbH{WY~A zjuX<|bz<mP;e$RnM$gZ!T;LJ3l@ZT{YvnMPmh=f~(wc&`OKaPOkin8zmQ(bNuHw?W z!uHNOU;+Wnb9UD$aQ|O|;s6%eUGDc@0otl}1#SqIT5R&m_3Cu?$I`wE!{4U3Zn!J= zN8y8&$>n2@W{QWl`ywi$vzVCSqONU8kHnroXh6UH0GR(wxl?QmNXYSQ5nAqal`Up% zv6b3NE5^Z+f4~m|oqrlvv@$W}*#n)qutfjt8pDJa6il04_aG1^rs(-B<)Cqpk|o#c zfaH()B`>484*>?V%tw<oEW61pzicQAK;iJzPCo@_4iEss4&BB<T&<ObO2uZ(@;qxf z7{$ig<c3sNQI_~!6=(Z+=$@1{hdO@sI>>ZIuJ%7^{6;&z#@XW`N}xnUE3ZS}*~(C! z8>(caku_`SH6s&cjwQm0LizszT0y10i=l>UX*5NY6xRX?c`Fo*Y78<%wM7m<VrGP* zS*H+GTWEESzDfnGngXktM8yR7P6Sav4s#*uHk$xZ1szl0sHG&;HRN+@L={%4$qZHk zdJZ^Z9%RuRGBCxU2jwu}tx`||M1#^ydM{%Gwsl-UWXM~o`{RIUNa30kRs&J$JS`<z zA<I)522rO5)3!lPX8<L&pj8nC$^cPd*y?8uL?e(LR(th01HUMcW-t^<mqp1X?YO>L zK{VeK<VHZ$7dB&32yAY^94XD}RU?WhW>S_y9f8BJN@+E{sG_D+mKulx#g<yD#}QZ! zDu@EvK|MdDRCB)VI5HN6(ol7d?pvBoFfe|r5%rOp>uif-h;nV|)C~&d9tBgYA7yf& zQX0zAC41W#>l2$FHB-(8Mgc!kVWX2vq>sYiOCxv42h;Y;p2Q+kBr}U@t*y%|qj$=4 zL${fNx_DQ*WR-YK;jC6DrTt6y347(?&o-F5ZbVE4bZ2i(l|Oy7R{r?*3h6X(i+nVF zzZ^JT`qe?46sWN{`u?HYQu$C{e=O(h{x7q2Z1c9I+E`J20nL+BjZX}=cPzI`rf*D^ ze)D4G-N{kXZQLH|F>#*(F#yf;-!+&(h)nj8vBE^y`7md5!womwIN!nj30Z91pB*@` zUskVLAq(bCl^K(V$<!}Ck(p!Omnp;FmJxklk)E$VDqWtrOFr-2S^D+vBA@hpQ9kJY zy!7byiuCLIiA<g}(S(fT<Yc=po=$vm<1Zule8CDZ1NY~prKRT%qkR1b8lCjq3#p2- zkPeJdk}uf4YDWQ342fuS0+^jHP6wl0EKvSXS$p;BWp${4CVe)kEoEkE*|KHw```aw zKKke*yC49-5g0{91tK4Q_+h(=BCVA%(W2m{n{F~Sl~i1mMO=KT4@F^QdjKo%2Sz7M zm|%YcsLX_YTL5HimoWp2=PO7j;U<xrL(+*!T_$C@EnCWDZ5?GjMM>2yPmgv$RH4)0 zGvZbYC;}}f)Ki!$s_;R&XCt+iwRu+ai1G-qQIF`Tp+yE+{7y}2pt`_d4fU-F(;r|I zMU)y!(pNo2259DDi)!fiIEpFCAxa)8!vV*=)oK!{>r~LRx=yxdX{F(mS%tu+np`tT z!Vwp+#ZakP;R}V<V6BdN)*?q0ZK$%eQcQbD^=%@3H{fy9VA5q3RDr(`im9okPMs+0 zBc&JTS!yjAe>3JnnEn8VHCYabq8yR}jM__mivG^_4Z>?MEOOFiflAVZakF(mG|%cV zZQp=jGe^p>+E^4}OP9qBSdWX<>l%m_D<C^{yOvT-nr=Ps%XVCRfpDH@JoS}2{t|_7 z1xk*u9vf#{O}0+Xn@?fVN7^xJFUOC1)NpF9so@PZ67V%RPRATmJ`*-MfH6`~d<xaM zyC=*3;kQfvo+*~Ld^SKzkY2m)K#mMu5^vy14JZoc-lSrAcS@8z)OUk)RG92MVuyUD zz_#N^f#fLkk(&FlLf^f;*U4QUu8~#I*{$oWsSim11&8Fy*A^>?Zk4$Th|$MNzE!OR z9Q(~XBv*7@EbmQ;mc;C{OV>}%t(K8G{^u2*0m;&8p8@wb1<<`>U#9NgX1V3vm2%I= z>*R_4o8{rY8|6>E*2<l_UsE?E8(g0bYQHJ?p9V&mnCsiOul0>SUHOF@Zn)vb#Tb0I za<9j|CkBu`-+M>i>egBMfAXqK9o=14&gda~m%T6hSG+HK7x$3WlV6e9Bc7M-%RZ1D zD?gHr^WTwKLtl`$p1n(6ed^EB`KgDb>&q|5&>@3N%m7Ns9yt@!O*byhKumxgC|B&) zuOFYebB9sBw`+YZ28AcX;)PfMV77k_8CDlOfKd#IP$^M2PZy_yQBofwAc(@rjnL13 z{&TtU#v2V7pL*&kOX2<LPk(A^=8`2#Oeu}PC@Sd9H{UG3{q1i}Q6;q($KLC%yH0-b zi(eRwqNtL-%SDNCqm=&P4}XyV`JewWh~+(h{_~%$uJnc*ZjiXRxUY=yRKW!(lG=Md zVKfq4Sb=PenAm@6Ufb#-n7MI?92kCw9G&&D)Z`}!<~#tR!D0C^2mTa3P$45AYK8xS z1_wG&Bo*AKwZsiKpkRZ_Xix-%+M=+cP#CCDsHrJdJ}WW+Y-y-G0Yot!%2(iXpvKZy zfgYY`UAExjTjPLX*rgTqwXr<eVAXM-b<`XTi>ze~W=yq?i4cZMD0*f9<aJVpsdY6K zF9M<p(Wrpt)@yB09LHeHkuW|wv2N2+b9J2<^Z=r?ISF|k7hxL*iZ5eEiS!hPLDX^1 zMagWcCv_L2rnFYWqKHecucpM(b8)-1dRt4+C0*8I$2(Z2AX=Gg<FZs>44OFBr1~0Y zdcOow#^|fD^J1yTHBMU|#uL)>Mg6O_c=3c)?C`d5&Si1{M_@BgQW!UfUM4CyKH(NW zjSbaK&29s&V#*!x4%d%5DaS3+P?;y$J10u?&|4*M>o7-oJqt%&g49~lZ}%scm@yGR zO4=<7>C}zKWcJqMvN}3b)<kD1v}Vhys7#rrkE0+y&}W0Gro4|@QiA^>YDC{tko{fP zMbdL>v>Zw+GeBiqr0`-;6bv;<AiVHcq*gToqo|am&9=Vg1QR3xllSvJe&;7kLAlpb zjVG-;rqEny5ZTz|7$`-y$Ir<2i*>(8Ek7zZytQ1e*QM{ASW|naZA_L|$Lx}8-drkg z=zfy+>!V5bcL+@6n7n)It+%ZF;oBw}-EhMVH!h~&`;Pk~%CrzQ?!WIIx$Wj_<@qOX zmm!}#CCg`amzYhTNZz3#QW8Hz$`19H{Joz@`u1K@5;sUn6NgLTkzta&wYRJp*-7Sp z`k?fF=@xn7o<GQ=_jQn$I=^7%o6DCkvwFdZ=_cPL5mO{LF8x4A+qiL~3>`YugfNYB zhfyY@OFhk!b%9mU$U<6hiEiH??wrsb0;mH5NwFnu;dEg1O)$zoErsG3^pKW|0TG5s zz$mKYXP<p$|MZ`K{<%5IVq64(vdsvLVs3Qjop;)TT#j<c`%n@2oBF4dVHD`S<(6CI zfd?KinB+Yu0jQj$3Ufntx?o?K5u=us9;@dwP4~oLHV#!62qv8wBc`^v@TITqXE1tf zQ4gsvO*0U&)JxKHYZZioUaQNDq~@~!R)eSzNUACdnB%-#tL#;2N~x)s`rZ~#p5snS z9VLorZK+d>X?3J!4(L*6s_SS}XbURT_)*vVeuXL2%W?;>7za>C-%e^a4Mgh{e4Lb1 zOWm~|nLuERDV7e1rWstKG6IxMfGCO{il~CHsiD9u>!~l(-wRtol)BDHiYxC2po6V0 zy%7)%8pQd_(<K0iHUOf|Ke8Q1dWnW1+rW<+(XeA5PP*n~i+Wh^q!wG>QhgofKxTdf z)GNqi97Jj^ZDmk;F=;ZBAuq1Rq~<!?GV>$06{v4@1#}Fi90j|nin`NWr=1Ls)mIou zn_{Z_kEvAawweN_+FEOxDurGk+d#C^IUm3#?!?~8up)}WTH(O1wTf{-)KZxBvr}7? zBIz54Nz~vQC2!lP)>_-M4boLfX{Bzm*UST^O0L+OCi@e!rL?kE>O3B)((6iIL9y(P zOO;^@qUDv5+vH<?E$OhO)n6)}{EivYy}j4VuU?pE%4wh3F|vGLraAV`+?-;)OzCYp zc=2J!jdw?yrSQ^DCIX{R4ccl-?C_=W_S)sT&V=krGeSa@9ixve+LbPA59BB?r&voG z43+4A`uY4r_7gKCKzmbco~>huZjT<RQ`Q67DYD^Uu5ItS?#tx<PuAPtM0#PZj97j| zZtJ<qU>h)}9`(Bf*K*9+xMq@#;lijJZn)7NB4SFEo}MmaMvs=KIz1sBZn;Jt`}6N* z==%@IiV2-%|MGVvb!Q(bJTypZQpZSr=2-D&>5?`|8j^;HKUtTwk>bl9r;m-1n%y5t z+4|Qce%_<9d_)Hs^4e|k;O$q*9k*O39iMz$ual!JRTEJKQ3la~`$xXxPj^r3M*9!$ zulTHCUPMNbv;~ZY^nS=%xtszNFVsS<syEPnQgyj$K~=<sHZL#lw5B={7^Q^^uD3u` z#7&nXanwlEN}O&1#>+3i+#r*Ga$uF$adU+G?a`x0zfv>Gj07pTz$V*aTmUb<NO>Kt zTv!*dOL{Nc<!{W608!S58B=U*tfd39e4BX(`-&RMHi75!X<&3>(C&qr**D*O)5eaA zF^@$8D9*{rG2lMk`8YXJSN1j-O<no1cuS5ua9Eb$)QO^=osgPqElqHMRj>)si&WtT zRnZp)QA;N^<qZ(ccWOv$NXym5Q(?+uGxdl-Xc$BRN$N;}9qWCmUno{@iE2ukEDP%* z#T6*jeMW^d5CyIPAFHKwdTts-={5kzAQ071>!^?zBw@DW1w^Zq>#7Vs*|w>p{G^5z zfF#8gSi|rK$f_x{n$$*|XQ`EJiL_fO7hgI3HnS~F6&G7GGqN@^IZoO!>C0g_w1u(g z-`3j242zt$Fu*M}o!0-fhPusS2Sia!*(M;W`xT&W6GNj0kEzwT_ga5c)K|uS0*G4c z9dphtH1IXZ#>Mw!5Cx(chYwRIpTRaM)Yh8D)ZQv-4u%!ch-<Fyp97*^N1=|irO|zI zK-AeM$LY9Us<UDwZB=iH8hD-L@11Tbz~>Sumly}VG-8K*I%ALQIh1VImWJFUsY=`> z#pwxhJULkwZH|?BTa!#NM14Fl(hF;3z=AmWeYd6Z%jf4>{U@N;b83|InsGp0`(n4L zuP=_+Zt1ZAAGM`@W*^k`Rkp(DgP(4Y=Z9`LD81u@HI^<sT){OtuUh&kZ2qS6e0wjC z-|^vUgT_IN4x1s<Z9P}Z`_uNDsnQyS<u0Rk${%|yw{+si`){`OcGYnJ!SqSJ?Y&jf z;r-PHtJ@A2m@>>b8P~vthpZMAHTAm$pBZY>DFtBi>wLI(yWxf#ZnR}Yd_0aFJ0=sx zkCg}RxlNwD|9biO^}A&4<d>vme_!z&87#F&hDdqrU@6?+U$Ub<m#qB*B<tW{$<fD( z4)&MwgP%!F`~ay>9xlH0(GrUPT$=a4D=mkgm(bxSC3ooyGWEl|<&7t=mnZMNPF{Kb zQJFSnq8y0cCwsPUlKC^ISm_t{lx-mtxY4fS++Y;;qqk^I<<|l6LN280wkKCzT%-U@ zQn~=T6QStjh`=aypGeyLvr;F@?|>vwyL$C%13YrDNwdWq2y-D|6{o8E?z_*FQR<ig z)(DL9deq5>9(qVdj~;EQ8N-JUH(<kDh*>Inn6e#|PsU0uYWMElEd`h~Urd2eUr9|v zo&7eY6IB!_WF3qVV7?Fm05S7MY5-2Y)WBwEXWOx7R&L?Kh1O~Yus&1gx0aSBIlk;e zIWY1rNniV!G~^#r5KVN<bTHbXFJ(j(RnW4Q9n=;o=oyGA5C*8DRFHI(H&RZ4(QE^! z2#8Xz3AmV1k4J&E4z<&9-*rH=%BeLB1|8!hphw};QfbX#h<eU&O(K0O0V*>Xa%v+1 zQ2TF2T~sgyl&Q7Ei5AydtARyj)peqX;>-((8Hh4Y`>D^d4}hqz#uQj{yhRZu6*-&+ zY=A|YutJ=_5_j1Q-M1>I1`;4vXfzO2I3zW>k#%7nWFQ(e$Tc7j8y3-z6*ZM(O&c61 zg_->>(*4)Z7Cl~dL8+zo)lzd6{+sCUNg6YtuIp+-(G73QoQ?s}K-jRTD$7!PSqIR} z@oNQ9U$p{inYEp<Z7KLRDFB-yTApR~qyVn9z_EJNG*eftKDeRIId8fi#|_wV?=_(1 zI5pyQ>{Gxlk($(PlD_;si5hT?<nNpDom9~i0|#BwT%R7iRXPvZD4X{kmD;Lu@gG_) z^;^eDP2x^TPfn1PJCDeoqj}=-qC9`~0$WzoAS<J?<o-VEt!A|I@EtO8`4My0UAQyN z>PIm~nz}LB(tFv~6|XLmjsv#HuA_ymFiMRo^_r}Ulwi_mF<GLPmDl{Z(=7SVE(_$Y zkJcD$4qF;8(>EoX8558SjBeC@8K5xv=a1K#i4rO-ubH(a#l{hr4y@+L<DYMqt6p28 zuUn^}yh|SbY@=M;eVN>@P&-3kPm1x0Eo;suY?4JkJ6z+quDG-NZn)vb#U5O%($Z38 z-rQNz?bWB{?Jj?kS;L-{Lu=lX^239rE@iw_9h)XaN2V)$PL<@nV<m3OV2NJ$x$Iot zTedCvKz1zcDbdT`kt6Hglk^=QOMY}eDc;jtJiEF}Ft(GlBs?tson0hr*~_wH(sMFz za3>kk^Kt3c<tZ68=o1<B$?MYh&5n9cIrdw#1>I=B!Fi2Ld?5Us;NNm?Fv@qfFVrI0 z-&4KliHoNl!6@*9DoEk(wr$%^3qKeF0fN_Fd#&7k_uXa^gu5*;2&7UAiK53WJU|LO z0;u=gbC1=50;d=fQU8frx_tR^>sgAEE~+E|`qE1;*=u<p3$N#KW~hKSK=a8bpOmFb zml}`)a-;%B-ouzk>phd!E<8`|D(WO-;lgvFf)76X49Js;3`hgqNl8ft+$g|2c4lL0 zv9!zNCGSbhsJkU=!vJZ>+HL8%D4I>-w9?iVE(SIw)*{6~6y=OQqF#@wqfU>}Oshwv z7ih3L-!Urklsj!%P(oX4KXc5~2V)#F&xwGjf>gxN$ZAOeLj_1&Zh4J?D8R)w08!j$ z6`%}kFcR{63{rtMY8Pog0_36aT52v33yA9a3`D69B{i4v1FER6sHW7z0-}sf*Tw!? zeXByFIrsw4w52IeGgZ}kf}-@HU~=pf=o>3Dob4zyI_h%B>NqhXatw=rTheUl{pq(_ zTWV?<Pcx=QRwr8PD546Bm^Aqm4g+;n2BOvqhcTnRDu|+PvMp!gpebC_tH-`7$HoG5 zGa+D(!MeXFnSe41G-|PJi}YsQe|oXv3QYR7jhS)chKwT01cFUMcr2wF)zm=L&PRcx z)S{p(h|-cLP+w&&cWRGskhB%=OVq$?+XP0bsf;>SA}<c#E>C>6Ru*rHk>cD8soOn6 zJZlF@Rh(`wF<#c}Ns@yp#rFD>gU3)uUsb5P`i&*hduEI|_R>Bjqo`J*j}==A?ZmZ7 zR<HTwfX(uY=jO^CAFOUon*~PSnz&E)DR`FE_$;;dk3Ck%W1nx5t%vjNHUIkLOaoM0 za4}U%E2y#U3|Vs6fzhe^Esd9&PyYU7R*co4V%U_YV_`hBr`elWB(IO%ZR@(b*IENx z)K=j1zZH`E&W#N#qvv*P39dDK>2hs%-6P#_!;OnEP#}SV?c27>z|VV0&(|N6^|RiP zlwEzLDtWZ{^Ja;+aFvv2Z;|}8osxZAm*mZI{O~e4xO<jtSv^{o%<dzz#=kAIzvwCp z$G$A9Cci8jXLObIlV6g<^PZKG9S=z3k$a>m>P7MI=pj{GdrHCP4`lbOZt~_+cgb5% z-6?}#zC~t!@}#U<xx(%dU1jvr3qX|bLwsc@2jnM=5sq_%QR+oWk9A@6>;iBk6_#2~ zl*-e=52>+$Bx)$2$%29h(4lUUN*l?sj{qs^DLp&~4<2lFlC)@H8+;JR&jvJsGH!TL zGPiEsYU#Jse3Cj$$}i7zW6t_W@nxK-w0wBT^`<6@^k3e?nAip?=eM;pV1M`^QfrDb z`)vkJZ5@mcg&22XAfID_LXC-2hYlUg;ON^@jV<(e;~sl#(c2O``fkbI@VPW(?hMyA zI&QU2s;6V9<J5>c?Mck62t%I+Kva)2YA7J80A=8$KxK-kf(Sh{t%V2>ssL(fo(iL+ z^rD!WvZmWI6CQ;|OYy|1wxP_R7S+wRrC>wtBjdB)m88rnBt_Cat;c6=zQSXg)0ze6 zT+~AY(PD!x>NXA9Z~#`I4JnM8B1%6@6jBsJAdXbzh8hP%ap}d$RzVHO<8eQ2VrcII zIGTH}UP!2?1fa{4%}@zNk}<T_ZbsDEa4Qt2zbA^Q-*F(WacWo1^oP1sOqKxA2HMtS zDu||A3a<e>(95<AM6<#m8u+s1j&94+Z~;*!4-{6Nx=<8RoP$dpmtsBtxE)7Qfl*d* zIR->AYibUl%GNo@k8zVyt;g0TAplY9k4kEGT6kih&fI|6UvopQuhubSNs}H!Pug}# zTiMG=(*;J)@yi<EI+9r-uPL0}@YWLPI()rs-E&CtW4B0N)N08&8YRhz39>QzxPcue zK&OhTI={R#Im%#^l--EAQBqEoj9PKj;O}LHK2l*xkG<^a*>dwcE3N+XRBpP96<%?+ zCEXW;p$LrL(|er(EUvf&wVtd$0;AJ69XCbv=66@hlnuu$4LLHo&JXHjm+Z-qClrWT zmy@QOZ)v(+M(&iWUtgkNzC)rEa=rdDPt*UV;2KHB3KMjcVz}Xk8*W@|!FLT4@(UNt zmCt&2lL<p!l-xtZq&91^_=^`wpkkBwYYs?V<q@eVPmt>JB&n@TlB%*JQk-`{GE=rm z!l5M+y<?(mS~*PSP5n^D4th!Y^mstJJaN4Y>UNWCoBBs7+jE=vc0VJ{yLw8?zCO}? zu)pMO>@7pO-YcIxcBPDb{t8(#_<6f0B!dmL)&<f_E8r7EFhn@KTTn+aG787J!6>eV zwce0qm46)$FZ9x$V3eC)E&wQwq`2ZLb~bg2-xjQk`Y-C2IN9I!PtEhpMlnMMu$~N* zR<{;#Zq_5gHh2$kc_!N+xWG_Lg{lj{0>W*%-)9;C4r#fh3ZwASX6J`L{GnWR)m3Kt zl%Jn}@>{+Z7|Q^RNekYTxUmmN?zU0lD^7HbcEa_ImR=fZ-%^=j;Dfqo&Z*v7rxp<~ z1XM8mQMkd-#|Ma_n33K|$}Q=rrkZA%iUtH)ZKSTp9Bye#LW(FLsu0?2Ad15);6v|9 zpDCIEV^amrxmgBtmKLjl$x?_GE|oav7CDM&2oSBY7A)qPi@U6@!xTc@k5F~4!Jiow zp}wN}QeTO|5FlzGm+XL;wY8ysw9aZMO%Wy4R<~<4qNJtrI)zMne>xz_7{ZDu`v{Oz zhYN@*V7AaMMgh`kg;QdxCCAE+0YJ*wb!-6zVgu3IVgpy}d5TJ@+o47iB{!0qiy~>g zUn3x@$0Jnfq*G&1<g~B>MD>3p+%m_&pETewh?*G_ZFltfmQcL|y`)`p%uWVTPl4kE zY>H@|)y#egqE%+VL<%uRRi5mFlDgt!**D-?$=^BlOGW!T2C1kR5RF-tAh#;eUDIu` zygqW1j9nTfW0yq9n8i^tWmSwU-<=|<1vPf9KUJIxMu9-mbFX`Ax!n5RD(mA(sx0Za zKYw=4X<-zF^X`w<THBS)abGH<D46?>6}N8pWEh>fB}JYcvQ2K$?a}H5_-uty{e@OD zLzcwLjVGwCk@b*T%zB;}u*J4ncT#QacL}cPlp!$jMri}rN;mb)4L97ln1b&c+;ewq z-y%yFO_lBICrDDvWXU)>MJjTa=y%yxy)V$EVzYj4Z_@A6&HCNGU4k|H^j;xamwkF4 zv`y;ER!ec#3Q0IPT{f-iFJlHhFHb%2Kk{&gUrUeYekHR9{7zzLJS6^2A4tp2&!i=4 zpcJg@BNMtmAd_DFoeX>WavAW(J(hKc$buMhCij$Xw2z1|4<Z>Eo`^m$$_a69FzOG6 zq`cZInPs7`!Qq8o+7pZt{G*{<lHSS;`nRRGo^5dB6IsqA&i3`h*}eyr6_X>{!JxjL zu2#0KBjP@cf{Ut4EiLVDXkqh%AN)Z6=5PK+{_WrX&FW>t?p~)2YWlpTX_7eiHHn+_ zgyip=Dc-6=2`V6=RH6<79H@v^3mP`R31YJ2tudF{CKNH!GwFM&@Di-65I?P3Ff$^( z^F$B@WFsJ2TPl8qOuk4^2Q9VJ>K4nLbpfIZPPQFD)WE4ox0UKFz6z@kMHOvU=nPfo zIf^JLw}2KZq#skB5;Nk#K*-cyg(3sds%%Hm)%|IzcUrrIFgjwtb)Q4k`KBnM!UCL; z9;2rAmSbd8=)jb<KM5GHHC5v*oMk3Wrpks@WdqTOLva-@vrbK=ty6)Go~$7hQ`*1) zslXw}4TV|9;44X%Ky{(*1I9w8KBI_MDTtDG97)Z^U<wdzs5a2X=@@6=5bLLHi=`u% zI7&5UMAqJ>%+%#3-8QC4fG9mynN%>iq-NJZG{=;2+<To==~`=VW3IWLJm>tO@*0Se za$N0zs41p-rtsG09+s5lA4$}}>m_gVAe#V)OeTESz*%=&e1WB_-thKvx#isz(tZ4H z={kCc0^u5YYT#yBup?DUD(e+S&s-Vhasc36{^ES=;fb5=-lQUVbNpUQ(>)PJf8BY$ z)simQk!EdSNX4ao^Nq24EEV@mVH8*29+URj-*1lJV=lbMa;nUgm-QxORay!#$|=e# zzvJqQ5=(k7=1UkO?LG%L-4laJJ^D6Nj={ACW4Mzi`P^`$y$1JM4Z#*EuWy!u>SoC( zZ<1rhO%k8qD6zSXav-NsqOuxgUuL7k=<i1gnj~4D&#Y*cVo$Tw_*%r@*uL)BF2&&9 z4Kbm(xJYtx(j_b7sKoAHCs8}5OG5NSDLpnp0;xkKaIBvMlKSe;&m?espajxKN+4^z z1ahZ{KYyC|3a5*&c)rvZ%#rfciITMaW0^bfe(B!vN_p^>pUT6x|5Qf5d8<?{eP5b4 zf383}SaO#4ks)2~l<A#+BZD5lOgi8DTY2fl=Pe5_J3IS~?<?K7_=9^J?yJev?9ro# zf$2#?)VaZEgFhg}<+YMg68st*UdSc2I4HFZ&F${OMlU*M&PjFUL|w`^E2I+R2F(5< z)1#1HNL3VN7vm)AVQEc+iPMjN{A2mYfBZ)S(ZBkuzmk9Yr++f1U>u0QEr8Lwf<!qw z`(-&ixs&7{m@nRHCuJJPSD*#pFo#qF(Q-!_^m@V|N@{LaSaqbW2mLBb9J3-o6d1$_ zH>?IyOA4US(-Np*8%}@BdIyXEF@292^(grGSs%(N>q6Zm#n*aQMhtTFy?`icT{$gW z@|=27s}TiWiygxr)KFAP+QQHe6%f_+)82&V4Me?8y``n-hFjJ+hCsEpz99Wm71mHG zF=ev!RbUv$SQJlMoUm?FSL<6Hhz&%;CPUVCh2x>%X9h^D6F8*TDCx<jiUOjQ2IK&x zzx24{)Qf7$HuYFz5M>}r&8Lo+dfpZT(U6@ptARxgH4vpXv?5#=TVyq&<|<5GZbO-K zyloPpOnewVp@=Hrno$#qD2~6tyS|@VRTNPs1`I@*pwRs`1s4$2eGbq=wIR|*Cf#Z8 zLn?JklhowI%JC&VWk0Q5whl3sTbjQcH1Q=5e52k+Cl$yCQ}@ZA6h8mlYmH1=9WNu6 z#LB%Nua+nKZ<J-bQ>CKXBY|(Q+pn$*$cNMSTOH=Wg<rzxf}QCGeYoGgJ0;4TZ%L2E zhzRG}jj?$$c+p{V%0<n5dDJe2%4l=ez4L=L2BoN_yyh3r&y|h?Hd}wxNC2a>aH00| z<C!t$j5}K4l=q{?zW&8-=|BIFd^R`M+<Ujh=bM2Msl2x+kn$6Veqq>l1K6wISSlY) zKOo6@Rc7vVuEDjO3=1Z{m<1+1=~BMFyK!j;|I@VJWzru2r8MiHLgw1Upv;R6$b>z9 z8N9_OAFpkY_f|GY_oZHWb)i>Y*5&PG4f4rapA6aRmq~jAvNS#@+tV5)UI7(|Ev#*p zN^gsJgYBzd{G}3Hm$+~7`+efA_eg$jhQuD&ESp!1kmw~{C9wM`32c8@g4^zQmL2;0 z?v4`N_nZV`UX?(65Ah}SmbzmDq$**c6dn9jQZ{sx?c?s3FW$aJp6u{5x%xN%BCp+l zjbu)FM;cc4lhEd&lD4pq^nLagnc4Xl(*Kcvk&d@qCY>I-LuSpKVKC~(r4!scFhPq4 z&aPd%?0)>D`-yXeQC~m`OKT*p$R}wfjbDSr3$-L<*Gi=)@HMx`i#@3KoHS{YwK<{p z<)z%0j+ES4vu2r55I1NSI&F>ZQ5%a9(`A=kW)S+ffBU!c*MI%j2BLiQ|KT70!3>}H zhxxVu=4!KJ<>-tTC4SnoQgmpU_&gN`H-Hc3I0lj^W55w0T3;=GQw@(hwVyr=dc2lu z>NgNAHwDspKZZB-oFtvpQ6v2T9jaNtslRMdP;=D4aswb!UIEcMrzX{EJ~7zAAjfJ_ z9fKScQA~qS1_Na&5&*uuRSuj*#*PXKoGQ#VDeO6EyctevF;GUIQ3Fx-0i`nJsG8J? zqT-VNO6@3Vv<hi~iZrW#G;n15O@Y>)n~@ebR+9=88i+PpdM~hU&bgR2(JBYHwZ5f* zVyUw}%!4AWRTN55qfseI2__8}fVFzlDnK;DNx6-HC}5_?(hR4ll?Ap<`7L&oQ&NzD z(FQ=Y!C4pSxjxdU>D5}UAgcRUnPn<+3koe03AMDQNpmI;BDKA2(`r?n8e!Xi)K~-2 zfTdGoY-Nh5Jzo`8KU<_KBT5qIy(arUzgqHlj<fv+Qol<8qBy>i#(y9oQ${R4ATJEv zA|3m0lId&XWcr$e(skr!d3*d$S-34xGV@FITJJk$+7vFksGtw@S#K`47yuEt*Y=qc zW2Qx&25m8=6ZhIj`)xGCBl>ybpnK29>#UCRr4c*jv3{FOHKiR3N-4%e)PDlC?@Wra z8rMi*kThxiF@s6kz@UKQ(2K(^FnUe5rDl`_oW43{ml+?W=GVxibxHDuf+{fDQ5UxP zaNmu#&82%X#nTY{&ThIV2bmvS_n3I&T7Mz0cW$`R7QrXhF{%_(Ko4X$%CaLt8L`7J zJ(f4fD+|2x+?;ypIHO)3no=kCOsJDTjq^x{FFbPVXph{Y%N=9u<X(OJ;VI7RI?waU z8;c!i?YGe<llBE<!?BPYEo@TIZIMt@i}{hTUH3oc!UoqUpRYl(v(jYgym2zS|NT<G z{5tV3`)>&@y<C@nm(bE*Nod(`B(&ncB)IBI39P$8>NnjZh3lS_q$QnX$K*$4PTvmF zzsnWU{gGeDz1RM;T=C2QBllhVTiM+ACCQ%NOR856mV{ZKNbgQJ%FGuplfDo9f70ou z|0~^|yjEsRooHs8Zd_Wyga_Y)-*9fv4Mqcjkd#(<<VZ$^953{L4H7TJ0t81L&6g4# zN7E^8mM-StAAirDJ?896{U~)tr@G;5@4<eOij0FUtz<GYGf#fWc@J*B*)HQ?444)D z<R?Ed00l_-=mV!bj)MAaDhBm^RcU)Ae(E!FWX20pdUTzos^Zex0!UhoXSTth^}npI zvbG|j(nLoQ^*eBb!iQ@vP~(88qe>cx(%aJ1$Q<j935ZfBX^JSSB59()8tYO(45FHP zP++}Znh#-oWFSgvtj7SxTx|hYh08#Bs`dK>@_;K-QK<{X)CUkn-NdyQRntHeHPC78 z@+FA+72Ndov}2*4sHMO9EF~EupFnwv?n8zhi`LX!AEr!s3ZfZb15p;zhb`@x<DgK_ zb{jcfw88;Iam+=51VqiWsl-u4F@LJaGNqD~ThfFLXi+H{n<=8cFF{oQ-yG-MGE-A= z-E9R?Q+@%_YU_(?potli9wR&6q*Vi=3ce_nricQfW)hVqK_(D94v3oK><xpa?z1Vv zCxB?skNRBYD5BKGS}L`pe3BMimbgLUr#&OPKl!h4pHT-=ze}*sd@(RVQ&3zkhmIG> z`u!QQbmwu|8=oQjk7mfG1F5p}aJC#xFP73uj|4;1V19jj7#m@5v??lFVp2+_#_ROf z1nPjtbul@XLQ6U@j=5Xn@?@t%V^O8o(r8K7rNzwFxO`c<J45DdP0`ono5G3-5k^R? zdv9W~l-K%Nw~IS2>!-#O@TDaSDlLyucS<W6z;n*FRM{AtYhcUza!TvuKyrzhA+g@o z(b@L<?t~&KtY}*^HO?)w!EV1%H`-$WgZ04{$*ydc&BsGBN&&Q|0_5{^>*d~wb#miK z&p9nOjq=Ft3at+-!~(S+uW67mJN>fsa8ULu^rD)g#Gc}^<HmUkuJNW~)z!)VeLH30 z=x!1>^m3`4dzJW>{!yXyR$Xop|5BFQbm<_zWp_*c(g&q<!Q+xS{UM1PeWz^h^FOla zt)Ix;H~zPb@BF`{>s>!kApNntcF&D6y~ooMJN`q7nb1o<dh`aF^vwU0A&>q*dff9v z`Jm%(WXOO|O%S*g?}h(TgYUq@hYy=q3(G0)CC&{-o0^)X#?v5s<I*J|+w;Asqp8Iq z*&KCD@`_NU+TVwlI>t?#Hkpymj2TXP!ll#-fDaw+wSD^ZG3eg0W5;P**_`j-0!QsC zZDnxOr3WiY>EHkT-^<_q-QUU0H{WdSY=GC(1wdDkyiE>I?kGoRz9i*Io1BzUQc5kw zGWSamB}G$#GE{!tF)SjzGlb)8t)q701PeHx45A8a)QB3;;j9}97z9)M=qREnbkzoG zIN)Mx1OOWNC@i(sn*yTdoJ&e-lVg-rnJGcib4hhI#gDqva)T<zUDo#{h$^sJ>Zp#< zKoo-^+-1WcYN{&fueyH#oWUI+N(wL#TkSwOX{9K#24Z@AOcAXqFbKE4o)HkmkjV^- z9Oqn9mUW#0dZp?*sckh7wG?IgpB5VEF-AbtUzKa1>?oqtqh^Ntx|TVRj;hR;u7Rk} z9uESd)T}ZdtA)j|2;i<VQ=`Z+x0+Mkmw>*<KvcJf+pe|Lp;oolsr4jP*)f@Fu;XD0 zHEGhETU~!Mj=lPSj(P@&Hql2`$7pq=x*a7@!6XYPN9`^?xKIvHdR%sW@*k2PHH&|& zlRuJY9g!Q4NXjl_p?-*k-+Am5Um9+lL0|@V@Zdqq#`r!a-`sHHT!MPgRK362m>iNZ zyZn}Nd*?Wh+&ud0P}x?C!RZ)}JUX>r-dWKg!?*ioS$xoHPz!3Bt&QNhCN0qY0fm?S zbK|SQ_W;*@u3cp573Am2`UT@<>~sGi1!Mmt^>ZJV`bD}den6K;q;7F1sbBJfF0bm} z-SqFbq;6q%sax>6zV-$2&bU`RQ?8b*5tqsM7k?-}`_~`J<v;zAJbcsdWpvk0vgVU+ z(*5r1<l}q)Rla!Yf64gg|53U>^b>izTNl~2Yp2~OGHL2UY5Na8S9mBO1a;}s#hi3c z=rKy18;lb54Sw0YH(uh>Dhv!yhQ;%_r0MpK<u%BLUGb7zs5g(_qGnx;!Ob_dMSv)+ zUKT7^U^4cl0Hk~XDOAS9gAXC*MHhoh8pjqVFKR;>Q@3v2<d1*+qx|e=Ka(48ywTd| zoUYvtLCqs-)u9QGDU5cOip2HOSmCHzK%@bbLKYxOx@f4vG0m|SDGG6<?FNCB8d{=M z8BjVcK(d@vQrfI27#XZmM~V}y)p?SdTWL^asiO4tR5<i|N+sySd?@UKYf$VMCpm3U zf)0q{?yK-*y+o<+^w)n0qVx$x?L;}mut=fLjEzV^B{i2mqzc@?8~rnZW5&+<NIh<% zu99?D_Dx@7AZq(uVM;EhNu;JW>AqRJ6dZX`RCPU$DjKrdO=>co8rKYGOuEjX&*PXC zd7OGxI}W6|T5W4dSP{*3KvY4p6-4!XIL1govbPmP%{dp4*8RbB2o)0`r8X82wfa>Q zQI0z%Q70HWv2XsW0_&AZuhr&oy)MVR(T-P!x%YBDoQZ)dM-2x?QB{MqzX>{uD94&q zVp6FqQ*3{P6y|0pO*zQ9EzeXKUEsiI?=XzEahwRIERnz6IQt;0g8KMDg9gd);lpi4 z*bO%>_Tb;04o+1KEe3$&_xR<FC0<i5NwvLs)Yk#Btrvil^xnJ1)ye&n>MRAg*O~^I z6B97CG^?UXs=O^$zj`*om5hBmUeY8hjszvI`un{Hx!8hheZ*OJ*REZXmX;<*;t$Kj z5kusiJAW@3eV>$?kq?PyOb4kQd!tlMxKYX{-X<lJ?vm0e_eu4v$E0@NbGp1PwevsJ zp97?JW`C)i^tz-DzFVeszFe-k{C~?&{^kG3)xZ9wbie0Lnbzew>2mY$<&6$MlBo)- ztKR;B^mzD3(&f3wWd8iQrjBwie5TzR+_-py&li&%ix)4p`)$e-`TK<PdhRglRT$l{ zGe)8l@(m0xq&k|M?~|D164|^b-qOw5qcTcxQ%Wy7+KK>e^bke8`?gfki#gaQOqpiS zo-J>`{kC3Q=3jKGFJq*q)zPCztqnuRjveKZM;@_0uC%N<T{J6LmBg=>*l`cZ(do}h z<&otQDn8<v6`?#TJo)M>Exk5W?to}hI2{(1kF;Ea!TJhQ8x2HB>!t5yO|C*}fu-+S z9jU(F>MSk2waP#gz-!Xqg9>PYT3s4y9p%(&8YAl>9kkp@i&eO3V%sRKfT*q?=UYpM zrJfQ+l+;w*bWuDN#%T3oW<&s57(^RAj;RqQLi{wKBt2KhYz9Dnud{E|q0(a%Qy~nK zNGmo|CLDQl&BO^8T-#P_ooL7zKR~Wv)<jDjwxMG}iNqloRnk<-K&`<h>oX8l;K!`U z(swzY947@@Ck@y!MvA27qKMk~y^eEkup+|&(K^6345Hx_WK*I^&n3kcb0|O*S72Jd zC{&w^F=NqVjylY`{53_^!iDo;69cH^fM#8x86aUk<<y8e#}GBuK$TvjA+L@x>~gH1 z4PQl;ZL9R~^79I#ZnRDC`9f9TdPnOht|e}tQ8&J?;FicAa_TTorz!%XwD860c2{~x z25s>h2m*wF;hkfh<&H1v<n}RjRttJENVfG7QA+Qds8Bk)UOrmgAXB3PvLiJlIhD<} zUuSyDr@Jo|^)0gUXh=R?@00j~MtjT+C%A5*R3#=R$nar9WX<YTGH3QIdHto%^3qM$ z%JGl8NoBt`rSj94q(Z^8q~AS~*RO*le|Ehj^uJPa$6O)hlm8&q({7iV=}$=Y^!KD@ z#z?7}I#dcqy(`<^y+^v-`eXU`AODm5;qrf%?swiHlRI^k*KWQ_`aOJ^tbY5SWoNH{ zlHQO1SUNp$oAm4bvCN(}P1dhoW7&fj(|w~G-z_+Y2tf1Z%`<@tKZO&{>$$@yj=H-K z9F@)c66N6WQggOF85+;m5}#Qk8}=l~o`Z=}R#DULYDQ0vLx&DoyO$3>_`uA5m^o;# zk&==k)2C0Du3fv@zFce|dUEiOK?*VEO3cWe(2Vz#K`LqSp=EOLi~Hp0)Q(aax4@tb z_yEX!byec4%#&c*aYqp~*IasM1|{gNG7xP<VFaAa?Kazir!a^%R4I&lz64QAvBfbK z(2Ue9lA4PG7giT>o~54AF)Ye*)Ke717U0XLuP?S5QPO^`J{5Rn`v9WVQhLLXR$+>$ zfhdML02GjBFo;^{v)3AclA23hYXeTa7#3BU%4jLc^v_ff6<`!N1lVd&eDh6_#Js81 zITyv!TCXUul_i>K6Y0XOa7(?afvDG6I0n>nlA5cq<}1TosmM`PBgRM=E`{q+F+_?e zf3}Z0ULe-el>ui3OLGsVmnahij0Hs$@C8JHa|Llg)YMnqcLQx|R#AJaslQDVA4)bL zYEHoQ3YdUsssp3}Pb-M>#c!&q^%_la6l|;M#o3p(H5KBo$gwnFQh3V~)=S)k$7I*X zzm|fi*(ZM--MC1C&sR=Pj!c*^!FouekY30ZR&Ka)VpMor%oqrGs}FoLeW4oWS2xR= zgrIz=(D%gjdRv~K<CWJId8Nm)1~V+`y3i}n&h{Fl-gF9pJYNf{?4J~hJI$<@x0ZR$ zy_a?|HNNIE0#d(%HtxN>*ZRyriS%RI%edhL*DcawqxS5So^QM$eR}nj=bwE>uDkjw zdH&YxC3(O{Qa-S+l=tc-W$*Nm;<sLv>^Gm2m{;zT^<DlTTi^Y6iSPe!y?#;v0Du5V zL_t(@Dfr?)q<YddQa$lushaqyLg`qkm^eyO`@bn;J6$DL{o-Hcx?lZL-ni)&nfk~> zGWfwOWo5U2m;ImqOtyUZ6M6r^AIpQ+|4v?i;9>dfl{aMi=r7Fl=A@KPH`-xv|3&#5 znV`7K<5_Wn;CAjX8Vof`Moyuu+8iyLqmva7t0k@E{HN<C=GM!uxGY()YNsTnW=p-- z*Y4BG7=K(`oQxYc&U*1on>Nj$&g=EIi!=qs&sfO~r?(!4Mf4Q{^xDI@iNMO4j1^!g zJh(_=M&B(7GoF*`<c*f%O4_Bl=2qofO{Vo2H3g8=OsDrKMniyTGX^`Tl7LDWL{Th# z`d$Y_X~EJ&PH?MZtbr)XtHN9aM1eztPK<g0QCfxoqSRE9j*9V*fheXrIbjg>8yu11 z+M1fH;{yQAtcV&=AS_Vlq~dxU7-U@rqNtW(5XG_9TDvIVS=}Wm$q|(gMHDzhd8K_! z1Vk;(H|W%ea?JI0j=@igqh@j}am}p<+#?{`;21;!+zOA@a}`BYfsNj#q(EC;XAK~l z?tm!unysn1%}#1DkPC=X$6K9iP)oa&M&P#INu#xtXds#~DWG!BOsNJ$nOwk`*k5G| zENR)myAQ=QtcV&|TdK4JqE>4gbRd)zUCvoEZomerIga@eskwltIrL^bZE5sak=|UE zxKR#`dr)?M^h?R#G1^`axVmw11)nprEr3vJM%(j*q8sfu0JQ3c7TKB-l7X9jGCwvT z>1C(Rv;stPtD0rwv5-vO7m$%V{Pq(gAzW^EriH9$XtXZdQ$w;YDI^PGgJx*-vBF)q zB@L~BieieQc`{74<pP*KG`UXttn<mDxPTnWYc%Nora<w<3~5?)z}|=JFZHMig{M9q ze`&!riwU#z^mJJ=d!qE|e5XA2r|adnzxkE?@<%_C*YCJhl7{t`vatiCd`Le9(odx1 zjgO?@rS~N1sn=!Oqfg4rhyN%Op8mCL?D-o>8+L_Mj=xzdzvv=WlZQyvlu435a-b}F z;a<7n^2_A5U;jehzvVhv_Uv7<pv&)Me%D{g)Gj}h&ma4#ymrIo((?~D%77d1k>QU# zC$ol(u<NM{qnCV~JB)IJ=c)I}#vL)Td{eY+k4ZC7JfG>h$MPFw&*2<dwe_GZS+i4$ zOZ8^@8=A7Wx8UZSe0@~KH{X0yh7KKS21m@cU(ElID2e3vuUxrOdiCmM3vxU)2<-u+ z&o+RDf&=s9z{tBKVa}^kogHiDLH_z$Q_tvCX$CxD5H(m+2*iMgn#<+}`eKF+kLa_h z5ElTf6sr6+4n!Fg<C+S98HkoC_*GcSst*vgb|Zdsj5P=|sLgiToP^;9=mF+%e5L(} zrG;8LFCbdh+HaHb8BF6Qt6<d%qNX6Ch*miJf$J^xmkOe^Y60x*{km_CB1(E~zO#K( zL@S-TPZY;0QehQ-1N0IFL_>}uN-Zhdqec{nG!P{<7Z5FRdUvue18(*+&*>wo>jOrq z`NR~+QH8S|MO63MYDB}63l4||ZM;DRX{Uutt|^|)90yvs=<l?)aZ;lFrm(Uv)-H$C zTip&o-NZ3Q71d+zfM})zstxo&4VxWtz7#Z@08w9!O+=U)8g>dM&Dv6PSr-!xP1MT* zq9~$#7U+*!W~tdk>Cv@vaP*&L#|Qr=d7B5>@uEeK8y8>jd5f5Foyo+O8*a2^0HXBm z+^6tJO6r4?>!thB22%s+<rxV+nvoh&z-+TVzc-`N+;dS;fjFLLq5mffzte`7=SX|y zeE{m>xS)*Q>6dp`G+25sZoB6*l^5V8ZTP8K_0mW8aru#;q?MtR>i^yemqo^m;=5R( z6%!`jN86c~7kFh&V(>4zuPNV&hzFDr?B5?Pv!+jxLGN~!(VefAyRZH?`T76*zw-N^ z{Y*yQf3GC<`alYX^^=m({iRHQmVNYzly>__ie7wAQXcOntN!%3^tt5^GWgN|k~QyK zDfy#smhvwim&$Q{q-xRx{d<foc<yPr>DND(d;j~F^5Gp<$&9D2lQB>JR=RcgxjcQ{ z&!qF!{~?2~yIt0F=qQWs=qTeKdr@YM811I%UV_2-#7~BW7IV@TFiKFX854h8mT%ZA z%Qi;Kz9V@CjBO31tnYYXK%x^16hsfn(sjFKM^wCc>cThfmq;LG1D|u|%rPKEVZ>A? zDJjW}b-1a$SoW8(GIK?WG3mIkzWS=wy^`+AK3u975rfg_IdWil2RW)RT9p$g{)RAY zDx}epg?^pDjj3@M`_xyMLTPXXXy5{inGYaZkzw_Le%xGrfFz12-~M42SB0O(hDyiK zrlwH*W@_ZK)La7_>JB4{D7`mzd<Hkvlu{E3h}PrcN{bSYK{94N0J4EHsl%8G0irhU zI_ooPeJ@$q51<ox1@>6#9hY0y0q~k4s^e(!*Eryaa)|OuKT^z%NKN)R%HfwFO8Rl- zm#tTD%S93O;t*VFDZ>VLRxeBXEg(v(8wW;lzqQ_-;q+u{EyFSOdrneBgO0J1DWcg< ztt)BA90vdyS7Dz6g{F!IYV0@!Io2qioEK7RnLMCYmGhxsN{X+w-SO6b)pZvmD^z1t z<$zEBXC3!kUgH=vg=>0Ay;c~v8d0xfb`?;#WlS7H`wZw(a(IOt7;%TeX#TG8raHUI z=*1PBUuw<E%F0~R4L81*pf0plfef|qi`{;Caz?!w0zEdZ-du43wnzZ2xXYH-HJe*) zsi)as4$wT4+Y5WY!YvAHnXV(dqFJIc8)fE!fb?49lc#6b%bzFI+4j$eTIzshojfqP z&Oq}E-M`)GPCY3w4QR4ID7zo8ZIC~YciPdQtUfZe-qc_EnqJDm_uq;4iu_KQ2pJnF zp}TkQmg&={$w%+^lvkgAR6c#WgUo#8M!E0mpUba){LgaVZ+|UYpL<eLyY!H(9v?|g z-#$_}Tw!#GE}!&~(l<Yn{Ab>g*!#N3m>chscW(Zj4DED<9PWRcl#F>?Dn|E`$_e8n zYvdT2^7J!u!{tAhyMOz0dHII_ln-vZQr^Ddck<Nl{!=<%ait8r_IBBF=QFbJ?iXeH z?T<<S$DflK6DCScO^r?LxpC<PDd8{zZC;|Iqj{)pV3f!&E|;x)4$0i*TV?5n{StGm zM3UPWNCDB<)N%#USliC(&HE%RtH4r)FSX!Comm6SjQaNNYkf#R{q$4Wwr!iGy^>a8 zH}hxmt<lE8HrX~eU#Pf?7A>+dV%RZYzyPbg#5MO)zJLIF7|HA(cAF&3?J6~SN33Tk zF0rQYQTvGL4<HJ#DO8a*O8p;z))EGrCQ@@1s(h$xq>a{x)iBO$SXrcni<uQ+o`XW? zDbsa&bqw_4bO6oZ#sdge+xw}D1a1IP;Km$gN#_ms-E5==6vH7PiS%0PJ^@i{;ZkQX zXW&WNYIr{^o!8QE(;RnN43TiU1t2kL(sei>>b0K+qL=}ZE^GCs;dEZ0jL$9#B;&!b zsDkz}^z`&7*wMnJ)M=HY&!YgElKcO&_a0DHrD^u}`cKbH&o{HytTi*;)2X_;f;E-1 zmRc$ZB8rGPAc9B|F=Ng-=Zu)egjrEh5G4yr6hU$Z$vG$Y_wMI_Wm9%{RhO2b@3Yps zK)9TH?zz`{&+qK#*;^em#>9w%T%xr!6Bq?VWkyu(mjXmWH0M3#dE$&H4U+1r#mGon za}|uT%~+g`W&d)eJ(<rbk2~9&#jJ?NNCZ(TqBJXFL=IJFY0YJRYNC4H(ok1fEU+se z`xvYfwPQ@Psx}Z6Go;q+S7wQ-{bmd9Bj$~&`>$0{TUU@MF2>vkwr5K<*CX%l350Cu ziCc5tLUz!>?|!GNm!=4zmo8nB_iV<wyv&^}x_UjUzDbK@Ii!G)*_|{1B7o6^$jQB0 z+HMJ~^6{d+sZ+NnP|^(OuNfTukmBE0);$tsl`Xx8L+jyuwnkcGsYX5zAm!rhYA>~Y z&bZFJP=iCETI+6cbqi7yAn#C6ZgHsU&+fmE0#my5hNi#1=O}F*8ZsKv`kBj%WE2T5 zU%sr)Cl_J(uwm%k%MgZLJ7G+(j&QZ<1$T$u82HtD`24Luz`n;9h@UnFsTPxvY%vXq z4l|K5Zvk@GDHvU|6ou-XG;i`;#1Ec`{oMz^>eEkQ+w%ilayCHjmeDBKyZ{9|wj+Gi zdd%%V0AIiJCl~<XKpwxn|2D>d^(9tx(Z`~%^|0uxp4ipZ6qgN0!Qa>#mwH)XseXUV zvvtH}^?pit)?L_py^1hOXI;iU?%lhWKYDgBTHnx!xWse?h(S1T@+!_=yNz36DTqui z{mGFpoI$;pUWuT4S-5Z`Ol{v2r!U{a-N^XYP8DrmR6;pN%9dAVb<&xZkpZVqpGH(v z)Zfg*d>+NCYK$u&gt8si+1Xhjl(95)%i<f}>-?Os1V#f7Ab6b~Vib&4W<`lA*iuYa zR03Tah*oRh#<@oYPLCAiP`wnR9%gtd0I6+l&!uwtSPg4a0ts5`YeQ8lh>E*zwbt6& znrkUBC^W!nRoNbcXtAcaYEY-mTXIfO+H9#%)^monwY^qU&DQ;9D{iYIDo`qhKcyOo ziYXAYNi`EAf~FW9m83kOh-xt?0%M$yWy>s`dSy=Z2@qAcmAp~5{Bo9*P^@57vQHm_ zs1|3#nNH55vVFB)9W$CB35c>CmfI*!Xa&)7&3LF)5v6evO^_OCf+X{vsWKxf$}nSU zsMa!CML?AOXmz5NiXtkDifB$Nh%(oeZO6H-AeySh?y&8dkB5(&v%iF5&Y9AD>G6p` zlK>{6SwK{6OCns<^CgNXmEHovbcVS1N((Ok53$u+J!jc|yAiy)D{jtKFnZgq{mW5T zFSz(T)0La=hwIm`7dQ0RwJStdFK<08Z^5y<4Va+7lZHQS@I^Jy)TI*ZZ`2`I-KNc5 z_oo8r(~56fb&cA-1g12}tme$~H4+ii9weXjBJ{F#m(II{;nR@~0_BA9CvVrL7P|&% zV-T-nu?06SE)M?w{s_B!7YPXoIO=u;OBc<D<%l8Z^!Z2l?XUj_AH4Z{4D0eKHe2+D z_f&K28QLGizy2K7df(uX!w@7dnt%+)Nk|(x8A(H?Ab!krq|BL*oE1w@uv!7>qQywH zosQ!M!!Y{8FEF&@hd4B@H?lU3L;lv~$lAIM7iP>s@6WzK&$l~Z>gQc>L~kH&_O^nD zfjQ3h9Dxf4BXO<QXn6D*g;jcHm_OPE`(5^kAoIGPOa3DkKQC!wM1#|rGiUOm$+Lq| zR#`<AV&YSA?y4X59`nGl3%78^FB*Z-nc{pKpZnvTb~%4~`+hF22gTyll_2apei_Hk zc_Snu4*5l;uWX#lk1f8jL`Ftp^XAR+<Z;ohk*%>*OJibUM2&nNCqPtTIr~WfrL@i% z6y7&Ymgdf#D=oyl{}%zqFRG|YW(Dp?@Y)`TIc$fjwA*MdjFVPT4MYhc4O%NJTS5sL z1RA#H(u9aDpb9|gxXU(AwsUHVsCqpUrr3JRTuVCJRu^k7t#o-MR1#F_g4+h7R79C| zDT-)=0@q@#-Lo)BfUr#wm6lY(BNfqlW}Q~Zyrs-}3P6`@^PB`x&QB_^6X>K;DvBsg zfm%V73MaQk6`1Y81YybkEYVDn6hKjp6%gfDf@`A|U@fgfH2A4fAXpfy#j6O2mP;El zZ!b_TAX=+|XtAd7ih+=VPQsxqw&p63W{a`bn%f%R(t1_6v96}|ImpyNl!_?lc&Svk zf+&?qQD1rdc)TT6hpo9eqMQ<VWgeAtvsG++PC`v#vVbVTpKisE8Y=%BL~FIc?#Gce zybvG}FA18uO4VCE&y6)&Yi^x-uBiI9)R${e%x$uLw}#HfNyxad34tp*!bb<AFJ-Zn zjPJt}CQK0HxmP)sOINR65h^(+dMTz++;6G;wFfl<7@cHkK=jv~b)RLiRkon=kw7UG zO50-^BtOr2QZB;xkiC_1EZT$Uled|Dp$3IjEq^v=`Z^Tff6~*^;JR}=W=@@qxig)y zX896W4d@NaUZ26d+q*FQ^bLIU<{x0utrND58iebU>~Yo30gDXHFr#l@9GW;5QEL_= zYT*K;&Y6x(hiOP3HWi5jCnL^E0qL~4$XT=qg-aJBecEiC>SvCzpLWFXj-O$lof*<L zIwNEA8U!z1ip4`l;M3o|35O59!O?Dmai^~x?wY9kGqeyz^rYTU?CWNRHQ)AyopFC` zU%yc<?&%cK*Ryzj##!Kv8#ju$R$gBIY+;nqDkWu=h`gU9#zF_(J#pytb)2~5k86SV z1uDaniV>Y!j{BL_c#u=~y~SD6dubJ-sD>pJ;^v(sobkMk!)Ls3!2PO#XuzF&k}FCz z_UCy^eeuQrl!QIbB;CDxS90u@En5aBCnvcYzjf<Y>A&{z@ew7JF1&2Vpe;7lRl+RW z9oo0{wzb^ycg)A5>dC)JQ*UIPQKciKmzS5^9|PQ}%yLl)<^8cum)j?>{v*9{Xv!#I zv^!!B*(ewd(hQ5(`blMy&Z$)72sCU%rORz)mV(CCyh;KO+cF<3qIH@YM#WQ<MYi-7 zY7rl04@D7u45G~9q_V2EONgl|OhaX90V>K$P+E|U;;eX-WW=B>H%Wdjj<gCQs|wRm zS^N-nHC3&~MA-^3X*iTA09##Fh>F52)D*E@n3<#1TF$3}zv|rkvfYQ->B!B>KtWbA zDs$sBqb7o|wC0v-@hDUrr7f9}F<J&{O?9QbzT{#_bG_CoT#_gNCiALlf9lG)k7=lF zg*>XZRYjSoQukAvA1|sX_eB)w%4{u@mGH;5+iI$tocC<e6j24G0?g`hiR!BEN1g-T zFZYdeqGF6xr-2_8QMwe%^P;r?)6_^DZ>eO{U`au8gW8AMf+T5`Zsz=LtLrW=2*|#0 zCY6xi+L}u^rot=RP{*3>*_z9+mN6=?fx;9GuocMj`IlB<^?SaS2;p@K!ZUn#BXH$6 zqKxL;KHC1}sH+!TeBb1I@6x49aq;5C_P^GZuAW=*dOS^tZYDO0fso0rN&&6*@WnP& z{%*Bbt+dk8i0FA2&ol_66?Ko0P|z$&B~?<+vi92kbVz(wi}20+c-pbx;_)DybEdT7 zHHvxRIbC&C6+--dF@K^RM)c{4fd*YM)>I$!hklD)Hr=p##8<HD@gatEqKVKbTymHK zZ-+TJHgX!)3>yo#sZ$WRW*P2n*n*%r8<4PY5z?p5Mb?-ZNE<W-57dgco{kLXxhPn$ z5NXq9;rJkPjQO$?Mt1%hhsTXV;=1_=UAhQc$Bc!^mtUjPZ{Nn^&wAlRw_)(;ISOvw z2V+a;zBtf*Aog_cj}@H_Ve?HF*o-vCY4=ld9;&O?x5zW}nqm5e;+kg*qb$1GQX#z^ z9*Z+qd~xWMC-$GXf+J^d;MA2sT=WTtSMUP_L}nl~E>E&apIiaanc~QM#WxyfuZ4)Z zN~k@cwsHKtw`7{;=BpF*`j^MV=jSf=TQjSNZMy9JcX4qMFeN0~+S*D^DX%y%aQ(uC z3-I;z6;+chv~;{>3mNB5xwswvjO`o*Po7YnJ9iET4jjO$RjXuHl~OuYP+q|}eE6`y zDX&`b?_cM+s5XV|Y4x~GmoR$70aZEoWFC<2wT$kN7EJ|^G}~z=q*l=Jl`gu);^tag zqP6A{iXJtvHJ2dTsJYNq(paZJ{=Tj(5B25wir?6pONFgUT76qMYf8l|{w~69`yn{c zAAx>1;pcr3e%H<*!0Q5n{BGcOpdZrWqfnUf0EsbSh>4CwL2jm&HLBo=`$n+JPESE} zSTG`jZy+l<2GtcBAl4RVpfEWU_wV@N#<j~hd-@dIj~&64GlvoGe+k)1QK%}_W?IG6 zhoDE(9x9xi$E+<wxq@U~7@@LXSfI5X6VR!uGUkPfC>@3?@>38W9g5_{2PiKoKz>#t zqJq7U7=8=oIS-_rm?|ufCHI?(DAi7B>6KPl4MfGswADFRfSX1_gi!)0L0Qz-);3^* zXiKA(+e#3nqRbglnMEzo)YodQ<(NOOr6O9Eqn;1OzfeuB)qdYPuPW*;+n?K7bLIDR z3Z`>zswmpL@IdvP+Yt~g%@75YEy8WeIKh;!9kv706e~06&=W99W%TK<OI^K~VrCli ziTU2i_gY=)>gOmbqBQB@?BnX|wc=vi<FBZWR3vHk!^jsZf<M>02=|k7sX65>0?q4h z)MAWVwZP)@nkD^Pi?gO?E|u7Ns}4D3?AU3)*RNjje1*b5T1pDs_wT{n(F0-8vm-|9 ze~slM3~_F%8QdrJ!J@%kVcD$<Rt>bm`LT21Ww!#J_G@v$X)QeFufm-z+Yz^W4<gs? z#myPJaewhzq|RB2EJp>T!>1w9Y%(4Un}XzVGm$fM9+D@`z~Mn7F!r0Su+!Ja>51bJ zvSdE?J2+r;cYS>Hhd0sv_Z_g{6GLq8WQy&b`(b5ABdq9TgmqnwvGiL5IDgv}4hG#Z z&B+Nj)p;n-b9MFl7SGlAxk)jrzP|q1!)RL-m6RhoE(KR_hA5c4ial-@;d=Cv%zL^i zFg@u(=iS>fTl(a3zZ`&L=Wi;Ay^g)dF3aETKcP;771#!c-N(a%Vl=+S#yCHw_+(OT zqAOZZP>_Hphk=>#N#&A;M{J{=K7G2F3sFTSSURXTh$k0+#`ZFT=ZqOM<Zt-5Yu2n0 z#g$M?V~)_!PyyLjCHM5Fmb7^`YV(KftIfbj+H{YOM^(OpjTV|ZHK4w>T9ih~uw>3C z9cJmUT304+x-uim2o<)BN?v4ZTV_M0%o5UtS2HVmC?Lu<*|r%=En_o7oLI|q6L9>H zD{RM(!QjDzFlb<Z4D4@=L8itSY-WNXg9oZL2!~vE;<ld`cJJ7VWlNXfw$CNh<i==5 zO9~z{Qj_57vI`R(?J?VVl0a=`DYGs!k$%q)$M<f;czbK~H|+=G-bUza+zY10Jz+6? zAl9#5hG1VWls`<?K$NqkG(IAT)|5!32`_d~MV~lvB7Co3l$=f~egsfD=dv}IDr)M3 z2&`VY6x+6K!M(eI@bx@{g|nyN(4MWxP6%slvsF;39(!Y9oH*HPAX+D_vTY#RDDK6~ zb7h1}iU2ILLa9JYo3XSmYY{Fy18J$J%Wm%D=Y=53Hfb71Jz^^{L9`?bwFSwNA4`KL zf-mQKMQK$aDh5!5>c?h9bP8@1)wsDdNnljWs6-L1R3NHe2ZaRkG-(^AS(Sh&9|v8G zOIks+G+SJZvu__m;EHeHGy6}Pfl&+F@xTAN)YXeAKId%J<@+N)Ty>?ZpQC6#6q(f| zPPxPOSA7q-p0)^J>%40bliU2WN1KS-Z`~tAWjBe*5@%a!K148Vzn?$HqI$~Mo9LWo z)Hc5Q@2Bi=OHNLP`|;yg;N*Z6qXuBAi5}(;>V*r_Mk8>+Sh(1jVs`(&m~Uc<Q=?bl z>X<F?aoU5x`A2Yf(+R}xIgXU0=a97bB<`*_1aIeqh+e!MNekB@YuaMuTFyY~fGLO@ zFbOfEry$;GI%3C7#m;_%;nc$r^UQ|f($qO{u~i`2Lm!>q>3~mv^9K68^%1P!>xgNe zbjRGUdSZ4*1I+DWfaN`VW4(znR`fT(l!3-rziOpixTetYk8qx;E2-xPqpYf$I;3ak zAv7Wu9^S!lJ98cTj(Z4T{;kEC)5E8(DG2q0e`qukQZi9oRw1pxx_Wx?KQ7;lm>Wg; z+uPe)jE<fJRGJ{Mt(S2zT>PHkN!4@Pwr!$_o;h<y=AYQ6ONU5aX?q%wy{^T_$`*oK zw{D5!7YzW|9{uDSV_VK>@H%~g(aO98IU%jDtCf~oLIxdT1w<8uQVFBd`51U;ic?q0 z4e=h;*=)6zwW)sIP_96Qxs_}MmE22BVIv%gYpxg;Wuh=Q4fE&C!ABpwk1n0Qg!QN) znB-)S$&)5v^2G6&<Y=P+(++1(pM=-7t8g6W025<FoIA1|RXO+NJ~C1hv0>d>81*tx z%MgnfEksOIII4@%5gm9L8<)>UU!$JrW852dRwFQT;#e%2H4!tMr@*YgDS8|Bl>4}P z!5t-eS(;H1L6k6FUn@{X#fY!5_ulJ(gF99UWQhYW+gS;&>M=JfP|HY5!j{dO;Id;Q zqJysC#QrTX>e&tRW==(Nbf5sNfM^lhgEjaS5S6yvW(`EynoB5@c~*j}W>}<|6+IO5 zCW5HuM*B#NowzS`#cA5}!V3hoMcT!K)>dgM*fse{0;J7tu{#7!0a3>0Bnx<J*{Mz6 z88nHrZjvmvXA6ibkmW@Ny71DFiQ9Orh!RBA>xBEQfoPVrD)S;oPRKC?Ebj=PS${-! zfNLwf{^R_g)RnGu_2<>l^aybeo3Ycs4wi?j{$e|7`$cthtXs8Yb25_UMHE|v87Fi7 zK_eDjuEDT<RT9(EeqVo%HS%B;PDRwq{=CXX<48i>jvYH-IdUYddiBCqtI^mzdKC8C zk5wQ#9oJ?!VdaqF3j7>#WcYe`jyVY5smBqv`Xb_7uOs=0H&TyXN7_jrJlJ&+Aq$V= z=FAg_Tzmiti?$<k`U*T8GZUEury$030)hvRL$Kv|_**z&j$R*3HZa4|!DF%B+#Ulu zcEwlkya)XbpTPEOea!8ye&1C=sGbp48JlAJfWg=^Vla-683vE>Bd~X@8El7`V#k&( zIPZQ6K9{fJPCy_^OG=)L7mBW4Z1H@$ZS2s^&F%TZD9@Fe)QMSjbuCgd9^!UH9J~X< z1WeCd_J#Y!n>c>{hRlI-oxJFSQy#Z);aUK$``$s&ofyO>W}vjZQrdt2k$~u@)OQL| zqg-7gpb{tvSZ#~yCf~fc+J45{@U~A{f&aBu)z`K7c-bPdZrwVVnVDhCm@(M7bEl}! zY)zKg+s1n2+&O{U8w@1cg>9rfevJ7bWHc4UYi*f~@@Q1RQ(Yp_9kl{HS<<d4AS(Y! z>NEw7#y3<>nOj+&_qa7ze$SRs&OEZ^m&#~uDe@m?WA2>U(CgL(^QJo?JirrqSxLyx zNk?8rB65<Vk)8Sgr9}m}9TW&h2V0mJ>EYb*y~xi@#61O_+qZ3oVNX5u?`MK-TQ=iC zY>c}9Y~&?`VE4wwFf!;4eZ3xV-LVOYVcsZ72}RYz1Qh3G;OfQGupc`bUAuIKy`3GR zLj5&^B<88s)jWRv+`AX8Aa?;eba)pBcCJHxS+3@sOK@exO--?Sob@O!DimnG;c*hF z(f&Afco%x<cf-^PcJMfL2-nXa!PQfHapU3<q{M}zriw~r1Bc(`LpVm=Pu#so+zIqS zRFId<%~s|mi{cs@;0@37$8p2+A|k`XkeQK=no1fb6(Bc7{au)^v?9kxhalMZ8UlSh z5OMn!Le$@7#D<_w!6RFFIfq)L_A}~kFyhqhOEd3jrb`NnAA@LXyEWD87Am3wqIGC) zs8-J_m2rY-I+|%@M587J#e~<E`YKV=>kAXqF(QbTtL<?Pwi(&MhY_&s3xUy`z<r|1 z>R|NcDSoJO41z{)uXCi8u3nDfe|^FeGd*bv^ke5Q>8!i>N-cs?)p^3tFgyCR;;bt( zQNy#E6j0ZT*${I{+wbf1Ud$g|sDSuRM$>a~7ydbl=N{2f(b%|t14axQ3jHs>fsMW~ z_Sj6o(echWKXo>|XU)f9`$<?cU@EqouETlrlejhYA|lp$A%3qPlH3B3bT|O1CxVfv zzJ{;62H#od;6M93A{L%R?7Tflp0pC#qn(jxHUU=*N8^%-C47e2Vt?OZF#Dt<hITT5 zWj7P_|GFDGc6c9yzV3?o>UVCVop8Z!5>DDWz}?;!7o6;H(|H`i=8s3%LI+%*VSx=} z2f%9hFf5xk1FP&OV5Nl}wmQ#7(5+i2DlC-u)qk{e(bu{7e6p)^*REaY-o5*?hf%&( z+APEQX2aFk)Pky-29%UnBe$Rw$r*WwxSxWMhy?iGxsO}7@4+|Z9)iOk;7)WB;*zqE z^{@!VrIo0tQh=o1E!(!$wrwum>rz+Gs|XczD(=^>AKrWKJ$(G}$FQ-n5#^Y(wVCOu zNDXjPFxm^zM;uX>eINBTT1-k)5nXjtTODTW&{$oHy3!n(M`SxEXC`UlBWhTSHXq2= z(E9RRExtu9O%ZJrh!KUYB1=<6Y2L%!QT6+WS*e&aa~gW~=!)eFrX%rw7^=&QP*w5} z75Ql>%TGsTSrO{0D-as!3r9OEnDo}ivBUcj=<kcITeqNZUt<gzWF~F8u`yAoRDau0 zl#IJK&tdBLvFO(28_b_Q9rr`7qbB#B<fzL2Ru`in^8pUHY(YP@jZPgq;__KH1=IPG zJt_{eav`Gy_ag6N(Y%@H@b23<xOW#Cs!G%`DHd>Li>j!;3e3||lCXB=GHhA17!iTj zaq@^O^tyM!KvQE(u(!hK5d$$~Kwp>-8;G6Tw;(Pq79}MmVxq;F=|v0XVb$`*m^))K zwys->xbOf(hWNsD*JjL~J_**A=5SCSgbNldz~O@j)bWZybx{UFeXn5m)-_nUWG+@N zU4ZcpV=-;=c&uK&NPTt0RrR>4OLAo{wW_iL{=Po)cwD!ymw8;a8aFk4*P1KNx{tF& z^Rx(?MolrVEm5#Z5UpUAXr(B$kJRG)s<`kLvE?^IU|pPgsjxOwNwiDA;!kmN)}J&3 zBffTY_0kkg<96@fEg|y%NUq|#`uVF3;%>w@N_MCT=R|*?D%!SsZmq!RLsbH^oE?2p z#m7Qt-}wAy91m;2w6iroJCIT-9^+Ppi}xB(`7^l;Kda(rEKlkgf#T}22aZ<u7}%pP zhJ0fL=f0z`-)=6BJ1xT5$xCs1;$o~HIt82ht;NBCZt!%thPx~MaBp)U;`W3fVgDT@ z9K3_1W8t{BBN)C5eQ<U1RRqlT#GSe4aC_=bgp8esh{2Ay-fJ|LeccPY^#|ar=_ojT z^({XB!&~_HPaPD5evBR;eF1AdBWxXQkIPf%A!NY{L@Zg3uqBHSzH}a<md{4)>e)!# zFcWu|Pk@X4AUK%yg5B_um}fc^>x_nAi@_jl>~9X2r7Lms+BHN)M#^VBFZ}7i^wlq( zFR_c0Fsjb4e>#lv<-pej7vEUeI>O*(4y#wI|JYS^k0c1WZI#zFqo|?*IYrgT%&$aR zZaGqOO4TYuMqUN73aXG_T8GlACREn``QLI|?CY=RQuoo8mDP6tKlG)^7jau#pKd*U zQ5pS5U$<}HmO0ut-+UAQ{LlZ4H{N&y-MV!XDBZDb3xZCsN8maG1*79oo)s-ls*Ebp z+A;}Qb!e)sQXr}zvq+l{Wb6sys|`e{pwU=|W;Tp(p`$D_KnZ7(Ls?g%#lcV&6&0~a z{kQ`43V?D_?qiPg1oZf}Gv>^gf{@?<q$S0QYM7+JG$}p?h4~M~1S#~E2OMoI(5JT{ zR;^sA;B6^P`u0WVZ#rVTdMqi)i3((CxRim0!URzwhYlWq?p?pZsY5$anH3>0EVGn# zWzxb*^Cdr@YZyPy9`C;M4i4|zjoM1BT~+&_&>~!-LVdAd<|K4@`%N4?un+ZY;bnA8 zH6vWAHC0gEjs{AWqlROGf<gc57jWW`3p#iF3V-_JAJDUVXAB+K4}%8uL&vYbLXRFj z#JGzePP{lUWXKSF^x+5S+PR~^>)PcD5Ekr<UE4PcY!6k7?aO>VhpO9JTA1Vf$-^kg zjKlSd$1%><3SHFSe)0LI=&9cW6C7<7z)pmL-nUr4W(9JyGr@Ua8Zd3!ya8q=y|HG+ z5~L<4h;#1a)?8+ZGA4)0IHP=MVANPA0A5>?CE2e{(z?wSZEfz8kDaDUG^CQu*J>Wa zdb9}K*2{LXyw<>f;Rh1!k`sEo{ac!@UT_gYmn>O=>({T#dxx%c^*oC2s+<!I%WT3j z&sqVuzb7VzGn#bPU3j?`5n0VIToL5?TxxN%03m^tvz$MtvC%-+N{OCHE_`ud{$s1{ zJeC(^X(~Cv)(Mjaje||kkr>ls2v!br#zFg4aC2OPlTPd4X1^Ls`p&}^<NY`_$`gUJ zgAuvmE}}PuA!b_y?(d93tV=ZF_T5A1>JVI;?vG0oZo+@25B!`@!`*Tz_M42s-tGf1 z?{htjdjAW|>8Ow8U5)Vh@BfJ3{P(Z$$N&8uI=}Y`91REHnBy$?&tHvuYqukDy9-iw zbL~LN&P_<&xejSAYmwx#64w?^z@&cq==9<H7|}%!n|qjHSGWGy^Q{?n_ZW!zeFkCT zNDGV~YlC^SX5+&7^C&7Tk`6u{kiNRb=Z+T^I7aw~Km6gR!6=8{8E#BaEUkWo!paup zSF|9nyhQ?!b4pu~%~kS@mfBWsnYMkd0(D=-RV{Lru%`YyIQ?727FB`<onv+D`-}6U zx^+XGF=_tv{`>FaU;gD^@K68rPx!?zeu4k`um3{Fuhbjy7=0XddIxuRo1-kj2Tl3W z(wZp%$+k=dAp}8jOr=o`jd5BXTqQ$OB3d5GY#>3UPE<wBtcY!xqKGoWq?{m{r74p2 zRT7m#z|Bp#gSpcj@xl8Y@J*+V7;Rwz2U}~{jU5f!u~x9Nv&HG-hmi9i1i{x%!rs~( zpM3H$x^(HHzJ7vV|N7VX_@fVT>C#07k;P~z%Tgekh^C5M96sPGbE5rv>%-sUu*_Zx zh|)EeIii(XUg^EaFf3m@AMd^U4t8wchVn9I|7z{WRCam02!9VOnBjyD@BB$qMk_dr zS)e%stH;XYL>Rm4cNwO}hOn|2iJMnW<HY`LG7I~E9p1(q1%?++AH#)nr?GtbGTDE& zPhY#{37;F@Ff!_eH-7(Hj2xmqz_zZ#ZGRtu+hvRAVvLmqcJA1w{?7*^Fu)Hpr%!{S zes?ULKOLztw{i9SaSR(g5bwPGHq83>!?sOpaQ(tj9C29(gC1Sv;sXPkE6R$Ha6cSl zEzLy@K63bw*w6EXK=M-?8R4Ryui`{)R+x&YdLA39OSKu`;xvh+;qxlWxH`4e9($gc zG4VCRS=+}T%6V7?{cQP8_go3z1@E`Q=!y1kX}WsB<>~1uCUIA;T+y8%>FT)^RYZd5 zh8uMlu&3%L0UOm(&T8%nsuxGzpWDo+eewRO(t5==!dU?+oq>PK<NGnoXnUoo%*<eY z6^a+UIG!RicX#e$#p0E)?lTsvMy!By-^m!OHyV4ax8R7wE*!Mms=jW6n~f{h_t}Si zW}diWcLyPhVi2|}7U8R75wR`~5$hiyYEwMIH$8yQ+;Ci(9EvNG)aT=5Z|t+!g6VqZ zu>9y740-oceDT{q;-laE9^Kx27kY2~pMuce<MX#Zz~GLCSkT`FryQ5!?!q0ox5@<x z+YTV@;BjQ1K8u`-=aF;aG_ubeLz<f_0@f|TqLBk(_{m53KLxTwI(&lTdP8tjZ?Jql z-*Y&2cIk(8os6-pV=v77))<pVj>bV3SA^faD<G~jIeK-AT|~aVzDxo6@i5Ad*@h<V zO|e7)D4{S*fnsW5Gm`V0k@Pb!-e*Q}vp{N5r3R){Wt;zsQp&JZO2f>A(=FZlYm311 z;fEjMm%sca_}_!;|NX!JhyU-F|AKDs{tH(1|5pTV?1$pWGYUrUX|1^mG}&Uwd{0qC znZ+p}+G?DmmZpg2qM<ZhTx;tUh!QkK5v5|Pz)#{}*zPGHsvxDl8uis>((>C-n1uY~ zFwC1VQ9<5Y_~8HEg@FRBekOfjYHBLZxBdI}!hvnek#WxtLEh(KV?7$b`qi)S+u!^z zzWnlYbne_4Z@u*vmMmI;xNv_|=G+&DU%I#MbKQeJMur$-)(5w}&#1>m5G|F*L0F}F z%l)|@8H$xl=Ha~#@5o#&-JP{|Rdo!ii$(1V_q&7zv!|hh0#-WXGK-W@-gcbW+FM_g zg1f$#VA`h_tVRvTO^*{ev2Tkgq81|ti5qZzbp>i`s*#eCf&l{tC;&BrtIJM!oOg$j zp+1Za_2BNN&BGD^bJJqsaqc+WjvN%L_~4)*cw9V(iB980L7iYf1~Fkl@Vs;uqec!x z_ikNraNlkeWyYd5{{bQcJ>h6)jo!U`<H*5%GJEQE@u=FTZ(wh4FN61dyfjeKvPKn% zme8~+LsLkb8e~8CeASm^pt&sLI}oKBtUy%*(PS-thS{zSb*&&;BN?Xz(Ym5!q<U?J z@4^ppbN1WF6=n1<y}Rk^MHI8r=swP`y3*D2sG5dHh-d}Tq5FQAbMD_-R7fYDszzjX zvz*ht48=@RzvM=&xK<1EgH<rt`u+WW){AY<rV4&H->gGf4aZB`@72pwgfD_94JL2j z4#B=X`(ZuE0rSmP!gb71%rsemG2Lx&eB5aqvU9^Ot6gwW>yV{8cANO%v_&+$Cnh6s zZW@9Yq~Z3WG=wZkN9Z!OmZu|VX$me)iNr~V02~^78LNkE#fVPB(e>>w@Y(O)#wWjf z7jOLPkND%S-b9DrbwHQ*I>GAOA(-Fa5r=J;<JQdG2wSip5sO@LZ^a(OZQPF}7dNCI zKZDG37m;=D98!+EA!PG9Y#L($%TAqP`QcZX_;F{<{;WIP^@ie<{!m;tG>5<O7<d?r zM6B9&%76**Q~zGs*%&iShG6TG74W^`BN@-S3xKa?@m!J5UvzZz4~J1c*SvC7+Xxg_ zJrb2Ny{H8#3PMu~UJQt~t>gl29|**m#hPNuH&(U`KTRp61wUg@I1{K_y7iaVo;`ac z4(EdpKES{J>%R(wQXT#GfB$#9_2wI}==DB!TKyKm+XkUH=CU>i$P7uwyHuBp+R6FL z7XF+OF0F<|RM46k-%>&c*K{$pkr`2%6*V_#ZLkDTDxy@{Y6{U*SEb;#6-0|uC6=Wy zGfw6?J9qjDc4J22=s{Opzjg)QUf0B^hekgMcdw%&FBu_$w_s~C7H|IX4GbJ$icK5W zW7DRMVoqda&;$FnuR&UDDC(-HzSgL1?G=@haOmfKMO<Yy5Phh{x2W4R)s!GKzzZ{` zO~SkHye%2DV#lr^thu&a_On4jW<;<L7R;TcVDw!a*}qGoR9ZM=sX&xwNA(KW*}fVU z<c<FQ`oL=BAo!?#KfZS}jCyv*^r;gOAA1iCt>#*c&akksfL@R8*t&idE}lFfMoTtU z<_Pk=j<P}p)C%Ov@>1b{^E!6z+94MfX3v@lM|&G||F$ze{rE#n7-x%^$UC@t=^Sj- zhvJZdrnqwUsDNmb0>{kcIPBiF6Z-o4STb)qe6PA==Z2;Dw)5BGOifjkQ7&wyrGiPb zB0gtq!4*Zcrc9dwF3nURnu$lP5iW#qw(g3<Z>u88_!z?d<7`rbsDjjb_4=tR&qwOb z-SA!XiIy`;1EWSN?9J`p+I01TipE8p8|7DB>FQY*O@=sEN#|UvBURGI+8(I>-ZkPt z6%L2iBcr4RO)tS)3eVX}6cC1_H%aU4*rQci{L6DzNsYEwV8;0xge%~G8IHqWDxSd8 zNb&sn^Kf<9i+R%)!?w3Gc3YjtQF|ZEGFc7l?h|lwoHtyp-LY=?POKfa13QME#x9d! zoFAKtn^W`PKPz7?^>y|`+?t&Q@0rQC;v9>MQ=@Ri&JUYMx?`^C23T~pN9Xst;_G)h zqWgQ@(dQ!*^!sEW%sw52kzb9(bi+wFJZ1~7Oxg>-nTHTM|1iQA9YXZd{kXq;4`Nrj z;K4c<#BX&){0;@Eo43Mm>0<0xpT*OA^u*H6hB&D=1irnka7KR^JPb$Tl3KTnt#H@G z7M{IEAx^<)*5JuV8#oamCN?<GYdBU8vBb{ht3)BCQmg~gSF`wfuB)s2VKB<)m})57 zaoL8Op@8p21L$8`1k<#lW@!;Ft!_bW!y}nxeX{t%<$vb%^mN_Qt-rV!=`wrvY;@?* zL1sq(<3Iib|NFoHi_bs*9M)E&ab%-2t}X43@V%BONw}qevQmtDXk=8&IZ?K}mZk|@ zG76-vJ(qB)fv8%{k(Ab4&WJXM`>xE0wzcLmK7?@~)mm$=IQ5pM%I_rFW$qm4_vnVj z^JXfr3PVY8AxaCfP?~igC8?pPDo{XEU5e1#fpAc7{vE&k92?fHLG1l|>P<XUVpBf< z^h4Mx7`t})Jc{yj1rRTvK7<j2Okkk4edgp*wVf1EL?45wg3Ve5T30W+!)mlSKKkfG zX=`p#kF$v}Hl<q2FY`Mi?gV1tTpAd?gCmCysAJS1?z1#@V(ThZQAV*u-U&c|)4mEu z2ja#BHyqu&ML>Dh%;`vofAHApnOB9WSR3ed!?txxap~kf^j0uBVY~w(?*^f&I15dc zxk!ExiB&6?V}O~N<gHE|Z;z$(XGkk=r;cA@qLU-;g$Lux`IB(4v4WNPP~5t71WhF= zqHvcM7r^({P4w;42M$)l;JSGkri`~iA7djNJ9Z4!)m1X@N!7T8d9q?!Rjkc`)@l(h zR7B~X%+_YDEx8p$tC>TaCON2#o8iR<F?~|60lD~~eqLXcB-?1HsX&IG`k-I@nZRha z&nC$~)j8_EJjK@~RdT*Yb)~CkSv-Hqt8Brkhz8gktA^2zXKV07C*JAjYT%#TB<;E{ zL-8?}Dj<!{Zo>ZHdRV(v2`v6tqoyCT^l9pJxC$r28bpPDIgZ5li=G~d2?<!Yb{(cp zaE9Zs>6m4@8OO%mfxBZk=9zAXo&IbErUBStc@B#Q?!bKY`?Y4Lu(SVNT(DE%Ii(O@ z(;mWeN*1n8$;A1I2?{`OW1Gb#Y%)KC^}|nKq1kp!G@1_^y@@dIItDg9#$lS#EG#o! zi;V-fV)Njw*fwM<4w&zThoc*OrW}Kxvm5+p97gc$1BhI_A2G}JB6fuf;#TfN?CNcZ zSg{UXv*u!twKe7&^}+hC#<<eU9QXT=L)xH8qJT!3Iv`X*Xhc6d`8R<yML}rJkg3R3 zD_z|#u<uy78V$#~G2?LRup25WDg<hD^=cJWemX&`bGjb}qXbdDF%a;mKDGzImvIr6 z*}hAaly59gQ%H5CtG`xPuU^HdQKRsi-~2{k^sBGF!h{JEWZ>H0_a>5p&*JXR!H7O+ zgObDm#gerWX;Le5qRhigRA5o?cs@{T&n*|X)LQlXx{^#Z)s$&7qB29OK(rzokHj@M zqZLHU72H;|f@qq+E!#mK)t4z4&BVMpvtVe@6Khs2M^<KQd#D1{=HhsPR>p%+al0Gr z2S@v{Ff#0k)2B|Mvb-3jdFcrBbVnb<?)dDp&oF1!Oo20ITt)?4#<U4`=+fyc%%3$4 z$@lMQS(*e<Mvy3Y%+E{^gCORU(sh_EicJd0IeRKuqGdS>YzxKFcHtananq%GN;Q$~ ztMycc3*yzb@+5OLD%21CO^jhVVi0azI)-EWx5H4s2gZ#Xhlq%X#}PzXSy>o9d^q|V z>0{rHHMnwmzrg6^iB5<K3q(yJXD<tI>B3p`@84gdYgR6q56|;Qao5)aD;7+LUiYp7 zuTi15;Bn?C#@VV5Tg%}H@H`=TuY|(7+8W6|wX_(C{(TJKU_DZ8s|T#r2WfC{FdEgd z(6U4s;gYPiRue=OSUv{PvP?01YE#Ck8aGv!C=g9YQ*nX@`0Dm_<87fTT34olXkh|s zOS9#|MpJz)GX3|#f9V&kFuGB}Xr5fW(6#I8N>{IM@w|nGMFho3r=LBDwk@io2XEIS zwdm!UzvE-)xf{Fh-0sxF=4cgqKc~mfMdg*-XXFh%PhM7alrS0{9gWEoCS$PiNZ9tC zg-s(b;j~j6&WwM66@!mrve6psv-ZQ*QI{~M-!@F{wHk{}j>2U~1kT!L;Pkje?6LAw z(0Nwf#uGb6dtu%1Gniw#L*0HWRt!0S4I_?Vv-uHhwK#;GmItxd`UsBNx#O(kMO>b6 z8CNHH;M$~%@Sc1De$&sXbqcqp9aB(xSbcv0;R_EwR!8qI--)or>v3+<46Geyj;Z>+ zF#j6^cpHsI!hrD#GN<5SD{QtYps8wQDDdR>*@Ly;x7{Xr-~@R0vBLI#=GZuUKC-j3 zbr%9(v7(aAiwJxzA3JvJufr%$a~MusRMjHwuP>t#`nOh^dLw&S)(WIA&-<*dUgF|^ z5<<_04I5x+Xb1xX1tJa(IC$_NVq#*nxkv>S#R+~0cNs1)S`_cAL1TG7>IxFks9>U{ zlIm3*xA@qws8NAhZDES0hzf|ZbytH_X~`{173bV$1yck`nGscRD^9!wQMFCBfi^eP z;9*80=1iX~FuHE-8f2#@Y9LyoDY9)UX0w9GJO0;Y{?VjwADlaL3Ka$E0*qBT_pyEB zYIN<=MF4Hn>cz;44M1s5JPz&Mg}%KF(Dj=yv1$Ek+zq{rw4?{fPPvZ<5rNp}vK>Q* z42GVbo*b*pRK}8Is_iDpS1O-W{i1@sv1smeQASs-T8ZfJ5JZLeAS&<*qSaT<on|H7 zL-39BFjdfJIdU*=UOk5sNA}~}uATA47hk~D)m3IilM)kg=Cr%acUoHvh1Z2+Vs>O? zpohs$cDR4n7d0i>Vgz;Yzyav>=z$SK2g3W(Nff3=BFyhHrmEX@{OWVqSzF+4pf@g@ za>F>A(Xh1|fuLKSVm?JMq#|9BpN!Q@W<g&an~y(yAMd@_L1qJUv$W?(T#gyxQd203 zr+}z>4G^^D!a-H8JhxQF*_zBq8)?feP87wI_ra43D$0C)G?(#tX#-JS?d4o8VRSD7 zmVXVuMIYf|$PsDx{UNF-uRJm{mw8@!d3o)Bt*gILY{gDaPL}`kf6XX^J_P(wrMoka zx2|;cEQ;es%)eZEwN@0&=hdoetRQUKnHqS;Hj3}Y%T$EYhZQY26WM^7=WEd4<@o_A z*Vv=g2vG1(0p?{afzilFOmUcqkp^Qiz3(Qt*+(cCeSmWllCj(JCKmMHhgCyPVzYwO zX~r92`RxqYcUy*8MtiZ$>^N2qrt-ND)B0@0qJjIde#AL!9_69Fo)Kkqul04fj}L;! zlrUVG8Vb*8!SHkrz%}PvYI)=O^lP{=^D1u6ybRx2mk==LJp5+4<HmG1T%URXUeor# zZ}x8Z&DjpG8SCKgG!GliZ7{>QKPGqSiN&1^r6rfZ`6Oumr9}YE96Slh1Dq7RI;rm` zBi76jCwh&*%2Bq63=2o40vlbc?yFPmP>qj|mvOiO0|xwM80B-yIfdftM@TPvIa_jn z=wj<Gjh5K5t5ZmI^|y*|Xgv93E9Qz7E8ynlCMWYXH8q+!kNQ2~p(Hs75w7NlI$({0 z`(CIiZB;}I;{*yJ&bgW*$~j3!ztj{c5G~93&a8+a%3R7e5G~KpjE(4+TU9EVql{Z2 zq=-RIZHdg6G9HD|F0&Pk_A=1N`ZX(&6@M4agq*Srndj6%v{dFiL;Vzt+S{U^i3!dg zKY;SA7>RJ9D*51kH0I8pp`i0CQAs_{98s{Efv8Y_tXaNLltn{B12{T5VEK{-Shr#U z=FXgozI}`Z)@RO~fjc2V(#lJvv!NtITw$B5ie%38-W@+IS}+&yDj4P8I8SoKG$$KT zUY(~-#*C?xaAeOGTs*!9CPw<OvKWqAH?HC2iR1XDQztQ9q7uqE%@xa+!@_(xjCyv% zj*ZI^AAK9a{yw74vW513cpz#jOGJIWe$5jjh7X0Yp*|MQoP^z*mSe%}spxOg2VZ~r z8TuI;;PkNrIJ$o)9L9`Pa5@@6fdK-5e151});_$Ct7i_Qf4{!??QedAj$eO`^Jh=V z^CG#gY|X7M&_Glj@4CuDnGw|#>@2x(Ag#CR^+IKtGonpJ2{J><R%cN}xd@`_`Q-%y zUZ`MpX>*g7rPf?qhK!rr5V+!N1T6akc@gK@zhu8x1gC<60*SpjaYCJVEL<oK(SHr7 zboKpWR0}g~j~+cLj><fyPdkPL(H%Q>$gDXpn&?VbKec!PJu<rq8+_=TTPbEk?ZN3; zEpD6lvF=74VjsSIk|599D2jMJXcRDK6ifTZ@T?Y(!#3YKq!zt=$ALu{y%!OInGTcT zWHJqFhaJONrvwF$vA8fH74G9=v3Z0C=9%un7V|5Z*lRWVer^GykB7s&^HezKFUDNc z-B>sLtN`e4D<2%TV~g*7oShJlixU%Zd2%AIO^t__a~!;<-@}a=k!szA&y3sZtG@zK zAKaYfrIsgd&GLZn>~retDfrAhimT3haeVwbY#QZ^mBSn{$J7Gj^!vf`t8X#AqdvBO z+fU{|6U?ZHejhyBudE?c5I1xZA}psO#CjY8h7Q2(ewK(fbx_-}z?Okl@bb8VqJqMw zzBlXY<t!@M>@cIBhn}9^Uxrb(C@|Mk6w!k31Kz8)*j`**-6DBbIv&;4-zvW0vmKNW zN_B*N?N45Hqg!lgY6v3tS|D<t6&^-jM0K75vBG#Q*5k2ZQ43lcs?eyQtgbi}wQS98 zn-S$|q*={FO<^lf7ZW5Znluv9KveyVExT%)%owGMEi)@=WK@`wf~}j^!QRFa2VFKJ zFX6U8E9Vhq2C=R}GaRbSM@&R0mM>ilr*Ss$I(G;a*)amg4QhLpC3)~TeHat%t;EEK z@hdqgF(}SSL_}zS<d6;<GFU;cp5&C8^f3~s95G@9)~s2LKtCT8Wh;nMAj%oghROoj zhZY5ME!9Oxh`om$+qc5R*a-UFIx7h5g??%q+EU*e8)Eh1Sqes7;b>=t`EzG0xV;U} zt5-34^k{L;9X50b2K4U-QxjvD^ftiq`O^^=bOZT0skj$;7Y=qdShI35k`rQ3TV08U zx>_W~MPuXYr5HY_zvO}r8eod?4mNO}>?CGKW+p}gs3#6=#p1cnSh8@wv;#La)QSO8 zQwa@(64kK_hMlb~KK<llSX)`(PJp+jQge2<Do;JX)e1!GP%rT~Ia+J3WU>}$Dr-{% z8fA`FJvZw0A<q#n7D$9klLAroT&ZJ4S6#Ml^FoBokMe`Hr5TNCd+FC!3yAtH{0I-j zPCxNV{jT`><QqKg{vSMeAP_on;zac9*%O@<)E_^7T=!x9Ba0WJsQlB!itXYjPo9)x zOC^-Yy6ruI=0&Skt&)pZbXnGwu6{~!H0nWKGh71eVC`1@Obv_v(qfA$opn888YPk7 zWfPfbNJ&6+k7$7X@#^pG|Fc?*yqTnq0}Yhwo7(Tii!FJNSLY%@etvKqVTCCsQ?SFr z9cRYJ<K*}loS%@4OA|A2)HWRJhMiHr^TH&fH8B2sBzk`~9L{|<Vf9dVxLEn(gu^{` z`xIQ7lz}T#b8y``57(#X;mWi$T%7y>=O#wt?8H!<n-qi#Qvz{$S^%z155SFC0q~m_ z0RQ=Z@R@xLm#3b>DW`qv{&(ZpxGiwCS%JlaoiOs7UKsIZcZ~R~3&wof1xq{j!Ue;T zVnD>!+#jNda-|HJj2jk{;W1?nuFZGC6}xY+*RU6E7+b)nw<Y!s7=zOXkKkc;j&5%B z6)K)n2L%OT?b@|4H#h&wFiI6Mzg#maY7cg=>Y@WMZNnM+qAOkft>TG$n-bc-2$N-L zcNB<PBEoeP9^N^I+N@9o8fjWvW}9<vbAtw=bVV&rSD@KCBZ?Mj;g$AI!lwE=Dq}Pc zs;e$jAgZ9WETa`f75G%s<fcN4i=nAaMV=_9vC(1h_d1XGyI!bz7^@(#NONW79Aagj z%ytrBiwpA+ekT}yp6<wq@k4EKhU`nDx-Hv9a~}jE#ODI8T|Ossmu0yM#3~9=Syqf> z1>{#Rp25DI8?bf5Ds0)jQCgO-d3qu)CJLo_X$nLW6o@9bf@qPz75AAcE=`qUqQY_Y ztQ*`9Y{BWH`*7C%7|xzJjMImAsGn^`(Dl<ujt)TJO;6kjy^XxQhsemtz^z-i;O*^= zOXttv;GV5;-L^*ZN26~0psX+lwUs3(ewc=U8<!Ck9)hycVwv^iF|5c-L0qUeJlqdp z-|lTVuz#<_xCGw3f*V)P;^d)S2)TX|3E?*pcH3V;L?{aK^97`vX)IKnC@OYJQi5bi z_A=;+O>37SBQ{uCb6aRMRbMG_EOk|U&T=&n<%I*ziAuZ+4V|jg^OYltIRRImBL%$j z+F~9pvu2BuP)k=~_52EmirJNVov3Xz<ljfy<;4hC`Z;bb_yD<g-P^w$Wm{}5PfJS^ zr|@ahrU`_;`|i8=;~)PB6B84iGWvHEmC=C%2jWkE`V-!M`)xU<&d$y_bLNaFpgiV0 zrflEevSkaFELnnpfPnU!(Uq=#aPi|NSDkwuyIT*36V;M|*&djl$I{!<s*ZZsBDJ_h zI^$lZ;yGzdZZo#~)nV}7=X*>>9;gz-D9(<)OvQ5%ra_!n??2{!%rLdDBX(K3;i!GM zf<Y>q3Ai#j8&@V}!_77f+eUd}mdSS5^;`hwzMHVuIsm5}<KQtV6W6B|z{j~1H=K)c zV@4rv%qhh6dD%EV`3|<*T*NB#{a7?~D;5sfh?VBsvDtbr_BtNL$!X_tb-ow87J1>s z)T3B7Vj;%pjl`&~12Fs>6Abvu04ASw#;6ZFVe!|!u<P6YIH^BWVp*iccF^R<Gon9a zC5)JgbK_^@kn?=F&zgX<P6k-f^IM!U91L$GOB^#BgS{IzBO@hML<C*E8pVsbyl_ho zmIVtIJRL^)CR8fzxh>xVx>tE|5T0(pwJ+gSKV7|AMHns5h(Of-F^Jeb7<qv%Xvhhd zw#+sV<yRW%)D<QPh&I)fYKka}AW9cjW{H-iD0s_K+)*SjB2Yu1k&ID-O|Gb%^$I>^ zezYQA)K+FuN{grhS(@HRJV-;0x^1O=-^e^r^>=I$<%}u+P35*h0aIg5xhS;F3Rs(q z9-yT(McY<wiGtXN(q_xH)24=cG}bC0Dt{<*riBkP6tLyVTpZ1sxG#*aVau<8XbszQ z^Q1kO``pO*mO2_9<!Nofr3$*$ZKU0}oQ}H>P+t&-#*$RYA8jEV6X4W`3IR*LudfyO zu6!7e(##lC<*WOvX1j8=+D@rJuf*XnPNlvL40Gg|&|HW{Lghs{s3^%pb%_Gl(j0;D z%A6R~<;ThFr>LQ7U)eg_TuKm4KxIKHQW78F)TvV#IdTNXSPaLFi$_tF9j!etbu@>n zmHbo2^E8!ZX|1`fhDDq!=Im-CL9{4d?N8F<EK!MYX>LHXI_Bb<Ta<$O%0jgd4cf(z z7PfC|W>mG=;Yhi%3<1l(g73nQ6pWtw?scec%h`+A*jTwhKnOK9HpcIM_dERJ7r&55 zB1RK!-n<$43bb|g<BD^(+qP{JgD)OS9@F3d{`Zn`%L^0-4jd4rlpPd|TG+pTKO!R| z+iyl!y81!Y<|2FLek1ITRX<n5o~JD;u)JU1KhqOlmh&VYm&}qD9Jx~uTeW}BYj(D6 z_1j&ErJl7iulq6;FMQFgXZz+Y7^G*6$;LL=q(E_(r2<387`Q7SJwGu8mlbU89pjH_ z#v3r#bQd<8d*H;ldkR4F6@ZoqfL?RX!^LUI*lXv9H6u@A+3<r{XX%cOR%fw#lpB@| z+k*vzx2m-nONVd63UjqaZN~gzOEJFR1UU6|fR+Am^!=;{^xpmumLGP+w9mR@;g>zJ zrlS!~>J3FmUmGMT0HrBWd&vADi*3C322aHHk@i?+I|*y-$79Z*-k9C17moHC3V&m3 zTrwGhrH+#j8yO{Po337s;`tpvUwI)nBqZeNFj`&zNM<ZwCsnj<vF*5|`ulM&y3*B; zDYogh&5TCu8i>3gS81sg5S8C6h*F?b`xr#q6j6w~t~jp}I7?Gy#*-jYS6QTGjtYb- zsA05=0vmDErCM69KvV%A)i@g1i1J8qQXoetBp4CUB)?J=(Q;`6ZDt#(fM|j^`8L*8 z$V?>RuMI@m0<3{(o~W~p^%|TpqD0(o=`gF{mu<%~$4JnsR3J*TBAGpXDD$bz$7Jhj zGw-93Fjg%fE5KTuEcZd@T*9`1Xg)zSMcWt7X{x{Za}cf3GES)+tJ~24iY7pW>qqLi z@HjrIE73qyfwMMen~f$uCaTnIfmWc(*qwTHj3lR3{l1wkw#A8RUklW>>IAw8&s4UX z6*xAOXCpZw7F)M&!KhK@urwcro$Hq%J^Ge5E2<s?VV!xhEM0Y2(~sAcln`WeNJw|L zG-GrOMt8$#0Y$om(Smg9=#J6dDBX>eNS8DS{Pum`_wPN=o@d+L{oH%bx#u9`<J>mY zco<A(1C^bse7P5Kxa<@L(5HH((WXc-c9Iy_>5Ld#%2DA~4g(b0a-8Z#R$u&G&`|Y1 zqzbRCPHtyH%gS0?jf{Z16L|?dUH=IG3oB>q?pHJE%(-~@&nT+falJF3_39%hO<SW6 z3yqE)-bFM2K@^{6_R!V&xhw>dj6QYv&0If7(}O#S&R3uopffQG(_W%U%Dtig4d~Z1 zD^C%&9IxzrHQ?E(%D3$CeM$S_KSH;%b<|Um;uoFPQysB1*{5@-o{Bs2@6zNtan!ZG zi@q3H#hZ4t@qO1HH>|vgoFa2lQ%$HkKK*?ZD`rH|I(7UE&(3v^jffe;bEfe`;Q-O} zc+JG6+q6^SVm4cq50D(yc78@{d=j-ej6@SQTNJ)$B*j~Pr5MJ>`&hx#+J1*}-|+|- z?0goo3tXl~<~y4SYwgNC9tG8~O}#d)wcFl=ioR04kLwQ)l;^uWbmQx?56Tv3?3SGI zz}gcajd@52J}u;69p!Nd`hi5OXJ;&v-C*rFSK_15jpkkpOb%VnT&%6W4be4**&7M& zX#kizMSOc33kf61xd>{RdV?(v|MaZNqXxibjABkkP(Bz261j42(dAv12_}9q1|Vlz zJagMGGcvET&<QE&?-JBgEr4OdNB2f&wqz-xLSGoukx?7YTERNSV-ft&LQ!3MUBPZ0 z0==ZzKU?S_MxlURMv4LY%C>MhSDnW!uBHTH)@Y-ldD{k4_OEyYY|9@==o&*<b-|P; zu<4TSn6uFbk{un{LQ#|$ddg}xc(`C00^(V!CWQ@$d~#B;8LTr5;gp3KIm}Izs_-X# z)nfHv$6>~fzzdf~nSvlAvN`Hkb$SC0tAbe*#Ax5y8}0Cw4Gllouecyke}lwPIMM?N z$YPjc$a;SEP(FODkoEfZlHK!U<dLxM&}kUMT>-H3`SWlje3pMpF&Fo$xqz!lNEH&# zk>g1sdO?2PJb6}@8(k-^$Iqa$=;n~~mPAWko`htXY6Z|-RvtR_I+3dW^}zu1fuExz z8@E~ej||+9ab$=?5Rm~i3>^6UpZ)c;+3=n5+WmRV(8)sAQ^Olo8XX$rz~>S3CvBr| zT%4Q*E@?}1?flhFBr|jCIua~D*=GI%k-CILL-0tSF1dFNPT!3_a-d;PKQ0BXo2KwC zUjGY@C6LftyJ;!Gq;T;~EJ|n>9@1#68D$>5DH)(!7~ahkT5fdM11|g$GNVlH;vBe1 z{edOEuMtw2DEa5g9El@;WG9zI(RY$=j%_Io{DXRdBueL8nDt&sMQmSBon;!mm|%PD z;rkDbk7L<&Cn}{c!D^r;e^KZU=JgW#))!OQ3$BH?ZCV)LQ5KylgXjJc7u^_!@R_>s z-V!mq3hH7g3R=c%YfbN+aVPG5SbAI4l`q3;>gpK?#65X!=zNH$d*iWcfv)E*)H6Na z;qk+QHPq&MR(Vz*)NZo+5v%m)&*Rq_Rc5_gVl)9{(=#&zUokYpz*mDr<_32MzoVWO z6}rdSzbAqp`tp$*#2I|pFoTMvmK)$Vdd_<kOa9tpJ|WsBK$#T!mk8qotIEZtE!s?w zz;>i4bPA~qpVcFFa4CdWu$^slT2aCHmkUFKe{zoVXS_{zR?XZ<W`uHmgS^F-9<8o$ z#+U*Peic;7;$4wB#D+Tq3Vo5+g2J{n{Q3eQC>l7V7n4gf8Ic#8RLYg;K!^e^%fabQ z|HQ7sv8jQ=&?HN+BXuE#rT<UtZG9z%zLEaiHRTd}g-m@&^!Mc;kCaCN0+(rGU}%mw zGwLeDYegpc_@^^QYR0Sz*yC_fMJ>I!mLNryeFSH<wDB)e)uMx@ivh<8&KfaZbDnz~ z7RH{22JL+q+5RzPXl4{!Qe&~&vJ-OY*NFqPMhba3n4Z7Vsd&D1E&n{snjnj2ob~A_ z+O`Y311_u@j*9?Nk=&-@b_*ag4C*rURSLh-Ekig23MgSkCA*z3gbG`f+FEdB%ey|G zhQ9u?-6Zd)K_i6wkJr}*c?G~LAkgP`+7bC*-v(&W3xdbCSNGrwF&jHO80O~YYxjVd zA!p>V`t4cY>m7r?>3<gq&pw%CSRZa}(I;|4O2uC4n3-{}AO1mJnOc`AcbKR^X*SI? z1gZvwx11pi@oO&KAuCsl`}&a$zZwjX%2T3MC*ow?fY;x0clez!_~bTXv=u*9$C9dX zxP~}sNk*KM6Asipc1l$b7@D$tthBvTK)w4qFflm!MQbsEGEzCf-s(!LaWA)<6eHGn zRI<k7w}L1ox5E0w7+;Tj;b@#$PWA;kP)wk8W|*q2!nJ$gp7c8Qd;rkKoBf6?OhDs2 zNym3l7<~m~+B4jv$do@T8$^m3B=v$p`Xq`iZ{ONJEwI^A&<Ha58-?i0#mACw|FVvH zvZ)`RPM3LpF$YXCf@Q-<d@!%5V_0=w{-jg~zj!li&-o!ezPB(SGBGuK`svI50TU;) z&SXM}HTS>h;vNTiqOH<5o$+N3D^?IZ<3#gR5pvTGUxmGbM6J=<D6Dk>!WPS73{Yv! z6d=VA?yMOwJ4@NGMv#J)_9mnoglgre=j6mqZO|}p5+dIqs8DaREJ8(GL-^k$)9b7Q zZVx6vC}9K=2jj$oze8@=T;K-}5%d@ml`h}#AV4wsWlT7FW8?hlaHt+Z<$^_WaOpP~ zrYisXvYbNlIpfasfwB3S%vw-m5lp+|cQ%`VpfL$JlxA0+s#FFCriW8edtCokT2BX< zfiRROX)v==L)rr6qY96^5<;-y=thJ6B`(s?f_<IDlH71zDddG+NsJexokY9@H&;x< z_d0_Z6-OOl%=S)s^Ae#CHtc{8N3bk_{tFSX)Mn(I`7)JR`onl31w`_+I`l@7U~aV7 zr-D%+nP6feq_xAu-Qhv(G;-*R`zj`Ut6~(EN2)I5Rz>Du->&K+OL(yOG@@s5wK(Gj z0JX7`2hfT=ac3HPI*kY>P*n5!`ueK;Iz;lj{qB)x{bGu!?51V{0s;-f7N(CITOP>Y zKuGH8)><r8q6n$YO9kRhVVoVzm0Q&hw3wo@8t;`PP5Z<={|6E|oOdzLfqQ8YQn=+b zQ6ceOr6j_tzi}>jKHe?FK$mH9b9d-3Jw99lebDMu%tsYS;<fpVCIza%e}F+61Uqm! zHE*w(F>$WqCnlf6-+lpF76gN@Y6mzGDU0F<jQGnw)@=i}7zsnQCMX4!zx-dF9x-aP z>J!7vvN98pq=@_C2{-=Y>fqZIt)EPiHhzk);`dtHJo-N!y$PVG($lZCPJTz4=^E<2 zpuBgLNOAejANMe!bD%7~!$U7f6iyue9`iCX`1g!n6)L!_1fqkT5mcQt{4k*vMC-s$ zO7+r^euI9+%bS6)S`JLI;{Z@Jk&uwM%|Goc39Y})o?f5=EtQRk_^qv$R6U)*Jz`N2 zi$ppe_QpX4>#I?ejXS9}N{386Uo=YJ2c#WWLVz3yg%l?nKRd&!5;HuKJDlI|>$Aa6 zxkzwF!w#?%Xq?MMa6=9(g%R4RPWR)b@yatLbT30H$28t!!=~37mC!hmR%RURA#s&0 z=<9kyI?Mh}LWsAOl1*}10gx}u%*xPb>=;Uz9S)382|J7u_9u!jLD*hJp&o)}dxf-+ zy{I2D)8Q;*AVK>72r1|yz3}()LQ2Rv06?87tt9S;i6LN3F*p$u&SdosVs0FTDJ50* z)rC%FrzwO$AD)}wycE%GX<1P}R+rQ2@<Hd1@_?9#GQ_q~=;A!^CEWRLN3C2O<eXMj znR@*GdF54+gA`MO;p}fK!5IugYU@AA(TLh$2L$F&f~=r?_jN7(vH^$7sFhv+Ixeg% zz-P81Se$lW${5g}0mjI0X-X(rfaG5<`@)xv%u&(MY8_{oL$oPR+s_B7_LdtsxJ_FN zbPfY;rjmbjcgq-klfL^+BvvFC28RwWO|s;p`!pc!qcq;nU_F<X9$c+Vp7qarA&;{4 zFjBZgsqbd;)v=7S@peA^F{|iF77!x;2&c4;hp4L7l!o$(&ud3@q!cK0{97HkUVXsn zE`8<sT%!H7>J(~p*wJvLdomGIRW!(qe*d!f{WrWnuNGmZ`PCaGrX}elmDIkN2=x!W z8bNREuGb>Uzj|T-c{Egpl!qP`Mf3u=W7`Y5uh_2*A1t~Wn(lAF6*Rr0U@~Wf=(oU= z2+8~#%c?2V%ZD0T@u#PN#h^)mqTj3D<4)^De)nFTx5@65Zo)L*16(%dMo`IKfi`@3 zT#CMS-~M(fjh5PzrjZ?-#}cWP&qSRmcDZola%J%dCi2G=+n8KsQi|_PXkA{uj#Sg< zF>IjV&dkgl9`O7?vgycMM#m-fAe2MqVjMn|bU^x3doHEw=@&iSKu%`JmpG8pUxWC+ z^;)y+UmPG7w<WRy4}eL!*xI$^Zk=S#3na!v@}ulW55u@KVNKbd)rdnH4)_-(6Mgta zCdoQ--eq$%<N_FcKjb1=98|$u1Ojwr7%#p<csLv<b;`fRJjmz@KVcJ$k>%%Ng4zWM zfQLenM(i1=@VRL{6(z<BWIMOhESyUlZ%M#QfC}oSRJN>2XObwlD;JIiOs^`_A^#Rr zpeME9nmdp1_)*vQ8p%+2{U1Ug?r#s}EZ$w<l18~jU5G%I^H=~O5IfS}1VT$9IugtJ zJ>$48R`+}6s(9YdZAL@}&N%i^_UUgA!j!tw2E6oWmba-ch*cWd`c{)<8$J&0A@H}1 z1cl?*JeUp_l~M3}Xl09fC9}5oUx~T$g_oNKuPQSe#H>OE))Ed$DrU~2$@p(2>M6_s z7Bf}V$n69BDMq?S-eV@<J_?-G=4xxpBquLl>#+?oFftlm&F8ki93t}sJzO7icz%b- zovyYp45bb=cMOjP&Dbs@`SUGva%G5#$v^w1|FA-p9S(9y!2FA>d}%z;Qm6`n<Qt=e z9~I2EEF#KIys|eIN|KHD^OlcsMn~8ciWVwgy^<bR6@-#5GNq#8&Z2`n4ncAZ@H%so zw!dHJieimkQO5iXNVQD6v(2u@9g;n?IHCz4^1MHeK%6r>Q*NT1Iw~~t^6A>iETy)6 zSY+d**eLsU^(A!SLR9DkbCp_h?r-=0IgRMH5B9UP-zkdeTuz~%hQ5AehOcBt>+Y2w zq6n$e>8QFfYbYN!)#bK#em}nSJG%!nRvn+&T8&UF><2|+et0+mjuPu`wrBI5*=M(> zb*UR14<>5>I-%6MQBb&-=D&5XK>O3j<C$$9s|m8mGu?8-y^9rm@<-|Qp~A*6z=~%Q zf=M-i^|z9Kg5-&??2B21F0GIMBFT!>QN(uM)z{W(0*st0Z|uZ#q~zV3JOnrIHBe@E zO6f?b;g>KK<tpZ(w)MT5!s3_eGxuJJRpWM=9+Aa^N~M5>1G1|z!z!~quSwf=SCB#u z{*Nn5h^&!`od-!0WpHzoZ4FJTcWrA_VpUoaYSf~@I=#C!3!|yw=emLiROdMyIYush zdNaEa87}%vMxjt)>kJu2+4#x69xsj16lCsC^njNVJbGSxj4O4jN!$LI(KwiO!|GB8 z3M@>WhR?zH>5Y<bvg+M`ChwGqIsDN{`WDQkK5eoz9zHqht5Ve3D)S|ntx`K(d-Q%# zB@#$y=*BkQf&+9MWSC^(vPl2sM9gB4DeMRUV+7%dV)RxnuNQPesF4nUhirutc7@%E z9P|h8$89MqJHj8pllCHxMVb;cncSsA$4TtvA)-Bv(R9NQqw)MqzqS_C`S+p14}ttD z8Q(%!^J{S^mmYMSvf$s+!FEMx<zj@Q6?IM><`lBbuxaNgwi#&9DmPvZV@hi;$pH8C zx37)^RRJlrkxr$;#j*E<z)xR8>=d%4qbvkSnvI}-E$YS|W%Vxzi_NoSr^A`t5)#2z z{zYb{*v}-VeWz{GZ%?NvjzL2L{hycw`Nl4L!d^~af0pc6MNwHmD12l2QuomT3N0Jm zM+E04YRzX5no#JD{ru@+4os^e(=$^Libr4K{d+bt7yDt&wOfKmz@Lzsce%daqNtz$ z@9?Ze(VwC;NX&QbC8}ZiZ7Fq}7pkLkZvK<t8}-qUHWQ)ze9X9kW?}Ol>u%+23Zh`N zzEi7HCFu$(chd5Q97xSRi#I%XTdpA9%W`7PV9^>WhwOq`6SaFd`trI4l0-NOgnBrZ zNsntZ<4q)zZBCu?{%JIfLg_k;@i?vT`K9Tr(tlAr{T@5LVm<bN4<Ktgq@oF~77nYq z$Uj*;3uJkt9ZWeF0B=5ezm#k?Lon$<j32a$TP%I}+6|+~M5JM&zIgh>-E*?k3_!#J zb1O11ZgJDoy=#Io`&fW<w`d~r=K4|T%n!C@LW#ayf2w9WX7umrx4MxXw3aFJ;-OwD zmXxY|n?Def*GTJoUrAM({%Xq|4(J`j{$CvOX;dcfhIz5HDT&M@RvAm+mxdTUjI3Cw z3MD5)WbEaX*qjz!0-qJTOz#kKk^C%|rjG%lgpoS1HB|hp_(4QYC{HVXyR1g&jUefF zoDylsCa2&V)`$PPEQ~U8)0ubG`#mgmOp4Kf3McglB@u0`Wr{Ub9n%eBVZ6BA&*cbK z+Fkqm3dN2F*D_{G$+1nlAOlT#T{9Dx?=L$YWjO^|tt6g$@<Ax;p~8SzlL4C8i6S+v zU#W_CSs%Tm7E_WF_e}Q}u#(*OS!ZTk8v!}xUCa4RejH^_#!7n!E0+KlZzd;MM}uUJ z;Nc%^D5+Db$OMUTruU<i+Csb845feu-oJ(w3Ta69knYU--O$qMaJR<rw@RwBnP5Bn z`ZNm_tOinMhssZjy$aePE=j+fIW&j}NVJ67e5asvWiaD|z?rJb-p8HO&0k5<bRuUZ zG8YL7iigo>j<p8p(R=OXhj;yA`75+K?o?=h{nZf>$ploiv^-~Jste_WvgaG}R1ws^ zV&umiE5TPED`2BjJfTKOW$fi=qDOpovCo9t#+rkF%-TF$e{;Q|4y1kntq#OnHr4FJ zIN_+BZuidGIMckRxM+3jYj>qtlsaN>FH+nb97SKf69+7GMGuzNzOHpU6RJ4%z*=uK z4Fd#a#9u#mP>KJ$58w={W{_*+gD*8_YREBvPjPHf>xety!~R9BxY57+So4_gDmq<O zT`)(eB;c#5XtRLb{O``fCvcVOozz~0eBPy;RvI5NO^3*+CFfynX!G)$a`V}oTy_S6 znB)G{Krc#Pt=}E=cD+4r>mO=Mgv01AqDrKEnN#%2zqSCjUaqmvi~JHsSf6_7(Ovh# z(W`BFcLt&3Ey<y*j$PWLQb?ak5Ky&nTt`MnMo`l;jf8+pKP*w>iOZM{V$@CGPp;1R zn*cbrQ-`<DF#ME-&Y@3P5^6vOdrtvcCs$9NUET6Q5l2Nn?=@;Fnr4xXsC@+)sMr;H z&||iA-w|RH5Sx=<2r~G$3TV1x;$Q$2Wlm>bw8<=cAt*aLx01ZyTVcvI_sTC<WsHQt zw!<co9F;FFqajr&7WSUoF&9X{B#}k_-R3t!Ek5KbQ)pV7$v9rslZKlha!W<1`iR1) zlx|fvpK-tC;ET9f6!7BHcTeu!D2pFMSeq4VOx^tA<<~!x9oy#M6ygA2i*$pz2dy0t za$tOc4jgb$riW_~*g`VsjO6-QD#^-mff>G>4LyQsN!Oy#tb;d;K@cE;ZEFpi`5&IE zd&=8@8~fs{s>o{-v4_qBsF@WGk51w(<s{D+uMQPH?UteS2p;_QH?jE3Zpp-JBDNX@ zhXef+$^GHz6-Ld3IxFh^^Cwj9*8NN-+SU)j){7En*`W;#gl{GJxrtT%j{AB~XwGi8 z-c(Jf^;RxrH1o1y<}&=NX*uQNbm_u8d{;5F98XmPe0=k8&gqYQzz<$Z`^U$)ycD4X z200T*U>7c|;0>T)WZk@rlY^sbhuRrcJ7OJ5(1@(O8kkCW*qBu8aBn--3jaWnfQ3;8 zFdf>q+?^7gZKdsWoxz)Vc-5aR+s9WN@S1afnRY!PE5h2i@$EmbLzpd5Yi8Q*^4Q1- z(akevL0<{CDNla5>Ifr0<wsYP(a&tlv?PlV4mxW{yilsw8WXE-W<#c86a)wM2Wp{Y z{YI{+R%RpFu_cLpc-CZIaJr*WLi2!>8eOj;zFx($Un>M}&FgqI9u*w=I^R1MrV_`b z<LPJs^g%$UAfY8WbkHX3AN<<v0FZS&kO3Qv4!-_FTx0vlBAY5Hhi&QgY->~&cC0@< zpqs|l9Qz9gHXLX`OHnMgO9*U<7af9<?Qt7e`&I|Dh8Q15O1{GY8uA!A0_g17tKzDf z?ZRHLS(<$V`__t^de7a#-V)!)2iq4UBOlndQ7~~v-_Z*KG*-t=ZO&aD5qxwYf@w$J zx2x<fb=mYf?zcZb&WNwp{Bi5@{$%!#Il>5`DsOGi2JsdCEB^u<*z(A1<IbIx-<vF; zrsD?%0^8YC&msk2DW&82mM@;cg>|C8H=cjmBR~rG>0RAvMcQsm=^$$57*Kakh*d<N zL24o73vt#xzDKu3sVnKei<?}#jFnAFqV_M0d7f0SjtYK~*poAaRd**@Bsi0-*6h-* zLVjA)yW!0_Gq->c{_$dhBlV!w$EG<f@=y4I&^JeIXd`VEL$`XKyoJX&QAylKttcxm zX$N+YXzp&gPd4oZL2kf=mD{e%nM2zv>3Z{$!1d3Fz7F@m3LarR(fGumqnN(RE*YO2 z?5mBx3Ya0OLf?N;M{HXihTsSfy+usKE_<ylo_r;!dLh`nO@7-%8x={d$WSn2(6DNN z{(*);t9>)S%HUe|<=v<6V=|@clB*wH2;b0ROyC)7nLrrxnp!i9eLRI38PQ+9rb(U( z_9CAvJhV;ru}ztc9n8+g(+Py@t(R=T2-vv9RS-fmRH%TE_=YTIeW0KwU5X?}VIdYn z5tKq72$4y?N<B&}r5knS1b<1Z4k^jO6fuG%x|R!5GLSgox1qu-7C2Q?uIW*`NassQ z#lJVoeN%p+Zer~zL|~mo)}|Z#xga&!0_m^FWx33CC`2tthO~wP0<qynP;Q2}_p8XZ zsX=k9LpS`RWENUxiyfvZQ?HE%20=++(Y1-oqzh@j#`CW4X-IAX{1jrTvnIXm^<9>l zde=f+Q=^};#&LlA&Kw|3wJ}?qDuoyVfxkcRb@_s|CpiX{wz7>&DS#gM!`M$pkHaul zyqHn#A@&L5PZ6`>DwnB3)mE1fx<5*Fl27U~$&N!B>H5124Q<}qArwL>SUfBD#7ww9 zoal^(sP)G?fWB$svr=Oi*f^kt-j6@qa6my}Z=@SPu+9A0$BXf|ZCIe|YDY>~K9~eJ z54LjH-rgJV37hc@#bF6pA&yyv^?JaDGqC=xCPhJk=;&y|ZPxijJqYB7lR0uGBw@%f zJ~f<jOZZM>BmBW@8QSg7BOoVG9@r=noy!zhRJ-fs+5WNrRP&mz(|41<^N5f1T^m<# zpJgr<^(P#Wm#2OA@iDxY6Jb-Q9a9E#yD#)^ZfS}Y^FB{pzA&LUj`Qq@-?D0=bw?E6 zOHtGlN#$2<orDvtX1s|z^jf9%$45xtqNa(*%K2BbU3LFeh)Bo3%HSSvF{^yJ88`%1 z+ADR*k{M23BgQBf6pG%L0(jtPe4X8n&F0@p!|zO#WZv>fKthLN%7ME+1%c8_-l8AW z9w;eFOW$uB=wA?ZT_k<|TI;p1jkFj9z44;@f}h{&)e)OiT`9j25f2mR#~92e$8b3a z{raMGV^}+B6CI2Zp;pQSWJ{;FP-fKWNhuhf4px#8G*>MwRCU6T4X4X2RbW*|2g`B# zebB>#L8ipggyPh$AiU6GHo_ShQx}_Ps|E)a_3(5;6%qZfsUUWtr18D1<~zdKBfHYw z-<?=m=-_$8dp})hKraFlZWqEK$BY+@<%8NTO!<#kNtJ?1Hu<E5FvAPgG6%+L+zkPk z*j5S7(1N})sn21<QlMGcEakPM6aXez9iB#`-i0Ighfr4P&j_e&vZQ?BNphkl9}XNP zNqHmp!D_Z~craDyUmMtSoT+>ITtB31_QZ$_H5uJ2L$&B%U2aMQSWU>TmH*HO8%}aJ zuE)3B;(_0iJ-Z~H<9TNS&=VpU3r%qyDwNo*V-bQUXEJ~s%N?o;;)jf~aa1h{wO#!a z{9=NtdmJ_Zm8sWwaB9Dk+BNyoQXD=U;R%eank_jB&<VTDP(zDDgT%|6g&nJ-T)B!4 zNW|f$T>&vf<@f9#2O7zTOrv%Oo2;A`&s{5|0f~~T>eqMu$ls~N3?sJAEZMaM8{-~B zOMc7S^0V&4(rW*yR%-(0{Y#6dA3U`ULn3Dsdm$x>iUfj3FElPqK4;M4D<-Vpbf>xO zT5CRYHS;~SqmE~t9ejNAvCRc&k>`!|XvHE2G6!~}2i<C54d$H7HuDd^e0hJ=)2bQK zQ+y%1k8yyX$|GKoqdM`kdC9dTcCu>QiPm}@Tq<%bApb2zI^)+!*Mg~ZInT3{yjt9? zPbli5YEFdZ(4M1GR7~WR^5gd^JH}2kB#03P&Ief{kB6GwH$xxjlmD;$v|axNM@rWZ zZFls66G;m~SapyXnNatPOfo@AehVUXm$e(d6faxeX7f7{0B$hBOY;n6{r=g-f|0&# z9&lrd(Uyd1Vp^#nPI9)mhjgRht^nA+BW<L%%wg4WBbnu6f|&So;AlC{Xe$(cNaF5@ z#bRe={H7w9V5e|oSr)AEDwG`1Py<S_Qy!Rp56t!_nXF~P=}FktwHElXz4coaYe(z3 z`0eYlFVMx_f_brp{dSH&Ai+mv52Q(6grYm6Pu&Fj?mim<lRH0NnmaNT20@S9I>dxT zj2-9XHf(;+>^pEWgSU9InWi*6#=YnBp#X5GLu#XRmhQLQ3;G8uH+@j4Ve!bNjrS2M zTuu5<JAqLSNR80&j|3L1=*K6$+K=U7SNS9d-aQVbn5E92bd0Szx41sVtk$@f3{@)N z0AuR!)Y@9#+>kz!{=$8F`Cfi;d1q|cqB2!!Z=a5VG>@(4iwlMWg8${|_(ZQ;*Ut*- zAd@ejugg98RAdi^V{DtxeuOu_8KQC|Id2^xHg$7zQ_Gd07)j$HBVv<GQCk?<)9v1y z3<G8R+oav=m|iJ-+>A{+u$vk#e0yGGFXt>F%w-aX_h`2zH&F<q?j!^BAx{N1?2e9j zi5PwzhsXvUBp~D)ITj^ividYO-luV49k&opg-H_d{S`$WG8yOqFSz?PlU3nKMIkyq zZdg2g61~6Ul{gRvcTZBYn|tw!XBKRnyZRL!;D?$QC!(e!`Rv6>qgIDOrPyfxW09&| zVv`fmS2TUhbo_$Epf%1a!|3%-007|Z?5sPAh~-O8&T%K!)q7sb97mA5$=mj@uLK!( zGID0N9k!YNB>@r4DZL_`)FFJI4&;XPf+(8)TXyveXW!9G+h0yK{$h=4ePtZj9$c~p z#HwZg0Y~FFq!JKB9u~~X%vvSG429Ml2PR?C8QiL5i*Hz9f0QA~AQ^4dVK41aR)!dy zek|nduhOLr0+N_uOVomrCHb-8Ch-6;=(NJ<R7EbYd2o9FrgYY%Qo1ysRZlHhkYMgW zdVkq?z${ac0~sGOSt4i{17mk6*WT?_%2Utw+|Hp>k*66m=*ECMlWeP;roc*;@pZJB zrdf(4oxSCV;1Y{a*;yeG%2>i*LI}7V9yf;8+<MH`T&)0?sN&%a8yz&UWxG@Krs^rj z2+}NbAmfP71OTIeW8UPGn(D25_egFg-*vM1Icxb8w?C9k*7`yf$}KxfKY<aK_<L2< zo)FOBc<gYSO$rS_j!LX?utk>C(n)md)>D=KcYqX(M5V54YJalFcmAaR^m_L8Sq?iu zx_SbHMqF3lBLBTg|5%6qD7?Gfp`eMjmk{?l)yDdpq7-Qk+W<I-p?Bt(kS)Mi2V>t# z?eKnkC_!}s3{tpi1tW)aeUyBn_^gdINEAQf$;eqrp@<T_SRbciW!;|%D0wbWK-cwx zEIzw8g3q(2>%t58>Q`{dTVn!#2Q(zu7-5_DN`4~f`AzPl-sbg>$7Uf5GS`nLAr4o5 zzrUDb(fE=FhQ03o*e!f}y%ub|*45YY!F+3vrO%^y4Xgh;opVa@BddQ#Vv!90w;2Z$ zR{`>m%dh3%Gx)`{&R-o@$TWJ=uF_7wmI}M?v+tJ4-e15q2EFdyXLy;Il~b4|E2wkc z#v>p=q@bX1^19^`S&XSB(OIvpJkDnE&<Fl-kbBP>7nLf?RiWL`6mEs5@l^eak@1Qs z{l$MG9|<<CR-p*=1Jt)rg^GMsko|#e5Iw_$S~TZEskI%u$T#pczLfAMQ3=P4j7Cqs z9%BE<SeZb_f{|`msR)&)$eG{!>wdP{J6pk&8hQ)sf`>|VG&R>qz*HS^h*~5nj|Xy9 z^}e1eFiZ$`2zilMkX&~H(e_+o5R_H(EdW8N^F?nd$-zt#959H5st*x@o&}~iEd$f& zM3<7Il!fFd;w6;h!*F8Dtl=4nIaMu8u`tqwP*#*gI}j@TsNtXtVB4k3#|2kKGHV!P zR(il@F9j6}yp;P{XGej2lwx(GeqI-Y7d6+XnS4pXCE}PFZe%e2fTeNYNfQ(>pI*|P zfRYN=%DgC!RlT?Ya*fHJGHWBH(0j0gq(98jtuvSQPZy^YDObeku6&{dIvUszF|f%= zth5_sd{F<!6-)AQeY7oW^Ja~96LFTafS(U?5=z2-(@g6&aQ+fBDrFRl@AUqB_p{;c z+N#<=KeUad$=e$@0o%J9Qd?CV9?NU5Zvfe!dIe-c2}OUajoLXzbD_u%NLO}fdxZ`% zAMM^l#F5<IXH#Hf%+(0VPeSqO!_8vd=@>h36bAp?<qBn~`4AJpBx4pVoT`=2j>TT{ zh0W8~JmZ+0w!FwPO6d9Vab>&hn#iMU9lCn_fc8(E3v?9RfJOB?B=S+@?Y9Y>@Efz> zAGw|ci=Wu5nrNaf^6&{QY={xAnMdX1YPE4k!YED4_<eK2@&v2i6+p$jxvu0e@BBa< z;7z?-LM7VMt7iq;tzZ8%HwCICPq>fcbk^~DmInj~0Hm+JW|EnX>9m7oXg^m}a7HD! zrc|_Z#-?uS$#CN5wA_I#XGH`_Sa|NXGI=a#P24DMklm5{<GFn7M^^&}Q!TB@J+IcI zNJWCW%GaU7-m<^8W3<2ibVw(ERg0MncsI|^?(_X;JP^2q-wz}ht56?Fp;VeTu1l@Y z4p6*sVt`LSBUA+55EAH4vR2BONOE8YKq)-NkZ^WvL<I@XUNqyJuE1+6uA&*CR3u&v z7u@t%tjH82jQd2abZ#(!y|{%62BA|}dN+OryAR$gM`_oMC)1V1>u;o=w0QN~)~9u7 z9v1yk$th9q))<Y(f~%=l)vG|hNPkkZqA7?sF3;zhlu53t6ZE(0HyHa)iX0^EetI~t zAv{ohb@jM>M|-X?@Uh`lPu1RIA>@F<#4oZTCAw&;Q4$=i#K<ST14T|4hfSLr#)q0e zkjIXst~QxuX_6y^cU-q9t^K8bNJ*c`TS-l(t<hG4>v$&EJPOfL@yw5?4kfW8h4T{5 zt&0rKjqJyb7ytAFu<{g6=~(s(NR?fz&Iz#LJ#50xZMyMnFQX@&<Y}btG*pL}ObGec z!P3%3D&3><H?)5z6+-?z#2^lPN}Sy9p0y4URrSw*{Q2H3@9Kn{y?Qs=nAxf`LqEW? zl$1mmqV=@y%>YWPN=w1a7rtD1_{Y1f65=paiCNKZDckw|-X7faNUy?HHs~5e#jPmz zMQ`;o55YrkmRC?*u+`g>zbO2HblfX5IQ>Z#uxS5w{6j{FP+njNcF+;5_SAc*DBgTi z<;M2xwEYDi4oDv9IG8V`3YdAID!9&w`~AzD9$t>R+{obZc<~lBzv`Of#Nw7;{s6S( zSHI?-{p?Iv)ksZ02(0RahflM}m*d*1oP=>RxR(UHJEFOzZEoLa`XT<0#m28Q;V9oF z(bI!dPIF@5HLha!ZYtT|i}!2AD*6I|YK(pMiTkO67&6s)0z_ms5+aRLsth)F;2$@? zQFO4HqEepOk)j3JZvI^(@Oe4Fz&xV;|0O+aTVma`a(;Vm?TpkPBV*c-{%5;8R&{*C z0bZov+b~tV>UqZ%(J1GU0hz85zy(GjeVJLhDSi+86qs2AqEMv@h>rbD542An1=-M9 zaZ3gx4%i)8tT=v7T6_iwzf<=q%N5<_XIBYh#Apk*;?OZ*qK2S>VV>M8y~NNnAcEdo zP1vJ9ZULTy^AF1`T&on<_7bA)?OEWeZILo`5}S=9kv1u2aL9s$2ov^utneWhjPypz zybPBrlr^E8-htD-pOKZHpv6H~W--@usTT$NGa|viHFCGnZm_5bL1EO%D3pvB-DP%g z>>=gaVx@y}rR|k>^ZAwN)zLH<WpQMp)i$PhX9z|n&916My@@d-%86YPfc&Y3{K*&u zhkKWz6-E)wxS8&PelWY^loX9@6O!n9n&85{n1tWWBpy<}1}wT@9t&v`yv7a5rDHFh zU%u$MT+9Bp?NjqdV3i?ON|bSbR9)TWdYBTWU)QW~uY*r>+L>a>E8)Y_P$;L`s(E<d zO`JB-L1`fVI#>(0e1bA4J?*n**T`Tn*WIE|<*QmTs9AaDb$h1vzg}DQQDnqSzigTd zx!1n0E+P3<03W$6r2dp9KUgoLaHn%9Q#(6lZ<U?1%QaD|qW06+1wGirdt{iv=Tr)x z(=g7|G=ic2##!P-;I)36+EDcC2l#{AQ|$HgwY}Pdr}@#NKjlC1F$J&Z6s%(e`cy}k z0%$$s$NobKIp%<OUhKDcYxlM>-qmAXlW%;8(^y}bJN__aSg&Vvy=%+u65X5oW|8sQ zIn{-8!l*(^?K0Gv`app%(@4J+KlQ;J4f)M;Lyn`P2e;<)<L9tTv}a7j0<A^MrJ_Mq z08aTcy3K7x|F6^g?X91ZN*XHedF-ZRK>snySC}DFNVUU8cL<u2vPYx5zahE$XahZ8 zIdN2}n9`^v@b6V$N$e30W6>@HpANMu0~{e9djL=8Cm<t>WKfK^7*1xpMXE7FHKiYk zk)r36JqK$D!6&?gn*>sFE_Hk)n?ESyjG>*x2^ItWbte6a3D=vCvWs%m@K$IgU9$E{ ztZH`N)XvQhA(e%|k-oedMXPl@Od$&N0ct4BCQEpfvxLB<CJ4%xS)J6~s0S_`?LrLL zh)`;LGY%BnB1x(&c;v05+*hG|rUug}SlPD4BZH1D6MY%o*9{gr6^n*sM=o3PLM{!` zuhjF}{2ghSs)EX8ui-f4O9)6$;pGn-o~cas5G53FDAh{gndtuC5iI7L0a!^WMD<f= z`YV81!v~HGqjFLZJ;|Q#NFaw00n&_PY%YVF)3IBS-zq$%v;`epBqu;`q+2NIc91T5 zIxSGoDZj$JG68!FPO4yG))2I?8m=`;@b0l<9H2ti*hNku<2*Xy7@S15u3|hVR_6M6 zz-?mK^evka7j3b^n5hW;Y+zle##T?OaC6gH(~s+4aJmMA@lrP1dfn~dYLuM$KU+7h z1P7E|`n*fWkrF=lRCD&wlo`t=N*a-o^SkIbA?i2Yl*nq&W9s&M+mEh_tFd6)A4)g; zt}7CsI*H~#q=udEb26`(&+|o}!=X9}xo3a4Dj1({f9k59oQ=C_egj&M_EjUDky@O+ z*iqWr9Q3c^=lGBQD}mijK3B{tZ>kuWnfcrH1<q_I5ln%nNz9y}M{eKemB{pq2O80v zJ7VOJ5&Of;GjZ1bgNtv)<-?H}NunuBFLM0#X`qQcb0sn>4FSBI37kRFI*_KQGZ$%8 zdt*ETI?1In+W3Zpn8!+AYx9Q?(RU7n@z=Dt3ETm-@Ue$Qsg17QhZXa;o-5TB@2-Nn z15chK9($`TyZ-r!`adx2o^ZY~{e@zW?<Qm7octiT4+U)r>3Dew4#rc*AS2W0>FBzz zCP#W$(>VAXF^QK7=akq>y3}-DpM14&)LHnnlS)}YI&VH0u*Z9UL8^mx8zK7XFJtH9 zwQ>8+FTdO3pnH~o2l`#TEP4L)LH5QGgC&eTWQn;t1ofpzMe|Fe-|8W(D8PefEI5wO z;&Avkmi<a}us$$(qaZi|0B!^RtFJf3J2*0OQ^}&xEBZ>Phnyi4Y=OyV1&s=Q^o|%- zLQ(S7&41Vlww};mVOJcvlr@gEYKF3g=q^_g_dCQlJL}3Rnt<d+5JGf9q>fef<6Z{( zZkk0iD!^oK_z}QpR4QLN>DM7DIB@T5Bxz7dt%bw_;{L;dg&`oxu5A4KFjQ+dmt$Ac zr;5IW5NNnC>}5y>OWna1y?6tCs1bLP#U4e43$4<OB{~!kv+Vbx!oxjYnnEv-5E%CW zehgSfQ$|-tP%+zwX;@{ipuD=!B*t0Xct8R<OmQZWa-&ZXbEO#4!<jQ^xv0gIryFF8 zH(ITL5NaZwn993aXn(g^NpSA54?90}tjt8x5D_{71J8GSekL;fC=A1`>Gmg5mi&yG z7BimAx>4~up-Q%C75gE2gUl@@m5cbfgT`{Oh8jbg_MlTkJ`6l7U2ZF~&YE_boSH88 zV>g<D<S1X*`v7u#e-E#y7!BYvZ8bCabkMq^RD+i<kw*(!ajxptw;#n)DGwjGGn=~t zw$sGwFGL=?8uDrNJPo#uQGp7_aMh;?3Ek}$%-)a-(sXvP(0AUv!pK6nH_iA|_;qpE zJw17j`WNE~cS*naW!ee!Nytz-HhoDr_v1i7K3K5e!obAs9XenQJHJ|(f4FDX_?kFQ zPlj^Q;}vq)4$Y^%35gURzoYf)j_nL{cR$9g$P{{M-S%-XeBznpW9O6KZ|6G<&X-R( zA12JZ120c4q@P~9XWvh_`*eiGZs%E@B8|Zh!(VD&{%}B$nj<bXn*$$P(F1z>`b=LV z$;`uD2E<bA>@4IH1E)dFiyU$9-s+%7r;y42OydUE|BbD0ASR@~UW#9Xps~X<as-}# zoseFQu}MC(T+@b0C50i=5-<R`5&Lbyv<1wu>P@(~YDRO{%~{fm7r`%-fwGVa6<`gd zN4EttV2k?@9USGw5B+heKJV0eAblrloI%VdBy6hf*upap(PNjf!lVCQJ!m-#1(M7H zmdUZb6o7_bS;yOo<oIqZA)4Br9OSxl(_S2gh~Z_8Ue?Ve-GyO2KDE=ra@VLoWgPoI zy=7)KE@(h{*l?%@79x6)u_KvxEU++}!HMEfc1d1$EWz-2#O&>U%XC76gkngEu=Miw zXQO2#7opRztUW*WzAKi0x;EW?v;~O|ALBPJun`d025HVu;lTKX^xu4RG$;^5k68lb zBlu|xaroy}%e>P;riC&W%D~xcbYMoVs=Gb<qFoYeQ!wjH?oP|B(R?$|p{mgdq~ULd zonIkj+gghW-$7mACMXos9?$p0TAs!0jQJ&d-9e37j(Vxi2x7_Lf7gd7ng0d@kI!>> z_5(it9!I6vz9F^RzIyL?u~qWbj4IJ_==H)d-@Xj^xGq3sM>S_JK|$3}RpnNF_+z); z(qHb>yUdA^k-j4Jtb%IR8E@~8il)`tp9i7Y0nS7H+eN-LcRKl%p;LEeq>Lp_bbh!R zpWSfI|LB*tV2W5%BR@W)<78HQWm4Tr_P_+0yaGmVSg423PnB$HVZ7JA>Ohs#edwr+ zJCa^6*m!?4)KPO6mMc2ti+;kU6sNmkxWKi5QJ|KOcPOqgZD-z)#r^W8uiip@pHM06 zUEILqHG@a~8t>5|AF2~6R}b2$-(o``L(%2QXUDv!pk|>9+Al5pX}rhJ<{P3F5xeH4 z=eR<F*yX#=(YNZ)FmIy3NL7}P^pXw2$2$4~B<2WoZOrh!@{Ru>-9VZxYKo%K<JA@- zNx$oUx1a~(e<NN0aGy?2pYVgO^n)<2riXX}&vEM=XE0yWN;zLD6uf;$>9I>n=Pv(} zSkWwrl>tTljpZ073U=sACow}brLgZ%>v+;0YrK+jNoB(bA|&GZgp8~w2c(!~ea240 zQH?ag1G;3EYSzx@&y440;Q)`-0NyB6vZG6iI@&rJ&|ykYj4T}ttaAOExVs<`pdQUA zxqzMToIt-{r;zkZ^~LYgShH56fyI5{5n@Q|BR^hxu;ELfi{PmxUZIR$5zql!OS@e4 zrMY|}l6KXR`5K@%$1fy&Z8k>q(nzXm4)(V_nsHc3hgP{zh!olZoC_SVP4562G7S3~ z^=8h1n^Y=`Gtih+*aatap?*tl4bjJtzj?u|PS%wql(4$T*i66q>i5!C5Fj}TLn$Gb zl?u#;F~gJf*dzM)DL)UitKIQT5SdU*XVgRW>YNhRs{LtTAYFtiH<)!TfWmPMPb`ce zA@|Jm?>n3I^_Aa3`DzVU@eN8rT4bDo?VMDI1d?s-5(?R$!W7OIz0)w9k(sH{_gC-m zj%f;^BOclQ4*m7=3b!_f8j=Lwcv3W9Y*LOJ1Z&NbK@K-3Ykg~NAHD67#cRUE%oXQ# zU%DGP^}(3EW~<fC46^k1mzg@d_$S5R5M{yo3*oTY`Dxa&migFuk}eJ$T%3)7bHeyB z<A{qembV4Pq)Te{k`5C;eI{)M$D1f(`zkR@^31!t5O&)Aty-BFEi#GmP0snfdD1)Y zAK+p8t*e31YDevI%J0E<flVHwB6iI2)Dq#5fj67A{zA+jk0SPiKD|^_d^E>0{G0a$ zvEyd3c%e#u^C%Z#dCH?zf02e|@VCqISMbNp7AJ<S?XA^fMSTI{E)I$}W!A)HNSHSd z=`G?Lj&z~fpnck;J!?W9Gu<Lx1O1T34i?ElhpR1!bP6FoBrn#_K^|}$;6#i`P3%e2 zx=Gmo21%H*6y_D8P9>ppWja~{XJ3?mH}KW0JC1ol{qVkSPzMD!Byzuz6cr+yE&4ke zmt)N$qqj^Z8DMbzwjndFhMnA-U=J%Eov(F?ojhre9_6Q1aw<RJu}vH+13cE3otIF6 z0d%pqrAlrqUXl1mmMIfJ^x#rQqpNLqVdF<RIT(egDj-JaA%uk%m>-e^Ev`A4Y?g+Z zs8i_|mHkqukBYV3mT)2)Dpy6mPr+#xWFTQyAwC%J*gX{$<h^kNsjOX)W)x_;w$Vh@ zlIe}{VjZ(CRNT;W1gkD}y1WoupkYz=oBm;)DXz{pWZR@5(P5b%W}mF63}S*A;Taim zztyf}pgWC56&OgfaI}F|J?@j?=ILjWOc`d_DST2Ji<`0NkmxgHFnI|SOCK)J&1&Bu z!dek~(W$zw@S*$ra*H$jPO{z>tP6Y~YWnTR2W~}La&vOxm{+=@NA@xcr+LmZ0lW<h z9J7ogjlh=aFFr#}wIcOi`_KxXxb2*cFV~zW|Mk$YNE9k!=h~_^>{IIo(ewM1L9IwM z7jh4SF#pE!rENEP?ai2g`3~1}KM?^g>ZXb8p+vXT4V&sItV7{n*jYw?$*>b5kMcfu zp9bn!r{jq3rWqwe=E?~{n|C3X-uUxnc9*&F1{eb$UuuLbz=eE>i7R^cdTrC!s$O<J z&L|qpoPQLR*m@o~&Uj{#BYU<7R@|-w8V>Yf22Xis?wit^x6MHVv)8BA$M8EW{8wIi zl%`zr$d&B=j+Ki`{qOAVcuvgEuPmy>mwCwhqVxX_<>2t=+k|J>)8hjJd)c|d*`Hnr zqnY97$io%-A^VtnNke26fjvGn5GylQXF8TSU%)pak4+al!~aMNJ|@np6(JLolr6wy zwMPg?F6|X|mFaFRdS_iqSlO{I35cWQClOL5lzbK4z}Bhenc}O82V{amr-Hw}VB@kN zzk+P&vjK+vxp0&O7_RpV_Fq3_PQ#S%vk574IyBV>N^Any4%OzwJfvf1qgSedz~dxT zj>(aG#on@l<W`wkW-faJw2{uDY59~65H%>ON-x9VcQUrBa?xy}-rrS2r;$v3@71HW zpW>=CBBxbT7E16=R5;R8q5;NQ-AGPt3q=4^RrlVjHksPT;8Bv64rLsScvE~XOeB0L zjs669o4G&hZQNXwR)Pxl80Cru(`CM;gW3a=F_BF4cucTE4kcJw^#cq{a;UqX&f&O# z3il#qh7s<q$wCfIhaX8Ia%69xHqRbTG^aI6N>j2=yIV(W9T0z&i4li4EzK0VP6JPJ zt(EiktyS;B=hud+-jV!ZcQmZjx?Q&C^0hXW+n>Ioho5U%7%`EW)y;;D*~d+Y=zZJO zK)aT5FE?HIj4Ce_|9O#7GL2a?yE^0{R+P|25#%@$431v!Fr=V3Yx{sUlv*(u6==P= z?ot7iF+Cf)G)CPE8_&l8q0LUD(|qm!*~m8iK2Hd1`1{)6rp<%w;-VX4Bey9wEiLV8 zhc|AUUe!RERA0?`*!bebC;cPmb;@_Gfx$T)&%rrpfnla$PgEc9Akq&{_=n3~ksf*e z5kk16`@GAc{imu@>zpwhJUk1QkcA1waI))1&kpR)yFXMO6OUtVfecgAL5~YT1Gnef ziC-pf>3!O2$sIp7F}Pck*{rT$csj<XW~vgTq;jBxQD4f|Lq=mc2epv_(j~S>45baH zlSt8s#k_VpTt;w-0x(DKYz6GsWBx42uib4|rnQ1lVoZ3!xB}K<I5td2zK?DAG?cic zoZS-Ynq5#>mzyM%B@tLOExw12d@TS;_}925CjO}N8aqaA?P_R^$&T@@8sBVl92!vH zk--RkGqgF_x{ngs0APD4cYX)@l$A@6uG$*YD1S5uU<y@?G{)CNCYj3+30c0hib7_Q zQLI%EY=37LLQ+_TEn3QT1<fkdOiujNq#;hqV>>1bC}8njWU@7ZGXspU*N}$;uq9F2 zN3x+mL3S7*MBwcJ&<d?V1<yae=VnA5r8E&gj&^blDR`QX-aR?^UY4t=YF5$B7nk(A z#a$upliHl-8yE&srN{SWSk`e=2<WHL{kRhZ`VhSP^;je%-E|~$Vo5P3kDtz$ns@k` zPjw;{>s1ruR0%6HGpLT|@&k8krb;ZdF5YA3YBy}E(U_h0=nH^F-kaPzs>iai`8?Xo zViioZ>?e^?&4W+wA%zs3S&xJu%dC%SV8G`M0c^WGsJnJk$31_+K;ncCJpIA6w7C>V ziio&i@H(53H?u!^jhUD8zP87jj@a$42Dhm50lTPvKXf3U*<z%(aIqy2EzsaEwO(9i zc%{qzrpC}_^Dj#*jfUgMfXX&Y#l!f;Uqxsi8uwvqm2Ddjn&Ou4cg;k-_jsqZb=Hc3 z$k7lH`Jbza$M)dn_3_GoP482b76c3qd3?CRjjgL@U1=pyGE@{t8l$y#Bm`=Qr+ek) zV6D#LWyls2rxPUC3%?ce$|E5dn_O;Ub+1i-l{7eFr0G~}^<(xvDEeHpu;A24g&*Cd zzLBv4uVVgH%&VoO>7S(Zq+y{{StKbqtoz|35F=&p6v`=+hAIxB1<n=A*Wk+6j#cwp z;atkIwE}p0MpZTA+E8KjEeYd3B}TkZm6=)oB~+7GN|obsfi;6S4*S>T8_F_}99n8; zM+b;*W`Tl4&Le0kO~*H{^;O8QDIIhwen=_ot+$y%spftDUjy`Rf%+0w9eeD4oY`i{ zYlf71gJP<h=0B|_&?Q7lB2tK#s-#}}y9fYT>Wp9;!m(naKSBrZzgAJ=R8G!%k!>8< zf(s*r<dptIE%Tx-@~a(5N!RGfUTJN}Mun5|Nl(+mvEQp%-2>Bk)MugL8NSCAP2O__ zt=h!{u_`&npF%~fG)KE1NrGhO9|M+K=8yI7aRzYBTDR-W>RrvJG!EZ;2rn|`-7#nB zeY|d0+@LCMY-`i{hIq@6kGv4=O^b}dh`8&R*6#PcBx2F9xH_0SyT1MqWRO;+4_EOX zOES~uYKS|cI>6777EXt&&g74PT?^5qbv$Ss4Sr`q*zo&Io2a0>U!|8&cU|R1oI?5z zvs4tBh8+lq{$Y_dorLcGuJ&l05JkVvgaI+WQ{bRuhsCPB=lz^UH*BQw(ZM;ohf=q< zrYUq(Ea;XPvHnQ>KbEdJAg}*zXWPqOw!M~@ZQHhM)#}MDEiJcf+qP}n+<W%@z31QR z={e`zpPScxU8O?Dk?VRLfs&I(njDr;I0|V&X0sMMugnA6u7kguR=p^LTVne2E_Sb~ zPVt^Rh`zV#No<K0ku9y5v8a!~!2AySGBc>GF^V>-WqHSKy96k7bm#(ORbl39<IT-Y zlG}svUVvYaQgnp6?R>Ry5SGN3G+=aW%+=srvcGQJb#onVWrnCZmKkwRI$xnX1IEru zdp#EdSGt4~OAuVi1i)ot#KDof*Mcu7FJQK;B=CGmI&KEJV~Al7HLvuu{H#qs1~QlZ z22DtyYL*bQBJL}uT{nkUP^L&P;2o?Pn8|Iv!|;aBFh&uZrI2tfFVGfCdjI;3&4@i& z%A1kMJI#IcHbHxaj?qs|WX^n61YlnIwrO^X{^N*^+!}X_A|jGZQ7av0V&WaW0WNo; zhw?}Ii~z9dj5cGHAvS2}27<_SFP#-BmbGcz;V?>YXC=+EvT9u*2Xt1&5x>mso9j>n zjtW&kDV44i&3QTh;@~8|!{CCy1^&zexvU?+uHh=~>sH6o94<-R=Jsd?xe!+h7|2VD zseUP!1FsFFp_np*vXLl$_BBN;6N1j28417dgvKbTpiU}%SEF42`qtchW$+Bnc4<2^ z<9&K|sup>leu(Azu=?f-bz6(gt9h<QZKu=2ZVN#41RK#2^gI>~7<-%AzgsqHpBkJv zpSnd)BdA{vXd2cD9S&^f0Q*t}jWn-C00C#SaM`y-dUW9dglJCXNvW^o9!GE0(^kK} z<ebX;j*jqHB3!eVN{#<^$c6k?q1jgzVKr9Kywf%zWFcW`zV8yy!HLT{I#ArjKt_k( zD4)^Y?KN7}xQVxREQEvJ{+Xir($)0t7g>F-?53X{>Btv6DzI>xCV^`$VO{H2XkWhQ zK1k@aloizOFqHQy_csn(opuzT1-q`DtdQzjJ>fMKk@PZBu9`tgfiodujh5)14Sk9# zuN&F54sYb|&(PkP{)h?FmVMW|gH&J~9v8Yq5^nFlVpWC+>t$NCEG7d;&6cKrz&5c! zaWyeX4lzAFqK|3;svM2OSF7Qqu~{?`Eo5OLfRkL*75zNRj?FV;IAqQsZYCWG!-4U8 zZtJ@TJCIoU{Gg7Aj|rqvljC8)3?$=O2)4x5NoEO7tGs8}FCM}{Gm-nVZk#-)y5PPL z94ls6x}|s;waUD1h)n|@AtNmRK*<T8v>cU?Uy%qviPeHlkRP`>gm$jy3lHd6Oh*2Q z=|>quk@lYh)(KAVT8#0wG&{F^AvU4V)hK(y#|Q<87o;8Gm2HfLa)>s7k@|QhM@lPF ztrk_;1E1%r&C6V_C(B9pb3m%!3L_|q!yx4s<Du5L#!m==g>4LwRc@b)2>~v?)UZG` zF8yllRh#DD5qG9AUkgfhty4c4NNS-XTj{Tf6g`dXL|T3CCGTu2^d-&phw3y+8`c_# zxO}icWdCH%l;$mK=w6`DKBJd)0|uYc5%Tv$fdj9+wNs;P4+6GflPRTcttD<KD|db2 z&}gp$VvJucn7vzKFG<kA&LXW@J8<svLh!l=J&Ue)ybuu?71ZnzO>5o0ugERT;-WzS z<NcBA?m|InN#AJ(X6)Hf`)Hs#a-hZ6koeALeRBaRua3vlRq9s0_qrWx-N-tVp!8UK zex&<@KZEiib#Bs3*6EXri?sgX`}+smN?l(wHrq)gEf{;Qw`UzoHldy#(;|+9RM0nV zBIgncSw&YgAm<95XZEL*zt<mGicZo(6uQ2fIn3VL!E752ep%wYx6$QsMnau;a?q-W z8O4qxwh@N=&>PrNeeGVFalv~C;6=4)I18M-Cz#pPs$6u?+ObJ2VDz}$Fym74!y9LA z9HctbLQAxgxKjX8*6SMa&X!om=^NV3@yW3|%h*<}lqwmQiTTD9Z0uf8FzM(FZYVqW z9D9@PA0F<mHM{H-xB9&Tw7#ZrV2W*lrrt!S;#w*znriec!*{|FhOzoIksY)w6jfgw z$?$3<RD*tt5ak(vpOz1t(RE47qCakx9cPq{bJg;a<<h5Bl3Dn=g0O2utKn!;FLMV% z>pjYw%m4EwkTT+j7W%9i_MG%xy}UKhMf@~P?;<XyLdBV)Slpd1%}!4zGZ!<MU(hGR zEDyoeD6T^CYoR9WzA*vD*1v`sTuHQs{X7+H0*-3jJGEo<mh28_%%9ugaWcpPJ<#Q^ zKM!@y=3mm;f;B~p8I-zgetzNB9T%a|w!x(kF`(kicjp{j$W^t7tew4+KByjnU)HK( z6&;2vmo9}hlVPS9!01;G98xvS!VT{=$H^&&R*Fi#7)4dGfd-DGl&QM19hDu<iU6?E zhnTBvpdv9<k7Z$jh$~T-iPBepAY~l@R4?~bS1w+VL@~8WSva6Q8yobh2X@n!bHAES zYDLR*WW>p&ty^s2jo^I+#9EenL-d{*{Wh5LKF#P%54wC0t983ZgTxSQTp9Glb!TOS z#DoOLk2fz?lPMbsH_ieiuJZO#8U>R)6*qWIdIwFktaWGmt2UWy<;o>h#sh^_zw%Vq zwNR$j(-4}$st^fji^A&ZRZ{77wN2Kx;fu@Ds3?F0aqa|}hu?y>Z6F}J^lQ=X#Fo_+ z2?r4wWD$k$7LpiBY7k9KkU>WRdbxybCVI&O`bq#D#EYz`RA-52{=hx4NmaF<@EHjo z{O1K*k+@Ai>3t+bq@D?{Z&l|}*GrowJnD_-(a)GgU8iTf&0)J9^%cpVJDBtqwJnF_ zxT#txnJB3dJATt%dEK3%6J~y|%TVT_D@BV>EnSc<Z<JMu^*Q0$okcWaS&h@v3<`|6 z^B!$N{d?tFFOSZ^)!^xvO|IvVr}?ex2wDYVp+YaJs*2^<rH>n`|8Ffb=_a16lT}&j zpLA?TydVO8ie4Xrkv!%+fdd4(1Ye~r;;@}NxFqw?JaSI3us(hks?88yW(mXCctsN< zz7ci`BPl<)Zx1G{)W%qG2b{7#R9BkL#J%}B1An?=Sfs%!Fr?&?u+$6Z*f<yW%jm(> zlK&7#IW4ei@9})v2hgukR<TV_HRiKvJ#_Wa5*N%$NTI-BXR7y}YUUTEApfx`JqbAg zfF-wTR1=@BM_2mspf(dxqMMuK4n32QZo@b&QkPDuO|3XA#iEi3Z=;Q>*2;5Ov!K4x zRJD;%pN7y=VE+?c79Wn+M!g?21Qh3fKVDf>GN2!9IMCFtWQUR;nEF2<h>cR7H%SCh z$)!SZd7&h;{kQ>Qe3@hcdZL)iIooIT!b6<dstDR1g_#mg6Uq=V&hlp`U@hxUv+AqL z)<~U>jy_bG)|Z(-;f#I^1-vXcW?gaF5MS*H>DYP|<DE&uu!QX>UvhNYV<LCk;t6~q z;4Kcey8oc=AHZg{YS&&HN1mk$et#O>`FtRLv5R_JOF5tHCt;vkCMW2-8Q4&o6*i5U zV*agD>4Dmgb_W?D9glT($IIRvDAe1G6=XY2%NMTMi(XF^qV-#mlr7Q~ugndTNczXb zYBjjl57obthbE`m?6p;jtyVwPh_5tMg)I>6eKIqTyeM-qXZ^vpY;al4RiG<k&4#a( zn!<eC`oZn6JM%@s({3$hl%9KMCAYqb)dSjoQa@~!J!R`bgD}cwu$=F!PucJ5yBoKF z&QjvX4{O?sKX$j(qwF|aXLHR>0)F5Xe$8+eg_DCZ%$@!H(W_{VyEtsvmTW6&DTz_5 z`YDcUCa@3wt$4o%W&5|wSJ44#x6Szq8y-DI1>cK|NdrGT<XGwG))S&&O#Mg4v*1Ve z!Jn<&5fLi5f$p#uLU0aqe3jmWk2l@DRi0QWpW;)~Lv9yI=5z-9GMe2<f?_cZS+wqO zZze@RLf^x~zi}j)ivuE<I6QvDM{lh2bE;&Yk9Z&!{9>lg{m^|($#>+(MAnv@BigI* zob*TIx9X9GadFA|ZO~`z%ftuTiH6!bNj^QIl8$>Y&s=(wt~z|a`6{2oazhu1<(ovg z_Eoy>CE$q{E7a#VO*e@df;I9oD~mb3CskeFW*}=nE(|oQnHDR}q+GBPcpR-D+JUJ< zDmz5Jdi<0Pos6WGyue7pyOHd}cdekxdKRrxB{KyD=%Il+u%yz#(b`;7uZQP4jC2<Y zBk5a}DnbMoBMc;Ht700d@DkEb2MqVzZb&%w%=EJcqt1p*!|J2?xdIfjF|!%yl#UV_ zItwM*QHe$)zM(ag!bt_ty-C?LgGMebo7yvPM$#Im`(j<G*_5wrIE5vfd4}>nImEXq zE!w>H^}<5xS=X9D8J)I^J+}$!9BUeX_!b>iDK|9^alpRHGmHsvyXluHaTaZ)?0TG+ zW*8PvOePI_`4$tm!hVx)3an^O-AV=B+zqlgqRe|)-Toq*vQq!coe!JG@qmLsQIW%P z5aUh!KBp-1$&Uy+2sWO?5zLFMC=SyZKMi~^?!W`BWP?B6+Xhm(qC%v`JDO5YHzFSq zG0xGR*jQom%iZ;-2muZhGMyIb?CCOU_gS;-$t;4KEH>sH&qEzXwMXWt8YoiS45s*` z)k7sSw`g72S`900BPw8FCPjO$<@Fi0KKA!2V&7RYl{8`Ayv2CwlkRlE>0sY49&}!{ zr;GH02JB&lXrb0<40Ks$uXh$15BrWo7y3VDZ79{QpxY!BeMUd6#@FSmf?$t^#%JF| zn5+wzzRt&0lP)e!1*oND)0_u^9Sban4qR+(GI~KGTGr|ac%<$;ej&IfSLBW&=C0R1 ztu;Y+e2<t+YsWP~|Ci%ykeQhmXWKlR5aGnWLrJHeu(zoHhEnI|flzw|>46f>O|8pb zVHUl2a`vyN&?CGjz0M!(J}6tivxOrO5bsYPahUuyZt7E0QZz^;AbWRqCUmd6v+`1^ zMsIi62TQ(fz`@6c&zlea<Oh$|K2$@!nEgyj_`QEW;)g<SJUjOZ<E{`&W=oh4DutpE z%Q5pob`W0mQmsX7o!xSk5oAqG%^I#3{5+}f8(XlTcTd3nE1c$QN#JpaTlscpL;ag? z)d40{*#{izqA%UM6v@Zbm*^NcMPbMehw$)Ip|4Y64{+*L=r#4+q79F|c|_K>=VV|Q zZRM!M=lH>cuM%W$U3j6F1V08_VaR<g0>sXUBCp=_&>I36h)D=zBOvq8t1W;+6jk{z zNs!d!j|c%aX!NW^G<QQn;@fJ!UNkpPf!r8*H52^Y7`c>9iicRmRgF^;r2yd%>|BL0 zRnj}E+<C5C)L8McYTBE|A<SAdH)k&6xxXec@)Dp!o<8F2uqO6R(u+0UzKjo?_`wdb zx%lB5&$Z!7w8|}kk2ldi1I^GUb{8}!Pjcc5!XTDOs|dBzJ_E(r%<S+c6?SxwBoSdb z5zuO+P<hwKnC`NC*^2(zk7H|t%|CvOB4^895kj1j6I3rlV)sia*2@laoAqwS3ge=8 z)3gvS@!`~eT+|NpV)Q2D-UYSCmFBvYoXgN93(3}8{$vE?r^iZhD#X8rS^rJp%>X%a z4#qyBNIqpTn|6?+D8x1IKZ_L~9#p^h96bn~c&Sb<#{|KxbK?Exhbi9lA@ekW<>Vu` zbm4X~@NfEE>y2Ez*!^fXe_DtQvEX|~m@UJ10m=EM5l}WDdBS;D@(^&(&VASR8g=#a zL^ckdXi-3qdE_687_W+oL<I%M=%)s)y|`B2*^yV@K4hX@6QTJ)A;CUgTfK;~|LMiy z<89r$=im{#gkH;<_~#`7Xt!5LTa0a+`3P0Lo+A`tdbSNP5L?fX6ora`Ut)SkMJ$?@ z(e#5C&rlfe;W%F1X2b(Aa?ySZ%<87BkWo)L53H@62K029Dgi5J8YQ3!C%e47Y`ims zwM0VT+jij?(Du7qstBpkO^<?~o&Xve+L@u^rlbYnfyFe>%BL(EQuwGSH5;|AC+Fcw zWz;|EwR9Er&S(t4W7=qny<L3E;=Ncy71vdUIg62bf|ca7&eRZO((R0$8lp`4Nf4hN zA7!%<bf4@*qH!r8rm4y%YhH?yC;U%D8L`%45+@|StE)>x-^7qkN>qMII^V}Zazzg{ ze`7f~N#|FbvkPQGWk76xX=%KxSpleTls?YUsu_P2?Xn*08aKr<uuz2Y;tc|v!EOw9 zDpG9<`zsAUr%qSAB9@zL3m2T+$)gmsG<rBT8c}+Vp;ZRNNHs)pEOuQOts}xJJC;#v zIH~KSp}-2S6zTDF%m<uO+&314jTy-9P%7xTFDHLpVGcG!xIDzC3YliUasq4}&<aG* zxtyu$CVu?L74RyXjU_~n8h)X`e%EI^DYygh%|`zdrw@O(jn*aMm(H!omrpqjzoWSj z#)LIOMr~Fp<E{^T@6XKwz-tul|B~p24CV?x>^;V;RF52mif}$X!rEvL`$Nq0Lfh>$ zyQv*+=uPO@L)f04wxP3#9Q+{t)ZkM*w!QmKG4D&~xxp?we!aAENT&V8XKYyO<oQ{< z`<5FY3^Vk3tpxqENryl-+nfu7qQAc2AudkoVL*x4gq3`fr&sd>Uvmr0@M4EXW#+w9 z^L4wJoXVvqsa$&p+dFDkD(>Y6hHn8mRLrJM#N+ym>uT9p(BQbUfK(kh*(7*Ky>>3~ zWL^~qe{u$6Kkh1msSlKb2lu-As0+)X=JmJvybk;Qy**M&N-_VFnfat0swh3y0)=46 zUes#ggIcnvyvAUqSReI#tWJHreW!r~HKU?zWEF+9s`blx3;rvLh}Ykvj!6)T5~8s* za0MzChS0x5Wgnj%No1GJ=x$$D)q$$}?_oNCb<8qoyA~ly0pMJf04K#y)xH9=eF`!v z#OvX-NkZL<raB0<c|iS}(e_`$<CyLBy#OPVv6O|JSFi|JpP-;+gnj;Q=E&nu-Pq}0 zXxt8ddBdpJf`y37)K_n_xOcHxii7O0nJDc-(17k`9U&c4zmY>>lNYwmaG!*tLd#H) zraWn7_F}7rQbwhv?OpuxtC<G7u*X!|i}PThJCz9et9pKZWpa_AsFyXGj0$ls$FPC2 zWYn&oB9;r{^L9!ci4!z*{NkR)$L_Zw#MX$lDVHHXR~TMdT4cVUvo4b%1fJkB*zf{# zr5Vus16i*J>dfW2GHJ)xpr+t!ejVs8c<tfE1=9P618<7+2OQm1zE6mqp13L83aY1` z94rkZHMHE^)}5Gl&8VC+emw4fs=uv1wy&S?@H`ZotdtTh`o%z=@Su{9{*5DKU)*)F zfF+WYzoK>Xx0r9%kK?6@Xp8JACa{*b&*PbZrTXDdBZnfAmfm=Cr>JTwo<puOZuUX} z9paSUt>VRW7AERtuetlpVxO+H;KOBU=z8yd+k2g7P^FS~;hM|tLsP=3V9uFME!nD> zN$TvzucFS)=no_Fg6^V?0Aa;$wjp+0d!@hmPgO3@FCL#)$dx8H$JNkY&S%j^y7}g% zUlE<Q<KdyPD<6BF#rst$0CvS=)|11-!;dN^J-OT`Uq-dYLnaf!CTEiNIkvJ-a=zo7 zdC+0_gs9afYf6bC+iOwgK?5o0rXr+`Jl1gz#mU}^fA?ev;Tw$l-J-;};Wx{<ZQ_oF zaFL@Qh_O@i7j7eSjIv)hT#skwv5e`sTD9w}=l=+S=tQC@mbA*aQW+<Sbz=_16Bg|} zIIJ@Z@jzoY!8tX5WQaoAfv!i$;IP|W@zE-M2k{ZH$KDXS@BO?6kK<M#N1=zpUrTi5 zN|kg7wb`}su5VXv$lzO_(aM6N#Y4!UYoI9k`Y<f(mB=dUcVujszOQRCv~!tV{bh^9 zGIhfGqXeb{E!n}fgdrhO0bv|NXonVO#sl8W(honuapJ|6FQFEFdML<R=kVDxQ$X>x z(L0Y)#m0l8lfThG-Gk@xT*%XQ;(E{Xw$AN|J1drtnO8WgRQeaBXhB_^k+2G<Fq=jO zJ7<8J>#o}b)nUpcw9ZsSsZ;$z)}dW<q)|Dx%@+WqC;52qHG`OxGo;SUotGt_N#&TF z{>Mq6E27#5k#&fIB+mm^DtpGccHyD4!oKwW)!w@$fEG3{50rqSn&Z8d1wQ<Ya=$*q zc96)a35Y(&Pk!3Cn(5SVZAaOspSne=F<Bx@oa1kSR9~M?RO;kc|DoK0a?NO{cDWe` zk{P=tXR)!flptZ<mJfzBOFs08b@YMK%qxfKFT^)ZRv&Gc3}3D8AEfj9k`_7o4(w`M zx-v4t2K}th1+al=$@37Er~p*pw{bg-pl%onkv56M{I1gl{n5<)Oy8C8>_HeR$%L=# zv#`NaOL};PL1&D9yZ5jjtKPVRY`0&Mb#Jf=6@Z9oNX12$a1fgcdKq?h9YmU}+Wmz| z^<*gB07hjal9Ivr#kPScnJ*?u_tM<7mqn%)KwVcn(0#3gFv8EO!$KE=&r>Oq@Bfa7 zd*|=bq93KXh!>|#ODX~85?<C4G9YN<piLQl?-sF@s;?Zq{-J)iX(RS)e8s7%@(>jv z5<XG*9_8>5+QEGqT>~j@#0D){i^|T<eZA3WL!5npgxzRMh=R|~=iyn{##PmY`?gH0 z;oB~`UW+Sd`&0YykfpIPIl$6e4B%WERh*ohmRj8CFPq@5ruw8jnr($<jOx6iaD%CJ zIDK~+W)$v#YGB&Qj!EE>K+)X)M#vazesvikjH#)D=CJSm`&CpU!d#20X<5RZfvBAx z8r6}1vB<-zN&n2mcd*r&;3(S-zneX20q;90?uDuQm89P}F@{`sTeT`~1(Qeji?bF< zS3|^^xD;mqG~-)}8xd??oQgs^<sU2ex2Injt28QxsmiKU?O!GZ8%z=#u@T!CW+;-* zwcwhfSslC=o^;D6O$0CjQI%x{$QmFh=x3&uNMTasP5*a$3cwtI$7u(uP$8S?xY<^; zGQ{#W0VQwT07|sB9X~}>4zIQqjc8%dI5>E0eFK|^z&62&yer}-?ns^MhL-|Z?I5a` zrdKtV<Xd$8sC}m%(|{BSqu&bN@vhl&s%wtai8Qp>I%+<2M4{#W{(eGycby*a6~a>+ z@Z*$!h^H`4a<H9r^ah$~FnT6U-&M+m)WbX`5~T)AuaPp$s7Bmpf--K}?MuthJ*hk; z_ARfx*wxv|jIr)dZx6I7a>E)Pb3?3pK`R%CRkX#G7+?zp-xdkt0W{qP({*lK;MZRA z!b1rOiH0%0=|){XPy-UHT#UI`)?+kYb*+f9m5yIZ$O3)+e>n0{k3-?}$kEMbj|Vh2 z#@7<<*mI+k2j&7J^t?YsA+kgCmDW>t#Z<-iu}bacs*9%>o6Khg!z<HdlMylQ%l2ru zr1<X~1*r`RCR7daMlWTrPE9fUtBB?$kEZzx3PiKVimhd}Ct>1SE_FrdejWr<YK)^C zD;ns~<@}l^-yYX~S_jbf&l`v3ciOled;9HpP$7#S?<b_D{`a0F+u&rSrMm&{+<gDj zu{3rXT3ROqSq*l8i-1uQq&qbgVW;P9=V@$tSF?*M-%Bo`y-YWtH1Es|s~5fZiO6gv zo=LCr8%PX;?a(KU)6xd4V<|T2xcuyXS@n_)7cX1ckoP<Z{@g#m0Ac()T(D~;mFLX6 z`<_uA9@@PV1kIvd8Ro?;ihYXJXc?~=0p7w!!`v^!7T0>!Yo8fl4hS&r^&X7>a>JXy ze~3U=Y&n!qV}llcyMn~Pz_=^6Wd8|#&2PZ=py;fvrhkWO?(Yl3XY#!D6T-4i<*?$h zWHZgVxTVnd?cQ*nfR8p)lcSmY7F8|iJvG(cJ_-VuGj^^c%26rOFAkTGzw-0A`di0M zOM?`0A#1S>F=*Cffsac{+&W|9^3Dt~S74t^Ehwz-o5N!N{Vk;;TUlHj+Wm4P8W<GK z{?G@8mk!||2?}y@au&N44&T4;k8p^Hm_<y$G9O?d0_aj!EO$RGST}T%zI*7ih>mcq zKjK<Msa)>5>O{dR(NR!9O2!iR10$yVAC15>Rk8hzm16LHo|{pbWJOUmep=#T+tgvA zusBwhsVw<?wt3NSbKDF~%p#L>Hv!nWb8>5MO8rroLV4t^#A<WhdfAZ}gZuh;3BXD; zMse5aKSgUc*rwgRwE8?&bd`L3S@tdHwH(D%S+-<xy|4Z>i-P^b3f)G^N@Q;7I!(Ld zFic?Vjv-h9B);ak8HU9xA+c<dv=834I*LX4ek%!>sev^Idkf}eSDlKvt9{Nqd|sSE zCMyL^yYhxdr3{qO!-pv{4RX**N08;A0&)-U;T}D7?xG1HhV;T_R(h%+!0|!JeFX^t zfq&oJUyPI4o0pfj|2G=Davl(QL?<R@N~6>j78W)@C456NW5L^lxria+k4s7d=z%Jd zD>E_(tx8!8yTHztYheMsPj7E~_7uPtRG<dmZs^t`>kMb_oyP>Vg0&<g2?P(a>3~EU z0esr{>%5N3m!XYZ+shi~eVMcMRt?Z6R}DSjafkNQC0G-BTK4wsQkP(sBbv(26YX2| ze1~hDOY1<mzRj7_rJxqOZ3D3Vl{1z{YDf(ZMgg<OMVzyg5@F{rI<@LhUmlDO$9*?s zZN};`+j*0j^ytq2F?_6(*aNH@4@m;<XQ^jV)QLI`w&)J1q9yt^e{V4iEZWoonVMZ2 zMZka<b=tOCJub6+o2q|)b9AiYIpI3E$g1G}OeCRGu|!qUr^#6O)##WDs))m%g^`<f zGsxNHUI!h$oIV}*+Ws%Fo4wy7<M6rQ)Gex_N~u{*nSk#gLK<ui+6;v9uwvH7Wk4S( zS4qt_^e$fdZr5xfKB;$~SMA_^m7&%70stoyW6y>d173wZ^wn)8_jh+gs!axSfq3a` zGQN)@A4pl|eAP1I7Q)EIrEdL>!=Op3B)`a}`!Q(WgN=hT63th^ZQKVZJ#9J|GMy)h zI9FI$*ixB$%9wo0dEncA#PX-qj9y5Hn1I{9FN4>`@py)+nRIUct>9lclnYhR2VEP> zr&MrLK^Xgf`9}o^Eu|MP({je<%nJlkZ!h(>;3Lbd-}r-HZSj2OR8gTe?{eZXXTwtT ztG8s5AeYTv#(#(H1uKaf0bXPOHi~=J)6VVX&Z<nm9shK(a<D|Dh%$IM0>>z66w{E~ zVS_=4h$A{bh#)T;85+`(Mj9=X#R#IoZk2j^DGHZ)CyKeFtM3P<)BXtdqnc}b0pK`i zU7!7q|6!`N3!jY5^^O2hQA+X_mqYozgM$|4!qPOEe_o*WX}OP&LGWI7@mE|P0))Y! z=oy~(RK$auvb)bEo8dbVJ+XyayXrZO(Sv&%WbI_yw3Ce=p07&Cv|-_XjoX-h00FU? zYVO$E&f#;zD3Js_JV(f0z1n7n-@cDGux5tGWs3(WtWoU45r)8`4oH&tg#yC%BhH{d zHLBLLg>uTcCrlDTfC?y;Rr496v>{c~%}cbR&)bvJzxy!TahbRj+eigCKOk(x|66z{ zHH#E{PsJESWqrU#MnIT}+R}*wh>`BOA`Kd#l#&srq~tok6rRdO9Ep?4(UlFL2s1h# z{~<H$a|9yJV6fc~gWrcGS>yOdM(8pSaE}@XmvVeAPl-WbjgQ4$TkjGKO{VcLQ%G+9 zfGoCPc}3VhN*<K-gFhj!b3RYzOl*kKG_Y4>JQPP}%+m`;3w|FP@Rr9Q8E5KRXVB>n zQO@Dw5*yP(nliD=A-FTn7fidj-C&iKe|nA+B6a7v+~L={7J7Am?<V-4Z}?pJ!_?ed z=41$P4Fx|%zkCVO&zlea_oGIoH>4fC0R6FQ-@tp-XjZ7{f>K%laTRe!ch=(2A8JDF zx?+x|u};j7tm(N7v6(Leoji%lNx4+b7-<nF*Hy{ebcj*aYRhd13cf)0D8G^}!y43u z@Pho5C+stCH@bt8Cp-E6u<8KKrZt+DlJZ$;91<1<&Qd;|L)tLu@1X}zcQE&Qs{1z; zatT>M0fCcl8XHKjyqE*~H;}3JMJlU_DS>X*>-7D@2sU;BBD;h073TnH0uVk<ti5uv zc`n;n>Aa7o$iP+F<x9<^|JhpJ)Avy>X28v-WaU{*X8j~E1NPErt79E(^|9r@udr^8 zGPotlt`g;oIQJ6K@7!cf&Zl4D$7%mKE|QyRog+drc$@;IA*K3EgAz{&DV+Z(+Iz)d zzN8JSt;&BTA0d+Pf{Z4WIojK#70qvK<o>6ze6ik2K}RPe;yeKQ)D-ZygW+okIRt%* zQA0XENI4YUKKtre=AG(l=1#6N%%`>g7dohgeP$z1615qo>~{TM(veJ6c(~Ez^i>E7 z&|hKpIPxEBm(#cfLV3%E`E?94MP3_^N&XkW6VOV)A%f;TleB2j$APtsA4&2&_C3@N zr93<7XZPFPdS$FsOU{`wI>-)n(TEf^P^Zk3sZ`i$?<!HU-mA$LKS*W3wjP;(2gVI2 zEw45@AcEXa7vz+!tT_6z&ObgrYKT)zmb-zq2Nu^O#ULfoLqPc@WNJ!LE6l1935Dg| z8EBQ=8zwE<qY?|C;p(O1$Ylv=<>rC@oGeo~bb4k^h+r6~grkxq*PZjt{0p-fmw&1w zvPdETf$pEE(vPTudgBJb%k%!#e?-GP^VB~&hOF1JQTN0-jHUAcjXYY`)9lFa-(=EX zQ)&)W#M@5l`|K2vP%BBf)@P_yMnE~%blU=H{BO0+YQUDF*PXEyEA)lZTE1C;`(1f! znH=rv0B`gc-ZlsXO2U^1LarbHRp<Cg#&Q$(H=aVWFDe9V^zW^M>!KN2;HdLdM3|B~ z-&uubwP@f^rsj)9vo+LNPhctH9To;gW`xNoHRq3{#HRjreb-@12sg3rdB};>zp(Vx zce}<8@v?0^lV2m9s`k2u7~3ulxZ}lD2LI)o31;Sok&%&qVRT#~lq>xO3(p!Y+koo+ zC&z0u@veyR7qyUhaeXeFU)GLPBQaWsV+zf;9WmW-wylgXcWYK%Gzt_Qt>*E&a!h|j z(;$w*Vze~;uK6Mrhd#0ZQMzO-Eg;ms6B9Y5FdHz{|H{hBO67C=BPO{#n7M2TB`qNc zpY1wp=2|QFi)+RvsicKL4v*71DHR>vdHs<92PX`uP2iwhh#G>n-9P=LO1)`vZTE9d zd%32Da~S)O!ukh01MU#~7BaS^`K@bi-9@Lt70=GgIZeN5^Lk;v$}<lnB17P#Z+Q%3 z(~_R}#ARNW7(X+&V+}sr7i^7%f7;neKmRSIq>a8pXUd|nnwWuWo;0*Z?ORbf43e;E z&(0_gA#x2FWgdpO#CVR>;%!NOJ`|AaiU0k0hn7l9U;U#-U$VLP$krW{iQrj2-(aVs z6?32YHXK2_THV**Z`2oo{<N2<kpB@Ii^;aU^8G%i4O~N`jaRa`;Prz4vW)_5wMc>F z6Ni9tU{IQU%ZAYPO6F#e?ekX+^Nk!kk$>*vV+Z!rteEFuv=zB5_;WmNT?Gy~kW?h= z#tZ2Wu@uO_mS*GF{C;{l=ugYe-i%#G2S77g5naPHCSOZhTno*-nScd`<^>}BU&d&n z$1nUB#qdB?@5I_eCiwMs#0}2x({x*emKn&AO?0IAXyhmTzDGLRH8sBQA$;r^CjciU zWTjlGw@DG9s};};>)QW>Y*vaZG=Zg8_{#Qu@%=R@+i!Sy_(p4pW3weh4_))CREnCO zL-vz5IS9!3#Va;HAv^r<MvSM~RrowDVz_odY)826j5>TEu7K;{U|b?7TKwy&I151V zF=3^@b*b&EJWGM6lgsmsne^ojbfnD{c?fr`Wn}jXDIa$Y_3=!>xB5}XRoz%z?lSm8 zz5C>k59+4PHZK)IH(X|Y2K#t#>7a__Lc0C}>;?R_2sgw0qUr_>E0vfB9lEQ2a0>HG zPLrruY>m8nS6!b1easiS{z(si|6BHs-vpqsK%<9G;6Jm6nI82#u4~h$Rs9=I-LhNo zF*;F~qms3vfi%xOAoZWuxWy6PJL(z;_Rxo`y$B@9@cv^snhaFRI&EEB;t1f09_&_p z-M_xb_U_yctdMT(o7<r=D?bU;gca+xy4N3T>t>x^UzVZoyr#bIFCAs98b+qr@Gj6J z<8w%zBs{Gcs#a6=5;)|bSA1g3F`>+Zf3Zle_?zASh%$O2^Z4;MVA5;ymH%>#*Md4N zjYCClZppYiI1*V23aS8+CI^m24jLT$ESP4Ae#s}6^I@ZvHMzK&8sEftE={`Q^5bkQ zizGbn2X(sQ!eh$=^yKYX%fpJ#dbZo<slJ)?oI2YXaoZa=zd(wWZmL89rGU^Ww1eee zwN2}C=q}ujCwUK&Hx}wS0ij8KFq+Hd1k#fT51h_gfs6P4N88DpCN~gWLq?3<L9SaN zA{`lIo<xC08`yR67P8qyDKY`qk|0B@j}!h+qEC2?>pf8bLz_H5JGsdZiD-N?AD-(6 zIUw)?&6jGG*U2>Y!3*9)=H0MVACyVj$DfukqODS)-AnoC6cS4+5VF6x#$l8SF$@I< z;`a^@SBd0_iHY44kP6Ya2uJ>0VZ`m30KvdLA|YN$3sYaIFof*X-g27T`5hJ{bji?i zVRC7G)#7l@`bU<^#MD&v;rn{b`vjJVt2P>w{#Y_Mi?yn^xaH8^aH3saiW9VKs*S3U zsbQV#>D%*d!(pONpC@@>WCWBzq363)U=S{56{T$!8sSy~3t=uU+@-PK06h91MBg1F zDjV`ij_gm&ecmFUwI1+d_rPdPe!?9xmO6{B9U;TWMuSGuKdh&Z3RX9*mZ!sIv|h^j z<leOzZ!T9%c)vmH)#?UrJbizxyM5a{Lo=eF$h)EHY5tKwx3Fz%I9GfZnY4+NWwC}W z!5~O0vHulD;#*@XMXk+ZMS}*GgXQqpJfJ|}7(4`>@Iw$m!pahOluwlOCYKxS06gn6 z0tkhG>#Xw}Q;7r)97gwgELj}A!M4XadZb0lQmjz#0B7ASXnUp?mDBGLGJ5?*go1lR z<7LDrQqNJLo4rG6_1a^f?<632e^SqvI>7F;EZ?KoGV(k(8{5leu3Wwy<uG>U`Vzw5 zxFDsxna|#GOa>jc7TzXhpN5rXAJ%TDAoB_|IPgH1lxN9To?UgOl8VFCVqG-5U(<Mq zVn1(80mNe~lPo5pzqlRjeUz>N38kxDLoBd@+aH#2qHd!Jo(|ja1-oP5&J=ky3@<7j zuOH#f(tjTwFYO$TLpuE&EY{8BXUZ-9{(*)ip$iy=G^=<~hmX3XHQx{vAA1{^Xgf9f zLU$V?bgro)P}QEYF&pC&P#bIapEZ<=<fEmfS2IUcy$o`Qz(_>Pq(qQ?F3dPfudc3k zFO*w*HWO?uoDN6{nx3J@2LXuZRkR--cS<Y>p3}u~hM>PeN`bZJ<LM6u7?|-%aIuy@ z>D12roF<1`PD9Uy5G25x^YbZRn@Ifbl!mjv)nBYNn@oO}ogwd9b<O$oQv*C=^W#3; z%Fy2bJyYoQhClQDji2b_>~iI8*r5Jy=<8~2+WmIp3G3f1QH0KF_C0U88Zfsty?TbG z(!#ue)Q$qmh?*<Et-H$lDTsARInIA3&Ls}Z<qqccxeEM)wIKH2JE20-1sU`}va_+( z0g%N2c@iU4f56D^5b)N4X*{42a(;+LLs?hYl=_%rrFjK5<F)50F`V%$Gt;gY??+7P zLg-nGtaFTH(y^gGlEg8qoRm~m@9n5|>cvlb4O<6CP0duI^nZ3r4$pkj3%rro+cu|1 zetyi0Y5IW1_Fp7fPo)PJ0hZxUD4Or(f_8|32&rrmb=Sp}bCz?cXeX4?<aa5KTPllk z#TD^NUKqDa39w;Cf`6;PkwN%BnKFg^Rwl}%n@C*Hji=Jin)em{**_j4J}$0>4{wm9 zKu%1c+5s^%HTW1Xsaj$g(Nw_4<x4ygA$K8Ob{XzN^u?$d)r=qfemr9$vm~#m7?;WM z!)`P|>LpZzgNmv@%xm_Cx#`{4RAwzFhxJ$e?2Q+BNuk%j;BE&S;Um?7iD;YpQ5Pkn zWe$7%`!<}Doi^bo5);o7qp|6_=n@k9(nv<XYt!0TWKlo=bA-~S#~J({6@u7MS=b5r zcxa5-Y%JO>;_f;Ezs3k;ngoay2i&*Fs7CF~%?kyYX^2GtWfPj&79?VUrL37g^i`Hh z&C`YQbJ^yz@F^cdEJv!kpD3T(B12&$B_&tM+5uX$7WZ@QYIeGJ<4A%Nb`!lHS1gxI z6nTSDTdS`Di4}hLnezItt(lbYBEXi@ZwWi{FEfqq3Hjf{iBC>pH&?SX0QXYEyduHZ z6ovek74U$3q47~<gRbbI58K$RyJDl2NT6hUa20?~qvr6F-E3hrayHl@0Q%<uvhnBV z=jGcVHi-n7eJ7R?f4F*!H=5x%xL&HHdi(X3o5#n-!_WN$d`3M`1nThjR07#1i9Z_{ zemr7t7((VsNGMQ)C6={Qinlyo`4I3)$GFZzs2+S;_uF>h^E|$6dA@&Si%p7GdPa{| z6hn=_y95#z;7|gDZV!XLp;lH_4H~W=Z_nm!YP|nWBqKgTRyWnR<fY-z)K|<LO4j(I zq9Wv6U`L_$@MHfOh%hCB3$-~pUBtL25%Sc99^hhL4Z|S0RHjGbOPZ@6pRt(fReF&= zvGt*Vo6AaTrQ?6U+0h2K{P^eFo?l)cg(;<s-)P}M7k}?z9XZ+9=ty#JllSuL9uaXE zSXiv@RuOlj&zeJzSYtrPBp{7C-_b71W86uq-?yQeF(kqp4-q@hdjSF&p$mLQT%i{? zM8xAeMA9g8Y+R<}>6{O>a&wSzQQS$;3baPgAoRh~k+*y^<fA}~iA>hZP-22l|6PLG zu4@T`0c%Z8rc*$`x?7EGassKeQG1F@`5PY`(*FhQ%Y9E)b^J-8tETMx<Oe=EJfvc~ z<uv)Kjx7Kx2Ch3xqr=&0IkEiK)>gnHY|4A8fhi3{#5X5KY6;jBDLwxTny&c_Xk~w& z_(&(0#+;o7%l8CD1Jw&~5i|1D5A!J;_~PQ?0tVih$+CjyF`&s&<E-D|yBvC`j`j(U zlTKl!$|xi=@Q0OSs>eJaTMb-C)5bIt0K?R|WiDVFz%RYniHWb(m_=n4B%-YYi%qM7 zR0FrltfqkL5cpBnGsfBi^#1sNRmA^yP3L_tS14wh9RfDwXS<509Vjr;fPT)(1{6k0 zTwP!B&QMn|;4PGhWOw|wd%m-!_zwT?-*P7b_h_Io);Xx_%gf9AaEE3x7{ln2xJvNu z415c=|6PAlof9ZF+PwaKbS3@TO{@F4GT_OSRaDNw$Ebbm`!OHe9X2~wI{bWwvq7Md z47}3`mXOYm`%3fTp#ObAg@}iQ<EcXVOl!lrqeV;@f>~mg6q@a3B;5|D-LP{;&Oe&Q z`Vwrk!g4jZpMRCIK19p>DxMsxA9)=MJ;wjv%hg|U8q2Eoyf#jhTRd9C4#VG0%jEZ1 zxI3Obm({)lsk&Y(LhQlR`0L_5&*AcMj>u@G`RK5UL2yxDZnU_SriK6SVIN6`>V>i! zpUjnbzuzwce)FBL1!!0T7+srK%=6izSil_ht&5z%Jp&Jb9-s|Sy5I}>J_)BtCsH5h z#R|=&82D>B*9IV1w)nj^oYOv+^vNc5|Na_&W}8u_@b<PF8t=2!1^!AHnXPjD-8L)R z|D{AQ{PcFA!|(O%8C!bg#sr3E{|m39U()C#`k*Fo^Z?Rm*S(>562XmItjF+{R+B*# ze)vV8*;nVdExJ1TN>e@Ya)0V}vo}nFEP2>J*3mpss!@3|k;(7w1(cWaz^WT7i_S>D z-^p}dc%f|Cjp~L!#lI^rlKM>4b{-y+x>&tj`=5Ux0!RBAhNLtYLsaj5Z@nsLpp9(X z#Y;{u%4t0pS6W&M=#QN>D{AHD=ALC`kbG<Dd+?M2a?t`{X)EdNxf5OloCy{QFGsCZ z{hg+a)eP#C1v{p(u@RrcGN*CF`<vPyRi)8n#`DV|MOm%CcuxZKz-Wt>mv@<~FQ{k` zekhlaha2qS(6T!i8UY!(el(kxC?NXOC#gx8-D0BdaxsDG+nI~e8N1mPO|`tQZ<}B? z)9;K}Od`_7N`pqDu(KI4WUw~Z2$qV4(3%wc=`^v=t3c1m704D6g#X_OynpSYA-bej zFH@Y%5<~_GeFN9PVqpp1hOc4?M;U-8#%QDPREr@Nw1$KX(0W2aMa)7Ipa7Lp=3_fx z5LeXLM?lbp?ic3gF9o^-4pdpNBJvgX>FM=#b=e<?vT`_vk+-m6VAdDRk2sg}|HAs! zPAj0#4&b7K&@L#l1lme$zX|5ZuL0+U=@&3qDFrf#2QCo?w;;c&zdMgvY<coT0-jS@ z62`zQzq!3tNnz6C2>9@Q<<E)rQ%PAu9fZsYBs#%xL!A088R>H1K|0io(vWwtQ%!Zs zy4c4B^vLQ1cG)*wm`ugY&o0wBZO%@&_?Ed}&5T+QM-p5He5Aa<HZo8TdmPGo!Q3Cl zqkcFuZb*0E*E(!X4h{}pOl^4?2?{9c+6<rOmwEXwksS6uyD(UE+P?PV8Z4diWLY+B zw={n)8|ADaGV9sV<sQsnH3<VWMD3>b^d0RoFkg#$19zR@GYH!Kf%<J8nD?C7HRDVS zFEgJy3!Dr9-8F0sjNH09j#Zrrp%S06!S@^?Mg57^;MakMB=p4z2ZlBJk)*%Ng&e{* z`{F?NNge2L`el1H{PUqBQ5oCR40WIDQtAZjEb5Hvc<YMlelLO7RhL<W72^rY&mo?m zyT_X`f5Lk&*fDZJGDK2Bz?q_?#O|d8W-S0dtpWB2ni`VIfUL1Agn;SK{?(Aje4q*I zu1N#KKkRn}0xQBx7g)o)64)|3-ahdCB>8<YC*bdgJI*`_qBtY6vtiOJG!mFfo1xZI z^lKLc&kj6z%`noa-)@1dCUxM`icRJ6WAc=?VQifG3B<N<7q(FmUImCVPv7-&Lr+>- zIx;%C{&(g%kG>P{ee~|`7n5}m2#LV)83nA{htKoMu)n{5d2qD}AWJcHBno6k`zE-X zg+$0RRZ*l1>o#|NqjU(C`!CZM>2J^T=j%e+<6)7JVvk0<|9M-;pvf{3WYS^Lu3_M( z*4^FBH3z92+Tbh%4-fzFaz-#kM{F$H4m-o=dXxhQDz2Ihe$XgXWZ&*AZ+pXMyYmGG zy0n2ZcK;P|Jgz-Gv@R2AZh@Lmo;PF83OukC@u!8dnoF>o;0Y)owyP|g?%k-z1P?H< zVK>EwwcV>L$5j(OO-)OZ5El!-2kCYrgEjsb>E-Qx`;E3{z0ZNZ4SB?hk6;FTl1-Is zh<toJ#Fs=Dto!5cNz%YkJa-GCy!bhNi+!KnBL5jY69*VROPFabrNxBoZ+}G;DtAAq zm`7P}xo>9DYs<kL1SowramG`H#;s9j>kPXEN|E(U>c6j+d2C+?xHp*e8rI^maMIBh zg?u&7nl_*Y>44(eeI8df(0g#MtqnDAaYH!S*=M;>SYK*uYHWG%I4q}xafNEH2nOg+ zA#iOG@wQlZJwQev6_5pJvSy2M9~vqH5>W&yE1!hGMoW9Iiv}&|GfH0eGm91&gnmsI z)*~$&+YF3UMKv%^;?f1<4Ln<Q=(K}`?E+)C65dqdhO6J-a8>{>TqxHm1<nOmYW0>l zr5I-H+U^DC_Jj!RJq!h&Qw$-j;Wl$7(VQeHVr4~TW#!uXx*gA67SDEj8R`-Y49tof zDjF3BjsxYY<HVfYgLvAOmy015O`p%8mm;yeUy%Tq=vH!ztQXi?5*ec67%wF?^~%0z zn&Y@@7Z1c$1TmN~0jITT3a{e;D|pM%k)@B%2)Ia|Y@kL*9#XSEX*mz*MwkfJ70=N= ziCN$5;SkhnMsPv9ILbcu41OhWiULv=nzIr(aygu+=O=H{uLwCc4oGqXDmUBWB9$<k z!VFr4hi?|pA`|UKt3W$!bar;ue*+%YoFoI4NY@pw%HaZgaqu0Qr<vHFA|pc{`C;7u z_zFTk$mM?l=`;FN#Jj{G{`p{c$ZNe*FFvCd&$n6&tYXI<#pua#MR)IvmQc^pJT*V6 zJ`eTFe42adlM2wB*ggpFPj1h#J3$n(7zaiS9A=40Xff>4jOQHVDnOPyiT*vDRj!O( zf_}imDFpv(hST@An88qDm(R5D1ND%&0#|rLU8tyZMt*oJfrw#Dj-&ppTdX3ipfsHC zluVvMETRFWVoVC;boc`;DJd!EWCIjx+KD*WDUf+Ulco;YI}i*qyI)!zQWFe65Sp~- z9frK=Zx?;wbiOoZD~hATbLMqT8KNPCTv-UN%hA;DzffWVc7eo(GJX(!#3`U+9uPzb z2Vo$qi4OSZ2jm>358p|PHv0@%M;QumUx=PAjC+(~@GTyPtzz*goQEeZL-;K-!3GNh z5!yl6z>?iS9DU)om^p*3c$?g^Ct0tp<KEH1>t>0D!Cp8HCql3Ur_`7Xi%=UVfo1MV zj4a8nC^!g#s0@C6DbjeqTb*BAc4yv)o~W7nT5TO&41+twTh$rt7BPN$o}6j0DOJuR z!OmN2*JH}+gxPL9D48g03V~UdG?;^<qt9s2U{icq@>yDK<!_VzC7;W+>ubQX9^t!& zd8-7oo4G1N?lqsX53}#DW_FMd^qq7!cr##$bishq;@A0&>q*@7E$)5N1lR5lE}$wX zb<TjR`{f!Mga0#c?9LSVXi+cZ<-3(>SZ5&;;uL5*C_U-CXl?ERf2F9MIvPpoV@m}2 z{Iq;8!u({ZC5G>{URl<(s@~6FoFUz)v3#kS#|cbjm$tkqH?vQ@H>So`12Rv6&Vmr1 zm;&kRAlje^3^k1+F-Ol+APWd%3#`gW2kEEecUqq6C*e?0fjSaJBnu>WVod=@ywy<u z9Y&{|MfecPJIq+59G!IUjAF|qSTnI_LtR{wMve6SE*iyfjdT|Eu0Uv--X>V}E3s*% zF29^zFbAwf#3jedtBCFA{Mo?G5_-?`V#GOg3kKHhw70Ov7#5uq@ZW2Jj3T#9rh@%O zNh}H{;8#d*S44+k`CDpG5>}c|$jnrOh5Aa_PspC&!Oaq2utzWw;93)zR~jd*uV5|B z{%uKJIELQ`iHo5=t`?N+Ic*l|1VcseI+ci%kY6E20{uo}PX|Uv>bD9CDsXY#V1|qh zLT7e(k>TK~V#nS1O*ArCJo#*{yl+5`ot`LBJ}I;TClsfA+cRDKhds^Cs7($T6puJZ zkongbF6FKa0tgaX#gIt^m*80@BO%^UeVA;BO)5fxOlGP@@<gxtkSSn-i~SuXCa!yq zF<=6ZU<&My;qNtLS%Rx!rh_WqGR^+V7qbz8pZ&7o^#0h%R5+w?t0)tz!~;@!vK6f} z1zSiy9v2A|jmKigXD?0-RVl>iDpJ?$B)oI|X`So~{9>Ee8aKO1k<9g^KfUyccWL*} zC%=tanmno=nB}JO>)i2iP79y9yb<gT7FWU>*=D?LdY+sb&P7+puL)aI-J1pWq*!09 zy6Kimmn?a37+%M_jud|AlXsz6c1`uh4Gy`qWp|NEP;Ou^_z<IFpaps$hmho9@1{9Q z*ptKX4lojPeg+@A<q0oB$40(HK7_{FqN0O`D$qg!?NsNMW%da&;AQv@<iUWHKSU5k zpmiM$z^Dz&fkcpNZcDm94_eo_SQ$7+Igx(g_-o%pLN3t<VJg605_Tnsu)~!;^Cs^e zL*)=&5@-2bK<7Y%W_U#;Vefw^8lAA+M^1Vof#ZoFrb2f$2~9+WLE!a9je2-SHv3!n z*6#KfAqDbqb929HIT<G1fJuN;O(5c#`H;rk<b$sEFo+bDyJ+++N0Ke#`{d4kxw!ZI z2vmcWcpi0A?ufl%P<e2L;9;2%jbPfB8e%f*X?Bud)5*zjG#i0k^2GuvQ+rbI3S~+W z;=)1WH4A}dr!XHP`_>waMBEhWJKBo+(?XtrAoB=gU3zU~p7}G?30rwU$`+mrS8p@R z-JH|B*Tpv+-VeF@w;9(0v(+l;dFua?IzppF9#U<~`p?qvP6%g9>R*JS=NonCqbDC` zPmZVgs#+Q+&hd(wV`Aqtid^pN>wVpPQ|(E{X4)}(UH{5fTx1fJ>qv$j?^3by@la@v zx@ec=#742TaHth~uzz>GQ5LP_Ipvu>cTa8jwSO{C|G=O0P|m=^2=edP!m{N((>OCf z%_6{irv>-)47mz5Mlzu>ww;0euo*wpRL1)i2ul_cDq*|^fsNPeX&-P3^#zQ8G$b6^ zggAo!8;}mdet*?vJ3~R`CBg(lh!d0SN@moted<Q*uZ7J(E0jT{&}Zw>0H@hXf{mY2 zGmM{1LT3TD<e3fFGJ>qg!jfS1FDdoZyw2I??4=09&AuMO8xBNZiNuYI!Ni9+L0%MT z_rd~Uj3A6TBYOfu2yH2W*suFfAiF)E!BSY?|KsQ^1DgEWFir@ll+q!f(jX!&-7VeS zT@s^HLb^j~7%&*!Fj6`s1{;IXBHdjA@BZ(nec8S|JI^`yecjjZI%mb`v3|BUcy*ez zRr_C(q@vlT<FB)lEhbYA>9EvL$90SE$Go$53~)c%o5X$eKP));_%*v4Q!I;X_+^Qy z6fdHEgfQF4x_yQUJ-!6vV29wz-G2Ej3P5xFyg7KO5#_7UuGahRnCM8@X26Xgex?HJ zWQI%9E)f%$PO3iMl1sg<0K`8LUQu8BxexM09kp)sdsO)B*YW<AME~{W8Q$o@Ubz;8 z<^s=fytPBI5>E4S9E{6aTPF9)(wtHO7M|H+%b7Lji`-ro!)PW;Y1QmF-}mCI`T@r! zSjEHNcOFNJ_>+V0M_HCvvx@GGHZE?5RD~KltNAx?S%k-QEKDz9(8d_c4te`u|NMj; zngcp~4q>?lJcpA{+nX>hHqrN~_&HHKLhlJ}h$uu&(J+fVd>KxXKaWa%I#G_xA|zyp z5*%WY<1|P6fx(Bi=Ef&Ryg`#h93@zI%0j_1amPx5b;Tt@NjJiWQPeAe(mYM%(S&=W z>a;~H0<pDkXcRESDzwsH*S+KwC6IfuAM_@EhH|}^<wQEDcQMK*^#NTL?F|6qm~z^f zNDiHDPt~0QYk-3JMLhQu+w^l^p_o84v3z_G{WCXag#1tUAm2CWA8Y8oV!=-08|cPx zKVz&=#wk+@A*Q-n<wYVGQKyW#vK@}Z^MQKtmS6kv<oQD+BPDqAoQRw9yYB+<uxq6Z zwg*q+4^w!E#H^GcbzNVsFgcNRFSZj^U(_;+8h6Vjw;2es5Vrfxa=$Kc)RhLMm||H` zS<64XrFmO=&@ML_zS_ogHQJ^xgp>Q!&K$~qUB?iw*^Mn1A_`UPJ^$UcF8G)B2daFd z2hMyz$|E2B&bL^Y#vJ=Ga~!`!CGnGa&qh{1c-`-OT^7j&Dc_#~Y=`Y>R(ZeNi~WYp zl~x!EGi!|7T}+QTK#@Ysav-rUGnb+dmzSp&cURLA4?%jIK_Q<cR%u*oYY~zABk`{| zXm&xTlq+LOCmP1BpAu-f6gF^}oqUg<Co~=~BWq{O&8S7iEoLoe&1=^bQo^9}alaNR zjwseBJbSU#|6I(A#DuL!%vfBAjoyjtsaA=~3COWyv&sptki_4a@o2}Nqa|{|!aj+N z*}MyIRn)MJ$R>(9jCW%B2x5L+ONn?9fu0@u?qtN5O9U{PE(T*1!;Fw1H02i(K%*2% z^c_VLG9N)V>l5O#f1wz(B>If|vwY$6Iw_I)z~T0<sxUME!V?H|4UN;w^n|-tzbqeZ z@?X?2xf;Exh?LH0AL^<J83{DD`}be+&NsU;Y9uHSU;c^LVZFl02<7iGrPz8CuKEHk z{y48{I+SsHBb{#l%tY}<K9@dJ?uKo963`$%)c?mFdndQ5UA368=A7;)Qt4=tY>MS% zOtj=*9=BJ^wos@}O*o*<do0SsB1kIDj}tA=H*fcl{+0+4MowDyjA<c%mDR{R^LWP| zjjaMlgcI%4_GA~P?ykaDBH?u!kY~(3dUv~NXvW6p*%Ng!80FY7rKf5Tke1lr=Hj2M zD3B*+@^k0PP&rBtWSi*Ayj{<0*2Uz}gf&)n%#nB_eojzA0D&b(CO=9#3OJybp8J4x z7+IcZJ=}0^f8Y@db>~%V<`i971=i>o^^z>sf@C)L2X)+Y>1J=k5`NWvaN4;vAfOV| zEsj)LJ;Z~DC;hqH9AYq8oHi1U^=Co0R=n?`cCwejoS<E+CCyZkytmPQj65Is8K;Xa z_XPSh&BcJGfoB&l68dc89)yS6ON{wUno^qdm@tar?Kh2oybJyMgupk19CDLtE;x0s z$fBW1S<j?N!>Kkd+&`Uv*o+S!K*NqY-Am6nXlER3MuYdhSUWGjKk6K)piUP&!9EeB zA?s8}|0wi~1=T@{izG`hFLKo+2W&8a`!<dkiY)QC(azd7fOJC!#?AYg8xO$lIvN1M z^DX^@xD8=i6CtC|1eX~bn%jAP&I`;40{z!wk}TZGiuzpl7>R;rN9_0OH<JvACAarD zq1H81?ZKfr+uwTvw*n0c?69IYDU1HiM;+5<M8|plg|ra_+XY+^uASzh4@A_Qb!lPe zt|ldjuLso7>9ZR|9pEO>Bl`F-%m3M!jHMm2(34H}3%Y?+KgT%ds-gvhoOgMP3F^=a zJ{)9mZ~K8nSm|5VDc;0cq1wT)w9%i}xW|;+rYAX`1$kK_$Cy|5ZO`=OrgT+@-(F|m zTW-yvCSz<9e}VwJGqMvo0?G?N-LpTxHzvQrhm+e4?d43p1UzjiiMDyFudWCJYuyTF zO`LA;#v38$gAPH#n}uS+gt;}-MJ8I!{uV7~-(|sOwO6r(Ze?GOg{UZ3)feq}r?Z4j z03lAHkKL@2A&c4I@51-{OR+0dL<@2cfv7cL&o@CyQbnWNF?xUQC+r>{Ozo~#FHN)9 z`oqP^ys3@lO(&tRlfvXTyD3GPGc7{w$k<by5hDMQHqlP<c%Wz5eQ$D*_Q)D=pg;Oo zY*FmHCl<`El3dB1i2L<l@1D(43S2WRJ`q_w^X#5%lYG1GzFxYin9hE=rEzYjVHbP( z@87I`(25+`(|GJDs}oky==<R|&Ru0W=nVyZV*hDu?XVxo@fTK8+DZQOW#HH`Xdw7A z8RC{<YG*b&NO<AlGw0QW&*W25n${J;QkF4d!R=INtrysMw2`P4xT;Df1)O0$!av3W zx&9}5XYfM_EV3qZgq4AEB@PxA*&pAqPA7QIdR-$uMBBPl2z!=XITov)TJ^_uMLQd{ z2#;x}`)dgd3L$3Kt}f8L%5lKyh7GdOreRA}P3!pP%F%cm-;XcIMB2Z*)P=`!_fIOp z-Ts|*P>-PW>hZ`-hRy>o)Uft#Nw(ax?ozHU;raeIb!NN0A%+3H<zBe=Sc{PazRpU% zF?ZL5loSA&qEEI-b%H~M=ov{b$pdpw>|Z&`&%edJe<zakNkweK!lNwEm&oniFq6m+ z2by42Te-KHU!|lTM~ugCw-pobJqJ&pr#<UK_a`pay^0*pD$)PPK?z0;VlI5w9flXx ze6LWDi_+tNlbMSg+%;74OQve>=Q7cb)}7v!W@MvF=9NoMd80foVEd1V=#5wLQNj4^ zL!;mvx6(By$<D7*2mh}&EwsitwfBgvy+x-+ucvdHyDg-pi6EarPNx2q+x5_ycYw5u zi5cpKhsx}Xqq5pe{ZEjZ9;}jZfV;bRLab|Pg+&J6{Iw0>SY0_LM*MGKZ<yu5@EIco zwoz&))#v+YB-;=Jw%qG{`M!TA=0ty#o-fTtwXDn9e9%YT_i+$!9`W3=*DC#!dX~(` zCA7qi1L|6KTA860^oEEl^k(OA!DNxBLkjt!euFCV^_YPPTakAgH)!j_DxRmxgMjJT zUpJ8d#ASrJDT9q!#46Fzok+u?h5)6J{B}HQ0!(~K4UA=Z-M8$3qvQ|yZw#fB7X4M& z|DurJO~3GRJZf(m?7O^5S%kBQaq(4IUUbQkV-fSxT<-_%LBdRUxH}}Is<m4bZswy# zICn@a?X~eWbJ9(9X2|_8gHpp}!m+Eo)$FmW7zu;jj*0?896}yk4%LH1ItpemsU4Me zDyi0P&CY6`z9D&ipl$V7S0^;UGkPJ>uhHw~R+(72Z}7y~m@t7NfoSjLFmX2!`Exf> zN34In50MLsP84oi5%d_zzJg(VM0hEYBovzzf#0{&pMCUIQ*^`-;hsyf=0FMTx^S@0 z;a}FlcUOv=@h|=k@l0Tv&e@4360Z#rzqC(Vi>!~G@_0f`+j5g<Kz?NT+6-*PE5u>u z*4irr0VzKG8vLr3HK0*S^9!$H<GfGa!g!DC(m?^o`8Fqk6{suec3~B1LY`Yz0fBGU zExc}o`YC8hiQKHUU&|ULb@^{yxO13&QjL?p-JSdA59w)Zq5W84!!fs73b^=c5F~00 zRUp(AK!jf&UQU#~3OdzD|5zOZi-~-P<gd@m#488Zi>~M*=0en$Tc2u?f$9d8v%>pn zimZ@k`{YCwH7&EAJF@sk15D8;6LkFG+D}u+?9XDWz4n5{s-HZ*eoA|$sC(NJVfDM0 zWmGDZkK0V>Cu;%fA)bVZHY8&FP<>Cu?0{>kBGq#8ji><&?OgmE$~BZ)m2KT;-51bc z^XJ822Ws3X9pLzUGU@X^=raC2m90MmnoDTPk3fkVgP1Jr`$6`X3R=N(a^j3-rA4l@ zDHf;)Cqw<sC;gdkXTWIgLy6T>LcX!;_uAQC0q0xzCbtKvVxmtms<h%&%fhYpx==40 zI~z8;zv`ruKZ*zAw)Xqs?0}nT$-G7+=X<|40WUHVx>Z+eZRlUGoxsz<ZDfOc#@3eH zp2Fnfo#WO$MEn?Y*~~m__ZwtRbJdV)@3F6pth{9op)9;J^i3j<bFaLA1Gx+Qh2Qjf zQ)MDHK!g6<lQ<Utzg6(<kQodl%`<UG0}kJ~v-gp7EA>Ud=R?VPNA^Q1<>Q`!Hv}1c zmLn3v8mqI&b(HmLNm&R@zy)dT*+*YCT!nH#gyV>C*QC-qQ<WCXzOwLo^#`?^@4QQ( zip0O+BBY=7t4mgYe#}yo6u;rT+TDfk5z00l>qGk4ri!&|FP>*}X`VS@b4pNPz*{-< zLwy)!|MTOW##i@@0W~~6OdE0udaxbE|Aw}mao(XUlu>5W#Q&^*)+gkfrkC=NY<<VO zT5PgDHCD;vRZ2T+<P{T85nSkkRp#_y2=VG=sjCfMKi3ZY5^V{Th6%8$E@1NTIsB0c zP+~%I;xP^JKP<J(jg(i#3MD;kF_;Q_?nPvNp)597UY)zFzw~mm4O-68l3$&y1_pSV zRF^h)|J-ol2!jX<Xtvr8@A+F)eN>H^8NPiK;x36HzzdXjN_&#7IDE=7ecU+K?(txZ zZ1sQ+w7sn!IVhT+1Hfu0LLBtJhd^KTJeB&^W~HZ6SS=1rSt<O4x-svOQ9eC!)!HlS zI8)fX#NK-TjN-*q5RySM@ePFw9GtOCtoj4<)7WRttpIC}J8XQaaugZKpZq5QOLZSs zVHF^aZqcy9-I8{ElfcTYfUWJ3yE{c%r0A<JsMdRt<u8AF#O;FhR>*qf)!}N-6L|qp zV=x6Zxjy@d$vY)4!Qz4+g+Kd7<)?^&0VYrS9OB#f1<NJ@9{lE4ST-M<%=vJH<6g2V z8^u~BE4?&t(3c3N%_s-!9s;Ep`?629*e_2Vif&vjsB+<EjkaVC+le&Z&xkd|lE$K4 zP_M$@P?mm3U8H+rptGbjv<<4{aN9k3bq&qa$iHLjX+Mg0DqJMq-<|u5edl4wvz)hH zRQodMLPsOHS!Q?gH_0+9Rbg#e>E7H8YTGPH8T#H?HOBdn^LSy_qt!L$#x_{l{IUa} z>)W3l>+ujig#A>xHTW+M63|&#UZ-l<;uUkqyB_#u>!yLTSZ~B2yz4vW$?Mlrq||ah zU*%V|`d<Hl{MmkaGO#~iu0b!1PUQJ`dV&_n`$J^jmw{62r7?y(jj%Ti<*>EFWLv$z zxtSkoM>!iroALHVlqoiC`h(Hrx6n(;Z?<Exv(Nj(mG!y0#?;dM=12Z-86Z1+ajY;j z&7%?qZG0_<XEV-gDI3X?KF_Y37q|q6HWvMPZ)lG!|8!`&AP;J>-Ww_^y1V=PJSh0B zQC6$n&RzG1Wic@+pQ%J>0)0Q=tGn6+^<QAoG8OAePm2^1p_L_}cBvZQ>KraoZQ6yT z{UbHZekeFxcF0rIDYvNtvkMJ-o+ufPy#NR~$`#y$HpdAeYOB*&x$fSJs+yGmpxe8_ zkhz|AF`BhvVodPvO`!mwdQD~d=fC{sww*}^x*$P8MKe{J(#5o>{l=6R$mzyXDe8wT z)1Y;qyR``fyfHF|$!o8?4x$5}+K}aldFW~>c~spE<}6RMBgP-u0aa*~8LC5IEue&~ zCTCpvg0viF28MzBZcxN|tz{nzJ}y3<spyc?<4gs+eXfGuQk4Nuzya&&(d1HE1cC@W zSZLC)ksALz=|ceumN?kp(al(4;oeJaizY`GZS+-9d?_>1#~G<1<NL{?59JC+HrR|$ z5y)GL=d!<2$y^~~%}$69)<n(i$0~iv&%{u7KQyi;bsi@Kx7)X(-@7cfu(@d`zNi02 zy7Wus1zE&aNlR;2VMivzPFnR(NSm;cKoS8WZ?+xc=~2=xWE-|y|L*Q;;M4LGgWhT` zO*JP0!OS3B6wt&Xu|L!V9q3kp3sluhbsBH(Z`PCy<Quk`chihMLo;heiuzx3C5E_? zWTa>~kW7&kwF|ep=~z0aN8wLuBBH;-!MZxX683gvo*06}-l_;_r2>F@oi&59`)RI4 z$X7_Eb78&N$tDEAc3>Zp`IP9IR9U@fwYWbw?}aOO?$v1CrivdAqj(wv-;IToZ#i*R zI*F1e)FYq7Ty=OW_t}xzcXlqSK^oHk%+lq%uK%t!WcWXeWb+wOUtHmxGFl+Jj9VwM z&Sb_&Cm`%x85CI|C6+@M4o$&pLHSvNn6y$p$+NDvUG5`Vx33xM36tdr(O4f9<+Qu} z0gJ@S-0z3JbK9nWvXC-9%_ou@w_BRO81V>3%~V^qUvutHp%4H06UOA8T2N46JDKZX zc0DV_(A6LMVa3Qi-&3IIi<}xe%I&kM(TRgjzEW!qEuBq#M<mx{+_ZR6LpVlTn2g2= zxN<5k8a-?Z4E%A$8G~3z(gZljhXg^odL~^*VGAw~sN{sqkH^#{Ip4AO$ECx)uD`Z) z3%W|~{xn_|&jaGz&s-j6x<Z;)%s|`s_VDNys->=qLq`0AX-&!?V%`ATq869sDmZ(V zgRVvq0qtjX4?_%kAUErZ-iP@pnbT@hZuy|TrOj=q416U(zP9s3K2pFF8k4IXB<5J? zYFoB1Hzfuk%d3jB=<*uriOURVhCeHC%%(9=47l&9U<a<wlHVp=$eQ{r=YP^{WlEea zef_{E3{wG^hDct1K#BI-m=h-s6$WG@u3!AMsQr7utgks)`!+exYBhCY067kIddX#k z<Ml;@zH;N;C!Mog$_=`mK(8}WPYm5J6odLKY-|l_m|Qi)oV%Q)YcDQ&I9cTQ^VP~6 zE&hQ}Q&QAG+uUxE@gP&**MD;NU2n&TnUvjWIz|D5*q^~r@{AzC-aF;a*2I%GjMR%S z2aB!871xTs$K@KpgZjGA0Zy6#M{O>f|J+ltlR`pocbyq3s4d)*DCn#%(u?s}#~&#j z_TH=&{T<0<qk3H)&Q1gXxV4ChrvfW+&H=7dGI>j4y59u|;79O>W_JrZjvQK$-$sYy z>CJPOY|82+1k?W2uMf+@V%pb)wtH>?mL);xcZ(lGg1UvYzvD_b*;~tRf_$de(j_hv z<EDGNPpN*=R(47{Alm_Hqf~C%TUZeoelo`>JCm-HZY}IbSPT`}U0AmEL)s%M!N11G zG*IKg5VEW9O=m)rS4!QE39vFR!zu{^h;kd~eIxc75LQYvS%$Zc22fF7w6t7R7CQk= zKA^@>Mz;?fZ)rf|?d{n#FpZCH-6feBX3cZeMYq53dD)9&qi(v(26Q<qoVpOOE+rL} z^6jGc2!iZCi?W^`=Z6Xr1w3gH8oQ~pyJ0|l92(kV%RIdQUM<lf-$V+oK2?&74^n6+ zRhn(6GP?C037X9dTwJZ5;0Nd4zt|_j_&<5XnO`J4AI7r$>6z|F?sv0;g@UGl%N?rE zPZ}u%Vn@gnFn~XPP~02mdsQ(bhUe#v_e=XOQCXikv?0I(j{m<^62w-re#6{myvIgB zGp1sn9f^DqhvL9R0q98+0TNONs-dAMuANGp?V8%!o3x|b+iR~DdO9^V`P`_tt>!u% z6F%PLIg7`UC;UYVH<iy@rs)g^%V|kMeB#aP!VKcRX76IG0?!98pBzy4;x_QO#RuBY z(RBeYBQY^ft0Ekh2m5T3#a@+HtS=imu17fM2{j!5^fITWd_a1jkx-KiKTGjYuJ(cN zEWIhch}JeN%wdW|WCmPjT_ACLvKEapn&zjJnTrGV0t>WF-lSGk3SxeEJ29P5qHPkV z1l<fZMQ+m8pwsbcCSf15F`tOiP{UAw4bn&gDQ`v+4{c8Y^44n=?sYxJF0BU0b{Vh2 zqrKtq*4=<J|7(Dsw!bph@f@^os2e#eB-NOG=Mb$H&6G6RI(6wle}nSku`$JSA2Szz zMgiT6%C}W)LigzN6!c1*M<{RGp($D)UZ|K<K(3&IAM1(udV7@*%DNUHJh$OvXUx9( zT6Ld6<EGT|>0*f|5-%nvpLglGuk8Wb*M#9rRM<8~Vgg$k9?EzuFWA*16Ms?@r0b$A zOXfoOT5w?j6UMP*l9P1-%>V+uwr5B0u26^eKaWb<6;bfLk2tiecG6(nFUW~wb?H{d zM@+mdr@#fVsiwPk^Z35nR76G^CxVr!s{Q=~HMmhv#by`=J1UO4v$UcXy<ss9Sxv3# zaoY#g3Y6|o=bBP2&PwmD=xD!M{KEdy)9YZGmO3$Ec(I%}?r!w6y0gfNtcd*QSBZ4O zw#_}2d@XOy6k}pu5}ah|jWfM;($MphtYO1*DsF+B=}jnwXn1X83nA(C?ZT}t6csr0 zcJr&w9ULIsZ9O%ZB|$S@973|7&JHJ!@RnHL9<b%o3N*Wf36i>BOWj@p;j(QauqD(2 z2N=QqE=ik%wgH3inxe-`V_2;%CZl#W=#kG@2R5xNY!u}XX;c`+cRklTLc!50vG65E z=W*bn#FRUcWU7;mEq<mh<u=n=jF)W+Qrsv`8yKHjFxEMY+y4`>tC2vxDtL1D0X0tL zWatY&brn-=dyNQ1WKHvw{VG~2mzjbr#j_>;_s<9j4$<Kre>o2h<S(8jcJq{cw8tM; zovumn6Y{Yxt_jvG??CA8{Hjjy_jL-V4eZvvb_*`QFQ{$oUM#^eJDLs`?f^Vpku7($ z%lXvQv=n;%Bsj_~7Stkw^E70LEU#pkjqCAs>e;yDt%0Y=BKLkpt=t8BmUS;DSEQY% zYX4YLm40Aj=NywI{(cQj#cHfR7jzY}N>Hk<Bljb;*gslnR{l;v+w^1x@lAPoCdbCu z-O(vx2$qAhfsHOr0ADL=!d7Yw-)vUApfLnXPq6Lj*Fw(1MK9b7SWKd$p|`5_LbI%< zpbm!JPOKt5Z0j9J^OAz%>_W2dg!7;3wXx&zo=M?s(!(-k!F7ljxaT9v%9t@4v}v-B z+1`Bzf#_6}Vr1!Nug=TQnuzeVn;Mbw+1GBW2sr>6(KaD1dbXg+>|`D#=0v)Xva4E+ z-B-V*wb{JiCUWR#Q`o#hwKh73Rm~Dsvfv#878H-YMDP@p%Iboq*7sdnTKZ{Mg`ex6 zT#w9u1zgTa6gNS^P{xVz25As~@f%8r$KR88hpQbb%t@|uD~F!V0q4FZpM?y3rqrOV z+h>5M-_MtyemJ)aLPA#V0^L(-19i)B^z8N?`JCFt)CgTa&b11V?fij44Y@=O4ct}W z6@&YZvpce-QM$94xvCJ9cYL*$x-fu_U>aKdOyy`nGhDc{<}uifb*86OQ;X;T?}kYm zQQq`YIO$;OeCzkz^sfafe(Sn34^J3SIocnqoxLo`q#aRHlJ$5(=lBq8W8)8T(6A28 z{EkBHoIGXQ`C)YuoGxDy;Ls-PykpYZxS~^`W17lYF?S(S3v>q$aE*1@zO6rG(nk%p zlo|XoxHT%!v9nU~bv3}kx=L)ErbK!nJe?S3&}>#y(koC|Sy<E2!b=FfA}h*X=v7vx zx_8i-RW*do>-F!qXntwrr^?SPQlS<=Lc`@Y)x6TpF>USaoKQyIKpifNbkn)k)nFqO zC10D2jd`-%<e(KW2`$LROj42-<QhspF>ZLJH^9x~-@=VD^5@Io(Dl<P@C&&Ak-6`> zD_0-~nw8P@b#%jGHojj#Aq&WE5iRmLx!p<?N&mKraN6L%rQdUcF1|W3i!Qyc!s(#D zSd$=J7SAqrOSIdmBsRGFXW#Pia1R6=FK+q(O7O?&2KnneDo;3dl_}OAdIf88is}og zw`SQntd{t0%JS=mOo_7&K8XKa*Hko-erO6@(A=QpW6Nk5jui7Pq{q2ptF{vWIBqZ< zXV$!QT9q1Ixy+z8*4s7J*}RlwY}fHqlMd=^VtKDb7d7Y<rPqYra*h}nskyV404F9= z6w!2j-`Rk5-!;iMe8j$hwR?+t^|9-M`iQ-_ijrJQE@y9VtS1~18t6JsBOSJUDlsR} zh4mmWk7z9F6zCb)M6dbrCiYIp0bG2yg1YMtxW#t?#ml<NhEE$=?YFZ7ZLMVIz##Qw zf@~dqYeJ1mg~Z}mhWEU?ZH(^he(qIQji}~`HL_B>!jgJm>b;+aK`Kw^^2Eepj0iA3 zh*2cVxz}l~s(zUE2YCdPGvmKjo?$Hw^4rUJV<>^=cx0PJxS$W=!|_(M9N>xLFcmQe z(Y*-k&U@;n{b*fL^F7XSZ~YI_zpbg8w#FHJ?<9H&hoCJRQ`a<(_chspljaEAs{9;| z*+Ki2XKw!j_$sc7H9M&tbqwwD3TE`kw{0Bkf;&C>sab%BA%^2UtR)Ktq~u4X8RKLN zG|h5yr7B+EOnUrv;^N9!7gnWdsJm-b=eVb*bSjA#9UKDLFI|n451qPdB@Z#1K6JK< z;n5V@gc%sVN~z9p2yfzai&sr3D3tDJa+H<|ODrXvYfr~Bi9M)R7CRL>d<{$!s~1A> zDI!ue0WI>ywoTVsU#kg8W_o4MJLLDi>PFdm@0W#QCtM)uOx3%{O|@(@Y{zO3i!_>z z93&E$sB}Bpu@UZCnjGYSY8@?&noBs)NzX;U=P@SDi7G?owW!QC$EfvGfw&p!rI`0a zp$Nho@51fbXXa`9oL=}@X8%N;{7*YSG{&MB<bNk9k(o5q{#*$L+WiM7ZYsw2n|1ou zi<SkW%+HJ={QhznXAZS=Vu6M1dLNzk!MldyW6Gf6;6hGZCj$vXwd-v}Gui&}H|)7T z;Co!7y_NTB%Lelr0$W(S2VNoL^WKOkbUJ&3v7Btud-+noryN80O`gbpX9Y<uQ}$v< zqc`croP2+Py}f!+DdfpiI_OchzB;6KCozGy)jl@Ow|}Hud&Hgh<fEZobM3<hgq^)G z&(e@z-NQ(aBX+-}Iw+ivaY;I+I)|mcJt;HN#zD6CdG%3l?S$fNlekY!VgmaNb0jXn zcE<T|JR1xU1?_zG&llSj(9ZHN+i&cBmgll4&AezRK}AyQ$)zYnN?NN(E4H^%7o&-6 zLX73JW>XV`lI3c^k9$u3=RKy$`^s)CXL*FY<+h%iFqhJ9KI%$OI8q6`svvFH=3t*w z7JF_F{tfFE|4sXAK49FGk0xMK3(r%(-ZB!EBc-aXF&Lr2z2xuj53Z}5&fnTJh{&6f z5IdCEx68}jQIv7}XfcKoY?`c^>9SO^_7_kH2u=24Pe@3>8`$imqNDZ2=l>%leQ4Hq zH7GnDspUZ_N(VKaA>hiybGta2sN@S#i14-*n6<mbznJ(!zMNySXt=kRvru#A-_p^1 zv$ro;1mR&;LP>zT{u*i~*>9@V&K>|BGPpEj{%W)IPg<sc$0Wzj#{#ffdupE%A1vK8 zubH>Ga;9saZROBsZ1<)I23mg7Egue`$`OFI(AYB@1W)iUjqGUjK_&7AMu`yBIo~*X zsbB9@elSx@Ah?3h!DtN&U^<wcNVa}@1+9lWr^u&g)h`ObG!^Gw&cBs#r4T#D>ue@J z8g)U#9CJs(k;rMMGJ?yTlB8MrUCTZ+6MRJ&=+F}IyDP`rPu@{2`ci;J2{XG%Giit; z<B%BKSM%bfr%R1t#)|f3PbXWF02r(chnxk0AXtTfboF7K2NNT<N?G>JR=3K9U#8iw zU~<bsP0)yXEM%i+)<?4&ZGjqNIOe`ztpWis_B}J#p1AFsiXWf+QQ-C9@bKb_Sf6@` zL-|L1hWirrq6cP&AM;U{Kar93%xVo2jj!-;Q1BIe!r$O@#$TJ|87E(#&&HoW2ZFNX z9ER5ZVF~#a|CM6Qr?d|!1pdJNQ#^p%UkpP=v9Q$%@i8F&sO-`yBWuZI{7)c&so%(K zs|Aq<ow(T4Cts#A(=+>0Q_BhBhc53{U+NTry4m*ZIW!<z^HYY{hCP??iErvb>6-G; z2~+iEy(bgS$L%Mb4aZ|gYXx~yIc)YKS(i)uE8zsf2R;0=2R*Fj@)C517pQJJc{v{g zEy+<Hr7v+<%v&i>{uDJb>3{$CELa7ATFkfaUT4oR8shp7ICvIxXH(MvfAd?6Ii%H; z=Dlnda?ba*shW;CrS@vEkDDZOooQb%I(SO@TQ$>??uX26lEAs2fQ`c3h3K+M*(G0G zwLx7b`kRJCWvGQB9D<EPu(WGI$#)zih#9@tH7-?lyMXr2+ZrK+131M6gBQ*$BfaSF zPzM4H-2kS)otV4qQWhP?SQxIdOIY<SvX)B+Y}yje-Bi8HPEIKmv>Mul)o(!V;U=@Y zTHXIu0@&85`^c5Pd}&JfW0p;tgzbUCL#TwktMGq_la7sHVuk)6IGe<15O1DnB*MQf zlnm$ny4Fi4^^P_rB23@)ZBfwu6%~)|Z=Sq3_{j7BgHZm4DfPZ6YG!O=R0BmT$ZaQR zHYH`10SKn+BTzFrK2w2vZ|FoYjTa-ZR7}waTK}8v&UbzvqI(k6oqLmb^aX{ZS|)er zrh^yL<PE2Vj6vA4j)7lY{9wKsoKNekIz4BLRD1it7u_pL-T`KqU5|x?#38l2DLq;x z_sV-ze=A=1e|NC8NI85B!WZDaYmVfIaTzkc@U)xKxISTNF5hQs1z(aZ7Kbx^XLfSE z4mKoPHB+|YNk?#*3ep;6A8_f6hwIt<Y{}73cd3AwkLQN9uJy-VgSedv5)+n-Tz*nF z@duZ-FxJ9W#BiHB?82_1KznsQ>HO_jPG4wzRGj`)54!xz(C(-Q_xy|6>(WuFIi72I z)NRf$g($h~gBDEu4b+9z@9*$ozkcJ<92JxSG5w9Ej2k8s6=3TJuG#Qqd3RISGdF~3 zSJy+vOkD;Mm`~f0M{QzcbT(1N=)l3WuC{(P#BodpIKwGK?prV$(}7YrV_yIxz%$_! z+Kg$;YQe>VJYoF)5<jVNqWghLm@@HVl(s+=tJE%x!eU$kNCmNrj3Nda#Jb8aeh`~R zFlSDgnwq|R`Lbc7Y+(yUid}K#?%IB49&%{N+YsF}USQtt@i{iVmlLZHAWLE%W*o)V zYN}gJO;#l2+>X;rACNWXM4U0)3vl~u{(s;7(-kAB@NBdeUa$Wea6Sa6)`uu`1pu@V z(`bGH0TqDdY_{j;v-KAY%U{%H6HG@TuMx@xfF$T?^rRvAjMO-NrK8iafS12L<1&k{ zvRPtf<sudhz{wfxszOnnbBo!;3RPat?gsjp-4VjU7uk2s_B9nN)qKe9po!Rjao%N5 z29X+^EMte9XHNRcMXuEuuSm*@D`pjXHO{mX;`4LrhJB~ZXVLAb1)Q}#zN8k5(*IqZ zHVXTgPqjt^s(5X{7W<~8eL2eBHcOIfUPuRMq%vpRql1Kb<`57p0{iK&_I7qKg-pMy zGY`?t#3;%)1)O}WR$MA3T^66&8n{GKi0hqyyLa->nap09*TSq6&;bd6S&RIoZQxEd z*qk~u%1^iQU6<Q>@RP&6m#)*AK#bx`3(GPB>+L;l*Dx_wmgxm|D=$~)nn_Z;+1qcD zLU{-^cFp%B$58zmJ}_IIA}+qeyxi^OqP?XkVS)?HG(1R}x5Y<Xg;qpL7Hgx~7S3i; zmoMP#a9+dF@lAA%`p4*>LewJBHjj^rj(%=qW3#X>CQb$(zvJac5y0Nu1t<tYn1*XK z2u)S<Htp4Jb?b^I3b{IP?)q&5`H!!|R8;t|h#uNY!<qE&Utkw164)P=e0R8tX1KNV z?iTC*ubjJT;x)<FXPT(=7S{rvy7{gEA7T0<6x&r`->pqVj@~WNi^|b@l=uCF@VnU2 z5T>?9;gS5PF{Aj2IMyDD<^8`ttauf1rksU5zNT|hj?Xu35^wt6<UX84J2SBKfibcv zy<@z?|GZhrQTNb<)N8lgivDhIYa|Q;9mbfsL5Z^&RcgY+PPJy`TP>cmpN}20sV}1l z?0TxFa2<EzxHzgyJ6x0KDi$}+nmW{)hztV@_i7(C63l8#a&;$~K98ux(N{V-EIDP< zJT%-vo@CdXSF{7G@2K`><gk9BdPtw{Qxpp6*k_9=b;`Cr&>K8KBc@~`N)Ml6)>&SD zPl~Kd8+wKhzxW8`csrqCk!4KkJ&Bkm!^+;NHy#%{o=z0|-V}18aY*L{`!-`G#Cz68 zu$@2s&S@bXqE>0?WB3VTQt4j>Z(4&YdE)A{?j?|(s&eoJ?Bc_H*BP6A4_*Aa4OEUh zZyLbeXPJgQs}l9D!Bo^R<$a<k`;aH7&`%x>JLbu0P^0bsOlLbp#HZ}zL^jw{YuNri zSsVYwg>FYST>D{D4_5E!qT)KO%0uPrs4LAm;kG0Rq;A#<>i~%6OHyr7gR9z}apVdr z%M;MF5Pzk*Z*qRQ66zGq5_CQxz|IseWSOrbk^E|LnDI<f+gHl{#PVu)b{HGl+bgly z-{!K&PtE<aHEq{&q%68M^^>`IJt|s4Nm;q^2}e6j-SU3)tqxEQA6}s~tm=jrpSqrD zE$KWpj-AB3n07~z#I)Y<`#Od^b3p1hbhcO)mpXMf<Eg^*Jzgm)3^1U;S5G;1|8J+T z&yjP0>;fnSdj4Nf#90YS7Dm#9qmmI)XgI7coMm$4K&&)vUMG`3xnwI$x-yXKL4L4* zD5<z(b+p4#N9Dy)3H?Hl7vM{#Ydl3^&I6AZP|wWdIJN5h(H*%)Te2jlk`8NYV3O~w zR`cB7x!bY53-n5MCB>`DtVhCy4PLk==<LILMmol*vv&~+p3?jFQqvtuZr8DWCZ?L! zmt$1_!YXYT?(JJ_Shlge;Conc2QqKu<a(k)HQsZ5O*R*p%;8akN^<SN%aVqksj5p= zT4M=3hsH-!X47)%Y)B2+TiS<};gC9bjhAL0b{IJU)$~8z)q9r~K#$^ikUVR|hZ91T zUE!0CdUKT3il2)~xp103`+t9lQN~&<S)ScxVAT9{5et4;pYmT{4FQoq9v@%K(LEJM zxpi{(2vw6JMAYg3`8%3wx9~G3%!OUJ3uW_}GRv#Z{B-B>3f>9~5(-KAIGJM?w(CVR zN>@~`kw=Q`m8D_tsZAEBWZ%m}W^c`?M&cu#>MLvK1aQ@h;y}6rha~`4^lSu7O*%Wk z$=tW8Y%>YIA{r2U`bikLDhIP#+3(%ydED>Ob*8|YL|G-B--@ED3af-@R8&3kl^)#j z0!@v393#}1B}}6;Ce*f4xB8$&;zEC&1B-k6_fM;yPb0?46mGu0Z8V9>YC=P8Jv}}D zQOV(E{ZaTCxw*~)kX&3-MO30BwHWGBl^PWJ3r5l6-kR^W{{8iIbOaxUmD;sfkGQzI z&kFjOXL|a>(m(ntqnpjGQ_<<sqW&N4|AeSCU!NJpcGM3G;?=oOxg?!Rayy@RBluHV zG7bi2L26p4?+t#xVVM9WM)_2|>w(mQ#|q5UJ;MDBcgYJkprU|DM?%gX1f4k=S}2;L zjHEXHajg?svMHTCx`QSdzog*iYWmDJkL1CW1K?dzq7y^6vlN{P$=`NEcL2M_QvvZ~ zSYy2bI?D*N6t@HfUmvjSuC=_(H|rX<+to^XqszA$JgrH+^sNb`({{GkszKNzQEkvZ zmMDbE8sLjyPTmo8?ls`Cs|+b{j)V-dOob%cMue0P=wgx_5LF1K&>6lG8pasFNI2n` zo+rRfU-jr9i_C>a%}+Hk?t)vGjFgWnk#pPNmh0wBSmj;$H7I))&>H3Qqx5qtp5NR- z$ykV=p4-3rRPbX(tg)#}c~mW5np5ePN64_W$)5Mz4gn@#k5fKBlj_V$)2>ji!!O$> zMY4F+>C<0i$pi(3vsO$->Ouw}+#!eKx?kCFdZqdVlMKzu6ejrjw1=KJX?qib2v>@n z|ERE;Ec(cq5==UGJ-jHhS8NwA=1N_iFcGdjJz(@89Od{g`i(I6w6FYl*^WXlwll0; zV=zjCN>s``j8pTBhxS|}>RJAmwT9q_yA}bWx@Ty9lxx)t>uOR`-vR>z4`@9)vnAN3 zho!#ozH5s(^A8AEBqaRw+#bbUBGrG}?MM9+pulpZp|I(Re~slUNMLFr0{QJS{21)q ztz8r2IjC=mUK3}={0jFs#+I+~i%q@-&xCx?|9crI*NAX_4=SVr`4%ZCF{!8|B`lYk ziYnI}v*D9vhKCy4)32KC=VT@v^>n@kxTv5!p6-tc@cOGoN0UPpl`EIhpm#+=)aI0x z=%SbINhBr;#~}7_ZUFtFVd|_G$9n^uMR~TasLJfl(}ug2N-!oSa@Wzd&2_7Ef2l4d z`}Kkf&HkKLnt+*}dGnZ3rIDUNvvG#OvJT}OcMa+hwn}&@$Ei;^9{}@fj>Z&J8p9}k z`B95!!1J4Q_)c@%ktlqS+4M!VyrV9nE1SCAjI~;xwaOyZ7O+smg&vm<b&c0tmUoCA z4oq-g1u<F_I7lzKxG3Gs7`i%1*G@GT4`WTl&pTE3C<*3~IVE3z85Ee(1VLV(fj4Wt z`2tDQ1!U?~h%z?CjQ#RAc*`C>lQz*+j?xS{X>9Kr&Q`0_4NgsGMmQi-L}YPhnfG@> zV}$bgRGAyj0Uql4i8h3Y%x;Hlk~xDHD%lR^hh>Jdzs*1*#TUEc=I8b~HOOVQFZYI# zMo&Ia)BUH$L&z>qt-i@=Y$IY3+R!+}f2r%OPCmh&?I1m&CW}p#A*PYF{yNTvjX(Z< zpBXB(%_WI$@Lu6YTJddZaoR5w6-jCP6PW(WM9Og?e+U+)p*cDGi9^4x_mN<VSz@#^ zALYSoX~m)q6Ya+2UHFa(^koXu7@|RMrxEh1L;Ug#c(?;lOD!;?XoDdjMs{vLkXEi6 z_?=QZ=9SfCWt$#JsLc|(MSr|B;}?YW6rcaf{{`5A=~#HB-hX+`j3M0q@%C0HC(cWC zlGJmxtqG|vWWUu4xD_$A(nyYuOXUDg6+<0vm5%{#?8xdw8}ruLnVOZcQ|B7>{nj>J z{AO+@_2ECoUhhP7w*xo4AV92=lu@CAT<{cj$-Z1v^Tk0W@Nt>1#MKjidp`d1tDpR$ zRuF&{)KQa2RVexsGvf#@d8D(FHMOYofxf+di~nk7j46*7RFq`f)`Tb?>Nd0p%AcE7 z4x(<-JdS3mu3xbVy#OLj>4Dqt@~MYnu6D?PEv{lO9f?3i-MGBscWqnpdFmlklzS_@ zMp{y>j@lp_vfM;n=f+=qhz12K<RSFi39jMFVFQ_5NY#Q#qW%=jLgo2N)kTXI=|O7Y z8n!5WQ51pnPArS8o{o>UVT;!<Sui0tTKq73_9Nr&lXu44Q1$e8(M)@jBw4c1k1Y~j zHM;U@h_mRJyHEa|-q$}%q$!C3B~PDej>HY`d7>PGG{HmIkzhSyfkB<xTGY0ae-aJb zyb}E|9pC&ov{!uj<cIs2Q-6H$O8nnuK(O8I?>jpChdSBhGvp0>s~aTmW;Pzq8St%{ zBFf@44t0Jdp)#miKz<fu#I$FrF+tC>7^^bBB2gKrNKsAeKZE0nL_k!;&_80iFx|DJ z2lP1K_opSP(b1=QRa$T4l>Oxv2zxvUcqdU2WmiI|?9bNW(yuiPg6}7(IGL<;(6uLX zo)HdW*?YVZX%r2(f<?R_eiWpIzb>3;L38!@hvX?DF3zY#-TvQ`6%!NJai6P*wI3O} z9CwR$@Og~9Ed^ac=f4HoTqx>(vBw$owy&xq)7jL#WrT;^>x@g(RF4*|Am{0&8ildg zma8*}4EYcx<*yDElwmP28I$P!(m3=}p?%6qt?pAHn*8!}#UL*}V1es}>a07it~|eE zHKA#s!J(7h+?Loc$fx%jcJg?zzR&Sr^_u-E!9<n()0Vg*tpdl+Y8G5Q8aNY9e}Sbn z!7V<#^_i8$ON&mH!C^sDajhUZxhbRGonM&uqzmnFNXQM?sPTi!VW%=~v6_R%1?8r< zap_9+uwjUv{<z1bV+ybiJBW1Ex+$vFs8SKVBmKBX(t$RO6mA`=xpbbfDQXlRxCJ)r zh#7Mh*YHEf)rQ#sGq2{WN!^8;xm_!D2i<1&Zt(BwwyCOB^s5tMDw|nj+7<L1eO5P- z`3>qh{u48IZM(>M&j#;{{Bo0ujiyGoe0~rfy!gio)gPKd`z)rttKgu7+I?5)O-5X? zTK~a^op16N4OqqGUH&_>SIaI6qJLLpXLG#Op9-JTh<pEzr{M_y*7%r$Dz8|I?a~H_ zecyU|db)60yVn6*b3U|4For%4{G(!n$Fd=JYK`(mRTxsNVtlc*_t}GuV(W*#0xm8u zFGpz%B@-n+i~p0t_>XMO1(i0CClc*q|M$c)_~aMPm9KVRu%LWd@p?#Smh-UtiT#<; z==YSMWZ#&Bp5LQG?#frR?k@j^+LQ5#ICnStf>3>O2I>@})G>w*8Mk|B#Mw=$*G8i? zJIzT1({jN2J7TznA@<g5I7(kbroIjNz`6|R6A1C9k>$P|*T)s7hY-gSH#^dyLHqsQ zFr={g4+1*^*l^IS7Z>*^x3<-#Vh63zF0mcvZfQw3e>`nm?5sH=6MVa!wQ1A};lGTT z8<y5;7VS#n4*UnZsJ{OD<gbzdUa!Hz59I6W25s?P%>BkT&DQM6^Vrkh)~ikobTr&1 zBXtRISC-SDq!e{S2g(!Bs!L^2M7BmlL$79?U*(Ylq?92AS{66xxOZgpbw91p9SB?B zIqK0H=fgYLiY3b|m!sHA0o_<dSbGZ|CEOw)2e01m*{|yn$YCyyqhMoYt3(ICBznWw z0qT{1t_)s5#tmBx;;>3Y5&Qk7bcPZTC2Ar))Q7QAD!Eawpc)<H6bv%JYDks^`k8{s z%h6eThPhGYiKqKzLpQbxuFeXlc2eqU5t<fSiu#?pX5exA+i=qGm4Bar?)QK1o2O>Y zhDF{xC+#&b(eMrMNEpqzO&@Z){@kv|i{VE*dWT90$rHv$bz=uFX=OyxPz7+Kv^Ndm zyVF}VYEvaCPjo3};KzIosb@E|Q_L$$=C8{KJYQCE>%6p*9L-%sx$y2k$E#rxEiiBB zP-ZEOa9Bx=ddFR7)K{TW=Ndo;79VC%fjXP}`%coHrl$E_W_?ixS?a5Q(D1ANNN)OL z6Vf_kdX7#T^#!AwiquBz{_*2m|F6IKf9<pWno&j6MRHVC68OaWK9%5L9$X>KL%IPs zrfpu;(Q75-Wi8IoW%hUUZY5K4)lxg1FmenWR;YfiGRyYl%t1f}w&{v;+UQLY$~7yr z0)ia%ww@gxW`V4JN(0`pnxOnV7Zo<W!AmXQ58rjY%BXVcy4(9|T{T0+r?t_#qI=vz zi(FzhFq*w%5nxep)v4U=5`UEKuI;R~fEzhl=Cx3%17l_?JHR0mWlFV&;B*K+iAG11 z5N}m&?z`{<{<)DI1W8?_v_-2;!4MWeV~YR9*m;55uN{Ypm?nvaMFWn*U`Z`0Z`Q_T zyiI^%HAU+$M}ul$cK3Fvc@_h;D$eci#zH^=e(kvBM|-~m+3NTn=ddlC>Ud#Lsi>;G z`)^dwU&+WN-2B12<MN)rdbe4JI<>`7>v#eQvjfgCR>M-~7qV#tRU=K#c{wcGhN7$; zLw({Q?Jat(*e)cel3G|g)s;2Ju(=;X)y*{yp`vmZPxaTd<{GvRE7z01CyzRW?~ekH z?+=3;d4z@igye;@M~gz)JU)6%zJJdoK|+F_lIuB1$q>%+w%D3@YuXV<c(Ls0$>%8i zVjhmPYAw>3e1aTR#%}w8_lovjZ2KIQJkRrKB$gAKcUceD&B~bH-TWrU&X``YU!*pU zedsu+kTIt5(<l`80Lea7o0MdGE;~LpHd4f7?xpLMn%0^skz!c?N`?q66_hMxKYFfE zr+{hyvWIaV4<CQiR2}kJoz%o}RjR$@Z+*#<@;AA&h*XUp4e&SReD!Z?XNIYu4DkNl z?X7z)BdKgLhT>lf#+{Ofx5sS1n9#W*pENIx4b|j|nlC7yRWub`Y!GQ{v3&fsdGN&5 zk<>axHF5Q!^crZaxt9QbUMVF-sEi>Ona`jBrGel~^XW*8E%OOet8!3!5P83yo5DXL zB5eb6O18wkKw`K+U$fX@8nVvqjU?M2#CgPhg&7T4f}Dh+-m|-Z>L8PE80VUB<}P~E zf!MY*HxZJPB61h;W&h}lwO+OR@=ey=S|K{7Cbjq*AHv>`)Z|shCC@4&$CIaen}P=& z2Nvf0PpTt*k9wYP-2PqDy}RA;JPEXTXSp&nbR|o4)73&!Q`?hJxH<kBSQ*BL9A9^E z4E=WS#n}4#q*AZD7vMcb3YC#cl4{(5LBNjs4*UIj<M=P;ZdtyrOqqMV$-q;h?wl$f z+IXf}C0svI@-jFZT4w-x=?b>MGtQxmAOMb8QB8DjL$^X=!gg4-{b=PP5t7=sv@$Wn zpBi#IlARHuzBwMYj0*Lf(k-5hFGU^Ix3t1FSC!IlGvCiL;=i^ndY<lUo;QI1j)=T! zwkh(Oi1Lk@X?&<&#;ZFBo8`i-cBS=L9C2Q<-&v=c>`rG64fmN`OoBkeKV6`2HTl^a za?8duP3;OQpq5FZlnHf-tlS1_P+^I8PZ9IO8Wn|~>&DB+hZWBj3m(&qG#o2n;AGC& zV;10!+|raaI%Amee>uz;n#<uUV=DVio$XXD^1Ho+)Gbi^XXZ;I^+D0G`x2XUdZotg z#>HgvO!HLpw3YBauL_pen;x2y3>X@aFk<@|Zhti-p?}F6H8xWj!Lnx)f6+@{S+cdr ztju<7hR(zfpk&L>zf*B^w5Le@(}_N@T`<JdjC*x4Qd3iF8FNJZwDk9)qo$#h{i01G z@`0m%^7peaO%rE+E{Rzs!UMzYIDc_+nd~1{X#+Hjwp`_{s`4OtT7m(X6{MVc*u&^+ zXCc<`nb9XQqwsD%*&Va-IUG9+q0iXIzBub|ewF{GruTQH;C3G_Y!*Jd{JnO9v}|MD zFCGZOe?LKE(PA&Sa&xYwe!q|V)z|O!foU-H9B0ocn%4`d^rlxFDer$tM2RR%TO00L zbXD+6{S#3)Oins23i?4?(7TWZ@_npC(=7VVAv9#4`y)FYPnyxi*Xgrmhb;GuGerHj zc@qqf8X)XM8PCK+fj^zt&!&jchxc-RmBx9Q3%lsJaPS%<``={I<TXY3Z_#;IgShIg z(s>J`?=l@s&RKdEIkK-YCq6iHjZ384kXioy8J!6-PxE8=XM_Gc`R-R-P&ZtPHQvmW zU{#>$6+|Ic0lQcu&MRdoAHkASq`XWO(0u6rBQH)nTH!T+&p1oG&r&a@Vp(Fe*Og?` zclJ4Ij$ua|8<Gkti>xRPM+OfW-}{ZV9g3RZcd-<rAK!ncfcF-W{XnG!VaOdbU?@|4 zNJAt`*|N%HohiI{%i0-@mfow*C6@P*q?HrYhy5(3aok>XVrlwp!NryFIC%efF>Cpw zKjAlJNX6{N0e<pF^`<H9ge;LJ?N2dlSz=GB#V3|8iOc@TYv(HdW_cm1^n!xaKUKzo zh__#vI555J-GvIYEB>fJD@{v5Ng*OEk+uXk%QB<pon_TQ%Hf}wyvlFWG))ANAI0Y1 zU&`c8lEDnfi^p53?pyh-Accg?O4n=lx;TlXS8EEcgjX&O12u(8cvT$3kNAwAJNXIh zmNDFi29ofDMW;cBq23jud)9OMJu)<9wcHdsTmwQsIlRow;su`WKEMjhLM3!4XA%yQ zZQl<QSn(Wu;*$)^<1_5atrPC0e5gFs*l~=8&)q%0x|SWtZjLsgjDQuIr<dAV20n!e z>;54p@=q>4UB7@jpk(0Z8#bE0mBJR4iX?w1zK{vgNpznFTQ6MGL%}rE7Y%DwB;Qo9 z2U$F*WYeYuy>86p7Gtl~x5Q#(7InRkbl#1DiT=mZRdB`GG|M2tUBcq-?(VQyAS~|g z?(Xhxf#B}$?gWAbhv2rso#1!h@7(<Zd(Jb{)7@3wRfWJct@`|#on}~PmzyXa3Il^k zuTvYUbv4StGL)RJSi6;&pK^Kc8V}aO*#gY@g$z_B$(nao)RH>2#$x?iIsZ|~mOx7` zsG$Ft&#z3YM#z3Nfh|tu@GMcNX#kF+c<6K`_1)9>WmFVtE#VAf(Yjf0kf-IgcYC(& zhdyp=zD|Z6RYVz1`~`-a;<qCJdUL>Iat6{j30QwXN_6)q)U%YeImo=k(4fhevx|$P zC3i%r%2mv8g4C6JCH41zK}I$)p>&5SDLJnW0}NDm1yZSezK<7&{O(lJfYK<~7(juE ziG)OA4xLX~MMV^hwWDMG-ibGB_UG~!C7=!5+1JB4tp=2g*j7xNohYVqG92F;nRHKt z6F(mEEjQdYYkOd{HNHTPrwLsRp=u=z-;Z8A_=Q!|88WnEG-Ij&bf1;hLdF7o-w=WZ zM|)f!;r62OMNIfRy}25b=*3YVSx^dvO(hRXp8sdZk3(;!z;{$S;SYf{125cV1vLz7 zn@HexD?qOu@kMfot&1IY-}sabpCoKJIhe0ynzTD4w_(!&juuAWD<{jiHiLY*r!XXF zB^}e+B=WUF%dNHb8i6jRs7P6!eUS0407CC6l4$Q7AHnOY>((qInvEMW6-^^0TelPb z_2_O8&-gqg&1iJphQ54iPBv={s13L*n!Xg?nxS5C#&BL%T(wgPV%rxwYjBVX@uO9- z6G%R13!=zmPwP@F_@Vv%6zZYR9^rSq#pPd=+T$hir&C)e{|kFNUr8AAskpgf<6+-m z6%z3FdV?P!Q((os?}N<-W#r)T7&R7bZ)F6R`9r52zCX=9mE!MKDSaSI!nbdX0p=2} zY6m67cVc^)d3hvI#oN>&!d;EZA<Tz{Dl>H6SoEw>Lrrv-x8ohVa|7=hpBW6kMw6x8 zC_#*iyjNMh6)$l(xL)>E5jfO$O7&FdW5mG5u?aFr(&Zs~ZgDDLZyz54)+6cj2(Y7$ z(s^XXGp9W()g51Sf_4bA{6axX%o9s?4K?eG9T&#o;*rJS(9{gJ3odd;)kLGj6ejx@ z@d+0cnFLq9m_HOCFd(#6V0X1HIoe`eA8g1QwHrkqH%>aRn(zjQCmB;4;~P^2HVWE( zVh#|tE7%OZNUycJo|F#hgCD}Pc0xe)^QLFY5BTgc+Ygs&X)rd`|3v(LW%v<)9Po%| z)!&3*d&iV~Ocd02pUl@Hj6a^T;1r^tsANI4Z{|a!R0cE_;3s-%YxKWs2-CA?r*iB^ zoegJ@)#MNx!=i8Nt4z>C<+S5%8`a<?rTXkB;h=&9zpy`^u*ohBlm$n|%#;oWi}Pg* z4y)QyQ$X5MEA=Qqv&k8oqg0%?@Q?J+##esfRc1tq;QfG>e+DB;7Pt<PNoJb~R<{Ij z#nTfmtMPv2{z5I8F0!XRN`*7GR53rsl4vf)YqW)V2oHuOsR1^#)?#n0^}`oy?MeUI z(4M)^*-B4iu$wk~g8~StE-hNldmX-QfG+1LInd*pXNeZv2gxf1FQo;hGtQS=q^m}- z)PLe<iBQ$`CZ>Yl#U2qo^pF8>zC~2JRelP_U5ln^RSPPFnH3dudF^t$+G>cILIUlP z(;~9Tk*S~3b~1GumD#BolBAUR1qW!qVdJtConN$}<4h{N)Kphb#nOWA$4Ozz!jCd1 z2<4B_29Kv$`%!dtbq%7Q!|@Du!UrnAD<Pkez4G-ZJB!)8$qUEj+$2j+MGw!BKYhpN zQpG{@$5WFmtZPGizhK<0I*wTK3D&A^k6Dwa(vqQAVj=jBwgEcIh|u#UFri+P&EzTG zy!`~n6K86*0Q=yWGjz2qcZQrlFp(g+ipL#2!#!+uwLV+}Z_aYKYSBQF;Al1QTB&LD zu#(wu*B&@l-TzMf<>S3x>u%igilF&$wijxH+Go|~ELc7<9*i0@IB`TpBsVb>{e$8M z3;pm~jLQ#Z)}hJ}lsIbXLWf+^WdhlP=U3M`uM_VYe?|PKDb(ansfdzxYAfYtv9=h@ zy``+bT1Kgr%tbKE1zfa=E9qqQR}kBDET32D<E0D8G&G@rQRxM6BpZ)>sxBVN8hZ#; zp#daH`Y^RTaKP-j<=8TKiV`_5|5!ZHa4MP71B3tI-?Ia%JcWgsg`sm>Pc%8ke0Zco zB8JB_)qyiyVgoQWj287*?M#a@1>LbCH$14DZ#iZTIsVj@dlPqrpS(qpNBnBdKe+$v z*)j6irDO4NjJ*=x_inFV^$&2~D}R1tD<@ACvjqD&5B$4fkVc==O>KFHf^Z*c+8?u6 zIhUY|7o(XC5zwm%qGJ@NrT$}KW_Bgtbr4UbAS=5=LR_Vz5Q@2;L6#wu>~@`%OI}9! zJJ&(JdCGB?zI0_lG0p*uGbFo3iG)pq&aD2YRD&G{3pkoqVFdS9v8}$Gy_H!7vk;&< zPZ44}iqsA1S|r^X3lVWn%wI@9u1Wvpe>{=CKFtOm&E^L@TfaWE_m9>G+%sBPTVL;M zdjZ2qWTt&!{&oen|M+;K=z{-!KwFxu?DhV<L}*0mELLE^#@c~Z)sH{eq)fF@LNQ|l zuVRM<y3mQ?YWAc0%T+jX_NnVF!Q?U9`mSCyVlaOEJMpmC1p67PXu@Wp%-o)6j7z47 z*}9r>t^?o>54=~qEgH%jcuOnvSp6}0`LVz9D<E{`{YuEJzG}$2lm6war`cL$!BQYR z+%nC+m=(j=u&Og15B387XFiZ}L~1E6pSnsMCBB>$x^AQuA#w+k0d%#cVfgW_BY6xI zYp@khvpf&qrFrPCfVnDHMOLxCWP%X4s?^^s%GG6j{GaqaS_LM7j^u@;MT9Gk1@-Od z%N+2qa>dDJ>WO~i0VP_VY#7V~I-71DKk6gPZD3D?z8RNaQZM@E@1}bo0y)3q@}U+- z4O*4i^TMp^)yzka`t)jr5mK`UO^el7Q)pDxMI3k&_xw98FL;8;olPe^X>co$eDS4C zq!B-+hs%&Dy~tX0AEkp?MttznJGy43qmB}qSVpy3H<lRxTyAFI={I}Hv7Yywm}hp` zt8fwrYf1VQ4Q2QCo3w82xHy-b+-|2@Sh~%I*Z>$EdyqVs7#FH2gSi<}n?W{bDnBgM z)p0+X=+P4#fNIwsZ4?(Z$0=}(u+eV&W!lg_hDi5Cdg}6L)3dY3!rv?zUs5pRQXZq$ z5tJ*gdXIclWYVk7RJ>RY|88c-V_^AEQ8?0uAEj8{9gBYOw+FtC2OcDT+<p59g9xUk zYpNGJi9p^W5G+wz^*0bh`(tHZVv(G73_+1EzTz^0wbp=U6DFd@p=^7tcm-1ilr>7q zVR#f<f@F&}77~#SCNr}{(+tVt`VaBhNo26TJ5)C`YXNK28o|R4yL0cB&qR`SlDx}* zM}}PrkuyY!L;BwzRlOc~znMPeXNVuN>dbk`qYKQQNWNziopNiCkNP3o2&XjC(!o=b zKLJpu$5zKszS_x;E-vE`t>;AyBjWRA|6-y?>@7USv^7r<qvp2rQ9%8yfT}!I(3~u8 znVy39h}BxHeeKt&k0OgM`=evrG7hytcd@onQkxWtMnfnjx+uHPwn<ACY#_Bi+Na-b zOx-*#|5JlXo2PKqnUurc!?;_U61p`N5alpvF^uI_(-8-UR)h^nQK4Q_U!({!DVA%n z{_@06G&-70xnPd&a4ria2ir3=*)=5_e-9>0Cj$tTqyEh|*xmXtj;4%Ml&1T^EpS2W zraxE9a*WNzA8Sr$c8q+FvsulZ`c0sl*_*rEO>e%`CSC(qAx_>xs5a)(o40s{l=EuH z7~3#T=c#zbl~}+f_G;_9t5CUy)kW+NOLB7ZKZU>qBuGUoI)BC+ewEUm=E93eQ2ERf zKBHob``lEif^(0n`1R`nu7kB;7eB;A{7PBdM=8_|dKEHG>-3Lm0D`GF8Tp9b^!;{0 zYAp4mn!N4NrK>AK@InkRH8u6lS0w<<*|jFk72|k6S*rk9GWWMX-nOrs^WRY1I>}!S zSp$Dcdj1tElKu4G1>$0t{d{5dn~jbxN;JQoguQE=i-+e12a8da5Wws<f;J_<!&YXK zzlpFgJ46%ul}cyDB$Nwku*r)mdxcD@_>-K^%iPXCs4mTbiP~80aY!F=7!ZY;mxJM; zHUYQcjgVYADC19Qm*F*FQYg%Xw>Bs_ikIU?iD{V?qh`v}z+?QVbNe-I{h(lrpY+_| zGx%;=GSiXf0oKCg0|HV;mKwXLVwDZ;JF`)Ifp3>8v{xtNnKZld_r>>M`hB$Z?Icka z=EhXWbNVxk&co_B++VIa8b=K8aI1!|285HP#X}6qvF<yTr&0R)n?!TrVHYy6#V~I$ z4eZWA)sqZVS!xBpj4?V<fX#NE&89r)gTIMNCVl7jT%p31D5KhK#KS;{g4-^b4q5Sh zu_icttpz_VF=_)|8d8OMAmV^So%q@rrK=^lsHd7WMsIOgMO$+R<B!{?$nv<Z{BbrI zaoJ?P$vA%@4f80;Eg(zfV1P<@_EzCLai&j^;U?N>Eb3f=S%L_v?)!5pl7Y%iMIPA| z*TW#@R63s-aS)V0Zwr90c*&jHXUKC$Oj>v*moK{d>{$>Z8$g1G$qOpvg=o=UXh9GV z3^pIp?0dW4D16>Lf8o9wGEK*%TBZ$o)3>KhzNuO<t9O-(x0jYuGchrBfG&s*r$O$z z3{p~L@+7}PLZG)C(<Gee+fyyDaD=NLDF{$NAj)SH0xHl1PH^ylHz}9Vjp_FY%3%y- zD&(Cp;3hSHr(E=n-oWQCwTKmU{qINqw(zi{6fmGXS|VG`&B<v8la5Q9pHw;-LpnUX zA$h9sx($V~)in8hF45(Y{-9~lwZ!=|ApnoF!)}aVlj@%>(72e3w0fy2$%rk;ro6er zO)$i7-1uPqQaW!~geS@&qWgp<tM3qQqx7A~g!7ybbhI2UpNOXB-Bvi$jXT|EZ>2vU z@qjCcKf+Tu)lbq3)w+tSJSx@_CT{(uidbdtA~z<O`!4`daTq*Cb%wr{uxUFy-$9Xt z3Pv@$d97Z(DH9+y!VddQmK_IoWJ&L-7)ZnFaCG_okjqO%W-KqYy<B<|DW~+y#A<R7 zWh8p+VAGs`v``==JaFd_j9Pwp5w$QY6>m?ARI8ihuoS^}OCy!D><p;A=1YK``r^T4 z3CY1o9$A`=w~h$z2Bub51Tw8J1~}`j6x&rK)-<K7G>YT$htq7{AhfsgG>k4W82{|H zKs+hS0_QPPeopQE=wY7=zoIgOwK!inPQ1T`+M`4=B^R9D`5caM)`G<rm5B0yqzoo@ zsp2MnG%G=8<<dEQfk3Me%k~kHbPfxitqQj*$(TCoqLK{Rd&AgVX>&edf0UKgRf#y_ zqdxGn3>v&mATR+@eyVGf5*kQaVzHYx*`?Zt5=^gNU+P31A{HWnbV-el0eQ4QgrA;` zD>zt2l@{Wl?%zsY)u5HL?LP{4g<Dv0NThIseoo2fsS^-Ir)q%5F>*=WUUjv#cmEqH z^2)mCf1T>Tf(GX1=2E!nfgACrpLuwBZ=*$H0{FdeO7>`OB_K|OlEwinzv2~*nnn<# z?F#hHIL6?H;XwyNF<!B`t0rm*#9tJ(s&kNd)0Mrd?83*J=)z8y$kvMRbWx@nBsk$j zPIMddP}7R1Z5pO)kaowy&%Eh>+vfnH9txLzKi!M?&h2y~$M<boJ=n_J#(X)ua|FIj z(%ixFn0P9~Jism$IlA+QE<m@um=`XH#@8wUFHiW~YGS0~R|HDdI>==e;GU`-uWfn+ z)oWLSR`+#79i{g}oh`~ydYvHn7cHugcOmXgfrSDSW~U4H!vp%C#?9=cx!#zu5FBSr z0ss=PIUW^wv@kpuo_iTp)<UdvxP%}FH3q|6CDg<t55)b)s29hMmsvXeC4lQisW<`& zZ|#nJtpi23)fAIeZUhakT9-=hN~aA<=bP=&+h<%HR?abJxQv+b=UzZAI&2vZyV#P$ zDIhI_nR{;@cG35)rVqW$UARGJ`itV!G^lSC0g|)kifP8_V16S;Bh$G3U@tig;xNY! zqgQBDvWZ}$w6#-NkO^x;EFu;HwNN6OzMwnN#%F=Qzht}1D~HiaTdSOB>-Oga&C_b9 zmCK*&o}Ha}$+DrwjZ7<p#~0yA{RSjwehNOv8oomR=bzEpo+d<J)YLq;&yo!;KACsh zEahS4!sjEsM+=>A8uH`QT_2JW#4)c^MIl1x$k}72;jr_d{9S_7coe@%HXY*kQy$!7 z0@JHWWYq12>bF?bATLN_d#{4YG&RJtAaAz*c#Tw$exX`%EfnQbFUR#9SNzj63<}Jq zqk+P)5ueN0^}V%tsUO##F=rlgE~rku6>Oj$eKDZo26t4vCW?DVvXQh8h%?JcM{y$F zEvE^clECfgYXamLs!BsPe$aou`Hrevz2RHP#QU?`93}LQ@c>fRjO0&#85*)SN|!X_ z3FPREgyldUg*sMCrF?OhuUJBAL`hZH(fVzz;Fe19GGjQ2R?h)NdRQnO=ZTih@KzL_ zcEaO=p;A2i02$CBUkK4Y(Clv+rKBw~Rx3r4X;!RXv(AAQk7=<3L!3d9(YS>F9jB9; z&~kr0sN|D3AANpyPl>y{1t6JkRtg*JyB}3Y_+Tqgq^X|ZL9K9Cp=Da%C5+Z!4qG~m z0wh#t#^~3JkX45nvCS?}o<_md(U?eIqu5*`RbP7&f;EUEB|)EO{|d^3E#V^Rp@u5{ zRz>HQ)uRZ}^6<;ZlmCT`ft!t7Brf1B`EirkFUF95g8-GihJ)p!BDLGKcwsAY6!-h< zWAy%bQf>;gqk<qTp`A28%)CmwoCS7IxpDLq?4@~@$!0i0F^3k*&G4ROOJVoG7bW>Y zt@6w<<T_>a5rWzMibs)n;C3?#&up>xsP^Iuk!0%%AA@j&@@^x%dY*s9^@Vh?A*)B- znY4@%YuCyh+o{oxb|}ZA9S$EY`HHh32XPz47zaWymS6{S>d;?%cfOuBlo?iLsVivn z1C|Wu#9!N?bJ~Ko2UD??XKmLq)seIW`Y1`PsxrN0P-AYn^R=f@W`b=!4Qb!h5D?## z=R2k8ja$u^t_t3>*U15Cc&7RG`~-@ejoS9Yafa-X^Mo@|?DXS6IjR_Y#<{~@ylJIw zP@!Q2!}+<@?`W}Z5VTIM?c9-2l*gmonVV>s+pUjRuW^EndX6IwaTjUvZB$Sil0Rk~ zn$@~%;;Ee#>tVFgsrOF72rl;eOqNMvr2D4#DPHz3{)q=uR^M|HxN*sKPX2YXH!OG@ z8(qpZS`162yb<-}0G@Jg!Xkz7OJXXcJ2~9L>=gLPxw)j{yx%*rVeAOgpOf38Ts~J! zAh$9W>Dgimu>8l4v^HB4a=j?<?mW4?l}4BD?)pa|AcjX4(o5W}P;Ry%OS4cC0U3o8 z3T95JCiThyNgk%n#_W1&k{zCIy<DN~1&~;au-NvTJNjVJdEF9kJ?4tp98;eB)bh(0 z<;ej?)_paa_{L>K3zuwJfHc}8e8PoArqB`>sdo~zqaH?-Xg4xJiCJ+96NpM%qtQMN zhbma3!V$bdXyX)UjTmB}9kI&Ws@(Rf0U#&*w5^c$rzV%{W-zIecarEWxpxwNtq%@? zT`SDW9M{cS%T#O@pI^g)?wjl`d4`0<BW3mu!P)6JgRAF=0`#11b`)X`G>Z0dffOq= zNqvs^o3>(&@@5cMG2fY69kAS$&;~VTo;rFc;Efs3Dm&aDiTtb|Id|&wNuLFN&aNMR zErLHv%Pd?;Q6}%D(yQh}xmf6A7=;Y-{LjtU^^g2Lb%!lAJT^I<JusR*Fx=LWsn4ID z;(66IcmR3i@l@%OFD=8+qLB{b<!|d+A{NT93RpzTRVW~o(r<}F1M{UoGYqY{g)w~) zW8C9#;{dCz%A~o_FQ^*yq}eW2^pV;Jz6iq6MW)$xqiM8?{za9_!+v}-^sv><#W1!S zH6i7{pl#jW8kYD-y_F>CoC%M`ban#({12h4X39}wa?qi3U(Xt>q>^tD9Az38F0uaO zgO)7`s4pmG8?1p@&Ll(AF&MV0v3wgocI|aiCDKrBS(eQ*aJd?5QBCraS$}zIQfh;{ zGX(XP&UQK$0bh{Sc00sA+iv{^{=)L7VpA~HnA~XAGS%9^gumD8gg<@>qFZy8bUfp6 zlga2L96BRvT$Uis!!i$J(v*zNrUsVO(Q!Ki9b{qQ9`9m(8zHYIO$VBy61k5%ppCFb ztk;dQtcH_q#x7Hi2usc1_`T@MR|9sbPH7IGAJua<`U!C7uqdbYLt)xPKzsRy6=T}x zvyC}=M~PTzTn5ypq|!c;&Z!zUD(CkZ4VLW{1iJFl3oF$|DO|y0A1*t-AO;6cTTdTh zTol2=%jndIJU-){;9(A4E^BSd=^2<v;P};Ac|Jpcx`eBk+iY=E5=0^Nc7|+(#xn_$ z?-UP=i;84%vIz+Y7^;quAmTc22t_Rj*$5>!_lsSz6wpVKRj}c-D@)6i1Dm8IU2R&m z`=M;1UL_kI#RJ}=F2s)%+FQZOZ$e&`(pP`xa88vLee!lN*4y@kx&HkSue&Alm^j5W z)el90D;$LTzB*FCGgg%`XpNj=@5y0nvOG`&hj^n>vy2=$s9QGYr%ik^%N#ZutI+W5 zj%pX6iY*s=r(9?jqJ+x-V6|>c3Zu->AVD75b2vf_H~45mi`vte31wNAKooGIlEtEr z7AviQ*(8-3xJ=H2;JD~1k@0BLvceNZM_u@W6@NXGFC4kIXeXunLTTvwnTC0f%{V#e z4TbCUE~N295gRPhO>}6nF>h)Zq>+kpriMOVA`aMK!5zLaIcsB(&eUdEk5`$m!3%e3 zU79d7iL}7A4KPq1wHh^j3y9O91O*qJ9MXW5n~!i^aoam{uHSpRlmK(-`$5S>!x6tw z>_FAFlXaLbExf#Yy`Iv}Go5r@#}GrtM9;f|I*POmcX&jAe+m~N*3s$dly8iR@AvKz zOJvB2APP&(w`NMDKun+-%%{E>C_9MYtQE-Ll!lulIO%+5jfSU)8^Scri>RfMmTf+l zj#0#W<tjFQZ^P3|3&Av3iPuTQ2J1QFGxy<n<ykSntk6x_z!g$QStL5Uz~%A~Pc&_Z zN*u!oeY6a{kueCDDeo*MS~Z(I<Qbz#m_xwXOrzUB8(MFZE>!)sp<mP$P;Azk2Wybg z2M^D+ygIhq;d@0#5X=%A9?#QPDOJ}Hc4Midw5uXr)H&b3tHrmg&SFxd1z7!L+1XAE z0u{bs;trkIUP|muVo)jQHLj51q*`pSne}`=Z^E!Mv_N0LOT(x>Pt<`cA%<%NgSLZ0 zftUbxg`ps84i7@0saPAPZ|P}+!U7)ZlpUyX0=9d}{)zp?pAz;x55%fsv)QD&h9#)Z z#9p^QG%GoJjuSt#0wa(>r@11T;fC+iqc-=qE`r4qkwCa~;)+vzs11XNpNpP|)m0f1 z9468b*GxJ_#%_rD+&LE8GgY)jiLI@zv60b$Z~E_=S3kfj0^~fVxiRBZyc|?iuyk~E zj;33))>9764=CpXm^DsnC98VE#LXyqPAn-VTL^UHri_}YK8-nck!xkgt7OWx)c^a0 zYG9dk*@2F@TpQI`A&C|Gjb!~)V65Pl?WFfx;rt6z)n<9}F<Tg4EKS(oV+i7ajAGax z{##X{;O#iGF+rTJHKm}3YYP~HY7vOw6`;9VP&6HZiFKwjm_DrHdEs0N#fPhFoxEB& zFaxis{nX>qakYc6nK-fG_6p{jRBcWsdlb*jfTxL<WVeD?RAL2=FttX!LRzSPa_OEj z(QG#!llCKlulSCm4Va-CZ)|b$n$7#}pA+PI*~$*%NX%`yn7L*8BQ_<2K|cMGT@GMS zda*+%zK%LGd*P|T!tf)@hU(Ty0Z*tIDl^q-e~eImIotOetg%!)BPMUevYY9-%=3h+ zPsnE{c4NMm)3bduuKH;)($?Bsn4Wl$7*LTa6ou$?{6Qswz^pHi%F_DmFpN}GRCi$M z0r3^1E~o{mUBAvR`FzgMYxJE6P3Y1MyB=n%Je1=BcqY@Qm=?-G>g$M3nwxQAaugi) zj|D%Eg|huw3g)a~qfw+(?F78(SnnyRS*_}Z$$b4qk%%C@7*`ZZj)oM#Y{<G&G%as_ zgt6cEb8O%TbWOfDv32YnTh+c0ObBdYJ8Fd`a^ai{!KYMAu`T;KMRgJM)6Qm@s#ou0 zdxv&M9G$|3ZC#*~PW^Gv*ThdP>MPej)}LIM5QK^5TMuvGNelX$;nxT$k_t^Djo0MQ zGHZd&8N4Q6a>U<<RAlo4ko)QhDP6zg`7FY)8v_#Fo(ln%8@Y3jrsIEqQB9H)^KPkx zuUETO>fF9s3w*;#ynbO9piMaGDu&=Ms8hd``$iC$88Uh%S#q1raOBv?UB>~<<ieJ2 zR^N-YBoR9!W(%B0FP+%_CjKiq;rG+I#S*dCiktNzcJ2|zsU!zlRs0%@-d~6@OrImd z(7!~G{TZcxu|tOqiG(O5inv6GFDey0JiME``!yRH`wPQ=hhimdfXrYVcsPk0FkE_i z>&KuY$(H^7$FPwbc3ZYNr<WZhAh7@P^!6Af*8{mUY(~k~TfRVuxA`;Q<aXbm&O~C_ z_UOOueI22!NIkUFDAh7zxCKB68DnIVRQS$El~FwFP=?f^6{4lE>TpF{v#b&qSon_- zTZk-(IEYw37bLa-lX#oehXXI9D(T;#7uJySnm4{Rp8G7>vkkR%<~$xbjEN>fo#9g& z5^+5MyW>euhT2VA0_*q4zRH5(qa{>+3a%#^#!=~hYhlkKOnx@5uVbB5JKOn)A%J*w zGW^Ojh(WC<DAf3gUHnWANUt5ld`su^Jk=7eo}ED!Mvt@RFVr0+h?hSCm|nU+Lk-g@ z{`!&D<Ui+fl&{)50&9*UTb=p6z~nnSQU*SS;U-EeO&uMti&R$zyNUW2%JF~bR`+kB zd^m3kero^nJ2dVE^{U@E>$v;kPJ1o*w()~r<FhLmqQ;YlRbvcO=ZG+&=+&2mT@DZ+ z)^FaR;tic=NXTGG?HWOxA{&~U_YFWuFj$=R5|Ce_1{ahp@7my?xWQzA$k7>goCTGo zaj7C(Th<Z9Mj4^?a&C{0Lyb2jbjx0&PD&ig#%9Ge-^@nO0Y56`d{?~hH_A99e77&` zQMHm~`rIl9pJkh7B(fF1xSZW2+vX%vE#UlPy$$`#S*3NKf;G8jlyO|J2_^N@>dCnh zKBAR)uEZqa$0$chPi?3WT*EZO7eIzGBQnz8TyNpQ16bewFXqTwF(y(jm!YmCO;c^c zzqTAFf8kNqoz4*xh5K_Wb~tI=;p6(S71^SypOS3y#)UAmPb@T#e!AaDdot>;qm9Yb zHQKDUaq@Sd7s^)!4&@vH7PZ3>|H}WEGrVq(#`E}<pW(e*QAY^3cs|$hV-Wnvw{^J$ zzMa_cQJW=x;U=E1w2SX5EvqmR%kGwqu#a`asBw(WW^R-<?1U44=BQDA#<mfDM~$Qm zOi1K-$6P`Bz+aJn=dDYBH`J^D$6d$EG|-9Ueo*I+%^fbt<}bA~rT1l!4wtj3tqo{w zOs<qC#NRDH08TAPNI<L6X^p{~EI8?GpXT{5iy#V9Bx|-*RwC~1?pj${?cll{=g?Vl zF$%+PEkmu6#Yz_{;vw1lCL3=CyLqEi#4}<#XvIK}W2SWzOC;=*wPmN7hH`}lTR{~m zA}2G{OO*QZ>i9)Aq-w`{irV}UNU~H7Cyv0^s&2w;tIryIl*V18`)(}pctIw>iSe+z zB=mECq0}LM@{k5a>K;E$K6$jpsuD_<WgLg_*K7nWiE-$y5-y_}3pcPfg&sn5d;Oz$ zOxc2~wVc?u0mn<01!OhpM88>xfYbO3B`gwYwow6$6(fTeB6>pp5$I!KVI*k|TOGj= z2|Ck;ZanKg@d_T7F*fu@^YQvc<ZiBWP}48iM#mGQQ(FNP<wL-t$FTF8V+alLY}&S; z);OwUHX0Ft)Qwx8BcM;&f!o`pHFv(=W~yKj303|`c9~7Gp<dLZwT+USf#BlV=SuHc z@DA>sk%i_RMbq^iYUMWT@FQ())~R<V->DWHxUcp5BlbLR*3{CSW+son^dBzbG&x9Z z;^2VF&dzRUZ*OQLZ!;xmeq0;p+5O>6J1i{B2x4&ScO}Vb{$C_PLdi|I?2mGc()G>F zP@b@_SJ#Li_TfYZPTIhB0QIsgPr<Jy5>$`P3ASp9Ds#>hri~=CrC_~C|B0s4@h|%- zHNRt|vzU;-dz@F?6O1eM7gyW@#G_5oZ)$Ur%r%1fLiK7^qJnRA8IAw_V6VvsNt{UN zlX&>nKiRh*yQ@sFg#S$`=gJ9ki-z)9wD-Cu57C^&B{m6tdNwbk(<({)T^a#aZXKfY zordtqk-7{?T$5rQ`jM#5sKm*p=LuZc@KSQWFp4ZA!rTi1C!_<B5J+oKTox=ID^#-B z?rW)z+Jq>3Ps~0Tls4q_moGn~X<us!>JUX9dpo1Zq!S-*ZJYK2!$;|O%<~W=ijaf) zz5K^nD3)fN6+dJcmn408YeG&@($EvV4L)*EP8!>wUNjb|+|Za~FK{<>F`t9%4YBU% z9j1=G_&DF}9cMV_9bepnwMd8+e>{;?_0=crC<&}px^)9_XZ{o11{rQzQ-`3on)8YB z<VKyHojq0{AaE=<B_tejFkvLYbdjg4jiw05$ooK&p(ih48ZyTU`vJH_cZ6mAP^4xG zx3%`dKTSwK$D~(nl%wov)T7$1#N)J?mrtpTQ@MyomAdD5gu{XL!xc_MzzAz$ll$M8 zjM0)sN)&sg_z0;~gZ9p!;__0-cFfJ)s&w}ch~btVO%^pmdVZ)GS!G6`+N6D*udtiB zm7|wR7LtSj8*CNEiC?IZ)mXWKsVV%9{_K-*bNg6$TUAAoy=`KnusKV2m?frRZ1$#h z>YM{S_Nr1Fz6DH_gyRV$hx{Y|G3`C&*Vp?MQ$J<uEXfE;(D<ls|2^}SxrP-%6R0yO zH;jC0mA821E3fKonLhvA<-q!U-(csV0)bqLFuU&cl}RyTwk{m*+}$ZHrs2F8tcM>D zc^edOz18?W2wr$zMfvdG9Qm5x0(b4sMEALTg#g#ci#~6(J&T4iW>*9#=frBSHep9u z7B7w~OH0=;hs=<qq9^^Cqrxh=vy(q0G}Idzs_G9N)t%)1q2>TLk4K-+8;@zAVL#%Z ze=Zk<=d|aS>9*Tk&-vg$Y%HHF3P7pf&zc%099-P#DL)XFdhGe%Gx2|u@vBV8618$P zaAyRN&NujLC|3;#J+j`1iU(|N6zYI9<c6J0SuZ{wL(k=_R50X$?s7ksTuK~kCx_!= zPEW_fELqt@{9U6K!F(zg4?zB)Xit>U<!?&%CK*?d26xzt!cd)=Q=qCiBb+Dnn2NfW z2=3Ys%AoY9UMqkgiNZE*fMurv0L(lnn)8u^yGolvbrd1G3P?A{CKF$VOVEN}a<WD? z>WjO6Lg2J}VUHRXcA;yi*BLULnp!b6BU4r_nvI{7F!hHoDkU?m2)=?&Grx>^ef3(t zRvpZ*9!KK{M*lw7NlJN1*D74}-XmScLWhz#v^s$hIbV`AjgXv0@#?N_r{6*($Ymf? zHXnONaK**&BOsDtWxL?_Q#)P82YzE0>yY<PlQ1|~quFgCdkI%9E;$8Et`YWj2pr~& zJ55JLOFR7Q7Ya8wcTaLnMINJ@XxH7fCYaw1rgeCwzpZSZ5S-r^eBI2;?(X2Q(TB}D z|LdXM=+WKmUhidneavyE_Zw_HlCT~qizWP!M92#R;uU$z<C$;Khkhm+bWYe#K4gV^ z61V3(&_)z`qq0a)PNBvBNyCwU*jpO|dXblWo{zXTFBkl<7D+Irp(c;4%o+{MVL~E4 zdIC4tN|#Tu2?!?^Y?%|uMLcQsPoPOFppt6w?b_YzAr0HBHy$MMcfT?bR^+F68)u`g zqn~t`$nQ-;7Tz6g$R)|U5A;jr`y{bsNO=6DPG+Dy#WtY4;T|S7v8Kh0<#E2?_*CrS zdaSitT540@_%9c1jo`ki;F0)<=>CW;NOQ@r%;r)onY)%xWqpK5)|17}=n7a?T&*6p zb2sXkghk<2E9a$$Hud;Xg@sf8{*5R%WmciaAwK9LS;D+Dm{3}^TGs>EP~QgDoKT!8 zs|+25GdO81OX=7Vl>y?Ag)<;MA>YZ~XlzAIO;~d?C;CF^N#|{)d(hl4CX~L{rh12& z{?qy^QlFUKsd^f!fy{khr#=4WN<+~VGFUngE&Xb?qtfvSg^;0c5B!-OM0w{#EoKV5 z6Hbcc<K;%)z4??j!DFJA7t8KV0>{LW#E2MtN~pnRXcWy|RSmL84b|VOwQ8x8Y4xD- zq1s@x`1REwh`ik4%y})=AOv*$R*~FJsZR?RJ#{u$W^rFDe(?tIjs&*kdR0f~qR8T| zsQ#lsE$Y7@uj+W`$$85$Y6Wd0YKxO+5r~i_U=~KQ<VikXbc5}!rALqG)cES9Vv~_C zH~5)f2;E#}MSYdX<j3h1yXZ)6(TBmV-x_BjYm?(;pBb(KoH3V~<|X3tKAS+GxzQ!U zT}DmHp$*dOGPH`7v|`0!0^*b9)t@s@0=TYf^f^Ulo$O8cC1VwY<cq`bZ^FGjk5zN~ z<P1l^`P|{*VYO4>0C=A!`>CqFe!R>5v;pFXO(fuEWpG4|DC{L1%~sWW?rK-n8Dom~ zISxr!2qZ$LJ|i6P$~XJ{$7N9ngMB>n>RvBeW!F2PS#~H)odC$JI&X~=epNct;4d8! zU)5*jHX)Jdipt>5(V)sn0B^R-XZ1bktxX-=ZVRkeq!g&JnKRRgGm+zJ%J40ckV|;O z8h1o>yBtU07<d1X+%G#8i$5L460mSvg;3DWdMvGdC+z%nn|R^bfzM$lxcJOFwUF|6 z=<>eU$3V}02C}fueAX3=2U-FeiixEX6vMKEL~AW?7pguxmHS}YVm#hX(beYKPux+{ zXJh<aMjU}2%?+sU5a6q0@zZ_Pm$N^a#j;B{$=PF3P7e}#TChz+kck76L}+){v)@gJ zTZ`fzTb{CKPgLI;I~h6rG>Qv*u#RW*_Cfd(iIH^i)a2yBc!u`2wvaBaBYLQxPG)gw z2^S*Ba?8Ui2Od0?3j_J|wZ1bb>HW88a3$3%U75{as-*n>?uWO;3LwNEO;FsxV7?zP z*k{gS+YWY)<wMDmBG;cL6saeq__~JM+IDS~iC3UPP2j(7oW&J4P@;|mR~yqXB?K=_ zV-4o`D%MPeGCHLP>d5j5oKfplCjxKP6I+CCW}DN{X`4)JjAA4)IJna8k0z{s3e3-A zenw@Qb2LjZ#%7GmS}j=rh2{KiU{zZ`R;qv(?x_Y&h9o_q@bNM<lZH}MMMP`5NBvkN z1y3%{B*LW5y`(<cgh>mbVzpnfJ!NT<{_S#^@Wyvrcl>Fr@>}yTSe9E0nL`l@nfR32 z(}3#>M&<0}_={zpk_bCsTpo4RA~PULpg?CDx>5zM&Y&8EhnrY8Nnp?uO#?gHdH*kJ zm70bo;*kgy2?_EPbkMW1CO%@vM@0=lMBl<R)xg1jA)hmJbry5e4VQKs_rDLy#lbPe z2>#s)DPy#q?|ZOI<9l4+g<_Cn8b&9v@7EdGY={yNo9zKciQ(COTDKX9(0JNp<5U3D zGPw$Ek_h8m)GUsz7|dI#JHm0(0W#<pGjf!hkh$WEgeqx^3LU$U093)PYzgNA-+th^ z-b2Mkp-&|ui~%Fjyz57|cV$c75UFo#jdE3SG(mT%)xBN>%Qsv%Uz_5s?N8c+33QXI z3G>Ou$tsk06H?^eYOCNk&+XCVYs9-Ho+5sFnDkSg;)y}Lr!JSY>MY~(Drz_YK<T8* zC6WFfT3E)&N7w54ZS_2FMH?W@;C_r7K;NHzQ}<}KPHtU@meTqwsZjusJI&nVZmeK0 zJY<*KD6LM~ISHIlV+VHp!C%7X&jPGRdd}qTW5O3%Ozu-NxM^x>O>7kK^YI}-3J@vI zX*Iy`<|eUjo82IU1ERxay%XbJF=}!6aBqpzKbB61D|0N$hb&*>=g>Embo5d{9P=vV zUXP<Ty>ZL8uz48@Bg^~xL~8UpQzD7ZH@h_<CxElr43v>cZZO7ftJW&`_9X~$9|Hwk zFB6<i2)&&&bY#w5quMCm_5ioYY#0gJY()XK8}(oFpiJ@XkU{F}@1$k-LXC#Qn6*<( zb3MO^V6q>2zEw~Fw(94}{8aX0$_|hI2(A~ijCUj;KN8S8C;<6Av3*#;%RDm4ZAdp~ z22$TfTmj#F5L4-q*^;|p_djcZe<4eW59ze<YTk%vL9jYu`T1)gc@3#SUz}^XlCj^V zh?~Yz5rfEQTtZ4vCj+c{o_K|`PPBlo;i9&0;mq$_LXB#pm{VLu=NunKRxTai;$ZFC zblS;Zn@N^?VOXx3YC1k-D}TcAm>G}~#5C23m52n#$Zya7E>y76OCOchM!DZeG>ibQ zjd%G5#L~tFpe0DZfhKo%cE}(iIv_ngCP?Lhk(-o8*kwEn_wV_+sJE9GG6Pz<X8~cS zN;_eTVT*`Uc^f_s6n_MKDEM8iX?lGu)H_88lM4H%))(8`IQYg>#K{+lM4JD`9r<r< zWg<H(OU%N8hJk^>>F5{!=+dpq0Tk7hO-<b(&=o$-M$kxh79HL!xI)<#IA@Cm5V`#l zXP`GZu!uDp>(}@VD@tydEe=174ddRjfp_0qNbwDUngwjSvtDNxPQuOqxJNg%`$ll+ zuTuKQK%Fp+m0beAna14pY`NjkAOBH~h`r`C4n{>2Zpr~X-zc3Pfp+7*B@sQH-)Onu z=v;Zl`mg3l+i(5~tFDb{WL%gf%PL+<UFTEwWU99U3{<g%Xau-}(u<O<_vY#N-$8Mr z)AboY5fQPZp$shnW0i77YUEH0$^msb1uy*}VTNSlPkO)2^9<ie`+jTA4P2^n=c)et zY`X^o{xfz_YE<d(YZdiXgE$*_lb*Tp&&8(`<!1se&j<7^yscy&wp0Qef?nj`3dc3? zx_>*$K29ZyzH32JBw3fYy1Kf!j12tR+S=<l_HL&Hy3Qj~G57GwJRiq?>ucdM7srAC z5L3d^{`em%N`6OlGpS$nY+XP&;o|8X%{}T{OuIF{vDmu6`}AW6F;JaSB-HNZIH_E3 z_l4C+4Qw29V~9p7@{1O=aynz~*FYv55kwYRYIzwzMZjP19cnXd5wDxtKvvfTsU9!g z=xXt_!A}Q&9=T?|?kFC#=JWM${b);^m>W$^Vo;h+C5km)fx>8ZyP@~o=eP;5tCFlR zd(mMS54D0CGrNVd%|s!WW24CLbQF-?PLNF}@8gM`I!LBv44JO!-SMveje3UqLr{eE z>BFh@1|l$V3$Irlh?d_%xML*w$tG2QUJGkV5<Q$(1~Rh2l=l0{`)!@u?|sekUzGmv zR^X$sUY8RT-fVv;^6_lo6y06kSLdhPGlRD7w>OqPSEx%)gi`-b=eIFl8ape|%4ded zft2#Qo5QIhz^W<Bv;SY-lI`^CxS;t?mJlT*C}hYEsdVha<@W66hKYqbSp3w$I87+U zH{lPYy%@UY6#WHP?s!8|7K0&Ri<2Pef34tm^SRr4%cEk!I51xPZ?tE>)a)SsC3w?l z(Q5p-;ah)*7vx@{YjbH8opNwre+eydjPa>YS)ulKXS=yOZ<N_Uj5HSkFSq~)RaEW# z6k+OSSw<(1xdB%{*L(trmJP#GpSPJxI8KR)mD?rDt8gD|MZEyIl`*IB#a*y{4`Ags zsbLOIH4ZvOKZTI#?6C=C6lqgGhPD@2vG?UEgw$7pX_1cz)s4V}S(pDIU7uHCK2TTW z1EOsYKcc@e7!H{A!Bv^Y&+C1erOkv?UjGpxY%>rQ64<yIcVc0mv`o>5hX<g(K5>5| zGNLfbCSx2AQAfhODzEe3;`*!kW6uW0z-<H?`b7=Mjf}sM-0pm(uG=L46cltfBx7hY zJD*e04-+SEDsuc(bcC2c%zul7wGIpUqc^RhXw{oGw<T_}sX*pQH+#0a)0lk}S=xJ{ zxpM2oBJ;5*AALf0M{uE6CwBzZ1)pU2HwSp>hevw2l1R5dY?DD5G05~o2caC3#KJ|V z5tz&}chze}3Tp(NQz)3G=ONEOpxwAy$?ViI?hO7R^PIlJ^;QQ4KdJ_V*m{2ioqCH7 z-oNH^yx`a2y%Ld{e?Sj6_RBbT@MKqWbR6q_rGb!i)1IIXK53l8!$U+sZgo)(?%~GS zYGY`qm7d;@sAN2!iY`q+ttFe!&Vi!dzbi(+3WTC|{Mzmu;{H48^?qa^zExg79zgiY z9+1usSlW8PuSpd>{s@KeWs%G$)pF!rcXW65n?24jmp_DK6RHTzd_NZ{nZ$;wcy0-V z-*8oOu(W#9>Dh#}NOPpI@{Tr)L_boN%&l#RCl{m{6nDIe6MD<in>{8aFrq;q8|`iU zbaLIs@>0}$?p7`HOX!92yivH)a?tJ_<5%fdWRb5Q^nDXgh;=E4?=XSyN<*^^uIGUi zGvA1**XVmlYlA@K<b_Wx3}Q>Gs|cv5s8GeO>NqE@ma`<Q8yl+VBnIv=Ljw??CgTDZ zNYCZ*m`n)ZcTbHxNdbWh*N&S512dD4Z@Y)wy3w}==8-+o7J361zh<^|s%F)_Kg{wc zt(VQbqC+;ONy-qIDncAgOo}Q>UWkjnV3#N6i@A!k-~4!+bJz)S{w}BlB1DuVHE7Jx z5pJo{DKz^|ev&>7ND{bNMlW9z7=6Hqx%yXNPh++c`DpZ~{y2oE{L|e*x{;d^t4(zk zCY5K{EoZ&yIiVv3@bq&U|0`PP={qZ$^s8V3@Ev+#;~iI~?H$^8`IWR}?G^v@7dHlJ zU%2Kd-nl*mhC1p-0G^(n7M7IU?GMtSGW@hE;J0m_I(jJ~A5}K&a*KY%x_^9>@%HBb z&^`E$Ap8{)5Gqi`W%J)DBt;hyr24CaK%Bb$m;V%hbid-V>Tp3cM`BM04rluy%tUz7 zh1$Rb+k-1$eEJ|336e2aDFB<DuC&n5(Ep0Bd^WY+K8p6JWN2tW`6=NXE1z1vcbKYe zzuBVGdCQ~!?KD=NA|bX}2CKA>iP*yvSxnKnX<!0K%jVxPqR>wH3YAPk{X8FDzGRv@ zZaZ)_LM>XLi^BHwpg23)QaDs1?JeO^NPgfRXbZ(t<cJooUOdvJePCKg%$lZ-Nx*_n zDMct$1}c)0k}$BBw5)7wiuP(m(c-YYQanQ?uIltm1>JD}{SY=VhnYULTbuS4aVSx) zhd|(~>+3fsowQ0nb{dx;pqDBnStSl3aL~s+ehO8B{;5DZyZRlyZ-i=NcBazNNBXCQ z@nTePh4<dE(<IJ+wY2n=Tyr-I3k#(E##uSJ#PHYT`c$Za6>rTCX#ygBur*Qs^6$gK zh(e3q$fjgMXm!R1g&%A~Mouxp<@YfI|JdRmf(!UOE1{NIe@i7b;_mU<Nd>4N$=}{! zc3Qw_u%?p*c4AhgD}sv5!dlF+Mwm5$H8(0FRZK%@4;}uM3_L8EM!Dg{Hwzq27k~KK z8=0u6xay@Jp>05N^HRF)4*qUz@<HB*&zeGK2||RI0UVqie#gVC&hHueB~Y}wN^h{0 z6F)6i`b_4Yr`dXG^%v=Ss;iMNE+A9S9?WCo<D)x3aIzK4ARYeT(aiNJSlA5)W`TEZ zw&w8)wIv~tQ0gZFQ1gpFRCCz(43_5qe;&FD$HvBrN=w6!pOO9B>4Z-IB~yYc>tqhg z=+xGqid2tAe7#80yKb+vpd~_7Z?Z{D$>g0X5aQ4m66A#;6islgV}<Eq$l+>9NhGmP zMifW0OjtK74}7g;NxA6nJ9|*Ab+`wrt((one3e5ujk>~ORaWuWpRm@G5mD@o((7Xg zcb6yoAKh1Z|Gk5j^V~`6T}ij+;Q3;7JRl;4r7ujt#HYL5#J-WrN5{R@(wwZk)>tGD z+oxzx$Gx+|9F(v9BjuX?L#>3jF!G2Z5Zzd^bou$^<uOaNk$43#$f|A-Tm#a&Ixp)8 zSVjQCDD>@w$DaL$BnDvtP`Da{5tpzqv(X)ibZX6|z7JmUOT>KA_t-t$Q)TK$iJU@Y zxceXMm8TEx|1OFrFx6YEzr*`*+bQ?`0i>fa2reC3ftsZ*IAChDOQNSm-Ye)?B%dP` zWn0{U(Ph+dE$8fGK|YBiQ2HvgT-jZP1KyuwC`}u?dsE9d{>R}=NBxUf8x94<NIdO5 ze?9zK<tG<9xPLhlbTl6eNp@cmb-wk;C_htgnJPc?hpyA|hdmZ2ibw<mh(N=@U=kC@ zL!|v;|6mo$lyxE%Kia?+i<=udVdPHZ{ntCvxm9Zt@H$WT@y?Tl1>Ko?Yzqc}**ti; ziuu`z(}M{?4*T&IaHTNM4uIq?W0l!>JPryzVf!6_WMy2P{y(EQ<hLF#b@p_D_XT;3 z=-7F4&Vx|V*y~F*%40ij`xMM}=vtxvzK=!Kg@~QTy2*BSkqqjbTopP_j#d$+$W=VR zqT&V=%D2!Cr;8r2_<t09*mnZx#j2F9upgp<?-0B7r)xUAv`;r@@P;>&nK2Dd{ty^e zs0z0AfSQ`j|F>WG-SFMRgUQ?TJ>+BU8XFe*)Xfzy<{G8Bu<hf=Wr(7&t-GS0GWPAf z2<NaeKq^j9RNXeL29Af65rf+$1!SYC|KE%tp8!vAFY~$8)P&?x1Okr0GsX5CiTV1n z)l%E2fv}}P@MBakZ33asJ}iE#(No4#gFB~(%QD?TW^GREM*Hz;yr0u#lsH|p>e&+r zhBb~IL!{t11o7d!FMgUvRW_yOC@dj2zMVttprBoTB55vA!JFE{t5&L}i(Mrh1*42c zQb(B$Ze76X=pUk=+mj?(5yhmORb6y~T((&8HBD~hBSqKNeA`|2W$K5JAQbnDwwuzN zW$dlVd-O#*WR8K3nc1JuA4s3}z|f7S`hNWL(L(4)jMJ0k`9JEaI)0cE-aIn0(m~{K zZwUKwomZp=yFZLBs`dxMP8Xi+Jh5V~dEw@HYxcFNQyY?X?h{1zMkIII^LN*g*BmkU zA@v2xTx`F?uYW9DZoi<a%7R~F>Q*4TGd<ZX;ipLcf%63!aCEQ^$4Yras{;mZnoEWJ zlz8qNd&<2YoD97y1w+)I4j=mHQhf^-*RjntNdCz8^UUHGySL{o)lAz3zSx4j`8Uc! zdMyu`8I7l0nDvv*cSD2}c+DD*5tr*GuF#~!4?IvVPWD6p&WgO@&?XQ`%h(HZEJqp_ zN24z52o`Ygl4i0P96WlmuYSp{E;7xEF<V|<zM1IpjXv{w-1G$L>rboxyx_k3F#{Fz zPDny>RL{)90{r$3J%LzoY+*hOvdzjU|5+#6cW7vcgeeXy%Agai=kaIepPWjBRP@Oi z!@R$uUbl#H6x0~Cm6gH4!BBg9drmh8a_rqKcN6Zr!}VSp>3i>|hPsDysCPu!f9VCB zJsfmD=5wrwItL|V96v~X&^AZl^M&zOkP161Aqr_%gG8;Kl@%4rddzpi(lDA`@CI*t z$bQd1X*;~$iN7W!6t?$bGq<`|urSML2awCV-1Zrd8j8+-tjjz4)^)s{YfRR}*{kO9 znl{(xc+dL1d%LHruP$^W(<bKsTaV@Hu7@pE-QTBUWdNF6<I_l8ix-ZIscmK#%>Dd{ zxW2yr==Mv+mtsAS;;F2x3`cVt#APg_K3>UI9vTYM)6>&nF-;%^1e!cvY?}NMoR#`B zOoo$?kWf)o<>SOQDd~zRCHe*t(qVu;2<3v-M_PM3JJ~l}jADU(c{txE0SVAQLK+vw zlbG*ng)+=BJO!>N`;38{*>{D|W}5Y!-R$l052+9N4^IytyK=MKc+c0;E6xPUAVY9V z3l~WQP2KKexo$Xf$9P)k-=*q~p^JD}f7ptAzgKpVcktjEZS%u5v#Q?s|1tH|VNLem z|Ac^&Cg}i01ZkzDQ&LiDw34HR4FseG9t8;{q`P7C$iWcl9xz~}(hLTY(!U#@=kvXO ze{o%7yYF|LKCg35g5OC`ffEaShwyZ(b$ip_-0JjCPglmyLxY;uU-UPVaoLWBUS6V` zTU%NA`JK^QQ}$AycIoNq7a&&qsazT8x0(WKva0618+TN3PnLGOW7+E8AMb*qPus(3 zJOXPgD}Ub9H8AL85f&1v09pVyUtXMk;=kn+{Nn2!z&Dv}_7=Q%;p--#MhteCaz6>- z-)>MDIj1M&uMNvG&zqOtWTDY{{^Q4wYr49+e?DsPnsG+{JpHlo7jbu9pY13{H4Nmv zZn3ZowNK*NXEan)RIWVBNbJ;<oW6d%R_@3_3D7g$aVS1Koa}phyEl$2_<AFzXW5;3 zK>3cFi;GM1&~5^+$)D#2Pmkgj;?VRwyu3vPk230WdKZBn*xI_f-;4E2-qpj0&u)S2 zgn}vo`&rza`F`XV<zVAF$^ISF7ZT@f8gtS6Zx5o-s%a@H6hJMvv><>=IhGjYyn+zi z7;x6X-_0|rK_J|RzJ1fXd;4}WPWOWjV<H|~OCcvGH#{=(Gjkn>qu}P|rcU?s^OF>s z0JhAJ+`mn;1@rO>FDiO6pDypZ9Ve<)M5#x!j0z%tU#jk=wm8=jLAU2-Vf<JqK=DYq zQESd2^w82WPJV&UZ+C%$_FO8*;OQ#={Eve!C{bf|^_>#K^01N;?u-XFZRW4iw9Hx~ z2PuJicQfwXzU^b0dp$?5`UC4oe}7~_w9a)t?{X2Bnd?Bo*WA`?clR&tt0}H6JqobX z(^DPbSde*PVPP|i-W*-zgxB-3bXKqTGMLr7J*fNfLs934y+7{abzJk8xUVnFdzEiw zS1aA~WT~6c5q~h0E)_96tSud(s5+|$R=c{d0*m*dR*5Y?eoVD8=uhS$IO(KM$P=a0 zr#)9f_@Bb?>Xe|%%ju?V`=%1{%ga`omD8cKr+sgMkvF9bX_ZHF)05mAk%j*tE!&A6 zK4i^A>*aowGyZ2k{pR%;d2-e3PjbJ^`EK`UG^rkLONvbWbaQuc|LTkH(b)AbI{>fF zc}Qo>N;}IOog(K=1adIQ=^Az|88-Q>L4%~#ZjzpVTz!LsKFDWjS1%n~lglM3X;b8U zeXOCzkp|UGpFds*p;<XU&}vvq)+80t;9nf+3Y|#R@iXqb55r-2iJu(J`3R`(dQht4 z$YlPsrE>haHzEp!_6fh{LlV-^y5kpo$KKN0+%v28HY_0Hb#N4v(a-G%DR<B4h?aS0 zN%`z|!)YeWyv&ki0{r{=$w8iAnm^kLUb!-o^?uu)FXp$_@j4Rd%qL}#wf!_RADk3i z*zB<T5ZXInf~IkK=X3h{h_}e((itLGD#+;YrP+xDx3KkaL7DR|IgBKTa8E2K!!EAv zS%_`(KB~}C{?6l<-I?CIJsft4rf+J6=;G++fqLL$${f^w&8TKy!mHiVruF=kUFp)o zaRFn~XLLM9F~s+$45jmn{0^%#26iYs6_o3GGdm6@EHYMEO;+rp#i?_M=!AXY_FD+( zEW8HR#8ifWnrSJ8);~{m`+j2`XgSt{r>gymRLE-2DpL0}Ab^^jyp1mNt+SYG5k9ZI zJcMqqfr^eMF?>i^fRnFyneN<>8TN~``^ZYO-)?lu=r(>j!IqbXo(k5|>@@KCSN>z& zOgq1R5LKwWnOr1up!SdLy&)MHlYGSizdt|21p|pRhIsG|Gp=%6h1Rl_(c1$Lcm*6- zMm)C`8#v7HbwI`QfjK>I`a_=AiRL~KMlpBm3yUA-vRpf{*PH5Q+|@N83d$)jZ5U+y z-!uOq$Qit3hTT^hnCQO+#mT;@7V*$;_zAipAZ|C9dS5EB6&V^3S{*g0R>*DN`zvkz z``k48sL6GzDb2kP1GQ;&_TCyyw`x4}JYU#aA2X@TU%Z(4A`SfsZ0_Vs0`%wkG&=R( z+vren3vu)Gb<r6&g_Efv|Aa(-qn7vHTP|7JoMX_S5ukynZMme5(*VhV42=}wM<d|| zZFBb?cClc1L#ip1j^8|0ZqzF~lo(U)x8>+f{lt;8uZW%eMF&aA!{4;tkArXSDMm}+ z>o>pP-gR})e|bj4`gWEIgnagA0qc%*I}jPxsZG`Dvb=lh?5M#&>PT3b#B_76q82yA zbZ>?zaJPCjoz=&vel@J|Q<M_D4g2}44Cg5lvAYfl^=ro8WxM+3pDxgohc*Tq34l3e z_Oobg+?KMsp1(<=#ebc#4|+hsVPC&l)%Ps5>O;EEMiB$+i&0p=$@~dMQhC3ZySzWv zGw<!;l-=q#&e~tJl#+Qj!sNDq)&?cev)^5-l15Gn%H{PRnEuqC;+v``v^6qYquSO6 zrP?B>C*F^2l$Eq|)?NGB7Fr?Vv(zc1dy^b?4<<~%b+8n*zfcca|9L|~w^g;wd#`T$ z%%CZNH`4!4_P{!YVNGry^ISrRK&bq{ms=KlWzFvz*gG<<IPY$dKhy88zVX3-<5fm4 z(~&8mu-s$rd|?@|dh2d{dG65r6+s5pAl!iHd}@aCuhhwm*Ce^(>G68=2(1ihm|@rg zwbKa^u4LI#`lw-O*@|lN?LnE>%e6J_8>WxGRN(?cYr+D=Nps5XolMuj43nGOEuK3a zV#Z&O5w*&{*^6Fn>>!i5yP>wBCV(xeI$&sJqe{Xir>8avYM9puO6gB^4&Es(6qXWd zfw8cbg^omDzV<lT)#qLR)Iyk(wbIL3kqFmJu>^sFMsA;@`%D_0oezuV5`_^bmxyc= z%qzAi?&FD5GjE#5&ZkqCmQm}4aTa9Ymt>%K8F}Ve!?9dP8FpX2Gk7wOW<0bufFcsl zG+0{Vvf;}oi)ux!S3@A|A~<n*zNl?fyd!*@!MOj)y_n9L4?ahQzWzVIhk6Gv2k#yn zMKS=h5KHHtX0cfeawj@b{+MNo!#WIlby-4{af3B3qlZFvHaKaw$<3<^l!1)&@YOE_ zzuybh7aAKMi?^&fngl`INEG8C$0xH8zrNAwUaF?OGcDN@+0UUhj7l=5K_kE%6*0eB z6ZOVtmO@baB?kWBN9w_vb6K8<T&uYR&GbZX<Y!zOby{MxAECfIqE{`3qycpWlqB?2 z-<bQ~>P3BRrFImqzn7`UY=hA9EqxF*8mmw1>?W`*ruU?dI<D=SoQNXjI5<&;zhH0> zxD%&_Fe!e?oDG!R#XNGw``7WfLB_b$c6tTKez<I=5B=5EIXx>!Yp~9a)6mIr#K&-4 z2iPtSu?6W!)_@#-0-d5>SJ%PGbHuN`D$pFb)5_oS=hvBT#9BiZyzub*+W9?#%}YH; zgfa7mNT&N4vDH!NaMjzMw0bu?V<g1OT)MbhswEE{kcO}}<O``93mu6NDY!&7v-|BO z^RDxs6DcVSB;Gts;I<^Z8np4iWK6MnbG|hoy{>}_K4<G+d{F#Zy6<a58a07>Cos|@ z#b<vlc4Z(XQ6h~@mrxB8g`aI&wPD3pn-`xXxpvAwysWh9TG8ZKhX$+4z2s{>+l6-A zGh8Z67TycnP@7}bmPj78mBt2%Bg47YH{N?Njd`OH%}Hu8@g74`qIEW~*~a8ltKXcI zK24-K?JVS>)QOU<;Qlfb9JrUuTp1KHlE6ES20{hsew*pTpmtlAS9Go#IFUL}F?!hT z=9~R@?tyi}zuP-<j@=tKr?-i{FX&}QO<v;$voRSxL==8@-pw{+c<%V1#;M(QvmfGG zGD?Q2Gp_P9w8^j|b{38_#w<q!Kvd_EEkrrz#K7sq>jVt>3Rn@oqv&DblJyQZ3j0ar z`cA=Tyu>_S`g~n$)!TGD1$Gk1SSLoPD1Gre+U|PV^S5uP4Ss=sRCO79vsQ*xm|g%y zi~SH9ka&$*S7mC&qjVSs?2Y_x;))bDk&)y2*1X+v`fLn|Gu!S}S9BY39D?4Xr4!)K z{3LvX_tpq=#70~1@h$5ZT=wDn>FK$IX`R6MJ6e)Yhf?kY?T4_OS@Oxyb^6fewe!LR z1PFn|(ftip6+op6%ct)(EAWym`5CR6O+KZct%8{t>jsS!QHbWPtJ6fopq*8i{iR5K z$ao`fe7MVm&lCE1A#%&$`dcf%Tv^CrFKX$ONNzI9w4&cmS4cQdR!KO|`0PwKlXpvt zH#pP#y-7F-@haGYxzPEw+Tgqjy6o`%mZlUr8OR#;A-Z$5x&)|h9<?D!WLPqffeQcp z-k=2k0YEYeE{&FB6sAjRG1sWew9Jr%LLSY{R)dmz{re-){zB~*Nk<*MHw5IK3GBNN zed>MkEuAx*m2`=a2ctall!Wn#z{YGxBV;boPwq`EdE~{(M5`vtaksL2#hug2O(Lr{ z&G$51WHijFt7WPRVsQoa2L?^cCv*NQ1;6=-!lR{6?D_a*4~nFlKdYT4h`G+Rox%OF zq2rzbH%pwuC>g#eZ_n?gA1$e545sP$9Y=h6zxY{nMR=E>HtJr<Q}tpWt`e<zF}u>S z3&TWRkqkJA^P1NwL@-SlRP#%`NF$ZsdG%}FD^Q+|#k~H622;Gps<FVk*G1Hh$6^@Q zj)TbyY*bgmv+kV!(1Rzu%dbeg8fmeD_BY8x<qwvoR+4^sJFv%!hr9!beQVk)_nxCj zK1Tt+`YT$ODYCmMl|dt2or7hvmEqnK>ux@UG%V7R%J=U`^;7V!B!6hE!cycAxXm=D zO?`(*uVy)oCT_kwKeA%*d*3;_K2aUX=z(=)C?n3vXf0(242pquT!3Q_>^eP_u!;B% zPY#b>W)>S)!iZPPUK)c-n^sbeK7!~WDjUT~a1G=&=WT~brd?}3Kb+nZFAV<jkM#|4 zu-jI_Okf_PQUivl_Kk4OA!~h^SxXkJQbRS7V3*O?nA))~)IdC`58u#HW@vn<FOAAu zBvJ$)ydRqVX2THKvIyGl9yODtb!9_!6X7O_x_XBmItc{ECxj-(Yqy+w{MPs6`?2kR z)Z_ux-gcEWAdRW5JmLJ>SP`>fN~q8GLsp-OE91~Qi}s8pH(2FBU#7wyOwr1HnQ?ix zDr7&LNg349VUS|#cQ7tyhWnC;xHdT$zMZ<NE9#tA&B4EFkc&=4ZrV(J;>)gH>@|+z zSx#6R_UBpI<qV-1jKCQrBYh5f@fb{^5R&tRFH3t(d%3}c^m-r4Bpy)7%y672BKJW~ zLBS0VQEP$H(2l`5BWo6h6TRft++FmVynw@x5p|Q{hoIY_f8Koy|9d|x5b9)fvQqy{ z67M%((aSmyLD#s@T3Y$O$9{YA6V7v9gKHS_%t6x@ZmQBvwqYIK$YgGEl>s^`dMXfg zWq;-5`chq22P}X~fcyt49vY-RMR3j0q_XfHJ;?)7IZ!*+PK?Yz(}N|5cjA@0C;_d3 zp-W*Dq42msZ&hzF*Pjo<B)q=~ppL{kdhMLr#8Ra>7Uvs=9{#<Axy=+2OUu4CvC;Fn zY%;KmPSKWFxxU@!mir|9@8nKUWYRHxl-I$W-poA_c{KZ#mJT6{Q4)NYwo;04L}@R< zy^8TWDtdKwt>;@)x2f(snwiFbZW1iS*sg2@n|1{mVN{%#M#U=d4^?2G=gKSDL@KR> zo%{jEeaBzNjb!9bRapwf=dl)M<I2cD<Q`E%Z4x3rY!5dB5p|SSyOg=ylc_d&TFz5J z4R0$yU<bV#L{?Q9bWeMWiYBSo&pVw63~KCK0KOU95?D9;>o%ZD<3#G5G-zm;)0DJK zbk-$Z=E-O!JW^VqzR5zaRzCK_lR^O_pU5LIEWe%dg{pL{`LFG%MG9<4evmga+X@}A zbxMFSNaxhLM{oDaMIxN$(~J=XH3*^tl=}~!zNWG0d`IUI{;ec_Mgc}~wEhfqem`H- zMpQpRly;zsVgGJTp<*GgKrdHUDBXu7$DUs{#4MTAG0zjmHQa8vb-W?9q=-}_(<Wl< zihsg^%3}s@&~FxJp7{DEr4(QZYHa&$`&N@gEhCI){)wk)4%W1!;-p$h_ryXNI$y#F zKUwa35BZv3P!|%ZyOBWsy5zZ#5qyoDA*nE$CKxV(_dQQ<#-AS?&zixV+3A6eR?=rb zX;y2nxBJ)~9~+v(8)vd=w>{GLQYjOz+!#1Fj@J!CJ@W<p&RY$+%<I~N<F0xh?6YvA zxVAlzkl6inFJISip4cLjyc?`1$0=()Wt&8@haK|AvfMv4OF4z69*&Zt7i~*5YC>2w z;7^8&&MB^6F3Mb3`7AQcf9regtP;ZnNI6jaYSU4KUCinkvoNXBDK&FEv#+~m@}xCx zE%!JDJ^XeL9r${2^viNd(}xszm<Z9jjYlVgo%K-ywm~>c+fdIiSSRLq_ycIY5>f{F z9-Icj?3zJVs>~q2f(s@XNJ<Vc4CfurCZlxaPO27L2=+9n0?NM7o2xH@;GyMY`t_Qv zTlD9%+y04!+wA<_s$uLgMj`|xQAVWG6gk@gm#2H%3wizP)*W{&yH86{oD>WtpcrBU zQo{kUQqiL-Mx`s!QV?)H5uj1-J>ZB|w>UwkQr{$28h?9-npO$S3uZhY{M2iF{ri2I z3#i*zbUgZ*!^1=O-z4Fzl=qo9;qh|T@TyG8RKkA5k1BHV`J<L28}4d^mAAg~yad@y zbc0HDFp<@Kh#f{nwmaTH!jT4!EJd59Y%TiU<diZ{<LfeYGJCa9UB7)Yci-@|!js5( z;?n`!c-leSGUnK3YQL%REun$ZmZ<OH=6mzGvlY?RXH}*lBXq5QI*uh9{Wm8H9mVdF zgPL|zI~3~IKHSmlCQi$@&Q(GcE}JPur6e;mN{Ucr)C*{U$5I<|W;~R&Op@>{0*^n_ zO9+>lH@(h?#oG3%V-)7pM<QnW(tQVi@)>fDF~ivDYu}Qs^iF%}I#PRD1OAhP+!(1+ zfcGh8KfV=_>w318Bz_Gr2C}LiU3X+F(XW+oP}T#qjspR4WJPg26)NzyOZ8T$4uRtZ zYxmDN3i*EAHFY$XzzP55F26Va!SmPm^ve+#veK}TfYZ7F^(*JKdRbe%7yr({X!}Pt z!g8!R``2SrsvJ{~_*$M4qT%|r;_|ln5_lt*!ow0~ZEFVPO0uK-DWNpRVtdu+3twM; z4Z#}|v<MHHCpr*(Y?KsFwHAMzHP|XglxgRZ|Gv$V&|_!Slq&@>LORIdb}fgLGPeNP z)9}WOBzr?|C5=(>T?e;$Y#%|ObHLdm%bu9+F;56p2k+!Id*&@eG4D_4(w97&XzNhB zpW`i8&N*iizk|TY!nf6j(Q#wf>1Rh81qV!OBo8(uB&8&wx<2lm8~KWuGyyvWALvp) z)Nv>Xve8dcWYhPPhB}vR;l-PD3d6_zSb7~w(31@Z0iOyc`t$KL6OV;#S2HF7Mh7@_ zMvf@a&|}A_i!p11KuLB;Qhfm|GAgSmK|%{0?ZOkYE`mF!etRkN{T1KyYEWx3=c6)t zbjY<A>p*iZ?=VR>yGKoP1dhUyEkNg#A-?R=&pdvmB`ty(x+lwjd<2RmOPli>D!L8Y zIs4c}F>jLeY2`dO3z`FI{;BE?ek$NZoG;51bK~Nb7U}JJRPf8y%JeuB<xjo>++}C= z!)ht8vF4fGK|4YMAbL<EuEf*=xXAq;FTqgfUR!f+SFSzx7Zjv9gIRmJSaQ9pv*PGx zBcHsS^o?oSNOx`#OTx`GvJk?-_=_)5S&sMm-H*0_gj|?Tf(^c9<x%*R@Lz6tBOYoe zg0pwH7U`1-cCFct8Cl1Nhvl`jb9gm2@g2<s<=E!=vM|Kr*99My8Py(EG_8^AzE+%y z=uk+b&<M0&Q$ddq<k-z3g@^8)52hby18%bIjhVN7ck|-LZ><T-lPEQKxwksukh+o{ zSmK<_n|kpo&CYF{xW}B<uqASGw=-%+M}4ZCLt$2m1n^U^9(c#-@j12p7QZ}CW|a(` z9>I`qAJTvU%vADP57}(j=QB%6V8pUTmJN8ciVOGv14Slo7CNh)?h`|;>|ezNR^v!n zlUmu<#WvPJ1Ce0^SpZVGN4$&5*~=R&jS}fnVYx}HB^M6uH1dkqnkF(9DAt^biRg?1 zrL?X(IH6LSs?I=Nzp%8C;$$m+MBX7c2X&sX;dm{9kcmZFUojt+DyBpoxp)6cI>#PG zc|{!sALPGcv#ha4*EAGOu(hL|k)(NqZ21Kr_e4Zy<jLJQ_$O1DLBfKw#|y*I3F45^ zJ5~iN2bwf>{i6b2?Zorl`H9Ov5o2NUWH~v6N<$4^jwe4qo1l*l`>*U6Vf7VV_ok)W zQ{6Vr)Etpj&&i=yZsV-n!^B@vg`RWv!jZ|B*9qdobGtxyEh`&?DytrIL_A2;W9@m! zWkek#JaIc0puduW^FX#3BA1PV)R`QxAqA&6-L8!kN!_kc_?KElI*LtppW2Yw=H_tM zQD(oX4q;eL#KH@Vy1s=?49A3lCE0}$TU)AULIR>9H|qRsh^Q#N6w6a(sdp*YMHz?* z(K#o_Xy7}0i=qi78*s(td&SKpN@*T^s9kLr1B=yAn0^X@qk-+gFoq|0#T{^RfGDvZ zc(2|8Invp-R$R-3-WmCd-a?Y*oA^?zg(wYaF4~r9yrOe<4vZgrMl)UAaZm~9I?%nF z$mNy`F*kHJJ2#E*T{9XOKNG(`|6Lb7=Q0~wa+v-AxRRTAcH7r^y6L=V2mzg5oPGra zs2~eVaOA8ZA=)L{Wk~kCDSc?{=ix00G#Uob>!3cE?RN8+hn18bNRysKh9@>KZ28N< zkXNbuhH*-Oi2xVNLi?`$VD2-OI#k?(3XiuX^)+{x$oAOLxWLJGx)uSlmIXe_wR)c? z(c;0j9vX!7|J;jIHI9-2qC6h6_4W9qoSNuVnULPqR1eKCm%Zn;b={3hJ9+x1;qmdc zu?I{q9q0&Cj8uZMt$J^Z&bYQ!h!dRDA-9~TeBe}Dj&pH~=cGyzBsH{3=cJr6dr3n} z?m;htV#Z#%abt80Ok~L?<448fN!5`)5G`*<iLJcWH!>5%g`Q<kv{3dy%<bjtN86M` zv*h1IsFl3*Zl{TPz4gf@`bP@y^L%#BS7k%uy&9vijd!vxAzGpK3(-_j1UG+GQVDmU z1o!Ou@XpwIRQTw*UwGb)SGKG6EuR^Kr0=dnufJa)#J_g6)>#abS56jjno65Eyf5Lp zI#;GxWmaQrXk3MGboDUA_hM#$?`1q`)i^dmfLiPn*VYofV!Y(o9@JbH3MLI)^#~B? z{nY`w`~K5)oZq!fg8LDcA0d`zYVax&rL8VAjHIGAk&qnk8j+k*Wl(-$i)@Ffa-cqM z8U^O(m5;p2e|<Pz<s<k4p=n$BkeujRbDVlof8Mnbo`7AF52bM{)>Zw%R<Yh~U;XOO zu@*kEv3arT0^FJF&)7FA^7K(f=SJ#Ov=d)Fy0PT_<7p&c)l1LD$nFRC5jE_3wA7}d zbY;QQ-C7OH5I#<O8(#f@1e3sSaUQeNo_2rKSc8G#7Pkh0#geE?r{!4f0=-+Y<VF!v zwvJU9%e-?M#Z+C0#4^`ZZ#Y<NWql5;cqZ;8Ld=3PXzi5sy^1;*r`OGKr*|!Oes;Vc ziS8WWg`8}X6nVA&TtAs+RuP?-U_IE7c}#U*SnzXFOEG$67D400Y|kWP=g0WW)7-~$ zxSq0pV)jI~vHA$Lzx0DaEEqqi`R=&R(EQ@@FGlBs@1mD%%yWP;g*tr<rF05R!g_~+ z;-Msjw?EQyB9>vVvU7qZ62#!8JgKFYrMaeW2suUHBuMWG#_RhW68jO*LW{|`xn$8` z0@NXia`0&&ok0!u40%aVeg<YJSsf5hA2;%rt$h}lu?srVzG)BAH_%Uk0%PaXd8<qP zSCp$r-Rq>z_eTZ)_-q#&!nE0J>d$uj_8N<3v<Vv<D^d~T{(1?_mS%C$`$1WjFL%GT zdJ^Jn2#6wy6xu!-DU1gt-gVPc1|kp1JRMPF2YK@ZtPO+7mt2R}+(@FZ_1KfrMbf!= zb>`rQm4Uqqvo%bPB_8WDe>A0~{R+|=P7<1(Nk@W8B*&+{?HE?8C=gcQM!>LO*nPS0 zwA!HS%_2mrq^dW~;6a;b9!;qsI(Mv1qQ>%43Xf0Bio@amQH|iXY<#Bit?9wgWomQ9 zGB{-$ET^mqP6*R5`o`?zq}!Ocnh7;y2SH{JP?<uW=>8gqfs=Of>?IpK`<o9K_1^ZT z5m2mlVTqdZOR_%C=g2$qS?#<yVInTfcO~9+cb=X-;`-pJVsAy*vNtidIiMCM<p?G8 z*wYRA2BkTi<)KP2$RzDvHlzT!0nF)a>bYza)5s(9s^pnP?|nT2hs*QvI^)C9kFRKl z&C-#1E$D-~?Xfze7Imh0@g5T-57Vpib3oG<3NBAs7l*Y~4*4Hee-787Y1@)89$jMC zL$ie_<xl$@mdjdaK-;(c(DU5dy^T{Pmus8fCG7h*g_e56hOj46(!vMK&qV-fu(-Z* z%B(TFO;8i%_xzxg0RN|_8h!FPh2~UJhX$Gxr({M#A@F9CWNI!^jT8EaYkod{@bh3y z)Lm|J0GxVO^}KN&R$(Lv^>mPM`~0OjcL=ME;h+qq4maWfj~MNm9%Y5ladNT3SmT2# z&V%Bhz0J9Y`6eY_v?^IA)RI@M-33Lq(eoYTjjdJ38p(N1WF@+AhkKl57p+d?T;42& zkyn-@_}SW;H^dkvG50&Fv)z8A6Ugap9;>6R*f?^VnCZXyY*u~=Ml)^uJZjpR!#Zk5 zwU{6Yhnc;O25tglr7!Vb`>J(Ej4>zF4DpXO)4pO#x0?r@ec1!}RI>MOXrg%4?4C1k zX62YMvnq`Zl>Ab&)$zppmC6%A2^3PiwGwVDgpG7}XO9!LX1g+)32JRv@=#t`TOiiw zG!wl>DZeLUR|j%czgC&RHCh+kj<l|BsH89OTM=76=>2{U%a$a49`7`wAdef9tNO~W zz+XYBz{K~k9JLm*o}98Rv*3ANMw6Sey`I9ncJz)>@U4$=#XE0oMNytQ0n1K}$hTN+ zIeMdivhUm%r~M%u$(`)2vU0ETTGBLoW_W`MYh2$$S!nb>zl=<~7<q^Lc+uj)yL$n4 zOLxdYDl3HK6;{%Zfl|*(wjt0hALomx7Fj@4ZCRv$m3-MDQn=3{L8qh|HjUU;iAXF{ zyTh%0^5<ItIxtHspX@{?-ADS&sPQyNPK-5nYc4!2*W53BncXBTD)n2ZMsm^$k5RHn zt<$<L&AXfhqI>_mET}eXnlt8v3iYNwO3(zk1U-YIzzPk8r%Z8GVT~M$HKk!XwEC>v zf!A?C8pgp6^#Pk!+R6)5AyUh79aKhlRSmR`r!5$XB2}h$uMxRXGE3I~RCo9wzjcoS zc^Vwg18>s?79WZ&>-^4XKBGV7iTClDuJjnfB%==vJXF03G)z3N;)Z!idx-@-f^>Or zU#;Xl{c5}FjNT`KY;3YaMs{l<v-!>t+@aZVJ;)o#qGF^VeO;KB>(<x9Sf8=bFQ?PD zamjamx<*{tAJQ@XV)fd`73F&qW(I0)lKpw`FrB085kd2TiyES{HD0&g-2?|m(l5P! zOOk6k+plm>?rX=rz+ioy7h7!3e6iBbfIZS)^vt51n<^PFk^owcueGE0?Xn#6rK3mS zucbV3o@$YBo--gEENdDK>qQ`Jyo{?Gs+@gnO==;A#+5C?yx?kHUh`UEUegA4UZbfm zw59f{5nPx2=15^)lRCDtnR>x8_@;?pX${78Dow-yJ*U@GkoRR?B(rgkL3XPY9!PqJ zCsZQ)MNT>L5bIR?@%TZO)e91Ox!i+Y)a#Ec-Hq!P6;nfuFcPJ<$0(t9#7ZDX;u`d? zd_pI%7)a0=TayaEo)AR{=g}<9N_e`!=oIC!J%`-8SP%g+N%i9fsBeGoM1Cm2KRqyj zO*u+Zi2k6M#Ge2<iipGLXL@vmnL*YY%^<(A1$8gto!gSlAQu~*&J<kJ--WdboK19l zT&H|cjoic=7?75UrbG_ECG$-~9^A(?k3q31c|BwpKv6a7W2UrPw5l1*4@oYdD0QgH z4&*{FXvBMOW!XC)H7f3fy5mNRE1E|Eo1hu@GqAk-{v>K#fB4a!0*OWy4Mk~QJPLxc zUvv8u9_aPKOP*5u-wQ$X>hUQ3b4T;}&xO}_SA(Q!d*hk77aH&ol~m*#yX4yUcF9vq z8tHomTm5dyZBUQA;At;AbqHIM-WbP-2M|}_UkPnvGg4bKx}(myMw}CPlV{`Iw2`7* z{=$pWOrAm#rE?3^=$S4J>rr&Y9QyYA?)lP8)YrVbV+QSo`+=q2-m->2bZIAph{-xr zM}}V~I?kZ}5gW@^zH;d}?-uj>35QXPKrIfd5^GtMz)W@9);DI;5?m+aSIV*Rwq=ua z)HFvTcFTiKNj7D(Rf`bLtwN|fOsM6c%+)3l^(&p89Hf3*QN=FBmv;h8tAWwQZ<NN7 zLS%pb`MoIkoR(fV#tP1{v)^<s=SXN^PMW=w%(8LuhA<x9+)qc6LlB4K?<+jNPh;G0 zn$S>iMbGSn&Hg8&87q+c@b-VN{y;y$^Xc+$<hGydsU>vPWl!!ELY&XET5tePU}1P| zr0NNi;}iRpmZlutAiyMLTwalW2PClzMl~gPqXQj(Xs;Q4#ia}Db}=bdsWgg~if*_D zh03m{ig|1dGT1EFWc<^FCa5Kd?<*r@zDdp_$J@PsL(v&?YZ>T%XE(H&z)f}fM`9&1 zjJggON0o4uO@S`Q8`{zY{kOc1%&s-m!cN-u`&E>HgNwa7iK3y)3~~ZgM|;uutJBZR zg(P`^+ibq-G-mb7hS5Lir@&VDy>fx+Qfo%{SOU~52>NaZc8B~RRRoK-6?ZqzxU@Zr z;Gi?l>?Z0Rz^sU7M1dQjkd|`Aei8(lyR~W6NMX4ryEkv*(w{8<u$G0JcVaehWcX){ z+&6W8q)^lSO6-5vTfVV;suW3I89?9RGHv{dT2m<6#09j5+hD3Q+f!|zWBJs_ha2qS z$N<XKPUlw#-2rGplug4$AMrK&Wa8eFbyxMu>kAKKC|F5iSmR#Q@?3te#XJ=mc?(tq zS+U6WIx83hRG2S<i-nstiG&DD!tNVdI>wWKSn~4Td-IP<epdT~2^K@B_J%P469Fi$ zCII9HNt9HsiGKk#L<nHy=VB4OMFd4Qc36sBafJAH86ed+`hw0ZF2`GGe@1(a;Unut zI;X%C^UZ73Wa9aX%VkF?+e?m5lB!Byazg9%Nlo97U%_lb<xdk^m<F+TcFLXUNwP6F zY`~=?O>8g|9YROPEat+?;_5!BqtfcZUdK<#8J%2rfL~;3OT`D>HmQ&Jds=!Hl4~z# zZ@Xq`^l=(ly|^2X;3ne|p9CaI8f43s@Uuh8^3*R)&dFX5nd2TPkh-!>1S$oouO4Ia zW-CL~)tq|4X6IR;(A8qH>>><PAknL7UgqgU4FE{#1PE6*+K%Fdyq4j9fq0d<YZk_h zc!-x-P}0T?Xf{kP!ha`(WvRhgoZ%-my|}@>34czgC9g$`B-z33^Qrd`m!>O*B@a!H zt!E?By4VSq=@-D}+vruW=-5X;F0r4YEuBj;aK(<V%;?}7W*cFB9xqtl@3b`|-atDQ zx{8gwC_CMxB2QU?9;1@>7yAV=d0#cfpD6oAb>sd%S;_=Jt_|MwIax1Qy>~|!DV7tD zU#S^S@hUhjiSA!(Aj9?tkzl8%-NtKcey{)wfR5`McKlk-vN8}yKb)xAiR-Q<b*QgM zEUY!XWmsk$k0~=rd@X!eJYm`V-9ixAvq<xIKSRZIp^6qXSBJL7ux?=tAOXtFy+cFw zXp~I6@*sT^EMmBi%)O*F9;`W7+G3S=7cG9LE6Z%(2e_c6H>}<D>#RT3DweH7xN&~e z%#zRf^>~uY4(=*P(#Iygxxo#vdg5s(LH;rd-h^+gar^n5KKI7e%L>&b@%s!6&+0ex zeFPd|U3kcfKJ#b3Mw^s62Rm#{?L{-tL5ka;w43TKA*Xy-a<78?hiU+*!$!OY00FDJ z0xv&XgI81veD~KzrnXxHE-!q`WJN-r<5?13W$+Z)6eU#Mk5qA39`*9Cvk)6RLO(gO zRGm0F{0&BPj;O0+{rQhOclonyizt@@wdEpBGk#P5UDw+B1-pWhw`c<Ic*qEOu7(6Q zdky1)(J*VxJ9mr<11tq-K3Mv6fBkD&RtPDTn#bzkfWhzsg)nQ*gC52W*OB}!GVOtK z5j49RqxP~5%H08J?r{zEP}HI;ib<5O>4N+wsdZb~c`jSv<hQ3*hn{PySa!2k4-g-| zT|I%PXXnY5>et81LQ8#(C6Yh682)kvC-j|lg+YMkrS2uXa}R1kUnYyK33ZI}r#C}- z8#;WJcHrFSxkrDmIYLXu&<u6`F5wc;g=TbreN^!J;pUVeY<o^>*1eB+eMFgMKpP^% ziffv7Y2t0-AzvYS81&$swZ=h8caE$lFfU##dJgHkDhal9=7A^lTK<eHn9sBGC>7#q z%Qdx35;jcAeG(8rJWqhuu3pZ=CGVq!kHr7-mzDm)E17MqW4Xo^R%>ev?)MsckZ(4s zm@35>LKm-bGv^+YC&&#sqP^A_vJNI7c)FyjD5q!ib)@7oljuq~X`(60u87?2`b!wR zZ3CUBJ_@8_c3n-q!H>Y&a!=^*b}xVANblY&cSRmxQfVuo%{4BK>-m&LJmrD0?01|J z%1~T5u5O#pB;TT*y58gIW118<5RZyQEXIEvr=_nhbf2dRO=c%%g`ZzRFf77L4}bD8 zUmdTNMkJx+EWG~LC+M$_94i+|>qh$I^niFAQVCCDzwK5POuJ)cb1mF*r?QK=&ykDh zg@M{a&j}kB<C=9@s=#Y`&jFho2d#?)+`y#UwSe`MjDZ)1sETKz`sQeo#L^sp-=$dP zXYYtiT^e|*ne7j4Zt0Vz`9n@k(pe`tCd<5cLgmxAm`O>^2c=Ip_s_QjWGuqrZQm3a z|1M6oH2cKGfU7b!qMm#%_J0erV^Abgzi9`d!-rjveJK5!8B>d%d3hN!{>)+z-?Tj_ zu=X6I($`j@DNg#H8YC`6oU?7DHo#_MLxK?Bw8Z>A{8`1V<x+oml+ro@H%plf3M$k{ zUN}6#l@E0h8XK6dSI#9%G;+K#;T#iU$b1{{{v2naMQ7?J*D}W@e--=)jbe{qx1gjx ziGXGIlU;=b?Uu5h23(wNy7Udiz?3-}>ldnw+}ni#*Y}*}<bFE{5y(*wO+C)!`DS@b zmJ!6w*UEbLEciv{y03RdXRxsVw%nqX5|N4Cz92)+c!H&uJDzR4+q-$T{jN2{IKYGE zhUlzzbw@i&R`acg7c$}z0IQSF8m@fgC-n+NFR{UC3CMXVww7~Kw>HxokA9%L<!{_{ zCKw_P&ATHI{0d*kBezf;n7epJ#n_e$1+i(Y18L++l#aO4VEu}Cy8ewC_dc6ElG+w- zNY#fE>V;s7fDoOVysj=CrTM!>g170&Vab#m1X>em$2q~mKT=1ogg?j;J!DdCtAX!Y zFMmos)=UE+X|jf^NmRhXgpVt%wrVQXs=}&1xr^3BALpQS2_C>3O`jTZwOQU!x@Ha* zZRjP^!f34%7}GqWdEz0PxBXmz%VIrP^=98L7e61f$U@Jr9~u*9daV7S6P0Lf^MF1> z>rsLrZ9yBpnqxFiP6GR<hU_a{Ax>-KhH27%H2X?RN|-EdNlY&f00KtZltDIrm8};Y zr;P6BO&=X2!2rs48oflq3+c@+Yj|1PI^hDCc*7SqOXNINT<#+zAs3`k%n!vUShaQ@ z>?wo!K~{SpPLlMf@uzgnBugjzt<X<>Zz`Epi5E_;Aw(c=w<>DX!F6}v6Ae7G7;o0g zEEz42?BxOlB)(V3vh9hOSaYp&nXRhLNO||yB<RYfC*J4Km^cB@FL8s<7b9m{OBi#v z=p&_TEj<{2kA2zSvHFVsAcn(|1b?1e67#|Kjs^VK7jpU@Oxaa?I(D^M?bD?PM6|F9 z9GK2;HRug@>*=+?(t@ZmFViwL@!cZC!}v3>X2WjPyQ)&3!ld%v^SYBP1GL*qqa68x z1RQL<Sl@d+TX0BHaL7jF&+UJSi0k}WSO}2KcU@OtInj<%)~)T81Mq1xo3%yTJ5P#E zK{R=>uM+YY)V(&Vwd`C+xxJ~G1vzFhj&43~JAWv<|6AH{EPH~1OFK_?;Fzf{Uvp_W zPYsa9jiH0%_M7(x)f3G;!HBiLwhgp2BKG@O(G@<od|3P6x1FWJAgDW|qr68{DmevZ zDuM~ZEK18_(3xiFkwQC~tvy}sD;)w@z!t3lBRL4nh0Ab15U>+AbyG0Yu^eOVflZht zu0F53B_{B%MAo!1Hqe+avl0;ZZp#;P*!{l0w(}eE>$Ur6T0^t&=d>0cDh6ROsi$wv zlWAQVvxaq^xYRLQ&jbvJQr)D22{Q;i^_pt7!~ImcyZW!Cli+3jQG}nlo7r#{Ag|5k z>Cxup6!b5flF`&<vM|&jc7@MTH6o!*eJ|0qH0~6InF{>NCc?Tl@S1`7tt4*He@wfr zQh8S?nmLVFH$Uoo;W@@>F5VNb2KbC-5X>jPqxDUVbd>vaRNR%5y8@r_%E0no@pq{F z=ewfr9w{#pU0fI7e-{<SUirz`<KhY|tG=HXID&nA^$P8^g=^9;$TtCHX1<!h?R^zV z2R<K4I9mRyC7tB|JHjHjdQiOYOW+kn0sn(35<T0;P4JgzdA3uDUi;^-x2UH)7Y(!t z9MW0U5?hWFD$6#e1#&unAJ6*8a>UGQHeOTucnNyCRX1^Ql~c;nwvQHZnv9Y9$k(`* zE#x#=@$UhIif2z%+Z8ZN`Q!plE=PIj6+8eif?E2SYB6g-{m3Zb`~VomtEFuGqey!H z=^I5x0igGzgqO?K(BB1yZgBs50$EYTJgc|&Ui|&CgUQ@iUS3{r{8}lv`u`r#OEd@c zB~fki+$uf?^54L#hMB>RdZi?DigXt5-}=#~wHDVBIhR2?tIk~yO^}dPW_l8zQ~c;u ziezH>JUD1%7Gl;1r%b-0>uSRq152nKmKh*A;7QaD>*zd4|F@@8FX+DgJte?t4MEJM zU>N)+n&xWxBl#>^ow5DwyjOAuo$Gx%Z_788novG%aP?UHF7K9j{z!@>UHM!xLa*EX zpmPhdkv(v_`Kn0+{Ad5^w29R<iK{h(M(f`$qW=QwKN~CX{@sSQ-`mmJN6AMk9vP=I zfPv9@wmizrA|r9&`(ABB!fDsP?&qRy9+<))A5<nW`mJYg8ul`$&cD4Nx4@Q96({Pf zN|6Xx4N=yZx68mfFLXm7yQS<v&iU}}s4s9IoMX4B02{VENd&-CeG~k<-(f=Q!L=(y zsQKl|#5P4v4oCktOv(1n*y5nF?vVsDcH48lU14;xz&6i$+A~*~G}>X}>E}6@(j_tI z*el3*G8K^5K3+dGCq9}g@NBAG4HR<PU#fX+r4GIS_mvN*0R3E^xFY!PMP1ma^#LRb zVfC`02MNCItT{cURu5>!{e-ejP7h(KiE<VXQTFrXk%-lAO0H5|-qh2?Iit1%T3&OP zYtq1d)#+P8|Ev|D!!-US?Tv4bUw(^mX*pgwW7v-S%QJ`kdrA!(nk9t^FD_fxt7$cu zL>zQ|-Ys;Gt&VUUHc^4Zo(!ZmJ$IfTNmlxtf`wZ#B6~c3)iglPI|m>a*~{xAudS|j zgS(vk3ejEu2GkPeEMDRIvZx_|e1)9suOI)r9qlH7AmH;A-dOaqBbQ~}!+{di>;-~7 z{+#R7uF<T;-7KsP>nHB}b#<ebN${rYObt7)aie>NpF>@!x<S4$P9z_LxnA%uyZmM0 z0d26|7y&*T=y4G`(knfhKKFYn;@!l&!yVV5r=RB?^iNM)Otx<Q<rz@2;_z(6YXxIK zMVmYR>j!H4)glo2BowquFs1T_m1M;0e?K1%s&!xPZ5~{!c-<bki}^oxzQ$km2x|cV zTdyGZe+`{0GSNs8;XPOzVViYpr-JM*_0-Dz>)>PA_!^@ncU8oJw8?0f^kL=y`2XS0 zbpUWPUm!U6oK5O*((3<ykONr1`B&C>ntSh)b-<NX{ZCebMgX8Vdfd<=O;hS%Y^*>l z1Do-e=gGHXC&@9(EjV~nKhuBmsF1D7AXzM@ZqCQ?s&3<dGUHH104&=G8GW4be`4+- zG2Eh6n%A^mKzZ!H5I{K(8M*ub)4h+<<0f`viNal1vgz+(1An01Z*GAAaTHZs@$UB< zi@(`l-~@EGW*P*`g}JKmn^V;uYbXDHf5;xVezAp&wLPF#XjlJ#JGtkTlc%1zLbs`J zXclifkQ|S2o;GMQDd@R>-)3_0@UBB}JurfO_)xiN4Dig$ycZLg$%S>{{6p~ew=z9i z7XyPwMsXgLvaM&!`~_EP8qyX_(n9^OC@#!F$3CS2&@OOQ836b73MjQ2t+MHJ@WS2g z@!eY)fNYEvF$A}LuS8es8hS)0Lq1{<!3GdV7u#Leurb&cwiYti><s}**QRDQI2sw= z9Mpy3e|)^Nm#JQz>%Z~j(h8FK0%>LlueP1-V=wFnC~vpAZ+;-Av6w?8N!rgxdSc-A zAG69gFAnWrS2|*3)ssXaAMX%LhO2MPmx?#Fq?+I&`~eusOQi1+A2H&;n-G={y#C>0 zO<6__zy(@7D-}{OUrR_l-pja5<*i25fMc=+*#cl>6ayQRch`SxsE{*{mo|@k11?$v za=U4}g)(e_fK;yPDgwsWu=E9byqs7d=CL9~LO(We(DAE$Fu~eBOf7ttHG*^8G7HR= zP6~INh}T+0g1=<oe0CPvjxQWK5Au*yP*dNn+Vunz7ga&K!P42hS9+y3S7DOIo$nN* zK+rH{_lzq<jYe`uR(!8taC9+RCjDmx!?R%l&&^3mSNCXG#S`6CcPU@b>-BX22VrN$ zU*sXlDCuMPnl`J1TxRNbU;Rf1pJN7#kFyirRSu*{)OjXqj(lNP-zEBGnsF68S&GsW z6!*ww6Lwob%@n;6CgvTeuw2)ksyPBe?{Qb+E7ePINs<BusP40e=PRPEsz9}+>n9t7 zf)PuyzEf3>v^ZwJ{gHe{SBzYpKVAzMpGHZcpVqAzF9OqGm|P#pQ69w?{8~ocKKpS; zYe|khga%)2TWA!62c$a`3n{Rn7e$Rt?e-I7qBc+GFa7<)gcU5O&-XvOzGriV|F)%+ z415}L-!8+?Q~cu3qTo%@oHWSs?y|S+dLs_*h&^9wbmy1o+K>Wt2Nb9UDY!U4oxo$C ze39X<Xp=ZH7_a`H0$2Jpok;B@I5Pk(yh34X%AD^*H;M{u1+3Y}g0?oPL0$t_<p#|# zQ@N~Lu51@+kk<rFrAnnAl%|HnR#W%K%l(B-piob7iXY1qx=WQl2OH2KsiUQ4u)*2< z)R#JHk8*P)daG_eo$zCAP4%)5;<;4Brd6~QVuGq!aP_Dsgra8f#1H@^y?0)(k=W+> zp7sR8bTpEeXr6q9G0z7=JhrmL;b}K6;U>h?Im*$4Are4!80_Q+hqR7V9ULGL6x<QE zDt#lEB<U&ZFnG1TyMdGw{xrk^kF%{0Zt_A+cF<X*N~tA3aK7<Qs;=d9b~~TDyxDyl zwsqetzb~LtwkdknaKa+M?_|q0AcbrU_14ZOp&mgcUi0(I{eLYDZMM9x=D|1Cn>|;W z`8}?~oUUB-{hf9iJI-HNGIBXxJA$I9P)!m}q9D7}w@L^!VeVD2K-2WPB`mB~JDhZ_ zI)Shan2{ym^{Dj-xt&MBSI=6fYzHe2B%&DnaO2Ldn9Xa;juCHCr?jj$8U#ljNKHH1 zZw?U~j%C%>+M&4IMw|dQ1*>&DzEqA3hLQ6!W3nc+pWzH*@OquLI#^gOLUMdHb!${B z;AE6ClVIk4-Iby^p4*`GGFzQGV3!HB%nT96B0Jm*c*Pa)Hmfmty)eq*z)ZWIG_Sd& z^6EjxnYU6+2+G1^6LZ9FfSr_JL|}$*(g72t#1Y&zz&8pSHr7*1LE6@`n(tk8r9k#P z`n~ayC@(nJBgi}Es*=z<utN~b8XZJ70kZ>#&K3Y7uW#sPNG|N|f5&Lo-8XwCo}j*i z?$&f#FATcVrWAOK75#A(GLf{RR6IJeCT6!w&>;I0z<286Ro=#qCoRjLl)3tGd)YEM z*N#51J+KLlAIbnil$0;TpRd#>wN8QgdKXj*^wW9B1C7kZ@z`v|q;bsW-gz<Zy`YA1 z$k2vk_k7Ffh+_>!QH@;C8SJrK2z8^Qu|JOyR?e=4%27G|QNVFS0#z}AaGEDZ&2r7O z3D4xM72C!pdp&R<di)U2i|?6<YoHK4@&A(Gcw;~P{cg{cF4@`mkk{@eWuJW|I_TQb z?o}z=dzZE%yBqJGc(gfxcEY_mKI|WE89$0_tqu3UpSUb7=vESi%n`Mx4-6K0xZXqo zGHe<)+r9S^{k32Zhdq?sqa?h5^G~F2Gp3dMPC<!=_znDrD2^t8=$yMBlA}g7XW3MI zD070~mjaEHUK7RtG#~@I;VG7i^7H^VAwVYsx(8}UWENZ#B>GeuC+_>XMz_}_T&+uf zf8oii{3YtrTLEcY-qqH^;`LF~#nkbddsn&qnb%_<d1$H9>IdCH<6$d{!In4k;kEAm zZEtf-w^9XRjlVmYs=3X|POArVMJ#OyFF(iaosPZIvccKGd}$4J4cvknQv+fqIeSV> zC#2(ZD68x(Fz-S@D?*L5j9cX83V19GM9ePxOM>`TW(<4F2rYPZYJs$F#x>w{`WUPu z$v8H2KEsf?U{08^JD$(_!o+YwYwG=~kE^xG-hx1&-b_YGnLFMNpG(jwm><ixvti0w zf0$%6;~8+&G{5Yd7GD+1YWhfnz_gi3O@NJ<bv51BB}!!49#`1d7-b|oD~jh$aj)-U zI*;Pi?E;jRgfGL{@92Pj4;1ciyaq5d24m=un!FSGJCC##m4o!`Kp)obGY!BeJ`(Wo zK$NnBK1I2(i4TjbbY#nlyVj-|rrD>ZpuPF}+=aduXODJ%vP$Q2Ormz;eb=kZP!Dpi zbNm3s_CL6q2AvMHXifOpbm-+ws8>(!fy5R1)*w7x2xCxs^T&f;Rj2m6k&UGF(dmHU z)ey4P6oshu=-usRync6sife7s96Its%=J{{*9n(fZW%bjVRT<6BB~*j6l?@z{P~#2 zY_&u-`m|?$d}mq0hO18)WG}mB`Dm-bV|4|mY=ZH)6`yFQTsbB}WS~ZnapmYM#NbL0 z((ETsa7nu`p<RatnLOCc+b$b!QKO<zog;q7GC~^oPH7D&{-n{ipO{psHgLPaR=ZC& z0vAE1gmk>L)GtNtx^RDQtg>HyW{0@7nrJ$muCd&lfJ3CKiAjk%&OZVAHJ6GNOESK@ zF)m$x^#7f4<Djs5^6_W!5gxN}E`fYO<C*K=>oC&|*f*%H?2C0ER;qlWZE6RzhO&Zx z8c`eZQePwN7>|-j9(){FvUOIgjMXM9x+XNzz-18XQ`H*9V_$P@0-oLotYtr5V^!DU zpt@GwJMIyF(1f@bxdh<(E%34-%Rbs8<~>ZDYj-K2zfvb8yfRho(v}IYH_cJkvK8eN zmKlLx2$PVP8XjPcPoEc9PR%GyWP0#zhd4WKk-7K3CMK7>QGbbPf~zt_P(Uj2_0rTL z7F-G19_yylXqTM!X2afPX(84n^O(yvm6Hy1$)W|%dT+TIZrOsABb6rP8k}mig_e1m zt*cwTrF5QD%XI!=iP3YbA>Zr{Ry75rX7crfq4e32p(Ph;HMuNaWB~}Hf$eP6ZIl!5 zy!lCcw^wrDZ{2VVeD1#L@?Iq7v8kuTuV6U!?hfU%z|M0l5}okLP?k_>B6}5|_0G63 zeDyImC$7p)8Hp=4?(OB;_G3rGaV4IzYqDRI4+LeC(L~42vqMk*bL9tP>8b}K2dNVI zQNIHC9n*-@CW(VW20uoXXt@<x8O`W{b&8ddj?tsBoCYs#hHt%r?TVIYnSkaUrotoA zt-$mB5UZ{6k*SRP1<m3j;*~^P>-4pCi}1Aj<<<SVZN;Ufd2I6rc9YxU(&7Qw>#RiK zhtReAB{QsYEvzpX-j@@@Jq)(r$5Yckf9v88z`E@wx)7zT2YL&r<Nfu3!2CP-)b%)w z2LsPm=^kZ%@66JhSWMIbJUScwn>S;S8yA@=HkjsFIDyI2Fja@scy|xMsFSHX<+(uy zV(MlU(!|We^9BuhCgRQ5UT^o$&xf!@NCWm`2Mg5DR^ow@AGWtng&D6v{F0Oax?sk1 zFsN1AyNCMS+gt1$SbSpcfb-`eW+v!i4QdGBao#F(46?X6L~b(3U^F?u5Vg3XZy8%s z&-$9QEwJ^ynePEU+@UDiudiCu2mW^Ao{CO7B5>&F@XqW6hwA@p>&oMy-2VS82BoN5 z_DIU5?4<05uB|DNZ7hSbHz8&wTNvC{MY2qGWgm=nFtUv5T1HXU457j>vNZPn_sqTB zzW0yc{4syc>zUVc&gY!-dEVReCd^+#LSZdHLz-mVjhOqLS)-2+X|p8^23N)fcqy(` zH4O83u{isvgP@oz(Q;hRRO%uQVhdj}9-Rv;t;K~M-Dm`rZ<*v`2Fc&}uqZ?pPM|z= z;Faz$df%WK%i4J`5B+YhNf8sUmJso?+s!zw=q4tl=<4v0M(df*L(Og4Yg%ZY%**kL zh!8}lP^{$q=5VM4X}D5z_s9jCeKXVP0*B%Y%m};>OR;qmW$Buunk^&4*jV1P5hYkG zPMs29FPL!mLepfR7-A*Sd1fTJOF-tsm%(6;pj`!bou?CM-)9^3E>?^}B>&`>z~JQU z3x+gk)Vm>w@OU9MYWSp>lX>0|hUFHqpuQr&OLapaMm(uYA=}D>>IA8mNjuBOvA8Oq zZ2@Y*O=?GqH+SS<n-IgYF=G>R+UqD`*q@M~{`rBG2}diw1Je&GjYNZF@HR|4ms&^H zE`9K6hLMsWqu4l3Tw|;TlxN>#xq4mrc4;{?dFy3j*=D_hk6-UK!O{V82YuS!Ck|$+ zNpkh2P1!@LTcWuZ$V8fDD829F-lVwYIk9ks<$1&C-{<z3S%L1V*`K|J!<8O|pANoZ zo5i$az93O)xS}TqRgOK*t3bLRyp0cEk;9)WgAhsa0epll)67+W9gVEa6E5}NbJD|$ zoA#I;yDp7E!a)L%vlzKfoI_X>4Rm*@uyk%A&|TU5tGan`_MMpT(h*LFGoA4Rd$8A? z!PZ?YIaw&YDey_8zJO0~ZJUEZD!mnBa!7OFn!p4TeuZhrl4A_}H3c3Icrt16WoXtO z7t4!970a3oHgV?E`E@4HFqfQ|wwwYqKhnN##f|#o4LP)mIrE?8@&1`$tyzy);7B~U zTc%pDRzfMf*L(3QSDI0c-KlNVlwQaTG#^Uout7-YBD=BI@3PKwT`C|x_JSS~3r-ds z5IMP?XUS25y|<l@A?v}5lFUvl4uAzdnG^uBI}qQ~@m3y=YW*U@b`t+lCcNr5uPi98 z@DVl<%c4wqYh}o^A@1KbV3;$GRr9y^U$bpHzCF$*Y>t23Lh`oLe3+M$zk}t>bLUO( zy7TaHv9{(i4LnYLcn%T|TjVulGSf{v9pN_iAS=hmlk$85A5^kB_qd`)p{81xs>z;? zp6Pmp43`XB#v8S%`rBqahk#-w`wweT4d^B*8Ls%&O%B%fl^)!#qXRv|aE$t=Yjq>~ z-426(yBqv)l~}h{AJ-==w8@LdD~g3IN$L}4={i%XAV0dHr*lf4MfXm<Jf$YrBKhP; z(z)Oev%;9ri=v5Wr?8}G+h~v@>CGx~nrNMQ8K6lv3#;ly@wdk6YCAC(GlEph4WdNG zd0TmbKC363el>)XFCG09S&|#B7>y(aLp{5qZ>w9G609xKZfJGGy4|9+Px(M?@)7D| zB<0Kl?wJj)J5zSUnP7o4P`;6Y3|ak5Z5|zCgwAJaNqJeqxeqn&w?2k>jDHN9ONmfA z0~O@FpKdv6E6#Yokt}GkdI00cFLd7It78Pfr*(^;&YYPOb{oq12@<8|v91qRxR9au zB>re~G=`*utOwv3Az7L|Pq<riPVzKiKY+~r^AZbq(=CEXK_i!NH<QSZAJh)2mCDad z&;CK6WNALpoMiPxer&PI|1Hl(1LuKD9d^C!71w9YxD@WZ{Z+7vCoR^sqP_BEqCRBq zMadjFyIRWKQwkevmn=U9dW-!%so)QbNr8gieSLRq<s|M)G7z4^GL)u1OXCU-ryga_ zHrngE^aU4kqe8Gx5%R+UkMb!zfL-ChikfvLzrWOu5h?yQD&y^$;qHSPn80R%Ed)vx z?|k72CP^TBSG&%e^bXA3{nOqwhb~!^*#kK)N1B%|cPR1B)xqb^`bLo2xlRU0U7@y# zct}w32ugFy-B~9)8r95rGTesG+^h83pzy7L*oXXmxYC2#Zyf!kn5!=e;c<+6vz^-r zZ_!KsnHgN>ytENSmm0wjJ~@K@gKi!+a+0&naS$F~T`?=PP8-y5&n`ZE7~e@rAgo1) z);qd#rltFJAH{!f5P%N|#LjZBTMDnsMtfvR!+SUSI7!!<y#u2YM=e*C_tqt{hW<rx zEb}hBP%XJ|!|k(7<wE1zCqdDY@b-Y0fRk0`CD5Y!5?Qh*zi7^l1lnn00U-{;^Q~v7 z{4<}T2^7;}`~~t7ZW??2A{P<MDn{9BopJv#Iu6eSU1pFIO&M`Xckk+pZZzkt*Z`!t z4TsppV8XaWPqJW)#Dm^1WIyAUX6#4||7S{~apYwmzUK!n;<>V|n8SyY?jS@3X8kCm zI3#$#tP-I+mSc%%-SGy8qIZvRs)BSSO35Ite7f4Uxz+rWXGRr|qa^SyR2a8_ko+Xp zC$PFPhNkm99AYqktvePH&Wg`k;=LCW<87SV)MI74)G_I~SO~KCu~=U8suH{Gq|ki` zr=-qBL=8*5zs~QLgvp{zp$5e?nho;~q6X)I_ZHd$hbx1;go^4@d-=q)i8I(yb+SIW z1_(U%Huu$x#3=cVHf+Dz)FhqsK>?J3q6}vmvnWhQtCL)Isf)5|J{tW_>u0z0=0%V9 z;>*F0rxkHxh=H`q8t0SnYWWUJhCUaM($GX$p}^zSi)|O%%((VH=s41VJ^GgS6b=@~ zN(WdJ&IBW@&6>2B>IN-_Z0$hkB5o#cBybHyq6rtJ?Zu_8WVzHiIzPeBLrp(VOm1F( zl31%5nBUi_1~cEk2(9dDXzjCf5aqb#6P>S8TcMg~#+=gaX=N?E3%R;d^=8AW`|Inn zIh_k4%Iy}#taQ2P8~tNj&YwSj%lJ4onSz=Z?DkmU%I>OdAe(xQUUA2F;@<lV;pQkk z7Cei)R$+=o1>k$J^((gjHirMesh4ek9v1(=v8kk=8^u5HtQzWPA^ro`-l+e{|FCg* z0HMYHd{F&|I4PX@Thj0g*X`y1i4OdO^XC7HO_@)d#KWe6G_PJjK-L4mP+2Q!dpk9* zZL!;STcHj_U9<WZtER6sjsu~hm($Msb)_|v-5Ymk!8~^jLVsNody2aqAiqwOA}b_) zhAO3?W@PFFos8}U5WgN~fAP}Z9Qf+ikaPWTjSqTIxvhS`3mcWboTaPtL|jMniLmPL zmoqfgfSe3bdUvn``G&QwAHc1Z{kp?g4xf=aWgS-g$(gc<9ds&?L_x3i?j0RtYn<4i z8mVcM$&Td6eR1J_4UZh)Nn4;WGTACIHumqPuJaoQD>00gc-niCmgEhj&yinYA0gEV z3aC`hgCdD+>=H{mU69K@7H3|K?f7bLOeFOF?X|zNzIY&y^U#4q%L`+A00)Tke*Nb@ zatF?Yd2<nbNYS~eVU+<W!heluVKL8HNH!bz2TVbJ2zI)_J-+fP00;jYCiJ`WO>_o& zd>y_yP}`LBrC<P6otfL2y2ciQgw12J5IIIqXrz4OD5P-z5|A2Y8qLMmwKzBgp#pJ$ z@5aLc%}Rb?!#A-R)vS$6n>5>|XS-_=d*{Od_1Yys0jbqH*tAOS1Egeky4Zmxozo*D z;ahWL)7vIpJ7H^|32dgauhwsj%`TAsS79X54WY1Ehq2W8x^J=7K|j9UJp3JSAZ~UA zHYMo(cz7k3Ej7?t{%~PPq)lc22e@qUjl(7@Ix+SL1R_;CIN;t#HY!%++#EjCqYlW5 zZ~7z@U}=uVlb}cz@S3vQsmiWK_vAPDYh4C3X9%ngE~>xxiXJ-O=i^JmjddW`X{x!n z6UmC-1BgvlI9p~l12G7GpQWb<gpovCTnHW8s#JSUMur{ZDI~}{BzG2yPky>FTXdFs z3|+oFSMW}*i>&a$koM_S;KK4>is8y(-(8O*7+DeFoIyV7;oIvY;nbywT}DI?P(ZUc zPb39Keq_@ywbBIC+|`4-w0u-`{6+!OVzs}OR!~~~aBRmz;tN}^$k=_&qxcSIb33i( zY33?MmrU49K4%``jp#1Gn0Hyq^qu;(@^HBR*D{4EXT7h600~^ZapB83H{+K?XN8`h zI7BNY@UH+$&yh4<f>WEnC=3lOvdd#zqia0?`n-$%@t0shRkyCz&Ua6}PM2YTJUC!U zEr*W+<DNtUwB6Ww3L~GLXfVSUv3}v6k_5-oiLzq=urt=fCZ?ButO00feCgeiXxf4n z)nlvEdG{Fu73Gw+jThjV@B<hL$MfYYR4AWY_Otud$dk^CQG&UOVKaze=M}I6buK^6 z_ad8X>76Oa%8Y}><4fSG=FoXtr(k!pJ-}gYcpEcdGZi$DP@~S~1%Nd&e_(MnDN7~# zPMber7nr2f;U9qDEKK0>=$)na!Uk;dBNianCs;?l_2zq+dGoY=21%mWEVj>I8xZov ze-6{H-eHqC1#AcRkU?=HxGF&X3Z1UZYz;$(fjuh<McY64mK1mC1F{lUF!$Zm++SNF ziDWMX<E75zCYaHlLeBvz+j#w|2jsY|-<2SzISi0{$Ytc(G1Y*H_GJKa*4k-{*f!=& ziv!%uGbfL4n1lcWmwAHFf7l#sF<^Xaz358V8K67&7lwdO2wwYu)=O4jY`flZ-E%B! zdqR3B3AwSiu(zui5%_wa^h9&y=g`rm%w=E@6daI|M?XPa*uTQ)wjGbSvk6RK8p<_N zZdShBa+IOX&<NHup9$SS#0)E=%T&U@@Hn@VszDzX<YJ(l2ih?@l)eC;WWMa)b5V!; zkE|!o8|!U;9NArQOwEV}uBZKqx5~}t%`{LWYc~yHEZSS5{o9Zk?I_fIo6FEHd-yPo zY*?DX@>XU1u^Aqi>9qb@&;wnf4{1AHfXOcDD!`z+5(3b!*|Uz}(-)4@<`}){wsXJj zS2O)}qM%SfjK+RW70@yKS>t$E$v|qGO6wW!*T`n>$CZVV;>qUvL7>w@TMS?ceGl0W z(HAubYqE13*6e3NV`-zy`RV!1VSFKpUy7Y~Zq$73u-)9-UD(T9<Nw4}s0$FFL#gXb z+h)z~P<IPhvRn>GbUf00!!cwgPnbkK#Y6mN=c8Y_06H?h#YXaWx0?5~YGW3r-lZh? zQYMp^@11WraUo4RWc7PWFEqL-L0SU#0R9|DfTc)TZ&0&B`gwM9dFiN~{uOx7C~zml zjts$NZ&jN<$?U!Ne&MRn%4#y~0Y`#d1G6o_`?yc*#gc>DWli`jvST>&U|+_SHWNNK zsF{x!2vS$P0EFZ;EIaS5I8)>)5$HGOg{Hp_QznxLc=XrrN0@X+>@M}nI{kRG41^$Q zHv$vTJy8Air9D<jX}{1IE4kLKQ>61Wk5sYvgOK-Vr-aBOZ^3!zn|?eQQsH@r+<+hM z6#G=qBo<d%TWqdkv}MH*xvX1zu-b6X7FE5&>0tQNwy@`&it4wud8BVu@(E!Y79w+& z|EPjBl!Vc~YGtniTfYh+C9Pzk?jTfgfX=RuA%6Zlf2S>NMH`IyXTesP=6;ty0N#VC z+t;sglyhA*cCmU@AyK8^e5(aU5kp?3^i`@vTl+R~9!&3KE5A3kw?<uHA8zmNlpRbQ z3_f>gTpQCp^>wF`&?v{4ZNU7{x%%?-{n-8g+3~F3Oz&+)>?uNJ5YOEWAHlw}^@wl_ z38Fk;w(-^lEYAUJNt{7P*7M&DbI9WdcjfXO5j$na91ZQ{DG*=~v%0ebJERB`?(Wpz zr`_`8NN?mar*t5Bor=xcj2BK*N`I$J*&vfOL<v{RcfK9p3o1jXEKU$N0a8$d!Ba_9 zRBzbboW*QHIQ=!kKgKvEeohc`b<S>;7s>`-5O|9wQGSO?qE85U=p>zAv!yz9s}OzZ zmo@Ncog}Ks6rQ-%LM_DVs|w-LH!|f~N4zqiE4a+aB8`@=^TnrSE9=jwqKnQcKLDxp zkG0y<KHG7)%z>N8@k<j8us`qdM*r$k>g$Qu61QX@D3#s%RDMl(5hNKSg3gmL0L0a> zp)W6KuaO6HK<(+K`uAwsCgtrSn?&2jGY%g(Ca&du>)E%S+fvUh6q!jOjD#kEA4h3} z11`GRm$yqL^Yd#9?3?};Dxg(9b)HnG7~FvfsArxzz7eUmLRlGb!0Z73>u!|>QozW} z+0wXo6R(8CUbEn+a<mQOnoc4($i>`PaqPOW;iToi=7UZVcshL^-zM34{j<1`*m+;a zE@wA4r8C5`vG<BAjV#j(qIkrd9ZwFOuHi21@x#hyMbRgFj?8n&GkB%3=G<OQDfjCa z=F&jk`OjXnxEP2@%G>V<{fbnikSo(Ggdl=XjbKKw#nX$P5r<x*ss>bO5U#05?tKN{ zoVMtL9-UGqN8tCZ1-!rpk@5Hg#V;2UVdaq)c-zKRsi3RjO4j4g*+VJs;;J(lTdd1c z_B=WcHpKu6MSUm0Lk}&|b>VpcaB9Jno3wU=&S#yH?WJ{8ndfVI>QE1wW7N6+t`;dA zXlJ!oTq?l-aJFScOENWQBb5vYE^U02h~izPs~#T{Ni578cT|#qgxdZ7W2k75T#DRB z`crF)h*6$v1jd9cjV1!oggIxc!30z?5@%yG=g=*$ag^!ypZf1N>YX&BL2Q$!K^jwz zkW1#*KbwrdP3>U8-qBJsR{7uNix}B`L&77)*UZrYQT#Tx8x?O;in*`v@HuhRE7$gg zXCIm{iR1bH;&mZbUQhD4Z?nYVTSak`_QyQK<xE`pw`ZOAQokJ2rrFxzt0YVvAB*KY zhVz#bvz8<4UFA@&;WDzP1|9MHzQY@q(J<XSo)P_Yeef(MdD3IzO-4$LoL4D|{@3nb z1_x?dD!=*k{2j$pN*cEeqt;>OgqD2mBJ3VTDsWqSU|jOvqv*PFCBTCiBefItX~Yrd zL8;MOOtJy2RZ9QqaEk?uTA)Hwq*_P4PVLLS#1U7Rgv$9N>ihnLwu1O%Wok$1UiHZ) zU}d}^hyCo;fTam~$BMuwu>7L*!R(-g#<i(&%~jXCN5;9Hik*23ak|BQ6Kw0R%;`qm zUs%CgIUl9xGN+E`R7tYXvGb!$R&rdlq==z?3n`ha&}qnFuz9k<SQWBTr8rclvk;{J zS7ZOS<%-A7*|>zs!xE3e(a=nn3K63g>2_-c>Vf*mCe24iH{p1xH20%O(NiFD*U9pv zIF;eH%~E*pIP_Ma89e0S(VJBL3~=pox^!Dd@5)zgV2XMFmDmA^p7m6jMUeT33D|}l zQwZ0<LmMJhw<;VVa*Lc2;NQ$)r=nnBD_<gH==s`4525KS^c3|sIQQGAl<SR0ypwoj zrooj{aS-9QPsiUzY0swZZ}@f|WNigH<Or^neEJ?z=p<o}mzpmwsT{8_@!oIN5>Sy` z=@GzVW8{8K*h;Y6E4Fy2KeAI%>D%=KOe9CANG3!j>9%s9l@46(0q4fb+x<ioSc%&z z0+-aJokV^&Ga>L^N{E}t_1vwIqnXCx1&iGT&Z+d-FG22t)^*4ml!-GrOCEYl#)iTk zpE=&KBQ8nJHIVOJz*U`X(l#%P(2C^iEUWHdjtiK6QgdBA!A_R^mtzLZ!rXW4Msqo4 z!D@zZNFVsBjbX9^6->t`z;#A!DMe8T@A5Mcy+uhEUbGlmv{>ZO1TT+>odV4885hBJ zz-nCjnc}TVXa8Zf?nH#VkjmbX#iNm)lZ+^uiY-R~q7hUaLgTs_DLbGgfUxPm?qelh zUKl=gAz2f8ZRQ~nEiPMbWz=?t51p;@=vZffoDS*T>b3XolN_(9&GIRNL?64vxw29Y zu4jTKR=hsM$DD@`9Q9HS_D@<7n=NGBO0hn-a<;6W;@{MFL4z_<>46dT=<gqp%aF^a z7xrBkuApg(XKQkC^43WbBrKVyuMF~Ph-0fXw@6OQ<K~44|9kK~%i~u5AyU#ef2#_g zB!xom!7n$4A#R<da&pcM3rkqi<g+SQy>-x1>Kx(JlEm6$oe<*V@d~GE<jlswaeA#& z*qU^kEP<urlsT8g@-kj@loT4gsFSWL(fyivY+&GxE>q#jZ>E$x3$J=Or=GO3<X<S{ z|LTErP*T^?q1hGRU+F1T@yS43w}#7d&TM>ICB*IAn6*K*+T&%<;69J(ZdqzHC05iN z;JO;xg$SkB_cfo(pu3_Y`0oCv#{XX!fyk31Xk8;-@t1(Z&HsK4m<|JWw7(^j|7yp2 bm-nEa5z5`qhQg{qz{k|k><^NGOVs}XD!f-0 literal 0 HcmV?d00001 diff --git a/website/www/site/static/images/blog/apache-beam-flink-and-kubernetes/flink-checkpoint-ui.png b/website/www/site/static/images/blog/apache-beam-flink-and-kubernetes/flink-checkpoint-ui.png new file mode 100644 index 0000000000000000000000000000000000000000..c9e95990da6e662fd23853637ecf5e8ae809c70e GIT binary patch literal 413413 zcmaHTbyQp5wr(j7Ed+OmVl5Oe1ean33KS{Dp~Z^3I|TO@r??YnaZ4#dOK~d}T!ID- zmOOs%zH`SL@7#C($k-WsXU`>bt#5u)R$|_2DnBKlC3y7c(Nk3wMV&{FaA_Ys!ZOFh zdAPF@t$Y3GQOF}zMY(rArU!Z0t$H(;ef$>8CX*&b5qMz{9^USIN1A&$asW9AED}Wk zHU~LCPIZqGYYzax0Iv9>5im#sVN$gJ8fn<E_`p@|@h@?IaHe}*+a-+^xJ$wMCG``W z@E-GjyR^uvybL4$x7+{cw^8O;`|&IW;2>UJ{TjpT<L3sR*QXi#TRFeKqPbv(4VHbe z6#qQ=f4>}k#Gj>6kT&Wt{paB+OkY3vyWGB*^UPP}r2X9nG6}Es63tx6Qr&W`{qsE` z3HR-q(R``$A3I!T@Bd#}|1JIK`#+tTv7i0&Pn&mQ8C9m0CXfak2=cqv!3wzRmKZ+1 z#_xFm<<66K*EGw6d;{|bZXCN=&uz>&L#A5(hkpK-?2Hbc9tkUzK{kPp|BMH%w^c~} za7~fl7n$mR0c>&~!(g~uuh7hws3xT1-y5DeKHt!i*=vEJO3`;aGB*(Ptqd4TMs?xe zrx&O9klFCllbG}q{C@^E9IO|UXWC$8^X|UCw#jv$?{Mj?jfhdYimZGTeisS$ydG}3 zSsP%w*at)QU})5pXT!w*Bs7oB&_=j$;-Ah2_C>t@9~6584D)<VMqB&O>>IqX75;bi zuXR5Nxm(0bU_k`)y9)Vo6w9ysa&%ugcdKV!M{WG`CNbYLVcLIRH!d)e8X@Vog%eaH zWIh~TI7(WnswqsW)yWlmovFqw+NA!$X<VCYB?K}1fnOd?OkgQL=u6T<zW9Gwf5eY^ zBsMU~Xi1)u&=&V=m8aWHRa!=d!N)6VOpo4wAj6rNLRutTsG;9_*5vECGR0LUXR4B) z8p0hq7|50k3m%pNO78zZ1^u|J!cowniezd#3Ly*oEq}hac46K!jvov5JHtT|ZFcpe z&J1N>@2-+KV6+3e$dA^H^Lx8~Y@v=gDVGV{I`#|mVUu$8v%w0L^p^|u4;n<f2o(Mg zI*GbXG|xkH6DEM-@lz!WZNA&+9;#srd-^yY-nrEC1xo80bmtc9c0Ah~yjhcI2>RB& zBK`(;46kF_fzJNL*KVrcu<|Q(lcX5@V#KZUN$0MF=3;(>#>uw#A0==6hY{ZL2;UrA z2MFh#?u=G8gex>T_B}Frg=|Ni{gDXt_UhuxGpz3I_-cCwkGVPTLK&eoy&W^BWc~Zg zCbfhZXG>Kln$UevS{QpAE%~3y$M1h|DaF3I<kEYXSiAq!EU)l0S=%?Y1Rn|a?|r2_ zP=oaMUY<FqvtBv$u`}#wcFf>%FXsncWAN>VY_t*6h0}YLa0N2ExmCp*R~@%|i7izI zYW+89IbYr${%T7XH+dEbg*3eQXJ->rk<mW=ceH9f#+#)%t{zvxkNK*a#4%bz?|W7e zJcc9j-L5fc5nU9tETsZrJV7z~DnaNADhjmT%+B<ZH9D$l>5b%QClIjDc1yqa_-M^| z2scqva3I%NJD|ysw9|=X9}G|c{)2B!|6#qaco(LG%~S6+JW<PMF20G{p4K{3-)(8( zh0l5K@6K@~Tugm`*524!Ni<eh+|)~GYdM9bRsh$2%9mrxo4qqp)L$z6O-EZVh_70( zT&px0`ApcT;WUZoKkP6-p@2@<n^<zr@O6%nl6LHZiED(&#o1ETjjtT$eP?}bkkP!7 z%t){tZ1k{N9qc)=roA=OqC?M-M5fMg_HhEK#xdPw&645sEKRUPbDQvMCmIpu^$#FJ z-v9q0A%I!lN~QA&K}?of!+NRy_0rhx{0ULJL`_~d!^E5D=;`Sg0X@%;elPA{R!4T{ z2*`dde>#z>S(+iOB&-+6lZKPdRTFa&Yj6JNfG3k9OP*1qiVJ_z*VCrG@V~T6`5&~~ zF+O<IYds_c-#gzs_h@gd$^?E*>=ab_s_b9tP0a9l<;^~={#>?KX_LDlwn_eyCQOsD zn2oKy+H^(O{>GT-Y2{l*@GO4?D#NWJsQGzSK3_`_`x}Rj2OzvIC;tx(7msJErmWk% zE!pLM$K{~!`Qn4D2Ems8MyIc*$0{I?={B{SQQ}?s;`wwL&mWOCMH-E|4~8}aB6Bk_ z{Ck6-Li^-f?wMe|awC}ntrw>|IvtX}!aUw~2_t&1Q#RPe9<VwiSk3pRJe#NXu`}e} zOKTW{x&Zqx$oxBY-<>^jzJD+P@bZG7$s`%n?$?4~)Zd-LwV`9|FwcI-TicNm6?{Qs zGAgT5?UGDxa_*ZAwN&P;1#;fPp#<70t;k_o{q9)J=EHI_si?h?dXk%V7&C*zgkD+! zxF#qv7QS1ddE@s>-Dsk#{QEQYs;Iy2)plPVSEaFl+iiHu@W=GdQF*_sW!_T<{%^#y z{-@tDH@_kYB|LVMrO|h19em1(^b;Ap21j;G|GraCAjCc&BcRUR*RPe67-#aO{2M*@ z^&J-)hxiv;oaaI@aXn5qRZY75W?)D+v4>5K7AzES`xWo^N{&`3AxFNFtV}yQV(8qb z=2cZ%OLCdWQ7LyqbVYDev(c>QH%)J*s<gtci;vYTH&y7Dc}IT@oHJ@mNg{v4SK~0x zw{}zD@4K=EeLSB8mZh5p)fJhJn3&$XeR<b+E>14~>2V}`{gu!^TaTOA(Dn{>d)(xE z_DLgKSao9{39>I})&Jyhx#KE-S=;j8SWy%@keSxBJK!&D{QKv>f*N@edi>qUqMM8T z@`piI8Gub`U%7mHoj|*FHeRh?JK1105RgcBbf*cZ5khguTWv;Ukqf!LF4fF39ZD@9 zFX^8VF0@c;;(l*LxR5xPz2eYVbLDSzweE4qbpM5jD7(b{#V!2zaG*!yA%Nj%@`KQ@ zfnEXY$0n<BuE6OZ1&pe+2Fva<x;f>)D+4br;n~FzJT{}nuVQp~*i9uO*+&8tWd7Oh zGDeK`<H%)inBAXj5!#pLAq5rwS4V*cA9u$KQF$Ey7Xqz?JNOwq!RcM@-A?=8#v0w_ z0=|;)zpPpDKbG>`pVfHy=sif$D}Mb{4{Unxw}!5pt~Su>jV7LZm<G?i$x`6LGIfp6 zihH2|yWH!(H|cVFx-ymP;%9b-whFa}%O9kMTPD9J5;j^P`55vq+cf{cD#_eu9Tvy8 z7DUVoM?L6jg$e^3yS%&ujUaiizf_gYDpfh?|B$ZAy!G8u%lN<{C<Dgv<*7EA@S`;@ zP4LsRt+~^ZDo>ubyYo6^NO-Ux3d5L`v=|rK$6Z{P8>9c?Ppu+8kr>{lXNfc&L88=0 zM0fh1`0h`7iM=~>{)?CNhGQO^Z*u#BJ5OVI4C-pC^s1}$YmKT*I=osrgoUSN%G5G= z?5D~SRszm6(Aw24yrMY!@mKF`tqm+Tk(C+D+G<<Dr|&(%<L}fv;wN%yaqQLZeQz3J zd<>cO24<rxrB-K`yFVKbZyM(Y7G{DoF1@LX8T2G<KOrQtjoO=ywq6v!*>l@%<8-SG z>KXR?ac%y>N$4G~1kp^CDcgly3(Ed|b1W&Rdb!%3dxcAOIR!^y%s=C08-~gLUkvYx zxKsU8Z`9Llpm&`<*z^z*yj|dzFy>6JIJd|VK@V{7?WEi!hh(6*UA%6$zM36zdA-8T zaos^j5>f>KS5&2fam@)Cv-*#4x2LJh0$c|W9=zj}qL3-WjV1S;(Y=$k-gdXGVTL(( zA?8*=PN%Cpz1(nyA>~f0I240S&=Tyu8}MR@*nHpt=Ptj9*^fIXx{I&t4+s)-kst_* zF1zXpwX%L~%b<`O8jBp4wrLy3%_eTR2)VZRIl&aRX^3^Y8#f_HtE=)fzc&HAECCEb zEhJS#zSlw?kG!YaZ!w~(=eF8HE;5v#F&qNo#DE3Iw&9-J`hl@r72yp!PP5Hx5aP|p zD?P|bb^c22;@2(Y)V(~QM2#HL8E6cd=k7R*j3#UB17FL>PrwUcan<G4o_sT$Q<`i< z>0D<(*I7|0(!N)#FLdEmEwUiE>3+!MuY$|TG>KA;KL2~tWi(?KCk^kEw~8|`n0tOL z%<8P#G<aDf$oZzmLv^xB_iw+^Okmh$qMxqWn{05JJJtTr^S`u9ty!nmqeTRJ-yMS1 zvT47zgq@ZS7uil&@v?il1xH)CFda9aFb*bHaCfFcd56eJ*=kw)f=`a4o~W*7%+rvc zs_=l#R4Gp1R?#K4adMcxj#SigcRbK0i`JIxkj&L#nV9tBxH$G&pJ>iswk*5qRu4Kf zhPNho+eeKbDX2Qbwj1fB9I5Y1`NooxeC|ORYt1)qA2_5Bt-zae{fwNsne26zrMJaq zu7W-Vk*-Uy(wBSz2UlWS;d{q0V_T0720I{YAEn>_F<8AYjO1a-$>8mgEY$5F3q7*{ zJu@jW{i|<(f7m#jZVoLpIm`u;eQ2G^zRD4^n^a_b9Y3DJrhMnH2O$Gce|i4Ch*;)s z9DQ{NSd7Q^@4mlW>?UTW2@mEp-+y=$-ZW=ARSoWjLxOF#hBHtNEvFbT1<By^31wO_ zr(E<3BmBYXj2tbH30C-E5h~4!9LW(I3x<p}+D+X_ccO0gQMVz0>c<;>7?z&5mxo66 z=AoAdf88--lpxf=qPb?rMW>2zFNB!YPTl?LF&n``DZjToxsVf<Q~$_q!s|r(N_?_s z$jfNS?&TXsVdi>Y8QnO_7k2~YFHG9c+uw!+4q?-~i@6`H_r=|uPifs7vdkiHhd2wC zn(s*jK)MHA&aiH}^_@?$@kT99c}zd1LzyraKh@k`9bZ8Pz*hb76GcjKPOtbR4_3N9 zICnnqvZJNF0ZK%Si@Nbkvtx42>DR{h$c-eCU{pXb>cn~baZz9Gwxr9&iLE46NAl2S zD7gCuj)P`RCMC2QNW3_E?ttN*W{4N@fpC{f)c<<xtBAvF%}Z?k#E<m!zZ2;eIF<7y zJmFDR&e#EAOeREDZ#LDT5Ly3mvw*b)dMUrg2R*tkvP*nJLm7ZU%x4X259a=!x)6Pi z{~@)Y9wU(8IaEsEyX(elnYl?^w^wabfn(@$t%9I?`hfcf5HW2T%M!9V@4;n4x`$l8 z4<9P+X&X<-IY!`b1nt+hc-)-teOSo89_9^u^|iYVrI975bliPgC-d3>*J}F{VgE?G z<mdU95cCMgsjs5tRH(1K=}(;L0Y6ImuY8GZe?ch6zu&j&oFu+U&+hicQQacq%n&5^ z#Le6Y`MXrMrG`5ot0?v+(30F=upFd3tA0R^*B_s-Oq-w)!+Gc1GWUKy=5mm(r#XSE zzUTHBay-E6wSg12ir8RtV?7I(RFPcPU=&`Whhq7Lw2d~o)`w7Yv9rqRAiaHHKYpZS zvNf~Oaz1#fQY5r&oP*zzeeB5DER6JqrxXv>FLd`)mqqFq;^4;3U&}=Nw5mJkr)&bp z>=%=NEJz%r&pwr`QDP8vu)f&i$>|CqDEVC(W|yy5dq6S$%DF;<+@VgO@V=>P|H&cE zX>&JqhsuBrc$~`p%Wkc7?{uk;Qd1BQxL?xX^)cnsnNv&8rV5pcvh5dOIE_26Ytb8m zn1Z#a?iD+^gIM|(U8j^)B}JtURpy8)wSum1MVx=&gho9BFz|X@JR<+?ug10v-e7SG zZasyu%@@@wNo8Fv>q%m$dG37uZR5Uot=66tXP@6TdL+nXSbntUxjLgr=T_F1y7-Jw z!LO_Cn-8w2h$+xH9l1^it8g4I(dIA58k1o-bDT_|ky+#~^oW9uvEvMMqYfg;F>_#Q zpU-dT*j)9cQi?$^*v}xHId|;eyi-$=6f#O)k&5cuw^vaTM`b?uaVe4=f<BIt#}~lh z1|GFc=@$8QoW2s9@Mu!0Quh47-MO>ADjvHs$=I7)Z?WmyZ*e;<%IngFBl$@Pu`?F0 z1)Hv8LD-^Chu(D#Uc&T{5+2o5=jX4%?j*xWR(vB5tP?z%2aNvkz+b;HO8FsUa+%75 zmHwN$Mu8rulIpZQa2r&I39}tbH+>c^2Hp^EAqESEuU89}p<>U4pgeI2r{b%U-MrNJ zE7Ev|Uu0qFuty9)uX9zVh-mX)g3o2A<>0cLMAR>34;Pv`G%ec^h_9l|1uQ%yj?A7c zY$Tnp7$?Of{ZDf6!svdrJ>Wx-3`_F${@}>Z%U_s*k$2lIZwjOX?=BIzn_`>1P?^V% ztv@jnF-q>XC>hJ?uo7T}HOLj6ErL*Ta9q;NJ33363Cu!l!86TOFYpqd+|!<6>g&L{ zn)gCY_A_dZi_I5G?c0BHZm|Rhmh8*tVfgdpke3B*l|S))Kz^WQD7LHTIy454EHP9z zLN1f#8Hj8>p_Rkw3%?m-Su!Kzpd1wSGZgb4wm?D=`-&(f#rv%Qm>G#ki+h1e?M3DW zS>h_z7b-;G6d-_TeJfa}?++PiMfUaBJPthm5u1pC#mJm}px40(cECZ@qsMkyu417l z435js(Wl~pAqO@A=W(5(hJrpP<`C2V-Mf%6K-mL<^bT|q0>@x4xLLBfjwWD^d~{>* zbo&f8?W%))Ti<@{OGgqTQIX^R>eKi)Ig}xQmu+3^MrB3zTVXH~cX3<>&T-soax@(+ zgUmhNqjP)!s^`rKDmfwaiLF}^8dPlOp`9^?ktVt0gPFEt6mFPadn;WXGGKr~miK&o zjwdyy_;Dna>4`hg)19bop^qnwXh4W^!8?rNEEL%{8`Ak3Egof+XREq=p0RZ3MVY3M zwI5m}BX*X&0}8%b>z9?;10s3?e)dD3T23XiDBLZLwl)w4aVU4>@bAh55vogKfy_29 z|M3;%!f}EEun5K*`;fapL=pp?=;vL+_at3%&z`Sho|2q}K;l!H(v3-cgb0I4+^Crz z^9Oxf75c2O;?ZHh$0<N<tBd;cltJ@gn;!Ltc349d`KaUViIDh@U^FVIbAm{;ba_nr z_MjPiNp_sc4da^ixr`zfGG>1D2CV*-FDQ|al7s~MUe}Oms4bp_RZO1d=*fJ}^{A(c z&tulewmt!4$TZjt2I(f2&PUHx>JC7NHvHl}xmbe;A3`t*<J2-;@|OJ*x_bmw-hD%~ zrW;R1_-73c)t>6xhBjwUnkUS9b(IhU^QVEe-VUG6w1f-S1QY5+%=pu33Tb_-D&=~- z;BUW|-x!>e+T8E?ftTeSeJrxPhz;wc!YTL12fdp-H|aI(r_}i~z~y_9ReRyAO-&7N zbRD^t6+3>nOb_uecvRWrcIm!)_f%*{w+fnd{AaY^hvwq#LA+X8TXI>Kv=$-kL!IyO zbM0ux`*d!9j7X@8?}qqQWkh|)^VhGpUKpFOnPt^nn?j-z{q$1U2D-(p&|MLVTBRdn z<k|!I;pf|n$9s8KGA9Qoq3F2`C3XGX3{`NY#vBSR;1aQ7(k>6y%&XmePf%gsrJpTq zmAG<xGgtA^TPZPZZ+1QZDlnaT@7%gaZN*nzWX_?*vpjz|nFGSw_r`w%5z3)YP&$;U zkmq+k*BPlEP@uiLvuFx2D`<Jkua#LgDur9+vj=~`^GN64bnv!bQc|hBNgLi8N(O(2 z*&ITVjRb5SqsnfW=C_bR&Hh$y_bT}=(QJrq)~C_H4_gkR`tcE3JUUAs=6B@M=PM~> zTqU~3gELo5P79xD1}lii8O#-{eqUKAxcYwgG~)FYW(c7ZpBj>+UkQuK$+*=GqbG@r zPxcpYz*yu{Dl!s{^?$|Udd;vo{X7pB^d6$F=R<Em=F&Uuv^o?;U%cC8U(0@%gKD?t zM+QW&M<hp(yIa%kD7di@$=Aqf$_>h;o``!Y+{vrB@dd6rud?+sBben`mRg7SlRb(9 z4VZT1Z*Worr;it+&(~pfFGknqfgG@1F(82X`1)zcAv169LKYPzvo%8uz8$f6G|1(F zVeSY13Ob;Su>LrQD;hGQyy>DSVr_eJ^ceSZUqf;*VCXI?6uQmd5rBNMwyM7gy!eq* z_3`(YZd(kDF(N<^ni(Z{q?spiZ&!{fD$Cr@Cz%}wlZC2$>uSINasZB6tMdi03ACpp zU49<(dBtKAW|8#e;U&<=F)P4)pVMqgqt?afBLoWM*5$%BglrW0mE#3CkxP9)e#Ehr zWf23mNAMR|y~Ly4D=}|fpNJ~F@hWL1BLvxv4h&f2jUDTb+L*f3EH<gExqB7JGM_Ox z%6(^Jlzk<rA0g3A$lv@6vjL+akv~QX6uAB;#t&*{-q|LuRzQs#=qhdbX6gJ9#xYc6 zy;th!rKA~aDy#AN(eh$a;3>_$i&anlh`8GV@VIkhN66B5w&s0z=9?uP0^f6iddHhF zV?|qw&O|2J)^E}%EsHlKfga`lioy+=(sCu{PC_tlr#f*Aknes(nfbWcDIKar>pVj; zOl!yhgu|yLK)GOvN+o{vWhsd({{9tgdE}?hARV3%h+}^G&_g-IJTWy%{EKlyV7-8E z>SbG2WUH+4FK`dA<JV&=_BrN-6y71+X{7=LZ=~U;=OeED(e<F0L%!rUYN#Fv2}OI> z2Rwf~ThIq6UL?Y*`oRmhk+ym=pN%+`I<z3W>z!|4vE(`=B?}mEr60>>_Wqvsh3?%e zu@iwF7Y(SiFCN;RPaq@&WR{!U+c{8&q+@E$v>>W~jde-!K^wg2+JAkHUBF5-;cO#z zqq^elAki)RhF#`ma3%~d6iVQmk1R~$yBg<{=V*R1a*HHj`d-%1EaJ2@pX9Jzo5MT3 zme_OLI7BOl>iSz}^;vnvZH_ARo8~%EPV#RWUMpxBl-;u<;nB35(?A(s^*jjWV%p-g zd}o3jA#@WcVCzm1v;BI8U_&V(IZ)yS0ffXVZ`Pa-h7kwTo~#nSSUF@6sATuf4KJ%W z`*ZX14hq^Vcc^Out;pR)sHElykm-c#L#n|E$v*2GJ~ZTa;U>-J-@Y|=PKN}SYz;lG z(8i83Lg_@;6B<vKC!TJa^$AEB@lc7S3+}U!ZY;O@X^oX4jYQ8&%ODcm(7E){{5s9i zwWZZ&wAwiCHl^z>R-btxiA;SD_F8rh70BO-m02s=lW2gAJxwMC0*dD}wCUuOw@Sy_ zJ6FkVtwI>%YSf?``sYka>pu2Ze?(aDoLGNMsA({q+6T|^Hc5|H&>lP3&Ro+P{psMU zzEA9*%L4ButV}FXs-HV85;xXt@YEhQyw35jo#{8_?F#0f99V`;SJkDR>nv-+Qz12` z!QI(jUmXk6^d#Wv2AcXu--J*Q;I87<<xP?0hd)DRN*q&?#VUx&<#E{J3zmi{ZvCbB zs+p2-%{=jHXUMIo0pwm&UB<9-a?74)vUmsuNCH&*blN-tFK6y`rS{)?tAR_}FVR0u z><=DoxMrgShPqOxdAaNI=$`u_(DN(T0y_R2f17SBORu~KBPDY+XY3m%Z!}i=)=GaH z-4FRG#gHnoOr$-qG|@i^MHG$$!zccMjeGYAf7o93QZC9fs+R1;y&K;@TbbwsLngdo z=+(cg{$}H@TU9n+H5R?>CTo0+PFA9CW`Uo^cA_(?-->(gi5PxpKAvsRwV4i{f2EX| z(i41F=FI!xnF^Sg>0B2eR7Rs2HA_>QjCSw()Z0%kI5Y}Gl*hJs3ss0Y+dgT*W-Sg@ z;`85%=MR;(oxvtbe;a@==JsJ*)hkH^W8Po+%qFtjcbd?vR7NMGx5H-jVcwdoW22D8 zp({eqDW<!y;1l%_283#^Y{8wyesZ)9k2i{-8{mC<F3uqBXZ#0vp}on~(CO9LO-8oT z#xkM9t+XHs7d21sMtlano!yHJjir*wyE0jaIg-4VTrD@ObG+G|q2kTms0Rn<#x<k| zCn_-p(x&-nTy^kQi1jsDRU|QVOf!TJtUr4jn_hX2`c52}b98%0s8<7f-H^I<!UVf5 zZ$=NFiUDZbT-F3$8oMMXt1dmaZyb2so8-XP>a5pFZJ`~Hhi~z(Q0-mDYJ1kc@lmpu zx2OW2T#+1K`*Pw;mEIY?+#p5w#sU>7h5YPt%oraf-ZK!vAT3A(^o~GLBG@ikLOgNb z5*|YC33O&wXGv<+k2+XG1SfMO=Q>>x94bV~Kc3`N(F9o!O1IFZb$48f6S+kJo0m1| z@uUY?Ii1GMP{h6jZ5}^hq{&|Z2Y^tJaB@U)6A1+^3paxWaU1b9UIPtRzj!F|t^8{H zRhiru3n4FM^Qrm>`F>1QoC(4;EbT+5<Or@}Q;;Rz-iH*m0Aklz+wkOdkd^cf3=|1{ zDj((F$krXX6!uN=$twTGlK}E!GTJ>Rd0uvE>l(V9a8g>koK_3QdcsF*Ve<;F*qfQl zNwk^w0grcupqz+)0cggexLOTMtSn{6K<c{Ho#x%Dj3w#g94m1uSnGf;=^S7cFm(>Y zKZXhw{7AzuDmaTF8yzS=FIWya3P2@2t;fDlh-Z8C5`BW{_`{tnl{%Q2UG^u?@{6kg z|1VY#Obl(!;7l@2`YrzZu#s}6`A2kT;;xa-F}jwa*9r&Y&!8`@Tsl}{3~v4eY`4}& zyoGPm<LQF*f?9;W{CrhToZPtis9)>DIrjHlX*b5RG-<)FB(i`{fLs9epa}?f<XW~c zg?^C!QymNWIC1|UYUj&A-5aHX6O6W$pyVt1lp{s;bsRbu`fI_?ysrWM-cTmQn$zev z&4cK-vXW#;$giR#oz#Q^^1-%oqVqjO!J!QG_^!$fd|GYYY~5=LSD6|IYImf!ngQ%y z4QvCJ)vFAn7NJ%kVys}eTZv2{OHj;HH<q>>>OL@E(j+lw`zejPNauhgZg^AV!>o5O zL_zV<Wlsb450V`B(s=I*+QKwq1US2~83!OFFS34&9R8%kcoe)Rv`=`AV>aQu%@?FJ zZ)H5K5MM8C8Qii_AfL;sCJ08-f3QUV%vlj9LX-R+5jao>k`Gjh17wz-D40UGHQ%vm z(r~)=-6Zkvk~!A*r`#?rxp%Xn*NEW?Z`o!aN4|!C(j>`2=3PUda>t*2l4o;xG9Uz7 zmROfPvV8F?XhdL}UAold?xE7;iMCj?XA*|zeL3GKThP!ll-jDa07`)JIR4y6ge;)` zsA;agV3oi=*1o*l*z1dl^JLuU_y+n%uf;0C57QgG!bknP3qafO^o|p_?&F8)J6O9F zA4kph>^&Jt;(PB07t3`Mkk~V7yhAc)JlD4Lh#ELRLN@|DeYsET#nNBTi@j)W24AJF zBYqX#_UBdQm`Ooud1bHEubd{)6Ow7Dcn~{yO031S_d0bSgmHyu2QOu6{L}Qzlk_S+ zPUNpiv@88t!sXunA2)Pfh`^-QwnDx6uA;)MtD@gJMMrW=t7{~EkEij8uag1WTso42 z6+!usm1r6t?4T)TS1H}(E_gRapeqd7lK$h^{Gr&MPrG`ZA6_|)Jzcf?@`6iBwTrj2 zOrY?$aMy%i*TkLBi_aYjaSP5fw+vzqsyi1(e*?RugVx9b=N;Qtj?bG{>_rs7(=<h2 zt03PBDU7O=y1!>50`v~K=X7?eye17Wwv1;+7j(P2C^IUGFG}h5dy71$+1-Pbk_u*z zty~~%#y-M!BaLdQ&#U`2vMS9wSKTdKH+Nl|4#$;~8i2%wBf!fP==*b@@8uLS+FpHX ztM@=V(NdK(lOdf?2A_}7EFf`^Sm#%MFF&QYs|fD^{Zh5B8g_!#+EtBz$&V4A=OV<g z7!%`i<g_<Fy{l1tpFUm9>3Q>6Egx>z=O-Cg#4P$+qjr43vB`=mZggy+(=)nAF=neN zt4~HWWiDEI<5Tx#qAau&Nz;WY(;mUn;#M6l&k%I<;!w>uN$Lxp+}G`QVEvHHAkP?6 z&dR~Fd)W88Q}(tm=M{P~>&H%$>u*iRg?8PXmc!{U4F1CYJ^r+(!{jean)K19sZ_t# z&p3FigWUNsAQyFC>Wqsi4{zaX)LG=5a{3&npDzr5JvTM`&gB|=(9MuqI<kG0a78D5 zry{6VF*i%)ZRfpm7s>OsK9e!#i;64RM?K5qJ9h4T?ZFK#RNL4D8PD90wu_b2t((Iu zY`6ozuk&(~!J_x^MI^+T&kO<S2-vDx$%G7^DUmuK>`Bs<AW(U#_);CE=K;}XI{x=+ ztJlIh#t~v;L7<owfg~x^v`9h8?<(Q^+BEiHBW<aIQ`5n_NZuPDIRpWhxt;O#JP<L& zPb+x~Z`9&ef*-`D?5ZjyetA<lx$_P}Wp<G^A@&tqwG*;7j4YuNs}%ANx?CTO<CdY! zH5}p7=TEfk4_1qjf~?6T#%=s%i$IC@BuV?BgPyOXnZ|!Svx;jN7i~^DNJyKHMfm7* zsGK&gb6i^f{New3jHz-`@-Uc^L(|Avl7>P|RUeX2!Y#BHQ!}=k=1F%~;`{5&x#<}L z1zjOYsGtw(PH)SZS@^qQ{Y&}pE-hklgjBYnhbPA%@_*%|*Nz`4<H&6zGC2l64wAL} z#`}&#8ur+A)CNG6?2=q}aO8$fnz|uQ^&MOLktIPPX&6~*ozrrAHz-sV(<1yQv^b9o zFr3T~K+J+4ika2xBmoV7NV49}-6R(q$4wbPnODS=Zx50~=)>Mc3YP}LivtHG@q`kq zjhd?XD`b%qL{Nf=Cz}+9OK|Ok0NlSgs*#7gZ0w&DBVP});xP<rfpkYbJ2vQ9KxWJ^ zV>DMTt^mQ`F+MyMxOoHiTigLTq#RVPQ{GTPQtlO^1E+{go=<Xewg*1}6q@WDn&eO_ zgGY``K)QBl|Lwt;*aBHbVB<l4!ZhRaZUBU@DA#EU<gqMuKi@G&y#B0wN5%=I3iL?Y zFFu}sa!N(YP(+raBPOb3>?)x7M$m@YoF$ir>i(w<9vBdI@R;S<vi}#5-c_RT`cz6( z@&+==s~+DyYK-N_@^$jh+@!HRs<T9iJ;Df)b$nDPSE&=81s3Pv+iiisEgyI!)S8o| zVaV7k$;HF+oe#w3ng}AXmVx{h6hRY#$7$5ItTz%2m5uQFESGa0L#;5_lOG?05_;CJ zzVyQ(wUr<MDip+~Nn!wajLe4}jqLnnNZ5S&nG(tMsk^uUO6wcjwt=PZh8M^to#&(2 zrJ38yIpTjaN`0L_&kTDDAO+;qD;s*bpFM)4kY#=y(H?Y{9xz*ymCI#WWiB%pLNReI z0bep6jAkxtOoaHgX@?jq<Us{QKXoF-TsR4XN;E7pB?lUiBZup*zi0-?#-Ojf+{9R3 z8pP0@zTDVGBV&F00yxJ=<HW+*4u*E1-m4qVh^Lm1E?<w!*FS{L1bn$%c34qsLZ?&? znu?%cESB}B7QYVNilzI*3&IX5@YZuKmz+esgI<CNw|xzde?g-_8Jpysbvpc_%=six z1m?x56k-^Si7kcYeEKA?hw34+7t;ER2pPFSGEB{j+Q7ls<_Lc8fa#q!L1mfKN*u7m z=lQann%~jFx&KYB#BG~|ZCWeGn2_==TLb-b;You9&K(j?`T<%^#;&Y!m2Q`gqdWjy z_CSy+hKe)&OiH>Gy{TXc8f)M_6hbd|mYVar@jHmPryTE!we>HvhyPgu%ExH8IJF^J zS|)L~G3V$(wevDB^Ize9uyi;j_gulZJu}5maFW_Eibr(hbf4h5hb^zxFc=Sr7fx(h z&kE17eO{`!P(VwNnZ1-*diruc|N7_6&=q+d?~+Jq$Mr9%*I7a4>WLOk?oZ`Ld%d7l z{+oQTsb`U%Nn>qwP;V#4qT`yT^Q1e)I?k1)KYH`TFxRf#*$;I<le`)$C?=}cz!+>E zzUp1KLI`j^I4Qla$5N<sY#?^W@izyyOwpJD9CCmAgk!7qGfi+FV?QLl6>Z~b#Ucql z@)ybJl{+|{<@t5Y$R&Vp8?wo1N_GS@^y4fQgey_BU5hvMluf{*zQZJ2xsI;v3Cr3t zVUs$$7b|nnPtJDs``3Qe8;jYK)}1&8hn1I+`_-Sm1XO($|BAECRh6VZJvX6t7?2{+ znp-uiN=VnNTe@^TbNkma+pM(=XZv|2XR5J(lEpvPbN$o1>3J<K^+bY&df)1L{&n&+ ztO?iQ5wQ&J4=j<A>A$F~1vj}%^-XRno#IA$sNfn=L|rq^NM5*rc?8d_eI5HyzP8GU zFp5oR!{>h3q`uSpjn-2d5jnO2jhWRHaFy2-k=hs00viMHvFjwS(b5~WoVJkRna^c) zfzkVY4Z73r<6Ie6r~*MpC#h2v`<bd`HK5n=jMfU)-)8rM$+8#l7V*0bRr|lCV7Kks zZ`AkzQeFQI)oMjm!RmK2IPfK#R-7(jQwCG&68SAS9FgvO?s6V4*6B@J*x(XFU{r5g z$^VIU6u~u>ngOqPx7E(`;C&Cjt1Umk{#|K!@w)pPXx!&~(~g2qS8pt%qNxAS^lfy0 zxL*Cc3esm|r3Cqb3m(VdFpB+ZkBcbyZlIX;w2$YZm~r3lIo;gWtcw%2q-yzP(e80% z@!V0)8C8{GjE*1c2hADVCH>ltOWnmIOmdFDQv@++C=-wlGqN9s+KTIC3T~Hc%5K!| z8-E6$b`M$cg?+Gwv>jZXEnUZ{8B|m;#-`{uSe8%zxK^*MoP&kt%b5CkRz)S|f5WNm z;MQwq^_%~((c2)z9n`*)4zZN-+!GqP8o~oNUL7E<Lt|qpc@s`4_`q52RLHPr>vO9U zE9_1)Gt~}7H$-&Bq%(B9s_o_JYnAkHU_dTRE25tye{8{HA1u8h(1nwv6X{o*u{#tN zOU`yLk|N~i$MGgz<0FsQ+W}Jr;qfywt>aevr;EHw%|)>!gby)N#@H-5d`Tb`*L&Ss ziL0slubjSCS=)0TiY**CG*aFZ`??}Or|_WW?M3!W<X3!nrk#%Zpw(3?-)c!Y$138- z$aVfaW0>~e{hjapx<a%1XaV_GKe$jL=yX~}-*z)Bg6YL`efE<FH>NLzkdU$&lztLq z``>lBS9Hce1s{31tf=gQ?5u3QEa;(jV;MrmB#8G5yNH?<FASG72Kb8w$^4vIjz#<x zmE4ntsIluf^GDp6`1tsk*I4f(m?MA6egOF5=>nX|K_qn$meW15xyj`8WYg&(5%K#_ z53k3>hYB8Yq;gntH7`XYum~L8&X_kV=*UW#@7G8glC_N&VNVh(u$+{YAB^lO2yzn9 zFDu(Wq2<jfvk?sl0}=0wVM8gQr0jg%9v$k;;{Zx#uRY#o4|bkI9Lg*IVf=(-CnQ<e z^dX-tAPg_k+yh9<@@Ls!grybv6gwsXF9c1$hl8OJ2zrC}JQuS~*(`>Eb9Y&*Fy)wq zfJ7;V0}rS?3^XUZ34<+dgds59Lwpp!DdMqR&OcQ>kh5mZWX1vHfGkG16@~kiJI|cA z@pm>jgS_R1$#w#wWJ9_dD6F6VVs$5-2p3<CP)dx1_8liL2fTg*OvyCSK&JYx;~C#< zuY~uTt%rMMQjN&LpKBLI$ap)yKF93tZGkem$GW~WU5{u!!Nqz=<TyR^de7`6Ybz8D z(7Tp#$FK-(Q`3|yk$)aJZ4&^!e5i1^f5l>o;J*)#=(C|Gbk!KeC#S6>gN%!DX6x(< z!B|F_)6arc9Iq0kp@fPer<83;(&B|Ah=^Cuv?hYLDTAIR?((|R=>Tim4xJW$;?jqA zEwfP$)ow~1L~AkEvSi0lK~r+@0inJaIc$usjBPlJFSRO@0^(lEf0lq)>h@z5jvh7D zLqGP1&4>J9fyI3Nxq(!t8%Schy6-{Oa|~FS-9QBzY4neJ%r=Q4s4YV}$>v4(DO`yQ zlOBbd03`F06&cH(epVZ`eqJMR)R1jKG3RZW;j60YF6wobPtEdmGU((F*LhBJR!d{d zH<J9p)ri-YJchPdp_g6=)kOBAT?HSY{2peIix>o}dunRSohte+huRBxn#DW9yN&zc zU?M%{aBC1X13l|89f+{D11ClNX6~O<=XsYP{JZUlBiX`<i<w%vHT;ACDrzAos@C5o z(>rkH?aVmj7YEF%wvP8WDg9k(!u4YzR^p?}fPWHT2xBJWf<fH*(XCK(*muge#I9i) zHoj+}dD)?28UO<*W3$DZwd))GZ{7YZ!mQl_xbLl6x7nlz*XF5m1kmr(3^(4Vfk@n> zTtUtxOj$XbFD)Z3QbZyhgzmy#3{``or?sT5|8SzU`F8}q4yrb41a2xNTI6xar1B5# z)$~Cr(VQ(4(hfq1q3nU1ep*o1oZmj~K>h3wr&X$mfPPuYR?i&QaZ#()n@rSM8`<lI zgpP^W%xB&2TN>!Z>=mt}%@t*9Tn`WGD8dLWEgmZ7?j0gGOjr8K=F|2`Pgxy!#3L?v zpBGM;oGETH{COxkSSC!s7vx54WKm>8+Knjp%O;wr>^`N93Ae_2le~p?*g7_<@&e7K zzY~><KxU*b>LfJ&s6BZXw;n!Icy4=fhJqFUqSoUcf9I+rQ<@%QRi(c$O`G}x@@c2K zqP8>#E_*$TD%Kt|*uT3y>ZAJ|W6vPcVu#yAR9kLZTD9AIn$T45@*?Acslbma$s<1H z^^lHq@hJb3=`8m-${!n?W5^_gb;FJI2Blv0vTO>gRL!n{PNnZ0>gMQ{$#ZR;BTv$O zlqwOB9E933$b03k4<5?{S7=*txotx2SIRo1-=nkm%&s*d_xrH!5Y_9ONeVPcDR(|k zS#oq5Pw?7l_T2(K1JHK?jLV6J&kUiJlM1Sy|1Hq2UAPD+Okww!*k5cmx$R_+8a!n} zYZo%r7!_BE>~p!LE2qCw#nH$Y)m7HYJ?{gW*<JB5N~707uOSk8TH8EjD#WFzG=_($ z$*9~2jkwo$M&C_ssq=A#IrVfd?F>#cxfdqBj>>HE8}5BP6*lO#g{x~8lh*TusKlJY z+*0|A2~N5$#9h$~;FV9Hc<pg;r<XP|{K|W(InRE!ERHdO@?J}RgU)%>;P>US@TYZ; zybhO%;D{GVG6^?;W8f2?bBE+{B$G2KpV4Wkp!LY+Uz9djvgVhP`fhL1><!GSTr)6= z$LVf>!Uz(hA8=QR&eXdL4dgNlcJB)ov+3U%=2oa%hXXwtFArokFRpALiNU_oB}oO2 zu;M%Hs!G|Es%&waZhm3Z`t{*>idFmW^_TV&SWmB66)ZW0M@RZyS9H>a40__McY>_^ zu9;+rOS#L3PBbWrQ7ucCCEu(_ap;@&M%muU*8wTW{d&PbnCIn^?~8kvrm85>OUvh> zlh&iPg~k1LS1XwL_d!Uhe!yVxnfH4lf8S1>veD|$@PQl0Q?9>x{zNyH4Bh_wc^~b^ zYr3|(?(?sfvG3R8Y-IL%$F9gp7iHetmPqrdwWdWf*&Iz&j>rWGyN4=0Wc)=te_cKQ zZ0xag5FPrj<0CGRpPqbJjO>Jp1nflCGV7Gq97TB6z)ccs?8g}L(Tm*JZFp-~u2@-^ zdyg%M@Uez48Zh>HuJ69MmPjiSDdG*tdFT>j#gJvU_LrG}I0hL|T3rCPhic}_7?zt} zPmzx2%-cNgDptvgro)q!3YfzIM$C=Ok7+pXZRnGA0E)@w$;!ze4odKnn289zS%oDO z6>Sw8Ek4K7g_2JQcQR;%z&HUu@x;L{lGb#a91&!e1kh|c&gOoxyII#@1PN5alO<|Z zIfCp?qX2;oab`Y860N7h%yY+FkqI{}sw@L1B1MrM+Cpz_BgUgf!rA-HKwN;RD}QCS z=ABnp)N8BDz_n1<^<YIMPLP#>B?W2o=fB^((e(^L;ncP7DpQXMuO1An4HTfFc|agO zv;#N*1<a?5KjtP{`wjd7;N9hK=5)%knLfm#xQkd;&*1UAXZUtU`Ehi@KoMvO@?suf z@nccVx{9aqUz6G95zq|4Jtil>*^&DuG+-M}=kb>WG_U*tN(+f$3ECsO09u5K<6Zx5 z0Q_zwtb}`IxIDRVf8vUXp;4p5DS)-69#8Tim+8y+pbZifptRy*Wnm0uvljFe*rCR} zbooVnY5oN1_cH_|C_thxML-aC(ne-j-$Hg}%U`n;0(yL{X_5?sN?Z?1t)xJf0HODl zIaUB?64PGNN8vqJfC1!JUt;_}h7Gtu&!j7OU8tC?*!71xK+H~8I_7ze8o2^mfSJdU zq#;iScc5IZL$r@$5{{Sn#1aenkuP6fozO^gFlM6hs5dcjNBhd5lFb}JF>D8lbkIl5 zHjKZPUs>iec+p40Xwn?!nhw2u$h<)un=ib3@vZv#Hnp#O5I^tsyUU-tQJ)=ra^KRI zw_*lKC0f;t<d_$Zg?4XHZHw%`a;RHOV4wsk->8m|(Lq@(pIWgeC8INO=?4~iXb)zW zetO1sPw@9g@c%OE-Nk&&;HvV(GMM0u8i+-Z5c)I70zZL|3ZRZm@>P-k#$+5!e>EH2 zQvC(>5^Z<}`uo#YA)f<`O)$~ZFe(|b&jtbPuPqBd0<m2&s?R2_ifbjBcW!dROU6bZ zKTWLTllDm!&9)yx@IFE=qYv@hJt2W#nE<0Tc?5@-#WJfGK>s?w4PHkvWRUy&DfNie zfm{GU9dFyXw_%n{D?p1+4sSDxXI3I0f$4fb;HRJ3XpzY_hIrJx#Zze12nWj2jNqOb zXAoGEldGOz?mR-=vMmwxMY6v=0sxPA4PO<y89<VJe<%vHSgLW4&*~o6+6J6OIB4E5 z=^MlFLBzATbuD4uLJ|%2Dk-c%HSQX}0=90FPz|3_j<eK{%1Tz(^S{YDb?J3J90l~| z7XGB8(&;gLrHC)`V*WLQ8qs9Pu+U6}icH4v%v*(OzVXy%PLXPa{!{QpGmc*VKBU*x z=|`Sl@t&xLUF&N$`w@ewUx69?CZ-pEwBN-M&P}eky$G~G$X|>?;^Okagt2%D7Wj!f zT~=NhPE{8fAK*WDYMUT#bT4>FtL$k`xpbC*{#tCT_A2KC887^a{+P8QZxo$xf0@59 zY3Qd)PArfpUb&Cr)+w2fWju3j+83Foa(na`mVb}i<QXj%I%=Yha}ZXe14+nkgl`gt z)BB^==ZsnqLR~HAm}UpXc^jNsyp49seOl>Cfj0@$ePI6EBKpJH#l}BiVvGHUbBOOc zG)*FB>&h8AE^l^Nvt5@imv8T+Ny$46Tt}cZ!`iK366e&b1#x&ku70vOuCz}DV?DcI z0=lYUMt7AAYBOC91TRy{0cm~X_oR=P^q2x`lJvn7@T58WiS<(ZmFAq`8QT&LUKD_~ z;DwIneRRU)8h_F?>w0FTdwTh26Y#Q0P%t;1$vi<rTu_6R+i2L;h3$76JyiQRGL7<u z$=ctbCN$S1Q&+q6Vzc@TyhUeoQ1yN}XzrdWYP@y+);lVmhC*ClFjDf%JFQQ@%LO*d zuPt9@ig<^nvJV}aNWMU%4r>MFbX*=xbsY0w-M*ILH@$f+<wyLkI+5RYCR^)X!1Uy< zWivd(2r)q$i0*CjCVrDAW%dGiF&QtmavHd5mhO+Jx4H^4bAd($G<z-*UC;=sE#q_r z1fJ>)c#VApUeS+Zk2wb<a{X;r=U0lm%)B9HID}ZGVnr4-V08bKMt=NLPIqCrRAWC; zSAH3cibNeYh2yo^|B-lcX&FCn`5bSi?M!$almfo~)yf21X^`=Y+ehfyT)@xUjJmwS z=YlPd4PeT2El1-$Y-Xlt*9$F5T838YHOp8+LrOMEGQM#`50#Op!LV#5)3va?ICIJ( zNLA8*SKJ(i><6k)s91p5@N`v}|4CZn5vdp)jYmBROAql0R|#9%8g=o~d@x?4oCEW6 z3W3RW5BbOKBQfkvx(nuRW+Y3tERMqLI1!X2LfcTD&6=zP56U(m&*OSeOoNRNs>hhX zm5Y_dh=KYrM^~gZvnaCI?|$_cX7)T`*zy8TpzRAM@C!kZcp5~Qz$v<3<+-%gU9wxU z?~#ck%K72wM<6R@Ar@LzUcTb&7Hg8K;H{XWY^Ez?3B5H-2?Z+RO~mI2kX1AW^vl@` ztgrs$&mvgR-W?PO4l55!p$H~NxgZO%?qcbi#}mWXAp9$JTMy`S#g`?l8{eFJ<bR3~ zZ7(=$!pp()X>Gkk<Z}in!B6WHW~;cCScOFa*QawZAnqE`P|~A8KD;0Y9exO~L2Q!+ zD3m)60w~?Jn>YgCP+{ppCe4U4r)?g0Qp+#C6ko>iToWN#V;~1WYN)G{*(<)9`6p3v z0^2+Ye3qvciH~IICk)n!Fvk9<;to(~GY62g4e~=RR$ULuSeV9b2CZ+1Ewu79LFP3_ zLJ@9r-gs8qbXb1hDne?R^M~+3pFyP~b_a}|ab$?t%e--rRs-E3cB~r`2lAF!(c7@2 zq!JURi@On$`-$jPUC0efEz3G)<m;C3ZT%2-&it6Ip97Dqh!0c(+PuN-0;JwZB2sBX zER4pu<H72s>CnJ1tMCm3&ErRi2x0{GHTJDGr{iR^e*hm$K<flE42swoxNjf>)9=fE z*reE|^pr?R8v_8DC*Nxsb`0v_{l;^K0>e12@j^pHO*%H|x1UP89g}{lYik?vK@yp{ z2DsGOBqkSUE#~@Ho5(#RudfrE(j|s6dJb{IsGr$<r_}ZKIYxMxC$bf#ks0#a{!U2q z8vJ|kh`nscbwun`z#Uj`+HIw|e83P3<r7PFkv^N)`q;9cKTo_bc|-PPeAm6$Gyo9n z)FnsHI>!o4R?bo$p*WD0w34ZFcno_A3U0$O7o&1dLF4vIxKn=s?2q)EM7+a=KNAC( zAw}m6M#Nge)BJ{hZz`M?IecxHpjsc1qD6XvRYN|E9qHRDlC#$>`ztgE`KGtB%UIb! z?d}Ex+>hHnO1CM;Y>>g%j3%V2a_DSd4a6h`3?#f;O8?oe^hC&DnGKE*D{y7_l^|i9 z2JnV*iRU;F^bcjmIBWy&Ih<l}Ta7pwQ`73DJbN}s4d$tXUGx1nrLSWe#+mB$+GwI( z>qjlg;G5f`0ezj!bqFT$=|AQE*l^35nknO8+_;2*{tL#(Tyq{!U_l2%&XH?XjqAJ& zGN7*4SBX04>3+K);J>^)#1yx|%{r})UD30d_cbe9oYks@p?-x2!hf7yWv3qMS*N24 zY(F2C(wevzM(JnOrihr%RTzOQ)%WChGTUn8g+5IcJrqiI7!&4IEcH+k@xR-ziJNKP z-G`faOz>(NOYQ|iz{ctfmGiUc=WjnterPHl)jvbs-?lN4SC{k9<acZ^Bpf8irLo7! z(+OXBZ>$XpnVGy>IHwV|JU2AURUprP>0ijoIaanSU|H=r#?tLGxD)ra_H7@}pNoK$ zw_Sn=kzfPrQ0c7;-zuH*wx|R~Ft^Z7m1f+LS}IFR{_n45zx8XoW=Gr3?5CQ*_M%SU zQeNp<@bh;Anzn;+RvD}x<r1ic<ZzSdXG(bu^uR&<k+Vl|!UU|JZD76%-yLbSJV@@d z(d->G)5K>~QK|i6Q<52zDh^4N+u4!YS8&-)LT(Y;fTrI)N-pk8Knd5xk2}>@H)-R@ zTPsEq*UXo7%$b_GqUwJZyzZxpbps1scbVije(SEh%@_1+wEHbmn8FaMm*X}2gTujg z_jlUeM)xfW7cG?(lh0S?=4o_$Z<8A2xmE<BI|lNL*O@@xKU7=l#b<W%3k(iC@HF*@ zxG4E>rZ_k%fffwrH@W1*?ZD8r<}aQ#!S&tnF$s*0*NJhPNoTX+>uR#iau&Kfnfz@v z({&7}ecv*KenR5BJ|d=9YDaa!fQs$c3N(2kc{}GYp$)$8E=IsxVEfCZycdc+u{`gq zv%HQWUIsr=n7ASbTQP7^pOFh|NWj*>@cXLfSUty*hmh$FP>h7qW-L%$z^Z?mv%m=e zWG8vGctuwzEUsSdG4si^v-W)7+~DJK)Qfv+IzhOeMyBY83tAD~)Dl&x9Leg056!n; zGTx}LVX?hJS8&(XT;Gv~;^2cuD?atk+QVwW{F1smUk*F&ydBDhb&N~lO%s(ubNLv~ z@*7Q9T6Y~C#azbqsg2iizm!J)JRDRYtj#fb``>>ka7AXk!AdPxFB>H{Dw_|q>^mfo ze;)cX(meE=MLC6aCR-ub^oAFn{EhtS|6=beqoQoTepM7uq)}k#PQd}BVHAcgL68Pz z=#*}d&Y@FEx|D7hkQ!1-x?|{^q5I79#`BN&oKNS|S?jRo1B<opn``#9uU*&Pzu)6H z(wZlITV1lf(y}(#6xal_L9G7RZK)&JbcnJ(x-<)loF}U|Jb#`-Yw#@wPX*e5gaJhi z@6DgrbndT~qaALg{K7P}@{=c=H0eQ4CHCv-=W*g+>akD%sPwvGtN8SM;WVOVOyVH$ ziW$B+Xiw^T2nl>|youHE5KQk30qwyJ9}{?Cd=6EiJEMK9ki@w?H@8puRTP&5fW!%^ ze77SZPpo-@^mz)6`YU5g52(Ojd>C*WMFToma*}Tg{QzbRm8o|WK<9Jm!4J9&BgNjE zI~^81BLNWVpc^GFyX?@O8L4o6F#5C02mPzfLe$8X%h3~JXgx3YqqP@fq;&pGmBco{ zkD+b?ES>(ufQ5P7bWOy|$4sclk9n6Mp6ANE@KBnHC%~Sz1(7>~Ap+(K81cv!HyDHv z^66;1u`O(^2$bHVX61`Jtf0t|{>~b5pE5OQcSVBE%NhqUR{dSJKoI6BA>fl$YCpJ` zaH9R6dtEWQu=(EUzH)>%Dx#V0t6=aimc`JFx1S?D$SHuUi@cj~1ofMzl|w0gqbuv8 zL)pqK--weh9(z_~HY|KcJWqD*<@O3T|0)*x5QSv~UZ!L$IC^Nm4+dPdr_j!X*;p&< zNBW5dro2n*61aR-p})afSAciEWj5q0Vu1v?e1>(>_&Ek17L$T_uq?RU%=BK<)AJb# ztftvFxlRPtM<n{J@_PNAr!N#S#JEz){Pf#-_u;AF(viSO>vZP)8ue~in^r`CD>g=# z(j@W5%f?7IIv+wGatas7?{#5+p`Z*h91)7f#kZg6Sw=QL>5delMRXOq)HggH@Ksik z9vkShnlU_2nGdr4?bN~NyvA}By+zdGNtNc{{WY}j%@c8alIWrUq!B*43-mpWdtgRd zd-QLFn^9^CQ!;}~EN>{=`J(JwiZ0sLu8)L`*|ePV3fYAAF^zq=n1f6kz{+3F53MML z{7z?|Lc0J$zhP;y&Y61szyl9L5mk_T_$6L)zw1Sq6;>Ht@4G=M)N@FC^hM%)r!%`i zQX_?XO8F1&LD1v(vnGpQ#Jx?RUnCe>yUwto=>zT;FXrNLiqcZnv}_${l`U{|V<Iqm z+VwUHXjVsOQky!y6}n>VG^dXA-V<7+uIaJxmZJlvwj(ST<LBr?tX=@<L3XUZFFDCE zG7n(jUa)A{q736u)SM%N@NqfiFWR|E@KV@~rsy16D*u^SyszJa)lt+m?CbnCzlaik zxm`9(er&FRoE84q2pjsi?WV6duh&#;3NqAS$b6L#Y#gvOy!?3Q+QVKuRYhNWs*3cu z;#u~F?tK;2&JVc9>w&H7SQun~d*)sBm56!hrZB_3{(}UyLYbn<`a~~q?RLRPaS*EO zQlwN!?cu#RCv1Ne$z5Qx05d$*ad{t^;)@Lp^$d`1S<RvmGmuU!pFm&4?GrcSf@UsK zc|=uZ6D|2By0-1M0w#8AP;1V!754FM5IovPwL9YqxjiaV?VUYkFk<^rhkT#1AM`TH z#p4#$tge|Y>_=Yjn3Qi4EGKN&4r7cpSIc16iLCDIQa+qX5^1&PQm?2t!+!}P@-=Vu zBYlWY(K*ki<IkCf4<W6hD@->uw_z3T*9J`eIudlzd5w2No#>uh<N>;hI$H&azTf&| z(|op*tHNKTAMM9(*gTU(o#DZejU0(x>rYv?5Cd>z0&2~7BXpL0?*r_TjT`0lNC84g z!bprIZiBm6HP8pWSo-{icGvosfOuBNBvHT{+bZtIJey{1jql~uOw=lQH26$AoV;(g znQf+Pq;@CmMrO(z0u+6&%Ugt1(H`+nOVHtUGs#Sb+tr@tZ0&u_8I5R;l!9i-`&3MJ zoz?^AY(hep<xqPMK~vH0$mI7NAf0`dpTqc6lf9aSCD$>PKA>`sHOh-sGA6~lKqI)) z4`u!4-Qxhmi)Mi-oB0w-DfXS^Y%6Ze#R~Uk`GnG(qETv19Xsk%8lHX--U+p!#CBG1 zY4C619ccaHXh}3^qF;l?{b^Rv=`7w*`H-Rd1pi`m-PF=q5d5&EF4>L0KjBiYTPa2s zU0ys{y=dWnJEUJBd-*I;GeL%0^NOW<JN}fy8<a9~fi)kE3sSy!-Y<LMBRXGl^TM_M z?|U~_99cydm$#_5{YS}<RM)Vbdf%dREI=16HbtmND6j1EUfzQTAq0Wr0p|X_PrWFM z84!Ya{!n4MGdiNbhIZoL5eN{dv`a9ol6F01BmomX=Ide(<dh(bE=D+>+x|k^l-_gn zPWzL0;H<{@E)CUsxJl$*XMExHLl^mDk7>>525GBl+tc^AKrVjj-3H3m%c4^+pZLE0 zR25Hqr0O~SplYNxZ1~dK-*;hck9GP{^GGJT5}3!^x~g@v+3H%?u`&Hn?WM%?tX0=g z4(!j^JfR<MsL@#tEhXJK?K7l^xc@~njReDy(03?>H!zCK?|awcH2Kn)>)wzStv{hC z0e=OHh#0{hr9~u&PHD!wy`>{6^3)|zTAV|#;mDASPa=|+`Kr&2Y9(5%f=Xm9H2%n1 zmh&~H?ETyJ^ajX~n!s96jfpKoQlm_c@W{|#DM($))O2bZfFLmazIb^5yF^iYfz?#m zsg1;s*FGDVLgD;$X-L1h7h&ZDF&Sc8ENk?3bZ>clHW*LoGh^0LK(+>B;`{XCj}Ocl z4=y>sn7yo&u>B;!;?ArEepsW>!oc}O)Om%b*K^}}o*1LRCB85EqHBwK6Y$cUW{J0r zHm$OHj_F{7y@$Y+;LKKl#%0;l3%p2|ncB=%xWdsT?A4>pvr*sN_sh=>LALkura|j` zQ7fD{yNLJo;rol{BFA<p>|0thRRJ3yzl&Mi)aAf^82*kE60OPULY)_3SS%#=KL$?A zzO+z)920;%h4Xp^?@8UCex}nR>rftiv&#O1Dzm*IaU|4NOC=2Brc)2stHp+v41ej5 zz@D<9|E{suph(nbQgVoC3j)JhQI{*TLd-G6Yt_XG=;E^1L<LuVn`>|`u5Fa!?5EX< zHz>J5_~^L=L*>>6t9#ZxFQ{?y%PA&K%@7S*?XvzDZpuvJUcD2A;GnRwh%q8VDCK+` zGO@i#s+!-0-K=Z~eexRAMm=|Kk|D?HX_|BI6*Wsw56KW37AwneJi=-<rSwwvtsrjw zMUCJS(7cGMy*3qK1wW2FME9g16dmySC1?>%F>eBqM?&t;Y2wQlL;x8@Qq-om61_}O z^UdZ9kZE!nANs@B<rjFKv|IKWVQz9!o1Q!Qcx6wJ3^@$mE8k*-q<m9(y@K@#=R-|g zR2^T1zLr;Xqza&EfPeo2eRms)k^RjDgC?V2PilT`HduHeS6?60E5ha5Q24xI%6eg_ z7V-P7*pG+PqFRi3BOsSBqEcI#R!=APTW0Q<YNW|g<9N@do-gRDOd&m=srpGzyruOz zxkq99Gu(1f1+jkRi4>gyzUonCXxU}i%bLZK^Vx&3>!19V%4_T2v^@Qq5~;H>g(TlJ zzkQKW9RBN~=b-Bkdi3B^5dZA^rm&<JKT8FkssG98sI4jVQlKiIH`tZu8xN7%F^BZQ z7wvyK*BFWHOX_iINh)Qr6-aC2l*rDe%O~{iLj1q>SS*@84NH34EuV0_v=&0lhOOLu z=6!H@V>CsdMK`9HF8Too$yZAw#k`hapr6ON4dGA<P7fy$OGG98JOt&CV0qVh*IQ27 zlx|mVTsA)=EOz7Dh(ee7^n)H`SDN%>emlDo%@FpUTyj32O*>?56sVJpjbpbjDLnD2 z<YCt>{-$QkU8M7>Y~LJwf^O?<&H+PrghYbnHLSKG{7#%NOKJf?m+<>7<ty(?lm?Zc zn^rRn`QB}6Wz*2B*fqKQV~&|TE26UI^E2pP%9>ta9nX4ysnd*u0b4melRVokv3BCp z?$rLC`FxXOmRg?THk2a{jhr?HAFeYb27vCWN3hAToN0^bzw;lCtHkCWV=vKyLg@Hf zn~hV)%#NZw1)C&t;F^m!>n%u5q^tVa)S{$xvFvs@;CRP)f7bKY`I~0h+nyAU*6eah z0oxnn7wkG8YAKv2c8^J~Z*Nz7RSlO{M;rE}sRS(QzSXyFfY62JopI05dprfZS&Qwe z_0~_GBcy=B;-2?`@Fl0dYO83UP7=^(z%Vs_v3A7A+3LfuzIz2W&8OBu(!yRR4Q?l| zM1&(qUX}I6LfEv!_jXVczV~9uUwcX?Ev7#nziioruH?}`F20coxnIrNf&<}}HrM8> zyLO%tq{G?w(!6HB&z5S*iHNlIJ?`oFl(WQCR6x3ezTht<>J~K~`@+82SB1}@<2~IE zaZX{ZoM$TwmuCMx!61tL<u+35tKgIEUsOXeUt@%#8W1`!$9iJ}=GXE3Fxl{c!LK{- z=^{Ib0d$G9OmxFkK<uX-QP{F32h1T(PSKG8*#0N3^rVbT(UlI(i3G2=h<oe{`59jr zQ-QC421*BFU12=Mi+v{aq@9d9jOHc!J;lkxw6^ojTmFQZBiNUijYD|2d%iap_P&ZY z-O7<qCE|fd-6#H$bwwB*5ofe)NG4<Qx`5ZgBmt~}l-;&|B3vDh!~-~`Z@#AFMpg9M zUMcxj788G6IOKWO=1bs3i%&<C;p-nH9Thal$VJ;m+k99!g1to{9^?eUdIkpBD-ZJU zTg%_P+ksFRWu*8&K!{+g7=@-Fyuv+yEH9~=2Yw1lH&K=(;gx17lSq-E{J<^ZJCBRi zPOD7&4N1{K`}l{a?@Oe77GdTVu``3Ugb&Ys1r9%>vPGI+yE`_{H1TMNV^Orb{BydG zbTK0!qs&$yKZ+oa?<{XPl4gnE{)8}&LN%}s$ztE{ieQ&Dzfq*AWOIyb!H6Ad>AX9z z|GL-IO*$PgoT4(FSgFR+qXu$s?`|3J^;;Ia>Pa`YZPNGZdMPdw`pXr;Y!o*<pDWXe z!pH(n^=6$Aiy-ax`lbm#&U1ZncyZe;-GyJ5@7?QREwi+6z12&6@qj3^B^pLXd|UXo z<O2;d5S<JnY`Y$NbpZz<=M57_)QRNSt67)+Bl2EN=MUpIw?c@owkgCJT#m?(LsANG zSN&T*iFkRG@;|)myhE6HZNX)!O~oLKjpvGwc9;Zs_^&n|hor51lwbh<K1aA4J(z25 z=#Ww7gG<|n2tSheZYUTfNS9n;3*p#Xrf!aCOW{QbY?OJ{0BwkmyPi}A*z`U5A%L*D z3JMK;5U|zJka`|U#+5aKd0Rkp5<6|wIzs)VxrqvT(eqQaz3x*ax&-ps54bEa``OD+ z{F9hTls)gmT|Apnfk{s5K}u&4$Tmi$^x%_&P`pUzsBGmj)s!p*Zaw@9*Bx!osOz2| zEL0mMVuf+L=4SnMIs9iuTQ!?ie)aLbUc}>70~{zq7cN1$f=`_oz2CcwAa$_)J|atp zRZ&JD5Si$hDx-I855Zhz5DF6o8*J#Iu!LST)6+J|EI|ZN-{-^>T4Hw_57wSiTx6XF z@(V>@n3Nf*SVv#TM)lm+^tI60RpZ3WE764EZwPLRWXF2RDI$e-3GKlmgSc0cR5feX z(2OkxN6M_l;(Pyh)<zihcUYulGwu$Lt!V8#b=pt)>e`w0G=WhSi^$%_$kdDH^9$6} zXuK{@TJ$wljUbXQd!E%>{azpT(WXiGB}G9}x3s0xslc1&8HZPg6^4g=I>oL#uO=y+ zlegA)jwfnmL8rs|Qdyej&YcYJ7n;f1P~sqT?xWdSE6({*Qz3kEH2KBcwk+C3eQYY` zPqzNCV4}N)b9mo8P&;mQ$x$j5m9w*zXi0v|>Kw=7O~=<TOvpxO&&$r}VmV<kYakP) z&$~CZ*fm{$@3_C<`nLJ}U<!%u0?_{&>Y3%$B*!!`u3CB!fNrpm(T_@ft)%UU_^<~i zSI^Zp+XuN#XU$`quNA@$KuK{jpSX1IVs&bE@?0{Sc5d-CuO9}COcyB7;F19+N%`CE z?_|7dOf%#)%Su^QSBvB`Ouq@a`rWs7#7*dO)M(=yf+Zf$*O;WaDv}J(AT>&K7<_x^ z9xYTQ?wL~-W_aRX+zNW!BnadCYT)YDez?zOc;h?D^HaM`o7f<&aLWS4&NzvN&UyY4 zK<Fq9%_w0-cb1Gnklvq%x$$K*Gh3e6TFI<!*fb}KO34R-!}DfO0Tv$wB^a9D&HQYf zpg_0KB|EL})L(s?VCh%=o<lFCP$rA)^p#=Q|8ZvwLwhpy$~Y7CMoTeVPx}@;ANwv} zN_L}V$0rJ<X7|~x%l-7oZL>wG`|pI76Lq<!zrE+`$sqs{pPZ6O>wFje5OdTI_slG_ zB9pYto$Im3hF_WOY2QbY@v6p~4kCA@z4k^N-+vux;CySIUPdhmzasqxy1j{jt7POe z%Kh0IZIGL&J%3Auq+z<w71x7(w5VDi^&7^&>_G`GRiTF)9S{v3JXQuU;mbFMTaPmg zw+ih)m|h)zHWt`hJ=ZLDAE36JYmJ|@nO(&xzl(y`dEZb$v7y)-@r6V0$BVF`qZ1{P z0vSRB3z`OlWjdu*FRa{cx>aXugkV~8YjY=A?0gQDvPK&{WTaZIx(yAhj}8pZ{)WG7 z(0vQA$8txvFliJhl4m7+ML01EP0hxb5eUR6#Q+I}Z-R=xrU#4qlLTGZuRh6Y6sCpH zO47y<U>0Ll5k7jNis}EDp1^@<g^1{}TE7@xlp|fxH15N`u1j09$Bg`4+>F$Z>A*OT z2@#2CZ-;6;rXVCDdX|kvAK`(X|6U_DttVQ3^sw(45c`sZm5)C2?dq6;8v?hB4Pu5R zxeS~RKPJZy6v$)gq6j3s-{dwX>T%@Ugm8p9hkm|#=RHlNYcKwo+(*)QWhwOC=TkNU zWpl7oCzs!$1I}YGA0vNW@!Sm2g>H!BeC})HlfYM9z)Fl2Qd#(CPx!m}Wy8?sYc@Uy z%0~|#ks=tul(%ntCBFwWbe5l_HLVa4^w7!)WgdnJxl)T8<Gp*&+ZNhvmgPOkqL(q! zi^nq*G!kU&EWu#p+vK$EwZv0MD^MJN*>|aI-gVz4+`(teRf+dbJ}b-T%AyVbf<hue z9}kUsbN+<G8d^?(dGzEl<87f>ua*(^5`zm7??Gk2=Z9k%K5wgtvbs3zWm+V9iH0L1 z?&yVu+VLf*{Ju`V<MdBoujc{3f-w0IJ}Sa*cBJ%v_lsr7+o1dS;aSdW=0MjCrY>Ne z>oIxrQCg-4E5S_-zP94QssP9vA!zgqMu9*2wGj*o{rpS_Ba7!yD62&-jk2eq?<x9* z&B09ouncNtahZ?$ya!HytwPrGR}u)MRcvk~UTj%+95OqISqqi)>JtoocoX2Zyte}O zhRf2V-&ati2h9PY*qsfb7Rj5%e_mkwg5TCtMCr9FE1%11?@7Z3*-)<F4PIwe7Wa8E zW%73kNCJ`22e#+5Ywx_i8`j8qjTlbi!lg5#&6Qb_D&E{Oxr7m}lKK*1A@o_8iC7R1 z!78h)EjMJ(&#@;f79S2=BBVFy>X;|ZNkts-_CG=PB$Yp<52|i}T<pc$m%3OlgHkrQ z9dXj^%Z3R(k<x}KAvJ#Gy=y<@1_d@ij**bC7bp@)KFtk<Cxq02u+^dUyCLP_9+=8D z3YFZjN(Sb<+NS<PB`R|Px{PT+Ph0r7oiT-s`rfR#v<Ww)A3V0K&y2%JX;!YloDQk0 zU=#nv?^mgO_qNMLD5*uhZ5|MuL6}*ITPe@UBfUXiIoL?4u(I)JHG(YP*1Eri&NlVK z$~rW3*#H~bg>QvD^-+}0zcMy@%kW^n@fO*LWI#FMc2lzStU1<Sr|(Kmia9V&CFpI& zLV8!Q<CL9DZ90?@Ze5_+e1p{lD_Nri!A(ana8l$hI+|dm*Sjl+c+DIK^o288+(g`7 zzyI_BvcLA(<`};f_i@O@7XjQ@NCn+)rYAm+2%k{W9U)DPlrqCc1-=js;mg?BpWWw= zx*j@)U&00byCv?1MPByR4zko&Q>GCr44S})YI&x{(fO*2vGQ`XZns{=klVQg!+NF6 zsnJT*td?ut$H{Lw22YxcTn=7-dk?9aJfALEhiYyGeEU;dwb-mPW1L|+b^AWPKWZeT zo8rRXsG^rlPjEvv-{sr6m-9a7T!(KB`?_pUU2Ct~;B}NtiO<J^V*W(i9Q{S0-9~u{ zkA(vo62r-dCg9q;6b-0ROO!WU*vi0id58nv(z!$yw!Z&Upf6Kv`NtqYCTe1XO0@iJ z$v{oC6#2p6!@yFIbQj0t3<K}0Qj@u;Z(@h~hF-?5(-roAPB#3dnl4V63?HYRUOh_W z)YrfTyqPGerSM`wadnh=2N!vr27jwR5;L5Ku*}w^+7%^{zP97m$-xP=S@J1tIu&89 zQKs0Rr>e+Bx@K28ox-Pr;OiV8OLeo}c|h8YxchFh+6I0u93e|)D#9@Fef#V@ugh)$ zy!HimS+ME{rR<)~f+ifS-#=jxx$I60xqK}OU08SnpI#8#*%DM1EdCIs96iKJ>%V~h zX==Sa7#Q6pBuRXCc&`h}I@LQ6GTWRv*PD-UwZk&Av#Aq5h$hQL&e3pU&deN>0-c8> z1O@X#GD;KM6WGNT*)*gN%guoRmG5rdvnr9&8g0eLpegCKCf0~ivPS#eYUwrHh|!e% zY2sYfE9Y(e0ielb$i;Eims^7~SUQiVpJh44-k&S+s9$`M@T{cr@pBo14~l1?o9$%K zI4o>T`fz{FwTAArr=(W)8y~Le@6__#i>?O-cbjeIW7(t<X?4G%IqpXAH?ZW5`0%KO z-mM>#x*W`eFd2GB&%S%5TCPvCx9Is6^j4*)HRH!lF%&+a`2|%&24zqe>+bRt&aofc z3lh#5b(=0<YIu?%?D}q??6$(X`*5}>A>Yl+tKvcN2a&brtdBc)v1{>#oM2g5tLNnW z5@{7z<tXJ0>BODdQ-cU1IkwRXoR{8E%Zaq_8oMsj=!V||SM8x%)%xs|^BJBuFJDQo z&0pfIAQGOtC}*;2=F-vU3T%5&gY2KtaNTGnPCQ!hj2tt~>^2v@yB{-8;5NII(xu)( zDU>0zB9asHUXc63$fvmF0tqCr=>l7xqR0Lau9}w%n#}Ek@r9`7N=2^$<MbO_j(9E} zKgRe<9&*M_6MscTLT!09W(gCA{)_u)1QYQTPZ$$%5)mB(aa3F~f-c^Uhny{51U0qE zbICx=7~Dq}59SG@k9XxsWrzo>X{KVw$$h#6#VMNizv0=2fIEA;GSlNd<b+eF<E!#o zQS66I+^GpZscjO;a#FihxW*l7{Yi2)4fJkZMKO0teG0qg?;u3FJFkcndEB{dV&z2L zxeKByt+Nx@h_A<nc`BLqW4#z*ijM#reu|&^OBIa<9vgZw|3YN;G3F?;_TPqc#L4$3 zctOnx9q;7;xzXL=*f3e37Q!bY)*=%-@u|qR0OTqBV*CxiLAN=H@z{HK!%3}T(?CPa zQ>jZm*^K=A{NK`sy{8U};@muS##QR@TE%%eM35J?F}?0AJ4Aw+$3#L8UW{NM_6wFX z0k_Gendn83bDwL9^BS4=8#g+iB-E%x5;n@j8Njm#Rb?{}$4v1}JkslqobmfScnDq& zh}4zLl>svy6T6JK6&#KFk~`1BrZW5Lo+fJd4rdGxsBCPcHA{aQtLe^7oHe<oAx@!Y zMm`+8&B7c<W;Mp*aGx*Qc(3Nlppv?rG#uf@HE8Pp2K6Hb$uNw2yjVEK%r>W$q$W;m zR7Mv29doRlXCohvNBfm?;x(<sy2cKzO;Vj4W2$0T#0(R0v$Rva9)%`J1?_3_aPqhY zij+a(>g^7<H?0j{Qb<B;a%&g|_q$m(%&15TU{pReYMww&*F>~;bLh2(ak^}?$$Y<i zqe12js%LbP@GH+Hr*u)I$fc<49C*x?9$1k-q}F<pEqZ9uVTg*`&GWkC3?rvqyKuLu zkcNk!M4$31Udh1&Mdu<xdoex<Koe2k$hzL?M6*&+vo!#R>B)TR)sJ3Sd`q5UY>iar zQ~0q)%r(O;OZ+uAgk5&gD3Tmr_hu{v#hK{y5~IH%MA}e6EN?$fxmDAcGhB#1IQ7&! zR(QaycgxNqxkcw3=EZD6Ij}i!!5NCE{_G_~cFR1~54=<)KaJ%>$zV)Mr+rFzLF!=e zoDzs>k}m6;)`k(ei*RSn0+B;dLZKXnmp;IpD@ZfGTBfsl`ndj;+930;SN<!wTDeg$ zc@~WvL%G9J*GOC?xo%d!eQ|&&e&jXn`TOTakil<l0(tjP^nj1u?(m;ddS?&)Tvd4R zO(xT@!6mHMjau%OQFvsUMOWMqemoocj5N<W^*jN|xI~BY%n?mPv??^O3o)K1Tyt-) z`}7|A(>E%&5+m*goTUxrbRgsgKtR3^s)v|oXuVZoY2f%>D+j_;H*?>C8$lh4I0vn{ zs$Z6&ANF-(SRc5EJtPGZLQQPfbs7s%o>0*&6UAN{hu&k2gP$pp@G7{Pj2y!;47I?$ zbiOfszDs+{zVsWFVapGYsv7gZIiMRRWz!WWrf2x3n#pDwKX#!VxjeCR9&^DtVRc4s zIA=~WBV-0%Yk5;4`h|r5O=<|Y6<+&={KApa@uk+>K6jviUg)deASv}U!-~4~XPi&d zLsS;`BenoG5U6RS(&o;2`(a|*YP4$_#BPC0P=Co<`V8*Zv{1=gM2yF`<qxvtmyGZ- ziRoA%bqT&aA*xdNMxIzA(BTTYFUvhb-#7W6%P9HhLY@gSD&?=oi3z|JvkH~6j!C&J ziAy&t!WxK=2U7RkSxlE3kE_ZWI4|8l$fP6|k3N7fEA?c0qaHTq;&wTYrd50%aDmC> zg7eX%2QNF}%6|2B51kGs4G&9MFT=R>S^_C{Ob3RI#x%;wLt{oZw8~KglD0Nr<H*si zsFL=d1k|;J$z^)o!)<qd8~aW44RX4af?kSK*Wa#UWEaYd9(rhqIq};qg!vps2TR6{ ztL)oYBo9)pb4<CnaA$+Jl)hckQBsP2M$04bP%U2uNtBOV<My=-usm$<WHrc6+G!4( zuJz2~muC5W`?%YR5B|+LDc`tb(hiIUM%D1%A1#toxS@Diti%@<KB!ak%#!~gZ`Mn@ ze*e#K8@RK;=Cyg3piRy~i@WZ*MUwU9^41=PC*-MyMl`)t1Y`z330sk^2c^kd?+gQv z`2d}|ovatj(kvb2C#En6VC_bO;e_b<{ws&s8jWP8?e*-}j%Ru`hvE+mIrYnS*=B2; zu{!7l-qP#k(<J+}J%V9I{+f>#{{x?jg|Lk5!#;ocqF6S&9ZKV)S5`h(fXaHkCV4{f zMrrE9*`q{RB{tn9vo~j5TL!0gk9YXY!}8eDM~mFeU@QgmSh4dUIR@vXN<Pn(LfK6E z7r~L=zC%^|*3g%n9=@vDCFfNhU;8mMpHJ|6%dB(`Czo8iKkHG+?AXH>4~(C`<`epK zn5io>Zo352+<c{4P}NeBz_KQ-C<=@Y&~dIOgK~XbTW*`A;0LVZNVaON<4j(WR%piN zC38z&su<|CF%tgePYF=WF-zI&_wS?;en3KFX=Ve2t8pqgSxx}rOTVVWK+`8lG6zon zj4iK|N-1!N!BUR8Pnq7Kvw*hoCD+7-d^X0OTvyPs!Wc2nq7>L+>M^EY3cYh9XZ-1G zfT?E4W;pvv+>G`#;W$pl`qo-;UICU@zOI4dGvt_(P0Kr&yvL8rTK7ox;!RS4zPder zJxi!b^+brvoAxwx7hn;AU~o>biu9>&6`1UGhM$&S`HFO!C&+a-?7SdVGbCT*L2{bL zDar%ZMC-25z79W<NwT{r_S`|QjI?hUo>x4|6+8NI_wMyUhMv_4ES*>U0~}oM{(}Ot z@4o#T(%mZ=HE;?ii(MOYMa@JGHvnf@*2cA(fOU5~{qZg9yqv@n8^f)m%xfBD)dznz z>9$Qc;b)B`h;&8U0plNv9rBlzP)W#5Z!H{_42fIZ@?GP#^oq*gSJSj;ZV{+oh|Eg{ zGRH3%6@}7OdYoE{8PYP>l4<d*&`eQ#n<4<)xedlP)KAq~$2AlQ$k!$n9CbD=L?K4; zm*i&ihN*c1nZphlqh{P_F=Vj;n;1dlxf`vo#bw4+98{!Gj$Q|T<c$WijB84fczp_2 zy`o7!dbS5d3mKt}!#`r}O%EVe<d6w6&yC~B^X_>S(?}#eL=XP8rT=<*OwJtz<5*%b z3LVf2+drnFVf-D@%KLaN^|a-X=G_kSl=vY)h&#*KqR{JXUE7A52D$yG;ULWq!2(f_ z{B)_<H=XDc@zYj13$UhpbuZ;jM8gR4vNh;2Pq>ToG!|=x4%Y$k)?ufczi#$k-<L5D z%j}xLXr~68)ifJf;6$|>y9hHO=FL32b)8)gGIu#DM6;B5+vE}*eAvMbA)>GN4#?YG zlz^li)`gfnx`a^fit#|b*0q^U2E2I`Wp~fyy%7M&%xWNbmYX?;tPX`GYnnScZKZm* z_4c{_Whb^p{0;DHN|tH-XsBrdt4wMs|9z#NUWzw75C~YnSdQTmlTLUp<pzt@qT+j$ zHL(}b-QHPQ(p^72Y0jJ@2?84jY-?#+5wx;T5M;G}prEj-i>M!EowE5kL%5WP-;S9$ zR8<ZcBr26h-Ss#q9m(B((^@R!Pdc{<e)l^dme=s}v{2hTs@TQ$VTD2x6`fMka!8^t zt>qtnugVh^bzIA5o;`!JTGfunA^PD?=95*g?w%T!Cm-L~ULKMO+v73HBT<P~(+wse zi%WKgKBrH-mkI6+n(;7X6#XURj9Jy{Fin5!MxIjCnG<g<ujGa<w&-&|VOK~)x8V_D z$S8lUw^P1Ye-(@#?_4dPSeaE6^4ber{A0g<SV}P8!ZFXgE?KJsMNU+$YnAUbT}0$J zJr@a4WYu_u4nog)>2m~j*L65P{)})spf+EEyuQ<;3fA{LE4_)b>fAc?rFXim+Tg-z zMl7nF%b_?~bxMSphf+e0ij*#Pm#|N~eV<dNR~6s68OhETp1oa^=ze|NsZwdONli7A zQV)KivH0NT=6cB^t6jX4g~2e$!^5|8zxjOSkDbp~nF7v|R^=jt`iJ6|4ypVyn_e}K z$&WSP&ItcL6^}P6iOLWqeiD(?C6eC{Y40&KDrJXkE@EtryRz!(IsOv=IlSh9_;~I! zyLZ{7!f!Ws3Z8S_^}M<^w0iBn@hG8$=y?xg--IAi^)`w47BjMEHq_ka6ScU9#NpJ% z#Mrv<ex1H2(E0>?L0TL<3l-j7o0|XviA)TYXVx4pxqWlZkvptfEzg3Z)5_p7BzNk! z6Aj)f6NcLwBKof(Lyw#f>sPO{d?kn9K}MRU2*9OC`hv}H!~2hGrXpulqB-Y~o7=-T zThlfK>2YS&<D+a0vrhZbiN0lUW#@ZieCZii$67!Lae;5>V979Ly59T7j|`kQdlc4a zv=|v`d7moK7PsX**k-<<$7VNE^0mvX*A0wWB6|2;C%hA=%{*}7(jTrD#PPYSfqZr= z0mvEt{@k~_ywnL)cClusj&13uo{9p!czB()i%S9ZM?U%XUViK9za4>W=#rHoOZ1f7 z_5Blw|Is}Kj|DTaFHhx#@V@WNooA|5U)q|<;jV93<cHLoq<nu;Zi5@a@R`@tzhl>* z{`Q?9eVSOP+IE2;UD{ohl-M|}BVKZu^~hp0y<OA&=v5xU9hdyM>^6BAnoLJGcP`vp z{qZhFlHM=C-{0wRF8Hl{LLL}Lo-Y^N3a>@nrhyf;!YH8}C8|+<t??*!h3E+5r)OwX zT#zHIg;XxE;3iwaD|zS5b>veBU$`EOa7*(MNPRoB)_JLOyh>^0Cs^!e_JVltX=<R& zsm#iI*qz|}dXBr|`A2Q)=HdIbkfkgje^{zpk*&auPUTY{PMYu|OPeozMIu~$TM@*k z9-mK}e~xt2V^2nM`!OiUFRzw7ynyt9A5WCwTr<vO6(2`1s>ZOv!s(rds=oBAVn)q` z?_!r1=G^Grt}JJgDr+zLeK?EzGNkuR9}8N`x$TbP!;DIg<f#SZs(o=(x1I(l-{r3? z8w1(*sj54H&bB0+#Gx^h3-9Z-W!zqrJSsGHUq*ZYC<_hH6|2|_n)c#elZb!$awIkT zyj!)!gQ4bN44=s|kk(}95j)6mIv*M24Zrzv!uF5_q@kJR`YCzRkfI<u`nqIoCZb(j zk{WiN6ClYfn9mV7_C{i~<%6BN#q_gOCL5we&3+b7Uh!@G#AF%j3H)v+17#s~-Kv+B z37@@xwrD!LoUoVesJ)D8&HsG(jTkTWS&>J=_^D&Bo|b-L88Ydr#L6LI<o4&a)Xwg0 zRDLTvzR1L;gJLpUPnE2WhB7-yunF;@hx44}`$e<tH}T|rui}vLDY0ZUGLCrp^?0*- z<oK|*#>}>Q2Bn5>MwJyrss6)hz9f+^f*XCyrQ<3S@KHau8LTA8bXtkX-r?A<Qz=EM zY($Jx2gMc|z5eFB1$9}#+PaSG&nMW6zt)+}_-U@Vpy6x6R@4(oRvl$<s5M)5^@~n{ zF42rmK};;XW0Z|l6r~|?I&xN|*7l}qlM}xOAv7*l3x2+1znOQCWvScFj!QV7!z({! z6UDg5q@m-Sp~tbW>g+P8M-nuV^3-OIDpB1)E}u)zp_46bKBB70hldN{qjO*}T6k!o z2IRkuooS<F$Eufr!l`1oRHP<o+&QywoK^P4j<pY#IKX&qRf~^uWB;R~K_C&jPkYb8 zcQ+CrQ%^-+Nc4mO-T>~BMMQxyQW-qaO}$c#HSU*f{wHB6P@-<ufysWgA5~68oVc$B zq6bhVMqAx1P282Sm$oxGZ<B<Qi=>ZzA~v?9V`o0`F=FO{?fOwwRXlQDN*dL}DYVkL zvXMoM2jXt!vP$5IgJ2qtdWoKv7bCS=3V~0c@Rci!FYFBh%Hz9X3$?-!D`=F5VfV$} zsJZNz?a$GNONkbtVNxYVvVPCVw7(N!u5*MKm^eUwFC!Yx<X+C}?gWIiO-NmMP4sEV z{}Pu|Oqd&xSIn;Qne`eg|Hzz6XG1AeddyZ$uL00>qOk@7OjN|RxyHFQfr3oPuMpH; zg~EPm`IJR>irg-K4~e*J9G^&B(J@}J!`3>Bvpb4?s)Xkiv))(ngOqs+xp>n62oH@$ z8Tg~lPG0Wo{3r)bc+6N26AZ!B^GD1gAWG}Qsf(gz-=Y1b^sQ0skOC^ECGWSrKF1Wf zb8OW*EWvDCa!7khlYYfWv7|7X!U7{R!9vf<J0m4vyGtuFpL#nlBgrZ=4zW)32lIol z&_)W+Ml0ioTY7j#D>snmKtU14dH%X|r{_lr$lIy4$w&IKX`waxx1xUVxPDa6yxD+Y zxJ5MQyKnuZ%6%<0M$OzwRTH-gD?j_s)XEK#T(r+k24Sv>Y*WV3<N-jN*eh9WwX&1y z6RE-L!t0SSnzQ13arP`kNxC|4?ZcIX!EaMXqfluEbj<!AfxE;V&D;2|1QK^5#*5R_ zkKRT~g#nU_S_8G3&f35`ES{PJYI8>X{fm*N#7K9Z9O`wmnVyeHAwH=l2EWB2{^U@4 z5dlMz-#QQC!zVhf0Q-dc)nctB(<E65$*>o?Yi~aZq?b$)_XKz>7sZi<nDQc}gaOl3 zB9<;4{m+y}7!0M{Oa_Cjw;jJ*5E@zo_Jap-;@0jq`;Lwj%zlt>hhZ|oK4tOYN$yDz zbxO;?yL-JBuK>xEva#$U-&_u&F+!q^Y>fGY&nUlg3Waxa<=2}WzOANXurQfX1%^Z( z$0Z{H5%vY;3Y%u`Z>HGFn#2zqoP9Q~x|w%d22Xhn&1x9xYV7ri=bTljs$01VnY^vt zyIi2PgO_PcQ)K5-2eGP}8I!{IUs=4#=DaJZB~>N{O7N}#e}HT<$+zj(sD`~wJ<_>1 z!wEJXc0-cJWzniKj?F|0e1NBa3;3^cBQuX3r(<vMqG`bB+a1o>gDR}G`9?=)^&Bt@ z{KbN~&FgSBwzze*wu@C9nBMuoLAB)>Iq{^q6aL$8CwHuAFj{8P(y-sP!k-L32?S1= zM;x*T(*Mx0q9m#YHjpaAjp*T;SQ^xz1Wl?{Lm~@3@0fv3(tXs+?jIgcUgrj}<j53u zZj5H8kncd%e3G`_IO-GY**^TDxEsRRYJ#*)++qDthl#deP!e*KRwVoJDTx%U$()bq zKQ-j9{7wS<(l>n2Re^6~5$0KmCTfBul<dv1n&zfN1!uFyod+A=_o2xuRu9|m;iI`S zhO@>bgJ|^tpw+|Gx54;GMyR|aMW=<Ie`+W6)^DLg3MT#0E;kooXxgZ5B_|x_e{;LA zL8ey3Q{qAT)lk~k__){aqtH_OyhAOp(1FQ|TK!SPBZcfKk?t5!HCRVnTmA%;2zG6J z{EMy@Ywv^EgsJQgRa5k`Gyli3hZdS$?XONOtfwm;8v5S8DNs!Hr;`l+u97A0vDAmA zF8GcHcqaoI1sX7)hS2!@D${R1q5f5cwO%@ymgWtAEC7%qn%PRHdyo%%2GNQITr}xP z?iDRO_1tl4M|y~TDe$qg0knoQ-m1KyfJjvCkZBbW0XEtF61HLQ8FdW?xm0Cjl$E*E zuwcg;(lK`SZtj@BxV%?Wy*&(D$Wn(@NfO`YjLQC}kpD|*wi@kb>Q&Fap>>31vP?f> z$>$n(Aep-d&0BahW#~)C!$X87VCh7Y3!V5xmh&3go{`_9uLl@640z5TxS8|^u!{!) zbIhH+^y<VvezXnW0_<0m+mcc~-MJf!Gx;DeZXO+K`4efDxIt!61Bh!f3xVn4@eODb zF=z;7pb{lbGzwJUqO$qBDC=9+@T$+F=8XSsF8*VknGzeF-LeU?GC&#=3w3vIeaBHi z_3%X&IW&5m`T>?ENU-48Ki*23Wau05!f|qszgqi6nFPd3Q~ZR<xY6Lu>?TJMLy^OP z;<rPRKTLsty77`hW%3bg%qx79Ngqc2^Q0;Z9<}lN@Bg-$bYW35W;cgWI9b?Ap%O>{ z9Tcp8jVFud(q2hmb`(wMqxXh+C~X(7Z>7J$D-%+0ZRA{U;3z&nvu#bfC4v-VNvYmr z{JXFjg-x@7qmj~9<OOS`{lfT;izHIFAOfiLKWjf7<0$Yt(yq%qk2T!m)xXPwh^l&6 zHa9khb&mLd_ZT05ExbzX|34J@|AZjK_yNtd^d30;HgiE@*LbU2mGOd}2amW68^hD; zZM(^!_{gIbh7q%WJrV}^|E&V?5!~tek4Ao&u-s4(LW`ifm5@yXN<pW`EXtYi#^aaO zNH~<D!FiY8cBwU>+4XQ0ZBa?2RR8q@%NZ1BfJ%qwSgQb~A@MJ%DlC;4JErV93SNm+ zSRMLM3OkUX-`6qYxM{yVW?~~~0Q10|D%0;d-4$RB&-vGn$BTuAcWCu2x_?M64H<~i zqN+2XfTHK8yN1NB?U#>KDMj3$nGI!j21O*F`am_y3_Pv*)&6xsswAh+ntU>(8U8v# z;XpuIT@_~rpwS+uZf`mOhK0Abg^U;Uh8yu1G<r}?+0A$TPOCp5DuMHq{QG;PU@d=t zw^{QOnd-@mN`421908lovkcG5`(;&1kUPL^Sbrooa@IH!NP0Nm7=f14zexzapm3h+ z|57P!a_Hw>0XmNXyuIma8uYAVPu~YK`@0SkJ(vrXfZakBS>*Y_3VJHj?P&Sozsp*+ zob{jfPEYS<w8hyB23o_M$+BO1NGa|@SvZ{D-K~&pzkh{G)#rA&2<yTB6@pLmKSlH} zD<V`)tF+%$-kWuOAQP4G3k}(7zm0k^-+(T#^Dee=Gv|svpjkmXOWJ?2t-Yn6&&VzK z)JRM`tD?*!XCsw#U5(CL0|XuR7>CkDq7^jDEOS*SBVOjJ-u3%4|09t5%QB-pQoN-B zP!g$`%uFD_wcCOZpinUOZ`S)Cs`}?RQX1yO{_i#`8*S?TpE#=j-=zNE?9)ujD60vJ z%ca_4nymm`UC7nEs7EMPH1&JCw`1pa{xWtm-o8bQ&h&q)r#r>piZK(4es;0Vmj=zq z(4nM}g(kmbk{8Q+&UV<J$kAZG7JLv8*!3D{SYQX`8>1P+r@z!jg{pW+5CvdGAFR#7 zI0W5~S(O(Ba~R9VeSZ>{l&dU=23xsuw-E;H-*sL$phxJVcCYVDo%K$;Q*{f!9Iog) zzWGN?z4+{J6T)xAx7g}U@mD-Fmal-EG0XalCTg6$xGqQI+dpc(zw?Zw0a0i9-U&rf z3dMcePxj(6Y>mXg!uvFwDJJN3@fscPoz36m-<KP<mF?1bAE857BQ(4{Ha`B8yR3J& z8zVvqO%zY=<!MA69@LmU{Vl=G>?iAsQ<~DEolVqkOU2$+hk#K<Mw=sLJbtwFS?x}1 z+lnXkces8&?z;Wa?47{N6}i8|JCx`@uRNTNCD<@i)O~qpC;YGQd2gor-d#T@C+Fy( z`H&t>?^>i$`toC$UYLmMK?=90Mb~qZ>Mv;CH}kQ)r<RjH!q8+<mdCHlisa;~9XFLZ zb!xED>{;M^`Na2mvV<h8?;kKJB!L4kabKGblHD8+gBqN+$9xwB0!8tG0AJzDhmRQ4 za+Mn>c7i?Mhkc)2*&Z>6vG|BKZQ+tbSt^ctvDKeDs-BB=@}4RvYUu3xn^Gt&WB-%2 zXvzQE+*tRT>(yD4qe(2EU}0e?e&c(JX0p2*-mQZ4;MG`8Jfo(jF4Ax2M*}cY6%vxs z<h2sjvJ!@(DYYi{k;lgYX=xNXHRgU56%~TVXx=N{tF!$GG+Q2nZk=^+3Xs8JW3c$Q zCO0>?n}^5F+Pin}F3#rch`R`w5E_k6=bS#`mk)Pj@}yx$&$01_!A$=C9Sb9T><m7_ zS4avVfZJoRGWNj(G(tQ0o%+0(pOlRBcgvKBro+tPac-c7jg4^4tXAAV<;06R|JWpB zKA-ai*WP4qn!WvfpgRfy1_IFJS>d4j8)C<e!BN7*@(&y66yp?Ce!Mv>gVvl{wyoXW z(%&8G#cJ91YjCY<Q>7o$xqC`NpME`9xZ~8W`i|yV+*)b_iC=9o!17AT%1?JD0-~d% z`DxSMSOsrwnR~ZkyAzL?d+~ZSN5UijTBOw9WgcugI}y{Y*uNZc9Sa370R2zPp>Vbj z1?GPa(GJd|9FG2rRkzuabaC6>CF?bLrjVm_TX-xf#h8JR@7^>JroOwo=gVqyakX2n z+UQ^cBeA+ft&dilsn`hjdjbLID1xGN6zwSAp})YB;tz$B-mi^&39RSpl&1F4r@P6A z2&r9ZB$X!@k0}-Law53k1fR~%|AGQ)B&|K*b!U=5;NB%;pGZ7~oKlshA~z4i^Ri#8 zv*Hs!)Hx;#TXM_(N4A45RsHRHq2$minrt#CF^h5+IwU&H3_`a#i~PlE$#9L>L^H-( ztc~cY6gQK_qFwK#!uS4D98#|Fa+JZKC~*JBAGyXfewz+7!K_o@a--vBW}iT7Xfc~7 zt2{~W-t`5thi2Eo#A5Pf`wwW!T(8YXtUrMbLq<l%M!ig{QbI;nUjI3oXq7TMJ6krH zE5krp<IKy83X%{rwIU^*OIb>h^s-i|<i)X$0A~jO?;qvmxlt3Nn`0&C+dokQMP<%Y zn_$0{?QwT}CRgwMcs`XWN!05htdf=h$a68aIa%Tpb)lrHNDl=--v`vw$UYh46o0?Z zwkkU$%r=$$a70U~hR{L5nbr;g$wAK5h&IRT`OD%aaeeLxq7HMuOtx}~8>lq!y_l2X z)MQnwOp=p>`0IHzQK-F<?0Kx3bpbk5{5xA|+5g9(95GjnrGN4lnQ}5Nm8e?<{}dy8 zg+$P^BEvRuv>C}ge@pUbL`ztGhl?;8%|VTi=2_1FBOCZVFP8PKJILe2{023^-Qb2? zUE`+~{PF;eG%oX}KnZBng>56;iso=IQX9$?)1?Ss@;b)l(r?OB8jhk8p_dGPKKus2 z<e^GW8_y``kEZxt`HOd3H5F5BDOV<X$3xyJ4=ff=u8<qs!j@^qhgp0Moive7+e$ZY zt9&4=RlE=PGKbL#e3QAPHO`QUe7Na1;yg2ID{0`Y<Wm!llFY>hmM|KsjB9SDNqc19 z276t34X_&Wg&wJI^JXile;ulKcH#;I8@pQ@%TY;8nW*>NstYHY)wGElg6Mf?r8lGG z*2;G;^X^_c{(!OPuEG7AsBtyK06&k_L7Sl#3O*Y3);l_`e9<SW&O;$C6vsI;9yIzY z3Qk7o_PjxMXd72^hNayfSviPBG+ADp+3wbZlUcRLhli{|?eb#vFQw*i-N$RSC*G7* zezPjK;_I&l(`Q3L#%|iHW_<D#yhY+i?{8Q0uVfB%--%I6Ov!;{g;n^}&&%@xSYw4_ zOAb(}q)H>9f`g3yT!WOz_`-qQ|Kpi?g7XrLCZwtr(kfABySTi}exEJDs$IqCfauMh zUCn{^#>5bS!Qf@I5=8%r2%AQOUc+vR(Dqh7fz=O<sl2@f0&X&vRG?{@g>^<=Z@KJE z`%V7T-Sx&t->4A0e0kO{ojWcSsSwsc694|{LnIuJO1R~94iXL3`j;0H6%#wnj6&PL zsnK+zY_`#~s=ys|Q+B}|<9?|DtZ7k~r>+HC?m2pwC-D@ZnP~%SArKm)g&;#h_d9_f zZAZ-?r!Utpjt4a__Kv_UZLbLEcIRt~C+!?x-Pm6#1ZEkAapSAV3@hc2+nd?^2>Nl{ zwYC4Va%YeSoPPfV=Nw-qqNru7vXLl>DlPX9CkR>D&+tCK68w8YJB<h?klZ;XQ~3FJ zBMowWl+Bz*<N0rf<K+-yM-7-yyGjq~r=vQZ5?pCSPA^93@GGhDAW+5DFD<7Tt;Y-e zFFFfD;H3^e@U~5#x97j})xVLX!0K7<#H&e4hqyZDZx6G(JwS<zN-}c%&_O>HyeN&S z`%pubSM4bo*+qrd{kXUcdO!zH1&vLqX%{BS9yGFXC530jalh~c+z#2^dT0KU&HM64 zV-kV%T84=035`Tc|JVRGy3s-l6Ro@8Dd^$yS^%WtY<_rW@!oH}2h{QFXYKgyU`d%= zG~VbXa<n}b8&ofGUatp<1#oQo%8XxvPGHwx?%$5%sucf*0HA+IZk0^`aA@XyR~Hw* zIy)uP0^^yLI?zneB}%nui%BmisFmhIbSdC~kcCl}jC&3wz1kYf$5AU#e5$Fb>2xr! zO%WX%8+&oNndK}D+^RC}^9Q`K+`1*p&oJ6RQ;XrHqI&^9eOk66hsI^)y%`&@@`6gs zaSZ(EuQe(QpRKXT-EB4f4OqaxyStkWP1qVjdk-;w?WgX<qQmAts`%doA7FR;afsO8 zXM5C%stv#VdA(y6QMS+)S%l*YL7crTqRPZub8-0<BvSb0epKHY)<E4^hsE%EUWz*& z0N4(TK7)4FdM2ySdTKfEVAV{&$HZ{u>mK4zo&D16elPqn<AxAQZzA_mQ{6C)veT9C zSAjyQbCVM3{kvhM#8Ivts=c<>C$7|s@YCz$qqV+9a~k~zexHqu8ztz8xaY-+cP|dN z6Svy(*)eTOgPk{hrz~v68^Yhsl`@#?%X5N&G^B^=8a;N!9$?5$Sy{;~o4Ns)IeDT& zTF%sgetTb<OvTa2brEqaWjmn-&_kDkvZ<BWm3qGF1ji`Pr}g|O>L7#(%PeQ5sW3TN zi4~pS5v;cH00KJ3t8FvzgYB%seTVNNORq8)rffvKyzvY6w)2KOORfIoY;LSxoDW8q z+}n2PL3+u!@zt+zE^beV#-FJu{Zrt;A2MT>H!7=|u-}IXg`@E;v(bov9==jD$?usv zq4}z4f9nbBD%$(NI*=VM#A0Cr>S}R6-Z7U2t}>RqPr^)a%7Ib{<al47&3-4Tww<Xm zxpI>LKurzc<|K?Bs?R6T+<$bg2lGvtM(N>d`SSToXr4=~2vY9JTNLvFF&!p14`69& zbgOcd)pYI~Z`<-FM1jElfIGd(naq`DacpCm#;i@UxZ*ikbuQJ=QYD$!9M{jD3M!RI zxb&<!B?W|Uv8UwEQf@bV>}4*}p@u(?y4cs~(a`)Ktn66oqNYiMbyb=)G<NiE$qkfL z(<--e|H^3E*Sng3oAEa?4GY<~a!y#q#k3QgKO()D-)yFiL$hM2vFW2*V)Q9;)a@{q z?3)qaN67_ryY^Uyqj1*lc{%OseEC*m<ul6=rgsCZzJXb~#r>_}@dDASost1Hqtm96 z{MP@AvGWXTa@!WQE(^L;Frf4zp!6bw^kNB3kRn7tnskufYhc+Rlz>PHN{P~Yq<2sW zp_kBW=q;4cLLeb<zqR){=k9yXxzDrz<sTn&^mmLoN8N$=whH})LuFda*^gm5;_WE@ zX3R>ol@PLf(xt(Ja-DB$GYb)R`@%5HC1eVY`D#^w^>@BqtW@&{pl!o^#WCmSJF1jY z${VtzS?`rs_!?Sh&(3gl$3erB+FM?b>4jsVZ%6V%(Z4nx_-!xXo1)*Oi2{QBuc9Iq zy(xL>pq7;$d~VZ>YPGs>O<|Qehxk6-^rz$)Kq;CMT?QO|{QT5_ecL}msX<LH`EQvc z0H!iZk$L804G7Zr>(dC@^))q<myl#54k^c*eQ9!82;QXpmefKhzitM4xWIr&Utm2> z+?+62WlF^gQ17hYedqP--K8LuhI8b3Ax!SDxY&?hfJv5qvS$*MsZAiMLyMoE|Cnd6 zyrxF|tCW*orK7&j#^`f(=jJHQM;cE+mRb5eKO}47tQPu89Xan&?giO&e#goNQO|9w zG{(6a8D#}l(qJ|nqrKzcn2rEo@iZqeZonMcfKtn%BrE3Aul&WWfVUa)j!~Q6ZdH)) zjZsq46BWr38*-7Ng+*YG*04`!_**i_>}{<tzOM&aHr>s)lIl~Dy>f#`((0ttzBY)^ z*mpKZ<Va~k(aF0C#4zel47fY1p7?}(kA0KqSE@+9F$q3y4Qj#E%s3V|ODENnevO9& zipJu`qY1IS?R8W9KqOaiVTQ*m3%LeK!8)=@r7kQ<mrAT}l`CK4+ggWhA$TiUpqneo zJ<~ImGl->sX$@I>|2;`0!#eZF>($M?rMT=bd%VuugFBr@FNNP*dfwGSwtHD<tCK!F z<k*__e#2|PmgaW)LwD|k;T0qADBU4`BqTO7f3*V>gZbmqpKJN8Z!gfiN$8WW*9%ql z-)6V(ON(6UOyIR_4hfq5eS=N<n?C_J>DZH@)PP&m>ye*m*So8g^RnQDDuE~vEYlSV z<lkA^)mlqgVEI>U@CV0YYB@_tVWU!GNMnF!Q%FHdlIQLNwMC#LsC+B1h3M${`q0_> zU=3TZYkO=cA=4I;B;8S7NZc4}-&^js>=d|jmm7w{3aH@<Ow}d4OocC&XuVg9cQX$0 zxEMPl`wd|@<1P9}x(wN7Gu98~BGpw!i4)N($Ax-aDC!ISsFV7q4Z82ss+tsQcP%c! z<GFbaV^(#I;c^bTtsxKikSvj4p<~!%fn0ocQ`3OZB!kszXn>1UpMva_=ULv?yWg?$ zU4+B;sm7kxSL<Dc-p*YyY6`Q->Zo*V;HInJ%8x8ncvB>O>uIYm3$Zy~ZMJz!&1UO| z;aXkY^_4BIzv301g6h8u#G1|KWkiskt7XNoIGF}<%m@@RB!kntW8LF=2NpNFW}=TM z2QMn)`bs9#b<7Ex|LSi)PB>^AXJJRUIu-YMyi3ak^y|*!r8Z)!XK22gv^P6q3%I^k z65fBkF_ooJcPC&efX2^>Uf{R>b+yP8?fDxDag7?uId{??EJ2QoYzmr)#}0#yRuefu zx$F-bG0K@wEBPet*gK|o>vqK72JYc80E(jj=vW$U#;+f)nZP#^o(X1*k`tgvvV!2& zH<JYoF6U?=LZ8u}chbmY2DmCi9#$Y&7O1gatlFgaecI#_<XdeDPQG{EX3;JNgK8!} zOGQ|Jc!nDkiyTgre+Ct^XoLY#f=wytU_dQz(`KT`NIXr}bK#3J&s_;X)&YchC0C57 z68U`Y_WFlQG?#hFzh`Ux2<6gKzxz>-T8R@9_UaTPX;y-A?fYo|zPX=OTS1zG(f<AO z6|IXvM)Q3<<akyVj@kFunLpUd3Yr+|7V1UjwWy}9h17-+(%fu9;22Yq``olEQM<|a zIuJH6hu!ha6_U)@)iX3acnok6uF{M@b=y@2?JVp3L0nuZFvbliR_X!GWw#PIOR&84 zX!CYTC5oSqumJ}tJQ+Ej&gaT7N%g9JEp_>0LzbqkbS1nGetF|T=CcpNw3bsG(byxj zVpaCO9%=F_gMZC$+L5pa(ud|V;U>XkZsWMJJXGGlIAhbFo05EK*pxmlKS5}Gk+HAT z_5c$#;`z^?@d*;I(RY0~?*C~{jsp{Z*$jjoT|8ixWqk}+_$BOrX*{QIHz+=$^t^}9 zq3UHqrohW`>8{Jw;3PE;J&F9=Q3w<b>aQ0-1Hm<l^_Gnx(dY{+e4y6c{g#k9(_!Cv z_Z;qL`tCV^CM4QDiMY=4&Za$9={8n?NZ<NAS?YPR$QNAuPJ*uTuX^8&fu>;pCd7AU z4{P}-7D4^LcorRO)(s-TCjeLR4YrYFU9R0k$cCMUG;w7nT8YXX0!iNnlnYsd1O%XF zG6~_4rBbQ$8ynH*FI;|oc1qPX=XXWqyta#b7CGSK>pOxPJX{@l`{^v|h!sv>6+rss z%3jw@(@^U20u}7Jb!rFj_2wee-TAlQ^Fi~6hTz4~=y>M(ES!Pb;%}&p#}=;57!Wj_ z+rkcU=4fH(R$8s^l^U+;7j~}TqvIsm04Q7OYzu1<w)PA!{UP1_?TnGk++k~~(AMLg z*)wIg{Ed&mA2!-HL+4&|-T`){RyhSet=j4zvYh`}#Zl{f#;1c*%!OmB*{;p6GDQ0> zE#@?=-ghT>NJSnzLEOkOoXN9VO(M$rxFT@KodNjF(Z=wDgUJIiRyA7))FpBycnYo- z4;H-qG3ROekIMWiv65yPAidf6koP$6#wEiRQ7qK^w2!4&aaT~@EkHf)LaSrWzbeUZ zKuLhncs!kYjoD7838+M#7Co>liOnHgxhU-t=kNSbbx4Og@B<=aSTe)x`&8i@<);L* zyc^%wD#H7iiF@VP)d|H#-m!CcpCcG1LbUGWn>buY-oLg8O#NMzWP1HO<`T1<lk}^C za_=P}SBc%bZ}haEFOH-E<JIbnT@;^F#xMOWfv;AD!&mmZJ~auqCHfvdP4H`yu*WMJ zqJ|g`1y7J7_k@q1DZj{#M?!+MaAm`-X1K-rFZf6L)naT?tOnT{Z~M{}Yyk4AEBagr z``&u`2A8r~Y83tXBjT8)bA?`vyaZQ;U3YkyO*>wr1;A1&$s%<^-3dU?{g!$xJ61(Y z0noG@Ixv%%*#O4gjQ;&jAbQL(k^J{Q?9`a!{nd5=GWkGzNsqPPhMiHFe*OBTktB2* z016+0PCoI=2~b%A$!DTK>ovfa{GJ(-U(h|2QuL<ChRRCQrOVt>f1M(UF=l~JiK|+z z#gg;cwqxq8oIvzItw7Qp(EsFhN-&PWK*y@BJE}4A)_Od*QMiU|(&a`k!aM1Qz{mNU z#r|FZim;`QyXIU4Q@Em3U+CYO#pI4`?Mgse9h9z&2g#X@>GQCorkk?&<$b%YrK-n2 z(O7b&=OPlCf(Sf}fCIAM@Xl8j_}EXUi%IUbk`1vzNb|JGCUo?j7M_G}JeW$vI_j$p zU-geeU<FjhqBSDu4sW2}ZYT^>8^4o1Z(z3Gy#2XTDL=V8apLrU$1;|-+BYSeo3R4e zKqqQ^-J^RPRkB`tG>$W_H}{$hI0Ofmz+S7<`D2A_Z)DzFtGQnH2_q@qHR>1>+RFEO zII-@E(+G#t%wE1?2=ZZ920x_qeYA>V#qFovI?((6yGALy^A2k&K8ityT;a`6+pVU1 zKBiRdcptDnpk=vc_#qvb#I)^zFL4IWW#g^Wk?*_2>mt&+uSFcxiEMdYdqht%(@5eL ze<5^FzkSOLnesMPC@xzF&hGC4S(jXT{4D8Ilcf$>P+<_%El07P(9d!6uH2GiZtEI} zfEF@m>Bne{l-?3T8Lsc+jreQQCJjf3hIX1k3LBw%s28~%vSO%C!<o+o&Kx;Pqr`6l zrU(Ea{tZJgF)V*$O%d~YLvf@~cSO{@K=?Gk{|iX`+6jDc>C0zFi*!vr>i`M_%2?J^ z8TV2IM^dMc1pmE~r54vI>63#Y-Jt$frQpLgKj0Fufq2#%Y1T_}XNpZQK=LnY0Lr?N z^SZ*eGxPaqZ>r29KuadC+|*=4vnRoN006!Y<;#gf2iSsn0~r8yZamxJEX6g%wCIFI zFY_}k3~#kg!W8`W&nYWQ-QtdSyYi>*N2@8;#m)4^$&~3ZUw*z#x9B<8^?bIKcPsj* zN824wrt8DZ(_12Db)oPTwr8*PVzBv$iN@;N7i2zWk}z*zjXS0>g2#_izm``zxu`7y z6)$r`kzBpI*!x+X(C=9wF6E%T<Cq$3z^Y=iob(0mjV1{@U4OG$Z?fmaJ1Z&asJ&k_ zG~~I__ZySLC;P)MnvhmQ8kPs+#-K(2>b5cU+{Kmm&hq9<bt7x_WuNc{{>&7H$yls_ zeiVQ!+8&j#t;F=Pb-nJLKvtd?&ONAU=lc#Zq`mmDj=S1X4fNmGBxX#K)h7Ae^KA(8 z7Ug;?Ow)THxBc~p%JoS3?^|>8mh80Kb)WKlo-c{Kyd-FLa#Js#=gq@6MI`6VVSi>d z-SoJi=kmX~HAfMzaE6Dn+S^~~+6u!LT6J66rsG<kG%=dbf5mZWCN$h}BC`va<;2IP znbq2|1YKv2yzAu6t+oB4!=PM9aoG8DOrU5mxSl3|!<*M=x2waTQ2w)B7iy305q*tf zM{=HGm!b1tFCci{;9q}gEU<s;RTESx@wYm<RhnNa80}*cbtml0R{@h8&IbjJ`-LG7 zR>a`8Ec+(c!7^cY^@)9HHSB*Tmb|^oviEa@z)`y@9+x-~wH7+8=KM+}TO;XOo6x9g z-B)e_vu5cBw!(k54_l*Gz)_nYA>>yYE9PKz;b%%O>_uV4mxS(^zTNZqyk{$CGQ!!y za~q_0VxFzPij2f$3z#O~V1?swI1hh+bpY28B_uAiF`rOLv&7~PK*uFW1w+CG>#8e7 zf$D&ZBT_EoH-U#GhE{W*r{=;{O+q0&%=h+m_q_f5M&sWD3>WT3*pI=C-D_6MI;as@ z6~e|?e`6Ff)DY)jyL~%nL;Uh0P+=UDmz0|U%_o!j)zt`T*U7b$Su~aBZnN5A6OI1? z=f|hj(rUd|j86Pk49uFs^i!SAGN>RW>6eevcW$lSG{2?pzS&AuCwcDnnVbZKaFbmM zJ|ycmGuLI+0o1)mA@VTc-ixQBt1s=Osvoe{SnVX+=W`^}8~FRTZVR({tp0K}AFGYQ zm>Xr@4>B?sj@F2KZ0c;)`f;Sz-453jgRMOnzdjtL5$WGL18E7#ob)FOW_g}|f1J;? z{&rsBoyYd}Ro|8Gc*Ogo%>i4VSwni!%yO%_YW&f0t(_A@qVFXqm9ZrAkDHEm)l>ot zoJCKv=X?dJIj`5jX((6Ac^G;X;+ph8<g&BNzYG%nqmOYEq?W~OuGyr|Un$Dhf6VM! zPX}JEhl8ezd7n%j{-yit&mUSFG&H{HPgyO;<#HeWqE8z529ZfDnZAN+zC93zzN?+u zz%1ApjcGn{U@zBNza)P4Vr|-8mND%3LOqe|t*z5OTEr7iqre1l`+Dw{2un6EhlA)5 zJHap?J_(<)nOiF;3{{F(@@HvM&j$ZtFRb^Nz(3)zcCpq@;12~-WB$!y`c_QuPwgZ> zOa#(qvocTs(!~9lum8LPMO%R420aK26A6z=FC}HCDLR1oddOFjHmZL9*sU1I)Cvn& zMdkA<Z>3cfg*5q~mj|v&&ZjjC;AMc`OH$D^^I_)+aQU!c@W^Ta7MW5=wq1F5FJ<pn zyW^D2O)tj$i<IHe4&%zg!uAj;r&$I;gCgZB5_rAq*IEu9ThOn!6qd)z4xyH>ioeG_ z#bmC|Np!?lH0n6F-!m*YyFHPQ6+qFa8y6c(eE9I;8&KK^805w%^TkKYc`f~nQ#sA& z@6KrkI*y}29jhE+Ep6YQ!2xvDo+6c@0|f^B>2V@8`1SPygy6$u9VlCj+GD`1wVqMX z0!hmGud0OpZZ`sL<8^`OPN|jVJ%fL@EP#*^crWX(Pap08<{-ilgdf$986SPcq5FP4 zn7lAYG?(|<+1yGv+S~ClG){ez?v0+!3;ju?j2+Zz`S;rpMoPwmtkQ9_yM3*u_0ttK zvGYw^xAh;2Tn*oHv3<arn&lSo;6t%ibQ~S6!TO<-_g<jdBCt|TUqM6=Xj%M3n{#-) zT0Xm(?C`Uj8QvWWJl5wca$oUwy?b$tE8@3r(_iY7j%FT?H{Yn<RbL|)8$%C+o3im) z>U2b}@&=R(I{6w%Ho$dkF~7^ZM#rkBTFkMs@51Bx(xX`^R>?w25~tmm?CYsz4`|%I z$^DJ2A*^)aU$|9?(*Mn4KVaSZn=38B50x%kZ646+m%bDjv=jMjX*=+fP_WhAB-rCR z47c!R1(Rxn1tClC<Lm4m!z-FYWNrXSy$P=z3x9=uM*O5`t`n`#tYxB`Bpjb82>q1z z==OCM83XauSHhK|?Ceb$u~FLTm*l({f6Yk@l{t3$=TGH7d}de>A;=|y8Ss?U%QpMe z)y^cg9jIRtdadVdiLFt_>%sKU>J#F(WM&y?vov$W^OTRI5#o!!w7BL+lV;1W5yLE1 z8y~I7jcrE<8_4^n99+sywDmVw-@kxtI1TMm%ZBuS7@Cjg?GAMpT}~{F4CD$c*h{^- zTFEl--I=Q=rNwx8i1siRr-7_vp7epzX(gY=NEE8j#Abv?RxmkbRYLno^GT>b6sg^d z?^P|WHxuPNG*c~R^wcXIr!GY|pKvDFi-W7`;qg)?e2F14dU<S8_N*sf$seJG>{6@+ zXB>aeQY7GjC$O>h-x3lSCaYbef&S+|uuZ?3{$RG|{M_6*pjF8cFF?7=n)_|o{LnHn zM^y_UR<f^=p;(9O*4xwmeD`WdNJyn;4dA8P88+!bg5epP1mgxj*J6_jzQ;h#_WPlf z)UDFeQpbUBZ(|!{T)iwyldkl{^Qyl|fZ&CFpQX!J#d;+kYNetbZw>7tEJLwcMf7{Q z0r6N_9|s2Z`g)^Ot5rAT@BFQ;o6H%Px5f}?R9-WBxq+$jYAs>X{$T0_M^)?as&e6- z(}C4Kr;5J^IfB+95(Rhv*f}bprqnlZv8vnuYH;3?@G@f9Y%aeF!0+GlYz-2&T3Qzi z9^gORd)&VE`)Kzl#H`U<E!D_W?BjP*ZlNO+k(PFL`a9zF?f1He=ZlL5FNQbp*f!BL z%gOU%?pz$rpx&U?ylnrr_3)<esd)Ozc57;YWfPqlf?ilR^n<X!_(E5D^wH_K$VHU* zsM>2iW3Qe`g>&t0&pxaur8WEh!NtLu+1vRVqJw(=5NJ!$af}?F)15qi)bvbhvD1{M z?C58J)G%L`NGciaQX!Wm@jz5(TJanFr;QBlYkWp&P=T=oK<TGPjx-}Q8|oLx)S{aP ze%)=p%+0*azd&wrS)7aX|2+vZl3{JD{H%0cY47#V4nbzcAlAY$_p7si*u|?Qb0#^e zPb3UT$~=S6szsdgVwaoZce+il7EF8b2A9$cm=y{Mjk-n!Mz^S?ib3Z^__8)f9L2it znnVqYxt{wok#<-=d}syw&MfE68_BJLdru64xl?uZA=?5WC(CqKfO{v~oR_c;I{~gd zqt~4xq%tne#n%Z;lGs1!wj5u5$X5M$!hd6)iZzh%8l=e|`oU5jbo^7{4js;QoRLfP znASCj_HlTOiC$iFn`4?3g?IRd>w6cA(4C&oOnpw?-@Q@X<bd>nLyyi+*EmhCWjeJt z7jn+>*4oKrYkaxP(Llr$)2XPY&sA%ri}B+M2KChM$EL`#$(?xfXi2*V9N&7<Aa+5* zZ8BuoX@Gs3rRl!Muwl=u9Tu-!LJ@aO_P%Gmnf+`h?6YL&xRIf;y&3ltvvu&fd@K06 zAyK@PSEiS$p)V~AjD6TGm?o>6*LrXR!2rIq<FPqaqs#VYFLZOf?A2Zc_(4>sW9;=r zDsz9D+<O@8N)J-S(o!Yl$b%~H(8pd3Oa+KPe&agO*&J8yz6kU|4vJEeu8xFut^zX< z#&urz7)6Y~?IY~W7P~CTkGd?Y-Xt}L0x*;rfajO5GyesQ=HQoK-@_YR?<_JxVZP9D zNB#nmD|xF@M<#P-1Cr10-%l?{1c%X8#)%d7MS3npS)_KqCgWm{j&>+16)j9bu7xq* zc6@}mjC`h<|8feRTF7I#R(5)V91rmn{oPlBHd4=fJ#a|xNH4gO+<<E+YYqRMQl8f& zW`oD+I94?OkViaTPT(X$EM}yc^xI7rTi+9`PPUUB%aoZ|QTx9tp&Yx|?~mnOS8ufG ziHkHzSK$jnOw|HyPJb!ayYc!<s}<!dIx-{~5tfhQg&f{?6}kFYKw{a#U9O=UsZa^h zBd+U{FR%Rqt#bzpMzM^$KpA7EX23}^Tos4@>V%Tp`r^OMNstSWm+dzK?|bSRg!tN) z<5Kk|$?fankh=Hx1KJfF>y?X2<H}^RxJ@6hibC<>EWr)0vGc$1Rl@adBJfYXCfq9p z+tIM=%*~>P#I4u9Tr{sc4j)NbimPQGEbs6C3wQfx3dX8tbB=x5>vvnxeYNKqMF@_q zL)?{N*TM4^fnC4Es_589e7BobvCccPK;XI1R`lVHFF9oAKqyM+aWA%TBlwETsEPVj zzYT?pPD97PaLaAsDQ%8Dkx%S4oLW65r*&`YWnSKF)2^@xdRQe>9?0}`HIx_V0@a$+ zWoxW|oWpGS*83^Zv^G4tWZP?c{(}1RQHmm&sU+r+uLTc1f8Q(E;S4#`Pw(kPX~%c= zlL9B_v-0{N$Bf`~vgm!{d(|}s>p#nH6`zYR{4zJUc_X-eGk@gUL%tp}7=VlcM931b zDilF7<Ptp!VR=xn$DVL^NNrJDf_0-x4O4sxG?+`n7;-3j8J_V@{ukV*uIl~Qg{JQe z_iaw|9A|R4roC?IQKcWH!XWQbvut}Y%!+P&D<b4S79GSM5lHRarjIls+mCuoPj294 zHvT#=`Lh)_LHH7yEs&Z7XNJEhW&X>dKVzgDX3-LAuKUii!S8ts(|aA;wCJMP&vBa* z0^wOAra?P%zuMQ)Y0<rDiC6vWEPAxQ{H3drt=MU_*Wxk%``SK&JX`Del=OZ$44{VN z4gM%;4c#=^?{KXGZ?i4_yJZ0xSOE(DBiL50f~f4svizQL!~2`nhZHTZK67fCe)|5C zA^|z%h3CB9Yf_)%QsCC*zLUo1pw)@0XC=3>&RT*1iZ2+9WD)Fkih6s`&Nk3Y&6_y+ zs7a<rP@<3(98lO;zP{f(8eByBLJX{X*_AwY(_EbQxmuI9Q?!6Pv{^G7p);gcQ1DFr zSOdz2_fj0NH9t(TQFdhi!l4UXD94%sFl<6J@ySSkve%MXkZbWXTl<DJGoWp7k?bOx z1=kmpx%>O}W2<sAd<BJpFB|I^%87|UNU(}-5r5zb?2eT><X_}Q{#Rgfd(0*G5nU5K zm0V;-FTv?w##uY2fSV2>&$A@XIh8XFsx1b$@@JalHRs<Nzi#546mHIHDVtrx9A8r3 z7{Rv{^n0e|?xy+4C$V6wTVXQHZ+qpx2Wgo0m>i5~vBIAfK;v5iL?MSXDNP<%pF!x} zLLayrfNfp|95bpBxEXxzT4{8P0i9ssC`8f6Us-vXvp=p181P<H6{2xA@LU^r^5~n3 zyDVD&ntY<``(bY9s^tCGM;EW(f~Un4c&q&SOW)wrJyXr$f>~OF6_F=nzZU0<C+%0Q zBQc3~<`<%jT7!A>=BN@wdiC}UOJWcjo7}$E@%8=rlvjrV+SU@RtI^o;XCoEN@`s^Z z(alGkt^9q1%Qy6(LUwU!QZ!k)J*pLb)K7AjCWbfMB0p*+fB%RlNYB_!Odbbyj=5b> zTR#;jAXgIxuBJ$B@ZJ(LViHu`Q*O!@QJtuFK|4x)D!*aUf(UBTo+%z;USD|TEGGFq zFzKY17^cs5MbiZsOo<Y~d=J*nUKr{QQ-=06f^(TY-SB3wE~>(YAMO?6*eGPB^tiT4 zNrV;DLc4_@<n{RsSeW#1rM_FPJ5Y;t!wbC@Y_0t&Mqi7~dgH=kW1@}4<%7ui{3)g( zHLW_`n*H_qCIHff6?+&ck(TTB${W+Ljt8y#<SF_I$92<n;q~^TWndQSKQ83`bHH!+ zVGJfsLMv}&L@SV%Rq1w)^M^a$r7twnk2?noVma$TdbJNuu0fu7$!&`{dGVSbKH2HE zq`iG1)dt5uo9Q|h>fyILcKNrYT^o<<%5nkhwDsOI4?)(NjjO<o$RFOj2E@-HC$ACm z4MGin08l037#RY8L;T8P^32`#Ks2^m>9Wny2NOqZ9oHYJe&~ZhWTt*msfW+rqaTgP zGeN*30o$%<GW}W)^M`5~@E?4@)R!}QO;&8#-u>_8iBoGG?PBjSzuH=EZCWQolc0hI z_9B1Tb5LvmF6}tue5LTC-*9(k*Nfak+Akw;ji}fevZ6F=#i~FdUh3vGl{4Cz%du)R zm$wYRaOmght3z7E@~0{pp1pL2T`$}Y)G}<N;R?&i(umMagM$+6c{N6g?1%c~qso12 z-ZD$NDqoIGIj4(UU`)Wu4<=-~m`mFO)YRoY6AOG)ZLp$!Ay`LS5F5=io#8K5x?jdo zq0BUOe8d?%v9D&jC`5|17<0U}#3J5LF$x@QiNZQ!qf4QlQ@#JA;K!?FeU@f0pk&(g z|B_!QHB0C7=rsIk%@_p|jE``Wo1a^ti>N$}>s1d-veS{Y6@jU31#Q@=bQ_h#-Ri^F zZVmypko`nX=lOmYRnTBL4PzLQHyoZCxfv+LlcTOITf6$ii!{A+AR77gdHriZu&+0V z13OIe+1c%`>Lbn>UUP%%C#=3cNTeHZ{iy*}_MaE};otMY-#NniBnGGhQMU^tcuE}3 zXZM)hWPv`*^da87C1gocV6Ld27b>X*Ui{{%;p{5aHCWo|_3a5fx6o4g=LeH(ySRKo z`Tp_|w%^K;GMk%r>xQk4E*ENIj`b=$gJ+mZg0F|G+Y3sn4B87`Z{dOH)xNkHat-p% zVDg;pj!-eeJsqILA}1k&@`tQfS~8RL@?ZRrR`-=!`F+n3TP9;BxfEFwsHHE>3k)co za0QdjyJ_l|+(=2g%YR${?&>#z{BN%xod-2#-ucE8C1<&xBy7eY^o5)5{*vs)&dmEu zY2KWfOKFeKh@`3BcNrPbm9$p0<Y-VvP@i(Jkx|L3syC1klCLk*mn#oEGRv~Be&M@_ zl~vxuL}IQY|HK++1D;Mb>K3p;@}EDx4CZu8#xh~oz>89nBJ^Aqr2XCYbW$YhY;CM; z+>0y9$=*~d_p!)f(#op`_G#W<jNRJe+DE6hSX)l1u@Xbt(dbIf^vRMe)Y1nWyB7LI zckyk7ZO=#7Rcvif<`6A#f9O)nLP=4G)JqG>P}#D7m_<vA=NObOwzuUpD<w4dqxIM0 zLb8Lo;!9&_(-EPhrfAl;cF2MTLkg{T)mE_~PW41%o)w`OR4Bbj>j)(^w0{v{wx(~@ zx|6P$iOqF4y|DP{qN&OZAFgpg!MYVLU9uP8gHHixN%a;-W2}Ws&DReovU58&HgOkd z<G4=oB#BvTLdl*e%aArn2)o7sN6rY@fAuafwuj$Xpi|hoO@X+Yki0D;@6EPY6xW^d z?NTjIs`->uh{ve*N=Z?WWKSbFWTT}f_I<28H?7KH;?a)$2IfGrho!st=+`)zPNDsK zdm4KKAuOPg(u#~USurhDl&;Dy5Wk&S1{^!X<D2lXrm(o9gOBb}y6GyGE&-?S?md53 z`Rmxboe$A?Pp3BctW!;jw#9kMm~v8A7;5=igvw_}-U;-^*c&yXZo>ang#VDG;LCc& z#Ul{>np7TxLy8r493R{gb4FbhE1SG*8MV%2ERkE(oaz?d7M#0YRCzS$+&&g|wDI*@ z@hnPqNTEHXW#Q4FnHI*(-9k>e8Ouo2R3=zWhmM~Tga1^@^+CRmi{v25-<O`it4j>= z6g7t~cx>OZ$!No|Sssyf-(-3sptP|>nbTJ}tqT&pjauyo`(kWdW`t|I8!DUZ!JA>= zO|zsjbI551Z!C<1#13+H{o&N3ObRKRUzIL8J3B?yL<b_aZ^9vRL%n;kM<=<ng}NL; zaZ4D>Q@4@Rwyy^1L%sB~<Wt>30$t<{8M3{$!s3EN|CABD-{lbgia+oENF;2!YLF1o zLUt{#$B>Cm9qYMa>MncB?wHev*cc_D^p}=4B`JQBayIj-MscXqIfQ421pi2!Px_WC zA<{`$smsPd>nr{RH*54Q!3UtFG6ZLg&GwYisBBoQ@!~X#P48#EAnR2UWZj0T>0xYL z@NDN3DdoMvrrlNS6lSVFofkLzv|aN{vRp96c%z)uzSV#0AM*Zxg!RDg8Nz|xJyZYd z%Jb-UYhl#w+0*svUBnC>Y^&%X4=_CyLOEf)fP5-GGHZUQEZ{fk40hQY;Q|y)WKdD3 zN_~Qw8;FQCVKNqb8FB_Z%xZb+7eZiafg{gmX@QL*)Mn+jTC)0?uzkqJGl}OL@x`!a zr&SZaWV2=7vR|vs>|AFD_PZ-NMq#3H{vtXmXI^DJ!77YMoKMjao)ii)rOp|Z?HV;R z6oq(DQ|z|!WD_R19j?l+m^2}dp^P58^c>-X1*hsq2=QitTlKNJAuG{kN6)y5b$v!# zAY(25y+rHx@MiYzQgYb}MW|PL%&V*~JQc$@OZJ4Uif^<dLtR>Xt>c$fn1}kO6r)BL z?U*c`;77x&H8F9aZ-xT)tqBEO_Y_)F5pf@lPW4<WC&m`D*j(r(4gCtHCaX*|@7*98 z!Z3gz)4mvZJS&cA6`y7rQ}**T+aFiG_dL_bH>ou2WbkO!+h3Q_Ybz&Ag^Q#lBVSrb zP8Tx=?5G}Jc5M;EkBVKj2$VjxObjggZ%$G0!Lla8A?*%0i@{s6x5B5Vq37#vS-<;j zO9~ccQ^e&kS)UNfVTn2oT~|<vt{R<HTzU06hx}pQ*p60;dvL{+&psn;BE@EV*&26j z1UqIj<6bd+{V~NEw`@Hch;Y5~lXgD*uhfEiw?m;gDe52HexayIDND#oV|V-XeyLtw zUE!WXMp)g-l3miU93=#A>{h(DaaR|jHY#Dz5R0!dDJnNZWqiGvxMD7EU(}oxY`&`4 z-JV)_=8#5!ygL}#NP0+7>>dep=x{C~Kr^!H?W>Ow;&*K{4cvvmHb4a6dw0k0lz3(l zi8v)%YbJ10H)h5C8*D$rLc{Ey;YndFx6WcUb!!>_D^C39kLzH-&wSu!%^^$8mP?SP zUPjh5Eh?4m@DMm0g1Qx6!sTYmy`b{$#pO7S7hzB#kIt_jc8o{XLk`yl+;E2v@ra;! zFVtyi8lULHl|^&t;c4snF>EKIrK-EQ6XTaYIj}fMHq5P8XgSvI_gpu8q4`wlpI-C- zv7-M9UrU3_Ed1<Pf!-FM=a}a*tMrgpA=6H!_gLvmOw}~+B^ka5EBIgB<$uJ$j=Ko4 zep#zX%;#r91=*VM|4V@U$3NaFa<e^*s<6-&rmDFABFQ2qU_J?E^yjUKypx9VUg4x( zi&B?E6!&ZXAA8+N&EidkgqwPSnOX=35wH#L5mE>LbSnS()&F^zfSX~t*aZ^^KB<GH za>mMTY{M4a;IeaM7ziT2k_^)y@vl*&-$m~~$ML}LY;p!jFQf(YsNpOo2w2#vC1=>) zb^#MlDQXjnrGjJ(!;nE`+*0jEiMu|K(X0Nd>@SGB596dh?1jk>PcO4v!y3N;OsDCe zrt^POfdBj&ZR_(A%A^-)*EluyU?^Jml(Fm(1nQ1~rMS<+E=8z|1U^Em$Hp8@%<}5g z3NDoLe>I4`l<uO@AgQh%vho3@N*gmyEr*3ED}`o%(?SK`SwLj*NVBu-rg|5ialO&e zjztM9j;iy#l$W=zrt+M6I1M3n%1-(Jonwkz{)pf$Z4|fO2m=?FTcX(;z&-We=+wd% z?@dMtid*mMmm-3F>1k=uYqZX)`t*_B{#x&uPvG81SiFpSV?}wlkAm9CX&NjzgS|yP zpGQ!<QF6hc9V}8Aj9%dfn}zg$D64tk7I4S9vxzr%LTn*SU3L1<ST1Ms-#NFslMEOk zjlJv$2a?=omsoWJ5fo$>D0!-z0TyU?b=44R_R5FFm=X6O!mY9nWsr)B(n1D5hOmSi zz5~N7x<?sr|HMc?#IAjq3mE_ST)b9tA)y`I=o$wT=f@sBlUbe1;bY~BF)qiqI-LUI zOBCxjhW)LxPHQ<&r2eOo0<yKG?}F-ri&Y+2Z(D&RH*b|qJ_fal#n$e|!`dxj0f#v& zRH(8gRS$i*s@{0^7i_{W));seVHP>kIJT=|^4ZbMiiaL<s{J~17xP3a_MUl0*-bcL zPJ;tlA00*hry{?|?znq1f1&#<c6|9AtnDx|C3F==;<{Bv%IURHiPv{s!mctE5!6&T z%ZA!OTH9<ATE4wg5vpdV{bia$$4&bvK0#z$K=CtO!}U5udtd4k<f>)CV0W*>KpyF; zGSS)q!+@ekLd|Cxi7kmlqcNq!Cr__5A9elId0l8}j_+|^AH5cvJ`LtgKS21aj*?$i z2ko`3G+5-~Ri$vJ#x~9^IZajF!Ll^gsDj7!iZ<M=T7b!aNBsXxUh|H|PYN>jsLK<y zzr|QKtPGmUm{EMeFh7hq+A62;G(+E@ryJC?Ys}G)jUMK)G8BR@Dj#U0RsGf3q0iCE zOzNASR_Gl(p#y)WRY>5<$A{)X=pJm0M)2yi$Sn|Rz|$|*JqZk;RUF83XBiaya*W`w ziPkrg8B;lNnFVt`*@;!>?W-4E*by!wi@z!5@^nHA%7CX&^qxLCyi~&TY9#nz1l|`6 z8RH5tK?6%2##F{{e%}Iyj<FhusiNxB#Zg#mDQujykXu3nA)l6%FPFr{%#4a*k5VgA z{bu$&Y$#7l9VdHm{1#lEMCuJq#VUr_e@o8%6tSk7K&C~J2Cz$jAv#j#A^!yVJ^M0? zuLIh>KkjJx3uraQ4cO$!fAfsPbH5c7Y|dD=Zpo`0;)s4jQEzl;fsGWka21@M;Uhsb zW*|?84bSur)KF>M(aKfOY(eQ1jYL5V{_G)LaVeL@de9>S+)I8{c28F{R9E_glj%+O zu!a)sh9ocVsC;b!sR=@Lj)57II`B0u>L*QI`yCZFpB<EO2anKPg?_FfW?4n@>n&<I zt^9?~ErD-*w>WwQI~)DHK=6J;X_eL3aDm}bI?my>S2b0GcGN=Mu)P|Xfx8oIcG&-V z=59ZzCEw-Y!XC_$%DXV6ZnSJTR$6Vj51WqT;w)Ph1hv1kQyecF8|M2rjt<NS<N1uw zX5(dK;oQxg;R!vdffa4LU+Fl#_;v-950k&h_!%^xH*LxVA7B3sdq@{)?9mUiBe`Dr zQAHOUFbjkEZ;sq_K+5u1&vqAWAYHFKi(6AX8H7Egg&Eg(Rp|Iq$&d_xqqDHNpD;Se z7^>b9ZR(FIMGf-t^2WA=YFj7jWT~mU9FM`;NCkTkB1)=}yYFUf5yY;kiR|{JiMQF$ z8jz>DB3URf{7%!xF6>OMVlc3UZ#JqmwSKJlAk007vXEqP?WM1KNC<kpM#i_n&+8_9 zGKlO7nk}9zbIDo#o#a-uX<0>_%Pk?4OxAmhg!{i7bMrAngJc?YZdFHOzSN+|3;8hL ziJ&tK&Y5$1oHCQdfqPNqpJ8=lKDbM?*Ty98WqDb*BqLD3etyOp^#6<CUpn3ic<~Sh zgn4lFM(8OLY<&Kmgf25J_)&sy5HXAZRy>*v&wu4H1{+ZWyHH6mUR12?Jb7g}Z3w2d z#U^p)24dsn@N9$j@){;Mg4KdZyk1&zEy`JeiK`vXsqH8?L-UHAZBX<}`Lmtj@cUVk z9B#qcf!V?j7&@!x<3UsDGs9t<zP&c8Eg@!4dBhCO&$fnm#Om_oE*+11gJ>W`k$PPN zQc#5YNvNS+mT=JqDsr6i8Aiw${{(lRythD;CsW^|XHHq#Pe^t=NL5b{T}PqlQIKF| zgXO9DMzyhch$*SyU?#*Ct*~x75j6CS<V|=pImySmT5yn_RZcQ3dZ27Faj3g0eJGz> zPWsWsuN-V>*Ijt3S`-v&HnFi6oZ9@A6)5rljibyDI;ewD#i9cCIYGi(3=$`UMqH}$ zoF0n|;^31qA@3Y2@2KJ(>vnG*x4>-f+{UTh2?UQ=vx0Ru1MkYTob|_fP~P<LCtnl@ zP9p`z;c3)1{T}Z#zg$Y$h|jvx8ppvXF{J|ntUe>q)LW^1cWHz7rIM_5><rJS!h?lR zMLgR<3q*2yqvFOPERNIM+;aJN?mH;LVkWJ}us@mKFlcKq?XT&xC>S<Q#*TNT@`m7X z$s#W!^UD#a9@+|WOs71Q=8~2O&Z-`CFQD?+mmbV~>+(ME?NyLT*T^9lh)WYpefIW6 zHKs14D(8v^S4Lt?{EP8&LJJ+v!cE9Gp+<Cj#3eRf=|Y#|<Sq+mh%2Y(L`s}h`q%7# z>-_Ep9ZkTL`r*bN#%Ri6wCqwBj3$Fq<&VxqMfZ?9*2AJ8CLE{d?Bb1Ic^!&WnAFM8 zM<BGYmM!4`A2_Lqsd_FYRvnBc>BQD<W=L=?7p4UxqmJ+epb7$e+2mW0y5-PKm~PXp zvi%8=*5>UpFUa;GWXdhLGVtyj(~J`QS!W=Kz|L8=a0&|Z_*^l#Kdn&YOoFEzii1p% z6F1{Js)tj}*VZa$KaW&JLwrxRAv`0CM*O$29h{quZGyd>TKq50^d|Pj?s^uL6xt)+ zi#g${YRz&8;{CAN)9-jP!h)Z5bzH)F)VxsI$gj5L=^gTsEd=vpKeau-yu=@HxC3)K z&9t}I_hNt?p&<KnxWzxKjep@y|B-C+mETE7TVM-)j865B3m9t#QOr3#&kPpTW9tkj zGz8+F`j*p19?cE)Qs|T>4<^IZPL_&cbrGr}&HiW$1Xz*yB-(LW+`4xj6l^(=*1SQF z)WJh*>O=didKIolSPrHMG@f<BdgI_J{$ns*&dD^i_q%e$9X2j#bKvZq!QQJ)R54ea zx@<3!$FE*?p^L)n7P?0C>2AGI5O6ed<FTAN<V;34x^}Y}7E!dJn*s*&W*)vRBqXEo zU;WxwcFTF)T{PNC^G~Nu0qxBU48hk!OtRcid(AuYSlO*Z5R!w%bA<ykA^Ji_l_d|y zzFXGHYhN_kXpz%??B8-)+EJkSZzLg+B6r&OGvZ_q_Ex~tnUKlj%<$_j$f#i6ceahB zNZ63u8=U9SF#``S%n-fzcZRNkwx&@PS-LyZcLO^T?PxZp*(My6;LzWAYnSTHvoc)~ z2rlvmDW965mEzQjy&Ta83Thz*h5PRsFp4+i#Og?$FxUQ=6nA5Yn=h3fQl|x}7YkiW zak{$x7c9^}I>iYA6VG7wJY5UVPcsB(4aNAYaFiW1gN9tV$|gKPtFdmJyOUa_O)3k* zVXiu*p^f<WEwg45l;g2TH8fuLl+&}d)dqR0U3Y|o-B6c~OQmY$^t03u&1dzJ|Ba-U z=Nfo-9l;aZP|8iUXJOopuHbPC$PXNL3j5<TSU{Qd3@s_H-e_$8KI@6$%nlI#0c%_w zfcbB8NRT)|72CHZo)QDeLcNwFHG0IEGzH>wsnW35=rSX);#9O_F`g0%IZX?OYf9m$ z8t4Z!>e>xY(PKI>wFkHr@baUjZGN6mzaZ}}ukCbUj-qmh5}#C^$>5%B4oSn7eq$=N z%Ly=XrS(V*R##P|%Kxd9o3Hi$b_WyX&Z*+cK|>x`B*w9rycEOM&c}L|ho_iwZBab) zjZP#ND%z#Ni<9c=X@1V$<z8<R{*AMshqc>gcsroGXuv&{&VN@iX};>-)<0Cxpw16J z<@dhAKnQg+FMn^c$3Ye->`=J{1iE%T+PFk0xI2Njy0n?nnga@}<FO_c@klF3SZ5UW z-8<Y)byRfw1YT5%Qw^Ehfqn6hI$H4T)oeYFbPN%)o~roD0GD*<+Jz4l`5A6aejoIu z2rH}?u29lTV`WRqXQ{gCiigwSAa$D&E~H{Tka<(^k(fLS6>M#187OPpv6*ns2r}jp z8iH<wOpVHAwnt)A)K89LU>ls9skY7Egcc3j8-p@E0k^WX{Eyn%a4_-9f92@IdtJgP z^fmEDy?+0H@BrIE81>^31-Pb<fU5nIZ0~X1!F>;CUj(snFi4QxK-~nUvLRJa79}vG za5gPudb<(kOA9~xc?xPbpH++^o|y7Vv#^u_VV+CwuJzcpRD%GGnh2yTE@zEE=!;8n zA8Ve)9;v}FG#N2Rd%Mr;ki~fAU2^!zW>ul*yWPxS)&(~o;&F2x&!KBmmS+uqKRr%{ z{KiNj5ViPJSB8)tviEUiIGaGDC1r8g52FfkM3}RA9#VUQd8Za8^=rVz!C>@M#6x~2 z@s^gdq4$trBe~jwjSC~QS#J40>)4lPNb{t!Q_ziF%CQY^UxQ7Lzm!f56rC3vdo;Dx zq$|jys9QRDh;8LrDeO8rxY{e&w-@_Qpdl!6J9GEOwcrDXXoN+OU`Ej<sYkLO9WCB$ zJcSZ1r76CiEyC0Ph;nD&Z=+{+SWBjJ^=*{%x`JN%)bGpq)Wk=j_#*fPtp5+_=jt`P z)#^&(T2jpne?Rc}$2YR=j|XD2DT%*1C&`-?{iAaWVV`Y*aSpmAwtQV3M;7gA?gWFe z-P@-}M~iV|5^tUjg#D;Se!i#qLGw0+Kil?%Q$xiT_jZMAGk~QdL{g7sENwW+uOZiu z#RaK-rBl9LJyBwyN@pc~cxzK@Eq-HJ@G(-*+)I)6W6Ib4@ka~SS+pCU$0J0&EF}l( zF%L(n_M+buDRpm?<%aLM?jZ+pf8#-AP>>S&pnp|5PBz_c73LcoC3`@I(ID5U%+KT0 zpmPN<kdbP!-)r>d$iwJOM$f*r@ND_5ui;CkZYUTsjox$XLT|wn+az*YN&_{1<>JhA zgSKoY2pe%g88QJyLuOl?kuZ+_xZ1_CvZ-Bd$O<m$)HLJPZmR)+NVpf8`FT(Mw9xbv zHEjw4hD^t<l|Da^*3z7egDK^91rNhPMyC}8Ox$wJnb<MP%dO!4?;v*;i*--G9B2NP zgdBA<mNLv^$l3Lx$&juhf_|~fkx$MTCh63h))`lJOu5GPyHe}skSo{@NrJ<SgHS8G zKgJ+iL5DeGKyKq#+=U9An$HaP*uodPHW)3V>cjTM`ngU@xW6{|rL+D^R^tS_3CCA; z29wwGET{A?Xq=<hRon%qXN2r^%6IY-&;tfD_>iS|^#$Nvo>&^rfNz&`M#dP)Ji2bR z^Wn8~@2)KXDiCYx-c*GvULl;@qTvd;uLk6B2;#U++z8lAyvfz4Y8QES;xO$$3nlvM zDiIBJA9VS<?CiACwKtJ)Y^2$+Fh%lOmQA84-OS$#R=buNjX%78G5?!-HMO*1yO79y zUBIVXv}X9Q_LRJ_(fsWG+mcFT#!Pm)Ifd5n#cc3)U0$^$bYw(pc9X#`VeBEE5bbf! z@yJ}cdO&6q=WHW}Ivw)w8s$HAUmf8O1s)rXUL4dChevP*<vTRpJ%77-tjb8C%&8%U zU2-yCOg``I5Hg0o1q~j77`4@{W|&g&k3u|te6A*Rw+yxifzsuY&5Ewu<cn)Q5~6%S z68A3ojnag~&#uvo9YCa!hAPN;=aWovmDGVuTHgYLGZ+hW;&H?0?5rZ@U;SpY2zOtq zG0sB5^+gnOFAfT)Xx@{1*(QFdi~P!LvjqQ>@&jRv%Xq(krX47Gt)+G`R^bgr=g+hB zn@`+#A9F3G<w2pnnsO04VV%Cvx4167WY2r3fZ|e%K=056nytWe<YCt=a);Mfkbm)} zc@VrMElbiq^680Jh)zbGj`>LziaD*U95t?#!uCgE39X%zgQc_R?CLAwv4GNzq#8u6 z#W8e!dJB~%bAl<QKbx1lz_H<^MI3BslHQn<+K;aKA~Plrov?ioDZNwCjQNR9-n>@r z)GkTBo~y!_^aJ4;Fy&e^qAV%rGK$>cmww23lsc}ux-Hw$xZYm3vLiz}M?KC}EG=uV z-7N3M)X6xZl+cgHSR}U|YT!fLZX2OLGFdR!%!URdDG_fqFn4T3S3kL#oW$r{fO9dt zT1(h}cQj^!^z#+{8%15cm)UJc0vCt*pidee#$n7z<1m}jxLPW9CH@;sj{gF*nHUet zDuo~Y_5^tYWyFUQ{4VG$q+W320AH`m$y#=h0h(2oSa{dKlM(Qpd-GEytVKyY(s<)j zVQD&mmtlHO|2VB0bS1w3nh?q6-0pj0KJP}q(NqLK(%pv~K%Cla<r!6!wMHl1C=2Sn z07A;vbDOCcCzYMuejbc0EQR(}K{))Cu){TGL`orKlK441AcQwgCuiynghdkKXND>y zp+<;wiZjD`rNV^hBW%G9&wA~oP=!M7KjN!3nw7+hLfJSaT1!F50`lSqwxZC0LT*si z?c<<B4v?h+_rg{%IArdp8i^*u$t4}UsF>7TvBkW5To{w!=7xXdCDgv&Tj3TrzA(O$ zh-Wpy48b`KQhfpew3qxhVi&&w%EYeV`xFQ<VJC}RsPkjeAFo6)=qgI+$ggM19xo*< z4jbr}>`(g)J-7Mvnc+!dPG3czyzt?S|17YK?Tj6Gb1b@*PDRnJ`FKU*r$5rwFKjp3 z63uAp0<t_=U|Rh4p0-fN-Nli2m83+PmOR~wP#Lxpq&ZKc?GoL)_HfLML7g+ucC&iQ zt!zj~q0J(A>vkA*!_xV${2SNFn++{zR6<$5G6fy0u+7}CS*{hsERo5l<h@uDE7`9o zDLr8e45A}i*b!!4n!0=6M5Q~8ECTJ=y*jwdsE1Qy@<mV0&7Tw@9BU5UsweDs^Hit> zZqD>99aR@-WnUMA2?jcHIc^4F%l;pmHHwFg((z+q=6AXs*upL%o2iRjXU3M^JHKeg z?2by(k?C(~yKq(!9Qbm;m62qr2Sd*HzZu~c<5Zfr73>K08>PNsf0&h@jlUJ&N4P^b zOdnpIMx51OKIbuR714G|A*dwx-bJn>A7z!B*@QUqnjeKfxcT#3?61jy5Y?uZQzB}y z0!ei_*>VpHhQpA0nMxt6fEVTHu6e)_F!$DT(kXAG+KI&05Gfa$Kv>I)D?Q(2affc| z4?cL@Q#z%}YH>^aXP0%t$%mGmOVpoCrA>#EL1xwaY?Mc0&}`JvR|$U%i$lzEE5obG z&RRwT;ED{P-fcIQV{H=7M2;<0XZHk`Y*?P|Azm4|ZcV-&;BaiGK{wC0MzrVaXT5pt zX<c2U@`WCzR!tKU1!-sZ)^<e#-l;V9Dl9JM+QB04Nv`<91{~(aa6qa-aD+I-ta;m@ z+0*uaB%Wy*Swej|n0iepTQ!z9{-_`%icRu|!n#*zJ#owfv=U7Um&b)xADp_!-a@_| zo{_kNN{!Qzo&Qj;9oTyfZ(VfQ=%xr0UvaWyi^in%yU2|j-w{_C)E&JAI2-fL#{E7b z`z5pOQ8?P8B2o6|<rOtwQwLw0(MRCWYT(kg54-QMeia;jwjC6X+z;G$k#W;n;9;vq zt=|h(_oyhT(?6>#RcOJxZ)5{=;iJ&+bR^*5<-`1M3?S}omuHkVuY6>4Z`^&Ds!8rU z`h+we>EtB8V#njhwFZ-|RS)@_dKt`93lqf4{mD{c^fl|F#C1(U7Vpa&{S430q11ji zmevzHVX75XrS1xo@2i%YPQkM&bz`kG?>@PNDngJZxrW)!+*aF!kg`z{H)$he^X?&& z%1R670@uTgn6c9p+hvxbRJVGkkTc3Gsyzg?o8WsqGyT<B*!OF9Pm}fenPT<OgDkEQ z3Zpk-cc!T46dZ_O)U|#q&62uRKJ0MA2|fbn_)I=tWs9u45Pu*MEanMz`4Mr84G6IH zkuTl<&Y*9`ohmNiVO4bS6q~UXP%396Im|(g{S`C};|k)Rx#c;K1e!X@>XdOvB0Pr# zk*q*@v$&f^OYiXeOls2E%YRZ-Pw+7SK#c{;CKyA}gVO*%bOdc=fH#rCt{@E_vtT0e zvBq)S#cH?(++SehS8as$!|oN%%AkBE_7_%W5}2lEJ_XMz6v%yvaGX9s!}bYpU|`$D zAJ=#p3qn(i-b@9lgEu`w(M~mLY5qnw?>UNsRD$%-jSb*p)2hWaXUnj=BAK3bz?22T zNP1$fR*mk1@Nd7CZeE?z&4+trAQ5|2WrbWxPvBC{4&>tdotQ$OrhVcu(ri++!Bjfg za9lP*M5vxD9{wM$-oziuxPAMdK_oksWn%0qWjD$)Mr4VoB+8nx#UR;|F-BCjL6$6I z3z4j27iLt9Jv(EUEo&I-jP05G_k6#v*Zutdh0k?e=XHFJ^L-v?{(W-r5zUSJR^ePU zUWxO6IX%|c^)O4S|CT@Po@kt0VQxl*nt{`wCVBZK9n5~~tyQnVNolB*t@}<d7w()y z|NkTwB<_5PCvqNC_R$w6BmRzUb`-x~eMlW8*qlwh>beOpDyjKqa5?*Fnc8<I>2bu) z3$gD5_oQ2~)4Ta8z9UcJwAxjqDYh1u$-GY&j;Hi^5^1ta$nXYE2e~M18^8u*$Rl;j zlsD7u?R9=!qP<n$uVg!X%FSr@51akTB5O6eD6sOMvU0p2rCA8fX9(pIf#v%Xj;4I4 z+yx1{Uu1{H6EK9HJPbkIyIWq*ec}{tYxko~^K?5U_>^KhII(9a+vLM0w9E9WZ_}JW zHEY~(_lzn-?G-8aWd~7t+Hd!&?+m^18>AIM%g}3~Q<H=KxfsZ|jWkI9rhU}SC-edC zcj&`ewodv!Kp#(t?G&9UW>JjTn#o%3*-Xu|gtH?8{Nvrr0X>&b@oUzb-y)=8HW$E? zeiQES`o&M6E>F}Bc4o+NMN^eDRCFe|`C6}OY?B36{YP`NamUrtZ$tTD(t9*0Z)*G- zfjN#frDFA4(TJbECV7nHRpD#pk2Yb|nnV5b&MG<*nnfUt^!o|nrNid7O*L+#PsVAG zf+s#;Lv{tej-o6kx5d)_g%(b!j;qL`;);7yQd6&g+90BCEEMhZsMuQ9V1dY)ngX?7 zArKasx>-Jy*k$DDP8K*6@rC6jL1TY$lyJ<-r}8;s?4qw)5MUv2&2mQd{mei{Dbayi zaBSa21bJRzxmnf6xBX+!XCw21LZ;QP{TG#4r@IBp5w}GQDt~k?ND2%yqc67gz==z> zgp&<w<?VO2A`uI4DIM0VJ1_Sx$udgY-?rmgd7im(X=1qd37~H?Xfv=HwPfp^0&;74 zJwuU}j0}ZY<7|poJn^aA49qgRwQb;;%jUJyZ`P<zsnPtW<f#MvLDO)WYw0Kvy<PEx z5j2!{H?XnOjs2->;TIch(eU%*axR$K(IDG3k53k!O?#^q6W-wz=DKMKg(GhHehVEB zCqx%{n)2J9_@B3lnjdw;&<0vd6mW}{PyQB|;X6_|Djj+*yEUK%vp0#IIbQmH1239i zb#mt@m{!R~cW!7jY+Djsj_QyGfqOifL!L_^CMa@(^6X5EA~}_Qa-ou@)(K9$!NZ~A zLmck*I$8ds5+1YE7Afe%wv|#m0+HDcs_d@ykd$*;%J83-M>PC-4;l&64$O%6tr`6T zZQ!~e?NZirY)yNxYq#DudRY@$Rhpj0I)EhHb$<(MNIky7Uu@)&^N_F%_A8I&3{-qy z4_v1mxqH*U)!&W@cZ$^}qtOia&z?)WF{ntBb8Wodyt)uHDZbSoi>k7jUraOQKIpc* zoxgC2hopX>TPkm};fvqN*c{p+){QW(&n-!=IlDo<Eo5z<BnRAQ4eqj>Rv@N4b*~;? zJuxEM9wj81+uWwTYoFc_-bHVkE!F8975laZ&KqB`rj7cq)i+EDy%E8z$o%%7|KNZZ zY{z7TURvD7af#^;(debn7R@6>VE?tAHk#F{;LDt|8MY=mqVeta@U36JX}%Y?d|P?= zm;K1s6GA<3)gYB$#ytkQI_ai91}HA6*~VmAVv(5BSfPlFMum5Ekm%*f-1<|EUmyjq zAJ*8437z@pLOV}S@TcKkG(hX{bb2faY8D`EwzwX&y{40^_aW2EutkrfZ2aM{vgvRP z<+^op*=m6QmtTzydfiq*D%MF5sLUL=R@wLbhOy4{zP{Q1HWv$<8@b2Sx}hbLOL|if zr*Ag1p7dLBoey_sFW{_8>|gM=LempH^BRlSIxR|SI>#OWom{tl!VP;&D5&Aq*e&xf zGdX5KJ5LsSADw7gJN}{+ZPsX%eJk>g>?<Tbtkp#xhMSj-j@;XHEUGDew4%0I<epMG z;NKG}%HEVOqH`(4rSVTlxI__h&;z`^{p%a*cJ^x7x27EqH=&@0M|%_MztUb66pin8 zhzV3wPtE+lYFQ+Q>%8YqmQ*a_kP`!W*UY6OJiw@h=44<6w8wc`%ej>2p#1%5%d7uF zqMHST7nq=1A#dRO!Dk1-d)JukL}!`KJ6AGR0kW@&g#Z~+Fhzv+FaJ?o+nEQA1=AMv z+aYm~pmEnwwrEo!LC&z%c<`-v>UZn_nj|_AI@s|aj$jMVg^#Z1Ik5+{OZDICDRg|e z1;|_{iY-q8LEY(pxsmV^arA=&RnAy}b_rMX2fATvq(RJsP)uqpy=f)^GE|S#YZMho zVnZ`Zj`yw3sQ<YJ{GRd5ZWKcL6nm!gej!(UA_!sHYXi9FKI4RSUu8SyKy8HzgxNV~ z*RnNKaF+agI}&3OOdZ_I7#(L~wF6821B4^y1Q?>BT8DKLsi)_8_HY_<S33koC2xo6 z|G^$#{=+oseNP<xWmU$0|EX8&CAZ*%jD1HC&#avx<THHY3G6O9_d8$`d_-!qQA-MI zw+!5<wjI1}QV8?{#6W9JVg_caSAVzdRhaqT=rB>CRQ;N<iDlahUK;c#FH0>XdJRYl zEGoww<|+(IHQ@B{$j8ssY*(*pRsaqY{W@ECr_W9gMo-RdL6#E#J_D3_H7&#&ju6(? z9FwhU@K3W1m9EcpI0kjpgdeOFWa0N~#lyn!3?5=@W0QP+!N$sxKS~HKvL_NAK>wkh z4|^rs8}EUqEH`)$xE)Ng^^b<Ju}{{anbJFpx!q*1ukOuVp2QCpOvw4#<GYQy(%9q8 zvr|v&59DR)$5>?E?<3kntQ#5T;||5F(e6Lqtmqic7hO3~GS4Ph4k3d^io>XDelaXY z=_G~Q5xRSYS!cL+Snz*Ll>-hCy*rnP$K%ggDY4CSdd8VGaE3o!or)fqgnuMQ=jP9w zL7&W_i+~}n@Fkf&|1t@pA@q$g-Nh(v&7-m2)!rpG10kp$hvVY>Ok!%*iN|s6-fq(- znJaGUbc?W0&G>C?3kGu;R<$BUgc=Np5!`SsE5K!}$sSNJTEe^3!ZR8KUQ*kM3uyHx z7S)KLNT1H=lNJ?am%Rg~x)Y_Qt7neIjU&(LPa>ZI;~(<d9{%KY4dh?|jJ6rCM`!^Y zn(=&LEs*3WxW4{|y>~(EayZ|ESX13qKY9kabg2Hn5V}5lZcRV>PYSJOUfSvmjkTkk zol54N2exN>w%eL#JDPg}SK)Ph32t9(a&D~M94+G5c4EtwZ6VkE9*IF@6IyS%UGal8 zirV#zj@+7)&hmY0G&%a$M}Hz}drt7@PwLl_GT*m6hAV%4Dwp~{tiQ1SQG?W7zYCM3 zq~(bZJWE{?-0&MPw5r*MF`oVNClnnS5nL;wm)xw&nv8|92XUoTo`8JJ2!?9YAT<iV zC8g5eu2K?JqP6D>eKImBaT$~~vN~6m@=3lJWZ5cwR_RZ?8Qg$_z-?_3aHF0pCj9e< z8K?@O8mA{uN`sdNB#x|euPplRb<M2&@vM1%r|p58#nHL7bNEkExK`FRK^5aIPv5ke zn!=HlV-o34@{-mmV)MB*0<*CqKX!R^rPAtG_}Iv`k@XQz54mNc&S(W#W1(v=66bLv zQV*|~RS^-;Lv2YjJl3*Vf*MRwpIhxeBAlSHn&uYI-;H11cPG}}u4x*19_;n=RzAIb zrJ&~F7pzD$*XTC0ssmv$^Xh?T{0zz2Dt0TMfb?*ZuT9--xpN{Rv@EsrZZ93zHw_wi zr}0X51p4LKf2i`%OZ`K+3bIYI_O&!K9RXGqwy$pzyF4yt|L6Kmvml#Fsz=wn=ziT* zTH6ilHvW;1%WpG*>x1=_x`+4PiP)#YQcJI_@K`~<?a%~K<A&Szy@p~V8aL(?^>nXz z-#29^uZ6AdIJM~@bHB|Gtzi@1TYHIm9(0M0Xo}87nF)%Oy%5?+%SmVX7jf8C&xHjK zE;s&!VuXrVV@b1dOL;kZUzXJPOyk}Q#YQ$dgjP#un3tAXm)<LSGK!kZ{>CPyoBpW! z&u3@svCfGx3u?PAAO9zL@9gzMO^rjJotSoTCO7Vh+2}x0jORq9LD2^9b`l(G`mR=U zg!8lYHl8T`4LVWp+MBPZ!tjq=R+EI!Dnjf|REo+pHi&pvmXtQk_lKJZUfTY*n)W<d z=ZftLZfcARdu2E=chh`Q#l9(24262rGL|criS#lGyzfo}9RHWxQ)&s<^}p;Np$jsc z8S5JZj9sB41N>oHQfY|y%N)&H*fXX<cF?Un!2ynr55hoQv98H&1e`Gpuuoxsd43l7 zENIm<>)Rf~^)VxQ>5wOo4C6Nd1%yo!n^i>zBYC1(GyxC|yPs_aHD(cZo@a0I&$yq- zR=^{nF9u7+R5Zp@xlCZLFdj9)Q6dYnOD_Rfj4psoHEYApYotGW+0=-ZqgLFW?OI9~ z+T9UUD;iu9D0iwvgchHX?t&6rhHL?fpx1c?ez8It->3PUJA2go+nVXAbP}BS(nb1z zeotx_7(|cMtm$s#%Ytwq1*XE!8LNz8^oMBvW@gm|qB>Zs2rup%4;XN=lb^WyOHxpT zfpx^DIg>g~Q?SEmnIjiP^xI9zs~baCUbQ-9Oi@_DfXk~q8}zRRx%4|2k;yU!R}IFQ z!n_0+Q-1tDmIk6={73YHD?bW=FWTEuoipoCZ-Ut_#Jy}&d1QIw^H*C;H~Q;F*1{)( z&)-3m-ZYRj)^*^{Z8TOodO2WGONK6PLPxAQEC{dxlbpYh9aT5T#H=H2+K}P$r`Bhv zz(c0JjX6Ifo}q`~v2HU+>U>}}k7k1HiO*2mY2<AH2_LQtrAq{60`Ztpe&=*A)*Shc zbHYtLhRyZ`lxSW&eO=;A4~s!NPLT|9DcQ6xBLFZka5FDi4bGpfPauoTzYP}Qj6U~f z*?f_DLz1q1VLq5xRdJY)MEp_R7X6`oDk`}Q-!`25<4KM(Y`zLWO;0|*5|tXRV-QG- z(y+@U;OMdC8FfkUKblD{4u;+t^(=jzOx%Z;=*J$<h~(~ZVOQH2#6#u-;B16GS)uxG z;OGP-t4~drm9TC{TkOYA_Nulo2esq>lr9Ea&qy)X7_Rzdc)_DPvep=X#wysja@alw z2g4?BQCy?@O*4_vS5A7sL=U>Q&c%f0XM`xJal2@T$&YPycC|Jtn$d>Kp2y*_bCD!5 zT@W0vNW^fcF7#ZV3k%S@()UFEF{`#v<eEO~#y<A-qVDCBbd`IlikM6DSdMUxIX=1R z@4rZcDs%khlau+-f*D$r_i$zK_)nn`(CVd~RvSM@MbtTcMm%VOijCdAIZMFZ)RhWA zWLCY`x)W*1uSQLX4)rkN^gm3ge&?%$b$R@$#`0H0|1p+84zU7>zn&hwbr*Uo!o@~K zHo||zb!kPWzn!TX)a*Hx6H6kcv=62)l_58ay+Vq$`?e=;bg2S9ItLMJsB(K>0^gpD zJ6Y9x>8v|g26WGdzgfsdDd!Kj;g2u;dE5Ah={RP3Nwa_$N4LbGI`AuvIO&R6Rr4KC z_Oy9epe&i?<m`Rn(I?IQzlJ>8`3yN9WkKe1iMQ;#mn|JKPZ#u-T;22tYqmjoGXqYi zWbac%Ot&(|qHTm~JcBi<PBwWhp0AvyoSfp$maw^c(fq`zc-d`D6&`}O#}*VG8k&n5 zN30;5-~6yDolXxPdqVizz}h24<{O*~_eyf__gmcV`O)3A+Hbt(gJS{8?uRRnR3rQ9 zF8U9|_MFY-?ZN)}R+%E=W_rvJx*TsJPDaL&=2_d$+am%+vN`?L2=|lcnqssyzmUo# zFY)|T)8!<c+!P}M(*3=1;Vltp`)KkOmumkMGRJ+NzA0Sy<OP%y^s#fP#pAd=pxYVs zR$Tc2ryNwAT=K)JOLkd0)1vJ~!rFH@7SrN0+K~EWy&~8s-a4@7`FbbE4pG9d%DABN z&G3&*`@eN_um0m^MH}j{w@d_?=vw)Cwsff6y@I;mvelJ~$}4F5`sR4W@jzLM@?G5e z!X4HQzikb!wi%L@|6w)F(h~4G3GHd1Zz|db`|Cst2=1w|XZ)_D))i5azpc6c&Io!P z@bny5YyF#|y0kpa#p<Pf&&P6TtsF)Ddtv<NMVpAZ%4X&jUp;Nf7{lPr^r7Dgo)Ndm z-@|`>Thp87^dfF>si+S%`YRX^scFH=o6x+2m0D#fu{E-7?||v@#>J}xJOLd~5H|(= z;XCb5YmnQvLCK5q`H02-a^n;y%h)D-K=;aWn~g-6<vC%P+p8+-N#ZwKkHX!c-Fnxu zEep75o>&Fx=$1z0-jex@*=?CQAJ@45!U_t}E;<Yn*;)2GP6-_1m4&_-1;jd99)b%U zQ<``@);_UI$Fbv+L!OLae_i`|zhRIrpsh}=lNKZz0<ZuDViknB<1&OG-JMJ}x46zN z<NkErFJ&uGvvaTcm;_+MdfV|-bTLl6zrwdzd0+l5+gIkV*($Ih#&`_``{MnahbSp& zYx=9_u8*VXrvS!~XxQU`>Mkdtt!5O2-07n_XaGC=n{n{(D^nUBdlmEZIiIW=@qQN` zcDr%|UsXD}91LW0!{1%<DgB}LKaow+lQ9Idy3Q8B1d#(|90w?EjWJFiz}P`ZsjHj@ zKC^l-dR0+BMBfU>>yNv*S3BbLTb47%DpNknKZr!1L3UWVAK(HY6mS*)gB<`2l9B}h zEf>w4_ocyh!2p@S?<~)b(!XvAHd~%aod4@738DK5xMZEMzlFsQ_?@gUF!V*UuXn{L zcFSp(WT_(`f`HG?n>cPi52i@<h4&;fKbUPhR@<o;dv*p;A?GnQ^Xf;E;A#OB?yYw- zBB_U$Y)vJz9@SO=)W=W&edx2n6s8NJWbgEjHr`!&tmGJ}CV_!Lhg}ptRxijAt%nAI zcQu5ggo3y*WIn@V<sd-#jHEzP&p!J4sV8DZx`l35*V;Z5mv-tAO|i~E%Q-HqG_&_H zZ4pg&d|1(aq?ZDnb#^LHHYM$EkrK^HFiAO$w){NzITnq02gMO|95qc)0<HyxK%CH~ zK4uWRNId1AYLLe<fVlG9O5#wQBd&WB8D|S(dFO&Sf8MK=4<Hn2ZLdq^nrac<zhv7j zrSCY;9g$5Sfy-Hpl%<64b_W8qXW{XK$CHhYLW_SrJRanWLq%wI*WVV*DfQg#56ikr z<VBgMXeA|?7G9J`@mipJ`vC_%xhJ^!QFHOJzNEWSExB5pf2D(?RfbAs4KsOZ8?E>o z&_sN3Fw|QY76jZ{Z8vop|D}lm>*RVBYwmJ!M{=k+rfdSAWTE0_3{<`A{iL*9g?JSc zZ}N-H@y)T>6_rp2>da&{hFVp|(n?xvyu(56-?QJCTR9tncI*?^Ds8!>knIA1A73_B zd8IJVcMPB-*0<-9UV9+cvhdR%MlQL@%d;)tWZ%(r|Hq9PYqKAJ=2l{MP{#MmY2<DD zvbxzwGM^MOIjhUlN<^45&^qU#^?Dg$$9jzVz=%hCuNHba`9zoF<i$qLdQ}rOuEMw@ z$`h$&O+gKJ7kGH}jo*7}UkDjY1xxjR?LmglxYX)lK;YX<QibAR+k0Shshs}%X6<ay zuv>EZ#f$0`G!4Hq_6`GO9qW2}y?oS_ct?qLWL1xr;p)O3Eswn(+oq0kkq}wl@1b(g zNr1DBaWI#2Nm0wQ`3ZqmRe4Y2ZvW=U4*}l>zCW-9$&U2c7m>d9S{DgXmpWjoscjg? zh|<-^wG^G;O2uHE-6+gzUhEk~(He7;(sB;!t|^kw;SgO=&f8#7Ne<e!;Gb*s<*dL8 zj`axWslUn%_5Hl}c-jb#`5?z@IVNf#vtJ&JbMo`f@N(}yoI&OC_Ui}3b#EwwBP<c3 zshS%c#rJpmcMX&xQ%Cmf>-XY)i<Jw5QYxsTBgMR}Y>=uEEn<HeZ-2WkHNyHZvHCN< zFKf<kpP8$+6}aZCN?Kd!aDP*QNpbIy`PBDxTK(FLkgY*<l$t#{rK?O4LehTfJ^pvh zb5Q%lKH__;-kPs(Wdvj?V(mX;K>@jMk6*cRiY%XmhRaB83Ze1?cJpWA5OBTe(YyTZ zC2g5*ov#<hmL7hV#Sg@pg)6os4N3MRQPeL>+v&!V`(_{7ilNFe$kX6OTMWC;lQZpJ zB?p1GZ1SY0{F~$E6SMD@p1Hrg8D#s$5%G3_KqI()-D|ytMZp#2H3y2A5{Vj6cuB<f zwHm03ljX{MZ7E*7Lzh}n@P^c?yn!m3=(f@5Od0?0&cbyGfe5xdHnq4**)O~u20e2# zvE-1*3imX`%i-9IUcBz{(daAwS0@f9vVp!k+er`ti~*g%R><a%x!N@-)yedaBrJL! z{f*Xfh(uPVR#K|HbOhcserq^(|F#okK0dElZewGY%$92!qp#vPAxD>vI0Q05Hrx5p zB7H_pomKMH*Q&LCCtv}9g|Hq8^aYni@yl^ME_U>#DC`T+lQ1Sw9o!cCQ3sL(_$@9F za?c<JShp(+FiLGV$^NkaS8ybNJ8Y^t!5Wy@{A2@WQyz+`-{kZKfP(1c09A)jFtEr3 z@NHFR@{paR#j!^ZS!S>5YUm1~568KVmjx-9^ooanI5vaA&hasj7>4K&dz!Q1P}nLm zPY4FY(LZK#Dd%}<cDBOP%tF}8K+(VTKgmD=q&5c6&hIm@3<f8_=G?+zHKsCl+P2qK zm?hv*8I*dj1)Le+c{WENKss6sT1ZfT#Ol)RME!Y<Gw-q>n%&3Wo=*7Z^f6mU+bAc& zKxvgnJc=6U2UDn_7spAz3}*vA2<HS`pu@_#@DLW)V9=m}b2o-XoXQgvxUSi2k1H~} z3-e-CMJ_yEjdDrUL(Q9Au`i=(%70WLibpXYVcZE(Q6TT!4XCd41l}VS<{85j19zD1 z9zb=Y2_WyeG<hkZZGJyfj;51dUf{&>Pm9O3C`UJF^IvofKnkKCEWZzkBLahAqHFLP zIaEaB?r!jE=Z9#w`H)hiu$BT;GNuBzwredSfc5&efskU4us#oPNIK;)<qk{cAbqf) zzZJ<6kY*r+7JXpV?Oo!PtDdd!NSmJ+f(!r36FPYnK0Ra}kTYy+Cy(`Mdt^=(6X4Q7 zc3WhNZ6(Ii{o;~Y;*kHfNBjs$l>(57Oz80|cEugi)BMiOu4g;$HQAN*7J|fRoSckY zT{0j2Qs8MSUN}=&7uafCfwP_F6gOU2k%zM8nZ2)%7sNcZyP+n)<l|L>>~!|VwX2^G z)4k$&<3S8^{3bNAhgg=7w8SWtxzapMmh?(BP>X+k3Em=%tD-HTj%=25j+)X}!Hbth z_1n9~v&Hw4(|DHY4|}os-}jE7L0S!%Ag_0V-KldX$ut7Lt}_#HQ<r*U35#7UC{qvm z2p&gZ97{XpLy&8Cm1>S!e0OILj)&9N4bC5TZ?4YU5~KFCr?1>0-6f=KF6%pc4PK%B zP7H$o@)?5`sRNC=nr;wzb3t-{Y|k!1Vfo{Z&P4LEq)dubi;p<EzVFIkRqsN|NQxY| zy^J;Q@||}aVkZ+)SyD+x9XevqopGeixi7dIHA`xl2@6RkngYEeX5$X-uBU%ZMyg7^ z;k=p^vN#j2q8*>lNUB!j;HJJ}kkmB46tUz`*dx-QbkV$2&*J+NR<47?@0;BR_9@6+ zMURp(a!c;34eRcMk;~-8gqz*k^i8S4cY`ZF-^#)G$jw;nL<&-5bR3s*7r9mfc5<un zUhpzF=KAy!TorTDNa;B>v^lyk|C?GVn}gGEI(%B{O%{w?t$^O6Q;N_HyuIccNPol) zA3yMztuk4S_rHc$6x#c|9dUfg<P9x`J#{ZdBXp+qyOr28=US72C1@8u+Gd|;m$!f` zW+bHk_kzl4?Pc%#|I|!L+G>=d{bl;2r$N8YiaEM@J)X^Kk5nJupRv#``p}zFi#ey0 zi?wG|ocaGI;AFY9R(H=-SB5v4MB&ymYX`sm^U5|G6ylJn&Q@=BB|I0#Lzah<qs>ME zt8QJOdVDe-o-{}aa{X`cxG=?~%8MMsoDF*mpzUfZRnZSPIJ9ehzfMDAkQ+@R06pw8 zn5f*vSqGl;*%g1hnZ9Dwtxp&k005C{M4da8=5jz|m9+1C2ML8g=Wz{<DXt8tD%8#t zIgSILXK<XjQ5n%w&?8!LF2!YVefMKqyy}l+|Laoy=U>M6N>1rcFb-u5?v_XB0{Z%i z(?sy|{3%EkfLS<=O~V=y6Jp1`l~5WUs$SR%-Stxtj?jhRJ~Le;&3^l(3u1nEtP}6j zpblA~PBZe^oj-pr2LMB~XpAtCx}4TGRaLJt$pQKt%t0Pl?+pMQ5Q*eCuRBD#TKQp< z6g%a<zXG$}fUD?;WecB#bI?WY*8A@Qd65PG)O`z$><HwK%Eups0>0D70qPu00ss=t zu%OngIQVgd#>8pq*%<%fpJ&<hAfFJ9Gy3b%%qRKo)I?djRPx0b)sV;S%ro?AW1oUZ zgB?=vt0wut-vDpwMf9~SH7wBZn5_=u;J`Fn;|HAo$PF+%P1O;TySSWMpha%6gOVYh zR{(h{hO-*M0MfSE8#>kH7TfP$T2sg8>NW%|j=!?#I;O*8Ps4R73Yu~qq-AalqSO3P zL$eplc*yu5s_F0>s{NqOaf~}#!9)WvQZ=?BbNbbz#vUvsi@6=dddO{h$f71adh1Dy z+Zf58`d$#&O<UmPhj7q7^+c{|N@Z4Q2gYA?SzA?`k3Z|oioJ}WA9cre*oG5F?`BYq z&z4o-?XMR<glhxjXFr{g^H^;MV||QsI{4&2#Sh6dshS}WDT)j9BXmNTJ#kcG7vr%G zIIS{q6eH2(AWS=Lg$P~8oipU$R9vfnHB<kpcRVy8{wnYd&Ol@7l6TWK$K+$EAYgG^ z#JJ-Z`xH)%vmcCj7gjzpdp4I}{ejQTCWk|Agzf5*j9)>m+X~Hn5T;Anvu;3)Vx@QA zkz6xheh)wSKHvPB$rZE~Aj;qFraowLwrxEc)la6b`OiO_YDVHn)%rBLv+(qQ;3Xrp zrMOpryQAma$ZZc+`6t!Z>lf#HzIbfOd5j5ce~dl(w0nD|YkNlMC{H%$p>3+|5IhEE zoH9ywy1LCL;yUfp_PMlbKDu$K$wmi$Z%G+d4DRWjtMp~GtcF+i)}zE$<6~_`>_&v` z(|qIWZW4Dk#Hp3(#e?c@80i>nS5L|95ydj7$7pIiA(g(BHAtH&i;^sl!2Hb@cDr~Q zNqSJTrfarlblPUZm2>>nvaRLQwG5YCf`#a1OYin9o!r-BQ7+v)g(G4C*VW)3_n$@W z>@@Prn8-#Ak3<r6rlajPh5tP@-vaJcN9dl3o!bj2{i`6^r-#TXQ?+&mi|KChQ~L&~ z<LSa#XVnkWc`O5`W+9xP%S>C5;}#WXf80izZs#xmURt^mC=3S}WEN}}->>Rde~uVe zwD?dN;hS?S>bOTQK_Ij#oLI9!QUs@|6qmKC*6<D{peuPRbniA7YNJlVU3hjVQP2%H zld%~8{rf>0s!3pB#*$6W{E?>~bzJjfzYeHT#y)+U>tedAvVrGusZA}nQQtO6W>)=- zj!Ox;kCr7zEWN||srso)c6(7rOB!Ck&gOUte(?Ni9H~c!8j@w1un?#-6HfS_G4HdE z|F)4Y@nD$i>IMfS$CT?o3*HF-j2I<F)1#JiYygdzLepof9eVzBRv{rm2F&`oQEU>~ zZMx&Spp``bo$Hz^9+VjISwP<p;?PoWgqHaG`z7~Pgs&NZv7Xs<cZm6Yx?E#_W1jo> ziAsG0p*j5ECIRkOrWP+S=nwpqyeRyc6?>?z>_}f|lns`9oVuzveh3z3>VBE_U;_9F zO=KtpN=!4KYj@3ZVJH<TIA+uVit3L?EYrVn6?>@za(CLnz0tp@IuLq4NY9)Ckc2UP zAA3EKH`tYbi`f)pFMKsnOkgY#LxR~ZikXFk0qz97O}UrDbt`$+mwQ4#UtFM?Un?S9 zw+?WF#CHBC2b#%P69ChrHPuFe4iBZ8rA^+JbM<j^($x)pa0lxel}Eg)?yBA}eX~M9 z_E>zbJ?^BlXoxu@TkdzMx2xNkXI>dTK(MtlLKZSs+chb>Omi@`v5d{@hG)RrWO-BO z3g9!&uN!&=*AsdGL(LEDn6}Qnv{zpN<LS2m52O`4Zo4aukc0|9HQUl6+I)a9TSZ9q z=PG`AtYIz)Lt7m&O&CKIFAOEj4U3xw{1DHj@Nbo{K_$G{kIyyYq#LS2ZU!aC66t)L z8!+BMowZ2F^kj54`@G>9U*K`1NmLo^w|IJD?_bP|4KLBqV17Lx0HXRNjv)H<5J-4b zdTRdx4<OK{yKk;822$DirHuK@&I{Jud-PC&cN>mN_Wb^-WGuZ`WP3LYd&z1|mwj6~ zLRX$ge+(LwZRI5&K@7<>;yd3@g7#l6O=TMX2z66Wi#@E`PTUiaMUGT2iKAp>T#_w3 z5p=;H-<00h95CKkSyg!8)pY2Jx?BV-Jx@PBlAgN2KGh<R!=HHEpVu;)otew2cx2_) zgVwp!78gJ?&V1HUQ#&eS?BS|rE63gncQ>|g?zy9{llSTNWFaKB7BVwvs@5XB#=5x0 zvq*ou7aQ>C-rSvlC9Sn9JMiBp9v3Pky(*2MlGxsI48;&i2{N9=n}egHTo|5M5`<UD z8r`dZ!0oyw3=hBxd75WeuJ(U5AzQc2Roxmv{k{4ynM}sjQ<@0vT5jB4NhK7zrXKhR zbd~YgS-rcWip4AYKK0^?TTBlUA(e%?_XvNJoVvrwH6p+*SR7>CmK7awvxbyov%~-4 zwyrf+CB2X9^NfwA(e=-5iAG9fUau!euN@<9%t-19O&KMeT*#@o|9wqQUQ5%lSodGl zFtwL@uPNTQ{!MG^F4Nu?D0n#6cz(R@KDeJ9>fYmkf1sJu|Hdq6SAuMrkSf+K81QCv zkH42u%qQ_Mem~+5LOAA`VdTlvSey*|xZ$?7ckEul;Omis@Y|?snF1#Jj3c2-a;RSa z=!b3sZp--Te$wUd5yADNoYJicNW|Bf1Q#WzW2f}UqjC7u65~yH#OeGuX-6xa+(SgK zlV%RM)y^#_XifK_%DeHSLN&0O%<qPPt68_~PA~aSwSV=*NVwg_JFw3cjgJnSX}T5p z=AUvABx-6mcQQzJG7oJ2VaeA7+8<EMzd(Buj{Y~^;|DEy<$qH%1|Hv2q9PKQh3S5d z`^3ghLv|qttBe;&eG~d)2q2gHgQ+Rk9gjheUv9tBb>JR%RvZ+Ub~73Ry~Vm~(9P<~ zI-Xc>&>hfi@f5a9YSkrSrd_>hz%SCBiZN}W??+$}#>9$%ywIMTcAO+9c&1LS!Xzfm zj$ZT*Hckjaf$Rg$kJil(>!!#+MgeRL37A{ii8h^&IPTDu$Ju52jo)xH<i?-|g|gX$ zyuCYx0GA=>cIzkjzRn9k?Y!BV8N$EmorLQy=)D!~90tQgV9SsRglVl{o&7JiEmNtj z`&j_BBxmnTfbz?BT~Nxm9tu>jV1N_TK8i#LH1pj`j5WUd(<KW6Yy*CwUb;H#@jAOy zm8;q~6~K>XKx!W?r-6J`MGLvne^i)T`g!cK2y2Qko%jb~LMsGH-MwJ0S1^fYMKwCm zj(9qhTzB<5t!g{`^)#0FetCsO*a=MGv~HK12X-(#{u2zd)VaJB%Hezi$hQpwXvC-9 z1U4hpOcKc2%|$i*CIR1=XXz5p<?M`AeNZ4-fhO#VRja{VX$9~1SO7J$sSk9-j>JrN z1!ol5qesP~;<B0pTPpXz7_V|SAPa)g-Y|rYPD3yS@+xFw0{+{!fHN=Ja89J}>P$hx z<~|qGu7oYmCMO;le{*c;Y`yiv9&>YMmd_8PZ7m9!ul6{v#LfId|Kazqu_fKI{Gsu7 z_u}<$Xg^B<)2<U_&;;OxIOg^vR4{P(n8)HeIPS9CRg}2Caia9^?dpj8N&_AD1>JMF zxc)7jgC_5ZCda#6JX`jVd62yCIA_>SC;uE7VhmBp?+NtS<-~rpS(1cbr){t8V_Kes zPA$B#t4NTuFUG#GN_LFW`%gFUuPAQ#IO8;HV}x;ZIhaCuk5`!y-ii)bW88_NbTrZ* zUXpEn;GG*Vln!TC`3o#=weo%rHTxkT;c3Civ)se6s}bzggr5(lUADuOS)0bvn|jnx zCJOQF_G!nfD^&WGD3`nhzo<R5+vs5H0ma6ALoS${YU5bzNK=cJMQz7K#t|WEMpn%{ zz88(ceJZ25b2Eb&e1NujjI&Cw1@G$|ZRlJ^xz}|14}$o<D-SPH3wmPL<c7)V;Ws}S zu3dWI4mS}GrznGQ-=N+Rr~L=|Qrpi2*P9``JD@CtCcEFGjznXnfv#I};J<@f(R;Km zrHX1zgl<r}%2kJA-udE&#1~f_EAY4{RYX@F47Kh*LNF>s-_=X_q|FvL#>R~NFg-$C za$Rr=Z2I9((d9;a4lz~rsHYVuVb0B`axAG0mB_Zrd7o>r`1Q=~g;e18--rI%5z3MB z9jEQj)_6A!{uWV4k6P5RAZ=P*ECjzA6q5Gb8rtON9jDbE_fmQMURlLX9yLr<V5~i} z_lh&7&15&?^p5(o+BEGdO;01vvX9}V5+ihTWgxA?uh1bz)$zc+yGr?*5tad_w95Ey z>2lz`9jwMD)N?gxi=@Zve$A2WeO+X-U|U=8{|qaiJ^rsT=Cljpx}a8uUB$6Rk&twm za`_&;Q+>}ow$(d~fD33tW(7J)v@ay^Jde0#rp^jCX4S`Blan6pD8_16USIjK3V3dO zOkDIDng@~tsQ?HAOrcKg#7QF8=jZgS>I{6h1ecTq0dXEaiQx7>Rbj`FFR*ZNfuI65 z1M;HSEnrx?vTr|!98)x%%?dpQ7}d-v{?0AYkhcb=bk%%@bYmiY{3<<D4aq)AK)?JW zqXRt%krz@xjhX(p<g3b)Z8ylGukGquriR!Hr2?foP5rnrJ~bWt`UY4(kUMp}Q4AMn z6E2DM^<5Qf;k#p))tAhUY%_S!EQpRd0kZ0=phbjjZ7Z>F;w&KJ?a<i|4vN1$nt{TS zGyRu0hA%h!zP%oh!{x;!_lMrc>n7}yyaU|iFT-3kn;ne%6^w4Q`klt1>|9%2AG<2O z5&FE(5|eQ@-Ge<fPTqj_f)MFu5eMEJ`57LDq4-|-r?@Yrce_Qd06CCpU{$Y{Sc*np zW-myplkxoy82T+M?jyxx5V<Zi!}w|N<J7L&5dckd=aR*v$kt4+y9QNtGIr%YadtM> z?Qb?Y+~L9?dfp$WG;v}mr7~|M`rAh-FXOF<JXuU6F^*^olJ^-Ap5sZ<TIaNlmL}}@ z&xg5>aW!LhA8=$bWs5;}L+XI*dem@J4^hA^(br^LG+JFJ{?w5p+j%we!DC4cN=`3+ zZFM~9juKvm85OBTe2qPZ2cP^r6>4x}9h3)WT>rgwe>A+`<Kpc6l8mG1rS7Bvs*$yU zQjN*Qcmo5aZ(=6ag>YKTSWUS1T&h=+!*}te$cSSP<PwrHENk;URqObU>SE&;%?hhM zeimYlSBe7~l2)6M&FkO&;f3KFS9-vxmD}(3Hm=&yXT;1ysgX2KlHOE3(aV&hZ`zYs zK;l?fq~{LAmCbPLGq#whZB*@DNm03c=ypuGk5uTzeSR(Mv6TC1EbQN05e3#QA8a{q zA$3)A?KhR;K0z2CpgHWx7@#+nob)BQ+FFoylw)NlR|oJq_4A?vg9qIoDCaFz{s^m% z2i*ZIBe`ye<UJ^r*Vc4n*ZJ%l;rl4w`MlUpvGQkBSKU?J9XQmU?d?^&vf~V!Q-RG+ zvw~{$9+$?lLuQA+w&(@SHF($NN<sWByO`R4<xgnHLZ4}1iydL*LkCyr;1g5seo~Cc zT>jy^i%R*+*Muz2%K4`j9ZeZ>8{KpI#~bb|k@>fkP2NU0QuVJ?OUgmwUGiAyV#*fh zV<?W?m}4yjS8(f7d4<vItMZv8_i1>5*{J4&9QpY^yCZGEWAzzDg!@-PpKhKAk4~Q2 zV<&ad{`C-&%7MikrtSrs14hSL{;tgtFI?(?jT(k22DjZ+5-{mhE?8Lc3^Z`dQ*TX$ zOGfD8h>eo_v_55qx(ICud^r8Z%>s^|wI$A=pHhc!`TmHjWVM6DSDb9eqg;k+4U{TO zZFO=_du}%ieC#FPo&QlyHJRGKPl<UA6`PJSTla{0@F8K-J{<dmEcxj_tqEFqLVNRn zg?YRP=fMD6g{5Io-~}ugE3uiQbq}!Q{tZDw$hb$whk?G5+Lz$3&~E|Huy@d+*<Fn6 z7md3&0RS|>Ku7DbRy;O{63B&t1VQ4^UC8dV`?Wn-3=X5h&DQ>0%z1t-Ol*#hlL=je zqo<1OG*+^gXT?$<%lvHi2T0UZJoou<Ev`$F^M}jMa@jt~Ww4cTm<XDUu)sa7{!`s? z&Ot;zLzupXt{)%?>CwlD7ou5KE62=bIE{s7L2o%o_Xy$wd;G9pX92FJYE@b8=*;oA zH-H{=_ekwdAY3aHU@;4NDH5T(pDjDfinE`;QN~se5n~262NYEAFKZTvhLk|&d;pU; zukjwIbPkdFMEa@)mQtpB1KRiQM9K~%3GMRB+v}8oIGJJS$5kRGai5yT*f@WT0K}vc zb~WTK-}*H(CffEnfg3+634oXB4FU53P_)r(Ii9v9A>j@B_i@r0by{$tT@zrGDQvGM z%xT%jL;9?F7^C)MPwJ(q=1v6C=X#dhlRa5Mpm4JlgOIJcCV-w(&tJH#k<8!L%wbol z@NtOB7t*?PoKZHQeI@`VN@bw${L6G^0q{0d!tpA@d>b4L|Itq!rcU8*C+rqO5LEPo z0$#bEv(=Y3k6y&q(7&hLuw@kPNRG95%>JsGG|RG{51wLZ&H{WU3eUZm@*OQX`gos` zK??oarxK9!P;-m-r1{hFa9y5ed{^C&iGjz9TE0*#2z#qekdaBm@41(BIQMHh^K0>d ztJ!7>|Hw}Cc$XWhj)04oWDl$5-7>pCuHEu@#k1{?h^W`ZtmqAzxwe=lMZuhcmXWuw zh3oFT3%EZq1gD91@Z1&6UY@yQeSYFw$gaxt#2wA;@heV^-#h64d_h3unJTZ%ILo>_ z%L*5nVqku%*x-1y(x_!r_>8_-0oTsM)bV_yjfeQ{e9aa2=uSh9o#<2Vorv3y5L;MM zAHM!8_SHtdH4(opxwOUXR)nmrtk;WPR4JteuzK6;Kp#C^AAN%R(YT@;yH4<EIw=9s zKFUaK#0#66eK>e08u!PE*<DVEtXtt_5Ue9M)6ZmIj;+|m20U8tg#OW6m(uNPwfa!J z$h4buSa+KnSR6~KtG}n2>Rhd8kgT30?nFmUa=f1^?>+SOVVTBTYtU2*{P+C9T%N*Y zl|%8e%IR|<J=Q+iF0D_Iya>B>5kzrodbu;x0><~UV02qZI%fu-f&?dR-|?O`bndZw zPlD4vU>#m!KUrB2fT8gu4J$luEV$v<{T#GTKE4)Jxm1H1Y3sbmfno82M(E-tvI2&` z6aB4oK4u+8h4BU-GTJJ~6x;5Yoi7E^uBo!?WmScv$5Y9C2lnyU@5R1jG50Cplf$&? z3JV=gd<=u30>;L^PFe6#zXk8#U5=n=vo)UiOFtx@tTFkHDL(zXFDj7G=)mEnhv=Y@ zqF<BUcSBHVLo__cqsQq=R1>j21;*s2<R-o?mKCG;Ib1iW=wt+oh@2cYAGx&dnK>Z4 zB;og`%szcE8aePRu`F`bz7o~b^ydGjFg$XY{cp!9jHaLb3YN(Kd_gy{=2%Y{i`<p= z7mBgV6<j@oOg@ZNq;e6<#9_u%Di2l;G>t^2^_2G{_MYbtOB@u15Fu;LB74HRh%H3k z3d|E5jFl74>QMw-fQ8lF(H#Q}CN+id17N}hY&Z<=O+t7@KmB#!RRA4YNs0h)^hw|~ zun%RnqU!?+kOtoi%4=~@2Y)G(2bd^K*%cy5h#Mh{<Hj765VKvFlEf`X9(D^$+RTM* zVD!}>Ze&-22qjY#02b}vOf?^47Jcok4=p_(GUspS0bNb}w@zOScLVtcOr^4?JOBsQ z8&gi1@~$Ru^gTTjevj@v{}gskaG67R$-J2@Aj!lZpMuXu!Wb-f2;!e8bhceRwa4UT zLx%jp=*~G?>f}<U=JvVS%2(Ff&Uw{(Ye1Q?`xLZ1n$EI=qx)Xp*PBS@mkh<j5Y<X1 z5c%OKSqv_{X{Q_hLQ}0T^=9?MEa4e+9#Y6)IdJjXMB&|y9Zg}$2$0{yvKoq@2xH!Z z8ihj{0s{(ougc&ksla%aB|c3P;2<8vcxx$iOvsLPHhjwa3Q&Dl1PuI2XFEJ>dn4h) zG&M}<-?$JjGch?NAY5128|$*b!W;)&Ug@?OZ<npsw&DaI@*k=4x0_TM6wAp}KYW#8 z@q;Nk2>zR-0y)k{oXp9-W#Ee69DYc^1Fqi|OmL_)eF?j|!T2^HF_k)K3kaq5-U7Nv z#I;CaW{{fK0>6~{d?XQgw^}p-PRJ|FjO_wOh|IqK3&Hbx{gOOy;l=yI&PX(dl8f&k zoeS4tjd*ch750@c7X1-WZvQ!;#p;EJ560`W31EDwW54k3_T3wtvde!G@8W}BTZzW< z<%21fwPh5Qtt+J8ZHHpuHGJ+z+riP}WYyHNxEaK+nPJD@2h%sa&NH)b^_c23JM83m ziJ$=1O!SF^;PUPnN>zZuU;M7y{JDU38R~s&{bM9k4#^!MvegnQZd_uGt|5M|XeFf2 zR3d_6Tl!?FEHL*>9_i>8+YgM|6MoO#4L;G{Z6Kve1uqvE2d}KZXWT&%Z+~A^T;mVu zW?F^VU;-RmGt1wI<yf!YwGy?CcM(Tk@k;kb{hr@-vOMkfXBzPEtNJaM^><Ke-0MA! z7}$OHx+rMsb=O&(IwJJOh-QEyjA{lYrh^?OeM%I7`}$MrL2kblnIdsZ2fNr}j<Gm~ z3P_~kF@}9r$+*?fJ0=B-OLiZBW7#DkZ{WGnt{g>JFSOZl!k_uRYEOhIiTbx(M0M=P zTR4<G*MM_87y3(2<euTh?r7xLm-|L2?6Qz`vAfl;qY;D9(;U{ob8bQm33XB<-znW^ z=Wpxepx_|frBtVrR*quX7Uk_HgbwGJFR}7PRy!Yj2^8Q92hBX2g?hhwublfzkv%kY zN#EpD!j_mD68GYoeLTccHe7J-{o#+z7B$tUo?mB>+Ma>JG?2#4DHwBi?e2#DjEet4 z;Imd8cn`|1q<ZArXiVt_x2q0l$ESegF?o$3I8o9-ej&4aFFuxIfz5RMz5WwM+mE%D z=CTjp*SRCGUE^y8xLyHMWM@zc&93}5gTHptrrwL9#OQ8^kysk8wXg|~G*huAbt0xs z9v8K9{CxKakY!o7SNnbD_7POB++#(PBr%az+ckSals~!XS7XX1tI#si@_$m+U;H<& zLCe~J*g-x3Ozl_%5t*DO=*J)zm>6gCd!arON{=140;C&jX%O=fo_!qYNAH5byOZ57 zbKP$SVAcIOqz;8;VGp5L2n@DrG{n-vaH@9BLHp~Cfa-!EXGHv`W)4u0f8PY+JNqBH zu&y@m7gBMTXR|SAcK<TEs3bOrt_Cmn^^k_nD&QAQx)$P?Z?p5FxbPiB-Izbty{1ql z9#Z_TnF&?(14{WQp)U)&S>9^9M;`@TgcPqQ*+oItR6x#o>5zfDGvj@OPmh)Xhy6l9 zfY-A;0K^-hZ+sY>7y3EKm`nIMS_?sscJX`*R06QRsfY8;N@@a_wi05|-TT|p{u-58 z2@YiG2}#?rjmBnaf2!nb@v?m!Jy{ma)!If$_7gT5`<1b`+yBI_0%|Zfsf?Di<JMT7 zM&-GhFSF+9pPhr!=~x$)?J^PK$o*mK#b<}~ob9$J8>IpBA~QUnQ1s0V6(Ue1W8)4> z&`%jewud`iqnp?X<}*?y=KEg5<^r5U=dD|aCO`p_G-^FG&ZeXQLBB+_86$FSI_hQ{ zI(29PF4-(qyWBN>z0}Au;FLn<F8BPoEn}0>YLoH9B6WO=2|57zgY@?R@tK2UIE{1` z$k+>&A2MEr-CBe{;_(?mp@>4{1^Ccm@+~FTp|7fb<z{bVPt7Icl-#xbvvcta^vKKQ z49!ZPXbp%H0exN1laz%gvO#0?+Y=d@i6Z~5C9^TUQLwXTCUAoo6*Z0YpM(7dOsK!= z<|_ekDC?}-Q4+41W8u=-ZnY^ZIl`Lw;qbQ#It7uTvKG#S_dIZ1ZMHp@WEXbqetI(2 z(kKP_nj5sLdbg1t5mnu3?unn5SsV4k0h%*$GG$*(i1yxZv3FZ!R9y>j$+2I5gN(bu zNJW#6Y>(s8*E?hbDOI>HCS(UCjp`ZyxywI)llI51+}*4KpX|QCGZk6Z<n_Mt;5FyG zOH`N}ZItm76HP3^(4X^N?7%v5o?p`SYEPGOstgRAHZQHC;cX*G=YPCy7<0T*DOQ%s zb1Lk#@~Z};KmHiBM{ixN(Qz5tKjLeZW3>=X>3mTsL;B~GEB-TZFX9Z7g+w;BWaast zAU|=CZ+j)+M;Mnly_5I9CVPq+b|*hY)k3#zpHfN{eZR|$ramgoATAm%4I-mIfBz`4 zB)mP@V>n_pVi3-Zni&<aw>KR3c{RD9MdH`8H@>LWIx%S){(>@&9y00l4M#5B4W=mB z;-I~I*?Tp&uJ2cu{goxwsJ6;&M`wNQrCys+E;5PMUW`iZ<bjv0Ebf^@@mgTjzBbaf z`Aj~CJ`$#}0lJ7Z(Nqn0+g-3FN4yK*8G^kl8+dB8kZk|Sw_q;^ltlKi8t+%VGpRlK z^jG-jebrNEaE+X2-(cOSF3K&#BST#GMbTz4EmFq6iKvTrZo?C!_u`u9kJ4gaVZND+ zp)pBX=Dz*#?1Iz1LbsM|q@L>qHNN1^+|cdRBcrw8qfqr!&-+C)CB4f@DlQ8=?A@}* z_rT9&mo+!4lh^mYuB_9a+PTa4;RvsLC6PHNS0B3>=oDg;ip5r3Ic6W$jmV%j$MOG@ zk3K;7e<MR4PC@eAA9N8f)UgZ4Bf{Oo5CC^}_wsRU;beMeLGeG`B25E9tJqzJh4HHs zLw^PTUh}y^i(xpyV&J^tI|Mdq&Yw&2Git*(rFL8kDmcHOIY)u02JCI@lQ8Z;z>~#u zPS@0CR+0coNVG=t+c(uLhqJ1V{sutDiPsH`xbw8OdVML&t4@{&re!@cXtW?A#=0yP z&3SD+U5Wvay}O@^A?Fz`?PQRbJBWy1vWaW8-G)pX15gNnSP2F{bgDgbuUPr&$#sJ~ z;ZteJYh-lPABrbSV{eX2g@&JYK>x^+UBOLc!CR@cK}VtXU#Qa0CN<ST7kX`dJGxW` zo&-g_IV9{$n}F$xD$XcR-`d`;sWLe$w4|Vg@t@0f-?{C+pkk2JC(t)I>s7<iD{Aq* z<I4i5_}JC?+?7pozie9o_2Ft-z#`2wx=UzWOUuisveeiGUHt>g+WmaP-)h?=1Bokf zwzR?uC&_p7MhES$EfpwElxvqIZuWKfQ4KZy%xiFkP3sA#*ljIK(uG@v9LHrwL8+%j z8*_pUJ3wfW(R{b^#;rvG$FF5Y&XS`(8(KrT(|#vYxZBt6POU4IqB2Q$n1VW-$KC~( zs~!3Ll{pcXB_!CMIn7YAG?%qczi=FC6Ca<#qxUQ7XD}V<<_&xM+$c%N)k<?)tm&~6 zo^x{u>#g;vM1y>-fbC~~DZk>EXj7U`xa;E{zqfHaP?%bK*K%ldZ{7%*(KE4aL*b8& zb~CU-#A?NFf3<W{tNx2excv?y=8!hL8#I~3sIYX1N~HFwz^|!62Y$`r66^NM-h^I$ zCPqtt%jK+~(M%50<1!)0=UmGF;qI-%+FIMS(OHE;3#1ea4y9Ou;=x-8UZA)`i@Uo{ ziUlw3P~4%oleV}+puq|5QY1(qWM{5#erx^zx97hP_Q5{bC%Gcm7%#lzef)XuTk8O5 zms>1hTYoYUvneFuev`L^a6629ct%NSu~=Bs8WDFoiq%fLm`VR0FXV2VNA&pVg5+^@ zu0K=2JY3|-%YA^qe1BH(g1eH9yfLj+uCmpnhyPEbgTx5O$lTj{5y7jheIj%()NiYP z4aaz5=63Y5&#>KJ{ATQ`Zfrm&n^oQ-;d)~*v+A~b@j{8>+?m5LX|UgAG(~tyG3UbT zvh#F0-Y1e{p20D&3z6(`KvWU^U|;!+BKNSmdz50{yz;lw$&_9n=G<}SSom%T+N%@z zEAa%p;JdpYh|KV8TYpKM$&@#{JluHsfg<kg#vt*6*-l=<yy@mO)B1h%)Z;sp@!jI_ zB%^;H;@0T;px1Hg@?4~ye7zO33(w3qxa0CD)3=-Ow8zl5=4$2K6&3`1wS@oME1~H4 z-^DrpCGI2|s=cS!-%(tHxJm5D;rN4Ky-q%`tzkI6FWK`Al7P=~A>hy!HH7FYhSs%v z7kT)1R-Xhe(D)~3U(Z9EhYz<7iyVz%is24u*3Q!lnBZ9xa;E~iV81(G1HJ7GI96yL zxf`<I-mS<9+S=E&XzuJhZ8}5l5n>1%#L^!#`tQ!Bo)=l_^NhFLR#vzQ#_VSV%`%+1 z?OSg<Zd0emc@FO+x6L$|f4#m-EL>3WpX<tQM*B9a+~y)0&&>9iI)@HbQ0z5mU+r9n zE+a%E)Ny|%HOK!KH_@(UfS~&@5_1)MXk@<4e^@(PaS)_XbK2&9k=$J4(FxeC=(OL3 zOK=+YGudN#*{H2@pMoEkAHUr@Fq%pZN}8<5!i+C_ztvSWw{OqPYR`OSIVdmvrSK~Y zUcSj&Zp$Bkus!|&V1nF_q)Q|E!cv;{fH;eyJMGOj)px2t)<;Tt$1t53?{bh&Cm)2d ziLYy1&lh>vR-#<3?s?dJ!5V8GYpH4-5La0tA=K4%r>l+fQXXk86|EZ{5f$Y1w`j{9 z!hWiUj*m&ujE@BxS__7x85`8(lW3M4obR`L)J<75xf<HXnicEjO7olzAEf58m9PkG zbUfP)J`i}}E{eU!yn8a`;A>xhTYn3&KpTS??mqJRPiFhkq(FNC{lWU@FZqC`(bzBl z{gR14CtMIJlR2t&IwD!{zx>cgu72>}*YV=V{LdxWckgQxt9bRS8s_Yn#$W%(9eb6V z#;o!tE$Wj+>nE)K*R}dyi2Oe<MyY8vQZ{K*W!h^m#Q&M3N#XTPI}*5^<oKfUt<ben za{H5pUW@vw?tfac<s_sJSe2w+ROWJ;Q6gy^fWN7MhtalqUU?$?+E2UHnt`A?8G-p2 zF`y%sWSCwTyxC~Lt`x7;Su_;t<GJ7<c(@#BtZ6^bD&|l-X|cdJ`cJz-DQgARsa#Ul z_gqpMZQ3hhixDohnBtr-U$5SqEpq%OE<G=heN&1|5n*MP*r)~c^kvzMYPRc=9o8*; zn)jXw{#EKNxVd3ptJT){8d`RGO1YaqIOVo&7^t>jbW?6O>M_JI`!)||QBP8%YsL4! z?RjLjUAA)pZj8_)j-;t?1<JI%^1ig*348X91#JgBZ?UM6AJ#cDs%sOANj^>SqkfhN zJIo3Vm325b8IH0}=`8soXA&}xC)V~I)u3XJS(+tB&wWqFZ2^_K--;tXoOFIkL&)*l zz~P;j5?{xmR|=g3AR`jnhG$5dtm09}97vf2TO8)B{<Pw5c;RDxvd((OcL6NFr-U^r z@UK($?*rABX2Ps*xhN-Ts<_E_P^d0r_|=ers`9gd*tGc!+HI^QA_KJPqSUfc*T_Ky z{dnq@Eou_&!ggZRYfK}SmZ+}Me&qc%$7@el>~zOe_-dj%Nx#d6EnuVZLvn{exlut8 zUa%&+Rh5f(RYKeL7QY&n4eNY>F>cS;IItp{0GR*tvAyO8|EQP#Bu%CqI*j=rl&_#} z--dqaSI|*0D*gK1YWH3nykyrjI@j__!P)RdlX7UEvA$iyKOEI{832$wBrNimO}RlM zO1|)8`mId4Mo~GNc3G2Ag$|QM*3GcA%xeJf67)eu9G#UShH4=Y-*~f`QkB10&Y@B3 zqJP+RoF}~5Q17-!!N~u6;+@T$2TpPs%s@zUpILIf&Kx%eQHtR%sW=XhwfY^=?FK7j zoc8?wi4F0QGP!)@hFQPdt%$UIvLnjpQbMkuP|Q#&JZ3^1hq|Co|L7}>MDOKs=}2|v zXj3b22Ap1ArP2Ax@|QJ2odoDE_vn@Z=D_z)BDmg1476=fQ=V%ZDqCa$%o7p1o@TI> z*d!wL8;dD9P*3ulQ<v&P`dl<M+r2$wpk8f8e%Ji|buNusRd*USovo7i)$P(~)N#$C zS`jqP>UJyB5!rl0J4vh}QR9ig$gxY!W2tR)$=<iBDK#GzWvL;j^L&jBMbS25r&sDe zw!Q6ocVjp{M@2IYD`EN#>kbbz+eS!-mgLsTMw?jmm0LVqhh}~$`(I8?v)4SC`CcWw z7c!;UX3$1pR)Vmy$eaA1(}sP=>QU$37sJ3;%wVdHTt58Rcred(i%=GNWS^v766~_) zp`m=}u7g${n#-?uv1yuPF)>w@TqhmXv!1?U_1W$uM4s9@+KheP=W`vC6l*x(<_NqI z<7hkaaNp@tXW$dR9jF5*y@eQ{N}yqGXYT79&HLA>Mw5}vh+i%8sceGHm0?*PO*#In z=3=NG^$hRP`-a{~1F^e}DU{hS8$*|Cem3puI+qGT!-@cbg)K9z25O3R*|RT-Pvo4U zbgY`Vq4le@O<gR8FC67B);)uH7LYn0aS8mB&UK*5tF__%Wz*Qn*=f`DI9ouks(HU8 z;6dDEgYJyY(KAo0F=TU<xoZv7u)exedi#BwfzW2AebS+N)N`PS`%$_6QJX%yc3^Q- ziD4N5b+cv#+@9ZS;LXy_cSiY!wQqm|jdqViuod`A%jTGb?RY0nsb*%=-lVOzq0@=E z-pSEToB!_8&}dc38MBfi1BIhLiuurU<r6(LcHTQR2q<uCaM*Ob%4S%U=OXx1gxS7H z>DTl4+C|d&d_%%)?h6wvRO^`@Np<b7+<<CrBjf5PsYX>cO2X{Teu*(I_BH1Js7yeT zrtk1It*2EV!~%<t{0Y=XEtaOs7ijh*cAh>Hy*4-ugq1QJ{+U<M&<|i{Ezui}sT*CJ zmt4==tCivsyPZ`Z)G8A>bWLN^ur7s}+msuo8uFViKl4G4Ze|4ZMuV-)_BFr}@4Cme z55uq$*E?TsF#8sL0sM-SDpqW1<fU8HyeQ9{Euf$xe)c)X_k?8B@S9Evp@ep@GI{lb zSkSvhJ$FrqfGA_Ggkg)Dsu<fwHCKMy=7PTW8P)x{KAaK1vm39plsYYHb9|1Z#VD&| zc>7f*N$Hj5fc7xgy0dGh|I?X%r8TSfi_%kTb6&qf;~jf)^^(yh&_nGyY@vFJE$mz~ z6WfPe03SLoiN1WhC8(;rsW()mCNby3PXZ*A3+?NNLHdkD4l9{+rLVknIogkkVaXZn zC2owrY^S7%xiYhHW9qj?Zj7vMkDwUM+-_y9qP;`OWu;{O)+S-kInFr+_iK@3+f*^a z4<!yF)<1iv+{gV1_a>Qh;8Mdm6aIv+r#gBC%P+fR%iTcC&EKx)-J9kFa4MD!iiY7Z z`e2n`>f@bXQ;mJ4Qb&zSvFvghhCas?5vgMnu2=#kG9zy#=4>G?{yld!hdY^An}1fq z#K6Quz}e4jKk;TYWh2*34Xetm(`Rsd5+G}{+SCSh+}w5544!_O!Qq9?ba6%7<)Zwd zu$#8SFP}W;>%HdjJfFQevhx+jVYlx=I2GHmoec+rmdGJn$8ui;TC+*be>15(k96T2 zW?o<OYu~YYB3XMJr3zlpW6mzvbDs|rhN~%W&hNo1a*uphK6So0-GTc$YStiRvYYm6 z7-NN18D$N>ww^gopK*01bl|w4My}bkyYgM|n{?Wju>$|`(Wqwe`@Ap$+A8dM%B$Gs zfOGm%SKGKRlq*z`?#<g8D|Pub<pf^yD`j$PBQx#D;9**nn_`uS%ut#CHmiTByyDqG zIE7&BgwCvjxlNP$+2N%Yy2`i01x-*sS(>QVcvyC!fAOK6qgj7=-m$fFzUjEEn8TNi znHFHu-P@OSOIp6@M0x1CG1#`6BEr8}SMkd#YOCC^IVdcfUxup_6?Syhy|h>F8)&~! zlzDV}CWiXGo(yzeeq^&;R|Vv=DGv&#t|$s@<(gYNiH<kzA_D?mR_6o;dmMCT0tFn( zIp#g#+$Gs1+D3uuo`|zQZTJ@SmQd7%H0@MHvDv<5=KIz21sK(~)#8OlRQ<TSEK7xC zDBe)*gw=vUl@EI|8#L7Cf}(U?MOa<z=BVZ;YJsI$&vkPv)VBUxmGJqjx@9b|BsiA5 z{BziXLsC_IpxHmF3l>RJ<;Jb@c@q((9B$)aWy&h8rJFLuy?&d%tlqYxVv)wHVv)#( zM>B|G=fr9ahjh{**2UjdBmki6e#6Xie@-!(vA5x||L>qaV?HG*71}c9TyG;sjzIlw zV?M*t1$VV6QhPp!wIb9`V)9`7dY`ddS)Wo`evv1lpm{k!CCZ^oeN^P=Yeu8ZZeDom zZu@c9R5<-%g8ngVHX_vu6)og*P_`{eLA$qRtzD*(?vn9l(?tw<p!k<=1^TX<#ZUys z-~HZ(?j|NzxQ{jpmit{R3{@?=RrxH~WyX*+sm&>jYUa5ZCReqEDv&h1XCw_yE?-tC zs^HV?Ex%?nE^}@n6}X*W{`k+<9{`NdVzVphj+2lw^q4g^x3McK&c4}6%j<SOO*eIe zK_>Sbus<3Kc~=EqT$I^aw<)F4WYqtD?0>hZjXJOAIv*TTY22F{KM!v|R?Q6v$2uv+ zSe2ii(YTX$9hI`Po?P*{3f^BK{!2Eu5@s0KC3{2d)}oNgS^?y@**k(RSHwwXVE0&w zec4-^)~`eDFN@^4eDhWi(WPPNu~(C|s35#iO6+!wpoL*6Hq*cg0augEK3skQ=O-m@ z1GUT*@6%Dx*cOv|dxbv!;RpAn4ck|Bls`p*zZmSd$9(!f91F&<S)DvR)8s+N6hF>v zgc=Duzh==;jAj|Etulg5>yCDLGA(IF()=_7YxTx}{y%!#e8w6xoJS?IJ@2IF>&|G{ zj3=~&@(5ZVacGw7;gbmL7bk|t!8WXU6RNbv$@!I+e2z=-8t2(d6o@se7Y+KU0wc#L zT=UwF0=~x52pf<Yi7JsPbxL-RiK;C59v1Oz0@+K1RSia`3og5aucKZ0orgBtK!u-f zel52XKDBo+i=VW(Y5eC{M9J!Xr7+;VEm(jd9q%XR(ddk+AHw9&PheWods9-}eXmh_ z{Z;m_(!r*|?bAot(CE7_?M$KK9!vpio798D^vgbdn|GhDF!_|pKbcmgA?#;dVB!0# zY(bd>U$4Rki&Lo`sQ8P}+BVEIxx5+ORc*JVSN2sY!5AG@ILEUwBcoZ~EWeO!sDkxB zv0L;H!6T)%)h$Qmw<3j2etfuJ48I~HUctbXf18$Vmt%&GgNN`kN9qiNST4)JEnm$` z|A4DZlA-~T-7buHt;MeIUfsTB{hUD8KLtXy8~8EgCZCGU12F||U||ceNHx;`0<%8{ zN0yw{&)Gp^|LH2{Ga_slyl_1HPXXlr%b4|Ff%X4K0rLM(7BjtXpKo6qlPNrXDS^3; z6Gxw`quHa!w?MD9w!5>guwbNlT+mxJEcCMB#mtuWXZLk10$zzY=Jrgf4gtO1qVz}p z(ng{7YdW=VpWacn?X_c4?>Hil@7U4%5@l$pd*L<z3Mn$qv8X1hbrBQh0>#^`zAHID z46KLpOQa%B1#vF%0z|&O`euQ?L12`RD0Sow7){XDQ;w6le}}SXiUV*84!(K2rUXl< zYhHZQ%AdwveyutF7rJ66J;Xj+>6YhMbkCPRe-!V1LtQL5CJK*b3ab2tXie;F?Fvja z#^3rKP8(2)1>v;iFeShO7mF~#C-G{acWn;shXPB^{nV(HU_u2ukpSDBX0EO)$>Xk@ zBfYbiQ_Fs5qu>J!mQw+VMW<B4Cw)C(i~U#a_8%)Fu}ID;wEpLl!nB~3S;jcgd@NM3 ze4IGSs&1JidF=KNhUHB5BJ<%QU!z1CA9Ww(y2i&S!+s)YtDfF@2jBMG?<Hu!)zM6d zi^P>$+uq5CX2HlBgRhI%TyotsNja!DBs8Pr;WvUa2g6*@%97jw;p@X6dUZa?J!Ahv zIt&(``#quf)mMj(lfDh3lf8<XR~}fHXXZ;nctb*i%m;1wI+J3`4i;^i!x<^~ubsr! zswO21MlIOAFGNUz*7##x(A!O}HY_AOpU=4!7Rb{Z$ZHY(N;xZyO7`vS3&tcO5*ptF zZY-t7VB1!P?eqTCn!~TJci%Vy%u+cG%Lfyg_pu0iutNF7gxXjSEO0;57(%I)tBjHX z^cn6tE_TTnw0qJ|t1|C?(o05GvC`Q3L+=ucjQes19!h4EHu4OW>>b>D?Q$_^$t)?6 zFJT@2lp?_(HOrWaj)%cv*?q=va67**vo&bI>&GLqBsoA{i!PebdRGzlyS?b1prY^V z_=_%iVV^R-8`a(H@j}e4U}Lzp9P=_0wYM+m&ie#R_FVe<Ru&c;RisFpuzc6xH54kS zEx*0Y?B1%38m6?SwPTq+`SszrAA#p?guaP^yllG~WX_$wy*}gr^fP3`69+1!Q%}Za zlqI_O+xBkj>+Q`^JJ)p;ufrn95Q~>MsMl;aS7&BVDG}gz`b9z2cCh#l!ask%9mOK| zCV{`yr+k7m7RbcFvV>d{!XpHJp(y|5MZQ=4?}T1G*vbys{sO37ZIBLHac+4=D{ioz z8`xF+K0N^!w<OymLnfL$9<=4$wt5vsP?bLqtv=qVaYyP}phnJSlC>>_o1Uyc3CObf zi_?KY)Vz4x^|eT5?$Aw1Y_i4<K2@T&k#`Y+$iX)=KML27mtgXPNcL(usx;@3NE=_( ze%>3AoA&ntW(I55-1Z&;Bhp(N$Qk3@X|h^T;o?*-;~a=GxXelPY%#ao)T7-5g{<wm zRp4oOvVBvjTmQUY<SgXc1xr=&%7mdmP0%vO+v4=?eSf8w&vq$-^S4w~(XG|oV`l!= z5J|}cGn(H*A{gBG=t^_2U4LVKX&Z^A%uC);6f9Z83b62b`Si2f&Zye>BQ+=!jNVh> zfumw*C-s*66ivLA-zzL<t^xxdVT;i`qhGUM%nF&k;cw0N^*Lq|L+rfNYs-np^6nN2 z1(=k`z}k*9Pb!+bS1BikTbFZY#e<w=;@1joNOS93DOeP32_OcMs@rp&{^2-%xG>&R zUeP$I>uRh2+A=rO@M7yeA;a&9pIC&j_}hyWUZ~KR(00}a`A@{nF*MiCK106a4(=Xf zX`DKT={H?`z}33(oZ9!GcCZ^z`6SoxRQ~i^4_1=Ou-L}ezW2D!Iu4NZH?FFi<^j#s znEJEO{Gq5yomz$n1GB24v0F|i>UMyWN(WZoJm=<VA3-iv)a^HPUHqL@8L4^KG?tdA zUx01z%d7(HHWm*J+upSk1FeU#Q~=qyktI1k<)oYjX`%qA-}Ncj>L+U)W9%^z5rn+Q z$pWyMMp4@%5=#`|6)7%!0=N&Lb-GJ5UQFQ4={G*@whmK|Ye<R-29(fF$|ohx&CG2& zEuW1`<Ze)BCu|JEAz<kaECEo2GXXLJ$q+acp&w$hA#JEtrI%cIkAb$7?7IG!zI=GA zqDX|1^cx1Bhb3I9LFbGzf!v&6@h&V8fCNw-l_Fk{yF-S`TK7|-C4M*H#p^M*px)=Y ztH|(*HN0cJ<&SNrXlG^))1Ji$*N5ByMuwQGHxV*IyM>A86H<gs))ZW@RX?URM)eO2 zd+{AY%5xj?A_Z^d(d52+v46zToAqCJU-kRI;zp$R<aMXxuVv6Pn-96HC$yTl&EBVz zfg^d09~07Kf$0J9J<d(K7MyR4$@ODbH)7F)Ex%K3WA6$v>x=Z%>)*3j%Q0)fA8p@3 z-x?X#pVz3YfJ~TOGsgG#Fd&KbEK~6}PztFn;@oR-*J|k<{4GXn7RIe^zXhtJM*uJ{ z!z02DX&XUx;S)j_2ba7`0G-%XliTJx%mcr0Y(~zri=hf-$;L;}@oul3CF>^d9L2V~ zdRb`My`7Bta+eKg?4S6?v`<Mt8hoq}*OBVbiP{uIO^VO~=EQHb2aZj5^E{_cQ)}d9 zJrlQyR~Iw#Vz=)&M&n!uC~MC`ot9;di@cxv4_@A4mba}tQ$n#M<zSSbQb0WB7TTbM z3z;VLwRLAJm6`l9UQ_kd-j7*n*!i>=F{_1K;^G<7xt7W<TVP+sbUP%sjVAPk`AD35 z_z-yeQ5G-Y7#A+oieuSrkGjtTV;wo2?CV~gP5k`&F8FzE9D>vUvq<dqF({nuYUy2) z+l!kuvFyflH}ZXjt#;h?n+#pq7OeuOfyN;pVnnx``0W^luv@9J=t+-Rg$DfP*r%#h z-tC?nvCV!2F_k5UiJ@7isaVwUeG86)fF#P9STD|f2KT7rkV5f7EKWal9@=^IlhUeJ z$7Ro<+3tgAx4+P=6LG|@4*MKIaHi8@&huy9{RpWOBWd)yzu<DNDdI%%ZYy%(V}D~t zz(v#f-&SyqPTz85aMba2Ce{?HYPIH^o&tA%7sO$faAT%vw<odeGnpXd)>w8XXu?T( zvTNr{J8G}OeI*EI56gU|LZ#867;!j8M9-Uh)lcEk{Y>y>{~(Sf1%YqKlY9~<nz1Y) zUO<Rb{$w0zake~qM|k<L@0!<QM2;%(a>>*9R6^A{{XVVt*-K1S^Ku4g#wp}PLxMnI zUTVoXDV{4n7A*gbn$rCh2l8jSocD&5Zz=xaXeGYMWA9b30;RL9(VIIR`B!)`Qza@Y zedr(@8ab-MXkR|mXE%^6g+$*y947y8M&eHiJGHah<P0wsv|%&kXAvC*vvL6-MT)>a z{Jco-A1!Y@2QvART(JZ{FJ<&jdst&2AyPmb7H@`VwQyo(&^uXbcMY{y@hlK|kiAf@ z&~wBjq=4j|-DJKr!trL1v^Efny-ti{>xPTK+Iji(Xd%^jY%`sLZ>eQg6lgf&f)SWz z8+>RP5xtsiJOX_W=z7_m*0Ue%{WEPX%knA3CChC>SnZG9QTNzHAU9z+$bW414b3^p z+E1*}Pc;|@GbEC^2HmX#qvJtIqx`Zt(GUDZ?dKcJ{2{g^-#D=l=eiQCd_1{n!$|;I zwy)LE^k;t}Keqz5J+`C>8j|@VkY8G*gsY!DwAeb@`h!Dyy6sx1Z1t)2@qVF1C%r$Q z_|`K|5kqaaDN*!DMJ~HK#X^F!^D<N|&pDz2?Pt9@1xxcSyeB9heDuY^Ph6?{8e*Lr z$RXBZAbeQ^6Bi6SlWd!$M|V0Z<leC>1<ytdc-=(7P86ry!k1pfbZVo9ZLME?ES~UM z&Vcj<hVK5h0txIM?{W`)vW3m%brByWT8ObKd4qv|+ImoR3>}9*fAk@A3X6%O4_$t- z=Ql)K)G68O%PBwXf1BN4uy=WqnrAp(bbp(dw$Oa>F2PN$biZ<fnU_YfIf2@%-fcd& z2pW2Kz16N^mWBGT)83jF$d&4$OSDHc)XJE3<6e3*Osk;Hk*JKjP2&;e=5i&11+Df| zn%>y@W9GH(t@zDBqTLBa&1FlNQ{bm(Vn;3O3GR$C<*vsA9VPwz>HK2i)87$T4;m!j zEyYqPh-Ejku5)l&suyrtI!oQZgD}QUaQ9Os-U`rv<`)0+XJ;_t@yX>gsQQ5C#lhU5 za4+h6XOJ&VyMz<hTEJUFFP;6rkgP8W-}L$dZ7IEba4N10rO>)$5zA&`?`{j<_q^{a zp&D8@Qp(ne(+T(b9*iW)OY#LFOKL!=?(uax(EKk0LL1b_?dP=i5&Lx3Ej8$qxWM;E z{ef5Av<asy!HpTFNv*Awu73->QmS1xH*<+e7uJf}CpO{fYE?&`p5nP(sRjoKN}9e% zAhPyiquHh*Xnb$QnE!m?Wt}<0n8=;%)hA!x5Hi?D$UVEx$a)Ex2Krelr|<V7q?`|C zYvmt4SSY05T6x$_5iW&Kf0m@KEYC1SBwnyi%S@?Xb^`W4a0Gv!Qjrh?P&t*?QZvMR zg9w~6=vnAlem6)Ui$$Y(K;b1Qz-OWax!?!SnJKqduL`<A%+4T}r#O_yaX+y*ZM8zt zzR^}ag9_7fZXbDG-4-pWLi(%~!R!mQ+yv2w(`D+3wP3Hj<Fzc%-PL-c(D$?xvDgn; z6&wAFk{Tv7bOf-Hk*IqQ9={~`05&}}IWS3DOXD_8@b2@X4Y*p3Nb6b5>x#x-V~Evi z`qO;*P0EPK7OQDguP0v@mH=Q7QyaU{Yq<we7nJFKg9DPeKJhOypv49m#;4S=j_^Mo zzS??Xolc2s)ev<Vy7s;Iq@(p+rVexd#6XM8j%kl{)_ORV3na&9uWZA&2ZVY^Ddb`X zJ3c3#p&)SE!WQYNM6e8G#3e$2c<wqZHSabLPMS#cBh%pDw4vXhkpRo^*CR^5x8gn+ zVc(PL+88ITq%svVc@{}%M921&Qs8wN{@G{bms(jmh(1}0o6bDZ5@(ljxpM^H^}F4r zN>9$`^eua|pQr=meeTd2L>=2mjW>&HY}N?MLglE!?9abCCFlKB*?t0q<d(s|sYzh) zT>59Y*1n;~&1p>3&a!L*45W4L`^LtJZKL?!BPl)x?2VbUdoND+Jq_9H?szwlW-L&6 zMwPm;ak|V40!&52A=U08v`{r15sD(X3vTcS!C!Cua;lzo37WQ1)`Mn~DscHFM21#K z>Tij$BA;wFKE1v$q8h``wGDS;Mx;l^B(<9)y9Kb+`wpzv<Q-yA${9dZxwKW__=j!r znea2ocHyu(pdl-D-9|WzEO^!<kH(!dwap+=Mv&8+lY~nat!hJzGO2O8TXFJB4Yl6b zDfESPEl{n9fA<+ij{9VwIr~@y>XKRSQ|GR?b^8b1;9ZizpB?R1k`-<)_wvWQ9q^B7 z%<;v!d<6G?Ep8X`27>)27(q~}=-1!YjgCC*9?WgbvCUmS33?3=^-^Ilga)7!5HBHF zxY2&&Xzn|$uVShobM)#DGO$FC3Y-GY0|%U1#;o2&evlJ))^`;@#*)v;L!HIl!GF9Q z*9oNg^KDO|{i6CV+1^`(Dvj-Hv}C)^`qhX}KRV?`bXP~hn$ecZJdrC`H?<7Mw6VJJ z*WZq@z~ea{Q)T7ey}CKOCZz?u0J}{#b}lZACleoxfa(Ki!6^-po!m<O=^;1ar^l9} z0CPwpL<KYm(aPs?s$0G4pNk{Tb#-cUCiwi2fGxi-h0`zt)ao?04hH{b5E~PI_{S+@ zU@W7_>9m+KXRNHO_}WA7-{)fS@whRNq9iFXmduh1=LCASZyXeWhoI}fh`D-n`@KO( zA|B^zXBuUNtUw%Uw-0J$r$sclrJx0(SAyZpDBe(+zML)-(TJY!km#?DriBj}@RVh< z+69-rOrQG)t$h9#WlD_0wV<{3X44>q<{>i!fBpbfbB;8*nZb_w)`Na>)2#0hVeYl~ z6Z7}i1pVy0M7e+#a>gwV0;_yg=TaVVKMMmu7!lApS|%^PHTaJ(*JN*Cvq|r}5gpg8 zlWo5&M3Ph&_?X)hFq$t#?u>Wu86f*wscYm{OENq9Ct}13G5RQLG93`J{~V}Gi529m z?-a^`4Byvwm>J^3|8TPD-vPrCALEGcKU`M~$cp-cg*{Kqb`yQ`T?b76MW`-Ml99k4 zPPJwH0>~PCtsq8JXie#g_v23?WxZ<RExa}1&mxB_ZJ%;@1Wh*_<X=UAo+>VWlTh$Z zl~c?~?LxhZxfl+yIIVt{{{pZBnO(rVDw}({doH0k?)S2@NhaD>NFV6!$cE~K5RrE| z`Ze-x=DCTeH*s~Y1eBfK-)(6RlnbBLT4l3_^R8mPY^{1072LfT%O-)L-}#(PVT)BF z>?Fr`7j14fEHw*_dC^cgF8>60d>bt-Y^MGF{HHFrO;c+$jShV_@z8L)M#jaT=O@~* zPRGLdSdkFXYMm5^T!j~F5sF19?V|XY>FX*dJ3E&8b*f#invzLKsyO+chk2X-!B5L6 zVazrE?yP<haCK3lgUz3mGxf6Tz4Wi6di8gyRiCsm?9;3bHM{Svw|il*nW+JVJrFwn z3+D#<L;@DTwd<7qt<Pl%;H)#vdCx$WTP<V!R^Zdp_NPCp4dIXA<~{FDR|VfK4yF0N zfKs#^JJHz6_<<sB?#+n)1nxhSH1r#YMCKyyozRnv#VvT&(|n2b#mQ&fe+USM<X{UR zzNfzxUjSPLPybM8nivp^WjSKut1GVc_>#gOvwp4=U20}%W*GI8!Itw<B|uP>6sj^1 zjIgL2tfeiZjnCLnM3=E&zC7x!RKJZoSs~g^!0}cSARMM!W-v*R*u5GJryKN`n`V@W z|2x(3R}4FX?ItN}b73Qyw-Z=!w5yb+XiAw)qX)eL#6Y?70L(OFH&`w;KIhk~D?`z* zAuAejlg5*IVW#-2Pb^8d>10+z`rnaJYd8;UW<`KDoe7?MfA<C*LL8*Bew1NJ?An=3 zOcw9QboKrS=OwhHij`-eC(0t~bP{I3U7;!vrHWFuijGf)kV5XCP%A;mKnt^+#LP@< zSm7zO5?LdTm8D8=c(fYC2ihP?hkPCOfTVzVLi>WT>9rd<PSm{u1f&!)X#yg3p5}4D z<$>7Zz5%@*jKams`T~0A?D9N$fV}Tgo7)1n0i+KgN@oPUzK*SZH{GKf?If$WMH<gO z9m-woce>ur@&?fO8PY4t$n%KVWlKaD&;X*nC)Q;f)k^;8NJsYJhpZah>gfmp-WiC_ zr^R(rC`2i#bH>P*2`S}d8t{he&BIC-0h4KaSBTTFg;b~v)l#}7qkqUBZv$PP-usi= zLVV?Y-0?4Pk3|(dPjc-_hWR2(RAcb9WEbuPZZc?M@OT?HhC3f{Lgbv$<`NCjqiOi4 z^r?%uaiH<LbI84XzI0n$K&2)5&wKiB>jKme?#cRUZ>_*jDgDAV>?7UwhpitXP#^K9 zz*Ytfhb?bWH<4~PnoX#V8+YqW?_JWbDjn3YPF8Ain0qGUXrQFeh1hQT$NE%xk=IEc zP}R0Grrq%x1!Rf5>@sWBl_<$m03QO)mQ}Cjb5>8w0Bla@uqPX`64bV#y^HNI_OTrm z`hpu1m}E2hWen{*a6EP1&IL`T@sr)7*yCACX`rd6Nxsl~v3FCV!BS{n=nrZr(;Wyr z{s8q&QLOQ=ld;Nbm$AC#Bmri=Pkkl+MJC9FM!!$l7d+H%phrG`(>r4=5_bD;k7OwO zOoxpp;UYOsGPApuO*`aR>Kbg{g|=q<v##iMJ~n)qA-g0yVzQGM>So%%;pp{TWF)%N z32r@F6AMEAiKE>T5K+FJ(=-1A4Z(?AyP>@1cqg1++$!jl?b_+WwSM{Vk^=s<MVjsM z8I9b^-S@UJ^q=7gM6B|%bGW{hjQ*%|d5=dc%{YYOA08t|NP(<Tz6vjZgK>V>)Gzks zefDIGDw8*U_{W?9-stk`*2}46U!$elO&)@Nrz74^2O%789r6Nq97A;Wf@_ssBsQ9# zyMIJWTbbFHKv^?I2ZM3g9urWdiMBKfb(GBi%Y4@FCUEd>RURHR|7AT#>w5YSc@uIX zL^DL*vs(Dfda8gJ&<UV`$cyNKGvDe#H)}69m^{lBHy%_-bOORD5glWF$d(`EG{646 z8zAc~${Lpl$cwM)$|vTML=%NN^~KXLQxLrAc>-po`Uuel5fn^7qyV+|Eh)luh=ybJ zJs6b#%7HoatzC6L2`?bBq#;nCa)MakegMy`zL2Jpnr7QIB~Z0`iT6^*bTzBTdbs5@ zm3zdEMJAr`Q-U~HG#0U52>vSF(925ssA;711?+iVP<_%Fc4+@)EIyVnI4K(6HJZC1 z-~=5>)XdMc_Tt?WOWWd!wAM$)Sd}NbJKtz~dZ)6b^de*jC;vm51gO~gwff_H5={T0 ziV*E;)(*-1FlL(MM`CM4eTwi0$J2skX+uS(Y>c$khL0Kgt;Pzf(72_KoU&R46>oaZ z`jdLbGy3-32e8$KU)I40S+#W5J{GwxaT`<TZu0`@{C(ZYQtacAn&mwCHlb@Y^4Yt1 zWb8F?<^^0J*4ZAs+WE4(=)zp*^LUp&TiR<5zgHwQ^{HYR^1=+)UYB(Fvv-1W?Hnma z#>j513c@Z9dl9ZBkMq^$!Udxg6zBV9okQQwhW@Iu!`Lg3vG8yBIdxo=RBLEz@v*J} za-4QZw&QjjqbMy1ejCykTVFJvs<>IAIBF?RW+sg}3CA5Ea4n=zQK*>N2w&+>DSJRF z&%FFahE};nd)_yQ@6|2ZNr`4Q@|T)z6ZbQZvkDkHZH<)GQ}>)}uX(|w8+jI63l?51 ziP*Af*@PTA{8n|Ib=uNF?q+w!I`CxaT(&4kFiaeO(+v-fDgKap^V6q^fy7Y|r)5l+ z*z0=)vYsU<hd&o_-&xW1NEJ6`F=`GdQoOxL&NSXOBKD`RkH<On4F5jf3>zVyQ`UYi z;x%r}EkS;zU0(bx;JgJ&zt_9t1zBi4!d7(<_%ZvDR%Y077GRm6Eh*uL9v>LzUa#1f zOGX#RPt<E%wru-M`)tT)ea|Tmnneh5&1eiA&CeQ{bO8G9GpI+AN84&y-*nm@dBYCf zEtT+>_brt`u6_ap6x4?4|GNqFHp6T<T1Y6_cEAE#6BVjRvex}<+FA}3$%_}DtNey0 z1IU8!%WSKKV4+fM<*6J&Th<Z;pe&@FKZFrqmhKq9a36p}^_f@_kU!DkefAuOYV0)@ zOhO{d8;nP<mO%LV<yiAUm3$fPnDn092E!OHmE&ZstS(m7_d~LN>V|!N>n8O&)1{2| z^Q&DaFE!eSPaooE<v?&i@V}hTPhfT|u>NBv{}Fn1D>DP<&wX+*mZzse^;NiC;P~gi zLZ_0sWAuYzSQ1(Jhz}IhIk!wo(Z5ZpoV?avn~&7@KeA*!3%U*|AUna4wXVcLkCXGl zM<TGh-GKE8<=clj+h309!+M3)XjFk9=Q7!*337z)kiomJGFE?Ou@f-q6`-c{R7fdS zTZA|-GV~<OY`rn&xi-f7zRdbo;5TNR-w3eqhLVhgq)zDCFrs&gxfa#*qNU$6-UUEk z=3^C!P!cw^gHuMkf6q1AinYnmLb(=hUTq2XW+a_x`pYPNY<te?b=|Ets#vt@S)!kL zYSf2oLL2XapOZl2D;P7NxIF3AIqg_`4lj|wjUj|4;>9p8QB)6Z!VGH^ZbJALM&tX? zhHZP+)_uINEK8|?ae2?Kl*sVrF1lZ7Y}q73#_9AxwdS}kg}4*W{u0S~czq$YB*hJa z#od^W+AGS<b6`6cBi}3jG_Ush{VuFOkcbm-lyj{!Z<^<J`h54Uxv(rx+1Hpo`%G{* zn+t|@t#JFa=nZT3VfISKhN%<edw!#dpE`A1DHTKdCRDDqD~+uRpY}|7N+=dcarsID z0eenzR*JdOuuq;vfRuB8`SAUy8^Pm_m<44^hyFo+h+MyWZ>R8)&z}<m8Qm!A8eaJ9 zmAG`ZZVxY<AVj8m&P#$$1s?uBFMeMPF9s5UeRif`MnbY4<YO;6ywz`xh%Slez-x}) z&L|`kUJS11oPJ3{SYs@tTSw)HwSf|la5@W2@x_NKtw|AZ=R5mLnnqg{)>v`bml#cR z|MaX(U94%rxNt>d7~Se}R0I2d{=4a*xMd)P!qtUsCTY~7-oCCWW##JT1@K-!=@?rN zWz(q%<p>MU37IU7$sdv*qqR0fnk2gEW!)IISN~+DeaMP>98UZBnJWHCHUw1nSD-fJ zN5-fOZsOxmu!@#s`1weZ)Z`Tb-(Ey_4ND#%oKfYNkxor(aqKzZ;U_x07!c*0aEj-P z-cot6!hF~|5i<)w5k$o+U6P>OG$9vAfBL4{k8Il`3!(Zta*|{O2lY}Wny!Zqw0^Kj zZu)cfjW&^wv$7h^Phw@&tRj&1ddtdyH0Y<Ug7982`3r<tZ`N1k5so2?F?s?Ehirug zxkWX?x=@+DU{Kh^pyzaa1G`qG+}1Ny*AdpNatS6rEDsf+!O5ck(rS|6;cJGFGGZj_ z?oV|AOvMB_Be3;|9KY|hBt)j~zA9NzoSX)VayuZJXHBHqd~6i!0f<yjrk_y^TyEpV zOvlx=Y7zaxl8B5J`cBU4nT|6Z=t}O$@Cb8WK5fx7?VZE7ovyJMWiq>vtGJCeoc~no zwjO1DP6vpQ#)~;FmRz3@DP%^Z92s*`yQqT%1pP6L`cVm&uC#?~Yld8kT!Al_Zi+RI z6|CE(VfC~{`E<vTx4xjjtpaQ{qi~afPauut6l&N;KYca=Rp_iyYJTOv)=Ar>vPm7| zE*TIkbQj=rd-ld1%c*N4A_b%v3%oY9o>Yg?;QFp&!V${DMWa!<_?reC)B&02a#mB6 zC)6>N*nZy5rU}q_T%J8|5j&0Y!aIMj%XR8J30j?;$o6snBpJ^m&w^>Tq_gdrKr0pb zu>EA;2a_idO9Kgm83Pf6Km+Z?KwTUiULBlSY%D}DW4h=a)PQ^2p_uF|cKHIn8iRcT zo_sR*ViCCDCB}_fxH|I@Gj?TM*V|1&JS6?I&3cQzOPRZ*i2>Are;6_WnT!m@;q*!i z^{~K?+4v!)neJ_d7gPHh&Jz$-D**F=!91rE1$sR3{>K);v9{o^R*;2ad)p^5KZY>k zp)&J9vtei1hwOkSM9-ZpR-G7Ko~}L&Cy#As=~$_h)Sw%RfiPzM7WJlW#_zV4?g5Mm zg@{sJZVZadU~6{Vkh2^&FfRQoGskQ^P|xgM0%}!Miim;j$u{NYN>8_ZpU4G-g;63a z4Ppf%1;msJ+P`G`LzY_QShoMwLP&xK27RfL{|b=jRJd9x^_Iz)%K2Q{Z6nz&e**I6 zeFy`A5><P&2!YHU%o4&nNk4d%NaGyt91EbqbqBpn#{sdVOMpJkxukIS2}Qn*sV6;2 zKi(bpKHFk4|01{^@u$=r*0p-X%%z%u`yS*t7>8vw;p}4<e(zk!a|BWyWbSHCJdq^t zjR!WjGYW?Y#Xb;lNFEk~41zM#+Z)n%G{IxoaP9EsS~+LB$n$tvp7sj?j*$1H^42rg zuv;gWsgm>2Ejm&2&uM+2Nf}fp3s!+;?!VX;M674^Q^Y!DVl9I}G%vbGCqWS1ud_01 z!TI}*WVEt{Xc{kWXEc*)Y$f-ID*X8t$u+<nOkEHk#vjBpkje+T0|1DmS|2;jo_&zU z2EUJW*h?9=d6q{+LR@<8!>RhH=(F8FyJI6OtB2s9n-xXG7?#F#re`g>8FwuW=SsUN z+C^uyPy1`)o1n^VH=}$ET;8&#S5QQw1Syb~A>X;0Z$n^r_xE+OO5w?+A~ooOc^9r- zC=uey<)b6KN?<*NC`uK_+tpGCUuOCR)e{yeG(N54+t=OQDS~w=7P6<v)A}VxhBZoN zs0PLxU43Y~<k+1RQNQ=Q5~|=>sa(K~U9O%27JF6{e<?rH@zU_3fp6!+=~lto$O<tO zh%Fj&P?HTEcCy5<7(Cx=W@+#~&V+fsGXn0liZ&cqX~NQe`}f-h5CKa>>G*S=p9U~U zS>1-a@#~i;6Pk-=U{y>wWRM_l8E1)YG~k`jo$2oCW}bL+5yM9Qwp4ByrpAkb!*{#a z+HGg+G)JbIx7rvEsOHU2Zf$BUz8c}K_Jv!)jUEcZ1L2QDK(VgR5^!vV_wNyAJ3Tx5 z_<)d*Zp<7n2CUi7p}8&s^nM$5qg6aH(D|xMLZJFdrO={^PIybWGbR$sO$2P+R#kx$ z!&E)@-etA~(tkM>i7;KPzvAc-OT2I*j7FM0TMZcO#?G^RM@ft5xO#a#s7-cHMu?p> z<3pobEaRD>xn3W`kg2|G_GW04rFB7|ei2L@6?a2nUuX=FX3e~zxI7rCQg7vgLBS4n z|KZpDS6kPUZzG2q0AK~0GgRjLyRd?%;X(wQS(bi%b75CuiBJ1AE}F_An&8Luea-{~ z-&jU^tmL|wRdP!ESs|uT<Dji;nr+fa>m=fA|MU+u_a832SYzofg9LNqQQa0T?#vd% zZUlXU5EB5NgwrUCgy0VK0m$|vh+`lMR9`?RPFiZ7;M$$v3yb+vRyb(z4P@u7hLm3( zntV(H6@}Y<HKxcQN(3h|AW9^HK^DJ943X14U;kDJb#CCX+;JT1zK(p5B;J1#`Ek&5 zgnKA2&UqOd9X%kCWtoIbM>TA`%O;{2Og&+yb`3_Ua-GtNF%X-EpQX5R?<r<dNcGgV z__Le8al9VBm4!$W^_EdCP;OIfP7;EgOs8MDKJ9<)FC%o$>`niDjcTiH-T)54HI-Y~ z5zLyw`lt5N<b|~4>>=8H+57mbe>e*|WJe}uiFZ@bAgTu4W77u683n$i*ruO>a=E35 zHhfW{u9+IAQ6_@6PtQX6Mr`+hhzfZ_mPGz=+g#x_9ZRF%5;NlN7CCiBy`^+}{Zu7S zPdB)bE4+svyUc40nJDssw0?@f3(K;ZMi@4yk9J77Fk-M5G~RQ~<~wHH4dzmhKm4(2 zt+BeIKXoXv-Vk`}Z;f#pOZzg8zj~&f7ey1OB8-evy9|ZBYc5p%+-}50?d%KOY!QAv zig~(AqeoBer;j@7>vTx+3Fpg}GK@u=qOe6Kv`7qUL7XZR^|39ReIcI#c1Quc#BVy^ zJQqs<^Ic?7#&RjgdE$G!Ki%}qW_;Er<%6DT7)|n@!iTBmaf%parnIj=Q!DX?wFV~V z1V}7(A(y@c=&`uI9oP<PhKZ+L>+F=IuikFM_}&K0greM@m?N&HGdu;ux^$_AW>Y=6 z!<vwU{vO?DWq*2aEtQqZ89&9S?_~b=9G^H=r+I%ipQT0I!$D(aa=7(_1a5FN?kDr7 zDaT4cqPLQZn2T5<>AJy@h(UklVBt2<lKQRJ@C%^nxse|;gEeb7HUTc1$Gt4#^)|B9 z5>7_r^KtZ}_9M7w0I`$A3Vm4s8pUM$VjusKhDQ8rEN*6{_DS=L$s}h*`+)rqj`QD- z<8@TvGH`7;4UNr;!O({Ipd;03K~-ft1cDpb5Ol#n_57K_GWg0>rD&NO6@OYv?El`s zl+k+a0ezwAp+&az@0d%j(-nG-5H*jGSt^8>-M^+5=iL7GD%DIhu|>Q?!MYZLDsx>E z(`dU}YW$dn2dfb*jLY2M5K@(av(&0OC$&|#A<}s#5FjSx0f94%%y;Rv3p3H=xHG~M zs?YR4iTklKN~0_erq1$_mVVY(+IMBtG~&sWfHN6`EA70mxfd}N=jZ%~nn0iBJVc5k zN97t41r~yZ%_?W5;Myn2`mYO^WIkvJ_E?7$sT8b&qmQ9dX?f!SW1@6tQP_C&BY&wR zy4|&lv~v*?!pQ&-#JQ9~5*F^Kl1`2uCU@dw;M?8t%^E9Y3>@nI5&0pSIH_#~i_N#1 z6muN*5XPHIgdwVNhOLno9r^VMjbaKdyVMA-RXI0#X?MdxT_ew^TBq*?@i_Djwc;uj z_NR|v8bSlUj)MG5Gh2-WQg@o{tOO5BM$-vSaZS73<s{y4`i*-{<r-Lyh`?3)LQdZC z#UA4@IacddOO4UnzevP(P>Pv)Y-fAE0)&q=@0`DqI5pukAGd__^+gCpzL6IOTW&e~ zkx{MP$M9NArwj0G0}|I4SNygfd)MW>S*|49plGI9&RU^{aQVGly3OLd4zNC7f))A} z+2(#Z@@FJLLcTFd9v_YuLOZhNQ;jind|j7C4M~S@rBHD*9cM&Azk9zND~-#gO3;mZ z$+fLoThA?6Kcu^kC#XG|^eha`>tLrzvf#pZ_ZDB`MAELdA4_HsO5HVwp~Tu4MmhtF zw&?>-#|9{+?g%+jyPh0^9LlBxRoP~jo}Z>+(%j5hJWmo~iN{6yg=*$w+vvEsO1TAZ z=~Bcw4ouw3-dxXfUdC*VIIqi7EEg}!E?hxeF$Zr~oM2`!WBBv=`D(&M7n!)+@V$I) z%$?T!(SEm3T-97LvdHNOmI~YHpCeR^lBB*nEpJ$x8eK<6r#oeze7tY_0QwB_d}H+y z+__q;+8Q1z*HPkHV@XZNmTsHsF1$R|8YbTj97nTN|M2Sv{+$2W8MMCWDA%v|Jh1)S za=)+aqDiLk?}ZTeY<@!eIAky~dOxtH^Fra2p2L<@pJVFP(GxJ{EWNIb-S*MtGpyK= z9_i!8rDH8*(Np-Fv29PRm=1kBWDrA|9TrqmnZ^`wP6r*!E7DZzIv?ie4kUIJV|3Ve zE3PI8yj{39E`fxBJvhu)IX?Y2Zd+;pZb9pNrEf8!JG|<M?d8t=Ou{6fXZ8LoM+1I1 zb!_#Ovh>~dRogtU;F*&T2~(xh>qs8ZK|xy5uFiU3mY#Gr?WzduMKWhVc)@Ep@0Ox> zQcPlkk3}<6Q|4Kj<<_+-?&yf8D}BBHJ`alN;+ZvXe`Tq7RR4}GD(h~HG=Htt+2EUw z)-6Flmn5TVA}EuUaV$H(@3W6{*73UNOImTu@!<pEzUwrxqpZ~352FV)Ix{-Pu(jw> zhv6-`tyx)sz>ROL><?O=8{P{=1G2grp}x<i(uTv8(kFUjG`VTU7gVvN{h%#cOq{K( zg&3Xdof!?{3%|K*5GlO_{S`eM!yf%lfc-%GecsIIm8((DveHy<s>v^=h&w@&6S}Ra zFY-ytmc}LKUo77fPx)UWG&Ib7;apuxj5w_9uNcJuQDDYC*Xv=-2JNtzM_I#7{<&ck zB;7bvRO05<2KsP*Azm$TEN;^zNio~BmHU}})O|W+$;b#QJ{BVsOP;QKtRp)=W=Y3| zI*D=&D5bss65S${aDDS9f)d-CD9mM(9cN28V{j`L-?`jpo3%a%C9a!shmpF+4H#vD z;=&CexEu7rB<xx*30ZzS-3LS*^bs}t$GI2!9X)Zbigb_MA8R{OAzJt@Ogxj;;$U79 zcPS1tOU+#hr(#w?aV+Q7&s68b_jGd8mj6Bu0ZMh|GWaprK8mTv#-2WY{nER!(7yLp z+6^(iePhSlzx7klzUES>_EP87d4{3js$DPWtFi5TnEg`$j~q%7iFyGj+~BI!jt$Ae zX=T$8^JLCmE`>6qng!%IKwIQro$BEC4%@ocvl+H=sW^mYy4Bje)9Tr`mZ~YsO}t2z z6awSaWwS2i){P4HHLcs}E4a~Jmf8Jylha<Nn&of#_mI0ps$Dv^<3quI4qI?4w%i9$ za_|AqGQ0VEJ)G>khXj(r7UZTQT&v51MIuH?MuzD!Pl0EybQy7nsdEWQRhe@zJ_l+V z1f%ue_uEQM<oMT=edae3dOodk>%7S{rH4JzO`cK5BfNcS#`jgDD=_H}ni;KP72K2x z9bC29f@>y=TfZ3oJ&j=}`tPkH#^$X1_!+Rn{-y$~9i5rn7N+rb$!$RkMZkJao@Otk z*wDZ0+~RXQ|Ah(A^h7^pVh8fS520-dGU{7kC-tyt$WQE&Rz3ePOEL0avlR9}|8uHw zP6KOD1y%BDc}RPoghVpHX6>4uoBZdnj#7|ay7L(^&^u7Sb#I6@xg=}RW{B1BYZhMU z?#V#|wkP#r+J6kjS*X+LTP*s8$mEkNYT~8T%s<nw_@!ky-L>vvvshd(^ny=K*&zVc z$w+12^xWc~7W`|V1eXZ>I)$?&<*@nJ_uyFAU`gEcz~s97QA-3++@>;?+9iWP$nVR% z$4)s92^)Kf3DTGoWZ^iTacc~2LkzK1?WRWJ@<P8=L8tFh1M<eed0M3>3p0od%Kx0> z`>#Lu^_-+>osv!7kal^2QRmHGL;cuP%ac^A>DedPF<Po@H<a#Rt?N%rB4>Ja{g{Zc znCDSJ`6FDC>oW4x?Cmg}P^W;o@*!4rR-Im2{kh0pconn=lmg*#*C_X4+|;Om$~;8J z>;=#2Ho43*Q_dq)(<-&Jg*2tm?9O%%#vAR!{%=p`1@`GER!=~zG^;;63RU`5g9#G0 zJO~v=j+`}Yi{Dt^P<v>FD`R(6irM`DW+>*1gy;AkW`czn1df{em&@nAnP>EeMH)}C zO@BbPjs9}juimR$BG_9lNNu}(3OAK?-(6PM4|lPhwl0Q|;7cy^!^JT(O$z*)#s3$3 zZxt5TmaT!}5&{GWR=5NR7TjHm2pZfWxVyVUppXzCxFrD!x570+f>waSy>NGzTiM<B z^gez1oX7igpVr5>sI}%AbJ!g6j}N%@+taeQ)Gn>QXA9b}UA{MkLbc-$8Cp%LZzI~l z9rAt-bLCMPke$4b*XOe00(mOkKIr-QE$55sE%;*ZL3*CAPim&G{P_Q^X?p0p6l6$; zCjv;nI6w;4lV;d&bPF=%J9ovEv?>$vIxN|#-zll`6xS@#%i~#REwS%GboV!9uxgc8 z9Ef{OMyr|eUv$UTol@Vv?C>0C%&k8$&Q1BHJ?GGD2ZfHiU--a@C|pLqFfI{n#=qOC zzLmfPuaIzP;)O6e8fI{O`i&tsG6YZGj-FhbXH?RnTJ@Pu82k2^R;eo6a=<N>EYm^h zHMVU#@zuiGY7u|aJU4BjSBJEHbf~4+vuu!cQgy}^Hc_e)1fp~cxsX_PhE<T>w8(TN z%n=MmU!OD>v}Xv|RVJ%Q>W^1seWMmVSDUHv+14p{Vw?U#<Djn%(8sP5x}1zg1dIMV zM8i_JeR?$_%0z4jwnwzKRynAE$b_}v%t~rg)Bze5GQ{gJ$1;cJ^Y^8m*&6&EIj#fR z_}c`<Pc$Ji7Ux@S3a=$Knek~#%0(P1dE}@n-r2JaxPpuAMh)E`e!k4$xd*F0kWu;F z{StXt<0Y(i*&)_u?zVco$9oS_P5XMs>M=Q;A+~x;$u^~H#{GEI^r2iXZqQ<PoD~sg z4|yyx8;(1bZ9a+Sh+riiQ5t)3XkM12lkTj1i?`r$WExhlp4+COQ~HK=%*E(rQ5ND^ zUe2Sr8+l?|G?%GmL}WvT-kYdWqDnKiR-?(j$sB5C88Br=Tw3aX{Sn{{C1|$nTKyUX z;J5icf1)`xk<VqLpRRgY_jE^@oS!hiS~r%%<7Q@aXQv6LPWG@p2CBpK+4$oLkqZw4 z&nFQE6D@H07+ocw63Bg?L&y;WVk!eKRkJo|dYc<y`ftDlGt-DT-3xVVcKtjNT>b6f z9|^DsQbTayJ!Y}NwR5X`&W?ah;g5J>H3YDO?A+nB3hV>`b{7r@qLrbf3|1BuV&r~R zGvn0g2*7_Yy^u=|`=TyP0C?4U^&SN9%ZKd#)=xRz0p}ULbC|IarE)#P&gig?0mcE! zzYpsb%NRMV3p!S|GVN<dmdg@JSE;7h@oSon<AG<ZKm!|ss!gGhVCu28DTngV?hG)@ zSD%00RupDCZKMf?X_L#<Iy-K+VzAn`E4?14*D-@R%&48ud1;qBJHn0((?9a3P|WJ_ zcT@xT_VNJynWY9ol`^J<stKN8>Kl#@O9z<sZ#9tU8P;Zm$l8=v`ar8JHwc4Wy*l-Y zEuU5!M@z{s0Xn4xXXSGY@M2!d384Pn&JDv#Eej$waWZ?j%)sxp;vw?znRmiE)^dPD z`^9I?V%GdoTmI?yO0ratt*SQL37d)mO}lUV?8!xz2rw@hc$_fKMCs@O*sv%&qi*-R zZ@DO2hca(UWoTlfr6}wP|7UI0V!$xuJsfIvKIg7=?mMheqG!t5=@`vrI~VS7F9T<Y zfnI$w3c$@jzCV8gBB4MC%^ivJ<@JT=c5UT1*yAU}2x1Cr<3P6i&5jw~|JPi`UGM)< zoZq23-#<e3pz=jeX-dX{S>;cN+1R3rTWVmM9X6u465H;EJol`I-p71Y&Q~VHlXeyF zzp;HI2>QlatWYT-0^j>Ba?_YR(K;*;Q&q1;zFYG&|G3Ppuf%M#Vt_+kx@mnHT%W;8 z3>g~|p4z#KNrr(vC>-Y6q7dK{-{VDAp2Hm4E#CKK&5!NWZnlf~ouP|dZ4pg;?a{J@ zLCjuzr7DXAe2t9oRLb&g`fkMi4$iEs`FM}7U5-&MBaWj{c&sJjb;p<;IBA{j+A9p8 znH5_9m=193lAk?HES$)*p-5pfmS6DPDs&so{fde&s^zgO_`axOZuh4t1E2qd4zBRI zqM?vaGiRG?DwAS6JApj~Vw2Lr1LYc`1g})^d{o-}zkfI6*Dvfd{z{}mp3gf%UfLzA zqK?4r3>SEm%SCuN0&>>untHeZLEOHCN%Z9Jwl{GzZ^$>}vWpik#b0|+@ZsCAmgvTM z#z2Ma9tzCGMsvC3(;_{MTxa5-3Rb0N)+!}6#N)*yC-wZ+{am0-p%?x^^_)4LDUk`3 zh1r+W+9}Z+j3_)Ufe=D-c^nXI5|O79zo^Uc-9T@<+o8}qzVQ3g@s4P59zBOiK9nH3 zX|x`%M}y7~m3HX2)yp~tz;ppb{r>I!?Bo5?<G9!3y`=c{%A?H=&vvU{=mN!_M5e>N zz=_jXy~-P>@(0QLhUg{>#1ZDtj}RBgQkunWP7vE5f?aZ;uif?1pX|)oZLZx%M^W*= zZu&Oxbt$F10yx|-k>8h9ERI2)A9zazHvKXB+r}!Vbm4N=Skq2W%`qJXW;+iIi_ED| zSI8>lT<~Kpa*wDV)J?O%0nK@8Iy$mQHna|W=BR{Zdri#SIYFnfNlWERCuf^fB(o)L zJ|a{^5y!0iQ;T(8ERn4$29Iml)YfN$!inYpW(Qb)ER%B1_lT*bvUjgK^nNJBPd34s zws!0`b`jm`wo`$(FB{zRclZ{8yqla8p*`>1-v~+E?fphnqr=LYGAJJoC6}dADpl25 z4%$_O1XN^I#MMC!*59do&nLrwrP+3!3);@6@Oq4FoqB8(SJOSBH~aVU@6=u*gWv0D z59>h?-<q8k9Y(6s#(s9(xF$Bn(fYsyqLx&f^V}!cd2h4d{;SRd3xu1=H)YyKSy^g1 zI&PsoJlSOSYU0arG9N=-a=A4orFY{FpM8NPhu(Hv(&lz^rXy_cI+j0h>;-^-BUH6$ zFuAU<*C>y3?l6xPcH*&25XL8Y)`w221dAzps?uvQV8P30*=>G)(BM^UH&efB#U`y) z2i2+9<EL^yTDcitdZ@Ui_Ldu@4thH(p<)iF&}(0HK$Mx@-fSVdHh&GI=+)cSYgHPR zt4QA0O~;)e`j@_?<oo|&gZn@S7&ztEZCH`yLrTW*Z6(-<Nwewh(H@cuu^Fp=*FJv# zDlmQI8~sMwUrh54-U*b{nwL26&_#GNC=tDSG0<tNh6R4#JI4K>qZLF}m7-yub*8=U za1j&yX3%qItURh$sgaQYqv2po-C}^GSkVZT%75q?LcW{+_+zeb(~MiPJZ1p-1Zun8 zCJJ~WY5|^03ilVY#+a0@pGU|ArQJx3iVEXtz|H3rOfvARM8}^^@QLU7)fz^ueAIzo zE57)EBD00xIzaY{%SGLl!+T3DU#XUIwrLa@1;B6x+^4rQ0%7*JV%~|agD}m7NGd+N z#R55<y++xkqaY%F>qTWuN)Lk*$9`$%Ebs|zhY_K{XHGrFY^?5ITg#cx`#s7gx7tmW zl$u&)Wn+Dcu001dj6)24D+%cPO4SN`!Rie;06zP#UCdcu)7?mHhtQKJN(LV}`7U-A z6i6dIKZx!_wBO_J6-zT)-=x%bf5|H15Ou#XvNrICmy0B~+fB-N>`rR$;u(5$msl}* zi1{92v$7OdsLRsOw^DnrneaRO7AZN!YxDe_pRdw<J}o?yz-l)&sugN)C}Wm`Cvx`s zeti-1yptheJEkaiJoZ%wSa4CV!fP==g(mQ<TB|~jn52F5Y{n{g8`0o4MQZG?#(uG$ zX3Vs%=6FGG|Cc_<Eyug~`LVL@sHBqLZl)o_rq&_Y@V>u6?`Chx_rC4b*|wG4DBC}@ z$6t(a-(DsZFuz-i@ZWwu0f`=tX)`Do=OJTKYkh@I*B(mPF63w839Uslo+NDb+zii^ zYHa%>Y)y$-RMz(utK3UT9MGmp)+a2dwBbgkwV;&m2A(P34TaySbLQqVVB%|@YGrG* z%MkC^?TUo=9x?z%-$e8}G+W<&eHp`|SMfIAm2HiAfA%r&2jBG=n>%!cnWe!Xj9&&` zzcOtB)~yVAtKE;IB`bQ+tnDg6@Cjxs2;-cQfh)IpJcR!eak`VAb)BQmrvbAFHfYeJ zy+35Y$Sva(zeO}cDNjtWRcPh5VSwGY(&j&+vub`XyE$mrCRA!%j0P4c!Ec|{M)W2V z05WYW127nqVi8DKs?s1|)|9&1Pmnp6w671Xmo>Yd;vJ*_Qlp?kjBtr!)qLC6`bz7i zYjzC}sC+I179#s)YT%`^ay+3Ai4`5&0tgc!e0Gn$avM<RQN#&es&Zi|^lY|p^fYvM z(f^SIP^vjN-nAlfvwM%f+hw7$^~)+}>Rn}PMMEml@KuP!!xd4v(eP>}XJB-YalpA| zHNXN<YnnM6S$?y}*4A)B<108VDlmsJ8&&#{uOfNr^yF_lN!$_SXPC;tn<|!1z^1M? zcRtNMBXT|skNPgdAR{-kV>{j27EkH_DPH7QgUV(8mx$?f`c6>iifxC7Lds|*0wV)# zJJYUylGB}Egcu^CypW>ae1d?R9OR(~FdOn1_E8GT`R(zl_B|Emh<V!TI%&Xf2UA{W zn|yN4T9jP~C2LSNWZ<LrpH^!|q~-Rilt@9DT)I|~?%&>OzE_rCWYz`VUy4f+$qHlA zgg|v9i2<8Nr7_7xtQ~n$!t1B9LC?W`c=vn<-LVLNxiH1JBf@k0(k$PlFWd7CMCTjz z@T}((k9WJjopMq6EN&1WOY_x-i@zod?L}t;iWX8b_4R~P#(d9q76fCK4T_+zA5U|J zPP$~~5W5UlmN3N4tb?`RQf0JCp7xumsBZJ5jNiGHmJDkd^0}T}F}|(uix>SS1E1Nn zIjEnqNn5xnwVwggjdpYOwwl#8^oB!ncs)b)oW>26^y{#%@uwn^U%}+nU%f*cMCg-m zfqn;ty-NBKfrG|=l!(0!)U)=dm9_RiEpIqKV4_E~{r}Ts|LXz~{-b;mCAu}RiGqCr z#L;F2;InvbJ6|VHFGkBZ=`^bx7MTTBgEQIo+gKqmlP=Uv+c-47zt3@Ho77M)v7K!? zH2TZUJKKWS%MX_S@IkjrtL7Z^RWM6)oMS%3wy)LPT%!tgt86pMZaG(t7=WcD>x&R< z^(wW>S!D|v4bM4I3{-x?@<&YRRiuB5N5TEpVHOe8&T+XQTi>AXguO*;7r$7P{LH_v zAOjz2nu;__V|gWMEKH1OF}uyV@4ha9>QpJ`yL=iU69($nEdrbD7KvvX9;FO<0wUwZ z4xW2#jj5yJ2{Fbvi*5D8HH169yAEnAJsgN0nJBHdMZuZD>JX?-btD2XE#J}sqFD>c z)>JLh-5Lk88n?@wJp63Jshr|w##0aLEw8@}P8?n5bsxh1p#k>KcYTKJ#%8<g(4ZpM zdpeR^yzhH)BT?Y;P(Xo*Eud;v7;H7v_~fgwhD$7P>_4i(C#r1xU=>TRvwimp7B1!y z2r`~!$^Z%086ZiQON00&uE#z3;c}frC?bsU$G7^|M*Zb!DRtFY^nQBDm-<Sh9-uT) z^w=qvC3*xc)NDuSu(#Tp<&hk9OLPjmN7AzO3(75}Rm$q?siW%3-!cYXYAb1$3Qio$ zs;Csuc`XD4pV$gK_~_eNVu7r3MU|;Uy_D!#PoL}-A5x<xTmRry0BM#&^uhiSA7>%@ z_}St&D4<+E%|EX4`)>nm9llDq!s~J7I1}m!&PKFyXa+L!DYLw{&UzmX71Aw=Iz=d~ z%9k?MVpeew>oBMeqW-vLKjjHXA)8B4DWl=dEU(RuMFy8ohUZpo+N@JO$oHUGfzU?S z@1a1$dMHrAEFFd4X<QAH(z7gH;^Dc+e)YSn_8QhT;+>!B3f3zgg5XDK1+ee_&CWtY zE|YS4MUoj)@l5JmiT!$p>E4sN%TB&A7rCt#ygAQ%gmJ7}Bf%vXknz|Ne4oyx?6Jc? zPM|=O7M;_FFRZCq3RA0;@Uu-o#w_tEh2-xm)en1$GT(cz4+)D{9z9E8ZDU&H_hWZ> zfrtwJClPw7^A8aks7IhWst)UWjky%XK_iljN*p$+H2Eu=!5WoCllq?T!$=SWt#4+P zulo?OLl*!#3J8JB*$8G!y4Vup&VU1~Qdo`eH)B*?b|?5{%!(XZUJ$sg8MpcJX8Rmm zH(Nf?Q~9pt$x+%(l$B%P3M*+Fh%5BrifayMG(bmM{vePkj^80%*PZlDu+pvatZ!z; zE(w6Sk^nF`rHvTTjqpXn>BD=g?ZSI^aP_RU7k}$~hLW{!8YZb(D>48o$9^+ERWJNr z4$D&`q7uIP$q4wwq2HG!$sO}F%M(_{sFz%>KK0R>J-f)V95S{{_k+)8mKflALT)=- z6E^EUC+w59%8x@*sx(EV`Q5c$uVG1#8Bdg9O_0|o*ZlUNtzIcX;?2UVwh!O&Bsb*v zvCpk13p{E(c2-|BFcPFb=C++8RTfwmav{H`YfDOm`W?1KA}}}F-d2wJsl1kvp*`a) zW})wijE?>oT)@Jjq_&Oqp6HX`gK({G>pU8=-RfiZrtqkeXVK{1Le;A!`(&+b)jSQ{ zy8T&FH_`~Rjd|4xJE#UIgw%K2w-B1IPxG8jmxQ=<s_V!l*<;PTk5fERB21jRP4s;h z21lM6J4tNMhiaX_T~;Sd^oPC(Ai`~HVR046EIJk6BXx<2My1c2K`ZM-!9x`X&Ymt2 zVMGmw?or9Hq&zJavajzqZWcZg6%7NMcFtGU$nX4r$JJ%Cy+zE0)UU;(xTpV^#*Qg^ ztLS^Hipi{^Q8t71&TMKY`j=S5K-VG%8!ODVYzJ3%ttLgMN{I0EN2;|#PX+Lit}wAu z_2u)IT6l1#(az-+c-$qkEz~-p88>r#Uryg+4_7qJ<jU$*e%IVj8{U--W$0Fn`$ap8 zaZ^r(TjIcB3V%X8qKSg^_9<oL@?Mi_lb%vsx)5#{Fis)<?uGkv`UX$jk|jk2ea}5% zBxzxs&aN)>rjxZF!TDxf4wEPzj>y0^A>DerryH(WMZ<4VKw5|O?icbv{t=k%<d2^v zwa>_FWfUkQId#8>tm82Bb-&`m@>D+P-`b1+@NK|mxk(PxSI|!u&Z<x>78aQxpI<<n z^m~y=*p0M1&(y5S{Q9k8X4|t^`Va1F7KnHkm@3J7vL-ph%&F1bRW`e~T!Vb@88ubu z4du5nU@1RA$&#RMnZ^s4*3{Z5i?S>Ul;wZy-3J%jDh^A<mb@g-v(VVgjzCG8LS*c+ z8+_^z0jidK>2a;%@hWnFfCC7U?faoW`WP$0C>~qgiq7~U6!3&F@dewdjps>-wyMPE z5<SwM0qrO+6KY&?pOKLsQoMLw)g~=74KNH#_w;#`>;}8DGrr6P{rc}bc}(}MMN-&q zgVX_`rmEUEHjVGB^R;WC+9pPy!3O#%x<9<r7<XNd<WzG$G*`^`u+nDR(Hh*j7V0fn zvKZDwcET#FUzEj4-nl)O*&e7@{c7N0?v5juniZj1@2LKq-4H!pJS_2rzVzoZ4IFvR zOhbRSVu9Q3W1N0Xv~@cHsYOEK#9zEPD50(eM#)-OB};M&ip?#4ov>kCP)7aFvg>?b z{po@i<UKa1hZ>nw2CYdgQZ9Tp<K*{I@so!ZWnKF=;%!Nb5lMyZzGXb%X4KHwJj+5Y zU$I%w?eg9N0)JFjup0sj$Uqy`OK(w{@4DX$rkWubYb2ia+y{~Dw-h@6vUT>vnLM1V zQ}=L<ofZ3r2<@GET8}r80vZnDx-D$!YrQz9C7phPg^2Z}3jZ!||CGLxF31;dhcKXH z{@c#=HGHQ>lr3Hp7AD1PP*eW9s%Qd_q`1)7$+;{-u8S>EEaaJw^bB3gr$8P(;kKH> zSem^Q-f0<jkD=ecNaIHW`@g)z*=w|59xw3}u{x2r)HW6>$-QnVtP#J>8#7n+bUvQy zkYs&Fb2;u|VIE$nyjyfCDssD<J6xA=!u)M8Z^zSJZM}tS0lv9kHaiHny!OOPSIbi+ zh|3jIPgn{jjVx4eFxnaq;<6mYhb4tl`JF|nI7{H7I`l<Alg)MucD%eUqe?~R2C6Rt zDKAQz$3$Hy@6No<2lfj4hT?6om3b)~Hgufh#)NM=e?@FRwK4}#3l0vgCG)jQ+=xk@ zQ|cEH6q({!KmT{-^r2daf0~5%y93mEHvQ24OjgR|+1t;RkxU$-TV9Nw`!IZ}<_nqZ zUXG+%YaerLETAv&grIjO^|+Pk=a}lNZ*JiR62F#b3*ms`7xxcXsocg-9N<2+6@3FL zFS9iR4UOK5Pp&f$TBVHqCJIDuf`0X|8qy`pZ=hcnC`W_L=RlU030QP1MD~)Gx6RhS z2;9myT3^5fh6^t;;7Ok+NuKB>{5CVQ>g<0${$K=f0o3OP1OJ<L{l{*rhd6M<gHZX# z((e)%XHYqfq>pg=L3#E*NgN6i=9;;Sfx<~${8HJMig>j@J9I&~4Zg#KpJ~1gYAFme z?L!+Kv^AXONi0|+XuTFcIG>JSi0bRSBzY|V-*v*>VKC4^a@s{-*~)4E>4;J9=#61y zxEV@+j;(m$yekZ!v%P7V{EK#wNaIcE@!8~IzGA8Z`-A@fZX&K|P(k07xJ+j+TCcnu z96>v)IHK)?WAn}DZ|1&z><KDG=ll3?rUmiwSpfT3^2fqFwYxN3+VU&ohW|{TFbD3! zKX<YQ{`Up2{IOm|uKzoiJo&>vCG#kN|G5lyWn4&4+@CuxtpEQK{y(BZ2?gI`dw~Am ze2stVzSSTYWVGUubM6BW8H5n=6j6^*VCwbYtB&aZxpi5rLz|P3(V9owfBpM;2r$)Z z_9E8cKRtVP<8-IvMA!T6f9<%}4ho3Lq}A_|laP=w@ynP0U`UTU93>ha*m9$a%=L%< z$!SHL(Edk)?)wTj98O0VBS)X~n(=QJHCuu(vn%$y!+9AP3GY9duLPfOvB?b+pGKnY z9*^{Y{3dD3n5YnNO{xEvO^~b!2LAVF{`FT5(;uqnWDTJJ<p1rRi2J@7XnzFFg$E%B z|Ka&Ign#)7zu5EC{=V3Myr!V&&0jBOuKf=$zG1T<C}Mro{68M>@Z^O~u&7HXsHHPy zZgpb0)GOHd0QHgvB_`;T$X^s#HAm>AzY*yl?+pU3Y2y=x40;9Qk0Q^tBTZ6cXa&$e z2)5m*n`CE1O|_%6H)74Vs_0mY`i*k_?_a+)8l8B&S}C>1EcHW4DxR#9neZrH6_4~W z^Fs49(N7raCVF&0&d_|E02KH?mos0$yP;0Ibi&wJuOs~Qi0E;5L+q;f+FAQ_%oL~U zv921lLzDEEs#3N@2K|psI8z|Le0H_RoVV-Q$<`gLr~4W8XGw1s{wNo%HXqk6>hhc& z@cGW^+y8^uh0Ro$d*Y}X#+cr(Fz(cm3{`lgzeqA1er`)HI6~8UsNZO9e0do}B;?<P z^mCN$pEP;w$V<9)W@daizHfYdQw{Q34WV&D_JTW%etalKF!$L_TfnUVdn#Ymvj^E# z?^Ar<-$BM329Fd|ItB)m2MaAll0I#c-bYf#r-<VEd+pg%s~*o&qhEnvdoE)Ci@|>f zX*N(JiMFE;RQR@A19=BVG=<%SmZ`DPfT%NONDVA-9ZQdqF#X`~iISQ^Vi9H~6O>Lx zs(H7&oeG;Fm_f?~b^oB<#^Wm}SsK#a9h>tW{n3H2$)(M*__@!T)$;7Z1P+zcBIZSD z=F;AMXTF<M>ruM-YR2_;QLgh!H+oyDJxw<e&(R&e<Xzqr`8Wd4ksN_D$#bzkcWF?S z6r@@ucB6Y59Vfgxoh$5y(U(Be_q$Nx7=cU&U+swnVEoc~@cP-b=5cj@xdbn*Z11i% z>pMaObXWIgY8w$ns!I}6tw7yN1Y}zRm8j=(?Wn|$QOU<RL>37Hf%+)27<lXOv#)6J zvEKBa$}Ax6ex}K-sNE4u<!=J{Sws$cb<-fVoid6CIKM@OS>=3_`gSgrye~D?)>(WR z5<`Q8dV%WTiJ~cha&4xOzJl?2t%WA{+uf4Aw3JQ(h@<z|+$MKeS0gYND(B4}7j+vd z<XxKk`1<+g>8vqqw|<MRmGnjNvERc*W?uEK9fG)jiT<(i+9M6Lf2*gG6PY^~8TAs$ za02&RRCCCs#z6~?&av>(c)27*@N$U*Vs~wtw+JwSGxrXDd|d21vr#dSM8m3VzhOn- zSGJh{c16=g!_$Y@|AvN{ByAh{AItyz`O|vqihVaeIDGdEfgdB}bMiHwi%{H<Qa>~Y zfoO^UE{#)OP7aOal~LLK*t=IYSm$C@+Mk<y?A8$d@SOuGYy>1Ekx-~=cvO_x^~t*T z>9C;7`ant_?X2<33Ob`hyUDVyMi+QpgTvhS70RilIC5V1%r8v*AD0ZLDh$V$UWI?# z@FK6IGZL+}93H!0ZhK!r`VTGgM-1Om)nI@|QBZsqIi9lQkKub0ML(cTU!<_Y8LM_h ze_#e8U4Tb56+)8yf&sh1%Ph1kCo>I}7UB}I)ZKOu0p24mnQNj)S(Oqn?#hT`z~aot ztvGL+lMYJgkAp&!pB&F)TVv)4T(AtXt|kH>$!quWdhnB8tyY`_phYo@c?B?8?_FdB z_fEhSUs4jq_Sg(PO6DD}a+vJ2^pap)YatCoPzo23MB35TjPcCX`vkxgm+QvK5Y2ir zXj1MZPlLg?VXM|V6A_@|*pSuzop!F^DdRn5?$KQJXNYrs#pZ@M!zY&+<mIE8K2w(Q zW3S0Up7OyrCFM8X1xcGRHbJ6UmOn91Ypq5n@mI*(%y9rsKvH&{;5W%krlUFHz6(Ze zzM8W5C<HQ`y!bSWnZM;AYOw~Hxj_%t4640Y9Qr07Hmvk!Asib%ZlZ5?ecyw}1yuBi z=0>h(>h01ZCBiWAf#&94BbDKO34NU*s0370_bhK?w`Wx5+P}Nc-z|t)i3U!Z+3M${ zFA7wii!6`69L|5a+XO+(xN|KJ5}3<+pUJW-@lD=^9U-Yjn~<Q#hnoy|J&)B0<x_GV zv@i_Zu_>xb>D>qY1VM+npt<XY9Nyi!yKJ?ZHsAB^XaWYleoID%O0NgZkcAenUAsk* zg5RDG4x1w)Acq$F1pS6_1DcoF4Hcu$M_u$M5MZjW6UdQYh7{{AGUxgi6mrktf3HnF zpVm!#GXvRvW5)b1Q&b@NeG?h9;aQ^lAvop`)hP(G)rU~W9ck-Q1iJSkuZzUwNSXG= z%;Zsc8GabDo{O|#thPB>(n_Eb1uBcOsfk>YujrXd^g4dAR``RH&S}eH&Bz@+BsMU_ z$(r(xeEo|IIb?GA{kLIif~d(-Cwliw+@I}UL<@p_bWR!t@zA@r_D6EJp|?F^KezV1 zxK?3)#gv&yk4KxODpF>)TwU}b;jypAgl}oHlM{|hR?wv7pZNSlxT}|snAVJP`hH~n zq_`707W0kxB@|~3+7qA~v8p3fS~o_h+(Pk^y5Ob$Wyr2q2u?#V#pTmz>m4Jpqw&hl zb<xg%b6yn}wsAJr`(LGzP2z0h>7!1=r(z>s><|7FLVl=?zEjgRDeK4hgu8uh{xN*x z#%kyDnv%wvxyD93CttRwUwt<dRWU+sZ*i=fLyNXY{nMlhcrOuTnLDz?cf!MwTx*!- zL~;JDw|Jx|=2=Ld%f?W;kpGo)4dy<RbrzRtC<d-UgTLzNzD9Xr>6Zlj3tj3_?y)4h zG4_OOG9^BjRXHl$H^sj^?6Py!;@MC8Xo4m>!|djo#lvleeh5@^I==YjE~ZTD1@FaO z5U}g)TJnkavA<27?WtbTDna1g$sR7Yc}%=ysGqiy(eM6a2kelB9te<Jo|g5&(6}gr zPKAc6c{>EJe7474h+in$jiB%5>q4#<y4{9;)McUOzzxQ;UZpyWU^c)l{H8hWbbj)& zhk5YejxTluJ^sEs5*z$LB{m}RC3W9i^2?&90~h{i`YvyIOn~Xy&UCfe?*=DhDvmpZ z<yu9+!U8VZj(aHp4+NY<?3n>|MsUh^Mt<Rs8+Yni`0;dok*EDl4Q;?-$%`fqrzkwC zvmh}c_pQYCob=qyX^98N_?C0MW#Q8sxL7sbh8X}7wQ1jQZDEvZRUtgWe>$c(c;T`* zY&W}yImI|PDl+8pOC=t;FJ?&-?MTt=BH_H2{o;o&hpR&kE~V&2h?#k)Mvma%x{DE! zJ;iRreIU`lJjUNN<F%nc>dW<8GKP?RzYzQmOizCc`*rf4mpqYduLSnmr-cNpNvm!d zomUHx`?~SLH3f05_Ja)qbPUq-iGr(D=lzDQeyezkw|cq(?QW|Ao`DZ+8;4CHp}!lJ zi2;w(T3_w0QhQPvAGMfvhe(p5Hz-z*PpUDmt}RiN4c~Oud9kxJQXVOSr~2{RdcY2k z>a3T#V?vQA9)h}6@UQ5VLFv(+`W#1iR+%{lspaT=4|7b>uMb1qJue2xKlAFPcZia0 zV$(UCe(YJ**n5<)W_;b^tAZE`9wN50(fWEJPQ{p?9*MlIk76J}Fr?iH$!N!t*N^6C z*n;+g45~sk=R6b~vSGGvCW&M=L4y*DU*ZnVGKSF*=00F4`e0f&6IhpsEqg43k})XF z+jE^tPBd<8oRIY;Fe6JOpJ6y)8i`nbmefjVxQ^{ZUZOtl`z5Ejk-^ZJMhYq?x+%Tb z6VFGmn_tIO=|Ir{MV&0H1$(PXbKQsH>gE%{g7<+4SAZnfX0RH@X|)oCgu37a!R7vU zB&1-ZH*(S%+Ax}z;P~CkrzXD3lw6MiCNO4j$D4iScRbPzT{3U;l6GVBl7bWSlI%eE zYfWwROFq5)LzpKBDO+l)U#+tl=cAmfG7XE2M8k4wJ16<Bk{h2R?q_?7Xbyk-+EgZW zf3Cso;qLkikHyFM)!9ZDMBTjTWwDZf=O_ysHM0Rc^^;;>>+V$Ll!VLb&%$yl-B%52 z1~ul`i$0rF0idrh2&yQBoTU=(`;!<v_)^I|dQ`fB-TI---IOC;2=ELKKGGE|Q=;)~ zi2yy>xA9afPdoes9A>}X&)QXPJBSF!Q>w%hjdvi>V-dweeVOprNI!hZsCw(N#u0h` zZQ)E&uZoH^{iX~xi)3b%NqsHXLBk4$3ZfD0hQ;-HUi@)qKUJWEYk8HEfti01bwX`C ztW<j!;vB$T;51WXag)5QXf1~{_)wi|-?1;EyuUi_`c%a0pzL<|lGl~W4bLb3w3j1Z z)ME$H88d}@tL)v&{ec=NbutV0INEPmk{B4Zi#{DXUh!Q)4B^liq3HVzAK{m->VI^G zwg>n<CF7E+6piOKW!_A58@8M9@1rxN=vpk0k9GHntFxG(ogyATX{!i#D@#5dJ|uM; zG>63QhA)w}S~f}OM5U*#pNNJNv~Jv*N<LhfwP`gm&8PqsPRT0m0tKrQ?o^(N`<_96 z=iV*)BG5q3y`Fcxnhu&Ym2sH-<<WcLMpbjj{)nKu@6ZhJ!UYD_oNo20rG7+|@Nd~g zSeyd5r^^0zN`3f}L(E4wVeteVcz`BxOE7(c4mB@*+ktk3$GbRJjJ)yzzBy#tt8A}+ zK~Wjre8zS(pe?0(b{GaEvc8Exjg_ok-owQic^P!lhE9`Jzg&w$a_7!VFm=F)V8}~V zUnR=cAD~Ni!g&j$t7qh!i?WfzUdxTD+3!Mq9P?xPNr|IeTBw%?W3jJp-qT069V5ZC z7-?Uxd+Cx@?l-d_U*N2oNM3UK%Yv-m1YKmWkEpMM;qm?rX4CMwa!WNE15)wu`@7qI zu0u}B4awjsdInG%py5WJecG=#v_YAv;1q(^SOT#y5l!bj+tzP*Vtv=3h_tc=a^E&g z;x*=;a2g{Fz{LA8jjhh*GNAaB=AK@}eLvzm#~htg2x0%$a-_TRB2Z$k=Al?O)$RTB zbz|LwCDR2hn6h8x1gfEjUn_{aPW)0k^WN$!+tkzDdhu7v5f2FJ08av(l^5hDKwr;E zmfLtSDC<a^SYOd#iZLK^UlX7WKa*j#yjkD8l~Rna3{|Z}_&=%9-j*X7&08V8S^Ruc zoT)ydV{*6K*+?a|pvk%fH0e=eKsg@bH)#~f;qRXIYrADK0pA21PGMNv*D0^7wWm$& zW5#q&i%XHFIL5NV!@<mZ&k_ZnOndhSNAFG1o+|(&>cPVl8orvBw6IC~orDBJaBgm? zq(tBY`WE=X*`_!!=scWY#ao8Tspw{_4eJZ!u%Un8Iu`||8Du7@dQK6-W%Cvd<nOF} zq)^<$Z<Apci$CU*qbB38Gg=RfmOkn!rbBrq!y!=?v=TvjhUy*qG6#N<fQF7{GVhwi z_0_l|P~<~w-oX6Fl<Ou##snf-F8p1_Jz61JmX)v|6nvs^XSO#eQN_h9xTp}v1@JCI zxM?nysg6L1JYp=IJK76mSv^8&%n+gwRm3rXfP$hl1aRBa{*3XsnK|(*`L%%M5E-sD z*{}tWl;^F(N_T{>y>M>02Bm^y(y!vzBRIQc0`_0xv(}iqsOjiTiB?EA2jf)NzCi4z zlz&i;h<2rr$Mo@YdF*K5dNq+WdmUD6ep3DH@FJSgWKD$DoW7B6&6MhHCN7@wx$PXJ zJV!mX&llmrM%~|D?m~DzjRRN+6Kf9R(CQ+`ZKUnlDj6Vc`VEc%6NHa(+nMX5R5RLY z<SqV;hj73p&O7@k11{uZB;0Fo+8Q~`Z8h^F3V&)_=%O{!n~DfZFzFUA<P>MU23T%h z_I4A)e&z#mRc<<H#M8Ni=F8HQ(;*Piadu7pu%69Wb~}Z>eo#O^4Jfde>@v(2@u>1T zG%}>lr#(HEq<7{N9v$+#*wf8U-So~{!rw#Pwf$WX@vulA#|T|(UKcU%xXPs?BOc&h z*4yx+oWc&;^g^5B-u=+To_y}2=$-wnxh9uA?VOA1c62v*H@i>hpe$@%SHbq<e}w{) zwr0-R^OE|=Om)1|-`=Jm1LmACYk&Y%r~rP@r5a}k3RwrKw|2_ou)N8FTFT~20uYn+ zncBHLHDq~>9|uAyn1fBCwoa_E&)$MA6o@Y5#!nD-$w*&mn`XBqd&SWTVkOXq`t(?w z8F`T7rWycK(FOUXh6tz=Ja-RFe)M@cKitUfJ>22lRJKKaNV-gVC58tpbj){M^Z5L< zB_61j_S<6JV(lYWXW{0Zu^RU(s|b5XoMUy`Z?2suC!BZa5@6-RHj>7njxhSz%6wSO z;1+ntsr`)S2}kpj%Lx3$z$El@K5Mka8zyQ7qxJ|AZf&T13XJiF(6yw-pS0)kAhiYs z@QB;;EaSTCmnf#~q+?u&Y@bQ@gzgtUGtOR>^qAvhtF0wwMB--KjpW00@o}+H5uJfJ zYU0SR-BDTTP}b0QnA)1Dtm7PPV;0VQd!L@rrS3RkG$1wJY|5w}6gZ#!!VsO~d#r;| zHlwcrO-8z%&+wE3AKrLg0Gs2|Q?g+O5)653{w)}t3|#nkHKgICPVqq)jvpVtKHlaf z&BeC|qBv^^PLX;d^##5SBr-LJ6L`2?f-qzRkeqV49WUlH^3DJzhcC@;ClIkBaTusL zx|#fyn&7#5T1at1@TbAJeZ2N|YdNCGAS?|7%ro_C$;V&Od6Z=j&d#b1WYrYjD)*+& zCk4lg{g;}ChQ?Y!=0A%e_x4D`>Z}1A9<_(ZtcZviGU{fDFA9tXb61i_3La7`wXP%u zPkX9!P28X&gdewvDpZ<jxks=ZWgLe!f{_`LPD4HiZm&E^(JObX0GB<0Unyv2FBzf& z_tk?Mi`zQ_#r6>hD&CzO<kz%0_-ohCayLfN2*YL2-Y{94yHn8gpRR}@BX=>FqS9e7 z<RBom3kxSb_onmdFn05>*|6Bt5!1-A)f+9zlVC+EG`M)!jMxHXnm?mo$jAdrT(VDv zy0j4^D)>IrNzwT;Dw@?jpZg_Nmf1IrCxaY{k0P+U&p7^3$PtZ?QZN)ZRO}*YIf6Du zDhZl&|6+ROB7;)F4%+XIrWEJRz`oW<bda7G9vlN7=H8#KF3eHrra++{As=MiGXuzQ zj@ZC^LHrY@GjVl{TQ9O!MLuW%LdX(ol_BYFLx*A`*i%@;eD?T-8(17Oz)($KQT5Zw zu69Z>Z`DffNhRojK<$1w&_MEz+G2uXWfT!3FUvR7Ly_D=l}2FK-pVicR@dx@HSL!* zfmk-9r`V@3`qHK_rz9$MQ-5%;T?HZrYlu*pawfRo3FX91zzy6Mu_uJu<DRYY_Z_0i zh*U^YZ`MoNPO#vDu$Oh1HZ!Jp7`PG6-nGSZ-cw$V%}@6xpJ@v}3y-$OU*qG10xLog zW1+UrhlSyK46oPpmc?A}tM1vuekY(+g%2B~8HmR5*iV+FKdhZA9S!V;!lg|H+=t#_ zjUKXA_{0-+)J|)m$Ol?gDW>*axKa_YxGL%}^+oJvJZ^`W_4%XSl#Tl*Fzwc7v;8aL zLnxk^gU)xu7}6nP7mT_I;!rOw6Es;(UM3sFjz@dKfywHC1ZcMx+dKt%yHfM!@>}2C zd_*smtfe&vp*xxm#IlPtJ72yzce>7u@l~jk8RY1%U6W{TcgM+;xF?PpUF<uiIWW6C zZXs)flRS}MlFvRrj=o=`Yj6962zAE@1?|&a<ah^r>JvfD1L#Q({DM7bmZbN(jE^U^ zy<C=CZdp2*kgpjI@~0;Cr?0lD{QYmJ<BMy&f_zAmy_RGkFT6aynosr|Ti0(3e2{1b zU8*ep=(9S2FE|~T9=98-1+0rV7JE+3*~{?-2GUWBb~Hwb8!!G~wmR76FfH0^j;I-= zrnP%-{(b+;7dU<1aW3Cd!`2qI)1$rhVm_u(77r0apNe!W-hyhAr}}4hW2zT#wSn7D z>Q%oPxw-HCRw*Kvw7@`=0N{3X?!DezOKP3BsxK5BvpAE1WXY@6I-2p>!mvTJ?(+M- zAXUT!YM~}yy6jV9B;LojqCQ>2%xtWYd~yx#ukut&2CA$y;I(oBJFlf+R(_-~uc@`o z$7s!T&x?a%y`q?$++`Vyq(%Pa?R}Svh@v3Q=WLp4!%oEB=o>!1)U&@ZIRl$i+XCI{ z4(8R%lfYFBj1VPeWebEq)r-g}T)aY7@L(XX9*<UDgOp6tTd&>X-=_l9;#xX*A)<!9 zI`<Wh52q8u7a;>~SD1NwB6-&W%jLYCzi~QMH7$<_O;+_gbDSoQ)jPW{Lsm;Qn#()q z?v{Ys!7Do$TC}~>>e+{{*Ze53&&Y^{52$F<Zo*9Wt$M4)ckn?4r6EZ>SBOrHOF>*H z*|TE`YXjeBEUn#0AnquQCh)<Oh9vVL^4=z@*&@csJd$SsZryvEn?r$41_nX43iR0y z6!&R+DVR!!_NEsye*=Ja(!bA4Z!B4gLDB|zbQB_j9E=UXMk4!V9t!NHL6HLfNS||Z zjg<*~B~iQ?78r^iiZj`T`G7`cP#JC>XU@>bSVm6^cQK-gQ>CXgy`5GLeGUopAkyHe z#5VepU{OmI*Zn~hn;9{;qKqMB=ry>G3FQR}zT@$0lOO$Dq0hTTyD5MVE)w4NZaA7U z(kN2;(0&#Ledy~vnX18RrY+NO(uV9cZ-NV#5v;^7X&%+MS(rq8HI6bYlU>zJjdY0{ z^G9Tx*=k?H`@`REcv0u$<-cIyXkuzC|4xqmt2Nv)?xVwWRet1<J;mII2kC)x`9yiH zU;I+jLoTNG;Z8Ojt|`0Or#;b9G6EIGZtqr^W6XYS8CG~Xz8Kt%TZd*bqwu-mXoZ?r zU>gaxG6|<DDs=YN|Flb0jYmqdi}?*^DmK%DM5slfHBq`b2EA>39Xxu7HNrg3Eh$aO z@FYFeBRwz4EtAmNypFp=TM-cmmpQ%dns5K?u%2PzV<5=4!Ygf)Dd2FHTD~W_YqfMO zE=4x%nejUrVK}JJ!W0#}`?){NDE{#H;i1?#d_Lj=_oC<n`3!Hpp8Vl%uxk*dm`w6W z6!x!E)He)qB5RF{Sfp8djKv!~omLu_=5Ftl@#Idj{h~dkDAbuW6k;u8^n&D~S_p@= zn9B<SHpv*-a|^FTn+pe_kt<^5@xZRdWS2(w!BBD<By=+(sP!PK{q3DZ{NvfPTNV8Y z9~@;P<<5|Cd@lP<RQH9{;OH<C|A*Qe!TSIVL!GMiFX&2wrzy%L_L51_<JvT<(+T*; zUFmYI4=Ly`w3rL}Ah**2tH|+zmGPynx)-F9Ov}(Fp{G8lIkr~xw&gz29=Sa$&>5jt zza6?op)U8BtM~OU_QJf)K>`c_j>e;R^znCzd9!KJZA&^Dez3!(KGUs>sQ%J1mooJX zRAkU&U3Tm80Z}lZe%EZI04t7C5Idzbik3@(uF-olxXq4a+Ug`CU_+|$%3L9S9$l^6 z(jm^*g@;Kw6C`p@@ncZ3KR5(pAvqi`add_Q+;*kg`t8T^eb^{?28Yz_BGj2mSlIT? ze4@+{KS$SfJ9<!|D#`gXf%`l}JtaQ~f{}!XR^w;|x@M|mkBCSn%-D|)<a&f#y-yb# z_5HG>xVeS**lFUvBaCGFE;ZjW%+lGF?%BD(2ye$Zq`<QMYyWTiCY(aw*HoV?4TwFJ z9f5`rhpFr2Su0UTOHq$GGTa?>QBei=eO@d#lE^~qb&>w95#ehp>tf_uK?I&m#6bXt zStnGj-?)`>uaUTH#%Rv-OP^-I;>?r$&0tg8k&Qx^+SIDSEo=L4gLNvQDPj-BtzVm} zlo%c;vfXj-qom;r-o=^m$bef%Oo=uVdpcW=?zVd?*s$Oe>qczv%eWg<=>_&BwcKn7 zGDr|`XfyBp97{{^%;u0nE$UVo)u91Q;l&8m42WN_@$O`k!u1$GJpPD`)D(47JQd<w z8kdXBp=8*ETc)wDWrW(=NFj6?dm{@sJipbW-NAC+RvyT6nWd)niaYL;f7Ax~Ds<%E z6wj(@oSS_W3A({>-U~gJ*-o7UNdPI4Zc1_mgRmzItn3eOdR<J8@L-FEojrV4fxwT; z(X=tpn7oAGMEq$RAP2$SGvHC{bv)1x`dX@!zkf>}C`!Qc%=QRb0-1gg9_CY_1f(Lw zIL;SGCZQQQ72#@FWJKdyYEgJCZ!K-o-HhFi1t1UUsT*=?t7ptdS$RIndlK!y7`>7j zWlte3qxYG-t=ENnDW$KNy_>jO7lrB~;nX@57)sbJg3xC!g3=863@eoRxSLV5lX2`8 ze5AYZE5ZWjK%q)Be<o}Dv9B?aDp`Kt&4jbOQU`s^7Wg(p^?fpuBGGKiqMYNnVlDmB zBcXdg@Fo4x02F-IdD1L%Rx*MYQAuUt`C0m@eO^F9s0)SRoJSIgc{@&GX98U12pH^y z1DJVrkvm0bO5Q-c{)J#dFYJws$U`4;H%~V=ieWtIX6{r{dOW<>#{!MflunxB^nu!X zO=Pt#a*3ES2zZfhF@_cx-m7Rw_4|qus)HEd+$Y5_g>TA)3^~uLWj7kVQ+al`XsYq* zcB<LKPW-S{U^);*F0jwp{)$Rw2H}5^6Me2xJk72sOPG2V@(P(gpMP-OqxW&I(~Uj( zwC7M^@&|{Wu=3w42euNnb)^_WOjEJ)23YeB9rG43Xzj*Y$b<0t#*A2%oS#Fz*^X9Q zq<Ubz0%a-#R=5BLiTdRUnB3mojzpwWM}(!FW8V6WF*N8_DXaPJSy2U2fQBDyg<teG z%}+S=TFCxxkJH!jo|t25p?g-g;duX>kiGEDvjyWfa+lg6<$c(<?B0d}kR^;YqbGIK zAG$J`Luee4L{$KyJ@xN5-}W$lSTAxB3Z%p1rYUtbejv^^YK{;-T<SW`T|GW1@m@2? zhz=2D7dk0KKg{(Fe$(Pqc>aBEtL>heUh-q#Pw^P6=tvCE(+8m-s%cNkt$Nh0Rw^Ag zvPSp0&~LXyY9FqXSIUEXgT_YMo!!snV`@^k?1#`4v-xRKz_18)CVX?}lSrzrFBnek zuPXyRUGamRQcb+QZ~s*NvEHM!a(#IEF{L8<>HO*hU|M__taz*VP+Yu2!RrE$VKC!z znPorQ5k(SD@x!NzJ=)Sy%&kSzlmT?Xm|KurG$cf=oP-+DyJe{gdBM5-jc-v5whe1L zfAL~*=5ztivTtS<L|b;)8hp<uocaHhSZe$5`dO&2yRgT#yS~#<NGNl5$Jgj<I~y%y zNVXC%XZ&I+No7dz>@!sW2B>qOo<gm|heD{ud&_mEHx7)OuDLLjHp6x$s9H3;x<wGD zhpTV8F)vRh;VZ^7-O}}op+DI1P4K6E_tTN@r!C&a?c5(eZ&D9ua^uAx=z=7w%00=U zhY@Pgak??@`2k%y7KA%9@hvtZDQ(OjpO;2g6BU_7+N%6|0y9lp{6Y54N(F}ZsIcGv zwU5Upj@P?W(u&|IwJ>!oeZg<AgWt5M5c9X;{DF}Zp)ZO%+8F%3jz*Mel>PlpuXhw` zATf_jkC)O6g`^3_XkrtfmJtz?TWnN7MnXFxWbY*a40zgDni-Hg1#-~vDO>@0j)_XO zlbI}$o6U{C-AM@9H%1lIAisrGd%i*q*#QdbEr|WVCu17Q?7-JK$9CRRU|>zktG))H zsuDiW_#losih6H^tmlL#;MQ@*Ayt$HvX*U+J&w}vYz>3vx%{AGc#Lb4lJ<a?jb4hk z-|H`&*~2!@`<_~RUjVo^4B_}m!nYw^PDq0i8huVEnu4XCZQlm8wxhro9Cuiaw^O+H zlB}mxAb%^G1|_~id?g~JXYZd0d`AOdqkW003c1Gqj0M)AF2u$r!z%p}nU~Z-6#peJ zX$lD$hk<z5tNu7dG9Nh~(1kOenfKeDnG!jeKiX8cEO!%2nJn9?{T(QgzPyY1W5DW_ z+ud{!3Muc&$3tC4d%8RZf71zu26x6Q+F<U6kL!&@Ea}gn06xMampEN&X$HEdeVNaa zmsBWuLm5UAIIQhw>q-_^ldlyKI$Q3GS-jvbGa9cn!`x36O}ATvZXhCBK@Uu;R+`MK zMifbixDJWPB(2v_I{yB};XcCz)Z&p0Da8hlf0|}C!t~#yyZxyW-@@Oz+Deq@BGyPP zgz+b1U<UkRx8Q`r9goRyi<ORWd<U*XHy0JHLLU+NS4%lKBGYhwD@a~FP_C06ZwK<w zSK#A$RfyBX*;xKENHFB3sm1HpuX&*T5WPzT%v;r`$FATdwm9>(yHiQtBk`P<A;3wS z3JQb`IPVc5ZyT&O3u1x9KFd5my^U7;<Kr35^UR{P-KmhFoul3Eb^N{3>Bh{L1Zqt^ zSc}F|`-G)+e)<Jy^!Dl_!`#`VCl0^SI^RC3=BOa+V+Qk-*`!5i?~w`z!j{U*_B6D& zCrn9p#CjnPZ@&hQg}m<RQmL!+CPeavA!x1FN+=SK+<5ti3v#9P+E9~qi=ibCT&s$o zE6eF}?mokzwyX|o3MBr1u+ZtSn0{(p^QRU^k;s6A2kWl2OLnY}DD#^4_1e15TmC5R ztw%__i`VEJ0tV7QU}@a$eIAszD-wat>CvI3M9`oj^r%-zG&z>dnbu9m06<xLh?ZTT z{Pvf|K8UioXIJR46M3FM>`4_>XGCV_MgoyqU<#sK!4&hAIrd0`K(?Q0Z&rjg^y>>s z{^uuwlt@iioihwbK2Or5LCqk7L~+TifY9INhbH$Az0Je3-NTZbhhZH0bAEKRR;Fiu zri}Edy<k_%g7@_z&K{RCmiRP{7{r~4zlo~MXWl*~;{o{Kq`vnNGs&HyXxFXDH%SK< z!>WXBC+#BdzLj>f(c<rUe9o=3)%SePu1Wrl4Hg<?68O2qfCABc2<$3YUxCv+%?-9k z<Q3^{C!Hdn@b55i`KbvVwIwRIA7S-_T3@gh-x05nenIVFosfGSw!u<4hQ7&)#4Ly+ zLW+iAA)71BfD4^2<Iggjhk5(bCM(douCc|Rz;@}w*R?AXg3HiX5?{vJwn&~73%DN& z%3aza8YtEV^f9jAX@ncTM`O%4l7>OR^%N!5jvoov!KDI30K%u$LAIP46n8bmQc14- zYK2DX8mjPAeilp&Jc`&C+-zz!mS)qu1;>{g4A^`mnjWBBF`QbG@O+JL8CA7rk{Y6| zsH%B7<QgTg*9k67to2DRiW5bO1{4qD+27=G^SQokc0c@fqbAiL{58-Gjg%e7>(B&s z2z+7LzlJY3YTuvKg2o!fH(_&<Qc<u$$4rUUfhO=E`zH+%`ZIIK(%6hBQFh^~x5L3c z#P>Ak_yO47aMZ6N6<aQq*vuV4!}qX>d&)iQ3{%T1ZkEPfMAy&~9DCyzcgaO)av~mE zcnOw#_8|n+7e_JMV;nL*6&zp%!`;P%UmjKf@~R_l7!cKL31L8q(2dmZrkE2nPr(6g zn$64B@E~TyybZ_?0AB_H$W8*C6?}rxYJ;RQd7pfO4TB&un3VI~!E?PvO{hPqG4vcT zcFHgT#V*qxGTE`1zIzxMj%s}q$Nvv^Z~YeK+VzdAAR*F93?(2S(p}OBDvfkX=g<rd z(kLY&-Hdd{4Bg#5Gy{xucMS2Jz3=aHy!+nI`xm^w%r(bx#aZiIYn>}T%WwhrYYGW` z^aCL!p%kGo;k(%<LS-Cc9!z`Qs1qLfJ6Yl%1X#{UxsJjVK3kiy_5xZdd?`*TELn6y zbb==YNt|?1svmr;Mj=DtY#q;uUlYhw=!=~xWAmy7TUq=jMs@<pk`q-%_<_0vy<w(u zWF1=^Pl%}L_g0!57wbydkP1gK9)p!!wSBMAdEw=~l^#S^uU~l?j`fAh-rtvG#U9Qe zZdNEy-Fa?)0BI@dW{nN?#;ojPZQ1-Bxbczw!q{VVq1gTtHh<o3{u)n>BzoPpCg^xD z>u!jjCSXrL_(8-_FW4b?o04Zlyf>jO(%LhvBBtVpW2DRN>#;u5Q5=9Ln~T9o3Lo6N zucjp8bZ9?Xv2FCc{d~}^-s{|^S!qrQ8f5$R(8_EMigNmn+A-0aVdrCydr}?EZ+Z~0 zx^IoGa*bK}9TK#-+zvf(8sh@4eI>pOx^oceb93}Yxe;Zg?tYsg$YyVKAz{-!lNozM z_>XwTvtE>SUp1)QF`f~F3`$H^O|f<KOvN1c2h%{fiUM<Hy_?O_a9a_5BqO>rE1yIT zP}er=g3PK3JFOnC$sHCbYRP)F@%^d$8tbh8j+l*|SGYnp$(eWOBM)(X|8Vz~maxQ_ z#dhfE!I#5hZ$a;CT0Eb`@R4%8Gj_X0vG^>f>#%Qtvs}F{%V-j+{;Lspk~0rBLi-|I zr6SzY8R>jce?G%jmvL72nMC*FVdY~%f40ZD{ERs_H!@v5Kf_1aU$6Crcr>99a$W!M zZ|pUHd&Y9uCd9fr4l6t?hMvva2c+&~>Y<Ohq&|fkR2Nhct6E2K8KUg@KucFtdX71l z$CY3t*(seqcfHN5d9Mq4w1o^9Oi^Vv9Drh#CP(3wSbZ8;+xzJ%Y`?{1qcQ{SeKBaY z?um+;EPb`WZzo5dzAT<@FS(AW{Ble*1)K#$M+Unbsh@k%8(=W~Dp5g0B{HeFMN4NB z#mBa{qXkF`jprCxDkSk^R+EOJ4!+$xn{!rERcKEFPwCQ|MQ&Y4q~Y!KoSrxOuG?G^ z`3xT@g#5VY-S<QI(;G~0e<knCnhYh8e$TiIqw~(@MkOAFNp-)?nrX`>J70U@y0=Uk zXE`)^K9oiidQg;kjYBFY>mBw&0-P#*5TLeR1U}3%_S)6jNELG@yul|ASqzcy)Pj@) z6PJ@{NJ9813_HH_QE>T9D7YSqV1dZFJIDJe&M9gqOb2cR`8kx*LqJ8%?DliKkqU=D zbRd<y!k1fN--`VA#B0;c&9*(_`E9P>{wkT<N|rPw#03bEFj@^gXG)adqbcQd-g`Zy zj_544%O6UKF+>Zc<5@7?yujimB2v!a(kjj765dY>A-ti=q&-h*8x$zsNi6omH=CZ1 z;<7tK=cC3suTxg@)OFvBmf&LqMQ8LcFx^w!!ZD*>WS+L%zf_S~>SO77LVWQ=MClRc z$m8-N1Y$aBNq#A*!R|3$I?p4jZe@Fc`FjR8dNl!`7%P;gYGnrq9%5;IA}{vzr0kJb zv&mlh{i^2|h%n3-lheHQ@oh`<vq*J;3X9@^&-_`jCR22^w&!XB4S_{?T5iP4j*H<b zQ?E`)I1{r}Ax;KFR<}EJV=|E1jD~s+52NWjHithwNZLUMbQmSiD+F*lOrZ6#E>J^2 z+YRB+gne~}iS{Pc|H%uzr*#Ds&hpbO%Iv>Up$Te?R)5gk?W?W+3hTRGfk#@Ib<xt) zovw;J#(ex3oJ5x16NdCrQT{4Z(ASrfz^1qJhF0p4SlXtvPY0`P6<7MTpLiglee9<n zZB4=W3WM$yT+gB%1G8dPe+H|e^d_E(?<!H_n!%4)!`yzquUtwaej+JFCHvKh6+0d) z7fW3bpHg1xr4;3eX>T-@TNi0>tVHyy{>bNilJx%2r%@ZQ!Zdl5-=j~ZLl1Rv=baCf zmF*#VQ5gl(%jb5EL_w))N5be85EsAn@zFKWQ+-;QHiyxp?EF!F%b)Lg)zIZA(V&^; zPB;n)%7VZ~#)<E<_9L|A<c2s+8+9P;Kjg8Wz6=5b>KTgaLzp%bP3`SThw9B<P?u}S z@)9Yad<7WbpE^)Jt3UJH^eDdRoF-J(w1KrzB1vz|N6Xa8f(4k<43s{-3yCygJmP^* z57_?NZEIpoPTXHh$hE_-Rb|*J)~~NJbibuG_D{|PDq{0GmsV%$3+OXZf>GSj>dn9* znjy!Y_~45Bw|d35$ZnL|LJ?+uZ;4&jUS`m^6RurmWXOX4J@_~J8v0N4sKsXq)}Nn5 zQF{L~M*MG{I*xJ(;$Q+wo-SWu2s)7qI+7&aQ>*y#t&nF>Mju}ey)|k#Z|iaoxKu)0 z4V$Vju*8fa3}X}Yoc6$&zvs)^;D2>uN-wz9SC1^HyA`9{VEnS8v~o(rCg{TOctJd- zw?P_P+y4#WuGpux(3k_$_JvErs(adjaxMvkJ2cCw`l*QWV{av$vF&)7x=@O&%6*KL zA-;9_V~^7a4-JdJw<4;(VqIPu2jB4za^`nr`1p^pbbRqYUv#bJg1=pD>C=rE%C~K^ z9BZE*qx}j0DB~O0GfC3q#!Z-R#J}^(YOiHF@Us>=ZyvQv@kr!Ifp)I2Y^&hQC9lgc zgPTZzjM>@q&~*AyZ+fcQiJDWHmcW->&bR8kz-dDA{iSIqfj@6mh=M|++Q(uMDh>MP zzXw;+nlfJO=VNf9{gtiAFsNzVF-87HHRTX<$m}aYp3`waknOAQ35CHw*NZy@<k>xt z>k>;nY+M|?M1xx8y}xvDIDHM@<Y<+&FTN)%AW_pxBup!Ro;R%X^*q(6r{l;97iTk+ zy<IKdey+$ZTptkeGy;6IRwESt-5_dfmd|Ruh2j>iD%&pPZ4$Q&wWh;tV8MDZAM0zg zh;W_kCL*ea^Y>q%i_Bkp6#6pqRn&qvp0R$9Y@ZUsA#5;9{#1=^;DsNo9Nl3xRuvZ- zQw($HaGp+}rSAnR_8xfcPIAArwUPL!-FIm|l0@_UgNB2)`g0pTyT{&P3P(qyW%Mfb z(|Z#yZTWtriOJ>aL5WyqfE`uwwyJ@8`C^z>l$L=#-}zicil*$cRvBuI)2rX8MXl4Z zM(dXKmT81hJGwdt_(unl{hp_8v_LbN)@Q|v=rH3`a?!Y%-W4#-0W5Tw7j;OMR<Gq2 z?d<E2cYkb?C+N>K)hv~}{_ZxN#sKWMS&%(8+Izqs`{3j~Re>KMwG(=nZ7=mcfqWqg zqql&fzG3_2_w}f;a97^UW8#MNW;}&uG`G&DMdjRWh<(f+&nM#cgTgm#h?oHV!GM9{ zC?Nho8MnmpXzlYc!K;^97tdBGQcFKGYhFe*tde~8q}q7&Qe!#B|CWskJ}!v-+Ca{Z zuIX<@nJ?h~*eyul5#553Wc(pTyXf@v@QF9QNu2MBH?wz(#i9bXOlzYBCA{=TPie_# zvKV2YPBZ#&q6eQ$!HUL;#lpejwWCRLi6YuMS``U+EJ<Ix_4+w&)+<L&IEP$aulrjf zGA>bYR2_tFcZG&H_;GM-WYQX%U&nJ<3qpMYRxXb07ukTvL;>Wd#8RB%DDy;aQZ)W0 zFq_Dq>CqSO<Y<Brl^&9eyqA^Y!BmURKfgkJkry$`s{O~NG(wnr)#U3RzIGn3&+_0) z5`d|bGmgSmejI%hM>$=!QAcj9E-ofcZuoeg5dE033_y}#p|b!|q)}{RWf~~!qNkr= zdlxM$YfvSME`uJ9yx5;y3GKxSB+r%oDpS|Z*BQ>+K`T|lZ9cGd`3GeRjK-OHG4ouK z7!?&oGe@$(7n{)K^qTdxI^y<NcjcB4;C$feOZdknV@_G>-4lsuqkN_GGbZqRA!u-I zB;LksrGg<b>&yjJRFfuo?gy-)nqX@m0mhklY=CF^T7$`T@I%RrK7|iHO91U-zRk?j zAup7xg?rfD#u-?NzejxPXh6cMts=*Gf(mYp4Y;1w+;vxXF2m*)qUN1>K7%(S05-(# zxL#^I?|CD05w;s>oAKZj0H;~J`E?X)o3<Nl+Q-+q`)%Ju2mLptdrQ~^CH_UMuW2&l z%DQiY=_kkjdSl*#{mZ=n<A(W`p*Z>kaLq1z=$N&Qv4X(qu{V8=v2}&9Qs4FH3*$R8 zzd&xwbMKp{65cH)+b=3EMY7KM!IZMX^RQSVtH^hD?K4_z+DLMODhw=rlf6CAk@w0B zlH?5TM0#)RR1_^k6;*e+AUNHlho$R*>4$?@5x8XZAvFlq>f&%TqzqdtW>tO~MP<bI zDl5W98ckz!JTm}aX;~Um!yX#qzFNCNeY*iyTQ>|C{kePeOj%{W1+%=O_(<Q8%98~C zen|tL^L!Xt=O{|Z#G$LA5I-p)VG7_1kd|edDuoloSqw>{;Enjq){vLa63ar_n=!y@ z!=XpdS76|CPP;Ri_&gDP?ncJu{4ol0(dul|n8##XAiB32rG+QLjO9%4iFxy+Px+~M z9Rd7a@1Bt`=glL2mPB2Q46xsSa5xN=djHAv&(%kO7YfV?&v036-i?B9rVV%&%>2eM zkGpEUuN*E^XFsW_?lAUTrwXHUp_jDz=4__IZY$|qpK9GNF<7OZ1Mc7r9`!fM`jagE z?g7BXF<EwYOnGKeCq;`;gz;6KweU@qahydTV91?UH%OMW=5SFOtpn7I*)E`EyaZjb zo8zl{u2=QEn`P>2qEXfWT;D-R*u2Ni!^4aIvxR^tHG)Zb=1~o338To+$fNkS`+MTl zm-yHyC40#Nb#O~)0eTQBy?bi=w%Y9dH2{~l?v@92KC?O-Ns(_Yck!D=Y2b_0Vmi({ zet$T$$vBjfL&!A5Y9Z%?1~1U$B74=&MZijf8rb98f_afY3;>NJOGrG~xdB?MnOstm z_pIGxs>G}7srU!f8CdLUc;~txnf3{d)xC67AdC3gUl})Y&8>@=cm)(!C-iM;_>m7> zyq|r03g9sY7zf=<n7~MzcX{@qgV9eMic;67MI-`W9(w-x^0f_UgoY#bL<*mL-T(Lf zmpZS}58l$gEv5GasJxZ&&|f(iI-|%PjMhJ#6%S37OU221T5bnN%54MS#ZCSAWY<9d zE=sl5a<ROfyS^*<2Z~nStaabw7X8Lz`;DR@h!I()dmPBT$%n#2`i`(hmrA@~;2Ux` z?2d+*CXbltI9o*W2U`KH)rzMq0pUAN(HRP`KUU$D$PCrv7yeAFU%Du0wusG=+P<3z z!`d4>GRAEfDc2xvVVCFtbe&*)pG~@epyW((Vi~SXx2`zItgMvs6Z8aAoWhBeS@&q% zRrcP{Sm2`OFG3$mOerBL+32T}&hLbbg5%OK1BGQ<jwdM{0uAm*JXe|se>!H6H+R2z zFPlbKpi+{0I?xo%9;^_Ffw(2mP0@=d;^&cyCNxKvw!<F=5q}_BtT6O5BD8S2CwJ<- ze*MNgm~r=uI^!gPQsv#A(I>dALT5NcHVUwP&1m(hcp*=BFb&htDDn$~!a7p`og``R znEvJMA0OvP&m|{C$6-`+PKcBhA4l9HFK)QphhyD8NiKe#>CRL~aW0*v7V0cBPkIww zx**-o_3T))+5+O_%!6KJ<R4#s#OihHXwuIdQtlfPTKCsp0hsc#_-_hxXH&OYYKOVK z;9h9EKN74B$xV)yNJVs1%~D5W?wSgrSLuV%V<>s@xMny9Es{&(_Ral;K*2xv=zB9y zzucpR6Hb>b+_G~obUJO^_)fV0Bj)8>Mw#W~D0F|nRS5*7ZJxf(F#f)Hu+5e!AiFWD ziG9Z>G~^@R(}B`QvU^WJerS8*m<k@F9_N#X?|Jf`1B}1QYPd7Hv2y~F&Y~+R=G&{C zu;z%uSO<wYPaZkmI@2wSy5SQWwnp0*v(ug}r4}7vYP(F$Hq->V$spSQWI1ur1B?`> z0;r!K^btPoDry*oSA02~x#YC&#eF{O`qbs(KCrg6Ii@+E_y-FKS^Z4l<p|T4$?-<A z^Ao-^U0-q+(9)P*z27~WiW%gk)w}p`_Hu1%w5Y|5^LTm^b&|29XKCmF+v&pc$7+M1 z^r&!IQlq<!F3tT$<CP<>Em5mF44Gymeo(nT^&PcTI((UeI>7TB3a0N>lIb3pet2ZH zS3rOKd`$bDkk#*WZ0Y9}I%PIg3hB*{z)VVEa-bHDF-1rx?m>nS=Sm1yQmf!I6tI-! z1!X;_seM6o$?3(poR^&J7YnIWkU^yTvx_Y>Z#mN6%7@`zQF^az=H8+Vna@zbAUqO? z#Sm>u$n*7LSYUd(O<Y`<ABSo0!QzE0cEdUKSGru5?9n#Dn1CKa(;j+Z5MPgCM)ZY& zH`q6yeQP!sbAw&C1k7SPWs9K|^D21Cx~P32xPKA_#{xKwJn%}@wJ#j-!o;Wgc0tvc zl|BHkcJPnq2goOsOC1C<jQ`e<8^Vd`O8WL$wMyaXY$2EO;U8>L0i&T5n(w)lwi({L z>VZAUI2|fYK722jHB^I!zkRmM*6w$%aOo@Q6{=39u@SHif>6}vsP2wnvFqdAg~hy= zab*=QcIy9gP@4UcULzHvdD`ImRJKL%DZ_xKT&dTvTPk+A_c=wVCQvxbWcUVaDB&|i zZ#ag$`|W+|-2)ncSxkySp~;K1&T%k&1*z(5sM^{qN~;~IqOw_U+1C1N*|^X2T`wD^ zRl?}L%JA>tsN0~RwsC-(Zp8VKKes_k+>38k@gwhg`H#}B-)LXTO}pG>s2=7n6h$NZ z7y97!A#W#-C(+a2jew5p-0H1WoBC4^LGsvBM~{U^39XXwx3FK=9Srp`4){2Zx(#EV z+}4EB<s~KddM-zX@|bT}q7%*VPU>WC-dN~aEnv~*F}!-Ezrr82C>FaiA?&aFviFDS zXeI3s?VlAgms?=DXWqKD<84x?X*Yb>iANH<clk!7v!pXk%Vk+`WaEZ&Gm|q6$n}`1 z8)6DGUSb?!9{iK9BXD}$7;`!VI!a5PDe2UwXPxoFt+cy{SP5Iu=LQV>xEJi}$`^Cb z2=#`Y4qJ&wC-V95n2)3iWo$H5G}{je)ri@TkdG$M5R2<1oO8L12x)E1ztaAbkQ)hN z5d4EA!)C&6CpYLX&<<2^VXI)@*nEeIB;MrRMRA*5&IMDE#Ep~+!x;sK1<iqy!IIt~ z+cJ*bs^L=O;mT@*)*)L%Vp#96(3;S~;XkvMMH^8ubSbLe>%m{8Y`^C0e!ZpvEadxn z;LTnzR9wB{M6i8HIcd6UyCCj4uvwU{C-jcedL>%uqlaV&7(VK3J7#Iy8U3nTq~z{4 z@>xi^r05cJr9%;i#4jmat(dLgfv92|XJs);#>bt3x|jW3+cgFL@%nq5jb?|v6-^Li zT<KsQs_vVTRe5j0hrtwJkpDHhyhxbtQD!&BtX=WIlJ8Hw0>-tebq}BGQ#2oUK58SQ z$}i)O`PfxuQv@_p9LgD1b|ahwPda^l+?c8J%XA5fdgyFxicBNkqLQ^)445>+umO$0 zkheG<+gK`2rG&@KS;ugYtfhL9E}AKml5|W98tzy@^uk;ttwW_V8K+(3@=w>j-`fSM z`GgaeLuYz_bi4%qFsna93yp5a>Fmu>PyX~Fk@l%dQVup?f9$uphzy5O2p5NvY^(`` zu_X?G%ID1c^`!ZkOpk(&3fRx(4zCTR%h5JhV@?PapDHa^46Xznh+@*e^Qy;SoOAhu ziZVz~PsbHZS*9wTcbz8umfk?77oVqm7^r)mN=N7Q6r+jrL|{zNl9nV?E?ece>c%%1 z(D%DmE&_@&#jLd%gH?roi+SNw-ji8Tidz>gxBT)XXArg)7S)0X=|k8^;-vs>(>x+l zJJ4HPUgsemj}CEV?BMroT8K(O^+t%v+S#12IG9!l(?yE@N9_menw>F4=|<EUHTz4f zn#~Y*vC|Q3%ylT(TKIujlD>B^V`XaaQ5`>FRQI69`|cvh8LZrtQ9&}BuKUYol1@xz zP|Br`d6Dsc9*~UJ=Xqd+rUVV&W#p9IWJ{c%Wxijng|IhDACo#aPxceAI97Z4BVR(* z^<WPFv*mDa$+a;6rj)`@k=}4F1Ema6rxL?>Bo9^U@-Ka8m7wpItM(BaERaYUW6Q;o zwoUf*JG>SjO*S8ge#}hBT-*M2q82lM6pV=3fhhj?&9tFBVCwyj?K}Q5kY-PhCHJ(| zeQ2I}`{O3?bUve2$&^J3*B;TwCRYER4&Ep$5vMj^^S1i%wsY?zLSio3_!Gh!i9DpL z6}9;5qFiXdfMCw8yjyzFx=Hr2$6eJKN3%oLo-l<dSbngID)CoOhpw_b@?Ee0VUlOs zCA%!iiD*dh&0?O%z%S;Tc?1(!4;YXQqyX-I+)jeno!%LGEx0Iuk`uMnfqG3i{X@wA zXH#ZHFk&HWlfD<9NrN8m#ymm={Dk>G5);Cwe3$Phj&`TiV@`vXZuIyW2j(rk8io~f z^7dAM$k&3Pwj+VB+=G!9P83?C=R)=z72BMgA0gR8&t7RgAG4q!I3=t&mH#q>jbkX5 zH`4l4U9Wha8v6w*aM3|)U(FoPOM~h&aNxzWUs~7suO%mRZE^eh_YIoW4cb*Dl;fE+ z;~NomZ!cMJpwv$-S}=c^qvw1|6$NZTmkN;Q9A5>ZDXcAudDpeb9c$76<=cmEiiK^8 zg_}qyR8!riu9NUJ{MC&j7|ywJg$Ty3l!d`8C#39A_tLV)L&1edxI@vdcn1JhA#aRq z-E_vSscLZ)zIxEty{C~_QXBLo6)uwJ^SE3#w=Y}|dB{AU{o<rMwcGtLjvj@FRX8If zw@bLEQKCr!il!EJRz>3q8#==63;M2M3swlxJ;Hzc^(`NT%~wI!vxvUEmtS3dw0>g% z@3+Ps$fW2Ka@-oDV*+HEn3y#3HXX9KSh=xz^JcPQfhcIKfP0jwH#VkwP*Za|2`+yw zfHv>CRRq3kxTN_`w_dz!rEEJU2<kAS8vj6`Tdl-1Rj@$G7cUK{l^nb<h-EXv7Ocad z{q4uH1cTE7L|0d7ey<{$Dj2qAA^8IfWPCe?x97T9N|R8li3a+n+rY6J4NP2lx#;w- zISd1B?9J8G?jnr}NrJxi16f>g;(PkHD`h>^U&hsrp7(`BqW6$5s^+L`C}eOM$vKg* zYkZ7y<6F9Pip$IXFusOyi^6bpv2gfRIR401pe1;u8RJ#m2hTGqVNJO&(ETyR^G=QY z$-`xJMEh2EWt>IUhhw#)li@%qlt;C`B+P4g0X)-YoRom>l8OQe-VjP8<V&$j4}Z{o zcnFY<eZv<KYlKb{S+LfKj-C0(>+0}Zto5R1{Na<|^$dAH^vtb8Qt%bZomPv-FlE#T zD)pmbs_kUJf-ZCG`~!s~hiO|!63mDGI@q+!D8O7|Us2*vAYl6t?G*1(^#xkz@{@)= zbQ`-z&TM>bOVv`cyI$WpFpEkae<~bJi*0?c=@tC={dDCIH;Jdy2b7CHHKt=(o)t6! ztm??EPKZ=2MOCD72LaZ09|4rvep<k?vH@0d@W3;Ho7m6m9OVOr6;Jv|&neV%v9V8O zDfX=jZst8AcB*%@oZ?{ubl4x>ZmL?>om&U}=5<p~`N<O5lIm7bT0-wI<-o~TE^Md) zsgJa=g!VDT#Qhab6m58PHvBDAalv$8=-O&Nk7thuNtYhPw3p_)N|i)9cR~{80Q2el z(GL%|cwOM=7*cCQaDxkoc0$d)AvodaXgK3t;K5~DtZLl4+Y9P!^-DkLA7BY)!2+)d zkt>uB<k5L*O;49~^|<#6=N}erB8E>IU03})oVM*I>Loh((Pj6Jm(L_ttN1uUC#(v| zd^8%;JM?-NKK=k+XPgO8x_6NuM%$byOV#>NL^gfzw)-1?hnc7C*WYtSvwCZl_C>71 zLtRC0p35FQF<;rp#MG%GmCBuu8kbtf9{8*t<)ArLGD8&60p<e2-d$l}(5F{tVnp+7 zu#SyK;HE-Vu20(8<PHC@YXkTIH4;G^z}2<Yupc*yT;7sNHi#!v*0V3D#R42ppUK{% z>QD&gT}8{1YP5jyt4P@+0y=xLG!dFd$D>gSWdQ}B%b$`IK^2mODTJOSNU1F6BxUua zx$&y)A7KwYW{P?AheiFlFGfro$~&dfAoUV`og*NivzHF#ZF)}4Pa<@h5FDpRqGA8a z>zpPhVDoD>ST#q%^98f2W|RvLDg+%@Oz0WCRsh|Wh=-twW=Z!7A;NVI1#tGWobcGl z4rRGlwIQliZF1cvVmE9mC=Ywk4NmPNXhmE#f9&hNlyY8XDB6P)8q?RlS%2p!W@RsF z)3{#w53`hv@=rdi*96`Gs~U`eV0Xh+36&QrcJ*)RN!}Euc^3+|c7<PfXK<I6f|*Du z#^a#o74}ago`~hq?jg7W)u>HVo9H4lJtH?291Fm7oit%Zw`Q)iyOmFUEks<CE$UZg z?@DG^6FoCidW!H)$^2pCQJTHL-VQrVNz<7bbl&?rr6(%a&l0-ZRuP;o_a-mcX`Tp- zO8j=eknuuAeW+|sid+NqfX=CEO!yu*S76O%#-!2-9zc~uEO!y*=;p%xL_emI;oMl0 zfmSnQt7(j33+9gXNuwY{>iyd+6u#b#Z)A2X*VcsXtE$a`m^9C4IhGHk1eLrbrc8>` z$u{Q%YOUm<F{!~Qi~4x;p#=ISd!$|cchij~&XT!`b*HgeC<Tewyzum+s6}Bu=k0;s z<#9p9=|-<R5>%`l7H$gkFA&Lb$jl>FEWN;Qlbcy;=LBZBBi~wr<V>b-hONe$aMKL9 zvo(YAP#IU$sCpS+K?caI8XKeZhe~&n6*q&qxrDg{Rwyga;42;xi(kpf$+Q-|Lqi}t zCZ)Bb*W`Ih^~^l(1Dqk-HR-MU3L*j6yu0Fu35lCKHv<i7d$XTfE<It0V(*c;ii>%q ze)-{n3zGs49d!&L5DspgE?Stycur0VISe%LIblxa^v&Qqob6RBQm-^GhRN+WS}bJD zTGBI(gx?7CSfI8;yysi2V7F6UZREin5CL71q+G0G5q4Nqr{MNd<HcJD-10ny4N6*- zWV~eJg$+SuOP_UumGCJ=fb`$<=DxNg>^c(+K<9;<9f&VH6eQZ`k+S>_+bIic2rddz zNcrf|@v1wsC*|9g{(%uF2AjxiXaSW#roYf$g;}C$CM_6vJEbGZ+U&T_Od;-D=C*g8 zq?jSZarNbH!vH37Vs5-yrs}&O-5im_<uISpo1X`#qOU^Q5dlIxeo5G^p;CS|)qT9a zQQ00lext$}7Nrd)kitU!tmig~01(|A!Mp-p+yhvxl^)y69Xi|_Qu;aEK5|(&pvp~e zZz2Od-Q>#ku2dsphnGT3s-oZKi5)0sqI;AlY}R#kH9zl)n3Yfo`@uj5A|}$`8FlYM z6-$(KGc*vYd6247-Tkz!!<&Re_}F(wrP}7O=rFiFr5bO4)lMy|-ydtAIdkoJfRLuj z?7{|{A(f1%U!wzjHO@Hk&-mnKP+uQ=1G{gS4A9OlSxB)I{)29z1t~Gx+*==}oQBi* zyjJ$`+{^BVMtPih6pq(yMo{m5e|~1QE@bxws=P!MaP3z;=;|u@$Yc7ghK=^LjT>+* zyb`pt_)@RI3(V&**K_lZe+(~O&~&uv$oVQt#!rhN_J>^43Bb6*I8GCm<>aDQXW~mB zDi@h)0{J}2)~chITK~n<gLUhhOClZ3OQRU8TGw;NLEG{1qNyRsMLKeRCu%|JVe4{{ zp;*fMS)(Y4j7+dFwfVq^)a}<dLVol5#X9)`(|z_P(me{;F$|)_=M-Ed$EQd8OFuR_ z;1x<}-@k{*b{Su~pEc-4%}b`eBRvn{Kj~91!|p%?Z)>uZ8XtYGT@C^wU`!pAVAUnc z&Dct&5yPsS<H=Hsef45^Q*&u}PiSwmGc|C@WlP3tH>6m3q3B$%#N1eU(0+FQ2A{%I z4|EXW<IEtuS+_sy^6HlHb&S{ka%-QB8E@6EDd)Y=c#W~wywCoJN>HRC^ZpusIn;9Q zZgH2%v0P8e5zSy*NvErmlU#D!Q}^3|SMM0N8h*`$9fiL<v7;9+T%VxVS|63lErKO+ z3tHZL&r*W?8k;av5R#jhtJi$n1GD`STdIBOG5fN&tod<gZ>U_k=KHa=@uFk%MDs#V z&1O&rpZz#jcxr@JS@p2U#Vm{o4}H}E<=jhvH$+Si*-`qL{z#8ct{*PdgSYdW9;}Te znA`Ht4SC9!R9s5P4+8<9V{<+mTX2suZfl`-vusN4lq74ZWqQM9ZDm}Z-VAk&PJ9u$ zW=QWh=&lJ08W$plCZ`Q*?tyJq(t@tKl8>;aR1tZzY3~O+9G<Zo*26Gj01mh*HI+7e z+-&rXh3Ax<9eJZVRzTtN=dGR}kSV0Dm4v2DsQBb!yY6{KsN)<UmEauwaiL6;zS-Ey zNVj<dR=FvW%I(rJtk=Z7_o+lpbirn-%po0o$s~8kVE_@^TTX*~`G!j++CM=7#TNTe z7;iZjHLsm13GM(J2wApQP4jzo@KR7ui+J|D8+4eQE>h9jOFC-M4Rt%<<9zF)<6*|k z@4vmn8SQp*o!+&UUd(5^skeE67HC9Lu1|Br<<S3$f*L~4R~n$PD)4R>QqdwU8|KoZ zkVXx1V_#jgQeaffv}6-LdNVC++-oQMf}YA7w}QEPLvFLAdfhgnx5WL=X~~v0wxs&S zBSk6jf0)SMoJ30fm%BJ8GRmJn_2xz{!4nyqb4-`Gw+iS0j&!fVQx;EgXiz}scl6#5 zoma7}6xdPgkeVA!h>~b=T#9-9`OX|-!97ea+0f@C&!Fk%mqDG+61#qT1|Hp?6h)B| zRm6s7gj}Jfx>;mlqi*_*CJSHhPM)Q%g(lSeeL_YoA%tlY8jfnUL{(Hn?X*<(%+N;H z!d0m<H`iS6e9B9cA^gp?CLg3_`rAz>Qe(pY@K;3i^oQJ-dd*3HTW;Z6FEh6Z=Mqg< zziFE}SUx=bh$708-LSHwOcczY8zziuO9nU;HNf5iwK4k@my+;u086ZHDRJKVi~(`s zmfU@Z)uH5+UK0oXExTW=mx!p1dH+Ci$u>CAc}||7DbQ&n2gIR60BK&#q*G80HcUc= z-JEkyox!#)CAx1oD$AM0*@}30cu=(iic>RQ`ivqp4)vu)Y&JtYj{F>P_ut&7kF1GJ zed^2<%C^5}vv<kM&9eyOe-R^hM|5}YGLa(&>)Nb5AE2)IX!YVh3-T9#mkR&hq3q~$ z8TMeg>9;RE*mxg1U9WPBHRVw$f8jXvz9(h8F4B9O9tkt@`SO4~2-A!;Qi>hZCY1gv z@u4W$+|%<KR@VE7g3lV{@nS2jyV&lgDVN{uOx<>(A%7i)sHLioaL-x7j}7PcaAg$1 zm)CAy19fM{2)fO9ay72KiMgyGI{)*sbu-7{TLaj1+NIR{FnhJ~H)c#`RbWS%X0EQR zD{Foo^+?1%M~<EkMbH;Acq01U5i}UV1;qD_Xw2|3DR2Jlu1{ru>=s|gzU*F$j6Z7P zo?pFdHSNEs{dHo7Kl|UDs;hT(=~BGL;bAvs|B9p8q0{K9Z^wYY*mI<$hkQ;NQf&dr zak2uP)2CkFgk9Xwv{=IL4lhwDBP<o^<F9%B-pQ9bXsA_}T6jqhmdp&D0C=l!Lx<To zT0K4Vt>T6E-VWWnewTsl>P^B7IvrG^Q-v3*C{6^&(dp6@G@1XkDSRjcADv%#U+f>Y zm>S*3(JaIp`NKz=u3OUY8Rk0qdsYkKKBImI9Lo^@n|qOqkr;8TMirL8;GP)Z-hi}Y ziKWKw(Un?c@9!HWi1%Jn@lZm7L*blasUPBr?bz>9jnQyqx^!}h+5(uF*Y7GjSoWVR z+kBNCxzdTh?+?GJ^gV1{t~|U&id?wE<3wGNwI;~k2ako8hoIX1bayz8iShf`F%S4q z$hQU1%&~GNiaii~z8?M*oYWXIXcj+1Et-9d`EZ(WcT|eM7j~Gm_xa1XB@1%p;i`b) z|JNl@RM(ZCY*KmLD}G<ud^*y2aj|gNNBuOSC-B(gusMc_J1|&wu9pYBXEeOxEs}K( z%)om;itqo$|9sS8Zsz_Bt?|O!LixU>L`{!3R2->g@o-OHjsN=}awvTCQMl^6%Kaex z<=RVl?{b_dueW$4n)q~go%&g9VRW!;vI1k{AT?X|vE0Kc8nRp5dr;Hw8qM~yM5j@& zYQXO4Xc7=v7_%4Cdg*_0;Y}keaE9NB&o*erje0Sz|9>u-n$lB0MH@)O4cEDX<(*fF z0Os7ud(oo0Qiumsm-hr7HrPZiHZY+_6c!P!%*8|e;(*T6)(iiOeShB>zbkO<{(dd$ zm4AMdJGta!PUP_D4Xn0v)c>Q^uYR*CkqayADDg;=?T7nSy^Bl--!Bix(78+V_<6EU znkL(eWlWlo-e-OP)2;vcppH2&b>l!;&yU`Ou{Wq*+s7n~_0>u)d+nB1gc$t2@Bin= zLtVo0bqrg!Dr$5GN&iJ3!2~(1uXzW|xaSNQvsM4+g}+()8yxH?RAZpT|7Ipwv;$I7 zVAP<N%$=l6i~jXBI_ke)((KqFP5sxcwTut{881lBERqy(zNP*TIt?aRWB<D!Iu#zV zT$)xTO#kXlID!^CiVNEpHJ#p5C0(YUcVGUyfgJR|muP6eEIksX^!`&Wg->`0N+HtC zT2TEKwIQMBM(I-6QU73(DCI_q`sWx~YF{F1N#w?NUg{A5Tp0dAc7M%{(>u!Sad#{f ziOkUh>ucG1)ogO*|BKaP(VC!IGF|dPtw@y|d<PDFCtsLuPEM}F@7_-IuO@><UuRc- z(scRz+1`|;1#WtI+b`rOL=^XSwg2@jYj&lf$X}0sGEXjLqX)Nbuxr!)YnCd)7<e}z z0;owUR!`7DeQMwCATa)4lYtfYH^KDJQDv)&{P#Rz{rJzV|3xJKnOatI)T{qzr=$M; zlK7hz7)<}wMmW)b7TZkWzZC>5)hB-wNJ8Ylsp`|Kzv=TQ*S`iXgZ&@6;Qn9q8SwW` zm+$#+)*np3@c+ahu7<Fpm}}YY|Dt&Lf&Vqef-@=8PtAQ*IIIk!Rgrzp3oj|kUFKiX zY#x>bnnW1(CaN0B7s}ONLa%J!n$?IFm*!4ck=*pb-P2w0<)Li-8BB4$dH)hrSgL`a zn0YabpVL+<*~%AAcbw>5)F|D_>g_5?v*m&)iW8<`!lL{zT{jw({G2Jg4uKUdaI)-) z%scB{gpXJoGZ&6Sgn)-{kr|;!iF5o#ab9}y_Ai-5mQk8XA4?5m$Q^L($Xc^Jt5zfC zrXA^?PT^3)g8+#ft6BrwW3ZAT+seDI2A18T6&P9g{}&y+%^lNbYaP>dr?N;#MncMb zduD@^2kp)ie4$^uHFu9^Wp+hfzn9K|#x_bk7H{y9?e651YOjlH=Ze_8fkQR#VY!!9 zEPEPeLTui;d|eQ;uEs@>X0fWJ?VMX(6lLRBkCLv5DHD6;I5Ei*w40yt#(VE~F2BbV zr7g_5BG-&T)!sT`38v;EGQV={vNu*J5A&UUQ5*;FFZ#s@XrNVr9e!aG!H2AZH}xKj zc}gTD9pt|auvF6jyyxArQWVCk$_8)DhjZbS=z_wKRfx}wNcEI|G07bCczLUmndKtc z1XxM&ut8<FCUDtO9=2bzkEy&c$CO!3Z&|)j71aprMtJYvo|D==^bd~jib5?G9*a01 zG^DrOMvAr|t?De6E`s<S&z={@QN$0Lenjkkul}_NO_;AqJ*VOu<UbU%P^J{JY{W~4 zAU4b1uP@hh=9kZB&~F0W&+d0hk)_BV9}|AUdWy{6n_Iqk@euJL$!g}+p^H*p<|KQQ z(&&y=97zdu(+K%JDyd;=;#7n*F^H_s)6d>c1bnGR0jeR4+mHh`8heBJMfpv>YZOja z{i#>I_8j(}uM=X~)BYncQkuMc%oO-~^CSgieEh`nVE<-`+aWWxAm(=X`Ygt!!}JR= zR?6An;40<)h1c5PbFl~!feS(4CDh!QYqRTu4Yql_AKP#8f1zPUUTC!q3OB`YHv_aX zrLo|D@^e@ZIs&s_aOmX_1A0g6S1`NG%$<c$>`$#*0%x*$_+7@|>eYEg?2Gzwg$E1@ z>{aPRz~<(dP>^$O3#>d8PQ~X-Zryt7?xj5jC0Sarib<72d82y3&zr23lUf;-mPXe( zKyg>t1}ofZa<jZ9hdH7|V>4S*^sPw#M}a|y`A%oBzpcQQm4>uV?ckqUdY5?}i?}jP zap8PJGi%&N?eiwQgrCcdSW%yIpe?r{4zpY2gKS{G#H~A+CXa9iA%vmPTHT`aXUymK zZxkFB`x!^|%txDV%)K|(Ic6ttMM^(S=ZO>IfKQ9%Hr@8PCZIHe)*Q{peBKA~96I~L zR1q||6jY{KuigIe?b~}QHiXSc1~;+YPg~D2Hzm#&eBRQscg4F?aF}8MfY*8#ylAdP zN{u&FNbZKd-{_^g9ANo(Yc~6>?mK(wY9h*1DV~A7gyrtGP54ekmI)<17j1>t`Iy;b zK#ukV49AjxGWYu6)=|Q8uzqKNI}SoZ<XG~Px<Kz~Di`}wsZx-V`>pd#-JqHM|B@Q& zpR&2i6*)vauk(O#_o;bOP&GALgdMDESd-VOGI!Ips)SJ#2pcHncb<4M=m6_Uao+-h z^VuMUr!%_wV)P}d5$w>1GmV8hV5T{31_M2>Z0HAVv~kvkJbREl>QrTlB6%nUN0Ytj zMj7N*)1bs&^Rs+Rw57nF#jtl|#rWrqg&I#5;Lucv=B$5-IphF1eM->n(Pf=IZ96`L z5tD8nXUPG<<{{N9U;WJ5x@4-!R$ZEF`EveO$nIY5sJ|#yRI(=(>foSf3p4&I%5PJ2 zPQy10fF2Tq^t>nt(RTUYk^mf<FJ5}D<G?FiW-@DeK@XM199V8o6P8LEqcU2$*q1%p z4Us3cd$3}os(A<wTdqAtah#qNn=l^qG1yXOU)Lhek`40s{UlrM$MQbR@T}p|EaZ<P z!`gP@62#X^e$af**r*?UzV02&SDI$dK&RT!_)lUB%y#y5>Wg?j^8;rJVZUfTTk$cQ z>BARYvL%{9O0cTU4;132-xOo!#aM;&-O5!}uj5KC$G&N{Djrtw#NL|=?AiN<%B~Mb z_P8EAqj+bZz<2m=)@B2QC4vfX-nQrj?|RE_o>6kseh>2G_vxUtU1l!gu~9Z`e7t%! zB{V4AD1r~5=~*998bLSZi2F<Yhbwc?_@1e9(so(m3?q>dx6FU*7G0KcpICS^8g3(j zaF^MX+ASN%4g_+>D9XxLJEq31i`o9BS<y&-c-KtBWnLSS22F^C4?CxxRn{;ru~lm* zFL<w20T-jk`Vc&E#DHd8W6Lj~;sR50=;;b`<MFL6zt_m8K^xexnp&!!MbHS$f9m-{ z)27GNNGj-lo=wE%TxS8aQWKR{)wcuLGoMCn)wq!$yW<q#YHT?ZhbYy^lKa|EMg0o? zw73k7FP}-uaHUMgBh3>GTALwzoNX3tOJ*YY>5%uf?%KNt(t(a!`#2&1LcnHOm8H?$ z&*q~Zt$o;Li`^22G8C}N1qQO^D%sk?&Ff0xPHh>!>+-9DB&dQ{!=)~n<51U5b`hV8 z{-Y1NAb)C6_oWS)a|@C)ND+hwI5G4T#ob;mxa>9U3%d418+Z(`i=F3C3AqpF4H=o9 zr=CNL3V|iojgR@+c1BzJ;Fjg0mYZep?>AmN@7AD-F{<NNHMKV9!rIkk^mda6V9;?- zgyik-mSJz@*=0nc9U@v9RtmQ&)-BGR9@w^;BrU1*9Y=10K?W;}I0AN(e<CcXFvDB` z?o@#5Gu<gD&MIMTFUNAYkN*HhRHiM8&sLbHg2GOYg?fZK^VpzM8%y*LcU_6xI@obz zS1PZJyQ6mEKIr>SxN&71wm!7@Uaf{5hjm+$nH-^NxZ*e+f9zUgYG)Fvef#Y6c=C^n zlS8=N-sAw7dy_gI^a}%s5WD@@E43$#e>tgqSzU4>)m)N`4?Y;^>!f7)&(Y#QAoEFf zWooW^sU`Dt6>ILNhTN&X%3o=LugG$zfK)Pq)XKWE?vaXV)e|=)!&O84zCd{Yvu9EE z^{VyEbMC=Cl`dL3ny1tWYK_T|H(Qpem+8-?;ANebz~gx!6w&sv@%Bs}>UM~|yXgJP ze1@au<EiF6M6dkmxZU!JBGeAGcuvR6BGhw9%r0!AEQir!f_6J5yC<_`dJTS-TgZfh zTk0St9Yqjc&#hRtxUkzmlZNVCCe9NkleLrHOKn|YDZ8lyT{hrKMKWmb)~)eoaWgRK z%7*~p`Nx_`ug196z-#Bdh{uhM)#QMpa<E*N*!2(#zeSJ9;6SMZPGULk9r#e0GCO7g zh$!p4IE%=v1#dcNBg?>S)sg}fd~P2JPT3bdmJ?+Bv}-Ji(_8%p%NBivu!`yxMf6Ss zm!THBl8x6Vq<crn#ZcEJJiXStuLgDZP0f_&cSqYSFo!V@`KTV3+2H#2nEdj0w!2WO zikGP%x&0=SP4Ee6>Adl@Eu@y;S9SqXVDN6@Vr$M8YP>0#$}0dss!js?8-O50h=3gu zm561#eN(V}7aNmNjizdZ8Iumwu;FX=E-~j;kU}uw2^7#gT=ibSQT3j)AMm*erRNTt zPNVN;{O=N*?=BYb^>4vO@cWC}{h0f$n4G%nOU!uz)%sV|e!I1PgZI~&4?<lZ&BHv0 z7EMIW?0qM&@64{pWBksN^)1CCDwerNutYO^AOfHf!b$uaKTW`t{9O$KRc+e$wXfpy zBhM(vMVt+dpK<y_I{canneTs-e<Xl-FCumTkkKir_5t~jiS=eq-C3vWT@v4R`i6~O zMNWgg9R6d2qtkg?J64CR;Nu{(@(oqMS)1A!_O?5yQbbAr<pRgAkbMI)2+zD8@_-&V zKA#Tt+7;X7Gs`jO-@pDE+>@~?@M1F66+4D_pU>|5w8PXwmGiuAm>AgZw1pUSwa3Sj zQr8CHJuoST?m-e;uM|^jZHoD9<`=45M06z+X<@!c9=3C<T=JtfE^=0Bjb4qwIq!}U zgGO^(C`|>PD7eM^SR{;L;hZ?z$!(Ur&h!CW#BY_k(dU%GcFH_hqge0DoTB&aV!R%> zDx-EuZZyo`Tep;HSV0pWtVKE2jhuHprG|z8Qdr7oZAdnM)nRJ{c}d}}<JZXu^;mCs z_0xXAf1&}8j5K{_+4=O@{z|_*Xup0`=iwNKjjt%*IPCH2anaLeSzVA)V;^ah-Ni{1 zXnQ-+Zav`+RwS*wJLjEc;DbW(@w;!}vs)ArE(fo}Y#k=)GwD32MUcPGsrZ>~+?Bm= zzkxx!E=HBApGox)BuhD7dU~0X2;f5dBrPLGbTTn@yBFemCg`i6<KC0vncw-jaxK6~ zH?ylHCBH`JlUt4PS=Z_`FM|@_T0$6b4Bb|DR{VPizk3WsA@rkFQ+Y+uSDA(K`<?Q6 zp3!%v94%Kl&DR^L7aR{K9PV8&i8x#DcU$da#P92TPgCK~TRaH9v5=Bm55h|I_HB)j zDZUR#xc@6Rq3{n(${97NsuZ)z4`<3@uRn7;PHyP?eAc9kaUoRlKNawwG(;7!^}fH} z45E2h<G5eTex{wFo{Qhro{wv?>F$5&zMtr~oft?sa}PkyW!~u^58PY5wczCP%gFjn zq!0?Y)F6ozem&y;*7qdVDNi|(!w5QK<S%+AIw$6|{8u+IXq?$vkYWB;VFXCLb^2M< z5FV=TP;Hr17+r-JVL6S?dzK3k;cQm*bj8S9-}d_7hTUB_Jb=h-H*~7byE^o<{NYzU z0(&S>bZ90DROlI(l>ZTNpyd03J6>r8NAcTgaU_rT^?KsIOC~byu0QkkV;!84LQ8CB zfTc@C%*f|DNc;-Tdml2>L3wwQ*lPFJxQSskC%?{L87timjovEM^;R4_Z2I3@ssT^E zv&)D3^<6l^jW_l~a7a5Aqta{5+sPh|X&x$lk0XqpUwF4(dbeLz0@rxZPv3!-tMP8P zGw-b@G84le-^T4-5nB3RC;F4`d^4CSOogm?!;yx`9qhKanLMaM7nEYiF)cC7G(o41 zWe)@{5;fXh?!h;>2jWII6N!5hx$?%MtJPNAcTdDfIm=s)LO5nw%A1J9{8v-WHeX&P zc3uf2?}qc;e#7^r^k3KVV`#d3xQ9QmXbrs%ihxh)snWpl$%tX@unIIxykL*Fqj%5V zee3ka@*mGP^I`vwx)YRhf)iDWp>ug32ZUAi29fbQ9lVvdg^7k6vDEHWf8^1cufqNN zzz6&={(NwVqh?6686~J#@IP4DnYsa7s{#$_?S>+A`8{EPRDSm;Hp9kAC4H~?k$2Z= zijm8s{kEo`H_B^RMw?75%if_*lY7sARAR;%kbV9|o7`iHD5}7*tF>LD>gpN$xk(P+ z_KWm7tGh78%K14UE5j~R>+lu`#XF2xd8-iRh&F4<wA2QVAoCdq!fRV~kHzh59%Wg- z1v3%6j21Rt65UTx39uZ;EL>{wUd9m}^GUkYYP05jF|T$*AH*&cv@2#iTW3uG;DGzg zk;t_Shu`Wjl!oN0_tzL;D$Rc8@LLWY)i*{RigfHJsbQ8^FH$)z&DCKxsbUTx`;j|- zJ1B|KIYJM!Z`0ltFjgXvN|XblaE4=Z&18IwVN*Bw>2_gC2H*CoMERYI*4-)1Fu|?X zy(vIUJJ{MeKXAEdEj=sNf!XD_VINkoz}e`x%eO!bsRt$+t1js3e$utI9m^?a)2Zz& zlS8;g_Lj24O%VAU)wHITdb<UCDak<jJ-qn-Pf5W&NDMz(E~yhj$)~hVC}d-(aVG`} zcwC*}c8lkt)WLVKCyIjfr=h-9#%qNt6~_Db!Ku8~Y9EGvvicCX31JbxE->=i&i$dF z2(tUA5F0GwePKsHe<ecrdTf@<^L$5Rp~_tT=okf63_RT!UUm4HR<mmSI+wueLy=)4 zi0$<jjVuNA>({qsMy>X!jI!!<5qAuDxl!vf6u4mFeR%*;Oyv(rXZ_ZLXY?v%vB{I? zwINN-%H78{K&VO^rT<AkQ;<s1P?8It(K}Spme8dJH<pseT|=mPITSLO@PI#?D?B5T zkEFEanpk2UK15H#%tB;)uxT-oyC^pD`{o6{uSA+Q#3r82pkXS*V-0(u+1ovw+>l<l zf<)u@o-zv-O$iOEcn`j5w8WrS_>b=SpONL^*rb3sAsWF<Cs}<ns&&8al!3(0Ned-~ z;XVC5qd~#m3{Un@81vFTYl&4G0Z4T53AwI@7#3HanR?7snho<Rqk?QDlT6QH(VyPj z2uhKBask+5m~J%J<~9!<`4gQggo+_5qDA(d`R%tLQSXajw{%Fxg|j$^JSDhH{I*m4 z`6fMQCEk@sWzXHT(VfObCia?fa~n+!CLxPXEV=&#Nee;8`A}&DDMu(@yg_3B2=R>} zb$0CR@57RSuKsN4u#Wj!Cxons%Z9?$J2DPVtrr6lX+0c^p9ffSmX&W^m@<P%*>qpP z@Pnt#<~t$b-+g+(G(N`rr<o5vWXSJ5QU9?@Uj}_*<F#M#olSvInV`vzK&MkL=R@}~ z?b?gj5|(lrryZsj66`%XE;8#V{esG^cGQYzN-POpRzLG_-9aM)?ABC;vLQpLg0MWj zV+?Bck<npt`5CP}hg4#gaTe^gx^5}D5qLp%7gbTvhuM&^5z|eF+3ZOz;Bg8Sta5=9 zr40}T_mCPs6fOB|;}|nh$7oW$vyavo2HG;;YMSXce!_}UZvG+<v#S)IQ`|$@nR1|M zwR!0nvt9MY(n7@oqQ(Zu=dEnKe%&@R_O{o<UUR>;1Wi`%i4*O5@8h7pC@sQyqp10+ zXU1*`dgmT23t|{CYMk6T>YMEgV`g>(Znl`N9v5OoMd4u1wW;y|5yWzV9B*uQGr+3# zq@6+(?o^NqG)CRlW1NWhgKmG>H=4%Ef9(vH9A;sIUfFbJ$0hDn<`lvz^o|`k*20>{ zlk69~hoJA*_XQp4gb#D51pSsR*mX@6y(4QVY7MLRF_rXEL1^FKj};6FpNr^Xf@;g= zERbLq{Issk8ENkgQ~}<7hlroYV)6ZfD3Q<c>gyd!Z}j{iR;FD^_uBg7bHBS?Bm91o z;9i%_k&xGv*za3n#Vv-Cevr{wf8#u_^+(pOYTI$gq>_L9|JZvEucp4PYm{C@gn&vX zlmIGHLhk{jMG)+QARwVBAR>YwT`37IGzFwYKtzAif`A}Mk&s9)B3&X0O^Og|2qE0# z`#$&k-tznh_wF$mBqKQ`=j^lA+H1`@muzb0ad{CdHI>uL^6xu!ktVC>AP!`1a4eo% zP(BvN%Pl651}SCyS!XCCMId*M)`$!xdSAfm`VnLOLKFX*TdRa!ZZeEpFC#u*NO2Wf z*&;~R2b_AroVdp|1#SgeSKamttU`e06y;}g=+-m4jncjL0GG5gX3bLE=9m`RkE(b! z8B?LJLg>ZE;484PLEF7cz8Sf@o6h9}-hBEh4e{hG^QlmD)IIIBG6SL?t`6zT)vb7| z^0?oo#C=8#8zz94Y^BQ=IA&rlr92b9)hAV%Ze=kJn3zhS?}Hw)2XceLLr$&I!Gw4; zqwL(G-CNg(A2BmUsWHSAUJCPc|I(zy93dTM2vJ$6qYZ`iP=~@dZyrp>97T-r;O+U> zo9%2u3~^=a7Nx#!Xzb$TcwHAEJ$Tr;yJ!H?g8FG+L~(}SASLbF$(#--sIT!^0=iG_ zP6rPBgbR?hGssB0ReaI<>g4?4CYRj&DNzs?$nLigw@Y~23FYuky;;OX@$8qzZ_%y0 zKCPbWUj4@dU;MlTNp@fzD|ZlNJI>~XPYS@7Wu?r9;j?$IeyRd*8%8o%5Cq-Y1Rs<F zK6iK6Xwo!U)l+<HC;BrIHLe$)f;^S_E@}L=javsoHF!*F{<0*ZOLs&<Wn<%Wt*)I* zu24DH&T!{?NJ{l{D*Y>uN^EQ9M8#U*KL)0<fr2rp9l?@F9J5<q-SN9~KxDJHj+CJ$ zg=tf4U2+o`ORlL~lk3t><Q2)t9YP8TJkwDonH}jeYcLs$7EyN_=zOvOS-bt^X%N5r z9dDb*m~AcSiDtXifT&+j-B2|;saHM`gm;E)_gq%>%qMEoDaPd_BfC&Zm6lP+Gi=#$ zT(FsPcUwpwEk;F`6EP-F?tj`N{^rieox(fcfvZQ+r_{T)KU0PM7SiY17w>SIZy;7h zD80R7#Av|_2$gTO9Aq${&f7zA7Oh@2H?MIV)mU`)^YtVAHtJq(@hdj2%MO1Y3{k;c z<3n_%UNrM;;ff#lJ+;5Jc%fCtd2$Dvb<5k`@Yl!ga)*zr8?zmEZKc(=;&>KLnW`B< z>YEFUZsmX}dtm3V<DZF?qAC~0+_42k9~CSKVV8%1$*Qf>1LcDP4TCmYFpFEMP>Sx# z-VOY@QLn5{fEc(p@PhgTXc{0FYk~PtYSKGwgbN~p16~e1mkj&Q0-wQos(8=huWm(+ zV3E5W3M$x&1KV-Ign{v8rGxUw73Z5uC2e|pl7CmPwIV%Q4oS)#<>&NAYo=~{t{Bzt znHcERIoMgleGn2)Q~2~(`(46j5<RPG+GPp1her1655DaW{(-CCw14sHQn_cO5Kt`0 z*%8y2j<<90w$}c049#a4wj+{P=NjkXKBhP`^w%a5Q9ZMtxKw6BhHhF$-2V{~a{bm; zQ}C~!avGI0zCV0C$7ux#9SAE0rAb#uzWh4YvdVEJC9wQn(u2EC5ZJ--UK%>z^B~pb zINI^f=imfhG2I^)$p!9;+Us?IQGZo|28pUz%u8#$t#5Gj7FYGl4HD6t7=17yCb>tW z{*l#sJmM~<QU2|v|D?w)OlYaS_o{9$?YIay@?*e3`}dnKh+{TB?q@Y`kALIlS?zt{ z_3Yb(|J=!*bse2m?`9w&;5VKBt*Qh+_3K@;Nr#2yk!>x3V*pr9THIclF~4i`hYQet zv%p@UKd!#Lf2YFZ;O=PD2AATiMu<WD?#O&z``dnJ&l^6YP5)ZpN5q(kJ!3*m&i7;| zut{{?6u7{38S-jeI=anxI5rRnBET6?NFTo}Job?UJUqh=5@IEvo-gyhD<DG!(nG&; zotlahY&u7MX~3;%CI!emJ@L-wNm(s?{h(jrH~L-7yS5ye<28y&Z%(Wg8O8Evm^qQ* zGBh)RbchH%KoYV0qAFYta!l8@0CHN_b|P6_|0Kv}NsJpp>wa*OArCb-<ta*pYyj=- zOly+sGpZ9W8En2{6GLgVoV7EA;P~aN@EMQ-SX89dtIXGsee>Z&v$Kaqw#0ieM@xcZ z*G`};1-z-5Z1P6GoKiDY=Rmp0+^oL+thg$88(DDKWE)bHNX7TF<4)l*I0>@Ja5@FQ z1de!bgS8t55z}y9Uc9JC0hHv*gJe1Qo$!k=)D)4LLgb=E=itTL%5ZN9%5HK2CJxG# z4~<aQ-MA>i%yU2+nOPljG%Q!FS4ar9p&nr40I7eUq&o-qVROy+fh*?;gB0rGn~QxX zO`B_akU6i3PImdw8JqyZPE1#5wk^)0U+yh@dzfHGl3WkS*MVp?#2fLuP)DB7K;Ui4 zUewWEhxn}1QQrr50kUmqoJH_8P^+>=M)?AdAQcb3(VOE&g%14@5fb$pD3n;xEM|@P zWz7YM&+x23ww}joxU_5#21wTyIDL)>a=NH_j&$bBa`1am*jJbX^C=z$ZZ-`sb6l0L z8tn`j3CxKjl%$M8`zjxW_caW`@Gms8^pLlNMjMDw4xGl?FLAI?DyW33y_}~!SG)|? z3py(1@H5tdycJw?D?U!^jCiJ{uDzkZ_F9F@wu?(fTm?P&O*a{ec*fNG`O@^P%V-4Y zDbQT#uG5CPpXo!_2ZNU3{o%zCP8}ht{nLTrCj#=fkp6Ck*it8aW(&6eAR-ysF@R$U z%nU^8FPt%wW{~47Dog!Z`qeFP@6d*e{GY-n5)N3KG#dv%f>|O{&I{^0vrmL9m(=}A zoF_YIrc>h+ef_XemF~O8QQU4nTcZV%qNI`aAeEN-E7~ikwo+SuUHCK5tAvjiE>+6M ztcu5P9!sI}6IC2)Aic~BaY;v6WX9rFGVv(FDHM&UO|N@}NLF8AYEb~tD3No9qgRar zFQ3?oi`uV=;~8L-@-Jv2zAheji?a;a+;4lLF{~XPOQc+(A7-_RxZvP_cxU)V#qo^> z`m%~L8YHn;$4Og#VQW&BL?O}}cgCn>Z5uNnPJO=L?q#+hy3=THTRzJ}V#f?A;8B*6 zTatT-<r3dfZTjkpAep}@^$~Bt#r$gJS-welejsN4>8R^uUj&-=?)iT2)8CjPH<%Ze zafQ=kmDmca?ToVsCrp}tDK!lGdH`Roc2*l$`uMM+D5d)Sv?;kzcC`8X!<9jxV^3~# zFaDC_-3tWe>fV=z<+pA1KNO60Jw0CNrWBT1Ga~=9*k{V|2T)4;Sx<Zp63yGK&icGo zzVb^X_;7!0ZF@fN=AeIho$uHLM{@3iuvC}Yk9IRP!1~($zNg7}pLPh{ZDpjSs%}Yd zkR!Pe6nuCmZ1;v^U1XAn`{L;%3$mEYM+ZeM$Duj)`mMIJ`k}wy_Bpmp0BzxK(5ctW zYNyuDoz-aZw!Z0)z5ip&cWxwZc3p6&j@}gB0i-uBfqh2Gy~e-TpV!c;;a1Nc-}m{n z`mJ#VsMZMhI1GOOy#2=?NXZ><J>~Fnu)*gD8b&J}a3r3!t{51fsNefGA$3mKzH#DJ z-5*bOiKf3L?XT<l51x{8>d>^GE!SMaNK2tj56R{$;5%cDWBe=o;}FMBYiWjuBry~3 zIc?Fv;(DW&gLj4p%h|qj+w-IK*2H%aK#{BN(+=2RMpDdx@TykKojt-bf#`kQ6mc+R zikQp$T=CQ)yXZTBm);*O^0D)oh^Rhw$wfT)fzkf-hkExP93O708|vLc+r5xe<xbgS z&o~2t!?&x&kaK(}^9L5&nQuf<!UB4b2UarDjPz}Y?#1DZxwGhTQ{N(k40XsqXV#a7 zN~A@4ZCM4bK!PJUPd`J!S~BKNEN14*2K3qQms~rWGzXIr-AV72p>*ih78F=!U0~V! z)&AVXSlYyL_*-T%1Sl_j+T7261ejr?D<`<6Gv<*okbkU5(!=QwNd^K}xGEu4`(o0_ zIYjxO)#gk0a)s9t0%_qs7zvizOx$_68r|l=Y9r&!PUJP^F7@S-0tk`cZbNW>5G?Q} zGT*&kq5C(O+#Ujbnbo7r)^Gl%jgYJvA$Rr?DdUasSZ}Vn!pL(w`1uScLYdQr3^e3C zxvSqeqwobgF{MqX`XD}5hTKi1^<T>P0Fj;ek;G?ZKYR-xdq<DTnI4ZdH#5zr5N1f$ zZh3Ds$kx*tXXIwTjf1XH`Hw)0$(&>SoXOhU5AZ1nNZWokPNO((9Vg59o3@%X7f&;o z*E6%IoPD~GO3;Ga0>0x6{7$N1nNyE_#`eNch_c4;3Ab0f0**M3Z~-wGR3(Ru(`J|@ zmnr4&Ji)1B<Cly{r;cVVk7P`1u3z`YjU}s(k$~752RmHQLG+G8jVK<3V=2jEE@o1r zpeOe0AqKJm#>WS>7X*BG$*6VEyA5s@p`clReDY34pjG!rBehn385DT(D|9j;t*fzI zvDXWq6AE_%A#fa~Cjtw1UIuvdBAID@;XA~y?kQ$Wu7(h;GZ-zkC=iP<-{|$pwgVh& zeoOMOgou>h7J?=PosIO8MWli^)LD4QcGLXV%Hwb+v8TY+6d<5{Baxg5dnRfUb8u`- zAg|nK`}slk`cunlSAz|aJ;;LYRx;JSMGfD}Im)poG~c9y<lE4MwuNzldIcAxJfN5j z*)fn{IcK@%v|~=fG&%&0-Bj7CHWF08arwFZQS0AMpm-HXM}kB}6lilzRbOD+F?y(g z?ghye{V!)Qp#ybU1FzyAc7>ILdO@*ze>#(e#&84Ncn;#QC(*B~;UHTwd`}f&SuVu^ znkugnUV-1!M+#m66{R1@VOaUua}RLfc(jPD^LH1L7<fY^()A}Pe9|&6cdz%(a;lIh z$niHHG<v{cKtu(fs)^B&P2J#GQ4Gutr07GROY;xQFOxM%c}YXq2oDK23F$)qDY~`j zh{?Xw;#S4^bLTW_>OuZEWs7FPr0s%%6V&G_a=!p6m0x_LYT;cGJmf95mNvd#MS7># zi3MJtQ|lD=^KSVyRC;GVWx%{w=|~)RyDQ)Hb!p_N<DSwy?NDe44fx!%nZSzC0}7+7 zgAr{`rB2HmWg#1}$IRe(RLuq}HdQruDG;_!i8>x`Inf*Rhxo7ZKCa{K@UavmP!P<= zwmg69U^2_KGFbZUn7)uh!UOZs&jUH%EF3?~n2*-)3Ht~h{yax(3c?g`_>awgU)^XF ztG_!%I3*egBo?ljXo%@A)(_@P;`Me5`Ps}RIrY3cJ@e{a>-=WIrYnsO?vJN`OT{<S zZMD4S7aq>gBo6#Gr%!2x{l-f@Eok1Zy;SfKr&hDIzUqDc6?S@4G`L1u_U+V6iO|;) zyD}AfFI|I5%>xDW<#|xi?-c#1xzG=5UwMNy9spWSdAGOvSvzby9jhX{`dy-CEK>6j zKYTPlS}4)tEX!t|z94%0K8TPdR{LzM{#f+=i@ynCoO5gr{!q}onfzrNKj!yy)U7CB zDAB$MyXaR0{enxy_pf?S*TW3lf)?A=wE{*T<$c-=0O~LChFjf2Nl!F_B|Il~^e2&U zo;926i(MZE;w|eo4-yIeFb}&g8&9bz_{ir3$J-V$DX*$kS0fD`F8`zC^hIuS?VMxd zz3eqWHCSscxqAhAUUXdsNOT4Ph3%0wi4>)0!cz_3`5nJ_L|q?Ha@?m1YTetd*$}=V ziAhW;sBimcr2qe|y+y>l$%fG~-Jy|>8l=-7%3iT7PI5m5e+@L<j(1n$7Gq<}s9u#f zc9Jsb({3%Z$rTXLf%xz%zzb^-8D?j=uM6Dx{`L~jd|J4|Z!<Ui6aEZu$X$&H+IE^T z>1l=)WRHKyTL@wNHL)NB^4aQvG>#(a#?s{)^+pJFCgTp|!TY2dxZb6F9ZJ7!tCPkK zo8a04C=tlUyi0e7pe$s%%@ry;{eec3Cw4L(rwHA%+TQ<Tli^PNCK#Dd2%y_ykY~-V z2uw59A_eLoF`#qs*!1SFQB<LTnAkV#Z=`<PemxcQ>?yyf)9YeB>Z`ydg=egbX{%%% zlo3ukd{(AX;WUN&t1R5s`IdaqQ2VE3p@763V>RNUDz~XXY2hj)1}-iLL&{OJ{=<3; ze!`tJg8W7i`T60$GSlRA;WZ%USNk#RAd7c-ji%+GgkE;sgN0{2vXIZH8_TY%BDY8w z9fb*0^@T_I0SYJXsmlms*RdT=vj(&tP2`Q<K+-e&H)yP>?20JXLY$h~YX*-bjdns} zIitjt4!7KOP=RjQ?m7{h64>n|Tww7dK}tZJg`03IV3bdrpo}t_y;{^-4~Noi%LkEB zIbK=j2A8$S&s^JcE}vdu-NAa!I^rdc9w}`lQ?`kbvhg`bT=}@;Ypy>%Y-a=AxMH4B z$UPDxF@N>NL}$o4eiyb~mDN-n7VY<_V&Keb%g1-89r7H6?x{mpx3aaM>+dsnavD4= z`6J|lBP<10KCCy$$1gcH4}NQI*WN}0;tXEhZ;XhYkt)Di{E;%{(vd?Zs&Tv?D>*DV zdbazGUGJ@<H4C%jq;!>3DN|Jou8-yO<sXqA)C8)>7eYcl+Iv6`jRC!Gu$$G6JvRB( z2|s`n4KT@v4uD>}#aWD=8WkcXLkV^`L5j($R6JHobu@qAOC}9vZU!bjquFB`{jVzc zFp>DTQUSK*@|LCB$NukNH`EFev7qaBSK@Uz-f$=!5gtAhi215Lor%0g!$ZUS4#lEm zt_K_eSz^T@C&iRRu9Rf%2(SJt3Olk_p)TQjDwMBJ<JLDcDbYGfz`wwppq!906(eV5 z4K*SA9*Ta8+n3T4e@e?$%3(MVMzLY>ks-F7s*1;50q#m#SM$Z~sXY3na%?0;w}TKn zz)ckZ6JWYU8s}s*YCHp5iy|jY3UJa9K>@k6T!og*tPolG22PR!N$g6#Ch|Zjf-qJt zRG!}JFsd^u8-5}Anh>I$*8U}#;;y`5G!KPsC}BHA6LP+V-#w4=!{-Uh8@<oo6OlD) z8q&h7mwU7rseyIyA;Ve=7UhHA#;<7w#_9>~PcL#3Hq-;H|ApQSIr+}r6cP48)WP<H zWc@l$PCs}y-ft$Ncl3cF|3tgnRPf;FnvVNQuw?3LTE$OEk*I_{>Rz>9Vx?}xyn|!o z-2@MrVUN1TrplURqE-2A;;5g#tAYy5uj8$9eXpnC`BGbJfK7s_2no02C-s}{<Tq0v zhg|=SdiRA3WuAH3-ojQ)OwH4inrB%q(;_XoxjwvOXK#|-epCBHT}nZn!77)y+P$LE z#gn%SYFQLtRS%xPgbo+#JR0RV*;jooi04l&Kpc9jx$KVcJYU!0g{cMbeU2}dl`5V- z;B;*E4C5F19aUCXQhqUI;U%%gh5sawn`yuQ<DO5DXwb(lp0Na?slk)Oo8E#|d*V&r z*_g<#m*o?lK$?dS9Xx+{&sxjjT<ehVzG~_H)C%HfLcdeQD2I)+`wkE1cpLVk-{S1Y zq8Y(-aB5+b<Y9y|GO<VNPrX(_f<s@e)@bo-OrJ|#XYi{2y(!tSa00Vq{oPwsKf)nF z#l97QbltC8c=>Xo>t)51NO0}dfvz6ywa42}iC7-3d!tva`+Pkso~G1ZyY;{V5hZo* zQ|k@5zsHT@fPTN>4*5Q(`-J{GJgqNKRiGfWx!<cgdVtUL-RM&H`p^F-_X5~jSph}J z8A>`?Mi?b1aA!M@O`>(k$}QJFEAt(M`$7#TC=H?qhv2LPwjfhhX_;?ix-PgS1X`7+ zifJoDYOT)ir*da}S*?|ZT0F`(JJ0<z0|^O(`_lZcHx42PNngQ0Dw_&9;130&^2#e5 zGRFuFsey2*O=0`g%n(buG}fvH#g@s#t=vW<lj-T1z%FT<k}@)5`<0dU8(_k_q#CQI zPp_s&p=4?3s{L$=l!*nAuKn7oCf&7z6Os;pH)I(uuA)Km%9z^b4oJpILoIb{(OPNe zJ$$W}(ye4gY>ubJmYzR;bWJj7*hOF)!Y)!`*0p3=P!nsBQ9|uzhvInyi_&ulW4Ow7 zQ`uCVgxGUk?)VfoxkV`EU4r|WEjHO%Gz>)Jf7@_^@?2+&xCVK77A&31b~Yfs%|?u0 zSL6cS&MnVOH)v1+pWv?iRVTov_$9@?KR^lYzrYz_DpC8~URLrfADBwEU$3hNecWb0 zN*^S^%4L<!`0XGL_rQT-;8|7yJml5-jo|Zp*52+mhkG^lhz%YeVIO$2Yrj27eN;kV zt^6M3lZY%%4lhm-0uTAOaG7Kjgf6pi10bqi5r(I7`iN%&gd`P+4Dd)+4ae2>D!Qe8 z{%ajikDen%@3cP&%jCB-D~933p>3iu;~k<&uyLhW9pm$A9Lh#y$8a;Tv(K?b;@xt= zh8*vbc5>?Fp0b@|ym>&1&Ot^H<M3u^r<5`oQ&~4j5}Th_0s0hEctQbBYKsphqz&DS zj@r=NiVI`XCkXq*S}^3b_!sS2b1xX}<ye+u%1c^4Bw=?5PBgt9nqD|{qk301z7t4n zWKm6!Z^B!co*#**bjznYw4UXeO%`5=#5GW<f+buU<+sYklr?1Es1AhifsmSC*Rfo8 z#ml}RrK!jq1$jCnv<mxmJ-_D*x5<fi)SgrHoU>gNN@?`+G9vO}m0L=~tqEK6I13}r z>VE0RP;L99-?D+32f99DKBo~toEu6l)NP-<4A5s9)U$I`$}w3c#CJ*5*S@59m7s?z zZJ6zt#tyx&Pgl?eat!+?6gh-Ac&k@)l$-QIXG1eB5EX_`6TS#}+IbCB1d6JO2PO6L z(Y`&kN|x1AdzlPfIfc>Q05RI0;2;dwzLK?R1k!rptdU%5F0gi>a)KYWmYnD>`;si; zK`Z&^J$m;V=brwk`q48!@{$h|&}}H074zMjXN%Cv-(leP09qyDzW~GV9R6l0Z=B9U z$wfUrXS7GOTL^N1!Vl_7{*%LFaPGQt5vBgegORSc%Jzs4TI(M*9rrBpt@G-^lPNi> zV*rTV{soX&;<A}suKSoZ@>1*Zv`wB?)Qh^`nqTj0uAU4Q8U=s`o_7+)ljA=4O?+Of z5BYH#Kz-KpVpIthmva5%A}kz!e6CP3Y#OAU$n)LG_Ny)RDcJ&&AK33gjw}5aDl9x5 zKb-n;ZT!jg$KlWGev7<tX&t;;ys|m9T~~C);bV4rQYFUwMo9SbT5ze^+CPQFcD-$- z8hGH#+g0u4@{1OZ)^LA>AxA3qH_uInqvGE;t8S=Ro9Nt|RNB7VPpZ3VsQB5aq-|Mi z4aD|U&Zj<J>r(Eg;m>@z?jK76X}lQs-wy1fAC7eL47?n+8D<|w-v#pPTwAWKzKbZE z>Sh1%eEinONXs7I$)BJ33f#9|`q$2j1zYS?)!VWXreYm>%8%XwR0RjV9-m3|%>i*S z-5{){_sG<;HgcxN-?M{Y%geGhQ?gjG*Y*yq=@_r&weaK<Mg8Rjscy=5f+nHm^_wHi z_nZ5NJc)g&raAygEJ)K0S+(i$n~_B(BS(IGoN@?uwhZ%~JLy;*o~bld-Eq2d+P4Eg zP$$23;8?e$Lw-7D-{bxlXTSni$+&>Dfh1U|h04Um@)GIt4pvu_-~6)<MAPwh@W<(^ z((pOrvclbNSv$BSMY2ncAt!?S)C20X!lfB2v@fG?$mUv*x}IFgu#sytH$63!#s+R; za_UVwC3=q!b!kR{8J~JCBfd7lU7PzBPSh?rBSa2lTbxaXtdZg@1ll1}8+<LVjfe%t zo5``7SmR<LQ2LhO+Ke>CY&fj|4r#PPhORzLj?GK5T|H4$0O?v%B%AyMoCXa@-@Xk$ zD0~CvcETq-vbvwHBfI~{R2Hs<k2e!Llm}TpI_W|R3%4NN`Mab)8_{M-eYQL3RVi)K zCoI3EgR@NeYok9wwBUnbM;i`6$W#Tt4AhMr4xbpK*zp}eqI&D65Dtp?CJc?DL;WC? zAZ_7WX8g;9v0fn@mzkT{aeVsT223Vx{rDq5YC5+GJGS{MRwKaDhF@D5FLhJZ#UM3z z3flzjzG!g`hPoYldVv+i7?0J4`;ij&O&zE?p$YHINnjYH<B(qz^gvh;pX{!=E`2^( zRP$Wz7mXqi7PTL}env1Rtq}+9c3qPNo;dp+dljV6vg?;$Pe+A>h~8ohi*4yEcH-0% z_K<`N224!ROmH)k>n=yaUXx;q8k=z}qmJ~2X8<b4NH7!9-xzOyp;sz?Nl1p%?{0zA z4CuPO5R!GW^RumZTDc|U4LZv_m+M72w7jE5S=|%0(6mmMrwmNaFRH583pv0xZWQ|< z&Q=Xt*+{$pf-Oi?8CHg`x!n@nl1OK|%Y}U&vKK7Fg!ReMwF57yN4obZw7l<VttFw8 zAa5I^W)<nnrX{eKIeHNV9t>%XM=<wPz(|)bPHThy$?Y)mI;4!$X8W8;2;z)M+H7d$ z=bZl7pt9LgCFYdG-!J5{!8spDBsUx>OY8~LmRg6Fe7IWz+p9Hn;#KbP$?)$b+Ea2h zqk{&`rI!Ou3XO&r-SPE+KM722Q4`{K*jI~&8RZLmtdUy?pZj!X{rmR{h*?)&4pJBL z{NoQezKTZT7e@#o`;`w!$THpPmIMFMt6E#E7y_eShH_)F`70+*QbriMioN8La}8HW z4C^ii_HOj+<5>H1&W=TXoc01ch8*tKR@Tcy{}%d(Z!#IS0V5CWXDwo?RDqhNtTGBN zB#HvlztYUXC^6Fblkva-PswXSfmwB_`#p_L)<;V!P{IL2hdhoYwJh5Wx?Wm{_#@<G z8HP+5CwfCF?_G1)>PY_=Q%+Kjnrw5T2;lSC|BSzL{=*eMK!4$jDZcJ~%I8>#3~XzF zy>gGASJ3jQ+do%LKcr7d(;0{mm;p)iKH?^4PwyXhk=(eO`eo05p#I|D?~ktVS600G z-hbb`>brW$)+0Sfi<_3#DgEyY9i4)Iyvu(T+DY)4yTVwjDcK$Ey669U;!194+>=k3 zmEuRs76hMbLWGjd`{o{lZzr!fo_)Ue!F=(1!gl#P@nv)4l>Q2dJ9(flmU&AqeM`cN zBdLHKNL21mqh=SybBc0!@HTpDBVpZm$LBW!TQ%u;+uO3}hP4mkk|S?*so36lX_@me zG_0U5q@>Rg+jSzp`K090=SN$TK126h`Asey)J(;#s6WzUKav?A?Or$}6%-`19=>t; z@#UypWeA&Je<g1!0ov88w5v<Ly2^)2t_k}O>&LAb%ijWYVoG8Ar0|BH)9P!QjZ6R; z&Eu|}#*>my(1f6@2Gg#~NNf9!SEtdXMa*mBYOI(t^5^_B!pzOhO@xb_=oeYC-v|Vy zmA|>el?-`zCQr}moNQs*HY(!rxu7eYe#MajHV|Go|5b%9RW_~+$NUQ!xo;+rAX(F4 zRIK|wQ`w+~L95}MxszQX0*;UktJPFVLp?|oyu4GP1Su%skk$8qQ-~UZX(3w+(5dMw zaO5Lpc!0nIoKghoHZ5A!N`NmQn(fniOZylVS@TCuL+1Q1jD67%Sj5T@GOClk0;(6j zhL2aX8aW}Mr$JS5XkWS_EmD-ukUlAZR*uBM^iD+yCQu6z5TZz!P-?>r_>;wO?jwp& z%Qo@$wq+R!Ay9-<HRYg0bD0Uez-f{P!YKuztJh^u$K=Sw-xJ#jRD!~r)l;`mwP((q zn{WE{`>R^OEZVk{52v;k9HR=tajcn*xF(4EC7ohvOpTOz*cctZ|3`P>NlidtYpjJp zzp}<v$tG^gRB?xY2A6;h0m?Pw>nALU)1zt7N>^Efq`v@V5s0)LA&ExipJQS7p!t_y z<s-3?ZQS%3L+@+j>f(Y3tvKU6)SY0_03)&uUFP;j$0VrbT${Zh9s4?KKogl-9)Ob^ zlkt970W-}FV<wyBZYKmktR&nyPtDwqb|t);+d_>YdBm-21=S=mK3&IB^6m2WO}R}C z12f0VB9|{L{fZd98E#Din-n%$VBYgUQ(I&p0#&%<&)D|zff8D>$TO|2G<zTxn@N+t z@bxeeYdj9b{kjBkb2HBq>Q#qE@TPYds#p1Mv50`#=OH9U*B4aCo~5cfei}hBgjzdc zN2}$rZ>2^D?MSJSO@qi*hmJF{L&Eyor<k)2BmxbBoeXz#d8mZecP3z@=BUMFR`2*t zGw-<`mGDTD>9@9{!pTtg^GL5z=I{X>^-5%NO9}RT5$Fi&l?%)25S~?;A(Y`b5Zw(} z$?vBZBqQ3uxU=M6KnCc{p0nFa27WJ-+^g@_9hIFMcqa2e8<DnpVhG4cjI;xYIAJK^ z6}*?aUbHEa7{*?#X=KW_7xYXsNjJvM;bTr2HJUmXF=|Z8)lD4W7tMunA*q>-D1VV_ z;eDaKV7giFCObbqR=2}TpntaI;_>gWaDvWd`MrCS_-QwZ*M7+oeu&6H@B*umyyu<m zxyavJDNj)m-B$AQ|Ds7FEl~h(Ctl+5gdgnXPr1PxxZ>bgi8y-R(xEGRXOsL-w*jSq znKd%@SvD9g)UJbmL|#exl4>OXkKzdE$%Ed?ZvyjSO{9Y^3U#yNfSV2?TKYTMQiMnQ z&(_)+;wRG8?c#|p%^`<zQmTbtXyp}?&10W;#e`z}1U%w*JMHO}BOA_5ID|N{FeK`m zI|^V9M8xEs=#mhyhJ3TK$w)RwO2u4w^fLBr*ZmA-NG}K!ntv)%>ivby1Tz_KNJaxB zz-r_2>0%LXp^W#CT`RT>Z&|pxDWS*my6o=@GUz!FInMP$MnS!FVHy+UeVo;W%|wd( zlN0))6BQR~^4@^_ikxu#lQuONJu8N^X?0U{tlr`v$K-`P_#|HbF+8nCJu?=-C2YT7 zJI#=WTZnn9pksC{voZ{lf6kh_B{meTXQ5r*C23)sp0&bB>8kSS<}QcY+ujOp(g+h6 zrSwxa1Pl*g1F7&})7Ydv((F5`M&QLifcR`ISrcxSQIKWMlG}3i^kT-6tOGtLE#LgQ zY0jHVBke(52K8%Kp6xHAu_b&EJ4BoXsnER|&{zKn!Z90lX5*B|wqJ98no*zoAQkcH zIMuPt97s{0jzC_1(wRAZ{!rqR@P|eSYc)|!rdj7YG70h${!1+iNdzp|6CS7n5-u>0 zI|of^fFFOTM1~jCqIEB>?vs(AUib<1Y?R;31z~tC;>RI@E+Y@xr3btzOxsa8a~Z~v zt8d_lk{*3{sV3l>a}h{7Iw&D1UTct(j-NJd0GcjBat`*^!Qp-RsqDmz$DVso^2Gxb zGwsw0U$?X}HDzpEB8AEYipS!Jy)L81_yQf*{#o@=$f)*YJPai^*)Oa<285MLTnD{j zLZWw66(rAQ>fBwW=+3LoAoU<WLiwTwZ%)F+Ya(TsGLpT3PEe!r^k?tzqRG@}t^Dpe zN99W8jg&nDe1?vfZThOh6}z9csK~%eYF{4#hzBW#L)bl)7SDdElf$qDMI+0QPXFoF z@DoEqf5du4D#=Mess3n_8<F_X%q6#^*Rj$BBJI4vy_N7;*m8zI;F22Yia^%vyKzBd zqmX>T@z~P`>ne{hznogwI$rhC5fWa<y}^6Nlg7P+;G-1SXb>syQL_XPY`v_nv6cNy zO;sP&iyBd1Gc}`>vGNm=g?w1%t6MtJ*=SyDEEp-*PnI-3il}g3^;RVJPj*a8<B`|s z-8zs9=CKw=vem)+vR@HZvZ=Y?=_z6GB+{z)mG!Eu*%c1A7Fd750i?w<5^FK+6283m zklt+w!hoWXcauLxH73}*`ZkoQ%8zsg)uO{#_r$t8*UKGIk%B@nRWv>QXRp$-mpFe< zxb}UA>ww{tG@Is^O=d;((LEWeh2^kj0$nx|#p4`!Vr#H9tHaAWm=TCBK0j*k=LJ0_ z>U~nE@p;)z5?`#%sFhhDo$$)smwd}babHaoVNsB*NH*4;fmN3A<@bp8PH(C_1S!@Q z)NvoF+2#{+hJ*1ZK$wG?^>~jz4<%Qu=%T`W49^oaGZ+6Q(ZH7n*r#qI^OxWIuWso9 zl1K>O-qjacQ9o|xefIvc16J#~T`%DqB=)%eAA?a1hF<P!rUp>e=B0hD)Xd0(BwJZz z9GBtlR)%E6Z|Alg{NW|;m=}wf#JFpiN_FXUlMP~31<W8ty0*^}-iUIm2>3vn;M-|9 zX{cQd3L@2YHp3C}$|^0xOb#0R%~T}IytpfhGui)Z(?d(Q#NbT#V&x1YpnQHJ(-`vG zJOYB@=Xw7ez-c~8u7Ur_%ptpU%N1GW!B^9>-nc;0>7RaBnO#vfDv5xwkr5s2cLXj2 zSu>-Ik>Uy`>!^2`TN|1!0VX3;J<1$r%CtEK`8M0J)J&!~N7M|h(9JA}%Hpv6WD<W& z73jUyQ#Ea?^6yV*Ntg{zcyLBwvmFNBpD`$VPpnn2T+O&VgG+v*N0^Fmi;0z?SRehx zgX%X0ZHNG1q(l|q=}AqamWsP^1fU`3x-ODlG{$h5KFez~vEfH28+LgJl#JNfs7XSs zPzZbmmgF*NVv13-T9A!b^~9Po1@zR_CPCel1C+K>AZPtN@{yL^UcR}+<Pw-gD%<%0 z8w@?Y8msT^R^&`V>L?W1<o4bYz~cnw`CE}}Tpl<@Ba*@7dL&BalLk;!$QbwR|AoK_ z<I}VTZ%tkb?G?ipU`_221*o`CTeaNKE@4%e0C!1N$K-6Q!-9;|prx77q{+0Zc7)%B z<kz;7@zv(jQjlL@Gq&&~?@s&b1&~)V_sa96R^c5=CtGEyFFyaNebs~^E?<5Wbmksw z#NBC>3O=STrtX#Sg$Aie;cE;z`G%K9S79q}*4dI=#+X{wy!ct{Re;2Ct~MiQrRE)S zaH*-1a>{5!^98gBlgzUsyf~|ZFW6sRIeR3KhvSX#PzF`?3jRq*ZX%o<o8`f$?wdt0 zN<~I(7rw0Dgv@xe&XZl}WEO1#3SKG8(Qx`mEfTv2$G_Z<ci~?Y(O*ywFwH|kN4+NJ z(F$JThuE|cf^ISofnVjG`us!5AkejKG4P{g&BRbaAYsZmD%EPK-a_TRutZUTJ(S#& zASnd@`S@pF)+4j1*;E$MYbY8eI7M-hd@T-iq82j9;i*kxJaj=EKl5D0{+9v?rB(fQ zzXXZm4e!jKzYX?UcBePItoypo+k0UF8ib?etK5ew=UazulpQ^I9PX0K$yGO<ts~T` z-V!dkNvM&(02tCgamk8%XWU+{gs&-#db9d@+Dcy~^UV;z+u1>}2a?c^!zAcz3oal$ z28`sx#~uj!k&}3-&x99eIwvi`XA_`b2aNnhd`4Bm`-Hg9476&@A0p_&oD1Ho(_ig5 zg{><Woe6q<LL*Bmfdwra9_*t}Xx}~)yc8!-AB-OiOZ8L4;?rI&G5(V?ic^RK6pCc; z8d%hO26OiK{g`@H2lnPYMsv7-sC-lD12rdDyj$`!01@}6rX)aQS){wJWcWgEQy26O z=fyKFA-N$caMe(mEQom5r3}@UCT4)&)a9A+5VFM&55f32rAbYgh2~#iLB3HynS@)b zp%7e;A#dpEWe}oO{b#0*2Ydq4BjyK(&=6;#L0C71a8ZWsVJZ(>;q|>cu8@D|Lvc=) zHAt-?ql~w;&((f&M#wkX9YTK5)6*$~D&gh_aMM`?txpp&M;B$x`?mzxxlC&#lOc!R zDr?fH&N}f$1A1R5^_AA3at4X3l3@=e@|ue+@kGi&@%vooXE;eyV6;W(NDOgFA+jpD zOI^qtDM2%H8zw);wiA<;0v6!`sR#Yy7#<uu_ho_KkO&kg1G8#x;wrTdSIccMiIwHP zm~r}O9{>aXleI`vL6}HTb3+OCsx9JP$^1t2fJ7{IJ|I!Jy>6sWcu3psy!uPVxXz^U zM1DtpDK8GXIP6^Ml}bqBZOakl!*N;k?zd*ar<_#3df_DIhrr9S0ZxStf0G->$((l2 z0woa36m%8cnw=W6f3h{c>D{v`Bzri<TbT@|SsjTqjsG+UHb<$4cZ`#L7Fvkf-iieK zR;2W(QpN#N5SMl|qqxH4=RtP^`}_`4jjVm4N6l`{{ZyNq^OlI?ZHg7<37>UIYeNds z;H#H8ww?tK`!5^wx0jVGGNv9;9u}@gx;smW^7A?6t&ZO?^JeR}yj<oI&dnyB&~g># zS8U^=Ldk};qz8ucy4%nTJ97f2#}Mm6^B|$^7mX<}q|$Pkc(9m^z%>}dt=C&HFh`4D z1t%8J7L=Sa_It&E{QSt_BVzqdZQlL>^%43}tD3g3|2i5C3%C%aOIXJ`YgOF?`<1Nc zcSJlh8nkR3BYY(`gY}8f1<gZg{Pe)I<vrG*D++#Of1$4bc%aFI&t_n%lK?mnNV6T3 zc>HGdD_+AHZN&r&T5k8Wo{5bT_QG7tx^RbKv=z76u>Q0>;6y~JvRgExX#jA}Hb*%w z&1j7905#}a(a&Vvt;|sKyn9~7JB=Fl+TF4m!&-X^P-~1Z=g}r8Vz%%?XudVZJ3Gqa zwM*|Qp)<Lx<wv;Z>u)^sr^~1*foH}D@#71(yNiyfxhwAp>wXN8^gVhHi=+zOT4Y2p z%4O#%C2rk*J7+=DZ^yD#?Pl*28ZL(rX{*SaO4bHI=xWgO|Dw=68z>9c0)()_8iEg= zKt7n?G%TYM|1ltK@A|`Mf?8~L;dwwF^;j3bfc`dmjt6B#Kb4oi>8`ri@Qb|}4~B5u zvHB$p7wv9yi3FGu_{`a4-o>=60Ez%PymD(!-DB(3);n~~K3)&KwMp!tKojlk!;Tt8 z{j0HSvo0B)GaQWIsLL<+UW`#9c6@ZR5fqdy;m}e7Qndj+w1s$vxERhf;4>RjlP*h6 z{Siw>ICJ<Vo;vFt=6iHDGw?8?f6pxcAX@7(-%e<&qAWEo@-s{VZN7J-9Ao%kLvAe1 z-twf|6E=N~alfw*cqMxXJ_4m#7rwswsl2-2D#X>)JbpGN4Oa<7i524wZG@bpwHYmR zUkEX<f?%>LOex%x4t^l0AWm>2=jesE(8O(G30#Yvx9Lxp4yD`IpprxY{1H@AyMK~z z=lcn?3yoj@?Ka4|HolWYXjG1L*24zMM&5ZI*5}f1?qsZkL`xqX-V&I~vcr(ZUjOAr z*wo_j_!oqDbS42%<{?wo*iyG7J%}rjmX1K0kT(h4Ub9h;ubT3h9Si+A-Lz!6N?V=w zv7t8*KJ960y`?+tp?K~hgIB%XKb<Ou5W;Ok#N7a&R;3<112u9I1KYJ|4Jp%?*V+s? z?qu0LuE<{Z+ihH@4Csn)?aF^8t47X*g?)k1S9XTvY1e0<^Yl%wlo<lG9!El~ukB*n zNb7sfRoNn9)_(#q>RIyy1R-lvcVS*`voXrEVaxy!UX?SNrBer(Jxs#x`GossS=?i6 zhb|ocY7U9Q%`_QUL^yXw^>5WbOeJ=NR|H1U>WpCT6zY9jwH02ii<cE(SHO#%G^u*} zI5x4sdW3M`{6y`c(rKdnf4UT-dsLgAMMv(G8q1Dn)yTGYmZja2P(TC0cmtB7kS;I_ zpk-!~B4B{hsbS$#zS(-17~Be*6lYmP@e3lZy;u@0lL_i<xiz^{`US(~ALZZ|SJRyW ze0AELXuW$#MZ8EKrUg_P;Uy7DclX%QwOfQ2n<3i4$~6biG@oJRpuHy)f}t{JC{!)y zE$;?1eg8l6i)QPckE(nMsDY$`lp}o*Wla8@!tS)YTeF4}{mg)#n9HquYu4M&=RG8z zbn8{s(%_ySS5P@7uG<7b!E38+vy$X7$Y-OM!^;od&<c7A6J(EeU5}so7oTszyNT9w zEkcwa;)mqT-MaEUsv9%9ZVpt`yJ8;z1%9gC<KusV^7pzoLsxfD-HHF@=y(|rE;=cu z%Q>v79w!iHTf8MGfrcqC?2F0#QMU9(N>p|{=GLfic6?Mou>p%m@vi(qBpvIXdf^_S z=W~7SUV(d+ojYYQD`B%x<zJ@Z|0<Nhw$cXYv0esRNrb2AIVVXb9h{nGs*)A>esxI& zg6tTweOy8ir{KemF1nC2m+os&c#>Jo+5O~`=V*JxPYg!h%5=+|dsGE?kMx{?*t9bJ zqlCd5WwRT-Pu7nPPu=2z8+lo|QbYI;kSQ;qrzOod6nx(QA5jy&|0{fU^ZY`Hz$I>H z3VgPh3#eBM+dxhPNUp?U_U@hPmt;FNVSaT2c*-geFZdSJw{8a>Fp6{^My134Q`3&> zEA#(xskE6)J{bLI^`}>@<c(m3aMa)+q{uk6+e&8fIIk51CH2f$%1l6)3&d_JzydKl zty{nJ_V@H!{Z#1Nl#Y;RhC_u*xzIu^A(9~264IBvHa~x8)GCmtd}`puANC>MkoCI$ z&wKCng8Gg03;sE*3S9Ol{pJ3X{?h$FTz`MH^x}(iH~#r=FS_N=ZB}kpVQ!v#js5#6 z$F*bL(^9!h3|iIE=iKf8{^kF(dG}jIc8$}6_P`DYo}PUj@O=9(jj%7)?dykscMxE^ zyvlE_k+<JR<Nuz3h|Ss8*mwVKcpLtg^8nzW|7P4qAM-B%dzAJp7SW`{e>X-;SVaH+ zUJupMXngU%9Xs#-|5N$j7v{fSm4M74B4dphdD+O$-9Y}v{C6EKgTp=|{Stl0<!Hz5 z1#GjO)%o9CeWR)C#)!Z3?D*T0gAH)5ww=Gi0td!Mm;|Uand)-164e=XSaqbyn2I{^ z$8=*4J6-7J^n{}gFbRA_juHn*hEa@b%wk=!;fx8`ffr@{=&%*tO7Cf1y?OS3PYBxa zE6gc>XSG_+Rb#SzQ7i33YvNzQ6?9e9;R*%dvnw(PM^rGFd!t6?g#E5RqpuYOygx#y z=lWfg34ATCdOTW(A5A;_dmKX55v_{QqtYsF{qj620UW9GSD2J)d9(!9Bu|YrM_i|U zhaG;mZwmJ7Ezs9y?8A;0pD!fkAFZ_>eoD#j@~9r$DLLATV%X6STaWe-zlTdH086Ws zk{RRfc>RBk?cZhpWvX5>y{XPvvV<izt+(j^X<RI%svOO?`kwS&{&ebWsD17peQQ8F z`bO)EfW{a1_9nIFZ<hI81-DTF)+X*R{rzauh0(+J15N{!EIv|a_F9CJPAerkK4_Ga z4EYsf-x3}=9X#cS4WW>zfapeJ;T?Jum{fESCbdk6YDL_6)Z(~^KBA)Ypar1|xhbok z5z9|_X@GQ0LCEs^GE$EV;}N|nYHyu3Icq#7Kd<}rAP!L8XaIC<$feoNiNMu6+T*BF z{5QAg@wNYUi;fPWj{NDvDu-96A9nBtkW;krVe4P^q+g#^kas0k(Yvc~`s>YtFlsv= z>F<2`>3ftbCA+cmHt#_q@WDr2zS`R1q0kTO<@u~Ej4!QwdnDjC98Kr<>{ghvz5c6O zg7%Nx%HyTpo4U3*!tP93r>#`^-#oX>XXob}^R|^D);d(~c71PW%mo1c@G(rTjdi!@ z`M2mwhHtbl<g9~+F_C~li_&j7__>bfi=uYKa~i+dYP%ZoSKP$yqv-H%adbE4Xdi<n z9(*DCP`~eLw@^~305$&yH50a<=^5u;DR4ON!kG6gEk3HV>%SQ;Ppstp)Aveqh4uB7 z*16rh*8RMc<2$9i^((Jnv=Uh2E$2|V%BDY`?HeB6ZKf@M=&P#?G<<!kYHsPa!QwgF zvG1Q=6ycA#+A!*F7$dIy&oShM`7dTsn_F&>D@TmW`}(2_Jt+!Q^=0=q>j?AaA3&cf zWP8Yx{8y1B&!5iR8GQrLXh`}{)uMN^;`Wbn12!|p_b8eja7$*8mXe_m$)`(23wo~7 zZ9L;#(=<T2Wx^g{^OoCVizk*xh}y*Pk3%A=y_&b20?fZN4)=&e->qX1;7^Lpibelc zrPBmccs&QZF^gUWlha4fOY~9y!<(!!Brz$k&JwMP$dM<NPe=Bu4kxQLKgjO*@61N* z{S<ogGw7DVR<Al`cgkN|@%(xNIrZ%cRnQ>8BD(a`%u^qA`H0)lf2h2@kZd;50o}!X zs>0D6Mq`)FYDAgQl_*{qN&C%VT*}!tTjS9qs_PW=<3``PzV+}$ecp!szZ1om@uWhp zlAXaKULfn^ioH=r)%2u)=gA#{Gp7J`*Z(mve*eWMO+(n4Yc`c*ER1gndCO=kt`>&< zY2<iA+x^t~GulDOIpi)Y2csQKH({M&77{vdKgi@8t@T<?b-h!_#n-rAp)jPFc_{kv z&AEAP=88>LSC*z)zVm^7WHfAs=}C*FcTllUW#hst4)O0`!`e`G`TJ8k;hES&B};Ox z9*D~QE3g78XT=rE2AOm_HJ(EZWE;K%XmMOJ{J_cl<fetFg4E!$>7y&0OJxzS3qRjH zC;C0p&S-+pag`=nf=vq)@@7lC?eR|KSJ{o6Ii`__RV?8Z4gU`2s%%8JJG}~8%A4x^ zeA>UhgI$N^%x_S2BBvjVaaYf=5ud6r_5aS54ltkkk0{{W^Rw(4fGXngK`y3`w87D` z#0txb2l?t=wyZiVEX;{Fx0$<`9vVYGa8!71AHsf>+(@};7rFDn#V=Yd$WQQ?JL}iq z;0;fo8=dz}+k|pi*_quYhd3l2&!hRG#UJtsFMoz0YKr;n64zx}WCvhJbdD_b2Sk0g zY34<y(hyb+=Wt4A-$gSi8RwX2Cc9tKY1=M`AEJIRp;OcQ{ywJKQsCF1B~GT1mwt$< z=FRSfr)?Ocn6Smdum|5OZ)6!?_hZ)Zo47A-Z10Gd7rxh73cc-c(IWcNW_9;9N2WW` zauM%X4Ssxr>OUEE@X}mdI{Eq-*AJi_>i?K3z*|hjf152M9HKF|4x=rC!qG$b&7?#& zj)#N{nG0#YmZ=6-pR8|KcbRc%N;hKMw-yKdqs>1sd=2lj7cgpl=4A&@Aid2Cw{D)A zv^}*kfAGBJn)i)2krlsI6vC#Su?#2A0}4&IOSZ(5x?c1@dQ_)2`0~y7GCOn>mG9{8 z$1ne^SyFhmYd?ehqw!9S_deWom2$NOFW>L!hPz7g51lSsW?uWI(5{bQviiqMvj}Q` zH`lmi?e1ay{Cg$0iklF{pe}BHuDMQG{?X7H&N&v_9c~DFenyGUZx#%#)=jW_DTfjL zc9JEL<wo5tqWAxf)o82#-RNr#a{1P5R`O`*a>UjIoqgDUPT%ptr1Hse_B7c{m;Dzm z`%K;=%CuR&sPS4}_garL5C3>YE_l7Vkpf7j*?!l`W#O*4UR!WZitkrQJG+oZwD{n# zwv3=R@Dm@7l>#;wc8|%rP2Ong_4^x9Fxn7W^m1Rf@$14L&&x28Pd^@m##l5ni!w*w zw2~k6{!yakTXw%rdShT*pm{r@TN$7kIJV3Gx_8{zcx`2i?`VT>j8*iIV~(~fBl3%V zoR;DrUXm{KI-{>H#zIlHVuVe2(lcnGH#g;hZ~Dd2%0t>%>r8pLh9}b-JwVl|b?n<< zP`{Ynzn>s9Czg~B4u{WU5&3Gqy$NXEx5vc{8*j-+xJPd5=d|QKjYz!(1g~j8@OsIV zTZ3h{{1xl2gW)?SEMwd5sMbN#t5kxicTcdwwU+nXYmh33H^h-CXI)~Fbn>Ca&1DvB z`9G}T%<Hv0<!sNGKSw*)bKi)LZati8{ps8FwEJwCMs#j{MH}njMBzwx_w=i}OOcK{ zWl;#M=05>+#`;kttLhBkmM@7#UnCFruTFdvyDWh8@*tO>jV6<sqw1ft%d;HUX49xi zP#*O5afp_59npGgY+<}IpY1a9O~57k@-NBPlqZpkPhQnmT&$fGbiLmjQYJ`eMNEjd zopj}Hu4Dfvc;i=81dI6A?ay_7=OTpFAKs9lsXI?qPdfj&Szi})|9`r-D}aA9f*H$7 z`OUo%uJ3CQ-i?dUzKX|JSKzgnhzXPXT&2uBpG3eN^?Cdq34;rIX#p3Hi>cS$dV8Fg zOZ<CNFkvR-lh%y$??cArf<0?ichRkSLrF{C!d4(qt-Z*ck{8RA!epd4S3bQZ6wR)~ z+RMzx9K&>u=*~uRz;YS0rtL>c8Tb5ifq2k##5l<F7ri~fW%IG(jImwBn$hMrS>9@o zAb@Fa8;x#>;aX*LqJh)AR<1<n{)n&FyHj`n#&@l>zR`C~U$o9FM12q~_;^DRxN7C~ zVD`()-xJ|da)->9DIb)XhR1y!{{bU7&fmVmmGd#^K1)i!&p;!aC?#^jBfAp_3#WXm zj(J1C9~7ChscmSeL)X4n)UH?G{eDfKQQZ5o!pUzZE+2b$UE6~>>+zQxiPYdsWBIwT zF1c)to$_E@g1?QPt$*c>Z}}}wB_|?CtkqofA3~Q$8~lW<U4Ru`#L4b8)c(riP<-;) z9)g^o{S}*=bG!GX!haE(>(AotuW$c9JY9Jp)Bpc>m3+iPk#owBT)FR~RBo$}3SroU zV%D5<4XND5<jPrex*9p>%xuURWzJ!8UvppMx6k*F-`~66+g{J-<9I%vL7=8q@c35f z<Iq6s_77mzFbMH#(x_x1Wli`FFrR$xV6zSUFVR58{@8RhjQ9=Qrm_Nl)@{J0T+Ofp zr`Nsz5A-dpONHr25S5PF-E_c-_o<kKwj={-tG7tga)(3oBHbXY_lUH)FNxeXt%Pmw zx52jN4kK^YCtebtj0pq88d+&dbL)G%t*EZWoMl@1b97Ho?Mk(nmU?f){%GyYpCcG1 z3Y;h(kj39-40w?N$%O|Ph!e$7J|ZDY?a1+`0wgz`5>|!bV)oEg(J`-KI<;tUo;WR? zD=@b1OYUdtVf-XY82Hwkcaw?jWyN7%mwrpeL;tu{9Y)g>Uc<ayqWb+R;PRQ0N+F(o zkH0{s1i1+J|4_%MgXf35-w6@jdy^sMBQGxQyH(}B=bCy4H&{H|a<<#Iw@W*Wc{Ar{ z?7%;|6d^xI2;ESLio5eUEt$yjU6tED^Ks)|KBN75#-ta^QM~s(R#CUui2KngE{#os z@n4U-v4fD5IAM$&@`@6p0t;jec`=3qBw%<0Gk>QUaCVnHe1pBm-_X4HQkV@^E@7~$ z#ZdNIRkU<5fH~z*2@;xG`4F?j6xYeq{3t4*yj{mmQ02l<*Dn!8G*gpnL`L&|e6uY$ zFQ5IYjdJDtR1xyIAl$IQ3Dw}Fz_!&)8mNZ6qv6N2va4qikm-X+ZQ3!KZ*d9RO0AYJ zj&g5;NftjWp3pWWk{~<g;S(pLMNZTQ`F-_&)RJh_fYyvlB-Wj=58E>O%5RGP(ciCA z+5}b<7K5#?0#i~nujzlwZQZ=9@YFYmOJ^V*G4R-ObzJ(UtGP5>XPuws_$wthNUQbh z2KZ-}+ws<wrk4iCZXy<&&9GjY^zLBQO@D&JuH4DjHi3@D34TW!SM_Hx9gz}xjVWb~ z&!L>6(0nKvxa;k(FBHdB)E5H~G$KC;>_(BIY}-CLs$?GV4_Cy7)_wdbeDJJBRIx$R z$VT{CerQ!JP<zVSF-(a4bcrajIeO2_{9pDyGn+u5v6QKx3VTSderK@`#q~;MS!ndF zm3Yy*Xeg&+&{f@KDwV})(bI20H2T(@V+im%jU8{bKZcOop4b04p%_*nJAYO<L*tS^ zHRQ1OKfI~2msdv*4;0A<Epmcqhjs?MOYUQi_a*WR#)jVg03J;MuO)>$goQu2>Uk(u z0<7-LOdQ-|qR?t&1$+yR!uKX;_f1FB$OXVYiIY=J{dy&6c3zL}dgIX=zDHM$jGLCF zc94E+<>GYm$2Xv#GDmZpH%1e);ZDHofJFhI2KW1fS6Ue{jq5veu+8r3kz~gwx&VCN z;5rz)5t;vA{Fv9w(~M^HP?Bhwl6up?j_>JKqnEwz!en4oCvxx6+&QHW`D@!Q`d)n! zhtCy10%^tc9QAun|Lcx`QcBi2Cl;)Uj~3zh;Z|hFmwA$9pgcy0-+-5M-I1qeuPfL5 zQG4Uh{1|+2xTrmf5@+joDuJAj!nD7W?u#qEyEnRjx%l|?>WwQzF+b?|^Wh5H=SOth z>GU9stPTqgo$P9J`#UqcV<Wnzsa_4l8IR*vaHxY)MEiHeo}|)%5rfx>Ikny&{+?~= zbCHlDS{;e-f9lHv0(1|(N>DTHm-W^b$e}~CJCR-Y`C!WfeAJCWg#_b#w3;`5eVR;- zNTpD`{T&;b53DE4_ihS)hHU;W9dtJzUcuWv8Tme@{RghG9CmMyG@reS7;AV!nNa`w zVC2F7%D0p#!04c5)kP*g`aZx<OuM7k$V~qYegw<}0t)Fyd{uOr4qkQRB}T=u^TV~- zFLr_u`Vk&KgY0ojZLG%nCa$6Yq0zm2gA>}9o+|Qn9pWsD;pUEmVYWZAqwN7V4ppm< z-HJ{YhEXDWH)5Sg8{Tqa0Tp?-YxT)z?OA8+8fybqM**YKiU33%1Hi+lt@C9#eS8zq zj-TKRM7lI3o4P*wygO(W<eWPEQS1Ul($*m(xWJrByKoxxZBb#Mj*a`?r3EvDE40|o zS^nPT8=PKMLj3*aC&Iy(*Ns<x*JwnJk3I@@PjwhfTFmi|_`QFh<a8YL{O<maGne5m z-NL`*u_aRjJ)%q@Y%EXU9=QA<Y)HNCbv{eu@5<54(%QTp^W7;O^IaL!UvuF`Z^UmM z_Boq)O)W!23Rvx>>?6$uc&og22BE1lF#_@ps#AXxS~s7Z-!@<WVxaUw{&DWAL$htQ zj7d1}#0<|t(`J`!qSr{{tiaeOPg>7@N%F^VT!qus1!>b6`7!5ghzIV>c1_bx>0QX` zOUI@gKLST;GJi%N1TDp^rZ#6-CiL-7jZ)_YF;!0?E?bNE!^+Yf=SUm>eH<#zT@jfK zAA7XzUN$xM=!j|)zK9oIHn!it2;7M73%x1j<TVw3A}yf%ISS~*qp1F7qx+z}3boB* z7I@6t_63t5h@M|NyHw4&xz{+U_Y(IjEyLFV@Ek!>t5~jH{UD(@;Ob?qG2c-1rZEU3 z8WRh(AtL8{3ZZ^W8KV7xO~)#iyFz%B8S`MyBvjYIYsz0v8z@J_H;TD+lw`x@_zeDz z#PtAbxwfCgTJ}1Ap545RsxniRQA$YNwI$EqgZx-9_Jhk23EX&}ch(g{HT!q+R_n&p zjB@b?*Qa~*-emOd-)_1rJ&Ahvlbi2=#pb`$NYFrMN=HyfWsMn`3RDMJNPth~Ah#3z zfO*bX8tsFZ#5!O<Gvp<vcpYQL$W<Ko1)TZ(Nx1I}@gJ|PIyOZ<A;76gi+Ksu9>QNV z*1CUaEV;)ANg!ZOqm}dCVJ~+=R=0nUx>KWQ0?M>wZ2s?s$$0Lxg<idAN52lp7+k-@ zz69!doUL#XNotrqdH%UTu8!q6;oTVJUhD*ObPol>Uu6#vt*_A_y`A4_J<YKWG}>DG zC{@oRyEQ1hKShOV<gaP$=145Is~v<wEmkvZc|rWL-?Q5rKORk>8DV1sWm-`2{<QIi z+sk<vZ)+#Nygj7@A31usgDC%{OPA=5S8;YpbCq1GYnn=TF$Pzyn)lz@9(SlH9mQo+ zb4Gkao>+}PC$ZvO|5hUD@W4H^6Yso(rlLY#xTo&Pk8C7yMHV_H8Uo)gTd&q4D|?M0 zm3BTa;$}KCpo_3R;sB{H!Nq>Hk$Ypz1$ur=?4Dc55JXUk0R>7~)dR;h0xsh`<?MGX zO6YR{=&#O7*w6ceP%?L;c44r8V9q#`FrA6yGM0#$Ac^t#me;94=v_6`+;#e`oPg=y zxYbOWQt0fy|4u}pgcRYk;~wVKo`Dc$|3s0vq*mVE6m99mzQU{;vI=N}ZPv6E`5#X= zGM|1dp|;ajrAv+|>gB7!mQKq=HLs&b_QX-o5LAdigI}o9X`<|Q(ErdMy<L7xwU``w zx_>Y52YbPDR@Hj9*?M$`C)rAGGT`kmfvJ*z|H3j@gfs-RP~-i3vRU0fqvP4e#>eMR zSEG9Z>{p|?PK%ePf7GqD-K@?>WFIykef#y-r6AgYlnMY;iaoDC{=QtzsL9KSq(RCX zqa#~<;!0bT&<#=zzh3A;E+Yc)$T8zhg^%u&0Z~u9)fo2}oEt={PD*eQcSJ*Cj`cFl zN=&wcGt*jZtG(<h&#cX~$7i3y^F8C8#um*Ue^t2{WHMzMYF1zl3AvW8S#e9Qremmv z{YeXEtM^NJUUay2e|b^fs*mJZ5SW>xj5*!4!ZDk}!{iNfENGzpvpVI(ol|TR9$R9f zZ_=^J)fwY-RORH!dm*mqJ)zGCpf^Ou5qySUAM)r3UGM)4cjlZRcV(==N#`TB3~rR= zzTDHG*W-KKN1K$U+fNry(bQOxP1~BDfb$*Nt4NcJjDXVbIDymR`1;iV>WiXuf>svG z$?vpp@64;GY!9D@gqrQ96s-m%Y!5pOm1ek>LF4PvD&U#$`J9S$rb1ZDrLkFQj$7_W zaY}OmDThm&BS|++lj@}U0PPvcFE1UjLruKR4ej2^e<AfMCo!``avM5}pV2ZGGymvz zOEkGip2OT!ZWmFNqAoex`P=Elqe>&W<+SdZ!}~fNcYjlZL+gxYM#kj(SlL6VwvyBn z`N&)*@gpXd{+kAN)v*Q}oIcMJUuLt(jVxq0jIim=<f4Du*yuIsU-<Snlc2I;?g$eH zn6_6d&i0;h%~fi1>NKHyM(Z-SXLY!7uEdW!X4JO>oh9IGEi!p2E<}*JMW3Lt<YnIv zcY1q>=T&*YKnAD%+JMRF@T_R<S#hbke#NZM#=xjE7DIyn^k0)o);djnFdsPkHOAjd z$;SFa^N0Kzx_sxU3>coOXg7hPCMI(EMczIgY#Xgw4$!_~>}T_)sQmh&unG6!6B|E| zkRt}%!e>b*w5RX;J=1@Rk%U#?T;)ukVI=)I8!G7d3E4x*(vi5PAtgji=gHgCjjc8O zI+{tnn);-=5PUp~OOQyKR--;1EmY507D%mXJtbd6^L-1*k*|<F)=m%mRD$31*h}+S zCXTma59cl+U+FTlggyvW4g9+me;}aw!+SqVGuxETCgJmm*V*V$;&kU$@@|hPF?sXR zHf1+4$NZPg*ldljk!Yd*BJ7%BVQ>VeK0@IE;UE3QrfWCvDDTa%k8K_>D;jD3S#M(c zdpQTJv-a<wfh6tK1A_Tg6qCh#yy{cVry1}ukFi-pA79JO!QH3B%?$L?)6jMo?;Cc= zq(27Ru|KcgS0Si$hYr%MEfY1a53M4BUIz!wmQ?mW+a5s6ust2))cP6h`?(KNYw!u6 z{9YCr?O5YfVj`BHLzfM@XWSit@jmZ2>Ubwhd*#qAWBUqsy)+5Xw2|6S7e1v?cRhVQ zE9Bp5nZkglTXCaET)v#fUE)WlElp>2Co*p_!9=OReEY|mqtV?15o%LxN#mOs0qU>n zd444;J>=c<Z0a+0=7%c7BTr01l@L{=A_VS%!xbCc8wG;c-WOs4czA{j6M852e`;4^ zs#Z_V=?I%p_Cw!@j~uw4`x3CZ@nzUFmIU2<N^0Mg*o#{YBx_mexqCZ=&;Qb-eb2(t zV<w=4Or+9oL@M<6+hN!r$zejZ&dMGwf(R$Je=l?+6!tan0!kr#q#RAz*I_Q@<j`%j zQZU8{_G4&KX|;9>I!c=-6-D*MFivW|v9*Tq^AzuOvI{|BIf95dwh+<xL|bSsY-joq zx-)}Uf*McV305JZC;HE;^~vWv#$gl*YK12PGu*t}xIfo-{IA^Vj$XiKB<}BL<T6vs z{Zwf2eQVfhTW`WVBn1CuYnBs1Ah)@x7EJtseI)iCi3sw2%>A>xKK{Cy<EYOgDa_^5 z7u?@oYuGlS5H{Yb0(B_wo#acx$dGdQ2CXy=q{K8Oh$~?zYp{X+_NZ_j>57tfvo*Bq z2rjrR)`;^S7WS<CuzWQ2*ZW9JG!>v%{H4+fQF3IVt+bot7n)$OlJK#44xh?swL2UE zNnv|O|5|80d$bb%V+Ar%8B(F6T|N5bvGsg>h{(kdVCdrnosI98$BH#*Q-iQ(YH#dB zNWG{&^hfmegAZLKh5u*?vreP*=+Y|BVRep|7Rs0CeldM!`sTNTJJKCeZ*PT}QGORm z16r^}Vm@0sAH<$n2STF>t%j!ZfbO!r0?T>cB3{nJGy-2Zh!<2N`E=;a_0Q5cNoKhN z>N`QhtF)QvNhMQjz$vdFs*Byv`6des?>ujh*bpA^Hr_gdB^2eHVVio{gO%wdqk)7$ z&hYii&@?Yx8<2!^o$&wxIyozdcqP``3C#Ud_;*RnvqK*^8-l37&$69{3U!W!TiCqW zeA$G9ce9ss&9SxV-6B6h>RIN`Oho>6xp92+iVjV13g<QBg`Bvv|DzY&=ERb}=3vC$ zi&NpqG*@i=o)XfuYIS$T=$Cq{<jx`!V02gv?fcV2SF;ZyLi&W3Oay^PY<2EOZ&-0J znUKFBrs{7uzD%T!J}<ouJlV#Ct51jfv=$N^y!$*HWTYE}%vmhrZodic{Zc5((VC;O zIEav>VArv?rI_x&Fg+z&tut}cQv@c=gUv5giUwE>V^guu>fR<dF+X-K5Pen7;_>|z zaQ7yVBOO#=OldhmqbX9=MFvgn-c<k-Xkk|JPKIv#6%OqkW+NucVbHmLu$U<-wS(>4 zW!VWq;KYn|!Vew0FKy}zZJO=>Og(<V>gbaI=}c-!h*X^W8bl!Om-<2=EzQW}FSBl= z;=*UV`t@`kL_-!*;mI0dF=4Z1NuH9o7jsu#$3n$x_N}9Pu*^=+bEzw7Yx}|nW{;|E ztY)9x)ipW!+PB_$+v4mwfBGljqUUGBLM7%lCSf7~H5+oW3D(xII>y?s5yklr6K!^* z4RA&Y(->FKoJoTjaWGK8<3YZY`HPTNLs9Kckj{}yp%<on36uE2pvLs{@QhrrOJu@< zNmBb)Qy(Qjvb4Q>;4EAYE&cf|2LRoRUVN>;02x~;2lkuHL(VCKvj<C(d04yQbLSNW zlXGcL&bJ91DTy6EpG1-tl!HL*>WTR|>j#;XV-Cz{m+jBjoqP+6=v)6n!osHb5FGjy zmsSTf8nd!DZmx4kC36Yi8<Oy(FR9pgq(&cZCt%R(9djeen&9$dM+399=U3J?4FY7! z=K0gSUY+*ausq!Fs+HlNG8JvQJB(t43ao5r>12Ffe|##;r<Hm$DJrQWxg=rWH1u8Z z7W$3vp5dOW<NFZh?vR^QqQRJ&8?trpr?0<gUZY6MAA(=~dll^~B(~QJe-HuGhNH9E z2u#%Z&PT$oJxLueR#WbvDS4Pfr?3i(>>?GP+3Z8p{OR)9ILd6acjwEn<>=Bng`h_p zP>b0l@YXAu+sc5H{??%2tf$4CIa>P9H-GKne03G{V&CO$%=#N<$1nL~1Bc5^Aq<*5 zEb^|;6XWAJr}EW_PubGr%Ph6D)+S$1=rChW<%YNVTiM#GR|2F7gKIPK^`Ra6<g+97 zR@quLe~$;GhHY0#uL;6i3KMVb+iiVeYf?r;Xd!M{n*R1Y*^Ga)E3-IkM?@Oz^EZ$< zt(!d^xUJt$e-R?J_2NhuK3J%}$kkkTSZ}$)*dT*wt@PZ^-V>h1(y$2Bl|juBsSrP) zQ5hXF+8NYAANu;R_)m5=ZC$9zn6fs%`QgL(>P!6u8#%u0?K|wm0mSkD7?EY`$-wZr z2%^8et5s}IU><Mws-{dpF3K;*_kasY!al?e55b_7qQv%Z@_R9<y?MgJ^1(Nfv}xm) zPujy^#RI9(A2+3mUG5t4{req*ip>DkJW+eEM)wunwT?_J_nYJR<4#)|!JM{{sz{;( z41_ZjYh`y0j<?C1@e#yap#p|0;zZm##Q7m+j|5%(IuC8PRwU^$7?`XK2vEmD_v?sI zb)%VLB(ZLIlD{0iqeDxZ+$j(JYp+Co5rm_zv8x%1-gDi<8Mhw`uO(?P$`a(jTlZvq zS!uUlu#sk!gk;YV5$MKCu0alO%Q?}ZB@47biV?W|jLkT2h^3bP-SrTU9MtXyq%o!( zFBYoHFXzB({z!lCL^Ch8Qn5}UAluL4{Lu`V?2^Gs{q8&8{$8>eHnJq}wmQN8Br%ZG zN@5U0fqDGdn4F6eA=>p`t+|Z>e+X@jN{1oztM<JLvH0||B);Azx!2(?lzk{z+G&X% zzN`n(Xwci-mZdqDuQdp-#l{+2tS=%RR>E3haZ^d`+49d3&Fces4S5MiV(9C=jobB? zoBl$8Ebhe*4#@v`iNG>J3obx0Wp=ao0`yr%6;iO196lp7-;+HI^3?r4m=W!S&^7ln z6F1SXRZ6}5?XWv4pm}wCD8-|r=vz^AGy<@I#f=i9Zzr+`gazB~-N+5kb=Sa#V<}?$ zlKjfJm_Pt6Ydqy*OZm%0P_D)oydg|V9+ap%ZvNP0Y4#7cSN5>uebdX>mr1+qsRSdG z5Tnz5D*jHq&fTAXM9?6AKrtdp0d)EM({ZK?rrxGn?Oj5#w<^@d4CSVHO=LObd0fxD zpRFo-e;7C!<es|1^nS@)`BfMbPb1v2w+tyi@i)+M%n(-Q@kYd)NrUn%gDFM)y>A1v zKxp~SQgOLADkC#cRCMa$E#q5v-X}Fq-DKgqA@fcu3+sFxe?^wwFY(;;>I0}r{_Ie- z<*E6c2hhNkX-Ms^y2OB|7jY)oR9QKF<B1JXk$G7=Ogn$9+w`QLSqo<qV6nb};aYC1 z`X|pfTU~dck<y~xS+3Ah$Q`ty!(nwyV>Z>{xXZA@OQY^mT+QwhW$WfNn|(w@f7z7@ zJ(8|*Wi5Gf^<O7mzm;1lnMF3*%NftTQkA|*R%eerz}g9WfP~>G?2{Wa+WwMfm{If@ z^XwdHcFEMAs5#0CB`Db6f^s6w%AsA|wbuDHAEPaW{0twKTA-Bm8CP9Nj}6++u+JT| zCSFv&(B@-q+dR_z5~O4K`~??dBL<h=1~Z4nu1WJ~`pa&RT+lQatL>kGjBoO>A7Qhz z-aYio3paJfV<aRvypVILXNB6inp(KqWjFs-Hud&ywJ*E+p>G?mo{CE={RUSuOPF#$ zz9;g5qTX);@q%RkrwPyLY~|G@<d9$QODI%qC1}m&E26&3MBX)Z+j?5YrpA5z<e0yb zb}cETlo6N3y*j`9aVhSA7#F_DW8Kl=eSx*qqx-V>+P6NoKUGi88**$3Xw91IK;9~e z9S>#)L$fu15EBFdb!i`Kum2H})zN^=cjknOxFcqC=qQ)i&_s<IA!3x4ozDaUhe@K{ z;*JYwzp@7+zH?MB0BrIx5{tbexjdrvz990L<F?JeZv|FkfLrOdZ5YDDcMW(Jwa2j+ z_vFqv_(F=q*xuQHCjQ2`Rd?L|p0%ew>KeTd14gjS6EwDN9FbJUB*J}TkA!fx<NV;w zzj<6dkk|jHRxsBGhP#JCMRioA_Dp6I1Gn-~hU$i{+JyqLZU^8|Jx7l=gnBRk!W+>P z%ieM#f%rXtkAD=zc|3TZB@y}tWsW`+0JO^>Q{kHc^n3Z$1+eIF|0^8o{`qFF1g?$K zgrmd3=#%ap4&tCCF|Psq3%!PR@uu0IcUd3hA6|eYTwx8vmP3>H>^ITLT$@ji7wjFC zH5rPu(Aa|ays7v8k7#oG-XiqV>A^KUET7{y8C}gXfRDNuPQfYR2j7We^YP|fQ-b3! zpkSs-hlWipR`I~^;MQ?W&!%M)R`KPRpBno(b^4#?YO;X+mU04mBfS`R38_rCcHql* zFs8VE$lbQq28EnJA(F)kS=l48C{J#;sJM_EwtIN}*G`H*`pu8dX!<IR3;^va;ff^# zMl4HS@w)BTY0b;2D-m2N^7L<{&;to7H^q0-_vYH<RkU^zqGA9@=`MAwO#B}z;!3rZ z=MVoFaaVS>>Dw4HZsMf`YzS>kiitHQ?la&`&KXYXT6A8_Q)32rfL9G)QSw~lLLSjE z1djP$44eoQgnbm@i_TG1d1P?v)}7<)@1+B6wmh$Dg;Bz<2)jG^S?ccm8bzH{l@n+F zV%i(cZQ2<W6iI~<UtZR@8c7a2zKLs#Nx}L39qjq@NwJ)MSNE8}iCnLE*n@M2po>Y= zW#pe6PZ+AQ%A>fCzyF-G1ASbWHabR0#AoR9Q>F7Sw$<cb$X-`f`zRI|Am>>1aRlOj zKvh(=|CFTT8Vn9#Cq5N$xP;N)dZs&o-(6$3*)4MvWGo_0LsgR~b^_g)&geXm`R+!A zys({O8c~N5ADtI^#T)N#H(qm0Lrw1lzSVE`BYwb*{DHyB=~mCGlhoAV6IHVVXNN-P z{+1JV0_9>&+4C-rF)ETkce57ca8qMuF{4Q&9{6}d7DMXF8;3xSdQj7efx+~N3vC)> zdfZy*M{*%62Y1i@3{h?@mu@x=p_d1(uGUWXexSX|{nTD0;YNrm&WpEcK3s(~BX&;0 z)x8^hO>@+je_cL#g{umCmS*Yn&wt(vAvPKV)RsCdvLuQ<SBWhxn|;R~ZVCZ>@2!Kp zfAFsn^nEtU5DOFA&;6;<A_><$?`br1Q@KY8(93%jD}+7AYasT%>RmVoOL3%wbgzkR z(0+OgHZ%qFd^~8_G-0}iQ}wHKr+6&Gt8zZ&9`?Lx|0m0=ghB8N{t0c?fqRj6WB<EV zw=+@QVD=i1w*yh4?zY&Yt0pG%yOmhcto?_>w5>ja=9E{2yq6;{&T!6j$(DOzwRzup zwIDi+Ew}|0?KeK+LGi&T{4Cb|Z2-sy^zeI%{U<E`_M4g`Nf(elC>QiCLS^<EN0EW7 zkmLt458ZUz<a>9X9^SObb;jiVeW)eY?&?p(v2gy$&_L7e2Es>e6t-k5yt&AhckC+A zG&q)#ZxHOmh0j*=d<3I10M88XC)f5}P)f(ct3W5O2RhirC6ggsh^ci`d-Mh<MNZJn z3Jdqbo==DZ3t2bc7Ch=r(_eX!0J>L|OMTOS)oHiQI9Z7%eIlvlS!VU^VRo;1!6(F= z**|B1D;hSb8`%uFie`E}Q<&=)*zH78ae3`t52ZL3q==d~RPq@<8^sqH*j1>J#^5!a zX8J1k7c>O1tinBM&p3ML;;!zx>9)jjNzQ*St#yBIuFtR$9WbJ|T619%ywS%hHjY;| z=kTa|WWKEU{H&J`oUV=`wULRpOD?xd`~GyQ&S;Tq>J$hUtl`lD#awNbCss$^4U6M( z#e^Tv{`tbYL!WUmxEhUNoUe{yX5*Vr4Q<P6udtuA)Ql)J^LU0X+gT`|jQ>e$X4dPt zvy$>$0CrgFaWOUW`kdm<LPBm#81HaW$bli=^)*=O#YFL7I>*RNwj=(vUhJ#7TgKf| z!l%-;rr`%yN#k|C!a-Zo`cg=^-0XV<(mLlJ!bnz#Haz)58~$Q7=5)?(H5)MUQ7QJ) z_8qZ9)xAw5uET*vm&fxUrPk26?v)nhBJ5ZV>_M>|5ztY~$jshg$e%8{|M|8gLTF|o zb6?qv7X+MUKAr5J!SC2JqQ_aOL((Jt29&TGf#N>>qZMNMQF|!*%5IA1i($X?EkH=2 z^yYhkLz8A{>*3k!mDR_cBYHe`$(EDA7%k$hj}Y-`dnRLg<rDa?GZ$C@a{{69g05gU zjwP(M!Nzy<eMZCexk<lcST|j9e^zUYE&H7fsZ*%gYrui{=zu%9>o`MplJIc8y)888 z6+l>^E>3HP>u8E|&RLzP@k^j1er5mr!OgI}J1g=|=l+fE!&&=05nepv@YoV)T7^V8 z6-!8cE{7v~_X!eo>hDG_gO=<gnQ%v#?tjk0LIYg%%EnKF->~HsN~lc0K=^28z|K=| zXT(y``R*ekSpPT^Q=73ciWKm;D7*NotUKkHM5dV{)@t?>e(2Mt9*5Ph-y`YWsQU#7 zho*AeI_$~tjrR{_X=FN{D_5t92KSXNbao)7CjWEeMp|OkM3?&Ls=`!GzqIOn+1ooD z9<#z;H^*k!3}%|W{^+@tu>^qU&}q#(*?qn>e5)1qhXY9m(Z}^ltXgZjr+t}*bE7{n z%R83qUMXJtUu&O@AMzrOIup2RjY00pv|(G?v;+F+cjkycOX6{))kMuyEah)N8T3v~ z{{PC-iR;Nlgn^1-VOm04KpsO`2@Z3p+=#3_tUjW~*{{V5uR$gwp0?=Of9P#~f^-NN zY{a?#_N5Ux5yae|`3XUdFq@qv3Tk*0wNj@+?dF^(sq0w9PD|KRctoHFH~LX#3Ax{# zDtR|{%EL6u7^g1TcbyRer+_CV*a?t~^u%mRgXFCY*0E$4lD{E!xcM}m)=~&2y)zaC z>g+99YENxs;-`{O(>;MRNs_X#$_ao&4SnHgY6Meb)hPd81<0nAtW+J4ealIm?oL7y z0ovjLqs!ki)%NO9_0pJj*}Q~10DXRpnjtlT9I$Rg-B+0>94`yBe{pvrFZ<Srh2x38 zH7KIxVLM~U--+=v1yn1H?;Hf!*K`=5OPd+hT=~8Q>+Z#GFImM4uiK7BpfluZEA2jL zuJv0}>&MpO?R?9U90Mt`0i!{~n(n>FsO=FOSZ9~}OlE4|s$-xtYmWXYad_s<)?E7} zRf^P$w~AA7>sel$Pdpo;E6}eQx>vil%Rci#qgl*1<k)hhlO;S_G{{RX%*%3ceVQT9 zKPBajM5OdJp9FL_((=eh5me}QFBOVsC{_EPANSb)<-5jU@Gf2Ada=LqI)Z57EZ)i7 z`NE<|dp@&SY*R7>M8E@f;G0t`E+Xg7obOu4ELxvpZ;4gt`LSAoqCjPvM`2j)CDVch zF!a)mm~fJ=M-8y7O8(uz8KFzDoTgm#FA@Q!Gc`7<)uy~g(4n>_wqa*plqpf~V!?p@ zl%KN$R$*cm`a(7-C9x>>rG@vr8L{kx^sxDPGdJdm4S@4I^DvX`Z_M?><%B6ElE`2~ zm3-FqR#4=Er7tx-G*vgXA;XKUYoMKBC4Ng#WV(CwRGQbG7w=bQz@f0y!{-ORQ|S7J z&GJPrZxx5Cne7~ZD3ODyK+#ljD!(2(%=>aBy0QNqA(@c2E`sBV&5rb|Rw}qZ^IP3* z#prZDLlyh^=WjMtYQdX*muT$GBxWPfNrJ$>xJ%zIP4*@z#iZ$^0uUgxXq>Uq==7`D zvaFL$Q*b%jF<A>DV*c${=7v<H!p*tS@5L5zsdd#LTYB%7D@hE51I_I}C_DL4h#qk) z<97C~W2u1SM;F;u-Joxe$4*F6BB|Yf1oFnh`X82Ew(u#X%y2>^&*3{CuVS0ot~gzh zg;^cIV>bE$9;#<Uu|T}529Yp#U{Uts8J~WMH)vQLvsA$owjo_F&hQH7eebF1e4Seb zTYtU3&aN7ldMueb<z>J5z$xp-#Jy}&;3bdQ%FqZzj3Csm%!5CphrNY)XQPdggvWfu zyE`YY5AF@kJpByw=Xl=Y@CV=l@>EKcf0n%}T>Kt^W5np!k}}9~wi!$zc=-s{mTudU zU+Vlw^O7NT!TBWw?1KSQOw)1iB}J%;u#fzh<&?Eg_zgpe>|wC5iF*FrKQj9N$ujDu zA+)@lh4}7z!@`5~9GY8Ffs>@oE!jCPZQj!%KF@=gSxt0;hRxqpc}J%E-8!f%HU(@> z7gfj=Nc^8;qq2PlSiZ%Z3--t_?T=Q9QDdBs5Q+TAN2)WeTE4HeQbZP7!<deS!(XHv zQhir`Y94T%-m@`))0g9u+L4FmBU|=cS9Q#8Ts)n1wU{M1jT<rZb;joS?pzf8rbkMs zqdY5C;JW0Y%I*6YZ9kNgek?(+TGvWw%Q4%9rd;muK6tI}V-GtXl&wqQ5WXKBHV}y> z*VTcXCY7yxb?mUnvKvF(PV>^+xm$erb{!|?&6Gl|H{%h?Ylv?AagIU{$uC9GGo`LJ zWz00beWj60btywWMz#!wS+VK9p%BayKBR<(Nr&ryMeK1L_u6)|&J-gzLU#N3HzES5 z0`^Q9hnKtKhz;OR!qqRqp{_YU7+Y|}?@m4JmQdW{3n$9z#Z<bs1Z*2gmwk{8tAAwi zH#+o9UsY^k440wd<^^!Wn{-7s4|a?zcy0zX;Yqr@@F1smx#woxFN$6~*$n6vJGqF3 z$KYpdnlfM4>}oiV&1OaBrs6L{Esia7csWNEK~mG4>wXH}u^Pv=rz`DMUQuNtu;E4a z?db;c&C>&mG*>TIafKzPZeRFX^;8}UrcPPiY15A@mnKb>7Qp4_be2%a{~H$5x5$?- z;OX!%@1EGj;Kdw70lJ%h-F0)-n#(huGc(2B@M_B{$@y0{<02~a4#Msba)@}Hr=4!? zGfO$!eZ%s}Dy3!YnMNOY_X<5Xbf2N4WXdcz0m-YLtWU_9kj_)QA(i73VfG6<p@QTT z^T{b#mRobJgbXF<HwK3gCXax?VFd?XaO3eK2i(-3%QXG|{i^koP>W{N%>;7T<MU$y zePk$!hHT-9_^yUq8X1P>M_I-6Z0STFS`;8#;k22IW}aatdqjguUI=})b~8N$M&U4S zC&-XAVBt&cRmeZDwuDDi4XG}lNOKZWq}{?@j0}74Lg(z)vln&4*M8$r?bftE8fYp0 z@oG(Ot?8As9-$S1^e}*dsMGjz!R6{Uh>}-j?#%n2h$TkKx<d#l3g=#)Us?kc2w4ep z&e5BH`>}LOq4B!6)A93?pq*9J0&RYVFsSI=RKEYG2Cbg1d+4{KyjV*Poh4tc{tdzP zp-CDMxU|%H#{i@Az$P2H<;CTlnr6P5@xABWd4XX$A)ODoJ|Fh;DH6vh2X(a`lm57H zaG5zUME{Y&(X*W3IZ50|z!C8q4-B^7@ourgEbS)4Sitx~&)y6TF#3<Nx_x2>k!rar zBXKZYODL#TpWXAG(N#6yMP2JnixXRrbZ&el_eDHQJSoN$3pz|q=KDn82kbK$00!e+ z-+4;PgIY#)&$FqT2`!NPvD{)Ek_MnxAQ_O`XzbYr^J_uL8P^urZ+piuI0>@_6DIQq z=9%9qUAx1YA`&6~kvx7^r;8NZcMcpS>)+4&_G-;X{r2b8<3(ZXINW|U&;Cp?78_y8 z<c)Eavy-N!cS^@{$12ySIKL@L8~O!7gDtQzT+{J-rf2Q$O9;xxX3E|MoowV!iD^mp zU%M&s^l!tmWYI{%)lS%7GZxkDt3h&+>Ei$3e|-W0#x7r5)V3|)G08TRuum?zYV1Gq zj*wgQ)wK%r!8I!l;Hr4QaW4%e0dAO!GlrXQ;AC~B$T%E)lgtMJJzy+<#fvSpLrRxY z8Z3YTR=+NISnP6uuKRiBU^B3f9q(C)u3m1gZBf`7iw-4a?zE#tGGf%xq_I`TrIKp7 z-Z~1AC;nM{*(aRuSyc*r1;1>rqt#JO;BI+!TN&>y-OJOsxfN=6)8|IbGm-PPV~oQc zd`ttt<8A!@Dd>HuJ2C4N@<HA1L*gbYhm-bpGj#<EHzz!^2bNs|w?`(@5)YXZM>C$E zJ>M^#f0pfYbYPQu^7{8!5q$>jsr9z_ZK55|R%cWwp~OKA8%n*e?1+8HlDS4Wr(`Nh zuaxlli(pJ%-2S-20nifo78dJ{mdW}lkeJ^ac^4{l`V7OE2na|71oNdP+-dlU*uXv1 zja$XGjy@3}_96DuFPnYBj@lVJWo1$>_md$5F~^sv%!vmT1VdS&9zl4)fF?nWrgZZR zvPJE9EJuDV{s~meZZ5s$QEbYktl6()4Ab@xjm0k1LJYvof_J#%rAWk2*Qa6w{3nT> zB5RZFnjNu@*N7>_8Ti?HhdWQv_b|YxPB|{D9H@;my0PVTEM1`s`tRW8#5<7Qz(DbS z?vw7e!zX<$>^?mLrDPEuJ}UXr35^8o<adhj-&2m-X?w?#=#?$knX~Thb%T4p>Vu|L z^Y&CzTpszAriYblc<5ViGK+Ig{+v0}+Rc6T@F%IGF6G4BfHI)c%aV=uyH$mmeS8ZK zYq+o#H}yY@>RaR`%|H48jTFe<HdoyLxaZ)XW#}y6;fJ7#UA2kTQc!mdNBsperiV6L zGxpmJx@p92^Z~1Z5_fSQL4hjGP9GEo@@zGjJkb~Vr@v#jyC=eihAsd0aPL7Km#Qe< zO78x!1JtSz3{(E(fe~^1iRnxh#s5JqDBJJ^#3p9;jcD0lkY4DIv-Yi~&bX{KwC-++ zy|P+}lTXnBf&lXqr#R-&n)Q70n*g$Sc88Wn!I88tr=FyV_xUwyM~u^9vrhQmMw!Wd zgBC>?3s8+8c{_jZB0N<C$WJ*&taw$001S7J$BvCN<~XO`pX|S{CfNIT^Khsha=jt& zi93p(#uu-=<~~ep{?D`S5ynexpXs~UugNfs!)R?_lAbv*J_?h`qqB~7UJ2}wK@O<Z zv`_pL!#p*%!Z-#Pd=CtLoZ~<65V@rXVD$4niMe~angJv88{{Vx3FxOu3Vc^nTzz>L zmjZn<{Y_N5{Tol}ALb+{k>Tw<SOGHFLT97H?e|QVJ0y1CNK`mGapm`fOQ@H@ZcHtz z^*!qFZw<6nKCp~c$VN}piN4#{{!T?9sWgNtR81wEH4)KR+^=13eMlQ$c8ZTm%H^?t z{9zd2;c%2#yUXb33#h#@#(BXqYat7<nhP)OkbdQjm%t4d0_7k)_*Jzp2k9=A{!6Wc zu(1j5I2ahO$v=r#ex!HwO!s3+kQxL<Uq#L14S>V)4F=qLNH<nh{gWRV)ZRP`jWPen zTcsT?*@>}RijMsyyj4Da`BBXmLo9TzFbvp+hz0|1&z@qZ#9_*bx9;}-7FT~Qi41!y z4_$XWK_x8E^h-4IT<jAQC!|G0GbnyGl~0y~GA1s!MPo2&;6@S8<`X)qTKdz&zNTN_ zpUVE3ha5c%J@BL?)@>nT=x|Q_;VZ~Q#!QiQA7<Kq-H!64WV18<|0{)f!TpLmo4rQ0 zgh*d8v?QA(lX8}n;PksdBgCkVf|dksX4--GYvZWlzb#S7qY&pfecgWZ^Fa%}8YPj= zVkMG~By$|3zYLvGJIf-J$aCXO>@x=KCDr^p?Q*i*uk@=QnI$DnBu(lI+KC<v!FiS& z4Z6Q@XXZtYqdCC!P7*JttGL#Sz}T-A4s*}$6tjXpEZEJMVI}`wZN0O{E9`2NK09$% zNKzRT0Me1Hk^DEJ5d~u({Rlqw1kYHE8EVG*Fuk6Qdl_4lQqR$AdeyA{FTbnIr`@=_ zx+#%_te57!54WGTr-0JgFfW@3sk1_P9Y2w);k<^0McGQfE@eZ(2Xs|x>MU~Q1*d`s zuK{m9N9NpD$3tcAPPk87G<p#{(y#m0P(BYdRqrgc@xzw)O~*@Q*^8ew^qpvp(Eg|* zVBHhSi8lC?YdIhk8-?uW5dC{yTK&s69-pG(0EdDov|%C2kem0dm*QW92rmK3-bDDv zba93Ko=fN#`_;u^HP7KP_~<if5lLm{^FfJ+qpIG)X3p+LjEw;icVXh3>M68)_6K1D z4pp))J&KD=2E66a3r@W+mBdDxGgA$1%1VA}Z_ht&L2l;Cc6WZv?!V1=oU+>y)I-WV zF{N)#V^Nu>;R=J;Vo{9~e$;t6%9C%%Bz#I1yF|=tl0S)e#5VY~mbDue+$^g`kjf(& zFDl^a_z8H}uaRq+uiuEnTRd()TB|Q@qusJuRg&t@q<D0%msQ{7Q=YAy6ETz!olaeU zmU@bV@4A$5947NQ;;w9M5}O9DeaWZY&R>zb2@G&Q8C1~U&Mqjb{SbyL<x-9FBo^wq z=Jl}Q>&|H!L1xF#3OPmrl}#IQ7{)iJ-_pc%W+wCntmTd`vMH2c)%|pI9_Ffu{yKP8 zXfcxeXacjps)-j*P2fx#5WS#o^Hht(p7_e4UG7k0JuYbVNI+kvH%`lMG^vd-C?-CV zA8=5DoBL{e_#}iwb%~<;84;RNh|D%$EGynq&@y0PV$Fp}{hmrMLgvOtQJC`C&37UG zJ$rNZ)nB>ovO*Tebo)M%zX@P{bleVksq`mXNrD+iY&*W*O2gM&=QTrie_^ZV`|>FZ zIbp6j1Ki<VrG}k+r!zSR{A7u{-*mQcv{;1K{qd>6we^mAV1Ea?$7aT?DQ0St6<F4J zpb~kvI!Uo!*?0riu%QJb=?Kzlj@3W@XiCUhW5zM5GW{V&+|>m=@2V!~`nVSplVPeg zhxwv^<L>Tpmr2Gu3@nH#*#N1(yB^}$zqRu(PG*j)q`<N}evJP}I{1PsVi5RQg9|Yt z`_{r9MSD5T9WUf(elsTaV@{aA`~5{@<QB6n58~$=!hcN~`X-Y-&TCoiB92E{&(6i4 zA2kDl^M8PwHelz8KNQIi4(*#zPm;`-`opg0_hoR7-hMq7e9|}nTjOq6a>|j*NsBto zcn^h4tWCvr*ZTUG(xxln8vm2`>81$+@dmzzREF{{8ow3}A|@HgsgH$bke{qaQA^DE z39AerE*`A=KG3TVGNP1ZK>y*UMjepO5y(d9RWsAofogR!Ml6CMbnm+5?_~HA4cWm_ z3Eh16iTF)%jjyemti!E;vU1kdzJ4=HbF<bO+Dr!|UII4`l^kZHX1B>uJcp83aTu(o zlSE9bQSgo+$|7p%QI`@C;q%eRrB^njbIz01N9qy8qPCBuUl03;)V^jxD!0B?HIr%+ zpvXUxc!nWohz5?xmf=?!6XS~+)L+Z%2?ven66c(-kr(7=#VN@7U%wglHJML@T2^rE z^11`RC*<hy_@ItHz5D0P_rboTSMk9Ac>@>>(oQ7t4xr4NIZ>ySRVY8a&~5<Lk<lAJ zVly!s>P|Z`&Py8X%NzDw&ar*Itk;uX96?NVbGV3K3$krEPD8YJeR7KZLdv;hk6G$; z&Boyvsl%j+pFqq!1vO7Ewcbm-glWD|-mp_z`>fRxIVz|kJo!e@Jzt=*tXAuz2>O|Y z)?tEe{b4(U#gmz5yaT*;@xM>3|GzF&1N+Yz;_jprWrJQvqhfhY&zTy>dYN9)aO0K4 zMu;Vi6~~Rf5iewmd6~^lwfOO@&8T8a?5pGj(0vt`D;$z3FKsw(#RBd?&3TRNc^$;+ z@v=PZEJ8c-H>a*l)Pm9?6tU9Sa$<&aG*Spw_D9$wLHIqRS7WFy0=%B6j0mQ26V$Ja zmpBl_yK*k`<=dChx6kU+tN$_KeVy=qfKd_Z&T^`nZ3;Huj((|NNHo(^rwY+cy%<dX z_Vt@R5RmD@eHK&YG&G-wtNn__4~vS}-2O0hp;B{hr(4HW%DKSYA_cMU(ShNe*KGn# z_xra#o$+|nojNoa4Z;?^2VK;8^8f^_PDkK$C}(<8mF?t_*9X9Jv$Kz@sX|RH3K5aL zI?*<<A)7_^utHhtJ4LvlPMvAG<W%R$fwY##upVo!8SmDm)o1aJDM<cieA5idpkN@A z%#R+)LT|`@!6+)5o<7|husS#MaLliTn;?(8C71HDU7~S=Il#1u_}(gH`A6bJ^Hd?Q zKU204bo<~{^~m>Hsl{wXnPd$yNtg<*H@qX$VDN+XHfKvG{4ZJ4fcaj(aFgVv#n#%+ z7mCkxm?IQ+ScP_E)`DE3D4xQUnHRIFc+2FcMz^_vXFS+ICNH=4*f4ZS0oVnN2n$B< zPGJ*@<iV1dm^lJ`d>eQ7Kc}g(GAH`IcZ8>Kj#LOJN!cvm^iE8pk25Ulo3adH-h!p_ zhdsD&>^_3$JXG}!3t>d5YL>Na8B;JKLUin#x5pjWi!(EV=<c++Gs@zyfS(GvFyA>2 z-zZN<Shf+3cOA|Uem-)2Q$yZ1n6rXcOfZ&=YoJNxza_&rl>s=(O?P|7JscK>$34s3 zn_biGt>WrJv3+l)M-3%%Ya;KW-vj7x6@C3(v&pGA8V|LA4?Qb{qlCQ(AT#5oSt|mZ z2z%Ypu5T*}45*)^VHYb$gBkA>H0*GYZYXi%MOnL3Lo6O@)?_i&9(x=x_S)psh*>D5 z)+D3uQYo&VHRQ}tmS<-)c*Z=JQ583lz#nUyC+!ZO^rr5=@NzA-4CW7^^Y4vEB?~kc z!1T+KXr8G~{j5$r@s6zGJDaI>FVO9m(CNGI$&WB~v-CRAvH<eKQKRNkYmKw2*viYT z=s_Ew$pH<D+t%oNbLza}O6EANM82Y*`+(NU1<y5W8dVe9TO|H4{x>hQ*H#x3FHqO@ zHmp)@mGRMiE&_epRqbMsqYQ@JYDKra9npL#kyvWEv-L^<9_#cd70Z)V*Vp37k{D{G zhf5Rl9rM=zF_#c^y!qtpJehBzQpSgBPfV%k04w;#BC_$;>`*53czIp<?5AU*Q^l|w zk!_C)DhceVO%eBbLGS`Nt#R!uhCGONV2t?GROK_+L=6ZMe1%;b2HRc?WJz38*=I#> z7smkB@<&Rl>%$nMB9|_OWfIJA`zmKtxhbm=XFVF$-;t>M*1fSD)|l8faHe1e1r;b~ zci<%w)D@(J9OY1Q>6-3I|5`JSznY}Gc6_;dXCD~8yg^?V<*sHLsh*@3XnGyKLjGi+ z2ZqTifk#jOHM|;@qY-#>3*Mo_C@b8W+vLkSw1`^YktNh$SxOw|9Oa_@dZ=?Kvik(N zvM4~p8s{he?rRit@69`+B2>}q0*obm7{WVJ9?w`}9>30N$b%e}^^a$h8yHkRjG!M) zbfY&7)K2Z9o~~+8>+b-F>Gp7ePfe#>9nty0Uq){2GEVB)D6pn%^|;RvCXyH{-QH^4 zmk?s3dx@a{RMoD$eTAFgrb5?Ek4U9^`WKx}4)+*y^!W>{s=ZCy^^B|HHjUMsUNMIa zVAE)B>-C=elR$|85unbxp4Oa0m^tfr2B1+)7=V4;MWN&%C~9u);h%Qouk8_J_frAF zB@N1O3EC^XkhXgy=8f)a+^i=Dc3r{sbTmS1^rT2K?m2w!PH5|u%2OHxSgg<(;X21` z=KmYRj}q5kW+t~a%_g}fL_5UBsq|`6qY%3$k4*?>9jFT#7oJ-a`}Bs)*n&*<oF^<- zzRX{F482uhr2Oy&Ped|PMgXW>B=uYB#!y+b$BrbckK_v~ugVgtfA<D=PI61i`2}{> zB^&6~n?FLghNEg-Ut&9T0Y_r9op*hVZ?!5>#a8}2ab%2v#29ZT=U^i~06Ilubvd3@ zyqQY=e!<tbSTEMeAGEfZ<hAe*h1Z=V8t9P9c16Gxv>Ge{`Ef^QwjdWtC8}}6vkCof zWfy+u*=e%sX^y@7@2lJegPK#UNNY5xcs3<~b{COVlyRHI%IwA1^`C0I$x9+(+XI&W zM)!g;x8-p%K_>i%Sc}fzD%gknZ#uB3chVd))?mFV(}I+F!}`<$>}=Zi)ClZ7Grg3w zqTlz8MSp{qKt(<~FBoRrhTC5>b|pJTJj*_7E*7N<>Ae+@WM@9-dd;OsQMnVb^bM)~ zJkOz{>StwBhY2>xF;!<`v)SSgHd<-mOmcj)$%*L@){bS4cx};m*L7aRewbf$HhUAr z@_N&9^_w0HDm>ctkU8odo6us4l`>C<@xOb1Tpj`dvgPKlv^&|`;5igK@f(~L1*wvp zUW7y&{I6AKSi`Zu)Mi0>-y_>1V}OHPQ`v|oVak8`R;wkuw?e$?L`#t@w?IlG8Eleu zsqS6*=tZ^GYd+ZSB35{O)a-l3U~ohf!qkiVV)4dJh1E2hy7#H@XXfukC?f&rHUzBb zlu@c9?CM!uLC-(Wa$cBG)nd7^A9h&XESM3&1Jx2%v!R){Fx4GVu4h`11N~*~<d_*x zHpu(ZCPSZ)HBJOOp*mI{jH&mbC2J2#Pnfu8@^MqS^3;@O59?o!4l4X79}p+Tf3jL0 zflDqmZ``tiICP|$P2e)yI@oR`<F3pDa4I&0MMk^@$sDv~Hp%+N%7DHNsP;O;o$@Y2 zzeByXDhKSN0M&pdtlsO4a~hO){*lR~aqx>%=l42?bWT6P4AxV}jVYVo8dU?vR~`Do zN13!L6!kyDt64U>>TZd+)zDkWp~x4y?e5j>7&7}r{KQhs7V)LxmPG$;(LDv%j1~;) zeR%_7fkha4Ms&lj0$ebdbB$Z7zvIvED}hYx&O}6j_}o`T{S=!(3r8RHXG2)i!n9;2 zTlB#_?=ADS<8IUj{Di9FSE8sJzqo=aOGsPW%ws;?{=)DQHYG?vHsx2d_#)jGqqi02 zv`DXg<4tt4iWj50MFY7m{L{A*-F=XxejsW0`6uxpEtn%yuo5T8N?~Lzl=NWgbXbK8 zWz|>EhDRLZtc~6CkQ^vxO{24GM~1cOjXhVTEE=(9aqmINFThEn9q@pQrqFM~y-gTS z%pFe{A#b8}>1>8~(7gk!`i|5)|M(C;g}9%Xd>VZcawu-Eh|nauZ?uVtsMOR~Th^)E zEL)Ls4->*<ex7JpmlD9e0@NoVXpZ(X%WaJ*N8#wMM~sNj3xLj!x`uF;NvWw)?jY?y z>rYs>`0~}87p=Y8G6FWdEsn|KY_^y97Gft-Qw~y_>&u^0TJqXovc1cN86*j3AG9}W z+F^Y&wxfYN+jsc=?ixcqiPozX^zj2juMT(R;EN=}mH*@JJ-nLQx_4nc7W730L_vBd zDD5B^dZKhh6BQMZrW65b(rd^;ks`gT0iv{sH0d2wdX4lBN(m5pNPslBJD%Tt?<wE? z1HQ}17%-CT&0cHGHOn*SdS<1z;##@t?yo#KOCnXgWWVgF!o~tMJkfezX3*;AoG#Cu zb-?FtyQnyIkr~jkd+N0NWxs7vRQaRGiZgsWMV1#>3IfGB)T^(YQ{PRyRJk7Vv~JJ^ z_44wOF4?tuB(ED@yQF?}Gf}pl%dW3RCPod{;=9$40dAWq=^A0)&c|!L`)=aMD@O}t z-FPE0#H0ANY&!D6PVbE)(}h0J3+%ak?5a_hcD^cdqk|R;pGpB9zQ7W;=5k755G2Eo zT!`aILd8BBJ!GnPQhdVq-FF=@DAJX@T+~~t5SIo!?PIv{*The#&B(g}{Jf@4z){FY zwYB2|o>L35tqYoV3SV%{u8r)>0cw06v~jnM#nE>B(u}<?d3z*p6!AIJZHEP@b)Ki~ z1?$YB)M(|*a|G08#OTZcY>N6lv7xdroU3f96(O1N?4h#w_v&8n`T2b=&h}<_<Iwy( z31=GpWlk(VBO2siddwd%N<NW^*f%_qXg}!`Xr;JJHDN@FV8Ge-wyY?MD&Y6&sAN0; zbE{kGK%6n-luoe%_|Af^GERWQTt=cho5SV~ch3ew#tc+^KaU#=|Co=f+@ap0OpHzT z3ZzDjHnaTrs(D|%IxtAWs8df+@2Q`jT{HiXn7(=eQ25L3`+oN^YLYHg336Mf9B-`M z{SXu8tS{bqYu)#bk=mtzWuq`@gAre)y*nBCXD6?R28l%uZGYL6_UybanOd6eGTW9F zQ^_8FKW)C<q~&^`zoWX*qfa6x5%AMFAC6pcW>}^3Z!$Sc6ThT{keWhXE*Q26yQj7s z3DKWetXgqTzTA}jd%S(ZLXTT~WOiane6)AegVI~~bibJsv^AnPM6E@ah(gMV`i?5` zQ;!uBgnfngXt)fZ_tCJ>?D0VMAI~kIM2TP*sf1Zi*`uE_LPYAmge@c=ztyN+1YN&r zpC@9$C8B<_<+B)p+ulN07jaXmv=A)#v-o&pORAQ7jUG_xmFVLl)WfnR&ULHOH|1kO z2QjrcUQY1Fd5NZRk?ptn%A_3Eb^x?7&mh?bI-z;Xn^X6X%?<=j)F^+ePQ&nAwN|C2 z%?!nd3=0TgBng8Y=K2NkXSoLRw^vi0OHMd1aRj^+n}1?-KG6L4s9pmX0(HUY+fYf+ z>1MF9o!QzwE93L>^JnM==a;VBrVpkDMYS4}A%E0ROEsXG-NfD@=*{=BXFs!)5>f>g zuAJ7&ZNs~^2_p<R5|F<I{*jdCQPB%+bzIsistye-b)FZ=cd9dRGLec~zCGzKeIQ!D zeXD9h++epQmQpb+rgrXx{gCWqy?YM><!5c;FSUcnrFXB!Yl&GpQ&9(f;AoT5(eBeV z7b_`!1~&&?O^x)4(%xHqq^P%i{c<MpVfzcb5D#r{pTM?_9HvsoSWPz7mItFHJnLIk zQxh>3B<#qF_H3JQytK=^m-J%i#;&l=YE_QYf!?yX8l8I+X78b;HhXK$Uc}>OY|x?M z*pmKZ$j7mBXa2a|V;6Zkvt-)rfeUZogG@6^8<#W(Lo2j#F8@~RPN*-_{RK`kO(?fE zWu{!D66iBg7wy!8TDya^vU9=cr7Q@!#H*XDJl*&L{|)GFZi>GCJK>fo=ianP96~&8 zqK^Nwh>8NH(mKSrSk~h~)4};K`f<s$m6Y^FgR?f;8Go>Wzs_#`mIk&cxhL>48a}}b zoT<I-{?pOwsf(pU5ok6_dm+ZvYYg^6uuWX1hEu^cUI2Pk_cL_Ywy`S}lx7FVsk*hR zb+i+^S<1WnrcA3tqx%c3*Y2BoW=fZs)gN*C8Qe67g4bX=3crkVsSa4o#Fk#J6a@b` zJ@%TZ(xIOCp)sLO*q&+YY0y>Oglnq5Kb&9kaT;}V-$phnVBl_%YMm&x&g;m)(dw<1 z&9M^S(8FEcNTzMLx=I(-lQ76aF0VSHh<$C{)>Ay@*;cn;h-!o7;`@)P_NF2SK%Jx2 z7w5g4Q4{k&Q8t}>+e?o;E-ELc$_=oi*qi9yv5soK-AXYa2)3`~bkNVBer4oz>W^!i zs@n!EbEeS8EU{#(*Oe<Nu_)lUC=)@jJ!&r~vS#AG;*$5?k-%%4>)U#}a}_E!&QpPP zbyu&Qj@|f5K3cO7dG7Rk?!?MY{rYcpDpoo{Xh^%DML20C^%s=IgQ?%7-Z~oHth32! zXa4+w8FI40t=W06<%Q{H%&I@qwRXO-dc8=jVSY8{a@7>+x|zaZ?j_`xRLFjjRaNhe zR5o^NbSJ2#eswbfW8p9P5JG)aU`#hF)7Z|f698(5dZMJ;ZbkG4c=<rL7ODw2s=t>- z^dh|e5{o+d-VR^8=_W!c=3roL|63)s&0AzQ9KiJX0g!I7db8hd{%d<i<YKJ4K&%?s zy-avJR5i6NC@g(sK48vY^7pXy-{pXG%V(vf0Y{X#tjXu!E64oz#@vZXN;{d3^Iwx4 z=VmU#+w|?sxlyhSu1<}^Gj6@>EG_Cl3a24AgI9IK!#AzSqPVHu-`cp6Fm61-(>|v= z5**}F)F_?HuPy>pcC|P*W8pzf@VcJ0Y3Nd8+M~o7w{KHCt6j1uglWmPwrewIQ6KMk zGX7~s>0x0+Yr}TIa<nyIo98+b`_{J=A>QMAAFG|d^0}>=$F}h872@gU+(uK?5-#Sn z=Vkso1<K4;`?!Esun0?dWRcmJ<5EoqufHNr{jJn0y~$?LLvHEyByb+c(Auona&lp& z#>SS{k2WqBh(bULimyBX+XCFYiSc0yimUuYwFLt`zwq5JE5U*S3$zn0eqZ9#y5`Zx z{gg(Vn79z}RM0Rd;eFcaJNjRPSZ>lK68ZT1I<ioIT&AoTichPx3?_y-G<MIZUy!V+ zZI*M(O#jlikdz&^Lr-{ww>KTopSx>mqIP{1o=y~52Ta!YlUC*Ijw`W6hV9Mejjd@x zyE%=!<WJs|Vf~wrZ#I;>jq=C}jE#|agx7(}m!3wkx~K;lw-Z7k<%#Z^6QE;PlNSWp z={d7`2JrK+%kWxZ&Z~W?BD@~cjM~H-na20de3u}v`8!M6B&%(C_zz|r@(uC_eDH~g zFOR>qAU@mnf{8SQ)yi0ay)eMfn^GsfT26v5%=_FDczX+*EfK(V%J)Xh>LAJK8arN} z@)e%p7^~<EX@A!_eak6<Yn(@OLWzrqsevF(a_<{zY&286;{=C(5KzNFSDNUVk=u@@ zmI>v!v-Rc9x&jj`l-z+7*|*$Kzg!x$D^kW`)aLuJepkDa==0=VwK}<*$gf#ocistm zYE+YhkJI4TiuC$Z>!FOvS~>V+Ok&L?oU0M;<7BM&@(|*<I?y0{I>i+~SVQ!voW`k= zuCexC4yqlsM?nn5ZS+!{k<*uQuq+(*L#`CBzH25|+v1p&KMq!MAy-m*uJ$S8xEq_p zhCl+9aVpCRl>_MU8+EbA@062fs9?1BQ0&%+rroKx2MOzy)SGpE#%b5@$6$@i=lyG+ z^Rt}3<Jno!SGD+2fWFAgmFGbaHd<;GB7nhku46L;=plz=^vm)DA9D)4@naIfk^0s< z(G|I!(zKmW3Vuklmt42h2M(gvConnNMRoYSm|djnK`rfL&CL<GH<Q*w91yZd(~yxO zM^4nq*RLpD0hpvFuvwz}y{)Ws%J%JJTP-P_GD{gj^DFn-FdvRYOBBn)_8x91({h%) zdA~5t5Pb0}v5RzL)!c<Xw&{RB%GoI5*0uJr73!n8YXH4g^*M=h3iFqZR|&+?1u;^B zH&CBkWU8xQ*Fi-QS1J+qVHRd6?^J6qDd$NyvDnqvGv%|5cEhn0Vh_5*;Ls1-AFb#s zh<2G8l7Wy$D-ol1yoOjWo0~X4_`6bXM5X4#qE_Y^d2qGGlVasi@S<-=IQ_`nz5;4A z*Kun!Mmpwazbkzob(m-#1KQLqzn#+dvdO!r^O6!PQi7@|<Be$;QXWW|U8nx$9C3Qz z-F2?EeyKNTsJk-SWKD1{7$=Yf&V<FtFDX*=skZX#%EFs|JN&<sKBa$cd;QDP#|nP` zc!Z})pvE$4#<uuUSvUV`s(D@0UKCqadpoC2;W3Fi=d*cH8N8;ulxYuIiCf<k&)mc$ zf4HMd9rUTWzxi4~heOG55fj)eski-B#9j872tzDNAl1S{*R1yO-88@LYjWg{!THDr z|7IPcFSoK>%v-l7X?{P4<u=CNx=jJd{x@<}TdUD5KPDY%8wDkn?|%!|7YIa^^tM+9 z@Lr<wDXZu1;rV=NJ!4KSA(wz;^q9vXJ5bmv8<Aya##?s_k~@kGIVq^THZ<-wXOh|$ zUJ};+R0Pz%?55lDp~1RrX_`0Htl{^*`sH_T+@}gcXX&x(B06w2;WmEvn`uMyD_a%~ zKULul*C(D##zvkF%a~AwD4tV!^3bVi=__y|KCguO71K)Iad~Y}Y<>Z#;o=NSbHBkm zI6pte#o5Xaq|!}eGq}UhffhZDb8T<@e{jhK&FRZ|({8veL=4X7)}<;+f@0OIVP|Ij zor+^mmo!6kiEYpI&7LnhH-I<8J!I>IV(=9;^KUe{e|WXKg|(O1^{qsE!PdqdXUCub zg~WgLt{w~XjQDN}X83aD$8GT(MXumj{;M6|O+zs0-j9xf6P7@3iD(wine$Z*QHts6 zHi736Vh*#D31p6q$LyT^p`ukDq>x!{G{j2Pgu=J#(|9|^PQz9ZUs?76%*PI$(C?SA z<*OW%tr^V;o>&>UPCBUqG5(fyJor+LkTOXGQ8#1EbTdu7!mGb<6h6bF+P1ety<q{? z`&==5iBgf=&NO%Va&hmDKt*BDC`olBwpPtEFTmyf?DQ&=%F{E%3H30rLR+>sa(bXJ zb$1YpsU1Bx1RI%Voz?ts*1S0O>sSC3AM?a|UMMlT(_fH#nY+DHZ}MdXT)z`eP+(O~ zb!KU4zOE1_m5{I(E6hH6=7jw%JU-97)@wrHdr9RXF@G@j<flnzSL-1cj4AML`3GxF zE`us;wE4UlUKLrj3efFc_M5zfoc7@PRP|whV6%3k#3{?PSm111>`>>Javc1er}Mq0 zotMhEJ>6|>67)ac-z2#oeMm&f-!a-a#@9X1!_yv$Cps#|=DA+G6xaW(nilHV<efmM z4aR#?R}c7?jj6tSX1mI9g%$0IOVo<NTyM)`Bvg{K9rf7ox{u3yN2)nGX><YM4>j?b zT&`5V19c9fRH%O6mQhCUcs5b!6dn9Iaq)8WT}s7Zj!XK+W>9;rJ3d2}L4!S^6tU>8 zfCkv^FN=UkUDXy`s^M3k@EsX*p3k}<IM>nHTXjKq!_7EqivAgQ*c<%D`8un_XntMq z$2>W!o*Qyjn3q<Qa)Vag{hO{YC-Un|iUDkQXv}+I2w(~-1iYq8i0f;=<GzgLj^+-> zH|vs?CO>t`t|^FBuA7$)!whE+Hx?GgGJR-~Sr~;cGq{GGv+xEF(@nK&l`EvCw_sG` zUSy#^sZyW<8QXNsdXf()ic_E2s(Vm&#p7m~hC9ka)pXC7_E^k*^ATO8@xkW))0k-d zN~X{Ht_bFu^|Z@S`MySTa}vL-p><?NeWifM^|E_}6{E)UKGRKCah6pQ0QFu<we|LE z-&`N36?WP7-9gwRrYLa!Ls0vO2!^=e_PAL9QpJBwEVN(s0ObaAUa;%Yq~o3@U(p1Z zA6(A6H*eh5GVR#+4t4!HMrq@KR4$ivF&$Yd)QPw@nM3A(Q)ocA-4*OYW7>KAy)f}r zpmX{nkAJ1@wzq9nUuY3_#~GjDI(n3qJ4?~9p?9KiR9fHWu_L!T_3kRwS6dr7s+SqO zxfT532n*DikW%sB>}Dg?W#{DHl28#Rfazad$XLoc!l3y@#q0~cFPVM4iskXk{d^V6 zkylw!p6MvKg<+HjF@6lt=I^&LCV%)e8tn%;{872_sX2EhXtgi;{vmy!kjc~Ok6N## zuxK+Q%E)@k9@$vnM;h(xr!LZ$s^FMwG`I!RRHdJvK($k!*NuswcD_L!`M$l$jsfb~ zdNpE88EX9wLUUk`$1-qd6)5W2RG6(#^Xl{}89%p*jCAL-+)Q;U>(wQEmQ#L^S9<wR zMpLKrYmIxyrFPbtC-cWLTJ|0~4bQ9ck<G)*Y)=Ie*Z(scSc30=2s!=zZHMS4!tv{# zM8SCiB7SeiwCaJ-J#3|9BiYJJ^!?zcFEmK_1MTa*Z{_XUCMq1EW5Ru!m77)zRUe&p zC@%$Q@5di@C>3-Or6YG02jX^}IzQf~eYA_bPr0va`iF+|d~lGnVoru)#I{3P5oxkZ zy0uEI!)14Eyw^u}2-Ji<|NY+m^2^}@5g-BES0nJm7Rf+^M<Wk!0n^SSA_p7@x!Vs< zow7SL-`2FQphn%pa{h`=w6!%qFaUn7*TbmE^-5$E%GZkb`9!N*S1a?}P-IcG2&_R# z{Fb_y&ZhTP?eJ?vQQ^Hs!@EF(3sD=XhoIlMoKv5nhNdQ%_D>4Z0Ji+mz%sTmCi~+D zChPW#^9=5AK>|SU?QjI@J|h0(d4HyMukI9L4bhn(w!M%D2mi<t<r5s-W95`FyVRX3 z8_Vbxu)!cJS9e`ckiGMKq`;)N9T)uK7k_cgaLSkKwa<HQ_BgeZ4%%71vwrz_<OqYq z$hh9utxRlfAjbf;{cQ4OovTg3ql!<;IBU{%tuOPyX}NcmX@h$JA^VsawY!Pd9bHiH z^mv<vXTNr@Z*b;AogjXvtdI5|hs?Jb4F-sQS+6|+8tQXbp#PhbdE$3~q<azc5X_^l z3>J{PTsgM>W$X#MnUV2n9jW8Lgo<5DpDelFb_Gpfv``Q--KyW8i<I-(n8Hy*zO)8D zXHw$MPYzjN3OuHe0kj6_y#kSB4R-(1AX!|06&NKulBkOu3WVCs)cf`BY5no)q0Vo- z!Jdt;PTCyjwFn|F-sW33j^A(Nac${JmgL#$d!{@3=yJ6`34hY;R@{~%&}fcF{d=(U z1KFB|giwKe5GiNlfe{^OlroxVtDCHTfW%R+Ztf_tvEMo_cZ{9&1>1SS>!(~p6u&Wt z?9HIQ1#I4Ue=>yaw-8<9(?5H|Ijt_~-*}FsE6e92^U_R*t7?lwiu~8ZFFx>sW%Q&6 zY?iMs<auz+VL?Xp&6ajGoE)>zr!%(75fNkAFB4OUI$D(bQZEwBm%S(}m|}j~4<^~{ zEO^b$m&W&t4juD~@ZJ*+7-`H6#EbYsib>YZoO^lquO$EX7hx~%d~|Dc<RQIWa)9mN z5L;fPZ+bydEXjWAwIDqX8Q#X(u&ixB*KwnKw>vTXpdsG%z%{cR!iM+A>!=HuUbW~x zt&0X}Lxe@*!(j%LDT%9u9(k<UsjmzQd*+B`_5xYNiUDUrEHaIi=rGhRQ@3`h-?xp+ zPU^T&!ug*dsEWu9O@#q-h#nLxB%J>2{>)y}A$>7Kl%BEQ0+K;rPj2%9NvHQE@pDCJ z)o&s;>sVjobBbVBgrW5H!IBOwjI!l!X{xk;9gK;r)W5Qs5cPjX_vcs(B(t9$Ir7I5 zeXSe!7dOSkxFefbZpv`fg`ST#Wy4&cH?@}asW_5}%dSZ3q&0|}Eedjs0<_|A1i1K( zq>o#M>!440QR2_0i4w{8(45-mDFP9CS|HZ==ZwA$se>^cIv7woW#{mmD`M5OcxJy% zQHIP-3!J-fR~K}kp)j)o^LYg0qpXyFs80hYL9|4s7hmpYvy`$sEGuM}LUv1Hr9r%G zAM{{{gC`vJCq^5+$qM;Y7>J_q$-hTS#Jzv%0h!+cFDmLjqOS+OQ%Sig3#MGg{C5%m zXGsZr3Fo7=?_NN?*P4H*g<yRNBPd^6MqWK2W04l&*A`HRPnOHyW21$aG!#@!-oQgB zXY%aWqV@FAvfA}RyD_}VrGetZc^WMr0*k8WUNdIG8+LNwSl)_OmKv+Ca{sbu)?vC{ zJF`snV=sWESxR)-H?#kAvj8A;kuaDvPAkh1_!3ewh}>;yF<=JlrF_o%A1B&j3Ih(} z+-z~-I$%A&P5;rl6nLpHOc%Ho=B@QY#o6R<BFPvHlI3#0@dfRY(>E1dv;(;>vW1JP zQP;{+4{gIgdYlc$mF2+wx(rhs4~#?zn;ccut?s42Yl#qg9hxx^QeHPm|AqzR0UpT{ zwC-qWdg>;2Um%v?M!3tJJq?1AJ71mA;-HjYA+SEdNMZQd(u4MzxOb5J=@tvj4(Z-% zfC}XJwv!nt^XNb~%P?;K(B89_Fy(<YhwS)ow{ps6zo!E4CE)-qyhng?vqg6in>f|k zGy7L=i{`@Hg+b$Au4!e#ODuq92i+AJ&kFv!;r=x`Rx7=0ATGqjkey!ofo9h<smRA1 zc3yaq)f={nSk@GHL<dNX=jDLAt7~Q1u_hc~g{gwG{DinLsTQ$s{fCZ$6N4F*tg(=I z{LaVl9@?h%mTKuho~E}Bk-h^x+|eK^o3EJIy!Y(*GE2iLb~|YL`}2=4Ygy^()Di|Q z4+yD?o|)J@$d98e#sd^+<N#2a<Z2Fc6_!y)VgqJ8{Lb~S*oge*Cdxg0v48&?XXA4r z33Cr&f{RbxB%U}3w}iP-EaMf!#y<XYm~|jvt7S@nZ$z{eo-SpLSQgDp3_GyPI{X^m z<3V6u$t~&tFL3|X2hyiUe<XGqg3hPEu;C#K$);~4q61~>x4sQmHPP2!Jy!UxKYw^5 zXi?JA<dpUTf2m<(Nnv96Mt}E`nffYHMx{bR78_>o$vprvO^+e=FuI#q+xFk3(~vs0 z-*r_epHNg9KCu9@{wbzI9||Sq!KGV-5G%xafkR*A>6M+LxlR($Rm7jXkU#iN?;Z%8 zZ2B*WRIE(0mwKH!c3h0R>PiO~efNQ^s8GVW8lUxwQGSu$gtmy5xTuzEEoq6t`l}d) zwbl&+-;S`J3O34NLCi8PDDWJuC~(oQ&7oO!W;>?4vn9W%_E|qyl6p!qO)X!I9OFRC zj->WKDOi0IR=O83(tt8F`{#rvPye3{epXzFQYnO5sL+3BlC8U;+^~FTKq<K5?wJ?} z);Iju<JU?2cP$Jp#twla?PicfJW@)SU-F;oa`Ar3f7QrkS(rE_BoCg_hqf*J*W(!8 zy{W*Veb?#5`RL@~2!waV|0tsW^UI~^US^5JYb*97`+x0%<eQ8B{||qvcIS*Zpp5_e zYTexi1loBfp`r5>ea$+RAy7)E#JE|Knhr?C`9R+iZ?E2Kqkp>wxZd;y!iZX|_~5@P zkze9u;%ZTy3m5Tqro`V~>t7ljBib0tx2AGaf!~OCR`ze7XG&B?qDucO$^TmWoO?AH z{||p!B|K{GGmU@eSY%=zuwL=kwE_OdNF7j{4g<z_=##StVArgTr2kGul`wvL33})! z#B(<LpI33O#ZFL3s+lgSIa=0tFSLqF^jBZJarIMVBEuN}^@R~$BTZ1H`>VO0W|T0# zTNlP~LA2(P^igEt0nnBM*iC=EA}!&1-?-#IFNkN2K8f~wF8%kFRl0a5bNl%lzdu)> zpJ6zE?asg4&+Su|*8I=kccK`&U`(Ci<NX@pUu0<RzHUTY^AjAG0kE@60vJR2C3b&# ztk%4k^U-0=J@P4l{95hZ$#j$awT=HOiiUiERENE_lLD-G>&6GbAIWBn_phVFFa-3R zJl2?@Af`0|8=)7`_Sbu4O2E95?A3VxI#=<(G}B*hlZKb*_f_gqvu0fD%>pQA;a`gG zFLR?beE0wV6SMyFe#r;_j4NJ$TXopke@&3{`DnsEAiv&csEeI0I~6?tQG969RkJ(c zXvC2Hv-faLlP}i-G&SDlts29dW?4xxf0-+Z*F4AmxpFq9S|X;jMOmcD8Vv)bJCIj4 z<KSFVMYZ@(!`PtRver-!{>c|tlf4#`jf#SZSpw2cv)@I>Hj#(a;V=mC!?0MYSt*5K zYd;cVbO&neUNjqt?5{I6;RZVVFve&@<@<bTw~Q%bK}5iWPFsRn68EP!gR1t44D+v1 z-iQ60;I!WUmvGteXo+ZS1I#nS_*Kh83O-AKzKl-?A`pDDz`+(i-;$1ZpZ>9SoIhck z4m=nBtgCM`z8!k7eb{NobB6`9o}?Yi*|guvEEF1t8gyZwJv`VmiuE7LdRPpEI1i!p z6}Y1CR`!#C<I}Q1+mWrDJ0AGvAfS&}E1uydB6iErK-NGWZ4z`iklI*I>GNqUqmYV` z4MTp00#6586!kPp)G#JYwGO(j0h5yt0cw+Z&U?uloR)#xcp%bP#qXdGX(bV3G^f99 zlBuMBi7qU~-sV`#1QXD2ZrCGZCvBK9))Vl+Id&tgZ=3u>AVLb@OgPiO2h5-kY0VBA zK>EQ1^(9plBi6@`0QEF6qb#U4n;!wQ)EPX`qR(=g+~;EvWOR_W;X9UHFm>1o4=VNl z%53p@h_mypVJOqrl_rMYn9nR*|D2SuS8peDfMX9H*zR9~?A5%QGQ`+HcHZU#g0Yg% z@V)Z^HA!anTP5Q5*UUyfy6VmDHEbgC;YiA0<T*dwd%IzM9u46pVvejif;u^C;ThDf zI#aT|3-lyI2&FGAwNd`D*lfEcfG&&qt1elv3D?1XUS)!81urVh0t22MPDIldhEc$S z4l@>e_?W={m*JQE#P_e>PyiA5oYtrrr_S(jVPBOa_JDVu3+1;B1}d*Cm8Vm~yBN+N z*Mgz<fMBN#z=8E@*)UY0Lnd3Cuo@-bXB?E8QUp791unJqVHOF5HvG|vhh`m$<Z3XQ zkO)Fd=@zJTre1o?gobHPAm({ym2&ePJ&k144LOmj!4ffzz}}G!S<?hu_QmYXt8M;m zZBD+pw{}JRl<#3h7C`V?XhF>6EacyL(_R4dPNo*{?|u%G7DZFqW7j$xi6j$4w8z<T z{9t8Q0%vK0>`)ji<KjFlcwlsABd;mIlR8q>Sm9oJ6(pSgoDsXE7)D2k`RS{&WB#Hi zFKD|NYW-8QD|2P`cPgQ;Ys<{6yZD&G6ofy-d7qM47-ejdVW5Q~L5wf6-$5-z^dA0V zv2QDWtqg0SzG{MfgkbpMfk}oQ4AC$u_Um<g2n0h7MzILMtx0^LCtHhS&(Lc~tzQr` z7(dm^@W{JCx5!L2`yn8;CtkpkyOGl&Xvz>Kg|ISj;(*?(Uy{~Bv%i7|Obl|k4|Qsx zgKc?8Jq|sijk^`hLg8bY*^F>1vS{GrBx6qx=Ub3IX~RQsrZa?;VG#jx$6|}H`&N-p z!z!LF*0)82AjF@Jw<)WeT9k>*z~c|qge5e;zq0qVB9;%rOx5>lu;#>FHB;(+$p8R0 ztLerVjEBxZ^6_OQu-JjGd72phX*(tiELk5UDpbmPcg2LP6z|W7hf2a!@}Q+``ZZNm zRdLmN_ZFm}BI*EwVQK<XE2W;Lk5Bx~h$)%;OGPJ&akD+rgJ_Mm1EGH_gz#Gk$}wEB z<<i{gyd{s^aG566_@zzG#;sS(CQE)$G$9EimCbL}yQIPPv_l=Uw$H{EzE(Thuu(DP zlSxSP=rajd6&15OV-nh;xvcCQen#^X8JYlF={&a0XaFY-T?{kU{46BO=BIa6^fDqo zcR+2!E!Gqt^;kRZUEn@h;BZ?Ya)xN+gXV0QUU#Wl#))E<HlK&7N;QeBuqJnWshg#5 z<f~4vzzkFXd^JpYTA4i>c2y;K1u};{{96Cx1vAGt4wH9|M6KV2;b>wuNE*Y*R)<F{ zbi^-miHcCk<_8;j@^$&ec62Qp!Z89XS45hvV{I*%J$m9K`m>m6fH>*$-;5AS_@BN( zM<;7^#tpD?L|CBa^5PEak8EpMTcr<SG))VkYKue4s4QrQhP=_jpRmr*$l>`ujXUYx z4yu@ovUO{vi}LBPYr@@W`#YDvH#9W7D)+!|Gc>3M5rCW?YGn&iS)=uCg`fvR4-PtX zA*HO(JT(+*V}OwI{Pt4$2hUS?{FUOD3_lZZ5X{{>#hBxlLNfhTqc8JoC<qG-v-j&E zRfKUv55_xmK*G@N2)GpH=t`IOUUj?rLxIEnc1{g%wWCVx(Wgu>V&jAAq7tPEw^vsD zYK54`kA1josU>)?e-^*VO*gl*M4Mb3T-!V@UYcT_+^x7qd?g$BMCfdMOh-RGT!&JI zIcz7l$_BZWV0l&lvAZ??zjk+o4(NoYvCvNf@<=d(#y5B7=+cFG`i4)_tH<*8B0u7+ zwg7eSF2s>8t~H^2J8a71Cfkj$xWHzfna^>yupZ^YaM~PmQ0JksR?91UBJym!FYJs; z|B{yI)gk%tyT`c^tgmRVKWcuzY%mVklRXXutDl}?OBTFY_$<9x-~Tdv8V=O|t^y#_ zjYSa>Fhej-VeWL|PQ-oCsrX{2Wsf4VW%&GYJHGxm>@OI7@KPKh0+CyWuIQ*70LZp` zJU*4DwM#fuAoH5MzLisKw|~knLT|xP>jONOA`s&_FVHe44&q+X)~!`eS5CnGnv*pD zj;Sr&A2lw2&QNPF6D~lETe1DG9;^FREs-1F$pv{>8{y_LaX&f{12yl^#7wYyl083_ zhbud<KInjGr5FamxP)g|Uqa+4xrz-A!G;mAQ^Hkhk+(F{g!nX{3V%)O@vxHB%3^;N zt_3=e7#YG?48J{e&tx-M_VK<RMQ#cUR}oDQT>d$j-El)SODR(l_$cg$P1sv|V!eBE zUeSP%G$QxnU42V3fMC4-UCY^NZ7<&5(_an^yBNvv4>W3p4WYE3EV3<y><Rduk$`{z zW`~X-;x?CQqL`@UwE8NSXqJ206;kH|o6JM~6{0Xz>2_BtiZ%L_z3um6(O+(N+Ph1g z${sIKJTRLLvp)^&RQbPsi%|P=fbPf>e|CGp@NGEMW;XfI;JVQ8?(S|1Ha!LyrB@dX z0270v>^%wCC6#mu&&+<%^_)Zuyr6NOZ`1gcx<eXRtG*`G*TesuAHyQJ3Z3;R-DFr% z*i{PO6P%9&pek7+mX$}@o_lI32=fj%FBvb+yJk-T4fz^}Ix%|j{xI<YH*s#jAzG5v z5=kHLz6?b_I@=#UbfzCbtzc(j7N+zRD9BjPEX6vJ--ns<!q;+C%~`V!!m`;TV1Wa@ z`fXM9i_+Z@cgd>i5fa*0ggMo9q1exo3H_i4h?t9=Z`d)w>=2)dP{L0nmMMGxQa#mV zD^X6@qI=(fXarE;b2uP0`+JZBh)kB<*c{+j?P=(c)j$JM=xHsO(AT&S<^J#_&jc4y z?szLb*@a1fgE;@ykFG;RAmEwQu^Pih8VJe7W*Iwvh8Si#`nuNm<jij6tLHm;56~vs zUxc;-RfvH5L&pQQe=0%o1xqKF`*}9pyTf@s*W;klB$*;=$j9mr@PpXw!_#>#;nzrs zoIJuiE&$jp8<Jbf77de~!9T33or%y9eZNF2*2jn-U>AiVj7cUB$e$e`(cGDAcYP-( zXS_?~GFkfZAvmC0bKSCmELcaFz(hzX9FIYyJ=J<cxlmBg7QFV##<omEH%97k^Dd;S zrq`K6LjfyvKRNi6jzVetq2tQ^rHe*pC9H3p<}t<uvQc({XLujOLvF_#G9;_Dmq&;T ze(~~80KWyCS+^Yk{`-4C3Ai0>f$oF)*F!k0SRBG>nnOZ$zf|}h;{1g?T+$p6E9@^K z*{`sd0KH4$zA7f|m8X<JD}6va<;CM@YQ_y4q2}e)knH;8;P;6SxkXWi-d$Q9LLAAF z-_Ho)*bBOvhyw<k>ha;Hu46@x#)o&M#!{5mmqO6ga4iziWCSn)+5!$~7d*RI<S=U3 zPx;t774?rrustCywm;D19=nuUN{^&4$CL1+UeJTZx2I#?X!;9tu?4|oZ*C&h2f|#! zHASsqZ?A-z%T)h(@hXl8G+(L;j89ktv5>Ui3~>DnSI{8{|7RQ>z0dR{fHZ6*Ya?1m zJtO&T#K9o@%GC><ADsnd$t`)hAVo%;{cF;2Qgw}79#`*`{waV~d;i{^DfY)-%AW5% zeFJnhU4tl(zN-!C6!^JNhKB76)n5HUMn?_^A6tBtL`+*f1-41X#tIAWIKcdbYZew0 zGX^?&+(0at6Du$4wPHFA4+grKxt2XeznyXO`T2@dMidn|wa0ZAD-nri^Bgc>3c|o4 z%=jcR0&Ct!HMzOt7)Q^0M%zn=zjq-E^6lsX2Bhnk2%Wx}cezF1*C*Z9g{E`dKWVCM zv{jcF+NJNmYg-(ca?Qa^S5W0uCnrpxZNt+_6PyS`jGObRi9o&F{RicpUJK@^o*&h} zW7}f)OI1$=x?M@Pnu6a1|A~XFf&b+OfWvhjGrxSkHjim)Pju3g&`Kt_TVWBSk7=_Z zMIz11sYin#7Zp=F1qMLTv}wbKMV(OF2qsbP%>F#cVL<;ia<u?`WJM&pzFmRv@F6kE z#lzejwSy(qoSUaFK)=an_3E^Yl%Qvq0uxVay!nmKDr}kOFv_cAc7Pfns><>S{wqFH zlwG5U_keS3{=_s|H!}Z;t2jm4tC<u?&Q^GhE-8c|X0EZHL>rNiTy(59hQw;OI)JrZ z51R)l;HQIXiwxwgU>?G6c1%(aY1CS~bCkMYItT@dA}6mJbp8)2G99997SYx#_gDC4 z>V9||4-af-I|u%TBDLa6_)~Z?l_1`ToyE`#HuFx!f4ZY{xzYbpXrInUe?gvVr&)d_ z2mKj$mOv>;uD#TZX8<w@JOcwG`Ue1G_?CfMGM*21rmk_@-C7TR?SEE_<DaYbUqEN> z^B5^Ql}FS6)Bgx})&Cu<3e$$7`~&|t=u~3*f5&&4fmz@Fg}G_F8IRe$E52CAd{6-Q zVg4tA@n7u8%@`O@<DZz<xnlvh07S$*A`iv3zr#Qyg8KGeqs0F?hz=R{XwS;v#M^1k zqfdH}jXs7q@ap(Cj+2$TZiDT$V7a#Th?ce|gPiT{H^Ay_0?y14$;#e1$JO7LD$a%7 zhC+SbG%BsIn2zMd^pBv%Hnb4_m!>Y+eu;hnV^xiHc7l~%KDS!sh3y+%qFO`a0*W4- zOBn~1EIQ4;q*cw*w<&>}zdJlt$0@9K&9Hd*?a;l9jW*J-xr@ck;*{9M^;8#FnauR| zz`{fz!u(yy26g;#kLxs@=RfWF%}+`!(^;-8y$?X;j<<NDvd2$?FWbS&ENkV~gB)Uf zkmj}Yff)bZJ#ed875U)FfOEoAn5v~^>Qk6vio7+f?y_1~90iC2-6ifl8?jAq)URSb zjtc@qrF81joGd{9StMW(T2P;_2d#(ug<A8tzoC!b@%A=8>{|@ekk2sY`T51T`NQpq z)8r?hoSg;v#2bE$XxhRYNV6U;y+!@DoYx0nFtPxN);N}<&PGFP>5zs!RC^xmZurI* z-4#x+vZdY<L=qCw1%PwESr9~DwUk9tFYC0tfNbdk-XoroIvxl;cSUvmhffoA8qYw4 z5`thmd*8t8-qxBZdar9xSuN0akCTe?$J@vyYWLFC&YrgA0g(bIh6f{6uTuf*Ue|pM zvB9u$wCs6V;1R`uwqbp)p$DDv0AduIT7M3Y!Y#GP1zdJyuk+QJ%9TN-RNQu|L(E=! zFF8pEdO{mh&QziXqpAVWE%g#mRW9YqPGqwT)GsC<RKDlkmK_1|%(F6HK&y@B2+DZ1 zWJdHb#U2VT59-P--nEx`-_YYQBILi5Uh>3z(tVaU-UWb*1Y&y;hL)s{yi)VL29`Fo zX!O`sgb4MUnv5tIz*f*jfU~tJ?NK*o0qs?^(lfolWG#jBAqT*?$+2=MG;IySmmQ0y zdrqr;!O|&%3vWxBl|aRE7P~JS#St+oe;DuhJf3<I=H=$kIsh+W<FS#EpS)~cwp*;; zgSd`wLk?(<Dn~nZ#p~4UOm&WBH<WD`IDX-@p4N?O)c0HaHsa)2Uo+jQF&-!wI&51s z8zDHaMqi?QtCmCiQ&H(-*5<d`PI@k4m2VS%_&_)X^i%EH)#mn;dn@Di&#O)-1n|l> z&TV@6-M}SM*&e=~%{N?A)r4f&iMZ9dG=k0|qHjDO19Kyivo;(oR|jn9a92vKt=L(f zJofa(RE07u4fq$B@fmBC#V*6zv9#OlRUT_M12RMI1%et!$<CoymowBq3DR?DQJkHo z1;{mP0n^8@Mw!rqnfDm*9v~JTBnB#!hmDv>aE?OHQbwv$Y8IZ%d+_K5?&#z0dc2#< zWeSq)uU7sW9BnSjeLGy*2EZVy3iyF|uP}qYW>;*n^@6j&0?4cu&j8?B5deO;nz<fD zzta63x9G|o&nSrKwm~7y=jUSqyfj}PtL8!41~9=fTrZ9GL~f}>rb)S1Wufua0T5F) z4m09!TR9D&a-wvY3P6*#%?j*U%J8N-JL~qRm=gnLZN`0p&eH&O?PluJ>5AK*J3Yem z<#SPo{KW#E$XjLStj2yLGs4|cwhEc`8-Cb0SX9s)JKDGb|DgC$1AX1qlICjog8`x8 z)i?dHa`zb3536{fW&6bWRX=ZYzo4q?O+b;25dge|r5<?HE3Q6g#ns#KZXUdWHRVIA z9`ky=8!YU0ovvuMYO0S-6(1!k1`<LQ>z|BY>QP^3%i9CS;j_|CYn;t(&mDt@kyXY* z_JQqLOXTa~P^BHraW?s`Ft14GTI|)1N+Y6-DFAX=M^pz(?@EGxC^FL+NK!rnQa-ek zM~r|*)iF{kpS$bH3P8{1T_7UfHdgIuTMb%VsCl`u8RmuUH$nT87gJ?RYF)DIr#<Ey zwwGu<frvI=WK;PnVOtTwP)o9IN{u%1n7ul;T?TA?b@z4b%P!N@c9kB7u^4^^Qk&wY z;%wDtH{#sq15?FtihwhJ;dPCGKfhM@Rt3_IrRZtp9|X=Qy&H_ZGb5j|9rY*Xt<7m( zUY><T9_)ZF<WCeG(7rLtnjwB16EL3W28e+^i%rjN$M_Tgkrl^zN2gTJe$F?J4Z9V^ z<_(nLVvB}V^UX&SNXpFl==RiY&j|3&SC*jX4wmD$MmM=rEqC6H77&YRPh3)suU}a{ z1VGs(X~g@Wq;=<SgsNS;bF|k+^;STZ8}V!<KrOYJ)yUO<5S2nB-{oI<nX@z4t&cmc zzk^e3@^fGs4?F}ykA~&jMvkE^Ppn!uC$2O^^LPgTs`?4s)V(8~()%i@xM^g!9K#3- zcDN{?nWv-RU6<P17UZh99HMSdoG`Z_cNxjd$2N^aY~m@a`Fk;yM;)?lWl5N-O3}jY zcMT*qyf&!v?Avz-*Xb)gvIHsvWGOIN5Q?_jM+?x^S2y;6=24yz5ku-EJC>a=B77|y zf@9xKzZQJpTe%nH{V4TmBV-0srjm~)Kfxu0by^q63cP8`Jsaiq)YaytagRsaWygk5 zAdTA1eY*#X$WW>%VmDfAdpM5(U+`&5*-z(VlxtbbK6{tSg=~RL#yE@=qHwjn#y+#N zR>|1%+g>B1c%LQrAyXbBSFZ_G?lRq&%EbOzLU(mu!$-95yQM7(rxrUkqIH{mSC@VY ziDbb2xcgj|IEA+J-Y@FU#M9a+ec>`4qU#0?zd>LH2AB-uV*v;|&je@s{*@AkBEg^q zAy|dJY`$$^VrscnYV=}``3fh_&xh6=YIU7H09_U6N!_=OGIh&jsTjG5^t&YCVVEgL z={KJoZRcXK0BSE}nl|~O**v^joG5CQK`RE9wLP1|NKq@@9@})icP}>z4H6x#x>Pm+ zP={dU;}PG_K~;y&rn0C$V}c{_hN;S<wUq&yFJ;8IeOPD@Gu=G?mN=`^Q5YC}3}^y< zxc>m20h5Y<V3gTB1n^Xmhb;czU#9e4;(7A+!j_@<nqR&=iBuPz7^c4Vq}xgXb(Dhz zJan*vK5(enP>#c@tw3jN6djhI$Q>DfeyUA@Lw2Ai@q0<s*w=6E8ePIyw(Xxd1Y3(- zChuRK9pY;#IgxM{78x^SKw0KAH_Sz69h3!mn9S2liZ-;X#=#zqOi$8bSpqSsd&Wl4 zb~T#XM}9w@cXYz^=DQ%RsqOUWnDMrLzs4X|qAN!<N4*{gvpX~Lpu+9Ltw`jaR6<4A zQ1gz5d6L7=XFIxTuYSDsQsqwwBKm9;?qGePVCIB#LR&p{5D&64#jD4)(wy*aFlKaD zb1jL8!q>t;Ji>u<Cr<NNZt%ca!2te}4}xq*<TLUg29vm0lT<Ig@v_GG3X@nbuS&8* zqUqtYm+hhNd<lwt&hOM1a5<Q>H5Kx7>J(2m58~N}P0Z~kM)JwPd{J>#VU<1Hl%n(m z;-az$w8i-10-kr$aIniTSt_Fyt(6qUI_x%^9ehOU%{OEnT*wmfJ4PM$i~6A4TT8&z z_UsyS9t#qO*ad7Q_DY~4CTMygOK%rIg^>2#ni6BPSld|AglsJNpa7Y{rU-7`PePxE zcS?Gl-IWx1L()^8$hzw8?oDjnzctr03;yP0=D;L!lwEz7M>9?f$CuH|EZiYl;vRR9 zB~!xool`$6ItEq08b_R{xP8}|5FZ=3FxOq5E#@lPJBU6Y+LjIQiez*A)>7vAk=If~ zp)?=dnu|-d`V#6Jeu8Jr$tm$ogQoYgLAIRC@-b4C)$Ia&tR{?**lTO8i)#w<ncg8( zu@x3HWFZpCGU+W;p4t7j)+*xpL`smqeSp=yz`9Tch155bvFNtr<iQ(<AI=30+=Z&4 z-`(WdW{=luuRpF0@=8c`i-0+K0MN7b4zcB+c6SX#)pxxu_>}ERn><OQZ~K)U>(vQ# zX_BUE<#q70*)|r(4lH8Bqr3a)8f!qnIiXGgqRtBmY_9j3J+H>{7K%{r!Lx*aLq0GT zxK3H++`k3EkvvC7yG%{iE}tBe-W~BNQ-tP73lu}U=;JZ+ZEE`zMpn&}*YZIBT?z8t z&YSLdoeV2ftf5`d=q7Y4m_<#;-e24%ad&I19pb$m>}c)OuP-GO*ptV<^#_yG&d==a z!7(YB9#nGV=Y*h5`$^W=uI|Vufww_%8NI#wS9J~4mugScK&gVGq!YMTR=N%3^W;}f z7nn2Htts}~-Pk$3NlwlQJDRjBB2*##B0ah`dj7?M-mQQUZl6G9+oG4{YhQF|UtJ<5 z)kJQ0_#+}HEq#Qy>vEbXj%K`$fcJ<?d2`(R2p_N&k~AvQlhay9(HD8RA1ON?Z(iYC z@{aS&7lWe{<yY7F?DO*`KcZB=!<!s>QmTS(2y}-|KoRU!L`>p74X70Ryql9VFJw*3 z1z<c9XyO1!1aAJySBh;!uTI3KmXdngJV`&{tH|#%_d?N>fW;!y$v%g+OQ&e+n>A5; zZQvhEWBl$=kEd>h>yRa7tcgXao9v1jX!?f#Kr_PGL~6SXe67#<C7JB3$@PCxYBv=o zegySc^OsRZfier*qnpm^kvxfbhCkG8`k_O^A>N+jH;xM&f*BQpZY)2oK6uwnPP>@I zb#|M^a_DmeFtEhbS0A-so&B)AAcDAMv!kEOs#w<dUMf=TxP&q5n<+_2zwKArF|8#h zq^4V(Ii7W~1&H^q#VMLLm^i<bi#(g?z|6PUIr-bYw_yLcM-w-kB-T%GnthgvS)TJ_ zzEW%Q(LZw9%j?+QvUt;7pXZh)9Lrs$%L;F?8s9eYy0W`Tv1j=!pBj?9)m_c|z-eD) zz%FW$)mv-(Cg@w|hF`(H<dSKxuS%(?C&EAd;cxQCrzZbuop!09Wy~!vuh^M^r|i+r zi;`Hk`SFTC|ENQ0Z;j_m>&7Y(x5*`PO|PYGH%E1nFLNw^x@~MGK%VVY@~U#Xlo=lr zcm?i#5AJ_+srI(Mm%&uDmNmF@NVQSD15SKkL>Al7e2!~$pA*>IG3lMcVC>AQ#)X68 zkAhhMKH;=Qkm+K1h>E@}zdK%r`7X1gH}+(BG@IAbrr$Bof|qvWX!TYZ)lSPM4dlvs z1406*soy{<+dR?-OJJX!3^~=mY~uX=WajNN>-W%$WD^XR)k_O}GSR8Wdmbdc^3FSD z7v(<~yzb;QE>%*^naZ&T&Rj0!7wPXfBrI`~KeJCtDsD6P-%u9f-btx_lO15?Qe^rg zL(N&irbHKABKu^1;j1`B2W165ov4H6JnYu2O&hweNLpATmXz^}WcT}on{Nw(7Yi-s z`f{iImeQ;C9|SxRansPP71tRSau&wk#JWsa&kS+SJH;IsNnskd1rS5m_S7X|?8gYF zZ%7F&F0{m`UH7-nP&##^d(=s5DKN#vqN^(2`+&S8OD=hZCi2c+N<DnTWJ*@0zoR+{ zo>ODNW1kL$*q2{U+14t^vj%h0(;HvNX4yHRs_!88vh|@Uy;mz)sS!ulA&b7^>yUZ= z+3}VkafSz3S9*GK>jYGy-3#H2mpoU_!xKDFb1?C=rt#51*As)<0TBqt4k3q`r$180 zGqtMPYVzg8J;Iy-9pjgH)?D`4TV30-M|?>hRdv(O5huKCIo+I_hXZd0F6&A%8HBeH zhbbqtB1+Eve8u`bW-OH@`dFJ=V;^WE?gYhprKP@P9!LosbkQD2br|ukG`HSuTXiax zgd^XZudpwIPUgg8+}d?KabOEQ!f)v9o44#at+abY4+bo|j?tvSPGgap+mcY!25#`} zWP}4E$6IKGuJzvj*hJ)RLGgE%#pBj(k4b$L1BlZHDTFP>a4&gY+41Ono_EeI=b5o? z7iBjX6XH^xMA(d;taW(pGwY&3tVa<SbUX>26pwxO%%IK3YfjZ+xY?`nz5I(1fP$B0 zS-mf>99|NyY>i9HkhdIV!4D3nL!;zYHWx6BI47Z$Wh~Gcp>ZIJ9)zn{;iTX9#%T>i zdDQe^NR~p}t(6d%`j|YzH&BelKit7BVLL2l!!9Zu-TM9#D;$5z4v}z>7Bu$`sv)60 z8wHv>QiJo`z5ej?S7SRwF}S6ui`_KvMcl*Cu^)Ep8cn)BiycO1$iNv_%W>$ACM5=) zoD(ntw|WrRGJD-}9niYBl_wf1vk?PEb*u(OCD95SZ?mE?r&h8p<5Qwpvj}Hvexry` z1nY;<n@Jzey=5s0UL`UE35l|E98CC5NMKJcE$`eZe{4+(M0$QrjpHcVz8Rk*z0Hoi z6w7LZQ}FaaY%VJ~%3O!4L~tH_NTlq<kA0ST%0qf$>U<BC$cg^`mSrrw&+093$VYcN zdze#{WBfk7tn$uiOCh@JJN$E@(xsir3r?xNQ2X@c@~+=E2kd57E%qK=MqBpI_rf=T zN)wrEHI0WEP8jU9n4J%$QCzh5o2jt0(&<a{_)oBrBWvBja}UTPq7MdrMP>a6XAdKP z_P7xeWm`+#$KbPbHT6y()}OhORZb)AMKTPTtdmr<;E+M@)-3Qj!qsz-1tAubz>J2| zq)vY&`t;dXL{-l^xh~1dmpu(-yHmEEN>V4=5$n!Yf30d8_sZP1JXo}hK~M&@1~Lb; zhcaif0fzr$%+ky(t{RU--v~}#+S)k{_x<*(%K6ky_PdGq63uydt7?Y}QxXjcpF3Yh zmH&aAyP@_bZj;ZPX8hhp(<h{}DgxjoIx9$ibjg-o+eG|__$Y__W5c#@{aPwK1P&GP z4U=1;XC5dfK4sQrf66SB=+Pvp=!)nX74Dn6!7si9{)3<NQCFH-Z87o$GYgOTz`$3p zM7PKn%SC3)mzb}tcX@Cq7X?!ZpN+Q)6bL0_-IKLJMmjTsd%UP>Sp-deQhqeM6LfIh z0{`ZOpais*qK_PHiBz&Wh6=<V58BIBLG^cDt1LO1!w)tcYrke&BkRiS$aBgwp-MX? zz+`ME+L6Of=vwPCBS~z#?O4(p3Ko}2(@~Xdw{o4%6h&RJ`kLFW39!)G&WFruJdIRk zwvf{jApK{&Xlfk9F_~u&t}{A3Et~-XA88p4I{wkI3*g1gh+h6$>8ENs+sqm9c3+P^ zkK8@4OUr~g6~jE$Rtw5Fp!UTay=yLsy2>Sd3&;R_W+$6tZNAr`<bki7)bZ2|@{q}# z7MIn7!+1y%|3G!S4j6JzcT^A)O%emNKGLdRLj`33B=yFRm&oNJN>H@@n>FKs39)C( z)82f*cw-7}wzHdtMjZ^-Vt^hpw4K!b`|#aifff0wr9-E#e#`MS)Ekn=rhU^Tf|fEx zk6!3LJ_RQYgO}AtWxZ@bDiQ{~j+2y<G=ElAnlPkvC6Av`sFHHKBbyDGar$&_-<&EW zF*JHe`T$Kjil|k(|CI6&d&QE6&2b{F3gf*05q<D7ee9N>b(5Gn6esVSk?aa_TQ-q< z<NYl+ZtxxPC(^6?k|v0B(bb7ZxC^&G1|d3Ut*>BuFT-crF!edI4b@9_jSHGVCBeyH z{CJ?ZoYn{bRl~v1A%g>DbHVi5?~2Z+u0DKaZ^PsKx?wp<&FQ8;44P(WO8kK76<;6p z?Va7g$T->3aPo4CL0~uatiyZgmM}Hvo6ndACc}qePN5Tj49gI{1=p)LbK=FogcS&u z@RA%_yCNAie*l}@K4U&CjdG+^O{L0C6Ms_VDP6i#SWLjvaL0A6>EYq2Z_G#J`z3bM zLOU>ttwN~R4qgwQF7BVtjBnWgo~KaKjG?r}o!p>y7UR+aU3o`wL=;7yo^x<n);T=C ze6#+?`$C&Q4v>|B%KDwCovI)+)$^dalU!N17jrJs{~)yZhqncO`kkKYEd|qDUh>TK z?T3S62yb=0l3~>En;-2cX7uu!NU&^1@BiZKy`!4izP;~rZ0JFydFVxofYPLe4oXLw zhzdxF)PMvybSVi(L5d;tYJf<WZYZIH1wtnjDWNJQK<FWq1QOo(yW<(-x%WQf{bvtg zFtS(n-fOP4=lssk_hG$W$h#kV;e_(Vk-V;abOvBI`0z<bb96^+b+)CuF`--%a%;Xp z?`W^l??JYpF*&Xsr-RhdTCIpt{ZzLIzXqVl-$YgUO-vXWl08o`vLPEaazX#nq`Ww^ zwaaRVI@TXNTal{xJ3LUoL@C_cDmzP#4%5y@AlGh#*gvWIq+UDK<+m;CYdFAx1bMce zX;j?%Fq6JOq=-6e@zi=!76%!+3p5=xEO!Sz>shGfoRh~*($MK|uLS`cWmS8HrxZX4 zS7TR-0^ybSpw^f>4F5;9EgP}ybs@638JZ}NRxH&tjB#S1LxXF{R@4)%%|Gvho;NZt z_VYt>$%ULZjf{b0aV1~pw$bV?c=&Xd*1VND;7Nx1>Q~^|hfI;fjhRssg_L?!n{r7_ z-wPA_uF|Fv(7!<|jN1Ln27!iQg>^_lZX8ygIL=Q`*o8mWi2q9A3@NH69nd%Fgf2}C z|2{%fuqeo3n$?F?7~@IA%Lk1BiE(WYAvjB(F@sQju%0<;#b#S)aSiuFW^)jgiw{1~ zfaIZ4BAB>g$2v!<KaBDYsC2E8J`Kg-1fCMDH>ORTzxEJ6y*xH%G3#g*N<fbFoO<w% z>dR{lh2u0~G)tbn)I_qmNpzcHCTh(ow{Vw<1su<6JNckL1A}dQ{jqAmGGeP`@pt}S z#+6I#(0^lUqg2L9Ec6wlO=LwXbmW6RKmLbPUyj|G`xF7J{<q8)%KO|6@w1c_-Ws!Y zk*<cm{(q~Hd#hRDq8)<VQWt8!3?nK;KeBU)o{27q4$~6=14|2U$8~W<lPi-^zm<zr zVi}7E_)>O`?XBMu^3&QZz%Rwo_H*iPC$m4U65!a__s>rAnsh1|OiEO(^AY{Qes9d? z-&wHSHS~0TXRT?pl%~|6<WEV3bmy-=wcc->_mkk>JQwohL!MRZDYjbQ&l<74{$WpI zO5^uh&=vv~@^<c0JaV!P32hzM&68&DV8;bKXJC}S->l@=__^-iUZL)9KMI6=(6+b4 zv4K2eTdog1P>zw47h<a>Y_b7A()xnfe+F<T>*eYxEV^3Gy?sNSh65$TCEe`mdh7RU z4F`EP&YVWOkp0iZ7{NZ2x;VXp?!O?^D*mOnH*#sro&oynpBc4czjh_OM&Xj<?gFhp z!R(LfIoV*<Wj?VMlzWkyOD23j?nb@6yb5Nc@|@R-X@KRdb^E5G&0_UiJb2Dhl}d03 zq}@TI`m{pn-D$(TrKb){BTJg4o*?@r0aW39pRN4~a;WP=kPvO<jv4sm^BshBmK&-@ zeWjymNej*B;d_Jw$!Hd29wnRJl$GKy;t*i3CKR;q)xag7Lv9kbsKu6+*oTKM5tlmJ zydL!<OdEiJqqflfbk#B+fL?c_XP&>k`i7^r!aZTlTd7+m^fw6iN}E#T(JvM9+WE^k zC4~A+CK`*auFDTUK-a3ej_c6?X2V+NXT@Il;y)P!wCW!67Fh687z3XRK;UvGR;SnQ zN_w}MO7W=?D?0Ql_upvV(M6oCsZwnD$mVK}{^aRBW+lL*t^~?D5Ba%<`dSOy4ygep zA==fj{{Tmv;#jK-#r%aD*2n$J(X$CQn*#F>*h1Dz=V6{lk3mp=BNdds4}_VgHd*;2 zN7x0#29RX5;~6P9HvPzc<1YK^o#?KAgD^ca(VSyJCzfFuqPq8a#w;ZXp!hh_lSc4c z2?K#LV5XTS<;vkLrT(Gwv;0D+37Jq79`uWc${T^I3#tt7oSn{(S%mv6!39<%I_ud` zVob+}wgJw@l(<}Q6X9aL#xm@vdOJ2^|BE+i(3pJ&99}}}n}YaCyAf0VZ5r8`(1YL} zs4S_Mimk|Cx^FCkzR#DG0L(d7M__qQ?@gz_9Oe|{XHE|e+z)sXWC)8t9?eW7BZmCm zrUARLO<z?TOTjPP{AUAKJu`(z#c31W(<LUdr8G)_7rbi40%t-j(?$tO>(p)<+f9{b zkAJSylt9iXERgNB@`rfBy7TJ8-44>3Qafpa2ahz<7VJ#JQq2uDd<*RN0?q!4^V}Ef zs0VpH(WG34=sG=J6rv59lWqvAbtj_?XUFr@OCkMMx%F0uqe?usay>uHe&`A0`#ksA z7DbHAWLRSvulFIm35Di#L-PhM>h)7XPlG(w)1k>xc)6f<`Vl{&(tCzO4VB0_r8PRm z=|KbWe*(t0Ph{U%|Fn%@<UEhRF{8+*VJ>rM=@YWQ|69u1Ei?B18a2BU7c#d8$g2~y zCx5Ra{LDK^et!NXhV+af(9z1ZtkE|QJFRlQXhx+ISE>)PPkNia)RakdqOL8vwYiG< z-^Y%`RO#G*&v5u8djB?+11N4AZuDJ-P$`qGM3%=6ynyqU)oDJ;#;f!vgJLo2$Quw% zCrcm`R~%+=93DaHKc83*rbd{hMmGERPl0NSHDzU%n=m(>ax}p*0?W}&w-~{d*CCas zDtw!l^9TKZz&d)lk5~=@4jNAb1dau?)nB!$2yVC3L#G<8m&aU1xx1e^NY~>PGvcIZ zFM~sSfB{N352_A}iPMH^<D&HskBZisFiwk=$73}8J&UWVopp7{5EUcZOzO!$=5r{- z^ti)11~el>UCHK<FWTqj@z6RGcIZYoJ&smCR0MZVZmp0{xYwm3j)yd+Wp%|)6J!TU zi1gCs8-;#>aq#=Cz#MuSVf8k(=(UqAwDkCQH^AyULcAhH#9x#eNrP+ThTodSD>{9( zcGXN_OCn#F)%thsz{p4THhL(TgxV@&`mFU$fHYv2ATyYF=&@uHzt?zD(5YUWivQi< zOk*#lsA$&92m?-L&Wu&38hl+p!OVL<#AMfHr7qBO!!}3%oR}kS$zuc$y!LZ4E%dh< zuPC#mBq1Lu?C}3Ts^z)vq>ZPHlBqu=-?pLLzIph05y8nXN8N<L#9d9lJEdsJYUcN% zg}6O*)91aQf>F#ny)3=RxT-a&5=rnuTX#b3ByQ~6{_kQ??TBo%t3N_VbYFB^w0mhh zY6_nKXJ`Bn4z;L!xPX3M5?B2$M|D8S=e4bJIpQ+1JIbi3yQZ)6nkt|Fi|<Ryj1Y2d zg}P)4cyDF<!Iabw7C-iz;Iu}4+;<^82X^;>0PHS%Hqk8@b_>A>Xs`RDfNAyxb5wl& zN8h1u&vPa1R?;uO2i5ku{Ap~?C<^Na(~)#on_G8fO7HoPjVvt|z)YpUMSb{;mcA)` z<YkzGQ_qrwPv?P#?eyajj!&@ej_!o|ygI`=+;@iTtZ4DaiC~&|*xX7ZCtD2RrGK{u zA^<>~co+hxb7Jy!s!P(sz0>M)1{moVB<GF2ught3WWauI>=>di(yIHC?WPPc^%hT{ z3d<uPp!6crKtQzxTSNFJwuFE#4Z6K)iDhs_<pPA@O%tLg37SqO)E>)E^c^sSzgHCQ zxpXV}Zjg5TsNR`P<*QcW6n+9YXZ?rgGT{prEIRolvtDEM6&w2;Anxo&KicLKlprc| zcun@pLU?cRSsCwCgC0tj75+x{oew=vlfVLBGO&$>k_rlyVe>yMxRxG;c1$Q)@4;+V z#VsSgu!Fxx6W^P5!>vEchq=`aUQnmbm?#iz$p+`2!03sN^}9?PUiwY?Vygt?iK0xz zo)JcH`wrU5fLxALelFn4#Z!wkH4n4|Xt^A`a0o#?JcBu7%j)1amj4iNvjh%)YV<qk zdj!c>o><cr;9(Qoj+YH!X4{FIXZ}SH7C`(%-l|+vFGLz5k`e(~H-du3&)q@GW$&07 zzeGb{B|1E3VF+09gh)SmvJl`Dh7j*}i`ahHI@tH8tS}Q*XU5%(t1ojNH}2+xk=AsK z=y;ZKB!Tf~HBuoS#CwjN+Rm<c0$k*wNlLBbk$Ds~sL@tzJ1IDTLnhCnCVp||2I6xC zpr4&`v&Bekc-mNH>EXF)PrW&IL=nlS*inHz?H?FWbb1*6UJPLe<p9l8P~QFVQfK+E zR&Jjai5)r2ekqYGQCg_I<lZw1oMp!@IoR}%PSzVaH>qwK(|0apF^C3Y1BrX-ZNBXG z0{{7}KEeG~Fkvb0hMH?};<M-$(s&u-n%V}>;Ki={0-Kg~DpHf(O`H<~C$`)W`SW!h zoJLosXSiT=abKJfq;AM<AER})VneNN`$}bI>gQ0)X52%CTzhS{_0GL0Q`34YPS_cy zFcY`fZG*p0niAta=)Sr1et2x%xv1}2I(XQVZR6ruy-hBg7al$1f${e(M^2(w3+G>p zb|rTn)aG_zUYA40jWyPt<FBfua4`E5fQ#PhrJd+cyi~Hg58!^{D7FYv!N9X0+16kR zb+v+AYS&1`CJ-1`gd@n}NDU0&mEHXyP9Bn&@csiP+5xXXn8jN@oQSTBM!XJjl%tr* z(ha|)($}MpN0Ssg+@!j8C=9a8A3C;w3mIgEMmYy$FdC~T8q26GR|b^R0^P6LJN}o| zzDUr7qNjQ&VHY!wDl{F_>hs|n<#d<8q-49Bv_5DP!A##5`+TAS5gajgAP7n~==t9U zu#ayoa|1`LA}S(43{xjkGQMx1^V3dSsWE38qF83-v+RyZ-hX~{%R3E!kX`=N#JefY z*V&@Q3P+({7ciW|I1Y4tGxn3)r<G8t9J1uRbGe$L$WxMRV}EH^-O*F&^Y@W$%9jG% zDn@;rY$<J*N=&vBHJ6@*1dO<KCs<D+o7YrO#npL-mc09P-qd%#ozJ8jL>}ij;1mnI zx(?Z)O(Rqe38@@h{y4f2u;xsS)fADroah123lAvJ71mfsT05F=Pi#Ec?-F$yeDOR& zWPv>3py!}IL7T9Z4ATeS-kpxj?JB1-P@6k4Bf42SV>gpFJLZIa`~g%Ljd`OFKNcqK z4<sMdY&z+E_}!FZ+)8Sb)pan`afU;4&8gchrvX!)LVq8yHmKyZycmG_yT12dM)}np z%g;Om$C?KUI8SE17P{mYE|@@HKyIFxo?909%Kx|eE$*k{$zbHBDNA;EjJx`F{dfWu zsPEI8ZD{+_n)j0xdjRh2bBGZ98ahr<;Kg;?Y$tUVm8#PJnOXKvJ(m~ZSzWeIjhw#{ zO8eXZg{CezzdU8itkZY;YtF1lYSc=d_{YLc!2rqX15DGqJo@WDr05`w{iY~Qbdto3 zAj!!#=8Kz63>EZfzj(nI6Mn&v&+B;|?41Z?wXj2UNd*piv)TN+i3A2^w;|SMI|}Uv zrv|~E!C>Lfjj{m&KfXsTl$b5lXUgzcyH`lGR<L#6{VoazxCtwY1%el>AFs%(o;|>V z@4I~cRCTdk(tY~A&+U5%?=2A~81rFiYs<JW7L|D88e=Up9IPz><Aliog!3AeYd`;u zU|@j@E)(wxFg_=b%wzJKo-c-%VDTB|3$e1U{gF~%god?~swql7Uc~gyiKz=7S_R4T zlK1PHbGz>$dJzs|o?0{Um&GG63;n5yL);r1mAr^&0jmZ)ya9%`%-^$rmJ6KQG!@Us zUZboi#GEqQjwd74Bi)~7Y%V3{0T{h5?B|EhZh<Vf|LxfN=D@y0xp2tN1E^$<ic>V? z2~HH4wx2bcEm<TRaLhR^;G=IO=j;Un)muy9pmA&<&1%>&RzGMw!OzK3AYYP4jdWF| zJ;8lYzXYxTaW>0|l3elM7tc6J!EU^gkK6XT?NT`&gVXJ=16b-8Smd%M&LnJ}eaQot zQ3XNCD$rMK;4h`&?rbs8v5Hi<7GNrRwQy(2K<hRf3aie`jJ9v!@G^?fvFgBbmP(>0 zo_jsohh+^J^Q&|Z@`HWiUq+I&1Rc6-0w~H!`XOh@pw+5&my7qiICA|`V&No^Q`C8v zJftf<>(Sr=5^5OsH)tkZSt$X<WKXhj(tUy5iy>WA4>2m05{KRBN(|*vvrl!f{rmXr zo;S&rP~|-n7vBB0<I;(WOu*|&jg58pIbZW!@E^DpN3reR5@hG$`I248f}w!9-5!_P zL2(mH>rA7m#I_ohIk*1q@x<N3wTt`T@mo&IL!t;ye;KiqleFz=D+o?lxA3UMlryvT zp^M4banbe&aBDrdct59YT^LBs0ernlSmvS-b3F+LN;DT9EJzc-o0B%UadW>g5Bw#O zB8T|ubPv*wX8tBm%|7ZHUV52RnnaIwNayRlCDok0rOPLYd`0v&?d;EGS&pq;FV6-i zcjE5Q#e*A(Y|cw@^JE#BxpgHHn-idmBO-R5q(&R&lyN(GjC0Zy>#nzgaJP4*lALU| zxU}L8mL`>N=WTrCD}wh~3K&{JfDY?ySjx?9>~41EX5HaMCq?q-H?n&1_;;77aWowP z$qb#|irLpQ<5?oZ8x?#6VUnYu&f)iv5Hs4SlnnFdH=(u$pDJoej9cOYkXW>ykMn@L zkLsu%-|F{7%pZ^|2np=&mYa78#A7$lLUUgG<_0#+6)ey6Xi>C%ZM|S<h-YqWFEJ!0 zfZ`BxT4o-cI80W!w|MXlnpV{v>-(W<h%rqDkfY5C!$?8?;@PN)ZxH;f^JK3Tl^!wp zW=Nw*N&0hROY`t<h)Vnp@mEi>NuTnN{@JZ>;g$ZM?~Z=T+l<xUFAxA6;q^M0<4A#0 zWO|Fy=c>@pWi(A{5!6P7UcG?UYMhQeJdKU0*K;m78h{Fa{mSp;(7rAP?JBl5=03VF z`XS82d1lVgjDa+r=<Xt;x`#T!t3l!c$sI;2xZNaCq$AqRm5EtmN$2b!y~%(*P6kc1 zn`G?&XpFbq8e|f*&{^PNnzssGPCpH}pMdv2>35b0?4L<8ou1R68apWbppPXL0GN@J zO(Ey1pO|F_$U{A4W;nDqJbws!iOKdj1hBxi1V!=aP|l)=`75L1Owfh|-QLx1k8OFG zp;~!FHbM<0o;M*3TzMQ3C?7LcLN=P8uo3uGYV8<?!rK6zMf)8_fJ^3Mx%|Z;Dd9if z%nyC9YS>8LoGE!-i{5Js&8q0NTnKm0U8{IXK5M0{V;gEnHYB!PB8LRc4R~O8ly&Sg zZF>C^0-#oHOfp@fyoP2AV(Gap1}K{Vz5VIGHS9a>&AAq^u(*%;eCM_RJ<RCVkNqpv zo6RM4ro9kGPg~OyAxrPw6=VS+gI9|BR;7GG23*HHS$!-#e}R^tLq$K1VS-L^BTj!C zfl>x1yF{j3rg$pY8OH8fwu1pSd(*d`=dG<)RX(6f{!vtS_5QDh>_JiXPlRf3r*w@K zk2kJMUi%=wDoy&mhnWUuh^Bh+nA1Bq?K9t%)W0k=?qGNB|NI)QY|+sz6=7|NC*`P1 zvTMJ3_q4W|#qYa6pi2Eao_%e>u))3Ver-~E=21@~z;;579_NoYob#Pok}toh)o>pD z63wp5Zc4b#<PvMj3UE{lxyO_Lfejw+<ml{}VN@z7Su{QUO@HfT_OQs;6i3hWeLRT} z;nr|wanp%|<8*TF*`D9=?z(FGK=V#&_kd%Zeuop|#**iD*V}u#!spiYf@V?=(ks-r zAM^C`d><Rn1ll(m-NoQ;ZwqUV-8|_L(%j$epp1&~#0fUwB)2_8xN+b70VOAJAh@WO zu^p^EW;E!s+@nnjqSsyj(^^YP6N9X0TQg4Q@P`?k-Uy1-nuzSI7lWp0osS9ne$@wi zIEx@ss}C}R1~d_@@H1BGQkbt>VfteC_0I8h@0m;Xl9d~xKIjQ3xG2t%3PbO3LD9c% z^J}N!!Y3nUKKEKTUK@SA;vVrxEL7Z<d;7j_VUR=yk^BpMDb^ef+dh<W623HTs89<u z(*}*F+{9eqks|iPZftQm#obW%{BGS&7Dq@sCGe4-=8|Ewe#3KTXJwhJy-tcd5+mRx z6P2#KEP+@8xN9qn0ac;qowt1`^on*q?~-}@x|DfK9AlKS*7R+Kgwcobo1XJLar(}> zW(x0HbMUupXJTt76cL@od}lLHPf)lBFp$4T@EgZc+|Umh9<lIhrbFJ-xWMHEzcCD0 z^%Y0Bx7|YpVw@btZ-w5(j7e$hY5<MzZVHe*G4g10ghhm}&=oL-0vr@HjZ=NUpOf+- zP9Jq4P97bIHRf1aR3t~claz8<c?h3?%nWCx3t%_tkJ4aKz{vwDDbOi}w)F{X0j2Y* z{`M3EKj4~?nY}APZ!`QN?H|T%?*E64fdWB?G#cj^6qqQr?wA9J7D<p`fMtzGtf$rk zx8ruZZ?{tH&-R#*-2R7BXX1bcDl>eHAW`qNZL~1+CXB*@644|ZL<mbKN%pfu2$~R^ zy}Wd<!z5&-b2;Upo;-rdU7w+1vf989_pmV-<hZM$Wz2NTCGQTuzuzN7CScp{`g@60 zm$~AI8TnBG9%nj1$oW=oJr5u~hUS>V;c1>{SxwnsXpj(zh9=pM>B|fBa^3?v|7A<y zA&m<%$mi`{JAK#XhviJRj3SM!trY)7;q;9En`qKmuI`TqzcpZT9e)#Xrir#cU`G=z zo1w&X-<)bfd3fN^z@K23yjQ&JK+4U-V5HBX9woxx5^@v4nL5T)Cfu|Wk(cem<=Mq| zzUs<L7OB=&Aedub4=FE|Qd|~RY?c#L230ReDRUKAn5-LjTjD=PX$(Gu-#J@X6+pG4 zB@zzaUWMo=w5)*~dX1>+$dDv-uh~ppZ3805?=tjQ9oP?4QIS<A4EF@M2+MZsZxYNB z1L=4&ZH2gN*gBR)u0^?BUWkaMeD#{^-yZ8<M4fbI_mom*!dX4i18daDh93q?yMO`a zJi+Gt_>ye3JJRc~=6)gCiXLdWRB=AD3bM=yj9s&LHt<hQqHc6zIJx5VHAi-{ls+e; zolJGbJg-CQpkiWbtNAx6xBca|!}vi=Fj9>eUGCNiO&gkC>o0`dp>FFmVKet5@Kg%R z=Cy-M)X3NRgF>R?ov%P^^nUHJ5}d&TY250dkfy&_9ZsE1F7*5pP{$zLne3j{7^r)M z>`L=B4xxZF1;c-SfNJOp3&cu=mWvsdc0RN68id7Dz9cP$OEYciL%<IGyk5$)>GNbO zaN>b@OY~KZ>#RL=B^2YCD0EC_r*;H5EA^YC=F9vW2y;EJ)kI-oiM>(z$W3@Lt1Fuq z@1={_09=;RHK{ZWHGNBGvRyhvH@0v82Mxh)L$7+cu)n28>Ah_CK@Tx>YNB3FXPr3r zVR2?(`@8shb}i9fQF8wvB1QDGO596F7nG~zqOMH``xikINrmM5>`Cl5P;o6by=2>* zP(V}^{7=N;Rd(0%#7>>95H}HS|AJ`vx|i^nwk2-M>!Ch~Jj40m(`VS_X1$E=4869? z4zc=-amnuWKeOWWVLs5u(pqDlx@Q$T<ec{{<%REHL#(Uhd3<2_J-BB;5n_5V&Anc{ z_ES-5_owb`Q$fi4jxQIKk=Ii6JGQ#3cQzw_%$NL$JV9uHp)QU!G_VXtlF;?1?d)_z z_JhU1BW9+v03Bxd*4O7?(S)w|p|L0`$)Y;T%8KUd-!j^#Ju1o`A#M=Ac3=lCt6q3i zj69>iJrbElIa^;p*A#3y2$IU}zNPjE^swCg^G2y@EnPK4SA<&RSvTHqFucLSuypgT zp!w=Mca!je#03>!5lszHeq|Lnv_SUQje4SSiO&{h%=i;5Qdc!@cs8hcR_6687`kSR z-is$uZcgb4A+L{Sp$qv4oAix+r`U~%Rm%$EqgFHOdVWH6vp|8x!i-b+P_z@QBV5FR zRJvrW_P0*I8wV;08`@_u-yO?2vWRL|T|c!v0w36oFTU4-K8rwqzU#C6c)`iW{@@xQ zh>5I5cdGNBn$Cc@6<A23^5HxP7nNN1x*=sz%=$<BnX4&E^XtFpaS@9Gnq95Sru_>I zm~cDH1q-8fI*bLL<d{$Q>nCvTwbEc6xT2WpZyFurrG3Jsd1(^(bwSPU$d3Dvcbb-S zTx#`V5I4;Gws>tU^S|IPHQ_zc0c-wLWesBXOP;4H4Fj~TBAJmzt}rGcZZcGq+kTv; zQ8P@ytEk3GP%mhl+(Y!wL?LPTi@r70GxzL&=<(-06QlkG>84kX!L`}7D^=$91`4n0 zNQl|-HV-3h1!6^Qx1Cv>Fq{5`e)acx517dK?0(ue;F66khv!y0N>AX9!?f(dEq{*f zfA~gw8^gVY6_M8_pvB7(%O7Yuh`)8l+&zg?9VErrhPJFE8OPX2*=YMzWXFKU!L#kh zwomx^NGf=zr4Ri08Oq)L)FDq)Rl?Y&80qekfxvBnz<x1~2C;2z#A6VoJ-jPBNK(e1 z)pdhoLZ^Mz(ZR`tvFxrUy)Mh6kyboU&{`kDZ>=aGJD%_r8=VUy2SW~W(3Lk)h2&J4 zDfPgQ;e5kq!<6X~0(2;YVFEnKs}D%y0B%+c@@a@^+8rxttqX4`qV2bHe1^(a3KuQ0 zIZ$>GA<Z08+o_p5AAh`YdN@QN1d~6<1o^06&3<*(H}^i`2g^p%xJGy}ufO9Ft5%=V z!Tw-7wJHH4yByd-DDYXb+V@nK?--^|#IoJtu9F^LM5S`(Go9|2ae{==*MEsQz-63F zsrKgQAJ7HTdCBHxS>=tgDq?nrxUpU!)ajwd$ptRqopmAL_V?F!pz2~l>YFbny|QN? zX-xq&e$OKm)FSwBUlNio>_rJYS_*5~Pj2qtkJb13s@Rh46Z<E&?pPi9M4z}H)+>n( z(?_@&V~xZjLSFeVMKMkHFV6hgE|+Kt3>=Fl0r81_RyO|egLEEczcZU^bLpbo)TmP3 zMPMJ&+ew?BfK<m@t6#63^0h7)&HDL+pYC>5@9H&Wq$_q2cgMN=WaB9{4P7)8W;Q=) zK{wuuFSc(jVR}f(wxrlbRGNBLP;PGzp5Wg}qn0$othH<r_*J>>fq~ZLPFH4s`o@jf zU*c%>d!^FO>mD_J&#?Z#Pl-iLNSN=c2bNaCpw(w=w^!I4-@wVUSD3{N#QV=^J5WA& z4IKFoK2mCIFildH8Evxt*+@dQYGQ|`t$y>dmTv!sb#bZ(C%Y^7lC<<`i|3~iJ!U3w z50jfxz2KBA@!7qhQv9&$W~KAF<)ctGpjboOHfYT2;#|S73rT?XOCY1WFs@kd!{wO1 zay@;(l+=CGC7pypbp6lI(+a=P*2zG#TQ=4&cwY}|_ZH1-f4MaCEogVW3n4Aq#hxiT zz@EYW(MH|Pxi^aIO)^_J!KW_w#tKN*@vYC2v^fgw<o)N%{FhH;s?&9Pa-uHqTqAyd z>5DDOlR6tLDc8&BYY%PmmVN0wg^ARQdKX*kw=|#&<lfMkd0}Y%$(px2)BpOV{`%Ke z(ygDXI=#AxFgYp719D5wKZ6Kdzp;koC)}WuFhU!gAc^|CzcHH>%PzfSU<1fq$L^dt z?hkOVdu-*LUo;eh$*<a;xd{)rC3JVtdqTp8K?ISAeOG_C1sCqGW=sE4byhZzKVY3J zUl&QGnK$%1*Xmy3PHt5IR_z$$z%2S=AwmTwF;GA;{&LvIEiiYl#L?811FhiN{4Oo9 zf%y?jA!!tEH+A#i`72^+rl~oU6-fA2J@$h4z-Xc4xd4wJdpqG5*LKVeqCQmf8DJ{X zwCm4#a+Y^^fhKzgA=T7?Vl38ippJBleD9t247mD=QSiW^H4bvr@U^@K5%@t69#pA? z`jrCJ<yv~E_ESxjE;_m%8dZsUQ;r?yKv@DJNuZhcrnT{bg0D8RQg0K)n5c!;=w|i; z3!D=$7uV+&ZF&izu@mP>8!I9Gxf%X*#!!J6(?QR*O9i=xdD-X0+*-0iz)0}z?#L4? zVs|FjRPM0;tIz`J9RWImPXHy-A+TviGKgsKu#d;UlI?lVFnyml0iCwvUA627ES*V5 z<DTZm>yW_+&g*|ZiU3#vgT_nO2kLbwp9@>|8WtO#`7DCQZ5Kb9*5=CL^akBGF^sM- z$CfoOZyE<rh<<Ldu|R>H43#r@NlUf7Q9<y=dd&`9m{cfW+g!5%?qLTPKndj$B`|bV z)oRh5F!#jpwLH|n37BBl*JkZ2Vha;M74l$V%X%hGTg7P-f?B?OC2-834~Xa+Py?^$ zHEUz8yNXrU>v$Q?K1-16`a5hihVO@%Cac&Cf<$o6TjD=OS<nPB{BTy#8HDmgXIyjJ z+=GY^qB-ElT9ZA)w;mPT#|50VEPq}8YgbF!)RhSXq@bo#rv2i-&G{l?Z|RAw<ZV(q zE<oW5%xj;8=K$5T?0+-?e6Q!5oEdv!00m@pzX22?CuF9-q&GtEK`IrN4BA|BhK$0t z?;_>XnD~mO+;qOhvh2quu_Hp>7@gnKm+v%;s1cBztT3Rhxd0kTjO6gL@HCIgluSSh zhhmz1qcC$OMZ!74EW5eJM9luj_nfbGHnSN9nvR3Rl<uTNLJiejX{(EYYib3-p|`#g z64=M9f^>U|Oll&!DjoY5%w!cTE(b%dv~nW+-e-GzSP4Rl<Q-g~x;-dUm;fDKutwtP ze|d4WF56mGv)5LWke<m{WfQ+fk${NWuOw?OM626=Vgtr9U`O=?q~OrUd>i>@lrQmK z7<MXG7|&_^ul^}3LPTmzXO-ZI;8AMK^wIGg5PrXA6uRQiJStvNxv!nnT)n6m6f4_N zB9DkO4pfXV?HrKqWs~`e`kTko03)M1rjvte^_ANHeoO`yS0k9jW9$zrR21)_2jOyV zMujw53G7hVKJAWxr?qn@z?eg}h~9arg^7Lr#K6@pPeP!j%-G=(3xdcj#wCE$-pUf# zOAc)Yu)+yO=79=5lYlSb!-@cl4%vWtZqd!aLo0NXJTl3^L18cyo^U30b3<mKI~&^f zdIj1m?0W}JlR1#wIB@#)DH1J#Ps`>AlRvj$0{qq0dE_ST=pg82_#`xo=U~ka`&}rE zkN!43_8ufl0QTR;b*YC@=C5fTI)e_^EH|xXcw-*|+<d&A3-oZ{FH!CWjdy4hvxT!g zZJclOAAVi3<ENNV--dLkpNySjZk=M7A#u>7-=M9tG^!ZdO>Oz#L%{sT!`<0AJYe9g zw@pK)v)_v6r<H3TGH*sAmM8F7E-HYYd&}$W4t9t~-LzzA;Y7I2db4pI7%S|s%Lz#T z58h<*@21rhQ-J*4;l4yJLOS;IgWd#ie-!gxxY@jQmUY^Tq-<r|s~>m~>!8%pZ<vCy z0*qp602_#HJrF%ew{cfY&Zse}sx=weaJ=9=>LY8&fe>YXN?H?MwSN6#Ze*%pAxaa; z-{*1VaKPzF<%>BZPs@}%BGz^m_tGNaUDl1ieN7DlGeCr&EtlWd6R)mx-wYsz0m{<K zZuU2veE#)j&UYTq#N)bb+pb|QfM8)~0eo?b81BV$v|8VaF*yV;Xu4>H>H=3QN&#CP z-Gay&aYy;kX@$C9{X2kbUNiU`Auu+?O?JRp5mfN{k3_RGsa(C*`T!W*-d{%6M2Uxi z)H}OQP#{!7*|#<q(7ay_0&1t%fQKxMM7A1sWkOF!p*c9gIP-F+C;K-0gdm6iHv=lT zeAJx3KWCZGj!nxc&rHx3=NaxiEgIVtpoPM+_ZM`cF9>hkp!kQ@-JPFXv)r$apEm2* z%7>CBptX1Jd3&+<Em8wv<R6iVT@TJ!kgyP_vu$AdarmRrr8)ny&NVjJCO~Ap4})XQ z+QohhHsg2|aW@hQHjY3)$IeBDJ565EcKFv2_`y^eF-ZE!==BiTeY_L*>=0ym<ZQVb z_iWWj26G~hcyQ)YXo+xA3Oy*y8f0Ykr#?_U95on_FSW(cVjXZNg`F*^5WqHTq@cF5 z667(Uws>G8$tH#+BrHecYaL-213;!>cSz%6VwkbKv0&ZZoNuYoRV3G$54GCU#LU-+ z@!=aAKF8~+O3e)n%NLq_bVOClYH0Rk6e4`}s~EuEE;SFUA)I%ZTuuK3tr3fefoehh zH~b{F0SXT6R9mOs^@mpPcgXhqN!fx&;|zm9wDqLeDZ*@{r>GAmEl88!iRD1`NrMOU z33nHsaJ&?|6qhw7HXHRsETZmj7yl=kJJ~u#9njiD)m7J9uC+!Oztwe~m71G?6^+83 ztx9-VBIx9KXooW>ur#bcD5Aa)8Uv}z&l`8~AKYQs?~W#pMc`LiwsmaTT5(Z{^CU(U zN8$M)$BX!A)+2Z#T$plkpZ^qARbL}3-ED+<xZRbN)Qs?KmabCqaOm&5@_KCDs-c^i z_e@;6TljqJA<xuJyTqD@XRh-0@$QiuyK@}!GanU}9$bF~tKJanx}S>)UoV1xN?hyd zuo5WWj(xc80L(_oi)6pPwK%q3_hTp!D-i=ZRO}yjlj}*3)(rom4;D)tOoxvfUF=r~ zev;8Lnp^V&$p0}Ryj0Uj7#L*YK(govEvDf$2!$Qs^rJzB>`qTxiZ1p;otrvm{yZ1m zO)KPcCQ)Z$mE^c&9MV5OJ8K&t=P>hSEpmM&nEDZgfTu|vt;oR7bc8Bk%ITS!@uq%@ z*zi0nZdCF<U|BYTJ8>dgnsOha(8=dR`|cBv>Co2eZ1!Cr{1vzqZc;dx6D{Ykmq{6Y zv#3#VUM#N-mvJ@-v<yZkn_V~L5a>0J=kZ@J;YIr<=CV#X%^eU&&r7I%J?4-q^)3z` z;*BSgGPHPz-;Do9a{!(vlU0Y945SS`z=sDUO`asHy0<#T%*V?*-3BJVG$6y;+qOy} zoHFSga@~vleQ^Yw))%abhKGHn4D802I<D^xGTZ7@Mt5NktVe0Ud-Gz~#+<Ia<t~3> zktaoNs`%jaswT*DKU*<-jjVKFjn3J8ZT-0_93aU3=0^9u*0{v*z&~&?>qFsSUd!os zi5u06qYNz=hDo)&feqh4lk2ZBd5?N+Xl_cVBJ8x<OyliP91tPvvu3b8()LjObplX6 z_+x~Foh7<CM}tSUT9V4z`TlOpL34oNPMdCi=dESJr|R%s2ZuCPs~LFKa9c{kn0pvl zV>S@+_`h=V{~IYF%>8B_w^%f;8>n?bUsV$ucQw@#dy>rt8FQ7N&^CCHx{)NuE=n^~ zg_(!Au#|7*l=R~dD`VCqOH8YiU(q+}m=MI2OD4UV*Vw0QZ&%tGc2`Wy(xlpS!9TPo z(8uP5U9m&Je%QiXs*f44nM18y4RK;*#dtH2pTpRwg0}^ykjjypzN<oh@u>xbDz;;M zF|~T9$`jWczp#~a!)jj~4suf)jvcB+RV0w(AB*nwB@Pp^+Une&1_7#yN@H4j*sJNR z+F}~J(gJtGhJ<<0GvLP3`HGI%5wj2J>2u0WS4^JZh3T#fhN0LCJPCm_OnKN6{Q&IJ zeS|AcxetH!6n@Bn?McAxv9S@0E+Y2ZH%}Wh9_O7|avT-`+5J6`)u_T#aF<q@#C%<x z=F~(Wwp$YyyMOE+4CPvu;dwG9VQ#-sKTH?zviP!>1NzhME}W<%BI1fke9rv+=J6#7 zq<;f>ScxmC^;9xVRooJKSEpUjwbwucL(>~99#S3pxO>%CvOVlf)BB%t#?zJ(6CT^G z(s*z{)OzYdV2}}aP>I!&4S)YXUZk)jy=1r+b#`IlJe;V|QbaAvjfWs8Emy+30fa}$ zw8Aq|1o0o_Hq(30J$2iCS0`J(X<<Yk)XmN@6U=#spJ)f@a%Nf`R{=!H${ap^zJ<3z zH;`TH2JuZp_YpZ)Wf7(-m(yvtj~@H1-i5*2?+f45zc8l(I4}t?LncFk5KNPm1Lj^t zDde-yl1#{m#8J*^JE@EyuB8d5;)JR1Oz*VJ{&<AUk;9;e5(wsr^^PGW@R6?#T`W9_ z5VYscU1xrI^~?^#?0wmyP8Sa7{@XN^+VW+!G9*1_X_H5#D0dv#u;JZIo0vM&{ImM6 z>z6jZZ}><TnRJQ`rzUu}cKqqDV?;~*+et?3p}{O!W7~MH7RAZA#s*!p+Q2{smzz(H zm)^<x>~t3SN^J{Kdz8&N*Z6ioH^pdU$T${MX<NNzNJ|E>Xn+%JmUBM(2{Nc!tYv5t z7ZFn7IrCvLJ>7C(474D4qqUsJeuZrw6NK#0Im7!8IWMxcrxy>;+zBi3$vS)<-HdEO zfxu}BT(ETKgBiLSAXER;QFv;+ro_HmY^*l9?a0=)$efd^He5HCi41r4)LPvDImP>o z@Ais0mk{Oi68u#>=uCpP9!*t9UOlbEmU15@=%C+O=SvW^r_53DF2YY52W9Z5Ny<+q z=VTIOMLSNO<zh6`Arp!R7mp(=Q`hfaS(h>ecR{OrFnLC^PCmWi3LQm^T8}cxj|lo= zhlaF5O|Zrk({DnFEs=H6tjH!Y!GCkGhc&g|kjD^_JKz|W#Pk{pX6rZA=n0b3o@a|( zvT81~nhBrL!yAH&swgUrYlHq>Swa1cP9LC~Gs&KW2fMOJE{rltaM#yVUug;-1eH^k z9Q|bCQAe`~-V1XI`~jyqdD->!Fi<4DzSb3b^^o%vmSSt<1bQK>%{{?5|GpN%P4{Cw zIhs;ZlhO%lXeTKx&t%uE8<>0akey~yJZD(MEg`G9`>`4;Uf}7P#CbX<Y3KieN}_As zjD9t3)Cv^!iA>9XRqi_oBMuze@r=W<_?FJPec8+#JA5jBE7NP9e(#5PAXg1!|2`F~ zku+PYr7EyAzS;89VgpT?p0Q-IKRb9EHcUs2OsXB@DOJX;W?I&u&6V^u&^*3&ay5+k zN#}_-G^kB$jW5^FD>sjfNTe&T?|2+6d?VeqJqc^;1lk+S()mbhb&>ZsQPrCMkSPs$ zg6d{Y=+M-$4nc0pUvi+Gy0dF#MzWrAn`+2MITX@)*(~ASTYDLw(XUV~NZ`$}K=jv# ztJUNFY`it`f$05lr}-;Q9p4ujHe`W_bB9b#4}A@TcK?VD{%OfQKwV=B`6X`+o-*hv zwul0L*e2{o?g&Ih_6r>3?XEVf;F)T>>211Z$X%B|3#7!9;Y<#O4|!{G56QQ6m~ZFV z92>0!Vh2@y{Jc8V-!9`^hh;pvK65;18+7&tWXNXvC-M8M94sy$Y*s(4hNF^6fTkQy zS!nEL^W4i^Z+{yj`9;Lx{H2~SGZiOrr!s?=9P(P#S>k5{OcL`spdEAy*M!5@PQQYK zJsPaM=IS=bDhDFWGC8xHgE~5m;#-iWL}pq44;G~>b!Igy#R>i)C?jH=cVQJKbc1I_ zaaJiQ(Qh(ck>IZ$hCM_M@OY^ZG}ja-LDJ5Hp6FFNkKok*!?b&w)-2_tLbKKsC{@9q zt8CnEDzI1n;4skRXkc7Jb5)|HDlAAJe1B7GCPdntLfuj-#mOY@<01pH53w4u8?IJJ zd%Z8>AhXWuoBMHXZjhd{RBwCoCJ8Y<bZ85@7%`rRJ@Sk+DjYIyL_*%02AFhgjtvGW z+i)s9>A~R{9P8&}ZEB{&I@WwvOQZ&|a1T5u*Fo;G`s$X|PRklodF0}1=VF@(*^rHI zYBkjwSUP}aDHwHFL0{ciivC~ldz@>5uhng{Hu&0skmm8PRBgb!g9x{q^0*Z--NMG@ zgEBL0*!ZpNxT~_sc}NK}{&Rt_V?I~=XVd5BNFUZh%-_y&vT@vS3~OtFkW*#>Nm|HW z^{v9gOGQe0@5i_vJbwQ)+38pEJ$v1vYeE|CKarKpFI|11Z9z~T8H5^Eo<kZd55m_? zh1O&@gOHFQwY5ClZryR2lKYIQZTV@Al047$CdJ)D(0qu(@3BncI(KPx<Gds`Y|zX; zB%`kDSSCF1VuC?pM!Ak}8*k*b$mn8AO+XE%#|YymswvHz++Dktd;@B4KoftYJVkmh zn{W&c-X?7lhqjM8*l;InANgOkyL-O%ohC*LM&0kcZ|%+g5~AgY?AQdcbr@G%GUgfL ztzVnte{yX+F%~oGuHmffd^JBpS0Uk!uEMauS>ea(O`->7xeih=q$SWWPoRxWDU}}F z-ZXgtQ9GVEtn+9c(RzhQq`3GBJO-TanXP%oMC`0!6@6OpzN(KqIr6rHP3$R^iGh+> zO7C2T2I6KfsA?@cNUBU1KnoFdu441rw)<fKdes`kT%y$>uo*vR2P;vUd8N9DjT#-z z+pg@t-yY}A3!jwNSBPe&80aorYtzRl7{#p3$xvqlK&Qykf4nM4`rqS^{{8nTUpYin zp`X6oe|{UhF2rq6co<Jtlts~4MwK&|M>&806Eg-LHgJy@loC+68u#-0?@!MPp<f04 zO)g<>ovT#g0$)kNg8w>`I`HW)WpTdK!IvR3wf}VvDBGwqcNHk!{rZ2#fr`q$&CS3N z9-Z#&J)6hG-i6AmS#AhC4Hu~KsXrW{wcie@4hg9~HBTo8=KN=n|2Z@}#tRkr66*70 z>1P_z65{S*rATwHp6>Su?DB_gp>{?M=x~0|%&){Il@ae5lQsa{cgdO%0Od7SN#Fd+ zAh5+Bx}HxA<^jo-2wg8`2vp@N<n<TLpPM;OHsHT|_*1@fne2UB_S0^J!*fAjk8A5Z zJVTF9731Hx_Vgd#Z8?H3KFIq&SM&G55l_EB|Lt38!t@Vi&Vv^r!NHqVjqG%qlvcoR zcv)VH;1`YKxp68nvvbS4Z6A~TZcg0rcp+?eySf?wVdpp4Cl#`*g=7KQiDznlGNG4I z&5mKfvx43<z{344<J%s`eJilqPZF!mbYqlg?SCE>&IbcN4*$TKD$PHwmLMGbHeLxn z7*fg%is+zBKON~fo(~2FujK{@F=?Ic(KQ2?U0atE)E@C@0bK=kd^)K|J9L^Hq`4YB z+w)Mv+`Dxf{q1R@<(>6|yDxP9^YqXU_uFIt$CCxrb54(+atQM}rZdz#Pw7SgmdZ!d z&aHIMtdEAQDG!=fQjA;ZJ`4@N$zD^xn`|XgK0WFM*#Gb<?E2H!ulsMS00!QF9t4v{ z4T2lz{08Kg&=HOGJzVUaP<^4hoAhx^^3iWnaUv6Jv1FU7eY_YE5&Zpz(yhKi{@LZ` z{NHrDR!eh@2DM*NY%Of&>Zl*`XOKDHo<I(V*N(?uE;6Y<Lwa#XA{#<H8XHaS49{it zLluYH62`rc)yj;)<|Zb$nzwmX$t$U?^25r7y>$;m)SAx4dpxoULEFZa$l!$#z3PVj z=<V^6irqZO5pMO&ulfc|zD5Dh-eW#BHg2X?#oP1!k4Rs!*~WQ*O&cY<W$)*oToLpJ zxD1*q7zvP~B0t@EVQ&!KeeL%=`F%0J4SqX6XMtOm{$Dd|#A;L=+esesdGBHi$_;F3 zn0!%TOLuJ<*tG?t^jH#BE|2cfNwH1gwBJ;}Elbe8DZp}IEpm3YWuN?6d5ns~LMOf2 z4=3m8&%TA5;v{5dc|zez;uPLFspY8$b$0*9pEicxbf3sl-iSTx??fBfi++ocV=1B0 z%5FywHV}|bwQ4z>w5|DglWG@1J#78${h;T~YLhTql8BCv^u2PR4DNif;5b|QsMY5* zxYetC){|-%JRdm!8+b1Vv*1e#v!de-Q>SvzU+ce!jA-nn-%9UgG_uimWtZ+mW|aJ$ z^0V!T%cS)v@NKf>R81_pjK7syIq$HnEMuR_q<Qjlr&l;p(hm*SiU%OSa}LJ5u0Ewh z4QpC9YdSmZT4$BLaA)4n?_Dc5wLh43{{NeL@Aa3y`<3ZV5eH-zx7!Ye=WRf_&sMma ziF9X=W?N^tV!x!QFI7bF_pFm++$B`hNZ0Wi89=l@^vg&a=?>e4?|0wHyQ94sFPhbd zm!Itt%s8u($r?SOvw!M)RW77WKW=oh`fZ@-dL@S@+~JimQxCtC%#JJ2C^t;b!&xu& z$!rKhbKi)|(9lDERo-uVpM?l@8b0{8K4vT<<yv)`-R*uP=G#QK2u#cNaSK&lc*%#D z5cL`(e2lzF$gpwC_8yRGlyn*g^?x*hvgZAUA{uuXi5R~x7q0s@D3qGvr-}|Ynu4OA zJuIIix<`M+r4e*-Qp+~84|oF!;rycLS3E|A()PAKtk<4xZ@wOT?ABuyL)-<KHu4Wt zTDT@#&E}qJ2Gc5LbAx8ez+!q00$r$1>`MO5t~I3!S9FxgV4Oo4H23XNgjG)(c`*ih zX4@TVG_~^^bWEUv*sYLCkyam4CoDBOR5OD99!B4tWijBjphRoBwvuK;t!{d(`vl^q zzdHA}9$Cr^SIk6sS(-1>jv8CP-&Csv*BqAHr&&Zi1w967){uE6Bes=@Wwe#=A(jIL zIm0T4l6lUt5(&$%!n7J1&h8Z+c+ex*uBfQ!qYj4Cm$sSzXDVrDas7}jip7+ThGJqd z%-ZMl5M{iMaIakY_v(Fm5i;FM`pF{%vq{mVYe7HXFa-Uku0J9Vy8nJ|TZaI%m9vg9 zeEnF(DZ9E^dbH^WT=+BU`>z;*1;}9qP^ylVD#Ak`CR3O!JVbxu5k3oE^`rXpq6qMl zBB5)A=FD(guZCyYqwZYK3g{b_qYb)!Lrr>T-*uvXi{>mHrP50G25nl2sc%^?fIZ@M z6*NiEqMdjnywG=hSh1QJ#&-gGnOxH+f8#rAk!LEj-=Y6)_=h*2=7=TNb@Npp2Pd>r zk7k2^BhitlsC%Qr0!M;=nRb4Ca*>JZPoK}m2?Tn1yl_gILnW%FRG+aV5Ymfo`OJDX zO7*126eg&a_X>KIt6vfk1V1mQa7;a^(oI@Uc6WpCbe_|Sq2(srTLN8~p~7wrJo`rf ze8_*kX~y;A&aX$kWDiUQ?Xjx}N-qj0ip=PG#w3!y`7bn|x}tNQoDiCZR()Kpb5o^g za7GQE<@|`79;vMQ7;LW8>GMqmO;@*3Qi-Yw9W**wv10JP$dnlU#^uh4B3E*Wlejk- zjT7f*8YT+Q`%0x(%s;pwp^wV@T5IpN1}Ugb2YJxw7B1pmw@p3V8NF<i^IEFMK6KS5 zs-`Ig{x%ZHKwo)d>R#@7C~ZDwAqrWdoAO26OFYVC3K^R!PYWJaD+O2h@)|6rF>QOb zb{JZxj1V5T^o#lo3wq^m*TaHHU*6L0F%hRqH(mbE!p5vGR^Y4f0?SF?-K-0+z@4u_ zZoy~%eQfI#cc*{r;oAc53i(gHH??D9%Z*E49Hp8Ft{Rq1SOMF6FL~X)@isC`n6H<A z|IC!Ou2mFAy(Z@yD;$xU<r1|#S!@|hUyOE&QqAryeMeE#1OiQ+lA_&gSc4OEmY5^x zRve&YZWr9$(wry6wbvrI1fLY=<KA3?El;Ym-E*sj&<>C?PANIHO!<$t$g`Ew#?X7? zzr8Yq?iiJ;3j0|@QWeMS!Km){uag_^?G4`YzwVUu(=S0Kn#cm{D&xvn`So6++PYuo z-(+=y7k%9ff7F%@H)!c=ZFv?!upswZ$d43h$jVv}%>aG265Mej8_dQtz}puaY7`n2 zuQm1GtGFfTiYX0G$?Vrgd-%vD*2`IDytXuV5kQ#|&;`kV2zME3hm6G3Glqy(A3c>x zc<8=SsEcS3?&ibF*58oDbGqGsFS8rSE+s1_MU5g=>~4@O7h=`@UI1U|IeVG2KIbvC zqKDkw1Q}E6L(McJ4c0FU17%V_5<7%$T9nQgi^1uSf6Y}jk58b$j|u7KyvBzFdPaP^ zzvdjVZyW^+qa%hgDVgRFy@)A+`bcMYHdkTc+*YDbYq(ty&UJO>W6f-)byW87j6=kP zCh=o?y613?ie5RPa2R-4T|H5Dlwq}Y^fP|(zgA}Y@;s>iKlZ#M2BV*7aaq4Tee`~? zTvR-VzRILPwdCSO<WH0T5aGM_y$-9{YA9VtGQ5-m3m3WE$0|QXk5n!6%~<`Gr+`qD zxV&ZJ55f0WMfMGu$;>e;$!M++C>^}115Xm-R+@epnUWFCtnZS8OU?*CY{gmYUPT#R zVX-P=;}X*9Updh&47j4V{Hmhrxtjs2c)#8k<vgNF21WQ^f?w_7eBkzA;|pYJoc=5U zY#?XZKC${D?m@x(5#f^ZX_1IDxQI}CxITa!Kaye|Xkb{zv-*NgAckE{39e1H7DDiP z?#-ttb`l=VdlWt#br-1Hy!v9hEYHyFiHX37bP<1cv1XA@0*mSG@?I4$F!H%Vx$&6E zj4oXXogti$5B=p|XR@j@TXu0Hd9eTa;vMnb{+hhF`+*XRXrs)hbwGRXOkR<*v;lIZ z!;eggmtNt)L#;atv3e;Ji~V%%{FaE2jRe2Nnmkzy_5ZQ=-eFB`?YF3kqHgJIl#W0s zDouKCQl*LX4$_5y2%!g5kfQVwq=qUWO?n4W1nE*Ep#&)cp@bGnAaGaQU-9$XzkB|? z_nhaf=TZ1$kuuku?;P(t#+dKtcq7QNcu)euVLff;9cH!U)DVtisYKJ4YOAXTm)rwm zPAFpK$A{2Te6^W-dv~ze!7jn8=)U}SLp85=`a%Lty^~A5+SZH~YrG$X$5iN`yK!J} zz_yIA{9F^WKlg`9tU|YMIp!MaKlhH0dw;j=hx_wB->_<lklx90+a2{5T8y{>)>mfF zxku(K_pz%g<27HC6D5O#V(N3%w$vVBVDT)5PwLCoXFgy(?SIR*g&u*0Qr^wsD{%tZ zseQV=%cysH<@IfP^+HLhsI54;YT>JjR4?DN&Rh}dzOa37@JRhh>MG)uiV(JcrK**$ z@EysHvTrYug8I7z-TWatDtpUZT~qYP_CSB4(bKR_cA=6nJxBDQs}w|~pHimxQTooi zlw?EG;(jMkCi-EngIB_%UcI$fMV@42eeN{Ti@LzSk!+{EdXudD)B?o~jq8cNI74VX zoBG?CrK!_JhMuw^3bQgNKrj_@aV&PRL-d7?Tp2K`p7^jg`i4~+%>51QwpaO=#1E!D zU$2X@8!|tZ-P6K%Rx;{kt!cgYuWW~gMXJolDX}k)*(Uq%>$z=c2}_2ZAX3ZgocX&S zUa!WZOXZH<^=05RGo4Vk9WoPR^itX(1H)pG_cuy{C+OZ_Eh2X8f>)_i_UpA;BT7zs zA2shjfAFVj&{dV{Oza&<8NcE8Tkg1IaIGuE&vbiHi^angYlT{%|B0pv|D|dvEjay0 zCdf@cE~75*PvLKuSqGt(|MupkN-hOuuc2~Ys%~eLAT_;%GH=xvHf6(a;p>M$8#8@S z6f}Zrz9S)iKGDA}@$o5V5qbx<s7e#<@Gu?-AcqdO6TlD^b`ND?)9XoxH7tIP4hToN z<4Lxb{>RhyDHec%;M!<qr;w0F<5i)NCxXLjhN&A4EY%%qDo1Q4duGR!(V-cWUOVFx zkTOh<lI@IkOuW^Ir#UdEieR6-0zbykR(X|Gl?Z{ICM+|LidyPYTh%DyV|umTXc4j$ zRZvf%XIv?LU6TDktTaWIfiRlA(}X4(tq@cjU=d^xgk=C1$qXcosxE$ZL}$^m!+&tn ztMfTicB4&DH5P4SMCXl_Z7s^ju<FcLp6=+*L@XF1joPltju@L2kwZ<|6T&95dlf+1 z2Ey-*a)+1azs1Uz0udh4f@9Y70}U?S!&3($wPgq`>-gb)>4JB{Q(qehKNjlNCEthe z;N_)OQ*cd9mF~&q^L4qpmNn5$$zou2GNWPRgY<MKuh}=?$6-!oLil@`PPPb=jgt12 z5JFwyA1TrL&Hvtl7tcwkX!6vKJ!BfTmQWt~Od>+{e2gvPofGIZEVAq64a>t~IyVQ3 zgcsD*)XIu+yh^e79=htWmeIHxb|rMbh(>w>GwxEc;*&QsjwWh-cl~dGR_}dn)MXvJ zq9in~Mqd@5e|I4^dsocQK3Fl~oUZ5*D-HnqS5j;=3e#xBx#@XhZw>8OD(Cr*+_g=N zGidXs8|lDtMguM2hDd0l2YH2%Y6gC+#sd>ca@+5m^vNCeQkmXG*-;LwmG-1a>bt`- z3@Xq@vttko&QXa{z9g%x%N(Wyv6BTClY<m!oo@*7Vy|H>l-Zp{HHGh|7bQ4~Gb<=d zyCmMGPgZ~z=f-=L*y}o6h<lUd^Hie=7bXr_9Rnh;*^AfQptYJzTZ-5Ftaq>$cs&WI zA&SY_${rGUfqb5HBHorFFh4y|-0F++POPgQ3l^V*TNfBEVvNaSq3*|(<DN@;@F6#S zx^JG(tU&4nedfBvG#X`EWp+lTbN0{yVFY=HaS_|G$V5zbgJdKuQnW=h^t13~Rx2o6 zOP!w1#re6YE`?!k?&doBKcjm&b&wr?Q~S05-a>hP-$KQPuVA%d6wHf^bgDI!mmJ=J zfwG6PSE>hRxrzEG2jK)7CH7+Z2NaYJBDjovw&(pdY-NnG4eYW!SnimG)R){~HjVb1 zx>sa?_FBwWmHpTt=;m_;HY!Y$>NONWSc_1*h=cxk4wz3Ze}{4mg^WQ^kJzx<puuoa z>79y}od?6>;(77<?3W6-EQCh(Lh8y(oJxi0d0$I#AG&hoei450fDUHn*7z`GP@?*? zn%GB!Hc?dIF5BWs+3Xl;Pd<-BS+VDm0eL>$<ON!-F)3D9uvtg4_W{Ru>!7Qpi8`{4 zW5qSKA>N}7vK6`UPQX4lSz>qhuG)+EK;(pDXQn?hfG7Q)sIXT$SZ$QH_UNxtJjYSS zYgs!l+@|j-1BQ|`S6AY3W>IV!`J?!~v9|d`bJ?pVc&!=gG{~(H!?;`gx47ic$}=BP zi#aViob=eqra#~9Na^$k)^xrL1WemF{+eWeJ&cGuE?u(9qMZ{RX|epaDsgyLXqDcj zx$TZSl1iym9CBnnuWLD8S*I(8RBx+EJbkQU3sudtZN6TVt}tB0haM?jEJ0y?mdePl z)-0?33m-d>QAX$eM~0Q>bn!^xH2xfrX;@JDbke1&XurW1x?UJodO=AJ3aMj9fB6HW z3!=$a<urCn6kt27Lj@hPsjFRK>(L^1R#AyJjNu?_R80Hk#>w1qFN}f<6mBnYmGfhS zJgdbQHMS@_j}}Ryk;=v9AK@*PGnGk+@}LKwSwpWdSB<uv8X7wHuovf+=Idr=s%o#+ zU!-AGN{X-A$#<{Ee?awWr;Xd&uQl^uUWk(g4FkP4r+BOT1$A0KJuJ$N|MPnIpAUl| zYJa58aFz=?-*}-rL!o(*{_Y*92csGEv3su5Hy+ImYhtF<MOvkDTSJqAvayi5Q603@ z`A8vWmwQk}+)i0gzKu6Sk7`@7oIQwgZKN7^LxKbBmmIc3@y4!VPfc`*oxYa5rzo&o zYbQH!<mRb)bqyT1n>rb!)U-g+Tz)UcwkwmaU@=V`X#lT@U8v8cgdEvm72m2G!8zm# zidhm~fGlaEapAjaR&iEpA#<Lt_y@t9s=Qbixz(O@&jD|2(dqh~Gj#5taH;*cTW6gl zr+rBghuPBj-3+2~vtAYX5?g!SpyhNK#D{a|o}E)yQqX^0H{ytXSO#+4a6uOK4j@%~ z<|88`6O8bxr{m9`uI}Q(cIvTzRd;{xsP}%~QE$`VK|X(Q^HKyO+c3{t$z!#~nj7&7 z^ZF7taxcOTOE6q}jX{bkYT*K`e_TdcscDXJ-2A*<-KIyjFSa%q6r)flE=#xMi4X5* zQ&#6Xs$+a>YhYsXIS%?cMQ7@20)4ml(Novs#Dr6C9zUvzUS?-;(Llq#+vP)Ol3tq; zxA&HeD339P83&f*1l9bdKkk%12u%o_tQd3xN%R=b>q`~+9C>R`EWDC4Zr@K7>n;ap zuXo1d!v{W;s;OGKFsxF?+yPImo1scf*34w$Lt?meVhV!tSy}dLTmO*VSffSj(~qQx z9Y?B;P&(Mbt4ts2r>B>3m*dAQXl6uEBM*W@;E{RcIs0WDqH&IXr5Z{1Kgot{o1&?5 z?6T{a%w>&Q)oi!$p>^Z?McesSqhC3DMSR;1XNb<Dk>1OO1Bt@xW^1J|gUn;i(=UD> z_re(5xShWg+L8?Nl`dBgE!_(!3F*M?)=SpHy*`&Ao*=Hkov&>N5^u1#Bi-JU|5Z2# zBCL%A)o*3yw%1*0?o{Ut7)fA-r!tev;50A}oY9DvSvL*|PGoV&Fu{$k^F6W?unahX z*?4eDO}rP^!w1rWrP~wlg7ScLT#<ue;7X}rf)fZGp`wvH2-FD2I;LaH<SH&aG0td& z%DNuEU#JZxKQ6^XzwO%uH_uYMc6gHT8i_1z1LlyR)S#ajPpge{-4U%y^N^i$JxQ1I z0@+k>)le8-HmYaeee_w-BN*FVjl<fhWtew!s~f|#tVxC+^!mR6T;$znKow{7*D6l4 z(>en#iq378!?7!!P9ItxAZ7`ps1+{YkmoIM=A6j}2}^AxXz<<Ydrno7IFON{PEqEt zMrg27Fb!L*EiJ2K$ta`cRApha=k-9R0o^bE70^EJD*QIR+31($aAcFh)G%96e+D(A zcn}xV(;=RpUUhqS8YY$L>|_SfOEIqvb+XW|k7fwVJKZa8gLC~8zbL*r{V)E*<5Y;+ z(P5xdFgkH;sG1%7?{9mg{9o_>uPQt6jH@&Mi(LBG^9SD0LF%_?903q1|0;&E_Iv-f z`~H)(`D?J>Z2Z>;{fnsi>lFm1{s;Q#ud%bK`<=TXRNekPuE1{kpOut<Lp;J=liL5s zeDRA`S+zXB411`6+1-7HVR4aP>qrhl6DI${xP$=^VDMXvSM5FYP+A!Kl|GaPXp^PB z>iKJsEsOx3&L~~0-s@uvososkFF#Lg&B3DOULXHof6k0t-E}aB$o%_fD*VgtLaf}Z z%yuXBhQQ;BpM;?1%ZRSZi&klEzW;uDKy;}a_37c}4)<6mJk~c1l+joJ_R_gVc&-98 zZ1mJre|KeFX9IXZo7+mloS?tEq<NJ9dh(-wgp;P)-(9|@Kj(BY#qgWBzq__-eoksV zfMjI-yX(92Px|pK@KqK6?)o_TuR#d?+p7>59$*0e`2T(Y#9>>V)H?rO-oS6Z38@tS z9N!oY>76g^1w3bX)R{%luVLb&>!-sN;1Kv=NUSW*i4p<@K-^!?teYy(?>6=Ke{c4L zm=k|?1V6iF`z0vWDMRcRa`s~G@Tg=j)vvPzxEFXgUfXtn!}^sSexRJk&tmoVKNiK5 zfb8Sav3{U9@AqqP|Gy#Y=M(+kDpq&#(iiAC*T4KV-g|+5i}wpu(U4%s&f!ArjgwRJ zQK)Pr1?evn;`b~cxV`P!*{<AK^)Wypqwv7-q8n~)M?W?JUXFJ)F$R2cNj*7dr*!?) zZQ<87;6VP{9!MMRq<^D+#@oyXs0jP0GW*%ppPe52{A8EsBe}c*X3~H%E~Kh*5ajAS z3SJllPrVmU1Sj6WD`Zu)?jtt=&Og-=xr>ygb*vitbvGWRUhw^mtlctKUa*aonV9h! z)tb>&Wf})pe!f0~TdS}Na;uwkm@#!J8{YsDj~SqnI4!gH^<F`nHN2??ofj)>dde(X z^1chZWO+b0{lonj;R`UU2Q-dPZW+O6p7!z10IG!<Fz`iY0Pgz1JefxGTe&y<Q%FG) zDYpS1v+`dDe-F?<{0D6@q%4f=cKr+l`?e|T5Yj&`vjjdlsD|T9r2q**6QNV<48U|+ zI*6YF-KcJN9|2_M7(mZM2wh&AHwlu6p8^Eno=t*UAxW}q6<6G<)~~_Wn7{4`u*J#z z{!{&m0G}$=4D?G+nIZIhrosTM6(sWBi8H%KLqbN1g|~^%@kV`0)%|0j*Z8Wo#8Q$Q zpnupTaGIM08rWo~mf6ax5#XAe@02&>t_O58`TQAg?8x1_hDBHbrV?@^OF#P~O-8>u z*HS!R>)uF0MF$RBeZSXHZ~2$G6uSL87FlMazw~sd^aBK8Q7=>y|2{FxpKYq&Z5nt? z&7(SUt!nZ+gN{I7RnPk6tN`zH*D7_&B&m%WJs*IrzfP(C=PdAt95O=eR~3%eE4!JM zA3fIC)&_2H+o(nG_0<Fc4g59<{mr}xo%@;?+*>N8fQJh(CIs$$Kn&Hg&fg|&{-op5 z3}8(hEvxLJp8*JHt9=>Z)bFfy{xjXs@4(BN#oR#!0jCh*bScw7IUF|NV;rJ=cH{jg zvV!#;d#jQ0Wl)KK@sVhr9uxK3%S>JmdiwAOsIw@^Ul&q;@NXz@%Uo#z5I_tQ82+gz zpGcc|>!+)lw?1C$)4b<A>CiR5*^IG->2YBC&;98L@S>ZtXGcRfh7W<GA`7R2jyHbt z{Y&mbEb2-l+t;iP72~~u=VD%uX*<5odTY=<Xg8hh60{9yjCLcw0nmKVoOcQq0+CB^ zeW)5DbRuM9GXls^H=xNR?^J*8G@vQD-<AS&vESVKM9vS7k0S`W>No19t}Af611;Ws zUMGR12C~NXL2pyzE-nDvt6^u(f<KDAQr#Sh38mIvX~UMeLjo|ZKB+Xt?Qt3N-@2vP zrv6uBpqyu~+P+u7Ph;YARkxaQ?M8JI+g^l9H8rp5ec^;C2ZppuPU?jIJ%!5G=Ye^A zr<#qoqiO7+1`Vs~DHDLrmc&8KCXS-nlOcqgG|Vq-1LL;I9CY|yu=+s;K_kL=uZ2Jt ze^C?yDp5xQ3tK=dIXe?#gY<;w7ncmpz~ex<HtEEaIGGzH6MXoI1qLOTT7U1BY+!h+ zS|HN!coJSZ?Yj`&=i1F4Bc9v;LRjzNQGb{|zk^wpKe~A7(9YSc@Yh*>r1KkQ&H|`~ z{(chBc8i9`6P3IJv9#w-ob#QXN~yz#yYRCX9+M_ORq$tynkH39J`=0oP7l|OMn`9Z zYT|(i>BYEvuh!YO)jj}y_^MkR<K@lm;+30rhveFMw9u{LBfFMLLfiv(!go9XKKoW~ zTs{cjDMGXjbjLG20D2(hU%3xn&5Vg5aOX+F$;tug5{>h(?!W|nx$-7obr!&3zO@<e za1Y4*I0OpVhfpU!#563g+{fmUOJ+@GjkvD~{}ROo&VI`^K6+9{zgSXKSE(8vDFCDy z?ni>Kx8VFtx*&(`_@|wq;RZmo1fvU?PaR)z7rL$kgcdgXR(Fp{`xM@48s6ah`|-XG ze^>(AjU{neGpC%Fy#Xu*_uL*c2;Cfb13He8`0*wo%r@<=Gco7z%1i>x9>1b0XBwk( zohm7&L>lHcKH;Et-?kvx@b<?zO%N?_I@(=`W}GR)z>gqcGKx`8kbK+XYsEo%8x9K_ zXDP}4HSgjv{_hpZUg_wNv!oJP&*c3rhv$a>PwCQ%Mc8bs+F|O3TBk|V*hq1HZh_VS z&)%r3GKQ~a>=vs!Hy@ydQns`NGFkgGLfrZpzJdB=K>(m(c;yrbbe}h>OK0MigX!1y zzoUzApavFNfjumrE&|XNOzPzh2QaHEYg9}W7!MfM@VqlgbWpq<!eKH{XYNu~`|3l@ zbTN2#YUMCLr)l)YI)51<jN7Y#zH{^X=>qfcRUTPVdZ+9E`dRH)VlpH=Uq!K_`ib`} zyJYsi<@G-Wt@Nwam={S>r#StYvO=9=4Z6(>d3^Nng0}R|u%;@;NBG{?yJUb2O>iN8 zt0Z^0nYF}8_|q1!<hd~q_GZ5T6@@A<G-02DX>-umqJYiN@Zu^Vj>agQrgAtt8$C)& zN)r2$-jWc4L`8MgA}o)cb9JF{DaV-h;9F8Qw}uFkDU-VKQVk*6ft6_>sQ2-xYjoE^ z{!JJG*#7LdJotNKMAv-_!7-AL(W1yAGxt{PD>Xop#@pxm5yz(@_?|EGejO#G6O?1@ z$nwriaWLw0^`K-4kNe{z-M)%V;8Q4Hlw{8F9Z~0kyymSl2DVj%4AxCe4&MrdP*}?w zU0*=ucEiK?Wv*yaSj%C*6lehErTIogT~v~DF_}9_tPgG~m|RoSU*g4hV`mW`Z=l~> zzo!FiKWsXYUjf>&D5<AZqnZJ((Hi~>3h4U1P}jvuKOTa6AB6@sQ<MduP*U#qyKGAn zZg7F>rBdExs8af5q(J%u%X|(kMV<5+k8Y+V=qrb7rVkAmUfnZK8&|tMG4wd-vE{mL zSibMhyhT(fyv9b(uVK53L4#3@QZ~i$>DcnfmZb_rKh}eQ36{(@gbuTmrwTCQJEeRl z<&1U3L{kHAljN|&6ZZcq4FJjI>F@isM?@D+dEGJjnZbwgAz&7~{`&M?IOEeVdKm6f zp$k*eHE=D!S-Z~=HQJSlSl-12Y*l-wqe{$O5194?etqQ*n^L~tw4hxa^o19WoWkH7 zs)S~UPuE4YfB6g6e)r#2tALzx9`30$Mj>z|Cp@ce-Ral6dAR@kZuYU%gPUY+dcRgr zD6WHk!=-Of|IoqyBksKga_avmrT^bd=>Nx=kTlalS4hp2stu!afAlhN^5V}Tmf~+4 zG9ecy?AX4N#3|@A@>n1a0uo!AGk)pQ!vzk~_WCLKwUNZl-FR=pZ695K&zb&NBRiox z_$)T#rRRfmE`%(@gHIFB^iH~k<~1)}*vbiKMgt%n1rb99dLCKXWhj0<69dFWsy={B z&<lWJ@|0=dztqtJkl!ZjKMQ~#;eJ3W)TE~^QDEj=klaLQ_dMecv}ewF@t)5_k}g(y zN6lv;TFkqf(NnV?AZ#Fj{MAvtFd88^z88FUjPmJb2@doj=;&@(1$rE;0Qn5~?B|#W z&~Fy7(HCm}jQ~7U?w)Xhk9*((Lo)}=IJX(-6CFNy9kfrtRBRLj&xBpJfjr^X0m4CO z77$C%6c=V8q2I4rtz0*;i{AuJALkJw2PbAhF->dbn-ga48w7Q7RdhyD-I>ryyT=*+ zKu&~xM5p==6XHw+kT_e`chS3ez2<N8Js`->2>Q9cY~%EyzdonP|2Q_R^zA@%U>|qi zPYg|;ubY)N|44JmVx`AJ0p+0&AjvhX2gKIi0O<Fy0TM<uU;D`p8y%rPw-+IiI4JjN zEy=eJTU;%K&&nLmg(#~L<iZ3s?l8D<2{uDOhD=Ytu-X%YCRa{R2ssuRkQx%ifUBZY zuGMY;m9`NhJ|FRP8lWtz`s)IhrBIW2;M`iX`hLMaF9K$62H@}k-O^t|Vm&}*{~y`k zzo${|HbF!>yZ2TsnJ&7_sWe2C3m*8i9uQ0Wl$6z())OR!>26KV2ZV08Q~qmZuU$d( zJs$`#e#Q)F@BB00dAdy=e0a|XNQAsg%9|e712FIU^9EpKya?Ii>muR4;i{sG3qVHK zQ-vCo!k8=l0NVaeX9A+O8GxHN-ar0uS@vt|2H1Vg(AehXPigN9&u@YiKf$fd>~Gue zKm7usazZr2E5YaCa9`joQ98$rVGEDXG|JLc!DqONGb9r*=k1F}skOy(06?_VbO3Y$ zx2H4Efcq~m5HLf2PHL2cG9i0imj#*vCr?Js);o306X1ixy;t`hWCfodDRa&O36e55 z&ULjQg^$e>c)<MKB^nifRnLZh1Ev4;Ph1IhxScyZM_sA951=`l86amea^PrNmcguA zp3L7!vDZ{re)?4k7)Jnom!6!>6{fx-WKGS0IHa!Mf7&T8`c?Kx@EPVTdC4GnHy!0g zAWzlZb6#FYPXJl?4s#dF$=v0ohD}tDme6gvy&Upn4nsVA1jg|*LTdg`tiYBdAP)n0 zkVy+bg}Ky*THZ{>3+HVCt8;S#s6_0<d?q3j&j#LXrb+p_$zso$uH}yKu+x4nU3B}& z62FEdpeI#7h|~$p<w?dB@I#B$?o-7-F$Vz9ed%LUW(Xk>xZnx62{597R&5igC%6JI zEPy7O@NxULPl6Kya6cE9<L{ZuKLVq}_06^08+EH6y#}plqG5OOK=;{E{9d8l#%Alm z?B?k)hX1hrMc|&ciV#_A;`WPceB9Nf1j(x$7)Wn~siGbDv;2Of&fFuH8N(AQ7k#;W zfTDJ3=Jh@YOFPgB9zsBVCZ^*7j6e%&c*>ITz>}JqcMTQ(<0VZ5b#u$7_&9U>WP{!S zR&X<r`Zsk0%Mw^e)gh~a+R^t}%58=&?^gF;n>XIweUo^XX@?@q>MgQMFIr44ReQN- z4l~NCvrBCs?UW(;OJy|lAJoU{=eg{`>PA)1SR63s*H?ub7MH!`XBSI;RLsaS-J~CQ z5IEpsIZ9XwcZ5iPD$bj~>Ri~$ea(j^^w$2fr+(!A9mw%!vwJ;34!)#ww!?o`MX*!s zE}pTTO%g!d3_utfU(A_&J=^XJJW2>)a8XU=byms6=ePE7rYE7?fgrO9NR&4TG32Mj z7!Ez2Lh-Gh^tGR8^e}860bReFrhv?|nb72O=Aj1fI24HKnA5{<IoUjE8~?9a2ww1s zZqahGf-bOUb!(*`z(h~EzqGHxTspm)gJf<XB_FT7v#eiMZM<CZY3QIb+e-Fi>(k3i zkQtVBqc<$eJ>t&C1Mg*pr;5R)?E@w7;(8=YXC@h5fjLUZ|7Op(r4w;6Gvc8{Q+{l- zy~3iS0>L|_o%!EprH^|b@sorC*w(TB<qy+CkuGt)RcT_gzP5DVGWkO8?Hm9`0k5Cd z8tk}{PyH;=9_(NYJbei`@$+Q6+lP?5Hxt^;&j4!snX-oYS1)M{jdP?3<`f``m-E;> zuFGB0B$RPrOIob~+e9pf@;>O|LRcNa7t?!84du%kQdg{|^!a8Jp0E$30Y-KOHGqt{ zw^ZFD0yt4#idp@B_d$7=6|nS|&geY|xn-3FJwegy1Jpod2m?A7%(wi5@KFjC)Q(wp zC$kC%?}U6L?5RHE21XEq;X(THB>Wmyn%VIBVf*nU>=Lmz)^<5UP{NR*FxDx!IwN^t zq_6AGnshMa<~P?L1RF%g6go%CmQftG79LJ#-rb*7e&j3r^@jQ7A)w~8NvJ9vSu!l8 z@o#c(bA6Kdu0Vom$Ie=yq8HSBg0cr5pAMKKf=|lH2P(p^;hwd<wUC+f9Muv__IzwS zGltu1vn5{_=en`i8X?`NBF<H2J(mMQ0IHDUEOxy}DKWH9q^!fzCBzlPANY;lI~6?G zf-EH7f$i@x?Hk#KcF<J-ReX8DA<4z?&Ew3lIG6#{^gWEBxB0Zcjh$gl9&bJEeI{B4 z>#2*~p?j|S&RB7br-R@SAh9;MbvHS%pPcb{f6AS|FN-iwgXjrfsA9c@&(_lgDVC<h zU%~7uJk|b$`kg|qIdbsk@leoywcr=Gtx<ganMVEsQ%qr|w<%>@jN@0-2g77uLV0o$ z6D2gr-OcI5WQIf<1ctoZA9p_)8ijH3_+}tFo4lm~qU@TINDY3@Z#L_)rxXiX^bC$x zb!adRL!6a@%@M>G`Xt0AvvJcgBDCF~F5sYMutaUiVn(Yo*r~O@-mTYgT)j&t(xt+} z?zrgD<EHQC9KqwSU7CEVOid^Kw0IuhvG-{UFe;(zFTWNViC;Ff@+=t=9kiCBAim7( zYke?Y5d@r}xFJFtT!o+7-~}_vSrnSX1Gxz917+(tX@KHKLafLwSE_4uYP%UuB%;#F zWp!Y_H$NSackG#m9-b(jVp+lV-pzEoFr;@@!|9q-h+<pWXq{y^bXG9gH_sJ*Kf~Ls z8EWZazEFDQJpgrJ_Fs7K%9cidhFEyCnOJ-{s18VMDwA<*4i1>!-}Rn)?+eBSC3LGY zLG<g<?@C~S&Isi4)L}?NSi|z0b);-P^zuT3s%P-L<)_-5iJXA+vO45D^XBPK(r|K4 z%^PEY6>-e@KMa#Uw_8n?>Gb$ZK)V{LKB)^mih{+_MsQ292vKng`}f1o_JNp%bBh+4 z;Ld&bN7R1g!Ty2wlgHn-!#=04SK9<LToL+!CjhT0?<w%>(CV_;*LC)$4`{C#rF<&t z+w8cRgoDUgq}-r(eDY)zBYVT~Nl%+U33xM@q@5fVAOJn<9<wFsimwv}_w~b8UwgCC zweF5K5`1dSKp)RpBV#~X-vof%CK4}1kik?XzFDjK(X+Hb-_i70eY?;zqst3uuFeJy zf70^w@$o?zv!?sE?c2n8S22V=Kcl;M2PhlWCoKWSEnm8uT;XUx=_t_wT;r+MISP-8 zfSI-Xzzh!G7`;69+*@SVH<AOkGP@n)<Anjhc3<hTPT)MX{gFLl?Y3IK@&2h{_1l4< z3`Fe{Hwdz8J0STgDBU)-vP>?n89Z&%&npK(?MtaoN?)ub;M8VJTR<v@$TzVcu)52# zfmVS4;NWp?7_9APf=#>+q$6j0qRzM>;5eAE4Ecc(A9i5Yyc<IBYq%U>&Q38)8HDm# zQ!I4dc7|X|c~}9tzHSXAIh8$>kxG_i;IM2ib7OIZd-prv?xu}eJ`j!W5QnH_FLg$O zA)+P>W*2582w3sYc+quP=i5hVrDwJF4AYip%<H?zZ=Ws^s=>}WABQ1c0{rv!%aN$$ z)GMGI8K;TC3+}Pwf^Sq&Ow1R|fYM~|?LTldV}-j8j~?X)qBLO>nyh~hd#^DcJR*KU zQc9@Lub%>3W@7-5L>R*sg|CQ;8*`RFXC@kjP414O_QgKf^^~Om&}RuB>@eLJTsvbO z-#}bV5_CLsvu#p6NK9V&gmf^ww3=d|tI9%}Xbv*>s%48#!dkwoQu;i95Y#Gr!H+-3 zHd4deHzh~GdnwtFhGb_&eEz<R!fc-8zJN;fXjl3=Va;spz8ZDAHcGVoDm55~C%|KH z)9Tw9LbY2d&Bu+Pv^lYIu%48d%Md<(pOrEgF6mpEi9b_8zFuQ|I?Eim)iJ{`yA`&X zq^IqNbZux_%r%j+yP@FZY+b#I=BZ3$KELWaxMrwic#&KnE-Ey237Y&#^RAF}f7MQV zy}kZGMxu{>ypD78b&|nZP_DUNl5$q3#7cHZQ%dW)y}dUaKW>#CE1ONm-?H_QE%XW- z!V|j}?H-uNlTu1XsOJ|V{Rc!4Jbky!+$L)+0*VqPr~W`|HUScpTVK6H`rhMKaksR= zdeB}Gq=*GFMuRM%y>t(IeuJa@R8jU_0DRKOw63v^yiLySJ?VhApzW}ad~_c9(G-a` zl`=~T`6`;a*R8o`!|G-8sVxgO2lb8`syfMpILe0KZkOB%kUF+v&R`g!i4Em?+PXHe zzS{#D^SXC4k*7mzJ+YHO-PBJyqyzavNA4$bxt6gqoL&Wb%;%ltJ?J@=R-9S;mYx~H z|0@hVi~V$3d*&PrHg9))EQxKCmozX$c%fU*6_ai}sIb9hqYBn&ad|aph>Jipa>#|j zDAjcLdqZtB_Ld^@-XDV;&h-b{K~1u$nadql5L)D(60kOImKeR=AMW-{8J~^TzKCc{ zj`>);W$wDRq})0t4A>a}-?lC%_;%JIR+2}i`twZpQA-wP@Uv+O_Vi0-h7W;VS-pX) z!_Z;*c%3H<Xx6#f9lGl5b!Dd()nzS$WhlP4lxzT_hyYlV@-c|jV5Xx+*}-pKw||t6 zjJL)VNJI7<I0#ZlWi?uHf~6HsZw@(gw(gnRx?NKGkeHd2HhsV@Tdn0bJ-A*!ql0~W z;E8aNVamzgtQ_X*-d@YW37!5<mO|e&U4$U5Q}%k?J+f5-OO&D@aa62P)x;u;-y3}` zTp5*qULnLpX`TQ{9b)yj8#{;@l2U~nFeI<KB=dA-p?pKL_S`HZ%#1B0ITf@NbtFYW zI#lA_8R5hkq$d*`?vZf>dn<u;dL1QbeA+k!EhpKemltbTV{8YUa<8U6+r%rb$}aa$ z0fxb<P&D9QI4jwm0nYfy9|&V>>*w=iX0U^0*-jiJ068EmlTL21-B9!on2Ydxa9DD| zi{#IctJn-0M}pDLkBbCh6^wW5;Yi^@gb|#{N%v0Gh^q`t%lRFCd|A`5+({@@vP}<f z%a%3gx5AC2zy#xCE{3h$marDRqn=N8l>z#x)Cl3w7iD<YHLhKD?t7EQ2a{7h)RR_F zCQrMWa;_-WLQzSbyu2Rr<2v0+K#1m7AM@tgNS{_Wz>mwpU%rG?Zbt%cK#k@V7Q0)i zT90q#RbIppj~hB%!)3+!;N$E(r12W?1X1;zVBD-gx~5#IeKMC2Z6@^~C3&Ck7IAMj z-*~3~%|F@+_Cn14qkWkl)spE5pV&ajm>_!<7%H+60ay=PsiTqo2Y3M48`x2uSJHQu zLml8!L`S=%HXCgH&dJX-$sU>o#6vb|0#W<`legrU`Ms1cBNq3<B^k4hbF(QSFXD&X zjuCxN^4=xc(-gjiw>S=MLz}*}DA+F6fxg=VS2tlVBcwK5{Ir*b9V2|I#*TEAvt;vQ z;|9LY&gzy9tuBYAg>#yl?Vb0SSvvt#BKZ8=Vbo^wp8ayHr#jpt^KP}d+%SqsW70f@ zkf`k3e0I!sT*59yw$y`tC}S+y^oKmSKu&VGe-V=D9V2=6-3LoBPr?Expiq_ulpa|% zb&2#rs^_HaAa#2Q137JjcTqc{<%h^jIzTp!AeM~$K~-(JOi73%ki}8S9i9lIiuMto zm?o9<yCp#p54L~wr~Z|S9lRJ^wyA#YNcH#N)%bk_IXj!Qf&2S9S$p^QeeyNu?}WG$ zH<0kqJvjNID*KRlg5*znClM#7uY=iNvz<;l$b%jb(-i=uz*>^e)wJ!oNWYMWcF&)y z(e4R;Qc0Vq1&Kaze&1uAy@y_>f(Pta;2cEgG6eg2=IWmFIWq{rVifp8qC2k+J8?dL zZX5E1xRGS;R^oX`h#N78cXG*i)@u2*xvZK3*O~2PZu62xGGt6C%QIPC_z7=}g0&DT z12ssryc90Gms4fORH-%}FHfBI`R+4JAWeN2`^shIyfZU(ZUkjt9lgBwfIyMXPW)X1 z3c3i<I35d+i0AvQa7CKv@@pe^uaxnGa-PrSA^t%6F~Iso#MYp4u+%|MiDbG~8)F4o zC_{}8uYboi%dcj`#ucl*PG7U8IqssRoJlR4X9<4!B~OS7E`bsB-UTj-9ThQ|N{fGI zag`BL3>`~$;beVe1^py~7^yBVEI%%vx|oH-rnXtHn!y1_8<(Vcy}CHU1tk`m+_EMF zf}gB3BS&=*{!7C*zStd4?C(Ms13pr{x1-R_-KZ71r4D**t6|<WaqRYxxO)dWKYi0K zLIIjK_f<7hj9FXPXn&W9+o~2Yu)RT7<KV(fs?yL0j3}+MJ)N^Lv*Du5F_}pHmxwz6 zdp~lDoFI}#tHAZ#5OIJSq$F@Cp%tw}Kj_)+qC;h3W#B%-p22hH{%K?h<}fpxy*(pE z77@;9)EvAO!opLn-J=37v@*P_<>kyUqIFD}FFRV1uTc7WSynOl-G(E}^0S(xPpxXu zp~B+}2?p?o#89(9ME(sR{j`VM<i_(rGN1BtA{d7pdsX<FWLHaS&4YZvAc4%n&B`w( zcng`e;CYF*wfYUaqP8C$<Q^Fz`|vk!xXG^s0`xK;cGYL;DTY($BA-o(boutk$lDQb zZOV6>$&7jo4MENO@9yOeH?pAy#YQIaiINkB`(+R(QJK}!vj)R4`-&w4z%73M3dG7N zlfo)E!To7n8v^q7ca8BsDjk<Jn7#^#+dmu1PFAp|8j@#<mse>Z$Bw#^5Ggn`Ml44L z36Y_XE2P2N!t~t2-yE`g=HffJLY{5fUJVUC=?>-z`9?fIVorZ?&ytz#<GGac;vqUQ z95HlPh*?Qh=vqWRED41?BCgDL&mQm~&F@KkE~JA2Fn&n)oDoelvDn)BRUsFW$CD0k zLnmn+tmyPX?6@YcM^|Y7j2`4)7M^tY@#^v)8OOxUt*^BiL&ieMot+G`3z9NF4PNTg zp+E5k3w+&luTD}cFckhMFI9O@9%u<4#wdIR6Da&e>wbfw@3{^U<=KopP?AeW_6qeo zr@*0rS-sv|#(fX%bmxV$KF>jk!U!G&xsXHt2EDDH5duWCQqCHhv=+6NvPL1HeEkG# zbq1+W&NGn@HD+o+Xom>ZV6TEVn>KwAZWlVRNzZORvk<z<itubtCspez;(w~uOnvF% zvn6>8ggPU5KJ7kFvCqzctNf~!=YWXve&oG%gH@%BI9BA8&&A;)9SYBRs*FYQ^NJ^w zsJ2*iKi`fnvy5i04)qUQh<IDx-X+wv(+L(Rf+8AJAX}pI_#pT&JeV6(0}|ikfGW=h z)^oQ*307>mhxKij$BE7>zWH*a@z&7%p&Q#n^&86xEe9sTxFD>|Y7Q$_t2`CI767|a zBBWQ_zH1~6-c+~=mN#vzi&$>iv)GjOo~7<rs)W|Q-k%WdOp;wqv%#K6#Npi3a;JN* zjC|!{>T7)#h(3V*VR#m^O20x|?t@I0_kjy++XbLiN8V|iU#n0mk}Zi1c%m*h8ic!K zqyA@87S#w>rq{#OyiKOlefKl(BB#*Zf!ePkwf8kV_IUj~m?gLtO^z&BGP#uMn@SP7 zc(chpFr;8`G2E`KPv0z4v00p-Nww99VL1us6<c_Q#_YTs3b@&aeY5Cuw#eLvLMMdQ z_=g67jy%h8+FSLWJ$J_os(jzbAAbpX+FOXZfnw^-bsh8&GTOLP?())iu)qj7eqt#w zi3|I7#$0C_-dk@n3YN=M|NGjtO^0nyY2;b()|h;D2ME)1msaB@Er{bPso~iS3U5Id zDD=c>4oN~3a-aCHvlyr)$9D*Xye7Ut>P08)fpMH$GpO)4@Vh@N4|^^XqDlPb%O6x9 z^w#X)=j}og7GEm364}`mcE~y--98{bWGl$}iPG}&hBOlUt6w2{Lfl1?kZ_AQNBttv zCt`9^MTSM;=8kiDT<GK(YcnZhT17Zvm5>uH5GRq&QF9YxGSdx&C9N`s2Ep?Kt~5Kt zM~?)u7CkNLxJBpI0!6M&^JSXG*PRbUTo36V^h1z!bKEA$XAt(0h9_PaR9|MhCE|N} z)j*i)5*C`+%T9{;qnn+cv@OvXOTW;h=eQzKx0XTY%4|u2N_g!s_-5loEO=L}^+t5> zJW{PUt|%mT*n5E*i;qxZ*Ih_ceK&Tnk{Z7WO}2Ysk~hLW*10Q_9sMQbM(c_)lgUDp z#S(AicN?;W8#{{Y?`-j(YhjubgE-R0+{`HQFX?T{6VCke4^TxyvtO<u+~;53<g`R5 zl<XhKD4&a$3GM1kM1hg!D3(8?cglj(XNu)kG+?d5d8jBEXzUx-y1>hlZKv9=4#}RH zFCoIA!G~>$VHl2fL@M@ThJPG+H$)a|AdIs-z^_Q}k7_Dniw4I_o{me7-q>qw>Tkq& zl3!}D`ij}(f(|wxL`Xv8&W+Wdl#%AIKEEimneoZt`@4IO;*Hrm_n7bzS4wlWn#W9n zkxFu$7qD`8q{v{Yt35@^#ff@!GpPlU+JmJc@Ve=D^P4~RZdC8?K|a>DAtpp#mNEl9 zAyhN2VHNHi2$sJtA6m?iE@J<vYtaxx1t|G4#V<4U`Q-&m+)$TZ@@bzBx*v#;2eyQQ z*_jl-q7QzK6ZuaybY_mGB&@q`DQrr<<GU)4oy+PAG2O9t$Pc*Vp{4;v)=q4j$AW#x zT&kPQ>soM|Uy0F5NOaPJF-LT%7J>_$cN(ep(Lg%5IZHiJ7-A*1z37*(32o+!{U6BC z>x>~nR_JF>nQBDT?6Q03(#{8Zr~z8yx$BhQWd~cGnKFa~i~(aI?Yg=zl|tZ!s@Y?M zVW+sB4;2CS)g&ac3=x76j&qTF6y2gke8eb{F1jy5U+t}9=LFt6Q}jK970!{;(ywsR zkQ&m_*7c`6uQU{<?}X_Pea2>b66;>oB=VTIQ?XqU3dxQ46`}>7OMP?q!$u3X-5%IT z-~*uTvl7oN{kuhOH<A7P%Qeq!iB<<4L&jz+SEL^+=eb06cza|N4&D`%N7^NDTqAv$ zUaL8OD2K3?IkI~K*Y}Ekw+c-Dgewm9;`PbOLFUw*CApPvzx%>Ey<|ex$ZX`(=+x@9 z7?+!?_V7&6$m!BM<W_N14l|UGk&4iX12%0yBIc`<z2@E;<bMd9+7fW|%BUsxixngv zKSsy8DCx|CRqc)QV5Vig+y5*<RACNTW>FL0rr_IP8Sq|twx)Kiwhr``7_g-;+E6^Y zpW3oJ;B!qKF!V;8+ZA|?xoOXpR&$q9+1uGMH{P1=&*(OW2wJ@MQNj1-myAi|%U2@V zvx3UpUuj(IX<m>F!;nCAp@-%A+PR@lUTtyt`|LwsDuC0g>~}^R*PBhw*xJ#@Cu2~V zp-3cdrg%M{I#M51sVR<&F<^1{;!K2>haR8$wBO8XGp*1=j+UCSrFxCTjx~qV?p@s| zVOinIf73$oWYcAObR__n+jh?%Y5BRqCH@4llyg;JYVY3EG_{)QEV}>x0VA&UwAkG! zQe0E`V`~hai#i&Oz`ybw_seU+$|O1%N{U6AAgrO<Z5hG6D{1RZs{7-DjhKhw1{0~2 ztCmA;JA4*?98Z=yj`WrDjwPSgPZaFK_{J=tM!AxnS&uIxydgFY9+_pvDR6_>a()p1 zY6<gLvGkYfN^Jy_;_#6OH@&(VHg_O$IHK$HC%Z#10N7sr2eEpM?7(_cmsFhLK^fIB z^%`*i$@K}2Mv|zH)Ca^dB!5x{6T?Up8PtTWEtK;*<U>w~n@N(!eMlPNWx2!WQqN2I z2{_q%hdjSUrPmY#wjTPBpr97wwDaf*Qj_K&f%xTTR^}67T6+}EO*Ld5<_pp$2{Rma z>Uz;39pd9KK&vD~)IFE-T<-`wdTv{j`RgA-aE0NOle+@igLI3c4X#go9tcZ@XY>tf zu!T7Z-wX4ts_>i=c}C$?PL)Dj9zNhy3M{^Mmna4MY7;dLw(u=6HE>*pZyZZE7jH?@ z8&{P)1@vwD&z{i&)oOl#7H2J{=IKnTkfS!`1Lo|lRE0pn4r~wP8SULUw+|;T`2%g} zyi$f_Wu3BRA}@jOXN@nZjwj+qh|BAVcWZNodd?v<9}RwJ+pjZc8&-Q$%u=NSb>`pD z`PSf;X7IRqtx@(z2Pt-0XoJly^Hy=@Qg`hv1c`B|0@{ttsV9HU54TNY1;RI|E#zeH z+^flO9Ow7d0L`p{k#V^QMwDv#0<zUYa=a3Hqp@qvXQak%A=_!J`-?)ZA&RY;=V(=$ z^<k0*eH#&z5u8e=y-NBmhK^0;R9)>w5e7KHMdrgng<!lhy1MPSdeCWaru-D`Q;6|F zea401Mqq;$_<@5)|50zOOa`tjH3~NbNGqL}ab!*#ZwH{N$89ie=;fTt^n3OQ-e79a zO~+0|ig|md2lVR@xV~ZmZG;?#Mjx{v#K{FI!8_|rar5YQ$g2L1shp84_B(P(_Oamb zN|v(RxY!;G_I|0+u#-bBG*S$*=!+hx)tgQ3gOCCmD79j-BTuWm7PNUV`*CfuG@bh> z8Lq%KJ(;hZE`gc$CU%Vf>RJbol(opp|G;fBlhid2ji`)lGktHme?k?9RFrBWYY1Ay zTx`Sb&df0u3js%286H>}el5ZrddUb>8>f#K=*^T!z-JPh5d9;?_@A?!;OOb5`Uh=$ zohOS#EW$NzfQKZco)~FOF)p7yc^mNB+s_S=5EJPFqHasS<<v>Wthe={nF_Z;R4W3P z?(}`gJbyc+gt(>iZUgv1mF%#4b~#WMS^<GW-1GB0Tti%mPY0^1GC=}Y9JY{iZ=VYh zS@A<xHTvXJmKqjks1J#E)kug2hzAGkZZah&&k6Y2&ovHyxSQB>vl>i1N4n@IDVKY9 zk8U~gyfwa!Y+tHAqYo!(vdGBDNl!|FuwU=zdD`}dMx9I?Ljp_ObC{0{nXlI<oKSEh zXIAU8zPr6D?c+vuSt%~{QE2j{<6_Jjd8N=B*aKyDi6!Kmxe@b2>-nCwq_dRBi_T5j zwYikAAn(j4c)`(Lk>2&&z^ZO*&5O?#aGF&eUO96)b8IfZO`e*aH6D+9D#LkaEPXAv z$w@hH&6Xw_^t$Q27NcN~=Dm@DAP4$*dZdZmNDwH@r=QX{G|Cs`NE02Sqe#JhIw3bM z(UAj<P=%{&$n8>)r1J8P+>=HGgVzHTg7=~K*b-hX-YmAO%do8lY~sYqc|kfC5|{z# z1R9Dd9+C<li<BHci{M5DfMN?mlNH)wZ5aLI)x6MR4A1iGNvBR3y0QC3tFM5s_}ySF z>SoDu*s02^F}i2~g^7|M9iDpQb#;(4Ixk*<ILGtVOP#GCC73Y|?EzyDMQtM#Yk$G? zY_uIM6LEau;2bVgbQ`J@<xioQPMwde;9GKL@i3>xir}YY;=P3zhM-X{3A^%vezon# z7miIOVP2k09-2$XCGEHpstVw=iVmu`xH^+YToMY(iBe)8-pEo8*7b%?K*mMc24&u? z<sX9(Zp#G$>`Uyb%K+>V+sGqL?!muVYpU5Z0<ULj!^<=Zsk+NCI}5X4NF0}=MLmKY z%@auW<x{;f^3YbYYG%ukO&;13VipxMjuPaV=_7hKHrs0=TO@*LqOnY^xNvZr)E5Jk z*5BfI5C81bJ9)0x2pW}k4>{+X-E-w!tE8^+7Y$13xT}?5=Osgw7DE&Qh<u5wNH!DV z_J!tLwg$Y34(8(`h%-nm<1Z6$lECQJh0fVO==DAbC7vdErZJSA9-#C{9!cy(+CV2R zbm%$}51oe7o`_PYm`lEe4ty0n=hzkXT#?9qGjDBq<XvClQO4JsbouV`O=SM@<;3Tw z{L|qiv#({+kMDw=<$a$V^{gsoUfng{EzBT>6pX9h-2<BL^oO#3`$E24cK+#wUY+HO zm5&_KIv!OU_Ah$6`I1{EClb$H<t7><?%8k=_@+r6%n*1Tb=C%EKH+zjTds16?s&Q^ z+hOkzeco<4DESXid^jmGh?O=U#zizXXmqqy4}hZmCZ+6M<JpKZ|7ccjwX29*M<6Y8 zG4lmR_{q8N_vI-Zp+R3iU^A4SMC+?wtn~K5b3Ppl+3h-b=3<#_f3cz}{kfeA$XWj4 z5!p@B#{rbe=p@%s$Bf|oXYD=Y9yQlI^jQvXYM8HW6lNBzey4tvpg(M6QQbp7(jk&j z>XNdv&y=G>l?<YwR2ex`e}OCC&ID~V^%qR014j>`RMpmwacN%Tv8PV`FP|Cs;Mn68 zomMOAif>$aY4@;VkO(^DlNDGj#dhBc=+Y*(`~c6#{tO4ApSi~UdUyJuV%Kl$_zPyx z05yqAYZ@Lbmw}`=afV=J@QMM6FFWHUX;hCJtZr6kAb_)y#>0c6LXIps5$jf{@%vtD zV)NqV!i=iHPom>2W#9t|>;p9=b)(_fRNSc=>S&(gM=e}Kb&AsVVnz#<&yMbtzWD(q zbIn6OsQp&6nv9d@!*t<$t(JzvYD#N`xVE5$rbT)~lpb&Sg6ZsRh(T`!e{&dRa3~+; zZT>+9s`oMVw-Qe^_tlLW$8BhWvD1d`JM-?#FEK<|qZAfH#30QS`W|Rd>{w!`ASY^2 z_W%Z*4n9`fW^=vxxK7g3#G=@CgV!BYyQ`HtomYI%zN&pq-lD2@qfqGT2F<WI<X8VH zW%MYKn1I`TY}x+HC0tr?)d|tWa&Q$v?kK>B+9p=zEit-G4Bb5KmnnTNM-)n2da^M) zQ2Hm_q^fUpZC~ih57`&}LUWGRU!Ij8w>Iggrs_E;(@6_9IL%oxhIkW4k!I6%CKXDG zvw{H|)G(t)5IX9X2gx7wzJlM(m!3=0lL&usozp|^{AlYsM?35+Unxt&Pd<|DE~q3+ z3Y%)Xi3w&jV=58~IouI4h<xj|Ug3)Va6BIIqImNxkUe-)p*FF=jnwNgTy$1H4tCxF zsz2plG4{f>el@H~z-c{Wv^(?0#dObE9jXq$#p(B%*&fW}K1UKhq?;}*%kel!xdcmQ zMfqjqke?}vcjAXayC?Hw7okrP*A>5d;48jQr?B+RH+mnU?A|OQCPqBg(}KsIHo|%e z1LLB1R>gO;o>CeUZ&(yAs<+=wRHfU%%sd*2d+QRrTY-W0h{((MR`3qX9~Kj_<+rsD zh!`+9IgSjA=ucW)-cSv7BF7fFS$Nha*YNC$f7_g=yozn&$wG}c;@607IcThU#w;Nh zC6}a>gOkWd5(96F>@hey7pBQ|kA(Faq1M)ND32&rPq8+Bo;=_R{%=O@5Um=mpkyn# zubXIPlLw9hlI7As7dOevfx)-ZZP6}m%Ymp>{+*=^Pm-gjf!I6bF%~Iz+@>B&KCGy~ zMR`Bbdb7^xGkd#wzZ_yPy47mgK2%%aQY%?eLag7YXkE;YAC%b4@ESy5_z;{_8tI2^ z*N3qAZ}Vgda<bME;6Vs3Y&()`QE!Uij===t2e+B-NuwHZIeSd1^mZ1le5+QtSEq&; zkqrJ#gT3gE@VeWlu;6N*NKee$<QvEF0n9OMrC2U|BjB7fW6f5sC)2o7-R*bs2tKT& z*>M09s&b&#(4vtzz*pW(tt7@kawxg`2n63R77EW?bGTe01|Rn?@?g|!gR*cp@AgY% z^VU9xW@>ciLm!*$U0u>A!~ST_&*I;)k9u_?#xP5Cd*|B?^)uK1(gH5LC?JzRQtj4r zpXHx}%ZQ=sRvq1lWh8#Uk2*S7rp$iSfdvNN4<Q3<Ydn4$zL`m8=>&FUP=m}*;d4R< zF7sp0W#N&S)4ENg%VV9}snExPA&RGA4OmQvF!Tqe1CL%F@kFgGzdx@H9|uDO%71ul z(SrRRXRR&A==RlQU#^-yqb(JBvvM`~;4v{-_`r1+O#1C!Yx#+}44;>D4@wXGHlT*b zCe6?)Tdle>d^3JMk}HcFQWi*TJn$#$i?9Nb3S0Hp?}cCYx?T$@5BV4<n6<Irvoq}G zF6U0CxqS3?KLFZXkR`U^XV>203E5(-@K`&tI=kN9<+AP9Izf_^A|i9PhNcdg3e(*k zgXO)Q%32QTE=J_|2a#Le$vB%QF9~nUGcRYd(`3@2ZRr}voF5a-DAgHjsBJ~M?7tTW zpA{o;xEbAJ6|hmlb0=ZFeoeDnWm9mQ?ygA%sunsWYrYGdN8%%MY7KX5A9@V0g>E(5 zy&od$Gum*6^yf`r-s93XN$Mu22e<uaNhb^nWDv-v^0EDQ3s#>7KZ(Y)kWQ6@A0Lki zn+Le~5!dTX1bMD4W5uz^yX*cEBp38hb)e$y>nM}9!hsc$#Dw={cxaVm)quptH!z%O z9TRdqzN?AE=9KQ68c$VdRZNV(_qu%C4G<9Cj}MsEsrQw&x^>t_Turgc^JIUlr@oS< z|F5ul4d$P?QLMzzTdx#nZ!PWj!xKM29SCEqE*+Tr083{MMsP@<Sd4mA8y}anp$mh_ z)~H{&31DrV&N@z35c`!{O|=z;u>{@cXsq{^3#R~6?_yi`9Gp%N+iJt_*Q3#UTryo) zOO$)M->wp1^&_kQ+}rRj9zVyo&%uyq<1%UO8&;TRwCKU3&6zN!l2aW)Zi;ECS5~nY zArGiwtZlH|nsveXdws&aER>JylzH2|3PgBVLJp!J+2htAn<GGbSl$1uc-w`Uv6_sy zFSdP3dZI%cyMmG2GTj{}dq1Ht)zr#VT8=fn486Qo%j8OWW&|Joev)21hGQ~f(%~p8 z?xf2<4MroE&vZ1b(`LqA<@eAo8829m?Yt}&O7_n*%J8clYN^u^8RTABaG*~cP`jOG z;{?l^Y=H^w|3B=#XIN9~+AfN8q>IuaK#(F$x^$4LQbj<d3PBJEi1c1WiPA-S6_6sm zBy=K*A#_j(y-Dva^maz({MO#*thMI;d4BEd^5T~ygE8LmzRz<%cL_PF9E<yr@-XAM ze7cxIdLZ!;R4M&>t6`bq+|)-v)w*NYjX~_bVE&KMoq6R@@PgpTZZ1Ph5n5xLu`(QH z8ncjNCuzC+!Q>PC5b%BKzHRypMOl^Y=}S2wByRJ7vfG7mrK$&}!hOXpn{-=L&=t~o zb7KAXdX|RwWMfavqIkaT`x__PN;ul}cjs>ku~%%9B&oeCExyzK59t5maOiPlU6D)% z4}Wo<s%z+|224V@Ug=AGbX&+eliHn1XsDnY3EFB~lM*MHY-*q``BQma;n8%1NzG)& zagt+#)gu?o7m}Ze2!Dm{a#|8d=$YCXlYH%`O#e=t(vX7NcyNXN$#nkWy+dglv+m8T z_*X^@f1Z}>T(+z~ut??nGi?yiJvTNZ+Dqz5z3zI3&TAH^J2O^FG#wM-FWg8vtZ}vY z-e2wNc?er%|6;Q3^u@Ka_0El9`1zQguVagB>z5@i7F8dfKB3&7l0T~B%fT`x-?hK1 z^0g8KWT&+}oe5rRfhP#*$g%8Nte<-$bAkfu6wO|a@%{&wzTWz!`h1Di#o87Q%C5tW zLM0C=4*_04qxn1Un?9ze#4OJGmOib32;eUUIuqJW+<~7)gzu=qCyCJXb?5CSoCkF? zdg{?Er5BF4IRzMVnG*Z0{hpn6e-3iPQRVmc+&jE%WmsJ9{H42?Rono0x5C?eHLNQD z1$???J8)ZI`zg5NE2OqP$CL(;ycyGeNO}q<WMNvlCY&3O{>uQ`I`bGe7jG~Dw@i{e zlMMzanL<aF2kbaT!TMi>Y$`tA-7Av#dVr4j3Cl{owJ5LuqTpIF58IJDF$tR!pfwQ2 zAFyv{iC<QH1<;cO*VkHfj3k&}8SzjTLje){qT+7_PQl2Br^AV&J%sW7i^*24L%o1Z zd?xU)CVO-Gd`X(2%05Kb7a+18RcBUuZr>3)@si)J6&rBE6rEXhc<7Z-to@3j+wQ(G z6Ic@1BPGhWag~De93M)R*d#eC;+>d6%9eXf{eEZ<5M23QHX##rC$>1luM(pB=49s^ zr`Ydji&EK6wjZTtqafJ*@fq>0-QXgUgU?wn3|szVis9Dw>jMhMsGpJpcxW#1>h!As zv94}!0iGO$ql$~X;XKV&0lyI?en9b+u}JLL?tw^m>Dn@JD$Ww`Q)rf5l4$|OdBBD( zNI+x<N7%F^E~jX`*N0<1Z35C~JB}W@5@NT~HrLe>9IM65vG~h%$xX!HI!bY}BYWbZ zNd_q3M{oDi|8ev;KOE66-{_Xh3$^2O967CSqq>_u6^{M&DA}JkJDteE3mm^Z-F<P> zO#|I9!nM%D$^~YNK;Mj|IRj?&zgiZBT=~WNOVz6f#opnRE{{$LS&AQo-|&4w=r@z= zfmXjU=&XKnG7zEoL=h~3qpIHM_+fupF3wEWcSk{b+#$~e;OE~tirXUTR9mH~;4U!* zZE;T31+5uo-#JQuBv_^9i%d+-hMTYH$Zq@g`**bnjNu3iUKN3xPLKSyoGO4d{Mz-a z`7ciJPf77#3dPOE1fF0aPwRrjv6%1$62+a+#|(S?)O+)-rh)vC=k*<4$c;?*?BjOc zhjM<T{i=R7cC!-U_Npkd@7gaagU=K$0`Jw7S^n2|FE8|F*x>Bovf@#L3y!wGjqf+s zZFvX;h1Vt()=^Ti0krW&GA}&@a+XMhPcMq@T~Z7eE#+6A)5+OSp`ZVnNOSc3-cQ2c z7iQn3=wo!)4X<ZPBreO}P6?8}r|!^YZ}zuJB2|sey7~{nHH{Aqny-V^gJ_M|*Tm!o zsxx1Z-|$Xq<vr??-|yjKB5MX*Wqkww#6*E71?OMBe_X@gURC`QpeYY>fv1{qCJE{b zzCJx0T7UK}w*5>Rl4}~@riT3a7_v1yx$Zmi>E4U`&*p6U6^;Z?Jdv8So$3CKI6k7C z)sffM?GlID4Zi0mIKJk>&z42NiNUBHtl}_C{?Xv1MKz<@P}gP`$N&)JiW^O!eo2?p zW#>uRo!FJ|x@F6=LLj%IuhAT5o_u%GsiLK8uefX3HFq%T(Si1jhZChg8_&N+tV~MN zrA4s&ITS}qH_uVpp}3ez6<DpNzt}!7>F@#A^Hyuc!n@;#p*a0RkM2J_lRE_LmbFm* zPQ2K{aU8)&@iw!N=;F%*K>W}oZ{RwhK8ZYzMDppq_V=4mPAQxXbg|Z!^M5ue8nCqL zLLi6pu<+fRsD4W9D?g|EGzd5JfAgH!j-sgTTS{9RHu0}7hfJ<NeY_8t2ng7$CsqX5 zHAsVN(EJ~J-dMjFinmDO^xTGc*;?ohf6SZ?=3}e5MJ8RdGHKWV5>Jm0KffAn`fY7m zvKBC#_~bEv!eeokgnQh@Y6(wlx)=>pqj>CcZzn!f<aX0iQ(>ijMPFDF78XXz2fJ5G z_u&Jd?!DKq*xRlxU;8N<0}|RHxLB^+w~|hW9?484w;oL@F0XzqJjI@EJ>P_5rnmhu z`{fsn)P(K1Q~WgHF=tjkE+&;O=asf@OC|0qGm7=fangS3^0U@Pah$}(dNs!}!IW#? zmPDYiu{UcYi9RMYqm{Q;jdh7{=MKKHXV(&U;VwK=p+<6EpksXwD$_R2SJhF4M=D)- z!&zs>YDE?@ldfIDL-aR({df&CqVC~Tgip{#ZyB_RDa#y{1$9I6l};<wOA(WGF3!%i zXU5d?oYKRE0)aCt8q$u)DMJ?O+(AUChw--a&{3I-X9*(O8S$1PjDs{RhbGreCQ9BG zhT<T_P;)Yy?2!%Dw=llf&`TqS`Pf`k#RfW)hfD%0tQORW>i|sS@J5u1u%F)q-D;R% zmK1XOpnN3kfx@7JnXuk`$?^l=z|jv~zt`~P#&0^%R_;QMA!08#eIB0!@u)6(OXS?0 z8Kp-QKTU97%`w_pf4Ww}@?a@2g#EDb@}Ob{k*8TZb&QgBIWpWzoHXmaOi);!6Ku~r z-D)ZawxCrUK}^G$vnwthrpDy6cGVGluGyryfA!v91v<6BBmI);%{1J>%$Pc<^Hd<$ zfJrS6x-}*8HDaO0LLhf{$WRZu{nUe+7fdUvsmm0OU{PltcN{V<VdrpH(rV3e!0M-- zY0aw{bCJdei&5BVbCJcTZ&A`lZ>e`WO>W(ebWoFZ%sz*iQlsN3tN00iNfm(y`TS`+ z-)9`SXd>+%bCD9AZL|;4MK7pGGzK3|Yaf**IBnTQUjZf!PKQNN8nRq<SMxYY8~kjJ zCR&AtP=zLnbg-UPBTH1xmWFUWf+kwvB-~-rG77lzFNA8xUxkX~KMV62se!NL0Po_7 zq@t{Gg!(6rMDH!M8ml-^J7(2_-fLX@%*C8YTdCr5=M1o+*%z|Tv(QU$S_c#Y4rb8K z0@e=PHxag7hMtA9&!1d`5{!+sk=lC4x;E(_nUTst(u5YLyvNF?{pqIkIPv2lXcb6W zspgR`sYufkjuGb>emYtve${^Cg6eh19OpIK*Oz*$?==WB&)m&Q8ue5q8mad!?=NIS z^*U`M4X86X7o3UK*asEzBYxBdN19FAZ2NQ4`k>{xXGJdmh#+=3vCW!Q2wnBkt}eAd zOieuxmeG1iTssh_HG<FxtxqQp7}!68)5{)Bb#!Ta^dPEHF&o%tcw+pv&S&=}8m*E< zU$Pjt3yiHsZ1pQ$ofoid)+4vxYb?0YMx)1RqHC%qMnY!C?+uLPSW6uivY`-lr!9Pk z@af!@i5aUv!X~<Nezo~OjJ_z`BPa#kmndJL%XRZt*;W|j+X>&H+o^=#tgb-l!h89z z=$@R80RB^m0}R{NtB9njqUrvN$$mIqeBn=`PtNb{%!bHCnk&al<Fr6xoJ3y<y_~D; z{EeaaG^q6I$&JZaSC_3W7p<OgXzOcS%`^VB?4K%3g6l!A`uUqSm76tnL8rf#U2a=J z#Dmt&W*m>6h%s)vXP?zZsig*AW*0&A?MND>_gn98U)HGez3#tAaBWX>Rz5Nzy~`8@ zlWWiQufg$4qa_m5VNq%b-})YfWZA8LXz;dW-vv7trCXs0b$W4ZhC6b3PORw}del~0 z?Bn)3>E`DXJhxUO&}!osYZaGk*utQ-Lgoh@7??(T5!5KtTZE79w?BoQ;##*?G3xd@ zZ78n*?N|{szvN8gD02290j55Y<Y+a6HuZ$AI(9#m9Y**+KFZTlJr!ez@aGzMaeo}a z6l3#*)bcVFSF&|G->0Z+IhJghDcmh&gE9=R%qax3h&+BWMxY}Rqq0kDdvXzNdU1M% zdGq!g$p!m5QTOxOeOT4G@#u(O&D}1sN5Vn+)VEoS8g7`gY&@rw5E*NmDalCRSi3rJ zJOrDpP3Hl3FTlnC#F4$(0{;;q)fADs=xwHA8aSjt&cra>*@|V@5Z;NViH@{gU{aMJ zR;dgEUc8TbFw`cA(v5pJZtmtc<HfSLbr5OErwZK#^L;l)h`=opts`Z#Zq~K%hwc0n z335Br)c>{w_;;jddHj0t8|NrA71C|<WLk9ddJ38Km~MB%j2P#XR`GhET$i7Xxv~~s zm;XDdGZIc<w<8T3zBae$7hR;%$bm$Om8j8Dh;_z^Y-!R{j&E`g$l*Uifx>!zn873) zUm-UxJ2BbQo-mK5^S$l~g(QDb?$2kUM|xXjadDTfakpWg<QG2~KJ!J?mV}pTcF?{= zD8q+~z2qg9vZ910>o_lH-fk^V#_I4SXy`YZ{6=;1W>3N%9vz@^T3aWT9QCpT9!TZx zLDX(GRv31uIxpOPG5@=+%rqOD&kNr}y(tpsfn=T;RC8#rXAPD!pMI)3224g8?Wi3) z>V@lP!!U?5lZncYBgQpolSX}voXBt)g`R2VxXgZ!czS8r#2OcUr^@mrD39Ty>nS&H z%jo08bNJgbSr1hCWpbX0C*-SN;bLebQk^En1oeYQS@F_N5Lg0am*$ZBL@RAq-5bJk z$z|)o(^Wy0jK^i3@D0kPN||Tw(9#+v4I4YFz`DyLk-aXdA4{Ko?X#W)D1)>DsdYwM zO~;OZgp!y&{VmV!%(kGZ)m$}MZEaOW<dhp>YEEAI0HaQwS1)CnT}p&`>o))qK!Qb1 z%+H#Cj|BF}&#aE*+L1m9<WN+2nn!&)S7+N`%5lX-=uqiT`=4i|&#Ms1N$#`dnAl-T z1dGQt#?U)g*c1EV-b&=u@mWW|*W;j)?QAval>_R~S*kw<Y(XVa!CV5$`B~|E*D))< z<qb?Gv^v)OZ(a4^wq}*Ppo4G)j%M@qqm8lOT0$f9*I@^X8a<9q^3o+yh^Yt;FXv)m zbiKpUcU^podi&R|kYutWWp(0vn!NZ`TZx4JQ_>9;uX_TM{CS(SI1<(_B3pr$oy0x6 zkkV6uf`@!CFkNxX%o=mi;D=vhR1sDSDLzvsti`L6y&NePtuM5Fq?iZVY}nq{Pqkf> zEl{W?m00fxghU(prY6GVZGXjz9)zd(Yx~``Jj(MZuu^f%XKwL{_<{$A8bVt>5ULf8 zrxHAP?`(iTJz&_9iH^90YX(Ckx7IpNxE@9$7xE1umaB2C(GA$I(AVqz3YE$SxuKiu z_<=jrkgqb)3)sq>GLKb=t*vbZ<#O*tSq#*(TuGOD<B>hq@@k?w0&3@EX4RNIY&<jF zykFC`DYc`-4U^26QfS@@=av6Vt5NN8vZNK$^E(^aYqF2e3EA3B-J7YJej2>-#(C*C zP=vVCa*8Cdto9!+y_2yuVWDOnlCzdxciM2B-nEvg2U*jN&9ckS5vYgse195De;j>) z`O)LN{hX3%ALBX`3nUj!U?wpn>)|A-^2dLk*+bdRSlcR~4S_=jnwx>gYS}GOtz`-r zY_t<bdEHy#t~j+(wr<?v`D4&D*!g0u8(TCFc}4ju7Cb!{x}D|Bx^6UO?HxL$Dj^&b zZ%;AENZ5ADsW$&mk&~I4z5e~K-WB7<tNC?v&_;=EnDT*yQ(|$B0o+8o?^ecF#jhDN zw#cCUNG|UEd@J0^z7bqM-AFtFH=z&lzBc}+nfG!{)@sC)vEodB#C<b*`Sfzg(^UX% zPu{0=R-=^Ox+Gcga#@4Po8X~{{*f^dsC!xLvBs#7+<NBM+HqDk{muw+<^L(K&SE%o zc;sn8pURZT;p?GYgH~VLwX-6(;R^@&+%_|Ht^v(c`Y)xLmC;|Jke1~tv4>x?kdLMf z3QYa7Z`;ES#A1$+w!s%EhYVN&@J0n7eJ>0?s|%h$_#L$BnAKmO?M7iOJ@TDItXz?U z1z%#BtxgOpkd`7r$?{s<7i_`jyPFCbVwRal$np(IrlNyaHmjva4b#Fc&bO^DK!eE> z(aHg%Rj4#a|HW#VtGaS9o5EdX*RK8=vsHa<0@3Ko+60}eS9n-Ye=DX%8@<EC&JAW; zr-dfKg#elknP3F7+dW7CF?E49!r$bqLKSkYz+YK?QEr6Y*NRb=0i7<I>;;Sm)BXK! zWm!av3Lf8+ox#x#wdtV|Ry1>0fI4gJoNjmJGofi;HT(MKWVh*&@eS8YJE6G}#Nm?R zd9#gP*C!F5fuGM7WS?)2ZCqOp3OeQyxM2=L2eiMmD#v`DmL5Kcdmq8S#wlE#BoD_J zz5C)tHRQ>4&*#5X<vu&sHM~npY@lvuBNyY;-J@pPzY53=P<^3?E5D)g-7Orr&@E9l z{AIRQc}JbWqE`C&Hp`<ViW{0qpOj5oBKiZr=A?vFO5yZ*wcd(OBfqaasRdSee&`g0 z&E}ql!4PfO7zML;edLFm^v+&NwPOvYPZ!vo<HaIvjLpp0tQJOLrgo>c_{j5{2AulP z3B!qoJvYT?ZBxFRQ5_vK`@m}(gsHk9bN`hTG2QKp=8UK<6_u3%q^J{3!gfBFfR-XK zy&tWSA0{J^S3@ed5ay{oFmsh(-3>|O3LkRstGn`#zqW+*{7zrjN=tkYTpD$+LUTv= z(X}8`9SmsG53Nj}en~ifK`oRH$MCGbM;sWa1cxUdz3!s9(Qfa4!6oMCooK+@bglXe zB-B^>PW1(#%W>)6NF4m&fzyz8C+WNpDgd`<ktmjSV(l-=g;FkNwlchn{k*@rnweHd zI}z;F*PWnH8r^<FFDimt`$?Q%3i=P9J+@=5N%geCOaf6`m~y3no`ZM6JL0W&y4~gA zJlVJ7UccdQL7k5cWBPsciTTw~Q)}cSL@*7n7vdw13#@Wzo<eW;n3l0A$(Hg1TR9@B zyGY)I2mUK<jSHM$Da6u_DWtyCY5m<{Df#JAq;pSQ_YAfy<HUvS4~er7_+zYp@?EP% zarV1z6}gJR)#d2?16@l<9SP`o%IC^p|3|<;Ohf27A}ERacckAEo#|?x2Un9-FvOj5 zPjRkka#47*F<)o7N7N~G_bEY{C+3NqcB*Y2`<qPibqEagQ*<willcwDQ>AbR^G;q| zCxYu14%-w`l;4%H(Ng0h(duT=zB3uxjHWzc-dn1)G}9HOQPwq3z7{1tjSqNkP)U9~ zULRh64XHN3<V6<e75Hr6DTi*Ho(4tk9N^G#fdy2g-cFC?kVpQ=k+3k^91+q(qRSbY zn+)>AttMBGqV2rz5Oxv`#2;5P#J~0ki>lI@W83!WSfvEaOr&^w4b<}S8aaH37VOm0 zN@B>;$Va=E;%WKiNS;^P_h)ISU2jjnig`KFd>G{2=B?u66ghhix|{>)x(2SB)d?D2 z^F3_X&)lBG+z<o0Iy158Edb|2z!#{z>s7mhs!^ohz^r8!r}Yz4TbOA&k&uc&Zf350 zg!L3OhXrx|#p?8n#`M$du>{@to;tt;A)RuCxh*PA^?Z6&*Bdyb&3s1g;KQayEe|oE z^EQx#jfOEdQu!N)W}G@EKuQzB^`Un4>f&Q@Sl@K&*D@SZ;y}nZ7H3LIzpfOf?!~lc zpfv6DkeHhm#KXb_f6jV3Pq!rX{ekhM!j@a?D$vp?)vYtnc(BnuVmi3#Or@50;ycnb z4Tq}hAeV04PbsrwZ`6wuf-2kvmC<4{{jBpM{HeIpB$tu^n*^+P(6V<L<q1R6z7h4k zF-7=v=*l%RrVuU7b$s9BJD^#2w0rBb{9JfAD|=26{F~oG^BLvARR798&=^z{NH>l3 z76k9{2LHk^T<i`qBb|sqn$P?hA7+1}K1r6cvp0+F5plYc`^g2k-It$FKA(a7RP~v= zU9S#Z^<3Q|%_K5AH{kT(9~aPzU#*R&sykewsvo!6fwe{j?q|#uB@H`Jg`pAy?gLdh zZsj?HCf|_dkkOM50Vl?lANe#V9Z>`1JLw;?X0lVi8&=l(->ful+<)9KbKX1StRl}u zT>UgIE^N4uoNAo7*VQa_Rj=TENY*7`C{aF90?xfeFZjqiSX#}tQY=bk>-$p|-aKAr zqIsfi0hWO?3{5Z&i<>Y{`~q<S--;JUNG7sLq-qx+_XLo?@K4cdgmIqA>42R0Y8vq+ zQS#-b6NzsTTdE@drkGEU`#May**0urNr|fp_9Y!??S;@Wq%;9XmJiHYpD7J#iamQH z-yE``EKk}e_hPg17Wg}PLrA-aF8!bjR}#6xpwH=Qp)^;!TNT+UTzCe6KC-d^fricq zDLXgC$CN1=mEhCD;PG+gy-q<$#l+;2)Mn#sebn7kJ9nn%*jSG_y1J&R%{1!aruA== z`&Y=LWzNDlAxzMA<Gi7kV|ZRY$%%z{oXohLQ#kj>$G)wwT9p7U_V}4K-y^<UP0Vf6 z=A&dmG|wf|_ZKq9PnCLjQ>(<VG~dfFkcUo4%o5k*%4)ic>Cw^AM;}xCJ*ePie7|~C zuVL^Mb1c+E&#-D>8>Yc`qh78yn+78Uy66Yg0;c|@l9=4=lho{&X3bqGY7*6QAdL@P zYP7#FWI->>Y7+3Tz|uAYEwPeS=v^+2GQD%O!gaCX7D`ZEIcN7ts&7vDVGrAU4gIa% zErl4ADwU#>#eT>wq~w+JoTKGncEeArUP-9Wy>EUB<2<9O{{=ny3)kWpwIoH8ZIJ!y zd`h!Ihy6&)RmnGN?<C8eYn{cBnFBA}hMMpwAG08LS;{i3Gpt4L->1=4eyVI5_w=r= zzW#*R(;?bgdPssP2PyAUR$!4DN5(tUF1~E(rtq}|wtO02c6=H&2G-=3aEDKuFND)a z!mh$DmWocIoHMj}gan=y#9#TB5mYZ=!Ff6~>;@d8!oRhYr_*d(Xrt<uj+IO>RJjJ} z)e&*G#F*lV*k3Cvo4S`u!mfpfd7_i*@h{OD+^~c2j3BUqw?M-XT{PMp;W2qNLwcX7 z7<piZlvKcVT3gV$r(XLO+#=Zq3}Y}X-!BFO!!}z_`M59|gOQW!2EP{bKaAt5+Ti>H zPJc^{)D5xA?|m=>Q^dIPovx2`Bq{;mFVMpGSObz>bltId@SG~nVhP+Hl{c;_c)*m@ zhS_0yuQ39pzS$6yo{l(&=(TfRDW`RA%^M=_diH<QfY~a3N1$Df@Rq70s3E3iETd(- z%(QXrtjb-f+s=2^`<cJE9^E|G^V-q_1Z!52K)jG?M{Fr*;V~7z@Q>kVZ8NM14|FU= zzXq8>FJH9G7oK0=EiOxL#eUHt3FwwZ6_$lo)0gFq|H{^OGOOnjbs~3kqRX9n+GFSr zqbl=X;}RV9oLLCPYZM#Nm>`s+<77_$ffj$~KoMqr0hCa;v;PNVBb*>-;tH!LRSf;s zC`~6pfA=P7>5<?#>e0^A*@|g?Zl4!ThM_9kTE9`0F0bCu6MhQho)x5P^Hs6CCK1|# zLvUO?iAJ#e4nhR#X1<!XG^JGgFb#K#mwY*{eO)%<5jeZFB%}IUgM1;v_gtYn(-LNa zch=|dsc&;sLpnMwKN)e0Q>x_m;H;pu)VK?6FYDALZVvL*=;T<S(zN*}yg!ayeHROJ zR^W4|eEIzB2X4!E6$9V>1QxOB9Dh^GHB*$F8RM&#=gBh0x&BZ})`D|2PIKc08)CSu zN6-(hp5OSzU#jZeK0%{KJBRisTlWRw8>X0~Qt89Z#Xqr_fIu!Io`eV8cN27ql@_l1 z+Lg<gnAvhM4+9yJL=$_rmd!g@mm^OTH{acKzldjAIwe3qm<M^@>WSzV%pD9k3^Jjn z8))%0rUvoFN_bRQXVrXmj{pVvfY*9F!zGmBZ?B|8Ddc0dB)am|dG;S{zo``-I?Idq z!-zRuM>Yc}vA0leJcxr7I-#wxknI-NLGUEtVK7X?OnD@!Lek5L`G#eBFjg~oC}8pp z`S_kP(a98@soY6TP<*x+pOcfXdx%wcvxMS7=})cKuT#kbe!1-@w)Suy#8*Pp<5=b$ z(yi^e)aGCR8>k1IEdcIf?a#@g9A{Fg_YHHA%JHlvfpTz#-(Tf!sKvIS?Ab<*nX;}0 zgzL4kOq&&9*ip+#w8y97yQ$ja^zX!&Xd}h6X`=BX#BNT`j}s3jJtyr4)>Ne|RxQuk z5Bh_5UW>-+n`MZl_4FG&{PU&O4ardV@I>V%833P$-D&>_|2*CCj`cpHuU6anp^L)$ zyVC4<2Q>&;=?JD^p<%4o1p~Xz9WvV}bk|N6%-$I_RJ}Bb1MRn&hO(WD{q#WiKT@KP zjO!YWV5loS2`=0CN)NuXC-h)^4Gm@sa7sLwlvTFI61)PjIZy+iO}sE8PCoK&wA|rU z45zsN1gLcqs}UNxRro8;-wU(^MhHi4v?sLEywzgj=KfUx(-`|OUbGo>^7b-r0DrHq zvw1}1>VP9s+c)mude;l`uEkmLNC$}yT89lt&@RdTu)N8%xrCBHUc6N$Z*@QOx9RGG zNu`u^b%_|WLi$0NkLn(O3H4#J`ukaT^TR*=mmB?;goAwD)$H$Py&(u=;K0bPN;;Wx zt%-7|6@J;x{Em=YDHx0_RqFt-TNgE{?ByCvA<h?#wp3woMJ3kn9{Hfd<SE?PuP}d# zW94C>nhOjy1;FIp&;Jd^!wnQj!#3Y?ROVEF8=1M7nt^YqU%RLGM~(-5+e!J=vw?Ac z79tWNs+QzOOMi4+UMFY>UjNo<t*k}5FvdgqU=<hz#uEAw5kX~^E~4;5(3*jckM8Q4 z0fn)(=gNBx73T@kwBMwR?V;ZJon0!7qyybzwe+OnIJK~l8KugqLX5gZTAtV~+K2ed ziA3CVr>>VvuFR$m7Ak~KUQpfW_M5-@gEL#T<LG=<EtG>NFGjv<tyAT~Apy}IKc#ue zU0ESG7H5EAFaurh0;|-TsO3o(!Za3syKpnpa<i`~OK>eg(w;fd%AoH{seIx%Sz$?i zlT*`|4cIkBw<o;sBHjSgbLP>i@XN8CgqjZ)kf@DLF;@tGA9z;Y67O)6Mds>!pC$f* z@a7Ju7)J&4jX*l)jR1_FsZ5NHHX60i#)yDJZ=0SFDJkdgajL}zSr|r{TA{h~ssDV{ z>a-RG%7aqcfX~x|4q08y1y@-P5=XsCZUPI8O=FHXKRXAynNw&ftjBLGjQ>i?uHr|Q zZu;8EHA?S<geqeD^?H1R94_}CT}~qN1_=x>R6(X{d2nTgBi?h$Hv1O$iZ?VKSKm7v zTlj(jH#A<7kJE1q&L5efl`lWw9lc82gspbU>tm!Uun<Kikj-@S{X30Uq7i8Ovim-# zJR}VegLe*-f&W|!DAtqEHQ$h^jZV(V5abRuZ~ttavcata$9dF5XVB!($I(&LQ0A2q zhe{m~jT80~Qpew%KH#ZeXN$(~f?eAry+e-eRE|>z=fGZ7+OeYxCVSPuO*c{QDwTq9 zA!0N{lXllQDSLIyILX7hsqQO|aSdFREoD3Z7{46#o6UTIf5OhFtj*~f_X(kK5Tary z(dF`lUWU8U<x+rGIaYorCv>*>Tf=L#$~FImYxf^|(Ny0@u-M#R%2SqmOxMXTfR;O^ z$yh_byIHC!%MXt1Pl;xzdI_zX`U9+uRs7x1`u$fh3L~kiMIo*}L@bsY6*Ay9dAh~? zt<NLMe0P8hzdtRix$1JCWuYSCG0jjp0k&7sIfnFpc|FxoQW??JRZ|iN^GUc98OS|z zR{H$=Vahp0+hQHX!v)%pDENa3$YL=1$->8tyO#oVJx|flNE7J3ZE%yuriN9$@Xj1B zhM<-6EOBZKWB%#(!@}~5ErwCe6Tgwq?h33m#3@M6sbo(6n%D{-Sm)^vOH6_vh<oM_ z-8tUpO~p4?mEu;nyG72wi_}Iie%e)j3)e~QYJ3sg+NA^qnQveB>^Uxt4A5>aLs?CH zXZD|}c^VmTw9rg)x7IEbPM@mUwfmA<b4S&w)H6D(C;4pV>Jlon*wn@H#yw!19y6Y- zzRIq{`l?o^QvAQ5xQw_)SEaIrMvbntZZWwOuT3aa@tWhrFxl+&p(bj0Z}DaeAs#K> z^yY1S<B4@LiN?Bw;rv$RS}DT!iML8ij`&ym-PA&I(n<*@V##`p4%_(FT|z^Oa!UL! zr?l&dOG2+$?5N2+$n(>emkLhaS*tCP2Bo_P^vg?GCBoTL_N$!;$L;QQb*bDKjWI-} zHmXZ>@qOy${-Ft4D|XEShX6E|ZAo89&e-F@Q;vOF)7zvK)M`f13Wwf7cCp8%wlqUC zqoWRLX=P91*-<u>yTNs${`bD0+Bj=PT(&qh5hcEVYpwRDilk|P+=BDf9fD6=s~^}` zQ>IGiE;S~7l+|Roqnz-~&^Kx^O+g%<tTjO;9ZLb4a?V5DKj{OsuACnzF4RRB9Fu)C zC`k1?H)bJatzOGMC)KcXl0V%qSKcLRRc8-sA8GSBd2ljQRAQ~@4C<o7%BN~SGyW`O zP{tN;y-lN#ovV_kviQWDyK|KNsncj@AYt}%lM&e<p7dA+I~_26eE6_5QgoYk_&h3F z!GS}L<(hCLqKos!RLKrVPHaam=<UEf^_FJ)N~tJ9V-blGzY)zr9HC!EQh1!{8EHN{ zM{)T;Iz}MRRqR<)ZFdAbPjCO1uVR$LlZt9?wtc8MgsVi`{Ky)Icf9}SlOqBz8?Q9v za|nH|in1;T&3len5j?(bEq$8j9DJJ>ky(>RYRDXSyYl)DL#uC8FCCi1#va~kb_bOx zTyWfMw(7(<29ci{>HAB64|F3^ikCqUXj{wImshgo`0~`dDBkatb4SANT`pgV@83$b zR;Z*R9V%I`-v^q8lG95~A&@!;N11$vzB`NQw``){P7@EELJ^<bb>l3X7Jh*~J}@Ko zUS<VHN|gyMk){Gjg#gnq|98VCt;()aX3y&1D$=A3rsmkLEN3*8hlqxX2-e8URz$Wy z?iAY_WGCI2Z|G%+IbFzT9M`y>D1>TMz%DobX?6;CA{Tqa5)ir3wPQANd%|^(XoPt4 zj!CPt`Rz#AwMKdw9z#RomLO~$3)%Ae^~*M@4acormPUyyHi6eO885sKZGQAFWw@^; zW~Xc?uA=SXZT&s&%U@lc!d>&(zSD5NjC|Z)&Bh2J>$)^7#8AF=)I?(Lbod1q37MTp z1Xr`iP|L>m9#2v?RLr?w3bQ>akOD217@>S^4NyLJ@@lr~u*)j25BvPCRH<*qi)Grk zHB-YfBmeOsz9HBWBB~~oBAm$)%gAu2Udz-#l+*;KK#!b%tSCfQGuu(u*OeT8ARNfr zHdX1kV|ei-+qPkfP?jb@eR6yn3d|oyZrpBL)8F@Rm;du68);N7GE*mgS}?PA=-$*7 zvxt&H%>*|1I(3#Fb{v$5Akzod3-0RiepwEzV(IxR*N^l&x#MqUZrwABX%(wEcc~FQ zP&6E=t=hJB!+Px;$pgIY{=EuXGq~vCVqHD$W4)v@p=Y#_2>k;~-HFvMw0hs}7vD|S zB@x>(+O^(B`_+p_7iC?}Qhi_7kHk(Xyr}L=2uJ+ez~Iear4N;GPm>7U#z(l#x*Aio zHH+v3Xq*R%N%9y^8v$k}^V?H|w#ic*hC}|VE&wQ_4a}LuTYzK@GmRgNMHFFn`uk8! zbNbt=NiI*+>3mgLrD>)?R#~O?SZ0*nMypn;@90RQS*H{`sG~-E%6g^>Cd|NJBU0oQ z2rjTI>2j<MEIFeYi$?fsh<!+nKH{nwXRb;G37Ks*ep-#~mj!JERQmrT(s=sqqVP<A zqG-&q$Ji6WM|aH9P2ByXsj|6nHIzDF78`qY99t-jJn1dOp&bP{`@V_C(4IpwT*)^@ zOs)97YK(oj1s0KW?iC=&Lb(3!A6E#+u@wh%i_&K8Z{2hp44wfv1=l%G{OTH_(bCcQ z3;8_fzF~OX>-s<5>9E;ggKMcr&C3eEf$$3(KVWR{Ckp^ZxB!)B-(CuW{yJ1>Hdy}= z;Ua8<&s`V-2Gu=S1b`-D0kc?#qf6V=ziU`)dtegu-#Zx&(o1T1;F>DvbYK<g;-0?R zu$(oqbv1PsYxnmm2+Mr)fB4gtIGB+;Z$-302Su82ZwbIZw(2)^;EL*I@cX#G*(ii^ z4FA|etLy|TO9aic%%uD^{RV82W?W69aSmYqdh<pm76){T|GJ3K|I<Z0(O>!#DdI9w z$PPG3|He5J+R1P+yU(!xeHj1qIZZ=2^6o6)O3oaxlKuTiq{YJz$jpGBEjed0|NX1( z|9HgH5Gs)!b%0S~e<6&7ZG9>*)Z*R$27CYgAAp}tZ2x>G{sPE>xtrHNKjQ!SoXUy+ z!!`ZS#kgqx-;4Pl>e=4~+5g;L!nXgsGt0^}(b$`RzhVFS`O4xhip}Idj`sdg4rX9( z3(^C^le@g)sc@Uc|Ni%s{!iptf-}Xxv#(f5bK|rA1x5es8!9V+-s1SL$w(U)jyHn) z?B|n9|NG<q1{#>6L_3WB1BY<W!LjT9U*Am{_U6|AU0na4Y1987ubt1UP&U7)<q@`h zl;rmDixeaK^A^XRwHl=$3+K9u11bp(nIrY3z+(5xwo@b$Rh1~xE7!_|+B^G&f|(v@ zj@c)szOCxZJ)k1gXrRgus5xa`Ofgzger^d2K3(5coge!%Ib*K8)&>9dtluzP>H(|e z-oMfN<m+Dn*;P`Y`HFyz4yDW8n5yWL*8p>0Cm5Rt0yM|&sLlHAs%iP13Ec4NH_lgf z)3w#v#nE1{>3Uphz&Byf=8YegrzCsWWklHPwX6QCb3K;Ws^beF1Ff#iEUy(oj=BV* z=#bH|RL5(81OAVec?&`pO!T8E%UBWtjmNpD3!Fjbn1f)jg-Y_066mhgic@WHFL=iG z!gs*3H@PwCVwOb7VKdgQ?tZV`<v_Q->{rng^3y3tHuo%6EZLe#Q~IzO5!Rx*&n%dg z*YGo@l0d~cejDdFYpIZ=_o-YXGrF}}$sp)%cSO>G0hCGeqzG9S7~p}Cv=!%j#N)-X zUS-m`(X3dixuKLq`9Cj*^-<du1t&8@$G?HNB-F<%g-REztGLf!Z35uLRh&oMD$W4) z;{FR(haC}>zd|5z$3pFiMW(pOUk9BM!04B_;S_IOp)7jQ3@}WygJuap4x2%{kyg%7 z@+Ts{P}Kv=xcSd2&LqjHb9?D`6?M?go8P#onowulkxtK}KuZd^%Xi{DMdz`Bz~6|) z+4Nz7IH_?gE_DtI1Xi5~G4)q-yve!3rR8C*owF}C?9(sfU2IQi;i$@8vzb%OiwZl^ zuFWvFXGf9RoclPCdzFa6d|;#1u^Rh9pgbiAP#ie9XN#mWv}BnFuWSJ<z{&3mFwp>D zRb;lV>bN={{?aUy)XD^Cad#Qe8X5<Wh&$jHbJW2;ja5~mfw-uJ(>xLP7fGctrUX$P z%_lhZzz)t6zn63UPpMyt@&&<3r(RHX8JpI=sU2ERYMzV50;d9SB%IAv8*f#f!2~;9 zXF7*org(o55DOLc(SOcv+UOY9n=!B~4}yCQu5ulM>)f6a0B!bBo#YO{e0K_7w~d!N zMPOZn(wZ)0UYjA&8;vT-7}HC{TEpK*IxtJdkz4+~q;Wel7-I;_w9jxuVN?49Ui<(^ zYVx?yjC<mEUv;0S&fBJ%F50uD4`TqzkG`~{Rs=4l!T&u!Aq?*d=Xkk_6X|F??n$WG zw%R#0i1i!7zSQ#{$8Nd?`#WtbZ2Rgtsm}Tsvjkr3jvDiw9K0>1&G9l<W_f_C*7Sl; ze;4A6y3P~<dO{&!lD3~FSW_LS!Lc(s;e(U4Y*U8+g{^59X@7Au4`iu9NnU1oI@kCC zki<pwx%|P9VIeMbYQpcs-3)Vaulz?d9k5mf=+3pnYT^Dn;+UDQ+Q9+s%~EUIrX6Rt z=<A9PT@IH>g3|L~G>GB@cS~fRc1_o(7+L|G<pj%WSaT3y{6B-1cM4gLv0Wn1O#mI* zrVb<gU}?CtlQF83i10akT!_b+|0nnJDG+a=6S0ide%gq{FC9<_W9^Gn0tL&RFa`;( zK<*^C(E%>eK`;Ass8Cl==>og@vM>{~ma2;C9y#^P%K&Sn+JEj#^=E(TGhAXRnQ#{g zaIw=B*pKQv;3qnY2x*geo3*HgX|6z#$EZ4Z%w`WmH5beZo6E2{e5+C*TUiRDN@Enf zVG9_L$OfkF^<FZzRFZ#B0NtAFL1BFN0HuMeL-<^&*V*L-Gy|8p!qSZs<VbCgzNOw7 ztco_B58dm>5xSu<91dseyk@39RtMmTr5Mxw+>Lt(nKk}@ZKMCS;oDRv8mGO6%Rs=& z^!w_PS9dw8rjflMZ`^PA3TEueCVFvV3tI33xA&h3&C_t&*&_o>mQ%doM$ls~ZRLp* zQ8Uf-7`ck*-GD@<Sf;|D=|e=F4ld*_iR_DrsWyHm{xE*)c&Y3#b&4xC9~umv+w!Vw z(!og|q;g|>@PDGOFX$1$ocfX!14g_8F6!&<cg98y@Nx}|5)H=xz}a3G;(Xq_<!zK+ zlcv?!>t)68rP6p(`=V8uIp<w%lZ9^6#LPa9nPQa=nwKnRwjYZIw$0jp&>C^WX$h3R zL_6LdPIa#w3Vm5KgP`H`m~`uq+)GfjJZc=#!EG<nf81EjRQ$(BvSgjFDkBt1$4bth zF<qEs=UMnobgt}Kw3;*)yA4x%td<-Nt`%Zj7tblzRwsOySH8QA{nR2G3x-F7Uw7_` zrdMD!TC=Rn-G@yoChp`Z*{b!)A2swtAw0pku0a~Ym$R(M`#U#_pUCo9LG8?1=k8_u z0s0L^h3)2j=3Oq9-Sho^*99rDX6#cjj5*k<jHFM-^2!6JTY6dT3<wgc?(A15x0Lo0 zUB-U0pEqg#a|_C*%7IIS7~u-6RhNcT_U+!@*R(i@dx`VZ29JEray^yo8*KH=5;}SB zS{=P#f%?J*5z1|#BId-y3&DG!rt;){&(bq1DjLV7V_Z8VrD)p-bh8tq3LDDQOOA;m zO1G_)yg!td9Qb4BEid<IWtVOB{C*`oL!OErWgV2-Z{Chx&CY)ELUiKDNga9MaDuFg zb#GcS7%n@}R~$b>h%)MxHR<_$r%Q{y)2-PL2!RSf5Z|)Z#mN1DA=%3i68W66Dww*h z`VN<=1@Bj9i>9R$heTDekJnVp_S@Y~)fn}75~LVt(!^jF7LBvwf$8&p|EgQt`qclu z;xFW7L`6DPUKzB<2rX>^hfG~a6<+NuznF5|&0UUJ{@LDbNEsR(r^B6B3d4tT!U?m8 zG6cwVIoS9=Q!mDObfI%@M%@~!I6{!!0LK#g6UD{fKUY<dHpSnKA;%*}kW(de`iiYb zQYfVASKk_R$RH(YgK+2NNZ1UYms&UP8Y@9QlZ0+u*8;9uL*hCkI3UB&lAiLh`W3ba zIex?745jO>MKLB8lzg5ZYQDxWs6~xckHAILf}v4OVP|0@Vvg*<fg^RYWG7i2kAQ?u z0RpbntXJR3q8*<5H>%wkNUSD(0%5RtnXbE%Vcu7Do2%I4i(ZhvnF)Poj*e_@0@owE z!m5ZF>?2oH7sDq8tVu98^TGS@RCS48OiU|bNY1|1tW28d01K8=3CmQcfHYyE%ShHD z7F)!}w;f)soh5;%8fr}T$jtIR_@)gl1SMD#lMSX`d2A{rM(=k!;v}aTX;;DdDeg7O zK+$Du7?HP-V~emQNOke}uDDd@{cSZ;9+72)y4PvO{q8kZdmE;+ZxoE}jaicU)1F}v zyEP;()h^w_)w5EGQjS1<D3$vI12d=EmwMQj_WPj34`1N@raM{HnSgv2P~6!|MmF~z zJu~^>fDOT8x-h0r?d%Ik>K%ROydeyZn{rg(pK#Q(SjTI*6<QQ5Mt`D}jMetq9OP6Q zHd!60lZ>Q}OVt@uNgpn(O4VFrVNv=d>9{!><p3cWOJ5_9q516mc>j?S7Rm~DG41~F z?d$$+P+*ahv!s<{H^i^@Fx?49SxwJ4hi#4}Tn)954mfpM2RXedEBg!?*6j84Gs@mJ z<2|t8=A3zM*q8@Ww5CchGV|z{P8>0zjSe8-)B-Jl3*p59L#+0iiVu$#^Um8Pp|5+> zw#Kbn<I~w!23$Ay=;pm!s;8?$XBNOEo^2Tmu2`rb`l+AP)*f}MSC42%t!Opnl?u!l z<k4xrqLqF=>BG>V`@dSM%l|U2nY6q)2>aG>Hnlx1HclviVq&L4MrdLj6T(+*EtXo# zQQE^3>Z4+`NfxfO$glTuAx*gw5r235R*^CB^PSNSMT8v_rvwcstPKCRm_5zQur@9E z<ze1fv2of@3?UflU#i*ae54$5R@at-GL~{UM8B)XnbM3Rf)n|T?V?SUS;%an@PCT= z5H1r%t#<7KQ9ZS^Z1}tQ7=^(i1*Ugl(_y0vwALn;G^|Xjx?)ef79G+Jf}o_A41;{= zXpWl@2G|x^Ue~W&+eX1aheA!FS)y9I{mL)1NW`!-@il)$p}SWe!<dN&S~{by7ZP7h zD*Br1F8WJwJ4tX3L#V&(fTG3k$TG_@rqF^wzhCVBHR3Z>#)6L|xbBU{_J1UBPVXX# zqryC%*R-q@ZAes^y?H_r5U#z^>2<oWyzrUeAi8bzAR>Sxp(;%r!W(NVrgPA%v0$UL zImqU}=O4&na7<;vZ)h66AhcJAHdXA%Mn6^KRQQ;_dCX{vEu*$wIE(e|v3<sJ9x}u6 zbS$Z2U(`LJ6`mK+InXa6^+a=KA<Mi*YT$M{SiOwbV-E<%syu+U{$k-$M=<^&>Rd5D zA-b}4hwgdj-#qE{wmTT3iw<Zz-Gyn&=dR|$JZ;59;zl51AiqBVM<lxUbha+mwH+V` z2nO!h`z4w>l)1$O1dMn_39|`s#UP)&V3Qer0@^5MH`~3&&DYWkW{%Hc$1f5J>x?E{ zbjQT`S@GBynC>IQG<wa}=}u@J0(zV8+(^{zm6M(4MWL3QQF^239~2A1xdv>9U-BBg z&Ui{ZNBwNd1{4)IuRCROW+OVaOiqi=!0ywNh6PJ+e2FE4FQA!UoZMrx(KfdkEcVDw z3t~T(1PqOK+6En`Q*-i-83hCLn7|`T!_tG8L6K~3qzv&!Z_<*e>Dc3@E@lSM7Xj#M zhy07+*8_Q?aWkHglWPv$zEWPT<N2z~)Jfipe-qq*iBs@gaLRwzqWnn@%MClhFAS@$ zHl^VSE9fG1HleZLVA$nF3)J69DW~L4Eal>U0*uSaH?L!yNSHYDv@1tsu35AhLEq$* zhtzT0{S!ffzZkXxyAdcWVjX#prkQ?{r_gr1!nqdz>Q3%1P(YZJ@cMt5MjBn)|Mj+2 zAc64xGeXX>+BYjI#xM1dr{H9~RzY4{^&XBA!hE9Cc<t)fG)^#T?wF__To0ww3HqYd z^6(nm7*&yPU#M7{o4!s^0m0d<pdbeY(W}vD9hw(w4WDnZYPJqEA>eOmXwP4W1XjPv zLE<6zuz9T2AymhtVsRD+(8=*iN^Tp3mf`}4g_+@OmChGnAsf=vYd#gcFOei1F9`eW zXYvf12n_WuX8hmN1v2vP5zKL$ku`iD`Ls+BkeL0O?(sW|=rLaib6CFC>y8!7=qSnO zdDp#P55_VA{0}!?hI88JDJ}f~`2X!>$UN~&NL1p<I%6ts?kgQwbNgaP8bXwwBEFUq znfUh6_C}cV9zX(is1qAUt1yfgvPj(*<5Z*>)GErNm}eMY9)T&kzmK6Y_t08ixJ`XS zpJ^S27$a2CiScu2o6M1G%mv%pn>Bv6bx+=wnW2j&<!w*iO=U<NG<R>*D$k3NU3<2j zxba@&RDkPKGxj?Qqk8!wE4kuShsSo-YU#`Vd5f6(SlcK~M?gpvWSMR$Uomlz*lIo* zpc)|{0+$<{RO=PKfE5u)AU&4)H`&Qpn9}%-F+X#rT$FMUq6*cWwCJ>;TDs@n_y;_G z7l}(Y+s+g@sC@5MC6`Uy+U@1>Eh4`7oTS)!b61)zVPZ7ibE8kBjBT;QLxmUX#5U_c z{@8NDxxMcAhVRPgxRK(APvV+ma1XR*bW^|#<JvQOs`&JCO;Di6KJVt9S)-(G5E`|x zJmX1Sz$QDNH$In#eSm4?sh3(aV^hr4Q~b`pl<CIzgB~-_r7#w*a}epAe^th=^%rUn z;9=Gm{{<QabTuNJZgEy&J@79@Iq+Eoz~{L4=l%d3QLRQrR(_q3e2f11qCfi}pHHOV z4K*p=<LYm9c&#}l>HI1-fu!%iTtu^WH5>v%43onmzJy^!U!B&U*hM<<-89fV^q-t( zk4CHPk)DE$diw8lgNQy5p@_VRLhR_szL8(bis;8UNc1&cyP2br!AOFigP@mq!I2z- z^or+RNLG1M`!%X=P&n@#C`cq<U#k*J)lMtd$TiSU7aEIW#h=5z!{W8M&VP1VNW*Io z{UofBOFQ?CH}1BF@=W4q-kn&Ff!0eJS+tDe7sVDvq~fXp4;zzXq_XWVk?J=hJF1P+ zmW#A3=gR)4X5kLpp!=GEhd1-RDP&$9&qdT7uExcn!K1De9EKE354;xC?5?MgV1Lqk zyo<}RCyWiEkxA4kRBC5E<BOC~4&$>IB;{5tvbt0&99Kuk=qf^Y^%kzrh?*Q<Zvb@W zXOiiAL=d|ZqLY=ufTnee?Z)JAF^$cX8mzVA?~;=zo5sX1q0mRfes}eHOb;uZ7k^1I zozN<92GTr0gWCm8R<1o?WUE|bE=0tqEfhGdlRQSbwH5^C->^s#Ie#aF!>gpvs(xrm zIL2mpOwhf*6c&4GyeOYL$ox~U*Q}8{!M7s23sS!E*eMJi9t*}B8)95%dx!cXkIW(W zs-Q4{&<7VC7$|HSznHssbV8PUsY30i&*Ewa{aq*vkw&h9+Q>Z&+;?X#H)qC~`}Cqi zb;sTZ`{c9}Fu)A<;**uQ=0~GldV~Utf6PSp`<xJD(tpTUGQl|7-ru30GZ}kVPIBJc zLJ>U6t7XI8=vHiE#*pxXan;oKG}k0N+toAbH=-Y7ad-!`S;s5{S`-KvPcv*aagvqu z!TWyvP_floXasShY7Z|Mk!rxWHi{Bg&+KO%IYZ!50;)sf4~|fV)DKr|xtZDW#2beS z;IO`-s!Qy?@c-&&{{yK0TZCvs5OiBVriVXQhlj|9<atPG+}Pvx6g|5e3a5jyE-FJ@ z%tST*xmB<tE@oPMEhZI*K(GN}1`+%Ozo>*#+5~d{BgYTIO*ki?aJm*vva|Y1G&k6x zpH(oIHuNopzEu<I5xu9t#6y}^#5&!IB(W{0A#L;m4>CtQaPZb275eyZl_$^g88YM8 z2$zWZX)VLLj9fl7B?tF7sHId==fcs$zbt4wNUzWH*-*^si*bY#whyZc4hmK0B7Qz; z<z<Ew;4k&nM20jd%YQ`qIIZ_oOTQ*~{+^Lf$yzSdQ*7w$UZnMsJ+>DCcvZTEjVh-= z)g>^;>w&p_gV%n2#R}Mxwh6?=dVod3J|jegP9R!?)LeO5T3&(|X$0yp7V<EHs)31~ z0BbNIQ5ZpdnEtG|BB}aez9G~xwv~pbYw-Pvh6W_=QZu)T9gjn9M)L5^zSY%%)n@Z+ z*p2)<A7gq~>8j8SiOZ_k9+B0be%}fYR81DVvjXMuzAbl_X0*aLS*mOl?jpE+5;cv7 z&sF#^jVfL<r*~%Dr|VBtXiYcV3+1MYdmG{%xCu^<b9ejF$VmnALH-A=kJZ3j6b9`t zZnMlhkL7BKnvqp!Z*|$^&*b=8sv0x+w!sl9x>a3u5V4)w1hEzfF8Lw-!9j4WEBS;* zVOPg%81yIJ%Jj{NY+wothG1`){xi%#HkZJO(iB}7wafJ}IyGQn*6F<H)VQLW4q)C@ zu%t_dZG~LzIJxW0ED_3Kn)9Tk?q02ASRFSlz>Nn~?1=UQp{4{hL0xAYPOEv-25afM zV?Ak>6Wk^RvS&3i{yRE>@zECYMF!hqi+Px9BVW?&53<aPqPCxvoMd=DGQ*NB>}^@f z-chm#dsSX0^!vRqhp@cYSUl*bf_s+jraT&_1_Wz=BldEj%bEWIaZg%4$+3tt<#<wA zS3OLq3T#Rnq_f~WLhQPNJt^5jm~xS_xJ?1#=feK@eMSylzg&Y?Oso2B#s`1|Bz{Vp z^E#mgoWFk4ObHEABltsjEvGq0xM8@sNAMzQo;x-N3y4g{Pw4}A=ss#In_@nmhApGa zx5rDMd_)17xdtJ?dW;;U(@$pmA_O%?=&q{JN8^cCTQlXxQ+;sA2dQNW>OqfnNWRF0 z6NV|{Wl<z@-hol$?_vusOx5O71|7`tXVu>gc~w@2p`V!gC&t4dzvy%CTL3B~HIH}o z_g@d4ZH*@UKg_)cQ<GcUuq_CJf)wdBG)1HdD4o!oG*M|%qJ*ZjO&18Esk8(LC`Eb^ zRC@106hi1A(p3~f=puxc5a7GG-)HZ4-uu}<;G4;00>gx>ta4s!9p`a=qov;L!=}7; zvfGmysEE~*P3a?HW2O!51EEzrw@-Y5qKlhDZpHVXM>noSO5mdin5X1jffWc>(1wMr zRBeTfKnK=JUi*t1#9k-L8!C10TW|Y&gq|$nJGf}<*-C<Xrhb<5t;xocU2C1}3cVwQ z9yA~1e0T%|Gl4Sv$~sX3qti$@voI}}AEe~9->#w6+LUqu!p}p`L0(5<Pp3#}f(7Xz zQI~PdWVL&r@OF}a`S2X%mOi5rE;&<2m{=i*RYY55mhO7+I@JcUu2wS6I|+i0*At9m z+k~!FT*f~k${Z<K+B$P5DM69>!o?{b4k3$1lT^ptbSkK!;KlMmD?&2k13f5en&ebE ztNa>FtPCy(fBDMYg9gC;bu&*x6R>iDsEC-Y08NDvZuCMhusT!vddv;s^r69N(5*b^ z*2B=190sE#%;LjO9Q8d@Pr=E^j(XQ25jh}1%zO?nZZ9a1;8u_S)0FG>)sh*Bzt~m= zRjYgX$Gn>_Z-Vh?Cw%$v-S&?8`nHQsk<T;i^B>b6&$GN9GttL0&c6+Xj)8qUfji^Q z$Yrw?yQ=^yVu4B&qPALcxvPR#6ld1Dvi~N;HLF$rCF!oYBR{C?ZX~QYDZe73BER*~ z`}{i3MXCzwW@iuD=RzS=Rn)`;jbDpv{WtFE{j`YSKndQp*C`sj(&a@l-!JqvEuw;6 zoZ_!eYH#;YGYY3p>X#^q;WFvFQf?9kp74v5bd^aIGXR|VV6y6G>$%F0^^H`UNFIvV z2%&_1CVQw}(I!c+&__1&R`6FAUU`1yvAmzn`3(^rq<*>`o}^$C%+Mp4zhP6u@!M^o zw<`?HR}bx5S-$G$Pwc<L7!C8kCcfpfQHeU-)p3PswNd_18?$zL)4G?aM-`d>tiuW^ z)nKRx7haJbm2R$L2y^4n%m;U@YiodHqInCncW8%&gW}yb=Eq_;ex{P-_25G82>U%` zz6Qe|VX*PF;&+>uiOPka4w*y^sFzW`!`oImx`0<Wsald4R#)MJH5e}mj&NEHOZ4Vc z@%_qsb5h7RjHwG#B)!8$BScAIg%eX>R-vWY<cO0<Fybqwwn=8g3kR>pNWV!S`O%N{ zPK6et(8LGM(qFFTj5{+UV+C55rO8H+SSIB{Z&~b5$wGD+x1o0jYiwh&SU>>L{n*pq z*ufo0YLcEGXPnDQf>9Wf&JqtifhQJWHQ`mPGd;Nz>%Y`s0bk6!?OP)lenTi;{UIn@ zw2r_ZE5MRPV0)ecoupFMZAC>RFxZ%<_M_?$CnLWYG;%=k_*%5GhqrxM&@FDdIgWyU zs~dH8$3o-l6pZNnY?<n+nv^8|oFdnd(QBZ$emi1G-%`e6zg2E%gxst`FqThY&7puZ z18$ObGK*wgDthhXpi?9}sdh|Y*?&I!>6HiOr?R8eYI@$Nian}SlzZq&$a46sZtIRY z+`Zhx1+BcSSj!ma)(-o62t+h#X0<&sy}We?q+dUtC)#t&L?Ks)ALf_K6Z2%Aj3^K2 zds-RmZ@6v15a2-?cY!RoJX5_w)Nh1I9trC0nUV_bU4QuZF1rfjiPgUi;cx*yDi7*l zlxoy0WnR=iWm(iR*^0^Pqs1c&0al{$ii6D!GRj-%elp9jSc2V^N5CqK-d`<0g5H_P zvIZ|S!yBr+H7zMZ;rEKu7H2I?@A8@p8c60#<h%3g;#m*VWoT6gbY6*RvmetPPO%qb zM6r?vp7gI@L@S!V9Q4|MwT)Kj`UwO*AE~1oTyrl82vbt^AvGHlB7FL)HieT^YpLIh zhdz<JUUb<>i`g$xFc#}D*Y+i+Gl71sF5V^5E14l`&*N?@OR~Oz(9OLU=j71+ZvKex z*d!()?NxKj6#2fPz<vSm34PTsHXWUb>Tdd32Et%#Pm9pq7w^V`Jll*+yBQ&D$x|iX z&$$<#UtY0SpUuy@w2-O`@xQ;7=;Y^&pytAH+K3A$f>yG=V^<VDaS%UU^M6}m-nsjc zzd`MKj^gj==Y{8Yo}>@$dJeB<c^=+S6S%>vZ&w}XiEvLagvTYzh$25X)$AAsYM}-E zll3r*oLAW1e=bcnZg7@dNQUhuRLh00cf^pix-)`+<SPhp!Il*TD%=O0{XT!Y8a@OH z(JxOP<J^7m^%f-FNj|-y3?yGtbe8eyIlRq*Mm~@m+3h-+q*N~7>z&8_8M1mhXAK2% zlb?gJ5vp}y){8noNz1a}0<%V^piNXl#n&Ph_(v(MJ-nE=O)z7KZ<C@0S=UJAB7M@$ z?*AS{h=DcqxDfd}b{Sp&hzBmS(7G;`o<2;U7)v9-Qcc}+{`$e`bRdKoybq7EgBtCi zkiR{7`oVh)6Mar!_^-X&YrAV(F`0%{8^gx%R>jEIdrAT4{+MR-i;~je#RcT==}O>P zS0GgF`Xhdx)cw7;{J!W$6FrfqBhL;~7b}@Jw9H^BL#wT#pMgN=GBKtm!;EwTbbCu} z@Uii(mtAcv0BmC__&Xi_j(JjB9st<}nuss1sG0s0`z#M^-m2LD<hBCn4(wmh9e&<- zA8-4<rKQn)qH|uWkg|wuozG?Xv)v<m_L7-Of8YLSqDEc*`-qz<%&YmABkecYgUyfW z<lT8L=6g8@ZwD_O!usBMFELTe_EQy<p(75<BP6((pRZj!oXX(k;GE3+S)-v|d-su* zdQ^9W$A@yRZiy)4yY};m7HLMS128K_X2){MLCT3)*1ry#e%3BxBI<VzBvxYVa4b5~ zjttXOnh&?abX1eS%ed>VK9ftf`NJ#tpw+Kj_OM=4fAqlf8<wV*dTDO}k1HQEa9$b8 zSZ#>So_$!`csan>78Z?s<!Xex_?;(#5c{H^{DmKY5l02N4&&g5^jQO63xjN|DUZ-I zMqFXSH}}fuA6ORq>Au^s-OF}8n$;TPww#l(yui)LxS*?Iz@}mSH5f#t<s8ga<-?Qb zlfe>jKhbVnH04E=fbO7Xcxn>8|Kse9j~B4Q@}n1k6qfj8i&A{8&lt$@Gw-_t!r0}g zm&pDk-sLI0>O3&5T?5VRZ8irYiW+ZKdMFVha<CfVnEP`FYo8w$NG|nz&E8rjQhq^r z^;h~2CoV1eJAZ?Hw97EQ=Bc)*?R1x6@ct$j$Oe1YCRR6BY+03SwW6_NY0l#<BeX;t zZsXSlYl4eu(_kXn@8wCok{SL`1@r9eA9=M{v}da0%bB_t<<KwvN;nHVXtACwlUC)H zm-GsGLpwR_snk0&Q}DyUf{~OFC10CTV5iRSg(`ltx$!G+NfozWr=@i-;x+7%@!sfj zbf1?LS5Rl;Xp2vl8SatUzN@$2_l%Hx_d9Ww^kL{qdo-1YcZaZ11z*E!`O}@bZwd@y zuR-h4_J@~p#w!Br4f8x!y-DRBX7iJO?X84<HFP$ZQho>2*DGp;f0$FxVELbbyKDHx zsW8AM{%>srt>cclrbU-&*AMXZhfLK{Ut%C_#P@z1@A!JDy{Lx=ya$G7zb2jYn3oEl zOjDk%lnuMt<|M*%Pe1FmAl`P4#0?aQbqKCaq5$&3<ia0R#u`CK^A=~N77W8Szj|MH zECZ#7%u33pcwkC4Qzn$0A|?2^zyfk5F@s{oWy-qNh@J6^LX^8TgY81mK9kSZtiOMo zw7#Pr_MKa$&EmqNs`Jaofp$ur3D^%rNEk^T{e}7X(Ed1$p$`|-V17)zroT6yg!J$z z>2zRst}n}}Z2^;f(!A3k?xiG&O#SbNJYrGCDJCzhWh#}oga?@3S7rO%Y%Dt7nR~;s zRMnB&x7;rO$^x@3q>{ZJP?7XST~;rUW1&YTpuchrDPOf1z_$bBO+uay9|W#0`%SD4 zM4-#;ycOgFDt8_B_{w}Ht&P~-Gg|JhTO)Z8EoAI=?dfzHVJW&p9aV%|TYn|3*qdH& z;|`hcTs&4D9dTc}!`NWD=iUQYsV^M3XZs|%+YjZX*dP1kY)*+e-=Vp^3`?JX(5Ch^ zugP?eP>!jb-)Z_#YUiE4R<>cpKi_b{W2xwBka}kUL}jgY;^-axNY**)2F-CpWt0Bu zc+l$I?+$v){YmbKL{!^`RqfNZDxQCufX@HgKBV+WxFXbb(&HAh9mKaTlM7l$IyZe# zNCS;E$*q4YKVvC+F|9=s(2nhca%kOJq9<jcq05gm(ZP=9;coKSy-WxcG{8GYU)O(7 zo3uDx{_Pojayxhvd5m1bboK(=GhsH{qC~oBkXU09F3jX)FmulWQ@GKbdws{pRG$4_ zL6u6?R!!1SKXO??<(iz)h1caD95A{ht^vd{zCo_acpiUvP!1=N>PVRZ9+uy2dR_QH zOK&XD(h8G#0bUYy6I(-x2fY}-;~JaeL{9|gNYWk#?DE(li!hrL<WV<Yx3<@*upUnC zn5e0k&9?GZRaif;^!#T@o$&KS{eNq{WN1$mvM?qhzEAlvh@784FMJx_&u`>v;DhE7 zFEmEJdi&5eGI5Vl#K~zUaiZbvNITvC<@A1j{1<8kNGZ&Ton>dnRD{r-_`LHJ>h+_! zV&Kv3hTZtTjNdf)UuKuTkM#e<UH|_FL!bSLxQw59T<2F;?Lpl9_yLzSVTufwLj*9e z>T%)w<kD=Z>iS2N8|m>A^HxL*l!nLh>f*F%RaMy7_pXS|l%TIOk)!iMVG{T7k9s6p z4xCk?MCUcB_?jf<BcY(bqt7mh!%l#h{9ih@qpqaN*0m4#JS&G_=cDg=L5cH93X^<Y zi2yK{R60*&m_LQL$#4(w(%%e617ufI@Dtb~QCGuE{jiI%GIRNKeAaL9u)%c$u%k7V z0;<lWH@|nGKS+4O+kxZN1DpUz=_%ss{23hQBJBHW^GFt*Biu$*YE)yy>;t41l=N0Q z?0rKO_;{42lXJkET6R=YZ69u^w~TF!O?nMUv)|WEJ4TXR7XTQ{CDkuGCLxNCINOBY ztLSJI3)}7lz_7M)z~UP^Bu`VsuQwX)!^xO=U?F`J^~KgcD0K8{r~ePw2O_vphA|6S zgnu9We#j>9Udbc}X{Nj*`m#S}>L~boQg3_iZ9MbZ2kisM;U|vSkX|W#a;H@Ri*K^) z57_&L9%8FB&%ob%s!S5`@t1*?NB>`51uykO=6M*vsm~e|<~I8<h;5(QMyW|nmkMiz zL!Yh#s9?CTF@xURC7m<kY5a_h3N`%XEZ9s2=rKpqnMfOz<_|5$FaHDJm6*+Y2M^em za5y}Bd}Mhd3?do`|7`rSj4t%TV-Eo1N5@c|vgq|wprY(wB!J@2=W{JOGiQXJC#>zn zjKJ`6<NrbE&lIz^{We{Jiy>zm67Um5<@jqU`1UcdgZ^#y1hCrHA6yGAS79XK{4f#C z8*yHn)xUt!^zz4RR-MYJ=T^)W)^MS{DnDvigG2KF+MBRa2<!L{>IZOh-+hhi9%Yj= zA?`7re6D7#veIY9Ko=0a3qxDsyy2VQ#duh|M`dT!w9{)5>A-}?51dIHI{c;&0DsP{ zjx+mOdkbLdo%3;Dn#M>7^p<mBZVwETCMv;TfTe4^?JV)x%j3hK6?*6O3)+Kd3XF@x z_{^DuR=&y|{vV`IzGSMxz&79&k-V&~9!4C1{2<Sg$MEhBgg1@o&2?>&Fc3bm!2=2{ zB5G}gHvG0pC?}k8We3END}UeFb9<-eK>Eag5$lWlMY2plA_svb@?PXak&*YMxwBCf z{k0Ya{S)3!7qfE1_Gcug6Wq#tp*bhhvE~?NU>c|$REEq`emMh_cu+20`+ItbP4*Q> zAoYcmlHTq_ni+uyN<WU!0K%*BGhgiZNZ4<`aaXl<8xv5hiO4rV#ar949O~=Mj*%Hm zaV6VVo7%y)d&W7?e_lu~-r<T2F1~0(g_*fzxLqu&_C)%zD^e4i14u2#3BxMhAw6tX zc1D=s<0o^?xrFRxV}PVoQ0UaWa#o|;8(I+zcDJjW6`>QN7JIq!d+H+mWrSvlpiDyf zicF&+2<Ipb<vrpVNb1G+sWK)s-tR?R9P^!hQhNK#hmjEbVr0}hec@cT?uy%Tijh<% zdV#Gf1AW*d?KST)Abseprg78kM@^^h%}Lwh%^^79K2#Yf2F_FCZ*R~5T$gBBKr$DK z;c_(wxI(KYrC&#9<G?^+U7*l7z+x^p?p1C<E%W@5=j271p4MRsvjcFeX5k@OW=<;? zrNV7hrvDr+CVAuR@+{Q$RmzT)!BvB@)+LwWUJka=gYLdr>v)+{*!OjZsojUC50X3& zdRIP1pP!T#2~o=cI2qCU^^x`%9(;pRny~m$?L=xG8CSBL+y0<;@AJcug*eg1y)e6R zg>~p8QCU6>6i)L$%yhNfFZ936d}l(-*GhrF?vcWk4<m4v2e0@1H)?u+57mC~q;$qz zd`U+Ze*d+Mr%-N0wT_uvwk(@P(Gt!0f#dyE9xAc7yz^4BEI!Uz`jK;&ehZoGiuqw> z_8N@fo3z*gH$iRc(6FNlekCS%gAcvn{Tj(pdmUkQ)(6+<gnYkQPRu6?{@5*2f1doP z(<dbh7PSIknd2pAbnrtZinX)8A{ni(lqoO!*KSh#L}lyjwW5=-gTR{?*7f%)j92sf zM<v5he39le&#@xP4={(*)WjHGix|`YY+iM&o}am$Y0T{-bhu@WgdCBsMH}mrL{+|T zv*0#uTyrLNpVG&=ccNbOyG4K7u$XJ$7L9tT>ZI{$v?!4HEx}>8u*B8fp@M}rLP#6+ z9@Y~q`+h46vHKQ8TC}A-51j%XbJCp(+M40q5)l5MrB|z9=^kqtG;dFLCT2@4ZNe1s zwko7?)4Z;dJY_QoZ9=LUxP`e;_tw)ka;@wNDY#g@_b>o?k?8xyQ1mwFwigwHda{X; zw8z^)x!i38zUo+UcVfMS^nvCINfNbM-uU5xl}i6tm`K%pM|iEws~_&X4UH>2A#H9` zHEO^f_7J)vBte4*x5z=_Y}()*;K*M5Ed8vDj(f(yd)Eqmhave_Z0uKb$gBd~y~wwr z;wf^hy;ka-zCM&a8$Lt$4;D+o_)LQk?pHB9PSd8gz?|gfXBGH?&@j12h=VN|?b^WC z=B!s<C+)iT35}-jKc#!RFFF6ixD;O*>}h`!9?<tuz2dF^eOsE;IP^-??<0x3^LqlH z7XenZgKKv~x5+?Vc7puQ{Z;Gk_qmv=J+ASNa@dUWBI#XKuEDNJps)I$NJbCnyK>9p zFXDMO(|4^-%DsqDH8tmnPb)r<Zg2JVl`JZ={S(~+HlY8`%~{+pjCxBs0-%o_`(C9s z%hW0;Xc7!3%u^$*ctd6FEP<-a+eR&_JXTH<qM!wPz|MeGtb5B>sC~CZ)?(L2^RXu_ zuK$WweR5B*rseb3`Tl*~2Mwa{%0ZeTnG#nS@2vqkD2LPrte#(z$JH2Pj^`{p^|5-+ z)P5*VjL_T#a|3n?w-BbkCZ&~-Ha5qCII6C)DkKuge#jmn?=U0Ze`!OQhbp~6d!AIv z*!5)b)sQv^dUb{qY1=lO8V0Q2emVc-fe1s}EApDKJjw#21p@d8Ft%Dm;g}~qM4#73 zhv)px0uXUL5vE{zMymMOsL|rd-C%Yh<nT|7`ZtX5UFqsmfXIIa(7fP=RmQkGTTxq> z`%cow)ph4oLwBV2HSkz@oj0@}#@STX61D(;FuLzU#}0L8-_TYvB`#mzU0P|+uV1XR zUe;enefC<keEd7j#xHsMh}0Y=N??0~i&$CvdJj+$_~v@e+))bKQMp^vC^wM?U%@DW zjhoDZ`|m%T*R`(A48-1Xv1z7#05^09`6@*@^Q??-<}X;9w;<~(N^PafIw$pNtlMgz zBf(IGBv<-0)Cl_4ol*E}8Wsm+yp^1X>OY#~B@B5ZEmX)crUknWT%P2C>8HFfo4MNu zVFUX(QM{$uwAagri&zg2jB%-M;6Y|l-CBsoBVXii#28FKdX$5!Uek=f@}1->X8p#8 zl5KUR(6uaY%wK@Yyc5;$VU$(_IUxhyesl#3zFKC=&PjcJW=tRSeYU{Q_gv04MOeo2 zu7T9zSH}5eKaXFkkYDqQjWbyVv9j`|_i_0?Hqw2KTu*EmFABJbhmM8lg)iA+-iNuD zG<=!8XR`xqj@9m5T`6D;bIU5lM=4z$822Xqw8Hd)jL;AL5S^7K_ydClwlS<IbL(Wl zI(K+w_(|V#*A>}j_6+Y?^CSmVr^iyOBX7EBzTTFfvV~^ZEkb0S-7D_R^DgsjXzRIi zck&so{{1Ks$o?;Bh)o;`SCF#q@ouad=`xNy+L{kEr(J>K1xRa&+FxhWH(kH-HBO0) z8`TH{tvlNgP3tq+oI6cR1TXXM(sC3gs+{eoXz9;#V@XXXiYJ<`wmL$tR6i(CxTcRV zlwPdnPKqvMmWfku)Q#ujBVJs9Lq7k-h{K!nRE^A`2FjinlX=FAK*uZc`$F8S<{uAb zl7fxPL=R1^qPA~)%~N6zfUI?%)vi<2o85?ru}<&e_(7x1r}{S?KF)Binslz!T(tkn zxoXT4wc-&`tRW@H{mPYRQ8)oQ3m=W<{~Al<$tg@E9Sor&%bj!rQJN5c9X+U{g7Q@b zN9vkxaMV^mUH1AEC)F}B{wKP&!MQMk?9<CP70a@WiX(kPxank%HVCLKP(Ln^^qn0Z z&A3r#bc$Ky%<Pw5W&hR1jz)ILgzVbeWa@q!6XV)5VlSq#Ss4z-T>#+{NavhD$yq;` z4R`g`$(i1yW1#c3uRgb5rGI>h+|T>is+c+}bxLv97Oe@vdcv)o-pFK#>XOG|{3c$a zolI9_!dK{$Rc4`dq;VQGpvf<_?OJ5!k+mA73I`rWW|%`%W)&<K-|c-=RiK#}lnE4e zW$wZIR9L{rnLoW`aJ5%Be43<;vBaw=?Hct?)`Q1AJ&cYf8KIB0Yq>+6%SgRR8-Nyt z^ZSJM<w-W>G0||>AWyzv9Llq!K;090oEC0v1Ur~RtmS#zNLOg?-8~mBBL)66MJy)0 zrtZhU=4EysaOq^;j?dAn^~^PO|6U{YG&N}~I@rLB;QtU<EtRR-9a*kW4{H>d_uZ69 z3)ym46X_~_T8__*(L;=p1|0YUf9mgUDt6YEux(ot-h}OXdsTB$`n;M@HTN3HRWtF1 zwPNos+vQDb@=Q__@-}w+yq~I-Y|Td5*-x=A?LpSrATm;clcISG1pMvbFrLMC<Nq}E zf6Hxd;sA`_mrMsZnaOPQK)uUHH5&2|j=Oug5?z*&us;VZxqqJP^<Q+O%fEDYz>%4( z3#V-rdPB9*VB38mPqZ|Ia)4Hh!hS|C>JKFmt3~mQDp>;Llom)8!mOIOuFqtr{vRBF zTL7EXJeH`@BD)*8^6q0*<Af$Xt&{L1LOa6dV|F4F2qVB$DXH+nK$S~@5<~grwWOW` zEu~1j&x+B}EPFe9x&F%=1N=^UOsHs&Sa!{)T~Ce$j*_27Eji`WP+j<Bs6=AXaTV<p zPpysBqbm4GK66MK(D?4^FtqVhE$(Vw=<?|b3Tcp1O8n+)7cn2AFQr_>Ld8;;Uu)@} zWf~deIW9JX%=)o<L9Fz*kWSJ38H8x74vX2$(U(=!AE}LEQZ<O#F><2TdIUU0Fw?>! zPQL_s_6{&$x}?auSsrm8uO^%v%Bq)b-a~;Bxj;wTGOYkFylTaDtnA&6eJ;d3q*nV! zM!5t+k~vL|>>n4-5tdn*%|O2H%yh?1F&uM8w|oYn9>BB4@ocsNPSY{jqz}f1vi<(f zWkQ#7pmg%Tj65*--BQmJWX3hn9JX8Q%L6>;JN+2{<t9%JX0b`8&AKsQ-<tF*9NbUv zV-y=~_2Q)a{xEu%4sK;BJ7+DoGS2PT?vb=Bv!&-~e1w0aWSZAsE2dV>tb~FJk70wR zb54*$*>x2+7rwXu#lU66pv*0Iac=-UdE0k~LVl7zhbJ0}z3%R#cJnc8wvj*6%)syD zF@N%BJAHlvh4KSEb#e-#A#txZ;`Y0#C(conL}dDDGRkyjh;*N8N-f?Emmr(}AMbiV z%(n)7MCXvOCqdtHtZsX4;M<~AB{QDysHP_Pk)#~|iA%eO1(m(?m#nF^mGt(>JJ9v8 z_;P;-zeIkupXQmwHTL&|rev)n(v6g4(}u?mDP=#b%1oW>cZ3NM_Q?A@c-Bt8Fq{=; zja)l-;hz}Z;a_QJz%oVOHWZ5q(T6$%L-6E&VpL<J6&pR<xynJ7?$~|tXrr4}YMS1f zHj%zvUSWbNOiHvGVIOQWy#STHbwccoLS~BR<adJ-o^+1}k~NbBt^qrG6?;VpjV2jE zm9^tXwAUjcUA`I;w0Jr@VIe>eMrS=W8SvVq#LIKrARJ4vgf)_m<rbk+N`6Pn79tSp zsrFvFIENk-#*^wBHH$sp<s7*-%bp5%7XMNM7kAgs(q*MB1kCRq_jZKtAUb0yai2f) z?^o*?jIzv7VS6`u*ZHCVInqxvJ*pcy){9KWl*_m!o?WxjACQ++4<?^YN5H3FSF8>+ zJ00}h*&Rwx7VP+;Ak2-7+xk-=<TY)=Rm?Fz)_D$HlU(KTyqcOKoK#pkOKGkYHfHNw zBE(F#=R(4=vJDymM#|7%^T6pb&)%ydTX^)SPT*P$;3ur2z0zp@Bjb?5R=71=VJg`Y z7BkN&dc3ZGx+Wg;^DhK=*YJYVWvx{HK?+<=P0B#KR$(&r*X1n#@)OlU6mm#q^Uvej zFEwalDVV5kpQ<~TI1W0l{(gp|;Ca+IY?7GdKr5eO;Pue<$`>8sDKwh*Z9d=yJGjI_ zRlc)@;_`A*U~hQsY5AZQNmCXzL>6a7-8lhVO7e@hL>r!{^53Xt_}X5i9=JGv90Fvf z*b8;S4{wGG(q(0cdpJw#UU`%LTN+V@PxZ;t=WnfcNWS<q?7bw6!bT4swzre~x%F}T z+AGtB0!g^7va^v$4SnxJ{plwn-T*_vE;K=;JATlZy8}ergSVme_-6KMwR<V;8)iux zo-o|k#@jS;Vt$#Y?_9wt4JVJQB%@YGn};k29`)hONdCKz^!*aQVJx~uZJQ^XO(*0h zXbr-|-G@gx@>KB|&{<x*9R70D*Eg#f4N*fwVpCdZD1$hZ&z0xowu}AWOP|$s|HfDx zGflINvL&;VaixiMaXdU-o)nLt+cG05KO~Ba3P@<&uojQ7)hW_YxT34Fb}>b$&WW47 zrL-bsJ}Z#QjT#yz%ngJ$bh7UEc__!CssL7>FY=F2B9l-QhVqB36;%#37HR%7V=$4< zi;h#+lB#V_d<9{^+|JIMX7Yr7H;SAbDe|t8?^v*sDg`;GNM@(kPvJ^^q*|y~ZOcPB zj=iqwMtb*A@k?@c|HNd<i5@4dn>Ucu<nyJ?@3o+=`Xp6%ip;3(jOl3hZy+Sgm*=M- z%%baJ^>hSguqI7dx&ZM|J^NnwGIf&X13_-*6(GfdDp&WG5PX&AaS`L@wq)Q<Q`w8t z<4SXaw_o|88|iWB4W^?!KsOfQ5z1EeDDZY>=H7ive^keKrF&ji^^dMH$6&gWZc;y+ zCN0b!vRf`ioGy{&Zitjvk>j4R?!-x{2*VK*!Jeb~p4yiT*iP;PgxZz&2G_c^iC66f z`|JA{zt*KajW#B~QWrEVuYzHNL$<TF1ofFbFco&FSlZ9rO*L*18)pOx?C#(`uDTwF z>8Glsp?y{JUF5q!A@*nUHmYY2L5|g1f7y!Z`wc^GWgP)sN8Nw}K3@gJjj>0Wh52&x zsGyq2dU=?8TRP6?aa^aMx}N$c$j5Sou30V7U7I;(LI==KA(N3I>BT2q+!|B2x7e?8 zyO-IMWcYXbS#?a`Cp-XXo!`ul{pDL+>_?^ZbejRJiQw^o(AP^cJb@oeXOz;;JWQ`2 z$XybBdmrqKer;O7Z@M~H7DIK2*y~pIMvpP8eW_5!K<#Li7vD*?GOCmRbl7xd0S!7= zV;2?eaEC!zE@tJU_4H(bq2)nyfQQ+who{6_$E))JgQ_|}#Wv1(x33VjcZG-@+*CSI ztC##-a>yO+s3vfqx%pUyo3H7@VU~1~@;@2O%lRi$3}ell|CRIo4gN>C%e*^Q!^JsV zQM5YV;EIU8hhkzmH&8}J%k0)AT46(~-0!@CX;d4iYiR1X)K=7<4P|@;?R=Y$R-u$L z*xQ;e6mW_VO=}7#-%C0J4s$@DTX<26ZhES%re65E-Im)e2TfeaY1%n&Pql}0r(9)^ zs(c%$^L#8d?ElPWBr_YBI8(jv6R*?%X#FU2Q1(f(f>S<6m>>f}`;mhR){KfSs_Pk5 z0rM+b5z-Z$-Y!r&q7l%dQQX(`XdP9!qSXfAnv?G_-@S7}x|#JihWa0jSX1{gt0ZnS z>IjAko=(SHUinL|kXgw@eJ}A=<Y`C{cP19PfzvDhb2*UdfoOWojgfS+{qyYG)wS(k zW$?h5C$+@Htu_{WQ=m>}G{)Dzj{_O~{f00@oM3*Hv=f`U)?ZvapCTaaGLhCAQZ<GE z|MSvDM>uQyybibeU)ho?ne?^vXK;7&C|uZSW$4)qBSn<|-5<?&(n262LJ3<dMs2Y7 z2FGuVV0t#P!rCkLmR`(V{!h^F%jlyp(|zUAy@&k(H5ND>dCFSjGBnkRp8o!={>3$0 z;fz%W(5^{SK*geTL%`F#Uk~%;WUGs6rVjU92RA-V9ZGdpr)?XZ+;umecVI?(c>YLS zo@tnr)ZydHXp0RcWHDx)CtK-30bETnETah+$uJ-6!TEga3BjHcv(kkq_0gnY_szS# z*`ti%Kq<el!TZa5Z^LT|-s^iAGFDSD4y79onL*?VTABSicH7eFirOgsoU!PDg4&4& z8=E4aL#37nCDbwoS5*18(T=$d6*EGE9Ku>%^-rD-XWYklkjXIqF*oB$xO?`VR<LiT zG;SggI2z75lqO%uyW2Sqc6BS+n|2C_)DeQNJLq1X>7ZI~&eR2v1n5RB_#aaF-#m-a z=gEdY`OJmRi8o8;i!mX7YDo#aJj~vZeI9qNy)~*Ss(^BqvQ+ohr8lP6Qq!&qbBcdN zifC|dKQ7J<q)K9Dh?*dA2vn8}M!I;inePBKyGC565^vD`JpK8=Wa8@s6E?X=Uy?mj z=`);shm|soFr3E2t0{gMDi#l4-a`Wl(CA^1%Vb$D|Ej~P+RWBk$J-AUDZ{IEp`~7i zq3@5b-<$_jqxM*4U5^+3vaC<^3%5U%RX((oe|tE5q4oL|VQJPs4K?@l&21k$Y-dW~ z?lx`|v#7Dg?^Q5WN}~nm?~Ej8#DAK--pI2h$jxkutIkYZ4Eu68lo5#3Qkma*5#|4z zk7MV8QvSQd?CSUnsAA-_nE!Yl{R@Bj_|k>oSiP_|OP&{gQ!oCi^_1#p56V8iv>U(b zc!wtNdiA)Zt?951^miV6p?Rd8>P*Z#18`iNqpCd{j%Wf~bzXlEHm=`E($*7_O{ToG zhZC^&Wi%ng`fUt_;RnSwM)eqk(009+!95&1DU3b#Qkg~epQrqp=|rxCb!)V1(F&Q& z6d+%l&+uFQ?&s(ZZLEJq*8+K4^9r6E>zfJpP4+H7(Rug>WIH3pkt3RP**|p|CdIIT z<0eYRl#ixBW@D-61$7ZhvU`#-^~^@{eNUd#tL&Y}UY21K_uSRgSFW=ypLt}5)!a_; zCb4+SXiN${ccf|Y(0rJp-Pnj>zA|L}I3^hT?V~rItG4@I?m$`QF5MsniGJq?XTCIF zw|H4*??i50ITNxHRc0fmlFMNf{K#&bBD7oxcRoyj7HRSPuUI{w?NT~llpFqK3#7#b zOy4&kin&yX^nfOwkl$2z`Y{^SutCA?r^j2JxL9IeM#?(o;`|qV__%bL(-iS>s#%Y% zY4vALg_7hTQQ|F~BAKh8q3~HHjDsJd#u(z`D7S<3SpH^Y4IxWa4f<`+ja`S}I^9wE zj=wts$=5rzs<T9iSZn#M(a^~W*I1cez1Db{??k2=w-a2bU(~l{mX?=J#Z`-rr4T>l zvoCDFh7xfq$K>sZ9H*6Wn0I*w2w@cdaJw|U(oG`1@cP-yh~p}E(C0z9G~hQ2f9@9g zNGDxEx}Lw39zQx0Xg0>hdXG(?&HMf>%R?QP>(wD!ijKP576lJ;g%jIFjk?n9sJ(hD zpqe%Inh%-TGYd?9^QBF=_}Woec3Gls&xr}iv+mceMN455SU<8oL9t(ICEfEFo{aF& zvaG%s8K-lzq@3YivDNjG1j_2E_<5(%7Y@VQ7r7jSpQ+Y4i!{Il9~4h2C=IXb=ek<Z zQ|gad+v)~{?W9NfEqnwovvm`_ccTwB7Q~^kC&RtV<TyJmMEu41ST6NHEro5<V<;)w zwr)<Px6RTo{Vf#rjjMI%{eRMIJ64^{lx`i`s~+|Q_`b}IrD%=g%3{`Y6yl#jv!Y&H zFGzQ1lA5jS2?kF}^VBLEsGzkJpG*95FBUG0%$pH0V|o=Q0~)j8aY|ojN+EHa59?Ex zCumzH2&EA(Jh|>LJDR16?%)4FdYfk#D}6<7v-WZ&y4|HHTkGzs=yMpx0P}XbIX8)z z#h;ICB;oJNOs1<QR?M8-tBk10@`2htN%b`9v2}>6Kp&3#G>WqO2hz`IL5MHgxwk0X z$ec(1MdAKqiG04_IQ!UJw^bm0Mx^ZynE@%Mk@kDPMMl+>kgv2ygGUrZQS!dnjEgq6 zngYOtSL7iz9;6I<$vya*G|~8nNae;T!&rAvks2;OL9zMeDRTxH!J29{OEAGi!W-Dq z5u%F^LPI#s@Xx@*9dyLZ+}TuAu;cz`<>zj<0&!*JADB{1lxiX9o}~EwRcX+QBbo_H z;HR`Qbu+Va%2o$8CwBTt8y~x5KS`H|y|OrTAfU+OjHk&t7F+(D0dF~W%!hG|Y_M07 z8GDi=k&J`)iT7#3ciPa0HX9n??Id-1r)=ln<GJ7Q^dwi~Ix`hqkzHokLsnh$vscgC z=qt#!$9OKITkN_%&#sR(r|ZXI>dwD-*yBNcS9iR8Lz^}ZCBIN?U1t3OyOB}&{@ZXr zuSlU5I!Y?pb&utqp{Bg<%D1TjA16c08j+@kaR>8<f*_lP+csDcNGp?=zviPJ2H(Eg zD_7^D*F-}UOMfOh>Ch^HqAnKP;W!S?c^!<aMTM_uCQ6bb%FADww~sWYz<NL%2~s#k z{YoK~j=oa4EY{M##Cas+RI+y7%GPa1WL^=7?=b7cAK{gZZZNO6bP7@fWo6mPrO@XR z&leOhl9tec{(q_N_g)BlmQ?@R$mx3PI%If7ybMAV>1HL8MAJ`(mGHb=eJp50ho=d5 z^N%vbXK~6v#VT@qvOrAb&MR2sO8W5d_)q|li;jBT;en0+fs5jdEqInJmUsASa{;3d z<NPwd=li-md4c3(54&^CMsVufn5KR!f}|&9kGrJS;G?F7V1tU0*5-(ZcSFI0{U5vS z==E%7PGr>sSIi)-id!A-T8BH5d9K5;8;GTu#6cSH+zK`imvHsV3*N?lp0cORH{}HV zxp$@2iL!nV9D})6hRg(=mI+RM!L0;}&_fZWAyYWzViIByH5)t8UdW7V^do9VaHeFb ztGg{Kqs>l~O{aEq+Viv4_u{^3F$^dWvI7%cC5A#gq8b;5N{Ixl{FJNRmpHRtGE^T^ z%2}6_i^+sx6>oy=L|q5_HgG|;LlqeF-_F;6<E7xSmE???oayp8d3l}M6W?(2kQ5VX zl1iGod2&|!viH=eN}l(|rJ6g8A>-QggXf8Ux)`+%yRyq!(=|`7djF@n0WB{5!kt^c zUuL>Wj9Ihx{3_5{bl|_rJ;KgvvXllh$sKxOvK990Vi)zLR-3+*LUf?U%l4dv=EZ<v zcZL4J;M$ZUhk<ky>m5i7X1m4&=QLs8xI6BBskcQcjH}Ec=>=hzBUF#4p=oDujSYu= z7ap5pV+qSzoJA-n%5#CnB%db|#O{>l$PX_YJt3#cJBUBC?Slq@xcrNaX!Fd2cE<I| zcW6eOdT5kD9eSN3+lHpaIj$tk@)Pwn7WIB31A6_raTnE+%QjEe1ISu2hrX$seFNsF zj}mFRr?w|l^|C>|UCMcS!_!v&hCh(<TCivLyo!-Nm$x4e56Eox@^S~5WMSimw<j{7 zPzFToh-n@4)n+rNKu@_4Pv01!t1|EQy6>~@46B+L2vmPzHF!4&)8)4OZ7U;d3_7@7 zALm7Qwxb-pq`ELL-Q+7ZZ#ap3Ywsfn<~u^FZCm^pPpxtcuT1EG<_?4VY6AR*IrD;l z(_#+s)+A$9eKkF<#cXgmP|L=wd~&D@=iIKdUx}9bpZRN{8kn;b234VPn8ND}j90l2 zcK-Z&&os3#Gt<SHVB%vF1OwxbV_S|JH}M3&Hj}Ax75T+_c>*!OQ-fbv%Y#wLi!1ob z<etyHdB425#>_bzG3aZA=fxr6_1VE4{6Use)#p}uLX8a`2Di^5l9gM4&zRQz_1WQN zJYjvdvdwF}hRZLj<vYv^9DXm0IE@&08uY|B5{|gmE0tE;N0jn%Z2H#rU_*RJlUA>% zaz`$y9a)Ka&pl^Hj-xzvE8100=0n4TFvF#%VUO>361Ha&1D4HFYkp5l_<_}%)HbKd za8-B;Id#~(4DwHVWPlw=k2R~g<ksjw%~@k&t{_8$U%5$0QCq){dQ?9#U-H__mD+!( z@%iGVtU|@In<Y#*u=cVAq$ltX*LvM#^GIN`U)KK#0f6^~g+iT0uZ&>K6{M~?FF(-M zm&^-m{u3bl^EU&0{}L(upYizbce~%fP|Wk+$MipU>i>Mvf-f*qf4@mUT;)zER8)QC z_HCf(><s??in9;tD*w{i68<@mvk%SR%>VngUmvrMJyssv9KQ0~`1s`Hpi>$e*8L6d z==gOR7NDBvE_Drmm=rj&ydOEHPEed#ex_tH`DA(Oan`i5^augo=m`m|BrJY&8Q(TB zm>YZP=>fi6y6Q7a)b8Qk3hOyxs!nPb`=`<UYBGz%Y|8w<r?I$1M?3;*33=`T;xFgE zWoBPT5q?vx__`p=$o=7!1TrA<swz_lNRl$hDQRjz@z<F&%#7~P6ODI_;Kz&jWgVbt z$CJ==-6PFY!{oylb@DHUgLQDV)I4tEKQ5=`qf=7&rZg*TP@Py%AYCPAiF+Z>vDSvr z4Q>((93Fd|MK<yXbLd;-%{xw)?*vR0?8L<0Vdg~pSfoFD-k{_UV5S<OsqUkZtCza~ z2eu@om5eL5KVI8e-k*!0m-mi$ht&_L2GeqGzFF)vGRAKQ>pKoF6Lft?R2PU^OKksh zpL{a@<xaP&3jT;+IXgeAGaAei*n`vOI4e|cn|pSiRY0kmc?nO`{>#vh)*4ROA|e^h zd{0y68MsNhdWxgp*1IEMZ*~eDxCfwBuz&%{rgHdR8KeSC{>3N_*23rD(pApGlA4KM z($V$GV6cP#$&z$<OY3S$mgU*?Om7~X8hjK@+^7TeLu&vVvnT}h@=N>Zf@5wQvi~s< zcXm8wXH-|&*$zeMkoF16-SOnCDp_CjnUACl$UXkPjG;T72P(1~IXKWN33g&hXDk}b z0lA!#dS(@~I%99reL^2XDp92%z$jyRRz-Xm=?tClaf%NFb07~CI#x{vGSV4$T(eou zAgoG2B2`rZKbo8O()Aq<iRH=xY->$Yi+48Qfc~)x(7AymaQT57zdoO#Ol81v^B5t_ zgd4FXay)1Hta31b3`(Jo7C1VaSB6U4R(O0T^XR%W6N%-Zx$PX-&nCilpN$UOcpgR= zDl-dPyT81^NIp6%P`$mmkzjKHf~}$x3~8daM+7N_z3~zi3>ir0@ox9o1Fpi>x5K}? z`NEDCoaeRMHwL*8>$A@PSvwBXp?c?5E?;cd8P!PbK(k03pY=$otCaRR9drfPg^VRQ z)Q5D@IaG%LXNT4(V0%meF208t>xit7dM~RpGqXX4&B(UWrrEf6JA9+JZu?-!8}1kI z`LaUx_-6h3<epMq-jTj<JCNo5@T``UZ3l2`?6LIEX(<;6ASXMrc?e(pGmkIMVD&Zm zxthwM=_DeJpq9D_>e&gf`Dv%N$TeaOC}KgRaHk0g^__Am7Q$KK^X2$;=qaGg&(6*o zS^nlR&>ASGmp3dsjp%*`c<5S~d_4l`<6iKhMg)!Yxr(v_lykr0&-86tiozdvx657C z-1jzI)kFI<;h_Nb*MJZV8~4mfa?KRZ4&seGtS?pb+yidMvl>s4s(MuPnIO&TOwncu znD&-Om}9bV%Wuw%ePw6vz66NsnaWo}XH^;y`}u{lYDn~R1#i7cZ^vCN6IK0nv1|&c zm0os69}d(hEdUkaeizOeIOe&Ofe}D)!PDciGZ`P9jHtpHs3K9UxO1zF4;5_&-$~_I z$dX3Ef&a#N#X>bPF_C@8jDe?Z3KD)SE4tMCYopA@C10(O0Ms@+SWswrZLMe~`uyv7 z-pTF+07PQ_!gBEm<HJeH+5TW)GOIJGv(LievR(Zm+u?xcu6a6l<h;_OL=Kj1Rl^cR zAy@-roN80%k$n)1r>%SGZ0yo{$sN83g1WTSZ4ld`Cl4NZaLVB)jUqMvo}L{Q4s(DB zI`sdUps$ZT3lw0F&$?7<S<*MdYO0AATM#;zW7SRFGsYUO)@CZmi~31!WpkMAe=Kg1 zOmhZuroky{3X>2VJ<Byo<%d4>Pa;;d>IE2hN79NK9sMh`)OLjC=(*K#wlg>q;_b@T z0PoCjTUxmXm!W!DVP(hu><b;Nd`ab^`i@uIW@8V0OW(GyI8`3^1#168_;UH0=Cq(~ zIj-<LWOS}!=1#Jz@(QD4g6neqa%%U-+m0y{565h?n4D(N(sLRwpFidZj(f6@e>0Jl zcI@y~sj72ak8&ZZ!-n~~`ijRy+M%$?ZVo3QN!2KCaZSW?mxHH|+mIIRo5<%_1=`V) zMQ#V$wRz}&Pz!H+>EWw&JtcuPbb%$WnaRcXqs+eb#C7#TP$x=dje`e!=On~M^t68} z*g`aF`>fQQ7Ff_MmpJGu)lCOkABq3D3=Fq)T&^#2;j2cC1d<DM=M|FtUb=C0Dp;rQ zGw9?Gbu4#p;dvn)e&qu|b>lduU0(mIYD}tk`VI|P$Mg#_oM>n&Oh_)-mOPqSY{(7) zbc?MKp7`@Pp({+=UN5$q7ZU9I*cz)nMM76!8tWuKWZqErqq~v-VGh{UlH&G@l?v%C zL2%Ww2oqP^r)Yc5a67sd&U;MENC`(4d)bougV{Y;v05fMnWz;iZgjeuck(1)==PL^ zI8m$o+E&i#nLc|0=mAwP1g@oS<s?*QM)R6^=-J|~wrRU4jYwC@9XJo*5wB-P7Qn+k zeHDW(Tz)@NdDIS(n>Y<j1uvZpQq^_wFjE5rm2AmEuj;;pxpvxVl7-5U6NB?a-m&3~ zzf_?wPWMuSmB4*=#gJoy5!h(O7_+^-Eqj7f>J+go%E7rX2(#Gn*dE5i;Dx8Dxx2*^ zz)Zf`keBm!WMVh_Clm<$_x~$O+FsD+kBhe8q6OJnKP-4@eD_>{!!I_$V#O`hS`(S? zPg#17SX~;Ld<|-H>BV2Ne!9=f(5<3VWT8i5V7}s3%2bx@{sQW+xgr20JjVa24F6ih z-%$Rn_Iv(zfq{ws=8F4-`K1+FidI3cX(RQi<jt9Q-KnR3{{Hn$nWP8#dKdwXGwM)M zUWiJ2O^Kte?`?t>ScP>D<(=sIf+6LG?s5c!M`P<cz&ZcX+HfnoTOlwcC0`nW+_(~R zrYxspPJNv;+cNA!ns!8a4v!_6v9*TzyrtREE3iGy?YU}OU#k|loe5^^8hSE2EBkeu z2}w?s6j{zlkdc)r6ChHaQL+*^Pg)l0BOG*xcI-8qi$_gss?-9V&DCYqoSgGJ#M~U; z*V(M(p}ttv#+Mp~gZ8~rBe29`fjqF4eZYj)ofme#d|Vp(fkN~vk{2{eU_OcS%(feg z>NT}{*TRT*vw-zCJ8yEIOj$>_bRGqm1)-!v<7k?TT|H#DD3QCd2x(bSYwMoKke6Z# zH~4sFUQPHui^>CCn6(evn<(p5#mg-3^ja5$?%0Lm64{b4S>RP&8qadf)x1-tJLEeI zfS0L7^!-GLrR9z|BsM*Rhl9jn)-uRX#XOH0!ANyr`@dbYV){8x91co@E(}kRwXJSm zDCcEc9CWRGYrBJh>@ue$s#!+v44l!&Z$XO!Iz`zDVgBqKi(12OsWLrHur57$R4AZ_ z_sn!140C)VQw(Xf-aZ60eUG-Az~g4VrsN*6@Ec6pi<anZz*oAY9`<G)R}8Hj+9Q6B zYj--IvnL&d$U*oW*Uj&7&F$QS3xq7GhI4Zyrkc^flfxIZXu2h57LwJqafAr^{V&lK zxNT&M*6*yCAvG}Lpy<X|h0azzUcP}lmQyL(y$H4wKr4TtiWpf*^!S=iJV&vnw!?Q( zN}b?J0kwB<;a#(ZU?lHZAZ9AP_ur)Vy*jm&T=x?>`a1}3R{dX3UBIBeU64`sv5fPB zK{;)C{)d&s{1m@S&6k$rSL2J~Yi6y3AK#AAYT&iHHNikpgyoiHY`L}1Rmdz8?c|FT zx*auG$7a+QKt)?ii7Hk^B=@aJZgn(%GLC#GnD@;?g%oeq?kM)>#@Y=#@q6xO3i8Q3 z$M>j86XY+AZwP7&@{DPNwFv#<Rd%t=)V@yH1j}E4E}n`x5$GtR?Wkf7ung4h&!?W{ zwXu$CcID2q4V*>O(zQg856iqTD-jzk9@LH8n#P6;Mzeiqt087=lIWj&;y-cj1$gk9 z<O}%gR8o+qrI6kw%J4tpTGw#ra$E|Tw`&vK1{+_J@KEKt7gZ=qLGFgJq(!m;dWv=_ z+4<73|CcI)!DU$@o7jtb%{R_Q4pf)4t~SPSY8!-rWH~fnd}KCK(qbeA0=>EW-f5z6 z1d)%+dqaS&-Y<tNJaEHy>IrB;?11*GUS4CuYewIfl9dHT%{aOi84MWhLQ`noNu>Yj zI)%MoBbo=oHMoYwclzP_br-Frlw+RjdtsHRVkTbFa)1XJc^0EPb}nCZGk*E=m+O?P z6iRegw-(T3p_j@nLKs7*Kc}IICK5P-Q>Q(#D^}rTMLZLPXA1wJoOJ>;*K_9wN|=4; z`?MT*#%<}=LJDuNRMVj*m?w*rdSr813^Eh@N9JV;qvb&GkPFFHtnXSdu7fO+#js;o z*$T$7=NzV8_T@Olh7zw=;>%)Gae@PP41-6!9ZeKu4|^2}u2f-IitX3lIIO5r7r0wF z6y><^$zjoy$43+1;s6homZGgo-o>9&R+4s|LAB(J3<_hiTgGH$5991H*Hv&8c4;h? z%i_wM9J>W|igBGSF?Pxs;}MKKbG*VF4W)E*LuLj~wy$aPLxR2s47gJA^w=e4(jm?- zep*f>epG^r`CZMlq_49;<4>~uS+!d*;F`BF>ofW4+xeyIxDI&t=ioehhjGCgobaN_ zZ{<A68o|5*!Hj?U0_<P=f?!fPL+!)<c#fV&Jytyy)YdOtsCa#Kx2?LYC@f_x%`GD> zc=@LEG%dRv1T72Fd3EhXbV$5$de39DDk$Xy+-u`|<F7bHYn7u}^O*&0j-jf>ObBDz z7G3Y+SU9sZR^=|%YF95dvam@DY&-qLA`Z@4O?++wD~QNudw8kWKt@N~G8Ppn@fTkQ zTg_@WKl&b(6;4m&?PdEue>JBGA;=fGxl?Bz{o@)sil3aHuxr)HaPluj-;V(vLi+Q& zY#f`@@2rQf?F*bQ&%W1QdVXhRWyMYGXR6r#Y|0A*&1;HD`O`1l=pVBcUbEfpSCP3q zcK!755?4V|IlWf9P*}HN;yK?J;WN4rrH@v-x+kM_%ux|c;Wg>rq|tPk9tNvj7I(NN z71T1Ib^Q-@p72C+DAsiQvG9&`nMW9LSs2U#0Yh1?#F{!KzwIE9Qkwkh?8_Bj)cYDk zY%=@wE&ptb^3-;S`rga)e=JuxD6^I74U4hU`YKSqI}(&Q{`s~b=`d<Iaa-J%eGKI( zy2PpVyee6FX8%>*v_PVxM|-R329x6kcJlqsc|?Fremd_M&G*A3kSV~LSya9b-(hZf z@k&O0RFmGX%20WnxjXa3VL2wTCP_p$q+boHD89bS=9ry_3V^u}ju|0V^nzr@L8qwo zkT7@GLBY4Gsf((hZIvS;EkbQ`$Q{$^hnEga@f;+b2%JuTQg?Jh-_DC|k@37;v+bVk zJS=F-JJ<&k*eA(cP#ut|?YRd-x;k0=56U<Ha35h%Ue}l4gL3fnDEYlrGT|wcTB3vW zS@;z`OVt#8o?Uaps%JEKF(HY_fJ^pC+0wwpsf6On;6qa|?{?wvoB{SWjEH)Q?&)A% zth>yTII){OECFPXU?XJ+=S=D2g-JV?LwihBi25{y6xhy8H{;Ii^j<F|yoBZ6_%jx> z=U4fbE<jX(BJhGl>%Z6Emm>dV{XKz^DxdOw)O*Lug<7rukGOY_XS)C6|Lf{Nl(XhM z+vE^g4q<3=rjt&F99GWdlsRXJk~z<roQhC6g<4h)rJ2oXN#$6IF>;<^lf&=r`h2h7 z;q$rv|NU(@_I`UEpU?Z_@p$Q6J^tLeVj|&#T^(#)<e6xAbtk2{Y&X_e)g)&vfspWx zTbpjX5OxsO_sp~A=j~Nd#irw5j-U6M)!qLL`gVT*Oy-}veU5#0klY``;fhZ7A2O?d zOXYYaeB^1N1O2pSd^~O+>DgauoWSnpFS3>M)1KDWg!iSA%p{dq&j^Hh_INPQU_u}3 z%BcK6!9CoSBpdwNd2|HiU{&NVv<q?_o*!>!zgCy;JNz;ljxS5t*a&i3c2INbv-bxp z$!Y~M!qtTQ<dTsjfteD&rw5Ad{Ef4no7bEc&uS)E<M@!{TwxC^xce-)_TwYj=2Zfg zI1p`E&x$58E;_v$cV@{+6(&D<Wg$R5l~gSC6tA;X+cNoj^XJYV;5Nn^W(*o2%2)m% zohe4GCX?1IG$qbKN;&z-@7;;VxQs4AYz1L~+4fka92OPLyn@E|pvUix3XWA}8u7V} zd>OURkQTr1D)tViv~2W3_9ok^rVqVicS5O^eZoG9`FtC%OhJ(N+H!X2jZ*r_p8C4F z*CXY^Y;U=DujH*EY2Dh~-5EHu?N5MFUCyM$h(jEAcW{IA_}%RzcIT}oVxpMFafReM zF&w<rqwLvsQbko;chmGxW99?}f^b$LX7fQ}*=Gvyq+y9MSA+tHouxwbai9jRWQpaT zTWz=EHba#0uSwKyQng%+9VTB$3F$phx2K-hcZ3)3iEghQ)+tu=PWp7fF8<{d*G5{8 z4QJmyE#_gm%%5pUG3Qg$2lys_-gw3}>q%4Qee#~2PnW^o#<{nZou)!s?&udEni5`- zJ`EJfN-9r;a^juF7a^d@2L=7A2nZiJ$KT8V&rh$9gVSGgqN937Hg912AYq1aRZP}! zs5*+>6VC4n)gvW>S?%yCTi{4pZzVZk9!y_J4cPpDs-CKFpf-8rze*ua4omOa%ApA6 z-a{o82@lTPQM{)-dYp9pAdJiCJs@h>L}i+3Ld8YgMX7?^JR7*e(RG!r&YSc{MvWz} zjb1@d;z8qQij*<RQ#-S*+p{J4O@{;<<2|o=qna*00sRvA%M=2lKF>|eymSf0ZC0*G zm#n1UpB*|61jQyWSnHP*%}9zRlKI4gg0TX|UzA>CBtAQ2*=g1uDYrkbNLw)3wFV|3 zx*-@BXq=>21&hHec;AGb7U^y7J>KJe=ux;Uv(Rhsj1>YOE~HDgqTiy`;<L_;2qZ6; z&5ACVL#@E(=H|R=(_)SSXIMh;yIrMtWed?@5BrX8X`D$5Qz9B-HvTB<6}I;cQ%dRa z5xsg-m93doz-jdkB!|Kq)>XlM%jKqUp9PxO-N7wwa+M%b1{<Q}j@Eu^Xo8G|Av&Do z&QZ+6deB6_QmmoGta<OJz4UvbZuw3o1^$YB<L*Jn3a5RmJPymfVYR=(DGlJFwQ927 zSwwCm#x@~jaAvMK1<?B`m;#>)09u(+HGzftS?4Y%>y+{r4oMj#mJ(z5>@H{tClk$K zOxtbfo5{v?ve&jqwD-6|J=+LNN-_{w&c~Up<b5Su*e)}RS_`!OD2`i@#P+mm*IWf- z%}#JYV&o)Pd^q2)Fa-Ghg)_c~#cMa6tMcGYjD5sTMoMCul-kjHjGbg%Z+72M4666j z=@BW$k3=&5YZa_-`fU}@kv`8V037?q60G$`-`-}RIXAa^63kq_i;2`?<tf&fWXp-H z!!B9H0(UvcSMi>j?&fVdk<5n$-eElh+p<5e7b5~iBv&jfE)Ga|U>+Ry7Xa^f#+lWP zD{Hy>W)Dnp6XnRvhlPO_d!y%e>-B#x%Kl*={eY(Eh|7O9MH6`=bF6sEcm^RDk%z{C zkVcUtFzYc_;X8xJZR#cD-%xx`!uJKgic8fNm6TaKNYH5dNnpgrqGKIsEhb8x2TLPy zN9Y;Qmu;ug4WEGGpX$^qN~&MIY_ps@Dv&%3brG2m4M~&b8L>D>;;T1{ezdUp-mKoS z@&r`LM6uq`a4feREQC9Jz?F2CA{^_wwY(2%eA&RHbawhFy9RC=N$W+l#Cy690QHj^ zC3odV4d0i&yIw@qo%})KBSs6DIQLQ2V%d0G=W`(CaREIZh6PBS&Hwymyr)Uq5)Az5 z+%J98yQX24cS)UL14}*@Li5uXKu==}&dNYAq@-HBIu9o$?ndStZn4D~I2!Q0BnjKa z&0S%EG8{6#TJ9Enb40JW>RvQJ9*U>9Z!F(LnY)?7gpRqs3h!U$BHAV6_81<eN+Vs& zDVoLnH4|V0&U#aJZPNn_)fa>^vyhu&pHWhU#x1h`NqY1k?q`5oRzLly<g@|<v!lDk z6b4h3=qZh{o?o;hYFjpb$4)=lCz-Twg=%S1I%<qZm__NP0$H7${O$N7txzJpG%{J* zKl>x=vssm6arl$F$#VH`E*|N#fUD)qI@h4Xen}9iAdF)-{ZZ&$1EFM!MgSRTp+fS) zPkm0Kk>Zk3<FT&lIpPE2Kv34CAE6_tIkyuOOySx#)=a2XGlEapNF@tz+%h(c{6g$S zaNN8TuX3MQ&II@7YYX~Ot^~v(>4ZUd)n#*`Ee}z71H6Jfw^79MpWWK*Bu&<?E^+D9 z*k>y=MMr%?as%HkAMiYPD$kE&+mZ(B#X_^miCmINrik8)15`D75ROd0*Cc$7^R$|1 z9{I?=J7lV{>PsffwaysJ%CP?AY~;7t^KmX?9cLd|(W;a1#NQLi-KU6z_38K)9*^2> z?*Bcw+b3iH?YIGq?0lX|9yZU@ghrlb3l+dcB*GLGYZdDlYttSO>mTd+G#dM*S1lH8 zatwhoS7IyQ_Jc~iw$c3N^y|Auy3sGYNLd@a0C5K-{mMql+(>Z>$|Z7FG*L8Y_4&yn zs4oFs23BwP(uHt~vP65Xo-vbnX<b(+ZE-TzJJwAX0<L;``I1LN$z2Q6{rAUPX4zNk zeuSDDEtNq7aAuJ|GX$3b7a3okL2{Ez!V;=S(J0Q-M0N&VX+TA3GWbodEHq?SUn^wr zM_yx@iOAZr5=H8uqG|1ZQ_B1IV6?X>3A$<?2F241;Mg6n;QT2pcqMDaS%tsAW1j&P zTu-_;r(D~Fu~SQIylKDCLrzWDsi^?vPwt2IhetB(<x-mHV?Y$u(SbBi`b1UkJD|k; zdb-fsYQueTH@h3ok*}qcbM3Srbj=#=%q|>PE|$tbZ^}m{gZB&NVZ;tiNcgm08)JPk z_?~1OA;yoBLddp2Falyuv-g{ago0(dC{oHZc*XHwTe}xzcHa37Swp-BoeZ~7MVelk zO5KmD;@poO*yuOVtACV{T3~*r?p)Jg&p}R!L71QSL(j~7jMWaVT;S5Cf)<AS`??Rw zD$?Vt`a69_>Ot~5HAn?r?Q=rpc)Pi?F*0IC1l=GR=6aEu@Kb<9){2;Ru7c^W8uBBZ z!RkEXT?$hY`;WD{AI^s-IU;4jGwp96dslxoc_mrB&5L8hO-ecUt(vv9zR%*fT&F=F z+{#`^Y1X9O#e5vSFwl*_dKRe%CxIRd$7K01+Z-N=;<44$Y7b4meG4a^bs{*p^#lF` zJEk5Ro-nPc6E13%_jZLCD(XW|tQhRV$RGJMqar4Ph#?j4C)4(;X>U)T^_WO^2=je+ zb)J+~E#822Zem(-w+kebAA42n<|Iz(&wsDYMs5hZY(u*m@#v{g;H(`%%57oB@L_Lm zx?7jk`4v!ved>EEiP7LG(PVPg@^dgD$))L^PR%*+zqDdO;Fc!E$H#k*|Acyp7^dyR z@pH<PMP*Q$y0zRda%-L?H5z|bEIQtj{_F06>$RfuqTU@y<2gkw`>#(LtCskAXgu$E zm<cqV$&^|258%}BN;LFpC`yEedQ6sZhL{{QYO}rk%DU;>MMG)BhqBRCHCHmtE`3r` zzF1(66K}P9-Jwn4GIe&ja`_Qo3NZy6>t@1RlDOdAnNWr}*x*;CI*uNA*{AkE{vD=O z19=)akh;d5cd5)WN*Ld$Nc7lFZe_Q<rmw)9x#HUoDo3;81?0LK)`#Qw7B7Q|^uB$| zkMF^Ya8h+Pwji!;(2jRMHEP`aXtRDs?0Z?8;&EA-4IVXU>KWuqjmk0#<y%}l*B|^r z?0vaBoBe6EXjdyj=mjLyuj;)~x~r7zzQMTtzdQo6#az8Eb8=2-MIYHz8KCdo(vV>% zfLW$d9rc|l4q@`DcJVk{o-kE2tZFUC+#p~<$6aM5anx;J7&n;ca+rcPdFgQ8m-ub) z>rqa|;5#R8I`;@?KnkxFsO=;O^ESoBl0?F{mecB<G;>L|C!{d<e^5|xfE7D@ghSYr zbrr6ej{}_UpBi^PM%~U`X{j(M9C`+hy7UQ!)RBKU#T!P8^jY<5$|x&lpXK`moO|y3 zP0HqXgX|*CX*E-V3zy+PoGr_-6J1G4NSMY*Y^TlJ_$Ql3o}5i(pO@+6ay{xeEgn<! z9g~c7jiabD#PTt!BgF|>1g!I6+;g@bnHC-T;6cx6DGe=_m4^#Yo(GZ^JVb6*n@i=w zT&SY_l5D}HxOVdE=1#D2euMe6{qr2VmcEPyayoJ{k3eiKA6+u%*$zB`zHR<+v@}j^ zxiYu!luN8bm^pa)7_InV(rkAH>xke951@}L9(zVKK==3kNZ0zbTEKze(?&(@rUt0L zU5!bPGr%kf7kB@^#<%3Zj&G$VWm{ztn3@Pdlqh;yw66Nj6|A*X&_(|*w}s>$l-#l4 z#{i?YmTJUSWdDiLT+eSqhHYF+;&-T!RUV)kH@P0jaY^Xlc}v$V5{*n~3takfitK=s z_gQ6JKj7gr^x@(G-}(CbczBgo$$%nPl3|BP)2gwmq7E!Z%EMr+?B02XK*cO~iMV%3 zzE^#6mCKOxSFZq`(^MI%=CTo=Vo5_ZlHMpkKX^~pC7cXyg}|q!(H*KSJ=yU%I8$V? z^eVS?B=qV|O&JVbmRl3m(ZsNigu+MaAD%n~?+!Dm6+jHPo2{jp%1b&Fi`|vk`hB-P z?SKV_*&vZQhL<Pkn_|OGfL5Q>k}ixfO{=28N&HA^0zv?!m1cN!>s9GK@zR~w6>*z% z_!b0GRWljIn0W?gyq2`Nog!fWb}2r087@hRkD7k$7nV8tmsVe2!}{hEX~_~;)Wb8~ zp%Co|@vFn{R7qIEVj1wk;_uGC8?^$dVpa<);zX)2Cd0->UqhzhEDdW!OFFIFc_2V^ zJdt0jzoJRB;<C(lP9k_vc9CU8n4sE(Jw5$Lc!c7g<ddg8J>Bdj4a?uc%Et|o4|Zs9 zA;>mVySHR?%1mlCi($X^_D4`)RtPJhhy#libL+<K`|bdi)|i@Z=iAU(ydF^D)x$M0 z`uvC^J{qN0pCPlobi-OTY-kw8_fEA^m6O>V_~l_CRwGh=I)lbB)=#gXAM2pF!Vnvu zidypEjvpT;P4v}b;#7xZ2I{(x<xNb}1MZ`fsPvngoqSudQX-%Trh;U8PTog#KY6lv zK%imo?~Xt8YyS`_Ib8q3qR<Xr!5!r%mc5^8E~#>^DIt6nNBO%=a>b~vn;o7pOz|RV z)=5*7#CvDX6~i3N%U2-GD@hmp5hPJhP&!n%(Ou3(YTYS{eA)ZKWpf`h^wZ)Yn-$`; zz}5gY&^z<>p&6G~dckG1L3QDa6U2V+PwttETccXRX{R>MgG(<C_XbaG>FSMFO#ce| zviM3rA(@(2mrA$B3b$GMkG+7|gRh63N)H`|3452}6w_ehybY-t_o@AZ;s&ILbW7)x zoq1aZzmrYsB)V=_rg3+L=0>PF&uJT6EXfZ@Q~Uk);@VrOyb-$X#&{}8os+ihu_g3} zq+9FJdd%uuJA|38JUc$AAJg$d!_|cwmFx0XO4_fcfFi~wI6~;VIbx-*FNd0=_6BBy zj_>m{_m=9I8a>z)tTKT__HY~;7B;6lf?RxA_Zfp?g@U<A$tu+~j%-wflkIX!Q*u>O z7QNfs1LKmkBQ`+q$M%S2upxKLd!}3arY?iTMvlq2@R6pLu^Wj_eL+{9dJ*G8VD75y zR~vn!ngsYibPn^(e<MaP)2>~r*+nq;0bGY@kmd7x&mrf@oVKmEv)`okln2e;_zV{W z&GkQw=o~)qXL?|><fY$NTSBFjjd08gx1d~nRsg7yLURge`g#**wZDDT-G1(4$jEr_ zoR>{*E9qMTx$1NNNWnwBwiZ1j`2zWMqh3GVIH`~w!wIg-2a)iq;qLV;&ZD3X)uV(y zw|p%I?@Dr5$MH**;vK~2|Lp2rT>rA_#qIO2|C)c|=lkFw!ZYJf-9CxqRfdqUQKb-{ zR-Fvs*q$nI1!S}^zc2EVEQ#zRnz+IRPPjz~thQR`r&rVV74wpFeJlb}J2(CWE~O!! zBbEQpTk86^H8$>m>v_VG<eOCG1mk<ytate+W!<Qw%#EsP|L0GT%`Rz8JszZTe^ixu zN5nrf<X^t@KP}V$lneiVrouvR89NMoKyY|(>9vAKjxyqgDrR3^khBG*a<&G2$)gK$ zRqxy*Tlr>d?gwf0ZdAWpkeeIc#``0;haL<u{$xf7eVg1fDtNa*j`1`gU*J1~<DajU ze{#L<#1yjkFV!!Em1bXI_XzjydF`7mz5}!W^aXVE*}dq0K_6xeRRE5t6;9j$!ZrO@ zcNwpppR}m$mEW~dTeH7+fsfz222H<rZLWU9qc~ldh20=YOp28H&lJVmv&OCDH00Du z&R>I+on4~LCJKOWw#UM5{E6Bch^pRYrV3a7=;D;bo!5aM1gBA>TG%ODJ#~Cj1O*7k zrDgLwki*Gp=R>FyGC)%)gWRnAiRWlc<XS`V33d8+KY-JkM7*Vby@0&V72(xN+ssU4 zX)E+HRJEKJV~0Cw%oAadU?gp7a6wX>_s#d<k<TD$;G*|9ZvEZVdtC~bnrpwV-6qJ9 zrOo)QE5`5oIcJnUD#lOqW^-s!ErSyUO~nxxdO5veEZGc|$b2`-NKzSZPlnGK(AK<* zN`qUD*TopbE!LkT>~<l&wEOi0!^2zeR?$C9$@tR0uz#VzYZYAqI`)3GNM>4U6%ue% zt3fWuTU8+ebg34A5>l&oIjj>Czozp2oFafIL-ZD1!N4xz*=~v^YB!A}$G>E$vN1vb z^V-vrInEiu^{YTS;qP6}B|U0|BmRd%i=Yf1m`d1vL6s8|6K*YcyuTS1kbdoQ7USzd zKKFLgG_mpzfxNf4qr^9(m@Y)RML_|`WBFot`_8Uk(iP3($=-;tS~&f~F4I#6N0_8w z&!(M5H_g<5H|`TC*=b)g&3Db5P`lP_pqK{>P8oRxnQ?O&Aq^L2&mhOs=YsYh7ayTm z+6V8tKR858X%yQ5A04yDJ@lzHIP`AXUnt4SrQL(}wyyDlLCv3CgEQ6g@u^dkem&=c zeR-}?!1!=%1|%wIi^sb|>~A9OoT?l@^3M>h=5HkJ80KQbiM9N5^b;J6i`;~eM^w$@ zj@^M7P(0DlBnY%l4NzlS3n4jmtANplCsliE^zLjm182^0R;f4K7R$hVAKZuRWwZZA z)?O+N3`p~Kcg+1EYq$8P<n|XGnfqn_*1cs2^lj#WgQoAU1zvHNf)+Enj@Z-0x16<E zMD6!cn`$K{@=^T|Qvp3*9rP#I^XLYjANh$x^1*qfO<D$@0=^eq{ih*Owzu`rSxw*6 z|01;c_hEcg%8MR|ob_v3CV$-p1Hbn5_5GG!4DI-<L*$oWghkx$3_JH>^f;n@eRtaN z3jXB%x{GZE#D1+~7q_jI{}mC~Gt1Gp{jgrZ(D#5mPU_)cH}c%N*HV4R!C}qSv|iBc z22G8Qhpz6Lv+Z0RyeN5cTcmwAi&53&r?;1Do*!v(W40FkJ+S}6-<((FZpdKeo2JPV zUB|z$;+ihA2T<>Jag0h(^R#It0|*BNy1c*N^zzZBbPjNKea<Q+eH(A4tpKKScUL=Y zS};oE71GWZk<C-5YVC%bRtgs#Htr6*`w`^ZE*g&4*iGim3C7G=0-=KG9<0FA`GY&V zoQQ*}cV~V<#u(E&<x%RuQ2or-VUzTNfgO!vPIr5VYf<2L-K}qv?{Psyb+iGv%bzaV zYzFo1POVGj*viUqLx{3P&hk+t(H7tj7Qm{_#b>>*SIoY3nHdPpIY*8mnr#*X;f2+^ zA&@7+WUVHn8A<p1&}l)B_{P^Z23jLmcWEgI*`vNIjJO4aLV82mNfCtwms>W^d}&=y z_H)=yy4ejKV#7(W&or<W9!HK{{#}68eH*cT$|+gF<*3oyirOH5$+7SgTJ)}ct`ioT zpQFLxKZ6~rYCSsA^E!uI02;2!YNOMONEz78N6c<y#;K_i!0h*2*_^3YErr^CbzPfv zx7_C6N{wcMA?Y(PcccgXdZ0c)lx{dw*pJI<?G6c0?z`@*80$H<9bfIcmVfr>k8xxB zk4e1g_;10KB<6gLl2li-55el1SWam|$?nX`ff7sJw7v8&jXL0{o|eS(ho=&E)O4uH zCewP;CfC}Q-`}Lz`}xe!U?(cD1$S#!jN?kJwezSPs3A^0ie+D775eO{uXlUWQg!Lk z8)?=Le=VhLC?9KPC~V7}_mWS^B9brz)3bm7Ikm`X#fN8_a=1OD-EYf7^0R;}8N7o8 zLNXmKbQrH)t`e5y60O>=?3Ff5>K4fNs%;L27_d!RUFE(F<F<GKSU1+eS<lX!A*{a2 zutWFeaSi$dOoGT4;KXTsV|P7uz|B<sEJ>6yK5pAnCY^pcD)6V|<6`L6ZZj)3DGV^i zd4)el(U`{{rUMP*2c5UC>&jnD=}TMG@3Nm4+~cZqw3%e1Dtn`0a<(8!V>(5a`1?gA zYS$<1Xr4;}m4-^#q<GLj_o-2W2@`}oXT)#8EhaJ|RC>hBu~}=OnsOJ&MtW0K^*Mx` zKbyWYZq^Z0Sxw4{BC*7QY(pVEM4PYjs(>?h79Lv)rpg4!O{g!vJ~g&<ZewlEHnM=) zzRSsVy*L>mTLC2V=BQMX*%DMG1DRb4q;l0^pSV5N0*Ri^I6+@g`(Ch8<n7-yo}heG zJ=HILfTg)8o4UO<wi7`$WN=Q7@sEcysoiRKCH;0O=WJx_YPDZ=&AWxS$7BLEie_$Z zF6XVI{?vDfy&4u6Ox#R=$4%?rw^900r#6LS74;UX;^^08YQtb%m;(0Ub+@9Q#_AIl zRZZtF9`h3h4>Tk$#0O0sdi%0(d0B(PKJ|tZ5WP*$_{yvW3@Eb=_-qlBHybiGIpt9u zr}KpeStJFdf1ZLy_GIV=O+RbVF`kiIS_B)|JfmuaD#}Cm7zeuYB8J^0e`8grbG&`< zUvYj>?)XygObtlg`m*?ye^$^TfcE`wE`MSuHka*Os`El1fBx9ynB{sdPv5h_#2Z%x zdS6I#LrQJWx}!9q5~8l6j%ny!4?PqRa$I_pdHJ$K&AstzYYm{EbPjU#@)O&03J1{b zca%9*!G^DtWM=&Ub0K}!6Mo0uTA0LZYeln{wuhd=Nfjlvby97{%b+mkE(Z=dJNUDC zcF<DGzM%i%50t<|T$8|b@uGkPPIC?Vy}rb9j4(f8U{bh3;c=A`YHOUJv9Al}R}U{e zDT$a8SMx6#6ItX8|KZaAx%Zs6(UKgYXb49!Re_u}YD{xG_8ghA9r9YVJ{3L5*LF6X z&cD?i=Bx^0TaQ`rRE|5q4>~8wncVY~Z0mdtvG<v=09g5*?Zh5PJFDYprJo?0^%F*X zexZdv@}u8BVo9S-D9t!(VCI~R$7&wZ*fT}$Gj~15wjc>qXQbJQ-sX7%ak4m8S;LBZ zma&4f;mB*Zr}SEgJ`f&1uObL%J=SiO)u-PdJ#;I54Wm$ZevD-<12{3RU#G}%D)}SF z`{}JxnTh0{CUup1frhF5YC`5=Z627beDbZNt4URV!SZ`7w2)e4j&0;pq5cGXZRJoN zf!3o{;*1M~7Pm~vKUIoYt!S=$AB@0@mQ9`JlotY#(38)RkdfNY$^3&R+9nggrbCMz z?A9uu%h^|BF0cy?J37nDc%PD?`{N9llX%X<9`b#MffOgF9Y)kPL!B@!yFU3?^e4u& zrG6C%?q-(kBhJYUgstIj5qM2Y^2~>F!#cw|GTz|MDq9UHZ@YNU9P^trpAyoYwi+dw zw|FWqOdWNJ@3nZ{UFe-1hslCdPj&R$NN)4?_3+jCC%Zr%b3PUyVM7Cwzg7YDBCll0 z<rQ%`n#@XWN7Zf!EqtEUgWgauZrz+O(02yubxYQifs3iu5!6Eb6Yq?XZ3=!g&|$PE zzvkG8hTZa;SyHc#HqxO2&ADN!^_`)H`e*Q$MSqb6jt_DFb$p;5WP_=~^yZl7CC%gY z&b41*7J|wubmtmrw^+|t81r9tmu>c1`|B!)Vs9E9RkToEJ3jmn2!b)RMkiEL(gm;y zT__l^W`~P-iuwy)-!M6<Q%gzxaw!+(-FrKG&j<WG?*P!qL=nVIxIbM;lNNt!QDa*z zb{=h#QbN?ScxbLaE||E0tvH#iUjqMZRQJd!VJM(@Q>?o#ccea!=4=oX#|IA|>N@%v zrfe$uqFVjR+&42R!P7h}ZeF!(Kh2<ZsYjgKRF(Y&_HGC@h^)+%KCXS~Ywd~B>?Jg! z4CwCR)-I0Gg~&~TbP>zwXMJX93M}CM`i9XfeSkD~z}CXwuAGvU(1(GH9-3Jpe}W=p z1L^89m(B~Gq{|wy1w~dl`)?f!Ddc~C?>)Ys4yyyIGjZ^`tZ=*KhXrZjECD}`k!rg9 z{vUG+^{<UlOGm;<nfeUJxTQ14_tvNNj(^ZnJ`K3A8LRfiMgmJkV$|z5&X?@hWFy#T zM>&rSW_$|76PF#CNzF_iL*;?$@_y|SS^-eLGDB{$vx_?lWZ@>T9g4WiVZ7^HMX6X? zPn7fVj=}eFu3btd@*~~G<6f$m1;FQc%;;NPd{ojt4zxU;x49ceVADOe`jTrbv|a9R zJ3ISQf^)nxIxsN?-KQ|@=YWD@To@rs9?JguLW9)U)U);sCm;DaZkrm|8$>=Po_BS| zGcw(HN`XP=g616|KVO!$B0Xv$r*ced@WLf&G&5%I0!y8*JC!VsN&Nvy?6KKw_v75Z zH<0I5s(`B`w&ki-cqorn@nw2!W+KiwHQ4YyqV)`6-Fp2ClCf<&ICSI`7eXL?I7Dee zdqk$oGVE7C%#l2o)po6b`q2S{Kd<{h;Iz2F)rvzd_lxHG6~Z~0R`kT+&0E``bHQDL zL`AFVN7j8Z&b@HV1J%4SHoh5x(wO4V!sIFjk9D*s_c(ir5*a}&mjB%0<M_vR&i(K4 z8IGi<=_vZTSDMDoR&`%uUpr|{GQF<&2r4Fm7EPF3Rpdv=kBR1sc8OjSr8c}hd(%$F z{z=)OSJ|M2=J6_M4u*CjHt?;iEbF;q^~);!{B=5igh;69GtsQqE?nQgRkw_qXm)<A z(g_13FxUkrH20MQcOG;B2lSHSl2?DdE%77J`hOhP4xYAolG~Evp?CTKQ>2g)$w`I_ z@kUxnkqORiTqt*?(2Qh7K`^rGwhRMk7<b6uWhRg7@67TG-boD~b*ZDB^0sz0-wx9n zm=-3XYGkGH&l9G{)kjK><iCatfas%%m&OuSES}(l?Wi3dGrZkLoVyos{E-R@%HdTe zLg6G4jETQfAF?uzzQ34qEEsZ2hKY|;K|q7KpX_Q)DrW%^Af%kF`;-St<|(WCt5n0z zJHsC3OMG}_{Qf=s81sjh?$TaN+sa$(stu!u%Ne$PphJcV%RxnpByPbtZ%7UwX3rBN zeo~>%QciBC#z4RDhc<I8g~4q!kyeC>iD0>(vq-;ez6C4CcjE9#ro?JLW_`7C1S__Q z`n!p~A3SrvHq{qwjB=wIKIISRP(nrhp5}~p9zVZrqE2<(4o=SP3@6?gITv*jU(3qi zwJUVrga%MFzO5Q^h2ijYa0}GRX-QC=d<*sAHVnya)Z$+2@4i3{E{9MAJvW4|k}^0c za7u|bf~g9$t?Og(3sa`sLLYZKkbQc}_uCJviG80w4gfb|x;dghyn%n~XH#s|i@@ew z*d;3H+}5)F!c4B{gDH<Xw?%edj{&8&JBw+AT+Rq~?XxbJrVjLaEc^rq_-xCa33;Hs z17&v4cxU%x?pY=6Mc(-l{BbTbacbX6we2H=WZ}D%BX{eJRoZb<EiSEy`$wkC!+sA7 zl#>;j+2_$zHTCXQdTu!NrsR&e@zisHWO}ZBvd4;EM2~`ZPDWA0(R%ZD0?C+6bbIDb zPa0&DL3{O&uu%2LzY;$W_FTeRi`d*b5PSVV2qJ$_@$>QS<C?G&B7vet)rt~{MF7AS zTKO#LadZ_SqXVZ)dN4kBUi0{{^3TtfPi#^Yos|R80EW&R|G0S;=(<G&fYp_I^y6MV z@;MkgaRDG9L}rXV<qaB6D93bq#ombxZJ970xRok*sBijoTXZoV%)QSv(4<|8Ys<6t zXASoe+<k9|1lUViHGxL>Q@U@A88^U<Fby*GrJ)YEokI?!=U5wHlfBQdWXoO&8f;&c ztuWr4xW1AS^KP(HxDIUj>D5&oklsL|57P*Oz9@SDi2Uy3|12Ob3x~p7k0Zv5{<MTq z4n2)2(Sv40%w=gX_YnoI+%M!-M*z&lQJf{}MZ!5$E%Y?!9vt6@Is_G@AIdM?*9!4q zf&Ti?>8RyuUF<cJ{20(ApX?aT{mHMUCa-VrIH?1^Cq~t#7i??zPQqdo>$-A6w<|Pu z((k`Id6ASNL*c!9yF;PZ{sPfA2rgSwU7)$vFn+dIJcLsk&4wOI4v9#OI)iJ$=v5Ao z&{Ax!n;zJPy=nLFJmD8l5r@}&MH6`4K@n{7s`H1C-i*rTf*TXSI%2d$J)}OAs2Nx& z)kA2bhRi4JT^2A$xH^pZ+NFGo8(UTLHJrWfiual>&MNipx@{1U7`g(!!gG_5;+a`7 zi!<|K-I9t|aO^qd<Y%nuu6H%+jDQUy`B|1QYk(Ybctav*>a1EtDOPK(Nr9$OFLuk) zfFMDmtO~4qJHN-Z@7Z&2kEMy>H8rlf0b3z1jL~>vDkA^l6k<!VA0tbOWND?mKQ*Pc z?dpOu57$a*@3igH9_zQPiiqs%3&L7PO?(I<>y6!|xTrz(Rx&-JkUzOrlBEVxX+|AQ z+HU|uhCq3V6+5pHdo8lo^0=Ri<_P(r{-lk|7-By$k{u*d`cEO&L1Ibr-?b~t!k3?X zO3J;mgkp6!Z@;=s$qq?aq==i^D_4ow3W}CCc^AIeQ<?l%eN1IcjfeWx$4CHI(~*fc z+mcliIM1_$vQv7m>=|DD_`Crr)(Fd;Hw0J(PH&9H@z*<6LA96{e&wpB^s7BRq9@?P zoh-tf@vOX7>1(8Ud{mG8#scE{a8{(AA*!@tIJ(@dbFZYnO>+o%?XG*nhL`siE7AMy zZ;tqH(2H)*y^Kjf1MO0}Vts?9B!8U(XgOy3lY(U0EsB(Bh(GTyfd@}1odY5({o3FJ zaG}WJOt<#t*>$zCL3}vN?H!3^q2~0<aW6RRgLcjGhHuEb5rp$)tmsBSt7T;h`*8FF zm6OHHXerZ!Fzx9s6mkAKl65{6AG@jaW~Sil^FR|#XbvgG$JsC?K!e!Lr@M5(9uFtq zmlEWcyB|p^Ivpshpv*U<{Lt>hjF`dF-l%k!^UH9@Rfvq;DN4xZgg)kcuivA@G%euS zH~_o>8+l$|ywlMH@W-sj;Fxn`Kf0dd0V-WDB{emN-RN|Y3YzHD(JmLbh)o|gtRnU{ zwaL(k2CzSZdVx0#oR5e<RHWOlQPaFRoyaJ69iW*$Cui_f4U^))5J;%Wi#PLs^Osex zY~f^#{^Ls99+Ogi?TXj}x7z9(tM7nWJm_nzDbY0%+=w}grkq|D;(Ab$>(>5qmh+&P zb!=+$1J~xvX#;9R@qwWh?XV6c@z-^vM_gUjpRxpLaDQSXN<Te+)7S{Q%+~JE?*b1z z3h3cyn(^V=;Vh6jseHojYp)+<c<9C`j$e}WF`T3!E`vgMier)>%Yaz$f=qns?B)~0 zme1#~Y-p7JvaPx1qlqm;xyne6eJistWiu^YGo1?Bpld5DHWxF#|KIDgG<)mnzZBG$ zA*mv&qL0&5@!@o(q)tauuXa?dYrB<cRJ+wvGs+WL6WBQUw=NxwNO%je#NO0R_U!Hi z#Rllw9;s*^f8&KWOK9im;+d)$$@08*hjyAM%XxOkXH`hZDB5<1lINX6G2m(8`OWhQ zAo9mH?>{z?QQC%?3LNTPfqdR8_(bQl-(@8}z~J3~PY0RcJsuD|dv3W1JSZ~0hm&^V zlIZfx`-8Ee%GDw*g)2o-)sMTf(}ocbUtOMNfVMIq<*36DNPTRD?L*JLlw$m0oM@!f zp)0!7LkGgQ;-9p=EdgNeLlk8nys*X5NX@9uM9P!6P0DfT+YEo8Ozc6uK!6-09agiX z0h=0d8pd!(CkzVYvp(=`!6XMy^iqTT{R=knN?5CdC8nx!YePi=-*pT<ZN=j~OEij? zV)X^>Io%yhyuma=IC<j5<-4*5-lmCx<M!%w73U+r+W5y~HvOfDHyz>`1sU=A)>!hs z*5v@gWZ660zROtpHG*ESccx%TpO|)VK~6=AxOw1LgFl?U<<YBYsqOg}oZ$mQ$+8(0 zn{8ao>yhy}Y@*hFzL-K)mI=L6dO1zOyAx|}<I}Dk#q+7Rzysj6ySENE2=xa+BpQSS zpZK(NpSGHe@v2VX>r>PQ<gM8)fCg^mK-(yL#6<HEgA($i;I;4UwSZ$=uc=bUX-QVd zqvrMOvCuMJ4vWHbB{Iv)dzG=buBX_PeZbJ;hp=`8<ZxO#<|Mslxq?4Z_Nq)@o`b*n zwYMn_6K$KS*%)eW(5U{2Z%BpdoN7-Xs8uQ%QW?ZK=gnGda>!cbZem{mf^$Ud37v*3 z&!y5~WJPI`$=p!7*vqkS8hM{MCm^=tyk>e2U*%;NE<1WS9Jm?v<(_!oo&wik<VH>T z|3CgU|HjWZ+z*SDexP#I^$pL(Usoa4H?LjPGeCn%E`M<J2hFG#fo4b7FBo3GX={l3 z_5HYz=`~E6UBc&tW~FT9FUs#fn;V`UF`h!+;%Qk}kKNQs_B7E^nya_`m51b^_KJH1 zB7ju_sInnUv9Y&(k`3Cuo~9dNP(o1S`cra}L)oGixcS?$2fUf?tse*-^>?j)owoN) z7q=Jr(14+8MOf_Hf^~^B&-7GG$;oAhYkux|J@@sXS_zxRN5{1Yq$jZLOcPFC$)tg9 z)J^dFPn>hn(`E3=17azmW4e7-hldiX4Aqr}VQ2AsHcBKhd;Q->b_jqm7Bw$Fh9lxc zEU@%y=jK_`H?33%pQCZgAk>L%rH44rkGx}LJ}($gRpL~`>C%`BXxD~#DHC-ga=6-= zwH_0gJtx{b@>?+n@9pxicrtUsS-F?|!$K1(JqRl4Ly)*j@*P|rL|DkQn4lVa2YYOt ziwB%F7<S|Gsnz!~b}VEch&ms-c?`8-D6^asDx1pE==^E30e$SywJk7(x*e}!Zw&pd zjBrkZ*$vXnhbM~?$Uw}OY6jfAy-;d}N1WNGZ^l3G5QkGw?U^{2KPr5sHxUlpJP8_E zUK_N8K7Z3v1TG?FHcm>N<#E1`>aUERY*ITOJeT!m>iuZy;%-QIwuk!Ui|vNyXIPkV zQh=W|4N(1lj>C$MRE5gl2UFD+XF3|QcsD8(iY0_0PdF(mVaJMl{5I1JPKafo6k#J$ zrIHlGw|Ttmv%cb>1-+R7L%uX6b2hGMXkl>B1&z~2Qf-BVw;35#K-ELig>_V6C0ERK z&h-cNTLsf?RLK4?rFRQ7ndHKJbc99mNFFWsLs@=KMi*);I01^O#QDwNhS~Zs^Mc9v z<qJ8RX&+w~rM;jpmvRzRf+uAfQ(`l<WmeTvzee1Tis3k%OfB}QRSy0*-G8B_kXG!M z?>Joeu)n7G&)^qd;VG{HM}uPYK>6SG^l_CB>skN8EH@m9y>Z~;0k;ERs!ke7bo=b9 z%bdjtn$C}Y^wzhkX={X@5HeG<1}9Vl`i@VOeX+OO-R3DKcNEF)r}F=*fg-*anBRp& zO*i=lp&sw4I+KFE%r?WzUchrDG_tOarW}DVrlViV+AmzUoXj<<^N79iV6UALK<_vL zH3$ab%2P&Fzv@pb|4Vb_pwEd`6!Wu6w>@4%Bk)H8FU<|0E<34x*Ka%69zxKq0Fd3T zS2vZTAX>mt*rhI&hIg=lwFua7q8|&CR<R^jv$e<P^&{BU#XO5CUZrq?ZC#nw#x$?M zAv^zP@W_GUHP`sEMr!OR{F3AY<GC~EZGV~7ctxvHM2Tm)n6DYy$^?PK7<e$dEo!q> zd6F$P6TNw3U&(!>D>;?BJS{Zwu&!)WdhN!fMm;?~vi)A_(9KnM<|DWuEXBjiVSuS; zkJp5>3HsPgGIWuzZl`XKQ*-jkf!Ll_?m*Hp^itWtvcdR+9;W3&PhEv}1aODss@q3@ z!A|r;)qECS&fSy+tL2D#uyX#?{My+>0BM`k+DE0kUT8CCGQr-=!l<E_y-q8FG4V83 zVYm)dPq36+G^_E=NOQy6y?EKpJ08~DZnv<p8C>uK3licBD^Dh-@b}z8ang*-3*_U_ zSSQ_wgX#mz83xk1Z%BnZR1-p%BSE4TbEcK!Vno=Sw2JzKYYk|<vM7k7^3v;Ubr!B~ zPcrcF%hImxRYZ--M?*1#+5N-{&+U`Z*Fcp?K3b3Th6<WWt}d)RD@0_TKyBAw(0g1u zSXYc?CY>Vs1Mxm8#4{C1+AT(`e1+8yAk3wd`Ye4s<A)^ghB+!B!;4gqQkOjEovdP2 zFO-v7-_~D8f5Q-9^{&+B=DE-XPERI_F<OdOz@P>0PsjE%-*Tc0DR*6z8NvftbRi?! zIb^9Jv<t#<+oUby#wHWPt}Te0H|$W;$EHp!-7QoZkeHy;t=t-%Dj*n#MMSaVMA%k- z+ze(oq)WU;n-GxdzJjkqjGPZa+;&*zNB6${Pl>YN7(K)baGd!s<D7{hvhAJgHj2ah z+o5!xakC`G3Ozmk<w?~mDiVjp5jD7Yf#vh^o)z3a?J480kV%K$&YY_Yc8vCU0DSJr z%fmn3WjiU|>+3*lDS9ij$^^=9#HyI;H=^_^o$BXZSmJJqCzWyY6EgPne14H1RQJSj z-JK-`gU{L{FUU@d#!b-!Da)l=JaeqW)MxOzJ10tvB_CWT9QlHFHUsM_O88#)|G18Q zKd{`&mbYS=))*n*eFi1%k4CN<gDanqqR;ya8|;@7l3mvf=x!K|cUUUoH`mk85LVw4 zrqPi!MzQQ|e7L~#A+W5~L${{OJosfl$BqEtiliO{k?g_L$LM=$plORdF9L<(4^F>S zJ8-+$ychC<ee1iJv>`GkwpO%mD6d3Il3Zu8Rvs1u*y0Ww@{N!Tw{<SI*e3E1VdsYH z2CMH>88D?n9G%mldL`dOHB~cG17WFHA9v|eVes#<&`(b2O|dKOt1})=DaaI$7Uz5~ zjanV|rZlUHX`fM?<)&f?wBz+gmp<3#lM{vYZI@K>lM*d~GU}0R9nvQ;m=?Y`>}$h1 zRe9n><tE+>DC1RpKX25S5;=DmQch!^#tao9!n^aa3wX#ua=I2ayKYXG0Jz`juW+@N zNu%jjplyfrKh0(~5x?_ysA>H*;sKZV<kn#Wr^ZFV{eA`O?DO{ONrnDM*0BBu8vX7X zl1P7x{gD(_X|*=v0Wxy8ajntZkuBrcI`ssI8MucJ380w0K}g8}1DaiA-D#5J#ZJSh z>xOe(Zk%+7?oc%u4EUPsyE^@pRcsVf;fRZPlMDmp)1!eI1|QnI`5r9IboA|w4x%)| z%oo_O*yU4^y{--$*tIyB%D&Z)HDO3Qt_aGhiIfy%<yM%nK}v<RT_EkO|MssOEZ3vS z|1kXp2mYnjB6E<$z&gL0JvH*mO477Mq?tTRNJ|UW_*Q5I@$;?V_CA$meTvZ7Ff6Z3 z7a4FDSGwJ^<-qMO9Wa;Tt>OF)fJDIdA(3nIqwZuCF7_$Xi*=Mx-qorz!eIIbTd6{~ zUgl|7oQQcfzB}RLWwPhTBq<w<!rDB4AbmH5=A-qxKZJs-E2Qy1$Zo6pn#7&#!&5_y zTysk)bf2}%^})4>Yv<1uq?Q#*ErofVzrHtnxsjK=tvxxd+wJqsQ5)Pf+dp^OlC1l* zN4LsrHcV>AMbG8%lhlGU+>~l!kBXoEn+szaQpGEw$S5I)9MKQlj8W}i`5AX&=^TP< z%Bx@`$T!uy);x;lGc|irw*~&GV^Ga+qx4}lF3q0uQH^gHLa4f8iXt9_ELe$8IXzVs z`W!MwOm)^<(%2bN9|Ak=o{+zOMWO8H^9}6og^j+jt?X?$T~DhenKh9Vq8@gtat*ob zD6ef1J{4<i^!NV09(D3ef}gPzBUJ*0wZH5G+AAm9_JxD5z@4|>Yme}K{aW-p(hpdi z81b}FSH<tl(@D)k(vXSI9&Z+1H#@TDF5Zk?VaMJ7&ke0%|Jc5NDIzBt$2M7$8ans% zdAj9RL#oF;{a1V2Z5wGh7-cb$<_5Q4ggecD)wDe`diK>Ugsc@Ca`1|Gb}6y;d%1N) z++j6VnDVx_>&j^Ppfp)1efal_yed5}{Jubw>$0K(Vs`O}Hz&5auw;XA$u^!o9Nx$k z2vb+52EXT4BBd;rb02@oA0rg~fKAPA4?wri{M1%v2>=q8*BFssUcBfL=pT*4t$$kl zvDhZ|Xp`^P$^gPc^Y`(Ejn}Cj@4QD;_1yJw8uhB{M^D*6r|@0OGdtuI+l>h`g5{6> z#cCENV~O~?+*VIDnwfS<RVLS+!slNnMsBFp%3fZb=RcA^1wtCiZhX9FjrU$N7h1C& zqR1$NR)2-x3%@w>D9?a>?CFW9>yuFuPwf6)=l#|GMN4igrH&bsGlf_F6q!-iVi4L3 zxlHdIxV*BUHS)6Oq)p;2!r_+8@#lNGYDSB_kWtUijHXFVY)vHa?lb5VpZG06-Y?zL zK_UB(d?h+A8!VkpYQ>*H5g`JC2viINqn=Z~6#d>`bwmJG?;RWe%E;aBIjUrN^Ml{i zt?gHL;r6M}aT!1{cEYu&PVr59*KeN@R;PcfC_WT-BdYO^&%&ER(G9$H^0@rNVY|Ia zpSmyZr6AS~Oz*f`kIUm{8bHKF{u8KEof&DKPa9TnX2Ie)c4`+q2nDY&>aOK$-lf`1 zK|AVE^((mz>Nz{=9z)xMzt0onP3Ezec3(-moyq_9rCt1|Px3Xqc1lEfaDjwU)yh|h z=k@|#NXBiG&Wyr%<CR;i({hDyd5qub{ZAIF2{R_$NhUBf^nJ~)0&A`_VoyZJ0g|fo zbCZeK`?QELe#?bF>OuqAuer~(9;)ho2RMpl>%>b(jUb0_Bkj|ne@L+^^D7;^YOfyk z9XFDzTiN&og7W|J&Zn1_mWmo(iGt{L_@7FJV>a_*3wHgxDtGe;yKWl;_T9ZyNs^+p zRB3}~97^f>hdjsqQ^?Q!lH@Udr;*q_b+<j@1+o$Xda3Aje}ZoRZL<w}3$cOrABa#@ z>tEpG|NN-;piv7u5|BE!R4RoIE--hkFHl*@Pw^fPuEdkK?vGUu2f&R-a<3Q0U23>Z zYd-Y-jaucG)A$F@H+b*(<D{0C^o)`O(zq=vq2`~D*m;lK8p(s6@`+QP+SoUeIELvu z(Y022BkSpyLfS9!p<DL-d7g5TY6(r<YCcpG&Ed!aKY3jkm^zT-(=C_*`_*&wDa@!p z2<3R&;77=y-{DH-v?P>16U=;VKMLBdLi(fr2inuQR-(33h3X#U+Nk~g_Y%_k*OgH* zzycv<qNQs%X62DrjUB2#*<M&LRo_Sja3%xe!!5MeLAz>Z!s}JN_v_~SrD7Jyl75?j zqVSJU!d|Gq+*d4vpxgJ`U|pkI>v`|wyJutK^tS`cs{`oPSAXXvRRsp}9lGUKhWzlY z(&Eg<Q*dgK3G9l7GH(>n;V*_Ap83FMUpyRX?Mu99aHFiW`pQss1e52`j09lq&yqZ3 zMWR=~`SBK?OtB@%y02>gw*Sw}Mg7mr)iEhi+wZb~4tOHVbyn-|eb2wHF5WYc?{Xku zfTqHj3SR>r3XX-dBEVEn(bBJ>+#0P4%)-92XBYx8SlU=iX+pGUBBgE9eaYMU>DLG2 z2`^dF`deZ1$kfONPvsVZR8Xo}r9@gl<o(7DX#J;7lZdG0M6?6bS+XSU{L{%@r!UiW z2V>BF+3_noy+a4Huy+dP?F?si>E#nF<47~EEw<DAkD-mMF05J9x@gdt#d--#8oL(u zcVn=<-R!^T-h@Cr8J`qbZ256zuiZGJhUY(ERHK-M|31qHxrh3=E#Pt`U+=&yvu2Gn zp+vwv&Vb#sZoEw>Sv&yXJ{Yi3<sQ!IF>#J0#KZoQYvI>aZXLShAnHrVg#n$#mLv<y z{mu(zx!+Gw2wT&;^32KghXE+9x2dJ!$FI9tWY6M50bjM+qq6(~VCwW@U5Y2mo|14@ z*}3{bCd}?w{2n)$V+5Sgb9>|E>B_7WmaCp)S|O|cwy>I|2OxV~*tMrRu+iJBFyx_U zYv1L5%;P_gYG?F6kBW~cZ&uy~Tl77v1iUJN2W`yFwb67>yf$XNo9GKS>FLoP+uR`( z1xgFuZZe-xc{j)h@flkwTCdtA6jVsn)}aejZi)##Z|nd{<Zm2{+tS~#=`_HmIuRsI zKgAb%)L{od$`HIEK#arMZZpwn7&2YOZDMje?y<>v=%KU%!l|IhZvn#kqXU=us9Xdc z8RnN4ZG$FJ%JPDbYSL`-BSGw<xe$SprQemcP6VS>3B*~nvijunF~<d~Ta$Fa<H{)a zcUN}x#^gst)c;vK{`#L~CGNcD=Wy_@2_pDh-Sm}X67AW^A0X{_i!Hq254QpiPlcZ0 z@eLtF&g}FqtgE<4qLb7Lw{q44ss>(vKd7+!x~9i+{&Vm*?SXEe^<<xoPvp?%QpSYO z_^VqEy^HI~L2oAqrN6J&h^;mWY!RA$*SccY!jFD9I;Qb`ov$ypDJ`+3>+Sm3Zd_Ju z!S#?rYmZ%5#cZFG9pU?qzKe!e%k2urPVa(fr{`~3*;||m1pwYw(g&VqJPuIj{mHRT z<~Fb7{!SCT#rRXUSY@Fz;0SER=Oi&z2VXgV&3+n*W?%HR|FUb!$9p*mzmFv6oKj^+ zyRdX-!LBS6*!}6TK;7E&RB82%nGL|(FrcsFHaSf1haeWpUtRvRA8;7_^mUg9ZciD! z|HUL2cCi{0xMQaE(p(a;1z^pQoD-+IS{+W(&=dMv1if3nJ%8-XPOjS(NXrEWGzIwf z#`h2V+}3zgJd%vaoXFoWxDe^jaCR)1(sCIInzwrEPJT-$?e?Mf=oUO(FP|{=_b`&l z_8gV@F9U*Y{=U%X)t=p;$XJ<niv!2)J(V7oT?*n>OO*z_sBFA0I1XOt*z6P@c&j`* zba!qSXhq;+zaGLY5=MOOrNLmhPxpyH(OQx8^~pH#&5|f2@tREw-8?3yl3i<W_$1jM z!pHLwsSJEA`^c3%CO|gdNz4DL*xOOztVh;UqY=UzMlXCSZULlGW$*LB`Y!n*QOT?J zzC^e9nI`F1#NMqBgF2wX$pYG&2wHStmsU8cS&Vnq`6Ph>t_(|x_FmV1^X}5Lg1tbm zi?CFVcHra0P^R!0^OsKQf|o=3N2`s7af>9<S?_7NmX@d`M$1N%fZQUxbMUX0^O42n zE@Ww;o`e@ZD^_~;y7H)v?(CF`RdLVo+3Gi{l~jj!7g|HKNOmo5=g#VO@~;mV4|+od zd-$==y_BYF1z`HYn3I*;0<PS;0eSv;g{RmK7bAhdV<!p_wiKrW&nCwc@2lA7m}QH; zKuS_59*IOJNlZ6aucXa8`t!P<0pA&!L>D<Fjm3O4{xjji>h;J%O@q{d&I@ZPk$bl# z=CxW{nOU_e1kYJrtp(?<vi-q-9EAIqH9B~Pl2*E1eykK@m-H9tP%z0Wdm^Gmp*sQu z0>xJ@L%62=Pe!8EeIxiOA>|IULm>zvJ`C9^J+*C?w*l1-OQZP(6ikp`1vqtS7pF3* zA;k!IVqzTWjD7^!R&-IX&9YUQ>1cD2QwXa;){-_~agbyE4ikUww1nuWrcNG`rt~x! zz=%^T78l=<v4jP^YKQ*RfQlKENQh=S0{~k8zX<^6@;q6_Dh5dx?-m>bbon3NPOSXN zNa6Tz0Bp?dA-*nx{r2UBu_<kUXTI_&xAi@3z)$y0O;1h7)Wiy)M>9wg8QPT1v&?XG zjRxEPP2O5_=SSj;9|lDf9t>7ItZ<NUq~ZF9)(xNdsg6u&fsWFb#Qbmxy94;@_HH!6 zZUyGEUDuijo?5~F<~n|wg<z0W>;Esh-ZQG{pzZdSCLl@?1*xG3h=BCoi-00v1Eiy& zS7|{Y2~rf0-a#OduF^qz6H5p+6zL@@A)%LqmJr~L_p{y)_j%6YTNW#8W%8d~Gjq+} z`#05OJ-uBTc=A;e-S{njl9LjK$de6N*qGa;)VRIBA+mY%wAf2+`~PId*7Yy4-p+{g zXQWy1S_+CxMNUdK_dMZMxrfuyxR#1}*S)Zlz#EoaX*RTsXwQ<ELpD-2a%B$-?ZoE2 zY?o(8%*O<#Z!_Dv04;_!lnfV@)>Pp<^RFyRAI80+RLGuVUv^?q_1<6ksjePOnUGBY zly9m(SGMJz`Wvm&UCy4HW`94utYHNPkEl~J<L>&^*LlXn^@k*K<8bYhs$y$08w`jN zf7p%u6aIh7Ws<6)(fqad-*MUxh8mjJ{PGF;$?@^jB7ZNexj^MPGze+3GKkQOf~u;9 z&+j4#TpBGX#Sucn-dE0ePIO#Z?l56S^1Hm_zvAtPb(QU$fuH7o<@f+__I<vgVL44X zno5Ic;Di!zURrCSPh)k&c8AK~dzZupL^BZ<Uew`JN@)cAE<h>BIJ9iJS@onAW)<cK zwedP_@2QX9wj|`V4Ia_L#hVxTv;mNvV1T6$xW>!P_W<(Nob6rr1pnfhwIBb@?~TM< zZ}{HdG^pE9-}GA8IPpbDTy^WardK2w+h6ks!+e$sfJOc%vYbY3f5Rg?@2S^!?ICz_ z7LXzO=z9>Yp=Fazu=ZZXh028kfGRX_^e4_~yLUZs?6wOWhZjd{C!dE{>^BD*`1kdO zh@ee>F7AY(Bgk)UIRQIA9cyOqsP_n`J2tl%ilU|?7yRu;j%kv#%P<`^J2u@3n6uu1 zv95RW+QR{u<{e+_GWgCH%gM&O07UH06n^Y*nK-PzVR0>l1LOOD<~lDw7UlkFl%^)} zLP>Mv>e#ADGcS9N|C^`pbcs}JHvkyGZnA1lXXXx+^_bevMDpllFZwHIxNXgw-vOpj zV~m47p$B`g#XH)sGHxE!Pt4P}k>ETg4M{w&N?8=wqoiThocsgWUo$1I?$UTlzJE1r zF2fIi(Cn1~9z$t~MO#L5Gv`VIIR)32oB8DVFX=Qp5Wa8w_=V3u7SQ2{O9QyzGku}% z#OGzLnAm)P3;6}z>u0wwC7$afnE@9bUmH+iv{S~mDkTfc>p%FJA}oXo{;P$>Zqne3 z2XmFjlvUMmqus1TpxF&{T(C1A1sbg#hG6zNwjX1D^cUVXS7u7S->o-tMR}o)RrqgA z41T!~ETM|bc6^clOjxz>KY^90N$=uGe0>g3WMbP-^LIM$w44PwF_F}%-<}|HI_q5E zfUW%^I5qXS!d(%Tm`nC-N$+S$e|hMlB8WK=)^oU4FAcZtJzjWXSh?xnJf-*!B?sFk zipze%jcY<x8*7W8z%ZyI-@In=&m`UqeE5Rbo~933VH7ah6j&42_U(PatAF{a?PsKo zoM%+$T;TzF{>;zwEmHn|ownY)<u+9DLJc+6-{=qHhYVz6DmTC_4~;p5A^ZS9?RG#; zhvXZB&=0A%I4P;0Vp4a@hDze~!MNjGqh;cl^4^>764Wi>v;C#uWsUrAStmVXu_!PC zX-djRW0rEoFmd+eYpUK9#lF|b^?QB(Y!IU!>{-ZP0S7HK*i2(Z+}!%zbLVo$X%}!3 z^m#(-*o5{bPD3rhDPeEj{0d^Mo!EZsT_Y1*y*uH(JS0{}DDS)8;qmr!yL_=>Q>~p8 z-*aDV)6Ndob{XyV4rLf26EJe114GG2@Z#xj1cr6fxIzIqyniwuE@P-LuTef^<nY>j zbVT*9t71Lh|D(r53lg=?n$t(!^3a#&RvM+y(oJkE)3eaoBMvl?jMydO<b530%$m`? zNZz(T>e}`c<_@W$cs7K)$B4`|f~(hpp!ZJoyGh&c7xH~1wt3@3&V}huEp;tVw04vW z(4=l<p5<jlMD_7W4JkEN!#OzU1T55|F{mG^a3=1*ouYw`)_;WyZ%0>kNB6U+9iIvs zRoTG<e{_mTujK9cLIa(<(`~nxz0NYCvBIHP<-`KG%k`gyj6i`NRF@Wp{xqrfKRGvI z|KToZX+WMo(8aT+7Rz^;hq;&RuEiDt`t$kS-nc*gRbCm9ahv~`l3~<!P4wD2VMi-8 z4*Yn*ENza3Q6e4F2B_>to{XEr0GZA+q9dR0Xq#2b+-GxOTV%OHXQzGjT3lfU=PZZ$ zpT#K~=;deEbE|FkO?)kl%Zwk;)7b%0uf`kzmwKUJoV|>;4$QTVBa2w>Ueyk$ZD0Sq z(w7K$NUXSdLL5lp=zhV3#u%kR*Vc_FGXA|6+QsQhMlI*94UpM8^&=_j0oPu!DUr8d z$vCpVI+jIVln6QI=<UUN12MTOE)qO<X|;uZW;mYiMtr5=uVcM}|LUo9S6Itg7;Xqc z$cO)Y4Vc-8{d6ck(<j<Zh&Frl{EnSiw)b^lx8VoD*O(`Njmpnf4LQ$|Q!wwFpN}_8 zm{r8O@k1x9+?<y}5e~BD%dK>u|B}bH{5fgDJoHiCf&&QFBx1dpwok(wDO+7%@ypy2 z$gqZCd9d-FbNyxorc`n0Xe_2A2TG)WE}TPDv|Ux*X+*p#%UKw_qz(ynZR~Eiow8Ju zL+>yl>zop5rXA#zBYQJ;NN`)TTqgbp{<{{fq~^>m7Ctw)MwWSrIVMWhl%vPvdp0@m zIi<QT7U)g0QW7)@IOOcZeOB6!$eBKlYbZyO$s<$k7po-92Qs_%vZ4~5AoumXM3lGr zjntnlq@X){1|R!{{VM<wx6_e~r+X6c-40ptZSOId_L_%T@PqKPza8GSRl8!8Gt)xw zKXX)`@-WRN>k~rzUo))6nY?3isDUM-(d{fj3rt~E1uw~~L$J9K2VM?rVnkn!c<~5a z3bVtxa(KwuP20zlROEBv%ikEHM{m3l8@jwKBhlEaBH#J^;*E#Ry4F9+bo|k}PicMb z8qi%=e%K57erlwD+KY!g?&j;yvt6{<ncUb8gWUehGYRp#BZgAVQzn4@Tm{S1-VF?~ zd68pK?G+Dv%EMP~<&ZI@fTJ&wU3v<hOg~~Lc|zGYm9qCNbS)0<o4}*0H$U<EWG|ry z`dhOW$=k8+r5pAzdfc0R_U;)!k}`KJJ&JEREA*{ftN-u~*G?tqd#`&qu0MG%?yRc* zfr3)plpFVk3q~|$rnq|8auIsVF$!2_tYPTV4JJ*oq&JAr<rZ0MHI&BkyZTv5K3}?O zhnm`I#OVih%;k>vff^^9jlk_2#bbeT1a;7WJp4>^XP9O-7B29iOTmNrTQXL4ihc@m z28Uk(`-*ZCb*HWmxI{D$zyHQ-M1-NoL#Kq_q%Jt8q$rK>TW2)pYt=a7)5uFTO<gB} z{M}BH$j*3F?%)9Kr;>rZ_K-NA$w6LA-b9|!du{84+>>-+xf}Y&fM4udl=Rj5TT*B_ zS8FTGn@hk{*+DRe{~ZV_m>b6k`9N92AB7@Bxp|The~Kt6_F!Md;xT6>pXU-8PQ9jw zZqi4kpF&46+@DsE=$_K$Sxt-18v|lu`dcr2>DLs})%o>e^yUI2L5s=Y?A8t6KOStK zCDnr^nb}`jrgTy>2c@H(2|iSxM}A2;LecjG4B9w^qZ+m|etx?d>CW^#_-(<>l;)3p zQ3hO5!W`c?B%J*7mu+YtZpPFfTjrc_jGP&nsRWD(^?!?zqBBCi$9G;A|5}_PoXl!H z5Ln2|F-59=)9A&qDY(!aspP?v*Qxn{CF*?6JXVaDoT5BN$I_6W0G-jm(Borj$3;{d zOCC8D46mI~SO~;tiPo<KU#<!nRZ-igO8M1GB=AX*Xs;oi{JHQ&SCcBQsB^DnE(o&d zSbM|p#~;7Oy^%f_U_vJr(n%ld*0lNQR$V@8yB9Jq^y4WwwyM%=tKa_#wRDYrSJ=S} zOaCW6@>|pec=Lvg)?GuN23Bk(<B!qV2gQ33B}xLHSyMj*&9NS(>^<Zsw=^0bhGfq> z_KiztwHL&^)|ySI0^glKFjbo32pjcXj_iif_27@ITawEJ^SeuZs4p6GQe6NAX)>kL z-kHc*jqxpEN3otIyiC8iFr7W4ygXqavZzR1z#!FJT-*6Tu#l7KSeY*)*1^8Qi%M-P z`z;0at>nTK_9q%)f~QhljX9U7xY(zs(2)N{@W+Jmni2m2w>0hJeN|AYbO7Tv!$PQj zzQe4gj*&oKZC}j?d;Bfi-*-CVn(o%cN^br{FO=AlLT3jZln{Jje~s23a-KFwsi&Eu z&F%N~e4(|~`+bM&G{LFnkf1&my1Ln%kp*0(x3gq1AZ$MZX^4UE#c4KdWY!WQ4n->J z6PSUva?uS*xVa%;GN!(dUwvuJIN~4=6YBP0S%MVzK@9NJan1=>r@TKom^{~)BIoM? z4<P2$?~KM73W)J-;>n&0+?+Hpb?kT8Cj<ob0%Ztv#oj;!0ju2d>lXuORB#cfi$F`5 zTb|nub`5}rd@3XC4NMUqraRQSJ?kQ=d=~ch#<>Q4Vl5&BSR!j}kC>aZf^mk{IBTJS zsDntK;b9H^wQ*kf6+KED2xfs5hCRwFskJ<h(5P|i#^n%GaP@6DxX#cQ4noy^PH){Y ze%P<D{LO7HPneSfJ2j<Tcn03ra#|_^2MXX}Am@-}Y?Kr+rLC^b8k#VvoUe88>q(z9 zr)Wygu*q=#&}^bO_Lxih`0|NmfXspQ6YjuFz}0yZ2CR)U(j*AM>}3|%*7-m+2lzmq zBQ!0!=d-l*N4zqSeO5j@bapCNvDTI+snL)<siO#=XsV#Zo%CIwJt>^H;Av3u<~?Ig z%U@b9x*pSH(|2^Tbc!o*ZQMPD@s2Q4a@XQ**7w2$$=h_aw0TW^7<;FDmN4Pjs4=}U zy@?m2QIp>$qQ2?L_Q+dqnHY&4)!?Lz)PpdnWQCaMyPxPP=^{^%T7HT)w`>{Qa*BSs zLyZh|tnE4aRhP`?X=RX~nf<h}D#kb89zMgk!xiw(^lY>I;N9LwO5tyVu87B9WBb~2 z!NJ}(j_G%uwx$Mo_j`#J1&*rUvB+81TxVm`8k0ELbC9HC^IP9IA=PKj6cD6SA;|1@ zY@A=H9d(iX=%qRp6{y&;nfXtd=|J=E7a8F@T_LMu#pW`vS!V9J_-7XubG)pPh<1GT zQQRtYpZ}X?cgWQ-l@r(GODhpb6ff?kQyO0_$2)4vZLbt1;pfN2Iu{$R3JO2$?ItHc zW=b#C2+ey(clJ6Sm>6~oB8$7ok&fxHh5ab$MI4aYbxs#B_hAuFP&Rl|Sb}%0;6hJu z$Yn$(;r8^<m+SkXlk{Iq0S#YqM;ya7p~-bwP@x%pH}DgPciu9f$v*E4D?MJ9vyXZ1 z<#)y{aV(N<%;l_i(PcW{@AR!yf0f3WWZ~Igzb#jQTzcVU=Z=7t0i;Q7ef3#R;j5s7 zB=m6-u%b|n2;3}UA*hoG;};ws_2^WDomMB2zvt@2g5Pb;M4y&EZ=vUZ^}7|tHBD@; z*Pq95#!UJl8J&LzMpex&a>uxr4$j#2V3-7v$m$63K26=!P4~-1#?$0mKRRBgKnH5( zd=sBs>=<Q&q&eDV0yQ2rD`i*%#-&%60>363Dt*u<9%zOsal|@LXl<pMt0Rq%RKL9& z>HmP*gS2Njw9S3wJ6Z@tjly+On+2W#!d8nO!EsSvHcL=Uk}GjmD<Uee?m_8(c(+&L z*qyVQE2R76zGtp>zm)rX&A_RqsTH#x=BWLmVQN=|*49c3ehoPA%l91j^P<mVe*4|R z<6PW`;MJrg!{wKMmaSKaPMvt~{zml*n-~=ZlnezhLqI^5103p%^YWxaLDiP$=kw6e z{vRsdyoxP{mlc`02e$-5hM(=UE$6|6XEjR_cVM;S?<d5+qhUWy0&$47-Ne047YURE z2^McSJ<LXoKG{soNf`1P>OI|={i%gzfo;HIyEWN|HM6m5p`nBR)dR}syIZr$J-j)5 zNw6*KIP4VG0qcR4VdaC9=T+2?SHFYBI!^7}0tpfuoCcT$BcK)Gkr&-E{kdEYDl9TZ ziVX>lWx=c{Zd{ozYgtloysWzhBesTihlWdu833m=oMoDC&4U$;D$=cN2sXgH!x#oX zg%NI33XN?*UTJK`U4fZ}@?s^hk}Zj2O>_)-g(@`ivC%a~cP5*B8A5cZ)p96%(mIEG z(?{hstfT0B4Wm=WypZneUUbwf^+jUR?Sw13bdl5%Vk&nYN`Gj-cI~0xQdPFuOUtj> zk{nC}*J=F$kHqQ<h1VqSuRZ!|dErL9z9rL>NZ=vc==C3UsRGe&67Mn+X}hH2Bn`jN zhYFptn`_8&v{FK{Ys1LRML$D8e%8N_3mJQXF41>6Pcf~6z5aNHcdM24Qp7;XhxU@O z&)7%VZdfVDe9VN|c}LzDz1W*GX|<%lyk;4^1(bEGDEN78R_C=Xas6h?$^P}?G}BZj zuswvDR<L8$&gp(#9DnLNFm8RavIG+J!8?+QNfG0WsA}t?M5K~+PqKuKdt2;;S>UWl zoN!R&EzZ)%DgNV4WPX9JjM&aOsZoS4II>$}45{4osC`D4r*YXMd+dhjg+BBb3!9a7 z0_hr2|7j>q&bd3is#F+}`m{2|+Zlu6!qNSq4b?jv4zR%xs35?lBK5&OnDZF@YAlOK zG*oi5#PQj#msj%7Ut;=)WxR3|p8}A1Enp(A9BgaNhKno~DcEb|GFqtM!K1oAO|D^? zoqf{HT0STC@?{@i2rv6v?}P`x(8t^_hR+?(4;;0l>=<MAj9eyawdC28f{M+Q81rh* z88XxKyD&1z1pCx%O(G8f)3V7Wl<(U)JY2qx<MehbD2`~b0+f6w8v;SQV@3Y)+M^Te z5*=#4l-ZM_UXxNtD~-Y;((;%PCgOYW=AZZeiB;h71R8McxrA704YaTjMGa=KgPPts zqUF&IPm0|?z+{*w?|oBjegf}yD=0Dv7NVUrIrn~FALny?5s)t3M-~2dAp{N4zu2CM zXv{nxjh40LBh8oNToE@8Hh|6(N@~kEQl=Xy4#8ViW-cH~yrbz31Z~_l*SVjcRU|!{ zq=k$~q{F+5p_eNLl$+aeaZZpJ)Sj-AdvX1kdjXwBhz7fczQ#nN3#=Mp3(R%P#?2S> zH43WFuE@ea5k7RLx(+e1eY&&Uv!^7T_Ue$^P18h^(AnzNx&L6dAZ=Q{ztuD8PoDG8 zAz5i>aPWPTebQF%9c*Kky8@h$h~>d@VJ%=im0kpvW|^GlJSgC_rr0puP!>3!WiOKA zRceZQBBDXF(@7ZlV-AA|`s*1BzN+d7_?GC0vea^RusUt@2kpu_U{+LsH0VG2XUbO7 zj<^G#&Ck-dZZ?j&^3^p&i%ri3S>|7%tZ3PgFNooSiQtI*3GN&_VH)d1H)kE@5sI}I z(}MK{r@#fdhK1!LL@K~@C554RVQV52hRc$vDwMT;{(mq0ck5mY>6yr|g}TV%{1Uu% zotpEtai$9RVRCckg_^@iolpI(U#WTq#}q?PX8r~Q`xH4f++o!=M-t|FINq4}QG1HT zBsT1(N@v_kFo|?ON@G(kkLA^(zOD-D`zGKFztHvCDC#8YfI~ZKd+EdV{VX$eR(4E| z;V8HG^CDSu?3cc8#-qXzl-``)TZhcMM#d@$oFI;v`nanfv)z0HleNC+I0{kDL&=Oy zyHp7RTP89(Hr?N*X}{6MZFiq+v+?dm90&pqi)lz~8pvcd5YS7l;h6G{e(@WxFO;yH z6M(vh^L{;$FK@2Q4%Xa+K93clj&)?D+)QmP3sO;X>sQ!c2{8%%3_g8R>z>h^k>=L` zh!ScR29n<<rKu55Rdv}S+>90^t=8k<mFhkBDjZQvBedhk^x9Ox)S<_`#K6lp&e=Z- zmUNQxz4mx?0r)osOk6IH09U2LO4UII-N?R{&*mZ@bGxHt{56#<ILGZG9>z9vH_k-+ z?TQ@SL%}wOBASd&w=?1wo2>20Mm#AsQmaYg$m?2&*ep}r<^zr&oZlomZXJkt&@$~S zx{R5E%^`CJN@q1@KUmL)+2GDuh~Rb?`gV$aMd_)-QaZ=x5L+Sa-n#2fp8JrQ*;z3- z<_1@437(+a`vjE85>~0!SM4jry@q`Dt$VJw+*s-W8NINqp5<Q36IQ0bT>bUkp1wL! zu*0?GraIv+m3l*iNa3aNIy&o_2e$i6r*Ztp;UEQWp2WV*PmZhX<ZG-^)Y0|vq|6;Z zm$dv8*XehAv#&ncue(UJ>%ZC{Ak=j4Nw0s*SzL>b_zc&~5;M-qG4TY9s2~BE0DQg~ zYe!VEw_!JH(&lrrg>={F45_TU0PM^4zi`Y-ZO}5&g}wuyq`2;4;`227tYT-^DAH2N z;AQ%!c!XhzzoiU}D?ITO#sx!TWnhH}*}%Nb#{;~>Hwq>$C8n4dq!6vm#E`y-p_IfB zhWj3gfiN$u6igWr;W3<{0T|+0K#{&T-qilC!@P3vEEMRJQ<bNu&<?{@sd9$rsn@A@ zG(@(&I|;ofWz--n1~J*-$_>WYp(!rK*xFFp!4c!L8~LtA!sidmiHZ2(lwTguzg+WL zFST~Zj1Ej?lM0@_Si(jGdP~zvZn-P?jl&Ou&)l^7r}Z?ZI8`Heh7R&#KcUD2b2DL6 zum0FLWE128*v+PbGQK2of(Al~5CP1TMR7--=T)W$dJK6fj8KC+{;wx=&F1HR31X3t z*ln2nru7~V0oZ>NM7gC{12ML7hE#88UHi8pbWbKdo0P;k3$A~>Qgmxo0twThz-E4T zy>3-{YsWV8Fk1&=Y`~Nhde!Bdw2YUaKoVW;Pccmqq_m28%Rq@2lPePo6RR$owi?3p zn(5Hy*R$N$`yA?|ve&hAZ|)i+x#4;4m#K;t<1~L54_r#~_6M?gk2(9_F{iLJDG|Lo zK^Bi0>?V{9UW0U?w!@mFKVlp{LBhwyk_~RX4uku;GS3#gSi%X97#P9aV{u>fc`>FI zk^V|WJW9<EY5ZPay2N1ktYGB^BJXA)#E+r{(d-rioL&kT!tiL1ib9J`$yfYSf0nWE z3F%)weryF&a^d{&Q{u5$WVV1oR(O4V4}r%guzmCPm)BEoOIuV%wk2HJ`dHY!_X6>& zV&Hq|q7cZCp61qX7C~-(a7?P}(dOgIvrM&psncATKDO^INpg`;Z|&PW$aa@(Y47b+ z@Q~Twvm<<5kbdp8BLX`2^=vNlZim)JJBjY<>;*v%TW+S*Z0cn1ybn2N8th3(xpwEC zJon7E5b)}gx=WvK%1{(Y>o|5W4W<IP6+#)%(2kU=Muv9ZSUc({%}4MD<XG}#fZ{n* z7~zsQ&fXiy?FA&2Aj&7=414D&`NUN$!mWuR-k(f^YExRp>MH$KcMx0qs-(xv>i9Lv z{|8*X?^J_sc=4u{fsZx~e=(~WLl|^{k$bZ@bgc%C<YM;}`Yy)0J6G)g%yV~sLi)=D zeh6K*14&A#`+ad+80}lt?-`9q)D^@eO+;k{trAzxC`yM$P9xXiN2jC($}a%b(!QEe zxF(<rLh(_&gk2x!<r7%OfjjQr3fyj2fZ4KQsM4WNnw6TdfpTx_M^rpJ6>vx7j~j{+ z;g+w=cs3IM<XO2wofoX$hB{M{<);d=g?yq;5ymLl1LGa*7m`Zd?+0@e_P2|9Xga)w zxMvZ8l;A2te)llD9!EY;f4;EHta6|zrm5`%5jZ9BDW#B@qnZv@JzV5nBujMPDh$n0 zEy5`~KF;39s=?-A@T^qNf-B(Tx5WI=4@91xe#R?UGl#l-ocvk#$PGbwJ;E;Cenp;M z4s^p2dvJZ1FHnr&PcRL1hPuEI)5z4l`+V4>Dc1GeKJk?p_b=OdxGxB9g4{+jStHR) zRXpBK6RR-76pBz9YvsF*nlS*DjHzfSjJ4*VstpxJ;HiQf7ISTD?0+$xafa;fy;Mv( z>|q(f89`JFWR4uAj^s>Ma?|f?A6Yv&?V64oSs7H*7M-IQMK?H7f&_F5fEIL^GIr;j zk#q9=-*KK;>P_lnj*fwbQ4y~wA!^E`T|>S2O8w2YfrEaa$lN8yxXE3#<Zk6%zhs7? zWJ9LBJog~D1IDUC>jA%RSfno%&w*{6tt8WwNo?nb63TRWz~QV13ZJ60(3y7r4UFup z5r=E7U8Z#+n}=|aO<YJHQ<w?iF+sUsI~6F~<G4az;A%IB>7rd%Sn#p(37$pm(OcR3 zOFr?23FdO1G~q+QvJV&aJeq8hzV?KJc<`zT_it7yfGv3Fj-^Di+@{$e6ZuL4+Ns>g z3qBZWj?bad5r!P!>LYt9p-quHYs!L&v>usaH@+#&n)U-E6C0VV<|XUatk0K~H$%@) zk5V>B?{J13xS*W~WLkls)2u-#iQZ9=)0)U5r(&e&3jRa0E}TK4V_&m;{rc=!g~wAK zeV?)*g$H|tI8^=KiMy#J#&jy=zrT?ffeeWhYEKpIP@dI2MX)DCBtTs11sBC}16dM% zeY{?=UgJm;^ZoPmoK9z2`6d%?imE_+RH;F=M#5ROZB|VE5`@4;8UyLd07p7Y{MpAo znym;Q=|l*z!|#q}Y}_n<G&}cRi)%O7MMs3YDgG=C6oh?qIm}hRRD5wlJQTw8VfS9U zZsJ}71rMj!;{pgP$!ARhDXcr@eP1=%^ocF~T8!Tp)_t*dtkg3BrRrmi*)P@#ds4GE zf6Q7umO!25ZlGsX{0Lwv27e?LHQriM=v*#LDyN+AKx$Z~%|XY+qbzr4vC#eserN`o zNf-_ttRdZqdIUXhq*~my>Khzu>`u#(9ir0Pz3u3zz`UVP`g->)38Ch?^UeqFm*HwV zg!+Qmc0XMtIXMPiM9HCCQBRk1>gUM)ts4<2W7I%FP%7GQ|FX=8E9{T$(&l|RSb0sJ zyTVSbL5Z822)AZL;tp1zCa+H)yPIeJ3#8cGCt5t#IHd92E@5jH!TXbU)~=699_E9M znR17fcrROv7UDkT6yU#2+rb+;C2^{?$*D8s1nKuZyHcooUQE-&!pD~Yc*!G45u+(t z%0@h&1>$h-4DSRxU6ySjytvNu=9tN~cuZ}WHw#rKWor`~C-p-b+X?P_CUE6^koi6^ zO*V!X9DRA5NHJ6?E<RAO3E7PxfTxdr9ekS+vfZhuW6b+KIcf8Mb?j`3LN5HM0c8Kz zJ=h(k@XNJY<i^~K?tJh}U*9o>6K-+~CR|_-$~a|>q~4*XjjE#H1gL+RaU?up4?Uty z2$ah3hu#)&N>AP&+!lyQLytG=Tz@t#Mw?R>P0lX(pzsaI{Cdcsq?1FEgJ19=tqQFg zD>Fx@GsR4I$(lALODgFntF>hM@$d0u!vluTOunhs+zFn+8FxSM*5ACK2ciPT#OMTr z6kMsynOAIHGeu2I-f7N;r#$2R)JJWc3O{RVyffl@JgR+S^J|#u^}c$qzd;FJUjcjN ziHq$o!Ru_7qVOE99Kk_Iw;7)%8$Naj<K^8ey>HrqGQor#@=~$A7si<fy<c&e<mv`3 zWj+(1k?DCZ9oGxmOjY10_a5Nby!BGgl1Mvo^>^zS8vNXdmKr`{d!$OrK}}>Sxw8Zq zojRA~ZyIN6`yC1etz;S6Ky-O%U5;<_c;b9o7M0?jQVat7SsFZ<Mw|?6q3e3q*5eai zpli2{_|HD<?9r`6OejMp1nwm<o>_6G|7^U~`-!)D#Q|gZdJK}>Wx()e^eIcn#*@Z# zyslhEtl#P&Xj0r9U8EdV4m=`qKOTI9oYV7Ty#E;vIbiF?%-n&p+Nav8#bj$$OP)d} zk`2Qs$z0~C@Y7W_EhNbKrnaU49HTfjOrlKR9^45jR%4DjDx4hiK#+UZdVcNg_JL-% zt9R<#&d~Zu4Z$;Th6zB4|24FFv(xUYHZA1TP#EHs09qpi8KRtB)pgysj5Y6>91Y02 z3ep~^`dGZJH|PM(jv2SC<3kuilpbc2kSJ{;)wqK6^;f=(uha7iSN2}qOE$A8=Svz7 zM~uRooi+MLm`<a7X@@8+W<I{%<$Ws9UolnYS5r)Um@;nQ@&kpptE!BRDGT1KVsp*= zRRKP0VFKv#PLG6>e5|I&#(0-gw&03X7i&P}Z-qhwz;(sN$9auipW&1EW~q*;`poWZ zt49;czL&lKaHVn5CdDH}ft6m^r=(Qe?E2gcZl4)t6nJMj*X{Ejs<lRPxdqqIi8u+x zrJTUZ5O%1I#6W~EN^&_M+3bLqY3!qKAe})JF~)zf_5Re-*sEnHPLu_z_`bq>xA%RG z6>ZUOUvNEb1`JwC1YS3GCEz)4#r6*I)>DSDxH^M}SO$Bjt4}tule4I5`8a_yBGXUn z3Yk5i!6@Ylj-Egr$aDBxztU?tMpdx5KAGyP<>royl=L=UN=ea8zUA@74k4yWdm{6< zSq*h*p7t8%K+6N{)0=11*f+sl-*E)JXHds{RDTW}DdVOvjmyFMVFM9TfgQHNW@D|# zzdKARNk0GG9Ft&Ansa_!aDh;cqIAPpuMQcg8y#4`wqbo6BUm92^Anm3xR#4zX=1uy z07Y|@h>|&A9FzdF_S?!-_8xR=bV&V&&%n>XSYr<M2zBGDm@+FSQ{5N3d#gFpM8{nN zSm<-7z1y$Zu3Bhv@NRyGfod4Mh56H^nb>=s8>|@^1tww(WjVg9W{;zzj>b3Ocz1mc z^?`JAo-gLFd**&C+%j<j_?t3sEg{@yhKo~~*Pe?8Q@gxPl}DteKm%!$_b$$gjh0GA ze%?$R2l)h;3+Qq=XK{|zRJE@%A!|*+--<_~WObR_%btKN%1MnfUyh&D{1hOr7MNp? z%@gc{B{IyJz;%jzsafWDy2IaYCTg>E-d?g*rvtbR(j#(40Xb5VYz7$Tq#aFl35~YQ z|6*#}??!berTbITKC;DrklD<mSbEE~k~aD=ywk$fV;@cCHz)C*ti#vSQye8H%d}Uj zpF?X|B#d+}eUUQgexugFpO&c{m!?ilN7$x4jd9$g4`A?4Lv5>)jCD*#1v@rZO(Vj8 zN;vpG*!4;f$ZNr@X@zB2%~OC9fBc{c&SDV=PPw#U#cmp~2!5b)ixek&$8V8G`)~9* zhomAuV`^$3bfm1BYd+Cq(vbWPHk(QTTq2z%2U>Az4W7Y1Ls;Md?HNEXU;4JuISwFb zmINfF)T{LV?o8BiBV}q`dknMBDqD;~;6-L0C~tmOp80xitg*7eGf*CJl-Lwywr5IQ zGtOJ5>z1tUt}UFabbK{(oPX{qrit?jTG7XpXtQc0IqtFrkL8{9`wl!cgsFnxny0Cr z3YC_gkl&|pE6aoGq#aYmP|8rVof?rk1bjBFc5R1b5ZAv6b%@&AFm#BUp_}`tiQt2( zYO|l3TKbfptARX~HOJe?V5!6VTxYH!P{AwkCzcNQd#Yp$RT7#kc~A|&8i(P_&9TS@ z?oAEn(qYma$D))?iceV?vdgZva-vfkq;Qhd@Yun-7f?3I0-h+5%ScXM`;x6W?~>g( zRR#^H6YCj(1V7E|JtsjxJ}4I@UtY@D`win4+)9x~>~Ty^g~rh^T3(vqBYAlYS%U~z zwh_tIQA0_(;GTJ$Igu3{Tzy~gJDdH^`;MAL^9o5b6X&0`DPR>q^rsEhg>aM?EESAR z!~R|C7~eQ|Fm}!#8q~DXWZ#$>G{=vBR8aL1mG(~iULeqMYZ6W@4>f|#V-@mxCJz-x zn7LYJeGalX>u1p5kSFJfr|k3>2=@XFiB+6AQ}KeIhEEge@>rtweyBxW9+2;mdn;MJ zcKpHrC-0(N$`HZi{Q#~5To4gDV$*m_dt>OfZHYg}|DzgcF-{fiw=oz9?@BT0WcVZ) z>zW_NKbU*|&8Hat+hGaJ@u12m%RR+wTab>{jV*`KtZaey4qX==5AAze^{e}0ITtLV zzT%PQ-KBw2KjL+OBqQgvC2C_n)rR1j@8~($1zWuMgr-m3SJRhhN=GcFCKKlX3K4(u z@R>U9_idt-t7KJ!Nf2kMNmmEyUHe)}cQ%NFdUiIeCd<@n{6st0i@j^*<IT0>Nbsg1 zXW+vp+~*Qr8C5m0aW9lzF!=!IvCoU^AuNBRw{??&*E+A3vd&yx;Jl&#%H44@9Z#h0 z$&a2_j-q|C7H57-TMn-sY+K-!(R`2Hew+q^Z=y}7vu#X}iVTYznWSm#-v4=g(bl#T zXOqQQx}39qp9}MVRh_m_&C?!yo2qPIbA^G!dZ?o3mOi>A{0VCs>@QCsfN>^zM_jwl zX3kU`X?(HUP>d@&^CuQ0ap}Ft6}~j_Fkz<rUFG09eQKMiYu>VhI$G^TMsgDgUO}JC zT`oUn)@%(43R0l<*wc*&|8B$MxqC&g{G;1ice`A6s`^c4?RV(6qwjsLI=*8eW~U5| z-Q6sa0vpJ%%zl&%OELu>9bbrG7?U0RL=L?yb|fhr{v%{njn)@dR`0QS_bl;Vef;4t zL<z{MjRV5u*K^}@+oVe!KLhmiyedw(IxzjMaNKk6o$l~`1@>d-kwD2)MHJu4)2@%- zHQPf>dvpbOzJpP$qj!a*>F|CRS0lBE(g5=gqJSfwm)!{#{fito5(t+Tq^FHLSWGn> z7-x3w43C=Wrju$6)XDQsiD8NFJq&fo_<lq9;|=_DFe%egdtv|au%Hm?PS~}ZlihhS zf$;9t#hO=Iv{6T@XIAaUm`#2B7+3gDS5=tGshIj8k;d_<($#k<y9XLQnf7o+5{LGq zX1QOSZ7Yr8GY81a4mV}Z@j_q6TW4BjS0y%GhbB@qa*}KbCUl4kr?RC1uv4chSRXC0 z2%1~(WC**Gx|emj;x7ektAENB>MCKP`6&l|!?7|#v4+Lk%0f;XwN%!&u;4fQ7<%>` zjAY^u-nw2Fh;0dNO9Z%jKZ$jFj<q3l>ho>xoTaBIj__}O=xKmp?Zv7UUqnC^QDTjT z{orcsvM_kSgi*EW?%Z*5TTHv~ca5DY0b6X?UhR;_O2Q85u-F#rE3#}^;?IvPu!5^g z!CsAN$FC{=yTjrZ`yZ7`aIt?r>aHNSppS`8`#^@yA24Li=J)%1J%@X<+?da63+Ak# z?g}!uf?a!dfZ6BoLS^N%Pw7h`2T*D+#u|&lUk#Gkbf$ESN%T#y{CH$fDW~yCptcWe zp7pes_ho9b*|L1ogO3MNlZ6d#Ln}met;a4+#s=6|G-~PrQdj9DrEWIvW*VJ|0A}cg zfKuoXuuxdlmhsu@=@$hHD+|X@6{P1sCCw-My$SEWl$B3T>x0^`5jXqS-<#r<!o0ih zH7~GVtD8q9-;W%fLpNMtM;^{sQD0oQ9k<00LgF%c-QsT)L^C#m&)&}Xi6{AK#chuG z=fD1Ot(9~1nWtBE#Zv67pOBG{E%|vj>=413<9xJ5)b~U3LvEG+utOx;97$H3?3K(p z5CfzsgR(y@Y7We9o7y6g6F&t{lUl^peH=Z_@zY`HxY7V@xJQqweO%XOb3tTc{=KHT zW{583JR9(+=b`eqeobi1N0EdQ55C|e+F|k)+S61+Q14rfAwN+@+J+Q2K!AAP-+RCz z(1T*W8j^;dZjTs^-uMWOI$)(jAE>66BpH>NRwxOc3BEa5%_`uW()q(lIWFhU<|YGG z0m!R%k&7H_w)F$a{JYcryH=bLZAwE@A%p(mW6|)2eBs@zqZagNAPok9R~EauOYdSF z?fh{1!OeC~b+ww28I^M=kE3V`K8w+e?qN<zA@O^))Axi+AgARrH-l|TWW#dExHmVG z+pl^p7u2-eMbwJ(kV6=ZviCjUQV~W~spV9zWv01N?wz*nu;prjnn#a)585&Y{0cN@ zq9dF~?6=gGZ+5^kr4MpT!mz&%0<-=o0Qyy)xa1N-9o`$g;*i=(_Es3V`oAt&Ck&Q< z3UTu&({H}J#di5x>c*Qv=<}j`wH&b^|C!)FD*S5bFA0tQN(`TJO5U{aEnn3+x~M=c z9i#WA{zhc{4{eX5O}jf0Y}YtbQG1^i7w<f*kfS?K`wV_~0nU=H?YHY_@ak;F<@Xie z2L3t1@u==J?3AH7%+Axvd(dq8eFR(}Vecw&S-D~VN-&;{Rzc{N(W=lPk@!GCC@sLC zYLPA~$oOtgPyGIPY|~FB0pvic1L!GnSgB)))q<4P9Y2Ftj!Ke5u$0;+KS}@x@?@6r zdaR2x#lB6nn8SPoB4zhyEf7jmbRh6do=-uUepvG|GE7$uq<F5gD+C$V-Q3*08Vh!q z`e3V`ONNTikHd8&zvV3^`QSh1+c)Gtnsu1cxe<PmC!=5bih4Z|iW+Wwt>Ga|Jf_n| zjyR}WA!RbslMGA7S_^io_8MwbzYjwnS=qy4k;58;+(`tz|0QVQl^eTt;Y*4+dr}S! zc=3k7ORF74UtUsK{iC|aqv<v3#-V3UoXx+_JFadb^npMS=GVzRpd`mN^wbjmm8h_} z#*>ZHjW>tl(pn#rO#>XS#f*w5YaYt4f>|53Z~oFyrZEn=xmXyoq{<+2=9K(b?J9Dw z&{#?xk+y~{cFtgCd-Fhz@&}t1OiKlV1hB3C9oS9LDYr6Ncmsj6H|A-^$Tl7Ab-aW@ ziB_4f!HY!*(|}%$xC!@ZUGq*V>*A|4p*}s~)6_PC-!O$_d0F99fF~^oiI~wvkC;8A zvOP>6F?%eHt*Z?}*c|F|Oq7w61>N-3S=ipBq6I}N{4ixXn#er=8Rq*&#*J-@f}28% z8fK`o>T3LaUVP$+?;Pd3bhmTJwRMxK+tN4>D*_)BG@CSzsW9d8D2}yi{#XB!@+F?1 zKeA*`YEc@Idj__arI)gu=A>7Ck>W$}%_uKNE25NC3;-Gmp*$z_^Z5}1hrB&Rq;LxF z4nq*CpbtE3H}&Xia&JMrA%W9?{jrQX)-D9lOB;6wlU*1JI(dI04zQ+x+=Iv+DGaxd z;NjbCx?x1fH2LBasF;j8>9IM+!0cZ1moFtX&4kfZPVrU#FI`uYHe+4|FMM1qd3mrt z>F+HSphES8m{uFQMkAXjb66;l9iA*1tl1;qX;Y10p00pH!%m0fQB{Bl@}90|gV=1Q z2;bw-rKW9|VuIY({g6^05pJ5`j4N#KemQTH1^v%lQ>j#4x1OAEFL*mN_}+biIm^xa z-(@l{Kjp<Bj^gK}h0bK~9ll}ytib*69V%YA(SnJr&p>sm13_y@NA)1K2gnb3?$>B_ zqh^z^$EJi*V!NDui|{xaw6yX}4q#LzG`jg(9@WHy8o95|&NurW%|_=zCF_B+ho(G& zVEd9!gPg_W8U`Xhjw<<-nsgYt6+g;PNltz{oLUv9zXpRG&K`I+h4uRVxNYJPcK1dj zT}L_dxo~J2B+i!h&-~J1vo5Rf<t?)zwwle%wvBO(B}IbEk3G?1S&tc*W)_Ox)B4{B zr!1p6bpNYn1;!Z&Gh{L(ZZbt_bmdb9P8lV_S{Rdo{@uvKO63Bxp70Ukwr?q6V|3-T zr*!bu`<$lulH%$&-S;WdJ^$6Xr&L+IZ;~W~oNqrK^3xv>GlkZ&)_a5$q8o-}DXH5A zCFt9o7FY`kS9W^#>$ZMazkdmK%Gjo-*SICmYd&~2-@z{8jEL*2uN36Y`j?X(fEi3S zJlvmu2I?3P)0QJB)`kwNtWWVy&HdP`>~<@pL`_nwtDA9-A<ccScPRenskqy*P@dP7 z|9*H6|6LybXGFGCSSLcVq52G^E90*9j`qvol}B&v0{DnA4Eys@17~EMwl+xQ^Y=~_ zTDN5NyPm@anWRl*IE5Rc4s975F<dc8xTXJEbMFWHhBWvOlW?_M|4KLMGdcDh1rxnN zUHSp-G6bG8Gg8-^W$ectNp3wfI3&7ywG()fDD0EizIHt=QidTPv}7#US*SxKDa<~w z^G<j_&}z+yf)CIIjr5p=bC2bM1k;J}i;3X!=AQq2Y{O+a%Eo8$;eTm$-@9wr)y;Z< z=0%3Ddd~6aN`(Vf{BGk6H)a*l9&42v*A-6>{(fnwsp$3@pAPUkX%+|ac;DVI+?Z2D z*Q_1I&rRJn>2laXKPa%!#3lIzwVS?IT`b=}fZjEY;BFr7Fm1m*s1)i;tK7Jma3S!1 zkC*#V(W0jyHBLd08+}j{k|cbe@&K;(?^RSzda_1<gw);=u29@&0$pT3+<U&{WTJuD z>~oy)B=4|?kxMf1W(pc$@K{K6(nQqATj(lfVk4OE^_bo$X1gcSbpleIQsTe8I&H@l z|8A%--q4Gh7S5NruS0LF*IXw*>EE=E)B?R0y1ABHvt<WHpDcxR0@)&uvC2K6WXocW z-TD-W_iAR>%)?nwt=fA<CE~9Nt){clSS^KQ%i*-vG<5^v9?}9%>xjCTE%ZUnq+-vb z^9N5Un#lhYN5fp<|63*GWbqbcpvzCLy?RS-ZLd&x+>@I~m*{i``M|YZ-=mYMFc4(g zCN|k4ky$pxRf9JEtzr~6+_!Xl5Irzm&P!!QfYB0HOaQe<5~XRNiM@_>)Xglr@s*uf z=yYYQz}8{cO{4M67glS3g~xzdlLx$u*$tIlfv1LZA1?>|67Y0R+}ZZr$;{V!xP!_) zkfrxRU9zZ^e-Y80+k6b)8Q%JQIM3;NUU}qu{l{5QSo-OjZG?*WaAN3#+yl-+-)}YT zW}zm*!^ZDTmn*>AF61m%D5&7+;ek-F;n93{L?ijswRbEZRK6HIZ6E#c^HwRa1D9$K zI9)YL3Aq!sj~5b5i)Z*e>8TheNt;pMOTI4t=7Ro%+5Wdq1{(Z)e2__BB~o+7i2r*E zF2=1-<NbD;N1Tj0W)smkKjue=r*Y=e8?5R72MLjOM;Apw*SLF){EVNrzbrHtvfI-8 zBLB<Gd@VI@^Yz2b9N*OB?ktbQI(_`VczuC@m7*sdylg4>Rp5ZFeWP%f(B;|0)3P$) z8t)U6i>1%`b{1QhuHUl>slrS5)!p7<r}&`v#%lln^2U|7oX2u<lB$hS-uX`zS1_aO z8p4SRW3TGJR?HX(+`sVY%rN7GJX1s4mj_uhacsxp&i5@aOvs8vY3N>paH0LxO}oK# zgsel<Q|bpxt~o{m6FetNuv&wkHNS16at1EbE+MP6hjnZ1KWIjM*v~waFgd^G-5CBA zLiJE>zWfT~K)>+0QYbBy&Vep;rTVC8%$ME;Av$ivQgiPK%|J)g@N8mD|I;0>{PyE0 zl9`unkl=@b*uM#achk|qq_cn4Y7H){lxB~TaAKjfQU?t5IUxdXq5d*cS1VY2x`vtR z>CtS_VZJGnT94I%t10Da>UU;#rKczlwFR*+c|L0a+a_OIkY-Ad8AYXX%5Atk7l}UK zSLml7?ZD8Zi0SzXX}RNfps`P{&&rqS))dtfOEvqA76MC!BIGRB<h@tg17<RP4vJ>x z&>w0if~5l4|8}fD%uzci3$1!sLGgOcuUuUJV-je+@5C#c%+(i(4&_f9Bm8LMfq;gL z3lT0`Gl;%lg?papGpKy(9`eo)isIC1f05+Pk{5gJZ*1e~zK_z&AL7L*ziq(^{Cp&S zJf9pVMq5%CYkUWD-7)`Q;GESV?pAj}IM{O0*wi2oPq-|9aeg+WlCQU4Dagl4aB)l* zobNECzqxPl)l>UwD)V|{2Fx{oO~SjMV{=74#t}hvI<})S*ZbMdxxC@vkH!=IZ&nfU z3#8lXE@D_PrG>ufFpU1U&{3o%Tw?~yzW;smW2)f5dvjH?_0}sKgCtcit?_Ie%e?jP z?v`~r_AMJ$jgk2SYNM#`rDXHEM>?!*>_Q?p(=TLvcn0-atM-Fz+_I7}dAP9*I6p>O z(Y(deH}MfPcWv)ReFxqBILLFRqZKLLyIl5r>R#|@{OivbJ~_3XeZ1_#h-~xiLwGiE zZ?Rr0n^&E&TM0_4ikhL0PrjN@6Qv0G;6SQAd{+=x^pCwH{Hj10mucFUtY1QZw3^Fe zkZpq!U8J2GnPSieKD|;!V9|Z3{*d+h81bIV1YI6oW#feIs5~ZG0fW{47YAVUp7)>} z@^xzcIuc8d_<P&d$D}KVPZ)#TfsSSARJLnT@CJ$YirriI_*y@pUrZc&OsKLM<6z3N z{tFrj{t#_FW&dzK^lGV#Hju`CeR|CZQ*K+zw#cA2e;~AL$^se`h{*h|M7p9~?Hn{Y zTpn(6M8WuAUK!rx2%l7#&u+?e72hh~IMI`uM<X5pHwY+gZLoQVSNlG3nnEE1dcT8J z7L0Rc&>qiD$R;3+fV##P(d2wnw{XfG6<z5_`jCZaMCG9GZI>GfNd!tP;rH_i=H*)_ zT*8|p4TJX<@x#k>C){rR)&DR<aEBfWjY)$ao#aJ|4JZBbIDMI~G^(kZC@+@e7^o`f zd5}Q5iYyNs;UTHw<Fzz#2eYIHV;rXtj{=mYw3$lr#DWgGfYBgg(916Mtu!Hp#G7qq zeXw%9+k;n`10TGSp?_B5rS+3##&e!ii<xfD*e^fmuGoGN8YfaG(vyE)Zl#v3B^_pw zFEOtOtwNb(a8`MuKg;PO++3Wu(!;&u4P1c@mJt*a!@oM#nkZB2Nr~}&3akH*_UV;- zvA;B2lv|pawzjiBd4D(Gh`KAvJk*O{T<G-y1zHuJYO&;gbf~U3tlX1nc;UW-Ge`Or z-a=7|VE3n={zpa|3G>lsPBZ>KQ8RJ!y)Q6HPL01qFsa(=!t1`*kFI_km_G(5RMk?r zc=T+g-UsRPY|o=UMfED?-i{df*nYJ@|1)DQQ}+7{m(~UIDEuzvD}3pWvvU`Whq;e` zT{Py~|HjOaq1?TfB+cN83K(7y5T*N?_BD`imTIpdr_1}7ISWejXyI5>66BvbORs(0 zyT@Y_<PE;(5bQI)M-JJZSM^^R$a?Oj#<+KRd@n(LyA}9v6VE%SI$x=$)vohm3{IZ= zZqyVl{V9J4WWpeG!V@oIJmNQVK^9@TS-luHdY@Vk?0Juw$_GPlpVm*wHB%$Qt}j+H z_3K(5Fdz_654TCNa0y&+ZTR9X{jVYXOkQC+X9B;8^qSKr^8)y<O2Ar$sIO#O?Wvvv zVbE^UY8fXLkP5m?t4ds$FxS>Z3I{Vj7LMI)COoJ*DENgFIJ4Y2aXfWirmxugGT#i# z2vVO9e4;v?&~}zNe35L44#oG2(TjiW!S2yu{%~5=9Nd-mTXi#$8>p0o{CwqJ^%q5| z-FxQ9POdNqF_#|bF>qnvgVkXPeni(mts3digYq$5sWEF}6b0K;6fJFu>_DKw`RI98 zrDS)RcKY<OD=#5GiloNHX6i>_8>Xae7Tn95^=;I04XuRVJ#xUxK)L-IR&&bNl|})q zP0qt+t8FmP<pk<w_f40AYH2!enyt?UPv0mEI5hlJ$_+NJ2r9_QOf55^sPMiXtOoJe zwzhoQ`JdV7u5d^nI4{cGVzk1Q|4;K18!O6vg!d@kd+}OYkcL$}syXUQR4YhiNLM5# zyHFA&$gnQROq&c3TmQti;A%wm(mCo&vK;M?S>*_VGE2IVjENcsCeYwi7q(dyzS~Pr ze^2nEBz#ec`cGWa=DU~Xtk?eBL?-?=*=3sj0R54Ap_$Ku!IWy%2mlG%OwQnME4jk( z*dXp}3%==*D)kS1C0v8xtKI$>QR#5-S+~=<g|2wS?}<lB1KoGsI>+C3yYt)3#5AR| zW&+dd(v2cAYZCQbpp0$doQbH5=x%bAUQO4ElpmL_WmrO2!{Feys4lpa_1myAU%#09 zo!=?E!#j5M)fY7RCW4c{2qzq-7Y?u5bo3jVJ01l)wm?@T995CMtL$HLcpXog9Mu`9 zoHHpZX$IBh*A0_Wb*7f2585tc;?`^ZexaHaXJGV9Mshb&UwQqJs9c6mfI`)WMzz^q z&4yU8O!gezUo9cZWwTfRk7kDJ5Sm_G#P3=%tad`M@Kb(@AYfKqUH^-s^R3kUDnafo zm6he$rh7`$Y_B#=%UJ`r$D-9{_P#7zy1BEcr(1AgOd~WrE`w4!1)*Oe&)3AhUL1HH z2pM_yVT+^=!Nol{*j!H0)*?%FxUU)I&n)Jq0ihFj(0j^SqDg!%@<sgcGWH*4$)43d z#@a8u!bH44+(z_sMR$(M2qDQ#!~Y*!XC4k^-~Rs+A;qBVJCTGek#+1#LZwiaDPv!= zjWEWVCEJW8A(SP_mbH*UiLviHO=V|nV=yyj%x`+W&vSpj&+opEV-CmsahYqb>pJJN zoagI&zw3Jat?f^~{#cs(&MYAee!WNktJ`W6NEI~tF<%N=q;Kt<im3R6h8^z;u7}Mn z7#a$T?-$0&-QnLJsXTG&F99_(M4n%AP;R+jA{PtK_9Tt%NAxeHDe0vW)y;9ji!W+X zi?i+xs`+DFrO9vYRew=DFKgVB2g5aW(ZYv>oHoSqiDhFwgCRI?2giTX!%2`;IVi`( z_ThECRl$^`#qF_HDD<}4dMq>fLLFHlMe1AMwJxc}SH1=hH%5-|swzoEd*3!#RwI)h zV&0RL(?6=Y&Ck&FK1odi9+c$&*%|za{>3|wz32YMh5T6^Fbtl1c5eHeX0>HGCm=bh zjli#nymCiM`rd6oCO=(8=H;io$X?yy`EUwZ^l7;|vRr2Ag;0&3ZagFIL!Y6L&GlJk zo@J&tqKN?W2*9OlEQ}HT!QE;<<VJ<`08YKS`R4|i<fYV3#2;6By6M_8@QCVQR)jz) za1%J!uispzEwZezjZAa3jLXl|)0Ib=J}}L?m-@D&X^4<Ew>Tk{QJOuGk5a<|g$U$y zhnb-Ev)XewwGB9FRv2)bvsYIYbtb?(9Q7oeso8_wPfR-$O+zwLAQsV6#v+>m@Tg@K zfvG`#rJ*olka?>obTu!CdB)FHTm>FB8-GVveS|ew$F@Z<Ep7nge2&0ch(HV920Vb2 zrZKU`=v}P%Wb@$7!-pGpmF%Rf)MgS|r%kiF9N#bnPjsw64K-#ib&F$C>j9lBXP~)A zyS}}QJ;^|ktM|eoPFHRCJ-bt&QJMtWyy5Ia@wNc$$FfFjDqH|{krtRY34ZR<N$>JL zeGneIF>EO&xZu`34_xuX7pjxs(7gdTN#!gD;alv<%Bs;zl6XPmJhlA`to+)}es=II zhKsz)@F?bGXh5O8+Vc`$mNIqr8DSp8RzRzs;=2WIwQ;JQwiiTkvm7NnHP(&4nVlr8 z@`ELz;(VB~YU6eBYl!3r6m3&_>?TMuH;HsU`@Q$#d8#oKwMUIoLq|vo<}RAnM<2)f z;A7<k+@k%8U3kg4G8x}xx~zk3bvL+o@DK9)P9N9C`R1TSRTW~_q*S&Qm5SLH84C(+ zh{9PeQIurv6!VE#KKe3yLZ-A&il3N2w#@utmr&T(&+V~}@{=B%=sRblM(esMvy02o zU*<(xYwmJyOmTq21Fj~+T%aQ5#6LNs)a6gs1Rj+Qxje`5(zQ8-t+#OkJg`lI;S^FK z#ijo)vyJV=T2_2W{?x$_eOlhGtj*T{6Yu1Se~kB6osOSg{`?T(ahXS0?F;P>I(ND_ zavoYAY~n_nN$1P|{HnFg#}`mk7;W6b<EZ+oIbm@jsrUUu2i&Kfw8KxMF-QvuHpBJ> zX6{JePJ>a=7TRXI=XJbB00Bho#aqJ}d_*>G)I0T<Gi)6)hFFB@_1EWVCYMH2StA*$ z`B*n{Am{LLk8?Y^moI(3bjL7>u~*b$Ee%HdeVsQ!kv(~o>E2jfgsMZonNs9aYQbLC z6s3IdSw<^mP5Yx;xmF7l_dH4X)6d2T<$F=cH7>XIcL6#ty9GiqpUF{6j=u9(e~(gn z3vIfC(`+8%%wb-J=SRvGoVO7OjhWZ#xIux@@+|KRYER5)I?IznW-sjuI2w(*vo1Bl z#p}q4X#7}HPrBSqatEOpRl!WKt1Q5=6+UvJQ0VUjiL=7I4(d<Q_}ggVhYgV|wRxHy zE}6#2Y}fl7<b8TKjJQ5@J#piOPfs^Je08ZjOwn&Q66eNndZ2WggS(Tmt{Y6!3(mZr z>-DWWnFYIV7mUxw&P2%&%KU1J?&iXa1GIdI68EH9zyvRpKJjksNzCwD!7XK_D^}UP z`Qf1$u|fo<sXC7oV_x@M73T#***i9c9Bk|=FA0##b9bsN%jL`v-CUx{u-o>{i(|xY zC5W|%PVu4?rvGHAPsY~qqJ{@&+DU%sHn-cjsSEU#U45b%!vD`!IW_3n)xMkWc7*qI z3%h;R;$gT@qL%oee6zWx*!z=?ppv0ND?+@QEI&S%4@dQ6R&a`ezwEX)4Z?U|`)%WS z>xh`II{hek$@0gSK5GH_wYJ`@NnbQFnQoS=kn8P`)p!r3{w~n!>;g?oT^_EBb4wYv z%L&QKA53zCCSiJi^K!>Ihb_B4ZItuo;;j2t@MV@?V>Rf-L|>2x+T%Zq7oIGpzhylC zRB33`7+#66(b`d;;aBaCHas2P+IW>`{33Th+Q4nvAD(*n(+Y0SbCOv9aZy)QjCQow z2hZ4ij^xt)fMyLV<Uot2TcGo$E2i_EkZqUQCAFVnIZuE~3X3YSCKJ<t)@y0%&=Vr# z8TQ(NJy~cD_A4A(FAcTS#nO?|LS~jN?>^YOxW=_ZQn)U54tnbyh$=W2A}(NF^bAS` z3X2tA<~_zcPVq-&%+Goo0v22h;SfnHIIju~gMiXSoHPAFpW)EY5R{6l!L@T4W37jq zZ2`AF%?`bbS}~$I$}AH&9Tvk*Bdu3di@2pQ@}A#>`fi#jSlAp^WcJhkBbxbcAT^`g z^IpAW9VL^egfQga<}RJsd}ehb1Z0aOvz`qo;+ql(UP9cJ1(qu@>wFM$oqp7Ez!;)4 z-kbPZw@X+Wkc~P|G!W_OV`24%Sfo8{@=RdRo!$F%>)i9(<186xM(gr6N2RR7h-_-R zvhr&t0ctzLFT8LM)tCrW%ozmpx3F0|_t0RHs+pO7DB8C4MENkopuov0Oy@XLur?<t z`0c5YUh$dBoCGsbo|385HKvhb!SY;N68dIzr@=K<zwv1ub1GT~c^K>Z<#@X#u5Bb& z&QPUl>~6yu^yK#RX~52eGZw3{HQt7=uIWg;)OEAjiauA>;Zeb;^B)lfW-!vt_GywY z18D0`Bg9Bz!1j~oZ5i@yO3D7bIS<m6gt9g+t%?rmS<7yZ#eMBw&)pdW%K(4L67kd> zIygO)G~LHb%&)OLo+zru3LoArJ#^<RK-GPtbQ#t10`Y8=5lg*%)rJvfv%q~7Sq?Y? zxMAa2p2R1`|2x~8kaEatb70zN)p%lVrLfNL_&Pq^IE%j=QtbRm0~j!1t@lYqv*~}< zAb%=2X4cC8n51N4?3JZ9F`8;!g40HQp{b7QN0sxLxXX(rKW@`1Z<k?X&c3|1srmKR zb*}6y0Gao)?VCY-i1Qm9kLinZA8Qmyq^i^O($vv>j~5M>(me$dvc97n32iaCI`TfR zDqZ)E4KM2KC?iQc??l`*<!WXe3?RdJZZiuKD_n%Sxv3B$8a`_#k<x3yJcEdwjj9Jg zC!J3}U2o{w(H;$?nVr#R<fdQJRusuVEeaIsv}?o&6>8aOeTpKZ=;Rry((h`4xQ&W2 zYrgX(5p9RF!eW9BH`zf9#yWA>+G+@vDUbnEX96pb)dMe#ppaAm8txi>IF@}}gHW6e zWE**pforDN4fs~#zB>!UinVVFAqYiSzxj6oHOd9cL8x(f`04E0=m){u>CerkUkk1? z8C_VW8e%Mn%EF~_98s}$>bg271WR#;vv-SzCN(Q;14kba+0zsp!%6o*?>r7fHsC+@ zM`|P$^h9|*=+QcFgpq)VGWLgGV--eT)kFe?u!u0?>0>^A{tJFA+(j%-^z|Y0Md6Ea znxEH$Fx0?pNKYoit^}_S1$w#Y%CCoFsgCBd8sfVpJv+l9NySI~g(fget_;2r1_~<L zlWb1rcVG&05icFr6|dd9fORaD9KG9u7p0U~fcX~2f>SC=fMF4sr*mGPE(i)&V-Ezw ztaE~sMCwjl&lkHC&&{VKJ*o1LZNkdy6EBU<r|8HJ$Au)R%@&e_V_ijS=fV!$WC``! zb2!GZ)Oq6BT=&uwxUcDg=UV|yj>5ZP44@&ttv(YOQUVMoc&Hy$*CdY@pPBhF9e@ry zx=^dKc(q{YrimFjip5yMC#BvFYwRTg%9lM!5nSQgODdR!%UvVa{oj}srFKR9t;ER5 zBrN%oS_onlrfDJ#Q)nnO%`~M(r`Ksc5rQ-yM~$RDUj;~``c`JvBj@2}Dp5O-bCTw# z9_?~wi%q}I^I$ly9i;c1$aakb&ROvw^>M>vuciEiE`Wg0S}ypiRrv(ork$VBh0_F^ zHn5$IxvA?Q#7z61?qkW~HzTeNZ?%wmIl{pMr|8ED+;F*geZAbD4NwNlmcHWbAVrZw zKo}|ses+Z3k?#$ACwJ?xO4w%uL&r(`4o|VOU(X1?*KC;@&$jc%3LUQ9f1IBT;8hqg z0%0!7pRFw#Ap^>yNWDd1Wb|z+<BAtz5uuCB?NO}NeS9KT$QlXHyEjMm+8bB$de;Y_ z6$T)bp$}BW3aK)%k{5Da48bXp0UCzH6<33)9&m-B2)uwFz`3D4dL_L;iYfRW)lDqc zT%EX9R5TCZa-4#pN7DISiT7b@{@dNN+6IRuaS^wxOw5%6O^U9COqe017pNFHdFZoN zD)j2mjjb$a-om?-fgS!{6B#|#aM8Gvi2FqnOD}7p%s*ETUcTp&;F<*5h+a3`=wVi# zTF)rJ@&~TB*h7SZAd0++wM7W>L137;+7XVaB{12`@x32+qO%&x@Z~LLDXZ3ar4w^b z2yP_0c0sh-r^StsDGyz<Lz@G26^Y3PFaeQNzH$$;rA^6Q*;dz2#QJZq`ri5Tq6`Xf z1&a;@v;3LawGt-3{kNF>i5<ed6khRi&BP_qI6obRbaT+16uHiuGGSPu@#J5gh2*7W zQBpTP$yLwqYP335Sj=CnOjAdq%QJ9@d|j`orpG^t;xzw84Ge{#o8ugdbqV+qm+{dQ zs(2HwH#|vdPxoM#k)U(YgXMUFlk;E6wtrPSmvtsdR3<p?znI|vStU@BeIa)SwRAph z=xImVVxgL!4HT;N3bI63gpqWv_}uV0WJ|0y>uo2WBie1cQ9}LWnxEC8%e2oAF0Jdl zfor!vdNXGR?06&}v#ZO&AkV~AUdomn&oZ@@BAgo0da7kX1?v&LIg=@I^BO*P(VDJ? zuCgJ{(=ji#l2+{tz?|W-Q(rDxroLQX;54lHPU+(&Ayxi9`uJDWP3Lw%6>}$dHr6Hx z0#^xP5&q!qb)UXgn-C!Jlv*S>z`VnKdJ_aIlA8jDdWp9^27Zlz#&he5);?I+u=Nt+ zq3cC>ZWyEpWy9Zldd=&^56g0|-&3R8^1ojKsgY%)wyKFqp>?hx5~WeE(XQu5-rGBg zmW_W40vM3uw%2VRk`U@7-A7Icx{+?HwK!P>XwGk`_{)Z#5teBg0fL<r3J@6?b(F$x z<xan<bSXrHA=+?VsQ@X*bJ@4W#U7;8k9!}84!M8|Smj6RB!&gZxmyRqC+$^^dXAe} z!HCD=II230q@7UIeCn3&+OjfFiEhPkQzNzcS<SQPJgky`X*dg|lM0RSV!a!0<%IPL zgYp)#3Ok*DYeUSU&nP!v-|r19kOmY5n5Uwy$X<n;lcA@wJ6-$&t^%n7*{TB3aPuyD zB?*tr<R2gS)sNiObUsKjtz&%rYFm--cRX0K;mf$Unv3VT;m%zQAyX;Ugq&(_c|%UD z@B+LrxF!Nf0EH`v8=0gM-Mj1?CoF^gD>g7b+vQf2pZlIhmP2R!p1*eSX|^hWxm**o z^1LN6TP%Py8d|y^TjVekZQFHMb0xglRB$YH;#c_fvmR~86;+#1<eF8yL&$_TZJDMC zhio=H($lkY0rwR>F|bWk7rFH0gS62;kvAD2l6&5>h-EhjhU&&4@NbC+x*@NF7XMw< zq%MB@s9Cxo-M^*mBKc`=*KHOH<h|h1JI_dBy2`XGQbLQTdqZbcS9d`r=KSfUOrfgw zN-S*!-D6f3MvgiDn8&QP%n7VbwF*YVXhfzFV~6{}^u+T{){g^&#vO;*u6jm9vY{Vp zHC)?SFZm6!{j*lvPx(<+MzM(7;vJ4!u-ylMdc&&0mKnGnF49Pw1C_gY3^0uvi`t&O zwwrWEH<t0{SaPFK2yN+d-$)tJ2;!IaU~XNTJDzW=xEju&l4jlxYFe6Xgo7Cz9WBKx z>$YaJ-|(xrDi<m-M2zJLeLn6Dx^S9{zfTG-GrfbXVb+e&(zz$nQ>>rlXeb&*%iU2> zC|k_mDk7Z`htpCO;#hm`Lo6r|imzQX{lVwe8@~2}Qlqdi!a2jYZ1icS;W_+QT)@!k zx>B(ZLneT`YA=1wq!rWqgLsZ6X2+Gu??Uk7FXBG4@1a!1=!3QUkYC{&C1^hMi<?EM zaCJ{<!O^ZNE9`?&_eM85evyp&gn0&Dq)r>uE1^CkwQY^vaq(NjASP$5yBJKr#GED* zqFXEE9zk)fhDERS?Zl2EG>gndp&31#<Z!kTBMA(bO)cF=Wt*6GiPYNhZ{j2bv)_EB z=2sX|uyvCO(@zzDfTRvIL<$_J9io_5U${q?tmZ)^3j`4e=AOQx39pI~SlGYzZ}RVR zzc`M2Zw(*e4-)Nhg(H;pDEqoaR^NAdqmq=WmTjK4YAVWjJAA?r<%angipfF$e!c7x z)R}jpZ%IgYBW3Sk&DU*pfAadTo2mMZyltKyp6(FGWgipy!++6xUX$e#90g{2%}=4% zb?r4vT4&sD860@f&xOX0q-u5+5H#~P%ZXQJdUSjO%nZOeHP3<jrp?Rc1iN{6|ASEv zOgm)&J#C?~?FNQ?cCoEO{<WA;lwqm3Or<*n{)_G?P3TIb@31-Tar&{C7R|Az7(f7P zEW}N_Mc1Lg_`8Uv8Wl+`VWZ^2MJ4^k_}TRBBdwNOwvE(_Qy*j5vfHiTuX@?_EUSM? zvn6~XzWV6{`B8E{@~I>5Z>>Alut26m=d34DolzS!vo!c#>R??@QsIz`nUS*Jd%;>? za0bi37wO&kN~pmnP?)<e(daQ>M=XQCxe#9mRb_~9!Emb9<Sb+HJm0U4IZFJ6z`mj| zNBtOA7Ww<T8f^!q*G~H%`g>_M%RL=v<wNv%M`3Ax&F1i{pcIO?skB~4{pGePmru+Z zwJc%Z!!s&VFdTZcPFC5ZS#eYW*d3v#(skc%hWpajjMUMrFP^Lp*W0uu`SXNX@NbS0 zyvDaVnyq#=a@uaiwV!|cwR#4`O}Zv_GYosox7?dA{QURksJmKT-%vq@Ijm7cHgD<G zBXG+eoLQGP4J6<gkJSGJ<wy`gLp#@%gg#OCFjXo}&5h%$&hN93J!L03`DQpZugsKu zg2GH4dqxH6WfkBWW`{=wi;X-SWuXkz^s3y;tau=U;`&OTF>u)XB>W`q=Ay|`V&zP9 zWd?8UOW<%T5r0|D-5evHhb&2&xH0#+DhbnNtE%JAIV=}GVwhnH^GnkOt?~wJs8WW3 z4HltElC#~PT3sVi@|vT|8&b7Y3L)9A_?O2FW72ICyWTNPFDHwT2b}!a+{-Q0R;>}v z@e;mbK|k3}ZjgSw2;?AL9f;y2*Hy2cvNK6PDfZOtO~u@r@CFz9^^|V)n)0sca(79B z<+-PUi#*)x20T+q4|<gi#u8NblTb_|p5<+QF8w3UN~Kbo_fiLv?Qu3(tgfCrBrUey zh4aXa*xLP{6k3sQ@MCFeMfv~hC8Xwz)pw;%n`?VCmL1`1waSE~z6z@iIP=zer0lYD z8D6;})n@3FmHQ?Da;irba1DjjizxB#uy-o6`Dk^@noL+wIcNrEv5GK9o^m`i)g+f~ z|Gbx~DXx{r|Df0O)Xme5E#9;V>mpm`r$9eQW(CJ(5~}y|6=#J%OB-DnMc8pK<wg(U zM2F49oo=xQ!+n>i*5BKRGpH-!rFOk<6b?Mkx#srY(lw;nH9Dob!ZK3RRws4+O*-a9 zVr4NH>?Km_F{&q66B_U<zj<9Jc;q?7BG$(Zkb+EpKHAB$a1?|o<sVH0CorVh^X{v8 zV@EqYeFz9^BLsZ8MwuIb^Y@^cgr@otuN%HV9<_)({7B6PUlL@f!Pp3_`zKLFoGfEX z?R-d5!Hv%$`sOD;Aee#AT7%7@Qt>-Rr6jzAqZKEC;A>>C-1Tm~0LKdSOBJjcG4={f z-NJj~&SE}bmL7zAYRBT9*R!lg)9PQXk<dYzoMD(Gag?afwr_7FhFfM=(UbHG?o??a ziaZxY!Cg0r*O_-8s^&Ag_%$2)U`@bZ&J(0B1g}5T7zq{fV$eZDU*48j2a$VC3Kr}3 zx7Gdn!MmOXD*T`kB&n*hdCUg^fe3}{MVO7!&v-SqwDlCWa-+fM;vxkYLQc%>w_t!J z`P-{fTY2ZCu|ak82eU#>SXuv<Yo||~MTv0W;;nzz7^wB$`HzEc|5Wc4p9ZI!`uTJj zyRA^d0P+mi)f{mNIqzORxH~ol&Fy`^?tvT|9XvYF(0;0R4>OyckX3B16)s7EdX=RR z!%daXJb6t2#_k5qWEG=*`hZN$<r(*xOa3_}>Iak(4-<mftxuBX<ZwwlQw4POtG=@J z{h-!v^R16{3XCduc8-5^{kU{+;ubuP;7@}jJ*Yi$1{)u~Ae`KrHFX+qlvyYbd@wHm zvbTK7Usg;&XJ-4RVbkpT>YACX{+h)^S3^?R+n08I)kkI~3Y0cU)9?4136Jy#Kx0#( z5+&u7GimG{RV>c*VWxF4CABY~-pdK`DeEW9j*@e#?15!T|M%zQMeWA@`_D;cdL;h6 z@5OvOh2vSe0D+$^5H|PjQM0Bx8h)aep<1-xe5svMd<`k(;lSARg#cHpH2wz=#ff7V zIxy1C9p$A~Jll9oN1k1$GO3F3)LhVq1+QF)uhQUIXdXWvX&1kB{BNj;S_iV!`x|;p zC5oEb2QdS22>%)3fOw%6%}R|`ZrmUOR;OudwEnpVL5?&_Zp%$EQkN2X+2fnPbXb|h z7oJo>Zx|~;v<@t9|I+;mf|7mzSEe$#%a~f}SN>bsUl6}l$yVQX#p_|(ll4c7T~85P z3)6xQtCRcRb>{wxBG>tifBJ9Uzb@i`j_h(W{D({V(_!&n2g<1Z4W^_{w|_mAf6vUm zGylZSP#2+LMcwKDb#PqlKQ2cjWXVGm(hyUXGBal`@uE%iG3B_8a{S<_mj%)e?q_yY z_Q0pPEe2&@J*5a$@%qBOl@avPsVv2Au8~P;B1mJ)=Bh^)si7id%lS#V*Q(OEIwz}t zc*T*o(o0-WYa{xS+mRbNn9-yW$dnyjS5f}gio#aL+UK9r6RRw+P9X50%)FAa3_5Wl zqbX#xtWlkGmGV<4x35BA!+nT&Io0$ny!PZPC-URTYWT_O0qA5IR7Kfkp*)3BCP3tg z&o671Lyn7~L`Bf-lKJRv(;pIUlY&pB+{|2hnSc9V@0M<y9n{mG-8lB49QsVP9oV!5 zdXdPb!rj#CZ%g%<EzLWDY_*+iwx#q{9ps%|KH1(m*@mwIDf>V}o0F3r%E@PD^ejv? z(32+2{a+ejf>T*-9q3u_yZ81j;U-jn@a}L@_vTR8w*MyZxDIGT*`$74;ey$>H1rme z|I8kJ2&JX%xCN|1JNs?Q=jFAj#Et6N4*L<p!`^b4u$}z^3W;a}B98i^h3E@@e0T9^ zF^fK_3MY*Sl1FwYy!H{2i%czj1$$Ea@pel{drv8!{|!dWi%xH91pPNlO^u*^x>hEl zi1816mvR*aXHLOAleh5lo4wKDruV$PsV!?%I<qDweUv&#Jh8|R39}#{5bx!b1ei>% z;sQv2Zg%wJpOgdG_j@WPQ(6W04h2aag3wI<dGIRrO9|Rh#?8s&#G`P^(G(R;r530N zd7A~Ko&wu{oeO_b0wR_i{PFzYT+%SjZ{HR<fZ~b-`45C--9wxx8`UQpMEdYu`l&YZ zT$@H$-F&&*nMHmI@vR{7?E#uJfIiQUy-_Ur`%Z`(z<EPoQ^*vyuN>|_z7GZ-f)|OY z)CbGLPr}MlutHK--&)QopwFzYy9AmB;XFM{O`1jLhDiNGNp0avhA_wRw`7I&Pi3U^ zssYMAXXo$`d4A|ZE}0rdyTmXmc1-;9Lzn{Yu>)&PP==cF@s?tv@0zvRj;;~#-9Y5O z09S1IZnPku5M_rnxRc~EDL%6ixLiY*Ck8V0f25LIHKI~LY(*cp?wt?FGh4SiUeA$< zNCpym%&REB1W$fVH_naQ?M!*(ZVxz3gzl38Me!O)W2G><J!*_m_Ba2aQ$=14U%c*Y z&Ys1TR71OipxwJRA&Wrb+2A!F*y+a>Eu`e>iIfsfCU;G(_WetZMJ&x_%a)%F1p?Qt zfV*da*P##4&J+;oJBUv+4QaVqAunr(4gk7A_r1ctnQ$XQEs$QlRHnp=&{lTpRKzJ` zJ?;Dd8Hczh&;6}Wo0^j$ME#)iVO2W%5xOJf2Uj8`*|E7E4p`E3GFA}}+XCW6BiE0< ztZHn6@XtY9;V#OY44Qq_^E~lY@wcJ21$!E+JsRmUPR=y%G^fHO3hI0PrO6VV0)%DJ zV_W1Vwi)TRdTaE(M98h(X>`><%S=YK`T@a4OF%M;q@$;Q7Hw>3xjfY<F>uXf!(T8s z7864C!~#ZP8NZmg9I@OxHxnBXO4{q5@`3H=(z`vk)w<cbX}80p$6jE>-;2Uu5{y^h z)=#aG9sTvkpMdxWY$r~^wx;a09*LYo@1+D){1L?zE|)V!PG55?Vn;`4{bA5sP7>Sj ziN8vr$^EmLawFS~RJBhums$zt)i{S&v4<B92n^e_L93orI~K{~GY&t5zU<cUDz2vP ze@OGwCw~xW^5LhpF!BPLyb!(nloHx@6guS-+-EcS$44l()rGe!=r?$8WFv9!Ta)1E zr9yV?_cDHe@P#6n=CZ0zitYL*N9Ur9bcj%J0gSkQ(1Ip!BeFE%Kxjlh@OWk@rCuw= zJ#?xts8ifVr~>eIqLf@Ym?T<fM%cLxXyRx~GtXrmThejtnp#_6Vl$FJo|=7QhbcI` zDymd@&=0mAE%w^m4928YACl`YlR=M&1v^r43vloscs>p}Zf>TkySvUQ>A|TtiNDth zLdY>T5h~Hd%IJzBWxwIQx*^RLgOz2vh;fA6IJ!zd9P&+6iuLI}^YS#J%DLtEutJW5 zU}+w<a!tFyuz0ftdgu~<=wc{1{0U73+LA-N)V(ZxFg$I!bK4YkSwD@Z(#SVS_%A+6 ziV*I^>Cc+0^4{G7RE1xQF39pwSnFT&h+=z=n6FH@AU2NlP|!uA$>+ajZ$XwJah@R- z#w5|q?dHuY?H3HAOj=B*MDN}9#^)A^(0!%zUf!23Slk4tV%4!I(aqI@a2WZ|Fb(cD zGy*O~v{xuXOtFh93~IUpYB0_H?4cw-sAwwN{A3nD60#_ygK^j4m>CeM#L^)Vb{j8$ z3~Enhl=4$&6pB*R7bDH$cuJ<Jcrb1MLT)sw$y{(&Dj5qc2f`}6AnU_<IEXYOlIf01 z7C)hFYYyo}+f9e>F*_SFDpNalDI4KVWK?rY^sAT9cAqMdh}Kfii&;`rRpoIqDU!w< zYoPZv7FF#us7dK){9{9G8n2sZypPo@bw?L4MRa#<s)2TMCRQD#iA-NKMR*4N2wo)~ zBX^F^X0wG`h_`AGmeM!r1))y?;Y*g{HeJdQEN-%4ecf>=0||&8;g~Ad36^+CnK1~R ziOFBq{V8OQ*&R2)jslzT8`Y%V>ZInL?(XS4?|Lh>6|m9z-Qy))ckaX)+8S4vR~{Sm z5J#{Y-OP>R0^;Ia?+|M_vO&twc(nR>4}4&<7`|K<eU0IBoFMEOTp#l@r!hUzj<pvm z>2)`&_Q1(ek-KEC?MtuL^k=g+%v(x#_wUszM3#L+-?Kd`mUzBJG(5u)*jvOzPU7=? zmG>r~@Jq&4xKoRG+sRiYY}!F2y#QA;B+mzL6V~fwgFT+inwBSLvy^2G5{yyTL+;4V z^q^#*hPe~QkT4N&p@%0`t!B5<O!Wb0-p6WK;^S2ps`xMzx>Z)1&!d-3bs`Kf1d;9% zH-!t<(qBV$LS3*G?*;<5reas~L%ox@aqS@Zr+^oVW^KEsKq8R8R)v3)6C3vPj38H| zGGtdl-Zc(pipCBD$<HH7{9DSB^?UoQdtuz0F2mKec&B_h;s?zt+fL0wG(H1nP|&-A z#<7dQk<%CD^Ou8{G+X+LQq<|!$2G)iviy>AyUNv9$IEO9(*+g5gAvH?Hrp(5L#-*u z|4BS1|I<njD#xBIOU3OCuY*oKXooaBy6{`?4yfH4Fys5m=zMUPXceE0L^A(fRIfz- zGY7a@R25C7(N#S-7H%7#MO&1$#Co&QLjNbZ1-aRB7E`b7u4mcFs*)+WEA=Xg0igIv z)I=Z>78%s>SPg&grBZV#tOFK^Fq{$hi<*pAC9FgQ8OcWmF1`7ws=@d)UATKtgafn9 z+G!?u31^uKHqMlNmCB&kFY+EBU1d^qTlGN|KQKV@W@Ytz8#`1J%0Qer0XukqCgJFV zRjd12{0aMG2~M?h%}ig@BNuSmPR6=QY7kq_2tyr|t`9D5xHm;b8=|SjS#?GuM(^_u z^0>=ehZ3jvTY2djc@qSTw>;QoKU@}6sb!tk51olhqQRkLPhx35?w=#zs7^QjC~d<4 zV_?ABYb|-GLC$I2rhx9lKW0t_d4yzTO_O|lK14JQZXaMYWmNu$me!TU?uwG5Dp3-4 zMo(ZHs$nLpx$3<xF3Y0l2<lBU%-kKRYx(m5nMF#AeK;KsBJNXO8L9&yjzPH;Kgd$T z(s{k54{^}l(((&_2Y>jZ9sz~i67h#S_Ub5dGmUTZfbMD@b4vm8Q6A*93gT6p;gY;^ zXwEfKrulHfmy;<FmMvNpHv)L+`5e_uKAcMpdbGh5ZxHr?GUYt<Jvb}JW5YOnRGc=a zxz;~da6P)~vLCMpTnA@-gl+cbtoY*SpbWX3m=_*Ey=nd7(*tBUF`$Y+z^F7;<8XS{ z!|SmJtcaN?X2H*Cex_wufp_?V%CsV_t7(BRXbzE&D+Rg7)#Bu~Qb5CPzVRpZ55rvq z*O&EgBdIQk0_2&LM??iBA^BuRoUkm_$FxTV@~f(<VeVipomoe92<y&dw^;`@UZ@)q z(K!$#+XLS)En4TZLqP@O97>Sno!W#N1z1t)=;b+tF^|%efNEiGRvT$~@!_J%<Nlu} zT@O`r$hVuYHDw{oztpz-%F(hjrGL#1w)WIL2JiphWx1M|?Fyy3P^#0^MKwgB7}V4Z zX&vai=|FTD2!QZXG3(7s5tn{4`#K-J@PNb$vC>*n<2)&X?L9;`rv1ELW_w+PKh4Av z+f9!CAan}bY0&A%dWr3-wST0{HmNf{wn_YPa)d~<P{W(HWAw=uT9ZLr+lMyF4<{LB zqPpA%+>t4iI%?3>%7fDMFfg9>51|GuWlVH|D1^`v8oiNCheteJVt<u&R_;0JtyPNy zs&I>e#fx>E<5a0GHZ8%7B>R;Ao&mX6o5xjxFcG;c?80CjL%-mf1-r7FeC3I*>WDyy zOQ=(mfq;`MeRjRE3NBSl_mpeFDWHa=2|rO)1wg|<cL-)GJ`M6+kjuw$7T%@exSX?8 zPfjm9%Un;<`4dzDz_qv3d`nDf!GkG`4t1L7LXW*=vJ((@?QYf=)LF_HtDUG3{A#Zi zDBi%Y=CVU%Rb~Righ6c$znySbMrrDBqLL7@*f3(aVfDFsN`v0V=UheM#aHir!~1-y zXq(9q?Co}$F|ZqTmiW*p4At?(f*ln#1|8$%XQIs(BsdrLOt;3Bi2lhS5!hM;x!ww0 z7>A>yE)K7<<?#_O9K+*YlPq|G@o$L!_19|+c$S5p!UM3Pz#h~pLK&yS;&^Jyf@72+ z^1Y^B`biF_WPM^~{TM3}b1ivu!wO_?`$nO0R1^G$l6=`yCJ)&O#GovY$42Un^kMh1 zJ@=ZYTD|eskou7@2pjL_3K$X%SRxHf^Ts4CN_Cyn%OZ;SGjunb$S)#c%ijTu(&Wue zVy@<PzA9clPt>4c=CeUKCaK>USi67b<PB(8s$hB36SWPs*9`nv3cHoHcNbgB4jMk% zFcH8Qn;q%i8uG#d7q}&GXK3GM0mg?o0%$!~2t}=gO2ernr>-mHsGTNyoUsA<2PtuA z!GB1jTh`NYszIIrhqzWl9j}XTDD+~X=T!2tSU%*<xHrn>4i$U+wHN%{@J~sCd|Bth zz00z}Jwj(_*Jt#93NSjXX1_sFSOudFj95AzYuiRX=(y7vRgcQ%<Gf;Bd0soD!%6Se zV>v&g;YV%n<rz7czMe}Hy+|8Fhoti(Ahnc!fAxMxI{TJi{-kX&MJ?8LX-G!1!)e(~ zwASzqZ_Q7i4yUEWQyPWFVKgX)v%#C*=>RKK9mb)Mmp1>@O>FzN!cyi>C{;x@Slc!7 zS)^OA?z1IVix?IfopyT_##f>Rm^*qe&bgK|ovuixa-&-D;*CWv&DgoX6DuXG0b%x* zOlJ)i@GU?JASi!puIHT$_$73yw7g!I5O50=P}+(q$U|%kH^bGV4!1od9@1^FUX5ph z{eqGDYSO5`0EDfhf!_T>glx=h-A9gbg7@O2QO$aaT|0yeVWMh&w6bjRo+=#o=GJ)# zg_Y2#+Ymr3phVQ`_c4(ko5oBYIO}Mr=vc}Xd{0wQ><mpR=-3(Lw5jIlPi>pl9(9Zp zA*buBO=nj3Q(2bbNKtWC9!JOp?E?NX;dg)p9F;F<*1xhY5ziJPW&%w*u^povI4!%- zOQ7#$ofr<1Rtg$xg>86U@Lbqf&`1q&-l|PA>S<<InA7Gb+@S2bT?-&7US6q~bJ#$v z3R1&F#uBoxOm@!~xsVepM`iH{pBQ*eq7SN%bD9MQnp*ot2`B@OCQ#*mXSX#$iU!T* zvP)0Sz;t&4jG3p#jki?|#xA$IwIHA0zHpbv;i{qRrYOpyd3XnZG1!H7NDmC5b>WnC z$TcOXv|I(XI!?)iZw8?tLo`PO2@O1`r9Lv5@CKHdbrV&==ePQtVG)u7*T)J=*n!ax z?VY&H@mYtmt;K2ej@FfyMLWA7>R_U_w&+>aRwHS*7YEPk5<HsL1NQjngfN$lX{ZHh zw4pNOw7H>I!tS(ECx#pLd#r}W-9yfWT+`h*hUx(;U3x_4<69icf923@*#8MSBshrk z@S9w@*(;#QK-b~Z5!M0gxU20SY1LsK=@<#>utvt;W-aFEyfnSCA4M=aD@Mm=67gvF zfoh~xj8mp@g#A-KN%~PHt~nL4S1C`mWrW!A=CCt&?V`{$-}fZKE*mZj1A3(1pGc#z z(xN|24FZJ~0op2qEGqZU?4NL=tj@B<{IIoZqtMOL5&Ur*)aoLAKzflY-4Wfw1cE;V z6^r4qXoQHwWJnJf>lHzTO}4~~t8d??`_?nqg=HX)ai+cC+g`q+eUu^F7E<S*nGesO zy_(z%f4;psZ6fa>OLGqZ3Ja*>^O68Y8MVNZB&Z6aaH`&=pEyr)spABsw$qHtz&JXF z(q3Xao!xl?amMyxI5J{2#<h<~jK3n{wR2233t30BRtXE==Db_H@GT+BW6pkVLA}cP z9xNJe%7vF(de$BYIOby!vcyxMblI)<dPSjF>eEk~j?tLma(hvs(>xz`w4j?M{N<iF zMUPgE$Z@y+!pM6PpZ14>E9v2`3S<|0nXk0MPM#z7ZcAezD(iObz?EHuP1wkMDInQC z=`s#VRVhfqIExqL1wo~}7Nb$(!PDb6V*v)163KfpcK93oSbbVR_A=9H7y4pN6A0{j zYT_tX8uC+Nb_I!q%deeTYqu^1rH2ZzEM#%)WrzYJ3znb9Ok0Qe?S|FzD#IPHgT*g# z!dQYdNb8NUVK^ztwBBxkntCc8!rX6cx!-o5Sz}Fts2aQmI?mKZySWe>YNC+q(=x-s zukXw956ZsPqsNJ}1Yc`*(b!X7zH0IV6SJDQ-BNV{<q!U{cRMs=_4_HWpWp1;hXuzh zPZon0RU%y#k!k|gbL8sk&#T-wb`+`VJ;vHCYZK4P7wV*)2{S3oh%!%4b!UI_4O!p+ z%?8`>P6Gci4I|P4(p$`wvO0_S_6Z6Qqp5^-%eGy1vKpcCk!seoU3A_EJ!+fEhM#ns z$#Xuf=B1n39hSk`!y4MujqWqqk2z>3Jt71CT<nmZs7^0)1{DR(<qr|IsyF$m+;U;T zkb&X1h&An`={)Vr2ID(c?Od$zMFSVtqEpSM8ic4C#Uc}a#Rct3g{+cA8k(pa04H@& zW}T(N;JpCXf>eSY595YZ?qG3^M+$7AR5#|VgR3|XLc%jWLGr8V8-b2nhZtAALSYy! z$^T+ugvNr%R=s|3K-{#pywD|T8%s<^o)(@ege>83#j!wXINGO?D!ML4UziFdc&T)R zcMwhWiWX|NcXSlfhCZp2GQ)>8ve&@k@iUBwPH~2_fmm>7v?jZsmfD1*Y$1ajd;uZc zi~r)M;}GW>kjz{x+H$xi8}Fv7t?qipPmiA@toNO51m%q>8-lB>t1D;pt>RfjOIsDz zF98i5Td9`0d@)xGP&<$`i)_#8FM#CPG_=`3A3aqL7@nUo)EEF4y`igk)g}j!>}f*v z{I&jF5=TAAc!!W|K-pw^gXWgPDeECP8J6}&I2zwt_=z;&>os`<drdAut&d#_M{aCQ zqyRj-I8^(XZHhmJE*U!4g;6aSo~v)ZmTl_#w3%XXhf_mndW|`yD!C*e>w76SBl=6p z+(d)ig`N`I^=Z*%C+L0VTy6z_4K*g#44kt%7`W_~)mS@qTal-<gmJ`(xp_(;#6*oy zs=3VjOWZtOSuhNge@oGT9`~v!ZQtfCAbebTt>;<cf+uyS2<_)8lI|`1#`pGvcW@op zCVbQq9H&ZUh`e<aS1!|NmPlu3jYJ|VWSn@FamuhpJw+YJeUS*T3;BM*Tz&NaQi$`9 zP|X!oCH24QCxFC$Ub{jp<ZN|qeN<`iI4@N=kamytxiAi{992rQPgBcqq2n$xUaL+) zZ`XQwAu!T+ofb)FLP>2shNfYz)SW%C+ODl>xmoj-!RI-r!83_ph={JxR6u@*P2|U~ zCJet8j!lKv&m3S|T3keo9|Q&^yE`|pS+Gic*!<Do?Q+=fuj<1cZL9Ud4LR6!#8UP; zLPyVy<c?QuSjsu;XQnu)w+Jn~3w{}9WUS<-=fUO?5gNm<(p4;(d}mooBywKGGUBOz zgldVNz%k*fI7!dGIR$zuCZS=7UXE!5Wv}{uilqLIJ-buc)a{sV4r}L&$>a@reHAyO zDU>1*9)I&m!7JgjH<0@bCXJ#O;q93Hhu`LU`1Y4_WfvDo)0L-p<QjOej=9n8=bEzE zla1aYT>FbG_NNU>@l~IHzxdA5xR>qaz(~sAMJJc-`J{x5?5@)Pj;?mU7be7ld2VDT zR(`BGRxOvxz&L_ZYP!Se2u~i?brp(S>noC5Mg{KUqi<@TF9XrS3GR>bxMdr$Vr|&3 zNcCPeNl{CqGsRiKj^&F0Jx~1@VZ}}@+oicOsRpCc1~)s^=Y3PJwQg4|pAL2~UEX^$ z$=uS@3zlS8BQOgmJ3cU(O!yolmd3txF7$im`Ql5#My>d71_V~AkAtZD5|}&H?sRgZ zEAF>ESs^M9dpLqQ==EO9{dN!t7Ur4weL2gv7Ovu$*urI_JGt=1GFkg@SxtZaNfESM z_lBv`J(GMLgIP6vdF=Spc!Q=p^eV|W*^)PG1*ZF1YUn)X1?iS*K5k5M!FNbCQ4MhR zZ`kX{B4`{!?!-oDDgDvK#Md#4vT40Jwe$m}6obOMq>4+}apRNg?V3v=0+QFv+*t_8 zF^3w@C+(JYd4o%p*6xqRLwyJBsVbkAp0#INQ}v2Ii?U5%`OXBtiyH_S`|jLSQ-#kr zO#UEu>=tkEk%bWUI8}L2y`k$9G_J^^4wu_4pknwmvHDq}c09R)TYf3qb_$=^&az$+ z+u6}<=|a_G(d@lckOs!?$8qlC3$MKR*X!iqPNA<_zdAUftMJMp%VnM*?A4KHzKGbm zu4+S#6}39WuDs7<Yg4bxb#1|%h=^&Oi>}RSMH~~Kott9IV-+0Exlt3?+y(4nX3Xw1 z9z3FF3_{nm1x7ps{IvJ^<=*_-yqNf|d5C9xQrpa<hCREatj|)(?S&{$d)NR7BC3fR zX)j4{Gs|@q64jZ?6X*fL;pRCD@%%ADB2p6KH$H!7SeImGV9@l4I>zXk<}4bW>*Tm6 zQTh`ei2;Zn?eqKl6$TgV6pAq=Xam(=%Z&039RXLC-dgucw?Lg!3kaf047O7`9I41! z*6bxmF~8?BD%t`liBN<A&SP963;R}k$V!e=L236)&3cZyh~nDWPUe<%HmR7HXYS+s zmsmN{%uimo*ZK(YcG_2dpr-()P|60dymxGze$z7VaLdKqE&B}jYf6+;8J1<6=;`TQ z+Kw#jc9R8hd)eU|^gYo}jY#8~B4%M9t<IzL(}qou2Mz+*l<+csLj53V5FjNFzqsL` zzOK(A?@EH*Jn-ClzIM^)h7rrs9;!!{EkPz&>`YG#rl)KIZaZ1+;odRisq?wde8W(> zkN)9dEPPksmvI%o#jt?**u^ZNHuE+Zpybf~<Mqc&KE=vpBas7<ZO`7G0q);};>QV} z(<+fDEwRh18=|SL7cGtxc%Bq}x%>|LG+BeeNvxI#03&0q`z2Kxh2sb=%np5myz#kX zL6agXemW_y6S^^m?MT)pvzLSoMg*rHgk3u`3}iQF^4-XD+}JE*pH4+8_BUA2QgAXg z?3}QRCV_C~qFlN}Q%xJZ$oabt39(1*+H9QfV+6_uU)D8SVNdQ!d|hr{`#sUqWBx~n zQs^7sJyD6r1DxT$g;TbH6&!1dDR!if+cS)QYe(l={YnHT(lrd`C!X`%<*cf+JDJF~ zos*czMoTTduQSyHf0XIgzkBgtuh+~O)M}){kac0rl^0~)gs!|Vw-oK6??scJBwOgk zpRF_irri>?tDCt=n`Z0wHTI1T;BrvTtdT&njj7WG_rU64Or5w!6N)zGJdKl`=nb(; zDm*ApoqnOr3gH2VDSx)>6b9YN;SSCzEm6>7-nC`~Urt}V*#&Si7^bowC8Tw3sV23~ z?v3_s&FtUbvH(}&OI|zcIt&}KfYzMqEM7scn?d^x-|$qQ-YN#Dy=LnV5`0ltvlR8L z`DOP__t?IKS1&&CB?yNt@$OvY`fUK77x>CK;H&xSQ;M}yk+sLiy6A$OHQ~$sQ(A^; zT8{Ch!O%>0rLi(X$`=4=3P2MOO<JaXgzQt0vrr4}uWVX+-_r>X^VD5gijl-dUT{zH zS{m81yN$KBS1GcpjM1ZJ+rL|G_q&wmpn8-73scy992wKZOG{DycHStkgy4*LP>(br zp4(TfK-!3xP=DB&!6zqgc-vp+Mu?8z(%xHY_lSg@!x`vGBJn~3YgE2;d@<Cj^mCob zk6L6;Ium#znH2L3>|$2?sC@ETp)nGWYUJIJ{+amDjL;=W%aZ3l%OwU-ad@0+JimSe z^o7vjvS)Ynez8-VTo+e4V=>*M+-A7-@b|^9D#r<rF8YMVZ(_7KF4KFaNi9~`9nF*= z*WR+@bqZ7hMqLIHPZzBHK2z&qYdEL%y3%(C!{y<~j?YNXn2i1WqE+=tYUslj_gqlB zV3YEXSDglx>7oV>?;7z}Q@d7<qEseL%?M>u^5dyMTY&lgl5?X<AY8+4;6+KZ@LY&< zU2|F~=2G@fzHKoFp-2FeNI2p?uKToqsTR4We7565KIU{l^Go3C>#1!dB?*fC;wP1| zx|vV^8Eem7p}I&o=hx_g?_NBPV@p;G&U97hX3#E5D1VqtyUTq#?OkBbVf-0W-+_ZB z$*MvBG|3A-4Yy>nQS?WCSog*=NmI~Cz&mp=2kWmThA?*n|DX(Q7Mp=Z6UU~i_2g3` zLux8HU%Lo#Hf9JF-%>)->l`m0?RRp&_1*;)(?8s{$#NX)07-0Oujt1-FyrQC2lnVb z;^X7HRugFYrEBN*c@@R3v70q-69o}o1#eQlYj5-}4@eeHXWxu{mP3vKMCB|6T;|L< z5~lZmA>Z`z<;>b6)Y5B)McKLEZS!1NJd3)&uUbKWdZ5HZ3x-w<V)3uo3$5&8+Mfs# z%G?0#;PqSAy5_Nl3pJ6gWs8`*ONkOXGB=Dl)aH4QYF|Sfl({i-DHq(MN%IQ%sg5WY z>~9{zQibU1cMJRZL_RRypl!aaXe6uuhD<}!MQzK6Q%i)f8yjpMdT_2Z<NTXZzJwuo z)8yxy*l&`uwC^?(+PJuQ37(g8^BhYDySia7^5j;^+g_gv^x~JK6NV;pd)&P7YI=qC z@k#<K@A?od?ODpEzIrz?L(R8GQ^LiuMKG-Ez1Putv95;jdVk)2DBmtfeU*&A@?N7M zzd_gl^V@~w%u<k?E~v6><v#VQ!l&t1b(8RNr`qwu$&%0?-){!KB>I|~DUqpdnAOg# zEn3K&c*;txN~r#dvyWe1rRQ(+$xEnAuH6Z>aG^F%)L|W;vuV>i&05)2xhbf>Y-rXV zIPcPb#<c!_44ykCJfDBu@DMKW2eD5uvX!R4K0Ah45bGw@+{L&|k#yZMS{S>yc-4fk zg<x;J<1c0qC{JLhmH%Ygn54jZ*3L^cvpQH_e{lbrZqC5|bZ2kF>!^U7v!^);bp_YE z{PS#qE=i%-6|vc8InEY*E=kBPmZusa&fJ?-Ly#)oGYK}HiB-OXYnyAU>dq;;?x8Q` z=3v&|cDiB&n2OkURW-vozi`~I>jcfADbethiJP#z6DohbPj1)Hvqv47g4z2}Oq0ob zDL*Yj&3X{Ps<^<~)dTrs#a=US<r&pJ0gq=do6Nc@fJB42Rga$Uwr%@-N7IW6U;E2c zB>Y?oiBQo4gu3^K^dw$#PWR=m#e+^4yC42O8FJnJ><+0px9kR#x=RVFos-jQXi(O- znL(7GS90~-_``qC6c@k8vn9_xS>ZNO<j}7Pr{3dqtAsA{>CtxZ3f8)}D5qodX&RQV za0D!=K{Mo5^$jVw1&xw6`}xfAjYZt#QI$hU_V!4`lKnM`84cu;IlwU0r=D4d>rzl0 zI&{)XKurY=+`5x)79&zM6llIcH(k?r@Sjub-+j${i`s{h?Q(6z`r`K!s-3%;{bimb z`rz}+c+|JBICk;LcQ?Mb5l)`d+EjQ<U;d@VA-I3@8v$0MG2YgCGqpZ%hm*o|;M_Ki za~=B~E*N@l(`@pufVo)8beZSVw|<3}uGd~TP^im1394`WarEmDf>qyyF?Jm~<BMF% zcUFAY{mu9h=s$6GQQtT1?%$V$*8elJ{`>EDH$&<^>-v7uWuC;sX&`mB?J55=s{QYW z+`U-J{|~4CzXwpSAMy|LG{=7(<2(BgU*<FLj?Z-v8q(jqx4DvZ<lGNVjb7|WJKu)o z?|@wSoOgUlD$-nF-?xjtZvzHTl6P7r!})@{xfN1@?t_@ccOFNyEiXSY@i~p%J(5+D z$*=?S8RT*PMsscA{yMv7^gjhg#3PdnXsV7{P)mW_cKMJR)%(x;Xtm~Kq&R=z?ioZm z$GpYj=3|Uf^87<i?oH_9x=;c=T%}H6Kj~LdflnLx(CLTzwvKS{lbyA(wf!e2wzoeX zrKoLNKecTcEJ65G@XdI13I_&MHO*N9f$OnFlixL-TfZAj4-auESPh;NX)y1Gvas!Y z9n^#-jlK2>jSh(&S14E|cQgcgkUk6LN@FDAG^!TZvyW&=OIBFbA7Gnpr|>!P6;2u~ z!g}yX>B(H+OBI}BYl*<1)6Se_TMdqMP#vxz5%$+z|A+m+XTjkw<jGIS1|qd!SvS7d zEdIf{_83IywR-8i2?fGE<Y9+(1(nE?gX-!(g}5phm9vueN0E=w6eN-Q`K`p0qqa6w z?vrbFb@PF5`W`hAR&uS-R4&XCbb`&|<o=`EbdvBzXy>SnLf+`&^|1}3*34`@<s~++ zVaNXkCr$h*elELc!UJtXA<0Y67Y@%NKTg5+spJ|oVTTjB`+PaCvdU)+0|cRfuYFp! z8ZS>N1lKY$T-4kRwYxC*ru#MUWG~euY0312T#*7hYC%iat_0@_L4?1hl!NSHRJ!Wg z$DkoFgsyq~cTn5$!o;UG^48@>cLC_izBvq=UNV@<yA$~H2z_$+Y2r^${KOyBuVupG z<j&7Ohi-IJPr!EkuuI%AERkJ9c(@jcxiI-I<fYnXdcJQs{>KqA2x6C?Sb&kR;0xc+ zMPDPZE1c9i34|jNJdJ`qPkv`_{%*Gn&263H^4KVbCu8m>`lSJf{~uj%9uIZ<{{3Gq zS9HmKWf|)jnTp76tl6~~q3lZy*&|!FK}F19%nW5`3WcmgWLFtuWJ|_YmSXG^8S7;G zz1`pY`rP;Z{a(NSJUl$`exK)@=W!gb=kYo+I(t~}3h0HkwsDMg4$;_E#~*BMKOZgR zz!qrSj2LG6Z$DfECIuaA6s6|~hdBN7G3-EXT)(Cy%K6{o;5->hJL`y^L(68lpJU7J z#HtghuA+B_wsz#mw^Umh`E$v-?f1|GZMMd36$$`kL(9&GzlXA?_U4AN0Yj)E<q?OP z-$%BKR279EN8es@Tn!%ykh^^F3K0F5o>wAgb(HtqvyB~xPum_*M^=Cn)3Vi(cagQV zWAM<@KHy2Fmqy|&6rKXwE3C@^N7HwgW}LUG(=VBNDT5r5ha{*rF??QQs^K@Vy3M0l z(wIT3sN2-t?N0gi%}3hdN9<HiZ{iyLH{j&iHhKuqnGZJORGaT^GSb_So74Plp_`fg z$T_jyFJdccZ8HlFA%3}H>luO#m4b&2{gJ*Ue})k{djsFs_X9ikdSvfOQX_1Vn~zcK zn|HW3FBe=knFnxXmx7h&LvwQ+-f<lD@?L^juV&A==GmoTIf2c>#$~gfQGisVAE?F2 zN~Zqa{}r#(%ynt+Pvr+cYe!^j{n(RS_htOjG@Ol##lJ0v4V5|tt1@~8l(+C$r0eF< z=qaL0*}E`$=zhy;<mwWD4B4YDc&Ce!+U)$NuHQew^Xq7PF=`wbs~A7z%#xv8_`njZ zKdi4hTKt{cNB1rFT+?AS4O|2xo;0(1>D@TP%*9<;Dr90mtOqG~v)#mCjc9$Za;*l$ zH-EIE-ILxE5ythRX*2CXz)8R)dLD#2JAMtJibP<myEG1yHZ?xjiER&u^{Mr%w;udn zPLFiZ<lx{)BUCmsKHZzJj|Sgy7sRJVP_D+lf|X2}uD_288Z=cc&tvwuS48c!i^;<5 zp%JM`Yf=Af!6N>H=$rX#I~ieQ9*m(mUF@3D3hj3On(LLlz$Ye=v)qvt$1GNkxVWY7 z-|u+t{Vsdm*_rX?Dk<BjE!EQ1^#(k}LwzcIxt#4-oLZ227h2l%JAd={KiP=y`Je$f zDHA<|r>P#4)FRE<3yJIy`9lAnicKxX{4mt9lc`3UACH%8v~IWfZ5BecJU#zf$IDfK zp<J)vc?0El4+pSOL0}l=WUpUcbqD$ayN|0a+@&J%f(ZJl0H(-`T;HTaR$}+i$?nvl zME<fB?ugCZVlfyUvUYCSp$D;B^eAIOMz|DVW9OY*BlzQSAuERNUU%z4v`?EDiDJ(~ z$$!9HQ&Xkd)NdM3CqKF(fEN<%nlJw5K*Fozue(~Uoa@`mmsaj{`prBnbJy|)-@&mg zap>*>AFN=om(g+bPa35pG)~~<l;;J~(*`f4?FDz$h#H;2ezV|@#s<$V{avBhKoOkk zClgal?cZW!Dk)MMU<z%r$uc_K@!?pt6w{j>ty+!mr!<e|h4y=^D^G(WnIdzAqk)9o z@gbo86_93<B!c8}S&MpqLbM8ZgI1MmHgi=&CzYTkv4i(ya9a>sXo7%aE$1lWjGm<^ z-eQX<aK~LtqF=o~JRc%I{#KOfwbH4*MSMl3^=ktIue3WxxGUDq)#fz{cDkL>v9NbJ zXm*$DKFF%C>e7a!Z&|6ViwsEjEpc;q9dbr4=y!wGI{)-iBKF(;t%{mjKTw*3ZU<aa z_HgjA+(v?<E26i{k43Zo;1t0T#x$z8GZZUq)hX*NtR@=X6Z@mj)M`Q;2#5tc)&~Z} zP*k>hXpf6%4e5NFhQ+rr>#ih@x(q*2!7(}B7&@k6KB%@{?df5$v6k0ZAUBnqq(bha z2t?XgTmN(O<!JBd#2l9H=)@7WnH^Dw^IeCLeJuY^zV-^8Yvk?k{~}FkyFUk3qcocP zmo6jh?;LVPR^VS}&@UHPT0;t-zm7&-kQDw1Uw59F5`cc@GN6SrZC!-JaIfFFDe#!b zs1%~-N~3MU`Y&kpM)fW!wJN`ee4RKk*_o>8Q^3aL;$h2V0*49GQ@qA@Mil)9hIw{H z)?<1epBhvoPY2~+^>G-X%{!xjadi}(qL<phttPkS89s=2Q`NzlRW!*t1>jP3z$097 zpsDGL909X=ksY4Oq`w^DVc=S>1k||$5DT2x(CJjSo;rSoo->K}j!p3QNtj^e591Cz z@P32s4ROsI{Z_yjg|}vOKcvY#Nk9hfJlfGaQxz{q`7@wL-bs^s8WHDY@CzckhI)Eg zGP?yy(@?;a2Z7(wv|~?Y!nf8G<#j`kSWzd|dk2iC_i2*Dj5Oo0YW#So){1|3Gbi%< ziR+U?=WXZiIaWc67O5$qTh6X;3|+bGdpH{;T9+<q3&bbgE8S(BTd^f<h<5}AT-(D1 zZQx@5Kz@!9ecOZQ4JdQ36F-e#p<812B(agh2zXLf5eF%&uPlVIPy9n-;Ok3!c;FMu z6;Z(JR?Yyp%TR@e8#IHUVU{u2vHOP)zS_{K>6#D}mycJXY1qbnFB?sq4LHh@)2B+i zoRe8FD2(NqOB7f)UFn?Cg%^yqkA6DbnXxnJ4qJU2KW0od9!{!XSC&lq{aj8t4}?Ea zhsJ`oVRc<k)tV~eI!>?MJDe^EkD$AVHRV*)r8yK_7wJkJ70d>5pPIg(S2y11>>6xA z0XFp$$e%|Q7KoAc{-Q;&)WH+9oHZ@7;pq~7vp0olR*^NST3g4wC&S?~LqjLTcHYlQ zHg0C`!;^U2!#i|ET-l4C=j@&N;Dx<y9F*cG_pV}H<1gAdti@bvr077{g9lagdvj>H zZ#z(9pPSGvM+N34z?4Uo++|e9*H@%b*|Mn)NSq%d@C))l#NwX_e^Kv$M))%w##!G% z!=SOO!l>b@p)<Fey!yIernh2U;Q5|zyS9o>qIe@vV})^Y*rYOE-@s#U?r-ri5Y^<E z>A^r<&cWpa;FN#NTp37D_1ut%YOdiNI6F;oXzU#@FtQNWRVpio#QqTw*Gs?ij$BV} za0`d$5v|upJ;4f`5YT>E-53p@O!JINyihGe;+tl<iaO)J6fZ|tozz-(EchDiZ$$EV zlnRV6A29LN#!k(qMHfqmav9RRG9Woz4;TfD7y~YTl@thH)WeI<2-E1#cUh`$%xJ;% z&eW%^p6)5|W$Go5gJW?}A3?NH;OMF=2=O6U1zgvmUoa`)A3EADT1QLNV&oWCWkKVD zGTVJ=X_5TqAFQ`&&p5Mo;oU^6aP8w99At!+Y&M6=Mv6<}H2OoyihN8T3;O^XF?X(E zWuv&c$sHJVB6o@gN-XOMiA<0_p5z-fcp+-<rG6f_>6~o{>lqjQho2M)WzHkqA%IlT zbojz10t{I+;$!BFOU`Y)`E&CGOayAS<ovVyWhexZ_bV2fQ|?~-=sg}<6j`H<GZ=mY zLuA%I`HYXrNjNj$amM1BaQW?8)SX*?E06;!cC`97Fp%!ed%o%+6``TWT7AFk`Z|=A zWkomcRj3tf6uzhzF;t54>CU?V-Lux%;$qfJ(mq7O6ykinZAC}jv?MmixATmz)wECY z=F1m#EOJei6_o|I)`ab88^b6w2F&Yeq2S-6_{Q5TdzVKh&(BR`izP=u{uaXm4|mI? z^7d#@xnRgc#C12;o4@sy>^fPM54zaII)<Az!A=1e^M-d`r8_l__P}>9JJ(FDx7)o) zWM8p2_Bs#cE=O~XGu$gxlUIk!{7WLX?#a0qfdGw4ny-yoA&iF_UD44f3sgQo7qAJm zQ3MbjbF2F&7js3HMh@DYN0RQw0PPu!e{$K5MZ<sF<4^0oqr7n1`-^7IC<#+HjBVkg z*P4in1_h~);L+lIxT{&O;IW=2@g7JZV`x_L<L0>XTB^eBQZ8uy;KMU=?Q7rG7=KAN z1fjOzbfVH{RG&^=5)id7#%5ee@@f@IH(nDI%S8^1-s24D5;%$N2S1@uU5CYoGhR>G zDLT1~YO(JD<?O6|iYUg*^dV7?OZ?5w1t}!dxqV_$pf}i{lvfrsHTO+%G#<wTjRG%^ zzF8CjJV(Hy0i7(2!MA;<4XY=NL9|-~M{e1(uAncEv`GRy1}pj(GrL)$ARJ0P`11h- zhhMqE59evQjeqE5>#JH^J2SRqm$f;4p%o=xdq<{)!YRq~1l3D-tpPo|elm%Oc<&cT zF(j4n$Nn`2e}JyhT*o@m1joFL4rnoLaIF&eHQ=bL5?!c<<ANxh+Z_VT4{j$Dy-lPQ zxHWwKw%ndIJ?Id#jVXJky4CPnmYNf>^c0J-4!)#9RxkOa9nN*ZOD?v>yQ{^UqTxLt zKr>o85!T^Hw9uiINN?WZh@N$p>1o(wbf<g6I*e2Z>yEE|48HD6xaO7!Z9fv0tXd8L zpW2GeiG5|M*%2ZM>M)J$PE{tc_*-}fy-_T@yo=MER#*`k6WdAn<ytAQ8`hKB;Ikon z=)>Ajo~-v3@W)ygyRrqo(UM&Rr=l8Zm9?*=hGHXe(UU(Ny?6z?#E1bC!qs%S={V%N zX#RlRoud7*teykD6j^LE-wEyFSGa=}^rdXt3IC!=lKx5=+vdgo;T514i%<jOh1Bve zdfW6#<i>OM+GCy8TeFhWCQ8*^jf@?ynVp}F>eZ7&jTzmkIu|J+55IPfzA+*twRkB& z>xbg&cX}$h2-=AeqKiMbMTABPjqlo0UiK}15>w7_za->WJ{nJN8J4%*I=$&-CLbqL zba|+i(2=#uaBZ=JAf{cT;8WeDc8@jt>i@UgmoS+4FJ+BHHh=5upaFP=N0wKk#5-Vz zS7vR1>a(gSpkB*B2E~&7-a5cO6zJYh^3M7|^Z4zhZe00VxmsV}Hud(5Y7vVfd57yf zeL&o-V9G)A&nMRv1#}C{SF+|Rl#NgAuw=zGlk0fxvUcc!j)7Ccz#z%2wlJkaw?I+I zVmQ3qHa?yklQm`{M6wgLq&WBbyFPy1w&VE)a$bqSD@VFTIXS|0DvL^6vrM(K)HWC} zQSw5jV%1I$Bv5e#rDkq(3K#83stIV@<WJ&jp)S3=(6tz074cJ~N1TjvBg}X?t!Yp` zz9nej0^D`|Pc%4=@+So_5yAk7-DTtE9UskXX2OCV3i=vRd>3o*xyoJIaT<L~f*?kQ zY&pybNf?fE^Ozi8yF)M*2X#jUOQ(?s@Pj!)Xj>lxDl=ygjrb@gpQ=yegbj1@P=pOm zrJ!ZBGo_=3ueN%dapZDXIU^}oHE7xLOCp&ce8cqae0DKSVA~b@{W)IN03?K$TpDv< ze#B0Hm8j`G40>iCKKe$KtBJBo!|xBrkEbM3z?Sr#pEb(mR%41hn7!IO<NN^-&pJR_ zy`3w&5YA0TT#ZZLTRY)j56n*ww;UhhK$swziyG%TI)0y3tP_ZNmXGC#c)V{^(G{?N zIp4gc)NI!hUS6V4B*s(XLy7i23^gzB<iib0`c6p*49BJ%Pdj@ejH?araVwo@saeh$ zjtkzXf<kxbtH@{WjSe-@#hejZVPP`Gd-l-b0_HZ5VgbT2Gns26(`Qr8dAP0C5FkOm zLKi3{CS1}?>t$_#ls@MLk-SLY#v*33#!-o#zVp<JQ$uG`KC8c5>8WW&6ohS6Q(FXf z3RriK!Cbo=zE&0ezM+yi?X^0cu8vsU=@dd4K`%$vwJuNwkClO!txex>HkSKE-x1di zP&iDM{Vz2;VLAw?qCWiZ>E^WF_|V*k4<EkB^j+X$ef(<Rjc3B6FI9%ei?uNBuD9VU z#aA&1Q)hN2Wkl2PGXB+L;p65tRwuq9d#%*nc5%Rv*bNDbo{-$Ahgv^9&Fonv1C(mH zg|q6S>LwRHc^a^LVG!&z&8X*`G_Qbpd?{!p%%ET{u+`YKKtR_6*~-SgmSuTwO_e0| z<Y^@6>zaLxL#l{v+UxrIq>~rNz`DvL<}=<vwA>fY9~!)v>Et9z!R;=CSaCZp)2v<A zKW1>pSg#ja9vn;D(0eq{(MswME=(tfQdsMf3?RpYBtc_Il3l+~NutySD}mXO7|~v# zk9|8yc(wx+VHA9Z{<NZmlg1T2n&2wIMig&F4QQ(0cAx~=afW@Iq>4$zOsrwx!HI9b z$A^B#Cf%v&N3IK)kVZl+%pz<0n?kEjknqlP1CN^#*Ws02oGq?zJR5Q1(-1cbe`EQe zW}Z3b^Er+^;v`AVprF<zN7McUigOLGQCA|y=s7FqZ#i$Gb}~JjS4a6m-O}J!7s4`~ z=UYx}UAm35iJ~^cR2V0RDDkyi9v)WDmxn@gpSfa{U7JVL^ZlcOE{Z2{y)I$*Zysq` zsU_LxH7~pzY<+M2@T(cM!XXvXicsaj02fb{*c_2E?U}s1*DogbxqPLoOYBOOZXTU| z>!Re#+U(AT>gxwRcN2G>dMHiyZx!WyNfm2FpF^yfW5cO>R&Uq|L(Z+~sc;>L)9e>_ zmBAPX-V<wV1LxhrnamW+RxpVB$x??v0L|#k(8c|&RDg}Sdan*zxs=(z@x`SV@2$fn zt(@JN?dDgVOd1or5&P(zzAWj(sz4n@r|n8B{dAOZaYS%g9u;k;7%zq(v-mS?MMfj^ z_=@7nArDc19wOs?5TAT&g)E#Rah|z}rf}=TqGcZuH2ojh5H7aoTx2k_CjbAI+h;lq z501s4t~h}Ul#-OG%B{*C%HM|^MD<aW&jz<+vfcwncqbU79N)W%l`y@Ju`|8b>&9-c zuAHrGi+7@&H28$Hpx6M4_h?Ma;l3}K2p=th-Dryo73Ewi<lkLXN&dN~r~X+`4;-NM zl>6Gi2aeAIZtny<xUx$JM0k`9T;Oj!F1lCjRJEfeSBJH8QGc+z1?Z)Kjg-53?<zhy zhy6>*iPupvE0<M81$@~!-#pds1~-%yrO8NxhlwXnP_E2O4E|v(0VXgwZHV&e7!wxl z251G`{nINW&1~hxMOCAfvBJAz`R0?<=W}ftzf9RZx4erhH3rT*&Bb}p+Unyyl;V2f zx9^!fhn_t4C9ympwbS*@_0~rBGmLZ*I(N1vz)EYYmrELfKa(c*0aq{$E$Ad{pg0Q8 z3ClIC2!-6BkC>eHKNhRdmZTVuA%hL6v7_Ck;Wu|F!X#E0!c|5Cuf;%R+}>uyIdGf( zrp0J+hEeqSc1d>k>!EHX<}PK{>t(jg=a7?p8Wck#pe<307qi5DEJo78cydTg=&mGa z_Kdl{JA8jl4th(K8$-DosK-$O0nHk;d^zD2qrE&B-c@2=nAupAt?i$7U3;G7HjILK zT`+(soXzyPG2E~_)}6ETOnRHy>8BhoZV)#R)6?rzL|%L`(qw1YC{a&>-zbaE`OY?g zl5}aaiyEI14cX!p(Njt3AIT}}MjOe`H2Dvsf2+jhQDZmBoV#!A_(SBoPY3%O=6qT= zQIZ%@Fv_FK0yoz_;BpxJe3iWMskJ?h3q$#`cQ^j)^Xnt~H?YA{{aYsMW#;nL&a{Dd zdx&=eRl1I)(%!j_UbazmPsIU`%1^5b@^!M*1g#F`hY9b3^;iz~bmM|)q7xMZE(C^# zKKti;KPiv1c--6E8Ih&`DY!WOi~Qr{fNF|v_6T@gQvR4Ti<&hGYWL#sr#00prRm3d zEP3^KKbw0td_wfsqo=W-4V2-$PA44OGvKfg1Y1|WXX%BC3v})v*bwUTi&R;5Z__cQ ztw85>$9AEC;>B`XY|zyLr@*g`Gx4c+vnZ?}cF*5R$;$a%KZGQ72kl}UA!9kXfcMau zBz{#>7s=C4uACtB?10@&Sme(_ww0@t>&IlIrju{!=d7jvymQ9lk+6Kw7bRD`S{3gi zfHMC8T3hy#dbf#aorp0r=ocd-PRQrW3o&3BjE|!(z~=BDSc<eGrOYOA8ua;^y;AlW zh6rZ5h$z>F96*l5js%TlsQMiw8$~_!cs36#0<@LE4VIk?a{x>HarGsfi+|b|ZgMB= z6EJjbVK&~TTTX);s0bHjQ8}~mc8Iv1{KN@X?~xrBow=sQo4Y14IX9opBUkUimFE0& zOjx7r`F-&2&<osLj5lesEO%4Fr%wc2)uuktlt^HxeOn{tOmR|DugkRr*?#L<tzW?q zRsNUb0e#z_@iZF2lkY!q^duRywamFhwB2)Q=8Xx|(_lV}!LR8WYgX^VG+%G#4%Y8k zC&FnkNB78L>4)>&g<ud<W@PWVJq)*vOfMP#lqvu=?a&JQv@I3RDPNd|yGeuO0B>Qq zamPJ3yI*RX=@<F=+&m)wnM7xQ$yJw~HyqT4tXgfF^Cdg%miWQotN~=Iyg|{uPo{M} z#&PMRXhM}@r~5hyR!t2xQ3QC%1yCm3L1NHho|DIH+<`aR-$H+w-IiQZgRFe5<?KUe zE)godQhS-nw;Jh&NY!Y~0RZen7#g9B<TuaX+Z1=A|7fdq6Dx|Z1Zb}ARdHqaGlEMA zk;7>YwkHlU415Ex*-ie8+$a&L{W!;kS;+(v*?W#9#?d$L`boy?EZLsH5xhXKYf~ml za|V!I+H+ZLS1}&wvLy6p;{QI;)mr{V%WvMfX>9U$@6Fy3E>WP!Qkq3r@z#cjFtl!i z#$OC1ydAVL#DSkA$?Le_u4ge&fAxCze(jcR@(CMxpb`f^X~b&S8=@X~&HJrwOR-RI zI4!n;p6t|nk3AUU6N}|lHIXq5YQ+5!W>Dl0>IIEb6ld_4qEyI@Lu$jt#%i-~bL+*$ zj$qHHHxnJ!mGd!!_59LX#NH;UCN~uw{GDmhkI#~j^RFUMqHA|m+mx8=^)~)*D{0(M znJcQUCp5-BviPIVxW<zkqpKwT<WlnKxq4-t+rX1l!^WW45WlBO!R(cL)*~?>I*j<# zgIZ|pT9rBF`s)((6HYG7;JmZ)I#Y^&OLdD<AQdJ0w0kP%5W>gxSJuW-!hQOS)D|Rv zd(J5X6xf*7osq=W<tp5S|BKwvze2hsc})BYmz58f_iyeSCP}n0VeiMzE2Yr_Xd@zz zPGi^XiQFKlJGuDSa(ARM=o@LuJ<PaXJO?k}npo+L0>v)eP<qnH?+nwcm{US{8AXDh zQcYTQlnn|tUnJ)a-&R}_Ts#*&d?pel;C%}H#Df+O;!~}&|2BQE?aE)aoaEBe{<HCR z+-94yEIv2A5m%v8>mrYl;@0ClkjB|ZKASPPZuOHh)J8SeDS2{upBOOq5R#%v5*3I^ zO40F_T2|Y|5jsp)$}+i1gTC?xsB@I`ayCE|y&aJ`HgC1d*}Y}4$KAXSZtAj=+%FTN z7>pech0)HqoP}EQXnNzl%8(h_q*`mKv@+snwe`W3Jbk6F-HSm}YHQ|AB_Xb0p`S3X zPhShKb4+Z1nu);s1qofZ5>?C+NYEYj4U*{TXg_!#wI#vc0cjIz{+6EPpD>`Q+#cOH zq%|rw6hsy`sS&H0XieRZiJXJf&1Uwt4&IuG1CaK^sPu$%2RnD4PkHC2wz%pr-f#bt zUjkgy|KKT7kJT2AmH()`z-|(&nS68b@4GLF<sXuK<jL;6*6Wx-rIF@`6Q|i@(|I-F zrC=eN?Yf}n3*B;azohM)7ZA3d0Xn<9%C|GaF;Z<tih?u=T`F~f_qZo@wkA%ije%7o zFrS8AaiTR{SQjuT^4vMACjtE^q4+(%0sLUUtV_np$De_kBsuO;l(A*Vc&F(+<NagR zr@1A_D7gOsRmDh#;_4^re36_r4w_|%69zCgkn70xi6`H+8L=Wk38!ths&?S25~q%} z%bk?BV9^E3f_{hHlFUvOlJE00+|3xjm%PuN`2;r|BR}oVp@*-e8~o)-Pwf_SQD$~L zz7#~%3Lu$lc0KGD@R!o}B~7VScNoB#=iG;{z(1kcY{|cFW5mS*OeZq+<9EXh<fd?Q zToNCGqiQULNkdJWkjWBA_>8KEPuFkPHoKIN89cww<uqgz(fCplQ$)8;0L4NP5}{G# z$EI__-&$2y>_85*g`ie*iKoX*E*{V{96JPvu{Vha)I3lMKSTdhF<M@#%e!^EPk@*j z7(Qgw8m-{py}gbOL1J@*-WV!ic-yH$Rbt14872o*@rD`ocXd=P9n24C*!1}oAJ=Hn zsjI5l)KiCD#^lC45KqMVBBMrNjJF(<^GsN*(v!d|Ctpi04~`luwe$`!l8aan@P6#O zc26!B_GgCHM;k(ynPX}Ep6?4Wc{;7GvDOUaeg*)zqiPYHFLhw4@oP=@hlGXJi|=ji z_Ne%aCu+Ci`kmxmqmj4!gBtQ`eTADkWU;7CS)-L#91HG{Y>Nn#Y4u*8%e&B>{pXsc zkl+%x(e#$PNi~Eq*Go;O5bg1!#+CbsLG?@jB#9d@{v}DIUD5KqJWsV_O&%XH31%PH z@k#>Su4HH=uhn?3U>}&aM+zG!dxPuU>Q`DDf@Wh8<kSyE=EhN?vql_(tbFpU4ViB0 z(>E-msal^zgJbJiKcs{TicQ~rrbt$D3D8;Ry}pgXp4m1g7-#N%KKahQCxACwer{bW zoT;?by+7I`t0N?aIQD}^!&~0&>VBPpt*fjV$Y_w|iFXLOpERfr>$>eF9tP?>S2R5r z<rP+^LF^i*WscK4g;ysXI&GzwT|c+UI=S@JRB)NWDX=c(yq@bhM!CzEN1!5ISCJwv ze~<YHRYI*A(sGhqdIVjbHFjmd;4U{TVrB$TJ^uUU_U;9_Yax??2-PoqcwwF|8UCxR z<X}7~(J!#or8J_Xa%^OvjwI#6(?15?#u(#D{``%}?(AXy#WN)@TEG#QqPSXur>3_I zT(zRzJ-{fAoXN~g6;7@_?lQs5@$Yi4NPWK9nKw4lbM=zzfp8NwlAe42YoSh<XtY_f z3f4tYdrhpeBc5WDp8bC^pZ~!}dHMo46a25axXRkOXW5oo@S>g>Gj$bi*?Bh6N;J+; zLH)iGFRWwb&XmFBuxG`V&NH6xrpLu!!%JYEXpap-Q&`W{Smh}22n>a>#G!1V4B58_ z|NOm#Xyt#2Nco^5@LG?2ELP+aM|`JPLi*s#b7FmB^z1D&Y}=;VU?jK$u0W!$+H5Yj z7%<&-Ag8bko2)j6Qkkp#|Ae}KfBToZ1W|3Vs*kbIl;YN@FUqe*;JEY<t2w=<M{wA; zHX25{ge#Feqhvh)5z`I*eg7qHkon)6`oErG`fb7g+}Z#H{hucUz8FmYmm~WB{(=1S zaE<@*;*UtL|NHl-{rs1IW}K7K-|@*j2%FKFv%C{Fe%HlJD47>KlO))AAEYxykjLtr z>CDx@+N(Zn)o~Gxu28!+*|4PP1z*9eE?@5d##fMIyJfD<(e&<g4ZR@9L8fP`SEZuq zp+GW7XIb5A&r|O|kXgsoM~CUpJT$fcAAuGCYMJ%xeHzx$`=#m!(IHJCt-9|(#3*XB zT?1fNkRB1(ORI@a+W+YnZSSM94cMSIW{)R@^b5=|jStkvjDH|ie)bp91<~lxLuP-p zwh&g3**Se=OLH!2eP3|v4Jorn9vYQqIWnTl-osFN>^N|aFgedezf%9<&hVWSaMAfz zQ!`a9ADq!&&^(ql^TeqRr~2)$wXXf6NoUM3QN?xv?R{X3+b(p<$Y8ZHdVje?F3cvK z%>~ifuyYj~S^}-Ee%+z6dsr4DGVPn(P}9@SHPzGI*tgmpNIXP}M<p0W74_FH&FcUH zMYmQM;a(4yVl_PiJ&&S&6Cn0SWz~jSAOc_F>&LqHkajaS{}J@M9v!Cs+xrR=IQ{1; zDn8q{(Q|WFE;_8FZH7x^`SK?%js;yMB;Ye-7kM;PZ-a$S<S_d9N#`S%wDa!Q5i0^9 z<5(a030VO+!MxSf;lcJHk>ikIW)MTba{-fEx2@d!a%{=bOO^z?NMLl9?F4eYx6_M` zmFMASRLjoGulIL8g}DOib+qPnpPxKkT?>h731dej*U<fU6I*68<S?Dz16V7uz3(j6 zfDMD(Nb)ga6*(Fp!(<pV^CY}I_&ps{QpxSSH_dMifJmz8L`B<XIKR02!tXEXt#5w= z<27q2Okg3S>U<Sh1lNIOkJudRk9tVF+Peo}PK`nXMcTg2YvkF<Bf#sPOcQ1Q=-a<P za7I<(bOT#@{3n7qY1{ESdq0-qgnVtLwMMZ0LG^NT;7N|ue(VtmR(b?7l-m2X1T%Y# zyxk<vb{!ddHBp{pk6y`jhiR9+J>U_WsoTPYF#^+xRVQrk$oJ!6uFc@Ekk%5I*pp?o z>=WR4;NGwtG^-xDbhX8H<VydMAy*SVZJAcq^pLtacGymyb5|~I9WAkH*`+tuqt7-u z^48e)6%RJ;m6*#zTtwwUfP2JWrx&Ccox2y@TGpg8Q!L-v5IvgFz9JGFz0uqF{qTrT zyVu(jVfxRZ((yF*u;|~mg)Qqb?rMD}rG0L*BMd3sXsdz8si22Bsa_S+dNlv9bY62F zRvHr%GbA+mD{kn>DR-#Ud4K7ZZk}bDk6!2z)MVcRa8r&gOSdcSTDEE{9b4%4<m{u2 z)Lt|5q8B{snnE^*tSg<?vqs9C0GLo#2Y~4OoGB%v%US}Fr-9ak>QBu%QlL)B*9A%b zJ&oQCeU(@w4Z?O;Y0ixwb)VV8ACdTaw|2y-38zzn_RsStYp$Wq983xte2;qzeSo|^ z0?*PM64sB_?`iWO&Da-gN8?Kh1B_YP!VlI7ZM69#xJ@VbGiV0(HO7Ry;9cuxd7^AJ z^$!Vlb3O%JG^>$IMZ;WHhnWs!HpeUBhWg}s-nI<USS1RO_3gEmZTr=wzcn?Q+q15a zJo=adSoump?2Tw(x)(7V*r&}ngyN5ay0$fpUzyX+NB)XQV3*1juFH5I$T(5zinKRN z5`VgipDoW^1%Sls`|(L{({LHnPh9`ztV6Ne@K%67(!8W@t1>}6#(Cwou@WuXVtX!^ zbhaD%9chWLvZ~s|TLZkFnNB}e6TOG^_Dnve!!%jBZS7dQa|#(cm_7M+CvV?-VlQ}v zyu7ZS@iXK8HzlhmQtGPD?NzS}{&Q_=4j~i4Z1U$rYLevV5cl^o72VT7U1F?P$oP`v zvXUd%%6%lW9-tvtowll|PPD%8Ih!7BH^2UUnA5Ur&$e4jO|toxx)4XI&ZqAF67yh$ z7-RKqu)=?C3)z$Za+jE$qFTWFz+y^umM*NPq1s2ZUAZ>!Vb9YNQR`Ma<68Vp^VH|P zf`E#vNdi|aJd)<02?^z`CArJQrVPfDiOeI0aOtX9_?Itb0I;XirnHb-@1pueH&vwT z%DX{}qYn?i#TS79ycq>0UD+HE3}6st*3~WgtbFT3<!U?e1OoSQ))PhN$5J}=)zPjM z^%9=AT*v@kzQAW$Hz(;t@9Gh!l4v%w7jZnQt5UNj-9d#W_SQs|sGA1@*v-8;Iz(`6 z2jEpB(cfZ3H(op)=VEhdhLZ2d>RBH&L#k=)T_iEa&P6de_=zS&9-|xDWe@*C#EZCK z80Suw>~wE$jySh*!BOrfll!}%3IhIG!>1J(-^E}d;YoR<V;2f4aKYqaK^nhME(Ic# zc>%uV#%O`abU?6#vuxXxo|((|)PR0|*Zw0U#E`Ju#jCNidvvk{1FPk?#*UU0jVih_ zoHANy)$vh`MU#G)(|+1rc#6y3#FQpt?~tC3YaP~QRUes6I~;zo%z$+I6j)b`b-G2J zJ4MXBgsO1|EXdgKhMh-@<T*Erj{3N@-|)ndnfOy1=ysaDuPtf?Z*vc5CKx5^EO$mm z;a*No0u#e8RnOY&agDH=hH>kt_lTh#Hz`<}&judeIEDkpRA|1obzl_R5jt&pmcQCO z{HjR0l1%lbIBvQ$H>OJaZ@><dDWE%MTb|YBLJj)$*n^CDZHw_Np6G_Z#Uu;%H;x`x zA~h0u{9&V!s&fD~@_pLcGw;0{rd4LbJaVf(2}!2giq1Wqsmn7UJE#s!a9w$_zr^W# z_HI-ie5m2~%@D;*r+-j&|Cj&5JBKQK7rTsNrJ)DivZerY8bL4X8mCkcchw4aLiU~3 z#$K1>gK7k`?4AeC1&%W!unxY>{0nr+WVbL;X0bU|Q!g3T1uO`lX}(TxrkyK&*F1a4 zU1%q3+1kOs<4fp>3FjhW1<<4PK8qXajfTE`MSiP(R@(Kfg;D!AHFhsasKP0j3ucZx z&xh9xI*k)W@v%o7pkM7Gd7vX)V0hpwJbVjqy^NNFE^q}`h@5R|=dw7ChD^cfU|aq7 zsHdPSJsz!1@vJ=32FW{M5T;?{%k_R9b(FCS^VvR{bFLwMbs<o1jnOUVZ1SG;_(r(y z3Qc5jPSO?be$Ht+(TC46tjpzG?<`xkenvf3GwR#7$%koI6wFq}gsY{nIYA7(j9fVW zE=4H#0}Fv^kS_2k)!PJA+rc+N%~=cT3fY3`KrRmb8SvHfrZ(Qc>s-2wtI(8jbzkJ> zF5qOk$~KU(dg~S&YKMzNCz7es8qL7*%;`Lh(Y%Z0O@`llV>puJA}vBgp@YvQqq1-` z6MGf)KcRdjyvtXjW>1)trIsbP5}yY{(<x#P)qh-sqU_g-+SMA2ir01bZFCU2(Ocq- z#k+01DM)<=sGyEYu&nb+jb`R%4)$R~Nf5r23}j=6iD@dQ7c~pOA8+c<C9vfN%yU<P zce%qi>_^1Yrpa}5eKHXv>K{b-UxefJobo2z!P4VAx+OW0h%7a3?0BSwR2mk|h~`JJ zGc>T>_nI<L4FXJC*`2%1VF<r+$oX>qx-~Qh3WF~A`f8++GWi@?GYA5CKQRm7=lLQQ zPs5%jeT}TvSNL+*uU32E?sdxj0J{Cb&J$!*G55$&TJx{qp7hXHTce$BOl~&HD(vC6 z_VOwGjeoSx{CeEKgy7BfTp^~2UO&lG$7P}3I7RT;E0%{xQrDOIq0f%#iX(3I6~4V8 zUs!ER&`GQSDNiQqqKpdjjctA>eYa4|nib_nB~^(kwR4|A<(f;uklj}eV^dx$TJVIG zl)!z-slD)%Als}3R=|WoFy_{ki(U(m<2XeV;fJ32kOxXlgsrzlm$Wqu<BAPIqQb&C zn-ew%o)tCvIm{jv_6NUbr?r8*OOGLbSMQqW*=<;!V7l>n^#N$y-VEEvo0|07=%Ngw z22_9c*xip-w3(Kb;K~l;S_vb#C(*Hn5REXDQh>N}AbxZK@tVY7#y}MFw!O*mgJi6! zp-SlAvLDY9p}r<Eq(@o?bezBjGgF54SRTox8c}7(Whr?Q<|XR?_+$`4R_-Lrms7_s z=<FGiQJiB=UR}z}=avSz)C`1p9IIaA(1l&RG^K(XYf~kN-aew`&|+G9SNldwoVf91 z@CQ<=gKI9bI)2K;>VX(oKi#2pFz2M3GhAMxGUp4ez@fx8_%0`F*olCT4<8a>fpaqQ zkTf4eO&A>tDf&!3#&FJ=9q0x<<K09!uxb}w8k6GOVvFT-Cc!n!5|3EsZnfYH<@-9| zR8ub_%x9tFd<%8ynb^<I*b1|R(PqP<jwTD5*Q_QNEm%<3g8RtScP+TXqf<IPT$1xD zMW+t3pqIkv6kw(ijDZOH`Xz`)hf{^V+T7c7ktV7U0v5JOv+sk0ugeK;iE9H|gM9$C zK|Rp9Tw*vNOMdr)8gc1Xxj<rK<PFEqP_sRp-|kc5)?WcbsuuWtAFF<P*NhFbbZ%5M zWuRH}i%kw)dN?aIfo*KdWYa4N`4u{wdY{?O+on%_^c(R`Z9N<AVgaGwZkp@HL{J`% zEZ*d&^Xo+935>PPYt{XeQBLcLM;7s5I{$mS5<kD^NAEIViZrJaD{CxzuIbcvXt=qe z7@+2qa?Iy=IyB`OpMKKFdAe||{6p^Qvc7-^0^+XpXx=h~=6TcG4pO6p;RF!&Q}X%O z)-!H=#u;&mPY;j1TJ+;^%-VwU!c5y+!^+&0dHx{2_)sC(=XTQM0B@k0owO-9@DUEp zVuTvHXs)CXOvEJP=IZ_T`W73x&h3qyF|GhhGG$fCp=Ga6)z|9i49mf2*QfJF=L-ka ztfH7P;`or>L#{*<$=K)pD9f-fs1OVmX@u;OXDsL(%;@k*TqzK{`aXKSXQcj}G4X@O z1gbp8=y~awHdi7E)k6{%+{x(FbVIXD8;Ha`kQ%ASWzve-Tv;4ZN)b}oIg=LBd&cUx z$7SMBY|cs7xGepu`ieN(HPJ3U-h-^=+;TD~UQ>@%A{OU|@+3?&9qzBTk#r<sFjk=e zjsx?lfK^c;t_MNYXRnf-0>r1nE~q#$SM+bb^Xfwz6-`s0ghC9NC8v63_$?goGxtew zTFO_Ek+^gwYyTep4+7p79>4ZjyY@;H{6w}(Qj0-F3oZzCKlr2Aa=YfTr7606(<L%N zC!p)#z3dDw0{Pr>%Kn1vbgoSobH_3Jg%oABZRhI=`kxd#lE*>DlZhi*eL59cZG7pC z`|?z{uCGKlMNt-uG^|GJ56c|y)&9gE*xME2lYr5>?qtkxsL=al?*fZFPuTj?WeA-? z5!vnNZT$pwS&z){y7BXy@u>I|L-<K=s6UT}>x<wnxVNptEC#M!Vi`Q!*sx?-K6uyc zG8v%Sv<C`|81+ufbg|&)yfuhR+I!~K)A?~E1*kj5oW-ZwIcrs0h<>c1RKKns>p&CR z-%^HIsUjqurq9cXTMe7TM-n;54k$nJ@<4L`6j;=(|D}NBgcPq(3eIJG-Wl;2`eXP; zpUZ`zEONPVHyGmfq%TsrME++XC<P92ROBb(EMq)(yhk+}@`<I3-et=!A^_xJF%59D zuLM^}$#ZYXE#&$+BD0kRenMNJS)+W$-4vK|4Pvrdl5WGR?rH8S^>E{}hQXi$Zz+)S zo8GPqH`#-s5!AP*)L^WrGHdyBoOfSrja4Ok7+n69>j>W!*J-Zwj;gs(n68G>(qr?a z42Lxfh@Vi}W_~!;uv2;XyYi{S;CthX>af|jJWUI%y533ksKp}z_Lb9)^f2PO>=V`j zuH9w`$f5uAE9&q<JW_lfrwc?e?u$k@DMC{?PM=%+@Zhf9GHI-OugjGVIHn{OC=awI ziXp&#0lm%&n3_%r$&S=YaicwVf<SgVoWz%}uf3?tht{e__b#1;#lJPqQ*X!^PY|qT zc^t#Vecb>Z+#a$?;`oijvkGjMtWduJVzvc|9~i7D6K44P*XbV@E9YWK+6f+#X1;h$ z__vf8am#Ph9A2M6w@bs`%1Cg)GN!DDx$_6Y{}xjz30Z#RyA4d^h0HC~Zi~0;2@D#x zBGHa0BiP^42IfoPh(8F6yw1T1g?7eD+8Zx99qH+o57ozttY|=21a<n(rEY9<@nd<8 z={J4UJmi|;L}}t)s|zHWdiMCO@;6YGlEF+5*HoJ^Z#wVZv}y#cUm%=5vvms#btC+K zVHRoE&q|{HExhOA{EEf)YSC9$HVx`YErfSo7Tf6+ofh)%JS(E!#jmAZu5I$eJck`I zZC?7VT+hOM6z%DdCCil|CD?F7q{P%<m;1>#!f$vRenng?qK;kUoA!+Syb2;{rHr(G zO3RTlvcg8h?J*}5R`_QQZv7_&2ICIffWn#Rf2*V&DUwUPKMEU?Ou9Vvd#7@g41~cf z91xIlJNHc!R)eJ1^+td<ip#7J6D9e%%9w9JC_wx&iy&?gn>7UL0FivISa?-6CEwC} z@<l8{DY5*lX#nQFsYmRyez<OC>Z;=z@DH&7(6=L|BF#ue5Tz@BH}C)-dsFwyXH$NJ z5wEDHJP&3dvom_`ZypIiYaiKxCn*gwU#wO7`*A2d(E_kRlT*J*!b-c!ql+2q&sM(; zeouEHnZuXQ<ps>tkXvC0eZG`4_*Zzjev?z-dPo#>4A6AhJh^gN&q#u*kNZDa$TX{B zK#JxTjPtUVMGq;40>3c5j3O0MprrB`nsmhqH{&#o#eGi?l@LvOE(e3MJr=P@mfg&@ zjAu=PYW_Igt<m5^3GsDvHoCa($ic<OD!D0=KOjut-ld-&IL{Acn_b3n#1F{YPAeUI zz`|)PvHs<^>yrzU21?cI^#(J03%72v(VyuU6ljY3FbHC;#nI|GD+I>F;3QxH6YQE4 ze2xRzwY>$NH~J!DP%sKK)cFK1nLBVtGM%o;^Q7+`dnCVgmc%dUmFPP{zeJx8L>a}} znnZ@<5sc@?KkVcX$u`ee4DuR#TprbQ4oGv(p@Etc9_c+gN=>R29|_8WoErY1o{!gD z_1UcU`_%<fGMeQ2W@1}ZojS5-Jfs$3COXh97Li_`FtF1pvK6bDraCoq-W_q>>EKF- zU3F)da(<usy!Db4@pd4?rR}=r*n9Vpj?{i1ZnI#yLoYk_!A=hOei!R<!5L~n!&Z@| zx{WA9<Lb|1Q+3)iq_=uRT*#U(BH`+h|IB}#qOyF9!*~B03rttMOh68n4b?yVbe(r( z5rpxob@QR{V<wd<jcv2KOn7O3DsU;a4V6X=|51Iy|8-{R81lh`2V3&tGuvN)D3`>d z&=uM1b~vghBYv4n6+}KNP=e_6VM!yS&ohnka4FBP-YNF_;O9yk>kFFIW2H>YmBC+b z8Yx%w?N8*1g-Ol>@aZpai^TM-zL2kBRA~O=Rr+v-nTw+XUYPX@*LgD7MyWk;2_r_= zo$SGyvM*2#jNhEw%KEt;3WM-p$m-0`f*knh9tYng{c(X3Pb1vGAR8zy=OX=H5b1m@ zE?k0HhiLYXr_r(B6^DZx&Y)SN9-8LNUYjgP@AV1b)J;m#2bG*V85`caZIuEBay8Z& zP8egk)d7+_+u^WnVaAA8y7qGiFjrie-{qV&>>+(sZWZi@N0qPzKNT>@%ZyR`Yhb}K zS0bk;6HU%ZvHV8A<+2oC3EStNV3lj!>C;ibfZas~OzN6bvyIPILO>XNNUd3gKneG6 zQoLsMj*+t{UV4jFSY^(?oUvd~pq4Uw4$3OIK6@Lv-pO)+(@h#Zo6n8z5DNykh6R;t zjtb>`@7@;vA~(5Y>P!06XdbfC6=8BKkmG=OC{FH~g*ItjOKYJMSrYt3A*SyvgG`i3 zd}9c`gZg2T{$!cFio|zZe~IOCr^BxK7Td0ss8(4NW1j#S@Ng_S<iad{RA(PJ@8|VQ zi{iE3VTN49yG!QVpolQ94|yhTTP%U)t9!m_F8()#ru(b|Nz->ZBSslbA67XdWb$(y z7^GX{$hjuu9H)YVDpm*l`om@p?wL@b2fQx<58IMnryDZcPco~Lsj7lLuJ6<X8V5zQ zc`<vOf`Y${F(o?k7l6k7|B`q*I<EZx^b%0K#=VBUsvy2f8_Ffhp0nqnj+%`YfwueQ z-L~+`Bv#v(VPzi@?@&p@%^{|i)&-H0dEx**T05t<{~eTXaRP)d{*Yt5;@P+!!?__2 z6J=72Woajz%fwW`$GH9meB%>dt1$jpEEy}aC3J3jPI@<M80x=kfSZF)>b+j;$N(y| zPrnm~xk0RYH0bmrWSwUg<nO%#ye6$Yu3;BZ%8f+^nF%XV@J?iL^_?nNtL15JY^tKG zI1dN&<&XZ5t#y%xC#1Vj3v4-s-}PzE?8Y-K6HUNq_~3pQI}b@ffnj208nv4j!N9E! z0qMu1=D!+b`dCkr>{=n0>gL5>GbFJHs0!j@5s<IwBkm>fI?^rs$q7P^2>9VyYX>5= zUfrM2W^%6Zt=+72wH%aFUrFM*9v!o0C?LT%3&Ml*rr)%Z%pFF*I|>`+bB9T2vtBSj z`Lk+Df~QXkJ}oAS4#mmu-Qye!Kq=|tP+Bl@(_Bzq7LQQC_)JXBFa1lQKKDc+GGx)U ztJqOA)S2|@tA(nUQ<UblznO)i3ks8<n?(Uvi+0fQwOj!=FeO@Rb!lauVRxe8dEm!f zKEsyqQfALX*e5YA@!H;(1f+Mra&Wiy%#K8wNkFhJ;Em}nZiYEqK#w?N5)K=<oC9Q? zVY5<v!^MOSv-?8sZ2)ZUpNm#S^lINtQh#OLTO28WGsHC^b$3f<tJ7VcM}zsd^Y+lH ziP9U9llhGlr+eBBVdJudS^K=7MFCPe%S&olcrV(hRlr-T<OAu6h>4-{o48q-DYgsN zG}=Jbum+DgyzI39JuyMXN~4yc!>E+sN|Wrv>F?XPJ;q~)z;eqr7V4{ApYDR_S<5IY zF2hZPIo6E#py}){#ry>Kced30pET^>l<{eZB=!`M+(fQ&D-XdqZYyUgM=JL!6FXo3 zHosZ4WEBXyT6lGpgeEJ(vO$fY?-Xf+KM)iRA>DTS8i5P?akWn`tE;KmnCO`zfr3>5 z<#O+iW*yGQGz1e=nDAO%;mHk5SSv&{;A4Cn+!H^a%Rxh*dPi2?Rc1JH3dW!Y=tdF} z(@MKzm0hc@yiL7|gmPGIOtIc%mFko#`aPmxIqN$sE+XKoQJK;jUOXtm7+!VVNpH!6 zGB01e=G=ojhKLb=kZ<0DV<Dr9cwfL?e$ojz(v;**3LSQgsck+p9swoZ(YuJc2S0sk zW9=JSR$)}e3B`XBvwwEMXywZ}r-Arq`H3<H7j<Y)otep3PTKDAb5<|7I>WQOAX@8D zvo%>fs-(nVlFHg`UZmj?*l6P2WRE_*_-xdAZ~q1)Su=&lU&5KPqnW^~i#Rkc#PxD+ zliJ-M3dJ;o%(~|^*oP|AQz3;fS95zpn6}vrbTfqiLwj8gmvX!>W6uZ6r6I4B*5>;8 ztX{9f!|YCHd6iBl1bnzQBk}R>Tqz4lOaY@^P2>J??<b^ik=M-osW$PGOW=^e1ZCtX z&}O5uk0In<PiOnD5Fyy1-qSv{df4&x)Lg`BNP46MX*e`rUim)em!+2Y@xZA`cSqM% z^onNntxE%brr-%0uPu}XW}9o1^5bNYyoq}4m2{@bwb+K=`fs~xcEirBH>%f75v`FO z6UxNe=xR(0lH+Q&<@nxNe%FFfL1_P;MdHf0ZIY%!_0~?YHp9fNdPr}WOT<U_e3M(= zZou&^MR{TcW8x*v7@PHCN<B@-+xB3l#%8n>H-IQfR@ryD_@C^G@NWgL9g`TRb=wnG zgX1OKk_C-Jv4&qfzRnF`zVSF7JTBx;uJ1bC71nd+Vde}qmG?4e3bgG*y=<)MDiYgc z5LJ(T@drDyF!hx7ap`b<23Rr2=u@aVL%?W;`f60%d*PI+AI6{cB~SifWF7dg(Z^J< zWxLo_#L7;_sAk>ZXbu&TntBi1uwxhvKOs?&%-d#qM=rk|t)Z~^=gAW~dJ@DAw0P|! z;Vb#*u;Eg9M5mL>tckvJ&RCC8VO}GT{dNZwn8s#2m||G81NK!>W^$iK{iUA57K>AB zDB>G^^1ds?>=ZazFH2mB_Cy55Zg9gyX%i#-*N;2N4b$zdG~ZMCXoszV3-eyqJ|q%| zu@cydRoXP<Pca&oi01?cV|10!w!?*^CCapyzrpEd(;hb{=B@~Mg9L&$s8&7nSS_w9 zgx&U>Eb@Wbega`rAy2c;h{CJkg3;fhk)Tt1H~1v9rW#75Mt_FWO7*uOeF2PTKCQI1 zHwe8}fqD+g58Pl^j9Mvd&615QURtr!W?6=CaR~aQS__1l;HR^XFA1h*xj;pbO3AX4 z<2E%>DkE9Cp;)Z=slfeX#~L=!PeZD^o=L9{3GMr&8B4lsO_4@G=cGx+>bh1ny6naJ zBlK4cA5QtG2@qsZm}#|2r?hZA-6)X`VCJnpR5HqyI6GwRg>V=|7X@dHuiM_ozkI!* zOyF1x43}Rv93NY4+%4J0b<*51c)y%ApC0uz7nSM@+THBVIRVCY{m~2m=;Ma}t^@># zbAlSO8ogwCt=L0)?Xk}XT_+7rikzAmOq!G$Ik@|3PPy+yT}d|av}n~(Wbe)&B`<#@ znPjyYV}uVtMPs>^X*C$cOdeUA>BAG>z^(JO2%YTu8OW0R3Jm54Uub9ZkIkhJ{5?Ml zr82*fqvM#2>r&r#(nbxOx1?3j+!F<DbMs<}let`syiR@4jC%li?^vV@$55RA*oa%i ziJ1ixO?|ugD?xt4f4X8f`C%0P%W>tn)JLq?vd3JRNUa|28Oqtp9S;uJN|<}MO}vkM z+}`Ly6e3gJ731>0mh@zt^f{yKDPH)N2CY%H)8|uv2^tnRvp+UPv;GOvNal4X;9XGm zlHhN{Lk7dth3p1q|MP84D}fL9#Pm4Sd*?U4?nOpypb~>IFAVMgy<?UPh0d7$5@=nL zw@kz3Q=@`fT~PZ#48VE{eOSaQ;(izSaUCCP)cyD-lL%idk`Bw4UPC33z9_FNxH(zV zLpUw41(zU{8(Mv^ek~$TXRr%71EffHOT&jIEA;T|Fax8VX2udHtTkLBjibB*-pVg) zx1KXv0bA!HgY2rO`u|TvX7Jx5vTnJ49t`wE08W$Ue$zW^56*%qvX=eZ$lj=DhP*ax zl+{0cL)1|1o9O>X-Ft^Mm925x7Dn(WouPyhLXgOaASGa^QY0u%MWurR(go=PfvBJa zA%uX^i;5CaihxLwqS6u|AiW9-A%KJyY9PS7GiRK0edo;g_j|qnaY0CSlD*en>sj}6 z{}9f3VcE2mk(;WN8rRBRZ;8|Cz3NEv@LFhZg}tPb+i)H(v!!DyLD{vX#&4A=^6_hh zPkoosF<7WpieIF`f(%00aL08w)1~lZ{o-nkS7u;Qa;aITrfUium+LnqJm0qTHj7FQ znIM;`k-OmgpZ!OAoNsDf=l?PL%~>g%?VCZ?EigwxOk!4Sf%SxphJQr8PS=XEQ!ms2 zA8$s1)Vii;y4+qx3>uD76_MW@V;TX1;ifaSG{)XB1TCR~)^5uU8XWkxT0YX2SF)?v z#*C5t8$}Z0+snQ34^j@wVUI_00X@zZQG3!*^6W8d=SfO*QmbUK#=1q8iKHO@rrBpQ ztn_g})JO=OmQ(!OD$heRc6ISs9>XKxRN<OS;RnP$fys{M#+qG(e5X$0*FB!|#VyG# z|9R!Z|EPq@k`7c*A12B+->66PJ$qzU7Sa5z+wPah%8&0KcIbY6$$w40|L^Lk{@*&7 zc|AVt<h5VLpMP^8rKhP5Q^o&d+N$9J7I@zP-PKz_boKWn@!#uoC$CwB9P=-w|Bagk zK`@5<992Ll!}ZFE$Q^oI&9BMjzyEKt>Z@g+qg?mhT0pc0{AvoAahf>@J#ZI*p@S&k z!+)PF|NdcA4-LG;3gATpfPw_T$)yo$&+&qn>h|V8Y~7!a^p2{YsWSeb>Z`oK-%GT= zp4sop|GdvX{_8RSm-S}<Z=FwX4W>v@b}Hc3MzObXB9%i*qwhJX)vGhf_l9#;Z%xUh z&*)Fo)GpYxVuH5Kls1|>(YNB^$tXKatsRGW%nzseU&Zsuz|Z@-fUpYu-@+;}pmhj@ zpI;q{Xr3L6Z(JlB6)plN*l&^b^Ti(p#VlK@Z*+3LTmg%=)QYQxw4{g6WQWM_H3{X` zbxkP<ZnflZ35$k^ug<2gre~G3H)^7L>w+-#h>cNVnqrsf>6d0yoAjBkGS}S7f|7vq zlm1ubREO1Yqwpj`cl~h6aPhI&#x;pxU&D>mQo~h7m6D;@D~-*{Z&6%v>-qDUJo{9w zXTQ$zNz|A9hJSnyTgVQUp{pP8G?i@woLzOz!iUim8x5c2O=+k>#`-GYN{d{95AHGt z8}@?M0Z_}Y8l9D2rpRKVEP?#J5KER#C1w3|a0B33OU<2G*%T<vrLq3(w!r>j$d)qC z%DQ~O;^5kcrndM1NjI=@27rS53_d_eI<4@QbpUT!Pi0F&<i1;4F+ZMoz`wwue@=5F z=Nzh8*Mw$XU9A15yshHP5d?S%db7!_!w1mTeI#Y$w~c9NdUCFuzpni-xDs2pGTHQ@ zwA0>AXSC3FkA=;t`(sz<xY1jUS1j&#QLEK(6TkC`BE|*cwmduB!4r%(hOX5g5A76c zth;?{ue|#jJ9N|ZL0bFVO}}w?lgv&}mE$}7X{Qu4vLgVwY$3*2Z3OS#8Rn;Py0pg$ zi=A+0+x493Lhr4Pz@@qdE^Vl94~0G93Z}>0E_a;Dc(z(qmed*){I|d_i~D~DgNrIT z$K9p9W&zYM*DituaM`oyr~0jC1$oo8&|1`Q^NZv`*Z9?$mCWK2QDw&oHfMMoyhVqO z$H|f12jP3Zb?XEDtVpPS;9FXM#jB<ITy6M;D~on4W@*fd{uG5_mjU)I{4Fbt_1FrJ z$UN02DOA`=N9j+o9ca0@yOFAZ`f=<nOC$D{CBk{j(%!Xq6^mEuc7a0yQQAa(hdyd$ z3XtNh=@pXMNa{Ok3^-bwYk)M#{=j$)@3QfU6*~ELzN8@%(wRUHNNHHKxls}c97yGM zn`M1j-(#-<B3W&sHmp%}{2h<H5ta4H9gB6T--*-?pmzLZp?cPbT%Ty{P~4gT>C}2b zzs3Zk^SlFxb1uo4Hb~CC6^5i(mAgO!lyblm+XIy;+m=wzhdBRu6A{vDNtdU>qukm6 zZddrHf2iWcz=BAor}O)u%$+IW8sDlPk50@4oc^dlCHBy)H%f-K0khogf~j<G&5Qe& zH<hUXzqONeYP;#n(yg<rI3^k}2HHC{ODy@{H~8=fIb5DF^)siHoO^#l?OR~46Asi4 zSG+vv*{AKF1*&LWp8f{#a%DQN75vbHAIB>I>}sCc`EGEib?8K-^DbrcE6c}s%XBoe zh%@|a)=p5(mMhrJ(df<~8AuTy=)3EiEEkTg`?hqh$*tI}Bi^)3^`?Ycmrg?buOjr` z`uF`qCqOvdf7v+|wDn|a8Dw9FKcPkr4(zsC-JI!L%X<1#<6e|zH@<b)W1th)*<QDj zk-N%K6MwV?iA!)EBuOmoun*Skh$uT+I1c2ttR&~Gy?n)M+Hgc&>Rw4MU{FhG38xx# zZN2_vH~2o10tHvIY9<^oX)dK7;!Ag1b>Pid@dXo<R7iHwd?Gpva+S3am3^Xe>m~30 zpTaxf3JilB;<;LKxB1j{R{Wlj|BW>d@u6ow(9=&Xiyij22Nhfd(23?mj+Z%#rCOoK zs(D06qrz_$2dAq1I_}<PYPM((;z6ua(-JrpLIfw8r!u^Xts{Wc(}qmPygkKE{8B`+ zBjz{7^FwVxd(Za)FP-!H8bK!l@^nrxz$F2JIU1b|j{-J>aKSGJX0{T~cZvd*$7Tip zzP|F@UD`wM3t4WCwaIG5zX4s`_--0c$xwm`dcn)_W6jSuZv`KS4WA4K40i!3aQO+y zjZd0N_^T{~PD<SmEx=Bwx1CO??+Z8DQT%``noty++Eu*X(p~h(aqWfB>88!I2@xA7 z3@Cr<&E_38Yi`|^Y$&eZ{#hO>Z2mD}d!|Ns$#=xDWrqrFF0R`cdHAd{_4V&zMLK@} zTES(VH)}VgpZyt2v&&s*ud{0x)|qi|%0RfF0kfsO_X@k<BE+lzbx<U=BmEX9+0QXm zW8N6MEq<TvYTnQBoX;hF<Qh=+)SO#QKmV084Yp-STT3TbFsE#W)=Bk&{xk;5J()Uk zeLv4`iWpAzaws2KaPZC!(3Rj5A;n@WS7~PpUEOqM!n>7m;MGb}inZl0M0bDWpQ;`J z=0i3$;0GE`u9XY&&b&`+Da#Ku3-WmH`5JmAQL+VctMA3mPrIn=+4-@5w1x&gQ*=p3 z!swqqhHU_Hx_5Q~FWS>IrIL$iVqo5jxE&9FQDkQglF_XCa*V}RHU|fvE#f=$u!DFd zd2QzMOAs0LBFc>LY7rRN6bz1)ea`6X6H*`q&AJ^E#h(C;GV?5GDxKL0tD!z7CHfi5 zG!{8t({xX>I&7pj=R+bA^KvL|&ESmWjiK^l1Q!YCTF->q!6|b|mV|d(K83ZtnY{^! z+iFx(yH=}E^F`7f^2`eFqn5->q;VMFC+9#&fm?c3{xs$3PxF^X59xN0>+jcg;F_it z5aZLCxuG4mn0hZIzbBqwjMd4^b_oBI@BLag1DrX>5Jv1*j^SfwC1Hv}#)?2|_<$-K zjtONrEI$yE3;o@9kceut$SLb12_wUr^k~d1<TKaPE}W8gnq*;Gyrx98XmP0+2~{bC zz1sC%jOC=j3;Z~%Qx@`W)!jb&t&I@$r5Q1aB2Vb<Yp#lH9$8ncghD88+hfk)D;(A_ zU!?*%Znu>0u&AXg$AEq>`K_mR@63yv2i)k>1`<Q?i7w|`!<kwTFPpvD?}W3q;lMtn zF?rckju+=`-f0iBU8IoRw&(n^xz^K=2i?T01?d-jnp)Cm5YL|VzSQd2>Y>*~fAX4_ z-E$%iD*hqwlsNWUA+p)Q>oa__P}#q)4rGLt63zw0o$_VFI)k&d6QyLg!6L_(U_C$0 z)9!M7qxGJ(;Tva5;Vom#y_<xPy&jy{)m%zjoE<)W!uIFYx{*_KOh3)dKRrmGfkZ9Q z0@~c3C*!DY5y@JM)}({B-_!<Xm^ty_K$yfZlqiFY=+501o3H;hHA3Is(lLo`nR5Ow z*)kxX8T1}-Vd3H*vle?#2AkG-(mhAsI6A|TyuX7!V|Y1(a1MP&L5}>OgZEMGbKF=< zm~?jZbN&k<0MtwTx}0y3JbD%vB%K<snQ}EbT?<s<-OUVIIt{o-e|DrRw~z7M!|*%z zP{enSci&~vh|=fGavutCJpV2}od3Cl>x0x&mDh7Tr#5-8Kt!FfVA+)@;clV~c~Mg$ zG26`U74Jfy^nN@vRm&;ls&c3dh*ZXn5qfYbtEKk>Dv0wS5CJe(!#gQMFzbaR^Ega? z0W<Don8?LX6Rcu?+rbM4dUis;k2z=Z-|{btO{luVU7DDo$1&}fP2_=7rSrw!Wkyy& zHkN;Iqwf}+LflGDj)Z`|=?nNuf$+yX*$$pV6E8+T^P=fy6v%FbBd@F+jh!{%*C6$5 zxd}}yG%iB?o2})tUI&;j^!_d-hue$S3B=&kEub8V1n=x;SHblA?go<>Fp#A1sP9AA z&A72%p=!GG!iWV{|4h1q4}JFPB$MMx5G>fpv;wZ(<}VFe0=iRnd*`hD^0@sPx%SH1 zL5b+#Si&~f1()J6y3V@7qsjB!R<^~~j?}2+z{z-F(EtrTtXaBL`|x7TNb)HJukqBD z%%cK}y*JRE$HFVU4o~VHMlf{`;7=6~JYdi)%Ez0Bts>jv&T4#4)49vRcenyan<BI> zOZ`lA+mMI34<Dh0oZk2Oj+6*BR-0&YFd-YC#vpQmq$cf!YLxEeiUNVB`SH@;YlHE! ziIvOWk{~*sYb=@-5Gk!c4{rYUap<+j%I_8xl0M#$fCR#B`^Zz<c|`-b#_s)SEqyNR zs`4ax7e&>lY@5h94(S2qn(mX49A4}4IX_!aL5X%l?9F3ig(34-G?^VSv}>sAn}L9D zi_P;(PW$!J-F@0Ow`A!HPqmol)gNfIvcnzE;8DUZc@<*sP2LaKsM&ZGRk;2*aimsq zpE((7mN{i`{nk3gBqTUjNh9Sc@8V#|8m*q7+4HudjJ*db@%nsFn;Csp@<Z{?lmyS= zow3@IziS`UU-m@+XX>w6$O_O!rW*-lS%kTNmjd&MVq~zkoVILnkK83TXRbMmmgdPm zzWbGD5Vi49=1oh%BNe&~j?i1xuQ||^T8_yFApXd9O-_GPAF{V85QW>ZnR#0^^Oa)G z65lb|JYG?yws34XHi*-at`gTza(4#cjuN0l>Xw++a8iqZ%NvEwqAq#zDyvlpVx-c{ zE0{qc3OV!q2C?2cEE|1B5gBEZY`0LJ+=+_0)bo8{%g}*xbx=Oq-Q^=(WxF}v5&mWF zN$-XGinQLut|@5mUGAo+MHCtrKIRNL$8&^9l6QF|Q|(7XVq%Z(z-jt2R<MZ+ZVCBX zh)K|u3>;pOE=S;NeyU*C>y4E07${GcYkK@hV>iB^sRJOzV!tQzOweZm0Dkp1r6;t2 zna<NMl|_?wpxwkuq6i>^o-_3Q@Hi^@D)jzT^zW0n1o{ES0eUhw-k5gYM3UBmB=ihk zxaM+FOCOWyD{<(7BQd!*kmn;0KLkXwJeG;37TEl#<oQWr4?i>cRKgz~`h<Dz6XLfH zHTsu`YBz;*F1C)|t7&G@PqUr@UIvQ+gD`s3h!Rp8^0RKlcJ}mj%nMDQCuYRLWQ-wO zo+AB;&yCam=OR{R9arw}o@Nt{*cur@&1dWY!si8K=yCaUW?}KF{Zm>5%gx#(4+Al> zxoHRe*_+~zM=Ecy`X!WF#-&>5sq&LI_VaSro2@w0&C!VTR!z|o39XB14P&XfYsxXc zL;?;`Y-?-p)_~6|KnGoXv|ePIvYp?+yc{ywm`l%S=;I7sgV}cUm)2ZC(yw#gY7pdJ znLJg(ns&)&VJaF{8Y=9QQ(e-bz5#qaAZ`7vTxQeBiXtv^D5&Vf5?$n63M$17V-Vb~ zw%4%IA}Zua`FJ7w2Hdi+Uw`jLGu)5&m>-{hq*K(ciJU(@X}4uabjxyan|AAMXiJ-i zHs;M&(*p<Y9k^hid)3r*W5VcHl_p4X-&~=)3j7z8)C%;KvomD7?}mQTxsi3L=Vd>W zoqWx7CQGK9r{8<v>_GI>UeL#k(u{9_pRyJ(WH-`5FE005>02Lm?95nVd+Pe;kvt+} z&?7B5@48AmUSQx1*XKNsgr@h}hpsqVIont`+Ph16-B?oZ#(cj6NLm@HmDS{)A#u|0 zmuv&``JK9BHKJonGv*h)iZu=r-3D#};YTmd<%N|G%kAa4F~5h9ft|X9<9bzL*VtkG z_rjICF?M}t2fb&?voijrQVpMOzN$YaJYOcn14Ep?Si-Kod_NAI$mTK~K0GvA7X)w2 z{;rQ2iW~akx}hF~QPGt~^q)v!#ynXen$54u9^x#J1^X+R)8_Qtnn2Z5DZ#?_BhjfV z-s3Uul=8%D2;GxhG`*~8%I~>0;0uYpw^5RndMb~rQOT{;RU~m3=!0@N<`>;GInmVp zb}X@hx~$f7zY}rWeM$do0i-&o&R%mX))!z>PHdJr(hu*hVlJ~`GR|N58X*m093@E` zdU|SViSj)<ZG>HM0k*28TzS=<Jk1jW{11{$g&3dIID}pv9o;$=1hs5Ru@5|hl;}Pc z9y5=zrLUf}r)YjOlL8fm0Lut8WS1t3%HK$=r19wlTf<a)%pD$!^t9P*2!}d}KVz5r zUDq&VUNXBgSEr#VHkV;v43z|&2EatkgfW&uzm!$TOTSo`*r*ox1BOW3HAfBKwpS&J zY735L70SeLPahGVP6{LLwSzZ=2{}u^N<c3B#Avyti{sT)mv<Vtw@=ZLl{!O0ebseO zeOyFwv+pOSchq;gqnSauoeby0E3PppUU2kV(2Qs>$-}FBFy~gWI-r)NpL3u;4=Rr# zg@ngxec1vo;qYxEmA0Fzbl!<rLRe~J;D=Na6fFR;T;QC%HKE6s<Yr=)`Z&hw&2Nf^ z8D*CjL>m`*ur$Iko%Wz`xd|d-MToyNX-9i$+xO*PE6%F0?-P4K`a|o#0~;o{0B^RW z&IynLH^m_K$f++T<(8{g`7NWYmHV^+ugzs=xC#6k{4tyYw-4dPn%#H*5aYfU;fQ!8 zwU<!<stK+2%xI+B7khp8I__0C6UZSB&z~z`c%Sx4An-@2^uhB>fVQ@WfnPrv8YVt8 zAz2~jT!gjoJQ(G9${H4~lE{noaPNIOq4Eqq0pEnb#8#+&Q+WZWk)VS|(=@)#nHe9> z$^Q3w{#vAqYXgcfe9?VZbvpVIhXU5y)c4t8jky*y(GnI_sQ>6lb$*?U#xN5n%yAhK zOw|aDde?0<)2?ZzBrTqFP6BddHt%TH(j@YOAC=k!1(Re4dAwZ?Tkr5-AwtahIQO-! zF&=Eju74JKEOpT6A>LhutJ{>@g?9(<Lljj+zLy-|x@Ky?9f_=3(XcOih|+&O&HEN* zhMej<BuPnh!|=eUh2l!-xfH&nI1_D{#cq{r#JUPl20uGMO7rsXlf6><bkGY1S_-cG zczW*{VGGacKaOI!bDv9DKB(0s<k9Gc0T#^YJ?7-G#8xK~df=I|ZR^tD>d*zY6x7jf z;cs=1+^_n2>Y74mTb*yy9^V%;c-j7RORlL}c(tg4#+_%*OUxv@k~FGa#KV^3G<jjA zU~?5I(DX+e@*bpX8b79c(I(ZQz)Ky&?W9nKzf6$?^LhtPhR?@A{N-TF#1u9EjC%jJ zaK9?R%Ps^SjnWsK@%vz%%blDT==!lv@SsOlm31bVVRyP9?;}v*YmanOc#_AcZ`Py^ z!8ejmCm317Z$xZVcOKb>+OFD{Z#1U6uZ_EzY_R(?l1%FJQt!r<+GGIElq6)RSkZ3B z49_s)sMqcJSbYh6QE*F|t$s1oo-~QBd9Cav0(C{Xqu;A7NP)<!!+$tn{A*meCq;7+ zE6AS}zh?WVME4wlIy?HWWVr-D5Xeopc(Ky+U;?fRAAyHYr>5O4dHx3f_DB3{T>-$& zCXf5p4i$jk54Z^6bP1m0o*$$nf#AHu;vO@R0J`_EbWKVFkdEE)IG(E!;^UD6J)3w4 zRFU!B=+G(=Z0?N>exr-^g@K0e&Fb>FQ`Jpfi!>*@FYp<hmzcUZfp_o2aCIB$tY*pR z1>ch?I}%jSZBF)FPLBI7X`#O_A5z_Zo)=4-4%hZm7tnozjy}}TqZpl&=9%L4X;sKd zZ30(ms6q|=YNTRgD>ug7eNqOrn=wBr)rmQPNKUrQSX>l|JDGTx^&Km)8Tj9KAo4`J zU{sn89-35np=dBKjIObQ;f@0f8EE&(n8Z?)u+mNKZp^v1sX#{S+0t`mZTj=jc^)C} z@4o&#yhbK_yuK9W(Y`qT+krMv(A@0HmswNFY#<%#ODnTV$n7nA@2%fN;DBr_v6zO> zBUMl`#m*aaPu%sxu;{6`^z8c_qLG*Q2NWIbX#AM-aOl8f7V;zAx*pvH)tT1G-imYJ zI4f<8X+g<EyJwlJ7&ZX9DfvCJM06n88CGO!zTK@6dXYWy3Hv}LJ5gB?XU<EaW*Mlw znd8B>JajR~K1P~pt-i`qgP{v~UtG?4Z5{kDzHKvDYGF&M#wyotue>DDxV{$e=>0>V zaZm2|!|?pb06LuTdRj}#Dq*C30q-)z4cru_nXK%HTMhbv!gv28Qq9ZCm|12SXN)nM zPmVExfj`G2mhZR8?!-mNvSo#-4LUg4)e8xhwudkd^C(K=9QB#_W|;qgFkG^Jd^H_a zmakbl4(d*;lC+x6tkuA}OOFbp&p=w%mZ54}PST(SXS(cu@s#q1j)?}_U*z6XEFaeG zuf~)A47;cGE_&2^rcrk)C}}cvu7&w;j+}g;Z($j5bqYR_SvBk}uep82qCeJ<6GgsC z_Q!e=%z*WYupwYXb|#+7I$Pizfpq|+d%*_xTFMlQJw8B5iOSR->%8GGP^I!P;G7zC z2w!-(bS5wANTrNA!i*v*!E+DPNR%Xas${|=kS}-{O8nT2_JOZMZUJU5R(^9L!cFgR zhSqrXc}c&WPxh2FfcBg*$Q#j3lhWBbt$1Jai@@_+@#tqgncPY(OMx&)eX#X25uF@& zF>p5M1@9a+l4!z7M+v<Or<R^>N=y4{dl>8?<JOeuZ6IEac=hm|ZrY0I1Qgrk3Vwmt zE-tm-ON_b1$N%z*1o^Sa%IsBX4>Oe<1C=&du+&bXwf98{s6Xk4a`nyA!E}P+!1>Aa zb_H)WU%=NVzEZbBH{V@W(8BAXUX1f7GjC+`U_%2vjYOCF<sEw<@J{g0$J!1IBhKXt zF-4)+=^dkQ+`yr%)&+XQT4lB9c(~_vY$<f2vNZ&w7$l`OR3xLPQg&=E`VxSWG>aTn z&2d)|B(5Pa%Zj*?t%OM8>FsKO?|TN0SUqVz$YVtx*wNIplqW$#rH*W<&Ep>yzEu0R z^}u`!fipXSOelv$zt=o|R2#T{Y_zAl3s8-zsIFl3a&xXlW0kg7JXj+~H0F~`#gBU7 zX}`DhE-RS#!v2Q(VE^0Q8$#XgI`}ST6wlO0RLBX(mE4+?WV1aI`#c6%E>%yff8>3$ zVYuE_Iq=dJV_5%Pi17stTy`((^;GDbTC)nB{s$?^q2Aw(>yLKNmY)U;VZeQ}{+yfQ zJ4@n;pjnV3s4;jaWxWZT<CKUbJlMTVre6M`t%fS5Hv@p+Q|8A`Hw>tYpN-lT{i{@6 zYy$M}i~jdK*#o#N%k6t@^!q%HlTER~<S?>X({+qI*{11{Uq)}J*mJ{@`~vD-GgXfi zfKUC3gMr{F`drnn64q1?39l+!W2h?IvlgFv3sdJ5O;ZO`Thkj~c~o`SM%)RYJ^`c> z<c6K4aj0}Yzdfh})Ma+v!=$%Sj4fJ5Ejz%!#o0Vy8!%>5%M>x7290b&ck$68k>YeS zH*C0pWEtHFW4Je|;Rg}&(=AS0uI5*ne6-cmksCXBS>Am%t{cdEt_tSGx_D-_^Q$lC zF(wZ~l(y2$Hh~&Z>K2{zg|5Vk@|Bag$ft)fu<cbCf+h^kenV<7RM|$JRk|qFjI^_x zVD3+($|~@P1Onurc2TxFh1-Sw5OhL0md!u{a}5Ix^^S+zCFjwv?5+NuSqK{(Zqo^T zC+IXI{o*p06sGTlQj|{96;yyH)=r6x4D7&}otKkBHC@cwN(VQ8)vPJFyiJE3x+3xB z;QTTY;{?y+YD!u_NnNM9=4d7=pkx{XoX+4%*Br!GGBruE#^;`NXNvz!kW+srTAIa= zb<+2xPuB<a3w;?;hph3a2A1Zo<c1x+*&e()-04W~i>MhnjLcQP_J?X9zrxEjs~%3u z(;7v=&o5PZa2xqq!X`!C*zA@Op)0HYNUqqpaFRu+oMj@~rGyvD#GTZcGsx3|;W|tc z-pI}u$VMh>(5?_@mGV+F=@)ntwh~1r{T#6hM@sn7W>e0O)ZMMwtNy$JsOi2}Hd3d; zH%(ODT%a$&o!rph^MF;fy(DN=D{gJG4fm|2`FaQC%<kOVM+3mQuHZ{~VRNmE`sN_w ziB1Af6y@N@!iOlC!M9aJ3;4~5mG9^ZcOA`GcWKMIjlaXe?Cehh)5QOo73fQlWKAz| z+L+oJ>B9}~1Gh2kRQxen9{L42xDnhG?hUU~%OvY9N**MA?FhD?a)?sf$LQSw+LV&v zYx-**P222(U-a`*YgMw~p76>3^Ys(Bct$vA6({<^-6Z*f&{yxSkeAgZ_!32|)q6uD z3BvPPF+Jk|2Yve}4cxb>W1ODO@o-#1<e>Lp;fzoT#53_Y$ARr(XfOb@E0{s3g+(H` zOMXdi(XOhyu8tkwMb)8Tm9^$CL8XS?QVNAnmu?u?+>#&mZijL*E{)B~!a9=q*Ukx7 z`zTNpq950O%`%7D%QdU5%mlbR5Pmm2;*TRUA0F^i@OvJ>aWj}Gf`Fuw4CIL((J`ng zi}@J?cZ}g+kwPj%#>G?OwuhK)JZZ`eIr)@}<7@y!W&%$X)IM>{x|Q3^K&l#u87E%X zr$52*o}sqgZo=3|;Y?A|d(5R8yGh@rRim5r&*OqRbZ-##!=x&k(#UHNN33nKJke_J ziYUq~=Pu+{;b(4&t#KqKcmAB?hs0Bc%r2pyxmqpDkS|9{T}(d+Pt~kov>V=9lwOc{ zoGJ`;8xlxr5mtwW6+#GeUb~5xm?cM&3I08(O}$G!yo_6QeYrYvMv*;a;fj_LW`dQJ zC<B2mekk_7GA&d6@b-vKlTvu5;(&fmH}?-|c}Bve`R-<k3&p}j^X;<X@Nlkho=(!` zNCTU;g9x{V@?x9!#SyW&l$f#nT&s3xglVxA2^L8t-73us@Q>j_naFFEw{{W)i#~V# zd~lnQ$AQrIttklEqgXP|nFo2}_?1Hi>zWV~$_0;RIcQhRTkeUq6*O)=iWo??rAtlp zdx!_=B?R@yrBTWcAVllp-Qwyno&HGdjI=C*0hH9rORCKqR<2eIwd;FftN+f&_~%&z z_e;N`?t;&;SHFJmGk6*xG6Plz^@GYdj$&_<gG_(KL)dS=41J?p$Xhh`^J>CQi^k>G zGKHDz(+7t$`UdiiIXIka4UuJV(ke%{pZ2g9Y-`p?f&pbv!z6G8-KGhCw7eL@v!W_F z{qHIYV>|q$V6d^o-zY9787t3RSB)%`@SwFm0YGOoi091MhT8Eb^V|+yn-J!Y{5ph` zJ7B$b{y;~hRh01RsY8;bfK4IVi&iSXpWxNHv^L4@#~WVbdV3FI+_`*;@l*UL(I*st z7vHSFgqkh(9wY|n>`BKQ-z0XaQ)3_QIGu~6cb|HID|V?eH#BQ)E^d-nmy%8t1lAN1 zydGq~oOOzfZE~`=8$Z}Ku7h%VpI$V^^+Pv*jHvBfXrt0~x0+I)7JiH0Uhiym3AII& zkU261e)-hG_ve^hSiT^z{i7ZT!8J_wHa*!dOm7{Rk?_8%ftV9aY66XaL^_yxCE@Kz zX^MM+D;c$`+7<4mkrsIl&R&6#(NnvH5W#`#U|^gCI=wa33#v8Ml?mdcDKKJEy`lXC zy(nAf(KDzBs_Fx~wJ~$}>|%N>Vswa%bm^6?e`-mU$sdS|h%`5uP0E17CM*!t`||@U z8N1MVkolw9HqFwRfY9^u(Y0jZumXDyM4RQ$1n;1Wj^R;oGNe^=PcYN3p$&veN`CJ! z2h(Y4u|Zu3W@ZC`>`p%a(tu^jO6W#8ifbCg7~ppc^)eeU8o#>~Ip7?3w>^BbD6<Q% z_`y3>Es&s)S#D0Z+)xbAOCfNz+R6S!Ku@+A#<rCI6<6tp0*2DWTs~|nchdot7rRl0 zk<%lZt~r(S5iPjY0hcV5ka}*&d8*>-j!3z@*P!{whK<OD5jo#f6CDuKObU@c^}hW$ z!QNl}c4}6l>_?{%KC@oGy$Hiate^GAhWTxerNJ{3j^R2#{cSqq2Z02Hru+OsW<ta0 znTi%fd9BFmZpEh8pey3D@QnSG+c!Olkr4guV(ch#02Q$pv9K!^Qvtm8$~H6d_IUji zbppcHb78g1sWIMub7r1U2QHary#zHo7`?Eo=_66D3LG6OqQ24#%NW;GSeo8Un<9|K zOUOX}-Td^(NY-USMTUK5d2rF+i%jqyHvk9z_ad{jYUGDvsM)8pyi-p)L5cQLq-uM2 zadAN-7_2(WKj6W_MRn`Gzduu88r2RYWp+LKYp4HLja{&LAL%~5MwX`tT=Sd%qO-VA zxugB)?~Zlg-zJ@a8jbf~vUxEe+f0H&`1?lp*XqI&FsbEm{zWuR0a4UZ%qKt|=dZaN z>xW5CAWZXrzSO_!pnq)w+X1$<|G91cm)Gqr5V-lj@8bXSJbo2`9&Y%r+|IMwz`T)Y z{-4IXe;-vb5X$+#)5%c%gwLqm!n9lQD;qnky=-)25zsa5bALUcYw5ElwXnvplV!tN zU`em0N60Iy7u2%?eT57&XTnpD`6M)$C6%VcdHFSnDUJkq&3~j;^0su;Yjf^wX*rv< ztz52Tj8Dhg2QSn_@=d}Ng*UUtfHsHYhIIoj)a~LOOFk+STw1cQlW^+qE8)TZF#z!S z1&jUqXlpegm61T9(U%20`8F&+RLDw~KA?C$e-}sw9Am{)e%k|RiK;uhEcM#PC~(Xh ztb$2dCyS?9U~xFd&LHd3(28D5hr@v9Ko0}IpBRe-wzEg}<^ZoyYIxw`uum*q6MA5E z|Ma%Bqy0Yc`<h*VR(i*T-f?htFi*~{V<8<Md?V&<D_L+*N)V%U>1TLO2&ixe03p1G zh4iEjX!Q@Upq%qvE90O^?s^~%v%Yjl9ar7=Y$_L!8ZEG7T_;pGaG@RA-!oYbx=|on z@s#o6w7O&1ezQWPUu>-6#$qojqyUeuZJ%bnp~~%?Heb(_h8FdWJr)9UWr+oHq|_!6 zR+;c<fzj#k4QorSVOx|;G1f2J=jULg60TZ|cQg_(Vkzu_)&*_XJ4<$kqQ<tT!24A+ z1nX_4j!~5vrycv4<ZsU8%9fJI^|!CIXVRw2&)x10KYiWuUqhXpU?5>@|H?6zs3E1! zib+Ly87)qX4gI1_{p^<a0lmY{Uwgm*8M%a-4pLI6v!{S8(FK-R2<;)}X7bMX(UD5D zFhH<4j%Wkehb@QkrhRIoZ&+{U2NqbB>RLJI2EcHmtW?9%!CESO)UtN+_feLW>{xd! zMl`bsR-`rID60IEki!$nil7wya%QT3(Aa^Vw7|-%yA@$N$<np~6F}$jvz+TJ5NlMe zfk8eNn>1g}HC3~557?<s9%X$4qbz{#*vbRxA7JfsR*OC8g&ND~@2=(69Qmd96<jm6 zQL-rzJvYQb^bq0CB+L52levSLitSb(>s2Eob)0XDZ!><Bz_jKr9P{8q7m+e<b<F>X zJNaWLD5*ol{!~DVX51Miw)Kf!`yG!4v>>3z#%plKYn5s5>^A&yF}xzR?J_*NiLf-n zOg_{8O~^>59>tQ2(L{i>POsEWPdhTvc;!#htI1c>RIMc4Qc1rvo)Y=JZYvH~*Ql+u zWI+WU>1eySYKoZ){RSWQN?RCb7gC<E+8DK+_04t4^kNp|FNnkX1B-X?^Ic&B<sVr1 zno!anD*|-m&)CVO;kcQ*g>&`B=t1&6K>gJ9bpqjlcr-K_tLwu8-|W4wm<xR*<OVMB zY=&UJ1w}kc8){z=XZhn4`o6G`oCSb#+&ke0^-V=J)ow}sTrl=>-spbyYe1ihLeHfB z(?$JhA6N-cQtkSy2Icynad{lvD@{h_n6I2SqYH=-6alVLCjxQVIUWoPQn@+7QnV4; zn0vEGov@vRkXq**Rtn_K8_PP+o%aN-+tmpiZR*{4JNTL0SwHZPnfmoXvw8QSUT|U{ z>mm|j0>&4IwU7yqY8XJc#rX{A-j%w<GF#cd)^BPyRr~f{WggzC@Bft)!S-r!cDDbl z=?uHptnR3J#7~QM*;9mjLv6w2t@!B)yzq1OOi6;1T}T)ea0)K#NyeLcC?yhDl%&uh zvKXg#$vFv~=_Ut`8!kL$f547R&3DHX3{nmyW{1Z{1sT#~#0Kd@L-8x1QA30SsVpI! z87qwlSsc^D&@Ef=7jVS1*yHo1NV~$=XCvOK#_@H?PRPp~7UhA!oWY}sBBc`MwMor` zvCZyz6>|v3xcw<n%>LGiDdpUW*0VS-n0z%Dwf@%8-sq%D!j9XpWoy;v+Qkc(x>%y5 z4?u*fa%u&`{PwD?nbqS|P*QTb(2ldQ6E);R;t4aR=AinwtgG*l_5ozL@i=|-u46~v z=$~S$@HV4+){WuwT^o<tzu9%gQbS8d*(oPB80M4O+VDr_EP14Y^z(>+>8{6`qnt>{ zXorogLcEq*c*4BH@FWgIsYqLHsCI!iY~@UlfOhuWns$euo1qHW;<dXhJhBeh%dV&0 zGIQ%%pW&7(ZrIyi;0J=xrn!}qmZoMpt=R8x7RSA+HJ50@O+{*ScI{HEenx(P(963- z%*RI}U5|^qiXV0^`hgSJx{t;u+;cH#d3=JAcY38_-a)_ZjR<=A=56P4OT*NPuF79G z>xBQGnl@cmVs4Itr+&y+k>XTA=o>&>RZMvgn2ko?yU=Lu+=n9fug-KSz>eh7pAeWp zRzuU6w`zN=Zo`a<03S(KQ>hT29O;RivRoC{jai1;_G-GsRlxm#VW&&srT8vyG-z$H zKxjShXaWZ$RnALtKu@LZ0&H7ljie@YIVMm_2r*mNqU8^b&hlFe3|j{7L(4sPH`@f) zf2#1&e<}j^j}4#+a8oL-PPv;XQR99Q%1c`Rsg-#kRP;)ZR?`XekQ<-*p49$xU`V7_ z8K?`lPmCwG;+Q5#MM1|8aadtJ)NLQoRQClF2H(~QlZV|b53*!f)-l2a`}YX1_>Vlb zclwe$>5{PCglm{Sf$n)&2LTni*?T?wo|Ff<M2t0O<mNII5UHi!f9__FrAGN5g?x#W zF5YTZ9n5yE5CornHKTKI4Q}ahE8-6Jo0GUb&mERkR8UDC@iL`PV0Z~NqalO{U32&E zl_fO&w(<n|;ElO6@5G;QAe;))IZA=bYpSQB6YpB9rOmL>YJjci)UX>ct$RL@0@`5+ zXs$ii`}gv#Au)Af{S+PTOQJykWT8R>)Mal~XFmXOLNiD6X&#+CH|SMT7i_J;@W^4N z`VMqW86+#L9jcZU^pqyNbsIH5BtDI|YQMQNsI488ZN6I?Wu?(C+XZCn91ry>w5%>s z-Rge3Iq<whto+}?J+ncCteYc8ciI+4)PJzX>Dgk(35$i&T-*SM<6z$utNOk0CvWEl z4L#+X{j8RJ+o!w<YLKVs1lbr!>~hLQPrSn}6epKlr(L5|`!>8%%j`dm2-Wn^=5D&o z91{u%72xaMrrTw|8mF<-Ps4mys~h)G5D8DOy{Wh%&)d1O9O%7{DJR`dxBW}b6uP)Z zU&RW_gd43nS^i}Pv)<IG8Ggc^QjBU4K2-Z_9Bvn4=v<9$+5PXCqA`x=4yX?F((H^n z{{D@?i?RlJ`EpKY2(?*z^pE+kh4b&@-Uo2AM3?de5iroHzIUwqb%9rLzI@X8M$wh1 ztnQ&}r!vM|B+Y=x8t*xyN+QyMQu@sM>+ACp)va8x(8S*sN${qqpl`if1Mks9eTlN! zg8QaASO-oAgP5S{e3>2K#tT}{=tt2X2-V*3@F&cfvXh<E@uuvs+M)c?l7pRs*t;-v z;t`I6+EY-@TUci$cym#StXT;8cCdA-iC>RhCAa$xXlRowpvPI@J#NGmCMCinkNCZr zn}fXpdSYurR>|P^oh=0~&oZyz7!!S&s@CiD$@we^|6B$CGA^QFCq((4Nf~^abWZP` zg;_N14qsaZ<(1VpV4~$9(JmroEKzzYQY}Xwu?|Hx$pe)KXg<?NUWE7?n@lxRW<%14 z;~=7Q0wo-gJe9FW#XQ}io#7rnSPXgJbl+@XQXydOvF5u+maw%UT{Tsx<&cOTY0ef{ zp)<vjI)taPLCNLHvVnX}F%k5~x&0YoZJVza)w_>4#R}^lnby!<4Ry=06N@!Od|BFf z++@N@<orMvnivD-akfTEaF_8a?TGRTASPyWNc-nyoXv(*=weP<5^Vo-tl@l3m)kIG zYk0{szH{Iml87F(IE;GHGNdxvaE$1<lQQJHSB)m}H>FzTQhIr?p2?hRPlZ1DdK0Z8 zFI$q32<NQ%fzs5m8STyC@jH#2?e#bJ%LX61Q+(M{YAtaaf7I*OC-J9KrI@Z|K)%9m zyEd-S61;&KgH4b;gNze4{re+fZd+CoRmg;hz6kAQ>mQL$({-Vf_XLDa1$;hR(JZs~ z6sc{wC9_V!arPPS@kEgsV^HI)X)&FLvjRGU<z{A|V%~{G+~m@tWUepV<S-0BnoVQ> znWfqETDUX{%=2Pc2$)TV%QF<%dE1@xAJI~aN#?5q99@48@IwEvf^x8Vc_AKiLD@kW zVSvdmJyL}J?rx{EQHuNKqni0J{wu2gS$`8*md|vOB@8vzznXV_O2Fgcr=#5*X0rIa z=d`PtL8P9u{hxrxZ4LBsc0V|)_spFrm8&wyt3KfG2o>tS_iXi|&J;;E<tuoY@`NWH zV`6%@2`mcthbPprM((=hY#-twa>U-doYIh*-fVfE!WCysZK4OQB8M7omhW*Ua|!eX z!uChf`QF@#;k+NW<&sP+>;kfj5gr*WRrzby7$BgJ^r-fk(8R>Fe28_%;#%omb-+wY zl}B4Sl+f<~;97!vAXI>OMrOk6Wa<#bMOx21S<4`A2edzM2ysq_?Ra?6)O)x3_T$4h z#iL{LYt!(dEGy(nYEWVlq;jr3hM8<wYgk`vV6XxKpT)KthNdA2Ny$hP`a!hRyk6~V z{!r-&hJ)PD>4pwyauEf3)c>MN(;@!{oHRc3;EeYCL{9ocxiWh<A#g8N0`)WSj7Hx9 zG|4TJ&v8B-);gPOaSQ67B<FHNJG1Ys<b7hLYD4c+RK*_~%GHPHJP4@jnqEBd_cA3n zfL9^yJ-gl%@3?g>0b;?>-+gloF_j_Q5e_8ugm%GP-ioe&G2D5~Oj>f77Pl=T%OLP! zNvE7Z{zKX@$E{Q?&)-%&#CMVvceYf}DmNY9<=ad_8O1WwM*dzOu;xXu)ZHJ4zXr){ zGgsK09yc`E*-Nh1!ZPa-B0sgOX$C6vI27JF@EF1ud?v-$xnX@*_fSK^HdWaG_02K{ zsA17U%yY_{_b%u#<jk~@loG2Ui}>=TWn6?QR{^r7yEka3fw6(wd$&n~!q@^{SY$sg zuYHv&1_fVW_asNajTI_9B3teXC=7SwOX5ohyPxW;o}Er-en=(4BF_yZA4ARKjJBXS zuGI1k^|I;eI%31hY^qk8*~am8g2e6$x5k1j#H1DVYowp7y8OR2jHy?@0fjP7JA?bw zcVfNn19k&<kT$|m!sp&W^i}TQ<uaBaJt{u%b+}1@)Or1D<)Cx{VAt>L`PVB;^{yX} z{BfmsjO<#j(hLvbJ%xFjc%JG3@*4A61F|Kd@Ep?mo9AyY8#|hM6LjL;GV_(_k}%lZ zTt>^C`eVjZ<40x5H<xzL4xa}P1}L#VZ?;^?D+O&0R0<-9zZvA`qs>~a!S|gs`79c< z>eh%CzMv+gs()r{Y9qfd!_giFUfa1qyi8%&QA6o$uM9>=Rin6|67qQSDYoK@x!A%) zu>Y4O&^`vD`nX(it@oag8!Tanv<WbRs%(Ni@nQ(c24)Wbb0df7!y<ur$CZj89x(k< zz_{=xMfO2LZqvWISAa#@oo-9oaC@tfAU0CF>*xvcuI5IHLZPXGwrgI{mVM{l^HOc+ zm-IXE&Zcba;E4sF5L<6R1_kTG3PU(~LjA3oTU}?P4`tJ{qak(Eiq$TXx|H|KXK|(e zS8`U$E9~494nYmJtd>X?VMjs>3E@;CmWZUi!|KFm$I5@VtPs9}>l!jvD9r(+Ag&r^ z(l4dUjxqkgHQCrvL`1w?{$)G#Id$M_$sX}@q^>p?5-|H94cKgy7%qVA-2(6O>Y^|% zPs|s4&v(!?JUi8u72VNK;#?l{>oDwZxAAT=QnObIgu68;J>HosPN#AGY7=GpS*@kS z>q__PnYXymk6&pSjxYZ<u@e$?x4;j5=Z4NZ9ImuNf#{|B+RqQj33bk1@yiB+i{I3o z`x;wUMRcxrj@c;|)&DS-pEq(%h@=+IVc(Cd=y=MGBBaz&OKXQ6N%@4G$r9D(;d&{F z4Yt+3RGlzh{rS;2o^)ziY(2dXtdwZVUb9Tl%0ktQ9J7a^vF;S}uyS0TWbgoXiFR%6 zhh^zIiMnt6>^o1hepeo($B1Cu28roMm;N5x!T&h6M-$te5%40o0z3ikwn{`)zr7r` zP%$V^p0h(bd5qN%LKiOkg33VopfUQ@#)&L|Lt<oXgL+y#20Uw^PgFv)&2*`f<0@6t zwKCe`_Fd4+_@K?2!MYT`z!U9vT?{2QW8U|O4W2<#BzY45SbLWHXc3sfP~A1#&zJF> zZ(+n_gE@W4H*hfq&+z7g{#=OUBjqxcN5Um4F=BUYF7aD80t6l&28hFMm`?Z^I{Nd0 z*-j{7aXdDluS7G0-<^P4X|MH0dSaMU+LC);A6NJ5%cjUuUa$ITq-h|`h37U(TnB$k zZilR#AdxZW51TLL=8a0^?X?G38Wj+9e0+wtV)DNGIkDyYL+_sE&P^g$!PS!HL}ExI z>HLQkEfJ$rk;o2ucJkFp(hRCmaQmVA``j`MP0GnDpTx>|jub;)H60DrHJnUEn)IAD zn7aGIPX;u1LWgc;U44%>1CBOg`V=!$ovA&)&D}ixK`Ng`6hBNP#H^rB<n|9Tt<Ed; zD~b2tk@%dkv|E0C!>OK6BlKQ;&9?=g%J^i3RqZ5x?Aj*#0Tf}Jp-Lz=e1ZSANW#xP z%h|&EtX+!1NQSqY&;S8*Q!2!Ta#Bj&K&@ejpLPLQfSg9Ia=81Q3nqc<%LTt&;VL^- zzubDvlRH$4KbM|Dh7j~Mq3@b*tN|)+_8O_JxwIu)@!J4B^L#5uJ`+7~XqYB4m<qHz zEEUClZ_7*XuLXxc7c-;jIkkyTWWMI=d&w+S)hUIA%T`M?TujY+pI|!dHGA?p{k#bf zsJinoLRT|;Iq#ItDy)uXKX9p%KCJGoYXBW0DRsJWWTJ8;zf2oRWIvXBAM-{rJp1lY zW?{i25<8+zZG$a<5uL;24IleBCB9Tum|52;;p3POwNKn}7%nwb0U?XK#KKjEx`rBC z8THQ6XCo-05k#p*t?HJXCKFU&)GlJDB~AHP;nz-N*?0G!YHxxN36(RzZmBXH3opY5 z0My_Lz4NFnrzK}NHmpg|pH1Zg91X|v>Tvvve04RRd_m1i>RjRZ`XK)@)5f&p*t?uT z()MN>9$$NSMbeRMNpC%F#{-<qJ8GS4L$YFM;2(O1tD+ZP`jz0bMoO=nUG9W|cqI_s z7y(kA9$pjxfRAYTld4eQj3bJC@#sWxVkfHMhde-h>A-AN$c!_%#{fU;LdX+Ne8s>D z`zaaqrla94yn;>_eTcmCUYNJasvh^^?r&y{6K^^5m{NPSCZ<)X97;zn-!QG0gv~z` z);*b}gN}qmsBASWnxbyNz>nNr2(B7VpCUeBM0yT&-<v(B^7yusYlI8igGa~r&IQNi zRZKtHDpw?wQP`5?V6|I@jgcrjNOrS>98m3k?sP(iLy!^&O4J9g$5tV4xI}F9CtZ2r z!>GBd14b(Jn`UPKV6N+e<9xEI5d6oge$Fb`KK_Jp0%RFLLK`hfPt)Dml=_c}9{OM% z<2Y0l$>?;3u4d=>cUrw%mg9#IB!-&7%&g?%VRHc9twTKl19=}b9w^#K5;-lJgz4%N z5LtDL-UKp9l~A%Fj7!XEksgbPkz+J4W#m|*S)iYIhwfO6`;&Qk5_9YLK{>TKb$%?O z+h}v^s>E_Z=}JMEfJV5z(Ms0$tlLvaHi4b#r^2yOyC+^g2@R#?qg+dNe-nuS0fCN} z)dt?7iK&VGp4_|_9?jD~30K_V>o&5C&Zu*!#}uhw3m*_j<-yY9CV20nq#794H5vGZ zt^U0y$Y*7}ZU+#PMe?-K&?l<7pu8jZd?{)02#4idoi}G39Eb;V&j-35Mt6%8m~n^o z;B#pa1qhe`iMV+w!7q7p?S>k<i@;T`lX;4JHp&HNo>S$!WI3}MYq-Qf<Z}LfQim?1 zAN=D<&0GX4WVB`!Fd|Q}CCC(cW%N0{^ouh&_uc)ImzY<+TY?y}7Sas#SPlT5>?zw- z>nNKk%PH%qp=}4=-v*MoGu~nNvCFLK%YD2us1~#f+ROL~Dgn(8yol?CN_f11U=Q?4 z4;XY#De~%oK4nbPp!)ivpFg!|{_N2r{7`Ag<1WL=g9bb=>8E`oepYz`pBaAm@|;z- zNbG$94S>veRGwcpkCCI(BRrWt;sghx-t4<v^Xy`mye*G&UPI84twVmT5|QWoVsY!l z8MNst-u)$j76CDS&mU=lKe|fjKspj(87ssLR4QmxWjmneDs33*Fu7QNRwFd6*TTyn z&yo?P(tSh7F*I>$Sb9)>AE<DY4^csUdUm|BKzhVrM}?W$ArERV=j9e(jDTOFD8o#B zyVueO0L>noazPB6gu6eht<8vrwhB!Guh-U?MZd+a*Jv}T-;fms`1ZJ)elJb7K%;aC zn%ZY^_bO7DWrE^OJs!rnN9mSYM7MzcQM=Xe<<5`h{bO~S*hsLFCZoN$aHAn>361`$ zI*0+KiU8OxH7|wGknb@#F@Xjire{1DkCR*srMU3Cz$;82`qtF%F{c@soJ+xIa(2H? zpo_m6m=39V4O^k=E2<jS4#_O~U*f_el93-#7m+(J(j>3XeIj7~h;$vai3dWk#IEi8 z>lR}QGLmo%3BAx0z7*B)p=rfFrr?Y2@iTk?j`k*$-gXM%X&CvqviH^6)d~7Bh{%@+ zERcHLf`hwtz1tts>EnNJ250u_ZI)y*7s+i$bK-8U*168*r;Lp+nvv;b+AX0>qJ9fB zj5owc4}p|N7udQ4*PnXoyV@#?I;B>-kK5nEy^18`_x(SQBVh=)tO~a;zS1CIqxaxy z-@_IER^iBJj-Ki}MYVXsa7dHh?HO+Wtx$)`U$gU5?Ef@r33EqDnW!MMx4w4t32X{$ zkVBdTUYsizWjsT?DdlE}AJsLJD1Yr|Ze|t_h=)gurb4i2PAj%J*D@Jx;u2He0D3*I z|AK(24cW&ZplXz^Pi1v3#`N_`4_rm2l<FpDntHg1#&PtGosWh<H6JD0nL6w}gTDB@ z%X1?%9(n}^1Tl#=Iz+Q6aXf!5Yg(MIb_I*tuyG8?_6cl@Jv)zpKYrOQHyB-+;>E#b z8WNq47?|=ceGa8R>GYmC#Rdd`KGu`?+PLSeC4F!dX!B9=%!ekDIF3H*_&S@94CM7H zRWg@*!LDPA0}m+((aeEhcea3U$TWzG{@uh^=_P(S$b(@O#Y4TuCbJ6p{;|jDtn%-z zoGM$KmN((8qJN45$G>(?B;6~XH1VyL<tFQ@49C>cQ!BQtvzofXOL=v+A=>d%ZeGiA zLFBHZa_ZEN)5Nw5*C^}=N~}4@;ScSH39YlF>)OW0hj^6)WFe0{&z=5yndbk1C-i~9 zl{hEu!5#Up5WnOGjx3ZfVgGv}Z+v>p*mbWpM~xNteGbUUMf&yQNjfO<^@^k={_};x zB{}|A{QExF_#2o$Mu<nWJWka~l<du-u%l~mRTt!|QndbsxS#;XBMW34RrlAF8+xs= zG<tJ96LPSD7+zX>1iUb15jJ8=5wFwx0MVSjDG_9%rvBz=^zS-0=9;@hu(=%m43|~@ zhT*B%Cl(DGH9RZyG*sjy_xcyKGUAV}zCtSu&=4Nc$f+h*^Cp(r9i35$9!e%k_0|MF zSKh^MGQAa^On56R;4YjOcw7hX5Yk8Ry%^V2{W=<fXi@Wik-Ym!+XwtYdH<_!(Mmgf z(5}4iq&V;R0Fd_rzkQtEw~O$hy^N_SQ#&Bu_ay8^u9ku6^1}?}>p5SIeS}LA$nX}o zgW#3bTAyD%&6ZsMuq|Gzd5`1krT+MSTK}bfw)1p%u9@frSHh7(*SCHd49vT%JNV3u zCXdc!g04wQ^>>4#UyR4#lkg|mPCZqohevIQzhehAFXFwi9yK0ky-s?KN#%9))()P_ zPYnl^we&dVWdiY|pf&2@Z#=r7y3pgVv-+MHj!k!mWXu@7C?(|}k9+RfSWekCjvp1q zx@w>MR~2s7B}zv5%hPd2v3&Sp4Tj<lTTh!N8=wFEPTC>$WUp*gj94}UcY44qK;rX- z5Q77`>eH_g3IluH06X2=*I96taF9gc*O8P=6o63v4{iS$)zlXD@4}&jNLQ*5AP6c7 zq7*|(knX0bsDP9py^1txp@~wYgr@YOAXpFt1nHn+Na#(9R3Y@9Ku93KS@C)Ie%^8Z zo-b#Nv%We8$Xs*HdfnHrQY5=Aznpx#5MJ?#H$Sy7!kCXOgpB)~oIy~!eFKS&+gvta zqL0~Qwp0Y71ve~u`I{^AuFWT>iEivT7&8m5y8a@tS<TxUX$nN1Nlpj&tfyyYHOb#m z_F|=kT7+V(a0s?Sx0A;Qdy@`*jp5nlf~swAnIi+X`|a0u9b}GGPCKxV>0OnG3wXoW zW_6OB^U(H5{4IDxOsUmOCufB1=T-W-lQr#&zPX!&R*l@j`$6E!{IIqcP2PGBpGQ-e zZ%DwMtT4abV9f72;7E183eomW*P*;ZMj+4EEkCKm_olJ0Hn&*5_laAo-rl~U?khK# z1IspW+b%`x%&-@%M9||}v`%PJ!%Bv`@))b)aVbY|)csdVaI;PkyW^SQRa9l})UGFs z-sHGYQXz4>qUML_!TMTco9V@&YQKb3p5a&U$#nr+SFofeWvE5901h9#YuKSyxz-@I z0%SZ=<RTSr%VK$yKXFh*Xj5fuopa1GWT#y7%1P}3&?TIL#n^8jCh~=nN+ye|C2fnf zb&`tcF*jPGIU)h?W%T$L9Vs$*VXO+r^g3+=Kd?ERx=3jx>f!#~LYAHU8yP`XznG+n zxoRJ|AmtqI7{5?1S&pvrLU*pp_cXa~xq=$XAGjk$7T>*Fs|ek2V9yksDR_`ujnfwC zd>6Qyr|f+}*GD+{HJ9)eEr`Y3q+f!4f_Z|y-9X0Kc4aZO<rnQ9Q5VnZ(p^K*D5QI0 zSTauu`#ed=6VSEZnN!*PSz3LldCMn+@=$i9Nh-Kz&};<1)IX}t=iMGvrO<VqTTE(H z3GRKnw5yxt{K$k8ow>`A5Z$a^9`e&(Jbmn*?+uwQ)bPn$oGH(m$?QgZ;oY-bDVPs( z&~0tqpgKo3*o<`G=`eFo%eYOl7|s%z(`e7pgq4uC&#zecbk{DQ1FN6-<Dv3EZ*0Sm zI&=4}#OkG#cIEq>euQO>4dm?2o;U#&JElV&l~TiYoS8^FGye3~h({IZbHU)GulIu1 zG@}@7f0W=|6Oc4PQp9(6o0EmC$GX$8RbD3%^{~#*54|yGc31t6j10Jg-GYr}rU|*E zgxxfgW>;{<Q~p#sbO+y_ldw_buFo&u2^Ax{@zTsDyJq%KQ?UDtE)>fiYM;7Gdg?iM zup|L)R~u0-i#2h*8ko_PmJ<kO*4S|1C@_V=OMhntq~wRS09n>Q0z8b3o8#aIxI@c5 zoD;`w*;%8Fb1FV#>1R5z?qSzbHy`cjG8oB-XC)`v7v*(B+V2yZm7iFTsSu!a3l8~) ziRg6ux|bO#sl9V+ep5o<!A{u1f4fiO-zXz%Ydwev&XhUc%RsQr<y)CTE<|dtsl&YF zR?d)MYz=1Otdk|AQa-D2?qd5c=i^M3M4??&qdB`(?(6IEQE3*0A6bc2p;Z)1l^Z%} z`-!(c==GFUaN(y;IH9o=zOuO&pCG#k6;RptZicT2XSyl(UU4$7%DL4Z*d-G(vSu#C z2ae**-emGoHL`FU@*fV#Q|Yj70FU+e9-Vi}?eg|!y|pr>y7#i}zJX3D*@oD^)@iA7 zo}G}@fa}^^B>fl({h~`QQ-6Ak(#RWUA=rRi*>|4H;2<<vc`wLqe+>vJEm07Fc+2iV zJUzP2(Ql8Dvff7ZEgv@8>*^MNs`(^E=;@$L{Z5R|8RXjgzvs@pHeRV8Eg+D8#-g*@ zsy=Z<+-_&w#qD3&m)FE!IH>KFh%gE0`K1c4IS52n7o%&Rka}(|w)=JMndIY`x3VQ$ zNTV*h=ZC+&vWyC>HV*E1ZGS2mAJ4tLlW_;-xyyVXC8d2u{PZI9-`&NjFY<qzo`OUX zKP)yD?qKInO8r_|(ZV9j&&eoZ2G>dmO|$M>(CqfNJ$uikS(&wu6$#GfsB>zoc9H+( zps~mPrOc#UCWcVawx8j9>4bBS2;=VWpU;L2S4;j~|LOm&H08ga_y>im{q>gv-2YNC z|K~6Ie+{Ai_%Xwef0kI4M>x=YkUp{WDJpR7hb=hpAU@MeZf@i<s-bT5tpCWam6;H{ z1ui{o-*avKsKPe!0M=Za^5~s=?qNvQv~j(^{)~Lc{}0@HeNIM-bgU0tF3J(&saCiD zFf?h_FFI#FZUc5DhXa7Q34F(NOuEhC+O<g_>C;r-;M@O`+Br3Nt2r2sYJrOOPT8uh z;`V|3zLEKQ)0<8d$DBPtDl~m;IBfnv%^Q=wnP9gRz}v8#5oQ()Ph|*2g^g)5t@eH2 z?ML@ZIX$bx#aYePE~sqU%{8wIwCfzyDJ<nw6JN-6su)Jc^%K0?Ga_RTW2$hIa^k^p z6Lr-dhCLt2FsP0!G;gBQ>F82!7&cD&{K&uWqZa>{H!=4jQa;&p+40N5HkB`0r;3*T zP-g4kV^aG%DvUC>4&0d4tyA%{FrhFvO_@Pz6U-Ak5l46;HOCIm(MILS{=i?X*Ohzg zpr62iq6aYw;C3xd>Ll}HDop~AZHH@kAgS}n_h7d?EaV_}6X|pi38vU=k5DPDILT_- z$H~T$+&;}<?x(Ugl1q;x9#R~PnCZMr7h&M4kqj!6(fNxwLM6@OWM=lL4BGKT<gw@7 zFM~F0!6x>}n@zhoOZVi>DT}zN0~|aX_blg-0)+34lBx1Cb0%W4oOu1w{z!}U&Q@TR zB6=5aOqq7xmrDNLPT#y!2V{wn`ca{uaf>cck#Ai`)t-q{XUvR^zI9unF78r*>JCrr z_d$2HQ>BF>^Mhvo+YS3^son=v3D#}64fyp3O5Dj%_~$8#Yx$deV4O$%hnGQL{{?H5 zrt`jKvKI%0gQnz1KNLi%pHTy%rzT;G&|Y&4u%9lM4Ib+RA~Hm%jqdgulJrLgW9T1v z1fYV*et}yGy><LWj-dc)<xVxHc>=Uqx%Jz#<$8vI35?3sJ<Q)Nb@BkVP(+}`4~d2V zp+=-$<`dlz2fr#yx+;w|hk4x|1A(TpmYBq3(Ks*_K=h!>v>s9+B}-B?{-S5pwdUV! zo@Oz(uY8QOWZxzMeZo#GlS&OfgcCA*FYb{83C0K3KvQ}mfTQJ00i708l>l!CuP4ov z+S92eKL#LzhML9Rd<337Py;qC9=r6!GA(Rvt~Z2P)h_8fwUd1*5A1g1oqeWe`%Q8? zdohQ;e&9QW+9St30o~ZK&b@)t*zb9q@Twd;x`wgy?Es6%=YiOLZ)z2zhEehJ;?C5K zI+%4^Gbi9|ZPX4xO+Tl9Y^1+C>xZxT(&RA~=E-&#(Sz}Zwr^6AwBsL}_;IHvIe#2H zL+s*|H^BCT2-u?LUY96;3VT<gxza7WVj|qP=}n?b`*(tT9`YBSc5eQes%y{ceZsiH z9{NBmEypzwP=z&d!mf^0Dtb06C1#GWO=vQzQX`@Se_;f6XHE+gxd>+_^93*LI$B+E zvi20YipUN@gO9H30Y16sH`n3oF2>`qSuT{B-kY&9uS2WVb=q&U*ExdvG(_G>N#yXj zVSZ4-FMZXA^~=TDe6YcQKT0-vN-_$tSt-3#Sjyc+Un^2<SMZru2HadKHGl4z#vp+X zc_Ev2;DgaWz>P*wltD(wHN*imi#F*q^}9-k#(bkj0*)IaXL<OJDiTzaSU1eum-~et z);>4@#BJVUhodOa^^NA))8%HE&avYDVRNt@K-r-#w!Kb`Ms}tT&u9kKMPanO#YjmX zw;gc3ySURdn!Qxt#gW&F5j(_&^Xs+=Qmw}-_51=p{G-xTz#KQiPwtnvqCHh)XK9@q zYC+Kc!_`78v;71siV1JWFNU?1$<@h#aa+K+v3R!R#}PHfmX=JQiq^iJDKG4)+qlYn z{*4G{jeyb$m(T*a$_Rp%n+uUJ+wqxwXt>ennXs>wCx>j+33C=R#aax`%WDylA#B4% zbig`r48fVZ^2BOIr|BJsy_(9;w})hEcqcxZg}fJ^6^g!^S20Amk9?wFrHe?YAME|L zcwEPHP<YA@I)1rRXyit7_g+;Gf27w$S*<Ybm!h?QyA1FjKE4W6&Ho?en;g#8t~=r% zIOVCsEZ?lHs>5q&*@HMU@(dHaHb(;s;crr|*h3ueB3R$0>PET7cZA@6MFUDThrH<@ zYFE^T2XCB_8jjySK)%*5Yl!O7QWvA<s&|Td<eU@)=dPp)b3YfMj;nl`zPI2Q@gV0% zRnao;eFajzSjj8i_fhUaf`t_uLxL$$s)oK=$qJ;N1Eg-I<Q}zmJfpvPkhMJpNz)m} z!OVANg+hH2x_z;8_sISHp|2zP=jEPJ>C|)+L*N>N+_<RbDJFsm26B|^b1n=}_NJD_ z3CocaS2(&fPh7j(PE7W#;5|-y5PyJN*ckxw6^~`;b4c+h&o@G9xMzdG!=GS>$?WJ) zFT$`3ZiFA}06Dy!!hsf(Ys(Qf`X`o6h@_GC{KG0RHiq!RF}gV65Kdz%{mS=xFegc3 z10tWwij&xkf5k-Gf8!GLf{1N&AGfEn=-sx{UhV#YpFac)SbUUFYLr4;%njFK7SF@Z z=vL_7dmim&E{nQ%H*S_%*|cJoy4jXfo;)@J7#B4rGV~cOCiuQkfr;^8fJR}Z={&k+ zsw@`M_5dk=5x&J{K}yO0X)Y3i-HU-Khs@A{&0_rO1g}Kpw~dF?i0_a@wG2gcrAxf? zRco|4+$tuQ@4CJ9*dNU8ZAy=H?YRbZWJ~h#6*akZ7H!`N4o#PessH0FIJu-BRhzBb z%#WuD#@^weytw~6l{0GX%CsfaT_l+F>=NGSB1J5GccK&4CL{M4pVnjx&K=Wq+P$tT z(K2e@xru*KMgRxpJY9q1wbw|p<n#kKpEm6&*!+28b0cuv14>?4VIKr#<N&ssKc_LO zOd$Xf)IPz7yCbp(G1oGriZ`lELbCGJMt&hL<<GwlL2SfJEfez1o4<(EHMn0!9nYVJ zq6ZWzQ-bj#5xp(mcTSEO>xhU+_rTykJdkO--4Xl5I*npMeBgH!w))?%pzPw`Hl{9I z8v2p^=buen*Mq)!vekEU7bQe4&fI8rtl>G2c6l!z4HVD$dirFKUwoR#*y#9)@e+5t zYMbaag{LP6l4{Br%cE54#L{sLleG06AzC!wRv!8{j90Qlhsii8cwmIyAn7ONu{S!% zy<c4iCT8+F+EK~-I-R*szB{asvcOZxd0nK%h0U*>S4)caNjI}lGf>ncazoITd$~fk z-&xi6>wQeG{MSRHGSdY2J%t4_7NP_k=b&|gE1>%(S^1+Mn>l^g-Uy9}6)$PMytc(w znE{Tl$~v}nW*4V>kbdU6uS<YiC{+d!8@t=4aGWu8s$`rxYMqsY=3)>VE^R`cUxpX1 zDt^+B$I%|-9BFQ;Q8RFdPwN*k=Pr4`ycx&NKg3lgyW>99PW>UISYUiA^c?x0qG7}8 zA{c+4`KLJx1Y|FGa_pTVXEhL_Sy+?Zy+HfhNzIK=wQBb;$qqJ2UZcu*G!(_pk74pS z6Oc<;WcM`Bm@`^urc;;i%4Fy|ZTkkr<^G9h6g(ANv(;esoCmBNB1o08dV%0Yl1WY! zG_SXn{-l}3WL#4tZ`F{@R<#+f>Yr|j{KgLG)05m*r9ZRDxD;bP^81>ka9<~tT(W#H zCt|DW0FysHx$#)vXuI4mf*tw&UR{wSNxyk8Vl&x{S=wz1SRx{J9-$7cx#lT1%G=P^ z=MdaXHDce_dGPdDi75*2I8Nzs>4_>&uvAs|*<QcL;#H)tem*~;-pRQ{8t<gE;V+Mm z9!2(&r$UvFA~5k@2)4FNFv*O40d-k}xg5(Mt5f%bh}@LOj}EUY$Aj~6a=rL2OSkv0 z0F07Id*gy4Q7;s;7&XFq<U21rhke1bkuYtXdDtFmuLf0J%?HZlz5Y2$aWhWNe<xkh z-Ryj6^_S4n10Yg&&lar+g`f%NyHjmJ1peSakr1^|#Z(mX;@H1_WK$6e0<cFP{og8Z z=H5i>yp8%GSyk+21a`>^`suFI<=p1>NPJh`Okse_4SII*b1D#6`#k62^$F)$T0Apn z0y7tVq+>IhNX-_0<5gxZ^12899*wTf;czLs?%9;Ha2=+$Acr!~`X$(p6T2fHv;5w? zt4yAv@R=(<>kEc4!FE3>!BUh?&MUoP!a+=mlMAUXa!6+7((*TS!BcviD1}DJCl0t1 zTtTnYmG%ZPdkwmIDX%IF5St3b<)`#o%uk(hkb_A6J_~n}(b{O9$z2Tp{=8gPnOL#6 zpl(jmg#|I_5>E>xe6(~~R)iV+7}B&POY|)%rv!ZC$@|o%0%XKsmA6^@&qYz@1gGb< zAh}D=FLRoVkaAl_jitO&3cZ=II@3849dK9db!Utd*`7TlDZo1oX^0FGSH}w{xu;k_ zFTGg~1AMA}70ZB)jzpy42xt}0J_g`moufWO16=L|rqCwgjY<gS3+xcPAp7(sbiosV z;MSZpjb@*JM5d0+E~*`p7|uf+ZJlIV051ANhoho$M3m~;k>Ivi<W0C3X35Bu2$}e3 zSgtCPYM}7jdhPOb6A})tT%eE%`jZ*bWh-}@`%Dizf|WlAg<ikBElsO`@dv`%>^r~r z5&$oMN`_PO*yklmkPDvWuF#U}KWl5vQI`TIE~(+<gp)De<05IwSK6CPWN=R@6~@K) zM!WMVdSN(JjtvhP<I$EoJuig6QMwMn;Tv|X($j~l9vW%lBO7lT<NwLsGB$cas8FZ^ z3%Ipp!cFEqjlz4LioSIU>J-jraK9}JomWe_KbAlAJcnCInGhEc2hSdQr>gNNj!wuM zD6I}opFZTG0gv^z*00FbB_^dgku@1%b9~Sl{|aI3&!NK`K)wIZBtQI^rl%a2%Id(` z;CGZ6)}Z}J)?k_75Z=C#sU?*5(v!7$CdKmPh=$qAD4$%_$3?K8PvCDqDGL4X3B?(u zza3EQB<Xtp^WD<Kl3@A$e$`#NdvunOCO#IaKe1eOIp{A3wlSc-wUL4s`C?#&hYG$w ze)+kHx^Qv}b_>=r*XFRZ#1>V&-~Nbw-=Cvv5p}0a8JD8Ko#0ztCV43ns%>vY&;E@z ziB0gxy_glRUO0K?X%^$vE4qUjVV3$3IEpx!qX>?8_1j1%>o;??CT9YrYkC7RZt2tz zrSQZ?LQvs2nSM69Xd7kH9n!{&e|F6IB1X152zHh6mKdY^Z=t8@)jlVZ*4_J(obnm# zBvqz?p5?&)IvT#_Z!gBRWZMKJ7L8;&pNdv^-5*@R%Or8fs3D3Umf-xkxbZY=o#V$O zRw$cf^vA=bXjS`UyRrZXsx6tlOe`j;SES#y)D)L&iE+6gx1D#0l=A<<3l;Nk6J+@g zM&@h|rH1)xyzN85M>XJeqS0g{WSd~DV=2$QK!eh2qC#ZD(ow0JsnqjFzFa)Oy=RPx zo0x8rJ32$!5r;3YzYdG77{$P0PE6eAaT%j=`XYNb{Oy)I`EYTC$`y#Wgq#>n)y%@4 z+(+!#1PN}Hqe4~zqEt;Mmm46x(MiDEZBANs$H5i!!lZKN5l1Ak{13i60oD?be41<> z4U?WYFV`Pu>Krb)RrT{%Yln(J++-Rwb}Wq7e6n6l&i1yuo<oh@ZNVi;fw8Py=soDy zi@k(sPItcSYBluzofjNQ?T?~%I^O-Gp#Bno8!UyuC4j7Pr`mXn=J03H{mPfWEFgmO zZv*;F41;sZuT}ln-yle}#3|(_yLl_2&S~F^;{aD&+T}YWktAOCJ6RjK#5n*pqBXr7 ziWs=Ud0p(MMAf@+Z|<>@;q?e2o!!p{q&y78DKfI)<mbLYB0GZf+wsv&Hw@2?B(41+ zqp~vREdO<{Z2zquQJ*z*q-)4>#8>;i^Gk8sBvTih8xgj2ouhs$L&vA7LwB3dkZNiq z@A^9B9Zme_!M7v6)h|<R1;+*J^!T)5{MtAB|A|CY2|!<0lm#Z9QWtgU0*s=xLLYeL zi;@F?=(t43Mn46t!e<5Nv0`km(bMOHmV(_ee1?v0StVZ|vL7}@<!J7hnG*Qk#az_8 zdjGpH%M-I}vCc&%tx{RNNR-6yRC&D9>wbR2hs}oF@mhreW+)i2F3@n7AJ1jB4;G1d z=Yk4&mF*SEmzq<gNp#M0dtwo7urH>yrWUwUU?wJkV?l_OE_EsahLF=hfO?^cIyw4k zSic8=dbJ8}6PYU(qPw=MWC3VTIN1T?sJ!?zWjetQu0qyh2QWaNC+X~HF~Qts4cFAz z&$d++&|STkwb=Beb|{TDqJkGr4BGUEhTGio#0hmGR&HN<7M^_Y?v`9yFD{h}7^Woh z4)j>+F0!xPZ7;)$(MMU<t$TPjB=rTlyXYA~QFm@NY`o(scg@OCHOe@iSA_TJpIm2v z5jbBkpALK@GRX(gGB-+!l@@?38{CVzSDDm+v<<nEWZ)Y65&3j<TR1_i*3yK}*I>=t z+hR?vBVCxBcT-LiCnne}mj}a{^F;8cbo(j3^I2WIRUFfc<P)3y8M6B3Ug@jhuJrHT z-!nL~iT5hoF4)%xqPNsiV&QAoUF}rAN`<wJSzm}kW(2>Z+fFL+mO#<G@LV}pi8DWn zX@f;y^Sv@5qLkOudGvIr%0Kmo6!Kh|tt8RCM$#KFw*IWE9{Q}x%S!CHLAJL#_=o7a zD~-~SuQzH>YN=|%<!j0B8`$KB7+G0lgOjKCfFjWJ>NP9h0*+u?f7kqOcTSak{&q#5 zv5zwCsn3TC(vf_N$Yw00Ionu4QkFB*_f1^L96N9Zz@3WBfAqh1oRk)zd;g!y69BG# zqbkDAlDsK1s5;U&EIRD0%PEQ<7=1CAh0n?@0|3^yKMh7O-4J#K)W;XPx5Y1Ufdw10 z=8S%b-X6ec>$v=a1j>&pAT{fqMIZu35C8`fJRB}6Z2u*eVkwpC{+Le9`-HKbVDjr% zZfsBWvoL`;NO10B&8WWoUJXUpEBQL|c(8>xceH8(PX5Tsj_I3WDoo2xgcbYcU1qYg z*8AAgA>>$DvYrn@Q>~;2&7)*5dVZpJ&b#~PWho^sNG7n~4S;heWGxWS$~i=v7R8d{ zq@T!Y6{4ZGolQyVcxGsoj(ir+>*R?+Kry9jhY>z10b7hFT;`Or&z`7hR_F!%ez=6j z%cgU1l%wx-ZH{x-4a6S)FvRz>2wCqU?sbk`<N*dd7^4Hz{IsPOmeE<G`c1y*=_+!? zD`DZ<-LtY>T$KFHo-P;5Iw?3Qk7NFp$nV$!AMo{38Is8vT(aMCJ0C7aK{&|@hxG|8 zvR;a`yvt)fe>OiR*jJ}Zw498HCrrjret1G@&hvF&MQp^e+O`Y&rsc13N<(AJc7+pG z{1XCbq1P3mzMMRD$!Zc>g%qR1?e74+>D{T|J$jLkd>y&``e7U2w)V>)O^GN~lU1T9 z`Q`EDdf9t12TPLWn7jHb<F!bs0_C)EvWWVgZFYtI_=jO5`6m-;cC%SLle1B^kj{|i z&y3*Ce9m`kw*E+fOR;q^?(l#7{xl48RKw8^By1k$B132Bph-DJAA;U^67My0Z&Gf3 zjY)Ri8T-~=Cz7ODtZY|RZN=xN<tt*`o$noGZQo%cSCKxbsY>p4LhZS>O))I=l9!qq zJF#BGvD8%mLN=)1jhD#uW8cRo%s2kG99^2@NcJz~^i*fte6fz?6U{lE2fyjHW2Bxw zb%b+ENj*y9h@qv=E_TT(G17I}a7A~R>?tH9OJFK;E)KobRmMr2y?y=awLnoRbWSh4 zQvM5eWC$FKCilNHDsslTffm7v!<v+eulI+C8%NIc>}gce)D;OdkGb>G>hkah&bFmU z$_gzl9!$g!uIY?^k}is6yj<t2vGF#fjk`giRN;4WuwXxQzNkX@Tl7>Q{rgnao!5m6 zc)|XdYyGvne1v4WVZVG-5)I~+s`rbJ$jst~RXtvM#%~NRrJ0Es!C0N{Lrz*TL7x@< z_)&;cPp%3OUb50=ipIzq2KkUK!HCXUMUiF+S`frwv}|{>#c*CrVe(1T744#=^Pcm} zVPYq6uU6NTIBvz?5k(!YC6tr3dLZY~o6hn*v6@i*#A^`V(F^yGk&n3C6!CAMdbCd& z2gJ1?V}P?WEtOeaF=|Hrp15u@r<iKask4KA3c$QPtIHTBI?cIn-!?Lw<b*#r9la=@ zIdM}Z<}IhBugw=zs7<q<k4EaGw{ZeN?c1%1Hi6&g9rcQZJR7|#@<W-0VUE#$8hIbG zF<i23;?UXleC7byn>dAT?fRH}{;g+Ci_}%NQ9UDtbD-1D4Qt%2HHEbrNf0w(?*J2U zxd-mAOuelqt$|dGxL)s|UhBdd@{J~QU1U>wG*m2ury7mnllp~=QOn51_y;Cx=|0`| zHxeV{X4RAd3_UYBQRe`q=5S>2rUB~uXu@J$jgq<v>c*NVNQcfEr6>16RI~~$Z*qt= zRhE#f%BsdgIbYu@)s%nIBox?tJVl@r#_PF3&o2zmctQn$e*s3ek0rconON2<jqg2z z3r+prinOUuVstAW_d1D9chEtLUlN29a2+Pfv3&67EoXG&7d|iXGu0`e#qvGpt_447 zx^YA{5bvClkoglJM7@KW?a=;<ZXunQ%oHyv<82hl_aG__i@nfol;F50|CQ^C(Yi}Y zLsU}#+Ru}ZZ8@*cq(v2fM1P2S>sZbxlS0>DAm`aA7sJ!NljD5GFZ)$p0@8c`iqy!^ zlZ6j%%uFNl&lQ5PQk@D30o^W!c0r#q+1B2$jtO?^y``!LSd(8*$+vU8+8MT&?1Yo# zM{~P6SzrlX_5f=V$FY>gldwTR7@?hSL)b7<CMMGvIW?O6*I4Y?U^=$*2kcHj8_7>9 zEKl~vs_ZRy919LZ6RwR2af6awJwmH$rpHRhJmbv>hDKAMW7oFSOYGhZVzG@Hxc&rL zPW~7^yDmBFDiOKj2w8Uj#yMeG(wXHMk+8x1GxmA4^*WLRv75;(L@fz=yy?aOBG0ui zPU-ap)RaTqwB}6A2rv#J+gj{h@DLGFYsllwXx|D(4f){9lentJl9p@RJXSoiUWQW; z1jdd-xSc0q__jrijGXR0yi~|!st@IY)5i#Y9ygfwqfg(nd?9?9I*7TC=O2eEp@Zwq zjIHnM8sb#gldr<QaGrC&niyjLq|v5f^x5Z5f3N57f^FLvYm^V$twRApx`LJ?CS(q; z%Te~u%MMp_up_Q}^y#PH9?&l+U5cxNBU2hMm3VE;2NeZOm>94|rCc)A;KZNLCVkJy zQRETQdP})0J<P>d-67>2KZTdBbG{V%I=z;^P*Nk0P`!)$w43~DtqXh4F8QCG1u!u> z<926^hR?@B5hriAw#E|L5R!Ii2S>Ap`1>EKGJZNleXrk(3IqC~V<T&EME--+ZHkei z8u?e~Z^0I<K2++6sJo8|3H;Gwd&9IQ&-e8$gZ0x#wNJrhSm1Ahc$*T`_QkI%rw<H8 zs)q@wCPXGOb`L!&OMphFol=@>`zPUfQ!T(z>k8|ZPNuuzqu0qmYGTX=O!;5&W|_hN z!-2S(e2T&g1sMe^%|z!$*P;#!Lx+S})yHzvvc>(<V5}Vba^Z~+T%|P92rdt(_GwXV zN+C8x;`u^AGo9P8=hK{#GdeLimG2nr-dS;__y>{2zP@yI#kas#8W~Yrm!n{U#H<Oy zuB_1oF{O{0Ch5Zfewy^1kKzz-B3m&Y+#LYjz~$<AO+f}rQJgpgk&qpA<v8jjgWZHK zsLxZi0vBZD@&qm4CJEDscbS5mbIPX$-_J%=Dq1BABOT*;6!2n2LS~v*CoQj#SkG%3 zHIMgMpM@F8IK=aJoUQxDG<-cH_8D4rAfXxh0-H~Hei;YMKU(hxwT5j}G$IV-u!ZO6 zfju9BlyJ-`YG*xQEFAMu&R3gMo8IUxn2_hKsD&~u`GrFm6?O=(4npV~Ox+fOx0bYz z+qh|d=*T7Jlg2HN?C8kD>C?k=DxpNOt(6x>&_3n|wVI;?x}6S3<0|Tq=i!*NUTd=T z;Q$TPC6}|$J7H=emeriRadW0UC#M^0txPEN!*FZDUVt%D)hk)b5DtN7H*abBs$Z;1 zptQwV28ScXSnIwIlhlIK%re1h%Z9}nV$X)eruiZ5TsQ0qC3VByg-X{&Y5~R$DO&yd zC6-5DAx-Vcdf**t0lX4f%E*<a0_Pv9*sCV-a=+K*pWxEn_D%g$OuK2R@7!?XH$?S& z@GK3QiMgM_gVDW76%?)Lz>{i81mCOoj<6iC?su-Mb-lVo?%e`joY}00v6gq|>3V#j z8TpJkL}S^oxiOs%k3wz<Nb%afXRI`~i0-ZG%{C76vjGkkp@JHGbvc_jd6-r`rTB}N zdtQtG_Qg*yp{4BJOp081h;8@czl_D-PcV=L*n~TMjlFhE^XsGMUVO*!vmp%U$e*Jc zhfaYvUFcI^@LUS*wNq(IacK(4p7WM0sqn7ssa72r;!kk(yQ1`}sn_m=!hSc)1Nxm? zcJ3HIe0RTm%|gZp=-DVxM#xK^P+Y*KHHACZr{3kW;asi(+rv#P+jd%reS9oXy_z>@ zku}f!jwBv*KP(-kS(@M$O)7|#3tf!tN8tL~n{r71AT9S`Fo$40Rl?|5c4WKEUc%L= z4Re*FnzGuy?oEnbyI0}?-lQz>4U>Smf6d6@5@!fq$;o6^L5dy(-(AsMQK3{{p{ttU zBWr&yA1`Fj7KpJ{l+@)x#57mL<_<rS^-DwQ=QVh;Z5C#ecM^=|1nt<jZvolW)1-@g zCM|2RjT4aQ$&E~K_7dQuI>ov-$7IJ1rxz7x{`yi^GYC-pU2ZG)i!Tv^`X(gCHt`R5 zJ#`zN%?2D;&4yi-VC_VOF}o%l(b*H94xTL^O}mffA4!cXM9eDfEiWllXEqva;HqFS z03Gbt%@Zdm0@l6oWF_=NL|gjK_#WX}pz7w0!mhPQG+N3o@yT8x%5sjdR*sY#mdivZ zHfLVXU#sx3jt_mqWA56aRwH4wFmA$y)YBoys{4NH11pHq%qbBt>Oq|e<f?7k<R<Sf zNNA>8?Z3c_G#$Am=2&R|Fe7ZK+{xrMpg5Y9plXUx96)quI^TdZ?A`Ci)v2;Tw~>Vd zLbcWCS+~X-w=USIg8%xj8$QEjxlvw5q0{X$Bfz|BYt(|{sx0YNVd1;#loLE~K>j6T zVVqU9dKhnxRt;_D6@8R+#>hPv9(MH2?u0*9ZG9i<8QLOg7uw=Bb;C@k<?*^ijv{Pt z1l_F|fAr~Cg>5l8Ee4SMR;hY(KUlE0j%iQ;l0NGoXy6)XVULg#*i}~NhRLu35GPo3 z*`!eNV4dEl<Wo%wpE{$g040i5Lul49@%lnfMti~>q2YLR5m(Ps17r)7nEVv!U)4H0 z;F$;Sye3bqc=d0K%smTR{fiRi;`8sI{-xwX!8xO$HRWZKHICra&5cOa=QY>Pcn|Qs zTT0FnVOp&yFk0Ge8xFm>NSUuFF{-cL;9zRr@svS1zepE(w%Ya)>6hXIllxzg!sjmt zi`}g=j6^!rYL(Jjkd~{L;?}#vSY!PQ6IU=0nOuJDG%~VtZ?D@f&QD&~6FN#qVM17U z7z1+7^6vi%s~@ub<$~(pWBmUa5q|tk{0krTFI)3Jt?~aU>-6uy^gl;ae-&<R<3Y#y zPY2OGc;eTKc7{F3FpsWZwW%B3W~sYW$YL6uE5A65o-gXW^lWVLOK}q+%PnKk@}yC! zpWN#D)%GSFN!1Ch>W|2gm@5kYa>}vcVRyChx6;A*K#Jf3)G0HVxodSQ%MD>vo8|Fe zEaK&@u;+g(brHcYQEfFTCbn{ORK?B}w3*jF0OB-TrpL`XZCvyLQ`SAhEQTe%3*r*@ zJ~~+Kr5qf+Qb2A0!{j~0KZ!mV7PG67u1Q!G{BZAh^Z3ZKPULVlJCO25!}d`{bIU|> zPR`uvU{CDfw_o0^FzM;iV0~P;pRLJ?Tbr3%1G?*SR32cj%I8yCJW!Vmrb@SD%j*<J zn8)|befWlMyLzXkj^5uK?wB5`RI`tV2zw?bDDiE7&snrPY1FB(FJ&C5IllzspUXKu z*=twExaEIDK%Vz`|6j-dtLlzd>?+`&LU_P|<T-c1Ol15JzWdJj($)c0>$FY{Kx8d? z7Pdq+_)Pt=edPfaV0FhY%8?FOtT|D}j6g-Ih!y~YQ`tk+JFQbwlh&z4_N;Bua#>qc zZ`c>a{y_RmzfxU3pFzx<gFmDem6dVm(uME$Yfn*;nKv{>J=FXEb*h9atb#3GO6WKD zw}>^W(W?_EY}?1^!YWDV9O|zdr!q0)aLR9vW5*G}(q1R@INigC1UNnvXg^sZ1+W*X znVB3c4BNy!sB4{S!<Xu<Q4LX-0=cGx4*{dXm(hp4W>p6Zzc}%|_W>>}{mY*W$dNw| ztE26@GfBqoX-ZY`A$~T6%D0*D4|~|W+eU(@?ekJC?4?{JjB4hpp@!an0h&lwh7%R5 ztM~2?>1Al$Z6fQ772#A_mdtZ&hOxNcG9^a+5w5uFyRVB;R{Xl7qv<EP%2)QOa1+A; zZipIw_=7mM*Pw2C@FZaJOj9a^Xcf#ELV_Jw`cgLhTN>7c8Ysqzq^&niuAjSM+9TP8 zi#KMjx`oXNO`CbMthFEjN61S-HO6nrY)4j-rk*m(>sJSyGSu{&*RxIj`xGG*c=_K> z(6j}Q1&gReQ%yOLY*9n~4AMvVnwGT3shX|M4~>Mk+HKR>KosWs2kYJUfcfg+*SbFN zR@bQtuxDvzZbQY_ALh0y7%o#;yK!oKU*~;X6BY0hTy^PlJrXyFf0cdU+aEtNbXMQW zd|d4cCaA|JuPUTnWvU{H1`XVsoZJy!5AL!Psao<hJPv2A+kI}@XnQdHrP)vHK_0-; zd3sXmAw^+pVrBXzAB;X-7$3__{qAyvs(MtS{Vj^zfy>LQ-$IJ|gQp02>xF<qZJnA5 zk}EYL?r+$#y<fY4^zhhFh3m<Pc_s-{V2POyiNs!SQY`8Rp2~1=UzL057Ah4idBrj0 zu@z!aZL>+-DG$)j$<S9#Byms)qx4!SQ!1WDy%pykUs5fvb{wy5ZrXeYh^lOPG<spz z5>pYL!Q<mKKnTxV%^yO^q^7F$R<Y@ku354)d)Wtb1)h8?A1+_;LB3qfAT>-)KLqQe zpesu@nnGUi`uu&f@MCN~`~>G!`#C-NlwBZy$LAoeMAhNo<(Ec>F^gf!7uLCq>TREs zr3s%x0Bo^#&51~3lRYszzIM8*RKWMeIOgkM%+4Snx0w#vYnX1ry`XR_DLcyQ@OQLZ z6}?a+Q(bTOO7j5L?$VLt_(DDz3u~L!vn^l#{MZ9L8k{_cJm-&#SId=|m7YfX$tt3y ztg_4V9oTOAOan5)^fT)0J$sSWs`=MnW(98mnIN9Qhd*YDs{aL2YrTI{qaOdy<PA0X z$lU!wRsSoDax*x74Tj7&$rp99ei<>S0cIwI9xskwoRce>^TaYO-XwCu<<)WfR4%dO z@g0A<<CK#pw8X;MMu<NYMas4V+_yjyXio7lt$+5}^iFb#iB()#!8y<}-xOy&E32;c zJda~LhA@FRdwg&zZZf34clqj0-G?`CHl`XL|IB*-_~-uSU7^bhaG?*8-4RF{pCP)- z2jBRbFVhZT@9~tLzIMEz)B+7iJx|LFRaUNNI>}tgieY^}C$LGoF59^EjagKRj^&d^ zP10I^J2(tEl<xh@j~(<{W~P_W3y0G#{bI~XN~e@J;yae&Btt<3JGzwz_tGDSxCt?X z?tCp@r5zv{>K-3XAFI($>f3n4G`hcOrcO9gUp%KpW?~-?%{O{B@<ZaGYgt_?$JG%C z(T5=BMas)2CBpA(N0j`k*Py#A2-^EY`bNsO<4{6e+17iu`JtvX4{dh4v+7)1v^x%b znYYyiU@X^IFRL^9y)%j8CvWkqB&opOT+9<swd-(2O`WkTT+9o8_?anYZX8L=&jmku z^HOl1!j86-`wbNCB<D)weqpfUwh+Qtb-N8~Vyvb>fvelDVIHcZnZU?LORatZepZz{ zFGgzIuD%>l5$+er1a0r1>x&Itx{&D`a2Iuj8t&?>3tl`9pcPzBrw*PQG_QQ$UZPaI z+01&dLIRSda$l*4A`(}RkLZdYPSz`^E9@IJB||`WDds{uH&7j)#6q-sJL(cSoGmi9 zw2nb)HWf!2u*#|uIgFegNUc(z;qFhN;NQEs{!;>TDbd{D-1eUnmp>EdN$2vGVane> zcrn6}2S9A=8AYa<vvWLf=VAY~%}D=|WV*{PIRo&mrCKJ**AS0^?g=Jg-OR!nl|v1S zz@&(izn>Wnbv7g8WVSm!<44eUIa1%kk8-xp5i`L_t>A1)q5GNf`P)h(TI?hCXZ|U^ z_77Mqw4Ko<-|BNJ!F~P}+u``3qe%HsxuHX$bUQY(JRU;o3N&x?WpDF$Cl-3>jgxL~ zQbe_TQCa^z8(`N0FQxa_?C25<4Gp7p+2l9o2i50xMQO}y)RCRT{#49WhtquPK4*FY zh?^QatP9GSq|bJ!Xxtglhxq(sl}IN^-w<&+oMW7{ZRN*2MQci5NCTpnf3oVkZ`Hw4 zOUnj~2$M8gD=JTh!jq_TG=%g)W}Qwd;vCBpx+fx`q~A2>k{C7v@IvS5v}h%13>cnn ze?)am1p&Yn6MwjrH%o@wc%ZrR4j!<~&5Asd_|7tO3eH=APM`A?PYPM=$>r)QX0>_7 zQn<QJ(^w=`&+x6Z^JW*l;mzX&r73>V9M4ur2sGd0Jk9ARN52Rv0*(Y?e5L1*)M%n< zeUdh2YR%YMl<}sbXr)|xUAU-^Xl~w^z-3@T65<LP`c>drPxH$qz0~)F<*UULij2-_ zKhLVf2+w8vGQ~WgtAC?g^Od7%i;ps!?Y_b!Dv@n0JSLL1{`q74I=D*h-Lz~A1^?;1 z3hfWjSWiQqHEF>PrKT7_0JqL4jcuvJMyi<1muL_%<tyM{VR)<*Qxax#(L-yyV>3T+ zAv1Lv_+Y{2Z8R>Q*>Ja&thOP3EBXQ32R}^_O(oMmUH$MogT@@;@u+?wcp+1bz2-jX z&bIDh^2QG9@v_FjQI(u%OfH!>?170gtGX`ZF;+t8lriZf!wlUl0dce>7}{$}uBuP# z=GT;5uXL!nQfx<o_Q03*Cj(wP9r3oJJv{`ZmvWIit?i@fWof$~r;B6<<q)<OMEM$i zB~ab3S1q7d{VWHAQV6!uOb`3*lKk@}%@dv5nu?*0T@2RHjeYX{BkgH>Keid(urfJC zK|b=^RV)n#ogReI{UVV5GyUV+W$S~rh5N^3`K{FHDpU|zPFx9=7eX9+6&?Phqfa48 zpkIMpr68N_k{z<e-8vLVITs06#v|arS%3Ll3S;N&2o2;{A4kg(<?=3kD2uO-5gF=M zT#=L|t9}0ciU*c=aB9+N%>*1QU^m<8n|frx(Oiu(<q+ei8$4dUW_|wfo9Dg_-Qk^s ztE2bJ-CX~B>ND*_`qbx$cf8Pyg4tPF>?lF!iO!M;^y&@TF?HH87urddD9P1JG`?SN ze)SsD2<LR5Pcr@R6GQ)bx6lU0sd>}1^;b3RT;&vq5yY+jWTsTKN%TwW`~A=9@u$B! zk6lsMiU>iT$Yb4KfQNA#98qCB*;bM1jxV7xO6pFPA!npYM6XSKW^q<kRu)tjvx%B` zvUWuh`+;LNxfFCV!qa0$TPMz&cCw6Vr%SzXpW$u92VVnYUx{m*lzIdH%8om?(`OVv z-8!`Qh6%WXl)ahr4vaZSpHG)wRIrf~nnS47cu0Jw%2t!dijDBh!>KO}x7!rmUldTd z`<8;#A%_EeLNwAK<$w~QvFAN>S6ml3gc+Ub6Mv^piyncPAG4Ff!wP5v8Aw=$84dS1 z&&oc9+=e{W@N1xF+iCIYf{jdXzkjG~g{yAy9YdrFIQp1B+wkmmf*kezXk<Yjku<2C z3&E(3h;uZ8GzzU+i#rUIK87u(jQ1ky_Tezmr}dvZdB=J(j6kqo<Mi^h$>hpr%=D6? zI%H~88sxaoH}=yQL82X~9rmhe<A56tLvEwx4$mJ-vz}}pi!(rm=cUl-h53N{7RQFn zek+9~8+O5V=nNnnM`Yoa*><!HjJ9A-AI%Rm-Ey4amOdjOnOpm63L4v2&hv?%7UXa& zNERj+TuF2SY7YiDj#9psqLRlmW|SL~d1NVBrQzo(NeS#@!_~g;!tk;qZ#AbKrJMB> z{Zg|<Qq{0W=#EBBC5y2Yp%uB89iJ6ppwBxHzrJTdOKGT7-If9#eBC|7E2a7YzODRV z`l;o!n-dC>+dSU$`|h3kB@|8gPd?Jyikk>Y$nj0M8f`II)|v!sPQ~^xQu=l&&vSDJ z`Ia+aqmi3Q`8hjEbO}9+uXvk#boSXpx4M~Vvey`V<zEU$FQ+BMb0E#&L>rbzSu!j9 zBJb@Qk4bm@Yh;Rpkt+xGWh3$m|2>ncf1e3n8#vQjG4#;khpsd;_(urUJm=vwl+0^& zs~O}`<@?0jcMn{k0a6nc5H;?bQYw(!jL#lFVf>^KByb(~22>_0tFAv|)+%sW&gscp zT>+Xc`j5LM7)XXNr@`SBYx<V&LQs$NSj2qg8$>uS&8EG&w$zF#D!j2(;ilH~X$m8n zMk2?bt%(Iam~iFp;u=U5r0mQS(`FQ2WncW8$wrj9%(BvUC_<_|0#34FRc!mB7~%S4 z!u~YQku{t)5UZi6KI~d8SP25XXQOYU5p&P>=NF&h3;YS@d;;I%@#umn(j~M)@SoUb zIQ3{USl<@IY8YvmF@w@!sn4WlI9g|9cNosSavP!J1)WZ$;l{*C?Ss6p@rIu|$GBC% zv1p|!VllD;rd%9*y>in?xaR;R^_X!On@DR=4=&(CAiYL&!fGYWvjXAg`Y5lWMFa~t z9~r3Z6z7HC@})ED)uf>jtnzcM^j*Ga(W#8~91Ud@91~ZEe2c5Rl0+7<iMhC+;Z`kt zOiT)|A4^2gKCt8&pM9YQ3SFaFnlmk%FCn$6Sj?)KKi<hVa0&utE$q9;<s7|J6<!%f zpHb(5-)eb0YOVODl5#)H(QS80WoN6Ej^kv5BoFtJoq>(hA-dY##$A6QyV>m|Naj2p z=E~zGQxXmLe)(p<fSO8s`ck{u>pW1x3Q+lxl1l<x)xYFO;xqRB)=u%ilv}GKa44Qx zr+)eH${ec*R)?L4TU_gUh#rf-M72ih9AHo0n;I<%4Qe$Fxe0QRHmO|RY&BP$gWDG{ z!L;_UIt<TO=+-i$?}8pv;&Up)B4XBBt$lO$#8zb8^>u7ib8#U}xZmk$O+^%CO!+j& z{sWQcGc5u0@Nf@5K+d?!wMf&sA%>P+97+vNA@1^pC~<^$rFkjafnaHKn>)`;k9tSO zbi!uZNj_y$C}0f#_j$;x&{r#TH>V;3U0huJy^v9G5ARa29mJ)A3Fn&VdYqu|K&lG& zm@<{L*|;`+s*gkT+~|cQ)+J77j@GkD(${1%q)F>SAkdwx>xzE+;JnXLOIBa?crMcr z?GssQrBW|ASZa#USc)bFRh}*Nq_GVLebT&adebue%w_QzX^hOH1kYm{<}x}EXc+UZ zXYP3N_Dxa5B<=bWJnh_0@k90RK-Ew)*Ey*wHw@zt{U?@lrny3Jdh&9|m)5JoVN04| z!3fWwf&B2BAP~)#<*H`O3gd;YdItUrtj>Bm>5BBnbT>ivTc>7pL3zT9p{rk-<CPph zfje|DdXKGJ<s2ORE7d|wvxOEv>w}+=)#>&tP8WHqHZa!CtHL`a&gA!VwbGVj?-x4N z*y*aD6CPk`AUqIdoMs;FL@al!z#a6Kc}TNy>e23ZZ<J6#tv~YA6HuW$h2MMDTJC>j zq%q&(4}Nc&OR}qSU+&;-)!3!A3I7LaD)ftCwpSfuL&wimkleH*u#X!-^4dh2*8fZl z^%m?~Qc|G}1S!(cF5gjGVM>}oowPUPEeb6JOO6{_xzkc!YucR1qtQYta-8T>x_L_U zgIf5TnxBQ6@6$z6$R~FOjLEd%0v`2KAyt<SL2>cdYcz&Y%UK*t>F&MdRv(TB&XPx8 zUW&-^P4Xvhn;*SRP?dl6FtqS^OIz^}aKT;Pv9<h`eZ|+{UWl~Cc8VK|bCE^m@62*# z9TeJZy^kmVa?jT3v7j=kFVo?dJ<fBu#~(`)EVI{oO-ETOqTLplXh`P{LC?lU&0bA^ zVhwy5uCqJHmW|g`uWepxnM0p$PTJ0aZFNbJs=d)nAe)-uH~N+RJek;(OKA>g!`AG; zLN!B5mW2=P1SXR+DU`3Clvz#ioxG#12+z6jZPl3;xkz{tAG~9E5oD(PYI=w7DZIYD zrTOwILj^>*9^CguE~h2+{2qitgIUgur^xT|hnlat|F@WNt%uqh{(Uv0$<jw>n*{O- zL5QkUq|ba%muVF>k16A$XOLj-pgp7yWJqHOXZXqx$nebJC5VmQL73;M(QxlckSi-V zT!<#1085$rK;K1UOxsU8N89+qvuM*<=p0vpq+doWi%5fY-?-OTKXp@ej{0pjdhQ&l z?8j$HSg-rjJpAfn69e0%TjG=`W@`T|-A4h;+K<8&)#?2D)74>*wCh}jf^JzmidyyY zZLt^7oP4Ewc%vri*f%S5UQ_Y<sy%!_D450f18Z0)YdxJ@<+{XWS`ux*4~PZ*(!1sd zmMMPM=n{Sv@DTkC)tPHJDNOD&DvkTYGSjp@wd^x=eMh;|<lI-XqEc|q04ct3ahRU} z%30R%XHAx!PBd&3&1}k#UEdzGI!yuRA>$)DdD=@6Y8~^FJ|wTiX^`Pk4m%AKi2jgi zi>sg?ia#d=#b)e1rG379I^X`ce%aU5c%A+12%U8%=QbwMN5Xx~rJy@~doJp8jehUz ztFP<sON;)~V9&~*->k_vBYaq47>;(Y%dUu!ad<}q!*+y4+{8XcgJGjA#_pIAx|MBs zB|p>Wau08s$2)3A3v(hjrgs?ouIkW4VQBSkgDvc{E$1x@Va-+dKSmj1IyP2repw~e z)PLt07b5I<HPhU|(d~0i)A0+?#Y2&a?)w!-ik`Xk^zxM)pv27!C2B+yS)0_0z6Le! zYTX=L-%+_f!ep!*T(rQDsE8I=C7lluN>4BjH&eE$vKLLUAQo)D?8dkooXp4xA}iMI zYKzKc4_-%K3GGfFS`M<435=JcpvtPdb+H@#$Av;O(pCe7a*QO_HA5W<VS82W{gW0B z23snWSq|{Nez*=2Z)L|3_><}8P~P!A|FlG~NPO(tIdzE02u;FT#}SSGwhCQBOcz=f z#;`+Asx++G_4xnm*_rlgfA87XcVtD`+<Ash(Vsq-clB->&8f4@P4_=;G#yLNfRE|v zL9Fx{4CM@O7(O$kjtRb{tHl!2SLijIO?b_o>`UIIouS1&IRHHir}-^yL100jSYtiG z+IFEZvF;2E^a!V~Xp7_KeEjjFa>Ut024{L5X2&@buw<O*=K|0DSL$*i4D&Py+Wi&2 zJMG|^yXxjif}T$n!p}u`BG1#E(knJ$*d9tPXqi#{WGE`syTcaG`Bt%q-q}Fs&1wj( z{E0P{D3m*4u@+Q!jI%-1IGEPIP6veru$!5O#K1;x1qKZ=3RQ8y7|1`d9GHSj1=x$_ zK}+T~(}70|@MeZBGVf*e$v{3)#H`R-{B*bsGNO!dqzI$!dUSD<_b%9+=CZFrdaDYa z%=i$LV+oXT=s8}nz3R7R(oA!vfNt~N1f#RPtRj=q{A6NyV&mNU@JEY1U`TUe&3KAf zr<3$xXkd4?HwkCE9F@4!e?op`FI>Y9+AGq5zp*bWWHrm8_tNA3nEJF=9<o)i{*@=p zSZUb_dmnH}M6M%K`C4|0OXmV^v6J4P%-?3|d+V8EMRwneD;a&pQF{;kBehw*1gRl_ zB>Z%5`k)q)BQwONW$S;G%tW7K7u;^@-`?C<mC8;ofVJzA&YR<p^n&D@mlxHQiJ;09 z0p^#b=o9jIo+EmBoDj_j`RmEnl<wM@jK{NdH5m-=tE7%?A)#r_bENI<?JYWw)ah$z zksWkzW%b)o=Lz~+7le68Po!9&r}z|Y<Erionl+c-j#Q}l;woCuXXm;qf$iuug3(!i zN~Kuj$ob4I5sv2Nq`YO62+c6AC*U91v47Ip@I7b&;6*|#J|#Tfb}ucOZX8TI8u~G^ za|5)fT#(!teiu2yq{De^ukSs0dO&j08ElNRRcV!PCO?aYY~9{u0}f*TeU#!#rGb5C z-i4k~bvqN2!QZ#&5`xL{eE}hQGdwYzHywJ;Xt_)9v%1qJmEQY6!_9&}@$E$31*yv) z*+CI&JlD->LccoI@ZRBV<x@Y6IUQh%vT1Qwj&T2K_SLH0egR~Yb~hgcdlY%h3S<2= zK46rqE(DUw!#6j8VrjoT`QFj>^}c9ctAEV`&kkGPTaWA3jjU;`p{d4w!tfRUrq;`u z9_uX3+X&$v8t1M2i*QvsxcwIMf(qjn=Vx@G+fS8M;g$|s`7O~tPXjSNa}PL+o~<Sb zG<mOpi^TMnMei+}`v1k&e+M<yc6-2ZLa3qy#hcziQ0bvb3&8>^B_awURS8W3NN)iG z+zLu>B2t1NAktfu7C<!gUP5R=O6a|X{>JZ|^SyK4=j=Zjh8gzkYprXoUwaD(#8zD8 zwFd#u5KQz}zc8Z>&~KJC?`P1=phQ!xX^wYw?B{ZHAeB<730{hwPDNjQRA*%<Ss<+L zTaI9m9)K4uGA9l$qxHQF$LrRTMPQdzIY#Y4up~WL?Ug1ImLFUI27#Et8VE3d>q^{k zu>SxKGB<}=K5HNyH#DmS4$?uZFRQdOUK~)~%#lRzmbdcroU&q$viuiAo%b}=X>RXw z1%|KHo5V#Ou}st6`sdho{E=55IJhf-XCX>k6(p;o?Q_zv*ksW<;emeA-sZvI+rCb- zy{TiP%CqUGBm^1<`c+)IIld+2Z`|77-@kduc`*_d<^pg)bTzO%!oQuHmPaNPI}({> zngjy@vAF8yR`i3T>t#vcnLFBC0id5--O@N42GTC)c^9_qC<HrnX;e6^mby6cd~sqU zGy86Tl&CnT|KT}WJBW4_WIVv5!KsG+fPQqAYp}N&XbW&D8SVuxE-NTM;?SHGi5V0p zxmjvZ1QVfcaH9MM=NBGG0PC+?;b3)|p3StPXQ$}~#Kg?OgCXXt!yE}3?W81KdSR}l z=E$=yFt2<Nj|WKM<_c%c*4u5_2)PTvc&+)v+=`=*E1RpTrX_emf4PyegNq6YZ}i)= zHGyxJK4=h=F`MTTEp09rhpaX+NaU9(1bXl(#7N?X4nCFFC{ttWN>F>tTfZ!En_k<; zbA_$&oJvbVm0tgXSVoU6R?4Jz?Xext*ppW5iA%Qg@@Sr((SJRIUP3#`^xWU`-^(lY zjSdZY`VPTnEuohbEWtCnp{Zy-ty%P2c_tL|^o85ZyBEYdA8Lh`(VVR^UI#s6lmuy_ z&1jlnx7n^WzX%}@zfwelfU^h}S|sZzM;XU1);8Tb0Gn#$iZ%Ji+M7@IuC+zoA0}A5 zU>Pg``@M@jNN9o11IR)dXl=V73dnLuyBKfVW6%;-LHu@kI$RTdm*zQ>d0}`FnrSw~ zT9I#h*iDtDk1?uOuvB%30rgqjKeHEDFslEE55D}5Al%Ll8SdJo3n^dZj0<7V3a%KG z%Stk++`CBvsCPCEu^bz-s@@gYjUm*FC&&PoM(M%3_l?n)G7}#t5n5a*e*|bYVs3<4 zsRN*ubb06?tm4%y6B(7!)F{vHe2{lpsT>0<O~fA#U%)OcvJ?B|L=t*I_I*!_cqW={ zjBYGWs_{^2Y6q&imK)P{p8(T@W~XD#Q;Gw(dh>@K7~i$`goQ8Dz1$9N5>QzTlUX>8 z*u5)`X0zRfK2Ty{xdB=PPG4_nT}-fw6PcdWI0OLDfCqQAo|qz$g|0M~o2kVwEBU^j z(L9>t=@6CgQS>a?h!Z*EApI0TcXQqbujcAgFJRjSs{4=Lx>o!XH1r4ekRLL`4EJoY z^Mv&tg`P3|7$Q~NyFPiNmtZ9DxHl&js=7Q^w*^;fSK~^I?h@POXefWTf*Mpj&j+hM zmLWZrhQ87pQqpvf_NWn`(KyS%<?uBE+x!Guf}6N4rW1P-y%_3qOimMuI$rU}AoYA! z!@?1PJfyuz^oJuli)$;f&^E7=Ml**6%Q&}mmuN182=f#rL8+RLWGE2)Ik@RLo_!C5 zum}SO&2yTfi}3#b?QSOlO+uzc%OfAQdVC+qc;@TzKmIIESjnL<F?OFXhMbq>thAAp z)do}ooFZcu4kM2!CE-FepL%A;9PhFmXrK~eNi#kK|NRH;XL}Z{j^}%D)ieQiEnjk@ zN&B(BuzyzzSL!(dvcycp{umSe1>du(lD;!ieH2|D7Q1p#-R1bj>TBzsu3*j1m=Eb^ zq&xK||H`O;3tL@5<jK#s4MB-R|G&p`@`h70_0;72$Egvxl77Y0K>LM8ueP7giwbQ- znV{AxBBVSYT|P@w8Tvgme|f%==FYO4p6owBq2c(o=WNs71_3?!#fZB%#D9n`N>DI* zxk-a>1=%xqxwM?MY)`r&@&5pXKeBw$%}Y|MqT;v(Jb%mtRhw_t-<06I#F@MYVW>!O zyJJ=I&quE=MT7wl4s^Sx8LQVtI~Bql)axyG_yi8|0f?YKqB9CWi^8unR;pj{zK~EK zEL1`<Cy4LpaKX?QWvo+Yhn4L<%@1U*5P@;G8Mc^Uzf`Sq1oj8kCo=ufHZkHY;k}en z4X?^%Y=Zumi&g_lr_{ZU%zl=PYBO8YI)y^vM<ct)^1h}i8FSCwwWU>~TzBAJa}C}e zd~<lC6NS^aQfRj&21v;O)J!c!?*(Cyp>f`3`L*S7f7(i0v@#TCh3fO#EsZ+=W0kS2 zbAyd1x3@sXcK03f1y^IPSym!anUEL@@AhUogtsp5nwf3=(>h>vD66qkyc^ug=3v~q z3hEeW*g3k5gdEsnKOO&D;R}!ks)p2h?3e6#CSSB~WUGobKvew+F);76YkE?vr`ZbP z>sA#Bb0S}~S{PdUsG8$iQESXqQ7!?KO6{W{#OpU1T0vR17oVCy`a#0ibEN&>yY^|C zed&E;r7kA&6a$RmDID&$T2io0ELS_e*q{@%U_;o<4m_C;p;GIQ`%hOvXKE#<6oB8> zi;J>iBhCf(q!&g<;xa!ms-uzz9~OTp2SrPiC)X~9^@5!CRyG@?%Gp$+9;BhOJB9tH z!E^a_xj5#Nh2yPO-u!@|yI>QkRv|V4I4Ku?8YScJV4G>b$rsda%%AB-Y8zELpx$mb zxW`$Kxh2SQ;Zl1qX2Xx8%H>Ua3sw|nj>kqeb_1ar2MukxF5r)ToXwn$lB0Ml^HS)6 zhI*}K<AUoBFId1gU;g01(P!O*x6ZbX|2-zuBLDBzrcg6qNSc1LVMsCUONs7gp+>&Q z-Vr3Dvx6?CPZ{#=Q>TLjpi2)ZVwdT}r9j9}Ki@0x56TZD%2Z?|Xe{{p7VW7jb$*Hx z<C1~IlA~I_f$PH==9Gy`LaOE$g5IQb1V<q6*R(xf<2+n(o44nGY!btKtD}mh9>B0* z{pZS`-o5@|B<TF`4y>}H)11LFrQ?^(OV9$JkvP{(OJ(K&_K58NxKU2gEat)|wO$6{ zPSu9qz5LTQrJX-VxvJR<lslC4)55X9b+sks8Zoaff8Ars``BS`l+5Abm0piPr0J<r z;DtremB`+I^J^x|tx(cO52++(q*dUILS|Ad+?Xo|WV;n@WgXx3<1i%j(Qe~V3ZGHy z<Kq(I5TRvuHRmaILdHJR-J-a6gZQ?;aAlC8Fvy5dH2VtEkv7}J3<%lqLzT?iagia8 zJ|&u}X`S8mMNT3I={HFUn%&O%!E+}?h5lfVl5Mj3qkHT(U=bM$-dLf%eDIKmi55PH z*bh;#o#`>1^rf9{>!9E4i}%>2DNBBeJOEj<Pma{4VMYAuhEmPi*e#2VMGBSc*fzCj zt!;AZ7feP<D`5AI>}5^|e@HT$zO8VF_-TAiW-y}E?+>u|nR9ud5rQWC5`P$cmp;=F z##CT+V?O*c_y^Y4p<5!lR;F1t{O;fOdai29Z~T;)DTKrWi<Wdz#!3wVhKP8@^*`_S zJE^O|zt74;DkiG^k3BN0yr5ud^J@sc0mbc5Z&_8$my}kJS!%mn<J>vqJ(66!kkTO; z?;iRb`>nHXd$~Go`Jc8c=Iqmz7MP~7bfO!#^S5J2!FEWSwm@iHXaWs-2OE~(pP+sm z=QX)!Uu@==`lN!7RxrA>x=ZQF-uFua<4LC3V?9cHNVB}hQk74znX_s%BpAKv3z}Zt z`B#ByXRdeBe80I5LwrAcw(GPMd>76z@jc}v(1I^*nM6JjQ|<K9o$qaFxiVZkG92){ zPPf%NcW`X<t}bEdX!$U;%LajHKe9|I2)r-)6_~YkHXAMwM49?9-_s8|j7mjoyEA-A zuE_)Q@$qF$)OtAqp4bB|(#~{x@rG*Bfc}ZF=zPHKah}dwT`kOhu3w$ed<;LzGs3?n ztRYKicFl^7ZAxes17@R@@nQc|gcFAv`cC-;#dRO$$;g5Q9h`h@Ud~r0Kq$@m&Up<a z;)kDPww_tK`-{BRPt>q`VmVTT9hRx*t`V35T`Qwj>bM4|JN#;KZL{DKG}WhJ`wkfJ zvCt1A!?on?h;sd(gZ>HvMrOGM{D1Lmo581vI8pSgfEMS>5zGUDs^JXDTV|cbATgHs zws>bvTfgpc?kR-?zUDLRi@F>8b+0O}U%8k^xU-ptIAQ#;(n<8L{^0cLxo{IB+K_%& zEN+gK@f<A(h}M%!ox;*%LbP|3OdSe`m#2<j?{R@SM)?5NqHI*O5;jY`4zs})Dnq#v z0d{|x7Fnr#Xtxp94Iz55Pp!A;e}KLL#k)NaM}9irGI)~Yv_V%b7&s9|9-PDkh)}U| z5>V+|YuK%P=L{F#<nF~kQfLA6Mm53m_)=E9X52zSekQkBs-5#MpZdb5$LsCY>x-hP z?*ScX%$$(8DMDK84p-KDjzA->BU=2n;SLr@QvNT%$3o}-Dv36OUV?#3ML=<aMgH3Q zzWzdgw^0A7@Ph5&=cTICtfeAPHdJeao^f`uSV$C1%Je1R8z7p|VRtSO9@ABW^jZoG zhVjB;i&P2is^enJC61|G@k=QzM0#M)Ur0B;-QEJYXv^T=TbFuy62u7f9bsASQ1i&} zDc-5z1-jFiE}JmCbZyG@8R;-cGUV#J<@{uZVMM&*dht_qFxDbv{*%vZkZ{Pm7^fCF z0?ipZ72-H-xRN<u=JE)`khKZAOKX|%rc@Qq?k~bz({OCUF0Mi39v`WEO8TFHnh?_E z_n!kbFQm^mLjAvAjp9>CKJApRRdKP3^NSnI2KyYF^s+WnprJ!*K&bzAn@DVw-mM)& zG}xLs0#rq_3<%_1iNc@86&Y?s=bDVC9$ivEFq?g;F!Wrr8TT;FhczIX_#ee~rw6YT zf5&4t<fp?C+2@m_)5&%!GKCWy?CYTqwIIIy$CrbK+`S=3w|z=AUxOY}(^*f<U^&}k zw3g(L6#yf5heTdqPs{ymr&WHf81>O)ENa{j*ygPJ#JR`x_XsDz!?b4Va|fm!*Qs>y zKmTvaga52vESDX=9H^U0e=G;rwsX1t_pcJtx_+Y-nr@&k#W8%mCVv@_RGiy=$F`k# zbfpQjT0xg5d7tRoK@1u_=CK`Xw_b6M=+$Ymuogrcvd{co|G!}HPkwNdt&AobYhDbq zS*8&lf4;Xu_4R9ee)ZR<ahm|a3e^UTA#rwg)(}3`%&1<a_|0{nfxgFk-;{<M_!g}_ zZ5v|>>Vk;~4hCE2J6$GqD3!^VLSDCT^{8!J1jILXFC2;o677aR;5^>C^w#}1pi?69 zpJ0y4lPtq5dXmdY5GH&`h;<<3$A8uGyh1@hcijdqz@17ON&K6E3R0(e-0+vk64_IC z1b{)h0E({lTlmqDK{+B~>Nf*prN%Y%;%^QsUKmGNdwsQ5MrFK8cJciker<qoh7Ja1 zA()PQ+$xVT=D)sV@wR=+h{HNQ5ZD5gT)>VhJ6*Eo4fw_&8v!Opo^OKBUvF2J209PT z@-&%{T30G*6}Gv8v_GgMUa<lgS?4IBC*~e&3S-s`+q3Ie+6rB)ulqHG^|u71<~vDD zf87tMz|wvoQvSLDCKVsK{Qqm-f1YHIudo~*D3566mbz%476Vhv(Ek5mU(yKxxe8oN zObk8ayoDe3JC)+7sRFx$X31*z{uTpz)Aa}Om5RvgP_tZ3@NQW{v$2{%+uNNo;6B4t zdM|Hr)x>g!kD&d)1vL^^Ep0YkR^kJP$mio(716V3Ri)S6fB8KD;A-N)hrm%LbQ_uc zdF;aLR?Fpfr2+tqD|(sXEVcAo>xW{r8VLYhN%Uj+`x>5ufw~R$5U%&jw8GnJ=+8EY zKJT$IF9~Z%9r_mdZuLJm8>Xe4L=fG!olnqySaYxF)z-h1+&e-31*-pV@&7+wL<B6y z`V9{h*I+`fzq`KzP!PLS8AFz`G(-=W2gfVvrR`Dlv%hVmz9J^YYAn}SELM3xwoAj> zoqt{Urk6HmVFEvGbbSwZ9q+Mq4lv(ST)`haCe`21wA1Kb>qc29e?yJz<QwvBU_W0A zSbMloKCgs({mJs%#g(~&J@YSc<jibQRXZ+!=`~w-U~XpNmoCR0wDXE8bE*l=M*wuH zl0waSe2pjpGTTPk2Ix)q|7A%U(l?T^O@a%2RO>g5SREZ58@0~h<L|*d9@Ab&&d<d> zZtk20ALR$D9#TogKHK}_u9%a~#*@ynekz|dGCt(txu--9YnC`@ScRDXmk>BgW8znv zQb3`)!1;_)bC6HSZFZh4zWID!K)H}aw3eWN^&y4gI~rX5Jo>;B4`0Yo`EWAO%3fT- z0Q`*BZe5#hwn7I2UIAtr&yO2pk<TFb(jgNcK)aih(Afj_ld+CH&9hm}x2tmsI^t}7 z3E*#ul4A+|4#!WnNxjoPd^&}v%Qar}uMYj|myRA)&nD=vZ298`92cC3{>XwD=W&<X z=P%Q8%Y)y2)Bj$RXW2M3Qaiiiz{BrdP+G7gTo8hv{5lhLNkK_bk&mBx+EaQzgQAZq z0G?8A2T`Lv`Lj>$m!@bro$p*54YPv3m5^ru%Y>erQhVK_2d;C4N)dbs4RIA`JVpqC z>mDDhQ~haAhA?uu%gW9-60vrcWcPO1S1*;;pr<9QH48^WGHBJ#<fARa@hOgL)@jqu z8dgjg?E9RQ$gcN~z6bsAJZ}=8yudOma{9}a&;>PG(U7n$vZe+#e&7EopeA+L_SEnM z9E|ML^x@MTAbI-{cqhxWkn;y6W(X(3$CV?Ck~>UQtsEav&oh?BsTp?_j(Qz|FG?h4 zBb>;`cRDs^&pybl)cY6ztL&c_rpo@ivAt43+a){dM#-dIRF)B{EAV^fLf9Uk*Rim( zot~C2eNP(%^ax0m^Rr!Snmr%zS`}i3kIeq1j)5DWCFDEGr}EJBpxrRkJlXY-Xtcc9 z0L{O|u$~5q!4EkVEVdFI+n~Jm#n~1l)2$P#X{P$JyQRi*V$B70<Cx-lA47T(m<#p8 zfB<v_^C8K-UUMRB{Q;Lla{YXStv1u(?V;D0ahuZe{ee$Yy;f)f`!wLDL@ajmx9c1b z`wb&NBg>b=W>$Kn4e<dcRIeYu`GDU)gf{1s-MsUPG>uZ6lDgTecc)HI{VxZUnN{1a zrmC2cJ(Jxr#RodAYzk!gmDTkhJA3@g%2pS_hpy(j6QVoO7V8cA9Pm^V`?E&J7&a0_ zyKCom>j!dP@~Guu?bOdZBR{>0&2K=T&AKHsc>M@2N()z1v=IBxM&Y)D%-VVtf|rD8 z05=LY<E*o#z6Tw4TCtq({aa@p&(z>tfspZ4RoAk#rGEYONyuLHcO~FqNVoywOUqwt z)dM8#;Cdt7lw?p+u8k=pgcF&c7lqzXima|$NNZReVz;Y+BVT-wB0grf@{A3y`Ymu} z(U>EkP}e&IfVs%2r3>`1-Tg;v;U>?OA4zXy3F0e2`0nN4mp;HdXTF!Uyw_fZT^v*D z8G%&S3+5KFLQIn^hdA^2u&KvlR}dr8fqt>cHXRil7R>59O#2rBRk64S=M{!ky&F*= z>{yPndRaV{?)`rMlwZ4sHkvP(W*jfeWn71Ti-<qn?ma*0&B6KOu4L>uo$&=6c8BlH zKM#6GVdLM^C#=mJ9Py^Me<M}A^X7x6(ykB<M`EQILue55%13DL72kgm3^feDaJb~* zjYAC1iC2~?#Gh!e8Fzd2ZV{$|av3Qsx>132>P_8rDWZuuUA5Qb!FFn>q`tr_LJea1 ze-O3l_vNLxc}@nLq>K(9z13J87*)d%#CW!wO%FKkQSuoeWy+JBBGJm8*L1soRCs2% zp53upRXnyB+`pTG?CA}8Mb%zY-3J+D8J~DIUWiD;0Yz!`{zklWdk-!Q{VtshCmL8x zm8aICT~-TOg7B?-lZ}+fM$UOkW6jSJ8`blEnUz5PrNXgh_*ADu{Wh*$AGl)fgd8eu zQ(K|yH`&b=;7UO}sf8yzz(7}g-PrC2E8x34AS6gUXnXsgQy=Q^H>HK)!cZckp4mnp zwzjsmm_%W!C-VwL0$rDadNS7y$?hk&HRn0@GD}NUQoI<C&BA@-Lb``%ag9Ceew1Gm z5`jh<_^Ue)ptDm6ho8!u8d$(%#&}=1#(m<7_G;JJ>g-wbnV+VMRN6M#2)R6!P{R}4 zc7(N*XT!*8D)u0zn)4@QHmCV>4V;H+jIKKw&i@2iEu6=+`gD6x6oc1mbg83{TSW+d zt>AxGP<;k&wxaRfnul4JgU70xc5uA#<Wl3p@;Ax(jdZD!w3PLW3eU?fEHdI$!zGm# z-Yibw>-CX6^FA>Pb=%@uqKq*z!04MT=H4awBh@h}Uh2-@79^`mT)qfA+hxS|Yg~~` zGHjA&AL|fgmf{l2>@Yd|>*0Id>9c~&&xQSOQAmeo@n=PsuqZ0YSLgM>O#CIO5n2J; z+_w9VwDBQbZ)txXvK%wYwqq${1f2Iw!=nyU)CJ*9XDS4n=3c={%S3n7QusTgi#D=V zuVm{*fC3SW?#cbVnh=!IBa^&!=?mMxzf52QrmmmO3-v-jF_EQ*tx7Q~AN`)+4Byo} z5^RJ%2u5xWmj-!_nv?W5p9tc!{k*=HQhH;6hP&OWBQ7(F3uM=%wSy`diVgE`6RSY4 zs&|@7pT2jbwN^bhNj_8vx?LUPZ_&Y%viRpO*L(cJOnl(sH)}PkhiIFE%puzexw9<- zV!lWsb^;Tff#n(US-dwA)5U|;6B`vfJEb36(rx-Vi2fD_ZDBCapG2s{n*`z+MStAk zpbWU~dDwJpO}eE|4U`Vr&-y?<x8(+ZV`%jJDgctC`+OMg%rlHxV{Pvdry!49lhrDE z7swD4*RyHp8<W|rt=?l$bv5l{<SS78Fy+<BPK<K>4V!dZ*Pxu@VGEKF$S+A()+zCH zKzT2D6vYN7zgsV#>!@3NY)PW~T9LA~SwT}W_NO;BXY+Gg6_clr1%zEXz|tczLeqZA z>Yat1=A)5Yf1q(uxWgA{MAnc$Z=o%5JWyYwhY(aPRpXK&Kq+-OxEeUVksEM)<wpM0 zP2J&0To^-Oq-1L16U%juY_KY1ibp3}F*OnB+|9Pw(=ho=E%u!J9v^fd>;I$&)qmP@ za+h!22d^jS8@kIi`vKp;o7UQ#N_dXz7_8#k3O?{q`sCc<vfjHM0(>?)*Z>V`&O`m? zy-gaxbjeK!#6#LE=#5RX!Y3(_;SCgvDUm{c_0;?y8@2@A9a^rP^*YM52N_9*tW%>; zH=Le}VL=xwFP%T2Zzvb_=<<&aXy2Nwng6S+IBBm&EmP-h9*=jwSeCuop1JneOaTSw z@_oyUhwR8<xM)MvmzWJyU*Odrj?q^d_o5k~4ne;gqhm+(_$}q&Q~Q}zSnBdGAD;g} zaVHR2{c}Uv-CqbBUILMeh4kDSlc#@4Fp{zwsn^3gLN8|_wsb5Lk7%FKB}!eTO`<ab zJ_tQL6Q9*@U{bmu%9Ji<rag|^V`gTBMu&g0y{jC<{F0g0EX$ZH&S|kv_^F!j1@Mg= zjni=Tj>Xuel)28I1i1yex2IFB!eN;rGJZp`1lpHXx~{tq4?-1aCY#>t2n2eb<mtep zf=*Yg1ZuGkDJpqgiEyY2N<Vcp>0}qi;;(Pk!z66FTG0BllkHdNg{f3T$ZrGSSIJE& z)}>6oB&J(!9HacBzXVP`+yjoX*7wDD6E!RDK}|2RqbLg?ik%MNE1E9Vf~RSg>eM3s zSL6ZMA4QH7o)cfatM}v`zj3D#rN>3O(IW8ptyN#~@0$(3WfsJrpBliDf<n-%_ybl; zi&=m4L(i+4>1#h32IwtQXv!ljmJ)F%1|&Tt=CqZxu@;WgK@43D_JrWG+f6Mx962Ww z%2HqYS66m(rBBpKT%2pfHU?fQ?+4Ar>(Gya`Ucra00_`_lj{_JUVV?Sj0tNFa%%D0 z|C~l!l&%|Wdk3q_*RRHZXj<|@;)L<>mZ{c-vj}l5IaT0vyj-Bw<m%(S&n|-M0L83Z znUq+iWVp4JUn`yL=q1VrB)FM%oPFjVaq-Q>oYvUW@8AHS+mZrc!SUZ<Ew=6WrvP!Z zFi$^+3SF|sFTt-V$X5g=>BR62PAiJ(Q4#J&mZ<D3C$#eQ5Gkbm=wCoI?6cb)71C-} z<Bslj9;qVNP0Hq~nIP@~0cbE_-=<Yc84*x#m-a2tAbA+yL!s7P9Yq$Fo&k)SqIlO; zV&WI%F^64?KCAX85a{MLcd@$Lfxb~BZKb{IIWDLzA!lu+g>R%yEgQMh3JbKls&T0t ziGt`d<(80sUnPJM4`&FTP#&S9U-8=W9dx;>A#ErXz&*S|7_MHLD5;87{qS}BnWGW$ zIpOp+{@QHPahOUPgEj<Lr%4W4#E~sADrs_kA6IZL>x)oZ6y%IMaY7(j8CPdtFz%Zc z@O_JUJ}6Zs7D91T_h3UEDe>Fmkm>ap;LOeyi!|n;<C^146<ke105pp+g0s#HTyZU} zcEdK~Hv_U8$ud9H20mff9b)L(?$hUo&DbMX&bocJ`##ksd-Oi(aiQY0Z>!%Qm#ex5 zyVjy0-plzg)twj9l{IJ;lCDgH0y)_f^M*oG_la3^r%PTOX1l}VF@tVHnwRPZpnm`O z`JM8B+y&E~{dlOrU-V$emL;vmC<6SAH1Rvk5tSOe0wT|oy-s?b<p^+gMtPl-jtmcZ zrY1oNuvzc_Eg@*_-pQkhMv6X5UE3cCEMe}WP${`l)*dvu5k)<TzweU1q=D%js$Q4L zR#Qo+We&;n*I!$|SQ&r$SVikXB$M^E(<N{yM~vwsF&;a5Z!C~|%l=QKGU{vV_c!6w zbSgtPK6bOSbNZ@4q5RAl$naKrnl5%N#G_bO*p?qV)-#GhE5QAgKrjxd=<&#0rP=I6 zY_?4}B8E>lj-xzmn`SUp^Ney!JKuE3@3yJhu9J}6pSgM`TCO1{Eq2*UGNDZDF}pX) za#;3m=2#vbRMp<ILrVJo3C=w6)C-`wjJ(n)#<8tQr3HrkwiN3_RD8VZn<X=U7}=!- zWD?ZLgj{M1Q4YDh|6U+I#5wWwekSsA>YC`*_1Bn-5<z)<&?0rd?s4F?wjPcUu3c(= z0N|46tmbQWfeyNfhXDPClYy7%;F}UVMrE<l)kc9jm^lWV%}yL-U23`Chjwj|w=K8v z2;Mtob59y8dgmzbEbc7S9=FcxM@u6)!6(k3DZ&s$j9za;9q+GQMs)r7-0R3jDbhps zWO~9~uBOf&jOA(8_r>B`?W-M6gapor!+RYR0oGnGlnu+XEfJuESO2{u4jl2*Uo4nX zzQ1&5l1fKj^6?ZJO@<ZOz&1AZWPstL?_vNxnZNRkAbmmt>=$2r2k6cT8lj$*g{?*F zFntkq@@D%JX|V-AVCtGl1otO;x966E%&U7_`37Uj&ap(XQ)7!v%7`Yg8wlMh&<Bd_ z4T=Pv`lu6}7zEdN4<R3HCA)m@`|Q${Aw-UHd2B8VJQxYw{ba3ps(HFQn8P2KI_ZA? z&LTtP4qhI@6+BeXh$tH1f22_VO#+>*I+X7+dn5CZY78lzTH)s#vM%3qOWS2Wsg;BO zN>aWFZq|1d)=IH9fp&i{T$2e~eB0&sh+MHCL0uCrh<Lm@&^)M^*x6gSmL+r0IYF3Y z3wf|EPK6Pp)7BN&g<`nUqNpRr?CbSOGB<ew@8}_NFsJ@+LI9d-{j2>1d={11=Cuy* zIr!nB)Nz^~G<n>0uX|9#x+BPR<M;Zy-}*Q|wmmGYqpZ*i?I&>lM;X=85HmT+W6wk9 z;2EnRU~|xARGr~xIU-FpC|FPReck9CyN8Zf;-=K7zSPD2WJhuwo8_l202aCm?2nI~ zPx=<RgBPl^4^vm7n2z=z2jInL!)oBzXu>+POug(#W)P44=JYw0mE+nSPou!?y6Hf5 z`pa0=TM=op_5R(I<39E~AXY`+<lcX-QC+Cny<%d$a}X89g8eV~B%~dUnfL_)ox{my zxTZS0huC&#M7!NjlT7}rzFsJL?qm+eI6j*{*E!>ht|j`1I5I!BmExx>$jD)1en{kD z=MMcE+L?ydjzkkpPV!8E7n)MBw4#zxiOq@mU$@?93Ni*sC8!HU3J69(XqwoEfJKPx zuQq9d=c+$#*l&5#m$5!;E`7sxNb{*1!c;VFr!5%HH~Qc_*5z~oj!5ZeygKXC@w~nO z6V*<6fETi=6a;mYg|G1C&m!M_%<oRLO$N|D>NYAuWXZHHnFMY*hqy;-IA0L_;qoC` z(~+&M`E0pY#p}Y|O?m%jiA`;b-Bu_pMDmLL`E)Rv4{IA7qM-Dov8mA`^GB|SEvhgw zNb=$E9dY(_MAjW>@7eBqT08Z_4mh0=K>Or_{wVVgvi}<1e(6FMo+0V8&EB@r>1{4= z02(1AcgzB%h_Z?U&vx}~H#3nhQGLbRp~39y2pWa$z}BI7TtWIY*!n|bU*t3kaZ@TR ziB!hGQy4o1(5BG++wLF=-b~J<i(mYtx7s^ZGPuE;FXO&aIF{#B`x~sbV1XzxVQEA~ z8HCuNbh&KkYqp|Iytca0Q%rdmRHrq9A}H7$O1=&i<W=8dP!-8>LzhIm!Tk2#SgpmI znI5@Go$W~10iu^Yj-(rb)<TdiW=M%~p&>)fu$D2dOrkGh6VHMWE`)Xv79|3CVj=(X ziRg?eX>&90Y_?3e5@jLceXHrv>ke%xyI}P~IB6jNCr2jlAj>3=F6F~HCCAm<Q)ALW zSLyrN-Gh^k8bCB9y_G#E+m*^0NyOAsx&sy|7fx$~ZHuk~I27pOzJ+I0JWW6QguL@b z;}mp`?CM?Yx+Niaj~#2P8Fc%M%VEfJT@DeAnN=Y+dTwwZTq8In#t<)_?}oO%I_S(+ z=bWy~q-x%3rx21rPTTb~x6VKr@V)MN53pO021|{r?V006g%NJ?XyzU+n`p=N46aTO zO;1<Za9LxXw8_qHrY`sd*!^5#10v0k<QeThq2s@*B4C@jj$6!3n0)k#YlZbd+14HI z5u-nuLkZnk&nK2Qqw;#5lB2YpxDPzA)%35edXKbC=ZBo3)-ltWV8+L#N0X%siZ2J} z6ToSyDYk3BypcURMT%XGWax6edq<^OaDQy~pD4}3x*Zj7i%$eZeMOM0F)eJ4$1g%Q zXeqsX?oYS?zsLS__{bkT)Z-5BS{^IO%XwrLHF4|hcXVv>_q@DLK^r$v^tjP)E*=Le zOJ4P_^hpMjMz2s2{iXtt`>tqhLNbqzLT962BGtCg>jmW2+zbIny{mp#*Zu?jRXpM_ z{T?lv7A^9XHLL%pm8{UlD|til94AQ@sk?MYx=R2y_Q=py0HfBE+Rq})j}i|NUjH^a zqqValb~4MZE*i@OZ5C>T#MZ-aXNfq2$m*AGG&S3G6tuF9hvG_r3ey2g4|JKq^^R6? z<38B)LRB&cKyc4?TsP90IrGR;cYD^>%O?bMYfCLMoz~9~%?JU))csaDPsMYhz}hi> zhV%{<*RQUYH@!4LN1=agxBvSI)Zv+zJ`Jl_`DsLvr<Ua&^&<a4Y}RF~LGf&v?MhG+ zYojM#K$1r4RR+iom^Rw;k^8t1JZ)S4(5i_qq!{4q8RcsF{pcm?wrq=ABk^?)RUqNM z=?Lu|2?AYNynXmr)J=Qu#6-r5Y|q|@`Uec7JG}-+fkqN<v~z?ckG0o1e+yp75Jt{w z0Rj`#Q4v#1iEE2IaA4r(Ec8#~4*L=!+hi?)$2Lte@Uj51Gn`h)R;5zhG@<<q?|6Z( zN~C{K7iQ1q>??TQ(SAQSSWt$43-omW3)MY9y)zi))5OEKZ|2DCfc7-^hKvp%HxA!q zy@AefpKKoQ-hehWwyz)g$F>i4q0$@n;I+g2XWQ$Xkj)e>2gHVYe>ZkDnyLh_7)W+| z6TD<k)rghZt-P=ujW4Oni9!oB1SCA~yE|S-6TKA6C9#RO^c@ag{IsFIKWbwW)pt&m z+<5g8G&@-&;}vhDf%-TJ+UCmHOB<ftdcRI#v?v%?^lY_6DVxSpOXgU%4B=1M`4;%} z66?Wq7?=g~fW91O8G_$s!}U;Y&1#3=$(;6cX6b{S=WckNnr2W28i$Jc{q5OKM8j-G zTr+<i6mOLCj3*|OwsqV$b!M&5+WwnVYm=D+8h2FBIpSOxtr^hA!xira*f_FjX#Fej zbH0Oj0@weLsnslAkID68s5;E>LtOgCBVMUUzR1|+ahHUDGF!LxM8&4!@U5Tk6kAcy zfxwQsGq>gf^41R6rff^#dM?x^_c(uR_*QZ8O%v(@E|{#hzOcKRX|dt4{)Q}o`gzjS zARFSc0y_9Sv105K7$K7?UxO(>Ha9_Q@7S+s9+dyZ2pi+IK2&t1ikqg*1NliO;$Wty z=g3<1Z8_J@k{hNxk@J-E^4LHmrDH+|r0|KbIC$jmE%?G@MPa8&eOif))W2ck6WDd6 z2!8gebL<AV*EVA<1hS*;gjUIffTFPmPX+k9inpvKJEUbxL}EQo)NodQPOoY&&WvAD zaDo4C4hOY;UypMTcKR&pD=Bin>FMS(+5YTLG9P!z9u{c)U-w{<IJ8eBJQC0x3V@1D zuxEr8%J(g**GMU}_vF!(m;CdZHQmQNOR(7{64Xq`e9=~KPKv+qBp(=kpLOhZ5^n1! zbc|!D$p#H#W}*_|?varx!T>mZ<W*I{6en5@+NhO5+<mJm*>cL|e=oCMhP4kSSH@V# zJiG0Je!oHiO*1bhGJV<h`!xR2JOg-2K3cr+$X4iXBkTn;w3Tj{4hfJ(W=StdO)MEU zO$tXzRRei}7a6&Ku#9TOQj?L+%r*x({OH8UZ$vZO07dPCt2A65O04eGJwC(UyyIxP zc<Jw)d--xKAG0<9Nwi}Y%o8mEncp~kD_&Roikcne-xCd5LW8FxOW*j|j>4uf7HDzm ztdScCt2twx?g)mi@&-3Y?f|(Dub-B2K`B{Bi%&=lmd~if?DPeW-BJN04fn8<T#=NP zn)gtC(n2dtz21b%Spdk>qATcJ*vF^-`N~VYOhosG0*RrFzG>p9trnBY?s&P7OMvJm z5+E9R#}2aP<^0VG-L8Ve!02i=AAmzK9lx!HKg-yeJ@B{JJdHoKD|><wkua-#hr^T8 z(0IVDE!Vx=ULJ9Cj;uu2II!mOKyUQQi6WSTLKZ*yISTu}`f@@6Uxra_cU~Q|LmQsH zS)C(qAgn=OV%KRD-i}6<>|VmbmAphjCsffxjtedhCVh^uvU#=qS(#7}uAeMo)f1m# zk*+MZl@jF~bFD4~+Qu{ek)@3{L#@~HB#dQgd~oYT^Yr%Q3R*ibCY{pz5x#LGVqxD! zFu>Z;JBXpzzaK}t+T1kVJM@QdV_X6gMQZQ-oySK8{#;bp7CMlbP1=HK&Q;=Ryn8-a zqs%@qW{7n@HgzxK&@9&a^B2c<ZhOYG+PMy1w<W(K*QbiuqjYKrpTJp2A1GmAJiUF? zdE$vQaTITP5T^djE?tS7Dlh6Ze#^liE=qA>>jm^B-wgje{|ogJu!z!{_Nr~kOQ>xi zV#*YlNxjd9Q{HU=9@g+XEaKXyQh9zH)DQ^H)zs4nPV@QEn>PES_fzCh)L)GSa3_zl zFc5~(Pq_m{#_>Jg-6(Q9{f<+%?6Q^}r)nh5nflsHy2^)hdzO_WHog)@t%Gcl1DSHo zwSDzY^xWMGo7Pcjt?#%T6l4|%zSD_#v7Lvw4-5f&OgjNPC%M%~qd{B4BNJ+$HvG>% zT{)dK{loet*1hsRHj6{AzJx1U;HWPgKQs76l}o6-X@XWEfhY&dq54>`_OUbpn;!QF z-)Z7)OUnN!1u<t3Y}VZi<8ru)xYTzm@m_RO`5PI&JQk`qeedy$s+DJAPVAjD{lt<7 zq$|8w3V?iz!I$k~>9o+}soTG1?v5jN=W>4!BwS=yV4r30?qII)&F4g43-QF=(+=T^ zlpzdz6C-g_U6Q*?MB$!l_WPkKNV=xBI_C;6;X|n}lB0`!y{k<!77Lsf3xQG*fk16y zru@cCs^4RS=}e8vOVbyPg|}^2^ljTT!jhNEJ|e(?cXBKkDiAEPhz?fzED!%ZplG(u zw|`~sz8HH(q9XG%=*;u;N?GH#K^2pwN>s5L&thsID29(@k)@g+Jgp3yP4ACjU?xR% zQ%o9d0Er0m8ji1^^Q~%qm<VZDnJTSsx7yORM+p%_f!sS)a%$zSnF{v0cU=U9&tqbL z@Q*qaz0}p>H{0gUS-POP{H)mc;OMdyKtc6`&#9w~d!_i~SAo8^bref~AzJisS7vPx zgnn=i=;BhN%}$TaY}1uF`gZh)pH`Mk{VrHQP86`kJxm0NYmlb?noa!4n34fgL?2kQ z7$#!20w$K+I_nJDeTfvLf*{cX1!);W&Zs9~KU$+hS~b9Yl21B6@KK$e!dniTP#kZW zhMC}OZ^Qj2FkB_`08BnjV;bn=z;Hw~>olEI(Jc}LwlK&gYBk|LzjEJB9)~Fopz}?7 zivM^|6Qz?eM)P*1ZfwhxK|y#{+CrsupSZ10(rtXnF$>4W;~AzX&#|9|HPP|A(@+De zN|y|A@x~Vz#MXc!sh@z<Ej+uRa8o8x-0WZbt_3g?VoSgF^4-CF-dW7<zv4%w`+q4K zV{fBWQd@ZJ6EmEV5G5Kv2<hoaVC(7Fvo%W#ts4kEpPs<ZKn*S!`O>-s9(LeA+bB2A z`3mJx+G!j?zB(qmU(oBD>O9h&+?^3s9pj@tOMG`W89bpoI;>G$qDGuF?+J|7m~>jG zPCiaJ1dk(v=Ah0o2o_|J>bL_v_u7{xfo<7Kjcxne9XA&&?V;9Ff2QouKA3R*P$m?s zq)Er3y4nxn{8h7>M>XM}yLP%8?T&9@!fU&h4lka+H1jY#H6tlQpVV#h?U<aBMy+w( z;Np}bx2?DuIm@mW0{)Ai4SkBm>mm>)yg`xRvyR^G@L?yHR)NE>D%bc^@C|?O+GkV$ z;iIvT2aKMx=CB=o5T|U`Z~Mpgr&=&|73Pq3R}iH6ThpDz8Kj^J9&&4#esfO?YgZ9h z^=z;B5;}qj_d9Hb7LkI=qKOXOkhkg(&FK$SOqCep_=-he-UZ<1GF)k@ed)CsFQ~bE z{tS22<zXLSzXt;}FVo()n;MPeWmouIUuY-2VDrN6MP`(ivsn`Hf;&C;l0lq6ndr2^ zh^b8Zn`xDXc=3&g7vT`@_Ae~%rb&Yud;qzn=KviTjA-r@kGD!(1AWOBr>+o~VWuC4 zo9dL7oV^G=CE^mnrOevyn>^N+h-}suxR@8sWSJab()*xi&j6}Dq$(RZ35U<TJdEam z8#JNQH@H5Y#F4J$7`~PY9QE40i(w*D1*1kN-DBU+1O!wH7#(VVqGnTx1sbqduS&ko ziSl0>@fw@up!SwCny+o%)mwr8#!}}Bu==yOO5Mz5x-VM-^_!PZ{s@Kex$nSFX$0`s zLquCaXS5DiWuLg>p<V;Y_f#>!5fIfF0x87Wd>}b}_~jf!`6a#{GUVHjIrK$qZ>v~^ zXwN=sf;Jl{^)C>Ke$eE#scdDXF9Ozz8IGWj-3JyDO}|ZOtFPP_v7t(ikm|zliW`Z! zDN))=^MyyX9x#bLF8Jbw{@f;Jk{DRydhc7R)s#hu_IAyOXrRLpcjoqMz_CNP#-cQ_ z`s8w7DsnVRV>K2Iy`@Q9ONY$S6m1#VE-3+Zf^Ih=^Ui1`+S@b})*=O*HwU>KXghur z++`U1K1!XjS}Y6vT~?MQuAIWPbFfQq_;;hD+z{|h39wh(jno-@T2yHS3WfL8l2*-= zu>k~&anud2UFrK6g`cor3Om0QW3QeNZdGnU^KIFHRTF&U?`3)IA>PU+J|I`F%hx8# ztxLn3o;z9xUq!vmkBr)=7w&PfO@bE~+A5&K9|8=$edX-ba4ZfA2HZxIntVQ1uP<3n zN<^LNZm=Q8GK=C9LVmc^J@lAeRgL$v=_$wfq5YnEx4-TRY*!~VxCULLxzOieuzr?r zLGWv8YapB|fo16(=9r0MCZ|0+Ib9OVN7+m65mM3rjO?!_h36OZbP+>*@&71wDyDj* zMJ~0F!&F>Z?0DPMR95cLj?J2!guu7uXW!QtVqg3Q1-F<~yr_Jew!+kRUAG^l=DY+V z`;*BdH}L{CoaYMQF7Afq?1CjoM5A)w4~_rrhEPM5DxbJnzJBy5q2a%l)YK>`U&)!n zQFFYXmJ2hm**cPzxf`kRQ$T-N>LHzJhKF8L<_SA{Xk{op`^CcHR{%-&_!2|cso1B7 zy01+EcV)-3313LJjW!iwm^Tlbt};g;m6`6BGitF{HUZTS*UU1S476UTGnSc|i%JZ? z<US9T(#1A=<UYrX!wu;lRb&m|ih+rf*+Fa1i-3)bUq|elWdv=?FFh)M?V(}8NMqY< zU!I(=G@R;l`=*l+M(Rv*piOL%pTE*q!Sc^jSQ~p2(JpdbYH`;BT4-^UB=k=(hwnfM zHd`4g?b}0MX8ihg_l7t7m$s3pUHHLjm}bR}#B}8JU98d<an&6c*6AD@zp<yU-*=TT zFam@nuOEtG=0uQJj_oCeB_AJZGyu0LG%7n^aNCsb5o!no&@2Rx_<eJO$#laFK|-z- zviMU;j`2t6FF4Rqt{z&Oc?FQ^H+GBK@csxq>ySp?4mz@7#^v-bs(Ff(rDOr?oQ%{w z<r)VWz$gDsE}6c5C$n|=l-hz`_m0HPdGmr-dwq6Xj0t`^Mb=+-j!NA^G%JkIc*L|L zPj6wJ4bvxi?j@tZn5|%3La|Ldegt{p{#A-_SddH$RXO|qJd_h{aP;|3ePJX&<<*o_ z3$|lXz)v6_*AhEwqS*)EEY@gJN!?{wn+}`6>-+bE58pyz%ARq>uU4K_-X(J{D?%4% z=Zt-tOrUrMsxwqJN3D$xG+s~D`=E@*j7F$k_BgM$t-CQv7sk}+nIhSoI@GR1{mYv? zu4jWa&``J+CuWbdd3(e62|MgQ;bhX4P*E9l*Jt(mtsWNq)wA2@C$fO-t$#^84BNEz z-buDrM<N2j?s}MP=K3wBwS+|1gb-3KxSf`z&jB?pzTB#L%00N4hHQ5sH0}}9MQ6F4 zH`$=^S+3n;jrZSoFK->9G>m&O-@6<aB-Wmvu{tKAsvC!QRu-$AUNxk3tUs*o2_%qq z&)*qDZ{U><6b__qd@OmU$0-Hr0WsxLW<@5CI?UTNI#DerZGm0BNb|wB+hk{AgEc+( zU=Ui{CjIcuaeVn9WqqRE{Eh|Q5*8*$<TNK-*B?(j&N=>~qRsD8V5{z6H`Q3H_Wctu zl~t`4{E75q;YQ0kvI{(M`~ku}85|oEgKv0TM`jy|Z{j_8?CsN~u6In9GbMct-h>}` z>||fVrAcQ-F_XWK_>*19TjUqyCrGDDPkk!e>t*YBQ&ZB6%8L&(RWjuATzuYBVh$xp z1KQ+foQ3^ak)<sqLZ$ZBJwA~kn+ds+&Hkc{Xe;tdd|6R%v>8Gn)3oJdixs)Pu)tC5 z|7zA^py>9ytVss}lM6RPw3F_kvq~O#2R<M5Yo=ii1R67ICwc%GLwXSlzGHneMrBQ* zz^1sO1SFTEu&dk6>GzWS<TX|aSJSNLS366tIJ5;2C2!G*(|Sm8N<JGd4U7cs4FsdQ zUN+w*2-9u^nO*;*l|=_;7h+fUrF$hRQesD1yNxJ&4d1^qH5?6(ly)}zwKyIs88he< z7@7b!wJv0U36>dH?#15`Y@6vEweG+7w(YhFF*^c^mVohVxz`TZLCVYvFJf3}8I*uh z6`XfNH(*lE{TelhFQ&V6ui4EXn`D0EFfU`hHhv<_;%$QeIq0Xwe1;H;IPj}8`+~V8 z&gvTSvDogAp&@~xbf-=t`Lnnnm@82~8SrUKW&4}RiDM)fB)-UVea06?#e0K9twpA9 zd!u=3Q0AQ^pe_Q!-ez2Zo(}`iDP;wFQF$}VIh~T|g&j(hvS|xLhEt+Tf@7=w_Sw@j z`X6kZ*=(weiyw>IH<_Y9?zZVUcY;I(lk(pu4m<woA$yie@4eCm2;-tP-BnA?v9|{| z;`TvPe=u;zSj&oHakZ7mF?!KMc<f0Wbg_-|7Zr?J#WNlm3Yb>sG&0(4J*Y3xtpiBT z(hCSBl7N*vzbaLUOuosICKa3FHmtA*8Ou#(6=jFAla=2+R`KC&$#PPn*0(p7vq2$O z*S>|rr41|A&BHdvzq^8u>C`>-rVTWFn>qYw$o7)`_6dqQ1}8UC);4DXb~|H7lhoTE z5_~ob1%~rG21$oE0=FM{)n%^liECZ=|6@;l>%8wZ%gIysL1RlFkDl~+`{tbThy~M8 zRu=#?P59ilff^JzYjd9X3|=r#31A7}*@@!v%!al`KRp5V?)KzUs^#vTU#r}w$XgB^ z%tR7hSk66XRJXT%77q2BCcX_uW-ZEsn}<BO2rFJ~ZmM@XhF6TjYm-f1iR|6_uAtmO zQQygu4(#=j3O2fa+R(v~)>UQTlbpRUnmx9s?$zroc^IYoYVz}Q?9vGW<)PUf1H@-V zoyYiHzwWKrt8ARg{DYwfYaA=rXWrNFc)#YjC?Hz3TP4bB3=(V-UOW6b_GaRw#E?C1 zhWp?tfiqTKQ}Jyf0o5KD>H4JRlM-7)-TB9W+VFB`x|sMNXO`)+W&SpNb(9UceBBR4 zl{G3UD%&>_O(yEP|4Aso9Hx4-f1I~(udrQEk5eYN1|?@`+=T0==-_R^p`Y^8j_3Z! zct59^AAPRLA<_oQ+Iikw-(K(9PKc5xSM8{GxY;CC$CNkU3W;6Ks-lf4Y(;6S@iRU( z9{(f4l}OGkQS`oHKjB`qagPdhp}zkgbTNAFz@+bOW4kphFp)+`yVASTNLo>V?F*sB zo&(xPQmk|=S3X{0r5k(ZV8_F|N5iL4ef{fe6iS^v0bv^e90!<H>;ICe5^wtC6ZmGx zzP!ll+V#>vE&nxu_KX>&sZg-AR`M?GJ1Gbn*fB$xxsARvIR<(n)@0>3{-%u?S@pWr zsv<)2(~|zFCyak>Hs>f55>pA{WB!%1D4fOQ*Iuk4sEfFq#Uz+mE4iA%#`yRH|6}~E zM7`n~sdJzzdkj(raF5iJdy%XCTQ_Z(6b#4*c;|PSlZW~CGs$hKlf>|bWp9fPC$@;J zAomM;jMjb8*sbR8ZCUCylA=<tw@NPZZQI-ItlE!7v5ejYNTz;lUwjWQ7f{=7-u%^= z++Vr}I$Qg=jY{evnwm8+HnU~%T7EdnbV}}3di<2b_lZ5FjOrKWm^Kyfy#zJ;^^HMu z$prPUB=Mz@2F1|QRKS1GDycJ6hY#hqo^qthH{(N}w=2?0)J~)J>2yEOVvn&tONiHD z7&J}q-X4?rVJ|#ex%TJIMMPz(JCddBkHqdhc#Xj+>KEsIjvnz@z4Ipwr}V2{mxx<e zNv-k%_ove*tJ8c&=eWI=d59H5KD(hhA#t=;hcX(DX-M2~Cr&<BN-37bF0Jnd+KNm4 zD-WhaRodgON{|u*Q~Rfj$=tNUA{kU6*ktGJ7|m@bR51^sy1S%=idWe?*$8k!EbURA z`pZ2=n_R9@Jf6lI7KrIq-xnfHb}03rci=U~*S#A@QLf^HDM7U9Rh{r1*87O3n&W>k zQd=$S1tJGG)cXs9S_z|XFk5M83D28NxzYq4675d_kT|fJ{myEDZPHG!%`FIKf5d3H z5Fm0$kvxP*_t9ylHEYfmGH~yLRI@bwMN6rEyx}Lwv5T|2DcI5O(La<Es0uDy1IN)+ z)8QR)m*q&_ZYdT|UhFuG5}6YVZbP+wG(29{+x>)`j3TWU?t~ej&{mbd&6U$^r5>+b z`fc2f!1$<acH@tlx|H3cHe@E*(t31`-XCJN`ocuX!K<1pY49ZyS>Y%ka8t@dOlg>~ zpdt*x428>=4w!TUM<*zSmO&{sALRNp1b5h4lMPeK)8L56UpL`{OLsRsU-5tZf9QJe zcsAd+kK3lGQPdu>TP?M#NNid~RTZ^byR~Y?3PG#(-Zj#qw6&?NXiEgKs<l^%*dvLN z5IpJc{{8Ofx$pb?<S(yRUM{ZlI<NCQ&f_>fpEt(;(_%eHHP;ZA77T&)<2UWL&k-?G z3q*va*;Y~wsxys{w?d4Ea#<%*LwfKo;R5!)H3|Yq_uR4yF+V}DqlBNx;&wwd^!mv8 zt_-(?T%aGM6E@VNkN=EuYJ4&uDl~|xWccElT-NFIa599&Uj7Z&*7sN5VOr{Wr|Fui z0$lQX)%kr&incV)J0$1V|JALsRz}LdL8^J&bY`3x>Z?|s4I9TYVL<_@+Un#wd}i=E zyyw;Gyir(oSn3^oXNcB$J8)9`_Ins18A!<Pt)Q-!5GKD&!CC)aP1gNS##5yQMCv&Y zOC#3*>UZYO-&^_jXFi|grkMK{bh@bD*bv?sqx`CBe<oGGEYXxV*aJ88`~&Dr2fyI% zLkm623KZXAIIzAUuEDoxdA-JH4YGF0EfSBov-RqX8mUHTYioZX<_%xo>^hfthwjpK zsTZaGu|+X)@k<t`fa4Y{W+vyeqJ*R{FXuBq*{+_2Uk^YYK7LDxoxd(P1M|ER?0B)6 zP-k&2!DjZ<N7}+OCHo|){($Yw!an!3#&4jkXkY#`2txHC8Gm5oJ&<PFJ!@I}NbeM) zwHPSy`)JEDtIqRoQQpOXAwO-_Z}4WGv%fJx?Y5BQ_Qq}=ADhA-`;Eay5L{aRB52g5 z`&S5XXL5xO2GJBBB(k5TS7l2!{qzblytvEXAgwjFc*Fj8nB<|PVEcq`e@Ir`ZA#bH ziGTx<+n=jmjNmmk`|M||$`i!8eC+t)+{}t}Gb%ZW%*;J_d*?ajKib_lxmPa{OCo;2 zUs!#h==N|)G}2&U5g!NSVe4tAipdi;w4)-MjPAqLgKf1R%l*8(h$VaDyE{v+x?e6x z*_H_Fe>n;qH63NU>HL^aIL!h^!?eC8moz7l;hgsLD^5<Er4Vo;zt{hR2_8wpwRtH~ zGm*_?sPuSkj&lCw{QV8M8%<;hcB5qJK@a?4HNm~2Ckk-o&9l{q>g*cfIS-s=0+y}5 z29Y%}klDW(Sxo8CgX<pA3t6slZPetSjQtEs^i3Hz-W@+IDqq0gD_ko@wh^?(ciHnt zLhOY@^z#q++g;Ar+lNzL5CV=5_)=2v)#oV4;Ye9U$rNm7C<WrpBjIpy=W$Q<ZwZ&5 zOb=$;K6w>x7ht5+eziVc1#BhWc{~*MT@5+s5)inM6_e_2Yh;x34=o;YV<dUNK|gqW zS~5d{Tv{`jTwZit?6x=HS)@&kdz+dZa3(u&m}~aU)K#*jPnO+<;jNGK$Q~{6SvKty zn?f@`^jO@uqwdpnZQqyYlgVh&$S1Q}JB=I@tP$`@O{=tyD;Z4!Byv-mzx)cmvad!J z!#_<O!KIN*c5y$s{{>R@%%)*b{qV+}OHX@Z)^2>z7U{g)Gt~6maD{SO`U&umQr(%B z-115bB}omh_om;O>Pgc38RY3T%%2u-X1l+JK;Y>Y{G&MpYk_rA1PZ^eMw#Ye?U88w z;HRZWz9-qE`e5=L_)AAcp)`Rnt7=Qv_7)UUXdvW$Hp;GB&IR9J?QN-*c^TH}FU)R) z70Z(q5Rw=zBW`{j$s2}KZBd_@t9K7{NTuAYKjXLNGIWv*9$6>pbVr+&ro^mm>-9H{ z!7_p`kzbAYH?-k5cE}4n^U9Mw%6HWS$PiJ8>f?7*WO)TmF(yZl-*?B)*X7_%q~b>Y zU2c%HFQaL*O5|zhX$V>rM+n2d3ulHsAAva9A1WypMw9fLN!R~}AreWyDcN10VnUh3 zY2s0KF|E{l^;#2})t~K`P007A4d1&<rWyw~W`AuaP?QC8zEDq@vP35&76dq7zeXFe zByqL%lrHE^3jW=mQ?AG0q&dsNy2ervr%=mtkP~DX3cqFIoll8G<*L97t_6>k@_TNH zoUZd)+=ZM}u8aWt3%&;}{fR@;uD7_v+lpdjHOd1a1L;5X7qS?sDEP<=oC?Uh@HS}g zZs|P=bGYSJfvohq;yRPEG9(rE`ZT?X)zX!H7CJQWvQB2dfnY?V6Z(_PbV&W`lzS0~ zqWV`#%;3z6BjGsjHVd+_jYZ1q5vu1rv$fnpoTGGeL)ptj_}A#gQTbaVjh!1tUl%GQ z%+zNm$(es^cX?EMkC!_Vg##M$2<>fQ3}m4}43b5d?jI@;ozZO1LRy942$jw+6-<tn z3(p2XWRp36aQ{++PUg&+&5g_lN}qgt>X}zza|o&Zbm{-9XH=Zt`GlumbdY4ut1M^r zf-`GusYYw%xS5qN?jsK`bn*S$uRe8Uw<fcz^U-&a1Ut(8g4GE(vQh?j1N*f?N!_<_ z?>?j{(w<zOOnUb1K$MxZMoHa$CB6p_-c-ft^C=(Ol2qjFCPG=;ibC1<y>M2}U&swI z_!(lapeQX{hoRg_WRhe#l`7qwOQu3EX(oTKM$*l`q-d(I+61tvQxuU$M1bzlqK7pc z<!(78qYGVPld)d*q6Q3%G#XfmJ#|WjfdggEa*)z9hUu7@1DGw4!~{s$ew0_XneF*e z;80wO3vsD7QGI_E1R`0f(V%PwkiVO-n=~)d*TkJlNqRIz#X^|0pR~W#j)G#L9$N4J z)|r_&+zxd3ty?e*^#wWW@Bd{v`nQT#A%tUoT3n@X55e@xx(>Cfu|Mk+9wapxBo98J zqNfnf3YfFG`<VBoDGfi%!?NP-4kvLuA{NKgO%_XbHxFFD1aTjq7}_L%GL4>iOp2N` z9?S9CGO#p=eMBOzue@hf*E>cJkj>C5pbOvr<p+>aM<yK{kdPBTeC4oTCCO?i3Z%3C zbn3P~bbF)g#-r&E2mR7S%`yP*cuk`Aq-u><N9`J;@Eh;GENZeEN}eo7$Id=<OowjZ zf6SMH|3R1sph&vZ6dkCUiF+6YYOMNuqklgMqJFphSZ2*Bjv^kR^BN0UnaMtrGWbH! zLksBPUy*aSEccrxhb+^zW8%5G;L=sHAs-xIiM!U6rZ6z$1=|ueisCT^?_%&;;aJgP zuy;ml(-E0hcgw)V`zy92OLoOOlu1rBqce}qvxRGOcRLI-k|}mSX$PY~U(Yat7no53 zqr!=|yUpw`iYw$f-5`yc%105hXP+0L;Ta)YHo&Ji@prP{B=WMH8gwh%KQsMDExXJ4 zkD5}Ef$=}5rQsp%pZ@jPsDDnw6L~M9$2LN5a-fKkeOhZA+MS^yu=F#O!?Wa)Te>H0 z$Gx)w@;oO%+hgA6hTPo2^+M#&|AH-ObIXh)!~`N7$YjW}gO$43SX@*`^5trt&?=x? z7&W*frzx_|odxUJYA1Mlu2yPT@3j?kT9tNdt;sVt!=Eycl`0X-t(zEfrIwtfX3JsW zBp2Z7#+wyKp1(iu<n4_|=yfbHj3TR>JicB7|Ht3_4#PjZ%#Z#WT>RJlh^MOLs)(i1 zemvf+s7WTc+-zoY^JgNXqoRD-|Jsky^00Lz%!|xPLoy=kgxo4Prg&R-l5}%pLmF8} zPA$((CIm*upid<qU_lo+5Q%}1@K6X&oO)cPVOam@oH<L0JZa);#2xM!f9`jzOZU7< z!YN88D(o2v`PV<?I}totdpVTsIzMhfch7)oIYgqgF~P&^ozub&-R;s-)xh1vI}hQC z3st%IfA4?2<VGb&3AA-reQdocO+NP@-TN`y|Gh(!#qS9V@1xJo!}n%?H5=vO9%k(o z&vsOTt0~Q`q-+D;d_Swc7$2C#&Hhvh@ED(X?z~AbayDr%BUwM(VQ+K0pcbDlIyl;y zr6l{jMh@-nmI>Q1q#R`!jg5$%26HTx&cQASFy_%KbS}H}l2kh3Xn7d-qK2V$Q25C> zOrZM%vSm(+>>-(A_srt?&f-F$*#(UJ$=MPuh-;_CPIWWKs9a6M=%_SNZ^>%);z3;a zP}T;Y<!$f6do|q_OI6!C+_5BZQHt?G$X1o<<B9i;l(yqZ-tUCNLiUO);em@`ff0?> z$3Qrg*azjCv}|@`!+%T+bo`tWezBf%?&g$qUE)WYx`%sy;;;Q7mrdNcE@Vif1Xum- zUpDZ=F8}e1J?XlT|7#ENe{3{JSO`rCO<gOxJhI$h+agOMAA9VlTZ<T03v>J;Ua<W! zOl2Gi4(W`#G-FQlN!4UI!^mC-|8Td`R~#>K+c3~i{7r~6Ctc;8@;T|99`Snmrn=~p z#t%bzDT6{gLm`({^w#hfBt@1YzJwE~=NeSF)f($V=Ix&$+fdYXNk>|)x*;-Lac~#J zyMNPpcv)uCkcRcbWliktvo-I`!zQo%oVF=_ZlLUB7WlJGTGf$M{RhyU8eo^-kW$!l zi=RvjVUvFO>mBDsjHLk~^U=eDgJ(LNpDO0MHhns&Z_obj1B;#YPUJBI@k?Y(TT}0# z<T+$v-6b7{%p=4HXx{w+rE(3urJVapOF5n8T_F}Fj7odVkMx2gxL5|h-toM~@aW6$ z>q4?C6sec@b8lMx{<E-LaLU+yFF)Cwz?8MFoEPboefe#oUSDJzDtqJ7KILXV%cF&< zOdqb)wfyR-hgUaWG~OXclc2317d`gW95|I;GOg3ay2uF0K)=PvW<y=9C-MTp#|!!z zAC6UBD!;3CdbC!ATcvaE{I|bFj0T$I`0&4bok)H)0zFJo`V0A@K~AUF$tAx@g<B7n zczR@)qrY0*t$lIHgI%=YEbpmV)5F<E&3@uGMID^4?>|jotOqQcoWw6>-~39|jQ}9T zLbbrrjAREj7bi6$FY_Wcu2ik)x7SntT2p=Zp{d_c{F`E=^$s1j%DucPO{k)K=StA+ zoMY>EDIL|OoiwK;6^Y~h;#VXKh|spfn`V*7pcQL;N&b|k%j>OvamnquJDnBb*Q-=V zcAElA`aC1H0}hdE+J|$}&coWjk0AE;9~SWe9^K+mdzXGTWL2{F5y~|bScIMu%bK(f zKH4)d^2!|Oo^5xXZ*_(L=$bF5s>~X@I*^;*^y|Z7eM<OunFIdwZkd8f=o&C#zsAn$ zHn+w83r6yYkFTJF%bAFP8H_-iSa5TlwR#OxcS6WU-O+eJX-a*sI7}^xc||B}Iyu!? z%6m3lmfd_5TxPqaUPXMqHu3~)eaEn{x}TE2T)pLG_Q@fC*!vT4m@l-B1FhQ?m<l;b zs#FT|qB&==OI(8thlSF#oR#rTdlG-<dVAcW(Ghn7dN9fzNtRUZsBQhYL!1qIMl9-M z4A^i-OTyI-#|zz)?rKP#*{Pa$YKxzUHOr=WjOqIi=p*M5#(QX6;x)%a-@|X$@05o& z-7-EY+nc8K8@UBn@t#wJ`7j&I``7G#z>N6#-DpC#?6v@xm4m-r;|<t2`LCV1tn4%i znel%uQAi%6hdn83m~)akrGVs!!7K6C5rSqKJg8jjM#GGbcz8KTJT?m_t{0WvWTmZ9 zu9Dib8)aTpdO?D=0X6FPA8n1ub`cV9(3$K~Ufq{K(A*fBus+Fun%|u(l;itXx@+>+ z69$kYf$cYN*=g^jc>mnbmduEEQ%6oZb5fuB8Hf~&!8!=&{8uB*i{YV8oCq3&GL21x zv&PE-7l-y2hf78iKOxoF%zoZ?dc1xU-O^u5kH;l7drC@5GC=rOAg;EvanRZLdwVOf z?UAl2G|A%tA;$H&fUMz&k5cQB(&JjkCe77B%zpl27Dy;7@WEH?fK(TULd~PYC)Yap z=ZAM{q#8`#ZWVblf1#j9xTGa%U33zT|H_3HJoWl^dC6|)U6V_K;LIj5gHE(D*$2DL zPwSpG>>QkLkDPmq{xx$LWB%DkXnEJt_EqA(HBs9ta%b)bo`$Sf2~z(q^S3>=Q+5?k zs(qEEws~}Ygi<}IO?vnAHsLVzFoc$Sfp5jH$*L!zJ9uZ$5If9%)`z~7Ietx$vKHA{ zS1sMf`cZj7Y%nk`Wz$;zdA^rZ^oHuCrAeh62>TFFXL9?Kpv98kr(vTns1jW3z8p>w zI=MxJo2>#SYbv_1=7Tq~tlNNnSkSaT83>}%CHg};;1rxxciq~0I4`O-XJdreH1XBs zVKeagb~Yf#qotM6PCl@rLvpKNE5atC@~!T6vwP(109h{H*Rv0LtmwSvS6wdX_smfT zesd?JR{u>&=w1F1FN!g0m91RX!qFHNjza0I3e+7^4-bF4dHZkxuWP1r&WoCW%&p(n zeVnD$+tQ{Ze`D&upZOHhvBK+rYNtr}l?t(tHT{Un<Bm0Tkz82$A7^SZ0M$7)WfsSc zD~VS;IM|1ptTyjuv)NnN6;i}eY<7Q*5eRXMAmjK#r9&RMKyubku9WQi{bvRrOpy`w zbjFA8<+XQoQ=`>nD}QNI6zEGf!zKeO+}-;<Y$l8m4C{wEP(ltR1pfm>{6Wr6rYFq2 zaG>Zr@av)eklrJ^eT&C5;TZ8jN)|2g!@;)h49^8JNjx(9y?C-yx=&e>2eqNy9kO9^ z`-sv@PkAZVv(-8&vv0#Zn6PPGGp7m<Et*Db>FFLXv!xM8{f3k_RA7oL8Dj?0YU8hN z;?`4lr<qeX3$`d9U0UkfMLAPq$H%t4RrFmwN{~OBJP)W)tI(}_KQ_J;?{W7b%X^`y z{v*GEkw~w$*qUqdxKO*FG=y~k5Ee~q=dM>0&e$<7aiED^J##xbSR@WDGD6A4)sE-< zj+?%XPgF8{rajL#g<+FPhcMt}1ZuYa-It-1b$zIdg(t-yJdPxiA(LIVEY#q5D%3^Y zLGEJ*Jv(Fzi?#ol;3;Au2S(87^n&a*$jOxmfX;$Y!fKAbZS_hZS)XfBuuDWL$u@!) z6Wp{4Jy4*X4b#6kcy=EF{tyeR$SLx>-(|=7MHxtvQpk#nO3f;AuepGJfwC!1tROtl zj#;J$_f5F<9}3PAKq<E2uAx@G`?M#&Tq!MS`1V1w0y2@B^1Av#FE10ucjxSXE53Sg zewtLi{#bRUgn|2G|Ghx9clrY#!q${M&5~#*KS|B1eU@^z1s$)E*|u_yHHTwo+-oii z@pdJD@NQKiCHb`xhujw5Q??MP-e;3%cNa^$4!lW!ZYuV|y!FWFq=nWMZ{D<C>HKqX z6B80{)@nQOYsGL97vT7+bL2Q1vLrrwCk!?B{Elt0XLW`1M3%MwR<%`N$A5zBa4|m; zapE65Yj2LJg07IAk&u(V%%zT;<VN3&zCpNEqmtwjB50xr!{k?}qJ9Gq^g3)eINoJC zm#9TtA|sEbFs7rDs`2dHeoK<?9{fxE;EWMlxjP?Kb!m<vNI+0<k&>0VkKBw@mI&cC z?mBJn3dKgtK>jR%OOh+3mQs$jE?r^J5^rI+;m`MxWERHZjiN9{+)7iV*2tGPLXbTj z6Nu1!`8Z6Y^Y7^*!tIjN3MY&LLB@0<&h~(^wm3{9VX5(5+a>3EvB(l9i>@Wbymd|k zK&Y&>{<4(@PmP9INc9mgiRxlQeb%s=VJhYe_q(*q)WMVn@0@ONHAhU!sq;JPYx=c4 z<;0RvIDaG*-gIgBAg+;jfAi%Hk}lzw_$*o7dKtDNT!`y@-_$vFZ~5{qWb=mke19h8 z?ON`43E>+8Eu}KG>KDQiP-l|>Ny?T#`i7Awwg&T%Xy`uMxmMux=M2Vy(VJg0##K5m z8I)75E{_L$TR`?nBIJ_Ji2KB=w+sYle#-pn6OTnkD4yTfpHT#>5l|UF_6XH05%&X1 zZ=M976V~(>jlw$n*}KXO)o{#ex6w1x<(XOBpBVF3@?`F~@^v?af0Izt>1Jw$Qam^? zIlDfcgDT_fE?q%A9Ohz8rGvp0z=}thK@wsx45%9N$eQv(#CyKYrsb(`{Wo#fHwCZy zxzqfHgAppP`c>9a*!Ri5o~z5ae|}!-S6a6%|JLgB6S0lpw!pn#Renbgn|&WOh|q6{ zNc4OQ@FZLVa`YiN-Kl(!Nu4@)>?+e4E@iR>{`O*f@U043=bV5t+`8VCFM-W~xJUX_ znWDcazD}(C;Bb9y<Hq@<ZQy+Upn0gTwq1t%Za=g8r&hVGbUqGFzf8eH**A@V9!1qv zfjkcd;4Jc)dz1}1@K*AC=uBK(KyLNSpkwQ^<TowyG-qG>Pws3XBo5d>x4gnnO`t~u z(|Bu-c~19MhiZJ(>-l5!)xzn5lCISU6D%R)u=)P5Nn|hH#JG<9ZspA=qEd6~Q2k)h z9+G+o!mS_P|F!b2vNFDBZT>#m?(;1{yj~(!>`kkY>3^|(`6%xx?8fAg>duzzy_okp zE1CjgJR?^{#@5##>A-SJ-C`7i|8hOqk&UM87W<D%Nx38jKm-ayo60X7E8L#;#cca= zsd)={C@q@)<I33k_=*`}Nz#O9{t{qoYVBhMc<Z5S(*B1%nB#@nQ;XOU@2k^|;NRvJ zH+!Gxy6HYSPNRwM0r2e9%B4(kopOZk3_o;pjd4dXPqHJwiZ2dJ-jV9KE+CaV>+?~g zDy1_XNwSbQl(U2IMMW}mXleHdY2nRSZ9%>6jsOY0VpgB``@`js-M3sqquVrQEaHNi zI#;j6O8a&6wRlJQY!CbMiGbok7ue%62fuClF)oV10;Yb1N%dG(S0%~tLstR4Zmm+< zRpP>NLFf-S?zNdql>~cV>&_@QO4i}v1+OU!>hqh3@DR9FhF$7d*qR2Xx2Y5a_|q)o z&-_^_+o{zPu7$0>?a)zj6M-SGDS2D3pt(HSQY--($sT&*nCLkMjQo)++s~D^?$uVY z8L6@k4t9w>St>7LU@h(X7CrQ^=<O>~V3(NpDnjDuS92L6L}Aj1c`Wcqs3!2azqo<3 z=}Lt-ToJbAZIvk^x~)0Ul}weBpn(+EjMnC~MW??ZuF7h1iSVSo^Zz5|p4ez0{c}i- zU8HW=8l%jt9n?qa{@5zCQP^V%^;K-2NwwY`<XqWLQi-)3lpc#v$0pURKkIazR4h)k zozMxF_;H(VH&|IZbhYfZRY{dOR;%Cu^Hv&tE7;$W0UCNUXZ1}Z8zI5NiIN``p_pWX z%C#Au8L_~r6*JfmnAWZ8vuK~yXP82j#<MP}Hhfj}E!zdtC)Ek%UANy@HZ_SHYr%#0 z*Sk3(<y9&mwxs0(kyp50rjXi^Agp;&EGQ3~I6^35B03mxu;B*oAXEI)a1Kj@*NM{l z)m`T&&c?Bx27xHwSRcPAnbka^Y|47T?Cb)#Uw#Ff-<s(dU!_Ud?ht!gxqfG{ojq-2 z)d*((Z=9~r4a7x=7*F62q<ps9jre8!gJO(O3JJQ+5T2tS|JC45ueGj)Zb<LT-bcM= zsCQ9q%gxKz9W?9gBP}r&j2x>qUyo)!)YiQG{8HMUW0s?n!?e-7U<C5u(c?$vLc_xv zA=9iSwV!LWqzDH3y>l$~@l)~N5!8NKtVh*>U$?YG-S*yUeNl@4CNk!>jVf?$y2diQ zgLsVU4kIyHb(vA|vZ<UZ4q3Jb1J&PC#ZRz#^O`frb0&NNGfPbT(1BSjnvyzDHDr!? zOcJ<UJXJlK#P?4AV*fdqzTS|L<Fpah7tNw^LObmP7Y~0pg_fcQ12uM?mB%f6R@+&U z5azm&Xur<-ANr=Y0;7U}ct(XI0+he-`uRyrsopJjZ^rgo)aAVG73*`fU~g!MFcqr> zoJ}NHb~cn}OcLx-%3Lh>n)zV_a}OL3x!$yZ%PBO)CWZU6$p_zO`q|wA(a5$OcIUGf z`3=9lKO`=!sS4Eop6+=gI8vx|!~iZPaFA7#??D4p5`Q?N<`N2Cz{;y4^}z`3^VFJt z{?c^*U<V<S?JFqPVCA(~R}~Fc!i$~UIC;?$e7~ZPa*gkojVTv3Fgv%>m3yQTP_7}M zv%|!$URxj?wpjJW7+*}joM6BDD%j=BN1dvkcz^_BvgIe_N;40WnNc-L#5l;z7dBSZ zD_Iohm5jxtA1*qdf|GLPN2)9w@?_)_!&hZOUN|z1+7ek_b#^=}656hm!nnf6OziqO zgNu#8@^2|8SUcII+@-A>*?y#kir`hd4iVyx4I_81$0Q~}v|kQ5@5yJ|;-(AV@{d7~ zIlvgC=lK`;g@S?MXJPpG%l?@q#9IRS{U*LT9;;HbjjK|vXg^Wa>998lDHzoEHcSd4 z!dY1e(OFFGnz3ZS=K~L1DAgh0c<Qj}XJ2a<Vmel&UYEO1JPzJ+B6k!IR9l;Rq~tMB zUi@!bxU^uV+Z?m}h-Gn|Wc-SD9Q>8qJ%ys1dc|Yj3fCz_ezItRLnL2H<%;jI|KiZ( zc)zo0I@Ktz54=7noX;M?VWb@*DInF+UJaY1@1wh->rNf0yUpIVku@Iz;30qJsEoX? zAp9w@2C)<L`1K`jv`wsD^h!po-jN6sOH%9=YTKUv7T&}#h+}!amK;A`q8+QHAZ*7z zshnm9A85ha$=Z$CY8DPb;zn;tLsN7_mW}SG8ZEPO6Ey{U{T<_(Vc5wsj2}ciG_8Vu z-r<M+H3yusmRMIspA2L#jK~@cXyOaY=Z88;)k?v6nc3%+S(AuwJXknIDF{yW{B;p? z8dm^5EMbAIgSJ=gYpt1tgL(q3$<IZLO$Bur1-VA1Jc`-u*DMWz%1RF1>Yn()y-$G6 z9@}sy<$_KRR{e0Nkq&r2H#knS+ny-joK@p1*P1?G=%KZ#1I!IQKs6ib_fzn)fDJfR z`e1;xA)nVQTFL5I-qs{CS?JVVPT-)lMq9&d!7osFGXH^V1ySSia2Kqs5L%oni`6RH z-+&O`(>~N!Q%`B9ZoJ$tp^SvFBpL8DVR@~m`lOpi&CmD0HX0Z8hEuUzxTtG!jj6hq zO$|`0d|w39ENs~wKk<D&z89U`=8{uQWC!UKw|W&+npER%b_vhG)vz9&4L7@F+|^_i z1b5mR#IdFZMvg)<tBoq&L;Fv?>B1%}M-DWQan$x3-jgN49`kQKSM)pO#6k5*3qZEi z6^O2Q@U*894G6G%kapTCt&dlPDSM!1`7jB1Hba#Gy~$qWA;vo%mE9_$zp?yP7DBTc z__9h9?%)~=hj8l{W0HV`?+I#aQs4~Fs-KS&S(eewm?@-;@9bK}NDE66#91Kvpy{fL zl~9-8y+D-#<DQqv4{!dPKD~_9J)e{sYvFY%s?%{aW$%l*l0+Mv>857A->&wM>U{6w zeDA}p&mX-4Q%E4;dz!YGN0C?K%GTqSG&_V=*s<CQCxz<NKEa=-Hkn17VzioyuLm=S z@4w2S01^zY_1eBp)H*Xz5Pr$-r(f(do(w0ggdAOk(QZzYcXdS0w2%=LGNJ3AZfY6* zu{2$n^7tdt>!+HWs^I^ic1=pxIYdO5y^3Sf?EVN>{NMyx^daA~{_vfR)+bfjvaBHx zCAOsaX=1V{-x}krHX^8MD#`Kz%OblsI5zsbI&>7E8uHV)gX4ryaM6=HzudE17b{pS zOT}>I`uT9Rm}8Ni!ptYG;<PYdva$Ct2Q`E`hK&<D-5n^w&!)<j<T#f!r02vWh++=L z=w-*(6l9iQDaoI9wwPqw-Usk})kg{a`a+G<WlHI!mP0W@#h6k>fPIG#to<0C>j|m| zx#FWoR=o~fnEAh?QPSn*vHaAtNi7GQqnr@TuVY(dzohXJjjz_{3F5~aS{;}~KJ;*2 z6IDXT2KCUW$hcPB^cWS;l%J7|LC&(|c<@~E94+$3tG09NFAv?W;9X0#d_$%qjb-Dt zbyPk(RK9Ax`uP-7b$dSn;xG6~oD(t3Wax-hu@sydwx7`UTP#o%fF&zY$&amn^+_bh z+9z{M=uf_F5W#HP#p?Ou6(_$owa;<h#;#KOWOh}rTFwwJ%d#c;%YlYYQDlp;fUnvw zl9l>RodnN@?gB@PKt1sw{1kEj2`tSm)6fC#1-zU_=b!otXzyzy%}aQ*J5jK$G~#WR z94^dU_ktg>>o+;1dFS+d?4hy1C_g!OaoXN?#0QgDJrwc20+!=qa}8j@nhCiHOCx|4 z20kGhx7`@WB*jI>J{G!r<h;Ta{Ck;=U&d<x|4rA>SorMxE|}r@8wIV>Yv}kChtFKs zT4u(g__XLH)33u*RC#u$xS(ref_m}$@!#kSdSZ&JM8sKX3~scmBY5NVdhL6|onoV^ z_@&}&lH=9l+#VqU=0pmcS(Ct`>35ZJmsCC`Vc55)t40$Q5SW;4iyM#koedFMy@Ant zd}8Z&E~Ylh+>B_QV(ZrXl`)j6cr&O|&iBS;;F%I@QZ*oVd)Jg*Vmydg@wBM88N;zg z2uM~Q%~u0n!tw^wW9_8Mq>4mdXg$F|!N$o1wqn+_m>UQT?A-$<QuttS-G|e?sb4n? ztLh5ZAQQQVl7LD5jB%T5dK23YDOvo;TT6W@szB@Ee)$$e21KIy$|(uDTI9~%>U1~A zWJx-ix|~3jC3T>~Cqk!1k(Aot@y>o(NLoHm2ZnmnBCHl+pL!R-!>T}ZbF~a#F;eT3 zB|L#_yzx&n@y}p``~gnKayAW8h9BR6X>$P>d!_U}d19DR)sSZi0nsL3G+Rb3j8FKS zGESd*=%mRuh&<$*Qn;dbf{=M#6!2_LmJ9ofuZbx)5WrI<h312fn}x^4&Hc;{N8HiL zZ~Vb|9hDhCf2}9nSBs0JvIcKwnw|8&+TKO+s(2N73G+^0O)oGZL|oW3bne#$jhZL= zCLY{>=gg1v891n?iG2?#2Zc3AnK7;5S5wObwmC(>sQ~>-)oOxoWQP(nid|i4r;@EX zwR_^a-)|ycoStm9s-46+fQMk{g>am#3<JkL*;`Nf*dV?Z<m}mNj!arfVA&1``y~a_ z-i=fFilTwxxM5qcfV%>gDTJ{$hav203QN*1fkLNq|C!p^`iq-6IA8N{1fcV(buc(a zb?zBJt^cZOCkr!-VS3E1uSv#c)O35wb6!&+L-q9Qf2Q&vgBZ{8|A~bY(%k$NioaK> z8L{5pLGe;(zqgPz*W|fds=)~0!SOzlAIn|5VJ2y=V@_pJdB_v|A-3u%UvU~lw2c22 zeOtB2Ez)}F*^nWjQ^F@F-5P(4;O*TcDD(8!mZaa0_KEgG6)$(zu@nJ>_@905uFx2w z@nwI{`gB@1Heyr{P}F5(#;(sODJSBest|ujkB%QWk-|2toRc)=T4bZT)=IpAqCBjZ zm$jjyZydfBH$<smImc29b7C)ru{ZZRCM%0J0mvzjd!kI(7m{>=ws;deWNE*F!i=rH zOsAi^-yMSDl_WY5AoG}kw<z^ReZmghImR|}`NOD;J>C>~OgfZG>qHG|)`?mtV_55B z?>mYVv0QS9)X;w+#F6GRBeahKsi&G_Z7X}_x_JA<Wy>4*TV^~@jM|N4BYYNmy8v7G z@9y9r^<B4LIkwe<d3-pP0j$dFQtg7=X8#ph;2#NGJP5_&`?vsaD~Ri{M9BBV<NIwf za^5J#(4k8hB?-dAxgFxK>;?Z!8Guty1T(+#{XoZ`{cnW?S2$6e={}85ZsI&eZPQ;N z%8<_p*B^UiNx&7Ehg|*ijAPlLPeP|eTxLf5EDZ!OIg>;gs0ihIaeI_N*pT5-T@3%U zY~Fma<Mu{^lU!Nhmy_BNi1xz}v}$K<@_r#6iD8$)y-6Y1@@;q+*YZwB*Xkg-Q=q6u z%jnlfwbDE}VVz_EIKc?MeY;sY^&8wsN5pzM@!hXCGF6|F9|;61cnR~cVtudch!C(l z0Onvs{qBG)41`rz<(YO~urdpi9t&6);%b3Qho|F!QkAW67AefCAy{~jpm~48Y526V zm@+JF8O;=xZj03}fIQc+OY<B~{Sy{0|8ox0OGedI*E`_KweC2dIf+6T5k`jjDt7#w z6QTc3Yp~}?{8L^-0BZnX=e~6={)aBw$0@sHpIwjRH~F<PGMP8@yI{WS>lv)+I<ofG zz2@~SLi9iA*W;UgpM1Mo_YC66KF@(=CF3^1T+Frl8Gb7y{(jNOzQIw|@Ji>sHNZBK z>+XAlswCNs@~7_2?~4_zeZLeI*L|-E=E+afqhE>djQ??<-7RSuLwxYAEa$r~_#|&F z{+9t&&-=~;2iotiKq&QklJuvRS>^TIJLdY=e7Y~F-J5iz9}1mtX@h{WoyE;sZAuPz z`Y;OnV&8?RGH#Fl{+g!7*r6=#OK){69r-+iSlm$DddR>XrK97rUjp<nB8PRbs5!i= zIf_4@!Ni;z(8~7jRvKvlQ$HbKEtK{m@l&zCG`SK!1SLfJq*40jhtppW3PB_O`jBxW z<Ixze6Lhow{K^QXnd+GO!l^mm<~=M|!~jW4Wesi^usUW{KM+_&1+E~aUaM*SATBfb znIe1dLXg@jB;&&FJuE$|1K-KMdU`?<Hphz;CX~)8D&@0gBK$=i;OPx?b4bUXn)cl` zT?w4n$*7qP(T{qjU$!X;_ox~$R4F!5f;0&c?q|_Y4pY2HG;#%|TVoBX<37#ZDxY%b ze37H5nLkzGy&O@hLz6d7T$z5-1I2cS{II|oE1aYa)f{LIj73BHWjpb05-G@D<c~40 z^9=VmGhYkPRuRef0^~bS)KWqp(`0a>tHKY)jJNS)A-!mpa(F#W(m6Bg{Sh3k-@0U# zj1F{~=VYrneE&*D0CCwpDt8sOxvh}J3>aYzKUr($OUpDc>q~kfF%F%$TnU>HtY7bS zEn<aPfx?U}UDriw2tYz^iAz|rn}~h=ENSjiUk(0K)ag4kBU*-?<Z({YB;&O%0sz(5 zn!T&K(EQ)+Ix(O4uqrP6QlyOBe`PCjK>t5;+3wnXDVGyR7cWYz1;*l;g>`fY^<&)^ z(f+}4>p~*Yhub<@axO82QPqxIhh{G)Cgi789DXXIO2G8r;s@!+2$xllS>LL}n+p0d z%0^NRV1f{)p6tUL<=wmC-B(ou>J5Fr<QT_qB1EX2ogxauR(vvJC8=FQFz?mSCXQPb z28p*HNb0rK-c=3j?>D}+WU@pH988N*fv3oDOdS4UcDg*zUf-zbl}sLxVt-U<0XJn) z9)@hAm0sUK>>Ya!2zGLKAEhR=q_NR&oRlL>yj3_(do>DH^SKZ*Ve^i#<t{D>S|!bG zRz<8^qPjzrcQ5_AOSv$AqT|{_2VJ0W{$*c70d8KEl*Fga0r$gW<+UiMZVM`h$}WLg zvs-Yi#RKxiQhvGWSc$GkWi9F0ic>6W8ad>Tx**0pc^fr}e|1541w*|b5%fu|A<{#{ zFRn10?3=<7^7|tEy<=@cS9c}!=y-IbqXukp%<Ooz!-v}^)}5;~vH=i+wxFMgI~~_6 z#wKC&>9;M<b5wfutx^ziMcY|1DAA_fwU%e1j~w2E(t~Z7^|=I}--AKwBJ8JCHowOJ z7+szCskR)^af&zFX&OZ)6<>1~C#5K@Mh6W7cKiz5eTR9R`OX(JN8*}~{1L;CMo{e| z`e;+Mx&Mj<A+u2%vf9X}tb_^-tKvq#Qi33$ojMV7^Qg558LTdSxFU!E+Pdq{%5XgO z@|;L;o}ZNPyT=RW$r<S}RDJz<jzg93jDM*M;uGUJamjH_-UtNDQtsDParbb1>(}`- zb*0`l|KBIF;uhkAVjmDv%`pyMESbzK#>0%291D7`G4?q8`mdrd?pm6U&--2LGGeZk zdS(3is&q}*Rf0*f`i&dUY$*()ACf9%0tYUPOO$FOqSw)FN~!#@b{UGoavO?hKrkvJ zM@9MSRHHegJt?XD{^OF`>AgtX$S0~zNa~aILHHO9^UOmajw5WDb7Z%-JeDGQV?2I$ z_9-%Bx$E+@re3qr*}SVgo#_rbNCGP43+ED&cl4fVS4MY>mqvj4)Kxyz_`o5&j?R4~ zWYmy165zkRdna&rPV06o&doL4B9mxz-oiDSGCY<t505X()As93G{~Q3CaR*3*HiH7 zIInD7!J`a$^l=k`=D;|)ubR6~HdteV>%xam2Uh5W288{T@N>>qUAy*mt&xUAf&Y@z zRcyWc9E5H-ExBAWy!6CD7M^(HJWd)H(suWaY$B-IeAkL|JyRrL#~A6&FkFA7!m|yj z<Q-Lqn93}P`*Y*R7LiTeellAf6<nzJ%A~RaT{<7xxgANSezzHs2usKL0h8S2c<X6W zTKQFNN4!;MqGKV+ohv+7oRY>R&N5t|mzZGL0bZP<QCY?f>i^Dmn%aa*j<pE>uVZzG z|1CDp{8S}*^y%!0XDD>0k$ZLq<{#Dx8k~JSK68z4mfVRpk^iaRv*3(^jNQUtyQM(H zt@a9*Brt7Z?{3ccLDAmGUb8K4&B(E`?d%&Xcv*P<Zg7+F^hW9WUeG;#={o{r)vBVC z2J9Z`K9b0K<^Dx*m{~q|dE@Nyd6`oF%<!@C3*|xOwQU}DUaHgk#Y0%7{`tmv>0|V< zU%OElT1g^F-qB0-2(#_YtV+RJry>xTElZ*Zg>^Px@;kMH8Ob%-E{cyp8fHg!cS!s3 z_Kms21dfJE!oRK_;ys_`4SsQFQ@QJRd)0{2bpA|GQ|QdFL6h*4HSqsCH2W7CaqxLr zT%s6HLh)@NDc^u7(Q))arZ$rTN}pH}Ihm^mweZ_fBfI^<ImSNYk%*Tl(eJF}5swfv zzYn*zm;6T8iXE?qDHSzU?A&|B7f`yhiW1z)=TD?-A>7`w=9z$~vh#m?aT!<&-I+Pi zrGcsnJoNhSKt=i=7Waffa`f_t2Xvi}4q9x!H5)LYw32NKXW3m}q0hmh7BUhSPSSSt zO1;f76RYTS>woVPy&n>;B%MA(a~mpL)%AgpUer|mzwhmTUgp0)RQvx!a`5VZ2IF5B z^7&Arvl5;g48M}3Ysx2hPeA}>`ZtyM|GkhD|34fU{>N2JZ8rX4&fWMQS6RMC`v1;u z{m+qK`9LX)UjtpVc5glP3Y4AFosjlqiku2|<JI@TuFjz!aP__Qk#!uw9_>K0r%F$I z@k{r5WKVaq3%vUhRp1gsnCWudnJue|M6cMfF^{{Kd^7LfNYjv~@?n0BJ$<fcTV}w2 zXPj>BD?|V9!(aIjNiwe5+xpvHqm16(n}Cu4BVNc^(9PDP=B~3H(9A9al&{~4Amh4r zT77m{1GObJ*+A`z)=u;HNxUJ)vqAfF-?6}L`GNA&5$HMY@Gnxt*+Z#*gY@kRHE~r2 zYUMlnH8{~W74$aon>SoRnIPlXlf%6-+G>2^)4{##jXf2Q<$w{<%3W3$n-*P9Jmf%Y z;C6>epJVFQ5)VT6@HCj^R2zjEikznL(~z!n_6qg;KDB&)N@|DwPApI82=MlT@}Thr zR6A3rU-!bi2?cY3YIPsBYM^IZ)OLd_t@A_CVIJyIJ$(@Q5oWGQi6yg1f|;?U<8h<x z)*Tg~H7chC>6^G^2QOYLpq#jmn+oyvjIXTxVCQX38FrNDUNbqNRF73QKQDUg*&_DT zqiOv4uMA+_w0b8d{OCXRMfM!df0*ARq(D5p^WcMwZ5LXx#;WVE%G<VUg@4cT@37lL zHw^8c&=>l_Y{8FP1hw`{1kct_*huyoa<phA-zDqClZlfZZ|LbsIp}QT6V}$EJDkMP zX+0+~cv@LR0QbU4PfOad^wwEg3dHH`ujRs71MJMB^@23)!Dplq4L)mX{<=c}#D8P1 z_HOy}=3>;7&pTflenuMTw)1XUVD0&EU}pm!)4q7N>%cG_j5bvI`OHi+G@m&B6w-!^ zuIot-EBXpOZeRR5Cm5Gw<k;$B`$w=Yhqfl<cg2Kv&=1}T!PBX}Y%Z^#p}ak`tr6_G zcjxeaJ#$drFJdXcc-(~MWQMjkWQCWredU-`5whL-J3|bUktZ;PU8Q;BHsE0HDROdy zNV<2o{!2cu_n;T2TYsrQTYrD6nM>uKa$ByUcgT2tHZhKQk@U??LPL_rf)lT={@hS> z>sc9aInE~RXGYPNDZ%zNaQ@Ro?{HkcVP_XMtH!45sIkVjYxSnL6?9d~zEy(=Q#gWn zcPcHyW@8Iw<UIyzJj2kTv%8DzNZ%poRz{=r)|K3U&77P7p!@i`8E)~6)ce4|kk%ZB zdfP`Kd)SJZ9tj6?n;3IT`2ZwP<%d5{oNwSP1d>bhn|ud4Qeg+dBcTfoq!+3y?2n=F z1dqGzHqRs(Or7p={WiT*dDCC6$^?k?L?%eEtH~Tbn4dG7()68`(o;WZO_d`MgcpL@ z`eUr%E-HnMmc;drEl%J6EGD;sc-T9gmEHffW?{AgDcjjC<Au9l!r}5ob?izUe)i|s zoEm#<dd>|eBl>sGYg+OhFp^S}S_SwY27u#FCSjW=J}IGdTB3=)FXT8j?J@Bdm3yI! zr_$7WiL!$uiZ!bjez4QI52=-sH)GiQHYzkK3g=Ep1fgaP-Us{PJt6`=-{7RJDEu3P zU2S5`en%XgsMJyN<VLky^3prn^hdG+bp(>LEhMe#R`Qq7u>`v>9T!MfpkHLf7)WUd zYf=xIRKv-%r%BO|kp>JmVFFoZn;Z#0YT!IgrQVgYaV-eG5t3RGd-k2P^JUsM!T8w) z(3aK66MucaM9e&@+-CfUNWdAULFfA4DwWLVqmbC~qH1Cc*@`PwRw%LB@**{cBwBd( zC(phIDjKEM$BdgQ@f>sPnF{<Bc(ifDyGW*JW4I5k4>`zy5uL`)oMl3%wTh5fuqGUN zIzI6j#c+|g_ZJ&AS;6aQ6w`B|G7>zTK3g)B4{_sL5eE0F<6cg99_Siyp!%!40xF_o z<(iub@(7R=-7P<$%JsFc<;Q)Xb4-$o>Y#w5!rG6!e!tvNN)(sh!;{B2^HIs}>mMg` zd6lX2{fCHpNWgl}L>rY=pslh#{t?lR<U=nLt^kFQ>oVc5M4jS-8tfBexKL1^T41{4 z4HXyQ+aD7i?UqTD2R@4@Y`&=w_j*D&363G-zCkLUv^ft=*d^CpwsO7fV!drWpUZ3# zIx~LNdagX*=7L0u>j*!`!Maj|Jg@|pomsuIk??u;@}AHlwuVFkFbo+Cwr<cy|MUqc zIVTP)L)O)yFSgFtasSyEr3LX9#s54FQ+X_G1AagBHz%4CAH971OUGt9_(N5(%)=#j z`_q|+ElmAuMXox<#nRj})6=gRH9F>ZyaKJkM{&XQRq;6b5xQd4MZ)l)`{c%}h|H~P zJ^7%6^wW2@5)1}0AL9rA_Cz?ma-W&Hsm-kG-p<4EnY3c+%S`qcmq)cMHx)}4`+bSI z32ql+PrZp7OWp;z4#PlQNTX;6-O%1pFmJ}*_Rrt}v!Z&I<m$S`YIZDZB&s2jG1_fq zJ*V*Y%y^Ei%9B*kl%7CKO;9!0kj#m@j0;4wGnd;i)KQe_1h{FM@`TuZ)puHIm^e`M zcWw+$`6KH-Z|vpCuuk}At2)K8a>vr%!L`@*mkkzZV>MZAiGDZRMW$E6A<04Sq~!X; ze4tZ51i;_CR-BrxlEX#EgCI=1Yq!7SGx@9AO-$Oh*jmB7Z4whgYiZFC_@qBmtK*s& z6c>3qlTZW3?vgVHb!&ypdctd}MXdgIpKNE&=iGlP5}KsDhd~T_y5FeR7VJ^?Wu5K| z+^)|*B=xG<lJ;eoJw!jd;}56OtGT<CC|rg8*{0a8uk7)Gd`|CU!J~NYvz}j;Bv{u) zJCY5v>ge;16FPL~Rgh54_UlaAg+ZS%*2Y*Dg1Bq{`-D^Y$keX09obo5p=VynZX7^e zI|3t<KE^4T*rz1>E5)W&_~>NdRb{A<Yh!uFDY0<Am6M?@ebA+KC9fJlv3ga#-j3@+ zn`FOtqRAXGilwG&!)45e$6AXP99ma5j2&)!jqvDRpgQN!s=?w{5wctSg8P8N$~rJl zh>T$6F+#)rgNitK;<8^<G72N&Fq#bNDGmxwB0F4VAy8NMDG}n-L>VxWj}Nl9CW3T6 z5G2ZWGn)G^tmIJ%1<q20i8#~;C5ORgNbegy=(fOOLsm^9?F@QPz~^bgVPerqzZ9$; zol58yz(pDn#THG3^%DL)?{C}vLr7V4vz|>G@H<(LdW}UUei*?^m09curbFjymGUn? zNV5tR9e>gIg#IVu+A+K9Az-C#tJ<2rBYtv*ceVxGKKHlKxf|oA^Ws{4deszTd=*%9 zQ=5}Cw=LNlIlOn^9Xp=h?d<*im+~B&(}46n^i%XLgn4OSj+G?m$q(~#*X~KBN^PU@ z8oh4RKF4Mn;xHrn(FLid29j2&>*Ap$c<44f97{5T4*$+pRLer<Tjag{t6!S`jQvx; zYX!yacCB5vA9nQz5-_`>Gje(l-2<VDdind-O+p&|qxv=+bJ_*MnWSEg(;FxI=3ou1 ze3&3Ex9^5%CYKa()+qKqBdZHv=@)m0vIofrq3ha$Z77B)J>J)vkPQ`;9!_{>=(Y>3 zef@kx2fOMPtWJ=)L@PrTr}ijT|DmWwZ{g7PaY3PrT|;ydk?cSt0(Gp(amMQ9gb@+e zjuZqq-U!?HHo-@mcsuRb<Glo|-!r^^@L}j{uF%y0hyl>o);Kk~=Y0uZY6<+#My!q_ za>-F?fUT6@3eDRk_89ksw|ER9=dGRl)9f6eJtWZ<mtw<dPw18a9N?`&bz)_hgo2UG z7?Bhi!ac#xHM3xVF}!2*bH!D`F<ajAKa0pdR-%@GFHL`6AmG>fwdOS0*lTC2AMpFv zTIm2`NQTKN4T-G?KE%vYBgBxg(UGmNN;UQiX$xIoV9nvrM8C(#Egk@Jyh&C7tjz~Y z%*U&jK|^4#cpOA<U8;Qj>3QXOW4jp>!ZW=8Ew%p=r)O{6O(!?^q(Nh+mAbph@eEP7 zV3&Qh1ZscqaWdmlJ;DPENAM4bLq7IVqcMC~+m%IRK<it62*DxJ-9d+G@u!&w35d-2 z=jv=dQ~r+$G|CGdbNB{|48^QL?|2IH4DqUL`bE@CWj4x(JLf}uZHwG~SkO5somBdO z{NPiBIS1x;RPD%3zvQn#DSWp&so)Or`PT8reG6IX{hw1@AM-^!=`G;@KV292Fi{>q z$j9Zf;yMaBte)v(R(96+x2r9;8nsmG#dRN0y_$DXslW29?4{>q+7|uC_;!@kiSR8B z*a&G6OSPikKX@41MN9lcMrO=+S4Z=a`OBvEd)A{P)fzoB1_~~&#ouxKyDwutj3z)H zdOX%Cvj?|HQjPbj@gy8?{41`wHr3FFr(dPhI7+V&y03rC@hvIk{sYVMz*zVxiDa12 z8BZXV-8v)54Z_o4g+(OKF>Y;Eet@o$uTXm{dr`mgRbg5m{Xn2RliVwlX^tzAULKEP zCr1Kw!#d>zu&-_c;@D4yK)U|z#v5w8>&v`bS7%vUAwQalQEIH(o(HU<lkB6cWC%H> z@m?mullfKYKr8rr5%sCQJ(R0*X?Q#q!jAnG8Z5bs?XP}BSZ?EUEV(TrxDms*q#Ye1 z`%dsX)pXAYF-dI++NnWdWuZO@&?&x+rCr>5rJ(CCb$}D|jT7ui;Rj;?N=VAijP@Cv z%I)v0^(9QQKBV}`4`8otGinCp9pRElyF~22Ah!>mfbIIiV5}<stg69+-(~5>44D1W zypgu0S;05f@ad#7ITRtUgm?}fxVwYbm>Xhf!a~6fR`Phm&`$Zs;Jdg`u+~o)Uu$IT zv)SAXph>Ttak^$RM!qL3Sfp&(xM=QH7pjm?Z1P0>7%}?hY-aTCLcWQ^pm9h)M=5Wf z?-O8J!oHJ~;6AIWmdB<945{3TYM8Y|8)9Rm{fit6f(yA~ZWYX02@-Ov(d4|-nl&uo zk_%+;$y;}{CV2UYJnC62XeEzrCa!(U_hBd6kg^(=()#+ASOPO%q0icyvInR?<8mw_ z=z8M_fI6rXAP{ANwglDE_9H9h%1D0!&+$H<fi*0sPpVyC*)IX&-qiSgi&Vf-nP7Gt zvqmDtd=7!_&Q+}NXP|sSk$2DUC*4f1TMO3QF=^TpRPIIi*KSVbtr2t0Z(Jr-W;dkE z=Sl1iv;W==y#8@FAax&kY4k)1S1o@k{-gygCcpch%(Fbtp_Za@GX8t~ro+di!npV{ zx*I#I9mNV-_cb|XKRvB*+YG1U=?(YPQh32jp_ir@jp+j()eTOOit)LI*%UP^RvTHt z>GMj6k7;g5uR^?#==CTa@|jaw;C`f`tR?8Yj`wC)nlo_x66T#jRj!e7EZn`i)dg^M zPK1-Yq@@y($iCu8MSbOaD7A~Oo6?pEZT+-CS#jfHb}39ty^tNhkx}@7{*X>M`lTv+ z!ys$_Yn{iUYf?=uGFMUc(bWb|MStJlO#Q};py?d(epc@+%EMnXVAmv6fFgGo$^~5F zONo28VhC<9BNvF1T$@^MRPzVvd{QFzuF)U+zS%)=xW5kqbnk^swoCs%yuEoml<(X4 zT_mFHvKx#nDY9o9yHP?rvc(8x%h1@yo~$#r3Ry#Gu{2~0qr}Wu2bFy(#+E@dV;k#p z`P}#C`};lj-Rt@D>7UZ8=5@_=p67L)=W)D`_d&5Ozui?GPBq2K^%iIc`gHIxg_1_0 z5NcXG9^R%#t@!l&9UC^?9f?{qOqEnmj=idMI{FO$s-#0(mZ;Y~;p+m6C%mf0h)t52 zjtDHZ97Dh0f6cKg+k(WyU>r`1X<{2TGYvQ$=*O-!EqF=TMFzmK9?%Xcd5i=tzX!41 znM4jM)@U^R?z$rMEf-e0dx2xoSLd;i9!+s)ZT;18ngfWGu+FU6k%?5)py!<eYhHE= z@#uo$vOB(QbyNp++@j%wCTV<<GQlEL+_t!M072=~Zv0JdqXvKApuE=_7TjtTAd#^; zA{)0X-R=9es=e1>xrWEdoAaR;pVhwT`lh4(IKR`9{FL|T>;tr9Sy?9e&*&D5yr;#s zYfpynToclD$83-Z-Z6}HbufIAUR+6cIj{a?{DZW3XLrO#A_u%D^C;&Uyi>uIQe{e+ zc*F6b{{`L(4!a>d)UCTM-e@E_7e08m07TWq>o&^xl048+v*raiauBNNUvI1TK`oce z5r~CG{B6%5G_iK$=w6Q+OnkK6*zwOZrP-6hZ5{Rk38vouYwCFmFHtp#`GcB7st3=a z{lD=yzrO^|C|*k3sKYv|8r~;qMyv&_WdvEKn@Z_{@ux|w)GJ4Y1WOmrg7{jO`vxG! z`FhEz+^pS+<j8Y3-dHyQVR3UmTW5rnPzO|b!Q3?7)MPr%<K~~+WHoH7BDcQt#PHmB zg)vx}?HVb+wXwB96pFjK08p1}*Y8?$zGifamA#y1$F(E2S&ujD`|xE&JN)|YZGoZV z9nM2wkht-hTEzy^(#ZsF60%X>YmK0Yzq9Y+D(Uy-={JKOdkxm|Z}Y}Yv$@6?UerrS zY7Imjv}us!ed6McOZtt(l8wQkyq`y8_$8etjVFOVsR!qe725R0hLcI)*nqIv!Yq_= zv7ug(q;_a33qjtPvw=lWt5Ywmd7~QvL4XXLt=~X!raJL&1*pE^zP(&kBVXpZs!Vi> zGg|ei>d-kH5&1Z4;0edDaUk}CFu&itu6Emz;Ip^EFA2;n=Se50di)3N@7_STGdN}C zjAXkZDMKLVFZ0ywDNfOxc-*d~9Cx?Qs~DxW81&pp-C0M(nNDGUToZ{0)*0b{M;ohT z&FMJpECTBiF1Sacn1mN!ONNsaU7jGRrrZj~;Aj?!MDmLSQ&_SwEFRkpa3bDjp8h<x z9?XRzo$S`07HIsVf(YP%Yj+;v`u<T8JMwfoKgAd=LbSZG>~9P=^TWgw=L8t1Y-Fl3 zMn4sTgy(8;&wVxX2S*F{fYV(gTZ?v=PhqjW!C5==Hs|M1-(cV7jJWA^`x2E2(MR6l z_`rC3b!ztb>$==P;>F1vqrr1DAA9%VcMga*RetEVIvtjsKTw{(%nFZh6NL9m8^^nJ zVc9w4jeDnH`Npc$q=w5fz_t!fbU)KrHKVBw#n=C-qf6G#l_2IaFk4@cA0~wgDuD;{ zc&)-?kM2zFwe@x}%_W+YHUzr!pT~JP>7bSy)%ZW7%GFFQaUQ#emD;ZsDm*G=-$kZ8 zO!H(NNi1i5XU5NT?-(k2vB7^{0YE|_t}pA;MT@TKnLIRkVDeq0E-unQRjy>z0pN-f zUcUJqhENp(y=_2w+|-%O^tgG^t)*3ajBlQAcrl~c7*)Vf-CAu}ITWB@4?sazNwFR` zA47Y^&#fC+N{V+o#JX?L90+>Uk=Q@i&D?pW<?*^g8DZ1u;N9U$0a5BhX`w~$=00Z1 zCCHT9IXKtNfWs7@hSQU$KZV->II7)J+lTYiu29jnf<w&9EQLVaHC2_A*~;?rQ<FXr z|Gl;@bSYI`B8VPsVfGq$NeZWmx@63NP~|E1sqQQDSz~Rg+O=E8<9rla&Lgok0_Kwh z&~{;x%4AFC<T>^jIUnekk{_=ToH=7{R3XTgt;(Gk@1*oCoTvXyC57v<nb?KX6Gk>} zn=f}+J7DZW+OBZm8$LZS6L*D9&#F!RNy~Z>GsAufAt7j_I^ZoWL>CA)pCV(`u(`iV zw-}-_i_;f<S4e@X&?2RS08CdV59!ygU~Al5qx-R|Xlb`n#9U#$c>Sz8w+yuB$vKq+ zd!wt=f~8SO<MzCKdh{_-L)A{B<@6RD3}i=O>5t!4!v&QsvS1DV*rv?3?9UO*^4meK z)J8v`xmHa#vbq+v|BD~d5w~!$vuplni(LjPTPOMapd8I1cSf5gBxf$&LU2;v($X}o z{Ly08n(8WXvq$vt!*FK!_e&A<9_fvauW16wKDcQadg8<G(FN?#piHnr-Il9xzlTl% z*Dl&)P4ly21+6A4Nkuyu>Nf4tgzPXnS)=rX*d=1?JyfO%PBAS?^P+W!D7yDJHz=mm zWN<+E%VQ67H~*w4rm4pJ^XLBxk%qE%g@aESyc*8ddoW-*Dfe1lL0(cGe)g`sbc%6t zv3Z&UfaCMCrXTM&Voz^X5WM;j%5au6y3Ad4?aK(7<8jh7Cbv$Td>A{3=ce)nLW?Ls zsI5z(ns70}qXbH>iZMQ_-R?l+)r|Uh^2FnlSL;o+#2pu;e7TrO)p<>vDAtY&AY-4q zRC=*e-%=6sQl6SB!USvc3CAlKXHS33i<Y~*lw#we%^j)8jnDQVg}TO1tF}tuSw5i) zOE?Zfi3bETrjY%<6idR<5?QCmvbX-2UYA8z$NHvTL35iV_Ko#gFg*%?X0^sBb?QJK zTdl3l)y~&p3~3xj8JEPlYkuXJbccfXI-DcoO_jb&&oBjiKb?Ix!vBig)CVS8@sw1- zQ7-`8VfX4+)D{wzxxbs{!D~l{ZI&R5^$bsR#2NRXO_ezx!X27a`2BB|*%K&f;E!A? z*!iSx4z0&eIh~BFg60BG&@<1g7^nbr6~Cv!AdGav#Yks3^5V22R@z)4i(>|D+NK?m z9k+W$57W7t@xTb$keCIZdgK(KKY&Af8-ocJl#-lB;KrMa7Gk2q)1T^J3<a?E6PT&l zDK+<0>`-!Ili%7`G{OxkI_|@L{By?6`73D)wdbaE@LD6lspb4$eiNhZzJpIh1iklj zja94gNWL+ofdQm@xjiM1`VD<&rYhU#Fd)~$g(BH9f2`T)uHvR4)1sQ`MUj5uEf)9V z&W#}7rGUNKAFsBlwHSF^*&}%(e!XzVN$2Fx#ftpU@H^R*=^1}943$RA<O(2iRjj=p z^2i+iFt*)!oWj<hmG^5cxqK4Xc0DTCUr7kVo|{oI$?rHy&lN2jydNxHNsqLo_m~1Z ze^i89QCcrD`WV!$^UIc^Q*GMnh#Ul8@o&Cu2D%^kHzqoi0FHH$az=oAy-7K(?-=A= zEF=V-qpx7(31RJH?<=v}dhw?1*G+DgpfTf{c4$Oqo?b+_#A{xAeDMvw46f^P2>I_| zc>)n_O3I6FF8Wk-0nQ{C%X_W+41a&|sUoFDGB1HIm2cE2QMPZ)yXa5l2a9~e3q?;R z;(<B7-8T{z3*Q6SPQzZzsf3?GdUA%eC<M-^P(q6H64C^c{>1CT^9ik86KwEHmI$Dm zbcmeJ68^aImgtC^&b~5QQUfIJK8)K8=wufRcV?qL)!F0Qt-(}?d-Hx?EEE-VDf@8# zw!~Db$95z7qCskB!WfW@<42TBkf=Ga2;)VVh!jd7fl-gjCmDCnd-%%pR$T^t;Z^_L zb1m_@Rr=wNNk1|bSQNeIcA0`D-M6CbHM^M!SA+4HRLk&W<Kl2+v@C1?0+z14;_<ZD z<y?6DN0Gd)aG)k&I<eO!<|fATwxxuO(P*awh7+I7t8*&g^|#-+Tvy@N>z38$kBRN! zAreB1tS*wTr&I2xX^(h!vQs4Ww7uFTTD&hugpHhqraI!DXO2pA(DUu-`BN{9YQ?|e zMNtG_%?qGBR%%{Kn#P<$(G#D+)0(pEUK*>j$M@Q=g2L#L1>M16J$f5|8a!QMjHU6P zTp@iq6C?duiXZ|Trx7Gw5#ea3xZvs)KFdS1*ssbd%ZSH0NQ9FJo-M_lPF|P|`b3vV zUZ>k~n|`=$jH%}Cv@b6$TEwpok<Mh+uq#@OYH$++D~x0h&BfOa-fHLx_v4Z`E~vK= zF4x?3uF$R+yaQ(6O}?6uF}*UP>)EoNF5=y7OX<;@TATKtkrcr{id-gTMGU%V-p{M^ zNvb2ts1qE3N$YRQPK6Nwvhdp{Qz)#{w!sVAZtV!3;ri^w9XgY0g@zS&{FI@&J%TC3 zC+s>`H%;>^{g`x@(W|oXN)En3d;RZ(H|P0FYT2D5xa9NOtapFp#QcTZtzw6Mw1mrC z?Z0`=QG)kn$(tOoI^Qpz8+)8Gtb>WQu7c)YZ1gri-nwu*lHITPRIAa^p`gS@O?J*- zuQ{q1xH6+ZwrVo4aupStAE!cv+m7yW9v@MHs*4+QO4C+fNhI^m@>PzM|19%Jembk7 zUpP#ew*56EVtoYaNMgDCAtN?y4fdY?{;EYSrDonL%)hC`us*t}wQ4DQ$IIx&2k&$0 zr4K9aRz!l}ro!7432hfPP0a^9sk&wtOsJd>r;YWD4O}h<l9+;7cpD-w8kEVkWxPc} zG~KN27h!SFCd|n{Qbeix&bn=HavxeG0~Iudk>|-I?Y)sNx3TK^n`5kh0t}@de0{xT zz~#i=;jHPCD>5a(ZHWOz#qt&x@607_rCJ24$1Ca$9Celm9LXv^gUXrWRm~5MdaNH# zy6A{8q%O4dJohH(JS&Y?{)Bf|`63W6*GT>|<L_6hq1Pwa@zoN}d26@ulpU!o_ZMDd z)&oh`Kb}p{ZDTJo?!K+Do#$0+=ezY~+@!26kR*^KI{kaEtj^QS4RhjB=MX%zm#pEQ z>^+~S2@yN`yJl&wKAcSPa?iJaSy$gT^r*JeyzR9gis+_ZC4&#IVjgjA9w6%O4Syk$ zRjjCXk5v*rDGs~zrZsG5`t}GSY;(G0==bfHVjl6Osb>t62$^K;ca5czphgAgELU^+ zE6vk3Wjt1z-Ikt@o6N-S!%nF((?SbIx4M7cfIa?DUE2i?vD2Q}Bv<^nt<$FHl#Ibv z7kE3SXnw_;>5<+jt#@Qz;EbN5Ze@#SmL2T=@;8TI)HYt^P{9RxbLZ@44$@pR@$kn$ zlUHW1k`_w!%=4a+FWnkAahZasHQHqEg}GY#|Mv-SAW~qMDShhki`7z8o;~ZEcrg>t zhZfhYEz@7JypXIExfO6*^>$W|T3bf=T<qrDlg;}^%qF2Gj!aJQG|t3!I4+E7TqH+m zkJo2;;~_Eh*m2Emk9o*m4JElmlINOQdh*?hemm#f=F!;i0ZzS=r}*Bc2?P)=)tELh zsjFJHuC5xW9WTioNPO?e!P;vW&g8T#m8LxgXd;Pj%XmXoyJVB+lvED6rzu)<n{sKA zEc_<Hdu=k_JU50WhVnr=G&M~!xB%3qKq4QWTV8RdTzthGuf12q-16rZHTh(CS4BUk zyqCW88SHMwL@h(p(kM{Cm7iBBF<n+JVZL?(bJmx16W+GRDLuKAGA^#|db***DP-V0 zG}yFsD~E$HEo00%@Vv~s*2~C?wBxO<tr6wm0x->|Aj(#Y0=D^SXbQ+__M(}RDkS1Q zc%@>#I?<c7=$=b{Fr!_YAy>)Gru4jPT5C!B_c8<|nLK-*KFz0mNf;*Vn>&M3a@x}L zWY{y5&hahoC3DHZ+HX^V?jZPJ!zVWQ6r6x}M)g_(y4qnU+Me!q<&$*j!ZW`SFtn$< z8*j@VMl#stBK#t~K@3WGUO{H44TZs?f>ViXOULv+XnK>l_4*!(b=ecO-ZpP`bS+4W z!4+Mb5OX=}e#zCE<S_K>UjEC&l^hYtGtjuQdoC8M3mtf5o*!LpNn&E{$CEwVc@^Ti zhD^8cGEKu_Fq4wR<XV-pT9#>cYp;9RyP*GHq(<a#2&m#I!Wlm^2_zOI`><2@4>YCK zBeA$R8I!U(if->0(S1>yGg=~$C(;jLn&zcL1&hPC@6TX8F<qfmruDTwZecYE%`-Oe z!Q9%fT|Qz=WZpHEec7&yny2ASrz3m@zJ`Xm3A<~~G!N@c3F8w%^XiyPhrx4LP|}pa zE{H*6&s3QE#!xX)ZOFcwj1caavOuJKlD0Ob9+A}6<ZS8w)O)dmqoaXxhwAqI8!dMU zSK#m3?X;O*JrmRbw5G&Wm^J43bWGhf*TDxHY>tQ#L}1o%{~Ksi+(LCfXf&5(RarMn zjo-SUX<m1YeI%RTKmSZE#=RFatX^S|57q2w9AQq=CMJ!WPoifgc%`1q<cd5>EFblv zy{%n=u4;nZT6$+t>S*kGVAty`e1QsX{d{os+n$A?wg-b>waS|vV&`rHCzFrP&+{J1 zDW?-{Ijp*h8+wNImh=bnN%V`3oClrD7IfQx!#lG;J=*`kt61xSgNGd}+`A1K0m_O& zycs?Zi%X2AN{}2Y9+K~dr#!?4o-x7_&NOz2WM?_F`ZwkX)5bl-oEoxdKj*q*gL5rl zJ)3nkW5<q_h*lzm-G&zLSGf!RJM6r5$9DCu7x@8Q4E|pPrj3X;FGlo|hhr}jd3x#> zivg6uXYAUeGe#>mQ?$@D2cD$q3;Lru)`JTb|AztW{2Py@z#Jgc?enI<(sb_zuLg%! z<$sup<9~_c{{IPz<?!d(Wy%N*nS)T*{!O!(U^xCl6D9q0f&X;b?%soTZw7vW>~)Xa z#RmG{{bm#QAsBbR%CfQL#M=HLasYzkHdj)--Nx9R=S8Y>B$uuZlZ3Op^QN56)(hk* zye5txye56chRro;s-`n)WbTH2Nb;RuDy_*|?O6?MZ{HYW(kLbWo9?f;vZV(E{Qf6( zf;keQ-{DE5HG5xmJpgvCwTDO+)*s`bt+~16bP%!!{20C$r?>082V9e%pq-eHq;GB1 zxtY~(_+i{3IYtF-vY&$Ot(EFI44(Al$T{SU&Z{3RT1DI$ENR(auhic0;aQv2fJk{_ zId)-4*oxNa?3V2xm8TmP!M(eAo1-5VHCpIBu@`p_sWdWM@_n!1nx$bdp@gx;*1wqu zBJ8y7rQVfqY#Us%p9g*Ff94dvGd890$*F8vgL%QZBBA5W&V7RY6rLSXU7(oGF-hx( zRDGNo3mX#Lx0)o6`hq^JX146krqgz|sGJvu%3;O6vn9bwORWe<pJ^!Vg%fw2Eds)} zrAW&mCw0sub>ujt<z9UMsPNxiUf17td6}oxm!(E#Cm~t)WCzbe08|d-bTG}|Qs3u> zJI8EW+AO*D+GX{1I4enG{NdN_Gm60-+upvRW81o`!Nlz_fYaFqgXo0qoT=S8N&g$y zLZ0*39=iUXLmHDSd#r$j@A_f>V4WZPC-z=$LyvTBK!jZctbf@=F{FR{aE5Fj!adsn zCr0140N^SY+F>SPT}F@Iu^xz}<=wLw`7sxKWvTD5vE}+9h&CH6%}(GeIAk7{0-I(; z?K%it2fOon{0t^Y2rZjvFV03H9O65(HIy_?(Egvep?P9~lgnLH*ul<Jgha(^_-13~ z<{|xbdZKpZM2^=Zls)z6FX!cZUDm*u=CPD`Q?kdphvs%*!dhV))ddbMTkVQieH&Ur z!{@=nVO|<)np0cSz*104|Bz&>z>Vl~C*~GckJBfS%PtqXUY72b>J2D*yCsK}ag0{^ z_E!4kHh`3u04#8I`xW3{Oy~HyCj+4*m<+zQAD4n^SCL4sA&VN=O^%nE8(%oWAMVi# z4tDjBEhnmz1!-aQZs`s2J<G0kqiBmmoNx%p8fLq%T^K4;LFOnX_a)>XTFwRmL~ks! z3Hjr_V-4xeIvy$3V((&kC~CBcSStoOwEQ~dYjZel9sj}6s9V6A_(k1pRlq(*QX(Ow z-p#t%F@-*6zq2y6<p&x}{O_&*eGUNWR<^&STOIGzYqI6$kUq_&lpbE!k~=U2OO;{E zp8ORBH(8$Vm?<-Pln~V!?uwaXB}yx>+rQTjDTGND;9s?b?*Zc$v3Cdw%GgHg-%y3x z_iE}ak#$>-Wxst`ak0h9FXVaM@|_AhP^u;e@+TvIy2Y1$;O_j{W>O6L!j$8*QI&aK z{ab=hbuQrrUa&GYzHwvJbEpzIE1y|&-wU4!%$K@BtB`?=K>$}_e7^4aT%&8FusTaz zX2DIQ4YipTwQEooc8I0mOmu+EPl`P-@vI!pSWDh*eW37F2DPW#$t-8(d7A_su2)9_ zmumpRRl=A4P*_eE_B<35e-hC`%UiWSRH8Zsz)zh>Bp(_j8aY#4B<2o*y`e*<?70A8 zk1%3VY5`!|+Kbyk#Oh#4$eQ7&`Rz;F@ft;}mt=co=*Xv0A3}cLDENd0CP`!4(s|$D zuI)#ap%dG3R>7Uh2jr|VY0vN3DLrS5?-2SMCXhY5DlMrygKxcQfEv?brR(0Yf~`Ax zR-Mb=K6ocv^bKkS1OM4`?+sM<fvx#gX89p`^?W?pmmcNg;v-h!WAEQ%Ldkr;-Ra<6 z9tUujftfoHHl$$N?iuUuroag6=&4Z}$`=WvCtvWzD<X2A`<EK%EunKim=cy6dfKO5 z9-aN~ch@vE^56vUo<RRktqnk(ijc2g6R&0LRp4xRy(RBEQLS=zX#MEsT1W3Y-k|E8 zE|BTU2x_i*My$h6uie)BkWnrF2c{xc$gOx9)jBA~%fP;U6JXtE9R)bSnbVSlm1goC zB`jslz)k$v@?%4heh0s;mmFaWhB}xeOV-&#;y2SlYZTAU^Z=fEdadJHu@^?30xbI} z!$0*_8k@thCtYNk7hb?NI3Z>i+I<_H33I+nU(Ix=2@aP@pR7vL${y;$j_AdD+${~o z1pyLNECKVLZ817)#!7j4+EvA{b8EY*D@>W*ixv-|#0_bN|Io$G@HE!01mU$6TxXiz zrvVKE4g)AljR051k2aSA<3psGXHNv<z|2vt$CZo*#g)Vk0hTr<O>x8;CqF%ESO3bN zoF$23E<OSqLdME}?OopWiZ0+pHzXOBc7ug2BuTJ9Qhy7^^D9l$NF#*M-u()5YKo9T z8Z;~H`x>^HDv#^&QJ@$<)g?)AH2gHu{+z05Ce()6AZN}1d{fNnXmx+|WVW3bKiiDk zyqHs@A2kK#=dbe`Q9+5@9J&AIe1qnvXWpcxOc57wbNhg9RdNPQSp~E}U|ze|qKPis z2VM+85|)%|%6dlKBtCAtBlGY9l5_<^NhXC1R_E8WJ?$kC&g`uV&KFZ^jD%k3A;_`g z7JA{9*n46ao{`2o<);l3%Rl3gX%fM&IfS>&E0P~SoHH4;&~iIT8-El+r#^0IT%DKQ zsghTuJx|Uc)yD+^sP5{gVSy=0P>%}s)LWv0lTQ#C25Q`j+V>*eP<k}M?hN}DrQ`WY zZD9Pz;2l`Ga$Q<G(#*FZQ{)xDQ-d^QMOZlPKB{`@2J)jt3$iwzZI2e*h8?#$m+m<l ztbW{KWT`qP*}Ktu(_ro{#D77*(qie#zmL64nO&>D>;{82u70%sP#e?jqutg?r_E=! zo|AQvXKh>=31zx!axFHvO_UmUuHDz<&M}l{;wJ=HLZ57Qed`Qk!>d1J?3Vm>d965) zn>K^E3e{NqJwC+xwZ2;cb>O)}CQI+v2`ig`>xF7~&d2~17Cm5$-2(T7ZJ&jtXy51U zEuiJjpWARB;&xbAdVyK|^@3hnZwX|gW);}(Qx6wGt1s)oHITj;CJH-OF`?)x(a+mc zV}KN2DoR1D{jmDtki%O|9}jzcZ>9R%#l&yf=F^4>I}X)dJ!nb5>Eu!D(Tm@eal_U5 z6N5CXI4=<tmPdR<AWE+VB%rDG2un%sktnIljw8!*m0KscIu2I4n4mJaA(xn3{X$!L zdJ>juW-nTIKlu<p>e`|{+jkX@Fud5HIEj3!7VC}nHc`m1Qd}18dg0)+`}hL;%Quzo zT7MV~qo<PT$?iHZ3Q@W!O?2)G%jLc)%;)VVIy`a7qE29YP65{sR^2>ociqj>IZ+a~ z7aspqN}&?t8_Zg^?r4WKTZ(kd2TG*`f3+eY_JOi@QirxDLqP&FeOhX0@wJ{q>cO^* z#suvgXk^8ojRJL|dx^d*KuqI6h~&<%NXnP-^kTC;GAFN|Qd*2{b=8e56i#^#NT)q_ z2&8jIFL|jBjffg{`q)C8>w@#>gxi%|&k>|g?kgRqX1<;Gzv(msZcWBZONr;zD6KBu zXRcL)0$o!J8f2sgtB`(Q$Ni@sc_)8$a><T>(9KuU)hd_4hT_L7I-IemH9QNO*le|H zb7nn45BZrz{;5Ph!_ydoDZET@P735^*=fG7GL*kZc2`__s&>nM_DNpS9Nug@i6`0# zO>3zqSa@0uaxTw;ulL&<T?R9(9zt@;v%XmIbcYG*&GD^UEejsl0l^MI?#b=5rh_`4 zlFA+0ajDGPz1UIGrUis)pK6<J|6I>y<ql!?-@Ud%1Sz8V?`cQ?m-_4zwq&Om>(ma9 zy%~GU#9R9nuN&V3J_9aK1Ws-NtjC+khwQjvY|5EB^P;HHu-2uLK{cI2Gt#5F%hKQu zglIei>;mE?!q)1*Nmq_VwZhGmy$RQoZy})yeO50AFPbqiJvbJ{fC)BZW^yrsYo%A( zq|_U?y82KUqxHYo+L`#Y@o1W{NaccUMhtwlD9oZAVBudRU23G_)`;#g7W!yvlGzF8 zC$2YkP1!&BiZnJDN{ef@e^FDH{6)BFGvuVhb#&UzzBbh)MFfy3xsZQ>q|!oJ$8gs! zZVFwBKf=QkUZsr|p>S$%6Os=1X-T5UDE;eeP?yx0l55FO-hq_WHsDQa%svoT?jnvx zri!A>GRZ-YcHbl5Rw9H#bHreeG1yh+?d~t;P0j{fx>39cWtOd0!fq`?o^b5(F_#z) zOR!LN#!X@3bjFNbdvrblOuvt8AXKgmuxk>b`DtLFG#Gn9W&zbFm|z;NA1zU}0F}|a z>wTsxD0RXUZthf)N=cJ*DJVXRx)exzAxcFN7rKWiDZyqbak^w@euPrz8NVX)IaFQ= z^@cF~Fo}m%0(IZQ@X63yog|V!1_SIorklx6{rSe|lQq-vP9uLCGAWSda_}3}KF;*Q zJ(;p`mJW9$9LaU`?E?}2gi|c$dtAxn3szh4WvQ{#k4vUQ!`<EwCt}pTj~ctrbb7*m z^9Zgzbi#VZ>?&89PE^I%VLQA-7UJII(3JGO9%xHCJ^1i^2}I2nqh1O39epLUG5GEf zvrbZM&yHAM&quqIv<OH_&FA%s%dBG*2=|xbb$-Hvb!*ofBr670VO<6dnYlwxkiq7! zDOH1=R>57;8;P-gTbeEXo)c91jDL!68@B;jSC4)rHEuiY>j5N(^ccHx4@zPQbDv7l z&Vkx&ZowbT=*(;QXEaM_xF7KS_rc#U7~shNX9*U?i>v4wW{*9v{_FJ%rpG#NG2Jq` z>6+nEF>Z8Ow(oV;dzO@ASKDM1jf-261U1)JE>#S*49X*j{Dso2cLW_Clo^NA(Xeq1 zY;Aaq6_>Vl8Se^TCSOetTf2vMs=y>zW8tzpz=MauSAe~VVi+GK?Gh_)RMy4aXM}Rh zOtFo_Fp|70CX{~cONN6Q>wMyV;@6S>;vg^PQ<?$ScIR%uVuM~*Qx@AT5&f4W@=~~1 zMrm>fsZmmf?}CW8i4vB><v}ahT9@tu0#EFtPZYiBm=qL@UGget?X-`*lA6N7+0>;Z zRbFT<m8WqrNg$mKc^ScSgDKQC8I$_BqU!{l%i6I^aK>E+($)Vq04PqF)}0c{fxy#F zLZ(brrHSo{9EPrvpHmZA7#hHMUOLy?cuBx{N6qmcc@@mvAFp4iA-o`k9m$h(3@{NG zp6ys=aoI0!q{sFB7<>_LmX<W>2^z2lg`kYVhNsd}$~u*r<85lQ^+u~M=_!e&NZgCL zSS2dqc13x?2P=MRwl3F2r4g9MzFSv9{kT9IUBOjA!b5u_9W^&)rb>-_r0?awCa2e@ zNS9ED_u2n2Mbl0c;q9w-P~p?WCqRkFg3DwU6nnBz@gSQgUWV=g9DCFvx;?Al<7G&9 zYZmW@46w3;%-Ex_HUr3j!+w%PL^l6iTo8$k0V$!^tg*US#u3(@nNNBEtB~*=A-X|I zK*1f_O6-@KQEk)0!Q3C@wE1UcCl({6N>Uo436H{_3l{bJ5eD_P6HC{}gkrv{<Ok#8 z5P38|5{d1ISE?RB3F$r$vaN2*Ygtf7FMdJLa3LLm+mejX!wjXDm{7r${Vx7<AC*8N z#NW0)33*#P$pcPgLaH4S04OZGM5{ig8NpjW-Bak6s@$AA0>Z7%sf78DFR}RDEe^gE zLFvr|FQkE!=EMJuZ`nKkB_^d~2PEhi>LqKVU+a1B=BF7aPL6uW-%cC_PMP=L|J30| ziLq`a660~aseIp2FHgCgh$@VHCtLnUiwCtLH!ay-rJV#JiEHteFJDwfyEM@Y-z=#y z+&y<w-u{~sl>I?5kb5`rlu9o0^LnL1_cE%gUlm+@d=AK*#0x(@2LBSYX2>Eak2iN4 zO5H)0nxPg9SS22B8TNyY7Rjc3ysBRpnq;{K2EA<sB~IX+V<c4%BoelSPJn%oN%pTQ zU9d{LJ&R%kRuZ~Ut+S&v9^F)P0z1)+*(8K0*C(u^*$ROSi5W!N7sAlZq&jlvw<ncA z2DoxjCfsWnpmx->NyoyH?~@(kab@zP1h#nLG1?B?h1bQ_@y}Z&I!~0OFZ}7OdQtzy z>4XxXN2i7I@N3b6q(;q1)l_`^nXt7?VO(LIeJ1m+L?B(kSZS))xVU~|6T9i)(qSdx z#)X`dyWXyo7K)VKB(J#?VA`)wGDx1shlKP@1-O4zFS_OkP+RL?F1mI>Aq_p=KK{Sn zkg`2^tzliK;0`KrY1QyEv^x-z%}tuV0{9713>P=q73w$ZL}QI=OF^t;;}II*<@Wef z0nb(kFKE6m*AY*0fZJLMrZ_DUn0GtzBVqVe$b9>?le7GK^A{ZvSu=s3zU2YVsOY>O zWW&=hmZlSr%zJJSE`_IRQ?FVkxA0`Iqb#se7c!!cPGz7zbX(d{g#0tZU7m^N&B+(^ zXfJ*&jhRX1s>hXvBIc+Wn^aY)YWrg059W^I^sOJceD08+H;A-Vix8W@_8a4ESCE#H zLhVs`a3@Hc<;J}E8BEWp-^zHynLtz+@Q^t2ov7vP;PyUPdndPi?dc6lvw$RbO+mt? z?L>(k`b|9=@x?3+6Cv~8PwMt(M<X^f{|bKV8W)3)r*Z9Po#=m)n)va0;g8dDFIlkL z{QYlGh7AlMK#@&hYJm$P-jwMMOwx(>CEqK)7l6!;bxj|{WJj-HdC#c}AQWuvUctPE z4E-0lQD(>5?`S>o_^4t<8L{s~i#<+kRdA}$FIM9UW?|h-)eazWsEaG{zHl$%l1R_( zZMw%eGAetK#_#k2gfY}(q5}H0;y$8f?1K<+;qaxvSdSsjD<jKKE@C{NOeX%Bo_*<| za2y=m!48L$3R)Yo#4`xg?v`x=vz9m)AZ2xmH$A2wUd$mkE!0f9gjhmBOR42W;b!vt zzcRkXn_|?^5SJ&MSqY}A9p<~J4mT}HX?nb`VSKEI@mT;B40cgaDCI&OevM?n0X0YH zBAf9f?fSgLN(jwL4lv23!D8swcS#UXgB~{uXb9kpE<g2aRxPKyofEa&qZcm+Pd*Rb zy4IA61cYanyNjj~o(AY;<JL4a9uyLQO}tE1=QN~&@1AgWsK<FfacTtkOE~T#AbR4U zqRKHbQ{89afGg02ziKy)c}{IQg`H)HEctZEg*RRFDBxa$ka;;O-HXz~aVb;HfLZgv zNLDyU6u7!Xq)KhgfV~DJhtq;2IHhB9q0_viIQVY!l@;5Z<QkO6_g)(^YuOlbIv+Ca z>HRWBCsoBoe-2oqmc|Xv*s3v=z2)idwAUmD{~k4d^yzzb4d*v|7-@Y?sxl+46V(tZ z%>yW^YYd>y@{oK7<uPWviwai2L-#!VJRF{@25M9PBq_9b-ZDTuXK&las>~VvTP(rU zGc0_}FJr3b1fqR)*I8_CwXPqcCGDPy8tuBbR4B4Ab?M*T#66%m@_+9pFaU8!5>25S zyTw0a@D0Cl#rclJ>7$Y72$o_@wvS^Ng3y*V+NKzt##&w{qY~1>O%gB#+Bbo*ZuglH zXW@O<VJaJItU6^6<(54sKQ4JRm63(TTUn9sMNPE|hbD{}JtNz3yCe{G65$F3H*+x& z7)^Evg*?hJ_A~bjif;gOJ+VKLMgSi>)gpHxBk97?wnPT)2uaU{UJVz02!le)?5p$K z=Ua#iFFh)L_7agUr?}p8IwY<k-=9hcLrc$+lWKAas~fNSDu4?k31tUdN0zoU84PrY zYJRa!IMHgnyC?)dMZXt!Gu|}#-W4q3ylc*uZ~@1hC<<Xki5X6nR~X@Y^3ZU!>P$<A zY$*waAe&5L3Z@o1*s0BKl>#&%KEigbM`6Fa|LxlA>Lr)9+P1iqEpx~~&Pm8Dc(Y0U zNwD-L=dJD0d+NYSg@j~Vv7DZ%idyc_ZD>DED#%rja4)*XF8*!g>CDYVOAsvm{{CVq zN~B1|t{Ey0AZPU*m%?AM4IDe#eR?QWjSP6-s*F@|_==3T9D;N48xX}UGX=or&PKDb z+pae~-Xal@W=-K~5pQVACXzrdiIaou80A3G%}X+fw0olZXz5J{;&>sCDE)T0rs1%S z88|#Y0jE7lK_`iAOVm{fE0JXLl+gkdf3)xGxbY4JSnyey;m<8O@unAQMVK+q0_SuD zcjwI6)_(d|(WBW%QIBmPD#Hs~i=3zq4dWMT-|f{-JK3eQw~e~3NIpb?!@Veu;r(B= zaFuh%K^el_-zyy8?PI#ePJ^nHx?b_`X_>q2Zk{~Bqa@HHe|-I}DKx~+u17|>z1JUY zat}1IAUDL5Lx!vio*qDKG}@nuSkrB;TEP{#NUT6IQ|VN3=)sF2i3dA@*Z(^LV!!{_ z{Q|(O`Ja<c*H^m5<#_Ej<~HV*pGHaE^cHZcGZSgQe=NqOC0edk+IXgB$gz?4IbRW< zMYC!6eZ>O4QGlCl;Q`!@tbl}?iy?886~o$2IM#04{vg(6Mtm&QI-ulzaJ<LO3#N|4 zKhj}LM)u;ayybMpFEM>mc(nPRq3T(oF~N4nPjaU77p;MXK|5;5mNo+k<rGQ$1_lIa zuBsJ=>gkfoh9Z)33$O1IO9^15qf+q>&a712K!M`cK#byI)W^cct>dGFqfEhtIZBJ! z4ykf|;0R!SAkN&=;2-#^AU%|dtN@q81Kw#`MZhF9wyX^bBKTG`mOYHs#ly*}JZ#>a zx2C`PJZ3R8%kQvikUbr5S=)62@;!w(i^g!8j0%Dhe%(QkJ2b;hPR2d$LYw+WPjrCx z2Z(!|;w-RD__<1?3MBeEnX`P3aVnmla9-*}ddx}q#5`LGZ%YRkLV`@vIf$=8a)E*+ zWt7#cIMdqV)@CZxJdP>-$kYQW9Q~u%;g>6@Z^!3S$W>>?a`lU(Qf8Cu+bWTr;Y$fb zMX7?IKl}snQ}lB{p$U@wX$bzLK*UZ?V{thItN&T7w#v@F%wmvN${KydBstrBwBJeF zKVxFj&U&p5E&h_LIKa>&%5zW^D3%*>MlL<I`EX0v&!u`N;Xb0`1vBoCUA<R&_1tBI z-8U;IY<2#Y{gspLU&CNjj9T&)+SO3f&1uz2%FG?T7|TJnkVp}=&4o<hF!rZ3FMC(f ziS$+^BZlTT=+&j=vQjmad2t1(Rd?JHwGG)B_j4l8vNcOu`$#ghpL4qXOCDQT2h>WK zc*7VEY!7O<JogsBncYfMQVo59CIfwxQCd)A1fjGLvjH1r#;9xDeZ&)DRv2XGd+3t9 zAoYjVtlGpvTfxMC<WAP?yAw|TJHrda^gft6%Dr=|%L<a+m1N{(O{B1HCyplW5z@&6 z2E1}7J&T8Zpvf+B%FGRgviYLMuJtW<$`yc6G+I(~B$0|6#9x%)`(|p8cO?XD&%57o z{iz8;)-dCJ9!AHA^~)S8x>>TiwQfIP@Hpn~(AE7ySPDbUbU6CFDiG2h|JJKpD+D@f zpfS@pCziAP$%u4^ak#kL5ZJAdcnbc2_Y9nE^=3YmMphP9o^R}U*s~{M{qAnTREE(* z54@Ea%h^Wq9QZ^`Dl^Viqvg*DO>!E~+Kmz@!VQX6JC*&$8(}7SiNvVemN_+I<y47M ze7@3%gCN9PlDn+dN_=;vfH<98+xbQs=Mg&UN=c4L@3q`r#nL19)23kEey=sfPA^!k z)cc#+Da~iB%|<Mz3HEC%-L0cX9x)kCD?J|}Y0&2ANV$QiO@CRwb&vT1{J3Ac#x^Zx zM?!J}S{OvMrhj!n&&S`c8bKX%)^2e53f+yp+Nof%r{cR+u)1)}Ou*k%7!qjZBOK@B zVrqrxiXdU~;^?|~k$d+YF}hI245QLrG$?Pp3;7UaYXaRXGfH6MPMF9q=kl^R3C=91 z#^l4LX*w;x;z|rZ4dhHMkUs59%f)pwW9RHNd#6UT+~*kcy5;@UEvMyr;bz_RY%#dw zI4U+5UVzp-n_Vznnfipzz(RL1P^_$xxp8G7?nv>bnaC>ZMiQop->J5fmH|nbs6H-v ztX+x@+MC7D&Dk3lO%8y1fKsHvhn?$38L||Ixkm;cdgkPItiJgj_xX$lbUxiPUoi7Z zxQ12gH@KnKY=%wzwLztgZ5<lZv$AX15!4=?!Z?xdDP+>K;cs$FpyrMN+@03I>PiHY zQ??!{+X=^WqjH2WHTPw_@;`}OKnnH0)sY;*QkAKwe7u8F;zZ&e>lAwK#&`{njP@um za#l?e0Gh7F-ZQZYes^B-3tB>|sPz*fu}0IdH^^1iGVY>*eyUM{pNMyr|6_L6Zvoe@ zJ~uBu&l0u318DKIFnC?9;6X@<jo0R6SbO+J?069@KpB!VUR>b05?Wc1=f+Bq7+x?{ z5GAm@e1D;r=*24c<#wYx!5KuCo1}^6O~6Vi$p(6{{_TF=%Ev{Mzi{~Pe#2#v1TrRY z-&3%-R|T6Cf0sxy3xzDSv-r}9oPc%2yD6!&@6k21e#1`sB^4w?1C`f0Dx_iJOluTq z{Z4{B;S_vPmAi~|#?qz4dpDzoWifV~fR@{TrD2CxV%Q#Ftddu$MFHicuJ?Vt_UAm+ z`;{~w6ATBdqx7OGN90Ziq+BV~&L&AnrQOjqRK~p2o8X*wOxY*wCh9u6?^6=|+RtfK zCs_u_=xGblyLvAue`H(CpwH{-&RA9U6f0>Mj)sax2qx*_gJ?$R;#uES*n1I4{Q7Vk zXlArnw9x^~IP6iDnrj<s4~}sRxSZP#)$?T4Hhg?P6I@yo=&-wWz839iGbkU%J+y^i z4?am>6ml7=XlZ)@5{UDrsN^B&=hRQ<Ek0||*TCe>zZe?vM)!61UN0F1Ua`{|fd31A znVk-gv6t3f@+1vpr=WcE<6lx7RQNyszzMx<sWCXmnI7*7NpfnJl7g)X%ztpN!V?9t zNnS&~XSv?A4CBPMg>FE)A?N>-!ZZ1;@)tjUCcB}ScXTLg-q@(uC&ky){oNKcN6)Ga zQ+0AuyL50VN@Y^7e8Oue+51op#a!CKqV#MtagucrF`QhocuCnUSA9;JJb<{76I^ED zZTUbC)}&WAR*Um`z!on;O3yPvFg<ywdt$~Wr-FF#Ey}Z_*?V)|6V~#LiV(IOxIY$# zHH;rV<0dn_@V?<7*<sAvv++`a_h#UI4JW<wo^yeA`KV>Ug(Vd_w#V6mZF4Ra9Z)3A zjtBDFUun5kYP)IcIL8>xn;O7k@K{Ll(aMr;=s<XmQxX1tgV-LSs!O#cMqp%YyLZF; zLEZ9py9<Blzc@}k^4ecSY{5oe%g6n^@^%YW`Qu{NKP#x>Z@LYAAAdc}K^<bZ#}_@3 zEg0iHH~_sA`I-%jo>y;o{#E@gtr~$6n;Cr&)`^~{PxdguMqYYkO1U)A8vb3>=BNI$ zs53Qq5wp#(1^7*8KiU^$6(kv-bDEz6^0WorBm$JP9VCdH`CaX!Gf-0>oC>qvvv_Ep z{wNN2es4rkPc%bIO!Qw$(!}rW-wq^HC^|Z)l0OcA3A)S2AGaM4IQ|rDm$aAqs0KU9 zkw^Sb(CDuq)4w<BpRe#&u@qnw|Mw#f|6r~4H<ak#gu_1{S3mHVoBpqr@xMN+{Tcdy zL176v7i&V=H>zu<`p(ZjYK7}KH?~_qWAma0U){(2I+D%*bs6>|yQ_|WdHvu22?AQc zQK8+34k}ctCoFWRZTXOC7zlh`$3{?<7R<r1D4+Hc?ppuYA1;Qk@!miwH$Tsq%L)&O zS({MbZ;$NGAhqiOG+fVk{@xlW>_YmLQ5)?t6W7LCxsA~YUU=e_8P~kd9lzWW#PZgJ zkjB&)Q%6*p96xyG`R7A`z(j99i{@BCFN>d=!>WAj-<Jxd>-hihoM{ZlV46ifGgFhD z5ETFab63Z?|F%016_*VMP?yR>@le?z?on}wdz|0e9%{kv<210CLxkdC)YE7ndCReM zC@ESx<YoR5A}tl{l4K4HR`QNh!|7d43Nrx0;ABJE11v(#`q7cQ`v5IfwGSvATK2~Q z1<&0<lgTVLKni6w1mG(3Lk0V5rToo38L9UUQKF$ksNkTdb{JZ#WH^^ye8><LWd!5) z4;zr0J5_ej0g>09ua^Bul1?ztbB?IqIn5kGV~a=$A33gcmw0IYI+O1OM=3VVvCVuU zX5yqb16y_`rjxcv|2V0j0BfYYuTK3fB`|*o1Z^K?9n*zb6snT51wQ0etR?T|AL2*~ zS~iM-XCgmVqX|d-c7V!zEa|mP!g2-N$ER=RP#4B@5CM=Ecc#0TB}5J_C6#)V1go+` zvV*Kzy0xKFK=0l@%ylbs%<Z^HJfB-UWU<;ear`@BNTj9&O+TDqKtuZ6_&@XAJ?byB z6sX|?tEL(PSjN)eKN!!UxaxG%?^~63S{6(c?>HuXJ8T{pD@*T-gM9oLcBqOv90n{f zMpI3^%YkQJB`5p|0!S2Wc#o%Jb@=*wr)EOvP~&9HKG1;lQIxo<JE!uYuPPJc2#A!< zEpwicaOON8j{)pk-sU-nnxv5~$J$iIe)>P~)gEBEY8jcQ1Jv67Km3*9fNSZ{p&9ER z*2+Yq?T5J|HGMtFX?J@4P@MCMqu#xHL31onXw5*u0cfPQRM#|no~>VFTcI`r(0kbb zqee1ou#g%!_f2OT@Gdp|qYC4cMZ!c%><}K*6Uxskx!xUAHiw2{s9G}@6)c1d<(@yW zU*jAd$a(US?@mbnfw5$eG4!gU&dV7;xoPL`*8+I?o*5V$8{gEKapvTbmjr&fl*cE= zb8#uQB~8%J1-Vwk?v_}(<8-1xp&L6D>t*w6K0W)N20yUvdyiDT8Z<<I|4J;i%zLN* zPVw`5TU^W?cyFNR9nYQ@YCo_cou!YG0VV6!>L`%$J)1>lJH{q({&?*V`4SF&{Ow3x z&gs%Skx3KHy6-YC5#I?eR5%_f-d)%UJi2guF{39(Ln9<4<ahN*h||IQn(aGQ2b~%2 zw|z(rk9t!K;L9NC1-<5xA!5bNv(GNSy8iZ`-?nd@5Sp?o&s8#joqZ-0a=CBIVBfs& z;vedn#=M*Ey}56-J0ry|aAn!ldf76;w@-aMf>!q)T|0d<5`I_UZc$4ydf$+wyY+H2 z46_;%X^~o9U0v<B(D(8|Tr(Pt{)9Mk`UvNr>A0oOnKkhiwPLy<H^Yt?)4enV2p{3I zovf_wZp=k_K0Opsz3?U_-^nH%(8N!E93u@{zA#qr2V3u0H%U5utIwlp-9UV^yflz5 zh`*5d!xG}hJY0qK^Rs{JZr#EdG=op^{a9Bwq~?$m5>!D~fbz&#mMLL~_>1AH;<=AV zhL+`w(b12*7#>DWjn;U2o})AzJ=@6cgOLZBKR<V0L=#@(M<sQz>dl(uk~^Hr=ATu1 zVL;FixG=S&mW;on;|GsCdVMAPx<5;zsBXx|xgSD!wCMYdHiovSda@~Uy6XA(ogCd7 z)`t&d6zsO4APh%sXMrPELBj**e0mS1{xm+z^pyQf#tWgTlV{uI=LgaI>e7&|7fS*) zDIx^<MUxBm5ld&Z*+<emQ@Lc(lPAW~0No?J{ONVt{dvzeSh6+o+)ln4@lvD*F@8Lz zZ!lppQO~KUbl6{U5YgN)*`w=xh4g;i%*Ou?ZR8FkEs8j(wuP??!rocNBR9Z$J)1D= zN|7pW(V-)zVejT=Y%TvCe_$Fs5Pw!k+56rp$fNrD`rDVK$50?r%P~}p(e{Zq;!Yck z#*sYw&O;m*3Qa#h7LB@lRfO>b^~f#pOOeJ<HwJybrn4Rbk2XvWy0cY;9UL5fpx2X` z+Q>!UX9oQrOZVMl7Hj!@_3qv8Z?5sVVcM-=pU(~rGw>LkztHpIsFEaPMkJ1vn(fJd zCd=934|55rW%%?to2%Z8F>@mgC)|4SbsM9#Fo`*pbJQQ7ygJ|SyD=9&a>$+jxk*R^ z3&U{9ZzC&%nF}-vzn9ro3&HT{^1^QSxb;~s`%_nEsz2LoB5{$I;2wC{f~fxc-(LT4 z@V;C8Li4f;`W>H&KHC6h`9(0YX6dR$Wbcu7sx0$oZ;@EmM0G}WOf-_(rbi*Lc|15L zAmv^tx!^W0anKGvEq+t)UBVK15lnw(?l~6RDN`9aH}J~blZ>1P@9fU$y}?O?X2tHZ zW|fCm91T4Rrh7JF`HJmvXrtgv@P7RYq+ptGN0T>h^5YYor89d+-YuOc@(4dTn;CKy zJKOoF`K{j8Ja!ZQ8Kn8msGrfrq5h$Bd8KHE*SkuJ7v-zo$s<d;o>Vz^MAztvMJ+wl zqXO@xc$3OEF~cPL{fP{iCbx|3H~WSQrdGz-ardujcrQYZ+E$@h`R00pwPy8`5CYpy zKO_7V*4b$aYV}Fc=|N|O$UFH0S#+uzlszmzvN_ym@5+_`Yj*rTC_jc;Exp})kt|d0 z^t1H(+wXVz`?iww)Ucwuat1R8f1ILmU%+ZC^l8v%%9$p`<r&BkhSv*5t5M*)@1I_G zPV{s%0L}&?7M;gE?ulvVGae!Op^pj~M6NFl4N*e0(MoDE;ns!LAC4EY^|4e$p2St1 zf*fJLQEFNKIjVfF-mE1m^lsKVV>-qc&tK2_GxGeAv$8yt-`4u~w$~Q<KDS+~a$&mL zexhkq4aa}?h8W}T$SagaHUr~ut?N(Ahp#bjAA$PM_j-3{DHZpB`*vf9xtz>$Ol>b# z1^zK){Wt6Lt!b8#BPg%vpbQzSitr9*ea5+XP|!`b{V_T}-Xk$X(JS(czNpR2i{WKQ zzDl9$B|fBbrW5(?D_~TOJuP3TE-tUSJ{4|okF(c!-ZG664M6O7l!|fN1qKFA-0R9G z8uTTmAm%&n`-74!EF#sztC@{4GmdM9B=Ft($v~+KX754eC)B%YY)DQmF(oT8#pEu= zv5;O%!y5OvFye{AdYAUZPujOpV2-1hy_3<2&nya7w?)rA^A~@nh>$&wd@j@Iv(8pu z>P{~_XyS*F^*11BmUWe>p>jKG*nr+|ug$7(rga^B;j0%%;Dejh7uQZ7%*vxDpLuGI zVPMdSGXfW`LL8EWs0A(4>|5{M>9QJrPQ7iiQx)_$j^97sb&K3v1B2a+uv@M8i`?0j z(*xiTi->oq{WKZPXLU#R{_9QhUoVtXPj@4lR{6M9H!R9?{f3DV2X(l4CzJ<rW2{9z ze0!-y6+-_}AY1@EX}*gO$~Yk>1opdpg{IK=J##f9Fn+(BED9-82lsV67*JDTo6ZJc zI}n=(FBF(##gSJcPro^muDW`g#-Dfb`PJnkT;j!zjg1@9pBa?vkDoqbFv(~uZbW{} zaO(4$EzuNl=7pnoj-AAy*QW{Qk?bEBPwJTCB5xd>NM-~b;m~J(;{l;6<6_RO#Dw>> z46)lBRo2fNPGC_Tl{j&oa+H(dY^3SWj?mOj9dRq{5m{~Wc=Zj|di$COKgJwlBgN_m z1_y7$5Rcy%-<jFE3e&%&e~&WVzAhHwq9JGVS{lb-fbji<_Yuis#oE}dEQ`%%ZnYhq z`{3(tt>TeftTAgS+G8!;pga_M)yn&*ehpt^NDv5mGuq;(_>LDPk2w9zpAwlGVw%iV zpw%gC!`o<Y0QSVCz5AKvP>5#bWBYkrVCj58QvX*8+M2+@XI=V{W|9{2M&mH06twnQ zU^baUT=?oMj&c63h5q9A4r-agYTq<Mv?3e#F){r?SfMVo<5m2{d}|^OeNB5Maz64S zvZk7^#NSH9EA>&^OKAJ5EveDwGot1elz1}ejdh#6RbkD%UEUh^RaTqd4t`5k#Dr}H zCu~zIf__byKIpzZCZKeK{E(@D&c+#ynp=?2zB1AWx)z98;_-Zc2Ba%9$Rs-x_&PDo zX(iP6+}KL>svsk1_FI_dpyzgc*ue(_#~w!U8KNlUjqn(y;r|Ig1Hk-~N~Kb%R4SE9 zrBbQXKq*1v`>B;P4*T-!f0p5Y{73oOAO2oG|J{i4*;l>#O)8a24PJfv<u_&c7bD83 zac$-7g-hk)<*ViD^&92Z?K|b}z5C_C!-wU`lPBfbv*+dc^B3jCix=hP%a<QSY`uE* zs=Rsgru2Sp4{3>`_Ycp~`-jhvmwL2b`%vmn+s%ra?A^P!Y~Q}U+`4t^2iJb~>{&T> z>{wZ|W=%PL`t;lR=5=>>mzy_lR$F)P-mUMuy1L5Kr%&rR_tvjpU*^o2Q`+0xtF~;} zQXcj`H?8ZCzQ26=a#_51aXEkfe0lx)_4mfOcJJQ3YIoPJUA@oU^Y4w(Yhs!`dv>{c z^=kFcbur$*f4>|#a-<wQc(9J+qxDXmI#nJ&eq8N|W#7JiW!ki9HGc8%%XPLq^XAW- zIkWm=UT0@#jo-S_>kl40C{w0PseUe8xUjBkuKQQ6Tq)bOZ7YWlA1+ImF0Jv3|I(#P zb?u86FV?lhYh8BS<BK_a_Ux&?`<%FT?OK^Xe}47Fcy>;lI8oOXzcFeTwTW7*qxb5| z2f4X@`*yXZH@xSAb=2E6@7deEdv~?(9$T*Sxvj0O_KNzfT)DE=-nz!|P3+bZgSGi( z?12LZYObRu<Hn7vez~SL8#ZjHW22VM62E*z4ZZH%xwHD|esex<+_+K4uxCy6U{h{4 zZ{A#E<d^poCQK+7E?juqW5${zK6!G@IJLii{d)CX&Dqd9Vi%jW)sP?NsJ%I7&z`M0 z;j13(>FKF!c|NSu(a}-oxwdTCvf3B&rf>G|-~aYGQ%Anb0l$nFBYW22k9GJe_wpmJ z`oLOht4888jt%+otlqhEr}nh|vu4zYEk1fRhEK#NHssm!z+Zi&H}n<T*43-(=AQY+ ziq~~9xuy^JWc}#3=wEq}6EVxVoQh4X5eGXFxB2q%uFsUAsvr8y9?JR{V;brj$eQ%m z+WloW)xYYmy7iacAA|ZUcYU$jmobA$L(Sj8q=C%cK(3v7w?4+2e2g&-b@inteHqgi tHGfyy`aSAU690Yw@*%$yZ&fR2{4b~5nCEpuxl{lE002ovPDHLkV1h=uqhJ64 literal 0 HcmV?d00001 diff --git a/website/www/site/static/images/blog/apache-beam-flink-and-kubernetes/flink-deployment-yaml.png b/website/www/site/static/images/blog/apache-beam-flink-and-kubernetes/flink-deployment-yaml.png new file mode 100644 index 0000000000000000000000000000000000000000..0932b96a96da1d1da9e87d9b2261c9adacebb619 GIT binary patch literal 39477 zcmX`SbzD>b`#x^q3lNo5L}D;XLQs07;^+?PP#P)e7$GGLgfUWD8eWVpiGfPD<Ybg| zH)Aju`_0$)^ZuR3V~@wq{y015zMs$QzOL)O4^&%I>Hgg(cW>OdabHCltb60eO&#LL zk?bGhFPxT)VK;7kyrBYqrSE64JMZhO-#aaUI|<VE<=gm3b(ez;nkD@BUgL`gAKgc0 z&qgkH@i`|so(BLdq|&s*n0L;VEo|-M-uUBDZS22q-c?xOPZ1fTSs1x%CERR-li`Hp z-uq9Itr<RxmBlH~e~)acRryk4MY-2ahC}-eJYDmq%#7a?Pfdr*FHK-OWw&4N-NXE& zJEL53;11alu2?_cHz?@1t_;{a4S0@ZQQmm!)G(`2G6AS+9Ydks8=zp~2O&qO_2Xew zCZq4d`-N`=cVQvP)B#*ZWcS+o@3Iq()_L+%%6nqj*VlIbg!i6n>x?=b+34$cnL!#M zgqOn4B_>ZBOvb7Lwv}dH3h=EQ2RvW;z7)59HDwAL%mTO+N+MT+%~H{%dug6h*_y%< zBBy?4kVl3Yrf#yzD=g91S;8VB<IYTS6M?5CAtFkeSuX2X0nw%dBt0g%NjDQc(evw2 zxh{Cu%M<sr!Ef<>k@f4&7oTS4DTc$fI$ZW;&us=D^YQWqGkw@3d|}#ks(jXz>89nK zfugKm!E}sb#bILT;cjHby6(>?CG@Zf@JA9<8h~gBSGQE#Xm@{(YE81p)#$zUuAB(0 z9&=m(&ABdUoL~8sG^Z;2E`lvj`o{tnmW?bu<x+y+0}CcD;^GD-!yy8vB|cKaKaI*k z-+V-<S7%?j+J5b`a$I|PvSb+I)_&bfa2YXc%jrAP_FO`jzsjdgni!5lEx7Kt;Y-tV z(kaFR59$sw3@sofpukm|lkH5(Obi%YGGd9p{)cs;C}?;-Dpy2&(XXl553iYp9f`6O zt!lg0tn+x=(2QNMFWU|Fyc+6e)vz7GTn~2OeX~M#Pb1gSdywHwXqG_7F^<yp@N{>7 za=LYx@gSJaTCzSEadA2uBJR5oFlqJ;_a0ae%M_<^oe!cyK@Jb$>*H}!N8w+GBfEpf z2>LBta#TuMIWDPZ%+Bpsmjp)(4e+*^wg2If_^^{($KtMB{h0LmYUYjQH$DeQE$b|@ zM;GO&OICe-{VNG+uCdHZ^l-?CZw8FUctN#lQF_vKWY_V|k73xUXd1ogFxra`p|Xm7 zFWFmDrRkLs!nGU-5vAqR^=@y$>M|~7Z&L#8YI4*KHnZ`4-vr6dy@0DxJK5`IlGwOY z7GprV?K7^yXU0ygKKNh%z7`@ovI0JM#C}UUWNqDA&5MwSVf{C~`GR$Dg1%DBUU1Q2 zq{y@Va+HvUn1*GH)g1NimDTzBhn!4bT9SYkAm@$=LdUN@;R3{&LpH(%Gf|5d&&ep# z*_^?!)#>=`yYhkO{Y}H2S{-UOm_MY}7Rsrdu?%dBEs%~G-@^}{Yt!?rC|Y~LJN=h0 zzNJLZ)KH*Y#aprbJ)*3ZDBtCwBG&H>1>9j^?5=sdEWP0a5x0^WwzqNuWd6bLwrV&f zC1uC5RIz`?HbS5^ai8OrW~&w;-_~B0mX@BolhMq}&Cqj1Gc9UOb01$`&Rk3QKti}$ zwP4<1W|SJ3?j0rQ)dNkt*yFK)Mk6hb#5QEdL%j=RKiaF^)3{P?C1@D3Gb4nooz$%I zWCXBmkt_k%PM{I6k%q)>|7s9qt0$EQxwRKM700S@;48@g3F9vfr<PX;#L8Xk)wT<W za*vq1#&c5~4Rt%cDt6;?IvFb3c{e;fd~m;<=0}Aq6J`1RAIKDhCdYYCE8x>wUrZ}u z4E3&iz8zcIjN8r}9`;(mfMd!Td-FS9Z*_PS`iR0;`aKQ9CSf_!i%cMY0p-00Us+}E zgC0<y{vz4jIW>S!H-wCtE#o-z-?~OQ94@!Nc7b`igh!gfe{ZvFeZ}qip&VE6V=BUq z;}DveIg{+z_v&yg>D3o;S_U~~Y`2ksMF(@?w)a7&wvNuxG8s*SEc8??@u7qqn%?G2 z2iSynyekVwOh$S+yR-;Kf2a>J6qcGp4RIO`&SfJ;I{W3bSq5ycoxZG<<%n&JgY?2% z$Nhtbe}I2(^O<H3V0!-XKtgZ2oJMwwNHy6H;;ayq@t?(3lJt%iG8Vfg)e+RSRPbK| zp`Gts;^MSEaf->lnEA~4RLM1Iq7M4?Q|EYHyo9t+D%rMnLUozg4$XyniHAWsMt>(c zsVo;-w#E9&<sK!hr)EinrS}=zrfCQN){#vGFE%5xIcuATQT<Vo&<ctWI>r7ft7$Qy z3Y3hPjia?c!4x}9oz3MSHCiX2JCmQ>Ya`;)@9bolZK&3MFi(Q4;*84m9J6q;AI_EQ zg@x8lD0@U1O#lBFVSr51K{IS3vI#2VHSLwP#|xmZIfNn$9=<r=CgYJ<F)@Ntx@6IC zZ`I`!&94hT)~}O;v?{x6mIDbO-Q&x^g|lD5!Pz@C*Pm!#jYJ5SWuazsu%@$U{#h>) zOnhA{iwA!%HNSk0R)l>E%Yz#57g&|}$S^j~mw(M9<o7iei+~ITJ11ir#~^M*Ft=cW zsnjzfZO(gtU`eoQc?ls0M=>!mi~r;(=Rf>=j7GVrOlH`Pxm?pd)f+r0sogtlB<QDF zrY~RX@4X?SkThi&7!=^w4f@C=lD$FFQY{Wa5572Tk%RASdzFL4u=oITx6W4Vu1n=& zuy9-FhCm0+f=`ayg9mG#ClqW<1PhO3a-Y#<56@K_pUMx2!-S;AM}3s{Y~MlLLdep+ zqLng~+L#Vxsu<@Z79g%|`-Ab>K~d8`-%Xiz<mcz#3IjtI1qB;3`C=>9$E{KskR4;> z&o<<F+5K@ia7lCXxNH2-nW$wkiq)fxY{CC?E93f|k^G|+JLf>4^g6K&kRJ%S9bfP@ z$&;$MdhJd|{xVJ^gY%gOtTh~hEg={S&nvG-%@g|L!Q*C$bYrg)Z@@*yJ{^OTt|dlx zH273j{I`0XhOejM)=!B$_fy-SLPZ;mRUIcKtxs*yUeOK<kNff*Kq*MVn&pa&hTXw% zIT3LD)bn99j@KA<F2%%>U#|OZN`+&5d=)o_OOIQj`o?AM``_OROG`F(gXF{5Ig%9N zUQt({-ax@1y)yxT!wAPta-hQs_)!xd{r%G=6oD+C-NwFR+@D3pd*7X5nJzN?gK=|x zI*$Eg2}O2Jwpy=1lmXq?dqa`YZl|W#bJy#gTI}z+<&|jRG1vZ=c)^U;{zUHw&$iaU ztM$p~7Y}ky7BUTWh@1KgSb?AXqzzKdoS|v3(TVW^Gnku>JmNjN;@k3zaO09kW+>Bv zfP#99`=5;S`IR5>j!9=1^T;^C*4@2Vz*LXlN#qlYMasFBF!V415Q>A=rM<R^KQw9t zuPGaHjLd+)4j5(j8JliTIOR0^v7<XYcH*jE!bNnXo;f@?X#-#?kvKc5&82Fqs;EdN z9UatxNZkp=yPNEnc}oBbO9cgmJ0wt=Z<&p>bAm9ef<6u3$+m~ov<&m2FdD@lzu{?= zM{=>v+SQ#B4x@FaTjjZ0P_AA=daF@O%JY&>!%KM@BY@LNDb%0X7bA;OerfGXeR~uU z-KSonu)_MIpkLRENu^E`+F!>SB-yncv4OM8lhpc2rRS;Vtwuf#Qc$2Sq}^t>UCEe7 zawD{Y_zLKW@zzS++&ZV;MEd{TK(F!@v@J;vyM-x>&x#P=UyKw{jWl`K6<Tm>9;NyM zPAm*KBhu<P?{^V^K9tUOECUQ_Vs$UPZAElZkY%sc*twXO=FEHkRnIme@3LZqlgMaP zlo)Zpce#6#w;vlaggf!dpXn|@p8fqVDh@Q{u<0W`Ttr5;iRPVxLbulEMSiFOahtZ? zE9EdDrnKhvs_2-#Iuw+&N<A--V{xtBtXPq9{dK{?($l6`Gw(0K>mZ|%kr02<pDZVf zCYRY9n=x3W;cRTmj$$exwEqGKmBfK#JVFyCn7{1CO6Q*z#jBL67rM<-Rmp*-9k2PO zTC0D>kUedF-CtOUVwcJ%Fg;EV^xyf;OFV)ph}Uiwd2;a@&M8zxk4H6MYv>L$?Ax{g z_7|s=|6F}|#%*d4m2=wT!FO8Z5%oSiKAA*SW;W!WuTP}2>U5w>q6Rs(enSfrU`S*@ zsni?+0se@Kr|{WujCwwR19-&`UoD##E7@_1=N_-aj-P+8{5=pp)vuaDZb>&0my%U> zjjJW>Nu;IH83QfJ(*y+?**lRgFKq@&wrt`OpI{#ROGK6BrZCpAqO#E6fUU{EoKLgs z;kJ&04^&h*o;EiU8P>ke0&323aQ0KQos~P9rR!Q*$-XY-UzXS@lda*DX*;BPJHH{r zarbx?Kn@>1y!qdgJCV`2G~sHigZ0cPi~k(@QLJe$oEgn1hX|aQ$jpX$rOn(;e`_gI zrIvnTWx8%KvMsQGZLC>AMB|t=hFAGWQetX&d2=XJz<<B^d{q4ppryW7$>*=o>eY0v zo%Chyes3oH3fL82G(FS%TDX-IJ_(t9F$mzjZ*kBp$GvKk!gaV?WC+~Rck?GMjzK8v zSZL^;%xc8ZX-*d=Kh<xJe4YHDKX>&>N@2RQiVf|7l>fvBqQa>hTM+TDT^i&POv#r1 zDve32T!cEG7HS>GShNL84RE-nHU}?;cEX#t4&k(LB`-VIQA#v;NOyfk5sa4Z&shUZ zbz~LSZVv_s-j=;(SX_}3?ldd~2%efl65Kui94gkexJw^@UY`s+0f{b5swaGNzU<*N zxAa|Pvbm%gv;5{GH3HYAJlg3sIxSFtFSi0`oHF7}Or@XtV!pGy-k{1ZAtFMPjvFnh zO`n-Jl}-B_wA-)pJ3Bi~@7^`w5^@|5#&x7dC#RlXFCjSb+hA@W+(PaPtz?L2E$uBx zA+I3!&*YfO)?s|qv{fdh@d~F}F@4I*i=OPJ@b4TZsTBG-x?NB+;Vq_niLdmd$$89! zFvER<yrZQggU4a$LGo8>?FjP=MPx^Nb3=5nZL5+4|40(_EhcK0y5o2aUp}n+<g%g_ zTM&Xr%Y%ce_n;A@9{IfDcFh`lJVis(95}_&9$y*r*;YUhvuW9!>Wb(79t4)pvh2e= zqOfmGn^c2JMi$NG?lc{^T$m9TfW-spBrukn2APA<xD-UDB6M1`bd;ixTPL#DwV)zf zVCv0QsL>vhV@S`#erCU<c!OnQo&RkE+aPOh6APz&E8T49Z$0w#nbHS0K}tDVHoaC! z>N5*Udct+X;{%~~92%i}V;nE)YX`Tk@<i}3^|MUV5Z33WvA=PzFH8?YSd34o`SRsY zcN}p7FY0P)Of<?l<Ly)i0t6|p^DryJ*-Ze2^~(b8(e4Jew`w>0uJ*%W8Nz9my^8(_ z6jl~5P)UB&JI&EQe*I^#<Bkf@+NDQ~6RK@ap{wv`K#lb$Q3k}-^wlq3HvdI0CNdgk z28J13?dNhC?5WXf#%aad2(2+CvFg`3KV?S?h1^f`zc3lPxn<<nZU;0yfNLyRDDg|K z_668@*Kkljy;Tolq8O_`$N0+LOMHP;5bC3L56ac6f7lugls_Pi`m`*1tDZDT?C!fQ zL9r6G!uQSyt-|1yhNhnx3MRgQK$^l5jlyWJ-X8AG?aBCOYAv)9rmD|5n>gO~-*6A; z$f=}&vJZ)~1Tq;^3|($t*1XLI8lZ{+SQP?dq0{kmB`x(O^%nmB<&6X?JN)DIW6Rn( z+F{y3tp{hK*0+P$5Q0B1b?xqZJF}v!6*A(nje;lAe7aZC+%X&+%6Flxq6C+f(0io+ zBx`hCj7;PNAmN%_mm`=b&FcY4?!nyIPEK(6*pb8H6FULMc+ZBES-@Q-0|qs6vJdxk z7-2Pji}Su*t{AR-4b)yxr4*eeJBk*S7Ox;j&i`(wwX*l<v9F{>rRe_aCrv&9a{t>u zM9>Ll_+3oWf5uWplu3JE6CH>pxBG#%N<9%Jtw{e@Q%Iok_b4N^EhMEKw^s?P(6nxg z^`gkJcI6}`hXXuL-EvcbbT-I8<RY?r`|7XYdJR<evi0Hc_(=0BNeKQzWt@NJSgIW4 z>p-?mSs(7upSemVBQo{gZsRP%sLilP|1TL)dYP~}bgs-{CGcdFPtEu?@7&4YivKLt zEeKmq{nuLb=sm2cXQ#0$SaOoDlz6+P=j^P?Uf8Y=yG5r}CwOWo$2C60d>6)#LFl~8 z36Wyw3*w8w=DH>BwL{yMm|jT!@ONAJTShy4NuWOH3cJ9sRWmZ<sLzyV-3!Xa`+Bc~ zYCz2IWXe6Ag~Mg|^o^qrlQ>oRHBWxe`u&Z{0rOv!rY=%0T=?%Y59PXO|BF-p^2PrG z<|jZ?bl;q5$BFBorc}JhW^4p=B0y_+KOWp8G{l!QWF6otJKp7u%KIk)@%m!=%m#5F z(#dXZN|Ul$d#!w|v#qMV95nAwjQ+C1kvdIE9P|d^G;l$Q1>Vzs#11k|`m9OjQ<zjI zp%Pkh?`U8X+;t!^UcjkffDG!}>hBC$UvB85;Dxj$dnn&eSN;Ckh#UtmL!3<rd%{=r z&>#@_Cbb^9@-~}PyNCxL=>x)7LE~%=o7CRq+8g~W(t#Y~b8l!Wi2PND2tNLQU?Yt{ zx2OIS7ev!%Cn@_EOj6NHtI+L|N=NgwkV9vH!<}o0JN*eWvqqtB{mf#WA&st73fW1L z+qL}(S_;sW%8*DkD*(nd6HO*~7Q<10BJ8oCMJw~n0xVk+;4fE{l+^b)Z-82r=ec-C zLiG82l3QNb5_Q>$x-?gj%Lgp+F5m?seDuw$*#3gx|E5l>Q&^?GN-!$O!d#W3nOjMC z3>xhDFD+rwV))6p7OWw2Ft*}Xx&_h7%Vi!ljr!=KfRox|9ja)Sn0*!8_~Wf;tr(|{ zV*Efhu};Z*%{>C}Plzwzp9@We9Ih?u&7(YIb($r_q1x9YaVsT&BnOp0yBKh-E2r8* zM+*yix}}y2vsBOvvTs=2-y)s!$v*KZ{^>3J$(H8H54cZ6RInFEQUN}>?Rum6y85Mp z1e-w>(YZSj7n&IeluPZ?DE1}!Jsb5*)|tZ|#&+N`{|*KObC<V1li5dNjdvrU&utr& z0=Y*mTrE@{aokFcWa&qD_Du%M5WyZ0^w4<4WhqqMc1URzjU*h2zv?5Je6bbMu1I&e z)Efc427h;v2r+X~LH_d-==C{e%eEYbfn|tO?zBo8H%|Ven`IGJTnOqq)VQ4WL75Y1 z$D?ZSQ+{q-iBMcnlEf^7&~nPHZ824f6a6(a4|lg<<k!?JO}r@=e}7f|kSxiYkItJ5 zUW=LDm3RUZGjWu3L%XJbUr^uvf@A-%IxE%qm&}wbrE}o;PdV#x$4<bYE)ZXFJHE5_ zWri+cZX~m$p~F0VWj5tMWqDtUYs`XmWWoBsO=)<w`X7TmQu^tcRO;wolHz+-NQpta z>-p^z79Y%S)G4=C=vrKJGlx7v2W6QWd!kLxtl=dGjV23dJ2&Gd!@qhlpMF#!pN}?Z zIP>^3k^1G9iujdJ^w}9oWX2DzJD-!ms#$70gvJtp<_x(53z(=9Q#fZwM>aVFsC=WL z1+AZJWSI_rij$O>p<kU{^srd8)-|;MnoSOy*i@lbRZftd40*tRIX8Qfqga0_J<AM` z3X&bL+o1nBrl1zEeKln?y64jLmtm%J#Blh}5poCduL}SA$2gBxP`ecE>P4i%$^Md$ z=UZMkHhCO}<4O6Yr($65CO8}W7D>x-HK5AZb%8}m=JBKi9PH2<7YbJKB%Ya=gVdlG zj@tgXMaksB27l2h{HYN)I)mxGI3M(I#N2<4^buFN$HDk@g7!1lb=ik_uj}m!*1mnG zA{rx5^%H76NZMI<BX^<xM&;Z+%8&zO*<YE;1<fe7Gy(hVRg3vAPy=af?*l=FeLo9n zALd0IZYmMH63IGfZK9ucNX<h%r?5Cd@~(^8ZT4k+Qu7{`W+pgYdCGvpBc0})-StwE z8e8{4tr*{$8POuLaQFVx8QlX>*5O&~Lag?9r;=6=C_g(uh3+EpRknmL53u9`i!8bg zvw0Hj3OqmfU(D_nFN4pD(bKP~(XCK^O7(RfDS7KxwL%9>eM`B0C)u4d`8IGC)zoc6 zqKvNVJ518wSPGDeKHN03q{`a@0c~C0)BZ4d^cE_MR^D?<tV<Zzif_zRE0e?AxxJhF zUWxyzW`e#E;T_Y#jp)EGlnhS}IY(ba!g7#-?901@{<p)!hqrs6)w113nP+n6tF!Cd z<!*|tNNmyHoHi>buQ-f4aB=OV6`%8uM%M6G%fXc&<*CXpKaRzGx8eJ#U|ebZA-BRH zKWOmANK<C%@&1__gPLJJ?OLy$Y0c+9SyN?ZTA6j7t@4MtNqNc!LP*<e+USE4`wjNC z`jEG*(^8Zb%ucD;Dn;q*NT;U1gi4&$Ks;j}@+kB5PH$L(F+%=z+5{az1*%5`Al{r< zu*Phw!ik0FVKvi_<V6SxMfaAT1@;>C$>TxcxDUu8OQ}fxQJF(W_D<S$%g#+ckW_-C zX%ArU%yxr@uVH?RB;7;aoeTP&cTBD5vz`j%+AYns>i<3LUk;GJ-2Xok(Dodf>gZ2B z1Tim0AveyJ6(Y}cN(8kINX5GGa(6yYeNp@V@^77lDI{#xHA=1j;%QMoE)u6~%*4~q zP;(Saz|#$`u<cYul8GSGS#U(#e%n#oxsWV|<~<fN!pjoy?+vTVaJ`_Vd7>LcuQT1y zxZOY?)BAqI{)-T`Hq)CdGuVy~o-8?Fv7<`<l87|i?8;t#ewTC~+EzsJ371wOCq1D) zU}iS?7OfnwCz?-2gJ>&QHO31}FHMG1z>Px0l5{a1k|=fb3y|?%6qSk|o{g&GkBLt% z+_D3{&+$DjB|w_xa7%F|GATN_O_q#>dNz8>I=S12>$FQr3R|$VQzRHIZs76pJ3Y7L z&8j%&ysBXv(H~M}@i%Uc^ILyF+UUIw4_~UOIxZ&O-X0Z)q>xZ;(bu=nsX68S>les* z;QGD_&ThO~@m`)qQwZ0XJwWB!&>4*o?NEqcA?-Au49}RXn-uW%r@ja8*sQ=RAu-x` zLs&PjodwcqFkJ+?xAj_14P4HWU3yI=CZzN)PVWrn3A${r)`_?Z0II4K8~2NY&4<Zp zHYsYoH59_P%?L`QXn3c7{P@D9F0nk^9Ock?`GsH333{5NJb1c57wjTw`Y}KDtEn^L zZ3V3@LfRGxHwqO`DjK-U`9~?W59I_dzUnjMGL9&xBq?JiiItwJ=}j^UW;lhfG>CAe z)*1zLxwiEkb+XUkrlQ1(|EX&fMLCLd`)<?#y<F0?->fBd-loQ>sG%WRe-|vP^5of1 z<emAqp+K&kh^!aW^wc>G192jp?KO!dqeNrJdW(&F=L^Y6RMJ|r5d2xsO6v(-#=}0V z&pa3h*W8coMy=c1gD<9w9Xrn8FZfW&7K%X=-d*MQ^+LdI8#*dFM#(+z?oWRCJo7v5 z2<Oq5g|wf^nN8~D8-scxSy(nBo?HHQIFR{^eji!K>_M6d3^fq{QBzrTu5z)Qy|K{R zS?=u;KcsQQHl4flXUxV4RknxTj)&$20Ei!!<Sa-!i_DBsmH*gB>G&i{tp1Jut)8e| zwV3uq9@W;KU9;YL9=&AA9vf#{#3BXmQr<z3sUDj{lsNh?6C4hHq&<ubq@c#863q?$ zt8qVG=v@ec9Qv0_G%9h9{Vku&f9%<HaJrgLc2beHcWPh30p5TWWuV})-?mH$ceUnw zbtK*;s@-e8l|D72ylG#Id&D9Jp?Ox`RE1w+=gwSO8navbzSIYLPhr1>5t#orfnb)I z*_EoFHVa;BSf5)n%m{Z<%Cs&K=(;3w9QGOE{^YhIr(R8k#Lc=i3&*o6S7(OTf87HZ zGyJL2eGu&y)i~`<jzQ@>88_@Dqp_PKWq?IMmH!Mirxr+MtP`WzrsP_v)OOl-R;ZYz z04%6KxLPjbF32Yhk{)S4D9Ac>x08jSr&b~cs11z+s{&ml(-<HjAt8Q?DB3)4%gm>h zL8K^R123cm<gu*oxlw`5h(#TMC1oyD_VhxCr<p`i9C>fs3aA}HPIi16S^g@)t2k=B zW)^##%1-QvWC|1MB4e2b!wIon=qfBMzW6y_%elFONGk;8e8BmA&~yAs!QT4hKEJ2D zF4<>SJqDSw-4m}b1W6RR)x$U7C>2K4Kb1M3v1BkR%Va(3o+KXX;c}%(!7}0Wq_h4n z^$L;-Lxkae&6B+{&G*9gD5XY`;^@pTi~g2Oqy0Gq53VLhQ+R*bjNRzM8o-Q^Mka6- zeA4eraCPOrh4fl!SSPk-$i}6=Ac&gonm)wEwe8HOE{A`Juiq-gIOKza#C4rAv5!OV zQX>nm5nFp6L^N%!C}{TNu1;sy1A!And((8^?f;Tex;ogWR!B8i>1?Gm+MifIn&^?C zsiVv}usIUS6k^#(&9f2x*(A6B1(xiK9UXb_2M(0=60isQqwRv3oR)$vy)L3-fkOw( z10)Zv`)sLFwMQFS=wciOj?*Jphwt@qqcevXmeTXtL12Gg<J4=l-_qh8O~&SOEVd|` zvVn|{y(#f<e&ecn4prGt5|%+HuU5GfrYYt6G5pkR?u3=B8vz~xgZRHP;-ZF~rl^i? zzH@-$#T7W7<Tyi=JibQ6u!PFCq`6)^iNE~0yac_cL0}d9NtlR>QgFmE*~1?hBeBvz z#KCa8jbw+fa5C){aL%eIpdzP~r>jzKk5~9C^jecDlgIa<VRYZZA)W(R?>|4Lw-KgX zVTpF@ueb0CB*3!UADcpdUukj$s$|23oZjTy*I<clUmu%3K<1avqSI60o@vF3f*fgI z5N?y$>-)c@GR@EZ&v7Gb*EY%a^1N}W!ldS@#kD5~k0(nFWKs>I#xN+PyVy{dV%l}g zKVI8}#A*#J>qlUFUMrbwM}F-{nWZI<JWm;QORb^XtE>0Egz?wgzd0jz{R^yDYT6x< zNmCLM5_IWD#%Y~p0<~E=*|F*uXmH`}vK;k`Y3-D_LNVu^U9*W`Z|>saaP@{oh{pf! z>@P<S@zVdR&PspZ<o(a|-Mt~jrDJitfA2^utU=Eyar!A_(1Oc|&-)oCyno`YM%qNo z^khaQ2@jJ2)7wO_r-ys>Snt{Nvuats*kshSe-WPjjua?Yv?j}@F=xM`7cK_+##j3e zlPYsRJJd%Q>5E{<Wp@1MljQK`q;}oLA1`4s;W@8=W`}pj!Mu#F=<JFhqW3J#AO+O1 z4A5m82U>WMM36WrTF)lTsB<OqK;rb}<C6!9a@ipsW{Sm(gP^ph`3hi#<SLD={rF;` zh`7?#n-A|7<%kuoWU`<i>hiE$TP!c{3JDPHnAHLRi6X<-AiVkS%p$F}^DaV`@phKR z3iV#SSW?nUc2E^(a(mnFcn|CAZE&j2v-k}$M@x)e?Ij~U=@So;>2N4(**{c5cWyV_ zNV#QYp|bHL$rE1eW9CHu-mf0cY3<d3D`=ilESY~_Z0{2#U5-6@@`TahLAJTWBa~gZ zvV9tNMsrHP-No;HI~$K`P@sE_v03i@u)PXXqRO(_jRphkk&Bxo{RzTsHxoGmG$X)F z2ZM1o2^D+deV^MK&3lhD123qnKc6>ly?m>%UwEJ4#C8hBb!&nJS-K}fw$Ew*s4U_` zU+R^T(fkTH+9M&gLVX~W12>&oI7GHpH#t394tgP6_DHCJPn0FGd<Xi=)$P4joYt~K z;eN--oja^vLeFge1%P+WP~27nl=o`m=5MZhTvqa#RtM=6holw*04{pZ$(}zIstDJ+ zAl<w+;U?3^=QlB{8>|vpud~zmrGJnc^Z}{PgZMLXJ&0oF?GmgA{`H=noxAja$rN#= z`MDAwGZeEqs4?ypAUsK%4mFd%r_87HupyJs@;i6IH;f8=kL~3e;+d5n2k#}ag?|(l zzh0rhQn67Xm%t!oGY<sd|J`8fLa-da+ARLfudQ2B{e*G@pXqFzFMB-HHRmd;f@iHd z(A*G%SF|V8SfiCp5NcgLw-l>!uk#ai>nd&M0I_q0t@&nB2rZHr8;k~fxeJyfzh>mD z_rZ;yHpV?<W6k;Pr<_ki?^K4YyjnghwVDU&W9Oarw(PB{A9_?Y%Rrg}4OXoyY|0wZ zL9dc&J=xzy0yS-mj|KvEh%(d}#++0t7|ceSO0`pKlt}gilFE4N1P}^*QD}5#oM_AP zquPvGjk>Rn4qhPisMN6@tGp2&C(YRyMMG0(m>9r0`V{!ab(`;B=(k0q28Nj<n9WZS zW-$TOM}1~m=|9pgwc6p6_I1jOUS@#4>iWDOJgZF*yO=jTbB#jo9@#X;_|q@0^5n?) z=M8%-tQz1=6OF}&^2#0g@ycd(!aN|gw2frDm2x}&USsA(WYPM0daS8zx$7ynnwPtE zqcO7ju^!H#Jkg%WyMM&1BzG%)vk*(+d-nyw9TEObBIT1*^W0?eCbLKOZk9)mPn!=5 zkT)-TAmOf+h=N#(_`}A%C?R}bZIuP}vae}A=U`Ah=*z637Gxl9bmuWjd0`xEzL6<l z`uaYFr!Z~DSqf;hf#xJlcDWs6TNBcsb-;w>jKvGzthEExniaI*Q0?v@Oa0W9=?`%# zNDqLj$NMhx{WP(8Ih&_1{}^ZP)JR12l6E%RRMbNTH;HU#qc>jVG>ydSl`9#U3g-W4 z(YXfvKO1uJlWKm)yXZ0cOiPQ{LBY%is=V{A9f7(<`!5XjuUHO~5*<`-9T%4Qs7P8p zt7_U!``7?}@MU3@^VUs-f{Eo`#R<pb4$eDj6w?Al`#UlljvwSWIpBB|pq+|_gQQX~ zmsJRaVV{$0s{%zq!oyJiS$9D##^dU=`**tja&lyMit>%PYL1}P$dGrAmQ0MfUt%N^ zzxMIlO3%BL3(EYNkcIRca#;J1AC=8;SM{D@;FiOp+@t0*x@USX7i-^X6xCo;*DkB= z2Z=5BY%3*>7-b3Zm>XHApC&)`GCR5t9BO;|B`_SJJ@2{aOXT+EaFn%b*bJ(K9ZCdB zX!eyxk5ZfWcNJ=^jjROf^ypOps;2bp@X30mWJe$;i3l;fy4(E{evAH_@O&TtrKuej z`r0>&GA~V5pFF)IpLy|ZI1nJYMIi-}p#@LQ|Fc7N)aL+G?n|KB+=@lLPW5_=QTDt@ zL9x%$0_$Y@uV3%LF)CbT>GKd4dq|@0!Q@5xs}@;ykcy@MGwxu;(nc<LuYN{HrrVs` zbbL<CWT3r^vB$YXLX20Cy?{CF#oDzv5WqX0l45BY-D||M7&Y1Mv30qs&8!M`o>RMg z&@EAR%OYf>^YQ4fLkAliN|0i*xzPxe`o}C{znA2Kp~RE=YG^m`>M<)88gYSCgOfpQ zyxP(;Eu)>p&OGjuK5!EW@9c!PXoopTk)4NrIGAK)VQ?~lWjXB2;P@-Gd@7!*;kR0` z&6TBo)I`{V$9E07IS+XC1@whGJn=7Z<c$4j5v*0q(iQI9ml?gP0%@6D4OMJh)loLk zhn-l=<`!v<aI1}TGxN@uo~0b2Cqfc$!e7AY;BpKl9^MJn%0QufsY}xu81lLe%l@)% zTXb15%hpVKl}tZVXhZA%UQ8+9YOJ=~zEk&Ol>#osPhk>YFVTViwm(At4yctwKN@{% z?}I-Kao|usIUW`dV|9`9IN8Uu&P~u)-P!7U6Z5^k0WxX@?DY-jr}`R{wJM@upiJrx zEd1++<`OPCGLklg9RV`_?I4dU=Y*v|pa8>}H3E8S0olN_`hp>s!%;P9bVAc3L<a-> za_5yvG)w;i;auwli!s5RGAvOF^P_i>gh$n)V#xId*$8Z)LL5Ni$o`9H(Ky9tNzAH` z4E3B-eU+)8lBF<R?D4_=P&zW{AkKX$Z_LMle^*X4^7ZQF_KsQvNI#k`Z9kAbm*`s% z1(-=q2cBaV3uLToT0hpv_GkQG22=u!QUQQrlV8y}#sd+ml+Y~jZ8ojd=m;<?+f$s5 ziOw`z{{yG^hXY~HF4-O;dG2%kkh!H$x{dT<rG0U~?A{qp;Sa*<=LIlc=@`)oRK`9T zb5sjj@wneNkD)?__;JK84SwO*k`Qd<nn_C1&YWiVl%qiytkn3@#(GtEBo%MbzWOBh zEf@MN5DEwEK<WDH8E^Uzk0%#0IM%E#5SHCuc_d2o(m~#HjaV1;SHryO2PIqGH};HZ zUj8fg-MXlD(#VReQw05Gn)Vf&fX$$l!Y71~0iUi^R^K?GX_3&@d@^0|)SiJ|hQqP{ zhx?<|zs)^{YxtCtTL@LxM$`{6UW8HMHLew)*bInWv#9{ep_XMgH%4X;dSRWJlyWYW zbAd?zow1E~Q@E;JV)JC3IDmNYoAnp6IWgEqY#jEzqLp$PUGJjaXnAD)^Iu=LtCvr$ z&Dkw&F3w0TyC~Clf<^Jl-;;h?^YL1c5LuX>-Mg$1sabOQ{2yRNlGPbiXYml(>~1V) z^FO!PGw3hln5Z9DJS|UIa~#;ca?@PX*6O;%*7?hEtSMu{*=NS578XZV+c7(BwINAZ zGBaZ+4UNyg@v<5-kqDs6%B!Jiapub%1|9IvX9~A%`BFLmH6PO;V>XTnlV3(ySOue$ z-FRNq2_7W1e?`0~-0TpPQ9H_}Y`=R;K5;`}@t)eRGaz>1k{>>C`3?zGZjGcFXD4|? zKRAxty8dl`ieXi>MKtHr+Gd+KV-(xOycHJuZjb$5As;i-T(w0+-vwE;;_aL*!<=14 zqP`gaEM7q?{3ddG*AoMv$x7hqg=I!7R#<1G@Vdcq$mh1-!HFDq*cHdZ;8lmm;fI7) zQ8a08U{WQ1@G$|B8%#fVBlzuH6pEI*4#-&%{AGZuZkUkF?u_k5ej^%#HHZ(u?46uh z*R_Brxo76$vA<nO_7@qUzxMQ>i`Gd68|{|>ZpyJDb9EHDW8W%q45(U1)(F5Y$J~LB z^~?y7f{$LcSR<wqFQ5(EfsX(}gZWo&o+UyU^^S(BXUksG`C_MQfws0m508ws6zqzI z&JlT%uKWv8FN6L>8K0?Xf82=<u=XG7G3+_$;G|et`JC9+<?)gMLcQFZ;Q+j~iL{UY z%(*JZDxXSsY@O#G!%Dhw?zYI~6fU&J4&Dhvi#}X4OMI96u-{{<U1%q9&C}XHg@-Ru zVQxg>lhrP~I9Q^Yi6T}4Lt7PfEXhvM@)_W0`d-`Zog?S=v!bKSb;isF(_Tz}zJTpE zlbk0{V_KsaBF9~d{#Bj2am~mC=?)rXQ8yiz{w8J<|7F+MEWUjXV9Vr1yM?wk+X_sr zesO;DFg)^`xjiDSM@H(@?=p2^w3OJ?s%EDNO8@wl+V$XQk)4WSaKMQq>OAIzI}I6V z8pj$4MnSF=zTS_BmWB)xr%9e&xhns4A(v4v|L2ZybLu3rf)pzNt~^e*c>xDLy;`rE zQ0+|!dYnknfy_0><EU&!P<_ClMIo~$hqdSMwoy~i*xN9n;WdyTPS?qQdK<9^S}@&E zsSZ~DWHSs8>_-EunuJ^u?x!D{GOi+OeBQ^^C>I|GZ28sMJFb!LNxDM6qpjR>=QoTX z4X(36b9Jv<rrrm26r=CEczj^E#JV(O{CFjmCoA=D*=`mZa;5U&zVw-v<f25={+Wr+ zQQ&WCC=jH{5YK$em*8Kf@<qZ^B2>a!kyOIkLotBmLxk{)1n4~#S4U4)DR+tJdd?p& zdhaLZJQ>RO)y&HC3D&3{o#~9NMApvNiX}hfrX!_F)Q@`dF^T-<BRJ9>ZjTIp?j#rP ztmGjzpP}<I2Yj6<rKVRB5<^05%ITTZvdxwjeHVSNpa0L>B^j>ErOlVcg<vLuERI!$ zh^;X|4Sp`Cw{RPOzUt-5-B6y*ScPQf*kWo?OXLV*wYFtaxO`9%r5}wondNwmt{&<0 zY<FKCvi=dg1>;Kb&1OfWIVU8w_or_PzoyC^QOe1ZP>70o4(C)dq5kG+febi!huBhl zFnNKjnJyR13AR4GV(6UPk9?d4xV6_}xI&vm%}?c2v=t0bzR>y59>8Q8te#K-;EpGb zLnXQTT-VUXzTR2iIp~3<d76tynULoZRml^gr(+`rJ`%}~Cb`+4sgkK;0G0$l0@v~Q za><mTPE!ii*uGkxzq^c%dz=%mx1h7^dm*mI7;zfUwo_1N+!7&8{pH@dvZ2hZ&#w;w z6&;Xr&`sgv)i;98V1JZKO){K!E0aGqie>YS3n7$l@G*4(64qCLS`{BmengFFpOmdA zd!e-Y2Iqmhh5h7-lc$XhxkeF~3&pd^K7PyDkU7oi4*>gxzkdC$hPr|_o1Gl2>+uQ5 zM#GcHdhj|rSNVr~YT8q_4Isrhi&cn;ZlMydKKjE0N7`eyJW0<j#RPs2&SydbIUiOO z{U)F!{=I)Kmgq}z4E%750+m5wXsHo_d}K0iT{0?O2mLZB=e_j)219MKV8^jZLV0lV z<<bXrh>dC6DXtje9khRY<{`slN*O3^St@{}#Ta|AT_GN>J|*ROWHOmY^@~gN2F09r zQW9JN?h%CB>tLrD@E$9nIh0c|*U=e^C3=8mRJZav=tSwD$yob*Uiiz!ocyERE=U)r zwTzH{RD6=Qg{c)<14hkQ!95uNL00Hj6>x+}42P}5k6HLG+iw8B6TI`}lqvJZt7@NG zo5I%GBB4|og(LE9?a8z%6ojjDhsqT*YuY;sw<4V#jX4#$3dI#uh16&rB{hBT28%f8 z8m~kpHQ>j__*bJ6?s5ukBGaNNj(+Lj#@VF}9*4a(uTd7{qOpC)mK>EENSl`}Jvn0g z%u&hQMCJJDqwTx~>%4v&m6}GfF=LlLCSx`p1C}tLAL<`In0L&}tPKeo?l601_H^<6 z5W1l14_nVDIi27VxtuOTIDNv1=|R-B-la$xv3*7g{kLjfoc5@OBkm4$tuj1sCz>^E z@&O*<4lt2h2*i&PxXF&t6|KX>AxvvCW0r=hi(@webHJ4PDK<0C+Friy{byWVmpkXY zl-lFdI*&X2l}z`Z@hvN$fw>L_^BY?>dAPi>e<<3q=~bnh-SNuFpT=dC3DRi_`^DdX z@++KGKaf-5xPW=$;WuTKfVMkHw)S%DaE0RYR1$e-xy=6P1=>8QguE-l;FkX$y7F~v zlfki_Z`VO|qcffT<vo%?-qW17mg!_X7`V*w)s>OJ>-LcnVa)B162?RPI}KZHGGsu7 z%KCy}5i@QYya4{?<DETxx-$Ol?+;pD$?a5?MI_+$K{RvtgJN`XOZ!~$lJd>1xu8Rp z$yao4+(O5;O1ywz43_ngrBA;Sc*CYvhVD_}=-nSci?!*8SWb_z;I2yUal6peLeAm} zvOsoYE0N@pOX7G?q=Ex(bb#X{cNhI$L6$@|r@HmOf;z7e!?SKkdi>oBI+s)8Ya88@ zSqZt`(O&^U*BfN4FLk?@t+VUSnS5clwoIVs#!IjMEaJI=xyjk8a+3Ke0u~Ta<^3`H zka*79IGt%u3Nu2FD}VkEqn>hTR%9745Ve9ci@l8bMUKONWI%P=l>IeZt0y737cnsw zL;+F3P+L!R*C`>=E}ND*<C>-^b40QPL0fBJt=FoeqH-*0Ym=ZwQdH@!Vmqzb;AmcN z(=ht;`d;s*ym7f$AK!BDMlYAwrDr{@uFZ=bHJPB&*y<GH#9!h-L%U~bQ_cuZD0yg5 zKWm3%%I#Z-=514k9laYLJ>!hit=DA7Tm%lJ&To$euu?O|CU!q^3V|}g;{>Y;8K%jN z_1k|d@BBf-Ss!$*#A+G2qG5CcJzktY$aNJn{Npg%$u(jX!>I_3ENx|`C7|LSo5D&B zhjy0fG$^~k#VOYID%w~kFSln{$U<cWW7695{T6oRE%b(CvzS3ayLv%`b=)KbV-%@S zdJxs+-9ie_<}pU#U{vE?8kK@!)Z@*Sgz5Rqnyl~W&zH67+TFNXF$Jd@`RoH+4C^8+ zv%Lb<Vz+1d-j4y(_)ri)gla0rEX>;0dD6f@juLY7S?t!8n5|a%hPT^dHyelZiF;<| z##tklzf^DA8&;EyIYTlB-{rnD*wC7tm`MwV{&XMmVp$4cwuOFyAlKzYrP*!GuHCRh z$wDuO<PMXSW|ENSI@6fjlk=80fvZ8V=+ceHgA<3+qd^J^L`US*w)-T}wEKV**+s0y zYJYsy{+p}pAR*ovQqO7Hax6-AXAVHI1E1ywIq!Zf>#!)VQ=e~zrJ2i2B}o^WYFiHE zyINZ0Z0=!gf5vL+WjirX>5J3Ix~Wzi>wcf!3U(#*>+O_am~D|e?Q(Z#u>_G*%@g*) zYg!+fAhf9T{cU5XhGIlJL@9V0!s4|^ib1%wV%Ks^eRym+EIQfQVT*F3zvps2T&wO} zPXzvBda&X16faX|xl-?gm)<*VyvUcL-t4TqKiU$d1?jP*Bl7(C$0HBPW8k!GQug<& z8a=}NZ0^J-iOcZhlRD1mrk|Ag)enYj3anHw^~o1*VC_a_Xuw}MiNk~&63e?h7zvo) zZ%~wjx=Jw<y-GLRnq-@0PT$s*s#i*l42xdN78_m2&)Z9bOB4qC>7uF!3V+VE*;SNb znj*5N*Ji~oF82xriv+(21*MQy*Ka*`YxD|e#$P@6s|Y^$7WmG;0R_t?&qGG^9DDbq zi3gQ|iX<%Z<0jihg4-vh&+y^lV|G~j?>N2#WRx`UE82f<#!-|xz%a40ZBdUeIy0fe z4CbzTzpJ5Gs9C3B$Ecms|B|jn+o^bs^v~sGLP2tX@VHajZj9B}K&V6<$<3~fi4n}{ z);G_F8S_V|@qMuQ&M2D4HeP|N6gdd*Uks9j@RXAe*q6zm{-qp2E)DOv=RSQVF^kYi z`oIjjK<dSH#=X|K9AR7HHEy>G%$6?psOVWcK|OjCb_B8;+Q01`K3XkJ*`p5}`P}W~ z(wf#|vL7xhA%e%=(-l3Z_Mkit0zj_#Hz!V(&3NzTE)u);Pvkzn$Z_a+I4+I0dmpP? z1unQaVqqD^rvN-`7g`%@<37utY$WYCwnB#7E6Rp80#-&$hmVBm<pu1Ag4^yIw-g^| zAAM#>vwxHRwE4JmokbzUksrr9ntiD)8~)dlwGAst;c(0Uv5G0~1UpM_7(Vh$Jx?S_ zCm{<xs%0;OYc9ZYdUHM{+mMfCFC;bHW^-Cz8ohW`_fL{Qav*df`io_vosN9l3A(*8 z8TzK9@QtCE=O^HBqC+}j>)J$$jHWEn0%V!Vp<Q1ZZL%14^YbVKka+KyrFj36s;idg z>}>5clC_hK7n?(;k{(stnnRgOb*&F%vAmme>g2oM?ca{z9x)rZbp|GBLm65k84nzJ zRNz#pwTXU4Qu78yq;bJXsk+x%7Vft^)Axe5{A#u|gWNahQ>}-TLs41DG0(FLOQlFA zI#n{jFTT1g9JLMihWh#ShksuD?eMZczb_grYOcJxL=n6dkSfr>3Dfq&z(bv+GWOF1 z=aei7?3EfeI_-_+!?+r~ywEp`$gPEFSXPW|N}+0|aX182Xc&F#h+Pb|*4u?Y33gn2 zYjcQdA9WaANbwUa{T{LqyrR2LVVw9HTO#;mhHfk6AUXn#Lw?xv+0{Or$tCdZXW#he z$&^zS(Kg9NKDM;VZO*D_&r%-va@BY7qrjW<W`XQ;yX}Ac^;_Ay1826(6rARp)4Y+m z7oQ3;r6|sw=fp=JU(81C7}FF#!z9FwW>2auOeHK8jrMrs4(cw?wFogA-LSKZ@~R&D zZHse$nb>{$9bDbPHLb4XD<*xW?VSnxLt=X|{*q=p)+-)W$qb#icNy<zA<bT?(+mK} z2^|EVzcH(_zq90@T3_b!gOwuo(yER6JN&wprP*bV<b{kBeQx$aiWs~WSKq&1wsXEa zN^mrQF~OWVPE{8|Lp?e{?P~kE_9JyvxeAh1vtOV7egOpnfSw#rd4%UnP~`(+u-M(a zwX3J_C)q`)KAQe<`@1{IX6ffSvUH9>3kYB=^LoXZJ3qA#niVSnXXBycp`}sqIxVow z-HhOxbjNAA)QG_Ix(5B{Kx#o<xDuaD<&+u|<TU+(>{3VJAgG0h@Fd`4KuL#4D$1?n zp{0o>nMYPDr5t(s=O|+^EUJxcmV%1Jyj1YMdkdvOP)l!{;TY+#DNL5)ei0UZ{MGO6 z)KT||?eS6vq1!SOwQzAr>b}XMpP?=9VaeLj&T^My+jAai?<W0}KQD;%fs!2e=(^L_ z1`#pKmVNsh&~fbrRZAbrcG5XNnN#ppkHW{Q_RFLGCW0`@C5uBu=P@Sou$!9oPdnS- zDB-w6p6S5EtUxe9uvU<>25B=WlB;}%9KFukx1Iiy>dvMkcWXb%M^Uj>dI+!QBSqZ- zVY{eX>DfY4NePKBP|~_5$(F6euJ-mLp9hnKwmsku<1QK?k;tsv3TO4i{t7By^VUkJ z58f|o$zjZWHvVobA@CuMf_&c<xa9x=l(UM~pXJv@t<G!)uYd0R;$Q{;nqY*A@<W!) za&80=Vml%VcLqBMx!-d8X6^qd{kGpQJ`CS9!T%1q8twr0{i*PB8{*x3e@9<iH}Ht4 zWO73{?w17~u355|iH_^oA-ukaF*;-h*#)x(?$eK-gsy(p$OI6!a$iv76TAhbxk11^ z)lF~XFdH90(TUjsqvKG1KZ!*`UJwRGV8lMDxp-Be;)9PyQ%2jv-+8hJ`Bc&yw*X|- zXxf3CA{IR#?Y9GQ?x!+wD_iZ3gRoFSaJ0<HE&bp}G`1KvmO}f1&sot^&<*3y;Jt;q zYbu+W=kcO3=O4B4f|(pKB724$`CCJQ4?3>~x6Bj?DnpuAuKpjdlfBCXt<Slp#&Qwo z`6&@!d6gRZ)t4b*{@&D5drxH?c=*uMuyMaQ3b?G%;7XJFTXZbzfPVgWdSpVb;A5J@ zx3zzceOR$861w`>mq0_n1NpNx%pY6=RV?Y0U`5nG_yOBt0xN0IIdFCNlrqd_vZ>;e z=hW+b{&ySQziD__Usq{#P(cNATljBV0xhj5>DeTO{Py;gwtnEBBL4kN;L$MWG>jj@ zc!KgtskdQ154TvsD;>EkR+cesQDYbDcre$|VX<H(J_W&6n`K1SPI9I$nOZqa|2s{c zk;Nc=H?F^dEcVTM-dRy#>|4SZJ!nBTsH{HnDf`Q}AU7#1lLMB(!>Pre%VO<}{o8g+ zm4#&uFD~FoMTB9}TyVBl(p1EAuok)B+gHHn*wORA3#tP=tm8o<Ei&>~+yo3ZddnY2 zb9waKKu*0cqKN&&O;8PvI@|txOYUVnze37|y7X;?s$2m2HC^#`^qnzQom@&GuUzBN zo+ar)(dYprrr=|tEMYLDdt>(swRp{1c^gW6%avmA3Dj?4_myqW^1_#v06TcAI~%Y! zXg$XfwH7tYe`}^$F_bcgAqc-^NpO2iZzs9UYMC1f*^%`xb`Du-7I_r(%^qPTryBg7 zs^%d2Oj_*>;MtMTxUupJH}<&Wih}<aYuhR42?g*B<j++f&V>IK;A)&4{Cmz+%`}Yv z^Th9~&GWore_$=)yYR<8NZQI>A`P78)M%v!<_o#l@c`9HqNuv_D7yJKlDn9S?oR`L z3VDm$as}IifQn9;WLi=KOih*V(FdX>G&_?Iaz`cv^KIAePD~bWvT3sGsBUjr*3$Jt zx8oF;Hv*p3>5;R12(sFSBWiHo*0!nMZMV8Mmz1>O${aFOEF!kD{FL>iz4YCa<Kc0B z)H#v}#*M6ns|LG%KZy!8pv&r7_$WyL2MZ0p#;Q#mBYp=<b@3IR%q)YkB)PmeaaL-R zD!&Q3GgB;>;0Z6Nxy*Z-?+0Fe*O*j%^nS5{*FN+yz0lD|fTs2IKe3-g!@ho=c*I7% zR<A-;lbX@fcUSRl-&{lb&&$L?z}x#(yBn)Hb^0m8yE!n)%BdjhY&Ie6;7(#!w-0;! zm!2rEbv=MBfZI}xBhE-eui{M%QrsI=`TtS%)p1dEUE8;c0s;a<iPGIjNe?g#-JOEc zAs|xH5&}bacQ?`v(hbtml0!EP;lOwBexCPzKYsjiego&sK6|gd*0t)|<luvbiutLC zG>9i5wK1nDX+&Ak^hHi?xclbNq3*~FeQcvyTf$qVK4Fp1Yx>SqSP{LROoEsV;0urR zN-TM+N%L)k>UbPf4C#z?0+c?^_0Z21?;^~p<3%QrlL>9LMObnQ%X{wkSz!(3-*YEh z7n&1Y@Q#i5ysk-vG6sp_5j~y=%tl*NuYWw^rBoMA#lEyfMRLrw4@D%pn>F^B|GtYA z!YTzHUHECqZjfw`;m<o!tGE(<-17}WN^iJ33V4YM*;)5}OyVLNl9{&`5ss1mt{pPU z>-8+iUZBtORDP$+eO4fT_xj@?Tg{<i{p5g`aU1G_{c5;$4WF5ONW05p)5>N`PF08j zT5amOcLid>ezaf>g(*CBnX{s{4Eu;bhGdecK$G$8)u8K*yZ)uWo9VCt^O>P=Ut}$L z;AfFRh{I9qceHmTOry+uPNp3Te$?mzO{+mWCV^O;$C=o9*h-hw*bdGtn?1<2Hxp*| zEOBoK?~lI*R8!omf}Ot}3}TztcgZy0oJ)LWou5^zSIgt`(iX`(>d)+U9ZwL>+-%=t zeUaTayAICfz?YK3{gGhby{BN_gYewy@o-vuRg*G5i14T`Wp}@CWt-I7?6KGA)lbDM zU2ZThS|l1J8cg6Sho3k(ks;o7ejaLAd&PTuX}GfKwx;`b=v{MNY20hA#rvrOx4a5f zcA;N~YE20HmF6K2d)}igjz?4sD`NNLf+BB4o~%2vU+AUU^_-}(&^wVO*vv2=^uU#W z&O3I$T;GDnrb*2eMW=BCH7L$2f+bbe)2+uI{1rHWOYL5fbz8mX^djC|@aAH!hwC7< z?UQI6oSsCL`Y}&c*R=WTv9&F@&OCF5^-dpLy*IUQi5xM%_lMZyNPCwofjj~<d`bg= z=8OP{WKi(O!umh;Uv2Y5?dHz1g&H=qc3F?W6+->3rUZggl(&$Hc65}iyGRQrFIOP| zP-cYv?Z&%9yBd&*-85rbbpKNTNeZZT%&6i1PMU21AGk0|`fW~frsQKZCt^AKmB`03 zsYoXMIk<guPv6>(wlDu}Dg9<ui%%*~(V>8;?jPi>Q0tM+E!?*#uPB|TeYq+)IO93s z(#^{+?^9*68+}O%Z(&Cl^f)1fi|&T8F}>OJPUpOt3?tUVH*cl4vDj%pmpU4N^<=5< z>dbD$u9Z^d7vM0cC=Bu?5Awv#j`=p37PGs@Ub+uuTs7^v4JI}ZWw)-DzWR;1-g4Hp z+RigiUu)@E*nISOPo<@Y1cT&N(9iiy9y*Le!n5YFBZJS|UL&g=dt^y#dK)UPw??L^ z{4`u>@n=*8RXICCuWmZCOISlLm#lN%aHmEf!du-*Dv<*>Cx5)olA!y&;_TX^2VMra z!sggEjD6BNO8bY+^%cRvL_kgOkGFelV^k{cT|)&j^n*EJp%#LhkMf9V&9f_^j-ry# zilWk*4vtzos-cY0gMp&W+}L#qky=5F<q&SBXhwGPaUu35<C@GRBlivILlgPOk8sL3 zecdkjq9QGD7pOJ(rK^8|_Y?=ppl=AGTFL)O9_KWJO4*3|4VD;kZr)#fq({@%LBVD* zD@b;rfnBb+&#v=8xXm~#DTYQepID*X=%}*68w;i$Q^MzCa%xeq8;DNRd*fgADF!|% zxjkEzr+9b0vEUu(?^h7>BI@xE0qlaO=mearVsx@w8e1G@8pU$K*)25Mb>%^rX$5VZ z>FyIC6oq+-gfGa^)?@h<*W2-RoA59W?o|9IfiqMcoC7%a0t3Nb&BAq=6*%E5Us!7| zyckeg*!^wgcL_OoIU|CKHJ8c!OZ%^urDtXC9Q1RQs620!Jw+H<YJf}%P+eR_T)z-v z9>tJmA@g<E$|#Hy1(rMzc$mMAf}l~X$%O9rqsUN5k4yB!Xm;$(Z*o$X@uT?2T_fdT z6xn!j*!W`c{W-Rk!jNo?Jc78wxtBN|z(mf{!w#Z~Ink4KYU&8YE~POjt%eAx`_E52 zWiI^V?%)vb^A9Bbk#z)FRKF3t^I0xVhUc%x)ra^gYpkaOQD&;nF_f=P8$|t`&sGv{ zsI^6bUccH>t+=?1b2;~3tB)CELpQ&EX)!584VVtz$vq9SUcZ{ddBdK63aOJgk9gT? zHV#|MYNv*0+T{{bD6IYZUCW|Ey-R5=pfOY0#v8tnydkUY)gQMN*FerOiBuRJ4!9^1 zcIyDmz=nMgF_JM}ak(M-(y>JRnAsjc9i`t(YM=8HE_-2=aT;CUr;tx==60!P$1!#( z2=Ki_Bs*4SKM7*fmg)$g9+7-KpPNVXi_T%kv-}~Y`p{vjWKiHn209>INb#k5u1Q8E zCV%;AmtcV!f8(oYZO1$$vu+b7ANCR#eQ&sN5GQx1>PDLXk~6vAq!5cnipDO3b~UeG z?jEQXgr&XJ{w{sKsykCo^;=a^#LAGidrLP~u>74S69!JDF55A+&wE_A#uu|iFP0cV z1Pt#zg7n>4bZ2)_I~sfQ!}@<^^nN@l6iFM#RqgEtIjw~9IAdv#&aU@$b6Ah{`u9hg zu)`iRhrUee2~v-u%;JXREr?t7b3<g>Z`|d_Vb|^S;~k0l7)j#b4f4|dst!_SJlHik zp*0m(_dbuDaxAV`Tdj<)nA_YY)V6;HEJ<0XbdO$FOC2of?HE~8cjFNmWPi{<fFq0- zOD#Fh#~tMu?fMaea+ffi!3u300}Fq~lJQ5PEO+!CB%ei3)8<IH1gOP8fxWOvfNc@e z3<_`}NBw^hjf=8KcLx{Z{1bw~PdHIi9ylfYnnl*gb<)P9h1)ER=LqxFUPiSmQ#mH! zIH3&O;2}nXC4S|%OP7?oR9#Imb0GWWgf$u0>=(a4b&7H@2Pg&MyF*n^)*zo}$m1xb zsl0Q0qr>(8F?QwesD0h+{HHLMgW6yg{V>6PNBmxE(&>4iMDbyiyE%VXLnw^)aM{hS zf&n?OF&52^h@2+KJ{$?XSfU7}bH1GCPHr$TA%IXGO1?+=l|Pl=Iina=X@S*gi!#EA zUk0HwVPffoZSJ++Q`rtWWZNc1>-j;om#MWXw;<wWHdOtsL${FgQ880b!qpJdLo)BA zQ9OwGquYV+ZW3?N51_fr);Zm>%`Zf7Dq4|Z7ASJL@s-U6rd}*9gaZ@qV?YxyAb|^} z^sa<3!V`Dgd~AeyA9mG;>7HamDh3*z(UbOBcRSgzNSha0{)ZEbI_@MpCVki;_PHX0 zA*p+<mQFHZfxlvR7OXFDUt_4c8zdlh8Kt5(g^uU(SNPEc>YgWYdyXQ0{5szwshF<x z#}z~MY?K!^HC@>H91+YlU_BeNFqTz<M$q^i<!8QIt4)LR?18pP@4REa&lo$(YRcNK zEd%q1;!<fsYPkxu4x~9p72mgIMu$|bwAx}YbJdqMT+kYejvLfHD3J+CIn(!{GDSej zIxml0g){$5;@GYz6Rl_ZK(Pk*ptdSd_$~Z3-7(qoHAj|+YSM$?WwEP9d8@|pAjyem zQn8mqo}X<@mu6qt{a36m$ymM+!oXNTJqe^L2kP->)>%-2xb32hSKo$lH_v2iy$H^~ zRC)iN_L6B`oW3<Uky|aI(IxMGsIo6uFQREubKGO_E(s!!CbtZGU>mqQsg5qZ5bV6I zEHC3rN3o6|QCWIXkk*-OOapf9RbTx4t;RT)jT7;y<x@EW-bv>W^Alw|(ml6SuGR;t zls}cn@}OEehB6X`oclVI)sd~dt#4Zd^fy~raIt6MF<q1VbDXZ&<}t=XL!;EW@Jco# z#|;_TZmuw8Ibyq72h(b}4_sR%;}86YI$^DY!A=Y~#wC^NYhY?%I+US%+wVZSl`cjV zQA;VVQ!Zl$lsyRr#Re_7Z{os$Lax*fcSn}~I3k@p>75C*S-Dr0%eO#7;7Q*PE}7W+ zl(#CqU!qATDArtZrGByrz!=CZPdH<q@CTj;mB(!1jgQ+;*pQ_RP0-KF4aC$Qw0LJ6 zVUN&0b_$IY0>os@MgY}k&`N1Hhm}S_?iDOm3-FB{i_Z%s%wfubH+rkJtgbv5k%bIo zGBLFKPT~DT1IBSuY`v32RJs8y0c0F@cAwZy#JlHnId?CX>wyIjd6q7doxjn4vg#Ht zo1KDB&~26b4S%(3*O4q6zt&H^wO|uJE0?5FV6;g+0?69Ag&mtz5Fs~SaypSsXi947 zu#C8<5;G^hPXp>JJT0U)@cHBJ&y)HS;LQ?O!q?)AijphF2NW1FP&C*CH2U<5()qhf zN~&`@SuMpF28V|4;x_01JCO6B?iC1;LJ4|zRO-tw8{5J*ISvkVSDLY~T}I8^`+TG{ zev(^QC@#nNRn#1dm$_?|MXk>poebv}8cMHcv~*H;tIWO`yu#pgN`EVV(DOS%+b#a! zlXv<k^^fJf96V?wr%@lz@Nf?|scUk+`v@mbTmp?aopVyHC2btK4vpmB{<pOVpNDo* zDhQPTJqx`p`(C8ZRC;w!f<wq}eaCPfIl^40DbR{-m|BvP>P-Gy{`A};jMYEy3ET9T zNwIv8<zHB@(Z*Y9HLMH6sdqh#JZ*HES&=tN3zm!!Z=?-zZ6D<OlDlXAIdooJJjM51 zkuVqITMhY(DE_sL_f*GbSzE~7K=CQoklUC}R-hy0?j_J3_&_ZJl*s;vi{gKZFdC(& zZnAllmoE?3X;W0@hDToIUdXvAR=6)bG}c_TQc62E4`2RB>f(nkcZ-i&iVt=(k5L<N zC!xjDz}{Ay*lI<Ppg(drK2AT|jv*blOHUn}+l^uyDY$GA;foW#0Ai`ZQr_BC;YCTu z`Ll_4fWRa?dQgRRsKkjlnT0f`)B^HuiXfU(stI~dx@@=89PjiYoo!NT{{;GKlfr~I zNiWHW<=W)GS73esjg=_jG37LWJe-IAihdKrL7Mnc!auS<=hc`2ut7M1CURh~OE}Kg z^LHeNI@!@C8Rl8fvd9K!3K-~cB|uo!(x&h4H!KvDL*F11uG6KLXI=pblUp=OS>h$y zPcv+fTWi^+7<am=9+kwV26<KMK4h5KpZG#iQ8U{3TX~c=U{f^MW}cwI>Q}hrA>0YI zM($x%MXy2#7=tOO+@#|3H`jKK3f_L~^3k?&$x`U6>0@#t@d5qXTi_eK#PdHAc{^tL z&{?Mh5PBSNE<Axp^;0Z3_18eT?G7f~=;2Yd%A3(qp%>m6KFjW=`RmZj8C9K_J3X@R zYK77az8<1YX!wsD6fI+d(lZTc)I-f3R>)q*YREmbnVbIYOR^230_phwa;v|%Dtgb1 z+dcf~Le>1sLzDe>hnh@{S>40s7lCxyY{5C>qKAr{i2o9l0QY;}&y+c@>d-K`<B18a zw%n&Gr^<w^^Ot~7L1Th_k)?8josW3FGov6XUvDcrOO`2a3hx98@?+dRS_sD6I0%6q z)e;B>g@^NSRi8N}WdTpm1GoU3jcx&Kdn7am;s>1w9TaYiK+ir0M~jXJNApBUMhW1K z_&K<4O9+;YoPRS@_l1|pIp*uUpv{Ki+_zq&qPW@S5-{!p#Ee>%Kqjz9&Ig#g?n}S{ znV><BM&Fj;#O=Aib|m~A1EmD#$~Mu!IK{w^k=xRAS^~<QNKhqoIhU;wkLRlZ=gdN~ z{}s6O*U13W!#sSk1fVxJcTL_u=@QxbODwV*bVBMD1vX&q{PV20^|zPm%a>0+czvO< zTR$0;U7qnXb#pMRgI3UduI;I!84A+CkG&+38kTmk3ZhN!xqK+JFZ@CbaQo~y*5UuC z2GaiO4EePRDAMwVb&o3pET2c1?8u|TTg@tUpHVDIm7@K+DLj))p_PS68$v*YMHYn} ziiRIoDgi>n2l|G;Z@TRFbd2h6KbeSt7dK3k((7+$CXLEQ<2#*)OLFw~*C3nP+vK|} z#mptkr_ZErQxv_McdR$7I->6SW~$EwpR6QcL3Qj>$_gh%e!<S%Zc88b_MEhO6=;F- z?6(r;#s)2kY^rn}&t+4+d;tApuLU}HC4VK43jH+73e;yQ0AB?wlCpW}w%~6kg%LuU ztI@nhhfip7m{o~8XHtBmbJwxjeZ_<eEI3hvR~bGQ*1flS(qVQqFJGPzj)jTf6*qpe z<4r#pKVzOUmj|PmUj{$)%6Yv7ZR*f}pt6fzHLz-)#oj7|e%aZ9tOze!va!}t`{A)N zwidPIpdffPEflJGGx}DXY1}2-i53u-VON(VvQXFe#Yw5;3Ml*}-Hk_A9$YCw=l!*t zTR$6HN7i~ja=jH^EG1c3l0P60afr~@v%JhmwTq7!<&k^suycQC2Cn6*bmtyTk=xLF z;D1<?0Vd^qTfgM`c24nZq)3rQpRVQ}-l&MSgS*{6vDcou)2&H?cGjUAF56TOtawn} z*O~LMCyv<zx@oDlgz=hne6@q;X%be+;6#>zYd<xgqHmQH<F#^fBR-Sv$*lvBrG_Sw z`vOFGkNrll38`W0_g^gZr_k;$JRq4`q6i1(B>gK8s?bUImffHikfFQfjUl)(B)ahv zm7w)Z7z7K(f4lA!I_QBaSL|=KId>1^9f#D{zYD6;+;wqAbaElIcga5nYBYJ7@++Ls z`$2ijP2?OIbyB$G)<@d#&PMLk+}C(O@8)U7yND8!JqMV0ddQPJ+<NZ;#T*QLamaDH z1QP~02y#ls5iAKh+~MPBd!us8r}GS_)<h0;AU*Z5*lw`GkjBjd`AbbEmR6wEMX2<r zk-})1JU3U!`iQjKND((T7-uiLYeyto!_3O?E@A`|l^9V6R3WL1hXLP_fufU`iD<AW zI2hzhdZB3l#wPvN_@c<O@i2aUjOj1y+H)@z5Ryamd+S8b!@v@rvJ3x9jy3#cM~3^x zs@^-Q?V$<P03Wa=WvI4Nv14-Csp6cA3muczd&$tVP0#C!R_Er)bK9jwt6JlXYvTfT zZg;|j`WnLrZUt+g;rRV~fbHfZaHn{7cGx~MJ9*(^_QZlrDNE1Njl5zEeGym1?a!Z| zsvBBFqFG()eY6qz>%0c;&HVOTYw_TRZ}+l6fwgU4QUFgdk>}2=`y-l~fS2UuC1hUc zZ6|DAM{l1^Nh(j%^t}NzF5Qa6#EO7|w*FdRc4{w8P+aTXpJh!44;Q*|fW0#Lp}84Z zOA_ZVEkT9#;sp=T7HagQiTpexAHn5tHj=mJlFZDzf(AJ_DEM1Q360$T{X2%*p^kbG zPN^~g)lP;8pzom{v<r>T4XbAKT!s($Mvx=`bfuU3?{36iA1Kn#K$X(kU+rb*7~|>P z{59}AxHL_q#1Hc)o=tHZZ4=9%@DURDhCjhVgg-IXp3Smia>vfCo)NZ=M2b~C8PGLN zy1WEt5ST(L=^DiQ4rl0f3(d{dpLZ+nYY&y4z^n`b-LO<c2OCOC%nyrsmGc`w>woWY z6mn9i5si19i<2K}JqkBJQY52th+&N+%^=&g)-0$3Pqm#6<}BmQfBAEVUQzG9BG~(| z{DuHQ)PgJA=M26jD|qTzE^p!^G}eT99u!xQ*5}EC6ycb)vOTbtx@odb;fLHiR!w>U zP1t>@eT}(kF=zb7t(XdzsR`2tmu@ujsqDnn4nz~~bH@&5xBDj|)jD=2j-@*K{6FOt z#%zgQ9#kqjU}08QS6QDvU4POH!MUG?DLPm%!T*V~O#ft%uC(C&=R%%7=(2O!`3ydJ z&Bqz%c)f?mx!~ga<=#18q$YdU1L)MO{f5SICo+1;oAHs(%*#$V<7_-?@zC24Aaa!) zKC;{B9;qWCCMUPMB-O`YwLEJ0s}9joV7Byi?C*)G1Sc<InjJQ9JF5?*BJ+mNP(M3H z?_Z!DcS4qVVHdD-?HI>wa&m=@#7ud;I#l}?(PoXyN=G&XkirV`+~8cfXC6p-tWAaE zfPesBQK`N<Tm8R0fpWi-CSBhU(1(h|=Kst;VNpvNNb)65JCN(1u`2%<9};I?1{9wk zUq=Z&O{iU#bQ+x`d>l7!K!X=8S*#U*x-2W5@bp%!N}y+sTGjrl>bG@?bY3KUu%qj* z@w=MF0T@GyKoeEdm0@e8QMxIquSYBBQO`!x^N}d$)G2Fj;k2D@w=}Lt`g>`n%qDPx z4o^lksku9tW|hE#SoyAQpoU~ig@HW_E?keOHth3J;(zDmlI#zUSZu_1h1n85fFESd zv=4wY&vzgGoE}~f4G{D=KVaNHxF=IQ<Nv~3C8@~m5VJb=-{-3z;Fbv@u_(br=?Upi z1~`EFPy~O#ml@cGON5C}Yps%EPL-7KY8&Upm-RUN_m&yvUYTjT@+1B~yad-C5YrVc zuu;+!&vj^mkGG-Z<)&Eh+C1m8i0WLKwBz!2hlV8&`xd;;FB`IN?svhUd`&bsFGvM6 zNv#WC(vJ78xzNUw&Ga^{T>7)8PhYiqL~H}>Z^D#X<AUV2g3ywO4V};l6bj}AZTOo< z^UvL^vzrTKQA?CBi%6q{@S*RbU*hfpWX=40o&uLMl&blqPt$-tOBy%3TGE5Gg{jcz zq<Pd%@<v;Y*RB8J5Jntez`yQvXsiYmhX$x<6{yO+&3OXt4>N{+-4?!5sA(3=c3)*D zXeaaHjT`6=%RMSUY<O);7|KB1?CrwK>QTOWAjwIk)$Rda>8uOm|9vW{3OEGd^0avZ z;XXoY-<NC8LW6UVo+GMM-8nFmGFO0OrYg=vA>gZ*ymNtOX$WWm@6_7wZA~*K&haUf zYB!#NHhvpY`w)*ec#Z=5*ZcpwcH<{Gz?z=?U2+;gwUWs_G|RSYJrs{GS@LwPKN&gx zB|0fGW%*3%CIz-cZC|>yVk=n?m|U5(njd3uRAsp=DmZG1s88lFo$NYkYF2=UD`N48 z=R8p;L7wVc1Dkuu4=_RefZZ`e#f`Y_{$1YhrIZi-q7-@Wsbk0lRQFf<wEZ&Bd2Df1 zpgPwr3YH2eI3SS|hp`~aN4POcCP${yD}S6uHJXiO?U)(a=FYhowdu?!+T;`VCf{`^ z0=>2XyQU12o_3n08DXEF{E&SGB4d7Xm^{(yS-@?*zVU;G8I#PWilMQMUrL{ep;KQo z{Y;zfhMs`FzauV_wJl@EQZN2q3GX>C!W5>)-^RCW<$R>`5op$0q_Q{C04(iWE;+oB zFH|E@u#R)L79Fs_>~Og}_g+GRFOF9Dfh`?eDkX0H`hF^X-0WaNwYmZPdJ|Rg$nr({ zyC4K!vy@+A6!Y3a>tEdpOa`sp-JxT_HBA)5iBQ-!FFjt?p|zBxNA@3fJXo+5q!{JK z|7w|f35n-=PlvLnuOe@vX1tCkmfIR{VOYYRzn*{LY9;cEg@zfwl4^*0Rx`N^knAVs zgq^zY<Z_!*(Fnw-e#PZ9`w5Xs<x5X-uhba2FQx205C`G^`A9PeswD@Vw$!D57^9(b z(<~B>Kcw&U=vkJ>02p&Vsyn;Ge@0vN<vX+Ju4diD_1yi~?Ii>qJX_;>Dw2pcIYpwK zn16;grS7$P<)NL&qB7|Uzcoaga^EXzwRzxXa9lU19<Lge`NBRYJy#oZwR{UiZ-`+( zRgQ+7sb=H=GSJy@i$b}O0}3^y;>a-1(5+UzJiNJsk~=AXtr<bj%pFD3_w&3{J3StI z(~>`%R8j5DCrXh7Rqmho$9!4ZRjdAPklP0_eJb)AcBlSZe(im+&f$Y>OSYW<%Y($3 z^B*?0<B1r+#S-Ou%Q1)f0?Ur^uX(KfKPI?r<egxd`MjbIAw9%BJR4Uv&nK05<sJYs zp`l^CUDP?k$a?WnM}VHl%=CWu|DX^H@HhYhq9mi4V%`k!TQy_f{2U75Hx|ZiEtA+F zBYi~%V-?vViud{%;JHZzP|X8K!a^~%u_iC#uYrC2H$I-uWcL>cbiS1!rBoh>vlNk_ zr5uk4uOXLW!kt3^&F@^z2yTG1S#O?L+O%Y^4+Jw<RQ*d<1pumZiF#C5GEmq7<kECi zs07c|%Z>Xe$Da?kG!(%y?bDI<OAr=$hl;FUu;2d^mooCrx+0)wenIJBKUBanYy?CW z=V}{q-!139ND7YQ?V<nfGHmA5eKi9~z%G63J~7qW9Hv+x5}wxhrHtS0RG$mXUz?Zl zcldY@d~O(jfzY@u&fpmRnm)-2SiSl|t=0#TMr}{7%|0lVJG2LAGH?*|x%n^gn>US6 z95jO{e~&xkJfkLSN$@KeV^<)9rkI9wC!j3xfJxS6Uhlu}SF^MX*26W5u_5E9lFg-O zM0+IcyDl0XCgV)!K4NGv>?#J^jf-<b(+r?<B1N=+gvpCr-Mdyq<+Wr<6W|VQcQ;oE zn^?J<!iiCjkiUN`^JTjAJz(M!IUsH!n8rW>78u(bI-ro9$OB08ZW{&STKCU05O`na zWfGQ!rozu6S2W;<17vOk@vR9;u^T&UMv7tdPs!$D${;vzRW%d=N;*pO4w003?WBIE zyeYvq(^dgey~#bBPe!Kj&h!4NQWPLq$2r_?|HeEg;oTj;{Ez)sU?eQg`Rrs|@l29R zM9-_Gb^Zm03JU0PteiT?4DJ#8i9QX{15-S^09Z5N0B7Z@+vxj<Pf-QABX`t>2cBPY zms%1=#R{c$?^L>xfYTJZIn(AWopZ@{<ee@3z~r@-@WUa7A-bhn1dm3wxoz=zw0L+B zP6yR^%$ci|ZLpm;?JZN$o=4KYh`@Zfjd&yj2YAQ{tS}8b{O0E?utHlkg>BTDcC(Z? zf`Y8)N~183FUqBHICG~KllfoR^aAr^J66Dz^G4uLoK!^6Sge;#K6^i=OtKNS%c)`- zYQgDEeo@1*;(ZA1ju712jRmjJ@=n@ZOMa<>t=7-q?r`Ws)=l+guu!f2dnUco?DTya z0Wilj3HUP=z;^XN3k5)4UbDcn0Q>$@yR(;YC4d>Y{qWn!5?_W}gactxK0>79^?m7E z(j(hy*|G;PtN^gM;eMbT%K(W;rTkq2ZXotT%UOhYpdyrz=v9+(Z2ms>AjZL=Kwu;2 z)K$6$%8a|IS3(m>AIxsLoWP^lCzs?*`|i-@FP&4pKgD=fRu=2iV-^tU&I*CA`?7<9 ztDets8{Sw99jUA-i2ZDP49k4{Zr<^-K}m50`u4U!LheD|w(JA+`HJ*6ETBzO`cV4J zv2p1GoURbCTf<B^=H+s`1K&_8Sa=2&M2*tyKDKOz{8jWTsJwpZVQ?$YeW8Zj)t((w z%7)Mv$P0Pkv;ZqX=;e`}W3sgz^dfC*X#IiE^gZlijP4ZLz@izyz5VM87jHYx$x-XU zU;C~Jz{BjM0}QlXATycs%k64tNAVA@6Ma8X92C#Wid|oAzh(FmaMJ)*8+LA_67J1| z3V#^&fv`64x_K8H>d8UF@>Fp*RD=DxjGl}IppqCr*yO;;>Ye`V+kv-Pdp7^>dVkU= z3tk+iY-l1-5CHrDAi@IVsXOyPHx(@M+zTIo7m`kbk1!IP^jH^x&k}F}CnIVV@MoxK z=3i5<Alq8%%9*Ym=2W{+ROw`3Gz<>ZD|)R;R_iiHWkb#h2lE=NaNxcSLa}W2-8649 zkVMdVW;EAi%>1FaK#OVTEi?#~u9|v~-<peGIGqL2AB!p5$SP(A@)`UfTVp2|HNB%x zhg!txL<_LaOptx4DCazInJ<%0;cfq3W19LfXhAUmzy=>=1VSa)WoG^WyiNeasKgg< z_9$BEU-D{9T00M?cr2)2?%p|nWIVNKC6!5{b(3tWBcn_a;7l1)D=?|QKMFxcS!PIH z0CBuWHC}e*H7WP{+t}CB+wTi>mR$<@T@O#=?H`tBAIMR$Gt~ial7|7fG(YjfESS45 zQLfQ6Xks{LOy~5_cghrH>K3iY=+-0}j_hhuDt+Doly2M9znWqA-{Pn~pj<~(BgG15 z<Mk+aGe{N*22!G!I;YzdPCwFe@(Wl0=cPKv-lp-U(UnTYukGkWLBwqEj56p}GlDde ziukheUolMzf5w-**}c=$u9;`ZwWc4BGf7)k)GSQ&%njCOC9n|B#%qRA>Bg60UaNH- zn0>I!o8$8`8{&J=@Xi1o^g*ud0+2R}WGDznZj>p3<{ZdaLpPx`2Gwtdyy5eklL7@! zO82<5C7>3~S+}CIkRZe68Ktr?fqa&m<+!1;w9m4{N5BayTpawZ|FaGN<&*EfoAW`V zMSRWp2Cc9D_!p_#0VO3hvB><Onj?5n+YSHxBH8#X`>zMi^IoMkx*N{u!ph^(1zlG+ zGq%g(jHTA&FQJ2GTh4YxTzq^}p<(S;>6e~6H&T*)$wfPJtfnG_X?H$9o4`Bi3zR?H zj{3K+;(^+d;0z+MY$Ot?_x!gG^tz^s90+(wq^kUg>gK2A0-MK7x!=-reYqcUuUIH+ z>sJu~HlC?a|K`*+^8<hLTx#>@>c-D!whnuTe%zzZT#-Sak-v7f)-Dzl?ti?W%2;Vm zy*si5Nb999%AZj>#}ttiuES~w<1e3bP!APBN4x*vsr7#><#*nNOp(4M2}PL#4)n0K z|L7!uT$z<K?N1H<@v<eJP5E8(KP(4=zx3PBG*e%TpzU1gcWz%nxrCdWful7fs{zt$ z;JlmI-d2%>Foo_01WS20f4LK7>)S2zvvuHfib{ZnT_jKO+*Mlrg^J<s5)7x8&g;sQ zaP=H8P!=MaRJ6u7dr69C%Wi_XD))Q~mUqr?F%D^gw!KEhpp$~Y1u<@VJRRqHbC&+Y z5#YEOJ0Q0<>NCW<(r=Pj+AKGFL}TPh9dtHdV)qs}Dq>1s;a-b3oEP8?UIdzBEBS#F zVUpxc_}g6==O@RI5%;n4$ePB_d$woy^)Bg18)~_xdr>-cCRLEcW-9ROP*wJRu8Ot5 zo<u3VnhTNIZ0@qMh>Z8#Go&zip3M+O>(0<<ASB|ujTZSudFdT;YJVc%{nH=K-6a*( zZ-oV0v$5ZuUxY86Zk4C#%;P+kZ+J$#<3_7FpJKSAVxuwNK#$aqy>Ua8T)K;ybrWJn z8t42tX*N8MxwAVGiE3;8neZgHkqSl5H3+diEyapj>6@b^(hznVp&xUucgt{q6{?ti z{SNb4mto%uBpZh)_YwblVbg+DrC@LR3@K!Ku;9A(JtWAfLU0Ql?ta)+SO6V5%q8zd z9#fCN3Uw{ComVfvW1{xu$0<`Bn^96qS2g6S8w-u2N~1B`xY;jVV1+TFi=|h1-xMq? z3ab*_wRLdNW}eK^@^bRSHC@OW(op*j4_CQprNJ%{22F{W6$^Qa^IwlO>#_A6^a4S| znw0)n#FzO}i#JQ+ODI1D`HB(#VAcEcwPfv~z+`U)u%)+5Py@t3fX4~TR$*|w0;{sH z@&~F^L5)!SetZi`kYO!iLvZBD+^Lbp(XPYm=7@7}t@CASqtY|K)e3Q$^NMH1%}lVZ zCxY=o)7|tmFb845v`@)~Hq?Kl<;zr^=~wl!S6@s^zH?PkXB8~%JARC@HfEF7YP1x# z?Gn(5xsuZ3+hyg?qUG{JUuz{3gbM_?tGAonP|t<BK9`PVf~5<kc6~ol>Y-U#8f&3P zQu2zMN)ztI2;^^R)!I#BSQp%U<7uME9d>+ty(8Ey=Y{4jxf}R~(xUWzTfNf-6NlX) zX(KM>+V!*zX%m+M2S;`8#=sEla(k#DH6`&~RFjl`7!C)a!&Ulg%z&}LXbUz@6e3*H zwwf{8ap+>k`Uc)A=ttXadFVD;IKqb52bW~0Qay<%Op;olB50GaJm&5Vd+^g?7k0AQ z4{uG-=S0gR<>lP#qY2_kF_dA@Jg-wyPK@2I33=&vjQprR97^b}d4YXSd<vx6)sdVt ztR}GO*Xb2`>6xB4l7i=cO|r5BE13?dHR3e?+=yl{$1u(^ZVVr%G2-rKwvMLPD8@ua zTwc+z(<rnjW|)exdUyk}2N##&_Tc;>0Zn2*aFCgL|K_DFs7(Nt?y~%FN!11{j~s9w zSU^fpg<^JT09>{i^O#cNEp}A&CtWbg;B}!OGOmgNqViUGkr(*4H?dI4x=)WAf*8*7 zHmL@OA`2^iZj^AIpxc8A$nxk2BO@n`HPZXDBdK(=Hth{LWY_1+)z634+fMQNwqwD& z2z4)PVKw}m{*Y=fXZMY7AB+5y@Cz)Mm8a=NXh!L0`<aA8#8$1}yu@625epp}8XBYd z8WO~5sLPh!f(ffSePMq=L(Nnp#DWs=S@=+)C|c0=d+#i$jOEl*mXhzQ^G*uxm?!Uk z(IvxtuAP*yQy8TByq#!`X#z8;YW`jutTk`Mlu%F6Jy_0wk~$DUEs|l8LiIx7wn2rG z1o>IIUwhYBj-f679n#OdyyN{F4?$&6>*z6s7OhI1JH+?OdyW2iR+WRLm^3FV%5xax zlXiaLtoX;ToJZfd?R&k2*8(Yhl#8B#Jo&d91C4sta+S2B8uJqu2CkB;1g*aJVzWbp zwM+e9cI*NPcv?1mnvgD$iyq-RFFFSS1R!K~sIu9HLvweWG=n5G@v9|SiP2mB+F^+1 zKdc$rAwEXJM#I{%g}=pvF()HXjhd==RJ`TO<R^MLixD_%C|E^CwUFULxN#2%^?8*E z98XV%a!P|Sf_?XkUqxbs5)+Cd*)BU{aLJe{Tf4en$MJ!v3_d~T|6NXoA4OiGXd#~l zwEb|%Or-W*`!P#mBPnngq5bxvpD~+ZS1>iOCavXwk;o@9D@1BwC}RR)m5CRg?>B$T z5P>8NS|yxH$F?MG6)&WAPHaeDNUW%yVAI(7%D<C|m4W#QS7l3XNFp|YR>iO5eFNhg z5Dy?&%s-^hU*YPzU$p?AwO-Ll$8Ki~R}l6b7J4&!<z4x#`Mnw)YGDr1>#(=qy5baN z#^=B($KGU^O2q+-?Y_uRJqTG$iH=r{gAcLE5&Ns)SiP8i{q;8Z6SqKN!C2qaW^g3U zDkq?qh9ENBM8v-T6)Gke#5a=sE1J#~dN`#_Kla+XyJ)Px%CoE3HHTnoVQ_APOy%jQ z5nhPg*HgZ8>h^@#su6-VmwzlH*)cvvekk9QG(~@>3XMk}3-z7cSS*&^8Y%l&)J=9x zp=5G+6G&eppPkK_^IN^hXrxe1ES5#zvsffspfFd3T^7N2t@fhUtuu`R9ykx)RBomZ zyXiu5zr>uypdGKE9rPdueJlL_JF-^%#bNF~skbSqp;GFqj>1t<c%caWp$utsudEOa z3<K{_sX;QTLhF=rKXiqG`NT!gI60shPt`4j6kJqHQ8R@gpR{LHYAxKZsJXd-81&DW zBAEX{3HZsGoXNLJ{S(u**yEi`hTh!r?4IKE)OF|BlN_d`z)N`dVsP~SkL_4M7iwlh zZ}A?P9X>%kNdiOD>ao+CMwe@t=EurDmXMGoXN`&cDes)Dk@BpuIn;XAl>jJ9O8x~r zwr=_u+S!SjB#q1Po3J~f)kg9@m8OVBQ%YX_@MFAhD3nue>Qk+xCsU6yxEE7@#i3BH z2?k@}(S%M4n;xSJ<8b!eb5W?)#dA*e4cyIrEr{RyY%O0Vf)Yhs99R*8L&`A5<#Du6 z{HZy@pji1cr$){PehQUSBNeKtPzG*Ov@@T#3|VrLbnvC+ab^Pja@lL6aUkiOhbAuG zrrm@TH+~gu^1((*<4C1C@V(p+dV|(ce7&;<%Paq*3P27G@JV2tC|HlXuFpKg<^-)& zp6Llv3jE=v9Ej6Io33Y}|8|`I{A85^+ukS+l8QrRf|2$4#l2F%-BnywZOl749QK{t z&!H=f;mBR0<kRU(;z8;kB<c0g{g)W{5^inO{z5_=1zmP+)<F@?*p7jC&{VP{^{4Jr zmqfgWmWy=#j!=M2g^|Me^fNhyBpQLKduL4O^EooGJ3M~osKm`nG}XiCsffo3Qf5E@ z7zu2nzmh|6aK=gFM)q=a-G9uME?^^)Trle8vUnn9{hTSjc1$GYSqk3(R{?rdbigF| z_7s;TOVOF*sV0Lai$fL;9m*mVR8V_wb}22CN#+A<0MR!Jg@APyrW4XCi8Qb>!0W^e zqWXBSJ%F=V2eMOglOBkBDFY>@ZEY>lg7f>`sQl=AOb20vj{GEJj`_!ZO3WQ4J;#pW z7apOBB+EON<zvtPNci!<KYxv>KGc-}#A@<AvYYUyKBDXWn?>7~uvdWTB2a!q^C}i2 z<H7DgB*mh(T`dh?@l{j@Nth^X5y4`Kb5~u6`35tXsAv22O|Lw_$^lJsrdJtz%fx^^ zL!})*3)-W~9!`ia8TA0nAFF7l8TQn*@Va)yq;OvYZJcYHn%8lfG3)EIsn%`zv<bSl zIDY~<5{|Zbl9*m5Aq*(8Yu!zOuWp@s&`RhorL<W5$3C}!6Dg!T6?@{Jv;0VVDJ5N^ zMD5gU#fchQlKrO}S)?}--i&?Wxu4pxo{4`Dm8oGsZTye@;vANWy`Fgd+qv78ie%Nk zg}xiL<f3$n`duJ7gIlHtJ*gJmV(glM`g&#cDXA@<im3<N5h)lU<LD^66eVp=gzL{M z#IZgIA}DuxukNX>a}fZPbr#!w)D-o=q1ZrUtd=cjlOkQ({V4K=vxs@Wfnv22`bscl zenar!94ow7_O5$4I`YcDNuu&pvGM&hppWs%Vxbrx`c&I${%Lqk*%Zn=xn5J?JJfgT zni}4C5T1Bz!*qDZL8@h_s=l{^>Aov%s7I!jF_M0_s&?@qZLQ?`_f^?~uv;Dt#`vl) zRG=HJXpJG_7Iz~!=FiL>9_>_@`4JCRf$OeJgnUC?xxvz1W9$7?*ce+;7jo+`>UEl~ zG7)V*2`{a5jY`mGQ@AMl+3~fzQ|gMaTTyfLTNm=iKz8tUdG5E`X7j^u9q$=17a2f3 z#U?{^cd7e)o#S`2GAq|)B1-)+IF#LDorst;g&=k<{9CcjUuMMRF=wqtPcKaajxkrX z>Mo}S(z3V-sWwpgN7-z)52^mV2{ie2zZ)p>O&j}Nt#-9`NaQEkQe~1;a*}qn5IofY zi#fJImzy_?G|zXUy`A7bKpxxsRCluIH7M{)M(eGD3)P3PfLoh4FP}e=&Hnb?n9)%8 zACj#(UzndzbHt~rK>4{tm@-;(2pTEmYjdTWNLX2xsYDxJe~tFHRe{R4luhxW5aBNO zQiaQWt`<WAIozR0&y(RWBOcD~z8jN36|I%8AZzEeeQv6;-&dgmi%O$q4H}fHrMf}H zCsEh6q_mn#gNz$HM;Ye3?=?vgQgraC$2~>qt>>#=0F3DT5vHsC+^;N?<xS<;wtVu} zh7*dy`-&;gwBsY0L!d2d(x`C~^RI;d7aXz@eRvmpsr45j-r4F4!k+D{I&@s0v2>n| zz6So4ay=w+TjD($+12L&;8wp?y486&RAlo6PF)SBqa4x3r*&P7`}4hbUXhLzMe&?r zde6gLnN5voQ)b<xZRZhS>aSiYgIL+Xtehmjg&xqHCccDv(~E(|XV>y^P~d6}I^<Bl zK%^2t5ffcd9{Qd9?zxbva}aI4#?|I0OCyQ&rpSh~JRo@O^oMPH8MEFFK#7LH^v~|B z;r~;$V1alVgKz^t(e`>G+<EimH@6^A0?iuvYP!D&T;6y}SjPQO1ip5dQEjgg2mf=V zW>4@TpUne;tK2&cD@$lwP}xhm!~)*T`gcf_MRpKt5(h$88aqSLh<LSYpiaSqk5VOe zyHb=A8jdTjNq>z#*82Kn5ddK5XAwya=>;Sid{cOWFyY;g)tG^Mz}NyRp3;s-M=O2c ztFk9kF$jPrwbRg`#?y-1*L{Q(YG*L?=HTDc9O%VrXYao?LIuriPx>S>%~aG$EQLn0 z2}Z_uKR4|I`D{?Aq9rVe@D)@KF)X${fKn_5Sb)+qz6b_(@oZgiH3L2i=k;^>J1QLb zTIFQ09Bs5Dm@wS&1LKWM5sZ6KSt?Jvk%fC4oTYHgj#UrHEHG<^vCxNCt?QD0a0$x4 z>d9qCq21+V-1MC|y}0#r#{mPg+s4WOONW<w0fQ%knPCKeBJscuuWrv^OKEPu3R}DM z@8`J7v2cFoX+qFm*LI@jj_aMW<GLy`>Sxd#X)j?KHzOq-^(mcVC9U+<ZCUv3OHAI^ zz;YFRBp)TBxD{FzQZA}s4JlLuhe$LC;GU7rGOV@a=o+%F=%^R7BPO*is3fC$9k6E^ z;ZpSQrAc0q=DAJQxTu9?@3p}BW1;CMq$vhd-wOS4C}=QNvv**S)?To$ydPK!`H5l| zLYuK2LS?U&Ry<kNKX*VvQo<=r3GSErf_?K0R$%p79{>xc77^>8THO3PKAmOJ+|d4k zQnCf1dX83AO>X#<Of0Av%3o*pIjm!;AQocb;iy6vw8)BY4Y!DvqmL26$o8$ZL(6$Q z&ah&(?LWJhj4(blY=xgWTI5UBi1wzoERNZ!`683z8Oti&Z&CG^;oOd>&o8hITeLci zA)gk#uR6oVSc<~wKZr&{Vh46#O`kw1h{w&;;p3M;cU0QUH+O<-T&><YZv*Mq$nje# zb`9q9>%bL_B(iTe?tx>A9VZKTJs!Z=-H&^uxhV<fjyF>EmKX7|U-6rUL9(y>!)GLx z{0uUnDp~ZYeK~hZqn8m}qnF;J-AgL%1$vU`%kkD%2Ls<pCj2n3!!0-o>7a@~_}8ei zUPTn1UhP-)$|mc~|NKLKcOqh>0iIsv7tExPc|5~Y@6^~zU$L|}82_9}jr@Zc#pG_X z+l7im9GS>;Pnf%Ge>0>C&tZsZD^x%@ODetPZk*oGQe@E%7-{thFjDfS7~J|Pkdq~) zTAlwW&#mJ1eqK9sJj3XP2fZz5L4s1vcT$5;D(<^v0num;mWmMp#LJ_i%uY1~r=%vU zNCW+>T-)D&{v}C@N+<Op?(S*H0;=s756v;t*@2&_mXBDp0*<FT<T~i#vRJqv9BUj5 zONXOM-9qh-$b%bXpzL4`K{;Uk7Crv3V#p1{I1xI&>mK&q8Ycb{i9}`5##?f@WFOkN zaogADng4JTCI-zJXru{njSV6w*63DtUOU9_IE4uy(4#;v!+YpXaq|86Wx+}Hxhx?P zs{%>Pd1T=Ov%eO-CZ1VF@|cA5Ec}`{90Z{&1ezpa<gx5%_tp@!5dmAz>|e*+73!+- z*8y61O#Rm%dR(3c^{YAlwD91cQ<r-xo^#R!H=vzmnWt@!K|IBU4S)E576?pEQW_bR zdY8+!63mC5LFm;D=mkp#m|MCF-6MS{fHY2{EX+03ks|i_edA=b^n|gXlOQrJ{Oc7J zXRC_PhRvBoftB@F-%`mA73EktSVNe=FRnLLoom3mgnHS<sUk}+ojj*i-X%^*6lAZZ zma9Ai){lSz79Y21z|;2E{ihyxRG9J<*(ze5lngv#zv*fyI8?L<0W0*W8{(UK9QTD5 zlq)qztnv!bU5~%+;n|&KREm(=SVXAyvB%icTKrM+jyLX3$D>W(+b8{0J(Oa8V+1HC zt&SD(Y(^1owD|S)8aQ2#mA^dhd%8<da}0&#)RFynL%WFLkV$15^`LQjb@fA)A*s!~ zIPC)Afx=+X3iO>S8%LGW*{76CwRMGF>`ZK$M0uYK9}E`Lm7a7>fk1Zg1xij=2iYE8 z;0;eDh50vW;^1mz40pg5&c0}vQnq&f)T5$k5wtI}Dw{D3Gar>|D}o}dz@s-<u*4EO zE0uz)|2$3^B&<Yr_!nSCKy_Yr3YaX%C~Zm3MRk+Vou!~uQ4^}YvxE`_CV!Kr)JKbJ zRM~ys0qf5k_JItV+P8y)>h#)X<8KsigHmIAbDD^jJA(j|u@?|+ab@x^WCZ{Ja`FF% zYKGf^VyrlJCa3Pa@WhRf)R=UcbuLZEI`ZqxaVcXocKDGL_-DPg{jfx9`%GOt*3B4) zXOw{N(QMd$9h>i+bB4X}v#$D3UuqSyo2q1x7O(mNlipBVAvmsm-3U@{T1(3|7xn2R zI<drx+2w+ITK5I#1w|d^BZpyrg&2oNI|Lft@2Z3;5AZW0zvuT*;4*4OGZw5MPv<ru z_dDWe@?|B5rN;xQpT!oE+PZ^{=$%lUdV_P=#?`lqbCH?sJf3mo*^Tl;Y3KaShLlEX zYhmW)1dm5|9WLbp&rF@}S_K-hJA|_h7jxwK!{s);rt(WhYs}xg9F^*bvamPRA7Dg& zU#exB60A7KgAvlMLCR%jTbEcYogg?p;3y}phP7g<**Ec@Ssk3pZQm5-PF2qTe$}T| z&aJV(8xT|OIzKjKe=n9|omyy5z`B_n=tlASXWLqW-+r+>(>s|e=^;}D-kQ%3>zk6; zmobohn-Hw|cF+#S1&<68)~h<8wlklQHwuf5-)UW;^0b~xK)I-sUS1odqOE+}xRqa| zYm;)Ps#IKms=jc-skE@FTO_#6bzNQd`S2a`Q|;cIrSpgy&pi9hPfoZY^MQ*k_l_q+ zLIPw~Gv*S@7yF*=nI9$Z^AKoH@Y)C4<Z@i9A{E~syU!;mC4<nZtddJrtV>nHE7K{M zY5i_Mw4(jl4y`J%Jm*Z=y{=tmq9K-X<zm<GQ-wjki+un5Lwqa)6;qE9ieojE9j=80 zPYV;27f)Bz>)d49V5ZKz3QKU8wT<m#$`gqMIUS>CiHGudPDN4n=wz_$QO@U*&lcA- z{o;`|K#RShkP2%2z`dyp-#d5H&DD2asF4!XmxD_h=ga5KsIknx+*nWa5(SiRwnk8b zyTrAdYf_E$YmI|VHUp+vYIe-SgQS(7_is`|e5q6dQiQfW-OI{7i^O1Xns@y*s%RJ` zWlbdz^nXeSI)SV{H+z@csgdhjG@0mUS&S;OjZn{`Xl?+Ie;AK?(E;K1n8DKAaSfQv z{cr&>A~8qA)m45OdM&Hp7zw%%7mZYZ(<9n^Ugp$h5P(FtQIk<aBZCx#J@3~5ilAc~ z<~uNbAReE?6k-@@2Lm|nn|$5X*^qtDzy0~HKagQ@Zuq9<rEiONdQDL-v&q2FKM<t> zIo||`sdbW99qaU-+#}9__Z6I4AK!RbGcbU@F@%89qm^}RPLdp@vp&@5k~@>ihHO6T z_`t!52%MG#Z;4Fnrkw^(;*iCYUJDVsaj_~3MowK1Pvk0bYx`{Yz4yC&+GQ3LxZ}Iu zR!Fd@THa;a4+@ECi^2wB2N75Kf23600uvjFN6wp9f>UyB=iwxt8bexWDYwew>cyLs z9s6}M>$GC6v_^A~?nnw`e>z?s>*xvkhtHmE-wZePCML)|8~b^<uw@O(3krqmDZ)D5 z^M!O$jRV0i6_|FIwkrW^!N34uavuN3_dZsLVmR>9T*%NH1#G3;A|FU%mFxsJc3<T? z+?`K}?I+#%Ti75m3|dRRD%AL+mSRgkzVN3&%a_HTkYIfwgIvg1hwyLUq=28oClzyB zFR=m^f&pbCjq7wvF^o!qkCr7ODksr>F^l5|&MrnU2-+JA$u4zgV9%mcbe{M7zfMYY zDoS>no#rikmBTSMsT34d6-l$v?>eZVuF*U`;5xe`lADzvRj{?R@u&E8yju{L5}H|g z;Lia7dqmUcAv0e{*Jd;A2)IqimW!nU`u{TXw{%UL&{~~fj&I^#+2#5;`DKqf5&l54 zOHTT&C)1&(ZVuC^d;<=*bVy4fpz56f|DS)4e(TTTX}q=7fumeX^WkQn$YS??)t(G? z^K=@jt*w(DjDw;NNW7GxUbmPIDD7WW?nt<t@1oN7%m>gRmuKcm2oHbs=Q6>yXw`Jh zOx+Fl0!x8>M(juT32L*rSD5vfPo=W`YW?qy)f2B(+Tk~<UJQ{rcZt2)RIzBqSCI#w zODg;$lpU;1hiTPn%WHFP<<rJ3<h%St!)umirc0p7g@ul-PpA*?a)=HAhg{I(Zh55Q zu;C<$w!XpMuuPfd=T46k=_nAWYoxF6RyFd+59~FMg?h@>j4QB~Hs;5_p#6Pu8?hLq z{XKH0W0F^OhJNSM3J`u7k?e?KpcH!%l%r1WG=?f%&xUes)Ia%4*0U&xY0=J98vnu# znnaXrTwWI4IOJbaUL>eyuHhsbu!><McDaYxs(V4JUMeU3+4`um`Ti%pYUy-VC2=KF zy*`iw0U5tDu6~%+Pa%7Vb3PTFJeucOoKKZ!f0mjo%#Y_djk1(AFL$WaH+g1mU?xGz zSKA(fI$^#v;~DfqSVWxEGYr89JCpwK3s|u*M*_u^t42*kvD5S2LF4kAk}(C0oo{<~ zwtFB9hlQT?I}gUQ*qAeBTQhs{vWchxeqL~ME=r@r@zeoXtEx3VM1QftrAiw`41pcC z$8lCbx`ebnSkBT;<PuN{mI$GhOT)3u(a!VBQL}yjrE_|UXM1ti)uHYI#by7Nj#Z}6 z&RRXwdh}@bu{9?-Qw*oa(d*B&d7JZ60k^eIVdkgAfCK#%kp72wx6FUy-I;G>xYyt# zjfU?gKa-%Y<m|Hzs<4Ibw<N7|9i(lbU+}8Y=@)FZ7C%;XUV_(VV2dS{Tmr!{{pU|* zvMuzo`~7!utYqjdaSe5fST>m_97121i+(NS%3AiHkBOFH;Z6D!E+l3!FIE+PQzcVa zC{#Y-!EkVkMD!#MRJc~}Zg@x-@4)_F0O}JJ?2=MGB0EXT$_Kb6t^p+|?h*HaKB5YB zs?O|k8PW}|6{ULmj3QLHU4y}J{U|sRg^W_Y6|Q@ySrwEBu@R&}Tul=x*R$QM<bu>3 z!prDn#>@OrAu97$8uPVA`+PmoY%=2*jZP%=khw!<4oN&!sE_*ui70ad8M=&;J~Dcg zLpF~-Azuf1pVS%i!-XJk$9W|(A!X$4klxF_K22*+Xgc+)x_w3l4@o){@R8o59tat^ zySR`!gqoCmDxfU)0ivWF`lLwXA>~J1{&iQX-e~yV8VrW(N5PRO8k-OR<cRhq#1^Cx zZwR7lL-Ab*2SH&C<$)-%e1}+6`!f@_2twi<#1+(mwkE3f0}tC>{b;pXd5{sLT3YRs zBm&0hY?f5{mMCP65<TVUQ`=I5#wj|vuBcC<8V;nvw?+|jhA7pF%;dwW4nldLJ&KMf zeS|367VE|O_7&%O89<cOLtjdgzUn<`g!!nG>+afU(sSG=grr+$>r`ABD>hr8+F&qz zSrinBLPii)`mo(~ZTcjUosMcd1Otg+ff34sJdi2Fazv6LCahVM-XR!N<uz((Ry`UX zL@Zi{!nCPjN2Y3Z3ZY}IQA1{6nNnvEL=Mu1K>fTmr>)zFAF@NkbBHAf;SwK;I7)mE z@&!W3)SVpyg#M`YnGth_^tnfVXplO+R!9*7hI~<`hvZ4;8o3s%ar%wL%1a^DU3sdH z27}?tprA;UxjnV$gh-NIi8CYsiDKPhwbk}lYD^$h-H9)Ra597f`J!6iI9{XH8cOZi z?FFQX#Tmp{B3N7JdXp;UW9eZXLgrYl5Huk_AX6nqAz8|1_lvkm^r`n=i;^A0-gCEE zi80$^^`|N-HE<lEKOn`y#~oVMABq>L6;9|ArW_z)rZup>nO5l9Y(9x2#Wg~{Tp#DU zwwl$#g@_sUbIiRtbB_%M!?mKYNc0*JSed8J%Q?sik`6+NoI$Q~*=d*Qd$91sqED_g z?vWQ&5`?TTU1y$r+UH#D84QLmid##f27|%yHDDxaFc=J1K*5qI%Jt!MR{?Eiw;TR` zX?#u@gTZiXQE(&*d6s&2Tvf3@r<hHq9P7EPxxX`ca@1X1QmO(ZmC2O0kHAP{OS`K| z)|`|%DR*-0f6q=s0Ursx;CpsjaGYKCy#5#rhT8)rQHUj+04^fRd}+DJgL4;=XnFjO zTWgQHXM>(f-w_6+OwN@T-Enngz7$?-MdqBKO^$0yeZnOK2*S+aIuT?(e)o%8y?da# zqBV7mDp7}!a#3-Kt8Z*kReTic%M+DKuqZeY-Xm4lpjx3NQupB4wA(#;se;T!B_&#3 z>nJLA)it%QLFyo_th_?|;>UD4?WScg7;YDoM9a!@yTV*Vl=;$f5#?n_6heH|7jr+1 zd#Ykj=ns_bSKDGL*8HQKf7Iu>rvsjz`(@Ho1^k9YOv84}ENz~tzpb;!Ep6OCQMBiC zW&6N&_YJBh=+VoMZh31@bwrmocHQ3N@7(6@W7j({>K?xMUKQ=fXBJhc8D)F4LVMFA z?y(#rwoF)%wY!II_t8t$6oqUzcaPl2)Vzit316eZV7NU{5-lywz0dk0qRf{TCbA&y zA`%4%GvDiqY0x^3|D27xgPxchs`bNxaPHNRr=&`}rF}`mN~PUgRy8ELA|wf6LWn5Z zLv#=v;Y3imXQ<wt9#RrSiGFGGzUvztb+eLRqk9T8-jdYA^gw6tpps{g5Gn+UC^J@R zxmcdf?OpEao1fjr{YS32_#9U%gTZhsP;izgAkNbFSfce&9nwRRORX_CEa(3*9d|E< zK=*`92@QfJit2r3iKmt$LE*yncgb-6)enDY7!cwfp=AH~yC2kAWLOWek`Ga4j`yFu z7K?V{#7f;1GQ?_Sn2(h@Hoc%>L5|=4qIEDuFh0AiLF%Z7u*{SXLWvj*hTDsRn-lf( z-Em&j2Vs#3L{}@re1g_No@McfTB)QnFG_f2%278KW4J#0!~7<ZHvK+$eu(6FT*DiK zATME!na_-Qam~woCv%(G{SWHFdc`UYUo-Qv6SFHyy5YVU42IhUCDHK702dKuzO-CK zd07(8MnIO~<3dIx5_O*E6Ky!nU@+V+d=`lU2qG6rTtuShr5Owc!%ak95;YhM219;a zibNSkR~6rJsnXLHWZ?WB&TlXn4BrZrM2m7P#spBU%=jIbu8F#V+Nct0iRXj3?=1S% zmqlDW>JQh6tM0|mug_pG7;Y1kM9=dR1rX?XL)5()@!VEN%#GDW-LI2zcRb*!>(T3B z&;2^(xji|qtf+8fQ;Tk5X6Z~l27|$H8*w3t2Ea#iyEE!WghV$xqUys5fwF68sxj); z<+IGo-ame}4uip9xNT4pJ<r)b0CS&^DC9XIR_XU+p6(!el8(8jX-}<GtkTLvnd=`K zcj@7YGj$jY2E%QHl4w55F;R5KO*i`c*V)Tn6uaY^5)lRSq7CJ~AP7PsxG$9_Dpctx zT#vzEFx*BciH1*RgWW*vOHY=ZPqYSu!SJo(LJ~C?3<ksb$V;O6b}ShT27}?6NHi`( ze&$NU=dT-|yKVHMa?xl^MU|1d<2!S@$mbzYg4hT`%YRi&1c4L8oDqP|J|Fg8K>P$D zlW_T5KnSI|z0;LfRGdls9I(efa*5v;;Qw1w-{|U_TF#_h2l#!8i!b&P0SIX$v>nI{ zp7*<cY0xeLG~x4K0vG*<!$70rvi~c?)o^7JC6os%_bfW`J-myF?<GR8=~UFsHAPhm z6ira%>|T#}ZoNI`#_FT)=Sj~U^?R<i%>R>w|15lO@9J}h&p)_{+2vE`KgB&d8%GHu z>>C(y2gh&S!06=Za+%|*>KZpTz3AGce$M-%&HH*>T~p^K=T=WhG*`d)gfJ)PS5D5? zH8i<}wH=kmLpgu{@oTp#bs~?q|MZQLHu-Am8r<~a`l*MJI`#a*`l+^1&PSsZLd(^c zPvAa$`rXKP`o2W9U-u55t5Sb<+3bATzdiX8tmcJ6{Xtq;dBP1(%;<f}RmYjOv+B%F zLxym5=fO$6XUb(AqmsovtgK49{U_fE-l#MoQ3;l}c2DwUmt(;bE!c1x=eG}@s-X9y zm*1b$Hi9y7*WTSH|D~~$G+t_JcORVj?Xr*g{|!&fN!=gyx8pjq`w-NhT_(GIK6%Nv zb@saBcfZMhcq(UG+29_wb@r*`V|JYvl{0)Eu0*00MR(jnOVoWg?78{osEQ{4GyxgL zbVdKLH|Bnu^4t^oJ|o{ZI%4V~^?V?vj{6Yr{cd084qsDzPpP*%J?z%+9iEZ3<L%nn zn-+3eQ;E9t$b@QoE~8d2e=_t!?>~Iu#%2~XnxQ|cqCJ*nPyg`Ap9(Tsl=BVEt@^#S zz1yv9?dyYrsPfSGjO!m7^V2$d^x>|ktX8t)L4yeUhsWKkAO6$_kbXc$AAbLDH!-)O z4@Dd0whuJC$O9p8(x#-}we?MIYJOG8h_>d1Xzw0AbJao&<FiZhp!_OzJ`qwFa2vai zwEczEyLvCsG@V`A)Ut?2-+TDnjn6Epz%;_rfqFkL4C<u4Jt3KA?|)O({h_fL_uY?w zO4*nCLZUC|q4H4i5;%JGgVg!L^`u9n%^g+sClUbv0bT?<2gfRCy(}*S+JHc~S8sp$ zzf$LO)pO-H;CgrDzR(805B+Rv?eKq}qgQ(G*W{kkFUSz_Ywqdcvv*2NGfV5L_e!LJ zTCwCGNk8@;zj6!f+=pTzvQfP@?gM(Syl`6Dx|Ae;`se@C-x7<N-wOR#`n4wir=R}$ zznYe+ZE#QC{H$&BU!(q4AOCdU{q%n{78lVA_Mg6XO>G@|zY*zPzxPn?-?V%4^S|W2 zd{C==QhsalLc~Mk(<;PGTB@$zZSOzPIuP#WHyR$F)!&=E2ed2y$K}mEy_Ta_ALX9C zb1m{)zx?<g+PAw8p6T__Hutlrs6_75m>ZdzQ@KW;n7?;-<=$qKFe4l8XP}7-Y12pY zHn(=_Z$|s{Gg|`E@cFnBiGo<fw?x<4qOP~vC)9VNe#nlcRxHnmrZ2X{PDu2{pr_)> z?0iCO8TY7c5)Je|!)@`SclP)siuE`=F{^+a3EcsaU=`Gg704fsxBsp@e2vX*r%04* zLSK|mhCIDusq#RO)+`S@WJEnY(D=Rb$Ax(3SGKi%$gyAQ$I|TXA9SOV9;n_^2cpcw z(iY13v;mnwh!8-vJoo{Lk{9x#-w*+0h?R}#GY`PK-~KIs__u210R*FYcw5@L+<Qsm zAw^XmE13tFa#(4!1<5@WtDR2>00KgNe@%`doOvNhl=vZ0ULFwV```XctvarcYvDoc zlj}!cmA+xgSBUk8MFojdh8Gm1SJ%*@zX=vRFAg4Z2!t064=iL$n_SQH55K$h`wulQ zBu(A)`|g8dwQT7pzQ*M5lKTq@bD!ud?L)>`o&1In2KCdo%BpI8LGV)G7}7s{{!U+1 zl)Eo4IwT623V1IswZmucHR8dJv`ZeYi~9`GKYsP2FVHO{%>5w-SFg0ii<JKn{~7wq z%Yu8r^&!p;xnnJU_{aa~i}lHypL9e4+4CDP6+!yX|N39O2L2n|=XbyU+YO9ND*5in z3-GbL(AMR*ZIa)1YwwAU1OgJ}Kl?ylI1i-%Z-4zy7v8q5FFkpQtZeV;f06sg|AYV3 z&i+%q*R*BB{p-QCkti1BoLHkz(=iS0!O$G#`TFvR&X+=x5bS0LX)(<|-yGASJwB)R z<a1dF6_+Zk9)4uVG~6GEiHE4Qvqvq?O(6^(HXgjK{l{t*VmZ#QZYw!KG7un^C2wIK zE~0+&@bSlF*gdnjp`=E8sNnNJ(Vi0Ls{a7;5c9x83|OCoV^cy<Z*&NcIzGg9Pl$y- zXk$y85=8JPh2;3oG3j$c#%Z}e`oY79)j`}QtQobUr{;xd*VKx_iUC-1Jme4rF9pb` zvu8j<nqX-?6AKm+YVYpXhZ3cI$mN7Y<zDcj;o%&W`XK{eHnc(iuzdHVT$l8tqdTo6 z2BBjKVGTlHyjUO(2!rVl(s)R@9`1qv;O^_ohrEylZS%sRe)5y=`TNr(Ix6JP%Lr?U zQ33aq--mj8<o>+;_=ge!gvIXw>2V*pml~ZwNSOMtx@d!Y1yMq%Q$k=|JHI<GResZE zu`nSa{!5H(h=x!m79#(j!O>~`pFpmJhT)#@+pOJtp!b>o887a%j70d&vDTRi2wv#) z9YV*N?2zF<_lo;Cy||`CoDfT%dyX~8J=_w~9FR7jzWw<Vi9Ya2lxt&T^HA;!0S}4m zwR!(YpAY_nw8M*&7XvSGUe5f-uuRFrb@0FEf5S@=s~Qr;A`U;thR?^9NEB!L-c;1x z?eZ;59PbyJ{rOQ8>G3Zu@%;Ih>XL|?Y4V5ss6j!Toa?NLx()#Z%D(){A|8$!UD#nR zDBLE)A&7@Jc&et}jmRIDI7!7NB|2vXxj~LRNT|y*%qI<Nfj<;~eCps&$Ad#UbA7=> z84#9Qkez)p%)IZI17s=zl7N7kS~~n8lROAKOjykjM@LV;##|z-2!s{%8B2}_$iJsC zC2UBFp?aV6iHC{$7}8@2(GRRS9(YKL>*WC(81d)BbYVpv+)=TJG+%S;$=oY11+^L_ zKlcSR2<ehW8w}Z*!=%4ha#*F@7uv+?hRDmLzYRjHSXGqAIuBj~SY`UaF05+*xKCK1 zTo0Bl=YmKHL_Ni7;~HrX%N1*l-%k4^dFdDZhj6(M^nu}ag^(U3&u;~>GTevcAPq>3 z`gMv!jwx4`Na!$^-;SvktUP`zj=3lFn|iRUi-h3$-3S#^T9(j0lTM$?rO%8?dWHB? zwRQSiF((dr^D^aoQm&Uz?k8<?zk}b65e0qco-tzLJl8?KAsNz>hZh~uVtKJp55X1r zjVqHi+JDg9H>AdbTH}3#{_if=%S#C2XIh2dhv!89yV58Ae*u9q#l>iV{|dw&{65re z0zub{E0HL3q72u&l0q^W#d%#=%8~c4r=36ftrO!}6YnQ^b55Y2_uc<wiXfOe;96M- z2<m_g!hbl@C<7_;z?_<-*Y*jP=bbG6oQI$<d<F>T384M*{7@#R+sa+$Q+=PVm+xgt z7;T|)rx!7@(}M5*eK>tR;kLuy!*!fG&XG4r3)chDLljtnShsvXZ*eO7T<)|SeF^m= zSK3AS@}-CK>i4WZoJq@od(H>}63o6|*?IlXta1_mzljwN@dhumT=g5S4_6{laF+ZG zhHHkAz+Zxq=GVoQNz`C47`|oPI1&x}p&ASZ!{3R&g+#lP5!E8a1qxx4?M{Eiz9tlY zBqeFA;-esi8x-fdsv>T)J*E!);d(Cv7Mc2oC*1C%SGwq7!ybdda1~sML|Ki8<k)?L zdsE;i%DKK&MEye9m9*CG`-{GnV^;DZPFCr6I{k_7mWqh3&a+|<apH)Na{gegzsgT} z2o~T9MIFSO#|z-&n5QmP?0R}9^?x^n<3B6lb-I$v%4>F?aTQ@zoK;<1nOOY}UzfpP zFkBW_BGK-oA7hE#O89>ghv{*@PpS|5Jh#&7tKQ@O1mPVI_-P!o{WBm@$P=e~J|WJJ z<DOcokHi9P6%t)-^GTGUd{5FRS#~+CN!uTUL?<)#zaRCKR3TY*G~pW<?61xSE1vji i;-hGmsKH>k3jRM$t`}Zwfga!h0000<MNUMnLSTYrbpvAn literal 0 HcmV?d00001 diff --git a/website/www/site/static/images/blog/apache-beam-flink-and-kubernetes/gcs-write-graph.png b/website/www/site/static/images/blog/apache-beam-flink-and-kubernetes/gcs-write-graph.png new file mode 100644 index 0000000000000000000000000000000000000000..02266e118a7a873def4a50407299a82204ba36b6 GIT binary patch literal 228927 zcmeFZc|4SF`#&rc*-IgNNs?^I&LBl)Z?PLBOLp0ZF_EpTZFWWpm1QhhvW5^@vhOn? z%h<*;48zPk7u~=6{`LAj-`jki=a1)~XI`(GYs`6`*Kr=p`&iE7nrHgDnv8UubQBa6 zjJIxHze7PmpF=@$te^H6@E5~Z3r!Rh3@VS*)%9<wtMltWcC&lrY)e6L^Vu6C8socd zY#C-R<5eBrY8|gVeop$yYZg-q^bM9b@zk_}-&FOBv!dt<EiPNB8{Igi$j^4l;PU0? zjJpriZkVvs`^@Oj%~v4;F@dP@T@<WqZ?5Bu=MRcZpO*qBXWlJP{60?KTzvKj)}u|> ztM~2`D(Z_=l&iDu^w&G)=V>Wi=J$L9e5vr)su%bN@b)B_G$U{Q6MBl1j2;SzC#f`R zXDL=M72Y(br7&nu)e}B-OVU|Yi-9%N@9io3c8@owa(}FWlJx>F(@z;vyuK&L?QwzP zQ{q`Y@hQ7&i~~WD>7h!&EELiT-|GDRca$Rg_{)Pq=OVN5(O5pOE2NxcP=4FwLnfOY z^M^U7E?#7rd7MLl&upU-5=G8Dw?3MBJ3;QMp@O|}MXLPH<ehv<54eQ#6}ksHkt(!) zS74C_w-e4^Ra=SUd`xd#WT#5=K!{gJg8PAb<kPS(FHX>bl+{@}!Z`|Aygp^d^Dr0l zexu(xs^&ol@0{8*zH#|8NWJ61rEO2{;>(3s2cfrgDet}|DMWP}&f9!yHd(vHa+kmS z;>q(G*YfF#ZPawlD;w>#J=jDZf7iVH?9Hs(5s%B394GmQTAyF})=*4GDya#1yYnEm zlzQprQILMpX}(Jc=E3%v@9Pns4PqN%9ieA=f*L9#-+o!$4pv+LV$;3d7%CUmMPp#C znjd&s_r#2b$exE(kZa^?*{8#0nHO~>+4L@l(%&hJvyY_9nx65WY92`|a_|RjUtu|7 z%+K;On7Z66p1CAAGYj#S8Qc~Sa)c&Q?ZPiAnxj%z&)pgn2yK<qXCQ_~GSh{(9{mp4 z$)sJkj4;{hIr8v3B<i8S<FBld8>({O!-h_dT%2_Oe0`PUN(k)w+}D<{o!Q0<%Ou<$ z%Wk^$wF}o8Zib)h9SB!?ruI$s!9~d=hbXRJb(ONbN8*S@^3x3Co({Yy!f`#?^w5aV z#ZcGAf?FXc*Yr$ck51^T7#l=k`J95y)O2K(C~(0G`e%k37@ttB&`nEtsBnxhmR!Dp z?EUl;)=L~?2&&|wQz>CRBe?jI&?ci2cdzWOH<#gyL3}IzyQdCcp{!Ho+}w6P->2@U zbE|OW;;>=0d`88v^wwBw#wW4THZjJEKIYFOjD~DR7C4`?iTn_vS)e7JaGUSCEn@QP z-eUhE_6E}EJNyz<lghBpA;ma8`TkA&MsZ4b7u#bV&EE9%Gz*IOAj{tfgsfx<=GePF zVtOi$h62?2^yyR0dQa<4iY45WIMNGD8Y5MZ!s{b)zM&!~*xTgIA^cw}LOBC_vB8oO zG&1g>uCF|PR4^9mM%7c1G;LPrIzaS=o{~wFc2B-!o_hV%)=Ju#Sthu^N@$ph>B%v7 zhBQk1h8yl@%xQdoEufExtE%|XfCOl-UHd@G*rTb<qV<i}LxUqy{AB2L{^FZgUWPwB zckOyehwALvneTKDl}nC(RNE5JjKBNc|2^xMFeO#Ui=ppAr_}H3i}ur&UDJHa@cLPJ z*4jMvRo3)qwycdF!FkiqoD(sFS-C%>=Dk02L4%pBDDJj+N!;x?Vq<mrGM!e#6$wch zdXc6}MUMGQP9eCROM@|+)_fl4beqtXROStD{V2Pdg{Dhao3|l>Lg2WHDWf$SKNVt( zsvyV7rQ={~8EfvduO!0sZ-3x-7UI5;7TkS@d6MVF&Hl5dtjfPile9Z7=n9ElaOGbS z%sB6JlJ5k~&GRooZ5hASpP%!%8TS58uR^cu9Vs;3#|W(2Et9tEZ6?CT!Z&VxxcBbf z+s|6)(B9M8xwqkM3lFtQ*yW#h2<Qoj3%Uu&z7T&A@EtD9dCKWIvtY(con$Q&t&dr0 zZ^yH13=9or?rz<SHDJ5gZ2-=z(c4K|r<#eolyb@E(&abzPCsiu`G)6`(0$jBHE-{H zG`t6Mq$`WB3N=+%)~VGaWZu5(ea|*;{hocE;ithoumSX=hGFSFn@>9TL=5xv6+igg zD$q;Mcg^m)GxwqU4&t85J7hK}Q>~Aq;6(O|++t&!#1{V;`;_xKBAxs?8MA?%1u41G zd_58=@-a82duMyfd%Au?K1Z4knmQLa7d$RdHsvuTjHZkhjy@Z`TP9K#=j6L+rC_Mw zoNk_;tw7`DIv?08-D|tB?#+%KM1y;v2-$1hmT?H9oC+gM6;cRV6%-#7OMFZ?MhG}o z8Jw;b-Eit@%rScgyoN)}DsKaify~s`$g%JbZ_};wC%u{q!k5FTG+H#yYMjgR*2s;) zM|(#TVoGC-&cG#Bq~9iQr_gE}CSOfe%5~~X#+u_zAtJS?{-0O-?>(q${@jEuO)fJD zF?luuE)AGH<r(fDzH@1VZ6k=DHL93R>awG#tJzD@GYKXMPM60nqs&aq1WiPX9E%`! z1@@igV&#Qpbx!JKYo*D~cU?R?6C8dNbrw%p1&lD2d6xbt$@^UWFmAMZjDAUW*f;ik zG?rVK+eEqKhgx$mTK~8H;_6p}R>{WVZ^aF3GZ&)_gtHRj9Jw|-+d74Z9;NoB_oQSs zX(UFwauP;Ix{?>o(Se;|3n!PZjAi%CyveH-L3w8_>d2;y8CM*26?L_oP#$ky@m{Ir zW#fIsd!N^WK~&?ajpTHiP=nN0&+<9R?l|xYI1EfvLtOV+?OH?55S*hoIfzLvHoQAY zJMMwy+g`i9(7r&~U7y`#yvN=r0xv#wcVn+&SH*VpN@;R+vN%;5<+mddRLIgAV<{6M zs^YKlUrWBm@^@y2W^StCo_2;vJ#~7zt3G}$CnAkOFf=?ujCuJ1SMir6P9Ip3=5R;R zp!=KE%GJ$DW`U)1-%jY*^b}j$7{%6dJvt*C?HoyIu=L@L0q$KvmXIc2%-NRft@m1# zp7HP~3Y3e+vdId&8jV+PKiSS+^F8T&^89hpXO7Q4K9hcic}4lk@Vs{%m$d6_WmMa- zwvgW~r|#T-(7yFLcuj5EC=PSWTT4`@R<Z%o*nfOZh*P^G-7oOv==rYqPalPTe*f9x zk)@nLnq!x$f@1BZTKJEt!pXW{I~#^;kR8z8>Al=Dpfe&@GHme=3p<uN*MEHazQ9^U zy?7K+qFyR%Ti^YxedL8yQtIn!Sv_HKQB0)%-6R3$PT4Ma3QD+y{mt_#p#UAQ@)t{P zF(mlODea36m-|Ch)|XFmwTZu^xlwoH%?-Ys^SbhK_iaA8JMLbseiW6XEXpKCbWs4M zE7sbEqocn_@pe4y8oxrH{05X~TfVcj188e#$B2C56<PZgjG(Td$ZceoR?5&-lmjts zCobstPImddvU}#EWnTjta>BN+H8jTF5Q`cV#T&MWRO_$Y0~-)+C4JYzEHz10TSE<k zXS5U&s>CNMqbs#5ms%dz&nOx><X;4@Im`^d+02{IYd5U5P=2(zLpStX$LXTed8eu& zjeHk#`(oFTD)GhAK}!U~hjBAo<FXG6gk#8X5iWx4pv#rfdv)kOyN20Rvuox3MrQYo zoCRG)NAVBMrpn7v9Q6avvcupy!XndT>`HpEqM4JDqodE66;sow!P|}jzPT%PHA3!s zHtFgU3AG9VKXAd&m(Xrp?sB~c|DNg(wU&k|^@NB|Onpp=0|J|U3%gt;oK;+}xD?|; zc^*AlcU1`NUz?A$$!Wb5=fr0T33>d0_^e@eYBu=QyI0WMD|*U~$^DP#u$6s7rZ%RE zU7>J!UzzPCw2YnEhCJM}wJRW9<#Djb+sSvk&3K=Rt)}{FLQrK>Z5HumYj(?zjKoxh zR<Df8Dc_m?fv<V?h+>3A4X=OghDKvds>Mf(30Z$x_(Xf{=7LYpa<}j7xZt>Gb#4%1 zs&107pyEs#9$Q92_tbtp-HPyF#wbmJAZwMs))F&z^gifa%1BWv@SojVGIn#v%*;*0 zaWyQp+v7&~+!pwKaThQz{s#%MMf4yjBfcSyS&9c`k?dN{hAKp<dZ=)yncCgGx}~SY zh;+U)d}AO%C<O1{&fc3?X6aHd1EnYV_)56B>wi1p`LLdn!`VUO7zM(YLRIJ3lVdph z*K6wpcn8Z)#{KZGu*P-ecOBzLK0GC1B%!OA9Qr5Fv0q#C3wha;EILQ|k*t|wVofd% zhznyzw_|8=M^dxua^|w1!fe)+=a~ITfyeILH(K4;vAlmO&;*|u5f5Y(y0*r*>~wS} zE&<oH6x2sJDUJeHM}U8zBV7N!);MyJg6iP!loS+{tpB>+ANA+8ztaAvgLgs1pfGUF z{ph;hJ>VUfvHzlS1pd4D=N-5{B659F)FPUKLY3myb+vnbN0ugzr*Ye+%dQF~j|;_9 zH=b4FKNpc1TUV8_SXhynN9(uzDevN?#Src(DF3@B?8u8DewIsl?+O>o#NWMJ{0^aN zKf%wUP5GN&J@yt7;&>-LBV{FdMcX&`)2<nuxzFOh-hJh6oaICUyc+@6uG*PRtvE`5 zgp!(;g<tgv1=$zFw}NklpcRGZB>wT#n@2CWXeDgBj+EG<!v(bx-Xn<r?B4r?25V4L z_i-8YhLcb7zxKC4N1HR0UHtSPPvktoqV0D57S})C@COn8ByeI2(?6c*a9vfR_Jp9z zKa>r>YG&aRD~JWB;y<5w;z$El<MrpW|3nuK<WWT#g7$v2^pBM2{}<$8W|vv0s`(V1 zD&s=uG?;%J$eXOk%3Zt_ZD{~Qt+KG{eSgb(wj<e^xVK|H`uWiW$b4FWe92XrXWBGi zIc{4PKL}HWB%<`gHJ_+@Qc7ml`K>*e#Lf<XzCiwBbQgXWTjAT^h%|*Way9lpDsB3F z#<S`DtylCKZxs~<imWO=Dtac_&|+)8{`{$5IhPU8h~Hdn$~P%(e0L)Xf~5Wj_NWgI zjy>Av_GKo)s~kJB!A75*hzmM1;MssRI4{HEV9X!h=T^O3{#()G&M=Hb-~#EY{{!1o z?V(m+)3z*kHc{A;-@pl{1QWGYRVAR5eoTV#v=31+S_zjY)&2oiUMv8t@sXBMbS!-i zdlnPK01Oz}`k-$2ak6nQm<KQ~#mhBg$APKos-CpLmw<~)3~v8miCzHs63e}s=PRus zk_(yaGpAX!#od;%Y2<T$_$+@?T~lLqItz<7>iE?NmJcUcv=NP~3FI&F|DGADV5^M; z(YFYMxOT=>;rhI{n_O;H^5b58I+G_zbCcFxY4VIbcVBCSpl^J<erxw^61cY!?PCr2 z3;MfsJme!A>&fk@#QKw<wKl!5K0QTgrHy&}Xt^OMmuxs6alUo(Q(h1o5j&?6kxe7c z$h-)1kel3|3vYWRWHbt1ANkQJm<VPQBlZiulA*a@o;Vp}1v&HS-Y*bc4tu3sl_4D< zzdNsQ36U+puSt|cU|*=;W*dUh?B$t0r=xwSfQH>B-@1*7B0ofJ<J$<3gn8oUdP1yX zV_(Krzx&{s+^}0?_Qveme6Rfd?Ahwlq+sx4rcQU*fP0tRyR8y+DIO{-W>WJ0gGIkM z`4(cu5Zutvuv)^ZN9RjicC+4??aT^68<EzHCOb-z9K@9uakO(8o<6qWSlRl`<;iw9 zSo&S96`tegNa5u`;iQabUafgn5a0fui$3Y9TN4+P>J}Y;wI7Cs|F$@*bCj5%ZvV+o zaDo!|Zkl~WH+budd&?sPeJeKs+b(SB(VM59W<YSV+<7qU)U7NsW)n}SWCgRt2L3!T zDZm~iF##4&B95U5VtZ)4p>XKdh@ukFBd#DV>!#a&Y%=`TCqkff1F+VQd#so!EBrUc z^tTm9OE4M@SY#b@bOyQV-Y&eF#YZ|{Gri>5IEGx$k?=!7Cu=rPvX_#GuXFiWw?+am z?rqKNF`U0Qz^mPpv8P2EafRD<NzzV+A5LqNcEk&rJbX5rqi}S55YkcWo_9C0y6%QD z0;>Jt+Ou6CteBcYL(cB2w!yJ<@_nxkef~|#8FA>2NK+Qyrtf5@o05q6+5{$sE(c*= zc8NP+e4~+-NL-ubaSQnI!70tB#bgo|L|9yH?uN-MRbc7>d3#*gs;DERTcaN4V42Uq z#fH(dMJs3;$avk5fd2Sd?!Ji`3a=km;4nni%4@@AgqKR)A;YjW?CNUM?7o%6tRvk~ zgR8Z0M4>aR#fB~POy32g!3{U!oZe!isK|<ztoQ`XP;UL#$j$|U!Mtu2m-gT89n&p! z$3Kn5$r+Sf$#%PATshWXG{@W5NZ3jNeS)Lv9(xaewsj&Fc?_{HgQS&K&Q`~ax#k~Z zWz0d9x@v82>G}3s=R3{dl(&y~Ha>Oip9~*6*3=EFka6L~1h=YqElDC3<xYu_+CJ@} z`!@abF5g~aUzU{Se%TLgf8B$-<Gjt+wzl|c<J*h}b24ATlCl6)1qmPS9!6;-YVyg8 zb627|G*T-HEJQli(|66*M+iNL6{ZTLzhz~>azc?6u^$@*>lN^24(Lc#c;}zDl4`fG zHS~LThP@4^ZO+yKKbb+CFIMRjP*7@l4p@ww&k1s+8XQ9BJW(Z`wl{jW>C-@7U-D0T zq0;mrx*XDM3W+hdW(B*NZ_mrBtGJ2MjG0}=1R8Cx=@AtJu}yq*%nDPhCmOJ0m`Z^f zfe~0eVN2^8dM-lQ2L<h4yD_Sdw}LP{51LK&4Xu9l#IwNm)=6#Rko__%lZyLb<V4=e z2)C{R4&Pp`SHBZqG&U~$W^*r<zP?7u+`m&9J!`{T&0n)U-@Gr!df4W~<Xw_SfbG~9 zAwG@-BqAtN4h3%y{vzvXzZqt)9Axb`nuVk7%VnZLTdv1CHQ^bX-p-qENE+Ex5(|WQ zCG1kyd~~lS1x>H-HvT~RFZ(5a{6ezr9#yQFM+d}=;$rnCxQV+%O=T&s{Gih{O--Md z4YN3fxQISOIqr>I#CuLdkuDzjt%T`!tK&jqbkHw+iZ#D<V06Cuqp!>3a`gO$hpo4l zzRf|m5`NR1^tH{KuB#(9??&W;9OFF3%Bs~wI!ZvUqczfjt6CLdBz*IPwZme$v*(O7 zk>l6=?+5{ed=Lt&$?2i_N4)8Ih7t52U#C$-_HH)=do4QtHlNVCh6MB+Bvz2@0IOQ# z*OZnRBYby{y(x?@5GJ>-ruja7A0&F>U?-6tTk@0sLfm{{Xs?Kdx6DeE%X$Nry>@kH zQp=L)7<pv7+tTM2r!JqjdxMPE`J}o_$st5!07fSQWKRT%3Peh|SZ<}Cwl=$;;5``K z)`ovCQR$(nuh^ADEN4wTKZ)fjO`CiAtAHs7zWIVnJ<VF)ECU-k(If)D3tQSDK5QKb zh$#)LM_VR_-fE*p0<p^LLBS&g;8aF^xW=|sXDuO7{5WZK6<^Qt=<36Ci<!#vNyLTe ztxm(fAv?;h0UDnG31u!P(b7ust{Nhq-i-lP<FUPa)`_oa-t3bjM@4G@QJLgpwJv^6 zd!^k9zciv)2fsptpdJ;DCtN&N-SSdUtE2)<rZfGyNlDKxHjb7guHay_2m<bI6TRhj zi2KwJu&KOWfffF%TjBBC3?B`@SxL=^K1<)**)ChNpQXTj#JM4=NqTtoRYthwzkGbc zN`{oZu5o=3)NiyegA?gKiyk=}1{HOYAnbz+7YK59UF69m#d;HSDi?9*OjHw+J}29H zdF<6lp7_on&t%HtVD2`62Pcuf3k_$x<aiSUS!&)iV9hY&9J)}FW#8gM*aIu#RE3g+ zwhm=QSwbKT>D#Vmja-^MGcL>QqGPv-<PO@-t8Thas?K@5trcsM)^#4Lcf~BwTTVr$ z(qqR;4%cheTYeXRAuJBQ`J~-rB$b2MsZ`?9S*PpOf|XO>Qg+hp_95|+YA*#&2KeSo zJPZCUTkq^;^>{YAj>RG`(d|NO^GHQrVgw&ZHVrrW^pk9iADlS{h8JyH+VKoumc3Bw zk1(zk4qT29X&Pu-+8bG4lU3qZT7AM(x>|8AiTGP{<pg+bskT(BKQKn>Ypc~(A=uQK zt#y?sKF(POomudbThnCoVB3=$+e6alM$Ev*ID+EGH{f~rwYKAWNU1chB@nt}p`g+& z<)c}(y*9fxYQWbUV__*#i*t1SMqco+#FBJVg3$W|#0PrSkpzO`u@e64vF39LqGpY= zovBUzAMZB37QWl~<JC`m6P}L~yw{r;p{cxpX$<I+574A3uMgM}0iPzW57mcbz-yfn z0fYJRNBiOw*1x-NDI(dx0qu*8oj(%7aF*_1NTGqy#h>A^)P2)_r4;6QcG|EeO+c}~ z*!pR#Yl0lWeiQUK`d2n3Wsc8E?dn&kV2r-!&u&)MTs{6+Wm=NH7q`RjC-M~S!G?2l zVIC7pQjyLFm&(`93U{N#m1Y|@`fxicF<Wzu;^NFkU{5`WAA?5y;(c5<{E`Jp>x$Vf z*GrSJ>61^<mO(gc^sXT(*+HTf46DNu>cgVM>Xj#y!9_KAZlT&;y!%86BRY=t9n>qP zqS>fRS?Tg9)X}@IeK+tsetjd3wpvK#;Ro;_vRY6H*P&u)_dsQ58`@M%s{GLVKKY5~ zOy+N$&1iIY;CQ3E<72)=>qlWDo43*rd_vqMBxLHyql?NaL=0i6&>d;pq*4+|!Z&2Q z!K^A#;J%H{-c36U;0Fg+tg1#nIVJSB$j~t`Fht7*z+j|EZxQlD%fhd3_G`hCcxi8H z!7_H#f3ct<+>&|kg3|8>%`hq^Iew)r$v7rW9`A?SxpUm?c>IM3QozoT=I`{Od<(7& z=+c@sx`8G_FKAByy;EX^Z)S&{yKDh*-(JQ9C5HCt(MPdkPJ;s4nyocXhKn!U0YE6- z*M$z@kFmUzgHhScEFw*vd9%47i`riMw3>}!?yljnzYils;q-iEIjOt5?(@*9%ofu( zM#X{CmO0lHu6`C}97CpeJm9t0RN1iHVx9}MGZ~%zknP!M1_nW9F_on~#Jw&2=HyaT zM7OL+vgBUrt(&rmt$K)os0M7RYrXj1qGF*=D{MQsRDG|Zu^Ck(U;*1vS>^$6f35|z zM<pgSe_GOw@J(kjcZi|em`@!onU*2iNu5?|Gvy>p(>J}{a%FekC@}&-a<Fw~W2z<P zuNHHb;&T;Wsfg<=Ks@iDD!+Au_l77PMMKbK_nZav&47wjc~k+JG8r}rCgCUJ8mqhH zWz#%H7aJutRcarnY@l$#tIHyFthzy+d5vq`%H<`54a>d9mGgJuOhUEm-&RBB(&Jqb z)O{+OZRd-E^sAP~H^gXByGU60bey70+TMcE{aEqQ)H}!e>UT>ACZ$lOFHDgz_?gK) z7@r0rtlGJ`#l87WUav(d5L6^OGrv9%!H9zd^jG&{<CjFH?iKE*J-M~3L6i3}UqFl* zt!E}wd<J`q+$(ydd}T-B^%tbx!poo+uyO&|M5MRFI{ov9w^zq|0nW$42wSTIe0!pA z{FXPi=;P!bxcW_|lF|w>6?53tFQv?#!i)$RPVKl-jHTx?9-CQFzy4%(J7UF99&Ux7 zH<=aaGPdgun1(l|=S7gDpUT#FkyZhGG+67X|FYLA(iBl&qBIfsGf1i_7xnmN-K{FI zB=D<d4NB>w)O}7R!#Dq(2>sWK;B>YR?An$=yUUZ(LZpCU2|138fThx=r@UU*D`!(s z;fTOZr^)ZX2{cIGprk9dw;>az4OnTX!BBR>53Qyu@5RPem90tlc`CljKB)6ta5h7Z zC8YVed$XQVu2`JX1F#$cjv4xFkfpY|^eyICpZNB8!Q_HUz7De@;Ra62ewnz3TVWvK z>LZ%gx?$}Zlfy9M*XiZ!qYkjbN=NTX>{!#=y;?SHdOl^(M>E_5ZV=HV&}VNmxs^>A zgD+y@<O3_xB1Tzh3FLO$&wD-Vi3;lT+3YW?(b<z~G~I=Ew^4Dx{sJ%ws|3q&9?(|V zq!l(A{i;c}OYvMsisYBaVhz|A4cN5~8CN7{24o#`DG2U2P1Sf-%kDmqX9T71{hE$u zM#j5lfV>}XcgJxjyS*%=&l!4H{6bX+$dFSm%bX-H__oSDx^T2_`nX%=T8sOvi5s1W zQK5FXig5BOzDS*q0}1UKvBpjKXeXG9B=ItD>t3<J2fRw|W3C|`ne1bSzB~C)PtP!3 z1z<4jS5!q9oZzE#dpo-IGOf6>okw5V4*&L%wvNTSjvY!F>2h65HLgwiRgexF_E2^= zxWDokpXCrz@sw`hP?@!Z)6Hu?pp#P%4d*`Iy%1*WJI<~<#`|f)DemK>fIW!+JgzrV z5C#8NoIYs@ddNt^0T2U)*NZZ^a<8<m>5WQplfLzw#SPaDg`idRRw87fOF6F>tRuP@ z=ipbAqKL9VpRWY|&T9%@&sB!RxzE-w!95JY8x2^$z5b6)N1v<V786`($I2OHQ2~>m z-M8{q_@?qF2DC&%(8b88SA1OkX2`t~K_GFQ#hts%DqU?eki*hhGNU3U_n@DUu^fe1 zlfu5NurwR-YNp{kW!Jrwmv8TO;iNXoz6UGewl(i*H&<srkx=wtmjR=NgP#$D++8*q zE8U8z6EiEn>{9ix?WG2DPpqmIkQbXZ&Yvd}tQ>Azjfhr+o4M6Wb9-!0T31ZW%6dEZ za3x+zBCcp9grUr<L1i(eajhobQ!UojR_$UD?AoN2Zo`P~Kqo($jd~19G%awi?oD6D zdR=|^7_nT^0yZbYOdG)?_jq<C5Ecvhns!@12uO~L9t^a&&sn63%msWu>)E&nuCz>& zul0#92Uz~U@S0>ixhalRQQ9;JcyE|D<g9CdC$3>#uE{?Qt<>cQuKle(IIz2pLHYL* znr!&4;I%SOH(>Y5))&_zm}q7x*LGymP2nE^KRir&gQ=_Qo#2B_HDm?l8;m5^JbSxp z4<|;n=ZM$t7NwjeKBMjv5pw$ll&Gj~rknx;7{4mmt43x;==JT)*9}-t@mfKKJ%0;2 z!jjNs2JyfJuuc|`G2ry<v@xl_qS3jq`Ed7*Wn4D&S*e+d-0?*FAF@F;SHAspHqVF; z9kjfA+B9x#o$zeTQS{ZTkAr%C>+e?6gf4SReIL^Tiqxi{eN>SA%4h!}uGGz_I|n*S z<9ajbyF_4p+U83_(_Nyg+~<s)S@pp<kYi9%IkW{kJc~Cw*0jE?aw$@3LS@5)zUfx$ ziqyGZNFcEenaBy>jgb_<C*S%7>KX=z+P<@sMapo1eYU*cyxp7AjAiJ{jooWdwY6Wj zo0&*$TC*n<m!S9I=sIHW$_pz<T1?PN{-_Eza>%4KNGnpRw)KXDvh8Z_Z=g^sD4w~( zN*;7%T2bzsJvQ%0lisnFTf*?wqVvC@<FUzYK*I%1g9>3*z0uV(w_kbhPnaha*l8-2 z)2a$#$Z54DrU_5JlLc8_{|Rkg(3$+bL43IqC*=df>qJD;%FOMS0`&$7{{UUzzFC(? z7iD;BhC&`qHuf>`daPjIe^IGDrA@-e8HW4meG}}m<F%JYfTy&#<}7`js)UKgzo?#B zZoXtTu?!-fu*UfdV8Rw(o~c85^mM+lTHlYz8FTID9s!#^hSbxsYY=a`Vo<urFb`MU z{aJ+Kc)RE1hGYSIudFgGuC*Q^ZT4hcBE3^B8@di?xboT)mU_JfQPdZ}4gB(K!U6nL z799I=9w=z#IHE<!JcWENt`9_EiFGjh?Up#B>?%nIZhIj<r!%+hdb<*w4Y7eNf$tiy zT6MMOzWs7E&p9uJ=uTCJ%+6H5O0KPN!Q4)hRPF-eA)C!PWW(FlPdIZvnehr~7h?O| z66=0;TdM9<w{=H?Fw@pCCVO7Wz14Owy6$)V3n{nO<bE3#1u@C~y)a)MnFq6L8rKHb zhP7UC!nU5DUUR_<gv&_<l+)(KF>Pq3LhRN&SAP%f$)wMK$^%G?Oiax34_oUcSSs#| ztJK=*_$>;#-j;3Vi@!H!RKco^7+fX0z6hM}{ge*|q96p3FzVhg7hx}^YcHf{$A{rc zzEMa^T&i$tq^a{WtE%g6d|19Fm5_sJr3*1)DuhT4Mx)hlf%m@oT5t^_74La}8+Kz> zbocQC0PDk5PJ{{&#<sJlS9m;Y7>}z9dXxkz>&kp3Q+rZ7ZaT2Jx=_o4ldvJKDbBR- zqO$$78nALs_*)p>j50-DZu{j71;JZYLQz(bWh=a~T`u%j_-d-tP<BwOvWJ1+CI@M~ zf#y~Bsw@e$8fU;u`Z>`!mIqQqA1^1J_<y#4S-{1Ex4IS^p{e2aEGnBvcf&OKr0a7= zHS5cj_I?lH-tT6tf1hZ2)(v!WseoCOU?s)-CF`@t+hMM97gg*HE&YDIG}&nqS*-u! z107D$X6LbAA%0J0rVUt@EPG2Fukdj4+YBJR3`IrQ6N89%x9D1lO^xmT8*UK6B;q8H zgC&w4+A8c3w$gaFER~e?UKAq6Bu)j9h64I<0isE4bzWW8j~J)IHJ+kXNE4&0IdRCO z3>%#P-Nu=@R7vILo5br;4lgZ%h<xx(9w0S|y{s2<eS{%o!1jlcmFb;9p)$ple3kI+ zGm%<*>?et9q<$hh6?Na;bFhDM%*(@m(N;qF5_Hx~q|p~vu5i>$0_MnRSc?J+sJFrD z(8$&DsD!Wrvx@9xBo?97sLqu9s?N<;f}os;JQ+lon@on2&};NfQ5khckQh@KGN`9> zl3A94=6xI?;>P5T-bm>r(cEKT!5v*;^nTeU9w%Tg#<*dJ8>rIO>qx4z{Za0|lnl!c z-9&A;(S&PqOCoYwTYhUYbb|=Y*6sn7JJ4L-Q@r@EUmlGCM>rJ|UnIXIx-tRaoW#3P z1eA_9NQBJxG9Y_bNCIV;<Tq;}gIw@UPiX!7sb)icYZr)TiIoCo4=xfq`RaxD^Cxr; z;br%>Jre)5U3=4{dg;D|@~hR`|22008-V8XRUZLq_}1^cNYW4s)vQOdw9R9`dsm$B zYP(&usWNW#vX|?{%{S|vF1^M6uPEU3gyGRXj7f!H&HDIa)6bvo+k}ww`GH?SQkRC> z>USoAvM_wRWf?siR;|Oxr|9%3DNP%Wya|&LQd=6T!QtJ9Mkh`B;)0e!R$peZ&xyY% z#5G2vjfk@utL1vgv{>JL@aLN2<HcVPy4y2k(zVl3vf5;cOkaUc3HUItMbLS)(e!04 zVjsQggPOgpYSG7bR-h6KTUFLL`4kAh+_%+AMaDioMN4w?QT1;oY*RhA7gVu|{NWF; z6%548`2s1d_R2(rGxLvJF2_Cp{udjUg7KWyisc_q+ZC7A<EKa62P51V+EmWPiv_Fz z#rC~bf3&^B2oFwhcj$h|;+o|dKh#GWqagDuf#1H@hoA+N^G|SSSplzaD?}pk#G)bD zdaEi$na!Bfs)qC*nq_HfsX&a&%_cFcxEuP}oV*5v?{d!ET(T#r5g>G!C78FcW;VmC zPx6-gPV*SmB&zDMcjb~|Q)$vce%8>(R0wkJ-wmLNX#CMtC-&{Dr#W(cU2qaeyg668 znO|CM3W@VEmFuBXPl+i_|0<eW>C>>tWe42W_V<{{1r$Q3mG>z!x5ld8fN6j$XeVNl z#cP}m6ZS(~PH9I?9reKqvMITPTdOou_JyzJ*YhzY*ze{&MURw25CkBIW>34KGzReo zN-?_&%ZR`((4_yE{RjeyMB{b<upC0zC-7|a0cn`07?=%6T~b^)H#uP!R|kKWo`eNn z89bK+N{EG0YXGOYao8daa`l*%^`f_gG8423YSy%G_j_~VD|8-P>&|GcEAzX{(gNJL zxTr<}zN_M6=Ke1CB?$dE6pL%Er=QI7oU(uS>iBQSv?xZ$u8jy+DbPw)2I@~uKp_e- zX@%E+IRRZh3Onvwh<wdU+K%XACiyhZtNK*X^kGb^<+*WqmHT$zNI%h9O)s>3JFLKY z$JEU#!hqtijd5wvHb{Q?29Dmt2U~n?&?8|*u9N^20K?Q1I_QLy9(5P-n{<l<_2lcE z(qkUiZ-Ughnv7tp6{}D4FuW)9<ASk5DvxpQd#pHaPd5CLc?dc=)QEDL9$=2!`=fu` z@&6o=T@~9u;*jZgX`}uv>CR0x=V`YYcH@>`rFgQzcrQ5E^z8B$xM!r=#L8)XI=FtK zR(`z9&eIq`PUFwJK#29&7Z#f+YL<}T)%H?w!ratGBojFDKN9s$i_F~PVzKHFb)S~` z(>uNv61Dv^eLIx2v^ngVw92DL8?d~hHP+<DlR7~B=@PR1rjITMcRM?#)B^at{L&k_ z1|&nRtR--Ak1kX8yPY(&#teY@SvQ^ef4&BGbl04-LLbNu!jg1vkgIrV5)(QSXMueB z_?mo@DvP!+>(YMjf`uTaicIFr>jL9f@|{{W1e1`Gj_eC}fX`y0&d0+AlfWV8(C5e$ z^X>ik^4){e#d*N_Y5sH2n_IwEv<DUn>*#WPjg(`^?Io)=1-e0uj52=@v1^O_AMu7I z0_uiYKZ$jwq3#PP#~#xqk2k^PfQ&1`%6xS%B_S2dWO}#uKerB;w6e2k^Pz6ilbL|A zIG6()@emPp0_G2?7C%#E1%#$!_2?fsxl@L~5=Gg<nyu7*KMaIwFYkAdc{pX+lPM#> zwp6EqyT+J3GEoPXI`&+LItAzwaii3_OYSy+zO^D?i7-(ed4&r};8+E;tp`x@iZbI; z{s&!6ob2<)5`+XZ*W*B|6n*KODATJ8I<7Slrfh)NYJbl3d<mgvU}Ov%bFCKL(M>zG zg@TbnaE7=lodeetkKf(Z%r_&yO{A07>vB@v<C*p%>!xrC@6ja*mlpF$tR2wnQ67Rm zDi&LK+LS-M7xQH<qbP`gmjD{MiX3|0UBem8Rp|i~?K(PMCzHv;Zzy@_bKFKh3+Edb zpG%j98Zix*0cT$N=Y!?U@KeoEO`jk2J?|HVqK^O;!h5mgESUwY%%}Scw@Nm<<5Q%a z8h`!Dd;+0~TDn6`drFW0Sg(Co+L_sIxcF;u%{uma9H8XyI*#2UQ^{kGdtFUJ78`tF z`D3OMIZv$cNlHFslJ-RP)xI)^Jo;l)RcZjsxIQ^Bkjb)(vHN=L_*Up(e5MVs7EhMX zSu#};8w$+Udp*K`aIpCGCcxDfyo2P1EmjYJyRmL~UOm|O#cW_L;U77D<WimwjJMXx zviU<f6j*Dy)Z30+%HILA#SJHt_9@St1I#FZ$VNftX>)J_5L{He?rS^Psj58SI`%Q) zJbBYqP4>s%5hjp1{@)%+RSh7M-jrkF9C<hT&&2<)?<jhT19*9q|MRdm1l{TINbA6q zs`k5mwdfvkkgFh(fQZ%T-g5m>kbN^r@AnZUQ$f7yf!VmBVcI{ceGyRY{pQL4IY#>H ztFP36jD?=e=^xxp-w>ExVyIhxu-3oD@;_eaYXjpS4BkNe(JwPV>|^bVzscP9ygV>F zL*&bmgODhgg}QIzD4A3Ff8GY>8agVl|1rsD01s}_Wv7xk{o-HaGv5FlFX;CYBa`ER zH}qvb;9@ZEDHd%^!#l1ke<1NtbsxRRpM=yf&;8Sndt<+G)6o_i^*=KB?*(UoyNTOR zU{empYu^N-murDc<o5OkrvXf2`gVy-QvU;vSA_tg(-H^CWbS`7FP@b}yF1vJ{N9*< z-641uP|DPe-aq^NOJ5J+)MG%B@+^4=>twQ1A3JtTgQ2v(9`>#3G`Yg%{}~PNNy>Ee zz^?_10pT;|<u%S@WU`I!91xqvUz`6ke*QhMmQa}mxe}fb2UtyzmbrF7t;B6W!l#}w zlFN9B{b=F%FZtI8TzY*UfT8?1eQsp#{M21we3EyM$ieuZ`%qB6t{_k5z9G86?77>n zUk`}or*81_JwoPG+Rt}@xo=Xs{-h$e_c{3T8u|j6)Bhgtu<xBkq|?deIG|}0RDbk^ zX5YCM&4+7~84$_rJJ%CK9}mD%)c}|sWqO7{rq5HA2gd&}k*)n>zN|q1HJk<hluY1N z<p*psgR1x+BnJO7qt4^BIkI!v(quCCKPd1U;FKr7A-M$pn}?bEC_h{CPaZ+|bJM@@ zYW`?2@H)}7gu(C?yM`SC1OK7=ssLb6(Fx^$B)or+Sp7aeoXn>G;E?^?{XI+7pYaO9 zz*@IlhRBsL4#*~bVKFiB2h>Ib66U+{fQnqk|NBve10P_!?<$;Tu&HG3ym{YM*jZqo zAB_LJ4+RA^y5#1w831r&zP#j;I3Si5$owt-_C)^$kqUr2yJhEi9`w5gzXUj#_+Gz? z+$7`g@uvG6zpXPzCdVm`G<f0uFlb&4&^q4BRcFa9x5NOR!{wW|;{o{T-veg9Gt{X6 zk0Sk7FQmU;*myjsgE+8`uc3gN34NQhA`^I7z5_Nnr<zZ0BK2<-0G{yy!cPB3I{!8{ zaxeC6k~{J~xdi^(r914Syu5h9KSqJS@Cu@KkfBTL{^JZCz^<%p(F3iVKMCwXO!&R^ zf0fLC)HahJVAfILP}hU;!MXr=``#HVBU8dZ3Bc?LOB;bdxlk4$Va*rA|5^PI#<K!3 zBb*{b{eW+(Ci~EDDo-FYa#NMucP%FDwSUaO1%LuK5yUHUHG2-AHK6eKu|IC@8W6m) zlgY9tsv^Lh%|4>d|G?*YfCqxYuU*Mh)Xcxe2OI9Y%R7Bv$mBTiq}GJ;Pj5=D3WhNB zU3`_R`LXMnsYe4{6)i0quwALu<W>u-OaPrt%^S0^F{|*nYn-~rp6K6~8}_9Q#zsl~ zT!clX6ep;1e&p)ad+6Vv>*^!~cC2p&#zgTY-*Ye}YDm=1yw@eSQ!9fFz)keJZ|ko- zD)fNB^M*kVeRk&p!?ldBg{ME9*5E^E4p@7`=hi-RqQsMsm2VON#(ofFA(h2dZliKc z70M5PN}zT>g^0S3c_}$W_r8@M5_HV2G}rf517p@_N}I5rJoUMedOv;Tr;q|Q`_hm0 zQhD^$gLg_|_CaG*Sf9YX&>Fw|+Q;=!)6difq;2~tp)QAVKrJ)+JRhjL;si(g`<j#U z(W(O0*e`9&w-!W4E`RO}ADH%4GY=hDY#TjrFsvuOtJoOAe{9&5h2#E<r}*=gsyLt* zxCaAT3bUqanSuPZ5C_R&PW6I;Qt`NfI=nIFIEw?=>YX)qt&l^dZ*7B{bU(E69<1Dn zC$%$QEFY}ZDzm>A2Sa|+Wlj=KkR-nO&6DnftVo{np|19)=)P6oX%pks1W})g7}fr& z1gk01XGdKQAM^<JiRfwN()Tuo)?`AIjP~}rlX?^DTD+z5BH}xB`IM&`9}d5rfPJYj ztMu*S!wAC1wgweRm(L&)U~B~gIfm6+Tm2(s0A)2$v4sxdtMMF^Wt5;DmY*CEal@z8 z&Fu8})E<O=)umrrkiZ-BbT`(GbSwbOu0A_%;-c^J2L33h*|IPX$-^eD{O}tjNH8@R zRIhxD;3T>^n%FGo*%4QORDQD~GYUe974A-67d<`Zc>H&L#B?ny{6+iv&L`=dbFNV+ zsx>J-_}Nn18v7nUZDO(%Z=ObSrqSW*kqSUV^_MC^9y1Xtw13Fnm$1<FYqei8j_M!9 zFAS;hh2*;?hdr9R@b>#TF%xL1m36I^#~@BFqw=9t9%f<p+#;+FWx}FRugulzhv+=k z*Q$uRj<a9c*|^FTh?slfKhtfAfYwbEns$RM*Sw8TQPu5Vy(5mc;CZ{ETj+J{muq6h z<6(BunpdYqGC_64Y=>(a4)C+SmO5IB@17ld!pY-Tkf|@FmZgEnvhX-Mr5qn-P+IhO zaO^U7YS_!^EimKL*wY~$jU_i2v+OtXaVcT7rFeUsdhON-^hCYL)Rh+j=c;{(DJJd; z^|fGl%0x!YS$c>iDFPKI*L>n3<EPO7^;-a%%5oM6UqU`ifa$j3LRojk;?`z|tV&mh zV!bk|ZX`f*H#X&aZ4&0y=G5`4>&o0JsXN_o=Tg!Uws~+(CuQFjWHaW8u@HPR9q~rs zP=yMf0(QxcxS%|gWk~ac^Js^qk?IG7FY9GoB9X{>hZ2@YY{z0+y={{6U$8thX_}uN zP^i``q3{9CC#!&}g-VJbb<nK2Hh4wP<`RDAmuES&X4iB7tT+3hpswd7NdIfD`;~9| zmOXd`vSo^Zme=<TbqJzBn8anoP>|Bl()P0&J0Cz(O9P5op=sZ@xfRR5b_FuG23$aj z55N=snCU@rF6pRLm`rp~Pw{E^M^owNLq@rQr}Gc@X|@2wihFL}M&bgug_wknJyAFV zDL8#{!He%UxaFFU%Ahp;oNYrFgePV*R*NCL*`~zZ*(b1ex@38+vhuZjQJvJ;@6LM@ z#FPoGwK@lE#<0n&g@BKE?vX7+YJt4&u@&Idw`_C`G?+8k>zHdgG<<sO6Q<dZu(r~= zRM%EFT{U29)c=lW$aw(gUZY3dpNh~Ig5Po)xVR-3WH&uGU|+U_mk;@Oi1t5S{HvVZ zBc+T6`q&Nn1L2`hIp2tAR9`MS6VRv40<Bp!QgSn|k}N`gVtGi{uMS(seKVf?b(td# z{z7pKgd_~P4wlgLjV<2|->^qPV~FnOkO8*D6FO%cd$sw-c9gj>j7K)t@?Jp0Kf!qv zcBq>7HtYwdb+^t|=NxX`faeFI2D7*G+>FQj=}@PH{qBl<yZ@dj9Xc)$Q^Ie;?dqMB zKe%ecIDL{Txqyzcb+I+uIe612od0e-FX7b$b3s+TLi5EWq%gY6La3U(6q8dNB==(T zlR`eYJV1_1JsGS>P&Qv25gAk>y4fIxsuMa3AuCbvi9P1yQ_uoKTfEoe;q7BA00vFa zxUgTmvzq&&VRl#KRDYytz}UC1hHORSr`50;nu*L|Ez>1IalWyB@aF7+(lf2QUy#o2 zYbk3G&x#0K>tYsRLJ)AM7MuC9DN6yuyDQunMPZUf|Kj8fq_ERjyb<S49J#OsGYTt8 zEq2lzJ7-4wp(1Ol>+poeYrm@TYE;>$;2aICMdU_-l|JHFWlO1JD{&qkCR8e6Tf6wJ z>YHG=a1>`|N61?bZOQYziOzxOb`IG3*`8sq8V|O!)bcIyOPBr>h>Pi~UO<IxQHW_M zpDjv#1h?#mQmeMQ8CL!9M16#l^4RrCFr=<cCAFoPeQoD0xZr`M4IFxm9`|DC?4fvW zRjoQ1f;KU}lo$W&#s!0np}={et(yF|xqHUcA)}srE!|MI=#G$=n}w$^*_EdxZyF-H zH_Po&z6G@T_H*C4^AKkWI-<d%Q&1rP^X0oo{}9y_L-e~4g!m@7tGiXhcI`yd{D5>9 zvemSeR8T-fMY+NNTDxs=u05pYc+;}yP|Oy@7PsJs81a0Kfv>$^5;`orPG=pSX~15l zJsqJ0<}XVTq`!XOEed0^T{3tFQOmC`T97_wIQX;MFRS1*-6f}=K{drbjxZ_s?qw{8 zN>lB+e`koH$*|I3wu0*L&GdJP5)gqqwPQfU=j();|6Yn&;4pN<Dqajgwr8}KY`PMN zVp<%vhMi8zsBe&F(!_ECK(6~HsrBNblzGA2lJQVUZ0?}4iJO^Wt22zK_>rBVq&VQv z;^z(ac|VC-1!H}llfs<$*xt9}MCeUtD(1#~wK9cNx=a_#Lat13u(ZXMjy0N<IgWjD z=?q)}e0bvgm=-)sBR|W=eBtO|Rx&{S%hBum8L(O@<qjrOqiJDas4XM0WbUJLtu_j0 z%amm_bS{8zaWqEi!yJ5U%CCBI=d)q$)bNzOa4bR_o;{3N5R5Ir2MF@1B3x5pD8{nR zxHg5uf~bpGs$);l14o{&h0G5G_cN3#q{X{LAtXQOJAE9gGaS=({y}ME%Vag+xiI)u zuXe3vxvWt&L?Wx%XUo%aR${;*tjpyYa!Fxs`8GA}5=$gX#iK2wzGSY{8LIiC1$lb& z%g3?C`La(2i(d+(t>;Xq=UeQ{OuOF~7utR>+4Ov{<SBIoD_6HfGHSNTBav)n2jydX zt{sw(QMaG}xn+f4FI!xM+@t9k<N4GPa(AbzpI+Ct;giPL+?aF~xSwLsn1FfnRZr@e zE$xvQ>5OM9c}yJlX+j;-D_+5OO@}HDf@%e!cx9Mvx7o<X{c@zUp2E&}KfN{9lvtLr z(Y~Jc3KI}G8wqv8k-9(341aV1S~uZqAaFIUbY2iB8k%yGq87|EEp6z+5uK~YK8e+C zJQVeYr73wBb9_DdrRl-8wjBU->8xy_2wV#D?l+%WZsLd2)C+=fek$K?umF9`5&Is< z=Nf-N(4}_IyL+f|$m9q|bc~dAw@zc%L|s3m<(ZAxf@iFK|0wB`UcQMC<-%G}8>ne5 zWfP?CVF_Fsjch@VV4YzFLwf|%{@u@pD3?P|@h2_->;)b*Uef-WIhI+vc1rZgn@y*D zBWrlE*R}7vUgs#T`oxWv%En5Gh0HInc^mHI6Nl)}XLC6m9}3vtcU~*4Wzpa(uvA$x zfBnmiEwV63M))?$(ckBR8R&hwMd|78++<&$E(ko;8Mf)PWzt^WylL3|uznrca9b6W z0qa>x9+{~*JW=I2=@1}Mt3ExS^26)glhehV$3uQo4XssznRl;0HPtF$(H|)&;m7;E z&64Y?{48K~@5d;RZkP&40I#X*1Oh=|uz=;_Q`J0p>QquGg$p9Q$7!H;&{Nmzc{fFj z(%nIYx$71Fx;&Oe2?k1zHTM0+7z=qS{I)<I>Y7|_U5xT!QETRY3rM=Y>!h?P&)4T~ z-&J-bUD@LtvSG=F%(hj+YACdvZG>@4XcO8;HT)2^;V4A)rRZ6(fU4CLFp*W(D}0Ln zDt`Pm!7Wt+KP(-Sv390d;WPKL!Wn(U>GG|#(ZGqR6C+SKk1|rqYOrvy_y$BQzfMyD z3Nz&L*E`INAO?Vs`|%0sK#|vDO^?5NsVvK);HTjU9vc;#BIg>GJ8Y(x?)-9yDE9?y z8#o8YceTqGrczIl*zhmzZ=5dJ%<p?f;qLuQZc71V=?{8T6I8Y!gX|gc1W37z+zGIq z%ufa_nw+RTOp?KWit!bZxv{~offU1{MgQ?8r0V6K@Pl}TMKcrdMsSiYb$Lt4v$F1m z4ETw9eB}bg%Ve88+Zv&t`>j<I_w};Q7Ku==cPkV#FZ&@sdh>LDQnXw%J+iqlSDzni zOX|@e)}7Er?F6*EI8<l`Ky6AR8}i<Q;nQp7&^zz%4A2q1uG~CLCGT-YB$5}`3Tpmi zovM03ffRQ=nenBnCiVJ}Uw%D?N|=W#z)zCIK5F(?47n2$bsbgdwNVNqB2VdxH^h95 zI^^d1j=;U|B<jumI2X^{{MkmP$Kh-wI6?86%JC%#@Rfxnl`{WN9m=wxX!h2MQnmHH z(!T3m13FzTy)~*t@aRZD$&QHPp%E-L12|{F42(zmzECarI(Yr5qzMb5?qSOh9rdPC z@E|9+M%VlSe_g75bH23C7*OR91x|IwngK(<a6y03KR220_l)|tVp&!wIAu7JdxDX2 zt<Jp6y6DO&+`X{ILw}p==YH#0!efOq!D~m?gY)bfO(pNKQq4~v?U|Oh*G98m=d0&7 zWtletMrDZB2D8pbe(j>}-{`(6skEf&%_D52XMSC-R;b+>_$BH8S`90T0%A;Gk0a)F zvPT`m${ah#FGeC$aS;Zr^ZPv|La!~JQUsfJEA8;j%ut~X3I;t-=uonrX3>Rz$ecbb zS^GK+Xr9gPuvTUP<qp50OrL+v{Af(-0=T+rV*H_u!T#^LrR<mc47CFQ#+{3Y%|4rJ z&#?al_owa^dbc#JLA|b$;NC6$E{E|jKkc9mn5v{u0qtiUe!kd`zHd2Fo<;FPl+TU< z1r`n9yECFLz||Yof+r9k*CUhKjZQ+;hDb#Y8AitJ#OEq6jjsmS35hLo9>Ory%)I@| zKvm5#qYJ@-`dTbNXI`HX<qUA7Aa2=-O;W9({SA996ps{rBZ`s>?fYRQrF=*sH248< zlEPjMf(+SEN`5$<2$TaM&cDO|bX(A3Q=ZIafb;w$Tg?d5Kjl0jdv8PS*B%Vs{?nE4 zKV@J*H_bM{{bs}q-oL;_HI@2P6MF3=y?y5?UAI?deSdxXf3BwLyMN5%3yq2W>${c{ zEV_I&-w&;`{d@vw5%93qUrGq9`MU|dk)ucN`HFP;For)--S38qk$?2({Iu%!L3Q2l zfA4<-iiT-G(NOlVt|Nyx05y5%(+{Es+SfpbLGAVC-h*?3`43-PMAfYa!W-J9JJY~# zR}by7j0R3`Ko{cD59rTS+}DcILX86=3jpr8#blNKwd=q5cVLyOZa^&qXILBa=Xa^z z1KV_^K1gLG%2gS15fhcd&YYaLxl-hz&f)=wS=$q<s(?QZ4ZGjf%gi<rzdyw)cVEqo zkLT(wF49NYb&r4Ckc<33Y5DsvUe&+~6-L9`f9PZcaE9}u%%3wa1<FRWx0UlDN_~X# zKCqqp9LNLx3I^}@8N9j8CA4qHpKn`DwCGs7tXrB-bsXqHUoN(60o$d1m_xn|G!TIC zCprI6>#+X>pud#n0h@yt+#&tOa!xt!Rbz+3yED+a>MQbR-d{~FpRcRR)e1)I9u{F+ z?3by-Zb+a1BQC&O;Qg*`{ps|<Q`U0oqAG1K0>Y#7%+<?_vtd32zOegEblnNaulo|q zOayc(;0wOxpxj)*@;#z&PSASUmEO5kvSp@jIT+jF0t61$_}6vL1Bb@bH-W__X#Xhb zUo8Nl?2%HDh*1e+dAH;y^oaBP4hzKA{k~hfQr9PIzki|60gL|ch!+J4Rc?|W`u?iu zt72$h1CQ>9{j_;A=n6j#&3~=~D4o>HGrgrikEzY05zYT9;9w-+3@{U;Tz_|Ed{)i; zK8ammfe4howLgpfeiEgpM~NOO5j$RNKZ%9Tb+*YIo`-$}%!4cVZ!Oo>=Rfw<#v!2Z zr#>XdLNd><^elF`XSloaxB5J~Wt@+kLqrbXr{+K*O>}?rXxKma4UWmM5iPk?>#)_4 zt}-I0*^=pJ&Y65%x%`Kny4(3s-m$sFh2%-!8>=7`-{K)4sDBx-zy|~BZh!1i^Pu&| zJokTZ{jr)mI(kC48(3Q=lvh*IcQ=1tQRN+vrkG_l$l`=H%6f|DZyNu7>iNGe7yzWN zX8m&q=KWI(s~*PWeU2$yVzHO9tcGfA$f^4?Jl2Ev4(aoiut1#C(rxmoN|S%uq^74c zLi0C_FnPQ~bsMwyFzbgs8wlC&1#fkuX!^f6yY7G*-!E<ynHdq%Fw3e4MXGCNL>Y;; z8H!3NZP(uQr4(95g``L-O|*!Lw6rL+OMBk>o##I9TfHyt&Cg%=>fZN0&w0-IoP8b? z24NOJRHf#_OCfgMhWU2yrE$W6ktUh%H<RaXx!|<!27hZ681Hb}h_ipjs6lH#Qgwwr zFwlAj{LZ;c#-)hDX`7klVwRC6?n@LTubfWX6Y<zC!eE7C)}g%@mba-34NCWYYHSwj z==a~A5H5w)%xaVit8SZwiTVI^0ekiV`&Y&@ZRMg0RzJ&0yEJA+U{>l6!z{lDp7e?# zIh-pLsAOme159u%?01&!rU(lH3<Qs|u3Ve5UpP=C8K`WY_lcK_Qt);|Ho)Ucx){1& zpd6)3)?4<s7#o>aw9bC20_@Gm)4Op6(qRvs?KztR-ofxZk4B3{Y}ixd1Fl@*yj&df z%q-sBvG$-qpVxqDpOb&N#%ER5LG}m#8tHO|G-wph>QAfLA4@~F{q{>sMcIs6I**A6 zH*ma4O~6YboE0Zo^*2~bK;<VJ5*6`td|;J2nmgX$jI!{U9ac5^*$Wp6_%=ejOt67P z=_5&Qs%A9`#2-D@zz%}&Jx47V8-q4sRJ2~zt-qQ)Wc`%a9CqU(ztT5ALF`!CgpWZI z!Q7|Ulg8tHI+2z~!=XOfL}|IO!6+Svd$bs${EQ%5nB*Y`vp0*(h&uMa(f%c>b*NZ> zh(TStH*3J08sXu$&z^k^+2SAouO_{;aqI)8f@kj|@J@sEoRM@)p=g7C%-usD=Z4LC zys`5V;#Yy->(<=bafDH6&V-O*v<Fp>&IWz@_=~jL$O8op2X`H*^Ws({Wvv1Gvothj zm+Pjz88C3*z-em=3k#KY=aXRVJ(|s7A7DcBM{X0E^TEL@$ZuS<6mvc*CY*Wc3Z@;D z{?lkS(=n6VxYC!1rEWhui0xl*RM0c%&O!p(5(IXT$$Zdn6N3Y=mHBt=nU)VHkgub6 z4?lo9f@R?>HDMJmSvyU)+Z_hnfQ<shR1D(Llwk9h(aExTtB%d)BHE|{>-6gMee91( zqmaRcp-Tz1kVLv(XAaug?LyJ4{ee_I{B5gg$O}-|Zt@0)oFOukZ%6QUFMATnqq^TQ z5zcEHC2<q5_3cpK2=<S9kAp`CsF@R!uf>LL8=Nkfa%-ZmYS}<4PIP%i**IPccvQpF z8L?e!U(9;G{AS&G!!Mj%zFhmuni2E&TyvRlKB1O3I_<v+7bb`+_8YifEl%)a%Xr?2 zBAXZPcVB0_>)GA661#1m^X+dvu_|o$u&46Z)pYFS)y==TFTEaZ(HvzWYWJwz{ZXo} z;a0m%9>LAVFwnBZHTrkztOOHO@$^Q^z3jb;xm8;dG)vl||D<58tN-?Q^KYtfn;(7_ zdVB0ztgGYLKOn}F&Kg5V_RgX2rRFL7=U+YgP$0ed-cHHG+-mkE*omA;J}T@LIiXV3 z8@Dov>bCK>10IM6%N)uWBKG@UDE}jU8jfRSwz)EPF;14*561YzoXeXzDBUk%&AkO% z-W|0vq=Lk6)(xXSB>9G=O`k#4Y0SuB?=yrBUdUL#WajmQy<zM(o@1PdH31imUn9nY zHaggRwZ8qqC7M+h&KrE(L`19JiHzrx3SoAD@+}FKolUb|&Wzq+^y04P`;-Uj;zPHF zuHd54#e_hRj3x)r9F((~`iG_M4jC<+J<aD%lgV+;{6G9QsH31`c~;mEm?|wVACa<a z|Ni~?A!IXOERzqYRA$&WD*b{mAHQm#2QJO71Z7;f`rBZ<2(gv0;^Fh>afW{8NtD-r zyz3n%<4$;>!JPvm@FVo^#qJIs*SObWO~BHpwT4*~cKKbwNyh>M4b0P4#-z4{Vq$yC z1>~rc+oN{~sd!Biy7F%AL78ML*T1VZ#1AXQf1SE*eBrin^b(Q!QoXAeF6?G4mwwf7 zr~TM5Tmd<1id%Ik!8)ZkVzs0Glhg@t-m~B)mzw%P>R)dN%NI@4u$vp*o?jCm)A3#W z>EVjZ<Dc*CFS=i%DDOLR$=>{@g$*}C6RJWBqpJ(b<F9+RMAV)x><nnAdp76E<z&g2 z1(H$UPnpJj<JYviKgtWV@PX@7@u|K5BIH<WVgqJW3m{%w>fDZ1e_--%3dDXLYTo&* zHCfL}^K-?eTBm6avwZ#h_B7qjiZ3eKZIr*%V^DI9RzUTD<F9K~FS@w6gf&OiL{$`& z^GhA?tS`z*T{<6zwdkqW66s@s6`%t~9o{mXEt?YeyH8Q`vBUN!PG2u_3?g_-4RC1t zVphCC@8!$_lY;QtFg=CE0e8kOQ>v&mEArQ4X?AqJl{zoDtm}S5XG~{B*A+j-;<)2) zA397(U0QqE=COt}OT@~SSj+c~3~2>p;mz0;pD>vj@~YwC(~l1>I}a)`;o|Qzy9AEE zOW$<<)JR1M$(V*~$7W{PRkR(?nX~CqVaF9c$1D#si?;km*}PF*tbKu(LnF&Zm{tZZ zv&gR(lB$+;E7(~qU)ZXf*VWm4DY45{FSkWg@=0jN!igU1Eh@AB8*iSy>)emK(yIw9 z%11VkXMhVv0_OQ}>uLKQ^aadMO%HnhT%abZ^pt7CX8tSUA`W>k@7YyV6rNAlJGS4w zt!OJ8(X#ohY8ed&$rOhQUek<of#+8mhJ?+zU|eXJeB)S6fNte1H~T7$&M7I1CC8rH zwZAx?bw)6*p`oF?IJ`|jB~@Ks0INi6RC+FC5W!wFa~GJjqM0f3vcI8lH*@u4ej9gS z@$Fickn#F~rJ?!uHj`EqMjzH1dbIYO<Q4xXZ9Ck}KbaL>(u<sVg!R#}$oo{a5Wmt- zw}r9MAQd_$LZ`~IY+Jv`l^hFn)U0#;RQTWNPktaGS;1EAcPd^roH=tQ(qkviIO&)2 zZtwDOR_E*5{qE6eRRV~HG1pY29LNj5CK+O9yYSMC0M3Q-%=f&$zP<}~)@K-ajm^?+ zcWIEiW4x|&Ok1n5{T?)?u^^-g#&%0}R@Zc?Zkz2Q_j8wWBFnJZQ7Fb?k8QI}p<{bq ziK6Mp_5xvk>#qaTOR@%~JJ{wPHm$BG9K|o!kT%`9^F>SeDgJ_x{UQ#~OSDJhGZxR6 zbAh(|^pkNsX^J2KR#$lYw!7Ldvz#u;DRQ83{{dD4NweMT8}+(2zE822InvUklu-3n zZ$sdwO$j#*JV!gqH)S_u8y`>CD*CB)<bq79oWfPPqF*`!qO5wy#MPo<aw09UdY6-> zqNYd1Oj1!Q3s1cY(>;n!vve%u&jm)#9pP7FJ>O5sT*1vV-E0oM52f@P!^}r`n7nIx zhQG^Em6_>(_1M@dJyx1e+u==U@i;$^(Yc|<m;5^`FGh*9wn`n}mG^9qvTaeZ=#{dv zvK^2R)^&GtJHO`M=tNNmZTH4U>CG><$B2TUSPrhF<?#loA^S&lhMVGv565tT?spx* z{-2eSF!%gJJU@(56Y2OmZFjlTN{jTCV%L}j(Z^ULNa@Ir*0X`h=(>V>9iJA}b~Z=L zUsWu>6LGQ(ai?5OUe+3efK4W>*TV`XIo`7R{zb#AIauGiTJx?LRST=xGUNBpDSume zS@YQ4mcd5{Q2$I{V~1a?pCT*gcy6xL<r0sEyyG8-sTO?raan1}oanZYz(i%Gqx&9D z?<#80v#Z-$`|F(OSjVoA!~mP7ClS>b;hS+Sp$=WmE#JF3S4}ogv$Nz)mx^i`Xc{;A zTp}Qq6U++T&^4faoU6{H=7-xd1OD?H_Nrmov6b_v%2l5MM%3W@|6S!ME3rog$QGDy z?RYiI&-YdBB`Iq?=k-21sqa}WiKcbKR4=R#pX}9<tQr?nXsW#I$3gM9fg=?bN=7sk zVFi-2>3G!G-AZ+?B9jFjzOu?#$xp%-NJezL3v0N>pD_BIl0;PV&rsb_8hWg7x!Tiy zi5v8emOp8(Z#t!KU9j{nw)WEoGIMfrjzzqG|Ne1=1-abp#|)X)m`*_pNx$T6bAGs~ z_>zD5GiNfFkCv-gR1)VJ7`0fW^}KRiZKA@Igv7++4OzBJJg%&nmSW#{6RY4bW5uq+ zri~??OMxFA9Dldfb8_yBO^MCMk+V7z_i4mA%#})vD6rbKti$b`Pii&bs`8j4N@cC$ ziJ(li8#<fgR$g7U!1eOUvT&nIcX15G5~UF;43A;f_}Ya`)iQoCfv0hF!6z(aZffax zZ5b`!*;KE$!}9ql{-qMS9p|Lhzb=mnuITibGpE+GIa*1Pbu%?#eO-NdHUMwr%ppO6 zXFtg|Hw&oL<aMdk38ts2b{=gT=uj(Iv{KMPY|WZAOGC{TL1R?L$aswN1t_1JkR!=v z68&u6Zma&nt`ZY_jtmWAX>YG+(eW#@lqbV5M00XeJ^%TmR;gWg%}b=by6X9lT;1;P zvaGotX`Sq6`;ucHfKjP3NBA^bg;YXHqAgm>>KaxkawcF^4h*g5(nfc2CIsf5Uww$J zyuD)3R@VRS48om=8oE&9%hMpUC3a6VXC>}kKv<i9wU8#De&AP?{#$Vv#|DCyG0Gx( zgnKb@C_1Y{C(9<@)x>>CfAuYvz&BHOZFMHl9)%VT@2ViGL;VL_`mp`;^_+<z<#FYQ zy{O{&+uEe{k*%AlmWciOcIYmQmRj#gc=^wuF(Y-wM=nXtB-CTwX>%X9Wjck!yPx;R z<5m!!>NVcspRS(qNG=QQn=kMcc}%*h9yrNV3B6f%fvZFaX<H!R<}dpxgC|{PRfqsb zJ;rksj?OdqtOQQhH6T+`@Ldz}ob9|nU2ojWIHDZB?P6K&$BbM>E`F2#34~)0ZeGQX zQxjU2p3gEOA+dK#P`1%(g_|Cm{WtA>dVdi|VH0DB8356j7@j{${amc~5Q3o;wufxF z`gSiDx{FEgfajdbQ1zn)Wy0B1dI^G;BeI&am#d!+kGMVad5@WesTw59Z1wqZAS8FQ zq|0t=TXSQ_c5Wg0p-5JO&d#EFL;{=jpXOY&ULb#OyN5*4=+E5xN1H*&?D^%QvEP4c zHgYbj56H;KR6NlkCF0rm15cjs*;R7%rBUAqC5H%=zCoPea<Gr0Q@4o_Ec!1@1@hr} zeT@VTi!`7bR@X00M2O<wnW`URD(?$dvF>hykfWVL{4&BG`B$bAM@)6_^a3_hF{Dru z@Zn+Nz8igY@b9DghEl^3N|oN*iV+*)IVy!xlKcd^{0mAU@K`?98E?)Sp_HL%JHd|r z1L`*<e>@a=PkVNJLrW=lpi)DCeNtAfl?`ZLky*jN>VrInH^%rL&|5WrgC@7A)lmfG zr)EmvgkA%hn3Gl1L_}7$_e6;_S?|~xXOi!z^2(MO0hAz${je2Rg375Owc=7Qf~Cmm z4BMaI+82MGPe2cz)S_UH?Yf{sIL`KsKY6iI6kAD=$AxcPxCBJ{4?rhZ;~Hs)-FZ>q zhxYTvhQ#B#74ML#+%!Z#SjfP%RmwDjXZ4uwILO1bKm1+j2tu<*n}Foeg9SoqBxF{z z)!v{!=?2DA*u<rLr3``@iIp=~P(zJpmYurG1<t=B1Xc?EHI}AB(x&yYv6UmfaI4~Y ziK^r0?W|zyaOUJ;?^9Bq_#dG}!Wf}F{8Zj_Ba@bMp;MnJ32{El(OI9^e-I0Vby`(j ze#s^qL=%yX!0d}_)Q?#N4-yQuoQ;2QFkqg2thsJC=7DgIN{7)$9C&+FX)x$=FCl#> zuY%8|@dI<)Kghj~k2+hTN9h@LeBstJFM!Hr7$3Ia2^&c;yg^~kx(edg|1GG-(}v0? zlx?-xe899vWiT5Gh!}V58YV<MqSX5)bL^=Oc3bh{vtjsKn{P`bp`!M}uXs-J_2Eb& zu{33%iegbN9>>wJ0d}cZ*F;2f7Z07gSZlNDeuK}C#tOw)aBD^8g)$eZwfCHH%;_02 z^R}dG;xmff&(j^f{__540k?RuBBIKW0;0Fy3;2ljIh2U61Z_qxuCaO_9Gd<E;!$my zZt6BnmBoPL(Fa!yBMxpNFqy&mgR?NK{AOMfAkcu4RjR*U;5D;Qw%#u~iU?3$>A~)} zgg_mtY}4cu#;^S)D!+VuHyj;wa;z}o%-9pqj4P&*|4OQL#4O??EU<ZDeNvVZkeIc8 z+UBE4-M_Tc0JFw5)-ai$X%tBN4cd;RdEjJLrtMWG1e0VoE5V+)ex2@<HuC@fvM>Ep zz`BD*xw8MPKY?ezfkBqA85M{Ekx4M`h!S<JG@Rz!3)i@M2KZ&Le~`hxbnWBJ)fnLJ zj3MkU>z{{wCSQNMkjtvPk=+$v=CU5^p#AXf=4MTzd@P9_XVDl7IF!K+hyDXHJ9;0+ zp~E^DaOmlG<N=Yw-7&6*QrnH&X>_F1i-{*ru3L{kw-bt14CWnDz%&sX25d1`fLma% z6M3<T`A^u-_C^qG_al@jIx>+R@WI7mZl(67n1H?nU3o4-1(5D##fR82-8&ohHp?#_ z6P7ug_h<PP^F@#V*~~@XGeyv%`sLt8PcdUylvTxfYvMWTJ!)-t;|n67`8R1Dm4sZl z6ruae*i=bF<$FgSXcDlW1fG<_u+@f~^ad26HNyg;WJ9O1*}W8;Z|g!W?|zufUytzI znp3qSJKWt~t4}cf9ZVyIE`=>q*iTD>+8+h8EyMoZan?aqHw%J(V0IJ%d8p4Gx9I;h zNuBlpVJ6EeAEfe_{)FCPjp-&_s!9b=A%)4jt$3P9w~;l$i1~B9{CJ2voqOCTKB@y? z_N?iAKrOW88}5TlP){vsJBi$yH0nxe3?P6~?k6^GrH$*FU`{bT&C?t!Y&kw>ltd&P z_cyK5e(aCwckv=LidrZm6I}g;99HXH*aQ2kVv?UK(QuItOO>m^Ig(;T<CM=Sl)OH# zb8IG?p}-F&WRAaYYX3WUS_&~yn<rxKyamh6Ixw$W?MrJI-}fH^dhHMwy=IQ|+J>MT zSg%<Dccq$HekO($Kc-rx{-JgR!~TtUjFG^QX4wKW*bK=$f-0xAvfJ>-|JB8L2_RW8 zV@EOu?q%?-l5qJ$y<MEXS&_CV?Hnxi2>ZA?sPt%dU_7C^{{fyP0iG=TXIx|NMyv+G zQ;id$)tKs34(kI#p#qGQNp~k>ghH^bE0IgMeMu{NA`VHr*Kt9r6AVG{B_mE8TE~8t zlO;SW(sHdGCVT9o*f08RpR*5aTeIF@ua%o}nG8l($%BdOahxFvd0;Nj?FbN_%Y(rD zW}h^tm;+F#wc^4>_IDXsz!?#nj0R%-{6Cz*ov<*283%>&1G)v@drmJY)MyHl;39=a z09JNp_679B>&v-8i`^6#EuQ4^-xgQP=gYV}rZNz$mClP+D5NxX)5_`OEP&pUQ5ueb zt9obO6a<*v3VJLxdY?zoU@B+vtA=r|BLKI#w{^<!Vp~#G%P80d+O<2zx~kyaRxah) zXQ9sAuGE3IA5xjI(jZX$d$K&(rLlYhT0MmqyG7SF_NG9$F6e$SK^mB|zu|*ow(@1# zfxA#BI3tx%#a3fcaP-=vJ|xciF*`sWOZ)_9V@_2-Ipc{TSGvs(h_J>z=MrIEfbed@ z=cDX3d-DOqt}eYc>Ho9pDdCAUDcGj{+gu`&n0>iN25;|7!9`cyEd1LGr#pZRXWedK z#9ED9s`Y-Ki8Q_uJ2a}*w{U|Z07L!9!RnaWo^Wu@&bj;X5i!ofekbnc7OJQCM`{AF zue;{!iE2FXRJ#*;yNk1)V9Nbfbi7sJ7wt}l5(A$6V3$gbRfJ1vw5ErTKFlrAp~*R& zNJ-#~D@<FPjCVE_z(Me#)}d~v6-5~W`h-*7U9c!t4Ra{QsZ``*Y&t^RL*K{Joy$OG znUl|l({`RF)I*-TvbVo*tJtiDn!a8I8-!5s@O|IdLz+mJ2V83VhI;@srG?}%siy&< zC&FYEx!WhtuoNK9+JRL?FWH`a0>N7J3^)6J$8~9Odz$dm6%#u@IU|LVpG=DAE{a+D zU-QunE*%m~G;+OVyu28j*4a7$OqOFU38~+W*uY=!5y!}a{x3M<!7+)%)F%r7j>0Q( zMCuE!c_Iuk|0jJiTHDa@R3#gE9LR}=1vFnNNa&U@aH3z{0%3kY2s4d9-66-={vey8 zV(wX(3;%;9UdmGlAwmiNmJv%E3__~tos0dk26Pi-p>Pmko;sa@QyqUXvm25B_=2$* zgd#^y6LYKpgu0^Cs<zC<{Qb`kVjHTI5w(oKmRvMKA(z6R{|Ai#@kRUH(!ezOh-g%> zyz&2|5%PRQPUT`><2wv1SK!xCBZfEjjYhFQVF9wn2XK5_O%5LTIz;6a-d1cr8HZuL z=$o`f!MW6YsKTSj+9ZKu&Eq8q)cc1(L$)fcgT+%XN5di~DT*DjP%r)=m(cJ^$P_6~ zzhZ%lS-ZU%=?6WgL!7q=w|Q?)hOi#(P-RmZ=f@V19<<{2J{d?2=;DLW0t;I8D9=kV zwzC5_L-fzWwmDr@baQw%5)3;=9Uq%EP%?nZC6sU7yt&h|itLzETS%eP^8-KpY3aaj zSSDVa0Ys9r9VU!_wh~gwZJoERNJ~Dn7rNE#R_B6l{lRn-jrCnR2BX_;cvit_TOyq% zu#reaAI#~S_OAi8WMgnx9J>XmZZ_}viYfmtcZwjfHPf6E*>V>v1{lY`Mb@J`c;%+2 zd3Ae)@Q*FCd=r&5xW}(J#(C!QbSHK$%)RlPO9?4o<vk#g;tH$FY$P3i4c>_zOW#T8 z19ep6*f%Rc7632G^abk3<{OoS+GL-6xibM9A>ge*Kwf8~w8F}58>sPCnv^m0MS<qc z{ohVwQ&~R^-VF}8NA`98yF-mSAtQzxS>Yo@f%;_@t2Yuy_>VcmCO|nmQKx0N0ALW5 zgW9DnEnkKo_K$8Nha68o=D)a!-t9p6eAyTDxw&c7bm8-J^0{1UO5Q-&S1~fm@+dwu z5(OZV-R<hcF+4xs!Ms5~IP&98vOwd=W2<l(CHcSL_4xKmUVOIJrTdSxr!u1;UQv^u zG!|30ThB9IAs702QQw#{0@0>11ZQTGd}1prD__2Zka(s}1PN*aI+Qy6d2Obj3Amu+ z6tNO!9*U$yuO5mc0Xgu_!1xgEKVyjL^$=@S^u?`oZ?szr;1`1+-e%YDcKKTNv-MHP z)@E!SF_%%zyr;tWqWyGIPtyoOQbsHbz8rqUv4~Abh9wA2MDZbFf-Niy3@J^uK9x$O zDZmL<=V)U0^gE~xh0uV_6YzmkU8}#W<wUG5lHvj1ajseVJS^>0wz>gPCGn3bF^&$@ zKc>WNpduk=@MJw4Ut5l9jz6ammn4Y&Q%)=&^`l?)S7$p3j3)pyX89uV1h`R_Ogr-) zM$bbT8-=eeST2V@f_L5)nZ*28=07II)&PwsF4x31i2#a9uFc;<^mYeL?BS%v!S0q+ z$A;vfZdrxkJWqPfpoGJ0AdpT%r<bQ=Ko|~-+3EVe`G#UPz)F=H1L*T%za!^U`GXqS zN7wz+lotaLAO!^+lcoX^!+Tm!+WEd$C$+)qcb3cn?Q2kR5O<vR90rtN8J;TJy64{J ziL-t@Uh(5E;!fZko`q3!XDt0=&0=ZV2NPeX{Ad{Rs=?+&-MGI2BmU*Zkn4#avMdmr z@^^=#a)CK^lq?+-fg$hvzrICrjDYik8=p^N|EvTWxiaO;{k524ee}K+z%8ihA3qNt zcPTpVf?Xxu$Ia(!EMUqxZ_VVq3W`P8m%<fc7}FLr7=t1oZ17xA*;O2yKMR>3yCjGH zlR4h+#(V;(Z0V6bi2Vp^B()zDob;sR^Yh=A|IRYK|7XGq;NdAoFQ;KY2^NO0LpTu| zgbX>YL=TSkEmIek(yW<hkC!qSsJrCA$uSrOn4Uu)kMLE!a%IiF`=8TUhdEzz$Xt$m z_lA-3xV<?W!H0-i9)YTZG|4i{iM;89UuKNtXevWk6RLrDjJ4Q@o`#$r`|@03dVZ-Z ziclQsQxg*3UUbufhmQ-^JG<bq!_sG;Arc*Z141J}<-&eN*xw$Gwk2(zGZCW{;NZzJ zq1D=im)iSeNEHquDJGAAW^MqzIqCZjigy2C>a(8i|McK75^V;$KJZeO%sBe*4Ms(Y z{SI&m*u+vy{T^h62|YbL7q57^oHGPF$-s8p7p+=C+b&J5Q_!;4N6ldmED<(RF{cQ) zd3fkpmfyU2^H`Q7xmjxVtOpdSPpm_=j;@;?5VejdIe0Gn1;KAerxC#S+!J>}9o>3+ z1^N75i|GZJYUZ%5yOwG!_Mg9^+DHC=ALif>GrplJslKJ8jNqHWrq8kZkU$qU-udFQ zg9N=kI~!u4!<!B9?$07qjPG`J60sV{!dDB@#(nhK3j0eOZ2%nsieDEIcHM{bp-XE6 zr@zTC_QTOPG;c%?5A$K@??50JQlG*RhM~?3jJ|nBc5Fg8p{~ivi&w8BHl*6iG4!lF zI2Kcjdf+|BN0q|ZLGY{*Vv>YN-T}Ia-r6qDIaj2BQtk7@s@M-@h`^E6r|~W#XvhMQ zuQ~Tpfg?#Kpr74U`b-9E%Ml0(7}^B1tR(rbArdR`5g&4dar-F!oG=AI8uL+R$M*@k zN)SYS6Oha7%Y=P|(WV3FTC?=&h#n&R;W-n~T(v2OBll480ovD4=H;G7pX#Y@!!H#^ z-k}dgU=Bdx*9SY-a}2+FO9L6s9CV@g`GVI7LZQaZ<Ie<j6xeJ+&w&jOi|zPy^!b-6 z3D@=W6%?0#^*e@ZKmLXdd<|$Onrw;%j$nt6K`^6{3cWKg4Fw5;GnWGkh<>kLD#Xz! z;#8*UUDL~<Dy-Pt1F3&}<RIjtk9t{(Z%x!!f)j2Y`@R^jCzgQ}_BMH8j)6O-JIagQ zAN>ximng(CJ};s$S1==obSrG<&@maDZBOQFbf8c64aLqb_1K|0ldSaz{4#BSK=Zq4 zgs+1-Q_n#A2YBrb=n6gY_lt@2!O^)WhL_Q|YQEbjVDIh~iiKbbkZ4%W;?)E{_f26> z2Ntx)f)YCl%xnH#>B87u9JK??f#|Dm$Rdxw;0vXOkeJABFq?=8JR3(r&S0IqL){Ym z#}^z-G~jmrcH<NERB@&8j#CJnK<3>{Y#Q(aG)Xb8I}=Wexq446L20?}c+S<k(2oee zxYE_*KQ^aQESpgimI8R(F$^EbaW!fC;8T(VwdmazNnWtZPV2i<IU@H|dz1Qkru83W z6ZeR)lIPFg{MdoDQ##+6R$L?|ZB8a?QWe-)`%h^T<>()x^5E^UVfzxVwj`;GUaGNg zwyBj;YF%fP`*M*;<Aa-WsLk`?%8@%mTO1bNt_}6-TA^+B!4=w;Sw|+Dwv4WQa8o%! zM%8vrUWIP+m&VF{8rl<Gb?o<PTfZHA-9Y4dh5cxIAo~i}&vajL?`$Y6(CtfzyGh-< z-t~iC=ezW~x_Xz-O>W&VUqL}QxG++3yrAHoip;WwJM0S&9m_geqGz(p_NR>3m8#qq z@he4+?zQb|mrZaqj+l2<sWVKj{&M>6<QwzKf=`x3By~_}0Q@I$7FoyRXG6`NJyEmw z2StGtj3-ZLLr6)2Yqa^s$ZwHm#ilXVMw(yVy<6{otZ7MYMxgyX`>yYTal<?vwpe$G zUaQdXXf1DXNmQI!(2-DY!8+3roNA?GJAY?!L%Qy0Rr^n4(+zmj>p$-cA%4$h90J{D zz}()Ty9@YN^sJ;jF1*7iMi(1qH6Oc+a`|9U0arciw=YLLY0qq|eWu|)KBiSDMkG+a zq_QDmt8T}&7Tr3*PhCZ+ffo;Jg;ra4>e<&_m-}`z^}-$z3-ie4{2G;-LLF&Vo6wQ- zHMS)-rd{oYQX*Y?_J^}3S+(s-R=xgl)`7&U!UuKE30v&&EZ>R)#A0c@h{?S*iDo<r zcZ&Ma3~L)l)MEXGv<^HsZrmt}610;>t+cgq#g1AUaaqxwSL9gg1_=+P6lZv{-j7YM zu`jW7kg9fwX^-FMsvB9IlM)~7Ssxiz)s$OYuG*!3`O1}e&y!_Y_CG}L)@Nme8W)bQ zdHL(4o2O^NVUJ~bP~EUn@H6HBYt&M`$0V1q7Q#U<r0&xapexee>-BaT@L|79v_tVm z(@rg^wWoe+*tJ|ljg#-C=EkL~9^c-P5U8Y>Vpru=ZSf*KMNS6RZ@j9c6&mXEVNrkT zRS#WMcXXw+f|Q5aU^a-AqEZL`r-zAUeBa&!R!N6FhH`Xv#0m!lDz>|aHGK(;(@|m_ zG|G84BgLU!sQ9U)mAPhvY+(EL$PO33L<hUNn)ri($8(Mhw#tXjbG?>`7{}(Q5}n$h zdybldDqXdQL*v8WHd8v%t60nOORc5trmSlf!raG*=7C`aPoH?d87XhnYfgwX-$G5x zHN`0|4G(e?mCt=kN})qQZ<?Nnoy2_j@Zlnj!JWk!4lLRE^6ekOzU6jJ2raa?*(NSt z{9Y#Ye4?<8k!|NN2f-50=7i&!GOCwdBU_yF7nj*)S?%8=%|gR56x*IUChAz`j}wiq zd$!O$IyZSq#FW4H3)VqK(X)BvLq`r42*Z;1*-+oWfVwuJXyHJ)s>|8Yc3Rqv1{!gd zvVj`{y*g}EFQ)6Y)`d9wDtTEXc*SMK%oh@E4Ngw~)k*?lTYg;Js)duT$aS_97d_VS zh|Z0$iul+FZOS7b6NL+6*Sh45Erj3D#h$Fp{*3*xv){70P;w@(eZ#%*mTv)}38zGD zSE&deO*-Pws!i6LSopH@Y*_gljW`d>f{9&K-<8hqab&q1PZmnet;$i7`|&xf%&T*j zVtZN4#oDJ&p}StG!);Gdq?w+ON63+!r_&@OzXl-4uWqvUJObv_L!m=NnGq_88~yw+ zVNPj-elB^baFG+AI%zRg?b?ECUxeySNR=-Oue~ei8L8!A(D8OBV5psfV%vXn0Nn#> zo&7vU`Fx5hlIqIpx@>be=UGLoe0k&X@^sb9(?%*SQ#}6BCcWMNRl^3UPR3-Icw)*L zAv^tBL;9zfSbKnn1iRTTIyEI#gL~yKyivSHf$@ScXVmzRZy6BWmyC4f#;*ytk6cXA zTUx`DC>335x309qplfM;o}<HL@t9xE51f;0-^O&TGFo@C)Yzt7zG$mgr%Ggl<EATP zy=ohDk{wzeImXAk+OoT_FP1E;xLL-6i@i1|y76R+Iv3c~Ye}8=5A4)-KmIzs*hgx~ zv8;Bh_T{}Ml$0_NG&mh5+SRF}RQ{qC@`kuK-ZCpn&e9cfw2hlq)Kcs@s<^ng*2gf~ zJq2bb7{s)_KmOXdYrCeU_~TAxDZ8DO+PYT_ma!hz2AvZYh7k{T9j%F7Q?IVGw@=J1 z96xQ^w6b8Ep}vKtiNdCz>Q%4C$H#}2vT81zn{2hWdf8ImA-*Ng=3T@gg?v<6`Xn#| z`-ISm0mieR#%xaBkdrv2ZCxYt)nH-gPTq!M$99in&0Vd7mCl1wz2)ZRrCMXzd97`w z=+V8AWmig8mw046_0zX5iAbI9FrU@&UCK}Cm!QKGMf2Lh7Q8tcg~@N=uOctK2eFN< zG)lF@sH6Z=TZwl)FKB2vaHL`zkstZZTEK*E6)5HTU`gv-54R1CxA2T=%+x9_y?6EE zsgW;h@)b4Wq#yv5y=W2LM!X5+FnE1nG)L=??=%Dr=wl)D5A-TWzjMsQg!3be(0yCM zCkyC??obHF>t<)p;y9O%=?*SW$>S;S=!H$z;yv}DHH$;(ZFe-f4T1!L-HBUe5^U9> zug9(*S6E7^!PMVDRzQeoNZajrS?x}@awulL`ATq^gdSz$wA~VL!BeeHITJzBhHa=F zs=03y&zkh~lZ7`_d0FVThqh{^hw{r;2C?+wNi}x|{P1BJlJP{Gg3@vDH{~&5DDn&r z1h2%Ajz&P83I%P#)OZ7R{fc=kOKXnj_6&kDsR<uvvx(rXhJpl#Tz3<)Bsq*SicP2B zLmfwAjp971U?ntqEAAa(sm*akFClzbFmC%LyeC@(^$|*s2&lK7)04z$urP}t_WGAq z(T0dLDFb^rJYPo47#{u-rOqs~TRDbqN>SyOlyeovL+WsVQKkOyN4?g=^kO}#>2U%% zcWb8!<A7+TBm8XV0V`s%8+1t!{ThkxcsN*^V<ne92}&aivjdSRMtL?-W6l<tK6yTT zmEaq;by`{B7R2F@25MtG|2T!vvVAVnk5xkv67>jS+OWD&+1KHTp4;*`>1oLEN5ckc zB3#5V*L8~@vp?0NAKc)4m6P2J^L(peA(p9`?`ZtMSdN72lf*tkxM+VS#LGzwq$%&k znq9O%GXi_XKy?origILMzTt4ud0XuBPs`b4gw3I3oy~D-vhEK_AZAx$LDb-NuC9&! z9SW*)j#x{MC?6iNYdrkZkldiYRO8;$*%!7;JMZQb;M>9z$s=`ol-j}@Q>XIXT{Zng z?6VWvr)OW^5<bIa+nC*vzSqyc8YU4`v}$jHR;o|c{#zSW6RmD~8mgu)$+t_XF?*Df z;+|fzET*vDsx2@ydeQGww~63JIa|PR@0<SG7letef~HtJ${Ww|T)j^Nn9P{`=mw_4 zYIr9&q?J$uYd3mF6gmb(r36)F#9MzJKl|b6QjQ6+{sT0?28<cw<%$DP?8{r1$VL#s zpE`(6rKWDjLqBO+p<BE^fs3oUC*>s;lxeV_X_LEi8!MStno@ol+}nP|<yG0A?VKLY ztcCDbVrU&PU+xNkz`->S2^~wMPhE(FNB6H&UWlQqg1@hMvWv}2Xq2B8xoSC>>1_Wg zj^1%W3yEDuS?(v!CG4j%E&zXTSf~;};Lgn<M%5N~vucz>wp<ADvEF}d9LL3+U<?IX z%u?Cu$maa7Zot6viIz|hDliCN26DM|ekLeJue_Ao><4%U!Q;*ddt4!mktIWoENwoD zxDQUW-m@gp#hG&v52F<}lPtSc7_;6mAhp!=I#Nc_6$2+%X3<*BXy<hkV8CF_Lp+HX z-wC*XpC1<M*an`fgVHqwD>iFAP%$>0!&!rPe}%2^ih5Vk?W!C;^XB%aAz2g0Lmz6w zws9OW;S(rXRfEi>bemLiRQ7?I;a&$mKYF@APM8KebfquN9`?V<lYW$A5LfFbX%g^H zVY6<g5fcP{)7~+TzJiO;52qXVFh{1FxBv8_9<rE<{9A+YWKHRP%<cY*p6w3KnH=6d z?`o~OzFFwlD0`(xF6$r9%h<Xvjnhaz_z_7kPwWpakYckevj|jxpu^dv#6i{}t0YmK zs6*=B1@*-{bSXpU#0q>7ygQIVX?YP@xu`D@lwY;l#zbUt#g!l5RJ-VgC5=o>bRUh> z;yNDo_9dW}-@0tgqbc}Q#ijxAE{nNvd<G|phH`n9xgeJU8WZSMZd<)V5gX7mAW$Ur zO)Z2-P4AtWmS2;YscW#q@Yqw4kTdIN9vjDTIVTu7u%l$fT`Vj{TS2H)mz$Dv^Q1?~ zO;R!L6W^IUg>}WR@x({t13)X4zjhJ1E{ULz1rxWUUTk&FJ&*VbjnRh0X~FiC4i=3h zxa}*XQCVuQx%3xbh67iOChP>pL4<iS>#q=)w}g*^c_ZgY5S=>|TdYT@m~{XWuExGg zu{;SvK0<HpcVazayWqr6p9?(3QLJY|p32KD1z#QI9R^4d@8RJ=AVur2UZB^-8$iN4 z4XOyCXO00I(N+JvCjHOPkYY>OGK1qn{)R6JD5g7*sEVZuPcQzL1GC(Ic-#zP@8sv4 zUN9_UM73-@ro1lSt#D|V%k$;tTxuNW!;vJCh2A|oak>iPv@|$#7VQ_PB*{UbZSWA+ zU`t$P_-9PYF{EOQj{dzUndyClOOt3|+g3$IjCejsoPO+BMA(nSgpMs7A$K|yU3td2 zEy8@X2E<9Z+qvSo3V<al$gRMs9=?>?T#Gj@fd*60N_<4f36wW~$luh^#(J=Q_R|;J zQiGmxUd|{9)VZ~!<k1{9FPJ=m{e>@6Bw-(1cquMQS`rN;ULvK+=1}Gq#BsYX-y+WG zs6cjg(_IBQ{;Q1%N3YJA#j$3fAM+^s_MWRR@wZ)(Z6@G)j_@0KVB2FSzq~;+=&8pa zhN{M0$e}40(M=Zb!*hsfFwzRld#@U>X_&5+x1QrcjgmlCLH~36li45p`UoUZV_X9< zuT;%~S#7WMqG{=y#k>HBhcO^h*V2nuaih*5td>p!PYN38zaZ@6_p1;CYKLz2Pk8s$ z{4c;lJ~=iAmSK?Fg2xCtqdtn~_cEvjvFkZnA8P1gxlfKePT_6c2YRg|o38o~NCYA~ zUJB!EOi>rnv*NWoDC&CbjWjMdl1c*B#bp0pSbY*UIb@^`raTE9RKe!qQ-LIi;(QN5 z8b2@>^uoyS_W|X^%;=uepgu?gtzp3(IutS{0wt2=)D<`q)5#ThME6-a4l67fe;`D2 zT7==^+tYf=n|P-3q}vVag^Yh#h%|zgmhGNc?0mpZk@_I9h?t80U4qckVFYAO*4T=D z>mJI}o*55&l4|Ox#A#a<Xuek*ecKhUff<KPRiThWkB4u<c^lf7dr5^mBpU2rUcx=D z-k;%Ad_)3&63-dVyZDRz8}^9E*&ID+#w;M^h9%oKVxxfqUt)~DX=!)R()Q7gs@PAX zJh1<Of7a3*app9_8IkBr8efqTh?RX98qVYswhU(m-=V1~SC^C%jYmm(`VqO8n!YZR z%M?m(l_I*zsBG7Z$^zR!ioH(-;~Hm^aUh@mrQ|60wNO5-;B&Bo<3uy2KLDrR>Qv&O zyy2jDh20MmM#EO*MlB8Je7%mu1K{N4gAH;qEX;;SXnd@f#JhI7MriF_$^MrbAI#gf z<&YkiD=sELtD>FJY<?^mbqLrEZY(9N%Q}dzr)FtzHiX_mr=hOr&T<TOJ-nW@9O32T zFAvoB>@!))S+E#iY53Mnb~M8vaBB;vHeMqku|wo606296Fy_406%$hrdQgc0O468{ zlKczMy&iBo!lSgYeRIIJ-G?7~2A$^2bz&wjfbDV?D`GnOj6--6wdnST|GMj(`Va7e z<L-()Q^=VzCg>8Moy0u?%rRhixE>GFBw{ZM1beCNpN->4bD-xCdSeD-BF~e2dAX63 z!b?1D&a(2XD+2bFZs5nHj$HzrRPwcjyrs_suC`=NIKP38w>>sAVmWsR*V+%gsKtJU zcO^VxpI1;P0X%bgyM67wJ9TlX_+N#~8I4*M*z+0tU}5gC>mQNJ2s`BdJOWzJgHhIG zxxBfXJ^u8<9XXdTuBft?!8nLOO<wZ~p&ynEF?w!!)}?Q_dKp;B(K-w}3F|;M#OLqd zNQ4n_)zd@H(+;SXX9}FSQeh$yi|EHJ26`oOD^u17)Uo^g2KNZ8MPrmM%r_)TAfQLJ zx8E)E_%Y|@@K6Cz8+wdcTeL+;E20lpBxMWU&aeTW$^O2?fluar5fOEOKjqY(sHg75 z>(|r6Ce*{i*Mzy>#<Sa)bHLityO2uQ5{y2X7W55}e7^+@epS<QYjR`c|5HTm>jGH4 z>&H5x!@8R@o$iw$T7yCKnt9qns#fVORY)LGzyem<kbsM5yB`F>ZM~tBNBGJf%Gfc{ zow(<x2al+dG^z#0AGMXt>-epnUv4YH(GC@9mdaGoKkkHlrc)5;hSzVh*w)<rIcbMN z*4#3t8abN~I4(5B<Bk9Nuya6}b7SzeY|PW(*NvU@VmsEe%=bKtE#*H}G~d&F824q1 zLh#F>94#+~_i%XFKDWCfY~shx0CAKW^3#;Sl~}loEkBFgKrhnurj7xip4Z`GTz;&k ziR|L5+xK8@Vq6`No3HzPaLKEu1v^qqXK*Q)R$B(&F8c8PDtkAvE=aqmJRp9|sZa66 zJ+K<bjX##Ncc3N$&#QaCMI5gN;)^`#eg=m*d+Zm%71NEpnKPfwRbkKI;9Z7{BbaK8 zKxuXJ9>YMHN?RQ4IXP65BOjlLEdvVsMTB8LWj&yuZh7q?Vxm_PvP;Bw?Bi%9Ng%&x zR>FfJ>^+&F@N_bn7#byJ!VdFv_EC{{0eRYIvRR8sbp@F!KpdG}+vtV@MVgpnfJ0qP z9%vGg8H5VcznhWlLfI-soO|=oEAv7uu(T6(x)A&Gh`2S&e%l<E9a@hJwru5+8h3`; z;HcqSKhry9rIGRXP+E)gRtCUU1WgXSpqHh+U(?I(49+=?8v=WCJOwGGkvN*k324!S zzdvCPshwiUZVk%E=|c1#4H9aS38EWB-eLokPnBVa8~V&Fmy7Wb4U1@PdxVpnKeu>i z9J*5JpJQ}YySpuRl$;Kc*ZH;bRYUrfJbJvDa$aH^frJZ7p0N8p*P`KkwNfXV2#}e> z*3vb7L(cQikueW5(GZ}xxSaVsLi12k0TedF?7=403;=JbkJ2I7jY~lNof_}V2$C|d z!?wjY=5Y?ULXM;cAKzMtWh%V7V@OmsC#g^sw7)-2OZ;suJQu-P*8N_VBhOx-8)`6F zAs0v@>>yZ7G;UDq&F+M7;1y6ZNYo(0iGKyl$#6tP@84}=p9%vH;+#zcf2X54w_npj z=*&$~DzRr^#e!q3@M@jm+P5fMnT*kx`qHRrxNN7^g}-t>oDVMPG1zXgp8bpkC?a}! z%M`q2vE^iRCk5wpra$oJoCW0LWYR4e(pPPLA653X$uhbSn>c~*Vts8-7GWef-HBae zW3Yogxq$N&V2S7O#XE@x^)&g7MQs@dcaChN94wt@e(nV3@buebO#im_(B8rDHtp(< zgn`zJ=_%ilMxuD|kQSLAAQ6PB7RdjluVVoiTdAYA(h~q;BbZf3q`sW0|GL9?uHVP! zL?h7O)?fl*;<hDniWuYeu|1{$n#WXv?0}l$tsKen2{=U6_~%lrGa<LSfn0bPOl)k+ z=!oPP+~k;mWPH^o8O~rkJBjGt9?8#9wTh`(0CAtG?r>m}{3{yfwY9qelTRBAde)d7 zBAlUe2z+UJ!sB)9jY~tDnPT}jax#PT1;lPGntjP#Gv0wX&QjH@L2GeB*MC2|mqkpi zA-eRr1SOBdN?tpv;da)qb?`V>*?KbV_7yG<ZZjR7MPV+I`Xt-j2ZZyiYiMLElD7!2 z<*mtF@k9#eTAy{{oJS`*FJ)!q@nO?KgiO#2GsvH%z*0Bh#Fl^osAT+2mQ7hFDBiGM za4(bSQFFQ%YY{3~^~%cRvFr^q?R5t^@*Us%7}$MBYSKuImOp_uH?PkoiYEGomN(%$ z2A^68v_yBwMh0i#aie1Y78G|w-JnZ^&R5a*9Y2aNFY_MjThHIW-<9KGeMgPDUID+A z)VyWK6i4>Si{1hnRrOe18{LR~0*qk4Zv`*KDxj-1i3@Nf>jpg>L8HBX8;~kpHgxLW z@Py5F^^jLk^fG&jlO|67D6RCc&}gx8&!r%Im&Jp^b1&$|h-R#xk$I!MIO;nc$!KeG z2BjQzdSwRARfR#-A#3UsFVfdd0QeD2eY=$-S)I|P4IMI>crvjZL?!J{lGD&m#IEY) z(Re;MOT>f!wox_I@VG!_GmhhYy@o`xXB5>}bT5aDejc6p6ryv;UME4vo`R6FI_2{; zf{c=Yu()-t!2@hznjD0m+;H8BY=kv=MEKL+46w1e$u5PnX5({)oOx8`*i78CE(b$= zSpq$mK#b+<vzkUWO3~*=wvF-@w1;@txPAm6?d@~&dMI+&!v4#_olaq>a4=(pIteV3 z0m4Mrw{XlibHv?K6rZFd=p>hkT5b+N-&U{xgmK9BJs_BjxH`_Pk6H^Cizs~iX6Vrk zaesKhtN<p-n$>|VURwURu6InabC@Q1jC33)fW-_*taxtaQ@Y2Chq&K#@gTxwf0hR) zs6n*0e=(bs9+J(5pjeyq-gz9qYyy2OZD+b~c_P0HQAsjf*b<AA*TJGqL;fZ|cwg&5 zsp@`ub%<^PxgoQ}+?{^&ZO?BZGQ^@f)zoI&N@B9zR1|qBUrY`6x{gKVK6-u1U#~^i zhK5|Iz(sMx!5XydZZXEANgm?i9V8R}hVp<yQU&iT-PotE4VEJ7-RYs^r`Es&mF2X_ zx75uL{6h7dL6ZuTzqNM`1u%2QYjDZjv6~}fos4jM<6ekQc!oT%#tgwI2UdLOFtCAR zalzmHEA?KUbodh=!j}09uk!qaVV`y)e!ErKKQ3kfg5=J0A&S<oA;a;=_emU~l2RTg zh8ofYd><=g<)nYB$D#HOj|}-EUAvbiZ3bVy=@>hR{jt~v5Mw{%BdhVFyz>U+v+Rwk zj-A~&)YsJ}^Cib(%dcogLb?v&Z~LI6V&m~eM49^gwd}NYcwf@iVl&`D&b|Q}>{a}7 zg|mqsN=PoV5>^;2tBz0cFTBIiTRRbZ3?5%0V9(K+7DJcP^PGD^_v6HDs1+0T{l{@O zyrmOi`DV)|Ytd<U5hbkLcjj=ZNCBBPFn&ZgFr2)58HHm7(m2;5&Vq;dEwhWF<6EBs z6TGNw<!hP|jwNWTqmYX;TxQ!lX)THJtyVI|e>o2fdek7Lz7Y`k>BD7Y$J_)hGHTO` zOD^`rR4k{NTQrR?fb?_e*yj%Hk4Zy~Xd6$@m=kR=lZ9>teM$Q{T?F`_eLWLjmnDai zNjz@?Nrvlt;+lQm$o!eNaR`fskbO#AZ$ZL;xTp>LVFO{7kRBN?V<=LSf2mU+aa2nT zeKe83#6;o+j+E4aJf>>xAd&-O>OlQ%;(^A0r!ey~Z*^32L6}9=7ng~OoflLS#1(5V zwA^+m4os}8t8=N?7z_ko7!rW9`9Gnr-)_^OT|xw+mhp6FVW5$F`s{q^g+$2ak4z$F zz>JEDI-Y*JaExecl5&xyLrwA1XV1ojDtv!cyRoq4eakiZ>IpkpK1v-ITN?b76pq&X zDl+XV^)L31t`i+DRbyX~TT~qyIj`EfST&)F-=_52v@<QU+;q>+&%q_qNoX9LmTCrx zhrP24f6>Ykp|L3`n<BqPM9VAI-HsWDt}-tE#vh|#Uo6|H+VRmqaja~f$;!y@@lqG1 z+U51Eg}?cWw1;IE7CH(UN7lAQz0vcmZ7@FXx2$#Dn+>lTR*qA`V8L7gmWAMMmBoa; zhHtM}8B9kT3d4`(>-a2RFwWREU8lAskag>Lfti=gW0RVw=7iLk8dmK)Q9E_GXWOv0 z+A>;Uw#)jCTG6g`w)sj{TWl+WV`g}E+!yzZxYfI0>n9onSuk_A5^-ibd0w`&vcFUe z&)uo>So!dsEjKb7aL)-zG(spydc}@r3fiYOCrGjC0^`~X+k>Az4?E7fZXYA6dc5d* zv1(_7YVkZ{`>lCRPJ<Hz9a`Sh3U_r_)xMFkTW4EV*r}D}rBLMx&FX^}+1JZD$cy|s z5i>@w)-tB!xrTv#%@?to4mGbNt07VRZT}JIHoIy34(q9YOj}^?`bKraz+8*&#K_O7 z;LzYDHjjBK=0x<f$T7w?$vWbb96BwdrR^KDI+Y`v76o2aR;U|Y8_jPkt=oA~BSF5f zGc#--G-%46H!d`(`f{Op<PrDrDqZn<Z9iHHYHT*=+178YEmmrJw?j)yzA=Aip@524 zBhO>H`QW@BX}wCnK4?9stFb?S?zep-<2lZYRcTNhrPOYGPd6}8sK8t!vfe{USk>lB zR^mCuhGDf2m0C`P6|$;@9K-8VjyEZ_{|qbZY>pXezRaz2SQksMxLvoQXoqNrPlLvY z6=nW<4Xd-fJVNyV4GvFjqR;&hKo<z66$j%vquq7^^~Km>wv~Hggrw}>^2aO0bUkdj zVceN;ynL(Yvd4`qrC+CNy}FVeu4o2~w^VG~TzjH%aGbJ3$-P>w7?!-LS9_kgZnLqj zFn?6li*LaY?C)yk?~=Cv8ZUKWaKX`JG3U-{wP*Bx@s6!IJW1-Oh)Us?FSpi+{ywJq z{h3%lOZhi;a|^>pC1(hg>{eaOnK{=nGcq!+%Zi7kS6LJb*H~LdvV?0cr#80Ne)hAs zgSOPv^FmK_gtptf-QXanSo>BrVVK^pjxbi!IUT)kIyLsHk=2dH=W~zPtd0EfM0RA2 z=%943rq6zH5%z5gwc)9Dk_DYFBF=qlsE|r2lL95bbV4pYH*#^-FL93}4THMmwncqA z+w$5vz-?nTQC~<UF{y5Nc%Je3cD)Pi)QqfC)*I*0a5FW|HP2MJ<5G#Dcy&wBImdwJ z27_gD6s=MfYGrDyG#&>8#P5=3dB@h(9fc{OaeGAD`HLfB>Rb0@xo4aTADnKG;dYFG zonT(gfXt+GANHGl@4n3DMC|tgw@<~uh4vleP9O3aW4NL5?Jz6i!X(n#reZcTd^{j^ z%~db!cKPDy=EHHJ4!NNT8l9{d!K}`@!26C7wKc40!T58Xxl(b4diCYUKTi`iRdDNc zldrif+QF=GELqlU_2z4vV^rO^+N@g+$MT-ZSZ$KFn^`#?tG-}})_4WPkPWp*dl%{D zeJH2h6GT}Z<L-d2Z%Unh$KUct8U51u6eFkG`OGzeRc%?gbw+h7OEo^VGcOfpjF=hN z#=AP?B*}OQb$GeOR1Z<TsKg3Zy5!YvsU-8*v_&wz#`;Y~aZ|47_?XUb{QR%Wx)vOX zZuLE0k#8Aqcc{MI;qZ7B(TRb>*gDkdEEo}!%rc1*W@$8YIVVhuQz2IDLS9eS;lk0P ztn8YE+l4LHO)CQUmztHGN^33dsvTR*49y#3S=e|stTsU?P^o=ROq*ft$(pw6u;NRi zBF4vAhUc0C2d57{X6%r?@?5iS69Cs>#b2+Ee;VsGs<1O4HM+Aik~P&$(Rl1r+%G1L zJcU_eT14U5+U;KtNZaA`?z8ce=v}kDF<)&BGp5v{*k80c#5AV8KvslRs_c<dB&$$y z(=>TrMeQ@!uC=ybPjxAZG$tu06pMB(GEHY?6#*PPJzk%0d?lvTyeP4&(J_8cwQWOC z%E4C+N?r9zj(!R(uY{&1i^%E_rTHy$sw~~Zf6i5cqxXppVY6)PWt4;9G2V&o6tN*> z;9HkB{rZIPnH+5IF=Jc2x+)wJ&AggBhDF#{8R{zV-BX&X7!f0?RC_Bc!KSdeGh4L7 z{&=2k`o`M8=#n#8E6WBqs}A!O&2uRHD&-qv`^z=m*e2;z@kZn0c5~;>ot)?Czf#)n zNy07}wzTRm1=Bv=^aU{s)aeNdfoI$HB?KP(N>8yX%F$t}b(C&ioA==O53+><Sx<E4 zjJVvZkBeI92{o)>nkrtnW!d{=enSxnNaS1p=V!o<t~2z}49FqhBFTD=2`E{sBH>8p z^8WZJwMp=$PnWk$!`rllceMe_$k>wGDoQp{eAYKEePZ|$f#PTQ5S>t~+TyE^NTK~( z)e0qI^bQ3^GT)Ztz=%7zPWB_OI+BZ@$k(gS;nc<T<fF;IQ>_;LXk-}MC|Y3G^eF1{ zIitRru!Q^O9gYT)tQr(R4<PDWBaR4ufrGMt2G9Nj>LJpqC~6jjfurXDy;qU$ws3%C z=Dincvo>35a`aM9#IA#dT%TF}+m4@1@2mHtwjUm`Kvlz-N~IIAP+|iWyCJ;k28#y_ zqq~lj4@<9xN6gx-g_A+=FR}Y3)>HZ>#7vOa?JxAGXS)>R7C(<+3gUiAW{y180yO{h zs|v;UR;pjZxe$q4(GL$C)>v;VPDP#ln6FXwgoXKHTya3@F@&_RX|GzdWz~kBbSIz6 zfeiM@vL%ae6Nw`5_F`@k<II5Fi@_;yy!U<L-+eN$yWar@=-yaYjvKk@i8~b(2c1IP zB|?#)UMrD6nc^vl`_tJ(|9S|>piy7Hm@w+~CzBYfjI-I7>u(wK`t~2VS`FPuy5i~w zUyRuqaEO6FOAeC~g)Y)8mayVHyY3lm-gV!#HvD*3=o*UQxuHzry9T>UvG7P=y2T_= zL;Oy0e|&v7HKWVJ*Qd6d_b^tiP1`Z?Cwt%EdPC7k*dv_#=Nl!5>fE*CIGJ(%2kZk2 zMyg_OBleC{0A!Q~Cn^(S19xLm%Gch!<DBqh=tl!hOAJLU=;pgKBz~e+z4^>}9(4&Y zu~%~ON;+qO>DN=X<{9ZEOVXPn-rXWAT>^wKQxwakv-w3dU?VRlhqy<y`wP?;u87~= zUHyLhkv+p8l@iE?yov2PxCoqdF@IxX@3pXq)wga$;l#sb9x!Rw`s@#0a<uy<4~Cc! z$-}$hG8?x?L~q1zD9JwFSaeNbYKe9whOh6SYCPuz7Li9an%$KBvJWmC!}~HUZJ+xi zPyAe7cqiDe6-U1z(+!CNxZ8{KNlS;qm(1iR31JeqLg877fj=4E>r-mm8N$m^m$zo% zK_y{O+Q=FpCQ1VM0)gzjs#Sr7j@Z;6K4%WcZj64+R$yH(ubtUAb>W7p7d}6jN5*pa zAVfu<+Cl~PEvPqm*E?$*6-xv@c4UT;RbJnFx5!S`*ute#B^Wr!%OlE#%|W1`z&2cC zybE*n-y*h@)le(5x(oCmdD5qn$^DRnQ<S%v!zLyZsi)M-1+OuuK&>zlEcd{yyiRR7 z=awYS`Nh7M0Xah$C(3AS2ENK~0!)js8$R8UWOg`k$oR0Y1bxK0F53(Sbt-BU2hkM} zm92jhNP@s}AUF2g-e9w{7txuMsinH-#yOLxPcD(FU1HHlc7qCIdcTKQf6y{KXCD*E ziC#a^3o9<5^lB$>8qs*y4M0vYy#nU-dTC@nq1RV?W1btThIM{551WWTm!qJ!(LHLN zAw8hj`}^HW(oG2#-Fo{1F-af~wq_>&f~2873YwngGIEsKhAoNO9M>k5knm*0OT_Fh zJz)drTZ{hPA<jRpNK6`ygqp}7i|6R*j|adBRtuepDdB{?hj-~&xbm-i8<ZE!g&=t8 zZo#;gD)u~(itik0%*JkGn~ldN^`&L(9rT*%`W`IDCfH?ECoz+_YzRoyr``rLB$AfX zo=#z-fD=@|Sa&9DC)Ya}^*TrvafB7-0+3_ZgSV`)sl5lZf#Z-L^D(NNV4S7K-FNNd zTumn?jc$&}NsS$emB42>alcJRmXnWzvg9#;=6ixEsP;I$a^fEWGaWT^MTd_g022w8 z;s7CnI%80Of=MaIx^5>YAn1xWnX}pKAqU@3a{fgIhyJY?C<*ev3I_T=epCO0N#L9S ziN3LaW?Dj!sZd9r@+dJ-n?h9{-wOK1brci2=P=yR$%TvlH-ckgMH=P%Pn+n{Z6H2v zNHDOT?e^ppM`3tNfVi!`>1>Csm*FTNw>E@ky@*o$oK%C+>c7#9LM3U)6g5or*24Wb z?;)aVmpL|wfIug>oyGdm^(;=aFqm}!o30KCOEEHwgM6zxg^|%!w~M{7)9;z@YARln zU`7J6G`e&BpoA9oP~P7Ja=DL)v%p_s?uBPxpBS=+(~hR2dat~sMYyP!(h(bstdT1b zoC%VIu3UH6GlzY&%y48hMEsXc)I2+#CmU-*aUsx{GVR+Ga&{{XlIb|s0d4K`7z+uw zAHsT~?*lx-#00tS8%|o0Ig7p*?7D>mN#85$<MKy(4O*WPjhun^pF3pBfPGX8^DeT< zsE=w6Wki`H2+10PRPr+00MY{e6Zc5KfDh{n=lmD3VUp5q2*C4?#XwuZ;`AEuY{qXt z)Q6OxnA%J@mIWKPO3_#%HV?Lu$~tdElsNTZPj7Z%4gvNSXVm}Zj_Ge9J(4V9?vLg> z8`@9M1j!ABki#dQA*YLUe|i?U%6`)O6ga`uS}1Z?h`L#UosU&eak!b5v#%S6>_Wjl zRRT#swVCxt@dNBabB6u^SlE!XNYblHLfr-{RXFQKH+0W=_=Hm%u~3$QmQ~HlJw$A( zPfdvU5)Re0POQyn;Tw=2N>|N}om?s?-$PNM-Qw!)IMyVCG7ht1;z?ZUU9$bhvu5PL zQ713n%)%R@%^eX_?p1eQ5_auV_v!ud1+~EICv-&igqctDsS9yE1SNFq`WxaYzwT(B z%B_nehcU}gPXTsIZWa`6r>Ci?lEN4X0=&d~HV*jy1oX2j5Z{X{$I(n5GD$3;BGvx9 zktpyXgq^xOF{BC>{(QWFE!NT~eEs?xdEch=Smaro<gW>sH0M-0scim8>we6sAbo;X z)=fkh4(t%GQ7MKb_Y4QaMka0>S&ZIWBH4BedMNWI5@1`N#rUtjXpoPc*&I@c`Z7#~ z!M>b#nSu>n#Sf(xqn)XGlR)wjl^~>!G=XVW)i%Ga(+$<I3{RU!hT8u~48{>Sp74*e ze`|S>)9O`*W}sBc<)Rb;hzvyaTltLcPkGzI^MBlac$LZ(RCJ=%&p+1;_^;shho4Cy zq#ppcw8Np)%f1-UeFA@ak&cwJXNU{?w_@`E*-}497!eI^Yj+SeL2CLqOZ9q^CY|(@ zMS7zvz*c*Cv#^6s17op0XrN1<*zGRBli;zVO}I(p3A#avm$>>2cEQ!9{$UVut6#s_ zU)QZ#zO9=9WRU4HBw{s2FPf7Zi>%5!r@gvA{ihe43!c7<Q7R&rJod(gFMnhrbeg`9 zMo|<t@W%ia4vwXlziIYxd`y%s-aiUAh}1RYIT?9>!JZQ>e$D9Y6<kblSdPuHKQRG& zuVUhW74G^jEQiW`E2o9WtOoxo*_S!7ndg{bT70C4x75-5?-tF!^=7im#R;oN_<3I# zeAj#Pk&hF2`(GG3#^tz4)QJ;!XU)0c{m@x?_N%em1qO<U{aPR8U3p`CuxoH7V}D{~ zMMO>X>3omso7H8i8podIr?Wad3rlXseAU8@l}X6=z2SEcw_z}dK;sJ>X#C5%OOL>+ zt33nr*$FkK2k=deb3B<BV&sAh<o0-fR2d?vx&<h&I%GSM9@_^GoU5u#q+=NW@f;e~ z=;rQWc*H6!P<FP0&(y<(x*Wl!bSgYSa*}TVKGzIbQsDNURzh=FQhImg%%7Qq*#rk0 zn)+ouoeP=c259-Gf3R8Ua&-7&ZUbTJP~Cgt!!n`^vHPb=QF`&oM4#Ubo8G@hbfp>B z_x?|K1j7&3FR#$Qf|EHZ=5!j4a?Hg;B#FCGB6F7Xd>O&|Mr-E{J`>7nbPUFIw>5Tu zhVsHY7$C)?F%-`xDm@?D($ezmSYkrLjh>rWM;-8jpKE^{Kt@Hox7zy!10@H`avova zUH~C;&zz43@b$$oJ}P$GVZ}#R%xWhHPl8!eWbY_#_XGo_p$nz*rB~bF=Np1L)SKp( zNV3H&IC8)9{+!1J90uey7797T-YvrdjG17WVICyGUhw<!H#e^F@#Eo`9_v2<c~`0V zNel6=G$Gz#oUg`9Mmx3)2svY|l9IA0d5EvL&m(fy9;pVY3a|=F6pcf}8u5w-!HQ(0 zdFFMmXl%Bo#}VBT>hI=fa~o8dbQ{j2V5YAgdmT<E;N5`p-DzQV+~fCFLF%HGg0WIC z+JYDXodOiK%q%P?j)N%-OlZmFZb?P~?4W+}d=_@~dEnhx#x~OWfePNZ_uB8XDmF{H z?7kl(`-)o%$Xfx<yo#(gNJ3Y~`6k>UW3>#faU~~9kh(}!HNu7$hNo5ZI4WYRnGe6% zZWAggn!j|RQxUhR8DenHWrMl4I+3SEwWSTm7ZRo*QOy|2v6@i_EYyO114HPw1Nh3z z+c&S0%=||Q*a6mVbRIE|q?eF+SCI^tC0t!`WxeXdDuo>!J+}S>0>J19y<4I~@N73m zGX@OE9h`n{_Kyih+{R{nN8=4t(?1cn)^@XXS_m61qlWleR-Hdgh<dN(_OKj71FTca z#^Z-^X;Y=PzOAio&fq<J_9XY}5_^t1wpc&6a}1#4KB~Fu*vJnDt0VQpgM~!$QedtA z?){QKD;m*lX`c$UyghPpXz!1!0$SvQ?Ncyf>vJ5m2wDT@zNjZ?k%rp77I#+^{rMrx z&%pXy@2O*7bT6vaj0pZnG!&|7_hLLXZ9aL92A&vd1_+;g8)upshVU++)rz?MQw-N& zJvv0%^b{Z`X`LnsOSV|2g@=#qMfdrxe3M2x>|eWJt#;%7_)YRylc4a~+SAX^lFcze zo8WoJUR#mvCr)=6hwq474>tQS?rgjDy-!=$n=bO<$jn3!tmh+aLudm(mh4u!MI7=_ z5ER`YW-c*Yi(CYS)J#c0;hJa6FBo8VK#5J;A?IaeviLwuY-YK+*~dHUmkbi}!*Mwh zR)5>66=Im=`wiF!p<yL<MR#+V;HHJFo_B|F2;B+!fJX0^3;q*!qHDP87tA8Ur0fYE zJ^tN4W2cx7tijw){WsovjAEr3+eA1E)E;27i9+Xz5N5X}2>0NrWc<AMMNZK1;~i$v zifqp5Z>py3p}RS%R|wsd8Qt?=zaNfy41B0LH5<3}c4JgOxbx3QR7I4Hb+g1h49U>^ zpV{Ot@}gNn`fMGbww762Tf47aL`X=>q9=FmEki=6prA_4`{9qtvIh#skwl1XgM-Q{ z`8f}3T0@|-wxcRxv@u+V83k;TwUpa7D^b6g%dko@exS0I+cgoktT0}~yCJ7$lhFnC zM2qEIVoV2++Rr7E)6c-Y@$}ePN=`=!x(8Bmb?Ot2vY3Qov?9LuaWfTmg1q06)pBD5 z+}v(nYPD8)=d<N<L-S0oF{b+{lI=-!E{h>k@}}^W1oEJ{$CyMwlIrCaDPVtgJiOE5 zX&}dB{;SSr&wKxyUO?@{CsuON*+>`gu6&NoyP)lG*dZ6K$lLxP=lZIliQ6EJ@4zqD zgSX*n8{bi1uk5`~M60_RgZX8J!c2pbTk@ErAD$Tc+t?b4Y;du$uEjRN<s^MUy`5mV zrna4K!2HIpS?I5hCmn##agwdZVR+?`yIi2}5ghTuy4g5PMV>Pt-;M}izAqmlIfl6y z25`Xgzh}Nr#13U%xEbyim~~HkcKlk0nv1FVf0V{<Twr7ZnS&RP#=%KaSmDoSV4qSQ z8vVLI`G*(GJn;TXj_m!d2l1tj{-edB842OXFOM8y-fmIF=}NJFOi75t*PNX41xJ8( z__JLnjRdOeREp+57_b6<qgd|^#(n|9WF7fA0;3Gd)8#p)k-Z?2u&0{3{<$K(_DQ!I znUdfm)x5s*1(#+XUkHy-9$QBuHh6i2qjGx!wi|;-?Bv+J+<$;0cnF;m3`Jbbmk-Wv z!IiBc;@FJzz6}Quaqw0X86@w$9g^nW6M$Rd-GX(nhp-RcCHFMME|?V*vNXTa@WGa= zuM)Ui2f?@j1PWdz9ED8?XZZ20C)LTjP;eJ5m&zk26yq<C8;=*7gWS3T6?4*~%|<<| z%3OwO>_r*g<n)jD(p>f|(iuH7zlID|dH+7%HcI5jmRW1sY{qgO6L<~IRkdInj!!yS z!HK<hTziT%Be;&$5K{$uTHU&Vqa}<L@NUTR*`!Uhc5^RYzRxv4e*onGkFCUpA*1?# zlwEl|)Z6<{B}E}wQnofkp@gJCxuu1ujC~8G?7J*Ok`$qmkQi;2v5b8gTgZ|<+t`=v z+t{}mGxIwiL-$tq=KK4{D~&my^Le)Oe%{Y>&H?hZl{mR;je}U*B5D3B(eY26%lZQi z*`7U1DT8^;L#&Ki%@2ZpVaJbUJQm^m=fjAoHvgLN|FHH6#+c)chBgj~*{}4SrM@Km zIot>WktF;$JNEC{K7SA4#Z<iVCk{`)jhOj(@FvFgC-J}56lej#uDgWZw29&n>+t($ zVDd|L@*}o0LoH2r_heEM_@BS?<7JmY$)<L4+>R1DnvaV}OXrVwCMPG~{>ORBDF=Qc ze5x<rTstrP$1uPA<MTD3T*rssSy!XUFm!H?1u}`a1_3%DTHttK8pEI7suR3=7g02W z(r=jr4%b0YKi$9f91n$mqDF;<m<`HQ$AEkZT&t4tpd95kyuK#(-vIs=Y}oK8qQ~Pr z_&%G>xcDlSeE|ypc!|%H651w@TqxJLeFOhT{0q)Tbfl>7m&_hyV{1{<WetG_6E--c zawE||UVwUKm}I@dQN7Qhmh)rRE-0-^l?NhCU;Wrazx^;j6guO1Ln3%ke?HOvFm#00 z>DY&qVF<t>T9Ge)fb4c)3YnLeC>w+S_K%<KQ3GN>X6)uDTP%Jea{+Xq-MmNC_Jkf< zK3m1-1*wZ!Re9~y)c+bT0buk@ix|qHSq|L-g1Z=XYi(8e3?@j;mH)?DdO5hn!`c7w zC-~}jo<l^@jhCbAu2s(#f~uBJw-)01Ah%^?!_%hYp>9IXzjsxy53@b(D)3O9`sP3G zPl$vLC{&%xrJM=*4>)uQS}AV-OE1tro$(Urj)U#K_<Os_$o59o(G!wV*kjTi*!@8f zH_p(-{xM70<iQFwsr7mGH?D)n9LzAytLeAQ@W+cX^3-8I7c<Ck>=!q?n_U;~*vaYG zX|Hy6anW6P+qP|*iP##Tl!?X+O28<mFvQn>fjQ-`S3~N%{{f78)IisDl^zM!;mYiC zHtm>hkTSHg5{ZDe9*zY#nUCk(Ye?DVt`k=hTrks=)%zG-W-haGYdm#Pdirg$aoKKf z?tMAqDV6o9Vd$)r;(E|k)zf_YYR@7@+_*@<R*PTFQFh%k0n1g|t0K7;%P8l=Y!Uo* z7wbPnA*=7VxuK!KdbGU@Om3lgDV$G6GGh7};yaDZ5^|}8xG3Q{Xk0M#Z0UoHE9pkl zV)$ZV76~t08sRd&n}EwX>bBB{{F1lW$wtFDS9F@#7#Qqkt|NkYTE$J#yet<`)U3!6 z*gA*f26%Fk7M=Tg3ib~p`r$$44?pvvNh;c|Uq6AJXj~k8;Zv{gTxF11v2uO4TUusv zh0b8HQ~rCKX`;`KSmtj0g5&qJhVI?91ka(;?4h^mb95t<u2}YIJu*V>t<B6+>nUF{ zMgv?{OxMt~cIIZ139u9RO{aA(@%fH*MBcF!ta^x;7dm|TkF9Gy$E@&DUF_W}&6$0^ zScEj5fPc+D>PJp=EWagRD(pTVm$i~|_4_>~#1#%gX<whbpWY?%cslkA7pPjzjekeS zaXWmuZrwvgDD&bb4Gj|IX`|f9ueH8sw7W4YIoI#$qjsMuyDFA4SN8bgNpf$=;|#Ob z`f<~~Qa_DRrL}Gr$|YOtmHq9E|345Ef3au4n~TeHi>LL#Kyc#yl~)bd0wz1|-*lZT zVkjdKV1lI43&KzNVAD^Nm+O61&${$qNBa=mNV7{f4)t_lw`WZr3jy~;y9u(o>mX`j zi%%F3Q@MND*)^{7(MbUODnbogwKWfMC<d~3*sMhZk6#ksf5Q?U;yV-+6u!ji5;6Cu z3TgT<weCVu#&_NnM3Ri%@efbWc9OH+`vxk(U>3O}QCWj1K?C%+Z!Cp#`@z-K_p$7g zPrsE4f6}Yl{)s`RcZZ;f(ZFMUXI0*Jl!d+y;yuOJcQ}Dw{$EN8|7Mk!G<p`=C!Z|7 zMO-smZiP)hHVk<BLC@OAb#ZXAWCve3lQ3-Qi1o}<OPz0p?30pG`S^75Gxz0k@9z(Y zi(0G-1&jHF?^UHkN?oSti~Phb(@SMAixPqc2Hf@h>E1A<E|UhB=RC!_*Xv>U@L^@G zn6mzH?bJ3aJxdkmD?8w-f8KbbN^O#^sZESE?L)<(pL&nZpT-Y=_->Cz*pH{&l5-u1 z)AJRrJ;UdJ_WRUnl3!V(y2*5EiLt}ZsDhbcwJ*`q#L)Z0@4UaG1SKSjM+9|iTKpP4 zM*;%_+1pdrd;X4VWgL@h$+GTC`NzTYtxO?k8Rmi&`-IlUK5nh2Hur+f#A5cZ#iI2T z=_gf7GVkT$vdA8@bRQ}`4ZQk0+hR92!J>_sFJkI-%UqT5u_jJ3uG;$h*XYthVJGXi zHuXEME@l#~XJRdV_gn4@h?8-05_QdA_7OCk9Z(aO;o7_R$;ima>ETGW(sm`O+wFp( zL~wiV9r+Jy+$Fg0%-Q+xgQ4n~mX@n5-A*8SPQXqmyD4Z^#{9Ol{fJ6DvXLU26Sg%7 zXPeo&&Y(;UusYt~t1Zt+lV*fTXT?vs9YGNK5_2Z~3tXyCOs@>d2$M$Mre=_1mKRTy zaj|7~Tg$snV`}By_4Lf>qR2Q8no<qpe5o#*qRZBfecqR3?spOTBn@kIr`=v#eB@H7 zt-mwBmTj(Fgvtl(B`qy2KZ$&`geETbKhSY?k{;xx8MW`@wl(G}SMZ{Jh@pH5hjw#p zxcF>O(fQnQOM2fMePo{%OC$l?ayzF7zop3}cjbApAu?~#U&(d6Xpqnj&VbXMn{F{i z=JgRWJL<PbBRO1f+49;TI@~w=l+s|IVs#S)*_(BgAbS_$)x*}C?m=_{4Jt6IzkcuI z`v*ukeAucZj3n)5L*TS-EkQo<+wVR^PQZK23O^Cnc0}~4H3j$<Gr){STH_Lz?X>Nt zk4%?J<VyRWnRzlu{G{*v0VnMuljk(qv4~{T!`By-^?GwMSxS?#J}ss$c<Vbi^S-C- zo&E>r0~rGEok4?q-2j_dW)S+NoQf#)=Rf}DC;M$XFg2!m9309JTxyYn+egEZL_Q?6 zjK}7QFSMDS%II>j%pIw=^lQz|K0Wt5SfV?7d2q5U9l31A(Q=>sa8`y927^_fk)n%| znfknCx}pG~sGz7=QU<OMFGZ(WI)4$9TK3R)=9^FW?MzYiPn5FqO%X(pKG$z8%l}x} zpTGR<Xi3t2YgK<VK$G}^&F%C;_oPPkkel;RxOLkInQ_maJ=J5o_KSZ>UFc+<gbl^t z@36=@-qfZcf|xEI+g}hu$}a_XQBTAwJ-sS~U%K%eF;cQ~1;i?k19yKry>065W?&t@ zwSM%w&bIZf-8c6Swf<vbvOJ!fwcH$(I&5oUexLDf$?X<7can<dGY^O4yGdED<Afg( zxD6^Sikr~=2{7;^(SvwyzrOdgUuU)2#`-H}i&(GZlu{}=6}o|ISGE1eRM&2u%=3eY zfwgWi#Co^b479Nf`*mc!TJ>+%A`7(<`kpY(p(JFL;J0Awto4x6e^KAm9@9GZzvbm1 zwKKlWw6>wr9ME5uhcbTjXs&LAda}X25l=zt$X5|dxi6~R8+fPm;73X?AJ{SF4`8p9 z&z~=U;(;gzjnGHZtLl;64>*waC+6!JFW^;wN{+T}xET$iHMQ3Fs+L2SnF*Ce-CM`6 zlxhrh1A_gBHT>H@%E2DCza9J666W)VpELCF1|y)PvvIOE252%pWLPHU^%5b92PR#; zP748eEj%@4{wE$$wiJ9vRGMdkLfozHtWk!qS8<-_uUnwkS}oE2`UY)O`Yb@=aqAc0 z4T}Riw+M?_=YcPP;+}BD$BFd^{HqO;AP6oY%1J3YK+K{RugdyDTzA4Fp=o5W?SJY3 zN&y9f2(P2VzxjXC6$nnBcluLFe3M58u)oZpc7tEN&EqFCSoe;9#tdk-BftTo`%u59 zd@SqGeU(hN6fouAAbCs&^FYI^er-kC<Uzki<xn?Bo!)wNP3^zVmcmN^!~#VUs7uSr zg3WEr%*<N<(K0zs3s!nzhtr@1qw}Ku`k4*?`hv2nm$i#ogi%^B3V>o471#f=AF8nz zq&BS6+w02vr@B2XfO$?1j<iwO+WT99M&ut@uScinTh?na;q<GGK(9O#)7KAcTYt_^ zBA3mCS{MkA)~W*I(B3vaev9=uYy+6Hh`@9A{%CrI!1U<Et=cIs<qJ$XpJq*-9<Hm! zL<y1q0lmmZHh`~&M?9FP6ccH{uu=~!{!z~XT<}I!?1Ny4ry>FrNHqiYMn<5WM%F3y z|EZWb44teV&8JT}bgd5BzevBnO9$i-|KAQEFb?S1v7KEb8o(ZP=ftvTr}gN!aR-nX zm_WSZUoxu_jf8s0%w;nuR~_-l{y(dU`u3a2belGj$vcToN7wJY8t<fet7A(dPb-}? zEd99iGVdex>o4__H69M@Y(Icc)W6V}gweOJ)KuMZ(C7l+Qfwviul8Z{BhFW^vmQA7 z@G`q-*i&+D!Gg<ThiGPIw(aB?;r^GrXhiMe*@B@i_JA^iYgBRV@oLSs0y5DuPaP?Z z=ny2I+eEdMQFfC@JxhqVxn^U@E{@Y8CZW{O|4}~p&Z=J5Vl;o8+~|yTIEpJgdFIDw ztAFtO4}LlideGc&5RrX}@~7{&ZjX|;PI|Cu^L8HSN1Bv}zDQo;O+UDrYFdt|{%6X6 zuYUHgKa@2$gA1TcrL*___H6zw9;HXB#VxHpb8dj`<08G<2iKnRuOorSPsjq*ONAVM z{A7JTfB)qp>N{9ryEv1$xOme_nSC3Kw4JB!HJDiY4W#X%A20V~Fm_tN7BTu4uS@cG zK~Y4^1CCl~m@T>z{ks%o-KYV>j1BJEy{3)-90NuLx+9&EB`7mKazMnqEiGnj?`bRj z#*=h^4_qQF;&|OtOiV%xe!**gT>IwAmOGFB{`lYjrCbk5-`P3)G)4Lbbc_lC6GfWq zx`>4B0PsjDwM2b`Y4Rh$EI-MVX&<0WGXdB|=D}Ok>te3MZDl<ru7RkON$*P6Y3p2K z7g_)4ckxo@`s(#2aM;cprNnJOG<PL7r|Obx?pzDEtiJlXjjv%)tKgiy#W+!X6sJqD z==jj7ebt47AJo}Tn?7S-m*o>yo)E5yLZ*!#m6iY=l`%gw4xTLAx_ztXuu_+Z;J_G8 zYAQJLOyT^iuQ%FW=sRCg7=N^GSj_3NA|bli*iw|BppSFzPWts{{+BwhXaU>2S8J)E z)7d{}j2R1GZadpP-$KH{Sb{4@S$hn86?htWoH4=YtYA9HdJB9Z5usX9@*}g6Q3Xtl zX86znt)*VNf(Ybo-G^#x`q@r*O6XpOlkffxP=de3`FaB-$wt-zb<-WMS+?QrkwZ7= zg_<IBml0KNEb8z&boa$hKYcxggQ^Fgi&Bw8&2&>5>tmlJQVpk`G1auwmr&bsD2q$$ zrM|OSunLx!?#)o2+rnZB_n~tR!i9nq2@f0>$#HwPB_<&;CGj}<?!VjIKlClD1(}zi zQ%%^52WTJZWehtzCt4YairsUg_DO&z;}fQG-Y3#a<@u`DTTcwi9V+~c?jq;T89R2l zZbdXDGY}rj<xfnw6y;n{yis{R>wIQ+ei83HDZvGC&n(bop8ooNE!MdQO7YEIZw+)j zo1F@hg3)&)&;;SG`!>m4-R=3pGMT;E$ATp1up0DejGjGu^jxHpCfDe<4!qNC>B?UE zo_#>39(RFpf8#oK;2%?8w}T%s0WzI-=yjxAYTiD4z@U3{Z^)Q(Btxv}-bYzCLUuYF zb)h0VRee0@{2pjIsc=(rS7vYzhTCz;x%k}Of=-(E#1!}HCdb|bnF}*I7;O8+s;8PY z4hJ}V7KqDMX?hE2X_c{HqG?ojZI{&en2znha9QYxX<ckW88e4(Vp%QxzQxJ#Kn1_k zxL_JSS4LP+!KL1zm!Ho#G63Z$a{<j)8b)MaS$A#98t|Ldws`_qSP;2-Tf9^O^`1d8 z%3WpPHZ@aIz~;EvtwiRY(Yj>MUF=N(&z+_6yImQ@TzB;9qq=chP;QbOMaLGWKAmmY zao$-zZ}8+3u5pQ=MUr$~%Y43ap}kCoO1|w%zWMT-Nqy&hbTy47(aEQzGoaEPxir0O z<tWfCNa9*3!%rqnIjU;ur533AHPA%f%$9c7$t`6oA0$Y)p})BwI2i77@;Y)Lt-98? z;0`%-Zn2<wmVN5XveT{_kHsPMfq}9&y6wTAJ1*>MZtO6Fv#>O$+A0$Eo&9j(KVEn} zhI_nZ0(IFN4Xtj%7Z1$Xp+pFEaS!{yzF}BW+$VA9({l{|X3Mn$Y>RGvT(}^zk#Nt@ z!Hz%%oea(H8%JeS1?~1Wcnc%y(o~9FrN@J6jK9?ekQFMTTI8Ha(#KS+I`^}*o1P9K z)wl3Ud^zu)?^Au`Y{b-1x215@B$c}oA$fUHg}ZHTj7}`G>lyNcuJp30cz3}}je`a8 zZIH|7wpvf*2{QW?ESr*W$g-)G&?$zmgOPfblVfF_x%US7`Y&y5%La;bkoMTg{hLnL zUvc#V4<_)Q5xquu#t;VgWee8_pE^8YU5C2j7{V0tEiUHjXeOw^?b20>h7;)N<Lk0x zqqsJGok~E;aVGN2g19U1e1(<EXI%ZD<wL4dIyX*GtM1uNkUTce<9yIly}kc^b914+ zyPj9sF!%Gj>G=ci9;hGh^4#1IrRdv+==tn%y|R~jl1Z1FBsrW(Kzh5i7G%sXdOGfs z$_TDaaI6{>I~(jqUu^rjGW%r?x$^Knmt4ZhurU**vcWzAwl;3Cq4su$?cSB#S4GBO zatJO*)UM?h7@<E6hK@VmY~^~pr&>N^F`ndppq;FC>2}Asr)=_$V|UF`@UPuX@=SA> z+o$$I?(j!Pe)R%Kl6=iUq-xJwvNpBq_V*!lo|WQFBA6=I&oX*c;xdyeHyOI-6S`^S z<ko}eSa@6SK63V)T&?)d0m6;~{Ol5snU)t}<t)=*47JM6l_Zx#9NLR{OHK=wq^8;& z;Lpo=w^?Flu-fE4E;NH;rY1Mum-d$9q@Sg_5OH9tV@Yzl;81LtLcYv=r@GLs`HU6P z{Hz+ihyCREq>X#P9UFJNG*ccPIoY}}NtiBJ9DVy~p_AopnD*AXz-nKTfm|#*FXm3h z0a=l7mkn&6YOBwMNF&dYLdI*1Qr%QRJ=r&$^eWCVbIV@lD}Nh@8jfu#Ym+y;Ch-v` z_tz&zw`p76Fjw!#2Fu>q<y!gsh|>>7GYLE>@J=H&ic<u+Nq1d*>X<-#Pbjz2Qj=bE zfsi}rh`&?^8maVf_!c`@r+BcN3n#11+}wr<b!6?Py{Sa5s(JgUHa(Xm(yrRj#N+Kq z+d`yqHeQ4E)MBEWPQE*uK{uc#bofo%hiJ#D0aGjglx3mq0bkuE?&b{^x=ReZPP;i` zPi$>0et9ey_4nF*W&TFC&;~J&k>AF{VUGgCW7xx}W%`d>YlA9Wxb=@N=qm|eUw)Mi zL?cJ&uciy&_wBb!O&1?yXA$|@pSygzl*Y7vM^jqxre_7>*`I6?Pu~*r&<NMhXL1V| z?7Xl9tLyg>$wD**0dx84569eecf4mC3op)M<QJwQ>%?~NnuCRYoxDgg_GO4;8>{ub zfd~3+W@Ty4`#}EKPgq>(A%3#Oci*&eLZ*KL?_6eTm1r}c^RYA*7>4||EyG_E&vlDV zlgM%iCS@hw@`VMVx~z|_gU2nbg3pQM<<;``$^TjJkqEkeSO~6oZ^Y#T!RpmCN-G%( zMOR5J+WC*S2<A_!JQO6gwI6KhD+`@e<Gym4?|FSQ{F~^BX<K&mLBn`y#P*BrHtbeD z_KEk$R#xu1@ZK6wC*Ik&*QZG>HgR^*SDN6g_oT-e(;ey=FmB2c#OXWp?)i;8;Vht! zyRqe+GvP6>eUmT`qfE@FMVra4)&0h%E=1D2ZE!%hX#<gckXPD^{K!2(X<%`Y8B_gL zXnAgq!RBsctDE#nUSfXzGD45QXR?e(oV+fpLuNQ>q*Cx#vBCb$6Po%C#PbrQwo!qx zwwZv(|F%^B^nYnyvR$4dWoWXxmbO}l?c*&1Ze{1U!P~Afmb5?8V_8!6RSOcyEqKkc z^>Xy+t$1mld*Op3;mYB-9C}WvED?nas(1J43w!G3)CSx5EZw**v9Fqk1Vvuef4%A& z-Q)P6Ye*_fsAgf`qhr%r?-YeC+tTh`AL3pE@XS{}yYr3q3a2ZauP8*Ojvm6wm=bGS zY-rzCo(LKo5)Ss5SE-3Z6cu3EW)_!K%0h5%%Tqhgl!(qOW|g*cZ61EAZGU{4i#Le6 zsbSAZasmuMc+1+<vX%|NAU~&v1GfUhw&M>Qn1t<icSN^zWx^Scu+VrVHS924W;Byj zIWjlPD!ge=CDz*@?Re*ucA}@8saaihb6b4${yRruXBjK=^5SO~Rm54aMC0d@=EAAE zOV&&Z!nu;h3Ob+3@w4vNqw8+jyOZqP7N^z97AJ+73X!)%YYdkbR~D8Cc?H5Lh-xq6 z`#Wc<`%_tP@|B}kgW8&W1{@heO9;dcvQPwWvKO~`nPKw<0Z`4m#P|5t1`tg7&gKhy z6c!8jOip_4dNJUK$;lkp%T=z%cp=ippJnEoB#Vz?rT_+i%v%JLD5@K%BZP`2NY**o zz_aVU2O5o&A3Z+Bs_6`3)7Ct-l6f)B&9{KvhGOUVE1fMt4hkbq;J$lyyR@z-%=NZT zO-hrDh0e{bbnolUZe%A;5__G#lJB;W*$tcq1<bgvmbpJpOUPRhI(lgHX!4yv`%k>T z<Y*?rW%GYti~y>X+FkA<(&>F9S6*^kB{6*n-_$N38*%SQ#Z(Bh_+_!G@192|j=W;Z zZ$5vFAnC*Q_$zkrN=J<QAfH~voUbea<Un2dLjm8ymXXOLXLqtXTpe2Ea*uZ}_eG+0 zA7Krcr;q^(WSMu&?!k?Y7jnHVyg3>Q#<>%Zml8rweBuLy8BBQL;-4c4*wnlogpvs4 ztp?MB9%i2WPw}dE<q|sL-t(QjlBQ@ISoz%f=;Oys%gMW_w%R~rI=-G$IlGzU7-?2C z+8cM^2^Z_FY#7~A?In>Uu6T95QHI@o1f91JoL%yB*a;6ydE+dXqnmRw=y*0*yn(#J zwgV5?vydTqFP`U0%INT#&GNxBXGLzmVc@2{V$XGIr$E}^t&_t{9w(Ur?0X!j8x=h6 zPX6Y3%jB!uAk$yM4g0++g#<nRqOylAWsfBj6B_04x5avI+av=VZS+1{VS_crM;oMj zFO-xIJRaJjC4G1zj`s#j9wWZsZE>x~d0W%8N4Eq7q%gZ4)zL}Nyxrkpvj@z=LbPUn z8}*zmmfY3nj++sc;Q!*XwL<Q}181iy+idOIKN8N0f%@!g8%%(a{~*X(E}UZDNln3h z=RtBJ{OZ)uDlNqC;PDk(oAwn;oKa&bGMvb?W{|=SDi+6ST#Ovvx-a-tdv3`t3vEzy zv2fnYxB+6ieF%7`S=1Xk3dZhn2e}5>xKN5R!^0vD{c+-{1HuAdOyvAWELSeXs-KJ( z<n?Hsr|Y!(pZOSNC<*go9de}@0J8@GX$M2LK3;`#%66MmH6M-ymR2WeGz@0$)9Uu< zG{+hew7OpljGo%+a~&w-L=COs`9GU+WG<jBg0Zf)>6C|@29E?=?^iwa3mZ=)SNG%N zC4%5(?+-nGu~8@zf-<h#QR|8VhXFQh*_%rd5s&|8KmxS&kKOswafNz5<XF6!oJY3r zzWCVeR+NB1Jm~Zn=hzXwq1V~40JBLHt+oG!r22UHIe(;de+Wq&EQ+u2xA?Zs90g8? z#LW5qwDSE+?1=^5qjcdZ;-{p6YHNk2e)Cv&>ql!*v*xtr1df=9E}us%<~&J}$zMmc zNU`&Y6ZP8;^}O21J(Y2gScvpJ_+Qqi8>~E$5hq!sSk-~9JPIW_H!!;GX&{e*=J4wi z53}+QJisI>nh!P(k4}l|g$*???pHCEyEtz<xvVq|*qsskaFcjU%KQ7<$}au#n!o)T zc?>d%l2f)XD9`zeP9aSExJk)pk1ZcwYpgS=jazy3WEyz&Bn}PM02|k1rotJWo7HG| zzh(G@FmF<ddZM7n;jWDBYjn+9>Own<9UD3DT{>WX=I?4GDU!Nt2>p7N>Ma-K&ygRu zGQpLwUV~@K!^a|>cmrRInN}F><Y-t0Kba{Rm<mkB%7+b%6gZC;6#p)n*LbZOhU^QV ze&Lr-_4ls6?vW$Fj=CK69U-Qw*|Co+Au)UJ!@-1YK}OWz&P!SIv`~HL{e`wa2*dyK zFDsyR(9^ssFMc`jBDYVEXrF{sEL*tsC3T~*Vwhf1^5HWtie^lR+nO#+G+TbqYxb6# z1K|5!AUbS!>*5<nN8j8bMQdn+{!aSLzv}%LTYeP>0)$PtQ{w(ram53=HF$o`c6#+| z)J)rHvS$%(cAkSH{(P=56-6aA5i+4xr%_+Ax>^-}{T56L+fRP<mTEP2{^`edFdqi} zDc04$@vzwqq$wm?ZGRAoOU$01TV9~ss;~uDUHF&j-$cVAUvJ^!l<JJgy`|c(jEa)y z_xZl7aPED`wrw7Nc1+n;z&Q)I#|{;(T0?~f@Dj;dX{<l=wK~@I@Mjf`lZX*A*$(ld zuK_Q5$OLghNLegzuknDpq2}Lg|8n=Xu8nPjni0IsCZlVVxq(y~Od*5))>(?^)3`mn z%Uq4fxyya!4`SbNy}LULn$_A}Y4`s5RW@zDeH_Hl_-Lf>YJ$SA2JB(r%&hhr#obD$ zECzEacAaP38HKe+C68A4xQ6?<{Sq&J^VCfqs=&y!UL<X%sG^<?OdfAr#j>gjp65zb zsHmZv60`aHDXUxJ7O?3|%I5G!amI)jL{3o&f6CessRQ9_Y(rVknpfI_VKbMRzi@sY z$9q|5ncD~1n=i}D{g)&BpoeLX02wIPzontf_8`PQ{S{cKOnI-n(VJBig=?N30#{+p z7RCJjvOk$5KMf>7kxK1DzrKtlYov*@|2f-6&cZp4bM?D^{rp!a`Uo@wGe1fHW12sn zyn~~FWO$tso13Er>JDCPAvrOHZ{CC@pxzfbJ0u=m*>w81hyObGD~M)K33nFa_-Xcl zU>)BY^Clu(Pud@JrZd3te{CQ=4{&%)RP_5V(!V}|hw<hE)W`{qHnV}hG2_)RKhHhC zEReqh3lN5uA>%HP9P**)HR)bM)P<uvp^}+oHys&VBQuMj#ZwY$a9)ly0&nDMCdqZF zY3}<Qd$3tajQbf>r}$lNKmO0d22%00)arDKK~CubrFGC!UICzs5Uy_CE>=`jS>VwX zy#9I-joFRy?e0Gh3>N1bAR1^~lVV$4-N6!D!lb~rq=s`mhz)?kQ1An?FV~9e6?a`A z=6sEO|GV1tX0xmkS-2fhLsvsLbS)mJ6r(N>PqFIAFHkI9+G^{xI(K#;xY{SjZ5ZJV zhnUUryn7YU|1<x80O*nJe5fr&9m+<)fql)+@1=*#RQ5WEthE;VRxLv)AVR00&R!mD z>p;y>ma0DlnXh12zC8v~6l*`i4?JxvU(nT64=&dPvTA;Gv1x96m+iMLU|TMKJ<W!0 z0YD#2=fna<|B+iEOz`=z&5l(8NCMqOz>dPf^?Owp7;M?Z)Nk6R!r?0FwKZbd!!M=( zWE88K_4o+hJCG@P3uPKm&XqSe_)3s6UtKWYg~y{FsF7^zswg(F5o6f;*hUjJ0tqwh zrlkLx#yFv%RTfxQw`$xoP(+hRQ&Ft-GZfGJ2R&-kz*0sr&OZ^PN5No9p+1}xN8#rL zX8cJsSbGOFC$Iwylr?K!g2|><edGfXHpY7AhLeS)t98zQO}d<5rL*)7EPYOytPuFn z@e7!2_-$So^*VSv7-(^N1BU~$y>t}_Pg8sR*{ZR~-h;4%pHsiUs!D92$7V)=krRy1 z<cY0S(hh)AfL<8t{czj=@V)YjU}SM;4&OC11YWRG`NYq2b>x|2pKGH#Hs2Hj!;j3d zf@MrqO#}Pd{nSRI%JOsrQ<y!jKwh;hu(Dry(4!~Fq`K-LAm@=m+Q{qxGtRq*g4HDI z4LckOi9qcoY_yciW`JQ+S^Dp<P2iFi7`DxbgOWULqH>cS;0gsglTb{T2m5rdfO~P) zCh%W|(0>KIf8ya0%JN0E6~OnUU;y7KxJUT`Qfav`%EZ}CJ8=HYPj%@)GsldLg7K!! z8o<VC-#n+M_|9GT)dIr4xmDCXn<qOZ++8#Wg*Kk&Pk~HMUcc(L6W$QGf54Cd+(@AO zNogZNUpfqQ|AM}qGCu!xV8chwBx|#-+AxG8QomolF9L7aGt+;&?;lHvZq2!WDdztY zie#06Y?M70DI$FJ6a>$cp?8J$K&HfV96ait^`~@EctgYYimx@GGp^_o-v-hKCwPEC zP9?RlPy)sc2xbb1ezjk<i+(8l3yEnhUTB(Y=3Um_^EYma@Xx2X=^qv*szV_)ph0%^ z)uGM?;o{3fFIT_y68xrg<y8|sus9##3C}g;ggXshVb=PWxBmTCBOV~+EvM)yjVO_M zP!O6FEnlMeH7HELxKZB_H^g9Vhmcnb^_~A`?DTUoEBgYPbp6v?>Z?*The98D1x&fX zdTM9t7#rro!;y0=T^kvlY(5mauD)Jd4fzd#qAr}3+zA*fkZx6j57{0|tBWDLUzoNf z1vU#+J*}KN269=e`fQ^O%&iSFi2dJfO(=iY2KkQGS6f}^R}TX~(CLP>zB6&Y_16CN z&m=%mc!=nRzO4bhe&xNP_p5TP&;f&IcT;K0tSa;vc#C9}nN)w&$g=gW#VaDPV@0cZ z*YE238x!^41Y=0skF2In;EYL#W%#(KIBC^e+El=Yy-TrgboY)?`zfk+&nP80ng<uX zeU<w62><@CKPEAZN?_cxp)2vxP}uOyV^Ux+nOe<t|AYo-Ai%mF41f`X#uHNdTH}fA z?(I{vu}OilO>-+QdlYI5P8k2m4OX>5255$iq@JV%2H8m2&0EQ|TdVJ_{$9`2Gjvl^ zTe4LW*um5bg!!<w>A*+5Gs_nutuLomYjyOnj-F{!x4%t&Y9oh!08VHC=H~a>f6YI6 zA~qEk+&>w=x(MFA3PLiAE)pnBjA2`Y>Hpfa^jsLId->(3_2uh{APAL0yY^EQ2SsIb zPwx0P>&#FF`=zH5_MlkecpJ0<**(lYwt*r0Tmm}Ut+-8Y&83xB<MNrNwYXf@Tv&i~ zC$;X|w=dqX%s&Hcwi_hah~nJ8y}z&kZt}{qPE0DIZH~Pe{R;m^#yy`nRm_l4ArPky zhg%4K5YoXk4%UQ^;U~NDa^{D>75fRdetvyS!ywhT)*)ve*V!&0C>Y3T5E>DepWiiD z7222E|2m;iNru>8`qJ+6aj9X<Nf`2(UG-OgERoRZ*xj1zG#js#Y0(S@cs0n=wbX>f zrFzk^wN(hB-_9U1?oS&?M5m{x^A6E-p7|nm!3be!_(4oULjE-|%*Tf|E-`Vz_k`X+ zLt|rq#yvgGz$<iIeC;Z@JQe(HWIglUF{IL`*f%RE08l&l!(gDR{6kstMtXnl4+6Pe zS_>N`kTdP(U)0A7qH_(vmc}@`7+r_f+ec}v>BNYlcdC-WrVu`+{)HjpJlfryFj-KL zkXC=i4c-s${Vvbt^d;(AxJkDgAwdh>L)0`)xb4m95Mn>kCYEB!Yst*-Hup{5yyDf- zs7SOWZRKP}yF~_Zp}lOUyx=&v*iE8Uo*Tiu!_ib5^IDDp)`+ni`JUsGd&gNQT+Twz zooN42*z1!-r`^qVm&rUus!@eDaNFqE!}eH?oWgDUTfWJ2hgT%NKu9n5?{Qfk2=45- zm)H!xR-Vzbzj}JMR>l(SGE5qIN8Qa_-*r1!$84&n@Say9%fo`ne8PAMo8Ggn`AyL{ z^ov9CZK$}!QKjAUs2&_Jk?|t3o9E{4cC%DJ+vb&0Q+Gmp_jq4`jDL2v?NE(SPyScl zu&n%|LR+21kCy{GlC><dKXJNF6`mmMKy%2r<>oC9hFQ!@6UO`A77>!3JpkAocZ8Mz z;$A`5Dm1T@I|o0`FgG`@9oQZCZ7oIkZ>_yb2l`5ha1)}iZraopun@Nr7PXL~Zj)kf z2FC*}2H9+`cXj(TY^%l0{luEB-t{_mIhEy;NiK1*vB|bIl4WoK2LI)E-drhV-0mIl zaMdIYkLks3_ilKz>h0N^*Cb0uM@Juee_YW$H&?JSAiU6|rkCwTvvaS0<?+n72q4{s zE|<w{H=sc&0&&uvyJ;vuhIBVr*Ex2-C(2pyS)#}s4YzAgb8H(v)Hs5j<y!5WHoAa> z>Ac_Phg&g4=Vj2r-KR^~#1VKq#~Kzj?Qb<eKp&gRKM3^{&^dNkd#47ZSEC6lv$f`h z52?kC${E|irV`?47Wz#kR@bGPx*nd;ktsu(4zTtd*~53uuF#G;gXIA>x8lZZr1;%B z9yb0(#1WS~{AgU~G)Api%M!-cft0k3IvNrexolm=X-p(628>?pC0qBEX1mzTH>qXj zBM6w$Wemfd+IG9ik$vCAxE0hsId;nBAGJeYSwR&gHpi(6OU*al{J<5neOm)dDK_s@ zt=#)4m#Jr>HBZhmp9A=fwH?5?yb0WvqLJlP&@d;X<-8$f{!=fPGVA664h6L(IMqN! z#TB*X93`Z*5QomrJ?NwvYBuej&G|GORu9Z`CNf^!Iywg#dJf}^r8_}uznF=NQ9HQI z_?A1F(5=W4%_2M5nUmQbRnZ3omO$4Vy<gMcu4`t{gN7@K59wO3s_S%HYPmn~Ls1cG z4$*F&ncZ1?W-LBNpdHmiYB3DRwjw^|ix@sX+jbp$@=n!WwoRPkW+4-pu|(W`B440w ztU%773X$u6k4MIQvz9!*Ap<yBw^yk*J*NrS2<u{`mDp9jtmKXlB+fM`IRI_9&r}K{ zlg2=JZ1h60l&Urq(||ZAKEFpeBykhwVL$D&04~^>3XNMfo%>DRaKpbUH8YD~Ibsa* z@HZK?6%B7}B6+dticqg8of-KUlW4~kF1*6?uoO)rt@2fecr=MCZdo5wM4+Aa4lA`M zOl0QJu+8xlPnU69qMXqn=P{KG^1h;SOQ&r4qDbe}seNn9I~vkc-`~JttU%>ufoOv$ zgjW89NnV8kN3;pEPF#zetR6P>Q3R{8Yf=;a9D}t;%b^Gti^El&uO*%oeFh#FosiLH zA~ejwRjFl}cN?qco=3MwaO-$G6k|}9;Bx4$jM*9~LK6<&lvOJ=d6o@FoJEq&P}&g| zQWI$n3PI8Eb6P>^#S`y+g-*ix7#@#=g@sK3**bbm?YVZk5Pp?`YjL67vfUo5GlaSm zH<Ql5ZEPbn`Q9r1A+gs1tK--Te27i-E@ziS;D9@67U=*U8l|1A4ewwX?6zxA>?~kW zRWGd0#FgAH9n~AUKCIhIu)4WK8bcRkSk=koBK5e|0$N6;P)>`k%fx_MjvBWRmZPG< zyO~8A3;XDOhFO_GO>i}+jzl9f{KVVJ{ZD^?5+hAKpQ4XhAdIKhj=#JrWPQL1W;GS3 zN37t)$>gAzh2!9=8YRT}P2G-5@B=QeW7A-{%q|d#R0DmaM_w0$1gQ9+mkzym1KP<k zvxq+OuiA^?IyJu6l^a#iwgYa6t@oqIVG5F%Ij)=tf(0Djsu}dPR&t0V?2J81G2(2i z1Gu`k-LimakRn*O((BFLCf;t=cfJ$Fk=XMdWr)tlk7j^Ku0<vhKlq2AFnp{Sda^`S z^WKqmH^PKsP_c3Ko7<=wX@f6MVi2BmAea`wW~)RKZ#c(KyMpK<#2?SN2=>*+zkgrE znW&$)c^Jd%WnZj#z`^hGP?uS1by3(JA;FKXrR>Hn^`EZl1X;pc=q2E$YE2GsX*Che zYPBdeFH1Nc*OF}5c#9SmcP4964PGh=3qC)Mbe(y6G%1&aACnK4r=!&ue3JLwM-a5{ z$8n~pznd0ZiI23)GFTZxx?2QD<74P%Y2vbkHB!D*hconz&0&Q1P&ahDOv2cbRPF7# z8|mo@5iU6E#Tr#D^V>9>cFIP9xM?*#MAJ3~E)IL7$+daOx;R{))$C%`EpN^{VP+{$ z+gHB(3b%1OcJy+ctSjEu8}*qOq^5h1woUJTuUBuig=COE#^Kcf06*i;<>;-EClY20 z?GUWcqOO&DKiRgylOwqepb1N?wyhuCr!i4Hz(b(#o4N$U58&f<4r;ya;CxSxhVIAD zM%AXQU$yPn5f$9_%m%iBi!!5ui+&H^f05F10}yV7SO}?*hCAzd8anN)aSex>qV_FQ z>N|)!sqJW<#Ld~XXWdt(%Y=5u5t-vvZ}vDZP4z~}CRvj4*n;t>FbkV=cg@v~n5K(i zwHs7+pKXZJpGe5;+Q+g2FV~>5^uobQFs9hV1k55f37nzOUiVH;%e>aXi(xm7T$QMC zja^@CTwE%&q@GYZ;XHRY4vkjYD@4z|cTBq%QB(;Q(Ja_pogA)rb2VHj3^|tGtXX*t zAexv&08%Vm7FvxHZbn}dzf%<wW8Uj!R5&+9>b>nnws()akuC6=5JfwLvddpXLldfF zx{`w-h8=QL?!+-YoyhLVNj-4a{Y5qsy=t%fcyd{X9IKXDvKO(J!F?*{qVBxaBd(Wq z#oGZ+_KRo8zbqDli<_D(Ad@nR=|Q`Ys|d76*dyWa=K&I|oRI}WktTR&_4CMc)hsLx zo~eZdb>O6ivCIyg9}J`010||_-o8SO?Dl!qz1|Z(y;|m+38ra3iF>|fEzS7vT*~7# z7_jei(!Mo{ft4Q%?(^BPM>}Xv|MWaCIR}<HmNxi(i4NBotucuXD>*R1&UnYkuEaN~ zD3!RE=|w}d^$P&EbK;6CG?kTI!Lpg^M#N`7u!;8>#6>cb_Uvp!oyoS0M!I1j=x;3; zMUX|Ka^g1I3$+L9xs9L<v5k#cKBM4z@<bQ-=<7_Bb)0}>{GeRJ^muXE7b&&-G6nG_ zgD3~I^(k@jHxH*!rz0w(Dth(>bJ1^CGgT{@yBx^d3f9d1^F8O|XN~tMs11++PWp<= zThw(#j~v(}Lvtoz@LW>~3vMBZ$aYL!KRUu0wg0ieq?)Q~#=e*6&xtOJiv{Goc4_PH zf4PvBhs=k<9mT1eixpWW5vTQn8l)U?99<=_P48*WE_B!`?LTRBDQ-gZooT(cn3!1W zWxjAD2yeC5c9%iO6f7jyaPOGBdRQE=L^d=0*>V;5h;Vj4Q-j=Q)F}7LIL+b^Y%^eZ zF~N6p)IfczOZxvqpG!k2q+F0RMYo`f(Pn3q;Hfy10hC>0<S1DGC+~~^M0mo>ibX&* z>vV^!h|W7*FC&)oi;9KD0Mxy34l~cN4r)lRrfGm{=$z{WpbRf}(hWXfOx+-c0|-M# z;K|sQvBJ@2(qH@f0CK#QdZ#+&oaeUI@)w7QQSo}WW0j|jk%(r2DMAsQa7(yjS7_3e z-1CjJmH~TD6oHtY`l#=+U1E<1?P0;4iB_{v*r8(+ItBkCGgE#@5ngTE>Pk(5AgNGR z&t|rVHECo!a8q!!k_6(heFEkpyn3=BIw0;zSDwq_N4cq>>huq|@E+I%1HjjWiP-3f zl%C?9`92sfA7z#qtH9+Qd~bVH%i_L7#2a9WhBG@I!sr^u-qBN&5yKrc4HB(Jdv4mh zbz(tsWrdVNkRB>7K3KpOAC~8|f2HFxk-mV-nasc_d6og<0jdZ26MA}2aJR}>l1`u8 z3zch6x`~1v4pOi6ol75a{L3z*pE1l)z&9&VJ=LL*3H)M^eh8cJ0QVX4Mcg+qy27HZ zYJW$}>mEYCbF#;CL<*PnVIs5J&CR}lejr%S;hd&dAe>4YXh%)Q9xgQ&pL&pR2R|5x z&bMhQhtIn-##|TI|NI~(5iB8+Ib(@A+MBe3lGMO5nW=Nj0>sAwP4i$KhvdA4w#vNP z5_gLNqJvt3OL~*4DBPT3+gk*%2iM8{($Q#DbW-m5*{7QCZK(r*ZMt!-I8%o*NH6K% zw4cz<IAK`Y0@6hz8m{}1=6UnY8usDWL^``G3-^$4js>Y;%F<Z^I0DezdW5q=9HEOE zBb_BJ^>yZdeK_J=XIL$UefI1+gK`y8rklDN#C`cpX>G4Ang&FRc2JKPFOlUTNS65O zv9&VfacReH#9}xheoWN839gnsx~!rQATrGhCC{ftsC5;a<~lq9qmy?Owo$!8))cI1 z35x2TtKd@hE8QzY5!iKm3q3`RcjjK1p2syrbXW?SGSAn7NNn#tcrqjr)dJ;KbQ8FI z0#4j+xUOfa!-d+KPa&iSbvF)^hAN2DBmnGUzH8U6D1kQG5qfUv4!DD5)46Set`|(G zmswEsG<aKf1to{Xo+&uYwdSp4;z<%%k2`?Zi=%01R3vji*evW6bwf{~9?pkLIXbW` zrS8g$-lKpmX2uAs=&V2>@aWB`77!Ni0mUYAT7TL7g0^p3NGMz%|60pp<UDayFkBx) z5;&J>(W$wsP))%y{Xo+Bpow`6ZNz-j7Sn3DEH<x)jPc;gt<yXAyo}xW?e11Gz`eG} z5(s$0K7#d<209R_LvI(TK3=FezY~b+gmT+m7I=d#oeV_Vr@LCJrmwiwflU+^Qqd9{ z&_j%}Jq2J9$!1wIcsUOnx+o~~boh3GEp#&qRimS$!znh#?K<TF1K3n3mm@JF{8)&t zo*sAXcDI2bHB6*U;=>WzAVK9gfgTjU8#qA)WBO#@iaQ(g4b`SOU=>(u`*e(a0GH6S zMAHVmp}%AlJbhjraYSnoEKzT?nIFo4EK4z{x42p@^_q&9ZZjsM&tIwr<fM64YycbC zmYfwyngI!QI)uq<lxLB?Q8ZK#m6C(Q#DQ}hj*SU5?1=JKiZbb=eF)}JF-055XHw|0 zFdB!>08~as6AqPzt{C^EVG#GqZsNW6b{I7|2?BivKn)pZ?f|X}=T^53_Hnmh{G{gy z$^cz4lpouc&lMRH7w5Y_ei1C+?(;nW7Nvn8P$b^%YAe<MTu$2&%27scJ<{K<!_pxG zq!VLL>c~CF2>Nrl8t=9*LUk$`aK5K={r-(|C2&|}2v@t;S{ED7b9s;RG-TIu@t~{w zV9sgt6YUZyrYZV(05fm1X<K>FJqy?`Q3DsKy&$3!RKT(?0^sd9#suZ;qMWa9P|<Nd zF2O=N+9EmxLfx#>P`ELmt87w)m@L#j6Tw<*YbXi4T(K#^FdZPO7UuJ+E$Z)j)F)ou zyk>m>pIUQ9@J<%2$+l!vD$4@bk?jCrRgU*Ip@Nl}R~(*Y-raFb1j(VdeS_K_Q*CW+ zQ$qKI36fWs>y;W~8KxNaihPDWcz#!}^H3R=OMILF+JsE&>D|6dqN7J(Y7&G>NV%Bm ze49M3mrq!x)aDnUw3xaQjTuE*1zWVzG%T*I(p8A>3C~y*upK5suQl$lI~ItEi(?2n zNEYHJDg=Gw_NcO(w-M@*bu!4JA!A8R&_?i$tM%CNAA3i@UYV*W8UKy85m9Y501HRJ zr=s<}wB>QK2Rz+r?;{=B?nRl@pimuv$O^%Vn>HUKfNb)EfUs#9fR>A0F12uP`fW+6 zyP^b7$Lmi4ZlU-54BuO;^Hrk_Cxj2D-Dv7zu~KW%eMc~B1B`-!Vd=9Ip<MQj*B9Em zN#YicvqQCxRK0+=%o%)R8lRGK(nJq_UoA-oi$YEHS~;E>y754#DGBI&J|qasqS~C@ z$b5LRIYFyYy<R5^z-JvVqJ9w0j<9_ac?`tBMLO1FNq!PtEza|!EFivpDFA1(BpBC5 z;Luyr+76_hXV}88?_PhIueHq-&Dn<c4U}f^=61`Higvg^k&Yc7471Dt;I9>|(QTu9 z-O&kI-+fp~m0R3~v}qce(C4SP-Im^QT79O%*^<f+1%wEsa67czBBgC@IK-5DnOC_S zsr0q9$Uzi<A{=xUq10994RX&-9!n5{ZBL>!yi#qQIjmgX)17v&LA%@~d*HP#%PT3n zVa8dMO6ZD@U}Dnfs0rJv=D0TuQI=p$bLjD)$vERs^k|2;{iq6^Dyl|2#BD8$H;#)> z@uAxUz-ihUO94SvVcqfb70?0|e*gQ^jG8lJA8pjD5D+6Ra6_=iz5IZ8)+YBVPIAr- z4_hy!J?@bI=_2vAyB}gYp(HG4x^%zdfjfHEU6apo*%8j8M{@v}>6{-qQ>;j3N?k8y zKidCrzIS5en6P$M9W{<6UhR`b^>ralkkOwXqU~YWFJ9%)X^&NncK!a8ZyW>?I|jOE z`@~R93sbbzRV=5v?}V9~fmG^x7)U2v9R2X|G4C;480Fcfuj#rGcm?YAFHiSOEDs?w z0Y_3ne`YwcC#sy_y0xL(byjgd;2a>X9dP&x3>WxMqc`PQswgZn<?`;~@2`zEGQ33) z-VE)PU@j4x#IdGVoi_neziDeAywT!Nehm~49Nak8TTt1zGoM_FHt7UOPd)^cq4-(K zX)bb!bEhsr-?_}6b7J^IfCH6X_tW!}v}(ejJaA2NfO)LDprHMgq6nh?<K+dAMdVaq zQFC00hjF#x!|<T=Zs)P%gfbb@E!rDs5Yb=w^pdgw&(Iv<G{`>ImTp=VncA&{mRXtd zq?;F<%BhtmwDc5udWQAoFM3$RmtF@Hu=*)6P`MK4u^B)u?etRPj3b8vZ6O|3NH)P~ zucXb#UHeY)*LMSE%?O+#Cknft)OshZ^~vr-2iv0*LcbR^hgo1^=SLd&Ql!6ykr&A% z<c^e~nDw&&VBi=rYq_aHLwGC^;AQKejxj+0hH2(0u_|CaX`S`Y^*RHlyl6qu!<xE8 z3|r`;*3NPgVpHP|qYV)a##2x(#$XV0ko?9paYX#<6qG45XvG|KJKc}lGPB<z>zzxF zERs$Po~1TmqhtI2vzzq8k$tlO0Nj!xEq&Nu3G#h2kk>@GOp@gwZtjgj<Z~J1OFyrP zg6*L9PQvkg-^)8;nOTMQs+1khHVr58eulRHMg>p-_&~L@i-O3Mg*MZw(6ocfkf*Pc zyIxEW61@^%x7=!sG%vt6p_z2UHuus#UYuyhUJT%xSeP%IL!kt;QjDG^fY6QWmgsiv zE2<gQYWfUk>Bv;@9LYDy!H=}2M$`Ebq>Iro=eeX}2AQ)a^N7VxV|4C}Pb@0?&G}+E zZ~F8Da*wkX3&Umb!-v#h+C$mH?|mWh7R*!#n&an2RIguuaq4p(SWo^ENR4xz2TS@L zsKSIz0Rb)yW$wPz>m6l<8I<#w*T1l&P&**_*mTM?KKb#O))3ad8H)|ewPV;?+zOCc zHk~;Xv@$?VA`OF^qJ>{p+oYLa#_9N$&c%CihY)lu)k1%u62?|=YeWo$h^24xuRsvY zHXHCqIada0XX0LBaMkMPY{J!$nSh(@@M`un%mNf|C-J?D76aUFthw4L#sD!?6Rs14 z=$tHE0DhZ8xY)D=pU{PWW!a7;9zd!O?chQN=wtBi{V(JJ)!-J3S|N0kyFkStr{|uZ zzB4lo-Hxra^Rq__8C#g;pcXmhdTo~fPhScaIJxcdEAAA%r)}}D*%3t;?MoKmFMGr5 zRSga;)Xp5Kbtk3Lb>a0fpepet!KYPvDM1#zilsS7t%B*a3DTmwq6$FtTL39`r(@t0 z{GGnvi59!~^Ff#{h>#R-(f7<cXX6#hVyF7a41Pzc5y}b)0C+J?FttV{t(5M!<j_#1 zo_M;44^z=g>l62NLZV>Vj}y@1i3W~FL$4)T-(JN6PKP0zCwKDZyRqXLEk(>B43F#6 zJH0cM&Q5>16w<}bHZ~8;Fdfk5?GUA?_Co6b;jalmc3Z(~;K`%__Z7qUI#6CfBOxIX zRx08&W5hPT$cY-w<C3k*8x{;9nA0K)K(PbI)d}zrakP94RH!Ua9FuW=fJea4PNH%X z2{6^0UJmmFK+9rI+(UwNuFI>?vGsPay7ynr1A-xmCM4qsr>NtpRVNcM?)MXSM9D`z zkhlyodOj+X57hwe?OFjHrHO~Yy)M%q*)|HUccGTr^c|*a$lD3h&yc}Ega`tdV;U&@ z)!N5i8K>12<cK)-_O0)Uk8R#4ljt*C)Gt}St3FpR#JfuPnRn$n;ZE7x+aC=O4Ya;z z!5pD_S_A&sF!UG{dQbOmWsBz$y`vEKz?FB%&;D8I4FKj%Z>o;53TfY^H3S(=C#WL6 zOhT)aM$#^zj&*|qjvBknEpKvXOk^%94%^(!8;?U?(Svs7LY-KzL+`@qLHDfqWk=8i zZCBE=!PaG}t@WT2AtG@vfzr#N4DG47iTt8oIM)~;R0CtF$e${rn}AZM1|^wJc)3u` zi$O~jz~QCu*u&f6JeFV#grQBk_E#<FtV)DBIwtL|i@^xlp-?lcCYJyT6_XSZZkjUr zoQopJKlBTDJO{}r<91()N4$de%{LlnGkcEE>gw8jJ+fLwGYo*;DC2(zD+}PZgqV}m zTIu%JdTz^1n3F;PrKEs1o(Ml(Qott@XNOYIbTr#{cOyk#g8JdIce_x+v9iso2<mU_ zj2at-AJ0=m>o&iW^KLwG<!<fZYoFxV!Rl1Nt|UK7^tpO&lZP2#=ZytJy8B{5yY1vZ z`kEZVtxs)~Rq!y%fdzagvnr({MR|1f(Y>JbSO=v~Q0hMcB2FxKdBmzrCZP3xT8SOr zAc_Q59NKQZVXyT^s54QFZ-IjJz#a$CXW3|Hb7FNoNoeExq4Kj1a!qqp);pq%Ey`X{ zg#X`5)AKiwTHdzMQAo|TJ^o)yR=?2?bu;+$%y#SoQ`};)))VVFw%WpR;SQ%Y^Tr~P z2Sae|Rb4a43LuiJP7c}PkQO}@AZZJzwe*1wflx=VdrM2_wvHC*+s3A)4O)6?Kk*_c zlCHZ3<d7No)F5Q_b*Z6t9A#YmHPFLawFI4NL&Ky{TNUtOqj7nXfiS*En^AI@$iq+@ zZ-1K2_0>_KE}~kdqzB%hYf;n;s4IKp*Z)~vn>;vZK@;QwUY*rx_bY&Q6Uh(9F3E%4 z6-X!C`)#Hs78xiX+OgJbULyTzN}K+wUjF09C8&Rliau=Zy?A6mFV0Ng74+(STX#&& z9+fIM&keYsxsn}G(UzsVH?#%}WDN-|tDSg&3AZ^6+GFIO+pTg#&u>C)kf4bv9E1ee z;NKO>;5qs`mK(XHEDzM?cUabJtpM%OK?h{9&QBQ$gND%YU38sWK-b?MPyvl#bJ?&T z%^4U|!>qyDqpzIThxQF)PgwPQju~(j2BT++5LX8=ItWHrXlx?^iZWW72E@cngN-{N z^?>E#?{KW`?Z^R*5?#3H=X9$PuN;bIXCql7uYwW%JJz<9(=<|cV>B|{*>Lz@<PpgX z8Qb@G8%2upAY)Tan(zm`UM36wpVmJcux$dS_Hu5G&Xa;Rf%)_LzN7S=Du8z?jInh) z2&R4Q3}9;8!TzMtJL6CPR9S{%K`cnxm8zfSQA~~hC_u7D#KN^#TbE`1f%^?@Pw~Tm z9zBz(|1H_M^uLHV2@>yP*w6j3ZRf@}Y;fQH5poUc!wzeGSdCDCn0Q749UKeuSiN@8 ze4`I@SxCcR&Wr(;FO1$~`g;k~ffb@l3_0U4zSmI!a8K{6Lq*E~`ZnJ}L;oyC&#D-d zLKWZDg1rsu@AEy&m`C$Uo9pU&uBmBg+yf<<qv2=nYhSqf*%E*l#gQz7))BzwXI}qs zF}t_Jb+#I6>oQ>9*SCazfU6WPSgD`OO9qfz3vfr8t#`wub7I9iyD_yl%`vs}L=ktA z>muUQ<P#O?V!VHYV>m8OH_y3*s15b3EC61l{rLeosC;?ZUfroRK;KzcovsbE*azC~ zQBbn@!nL8NhXMnx>NrhHiTe%^Ix>%%1k8B~NY1|Asd;-(QBm>5IsU^a*}<e+#qm*5 zNkLa-$SbxkfCm@{GL-OALX3U;;60H$TLY26AykbD4@WQypkP#y(dnQi1=2WTtk3s( z0OwPEg)rrQ7pU3*QT}l*=`(}|p?)qYTxJ0=A>6u@{em?tmj0o>GwOy{x6-*OP#Y@g z0(c}QaB;G`n;tf&tOYpRLt`wPl!ipb48Ck{2U*F%vND%h0AyN;i`|w0yFMZg$&jCp zM_H`^_HqLMghd%UblDIIb8+b33v*3vQq^jm>Mb@0d1^pBc0W|eJ!W)NwrQ^a0o}f^ z`1}E+=44wlL(btz57<U4#U_sy5PS@ClGe7hy=3Jd(#O|oRGEp3Rf94!HCsM7p)swk zO;gLzPfVeA63VcpmwL9CT3BSPR&(-~48{P{#rk1opbjd^HOAgF@AH#zc(<s|DKLfF zlu`Q8q#I?F-(geP3hIHQsO+&2sx8}EKn1N4#dM#ya1JEzm0V8!yP+!Mc#kK8zu3pz z#H&V3ptFP#l$<g^CD{3LneAw7G{&U4?F3ul+y!6OR`BrNnz;N~ox-!&!Z`p1yFg2! zDQHhYCIBk(GO^`A#sktm4=x*J5!WENYBJDtw;QNMxPZxZU~a$GC`G8PjH(sba1Q5Z z10v19kySm8pxrHI9B%)5Of)IBNegz?b}-Z!L(n=m2YARjjQ!XkY1D(EQm`ZOxl2;s zT>VvXhjI1HE64-X!+^&Xbs-ZMJBx({6VhJlIFEc%g~M4SpXob4dMF{uv|UxP`&R!O z@$rx8wvD#;?Bqo9P7S_@`vdHR3_5Zzh^B0P@glsGWohP^a!ui!j-@ltq0Tb;_Og?J zLo{-(I2xTL8-wx!L|d<_h6X#kcZMDSrcu~k;w_)A$q?ttGZea)M*9JgodNprGC+UU zEx*gX4c#k)5)MT>`2M2)4nK$J$g&pNngbGu_80P8>`n5#g?9Fl6*avkU$1ixSs!?D z`E*D=pg$x7|0)iZMqmJ=;q*esX19V~lD-F2v`DN4*jpW&sB?%o)Tfx}If98+IikfN zx)X3H+;T#`(*H-<dk13Ow*TX)q)1v)3T?8>-h>dM?5#wyv-eDzh-?`ZGA?^wb}D;k z@9e$Tb-DN*Z*+e?&-eNCcz%D}e{?4<@AG}0=W)E&fv7@OgGjTx%_Vt&K)>vgB^as1 zC$#_oVrPmA&X5NXT)CBCz@)ph9<G2AhJO4RqVGX)6vneD^JCGZeF*>6EPU7iGoxyp z)za<IQw&nO0yb9G+}S)3N;IO{Tr#Ce)M&I3r3~7|`AVrJQ>kGx;yP1?M6*X!*?ZDR zBd{S@at3#E4?7Vn+fp23m^KE~dd2`i-2L{tGvUEzoB*zpSPGkHZ_mzf|F9ZP({4Cm z1%8cc%<|+D0NaJGu95S`kr>*HHIynmQ5@FkgT=xQcKv31|EjokrYAq_TW7*W7l8Nj zCRXNuCQJhxR<JzP(X4QUI@o09A%Xk&{7mq#)<n!Y=s`xTK_cXsNWi7(w5*Unk&q~* z&}<5{dvIir@{<SGocYYmf87`BfP8W?!=SHLz~N;&_==EJjk$%G$%)!1+!lad8KJnX z!&-{#GT^w*pADY7xykbN?gS?BV9^N#_#1fufWbW5uK3@?3w@7GQA6c~%aiQC*WG`l zT!`aF=D-_`U51(z?$z=7$FZeLunu*bYvlr+@cS7ha2L#=Cx7zgWxK%X#qPLf*7Y>* zqw4f3(6OEVGW<`q9Opf)CVo`{e4~>fUI6>@&Pw6Hi}OYU-#i(aF+f0~UGrDuUM}Yj zdRt&`tR)bR)z;R=n#J#LnGa3rE&v>)18~v{9SmosW}9N(AW|<@&q=EYAiO)_f;9Fd zReU*>!ogH9=T7u9p=psZf9hnGG!Y@Iz)HJL@`KEuvCm918}DTQIHQ)?sYLe}1K+Kk zXSP6xFfcK6i8e9mQNqb$0D?ulC4PJ-Xl_&1Ay$gnfX`*Un2I1xm?K5+quui*1L@P? zzUV@9e7Pi)`ux{rB8(<7K+bHxI%A9|2P%Il)wQj&<)B}{@mBru?n7-mlZ;w+cBuL) z#&s9rW(=d%c36bVvIA~N_2Tyw2TU#BCTIcDi$yMYPU?<hK$<8FwI&0EmXV*b-zyJ0 z27+75(MH@Ct^m|eO*y$iYXx5zkzw*|mW>R~0snl0(+feMpPM;1bt{Jd^D&wXe1$V@ z9`yE{k3dtbZ?%k$Tv8(YC7)FQYHHNVk(4!nWV(e9fOdL@>CzgO5wvQpT=MZ6kvI8V zPb8Wb)8-&-!OUi#%NDLDnlNFqC<&A}$Z2UqX4g{2Y)_*X6QF1w(1&{=0@@^N{*5_d zo8ot8Z(cU<Ti3C`VP;*nx~$@7sZ_Cufpue@=>Lxh9`ViR$$lZ(t=YvTqZm9~@_UyA zRdtqmKoqj-_QccH$+m4Zq;B$*4DSipJvWZyBDXfmJ(afRyaYsO&(KLFKxTPNQT*@q z^lHZQT$?dLn(1je<>=ILki=}5+|(isRMh5D#{-6m)WmtQmP4aLLr4`rs^#=!3WQ^u z?GtF4)v)Q&_2FH3Nu=sS&PJQA3HkYJI!bvOrAD4<i3tf(1EyIFkpJ)bn8$i*ED-0y z%J@!3x{aE|WmDUipY)V~3}N;b5bPETkqTLw0rA6RCWO@qs>_(k6jKCZuDq%E!xG<- zbY=_wkA|+H=NjofB{_YyJA`KhKn)kW&&jMe)2Gg<L5LvlVs7Aal*dn2ixDXX^y?YI zTiC&4oc;S?c298E$kr7xPdC!LO3l0sfb>v&_BwcaMu~`^ITE-Dn|J6o%eON;8a}F^ z7u^;UE+$3g&k(Q)94l;VC%xCoh6yx?bav5HBhsI-k{QP%P@tBH8Sf3frTZEYGh_qA z{>5Cp$;n$_)3&*c(7yj#Z`qZBdz8*=5A%U{>ha(xqAgf?MJ5%=`qZxn^nEitLN51j zuNoGqsckw|t!4%2Y{^Ue2ux;7x~%h4gHQG)!=CkD5pq7){?lZppVqJL*NO%@)(F)Q zv%em|Te~~v31%t#5)vrr4knnz?k_=3S|X9j!t6xCG*ghi``<X2G%;{R%zG)ZXVQa& zUmrNSltcq3;Ic)HiI@ryAQS4lT7{J&P?BB=12qMmJWf5Ji}ZuS4?s9^vk1wjgwjw5 zaEjv9WY!o^#5X3jr)+$1!VU@&8=*1K2Ua8(VGPJJm8=$^%wacoR!}SkMpH^5kJMMg z5S69gucpvpWlWfBh=co(lI!~WS0u5Z_`XTyFqJk_@TU%X&%Y6d2+0*}5%#;s*?#OI z$eN=NO-?4LY4Qyp{IK)PAFT}=r99ezfy#;dL7KV~-DBtPC&Y4_X{IQX<(+`0&nKKm z6D$dSGe|_~0iDRfmmAfPlEjohb&G?{ISHv~`GD$;l`Wt=EE`;umey#gRVm`;%vLzt zotNQ|lF$ZcNHS4SB6>4Q$U}WiYrq=6oiwxUzyXZmzS0qmd)(t!ZKTZ9Vx10QX;xPg z#FKs3fyw+$sobD-UAz^_6t4;`$(KqvUKOf=Y0NP=nwooXNeR>T%vbL>dmK-QO{ve| z4}<GU^OrDPn)~?XL*kxhOgR3Qf#4?#-Fy^-3%@RpwYMn)RK2a&or#fKem9!*?=Q@* zdv1IOA_6Dl$v>~kv80kYSG)sKq%O_Igd9j2wF2dMo)UhEhqOW>5KSmWyit)$MpW{M zq;%@}se6a5+a_>`@-yzEb*LKEQAVsb+P#m^l|W5c*n>!Il1*8fOB1r3P&&&ogp=IK zK&M)RE+#iBbN>{fDV^_DTk&R#B2cvb;~HkF0l$EM)F886J%1`>4ZhI~;cPD`AXZ8a zkmMSS6_Mx8in*;Ld^bRfb~z~t+;o{#$6~hzskRX3NV(@|3J8D11+{QA%8p3IYL%eZ z3z%%1(<f~-ZRBJD;X3Ua%8SCWncK4t(P4hYibPx0rhOYlZ&^|u>VhV2N<E8_vl#tk zx!>bx83{YhP_lz~n}GAinzl_KO>o51$oteZW%xa682a&OS1yZ@Trc(|Uk}Q|5;5zu z7y0I*y^f|EW9~gjRd*TI20}6$uEOtA4_xwuNJ_^p9mXl(_jK9k-ieGe8(uVH)>w_< zXTiLkWTG?zh0@LX_dWr}>x(z2+JqzZN)yJB5IJEVHKM>XJ4rdiDdF;28=`LkuVvn8 zga58=1gPgSG|bd*E!}sKfbqkbx^pQFrAts6)FQX=dE?CYAMGdN?@J{pqX;VovZ2U8 z`)0Ia_hrx;mnTRi5e@*^XHhzmn2e@vQuuT{n>I9&JdM^j@uc0SSc4IM+vG(XAZw;) zJjOqZ`FN{TwtMw?jJ_n5Qkp{u39C{E&t&ihOn)74T<x(*rp;NA(^fB?hZC|3zX`iK zYl`SmKCiU7|4d>|N#LaWHKBI-6qq7B%@t_d>_`)48=^fdl%iTW@b#b^677O&Valc$ z`w@|_uDEQix}ab~M7u$=*av<QQy1GI1-E(AqV=B|kqJJD*9c?9j=f#{>0<~#Jm&xj zZ^LwqSzb3{OOV^g-+h`=F-k)^paHn&)4ee*=|88a^c!%9TBn2hJ~Lms+*^)Un73&T zDySxp8}?_*asVXA^%AvT8XHjh;iM3yp)gGbtoY_5kIe+3>{=I>B_Q^kHaE0H4zHef z{QASR*(uq~Q9&U<Ygp5x)RK=kI$^7p!A-@9V+#7~``8hOXxO~N*v_gf7l=TJumjhk zhVmbC^<Ot{2f#M<bv~bW%1iyA{e&X_P>y|prNy`5cJ|#iLIJ~{0q8qV)yj+WCaIvT z5<xY)(_TVUiYOO^Mn}q_(nkuXNjUIj;BdW}n3R+Wdb~3H_$(P%?EMu9G>G@8y^=ce z!ptp$^*2mK%%1fv?xn+^-atTscb9B?;7lrV!B-em&{Xs^!MAWS>mex30oxAG#MjTC zeun8^+CIK(l4T1!fe<o5n27_&I7?m-pC4&`dV$a!#G8aOK)@hY0d!Tv!0YjZiR7rf zgziMNZSJ_m8jwk<%()Q;%s9A@m=#QvTs6zTK0;%f7NCfoj`BEW9w*mgqu_$r3p8RT z=Fa-zH(M1V^e9n7BF0`$&L6v4$1|Io{P-G6-va<myW(c^)wPz-HR7(j^u%F|#ut+v zv=&3Zj^_*Y5e+!7G~Rrb9lswvW5XBqso1igO8%AA+Zj^Om3Uh5(JwSfGIc<UZGyD6 ziEG8sf805?Sh|Gt)Oxj{!?fSyu<PEIfx}_h8|xp28@PDg1XB$#zB-sL-}|_eQyP?E z|0@){YVgyZ>BPq?^5I@DIAyVar?z{Im`X4f3+)fF%3ez(83@t<HHQgZa_%P$KveGY z(TFFN#F`SmuM56;g*KSqdBYabb>kS1Xpvg31{9w1Om#|iKx|tBO(GwQ{#@yZLWs<4 z8GD^5md@24>|VLYC!hoEeuymar<_@i#>50m)9emC#^akHv+V*ql@46#TAzeJWna5B zU%CV`oG{L3pLk~AuDzwe*VY|RUYGz+qg&g9cnUF*VAm%JU$`JQGOdCb@x{7t;A44= z82GwEL4AW2jX>M{I!)LXZrHsoRaiO%3^;mu7ZiI|esIg5CKzEV%RnE(thkkMJ*5gq zSep1PPZeFdB!xNZ9ke>~tp{Ti{|LRQdb=Q&IX_{Ot|8g$(UPw?yR?h0(5nB88<Z{e z;5*35%wKd7x*gE0P}##lCAC^@KMg$+f>q3LdrU6kA|SBYvnv)hW{x#=aBt^xw!d!3 z-M{cHgk_o{9^Z*ia<fJf|8O^|S%VoTp@FZ#cOb2jj~TR;F11pKJEkbL`fvhquAJ7s z_p23ZX7R1f+)@vMF3=c%J|_27t)+;RRWVq@ZPHe{rHYgvn&cd!P%IAKZKs!6rhP?- zx-hTI(bWM_Gb@y5EkH|d6+H_g$4e*cRz?pOCwxn%WyP8q7NlCOaTbSHTvcR7Zs}~m z=@5A6{E-#R>we_QHioH@$%ZZ(fJifIJA=4MRDpDLa6X0bY)X0+t;tQ>Zm9mdFRbIp z&!3;JVX4pT$aoUwUI<le%JM`@wJH17abf1*nesC!7|Q}}H029f7qr;v&^FxYY>)OB z|3_{6Sf@LTz8ojqK)Co5zizo-q`)=TW$ZKJJL{+5;`HFK7Zi075+tG77?*KL*8(TR zxur)lZio1^KQ-hPIRJJ$hB<Pmc^S6~2T`t<C1!PYcJqOIAJb?5Si9qp5m4G$IRi0B zGN((S1WMZLVsx7VB<YE%ILXe!W%3{mx{OL}sio&D=v>5`6ZaRr`Zf%&Jpw@Xm{&7^ z;CLU|VQH(OSGU{WkTF^7sDftWqvJv)qJsE7tr!r9cyoYT&qrAK_sA;E=FjpYu&TPJ z1^)dOM{(}6Qd^*5F<KgH1TbuP|LCjR3!6jy^oTSd%>8$Kw#aq(ffGukMYDOMl;>IX zeQz9g0WE$$VIAU@cy55Rf*cU;m6un%nel6(XTLq!mJ+FW&G?o3xbxdm!kN-zI{3|z z&X}iiV@fUQY(WEO!1MOH9g&seM?o1o`LzsCmnWT1epOa$3q`&{JdSYw=jHa1*$WB{ z7%Nb;2@?*)deJV;J;%Ie^BTaNVwg<f92+?4q&0Sl$Z6&`8-6<*BcJc5df{BeZ^sUh zZ4L&+kQwlD=CASxC+jQ)9hM))fmhKiGLmnlz3c@E73B=AX*I-q-02#j5a6~(^VyXp zRd6PQ!>R&n?%4b%ymxo1K-GIU!#&-M7m%Ecn>0PUL?I>c0C576A%~i+t}7ghfDVj^ zHF3fWpza#H?}V}S;atsi0JkZjRLc+1J;vSx=6;O!I99h1b3EI9j_|uz7;S8#IVc-a zOtyugFT7TfR%IFhtVAO1NriwQC-4&C-yyv~%<lP!RQn^#6K_Pr$8OZloYO&&3s<|R z25G;EzW&SeQ3}4!z!bbYSW5bV4=_9w8r}Kx-TAhOPNx8Sg-byprmv(N$J9upsX;+Y zk_P2<sW;z>a!%6grP+c-?`gye109#x%6uE?vDj*<7f`98D4doW-{Wv$F_J-{?&y!) zR+z1+o+IkNtTxN%eWEWC+Pd25hT;Xj>WpOdxhLm8Cfj9_@{7Lb3;p86tt?(z2G$#r zqNuFSDgzePtgPuKUb=h{MYZ||X(FD2l>(8bSS;JoGPb%B=0yxYC1>#>7V8`-pr=oT z-E{id`Pe;oBDyy35gfy}{jW<u2EE@37dp!6>Ap=MLdeQrk0a`;BWngfp5kx@4N*eT zi`G-^4+Z8@wM+Lq&?|n`r2A<z-W1co6&PSbp9}}Hv*pCE_y)U~6_v04`;^mWUTr#0 z&L%0u_lBYe9b$7|kB4!XvMV%AwxZ}PMrv*}d9^4#O;P=*MMcw4e-=m4+BfXixiDO< zTj_W0BpDesqSt>fjH=H7nOj(%3W0K|Wx1(P16NJ(&9V!h9$G00*-Sdh6rGe>{Ss@s zJgm<4QJcHevJXS7yH{Mnn%BNk%dK3Wn%!(QbHp4$7R0qC55lsE3jlZ=;o>mv{3U;@ zHKEPhxCd=@{i_`m{FhrQ*`6n10Rb?Q8{oZltBGI1qaK71eo#bL9gIa$z^X?dVUGmi ze{5AwR*!wl-40a-jROO!wQ50=Es0O21KA8)#qZvIJv2NlQ6Isl3qJA{evMT4u|us@ zHL8t029iOH0f^)J;2hiO#7pwVimqKZgOj^ZgkvX8oQU0TpQwnk%y<d~Hpjr)*RM2k z5*>u&_agZ<SaTBwo}dn+NZYcW{&<EmNOtQCGc8g<fVk{Qfombk4-H4P(Gja~%vZnz zwYqOvTvypx6&hs0tRXz0+M()!nHW`O(9&2Q*hgGyaLuB;kKm(^;@eh&ldMNg*$m&J zXOzb^%cjX`x7G%IYNP9fY>Al=d^45_#cYZ{z-E~eJ&2?wgvVXdpVU-osnyZZkv8kG z5B?20T4uK=xWdNRGxnxQu>nH}jesb|hLPfxJxs|4OaDt^Uy}ZuI@RHZJ~sq5?xS!@ z6%5;C9QQZ{#bbn>_%IZU?8W|d^zg3w0o;1e1U-HozyP1cncI_?h{pO8sDu<D>%{RV z&o@o<^z`l)JL75&khB*4*0h=Fe(J2Xu(()zfPzfRG0o}gW*sj83wr8xk$9lbxBs-< zOGrY(QEa2=X_YU>L=Q8z_9ysQhNSV!`4Mc*t0W8GRc(;?#Iv!!`b+GEN_1Jfztt~w zrps5AzY#g_Jr|@T9{6Da1);j#L?}Y|Bh!{N4;ULYrG=}=tA6>$y^vyy>P=E%k$a`= zQL4zPTviP%CcXTCa(!b|JmqJ+hs3@HvT}4@hWpgYzLOtoJD%3UD$fnjat1jCh4KD4 z^1I23s8Fky`W4D_uMN#Kb2v=8m08!~by|!&xh=;o0`8OP(brn32&hi3p{}KqapNb? zBdW7qIrSG1%lmjsVvyBqid$C4VhplOi9-!J6^&JMOpJei!fTxF$_X>*?+iLLgo<rz zARHj?ZK3T)uo3;QT4i12uUn!Dyc-GSC+*-Zi#Am%T|($HbxG`4d)ntCo|EY<Npnf? zmaQ_WWlL#-V)ztI>%<Zai)Af7001l-3BI~zqv&;IQ+%K03#bptm+T8u$fb(=-HC?4 z>wEJU#vy7|IxN)CS2<C3GJNt^yw1@jXBjZiUiiGFudIw>bxkVfAm3;ke*5;QoUvw4 zCfA{6wu<Cc5swBogI~;}Q$6`@wrXvS7I0iEef7(F(Zm*>?2U0qP*}-IC}Ow4vQiVY z3t+I(tE#h!y-#qs*43{?qoDsX=BHW;JGS(YbW~5_TALRe)K+Rk=kddRawr`eu`xoH ziQ}_xH)kfxE@B+i9yFVtXjsfbil>pn<Z4XFm*NBDciUEj`Hx1=Dmf{Cvp;;>hmJ8g zIg*%LW?A-tlXBU7;ipl1G>{URUrm>TPUmz}%6@yPEOC}nCxzhm{?aAEg2|eHhX4G9 zLz{(nX*HRta7TX8@4w&ef|P(Aq0Laec#gxz$GW;9cgqu%QrJti6Um6l48?JP|N19u z<g11e6&ffQp*|!9)Swz{m6d~!#2T>a@b%@z#Kwji47FOL9?zVWYMWMlkKq1+GW_cz z-c$l66tvQztnZ&cKPZ>hv3`NC0&ZbAj>Gc}T4+!4w>YUvtCtV)lF^k+Uyy3EVBJ^v z=m(+PQ;o8bPS)e7hpXSei^!epF|u3}qhOUn1vIyab-}h~1ji72z$XoS<u@!hZa}dC zaReWZ^!rc=6v<lI7RItMDV19Gqzv@_ctchVpIs_fy*#FfyAIfnGkMhHZ(C$@hx?VC z)DH~9tJ2cQmYFE+QFP$Y(c?H=lWhKrc`^|6t*C5`2@{i_&2isUG*hS2Y#<<6>g9pT zvtW6$E$kV@*Oyp+UKOU2KYQ!(%k@M1h%u8^S*pb?tggJEpFb7VYFtN(?HO;L;AzlY zUKO2%rqIQ>&#=(+*|TRqHC(*Z*>HzEnTpoF71%(A!Ni2`?JfXE*cQ*3feq+2|Gr0T zxIhBp(x8~9o~w`y)9z)`nJKZfcxP0bjkpQ{TI8E-j>b%G{svNBrjGy$L$=5eRh&j* zi9-gwA+aXItJ6UCaM7zKQ+6aQ5GcI5!f8e?wm?Yf8vV%rTJJ%|KRWckG&8P|aLKJ^ zoUgLn6@|PJyhzYo{d;g)hjlC|zymc=O3utA(@Z##mzNitkek?$tD{A(*sA?RUS9rH zfIr)i<6OT`WnCTPvU2$*O82Axi|#D6`Qyi5vho@dN&BryrEGX*4~%l_5%(In#IO}+ zA4*|cOCar&G}L9gfr?z1sjseP8L%ey<54uXY4K64Y9$=zrsnJAdThE(dg8>xgxLIv zvKDq?Z^MRyr0h<wUZO&3)nP4<M)+P-_QW`f76Sa+w{O1=f6&YS27Zb2OEyBq0fJ$Q zGECKcG)zAH?f2m-UWibfnD0y};$VB397z=8q#4lp5dQ+mCHDK$&7YjD?WLs~(QUsX z-s`8&MDW|4REmUg&B#=ihxPc8;5|bdY$41U?Ikq6Gabt<0aPo6wB6UX&VN=wZ9wB3 z1Qc5H*7b_Fz-d=ODVws8lluM0mlz}e%)IfHn{Jd+0)AoCD)<7Op?5yaUgA3c{(r{T z{Y5k;SRw>J^T459UFN~p(1{A>H5akSpep!*m+#0j4C*#3BQ^(tKen#_ED~EdCjf2F zCN9x;`~FJFq7Grcta$Q8dHbYWHgv8HGn|Wi_<l;TKo1w1*6cdp=D6S+^4mymSPN3i zM0|j-9+6`gI4>4&ViM&O6{SHgNz0&`IoHL-a`WKvuW0zUi8F<ClNJ8I1G~IVO2s`w z<El&c@1;Jp6@lMW?Cki#-A7S%MQ|>KS5*P4OdJck60LjToR<a$2Fxc>IaL5RX=YLX z7&{?YVv4_p9m>gRjR*!OBzIBK)fnsSRs7D|Gp`)Ef<NlSQYg^L-%cRFC&%0Cx88?G z0*kiF{HOnL6&weN-d&D`xGT<EI2(BP15;C(T|&=TKp!e_x_{Q6s#iYlM$GQ|whl48 zS}~7efB@x8o;t@J5brO?_QB(GK<QY{p11h(2AG|^xHfw58!hsbTxAZ=K|^ffp+o?q zPnc&Tl0NGRi-@N_C);S)Tk9eRGF)d``|v+-qX}d=TD*DZzx^)@=u7&jnGRqDSI<qd zT(g@~b+~%u%A;!!ZQmAB>Jvex`?NR9pSZ2Ref9!Kn(WChTG|e4h7U_<)Sa8wko$XZ zkO<}A;MpIU<2k;qeJ-KE+vLytthih!@mkG<guB6GoWOfnT+_Ko_GZs!fcBno)c4@! z$K2?Dw*6H0?N?9$d`zLx;_Y^lTS*x5lEzQ+I0>-yTXYnyld(jNtzrB=bavPrQYyUy zO?Rn~P*aq(KEBq+>YbMI50U@xhooyUaz5tpR5|XRkBDaQ*2QOskp&Sffxjeqs}>QM zF5NT|+vP94eSUO)d*97lNxRnA%6gf;A+~<6H5V*FqDc6T-nH5J`SSR%uM`FtZ{9)h zKeyLNbYA1{01H0%aToL{sjX)BYrCZH-<IMyrxkpQIFmn!hdi=V6=ni&YHxo>LL~7T zuY%v~)!XF|KVDlDpWpKnX@1PSKQ&3d;Jc5zS>}Z7#VGY6nNv<W$vcnEwH|p)d1~KW zw;z)ujHpDDbU!B~zx|-w9D5h4k9xlMOavJgZ$;%%vPa@6ALswiIL*HbEIuf~!rJ}d z+|9=mfR8;x!14aZ_CK(N*%Rn;38(C`Cx;vvyEwKU{UoE%((-%2gPl??<YIOo@BJ0C zD=XbuyZ_^9?MI;1{vHdxo$OTC<5HtRI|Pf{Cs>m=p=*<YnIDY(*g1M4G#3Iz9Jm_J ztURV*lk+}&Byi51YVQe~{mHPQZH75Xb~Jjl^l&*4_VK>h-X;BPur8XPm-D<odW=gD zaZ6ZwiASD)B6{?tw!J8g1@RusydROOYj;1<+fjZBUV`uD$*H^phn0XRg(O_-`JW{o z_@5>AJWy_al}SMBYcQp_&Bwg|tO{C7FIk}wsPd-Z8T#f;ah0CFCdf<mu@O8R-_+fm zx`n(8TdTq6w!hN{L=?JSo3L<B;#u|d02tVV`lBvBmF4-x{&3H&T}3^&%F*X1`X;cV zPg8~d#K3^)J?BTAN7tbsd~MW5tVFGOjnBc71AB&1m7YkBta^h?%Qtu|LxXZQ@jPPI z`S?(MXBXMRI1Dt|v+?bukEkjrh*Q#Y>QHXfB&t`=ohJ#g_&hi%`M}^w4kmtrnxeAS zKgpfSSo%tWVwPuuN&QZI=U?BO4Y7+nJZdj`xzw#^d#`NSVx>VBdhdPZvQ(}rav?#7 zL6dIU;JRsEUCR+V4G(MX&%-)+oJJFJY3JFew%6$^FC2yn5l*`5TnxHlUuvaum_8O< zYt%gU-)%%!YJAqmo0V~ql`h$)K_mj|3=S(ZY(4peu}}0U{rFw-?%z3DEN4<Q|Kg+< zTpA*j*8i-wC;s70(iQFP<j)e0gK%A*Uyc5BXW)XA%ywHVqWEjrX@vV=v(tzo^YQ=q zD!S2LpPNka8t&w8BRiLsh?uTj7_Q9b$;9&A6h&Lk$PZm8C;oxooaF;8x=<+VQ~DrH z^R&Sm{2A)gkA<2W=Uhs#HGB_u$KI?J?%V1aSj`DOf_zqm2d>333JckJd}F(>vXk>a z{ZQ3E_xbtV`@HH`2=yK%Mcman2cd)5HZA8gY_3a<?Rve-<`ro8tDE_K$5|r$iVF+G z>-0P2m%5u16f)|q)XhkxU~reJ-PrqAO<(T|+gXpuHAM*>D3^283(-c`cM~4kAzTIf z4<Fwk3U{cLf?M7#;c7`=T!Li|FH7r}++l06gUzCcW4nrv(f{?l+xZJw6Ss#p?|iAU zWAl&3);l~c+L$&?0o)>|3gmvY`PQ?6AjTBW&o<Uw(<u8gFdo;{W^pXm1=BIQQ1M-& zDKqK;CdpmaUjECh^Qp`>xLZjmGozLN9DaSrQ{8P^^+P#KO&Am_CP5NeV?~QWUz45z zbLuJF<&Z6>5Ge_-^zHMiL<T>p*axaBXKvxxVfxguh}ofHt*zN&$<7zmB~R3Ns7@ZA z6rjL<eq^uwZsOz_^(z-gYOd~4;>fxIK;<L!;k1;?Qd(s?=Cg~WxcJM=cZOG_L)qUi zxgW=_3K#{1vKij3vZyq-t`5|r_by#3#I988_{WN8u+-5UjuE)AJlV3?Zd=%RqSVs# zYt!zM^D}DW&n7)ov=r=ab02RZEIK3Rl6m4@jx(JtQrhm9DZyaHnfc6SaAI#(D$3*Y zr;iAvV{SWe`JOo(OVHG#m;CP)!2cH2d+#SDm5N;W&e}0Q=yPkvt?}fkQ^q6@#|={? zs}>i(=vHT3wc326mrp7+YtzM$zSPn-45?i~L1av<1D043DvddN<{q)p+MSHq<A?R# z-ks+-6YApV=ooEfnxb4<{QB5;2M5hM+$bP{*4Eh_NiyscJ{HxLmG8T|)f-<=1KL-Y zKMc-h^QE!-8GM|a$_6b73PWd=&a<2L(%VNmtj<g~QuAnNRNEoJW7oTKOoN>8WsCs3 zPVE;6R}18vff&#V;9{jXFb%h%!M6xvY-?_A)&*+E98yRmpj%R>EW2}XLsYO&3A67V zKbOXX>iYU1DxyuVB<DKzr%m4rww5wJ^G<^fTX%Ei<!}D{@RT?b+y?0KVS9YOoo}fb z9UZNwhFzH!709Q}z_03yt72WrGK||TiBtW~So)0}E5c^ktO)WIq*71%Y_-1pFK^Tj zT2WtKk6&A6*KRnzp4PuL>aw^TgKS4A?CiZq^jUd6_%=xX^f{;TfP+5`{Gu}5f98$| z{y=Vlv$G&CQA*CA;kHPFXQXk4URieN8+dJf6%rLC;4usX&HP(E#s)GjZUF{n(F^Wd zYVbi@zUL3;(7&ke(Mc{l4e6~(>UmA;$5WCZ6HzI`_V*D9Y3Tsy&^y{18pc8AbRCfD zs7(FFZ@nfAxsZ9n>;lpX=un*Bx3?>Ipsf;HC{hluy>oq$2Z*|k3WSy+Vk;!HqY{*i zQ|8X#rwan=tr{XgMLdQTn+YB*W$1u)|3SXBv7yDQ6syTrhF@Y51d(<lA=eMiP6SJb zu~&j*>!Mn=(R<KD+yUJEO1||pttE_%zI`D{-d*q~MoNORptZ>Se@@VEZaQ3kN^^b0 z+g0%j70H`}Ro#1@p%Ev|ID$7DuACYkl;ID^v+#?lwL?v7+at%)f{xGKroz>beoRfA zUIqUA)@b~IeGC;;IuB#ycJ2{JWKMJ=Hp8|%NN>>f>GYc!ON(k?Lu=*U+S`Y*8%uFE zr_rIlCgaJXLX)RK?ZIS=zF9hd{2chwYrxN8<N#vC7+^&`SMpUm3LSFv+3a!Tk&;Y> z^8Db4#xfJ<83?K)!FsywsSj6*<Wvwb=hd4x@2^aC$Qf~9TEr0g#S$SZZUU3oB1Oma z1W>Wp4d>3Ciw(7m_Abo*6ws6PAfLT3=f;g2ng!$<1feC_#$Atrh8Ph`j8v2ED~20w zxe#*dTpVM9tO^Db`429_Rgxh!>%VZY8Hrmw#cfj%cxYnz5;-oZv7RdtF-aV3W;m+p zbob&7lRUl%LE>1de@c(4mD+3U<o|2h`Tu<wX&-z|6uV_J%4j#*<0204!f?|YCfYAC z>k<5IeKf@zn8bqosakjX`xHFE{4TwVa7Hm`IpM5?IB}cqNm9ZpnPx@5=!d&ni8P|z zl@7-$F>?m(Oi(Bx17?k^hzPs_83l7~#Cn4t>oqQpYBZ(J;l-s|1F05l&?|&bDxUQ^ zP3OmN1pH%!n6La*aD-GWjW<tQX^j1f_jd%5-Gir%m;Yn|=yk#7xk`LK(AdbZHR(O1 z&s1n_E|AHobWd1zr0ZJeI&G}Y$9~Jps_#)E_JW|F&SNn)Q|*lhd)ZPk;D3Lq=#=wq z9ss!@2vN`kO}{199M)|4G3wg}aCVZ2xR48?WTrQ19|_TAO{!;F&2+2qZ*45T&ZTcV zybsSwdkFTOHCHphXh4|8Qz-kqeft6AG2)T@PBmqW5cc{3z{b$c%mws{ma1B@bK&|= z&)?oiu;k;VF@Yk=DG2!R1OBVHAsUWE=T`khg~Y(46x{N40$Yxa^#`4bdDv-Vc??q3 zRrXhUw2J@yDLk^!IKO=DMwW#xaZFRPBy(ZYSl)jg2k%OC<TRO7TZ7$f5r=q&ySoX| zXuUr<n?ZZV=A#~aJMAf3t~iCH8VBd(jv}+hM=HT+<&okw_wOQG?_Y&*58jZF#0r`e z4A|#6d^f{7YN<g{ZfNs-ez&T?gMNnSiqACqwt2dD*t_TxY>zQd_=u>?2n~f_9DR>Z zbS9WGTsQ{dQ9wceYewThkK$nsjtI_Yx4)H@iF$;Q7qlky$8{E*kfH)0mKU_QditvK z8g%(yFJ5#Cx7g{QMKXQax(*$?KocJyzv67;F>Xx|kO~dY$%X?%ewe5-V3=w>NXX|} zR|}W5z4&X)8$~@M!xaOec&-A?!CT)VswZGtGOUf@Q}grltHFS+;S%ERUCqmnBJrTt zD(ZIRpYF_(gp*CjCZ=yAsxKl+nyLWo2TCGzF=TD<`cjvzygn^1*>U};Mf0@eb<_1h zoA%!0`w$SL|2m84f2?EiQizZhAd>mG`Q?k#oXk@>*<UD|S;LrtapK2@7<QFZv-cZ; z{#WYxS>!si-m&OdLe`FZFz!${qnP)F_aE{yhP&M#g%q_Pj>Z1gcqp~oiNF{uOk~NT znDK`cMcmTp3-=i7p7WKyD)XM%dHJ8BEs5g{9Wdi@!;r+4k&gz*P`aP>^5xr^6S;~C zZ6z5m8ncjGj@YVHMkFdop0Z6Bm6ZHY0acWIso9ItUR)8SdaX8;%^!$lnOiI@q6fZq zc4iyXsou2v@tSl2uH|Y-mYbe)FfvMt*LIxA8?8dTTi!XF>&T(V#nmCNQGa92r;pvZ zvjQaGv~WM>oQtyxe^Hn98Vg42FREDvvA5AHlie%RUDUN<96^KSK4A?Qw*v%I6Z77s zmJ|*^UOIxZ^^=<t#a33ECLOt+jR}bU^5bu(XsQPE<ONtAYqbNEkd@@bSk`iP^F5wg zd3P#a+f;Q}yOC6|@l}8jrk3}YYMA0ZrSrxRD*1w|K533*zscdToa^6z{Ez@)cFn0K zn;xs60x-bG#=&OB6d}J&%}w&7Z3H4ofe{gmDAlZ6iI5`~aO8s;w%ZOIJ*Ks5OSn&N zt4M&~%$;J*Bq!|3NY_!=SC4zey!*Vhsm4-eA22(?U11IZ58@^!$y~w$*d?{A1s$7^ zFA+Mky!iSI=YH@WeZl{D6)pU@f8XgE)>V<B3pI5&ExZ#*X)m>S>^Iny+Od)S4YA#a zCvzBTUI9(tv=F{0$ptwZPI5_2XwA>=-L)R4es)A#_RP=%r5i*hFBPdN%T+k6!Y~wL zS1$<_iB*OS8$hjAJZE>u%#gw`N*u{(E!|4xH!Me<NHK1F%YV^<-8wIY_dJ}Sw4)od zM}8noZy=+F!M!$SJz%S)`yY4=Vm?bJ9yhd=?*eHbP1y@#mTGdTtU%Po9A}h|;?>rR zS=qtCXB&?vyof^){!mhX=6M7)xlV3o)sneC<B%pQ+kE)0+`O{zP#p?FNn<ag;pK!R z(5;h&FW4tl^?v*iKmxuBAP+;&r)Xuoxtv~Yu`vMTyIN468fEirejacp9xznyHXyfc zam_ze#?~;?Ud2MdCxaaadCQSGo)sMiLrxG?uK%_bKskA*kpF5UII4fx_HGn?=@MvI zq?Z;&yCC&e3g2oH!&JEBhl{F$^rOd9?P;xyIhq8-<c}#2Hm-hh?j6hFQl5s$-vEe~ zt4jiHoG>_2@D}a9kTi{r%E3XGBs|>)ZxLkh_#=WX-V2`z4A(RhiyiWTTggr}U5VU< z$_ax0fy&`H<2ycc%i=#z@0!EMLpg3nPV-!4Vv-4sw2Ngm_|<UQW63kk?$J@e`l7|D zcFqR*n}FF|F5Yr}@Y7oJQeFqOdT*hxYKtx=Gy-2}%z$w<=!uTUA>tuO@3~$Qk0)&m ztX-f<4?1MnxGOuvcLnS(I_}}*W529HFdoXB%{W=KNLW-Q7N+CdWkj<kmKYOh)vB_9 zI8Ah!HR{_84X&9HD;qYyu0e8fS%D4STnH)-Hdc+%D4W4#3Bb$dU%a`Ta$WypXeh0s zwj5o6;Wb-hRZk}nI_RnRAX*p|>*ablt*1(oNL5<~!$+GzZr!+VqwXV*^ghKcVfC;V z7}OjpnT$+|Y4$>93Ya(~5PnzV17JwxB!x`&wICf1o#t+X^soDx&g;634O46LgS9JX z7*yK>=Gb@>0|uY&EVv~u7T;-Wa?7@5HN&~ftte-CQ5*6g7hqK-%*;~lw$w<fa#@Oc z@@*um0~u;L4(ad10WZn;s}?LTmyiOyBbLOkCX`JQiK@q*MRh07-f-IY!vNZSjH}m5 zN=skXxG)1|)6$Xo2X|j{O6%wa(q=w}Ni6Q#4~86qb1zFRKW;=M32gR?K0hdFTS~GG zm;5IU_Hh~{Ab8>RGu|B-!Wp51F3R#f`slb?>2`8fZ}QFg3JnXMKwBvlYlD&pbVm#! zu*>&|Ba#(*3sP7b=iqeSq(OWbyikE$|59ku!);;Y=vZLXnOS)ug|rk^lA+_PCOiZI z(g{aChx!w4K2x~a)0tz`acjIQCq<=2JL`}Hg&Ne+L(t%`g|((=F(FDPl})ImMaw@& zo1zXU3$;NTA~%jpE2m^BmjbEC&$s6W+clzxfx@qo=WBgO9{3UBm^x-TBawO_nKFfk zEjTndSl`>QD;ssBH8jB81q#t1n`4nqD<S$Vw#JBvB3>}1Wh`sl;~;mvKefzV5cL?J zC*E%uV{HrtRv?&9y6G25_D7tdk!03L$<wVqBOxx%JnN#-b`iZS>*dVj@uC9u^n;#Z z>+_(u;RV^*$OaNHE$)J;J=7{phv9l38L2g-3WAx_PW8CHEnH;BC`eu{$7Hd1<W8m& z92dvzNE@N5eV%TrdBv`Qyt3W4;%M^l`tnfk4-H3)D9sC`;F$Q`#I^@YgokM91x;9+ zcXoQo{fod;O!oFLFbS*$T#-nt#!;G5Q>5^HZpZ0%M7NZ}DPnj`BkhrAynHg=7*xFS ztQG5^9S-S7A8Kc-i62<@^*w$B^dteVAh(ZJrD&u6ZnC$~3}n8ZA&i+6?Iw<$w7PQT zN-Za%^;5~WMyX_P+JW&`O4LR}E7vR<yrkkYa8mGS^V`g*=i8&Lyay-!u1P*FamCYX z27C7ql4M`)y;xaQb!*X@SLQ$xqO1x-IJx`x@9T9aF!y}qdV8MZ9;<wcG!;zUYW7BH z6+2(IO<uiIR!r`DEqI3JdpZYx6}c3+A97(VV1xWs8%iH%9(Vo^NYgp0p{uzC6!&nA zd&==<Q@MuMqnL)b7D&3LRDg9)a-01^CG{{Q<poSl87->JBg)<Ira}jsQ|xV|V66sg zoVJoio0rg%#e?F)=uhMzA@*A-ocu$DmiXmt%*%c#ytEI2V@jm;UjN@6&_8sIEmT(M zAh+>919V;!%InQB&E`0sb7{QG6`_{tXWIDw^{>N6{Nf{5eEUO%lC6WDhUtjTATuiD z#o7pel3&$8TgD^Ke)F}tho=bJ2+I&*5vzWKAnS!>>G}}nhwe@c<ri~ah?IT%Hsyrw zH`$S;lLlpxpuYEJ87gUHc7f#JDjQo%5~!HVZar<f+DJHxNsvxVF(h62-5Jhh@p!1> zy<X0tPPVaRROULT;=|t5haU0W$)kCMvH2_nsEaAl8pf)Hnu+Z}t&kvb{kyZwkq_5` zll#7!0fE~UZRR91?!kz;SOcHi=tWbJlfz=aC;&Cx<dBxpJhi&|`^TFYm#s}v4AKYb z0`Q`Tg~RpX&&L#&fD=I|n!>X-5T01tK!VPbkQeRC=gC2e&AJYm^jEK6|2gsLwkF-; ztemHm@0-(f{eKYPjrml?H0t#ZZ-g&M=`R*`C!+jr06>1dUiq$G;18_$?`3t=WoRaq zp5JQQ{Svp^`vcBxgz-H_8L0)S=<~*)zFWTi@P3gez*;#CYhhs#bMGVfrxe%@fhVrq z5eFGu43wP<P&$MiM@MwW%Tcd`oEF-#M)OvcCzT)?R%#xFlj$lFW835MRLBJS4RTkP zv1qTU$q$)kpy;f2$0w3%u^_H9XtqPdnLi?8qpgT@44Us&amyIT)QSnO18UD!UIb<0 zb+be|xD;k;F7fmCCU1uO`|Hqt3VRbUM^d_U(yV>>zFYdi<~XU4%b3)c?5Irvm|o#< z6!9GBx40N1(?x2+IO=<qTJdpH2B4wZ0KM$UdGaMZ@BaS&%W>sDe^vo_P{Yqxo~Hp7 z?{)f=F3_i5g91x5S{uDM?@f}*IlI_uue<5A&>B9LGqqphnTs=z#5PW&nMQKwgVUE@ zt_5+Eh{vQG`iA89*P=OQ)!%FGfwc;j^ua1%yC`&br-;hqxm6KvI;~*t*%>dK$$4&K z&v_5sYa6RJIw3)(VfkKvcqvvotO_xNHK@{^HSI0vm{|lHFsnj26Qq>Adh^B~W((bY zN6se2w%pJlKo=BvX2_Z&m>*1lc>wO$*s>l@8Kg_LnrgpqQn;+x*sy_kKq5a+{$MMw z8iKD#R)-lfBMnU>7U0q8a<iH4BOC|1BA>+z<U@^jCp(vf@TPMzVG;xDr4@(mst+oQ zYG@r}Af$8T{HY5rUThb8bD!xn_Nm}D0Q|AZ^IAa;TG{*vPP>JnM^xH~Y;6N^KjRFu z7NI*LMF$RY(Cce!3J71Ii=8aR9QSeVeR^1vs<&~U1C+~z%Y(H6RT%70EmxyW2Z+T_ zNiktfiNyZlO8#;Vxza)PmMq`CGYVkeeofM^Z~Obw%F2hMtCb`+=lgm>)nwl_KdzdX zd60jPIH!(^$2tiSs)l!^>q>SMIpxQnO%5nSt-wCKyE@ZzjfO@XfbZ+70c&7tWOb99 zJnO{3P%&-7tZ^WQoW3PiNR~q8DT9E37PK!{0oZ5tLR`7#y>_FdT*fd_D)#rn{9t*E z^>n9se+eE)uMS9t$vk@WsQ`@Au`+B=dkgYZ5J}FHV-gY)w;WcCnn!_5sIjf7+S=^M zvxws>hPO0yj;YjA>{4Li4OvxY+)$<eNZ}W}rHTsiQ)8{JXdS5gKO(`Adj5xsz*#$% zoBv&RdaK)_J^-bb6KZ;@9keGRqM~`c<@1+L;#Lmg&fu5F_MJ|H&LKm>V+pf90s^x7 zL%41($F)>_D=XtC;m=;2b11Z5u0iK)#lJ0<>5Vvr3(~l_dwGX9ENs}D&;DlA73d)G z{_+S-)h!bH&lT{0lPQAN5S`7?`I6zC4f6HqHqwrUDry^Pm%lS-s<>6JTOL##-t?=^ zKuhKh!!xEYv83OSIL+H_ZL~d#YY%Mn1v8-5{)8u&_!NH;0z5iZ*-~Qg^CSsli@{aW zly76&Tn|onI@O2W2C57jAfo+HP{5lp4pmnGgwCqv*CrnN4uaQf+}xeL{3hL;NO*)w zG2E9TqvVv7CbMnK7z8Os#_QB(gAq%3q&DoytWCI2#w(quLP(ZPciD12)!pW=f#2K{ zQU<rZ6sg0s1zab7;o(VdV#nL9lF9G2dEqbFC?#?lnkmi(u#Z-)>Rs8o*ni2b2@vrR zr1vV?Sdg?3b@G|W*{3W`7i5^K9#-T9NC8Si*WL*Z${28Fj3|$;CR-3x2QdXffJ~=R z5qR501x%By;Q~1o-d0cMv^E03_{kF|?$ZR%tZAAb&Fh00n+3S;RW$6Dvu<tj+gaVV z)MRwqCCvG$?Ib@HUN}sYN4+qtZS5UmutWWyKHRM5{g(~(582W$1UrlJ`xm;MizVT; zU=)6{5>xM`>WGcHD>-rX2T;QccuxZInNT#n0RCeO`unvT6S}etTGDgIzd=CCrS16| z#(;yzA~rqEHvIC7t3Y=FgPqi{>E}KU?`|rcyJ3kVOoO^kZ;zmx{FLB{3js@$N-K{1 zb=&0A7<1xqnK1TRc{Q*E`%IhJ11AGmL<sGi{U=6}bGN=1#Ky%{gNKwJDT2r4h*@b8 z@l+DI>cq-=RaG|O!aDQO>&0jmVLm%W*A*Y$M;{@2-Go9Zf=j=}21SDsawM*~7s0Dy zk~e(j)EyZKiTB7<6!qryOL$fS)>FY;qYzH)3my5X&XoIQg$bCI@5-S4f>ziPOd7K& zb|>H4!aYbU2i_FwTQGd2akVfTB4#Q+iN2{x-~3FXD;3Nv1XaQnjLP7K-IsEJyhU4+ zR6f5MDIA+Pf1;*t3-_5M4hhm^HzeVmb45VS^z`%^t0aT*&v)MWg|Ivc)qz735k*>v z9$3Yk`y^eC8eFx$5<Fx4uy3Z1D>x!E%3s9h<<hi}gPA2Xw=X4>_E_G7U!g_XS2!=V zt=PGkioc&BxM*cs!~F8<7cmq2WnY7)cpH`+Ow-&dY@^yd@?P~qd#m@(lr9_3?2rq7 z8cH1hJMYLlCei(8hZs%4^-3+Knr(Cg33}*>Q<U4B$ro_Ps@5teGVr_@oIbFXV}=OC zzIhUfmMtq*S5)|d=#)pqz}UvoxarHQ0_e7`8%^Z)gvtPrcJ1ob@A6k{x>KsjF`79h zA2njsbMg`Zn}IC|r+|Tb0itoUzW)VrkEKJ`6|a;MQ7A*#Z>Zdt_VkA}1sO5kP5~Ey zL*`&WF>jm})Ut|$tMv<1IV(3Yf<F<ZXzLc#$n7n4r~^w*(Il_*Cn5S3xwkT{NwEo$ z*Wg+A-Re(xnyOL$luuR;!pZAfRtpXdE)SeQB_}Hk)N?0pkQ~Y*7kk!)0H3EgyAWxe zzx3G?bVtf`rj`Yj5k<Nb%YUm#7qo+7AxP|v&d#AZcgjtWC>C2~+)VphHCnw35c44@ z1?X?F5*u01baxmb`R$?y`s!*4%_mnD-~|aMrBei78%`JgL(`I<&Pwo`#8moIi_&=- z4x7_`QGr!_3f*Hj9305}uXqrrQ>uP_ioXUap?4x&Htq6F`2;;uxmO!Mi%9}qM)dJB zU8^G~&KV%RI<OleUaku!4mQj=q-YjG#hoQfqh61wr?K53Wgr253?Rkz4=H%g#vnZO z1<<n^SP3Hhd8<A|z5@9XK>%J(COkEmP(Ih!Sx2{jh7c*UoeWE`kd+z$2G_ew=gE`K zP*O5>dMVy%GrekNgmFN%v_d0g^6j^G-{YuqE`G=Nfq{CxZ@fpza>7GGZXhu|N7AYI zFk%%7DSWnb+KpbOOP5Y32Aq~^)1h=)x*r0G4_pOsZGL)aXUaawA|tKuc0-UEd~({G zBP@phUON8^yub@kUk{7-(yqiH0GY2IV-lovD3LUfZ7CVg0`+A|8P`4UuA-uH-4#Vz zZP=19oSWFLOYCVVx&r0&T|oIn>0*#*_8*k92IT`~UeZ5?k*bI%8;(J5r$C!yU_;7J zi`U4{W(9vsTH@tuZY9)FR6uLXec=+;gS2Xb9^)8?coO$q`?KGlIYO&@3a7Dpr+M{( zy*+w;iW9hi!e>W(RwbBw2EQmImkkLSUey`A<j`HIc5-@HP&w+6{l#wGkv&F<xURzM z=TILQvHSX|z!kvnV0c#f96qbm`E%!PAQI%xMF2;Xf3k2L1)vp_G!VYY?v_rytDT8E zuV1>ft8Q}laQ`ASE!*YxM@#u1dKG#gN2))4--+Nnyau^ymIC?Fc&r~a@~tc32pR@v zBVc`TG%RFwtWj7oN%@2zp%jcME`XrE9<_9ZIIU4_`zH5$CwnfXPe}Ej7PG&~A(Kw% zd^k4p^3}LKLJ*Hf!B0Q1^pnLHc@^ABkhuo!tHreS{A_FrFwJD;wwx`XdqZ&3GGx9Z zg)agHCA2DE*Za5-iIv2)w=V;p58PZq7c+qy>f@8<m&`7ajEg&@-oLy||0-zuS6!<t zDblcc=6^q{aijo3lA2Pq1TqqzHpkwADM9DBEx#rTcOZrC2ZX!LtK3*PSgvhf|FF?~ zxkAyIOGe56a`4PuPvbp~{)@7T@Y!_(esk`;_ZtvTil*wR*%y8n?p7FXsSe!EDc6iA zRfjM3OT(A;{z7p(6>~TwCItsQB;iDAimBOuY6`ubnqp{HPS1I-pZsq%1zbW2=5c4Y z#Up~Eh&DjnxUW!AwTfLLEX;(&R~s!|x~D$*qU<O5J}2$kcSLxuE{OG_9$S^P{jBdv z1NRfZzFyt&M_9|ftMhZ{=_A{g62AQ}g%AJllRG{aysuQi=YG=(1j%wGKm^Z1HMG47 zAh~on^hdrsGYiZY4tI=2V2^XA0PM>>4%tgPO2~cte}I;W?$C?c?WY<*3tFbT!>jRc zpbl_2_9ie<EPEk--p+y<(b`J(?eXZ>au8=-QE|<VL>@2lG^m>Wb3BC*`7It3d~eq+ z(p0?OutEjJn(EXx|D3Cx)tg0T{mR}$!=sXBvc9ZeDJEu{b^rga@o^d$s92oNwUhD+ z`j5HQ_(z3K>jqD6bX_$zLGyhVB>cm`yN|C-KiONLNCR;0TiFXcEuX8vk!_W;<zH3K zBax$zlj=VVl^RZ0USk-$+DLJE+R8-}ynWo|u_qpG@+yq{=^p?35GmnAn0Y9R-r3Mm zw4m%vA+&jQbGtp2&xF_S^_kV_$!)|s_zzRDSpe%9Im>9V$1zZn52sJzq~z?*k6-%& zOBGqbaCAFVGRhB{VWkD)xwPv4weV>t?4NM%R`uh&35s8z%abhBd5-POZT~v~c5nm2 zlF@xV53WDoF7y#0eX}IT@54Li-YGcuY*!1uocpuesr(Qn<jN>Wk~bY~LiU;n(+iQo zGa6d<JAS$nYWV2bA>)U#J8v)|y!A76_0izpy(`2>Y2dJ&Lsay=CoYwq)SMIE<59s; zS_P<ZA^qJ8JISp7k9&p}{<7!FTjyqpU7byY%paW%c5#2HrAtwM&0hS$EFz?iVBp(z zZe_p|a$0F+X5M~8cVSG9=BqV+`*Bbdu2g`W@bq44LfBx8@F4OZKfJO7cyB^VXixDh zQ^xi$YX$;nw1df}W8gnFiIvguP75ByMYOPZjgv;4{MWIpoQyphZscA-#?)S=SlN-( z_s78}$1VN5#jyR!<)CbFzU%00XTE#7)-!CM<n=^i=8IPo`@OW$0)4uBZ3Wj+_*EQw zKTz&g2zE$?;PU3|MdVi#yoT(HgO*gqkD~~hmseQd9bTNz|33Gxciz9Bf+b1_L{{}t z`@+r+kdJ~5MfiQ>!S){&?!avqkf%V(-tb3$TKGpvz1oJK_y}DG8nfbF-H|==Z2z0j zzC>$QiL5Nmfos7L4wto#>5T^4iM0?0_)Z9A<L#*oaus|A5B-h*!#@L`ZKYlNBir=H z&mw!W|1@ZPEeUO%9Vb{0@CQwYx!h!V>z%?R5&scqo9!a?4z_0Bqw9MWX~@%Z@C>-O zvjl=sSMl6ku8pT|>qVo|;YY};EEGBQ?1BK4j}g^;N96WZ*Zk(czUq)f^QPzRKHKNS zzWtPBb3gjEHq&@n>Z3X%NHi&UtZSAi$jNCT&q6hX7kofWrKP100Lr^XF&poZo{=#O z(HCc&j2oid0E-IX9DxDfO3s*ZYCDK%xXRC8R8xlFRO4#WVASom!^3qquBwQLJVzMO z1ck(rR?(%5et^DyM4Uf;`YNzZ^qE_ry7UVRQ_~Myd_6ci09v!F;JWq$)bbpsS=3@Q zDSGCOOC;@S*Ee~vO<{E0Fw>#)yHDw_MOf5J_(d;YHTqMVni8$4&drJ_dM8bWJ&Ns_ zM-4pS`)2WQ95Mzq5InVc6+XpqxJBKK02`Q=aSOa&@tTF`FJHchD=RC<kA_d+*+W30 zqa4EdaqA_D-nml+b5m~stdLdhismU@g5RYAGJ@zphV=ouuAkE-No;ojNA8z#14nN) z!d3u~xN^s-A_juHMl!x8k(7^Bk~7Z&J>TP#=m}s%2(O)aw8pL_J+6U$Gyp(HNl+Tr z=(&&AaVKFz{|F5!75_N^C-;*!<Cg1gG5z}v{FkTmuXTuP{f>>Oy-IEz-SEmJdF=sn z4`Ixx?&kd#Ue5Gv+gsr?_n4jfiky&s^2m`R?-9jhygVsnGOp@E=hFn^-{ju4w=iJA ziKh$5N9?yWMFA54LLsbI45AAzA-z18=z%YQcF;aJ*8!8~piLhF!Pl*}5SZhec*)L6 zO+~7Jdmp!0iR6Bm9`p_nT;YX<1(5@TAUE-YX)Kih+g$?$0#So4Hb|;4xd8f#Xcg^j zzvt1STf{6aGkMqgJz5j&S_lj_VZ3>msZ&G!X(eNX&H(f_MB`wy#*wtzwu64q!3z_9 z(8eIL%%Di7QvBJ$Ls@*Q4H3kZd)A_BAK~&Y&X18gynP9~HNkPY^>FCI-JBubbN6hh zgOdj<>gw1@o3onc<1?Y3sjdp3lg=6f<t5qpU5_|q#PE@HfPfW2Rzr#OgHGX!vM%t4 zArSNU0~T5dJR`z_-2*Iw$9(8z-aWenj!C-+{g@Ej-{ebfg1R8rxdk*BNfW37&OxdP z-$C4+=MY&`2`zn0QeSrzbsRuufQ4N~dUR+nI&cBmbAYY^fv+Y~HVQ<w&{Hp>(xnHl zI7<i)Mx2|=GwR!Tb56q2(MjU&^wfdR3p2;z2x<%J-D7jjiNW=-X{!`+YUk#Qf%}2% z2IB{lMmyT@858Cn^78niz%$<>faXq8%c+xBa)z-o-xXxl6PXbsxxBP9WM&N|Wz?m6 z4a1kuM4dcymnHuBrDqZc@2LoujbfyU!06#irfDmLsIb4*Uy}^1fT)TXi_y9fybrf| z-l!lvE+Uot)R!<1iFT(g<Qr=-snrpD4kE=TO6M0pjlFj<x)iJ#ezeO;;^J;);`L#> z?rHP9l4s|qF!vetR3y0gp1<9mW;FujdOYr*ymRG!rQHwSQYMBECOu*@&zpN-n1r5Q z6#@)0L*y*KA?L~@=)fO?<Rjg*k3RziW9uO~Ij90?WPK_Y(_UUVigh3yKfqXfN!Up- zc8ep|Gr7%&#4WlZBvcB2X=B6j9EVAGrfFX<f1lj|kQjlWN|6A*VF?1id%>AZLPX>b zG3~NQL`d=}3JM0~914eXcx4_Pg$-g*yAFc(TM#^c8*3?=$Q>6GQwd*j4WNvs%%^fN zG0&nV3luM!Lu;TWr~+LYHOv;v(tpVhvYI;1!J~55Hi3A^@?m=O%F0aTElS5u6hy2` zEJ6pyK_=)A#}e5&zap_r1AExrQtbs5c#Q00SXqHLkm&{!l#V`o_Iy&>KFPUg%;8?& zXiOI1A>1zzmFaC^VQErZr)51XNM{VuZS|7m`@R211NLR`l5>s}bI+{FjZ=FJY%7_B zJGt;)4gSue%<;Ji9*YfPgGI@bBaX{ICDjM_Q2af0lq#DFP|<9s7rB9#SqJH9O}gTp zR@oI815|^-CsV(u8n0Y`{^q0Sh^JVuxW&yRMI|Ga9;T-mH92P%&&`?Hf$$bpre2h+ zoOT;j?H9q5e*%?m!Ul7A+F<HUhJFPf4NXPC#LdneMOE*2@3QW_hY0%zaNJ#oVPDK` zIX0*1^=UMO=g84*6Vv3@ZD#0f*p#6~6A)6;igK}3=jJ!&E(Y;mSDVezrl21pSenE% zN=acTp&)OF!46oV;a$UXMn&q%Ob;C<N)mKB^qK{^70ycMHz7*25>k`skksC#nI5&J zMX3X$m?FmHagR%2&1WWw8~guAd+$K1+x`!{BBhc_QYxA%Dl!UL4P;k1_R4lbcE&MF zL!l&;kV3{eva+&LWJT7oN7*vMvB&Ryox^=U&;5IPJm25%Pq#bgd_LE8y|4Fp4Z8^& z$#x?1aTpP);4^e*0d;Fx6c=vC{K2v{D=of8I13ULNr8p9loe)M%wPFf@p@=UkUIjy zv%<gpY>ee8Dcye_5~19pBMc(uR15#Ibqf2bZSMfQ<6Vt^>B8P=t3Xo&Qlkux5CCO6 zYyfyv9WA$);S=f!o4i-N)CtI}ryx?Sb}48jmXb!1rdP9F!KP$0!*PHZ?`-P*m~2|Z z76Q`VCSS$?{@>VU4tjN7+Uda6i2gJKvL<q*9wbtM$HDc)W<n@t1PrP>8oU_-ex@3) zaZWKrt1tnL`}C|@34xt_x)y2OWkF>y4tfhSqNmTE9YMIq3-)Q%{seCWJLF`UpeW62 zRopNfyq=o<TK*;n9dM^Y9O+$q9(WhxEss~}lV1B(BXB|&$bB;6MlXw&r4Gy@&Is@d z0uf0BCR0W%G*-WQA(eU#TAPw)M8YWq^10cVEDkt&eUbBMSh+Au3V>-?<?o&2GcbN( z2SkrQAVXi^)Yocr5C*B(ncw;O`ElC%reh#F$<&L^$^q?H;m9`7FJ7Ju@f6t3)vsP+ zpah?D?L95ZqE5$)9iXXcrHj(YqZ;<E5#A$<<g>cp_=#AnZEtKM0!ji!C%#`hE;Rri zc???DS0H7Q^RE>t`SJ;7%7$={8~J&+^4Z2!Y>LK?YEX6fo;r4{_|8|79ZcA&0o#$1 z4LcB~d8f!ws}Sq~3{@-!;2zm9RVlbx^Kb|T;oQo>BN;!pDl)o4=L%sH)b71h0mCzI zT^Re+NpBj|@Fm2_Ys`f{#=AO-M{uuy8W7M8Q@J2em;5Ssu2sq4_^ilBfFM-Z<J`o> z#Tf;P9KA%|k3kyO@RFplmA8X{Gp(__dK<(`<@js|BS22*0vNKI;9qPSnmx4viT;cz z<0A<4C6G-NiWF_C(}2SEp6|81?H5@ue|L8l2k<E4Q7yj203nmd1~-KaXju2_jEX6l z^~MQ2v3}&FP=C$F_J1alKZODy=xw10C6Aj|I#A`=KT|x`<ullGMY|_SG3yPpeBS7* z?4$4O8@<%Rwvf%xB1M@?A?(M%g!^llpSEPY7dH4j|8(W=z@495JMS`Gj^CkU-)Lx! zyeM&*>Qz@`8HHdRww>;US&C;r8cdZvEW%8;{$>W!MU31_qqtZ!eXLHmY(G?3#a^!d zcn0pY=U`EBC3kj!<GYv>fSqMu@Wy`qEog)v)&{W5Mdf-KZmZ&quXh_!tq7e`+(@7E z*G$tWn+1vW=Cc)>pFDYDl=;?eVlOQmM1A(?!NOxO?#ZI{!QPq&N)B&b%`1*n#P(TS zoXV4dcBQ6Wow79qhZmJ^3-8gFffYo>943#p6AcsY-`{BC2;m6Q;x16J<85E_`ob)j z$|k=(=;Xp$VCxq!t9);z4Hbh#0+RDiL+wvIU-og8l64duqHRYqs+T$1BB-LzX;y43 zm;h{Zi1cUX(b}uA7(wDdD@^21D`~Y@*&n_89{=zMluge-iKg;*>Y+0DUOVW*u{Gn6 zx)?r}*&Lk+bS5Qp%bMuNLLj!<6AH*!^@VvrWg`SvtVTLPp#C@OLQZRv{yo}OvuS&v zZlK|BX2OAy+hOC@Qf$M!&rekft_wWa0cLzs<F`=;4vs8rwz($X4}F*Ji{hgz9yG21 zKzSGF&^<w@+|V2AYkvOtapdsfLL~zS5`KLYY2?8I0?78|;rKVy1dxehQKDVf0>z;b z)tz13uu9VR?lvTSjF#URDgWO)TQtMsn0}D3PmOr)%TQcDv-HSx{jFG*?dpWN-jav? zPMziDvfsD$K?>guC{&}9KzBq%15%>$QMz#Y#2xeX*}$ke0os6<AB@8&H2~%RX0`7Q zBB6V~Q!u)8U2sxt&4K)h8wy36Fd)FfEg*N)Hv8v}UCWuqc8>e;<GG;bFU|jEl<-t` z;_DbQ)9@=kYIaG8&d1o;q+w>}v*kzM_cxFaT}{yvy>t;|P4+2<(Fq6$L^4*Sfsc)3 zWMpKiZI-L6$x-`a%WLn8h8r#bg%8l1IbV%m{5;d9$`(9Kr@Yk9qmPR|La&WgIIed& z;m8)hB;tZqXMWPYk}@D;ds>>BUd?t{qU-WwDX0A}?K<){lt%Ic$@I>Y_c<!#AqE(- z$F}DngFdd6*x#?4H5n_%L66PXpc-IiCQAPV3xg++A3mh1z+<j&oP$VruG50<L*>hJ ziJu=9BUkLV(b3X1(Q<NCEdzO}y0eg7bVJVR-`8gn0%Cpw70O&%#fea)KyZ~dt4}a} zuC?%{?nfo$dkj?yGBN?nC3pNKEenp4&%f^^GoKGD<X#teR9D!9oWX)i>BBAh0I1=1 znlPcIqYDJg(yqplZ!~mS*bdx}&kw9&+W#ze^I*#7-x@pjdG!PM<SA6YGBD2iTvYTT zSK<6nDmQ89ffw9Z7r*DtT|qb2|2PlOh4?t`L+O`^t=U9v#-(Lx=UCVuvqPFV=m@zO zPD75WmU;FY4548d{J{GQdQayOpPX(l8|Vc3!x2>g4NQ|)m@d-mdo0`wIslon28)=+ zIXI^vx$YK(N_;_w;gf)B^Ao6q7v^9B7@+~klAi|$S94UUuT{bCLN#!+KD+trBLP3Y zohr%5uxapxIR}CYj6vEQ;+{eC<yA9ExB&(i$R_vVD{$*xIWVaa-9IeiO&Z?6wuLPE z>*BoUrc(6~EzzG7&6^SZ%H~lnCn-goBa)b?llRpzF>f&oA^v~tj!5_Hhu3(Hp14oY z_6i7V8H>;pawgR?;kP~$Kk;;s2_GF4U0_@0;M0m16NBE?ep}=bVB!(@O9bamCch`e zLhHY!Z~5VnSTCjOuIym;D-p?FZiaL*;bzCR1cVu7(?9)mbaaf4ybW!dO|7Rmo;3kb zBt+QxLr&!;Jv7Jj-m>S0W%@4Ot8sxfm4bsO9ijX?!q4B(WSFP!0Rz&6f_jJw&2tI5 zSu~P-Vgn5T)i5oZg?y<591RY^UG=JwY#CmUHH{zvqtyCW>B;ygn$IwTHLOpI979#^ z!Gj0c{P*s{@N>nyMwZsqN-A!E>ixC&kjIL+oS$_-hIN)c3RH6{KC;4bP%+`fOwS?v z=Db%CRcqxr$eN4yi0`fyQdhp3n*){na;h^~Zop8zq0C2|YJ#0}nIoK@nfGg~$`b%Z zDt!PB2vYZl`FHWJ^rD{Wp$)U<+jxx1x#mbO5AddHLt(HaqWpPZW>M^l6p{TgG_Eyt z(}Xs<pF^O)D&=N}Fq+P#L>VtPbdF2pGkLm2-K`PcE=}xr1}+Ou&7%@!hDwa~Tb=-n zMviy1m7sNg5122rSTuqD-*J$}x{;*(3<Lw26i)$$qgd$#v9#ZJZSArkqRD|=XhLzr zP(2<yZnnP0%hS^UqHoV^{o3XBlO$AXhTlR35I#92J_A-t@0e`x0cI28ph<9>nG=rg z`{0N##*)bXttna|{4}#}o%HX+5|7?{k<=Zv;w45JQzR{NXN>wmZ?`V+nZXa=q}DT= zH!3B&(`16!;}IzCU(a~2<2Xs6w*qHXhMt10xJrHz&Bzf(+<~{ixP_*pmNU()NbOWQ zO<T6zAR8$3@zQ(rtwnt$$1CFPe&5zkpqVwYXlnDBmD>tkx9Ha1hm0HsfIKn^$C&#) ze8@eg2}MD1`{Lv|N5DlM6%uOsNYql+O+P<X<;gjZ8>>?4E0Xz_0yKz#nQ0ZIsd-~| zSNaMmqLg*#;gufZ9~*4pXh4#Z?(MPMqR(5GyJMc?bm(yG)vUG)#kyG-9^$~KFMM>a zX763V%xDZc7WXkC^|#XNcr)+?HPGn_(T$;z0v6{9WDHLG(5zkkf)D_R1;u%9iy+xS z7LXnEB=?c}n+5xj#N+EJGTQ?Y-Ray1NVYFOpxOe`ilL5lI|8GuW$n8sCM(K={$%CB zXypLDk7j9*fMmI$ECYZWFK=aGtSlRoqYR9UUTi934$A|>W7##Wn_33=kg&aMEkwe~ zK*+*;FgD*RE6;@qO3)Q+;EymkJR+yv$?H+C57r^Wu^n;Ph5REATH{3f3ilIoAB$1_ zfJzIh5L{njl6iY>t4nlWI&I%MhEOOeqPb^LXNqbF(!(B}5~se;f)0=$jzLBkTn7r^ z4_pfip_&k<8fyZbV$KH-+%O@hk)mHXthKtHRy#7Pd#@GJYTI?!&82%JfGJuch-JrD z)$kJ~Uc{yx%JklCu{F(Mx@1Mq!srrLtAYi4%_?h}?}M5P_hpF7fwDtAKgiSV8&s~| z)pnD>%p)?pS=5<zL*jSxY%oF6?5h&TR&*_}?KXRbKS)9hoE!T3c)3ZenOU66Xu*cb zEGUR3h8rfKr4Gg52Xt9j&re%92YG8S?AY-ZidOM&-@Q;s>>;%PWBvin*EG{^64aqm zuw^Q;ND5LQecyv1kRZ&rcLS+rTbAz|=!lVXb-?b-&(Dm@bx*Xr&Q)mk08IK^F%oh= z?UiseBt!pZ#bXtOH{l4yei>Vu?-h;our{C7uDy2&!fV!{La1Vi<<3js?Y$Fs|8r)) z7x%1d?Nn4QYnthm90&<)y4vxWQ@x7imk!Q`ya_+StV@gz7ntpMDB@lNDE*7Qh~Mco zaUm2klozhxv~+kjY|>~v3P7i!$8g~<#9(-yP31-(J$QM_VBxpF`u|)O{eWv9$fx$p zWs#~bz?beCF|o4hn6x%?<nEeJ`P45s;%iA4G_c(bL!A8ZC3eCMwDTC&5v4oT1#;d# z4-7P17Y0m?9lLg6e!XTEs&h?ANr_6;*Rr!7>yrBJ?+l$fIX**`;%B>L)4zS+RH=Sf ztG8G6cT3aAd>pa<ss(gfVyA~cwmasWWa~a-KOAmH&8{excSaW)`qyLR_dmFQ|G3rh zSush|@Pp{J1$kGKg!|`j$yhN}K}W#yOx0B<Q{r)g-@}$+EQ~U&hc)AW{l#a#Ls8Mn z{B;aev}mJM3SJiZua=C%aHv0!y3%kt8JC40#Yy9S-64-Jq{ULJD;1|`0J`QFkMXx> zaD|n^Tw<{CTR)nS@*@$f7<VSInCNBIh({DuKn4^$tRw6;tF_(Eqm!}Xqq=b#rsTHa zxx%iZ8OyDjZm?tsfB@An(pyHoT`Xb%O%Da9;<zOTR`?H>F+KhtorWmv2aVknKvCRp zJh%4x*EWBQvMbji1_p;oyuT<XINuFgvvVUkr0$TM=^s90K*U0%+zKw>D;|ODbFBLy z5BTYB<btPOYhTzJYOy>*7CHY{25;^GgT~ftJ(St=@?`A)ogw`JMm6uGs~9ay)`jr& z@`?h9)#^MqH(K_0xzcD7>>>_>;KUZ%c2-G*AMs!OS5?A0jpIhEA|tllLNJAsp9Cr0 z_O-~obB#r^pZr9!zvGG)z;6iru)o=$T|WMOJNHR=WCM6)50|O3b5qN(F`MgU<AWtR zKH1L_z1VGe3ntiYQ+6+Fz5Tx*clkX8Qs(<>f|k{Ex#2jfcncEA?5$BGhnDMl&&Akf zrQ(jDRNSZdm#<K}+@a6y!)L}<(UQ+sgl#2!R7#ZcMU;@_mU5rb86~vp9-9EdpsiP{ zym;|%ws2s0P&{o{*w-MhMgb$H0$Y`fH&=xj$iP}0T`~1p3v+d0cZ;)%WOc^lDKIeP zR<zRfK%uJn;NQDL58609QTP3IX*s7~uadh#b%LD1v#;wdgDPCXaJ#kAlB<S-3P4{# z<K|lD9R(BaIV9PAs}+e4L0nLK%a)hRMWQjhmA?=h?g(Jg8o0T9{OJb#lShy5m3L`` z=_m|qwS2KW_%3)jJ^lHERlaV=HTZ)p-EVm)XDiMd-a(vI`0k_Rxj$LeMRg?3HEl(z zO@cghZr`<aU)ht7^DeYoH$A0EyR=~LRK8)gClP|T-Fj)+Eg4>uA=80r6?+a}@;-XU zkHoq{(?m5({)Ro@i)9R<`nMtOA43BZya}y{_bU|(C?<|pt|%r#C)zE%#b$F?3K{{{ zC@h0`?cCJ)|1OMXYHEWwQ_j~PPr}CAS7G%woav|7!UAT64HPqP%I_ls6s{(%o5#he zTDoO010W^Cr+n3OErc+RiYsQs@J4pQ8!=+vN_it<=#3os5U$C-OxFmLqfGpF0cd;~ zy0FMA+_Pdy^vAEw0tk`egIBUB2QY%20<60A^H1`TyBd0zd4p#?^Jr>p6uQa^e(mE? zKrpV=gH<T({_`I1ACjYo*a3+B^mnq$_mTH+DF|8Se7v!5<ga^R_GmeR3286Ggy!5P zhp7|dtcRAS@w@MD*?Rv{&l$R_JbSlZH!IZ0F?^!fom5c%NRa*6+Yr+?A!jxAus!SX zKN~uLPsqCx<M^KU9t~c$!$i%jXAAp<r*6YBDH!LJwQJA4%$Zyy41UTfG8V4vRTCrD zS+@Rpa6#!WtXbC$_ev0x9vMi##^V4;B(hKkly4(Hh#88BMFyHz+5Fpto9(a2%lLQI z-gZ9k4~_T@)i=3!{c)pGeldWsFRdBX%dyuL^%)am?tLSl*4h9#gif9g)8<-U{Wyuw zQhDEB?H7rBn61E-bAxGeHR@Ep4xELhWQzRU8jlDFAS}F!6=IYxjyTgas-b3^PlhFF zw}k~-S{f$|VG}o0mrgE-U%#%-{`%m-9ecO4T)Cuw@{c=e`4=5WA$VwezTHE4s8Dpk zrBcLo$e-kbHn&Q(hh2DRf>23GdD;eXuGVPX^{a&mPh`UFFUMa<k<1l?RT$q#S*;Gh zcDL5t5x&RtRVP!C`}*_dW6_mEC#<B-Oxi~z?A#S}<w9f7>7mHOREDaLs8;)+kAOF9 z^$F&spo`oU(Y`f|?jvJX-8oTGasB-J;XqhlmaEq4o=0Menwve6FLDWAHF|2b%J#YH z0BbP#?fG-^tJpTG0WZq+JfAIsR*OZjz#k_b&MmYYVwN4@IV3;(c=*@oc1QbmzTmcG zsf;^(n|=#D5MJ%=Z>xpR@N$~nL#D%vy$5tGeSL^=3;EwE1K2H5`kuZr?aDfbKTZ0O zn8R|M<I4sa!(xv^Jo_Ly;ErGQJy8VDkKvaq+l}xfr1Ns1x&0Jc;H5BbKXUdg1`$MH z&^c-v1J%ulRUV|A(zkDC6fe#;{6+={Xfjz+Qeq8eb+wXApSj~=%V3aoxmUF}6#xvR z>Z!w-{RwI*U68y5p-BV$d;24DR;SkS?Wh`g3DWgf^z>fCG+g`8p+jc2Ip^e2H}*@% z&BMny6s);zh6*;^;L_TVo~awYN8eWU-Q_@B(kd&2?rC&q@TpSLngds0{fU*Au^raX z^LQE*bnNKSqxQ!>eflK3#<?TkLxk3Q>31<Jx53wKbBhEHD7v8?xVs-5zF{J-bLjA4 zE&92eIk{f_1PNx@%CBF!IJfQFstdhXG#Y$SgL)~@lem}tdBm4Nb}IV>?ey0Esb*l! zbOX)JX0EXTwQqo!zYmNVMR*`!^9;CWUt27F_4G+{3yaEZOW2N}o%)s=N;iQXb%<1v z#~-s)O5klpN%vkA^=08{4|eML6Vp>AFLK6)nX(PKvo6;Bd`r02V>34DtNbXU=bi{m zW52(*v$9IMEGa1yBmYWv%{&lk0+I8Bw2Vv?j4rLAx^<3{9!1P}$TDF#YH)FcAZ$;k zg&nPldrFuCO&c_G{W1bFgpN@0%86<vPJPMZ1T&sR<S<tqr_8Yj6potTYd)UHM|wCx zTJ|B$0*vN;J{yo~m_Qd^*9n7pezQGSi-w-%Bya@qeIK-2dn-VOxJY}5_pyMwnA<R3 zEv4w=MjNP~W#9!Hf&fdhULno*kvGVU?^tE-q1l>|W0DGH{`Mj{&L4!wx>Bj7<R7P} zQyJdXb&4oI@^h8iKf(g-g~uHD<xnfexNdRVe2v&FRprw0X8R@O%$Bq={L7%b2NMSl z7>6~~+Bx9sQKt<}MQOu<u-1s=!DRp$@$17x22DO`{F$Qs6EL!FC^pN!vHtBGRL7uw zW>JyhO#o87HVk%eonJ)VPY|d$t=_41DH4>kY|jt-(CJXu1aaz|2JfLD@O+X0L_yEl zlPZUWt3Eo~gaEw8gfvmQ3)e43H4FpJo}HE;oYvR~h=fP`&trgyo?2=#lD5#cC0fPs zSK8JZ|2OM_lC6P;0gigc)o-ODIm`O2`>GC(bNgYgiFV#$=mLMYi4y*W&Zz(eno_xe z%lQzLj4y{y>}}1IA65*1Y|*ZB<ftKM&0X(1S8iUujkCY&yb24$-JKO7_Y0+7%TQt( zb(R=z!xCFX`*9!9X0=m2t~vT$SK?ztp$Q0_uVnU1ZF|pI@G5(xDLUn16hxnPUje#h zJ2Y>};3hpiJ#M!}7t^YUf+w~yMZ{S}E@3t{3E)kcYM-v!G!b|TxxxsMY87|v-HT-p zWsqe87*J!)Cs&yckmLveCe_z`52|kHbfnt9tc1Z{36xG1GQvm=W52q?`I54|q=mVX zvclH_tbbjT?R@&?I?lpZ(FI>B%f0pgcEN8{I(Rj_KotK>q^IN~yHl^D$?W6bX@pRM zM-McLzAG)|z|8I~RE}Zirbk$W7h$wh24-ztpfEMXZwaG<F_2sG!SGrFqYf}V7hsWu z1cozy{<2AHO7j+Qo9^5&1M(4a`G6e*uRz4*?-3%JbucQ+5(6e!09SIZ7kdRc=p+~$ z8zVDux$a@uZ>_&<79=nIf{Eo*Xfrt~-5^T;5cEbsMj$UAfmZ3G3skXyfUq_%Qj&H> z?Nb2(H9}?&bGkmv&L&CF6+Abjdd$SoYDiyOJB&7EpurjNKLnf>xO%xjyoe5;nVG2u zO1S_mt{emOIr(r_d|6Pa8$yd+Iaf7;2`g@FZV7frV<4F&Z3y7hWa$)ExA~-#;3ZuC zgr?CCT%G#c^YUWL3Fbp>IT|oHh>0&6M6kJx41QCP1OOW(sZ<4m&$sjsF`WE;ALEUo zUxqXcEkp=JkmtMrIKch-QMj}w95lK9^^)mvdT^%Bv81AcZ>qXC4tmZ;VOh5QH^Mdc z>koE$danC^b_qy>!M&$}W-P=cD3PW|d($H|m3KVnopKE%(84tFqHjQ4?ELt68ELNJ z=Vi{Jca`l5+RW=JYTt;-W5T)&dYXhce%5u;vwE=e8Lq$j^latQ+4mdDH?3A{P$Lh# z2si3A<Y2W?VlCc#C<`PdvMlOnoNzQtr(e96_%O7{b5*(3<AVhYy9~=LPljeIHeoz> zre`n0SJH0KxS&6%Dkp;GTFn;>>1;t3>?%^$2Q=rI&d$xJGxB+34l2;`u?pILwa|`# zg`IB7%ER_$B|{G~hKGko4#uq;SW}xzVW0*kOyzeH0V;EIBpm|-sAJ4q^JaQpcGoH@ zq}j{Bz0sG$>yjV?NZ()WdF=_<<LYy*DaHyZ_$4LG87->H&AU!=U6_LiE$0hH&U4J> z&SPCA*RNl{;fu|#7e<gX8$Z;k0>AbXNJ{YDle4z4fn<OZ_oeO%N3lXE|M$?upQuxf zQ^1Oc_d_@hb__!dz&tRd{pN%D44XF#Fv%ez0}C(CnzMsqC?nl%%jOSqAi$&l?SyC$ zVuKF_(M(X&1tErIPe@hog>ZVDzy&7!Bw;*e%1X<2YYI?wdEU-}C5i!o$h2*OdB4Jp zZdR_r8QiQbg8>oCE2gcgYI5Ez4T~!g3mVux=s4@??$pN<@579YZxXP5NE+?jTIUom zw{`Mj;pVtKmv!ocAyhj~meN;4@09b`mfc(t^LmMv#NLr(iS*tyxYPc5)-eWep?|Ao z85a8dKuK&Er{?rWeeI72K~m>N_0Cmx%UZ}~h1VPlJV?pXZt;8Q(nTx0$h)@7?Zfyj zAx4%&^Ez8RHU8k>@+E@NYrfYUJ5K6*<M-mgdyFuV>jRA-pO6Ru<WdO5%^%FM=lFp0 z%%D09j!%GrZkV(C!e^@m&U79<Fj~3@&Tl3ci_x4;gJjQfBu=>?ey=Ps*K7%fdxjT_ z02LG+nT|0eZW?7ysPCA+L@#W>{t?T{u_@XO5`##_PCiXSfdi3=Jykjb6G~jfGLmd6 zg^712wY(}3IU4$v8ZXcJco*`SXnk+VU^dmA%<K&dFf4lxI7-b!0QSnPyR%^GsB^pX z?pWT^Tpk<0_BAf8W+e6+3;fCnh^430=@w>s!_RHaT4>Ps3(3`ZtfA*dheveHdW-Sc zz%s~fEWQWPOFY`G=NsP*0fhsa=}2qgq4yX`yFQX}$PzrS7^DH1k{(-?7}$_DFpI$H z2q`I$=91<_H-t@d#x15kolP=SZc96bGyMUqa<1{ngNq;(p_b0QcE0oxX}u?q;~!$G zRnFm)6JNX0(c}TnFtReoyNj=j@Kc{wXf2NRLik#(iMtF7VJGtXbo6dR&WMn$&2~AE z#bG=1Ek#ix^Hq#KI{oI3(|V6D{`#G~YTXYRVDBDEs&il?Dmh%L=SaR!N(#@{Pk#cm z+4np43Sg=IeN#K1Rx0-_+#i;o%4QuQSZ4CQu6v&cXdi*WgcMovTZ0jd&qRPEed)8h zo(2<)Z(qyO8DcZksNO(>HTeZ^cp5Dk<U(DO#yt6G50B8N7`Sqn=wh)|T2v!h&*-DE zcP;6TW4~&}$0JSM9-%>24qp6)!HEf5Z2F7~C~c5&3ZR?yeNp3J?*L6rTI3Wkiw#Pj zov21GT~gYGzw09ltm&Gu`RZPc>LuD7xFGsyjf2)X@`XsOCbjNPEQZjg-kIa#o*M`8 z>l11HU4T_Pesqt})21Vtg(ElmIc-eUIgp`IN=g@kDBy$rG6k9ZLoAopoE=CM1{+_Z z8e{HB--Xk*ooD7gz4QoUc=K*=nsj!qQiO6d-f$(KuNcGB3%CgCi0On@^-3`<u-Gl+ zCQHO{mKYctzYLO|aDGEz^6a-ewQqVTo2KUH4vj0bE>rDq<e83(sIs5mx-&#?o2<j^ z&1$z*(uj|oP@~R8r&6dR?(LhYv0Mw0v{-}Y;iu{q3WGhble=CsU|ojbyl}vcT>VCp zKC}C|Mhk)HSjAq~9Rz0Z^70}x1}V;fGiK{b==`0KaFSuSgYXi>+3ongR)Ae9E;I{b zT6MGRTxS$=+CgIyyKaO~Y{md%{uu;H)U^7-NT>+0!&fZ=5c#1mDu;~1;hkMtV4kk; z_&C|`x%u=h?4`niEFG#J%wfae{@VN-$-X}9bka65iZ-#kr4_eHE*m3DbOu;io04gJ zvZ0U3RiP6nN^<XEtla><hLGJzqfHGYVrd=F(ypChkmtWJ{Bc}<&Iv2tT=gNfDH|i2 z{yfgJzg~b5XVLtjLP>(CKfBNS?%j|Y-i$=Xp0wFTa5EUTip79t)527Dq|a&Y!0*EH z?{x3w^1iq2zmD<<1TCA~qr9j2m~wI?=K&M(Nw`VAo`?y(UZVEPK3U8T{=Uv0en3e7 z8W^w>0OEyR?B0Fmy$0P<@<s&VhDT|*A(EI{qvrd{Ek^}UDn*v&s;&YngF{GZJTI%M zUqFPeo&c?8-7LnqgyWBHu6Hj3zunE8&wxVyvULeSaE1UYj?Vgkq2kOl3ADhoCDcWy zCBIi*$jU0P#rxtt%&?8=mf<G5wF|qLtAvJ9YkoZ}a*{Xq?1v;RJV7fPW0BEG<0su) zk1KOiF91Yvwv4N5Zmfj18kcfB9$RNe=LwRh5iW{G8u9lA8#KKJmG4GFzX<^XZ9!t_ z`AM}yaAo_C)s65O5OAqQQ-GTM3d{~+_GrLN{2YVG`f@N$-El@d{!SvmqDw_N>S&_} z8(^Z##w<bPO2>#06CslfZZ-+D#d{|H%mQCfxvy>j^Rfu6#4oB=8bY#%%1b05_&whZ zDtuE#j)XybFcLKAt9p%5RQ6uN2Ta_hvit5>n~NO7l!}PCgS%6$XWN$+9R_VhsDR?^ zv__8YQWAu>=A8+v)%;@HP?O}7l=0W)LIlf&tDb+HJe~3?&1uc-N^n%?C~wKK!#V;- z0S*#ZB8Ko`7hP~~PY(o&1w>z|`;51Vo2?bI0``eO6gVa$4%ChvJA_`ef?_rVK#7Mw zK8ApXvR^;dL|r5KnE9yl<gX_%XO)4xy2mQ<p*?^kiTW`29s_T8DOuS^gBCNu)Bt2# z`m4a3Eikr4EDxJhSnaYah3#;kven{k_Mn2MEnY!M=~=8vBIiaIQk}|JhZnsQbq!v+ zJJ7bG4WU3BC%#J#4Hd4hVLGLLBr#|51+5h*Kn3TTd~v3GjlHVH1W2h)E4r$|$jU2_ zuBkaJ#6ZF)63<K^_F|amvIfVar)p>Ff{vIm%c^kXgws@$qBgzK9||B0F7T$xbG6#p zufuAE?_HRQ@X2+5Q)xaU9<GH;dh5csIM(ZkefhAz?dg^_`!SW_XOEtBpA{oF&zy1} z8H9J_GlG`$t?EARRq`1T)UrNoVzEpH6{SUSFM!Y4uspyq?Zv|;nwq-VCz32@I$M$W ztkT)U1S)2G@D3#u*mCH$K@b!GjcShU+F!(~becDINrXX48WW7G7bVo?<Ezr_%1Mgx zK3KzF7rbanKm)YmH7pe1TXf?RvH5}!m`J+<2@t#MF2!eHO>{0zG$lfI+C)@ei)61# z?1);%BZ(&6mTq$|c`e&o_9YX}_XO)-j=z_EI&hC6Kq(H)uOuo%znH%gP6k`7FRx39 z3|gBockk|9DFh^!OCszA*{WDwOV;BWGr;T-*U_%h{phO|&M>&A0nngG&7;Q_mW%Q0 zJ-WR{lNDgVd$=oc1N)}v!Ofc>VadMP_E8Px&NIcLiwDh{QqK(aiPmr4N!Xf|$F7g} zC)D*oAau}2IQth?eRf%@7hJ*X$cg$#9(3@y8RzL)giIEhy<F*obO6I+5q217k@nC{ zixaw+>iqmo3L|0W^t{`=`kSYiq@)^sF1<G7Wcwm<JNN1h`Y@!I@NWdOfBv{p5SoP- z&UJ4gvn1E7eF918V2ci~-s79iTG!tcN7#*DRU4~%z?W4ds793o*deaKVm^LN<&2M! zy0dvp3zmvf({BUhpQ65w+2s51wvEvWuLPA*fs=Hx!Kns7U)Mo~ci`Hd<^g}kNcivM zdW0{V71V~*Krs?2qJf^%%~$H(F^sanI0z8;8dATe!4bf;YZuaDGx+L%aFWD)iv^HV zu~!M5Zuk#cucG3rYz+%+>52i0(@IW1>NLIDK`!ED-)@u|TWF|pX~jQApm=N0$1a6# zLVS(5jjy5lHHar~rIG=)Dg|iDly8D%_=<<nyzo>m(56}>ti3(>1Ad&{*Eb#!xWso{ zvn}In!31J1C^?~mWGJo?!#WY5n-wq7K^W9IQ&qYcxFEj1H9RQDIP4xwwTS}M1rvhD zwm#e|%Q_iybJ{w64_5QWW~uO{UY&uB&k7i1>7RudEYofO7!7ZOk5JZ>oauH-i#(5M zV~?(rd0ezY-Wlz0LP@6h-nvb*MV~(hm$r0vGO94)k3-roOJ`)pcoRP`q#o5YP=$0E zp_|ae8x97iUZdUd_f>Xf*_(?|bue!^&CdQ5bnB7OU_~b+5ZBsn|D=21=NAN|jSTHW z>W?)NflBC?0kSqOIfO`Tss|Fj1JF0^kOnRdta%NHV%ev4TxX($fbM3bRXD(`P85$n zg-UT=9R7F&zIPx0EivYE7MgldBuKPF{4>LhPbu#K-MPZim{9A7{WY}5LUq~QG>8}g zbnNpLnKj|r8lRkL_sSK+8-#%f#*%=wZK>A`q#sh(vlb?T1kvE^K4^4EBY$heFC4bk zNd1(z_^4{27Gq~OxSn<>+j~@tDXrRA5BiNPoF`Q~mmWUCM=Uz_zB-1KnXv&zvdo`V z=-H#Qzf-q(lagh`MdK&#?u;K;f(wB?VRXbCco}WG=)O%_CyyOD#zSOrDQe1{bUN@a zlZ)1+xx?Fvig*9>B>nT8MNC5P8f#e`NV&)OF$aRd(cCnk$MV^i@S%kIdD7ti!vXgk z66!MpU%l;Y0(0^&)F1dF;xB06g@&?B#I5K85ZoTYs*t8Pt|5-!wZ*D9BtM;|1>ZfG zn74~=+U))`n3ty5m%;6JEXBC;EMrm{_lLpj%6;mUpt9#<5{!(igFsvOK26}^Lx*G; z!d=E5ltYZ7{s>^?3p35_i$;^!m+Dd)#jky^c9!_#1m!1oHxW~9VrWi9%o2b+Y_}6T z$Cj_pauOr7FY}Sn1`-#8QoaNgj<Ydy?ZAOz$o2L0-9={&?!sygFS}85dIQUCF&I6V z#wBWajKYTba%b32aNzj&xx07oss&w9mOwbG@>l1HBIz!=AK{V{G--!E;R*GlLPw8& zQF2UYc7i@F>O`@%go6O@&@Q?z#HB#yBzr(YJx50+;bH$kG$RQg#9}*KE2(p`O1v;n z&{a!Q$G)m6kyP!$qNu@F|AiE_hUlL?D6>J3Eo~a^@e01~_;3}&c@R&0;H##yyXyVN zI#JShrfmuCw+uP=2Jap7`?-F?*j*%hcEJH;;+osDXNBKYE+!k69f7;YWn}us`+6H9 zfaP^se4QzxUawTWS0Tc$S$7X`h_0%K<d>_yQEE@&U;+<-a9M9JN;ZC&t9%&VZJY^V zH*+X%tEAG2*0c&}mKIQd<g;yluXpLYyA#NN$%*#?+3<mDf26Wk+QkgUuzH-+WA6{7 zeRM3?yB2gtkO-AR6Hd-Vvv3&m4h}QqJODf&h(lWPKnIb1p8}Vv#p^*-&gp(gGa;ET zREv*94Z%^Z6TBH0fX!$cu!IZn2&jSGd4(>hn&%jK2e=%*Fsl;%W5a4dle8bqj8X57 z_gP%o%1x?9bYtHfja9XS))5M_HJo_1#IP(KuLjUEAA;VwHmLJsKUFxf3M0%Ua;wJC zYkhZxI#3xTPYq}7fX7z`P2fSJF6Ln9<@^!`JFyd|Pd|sgx-?WOXR7qGly-r-RJ{*1 zb05Z;t_s`EdxcQ<nr}<|L9JJu*zZ@fnqpJF<%IFTAjn2_B<d#!B>N{3CnvtFGnbi* z=A`OFqCVN5K3Zrf&yyUhGOKb;bX!S+L_qUj+n*YqWzy|B^hR4zUEQo*COi%N8D7{V zxUuupJDYYp<JwYLo$y&;xq@ROIg8;pY}u{rM>Gql%-`{~wDc)7sK0)E_Zs%f#YZC8 z*^`5nS)goK=%O=wqZY(0UhkWpS{aN|$t{nhoBKHkj5Qi(X@lNN9F%9>u|OGj{g->2 z6X|5^I8s5mJ~|N4%&p=%MuBM9oh4e>Af@%D>tkN}sQQ70#Pbh!?%eqmByzjG=tbTa zwmWH3<uYFv%}#hYiRv(?2RQ_fRzP64Dbd$&`jvHALgFFj8Z;Z<WwwdT<Z+DmR-Oy| zj9-9;QY7sf?AL}(XEllw6!83ngXVOh^q`V_=OSLeK87Zf?@C<3ZS41mF=$e7qLkJJ z${ZFzLjtKrF8X0M%@@A%gtgZA3<6_qrO>!+{E2ZjxTEQ_XkKVh4KaVKH|rt$9!OT! zxsN9qUa-lzhc&z<99>o&B||nk_i${jk8lQ+gE-(|IAY#wJSKb`lWsNSRECIj1m8iZ z`&}pHHJfF^=#rD@-TRW)^aeN_oA7IJ<Z84Do+Buast`0!pn?fd`Y&Z{%hmN}vjzz~ zfwQEecl8RrL*iyJFK1)xN&$FSQMNq`f5Cpgk9Q+U%?ftC6`lVe&%3(sq&?N*R6|W4 zM4HuK>*yE-pR)`TFj`OyNn#-yUpzG-PNg*wQf2J)`Tq83r;pxa&}r8z(v$B9cW0~` zX$}qDHy(7<)_->2U`hmM!?UH$40aEo%sg%96+bjF+&Jk@bt+if<-L;qzUgsJ>us{> zF(IpFaK{05^-X^T5h22~ak3+v=G_!96mxkS;@6J9IH`U?E8Y$(ehbg5RTt@@M`sOM z>B&V?v2j|$ACSVPcm0{mmoM)dww;?MGj|Mz6lb=5z*TQ2nB3MW#zCo)tsI{CPK|D1 z3HYs%uje7Mu-2zWo;6ERC~RdOQ=!VSRQ3_!VHP4f5}7u@JXT1a_<JBDjX4jalK3rh zXrLiY7=K;sl{m)_4Q)bTYkbz^t)iw8z~u(7N%n45^HdSjc~ljyF`J{&+S~9_y<l;! z<mK~%w)zRZ@DguMhrQkzpLabK$y3Sqhm7P+;l-nIzkF6jDQvq6$3kcW<@!C)VFOYf zBFB2Y$v@X^>zGpH^XHFJUnijE_%vXKsI#j?wK-j#DA4`3V!hiUX_sfee8`zuimufk z2f6N+Fov8f3dvqh<lKEgm5ON#hKwngmvC=aaOGk29l0{2>qHq3|D3i`qdE|5AvL<n zU;fAJd(GN#kfa)?<vB+&PJF#4FWhWnQGej3uEdrTdrp)M{VXx1eHzMaX?Y%JIJbD{ z#EDaZRsP!^Jl^Oqd#g`$m11t=Z7Ik%gASO|Q1Z=Vv<ccdP5A5w4<1-Rhx5W{BH^wn z*}pd8?7Df^xW~*yB{Hr?<nsT77+D7{(N1d=PA`iLMR3E?1eUxf|HT8r&>7e;{>$0z z*Pg;4sl@@U=uXOd2TfUCS2(rnZ_vnKEK|#BMkCbopgNY7SFv${0;bB}0C^bBz5En8 zVcUm<zN{2`i2J{llm>*BYi9fH!Ok@e%Ih2q9-i3qc4>LLtemHR{#g4MMm||qQOcCU zYF-^aOGknJ6&X4B?if54iJCPc{S>RLLU^;{kEb`*-AE-AOgxwSbTr}G6|qM?7yg*? zFaIL{2?TdhZc%~+`2xs)0$+c>fVnNj%F`D$iO<~f=L;uKBTlAEz9P>eeouY~P<crI zywsOU+-M_p%`2gRiT$g<YyJal#5b5tki0QF^x;uNXy`qgqZ}$fNbUh*I5@>TV4u6O z3Pu#ZG@yaDP#$L69r*HIx2pG$n+f@P@a0*b-)As8?tjK5q${z_knm9E=E44P=eqan zmxOUP1tK*90G>QAIYCC@{FfV02f0$G@lR3Er1|vd7#19<qNp9_K}Z~SQ2a5ywR^cE z=!-gnb}@;lrMu<--qMB7nmZBcMcxb~1kYlrqMCo+?bY9J<1DRtT~_Ro)4v@#`uk_j zkafNO+Y0cbLm=5xTu0es{MTVC9gcY{Paf6RCc!=vq}lqZXPnZbnfw}1v*U;WUmJ97 zHWk<&t7O5gk|z8EBGEyFwODX+>f486ftWBtA03-slV|QUb<_Ed$4*(!lc%3}CUa^v ziAy%=8>|{$J&`~0=r`fR<gc;u20Y9Kp_wh@O&4(;ehRF<)v=M9+nY0BzAjsKv(<xB zM{ur*hfONnPJ`KksLch1RZ6H)Cip9h+zb;GoS4S}umfkh98hr;Mb{0qqjMz}0=gpz zJ(M+W{YhTqV~KD@tmINGTUUEYYu0kY3}$NGA-Uz_Csskdaj~x4CxwELJDfjqtet;D zvsUSbnws9pXBlsI7xV07JsG%v!^3iZfTOtz?)l^Gu>1>cfcFhEN-!^<9FG?e$>SgF z>Lo*0rlsMM$bGt%Ft-xOFC2?Gdd1r8W6n@5BT?7sf5b|O-GUQAKRjbRLD^2Va1-S@ z>GejFf4sN7W2#2aonI5{f%1$rj~%kawdSWAjEeyA7p9F_WhFM6q16>$*h2Q2IH0;7 zlc+s(b^=s{(}%L!01-?4luA~sy1I9*t7x6XV{F{vl_M@cWGij6Cndtc?)3Pa4II;j zo!>94(w1^zhbSG>(y|-{mXP-~>$n&v`D@OjKD}Fo<N*@fGPLZVA;L$Vcl^h_H+Hpe zMGNeYLH9Pesb?{5GhIhHsWEUW9GV_|*CBsF9key>THk7W<H9A3vru>-1I9Kq1-8Ws z2dVy9?*AiJ<;TNm%PE!8Uq0MmR-MJu^FW)j=D?1~vdr-svgmJ389Mm)dIx^`vE#(? zx#}n_+qa7MF62?5iGThmkYs{Sp_bxV{uFeH2N?O}uIwxzHN5p27l9aT)3#?d&zAxO zwGTFWnM8q9jq1Rf3vHtO<=Y4y#H;5w_1Ag!dL?gqT7TEXr1noz?GDD8LE5lp%0K+m z)p>xTif4PHZx<ySu>-6q<i5hqZFz2a#otx{iLP!fXF$#hOy6|KRCyL*C#9w@KrH+U z+Dw;4K~X*mX3UX46K*Co?T4Nq04^~0FNzldkx-I4v{Te??Ry9q>^u3P>O*23n>KHb z0t6A#Ztrxo%M8$U(i*6?Uj}|`qul;lCzgkP!^^|lP{EzEk5@g|XkV_s{DQJT`KGWK zxO4h|zjy{Tf6VPdfMC2_GxgwD*m3yw&-(iMHdV%tyu!Fcn}aN~g>-cTIaEpkN~7m+ z4ggZ8!DRxRz;y1ewTo|k0kAR{)ByATPO3$#I^u5ve)IiEG2l}86~){vK>g+_7^VsX zOf6}{_kRH=PUdE51D=Zbsi4*M@66L==l>y^jG`$_Tc&XrdHpW{mb%p;V1!wg=pUur zt>G1n;7H(G7yQlS3JzRZkz7c-9pQyT^13c)5TUuWBPe!6Wn`#cym;{~V2Y5WQDcVH z8{rQD@`6%DzWX@@C5T_l(%A|o5Lf|<d!ylTpj>90?F}FyDJc-5gFcHC=!M{{6)Zf( zA~v@`8zR}Lg82dXSHfttaej*$Xwjes1VA^tKQvp6H&#jjVs<gLZfUwY+qn7F3h28z zmgD8-$q)jAti!;TYRgrk4@HFSc`8Mx(_Nq16I2d*y~M|g3-aPdSxPSR-Q3Au@hvl> zNe=Fs6>wm-J$N;a0i{m{O!Q3d{swlHDnMadzQvBvafoS)1Vca89-$9oZzU+VBh*5A zE_OW$B<4$z?hF`D5gzS6^U3@;CYz5o8cPNJu^ti7sk6maGVQhLDSrY|+meX02g{wC zadb-TK*V1_!5N?J;OMpk=OGz{1lf2hh4kazh2Q!_YYo!Lv!8)Fr4Z~j%l0o=<o~j@ z*TO(sxFaTbE#)|OoY;1=VB%HdX2WJr)39x64K+>;r*_m#4VdXT0^|5>s>6sj!$AF- zbj&VMm~qCn=5orXuIqy22vA|c2{3j9d00aPB)WEW9!&Y3KtDwqw1Y0kA3{E-!^w3P za^RIoMS6b`<yD!6Vw+V(C6SM&ine1^v+5MME*_TjeO6t4ojf(G%4(L<Zi9noc=g5j zwy(Q$Wye%TlBdWJ`SW&^D_~B9?$lh_iKOkF^80BPD2enx0`$NB`0pt|2d{_j=IFus zn;)^kJG)(RmlYYqZauXPO^%Q-R{|9|Ix{{Jnu*c?dChABy**ff^4M5gcR@$?IpEP4 zzWsvXI=C*VfK5r4Js~X#WPy;^qydcgkueFVzCE58sFV2yq<e-P>ECP?=0WeO3-E+Y z(7z{50-e^!TO9GY0fgcMI}8LKE!iWmy0%EW0T8v>XS+YR0<K32;C%ffbt+4Y1JIb0 zl#qB)`V~-_&))b7O8@xr<1N@*AvkaPhly$UZWeJI0E!`56x+Tzivf2uZVYoI^0SJ9 zMxr%v@=HOt#asgn0LoGqz;`y-1_{sa7qsQV?FY=Wq-X=cnd=y?z{~_7zaeq`vrlDa zzRWhhYc~d&7^j&+?REfs+g)cqdCWf`@R&eI=A^b~|Jn5%lndmK7w~~?Yanb=>F(IG zEbu4K@XoJ8r$0C6>G>a;0!mKrF`9H2nWa7@OdUOvZ2Fjx2!1f}Fy+kx%O~X7-8fYh zkw%jZeIYaUr&xE5!u~8W*s$p~oX4SY8EI()@KYRvKKM%<sGCIMrUAiT@qy+mp9z7n z9FP32lW%oiR0A|*n#{%0xG=N6`CxtDGmNd9H*apGc;+I_6Hp&90NtlaJ4QUxE8+$l zGJ)B?H{$V!HGIJG98l&iaAozl*5gR~%r%OG{QY&cZmjD<Hhf4`W3G8aDaOn!ig;Ql zR}&d#U!%%Nr&;jv@sUemF3$rH$Pr#%C0?Oy=P3)DFagy`KjyO_#|f<#<Dnl6YpKl+ zM6^2~Y-al$5Jq`9KR?S8I(+!jL{tb!xqXE(St;w}tUYetb!_&4Nmb(dzw_0+;qgvE z6;;hc>kW0XQhj`dTKe~`QaY^hSOauMmi&OSDoSQJs^T%bX>rK4m$7lSyjDFaHD^)Y z)HT=n==cDy(kp=SRG@%%0;WoPEix{AgZ5rcs`h!q(wzt^5(@HJk#$9UCb{~>TNz60 z7u45kU8<>124S2TV%{Rzy%lk>v%syJ8AD{BQ4l6QqmA|3CvY8&dIoo(F(K`vb2l5V z5;A^lvf-rD$IhvraDhH}PYdYINL|1FRLu>+Mj>T58(6`oq*I3o>R(|S-@MokprKG) z9C+34M{)+C5Jsb!=fRBcB-qrYcJ=o9VA$M{y3}!<${O}jxV^CYQ9n~=U0dylQotQ1 z?JD{SBBD=_<)la^#wZ+2Ab~x5zB)#M6?}xw^n#_|##(tqGmaGXvsHVq>)}ZI)<tGE zvO>c$$Iw+`3#-USGLn#pkc4+=7Ss;_Qmi$P40$KbW7kg7ZdHZugS%9u&nkyy8tKLR zMP$+~w{zx?0%SRlbtFf7(k*)|@tczVsYs2dwi$CFW}*F#rJW#RL@Ug}A!BW0BbTO& zZ^J8R+a+D>LE4i@jAN_uU=lK<2GdB3NmIr#u)yuJVBl&8Y&!D4yHj8BZE5jMujxiD zs)qjlexpRp$y^XK>4pQ(O+?<cgrMMCA26y*@CQEg1zX$|2bQ+sMUn+mU~>aV1!dkF zuGe({{DMFscw9{uXDMBpWzHa0f*58QG@mWDD({j_d>rU26E03S4{@HDl{PaEcxt%= zi|^j}ax;F%3&49R`b@J<@<@c}5+n`SCIAyMwYV0k(GBX@fSl)ZghB)_6vH7xO6BbQ zf|9XUq+TAwExPEbJ=>2hNH|Ppi`2ip1e3VPFPW>nUFr`|bLWTYNZcdej_x{}YK=!n z{C>0>EZ7;Byl(f2H>~M+$JH;?t8coyK101XNVEiy_euJarhxgzl|tYqkGueTa|r=x z6+1OA_0>lP7VzE2H5vw$0Gq;W=!0llX%>c5U)qP9pK|h&10h1>D;*LAsCXB^lzC!I zNp?B8Fr9e{_8Q>WF_qD>$-mvY>=Xiy*=TaSz&h8_`V8#GANc4!ig#8W@?{m(2Agx? zVeH*@t&wYLTshI0FJHb`!-0^#a^-$G*hw>=vk%vy`-Rov^G_@w!OW*d6ey)NpSMz* zow^FgcQNoT$Q8WI8`D)*NhHz5Tj}8zUE?IC_>YQF@2XVgw5%*mmkda%U+6wQ=Y9#Z z+-HhTKlSpCjOv?h_a;IuR>}R{-P57@<E-&)k`hpEx#xq_ORJOGwc`y#)SQ;J4x^Tp zl9r{<%d_M7;@;FmUrAZoa}{sHAqcNn`@2=E18B>INav%3=zMqh(tONLKnI|~F5Fl) z=cD%u`oYzgz?DJps6;6>@<Ra$r0eSH?96wF<LB)Ljv#$kQ;<=#Lm0LPzP``ZQj8p1 zE;*>KwI=G$f_zywY~$yO3c|`DiPEzvF~#1D&z>%PFp{X&=@Q>s>e-L@FR|}kKhYlh z`pTirl#qBbSX(h3yxGg%N(H<H`c0(rKzugzRR}wK58gsrT=&lXc$B~ME2Y)wd<(iP zep@#OzDwp@jeWRnqdffc0)df7<i6j5uj|p6uqLm+V6Q=YkW9SGjzbCpmln@j83p$4 z8;$^^A60`ozC|6)@qxwl1iY%tQ22{`4&xf>o_c%ba2WFBYJ!7w&(557kUBD)8S9Qz z&O^gk20?m|iHeLogl7I4fZ|aQIn(O^90nGrzSo;9_1Av|)EQamXrfxi!d&^bRpU2E z+CM!~m_t|R954CpFekXdr~suCV=!KQIS2?CV$lGyAV&T+fZ0EU^qN{@3h~h24Bb|| z{avp6q7WoW&oXrZNw<TAW!Nf-6vVJH@GiEp9C<gw6!TG-rD<HjV?OZw(ZhzBt(g;< z!beY^C~C9+QQG5)i*ANyC%^N=p8wWrg942rEr0aXGDmJVyj?>@QZeh;LRw52Yu^5M zx7y&sJ*3rxW4}&#=y>;-N>UA9uq$Nu(ibm!;wB+`N0bdVE)mdxMcS;t^wm#j#k&t8 z6>8I#j3ivbyflEs_Sux5F>(NclncBJ_`h#9aP_xjrtH7HH2u@ENIYgm3&8Iv4^Pc0 zS&Q{K@p1bs$Zr_AXc-t7TtU#WRPgok8f#5XM`QKxw@jhllC5eH)J+3i%K}XD{2>Ob zLy~6zvWq?3@p(x}N!<EHTT&9wVlaI<abmf33PfN76PObY-!fP?J5~<0=BZhm>uzDG zW_6`+3SjQ~lF59cw^5u)tIMjT`8?%tvNK!(I1#J#%4G5Z2YS2M)!!~^$L5|mwOlex zK}xq28tK_>Syt)imv}4JchB7Y%TvD;JCk1gN&s=Bp_A&VSHNl64Gp6&Qw0;&+R{a1 z`<wtvyo1$kI$vE%>VYQ6ouKopATskd^=4x=J|6C7l;(Mks8`HGwtA|p^PFwA#JPku znE*Wlgb(+P&l?$z=PW~vNEzRntf2mp22ka*#M?K?uMXYh0B%qQH^rk!BFdBi!~~Xi zv2{>@cGYWG27=R{qCo4wV5bOeG^_mmpBrms#q!eA&70Yf`MT|3eZ|ik2<&qVE=1&k z=l8R%?E+V8b7ZPYh&z@mr}933|G6w1!x+IEOJut4WIO9?#MkYdA=2+VvyswX|6`8? zejy3P_QHKy)8z`K7g|zF*l(^*6GkUAf4AwJFRhQ1U(hiX9X^weOKauV<jTp&XrRt> z7->5N<;ZimIU3WQK<J%ZkgQyfKnPhp9zVMX$~3^{lq)Adw?vA!3*R{g1*kM+%6P?! zUId&+$6Vu!?T`*g;?U-|-zucD?g=r@(t4xkfFH<(S`SS_rPMoBIbxuEy9V+@CttsQ zZ9yFHp42IRjjL9`U$r=e;lcrEi(5B;4M4|hrAyUMECPb+2?#)5hr-TeQU&h)Sh)A& zYnd1sOOOfRu~)WFn-}avQr!mL9CRzI-V_}U*ez?w!7{Z9X&}MU6wAv492q+!P64=9 zG8P|yOVqquyi2_>xQ&9q*K11u>#jO`6fwbC|9|Y>)|kGzXFOeg3wMo-3-2HA=XQvl zqm_JbhbrC$5Zina*Q$O>D?S=avej}0YF8$d1mc&Ppf+dAFx~oW;Z1=Voj3OWb&jX7 zW1D`Zut)3IeiujLh722tYqHz)5q21l&=>wOX&+(7jsdjziqEc7vRAU9S=HmOJq3h_ z5@>>yPi9#CM2M3tS>BKMx@x`_JY%a}>!w?SQXm>009f&wL=aOAKGVPygjX$?@J}9y zQ_pGP+lLJJN+GX42vrZ;ZkFxa^-kl_g~efh6ND*|2A&i|l}9OzPRh-26^6Y31)VAr z-;kF<3**pMD*4rz5(T0{){sA|X3tz+<Drh!=)cdm&~Dz$(~f<0eo2i_-EFb2^2CBb zq^QG0uOBB2O>?jIu&^&zfM>mHy!!8cIePu-WRy&H-xX)g)#1OlV7t>3W{grP$r_5& z!}J$B2mHEY?n%CUHDvkf;I#+Fnc=snH2}+uU88s@2zVrcz)Vp=B$4)O9HEH2!FH7p zSb}c-^A5w$;D&xFHoeC<^q{G!DVsZiNzKDT3v|bx>iY>}#>U4hGJ#gc7g)KM*W>^I z9+>Dn2O^b;X+JV)E~gD-dY6;N5r_(02wFqaVPIY(2}lv`_+D8Mb%T5MSQFsf&4JhF z^&?AxDMuon5R~}NVtq|>s;NHvo57e#6|dK3uggX|3%il8>ENsa02hTZiX5+A0@`jj z*zsKv0JOFoD{1ah;Ux29IJhqFgQ<JdM>XC@fS9)f-&r2c?R_o%4Qjwlw_i8At_;`| z2DwWUb60hqeh&xk_02P)#gIU59Zz{}k0<_O_w45z|M9D7^U6$trkXsv$XbeDGf9{8 z@gi4iSp+X7&HZC1$o&#zZ0Bu%^qr+t8<NQ2MbUEnQFHk>`!}qkLbs`%;}~R#Jt_&R zdT{}$JY#X}+OGJeISUdgR}O@<AV-<cgkBTXED$oc?vDz}x*JK#z=-LjnmnNm#qL$N z`-tlJCexTk<$DN`mL3t~U6Up~jkNi9YbDl5|6q0^>LQ-AmLACN&q2qcVgWIs=hRyh zakBFA&ymO;-p@%EFw=_w(Dl)dS8nCoM)1LC6CaKs3ERdGi?ML$^wxfsu2e~c=wH|3 z4Aojz8MR)v!^mlMoOa<tHxnHx4{Do93)~*4u__8StxMZcmFMpt5O5f9;b(}fcD2{6 z5&Vwn!o<<-I;Wu3d$a<lgi1NOCo3b^8law@H!terT&~al)b+3fTe}<!P19x1pgg1< zM66LX?Uk06mNonhVS(Xn;>71<D&~y({0y3CjWjdmp$=P9(^v#1zH#Hmkwb?f<1}vX zt8pG~NV<ef&S4Ym#9!17K>dv9N~Fv@mDtw=2y;Bv?1rJ%o4TsB5$1QInpGboBE$gY zr5taiKJ^nC3qtEB)^A@}tR5)@i}p%}f!pBuEJzdWcy2AUi^I2DRa@E`r~j01qR?XI zXROuA*gALAf1PU2lOKyrT&<>xgPKb0m0A%67Y8(L>YDicPJf-vs)5~Uo?{FOLY+ih zlnscH3&BzU_LAq=)a~DoNMn-;F3uY%&3O&DawPBFl~!tGT2=mqIvhT)<E4!R#mUPZ z_*DmQ#pNTVZ%EmnAQJ~Iht}>|y(;Jek1E1Px(c3I+k{^hz>w_i(#_g>0|wDN5X&Z? zt;XOIt3u%_>x6MW<vnaHuw_|SOp9tRB<+O`CXQ9aB|{U+R^PoF_CqUj2`zT}ID<(} z*|I`L;a};NDN47_>Q6AOmdyTJy5*j)2Q$n4H1`Wx)*&rZ0A@;{jyTc;1UFlC=)eno z)87Dl^41XmZwT%XCg2~7!Ps|>J7z;o`UvOKr`6YKPt7gt+q>Pu&u3M{(MA?{Z3R5U zM>{EBGjBT>nw?YZeCG=_nY4kyTQHE%zqSya0*ks2=79pngmqCdfPz>?b?D!d;A%}$ zlq_Clyfq3wIJSw`s<$nlCM6a34jsEFezMz6eh@s$Sk&36>mJK%cZR!uGL87^!}Y&g zUAF2|{i~hDjsF+{Qkx;gAS3Ni^7y9%4K!x};4=?eDGQvN>sOhSisjG14u2(j?-3<^ z^I+bLkTgRt{>!Y?_woU^ZuyccY8L1IiX;IT5Qlo(xa#@(pYSMM>2i?OZ{`<R2A3V@ z);DcDqWOvWZo|(Ts}K5NcIkAhoaYT$^GGZo!E5$z*!%bWEVPkykPFvfoZMN~+Ec$g z=b!B>+TIQo6wCfy?AHu+Ru!t*rU~!O3!A&=6lJwReMgXfxg{J1zY>;fP@JNGwL-A% z$I1DfDS4`M8m2l;<;x@6KaX}kA}i7JzNr*Wru!Q}jGQ;i>N|-(&fO2;M>f7_&Or0u zpSr6G)9gb7v4Q|eB4rD|!$Z_*Dz(~={+QSSG>PwOB(*1XFJrT|vaO8t-v$oN`MiCb zs&w(v#Up1AwMc2jtQJBKbRacAZLznsl!pstgU>eQk#Hk_w#z~IiK(>Z%^~t+-{ODj zxpyF?y{#{~vV16=u5<AC&X=B?C2#I^u)t0c74K|{c$OD6x{$J{oV~VfQ8E9L&;=b% zSgr3P#sx!kdT%wON0bfo45`Fzy(33?mQwiO@!hpSD?I%4N>y;HcBn=Omx^7uWo48< zMZ^3#*cdsM(#vhrX`tiV=7k>;BvUc3(!(zY*S^wV_82ReSZ*3v2?ChvoT2S9+vShD zB@xZL*^fP5?iOr;*Yz#<m=gsWg_MGFNj(;Bv@7Y={=cwyhEdbHvA_7Q8e}z7WP+?f z4tSYTNO=MODJS?x2>MSb1n#MKvMAZW9S4fScfHp;lqbKeN-r*eJuiH_=f#udy8PTa z>ZN%j_P&keP45F3LQO7JQl?nZsNvH*r=UzrGEt?ytQR+0M0bwq7f%Je25&a=|MZ>z z<-c{DMDOPXrxRs6iHQNgW%GSggKg+E<O5LWL`QV(HLB8O#9bxjT33hO?*9<sZUfFJ z%R$(5#XB(WAcSY;E1~3v2UI77=g`o-SsQ`D979o>I&I9aZ})F_EC-30ezIKNACq19 z#cd=4sj}OYdU?B<9fA*HQx7FSR0jw8An6x#N=GO<04LMwkFFf*XN(9edSZ4JNT>T0 zYJs*t+eHZ)qs|_nmy%67bLPx<^K8O7fE}fA4N=fv2sxT|?I*M4rndmUdue0FmGeI1 z@}|;UTT`a}rhfNUzoT7XNjyZ0A8n&N4L$tGa%5SLQw$E>w%YWSW$%}uEPH%zw$jfz z2HC_jdzk)Zb^lYZVjCA6E-EATJ>;DPgFz-Z)i{oC<SSw&KUfs3UcQ>MgOIS*@c-BM z1mE3NRNR*QC=o0WGKf@y>9!2V{{mUIeEU81_=i_i@=viR{YK>c^!VrYY}ut%Z|na7 zK$EFpxNe5SL0Y2JdhXVW%9y<B|0OBChmz7@V+qOzaR*yg+jCnf(U$xs9k<P2AP<G; zjsdNH9AOpCAoTas)z;bbBE;&evOHePce563;p`0>lt=?U^R{7BZvPKw-yKNR|Nmd2 zL`6}&Q&B2PqJ_*`y^A7*ND93rWvi?!bXyui!zdZ0Y$-xEsT85I$tWQ*vbompdCs}# zTq8cdzyB_-*Ll94<MEuY*TH+_TaVwshAjv7Bp4UXgA@_Uhf^XT_6G*!61)thb#t)U zwLmI}J;LJM1M~qjbg2lCe~5K_ZMe3fAy`a|qw}5n_q`6BR=ECl-xaB6mEDkNOfL&J zxVMqInBnYB1e#_sD8pL>+umvi6`(&^!{vLaFoB#d$v_{vwBFp~i_DYdX?$M%KNg?S zJlTuh^|UaIO>U>~uDj}Wt!4sa4u0|beKw^2j~7JFV$hvD6GTMx8C<|V$H$*!U^))& ztdoUzIUR&V6O-1I7T2E$a+C|;OTzMX(s6L8(X59nP?1|E!5>2k^lQ824Encz)Jx4> z5L&n)&+yZwYPaPpmKLtuas2%MnwXA;pr>#7x`mjtyGXKTVjelHb+Zc<*o6%+0jXzv zE;^i%b|y|1E3VDQZ>cLR#-^LEb0i(){U0GcsTR5Af_F8aH;QxM9`fK}#ot_o(1$ha z0`6H`d|DYUpepO*FPjkZ7S4dR|0gGL3*M8h#BhA-Q8+Z*CcptR2fi|<$XdM@Qld_% zevER$k@d^YHm<r%o=E!RkV3fa6Nbp_ao!9^oGBRqcdPo(ePX5|xfFs>;_J?%ZywA* z{0QsZh#rX3QmcO@ZuwsVE`rHlvherUv}s2HG-HSwL}%|!>1t6FO#t@4A?goveV9^z zC+yjgGDy1>QbQ?v^@{)Xy7gLpAa#+ybdnqD3Q1-lj#5px--L12F#p8nX&qy+dD=** zDbfxS?DlS=y+?Oym0j@|u#TVc=UL1oA0an5-iX~7KvN7pA4d9QLN@!D3n5n)T1a9y zl1}NQNsg?hW&M|-3VBFbYRCRH{C+*76+nMqVb(CVe&{^dUeCpRVO$R>%TW&+O-zE- zXaQslF8CrhJ!Dw0rgS!&bhgB@)?aPpQaG=1U%o4SDOYZCd#Z?{x6{TN$nHC=I==FE zw4WU?9Q8mBo;Maa*ib(Crjr^zIVH6`s%sfmM*8|!w4b<R{Ib&q&G6oJhV~+GM)uG- zK(6)01NWxpCo2CU3UoK-{iNOr>+CA<tsfO8ABNkLnytKDf@Rj1mn%L!_`3u~@VK<o zWwqi>t2Tc8$d&W9qwXp60veWs<3CSx{Ig~|Bh`n3*gZA4z$tP}v=MDnY)IZ&51)lJ zkH;>u8A{FVMsx15GgIT)M_E391zcPuyCK49Bw!)W&(uCyk35}tpWkqskjUtTtKTF$ zY?_L@w%gaHZLqo8E!CbTFFSHF?AuoJ$OXUp8b{gI`sfCosm0w<s0AWMnK|Fk10u*{ zB~tJdL12ke!iFRN;}k+*H51J02+i62&(AOJaGaZe-?NW>hD*Qd**7trsS5XW%!`M! zshQqvcQ}#*_*70Qi4O{~20N#ox(%n+`p$NLF{^jZmn<~T3ea4>Vx!KhJ;*yqnzZ0R z{!o~6H=Z&#lfZCS27X<OTX?PEm|P>*HY~Y841(y}Gb~<F*8Y*#W^^cMtLpW5Uq$wX z7XXaj0phui73sN|S86uoJunQgh&s0Ar*SzoSLAMN|3YN&K!a0h)U^EUm*!@;T)<0U zmG63J2A0%2_3VcWYlAQ6V+aC;49Hrn8aCsqmT?D*E&tD!!E)sAmt3T7RJQu^T)owe zkA~&8+J8O1Dq)4V)04<!aR+qm+O&5DAD431r+!tDyO-Ras8!*4_<(cDru)@f-#g#Z zUaZvTqiciF#omuZlFn}btfn!M@fy927Hcp!9t$N{z?E$?JWs1QjaNP*w1DFVaQ4)> zqh4!kLxaxA+KVGw&07wxvVGlH9o9KAk2mMjKPn1+97&1LM+7Umm+O`w*WeSQO6C9h z#QvQ7@8L7@e)9@#?>`%YLOZJL8R?8pLml!7i>!Zr&NTB~NPqAsyPb1>K}>gIo{dr4 zNbPoqsv0eurArk*@;WP=&bsz>jr6CQWX1mxfL>A!gi&T)9&5N}rLTIFv|c4q<@xKO zt^0X)ys2{{D-~?uRpKQVdv^Xv*Y%Ri840L7$J+3~GAa9=Q~YK9q=19WRhS_Vkp*Ga z{9OHi%JVuqh5}lzJ>pcV(dtZY85|C5a`3%w>9F8`^<ip77}A)7qep%y{v&0cvfx*Z zig}{FYxd{t+W~w+T2Wq&T>0YPyX*@#d)Rkz`E<eEs)q0u;=7M7JpN|$TX<C|wbD(J zEqJi<ViOEU(Ono#6*-A!eih!#?`<4A`z~kenMR97rg8?BZy7ufE7kO8v7OC}D^-Mg z{?E+8V7qUP)Wu=QtAiDnrLSIm@Zoj1QljZChhfgo{dZ*>P7Qp|=B<BuhE37^ZMws; zb`@xQ4;Gq!=Az3L{}lKL4G~oEfPVyOngkfR{13Y9bOK?xyL}#8&O7f#ULGFTVt&Pq z-n+JNd09fWjw+w`f+VFMbw7SKMGE9O3Hitxm8dV_<rUD2)>dDp%qxHxuu)^)o%bhB z^!ALz!xL*xT^X6kKM0REGe9|2mjB1onqT>w<%ezKXjKQvGTP<DHThp@_r>EHL0Acn z;%=%yYLeLChamUdQJqJ^;?*7-y@w4v977s)`@KIs9CmeGKA`f?*?0A)(pM@444>E5 zAC`!1r?`adSI*ZtTVA{CtijQ6$JBI~=jcoL=P~WA0As1Wcq`+_Kfmi+wFi3r(P+XM z=F-QxpGj1q<faI#9u{@%@)ACqbzA%+<;j|X>_A0$`wGX;T#<AyZ(XOo%A0hYhrY-^ zJruMrSP|v|VAe-ykIQi(n3}%pxpPmFP1BFoHiu_2P&hmBl$IVmTK9t7W1O%E-Zz(I z7E8aKw^k0XE82e8ka@oK4>DF#GqV~CDUd}2Mhmx$8fDE*wEnR&28O)%z)hKRcBYwd zC61<T_?n`<&G#8HWbkQogwKeoc73<<N_S7AQ!<A0%3E27^{<YNY%`EUT8_f`!?@Ha zV|sXZkpo~oqp`N5nO4;Uj%9MTxs@_EGpK9t%Bs{h1TUDCC{wp$;WhP}J??AGwwIF* zwyy7e>Zf7&;Kot?eI`5I1l;N*m3<{_zBYvlaasEKG*}&zvZ=FaVL2{uCJ^R(r|c}d zWF~LeERBu<*MADG-~Yk1mR$PH#J0~(W!81MS(eVy1+mfXpU(9;E-R3&DQN8Z?%d~} z`rxN*9#3ZXb`^OuMrL+C(#2V`876-SV(<TI;j}Yts9q#n+<By{<3YiND!~KXWe+Yj z%?axH5B(Pp-Pus7%im{6`+v&3`5KLy_uOMSm7LqNlD`-yWQha|)Wxoz;vvYtdFa3z z-`q3}&^2nmGr$Wp6FMOALTMu{XVP_Oin8k~nV0%CxG~^Hb6jKLk%nwz$Sy>du~_vM zeRLnLnn!#7-!{U?{8Ju^YI3nf^U*&`yn?*-ahS9MZQlx4Nsflwnsa&f3&-_bJsHrD zryRTVc7E}3_4*szy~D-+!W;V2{8gdJFn_kAAIig%hu}fC)YZ3Pbcp6N%)8uF9KrP; zb*QQ69&>Pr)4wchr(wlsabQK-A@XAM|3A1vuwrxSnHtUtE&bmOexb+H?&xcHoeGMP zxc4o@n%~F2<YVZqzlxB?!BHm0!a<AX+vtycJ;2{E-a8EQX_DvV2+faLA7Hr7r#hw7 zt&+E@wb-#C?A{^CLki><)5(p*rw2<7JPa{M1(orbk+G2N-OtrHwc(7$-qmhVaeWRu z{2bbv?0Ts0G>ZEI|6?rVefneDbX*<aWLGL^JZ8NYmC&jUd_E%ztDJGhJyi=#i}|Og z{`S-O3hM3{m9A6CFDr$jk8ON^vN2|AeTLnC2myx5HiiqK5C5Y8%$D!$=f{La;`1CM zEsc_-1i{Si)aAW&Sx;WOQ;5&UzoxtyT1T*?v<B$H<Lx83fED6M*5*6HqGPu6c)kOp zivpv2rM&O>bSqL}#cDkj3x0eN+&mX3>05{nU-b^Dd#B~UvP^rJ+~L4meOohm<-6Uh z^Zg~uqf5zbw4MCjpf_-^E2)%|?(doSXFHz^p6GNA3*GVqBgb^~6nf88wQSe<^6OaD zj>G;6CJ&m|OeAJ3Xw=RJ^$KABT#73cvAwJ%q0y%M9PRmcDvQcIo?e&q--0gjuew+_ zX%&~HWl8n|#@-fd-L-J7x2oeRER<NfV=>O1u>{e!_bVNQJ$f+p4l0|cy832jW_q`~ zy1L*xb>iz!wE`s*9&&oa73N*KQs&O*6STS>gedQ8`ge6$&9!NaY8GXPnK`)8$V(D4 z5tn5!BDvjj9!`E3hh6KlgdbHpyxq+Y{hil`m-?$5zdet&!n~zdcBA*rE#--urZWM? zP61G@^ast?G1bxp)q45wDty0BHE|nALv7%9gOh0L5y7|R>H>vFgOes}>X_fyF@W9+ zWON-ypYL%Peu`99uA+&Ja{Y29oONN{;y7|X{EW@RsMF`4w>{jb@-#{F-E{2cvy_H~ z>JkT*p-T7xK$aYw9b$)VQvVZEym|GFih7<KyA98XZM?PZqvLY%JcriT!(Xa&&(y1J zSuf>{nfn+@@!cZgBBcSmbnQ^VWJlREABs{pKkmV)E@DCaPDKp6Qv0&xs<PJNO+Nmv z`O`^|^c)sEw?jD(RR&5HvS9v4cwq(h^DCC3&8aJOOxpW$r24)Ub~N|gIwQOz%p|1z z=Dx<}=V6`yb|x;I*4_r#0mZUcSnom=(rqnVBD8ih8wYJuY67Zy<*wAueQH`}j;qT< ztYgJ}oM^af681k#jvI_6o74rP89=RvH6lS3)nSbEjMuCy;AZ-0-Rn(VA?gJ&2s4gs zUTP3?PMfy3zf=7`oMaS=Wl;o{`S!`C@I=`pYgKF5uZc1YpUwl#q8Na&@XpE8LYYe{ z0mO)v-PMEhVomSdLOnZsN!<(e<wdbH>{;jIPqo_MF^vso+yM!?zpX!kv4IWPaN*M- zE*cx`SW`NvQXLJRQH+n{(>z)VQ_8{ia)f4Ld#l0r+;=u#n(%P$Z4_$GsUmOjyU$~n z;nBxaCmu}4C&DUC&@sZWSnEKT|HQ@ADN(+*PizMEeRaf5;;6V(p5}H*E!%^h0$=}^ z9ZNhx>hGD-$`c=~a0`*uhIK-j-3_TW-$s6!N4K2~<G$yYJ1vm&4T4sc2RvRh!2&5; zmJ(Q&6o6q_!Iif*1_@dr`sDTRpD9eYf0rcyjGXseFv+JemXwtA?$|CTm)>4Ym=8&e zv02FXkY2UI&8R4mwS(I&#^6q(M%?tcgShcMyX`R_{Bu7<U@C=0teBEdh`?GF1_#(T zpS-Wksiqf8L_mKv3p*rA400UuM8)<~4VLvbDCVRsdyPRyy#3$S&f8DBa=w~sEHk<b zM91?ar*`H%`%lSc^YqqW%E}yU<lq!8fcn*JghR`GMetp!ZBX^oIoO<|COVwT*1j*I zs;yWS{P6z@#(WX!J@O4lJx~S*z?t0Q01@28)YhpcciWgorPf3=eLJU;9Z@P09G>Q~ zG`7%mZ_qhwZ+&Xk{R@*cX$%1KNg?Y$@H~L7PpSda8{F5lm&QBF9o8Z(v7oe+J4J`k z2%^SR+{R&@m*P90mSdPl7_-(mg)LtSFxl3kfj!&}ti{PxQjZAslxy`x;Xdr$N&#|y zNs_!8pNHW!g{s$MLzHEIEe^nWHbG@EDS0dg&2U|C-~mcxU;u1jxTt$GO<Oa5F@>Lt z*uU8GS$XKD{Iz{&s>cL9=faF9MPNkG03T|%z+&#RJB={tk%tFKD<4*&L5~D0C|dfT z&hsh#Ik69FPgk2{x4O6W8Vk<=dtxBk_0LNTdx+|t^c~Pbj*fw?*p5hi)(jRYBg!fE zhC*QxD5|9)sCIx;NoDY9k^DHuj#um2F9^lG9a<R0ia_x(x5G4dm%)vTkNM;VpEOJ- zN&(oLGN*H)vnZyZW=1Bu;p<RMc3ceZmo>;6=ad@%3+pcBqRll5&0(AJ_8D5OPlRY4 z!q`vh0zv1Wcr=_X-Zu@&7c=&Nmhm#w8B7Eik{2T)yF_i4VB1^OzyM>BnXncKsZt+= z`Ak4!zCT+irMhGxI61hhy{l=!=n1)Xz+L8E30w8xAM;lgt-AJ*3cm6R!&PVE|6}3t zD)1U3%qyHHf$Yh>c(C>j4b}!vVL`-ZNC#9D9mmUQJU>=#f;IKI0+vj)o~`RFMSXNr z$*Y440`isEDsNZmM$IIOiU*Fy?hq+P7o?;yM@B`pIur~I4W&82ND)nN<s5j(eU%8g z9nZ=6=x*Zt(Gqs=jEL5>L^H2|$VYD04=<EYu9rcDC6u&bU<+A3LjsM|;=zH|oQZ$( zB{9h)T>4J;iAh<>*d2p|0J66T9Hx=Y!W`V^S(uB{8@4#~I9V#bq`JNMTw7vDUGZ-& zb@j?RmNPWNTQw7>AHxsY;inyTAEmncAhZH4?(6<blZS#QUL-h(774EAA^tB|pS&#i z*}jSDIaWUgA=c@0c#WZn0jP-VGJ}$}gd3sP2R<p#@#lk56$@rkgiOYN;DbVAIs^J# z(Q9DX1`_9mX;5##kGKzV6mD>XV+^kMZt-*&aWSJ5_&ltRvED!sh`qL_l!iT`Vhj$K zn?zEiSBj){ng)|=!0ei5glI@Kb$e&nU5v4Zfw9q#baBI1@FbE?<8Z|JZ!*QLE)1-0 z)BVV4*~=&iyC3-nRb0vzs2lKRJvMoO34ogi57(UN+NLNf^G^^Fd5o15jl+~8@Xqu? z%`g{WT6&l`Zh>Fd;Fy+!V_Lx*v;q-3g9?NKl9W=7p6*6P_t_RuIPD&JHID9w8Hy8& zpZaf_S9t5{?ng4#TbtfZkH6hu2_d#(8!`2o0~tOqcN7=l^jH6>P_%fLoFMzFc&p5) zl18*lz|EvZI&bL(t}Lg#bK_NOJODAfLV?TiO26Csi8X))yJ@KQvlz38gLnpQoT%aY zG2d`{-)s(BKVKkkJ@Mfb{xRu``;h-3rxQ@#k<t})ysvI4Mq|_j8Zqf-f!uhjbM8%A zQHj~KDup91SN!f#A;s83+K=|P#l9$~gU{8w#ds}nEa>36p&tn*oJ}5jh=ptP<vTYb zmUc9kOgOY|gXv+63g%TP&ao6_4TZvrps$pHlao_TDv?A`T3GSSbNNnM=J~+H7?==| zI1>$v1`Q({1#QK^K@`|b`9j`;YuFBk9?!f`5sS4wMI~H!9W}+;9qq)Aw%SC0gN#-) zM^j)r4!=a~EBB=d8k@X74vO(@l<8Lh&D!Qs=#_)uqcm^_b5jpe-!%2A8Qh@4?%zMh zDWO6I_~F5Y;Oc_pF=t-kIj4|s>wEL)qM4*Jet-&1k8XuyE{6mHoR{;vo+dg<5%ZY% zEG(?`#no?Y!Cb6YB^;W6o=9)lIUSxeUIUjbd(tu+li_wqY(q}47~?cA(5<~)T604@ z+H~E#VvNJ|lrBasUJ@@5(bWj|VvNkLe&8z0ZXe4+|2PKY!EM?x(*o@6zVC=)Rq39Y zn3^o;KzH81SgfvJ3#^$VV1sKn-%=u#@xoJ?NK+)0QDZ8M-MYmZ2R=HPv?u}S4MJSI z^6>(MUeR8}4~?qKqOB~t4+XQz$hn2KL=|c6a2fI6fBWFE3ZeDW?sX^!^iRW*dxC7( zY3if@lOZ~_TQkMs_hsFcg<S>z_EdE7M>mBs(epYtDoX(le}ADE$kc}?%%0;QgRZFn zJ0LF)d9t1+@(g`E6V=s4@)+$3B#)_|NgmS#X`D)_v<8L%*VcfRaGr9(H>tLo9<M3y zHIv`)gO=Ml8iq?--H#??!4m%U38Yh2gPO6;E?+!RYbb~2<JeY5U|S3g+rajO)YYf4 z=nM%6-`xve^&FT0sIJct%WIN~$m}+sgT$7+Gl?z92)b=!^_;Ycxohy`)1ePuIOV1l zt+(RC&dW|sL@u|8D$za>UdQpEmKjt*KBH4`B9idk!V&FRDPnJ4Vq@%0BkAaL_V!a3 zvid&$5<Ya%{?{256%}c0jg5_cRmnu><AZlbral}#zUm|EEP3vrSbJPuue_~;wf78i z+a?HvRkBxTU?sq78$l;D%w2HDsJ0qS^h0TpTESN|jL;I5QB+qKnz&E8gfb*R8Qin( z_h3$I5&XnE%Sw^<1@1YN)XZ)!U4J(&J$tcx$J>cY{)dxnHz#@VF|zU)m&K0HaH0OX zh^g9c*w3yjNAs&TQn55TL$6qVP#Z9wE?)f~Ek0c)TKT6U7sjS5Ga$4sm7f8!iITyV zzZ3wxBf6sh3qcD3TTGhp^G^cV+;C*uWQGTqACchF^SS19Sf7ZLCR$JLz+%<RN10Xl zT>z^#LtbXd<BV&`Pi>@+@XEVhpGu3ZE;W5rFdL`BbP1ve=?dK<$EQP!FtrbY)S35o zvO2D=LCc7mTaFJ_bK|w%pDpVQaloMYOn7q?;SE&^YfQTS3lCe{vIrNF>oGhfbwg6p zf(m;7AXYPtj8icYZV;vXsSHMA%U_IvvRw+efV$h#n1(}(Eoh9j+>2_Px%LSFz0wEO zMbKx~dVnllu5@zK%z)2`kYxm2mJj<XtOkRUQ}5=^py>7m!MgnFkC?8Ug!TD@Y(;77 zgRUTv_ce&TlMf=0pUNC8z#oT;cZ`(~VX){{Ao$J~sY%rY(F>G{X9)%-t;5xsa7Ub^ zZmN@2|K9GVw|=LDmit8{2j_4&k|Rq0wkEN-<qBB5E_7hgL>S-;!r`tMg}V|lNXm@- zCI5NGaWyh7P<L8&1|2KN6Wp<Tf+n)trO<^&?a)(d;y^Ipa0Zy4aT+jRzOGGU;&U@T zAQhUA<#=^XyE3Kkd)4W<%G8*U91d+Ez0|+y7S5BiNG|Ss$oL9sDJ#(4T}{@6-rY}~ zF&)u7(<~b9&kr`Tab=mtnS7vt&tG&clnJHTQUf?0bkiO$&wGea#C4qos#9V-w;x7B zfNzLC7|sOlvmjUGO-tlJ%_rjnM4j@o3{3c@_HD#FK6~JvHLPp<pxbcTj3MI>@Yltt zbuCJ($8w;x$aVK>8t*7~h|DII@c!{*GMl*0V%#s_0V+-n0!@5eU<HZXLF-YR%J>4$ za&M@IOEJ-(NvqHTsg5>VusF@>4d0)E)5`+X<b9q{h%K@a7FFLecQ5|jeH-aHsVJ6{ z%6%pKcmILdz8y~GsBCN+oK7{|por%rEnb6F*&aaKwV=D7Q-!9|zi!dWG#`_Vf3tp+ zz2!?cI&ITka8oc&$K(%Nkw#5Z#|BIjR;WP7)y_N(Z%Tf+N4F8lSm8FJld!e&)|=_@ za{Nq)-EI(}0Asf`&g4amw8rRu0m4@P4;qJL{QhzGFkTxYyxzug72hLh93-gu1$&52 zLeF-jlQ8W#4oUwHXvJl58wM166d)%dH}8AGM<X~xff(yB3ZRX3+$s7Jy@wX$FHvgu z4)4P$n9OBnc8N#eGQK~O#yER+9dJFu${4S6SeoKRn|2i7G{q4tYMKcaSt7Pl*TPAo z6vYmN5K=SaqPd0e=C`-q1BGD`U4l$iM7nn<tyz2G_lbW*KKPT60iQn?us+b3eR)_@ zn!g!;CED4`#`NxD$}T;eRws~l(mez9?GxcWP;P9I%~@CaNy2uT?S;$)fi5Eml=6~e z65qp?+ST>g{>T&3+3$#!Yl_hVW)i#^ULe$bY7S5~L3(avWei8G5ff7pd=VcX-@b4H z6uY|P)zK__=3sir`9D?75>d6-o>M3R9!SVi<hBpa#-n_uJZ8m{n{x!by)NfblItq^ zDCj46^5w!E)8%>}?t|(EzkhZFUHvmJtUe+^rga8Md?KjJ_mpNVOwL0fUdD3<;C2}S zx5(%O^tlubr2KYv(@nhO@2XTA=Usi%*23|#lrwqcL8z{S)5*B*-?g-6d0!G#S<-cX z%S?wu%+e674X+lJ#aM2SKwQ`P9=u>G0q=Q7$JIV^nII4sfkPZr$R+n$Ps5OtNN-fJ zPz}p1e%%A^1n{NehUdXCrhe0qTnXh}vfvE5DsuslWhHbhFlxhqKPs)KIdN*aZ)B9@ zNgYbQz`rvYuj54c4rX@$_n6&C#qX0bPxG`n$$Su+lI#9ED8mBNL0yJ>y77@b(kiqK z#A82ytj<xVhqu5{WD)TmtgP&Tq!2~!3KY7|;nlZ%8CvyiPxRxOd3fFzJ(H66T!>#} zxwR~?a61ensM2G{iI?8FZ{QLvM@_LFHN`Z2vR6o`&2**+nV!*!1IAcy)Dc!~-xgYm zXOn07=oN<i3r6%Tcx#RO+(~#aMggBfMP4TA-XSg&jJN{<oo894V*N*$E;)g~oEkBr zpea5k;EaDV4Rp+An7DBw2I2P4sMe>lil}7Yik6m2$(m>P6>`45-Wink#*2>#m;vS2 z6A7xC1Dc@PcU8OC1{It49rf9$ax!sg+%&eqt`G(0U@RX5$7IJdE5j1cP8a1ViaHXl zyOSwm6TUglZ%`rGuhvo)nbbx;NlbBWSJi%aJLiYe*#JxfoMw>Lebz%zj7;W1h4tYI zTC$LwP_p3GLtWE_RhNk{^YpY~CdnCG=V0A0VRT1)>R=%cmV5t)yRbd<0g(Jhv}Ck- z;IgDCF%0!K(5++I+`|Co8fD{yN+l4wU1O2Dp^9ERje!(-y!|Y#fri}GxYHA!Nx(S_ zK~Q_ZGjXgU`w6j<QyN77Y{7B*1c40PlVeqjs2aSzch@iSbhWY9&`xk4OrJH3Xidfu zq)J_1w&Nfz(3H53<C=Xuj%&F%5<`ZkZN9g4I`vRPdeKRD1hMhH1yG2B1N_Rkft*Ck z3HJlwmP^_+VpAcG(_>Vgu5c6q;DPg2xf%>iA0Pv?Kg&3AYPlbhTEAAljO%r3Pl0>4 zRz~ZlOT3Dm=lB^|Vf*h))NY*f9Pe0xJD+hA$w%jRA}#PL3lCITA`^m~CK?r;azF`4 zOI7j&IkB|#8sbi*O$2aQixC_HYyS!HM%WV;=-$ykF_Xm^h{O*WFChVQKiGH+Q|e?; z>T*B799*ekIu+)g<qy+lN{t+2VNxzmbq14i(uttH_jt&2FdKw$mfWx}Sf92>V8)nn zw#TKSp>x{te%@KtY&#M3zs3?uX9ndZ6p?u&tW2<K@C88Ts&q!2d6ew=_(<E&muT6B zN}D7=#t};@56Mlpv<BEk$bSoNPI~fFMZ6XiD$P{rORK3=&t#Bkyw;7;_n&I`sCu_1 z4stz+n=8GfPc9J%n?@lhuMk2Fmo!f%R0%1vY1~gBifo`=PgRJ{*<W`*K3^P5qAn4u z{XU1qYO>-o?L=O8w$OLqcuzo=6Y?b2-C(nv&H0!&=3jP1N2gLNbHXGAV-jAj;YMA( z3(A9iGq5<z3jp+Wl>_V1U44iIB|9Y_Vu>r6Dib}W*%Nib!zGngUf~Duxh$QNjH~&W z9TUmR#JV@AbN0atL|N#M6+W5R>4F#ABJq*DB}xU<?ZtS$JxNB;U@->552<7fpikCk z1heBO+0rH+NKn5*3T>S!Oohp}%^>O?o=Mc54d^E4?`we}yb>}#N4{_qukkE69B9+j zT6WYS-L?U5FMxZJZ6Ri&=Q%#=o6h-vK%Bq1e8d)%y?-oW2QHQUc&2bM!wJtx+0c-i zZ4A)cpWe^*Hy-26-iI`Tme{zUZ8XXk#Eq2HG|~QpvS~hjRyVfj4Zn#QZDhrN!wf=i zH>g58o!?f2N#OzHeg#9!J8%cV9gZ|{b`r-A*wFUrLa4GTsZU385Q0ITDd$iP%3wyS zSl8Q6(-e<1cPzRsu<7`7m)_lz9@<4z#PR))GojkYpbc!#I^YFw@Ouv=mKn;|G{U*D zi<!VTY^J~^55h#62V^sAqKvn|agWGQ1wvhNTCw=eV-YNU{?o3d_|lPh`}DqVf<8In zCDKVwpFy};i?kr+yE?H<{litzlg;l#=i_Gh8L!@aw8yJA_GNgbQxi<_f2qgBo)E#V z7<3AhrNCfp$XPUP$hioaojkpfZBobfy%qs=b7;+kf|c(PG*+Jp64?PSvvlmS`JFLL z7mdfkZ5S=p=!*3b0jGC-H{KHX3r3Li1%a)5PS`L{#B9FGF^6zW#jOLW^a9?zk>=^U zr1Hqn>w)YVl!+u|CS=ywT5SBR@hr+^yp`~?4o5d>Q@oX+ITN=x?yLZx7HlPPdB`bT zbbHVPRYvt&e5%dNp<VO)5R4qfO*?Wl?li;rV^yH?5UJ3{+g94r#@k%1yFcgC3%y%j zom+`ds{M`Ke7}K4?SMckX46|B+IOck2+@pVHp3z8D8d?acGtd~-cIlQ`cM|%7xC>e z9~K}@tiTjqY<Tx8GP|0e7<!Xtrye$AN(vRZvS%HkmRD^%6BcI!#>-WI;6R_nXN$0y zC+r$d3XR2h?AB<AXL=P6Hmdx5ChRyyOK~_P_`)3u{*Gjy`Us`+sXAo8e&-G?rD-A` z>(~n1vK(2p+eBl5_eMT8n(_Cr(*T(ad7o7(iG~L-hePep*KMS6iy}Z&;69q5wP_U; zkNsu>LetIe&Z0cVvF(v(A3hwtez;;f9`OA^#vtY;_hZwJKaT<vO6!VO(3qefNZ5!g zZX+qC^r8HdNl|dJAbRBMjorb6s?rmUn9d$j5}<OS3~w>!xhW9Zv0W)v#Df#stYRMA z!p+l=h8zV7KnuNEM@xrm?IVVZXHnaogs(e|jyP3S={dLet_gI~d+R)nEAIwzrUpE$ z#)`{Z!TI{%*|p&wu8?ZPpl@^%ZP53=CiSEt5{Za#TZ<0A$n46pk!p5Yq!ZGJYugxo z$9WoqRIQ`z!zSU8=vh2ekfaHjj$d<3qG~d`+C+kEOG8vH^rR>ERmm<Ixs-H~nz66> z-Cv%95kMI?b~TnL%L{b->h93<`D%f;CDN94%Ni$UG5gz^kdHZ5NiLdOl>E5IoHT4z zBV%0Pk4H}UcE&8qEeDFx;hX4&%ZQ*$Wi3AWSbYeO=|j+XHb2PexU?{EE{QdiS2Jh2 zfkJsm_T+AQ8BJQ9k&Fa}|E?*3dlj&gMWrU<kE)NSR}R#7v?Yz&w>L14ei`=cY@>FU zI#Gs4hF>_x9U~1h@9CN}6GGR&3<J<VUd5l7hI+_R;^N{Q59SpVv?trrD<ZX43~`DU z*(DW&so7BVJD)-x{!y29_hUL%$(GmdyW9}8$e%a9VO~#J>tmj`yf#NhUhHsiqp8&- zh=JCcBS2P@?~R)%Ycn!{jCsl}@Zu!&XHpxUFjFIunb#CE7`PfdnsfBe1-%aU*Y>KU zR37Dkp$d_Aaz7$QrbTubnB&KwZNj`RDX?ISE0qnn>sGp>J<j2(YgV+2twZa`?h9aF z{_QFx(s0_9(P@gXc0Pb)Stn_yL(T%o`PY3nzD?V^=)Xv<>FiP35vJ6G@l2~YRoh3@ zE{F%pE}A`Ts-n_EsL;d@lrI1X?(eI@&_C!u1)lNO+oClhP)$DZu~2MIF8rXy#8@<} zNFe#-4-$|kQBORR#!9Vc;mn<X&<W$KIIX*h-mZo!QfLwzSi-JaP%3Rr*9E%nVE@E@ z1y<zKNW~!J=`jGx`Z;Q=XpqOpkB30TEWrYtiL`^W4q9+T1%Mrz@~OO))4-MM16bW> z;jv_t0fA5?Ypa=!!zoFQh%O^G(gs(};K*?x{mAiO44BXI7+7#!&HzT^hZhjCUULaK zkE>n|S~K5!scHVZBS_ej3a5?ZDhH{pBV_9I2dLjGg0oL)h|)oq`V2W)9~%*p2kH^C zdDXvZ=ky1AjN3n6zj%?nZ|*ZC+VdxyOvXjTI2N>EYQlFLl;F&xoAAn2OuXfsv{S?z z(vSl$X73eicg^mB_Cb}Ns)(PUE7vvi$zX#2^LU#8f-MO70>Sj*d#BWDhpmFZ+04fq zXw&19m5cB5Jm9P>ADIc45WX4xWBBpb{eyTOAXW#Gl1+;(-8u2wI0-NGY6HF{EtQf= z?pRqfVm3JH5i6D1{b-?v)FeAO!HOJj<!377VDXKfj~x`4sgiA7_s2kPM7%D+c`$+Z zNY^b^XKCI3pi%R7i<&{l8eSe>okPxb-n{*$y0V-~1-WZ`CI#-?cZl*~H(ow`a$tq; zDat&iqU7)0(++#b2>#h!|L!%shvGuBdI0svfVVEyy-seZu%>YEnCP&9wCrCv4J8QL zNlQ6oPC0cUC6xhA3s*;nBviDf^7c8&949&Q-k7rV!&lE>+m^yx(-V>=;I(Wtw!g5Z ztI9o6Q*`)_^ayR(`=mB{Bz#~(FOyy3CcV>76^#DV;xk`bFc8kZkZ1RNz46B9%xQ02 z?zkjz2k_xZhTO2x=qBBPYB^rH5d&x8&9ctl?+8n8@)>Cu?M=`2tF-;WHj*jwHofCu zP4@@eli%F~`bQ-rbbfiKbwATh4H*2^7^9l79Mfou0L%|MGf3;<V_WQYfk;k2kU}dt zS6|D>Vq`4hU0-)@D)ZuNWr*m}yB*N)J8C`jt-xtZnw9a!w68Z#I(Kpe*y)$Eu>|W4 zBx}_fjSdFfvKUSfd9OG8h4+SvpF~GS`>CcqncbTXDn?=Oqynbtc&U3i{w&JBV<vD8 zj-1*9j+CL<oukth|Hees=QCr_8fjC>E*Sn+b4#wvXe6jVPH!-~qrQe3)&I#TZF5>Z ztKRi@qeJ!mEv3$mYMq(-)SQ|Tsg5q^cCH^%X~UI2qKCMgcPjtnyI*al35gZ$5H>eb zpze4Or;KAk))Kto{m}|3y;<z3eZhc6@}<>>H*r2Wwh<2bpJ>mX`hEw>9hM=_bgJGv zxgq&c`TGZ+^}qQhp55+Cb$r>8zi+5(w9&7>J49}!(CBgDk&MK1smuH37CN6D2>HP^ z>XObon$J6s)sZFFefiL@A&b@a+H=@~;l?E?5lt$l@QI82zj~*=iuNvWY&$phQUZ2~ z@C-TX51XNixS9iLz5JtpY7X>@F}iJhQAtfL8%}i?Ow%0tDPZuVKTL3$Y-0O4`pK<H zz){Bn7wI>=*1MhFYaq93^oO_;ztf+2BfCF+QW9t%7|`o~cVr+;$g1sI`l?bVYJ0bA z``p5s@-kOB-2sD+KIbOqmf?p~k)ff2_X1L*gYy1f`egs$Fzj*`%W)@f0j_I5Gw4x! z_Fd>09I@{lO3<Ue@82`}QR6J|K9nJ<SL+Fns)b{{&|sQmM;J%yMQR!w{U(M<p3H6a zpS5s5$I8C1&*XU0zYKJI)}y`)aE~=A*as)&f0vG)X&2();CPvw%p=$BJ@VhsORkY$ zLAv)6(-%tn`0rfIgt}eI7IvgCTUtM{FzbdMlS;#@(?H^D1Yu6Yz@l+NUn=Ow^w)`; z;pqR)JMg7HZnTRxyHlm)hp*hI1-u%!eMH@PYufLcQ4Xu7f{{dtfhLvg9ueK!A#z<0 zUU@pVkf-ZOoS{#B9-uYaRL&(D=lWz^W@v_qu_Oe6cE3+Lw0WXo;Yj3RND&!s@UHWA zu6K5`81NE#YkQ$Gq0OXT$Zu4<tetwVhJ$w~*ZH6HPPt*V3k}=!N?C)`V_*K0o<tQ& zHHtnyTK<~4t@MZc(04gbIV!(gO8cO8+G3@G&h8@G0bR)Pos=Z>h`y%XQ@oqK4YM<F ztPIV=ce__j1p>h7s?P(qBlBujj#h{}84P_=(YvAI*sT6BVkCNW@YGO--jHI)!Giqv zo&iVO97j@X`i5V=OkCO{?v!Xd>d@Er&Y3(=@Xo?%Ah*N7c{ttGBGqoofRmRv)HwY8 zD^LW;*aAU?`}{@|!Xn%`CAgKMbyL|38&%{wrA8Eh42I@x&n}s%=nUj#aIO;=(JXTA z;eCr<f#g>+An+qr&si)f{iaioQ5ttTy!&$~A$nN6Ls-w}_eyI~pX%X10pF-Sc0C27 zaGs<==~#Mip%uLH%jy1(ueK2E-mm$NlBB;Kz%<Cpb|X!aeC>G3{WOo3ay#%aTxKsr z&nfjjhkyrC;{yUL4s!8Rr*GW{0|z4-qep+9O?gR8f18>=GD5vp;3PKE$u_LN<UqUA zNT5EsuPc2-+-c27rthf9kb}tPw`><i`>f@1x}mmD%+gKjo7klwuE$;hxyt3AONc^N zSxz4?US84<4D;)Et{LX~VjTD5T>5BVw)llCqdJa&e&`r0_Bb#Vs>`gJe6L6haavu7 zN;|E7Dq?o=siT4R(N%*z-I~d%(QLDO99Ljh5B=30&p{@tnIpU>S3}kDe%AofL`zsL zx(SJ6;KbeeIn5BUpYZ&?Vk(t@Mldo)T>N^+q`6A&NP5^ne&IrvqPQ?KhsrU&&n8rG z;s?rXq#iP>SO|Ao#24x|A(ML*(X^uhKqJfS**%9srSM7y9&r${6E|{ol$GymCv+{6 zJQCXkWKBdp|FIt1@owqBW=h)sF`Q&x0he42U)NuSX`wxG%Sjez4^EImG;yAj1<7qe ziO;aJtEyyi7U5qZXvRPMgu}Kn797Rc{Y4T;#<ki*n&?N383|;CBui|=5;##0Az*5; zd3!|o;P0mCZ?=$l2<v@Gk;a<gXOYvWx3(DLcYAmo#hYDfdUy`Ikg|O~b7;t!<;GJE zfX5(daD7u4HeSC1s!qc`mTCCZ6QAX2BrEQW)F<ygaqmA|y_|QY!m0NkIiAG#Cb2}p z1=2OihCmin14WVH<i}^Fu)8gwx8_A8S-S1C15gG{flVzyu93y5jZ<dvUy)tA?F_|s zy#V^kj-lTc;TR6beP}nJwPO$tMNo!jZJ)ceICQeJK^PQFXV4y`Wwhj#0g-lXqBreV z87z)6I;@X&?JJURUfm5N9WcDtGF*m_Tu)*GpJ)kyqk)W59^%6@UL#Gj-#>eZX$F6M z_G0G?A`m8(Fd}t_YlQ;SUL?d0w;AeT88-JGVD6R`NnW`1@v|cn0U<~tFZRzex{m$c z4Y`KPl(~k!61bY|^M5K3?o&EoEQ_~&;5WWG02ll;fE9ik;E24GO9lz4lfVsjUlxXX zD|5~z$9qp*C-@4(MD%Ok79dA*FTU2r`N8Lb7l6MZeHciVqyc#vPuwKq1~w_UqC)+$ zT4aCwWc@A^po^}$&y|M(3|w`rflpdfIt=XV9t%}=lA2`fK_<vQ))2)D=AQ@uZ;Id{ z1v2_}TLZu6iFgF1d+Zs`rVYyL80W5qe4Fn`G?6S#q*M(V@NVu9oK&{+FvGLW^+U#p z#h#-;dAq%|P^YywTe3%}nU}wVsLD)qHXd_;#1#t;v7+D&fv$;($za;niCrfr;-FqB zejDD;>T9HGrnY%S$&RZQgxSt%t~$JVY0&u}5rft9MlMf$4q+u8yWoKf79GS+vefPe z?F-QO+=pv1quZKVtwQ%F_2ts6-`K)<7lhh6^a?fG`=?an=1I62>;s%F1p~`|WW67a zB74EK+v_Zel9(tXU|DmTI@Gg~ECn3R>K?+;Os0}F0t9u*8LY%Q-tMrPv+G;>1&wo- z{fTIrGq;!eiRBwxs4b5~zhZNh-l6vAXuV9Ys!#S0?*szr520`(6StOv2e66T%ZRbP zdO`2)ODkkV2g@nEGh!U^%oz5dW?m%s-7r@;P8!zJF2{NhqXDT7vtxft{@Rn?@c6+U zvyU<B3>s&zfBBTOB*f>&3xhPB-5c26krW?aQ4~3|MWymK2lMwsg7<^($KO{JeY;up zk(l%WifsSpc)@lr{YMgS*ab*y%at#(OL>`U#|Y=0K6So8vd;QKeu?b1#!vRG+3gns zUiIk?#+6vKrW&1UA8nazb9~T3$?X2ob7MJ1;vHo4>`@v0BBry9DLZ85?OjaS%&@!p zxIX8UNO2X$Z$(T?L$PyH+{k(2U#4b6MN8jcw#C7kvg~6F+7wPz_xf`2u<V4;Y<KZ6 z@qdQ!@zjZC!>n`!@aD}*{XSGI`bz+t@2MuJ;4u<Kur{j6Z4QGDm%{2EQ<9ANmTN^! zMG5962k~zZR!jS!j|^C3E;Pf6hxg$?S?c-Y!Eb0%ryQ7{(si)!s{E=8;l;s~?P;r9 zWL_liFuO8~lE-wVNL)OIapHn__9Kd1N7_NI#HJ$!A&?-iPTI8&*Za7oe@eP5Y*Mee zllYj5>Vjh})Z8r&x$|16S#0fhr)$~x<+)!17q3&yFhu8s=qL4h>Jw$0TJn76kkW|s zI2&n(4Nv_ShGwT`dq7Laj+&{9KX-yUla7Oh{I2M#Kmj}sbU50&&jjzH1+p__uaa2Y z_IhAM^SG{4mErfO_QaRy_A?o;6wT|3o$?lh&*+thr%EPCke&O|#RND&{AE3}=^77) zW~-MizD0F9MRnP}MZOD0@Hk+So0y9n_-FszJYfUCDIKoY!xavanrU^9eI<hrLTDmQ z;Z!S_TU58|KQK2(yOQ{++7``luZV88ZYehOPrw>X@a_UNPvu+rczGNLyIS;dZxj4@ z9xh+}%iPEW-iI5l6MqkfKKT(1SICxT2wJKzyXUp{%r+_p$wCd|2UApIRrgY|T-a>4 zn^;-BG^n5}8^wyn&V(uv*LsU&OWW-7EsxTIItpB-(jC6f$UQQ?u4l1@Pk?%|*gg+N z%^bAsz(z2gb*&P`&m&2?3XTPPE3<}Ls5(VLm6i*q6iPgVo@s`XLKwbw0w8)Pv=?FB zY<51?!@z)O;{s!?awcUFVHfq=&Q1J}{MRlZ{KEvTq#&bZns2cImr&$v+(Joh><Am| zY8|Itqi-9lyL7)sL_<Q2<DN1HeOr?XnA^Cwm|Z!*H92)seub@1i2>mEj>;~=ilqy7 z0fsPN7b9xRqUV85Udgb9OSm6iw4{C(G{Z(=C+Q%w2Hbs_8go1XI~Ed{3>4b(oFjIz zUWo6JthTd*z>7Pu&OXTooVmm|@DG=o+%U^lMIyiDHc9Jo96hkcvW3F&-Oab%qPVxd z^!T!Bsy~lTA<X~zM3TfFjokI~R*K)<JLUaoocn-8(c_QTpoXG<0T8k-HeLV~IuaC) zFo9w_Z!=-%4k0;6IGUeYf6S_**Q%{L4X16Qg;6enT)AEuXka(Vxx#@V$NNd~M93$) zfKnNq8e*``R{Fep_A3}jBKUwC$pjSG3$`rsUvwx3%7gKs229u2N95u2c`mc-F_MJi z&$l*HtiyN!Vn^NZHZ|<rlqhmi^}9p`3l5qYr(v^>uhQa`^|=||G-av2T8Q%j{|e1o zR2Ra(8g~b2qB5r-7km!>h#_n~;|jom<L*B;ih(M#zjw=KZkme!8zDT_{o+<U=|)x3 zQUp<0QbQM7v$HEl?-4P<FZ-Q{9A$`)GvLLB-o{z;dc?hPyiG7+aAbMqQfQlBaCPNv zXW5AY!}msVgl+sTEIm5$P6c+sXBkp8=k@G<W`-QKP9JXLc~d<n8C5btw+xN~|M|HQ z5qT{0L%2fpim-#PlWoCu50JScr<X7zLP#$R5#ZbEw{*VJzaYOgEjE(~pee+9m>QJr z*sO|CEb4@UUCa52VD|+48dsEjkxRyV5F^i^&!mb6ZM|>=&)?}5V|Qr0xB<|-o^%Q~ zEoM&T+OjzUZ&r%e)P$GjMkjAd>5&nS555ghB@JRFq8^-qbmmhIA&H+_X*^ca*acpb zwPH&+ej=0a#ba=479R}OQ+BfetuZ<p_}~%271{^v_<N)n#*&+m2k5qp9FoHIlG4VI zpy!~H{Yh$7QW%Q(?>N|%MRWr~v5UATSQ3ana>LjE@VfxGVJMI)s9}bcW;mfkoLal` zDVb1^<B0JK-TsG>MfgvLdcmvagmX@tTTTyk{Us2@=nY3KENY|(YqWeu4r-|=EyE9) zu?vR6zENz*n9HCj6!E?PoY7H5=8yzI*U1`#*;Woj_*%h2J21WK1`ViA_FPA}gqs4O z<L`cR>{#>p)<sLQ$N{gZXsbs8OpBT@tf^TR16%iMiX^U2Qe_7%6j6{5olB8rx>7jE z7>hUaCmNg}+-yj`A*py6B_iVmSdGiNGxW~qy(JK<?f+OQLdk@M=83<;j%!bpENIcc zfFf&pZ%SgZ^I{}+<}Pa>_V#|wpM3Euu2uGb!b;N2(+97;=*ws;d9|2<@g{f<-EV1! zrn-y#1#&fVRE{<l8QuqkNPX)ULJx+7VXqd4xL2zlpcE?Re%GkeqqZcTcJK-C+kfvG zA^r&;NFQORiX|t)F>|_x+z7W|ZzZ8(f|o!5i^;oIGMgR;JN|7`f?tI>QMP-b0WWgY zJ#_M|-=qQ}ge>V4NpFk!Q_}aX?4j81i=+-<=B*leV?tsR7)pjfsD1CptrMsQqr8d) z&u?sQ-(37ln<SBHw2^fx&;JH-UGXLwc-4yjh(ImOYki>`8W$Lv6K@!P1=>V4DN_Kz z<6qe|*-su=4LbYm*Lu&jFT!X(+BC_HY4R7b-%$Po6W|KsEy7A*u7_*k_3kSNb&eft z@u5d0q(E1O>W8}Tb=8d+xDdE+Jay{LQZ9>h%hlMGL=)))xskquL|IGMZNviJeIg)H zvVj93QSNjwSNEE4tLD2N0EueE#7~8!`;fCQk~%tA8E<?p2g_w#1lnZz0*mK!Du4t? zevda!78ClLKx9y1mB%Kmgc`mc;3RHUG0~8rtOeHXf8#;g!FZ)G;*0yy$?gP|Sr629 zqvhpH)}h$^3w~;^6(zo_x`kq!)|B?=!)2AYqt=5};#X4V*(9LAz8Qf1Y>#)v-!+8f zh~z(&eMeD$z5~gBtQt+wsY%fBz`2M7E-RP3tw1uX=eav3-LQcv)B(iryiry+L}6SM znd%I@R6_Jd1e3s!zBnihGk1%8UZq^KQ;y4K`^dmVD<U}f4m4jxSuYfONd&hqELRrV zN(80)yXiJntv4<$JIf;7(emI$N>W8+iY0mHplTEWKI618FbU$q;#M7DEVM_mz~rw! zbbhV|Jc(8{GG?KO0sq7n1QE22{}4!mb(oqL9L7Mp4k^BRzZxcXrgXHn*QC8E^vz1P zs-i7smu6AUf`9amttr9}b<!;o)#q@;A-0YbOQM|gKwfH)`-Hf8JMa=BkWA&`wG0KU z%pp>o<2FS%2+~WigImSC##rU(J#z42*(oW)lL%f%+}2ZmjmsI#pm*{YjVE2zM~eaU zZ@~t2uEZ{%6cdnkA_3is;iq1V@f-Obr5)b+dx~(M@*Pl!t+cRYGPNv7M90*^d3X&B zuek@1gX7}-y3U=DuM3zArL<^dd|`nwZNw-Uktmhnm(6~CSHc-Pa4I8?VFH_RN(=zF zd3oYiPNrN*t=7*XNkqD{Ry}ItH>bXZC)z1hQD^uWneXd>#eW+PnPOZ))d>u~v+waT zR0$|)OpADBzrD==@#8-<H^85Km=iF3V{f?F;$NKX%I)2vL@YId)c4Tkl~pzy#9|I6 z#5546j>{vX4BSYzvbhW6`^}Jj*_@S{%1IrZoq2v9c9TTtV<fkfI)0C^j)#OvdA%PW z87`!lO3>0_r5(#d@M9eK^YIulFc7hXR_<bq&LE^*@$9)7J@!1Fp;LIPdQY>wz63*v zw^vr!`}FR`$|_5ya?iA`Y#jD(1l7~P)!nx4IfSHKccG?LmP?qHF*DsuMDNRbh4+;v zFqD}SiuS1&{hk~^ZCSqv2oG8o??{B8Pgw0tFt}Qv0$#$-Ys&hG7}z^n;XqbWJp?4) zw23@Lkl4KrwA;~2+lLg!RFTyr)!6vP?Y$f?{-B90)PD_>Wc9q-fqp@i1(s7gfA0W+ z+^$D}h4KqbyBJy4|9Km0^ro}fI9Bsb6VvY%#eY|D$^j(Ue8RH>V?{SOwdw%xXN1(l zg@8a520aA$9{GnnnH(ozpt1m7+v6zV7LR$c@o6Wpm$Dcs$1q!&JtWO$^3v4I>S)Y= zWgOr5r1gZ8O)ye0o48jRR+S|IwwJU<V&(1w0C&0-&(*DP4p9~}6ZOZq`f>F~WAI2? z5pr=;oW;Ef$NqI6rKxhDONojBFJB(2BytGSd<480{hr>0OC&AHXLdd^9=a8f_~3!% zyLXi?Df4`%D*5yPU_|Q$nBcc~j=|i0Bng>0|J@}m1VlCY{PE!wah(BjZG}*RAE70L zv*0{LPtPjG8{t7puO&5qAo0a?6|OR*yIS%sax#*rZbAIn5f|e{lvParcD+hRbdQ^D z7JgLC|L`3SNaG#bMH2GHqeVB@;G!S|NFlR)b83}YxiB58H@fQbrKOJin&)rY`s>%A zTf+NX>6mt^%!wKk5eqQ{G!4xUtoyzBk?u$HV!=tPq04e3htPd8Td{5s=PooJEIu+Z z9(<0(TvY$|Wzups{Z~b)zf86a$g~_k#ummJsgfr0-$GpTHpb3C^B_5NJjbtyp>z&Z z$h4j-?x4pQ*O7;D2iFehJ2CveJg_h(-jd2kE}5#MwVF9tiUnmQX(0p?CS^?zv0GH( zcb{XC<XCA(Tdh5dvBj~79|bGBZ~yt2$x|X<SxSS#{s+Df=4%~GaeO893crwOqS2_X z1}$c>4i3g>)`jOdc*fudMJ_@uyLku_HKqcCT1ltg)ZWepOJINA3D1#R@g3*7=crr@ zYL}{B!F9x9D`&IZ_wL=+E;)qqjr%~lfn-KS+TmE1ehrdf*-iT^36Kd5CwbO9+-#?t z5~Qi0YUlWM20<MUWP=th2H3b%=u^<Hb;*Z=2<^MJiJlv3_HBNi{xSKmYRsKoHJ_@Y z5<fm$WFg&tN;Lc|rRUJc-uNm3e;E!hEiGSG){wqar#AdutwIyvA*q9UzY)9(3<hxB z(Q21RH0FOP@(q@Br)fU_{N_^QyWdKbo{pXymel^p)*BN!k9m+BNGEhhWQ(xNzCR<Y z*PPTOB9PE9(#khYml}(AzLx*zh;x?NU=~!BA}$vjDB_CgRekPDuc(|<A(8<8{UOXL zd~l!PrK{NaS5+PmdByp-YlIV6Ajp0>tuO8Ohl@HJ-Zu|EuPKcDkXEW&y`p5x$z!hm z+|I(+mL8F_`#oa$$IG$KBI8P*;3L`Zi(WYVokxPVVHanT4(vsJ0_~2oGm2xuKrkXh zbIxz1;M`j6?=%4`b1Pb#iAv}MN4pP_9oel%c`%N@L&_EAHKG3yNCF?$3wtb#<dquw z$#l3P+0Nb$m}at6l}kCjR#bJuYJ{dpf^JGlC}BoG1_xIf6c9J=L}V8G^<6y<HP&mH zLtSQ)Zy-nV+=#RNW9egNFVm>~d|W>3v6TDI&9;9^hQHSe51ws29QEghQNfk^;^I@W zkhhg2R}ekg@nl~H`q-Wr9seu$ixe^{3rnSRBvSXyOARJOJl+%wfH)a-Z6XI4uQyLf zRdbs0!qdDg|6B1c1LY6UnhL?Nu^&{Cn?)silf;O?Z0ytYnHjbOI%VheHJUfoMh#|V z>h#v<N9=xVNnT1m#4~ms`Y!c-kf^2Tl`y(R)g@>RvuI)s6J_czd4iYE&(gBg#(W`% z?@iC@EyFuRoEoqcU&P^F5$!5#!n*JerrLnpx#u;pD0h4fjP{+I8`*38o!XP#t{Auk zqy*k{;1Xf8cx~pqZcg)?Ca&`dnEldsK;vjA3{Q{`O6kU;a8V7r>S=S1B9d5GJVTJF znOoI@Eu{TX0ISsU(F=d)$VE~WsOayKx(ckiupE|JG-sR!aK~jlPIkMt$9C1ns=Q2{ zK9|c0ExCSLC2w$GE9O!bF%bzDAFqr!6BZx|-F1I!)M!b_*nwh&Rrpvtq}-if)Y{tg zrfHoN>F^Xo^4NiAWgaZnsfU_^h`d}FR$)M%L^&}(QaE5?;W^f<ibt*NzjOjO&+d8V zS!_9!xJ@`fhRcktD|Q>25{=a%LO%m@VBd$^j+@bPY~V?9h9CjIe5$J{#YW24;$n@m z2F@wwY38-O-u~%WOvG;CqWfD}-d?0aojO3L=W;?D+dPYMZ>eadt8bbNn}x0a?t>{O zkStTZFdh{OX+Pq@z8(t4B1`0VfhrVJZkgmHOK7omKV@80WxahxQkOzon+2^B`OV&N zgLLYv70NqC-W7cNr1bj7k5<F5B=X-4a~4Gy@n{e7Z=zrsm;$@4Z<X3XR~V0_WrD7< zab~hgBAc!h7pf#^-ZI=KWqY+L@v7#7#SHh@uUn|3_Z^pi^D;|C_r+l9EpHd`Z}I6x zv#q~(vnCy$V$4MYscQdN8EKC`mJjYmlu_AB2USWZbN7;c0f{HI<S*y1bl&txE}>|n zZ<XUSOR{3thqX-MD`uj=he4$xDq<_JP*Mz3qh?n2n4S*P(y{~GGL`pV-|kHaDFDHp zoYPWnt;E6YP1oxxx?XqF^_o*zglrMU`~Rias#25_SH8rae$%AAxNz^1(B{g*tA)4D z-wz|MhZyfup~xpXMUhXRl_7{Z$SC4v`~|T{xU)4sK*u=$VTBm)DVsJH(D?Gb2RVGM zlZqJ#aP@N7T(PRb$ml52i^L6a+y`iJ0oOj?s+a^Ve5%G2t1Dl+E`mfNmnyq=8QJP> zc;r9tndyHx;I@qtfGhML0PbRky?%dx&UnBb|1236rH!usQM!@Y%}pUX@3u*8UbxM> z#rPo4YE?$c9@~`*UD~AY>Uh#JDJfgsFDxopXS!pX_%@E`uk9SZgojSmQPa-_9oGqt zT{BsFlo7_8OZF9%kj(~6kL>KQi?B1UyigY37>>B2z2h`p)eqNnPE6BPrqXn{d@ps) z%CLVh1LH+K%ciu~ugj218bJf)`+UwFuKs2x7KB9OlDzr7`+;Bbo9=JBZ4MPOuHUG7 z^&%O%y^TZ1%D(hBPT{K=bD$Zc+q+7`8O7=sOyDYo1GVuZi%?$XWiXr{)VQPRTSxZM z@7)s4HhrFNn(S3`)^E1_^*reNXFvI~zpaQMVd*m>NU(-d|5u@AqQFQ<?ZuspuQ}A= zNO0P>mu+*OMOhA@Qu?}U2RhB#j3mT0X{FfA0s_fj=HK7x>VNo#nYPUE`3SpH+0RZi z+&zwf@Xa-<V2i9FT=fcSY~!2_isd#GZjy?~;p_EA+X~N=CqC%8NPW|_t7p6LYyT?? z<`)5sq&hi1$BsVpK{CTkSH1=Efd@`>M}Da2EPrwKM~7xdQT)IQvx39uQ`f4NTqIw} zutj`A4&oCIlTSA;JR0?U^4-RQPf5|UyQ4H^nTnZ86E>cfVW>y?CVBy3ST@a#G=BAW z6EB@$4}D4U%=%Sse*1fOvAO{2CA7CZl8rs5<>RmOVYBf=(rU*4NW(BIP!N?{{Vq3) z$GnoacU766l(X7Z!$lz>wf1BO%?sx=hn_Y?hRwSJs4+IMMfuXfykH5*z3;r*3+WOu z<&Z%+-zv6>Nb5E(v-KxsDV@mVM07zBR`*_j+$tdYcp+i+MM{CnC%RLf-)_|dahQ~R za#PIuv@RKe2+o5<tfTpxFl}EABG&nl^;&3iwQN}U`Idy5TKnuA9iKmZXIW2ct|kSl zF*GIlE<gM%{mGq*lL~>pq`{8%vi9Jo_+-h8#EchnehoWWc%?;i9IHu*-n1()9PzfT z+O1gqjR)dRi;D*I_q|cXD5iW;c{eQ<1?V0LJuD$*vigy~+uW=tEN>G+Y6hS4e=qwk za4zD=tBcgNZ<^}Js=s!fY+D`b0UF7CGCDTwV-`6`ze0C;e0rv+=erYd{--xaqS_*3 zZF1Tf2A=r~>iP>-B{y*wcu9GQH_xXvi74SLZ`%{T%6ON(??1P1uzO#tTaC)I_NX6c zt;ok})D6>BR+JRThsiWKyt9fk;&0s5;Mme+6#t4^?5!K~tS9o{@mQz21T6CI`TNF% zM^;ke<9Xw@G|T{;(QK1kqpX}8**fpOoUHVZK!tEfKj<QLjAk6a9-cr94qU!)b0jeO zR}atI>ZBeHh3s<<xw#|kDXGHmOPSQ<jIvf;p1a{@gYTsk<>Y69^6}CZ_Sw#wVMB$6 z5+kwM|4E*Tt#>Qna?37?7P;U5%+rRoj)(Q<N;#hBLoJEV7rtxbjpJ7}qS&RHjsEj= z6X<=fHkEp}Li$&MPLo4sO8dfF7h1WLYMK|HOkD%Q;(E+ggbm@jA`$MVS{q^rP<1(l zTR~6CK@9Qb;j!YE69gaf!?Y8*`J~I2mNua*fVsv?!OWaWftzOcKm=JxE_<yl;dqR# zd3}$^^X68evJGkj%}?t#RDSLbl+y8Eb-^e%SvcVMb4N)TMioPQwye68{eC84y4EX# z>(*38hmFS8XKg)@{jcP*uQuxT58L8LHPT-lTW+Hxn-}m_v@*D;!8EbXc_>x=XPw;2 z+;RgMPc}s%hZV~@`jv+7o1GKC6{&qRy0Y_(Q}v4lvi3ciy_Yi9x(9F>AA0`s{I~4i zyO#Iu2tVc+8qb)si9NcqRWG)#w5PY}cEz%2?Iwo@ua7-?-rOVP=%^^xG#Icw^^SUi zebd8$27z4D0f~cfPQ`-T3mN@$O%2XR?OC4$dNF%;6fu;dx}TDg@RaqLZF_ShS45j; z1E(V08p8!GR12;ZsvE+kdtR%84SAp6?s|OGI9K#qmJ@q<o<MP2q;lBEUf$r5PbIOH zm+Q~Vb|+o1NnE26Vz8xni$i|!+3b^dnO8duopkp-pK>x+uk4M!L0EyB-m|D@o%{O^ zmk1O``^tA7$bEJqp}uhF#DaU;6`EV~yi_W-7e_N^nU0<s+|Xkbf8mI=isL=%Ml*3w z_b1UwaXaI76rU(FQ#LzLsbZGzl|4|^>lbBEp^2=qbwTW+stfz9l<sE=M7xz6>=Z6+ zwF~X+xL2WDbV4$%&T3`$%Iv+Z8+z8LE~3~CTcnp{|8@X#i@IJK*u6t+bK}GFk2R^( zhf0N|1=}6JFCX@P8qInpuSlT0b61)!+^>H;Yb!+0-Y-jwumRNt$P34i_9hN>8`pU( zJd*u>zII>9J8dcR)I{xYZic;!LbK<c&JK_&6xSg4dpdH9ec1CNP3TbI()S&FTgAi^ zjePy7t97c)2DWyqZ#A<h+{b29m{p_eZJ}rWwXTX97T|PHOxs06Hd5@=?}yDk!T~?G zc-gkR?X`{l@Niv(d2Yx+*7EhqCm+Q7<_Mg5(5&|Kgzji?T28OwMqbTf^?;|X&t!AG zRALG`FZ}12tbH<1tujeCpn=WAr{%6;h55#Sr-tE*=~<?{YjY#5H@4dj?ma5oqaLqZ zKBCms<)yMHsP|)=RPh!a*^dUB9a`2VoV*`gC~D%B1Gdq?W#|*9(sRJ^UU1<HM3c*} zy$n#}?X58S$Tt!f(f{CQ$5Anho^`70DBV4499n<%J^X6EBwD1kv__(JdEw?)Jsm}Q zR;-v>1b1$i`6cR@s!+M#(O1}_-5|63so{}*ptM_iA4;N`IAu0O-18V;;jPz;q;-rj zkDJQ`Gn+?}ZD>K2$-6MKt`nykpNq;QeBQWici;yGIjgSW@4w~<oQ-(hp3(NFA=%=C zpLp||eveH&$|`ITF>KKlyhm>*=^n0dBC|hOB^K=BaVAOd|55gyVNGpY*yt8Px~QnA zR4Z6$QX(}dA{G!76p$_g(vjX{P{2Y_QK}FWsnT1h2}Oxgl>h+(35b*sLQO~@B)KcH z_3VAl_dWOCU-Fn_ty#t#bCmZT!^4n~S!|@fvtlZ^)2lzXe_@T}UKK|7NfhgKb1pIh z%RN}Xk_jdph^0J%{mdjKnj#SW<1A;=D~p=Ort8B^PQ;XHp@tHdhDSS1us1EyYd6=A zB|I`st``(%O%j*=-fA#IL6TF;sT)Tv)7@Dg*2j`yKmFZaYZr_Kw-xD~?My=XH-u}Q z!eUB`bZsmr&whQ6nUW6Pyhk+uN$@Lz-MQm=yAu20jfd`B-jKY@+u++@m7#a}Kdgd7 zae1Xj0ja$sTq;b$ET1E#DL$F?#u%b5jJVFx{m$I$M3bf3lII&g)YSvUBAyLqu$#wz z2yJHm5#>LIO}pF+!5owVh1_}}rS2YPh<Njjc%}x^{2auyVBqcWm>FLlTsd~H^|<-b zl;7cmuHv~rZVjlTmAurs{p*cWf%t$GqiY{x5?s^;D}|1l1hWsVx`4^9yryE3&?vTZ zG6=U4&k}iEFP=E;P{xO{!IkeiUM~(Fo)QPii-B$(^xf28J`X$NoX~4-hIuCva)y|Q zxW2yehSiiXITPiOm+!Eg5|4VI9*tIPvIs|0Pje8{ucR8;(KrZhH`c;PRLupr6KN)# zUSBXu<tLpQIADaj<7%ir-B+F@YlJ2N<naK(Nk?`J&aJoLf5Qqoc#dJ6kkN<m57h1< z|EMB)Y>BDUUem1nZIno#g2DY<6+$P`PJL}1${EqaG06pu>@!i?M%HNzU-z8A*Banf zBy!=zea@PpML7;9LJe|OB3v^A#XeCQ#5Kl%Lru>19FE`=V=?YqBmwE^x5&?PLe&IA zDRKBRy^!2=yA<z`vzU0(Zj7qmlxpef4fPw{iOBitaX2)S63V1luCA?ChmV;qY<id} z;-2fQX~v?+@#}Tdg}4;l^DM__Y}{dg<_-D(wS3?4-rJ6T&~^c>J4qTc9!&G)PI&r? z$9=Cc{_lt*J4}16@6bR|y3X1=Cs!o*&nvBE={mUb$y!2gPNOSFJgbnBzt(@HdEGJ( zCueJkURJwo7e2GPO%I>^rj+&Ed=da1b=zis*-`)89P^<6Bv(fz(k{ItBXDw-i#`j4 zL2-tcf)4`TVduANbN`zcDmxB6@XF@z3;vF+alM{m#mnlcN_$HG*CaY%Z7Xb<ya$H0 zJyM!wfHDL6?_AUWY0Boqb@+;4u8$v2=3!oV^L6IJKcf7(oU<c)FLC(uWy4`dNhb<M z*ZWHDHqLU4MG}(bijX%Lgi4~VZUw1!qwiXUUM_DeyN%rZ&w#A~o{;yZEltE<JtHR< zV37VFhu4@3I-lO)kp8$2g{_>42M_Y)p|pAz5I4_-H(J|y>Fzn_hxzV3+jF$ea&-sA zYJb!}$0_wKPe-80i$a>i72%`<K8xUGVkLA(NXSuRC%*HmXQ`R>UMF6N7$m$STlcb# z$0f$udUo*V@#lJ-aATeW77+E$eT$NkWe3>1T*O`f$$TCte5kWwtj3KquK{oRG&g@x zRX7hYHF<wvRsQ&jz#pzMFK{I|BRj1f30wojz%n3Of?p2Hs*X^<V~`cbF>DtSDx+0> zLjJ8>N~WKHG&}o#C7iw$wh{*Ny|7)4V}I<=SW_v{hw&Q1krF3^dqr(6wMB1V*Tmh4 z_=ofBxaZtj^HcA?!;xLZYk%}d-hah(1$3pj28~p3DWbYW(QwWIP~S<arajq*3L6m~ zx}2qzg4=x+4AFjEP%v0H><#n_c;5bn6Sy{{pLk)m?4X;#nk2r0+x|WTKu@xJx&%d< zW>eWNC%IvjLhBJv7{`|B$kuXoPV?D+`^hlNigRpWR{np0S-<c83%g89dv8d^+Q1Nl zeIfiG*eew75nTNM9G=UI*2YO(7cnTMJ}`Mb$e$~cBi+Bnmx={UwD8!WfIrztg`0o} zqP5Uv*{6S1d#Sy5D-<I7D>k*!T&SvW1scaO_d8e%fZM6ZKHkUwOL~lzww<TA`qwIi zRBxEL@wn@akD{I1YOZP6uCP;39je8;cBa4@WWTE3intrIYpZbt#uxs{wvvxYy}${Z z*o$<m!E8^+xSJK2O9Bkba65~w`fI-@-F`|cU;FmLyLp2H;WDwR7qa$nIBPHanb-O_ zr=(@_QAgKbI;DHoQ}%V+zlAgOl5fAyICMiCc-tH~C+XKipqaqlgyvRA{gytR7S6j_ z!KC@&c#tQ*ibzpnNKwz<KJ{Rq33j@zM$4=k0Br#Ym*2tRVgx9Erb6alkPfIKKTl!8 z!A8^=Z#I?3QISJs&y$wbx=y*BO8k|8Er4n>8r?)V5%FPEiaj2xWlx#EY929XF+n~g zF6?-}&@SX)$QAKl*J1MuV2CjSuKy_$=CJz<!ck+vR3`ik`6%|vuaL^{hTfwC*I~nT zrww}jnOFK=-fE8K1)7oSgg>r5I^q{bWPBIW93I*DP8IixL%j^gED}<#3)5{vgAb7o zEC7U|E#}56okrXzqx$Kh;v&_&1g)i36`B3>kg<K;EAx;Njb9PH%c#Jb>x7`*B+2z` z=?cAWLKF8260ty^-&}FqFW))V^MTKKe~qClX6>5YJr2TVPiF7%>JX;%rGLw1(3yPg zn>ay`TrA>nLMo@L>@Hmb+PBg^g8JDoMv}Dw<`V)#7U$C18Ts%=x_`IN1o@8Si759* zfSA;vm-EXi008qqiB+N-@sHF29ds^BmN@T*Cy^YG-o~G7Zsry9ZGP&ef^be%5;|R; z4z@>YH`!*_{&b0#I`HxJ^0tig-*%G_)x)&u&BlcB(9CCNIX>?)*9w(`ADKzMoi-%| z4ysVoTpKs(o*3yE30a2$7S~FPNjKpNBe2_l9ctaua)t8c9yL8OXZX&&N4YK$s@`cG zk`c@xBeb44Vtzow=VL&(TfkJZZG#egbIQ8IR|J>k?HZvPxG=xADHRHz;==nL+_!e} zMG5!tDma6hsNEaRn5)!{X{^oKcyV4SUHDS(!<1t>;~dZfS8d%I1sOk*i;=YPObzfJ zCdL}2o<#p)xfY%78Ln43Kdf)`n9y!H07uUD#XB18N@0`2kM)zbK-T|eq_KnlTD<q# zIvyN{PG!zstuwAcQv6RglzYGN6xwtBY@_gYNg#Q7d3OIp?Z0!BQLCpdqn_>2gjHF> z|LXlfyoF&H85+H*CpfN0JzXP)YXtGd!;?QRzflt9?v@O`uXUNZwAxn1Y?HJS$^vh^ zLl=B0JN%7Qf0Tx(Zad1QSLx>bDBC51??o;_`K@1;@<8@nn`aC4N_7JklTkn*Y@RZl zf@JtGr+xP|DO-apFwgxmysc;FTN+Dk2}52i;xXo)zWVg6keTJmHz)1M#rYO_JRD;S z@V)Om8T}KRBkQA|pRXi`<T^XI_Rnun-A}%b_&$&vLt|Lm$Y`ldcL&-^u89u`|FA^& zwu6fJ^P~^8@dD13(Jp++)}i|V++)(my^w!|tcmZQZU>+ML_ww^tzF00y{d`+4N0eK zntw;!jyoW|u)endvJZ5xX4eiulV%0WOzEU>;=K?J$C|Lc=htXJB0#IuEaWtCO{&Ur z90@NDustQJEPRpV>9`Zx&AAe3`yitUGX?RYR+vpACuokaa^_4Y2x}V<VG%iPA-%1Y zlG#=NfoV9=0+@It`&QHaF>QSNbH|<OHIqTAJZaN;I``AA>oOYTz86E*KOD^&ZS!Pk z6s*GC(+DAnO^6S~t2cTQ-N7O`Hrca_Yc;u(=ji^<=C)p!Zm{e<O^lmBhW2FK+cPQ_ z&<A+0RChOhICVb!Lsgy)^Jee-@m2fjA9d5G<bw(BtA>P14UyK(1qux6g<Q+5s^gEw zG_I{IB?Tf&EacusY)gpb1p;$MJh-iTW6j6dw_t(S-<L)6G0vMW9_&5bcBLw>gWk2H z&%v@ut)W9J!XA6~TsUA;c{4|Ex=sHbIdL1g?9bUpQ4^I}l)kU+8s}~oegIg{=2A<Y z@Wmff6M}G(<eebTe(knVX07DoT<&PXwh0x?g1hR>RIgB|hb3R5Q*tlDMXlS&%j!DX zMTW*^c#-8_^vqN_-RQ{+GE9KYr_Y!?^!V6Pz1P%$WZ35Jt@tMIFNd`2ZKt@oO+-Tv z*2D;4lV>VRzIRJ=oyB(l2vSE>PFZT<DzOAA4ZMMv95z(W)2BT5sM9bxcNXf2n)woN z?uVerRDGgV<xH?T5+0&W4P_=Vh-N+pF@`G!@#SG3lU&mE4tc8NXO1Y~gG{_{X}8Ya zB7B{zau}sr79Ir+Cyg$DGwkMbC)bco+vE|BUbTclGi7O5IQcO=SNcHPQ}z<a;^P0y zaRRb}_a4QGavgGL_QsZwx~=&Og+6lcz{l%sj7Q(XURX<6;^&*1otUSb8wklJ`rL7P zVb7P4_bq7T_NfH)$<55US4mf<TtdFt)3qmSZe~<WExQ?qDVXOy++Z(n)|^wzYCou; zv`_Ix2b%mLT1YbDR3Jv5Mf*Cs;vn=hc}A|to6Iq@$}**`S67dQ2h!0$>i84hmC0;d zR8+3cvyMG=ABXNi)f$gQK63aV{<Wl{1yz-}_n`sWWk}+Qf#U=79PfLf4IAQ%RNCVH zu*E=R;vEE_8{Tt%ke<3(&B!HQT&o8(=PflBT?<kPs18YL0qn<e^2RC34fPY9{bmf# z0TcR4w6D@SloBO(e$BHpL7gTSn~4eSXg{t{K$o3#wS_Yb42`CG($X?}WQ9!DJB>Cr zU}nQ0Hf=A^E^zkv>&!vO>KFzzI4nGvWF{&f{1q9H>nO+@3)FuN?{BU7@3QU!XI==Z z=NeBav_4k!JCU#LE#eWpT`{aNV9gzcE$4ax@3N@dS2;l*YOZL~(y~T&;M9hRB!$wp zVAYemMrCC*pV^-2Si9xw>*`kMe98#@Xs3fEggC=IyEz!DozzgU=6zv-<G762z;=XZ z%PLJecGCGR%80lA{Jv{)FPbjpO#3yQa4fY>2SOuM^)Rx0X16u@J6I#bGGe|Zv2H56 zUi;_5nY=Q&^Q&)jjjHZ{8@DUm{Ez@MP3VpuLnP&Wvkzdn^C))9n;cLa7%mIp@YjGm zA9+Nmf7KNKffz&lJg$X_D1x)}Uacw6k0E#FI7rnfOK>~R((SnhxEIHIWc~OnsC&X} zZL9^I&2H<8Ie{NEJP_5Go~fub5T3QPyV<hZ9q}Df8U2ACIy_ZDb?e&%w&jPm-N?+P zwzX0&QfD>9dR9iV&vx!9lzhM-0mdfng^P&Bqv-Zt!BnEERl5Pl{{XoEHV3V_Zhkw# z<#zMM3By3^^3GVl?Mh?g2I!gVnJ!I=@Xo@+a;NighR(H9!S5X2ktYtB*}Ji$(Q^ff z;>bn?AO+ktKROV!+Ie9jt#*Z+>Xh-r-~A(|tX0L8XWv!KgrD{Gbc3nHkmlEO<!|ga zM&1k>GBOa+jzBQuQX78bGjb=A@f=e9{YZ6Ma^y@XI8^IC0=aA_7g18@Y36G_J()NB z7i+i73!SyJq(apK;%e5mhkzM&#qj>l&Rv6Z`DIU`oEusZBf8Nfjh-Fa=te)y9D?#* zPD22FTO+4Xv>HfMNtsU@CSI?{)glaPY(s`a>E-Dh&q=nn?Fa8Byra<ePtX{~FB9JF zH8ny6Pb{31-+^4!9!u)A46t%+ot=DDf{XZ}a?D!eF0s${o?1YVGGw>xnAh-`pv;8^ zzXn)x$laS@P%b0ce~4L~-6TjLMu5Gp84)4NvY~B;aKsVALwi!#)lEOmlKzWi(62AW zWnJ;ENqMaF;#&0mv<1qJ`-KTvA;^_B3mYi_mFkbZx_kTF@|``U4*mk~@`#>rZj&aP z1fri<Ua3|=H^I$L^$T-}J5vC;&JQ^#!7yf8#-KND9k5xoAZ}Z2f`waDs?oKbwJz7Q ze8D+<-RK>^{U^5NZYA|i;-b6I`*`&)IA-$QXJY2azlwSz-N&_xak5{u?=M*&(Bj4T z4n=6g`Kw@9(iCYX_>AO$e6b&YCY|4x)O2Hhx%hFP$#dM6J>FwGczsoXH<n$&)2eT6 zD;LRk?3c4W@^5FGeELOe!0-#}w1vRX;bw<4iCf<k2W*X-@R@U7!B8@@@l1xiLcyd= z)HjFXAJum&jT$|3z6I1rd(>QiIg>EAVfj<lsNPebxs*8Z%}6}mPbIp7MV>~Tijj=w za0HO-zkMB`S1eBzx2?%gwbrQm{jzE~C`EvnB#nKv2mm}Raq;_5N54fds}>bAu8|xc zG%`kAVW8RIK}4~AY~FYsQUN5;eb;jmAUYh(zYCCH(K<xS)Mhpw{1M-sEga8lHstzo zMsCNim)f;*oUF}=oF<S;T<Y<2IWL@4t1MoRJGhntyQ+l^Uw2Por;k%?%z2cOi;%K# zdfN1^T4TF@FC#CZchnVCvO7<c45osA>h*u7uoa=WC46}Tn=s0{2sqt|6Pt6cK|dem z?QqR)-p~n)vQ(2eO0u8#+&15Os}aSwPxc(np}IkCTE5Far{Ht!d0bNF!ynjX{K^6* z=ZY?e1G|HtMxFRCZy)*<NX!}`g!;GeEqe{6xXMkwHO{rHMlccqX@vMF#!7%c=D`-s z9{^cd4ubfR6E&w4VjqSe50lL?9t}01k7C;i7iKL<TK_|o$lm+J{%7^nfH#?e5`Fhm zxZ9eyDB)(94|ZX{X6YTfP@mSe;Z+rk^<ep7h<(YUwu`!U2i}o6*QXmwh3Gry*&H{( zUro`z?WwT^w)xBk^IQVwZ<R*{sLp)1`7?#|wSxHyXF~Zt%rwY|XXpszm%a^Zbl@I~ z*mie{pwvAT5-c2IrO+g3&C~P<^B=PPmj7ye6M7UvW7|%y=lXa`B<YjV>BT{0-8^pB zI8c$$Cpe)aawdI`pd-}XThl|C>kHcN(1Kgz8}-FXfaTYC*O60o-?@$EWy55GGvFpi zRGrZ<0RFXi9ob4!$^dNDkSpp}ResqUK&t(ujBF5&dt{f0A=6)|5Tz5cMw)$DXQ>nP z#M;h;GmT@|!P|pim4dhzXl|LxOW3I0eT(lM%F#*Q8ghE$QfK;~{MqkhyG(;lJv?}x zIce|jlC%3Z<do)S3kCUDrc65%Cmy)eX!%#DX0bTah7%Rz<@y|u)C25R&AYUSTLSQk zogt~3G)-qi4_swmNbLMz=NN|e0+P!S5AM>=MmWK}+;*bLU9B5_mERzi`!6$YC1-`G z3cMZX>`Qz+agOsxut@3CkJp@A>rmRmG?M{m%Z7|hOCGccNx%fV|BfT+F!|iV2Ijru ze}hB&t+E)*>7$Shzg0n^5ZKRk7E1Zl2%xN8svOMUfNmO{3d-cbe*K=_sIgOCn0>|+ zjsR=ByJh6eug#MI$e|%GL-ohlM?R~BgK!DLm&TuIM!bBPVOgD3uM<)ok3L&{0#GCw zla#a88dxl_&%<SFqXkYn;q|*UVVU?rp4{Fjj|S@F;S7xt5tTo=w%@BORrXYeuGLdJ z+a?)QQP-}potSQFj~)i%Ilb0S{Vjnlt5yNOVu8jc(3jk3H0Zbrg_4`2=pQzgP^qYh z1-dfq+e1qff|O;ez$k?nzdu38OVTe|6C-D`mE_uz(7ol30);5JG0bbz63Dr8Q`q6r zt?<^s4bMnG?JS460H)M(Tup-qW^H<Y+V+uo@LJ^L0Cp492OuoAr1{UNzoqNcd!Zwc z^q1}F1Cx1|p%mPm%JrKVt3V6X1tBoa6h$sO(HGIU9A^H$=9jXb+<PEcfzgE;glKra zjXTwb=!0GE9aLfPQrv;Kf9HyHG<rMLy`f&~vH`wq4M-iT{X!j-l{dr0Ayp1gvxQ)b zKU5&Q6%N8oGuiiUnp|fuG*RS?nP6XUchimm2jY)%#h%F{2yv~53cz4lij#zrPdsAn z*mYUxEc!%!6R%dcQCpD2ht+pjm9xwXp8;DV4y@7i`nFr6Ryf<<*q1v4$khJ<th7V; zGBv1z8nD7abTLYZnDskU{MmS^asOBmySHOLy8(}0gSJ*mkE%S^p^B-oBoVRRtEoT) z6d)-A0H@z1#i{+5!7({7be|vG*s;#?;~ImHENp6oy^b>Gn{8jQ7c)|Pfe5HsP9oRn zc==-EbT&+nemmWt9@1Q91|@r+m4$_0PVHEYUOs1p=^nm@!9UA{Tfxt6qj;YE(;4j7 zb-~uS*}E@O)g5*e!00;VswcZLi!H)LLvb?`LM2G6qZ(eLZ258*N@0g-q3XP`K-p15 zUB0kus3lheRwYEzNC;i&_JKQLc*V2RIa%o!Qj*94S(y+DFb*61XoLQPC_0eQtPt6n z{V_vU<d)Q!?G+)fyIKunzF%q%GrC5-fT(d_rI&sJHs0$pHGo`HLC}QSbA6kvnS7wx zMReZMsM#a&q@6o|9m79j#va#8-VgMQquBEaq7k$ule%jXj7e$V&IGM*76_KVnQ}6g zI?|cm1~4~gO8yQF_g}I53BYyj<132C5Qm9tyVRM{_P9LkN^qyAX_ADib20&RPP>cy z*>L^V)E78ngd`VI>i&(9$`a!sM6>`Wl7lccOFuGD@nsvi`|V0R707XIT>C4%CH23( zjsIM$UjV{c@A(3U8zC<;JQ~J>ewQctlMs+W=xnGVVa*$JzD6<C&aj>6IFJddm%LeX z>0ICm1KTi{l!m*EZt!QXx<m0dv%0pC<E(yQ(Pl;&0yDj`)%(-8KzDnLy`KJGX?mPI z?f^6kQTOss{h&XB@n+YT9fO!mt%HCUmHx>3O8oCy08sMUj&_BR%`U4d?bt~|Y2IgK zQc^qPLg?!PTwWot03la(!j%rM{+E0E5B2ab#=@WZ0t(cJ<74kz_Zk-p84ZN+`s${= zfsR$?Y7Jt!qEIrehGL_A@`GF%?)mNVqyo3U!Ygf_%;>9w`YQg0<6Zr~Q_=xRR|D)O zHJPnu1^-l5xAuPFA8ii^;2(3smq{e=Gf&`~+Th8e;%#x6IAp2{&rjY@z9%DqL%-s5 z`u|K{f<9x@T#9s6wu<DCkNh$^KkpY2(O}mN`de5{uVvwNHk^tjkCjQl$Cya~G9xw0 zBb_v|DF$6*KB+Y8DMD^TVr-8*A%4<+>eXZfJ68<80MwP#e|#7k$MvJm2cY(bSox6L z9X3X9qOP(d+g&{rq^49p?!3c4fP4Q~>NkLtl^U42ZZ#pA1rkwBfv2ziPP25JQJ4ot zw4bz%Q2aO>y~c$}{IpRvrjjJ!dio)bfbBFD7%O3$GT%KRE?EwTg*i{1_;&5DCd>)` zLh30;wSSuP>}l#e;QYaOM_c{@{Xu@3wo3-!O@(|9xfs3`13th`Gd*+{`CG)Gyu2U2 zjRSB>4cA@UAENpB0l*yvJ`m6)w){!q(Lj0OE&h_#gUj2_ed?ZJRG@&UlGfvc@5>GM z=}LvmUa9^Xx;-Q<Sv<ECR1oIsc^JT&2Dgazqc@cSN9upMkI-e_4_U!<+qUPS3DieF zr0HhMX;Hnb<b~~@>EPv~MNG}uZO}9}L~!%3Z(;0t<>vu|*P>g0+>&FcEB~iv_sx6T z3CqPj%SummLVQtLn`RWdfB`H<SH#TqB`xcOXoVs4j*!i%fr0A#T>y|QJ!3oJyz`n^ zN89z8TW{gR0jq;%|5Fz;KU=4sRq-ZkU7>NLZEk@epxdv?^O}tt3=7X~|Fu{^;hGjQ zl-J`o1l;%l09@(~P4p1>fP%j^ooM&^{%B0E)Yg9I;Ims7b(-#q-&vlH)p&?I?Hpo! zDj)YTW$GV330oea^?=Tlp4^#+dKkR<3HfP+c?HnERNo)Y*`PwyItg3Qq4a!64xm)l z1B(&}SKN~9f7dB`s1wI|S4xW>KpxsWa9Mr@`(6LL-Xvc%%=V7UZOBwt-L!%s6`NbI zK0JM@5(`&}aS3s<53k3QEFvm6f@kiE_qKgqi)@mHzTff9grO>$%Uv8f$Iib0p6AxV zsgZjT!Hl=eZ+&i9Bd3_Ek_h=8IsHhCzrc_YJ>VR$;2O{>Mb9*lkB5{xCu3TNed8c; z_M2zA3_&psayAg$x3^1@&`35Ej7`k`b@XFR00Z>x$lC85+JO=s8y?k#^Sf#qR?8j{ z$Ea1>(p=qKy`aRxo=w{LanTQPUL4PaZflwrTnCF39A8dxg9N2gla=ZRKn%4hW#kvH zLRU!sa3>b7*=5l5U!^DffeYOzFO+*3aemXCT<IV3(ckBq*VpS_D9<+_=G8o-Je@&} zZqjEh*@idvbCvyOE}{j!C&F#djv?m;4p*ZX8U2aGwy8JO-Wy%6=iojF-}Y4#tW#cV zJW0vkGe9dpv*AVSOa8akcleMD4WuZJjspuo)@C3QYosv|n+DqOM4T<-mVHQlMEKDZ z?G@dgt+IPM%fN|zj*)i#UB+zaveZ~JC7K-fx<<PcKVCn+K3Mz_;iPHSz}0Vpm1gM3 z478OEaS)(RMUEj)#84j(=_Gg%rxL9yW-M^^%l5h4sljJE{v~Qz^MD;T>DQZu8DCVu zW?cB>l73Pw<HGmil|4I#D-i{<*#Xyczn&$5Qg2Tn?pDR2JW8px@2`;aU!9*bYjfq@ z#6noC=+K2akd^<`n+YZGs#2@Iw$HK6ZfWh^J0&(>Esc3WEK}0b3U5CG0mv(T@W%Ie z+nm~sxsM==&s)XLbbvub5*!!LQ~u@<6rOm|^u1|VaDf&;rlr>1WAs|x?#{kXy3#@c zG_u~uw2i%{Qkm$}(h`dw0|+f90Xg%J@|D3_u~*P8=26_H1+sb)xx|2)cmmYBM$;No zIVIXuaHhc>P9Q&&>rOyjvW|b$FBCafKo$JgEeMpnCCrTZ!48j^4b6o=_QRhXb9P0R zz8=$G!OkK2HiqpBbx-wvy!ICKEcB2?uW8JAYji-pph>!kCEDjJWUq%IU8b|F$?GnJ z>@%7ZYJaPdyQap^Dc>5Tg$&QfR+fiB>3Ax0DFe^wZ6$ivYHuzHHDv=Ex=3e2b~Ae= z8$n?xEJ8MHJsI~J>7#dz3w?T|Jne5X!aq6J04HaOYs9Ohz1WKq4cFONqPOqojlF3{ zNliSwC@&uC0~@Q!ka+Ak<187QvrgD%kyg^7pyl3OBW;~rIn|^o&{?8XNFaZc^*}xo z!r#Q!s2kXNaSi+eC;u@qfIkkz=xnU3jOAUX9<K?JJTgYTLJjT<&#>o&nOG-YtLZ6T z@xA8aC)3uSTUG9ky&GJ3?T(W%sw%^?%{0$6xoT9Q39U-hG(cH3B89=}86qUc%{98C zVhSScNBKKHmiy%AOO+b(q8&Bv_Z*;J%t6aGTV>n}=4lp1r0orC<z#V`B<rVi8#rtd z6OFN=&n+beHdfxUiFsMD8fWA_i_%^zcM5G%Y}z?i62~65aUh_H>m|31{H>z|ETzU| z3_-2jP1|r0M^y%^=RqbhB#%(ej$<!MgbI2H>MsH4n>DD<?~v!(_;Fchb9bnpb?@L= zx|CA4k&rT)tRQhL&tt$Pr3ZYTqE4x=`F%IDPth+Y=DVMy%oufeE<DrE_VAegWvbw_ zAf!~a7UjZV)-COyQlTTB;I<;SJMb@vf_Q(?+{r?G*#_G#P`CW#n`0>{2cumyw^YY^ z1Q<<SAZRHHdc2d5XiJU~LC)$;jDhKB5%*O$#2T?xS|ap}*apRxU&Pw)JJ;&RI{@-$ zAag+O*DYTLRL$GiaqbTX--ol$uy@{&tgd<a%Y^a8hdSFh`}|K@zVXZGNXp~R8eS>3 zK~-Q+()|E*oQHnMc)F~SkWA&2W5|~`#i8Q4=wP&$vDWY`{+RtgVLQ-b-nZ5^*pX*k z7N<k;#|`sfAo!sZhSY<7jHjtlr0)HBE0M~OV*{6?T_!&|_bTgGQj01&bEYP}?LBkV zQa3B8zOS-jMC?EZSfaNf$m#<<bJ*bowsoq&qx*eod-^9Vm_WNPpZ<w9-(~uQ^<!op zuc>alZ@<lE1#}SZv&dm229`jqNJbdaHZc$zT8kuYMx=;V1BV0#jc=FaLDE)PYK`RM z@^y9*xA^}_<@t}&BcFL8A@(61+l2cvU-M{QaLLaB9k=Lt7x2Ag(sb(O_7~ec8btXA z8Mn9=!{S$MBIiV_HN(ex<r2epWlrZU@&mG>uATV!pXGuQ_ss4WvsF))lmS;(rCAQ- z_|Eut>RYEQEILtrnO4bgA<%{5U|31#jO?iHaDPn{C1-jt#-$$H8>8i3QS-cty6hCO ziXGHNSQL>~iRahc*Eeej?l*ix?B$S}f%Z|B`wz5Jq;W)xkdoz2LB%Q@sd2G9nxNTa zw>;B3$@afN8rQLK-zi7mINPce&qhUtfw}gwIlU+o)g;^GT=?z`4JE|95=4YU^l72C zOveci{8|!!&AL2u3dx70hax3$6k)JLNjT-BT#D^LBoM0T)m*i@MXK5=|M$mC-X)W& zQ}2=yc^_wIQnAx?3vZ%u0ZI;U?I0v}gFNHB&B8rGLYV7+={gmAo?nZQ2y#+&`3F%I z`ijp_FkU{8H(uW8#5r6X-0h*CLCdlyWv;q$-{pOpi*JhdbPArP*#yLvU|<}ZKd#ne z(2eZ^hl{kk5U$ps5!H?NMh&OVSXD(x-k<6>v7bs!1i;sl`>x_g?+;zu>{(<2q7k@h zOiKer_}5Y_k?WD&sNe?&TgV?v5q~+qhztx{)YxG$R{u{`_si@m=PxqoRi)OQZDQt$ zV{!-c%wY18p;7aHBbE*{G<S_$MQP{t242rgt#q5|yN~i#t&Bh!{LAajf7nbmWv2H# z=Qg~Kxy$vVVQi6PgImufd3N5)a=SM0cwjYy@#++^K8e*%2GG6~fzHDUN^e-tK}zU# zOlnowT10Qxon(?<U1_2<B;^i_z7CW0bl<!Nn-YfIZ`9Ia$!mOY9EstVkFaoCfgq`f zB_wNH2q@&Ix}U~e45qj9`=y4`5+nuiYc=o~c~*tIAby$T)nr!yZ(=mGZkIeK7BMly zG@<naJ%1wCZ^<557e#ZSKZeZS7x{;G0xXYA>BI5hVEXm$Ul*m$Fju`+^WHG|(B8uT zboz$=;1xK14ktUSHn+c2o7MVaKH|1>J?G23`q!Zomy^Ev#W&Bb6W;dis^=O802+07 zqE7PQP6b7Lz*0UCSF9tpooE=lVieebSMo(%TMDPNt^lYFVp|-H8f@=K2}{XHb$#3i zkdB2DDE(DH*0yRPG-4vTRiCv|8gd77;jp_(3UgK;e2BS>fjHDIw_~CptGbJ|4_8Ey z>q*|^p>prVQr4YX+WY6fCHo_v*+2lJQ<C}830)KN{?>qA1J7QI=3vX8xEml2QT}rf z(3)bh&P%ShNWBGd|D!MD#~Zr?Z(Pk1;ON1hCBIVN^F_p)Io%+0-1y$M8INBPM)x!0 zn`7*{#(Mj`nfvPk`{Q&F4_8(3dMBZ0d0vw?t<@-|=E2iHqpME7Moe6AioZBM+HsXe zXN^lnr=}8?z$|)cG@(F*d9w|jcRi|<)=|2XzP2C;_Qx2--(%^G<=cha2z9zZIfac1 zykG#7q%`K}!TzdcFng_e9V?O?#k=<^0M*Q2Fg^%?QO<dV-mTaL)FnIz;UAVgrI#hW zZN`gT!_Qa-81lP%T}K|EPRl0gd1ospAMl5)X<)tpG?Vi!Tuag7iOylJ4j*PN2&VsO zOXe8vX1&0jRHWt=>?qsSJ&jyP5<8p(&MX}TZ#Yb_=#<b_a1$wMX~359rkKRolxN<g z5n$^H6PC!S^j1tnx2WZm|AE~GKm|<d_x1{_Ao~4Rdc3AKz@o3q4L0)uYkqpPF70+O zFv_V0^X;c)#v6<6@?39v0GQcof~#P0*27hJWJckYHzMA++lc2XWv#$wuuIwAvH@oO z>mhvSyiN}lq?w9B&t@Rm&PMB~-_h3*xx24J&cGA;uzlD^^L6~E#8R74tZA=S4;EQ= zqE5-V)hv9<?6F&ExPQ=63s{gvNroJ6%vEGM0PSb&0|El>(dB<cbD^IPofa-KNtJ69 zzE|PK5w*t~x4%%sG+M_N7zd*Ei~)N?@LL;lr8yWYoEXKk_a-p3FP=9}vNr^>wIRUH z@DGTe(#?_wrfO@o{{(wOytX#v*mxu9>DLRKd?$?uR|EgnZQ{8nU*m4vzk7s32)uHc z2B<gO(kEdfWhz)jWuMJ@Dr`5CBGh!$s>Tg-aZdSb24AkxP2@wm95D+1@O9xuJ9j`% z*hOHV?|?;)x$gRiv;D;QrZ~I&6HxeeHQ@eh)r)rXz}U+Zv4^4WfPuzba?xdfsd4K| z^$%AAXFsT3W<LqP^(0^tTVIkho?Z=H|0I3L%Lt5BPS3RE@olhG4ghP6nvXj`*3BkL znRBJBA8jdxWp0=pQF*J0x=6ylh0$W=;w}+faTP|Lvrr1y>I?F&y4JvX^Ctz9N+N-+ z{#7zn*c&%kjH-rT1zvoLl3UYn2&^Yu0)sonbIyxRIW;x|*3LTArs1R1_~x&Ni~$nt zPkcZsex;Mzi?7a6vS0t?(7lj?4ZoF=iMPoq`JTkb4hQ8c#e9E#);1{SdjwX>_!4j? z4u%?L@D>4UB+kp(aT+)gK#|GdaIUv-3Mo`p9tn(TGd!P*vd^122w$vjYy_5dWi2C3 z*qf=ogXgswFfts%WOV}&LUv_2n+RwH40$>)6YziMdpbMt2k#hL0M=<o0BP{h?KS?z z!8;zZ98>$faY+rVegqDfVF(pDm~06=XMt%#4`DxNc=3+4ATUFfjT@)uP69J@M!Nae z0YjQv09Tj7fg#}-r4-;H%wG@51cnr{)^OWQ7kEfqME|q9Ky*N)0n%=xA%gq0yHMb_ z+l5E|f<6VdS^AO}igO3><DCZgw*z&wfc<8t{n$bg_-~>3|Mj!1k3eX~EG?wRd*c7^ zc!5bneofjF*pE885s4@ZOgfO|77k1r`D@a74_9k486^V)yS$mVvn8wF4*|~OFV@-L zalHRnD!(Ip|L6bFnE=pJ{CK@n|Fm%bef#~N9{?~ieywW%|8#Nqe;Yv*kXe-v6Dg4T z|MHrDe&^YL*Ngq<hySr{|97H$ZS2mpkWt|EuS@TQ!p~Xx58Rr#RSE=$U$4&e#^4!E zcp)Vvr3ng*3U90I)fQcV;OV}}-KR$z0=;TC(Aw1wgSIRRstu-1Ha-_ZHV@idHi{NO z()1?Kr3m$~36Y5N<|(j6^TAKk?5-0N)Cd-2My}QaTeH-c+<43?oRX(PSCFk5oW<bX z$ZnffK3|VdZ1k#q6YHffj@J23ur^yl(<|kWn=9U9&Wz&5&#!k&m2p9-kn!yk<N~xv zk(!p4cuC`mNo%L=T18n|**8VjguVo?Bx4qp=2%*{HZbw*bjfTAUVB+YQf-emb2$O@ z+=f2#*_+UlL8^=smMYVOItYQq09O&&WlgW>lyl2$Xb7+#QDl9Uh6xBUzrs{I0eY0} zp|wRUMiCkQBM}yMM3ekZNacA+-SkxVV&ce*sI)ORl;-=TzrWu&eU3i;&S&HRw1gx3 z93&h+`UWe&5w(9Q-1l9B{uGbjKHx92LrOdX)>4^X%lP^EdE*)q@Dl|9#c0U&!*_6C zV4%$7@=VEMI_Nn3FW`ozA?jX?y}!vBvIqF-uPgf8HatkA<eF*zV-5%2T$IA9(B`;C zs|e(~Nc18jbFstm8QuWN<VVPKfjB0S6jAD9eTiV@%|tI+?x^D`ACh8;I2lI5G?uFn zfi`T&DAp1hEPVU+?KDTlu!GdyV6%vhyvOixx@DUn26EjuB%2k%!0AqaAggfIJ0H{* zJ%DDQBAk;TYr;AgE*NE`Izs;v{k*rkI|{;DWOhe)eoBP|;U!ht#ug#t&`TPn0;R#- z)rQIL_eseR3Nng{pbGiIsn5(BkBkO%!mSh*=lgEb-pkh!B3ITTX`S2f>uZZV7{tb{ z?jOE_EO}b1yh}VvG!waoha8_DD$fpoGf0zEo?C5VZE~lfxq%H`$_7^ihgzN%J*=<| zdc#WchuUfaVgdHJ7s8l{vOI*TZtN8{(%c=f3Wwwc??l<w*VXJk#L|GVHkN`RbXJFD zqs5vmd~`o^Q(n8zKSEP}l8GSaf~u0%2`pOW#0+Etsho$Lj<Xf_WDI(Fc6Y{yF+dU5 zf@*hX#lbWnEAdNnW=&@sp15#C?S6t6YNTM14(N@>l{cjV_>FFSEy2eHN^9y_o<y}Z zMEpRR+(0eaveF?U5~1_u@fFIudz<7pT9N4|EP;>b^4cKlIkujL&iX&($>E>ztk8+1 z_VrYW>z_o$@XKq9o^l%pONS&QHo&8)*2OB}C?nj#2nv!xohF0ACqa)(>H4fdw@jeK zz}^GX)NU5VEJ9@z!lGZ;oGLuv*#sP1*|5)Mn@vl{<6ssQFdHQ5I&mGLcIChxa}o9r zsEqH7iOlw10^~alxAqfl6k`-lxYim6Ldb<Nz!4@-kO~>vfd6Mea|Sx!xvWG^SmJ#` zCq){y8IeACkgd-9tagsJO~iT=bxdTTi3&OdPG;(z3k^8gNYgWmLec|~Qk#$3sTIpM zc68|oaRv#$J~rLk_`gxl!#I2=orX{|(zD3mil&6GR`zO%FeiHv8ez0H$(;$RHa6bF zCD!&VX7~h=2`d#W9YP`>4Kb$KnZ}x93SEoo*wfx~WWo{!1Tit!K<;OvN^OE*Um4Q@ zvXh(}zott9vI-v&#85=(@)0;S%9d8g>Zh$!b4Zns=-uLCKud}%cIJ2xmEKfF@3x$i zJH7ipuK<xgO?zJ&$jW1lj(E~9hwY%V?ZM(B)O5?emv{uulr=VrMKDDm1<O98c(2C4 zR=rw#F4?%)PDY?X;cXZ%T3^$eICa&xh~7=|Y3GeK$|`Fi)AY2oy**iBv|gs>W&&&7 z?J%N_+KHUMhINh#r>{jPhi~M8IwNR!=H+u_QP#)c?9CdiCohbWhEc|m8>Gd+4~}fM zvo^ZhLHie7w7=97t-;@eYW)pS0TsevbnP<J7tG=x{=uIMS#yqXMSED6^{sNjo7QFH z;BO*ovGY#rpG9oNb}}|fJ!%aE3Yi}zHzu5x=#|Vtw+oFtl4?H6pFmkmS_0$#YTT5| zc3|7^qBs_l8nfOd`f75s7qL*k1auUYN53z&=NtM2f<Z8Wh+VT9cplnk_zy5`9Ja|K zWpI!7U&NtMZPC)DnVI|e*PdnN&A2T#Nga{mh?L^Vdzdd4@VPSU2v7dQBRBUp4AF;0 z&3Ezhy?-cJCMETZt4jkpk&)8vL>=y4K#yy{dOcr#?Yg&O=(K{L_hL?#q-q{dsQ0LS ziCN88u;x06zoCC-UttUvi**&ova)B=M^`=h?L*$~#Hzg1aU=@~X>UFufbFt`)7Iv{ z%Mz<pDw_*38qy9bIj6~bu&SXMPw)19pvE$-X2*!`zW3tSrnpqK>ND(_o^)Nw>F)Fy zr#CNZ^=&{U4QsBI>6D<7UHfv~bb7S{$6n6fSX-4xD0p1g{w(rTLF-t{Rm0<1p^S%* zgg%yqgf3R+)vc_|&)DC*B$!Ves(U_sZ!&I7N#SJoJAE<IjxIetUu|o8A~x-|bj?!w zL?6HVJ?eZ7w#0t(z25WD*d0ci%OBQTY9?zu?lefcUS+?Q-lK6TeM8hUe;_ssteU0* z{ywZf0#(r3E+ne*T1Rc(X^hU`-g#C?iwq<zRkD0~-l|>CDnZ8!wV!HlUmGe**Wyju zWqviwDxh*?@{O4_>0;3#9*ex5XVZonQrCLdNN{Y=$h+;j-wSs*cI-$RSGn?K1c_I9 zs#I?e^2J;jH{zJ8NQadlSv_4cPjn*m89alOL*|<pYZ^VOxJgJLD7>^$a}qv1z}QGf zNra9TA#ov24Wt?slX<C%Qs?fo`a}4JnGt-`6{QS25G#1h{Lw?6*KM9jGK_iO5K+Nh z-_N94x+CgDt<Jg)x`Q01$Dxd3hsd0sD}Dwz&~u=SYtC+&<T0_*&ru?_b7F^zo7V1F zwEE#dD3@|UAQ17o(=%Rj-G*p<ugw3<`xnOyX2N~n9>DIB$z(5%c=Z9x6l(WpTWKF1 zsq5@Tc>9QuadWKRe=^2fNl<HTsa7GA+I^VswiHlvp_`{4t-iazc~`X-A(6~9tJh^D zD|zdH(O>2?SEKGiP%8*ENyD}CS$lZR?W6gXoVs|(1+hz`tMZ)md5E$1`kQdy{a%o& z24!x^fO|d4x2xv+tR4&usG%E`maUU>^*#=^Q1K90iszRyORF?Fmio~L5M`gQk`T^8 zV#tpfv=Nwpr#ksb7XBeI+v1rGrfx>P{b+*L!d#(muLy#%&@leRA)Y@Cu6dFcu`sGo z5biRuV);GrK;E0gkt+TWSm60fR(hh(Bi28^@wcj+R!g)A8J?2s9@1D0aEYKZ0^wsd z)-gJmwIbgT>1oF2BoEsxPH_$Y2L#`oYh4D%I18WZ)NRBVfzu^4m&SElJ}2AkA4)xH zB{~aXWPPfvcbV<>WTj{<2vD(5P3T4=W2fdZpxM3T;^>sE9BkjqnDzIkCnZ9LNQ(pX zO@)vl**5F9`&t`v#C3!ho|lZOy2fG`v0f#9-!+SLv+8+&hAf=2S?k!X58`!-U<Tt{ z=Yr`pVWS-w!2`3IEfxOPJ@JZ)y@B}f#gVTWf%wE)u-Q*T-I%&CA+44BcV8V;a;j&9 zMBnMvl<TH}@O}>vq<BQQ`(0e1bkj`VdQJLi>$^@z4jUVCcG!c6YB_5ob)AG3BQRrc z=F^N<vSp8MpUuqsJTin(n0GRYJkFW4M<!;h$eDaNMslCn*iV{3df<W(n3JUbc5b9v z{O7?Vkm+GoeQiRX0q-QAMT@80<2^1((i~G|=>wh=xsw~Ael3w)E#)W`Uy^A+flw(@ z5%-w^sHW{&)bKK;`8t|afMP(U^yfVs*LZ;gfLMe+C>a~`Mt>71%JxFGeWuO^U%#Ze zn6Bhqwhd~^9yChmjA$@^`_2GlU?LMcu_f4-^|QnRe92#62I>37g1$^FWK+#=Sl%H7 z;l*-3t8?}VvuBYl;>y2@P-9ddk>-5Q+2IJXMVCUnhhCyCKJTgPp-;cPKeP?{o+n7q zrMEJaYXf}iym|~Br8SEKU70k#Au}C#W-<Nr`>zo^rV`r>Ef-rWju<32Uj%<EcJ_m~ zT6Dr7Ott$FcipIYHI)5ttC25wG&b6w<_{dc8fJaaez3@*hv$%jxA&#xJ*`bkeVr{0 zi6^3#K|3u@SzS~2;=O&G4-<TPmOJSpXc@e(wbiF+?~$_LIfAno>vSn%@Xe1X!jF4R zKIguB>{%7D3Hv&gD03YnWFHd$T*t3i>P0E!+xWq1ykrEea`9#UEowM9@tA?_{m;Ji zVesxdHLi)DCwQ?lCt0%a_%9=QPo8nA;=@+sI!l?Y-0}v=JT{I(Vqw|%H|6euSM@P@ zb1~_ChYfn0+b+o-%~~s<oA>72r@3$VVFHeGQd?N}Z-%hG9yh=!S-Q-VW3@xicLL8l z_48BWNPUHKZ|Y<Rxs`k#H2U+CHc2Dkx@OWm|G-yp=<ZzUyGJULcFDAR4*s=!j9%Kf zac6Ki_<Q=rb9|qNhQ0%=>uhE77#&ZJ&qLjNQa+M)A|7164W*7DP0!`w-)aL#rRv%u zgqu!#mNeA8Z}(j=TRQtaGZj~TG5fg%I>O*NH$9s?USsp1ecFEN9)#xmUf7KwkB>rS zxcM;C_B{=1+qp$|eYui#pke)$HtWIN!Qj;y=ht~H8HdEiJ`i~A0YTH<TLe_#;o0i| z(myRFePYAwvMPDG+ejcy)ou5f_!8pRBl7ohM2&QRf7F?3mL8DDndh%{bZKV`g7%~H z{vN7i`Md)T<ZNfZ9e`OI$BtRW%68GGE3A+3Md}=Z-X%>-X!Y*o+ubr}*XVWq;`L*k z&rV1czn?c3Tzj&k8sC<nKIo#Bz5F2KQaREg7?9tuKx`<f`Rs86vzhE3q6|^Z>gdl? zN&*d2w`UA**q~1z&1Y^@k{<6b=4g8OBpTFi<UJF7iJEC&rFrE_S=zA;M8~sFpQoxZ zcy9u7W5W&3_>!Ga=doz7a^?)`>R}Gi;PAon#@R7`^zmn_9Ef}o4Twzevm|pHp}kZ3 zDGOiAisCp|C_*7G^FudDfhVH#auL^*G-tD0j*q~O8NkuHb>SGH`ompBv*|92Q)iNI zyxDtd&(N$9pEzivt&<;Z8?ydJr&3}z;du~#Zq?=DUA$YVucMTKQ9Z|}fgKzxXMM@# zhGX+w%NrZRx1$pd99-C-U~JoJ=u>94b#!pD0^>Uh=1oV^d<c>}%QbuVAF@1o(ayK) z0ji;*X~tRJ0AsfYymWklVU6<g3G<y%+$eC)^k3hdiyE)AaO2tQ2D)Z<w86wfTVm7h z3|`_{d3m;@JxJfBY$tS-^I3U1N{d$J*3iqO))jGpY2Fo~NqP`{*B!4XjqV@Jl_d{9 z9}_#`z{R1)3y6dfbM`3J@<CtdqVpRNc{)ew&>Xx(Xir1U;<v9}PG?juTT0kAjfsvu z9TUrGQRjTc3%tE^9g3rVY6EPUx2wB{JG7S&!-;mh(2m1^Xw17UVZD%b&V@k1BBMRI ztB7Ir-M>E)D{iUA2zoziCd;jZ&8(`)O+E%GfRd6?96W4n!|{2f`}0t1qnF%b^?PNW zPwW?g+#cNZ9-cCb{`@mE29UU2!l!ouuxjSyOVyJLU+&FGqpdd8SlXcM&D!hPL-_JW zub$?wJ2(qpvt^4a(eJ~n)TwtHl6!?yZn62zzD8;vjpwlVB11@<N8L|db~e-yxwKS# z2R7gMK6>}deYQy=??YSC)PsXll2HP2>T?m(7U%mUqY(LkGT4(}<LX-xYTUTiesg!D zNN#6#1?O5?h;{bP?DG(({GCm`4N^-IW$z;+@81FDgXT5u2hVapA6fjFpnY9)pP&?= zUZ_8AJoR}Mc`@L|vZ1n}HZ+cNc}1>Za;`T_vQfc0Gn>IPK5HO&bw+EnfHh#iduqFV zz4Vugn%(Cb%)CG<a~Rv+j>3&5DtLUvvp?NI`DLaf!0_Nj4`klcm`w;5BoDE&bhjJA z&OqHqMekLOh5;wxA?3)O>IagoGmVrd3tH751@#{}c^QXKsg3m3jz`sg_XDnI)m=+9 zp6>#DOMG8Bi?SNSR*T`*6?fg|@7E1=-#ApHE0wBK5a3(Y-vYw@%ya_@N$$QBsMTco zt)#7>=aL6*<rt{U3Tbw{4DG97w#3j0OY_S?dG_eV(Wc^bP6kw(f2CJ2AAW;z6k}_I zmcv&`@Dkb1;PW@XH5?f(izste77Xxxy@R;x@(JU@u`{9TN<`D2jr?*x9*VAz7^H46 z{;dGaoaeQURPorEnG@XNA&=eQ>S3=GGN|3htqxRN$oh-JWqw+O1-aC#4_Kz+hm+YK zaasNKN=NEATRYu#x5t6<8fET10<z`Ls?(<)8tmMqoLKU>%*Q5R{BR7n_;r%lteRNh z(pUO4haAThDdWGkRcoo?60dvHo?s(|q>Xm|<TO*~ul~x?riNWQ31~7!wkCTEeI>|a zss`QXchRA(D^;ml0btM<SSGC!*F|iUZwEQ<)FAKjZ3%7?eqJNw@1S?b>CLU%ioKfa z=64SfA7F=0c6Lp~h`aWHsdx*QqM*JZaA{WreZhf=o_STon0j_@svjMjf%qER6#m+R z=mj}#R&KB6JJX}mZRGAzGvaei>1X?Sr>K3md8o&W&)75$nYX})^O@03UHRCGSLQ-f zxvYoK>Hhe<ltgTm>TOf!9ai8kIap7uNB5wmmtzs!QOUlQGdukqM$I`jJ3;fv$G5f- zi~%v%gv!3Fr&Fz2<=1LiLZSTO>LioZF)F^HR*O2?MsE56_GAQF!XNEa8+9Mdf*$n8 zdOPTgrU5;Oahl%vF{k3{K_8b*#)FCVnY2rB0cUS3+XZ0#&3wnI!`mFnIG`gJvig}5 z6i~aU#^p9;t&*pN;cUxx7t7tX2US91>&QR&yP@3kuyP|&lsJC$3~gC|byUg7iKJ0X zB(%qiDQkc{@7=ccAMj?)5BtW7oBpW5SO!}AVv9lP17@~k9(c{o+rD)1z_d=erXhF4 zx>IiV*SwU;G%>dx^~%r`qz8Sc)AHi;?u@`#RWoZKAHMu4aXMFYN2++>khVb^AE)bf zXtHhaz>tVg?!Z%7LkwDdpS7a~t;5{jkk<HIL`vgwug0*NhRW!7_xrG6lRN`|(oK(> zy>wYsyk=&%z`=mZy48|UD^XKX%7i|45L~nFxV@-xrNgP&ACnfXu`nKB6hxHuy<X~T z4fb)ODj8HxOV<Zzh5RhmZm3(>fKE@uNUBgr{bN#T3&(t?ZkeTD&kIoYu!QrW*~_Ps zW!|8MQR_|z;mJ+YcO1B%q>cgcjJle|Hzncw931pJ+dbp^?8G^~a$c=Yzg3v|=;)<* zRR<J~5Met9a&8KtjBZQt80pIQD6N}5l>7a<MxFD!jOs64nOax`b?DQm=FdweaL=(K zz&Le2e3*mdAp1EaSlB|~M?PzZIF+s^meFbcgBB?9ww;;Ez~vP0;xn%}J)fy{HvXev zlW>1Sizv5{fQK!Nr{UM#aE~Krq&pBF3OlGizu)0AVc{>JT+{crgM|Jiz-HKa1DI8X z@MQ-OQX7YG(57;ro3MoG$CK&x>MCyEJIwv#=w9t+uc+OboY#0>!*mWctOK`<N}k`m z`ynjrfgE!FrrAm2{1;6|SEbW@$_=ONxVe~pQ{lc(BKPsUegk~J&e*cn+)$tViETgs zzQ}sZ4ym1f>>uS}ibS~FXCbSV1P>BaihRq?go`ewC)Nq`LzOw5y3;(0bd4OG+q=Ht z-mmjc$3pzSK6&o7cU2>a+WjEJT{r6)Z_vq20V3%Rtb=FG=;(}|US^~(SFqJf%+cWf zo_)%vF1eJ{&dq%~hC7|F$=WXr#2!TM|8yfKDr?ntggI<-n#H`#_L868rI$lN8{fZZ zd-%K!_S5m?yr9ztfpq;ii#yJ7CtQ7f=;L4WGJ9lfP>CqVxU2sUdv6&Q<-h%nDxe66 zC?QIUfRu!EgCgBZcSx5s0}P0ign*KQGz>_0hk%514>=$WL&GpM=Zkyq^PF=%FV643 zuk-SG=fyR1J9E$auJwtv)?M}r6%L)+O`+y_wf-qH>792Xw+Ib?rq_VcX0khWU7V;# z<T}E?0Qz8Y!)Xq4(VLJDLzR|*^zx91YD9v$U{nfYd5wA>q5wbqY>kwQCQhW7o2mSh z1w|Yz8cS0}54lAOuGGwr_Urp!!U=F6KG%x+GUlaFZK6Y3Jk(;1Z2Ihbx^7V<Dwp%2 z(<RuiSVwNYqR0YEdg>_6LN!t4OQ#E@YRDGnZH~`Z)$@t^ZMQ8MD*fn>=<{3)=kv*F z*esJ@gRFJOSIb?5z-ST4?b(viu+5xQML5i+t&<A=e6Y*smM~1{uJ_oRYiuB!b8=XI z<vPG8V#?zmNqPL|D~Zkj3iQcq9(9dnEIRYHx)+@eA&ima0IIvC*e`vX#BT4wjM)E` z6LM(vFQMdLrG}lqaiE!F&_T6QvabY4606%rK4ft6;B$d!&5DN_3ETS(=u7M&%j$^P z?|}h#?!3FtIA%ZYh4#TLav#v_yA9L+DHTpZLgoK&WkCbfA@hs4ShN7dI>Hm8uz!ti z$_&_W{l#xz*CBe}^FQWQ_EQCcNyc5IaXqGjJ4qrmfIhJMal1LES8?nl^>`IB;$x^@ zU#QXWYlJKp{<^?s8{o#&mTs7h)t)W=?=pU#nJWj=W63e3O(L_CRLE`S(xXv{qQe&s zv?2iu4VR4YFQF%tJ!dO=#38}rWA`~OIbxX<sKx#$!XXnw)0gkH<&jpS*#bq{2L({k zjHkw@Xy?^BPC>Vgh4Ll6{)87?QSWU4C9e5#cdS-Y4D3RRP0(DhY4m`XGSfcStKo=x z&mAw$x}M<XQkc~LRPe%)dv<Ml=l$x@AZ?^nCK0@toMSK>3W#3Wk(ZxhlMV3zF)3{N zLD2TO($07reI+e^^8L$mJa#v*ZwY{2_+c;Bol)Mu(TFqRQ0p3CLtT5j-_l67=xF^G zZ>sN}WQ*2f>){Lcne~XP@#&o!PH(8_H{L!wS_Sq;tT;B0ZX!j-W~XbhtOM1mpBj5M zaZXhs=OeEA_$K{Z8-Bu~ENaB0^b#&xf}{0Mtjlq&+sYu@wL3o5pE+iY=kp&Dg2W@O zd=SRoRDsD|SAX>W9Ea8a_@I)QoV;E@F46VQJ0iw*`zk8#nGZ7f3FBdMAq$6{;^{AU zZX8koC9f((yCjLI^ghnbYsy|O&6NKAJ9(~%!N`sv>DHFoMQs#uzu@W|8+3R_XLE(l z<kw3hCcigZEBw+p);bl|TT_{^*F}uGO|?&7ZC-Q5>maIja*1BVtQ@nHA%vZw>-Xl` zM5#fr0F93^0%SAo{*gl=_tqo_Gm7W&93r%;wjc&yJh6QRp5L`xmE<~+dri<V*s*>; ziTVg$FED4^a4^;cKkSL`-CBe?2Y!;d;dd9qsUgR`6;WF>$0VJ|s$1#E>o_x+s+?)| z7u-<LcLJK0A|88ZMC{Zf6se|DDIl==_>}tuYmx+KM7^dVWMjZ;XPpCnbNPiHW%6b3 zS9diCR)o*et!Gq-t9gm|&am9Yv!2pIQi8mA(l8Pom;txkAQkGvp98>yKd~H)!41%= z0`cfsvHomzk%e>$4Dwi}ffLW#3^BHpQYvQGCLQJ{3hs}N^8&HD$HbP1U~^*IFX0B; zb*qTyuL#$2S%05F-Y^FLT*)tPcI({D@B{LgGlJ=2?>@|n@VO27%?sn4qeX1(tRKEi ztu8yY>K8J?F%m|ZoJl}hQ%O><ff-)y3KZA{tY?Vle#RP}>Jx;=8$x;b4?ktjD$wrF zFbnZ9WmT4zCPiBkV*6!d$(1Id&h$SziBzNZIjvMQX}&N2(0-bf^<nn=>p=cH&*`Ip z67WQ}G4gBXM~^5j%t}f8`)~is3{A_bB1~Nm0uc=KeNF-3<>CK+d5Xi;UA5^kTZn9n zQqtxWk~UH_u0n`50^RqRf8qmJu>?Mqj|S$&%4eU$yJOOhiL%Om0NoniBYc&{cny33 z_r{sFtk~+-PV~V<HaoN{e-o^1tpoA&>NVX5z|TQ$F?so=tL8c+?oepZs64~4+FJ1r zMssx{zZuBPL!D`2WH9}*x$12mdNmS_r$u-?5;%3UI76U+)*KhTuTQvUR`*IvG5oA) zROnk>aGnteFTICDdXxCr_g+)`EWN0{D(fj$m?`dh6|nB}PSJ&V$ipx3tEAY1vgHtS z6U&ZU^Ci6AI}kkl=wXaS4%WfOhs=@BtFlzv8)IkQn$=lic-*&wx<?u3=C)FO{OxO) zY8LzJTN{Q*uF77~-dIf6!$^W7Jle)k+g{svbo=lgVu+Hacmh>UzndM6O3@`vDhG65 zn!3I!(~XRPaN>tyJ<u@s^zG|GNRebYMTk~e&0`bE_O4>Kx+s4%vwJ$9qtwfjn>pzG zn?WJ<#leBm{5*NvIwHf2O}p&Z#pB$gvo?ZB-Z!;ayiF7+TF*ToKXA2KKT#l1EqT*( zC#y1v-d1F3H*@C0_>a`*$tYU*4^nH>&I41v&X5v(KQSEeie(ag%t=(Q_jw4z)pHKx ztyoP%=K)04T<4|oEl89+?nIxg+9!YU`G%sr3lbYX^vM88(QMPn__X#T9>p>Q&Ts$x zj{zH}rV9y+5d#UYw9gJx6n3fv-39cMv|Fa@vwOL-UO@#W`m{fV<hElCPxK|~%h+8S zTDT|+A53?$C9Cgz>7)WG`BNGojtcr!xE*}(^S>_T-KP{jqov!m`A~m{hTrN@=)P=2 z2DuO7-^HhZfrz*g*Qi(bvK!eZMk|b8txl!zc)9&NTj6)DqEVr`4mguaQozy&Dl>@h zyuSXT*_y*w)v^&`=V^p}o@wBvjE>_pQ7J#!$5khMu&D^w&{o3=HBJlpXw^k3Z~oPn z|Ef;kA5Gr$WBwaZ$srAtKMtbe`)f3JiknlGg(22gVeF0H4}Rf2fFZG)FMkbF>&h-{ zXqTG{5Ht(Mp@)+XK9o{kGxu)A{e5EjMC-Q*gc|$j&;im>)v!tTxeEc3Oti2M;ufjd zJ5(m-z&h06QL)%Ox;*8@(gi;mH*3esV5&b-GrnT(_Ac?d?D0RP#0I%dR5CE|wQAPB zivB6R4-010`eBBJyG1=}Kjsr1{yf=9)VS`bXR-8dB@l;)V0foohIV=vox#xzXKUeR zAXeNF&`2TB{7QQlDxR<{TT-BimEyUg?XEn>YpsB=Mn3*gm1H9o=iTb^`6yO8Z`^L$ z0up+S|J=`&a_9%#<^x}3<Jyi|<VNn=p?!of{R!@jfSg|$0%-KXV3LF5)*ql8<?hCW z0m&!u-;$5Q^=aNCLb%vza=lk4K3B`vJZgN#vL0nDfA)bq_-@Zs&Yi1*(I;`Q+DdeT z>g%i?@>2grK|Q!2D9SIFqA3N92Irp>Uy2yI>{cSknJ;G1i1;r)y_E8jCAAc+Mtm0Y zCYTsB*CDjf9yWWXi+^Y6?8tJke(#7<sJ=>YYeadL4Z$akQ+m&D|Gtb3->P6;x6M?6 z<hVUh4X^nF{e!=MwY(O}2K4aB4nPX~3*{rlb?UWw7&{)WK%GyD9@D@4Nd#Ug?^5Kh z5-D~P!`{uBZGqe`=9&?;;v@E$+vM22<yUrp(D|+1&bq}_lDYxVwQHeHG3ydvl`{Y4 zsUte=S7{qK1!)pw<mUa~e*{jtj3~~Q%@`g@m#rffpPOFI@LWxpS(H!P*C%@B!@1sy zo&0Jz@6I>ho~ebIg{`a{7Ue7pS3oC{<k{<ID10W;SI(WUlDTF9Pqf)g9kahm@j5S_ zbOR(>$L69>;Tz{zgGnT+tpT!<kQvF@*uVX)Jp4FfvfLu+Vc}FlGC3b7Cbk~woPOLE zLyC8|o-3+yCFY1B>FreV_i}}^;D+_FGvN`-oaA0$iRlZn9PwWBsnS<2&J*du2^UAx zomw}O9}8HAYdf|JuEZHj^m-zTki(~o=g_UnzTbA!34OdaMlL?pb1%kw&(HBj)<^Hb z?2yrVQ)_Vc=1c2)BIk_dM^iT69MQ8xv$4(>==QL_m%$YFyf%YB!|Uuk61F#dF8WM? z|5SCR=k!Iu=;~tT^t(gTi$2|ycmL+?^2NppAtZN3RWRBQ3G^?=`V;`xc7|U0n8zoR zTO<!59m4s50`~JiWPaR7hzzdhv6pS$h#KeVi*T^!)$)aA@)jOP>m*y&xrp+xL_SqQ zde)V8B-NVs%9QU(k<-IMq!EC*PqWWhvDF`%lwbLr)y)D?ztGBOL6&Uv9ofWc``VZ4 zpvj^-SSn;8grQH7?=Gj%b_w*+0O0JLH(9i}ltob$k6oa?BQ--x0{Mc6D}T%Z8~WzP zP|ZWBY!x5de~*mIjd`qKHCZ`#wSH<NLTJ+{LTFp-Ou?X2;oE7*W5;R6gPsgY^E45k z%w)3Dr9VzxB}qUjm^JM$<?SgQcVnVc_&0YiKYeh9+2?&Srn$n-QeZ??vtw^VUbr0U zwZCrB3p`l6)GNxZgJyM_;3xQ4dC4EDEk-8#a@>)BQbNu)8#Vh6_>a>tKOd0-!j9MW zFT(Ci2#{ve^7wRb!t_o_Gs%^)h9MLshWEL^u6Ft1c6C8Y50K{DK+U2(hc<L~=O^qD zJxlP~@%mX%+d<aPro@b0FMtuFe&_Xdks71YtrVE_zPivuv)QA4x6OS{Pb>~v@z}|b zlCz^f-gbsR5`BK2Y@N)QFfy=a>{G?NkJFc!7=-W6O~jzY=|1ONJo&qbpk`a=EXAf> zrZ29@<X`7G;yNLTpb-xCHCCS{jr>uH`EjhX`^h0heFy9fO9*;u*QEzA_u_J{h_*9x zd*M4V!QdUvMPzG@Q7Jv148-^i4u1uT?CaycDp+<U=K3Gn{u?m#45M2ZW;80HvzrGG zO@PBy`#e7E)sy?zH|Ns!^zI1i(0<!DjaP$utVUI${QV#0j{b40f+w?$!A;v!`Lm_q z5>G8|VbBvrNgaL+V|$TbP!>9AGa%9nWF?2DlOx@OhOq#-1ICLqmjF?^b?(0v$z1s> zJyxFI2YBm1!VB8L98Ui&nU7p`XcvPrp3`>U2$3S~C&gwYw{~RLL-UJ@7_D}<v(O)p zC<OyMcRQ_z4+CB+Z;+52UwaW{OU9*TA7RFCMV4{c40jRg?VMx}+4LT1r7Ly_RNIbq zlqC==u8dQA?55AJJ?pf{ij6;UBv?+lm#LRRt$tJSPl59_yx@d&w3vF1XFtt&c%KpG zTY~Y}eW&@N|JN_B{4Ucv&foQ=BtL_;G`T3doU?>;R&kg2&vW;AK9^7J*kwvtP~3}L z2N*J!9xFmDJv%8-oND0U){egG@ux2#zX9*pysb!-{RM2FS+yf%-7<P^QBL>Jnzm>8 zM0I&i*gHVTEi%R8|8Wt3LlaBs5ce|ehKnPC0V|X7h<w_j)&7W1>(!I!YoXWY>XbLp z1~M+t*oJBC=`5D7!VRsa67d&sZo(O9*DOg|d_58L^9@G!L?7_f?*hX4bu38e`;>sk zl|Mn*{T|GNYg=l>=7<K8_tm0H?`FLK>EnfXcTnKW9Y?&LO}0CpX-|7f19N)dC*Zet zDQv~l=&mnUd+&RXY27feLP>cRXjFv#-fXU9WzvK1D2U)=7<zgSV;v{Pgb$hf*I`id zP42?+<G<!$?$4T#<|3b(#KyfCFLKDQ`oLxRp;Z9Bd~l_B)MX!0Ei&1J=X=VOIj<-9 z*~I6%H2{-|{C+MchWg42k-XY?(k-hIQJ%S}2kg2ImD$>Jp67)HPj;HP3iyuceV@P3 zwXLevhI=yVkNY<a0D;1EB&`Kzo_XpUkrJ6&EiB-3G-)xedqDv6xxn9&Is4ozhhvbl za552e11m4}=U009g6icT9b{%O)q|CVf&c6RXbZCEHb%ii$;5A9kUU5b^KAm+$a;qo z)Sh?t9gz)Yp@l{XRWHu{58{0cD!HHa4+a9rUKyP{<p8}GT#E<a%*WI<T${gKRjnlU zklyHjh`|%^A^G-Xmr!5tx74KO^JVt_=}d%#6>qBE%kopGv4z2CfrL#_tyIywUlKPk zVio2)seLQTZGG$yQ=s}nw5zZrTicDX{CE#FaKBAx2r()@-q=a9X*bLGdcW%_z=Y4~ zUW=zq0q^SiG*7XSU<ZlczKJ{?P)HS>T@%@#vMJv3IVh@{vVLrpLH_ULu#RK-@QX4` zv3w`o;@ef}&Ntq<0zYkw<OS~J_R9#M?wukfmF{HGsIayka+A;#U_H5U^A5Np_dneD zAOCYy0prey&5CQf#((qffB&b}_%{RAD_7pAs{Xeh16+tF`bO(vB1Auz<bV7n{_pSZ zrz3I0*t4)!O^x`!x*LBlB;H5&fuqsffYkcde|=lU)5h<MCm7hUk&p!b&wrHvd><7q zoF@v!-{65dFaNt=_xeJ)JXrT(@I-k@Nt*u?CjScZ{}U$vdnEk-6(&8&92b#dmmI0y z`yt<WY~rj2)BGmiMJx6iUhc;;HwqrC^z{fDBn!GK06V5^A)huwKAz2dqQn^Hh2A!w z|LHfTdTa6V`qcTr)wyzu>vfLD`S`P-q!rivGtW39zoDY+M6I)ha}R5E0@r}>$lXUA zX)raLZ-t%g&F3c1lv8XUvg*7836-UMyAG*wR-5N=x>-J;C&D{z%T@j-h<v4BMAHyz z#ye$6TWyH;Nw+im(;8@gdbkcVZVM9VDJr6@dv?lL0Ip!Z9IMHe%iMf<nqMCwr&n#S zU*64}PaIPWqazb?E3lfatF)S_DgDOpXrlN-qjJ>r_ds*);<$EUiH)HuJt@0^t;+!E znVtuF*8SRd+}dsnjZp2sP#?m64%C*lb21+5cN%(*?M@o0WubIwUT@Ks+-nB0idthw zRu%UGi%6}ja}D9;ifHt0a$6(N=Hpzxk(a+BtdOIW+`IdyO>rPaMDgnK!WkB}bG+Zf z{@E5fi4GXXAs+S(MSr}7r)O9O))FvKjf!QC%`M3h4IH--h$UPTw5_UXwx;~y>ii7q zqTf|o*_bOg{#p7ju_Y>bYCvk<oA6+*ydVht(ER-KA22akT<&g?;EVZQAPxNw2E@cl zGWwYXP;<UlBEoJJR65nKJ{z5FRTK0J?KI5S?=<CqJ*A^usu)pncCBbV%$kj*0(ym{ z_N}@2=x2(!lN*BY4x>P=ZHfYpMqZ8P@zIknU_!!e#<BA!ZdR+8l->Kq&wTgn0mb@> zTbg&SN9A+;#|Yfzf!xTA3czn~5eV;!n?MV0uqzlUP@78{u+8PFW)k)k_IKg2)jSc- ziJ%mEs22-UObR*yBN2%o55{5w+-Dqv^P5Rv@EB%wc@ZDO5=M`l**oYVPk~ktXNwaX zkT;f(xGkf4%8^C&4->POT+}DAUACsilINImzQ1v5WJ=@4xH@aTLKaERJge|`!VofI zwwx?8+eLWo#JWYv_3D3_-QTL7yIp2gFethxY<A#=9A{17vQRSFQW8O_#UpS>h|mm@ zQ!P5%wMA9-lDo39pIE;BjA44)iZpMZ3aRb<Oujli;pY-^M)G5|cH2?t28%`F+P#a1 zS`LjUy%cF`D0f}{8+d-%zkbzMu=oG09&4@Obk$ejjWH?sA{TG93<fvj3Zl;@pcO^R zwXF#+Ofhf<=|5>%B)fj3jo^P;LPg~-aZiAb7-nDE{;&lMz4SQAAlC&a9GWjaB8!_v z^BZ#4bUp_7ULK@wA4K||)8-mB`xrI8tMe>sJeq1*P4!KQZDxL&_bTD?{9r&xYHPZB zKrow;R=*sNy@7*8@y)O1OB8#j5~Q16!2w>BoWBbG9_7+i)~Qan%e?a8<PquH{<0X2 zkIpztE|F`dB2~A6B}_oqA9G`-lh0=Sc?p$xkJCorj0E)b5GT^EF$Ei&$|H?jCeziB z0Y5DyxB_Uo^Yk-4^d6DaIWyBI39kd!H>&X-%=liQ;Kqh*2Fv$hnyzO|IUKW|-&`-^ zQY74tLYxen<X8s6hEpB9wbLT8HkR+_<UD{yQV0x-L=uP|&wm}@OI4i5E8Y-IbKgM} zIWS2bB9S@~W=*1L6w7cJX9LG<zt4lI08gsP$6OdfNfGr*8^y)ohLAyYPB!DLMqmPt zji+Xl)zh}+ii26QMAe5!Mg4F6r59;Fb6g}jG&%YUqc)>t8uKL7#rT|(=W-E5n9y#L zaYJ9IJ2e7$Wg(Iqe6iK)2;ZxVU)%-((1SichV4k{DQ(J+U{GXOWV)S)40ru)<~p8! z@rF;$vuTq+PFl);h_;@Dh*#o{(96U>g2pR3JoBkgwvq!TrxO0HFv8TSA4^=JBR)9P zbh-Q+xtW|t*|WPC>!9CPB>z+H+9_yG=$m4H85|&#7m0N^_5l3yJ_91?8s{0~V9GM3 z7u_-xOMa@3C#~1rtFI*+mI+}!{*4>&cN)<vLf(@{%S0X|UQyoRVXXO_Bh)X>o1XMB zC??z2Z}GZ|EHhoelI**g7xp@-2|#m%dOS?;>Cez@8sB7>)MT*8#qXDvsO1Him3Bf6 z`Gmx0(iC#&V?c3^^H`Ql2}xkc@6knX8xZk^K9DXMNH>iF8czQsGL*WRVtSbvhMWrZ z7v0y;BY&kAW1!^dNoHJ^@H^UB;CAj(IgSg3Z}GaPx;k~*;>x~H0W_9!A9XVpx~aF! z?r~6SbZMg;yPR2ui*JzN(@vuK&g|s!YzLElaNHUBnGew=_jdL{?dHQtJE5(LXk;** z=+(SG0cV~dWPfS!a{S`Doy=?#y{f-C9}<X1dRy%B*i=eez4Fhrle!4s*M6iwsRj0| zoA)TR!XP5;ruRmwlYW>Y?<Ae~${`ZYj*O~{ybuq_dU;3t@uk=+^y}ffZ%9x-YP)+# zcZ&7ZkJ@-9ANF*HLjPIXzu^~pryvdMNZEtSc}0P>7u@lUxY!Gx%`e4$eMgIJMbo1P z`74D?VnIlqb?@Gw749=xA|>Iy*Ulg-le6seYPck7{Q}9~w4l?edcf12d)&*@r0x}M z;gead7uiGmLFD#bUFMQ^%(G)PoE$Y0?2l%g2f{C=N=CZKk*dR5k&zvtHsK`B@3MdN zeqz(FO8~X{+xBdIk{>3vMt6F_;B<xMP^;EnM5#%3xasT1b`5(SE5O*~SnT-B)k*2D zCsnuB@#z7rLT3b}qW^9DaMYzk0mrOXsVBE=p6l2^XioTsS1S%f7IS8T!Z<Rkb~#Qr z9JOIrAl+qBW}e;G$<N}*BSis0Ui^r-^yoB~TrT&2(+i5a-KGK-%wevn!}G3E?_9I5 zuQMRF<g;FKy4GFp1V5C<ZR$&@niY!;8pBD{7^wO~M?RAnCZWc42WICqgm;FuBTG6w zCtZ=_<u#@LT!aVkiY{-3&AT`qxZIS!l^0T<$-aS;&9};=^{2!%*CN5;YJo8&J0Q<q z%(OFtr%<<=_cJ*kFHo(XXK!?b4)xUmjkJ64bPi0oeC^KP><#lhZ*Q2}QsVL*j^9?n zSDJ>iTHN*e%?h`y_+rIh6RTZ{7(Q#7I2AtIEbr)#GAV#tnPPhpCp9h|p4eiXe#nDV zJ#>1o+I*gSvJr%bE30Ficxs_%JU+3yhC9O40*1ki3+87~UT)^$1;QJPKyPJ_fA+hm zw_BwhM0Z?^KemuinL(rHO0=@5<3O>;Sb@>x<cz1!`lL<ZfH5J92VYw-fxF|BCiyFn z)xIViB9K<)mMuH>nzL8hzx6IQ(q(^#hK@LVv!AVdEYuVISj@*;%3!j~6V~&v)PGse z@a%9z*&++FS=!k<Bu%}QBJcK7xuc1j{N!|$dwGZe{@sVa4@LVAIr6ADhC!`;dxM{- z`h%rgA{+D~zgbQ}In3iwlyt4k7#@_Ea~@bm8!$!2?$?OC>x%kiQrPvhq}6hDb|`eh z{~o0vmd5ulUo;SuNxXJsU0hj>qkbgc>$akm1ep{_QMXf;Ug+a2tG_uxg?QpN0DP(k z(fIi&j#R%i<3aURCEOrPsvdO<?;#dAHikx$t1UonC)IM4HEMs|Vv!1si{(x2JOm{c zL7Ayv@8;JX9uz5h+66hX1^BrHfV$fn9#p@O*6rhL%@M^1<sa#*S?yDOPJghlb$5kU z=+%WbbnY~rZL`gSnW@dVT47kvSq6#VEo_SS;zp7MEk0*kq*@O=i~?K!{IJwV4N0Tk zil`gf8A6t0jRl)cuz9gozdI|HTwR{kb5=GieU&deK0ee@O5kXjmC)!V+62Mpmymz4 zyf2(5oL5hReXB0@mu8Agp_Zt}dUkw;(A_|eqmbnX$6#g^FG67Dk^5X|=62ZnNX{8u zY||lPm>LW&rbR_68``$pH($89v$`s-_9ex0)i)?!Z8`WlU-+4Gd<u3#NJI+n%Xl=e z9N4ltWV)q=%G+QU1*nI5D!>Iklks%V<|N))_=!nyca(}dNnH5+Z0^c!-3b*Ic<4UF ziMMWuYp01YtrO0;hJM@M7om2C6JRnaK;9Hv{nkeOsDU#?fnAe&IWiS?aorgPNgm^p zhC1eJ6n+HI1p#T(oGs*-B~2~$-fh9w7fW9q(9pG8HXj|ldqb7rZ2xYN&qfz*qzi>w ziaCbrj0Ay5jA{xmE)y5a+i$cBDZ@Dg5E|i+<2`=_L_T-A^rbwpx0cx3?O|`m(qJT~ za2b4AG&ciTO~w?;ed2*Etmgds(1b`$lbXZx(9+!pq^|DV@;Sz=!s7~^4bk%*-z)O( z?g&T=D25L=YZfE=Z+rX+6f3q!hkod0i76e9<MM_q2DSvVx-qY3a)#oZ2*b{t-o4Eg z*#R(Ilv(fLz5*^07&KF#VBN@9vk-CEztu;W=2E{|CNEW%+vy^L?B`z$|A|fl6VUiy zQw%kXHn(jm^-%+RQ9-d0cG!1sv+7pe-TbL;A(m(O$<vIzV>=Z7<NI{mfH%uw+;$_3 zrD&AHYM9Rb^XAW~d2s*w8I`DpT`%5Y7gvt!*pGknAam&HF-N^@c5q2P9C8K0plpO~ zRkNBQ9Ky2pt>^;GL8MQ!WhuMjY-zrL*VUc=Nwf`vL_GheVusb7N2x!U1J{$!pcJ9~ z7kkdEjUV(Oq4);j&src%EuK)93_Kk6aV=APhPuHzEV8LpiHlQ|1Wm&cGV=#AvokWZ z2(%j0v#BF$#D_e6-S@`5%EaDvym>0U9DPHCj7}j|r5FA?Or%mDG!BZ7z@;VFjOzmU zx=kB-glNpTBLgy-<<B&=&0<wl^0s*Grmzg&J>5%5U_|&<6b^tIzZDwJ`Bld7dtuJ> zcT9!?J~Pd@qR%Mq&GG1f^>L)-U<2e}w`M%7ghdlof(VrGt43|EZ$zsp<J4rfcy<tI z7(mQ4VH->@%f;A{jzUpg0#w@#2mQNLZ*)=(A@sbO-+%278Hq>>wM4LcCA*IQp7$72 z*9`oPXI$_L5N4+mz=(rTB$$RxMaF6>j~*J}uozgfsd`GjwDLX;9qvGTwK$)Xb&M=e zv!O+@UfwXFBC%N(Y-)BN6NP@l9ZBM`d1xRu8+NH9w|R}cQ`=cDe%{APT1+3j>|bT6 z@MWsdKS90|$+#`zdH8-c-A}ys<L$fadWF&uK1&ItZ0n)Zk!qfN;t{Q5>gg?CJMU@D zt12|YtNz%dx!H0k<NeRj{dGu&=tO+_OQAY;61tRdw60tZKfdjLdIBdjrCV!-DEn+N zR^t1%-AzX}Zp-^o6IS=%a-Z+E5wsjlJ5cY#QM>S0->-2t2ZV<Ar7P+t;p;pEp-+cI zx*dI@^46HM78Et+Pok;!Vn!5MP_8%p>g<EbBWa{c1gafx75lA{5I-@DRA=Q-UZ_U6 zJ0F~(z&j&69~8XJdL$*%2eK?v@=A2M@3hPjlr8Sx3W<Q~EFZMsji4tjs4X&?6b~mg z09~K&Aa^+YJ^SXPrJ;uR2YoV)w=e}q6B|SGQ6G)7%j3oDW<}h}dQN8DmCx=er-Je^ z+wJR9nsS3bi5%uS?ArVZL?>Rre-h)%kmNI@dUGzmCoB1Jye6KId+~Iy)}kQW2b-}= z34R0+&sr7U`rFu&2^)R=i}#9E?G*2u{;537grio=A+mv^k0c4qbw|++Y@F;9kn{H9 zkS)au+C{mWgiP{I7-6pBtC?Vj6adPp!O-xPL#j2_GAhRqhly@9G&pxnkWC?8^Igbr z<9&&RbW0gph4_Rl+306QqWcx=B3bf`qm?$Ar0fqSE38IZCFpTRmd*4sghKi<sFGr3 z&E!U(w?u6`XAi_x{R8um^#8SM660c*5l+{2^okTmBl@El0c0d5*d{iKP9dI8dMjF5 z%C`xRl<lrSS6gFoy;ub46F{~bL0;?TYV0N=-TTsMJ*JO<RU<rYs+CQ$Eu-#BXE-cJ z=ez?}{>9-+lEdv_YM+`gxG0Vjc<j3bf$Q1QAN>?161BL75$48i2}sT7An$N`h1e)V zNW0xi?>Dym*O!F$ECv`lvpcbr4TBA3V4AD&K+ZZoFvYZXhX$gjk?e`y);A_OFLh-< zzE9$Ia>QVq+JG<rXa$@6S9?zFFO~KqcdJv<F%iT1{nkSuMcj^NPzFFgWp!ZSp_dP8 zOtVONxK`eKr~Smy*t7O5mwQXv?l*z((3>r-bGq>@U^fUU&*Fhfw;vB@<&pPafE)T= zh-28GRQFYiAKz#w(f@d8Ir!nx@({ysBHu*L%;k8~#`~&uVoE<24xMmEcw}Dbp*FvM z(1oj@<8Ko&8617w$%I}UOk$0VVJ{KFa;#Voru$Olc1yQCLPE@MN6E|)Xw?JNL-<~G zFC06|_`9k{2>F4b7c();*I5}x8K+h;?7@!g*0s%*NnlGU({=G}$6h~PeU<K}s6NSB zcBN(rT|ksCdcO-ksTg|f3x+<<xG}p}iW?SepNpUmy;T})sT~G>%y_pRnrQ!^j@8dy zw3N!IbzT)rTAQ>;buFi7G`Og2s5pxn*xt8dHm!a4o4Q5d%9}F>Tqy<?GjG?%QdINw zxfEQ9xA6Ab+9bEhuREP!V_%9f*<`zCP*}XJ0(Hhw*s0=V<Gx;z*U#ycOLP<|?5yGB zR4tq7NJmpsOfTB*#XutcKc&O?e%2%)Eg0_@H{Pf5s_1nd5P{>++!NX?v2<xc5+=!1 zu)YX<y}kE->5Ft76jaZ?iCATh2PFFP3G}KTMFAHoCz@E(LT!xUZ39Ra&RO|IDnZnK zhF?<}M=ko-OQG$W%;~ut<<wmdN8>syGs4d@A4Gue#*u8emtlqDyZ91AHF*ROF*01m zm}jXT4BavreP9xyKQ_JP<`gMDVY^3UC1JMweBoky;JQJ0HWbm8%`Od}^lAymTRR9@ zUCP!o*oY43-^cx7Dyu+-Pw2;D@RXCiVeiZpx-_eEmf7k8i5zo%qrarhP4=rTEHd~W zCE09je5tT*-uC|R^9Rn{4Ion&-IFcISnsYhP;Yf%zYOz?XJ4UUUZmRJa_EYpA<-}c zitcvZ>%bmWW6d{@37yvkt;IubzdVkj3z8d|%U(BB3-;dX#v8dvyN91UB=AdQG4K1; z`Kp+GDw^MSR8_imtA?X-2C9~Ll=<zs3)NqN5kOd_v4K1KbuNx)nQ%><2(e2KxQ28D z#kVhrowwgSd0$IN0%y!js)yaR@ezo74W_?wPtI*QUo^5dlnEW1d4m~VHDMC=>x?~B zQAHmr&SvacYA1=g@s4~=)cYECC0Mq3y@~Wn8TcFUuJeSnO;qCy#7V0}Iqc`VeH_XA zSvZ%I#3bjZPKW1c3_H|fnX|XdfkGa0eLLD6Tp^{{9+8diD6|MAxNNANG$-fV9q2sE z{<TrJBj~!ZlnD;vNXb5z8(m*n#!FQnBNa97<bf)D7Hu3BiMogZCONdhNGvQS&F|Sl z;hd<bj7$+fKJMGyl>M<Ls7iAhcUd`4S8H(Yg%d~^*L_%I#-54)N&1{@Smghd*+0F` z`m<rZmCw1eerJ}|yYFoOy1)O3ieUe)H$d&?P%7W^J&yGN93nj*oK*S`5!fLW^i>m1 zJ*-@A%S7ptlJM@N!!Ha-x(6me9YN<pFHcr2Mu!|^)P_NhPXpXolo=p}RF4ImRUCa3 zg|Nr_gpOjFcTZ^=kD3pM<wXVce85}MEft><Q}2hkXM{K0k#D7=T=c`H+Lf%y7LH}{ z0p;6idO!CE2!>g_CHi-AxQH`YfXYymvM{M4sbM^}@|Z*_AUaGljg#g{wKb)tok5O6 zBsc{z(g=4ve7mU%N63<^0NMVF2$;N1{HpHQVreIp{FMH9O&%f4%V7-j_T8?$=G%mH zDD3@-EeW$32?Tk*xm+%zhjNy#U8Q;gH}j_4K4VdU!Fv)BsjrWi6@BAjzN*Rll`QNE z=vRD|B(q_TEmfyf_#Q)|IFTgxM30TCDM?_OQq;@$;#OIHTzq%xW-78z7IIhWg#baK z<^u3`rI}RX^|R(#w`o>WcM!7Sze+#rnRR^4M(CZ76t@U%Nns7}HT6R2_}RICwXZYh zN2w?o*O?a%iolzoXTX_`8UPA>!k?GgA^U%mHyzH%NJVT;if8xU1V~8b9*nQ3Cd^EK zVec*@F^4&nggtP+$ops$D~K$1sko%%=dQ=Qy|;?dfDB`S>C46^H=D1xZ6`Riz*Z&7 zxoa`$(?D7D(YKiVh~)}h?)MT~uX&YeUasaGZeT-qTMu*5cFoH2>^_lHwDFyX+yxEs zo_!yqi9w@kBdgF<@S9CGL#fi9Jp!m|I^%5ENGz|N2N3ROc!r-~h?NYZ?aGwB2g`Nf zZ@8$Kj{OpWd|jcu5Elp@6dtf{Rpjc5`YTe^XxaY95B=X=tN*v-9ZFa7aai}J-!}ng z*uxIoi<z>rCZ&4&P>4-;q&eJf<D0RC!`ngQ;U(1;ja-{k6w;#L(WDkZ_2bSm3?tK9 z9*-TTN~j7ylNi+>liSy{xFE;-Y{9Mvm|0JpwiW4fEu(_2F|Fbd^PvpM4o?zytAbz^ z=MhD|$E@m7+*-wXf5j`+@t%xQlcM)%po{nYRlIWCxc~t&ZM=~Lpb6iMgUDBMWaEsU z%aG*CH4$FC2qp3ScFp6pN+U`$lt`gnG{Fgg?T>3cXe`ArPvzbj_T1(fmuOH$fMq=7 z$rd0HhohdQ#O?rC$e|%@wlQ94Q!vPC)BLo6v0bY;bp<?7ly_XBT0Zl@dMG1s5{Rtf z-JgOOX^~b#=Zn-=@4pWSBA+`S{|P2%i6e-<Rs=f0aKOQcYt@39Y2bPXZfl#13FPVm zU4Pz9L@+L2Q<@^!_Y#|O$>ZcAFO`I$9+}4Z^<z;I>b*n3jRCFn&-J7(Vp=bv<?XR~ z9wF%`*BXAw$|Fx^);!z&NeUBn?Q7e$jO&C}k1t6LJX*B&8cRP?gK@|?GM0OsK)T{~ zyYWE)C=1~z%g5%wJ0}>*{BMMloRJIN_&GKkn#7~ku0@kmiot%cmKjNg44h&pj#>Uv zjo9hlszS4^e6r*1@d{^vDe(ClLOQb{4U5BG!VW=QN-PMTy3|p((V&c$(8kXs%aWNh zo)xU>9U|`}7q<;m$W;xa*^e)~aN}orG<A@bV_FX?dSEsd*hB9~jIk?f&bT}gO|vsV zf$6}t)u%G_d2ykicqxh9121B0*nMRxf=$k1!lkim0aOz{_lUZ=LXQ69qkNyAp7uO- zZUEoE7`G`0;03C8R?^B1zd~xo4oUye3pAOEBuOHxG(?^N>2q!v<LR8n2Zw@q52D7V zFSF(wqd!*CgQ=YC<~*}qFFu{#6yb84b=5Sr2)h#<7Mm>FVrMfOde#aZiZ$sDUtuwW z0?+Xq`VvgciLJ|PD)^5a@!;2OYN~wU3Tm=62TEVIe&&#rc}|i}l3;$Jp0R-lrfc!W zW4m@ZNKP2~43n63_HN@59FeC1K&1Mu@8u}AU9&jTXx*M*oOmoJnv_;|`T^1CMP3>0 zwl(!w=zhPtfTFIdu6k0Xe?6gH9zR}laxsBS+eEx>|8gYQ$l6p>70lw4OjMWPn&{rg z=p&UHv9kKxp|<9;7d|)2ZC1?GH=SJL#@1<--nwh#gr9E*?nkb#S<q)cqPnYeUGS6m z9PGz+Y*<YjH!MbltR$c@!y?D?{vG$8xym*q+Xt@0e6KFGCpS0wI{MESY^{YDe^ok6 z8kAiT!<19)3((MNgXb)hR&ImYs<j&y7tZ4(ajW_F>sgeKGTbkYdI+5nh=GJTU)BOc zU8`8Zl|0RRFIb9oa@I^kLQtODwl*jd>o&KzJDu7CbV3qQEF<@{xZFze&-&8l?p;>P zAJ!HGfCL42Vniud_@fJhvhth&*M4};L$5-0%YS7oJmYeVf7m!8Bk>b&(&E}oeLHEq z)@FFC-o=xPu>-VpAu}7=u^>`a3K)>qFl<Fuhg)=iq!FUA<G4euS%|j61(s7Vc+1ec zDso5X&+A94ni+_niwJfo8O!*cf~ZN#w^Th)nZ*}Se|+Be%UUf}?6B5sbOM!#1q&PQ z(uFY9QlLO0+gu%qL-6DQBv!%X-7Wck=UkG9w>4~+x=TOWx^DtCQYb1%=N6<vlX8J- zq%mjT8#N-z;^ee^T<mS2AG^}cQtZ<C264@tOIp7h0m$Qq{w@|LyvE$+l@(c@!9hxT z)xG>e-@w9s(kz~%xnUIf-Q_vtP_q^s3O6aolSeG+T4TkAbIX+(2yu$#3q!N179uv$ z4IJ*wT~2$oSfGx5f1;$p7_078S#If@ri~Xf9*S(Oh;<}@5X`x$y!804$0C<0Na;3V zZ2a|=aK!01+KiOC=mr<q`p)nnd!Bcb+p-8um&VZ}1yNtVvF6{L;>HmaiF*$DZJI}L zeH=GUrXxvqYmua>s1`U^e5VV_&h5T3|3QE-qXV!Iijwo1heV9AcaBYF%AQ89TXpyH zi-(XmvumkUzz=JxU1C)&8FM;aC`ayb+<C{9J4fsb<&JK--bf`Hmf_lCajo3VPsfnJ zg4GpAGz!*6lbb0wiQ?XN$u)_JpM;s|b-ftL`o^=6H)wOphn^vRSi0NB*XEJU1gyz* zHx(1mxK^-hiv3iu%KE@<*K(X@STs7X)qbxWo@+<f$4@Nscv7Z`vhng1`KKrp$hi^w zCj}f+%RS348nrasO5SQ!k-Q6b&9lsYTA!bB4z@~;{y}|g%=Ri??~dj@zU;ndfG<od zB+ih@U#YK8DpE}hHugjGB7<IS#&c7Rgf))mMT@c9madVQQ2xV)D$qgLKwGb(-g;Oq zYB2wLg)q^kF=IY(6V&wa`T`<pnRm#*p4tctk&OBZRpUC+XxSSlOVP?J7q5HJC&RcS z>e9#)_!tB8ksPhUoK;S;p1?S{N737X&`8{o_O`I2${0&wvW7d-ucAU9LH<PTpLvfj znpE&tr_d?DBdYAC>zu#5S5LQr7ugFjclCqa15f$)dNq}iW3#ba&EBFRaf`;kK0Vkz zCUil1XP`g4erLe3EUB&967zaPmPR(gUZh#5rJ=i=Di#yD%ukz7O5Ayi=j4$t!Mgpf zf54?CFc^&*nJr@014YJ7J*B+OY}I0cORVn-|K&Mbu*(fBA(41qW(73*mB0>-kdU8H z1gcKYIQg2R#A~1%2?FobZz|xhr3U1Rdfco=C2VJ6%sG(!r9v*^1DrqffhJPO=f!|A z{vjH2uyTv60hp}GMqq&cca8f6L^5!_i87`<Qf|bjz(UQPa&wb}=3q_kT61i(J|j%c z_bn@@3z%$(;%B>CAd`>+>)lN=(_xbOZV<|-KgL?kF-tqh)z=?ENCDRP`6Uaq>od&y zkGB;A;MHVD|Gz*aiP&zobR^Z|RlOmpCebH})hZoB!e;u0Wpu|`tB&b+X5%l6!;<yB zAKl_rWCC(mA^N1iq=7ZpN0>^u%_K!u-K<*1XvZ$?+PpGAh1&_4ewNSoqbj!;xQA8; zKb3O}wS?Q(SEK#={*OoupXcYU&GsK>KE=gt!l$3Cx<#V$Izw|iMZ4lJZsb!e;VR}| z@eI04!9Q4N2v%-BmYmd9!oW`%{h%TC<gLHmVly>vsp0UQzai@l@-V{uM@W@chol+i zKa5)yIzMJoSia9b5UtZR)eKXM9uXj$6mZlVR`+Nmf{xdInSJGe?#{O;xjaCrd`}Q~ z_)R$=PTqx*<thK}Pt4iU#yh%a^m1``7}&&*Aj@02XPG8C?B>@;6IzBF4?JKX1PDIJ z63&+QE-VARPRU;kF%;Y3g}QJsnn`N0OtiCPLPu~Q7}vlp&q$~IhmiBPO(sEYz<OUI zk28e6mF<~_pZwPkZom|GGn1nrBam%%$=;)a8G5c~dpxX8rc>|**5j<>(uo+e9O_oq z1DGMO0*ZQbv8@eL;(9VTX1(@e9$kSJpzsv?anG+Djhw(FZ<ZTnE_E$>lAiy1vhIo+ z1O<mSO)DF_ZTOL&tVu5SU(6Fb05AF)u-;8eupxK_kTK2fLJdF*vh}U=11!u6#o?mN zif4N%+3M&lu%XPb!cvvr>*a9xvuhmRqmbwP!!hvFh&)Mlenc$2LS#y_dF8d{IxFG{ z*npIKpuZ@%IFO$H#a>?e->!9Ld-q<E805$&iEMQ#k&fK@wQyf=^q5KlTk-k``__GX z8^a%9`0WA|DQ9mpq$MoX@?YFr5_Ndt=<?H|;?hUoE&KEcaH~G)S~JO3mp-g4Z%>1> zDEWkxpxA&h=H{oyjbc@CC$`g;b#LIL{b#KF&a0ho@kU}b&4|exD6?(ydD%sYo+)%O z!$5O}eQHX(9Bt&%p30|-uTlvqE7zLQ=Cq7jg27wc(qohf`9BS`yMERWVA74S)%^e} z<+q>QQCURj)`yDpV!oH>S#)e{vI!t&iiKVVzRq-HA8Sh6X}&@f2gw71#!2NAQk1#A z)aVIX;of(~b+rdy%TDdGzlaG4$vuah)e_ifs$SE`LezrLPbmxY!MYZrJOaWtxkuTN zSQMgN@-<isf>K|_b=Q~vc<e2-i-zfX?nV5v+KY-*Z41JmEsYS|V62Y$AV$_u^ZF^m za+jhC@acJ=FMi&}g+~`<lMYB0`wz_D;&PvLjgJIjeTRt+cxi6k<wXmEZkvx_pNlA* zdtcaJeX~e;EG7RbUl_Dw*#UU+OP%UxrI@G@4fP_C5`W9`m-sG^sOshBi!BFxUv)19 z-}A+RkD;Q7HqQnGiJi^-uj_`ne))_abYGg;X=M`(9;2g#!}1)4QDIdt4GEeL?V5cq zP9LY7WVRO3IVK2q$bBl_A4@&?m#zeP)(*<QM3Y;re1gw4k2z4Gp8wX4?Qwh%iaWxR zd7VB_(K|aE&0`w6^!FBOffYM*+3}hDtcUp8)nikFb6CxPw1^!rt($V@&SK@#krIF( zgpBmi$Z6_BeV<1J-k#f`djIpL@x#h5V9Al#Adf;B!)S4S;qRpP+@LokFP{L6Fassk zfWPqN2S?>SlX;7x5U;5IG3+ceie>%B$ns7-($JyCYM1s{S}@|9@6CQA0udI9zHoT) zS2fL>8%&qb_lgpEh%zXGist(6^JZby4$W`5<9}7ar4)2|zbuU%xdQ$ZW7oCx;JSU( z`Dg4&g2eV$TO!ufE9LbIW-(>6&3*YlB-ar(b={1-Yt=zMM1{cg^xLMNl#{I*EirDR zAfGHWiQ#_2V%<!K%E?P=HZ=6!YY)#`LB1W)LvCyYH}-)h891@-b4>pKiKG<9q3=p@ zx~rKEKg`v1fk;JqstwTVu^$T4^TtIfwBEr<7X@X%pn%ZHi^dvsM{pi_75nZl<ua{j z{sC)@T}H-wyzIGWybt=eJ@(Zo2u@zZ<<3Bh4>qC=j7-@r3HzUMc7#7(>@Ec5@w&tA zWKG=c)N?~P6VV9ts?OaRI`8nH>N%fj9EBL^1x{*^Vy0Fbb>tYIz6>G2#J=Z((z$ME z(=>pzOV?)$f0w)a+#!Up@#IN*b3{F8^TgV>rN%#b^qZ(~|3q2seio+_Z~^bl0Rcl_ z6oM09^NbAacy?OVzXB^uBBZeQy9!LfV{9ZJ?@dkE6gSQ^fsHX-S|vs=YYa`a#Y)5> znJE$A&G_-sttlR}jTW-Y>qD8+S}COJ%%Xy8U}qt~KyJ%chG$1xZNfT$1mo!}|KZaS zgLFEK=BjF~bbX=a>MjV!XXj>j`vzmXw(b~5<EkBAgx|*3saXhR`3P|M(2Gx=48`GI zGM#b9hHzzSeBw`tBx^i_n4*iJs=@|U<M>lN$Oif)It$Io#$G!F4$H95TqN<-wGiv_ z9SE^=NqJt}p8~19X&*YK=MSG=9~TnCZP9LcX~I$FS!dX0zaX;U?u*pZaCJ@nu~|%O zU(q##U3A9n{7-%#<2LYkRAio$0iCrxKpx*@P!vKE5%i`zJnn5jiq6rq_Vs#R&W?mU z2`sE0o#BmuteV~BQ1Lo&dp*|#<=O?uCZ<I2^aGu7-Nsi~;jy0yXzLqTYv%6iA6Rk% z&A55T97`LR!<_f<ihji^U(E^==h!srrGrflOTVKrC!y8@e;WfPxCgTG%5>Fa@o!H< z(X;xEX2J6B$?>mSOD!x^n^({HQakWQToLhlPO?&|C!p<0@J#iLJ3suo+FEISrWQk> zO0R|a6yHmT?;XuZu-)i)a=dNh8kbP&EXhNag>L6eQ5NlMla4umV!m-Z3_7!YXeMca z|3(m^6A47wHw7FbhN+1wI{EfvLj+z)^vnvH`qm#=VV?K@sT*g7^C(qyVKi5KtW$I4 zjltmJQy2C(@Hc0}o^p}nUJJiM(iyc|D=6J(URaLhsXuY}1X=`ZiQ;AWhbnF0clRo~ zqg3EIUv5aIu+YbSVSJMz4vC8ht&5dTu~SbGC8XpAniTP<jK?C{u5KOHk5?s(yRj%4 zHdSoeE`lY;jYMW}6PAF|bLrQhn#ZQYEXrmOo1l&Hek(e-vp-n~`?^Cwiy-8ACRjVA z?~49N3|T)i7_1Zq&Z|CzP?-`mGDKu!O%oc0)xM6|n{U)~0Hwqa%dSodS5>OGDRF&B zb%6w8K2|G(#mF8!1j=#|H<`>6njib2P|!Rx1D(J1olpv39~K=w4!2FMc%SRmuvNwv zsRW=|<2B>m*j%;Mt)6`hNlN+kjC7C%iWKy@^a&t_M<?U@?)ug2%nGE2Vp=bk!31HP zy{_v%MMcbw=TuHF;+}NPIphTgI3ma!``f)Z#*6~G@3rF&Lr*JEbztsP;z3EjYoy50 z`&U-8BPq=b^xmPtPID4X-XSl?lw2ss%tsw^4M4{h{tgMw%3=#%{pj{+gj&#kUh4j8 zhL<QpdYMondwnN#%h2WC$y6*Uv@>wMJLl$8ldOTXm;S`D56&ULky+l(9Ii?QIUpEg zgtcm-73Jj_x8{A00te33iBSB^*}!Y7a@GiQP82GZ$xA^!pXp<8vNWmT&ym5U0}2wR z+c&<2g~L^H!-s1JO7#!?j$#RcuO@tczzg38rsn%#x&4G9$_=se12zBBsVY12uFUdZ z`OJoJo^1hP(5X6*pLg~_5BMZ@X5BGNf4CI<>9~#Lc>So=(ot*ktjW#*ec_u9lyb)7 ziFtDo!~S*B>LL~!{pHt+ZI&cT|DH1Mi&4!y<pbyhXyIEs<(M=QRic{X`SgMcd>G!y z;k?El>~F<(Qsbv0g7g4jWr~a4w|quMsf#X4k<E*yUSiZ$!#MHiGIMVq$)5Fr&aAfd zxc5Epkv1bC?-#o|-<@jrb$H#gU;Z?=IXn~%C39&0<WSmr7z*0E;5w+h4WEhW=fEaa zENMS?HNpAEx=Wf4AX2USDMgX!xFVF(onh#SPX9NqPRMy&$v~KKMGUq9<RDIlSu;x0 zC}f=e6&g~>RyuaJ)0~>H70%UvXM{OYSep5@B)sW)bIh95!UA~M0FX<DvYy;bz?u_{ zj3eBV@c`xLtoE2L{gd!pxwVb<UP2T%F!-WKk371-4#>LPCtS>ySBsHiA3>WO_Wsl+ z^4W{fh{x!=+mWOscUU6Po8)F-gA7YCDT}P$V}g@_bvc&35uoNi#Eo{<3kj#o0%Dm# z13yDt$bN)Z3s_u_{k?i&O~7Wb4T#T}I3>hlMn69kI4!5odLh)wLf%M+KYGwLQd9-3 zJ+twPCj&9S7UU*$15QLgPHtsk%iX=Gu+ri6lwC~x7(+ApC+HSz>Hw-jo?H{{yMpNP zLfv4l+s6l*x?gj3%W;-)mz#pm9hZ1?p`ouk@*=M&032$0t1Pk{ZT7seMBe`{U69C1 zg6XTFg6>H`<Ud|hLE1=Or;zjtc12%Jg{urxnSi)_{beFp4_^NETm05N43pSZf3e~k z;Oh2FO8;o6*pfClrbnRQnhPDE+Y`I)GBgZqw|q60h9#COWi>j^4${zkW~3-?&mN|i z8s9B_cd>a%NJhD67OS(>?^rY?X->REOwp87;ChM|fUmKugs^;J{KLTuB)0#L=OVg1 z<iS{hvNZ9E_Z4+xuWi7(jc8cmC*7VnRxa?7;)a|BnX`^olZpvc61l^nld-kmomc0@ zQ=Njq%8$fSX-t?)F+Mr(^TpkEpv-jJ@5PRCWh<wO{t3ru+WJiezo7AAV)D>cwo~Qx ztPmTSrwHk2wA(sZC4#XimI8Z<xAOM)V~g_J;Z`b^Y(g}tv)TLBUW4<u=T?^qV~`X) zT%U&9Eo}EzceY5gf2&7Oi$A8$&F5s^0O-snRyX#dq*{BS)nzQR&hEvS-~X3C@}JTp zBX4xetRmxv=`CNKM=WlxJ!U9*P)SXaz-$OP%RO!l3cb-EI*w7Lt)!+RPsv!`{7yoH z>hfYdKh;!c({cZyt-FpYtdwf|Gl|b*<b&#rr9VDM;~6c-`s+!ba1y?ty=A@9r?jTG z7&<(&c*f715gDXxdX@n45t#-~cHq{syi2SudFrO~B3h`3gs*mSr6<ozs-RHX#8hOQ zoaO(T+ZnHQuh)FOW*Oa>bnV{~j;X7Qw;u{gW)1iB-1KP8^^+5>13O6bx7uqiQaZM3 zyJ)bK_8y~auk(WhPv^{$Wl6lUY%Rlyt(n>u+CO4W*Yf{lUR{6jZ`#p?>uO*B(@=Xc z=gP6S3KQqlHp?<3tNF}GSR--j1|z>^xBjyI!HU3sL+h5+b40ZZ-7SBw0H(k5imzR7 z<!rqgcC72IwvDp(J5%5awes2}x3XN%%Zsk4P6UqFALA1$I&b~@tNCLK;0U3{0wFW8 z|3A7b_urno?nzs7^i0v$Yoj>bHT@^PV!m?i%bd%`TNgc#U1;VQ`h~H~j3=4BU<Pn( zeda`E_h&3YKGA`SS=TQw^DRDqvTI&;>lenOM&~M(P9N+8j;b6A3@z(+U;lb-oxl7| z{j!7%$J+Vl*DuW}zq$X~>8hl-n;!Svg6lP{?KMSCd!??}e{J(lwcFb~pYPH+E_!MC zvYpqG!bJam_>|YJd-}WDb2eg1%Y;hx4R<&%Pyfz+?k8{ytiXnOS9<5ZJ$OS>*auim zGcYwuxh8=2J^i{8sr&!HGO5s)QDKs=m)^e-=?7YDeHOS+DLeA*N<H8asu>EcUyr;i z%SxEamLaGR5X_m-ef5gA^Ill8M4ik-7hMnn+^OoV>QM*U)3UXr!RFkl$s27p?f>zp zJMDh$_qDEDm#+eD{9VcN#IfS}-14&KVBnsvWIf<fzCqn8_AFT}jlk)*>!4-g8D$;u zz%#^w2LUX1yVC6iJUN{qTF0*?m&naB(>N4-;-{@sdkz}qS72}C1fE2&`JB~hpE(v6 z{{T0%q#fopHvvYVF|dtrc4Bh|heJZ7(t?8;K<l^9w5#1UQKJZWbfv-jJ)ir4+s8g` zDw#D0xPd0&#f61wz>&#kz<IH<=5@2oa<k+r9yC6kz_L8%GAN|JXuJGH4-@RWLKnyY zcYIE7R!RkL|9r?2SM^eLGjP%FYz2-pz%_CCZT#|i-)<!LOT}fcVrS%-(%sPVQmW$< za53V{{r~^I2Q6n`23%eKRBK+5LO?DHsGn1O(N)}%|7E*OQO8c;;`O@^R%fji0Pbbx zW4e)Kc|d@dn3Tl8bXo+sPx$ner%zHq9(kg+A>*Rb=8BI=!N3mSoz5G~j2t?fS+4wh zetv%bHDL26^_z_4#;21H0GB|X$y&WO4Y=0v=~H2b^w?|;V0aL}qg6o_XszbKiM&LP z&|ob_rwKXCI5!uo6mhY_06Yx_xFuM~&Ipu^@TYY=3J#ULBQCIUSfSAg+-5!L!$e66 z+yG2o#r(uh%3}*12TxE6ZRtOQ!;!c`$L4VCa|Q<1jh-%!At(#A8Vu%E;|VkrwOHfO zO9{B=Tcsuic>Vx32NNgS^70Ndi9SNH4R}XG%S+XV%El<7Slx@;kqSP~<#2@`G3GW- z0+w(3oboe?U15#Y$B9<oi3>9vh3*s=;6{GWA18PyN$eb#ce3LSD8dOCc*q8@$~oko zhQpDBO#r4zV6kCKVicm9t11|g2psBi^i-WdQKp@i4fM_Q*Qd{)q@V!On0tV$kyk4X zxD<BX7GLwbTfPKoGE!2MOp^w=YPR>;PABp+@Cq5AA=>TQODQlTA`eur=joZMbCI9s z78C&uSueA03I%2)>{M(psNK2q${Y=1@+)@70#Bs}7HCB~g0v_w<7pePu6X(^sCW_9 hJWgD+YqcNv&-i<NhtC@=?s*JA;OXk;vd$@?2>|9h)#(5L literal 0 HcmV?d00001 diff --git a/website/www/site/static/images/blog/apache-beam-flink-and-kubernetes/image1.png b/website/www/site/static/images/blog/apache-beam-flink-and-kubernetes/image1.png new file mode 100644 index 0000000000000000000000000000000000000000..c6d35e7cb63bd37de5f978a29d5c4f9bef6a8e01 GIT binary patch literal 103228 zcmeFZcQl+``!*~M2_c9UEzyY<y+)!XQKLmK(HXtBAw{ndU6de*UWaId5WPel42Fna zMjwpwZuk8>zwi6+`}bS#`(DdhnKjp3*WUZwXFbkiC;YYQi@SFo+`+-Yx%=v+oCXdK z?lT;m8`tr#gKyxS8GPWM8*VbMwD9rqkqc_`I5-b+UdcVz@=o8J^+}=Cfub+bg{5_= znQK#&uX2askQ%)?hYFX6xF2udJK_#9l)-17;c8)?31t?k7oqIF^B(8g=V$LK@XXCW z>Qq?vMRMt3qZCGLm<MvE>bA?u#9O3$9qStXI+M8T{S?p$CTtwmT?e|6zJ7FIJy@ur z54i>=1g_ph_4TVMg6m!VJ=1@`{aQlue?R>nECanfo_|$BG87$|92cdtF3yKyC+VYr zJPMH_*iHH8W=fr;8Y`QnmA;`)4Tbi)FQ17GN&CI7I9=tOC+Y+nV~<+@(W|df%4$7- zgKW4JxoZfoQ))X`ebe~d6Q)5Lo%kb@AQJMgqP^zfr%p{HMe%yjrPL*mqLK?ab}1)F zwXSphGgfafI`WIvA9CX&Wd(nubw(cc|K8~S8A12@fj)1IKsXM;Kfg6?)6?BQ#;^Ey z1+ORm@2CI2%AiDp^50F``9KxI&Y~}Uj+adIpN(~<|8SpUsP7)9UXuSM9typ=Zdhry z^ti00l9QgTy}|uIuIe3SFXr5QcGldSiIc5^U!}ULs;ID1j3V;*;@31xV`LWk7k#;1 zjSfH~%O`!bJ;<+3umt@_QJNkqhObx#_#=s>_t#p=zn6ZYReIG)78fF~Z6I7S_v7to zv<N)kd|v_4tb2<}F+7NOVnV(1KkC$a@>|?cZ&;VCc{_rr%VosT*WcbBt%-+j!Y(wT zo6+c}=dq7<%687Jld0{_(_5^UN@cmkGdM9~is9^um@2Va!ynHN52Sx#zFRr<BA4xv zUB6`C-TFr!gC8|Mdo~vpbzGirv^Y5yWu_ltPX}k9m<-g#cHNQW#VRj-Q#s5-otad% zU{;ycTrbHSS@tD0Gm|1tm&+>6>K;y>#+n6ZYLA6X4B4Gv2PCI*f}u#f@}Pwz?_bfx zMD@k`wZsYW3{}5OCw;4Kmk-HU{l>x?G&acrnUq)ua~IE0X5k)nPk${+JLsTNFbm$L zS|F>+PJ0_UQGIB^PmS>4?sKJ;So9GX9nD;k&wbKW8D42je^d9L1<W^{z_U~fG;J%D z&3pdHmtQ0z=%oYfQJI5tbD2Fo;k~^(H;5%C$7k#9meQhWM5K=<w+k|-bLCr(mJM{u z^;1>-pipai=w@7I=AuAPs&|v!crWaWu*Voe$fC}Pil5KfaVS34(~~|mGqtF+5a~0e ztvM>NpeW(}L4#jN$^!+JmKJe$v^Bs+i_Cli-iBXN@Zfiu3E|zlWhQmD$O*c9TihWA zDyqr4%bxhI*bJJdl&&&DLLNFM9<w8PeRpO3O)y89zIK;$=Cc{`WQ)7rhtn_`abc&@ z($Zb%6BQo|OfHsqbGFI0FKv7SBY85ZiOot+&4e$6=r|!8#`hI}pCmv0k1p^0vE_#I zl9Q9)y?d9@*Vk8k(I@0lBZJ>DP)*Gi>~L(>(^2e6;E5q34)2#PHe$nsg;8w_g^hOd zKb$jpI)-XAVdmDTcU(L|%p9!OHJ#9bS+sp_udEEeJnqzSY*{>N$7_^V4ey2_nx7;p zj%&$RH>wWypUsC#|JWGJg8pS{fdXX?19EsO?2B}eZdg3JJlk(Mo|%Yo=;crRJU2IL za@nbahOcaY#ES__Cnt&9LHqwzyIASE?6_B5y|J1MKdTR=5BRp?o%}Md`*(!)7pMBA z!xrp9tU=OSswyv3X3JS5ap2yNG-}G@AWQmkh^>KvBJOc{Q`!A<XgK4)!2oAPC7hqI zo$o`7QedD7qLPQCB|l$yGw8#LmX?U*@jP$9PVOcAv^N7CIOH@oKF;d^k#_yIFA#{m z+{P}MBaC26u}?4XJYXI#Bx3!~X-qErt)C_<=RZ9fmJU;B@3A<2A$>823Ro^rDmA<; z!<?OJ>lm8r9nC>wQ76?B`(^2QatUN)hv|_99C>nfA0HcYuz-hRFaHv6H)9sai}jx# zRid^gn>>82d=c#}O-E@uL3I0C=qcWS6F1bwaxBVuqxfPq@WMr@FUPzq=Hj@P;cTnm z(zm@Ky2W+0SYED%BKLeyB5-<Y)~)?e=R6j5`dak9af{>Puf`?Dz5lwcee<;5{pL|? z-_f;%6~DzrbIgnzqAb)aQuK?kr&i~pek8kPTIYOy_T%y&drS-$=bfDj?R?yQr@aWc z`_2xDOFYAAX{=$uNup}^bR~49_1Wc?bn~KH7IvD(3A*3l0l~#R_1rp@SCK_pBI{Z< zw$Mg(j?HhoEhRivRiiEt#EyZg9EyaRONj+lm!wY!r<(l`%@}{TwW^G|IoAz|xr@1w z5PgwcUZjf;6e>GC{vw9n_iUG~tT6y9F}FLHAyI}Otkt#5opX*w`yK4Ed7@{0ER7rt z6GYXqHi*iN^2SqhhV%B25LT9-w<?TP|LYpgg6ttyR_jWar$gDVyFF&h%FL~?Lm3-1 zC-Y9tek+zb`V*tc8GBGE$v9uE0jkn{?Xzjs=j2@m@A+$e8jmN+X6pA9k+n@xTH4z1 zKjb2k2RmhR@laxPfX(@-kTrJjAYEKrk7TyNvI`ipT*v;m%F6t*vH&|tsBzwd5XpM< zlPAeSb%HJH+T~^C<%g0_66_ouNAyQ*F@JkN<_s@2s7(p+DZ(TGcWA3$J36U>K%qp$ z#WhSQDdUto)%-e<sH4n0hsci`#&O2Y>HiZD)I+@Cn2t;~BZL4Ud3lsQs%i};(sD5k zDOY1I@;8x-471^mH&6W)iL9Y0{l)h!umauR`nq$V#H4Y$w)Elni2gJ(NxQ@B4YTXf zrY%WodO9ujm$0DryU{AD?BBAc(z-_tp7LtOFD9A81Yu)iV^$0Y=cXQSU2SamD7=ms zCND*@P9NvVL7pU}rKE47X=v!FRaxWPa^$5K5n9@U6C$TihqZ72j|#>s`i<{)pfBeb zn2R9FeI?~i-^)wu?3b{6Gkr3)yO}g%w(5-3sPdBU{)7896eNB8pL;BJvfu@G+wrGr zv-foHDKs>kElaE~mK**wl)a@cn)Ix(DTSdorl&hP@6s?XynG$5v7S%k;ou;vt*z~7 zm}epv5JK?&Kk+v`OSu!FFI<8e8`Glce>tN6A=u-WB;BbEHw<>Dt^Gt=z-?z|Vczj$ zTMGz(szc22WCD?t+$3?%qK10Zuktk7ncB-XLS$5T1W9@#i{v70zelYsOMZo#NbgHJ zAW6eJ_{+Q;R}EdQSh0e6avU!91vxpcR{r_WwY|FLLujNvC3fnjYPd#kZOP^h3mq1V zgoc0f5xz^C;mj4EOz+I5R&5V`C3Xv^xr(-dW3|nI)!z#rlUR~i;B3U`IU#<2#*ym% zz2cgr)6<a7kkj4pLfKGaW#uTnB;9;M!YCp_!j3=L)^*=IYWkfg>8Ys|3x3shV3u|k zWSa}rB3-K_jHcl~)sBubpEhqS;As}RSHRX%Q`gok*zeof>KPg_Nc$iEF3gP2w^hhh z6<n8NiXkO;-Tcr>nAb-5Z)&`==Y#Tl>inm}MI`<tFzXbB_Fcgi6_EmaJsrfrC4{8* zzc8FtllQ$jg>&iGSU|dt{Lp@9gN@sFf`3ZzHZo?YUT#`@%>=B>h8Ml`)O^I&<T;Sm zg^{esR<gm5^M*<d8W${vcFWI@Ew(7wPtP6Nq4+P3XXD(+GPD%*wA$o&B|JQw+n_dY zbj=Tq^)YgobqbiT6LFe$T|CseT#fO&I4m}yrKi@bH%rtQk^Wt0Jw(QEHbAVcrIqkK zYlrr~o*{TdpFcG!Ym+|9Ceu&dh&4WyR?6TwJ@K*5<zT4vDf)6ZQ35^ffi7b+pdpi* zy-^xj>;ygEm^(Sb1hqdxCBGDP-7kj+pyADGo5;)5z%w(WtvUtP&pHk!e!rF^F0i3q z^<~BK5bj|c_l*fU%#0U$H@@Iv&*{Q=Bvsd89B=M$;^Lw@<Hg1Bev_$#xwiHa+r>d3 z^blfl`m5~4<sV+@@rG}YQWc3058+D)6u+cojm0ZuBkGVQK3=nB=5TTr^AzrY<z(5L zWD5ONQQow-4&1kl96z-h|7^jaTz!CFVM^B7Up6J98~-TUJ(69?)bs>O=kdMI!}@Ab zvAlK$_0WxpQUlK*nd}@j&5)>=s9G4z#AVz=q83dUnw24JnWg~g`*U=(G@v4+;xQgx zRJ0`_3}1_OYB&+9tAWArlJ#{I?d;a4k;pF(HfT1AA&?>`_*#fp)7I=45s<&cI_b99 z4iv!_oG1CB8m3QAPsMiyA`NKj(359nb<JBIA}vpi$(@X9T=$o`m6B^8TZX-ql{Ggo z@N+Xa(sXn1Xmb5kb!1?SIW=&Cx<NL~Mg=!EXpqQcSRJ?inEqpUSy2RPr?*&ohXy@8 zQprsl!_JaxzBs_$SJrr11-z>c*!RGQyo3(U4&7&zF&YhLesovjc&8*Q&JDHhlk)1e z;Llv)$fNB_-2&<I2Cf<z(sOSchsv-K$|@!xF8*0HFL(H*hexyn(r0KFnxV+@LP0^J zPzwr$a#4)xkF3?arAo52%;g!{$HP6fwzA5pv{A{KaM?6P<Xh@5k3Jqs2rKvCOng8? zlaZQwJ2-uPJ?$m)=jMo}BbKFn*3K?AHa3;flET8<p-f4ZxP`BdJ~<$Hd_~$_&8bFy zvrk;t_j>VfuKuh(h4R`P;nq|Mb?>=?TbOJ8PYYH@tA=`>KEh_zS4kydYkuAxZ0=fd z$XL|v9r#I5LGe5LpS^z<iM|ygI)ts6k}0mw(mY^+Uw$}xilNV{BWKZ?%l=OR&g1hR z>%d`${O|JVO(6k}^Y95=`svlh!&s@y759>=vw8tG;p^`1Ze}I}G1AjZeaW1lQP0aj zM@PrNfY1j@10vpESWIkeY)p(@Y3{HM*zLJ8HdfY<(9lGUVVj!#d{(cPDFm*zwvNu1 z{8WSSj*gDW$rLmeWZphP(oWh_6EB%Ze*gZYGd?+~TY<_EvM!F0V}!wAKG=+u6e?=! zVSPNQRD<d;WmVOfsHmGlD|G6PPC@Mp4Gj%V>iD&WHRY~;OcasiwYZBkH4_sP6fx>M z(>8+oIy%XS2b(%_GTE=q&CQLSlvPwfs&8BRF^wSB%iCLYwOEevt==1B$Zr~Xteu&e znY+8-_aYl<ywU~d9~DKL?Dy$Vn*os%l0rg4l9HR)06w@+>FDU_%3ub;;L1;?w9HI; zT3Ri|VBi!Co_cMkYZO(8)J(+3$G1sX)=pGbJ{j7-(<LIBGd!$5wEy8rFHPI#?sV<! z$jD~|O&dvgm;(~br<%@5l(wv}u+qV=7G6|X*s~E`Z&0mGUg`sC5*r(Ks@#x}ke?F^ z3k&n!8yg!0kipuC@$scCFDt9%nR>Uj00;!KAH$xgczAetWt<%T4*aQ`Xf$zWU4=*$ z_O3!jMY+O@7Xqou!~2{_{e!Ul{Couk1<4@+6}RCW`8~rP3s-07-xz3L6)-%4p`Nd7 zbX|p|r5>~eHcmhSGyhD=Z)+y^lQQd2h6D(}i=tqxU@0*{K|$p1V1Iv9RMd|$2+|X5 z{abBWS=r2$sahi-eI0G_jO~pLG7@AhL99rIYL=AW8xBjPr;3US`+ZJ*H&I$uX3}6N z5H9VQq4%-{glI6mX~KeE3`o17%r=dio7)GCau=84<GUB-q^YU-*5sD9UefsQ-+V|a zaypb2r;w;9Rh0Sy{-Bh?R8d5X3~(Ow^r{7;)mtJq+~Nv_T6Q)z-5y4LRqN~PdkSqH zvv4?k-k)dMce$TBFTqj|?CgTGX!&wmTbnZLqNwV^u@y$b08+a=Jv}|8f4AMn&aQXk zF2Vy37Z(IU+qCHDXrIzG_tPuEuZjo?-aqQBcU$2HvOTwo{cZ#CY)P^lM<7-lruLh_ zP+wuSwZi4)h6pLIZA0fKc|TX6c8QAQPzUi0`Uejb3r5@6<rw2~gudkKnV6V(%eDDv zAHZ;SMgY)Dfa>q>e}4n-1FzMB^YMCdd09zGe=Tc#PqQBe*X5bE_IRxWvZ%PYx-?aJ zdH4lGMtb@IR66dZ{Zti8+)E1s*hHfjn&@YFL_|a%Pa_uwR<^N?tSfJ6F_GhZ{J72m z`9@PS-eA1#Bi>ND7!%|8_&A6|Wu>J94pQ!`zt=_!<eB4M0>l5iEt0B?`{Mps6DX{T zii+$ZoQdMzJEl7<T`uVxWsN)-?}Ge%B3jXVH42Pz#bphg^s$tsML9WU$XbVb?L3V_ zDE8<sA)!qvDA!-ScwuB@R6CA2XUp;jYR4CLz0Pqq4XZ!2f3!V|M4_ChLM-(V#(oZt z13=8bvt^r`n?v4$unQ@xXLX7qVt7h>pum2=;dH&&4FWd0Z5KDU0t_${Ky*YCbiY#9 z9>nST)Qkz=p-idgtt3RDUfVOkd2BTJVYcs{az2Q5oNW*i;dV8`bjLBVsbvy&dX46* z)W6aMtMWZq?TKgm2%oO8)!c@Rt{p72hpJ_WYu98MAqHxvYcmi!0s;arVrbXa)<{T3 z1<M<^8)m9`QkChYd}|5{*pBt#ZYzT`Pr$z70f9hI-S4S;0H-1(v~<)t*X#$TL_>>C z;x^c;ss?k;cnY27BShBnww&$FaQkI?t^UrAXH$C+vLLED7hr_gT3;7;`4bc-8sj!) znY<Ju!y13vj_6M*orLG#3>`dTdO|y*U>P9gzG9!ti*xIt%yE5#B#*1#)>`qWe#9fj zg)WH>g7NeX4BQ}3eGhEByu6&7XoT&%Rn#PhBs@2Db#!$26lg>o?SZ1Rs${NsczRly zH2Ul>&rB$@j!sSaoE^K6RB~-ZJTB{&n$zMO2VZSYJ|THRxCm5oP-^b$VMjPw8|6hG zZ;PbY{SVb;|H{E*V`8RX6#|`aeXm)ddJ8%}JRIlO)P{d_aB$Nk7HD4*!uNP*p{$H& zR1gI5&EoCt?ZLsp90Q_G%<=BxbFDG(fI%V}J2mlB{a#$|i`Yz27a2%tf`;PKR$X;< zb#bv0#DNMpHc(G0xfwJzG<<mCN^3pa;L(@F-8YNgUF;$T$`UnvdbmN!t)K9P+|$d8 zLCjT7?&pFt-|5s}c?fiKvJ#AGdJ{}%0yt#}`0VU#ThiV3$$B?yH#av&CwCw(506_W zE?%3HaY&j}W!BZ`uV87Vk(9iHbZuWnKJd0)3D3!CXsZ~=n_j}RpO0`<kia08R?PKa zb@;1Cd&u4T=0}V`(<Fh>@O=Oe_d!e1ST@ykQP;mOcj_x$7CV1MQd-CAK%jend)h;Z z@t*8<M$@>xcn+b9`Y@pnX}%KXgz5bc=MmjtGhHzoK67AsvGh_8XwlO!h)$VdDt0<E zGjk>lIX&$<S_zbB8`UB$aq8gUP_Orv?^GW$Dkvc?PQh!O4t1zl0=jpLN!Q!>@Zf;< zssE(MY)&bd&g$l-a>1x#A*dPt)<(SKJ8gp?%1yHT<QPZ5rXQ@TqcAF?hSgRB>*K|4 z2C;M!wSkubqk?zabzyenU>f@hJFV|;w6T}y!#L3}V7F>ghgv&42H=S*%RX1^dVi__ z_GpXXRLr~JZ*8Wy2j^tfgzx_HKwmQN5M3}(ldaiC5vn$TJT6Cr2(HY^#JGVdA1dWs zU6BnXy7go<O$hR)L^rMu_0<Yl3szlk_A6p0eHZ8pFgN@;Y}-jXc6x4ZjxE!h@puOR zR16aZN}+>`bL_{L!0bKhOW|nD(Gk74I|!HL@HcWFhKW19zzv^s+VuETWd#nflA^;6 z#7ZqLl!Y<mEtgS4O>V9%&in_n+m9-oE%;L_D=W1)6EWU9n#pge+G3yk#M6C1yZR6> zOFcUF8eXv}g`#v6$sNTSHS-c|(}J-B7wp1F$Va@7PPgh+;OG@NGa4~lKaM-pkF<ZS zU3)P_)MTTrSU4XRab)p3ZB}UxQhjbXNi&6KUCh?F`*Q%7*oq;`kKLN4V?J!}h~A%= zu(Qr)pk9`*>v*uTYszlWvBLT0ehg#z^4H`Kc-(!Qn?quDgAnc#>0^B%B-O3OFVT<g zWPO54{Dm#Ym=3>5-FHagF07c}#J7sf+VZwWx;o}OG0+ZX;PPSXr)3gF-n)quQK??V zw~ECy4ff11ar!nrpQ;aibG-KxFK^yH3oDf(LdW4%9c*p%dWDuLWosqJ+Wv@-Kw5r< z{&~+y)14!YA4~Pbp0#fLn6{tl-PlW+g@Ede*9Yx<STSMbidscmj?+R8oAO-eNA7;% z56@rul)7C8KEn93H(;8EdU6U5>`Utt!E~GPBH@(FRBbb4xvO2ONBed$6Z4l`!8Ai~ za6=e#z+Pe(L()%lYM&L<=4Y|3mN*r|?){rDYQ8iLA&&WCxs92pmCmu~vZ=k1wdlr= zH67j;YiygUO~-N=*~ECIhDAk3-y-{KEy076f-Esf!s+K-TwLsmp`)RpiR@Yg(QxoQ zykoy<cXzj^r^htl1xQ=w%$k7)n8s+oEe=bU@N%@WVr~4+S?*zDW8>;N0>*~<NF#4W z9)6f;LReA4djtr6N7PcxI=yz|+kZ?NexE0jhMM|z45D^xrtlg_im`W->i$loBlT-U zI{&#6G~!-<YiJO6Qdc-)tF{poe*pntzkx+UV@F8^2%Dg$2Um`H0z&@>EiaNL*nmhM zB-?V|oZse#MS(0lDvBzEgiF0gBF8k_q++Q_UVF;Y<+(Np?>>DzRekgyZ%OzV4A$x= zjnB-ai@@7*x=sDTg7+2p{FZO_<=?5JZK|zUu8(D}gE?PIn-JG^CGSt{mdN=k(%IEj z-Iu&~%1PDs73LiEp;{_M@AZ#|kNu3vk3z1JbE3O<slnK{I2(TRey?{{*RH7Hrpo;} zu~{`?+UEl$Pam2JcJi_It5`Cf(0vOzYJPn%)9omx%buX4eH0c^i%YQm0nTYiMa~tG zUMkesKkAYWlA}gvR?(cVu+_bGdlPb_Zm}08OWN8$yt$G3x`~q2gnX;TAh&A@mYLe+ z_H6<pRYFGa7^UYHnGQpS-I$Q#NcNlBw<5pn#jxv|QuU)a4BsU=a5vyM0Ki|j$QJ48 z;o&j&Cm%#~(^Tu>rBF8pb<e&wPYd~=|8fCHtAARSPNhZikP;@@QIv`y?U0^~aVe#+ zRmjKS^yk{aP;xH5^xUj!Eca%#>GLIfXyR5+eK9{_ZdqAbh0yn2!GWK*Y}zyiO#8y+ z%xX!vro}-PAuj%ai;v5{;O~+4f?2a^;j+E8Q7vaXpB)53fT1}^v`q__V|)-z0d*Lj za2KcR7v!C$X)|k{I_1=TO$}1ssZ3?o>OSukn+(7E2^yeUs-}y3O9i$WlvVYmAPEO9 z@m`#KBn7vA4SRW399u>byqX_de*HaV2j91B^^#jP+c7u<y7-_Rs8T}GcGd58(wtOY z6~UH&CM(UaoHyTYMQN$SVrO(-*>}))U-+QweT@Qr+w{?IeZR~CSj;ao;JWK5MhfnO zr<UuNmzU=wY~g51p1PawC_hl^w#v+7FLrizKwSrH$HU`rhH~#}0ONoH)eOZv4N(2m z*X2HcWcAOh1uFi7+46?Ys}kgu<;?s)LtZ`A+V}q8PHUvXyIcKApcVjySpV0X;Ony` zpSg|>Iq?i0k4HGm?{5jF68^AIR#v7Jb$$kE_S~8V`k|401?{AZlpYp>IFl)<r=wG6 zH_nNyRWc%ZSJg?#Wt!Pz!4Ja!`T12I1*%(a53#E;BNu#(0|PI+ySqWe#lyo>G9r5C z3Q&fA0Z>wY{P<CSqqnzLj!}Kx75w@gWqVagNC+;#kID-QLL<#_QfRJG@C^Yi_Lg)+ z9u;X9g9KX9hzR_4hY@Vd_uATts*<#{2W4m?Q^$yOniMaM!i0^6xR)J*0~&~{PJNR% zm{@#IQ2X1yKi3D;lnV0DB~<;wZA$-O9t3OAuy;haeprAE`8B)NI_K3ak~wLKk{fQ~ z?BDn=UeWvD8U?rmuK)KZMC<kCyC}@)S`A1s&7IX>fcsqcGgtC7IQpvWk;|Szj4}*3 z!T1&Le9Yb54?r@`SOvo9|E!i^@nCHfr1QXnKu<MFjxmp?po36WS<}B|%)>OZdMh|7 zh(G*BiU=z!D=v^7E-uhDMWZV|fO8um5m8KZG^h){X%uQ8G<@?p7Q$?d0Ha|jn8_?H zkLS+^Gh7NjC-2M^EIiES!U-2F>{iFVB+0P;{>QQ~c9z{AVM(df%WoH&I9L>4mB?^% zDjM|$Uug$T+)L&M(GNn}YV597#0)_08#itwCML!#CNv~fnd)h1KJNL}-yimAZNZ-^ zKS+iYh$|-YS)B8YeR<Y+Y6b?%tNqgZHq_cD2U*^^L8-7KvEbT@eemNo&scP@M&a1; zu{ZnuqLPxojN3R7S6wvIH`F@$`OJx=Lu3nGw)66%W>lmxF9kVkje2GLO*CGmrX@u^ z*58NY8gp?ghFiYkVqJJyZ!wBPA$--1jBqoervI!}6qO&1v!6o2ZKkJ@@kxU_7@^PF zX=x_opCuyXd<}3lz+<~b!JefyIF@0J_pal-4l1|wRW#Hg{K(tT8vIp`qsPL+uF3+z ztXg2{d%9LWgJaw<Uz(tN$BXFZ4*x?i&#(BXc=v`ec^rZmjn9gS{H?tfiq(Mau`lT| zYN$ihD&OVt_5Dc)21@dTXKbAR8pz>yn=~k|ami)sR7u#`X07S81{WL#zsi#u5+MBe z_{U<fsL$TtogHTYLqMTYEB$#NaR^dG0K#yD2Gv|ANWY6gu70KSyc8%Q97!D~Cv>)K zD8L$YO7+Q!Lcy)xSy1PWthw{2mXwwX)#D&pKKjtMHa!s*76uRlA0MBXnBKZR*voeW z`7JGhuCA`8rlwX_`C~9s`j>eciX13R{o`4dVBm3@7O(*oL7!r6|JnJ>l^%iQKAd+s zDewPP3yR2Ji@Pri9i^{K(D3H2x8(ghKt04_G{O55IIMsm6rj*^kYy)m82-}`A$aBI z=O;-Ys{o*_nlTs-yXH+W3V@A(i!Ce$Z^7ZIc+EaGGNJ*DFF_~??8?>k|HajEA@MSu z1K8G7qgQztsIr}zWWZBgR3L2zIWok~PQT94NR*Zbc%fbr;6t21vJxE=a|)M!^5ppy z$hj;8XbQX$6wqr*&#70@Xtg{IFe4CH03Lazkyn-t1;ib4@LJwPQWt1y&&y7UA!fz! zY^@H`t|rmMY&prH#~!zFdg-p-4;;u#W-}_>mPD|C{bUXP)X%d8O<ig}xdJO9ca<VV z^X=`0%8czMx{MD9c@i=rBJT86adL9nBY8ZyaeD7xy+cq$Bvj^6o)%*qJNOQa8$!D4 zU%&Wr8T&#Ygm?MG`n7)Fg?gZkMvF_lHjR1-iy_sUQhjaULiF_X)YaAh*@Np`rgMD% z#Lh8o@Qx;L#hxCLOSUflm~xdxd6jGpGrM!|tU9>5(S%iV7(St8cU4JJQj&(wwXY@A z|1zXnSswj37(dTQ)l&STD7Qh0>AIJyxaJq@N5*mfM}`llwdr3@$NbV)LZ1@0{YXvY zj(aS;Yq93t7tb%}%A$HGAK?5D-0>j>6PVA@R6OlG&+=+0D@Yt+<ULf=T|?6irc+~5 zgKps@U;md)K$(H=GnYzb%7r`JC;s-~@O?1xAcZNlB4ns#|Lex-iq~UVoL2u1`I+}V z=KlU?I}7b0Z%g#5L$K9At9?2Izgc%&zS8L&$%)8Q$>g;qQR^7_LuJOk3;UsONFvMp zEdo`}XVJY~J$Lln2^zn4LWzCoOmc|9Fg*4Dk)8eaWN+!pxzXY*g<V<SV1@(&iG0im zbmj`4Hif@dv@$%;Qp;!i4A-=8&ML^O?Df9X_OyKIi25=G?Nzd3UjE*f7G+R7#T0+q z7@MyaLs4(sm~Fva80gDBt4mTA&QWR-)86*$_vomApkRVxURqij2-o-Jfxl%_<QRyx z0>c>Z!Zszg=x7FY88tT9s7qNK{lY8s$Nz9S;5$6s+}=7A=B+I=w3O#+ZecTe%M2Y7 zRIc8E#e6<rIXkx&KycOZp2p4R4C_D6(;8m$!Fk9COihxMC{#B4lWiKnWP$ksg?Omg zGykmxoAAZ`R~k!3MFK7T?@J7uCW1Q)?&J5X+*1@$GEDxKnySzLd~P>1CJ&cQ3Rwjs zQRkFy*I>*nK#OgcL4-HVLx!AAmka(!jlVo3edcmy+Uh1IEgl<h+WAPs<qEI(nBD=3 z%o%ab1p-(+et9c3sdhw;e5aK4*kOE`Xwujlc9K^OzJg1{L*AL?(aQRAUp(cgPYn1I zXc=|`hPU93&~6QxARz{B6L?eA)x}Q{nb<&qV;OO^QT9kjJ3G#QjWp#mS^L}#wi#-j zFV!VPZqQ2Qdb@C+K2K|h$V5dSBk0-@F807VKmX_1F$Cfy`E-B3S(DhkV8n6SN4&i% zvyqI%1ikKp5YjKEQ6?=pVZVQe+qmfmFrxqVD0n3En3vYO{6-79q%>bGjInd)$WI8R zh?Ky}bhpOdd}nLOM^6e&>7%(Ez6Z?z9@`DTOwc7Jm&cSj-gb}{KhKyYD^M^bsZ9T_ zp|G9LB^#1|-dr#%udTq<F+Ow6^xD5{66XbuI5MUSC>#HuK6}LS8^tnX%jO0q!T6)g zwc2bG_d!DClRn(p$|smYfwi*i{_N^Plc{@t?Ssby5`UO9=T^39A|E5XtO+H4&?gOU z_w8E_P4%C$LX5~zE;y#w{<L3prCM+0Cq%IxZsEmizLGw(|E&d@&BFn`hb`XN`9MiA zBQ(3Fg<kV{Z;T>>L0V`tlX~#%#E40J5#{QWm<Pxp-0&}?LjvhG+lLj(adS`5@G^I$ zL_IQX!vQ)R3M$}&p6*Jj$@QCIK7qW0odKTo(RvgQPvDXH#0onJDjB`?YA(?I&)Bn~ zN6d=0>UJu{)h@Qja)Ij?xdOHbr^xS4+o`@)yl!TE<bKPc4^dt)2?z1Y8CX!vXSTrx znxPa#DZbI)#r9kIup`MKKEn2t^Qddiwu__JQm)FZEZ_CQ8+b&ZUvZ@=IPdg0;<OZE z4y~>D^*PS_$IJFWBRpiJ&N1d}ZNo0W1L{6K+O{;wAS!wo?H7uc;7c7mR_ysvEbZ;{ zi(%GqKcAbttXqJWa;x6hP7xyA_&m>j<dFEtk3-E$uR4rhh>vflgG>!zT?M0SAnE+7 z{Nx5uP0+uIWsv5%YQ)T#QsLkv@Up-s3ZnupXs&a**x#P&br};N?sgv8pmh(;Ki{mj z@4^oyL=LLu)u2z;IG9$_w(~XOsZskdDmmT#S58+K8l>q)Mj4Buc|3T}#Qqbc<mDiA zXCR<`m9*u4q{OYLdt|HAR}!E7fh9<KDtX4wZoP0Zo<z3B#-yj*vB-brFdY3<bZ-g! zqB*Lp=AdQk0m_XPS|MT^o>eD>Qrf5y-U{Pa=FTD1GqMih`o&VQOo_rIYQWJD$4BA& zZE2%QcdkN}9OzVnaHqiDVF~o<nVvvm2?UmW@J%hoszi@f*0%I+UYYK;VdL~^W$Fp$ z1=~7pUV`|OPrhfXW8oJyN3ilk8-=;%N4~$NBe0B}E$9c;TE?vTuoYkZFREjj%MlOU z%&Q$Z7Z|tql`Jju9FU+Yt93|jph(C7O$#BRM67{-J70oE*#;32krnSf`!p*_`W4?c z0;;RYftH&6T$3J0;!Ea#(&g5Do0u;M9yj<|3ZzJn?XB(b-Nepy3O0Z3pNLaDY`!Ey zS|Dx8jnl`!J3|@!sOz{j=D6Ix)UFhU$x45P6=&tV%n#9r3=Mp<{u#GQuEfag68}na zU)8|CK=48IFSCC#o3BmYnO^H-g=B21H!;<BZjzO=vH_5OuV?dPtwXtSa|2L>o7#UY zLH5y46SD|neR2cf)xV?c5Vm+>U=x_txKab5G*}|zdUSSSt_e>LSn8a7;biOot1}gi zeqb~*-?UW}l$Xb8*0hz@kv38BWw&MjcXyp<^qVQWS}v~mS58I5PL41-B@)vC3DCjl z`OK16c75q3L<;1ALPB}2bLVX+*hHZgC)Sid^{=B20VuA3FL*~Vgdim@)+xcypzlln zvNvGX6QBLfN#<fZvoLEj<9M>TNTfzy2Nn|AB~bZkcimyPPEJMELT%-5;F+uY(3+O+ ztKsM{(0SJGn4Y39Ej2wQulSmhKQ%KW2`&MF+&iS3n@su}x?BJx-m9VvOD3k2yecQ1 zE4hlR>uRS%(hfs{@;|3+xn~JHJw1e%&x-L0CIR%AkcIoUpJ~T=a@N7+f)`%qx{89e zTDe@u;dn^WWWxkIhW;i6Rh|~e+mw`)-ufo?Va{O?#2YQGCi~!prGI<b^EtouIc3(X zYIkxtM@RL)9W$^@*KS2oi;2$T>eRbPk;FwMbq#)Nj&h>T87tw%z+yt8bxOi@M7hAr zANExxap{r9c3C&lG@`M9pJ?c4*(M-;(xaBtfQ!Q$HX$D0a|~yganBuF!XZdU_x0EG z@0pXnqf~<wMk&48_RCa!#mREUdtjL_SCkV{`1hHK{>GnkG{p8hU%;i1c8iZ}v_l~l z6C1dZ9hds8rAq8z`7v3k_C#v-*lLZ=!3smEeCqn)7gC+_lD~r=^ZtmE#x#8sNuHSY zF!EXcGxIF-HHcHepgNS|z07?E$#}^u>^MUOsse3#V|v3YKLm=$7zSy_G%JvYb5T70 zrz{^*i%d}zt`l!m)eXg)$uW_xN&3Fn#f9l{#B4V&N=pa|r5*(OxUZ*`73w%e>jX!0 zcI`l)L@{1VEU<(^|3+9QQl5x9yWAxr0{sYP(r^}~q{&*Nb=TK1Aguw-5&+hxV!uvI zyQJqSCGVZDYCOAAdL||&fDnYd1?IVP?Jh{c2~3n2?Pe<YXT;Knsf|R{y0l!+(l6Jt zqV&W)e8+?mBP4p=&a>#!CEdl>KrgqqI%1%2Sf5MNsG8B3p-xXywEXbs2+RP06@XO< z>X5_++61ONEk8f(>gsBD_e;<G%!!8j`ufVsxO%_j>z!07B9cR+0DM`wMHvb-Fon&Z zo5nx}Q7e|JS7vTW&7W3<o6m*h`#?}hqbL({5N#b&&@{z^%G^7Q`BfQ{M-HeO@IDKE zMGl0Abz?WlLq<>)gWBDl_63BSQGdf*67Ie>as?!e?6`t*aXH#X@ImA+vq~z4-ypvW z7}l4m^<64jt+E@<SklnW&)$69T_c;5An7(LquFD>kWdb@eNaASNJC7c+>(aA>J}o) zzJE78f4DpWq&7FVjfGDD^*uWB%-=K;*OkHVrvYx|i?D-+A`5Bw72>h-qOgE>yBM}3 zMfO9`mYaKg0@l|qrKTLQI+d@S>iJR*>&XA*EjcQ!TT7wEQt5Cn<WpBd#z}c;DK{;V z&YL%&PSZeqsLD^!ueG0QZEam$wb9fh^P+G11{ikxAr46YXE#zz5=ckyms)yzyW9Iq z(%u1$T+x^x+tp9rd~zVMzm|Mmwr8m?r%{cWIK3~(%H)pO^rJjj@~eSLjv+)O*`QEy z@p4o>D4E(9YHMpXbl#t|xuiQFub5UZnSV|vDBV%QzlPH)pPlfdtPz4}FZXNT5Vlt~ z^w37uZbe5Xms~Uk9(TpXsh=ww3q5)AMAX~S-F`D~CXlYR=_^c*?r@^-Pv!)?JS(Ow z@{FXNh$d<O0CMNz7%ohBxEzy~&uN`>7=}YprOTBB##?YMBxvjF1^gvxz4ZqmTakeC z3XJ|tv@j9O1L!ppeZ1R<+P6axiw);bpB~wiaF7aZpRPH;(SUPZnZvk@J+02dA~A6E zIFtEyu~S3%E8XPlrFr~3EcFIjoGgV2_Y#wmCN#)}e_>)X-Ll|>ZoATo)cF&C{EYdY z3t+n_hjYq|=v#bM7=^ned77eVU;HGQGsNNHxvGWV5vXyjiBBA!GX&H%cy5jBZ2Du7 zsPUOGaj_>ClZ%Zlm+cJWV-r)gr3vhLgGZPyvW=qRKlvPtIfvEtpw^>O<K@;;^uMg{ zBTP&Q36nfO+Z6c7@gQV(v$x}GAzw`YQ#Q1c&UXCAmE((1CAxhGj`KyrXPuIWR7HBR z#56^0btrv)P;P3?v9W?w<LlQPECv+6jH?T4g{^i2ZoX5(1MxP?JF_5xUvUot!4YME z^8##A)V1dEgJ}360_8KHjksC(yJ&o)tobNJ&LgLx;q)N28o2Biv7T>!G0C>e#!6S} zKxQW=XLM2D3Dp%lJK19B#U%N~|3Zx7zZ=0^JtU3E<y3PzR2oBL=mspB@*K8e7TxJK zHn+9_%>r=MBQ2-SZ>$Wai%IeGgLp$8t0N*($7ND$_;*UQfBy`A@jjldp{uitmJN_= z$^9En;Vs`iT+Gd7@7W(7VnB;BHadE;c64T}$>NoPy!<Ux>ic9FNMU%Ih*L?EuaEsS z6rNfpzMFom_o`dC$ox(BpTUm=Dy;EOR5YK&60~yt1Mq>O^s12y6)~&SE$qAYt{$|q zhA2teKLOuUxTYmYN*ecRu|yYcC})Vko(u(^)@5O3xR_(ir7w=R8Bjifz43v^8){A4 z%ObiVK7YI8E>E{bT6TC*_b<PPg_U@tu&A?B1IO8>LZ*~vA>pS-Ls^%hWKzux7Yr>@ z!s4e>LxBr&n2%_`O9pPerUS~BCNvz2Zr*rrG%Aagllx|A-qk6My}UT+eOyLDl4s%T zyBs%53`OIiP8KjuWR-HCcN>y)$6n<Lv_oSrN3B~h%O-y2{A*H2$4n+C>&Z@+J#>sv z#wxQz7}{s0(HJu=0(|c70JS$qO_;;?Q$@`V{|oCC+y`Q0+?AgQ2?-DWi434KD*?bE z*WO#B0WR>GFkh>S<DURM_-`~&ES#R!2M~beprE!2Xtz3;Ih*cmppG%&Jw4oX$mt}m zXodE+*u|^#!SmOV3KoN=j9V{S2ZXzb??v8;Z>iW+X)h7w0SlpZvoJ^K227`AoUfk^ z94E;@UpkvspaIRi`EWy+lVuu{I3h4g=5>BH2pS7{4xmqaHX)>A1erQYh94yknm5-q zEuS9Mda0<YJl*YB42|&&1SCm2lZGZ3OkR$Nb`mpQ4n4^<k-VT`@Wv#?I56ch)t&!+ zVLg~WTj6u~`HKm#%T0jh&9Obb3=Z@)MUXl9_5qmAFhhCp>?|`YaMz@THdbeDufe3+ zVS?uxZ+H^r*+4pGnq?j!!BSFEWMpIj48Ez$m;(dH$<pI$ltO~{&Dn1UvyhSioQQ)G z;L!hKx&b@MSKfX|7NQ5U+uAU}hDcxTODpD&6!>1+9u_4gn*@yKN7K=LsZ(H#nOGJ= zG*y}J*Fv0pp$1Csvki9{E`Fm=0jIHuU*)jWDe(NLaXa$?9rj!;aJ`C7hg~*vrg4S$ za-A0q57aS-j$WR2h_~~}CuENucZ`8U1HbQ<H*Ie@L3LDApjTx%|3xSR_IUMp@~Bh- zv<n1M`;DYq&M(gf3XZm<C#O6tcK;+jOa~Il%s?XGiw<a8CzDLo0{OpWfk!$bGZn{B zkX`+XOd=$XJ==MKYQ|o4&0z-(q>tYdBgTb(P}da|of4Rwjl?=)(HrG~+D0mOP1tef zWn7GOIF=g9hR!wR<d_o^DxbI*k&u*HK2ptZlh2k<`0TE)@v2ajtA;T03M~K_zbNg5 zK9XS54Zl_98%*ohW7Odw1~BVTP>dQw;9g<YET((A3-d-USHjaC`w?}y2ozXk3^C4Z z$1Jq!6KP%<k*7rIm&wjzJ$L{;B0!au3(OtoBTxrLh?IQNI2}Vw?+bdLRd8NoSO>lr z_;fE01+o85>?tS|->}dC*rON3rG}MzYklV7O?!zV;bCuI@}~-`K&D2>`=%+S&wj%f z7M_#17&)0bKNOIHnN3A>cCvO=%EZ82+O6Wqq@>vB0AXcuC+;I=2J`NRk<%%xj#s}1 z0Q+4oWld*k*zFc{X4d)2@$qF@ap~JP_&A2_pmqX8Yete$|7*pHl;Wu5o>#95vW#!l zK16(2V7#rGWQvRG2-4C?UaoDSynUr%g2KX{?(S3}0Lw7}pyK?=2|TDhTcmjxQQo*r zjz_fVXMy>6d9lTyJfJ*#G}_qycE#)IK_!dQbj?KD`a|wN*gsga)Dh^Jt|;fn^Y~<R z(+r>1PM`N^!g%q-^pTo;$qhVWK@lT{&0m83z&M44gnI<)n6wlDsm3iI)6ug?EPXaV zwB3B!JuIN4Z?0<TZDz)pxMz<nDJ!F3n-Te)>^Yjxkz?s^W+?YF`;$g5X6ZQGk$bvO z7iK+_8~gOA!bV*&IyM%xNFqS*siLMX`J2u))93n?X7Zq9>skbPTV7e-K~ofS{NFd; zg(J4RP~e5~O!eM?QUc)8@AkjK*z?jJ(0ut~0<+wRfs{j+yjn1&CQpyA4taTxlOqI0 z>K-Vx#jYp<Vy;8sw@k<HPW`Gyq;c!(c71~KoS%DA=evBl%>W?_ImNJ_NQy#uogH*t z9%eYr`3(hz2hE)&5gP^$B*seBVzp4Z&TH1~pSF-Z$hpeXWMV0oE;{My`u((p1vlyA zwz^-2l>z6SHYG9;YHDhAeO|}1lMc40y55Ghzm#=xete?s(ra+HEi7iIY~%Hy(Aa+W zd53Z)*!rp?*eV})K3++&D^>VP%fw>|R;jx*KVOgC4)k}fkdE)MC?ESWP+5~?!z{jN z_rwhJ@7zR00pscf-fbf<Zz(+P>w7vOa9Nlj2@6DpK0Vl7mLF$0p3j2sJD`oKzHQm8 zEVREM^663bq@<Z}3TWqc5*0Zp58P{c+HBii=Q~7nH+Hb;ytEuT3vnuq=z^MbJzmuR ztGm+fH4Brbh_)>6$w}DrFNaw^Nge|*hPT+&A?<gOW)>C}mlx3DH>DuRS8o-R+nVVm z**?fg*xhtWN~8cM<ld~Wp@i#r4c?H&7AkIR`=J9y%bSHrZ`*PgYF)*)g3khs6ML3@ zE8Vg50mqb#7vRKD^JO!G;RKvn_uFc|k&jhvGU?iy*zRxH7Q+_DtuATL{r>)Mw)&Bj zbsqL}Z}g~pEIRncdJ~IfvOm2%Ocf{yT+07aMMGRRk|%@-0yw|7-anHiQ04%~TF%xo z>Y&REmurV2EvH3~%N<M_W?a%2;<B;ZmzPtG=ws1{jcfBxRv2GIA;zrdv!Qdmp-bEB zJT5LWh}2i*`ZZGgY-Z;7&sI-J6!Du=mP1_7&7g*#sw(wdHxgfShk6ABmWVNYHsq@s zVrxNn?bLtCXoh;RWjLbXsZ*wf-#1axtUNsGtnt?>f)Xz1>q?uNN=iy(^4=6DD7uB6 z<jW)A`xjjswVSgI4x_&xF;P|g{CP$v;U!$-J_8+Ee4D*st(O<ELQCZ5rkVTP$p&qM zY7EqoHP2PvUF>MRdFReP!XD{`{R4Rs`vAg6MNK9jckP{^ECtRtW(-ZLP!7EEWS{pb zKefv3@E*x7vGkbf>9nw&FbC07oBwhFQc@-f&L1!a-IKw86qwQ~!=CceT$i0F>;=Bj z-i)J{thp@Id_@B7-{3d2-4S#%QDbUmM8i;D-&n&1BjrU%%NMpHrfxw%+3&ZT#mkY{ zTL*Z4xx+CJ#C-R(0k2r^lZkFcz(7o3%VB3lcDrcI^2!ROft>;ogY(wehL5qL{1es) z&w|kNng^U5g$&-sYWI`<Pp3j}hlj&pLU=^<u8mUkja>+}as-QfD1HCb<w6&F%>x&g z5J%9?!P{HT_p-ku=-a1HzkS@wB|Oh2m+A0kwv%x!h{o342L@D5PEPih2Ti=&Rb1-{ zBP;I7e5btq;TZ>usE3%$s^AanEU57BCl7-L!w0Oc$b*1D`ldLs$N9oEh=)u4W+>gD z#_~s1(1|)!>!V8a2sZ2iwKNxb;!;xYpsInr{pMW}uBN4YBqLMqwRak%bmWV!$c>Hl zm<)C)XLf0^Cjq~|K4$#Mglk2_pP~g58W*egVT(}0fA53nZ(1qC;~&8zmAQSbaTb2r zO5e8F^e5s`D)kNjQCoQhh2pVOS0!5)Tc5XY-+Jp#a*Lpf#*4lf&?s106L-`0yJ;1^ zMohd}rlgnhJnPs#2cqhg@@DtiKgy8sa*%e(VP_Fz5f|qd=j36@%gU{WHRLb9o?B>Y zKe-!y$icy($E3i*%&PtYK{`lg{&f`3%GE5Hz>vpqk|2AIE%~FyVPrr(fDU5lrPkfW zJ;dp{T&z*qYGgMnrgqxzoNENVQ$QBHx$%&&oSyMMGZ%#tAj{+lK7LG)%p|uev+w=2 z{&v&cvo4Z;_FJ7_0l^3kFRQF?+j1=)Pg`A0=?;5Y`espyh`;l}mj_CUsx@_W5Lnue zA3-Urx|u8Z*NS1ewY3dZMb~kD{$%{KNUE&BboPTM`yoES6pYKeO!?B3AHmnP6tkP# zJ<q(=!LET*ORDRX)YRj@Cw~tQJN~Bm+1`HZxnaxO0Y7w$fIr}&wpY;-;o;nPn6&pf z`it-~J?9x#I<s-D)5MMT*1Lw&hLDh}6Dpm$y1L%(T2WK9z5BEQ@5P;R@Q0tY;+auB zYa}muBgrQBJ^2Z)J~x$0%WCptHnxJMrWqSSz~P-S<vTETf8Mhd(lGrr(d*f@=Nv&V zZY~pV-x>+cydwk6c!)o8nMZ^F$UQt>Lu+9HZ0cL%@>Z(c1XsyhNAV-VGB4%4y=`p1 zvxg;U#N>|LUyFKgTt2!ns;QxI?cJiN;`94z+@jqshGu5hpCLHL<w@N2w<@xknv8k# z5?0u6ghfPwzfa<Vf@6V4zX}}d{6@YNf<1o4P?WI9l~nb4WMoFR9yFfr%X57G-rXG* z)*Tgft@YiC4+n0)@uQ&E=+93pZ#}<Pc})Gt{lS$VusJR&bRpsjdVMc$P);;PqTbs? zjg2+x8cV{SR>KyGC?G3fR?SY;D~)NZdtVYKY*UK>=k*;=KWkgPVD=w057vD$H48;? zOmVtm7`Vkf3AM7%M@Pu-7&quXr2DXPEWkkW$UO#MOuK;T+)UAHH+h4fExzaG^M^t6 zw7U4&qiv6Z@NeB!zrV4*j{k^+ge0Y1W>u#BQ4o~;(Y$-Z1-cYr4BdzBWKv1mFRyOe zCnI5(^VvU}U^j488eDeE9e-TtaORP=Fd97+4chs$vCDzC&el{tYE|tKmh+Nh+@>aB zk=9e<hShJH8$!RVlY^$$YEL)j_WOxP>JEGF$&@1u)Yabt8a%5eI+&uw8UN8bIn)XN z5i4*Z_}TQP*NcgB?*laG2>h=q53K_o9RuW5aurIum6hnFT*tyn)KjTHYH{`(8$C;D zZCFj|+dA@XW(!4;J`8fEWlQeiU|HGC-F}=c9-GDynYY<!Y0|v6=ZC|B^TerXw@AO# z=dvAxbjWaO+0FX<_gkb%=&zJj_j<YKH*vtJ>!k<T>=CL3`(xB(*RBU?a3=TVsdqjF z$KdbXj9Izw`fKTp?AYVmTz`qlbieByG~ax~;LC~s2zdCFXZIh0Jks+Rw%BE|<Sr`5 zO8QWecXZ>fZ!>Pe4;kp~N2YZWBKG<QZ}hqDF0$Md%@m;?I6uuhQhG&j`}X@RO+h() zclXJ(G#YJ1ZEx?6dTTrUYjw!;U!T=3&?tsZw&2u(bJzn1ISVaWD0F!D&qRn_`F#EH z6tQBPhC0l1me|uEROg=65L~#_<N>y4A_A?$P1a;Aqk7lw4X3uP?b_NoUQkGgh9XYu zk}E*yb7nLsD3keo_73jF-<DPY-W&1uI>FUalIhfM)+<xcQ!Jypspc7}zoSWRIz4Y& z@RB~0DgB%7IJ`)b+AV0OXBg$9b+md5!A&c;dPi{(HuCY_B+L(Ma(RKcg&1huKf(ic zBxVKHQ9OQnqVxD?t~*o6UB3keUV;x_wTc3Cn>Bdiui4;S*KF2&{Rxw7eVHf3VPAR} z7dL#ap#jwslleMpcaHI65Gaqox9(1mgP+rl|NfiqXv(m;lcD>5>-&!!EFT`ya`#>i z1<p0CElYGpXUz9`f;Ftm12Y5r@_Xrf-*hnk;al3JpR!{l3(FFYjW4pbYU>Q#_5m@( z_37Ec7bf`^S$@mt)YKc$j69MLLGK=3XL;6Lm?*+=t6aw(0)ZtY-M;b!i_h+>MYDXn zkALksj?CA-&lw3a=9(V|*7hdprlxXbWX2rp1Z5&J^bHOQzU^w};M3x578EBd;_W#d z1aAet<{(7<%s0AsJtHG#roEn{Z~-tymzGX*a;_JT+kw-C9<0n`-^<D>r7pUi1O#xy zb8H-(ic5-g8yyb|l#=SDw!7$k7+J+5?)`L;R1evgjBQ`NC$oCJ?Iwjxo1DH5m9_JW zs|61|Yh5Id#y$UAoDYHN*CB=Y{rU%uknpp;{h2fKFT&+YFJF$?!J@9^&3pRU*@38+ zwsoYcSbtCRrJ>=6H^o+jB@;*MX+^!9k%JbFwzeFc;yD_HLJw37?P<HzalRK8I@sG+ z)YiGsTJupYE|PkPbDKZJm(kQz^6@FuE8qNk)8Iwd!r9`3XjWG7{H~vmWuD#Z07Y^` z!?fy)0@Y$-0)p7c=o>hlty`Wt2rV&M&*s5<<mBAkuIrwDy}e3Q=Gn^?pYB!HBpCrx zJybR$Dp~$TmyKW=j~JKC*TTYu*0%Fk;kTI-qOP0PhwgJ0b6HhaDm}^XG8R6L$X@J> zvLu{wbh5T)d(@$Ef*LUJn)P*q{x7P&G9ap`TYCV(A*7{SK|;E_K|yLrB?am3?vj@7 zZjev}q)`MEq`SLQ>HhZM``-KA^Fw7EPVBSx+G{`SS<kY1HCjqYa)TH^r7Xu<Py7Q+ zS4=!DC~{PPzhXqE1cSnh7s{n-{+JKp<}WUp7LoyzgVN)N7yOW_eFJKfR8?*+`sW0T z6f;v}v#6nl=C7wJb!RZw$@m<O&B|*7^SPM0-MhY|)|l#Ba+w`rQeN&x#FUpaN=iEM zhdMAZmg}%g6rjfMf4ylfDyq|Msy~=(DnHyDN}EOrmPo=T1mRI)+Z!+W7w<o{<U5{B zA86iG!zxzQ(ExT~>9RY`?)>*lFz<DlXI%hnbN5TtLbXYEe?{mKI!XG>_YMgTP-QmI z<n#Uf4@|&n>$9!P-aNa<XiC%m*8u}Ih?sHVntoOD;vY8xohG~K(Nu?q$dyIqeSg;l z-)9y2)4Hy+<zp@i2B?TNT0K~6lx;rqzu$JqjE-wq&wni~Ep6?dex-t4U@YvG#hd<! zcnb;mZF`mIj-{_uwv%hcg@pj|Pw86+#dpywh~$_Oo3J2fGBCEF5H0Wi07Z6U-2rgl zzS!Z1kU{27NQ@(5WtJpfHS@_@)=?D+;?^>sk>RZ*WbfIZE2)V=!DiI+<BQK9<Y3vm z!_YRH`7iS;ZxxekDk{dd9B>fx7LD`7P?5e9;i1Q@ik>f0*6+=xreHgsL^2?8z#NZa zMDCW`7TnJk^jINAm`xo^)ktiRWtRqe9OR)H??Wo;#cIK1ubU~bv#qw+D?0zWnX39M zP2sYtRon3_u<d5OsY;kayh;;9+%2By$zClVN9JEb-J0j0NC$hKu361CvWkjMELGzn zQef3+$+=kT8*`ChpctL~Wonx9p?4L<dkHiAyXY?oi&I7T`Y3%I0JIsoC@(z7K+4>u z5keNJd+dyNRWdw048Wx2WyAD7f=w;pcj-UEV}lnHiqb0hc5!ckEkbV$JzBnz88RX( zd4{TEr|7T@^po5yM^Mtz7GY2bsg<f}OsLJdbSnh4Un=Z?l!u7P=-0#Q_qMG!e`nSf z^aVG1IwNexEWTLO%2a5a9%TQy-0_VF3p>b8DjfmAw1uAMO?LUXl~6*m_G|O7hzNbV zx$gx9tD?d;8$9KE<Jvaq{5K?^DocYY)z>i0kG{GC*<Ihe<*5M6CD^YMur~4w2#gyt zcplvNvR>GnY;%{RJZh%le8Ma&T%&LC>d&XW@w)M^lV(-j+PdnwDon3Xq6)tckIv%b z8kX3aJ!fV@#wDaoo)q>+hA8w!bw?9Q2k)p}olI<T{rXkvka@A9)Y^6xebu-Af#rrj z`iNP2VQ{d~b)m9rypAbEfjw11ku#X^(p^gBX%IAkDm(>G3VMP8pW2T=coF|gO6A9A zi~DhOKnsQysv+31)NA8QXKgI6;N|BTsxhY02T)ZP{5X%DdZJmO7ay;=XgAr>A!%f^ z;w<3pS<_+0Mf>#WmNN-m0F~G~snO;J-k)m;VK8^i(ci>1ZP$jcRUI95y{#P_B;3A& zp(yIuw|8+036*_H+x1m=7-L8D<(->Q*7ddIh3xYYx|j@MPalBo7!9`hZvQ5BtlOK+ z)VKSRL&eI@&d({#Icmic@-vYCAqp0SL5;JTx+ccY-)tlpGp>8(73IG+vxldLz330@ z3>~!8-f;~CazO?|*-I+R=!IYj2}G25c)dq#PMdbY5=;9fLJ6K!)YQM)YGBE6j%ICh zOAr*r;u|}k{o%v~!H@gIBSqbvT@fkm+mGZ5<FLOU2`Qe2V+)YKGN8uiqM@Ndc|;<` z@aSr*@!(%{^KA#PA8jBxt7<)B_|g%D90ztb;!8^*A=7zzG!-&q=B_T&f(R1QDhV*f zNOl!`Z~0q$GW(|B@Sa9cdqhAk=y81>3&a&BgP<z!@~GsziQ;Y8RNe(lcJ%lK`E_i8 z(n(0yvf)Y}b?)JcUw7oO|BY(=-@h@L^!!WJf)WzHUxtOrhDRSv*BqVB*i=+i3BOU} zqH<IQ6D^L`vW$^Az|GjR>EtlCnSG_}Y4Pz6Y(x?|xfczp5D6$*`}%v9`C4;x6R`z+ zjt7w*b}EV9U8j|A^u8-rOvwBGh|>G7xo7dr%HvE8Qa@gP?yWA;%51@n#(l?*{G<4^ zG!O<xfn88By3=!Lt~@>nRMAKr8iYVLZcv2^m{&<mwe$3}pcTYmkkl>9h^9tOE2--G zw2*KCp*p68Lr_?_gN$T+c)Clar$43G+`{5@%lZ}Jf?G{tk=1534gk4uaE1z#<PbNS zxhN4Jn_rx{xOo}IIHF{F@Ayn&*p*919U3Iw{eD{?k=K6o85nE*!7PAFIcAt_xw(1w z=Q2gFw%tLDXTHULg+-Pk;5?q&90>V5IXSsmlCqEb@f35Xj(jF>g$9H@*<RZ*G?2nC zbJJxiOlSLgnlKMAVLK)kwj5$8_&FFk(qLsn778+|ceklX4>2^#9|u6bn(0-66%y1v z_(vmBJH6}xQOCVuBoj*HbeOQRuvnN|M<~1++B!~<u=Kp@IY@3jz7)*we)n9*&B&xH z2kePl7d&?sb35b`i>{tvQym&{j}-Oo=(*pVd?`rTp}6eJ5to#ZkWk8?zclF#N%-NY z$D4}4rcAHy(K5eBkf9iAJFFzp4tRZYHgaLGLYjoXLVL+^MPnuYBWr}KW8)e~hYUBc z{Jgvs!#vl$nT6?L9jA{_KTHZPDLSN>;+>&w3rnC=k`+htB$+`+k<bVOO23L$-&@E{ zYcp5pzUYUcKV4o|=CWQh@h*l)0jffuXpxw<B8HD(N-GV`jX(un1s)zibqT=VkV2|( zyWPp5J;6F>Cjk?MbbK6Q1?ZPAUq0%bc$>X9x%~3FrYf}x*7F#R>VeeV%Pb<JNCYKe ziWqt{*q_+3=_D{G$4s($jJtlPI!Z`HCP+9s1}@rV$gAyk5BmAZRG_k_O4lo^s?N>Z zL`6nYJ>bJ+Wb9=uw}1`?oj(ZH5r=|UzOblhl$(n}B*Sz0$I#~a;1{2*=136VO@4g~ z<XRN$?NP-}z&^Hd0JWdUYW_3MYB<1hLVd10gC)KZVM_=IS}%BA8SAd>Kau^bJ!2&k zc-k$T_Q*rhtmO4{GW#ptT#;Z{4nEjND$<8jopSXP=C*(T?~C^)qYYwTubx2`udb`L znQivzO;SLXjV5tQd(F8aBQ3*;q8C#<Ih-@09Gy?Lrekvuo8Jl2S)7~d6aesK|L|+o zqV`RBJf)~lUR|X|#W=)oV#i*9Tu7E)me_BG8C1bO*S^i<mr&~G@%w<im5}P%kdX%k zlaA9-+0bxuDd2lwIxi81OguwDQPJCzw8v|SkBnq7$+_&_!%DHG*@#{RqF_#cvRs(= zZXzY!t_i_HMMrlO$OK!iU}@TqdQD}e4MRjcU-acH&i(`jP=NrMGH)fs)YMohgYBh< zMUdYFlgqS#Mx~Zx-q-V=u>tl*vKOa@kxR>NB?r_Ghysg>x}L~lZmjP!v+&D6QL!R+ z7kt})#=@5szGy>CbX@^9|6pad*zw0*ov(q}P#@RO<A3fcOY@}sDvvK+KclLyIx}iT zniA`_(^nr!!5t0zPGnZwXdeH$-e#f2YVGrQ9S>r)<C$Xe%$jItlU~(9@~l$11_iUg z>G(|5Guo%JQc^nH<`#w$xP{u9A9eKf^(iedb!%L#6}-LON>un}aJ^9iRai8oI=AP~ z&&qv0uD1$)1ih84@u@x6yz|<txbwb#t0%5!Wo5OWI~(_zBRTo`*Zq)$RUb$@xRdE0 zLGOYk^wgY2xBmXlbY!NMs?eaNeHxnfO3=loeg+gdXEMkCY9fn;P5LeZbQqb(Y7?p- zg=$0<KNR%iM}ZHj^P|4R#5J2U`T#r~T`73*{fE2F42qE4n;2hn)#uw*Z9Oy1qK&t# z>T{I-tg6w%<lVccNo0BQ5AHpzOH)f;UH~>OC1n*X^C%J*KK&Xa*}JTGw2d!(AoTq9 zpY65AKQ`H9r*RuP$y`FBWSEWZe&^;!5YDOT>N?sjj0awwB68jBFSTKjax9N)7bE%6 zheslG$1XkjT&$3K{YREnU0n@fRG-`U&(GKuI>n4e(}mVOZOv6OKIov^%${%aT2i{` z)o#3Slcq<D-d2vDcy&Dm3Z#C9x%NFxL9wX`7UkbPUj53lo=onewAF$Kf#2-rYS9@A zS({J${}$dMLy<~i<jHQPMepp!3e4pS|8_fLQFCxKR!*U+KSZDszIkNsQ~(gM-c{Kc zD+E9G{Ok?)R{0xfVYSU1BgCx4zNqLlnNiuttD>C@Hz|Rr&AcPdTCh%+-v$@2^Y3yd zWbwo$U8Z~|V)k1`_smC)KUVyLqX6OK0Yn^*)V%zB@1wJP^iXU&fy^D=k04D>>alB3 zO7DLBX$FYD`7hj@{wgV<|NS;s9)bVJg9a>bp`C{OPMg?}cgDt@KYlzAAPc(Wi+Pxq zzSup&5~N&(BmF+w-!CJ(PWzih%kFq2p7fA~^^x1Gk5a~A^Q%%dhD5p2)9rUm2r(1d zK*J+Fd8Jwq1(OV)GPy*VJ}sS;p-?9OB}=lr<lgkhalB{Ya&q!+S(H3ZQecBwsAHts zF_a$+1S8A=qIFr=yA~deSw6wE>Eu!S1{tDQ+&evw7Fo*uCR1ZW&_>sLFIVy*K(XkQ zveHsMP=ybF-0SZux?Eny13V#nd;6QK;`B(92Ay@virzIF+xqZV5Gt9#m$@I1{ATvd z5313?4uO;CRcp{h=`*FA(uS9Z9{kaqB$9m5M+=P5fU)WI6A$_|43sBEd1KREDJ5gu zg@59N4I{zX-GnHzlDAGmdYSLw)7F6NCQrW-sC`9r=1uH`Vo};~k$AE;=B&C%>N4vL ziW8DOe2BH5E&S~2qwqn?#;2?OD@h>N5uNu)AVI=S%HVcUkLn&7Oh`%sm2rq-8X6ip zIyyNW;w3Tg#g-6CSIr`F*N{rvD()zJ3Q!2YpJD>~iKi#o{jawr8Q5GbicA-T28WLa z=3I}1hm1|k7NP?nt(>c>u^+O@)|>{)uT&Z-ORl1Yvp!sPPpym`#D1Ar<KJ?wdRl$D zH>0VmTQOt?wj*#*PMR=*#R)M227<E21Aplz7n@`5{tWxQcK;42b(DfLrPpDR?_IL~ zt-<_k=di;=`uG1u!cLqW8gz1%*}7Qjdp+Ru^^&$<xSZx>H*S2ZwdxCoeloRmYxCHa zi;ynSoU4%TQ{hwo<gV<o;@<J>li}Q>B3LP3v?@J5-_(w@7?g}ZZYR6NV>7qd_Ur8g z1H;Q8G3|8k>%@`$Hrv(?=f48;r+cI_QDoU4K9J?O;Rv9Ll7ql3H#Zl=DIlhDQ%AdB zuuo{i!oqGYPvjL8#>^B2)$mU8?rzVx7xuJmT78GU&*0(4&Dz*Ch~=v&rF!-0-<?KR zb(E>=YiJ}bUE9zvR!1Y{=H#j|0<;*B9Qj|^(*kPWjZI8Y1j2ifsjKZr7#HSZ^Sipc z=+4n7q;a6ZyB|N{(Y;z|1cfbbLI50jVuUgAUdliC@qc0nI5PF3a#8ZtdNzv*U^5SG z8XDnwxe8P8DrQF^!m)|_JBNSm=iXnS9_6H8+ht-|_3`=HT;i`aEuY~3`=tQ(pFJcS zO`eB~mE+@dS$Ds-vpYMbXX+}BPS{B#Jzg2$Ad)@gApW^Vl^pmAh0K+o<usfHGXn`d z*MR2n{_+?WJg}i2?S7y4ULISI>umPs=k1MP!OWfaowHui*OR2Hj*fgt3KcY+x2knx z9=Ww2sBSpQIgk|o@WrMsq=No7*`a3^{^U5(Wz>xhrIti7#Ucg>S%L)@iM_ev=ni}> z90WcH&=w)sq@?9)j7R_~xPQu)d~kgJ46B*|0#JX2vI_OIBE?-!&%^LcY^ECDSh!#9 z5!PGXE`<cwtA1Zw+tY5`<-EJa>r#4z74z3N7k_&C<3mRm@8h96>sdDok#pPDTO!E% zK-1QK6XpJbZ;lR2HyOdU^aE2vNqQ|cHMP~%Rfw323rP7p-A}Lkgi50lsz3H+uoO<7 z5f#-&@t#KY8+$8siIzzAL5pu@m(p!IfLKXOODmIa|KmLlN?`6;9=e<jAO(F#W|lRI z+?%d7A6jNG-Q&MXBfUK?CR|xjVpe^@095h3FQ=NS+HPmJPJy<H=Pjje6s73i@b9-5 zJ{qjwCOvQpu_<s;M86V3lgw!FxtjH==)?2q(E_mHhyZ+iwz+NU(_z3HZOX^zpohUY zEJlOs?~YDVn;sDhE$4=2{Ta~1LA16W=bgX4BL1uiQP$SJ*vxm`oo`XcA9yCcD$n+E z#NlG$<Y48l%F`$YOCHKbCqRao-nZ`3gaBc+cn~X+5FuGo72mst7K{j~{z&qlS;LqZ z4#@YgU-mjVMh(x|P=p5bz54>HBzTy$Z`VNKJE)<&5sdszA?n@L{F<{0fM|!TFU~w# zNO>zRSL5rwnQ@kez$}P)A9?B_?0IH0@N0^>mY18i`S3elYHDCd$CJhaT2%k8u8<bb zdPHmp0S>XST{~N%9POASYJfp)9SCtz5kYlZ^^73c8gP@4Qx~+%m5mSs1QwfwWc!Ua zcm}qXZ=>crP*l_tw^<-ZgA(|ut}YGRkSBD@3FR(uTELpNQ#x4oubg}|5G#?A>6GdI zG0=MH0kn1hxS^vtFf#ShuRTy9;&ln1vf7_HdHoHl6o`XHLQPF=Zf*|5%U%`z!>&LY zrz$KaW`nQajH_&HTddom!2kp5Ka~Hhu%LkTjoVdmcFWP2CgqFt{nnk>V07Dv7$uEz z_1h^X36P`nnHjHbO?rR$lTgpiv)zySO@$tQCoyNG?x*XBv%9m#FWha5i}Nir8~wV2 z!v5;2s_qBgyf0i28jqG%3vL+`(I9-7(d2@S)>oIF`#zI9Csqrsx09c~A0LZ@iLhe) zxv*Cxo6|FQip=5q{L>>+4G-i4zu(lK>|b`rJnm0(ww5%SYi$wa;C#rcd->X<YY_l^ z7PTJcH%t=kD}Q{BPGqxvFXm&v3*8xXMKv(e(RJ-PRBBzUc^<Sda#!lclM<M8*|R2- z+`|K-7S0Y19MlgcOvUdfuy@!ugF!PcvhF2^8NztAxA?@wk4Z^ueeV?YufV1bl$J04 z5j#W3OiVQ1;rVzxce6s7-L_r!T4><qR`pVcOqu%j=fBGe>5Bckw{ycv8-3b?Cn^xB zAy)q5m2^cYaG>)`NTHZ}Ake0uOdd6{2K;*#W?`Ss$FI9%D5u(Ny9cSKzzZYg@e=j9 z{&If)E_$W6M`?9^KPoz`yzDcdZZt?JYirlMDO_(*-NK9xoNIEuxhQ?uB)fd#0r5k? zyZNimmvAy1RQwH!j2FGa<{5lH_bw^2HrPEMW9LBuTwbi_B|&c_tYl|XTUV>CT{r_w zo09XCF*P!tLcHgqpvXiq)gh|t?$3(sM~#`(1uciaq4C#u|K$SYh(R;>?2k8No827= zD_+<9Ua^nJQ&vmJ=3hiB?rpH1X{45vv6=odZBeVCmT-C?I{1!b^MpjsT)}!Q_K7?2 z=49PCa!wvbT4rBirF1@}pP04B&!dSbD7cOPa=(7PQt)iYb6WH$QIe2U!{9%w2f!2= z<os9KeR&+CSNJfS=P4YtXav>M@E;Rf4yIpQCK4eP#zllj{;73q(M#?v9WooB#ZB(@ zJ~nHm3NI{tmX@|1L(x~9kj`b^^#hfY$Lcx@0q=pW%FxpE^!C^D0_&Lx(*mrdU+-*U zJQv$@H3u3iXJN1pzEGmglaP{4x5VvLJ2i=%{EngF=~cl&VOLQO(|#IDPfNvqy})ll zj=IR`z@qNg9`Hvpxw)C_%U@x5&!SgXcmLEL%(u`A(Rdu@ED2=xtXyodtd81!0EpA+ zDNl>uMm7ZZ0dNf7u8+|Q?_O=2t~>D83e2^f30ZKY_Vz?*r;*|<$FJbQq*Rp4M_uPG z?3|otQ`qCnEE&PtU_W=6=-zN86O{(4YB{MK-|N|;J&JlC8O~1h3gmbDrj0eD(=+0R zv$5r%Ts%NQP5lZpA=BGfN9R=nFC9Mv;KLdrGHDt7OPIlDL1W~Lm6JDb-iX|uS8(qn zZBU9Vww<gf-5&MTQz~R~09`3*PSgHa8K_*-Ve{_^>ydRzG&BcXycMsPEONQ6t#1!n z_gY0j>PxQhw=dl}=b)Jr-m_O5?!?p|Vnqgb{vaaA@yl%`YAPlIA^vth1j$*OdRVDe z^QK}<H*GQ#Uvx}_{3;&{od635P1?%&!+qZjb~=!~T=Ea{n|lk&sp6w?Nl9(lLZFCj zC=UE4RRO5?n=F=OpfV5+BOxcRP^O2&_y5-94wD)n?HL4);9z?)!cDtNJ_(Sq0eOoX zEZ?vBpL>RlC;_#?tCO%-)60HfE@sxgyCXMuiJ3VH5ay1V$>|$b`rY+?YiT);&}!c> zg-bZsu9a7bUY@Vidyx|`H}8?<8)87IVF8hVBGU8N;oQE+onhqZi3;Q6NPh`x&Fe|= zG`bWRFxxJ%H@(B6oNIQI+S=)bGjyscLk54z@sV-voe>EM2^t!jluAPH8i3<W7ZC2v zDc3+5M9(moAs2~kB+*;8jGgzKPV=pIw>v<6y~uIvqQ8lCM!@>@ldekLHp3IF1*ZeM zn@4yDZFhf0M=?S>S{>Fjn_O*Ats4#PG+a&-H(GMCeO*eg_9z^cif5j>wP1Gt@{gjy zBc`hiz`GwuiWph!2kq**IPd{fURZW%VkiutS|pr9kJ_LHVCmWJSV0(P2nP8tNRM0$ ztNv~9UD1Gu=SI!IJb#B#>&ZC!9O)yE6{DCNW)6<iI5+vQ1)9L!RbTo|U}U*D)PTCi zK@2$*x?R8L6Xmu_<|-@872Z7*NNSCQnVFjdSFL$N3TV)=7e_X)<VmTLnIL}HZtGi` zZI|1=`p+-23!A@Q1^!NLF&o`Py*jnluiHHuAvgQ^Ar@=)E;RN@)heGysk-1Vx2%<O zd1gi@mP0P0+?=;O=|DTfJqrsJ6&0YL_oI7~78LZLK?iJcKwdD%Xseef@aCb5&(&Uc zYtq8aekq_dO_CYLnOG!WN=Zr0x^Hwa+zjh`?%4(h^SAaNt<M1wfUzevRX~9zoy&e5 zc<*$cyK3hhfjdczpJW^!zYBnm#3JL1+-NxJ#dCgXBv+$<vBK9Dz=#s*ueL$k$F*S0 zD&dDK=R2FDW;tr{sPI{Rmb`z;U58xbuC)1t<kf(eZ)3~gKrrg_pn#A!6^e?AD3FXv zT<~U;J9g%wC+uz4u0Zq=Kv2;uFOr22wuOVlJe#=h4#oB5t#p>7A`Z~F(;GkH(ue1$ zF;<cBj=}V{v)j`|4>W<-kN@Qs^AN=A1T4)!6ztaoE4`VG`KDPtW8(JUVD<U=YC*wh zbv2Z4LCG^Kbb8vOctjEU77~P=&n{Iy^isR<hw4&nK+?KQaxW@`q=H3+g@J=31iQLU zduI9#lsaJ;TeU>45Cu{`rlG01caq^1M^WEY)<YBD>kokS{QQ-*znru(P{3@OpPqjH z9udJH@=x<~kcLSnNAd%DA`yi@z$O_3$WINKDa!Ybnk8U{HkFosV{$|A+xVK2!uR{_ z4emV~&0rfXHzMRirRE*BM2ke@D{}0ocA+vQSWh09*<PD0Jtrn4p&p!{PfD*}nD{%H zKL8T3&cF0t5+Swd33?n$-E}E8DU_}XXT`gMEUCw3dqRSf3O+m)V@K9)FI2eN%&PYD zogENvt6%Ub&9BMC-F9ih#f9xRA5mv&bQAr2r>w4iwC9>#somCGz34Zxh2mFIG6_PH zC}QEQB<+1x`k(G6wQbwZAWP~#QngRPZGi}xs{V83I)9<*Eg9wXex)1z7L<WN3){7v z6*{{qE5jorg>;1yBxFOaw}uCJY+oxt-$vHf@ho@_<O8Zy03;!x`^m?JYKN0;Tdein zKT%)F*z7hFjfw$Io&Ji>_o&bJ%NHI5NVh$u_j%@wcUBb!;=d(II1hi-p`Dc&dcx8r zmXAP^)6335d?<A6Ru+OSU`zH;j@BeJFmPmRwt3BSVQ%-MeL|Lt4yn<H4<EQpei!IA zDY87D98{huM1uT$*X(PUdu7*FV;G6Rwk2zJFfZKZyIL0)N3$^gxcBVok75X<u-fyi zGL)im3Z}B%=4ov`R;Ir2@y$tJPojL0Qg)l^89}M{dgMjkPx|1rw4Up0RcBLj^1YpL zX0So*`RXCXz(-bjHt%_|rkQoheY-X_Arq7JG+YciX_X1eG*1_9Z_Sd)(a|v<6Oega zWhN6aHyU*m9i*p|6u4Y-Uj&2w8VG?-{9uD--}X?F(c7%4Mg<EZHb@i>@GUbQ1|yD= zY??jX`hi4!jQ&#fx@3$>^ML@q6DAa)D~0}iza#ab6$c7Q1vMAf_laeDPmkSG^eF*B zz^J0OJ6xRK$o8F0ZXw}!xHgKpWoEK0m_SdEc>Q|o+Oj)q$sX{2>P!!K7IfRptNNcw z$$hifyS}}uYI;{y*|bm4qK5^+3^nRYJ7_emy7s<UyV|ffa}xN+_wb;L7k^J|#bY-q z-j0fneQ8XOP>9R*M)|?MR5JG9mROgh?89Rxf)MOWVyNFM6bo*%FMOLKFRHUmKBlN$ zI?=HdcbHh7&8-;|KJ}*p2}P*q>8{tZ(iw%I1{HTS3HP5;(Ke^f$AB6}^?(fX>>*|I zR@=m}`H@ZAP5lu2pFAo6S)2{a`?4?#&acH1BgvV++4f^lHL;!kB8M*xfj)Ya&A#%+ zyrp-%&Lm1^;>LK94~V&1|BAa4KC74Yii)|uWu4G<jTkbEgVlF-$}V^vx!<+iUJSSb z9mxTe{yi7(+Y3+7X6j_#-G4CV->$kr2kfea24tnBf35u$3k@7J_TTts_8^dY(4K1a znGTw?QjnkABRTb)6|s`4&58xf^gfN(uZeMSaR~@y*ePA!vO!W`@z7ZA`ril^5=0a} z6=C_~bi)wb*tmUtjnqoeGQYVX?r}zp)CH0l*b+V;w=Z8Czt*cVox6N2-*)voxux-M zPw9ZrjhsxE*n-qnlZ)-;!By3Ti~DuVz=F$+EmvB61Zq)1K@<u1l51O&`ON8wZWdSJ zyPkw{XAgk)g_f5yLq4SO9BvisY0%>Xh}Uo9Vt+_OT|0^SYPaI9dDGJUIJRxxao`E? z*kw!3d}IKJU|`{Av?k~D7{*f)5`FjcMKst#`T;S&x1*b3z8Aeftv)IH&zl@hc*;UN zzqM6FR5Y7CDKnEC)SYbFfq+d1&2$c$s~6NZ3e~lclamXNh<Jq3@gC<%q|C~ctC%RI zkpWvuS7h3%yg|$1K=;V72GFTNlbGla5;H9ZiN!}ik*Kxn!w2PJ)kSwjMsbIRxko45 zprp$AWQ-+Ql};7JPTie!v`-6&FfJXlF)TOw737n>dwf2+WP_DNN#w%N%IXW3wh-LO z`tTj7QCRP$ygZvbBN}kbgSP&B@}wi%_jd2o3an97q#I$t-e=7ua>KZF?2`w^-%i3i z9W^JdMsmvJ>!#4G#`W!;!rerCn5^to1gkKE^9v=)<HL>t3~~Y7MYX<Gxxx&-w;p}E zZNBTN3z;M3e*FmdVT)dsX5&}t<>m7GmAwSZzCnkqJ1oy_T-<uInH@zY-yG~#Ta#vI zk@US!mZEs)jd{{<jua9CX7;A)ZLfG2e3tK^Od8Ma`N=zEf$B^cFz2}z)14v(QLnQ( zrOQYnc9ZRdsTvdJ_c2QAd>$+ZrrzFB2dB%Ou5;cK9U$Iqt;Iz3AFoTwm%eBf!4f%t zmKwPe=Y;p*eRQ<%aqPzJ$zRpEY(6A|*3|0Bn7d379{p1^NS4>%&R_9W8l?%X@oW+~ zb#>Dd)Pj~e4h|&LqN$kr_~Q90#>P?Hp&{J7+*uco#WtY#oB%lnqe$3xlaAm0_D-3y z-9Cc~3ihq%-_XOciMY%rOH}Yqcc&O@n-2$`HR}PX+laV$)innL43K_f+5sXvM`II| zYwP<b3*KA9O7%8+-Y2`HGBIgxe0S}4B5reg+l>ZC{zy%rd`jJI^)S=*+GcamO5*20 z&&yLKQf$2VL0XgKk=LraOO`L?Ne6G(iIGwxDY-4K2lyr%h`G$ZYLp+J_hr0bAUo-` zD=awXC3+<sf{#NnB5K`US;X8zF}4QJLn4D6@Ox^>5M{YO6UQ7Q_%?j7bMeF@A|hhX z#Zy88(FZ}yTMMweG|I<zPTo}T41PK`&Vz=9y~Gkd{{3BA^Md5=<nPFx$HkVp>c%_c zm?uF=$%Iw9H@ZfSk})xZ!{gV#p50xn#@}UJh%sq+**DXJ60cT4uk+39$4fQ7?)22u z?mY=pqK~vUzqrcP=NlO+x$<XY51KiP?$~WC-`yTufJQTP0*upb?Pa-3YGWyHDq?@V zn{$1Kksuc=7c4s5f^z>x66F$8QlJpGw`kN(0$?u!^fPmnLGFpODk>@<U}6W<$R+l! z31ni5Ub89z*c~X3VoMM0x$K-=QVhgmZzCOhw8-S=CM;7{p)26OJ>Cps6}szRQJP&@ zXtlWsciz!F{@OdsA0hiLO4!a|;_q7a+2m66j4vT+y4lX|<Y(i)Z-Gq(U3n|WAollg za7df764}9^cSxPx<A448<g86JDIsHAw4!ao(*X$06vE`|Ez8Ok731ges=7TJ6-4E= z%%qG|Tb!<H1{M@vzSRG+iP@rez*NJLs;H|1iV9pCbS!Ecdjqi%K{_kHr9x9e0^+w2 zgpEd*02A^Thh|1oK|#q9NR1Bk_SQ^U0TS9t03g2t-RAYxRY2a(ZsFZ_+;#C(QHlLh zrwMfAoSnH9v4Sqy;5dad4jW2BoMlxDsK#gui;NVsybI=?y&aTPV%GE8z>B$CNlX@H z+w|e+Y?UazzOHd@yXecdX}y>iJaF4Q@Vz?Ibrsp2Y5aRd+vH*`va{72fLz~t_+9Xc ztm21#f@HF*0be~U>smQM4itw58V;K2i3LN3t=o7BI4bk@ZU5`+p$510RL$(mqu$mg zU+?`h%W;+yay9(7hdDT{6##qN+q0gN%)_^^EXO399THZ{dYB4A)Aok9J_JH|GQe;z z-vfRa5X6cB4t5x(r?-D{+AsYApB~701H3hW3Jd`u0l_jFMKWoZ78ry%n`ZzkATlyG zHnz5Qqot%imS;x+Sl0UQ3w^IvoB3H%rHQDGi;dgMb^W_b84HULSnm4>d9CWjao*dT zRNrjh>%EFFykbC4V@j0EROB@q9GtPa+G}<%rZ}3(RzzBO;`m~6wAAsMQ(r#m+V+<w zC0{0sgdh7LX=JH+YHP*a**JQTlde({%uawjM%4a*zZqhhoWx@=ls9wGxPznLd*FMw zgY@w9PzVAs!p6fpnEB#7(+F^JWdDm$DTXiguKRQKExr1-675?~K1Y2Ucjw)zpwUE3 zg*Go6o#FZ^**_xgHaaL0dHXgOv=?I02gT}!pJbGQ3OX2-%*;&Ko;6T41Tuq2<oJyx zJ-xj^H_bb7BIrYObaY%CZfFP2WIMKmH^fK`y0;*D6U8f{+r)DKtl{f+S-H1102;67 zJ>{_|N*Z=5A99mKeG$v0M=X@WSMsj?>M)xCaLt!7fTKc~%=h-1OP^9+y0n)zgI~NO z<tI_Pi1&W>?Qve!A68%Y^Iq1w4JQGBo~!-@DOCR3)r}>fs;pUg-EZRx!dNlr`@G4~ z(d*-pwml$2EruOb&9<^vFU&V)%Tp;TdSW0h-g@k%&Um8hdZTAiRaw`3)h~~X%&%9K zjQv5RPEf!KWHGJ)YH&@Ngl*R}r(O4vCspFvs}vr`b+6htO^zFM;_a4d8xP>_1W)G; z5%42mJ^||)ZeXGQ0leRH0<srG1yU-Y6$D`3{H?{70E*`6>FG#Bj~i2fayuBirn@0q zP#;iU-oc7gqJ;YZ!MBc;LmVRaE5|?3w&i33ATejP11TcDx3lg*KZ^qEyzLo=7`Eb= zYj+oIa8S^tCJ{Rd2Qg7p@p|az8Nrzu?Ew;0Bfzq}M6SQRoWQyy$-X|iA?2-%qwwAQ zu*+XVc*P@peORH-askxsJee4ucPo+20cT`cC42@}nCs>ZUl~2tr?j*phwXT+N!Qnb z@~xLSjYWtME|R|A#77sDvJk*pxuqPqv)`LG1~JE4(r3wrtEA@_3-Wg!jh{b%?CqG| z%hVHpAEt1=6l&|TJ2}~Q*S@?=BYtRzGv3htau%T6z~NR_M#D&OaHx}e;R%^d++mQi zfWemA=&`5^KnovG<qb5c(G3hXueB;PB087<xOt6y!<NWUG<A1xcvGPPXtR&7bj|YU z2?z;itm;^f5MDNH0Rya=`Z%$J^q4||Frd$*90>IhVVKlhI9dMAMfYdNesJj4?z@g4 ze(2FW=)Ed{K;?s1!plA8>hxvD>5YC|1h$=LPT|QYGEvsX8b>F;42V>Hdi%~aHPt^Z z*f|JXc<$>~JFK+=E(74s@Upa8ZVksH5ee?q6=&b>Jr1l{Ud9DKWH#|M>h5P>%?|YJ z1m{Z1-W~<J-kqP%!Fqk~t}L|7epLD1_$e0m{Uo{up;GH{RdIm=sDz#UWV8Z+wvlb8 zT5k1*22W3~S1R}ng^*X+oh2n&wQoB9q$KfcYQZyFK*{%KZ*T9fUw_)@Vz4D#nr?s? zYU{T^kpx&oCrnqDyrg~b!oFz^%vqoksynaTr3Km^0O<}YsDJn)-3O7MKvD?UAVoc{ zgcZ+B>=Ri#MXu5AgoFfqa1am}?j3ecc#ANUQj~S4Q3b=cnp%8r!o9ERI9(jbvX862 z!#-Qjc<5)pe@`E%GSTv-zinC(`$4HcJ17HnJN)*H=82KF_T5QUF~AyyDR?}NgJMLk zea;WrTNT_x&wSgqcly#b<)_MtM>h1^7AJ}^m(KWq5m}m>C)hO2R#M2{Tu1amh#(;M zqg2(fe)){T_gnqa1tG=Vb~zR@!m-B<h~wrx4~%ZHK6_s^j){6P8CMMtqr6IWP)N=< zs})Fu>1wMwkEz#y(ur2@86zWH`nBrILmh)MtlXTXKW+s*7DCSA&<kjQvWCVwsn8^H zw>{8Q9<iuJ30zw44(Si14+ko>piF!&c{#4k7R=le4{rb}iG7zO6t}c2?=-ylgX@jO z7i|f$=}b(ZRH}=1Mmc-Qp#e%w5%Vy<hLD)rp!UtqPC3X3oRdU^|7A|H2>ezsRMON; z{fQkF?^!<x0-~JG#x%1Z#k_wdPoAS;DMSHu+}A*?0ARX?s=Kk8${io&F0n!lr7W)~ zSSc<twTKAlIyp>NMJN+VbTaE`>+|Qr)1sEZ&m9n%B|Y(QVNL>bKf*Y{@WW%qj_tw* zcTax=NQ%U7067|)#-y`*`zR<F8yRKF<j4Oq25pkCX=p>ZNcJ+OA^<HbkNzobq4V8g zL4v%ZT)KdBv_$2mlT+E)v)&#l91N6|-6>u-AvkM`evQb8sZF25=la;^2KI!8=Ai8$ zCY4wdq=%@_4sY?g(F<iGB9yXtocmXQX<BkO9v*Ji)e#^*+rWwcwJI6%Zewll%tHud z>&K_;$J8~Cgy^dKQNO+If+9!!{rea28KPG{*VF)3-958AP8sy+NKfB!X&SJjAf=?t zPEBQq|HWbgyu+8eItri;Z&91I^ym>579~(kkET6{pS3yrWYqnfpf|NGkF#sk;sj_J zD$tEcM?A_!X9P7~pi6}rS2PhjB5UK<Ib}`s;+KYJbkTuE-Qh{BcZPF~YqHVn7q)3a z7uyW+ra)+iiHE1(^+e~5IDB0ex>&tw%_=QT)l^7(Oin&CKh)~G5Dk(N7d*kivLChT zo{5Nj@&v7f9O}5TpN9b@R+PYBwtZ-J4LVU#&OsSgDvatVfnh6rTee1?o=u%Fn6~!V z+Zsf%7cb1bbYN0aUlfZiZ4av7Xcu;BKSIfWQ*jbJ2QasA8c2L^qKTu{8HZ^(&{95H zo!kR}UxUv6)zhRp;C6>~Rq#P-vi1$LdNzM%Wk^V4+H_ADV?_5J@Rzx7W6ev6si|}G zex_-YAXgDWpw#T_pYotVV#O^ZYYvD=4LX%$r^>nGO!UEr&`tR#`0Q|BQ$4c>>bQeG z05#b{{XG_m&ChF20sti)93FNEdH2u=@SOoAf$r&3o!^s#?jMrk5=6EK2BX+{%m#A+ zjz6NlnX?EQGNi`GkIc^Qir%C;rL(W!wPa#*c%2`nq@=Vz2v@d^U%%xf!T?CRm%hM~ z2rJ<;e`zS0z?TEC-E8*mz8nE{!JbR;O0D>WXYphj*48J1Z7uc(S=Oe)KeTO`Q@M8p zvUYNtfBMemak(8jX*ao2NtK!47SQ5=W;3!&tHa^KqE^WdcR%$XG)u-!ORJF314>DP zLjvwyg>LlBS}x0n*5vl(Ebr4Pv2H(XfoB}4&q3Lp70(K#bI^wZ5EnN$lR^4F9DL}? z1o>Q`erMFWf;#~^k|rl_$ZKU_8vVAZ2SCVMb=`#y`;22vy+W8d7fFRixiUSxL=nhG zfrUw<90WWv`KVY}KntZa8)4LxKC$Qhduaa9d*3pM7<RkKjyqb}yaW`VU|slflN`_s zGIM7>tptPYhGZpC>BNq=J3*j$b|$wT{%f20){#9T9EzKsUdmUQ9I`CQnJnT@*8KV# zPn^i1xn#2Z(JtkM+wN60B8#jTVW4~`VTDh^2Y0xE><p3m0oQbdN{{Q%qB$E#9N>Qh z?E(%~34A_|erV-<>G=M|2}uGh@SC6Gn$D$L)`UeZi2R0UTpM{4*oy#PGO<!5Y~?Fp z`A$x%))3C@osN}*DNIQ~5E2ni6XbzR4Orw}M|<76A&=%=7bv4-RQtcpSxzvYar>Oj z701){XMh(L3`AHB9w5ZU!F=wnOl96hT^x_ppY2Xful)hyHjWD|F}n8BNw&%8t1wbU zm(tFWvqfwP5Q5Wdff}WnDKB(E;JrcSOIy45=o<8Nc!UyoP}M9~_&v8fSwsbY&!wq* zC3YVs2lN5;R8>2ONQK<&Uc7kE>JQ2a)EL#p0;LH-g|K#$3-iDcfM`KW3oIu-K)79! zz@lE|o2-6&!d&0~w|3Tf189XeGYberuegQgbFv;RR<w!Ko;2>*xbF^a7#gkI#6JY+ z?uxaI&2t7}u*ltZxIYxqI~dHJ0#3K<S%H*{6iUH5^kT<T5_&bL3g0!ea3;XOcCD$G z2!trckvI|<D1k#}7zS3GLdtTrZz_BvfWo$k$neg|c?ih6Y(IXmIG{AY(<JCSUj;Bf zUR2a)#cw7oxItC`)My{vF2%UYleb>&uOOq_ghfXmRjm_9FS?6xa4^!)Sd9MsGfWG3 z(}h7FvYw?%Y!Yru=2PP|V!%l^=ud(aW$~r;e?8XFI(r1a7z-sxADPYp8r0WW32F7< zD&y%d4fgh70d@{qbSe*0BYXQPL|t~@l_D7#SX*x0?!U++K2o@oLZya1ksw68M|Tv9 zK`-UcWo12kE)N9={^eOcqWVmjn3%wNc^DNTOQ<L-+jPJ)pc?pX()f*Zmst52?T_N~ zH$oIx?Ww~Nd<NLM5BP$xb907fXLYo+pb)^o<>25LT!Pcy0b@uIlqZ{;gTyf?MY+vi z846w$WiFav63~riA%_z^K}Gwf<q{tSL^DtvAq3E$kRK)Y0>721SZwQ~|L=}2{XiL3 z)x?e+U$dSqNGGijAyu3{>-7VR*r&$EE?!e1>cx|AKmgEfaw)BfQ7rCWi&SX`d!*bP zafZa(82RES@Dt^>27zQD(ZJEU^cjr-hMwCJ`NABQh?@~e1TZl%3t>6%f7PkX6yy~? zPbc*z2gO`?r0HS|bk)j!OTeH5WZc}`WMohVjsTUwKN|U89#JnIpz_|md!LDz8nKB2 zU*eg{$aap8YP@9jz3VoTjlhhd;P!XW@%HXxy7TC1y8{4HUfwby4ugcK|9_KDN=gqQ z&t1C`*#FA~SdhL)+q}bu%(#d!&Y91av8CUKg?X+e#t<-P?OI;%Hz@%lir>$y%BnR0 zD8E1^!32R6LZ~DZ2aPTW`?@Lm9>BV!xK8qnjUsbj)4n?^l&cVu%r37?J4s)-W?*MW zt<l_s`Tc){Gvgq@x_-xLs%OcTa(i{Vlg@iLSSab`wOmmz!spzRk&vJ!4t)^`b%tln z?#~bE3;)urRd+?D$7P9Z9xfpPWOu5@l@tOf1LWKe*Fj;N%Pa9}a*xOLa^)YwEsidd zz5VjO;D&m11K9P5vCccdo!|l!fWzVRc&nz#D(;rrX{#NM$#!fxjZaJ<iAhNX!Q6|! zfm&qvWsz|JJgu!=KC<=w`**%*bSJ)K!jGRMz9HV@UVt0<L^c}YCz1O;G7<y$iGK-6 zOb~5rXqa%3j9Jv`Ro(j!d7S$xz>?exq{6#|D3t!;WbiuI$vu~tZln4c<P0`4P?60w zhyYXy64ewG7z*)y9siTT{bo-Fc5r-vBL{Yr_rX3u@Ap8XoNoL7Yaj^;Qc(8vzI&97 zDu_kpp2i6Z7LSjeAnj=XhH^AH<rQ6~VkmeWxt#Ea|Moyt^ZwpTl$PCv_@$wls96~I zX8h0Y<VnEY|I5XLuMN^Z0#AScKkz>`<3c(j!S)SI@1mj*c<!G0-}dePD=qNcy}CMa zR9AI10EzAcH&_td7-axr0Ja$Pg^Y>7AJP2pBnDjxzySgXpoh2R1h0d1Vz`IRI{=(0 zMA4NpvxoUV$zPJ<5e8@g=y&DO0-q*MVc+@T^)LxA2;3(ilB55-?G~9!p9p;?90^mn z5h6dTnaYTZ`);rE;>C;qeG^aV=s3D0Bqi_j$l_4ly!(vC;FeQrYH9+k6#u>+8O&^u zM;bL?I~0n658%js&1=xcA6Eh}8jX?XitEHi{!tHd;WA~&0uSKBwZr#s^ctlu=)SD; zu~f&xZ_fXJOq)doXi>m=>@y^Cq4!UwO!c4YXTX3~v$kfYUWKce9P(8j>0KZ2bJn!5 z5|BqIR%?baCcNV=f8Cc9vHO*-?!b9js9(Xs`$nGZnSI$&FEc~GvsG_+6S=E%xk?@y z?I*vdFZ>=SLf(Il=seAI=)W;~nSvYf0`2=g;=(}w27AjSd2_^6CjU{t=X`!yS@#Nq zGrS{gVbB8tu}(GxN5E0^^766)2}vOLZU~l3f(SVOophWkq!aKpfbiH4L-sG8NPdpu zi0C<n0D#{E&E`PMLD2G|Fd7KVK<4$5jLC&SryC4u70@jtQ_-~a38EV*_^7(6$rCgJ zD^)8S{R9z9{T^-tdECLJKpkZKx}+V*y=ct_VS`Qxo!S~U^-YzPm<qFR?V0e=GL9s2 zfpAc%5rc5o`~1-d^PfA<7$o3Vx9gAK9&==}0BWc0%SLJIAn?Dux{6rS1j6F}VmYJY z2+b`-MiS6ShX}rB3Qu&OT?;Q(RL_{l&RvPYfB#+~k*D9Cjp#Xi6slj~+eYPVz54Z% zL1E-MJ{sse=;roS%d?PP4m5@{H8m~GM{=71J@e40L0nj^hk(O?=I))4LHoPOPu`>^ zcBW?1A?Jkwf*MBEM3YG$Nf(j6S7#15@8YYG-zGJnVG3{!jnQNp<}VqXbqC&Yl&bZ7 ze%{~Ev1Ddrv%k4X?`#G^&jpL;!C84(IL$|z;&{Gdog}ABJGYRC@BF1C?z8V|5(N@2 zLcq(S6~6<WDpCkROF2+h4!Xs}{~`*Z4iZBwe}z9m1{ft^41kg^-$}=CuKAl%N%|*| zu^r+(q`A+Cbe_xnNR*z0M0|jcIz-C?=z;*PiG)6r2zRDY!33nPl(I6GWF~AV=nMj6 zh-;>*Eg?yK{e-3@h>OEipmGvCliFsZ50KFVS*e!{{2LT8&v3&)6OiQIV7ctLl?*`f zF()|%Uu;kP&qZ*9u)#i?l!SzaMqvbKk>m}POzcQoN`rIJ``!;)@8_bTA|lX0SN$t5 zcr-#wV2GgeBbNe$F=zuPFOS|&2T`VbH9|&CesXmKf4}!u^oZ9loLxWwblj`j2d3!Z z(Fi_=hy}AN9D%K0V<kFu601Jwx)r_q7?Kcka|Kr<JUc#y_x0#TeB90!sl#HLW>Sje zj~tBl+%NoXS`l`hGW}+w-Zqe?&-o50BgG@;G-#n6AQCPF5&fdGlWsB$kiR<E<jB$0 z{NUXH!9`nIyw`dXQ=n~521-b3!1h7vlASCc0SNRHg|KsS8oegi077?<H*g^IylF)^ zv3c}-V&8&O($Z-1rlwv~OO+b=2M5CM00;U2aE$miM6#>ypM;hHAFcB>KL8h=f!~z` zr3zb4pttDsnnR)hK?wvV*QHgO?-vX=iq69;0Q%WTAp_xIosWi9*ff7HyuR{1@Z(Se zDx;X8>W?RiA$E#<Vh}Z4u32QzJeMk{`gz2AHT6ziYzV#5Pd$Y8i?<Kp>cRJi@gvlK z{i4B7^p@Z){pWMB*OKu0qm{(X&(F`zt-fmnHzY_kA25pR^FQEu7>rAe5d1iT;Wq`g zdXqpncz>=5v{(cm!@rus3#?&he}8{(ukRfP+;jBc6L>&Mh%UTZzd}7kvRUu_66p0n zmnfH7lqy%~qznH@xd_-(36Y~^wh^)u{6t!>xD?`HjNt+PERYOzTrJ4sC)iL>%w^y2 zEu#NF7Xx_Cg_j6E8h>U#c<T^jG*mFqOGg*uA74vKr4@mJgrCdL9e6Q3=mVf*sjicU z*gmzi2~mCts((r);r|UuF^yyo?NHgK2mLeN27u`5-;u!m6zG8cJ90i!-_MWxpNt;~ zS3?*>-V%jL1-uXVMm4PsAEx#^6?%M7!QQ*}_v$9{6D$-7vIcz&zj1;HXIEFic&kt8 zx*tz1GLSQ9Y-++H7XXt}o*W(Y0KvD1s6|@wq$?JO(Sx$4|K51~i>Dw%Ko{Rw*)+HR zzo(Q`!SClc!Z03#-%oXlSsGp84>3YWKhpj6h%k6~dGTTw4*qAbi&|UtCQiht7M%qC zT@E}MW5~pgZi53Y&XiRheDJ`*W(zud-uSfT(f=Dede!?G+?6tC!|Dw0wglc=B>TD@ zpBy<d=sI$I?`Pnj`h5mw3jYWE_`l<p`w+eYVHla1gg8=QpgfjlaOS)J+R>Bjfo?<g znC_LM3nC-<6zsua1h*ygoM5H~bVCyMG%+y&QxX2<_GV%ViuoE-C7`Uwlg@l^Fv|YO z%Iu$~tOPRY@^n}<;nNx%>wUYsA1ilC)#N6eL2cQ#9VCY=7jD$Q)m8I90-;(6gaEHA zB_WPI<leIGugFIhEt4O~o;~nCXHftaldB93?Es<Cy+68Nn9jXgL7P6#E<V}4r;v=` zJbK6OhtE%{`F>AeW_t2(kljyY05j$)Zyw_jx2GFMmNGh!$V1~{NdhR~%3tv>yl#Q_ za=fuAOxSV~NbY6tLiG0o`=~6?UCjDEbHHR?eYlR%0eY)sYnKMHC%vMJ@Ww*&_XDqB zAE3VAzO1eH+*yvg0Np(_bfXRwxy1b{!2Z5u_dD?qM;FNS$jXLxEV+3u5P@+kjDsmE zDPdt@k%q9JEv~M<12kmlgYXf>!i4)IDU$Fg0XwL%f<PF@&d<+*a`9HoZ)YqJHKjJm z(iVb94m=KhFQX$^-K(k&j&E5V1-W8hWVA|x&c19!LIYCipaY~K<I(XRTi4%<>n$dK zC^d+Y<>h;4{<wfgo)_r-4DJ(DWGIS?64)-K6am-5f0lzSG4cgyKqpbjX<^?xAT`GN z8SDBPXI;^nwLQ|y#SjVVFP+T3X+`crSynurx8#gLP?W%pY%YLX9$j3g(!MFSm6Zd7 zC`>UR6bxFXu-%29d`~MvrS8ApnoU)dGq0IK^l$$z^AH^kFB1NM>Q8aa&CK}j919XC zVu~9Yh;^lVvS;>eEG<>lPlU)R!e18yO1}BDI}do~wG)ql1%?2}`N2VKv|tutLty0f zsd@CkO@qF;*Uz}Ar2qgAUPS;h3Obh2Yp}w+))1lqKqmiA-$RI20Flw&-kwj}osOgc z5E24Q1V~)4*jd@3@@;@~fF*pxBOn2$l3>PlS@il@+rq-oP%eFsy9g;*0l|8bssN#X z81feMw<{VkKQ;^D1P&DR<(rs2dL;p^pP3sQ3yz;&4rxJGW(Uiw3s@q-lCAYz15AS) z9k4xJ>rECUi>|@k3c?0W&_I*>C8R+46+1rA2_6jCYgc-sk3dDbq2WS;E(qT4h?h6} zM7EQoGCCUbHzinDh87m?rY);LuTH@K1MXoBNXqQ%6pJzx&e`aHG7Rordvo4y05-V) zCLvw?%i<Ami}1NZ3?iZ-v*X{?nF+YAPnduovbC`hsc|dgBLyOoS;ekbTlmuE0P^!_ zF|C*ZDlnjj*AX5&q+J^r`=gE*#UsU4To2ho*~e#Ry~fKNyR?ArG4S*+5d(vRaF{`e zq`(cyDCK)OcyN}TRr~?Q(FN8`GE7-iqd<9OviF+m>X{dApHP1W+XB3zzP{eEUrE#( zf8+I++S<43au7%f)xGVnJj$dgOaRL<=zeWdlgP%SMFe+ukw6<BJVEUHfej|OqtV-S zxoULYAu=k)+5*_r!k13LglC~0L!djU2cZjtF<4aCNu5>F!Tb>Qrg{5`dAdj;9sG`$ z3<l(Id~{+0c<vWZ9sH$>s=0t+0q#NH&4QujPk>cC7x^m1h7Vuwo?`-Q0@N&qqE7+e z7C~Fn8z0}&zySHDK}1AEd`_F76OB}X`SfE=9eZ;o6gu26Fzdh^T{g`J>Duyg%hnYT zlxaaPynH4qbpg+|^K#jYs}TgNqxh^i|Dy1xY3b?UtHIc1Wp!V^<h-fLv+s|vE(_<B zoSB(nW%iUVPP3S@;_3YyPCl~E4FmQb4D1DTKK;f|C;>cQQWC+tswB+XQTRgNehq4{ zIvYKJ%T!f$q(&bT=dYRB1F5O|g{nTz*)Vc!bo5=#l$+4}(2)NY=xV1tB#}D^TCx@u z7pJGCDO<G`7@$#uQ{5j4+^?uXSH1h=elZd2#FqH6@&`Plq3Hd}<Jb9(s}I!QISJ@) zi$QX{?kx|T%N4NuEhj4fnLhmY*ZY7MtJDS589o40z!#7N1}^)DB}Bkiq`tyc`Tt{( z*cYG;z|_>#a?j)Y1xO|MY4!2*HP}0X_tiPLI&3oU9AgEgB8<sd0A=l{`7pp(g<7dd zI%th-(XfP(KjTNT%;STP>Mz0q7}%Z9_Li6GK5VZZd50%IRZYtJ(aJ=cp(SR2j_+BD z#4jk{L(G)?IHaBUlh9-et}+=*WugGKma>-UnRAfz1^(K|Q)HH)rGS^qXbp((^n=0{ z<z5&R8gV&C{zgPCAI%jjR8eyXDJcq617S<*L%phrHHXr&?LTho#(|BBHK@bR5=zWp zwKC$<Xng(1qG6$b!R{_zvg;!Yv(8AXH&@XE1=(WZO~WIXLdF6z_P8f9pZ&Zw6*FE{ zj0mZu>oe7`5Fo!j+2O>~7*CK`MpC8VV<*(qw(=Kn_6ZSIUZ;tLw;~MumSZ#&!_4x* zSlQ}5O!eLN*D(yju-Nq+10-ZpdGcRE=^GZ&7sVSTcaIIdhVX~lzsEhLbuTNai@=}# z=1D$M;1-F7>{Y*r`|@Rms`DfBh_fbcn*uBXLju&0nJn99FQ2}={OX-wqhWLuKS9Jh zEF8=bNztQVuU~I*>*87M)vA${AbWiNKK9*tdjj!G7`C;;Xo$KghzGR{;)*UAL?j|} zPtjO{6Mhg$Pt#9Ur-(?bIbZEOU+(z9mf<Jg^`5<_4@QBW_W#Iw%Ydl9aE;eQK#){P zk?!usVd(CV?(UFQx{+=WiIFbpmTu|p7`i*}@_){`_ru*EWrV$F@3r38@AG>a6|#f^ zLk{0*@pOqxYIa1Zz8+1fcJMX2)-qvK4G+fZmo)6^8|+6Q9|?}`_}+r@*0s++e!>_x ze!|pWz}kd)o=&(k-QkaPwjx0S0jb>fwvw34*RXD!=p3hT>Fn{W0a%5cp0VjsZa__7 zwDI@+M4s>p!<3}>*Qc%OjZ5@Oq2Ww@4U%pO%(*>CjyfFV;zOZ-8aXF?-P(eU{q^I= zP@ZUOYM8ICDQ4g(Na{Uc)A1*K3ZOgLIa1x`*3?QD>KFJ%PTrMyq$&|&KdiAV_sljq zPFePSmTi#8!$n^xSxUEw%CzH#w77<<S00%{UCp@qXwr8~Uc>|f39k4yzV8OnsFUEA z{44pMgvsD8WYT5JFB6nSfmQ!65&xKAJUB2whKck|<@vwmt_PgA4S{7;WpJwHv)0f5 z)g-~)YrvNR9KikUe_8}j^cKUz!}n&Z3c;uiH{BDi_y5vNQJQ#3GEDFxkZ1lYe84e~ z3QPq0H$Z;p<aNw|{``q;gys(LJcBTUW!C8v_kY>162X$g5cA)c|J#)SbSI&YKmNb& z#=k0p;7zwdc|ifqzx*A9KI#9qg+D7tz)KIuuprcy;osk*`!F~-2%>K2PrV<npTK>y zp9^*Bh5yTY#r|F06Q*G<IGCy$R{)kg!dPJ1<bRI;UupDzKj_1A|JM%auV6_=F&x-; zag5w!{r59E%s~B|F>|otJMrNDOKJNMVmrv98~xQ{jL42PG?o7=azJShvfH#%RD2H5 zsrZ-FgM&#BWJ$L-Hy{BA28A0g4N`<};$D6l7sTI+dg<>x;zk8q*g_m+tF5#Z)03z9 zFHrph6B*^i%~-wrpK?x}ZS}bJUsVE`awaSL8XVj^MPbprI9JAK9s6GQ>xSa`+i%O= ziBUywDCL8y%s|C2ug?XiD*Tt4e(lWRYVH(zFKck}S9<yIf_-ScimAo_+K((mC(^B` zJ;JAgn8<!Tx?Pjtujs2E1|T4a0}B%<$o2l;Y79g|@0J$O=&-3t<atNMzo7U5za)9c zQr3lfhL*$t`hh_wtj<N+|3!sb*vtPCZzi1iGX3Wm|69KO`x<P4Lu30&S)Mt#Li-VZ z+=mgnRyp6T7`$363=GoI7apFq7G6eqi}Au3lO$jN)zxj>t_=(7zUbh82d}3-xSsgn z#D-(#8Gmw9QUODYK%pv4thJMr);rhY_-m1z-1KBz(KqobGd=8i<%M;3A?ml+zdE0Q z!s*}fR@mV{(7x<qSv7xJmj5i!^gCHh+UJyhyPMPI`o_|N<ue>vqHF`5*WgjZedpuw ztD4cgMq6>WkW&w;HA>pv=B!dl3)$M#IsTvi2&jBxYCjWM<2-n9&vhX~k+Ul=gY{$V z+ummJIJs)`RIp6CcN(42temmv(>whI?GfMJ2aToEHE)Iu4XNXtoJJ8E%KhKIG{h3` zb66CIRgH65@P;ggBcZc*=Dgh_r4|2Mm&$XK9n(a#I7Y<l8Su47iW`d(8|0u5zNIUR z9aiok<2N22a)-GWMDk-C)Shl_y26Fe3BVGi;tW5@yV*NF8u_iw{rN&tJSJ^nG#{m_ zGm^QmZ-`4_h@{4^#8MH#(14D3IHlcz={mFfUTxwPhpHULCZ;OOe|2P>;*@h&Ql9zN zKYPnEs02y8(saImj@8J9F|HwnjBr}i^mTX_a`+2U*^W^2qSddTh!v|KB{oY_3x=iV ziw~d&z`IlSqX>Xkwbigi#(sl*GnYg+9fhCyAU0v$RBKn9q-*+67q(chs-%1oM%Xp% ziaX|JuT*#wZMj#B!I6QkdnIuJjTw^mwkMe|@|j&YI^td1A;0twV)^=Ct0E&~vC8s2 z+;c2X=AWbA3OG@r5bWE^*CZr)%vG9wDMiyvma;4yE1ODhQk2yt4ILMBLX+w8Vl+3x zh^)MhQ*RHh9rI@XqP_0xUS3{q_PPZqLeOPcbEuOfd_%`YVfwl{2tjz;_h<UPEI%nG z#QQ!#ny0f=1Awm_tPLw!GSGKZR74HBv*x3!O#@}O-oS#A95(ye8DN#OO6}}5oKe|W zsvG}RnK@(IS`3uAptbm4Vf19A_sxMcQ-EUur;)Yv=F40*=KUOdd_jFf+z5@dB=-uc zusT1(;cL;QOpAq9ve2?0S{-ZlB$5=d0g$IiMdae*7dy}&ya%H=>E$e)5@`ymYL6EI zf8xc~bekFHRg0@t9Hn|Pn^La3q<W!!X9dz*=A&9#BSZ7Si6f;!zJ-w9Uz~mfssaDb zEk>Lg*Dy&C5qy2n7;#JuGSr`R(Vu*AFdWhA=rri_rDC~Z&OZZ*(2l>^tBPYX8_egy zd<|d4j<Dk>2wi_yl#F}9pb;JzTcV2=l-oJY{f-Db21P=LvcaeBb^$+*rqm!wQ6&l8 zshtM)XFc14)r-<8l}#m?ydl<c4&Ol?FR@ADwtD2qi_i^5@@_q%jqksH)qi`@AMVR9 zeE$=NB!e;XJGd|?-;q^-6FWt}MCT5!3wGnaL_d{p&4$i(ur!<(zPeh@)^fzakeG;1 zlSjtgJ~Mbn92X)^79&jox}2bz_}92g;E02ZwR@PRRXZA$2KN|epGWMZ<6Bwc#ak`j zY*^-(xPSf}v7dauJxMdCo#<AobKq{^db9mJIV8vUEz1A{?v(8LQ!*ZIr09G1@fw!6 zMRk5Z9rbv`e{CvI$4-^%%0WR>{J*d$lrK&qO9z=38;}?f9vG|*>yeq4$%$U?l%ypg zp$@hP&WHSBDP=t{LM9!>CLNX`#6ryS1eOB`&z`0Z2ELfYbfQ~v+xAbCaGNZUr|va2 zbuAsS(Xc$_F@s0%Dcz3FAs}bgj>Zm!QC^y`1dLl-E~wGF`fO=F7W}%u9Uk1no#5`b zk4XAiK89S5EgH+cUso<<`f;NQH%VOG{=dML<_L(ppi2h&r{g%$16cMkZU!$6xFu{1 z5U_lcqLR{svDV1QF(}Q^m4cbgC0QA<U%{m%4Pxvz5R9QJ6G9*5cc}E*nEP=0lt{A- zsZ&K`(u-*%#U0G8oMUIv*Oxqb9(M1OLo}pTe!J!{MPp!9ZbC<IqTa_a67+dp-%sGr zrIiwhw{ZNlOrWNMp+|#Lm6XXfV$3+wQrMqUB>G(DigfxCsYp=|8jd9qqwHH@-SsOe zAmer-9K#SA^P-OmM0=WcEh#8lq=tJEHibf(QJPVxQDhGZi9rfkM33x%6rVVsG+ob^ zuj)D!+HuKbCW-Gf+}gfWFC(Y8a5A3rKK)8A*T{k4X4V1!S(&Jdfk7z=?__k%)8#he zZ=@{Vh3M05tzO!kFhudlX!HH1EXFs0>pc(=C6~PZ)e!Xh!r*0ru_jBqHN#NVSTzNf z+r~<pohg&{{y1}VS;fb1&JsfdPqjNNsWUmdU|7he-ckLB+SBA5IJDT|y<bN2sM%G% z8QdL1X_9-X)22jx3w)zdz)tqEC$BR9icVr1^;%np(?MMdku*6nmYa+}BFBzILiuAd zWj<EZgF$#KTkjU-nn=uZg$ez&v>a+=tqx0`-p!9899>W3C1a*9--xkEz;_nBk;7`^ z5d=@?GzLDB<fpPEuu-h>C6{AUaFZHTwDqU@?hr?8nxIPRhe8r&w4|?PcG|l(4$R>X z=EJ;rCUKSTygt=Ip6wbikqm`QOOwP8zU5<p2`*Dm_tqyl3a*i29nU>1ThikBw%k6r z&U+8e27G=DP-y-kcs<%|d%u&LQ2&?6YcDeIhDKjA%4Y61Lbn<Fq^{+nGiqG)?pM1n zIYdERAvidPY|PV;M^h!FMIimNr>}7|*FJtJRR=e-Xx?>wcNUnf1~Pswcw=+(BB<|q z(%iaml#~OQDMCQI3=;;U-}H9Bks^!6Rz#G_aQULfUUkJb#uR-;Mc<YDvmjht(j0Fy zpvCUj<rdoGrXbyz=Ms^R(b}MzI>jt5Jt1&g>x7BJ|E#UQ5$y=UCW#2fs1!#wh)Ke> z@0{M{tgfT(4v9_1!m}?~Hzif;n&WW5{_sS|oLLgTPhL`KIJK#QOvPiS7(b16XCh<d zZnC+kkLok}<L!)p0ShUk6*B2hVqx#CK)as7F^$Swe|ws-ww&K|q=ihJYkl(^0XZSz zgXnk2l?VSf3m~mvM^;28uBXk^<xhS=NZ@%FaNyT1%Hr0(_M3fDb-{UUBUP=2xr?C@ zub)li@msTU!A5pnV@`mTVC!h|UG+kc&-oQGiTCCCZYmyWj1ft?mp<Gcx%T1OpleVb zvvXIP&i#H;g1W|b>0z8z&a7Z^8oTK{ftOTer49w#@N9fc&e~@~L}qMLN3W9GVt1#s zel^AHcimNl8urd&CH$E30}VDZQT=0fdb)Pc^Ef|H%P%hK5O;vKW9VDjcS^#jUFsb1 zZrfR=VsFr>3hGaQ=wi{nZqO!`mt{b9xbf`oBvyO&o$`Z<<o5CP&Rn)-wbhZzwM0yu z;>5TZ(N*WwvK?{k<CX4RM*9ZzO`$T5X-MytA=({ZV*fZB$Kq-7rS6wEQp0cEGr8${ z`yQqSd?P7-NQ)a>YuMxk^tWk+Q^ULvxS}OCF`^zCby3Wr&`x{~NK$@2L}svc&`0Ql zpoX|R#0>*7AQKr#{wYQ#I!yvtv&G5jvqe{A2!6*;`~E5t&MiURn|CbN(!MmY{6){w z6Ry{mC$F5H*+gJl>e%SX25V|AkwhLF9Sa>Ctkt9&ih;VwFs&ccqGGDQmsDSE)3v@` zfA1|>kn^Ld%P>`Zo6Y0Mw)v(b@$o2Q_<pijpmg_{qk1?-^n$g|n#@T}=9!1~ly39g z74&D;?9`<4XS!;G@MyJj-nW4(wr-#~yIAK4i#s{#XDkI(k~7!aaArwUmKM(~_|p9l z8e3FpZn?>IMZC(&(e`k165p}U<9`3cQGwQ{Epxx%;1>!pW;b#zSB&7kk$(G{$($lF zaJSpynQnG=(6Fc!bNS2^+*Al6NlIE;T1JLWwUq`N9#GYV=?e`^pKYzPE_tnPg*`{m zU?oa5|2+1#gx8r(&j`=tu{k@+=QRK8oUuuch;Ohoi&EVA_s1~7`#(4MoR%D5)^OVK zY)aRIM#78ee$-U*Ix>MzbYOhYdTGJM%4|InZDGLs(tzTGb+gP8qH(qIA`VUT&uZOo zwpu%y0a>umB_&nwGF}y_r5TYb%I0$BGFg{g?KFtm+<YDR`zN}|mxYyy>F>e<Kr+|3 z>}wX)HaShd-8Ix6)7>vV$%J8sVn}eX(VuAWM`M9B$8E1jz{)asdPg{g`}-8x8q^Q# z#%fk_gU-7vL;<;ffKnwW(S9PXsI#y%EN8Ev72+QcqR{L1t)sskl#dV5f$zE#s^|}! z?LiyUj3Rz!g7>Vr+r=ZzML}9SeQ!vXYy+2mRQGYYw;0njoQslLRsfDVb{Seq-yF&s zV|u2+IAjqAg`mSrZ{K##9YWBvmGJdbWSTY#*e!XpWrOH3k^Ux+p>k1Tl95rTb>Khu zHWcslNqZph-r0X|7z+*8H<<90$W^@d)Ip5y-;gAfGByl1SuKC`yy_j=u%=Zjny>O0 zZfpPN8sp5h2lM*IrSQ8tZ)c6@{)@+FF^brXl#JP%`AYs<kB80D798ZjgO*2OXb0|V z=%0{$1udb3sJhgeaAxX_mG&S|k+R{T2DKMx&Y1F>hvb)IiRltpG}O;*{*L${;B8>X zrlrg7dixziiLG<DtkfM|uq${Z3!6%p$rFnT-}83wp3WLMe4ME9WnUSJpbZvRh!^tX z<=&y&xcx>VI`p;4eX?*mj`P8vwX3#_Zit&1)0Cev)13mYCE3dr8$&kKDD~Pf;YZPT z1S~~%kMvC!7=c>v`Hep8H}b>Ic9mMF-CMC;S}MVE^>9aErRA(G_RJC~dao<+D$Fp0 zw9;jtfYaXpyAGPiVSI^9v3LI8v8H6@$&_C8C}+w`lnqz3brq7{e0qc1y@rCq=Q~p$ zQurEaH%5k}4I~P`>vM2Y_wnLm(P!{gsJ(?%1f9GVMXZDWMJG4G=gB6jv+TE4D^(#( zVJmjE(sbKMxj>~&c`m>iO@|%6f_3cK_ACEHEy$CMoVm~F)KDn|{6py*|ND$OYZg&t zc)s*QPY;wN*_r3}<c+ws^AYFvhoOp@KjqQA+fYr!mCIT@OW`UN@1IJzBmt@QGgCf% z@x7+61Im`hX4MIg>cnG8@b9}}PVHI?BUHJ3JS=XWBs!lr4$TTOWZ)HU+MLogV+Rhi z@YQmA(Qpb1lN&ME=!~b#^wgyKzCEu_A`3_X22jFfJ59zoWQyNhr|kZ52`auKX#X9F z5@jAbs{DR6oCr4~DV(xUxLG9~b*qqFS6+jk5j#L<?pRWvqWa{9wshX}ywv-6>*4;^ z#tTmPCg}cL?l+FoWa(V3eY>VM43br5zP176WV#tLCc19t8@x6xyf|lR_wR*fsJt~~ z)3~A~b3umysBZRMZd02J_?Wwpum3!}zx+j`>pa23pXsw#C>KtEr(W)c>T&NS%5NKJ z0kG6F>rpRc7J7^;=nLmsE)$nR38T=?K3M8Z9E`tnxks=4(iwHIr%b{jz|PBW=*sOy z?4xbzXQ9-&%WLa^zosS88M_8TX?{_=aj8WSl4_9bqNp#qF9ue)B181i=X<RoWJU6( zW`zKw0z;RI^L(Pga5Nbm7sIUj=q$w7eIe-2PG=HHdJD7zTx*8zP9K8xPgrST!NFJ? z$Q75hi#o-gtAWcYqPL7LxfkK@W($4;x3z<ZqT%BC@-D|+ciWEAi^z3^wfczO%8VDe zWdjf!XL{|=netqHr8H<DZip%}lwwgsBxMw^<73gonOlaP=h$~~C}U!9@G~VnuPxE` z`FSp`qZUq98mh60`1qVwA0IDDF9|-}_Q4ODPV3JdlV8Cih`2baZ<mjRhc8jO6;*V) zt1LeYIxjh^74Oo$;R))_^SN1%%;X>QGIPrNXp}x|zm^wyYumEA<4nkrDysMfMUD^H z^$`g)8yfN84kxEiWbjrOxi8VDa?;smgi{LM#uM!mMechZq3O<hU9|=2xI1+BJYJ4i z3f#7()-<~;FL|Fp9Icn79yc;J2vMCZwnp(>1Wx8N(!K6KyAp-L&L?EDKGzgIw9=<0 zR!yW!3))Y*o?ZFe?Q|CNq7mYxqE1i5)2m!A{N6Tn)m?L%4p#8wxQ`aRbul&EYVo?- z%*f7O@-n)uUNFpj+yY$XOSG6@;TVd{{K`92Ly^n?W9HZP(QshHVI<#MJ#lj!ddlKe zY0KIF^YC#i2)e_rrrTh0?oP7NGXe}a{i4Mu@E?9bB*6<4Qcu{};^@5lKKqow>G<ny zX?GV63}pKA7nAv=Vc(4%Zx~gz!7C}AbOFyRJCbNKYiqDWrl~E57KR(g<_A5lga~C@ zw#y_E3G&c|ylI|o*f_v%($Dm~_j*R0W2^yDm)6htTH_|-aXP!%+nn#@sXF{#>jPO? zAD*E!qWu@OrPqQ!o6EHPO83mBtvX{mJ~Q-cI*!ArSvSFa#fF)zM)+IYvzI%~2c|9* zlrf7Iw(z`?&w}2=6*9DT)U{;Ttnu~Q>wL;~tfN+iy#?>O^kZd{$YgmZSo5~Hr+Q?& zw+b&t{QM3dNyqRFs5*Ts6o8%jKM&Zl_7y5;y0RbG6r2+S&8!FRP94Ig4FdY%4)u52 zL-Whf3vK*`sgjctb7%f4&Z_F(MNN3HX;D{MHP4Q;N%5he;QijW0kiosYNyEuRY=Tm z$<jnxzH9bf0SWOp=TBUw))5m>2Mdb(2_oC7oLVbmOO@7sm4Nhwd3qd^N3$Z+!!0Ww zkDb(R<oXdoyxlp$n8CNQx1mh!OhjMS-@@{=o9gwO6q<IIxBTP|@Cl;s=Q2INxOo^d zB>Gs*Ee;DF4W!^riP>u4AJrj<!5;3npQ{zw)H%rTVh|D#Ja*d(9=96a?mVmh%xHV+ zK@b#}r^CzSbcu?OWBRh)bF-C;ykg1cY=hBrt*(F0vqnpG3AfmucfZ%^U?T%7jqd(8 zM>$78?&yr*-DP*p?*N;oM*ir5d$Zf}#5a7rxN`Rk@Woww?%+y8Kgz{N*wSo^>FUL9 zs$h+q21TNe+u^e5!-*Pk<}E*`wW4*3u||Vy<oCZVw>=lHGG|)KM?7}pBNB@(@`HI> z6O2(&0Yo4o-uhWsus?dBKDq+1z5g7=ieq(tyOX#)feL41;;LPXw;?fZzX2Vv^}y8X zz1d2ir3V1n({jmpFDVhz!!j7lYTkd9U7?4QtxV&*b!0QtzhU(Ur{AQ~)Jsc}TR<Q~ zob2bw*2zZydx7ejU`KDe?r*p$HkRs3)h@~UI3w)mrCzSx{F&(|0r#dY!{VLnh!V)u zbZt0>Y1s?<*RhHp7H4joTHKe9Cfj9F@vG-s$YTcxzS#I2#=cTI?y%0R<<xr5ot(DX z!R7HFCfCq@5ZTXjntWHDE>`mh?<Z{{&3wGbRAX0_QWj!#h@lHWW>Bz;HYQ!|AR9}X z{dM%iAd*V8MS^wGR<ax?HZ)d>cKp6c`g8|tYC(#lJAhI6#Zk3|g#~f|gd-IdbjGOm z{+8N?9)|IXj;8btImjD8U&it~PS3IvG}tHKN6IwcmYg|uvBldgt;?AfTl41!U6Q$? zZtSNidtd0+lVOIJ6pt|$4zi3>tK~V=ty8&%7w=l!{B7X8TOPSj;CiU@xtk$Wqi=8# zhR>F48WTnsX*$BaRaVY!N>L1@mAImB0{gYwF%ye}y>FUbi>z#!q<3L*`&V<ef(_2Y z(_sy0^p+)3aAbZl(ITuF*QUe1;l@YNsD6yh!N$vkHA|jUf;XR=_3loRwWAt6zmIO} zdaW#`mZ)#I%!*O2R&MX@T!^lQ2W!0{AKkivWj0>TG)#FfSJQfZFm?WGI;Bivz#yWs zsP4s-LFak=U};n5Zmp!PLXv#!k$Kft$^Urmc_?;3FxS{jnmXilE+%()_`zo>o%Ugw zw1&a9m~m@!Yz}0EpXK<ftuIw}c~bBSf^nh~q~YF2XN$x0<;M%0HYXkn6U4S6XRRzU zj&^oT$<hgqss;?zo&t~W4hvbk(jwhkYz6)lu!TZ^aGgA+X1>0Y2=V6r8!e}9-Ffo+ zi~R*wk4vkM%Rbs_oL$%=Dig8AJMQim0yX<?x;<Ek2#_qMt~;nv%I5fZ+)p#xhw%W1 z4D;%=Q%?b(v#k;1{sa{*jD(*wjCXr8!FZZ(gET??3QLpfOB${};L}3G(PnC|)pRdi zxJ*L}usytum1df!E>O==W}j6HIrnTjEXRHx!hu&R9m9N%rlX6+ROhSjhUA(|XYy3) zoHnb@8Si&8eQr(|b$;2}Hf}DavNaT6d6Jqkcyyo433%IjeO6+KU3fThl=Fd41?h5E zKjt3bAGGYJCCjK|8?`YwzruAPId{%C|1{tE+a)6Ctz{3om<$V9I*+(`o}JEHz9UJ4 z&P^?0-ghg$EWl4{xQ47d4~)6@KTF=lH)(NWDq`WWv<YE#m>CaeXt~$CY4v}oIV^qc z!HqHV*1d@Hz6x$qp)ppB4S$t(UuGhq3V+A>dmSSCT&x*6;KgJ>G|RH{<N09=tgvcx z_;D(>gLO&TN$_gDc~@;>$!Bika`8)DzjOIzDH-#I_Eb^b%E{XT8FB-nhTD|KV<5%# zK8Jty5b%VvC!o(X%F>XJal;l4(5~;iT^+TP97WSmqhuc4Z`(qYAJ#65TOO+m=WTc~ zM0|CK$>1%xLFy3q^jRX$_3Dc*3lfqp2e<p85gZ8Ky!R5l8l7!1rD&DPgNN<xS!<<N z{zFIBxAH;DS@)=j=Ej=LKNm~QuKg3K30Uxs8I|3HV)LarFQdDw5Ot;YUgI){`k4{= z^WICXmc}ZZgHl&&pVI&^?fVfuDLGjK4~u~e11tW!dYWtah!xA6>(LX1y@J9LWCS3) ztuC^3Q<7SHJ*S(Rgo74v2djFa5L}d~yU4-A5z}u1gHdLBcv}vC^XvVxG{||+7&TW; zCO~H^Dw)6Qu5m63Mo$zLr%@U}??^)x%-wLq3Wh`N1=v~2cPTx{MhfUHRZ=%gkww*h zi^OFJH@wmEx;%e#&Usn5&tScC)j=E8ynN9x7IGGn6}=#Lz*p{5a2}cvdEU=1QykoW zg4rpqX>iul^I+|<;5<ZoFkP{k5+PgKPlTl7x<r4gJ3n^%MQ%9EgLBa{z9p#@GY1wf z1J=>a&Uq*Et#OS`lk4um)k^XT>GM2&`<F8zoxvDLqSP^eKILGf(iX)*VO`q3Nf3)9 z`zVtef9H=f=J>}--J%C}(jOe9qc(Ole~m^4Jt?E&X5YQ(>QKsFJpR2uaIq@_kxVZn zs^@cpBX~>~O2Y=QkQ|M(*8(SW8X#!ROsw&2R&aql<CoUvY?G5l_9jJI8%~1?2S%1k zgPC488EA{%=cL2??nZMNrk8R8Ew)!Hx<vQca;RP_t_KA))oLw>18Xicot+Qjq89B_ z^o85_I39QTpNRdJ)*7u;`Y)>q9}<IH1kOg=hJCIAZ*3N_Pdw@hFb#tOW%mR>WG8vw zvRlr7c$K~bzX>dHjn!CV<+L`lwf*6HOmQd^lV?{mX74Z})b+D!B~@)w0ArVkAa!UH zd5pEvPiedAJVU#QG`2jP{DmQ1#z05xws7o>5f7^?c2BBPw@QDatB8b8tgfc2vdTiO z{aRN97=q~elboKrz88aE<sd8U_Z;|B7r^SuiD`1gDaM}a+G&{1vC7?_sXvZHMly&b z-)7r2eShqGe0a~#FG3c40zKz!ay)${cogrt=#)ynMIaGF4n;-|kYngk7Xake$+<a? z%{2&OVt<v@OobVnracm!$jYtGw0X?P?`8&drTBiM2w$BAWqhkk)Qb;c5i;yfY=it7 zr@#F8pYsyU#8RY-qle_}%+eFp?PM%93D!4uvd{-4+?m#d<JL1z3dUp%u-Tf~XfK$% z@`R2FHa)f7Ex1T;nI<GkmF>d(R`5}QnOaJ=MoQuB7!xl1n_Zckisj?Q!&ONBne-?T z5?us7t5gJ;aMm;C011lV<Ji~wtno>rTEtz+7Nj!dG74-Z45!J7uU?}O{mfcf^&N?^ z7m=GajGh0ROIV5c^aqOg<a+<&lECTVt3NN-=oaR~cIO_CQA_4HYIA;MHeo-)ijUPN z>+8nOU%4tTRa2MckDb=pqRI`Xm=3U$kW`Sl5uA-USK_(XsDYQ041wt3<wFUxj~8>U z3*P(n?_JNm;3H~Lv#U$q1@K9F%SE^I*{~)Onvy`Tos8rnR1fz^2?slg(G92O$4T0I zn{G9qlRvaUFz-Vt!TTo0`}J=vkIVd=Hppr(rKOS*$XI?>wVj@=Ov-HYR6&;mUI@o* z?$BGO;2m-G#HF!zsWa#%iEqn_cppFV$*3NTk0~3t#Pn^qe8T4RkCwmw9MjOlwlRHm z{4FcXtoSf>>E`Cr=gf<mJUU!P?qvZF5KlUvt4tw0bL=(CFPQZ>sIMxZDazLbFWQB@ z)+$O=M$E7?kAIFu;u_0H=2J<6AvrKo{}G+and^N7vue*q5ZWbgV~RfZhr88B73_HX zJou?^Q9kL|^H-6RHJmIi4viEj7&|D|&8L!s`i}MKD}mxUS$WojCqiU;$?@emT``@( z{$&;0MQpi0?M3*LE#k;<4HxBG$)e@pW32g0jiNDejuJEy;rSf#)C1!(V@yb47!0b4 zn3ReOIkn=>(W@T^Wed`&@n+ao(~|TCNNl3VlTEY#q~>stB8$C+8QQ#_r$SGueGXm@ z-WAl(93V|qV;!*i^aT6=cvV9VGnjeE|HFuuQ)t~d)a7zv`grRaxm5PY>k6vxFeyJ= z{XL6J67hl)zd|QfRyBY<`s)~|8&+iFO<#Y4&>foXg-=c@2OK=CJ^n(XZEz2=nQq+K zV>E=*Ik>gAQyNeQuom~pc8K?hPd|Q}D{VHghF6vnUOixekLC%S%0(GD1jw~qA~fXj zD9h#y<T=36ZVV0@ni<l3^@nmCI>eP!hLV0#6?E-$?5Fy8&GlU%oc?BS<Tl)wscN~7 z@XtD#6Ol|FhKKa5*ePq>i*xIRo5c3szj8-umw)TLb%oT<79zefZ$358&|UsOI#!-P zhN+(LbNZkO-ejp-N*ePfXl$bbNdZH9VVN@r4D8bWQyY&U@4xQn!Y9HZ+`}!qX6H8q zbvv<iZ*Wq2A%m*;yRMrc<eA1X4+rbEyYLU%lP4u|%~QH>myuoFwF>f(^pRd?E>y@{ zja{U=&Bu^2Q_DrN;;g0QF~=v@8{_4X^#)@wQ;P`1Dg~R%rnc|%-6n(}5j$+BG3A;} zv84toofxP}>zV6}@<~@ob&7^Uw1`z3yGi8k{lt&4#>5e6ZDUBV@hM3Rq<`(i6m|r% zQ26(3IHctL&x|=a3AGugjd{kt%rwlgz(JJLb%@H62CXp#Zc;L2&w{q0sKICLbr`RK zPnqg)VW_F|*E(X8T?QAkM@NMNtY#F$ZL5dBtykcShK-5Fh>LfYREb$qTlapaGr@{m zU6th!NbSQcH())VQ5Hxa8!n19fI`DrMbj`0DZ307{KmBY-j-TXt4d2snWfCms4guk zpEiFkx$3L=a2-U{eCMZo2$GLf%RkK)Esi96U$(BBYHT#5VCPNs>lNrj!8n6e3OJc5 zZrev>8p?zhGy9sY@a@oAG!#ldoy&O3MCO3lxZ2`|Rsc5vL)V5@facz&S2n()oRbW_ z6FPiYBQ0kHC_C+D^V573M&4PU_3p=OKj9!?m*h~ETc<J>s)D8{zJ2a4OsreZoM4{e zF(xmY6W3|{3fB-!41@8?Ecss!W6g>^z0p|h9zpl)%?$$gizx{R)GwH0AP3%?@unX< z9S$oOJ|8sa_fBbfHh;{YWwZSYfBh;4T~YGit#8AjT}O^3T|*^QD4_~JwW1-V*UzK3 zou<rHW@p!WUtVV&a-6)q7CVX1Z&HJityRJ_?a^UVb6t41`2BHxI}u~81JrZOL%~$s z{H)AOHtXT?*71z%-8BAT$r2qVJp(b(F$I3uuktw*_j8U6N);8!>?P79?rYgzMoV-Q z`+g)5iH(^<t<j~z6Bc2P3tCxO5e|AILK6L!R+TH=RBo(&wcE^@%-PUyH%RRM&PpkQ zi?n5ofsO*Xt}Z|lTQ;Net<GDGm}hU?b=4*<S?$j?x4*a6f2_B8xH?u7xE3V3ZCI*1 z$fH%|m<nK5kuW}Kad)8%P%iCnwil@^ps|7RFPG2x%`W;qC(TyUsWDuK-s$?B?S-hv zY@ycaQ20-HnQeu#?|YZhCJ%YTR}ypd2XZ>SIIZobt$Lj<>Ih#o*>u;J{OZ>~txX#| z$B@*78X=8LD7x;?Cam;!w0L=R6Hb*ZG}aI5PsQZ#tz?D7@1L_zw1qWEDsiVuz+Erj z)Dm*YsB}@7HXon9KZEE`%%%($&WW!S^5B)KPdMq-!gYP_8mHFkEwo?3wGf$^cvP_7 zC<isx`qex!F*RK~=EBGwGY6~T8+rxI^l-%Ro(#dJ<$S25#D!WyQW4w2A{0n+Zi58B ze9=2n7swoAlRGAlksTUrJ8Jf&3RN7?0dNs8GY)cxkxK_$!)<7R?d{^YS^aLk@#ll4 zk$u<qj-{amlGHkGTN?+(UggKVU6EV!h3mcM#?S7f`j8Fj-e!6oPy*~=M7Q9DYv#^; z?H`r_Z8@7^6IN#0nhQdFfArA2-^Ke{-sdOUf=lNo?Zcc-(A_!w2;cJE{CzLR^K1OZ z!s?>a08h#&$?$v|<6osM*841St*I#z%~yrR*+G$5$B|XE$$xSJ&{}^|pv1hBNS6)> z#$xWpUZt@QpiWs2#n#dsNb%Aks=|G))WH^GwDA(xop@XeaVjQMh46=#G@znX&VVt) zVR&?_W<43V%5G%!^aawCyyDo90%NBNJ_Zs$<C4{f-()Ny2bTS%zlMf9%F2$wv-Bj~ z_#~jM+&o3rbK~}N=B)P*QoM4@9Ve30oo~decMqwP#rqZ{U0QE$yEaBlFC+H0@b5>} zd~^uUF5ZNr$x+Ph+^q?3b6SBy|6cbHH&YsFDZEy`{(7N1#%-7NvSg_-FM|3`?{QjJ z@W|gh`BJNaNAtaGPDNC6S(NXr^X_M@Hx1qUwTGKBP0!bcH@che4$*XL%1H^12naK# zU9DD!1>EKuO6KfFY@>hC|F~c9lWQ!y{XV?Ehfna>dC~5jxK?wxI6t8!m4F!J*>YY| z=u_R~W4ZL}JVMNNse)$9wHs?2bWDAYs{4I*wLe?AC>t_-U!9Lt6jN}+$@XmiJLURO zfVIKkd=P9pd2SZD6rTh#TZ)VQYUdkjJWeJhN~hX~7PQ~=X;EQY*x0;-0Q~?E2@cWT zZFNwwvw9d^M*Etx(OWuSFXY)*q)8%cOikhxyp|yh*`KL5yxy?1W6YXoW|7(L_&Yt_ zc1-lxUX!X;t)9SDy(`X;(QwVL=34Zb!T-BrAV~NZUB~n;m$MRYy-8Zes;gNud-uIf z#9a7%_HA`*4(er67F<}^d&M2|O9d*t&bGv@rM#>>Z<hOCDYhr?3VPk|(C@zgbgd$g zIc!o1{Fr}k4oOk7CWq_e0MPc>?;7?$n6#5aSfGal28K$|6D#yICE6v`H*?HolP2S+ z6%(+v$}0_$TYBVRI|oBzJJUaDVN=fhvAt#~o5x?kbh%cVh-Y^v+n^KAAkL|K(_2A0 zjVCb^Zpeq0a(CAU;#&l-s3;^9*zzR$d!(p`9Nn2-R^6;S7i$F3yq_Ae2?1jg&&0P~ zc|oK7Jmu%ZRdUm?4euQtA^9qlC;CxN&|~nh@wPtYduGm2Eg=X-x2qWsuuNX3y&hST z`x9E8#~oHz{=4~u#B$x(@&B6z&~cL#bneyHh%pnQ8IhZBJ~kab=et@{3s(MlM=9uZ zfA}~_k2*md!-9R(2)|)}ya`i#Tz%w|m3_Y<{c}dnd#h{7ed=9Hw+K|NX#Xm9$xZFb zbk6(cFKy{QmuwEYF>u(bYd0a4fb{Jhv5gq<)Lft2iflRy%SK+*Td9`c2Ie%{;9v#W z)L%CV)U5e<-@hdJWE9<fE0$Ti`7l?a%ogBJ_G!v`?`HA+(q*x3<6T(yg_VbSFm^-{ z!Gwy}$wK7ncSS@Awm2N;T*sSdhd;^x>;kojj*7~{LFE~{|EXbkVBo6{Zg=_99nF`` z@L@vO!c1&TqmG>8?|_ch>?jn^_4Z6NOY5=oXCqrHI_(uhQ)jbmgB{}nBs`B}dc<G{ z1mOmKWVO~VK7sG|T5Fix!ACsJX7lsyDojZ5PS>oSHrFRKuUlF#*T<<#tX?e^Ua41= zy@eZY4OJV8bt@zMP>N8-cQM$bx^HXGvt3P1E5WDX4K<hO`OvO-PAz*lY-tVYOBgj} z36*7O!(Q{S>4*CoUH<dR;_A$%f>?<-??a!u$4I&6hU(mAbwjh{$uBN7);~_OeU9^q z?l&$!+%GCY=cR^H1?=w|1g<~m_At^k8tRfn13^e|r$Ogi9H23I+02wVe0hFO9^3nG z9&vKAGqXo8oQrvgjM;LQ1?JCfo<d1S%V4C_v-<O|G_h#HCn#hkPsG5z-8qkO;GG)Q zCE!afEeW!-Z`g5h*e*754Ly?*Q&E`!N--v!qQ8OhaEz#KFqRzrOC6`ApccW0HAZ_Z zsn&e_eiRAznHw$?K#LcRE&8Bk_(38LbMNB9vr<>=!opr}BV$LPI=!fp?K{oWA!N6; zj<dZZsFj&h{SW+=@2(oZtLqr{TzF+m&boUlI-qpdnsSdud^5B1RX>ZfvUt9Dl6**& z;hYHk^AruEAAa84W}n%w@E$2#j=Uc2jq$0wp^Vx6i5b;0|IEh}g=W2Q*?}|Ea2aVz zZqxL^s>0b_KkwoBqNgzq?i5v*M?E3iGoEc`)<ruO`GrWgDk{Aj=MQVKdE^jpS0(K) zHpCF>4e*hCsHT#eqjr41IVE*B%#_lww?24jqwM@V>hdBAb7Yf!e`NOiobzm7UwCv} z?D2YZ7MAeZv9Ar=WFSw9MT&)RK>@miHYXZt#&@WGOG6fi)3-t*h8Ow@3!#tQi*B#F z6z07iIV(MyIMe&m-4A0LBHE_)tlUxQ(KRu}iO(g_%;ipU_|SS&pGkZqcJejvng;%h z;SKZ}O&Z#i=?6~ri>{`Nf~o@j4sGbI2~C+kRttMW149QxL*AEaLWvJ;pEKs<C9_AU zAX|n=s|S~#KH5D^w*2^_P0wzCyNf*bom5JvL9)O#@rx{ZP>~$*nO2!3NsC@~_|LI3 zM2oYqTGPaz<gk)M&=FsBcM%YD&sn*o%X&o0{UYLx^OZ(0pR8Pnk_%*#sP51rj^W#N z?adj(N;^9xuw@*K&y%(VbAg6?A8^s!p7Rl3tP%_?BL(qdA{IJXDCLpIx7#4#tQ`}Z zP7~$$n<Q?eTU5&7elUw3|4w<2GYUNsstH6$a@0SzV?qDZCb1+w{N3^kA4XW}=-|bu zmSG?+D^4W8g91y&QC*Xv7pdX>n>xv8m8A5H#QYIUr3&Tg_h~rJhT+!a+-rW>e*5?O zK2(*h2`X{f%w|KgE9p-qBohkuXgnqDzbVd+jj?iZ;Y5g1=V}(aEH=8CmIwpk8gO!< zjFkW}C!da<ez32PXVB5nH;f>ZM}y6BfCSOR+Rw5Np+#cGNfNiOQY%;jB^oNY!q12L zTt-p+Wfb>8Ew$uLh~{?v1@V+~X8Qo$5mC+)$qoi(lhv_|j~Vdpt}Y<>ZC_!^WlNSW z(P`4mSN68J$G#pW_J@*S<SX+O5<bfH0j37vAK?VH;Q%*mq%3Yhy-c$w2x!lmzvePP zJt!txxV{VE8OR>)2w%OBfXc<OQwfE)Haa~tgx#x71mNC>K1&dCArv1b%G6S5y<tLp zIz&y*mm6mg4l_s|#t{yu9zON2KtTxqDo-rUi|#J%f%&II3HeDJI)vH}H|qp<PyrIq zsG!7%!#W2M`UXeJg*bBM`aJ!k^fm^|<&)Uw_ZF!)#(lqH;BG&HqnRkVe`nKUX2?t8 zt~`l4_);%A%m5GjdVcJXg`{rBM*hpLJo73Yff(JY9_cB~e3jVY;h1zR>qY%v^p{0) zul!*ILDX540#z7Ye<Gxja)pk^`A}dr&22cZqb40G*;fKcfi6Aef~gZl6i0z)tV*9o zgE-fJ4FeOBLFoDvv3&07S4?C?j0~7Ra$$g@r?ad3(|-6V5G(?C#nZ2PmU*g$@Zn6K z&1VTyy9<@3Xx;XI4p7>st*RRv`HEFNM)yr&U(Y4YcR5$f!*lQ{>%TcFqyR90!Cua5 z)6gp}C<W<TTE0Tti{S>u@mY;%l}qUtMM59{E-f9}arwu-c@hF$?V~NlUZV<0BSn0c zp!Y?D8W4s90s;WM(Mi<z=hv653cnqDj*Nn`cstT$$Dyr`5&{_MAJD=Sf8ZqK9bb<7 z?~WQ~dYTT)3<*Eo^=&2>U=w?vh(n}}mv+`BhX@INtM~<(=S^03=ldVnS8MNnpg<sa z`&g;MPf+||4}21zvHkagSZ{=%${J${WV2G<{mIekmaDrx!$|Vhg4rlT*9n%-6tQ!e z7G8gs|H^8IoK~I~+(Y^8hje=VSuG>N`oQMO%z+li9ER3Qj10+pk5N4p>N@1gtDI1> z+%xrX7fB3AFgILKI^!Lrr?pkc$H#|@Aj5{~JQIhI)E{^)5)c4%%gdK9!3D2vZO<Tl z`qO!}<nM1A6ZjP8$*1THZdq)qAjB+*9|Tax%6nJuVL{VRB>Y(AF&bGq=}3S;{9i$f z_)sY&0$m?_N{k)W2_ApIZUU@YSs3peKaZ<EhPJ`wEPKchl3Tdo{eld&#D^LxF+J6h zXG$zmsdxSj)F=5Q$m#$>^WOnys#K4;LHCDbm6Vi>j3`KMhp==6VN#e_(cR%9z5oiy z=ih{MbQY~H>4b%{D^Ot8?DWbqggXOKKb}0nCZ>r$wb15+QKje(;Y+89KF6fy(RJ7b zEtkc`Mc@W##NuY4XQeQJ$?eKzuLwcpCU}!MOo_OQ;wXD{lIbVTOwac66g4Y~$&}f* zv-6s1E`&fBy9_`k|K}LO3xNWeHUYN-8NnM?6vv+j-=!(Q#sDx7$<v9%{{B?rl+Da1 zd_L<v7(lwNuAUSf%?xDR7{fsY@K=`ay_f9}{k~GYghYS;wcE(lxazjJ?F9Vi5d1!c zABXI`?fT;6q>OlHTduCi9FIt>Q_6`Kf$RgcSAK0wO-{fqJUU5Ilq|OQ%-yT6Hw^9; z)y>)jU^qEH?I|&lN~cn#WoO)};wAmszSkqJ9oJ6nZq(-c%L4UQ>69Gf`4>Ll_}uTm zWV1(dK#T#951ARgo%5SGUiw@%%#f@52j%=WJ@MR|tT+U}Ha#XT<NlEH5i{vdF<9Kq zz4waA(<g!YCQ!M|q$H5N!0@O_r|?6d{<4y}0895Bj|u|hWFg&ae^$hE6|+Z@>KA~h z9sCW{Y#<ulJs*>z1dK<tY@JfM^rh|RFP{MJS2p*%Lq@!a8&3h^{5QGEurZlSyf*=m zZ$f%EAHn(F-e<{EvCH$M+qid;wJZ@KzVP=m#t{YAaw!XP$rU^GE5ZH>(QIaR`8dVQ z;rAiXMx8?m_DMUg`SI~}AnTBn{H<C1t3HE(rncYbx0m2hWN_N+;AaWJJw@NpV~|%F z@|NFXfP-7`=nEfsk_o!gl@&1Ot${{(c%B!N*a@Tq5TJ+3%@rwD-90A~Zmp_(N%A87 zz2dkSU|9gpWtTy@QvP?kehzkaBw+n4BjfGeeB|+8O?Bbk1o31GfGDV_fHKw}oS^I0 zUt;}Y@)x>=`T6CVuxI?hiv|d{5erarm9s~H%ow<iWR0&K698alv^e6w;I{H7fI8RQ zz+m-nmT-Qp)w?j`p)_u4sn+P|=*>-&6=X3)(3S&Jp`WBM3G{e*c>E2s;x;!nRMga7 zzsiuN0BN7QhX)cdA~8f<M@PrYt4ZFPBzkLS$AOE$*TRpPfrh5w*DrKQK>q*$)w%t5 zv+?3&fTj(01m8f+uud?i1)c<pAApMt*nMRW0Eq{#(U*!e4)|c@!sJ9!!A$~#3)EtI z$<ozdj#tO`H&3sDhbZN``j0s9Y;ikoBHB0+baN9#PEM+RYQ;n-C@GP}5rqVgZCja{ zp$Gb+1BeZP5xq=~S*ryL^x(iI%LF3s?tT+PF6A%E9k~mR!^8LM^x@?b7TUkfqU9y2 z_qVoi@bRV0-N1Q*v;FhYkzi_cltn;*fuIOVLJgw-@bG)^h?{|n0@IT5|4z0-6e#t; zcNW!6>JO$uAa5yh%gdSl<#%$zf`ii#6k0JclG%Yg5vb?G?L|ZawwM_I?MDvYLV%=| zHtu*7Yk53oF20ThsWp*p8H_u}g;3fg)2zB>2nxjH{&>+g^5WUpCAohN$2&<_SOMR# zTuY$oRxu<9HpS=l9D>-TZe}T(&1#WRZfmi&Eym`x6h}pICusbP;*Q#iuW3_x`;0z- zIc<?Pt6)5mHPW@KiEqs=&sE|VcU5Wo3b<vfZaP~760c}fSRrQalmVmL&)L_)5pVO{ zd9&LgDwIto{t+-Dx^DY1xwi01)~1E=kC_Nr1sa6S@?d9%{@cV%P`?QYUC9IA3;O@v zZ{e>@D(e~>JpnKm9kCUu&44NwP&O$k#mXwVumEAm$LCQ(u|O#lodqufFdxSFqo5{) z?oG{rBv@L?h>n=1e6#M>TvQ}Y6VHMY3m(cF?d%c2XnE=P>B)~curY*n#?Sz20#MBR zy5i>oFkl`^(UVpyj;YDXt~L8C?zRScm6U`84%4AD(CDdHOgZo~snP_{D%xq&r+~+t z`XQK}m-lm)e2yf5{q9(p;!rR$%H=8|#UQscQN&Vj)PU{oH8L`YVjpjww4Swt;X(jd z22@XS<=pU`CqiWcr?>a{Sxl*e`O(US|D^lWuj%xEC~`n~(o6F4D2xqr<*D+};&e(g zb!Za%VM{g$e)<5#Kn?HMF7dRf;l+#SGpg9zv(oqct!Zt6!`ub*>~#HkMKvf0!WKW& z69$Kd0LYcvc+iY3?@f^L%+H@BvAtx|F8bl92*L$Ye`;3ZDD$8s%*oQ!YXTuqMGcMQ zidig4C74jYGBEbN@oZMkpRkZYjnga!0q+fhAMl*Bvf6`^(h<v8xVyW5`qY*zXHP=% zp5^CPauSwX^p;Uz_y8VR&K{!*kO%p-fqX`ggCoKHtsHT@B$%mm;$A)jn|WJX+qg&3 z6JqEW=IR-%F~g<c0a2X5^0|Gmp8{4ju-E<&HT)gnOUCfvAin~J5MuJ62|##`T3`VC z9SxZI85@9dvazzBvgJ~Xbz%X_1{k#pO+4^ErKE=-=1=UzWv_<h@}wWIW%B)dr(wpY zk`Ia#hM>V;+#4EhB))!&`I6+Ay-pJLJihE${S_Rk`#B51AP^=&R-l+K%PN)D3}Tmb z;%@=vZSz28=OzpYkOGGnD%-JF&Fyof$|#pYp+o5Zvog)t@BlN9S>RNp@xROOsY8SG z=rJKu(BI#KFtOFa*~D794^3o-1k1u`si?%|<m3WU-X-q?0||g2&mNfuK#<E@(~fpb zLJ?C!aHSg#GHAZui%f<HSe}B8fYjyS04nJ{87g9ulWo#}3#1Z+K_;IEPyfGX<COoK zoxB3;gOhX0h^4Bg1`uP*={yX+q@#tQ`%!@C_Uo580HaGZ53!PAN|>1BiDqx_?-SzS z2qA`O*qAW7G%T30fv4$ojvxf?7N`#fnE)>67c4B;QyuzrH;t^SUb1*v_HJX$N_&0& ztA3J_MhMuPz`Zt=PSj)|7`7+=cK=f(?B?bsl7Jg*&jOlum&#g3xr!QKnlB;Y0?A2r zUF#DFC6klDJ{`Cvygn|5nvl}<XDl)6Q0L+p0-U0bjsQPmN^o#bGPp)l6v#)q%q_~o zJv+XodH%HZFR`-espcl+DDpV=?gVSG)lN#Pv($uTf*;UPqTISD8fS1foyM;35HXS? z5KEcjp5=c1TP`R9HDT;Vk){&CdHzhY!}3P!MJ`1;HPv4^Rkf3PKCCxtEEzr2jPm9a z?}6&#duFCBFvRxoSg@&S)4P&)L1sxF&|`Yx_o;9~Q(pdKmt~7knPxFfd;tvB{hzys za*{PhbY-MuE+*t~15h8=*Mr}~f_ub78{LND#Mur|{ht!o`Lmm;;P|jmY#|0@hv+^U zE*!{CCE6xEZp<-^NvdEiwk}OsdE6BTzEYj?ysqxI#-+GY*Re%^7XFk9ohSF{M>b~M zFRSszkMq-KotB%P51Sj?CQVdm!D*ySC}8;x0~h;Y{4f!XgsLQfj0=Mh-^%MVFaZ-f z#j78UT_hx#fK&U0Y!f#X@rx&4cm6q#-^4EtGp2TVX7~QO7sO<HuPN4g$6<7V#^fv^ zkYXeC%~-@LVQNCNeXr}9VfLn_)9!R1sv_F!=~S-Ek==9ff;d_nN)hYzD~Ss^;3sf# zPX@cJ<cKib7gDSWEFeTlFaq^Elo%Zj^vcyY2U96uEW9U77J)usfoCav8Oh7Jzl~DI zOO3%e%#pR(bxDONHPq=@+^7-XK@y6HfxRU4M?^7$2<^xa@5hyrXgw~3uTLJ<@-Q6S z?gke~k?V5mdrA~&U!=bCp>inm>KR#EnBw`eTOeNF`4KD*Q$qi*id(z*>Mi>F8D0%Q zLr#~`ikSO(>>XqCm-7P46Nl5GGMex<Zh^&x1?ogv3(K8OMm-YK#B%TMp{Z&%tC_>h zBPECtjhl{f+ntJ30b_~UZq@sg&Oz?i5UO=lo2zcx;)WD5MWkGR%!2U(ndHgUx(yL$ za*dZW{-aU%_PvgtT<D?gW-QTZDX$p$aum145aHqIm<x^!>SUO`*ZYAKEcgKEBkfH? za#H@WxiS4I;`Or?!rRZ6m$K?{dy@yf_-X@H#z}$*Z?~0E4aoGFA&5_&=Bid6w0&mC zr|(Vq9G#hJeElV%4AJ?Lm*H{#kv>WHYJB*BdW`C2OHmQq8r-)2i;c+QYRaEIr_+Es zEdf(vf52e$2Q@*1ok1MQi(WF<<<`aShYPrCjrDNf&``PQOy${#`ZzPsg<#9g>~zxF ztEJm~V00c^7>^Dt)@u)&X-k_97Ip7WF5S~*8@oh?kP(Tef*l95hZ}FlJ+_Taq+C*z z8BnEPyea0dJxF$5y6#6SX?#Rr=uO|Z36=hYgmnFEVeUqn+`U*MjQJvx8&B6-^VM!) zqT2jLQ-jo|s(4OJl!hbpD=x~%w=I)kd(lEX;v)*{{P&$D2FOF9>uV6~vY{vb2%-zt zUPHOY$#}VY=h{s&+QL%1q2!92N&cOBLWqb@k-st@`9Bk(d(Ou#``iFW=n3D`C#{HI zkcCz{2J?hJF(j5~Dtak+g=kV6s&8`jGlu7-WMh}3pgR;{KLvKlBV=fZ=<QF4sntH| zF?~e-Scw;}i1Zx%wkLAy$lDW%<E85IxvITPeo?lspF{lDnEIdmFsza}M9NiaV8Z$G z%r~2yk>`&mA5abh3{RlGq?8onuhOvR&%K8EE|sM$J*>U<QKy!aKYz9z$2TlD`+opx zL6yFDPxqE>6$|Dsl1L@f6CPj;2tgvjGYrEp!1Gg=D#noHNF)-`TdFl0jf9o3ED&<c zmk=RH-6+C;1jnFMos=GT8{wn(eae4Gr~zQ+z8FIyK^X>Nd|ZD79dddNr}<sve1hls zj=jxWpW2a8WWMI1n@Sf{%H%Ttgtu)^b6;~$Zbbo^HvJ-mFoP+f9LGy#5<&=>Hm_oa zK?qSo2qBokN3T!8M*&fK+{Qvx08`sf{fIHfh)?neBg`;_@ToXlN4Ez;2vEQn^E|-} zCWQaQKH{9CAdDFaE}2()$t^222CdiaJ<`~*XYJ1ZgGbgqzDD*fc|mpYbjJ|PV1y8t z<S@fvj0jIs^476Y<q05!G2)UO!!iJfoPlpMLI`1u5FFd<=)`pF{GNE;g(bnX-w{R_ zW1iy?nt9#<fsoc+IPVB!LI}qAOu8o`A|mq9a5x<O{rx7Bsj#q6E|;I}?udwpoQsG8 z)(1m*a%5<9?*Z3W?p;`DPdnT*wzsACvsW*%m^8wAF_GjNng^W|{yF8j7K5g-YpAJn zAj$LjHtQvGOVw(Hpy2R4*}A7S6pk&OQ)<vE-*4#d9vtI1zN{c?er3K&B|o`OB7#Ib z+0;Gkbo(quZGBmeRKfxXLFLhi)3d+1ciiJ6lxA8C^QsDLX+{7F3`01srE93Ad(ayU zvzW=U8kf!~F&H#Jh|BG7>l;1X(GP0Y8;oq-*WS@Ls#Pl%)f582q4ojyWWZw7))d<n z3fWY+Y>Yd5NA@-M20{_JR8o|iv9PK@u9Q!``eQUYK6$8Zz^GH#m*vJeZp-eL313hu zVT*G!E?Q6liR7nu!p>g;fx!6qH~?g2WnCZw7DB-DWYaqv8V@&nT-}fS=#iC|uTrbj zf>@fHTToS1YfZCCSXOv3WEhA>6W!gNy?s4i-z0+>vpMbJi<fIPN=m`$9B=PvYi(;$ zE2ZP(uI)Rv%A_*2T0M8}0tw4t4B>FRsrgWUe_t}0FdEHub#v^tEJ7d_i}&<&j*UBY zT3vnJT#1AQfJh|L*52arc=GcK^YaTBhCv8Cp2?=>MnXtMd1Yp12E)LlH`v<RJUTiO z3I*kIx!s;qSy7|cYXM+%Y^=4d1pyQlmE`5-Ox;WvTrPK8duuY8sI08c%*YTmtn&|q z5SK_EdS|~<B{5~07hk(fpJo7njC@;GVNN&@ve>PJOeqDJ!SP7Er=eq{XCM}d>CJ|m zih|sV0wIDpW|;1SZSIlr+{(hN!tB20?w-aDgiua-K{ybK1|zoOob1v(A)*#SDC7_I zHFbsj!R*q!%mO>Y2xG=G=I+|x;u)L3EKaj$6wfKwne>DZAw*aE-eyW^VNHolr8vBO zziVV%XVx#ic9|e>Tu^XHUTf0lRumXBEPRqHomXMbw5<8v3rF_%G`)KutH7?*s081C ziBvk&(bwJ39`ptzGHGVMy|AW4sZ&ux5kf()zw<zwLZvLIE{a9st-Bk&E{{|$%dae| zSX`ToCr)F@g&CYoB!}AjdRw}q!H7bu%C9WSttez!hEnRA@b<U##KKW)c1B)Rk+2Iw zC>D<PHh23cd}-O~xn=pRRDv<~x+dHAHhae07$b`<y{Nv-kY*%=00J(-HSXL`Db24g z))=%+I}VQZkEnH;i*LFdeL^unMP@k+!ypC&r98)r?$0704Y%7pIXRh3CT%vGC}3TP z2<8_I1|i<0@F6$#Wq3B4L|_D%0W@p-e@-F_SRVvJ9&cb*Lk|F{EwR^^<*wb{xUY3^ zXl%k_&<GoZqlvZK_W$~C`@V7iqIe?l*gLH-HUS`I(p6W~WIy)3J9BJipuq9`7k=~F zV2A6mpWnE<x$n6*+JFlINab1TZ(cw5E4N>saq<j<C>V*Y*|hg}f7|u7`xcbsW=N$H z0Eowv>vtZ0^!aVOTF1a20zd%rTdphr@fWVHD#>nW?S1U!ch~RjarR9B0009f`5l*( z{N%ywiVLz1bq-#6-{Tp1n*0pK$hhyJKdmFq_=@F4YkvEA2H~|^_y78@yT0+jqI{c0 zp^yPWk!bwYEeC%0?6#J}PJko;Qrq;mFDrZa_RA^?Z3qAm?&ur4_IuCYc1gkNr8Qgk zwLkOnApiiiRHjqB_`{oTxP1Pswj@a;5&$5PNX~B9MM6j_FZfyP#-|BFFf=@{e&fbO zGQMp2^*7yc^K^?Rg+iX6mj_1^5mHJI96b2!bN|}6|IqMYH$V_T2Qtg;H(l|SFMp%3 zAm?yX!~OR^n4N2tN!cU4ec$}<4<bI-6{}X&)y<Jgn8BgpKmPImHf?)va(oDY1(4Uz zt+?-=`>wn8CIDz@YyQr6etG%IxsUwl7x{U4gh2m5|6d;e^Okouf9u=ddf>i?G#WJk z^!FdR`|fK>Di;6s&yPoA@$K(!dEv$9_8e-9`Pu<^fDCJ9=4Woc;l6vmke+Ud$726} z`T30-w*LGVk9_uyd!<r|AQvBQI`r^=eLv5h_J6;5G%GXXERMg(Ckh3MSriRMJfm){ zNry2;2x(0^tyxD273`Z3MiV3BFa7rE1F!BM_D*160BC7h&YeHKXTg<A0f1rfwih?P z{P?Rsc=88B9sPf}^-(u?it}<m|A((^d2apAHTy38?8X1}<c~Q*0Dw|D*mmT(U;OjX zp7;K=<@v0l9E4EUftJU<`u`lwqmvK@peE2<eBYAWzIUG`%L)L2$-wjf{S+(3SKNE; zU`OBU|NCYL`met0?nPH$f>?Cn%tZq9XzTz4fLW|F=`XqYisA0TP5*CW>&~WyS6!^s zs1QPw(4DVs`P;pJnT-0wkN|)zjp?dyU3JChuG6O(5klhw&SxHZLa9-dFR32x8rZ&W zF92r%t;VRn`6supyzA;yc_Si(+{5GB{=M<t=eBlr4*?`(pv<zTt$L><-8ldNAOJ~3 zK~!k<6?b2!F=`o%_q_Su)+gRAU0U$SYrmB#WRy}$siSvz&Hp{ObK|?edinQxm4yi7 z-lon!J@n}CVaFr{0k8>Fb8oJ{>ygiA7T77J@o?<vAO0OMT=V4{z2lQ>9(maVL(8wd zdg+aqA%re8<GV9QFmpTpx2bHEv*l(H?E3(KQhMg8BnVI1_cXQaJgiSMEV||romoFy zj#uQILa+-?Eq6seB@lucUnc0$oZp{nH-O_vbNAriUfnU^@W^G-TUXYvTvYkw8}BC) z$<<3L=hYORO$`X+sW8-(oasXlA%}Bv-HyX!6aLE<lrO0*yl^7Ge4>Z~)(3&mq(Au7 z)($YMDhuuDCS87(X?u(FaMxf}kzFbkzCTk06dB>E^#>Gk>5p%$FU(GB=^6RoHyStW z?R(+v-Cw!=a*bL600@XJSMjaK*PB(cUw!@JG^2XUzOMEA`+oJ8_X_RSd#+!Kn3G)A zlmaCbFib6;5dfiE_q9FpxAko!lS|61cdlBcR7f|x-<F?cv=}si@z9v(A1^elUXgS4 z-AnaaWpmf?zqhu$w!3>?#bJB8Db1k%+Pw?92FF|b-6pN#iwjG18o9+_kgyET6ND(B zsXR-75aIX@?=?R1_l*PNf%|T)SvI$L%<cd4Yx|yfyNToYAARB4e7h9@D5Yi%+TT9( zcEdoiRr{Oou5x(-@9gjCa0WNL*RZg<FvDv2gu>o`qEJc)2M7E6`vIW3y4qwioyE>* zG#bJ1>cWXli!pR`wtIa(x6^au4{v5!NG7L#pZ<;@c+Vh&TyEDZYyQ3BvdbTO@CzEX zx}&3Q)8@@DzV>!ab!}$GXH6#az4zVLbEMnh7&Yj1mtS_7R4Prgq)S+d%jJ6U#b;i8 z@r^}`tFL<aA!}Otp+o!s`qvktvA>m<SCy9)XJ%yP7Z_VRx?C=2VL=|x)3Gtf;LxDa zVjk!}5{`tlS~btpuI>&1g*_`HGb4-e#4$3wed|ki-1+eQdEa9gwy~*U?VB6^@ytsl z#br0%aARhsZEpRXjT?40G#tEq#j5P=Yyj~4Lr0Eu1t%t#-*roNcHV3jenn0RrBou7 zF1+TVcVBp4=9+x&mrt&^^NPZnQj0x<VR6dC3uEk`@V@!@zc)R(rTVhco3H+iKFv7T z(Yxs{8~^>QXDv2sQGFQz@I0|d^{u-O_cb2LoNK=FvXwfsDX*d+ud1MPcgOnY{`sXx zzbRMBh4+Eso*`$qv#M%=H9LcqutObv|M<aQTt}Q2Kd_`?Q7xtXo^?C7{c{H^m3-;< z-=Ty6C5VA&II{D#t$q9Zm)<cyJ@5L$nlgrAF60*s3PK8BB2&upDvE;i4d1wLtba7C zD2MR;;qCjM|M^p(<8S=RwYimr@ksRDm$tn4)0fqH-Ku-8W2F*G2_wVq5!c2)yq&u^ z`|H1dh!C>(&G&~}9b2DSUszjQI<M-)CniED5)5zp*V=XevrdsAyYEMLSaUN5+xvDr z_wL$Xzs@ks6`#9KsZ*CPu4>-axPNDhd&p@o%@w4R@xjrNmf_Ok3VUguR4yCt>i_$9 z{^)2Lx#+${l}qXnLXF$@?|N>Z)~vn%R}V{NQVJAU;F5{mZ|)dw9<INsDy!tG?BaZc zMfK{G5JDWiL-A<rI8RcHWeSBU!z@+E=sCX%60Whafe{3tHR{c_^fS`15F&&)`i7tS z#dGx+l~rC;r%y9b{#-v;k#i8E(dc%&4F-cuCOg}GxqxuGy)KUr5EAqm1R#;H2A#%g z(H~1yf96a@)1b#iwmAa`q5E2TzxC*vecdkbMghhys>zGQ;@^MrU6=?J+AZ^Hi#}XW zK0XMgJ$;T?JT8++@@!U_j6K6gp~n|`VdH_F?;kPzSUJBkUnXP30Mq|DqJZ@QP@Zq- z90-kgA>C9}Y$rTlS!e^=dSq}k6pb0BYGE@3h;npcwe>5vU$&&KM6Xpj#wQU*zgu(g zy+hsCTwGUKngakBA%q|rN3TEjKxv*$sgN$NtMCL~JcvfV|HS6ctXiN_DNgJI0ze3W zK`BvbWPI|kue>)n?$c}4|M=Ox`8F%d;^hmgB`jk#>44*x&oAHp>=$#h(z2{3iHu$9 znBXb;-<S8b_6)lx12rW%8TViJ%-S6r_Y7ATX5M?lrNz1F7$LPn<_ksu0GOj|FN}^1 zjX$<#=g4I6s`+`peE8OMvpyP6WLi!4|KHlD-fCV@mH&lIGm@}?fI%pp<ZoJ7^x&;4 zDvRvlXxwbv`lEmCc)w-DIpNJnH-2(q?>`|Tkw|ZE@8IAdA%y4oKp=2-1uV1KJa67S z08p#dXR#ZfCJ4c#$CF4T07fgyD}{wL$8jSg!;xr2Fa$sd($g~x1_J=ht)Kt;>+3D% zv@Bb;Ov-pX-U*M}<9%=Yj_sFUwz{^a&SWw__uNyR-Tiaw^Y6XqL7i61Ff7mW`}XgB zXUpdLxg}ry$~PA*SfY?i>T2h*63L(c^!oz`_Li0uTdf&+xj9?6Yzzdvlma1SY|Igk zL^Ct3V@~JT*r+uvo#**odv>V}=>_@uTCI{&Sg~SdMa9ni{32_bm0_T)tg5^F$le1@ zt!*t=th`dAk=NGFOV83bwX_Wm4rFE7Sr$!BPPDeS0hScx7wL4$r09Kno`P9iHotn+ z=a+AOVmqJg>Dty^xw^c5<-DrJb2AI<Kq&>vCArQ6ZSTMGe$5qSpZVtJ%H~%|<+90z z?yxVk?oXR~o4a!>3S>$JLLfKG8s0vLBa)x*_^mO`gc(Mrl$9^8-um=A0OsxkZMDni z5uQXt5l8o+(=)m9^Q*KLeaIVl_mzz;yLwjMbIA=~zSCBegAiD=vqn3{*8Fwdbzi<M zt0)@)VT^*FpsU4m<#(=L{kiLvS~Wr-`jMru*wYgbjYr}Ej}Ia6xh6M1yS~5Oare*e zyygqHsB~&R$x(tP`X{zM`_3gdU#>OjU@8m%Uir1lFTHiOttc0OrrT}L|MDNBO|JHR z%{is{GUW;GGYF0Lj=cFl>sUR$_>N^ae)&#?Mj7&l7>3>Ozw0;seS^IuZ|=%P1vMr1 z%4}z^r)}?Hds&{~2Rht6;BdIFd+2IQmL(DhzxUeaww*nTZ<%-9gSX{Y6as*~C^zT{ z?)dwTEAG9%pt=YEA&g_8Soizgm)^JH>d)V-(Q6S#V)pk_fe_-8{F{%x{QBSDF@ll< zo&ZIFv?gWlyzKc`Et-GDQgde7IWGGmj9kOcFD!b91<5!5^rfple<OFsUr`Vs4ahJn z+82m?baHZX3<iT-F28WT)Sh(+p*tJ8Kl9ZWL4kk=NK%lYsuKOu8r$k6HCJ3zCy}xr z!i@$YuRrw0mOTo&d`W$|J=1ii8Wuv~NNito&w>7l6;<gE-m=VU)MTZn@jOv0aY)9( zS$y~)#3i|3Jh9>VR}QXPS^m^d?oLnBpScbtVX!E<e7uN{nbU$$g5#fhbHCK8d|-Kr zQmay{mBo1(Hj{kk;r_m1*Kr$1NKv-6zARs_R|7zn)wr;x7)zPHQSYF0f^eL$>`QQ@ zGS@h#wnU|rA%sfvGB2H5qL4|)dj}kD-wBrICj;YyZT;`Ijf9+uzkl<Zk^-AV!XN;9 zra9eWKmb6gR-;(9pfWexiVy_+5r)AznHHH0kGTA9Z-8YPlTNLaOCd@ajCC5NPOsLe z6(?yO<DCuNErV{(9sl;-E7Q}B092urtyoxf=h9MO>BimdzF<Vyk*Abl*xRmLIIl8a zA(xsh`l9R%Im>j8hU1AOoXv^nLqiA|9Ua}bZ(m<upAZNJ&H=|ggpgLN)oQgDlmTii z7Ueh|05oc~;9=+Y`~UgW-@p0ouYUiB-}=Fizx}Q6e7&i;fnk7Buvjef=Pk(3&!dF+ zd;x}Gva|CNNq%_P!Es5MjMeM(5{V>{;AB#%L2ob^bQ+B+8jW^#wRN@+&7ZrVw4@vX zM5A$yR##k93ZU=n@130VXJy*5?b!gb_Kx;oFcJ&~+dDhzsw?N#&ui)E8yOiQgz!A~ z<{N7)CVgGq97=)bAvY&~@uG{;t=3pP?(>J_az$B5sYJr`_xFWDL7s>Dy16Aq`IFv2 zW7DB{JWdFVyPQ2eM~aH9m6f%k`PF$0CDf2+yyN>1-0`#95yxT$YT4NG<bzNA;Y<Jh z{@SgvNDN~f4urZ7wS^s#!s?Q&0z0KN7LLl5ioD8Vtx7T0Hxl#)rUV5FBTnCsHvT-P zG+(7tE7S^xVbbgwxwUy}(DgKTAV3IX*O0SyM~gkvT0FO0u8_MO;|Dh!FdH>hi|f;K zGZV3RGM=zxSxXj`$U)lI+=Gv*REcnM{_XQGzvJq(>~y6@sZfir7Dqv#)F=TGlu}A4 z;mJUIZ`aPA7<d=1zC@;w#UpW^BSm#(=|vfb5A6te{4n)E0f|&nJg355oXbiWRw}7k zHZQj}k5QnW1D%mzc!pGj0JubQ|E67Wh}$b|i>|svrB_o*H3sdC58vsbeerm#<-G== zV6>Wx=axx<9qc+14Mi}6z2lxk8}_HCn@Z<bDm1DI$9UJi4y8m|bMYKoVNN0%=i&*S z$&itknI!yR+YyAO9x;4^FIZT({MOYOc{Zh1C7$;W1hY6zuQZmb^X6w2F0kj%&r+o+ z`WyTH`PF}J_~&bZN&n2#I*pT7c-#gjY%+bvlWunU%1;iZYLJ;#3UcBT2u*M0By}Q4 z{r&;RB_eWCSe7*!jiP$>>0_`ld0CRGGEIG*p`u9Z^o1XLsqva0KL0;YzTt3sX5Q~; z>LU!CI2B;z4@U3%{x81$_&XE65Fk8r;yTri0RCX~$j~_PC+An?Upl{X>4K{KY%9+b zEKDsY?f0?TPFghM7dXkpb&A>SB-J4bdI?69ocKO?>Vulr_Kc0spiPleAj*NK1)=^C z*E?HV({qfunZ|~u9!jV$5Xnx{>}nhz8K0;x%RTn3iNetr2?~?~Mpm<-CQ~yJh`T2P zi6n=Qu31^ZPjD(>5aec=)e0$qqIYP#I6w1)dqwDgV<PAXWfmGr^RfVr+ICJ;xGABo z$v|80$br`W7dAIHANGKWWUAPyC=8TN563`-)PaIFFmnW8$myBzN0r%ze7kk32|_Fe zb$Na|u&p~<heMIL!8A1j1Yj!0Jf%RXOe)dInQ&w_y@eA(!r^dFPtWl1FbMH1M9wNw zsf=MT0E9vzArBzNNUo6Uv^qfGcDwsWCOM7+0EB?!c$eEXFwnPa*Lyp6?`}A7*labW znY6)3lu`;5i0}^p1t4^4eI5>n+%A{ir05&yfBbKMPWi};kB?c>wZVWt8V%>>X6EM? z0w{Lx*>l@x?utah@4dh0>eVZY3QPa>)T<81P$H4&?d<{Z>2<o&l5*jGN+y#dj*-sJ z_5%m^?LT<v@S(=E%rvu69f?Ibo(BM>QgYLcx9)iN%NyT$=lW}Jwpy(%txcoOiPbBY z7ZwzW0@itt(Q3Z!zwVuP#YNqRT08c&cJ69-^|^m?#}7aE-Fxo-@fQ=(_~iJ6)GQh5 z=zsP1&tZ&(Xm8HJF(ieMFO-Z=S?KWzzN)G)Bi}~&W8sr^CPUq2^P1mm>27FqJH}0! z7U#fd^Wm;5Z(p8Wk_!NdSgg6>NKtXd;dc)Xwh3ZABw`82kr50K^81A)7X=_O$S%t> zrkkdx(h~sypx|>)0w@@Uk;tW#P-p)rmO{EAYv-$*W04pDAcPX}xPQV65RZi;h|<)+ zB8=$Kxk;nf=*<Q`LB{)>!kjh(!t<lO!!ikLwp%g_>{G#FcwTGN7w0VqNBkpQ!^wC; zZPb=8tg+8~|KPeqORnjxUOLA$>^!uu_0p@C6xEhcN~59hV8gIBTh+e1d92q#_^I<g z*wIgbL?hAEv2js8zp5asFnjuSC2~3t#C(a1ZmYZdk%tUvMlQ)Y`yFpT_4<}4H$DC9 zHJ5&7rOq@J97>3lmWam_u>_@5Dwnbn3Bw-q%taUxjwBKZE|KIoPNtBv5|)*)#};pd zCK7RkkVGb>l*XemF2P|2OXV_F!U6!#@$pEE=QxQ>DpknNR7*mL0gTw0+LMq{GZ|0t z9M7-}E0vryQk#(cl;^l;ILalGGKE|!lVS#o*N=$ExlEj=cB|o658qT@p63gMJNrl9 zdbi=FtxZ39{5^#5S8iWv(5X_dipeCG<oHCAr<BNLQmKSx7&NuUPV&9O;{ZiAgFMM` ziCBUHm9h*%2!uH;nT#bkN{LJ&k;x<sgN4^vLWnmI_68y#MJk1y<4H1}lt?9~>@h+- z$4BGIM3R%Rj7-W(r4k|LAOIvep5qBdNG3g|b;RQwp_FB?gq=!3LkLZ9NuVH=u=s=Q z$`K&I2+xq75F&)c<H=||$zUXtvNEXz39l>1L8TG~0Ale(IGSJ?tdL2iQf9XKCY>vY z0@i6^@b>*3U{K3gcz)yl7dAIgN+lRYqcK*4_8dO4VnLNgGvg}(P5q0}l({{|$6f98 z_^UN_&(JfviV@}ccrpo6WK>Bp#wXbc<JPXhXV>j|VnZWl@b!xd?_60uIO=(2d-I1p zegFVLW|pGkg7pfkFN_dBqto=bqre!mvq}Jp$Kx)SYjALIa&q!S>FRv@kT@HThEhng zrYjV30Nde~rt<Pqo(HvBbH{CW-E`w;Mn;C7f8m)yhgaB`5a{e|fAx)*Uw(a^L8Dr- zXu%EFUfa{twQ=)ixl(hQRyZ!1<dQO(#OWOKdVS=m7R<72c6PSelE$(Uo`<5M5&-64 zQ%^AH_e@R>_V#6FWaj4=6qIDNcXkAWftJ=LfQ;heB8%Aq0P%R@;K98wz5K%Kuf0@T zzo@#h@`mfKYivBYd(Ym~HyWOYg$tM1ateGtUsH4AMN2MeYHspP^j>`Nf1l*>cTRGi z-4>rRf(B7gU6fOvS9|IF_PtGSJ@z^)W1jxa)0cnt8ivJ*coH+<9&!iVK{_S9B1Xbw zmS!2#O~PyHR8>5g9$|-4DpSgG%ks14qz^U>j`j^}4BEctZXGCd%Ja1*J)N4aDWTNS zJ?!iqqeo|UR>Bt5=gFpLfTBPNIrk&@oE0V<E;+~*TDj3`1OT4n5dyhR+PbTmOHAod z2xFB_Tad9>DwmyV5QV7yKmkjbliXDZC<Q*rA&h2xjHZ-oESk`OFCI-K6G@3&npK#c zUt75LiFaJXW6@Bwx2anV%ACqPz12twC4@xAV;Z$=q<h%e?-Wie!iZ(rytz5Yop*#0 z#%CVMTSS282$x7C;&Do7W}&@eQSIi(-vI~_N`(`L5aKz0sH3mvaHrcb&Lugc)s$ON zSX5UggsY>Jh9-kUo&Cez1CwL!L@che7_thoOXgPS(~N{C2;<2y*Y0)O)q2g`l}pC@ zM>_Yn1Ux>8LRM5;R<XDi0czjV)P1-k5(w!``pS#ybIS@?ndGBAR}ey;QFm{1_fS_~ zG!#*4RMzawlDU-@+i^~j!gV<@GVbgj?rZA_`+~*{OL2X9US&}#44;UI$T<k1fDkBT z5`|2nNHb(u4V8tqU?l$ZmgY6vnlGv=xpZL#0L0_TQMa$5qkqKdnGA#pA(?50n&O=L zvRt`R=5%^?AMV-H)T>U{L}Kv`?;URH955TSm&_|OXjP-m$(})H_u!age3DRV(rIeT zau-z>%48Bi&^s{x)_aX(o<O=m+T1<-;)dOMwzR7+u08473Z>wh^dIgVJlrwh4@cDs zS)MIzVRcb<x{+ZS!jl87eT|(1TD9`3rL{)ATIl&pZ|{x7Vwq{i>n@)M91j@x50C9Q z*vawa>WgbLt;UZK?JF9MH+BxT9T^#NPOuDPvl^>Qa%xJlr4shI+E~Q#w5faOaOc3l zs9PbEmgQ$Hswv7!H_pZg(Pte|z&bSuanZ!i#%{d=>osb#UM;LHF-AU##K!TzeY5H7 zcdgQBX9!p*wR<QMn+(M<LUO4D<6{EQjHTyfAQVsX07<i6J6#Prr41s(s1!25xOF@f zk0${mn7+0|C=!YN_2qXTd-=eE>hzy{@v4Q@g#c*7&ZfV<-I&^n!kKBl6x2$&M8^6C z1IZ+x3hmDEygw8r9G|9^GP6peIt{^Ku)n|G>2$_ou@ki_jBy|k7#|-$KUQFr0wEfN zRA;**9{{D0my@SbD$_GfFTC{P?YG>D5J;tx{QP`|0hv^$RH_mQ9sm%6P$=-iOV6%( zechbeiZ49yU{zJEPOJO(%g;aa%&XZ2(*&%eB5Ud#OJy>JLNV?RU47M6x7>7_N~t20 z2pdoea)m;t)AKx(msJ8J;sG)^G%!3oY)Q9fW!dxc3)0i9yY?Q8M8n-Z9l)w9$}6N& z832rqj=Z$yg*V^c{@J_kzx}qmvh6uMPrQ?!w>G^y{Q%7-%gr}l`-k8Et)cOt&6e$O zj94<tD$A>8g@A<+Boaxd(>=Q!($#A9xe!QB@UNki@;r|)%E-;qTZ~?(=Z!~SRe*M4 zWZYhoBb7@Ney;ZRxfkDbnL?$UKL5y+a<x%=ni+{wnpt4auFTo^<aUqK<#T8CxAkOJ zq~%l;$dqybiV@a<ijVQjzjVdi%NKJ=Zu-7Dlb-PWCp!eQ9L(U5H`v|Srjf}tCao#M zf-sWGq=ct~!_MzK^;5M`J6-i7gjllEj|!2;J>?NXTp}3_M=*<2Iwd;s!Ab#Rq|~ZN zk|yJcL^Li;Q9{l*XM+oyu}ZI!DdmE9YVn*3`oyO8Jx%2cYI_cMSaXd9wIx&Mon^E+ z>O?qt>6cc{U$vA=B&Rh(H3lsq{IMTIK0F8k!Wd%)5T%5~!jWW%&r)Wyj6|^3=Xt*S zQ2XnDeyREOmPv?UU^w6wTw1aE{_8Hfeg(qVH}2W7X4B4Bw|DIyj6wu}YC)4zmwE9m zE3W>+%~H7(A>^O%{^47{s;ZqEk0$oM_1?~n2LX@?RE5j(6VX^K99#3d=Q}%kWgu6B zV$ofFH+=Q>oYMS{@a>9Wnc=R14gYw(X<I`_TR%V&fjlqAy6C2hR^NL=Mt)Z6GGU}R z=nZaqX6;bxU}r}MfFw<7&R>#u^%rkke9f}+WBDW^a=t<Zr<|k72Q6uayRW_Ii5K>` z6tcZ7y_YVk1OzQT!+&{k^SXV#o*|!LpA9DMs#@E3?_IrYQRSG+d&@6hH%mCHMjDR9 ze)hsna7S*vuBNUuH<9Fj{O4Djx<?us-2e#y22`>|mDZm;c+F)CD;Wvf)HU>j->%Iq zFsWqh&gOxa-)z0{n(8Z-RGq}v4x|1d*AuU9-?X=*v0)S-3BX!&jW;hVzvtSe^<}vj zqhL7lm7lE1Dak0!%`_S{K*8bi-t&j`m`ljis>sR(06Z}GaM$qtkGxS|owjUV*;IzX zGY0_xdTG<Xr{CPQqjem7Api_U<+AehZ{EG~iY3+3X`dKPR@wWF-J5ndZ`j-e5Cc#a zml<zbUh|E+uCSQ2qK5Th5e2MMgVEux!M46}pNIeJcQ0F5QzATiF~(g3V_*KwzoFN) zZC}To*Dsw~g<>Ft5=sGKU}!v==oxhMkA@djq}kGq5-A&xB>?~-pp*hng=CzeV6?Gg zC=`hUDD$$+Q@764H#&sSi8>Yqj0)^&X;!r>5Lx%$;o8z1EMbJt<2=W+Qpv#R#P-I1 zh{b>Wz^c_t>wrPYSQ240{qDdBDP$4=%5gl;6Tk@Q$tCWR-1IEHYGB;+?!nH&ybK}z zTfbxCK-(YyT~e8&QOVDPC2lgAY;SLOI-REq*?=(~8X6iL9Q^eA_DO&c*s`k9((=yn z^&}yXo14G#vK9Yc^UA@!dtQEJ&6TUKQ7Gg9Kq>G%PY4l$Nh1WoVDOozU(3oiT)FDX z70WLZUH}t`I2`r!rBnXz2q8oWA(RT+sFbSo^h~)zGB)Z^D3k>S`Kb=@JYWoj03l$p z7*}1nV(0FCP0fw{0|TYSd3KvU-J&ZmE8VqoTVH?g(9nR<s46R|P$=XaM+S!m+S)p_ zYS~RU-a2<qEv4Y`_!EgZJI%U+Qjkg|mtJz&+I4UB_4jStzIkY5^s~3$l9rZsj-MI< zgdh|OwRh}cm{Xg1pJO0|)N1vzWy{X4O(mtY>tLH_)K#~9fkLgq2vGpRF^3Wwg@jzC zkSXOExi%muFfpl6D{VzN=;*#Y&oc}|gan+YYJyO$QD0JD@dj+_In+s$$k{)dU74Gi zZ(}43B~+nSS6xxswY57O2<g)dN{vdmNm6MLMU$%&K?q@tF-ANOA#ZTk+O3D*IjpiM zDwfpfEJn=W?D9OhN-hEE_|RC{!YY8N2m7>$LenHIj)y5<b<}&fYov1sF*vunK&FsS zo!BXVLj(vhEL&V({?<P>yN6wa9eozN6)_k9^)+=lI7c?l$g3*A7*k3SLiNk%r^^iu z8yYJwnlsk#$g9h<7v-=lO9@qLHTm;$8((Ykx;!eaN@q3*tBzymeOAmq=PE45(O_h> zcQ`odkA|X-o}rzuZ&hc@SAG6!oyACj5<>c#dtUp~iyhnBmpybz_0l;?t-80lYwhn| zeeRK`vx;&Hs*4#0``lh*hUtOdEYHZdC1Z&_>)&hH*7V8`)|4--E?Za)Q>RHD4hCQS z?Tckg%fIpH!~O~1fsMPzj*LG4-%mxou_d3m_?j=@Jk;6Oxb5J<*AJG=E6>Qyl6^=^ zD1=Zf6n*w*e?RzoL-yRPul?>T*4&KI-jTiQc5ZxhBPH~vhwn6`Pvy743<~>0lVhHx zw_JK#(LDi=Z}YS32O9_0{q;3_ajv~IH+7APh@6>e9ghG2AOJ~3K~xk%NIV{ASym#M zHH(V#7LKnfw6S(eG!g9{ba9C!D`i;*503k8SW>d=>!n7Wy5He`apV4tje~Oe*1ED> zi&6L3moDGDw|#HxfJLwR&WbtdW?iO5uhS@ap0p24R2F1Bc*~`k7X7HpyY}6Nw+|2h z^YtACHfv#CdQoorx4(G#p5`O{quxc8IrTT^mgJ^OSazm=5<>n!_<x_?`1o55fY7hM zcWt5F>Tr42ymR0$uOAAAV?Y1WjX8E}c7~~_JiTYs-#aosuetyjy!&vElyaFGW}w$S zIOxo=TawX4_n;FJ;T3ao4O-QQewju1wRiU4_nkF*oAmzGl`9ujC6e5`2Rq()zvKIV zUaMBhE}dVIdW)B_xUp+^`P|}v{^ky)LiWz?<~2Lo|L6H#IT@xee&*7T<#b(8hyvEB zA+)2<(L5TMQ)#X)&dJHPg7Dl$VA85=7M<4?{^`HA-g*5};YUz04PC=K4zw?<DP%C- z-*V&+|9&5&=;AqrB{>;zRM(ZVcxc?eVf(?VB0I;EEqmM6?&@^A<3IZ9a-&uyEL9Ok z5&*Y1FzE|xH7D4}5K>uayR;^I)Bc{HKCz=TH)C;aAwuZL@ObaA`}#}k8H^bWAdw6> z+#}8jjB$IfWBrasz4VxhEtRsFX?g${ar+v(2DECqgk{o<+L?kGA@eKqFR96K`oe#D zX-9RjJuA%+izl9YYuA?6;S7W7rWN&Cwc<SZNJ}Y=$K#aJ(*(UDgq$R%>8A+BSm@q= z-V=1ZPv|)dr67|@ues`muCC4lN$$^&|Iy`gE?#_*UT5GquA%Y3=$M1@WSWNy!}FZe zIp%bZBZN9S+ct06l$EJZRVbB8%@&JNF6r*>X=pf5Sy4?XHJQzIwR37JiuN6BT)TG7 zRaahbGFd1kUa!aHcFmu+NGg>GU!+!FaplGh+ge(h9iwB57A(-|41_{`{aj0C?%K7l z4h{|1R+pMBX&4L2mtY$KBO^mjXF-xn?%TV2^VV%f{mh#>E6aY_rOUQ#+crM#l*y#k z)pZJmayFbF0m}3I?6Sa0CX>QT{@LmP#@IPH`p5fz7mvs1Ty<#aO)HG)W<b!_)V<|j zn~((Ztr=NGcDYJXHot0b)y{_XjTW1A$#u*1Rukcg&*gCqIp<uy=yc&B1Q66Nn-AdX z*xT-R`J;h&VQq;%%_O*cTe7SRu3Fr-sdL}D-8!?O_L8}Bm6GSUp{{<t*_dBlH2bIa zj~SN1VSjM6Z`e06=^7s2{l>c;yV^pI@cdinFIat%QlkWbtOEO-tLIJr$+z)eZ^~7Q zys9Er!p6hV@uAVQ?2L>&n@|-(kccM^ZP{zgu%y}3{ci8(=iVCabm}rSRZHqrIyC^r z3`PhspmsaP^CN{arMzNsZFyb3tKYTx+4XvhL1!`eT$5`aeb%ZmT5>FNRxS|M8iWwD z&3f5aFMIp<>)-n8tNxL|Mb}<x%&-6erPQ2dtywnr&?}Am-`t}!8RoCLNTpSANzTzT zsM2ZktBcMzfB8=rsX@}$bmVW}`Xf-_6I>)1iAQ2Jm(|rSo3GNUfI=)BYunR!=#9oj zx6QxmftzwF3K#|#))ae2U9bIlee15qg6d+eNx$;$YcWQ~42x2uqJ-+xjP4;<ldIEl zWVo=dR4NyibeQtAYI*H#-?_(>VF~*~L7#v3nw`F3|Ls4!>*AX(S83HlUHv}S<lgrE zu3={^5<5wD%hQ4xeBhnkjjuOpGu5l^zvhygS1L8?us@_RXkT%?@Wvn3&%I)y-f9#! z=3_){(A@a7JF1t~t96=qG^Wz2{`t+nIa){hT6(eyvu8!@0+F-G$jHc%BS$P2OHomg zT>jy%RgsS!jPc@J<Du>mUnm-lB{UMIs@VR!Z+s>z&5&-<gG|DOV}y{w@ojHx?{#~F zh4!>B-h8Py5P5w^S9ZGb=F8{SmgP_i5{5wtp8WA$I*r0+H6po`#uIw2vVHK4y&Yp? z?#aU3^r}MJ-PbO0xIDXB$LmUS9=PcelTOXBCq#*2@b1G$9^cdgasJV-uYTyZ%j61q zA{N(cl%1n*KC`Czx=ZSEvMffOX63y6o~QQ@k4=WdF_l{0)ID5p*XdNUf!;`C*I-Vr z)#D2eI3|FVR}^GwRf-QgG=sygmpAPNJ-T*b(YHT$WobbcP+C!t6%5B--qg8q=i%z& zY?DzdY(5%FKD?s#;XAG<u%`iwYfJ2|$<SMe`gb<;T=oCiyYKk8&MJTW=eccqpV4R} zjk;XqUJ~1Br1yjn2uTPbgq8qH-(BFlz%F$cSa$aZgir!3OGpTvKoSTE>D?_ZvMtG~ zw)&{ejHcXspXc|-$ck*sj$_N7;?Mh)S6=dH=H8>ZGq;?3p7WfyvMPdsgl`NISVzFv zP;7EX`#y^4=fAn8%<mz?oVn3$vuYcb)o$+`Xy4x6-81BK+lfe1S?|!qKi>7HRrO_( zB<<=R8k$H2z0S8^x-Q^z&AgT|@>uAGFW!It;s^=a(>u~Pn!NhFh8r(im;08gs$3Bc zrH$T4UTAHodga46Up##(Y0h-l5|w4XJKp-H*m&Z}o%=p>*Mp0zgFd(Y<z4$#Szb_G zwz|1~L3L=y{?TvV_teWR-I}62v9;Tw$sU_Fb4ko3sX9`&{>+Nbf$?wO_xA^%+!`$p ze*Od39B9n*+PcaEp4+ZHtFv$9>74`b|MI;{>dF#RnHP8MH+A-rx1M#@%BGo;OH&B* zuy3E4D>o~6s;VwrxUjXgb!=>m=4yAvIWI3S4+H}F%KI?JVzJo3zyJ|dR#tkw-V=;O zRs6-0BB`%$yz`EKnE2*bUwCfo|Ni{{9{A0#eLim{lTF0q`vyjqHP=KUm7J5y?Y{N4 zYwx}P_mBMbFGIt_MB<63Us$rZfpae92JHy4+a2|FjTVb#U}*UJKfJ3tTI2P4KlO>v ztX#4B#v5+_<Ujx0z4tw^ar35y3l<rs-rCyQ-ygf{JKtTge3@yI5VT_XN}|Nx&W=Pf zT@|f%I$fqoOBOHjx?LMLZB9(3-gMsi0e?Vnl4Yr)B3e@&jg5`{-@QNGymezLm3rpc zmjiy6Y3he;JFm~bY}v}cJ^uLc$jI6?YieuiB#9Np))9hiHv5VdD~c<DmG`MU1rcLx z>7`45`-2Dd-)FY{eVfB)XA+CY;^ST8WsAz*{na}h9;e`<e))n6-+B4ZKK9c`@A>P_ zXSdg{Y|5rHT{}AC{c&HJf6=<-xy5laHy6jDE8-?Eud7_X^1QLWk*SHK#N^7xYP)NC zD57T7&baiPJ-6=q^Uwb_H9ocViH*UEV0<*b?a^(G>l**{H{UdKt;7Wh<Jg)smO8~H zw>-Cf*R#8X!41<)4kw3F6PLdIO>g<=%@vK&Tw89p-+RFgmvwAyZ+~v@y`%TkEU9yP zog@24cK&hK4gYcDHFvyCwP-{{#3Y$*d1Uk6%^lIj(ec5t!Onq!fwA{|@S25dm*hU5 z?tn+NYUO2~NACH3s4CRFer32Oa?OX{@{51_za39(pBVnGKN1-38R~1<?+UrDzVrI% zg1T9oNt|`<n;!el-(r0+yVqXRT<>r@a@$?i)H5zUd-tt7{`~WY^}8S3^}_bD>TrB? za_3Vm4Qm@eb<gLuT!eyzDT-hubAb_pRErv@3Mi^#>gG_#&}2`dyQTB_KRw|q4}>bq zrs9eIj(rB16?IV()Ze~OkPs#h*Ocp|j}46I>8z~Ek-ADnRZ@vm$F{byp^^RVJ;a2A zwD{<Rp2=#~+^0yGDY5LFRgt<%ZgPKlARLWk#|?Mbv-;xoZofwe;R*Wu<w27~I+2<^ zT=ZBl#!|^tZ)*oJQQla-=*(qSht1G+o6Fv`x;YRH^|y_*ZQix`jOCIn6Ok#n!{w-K ztaiAahGD1{t!`;!#nMR6rry}VNO~$|bJ~mI<{9M6(9qDt#Dowc8jT`>m4A#BiAjv< zx|uah5+uv=nk99lDx<>_U46qtv9Z3PQH#oGN}q_Qh%w2c$g)IPksD#4Y8r7fcVJq& zpo$cEay+qjPxts_qHF(<MUi&*CR3RV2_Z?6RaGU%NEeDMSv6Hr<vBtUVq;4u<GNt# z^%t+T+AL(6nngQ*?V_c%FFYAfJ@nj;Yc5#rb32zbR1nEs{UhV?WL7si_6;woFRKiP z{<^KNy}OTyjZP%ndxx4Adn?KUsv^I!gB@f02S;0bM=32XUbkpLO?mF+OB$;-tZ4Yv zGi|?ma@TvWKR*!gOh13jsI3fDg?+?`OkP_Nx^%;$KR?>Ge|T(Ybh4@<Py)3erxFra zN5I&?$k@*wYa!m*RDDgr@1|J=FOn=>v~JPYzWX3i*Pou+eEW6h2|-z1T(G9eWw+jS z-)7R2MCQ^}b$5UIhE<De4-8!*B1&e=YtC6%5%hlTu1AT8$h_@_^`Cs#Ws4gsavyz% zRlD~5HQ)W;zjTdQ|McvxTdqD!mF0LcL%ek|xtt3kL6XeQTeHyVuzux-e}3>Uo4Q&@ zh)g2w>z}#0vAW#huzvB*Yo=1!zdW|>|JsI$)Ng#|%Ek53Yd&{B-#=-XTnJ)9EUc}# z_3HH>{=zQ?d*Uxs7ZLl&%@=uH_QX_%MB8L?HRmF?u5(u}{PsV;;|q8H@xe!4*|v2b zkx6df4?lD5br-D(c%8%r$z&LOY~KKp-7xsfU#lDD{=Fkaanm#l{F})b3x~r_r*rS# zz1`j2obyAjZ#n0oP^iAX{zywW1tdw@zkfdwg~Q=!G<qV75`rctXmYX$I&kL>fvZ-o z{oZ$e^rN5r;CtWwF3}K?n<z<Cam%f5zw-khidNN_Cb?a%cfIrWSS<GNpMTo6=UF29 z(wBd*{>-y4y=*BR7{4P)?94OPf8f3E`<H)tUte$A^OUG=IOpb@Zf$C8xb~XsC0YK* zfBovyPyXiVCnX|I<o?7bKU)#0nEv()^7;JBSDm?S%U_77vZBi6atJ|wf6(jozVhNz zL=>(1NFd<noEW3pn!4NG^`32?{`4y^J^ac`4-+Z>^FO~J5+DEg9T~37%)%=KSuNU< zCCe8rY<%{`EsGW|3I;>@d9U9%A;@aAR@W>jj!6LuhTuF@6@KTxd|=6jmB0GheY;z? z61j;4QTnQPzWLe@zO8Oaqu@daVRzXtc>84%lYaf>2VQ!5<4aF(A<~KLANuP1A`MZ& zImy!0<kZ0UtFiH=b6y!TMpu3Cnt#9J!$h=l%@xrFwUQ)FPhiJ+s49Hh7w(MIRsHBo z_qMgS6Uju$wHwa4>W=GkjgUC!1Di*NvfDE0YftASmJlKppBfl{bzs~@WDsQnw0zx7 zXT0ggiyK!g^p^Q^^ZODJNvv*Z;~n4p_+20RuZ=J7+WBf55hoH?+;qi~vsTE8Jku+{ z;;^1|{kp-f{=Yo%XCfQX#Et*(-fQo?(dMw_Dg?vPisk38+VsNq{X<(W>bQK-8Ou~n z-EhtMo{;}H-@5PRr=KUX6UCP;KlA<H`f$_gMVdvMJw;ulu4?@|&wS?IXKF93i!Q92 zDZJ&J2P;Fjf9b;u*EWCaV|VTC+)JborB<vy<C5F1QWPckw{&h9Owjgp@`BTOyB~+_ zs9t+j!`nW7i#y=uoO6@6Zfd#vJ>Plo8xJa~cGEw<dulQ{)HCF-aK8G~#*XcKa--^) zB#jS`siY()lIc{+X>kkApa0WSn;(CnXLnEcQ@u%%4ph5TSw8SNo8IMegH%hdMpzJI zvMP%MQ!(TwxMYmUib5nz!+b-v+qt<CljDi;p>bkVRvWTA9S1(V+#B>dy-tg)40QAx zx}nL_mzufze+Op!6`IBB4g0!`o_K5`lgij!c6^%PC~_ZB6Vsj+2EoO?SdtN0EvnU` z5E04J$jHRq4?Nb^)wiQ_WXqNj@=CP<jilV5=G>lW=KNN?)^}EtlF9V%9((24&AT@4 z?%TX`j5JYOrcqYR?3WHc)`(eG&j&E3r}g2{xG?xR%gY@$>vYyh@TziuMbKkcrDwMH znw(oTH5~9cE3NH4Ly4*M$XKGKYvg^`tXk3BwCss39s7od21f^D6Ycv(*Dj7!hW)QQ z1>xZmQ&U^|CdsXZ171l}$S{dWvsudg9#ZA5JtO+8zM{lQnz`Z<LQ|FMazBwdI+;pN zceI2z012!k;C%U_+JV3QJ26Hkr$Z*R7UIGU%SG4yL_~s*jZ82iikp77?ellM<!c{% z+s^jBkl)!@SDEY6Gy5ATi1rO7fBLPDeCpj-wsj4Z`Q3H3<s>-G>I-nmxywgixu>mr z!0&N}{T?Q>yTAPYyT5t|aY2TW`>x80bjI?AKmOoTJzWEXu?bC8mNiyM7L`nsIA6N3 z`jPK_yrXw$cr?DGvC`>v66fNQwM0awNhT*DtXAzqw_NtltIlcb8I%|;sEs(BR^sB@ z|N8!KfAOGoDT=(Jx%Q9W`{ZE%Xm|gx&8jVHswPExtt!nlFF$Xk==v2AkzrD<@2_FL z`I0k4*Lor%!=PeleBf|6mM>pkQBkpD$BxO#NjkU`EAOK!oKC09<szaJzblpRKjn=P zf+R^QD=Yig&wc)%KKa@GefvgZF`La6t*-HU9DMo{O+>V$x%vBd{kZF&J~tVkT(oep z(`n(Hh6V)@>H2gQ+Uaz@@4fH6_L{f$^ma><R9{!`ayd+stX9iSZ-4t+-h4xEZ+C1o z=5o23nigm(>H74-xu9Sm@V7@EnXZX6r*{lqulLV?_(SeN&Pms&V^$RTq6;s0_K9cO z+FNBwuB)$eIc-Grj+<{3f(&Cip4<x!Lr<krHC5r|%U8Rcu3|gIFRIT|!EgsW7v6OF zg*RO>vVUlJ-=GkptR@nS1O*o+pFVvTLfBo7tM7Q*<+s0iptEm$Xw>0xMi<tq7S+%V zA%vdQKmD^We&*+27D5==LyP-}$k2_;ZoBr1_q-*yG*dTbU&IO_0u`ZK{@;7A`-hu` zdk2$~iSn9?Kt)gpVHze8mDg6>H~D);j0<k+#_4%cA%yUi2R?E47e4uee>><=B5rcd z&4ZeS2|<k2G}nLY+3!ybkL_#kVZ<sLqMl$boiW`XT@aaBzT~Xs@A}*w@A}s}rzR3r zP1P>H$I$heP45bL-tpP@UVO_{6T{=>^_9d(aA9-W&%FHX)fcVX*VdC5oAi|jDjTYW z;N0Y|Z5@^yC(fr5DLrj2TDLU1ur`~{ycQ!D-ZKBycieE%JFXhuH<%coEQ>~hm1RU^ z8iwG)?RP(v{R?AEaBiHg+x_^M#5Jqt3Hm%CzYv@etEi8*Z)ttv7mxnrt6#qUBR2~n zOx+ZMtZrMdri_Sk$)d8laO0Y$n#Fahri~B9?)&#2|K<nxEL?oXs!La0{_blPMS1#n zkG;_I%3&)!kSwBE6ZFhHa$IJ=S_mNo7hG5zHd&Ep7Rn^X7%_=VgCF|Z*>dw1aiLmN zS;n04@H+f{|M2jz+wDdVto&naGMQ@K(e1DD)l~!>PCIe&(w5fsAN~Oi#xFXr`ug*i ze(6IEo%@D=^2cY1Cy%x$iN!`ISH1hcdUy3SEe_vy#hTB)CptVD|M4H5>FP|sp`e7w z9qELk$V9L0adKUwCC1V^CqX2!#r2gN8UsCp6Wx8W@%Ys4kz_dNU$wA$?ZWcy?E^d7 z_w65w^-N~pQ5_BkJT&LTNe8!e&vmAr$<GNPh?628dUbf<pZcNqe&*WySv5sfqyjJU zC_qSH9U148J#x;)JdmcD^|9q#WU{*7@87Vxi3A;*_97x8g|VUd6zPWF<5;(<aqg!X zrqAOzb7dn5BF>2m;)gv)5Sg6JY;{GTx-vK&@Jz1P<U~|o9j>nl6BjgHG=A`WQbLen zx?Q$4%j$@TxR}l{=|@@@4~N|0GS75W`oV9`%o}p>ACgN*7nfi#xOVN@&d$#L`}b$F z*_l04-p}BJ!QkrEt0hVDdcCjz?2^MdWwWHJN?mPjeO)ch$`;MYb)ITysAuf8-<pSg z@UvO+`MiOEH@A@76NDgLCs}57b+z^NwIqlO%4QFa;jl+$7I?Va0<Bidisj4ZjFyPF z5Zo}6$;9?;TlRMJoO$Nj+Ui<Gk@NjogHu6pVQ_<peC2+BBuGR;aO1E&a4s`Nj4B$c zDjTDM1m{AZH55pHy<tIyZp^V`KOomr&uX_;FRmqGhs3}+=b5=a-=`UI@O%EcxXHPh zb$FbM&R9xBf^*}r@Beg`P@<})C=nB!>j&>j1m{d*k-ExAeU;!mmnIWJ7`j1>RWGV# z#DoxsEQEb!Tvbok_Mwp$5e_X#DJ4iZ2uO!C(%m54pdcO6jndN6(%mH>-Q6I0Xx_p9 zecw;KAD;Qb`JFTDJ$v?=wXRvS*19xMTQP0WDxAiKHoY&S#b-iJCKCgFO1L<=Yo-`R zKC_F;fOU@|`zGJ8p1f3xUzzFfxkO^X@_9XBOb}gMV)=}9I_t2R&rLFRaqVffr2~@7 z>(aQV8g@-*y=okS<sa7T97^w}cqeo}q<8A?FV@?x8ZsN1rw`{t4QuzA=tFQmJ${Yp z5Iss(C|hAh<v(cdaHS<<a=Y)P6#POkg$$0AJS{auo{$F?cWt6pBrjjUdi%OR`(0TR zG=@51xOkf3gqSZjy{?WYuzlIS7EM2QX^bFCt6+mOvPm>~aJRC=yo4qbh;+NWZHDN^ zL=Dq*-fXEx^^brcGI#sef@@40%W+EOo|6cXX9q;;J|^=Cf;StX!y3XNH*M04S~!gv z$H>CT1yjXDF*&re;*RZ!a(pc2W{#e&eEBH`PH*Vcc9rq%CY2Fg4v7~Rnrtc4LwMGY zzku!3Z)Jb)djmC>#Xo<xs2-22#!oXim`Yb@D5<P%-x6jS@zB41XCxVAOj)kMfG%=A zF%;aP?4B!I{VMDTED0JU&vMT{I6X-)|FWWzLt>KW(Tz+g?Ex9(woqL4lQ&<UIZOn7 zd+uoPL}Vb0I5Uegkmr0tQHzA-g9k&^itYEe<BAh1nyGT84h|KIhqku1<mBYT9{Va0 zRFWTso-Hb_CAl`nGn8KYC*_f*f8ak)Ag_O3927(q%)`s01FF%q2raHg3f7gHp1%It zlwE%8j~pn%f~yrY*~s*?qf@fT@ToxwQ-Ga;KhBE8q7OyxTTRrJRPv)=?%DfHh+i!| zSh~6byQ>8&xCJ=wVAteNsY)*Q#xeT|$jtYk)z`hxBxRzFH53syQ$y8#tcLp;Q)Mz~ z0%!A0&-%JvhX0-zWGrOXRBJX6wmC)H6p=T5sv;@V(T!}FUp6or8qdV*QDrhF)1I0v zll|m{aZ^2JZcKhWf$m^#1^tj<`}PNWrM>yOZn@}jbEj+mRP_4U$N9wusZO>B+t&Lb zzg1r*TNs@kXp=slZ(oho`B6P>P_x0B9hR6uJgv;bX~PoJS7|*ze_~KtJb+(-_I^(_ zYBI_0+n?TYl3>we(oS<J1DPo5cX;~HtkqzGnwr}smtmfpnPjB6%_2=9QNx&C>;2n~ z&Vcps{`lWdHr-$H^5G8~g};e^lZp|OH%iz?{Q;xN<51w(vY&_^Y{Mxj`?g|&WA(|B zuX$3Rv5MLVy?!1XRlx8b=jl}_8|zV9_4zI{W;Ep!g7+}T!vn%03}S*J>vwp;h}>53 zotP@H8|%%3h`W55YMwQ-p!lePoy@4C)>DwdBi=>ZpC<K&&8p&iVCoey&EXEQo5|RL zgDRs{0X?l4S*R>uLryoPv~cOzZu#TQs4qT>zb9unkM`ovD^wY=0==ldXBGN5NE3i4 z!ouVRrcKMxadjHYM6EAWQ61-J7<Xr%wb$aZ^mW~S`C%Q{;sMEgKB=<e<iH#7EsBk~ zJFgPjQ?8|<0cwFdA1?5-u~GTkabcxA#$7^`z_p&Q+s-1tjUC^|Lee6?`eRL??Hb&f znQ?>mZS8~K$=;36_-7n14-fBK^5(5Y_BGV$p~lN1dK;gLg8a)g5m;rWW?Wh8RCOLF z_Z*W=a&Yj)(5CME(OR|4z{Qf9trD9`mD%8~8hd)9Ijo|d{QTZ#US%_0tRIQsHtqWD zIkxuYLQPM3kp^Rk_QB!S?D;_Mj+j#xLfPisAK3YzHOJ@%i!z1i=1?^XV2&6`pE7pQ z(rzw#!NPGFXnWsBV6Vzh5XMwAxSpb_^P}W^ulclL<|Oct?_1vMb@n(f((Clg`BlUD z)r7^H+`0`K$#{&kAc?Y`_Qp4ni?Cl8zrOcx9?XrL@7m7g{ml7n%50_Cn3zUWBLo-n zMaASOf8a^!-T@&=XgIabOcN|))8(xD^)7MyVe?(8M31!A_mCBbnI0zV>G|80E`F>O zV~_WT{LMZbNm4PVntplIm`}5bSC8@o(-%xe*g|tTR1<1bohdE0NnD*z(hH84(o61i z1+R`pDP?zdsn)53k*uX6)2f`XvCg~O;9#FkIdmJF4vvbozh0e_2WJ<s>Mr(NH2Qcq znNodNr^ZZ3t5S|O#%(LemnP0)i=Hmeqb)vh8XWPR6w6g_ZU`<vGOelZ&Kn(MZ#AQ8 zF}nPgjNK(piwnCs?=4-t9q_Qxbn4ZRh9}!x)U}Z8%%c>zCX}{NR-s@@wrEpJt6ynt zuAaZ2;%M%6YOOPwe|3K~SN&GU%E`58Qn!-k-A0O}9+Jq6A@c#oMXK};ecou&*SqDU zw>8c&<M!Jc(r<(hAKN;%KYsSnX~loAK>LS{#v3OwC%2aMA46WA_#*K+ww`}Nj9%_L zPetZ)prQGKT9wsS)1d0h=G^AVIxh*`3lRlbr}<{jgE8lNeod7|&uOX1+0R)}A@e69 z7i}MR1>9qvDOP<`jm~y!C?XKyYCPya{K=Ej=byQbKFRb*HI~A|sV}k4Avc)s-?0EJ zNpjXcMw=BrbBWY`K~Ijp;C<)ktvk-FBAEBgDe9y8Za$t#zW}n{7H`j93%WV4l5++S zi3Hf}&CS(Xsk8?a4xx*;W76YKl;$tq{fQv2@nlvfCbA(kult!{udnj9TY4_lM(`yf z?GAako2jj?&GBQecL!AiFJT0^HdJQ6e}%s}hydh`%fHGw0srx4<o%X|V3Y($F9wfx zm8Auf+hp8wvV=|Va&iyv-Qs@p<)N-))2Q2I@#6V}n?agkftZeaXL;cQY(k-ukMn1o zgFqtTds6d-`l|iIMYqwXPNy&|?&9iB0>-9|p_<PHec0q;=#bC84gvy+^RZ#pSI4%& zqoPFIY4k*m38^C&C6+_eKQa~Ljd3a09DwieNLP3IItvg@oZ2V!n=H?1RXpR1lGtB; z?pc?so?2FPm)xInUs-ly3Em0{wsn)9#`vV*T{Xzgy$ft!JTkamlGSY}Wm*cMeoc%Y z7uYbA9-+u$iK;|nsla0lMG0VkwCp^y=;=J>;(ZCm=9$m8<<=cX{Ic&>_hzZ;PM6R9 zFveryDudj^vT-%*MPH(A?T@3Z8m&<%<5#&j>345bf;>xP_gRu<JeRH<tnkCRN+S@1 zJc-}Gr7QW`Y@*+@%VV|n`A^;6W|75=yZ)iPckMQ9v*%|C%A~%+v>Ki2<5A3hb(^`& z4x75;BK~>5$flpW4vSZXg;z*P9T&f11+DJ;_6brOEm%ezm99ml2%-z<j%k}tUL|C& zH6coRF2gSttEGRqobHc$!i6QFm-b3nRhIug%w(vRL9#C@Z&}&s@+x$~P8gTBh#nj7 zl}0>^9kv|uST@)|va>VB8h4shNaD6fz<pbL#N@gD(V?XN3$!<6VBuoz?b}vpp;ZBv z%4Z}3)j(Orr$(i6n8G5_IY@=cr1xvlCb?_R-!UflZ{}N<lxnK)o_pBWlBJ9_9fUZ{ z0lvE2_@?uP{NC)r{VBV7?$8OQ50~?z%JtbtgXG>GKRIxBiAStvxH;3C9oq|{np$V5 z&ql<Y$}>@0Np)a@<hR)-6|rLNuVWBY8Nb(_k=y>Znn-%_f<1KNr^ykoz=!uLO5vT} zW&{NaY@XBgB`->g&E}_AS@`s~#Eq54`%Z?{9XH-d#@kE3D!&@7P@A5QARV^epBqVs zN;B}7Dms1`1bvFkOH6T}h)Cd~tG%_kT*CxN<f<QGHhp@oqOjsF>}qBmlO8ye_0A01 zh0f8w;joC0M=dqaxgE&V<H;&&$<4OS^h5^~6xcUYN!D7vFI1_&Nqwrcy=Tb0FJ*k% z$2u&wn@n8T|8;R@#%;px*DjTKhgi$%jn84Dz2xo2`NDFM4Tnk~*QRNy#np5|$>QAT z_%x^HW5=`#3t<OCM<waEvxEr3Uju_AP!rnSTM0vV3O`AwWF{A^k)3*waOzZhzqG~( zr(sAI5$)*g6Jx)>IiR29n6$Clj-VamD6IQdw|88jR-~nz^&REl(WA$Ic$#m^LQw*` z%j-TI6d)D6eDWx?z;iQH?HoHiIn`FZtYx*F)P0>vV(j<Hu$eQrl_}fd))sLvZ3S7L z@$K<d|Bwzi0Z2`n>D{lsKiOB)`Qbbglj7`h)@dYJP>2C;<S<f4EfD#fGb^;)xOqAA zsYnneee(G^mOxFFwH=!x@ljf1!FB_C==@Oo1Xj22dL#4M?$kAW()3E=ip9_+I+hof zth~9x)S-L+GL0C1Odj9jhXI5!w&1N_O}^d62B|c(p;kO?(Xq3LNKLc?xD-7G8Rd3O zt6bdCT8rsqQg)<^(Ms)9BiWZbm2QdIZB$3DnbJ&4F@ai^oiUDB&hH2Wm>pY(okU6B zWyr8`7|2*lVbbl1^2!*G5qkeg?GIz2kt`64qcT6o*AGhyNRrFNT|Khm$<km1=V=uh zMjH4cYu?1<X%aFWDzYa5p2>p-TLDqy#wv>JaX~+OU*}^OW?K}b)=wD@^cXqt-!ZC6 zGjS%TAhrFH)re9K<#;PEFYYcbA0{i;88f$b@NIJ4Ng8D_!Z_>^1*_6_@RTb@+=!KG z!Z0{)G94Z@HgeMDR91<4O3fXW2qiTYZ94uoZem>SNQuHNY-HN!?(x~NjTjZ}Ysqd% zCZ5E}G@rEWJD7O1oO^AsO=88hI*WL82b)-w%H^+fnXsJK)3<(x8uNc@N#RPtenXkY zGI`Eqd=24*J25p<UM^gYMm7eeLGJ5p2Aw@eVWccgH%*=vmgTSAQ8wzB5zMhlvx0&i z(T-XPQ>UTY0=nda0Z&B7_C0bk`CL}vuk=e!Qeo>R2r{Op%OkzYG==$6Evt?^MbRB5 zx(4XN+g1~Rd+dh>n?^bobKHn*%oE@!%zbSfXlRtoClE)LCXqW<H=i4KIp;j9@As4N zHHM+5y#+(MQHLj4Ucdghvav~|seO9yj_5!^KH+>&I4!Qyi{|ca2j1Tr^1Mlik}p+C zDF>a8C!Io`eMHLp_)bzUJR-towpuoReZ4=ygD?Gw77?zx-O87iPi~Hy)Ia1La*|tk zMpa-@a@B-}wUaXPuiXBOC3Zl|CcW8rlPc6`O&0O_aat*Q%MDTept7_L9ghzx$oOZb z3r)fXjTGf?L=YYoje$2hT%ujyw)_W_buF40^LDOQiff_F<Ss#pI2|;$GD&XRP7dQf z>=^RaJW#9kDMyGS5g9mYL-2@t;{vp@nPOm(fyu|Wm-QR!{aW-qqvRg3Dh=nA+i9yA z{i&C)v3+*sKI6*h>7!BzjU*-lF(5S#>s&p0A`gfeAT50ES-nIQ9f~^7b5r_AK5RQQ zIF{rz+9;#gt9Bn(K4-R^DBYm8Mas(nwS@EY^Fe)QN1kxRi5E_Lx>MWpqpfjr-KT7Q z?4NmN&?E7X2^?hLKU}=rz*vQn^NsfFx;!oA-#;_p>dI-|Z7s&|&7|l66l0*A5NO{V z*M`8M71s!o?{i^z^lxTURvh*7HWkxs)&jJn$PZn5m}_Q>=|bjh)bKEH;eBiM4<QO{ zV5VlOEC~$-zA`VCJ@kSMq<|{U3H|HN_1a8oO@?s3@ShR!g+{(my?|nD@S;6*@pYr= zw>LI6ZpwoAehQ%QfB5d9Dg64<0{_p)haV8}|MeMy$42!75)y2l9H77U?@Q$F@@(=g zDnf*oMV`3$X0oubjEs&__8+fwhO!8vKm2EG7&t`Nu6G6pZ(UvDvSY`jSP(qQhbwjU z^hoIHk{dGM1_SYW=u4edUk}PW54;MKp!x)EJn-lZDqKham1MO4A;aH`5lKcv4KE|$ zA!tM|!6gDfapu|mREN2^hY|a}TXC*eQK_1?3i%|h3`rzD$AXAG$3rXDtO{Y-fL))E z{vA<ELw;$g3m1u8-ji@v$dUwjTsoq1hJ?7d_5VCpfd!~E`_IgH^E}-7q5WSoqYuqB z|Ig&c51Md;(wAVQ1b_w<FTA(6_x#+sOI=ARW^8-Fh3oGXPYgkt95}E9`771cs$-R& zfe4V#sE-(YGarX`*45Po(LpgN|2;Q(d8T@L&yr*UL@2=Y>B&Js;Y%4X8(1Jy6dL4- z0x8Y%tyX~h3~%S&9?S$G(QZ}KV+lrl>}+gD361MA?J=oXM=NE(@8@GE!CS!XX&x0A zXV?}jS$f4{Ys>XKTIxDWV@>@dX#ycr3Ocdj$xjd8w)G=dadC0r<S!_=jL3n!QeD|p zx3h?y%lp3cM4f|kJO2|;|3`!85)%?HKJyqgFM@ay)~>|iP2QuNURFumM{_O#Pg<aH zcHzwfi7M|_w1!=t<sk`YOGIlYe+|VKA9!K@AwA0OPQexNQC{|o&yG^Xti64)MX%85 zXt;c1?4*<ZtsRJh#Irb32AA!o_Rl@L-$AI7@Q4YMMx}*X@}CE>@k-h^ttQGN3Htvc zpq6YE;~-%c{?tQAmBPCiHGm*H28#5zFaK$nODtEHsL)$NBoU;LQ0!5xo1bEkVT#!& zis5RarPVmNJ(HUF#wtj_7)Do6hZl`lt^M(_D6XVpr?EbgqkwfnCF}H%odg5lC;z_e z?>i<E;$M-iINq}nflB|;%6}5R=lNPtyAZMjpzCwViF{#*$!lPH$!&t?i4;FBxlBWJ znC1BCXVjtDx9_jS%cY#jFSDrQUDF8)q-;2;p*dszk3yHjqp3z2*6gF+{Rxa(?Fx!} zf%PWH%bo9qBNNxbr*tY+8V_U3;*F&4Px-8!`lt)GRJdB4NZpXK27tC?%B7{$s{>Wu z3O}7Jlqmp#)n+}ZQjZV@>Crwt`z(niS*)zhRGVGW1c`VQ)AL^V6CI1l72c@&pDqbj zbB<%t9>M_^x|6WrwtH4oVfT>HN358s@-1b!qG3^)QR6BG8k?8K$+3gc=FvhPkGM=e zNyJO4Y&)90qX@z^p=qs1RTy+ASEufzFjqBCugFLh9sT`;($N0n?RyHnuxaXX3iFrr z9mi;I{z#@vejr^uh0l<PmK{xv4k>G~yIMiciOH6K+?~Wj@;XXdPosAy^Aq1vdXRy{ zgEs!#TFi@n<Jv=V247*cx2S7Ad`HJV4Xodw=UX_)=R(Jn1W0VEDCNk=h_L9{*_oW_ zcJ<JMy7V0z07}NxG;esbu%u){$~D>$!UYE|j28~ncB2OjH)`?lEj&78!w4$QlctK- zX;HPoi=T*+dLl}hQBW{$Ax7^Gq0<Lk*Nu&hPo6w!)x&wj07-m;s}2$Kb4a(;44iO+ zyrLar<K^X@ot@<;8wQyQT0rGK|MY~?A7rb~H+j{MZEH1o)-E0nyNE&{w4vt7VP#2U z+P+y&K!sbV;m>*pO<`=X<n?Q2)8z8Twky>cE+dd2Xo?K@y9WnDCKb=0Kd&V}CViax zrL3%MXh?31jh2Q6#Dx72hWqAUJ8u)(xnf`YdrUvtkV`);E$w}c&Hmxx+pd$!PiWw> z%dqPN#`of)fHnmf2ccFLjI72+FAEE)XetU(Q2YZ-&G+vuX}y$jy+-ct?zqAJRFa~A z`0zJ)o?pMd9R7qr<U-8YEGni+umT57`dt_l5YP9fV|tAowXp-+%NaI#Nw~!RD$(TJ zxP*i!kD1=WE!lx7Yj1C_H9|3DNRxAMbiBCr)_R1Oe|LBH@gv-jo6T2de?Z@Wh6w*N z5Dr4!Yg6+!OEGo5jYTSNESici;3Hbm7+L@hJl+$G6J%d*Ol<7K=?c^HyN@jRf5~)? zp-)L_!23QwKeyuGt&k-&>@_N{s$!3mTIb||_{Jcb)<WzY|B4V^jSP*0MEUc#@$3K4 z-FPA)cXz)E1H=M=_~xQYivj;lGd7SGE`Ev!U8?~6^&o=eVb)vy0`<bGDk}l$X4+ND z(gi97GBWbj^)>de4ojD#_Tlz6Au(~kwAF(|y7>bO%fbR`Cx<;qB@z{W=>B{?OoEyl zgh0s20qRnM`Ur!lS)JN#5H`<f$N)pso|N|j|4XaFk+Pn;35*Y&)?e8Kk~}(EZaE3Y zmvpEO`SuL_1w3B6b>62<6hIKYyu9)lUUJz0mXoY3L%@SVM;yyCvAEdmbMLLS0Vnh# zxYMAt>B_>?)D$RA7Y2yJx4-YFOuY#72li%lb@i`^wSmy9-2)t`VSxVW>1iUM#LWD( z3Rqj581ls_I-0z$?ZQDFh3ve%yqcPt!b0@+1-K=NSYT=(-F<AbL$x^YF7t5D-o_hC zMw`&G+DGDy>;>l#8YbuJJLk30N=7sEC&`#>L?vM+^CGU1;fZ`N;4z(i4e_OnE2K$p zEJc8DAwI06c0fF^@7$HxY0&#HSRA0Uhk163w0N`U+<es3CR<v3GwBLdi|u*RYDw@7 z2bS%CWBnpkZgbl7Yg(_7J+WN;dUbU*5G+c~I>-Fs&6L^_2cER#-1XP00Ox=zy~-+1 zJZS|5U~NO_6#mY!(6b)~ooJHLH{L!E_~_F!$>_=5w97mk12Z!T)K=)pr6*Y=ZDd*N zhc8sU;0jtLNpJU@>v2R-mrpL?KMIi<g3;##vI4Lw`q#n8{!S=x%tFN8j8#fdMIX{Z zzS*sd6As^`F6!y&==s67n792E^Dt)=->eq1e}Sx?*G8p~rLF%9bp1DIk-K{(vnmMj zZO;hi6&eBjLV%=4^f$=9y3}|O#BWAnVPSwj2?VbuClmb-#I?0$_V#Tc4kZrie-IaV zu{AX=fJ6xo?Lqn-+HgdGDoQpfDI)3|f4H22l%sY6Rj@<E6Kx0=t}@V_m?|ny#iFPr zRmMHL|31Vk+alFMoHm29p7PWOc@+XzQedNu0)$BVVVAde|LKO$puPmV)@DkW-5>fd zu?)$K{cqR{Fr5?@6daQx0l<0szmcd=uwK~U<!ijsr2Qbw2N;{jjX{A3%z>&1s`e!J zCaByoBs(^6o7xMOjEoHvZ5QA%Hw%4#;g1ubKVT)Q5s%s4lT%`1t3dfq<tZ^9xPl!0 zgFM8{A-4ZCgY!)VWz!}m03SzC5DSezsD55+-UHa`vkEMj7AYhFPYEEEAT4l3)1rwn zx3-oAzx%LutsHR~hOtZl?E*OXVxk!6=>eCKB3^0_oX~cU5o<w#6fW{3xRyD)1q21Q z=w2f=wFD|jKtT(9o{P&6p*ygvIsl6|_hleB_{O5>xVX3&SQQC!<k41A@;i0;D4p{T zyy8F%q`_MU0|SF|<Uu3WxVShlZlPW<$3R&DI;QgSci@Hm1(2Kk5HUj|qwK1xIEMrK z+6U31Ja*;dNMVeVBGu7kkGc4UfW-SVPx}G{?t@%uVq)Ga<n`1*5eguR-(nq|oSa-- zx(!$&yH`&x{{if&{qT6<T57Rc)?$B`%hJiEj*H$moVLxA@!!AoZuFKbrmcYK#SdMl zyLO}SpWO%QRUBAWDX?LPrdnnofJDGQ0Nbc~)hJ^^0!GNI=S1OvN+e0(AP`3c&^N28 zZQ60kebBQbff@paE8Jj`ZCs`rVM|L(ZKlZ=5txvr>`^v=00tD`Aa6hR>-lCXDk^3a zSR5SFJlM4iP>CuOpAi(F*mk`7LqtEu1}Ku~U%ouXB_=_U9RqojAgp_BZEZPAfY2~X zA{)>{F_*zXQuzi42LXrfLnZHn2RTJTd-|}16=Nl+-s$K>5Gz4mr3`OcT3GP#O@kl| zS3c<kra@_CyE(wh@(BsugeEE#eC^%{p54qA1Cj%{cfqP_Ui^9}1lIpQ2IVw!#Egq1 zo&SphYKFO{gM$NE)~8i`w9hYJzBHkwcJM<1+?EzkfGnEmMI`?<dw{f!UrZ67NsEh( z%_!N#<{5=y5jcB&*u1fEk=081gYCr8(a{|8QxQbJR`4urt0(~N$kN0lB_}6qC#I&P zs!g7$GjP1-6N@*_8w(zJNsTd<1OQcR5ICBTkB^@V%UQN_-5$-^n%8OY$GJY=w=_2g zt44Iqj%$eTMCR1y;ez4j$;HLR`}_M~&qt;owqC9*5~OwM##EePJ~Nsl4W{K7%9yH5 zSA#%D1G+HD!a=YA<Cy*JOtGq^wKZe8npijsU{?WgLlpjBjBE4rA5GIk=&Yx$&Lv`Y z+O}m-0fUbHn{5XZiz54^qB(~BkW6Sm#;&_}*5??shuxnzDEhO#-HVCd=JWzWGGtH& z7{Thbm<q><P!WM>rqZdX?C$I^;D_d#M@dkD<pfC;1$M5cW^(roFbNqHpd|u#g&h|k zD=RB7Y_T6sQymieKerp3zj@eUcNmlb5)n%K`gyRLT9%=PWRZYgB}?{QuN_YsVD%F9 zw?>2H6ax!bUq8F=?9r$L2;~m#zWKiD>g(&H0XS@SHgXhB61B{z7gviqjsvJ8I-(6Y zx)NmHETP=2j<u`b4|_u280kqT3V(el*U0Dg-g2$??{{I>k+Cf+shk`f0qx65B+%3c z#EXY1rbs0Tbd&MFlrNj?bIQ=pOgbnPq5~j+0kQlcnI!4$xFUee2w*_Z4x~IhlZSf( zFwormH3(FaJb+?D@*oj0kP{KqXz1gRUWgu3zH$MWl6_~e%az3KW6;AA2_ofn23BH- zNd{lUEV<gKHyVf$9zy~j*jz#&5p*dTuBohqZNA99PdcE}Rz@ygIz@hV08G95MJ|sM zl%@m|%Bu!A)oRQ}D1RTMVSSkekUQ}`hqR#~8~f+Ir!9OqpQ0s+NI|im#S92U=tCbS z6=0sL6(OPU?_SL>E*3lAVf*;tUuYrW%FBN->J%FS86fWJVWUeDeFp_&P@=;tC@x-4 zMW;|!DbCOT0E{dan%v{R1Kenl?K3$hMHK4Q4`T(a@es%o7faX9?k>VdUn<F`PoF+W z!{gR`KQeVlX9t@sWwbH(;zvlVfOg&#DxrAODuj+ZRY?r#MWO-$1j_^pybN1QSlxi8 z%7~S)a6F{8SdJe00`0H9mY0vj$;{2|TDDu`C3%>E&rtwf{s8bZJ)Nv(mNZ;k<td?! zD=C1c24%oE!ZrSHxCJzD5FZ&fKn0*FumY-7(B^Gip5XvZa*2fkL3{ic0KE#E!x4#C zp!nNO#8I0-1RC)1ufYXR76Sa?z=lNoms>piPDc$`pyG|e7)?qcc2o~S6Rtp4>A3w) zBmVX;ntAAz$oW?rcLHHn>=OPdJojv%9@!SvkgBjl%1!`r;K6xdudCkm>jr-*yPd%w zLi=b_Sy2-0AQQf&m1{GMC3Q>g$)|U#c;vaisUBez`g4jz_db3mpQ!x4=}nR;mkR`J z(K8XiorEN#vvC=q+}^?ZL^IOj0YUm3s;-8JiWbo;?aEEQR@Cg&@c&Ez=#13<+uXo6 z%t|r>-Y+KNF-y`Zo-D?fPT_ak*<OFV9tI8?9aKCZx9gE1^Aey;>Sa42?eG)<rsQw? zre`NdV$HRT_=xok*YcYxPn8jb+SbCNpuYY+s-l{jX+@mo8-a(eT-LHOGLDXpfWFR* ztw1jx1(JyFj0r3w;Q5*?(T=8a<4Xr?_`zg?_DT?}WbktM$s!jIVXjgI`)rT_g1@Ud z^<?heLGYmyBo=v$jnFXHJUM5)90juR_^%VHpddMyrAt}m0ck(9lP-&@zH+N7O%`t) zNCct>3UKG3eU4|)pp=Xr-UJ({e|qJD{8EE5*G8Sd_Nf0tO+~7fva*<lVfs?q58db} zm=X%h$`s?*JyGBxpV5H{HK3H-dJiBt@Ry+iWy-~)h4J?Zz!yToK&d_a^3c$LZ}y*0 z53Z|U2teCqzsm;_D;Zr-SjdPk-P8{30gx2(BiDwkEI{2TO8J<r9p0DwFMLBrL2)^l zdm$M;Z&O-S^dFB^79fiQUOFSAj&*x5f&T!G2Sts%naa||y$SX)e?5l2ZeSZgC7El! zA_`1D;y*?euM293@n4#4HI+VuQu2El#JBeEErA~sWF1jSq9OY|plQ(b7T7pAW~?{> z5^>~t@ZaIl{ap~920(vMQ+BBXF!J{qIOXyx^7CJC&OCbn10R97d`nyk!g~0P`jH<B z+|t$$kS+dC`~SU~|N7zEw+s~EB?XWTaJWQMiBUdQQ8@wrO3+$vBxqu;VTUkx;H<+L zeDJ^hU3kE`$@sonB08H+N=!`bKQroazPG)z1ESq5CyI{#z0&Px!B&TD@P7yswInMB za3uVF{}~Sk$t;;hG8#t&yzhTwYX*GuO)W*dFml9yd4+++r<RTfN6i0iv4gi#&H${_ z|CXgUZx9R_fZls(1GF`>^K)}y4|)adw<*|Lll*>|y>C>@5AO&IcsBmk=*+;w{qwoy zMF*7hoxRIT`acT$YN-#3jV9GWh7sf8=bUafSqjmnEMFyI&H-hflwP9Ov+r??%L8l@ z^Yr#b6le^g8@(C?hN+=**tEu%d7Nw?Q0R!Z9qXE#b)&P7kL>{y;-TH&8*RMhBRf1i zbg7@uP>=vmALyWI5JttJozfx8V_qvt(eXOvyz&ggA0b_ugTZO4IQ3-cww>A95>`iI zT5NS(w?}I^FPeH%oo>v%lJ)OcfQ$r}Jxmn7-FSA_)apy^J5f)I{t}y-vFge)sj;;f z7)N@#!oyZ2ATZEnZ|Whk8F*O!Gx0n;JRq26er6_FhUSE{-#oZ1s_a6EdH?D<l=trL zCc<ihPs48yN|@`qmy_n=eNcma<Fu;GBOOMqlq4uvwH^<O92CO%PAx2l9g)WPibA8P zqNo~GF*>J2DQ{<+8*fTV?;bi-W)&w4TaJ9~y^JZX^js=zzSx@dIch81r=obuJN$Qm zj@pw>{IECwk@h5+F1A5O9&4z0lOZ=F%Q-B;W&R@SYjifwSI?D&;nZgDM`_u@&Q$(5 zRF8wVC+NZwb9>lzLTB|yHVW_L48WmP^G(0BUs*5qoSnCq3Y{7TwdBC|4^@;RtU{jQ zC%UXFS{R))GtRP=8nhgTGad`cpP_HLn*W+Lu#HA5c#5>B1)(#jCC&_!KxLJmtoE)w zaTt1Zs{?@qfix2|3KKb(<yZqNPs*~2#z{5vQf3$HgJ?2O`;$xJ_|%TG$58`Vc;b%& zW$UkG3k%!e9^T1|my!;~UeLzYWmhXl%A?tCPN&oG=CkOkGZkaePyTkiR+HL4t9fuy z-L?CZGe`MM5)t!5JX@0n7cZ=KN^}L=xjB*4pvaAbv|7l(;|>B*0Wy({1_j(6wk&Pz z*D`y5)_O*KJkVZ=Pg^-}mZbHH@<B1!*_pl^E9(tUo%&D(g(|^pUWw=Sx*Sf&44M!T z<8zTXuK{oYHu_L-zB2p(0piCZJFec+=veP-*F38PiL2dr1TK|_@-lMVq!W`75#=(t zDNSco(o{h*as@IXU$_hPCkDp2Xf<eJ(3M_=iqHre=rRh+L}#jGm{Gm*N@n#KeJM$) zLCF+WRHCkoTLCgaZGw7njh&eMv-3MT9mk=DRFe5&Hi-=bhGpMh=A%6l4(Nrds09=} zhl`#%9mUci7$yp1$?fa76oySxad^Rc*F--VX>1{Hid26$N_NznGBlK3>vK@!|1%lR zR4=QDw~|T~j3;S?!Q^i>-m~`8$7$~GtK~?sZvBKL$~N3t0#VQ`*Wl;W&If6Bo?9Pv zCNaU0u|zbLib|wcSuGasiG5d`R8Er%tmdodK^*U*!)~D9>4!owfiB}AHH>J{fL5ja z{ML+CQA(SdX|N9~R!zy2$TP{DOk<o~bndS&=xeN5biU?YJUUP+lY;oRZ5+7*8f!)G zI0Eqrv)!G=IUd(}Zdrm--K)0hMn?WDVi)HjGO-KTX3Ycj#aX|_qUSOpHh<&Uq>l%6 zl2qVrdW~+=!HUyhIYSIiYG|I$!rg8+M_Oa~QNfAo&D~MBkB9fBDC<+q>qWOS4PsTc zzGJO*i7?~W(|Mfx-g}+eItlw-{gaa_$PSSXLoitQ+-)JPv&(_J@u@_Q$8449*Qy3Z zm8TRPqI|U<OgH`HZMNqG+qhv|q&2!O2h^sF)Z(j8dQtAKj=R#*o&?95AA?L~vzFZf zqe14m55;3PZYy!xk<GtxAGz5!SZRy(q`XfsPu|x?fZg2XPYm4^=`)m+He80BS*Gl& zGm+Dlr8i9tlY2%;I=1JSUd>04-<{`g)>PXwO)%u&w!CNl{t_00(?ORuBv7Akd~Ija z-uJm92x_^%RS(M`sIEE7e(kPOdmm5FnOa-aYBMf3ea=YzV6vWMmq3>2pjo1PM}>=X z_n(<PW6Vxd?k@!=bz#n-Vkx!qNm4oFH)Dqa_i^{hM6)Ng!#?}f?-y+=G$L6DD}C<o zL=8qk^-K|C+C)KIQKfuk3%;WOo-0XQC&#^Ur};yWMxC*$!~2|w{Oirho964&<{Mf< z-e3~Z-kpF7!Sf7k9sAxmfkR2=4^KZCltty2Sr;2F%;{#EzqILqhdlF$BM+0MQ6hfN z;{lVKm?zOtRx8stKYKFF2PoqvxeE#CLPS4d#JC&~U)+1&>^0G&=+0Ebe7uhlA3rbu zqI=!TTqAHi2GgzcPNI^g;he2QIk=k!;r7p)Z)bBC1sgs~;ApIWIrj_HB)c1aU*bAu zk-RGxWs~Sphun0RWy;J$Z@*qk9|UFTGdT-PpS~)&zp&2aluLc`f%0mbnN#4ZnY>|b zFK+%i(B@`9UdOvEPJ;4X5Y=%nOt20Fdz}qgGHdZZmY=_zZyuR4@`_6f^1jI{j_tw0 z^jevaF2=7awp>bC^w|g~B+&9&=+hchdzp4QZ&=!Jg5I2!FJq(a<CeSId^qoVKe^z3 zOKobAEEN_mOXG8Ufv-ZT5=HJ+wq8c-j58hkWqo<G4!8GIu<^Q2V#?Gg{F>bDZfIE0 zb@6_#p{O?A^emj0$M&}7Ys~>P)0cf|^zInO51WJCyj}D-niM=&EmV0!IzbE$h5>Ng z#-WlKV<`NAZsP@5j+h#jqL1xdH}>erzD`wpg?10+t>|kAOvG{-#t!bl?vFR`Zx_Gb z9=MRX_P<~7UdekOCMlz|pzto;;p%tZ{n0wX-JjdT`?~`AQr?BL(KVx+JKeiIdfm%+ zKQmvZYo@Oo$>h?~(g>X8yLcbR=r-S;V_!5Z#JCIeLW{L(H~Muig(;)>?qvxmJOB*5 zWM?H>9Ag(^tEzlK^a93c^PMW%*?Er~N?00#XfKHmLO$(h$7W%ewmbI+&GW7sqjmFX z2!gj`p{12rp7$TSi3{7ptJXn&w|KJQ>Vel&;o-<8=l)|Pm3{tuK@S)ynVJ(Ln&-Y( z-)U}?xYBQriaGj0yfGPUY6>`b^8~i&IsXPKSmPIG2PGSK33v)2x(uXOjfn<f;T|!J zg|H|S@vJBlHm@P>>)Q`TZeNTHgZ!u}ymgyA)vsfn&I!YZil#lR?`eI`i^j4&S8IkD zN*J9Ao~JK_L+oM`kCDP#o{0QL!ojXQyPZrM28rCE!;H=A<?K1>bJi$49s%O`lkxCy zQJekR@pDdRUz?0n%!SiAU%5{TF1nlR-Y<TgFaKHHVKT9JAC3-Vb7`(Dj)Po-!maqV zd(jNDGVaJiFV;yjpV{pgqAE=d`Dr)bTo-=Qsnx%2vmDYj8MXAej^qT%csCuB<gOb` z^EsPL2wkWYG-*?t*z?|#h3O<??ag;26h`g49y-%wjmEm0S4_4|^CB;-i(Y!}lIv<U z{y;M_W+U-%9$TnRk4h=g`O_;s!L=VRMCUmf^Ip*TfO0^`>s4Y4IPUV7qxQ#H;*!f6 z<GWn5<Xk-EPI&E8UYu96`DYnlvE9-U-VJs&6@!$4hfI=2r_)L^vHMYmCJTZ930LiR z_@<x`8+@oRZtmsp%RrvC!nwYyQYC=|d6ipY%KxTU`RrLNS#e4fDMEMah3lNzFSsD0 zaB=NE5&p+;&kiHtBD6`g<EJw=a!u1`mtN9ox*XJgy|}<cO8+F(QnNgr=M9_n&|*sH zv`U)$kU7Pd^j$ZKG}3)}2b)&DPbiHZ;x9ArdaSTB>`*j8EO@4&5yhVQWnL{NWzdx( z<~A!`+qx}J*B?jZ9xeCBM0!T<5G2uR9SLjcC#8$GVy*s3Q&YzB2)r5-jyMEWNF1*C zyW3U0nzm~fWw_j!f=VCi{$2kG2wCvXaQTrr#eHN%!Sd3=`+S^!SVxi8tdrX5X*oQn z_sL`3Fze~r^RiG?Qu^pU+9xlb-yST72L`se;ji9bw;9tD^#!+#tKsLpCX5kADPO64 z?|oiXDE851icu6Vo&{fCG6g96;e`4ow=szfdf3j1+78<aJui=FgtpyPCUq4xL<Z-= zjDdT-=*VaXJF=n4R`0Y(bJ)k*-AlU`h0ck0&vfqU?as%qb0?#hmPG73VS?~kM*~%P zYI8^AYRU;epLUUOS0j*OU$qCrEt-i(vlHo?YZLlvI>b`HOIlVytn@T=I!f+jN3MWl z9W4()%hI{_rXPL&l;#IcvOJy`h5wSMi2tvn53w8S)*}raW5aw^R@1m&XvG!xr^*<) zaP8ODF#`>zXP>}FQ^C$)j*gYs*eHO8Tv%99Oe}}LyM(T&u)e0MDkrZ##ezOL1v^Pb zRZI~LJ+xm+np*M^1~v=^L-uQLZ`DteI~;IPQ5#=dUpLOv#tk;|c>CT(&6G{x;0SMR zlUqzU@mMZi($o_UkR%-TNW3B;tkAHwaS>KA4(~OR$je!1x|^t;_%oF_q2>N_{fBrK z3jDh9x{aE!czj!iovtoy>DSWMR#30rkV_pt8v`2=rcyL75MAj;FKtr(H=s6v;8|}9 z5y9_%)~WsKfPfjx(o4~gc|q@3%LqP-Qch1Uj?K(OOLSl$E164bxVhzsXDMQ{v#~k; z%+bi3EY)hNtEqwgTJ~Em8@Z}O3Txd`?=fRz2TZ@He1=cb7Dq<#q36^4ZtUYu+m7zt zLoT9<r$5o9rKD&!T>&+D`oh88UDL?O<Fp|K&3CG*IyS7y;}f93wzRagu8%xVnlo4G z)AcwqdAOmU!l-Gie}rA|aiS%1ooh}>Y{)OyazY2h8)GHL)$lm*W;r;pL2-SH?+l}) z`4idMaY#Ab?8ao_stD0Vug!=HLtgT6^Y5p9jAGG<raef_#9L1;l$!e_4w2jt^WHz! z{IT;a?%)W^kLs(0a%<TZf_%gF%Ofps&B4B>PG2NGQdzQn5`X?(cU~a^zwU%}JIvO# zXqc0gDB=t&V(RSXCmm*F#Ou$&e8Y+bueu<v5om2?cUsX$>NiKG&g-uBLprM3YHIMk zY;;m(yoc9cb*?XUt)S+l3k`W^4X^PF-bY?Xs}*UhYwzU7z7y#~Bjt#pS&IZKDE@Y; z6$)Y5V(UE6`V!ZzGtU^(wjD}e7^01z<-J18czUM~kDlT7JV^!%6H`$N9N%9(LZk3E zq57j`Zk{BU*SvW^?sKxc$dMH0-?FvU)A~mC=~H9PVw=tM>fZ22AB|XRJ%)z@$=&qY zm5AY1Aa}<*uS=WHXXs0RcG(QiU(C?^G#=^{3rV3ne0AN~z9#eb{4IDjyLeYrnSJ^z zBe(nRGBS<Vh16o^5d@NyIVwh8zfzuJbEBn`uC3^cgBsY*+<bf!BhP<%zA*@alT*76 zwq%Ov%+xG~ygD+rWWb-l8rmSx@wDl$)_!wq4Kak2bakTJ{JaY&EWHzDO>(`ABRKH+ zBb<&P-?Sf0`}Jbi3p?Ou6`TK*Ah5m4dNy9xwy|kF&06+)0fGZz6U(X6y=mKI((uU{ zG)dC2@Y{4vQw8Uf2ne@gV$?KQ!iqxi8I{?N_Cil6KwaFMuH10u=9`s%W`4*Ql$Ni~ zT|1GwF1ItUhs&m|!i3dax-mJmD(%m%$nXEqVhcFl-@^DGeYqMh^?6K&e!J9sZ+C=5 z-vmqJl?n+RN1XWc<Njmdf!ESu<n3Xp_l&|KI6HA|$J}(2QuVo|*JXO~0uB=4mqBsw zFf4d`+?7ycvw-;vMbE-a%dLO#YwcCvE?4>~44V(uHg@lM+}j?Q47c*^XKB+)s%2@D z_jtkBpLQ~lExcC(0s>rIW*jLXLo;_nkk20jE3&boc{cTVkJLu3tOqKbBF-qQy6aSV z^Da=XTaIr9U3$=zXb28e62);C#2ImHUO^?ZUu%9a_yGOPqM=9w5>%fFsj5!bkQ0$a z(kkT-4CKr$l8*ZdB7ReW=D%<z8FVwI_A=5qAcA^Zs`XdrMvxA^@iLSw40x>fEvHOW zQy(pbWQDCZ=8D!0P0AhF58}UCOPNPXB{m=ZPE%vcSCR1eC-66WU~8!2X=KDgT1Hsb z$07$3JHOX&op$-90XAvt#k>B^A)US2wSj;*^q2UPc)LzS1NvSNV7AQ`poE0xHa4JS zc-73!%?)e;j^b1?Zb*Wg+qq)8Vcq%z&@D_{K4}bZbbS^#tU-NRdoS^5ikS2{_4qgh z+3P#^VO{4#%er$=d244UM`mM$maG|e=(Fj1*Cju1HC)F0z3|)}rK=4Pr%~qh$H?Us zTj1v9?KU;v?xi{idf#z2nZAN*YNl`kg{uwYS60>haJf#N6>CS*aI+KIe0h6la5S~U zUZ_^2P8ivpj9r6)g&Fd;37r~|HEH0S^pNl9;<=iZo6|aNiJ{E&)zEC3HowcwX#zR{ z(+kEI{bkd;mIb`^Jn!-yB|+*hwVashfC)NWx#jSX;Qcl>ftF*h+vbfrUdOGhhs*wg z$2|g!fFL2=dq!h=z71+evT5(8<>u`3>|$ya8~YMDfk{L0y%KGRa(tvoDJ$Xj=Y;ge z!#j`J_p?o>Ipja*)8P(VPM90)7H?rwMb%^SyrG+18u4&j7Z+2D<QKCkw$_b%$5k{j zZX<1-etx%1c?U}w?Fa~sIn<py`3P`6KtC%c)&Kmij3=yW1bHSU9!~}NhS#>dl@%%I zxTy_;*WYF4ugPp+WhHYnot5*`8GdBM#&*7@6nS^Y>{FOe*RXF{v%bEuKT~m1o3KDa za(r9T(6;XA%@s;Z>Hn))9%M1PxYP`qye0|<$K2KJm#EuESk2`g-(Cg!IH_%}&ej<P z!y%<W#6PmZE@rQ9E+Y&XY(;!o#Nr7dB=@0s$Y0`(z#cOPu%oPTBa3DwnRgmVxubj% ztcP8iR#);cT<s3MS612~LJhmSi{cd0_y7JlZOfcBI&<SI7t<~e>aSo?Vx@>{wR5h# z3=-=irYpf3Ezg;@3EBEtGLKw0%ap@`x{ddx0=s=D@n7{^>4+b&1LwPZ<(F~rzK%S% z>BP_d1^Psme}W*THSYP(=ry2rKvBhzijmv>M=xYjT}2)CrG1n{^}9$MRK%NM{^sRz zHI8}b+2XHPYlB#C1#&8jif-cH3&1**<hAXOj0sYG6orMCbq_DcK6(W<A9t1(y|2+x z#A}q3e^I1fy^`*cURw11+!aksagTf}%b)LjE!5+DUto2hg47K}EwwH#y5&*jxCy&- z!+f-~Re*?NlL}n4lY1``K#?HGEzrcl_`;F~=jFDX{)CdAeg`{}8k4haPR|X8d{Ot= zYDRRT9j3aicaeF2Pz0N@igWHk*8Kin)h<BlF_L)OVw<|7KlSvD-QxP!H@NN;PFi*E z4&~4HY!`g=Y*#;Z;)ilyqsv8Pp%dZjfNDDTC(X^X)qpk2Z}90lGm~z${q~A+e?sZ# zZk0uC;l_p^w%~DOR9Vmc^&Kgv_YS@y-t(v*^Xb>C<hN#Eu@ey!Y))ArLRxS_yJvW3 znqm94%-VIIR^2@0BINSkm;YG4OsKA`s$tS-&?=pzj>#W7h&T{9oHhkP0R>8U3gl-m za&~-rSZA9IcMT&2nf|P85Zhi%#Z7`R8&|%^&~xq+ZJCd60+YS(QtEdL=XEbfCx^Ja ziskb7cZ(d1HN2hMUlZ$ktdDUn1U+K#|K$lvSyD^ByZ5O!yzF^Kl{U0zz5Q9VZcgcE zL7B`FH{_YnJ3{mJ_bAA3yB6oB-LJ}LdJ*F^caC}lB^i>i#8z6qwV02Yi+066M+wLp zXL@8rWqz3c3>w^-{iM=>=2FKJ=XLy7OE`%mf6_e?&&j3S;Cq_LhKAMjK$l`2m5Z@b zovMN|t*pn33-ig*y2C@XWqbS)UM2SOWWEf&QeT<b+;T#S95FHck`Tm?_b0H`%R^ZD zKA;zbSoHHHlxW-UBWVjZ>`q$`F8sMAo@6NK8?NyE<o9ENTt<(A%@$N}eu1QA2&FjH ziGGoU`%Z20ho|`}FK;yfxm?F2?-!>VJhwB;%h8&BP?xs4{qF7-Hg(*qzI9YMZcBxB z5=C~u$N^!|{i+N*2gFv^ybT8p4)*qYe9RDk1`}`!R1%M^hG$pHhlE6?_lhj^t$15G z0|?~v>-f|v#KpH^KLY|_*3Xc~jsSSNF?{2NVE7Ep6JX$R6KU#MTEiu};!?}+rM3`c zaO4^#v*lQaRJATUSd6fs)74_YF=gfa8Ls8$xJm!83q%G_p|qqVAJnEcf|5tlL}%fL z=KTEW+MS-;Dr`$hSfe?pB=s(Fa06J}g{a-B^0Q)24r_1A-Gy7(K=O<3=jaxNK36fi zb@NORu3wcf1?k}e<1^Fr57i9)W$H3`>{aZT4I>K^3-}nw_EtNSBMeCcTJ}ddyA#^k zW5GKZ9hq%youTmp_jAPcu)VsiHfc;`Z7SU98e92a1ZG-4rg*G_--tXTA)9S9eftbg zr|J7$8~Q{-+|z#6piHw04ahU1@W#st%P*1%2?@-aKa~|@L1bjrR6L_b+4lbagGhpM zPdBi}cpQ<f_KXoTs3A#req{YtI}3iW?c9RZ(3rNPd%J0=X5m;61{4lgJ{LY~71q>n zzmK5Zs>jt04tT$Pk&#|DgNH(fKX+%&Vmmrsiql(IAzjvzynQ_?3QwOF^?U?GJPlJD z_JXpLa5PA4$ydklkVFaO$9iEO9CJ@`4Y3N%*9*ELyzkKuD@kp(iy2{%x5BCcZyu*e z4xh}?&uWSVg7WV^>z`P|b<hdT8E}>G;^QozlTgQgk)Y%!3WleK;7wIBdqQ8QoLUYS zdDKtWH|F>6V!CTNf1`TEcAV#9Q<cK?Ho{HP`Us_;tv7@fHzWw&wkjD7o%HkPOG3%3 z4r`yo(YIdD4KLlu*_h8C2Y^+&5k5CJcXj115;W)>_Mril!ckPbFE&o~_awWO>s9c= z0^p|{nV<JM>n`aXbm|F7y=}n@eT%U0aVB79Fhy6;)vQ_b4QHGTD~*!KN-*Lp1$uDU zV94MRB`oZ_u<)^+5rvm$=A4tAA2!2L7~zU$jRu0UM4qt`W%EHqi3k`FAzeandSR`x zj}Q@|e%00SLlM<}fyvJM^A*-=(i#2VCqsY%Ng6P#f;W7Ee$4V&R-?AGdD!FoWlP2z zS(FrPKzLZ<S#QR^JC?+jvi%{3<rQH#fsS48D0Tm^uFD9(88=7RF7<qE=h_dDC++^| zbE&#Dw?MOe-g+XXVB_+mG`Vi5f>b~t0uq)I1HL@9(p#)tNCv>h?u}gB2EMC$N73sL zFO=kDzjnBUvF#J=s+t-Ej1n!knbY!Oe<8}phvZvoMye_*8VU+*+)0Xz6Cix%d~bG> z>Br}G!$|(SFAV*V7GH4kpNK&Ow1Ko9@VLNrc6P?TOBpnI1vTM~kHqNWL4hxBY=uuL zDd;d~+2QhDv9vD^5cJR(Z;aeqnBK5FJSiT^M}!DxYrips2~?RIF&MliPMNtP^Zw8+ z#C4VvYH2CO|CV<Dk)x5-O_S+D-79rlcEaSD>t=_^6gO3UW3Q3pD^}pD=?tp+O$?oN z{<gYOC7l}?J!<BEIOllqv}OStX+(G7zKwm8d6<RR&v4mv!RqP?bGM)$y${`<gM4R~ zl*rA{)Iq<gvC(B~0n^r@MUV1%muvcrHI1L#pNKA<*$JcHmQ)%V%GxqL^{=xr95s&$ z$#t(cVC3`TJB_=+lM}@jOI*h*8@JXAQ7pDAu*Ivyk)r!&X9cK)7-4rihs=bhD`I)< z8|&&u8q?n25cUkxC3ine{Xm{T@qG|h%tjquMMXG>dMN?LX*$%9Z+LQo9o`>yFw1C> z9J4K>E!#M6ld$gMb^E;sO^7BbXweJiBQH4gVS_;l4M@wq>t^X!?;T^`yUfwoce}(j zfTf84ezb5$YRX^8k?*A>Zp-hGhUsA|i@|)(MfclbpUd0o;!=bTW*wiC`^zhWY8wxa z#k)5<RV2IG@_0+djPJ=k|14k*Tf^sjO@~M~JI#WMMp(A9WhXz=5I(ik&!^vbPKG?S zoY3{09`-p}48*Fj@>~rcljC)mE|1%G)XaN&)wr-fdw2L7E9!e2jhX_eDG}i_ITp*% zew<S%WERwA-~l_#FSa&2G$V!*6%=XgI@g+d-m9Xxy$Ca}(kvF|Wz@tErA17)6qu{t z4yVnnbiaNX;gEw$!KSNTtXeT0)?O;Whe#zkJvKKz%k8{e8#ms`6tPKj&-X-%KIFLS z^M-=$lF<A6%w$|AHwjGn`{SF|X1>Z<Z5u)oSrIl&^NZ7UQ*H;wx6O%^qU}+OH)m;! zbIoHK-%l=Ap7pkeETicD?68{f%e15X`qdH?b@{>h^R5CGwx5sBEvFb8#2m*E12t@c zpmMvYw97X+9l*Sg<`#9#-0h|7=bt$`hrgL=F3<R$+2b_mCVhYY=ly%l0G|6z#XQ;j zq4QSsNtN(isQ<f=y3=Plod<Jbap~G9@Gdq}XKT3|)f+t}c}<FH2k}qE(6Yw}H5#=d zR)=)QF0Prwz1JgXeJ&DYeY~6}UBwAuUBZv<&%xQmah0^NsxNMrI>QS%Dm?1l7XiY2 zFD^y4x1GxIG`5eintwKCoqdCqn7S}C56fn=+BUpKwcp`g)JU;p*N5LlXhFmmIeC;@ zf+IpJ7oFoUk&aM#Fs{bg71MiFHxtrl!D@~A<rykOym;u(R+cTJ`VVk!o~7~}l?8fw zsQKrrCj}?x)hkesu&EvhVpXAl^AUl-y|Gik(h0Mb>qSO0T+yfJoyDS22`l{hjxVpF z8gVQ^nJrCT|F6BX{;R6#0{sCb1O!BskOpa_rBgx>k?vBu8|mgj>d+`%(j_1zB_fCJ z=8)2HK)T@$_`dgf|Azay=LbIm`|Le?X3d(J72oCQd$eVM9Tn7ke01}DC;4qcJOK{6 z%*oDtiu}O0Nmq4jP(k?StQ7t-C+n%G`x&8KQ)ykDALMf9q-E;QOiITFG3@iXs~jT_ z3HIwxEa$KZ+navhSrJZ>9g20GX0L?y_xDk$H4K`rnQ=wZtp?QSOiAUpvnRIb8|=?h zIE^xgew?(Nv{TIN#nryB`@S7ptgVymDG2vHT)N>%>a~n4WpvXA(Vp?Kpptud2X}I) z#XpuZN^)fN^(ZKFuO|!+XYE>8d-h`z5<W?`HU2jifUemX_WdJ9mI{h#xri=p;;V<S zX2%f_TZJDoE0kN$K7Sp!HF1cG$|v_e6cp4@U_8o&X3V;(M@`h4*a_x-$s4`e3m+d< zd+b+2N;Z?0$Q(xcVaCJVGwQ^CEmJp#rJSv4Cz~fX7#d17Q@NkSJaG6Rifm^enn>T& ztSD765|?pMosu}+i`Jxo^25)6DD&c&^%GmsbMMtiz1E5XT1G3vCu#NDn>_>i-Ft_r zkB@*0DVJV<_F*WU&6BIriLxfs*=~c`o6AA_&KM)QdtypqLhd(fT%RU;zxQ8@F|o3i zJ3^(ig=DbO_-khydl8*O79igQhV6j>UuHg97QKM=$Jn#bFiNehmWW#I@iT9t`lG}A z9_A=iU4Ppgb5RJHmFW8Q$}zIjz)Kr3)x$|15;^!m4VoXHTK{de(7@|@W8y=N^MikK z?u%y{G&xKGr>9;+<C8qxE@^tmBTZo5A7l*U5ckWtJQZy@nWb12V2#9j@`Ug)4*E-# zno7-*@K582*;xU;b-nt1Zl%wsmy|t^Ea+f<Am$uGlr!WYtL&D}iRSNDUR9ja*j){n zlhr&&L1Ih^09OK;rddM|p)o+hT$ro3lCh8trc^YHenrPcY~ff+A*JMX-MV`0&!zrC z-?=q6_gOF~9RO+#?seG^L6SNr%<=d5s2)|EUA^?4)k!SZ^le|EU}j=5#QZYsNFB5A zD@hcU1BQ&`YXx5{>P~JH3Zb+Q@}m`(?#4iVi!lVWXXJCyd6Tb6`tD9!`64b(Y9?pP zk4#-NNJD5_r3iKR0|kAEN$)pT**M8)n7wn5^41HaRCL#thH|hIMVdS_5t}m?@v5t? zc5mP3Aw|cgOzZzcN*|WI^7>PAvjczt`SRgEXJYw0qoni)2^2dYMF07@{maeo2V3;l z(eergqQlw?HS^cO%wy`$5|pg&(~c;62VZn!Y3Ob}Ko)DB?Zp!6R6(bRjL7fTRlX{G zJ??gugJ~4}To6LaD)!^EczM%OxW)sJE|&S@-ICkRz6<knQQyZ9l(Scb$S^*ruGev_ zllnNSOinDV;I?U#5^g0n<|C==FqfVIU3I}Hp>P(olrfA5?~~UWD0qqliZ?Uea_*rA z^al0ggkfKllP-0`yGqct>VIE8*3ivNf1S=?+fLLGceKXBZ>M(kXT6TL%Qi%9)Iuzs z!2lAbtiOFZG$D!<<ma^vNWp8`K9_w@M{m%y(v^q9AfxIYmc^J7G%<Uc_!w@aFZ5|h zg;_i@*!_b+@<m#u@%?@+Px2pT0nM9_do)x8UREo<6(5?e>Ys*>?(H+PEw8SQjgMM! zPdf{Yjlfr29}_%zg6s!ER+HpY$G>~89iw6;^6l@ViYpSKNDvvdE_@YTSaQ#*uirT0 z)vsO^7^UGt0~Yhff<1z;8@J`f%F3|c*~0)t)d4LH4AgIOb$n;-(o7ZFz9iBzz`x!9 zWu(06*wMhC<+;_KbsQ+WV9w$SuR*{Q*>qULvRloS35(^-IOoL6Me*WI&csIMW<6b3 zwU0(T1MvW{HpBC}kmC9`Zbr)nuwV5vu&_LFd*?QJGu$(9cG-4@Bjcb{sNTb-_03t_ zW~DiA^<t+*Cx1kc^o+El{rylDm7SKBN!hH=rL7q@w)DtRMx*sbKUsO>IcU(Xg_ZjE z)O2DBbz+pbp>Ot!v0fLCn!M*8JWXF<(9l$km8aE)<2I!31V>`M)p0U)A5%df(xOC; z2ipntRy&>8{S?iOy@V>iI&hS2`-1~bpdmK2YBlGlSPayfZx1~Jsk0I}?6<=-9n?O; z#h$AYJ6_)Uh4)jKWDMK!vO?-dj%Z`Q4TF8it4ej6Cv}d{OO<3HWcJPq5-01toOi}~ zde9OKYK`^oB!=2TD3j=MJvRx}mhSExT?2;scyO=%@CS5ACd?vrQa{<knO}h8zaB(+ zC8F0b0;zWp%)ptM#*C}&=D~GyJ)6Mnf~O1+@nA-5KaQ_jHfH6*<<-?k!5Z`Aq$HN+ znuZmcKaGswuG@9nv9DWv;0RI~r|sk6VNU*}>BI3vbMEB0xCW0gi=*lXxw$5u$F(BI zr!;+i=#OEpJ2QpsgBCooEJ!F){@K~tc0PMfqBjVNrLN|T>wze_*FDlNX}nSNW7cS5 z;<Fhb5z4<EzY~rnxVoIo-JCS=HmQk-kWcw$c4dyZoS+g&p($zzIrDOlx;URSRyr67 zq~YJ&8-D%#Lj$H9)4)0fk=H`2=gH1&+AG)x#IzCT)40NjRN=J6kIqcO0xwK?t#+Ds zV<O9KoLAPuN&5c$vCo(geL-4(mX~5>wMi5RYC;WouO9e*-->!GT0OLTqBb#V!K0mF z_qDe6hWXBFw6-TSKm?KeSwWk^$Y#Qs+|P%I2%+%cOpuS1ikt9lY)wzOC`%@WR7lF{ zIi%ug);sp~@y<E}+tukzczKPBb9_RgG>pdc_yw!(<Y~wJb$w0EBH)*ub%@@Ky}UW? z@JG3jlaoUN`P~D}fe1G1$pzPX=-wU@RS=%Q{mEbRtXheW(XdD9BaCb?E5AUy1o}<c zWBdgwMkqXAyC5x`OzM&GdB3L@+$)&-<U8_C)T2&tr_QLxNDLz_+L&`{VuBDS<j6zV zXC6A(l9DY(?Nd_zc#WZ&<lL|Jt-q>^NngM}!NB*67xXNguvEWVj-Wb1oGEQ*13|6Y zlKJkPo(;;tK|Fp)o0gK2N$ti`#a?=P`Y}`??-f8nYge5|{04<Yvmc&$g@o^O`jKK_ z<34zhKtRZ*S6TV>D}#UteyAj<Z<R!Ca8+S7pzn$EHG+oCw<~r<J^V4GsCWxg7)SAo zJs&A%B=9WG*<VEhHyVjjVNsFRWL?l2zYbJ*U=afc7ga{uySX(rHI>b4XB?z$7<+vy zDl%=~A0zE>Aa!WK<b!rOdi>aW5c!Xg?LgS$;%>Ru1pH*sAWu(En-^R3qI-ppd(@?W zwwdIvgcGuIkvy1sTO*y5m6z9?)%={T<JV`>@OGM*f}u=<#;<!LyuRm~#bs4fEMds; zv9Z4^!zwYu-#o>*2Xa0=FFbg%>paFbgRuzLR`?lmul)wrD2mG%^y(d5{PPXF54du< zX=GzbtF+%>ZGtxj<<>G^u{KI!NE%h-y+rV+J;vH3RM(hbN6Y@fX0EBKtDO1Jf);;= zg2*_m(s?ht+AK{^N2Po$%p^B~N6jQG_CEBUpkRZvbU?j({aS`+|4-O6k6G-tzEw`x z599Q#CqxqKIo4iYjUcEymbOVhO3grAPTUQzQ`bGmIHKAa_xK+2v*++B=FRPzB&=v= zOsfrYYvMzVC(C+X=g7rcQCcZl(tKa*{2tAD_P7W1o>o&ynB$NT5lsRHc&>9-?6b48 zvnnziiTpZwH0gkjr&^-CjA8gVQBp7(kF+`4fi!(Qjl`9qAw`xaJVX($o;${JQc9ES zlLZ63ARw8^gwq{IMql3t%4)Wc5hMmus7UvrGSu19D!RI{pSBDmj*gG+A&bB$d3nn- zGo$eZNU`nXtJYR6C1W@!@8O4j`Cw1EaH0fVU0ES?!^H10cC%J3=PlsCLk6+2RmC(! z*kRf0j^D8`F*`bFvWB7+ik+4PyJCv5urJWjn<~sl4Y{AVVUj^FRhf8}pJLy;tfY0v z#MD!$Ws$x1GbkRLM$`+5x_srQqLkf#WlqbDk1YdJF8nrRZqJumU+?ab=E`Xzi_sY? z#7RQ(=eRwGgL0wM3CJn{^;W;l9@>kEq0Hi*NH=3&&li20tK1b6V<;W#2CaoVRcj-b zV@5V2Cg1}OsZ-V<(<TmFF)VVU$W8=O(dgo)Vm@W0q~s6DI|IE&y#D}r+AGXeizk4; zVajR0mu{rdGggJb-qUb8O&r7f3&?^yIy{`Tt^uN#ET1=pV^;Hwdawa&9gMUp(H1c_ z4l0bB+CXoUlh~Py?kkaza9}Hv@2m2$(Zu>Afd}dy$!gdI5UFtS(LPsN`UiJ@002%n zM!?Y_J)hV#iiatKuDw?+Kv+B-Rw{+n>F4CwWRZ3K(%(f4z&^CZCznsY_$o;wN%5yr zk*S&(YY5U02N@MT6{v4XU!FCWgS|u^DU%>J&qN*rW^TN)<0TPv3?=Si5#ejb6+X@- zmV0vIiAmWEL4E><Jc0`w8UiaUaYh(<)=nkfQ9;5E{wmqsw$I`n<BdDGq70F-sJLhQ z(*)7c&?2OlK&rU)3%GR%7kpI{8H?_EhTW}PZHO2JEn?yKhSN{R&p(o1w%}uAEiLgu zl|gf_g~c$)Iy@5t`P8*#X004M$0CB|kmeFPy5>ywzl;p{P^Y{RJ*~iz!Z?F)$cs+w zvAOQAsVIE%jy@6Usd&Ms9(Uf>#ibBxDYxY%uLm}n6@P@vaY}!n&(_P5Rd`dU#5|hB z&ca0lJNdq+?(Xg&#V{xusHZ6?D$v^!7+G<7-@e6jfMi|Jc<YV;yyP5vOezr7!yR}g z7>t+2kCP8dRh#}IYFpHJv^qy1!Ht_1pn|d1{T&S)4Zsbkf<!9WJ%jNCM0z)<|Na6` z!u)yGrX&4LdUjOYvnG4#>BazHe!06C{L}Y1qJnCkC|9z4<a6ZrxKvYW+9WxdY}qPt zkpA+yEm`L8uJCX39!X=^6PTtRbwLP-<9(@^C0~Jrp>k1?B%|?CEmfB>RA#YQ(bJ(k za#G-wtG4M&ajj)3X0HG9O}a|<=Gsy7UWjcU$(_ah%g9Dn7L{D#$Ogf^cQtL|cg<g4 zzH?=xs2B6_@_te-85OF1$jK>i-J9n<C-%|vq)(Z^lw+e@;6tI{{Bij*kku2BeS*Gn z&ZiGRn2&OJ&cokNLxnDX*MwxuXlSpQmoRas9NGVj;e^r3zQ>kCa$>B42$F$l0z3?E z9v&pfzyT<u{qMuq!DnX&)UXdHWCK~<Kzj*1m-spKEAHuuNNq0JH`0hN3+8CAI0@G> zY6^ZLdi@{0#p;uf%U@ZSWkU@7upWF~#<Z<ZEM(@M(0(dXsUg;x1@%wyIsKE4(93D? zwefCwSv%N*eK+_U&H`~=1zI=%jyzq09Aj9lLOwY5ym|8SMgjS)W69;u%F6?d3Cnxw z9B7aTfBpOlok%V-*U{mP4%zcYW}PGbEoFS<AJ9)e$$QQbTJq;l*+j1{N1pI)o611X zKJP^nACnP>pG*IQh-keNyIvtByW8atHi}~?-LNl^wEgXNf;M~ho<<43p!dpp^=nQV z8CHX#aUY`a@Z8A>(1IYjVutUZQ&EM{zOJMVL&k$h@h4w_d^jWt&O&UYpH_V7OL^&0 z%9QsoAYk{<DaV~YY$e5HpUJKmU(!hj%`;wh<1`ty014T1)sb!QoR-%hDA6W{*cgx% zI-Gt815#4^BgQ6itjvG9{qQ09<WW|9$eAnrn54LpN#;@%n0NBRM+ybW43Mu=)>UL! zw{i|lHgB}Fl303N8)nV9`(0VqgSsNt3dHFxtCK@pUs=+HaAX=AnlGE3G@o{t?TGpu zbXOvGWKj5C5&HU|iP@_s%p4$!pKDh6<!sjO(<XFp^UoagpzFKuPxV2PtNJZ3H$FW_ z>J&^~{Jp-DS7vv!vk@S)q#@!kH~42Vw^kTC)Cbb$qEU7_XZE~^?L<CafhJnc1jhyE za(#y|-N%PQjDT|OSZAPdC5($X{D6^44B+$5muSj;x2|hjynFz<yJ2qSYp%+$0lDv4 z@rRbH&RRA-Wu3zx2Rgooa6$+^<m|)E;eN_<LA^nYJ{uI#`nBL15egIKcNUAm2A2qP zmNY9KPzXq8=4{-z&|FyH-CZC7I}tYUxu+r+{!Va#E|5|`Yny>!7`*uPP)q53D7QRI zcLxR)^?-^T<P{wa`pd;u!}SUc*I!GaA#>YI<%|Qy!xfs#;iA5mQ#XBDidv8$W6t5H zf+@bvCr-QH(8HK}xT)vIDRd7CMb##>+xBvTRbmT!?HNc(9(Z4Txbd<HsV$lOG1Lvb z#Oqt=F34B?W^9E_uE<oO#^<?x7cCe0W;*{+yF9RMQvVax?sGkun0M??;j`Cm61kGe zF7?hRhR7`>r{cYqIt1euZSm7^y}r-$=4`sZqxK|zqRv<gxG^#=`bC*A$*kU5$$uHX zm`3l&;dt=&9uy}sD2G@I>8HazdVC-A|5aTl9ob7Bjji<hQ&gCwC@`Y+>g1|V69NY% z5Xe0~XtLx^VPD9AHH*)xZ&|HQWVx;2;X-(Wz~z!B?#XCEJ-`16YKPCoA1n0lAy2U! zZCfm6E5FAo(z@=B*op3+BQ>p>HwWMiV(9uB>b{#9Xz`dcF=Gv*QyyrnKd~FI^>y&P za4!}JpRFf}oLqWvxw;t18n~D;!87C7PK?^0ig=SYe>H-|wzIvs>#^E;a~&}C1sx)H zA0oq;R?sKmhmTBmeKf20x@acKXV>UidLsf|!1th&u5-xNZj;YS%cer`WWu0je`DgL zv9UC_*yyrie%_dC%N<&ke5i@J^)m2(G=7t)M~vVo&$qK^UOFyF^l*0oH9SBrYQ*8& zLf0$h?>pbRcYUtAH}MG<3uaBH=cng61~;&v-Iz-l!0LQ43{(G*bd>j6-TU&VTKpPB z(>-gcEab)J>lu~)^#e1XgN&^*KC@tHW@JBUg5#P)k%nr|lg0D{b+*o>I#922ZhF{2 z*`#5wa^$UnJ{E*rMu9v1a(jSR=t3EuSFHV9*lvoWoW882<Y>mH=X~WTmvo@P(MAx* z)um}zh=_=cJd)3m)vUY#4xj#{MD$~Qih0YaR6376Aa_Po^6&ZePVxsY;u7PNe>ZMD z?Qu^>MnUT>m`KJJfV=>u3_Rq^7<<^*y6NK~e)=@Y*xHJk)JdvDD26)6AN;gfdaFP@ ze;B4vCnw(f7wZt^5rt05T(|uak~Hbrl0gf*5*CkRL|NjFn6Z#8$2z&YO9qtz!^v`q zIRi>Cl2L%PW<v=qu4*VH5!dBV&ts>OF`SH?pWCm5U>P2^@5Gmm4(pZHtM9;3dPdU~ zTLQt8!;qIxR0X6o$i(=0GzfS5&q|Ibu{-cew}{c<s>5Y%vpW0xN7`qj1rvVRWmCN3 zN(53$5I&wWuKF?oB#1OxNL%WwUXMtc4s7f9SwEIoWDD6})Nf``{*i7%Gk)>pjfrs* z(iRaWR{VN(_S~0{2oPByMiBH*HtDXSn^0o(@WzsH8j+Eu_5+X)i9ArarhH=fXQx%2 z@6YDTkUMA1EyqJNcXZL-encZt>P`Wh`v8;-$kqlKxy#GFw;EVhp5RosSgCkSo-Scw zvOb@2v=?tc2{;8|1THh<k{(;Xa9Rmqe*V%63|SCEyfY4-e@Yk#V806^JQU>^=+al& z1mH-NK?(r1F_<pw3Csc;Cl_XdPyX&!6p}$!Jz(DN&bNq?;tYTXUcHm~hvki1#>i%B zz3H<yX_`=fP#C(7Re}HpO(@A+?(7wo2*??#A*Zna^1ajcg*hWtk7ep&Jf}eVpNp42 zlpqk~3eEQitq{oaie^FQ4f9!X*K4(Wylh<JyL~PoT+qb<4oBk3|E$wV^0)g5Wdf5L zBXRl&n#!N=L!E`v!^qAIgkoZsPaGZ%=xAZZjQ+4cO(;c=tDwb;mH+Z$*B0sP8+f80 z^!P$zPZJqKmR7j|#|2^a6vEn4kk@I-VJW#d?edzk_t+I(-^<gNDF#iVZE^A?n>E)u zvE)cTTcJPWM$JheWWAai0x?ISStPcy$ns@QmH0T=$Wh<Vocq~B`fgP6ajHdN%S+Fb zFNpl!4Df!yzFeu$7^U9X#C3;?I%&;@wW~e4N$1RoHn{>Bx5canB?cNN=l<y-`qLJp zd(U=ekmfrPC$>y1T>1IlR=bWEq7A-AEo_?F>!dhR>0UMiNOsHn3D20HGFN9q;#o?W zFMiTLxmnrpKCyB06e2lTUIR&lS7%dc6-pbgc}umX714I(%HH%DS1et{DakQol1120 zd%<@py!L*M5I#g+D=C=R;-MFDz`tS!3VX2eUDec#EpP6q^$Re>cdvwz#tld@Pr?Dh zCQX4CUXghM^LwT|ybg+sMV}n?>1h>`qy9KNrP$YCp^B(5)(&vijg8+Xi?g_ozZXG^ z!mjw4dBEeMk5CfS-pae0v%5Nc20_+dJU`+Y()^wDNjo2V^*6)N)-SvHmdBU(7w((C zdKrB6nnJWTE89;I61)Ou&1hLot=L=6vHNB-y)U%R|1rPS;<DeY{}dKUr~pffA@I{= zVtuO`ZbsRIDaH~P6E(h%98F71ncUy`XatSXqJ*i@Ug~UK-}3uGSJMYuK1jgo!hDOa zfm+c05?pkC1$SQ{;_EcbasDxsuZFEV{0A9jlzt$P$XleOFtUtfB{M$|4%VMb%XXUc z9UrHU$GnPf_WVU^1m;_=ORk#9&d|^v>&0})%NXiU-iUnxC_W`EX=qbZEh}DFyqn{I zu5z0L0VuXjwnOUW32HEletCF`swTBoZaX<PGdr|Z%;Zw8rHl&E`B3YFAdK3cA@n+E z9&k$i>i*VBD9LeEdq7irR#We=EK5KBAL$N*X~#@E2BJuMCl;6#PkTOF89om+117|n z)*bE-t<ldA9B@7V%@n6Z-?N|CWJBhCEs^iN65v=Ri~D2P!eXnrrp*6cZ5z18GDo~P zJ^O>b@km))8j6qMLf1h3`eAUetssLFd-+d4OvP_Rzf|ZMBkmS3b2Omg*XQrudO{A# zSKYvw69WlKo`XSv66~K%I)&zkBtQNqZ7&2ORu~^cJrwlAi7p2R;)hRpj|fEz393N* z)fIKlV||=!oFyVSr|X;X3v*N@zYUiWhIg3Z&p_2Z*p~@QXKIFRy}&Pgm_f)!nGBCa z*BJ1Sv7y&N5JlVfdcV3TkRiG9>M+<6q}O`T2{|opika`(BTc7M4dnl;xt$&uQ8rC{ zSLU*HDj|Cuyk-F%<EwZ8>EtpCCSs2^4%D|AM1iE-=H6h^K%|?-3nq{u@@!0O%#iMH z8S=u9dh1*hv+<B$gukO6<xg%B0N%=fmxZdZR)=wPRia<YoMM*mDMv$lX)kK~U$hlk zS0UZ`i2iQ<l8cExKbStyP%SBNd4pXfK_Pp$L5w!x$HP#R1B{fhgnu61ZKVZJ{~pHx zT%lSUs9Ww6_T2Bk+%H%^F=19E9sMv(e(_NR`EGIo*4YUH-a(`TC{QCCsR;Z3Y%}>m zDlhne(>5TA%F~TlhXk$?p^+%ruV^7bhy;%%Upj}i-FjQ4?{q38y35$~U%hV5&C~;& zMA7z0hR!USuA4NtcZZ~SWjRKwduzCNC)q5O`vV6MN(|&&XP|g7`jqb81^_!yQBmWS ztEPnD6_j`Dg!6EnZkZ%?OiD`e*!xi1puyh1YNcBH6N}bQqPB;(ZZt64Q~}d%|KI?y zZa@qN6!rFr^rwVmcVn0q`$wFR%Gv9b9GK<*>L9A1Rc>$;`MVT>UuN+?_@B;}Fjr_J zxjoeb92Y2fs%%-IIr?w+(#CaFRB)szm~ke3B?2%I>C<t)!N!N!XdwT;-uvQMk%9Z) z)&owM^?Y*^$h9Yry48SL{Fvwtl-vME7^=$DP6SXz{~m$*PEYHy89@GfTo+h?Z!f*; zIMR-X5)8Sd9ew{Q(9HwZfdf8YJSu0F|90G6)Jj~}|2(>Gc1NDhsA}q)HsY`K;cw5q zNO?*^0)a5s(FEZoPY|UbFz_UtBH9+gX^gaEG4Q#>y5^r*q%8M9JZg-e)-wKsGN@EP z94KY0$j~M7Nuelmrh*tjgQPDR$IQ8_$WU+l>OM$-3L3;_KvhabP~r)!u%ihGE0(N$ zyzSVN`*$6C``5sS+}f#IL4&ewL?5Y%u*K|w2!Suo(SvmlT-q10)@mCv$tP(<*;7UK zK;nQjH*+Ge6@%|7`RzbaK>oXm^O4d8X|g<Fu?rmx2~=j+`eB@e;Jba6iZ*~~Nm2`> z^_z1mhaG?~as8-81Cdl9w;2o!0$v*;k@fH5pI2UPw>4ghj!PcVv2;5O=Fp$wq+r-X zC8^UCxPS?8^tR(}+c+ruC{z-IYRH`X|Bek1G;X_soI=K2z7mqZawf-+8^Hlx!YJGK z!xKOw{m=Rr79LJ)bQ>591pW59^)G<TQmEQ(Tf231e%l&unTP-M1k!N7WrvM&j0I3G zd{Fs&VSE;YoIu*DJNYLjWXLPYOfy3J$;aykt#^u4a@JbLad@Hpp2Ptfl_Wv?6!fpg z%8H9n1|&u^!7%<n#f5tiQmcS#9#Y)=@jCR9FM)S&bKBX(0cg%o47<VVw911F6Iys% zh=j-YuU<-SF4OjaK=6%aCB;>__>ip0qhq6Gk&W!-BD2IPFwru(0{JOA8F|d_Tm3tY zobDS%M~KlvD)4aHC3bPy{Kk>ham^K~pFO-4y~{`8?ZX@2S`B~1{++bu`anm4jxgu` z9f!asYTJH%PKhDhUSv>sYbQI9XL$mENhHL?jBIQGsj<vwTSN1A`mP7Q${%TPMTALG z=jP@<c<`W#tbgVu2A<cnjQLMS9t!-dC}desJI|OdZewffDFsEwe^uMLA7IMJ$jF#z zLEGQ+@q(Z_uiFWf&Os9SA|;aJ1#motN}39i{#6VjALZ$|I61Q!!WhDWgM*h<Q}=*F zCiJd#*c}eQ;MZjws|*}rT=2!e>grG&BS2A8lcA^b4sv&EGdC}5qu#;jdmT&rP~t1( zPU7*BTZoJ*Sybd0+v~-_6uFI{`tQI{ap%8Ff10I&zMAt_p93du>{qlW68XqliGi&k zL8@RDq=;J!MYT-dNr^d*EG<5uFNR9~T(s9@Nu0t#r{ca@$kz>00Oda`OG{btFh0xY z{r>&CyIZb=1_Td)udRN3MLP_tJ{_DH-}VF`fh8lr96Z4F`6ackD#kdrbtE*<@xB5I z#R05B!T(qv>4GGv$|sn6Dk(8YB28~?ZGEqS%w(zfE94g!7msW>1BN9(Kfj7PeZ0xk z?nzW+B$6LM2MDB1IblDRQqp-WRiO={lAXGD0DGALlLH1oR6Q%n!<#1;fvuDf(9D2< z07jaYfY!|UT@4*T{yBChHflArcXQJecJ%RSro@<1ij0T=NDtzi_qVJ(MTRdm-)DCi z*VI7ix^`nuKr(tLQ5jW|{sH#6Dkqs^6@W{{D1Q*CcybDwLNJ0QL^5_YOF=XJ#UK!D zw3p)It?@Kb;5Bh@l;t9&m*NkKIHDpV7{cDaNC6<V`0>TXcSjy(jWYgXhQ`K{i6%!L z`QLDGaf>Iw>jIx$-k~uA0idNJLqGtWqN07t77t0JLO$1pyJ{14%X~R3H6w#9?EP`G zrD#v3^^25z-EtB&b|TO{hV1e2@y7GwFii#m<tS@*BA_SoXIcRzNsJQE6{iPAjM8TS zg6eIkrM0y+03XAcks(LqNWeGG%)s!uvV2+&gf3w}|HRy&772C%1?95&Bm7Njh)7^m zg(&(GawbLSm$NZ3CC?R+YZ0_pR#q~e@RH&H>Iv+#dNC`wha>P_7Ef#ez@U0@TIDXl zm$LM59m6IkCyk52u*_*}PHb;(S$6O4F2@{scsn@k9jXIV8^{t6(0li8@z8Vm<l>p^ z@P;#alv%es;07-&WM(q}^$S{EpACR(b94F<oe@nBLnQ$P6v+_g-2sD*GIJ7PgC1%~ z;CKN(XGJL$yZ|UNvT+!p!F?9C!dF&V>DWycAxXVSw#--d<%_XLuYS3P5&;+;*M=jH ztBBH$?=69Wft!!t*hNZGga5_ST5u-=0cR|qH0Rzt@&IkJtOk4;l`?ZLFHkxqSGkb5 zoEJY7Fu+uqa)-<@v9R)GRu0Za4-{Im&AF#1COU{l%)zllCV#}F78Eq!*|9yk{;pfb zcma)yD%JoSDnQRXY~H`P+7t$COBJY)yVNGLnTNrdSF=3bOUBZIg3tTuK)(Ur>F9-y zvNGotPoeZP3<)4!ho8rAFS+!UxJlHaN?=|k<HSbLjwL|MNs<Ryg#-iy^1EGGSy{%M zg=3caCr*HoeCE|<tW3~OY;=6)wc(t+<{s5y#7&Z)T3cTJ_RTvtA@aah_LW9JjD=8! zWshudC&`Z(sT(}KomVkAL+gWsiUMil=G?#>ejXF@&p*nA@eE<MwHKPD@x!sPv5XUf z?)uQBTWLEPTUan;O8D*G{PnAScMQF+$~|%WhS?EMqy6V#+r?NKDw*BVBATW!Fgb8X zMB`0UC>G}THYlGcp3B)-13br<FJJDt=|)=0i2?xP(e=KNXcG^4lxNeNvp}CiDjWe! z?<(437q1xsP!s?a8`Qj0Q6PZan~{MFcKvtxfGR;idz6)_i<-)xT|SEH03h9$S$I^E zc=rJA=87*?E>~Myd-O1|)o^gb8EBWd(pXvquo{p@q4;$f1L<>Z@UB!-iU)jRI{f=Q zkhre+tFOTJj!Pj@l2?n37T9XoIma9G>KkCPZm?EOm5f<}{c@F#pn!mv$h`5^q4yZg z(?~ED=GEHtIsj+}n4ihnzxnztt4$}g6g^+Qp$*Lj32ap-eNegbi7n^vjg5`l&B=|_ zIYR|wmQ_HjSy)<P21~?Um>R^e^lUb7N(=$7$3-LXPL`Iwd1JUIWLK?SMS%(64}fJk z{ZwcmrEhhc85mpur*iW#xQtPlQFeH-Eo7`<bbLEsT1p9rT&Q5q`&Q_4+7>@s1A-+0 zFIv8}tkynyq<ZWQ_{qR2HhRGv_B(#Otn4`tnISiclv1yV9aF;Q`ub=2$Tu5nVCOzN zi#YP2-!(kmVv2bM@O4F{L3p>l8V1mZmdr--c=ww0MOPXDw9u!;;LUnL1~Huuu@Tr2 zr8epxap4<-G`8c5t9WJTQr=+#tt=oLqM)E`l8tU1sgxDu=gS}-q^|)VXS~nO=BBa{ z5IPl)mx(KY+|HxKfDxcx?9wnh#Zg;RQzWbALYvpI)CDc7t_F(>z68K|OI3XR%HB|2 zRaFH9yXjUY!@??c08|Wo#yr3H#3uF9%EgY3jg7(K#XH4wSKH$N$8{7~kQ~?7j0JiP zxVvqh1<7y79*xb-&zCb`Qc3nbGued?4Q0`pj50GZFm&!jCMG`N=>m-bOUDEj1TYFk zGNSN)I;5W03pZ5f<m7<2xO$N0XsnfZaefYbp@3J%C$|ybya`@*W8+;!Xm;m;*Ku=L zHrUD^9v^eU^5#(?fW-{hc{D)<NqO5eJ(V+KW4UvvM-LiHO02tum2;Hi=)giWbpde8 zH9+d_lku!}dHx~-h|R6_bqFL<`XwVP3rlJ*Di;jI69@<jjvW*cM|XLT&6!(SsoWIS zn9k?S&CP*<)k?firE;!yh1fwL)O4hjSe`%cZw>_5YEUHig8#w;DYLp-?TvU*t%aPG zj*boxukSQQEDgL6Lqo&cKQ&=RRP68XZ#M&P{_VYYDRaNo)$uiy1BE;`zM;ec^YHWp z(lq9x%1!ddwb+ADTSw<U)n@r0uo8X{2Wy3<i_4L5aW(@Ocve<cpmJ_n(mbae;GI7q z^0VjfWo<<L{dET&Vx^wA-_(Az;f^N4Awd|Y*vghN^Ms=ieq@r{lQ&huXtql@yx&zM z@P+m|+tERYyyg0Q%U0jl*Oy`pOyU}wM1z~_i`}7|x=Vu=pR}4t;TF%6=hrUWBqnQW zB`}{Bira}y`SbuNPP>+?8aU$jpn><1+K03yIoY=XAIwI%`$BLje6EfrmiG9FuyI1# zPUd~*OMNa+_kcF%BgH`rM7Umn+JUacAfcTD;By+`h)SEA%Y%VhyLlLOt<41j5OyXu zqeQPyS3VhJ2Rhg~4;+VtgiJa$Mw{xL9&U^@eAxYsX_LI#m%=+c=H}*B2OyWbCml-_ zHgF1`)2s#?RY3R#87}bKaThEC-v#E)jsoDbK_F~SE<&RyM5TR~R4-Rk?7m-lgBx5N z+oWpPz?<C<=+Y`c&4t~084#({_F>+80^t*3*)R(;u(h*mn8)`%Kit^ad43MBv8m~I z7;jewTZs&x^Fp0S;`Rdd;;{kIo0-yPk0Yk*3zmELP91$DFmA|!MiWRgoxg(3FTK8K zTg2mKn%%VLYiw?YzBP^OqV=q!v1R|gG~_1KUj)`}ASL#EU*I*BBpI9CmJvlZ@YA{} z9h+o2k#Vya-EDqTZj#7K#*RL-2!7ansRCVGczAe?jgj<Jx{;kyQ(ax>u@FflNLW*d z<hMO|5AW&ev4#Won?VLSP6(%Q1iX>xz?hS<xT1F5aJQyyk$rYJW4flqRgj4Ha>}c` zyj%snN=EeimJoSV&-2s`!w7y`!++Y|)vI&#pMFdw<27|?4&*4R#NsHLtl2gk;nvX5 zD8yRHB`_%Mi0G~oKB^^EbEaDL+++&3`r6p7jQ;kBSAfXqxhJ6WR?qKx`c^(1TiPS4 zJv1TAma&0Ln@$L>j<)Vxy-j_}vFc3cT)e>)-dL^!*pwTM0%NVCIgX{nS{nTY7RjAE zo>6e_H$vI?WZMXd-r5hP%IHRyD>W9XQ-SPnMt=EHc=i_!<v7=zr~~q?aZUd$j&#tF zV9uey6vFV}?0dN7Ha+}>-o;9>Tv6hL(n&(rhi463-N!TBeZVz6{JG~E=BX71tK2l^ z44vqzyqL0#AKP<FEgLMsnw=<i6J=fV6e3@eWAuz))q(O~;~PQu%!K94^bYugd(~+W z{P3EKexJ<<;j#Jcqaxik=kbI!;}PzdTo~M3>rFY+o4sxdUy$Y7WHrY9W>e{ae*h7- z%mLGNrP7;yn0(UDE1-GI+gISl7S18#oM8H?kN1f8x?U%(&vcg9WX$bIfx}^Zel&Pp zNlUnzWLD{@%&C`Qj!(hnDcSa$!m{%6T#X;193OYX{d_A<$*r68c{4I9yL_TS{sX{n z1`%jgTO_x?2MOvPwUL`1P0+XeCP!yy?)AFqg|I5U+q5wK#!erM8vXK&j0_`Hu$#G9 zf5H`}Gvhc=JjRrc6~6YN5>=}dh&Yo-RsY(73jCOyH}Y(>pxGF{U&k@83K}|PDI_Wi z7Uqvh{Te;Qte<*Ro9gQs*Pa$X9B?%mJMw7KWO-j&xg_lZo|<ge%K#NjQ>eN}TjUTf zm@&{(9kv5z?KUsQYyxSdG@C)Qkc<8iv|fi*#>EHDJrB$=DSeeTyeWO}E7nT^?zYb~ zPCaMaPypVs<WNuIP?J%ZSJk8&*=Krict~(1EV>G|Y_qj?wea@$4?9?kXK@le6=p?w ziuiU4aq`Cas@7D-Mswzf!S=1)VeP=Zo}_1m6XuoYVan5N+-ria!js<Dt8UaV3-ab+ zDn=^>-?@_eO!Ea!-xh(>AE+aA1J}N?sTSHfvrOIv@ks+~S}wKl+q=nu+6OFeCT(lM z4FeMPd(!J9a;61~R1pY?ZFhE}MxY(G>^`XS^72mAtaSkG_as~@h^C1vjQmLb<l<^+ z$;kQ7>gsB*b^`O1FmHGDW|^^FRJMnu>f5((E7T9p8dL2^)PyF1AP<!$!qX|707uGN zv_<{m%BMn8GMXUjX~y({uZ}P&PE=G>{y0!Imq!x@X5T5cyoyMSM5<#!yG&Kdd2QgJ z5QT^+SI&m>rmr*V<^26yf-a;WIRoz&kFLmrbVtNgdvwQC{Qp1xKi(VFHxNns95V)g RjHz2Okd;!9ER!${_&=vlo38)> literal 0 HcmV?d00001 diff --git a/website/www/site/static/images/blog/apache-beam-flink-and-kubernetes/job-metrics.png b/website/www/site/static/images/blog/apache-beam-flink-and-kubernetes/job-metrics.png new file mode 100644 index 0000000000000000000000000000000000000000..1e11248a2f0c99e96ecc56d384722bfbce49439a GIT binary patch literal 348117 zcmd43by$?$yFY4DBArqSQqrKnkOBfCB`P4zkRma_(5ZwphzLWcfOL0v3=AdRHA4(7 zJuv6-efR#p-+j)t_jS&7e!uISe;#1Xf^|RZj&-lk{kaKwt)_UN-~qvn8#nGNy?pWJ z#tmGY8#iv>#{-@>zD<$D-njAUhSCd}cW#F0CR`^?#p+_u9=g@0%Y|*|kR)osXFm7R z$=)^1&Xq2=I)Yrd?L`bB4!H_H2$c*)Fo$sn;sx}AP=w6>cV2Pe`n@1&!|Gw@Y}{D_ z+DdEq^{^uGx2;O5K91)<(Hu|3CTe1LTJRqQkjUcw+v5;-ve*cB=OThGYAPZ;T#=ha z=qNAF{U}F;EvC;*Fya6F<&ribK<584`qxx$ruhGVd^GhoQ05es8h4c3!@cStqw)8f z45V?dA}g8*omp6p%B;=*>#LQcs*<mB(@dnF5at*)eEC)R^v92D<2*ZMWnVUB<y?c< zzxU#ZMY%`Qa=~eplZ(|pGtTE3s_MV~9l>en%y>Z#N^*}NRh<H-#IV64vM@ZHP@>kC z>FnAQwRrm1_V@11HMkNdAl9~Lj_IOQi|4xgXi@JlI#7;(RI_~FZn4f(inC^MGk1nO zP)6b28~R4~qc~6nqlBArcUqdoUy8gQ@W)+TohX{E-BoE%{p}K`N7LxMt%ZOG{-bC& z-m`Ww?Yzt!*m=0#yxUAnU|YcuFCoF7#n#wGv31^;)k2mYhvoM%$uUpy{r|C$cD=b4 z-$m{ArEyC#oWK6!A>Ds`^OqrY!k4lM{+@pxZ3Es{6!wVY=lI9Q8Y2Q^{;U!#q0pZG zZvxT2q@nn;O0VhTzs2~=CWB$0|DR58(~iGXb1%Oo2_H3>INhDEz^2_bgr4piKqW?% zHk*YlXNK;L8S8?^po{*<5X)<5LksBb-dq{fQGAzcchT3_d^gOq9?rqFT|C}ZF)=@c zFV)fHdCG&8uCgniErQJ?lgMJ`%E|_m+m~Xxbxu*}q%>#!Wp~$Pfs-7l?={5x(tGlF zVJi&^FW{@)4wp}b(*2{uN8tTfRxYl4q%hv)l`eaIX#cc9w}_qDlHbvETZiJ;@82=; z8*y|A2_j|gaAB1c;oC<@zRogJcKh{L^pT8`7FI2I1Jn#j?CNQ4A*C_XJD-W_JAXXB zL=Q|?AiW+_EUv7O*`M`lbaoo{UC-T56X4-txJQk<`aOn?0fq@Tc^5}~_gS2)TPLf2 zy%+6d$>P55rMZ|&lC$&1Fu2gLf{IQABI9<(YG!73fp%}azbzysMT$$z*v*#9!yszk zo7W>+0LzV~j2d(BJa>Fd1loy{mfm^rD8OwTERDJk{<^8}jMPW*>9>M<=2qEyqJFHI z(K~dH#pJ5ZPMtp$wG0jqSFnG{af?l*J<*vi86?X_#3UK{^XCWnURXOORnimDef|5l zQ`+(A!1hT>g8-|&7!sjpeCibk8mU-Hb(MhAz$R;v?@x;u8lOP_30!MDWH?4L?_D_a z@@PbSm=&VvqR7Fy1EAP%-$JXlh0`@+;x@*7Nxr>(Q=TFV+HSqi^2=p!I?QpsKWlVS zQ$X1Gmdnn9|M@+tOq=O)QTM}lCw0=l3{Dh5qgTsN5e&xB?6JoQ!>5w`iOz%y)PRnB zn}C}Y-p5~sf!(T!@$pI<pOPt`5|4irZcVJBtB@Bx^*&2>^6}BDxv5pG8v%rm$82oi z%afF;a_6L28ASDZvoH!}m%1*>kl&#g(Y2O0W`B|luRB|6%i1{$I&{ZegQTIp2<c{n z2G=a}0%5o9qM8F_5+t)V39_0M^$Oe_URMi9PmZ-q{oQaWG_(k1gyW~N)(9@`?G8c3 z)i?-JrJz)?1(R*Kcco=AGPC#lQ1T-`tf0AS)zZk9hy*I%$cTupjk48)<)AK7YHDm2 zY1U)Q2dQaE>yD9@v}<wssB_R5r7Rw#!>60B{o!TNZCv6E#l7o|g@Uw*Fig1j=7t0L z>wG6#l{Bf_d3jdTleL%Pi6~0}M7+?)xyN2wncRuutD}ik+tDjOw11kJt0aMXx@Vb+ zeZ2@BbI(n^IeHGh+^5U3Z6G=!`zrozPp5cjT3tG?1qo7&N_pPqXOymcJVPVx@J&c9 zrH=|na;z`Jvc4{*W09PUyv%da#}c!Uv<>H$cG=0=M5X$k9;?6dpL9<}xf7w@=K>Dm zC$DTn)OJYqujsY^qJ4=A56;WUiSHj2`5gOM&i)MSwA8JtH(V`8{jNx7@<lnCwG{{& z7qw*6mKyqAa*Is?)Pa==709JAX4c_)Z+4s6GV*0oQar-sFRlyjJ()P}j480bzCO0E zbz|#TD=VwukO+A%S-<hB+?eS7(>tiCcwmczq_6SClYetw3oud4EO=w(C@wSa2#dO- z>cZ`ogq~R~Jn%ea;8$duC1uF<Qdo$J?8_UoSeZIljZ15~Jjqg>J^3(G2qIz>vpKhi zksl3OhDWJCe{z%>8#{k;Ce7f{Xla9{L^V=~OIyx5!QSp&-xhHE{k{SB=<C<7A8b=K zA|-;{%^rwA>iXLZ6!P-&EI@GN$&R?F<8RUW<Bh<fFt26IBKggms{&q~c)tqAZ!AvO z2D2*5>062U+9>sLQju<5d#+S8Z2Pwe?KKj^gPjo<uhd&Cm~H%GjN+K!3qnkogtRX4 zp?1L)p}D4QU7YW>0qxw|dkSCPS|S{(S=8CHs^n2p2A;naVz;XJ?T26ALBt2%^#20r z_`!$SVg_FQ#&~buEEXMN?UfbPIpnv}Mq(BXocoNCH4B#mnZ}z=>)s^>4cEPg5S0!d zKcW2CA-JOx3?CknRz~zK<#0u4-u@y>+_k*W$fTYdm(g{#em73Ow4CAbVQq{W!sTHA z$wH^3g1a2eT%9$sZq0Up@Q%1%+5lcVt&rtvn4{)!o}M}#e1621UT6wCGsQ4eHcRO~ zba2qx*70GkTylZy-L0|Y=49_9&0Hp*me&p{@lU%t6;k064uosob4U1*tmvBa%%I~v z;!9+*Jw1nZ2F}MxTboshqpqj=P=B}1&dwRpwxhYphNBb=|04aQ+urx@l|9|I$S89$ z*99-kg|v^QDK`o~`Es9TOjW%lbc&1q^sQ~x&C}yX+^B5(U5DdDAw;KTRD+$Hbf}*< z4x#L+p*IiNh=DW(`b@}ZV5MR|wK46IrsrwoCcWOdbdO<VLF#Jq`XX(~%F3z%H0GVu zDzQd7{n-1H8aoF=<XjKuQGqbE1J7yyMVf#o(E{f?vVzItn#CqcPR@TF+1TfQIZfnB z_b@n1MI}Sd9$vk(F;&I>@t0L_y8FEWPKDjMX44UUWmuqb`x<AwsC)jE*DmHRRLh~S z2PZ;rWu91CZs$U^v=ijA^?Ng@eW^jz<998nZKE+s9FieVJsPbLG#8!K!|!@68z#8+ z#$dL3kDom@_jdmdw|<S?ZJ*-;Be2r`8}eda{n}vOdC%2BSLqkgE7=0xT3<s%^kpl6 zvyw*YV>pjc|7DSG1yO-kNf+jB7<$J`?1-kW$DEzkO4NMIiGs%vhl%n@C@D+N((lcR zi3yr#Wu_=QJ=yT{)x)d-AqGdgo=1WGIx<jxyY|O-22&ui46yy*0YlH(En7bOk7Q{Z z-BC*mNe3OFvBR+iwjml~RO$hN;Aki@?{v3|I<BdkTL2XC1VuX`#b!U^AUfop<8ePJ zU2$lE3O^w}J~%#}CV<=M^OI-WnaWX8$e)ty5@VUH%4M#5Qp&K%x%CXL9SH@q1;?$8 z#<w|-5SA3do5O^O4XvQ&Q4k~+<fhXt?y@Vm9A28rQCu`zI+7Zw+*ka*t~)Qu-kJ_g zLlY?I{xLm8$l=vP7x7cnVjZ0n`F6R=dC72|HF4N=`X{&DIrm)dIAi31PW2j38q8!) zhFwxYKqZ-eqcTNaE68edGdMJ|dzduq^?t?5gu;9Fx3?{g*SDV4I)O${LQEbw4Id_g z)C?lho_6&|!I6UZU0#Oc>D1fQRgq7yr&+D%S558B!A2^*)VP8S8Y(VX#4JT1^IvF% z{7Lx1<xsEcD!LO>S-fSgDXg0+jnhS=3qwPucW!Yl4Wr$aVRUBKaJ`&1T?>fl$cMgR zS{<6&N{O2zdWHUSKZ#Lck=-PJOp%ILg9zgLev0<!%T)eN^wB+PzA&Y?In!tGb6gk` z>dY5?hEIMXTI^V8;bD54ZAKunhE{C>YSO->U8Rw0D1@H%9e%+_U{K)RB$O)Tk?idm z4}}=+Aw+tzrV&%(-F>bvHwY@0%ri~|Lr`b^b$5<5vs}C}7p^548e>$#T5o@5{p_~X zZG1cx_sSduWN<_G&PGRb2K9EX;j*#<)9y$h*O4T%qVd@jQ~<eiYvoIX#&L%&BO3jo zPf|P1M|S5uxzp%{sCf)Svw9-+u5AKwa#N9f`eyJiP_+(}!PZ>R>QC{jjSU*QOE(D8 z0q`WPIBt>9&-&%!_A@GQ5tK+W@c{-Nt)Vr<oqG|HxVnfvD*3RGpE6E~y7RBIj>PUU zVY6Q;dam6#t@$>ON3nH@4;z76vRFrECLYUlt@dz0-{+0dB6g3HO-o2kjn(8x06)Q7 zr~Yxs9R1yUl=4jqgJo`;f>6J@z_u$~qQvKundV%CG@?@hoCL}g1vo^EktZqj;Qirc zwF}98gQknA3d<-^jzn2D8*uib<im728Bz>APxlmSQm<a0C+Yq)EJy`Cq-E&EQ1HGX zyGLDjr@HTLZWee<>%&40%a;fmbD{<z6k@jG4fmRq|4s6o^w-g&9A`=b;;6MgA><2# z3ZL7$t{Z-a-ni-5^ha@%pXI)oE`F9nhMT8;{Dqn5Y<|?CnCUqrg*48p?X2GVvOZ{u zJS?PqqABvIO1<Ut#!M6SgPITiySIaMbatm3sH?WUf<RED&aXRZL5OKBTk%Bx%>*{z zW-k!g)dg{>wae~8jCV4&B3#UM&!0eZ+xP-~K2>HNg`dZ5O;F%u@AI}u?1`5qt@UJ4 z%}J@-G{G;(ME6O^57ATe^^a`4C+M+7PXFr<!#}d$;>-^!ZG_e<w-u(>eV8l@ds`!? za%MAKEOd)aRnAGJP;xBU{pg@CH!VXBWjEnM6Y>hTh|c99jMqS6{6W?0qlPc<%RWZn zXNTLZ8Ag|BuMBLe3dBUtC`2S?LMItFw!gD(o3^Hdr<zS+JM+DGQ<K++!eY*wk!@;{ zRW|cP&fCxIQ!PKs2Mx<KS}yuLwH+swOljQ5*hzYb9}7<Pvx8X2?+kPg>gueaKgr0A zl&gBaE6^a`5p$Anx#>SrrO5XSGDTCQTUGXwFs@@_M_lEXnXWf3VR^3(Z=a%NF^TLM z0py-P`Yo-kG~}B3-BmSXzIMrSxbh|1V<9Y)>@HhmM>4Mc+%|gPe*WayL>x&}c(@?A z<Yh?t8H!z=-?*2bUYgVbTF1-^PZ;i^j;L+q=y{$4ouG|6C!|l(tKMDmlIrdLh7<9? zbSb@UnshbQjllo%v_T7JDgUN(3J7nEf!~ehBWM->Zo7~rVmjHRVN6}bZuHlMQ}L7G zELUkS%{rV1prhN(y~}k>0=`J77N%az=R9dKBkwI+2A38~J8e>Jd;5NL)=2jz3Kujq z*+ts}a$OEnEesMu969LoYA<uq6^J6}C0)?b>uP*o6ph#7Z;<1JQSkX^bSO^4`3no4 zSx;uTw%im(5;=`Rw!<YE#7|`1+-mXD!&kVqo0-L&wpIo*l2_HFKKg1i`6ri~NFHXV zbt%V*tvXsJHT-rM{W$zHWIbdgDO~hN`dMFY;EqR}ypScM>`nXY_AV;(UU$9u3)Ou{ z3&x*^XQ+NTS^r-bp!SG>Hf<7eoR}B^vkf4Fn<WA4UVenkd?eo+hHkp(1@5p8yK{HL z2xx@tZpq4O>riN3bq~JI_F(Yrkd%^QTxC})Gk<uFbw>&9D#?NOk6W<w3pu$lC9;7f zr8cy%DwSs_=C0(0D;(3gGbO~3I1kwYu`ec!XI_4}pL{|*u;AK1DAVC+0{Ox0>CHeo zv1<+jNBpN^Yg&c+5z>t(!qt#P+jF|7W_Owk-%}QeIX%0{E+{zK+3G6IYeV{pqIBa2 zCjX7Im(Gdx>{ukETj(O)s=(!^M#D`DM5<{t-c`z_rmGK=pkuNmE<*(NdJnt12BxpJ zo3(?SsVC6}bSOqiQXRhpm%6rm=D^5<Dz~TRnQxrO`L4#<9%yB(P<?hlqIG^@s;NAV zH!V%J*5T1qZSU)C?uM-#1#G5C`!WLN-7PYn|48u3R`hDPVZ4Mhjuht`9Kg(l*xBz6 zI8r{*p;&45zC4(0Y58n)bwaGxq$(^>)ACj;4aRLbL`Fv)gvDGooc(xDtCA>o-+H1* z<mL{Xz$ocw_wOiCcV3MP<)_fp&l-W@QZ}RIk{`#dKaSw)zlj+kz#j5pJCc(r=4qx; zo1e_5hc)_MHhy$|kZygtKqF@J)M7Bz5-}4$#Z@iAz@l_1cC5cq!HkzIe0#Qfr**BB zc4MKUB&3y^Paz%q>CqVsdtePMw_9i&e=+kd8!l#bQu*_xSW!^agCLQGOTr3&Wa7)d zlealun&I?9BX|0<;c`2)b%fj9gL**TW7dcTYkbG%KVVZy3^mTZU$ZxiFCXf&{C$kb zlS;wPX=kM8T6k{#-I#k^U>IykCeo+Bf{9!<n7S>g{bAORq^gfmLoFQshm-}Y#J--h znFEq6Cp7n4EcblE?CGAdwK5-0^Ie~}7}oqe+=!<^+jqMVICb@FRrh}64eYz5<}zrz zBd+jMLuGp^J1E3zKsq9N$;;h5K`63NuZ7fkYeH@u^3nJxjwkBHQFL<qLl_dN%o?Fv z>3nTu>kSPoDVQ~E5C3iP>uf{7^kCQXF3*E#xciV(_a~9XYnp7hrek-GYE-x~H*w^F zTyohOtObL$k?J_`KMGQc$xS!OesY2dB4TI*9A*E2J#ZM(`kYRW_I&ZDgK4A25>L2m zchnUw@r)PzsEbj7!EUN@+VH|^u8@_8LF(fOnx2X7sZGG%1K~25UHmDdByPHWw4)jb zH<4eI*Z7vt(X4dVTU07y&iuB;0D)}XVeJ02W#CRWUF2XzGU%SXOz{npdK1d6aY*nJ z&4L6sILs^uN3EyjsswA_-5)J%;ZRtpUqzKzvlV<q-M*6zBhBhwnQXbb=Z74qasPHg z`xk=t7pTVmHySc09SI4=?i8IKz$O=4saanoR75JjtYdi24GDFc(+eW{xwKwD!tXBD zZ2gOn6|PxsF3Wl7u-Rq7qnAYUB8a*qLF^ex3ue6JU|ghXAKAcU+}_Dqjebl=nBwIm zp+Ubk8lx<$W%9c=<m_q>3x?~Bgv3?2Z2r=(@wqL<rNwAH36X6eG8k^yyAfuX)jC;d zPFHByfQW-)_Q%RaoiFIyS!m>iN8=P{8_f%{J3Q0GeM$Fxb$=zN&RItfX|%}5g+<od zFZ-oHk!VokH4>K!L?z0D2SswXr?;_a6u)tPmuIj@@1{R-n6}qs>3JzoR5a_#jQ(X6 zCUInwW(WRyi!-aQ+GZAPKCsTUS<9ty!qt%KEa=_M{nTz|juWN{+<-aiop!!#J^T3n zMF6`ro_`Tql%YIzKgtnV_|VJn$DNB~CNt82v7Dh+07&TuFe#fpxJfqG%4%lz2xOR_ zTaPi+Zr)S(&SUxR+JG0B;?~RRMs665xulR|Ks2dlfsd)I?fxhUW(B|oopz?FEBvU8 zFMii)@tOB)zNj&5+DSejXIlmgF<XAf!M%FUSbQ~Ag+f{}iZ&{L@s~_p==ktY9p!=e zXS2!Q!k=*N5BX8av|VMq_X&S#R!^ZpYd+=MYK%D)X{mv3u0^<P1^Y2%opMYZ&H?_M zv4mpfH$}(}q_y0U`x4ovnD+hotc!uF#QLb?cyp08GZS1Cy`}^m6`T>`Mt{fu?qsZ8 za-71G2Zz??vYmo6O+EU`Uz6i(3x6;{OeWtFV3-+b$c=fA_b3$?Lf{`xQ>c+Ebv+3& z>_~!mhet_a+wZU_z#0NUCkDbU;-<Yo6e#5vr7*q4mf0%K1TsClLl;D&w3u%y@;&bj z{KSuSKrTd(QlrfA{`ZozXZi06LTD>cGh`D1e2>Z&k@rM15Xr_1nb|)VE1%_$=Uei< zU(=UgA_eca%8-wry&*TM)!7~(r@F(Zcsz6bI^$5}J$)lZgUeoZ@9ExyrxhP0zxf;| z^1yVe6n1(7WZIj>ow{COkRmNzGga0_7*V9l)+psVmsj9Vj>&j<-A4%f4i2I**mUiH zoPGF*yYhETdE+3tqyMps1or?l@1^+Bt<YRdI5Wx!d!hZSc=pEITbAGx^xchl&xfz# zID@`X=IerR!qYxUV~<(bA=|1PjX)|XnIf<oBim(DB8yDoQ$JwMGmjiJsJ%0&TPdfS zYp2tE!i=b-J-eK}YpY_ab146lsz^-i_^61drN8dXPd-dze~ekHS$e|dUl;Od>1h5l zf)y&9It<IA)81&6IL>{B&$_^IVTAq6)=8qpoATl9?c5CcZ5IxmJRB)r9a8yMaf)>% z`H5d;+de4S<0T4Phfs29P0wyo;N+^@@5CNsF}GT--`>PrpZZpM+i!T8FpTz9La;aW zcFP=&gwGnCSiLlI1h1Ep;oi%&->ncc*>Z=i87yuM#twQjl>;!!kt{_fDBB^A8}Ifw zr1jzNP4OG^9w+i%UblD+LOUO#7=dH78`xJK1{lBFO~A*w^=Qm04VyfUHHvhdp_l_h z#T3iSJ?v@AN{V0L!AV2c>-1Jsq<6UM0%tIrYLdlX%!}pi)hKD1gE#BL`38z%W@ZWQ z1k&7)Nk)G;bV7nKJZ^+KeDuRyWuW|k#yf)+Fv~00IT?#WWO*;Hm{9Ie@>Tp-GiRs( zJ%gB&Y}M8?oAWm!jZE9d8~Xwr`WP<YESx^^ovAeE%J<eAV#eQmj2>y=!y$aI^4-AF zX>WWeV5Mx8ZDw(vG8f}>6?K~8B0x6h8cd`g_ckU6$fI6Sc6N7zXMRtgNn=rv1DT9( zYP6It{kbFyZmW`wm%9xcquKY!Qp(NG?IAD4!X@PeMTi-_Ab!i^!1cl4_49~Yr|CG6 zB7<DsS0Hm4nX~=nj<ZSMD{TNKv+_%Ckhb4zprECUX!ifIDs9O3;nalTO)%p#4|*J; z2VdqrPo@Pkoo->*2Z>)V`IB+%PvoklN_R+O&q+N_wj#W7iF7Lo;9rz;1zK#3ie_C( z@o2>i-j0p(j2<=MM36Q*B~)QmlWL{~vjAxn^Fqh>H>uZO=a^`!I9%K9&igtUy%Ymm z)x`n@9}Gp~6T;mYe*o59X*!iIZPIKwM*FwRhncSTk0ab1#}Qy%Tq1_<{SniVbSve? za|ceZbIoU%6KU)hFv3ml5&Bo&*oTE;u2AiKD?SGjL+#o%*x7vAVmrk5^g$?$H_ZK> zf6*QrT9MQ2cw-#bNV~+J*~lltm+DDm|DqT0DV6TEq`Om8+wBuUQX_7#lUtSU;&K+5 z(ef&D_Un9oR_^zWcj$!e%Cl1&#9UZ;5u@oOy&M;SygC6UcI^+1;?D;mKK|nc+OuRS z$wy9G6Lb-})vo=I=Mk$koeq(v?Z9Q;CU3-Ir#{_l5a)zh$*y!EGDY;WOvjh3C?V8} z+u)%=gNx+#*-YAg%`JrJg&`?j8Ue)_9;Eq*@s(!Y&H&3O+K|c1kW9}NT`nP*@6#Jx zcD4fVXM0tSI4`F#`u)TpPwW-w_-MDM>vcYQ@_pUMK{>EJ8-dT8DWt%_J=AoHxT``S z0i^@REk30~l57!s2K5vOzr6gbK=)V9@5T(?rwlMh&6wIh1`u4+_6M+U=$WDh&|O;A z=j^l$;4muOtfe5POwXv)6Dl#=&J70+?ad^`fHpQzfDCWbajx_g{%zd*@<sLc_YFF7 zstajgWtsb9bx`WY(}l};0caf0QEz2){+kZwDo1i09ziY)nQqmlF;Q*5rgfq*6*YBG z47+;5*j|>b{DFDblWJcgmf_l`k5BLXct}lkVg~-IY$8Y&sqFX$lTq>QlJFX}x6qzk zchBMc(472@si`R`Dd}eb5aVJ}5Ev$XKuKvHP2ikj$*oh(%%+C?W>owVul>+Pk<ex1 z*Ld8PS)D^(Ie@vnE_<UnU{u-JpELrQ+YVCBs@|KMXFl2Z3CG7$ad<hLPlz4UO_!TC zruxw^w1)c?8kBG*R)6rfopnt#jUuJgPpottha6n@1PWL#+~U56-=#hNadphZypRVy zRW~*o84=b~a(%#vL(Ft{SIB+)gn=9<N!a>JzGi_g1CMUwotBI75?fAHh3HaM2jZ;Y zU}d&jo&^5f6?2`XvudDJ*F$-kSWmn++~y1pR<#<YJy~AskO3uc^ppUm#~0ImVx76K z*u&rTLLtyFG`#$0$Y(i)m6AQG3mF=xRPgWk8-x!e=z&r|sc`pI_Je2E=J5G)wP~OC zyiQ*aF|5k4EQJ&gD)1H+j$V~5_BD+}8V})w^>Al^48b>5y~=KpXWJ&{p|Mo9ed~+$ zr|1Z!x3?VldV~tFWU_AefCD=jx6;0QQejYU7A`Y)6zT0ce-TmUi<O}>I{Wr=a>0K; z%Gc+Mm5_z*(?>>5c^U<gJh+JKMOH!=nS)o)$^+Zz)LO1JqQ8BZ%)9p`iY9QBGCxDD zL(7HuWSpXgfRYDHI!|8`Pt-GFF=*hqAf-yPHZ@WP74gz*NR>aVfG`Pv3~#~{ge;q& zXDTL7crnE?Z}skfnuIdViXq|$&4=p-j)6*jE=JV#!(tQPEA4{qaI3N<sR4?S3ql$j zzB7K4eV`<Af}w1-J$ncKO1dxUaSS*7_?>x9dV%8;xI=NNJuu8gvIlWQ%{Qa(A055g zt=@^7ra?1j>%KPi%xUdC1>a>=gu`6N!H!>AhWRW}9AnR8g$3Y_L(8RIbS*=JsVp65 zI%=<OgSkI>SNH)caVizC=e1u}6_!kz^bfZ7C%c|&e|SnmFY5M_r|cyvQ<)z9S!B^` zHcr9JdmyRh2H?Hrc-7?WY9>0p1i$hYX#zuG-W-Gr?cbKntJR54%N_3dkL9TLG+;5g z+1_g#4qgZ5td+Kl_@1Y`9h@ks`Fdx$kxY5ga*1Ebwn7$&ZfoC*du;0W2Q+ES=8tU8 zM@+ky`LM@EKM!neVN2_k#G!9k6r;MZ2a8xE%q|nt%2WtHociv|U8~Mu83Mab9>V6M zXzM5$F_JnF@DY*+sM}d_6B~e;85x{Y>YWrja(GGy&copzy!UnN+JB)zya(Tu<HqwQ zJDDkIM!F@uG#V-DU3To$-ig?OOD)rU+)(qrney<s>K2l~A2<<->CZpks`jEj>SpcA zxr69b41BemLQG(K_EP}S^sRjCFu=JQ+xm#TNy}|k38s-2UYXQlFOWZ(5Ao!}88k4y z_4w%Pw{Iq%r&lXR*SE8q3yl%K03bEX6vgP)u!I6KzzBTsW8^FP>z$H4<KfciC<1>~ zgZ76EK?Bg~P@^n4)b3Onl$r=+-rC!<%%U*M_-Tktp!e$b=0hH>eD06Flf(UIt)Kl7 zLfqw&(KKzP&{I3|wOY((0~2Kx>jcfsmpuotSnCW`ShH5{?Jwc2)_0{2NpNTdM5%7$ z+=q@z*tGBVj@&$(-&=gX(Lb)%l-Aa-SukAuR%9Tdg=9|dW8WxAnCL+`jm=07!<Ptp zTOxA>L%}GgI6i#C&@fWcpNl^^6Kg((4JOqr4QI$zL*7Q{I;$F=hX0{GKvq-%YxdU{ zYw+$V7S<}ouEHhmO7YlvE;E802Qrt>zYQxg`TA)VOp3JxnO*?s{Ty;Bql1K!!PN9s zg>Z^Sqt`R?Aw1v)7X7B$i05+aZkS(6Sw*yv3xKkOn;u(38iP`sF2zh|ZiX!sYg9-P zP;=TwTc=*qlab@VCtsx0tN!uTPMBT2>z*m!DZQu@MQp*iLY|`$_&T$TEnhX2EVu=` z$}rpWbtL3=N?n_8^EKnExXTCIBFN_N97f52HFHZNsM&DUYN)er-C|3?>lZj7hWMbv zrCE0F#*!m2Sq|H3XXPGOOq4jBOUD~Rt-CGea4k=(P`>=9_DFi3naY~%Y1*5;+3)I3 z?S|(9wI%`deu)97R@Vl~gwfFZXJ&Khg6LhlkA1P;H-9sT{xbBKX@;^lThEQwsmG#& zWr_#(7ZlT`4cmchAD?v{3y#i$e%TUr%UPh%7xmI6^0Z2d+J7-l4*%nZ?DRf@UXn;I zC`#~ia3%%Zn>`Chw@Q8<M|G#@N-iie-8{+VY$Hb24V4mgunt>Hx0wIJp^=dfHHZi0 z*J4a!kVS*j7S6EL$n^Aw6i<F9;5$x2%#5{$B`Ekb$bj=z7)5zHe3_%`jM;(sc3d)1 zR0bEwcD<4gr%~}xp?Pd`FAT=rg^M_65O;3N8?!7_EiJNF&sfA<dH?!3*1F)AyZSXc zSf2pen!b!|WtoEqdlEKZ8<KJp5ebmSoTmu6GWN$6d^)6jVlp^$jC{N{dqROLeO^hj z;IYj^!DZMVr=2zSt)1yKtza^}>@Dw}SkIzDpWr&rlh360QkiMsNv9e1lj#m`fBn2- z2#w{o_2i5D`F@Zy2}C8+d*o)sYpsQZA9CpAhvUaOp^-SSWizw269(~-dO7OG69t*Z z>x*r-tmikq^l}-I2qQBhjR~HyoDa&R74P#7KTKp+g(S_PzTVOTgcC43`_S3N{(ibg zKcolZi!GM#FJrLhV+n1vovoGw;$0E%hJ@K2*(-&?*ZUn&`xCQ}ER*b#!JF`MbrrSl z@2$@<=X}0pz0q?4G80U*D#2r4^2KdGnV(0AEVN25oa2pueAW)&T?0z08Bw0+4p!r* z3w(Y+Ovdd}_O`k_XK<R~IQ4uWX!G>?a59RN9NLA53X2k9-}(y9DwnqjO=$8UZe3j9 z2)G4lwmqK7SJgvLH!vxwHQY0#+N#`Bn5wpA-xYTk_g%iFTeb5^!}ON@f|W?2;_f}2 z76)j6m~#hSq)vt3>(ZrMwYrv&QFE^JkAq@}MRikhC8(#xqkw&!1qz&LTo?2xIfMA= zfKS3&zEHJVg7EFYFp8?C9o~Q-y#Tl4#HxJ}%fjXI5gy%|p2fwhJHX{x=f|VFB`yqN zAXAxjw7EFM%wlP9=oQ#x$tld<^r<uq0@Ou0EX1L=n}daXM$Ir+OxRXB2t_NZ3!d(_ zS%SVB&b}kidoH)iHkf>kEo~wc^pSmdeb(ce;&Y|is8>t>M7ty+6iDmv`m!52?KDWN z$8#k6we!H9m*W@3kMg}Q878E`Q8JN`<g7xyp(~8ttoM0&Z@dG!jq|5lYH8Su?ZDwO zYe<smKK<_OWVy5DyS$5(c@dHK)m{!3yMFQrXdJXV%5SRCxxruvuc>(6BlBe$A3FQd z^<oY7Vbb3IS%A{p{qEfCGWVWTbnNZg3%L;!R`fz)!-6;NimfHL`E_%#F2nj0O}S&S zW81tph}*T)M|69X<*_&a6u6|kZPLth+74orW3`Ll3~rfX0z#&p|EXci0JX5pVbac* zJfV^l`19+^8NrdlV*O_>5{o9NR8Sb~4$B9GM4@gIf?k_W7$RWFD4z@Dc2u^l=G_~j zS^hgKU19GlEQ+$*)~nC7E@zy#OFf4e$!%81$+-J%CeYJr^-!D5?qfQUB$8EHTNFOs zY&INL<}vty-tq8dE^7ukm!KGqvYfWw!kG=5C}?MMql}!5K)@tw|ENH_<dNC>gD{`; ztcW3!9sQON4X=z0QO{ld>aQ*XoN4B8SrUq;J)H$v#HWj3u{m=4q12{$Ay-}tM54Lc z3x!w>e)zPMJ5${8!WC|*ZI7B>rVUUteBa_;5F6N_dNf_9KB67_BulGkM%3x-=lr%( zsmhrIMI(~W@Z)*B2qg6_Qv)f}9@)!z`;d2`IdsX63k{I8_08fVw3Cxy{zLRH&RONJ z{7lMza;i@)K>JdK%7!)8lf^b=(d+fIoPRh|?<Ge+_d8Z)Yo$Eh7|^eGN&w|T<3i}H zU9GJCRHU+nyt`EfjB4*RobJ}J%FnMn7?XgW6uxERQVBN=c#<;A4VmZ~$zm_%HjBj{ zQHe==+bHRQj+gH}6cURcN^SUb_a1Cou=*YqZ{<rxf<h|u2M)A)H9Ms*sgbZuf$eyB z!GUGmbT&Mit-E|)2x23ik-?QxO>aiZmQ-y6ojI|aJw0eURWB9?XlkU5tb~FLyKp!w z?376qV4PfvD7=AkOS^4+-P!A>e&r&z+GUg5Vw%>|q^B)*<X0M#yCaW(vL1dhTkdS_ z@`I^oWBA#&!zeBB6mA6{ifoBXtFY3%BtemPuhD*)IoBvPgcT9rFp=QDFVCt#ap<8w z1%oV>b##M}LFTRI*{C5SlLAU^{Rg1{P&w0FPah>3wRTiygp@E%i8*P$x-?K$7O}M| zAYP-u(P5+Dmy?Z~bz5RQ5*7X;?v34gw#{#f)AKc3nD+q6i1G?$-mczh@TnXC2`efn z&h`Ur!9kHnkwabQcTogDIoigDqTH)a9xTX%{LH7vW7M>eAo6(Tsum;Mlk2=KJk{h* zl*Lx}>r}-uJnJjmacXy<hM9$Brx{>R$-mrq!bRQgfpnaot+uTYv#3wLXD`*&Sf*+( z#rrQTYFPmwhY-NOMY6^5F(}!+&%h){Dv0&FF{ap2EOR^<Jy67@VeoO9C}W(v+!<M3 zD`vSK__FM>almMY%NlhcHCGX2axY?X%&mkSAH3`24nd9xJqSC^5bac0Z(g|KwhKz6 zv#vzc?V?zAF?BP^%JELpbb$iM=$)9BhS^lm5Tn%XyMonvNWPlt1cL=FqzDgBsD_t? zYl23=e6?wie3xc{&b_Rj@Mk0YR{OeA(%S=+H{Hj#o>_us+!4@nqPg|XA)5QWmZ#jF z*F%DiVTA$aosA6)>AOdTgpAe1t_^3)G$|7zjBsVOc*ZevaZ{agMu|OJlxW;>Y3g~h zm!Ye>o~L6mN<iU)w4volrMQXO;CN3*OP}hyIRXXHyIR*At(+?B?`r12#T;vDm}b)9 zb*k@qcO^#htcfmy)`1|^+&=ExQp^8xim!ap#CLOg?+|*jz1uuLKj}E9tgKw)WMFz| zD~@DjkB>vq>@cd&rIj&Nd5KRt@@Xx^@YTI<!R$k|r7t?-w?i%&>Bsc1E+n^yXu$}l zbF`P;$ynoFj;;7ext#WFQoOP}?qEvL5COH!kfHa^U8*6FADtoGs{tj}Dfz{o_j}@2 ze*wzj+QuEO5eZtJ-aUvn?~@R`|6mVZ7GRq1nOtGr>^cZ49AHGk*87B)&S{4%b?UqO z=%$ajPTl(ISD^05D64(?A;+aAQ8uBdy4a|HGA=%VC%k{lW>!Bx?DARF-mkl+-4QiE zKPPJt0vD@02RZy3L$3>G8?Yo|E_?orO&2ChU%>#^FlvXV(P34A9Qic&pZbQI3qTWI zzB`pl%UR=nsb(`(i%-q_MVSJaaq!|Xp_F)d^6Iywxf++PO4@3pgM(Fe?oqWPrGF7| zgVVkM#NtLgY`Txlxd1Ti_mT!<WpCrm7lANudKUF>D@pwaA$d*Uzw(@)|IJ^1!N>xB z{jV~tY$M_S-sy(<{2UH{t#?lP@4W04^Zn64nUKGj*S(PcAXERn_k8|of&B4ffB7ti z_P_qp!@p%`j*9+wlC~kve{#EN{y&+1P|d%QYCYfNImG5#*9LXNAX3B;Sl1mdb^Xc3 zu9QM<<y9r9U~11QsD!-{xzOZkfk{BkJEo4Mv@j`a|D%jE#{UrCoe(9yK!%f`l>^-c z&WS1CC6blZRcNY-tbwB=K#IZZIALfgb6vctUq@z{Mo!>|kB=|vh`!^pGadFecW5b5 zC<>0h7yJEtz^a-UDJcvsrXC+vfkG@}0>f&T2XI%D8_`o$vz;;v*!C`afIKbb81XN| z59k&DmreW;0x@1NX}_^gs*)hg0PrEZfu$M6*;!dB0oq$<XW!>Pg=aRE<WQxzxk<e} zea0px%)&Nv09$7)WhfWiUHA|^S9d9}90di1hI(Jz2P$%7UlZd_ddq|?hrlYWcTg9g z;E1Lbj1Qxb1rBJ){kCExV^?9iJ3#F^c&YKv?yRx?OUn>^f8t~(fPxtekekz0FV)V9 zWmxXu21y8}nY>%-i^=78-+CH;KR$T1r=!$kP|QpB$5SFEP$(gaQQUQ}t@eQY51fWx zXdwPPW|;|8;9sh%`#$DTqnFbzlr8(V{<_-HIa$J4>~d;n*0t2$fj@t|(BIG-=b_+} z2lydMQ&paP<>lq2=G($fd-K!)RzKLR@Na?w-0r_y4QK#CRoGAZDfMQKdZV-A$wV32 zlws0>X_*{7!Lp$~Ny3meP0PscY$@ws*F={@^e-G81-sRg{Q#oTs-xMD9T~R)r4n#V zTySKhkJ*syb>qh!F$x|7#yxK>vPW?3DO9783Yzlwjx>p8^Ep*k&z*;Yg8iLslw8`g z^!C76%O(s^)Op1v|1FT*CT_;vl}?XhL6I&VEq&g;b`Si|o`r6`_J?5I<YA46gvr9> z%8oAx9j_@F82sIjs55|qI9{SSF#y`gS4`~KZyuUI&i#?;nRbmA|MV`v(4_+RUJ>9D zJ^1KJR)6~FapMpGU~62>*8c>`;2yuXEPJfzL~Iqpsu})5Zz(pr!j`A%JVk)IhKc)B zwRh0pGW@lP7<4*^>nifLesD|BfcURG$J7I_*B=@^_dGA9MckO0qf)$fq>9B)Td=L3 zr`H1i7%K%1{@7}&&*w1<2{yv|sJnv+K^Fj@Q7266r)I7czySO5bK}EfR@N+2fTth= zCKw$Zb=X_*2AHlU=-LB->RupEU%%Yx095BbOqBC<<ET8ZVQc~zZSI{t?CAg;6ff@k zsQZpe=Wm?~0Dvg}F{oSZZUNqJ^JhYa^I0ssLTQ#`_6K$?2h@iWjksGROa62p`52yT zMams!0bsamt@Gtgmmu>cb#L-gRJ$$U7;e~p-pNf5)kZk&e2*yw_QS3z1}&>XCCt<i zvR~WKTYoC)nByV2D%D<M20j}tcwf7U!z(7eSle|i*%M+2<TKC}AI4?xDUKHH>5Dtn zI$~Z%u`Lza_f-eHapAw4`zI_*jfUgnYD#%7b!nXFN*2qV#s9Q3<Ug|}C8x7ij&j+a zrc;LT;KDHI5!Z(bxGJ);CYkcsz#t>pJ)hh5$bsn|W*qHeLqK=+#y~29mng*4bV<}2 znbI(vyZBI#*Xd&5p^EciTNAM=jm`MP%;Z>rOez$m7RO`$YBe?R>FM5(iA0(?G9j{M zRyEhh>U=e!BDp0j=>)slgBdJOZkeAwIlVj+@U7&w?m<&CNh5t-X3NkQS0Pt(1s~8! z2FHJWsHZ00y!i{_V>RTZRgFqcyDT`+*+lj8?VRTbu@5rYg5cxYO@7XB*qH?Eni@D* zy<XGIL78utoQC!+=m)eloVF-#9bbwvB?Yd+b{`5J@Y+d+bN^U^xs5xcZJekwV|J%4 z6KWWZ0r2tz$=7fzI>=Hv^!}K0uGi3Ja}85dNO8qm4J%`bd}8I+(j_N)$K9cA?~s5t z)F<~JG3ed-1`u%wbtio`q7&Ktwz))B&uzSR`#TqV9r3$r!=Rm5a>ux`nc~w74!0go zqY&+dput$8$Oczm65oKmvI7Kr`_gY4@K?3>No$~^ve<$NV@zis8{X(=AX<LOU{L^y zbvnV}q3r|db@BY}PuknnKq#Z0<}2KhQXT6s`hmKhB?9&$fT3!9x+_^-0?y;(n)#MV zafenCxSV+b6rU>ru>Cebl^P%qkQq9s<l%yw?=GfwO%WeWON<@3LaSnvmKN<1rq>^| zQ%(|M;izlNQ?w_=_UQpIbjlcd;pzxN`gS*m9(#^myD&-bLXc#~LPzpf<#wEe77I=x z7pIP(o<;64tDfY7Dc21B$jca=E3qSs$OdWW$iw^xSQn7xAn}m0k&*(=WxjR+4|Jhm zuMg1F3ZN-Z=G%~y(4J;`fR=M3i)_#2dV=nQ0MA`bwjb%9ub-TxGzVHIoH~dd60w!k zebE|wGxC)Gu0K`Ivdudx1W1eZA+4_sb=a2aH<Efja?Z(Jy%&$(=JMB`+q1nQQfXyM zR9IyyG98{#@*PUhi^B7FX%1>gkZ#y<2a415`yhQpK2zKT*}~`Jp%7tq;XlfYHMT0B zKZJZ^4%_ENuK8}Dhkwjn+Lx|Szlkk61<wvTHSb%i!1W4;_Urj+!mP$D-%~uqT0}C4 zM`l0D5Oux2b+RM<Nno|x6n&=jzQ%_oj{ClhKO2v2%X3~P51{9XoX8ykpzAbr{G=@1 z;m||Z)#!sWTyQBD5u=0|{HxDoI)47dPk(1RVP*jVQYez|DR|$+P4omkAA~HLHCH4& zq2<T397;LuJ<r-V+y$*3(PSCDO+mSjI2p2<im%LZwn!X3%T;w`j5&g0g|wMuJ}(2B zv_c=vAyj>Zi^JQSzh_1_mBQelYYTMJIm2V=UwjeJC)Zaqt}}q{lIk+=8jvbqOHm!r zs!qA4f_%f}iO~=k-`W~xZZX^zwzy9FsLuo8jzQU!gz{<PJ@VPD4Q#7%B*lSPdPn(F z?;7`hE1+pud6HuXZw<O4Cm~7v0@ZLOsXY)&R8vuCInRr)JQ2INlWnrPoD{5F7j(02 zlS7jK<`s4B>iaVWGdFJXS8WbgpH-ZalD_oIk-vt%Bl`k6o$@^i3YJMt`szZwNlOgy z6mqwFj+4R-ow_$?>2H@;oX33qOQnwqFK{MW0%R6jR~lowdj0eeOwFB?kdBjAs7PSc zeu%bkgFg1>j`x(yX81S|A!{!P_XzwsIr9K*`G<T$2Lb&n;djI0mVg}TPdgM)bBO16 zAjla?1`1WV;PxL@!I2?lK3tNL3`sRMU1q2w&d*~4TpxHN8Kl7oef9WVfM_WIJ<$&% zGHbv$D0|SM5c%xpVNMKcN8A+*uo;8H)cHxjZXVQ^#;}+1yK$P7bV`295dkwr|LM%^ z@wuOsHyw9)50li(2Yqq+E${s9B>nNVsi)l;%lXdb<22af!C}W6S9Dsf-%B6eO~1v@ zQ>jOCi|AsbW|A$e`nKO<oBnk|vLc2U;W@3n4r?(0@_?vA)Aon*^<yKh0+#Um+>6e) zw~5SGs_Yy-Nb}I*4lQ)AIZuvoJnRc-TS}pguh|rfG_*&)RC};}YMw4<94dd$aPWu1 zNrxmYzMGWTJw@JyvJaiZlC+}syTT?hdkG&tphU50^~r%>n-WAR-Em<ndnT75dQ*%p zvvGXh3t*rP5&Ctpxp$ql&==po`EwU1c-XbH=@rdy1~P!VI9nXh*4Q&{>84Ken1ZCZ zvx|lG+4!isnv%>rr>BC91f1d5_lTnaDu<#2Gxq?}YWKR|TY`CoEHWtNg_DX+v8^A< zz!~W_2ixBow>}YvF*Z4EmN$V1f4lh-L}QYibNh}&MZ!ax4hM#gB-WI{!<mtXQT)Ao zwCCoL8w-0JEv-ILtKL384JJ?f3iU5_d^=eQr6_!^Fnh2e7&bs=QZVM)u%^?Xg^dvD zk77+8a;QUL_G|wne9XS9oP@hAZge#j`qAF>w`c|2Foz5(I==Tm7mNvb&p&G%dq!l- z#<o0MH!Et76lN=F{vvhsM!-hnfdWrM=xvmNhHny)S^81syFw{9a@z2HxVz~v?4W5! zI^JognTKx+zpG=T(=)#G@Bv$4-51*J64S&ZY<-7_?T{R!FC%b`#Qn2{+eYo_5Mzfw zJ8Dw`3;?OsVp}yAn19dVn0A2@CorH5^C>_E8+erpJYTe@`?7$CTmHIGF}FMi+Yf~h zHjW4}>Z#2u^dDc=x{M|CESxijY@YIaZr^|RAUe2?u6Sj5Gl@g$;Y|iM_5=<$&RyO} zqs9n^9zg+HO|kypAsxeARxVdaf7%Zs&+fK)3Vw|{SPGG4?3rPw7evTgf<L?z!{Jq_ zFn{%yX;<^C@g<EH5#X&v|KY9Q`8UL;3|WBzz(D+MwcCvsR=0#66ME3$`(H=>caj^E z`h#iDpfJ%p{LFj9sy3ot<UK3^4_k#Q2T~tdqR0wfjwD1&xZm45j#RwlRGRWyW79%6 zex9~2W_EJLp|Tr%ewBg0A;LEp&?*TQ8L@dIsPIL<s}HVI>mKyrJ)%LHfk-_h&y_UI zkvf6#l0pi=2hwi;k?@!gJS<Ww`bXeuZD2sIJ$=u3HkLpB6o@GJ{tH0oFc#QiUvLXM zco-l<Z7jZ|`3?CiZSCco>+|@uin7Z*S#t9de@Ntv)1C1#jf<xFA!QI^zP)*@X4qPb zV2dz9F-Ab?gW!{!H<d^21y_iwQ))wQ9*w?+xZT<!QhFZt$qX%x(tY(*YDbEVS6#6o zL<w)ke*T0Q10sC(zeG;^pILCsgu#+U99)p00XkVV#1u6@aItW5v(e+obFLXh4}db= z4&RuUj`fJ`pN%D4TGUBFSEb^B&6^%FeH{zLWZ<<K{ztC|CJ2+^TrmN9zIQ1JJVO9& z1KS5!D1afn@bNcYV2N9$ncrf=IXtEBIjhs-_#M!KJUNF0os!Upe<H~>pavYO+&?yd z@&p+8v!+c08O{--cG@HG@GS7NFJNnY=u_Yc6xFW&7}eiR-tB*gYX5JA4A@sU#0T@E z4(1T<WYWhNOPN2`V4y_v`#!hYA8Xms1O~qTF>x1GLZoD|6kskjfKgxp$&9A<fQjmW zWFmh{_NRpJdpH`{h8uZs?;*Lenv%SQMUHvszGw;q&<*`ucSiF^ZUeW_5z;h<{}jfh z_G-T8Y+6-pA{qBb#e&|SmoF3tR{ih7EHL4J*ziB^lFz?!?MFh(5Me9sV$zlW?8H|^ z#pZ-MTN%b{3UoZbud-m6C|V4Tpc7$1|Jt-U_{v|Dgjiw#s9!M%0X3CPnpdx0weL!L zIsko6KeBT@ZCBX-s3{V7$G_FI?0Be5^PBSDga7{^)v&MqCHQFO5WX0CcE0lQk}vlP zZpJs3I)QBl7#1Q?Ks#9D-LT<$Tio9pV@!a|-IcTzUz3ylfF^D3`}p#(-KOC(s4H+| zrU9y*avS-YQ7>)?&VT<-QK-{E1T+=MeAG>x_XS8P8*@eMez3rYj|4_m%ZvyY)c-`) z20(Fi@00%$wg9j(z@l7BcV5v9c<Ort2pt4)b1b0CmeR!-{*APKcfc5dF!7g-eV&c# ztCKfOl8(!DbGF|m2F+%x#4&}h&O{x&36$|674tg715%4X9=)1n;Ov^Jbm#gw2?;nt z@iFuA-UP6cws0Mp;Ly-&R(Z%=?+u2BsWJRH%8wk@^Pka3y7)m2=&Sdp0;!6`jR6LS zIC4-a>5mV1eEUy<gW!EYoYMYd690Gn0o%IMv)0JW7jg9NJgpNR;9aFhGcYY>F6b+) zjlC42!r{EOA8O^&UilgwZAH7#fV}~<>qgw8ZYQOs4J^A!&Tlq_XER;R0MJ`y++GA2 zphED0)Cw>c9)PQ4t{hoyH%SVQ3z{LzTwwh5pRxY0WB`y508oHqs}1H4yaM3W|F)&T zhd|`~;~swy7VH}E&l`4!67ba?o0^swAO5nUA#eTJ;`TzZXKIgU*6DeJn!!k>vv{S| zA}s*?c)y^33g!pVhWW#mXT|QnE|DQeC7u9xDi5<h4ahPA;&aXngO#e54g$+aajT2P zzi&?5@L+--E&@uuH)qre0j(F5A4_pBh7TkO7hlaeSP<1>m!x(Ve2-JqCoF+q6Z`Mu z*B^JP=YP3Moe7VQj+cT8U)-nhdz-61yY`LzE6`k!++q)4b-c?!QxA=#-L2;0`Qbh% zixWBi`5!;b22v|-0Nla|sVkgB{#x@{;W@X^qfbPSiLNO!H-=FGLDJZGffg@b1;Ycl zHDVR`e;}8_eJ*^|muC*sVKb_zd*@Z#B85dL2{um$)7*`Vl^=)4Ib{;(>$9C)u|?|P zSGQ*E5JSFd-7fZ56VH;vr9bbGCVh2cA$8qdaERzQWs+Wmb@#=92z|_a4xaw;-v9nE zmOW$VGoV?!!@|Z!)dr8t0+17+*)9##?#9a=ucV~(QdwC@QUBG)CscR&<U$TLqZ2PW z$THXn+1O1-zL{_Qs-pgLbvnOC4Icgq#uInn3L$!C*n=;Aw|QV7sXCdHCx%E2b;?dF z-}k#<a-i^`BS`VYtGiy^c`NBsNvep^OnM}_y1%7w?fb8w9SIoJjL*zT!KlH-V13&> z>tV0d=XLnC2nyI2y34?VNl{Wa%#^YSyZWS%-IC**#}mnfUCDA|W&0mUCejT3D;z9I zlE}i&Dn)-7s(WK}nWhJm!b=A{9tC~<AFcB`Sg&CJ7sui4UDOF0b!e?4!KcoZmCN9| zs<SZU*v#ja2N1GAVNvR><zHg~h8i84YQmR$)E%Vco?XdyAxcVrB7jzjK@{Nn%Ru=x zUKB*4s$1>Z$+<*^^OmIEky0W2L68{=wZD%?$Rt28W{D_&n)W+8%rEOwnYP7iY_0`m zkqn?i+f<}bMp#pP8HV%5ei#e<|KRH_qvGnKG~wVH+}$NuA;H}P1c~784uus?kl-3T zXz(OxDBRsOxVyVMb9vwHZ+d3Bzgp{5?y7$|?6c2)WbZ2o{Ljxx2jZ2@pY+Zmlz><v z<ijmVZ{D|h3_U1JW%J*^Kp`u(|83i3e?WR5ldP0HbLq^z#jz=idZjlPn0n7>vYpSy zPvC_-(2+#N!%DP2T->>Jz7M9kVk%v3e44pz20=cZL?|t}mjp>1|DP%Gzn<SG#BExD zO|1ngbi4OSNtyP7h%FjTk<E)DjL=sM=hIuwGGh^_^4|7$zD4mTw;q(ytEZ=>hYLhS z>jYs=4Sv2MW+R?C_TWioxu*jyprQX<tW>}6bDa~=O#ejDijVtVrNM3JY5O`m{bV8i zoOOz}g|BXVilbc-A5mZh7(xTT=y@TLifUj2C^s+MtUtH;`uL`NVH|_;<(w<fp^Xk~ zal{w^;b%hC0B=AStrrs4;>N}&;I5#6!7Zd>V_QVEn7<+?;<AO8Oh0E9_e5KYx%R`q zGVz>rtcf6Wtqt!nvidXh!B7q9*vIx&?K%$ngj*9P5{oa4U-q9%LEEy((Wg2=#=1ho ziAi}@u_~hdkrG*Rb;P5ErYLR{#)|cE0^6vWP<kV0Y$s1_90zxY;_J7p5TL4*`+us+ z4HwN@?xVXz;N}4YmG)<&WpYBo7=XN`wmkAaSgfH0QXubZYC*v}>>pi(kBy4Bx2uro zEW(Zs@Y4f=j(;Qz<%imV-d^{EC+444@{s2Jtu4r4>SRxhpM&uf;I{yI9>8zcVWv>I z0+;z9`u7N+qiPM9$ieS;5=#tMO;Pt0w|vGpkp|#q<KO!1+JEqLst<dY+M|=wzXdei zV^RSlNC=QPI)DKdO45T`gcs-Oz|0R8^-{NCy<qw_MYi|$@yV5sY6d1vL2|nNOH&M7 zZ%}n=?0Nt?e5faj`!Va~PnYV!+lmrjkmFU>OeE^W6C6$Sbv3tJ&<~5yPCPm+stai1 z0GZy%N1~*Jy6*FR#~nKVam#(&2j#V$vyJmEMfAhp=d<lpC>)cMk?|6~idem)kuy5E zZRr)DH{2avy<PEoFSt;?^iP4ZO<uU@w5uz-k9yn=@CM{amOgPOqAqx6ip&{L?G5FJ zPR3_6CUbvRQm%zh>WjU!kR!x`XXOgkoD`(5D!ed{R@JB27HWOSy>cOjMl-hQ@Axkl zuKR8NyK3QoAB}T>naDDhY?Lft-nwbD8c>H?&%>DXsI(93HDCiBS3vi*t?}z^^MiE( zr0s<SXF`oHqavio{8`}pVm6U&-I|n+k=q8H+REbxZ=m&tq8<7(esn?2|6#bre9;rI zQHB-m5Dlh_Om{Zi1O>_6(G(2f<JZ{E+mHD+7T){voac+BTbx2ry)r#En?^E)Ld;9% zfJWaVhEN6u24jG_DtgqTjm+Yigm0U)nM$pzBGez*E(nfKjG#(2Fibs~_rs!<L(0}z zhS?D^1PQwBMguxbvG<HRufyyMlPT15pKFoz1T+@^wu2-)H;=4T0kb<s>vUAm$%OF& zLoj+qS97t5@$EJbgI&oeu>ApFaD~x2!|#vba|@ylfctc&+N7}L;gY8DlhH{bZ$Xx4 zbD6suz?}{VUE9;X<t9eqTZ{GOehoQhxr;52{{J9>5nOYS#0v}ninhBCL>tQ;L&Zvk z7=9@$!f0fK+!^~g_MIkv7Zk;UxB-I=sRAmM4~!OctKt*-2ldcJ_zn%2|6dT%<_&+p zGniKEcPMyd;KO^L={$;bn+-;8`ErgW>-w5l6jj9Ynp-V)J`OO<w*^E2(|y>&kEAt; zlrx2+0A)oBm}GFxG)E>x5u7{h5f?W$8?Z3zydC%byS48@P!K#&AhbOdI>bKq&7N=D zePuwkiM#{KMkWjaH%{U9@qBQ82SNk^?b?|cTDW$(C5myGZr@SFjacp^$y>WHH-NsM z&h#M&OG-(I1RB>@CwBiTXR3TB=y=Ey@FKVerY0FNK96y7GBBMAXq0G$yqVlL?CO78 zt*|~@xqWiU_f{2PF%N*Qbmrmn5QF+4U{VVIEu<urU|rYmL7E7IT)>*<v+UM-T;wgx z)alvi=M;XcUov|YtezJ~Hm-?lP-axOzZQtIZ&CoiMi;m+1D%8momMugLS9vcT~5|` zb<I!-CEX}4ht<TsF!|gIW`19w(C0NNCkbG7meuu!CE(Bsw{N;jNg1SfG=Q=L#Z*e# z<AMja(I^|LY@Tjw0h&z(t1Wi5{!CEsM5so0>aL>ZGf+yxMSOe#VO~IH1f)#>rRy0W z2a+8#9jQU*N$YU?_;9iWBr~_W3}QOsW~4NI1n~w`*xUMQ*9BC#-6>^XQ-le8nY2h= zomwRHGI>+xcu9<d*aTq4=a(GZ?h7CTKbn6ilow<_$jCq9Ehd0pb4A{HIa*cQ{6NfV z%JxS&vklJTVVLl>bmQ5m0pzJ&CZXih>%^8@0u#XW4G@Lp_K%1!27f>p<9FRd|NhqD zE8u|z<`u;8``n>NlL-U^y=FNo&uu%i7E}d@!_v~7^tk(Xm?Yr<o0$5A2&WzvaQ=4h zy|n@2#oUW1nhf^~(1J@4#c$+9Re!Y;IK1dZZMbGiCWLJcYhbyfz^dKYS^b%m&*;y& zGpN)ZIq*Rli@*(a^Nmj+*JcYZ%%0OH9w28105&Z_5265|IbeO^77Z{+AtAmdj#k$5 z%8^Q@ND;B180Nng^*F;r#3T)iK}#+-0c=wp?euL->ZPdRi6xP(yOjsCHTKJ40F(Va zr5Cwdt8$xvL<Bjb{<3rCm4?S=^6hWNBTEsIN1n(XvcrU1fE1U1_rB^8wr-DJ#t5}d zmx9-@oi378Y!}-oRs{HIYM^Lry&Qb5PA1~JFnp>jn~(z9nb<vvGf|R9*RLl7d~I@$ zy-v7p&Y91`FXU3D9Fh0QqmQ$?ev9+#!uWR3`H3m3C?{bWM`_0at$#|b10H*!Op2xZ zwQo{bP?@BbJE>?TP~Ekz{a;+vbXxn0k40m{ZX`s*p>SQY-(yga2cb%m#js<ys@CUP zs}n7GwB~G(OKDDwq!Lr5t+_Wt?CuPm7AK~MQer2SUcOIh*yF|`?Dxn?DPS6Kc$~TE zGdbE+brKqJZoZ<+bfX**$hh~-%(}<<AL{EAG53|hEbgR|?!+@!tjF#4(5Hcq2P#># zdo@`!6;_N30=_uQLYqk(h>oZIaf=Q@&ep>XH;-O(k*O)1LcP+Gvg6<uKmli5@9g^s zl;s>iV~~wzxnrEnp<Z-&do9XGkk_zxM7faT2~{_OlK2WVRsOxAR7UU+77ql5T~<2V zd3zY}7^~Q&K`Xb}4G-&I6A60fg~{Wc)=m=VV%J4(1Jl7k&g!u!p+8uBb%{2(LB~MH zSKA4;NxFA>4cIa`zVPn7&Ma3R3Q8D(4-v=;HHi&VCMZ4&Kq$s*8$Xprr*tT;$oWIJ zO6Jx8FyzLb3<c~8jCZd|tgX&Gl7oYc0mC4J{ZI;V(3jKrbPG}2vG|Ya!ZBI=T`Y0d zWi;OAvg!77e-Jv-1)qK?xZ)}ZU^*ou(W2GZ%^N9(CH>T#Orp%sn&FOe-X6)78)?2d z_iGA-J;s+6*lw&tBbcT5mbs3(J!G2;hlnW^F?DzfUR`4lEfm^(ZdHt%1L(BCtT*T% zb{f(t{42bI4L64UQ=ks$Zu)c9YIs`16?u{Hwh=QsWR&3^Jr>3vZ?wsx7h@74O&6ZA z<%-1G?~?x$k|WLkZw|pzk;pX9Kza{hkY5=4vBsqh3b$^DGD=&41akLs5bEYU?Ww(9 zVUO;nsFrW5L@7M@no5a_fI^Ggkpbn?(~U^EC`u3UOM1~TkOP$>))}_-SCiT2r8yZ( zPcM_+Kh;5z8B-RR=FbNZ8J~5#1MY>-U-JyRI<=ptZ0}gIL)wNaC?C^Ig~DTEYVJ>b zy}hCpW6B*iTu+Ay2@JwD{|8hH@^ifYKsSCDEptH`PPKRZ75h?ltT4rSX+Mj4E080u z!khfa`6+{vB+Px&FpuzhIE|bKQ!bjkvgH)$4oY>Gt%zJ!3;bsg(@y~C^#^5gG(Vsk z0n_&&n(Ujs`Epimic;l{V>lO-Ao%6Cw&9(fbimj@3P9pFTndoc`RhP62R#k~GUJS| z^CKeqALy+}+Hyn;nMm{{DepYaw?iRUspY4@L!nDRly2v=XwCk?z~sdIarmcXQW=y< zT$a|!cI`GUPVEJthJ9NH%*&ZXliiJHjTjjXT6DCJNP`*<jF`FI@5MhUp#yq^$?h3` zbqIMf-Jy}9x_vplY^rFoRDO1fLARL7%^<5m674{c7c8&!T>oCGS{B}(yYjb>S%CEc zyyuGyG)S2YRG$e&GhTF*ml6x~UbqQeelGfiU^`#eGxxi}`#q7cU)#@+W+t_I*f-vK z<ZvPAJ-fj8KBSZ9%InN!JG0%o?H15L=G*<@>040c(aie37B{4M22$Gb$LA1SyJ!b- zY-m`VoLu~@SErEp`4`n^W(&1X*-Tr&yj6iO#B>s`=;UGPU=XneKe=xSbHIlM$aBJ= zhur_H(g@=RZ=VPqH^9YVqM{#(&%*>pjvS1mzhR>f<y?>&ktPgYlsaJ@SzDz&CcFsp zJIc4Z5NxPA?^m733K4p*p~kK0pV~!8czwZIkPI8xh8_RWzilOK^E2@M?&ZmhPt`E? zQM;)Gl{urLs5+^gQ;6nq{@@$wJMs!9a;Ax=Ozi8d{V+qY;~Z;Jz~mHHojuE%v)x~H zysORZp`|sy8}a^mb9hAb;h~`jn>aKL$&;Gr2j_%j4N*w?$l6NfuX;p_VS@yYVbP}x zc$!=j(}4E2m5GT2?|1G`Yi8iov%}eQOG|6tHW7sRn(uRhsfG)6kY|q?qB-Xs->;-j z%M%Pj9UZhgdq)zzL-Sk=w)R$W?{KaykdyF;CR%`1Hgm)bDpCQK!@tX&=fcsN#5s(( z{CP44bRqUgjp+9r1G*7wrh9mzG~T&>CX8&kqe#*bI!QTC%vF_7z{seT<+H9CXm*7D zGRcwwrKBfqAq37X1e`=CxxxoR3+_WGAK*p0^|jKr<f@i41ULwc`<ShIdj};ItVzbl zx!~7Qh6vK{j>z==&h`9dIKe1uIp_UVr0$n|Em9F~VwrM~AKr*C#?qhAn@6|)8e}{i z<r_Md6jCDAv(Xcgk&t{du_D%5GyhYwOn2Nb@JwisfpeW7`LXzlT@;dR0*mN{fc~A0 zpf1z!<Ob0&R(CsoB$~FRTvNB;%v*H5sC1eev(8-7=3RYM{`@FBo8GQ|VySUzk|=>} zQUBYRg$G!+Cb6%Rf;TQDDQ#Y5R{q5g8I&q5w@aKG_;^O1o?BLTTtmAG38h9O28{vZ ze^47C2PR-CYx0XF=b8_Oj!1JDL>n}PK5j&6B@k6UO|~QHx-YwwBRz8-&~Pf`jb?>g zAFbZKckxZge3c2K@hR-6=!kMk$_R*oj7b_V-~T)<iu+4kX_&!%W!f8P>y3QB5RL<T z206keGC?M+nT4L{s3>ZSY0e}^%}gO{G7hPTWzQ-d*f4H&tC=xNt%^rKufj1qq*Zu< zZsZ$OTCBtwFb#ap*`6M!TmCXIr;c3yIQSUV)C9=4;+pR8zpSm}zGr^rTk<uydb{!k zTJcjW8hq9Yvwc|Zc>Cttio;<OzvE!vP;%L%??t6S*yifowyC-KdNQE@IVfkuiJ<V; zF)^{mAwbW<6;cNs1FsC<`Aa-10xSG56E^TJ0a!`@E;*@~x)le~HeGd?0tY!mL6=HL z12iMeqT;SxVm?Ncr<zY61hI^DhE55L(4R-@J{o`E0@9CvTj$0nT}%{wjU4zuzZOJv zoE&f&|523%2s#i_HaX}0T|KmsQUaswu~O_=(fj@oVrg$LVJn%ViT8An+!DOV$*`R+ z<cCsoyd<UP6F3-wa?w1ob-cc=3`QWiCL>bn?(Hqol~WP%hEyx2XJs`wYHeK&unfo# z^me8)8pfU@9{F8QKc9?nB|9ANNcu`(vl}!;2X&f8MtQIyq9jcPw2ia~e?3nw6Uk_} zZ>5^1Mi$P^SvPI+??Kk6+g?-q?sdo0DMF4Gq54Vp%~Bjgx-zqr#fZVg_%Oo>2S*DQ z{^z&anbC@~Z`hf;(i~bib`PEEGL!h`PeP0P%2`iz3wB3*Z%Khv(Wn2l`Sk0Twk1)H z1m*XHHI)x3Y1tsg8u5jdqw1D_0`LvdhVjmLpv;Am%bHuq+ek06Kkjv@wOE2Yb~5%3 z48M>DR=;PRh{@C#Ke2#_$T$V?iO@4O^<Td6UY^|G5D-xwoV>VI1y!k%0z|E7Un#LE z(Nr=Ldpv5LwGGmW?ew&SSOi0#&QQ^F-=`R)`W@X2bUr0fPEBBZrde?O`+VY`P%|_m zLbTz-Z&l=ADe3DA19Z&Lu`5qq{b+24HcIm1aj?FbV&Gm}qoG$$qtZkQ0p-5mIu4ej zqPwU@ze1U^#J{<^r!+R5c+ySl_BU};rg++Oc*cIdba<ket@Ah&%Ie|Z++EtZ7kkjS z&561$@QUm$(yWt9y~5~exszqkYWoHR9FWKF@aIJk5HFd5#rE%tiAF}E9QHghv_}1{ zgB}CDMUA%fWF{A_Sln6&n{xSSMf}GHahm{M7`nBO2eNEk_BrlRX`^Cm80W66TCYLC zI5U|Wn05$nP+rz#XJI51yBSl#^1Ls#9g-Nm{%i1{AfD+UI`Rl#w@kIYp;7vEr34Za z=DK{-8RYiN>JNd!Hh9wFq0D(NS(8iK`Or!pFW`cDAbq1t>lS6r{zbLY6(4Xx`9lr2 z73|YARzp|)4omG}Wh&IdG^m8^diVp<6-0DIJf3WbKpHaPg6&aJPVY=r)CXBY_&4Cz zW^Ukb-T7O}k(V<?Lt;mTYOy^@scVk1i*r8A_2n|uKPZj$qZ{qAv8kjqQ~J5V_G#;b z4J0IIYj5b%n;f;GoiY2@4KfoR3i6F?e#7h%Yn{dMOuzdu&ehd^p{!Y}>3*iWwLW*K zn<D4)_{i^fiu0$6tJ`VA1#sySM>%UZCu@8{Sy&u$m-cTXxBP8*GPWL4J~9_KoyO+H z3SOpTH?Wt9_#I(MVI^*@Axt6%?cQ1))5J6UpF$A^5oLSljys>5B5~@&Ey%pY@-6?$ zo}9N%0vQBoCu6bm1<}G?ASo7cCDT^3=|<aaN|9vU*pB>>Yh6k0=Xk`Vl=qmATnv;5 zx5Gp_T2NUTUMweU&4w%o@sQnb)GCqfNG9BqY;0`x1@EFS=&V@xvfZ9VMTjv}%8;h} zvau`7GTv`g645=}CucpuE@uibnuXX=^36VNoa4H0PM6sPu7dmnE~AESPS!`V0wk1` zBHQ2tF?R_rE$fW7*D8Sk#A~A><cJQRIYQw-6U&yea~1tb=+U<FZKG_d>uPt$U*ec- zhY32={6J5XHyxatZL!b&BzM6`N%U!-UP0jwFO9R5jd*TubZLyalusbW+QvrC>~oZ6 z7k2DoBQCqW+-ywcV=ldc5%{P3ZdhcrjJGitBDAjbIFPZjD?D$)1xhk!hM{tu>%us< zVC_C9cE&f{N1MA&x6nZO)z(gb2(q(tFj4w8RECIcSi9<awQ*r|Da_r@Ud86EsEx}+ z^zR==y%VMDEw_!|{y5D}M<4e;{md?czoyN>!p9BaKlJ%KUO759$B<;{!ub7rCNovU zwsU4&TzpLXxRZbQhM+g0_IoZp%ebV}48|js+vcCdS}N3=YPdZIL0!-V6XMXR@oRl) zsJJ?Xi6{6ls5SML6q`7(VKq)=u@MKEtE_lRV%6+WTd<N$M!IXQ*^OQ7;%d1i&ZD!D z4G>Z=YqgT#39c^DitTN!<8J@jcN@G^e>xexY+{<SX-)Vw`=z-KIrGmaid_h5+C=6g z@o`J>L28hbt91c+<0ncyQ192R?fxxlg91aRHy|S6iby8loB8GG01+PD`f+OXf@)2O zW(=fDOWAW1L2w!+<tlF*BavSPANY|Va!9cAyQ6z|aA<*$C9(at7?Wz2`aOGMtJa@? z)?tY;>l^K44O^+Y?#G;BNwXLkMBoA+#Cl8@=8qZ0ybzB4E>dfme+<OMh~AQkh{%Es zzz2VE&o*gW%-3^}8Kr>}?|VkSac)q8joy4WN6LSDXytv<X(3wHP@p%$!KUH^kxk>5 zwvI={{bjR%GDqrQM~r^vQ#ZOr%IAdZfYQ9ohmwvCmb|<W^~8*6+zd6_knhxD!A*B4 zCu??83h`e@&Dsoj=3GQzi(cSJbpDcyWsqc4R&{(97s}BC)+IC~ExEsODiKe^bX<6| zIs06T0XNSaO&cRQKUjynUY~v0z&icZb4j$d-E?$)p$`#=UenDy6|r@lt})Ng%Y(kX z6}7#bB=sd4!1SzD@4p-29GG(z$?UP|72uz~7;5r1e%{_k^6@)|`&PBDkzO<8xwYa_ zw;_5Ubc;>px7~2F6k>wBTu;;U9=fM<Z-oDRb@;xgwNCK(^gtk{6XECIx6FscJ!zu9 zeJR!##t8YIwnUoGpyWmmw;0Vgr`pXNZL0xS!*X>caTkBbBfQYhOiT+l=PqLF1Y&G< zyWSFQo&I(47UaFyqCf6x@t~BPuiCULxe+?IwDU-`sp8s6lRJ=o2`H|aWq5QhipqgX zV|;MamGDBuJjABz&Jm&e)ZX+u?}v3c`fp=!kpt%<1}xFW+GYD5tnRt8l6&`RdK#8Z zLPz5=?f(lXunc4GQL1C0G(Tw($xRTSJ5*$LG)((tB)#a)BDIQ$m?XZ|5os+l*J?qC zqi4s+tEw<5N^_QHBlaf=Ikze(Z(;-*?YfuzC45V+b_orB{QN0%r+M`GWS(zH*3GTd z+y7>4+hBFP3M;j!S0gS^>gMV~i>C%E->`*?6s;xT;eNkpvUao~D_<lo?mKzYXTjJx z#8iUdu$Vs(&uDlu1j*A|Ku15*%TjhU{U$Zds`AwIwRN8#t5AqQIp-l}|1DNQdAjg> zzKoD}c4r%DgZGqwbxx{ao?7-fL1Md8nz}fp&bki%5h}XHMvs?b1wF0RsW$Lr1Gkia zO{jRzEH9H!uU>w23tuNlvUxV~M2n<7PmXAiqi#k&(u#?cdGwpsF%;{pVk)(vsBV@o zkzis5K8Mo(9C4UH=uMio;}3T#%$o0|H<U}?QV3iI)I~l;7cYHM8jQKm$`RrmzJHB~ zVyK6PYe``zEvvybF7z#KHZ}b22Z@rk$FD^5XGM&!Dd|8FuEJXO^8L?Kga4~QTOk&u z{N7#V5HqLN=aY}vGefM(Z3kRG2F9XM6;UKf%T{zZr!7f-CHtU4Xu3@%L?Tau=Zytw z4SaglH|@A#F%caW>+bQ;@2E%#U24PluVu<Lo9<CgSlS&%((!BM<lT?fKI_)GebhBN z@mMH+h^krcSs5L5HefniCe{*gzYdI1Q`7qTZQSYM@3mjhZ$A6AtrDR%PdZ^P66%`c zQkXyo#fDxV3=~AZIg?SLX6?AIycKz40mo?FCGeRIwNNH6nd0JRcU_0Jn&)(fgF`>v z7=PE`Qiqz4>=-tNWVe?)2M1%lI#XW5l(+ai1J%#qCWIt2-@_qcQPj=UQ`ZKO@tSw@ z&BFqoD1$29@g4(JSRy@>gfmEe?q*?An0M?W%@1bjm-<OQM&{<~S0F*uQ?m@$1`9J* z@0qn<TYZ3UMnPdd5jIe0RK1mojgjmKJ0S%%c0PI4&wU_gb8ZxhAX6axDMfL^K_ud~ zCCe~Yd|Ff}Aa9tcG5JwP_qS-wLFOk@7kZjvdU(5NbFc=4sB7bsA;|rV{R}8vS^pzL z!=ewD3{p40B@tL8CYro!DG5J3TKIz$7;pY8HIL)JzR`%QwfHmCCMNct!W5_l!~tjz z6Zo#r>VpO=0otH>gm`OkFhd1{(mTbTqQ&UBdc=nozsu(zzv^YL-WL5Vk=iK{`a@ma zXFBzi6$M#()}$Jo<BfvrxhN((oC&eCwkO~t@w=V&3lX~+@eAL8Dn&-ctonXJmy~p{ zxh58OeV?JpfJ(&5(sNqY9QS@CkiOf?+VftY1f#O?2pO~aenEV<2xu0s=u*R;7pb2X zK%L#c$1<+*IX1#xA83yB30>OfSchC(5#_Or<<|79sD*NL2@owFkbm8|yCM>P<EjA| z65j1vD!vq4EQXGotEfuDq6w+t(C6!W%B4TGqjlwiF?XxhLA<&W$+_pP4RZ5QDbM#P zh*qcLZ>NGUj_59gqSl6^W0UQrxg+0M68-8LT!TKFg-HItuyip%V##EIJ2<vp(UPLH zr$6v;*`~%$^j=nbRzq6>TzT~ex`?E;#7kRl=%~w!J0h;-k<Th`ZfZ^qv}3~s1<ut* zrs-wf?B#u_hb%uMr)teQ+L*kp))LlN{S;ZI;u~@;dPB6$uA4M8_&Cq~G!#`K<*=Oi zx?IbCe4HD6cYsg<zad1Jm_W@fMA$)_yE8sq=dxL^+w*>qHYcY`60EpZ8@CQqqPJ-6 zUfam(d-v$K94|i5ua~r*5$Z97(4R{KU;vhCuY=<l$)h)*A_GoeRP;TK<_8gL!;`i> zIE-pn4g^G??<^r9`Wqh6f<G@@pl<pGg@Rr*FWxnwp|{)MbAsLOYAzB{DzR)PCoVHN zSkt@w7IxJ)B8dc(q6C#HUx&Zjh($d9wuRC#_P7V4KR#aVk*D9fGO@Ptcbx?@d5lMw z!!R~tIYjNh+26o6x$^f~*i&Qy7;?Xp4Ca?<L0H<J<w2#V7~VRblKU)#)f2}D$v{o9 z(n=*Wz}e72O`20l?i3ki)M=mWb?PA53R6j363pvzO;iP*JNU@=LzLf=1e9A@i53`G z?V2@kz2zZQy;`UWwzft!kNA8wu(r%TP1oI%F`ng&Nm0+S>AE0hYwGgJclpzA%yMhe zHc35N^9Y3+qW%?jrmwVYXq8=NXV($i@XUA#hCk;+<($u-I8o{oy0U%qTmM~MHvqk4 zaC>%L<S${yS(NwBM~oR%3ulLlSj9ierafQ@hk94ZDR6uyW~OttST?j2urmeHEF9GI zDSFRC;k?$^vfH&?HYCKMtJ7XSA53C{z}4_kZQS5hhb8yI05_2KPbj=zBPLf})1yO` z(-2w-(#~S7l)BG4EYLWE(hhS=07@?5LTrb9Zwfb_d&Y1SEr?RM3c7H7R+nt-@fk=X zAAt2u)I8ZR_OhvCDLAFt@j}4sYUr@4_Vg-pZ9hk)O4Q;`yk_#BXpX(!Xph3Ys^HjM zODuc8_B<==L3qL+?cN-9#r6r8r>U57>(L{tN)>E3-?Qv9eIG5-5Pe+v<TJg;NjT_r zzuGq#c<;-)fDP6`{d>I*@;#?8r(HMTCxBuRhx}BePC<!j`!(4edMC$N^ubUS0P_E? zZ_S$i5{v!GEkKlhl!HOPn>xDUVqzx8yG?$N|7p}~M(O1yOar1OMLR|MP?a9_eccJ` z&|TIaJ8USwxm=vhptz(Qopo@(Q7TVZ+E~v$$j6)v<<(*Yw{}X>^N3}epq1>`Dx%r$ zPo~6wRY%jJK791oT%mW7?G^eP?nuk|e2s(dgzaJY2ea^ZG;J;;_GRE?@9sOfNyA%b zw0_-<Y9)+b{g|d<`@WbC1_leNHmZTeIl+Mol5_r1t`9NVI`nsXvV&!Vw|e;ooa8V3 z!oM5?1P$@FLLt9S2`xY6B=YdxeN*NP`+N3U$DAFh*{tNGeWcI-?7OPQ_}*OyF`qh; zo#yUjH--<K4yxi@T|C<%#{<QJVY@DvxS%xHxUMkfx585e%C8re^FJ4n*vnxzcmmG3 z$}Tbb-5q%}%N3Vn`mdggPlCcPBvXikZb<;D9Q}>xjc@9I9Cvs+7d|b!lGvtb%eZPC zaD_A245kTpQT|w2S{;H4&>|o~p~mgl@Y6K?blHX`1c4?iro9oFxG-v@)WyEuop+L# zhQvzjv<H>6INEY)U_c-}yZ^)41giRhk96+G(MCmW_O%Y)Afb`Ni@&>CnkttC0j{)p zd-#0Q%H9A``*27*lcHIMgkjshm1=<MuRkc!ELwF=fDdN6Q8AjWm_57yLo_rR9_*pR z6^pq)@e2C=*pEdod^7QN89-ad0SkBF$i9tg5wuOhJti>6KyY+VUoG=DBA#F87Qen{ zzU%dcL}Ja5sv&69n3Qu9kVm!Van8q$yC$N9G>?vev(ft!f5r`kt=Xc$nQs1%CAy*0 za}h;){l>j*Wy$le>IZF>>Pa%c8{zevWHlffoliVG=H&kgNPh`Q-19sXfWQPGQS9SB zEOkAGo@Q##&LBuo_I0RTL(I{sGHc1vqA?a5!lI%*G>Vt@CoRcXa6>PMfwqZJ?Tev= zL3FF)ar;SSBi)zWw7Q!sd32>K2jr=gXD<{)LJh)L%h0aJ0wH9Z#;{6!g9w?e!L<wL zZE%H$xfQBMlhztdBEh$i_e#Sh0TSKGNxZrTjOeEN?oeU!?|-r#+KV>9nA)-=am!g> zB|k>DI&3fSuf2|AI%E;AEP?5Iyao7u6W?1)&DUH}|0QUV5NAaPX@0UGX1Kv^X<Yfn zR8@V~bED0<tUf?f9JmU6<B;~F)i9gh(9tvI-ZEMbaS+&zxc`%RufY6|!!2f<dRa1b zWXAu-Rc3d^x~0Dv_00ifrugwQKqDIq>mwqfH0=eyiCanxdO}$bo&pOc5`Gw>YGQo1 zGI;Q9{yBkamorJ7a7<t-I?`UU)s}xD7dBFOPH2#Q$k@otXt60jr7Xw_>XhfYI+G3; z6#2B~!2uT->6P8-HMY4h`3tS6JfYV&c6xt=x{?D0lcKq{NKRf|Nb@V131Z|hJ4{u` zJ7D*|NF-@&G3ymq&yjYu$D7r6j4^o07P!*UGzh)^nQ@A2gVI(N><qHm9`EOxa-@7z z&J_=s8k04>%0rb$H*F^WRMGeXs{bw4ju6q$Jh#(v)$AZfWXIopjm2d4H*@@TZ~0fL z8n~~jiGk`{8%T8kgx}M!y#_X}kjUWW5dX{E{c|rTTmW1iBs<w(0v1b~#q^4fwvJo< ziUd6vdF4$>KJ2fZVK9d@Fe<*1Rw{z@8jCJ{A=>iYtiV(-OI4wom~Pck0Nc5r2#6`Z zZ&IeV%3?gxjH>fKh2h?@`3!F`{*frHs$=6IQ0o5Kb2GN}@5Y{t={|WyEZu{kl5ql1 z4Mg(&t+mRdpmrq$*FV7bvvm=j&tOke{Mhs&sqYc&{90n;3p)(V;0(FekyNu|ma|tu zJUGHG7S-<Ry#S#`x^Se?sD3^+o_pDl|JW1)^yuN4*ij1R`@0g+g=L?|9Z~3_k+`Qt z&Xme~${9@DM!R6xK>sOU|F(RX2uMsDJ1s>Su&GKfw`|4YM?=*tAf%T6W4%;^@7@8w zI3jrx6XngT4&<SP*>N)M3@whDZP{P0tp0lS{h?VYrSR^m(=*DjY-E<!i)?D{%_FA6 z8(?@R4qGJ)0yr4?P`{h-EY0seP~yqY-h<#Unjp`m2Vqe1+tGU+x$m(H__VP$&LZ6p zdt?glWkT)hVt5aI9+sb}BFgT#K)tmfzi^~8@Xs@zKlLBtmeaoC()XwWdc{eBs!LRr zSd9gP(P`*>Foer5V&9ce)uJTW!b<(jQ@=rD-H|Ovrvt=qr*%%}6TVy6YIu?dEIPXb z_EsvfKRaoocfrwg0I-I&r*F}#C?iR)rYU8ERW7kxj(7qyp8Xp{_Xvm@h#{Qki=SEf zYF1CrhWEym|Fb-K`S(9d6b*j~9|eC2df=0voA<(ev+$DW_KEQFZ^a*rKDC?*J}*OZ zfnbx2w|=QXJ^*ohB}N9q7XPm66Ur47*3fgQ3uJ95?CMgm%%cZL+2!^Hh+;x>l#ZgM z`ou*Uw)Hs3u8{(DO{+(usp=0geB&Q!(d$`Z`c#er6X)Z<p83;Me47uy(LEBAXA!7( zqF1GZ7eY7wW(SXmuu<YZYWZ8n*bAy2JdpWiK`YOtb3Njl2@@Y-Q+W-|2-s#9TN{E; zx!n}#)&LWsQVf^s(80~SagmqXnn1g`U7k0o-1!H{N&oC&=br{rP;b4Spz4>kX73W8 zbXc9>N4$~-t2(kt5{73+CgITk+&(nKGNZaWJ^wbU|5nBUkew&qbcO)N`m=!O><A}n z*1p%(oZqo*^F92UzQX?gSnj1pUCs9+;hvrW{esJ*YgWIkdyn}!2&k7(bJU)P%UTW+ zgI(J9j@a^+W)M}duqnHPGk7&ljo9UmG5A<G=4#>RE^{4Y7sB0X7)qU+udnaozFSUI zD^*0t#TwptJ&u|YbJxOIL@5^>{n3aVUv$_ZXVISo2V&bpSr*KsF|LWwH_pVt7Qqjg z4q7aj>W1ZfeWG87Ixa`FK6Ax90GzCA6mCYXsw>X%=A(A;!_<71qYw?$Sf3u!*+=27 zXj8nxi(B&c{jffmP-h}5qAsyc3Z^k7oz!1~{oq>Jg?c*0-`b|GRaAjs#zHQ3K0dr+ zR_(B!kZ@*~cY@0d=Xd*VS6|_j<e=E~H&R!r$=VM-hzQ-f{)qdB5<ihRoBuc0&;S!? z+p#{^UUk(dAzC}*FcuI;X%|$p6yVy=I<<<Vw!|yt-zL5<{-uq4DnA1V*2n)`;itga zZ#mlp?)?{yO+i6U%658>ge2};l7IIWfy?zpc1@XqX19LLp{;|t`2vo*x<*yq0rUi~ zl<i2?lv%k@@u_7RJYJcS1Wb_872@ZoJ15MdCiTn8xI_*ishT$vv1ob<x~95zw7Fey z;A#`(XS&wm$l}*<tYi3EcUwy`P`0?*+8Ok5D|}zC=JO#7ktt1N^c>*357IF3BhB)@ zLRj)Vrk$ALLuTE0Rhd8eB_g?O9bh^-<gzGwx>%+`qM}c_dAG+_C2I*upI3ZMp>(W- z??_0}PHvNzsYE%6-5N~ZLm`F0-<^;)!~!vHl;UgmYCToM$K0>eWZ`#bu~Vs`q81R< z)r7p$H9DX=Q?q^(<ZQq2oWK^b%e4ijD&!y+Xxc4rkjvUC#B&q_$;ZwdS;|;;m@NP| z*W%WGL-niG%LP7;>PJyqZmIb>+=yKzq8X8?u!(4RHViA+aSCBytO?<nl9Y7Tn&S8% z03%@=id<-$*2`hpA;$}+j-I)(HJWITDSQ)mdAt!x5@|AhmV1%NJlG^adU58ih2e>o zj{DtDq5s870O2!?^%>j#dfdLjqu|xFd{#Y1QPNTyhT)ZV(C38(@JB$^0R`^A|5mnt zK~#)j*72DMAWZJDl1rDwJ<RX{gp5s^{H64W!fL$<y<>oDs#K%QjC5%^Q!}bWpoh>~ zdp`4KInaN7dF|VWjjc@+Qy08A&ePsL7Az^o=W5Ht%AZvuXOHs`Wj#HS_G7u6(&p9P zNWN=H9^w6+oz6{`=@SdTr|l0({20b{)>d{Mw~Esz-9kMlQw=>QZG{Gg?w#yrrX{t* zQXrqBH8XFw2gSyc#ID{HO&A|h6TWZ{t167OqbB5pL=9mru`?<mK1zVymQe#UZ*93* zAlH-`#`f%yes|o|MwWE=`lGyv$bv&R31kgv<rNM%31vzb@WTe9^Z~E?>j^7UrV(T~ zs*@bL7tByLH8odkFF(5=@&XtF8QofM{P#m8-@yJjSv3UcT<q{vfZ$|2lw_wQ)iSUH z`Me^d$ab`kw@g57zd)c4M=0lqy%AY&C%+ddn_p!*JrxTbUH5er(ANH}sQ{|t2h0oG z#R!29X;^>Sb#uD7ao$R80%=1bvn4H{<<60K>K!*A4j8;L(!pMiu;(QXxhSQBv0m_t zLmS=isC*=hkHG!JEAE0aw+ceWGOc6gBcv_3q?kJx(w>8_?l9EG&cnf2vcZsP`Fpb< z`qdMAAQ!0M<NvMVmD{Mq$X1Xzvo@OG?himaTX$5UK1v_tSicAFnJxbfWcs(>1TM6v ze=apQ72rAxKmGhKdITzhlbukHL4<=4%o>V^sfNS^45(&Jp~XV2^EJhi?=x*)UO{e) z-z(S4C?+<xkRe{yY%}@$XP_Sg5pl2`mtQ~a`}g*h*w;{;^_E$E$8_aDM(s^-q?8nW z&rFKu^QpsCl<=_cxBZ26TxIkWHc;9S5oK0v1?=`cAfkpwseI|O#@8gq%_`F4yMwYB z`$^eIHsflc`fbnAmY@|@)g6cPT4cB3mGlA<4`)&TV1M_nGa;am)11rXFGhG+PM$F_ z3w*x$>vvn`=fAf{LcA^rg|_(cku7_B5sX!!)KcOHUW-0_D*hnQqs2l}7H6c_gdN@( zAjx)&d%Cyp@MCh6!K{m~*^qee@UUaIk}x(ggVwC3C=VT=m}EaGk=@<B|G@5+>jM{f zU^}x)64k_1R`jjEr>kP2QL@qN0QV=8-66Sl&XSa;Y!coTb=vD|8k*C+<0TJ!wN^Al zJ#CuJwZ<gQ#-qC-op0~hN=jh((h^aNKHICa)-&ZQj$C2GkrZFbgQRl@b%RdpIG+b# z*<qQk&(Oo)djK8~{A8;(oVM97e}guLHF=^Y>;~HHML5MQ-%T0*mhkq%1G3rZ`cC%) z7oj-dfY=Nfu&k+}s-mTMJB5n=UI$$BglNkhXnb0Y%ee^P05i<qizDfZtNAd5H1(V= zJN@S%Y>%eYP|lzq8V!ucP3qqIR0GTQox`?s+<`}@80o*Cv{`Qu3z4vFkO^=0zC#c; z^->Yxf9@{61E`bE3)pMwzsV`~Hf`C2SV*F5F#xoFF$X|U5ed}^O&!J+6D>fMJmd*i zCX6)Ytep5c-r3SuRU)C3w(j%j2#9bmYCFKCy|8UB<OQ&F{#_KWIlV;CVn!`=q)lSv zB;&#BqeWj456R{`|EM!Yi?6B*UVOXqa{b&o+w#2U`mhF!YZPSAZ&j2r!#_Ji1M%<# z0Ss4GCO^Hb;`NbSyC6Suyw$6p#HqCePH4c(?_@qGkdT?IJ2qHgdv0gC3(<Kov2pY) zdR?o1)L}gLZZJVtt2iYh&3ff;EUiSwu3r~D5ab6&0a=Z#Ed&!poT7x4TOPY@-o5Xn zrxQU!6zJKUvZ*-=YO}3Y$Wm1P@KGn``**<#o?Nq)s#sOsdMCKgpWgyT8a$Auu69AI z;TLIjiC3D-qoLMpi+-2D>#lzMcn7?v#8acH(PWnDRl%i&@dwmq0axvT=hN*w{CdQ< zP}f#N%VZ~GAUpPm-P8a2Wy2}fa^m7mZCGB037ABrt+neD{yHf8gq+-~?d<Pk$~Pa8 z5}6tXd~5uR{w)r8&moqi@7;n-$=d68Kcc7Kz_C5KiyA;{^U$Bq{`w_LU#4o@wHj)B z5{Ta1aF^hGT><XjNv@yjr6?C)qDD6FmAwi^w|Qs^IpDx-aq^sWR6p3Xvj^7pmTKA8 z*9!Y48QDdD7w2nwVx@OU!H%ly;EoI8bSXsJe}u3snb=OTw{p~RBfbmfzle5KZ1XI7 z-D~+-b|!V4m}4ry?HXiGP!Kzg9V4i7iS`Qc!>xhX*m^`T@UFLijDL9JQT12GM85}0 zoLWH*sQ0hfo`<}tZS!=RX6b7VEaYQ}cmC9}iChJ=8BK?3R#9;HVp6|Njx|$~aj{8J zkM^vw6R0ESTzsATKGd=<s?91QNwm#t%2%G?_O{{+?`XD7*Q{Ah0-LsK1uN*l=84kj zwsl1<Jr6;LHCYHI@C@8P4y~<JK2(@dOCw{g@ZpT)q0MK(ptYZm7JWjl&U;}<RsdA> z`~ukjZ#HCQY_$E1mgiz*{dXtlgICe(mhtxAr0^78GH7CG#O*`EjKK`*cx0ah%K@TE zH1)yXByKeh`(Y~za<U~vi7ZED_)bZ=twabiHo3?)pY~YhchYmSk~<O`n?OT>Vx=XY z4cU}sDqX4#1LPvySbl-GQPWGFnD3Z>me1PW99?KVTvI-%zM_yYbhRUIe%R89jGP;r znTceXmg=H8Ik+9T+1r2s{w17<mW#;kHY%-NZIHZg`}NZ;ON*@3$I8yYDSz<kyEVNr z;|kQEyIBsul0`B2p%O$7Mo&XJITU#D51xq0D96ZN%U;h?Jl3~uhbIaO1AxneZGH~y zJ;GCHGsm~&Nmu|=x$#4#5vk0`Ps(0a{-)kdKSC2-bbuu~O-r<8vVXFsa0GbSj;CJ# z$27MmSuY%WjeAr?AAgQjr#jUfeD(2z@jp&K3O`T2{ctPTP`@cGTQBijzU*Pk75z2l zKxf=pfJ9-++|x?WbFhZ*BgvBQamsX_Q+!`iV;YCBha8HH-?dOk>aNyFtL3q*qz>K@ zZ*X>JdQA10d5RtbjD&to%Dyh;PIeO}+i^dujQqYr>&ZoI68lBS;5tWGPjx#jcu~u6 zP{OA9&;F}dUtYbFmiVM@@x3z+&4DAc=CRd9TVL<n^_H*CJG%;^46?4ZGa6Y7+upb4 zy-P0mQ#deRGtDsjk;3UqT|#ch8$GFoarjafmSc|;O<rGjW$f$)PPz^+>KlGf%V6jn zTiiALDjdkJ8ilKO#s`x(Hk-dW8A%Yc`6Yk3I^aqvX+EiqVFBQ?a+-DXAPlP?CLz7Y z1LI4Owc9(amDa|s=4cm21x2+fTm#Y-zoQ@M+IQ@hb`uv1P|Z57<@v*s9alFrm;5T0 zfrH8izh)8dRo^$lua!7pH3~QwZQeA&lFjg1!ojdRHagk{wh9|ISY(lRMeQkCQrflk z2bLI69<*L{3y;Ob-YFXQ4Y`SqZU$e^N4Ylv+{25J>c5z&x&heyf<k-%(J7buegn01 zv*83O>-&k<@x`iG<9x{TE&ZyP<u-u5uGbutzpITOwH?6DL!KMnFdw$mCP{<jjsx5p z^tx%mC}E*0G~SgFlLjB0P)kFCa&jvbBNLuZ#2&MlSjzBH<k0}VRKGRfhMMVj5#YT0 zq3P&zy2)qU1=*c@y?_>jl>|NjhF4=gUx?_A`Px`ob^>!=vE7cFTU&;Q?JI76mHd|T zH0M?Jg{U`DjE`Ee7{-!qEc-s8?(LJ(W8`Y<=9sKw6XT;*&eDKyPX4a1^(Rb&m)p+? z%R=|x0oND{7opQ{$dV9)VeC{+-7P|BScIIHT*w!(QbqKb$hBp^OC3OJ+2r%%G)RH` zsOfOL^ejsFn1xM7Z&=<!Y2fRMBX-v}atgeQqOca*iBw9e+V!{w_fUukuRK3vpua6t zwx>IcL!ZD&o#0OEQVac$r26E<nnlY)RP1RLK_HYja`a7yir*<q&{AE*h_W|ZK@G0* zc5sHE5S!ZrS*O#=a2q)scd%a>A-@Oiaz#8w!WF@~z6n!sx_X79iF@|z6z_Y?I|I_l z)@@eSp<EN_`{LVhuX9eGx1p(Z#Dg5eHm6#itKaQHHu05ed$27+Cyoq4D|R`pG!6J2 zkR%v7=N@lMeSoAmp(~wQ0na;)rSQHh)cUntRTWNVmjH1j4iQf4SH8snAM;OZHJi|> zOzvGUklTY}=g=%Ed4=mlL11c^-y}ui0o14nKC{}r67J`AK~wr&8TD8%l4MmrEZnfK zl1G8*mqWi8!(bV<(m-vk_ckh$?(1wYnt|*rIo~@*h0GI#kw0k{6E>Q;V!KX}&_R2$ zZHi62ZXu(A>dm^`>^!e|sfsz_`K&o;a&|H7`7Cdj-Qg;NktZx~Ri-Q1V?K_jnSGI% zBY(dCdBR#AH7ZDU#hG5TTQEqLkew(NfLB4XAUiKwST&#*jgiiLkNVO1<2Mx~dWV^$ z*D7-WH<+N#xS#KH)or(b>ZP^v=)+zHc>`E=Y5(UcXEvRsHg>&}lALed@41}vxu?SS zVN4M==Gmd=?j)I!jR`aeC=)J*U8f;GO2`3|^v=QF_#9Nm-;&vCmxz=!P%cepwEhTq zs~9X|nhyF6@UkYN?j_OGbKE`ehg+0~t0+ik|MC9ELh!-*$jMVkmZ6h1DxwrAyIx)J z$K{Mf$J4XVU|faT>#GLJP<rB$j-5X|oC2SPt_4N!4$vCU_Lo<0l9vs=7wYIimb63< zU1jjPcdV3kBf6$gLiB>L-+xuL#Ij$iz7!7ldQp(P{^z{>>&ITE2|jl=L;-b>PVvp* z@e+1N6%eO*&lF@(YF1&K7ekZ!u%|%^;SDQd%iUIjCIt>Z<^2vW9zNlKXtMv#?<;)t zGY9qPtu6KR21k@1HOKwtsd<@7k?MgkK=~%&X5Xw7UdL$x*w~)#G5y$&<b2>bVFHy4 zZFk)FE(Cs(;eVcV-(NH-D*ol+RBAAorQ3s2GvX3okloxR@YJN1ayi(}Z!4{=L-6KK z5o)ug&u`(-0o!fft;(rN16osFe2wS2nY(D-dNWwtx!Yw^O_4$&RZ8s-PJ}aatkYBZ z%c|;s?QFBF?NHiP@}0C7lt=Yvm))8*(Q!z*AB@->VdsRE%rertv(tUhNQif{?9kNe zLPW$zol8sn?yy2r>q?vIHml}uD}*$aM8a5yr<p}4y1W1jidkd)!RxtCi_x0xGxE>6 z$k90)CQbt58MArUCEK?riv{L|s@$q!UamDz$j{ER@$dw_B<zvK?XNZgIM9pC(Q%qz zZ_yFDZTSnxbcS03DQ3P@V^*z}sz0N8Ip?#M>woXQ@2aou*8Q4RrQuk$N9H8NJ@$6o zzYT>YI72EWB^B9HYr;S?)>md_a$#vAH5YH@T0k%UET0T1<*A1cHU8+`gpQ44(^cb? zoY=l1rmSXcHQ%pQ(Qq%+ywbYug|w?<9Sm(jaJu9`8^A)ftyn&{X20fM)&t}U%)A#$ z^&j;4@2&9f&G4fuQO<Hlr_~GF?ZKe1_{LAUS@>aD`zFRf$C24YGJ7pBL@xUa6S=~> zhNk*A>sakXS6VLQ-_yH&V0%pMB|z?v=1g;0U)ua0w*6#c#?TPGyu84x<M<2p&MMiu zii*Dv_T{Ft$$H{Gd%NxlJ^aP=jjY_}KtZN!05UnpNtu|R%nH`(`sVKiofARSh;X$} zx&dg(Wou~;BfY*ZyS`rNcdgmS9d^xPuk(EG&JA+PYw<pQ{JU20Ey*0vR6hXezu3a? z1n$(f@6Y7S44oIcxvAN}+}y?n9xAj^^l5WiTVIe4Q&XRxz?`T`2-+CY6Z3S4|4Dti zU&MQ1kTJ7I1aTz2fyyj=iEBi^HZQT~x24_N-nR|p5eF<p8?OS>xHBKFSf3szr`57Q zM-Cs^|2Wk}uSB=TU(*i7x(;Zk0jQ1jb(NTPed!73cqWHn{<zdcT2a5$8*kp9`-NE< z;(B_4jdI!0+WJs)ohHrT)C{<%kZT$3@bC!9=>>StZo#-Y{E+ej(mydv8;#CAUOQ?F zv_l&QQpR5Bp4~#_XCVI9sCq2*1nISZ9pMTGs=P^mbuN7CdFB}eOf}<<gmza1ti^z@ zYe{us-Jx}oBgtE2J>w|^j;7mxSA^ee@~r(<7a{S+s^zoJ09a!njm!B*t^Xq0sYn4~ zfvRc0R3x>o?)RhQ+Gpd!-1o(Ayt*6c^e`GUTL29oP*f#LIdOB3jGjMcZb_Uw0uI1C zkFr|^`?tCsC3nAj4~LNVo}xXm2=ALp*)jb*1T2kb)6)!M5i&}`&3vtWam)&6=-i}l zDMhi2Gv4qeu9z>HNkbQ~ZBAE-(+<!Y=G#6}bwYq5{^Wly;>Y^o?4q|lUNzVtU~Hci zw3Q-^&`f(i9_Qt`fV$k$oC|~UOz8>>L=ghRn3-WG1qFo=(4Q8seS_>x%DzZ6Ec(al zKpgkoi1tWHx@wu}-@<iu<&Qse($UeasXSe%fPpu^rwpk}_b{xKJwM&M9}Yn}S!nda zd?9z~ZZTeK<eYphSd7iWVe5>Roc#XK3v{r90?7T=GgO>ve-v<)->Kz^gv7=kz41Hk zcR0E1bBIgKh_K<4?6N-fmBvFHB*?SAuB``TwR&el4m2vj9#wwk-&V6sO=SCxlU-u@ zCpY2A=_yBCMq1&f9K+IhbWay!xD0iPx|XlzR(E%I*Z$^+!EH&ao@YPL=wwL;8kWyG z8VAuqPpUIvTw*-S$PdL?j%LBHqe4cYR7Cx9>4nm(*XRl~^}4;oeH}enzQNPW-lUX1 zcLLEq4?op8o4aF&7y{Y8A1%d1cK2ifb^Z>;Cq>ood!^(-rxA}ocdb5s`W&{eR?o26 z9&6_g4<PvdOE|c#w_{6nHw&}*S`!%R{i2YMXdj;n57KH?2V|TB8oMSDE*d<st!e{H zE8P53h48O8xg&XU^UAO=YJ4<e!Q9}sYM+TOTXC{V9HLnxqTn(K*wTX|7eFMM7O@aH zU4mD;^_l&{PsZwT{>4kFnP1Se_>s4TD~3yv`pk>l2z>ZjlOdAZW+bO!HgedMO{E6o z6)ao9Ky8nIaVFmEKG=71-ilnkLX@?5m{yJ@=9<Jl$D$Zb5p&8QJ25OT=}J-CCp{fv zwdshwnPj}+&|IQ7-NyaEq2@`$j_35DzgmyE+_(;pFK1pe{bF+ck3Wrzwa=gg-hP3F z2SIKntz{J-b%_vm4M{0r1l1PJK`}R|#^+oFIx-r8&BmAoIe#}Se;t!tqn`{*{X(hy z=c42XB0K9}0boK6DR6L2E@sx;dC1LXl_>#@{xS5Bw@05xlm_Hnr@NLIeh)X^#B}XU z8<HKNA}J_I7@WDJFD)&z{=W!&tDrc;Zd*4v3GNUa0tAN;+=6S+V2!)GyEGcy-2w?A zxVyW%H*P^2cR64FwRW9-s@AGQRo_8z(XY)hpD||5w*FR+qEOGSHRWGVSaeUtW}G&+ zXWC&#h7cLdH@eL}*AaC&?gC$|d!XZ41ZlTJ`}%`qAwaGUPEObU{vMtIpd`PIp#R9k z(zUUHIXuGH$7yM<(V&TTLH4f&AoO&{;m9a65y|U$Pe?{4DPx&#Q!ELU`KIejO;un7 zuQPR_MDJkaH@&^P%edCM=@V_NlM+#RhV1jS<mhwL_ZIRNg<zYrTzT@@+()#!5s@>K zxtewNsKXnD++ufgTTz(csOKt*$<VZ2DE!r&H+*)OY$T;L!fL|O&JyovHfh#{UbuxQ z_L!J32_~7V(IA9~h$uN4=k0|}z2ONItN6D-SR|=@ui**dgmHJpf=tb)gd7eLLi+E? zW=>8j*qlm<R<OD8M8vXR5ddinHRBp$M}sdf$6?g=?MK^D+Q6p&aEb(`h8>G|T@y*U zXVL3s!1u^87mVi0*2;VYJb!M*n{9;|*Uw9bxit(XtqUcXPuVG<Wd+f|5!ZXQRrpHp zO&g>wx>9=j?K9ef@wM_9u|>*^LJN%Ns+I+@l`aA3H2DK-iXbdPI>EfmWwWOn;SOBe zb^UvB&UJ~EN|+jynxlH<7b7yv=GaEt!B4-aYES>X@%?%I6KMSPJ~ZgE4Pr2lCejwV z0b~LL8~GZ%SgH9Fn~_L~_W1o<Ldnc{2v#JZL0X30sLHzow_e3FEEwb1IE~jAP41f6 zqjn%p;s0to4I{+>orxl!`x1Z;ah>hTOPE&1JmTKp$Wo1?8C+N737vqQv~JoZEvPy$ z&QxswLQM-R2J&%?Iz2+->8JGGc*-^=jj^Po&FZH2*+1hB5^lyPC8;)anMF^vR#^)y zmpt%G*cNrdXeWOiE1SAw${msZgkOzy%FQFhPN{Bf*RbMU9<~A%Fs6mi*`fD2RX(V= zQ5oHT%Z%Z}@`XO;Iju-2w9o%BhCR2Dm12Zo9~wMuVjz%uFT>K`s&5U`m=fuzJJ^tn zmMb$N)|@0*iw|Z+K?0hDDSdbbUK-eZC$>lrzHdqSND#&A*8;oNnEavQI!a%UVx$uB z?Qa3}!r@EbmCiJx5eB)9faM3x3#OL%)x^-zQ(V-qTDdGQ(|o~^9WM*m8}ef<N`jg( zkakoip~Vgf_4X3uc@{f5xiJKhFcJBRd(+rn!4^iel&WVW$J&en?FABJzV8ptY;_wK z!n<F-A4pa*s{3Q46qa(L&TaUOFgx1v3v$<-$ebsr<FAl1H>!J+`Qk{v_Q)#!YmxU} zG?TIoa)ms@>dMi%_jmGgZ~DtB_1>EBPdx0C-#wa<3rGuS?DZzQlv0wI{xZ6^2s&TF zknsmv?45P?29xYD=`#y$1`dw)l-WSN50Jb6`QU8>9FX7c$UhJRKra9YLyxxpvBi?y zXMsHVMeJY$5CLq%9U>kFOs@+miv{tGYusK{(Ua0r%J1JUz0w25G4Kq&fKO{X-r9)x zBEBiI*uG{a6C#qTk2!j;&)y<&L}X_hjvxqPH2HZtudP4|(mao$MqR}r3yCXF?rcm~ z{)U>AH>y*c&$*=+eolofcNilX$}Z!*uyM*Uxc_xRF(0B61#67nU(|n=_Ck51tP`F) zGpyF5#JsgarDREuH6n8gH`KaR5#p6gtl2I6@~G8vCB0^E<VV8X6aDe<vboxnKW-AA ze$v&>Qvn}pQQVpx2n&T~5lb2R#~imb=H_itSPOSjTn6rTYT#cn^xKvgC@ue;^8|>& zot`LwQ?ANDtJkDk%AG?~r=Cb-w$*B9z1qo}{`RhB=7Vp~$W`r0)!c^U2?8q5qx;j; zs9j$za8>`W?~=TP|NH>()drWi#T5Tq^v;bY3|!<0l7D_o3hv2`dhh$wf?1sprxFd+ zdtuIgB)1u2>}*SPr`tjFLiT{hF<_PosTL+?e)E>uvJHFH3uC{eT_mFE%?L&g4E|oW zXLs;)yC@9*BZ?lGv0oqfM<9tKCGRKF)@O8jmLzw}gG2xvZ2O6L%G=xwnYi<&@lG0x z$!`=U$6|I@GG>%2u6uJ5qwZ1Ly85<vE!rm~%Y}PI02G^VS)vI)&Llr1^`^F&`FKJ? zRQt10Mh}*KxVV!P$)W8W$j@sP?ie0b#brV6w$bqf%gFcoW2fV7U75V<ydm)495v$v zF+JBt4mb3yE4nCP*qIB%io#4-rJKpk__0x|1ihA#PQ_Mw_w5i+FIE>%t62(N`YDU4 z@k!l!ezDm#Y&ipHqfs{Tng>c1GnTYPv}7x5t@vj94L{{iYeQ%Eg;Jy^9)M$1E`1#k zGzQ&Yu$Y~gK6tN0bpm9wGvf)y#AVah^Ve)w8%E!=k#?7>=zE`6vleOF>u0zBwbp0& zXG)+y0D2sD$~sJiDAlrZM1X&TE&6t=%gTTN3Z`stFz*8=1p1h`&;fMOYDygmfzKmt zGu9F|;^XnzukvL##=Td7lJ9dTa*d^!Nec>&`Dj8s6Sqth3t2;j*fhc(%?4-&owfd) zLvV!2TuFqP5zl=syV0o?;)roCsO>;rH6|6&jrzr@Xyx8|Trwv-PltD#hw4Iu0jQNz z7c8Z_h<03h;x^Vi%aWN0U*KC>%kXr;^O_8p^MpZZ_1+&FlJYDY6-t#NADrVCiK2m4 z(l-qC@VX>lH+a>^@}f71yRJnBajs52<N3+k7FSWOZznN=HOrCjAdz;BdpecY*L}a| z_EV-LT^6;c?CiC;ClMSwX2OR&nf)-x;p=?t)BtQRJ@qO4$B=s1)0)jURNR7T5JO5f z200Rb5Ib}~b)Tk;ZlCri1j>nP?$ZE^s71~UYROo}iv`fXGA36;LsyJagu1Irh0H$D zEOH*;sD+<@=muN_KlgmfUc~cOrm8I0Mb2{)QCNUu8{hiSu#7ZAD-9nA4kS;8R|RlV zHiO9(cNQZbACdyM7N#|u5A}+WIG;`%EaBVst1al|@?~RgP@h%u^E;nMhP%i|JRXuO zOW|u#?}2{?$3AnRz`3idD4VTu1hLbqjW+wUFJb>*T@*MW|J!8x|DtLQA3<mTNZh%^ zp!e3ikW0JLa+}sOx42$}NvsC6UAR7g<%w|ZRqq6+7;o{__I{)IVxq#t9+W9O`U<d% zR=b<nfXlXV!3PJctYM0OD!RB&X6M$7@+UR=JUvRk8EIwn=-_8s>M}aViJKm;bY!?t zTIP?Ocs~pqxhyCGcHK<+LkYziJ2wyREoIijFffL{+*Td0@UnB$J|0DnqFP^<z{;Gk z|8=P~v|HNt7f}*N2Ksu@Svf9+ZnWJz^U-cCBHpR8g)V(4mDHUypBItUt6PDauDX1R zFWR-tgu>xbp*jacLA%e?x~UlczTf6UtpsK3E-f}c6@Y70*#TBL+{ns=7tzIENHpYk z@c|2ChlzQ~%<Z>&mH-Xgg*mc)@?br9C@axpz-%|KkAItZx|<##LY+d>Eck1Vv}}^` ztEtMK8NN9elR5C)r@%hB%=SLQ2Fxx#ti3LI5_TifaQ~RG^1A((tw#*Sl|n&N;Rdyl zaim!Cyrua7Z8VKtaPE7#pR;(0C5olZ;5nAZ9a|1J3`8K3uLFdJC@wibdF)qfqtYIa z9Pi~7#p8LQ?c(MpShq)w`?<D`lAZ={irsN=bTo8nRM01v6<W&!Wxt`K<$?*3_z6qo zyVbHtYCJeNEaDf)wMLlx#a_Py5}R3kYxz3P%9OGWrU4el|JV!tzfKMj9g`23d3ax~ zG_o4s>ws?1{V>3d;a>>?ffD>fy~IenQf*}(4Uxb3(>#kyWXLL5LY}4BAhU|<4-G0O z7%u^_3;)ueAFv#ZdAL}4&Istl^0Mp+d4;pbk}g<JP?mDxPN8igSF}X0xGpH!4EH^e zb^S_>u#`8DQPMVKo}wiP)W<bgc3@=|RRPJOD`@>3tyUl-VY~Z*+$a0>9;6i=@B87_ z=#@2v69k^BGZ69T!dJ)t@T^W=X5hBOZ0gS;-RXUdi>gw4%+ciK*mCf`f4`s0cg+l2 zqs$rI(f$1tHiI`W{w+Z#&pg~f6lpo{KB6J4u^m51DgzVbkAU^HVTw%?9IutBUO}SL zKuBBbJ<~?J^1ba%ur~r?daNo+B5Zp3d{(mW$)aQmI<7S6hR?fqiyIT<*BCWk<p!~A zjX|kfr>H*>V)>ShT{7EdVee~{1)L*qGmel-02GrTtwXI!2(#X<NYCT3%+nEVxShLz zk!VTKQf2L_&)GcHtdAiJf9JBq`K8F_<64DJp7hPuE)s2OiQ;y73OQi<Oj`m!>%+ky zHu20&E(R+Ky<+=c5WJ$j?%KD9Rh>J&&WMk~0g}KGVAZ<ygWdvi6BD%4OTxna;1DhA z?m;^^L~7$XR;X{a@=;UsDrVbft22HBO5T1e_H^S8H8L@|I$*z9rTJ-kx5U`dH89W- z?CO9F7UL>3H<4La1?dHD>u2Lu{&PP4_WvK*&;NrIE2;1yFIQYt1igDC;sRKb%y6ad zoljI1tYW2j45KvFtnZQ9!z`%=LR3MoQ$JV$_aUI0f5)RJO9ZS#f$RbU=3OcrnlYEj zbTm38f?-oy+LO2OKhj=-OckN?)rA~Ze9swm7aq`*nLYk(c#2G7;{iYNR?CI%Vd$S) zfx(_*qYVP~w;6n-CygsEx)7;Po+))s{QjL?6cP<>L#vJeC6ARyWOGFCsK#R{BBGC; z5qf0WQFZd&vzeGpnz?j*)31jV_j>>2o$Q&ehp)QgpIP@zbplPgdetJ$bvgoCQ;fc} zdGAJ$FM4W;g7sz;{EKhJ`7h1bbFETv*Q{QkND5~?DQ~~aY+A_<cYn0w=)qTs6N8@t zJ*%<?^A}8i!We3ua+m_XTdoPNMX-L4&G#);gRCIR`jB;zl|hk7wFVX?+1i;1G6O|& z#H5G)0Dz;mm=kJz($3^;gmp|LR#zt~6uM}oYvwOO=v}r)GOp%BZFyCFlS$s>P_cwc zeT%9?{@wONe>Aj@jTbt*r4g=;wx9$_?by9tjQ~8Gr(F~STAX#)Kz46m+oQ6_<*Km* zFss|{ULclBh>@PYCl(k#{U1g1M*_Adl2F(}=o&h>szIaljkVR$*^K_n`+|a-{pHW- zBsJBSXLSDO4R^7x_Y+DW^_MHc)zF*{4<g~a)ZC5HCKRAy?BuJU*A4;i1Jtm`zm@Z! z)${cmARZV!0kvGkf7Nn7&DjPyDPrh(#&dJmTQ;8AeBH?f8R<_XZAzN@N1IRdh18Tp zCIU}b7k=w==_wnW(fm*w+^9vMye3dyOpWZ%2Q97^XzJ+-xfo2+!D3ES6Sr-zemr3P zVy8%J$E|}Gu&0&L*Em4Q!abqp2Y%;FpLMyt#mAZ!Szz7wwEmV*N72W;e_{V&vf+)N zQm@1-x)iaj{Jl{Z(MPN|lYC+L`u|>^Ks}Y-M^7|Jj?XsO`U7Pa@UK-}6w+w3NkmrO zOyN7ySY(o22)EKy*__~`UI_k`y_Ov#T7-!42idlU;>Rwn5`}Y%50lyO!(`(^X&m-9 zzc+%i^)eDVFFvz#v#_ukgSxS%e6*J_$bBF~-uAzM_9wi2S;1G}8cT`&;PYc;D5pvR zCRBI(q{U7S{)+<I51Ty9y83r)b@YBwDz?R#8s*3Sf6+Y}jQl)O1b^44Q59}djjQ|9 zzfM4h`Uh*jUM!#>a<kA~v?t<62$xNex0zIc>TL@+sM(fVlz5f?dFyzzE{GCP9krbj zN6gW99ucHdtqywpBD{&(OLf@MAq|{$j3O;=`*sTH6~RSg)UF`GCxytGx!0tCp%t@Z zGSmy9V-DTK#KAGvZfpDY1{T5(0)NV^I{~^16WfbeqarNtETV5Cip%WrsYDgvX5?<( z>}<sj+4K|W`=S+8X`Dg<e=u*;{oM=3E8go<V(w;34ME}9HWX;n->t4dxApyB-kk=o zf`>i_3}=9z9_qMvv#KxNo#fE%=KSx;8`qTUS%F7*-o#`9+E^dyPf|5+nh=Mp?~Lxh zi6HofjXiK6AaY}tky=o8KYvf)H{Q%}9}e0BhaJ!I#Peqv#2G=<$ykj2Jke)BB-Ulg z;T#YG_r#$SM`ZhV5=~7z#oFU-K|#2e!b%2bc_S^i+m)I{Nmi7Y2csdEeaQ&5sN7pV zbKD<KSTS#7;l<>~dtediyT6_ur2-7;JGqEsI+;iYeUIM=fo(9FLO;}0R}Fs7B7oqb z;!G7S2;e9bUAnc>j-VKJIQ#|2%*OOP5wPoWRK<vpXiJ+KQw40iq!94;1}mLh@$;_q zwvK~`3V};=7-8n$AQgzq-5#u+6x*DKWLP5(p>?M^zLk944k#gTyxPk}*QOxP%Aoow zeB3Luk+And>=kAyW&mLy2|TAQ!`vAIy=L%t-gCq)r&ExlO<p^xOTD-Itdrz1HtwnS z4lzCVs~FngsqDg9gL%)5R^FP4@DEf&Bf|`faLI^br5j`c)%#>kJNrGdG|SBS?10{S z%_@ui8g8!~MfhY0Mok{K)Q#<Gw5ER(YenPQsXU5@*3Ew{BbC-vZDQx$U4pGT?sZ=n zRHsUA01;LX`G;)+^!9?7C{Tp0sj8|1?G~IyCqdkqnVD%Mj3pbRXQ{;ow^nKcwu(0$ z4`Wb=r(W)VaEr-AC($jF-Rj$(Ib!bfIqvnXwP$1)@LZj%$4PWUJt|`Q%&6UKRsL<m z7%ZJH1_G*3%Y<HVok1mDY^{%PtNDaKQ)FWGmnQQG$i$iEA^dH-&w1KQ5C*DU8#>8p zrzbUmX{@M^sZen=QTl)sP5beskF&t*wigQ106bU-^Uz;Mf3wObue++d=9=u)`3kqy zp$`dBU!c5ULpS$4NaT6omWWWm#P#-m^D6YB-x3N=m==liZ~7(@Qh@pIB0UO+R@OFu ze9V-nhXgvd9yNuGehKOt&_3-3Q;Uj%`aUmODVwq+8dNrfm$W{6aQ}K%h5^g#H(!=C z%uxXCyMme?;Blc1PlO&z8eit#_qEj7PZWRYo{4tf5EeY@4=SFrH59)jU1F><{4I$x z_j>_W1l$<r3!Co~H+j3RPIKScM)=49q&)S^wvGUhv&?d|Zji`#78;!80wv42HB(1# z7RGz@;AfYJ_e)O1b|>qBZ^l`1RwQB5cVaZfiMuiL1sevFP#%v5>n{vBP7nw`{583k zJ(9<ys8e5a+7i*zr>%~*+|u(b+G>MDa;QjMdU5DihU<FP2I-UuT=XPm2OZW?i5JN& zj^>zDsr-_l-xjr=uI$hJO<LO7dlf8y$BjMnE02{JXwL9J-2O!x7(m!`z_aBZ-$1hL z0)ii&BfKpoOW7<bd0T3$I8zJ9E}}H6HLP7XOi#rYzMJMg22Xdh74!GMp4D?U$d_~> z2G8nu7tYhd7mxw=?arVsNb7aiMwS~!cPr5Cqh_W?$zB!YdD-QbmX;5i*~PZ*xfOna zB`6OyP<iBj+~~@ubVOR7US>GSP~4LJOow6LdMWh7c*|k)Lha=BobBdjvp`Xif|rvN zh{@Sd(BygeO3O;<bD7N?D5AxKf7D=F3<ZcDW7uCf{wuN@oH137CqjC8)pR1n7~>dA z8+@)rr^8-Mjpn)i)OUt5{p;{G-nXOAC`<%QHBTL{?1G55r30A_QWWixptOvP`Q^3{ z>i7<u5*~9ivLiwN^oA(tRg^a5{ehrA(Rx$UJsk>DBrC~+zGi~k@OJvlh1R;~NT*|j z)QEnHd0VQfy=FpVwLE#q-KQLv=n1Zo*F_OD8)u$VwNt+T)SUG|b3lL6*inY-vtT$- z#JzQ4tanpp1Cl0g3nXTrE?O6`<MFXzbi~(V?pSncUB|u-!v6O6xm6i`Edb8LJ$@J$ zdFw{ax#N0NCSjdW%W8uBYYn!h4WIF?h#}83K%%(HM*t=N#2q@R8sspO&D+H}qAkls zreG-Tt5itu%icZyL0q%^%3ULS5wUAy)fJMp_8Hg~@bjnQYZ5O&5>GRZ?fc??&Y0tx zf)xgwQGdWFxH*>p*ZDmH@<~XVVoEVgEpLXhb&0r&IqILX-ON+522uoK)K9O-DVp_w z^JCuE9kez*%$hHVZJN?es+~}!=N4UuBLCBrN?6?U^sXT6uLEJ+dS<jxneXC(vrmS8 zwwRrL=ojhwbD{GBnA3E^IWK}Xw}N@0QfPUMrI-iNIu`xokLn!T5AD{)#c%q43D9>4 z8BDAb?GyG0!aUSY>nbwUMA!2buekhdp8Le~MfqspFW<da5ASQw3jVsgk)6CwVtv$S zyC%V<nE$e759q*yJA8Wpmn(6{#{QSsO6c0NG>i)XF2C2$<%{mPvwscOZ-2=6B6JGp z@p9p+)8>eR54f?(EUlrPnNU)2pOvp&e&Fl)=x`T5xlizP^(S0hylpAr=a9}$ISu=~ zzR6I4iHEfM-XgGRKER{e)ecasqD}?%^Y?@BtWglx*E2)a{JJ`Vs@@g_oD#$!F;1F7 z7I}VEtcDc*K$KIRxnN<<A}H+q)LmKRxHhnnk(F>B^$Kl9DH&bywAJIh-C`$2+&dVJ z((de59@G!=sXu>m=s8lk>I05V9!XmGT>(WXKp8rs1z)G4-297xA=D7nzv+4zxWc#C z9eFZc8TEf8UohTyAlq&&V_z~v1$B(v@0I#gIyN4bFX);({C7meKatsQtDBWscX;d{ zpcm{qee6FR)0J1)vn{3RTT7*`3z*MlQBw*$8_~h4W>ry(=<t&TX_8WJQL6<ezcTAa zgmSN9H^lPe%|x;lxE3mx*3XNIK+k)S5b67^y$s*^j+3QjwP8QheUV#x5UIe!J+d;? zCu2a8rJ1~l-JiO!;$X+n+;4J!uWNtN6ObJaB|>H!ysa8TwSp!hCXO6QIx=p#s$M1n zAyZV%obo~fi$^!&)|<Us@hN(vsX6Bgj<)ZSffb6K;L6VzH<!(z-WnC%kv6dHyymN^ z+OoZ=x0b<~*^$N#Z+<`;@t2sc^SYuWmVJ{bQIN~hRqCia`}@fo-HTuDH2Ic|dJx0M z-u)=c^QHwC9JOCg@)L>`O&95!^=Vu#ysHOnHptf>LDXi~g+CxFs1w5+0fW^d4ygbW z3rN+Ctk<k7%O6E_JaE2U@LrZ&PLQt4KmGpc*oaufs9<rv25R$6znfC1DbpJX6JPTF zGzs{<0ujKq?|iQ78sJecmiOk>c}R@6$ZDH^;TvvZVUO!KfA6>tg94_VH8EuwB3im4 zCCzU^ag*B86NBQ~+WZMK)8{~g8G)hPc815HaBedbN+RYf1{WG?b2k8e#aw@!>agrE zH*#^oy`?sGpuXaghl9I-=y#m18duhqlH~f|qcwBC0UsAV4Fs>C*f5eAI$42UPS9TV z;rTprFD)dA;lc)t@~iPNa+*I~3v=^yJ=WbN+(6sB_xupYxsl_^$pi1a%D$vGu#(Lf z8?^NC1}vzRw!;l<1;2X==y^>egL^~=ZAib!KF`@3Jn@hlJp1m+ep?Up#T?-nse5H_ zLs*xRkE&T4sb)O`qor8Bu3r-(FE)mH5{^_SBuoP)$!+1iZ|`ERGU~m;0SMCi>hHN& z72&3+jgdDlAE_<u`BNPZ+aQ7xEJd?gL5Swk7ntfOdhCnGR+jDeVb%!84wR`ZLgW=S zY-R53ud&#Kg~&eue=X#<a4dVQ@4)?ONEAy_=J&q2{EoD&Fd*hho{2h`7ES@js!YuY zE7^V&n=O73iw1LmTnGrCl7ZK3O_|qv`Qlo#)dCL-JoRxv{NiV`Lh}rHwjAdfA<3Rq zgR)k!SQ;gdYW|w&QmZ{5|4z&=*V{On6C)$wpjMAh4cAt(N-?RBo+<<6#rHQ;2KJ%~ z_RRq+C-WM6C^Nxts)tqD^RHLC4+SnBs!gDhpq|rjPm2`A)ao~JuA}XQte|xK+kMJ5 zK_YNsitDx*qx#T<9_ysBep622u@{N-<e}b)QXh(Itn0`pUJ4e8Ci}m4kVVyx!~4~x z4NI9=^FCs71y!!=yQcr5EzqSH$5Z^Hmy)wZR_eBd;+~KxYtI=3MvnGp>;RPx2nDD2 zYtmCw4NEMVcsuIMmn|)?Afn-pu~+wam$U0O@ouz*%n1b{L}KeD65Itq)`h>!DXwmG zd+9y^sUzxN3hbwr>FVuG@zi*{H$U<5^Y5;f>tS@U>U*OAxow|rM#jfxa<boL+7gD= zyJbC`mg+1l&PHLLyx<>g%se81Y^WOtHuR3pOObLO9W{Ud>FE*l69Nu(*em6D^YI#c zUx?laQp~@AH&Q$e3;uLPGBft}tZmOrjrHr)lNko)q1@vpcif$ta=v%AHLU(^-j9dE z?YR2}Yu7}t)DJT?f{3w_t7<BIf3`9{>Mq7EpDaThL3=}Fu~I@$i?P3Aj>6mP#P1t? zr0YESd+V1P2B0F-%ja)|8}l?U1*36Vx9{-ixwA@aX`W42-2&I}a)kB!@wPaJTA+;` zKF)~idL8ZSjqTgln05Cx`m_j$x#3{GQx3pnf^VG(G=7+bH7|s})~-g7Oze}%-9Kwd zA4!V_QGy_eb-W<<rj72!Qr(-oNLv)CRVB`cY)LK6qTN2fp96jo=Rj&oT;wsB10Q0I zhQ(Gl<KZ1Rid^0D2{2nek|WxeaI-b42V=Sn4Q^zS4`lzch8=9od##pP8SXJ%;x7Z* zNp8%~ZVY?c6rB{>1^0LuPLH2V$osBuG>v`I)`J<MxGAw$&|s&ca&~mJ(hx*#_?OyE zZwd#6Z21A7HJ&`%VC{~dcW{;Io~WfaH}(YS)Ux?0hNjK%dcDMl%c%LS+KnTkqbuR3 zZ=(w_%5W-t+a?q3smYtsy`N+V3FlpKIK}XcivP0u1$Q?})vbxmNVj!z`cJj7JQlez zD_SNu#HVrT-njMFtxA=&O}$oz$HnjRwMU<7M(!9R3ng0Gxm{B$ACZqRzY2RpH%X)t zetN!eG;|*y+nC?loVZQcxZ8Qvdz-m67uEeaEiXTvp8GlUg9%@27B-3EOVEc8A{J$J z^Y7o9Pz8<#OaBc<#7<DV&h}?bXJUIB9c(}$ltvs7HHeJij)CF++UL#cEyv>f_Wmsy zG<6++p<bK5X7|V3=Ef+6kL-A)T2+zhzT560M=Ov~aEpqLmcKMf_2_KWa~F_x>=C1J z=kFbc(AMTVL9luM{PF_xO<j$en^j*Y;#2?kqBLlJz9At9#D45Bc<X)aHp~5aXfbx6 zYQ@{I##N>@A4r#8>W8xX(<W;5NAUlIdyAN>l}w6T^=#u>di5v5AkH`vOm*Y!vqm1g z?8^Gl<7NVcSCKkF+klXr(s#t~Vc?*risENIq}u_Dh|zhik_B$U{~*|4p~y!#`%~`G zNMG_}T2ev`5a9W3ZZ@yEUfmA^9sT6tdE(lBeB{JRVXUfhZvJh{L+<vk>l}_}IJm8! zVdu&VEAHGodIY9^5bFZ;0c5}%)R%}=ajiyBVePRP*!+NKvcL1=SG~f#lscTZ9lOsP z{UomUU#r!7e0f=Nva*s=i|={cteJ!nU-xbs-FnVgA-Sf*m(J0BMpZ)`)@7IFbfl<4 zI0R1sCx^B00ROgM+R?K<g9{F^iA{#UB#OYbd;{JngUGw&jo1v_gfX3r1nY0aY26r< z`yvNQC?mZ3ea~kTO*{oFHKs@~5@HA5NV<!2wG|xR#jCnFcjLS=%2o!In)c}h1Qow1 zrE9}!>Q6~~1ETU*!@ugcoUi&{eJD5d`fLNkYnxa8Eb6tvA?j8p1pMg*1-uLi9WbVO zkY37N!lFpssdTvrd(n9t5ZZ{USfd)DZ4%~?T?jzRBTd;4NhI27P~t-|<ts2N>}x>i zh;QF7nRD1)`)RRsGxVbu5--n{?m@P=gvf`RzXSBQ4i=>Rr^H+l$C`c$ukM!@E9^JC zYtiAqC<-&S8+Nu*c4zWV!5+zqAntNnAm?!VC%^8aI0iDxs*zA>!3lCXTAcX~C@yti zh~12`j1x=Om}_Eef(AdFw75~<6Q#Hxz7`{M>eFJ>fNNm59qK?W!%W#M<I3vB?3<j~ zki5=>wx7lxR;LFfWDA!+v$02+>^ewUQ-N6nWP~o(_ce*-%IY&##w>n&WEEvJ6HxD$ zXHlV)>xd&#_m=6dyqlk3k(2MwIp~OW^rWa(+Q9OvcVYVtO!{MvI2!At`FZ!8`|=%_ z`VpC$wNdGC{#$IZr;2ERQGR6M_AYZ>_k}1I&Q5cC&HVm#_M$5GF-`t-;_Z~m^fVSh z?mu5Gyudq2AEW9o9mx+=3+y?6))nJ((rG9KnCcZ4&NmJRQpjI-q3wjJv%YAj+sz$E zykYMCd+n09RXTQCx{nwBs+mN6rn?s^Yy(JTWo1CgY6<K#A*jB_1Iw-ulD{*tKa{SD z1Q})<C*f>!Aa|?6z9Z~kOqMdc<Avyr)kb2jhT6i`bBAwGW`obvlwrJ^q@tpyCxXZ2 z$*m(mdtY7Mv^$JFwe%SEWw+eM#V6AkTZTq#-V^<xp`>?RaW>8oLjpT6`zee@4~ag` z@;MfGkP;(;*QX>_KQJ`Q#x3O39d5A@WH#9vG-K>x;fNl=#o5W?eyp6Qs+;e<;IlA@ zTNtX>Skq!+G3B-n6h9}7V%4x1lQ76$M{(?JU*F#d5Lx4J85tq8{aH^{wWur83gwq# z_E6eI*{(Fy#uq7tvkw#jMb2H%Q#8?#CF`30C4%B}_tq;jTAirD7K*+n+tRyczDv>X zY5T4%%|ZLyw^;zAj?U<wpCH@lE$JF&lwI=e$#;|xl5dr^Gjj|hV1wv$I!nj?W*mai zKV-cLDkpRY+WibhzNL^{wu<A+M0v-L)w?K@I*P4QD{8{ob_z+FDa6T-Gy!L!6rVA; zN*@FqA}c$7l;Eb@78_O7YhkhNyYKw=&NKLGFzVxal^Jgnj`OCTB#t9C9xWgHG6Db8 zI^tHxM3?YlRBJ26o0;7&*R(fE-uWGD$+Kg}7MD>bb?@$t6I&Be70bZB<ZfD*D~#sY z?tz(jNtcrvt>5lKU-iY~O|5SeFy{c=1!e)(ss2j#Q>!ene%*F66b}uQBDby&43t`& z7Ak;+Tnt7><jr;RrdhVQn1@KGmak$$PpYce0S202pH)xdt8QCk)A8|PP;uGUXF<wS zvX<G`RYAAfy7WF2^nsn2cx6p~e#5F|woiPApaPYNptUu<syEw&VPx(WOMTl8da?_J zSHF7LF&CPUkirb@paDw|7JL^Hq-m<CJUXgRAW=I=>vkC3A+)`{J@tVq2Ru-OK*WsN z(@pJgzl?UTZ&di_5|gFAc>A)IM7u=`WgqYdi#ffZm#EECdI;!2Co9Etof~wBdt#!Z zqNJU`EPJ`?Rt3h}p9l&hsLMz^nKBu0KyBxm!Dgv;kgf_ne_|m0P-PxIcT3m8N9Sg< z>S3;qV}i*&g|s4ZS0-}Ia3Fj~SAeA0+H?s@;Ao?vu^yh;0R5gy6V$dd>q7FdMF`vk zIohyZ-(B4fF4UgR^k2~SzqN2=@)!DWe5yHLTNmmx4~zLTQSqB=;S!tBgt*n`y1CQ1 zzJg~?9i5z?e&5u$)TMn)8VNko8%E(jKV#NOQO4NZ&%ddWZ#NrX@7e`7p=kY%C5^UT zGGe5z4zg_dwe%H?jF^owV{vL9w-^viLTni}(&uJI4iy1eCuky-&5mnkAt9iuFv0aB zp;#m1{vCK6Rs|yF4dwBzi(l}zDr_Z7rGUK=6rW!V)7v1v;nrswA$$O|t&up}r9b!P zO{>%5Yq_NJ)v-~Yk`kooIM{TA<G{m?mDYbwa>etqTdTdRbQ&M~p{;v6HXYfh=cbT6 zh6g_n5&K1tHs%`*YX=y8CucQc1ys@BEng@ZGUDAP)QTF9@&uLBBwGXy@om4>vP5%> zcYTI1BxYXsLa_|+v@U6K$jZ94%v<;Bu$j?RLD;>~A6F$R===r*Ohgr=eg*W7h5HWn zLK?4-%D`g@19QDcMA;+~#Xm9cQ*6}wvz_cfZ?H}D2463Dakak+Ag-o7Xsn9YwTTXU z2{z@pD<WW5qjB_am^}(9Vp+}Ks%=|M-7~kDY|`%)`S-^ZXpvL?V8Q)4PX#l*miDIQ z0hJWh>))B)4xH(_RqHcdd0fDOZ!3ocUQR(r#0Z644*Y6I+{tu~TSx&vabh>@*O~Sl zTBN!rpNwz294uQ7J;Zvc*VSj7t-oyQpT)jj**ALJzh_|h6PK3seL#_F%8aX^s`zck z-8J+)wPCGi3a4mt)&|o(p4;wmEBjWs>yg|KG54s!pvLtb;lptx&fw4(0U_Yj+*y>V zx}z4T1=;er?vVgC#;NJ)?=$->TNC!}u|TdqxqfcP2mJ4<R%GeFZ(^gI0=u-^Uk;&Z zsQ}5K4}^TiVvu%0U#@6{1YK~+eXi(%s+&RGG%-<i5RiXb$bEu@+wNHa*G}9{<NzI{ zw76JpzcpO=W!LuU0mEOiv*$~oH=-B{FuM@k(fUj##P37`B&h&17gQF!3B-DOKDik; zCd5@b6c$fJ$_}4xnuncF6@{H`(uJMxceF<i0Cm1z;g3J>>{}n%UnX83gdP75zNK#n z>=FU|&AK|KK*quj0sdwbLm1%rB!~K(Rb|CGk`2ZA*sZBQKK%Y97@GU?!0B@~M2VZ4 zGL&W|9DdzZO`^WL_5P9RN9hkyCaOym6n$7AL&5nvZ$k-k(-vv`M-bsphwiDqIeajv zH2Y7^yA9{<n_+`VNKA<kHzp1N>p^r!dsuL=36G?t$%!#=y>H;xDzgnDZ68mU_Ih8^ z3$f1Q6Kl!u-2SaD?Ntv?nuo7NYiP&b-9kI(yTu_i(QWj1lbu$;H=p9O_@X0M$P{Nk zZ)CQaiJ<kQXb~eR=U=mLh81~95eTT~UdHdKl}yI^e$SkDbEP~kLXd&!CCaO+ELF%2 z)b$#`#{6zzYGOtRP>K^cn>1x|2MS4({U@?Bw)73Ty-%1q$u9U5L%1hRo-+@UU#I)c zX_c&)d*>23Dr4NjM|8yeshyT*u)eutaF4Kz$Pp4mDsCx9=W6DK%py?aEfeH(#$syS z5-MMB8bxT$=*GMp2z|a@h?E;K($3LLvD?v074kBCzq{Jk`0a{DUW}PAt-8SekNMmz zubqUGwLtd~T;H$*wcQV`hE*Kh!GpmsG8$bqt!-Jte&fDxmvmARSvKZ5=OfPgzod~$ zG70LECOEzpr(1qrA&xhb6%FJ>(%t5oT`D=NoW{UH>)JwhE34kTXO2OKUs#nIQQFSH zoKh_}?{Hsc6RB`5g)_fY=x*w5pCum9`RD12Ra@0%<c<974BJ2F+o%iNYyUsn^zT8` zraXF)f57^{6JbZ>GrJ%8W8z%m@;*2FYt#sW`Q#~^t%YBS+=8C;-s+zTS_<bDqgKa9 zbi4b9d|))ihP;an$>~}PhL{`*0|Ubu^rgoiH_iA{x?$ZC<{2BZ2&Lf`dNq(9((e9@ zMe?A@w6*s7*Ev{9Ji>c$dq8mm`npMu;DGgalIr+EjXcnphQ`!rS4~ej<Ty*fHOkuw zwY#sku&#?@zY$)<|MAWlg5nW;n89aay1(`JT=nkaN~`vLG%OSt78ld{B<f09=8<<X z3d|qVt;xr`Jr(Y}CT@Ut+1uIOUW}LlJZe~r9=pqEI@e7h%gtVaA<2h)uOom@$*Yc4 zOVPxoG@$p-;rl`plg(WJ8-AU(mt)PQdmhI_Yhc<);G0>f2*sMJX}1q%%Cv0!z16qJ z`8m!`;f;J9_vB#>kQc4c%M&tS%HDZEHUj)bAMUAlOxo?%n&HC0IX`a(#0CX_(GbQ& z#|{8T2)w^qeH7XOFf-t&rxYbf3`F>_ilSso1Nu@vLVC;%`yM4ycEj1%luf!sK4s{i zbK~{p^%*%fNOY=XIds3No`DqAPJ|fH3bzikYpIKA&}I4;zcME|>4W+w6X<K0J#`yw zKv+lq!T}=lq)1$<mw$98z8a{YCG6pwyZ2H>MxN9r9S~ZIi91&|b{C+u*5PlaMm|p2 z8h^D%*h0l3%py61Nc3(A0FTVG($*4jo$nc>WlC+6m$oh(G9x-2!$+`b_1X=e7SE^% zP1=Oqx6c!_N1s{ua#{FFj(>W5+YjrrUpQr5nCYH|WfCJrE^TBz<+P-f(a3CN9Waxc z<XECf$q(&9bcw3Zw;ztlPlvg;CE`SRQ`(?WP+p|s+d-(c_f4ETgt}Eo^EsQUoR513 z(ZX?&dx_fDC74fA=Mty4Jl>-`ydSd`ol6k>dbGX{31dy#(fowDfK7=X78M~=(%4so zGObu8$U|WEA=JQBkscK;nlS5-z#(JueYDB-aX!sD=p=NXGT@LLDrIR-`Qv>LX2}D# zP{XI3<T0-aR=c93MIN!F#(mu*AAb6^ZSbEpMKe<D1<0vp`wG^|r=!4K&3WD6$!4U% zx<V8-NDMHB6CJUQ`QIZFxERY9p}@m26L?A1|M$g+B$jm<GB4Z-{oxmt8}iEfv^5kv zFE8wWA@=n73CV-w#r<PFUg;xMF^jG@P4mKsH&Ky_;&Rp{QBuJCA$Y`3EF`N2{k{D? z3$HnyoI$m(=V1SYb(8llkDhx^5<Z>}S#QdaVuKv7dQ&~1J&J&S@O!(LdZUY4e4uxG zq@o>P3HR(HGD1k7QM@!kesF@zlopWltC<|D?f!H&v@hjwvxVn4)kH?P!)IL66|nGf zM-UiP+ghM4lM|*Mju!oOaCE%O{^bnuw4@btC!5#(ZSEiE-rq-s5<5#M8cG@(tP2zO zxPds7^uj3UXgJaX#!fcN+-w1{804y+U*kUUdY?0pA%x_Yv=D1GA$XirK&kk-73Uc( z1l<q*U`%sVf0)2JgwVC~p@8$%VkFq^()rXA`Ort8H@f~w8sB8ecD&TIdP_d#ThArx zXU8{J-QEnpDo;7yeaBC=>Uf@Ib$(ebKIGt{#0rbe!3<bhG>NNrjYTsNw6{T}n*gjb z=5@<wY#+s$SP#gOTxF}G@$f2O5Y-;}T_2eI&W{$(oRb%SMg8JRr}o-z0>~w1ivoi< z7Rn~V=ikHRwqndVnF~q8H<QU0PS#9T-i+tWT<sRmf4*OL0HL+J3f#!>&9xbCEgNJ1 z?hODrF9m}sjCp~P=WF<7I?V(8=Y{OjMuAga(erl1jv)gG9;v44?Tu8~Ir>W%(S4qi zK)Pt?JSdpqXEuV-?(R$0Zj!BnR6=t{tNHV`Alq*QW`S(5VWl$Djii}=?II*fvtAYO zAl9U8fX=m|##?QC$9$24-d+hlqIXkve-`DO=VyjH!4b3|{nYsz8*1H5uv)sPaiw_k z$M|#P+(w3Fqr{ut<Alqftrb6;ifTR{u>||?qV?7XQ~#lXPd^#@{g)|l%ur&&qCu1Y z;%7ioiiZiIY8(Wcj@_1HIRQB0s1~qCMh?DVTzhi<&Lm3W7Fr=GGuj<|<%X4iJ9NQU z{o&XCV3hF$ZHQgCJIiZh68B1QL$5K;_+Gtvc%=Hx(n@^E#AH`q!#|PW-<uKdQ4rWS zs{bt#*d7?18Ccl3p&r;q+)>=4Juw_g-I?wPmu33l-^bp4I&%Ju@wu_vnk1aAv0URT ziS_l>Rb%Pq!^3X6#7Dx^xbUtn5yw|)f@t?=C<=TDK-<LnpYsQ@>SS_d$o~_b$8$QE zQUaCPt2}UVsgz>o^<0EyZ&O`PJU6Rj8^3=*EyP=Df%<<?cZ=GVau|S4nbI?3#%2pJ zE5)8atHXg$9dY)(IbB`vR>{``WU^y(IqYNA{Myy;jUX*MNuQryuDKD~&zJ8Wm6h1u zy^GWdI5|RrUCbd@O-6~Gnw<PyKkzP#O5C+<WI`rAOURon78dNdox*y7^1Tn1jk^Qc z-xo^l*fne}JOb@6<A=p3l4JcIa5d^l7;QVzwXv^Hf>R=qQdO}o2vhgO!b}eqBcqnp zWi8uK@by(N!ok-3#ARsZg{Y9l2fjf+Lg}=r*dd!p@Qe>+M>ox9Xi!8a7gKS1FDWUR zo5~6_qW4}eg<pt(kn@P^kGc`q3`K>e;dmcn2kNWv2@7`*nl4iv)a#N9lfID%zWlTz zuliec>(ys<miz{MWUH^JeLoIbMkd_hwPaSER|^iCb8p!B6&E=WlWqv}EqxF7{I<-s z3UY&uT0MGDUe%cp{X3Mcm$2K!A!+tK_^h9z7L`**fXO~`o~u?e!sZ>G+*C%+yV+~? z7W_qxk4Wf^jhKpTT_N_5z~Ee%Kg}b(->qoQ*^tmsRZ1%a<Wd@?O-gy^*+<6sreiji z-@wqVIe2$@0q@gDs<;BVVdS}T?J%^QOm3$6C17w!b(E!3tOv?XBhx9DU3N=?CRCum ze|m2P;Zj;7uPT%--0Gd!4|Wg}$y*rb<xfE6LgSEW+iE+AylsFo%pbXH59*#~zV>3} zID3zrcPDSG#g<#%YS*fRERY!OXmMCJFxraJ2@Of}7h;5}>m<HqfV$d$H=QdR=Io%B zD@AI6O<HxjCgLN!`lp2X{?iqVI*Nd*N%p_0raIv6AVYPGS>-juvzNR5V<&rjcb%Z0 zjPZ_%_hReB%UMCU`9^lyo|SosC%A*xor*B(t28ECUsH%C>O&I~BH)I-xMj;2t>&_= zTK=%M_8cH(bfB)S5%v@$3Ix3k^4QtfkZ{{SZb`j9b^Oz){t3@i43b;>&-WG3yoLsi zt=FmOeOrD1ojrngA6i{@GkTYeNrd|eqdZ_x`@8L+c=e(Nl53(U9qX?Ltb=2Wn72hP zD?>ine9n4bh?#NMA2)CuPnIAyn2X{cEvz`GrF_F!zFwldgV(#h#gKjyCJU=VDlj%u zmS^KR0=aK^L_3}lN}Xk62#Fdvgfk$O1oRxYiay<3bN>s^1H^@6cOYiI(A?~;MRa&f zPkup>lCm0gO7zolO^?*nEQCrulgS)Je1n1dj;Yn<nlB+ym4-EHK$@Vq)y|JQ+Td(t z#sXS-jKVfp-*aS2q|Kxb^V4PfA=O>Ddpq1h&D_@rhVS!drUfNst!NA(zMuf4Ko61^ zcxdY@7LaXYkDQnm)#mgS9vsrw*I56a@Z0r3b7XWRFz5@Qh9*aQ{{CI8&((NDM0Fyz z=RwqWNf&Dt+W*=w!7*)w&x3<9=8U<QqjOGqy4F8ofzV~#{PlBNesq8c!7Q<84h>HH z80Ei(2}&Re1I10HFy)26T5FZIw84X!^n1R58=s9KP3-&OjX0zkWM7(jWrPY!6zwBz zt$KNx982Go@go0j1`nVGvN40zopJ@>P~_HGx<5m{rv%c@jUyu1VVPxw$&X+>1_5FR z7V7l!kbse>kzDQFDBb5giI3ZD=NzCnwg`(J->(b6T?Bof3|q!e2ER1Z4>@0kUSF-y zXiD*6E|9zx)1RsTvQ|Q?$GGxw)!8l9)$rT3OwUXY8#-a3@SkHA9Kydbv43ILGvVM% z>9`3Y-#ykGUG)X+e1i905Ua)^a#PV?qc*D=vGy`c*6>GC%X2M<ggc_Q|64)uMnWud zVQZB8C%E<bDFXW-3DDX8y`cbcL4b#fWXEf0a%ZD|dX@t49JE&F{E%C)3xDR?S72wa zVu=KJA_BrFZ$!jQwBN-g1FtVj_PJn@{`SZSxwu6?mnoU>kL(wJ{^WxDfl;4NJ4M2& zi{N1XD#wVsG4hwFigqDZeAgZ9TR5_<KNDX8B7954pt5%rcoAfMDeNu8g#l;lPITt+ zDVm$74$Hxb=8DT^i8;kzrj~C&`bv=BdQ+vi<@!77<dKvSudFaP`kaI|mhGHBsre<U z%uFq<GM{7Qii^L?`Y>6*|E(X@#vTfcpom66O-T|$0#~c&mq!k~TxLQk^mNyO)vssm zW(VVUQL^B0F}nbym?b#zI=>|JG8Aghj~?rg2zkA0e}RGJc&l&8zBsuUHLp%B`fX`d z+4T8N2KQ7Lxp(2P#T6}@kcc$-0+WsB6c${3kdB>e-=T?m@$lBr_Ziv)eG#)1Ykeu8 zQv{|k5(d}7*9I55gvc%q@kB*Of8ENsxmt>Z1rv$IpVO~y8-qyoS{$_@W9{&WV<(MX zCwwY1(<bYIfiS%Lb4LxzAocU87T?@~F)RH55wjq?wW+CmyKzFwD<4AVAL6ib?=q6I zf}%#{^ADrP##nK2V@0fNkB-IL7;MnxJy}#h2_6|Ji5uS58}(Rh(kPYXRSG7VGfEaC z=a5#zbC{7kEt+~C@z22q2G#4%se*~#SYDnGuG{@u+#f&tpQc-xey|AXLMDmDOAA&J z!K2WGVe-wMyW_OpMOn^1VRwF+DZYw+y$B;U3>T5VwrpM7Q{A6ul7dQT%r`Y$aTMQ! zm8Z2R&b#qv?+;jNy}M~m0TWJW<%1ORpQegsNf;-->3BeHN6DW9%AjzANws>_!TLhV z(oIUIWsbP{l<)us{fb_E7df&lK0+T6CT~yz=gmS_!|cHBo^#?Y`YqkxVzpa+4_}JV zxt;C22-A%v^5GiQ>tgwJTW4TIjV%i5A1H<b8b#%}KIGS@mu)?u(~{MQKF&tdF&XQv zS}UHDMV`8ttDXE`52ZSDgGt`VgWvF@_{p1@QztD&Yf=BCw||$&e>Tq%Ai=rqmh1m} zMj?@+9!~#sty<U90bwIcMhbUxy(dpjN};6T{Mt!UHWQO86nkpd5~aP!dd2}d*98XO z7`0;Aq|38t6eLYqbU%l6eWWjmyBFkzzon;VOspcC(AHQFSaFALKJ}F}Ivl(o*KK5q zvoSKl#=}eg$i#pjmlT)v{l}Jy<A|1~PH0JkPv2pzCwqO}X4V5sAoZ&ECoHOI*ti8{ zJ*{Xtjq>fF;q7x(^Z1l_=FaF9<Xz+>3*UaifZ(tw3~7``DmAju6!mnC=8=ef?=v=~ z6MwY2>+6Y}>&QXIkNp*|XQ6NS`{y4YAT*ANG9ywcQW_c!jdFtiLr6gSpV6))vP`5{ z%yo(NQ7_t2Q>2E&+<vG}szAgMS$=G8UU|957lunw?$W?6LlykTe0{f?Ux3&^Z(JV& zgf>ufFiLHG#r{0=On($?;J}Nty+}#dfq5Sy;Us8R-gi{E`+<@%w(W`Uo_m`aTT(+^ z!`RPJWWI7}?DP(Sf`(bHRJnnJY{{8zZ~_?t?eItW@BSO;%X7y-*-E=gUO~b3blRHV zt^M6dCUh&ZfUY-{L0DTTVvAjR(Rd3xBrDM<QDuORcKf!XBP1-4a!YDa8+clo*f4MB z!Gw@DImlyp+N~ua6A?iV9DOOiGu?YUtj-fqwbATFIF=BOSUTD1P*50B2-6`Grk#eF z_A=qys7oR7NtMC!+#^nd3<7M88oDS95sa0W=$fY{wHet-M_MBF7Q+@y-`hxlv=e&Y zUE<4w%9r}I0uzcZ&?D!J<or2Qq9cA+r^+#vT-BREN}2=PWm+_m*GgIgH!|Kt7<zVG za5*s(SvQxOI+BUANn;t{z)Et3Zh6cPoY4!#B6>&f+pS=)l%ON(<3p2khFG!~c_>fP zZl}$5{<J8Vq^rbRkKbevE9<UD+PZriD50tMw_vT3ls`<dcn%q{O1KZtPoax*3Z?9i z`}u<B=FI;B#=&Jqw%>eyqdNp$>H1Mca4W>Ow(9?C?*FqRoLFD}0>$@#b!~w5Bd{B+ znH|fP(H`olVw`z}Hk?hYr5uLFG}Un?x;(3jAQP2!7f`*G4~4DUN3nkwJpxq)$nvfH zl-~;7?V!L!L{&AeRjFfNwr3u_@Ke&ud2+QkFMK)SB{V_&%bMzE=4%J(UD#*cP&Pm& ziNQieMWy`s0X0x0pQ4WL*m$eZ$g}IR5t19ob)cVWmR$Hu?tg>u!)PlQ<X!V)9u5qz zA8dy9%(*3M-^;tzH1eyrbtYLy!u>6*^>eS&Xy!tVmSemtpm@%$?n>K5i^cGYeVN;E zVSS)G`CX+!(T^DuaLAAR=XxM9ltSS5xx{yNk-T|OWPfY<Dm=IqzyAxrwujE{ZAw7> z-+qVlvePDVV+`7iA4)UYo7atMxPfoCe*<s{y!xkh15xYGRGFff>(~+2eft~QexzD^ zpW(l)O8%l(OE$+iJeh;#C^^}+Zq5L{>u-QLsMu}b(ZZscyx@@R2i`C4zMDVbuGBs! zqHI!i-%iq<j2f;JuXxt?lQcE<OPpzlAb0p-a+j)@dN{<6`_tmq(YQ$x8vOC7qQ+Ds zeOcr&^NGpPjnLA$ThVeMpgX!Yp928KqNJ#&LH}cy!S!JJ=g~&^(Yc@-KcFl~`v~{T z7b4^eyr!fmsVj%z6iwv+oV$Y<sVtO~2qJ->qEOynl>raF4}P|mw6bXnbF0z+x>c66 zzN4QS&D~>1lH8jh25dlZj7IqyW}*V(Kvej5yhJT~;n@W^@Yc%=Z_vf5(T?WyrcT;9 z#B_#A{)h}Js-ybhk2eEb%V);ilN*EC-Fd(OQLiIVdLa?PsUL*)sIJ#l_oId<)f@XI z`Trs8t%Bl;+APq7;BEneOK>N_-3b~bxVyW%1_;o&JHa)$ySuvvx5lAy?#VwhRrg`$ z-mcmd)em&hXUn(Ow}ex1UWcms54=>-L6QC^i^1Rcf(~Gb?7cD#lq*kHEaaKT;+7(0 zPHCo|#xb~#kBcz|@;z;mkxb2J{T90#HGS<xMyB^6;Ee<?jC7B(S~Z!C3ET`Z5I51! z0Eyc-O2W}+Xdh+QfX~pn&h<kzx-twCLCCOfnw8?dM97t&18)}Ie>h8cgn!<kDJI*> z-E%Q?LcSVXPDxEyG-<tl=@G_SJ#+0MCG%2IrkRNvo-`slD_H9IX$FfiW3Tt_?+iY( z>}kROAI#O_d^F=?-}h=~X^9U8gSW5Zt})!+&NoaOAqJ)c9{O29nhRM$!fx9fnja)2 z#3@*4WVM6hz6*L|11dh=NClw%!h7x)CfISA1YNiLVU>{+FY;ITV3pq+kMBdepp^b- z)`q~RE_dZIdL)~QaKi2!BUAYLJG$yDt<hgrzqdV#_Nf(@b?Dwr6x~Cyttj>38*;Aa zI|S4NX&Fde;z@kf)<$W%DPo&qXqutO+Y$-YFsaNXTDR#>DWP)iYZvGZ)u`dijd4wa zt7EnS!UuU+2JFH^R+E^EMRn#GDMVVAYhzz>tyPrq=)9ykkvSXal2)f~{QJ<x&ZOcW z0jx`glQOp}j?&)!p@V+Uqq!*)OXV?XV@PPx$FAezllJW}2Y=(&(1#)?<|fztf~g+P z>q$Y#b4OCY)gE-L4+ynN7yCVj(bRgyO~A52&sa}F9a&)K{7Pl(3pK$`A53|_K@v=8 zRGp4aI@YM)0C(&07aRUdUr5=7;$#t9RU3S*PIZ**Sd{cwuj_5wmG`t!r__J?06PC? zyY4>=Nk80&ba9U#rM6?U@a1w%c)9~nOFU-*OxhcVEB%vGk+SJ95Nbq9RaU1rO4ZA) z5815kWfSrg9wrVb`)!|{P|yQEoA6O2(4}QuM-2<fKPppI;3jdjuX9m4BZoJ$HjYck z)8c8nt?&1!&8UcvLKJkUwngW7;X{ZAvI5FC>pBq`V#73&-!4<6s$b8nmWKOkK4lOJ z6>@*1t++fSI_c*u{bFz9N4I#Ck)7z>aj)~JuiJBYL&GLVOVhAtU~FpK&*voA;nUS} zAhzt(sus_qwkgJ3iK{E-q6R}{N<9Z{D?B6f#B}+HJI%flPJK(}K7$^aRXy>ss#`03 zZ~Tfd<uKWsP0tC*sdUYYh2zWdl}vlME9IcFO%j9_H;u2+FFR|*gYXL~5PoB1>mV<n z<=GOP`tZp)+9fx?7Tp2xIN<0<C^y~U;;!LYiJi~-_1TEf@-z|e$HVT2ojoUW=IZtu zJ;TAn()45qcghhh&RlzM!$lJld_X;<kKGnv2f5tQsjH+r$f6m!8sJ2fMpb84S6tOv zJMr|K1byGo2&6rz^Hk?HdH=lUbZEgF{=)P@w-)S}+w1gnd{lB10DyvAS+}%lPxva? zduzL_ZJF8!St>6c_Uk@#C1w|MYYC92W3!;$`T^XN@Xh(|9LWC(z<p0<()qWP3-74$ zkWqt-jG=2SuUJ@E!eTLcLW-`^&xGF=qy0jvFE0=NSpuTM0U-6|Lo5~z6Ro(pJ9LMD z9|D!~YAKfGr#VK(;r#RnI>HMU2R{vC4YFZhIz8ptFLA~_XVpHIuK3~&-k<CGa(s$k zmM`_z?8?F)(p3k~iO$7KS&7-#6Y)}sJ$zBJp1i(XuGs`h^bzF6Y{oJob)8+eR9*Jp zytqBQon~PD?2rgv;172A15<ll%bXlR_Z0q(V?ZNS-+do=x(AtBDWMKIAKgZ)$~@Ud zZz~cYdz-1w^10Kxq4`Pb4!Jz+CxO;kPo5Ob*DSn()WbRxPNT279p};>zL)M1P^wM( zBOGljucBB%G>u85b$9QH7w+?Hx}b|nTeD(qck4|k^2k)`!RFY`yX0kzD6vrnf1+=@ zSoef1t)Swxwqq93KC_||fNv+SPdtbjDcv3v<mHeEca23(7TcElIE)XiL}DpPOd7~l zoRdha_?2&|W5a9_B~@i$rN;|sZ0G^M5WH#>Mfehr;=5sqH}~vg!Tkd72koiR1+U>B zNfCfYWpYDn>g(8i)nu+wF^c4jfXJZzQ|;sOJ*z9-C3mWEn6RRk0AW08)j{~67PC3T zbKA{@<?tU0_4L2qz5zDizbQ~x@&W@}REUA`Eu#B-^*cgBd=XPuCN{QOgOk(XY2(1n zm(crOg7YDfG%wPD9Uel|&r}6lSM>x?SmR*uKdhA)=4Y3slZI=>XWwKrKQ%`U!KS#U zFVSM0c``CJ_rrZ}dTR%?0~tnA?EMrSl+xvu+N$BY(#PtNXq>fM*z{?JTDj`+6{%*- z1tA46@4YMvKYzVhw0&*q@SsZy1WZM-)g1u?)%zI%LgQAPA|2JFtko30&f0mg(TU?T z#9&Uoo-LI(nN>B}V<5(hN^x2FXZ`}7Ge~o<Hu>(_QB!vzoe5$R9;6Lr(_k=Z!?5Zw z?@HeyeUUGquA^7p<m;nN0KM}aX&X8A_*_4~v7q=3#c<k0c`@5mS}cuKvD}zsQBXuq zp+G(0qk2v3dJ<KaJ+FtaNnDGB>#q#F>yV)229o<5?1_g@k&~dZuT<brh=PqB<rNQG z>_S{{-$2s=ZQ);pU*!*nnTN3*p-5&7dau3@OP(vX<jyYrZayBXn&FkApOo)JHq<b^ z5^=J#o=(LzRcs2!G(Y=bF=n@CF$+XAHtXjEb!Wx`*a<Um0aQDM00omA@Mu^6r~ecN zPMlO<Oj|ko3~5FkY0tCDKIX-ZSKpwz&8M!QcGJ>3_h_i<B3<{TqAVa~qmwWHU;&c? zWU=aXdck`=L+8@zbb4sLfZPo*`*%Pf5IHp`3DAyYVf|YZHZn}#HE+cU`1#WhUMIN= zFg;g&jWuGZsAMJBPX8S<n-vi9!|=#PEpog+#plz$af;;1?pr2~QQ-Wl74-4)00RJn zecq3cq0OVkcJcYa+>dX}YtJa($>r|;2*#v3pk`rNKPa$zL9LDW0#$j%nCf^M^{X1j zi>l@GnVzawTHlM;$G(}};3y%l0+7Mqql~^VS&jkn$+!T;19)o+B6oFg^w#4OeEF@U zh8hNtLIeHtDlC#{__0mJeq~Qn#i!A(Mr|9d0hxoWACG#=tZ0Yz46eG}F>t6uIGVMI zl@&y4q{!xL3UM#06O6Z1fSyc>bp;+~sldET=e0rql}-uxcqME1;l#P}URF!ycsQyg z<eLXKDbT!Jhine=uUl2yOtbNqlfBD4_UJde_Km;!2LiVM&**;#fh|SbsP1H#m@QgU zZn)bj`Ej0(?NmD3o3^fkAxZ@E@mY0gD(MGnmW*I}ATT?s(DVafpga#Tg+aZ&yCdUd zCPDiUgh7b;^Z6M{){6oD!menRRyFX)kNGPdUk+~Wki2b7Y;2r}kGYMzX+ssrum~-s zUL!y)9#z=kIXSM2h07mW2p3}>DV)f&Io4Bmz@+;T*FQ7j3r*c_-cVZ6MEmX4u0tUH z8Cf$UDn%^_x1oB2uRCXM?>YEg;nD(2xLLhU_4B~K1jZ~jlfTRBktxnxM{k>d6>RW= zpvqos>+tz`lzIkI={k=YH?5z8{=gMp%Bg4rHwriS$bPH|hdsGNBRIDc15##Ja#iJ7 zl3A@2IRV--?6%Nm;0pmLJ#RRfj(meRF-;j!w!Uq&7muaCzzGp<hK%=M<@dV5N3sl( zzJ;D5lT}A6Irl6(q_(Z9nk~9GtN-ju&aZ`U30w3W;Mn>1W+osJ?s9jRwre!4q|CGz zzH4^>CqR#rE&ShCOcY>pWXfgne|27px;fQ`+tZz{_v`oaTwLwUx~WS(()jPyl*8-s zE1UH<p}&ndJ&_ZyCK56aZsw4sf^Z8h!d<v6@J-L3#wl;xVeFpXG;1Suy%>H})gK?& zU28gEBETWjw3UR7(=wVqV<|E6Q>fBV%K~h#P1mt9f&O{~+Lu@8)i9S;7is#>x(de1 ze-hSBaictZTSOVE_jYXe*z2O}!d+;Dmwi7*i~-r4ZpdR_40JWjV&}pQ{#>St=g5?M zX(C{HRyq6BVbay@{xz7_p_NG%FG9-VfN+wQvTB!i)iE_7gex>RkuD_F`JWIr78{E% zHlg^$yUH@lBJydJ5V}+VNbOn1lIuTbk9Nl2rrNaknztB1Pa|wgFEfF94{|CHl_CCN zvwh#7WMIbsBa8j-Gak60|2u6~=IGzULeha28<Rg<*wDat>}5x{wF>-otC>CGb<PFI z;_2a5{~r1LFDX%)5pX|q^Skv(_i}lc^7|zCUFZC80^vsWH{g593IB6!IjoT}^h&tA zk4$?44vXk*%c7R+#!LIO5|_@M1i4?s`(^V0Ux9>vFS?la^n*d(fl@j`*rh!}0XN_J zq0uFuDBAKlW>1lXAhx9zCd>__scWA(%5$9ghoUzjP(?ZLv^L(xxvKvvGg~iV$)g$k z2{@c{R{DDnj=uT8OIySI0W~0z-hgD*f++s;EiKv-Bm2S~%pe~>S=zUsW*iLokljfR z%Uy<9s7$8a$<D3N_s??%?E9jR>HwGDMd0$w2O#P#Y9SQ>|8XS|*xc#=ZSHLX|9+l5 z0_EF3&fR~iG?$qCe;sQ|mSRUHRWlpW3d0|Zd`o{$33!Wuiw4b~eW`NZ7Uxx3{vV4A zD`o&y2!H$o3bXoyjVc4ZX7LZ_Wb*kGQu}!H@a)=G{_d^k8NOq$G7MnjR8i*hrmsT8 zUT@J}u&y(dv5dmuOkRB-<s>7&t5t*X-9xJ~WKklbFQf>`o@q?q)K@cSNp<TtXaBZw z>z>GUiW9VW1ED%0cw31*j}{t$6OiRRs!+c1GmlGqHS2i(U_kVu_T;e1d?*bAyBLoI z%q5A5ErjN6=thMvvqtg|ZKu`yECF3bU#@sB?AR!>KanY{?1GeM8XhDknm~_Wjk_9b zi-RofmF+;fm=SQlAT>GuByzf_^^}gWNf>_dQ^?~Xpc>s`xGhs)Q_JwvvHsN4iVVvk z15?l!3S0#*PyWWU;(zGE{|cl63;sXz67C!DNr8ObqycCZ<xNtW#=}^gd6`By(RSd- zzDL`&pBW(J1h<cNSSb}uJymNB44*j%5o8~2(<4f?6NnsmVyuRRRGf;1uv+%>Kh8U{ zNR9UKczxVXqnMu4i_QvXx=(t@@oTXg#Zlvf9G=oRj(wu3{%8mQ!b0WXu&wCJ8{ip` zjVFZts#NOxL~dVjkIb~2*;i3B3dM=r@waFFZla#3dYy-gu=tRvyZSWRzVQ@4PSX&5 z8J$iwV?&q0thoji9KU`{NLIr#*`x<Ou8GCSGHN64u0#^NKo&$_m~&&XDcw9XPA+T_ z+8c>%g!nvSyOm){pttY+Gt{NAUH48i9u?M2-nu9qCTO$NOySL#nQc{RifwAQ*<*9a zhDgrJwgX)eXaEo12AC6l9YP}XqD^|z4d}id(6X{|vii^(*0d@CI$#7-;$<*a_<vy} zkkN6Stl$NxOqOc@gWX*|{(BV7Cu;xI2y!{%;W8vS8>ZETspwO4wh%p;Kael4IMlaC zatsn?>a)-7v}yWy$ruY2c=lsMB$r*SYt}Ywk6l0z<gLXK7s-m}fa-My#P6O>l`?Ih zhkHA5{48q2;tfqQaz0Tt9~Mue6;@h}!I+)+^eKjQLTAz)7jw?T2t|P<_iD?Y2VMK+ z9m-3+VQiyDb(?vvIK&M?XKSEeI!T}8t%T-eWPVyDDxt+jp57C)k(B>h6=JdE5BMr& zeTB{aW@DC7)^2&A?fnN;1AP-a1gJR^RuC~%cA`hJluC9oF?ArU`?@(ixQyZe7R2Y~ zQ6LU)?wW$GP2g{+MLe+3^ma7hDxFxbQ*SdH^FDRo__qDY{FC_xN6tbG&@gj~^L2>V zk9nyw$tOZ!oIWHv2ihjg;#`xs?(lFH!b@w5{slyysiqW$&0bi@SMD-0GAXJ?EFcBd zI4=WNK>xY3gT>6~Y5$$;_XKBN!!=$Rf7z01VNq&TvH^DIj`Kw6<z{)?Zf|!%d!w4b z0mNO0Z&Kbu&%fIi*vbGqW`OhS2k<xk-<;onAOicP?)lM?u#617=e)LyxjB3<YG~vS z*H$~XDkK&Mrcr?~zp6R3%P;8_SQ5VxdV5hLK~deTY=Ah5QQvgHi!8&Puo9)|@vVL| zcy)1AGFj$~wR-j_;TE%zO9GGy#nt24UTc5(3*kZ|R<TL65*y`bw3Ym~G~J10EOP`o z*ngE)vaK<6`p-;=e7&2`%`1ZjEV5hit!#BOBeuek>WF&5OpO@~*JhoK^u$Kau43Dl zKb$`rS=u_W9;#xI!LJANWc1c_X&a7;n%lh7JRMI-&F=0}5v<`+iZ%R|4A!2@)J+iq z4(6Fv%)i<hmwdZCkMhR_>js!*OamPh`Ik761|G9fG!^D}l9hg?3fr<XhPIBjgg^LA zjvt8uO%3DX*vYb7HSu8P4vqC>e?<+wVKWCSCz(*hLK?3Pf3K_NhEAk;uhEU%a{CMi zBuL#hsI>Bf6;NtX67w6aTh=rQSlhRzgELQs(<~>wOZ>rFiz6#YEQe>3?eocJYL_b< zh_K6gHQ<Tj21`71u>>;N$7nV2;QKn&qWPQNv{rKx+xVx8C$qxQid);TgBu*Y-6mse zZ$gv3M0ocLcodYVtk$8f>Y_o=eXy8zllv!uMZ3I`<qN`hSA+nsSww_n2cT;9^;37* zy1j4T+is2=;1Cm&_?`1d{Y<8@iJ6+}=u;}-N8R7m_3rqXz+wKN@X;IFX5ZD=*rY5x z8KP(C_>b_D{O>fo^8yxrAuyW&!q0!MD*x`ApYPU#t#Fz2`j?TCW8#uP>g*7d=rFN} z=^Lo><#69DNoSJePOfAiT|~sjCQ~RkZiX#YR6$m@!`atK5>9~kk#a((?bk>tKKuEE z#L^CYs+m@~>gR&|;fS8m&c;OgEeJIgRZK=4qLx|m*tfM%?JPe?2Zd$j<SqQ7FLT!P z^;-}_1LN-!lxhbAsc&+UZ(aV8QQRcOinr$ne*aSPgpF0cuB`HARn1>CF0TauoV$zp z#RHYh&@vv*(QA*^y)xh^K3uiy=pS1{)_h94Y+^!Jl*`3)0`)X+Vaw3;6s<cWNkueP zIuO*1A4zdi<{9-du#THYNsjhcw5&PHtR;e0`VcEUayCvbr_>LCf`pZ8J^J)cMQlMZ zDaFU30t2&Znq}#JjVC6_3`{}ASK1tro7yec&EEJlgQ@i%U^2QgP*LnK1Mv7&TQyi= zEb8}AT4+k4Jy0S=Xw88a#!dU2FklBb#7T20F>5Eg=or7a{cd;c_EE1N<?6-4R<GSV zS5pjzohd3gAyX&*4(1ro2>WcrVBL!|#xz#?xZy510OZR3A$A+7>gF=PSSvqCbtOQ) zA|_5JipND*kz7&9=4ms!WL>whtjY@DJCFd#_5T{I(q6RNKQ=Sg@ae0i-LA#)?81D$ zRCr!9q#8!q5_1ek5Z;9ozrQ^#(|F@Wf!8?Oco)PcujePIG|P6ecU0@XGu{|m3Y$8* ziup#2hbIQaQUEv;Mt82kMd#Z)5?@znu%%<ScgpJOB^MuN<JtXRfPiJY*&LA*!&3dM z<y&qyheyLpjaK!xN965zpQ8T3w-Zc&{V{oKDt70h*#SvseVy$a0-{HGrIRKVcXHMG z_k%TVw-aEN{vSGrB=zy~6BO#n4~H}L(R*1zPG{byyR-n4Ag6lRzbDTSaEn8L-dT!t z(x;l3M{ru(;m~xP{sr>8RzJIpTJ?-j^|C!_-&<Yh3pV;NalWqWd(U0*oRZyiF9rmL zZgv0GPi=&hJv?1?gc%iw3X{1*dM8ynC@8BEf6T|(iF6;QX=VtxDKd~e@b8B&lAa)F z%V-T+Mf1K9Mt|`e8&%WGRvWCqYUn+8((P%+Yiu3$e<~_8Bj@aVx9<An#5H6_*(fJ3 zFF^8v*tl9iJ;@hBytSi%w$o!mQkSY@#o90-uifsetb2uujN0EEk&JQ0rG1>T+S=0M zu*m%G^|&>lo9v3*4G0T*e0*%ssr<JS!;cUGe#6e#<iPFMun=6(<UWk&R`?euKW#hK zR8~w^l1eEGq4<;*3paT+9AT*cqVN?qpCBHZ!?X8xU)pett|xCdiR*JiA89vU$m6S} z?1Z0PZm}kYgelQQRSUwXd21hNr8wpM?2>7>uIP7p>yXnmD8eSan#Mu{e;u1iwGeuw zmCTL6;Ix<8rTY)pSXS%lucQtgF&n#&ftKebn~CYAo$GNz_|bnE->Bg~ze`Oa>b;m_ zd8B&O$F~-Cm88o{CqepdukcS@y39m4K^D+|WHZckQlNvUJNlyo>ZL-f6#-Ub(a7mH z%@Ko;v{l=&LT$e9J=D)eGQ0qOJ(^Ex_?VpIr&t`<Jwcp(u75CCh8%NmT_ec|g<T-| z-j7A*iVfE;+cuY)hXUNa84Q!@+CK&#h`6r;<-!^D9;G0oSiF=;t&Lrzo9?pG^8Mqh zZkG?ZgS#kodl1e+Jwn+llpef#8*_I&#tN$EQ_d=~&OcJx5d+2HLjZNF8fy15V4#DN z?fn|kRC<vSa{+GfyPuA}guLBgB_t&BZ*m9-fV4W?F3UDwg;AWV5b#b56Z6uRF7O^; zLbyC0%iAa$l>yl3?jJP#-vUlZPYSSK|F@2lk4tGj^4ozMUe-O1sBFAdzS+~*m{a@W zwN#5USOH_=qF_Y<ExQYjghV7SP1McxHm4&O!({K{6&vHh2yMwHKu7cOkb{K_KGX=> z#gYVyMWhW72nQHJhro#UTW+jUII(Fud`wOl8Erh3p`t1RLMZ7B_{<g60G3v8zDlCf zcU{Hs*jSABQ3wd%!ongkg`BTJBw1n-NH-1;acI|@{p$g(^S>2tf?{bSaPReH$`r50 zlU+vVE(M;po0`N;I@hNSgynSuGOB=FK0t@ftS;*LD_6w5tuNv0u%3h|$Hv<Q7Pr8G z9z*&}4PxH_^+=0|jZb?dnCsweYnI|_#y+D3bwAz;j|itM0e6wa@%n1IYUCv+(n}3< z6>d>CP_)<Ks;m|YwwZ~oY$OIu_4yP*_Smnrl={s-6xe38%Hfw?;r$;l-F<~wS60SK zedB;6i6G)u;M(VvTND{}@P~Xg1%zq!nl(V<7!mOt=_X~bX6({`D#K3oZB2AFMVtZL zK-ZIgRcYu_0b|Hs8&BR=8-3Tc|3}~56unl#Oi%N3wnm18u~Tv1r?p64#cB@cIt08r zP`#J!20JSW!z8j18_&;GR~RS<3~zH!bPsT<6kFe^Mx@}zX5Acu9kPh2R?Wz4SEcrw zM4$H<<2Zs8b(af7t*!HDJKkX~?TMg2jU9lwmu)_kG8pxj&T|=~cl9EWlPslYN}<EL zf;2E};5X|iPbEI1PrdxE4EHd}5N$arc+wJiz`gUhtIe)U2gubRjUu3;CLhcI0l<?O z=TEXm3M>F#a40xJX|6~?7=G9m7xqHE;;qTgUYD~nYzPO~b9a+br3^JcNLF~(7;NJ# z$N&oV;&@G=)$xC>U^o$FmYNtdjP|2KSXsV}PPbjxUP#D}&+Nri40VqHn*^P;`jR=a zsN-c~4>35N6qR9F;SyX65q!oD>8HHn*bbo=X!2qfTJvKn1yiwaJwIP4rp_^tSYO$p zUcB9!b@oL2UdaG5lpF8Jf6f%lpzJ;?KTlCmKPRr$34z1$827Sn4s6eJ`iG6oIAG0G z1;N6qMSLn1uioTU*SEJ=H>*<Jg$k-{oUiwN8+jIn3rb6D^+hAZ!?wbkWL%jQh7CHN zF(7%tSss^hSL(=_c}}Bkvwk1_qtsllXSW;`&IN3DnYGUsd#!C{M(z9bnC|{kb-tx& zPm`90yv=s)&@@VE=3_+Rdl#YicWfD{39=PCremQv<YTwi5kW7|*mqo(Tm8-OnMmN3 zlC8FeGGF5Rcb0#t5P<LQ&~ru(czdtx&nnp5(ssayU9A&fRRZNypqATNUR|9w!!c%j zT))rfgTqGem-$WB$!^kY_6#(Z%B@MiYkc49r$vm{&xrhYy2i!_q%2&>67iq*0ILt@ zoBUufX)(~o?HynBZE^&|0%|o05yPurXWwXvI~zLQyFH(cB6X-$Z7Mz7bd$V2Cv^$6 zAet;;>Owr~U)ov<M+G>cajw=nAFf-K3Lzx0-AkrPxKR(=R|UZH+IJ!VqB%u;ecuBi zD%GDv5gEh3C%N=qFio=}T-+j|4-GcITYaauK;?h-N4+m-Hb^qSoTE<vyH6<TeFj}G zK=%+U^^}_&`<YuHzTn#M)b>x`_O<hCsHYc-ApyC*%PEn*<~Qy90&yf^*c~X-G-XAj zD(ppbt@*WCZihrEtJ+KuW&h)WP~?Z2`4uQKX+^UR^~<RqKnBKwzzr{i|8xQnJOeQf zCZHzj`}$p0i^UmY@eY1PobWPg`ZS<?iZvP&vQZmv;01xhm>!Qk%VMn~MQ4RxI^LaN z>y)lYFwoTOA~Y4ikgktw4l)#<RfyG)p&Haj+gCkO1rnT5i0^eSB}I3HZ<tjNHw|^N zvkaYI`^wqLD48hVuDP7K0%=g_a#P#_ymaCE4Q?})E4r)zsmD=a>exy)bF+RDZ{UZY z@Vj#xp=Kg54xTYTh({9o9h}NK&SGkL_5=ofoTd@L773tb?ho9p&Cjhu4PbwTFK!PC zPNKR0p(|NQxszG|8S%+!?7y~#Mjnwt$)u#I*^f9eV+@19pQ8-~z3jIZUA6uG@wozC z^o02Au(oI3%$fwbrGMv=vCBU+dQRwCoTW+I8D`|YnV2lY>*K`cR3&T{YUl^aPQD<3 zcQ%|JNm%E|=Aa>`S!!jOulMx&v{@Wrfus92x_W;zh2iaRn%{;n%y%_DP&MzLSSfY{ z74$t8E8I=+-h8o(^a)Y7xAJ6_A3ErGV-IRiIM!?pnK9y}=D!a$08#^hwj+lV-+vtx z#>L06iFO7+OnBma`vY!gL#Gnqz-`D{vu{(x>|&O8_I{Zs{ONi3?yG$NL|(nx_&nOc zj_e{m4T%3fj^SVg+9;R>pp+uIv)0!e0PYtw?3{PE*SP4)x{3jBJA%)W)ms!>H7nMZ zS*OuYXvE=(dh)gcOnR3XcD%oT8{OYeu?7eKL@MU?yutImh!*U-O?R7`nW4jR4tDJA z?WL(I7x6SrdZs#TK6X;pt)X<*LPA!mbeJOq5xyy{Nnvt1`Y@;xz0K4Um|Jte<MDZ) z%G}(d74z+f_kZGlnToq;s3HMna8Wv(sMfF;y%NKQY+qAQ_u<c20s&@kKF^PCe%MGu z-Q8r^*omavlqf&u=b=7rx}W&qAy>rtB~A9`=+kx%57S<nC!{5To}WeDSE*I%NZL!K zBpgT|2ukIJ(D2hafJ4w-9<J*fDs{KveLe4Kh@oL4ZYfv6zq`=eZYSWl@{S=v3aC3} zwFj}8@$@!swI`EVz_O$2f0i3p$Stq^ZrBe#;Ih!}t2inVb9Tbxk!b9I=d9%dBm+VL zT+Wis@rDmAa9L&bFibNL`6Z0$VxG(Z*!R9C-?}1Z1GXplKBpT3G@ZLOlf}Lh@?tsb zh`6>XLv!l>kn>M2?fsb#QkI|qm#!Ae<+Gpewj#xVWU^*;HXp#bD%sM6D>e8tkS6~s zO{=-9fT@{8NaohW^(zFGWG@ZHCDif&d09l|-d5yd{SOwYY0cP}LxTzu2{R%BsEw0w zp(d|9tOA`70O(Ilm`;jR7ll4ky(g{Vh6V^d7h>x&KnJs)Urv`Q$3(lnU3&|S@n?H% zGT0KR_p0i>)~G5|Mr|l31<f=t&2d*8`wM7R$6WjFder~4!EI}1>AMco9+(qZ4uteD zPqTcd|1*-lU4PjRZP9w%*s7bFd<*=Q<QG*<Epo(i?(Lt-W!ts!II$fc)f6om*I3J> z)UvBT=wEan1CjT$O=-v`2RK{xf}J7c>_Z1V3rkCQ0)mf}m790*ob&g{XNp00R*QDI zb>+_?4P-b$7ODSyLjhgq_vg)nJ>0z7GJ-+*tRzk>pZU)Kb2<hfHmhy5qX{mv;5kR* zHl{VHbLLT*XJwzeEA4GFF!jn>;^6SK>TPrcg@c23Y$hk`Q|^(so&&*CPU$#UgMF1L z#LRZHnuhB$AsA2TLEk%1%^r~LTNiKED(m}BTzJe(GRkjH_HHo{CnlOjk=xp$7cf^@ z4}`s&7oE}<;^xkK^6-m@BE6Es7e}lF;3vt$s|p_J++?fOXl>N$aJ*IO{5S6vc9J`0 z@8@A>n9pQ)6yDDc+3EDn5iUCMN0-VhZbUfTm6loclgBWLa=-qHijed+jC;r2;f(kD zl^UL6skE}9ck}I9!N8n-WG(I@hPjgsV~LfC%;+2=avE$dq1qtSJN1Y9UV7X`@bwgl z4-pZA3y?5)IJAFD{=L>%S}L(W^9mt>e||VVg}H4u(qWNu@bx?EF##%yurpg+{Rjo= z?vLzdcI0^F<voDpUH{}5GvbRTADga$u``2w$#X~9{%+lO?bn$qE!8p<;cf#cI)qAh zYtIj!kf3NqB~`LN(afmH35iH^dE$Qo)st50iPPKE1+|p)w~wPzciC&%7i_$odJ}I) z2a?c94O-=J#K<-6)nG=rAX{QpAT@jBjzfUkS*)R!AG2oT>bBRasxv%o#j{!r1recN zr_-yrtV+*`$d2OQViiZ95(L~)!_2E2?!k9-`qcexnu9#{wcD5@o`}_TFm8F;$c1}| z-BPW+bvp|-<!`|36jv8sEQ~B#;@@cA@4&|Gf}?&?+|UUAm6b=Nb+}%v4vJ(?F`DrA z7=ll_kq<W!rDtW@H%nDw{VEWAez)YD{w_Gl5<om@&gqtwR&NT|OcJak0pT?R1IQK7 zY4`H1fU27@N~&&G@EYylkgt4+Tv5A=F39OxN1=!8B(1OShKXIyAIVBtRgMK389y|a zj#rPZUpSbXcI}BM`ee-}G12xWME+@!Y|{3||HnRYszKIfnaezskN4PMB3&(ZMMB0L zn7{o&R?3a6jI=q(Y4U=qfJR=)jv3NG2t>Yv)bAP;%bHLts*?rZGEB5A=|yCnnwpYt zaS|xuGt$!ieIhH<DvsS6{RMm{thCUO*+}?HtD-o9r6U*Con2aR)3h=~PxQZ>scqsQ zDcQ~UIyOH>X2Jwzg%27b2RF{Y%f7_qg5N#xwz?W?9&IY2B5QwnsmZb6NWbi##yV8N zir5QsdmT@K$}S0)^%cg5i<`ZO(CehKL&5yacIfea$mlYnQ-fv5?CmB}If{atB@;(R zY;noav^RY7=*&R-!=i&Ly<FKd$mGE2D-#SB)tAD|RyW_k?}I~!m3r5@1swt1$pW*6 z<ezu6hRXFOZ8T+1PnuYIl5zPOEf}k9oAudvIDGYwlYOrL7>O;wX`jF_pZ`wmJHXY^ zthM!6FSncYsY|*?HuTC$n{i#2_Eha`+aCuh>dyChB`=oW0?T(Y=lrQOx$%7Q>rTc; z-0)S5Ze86IOaYJ6zR7d>s&wDN83pem2POlY{a3p`f7)kHX6lW$pIyAYwY9ls9Alg= z4wtz4s6I<Q3*Fg_c9ZH@zN(Ddtxpxso2xkOMcf~cc~m2Ru8a5u`f+`PoLYpnh0*qM z6xG<jgSyjsX`<0`k)KSYIe(c!X7HB_P=|V-Z(x<NyuRE%L|6q1Ly&0B9{6+|A2+>7 zV{9n_jdmAm*~foi(|xiY0^hH&$Rsf<!IqNUl@-U!rn=9_s6LUTiSUo^>ydtt#%KX7 z8L^3zs?96F_#}FGwEOiv+QkGMNuKujE%#3}4?<AzuCYgR%2vce2p}yX1n|aKEAA@a z&E^wk2RMMR0~O?^!mEFeQ`vj@#jUwk0G}YeFYiWJAHF<OM@m}0yZs3*pQx4}ePxEy zChsw0%7gbmD;3udfS|){5OMAQXhm#m%|Fk20A3{k;bvv8!6E+LPo%fHEQIprP@oy5 zA-3_Y`jrz32zt2t%fVNCE1uYX(|jVn0EcVO-znpkSlF#<oIV3H1a@g>jc(s@-EA=Y zMY$>8GV3fn<hfgK);+ZtoT@R9aQ#XSG3|Rah-`(j!XYOR=^HOuO*?8OQuo(WfC3f4 z##l!RXHkmgpb5q|)O9!XSgin}B6YRY!mke0%bfKtYR(M3v7gwRSi~aswBA&*6Tnxy z9ZawLJ{{f!>+OJXQeImbJX+~FX+D!seYQ2W%dQNaYRQ;pFyQl4zj>w(0>k2Ar|?Wi ziO8Ui@I>gAdTkW-CqebGONQSbx{Aq|3CI9<%O9%*-UbBg8Uq9rly_ToxaH}~9G_a* zMl*i#nk?D_JY0abi>^x&wvr_sK7R}qR0a{X#;tf^#PphES^7f_IkJ`J|Hi^92tro= zl`!qPeN#!Lmi^a<MXHe|l6A<pxUovl5bc61*e_Ls8|sPc+h60Jzj!}Bd@qWeKQs+? zcwg&ytzW$iAl<pJPubGnulIcL;XuezuRyS0PE9GIjt>8Qw%|QA;Oo8!SYOe2Y~>Fs zplIt<ij7{LZ3-kA?c!ofPg%>esbZ0AX6^Khf8LLdQ2Tf2KJIIcC6+$XGQt6!w%g^q zN~15sL`GBBP8C!WUpRP*+~=_vzj8HPJZqPQtK!#5T|R3^gxk1>SBm{0^0J*Z+`g^J zcI=GFyX|W(=N-s7;9#T@2-RMQ1<q(g$7W1^uF3ViT-~}X?Mg8nT|B-<@X0QB&PJen zWXbZmex0DW#tFH!)`9}1WoPAIyXs=Fn}myn(C9g?Ky99p5e62?8x-EnZEGl1V0R1b z^U0(iy_4!4LdH2PAACZxokjcn>MFDT&#sus4-*Fuj%R)kop1CL-ae`fd&Gj_J(GSf z_(1W`i>YP@1Rk1A=*4)j8KCP(=ytkD`tMA|h?ZZ#pF<;WT#AZ0mp8@kYG*iEW(B>> zEFQqaewfqlScQR|hcG>@d3Pt$EWT5k*Url2ch3UM__(RB$JhfBA34uNPeeT5S+X|_ zvjQL3cm8_+QypP}^Xt~tJb@+X?R%myE@1a?IcydRgY3v3a1niAWp15h1Opwq1MvId z64X>9LI@b?5Ofi!Tv)gmnN01THiCwYaY)sbG^uVUQ-UH_Q<C*anT!DUrvB3U@^Yv_ z`3w;kIieP?)3C_!kE7FRD=jvmnNz?)+t)Vj$prL&4-2W88TZd5Stsj)rP&7~*ykk5 zPm&@o5fR`F)3Uq%5_($XuB{qwZ~)j-lq%tHg~Jd>QLfz#(^R~}inS3DUdpkH#ut7o zW$pXQQ(AFVEksXhPd>E96LNq1F=UzYK5s41q7H0JfB$JxH&wOgH+D*WcT1@ex3aB~ zbaHcmqNWN_h61d}^ocFRGusjG!kMaR*5s-MS_WHUmXjh!&SvC3IR64$x9Wc{ubI1j zQd<kQXflz(X=#&O-q_K~14u?8OfHjlFqTuL3(a4HHcE)=Q0ntqS_uIIvf^VpwG!n1 zB=72_af`{N>bq8L-o-OSt9S4G-+dDkRzZX9#B$3j$aKbffo_;aog~Wl9Ye*+OY4a# zc+9MSEUxd{VFBp`l1Z;MT(ymgTgkRKGS=)%`-w}{6NMAZb<jC{&@`%wNr|&Ib7uGX zeFK#Scg)JcO@+i$W+7bPA9zQ&n_9bD@G~e!W4^M?i!Npgzn;QoZQjfs7kS0Y>#y6n zK;B$_h+Wp@c<<C8#+7MSZo$V;RKT6%MDT@Z!~H}jqtjiSiu{vCGiF^&ZLN%U7NLME z=3pluMSNh&z1SynCc6_tuAh?qe--RHnDt|l(Dkxg{YhR=Ci}=^Y`=W~WZ*A%I_%-q zImJ!be%?Eb&2HRXFCr-BM@c3{eb?y7XDuB*I7Te|W)LG}fe%4&;u&Vv^>04&yRg9T zrx0jlPur;+Xk;d$a=f6??wo~sLS$m-=B^xKqwi(ggx24>at?67;qRO~%acz-4%IC$ zOVWNbl-B+6O;bKP>ek&^y!|!I4;LNwOTu~W3rq3-jsp2_hnIbuuk4(^^YSp?j?P~R z($ckdJ>>1<6XU6w7ad<^(YQiE8A6JRmrl6%+}zFge>-1L1v=2CRt0lFnPGlUdsa<5 zZVI~IJHF6@gYw>#Nui+~Tp+#IJB9n32qQdzPwFi~-jglXV_N(7*CZp*JINItFHoMT zwZFetGZY>;I2~5jlWFU!WJ*@crdF)Jm)<Ms$VDhH`FOud9XeH28Ge0{?)Ln-Q<hpG zyT7*!of}tUyg04aX=7ruE{~9aL{h^8=AU))KV`vDE$-*6_0H!TH5!#`kBE81PA{~J zhrdQOpbOw=?%#Ylc#HIA?=<sBN>tl^QRwJOy-tR~-{1eZ9X0|9XP(^J-m|**^Sf8h zczlE!E9>LplLA!Ozd*Rjxe`X{_FqKI7r>YCdVlARobFW;Ynw2C*u+%Y|7(~$fjZT# ziQlg3)QtuPsB=60>{llNI*+YLU7iI22l)g#?ZTJqh^hf{<O=N-Nm(h5fo8*&ajq;4 zQw*{;?TzQ>BTgB^X_0%Jbv!DhIpyok*7;T1%Qb@cK?&+Tw~ODNd*dp-t8a!migsL= ze*ikIko{LruEJ1OwlC!kxTNKOF&uf=T=#euutqa0$t^Gv%!YfyoR*xDZNj$#?1xyU zNEghU{f3&%ovtgn#GWWqIJ#?>4H9_TG@Wr?_Zrbv4yvv!Ke|=7Qf^Q%evqT7#_r-U zT#j26T6|~Zr61$SBu`QPN@fYJ!%4qdYpaB}S1-gx>cl)zKZtYcfAThZlqdbjNaL}| zQkjw8j1V4WL1-xq7~5{+U<x!>2)On1AIb*_FKk}Sy;G9Al=_MZ-EIwHhJbs%9!h^L zVV5YsXdrL-n=cM@j|ny5P@oKwBle_7@AJ9Q`!k#EnXK0<7^3GV4peJBZC#9|y%iSF zu35M_X<Z~6OBRGjOC#djY!^IKoM-^f7hX0IWKBHYFr7*rQq{t=5VT=CTPe^B{1e+b z(}wlKT9K!r_f%wWMXU6v_XPO(05%^5w3EMnWzUs=ndHPkAW1RT8RGo>&|nac54h1t z=z0QaUtW5GmL~)zY~2C*^9$i@w?r-c#w1c!eW)(F`<dNCM@ZS5zta<Nek%?{@D{Xx z2(o-uQj9vf|H`1r^FI_MA~BYsgBRgqox5r8v&y^E`84<TGxuMboqirZ(2{Tn$XB1l zCb2ZDv_^C~;ZTv89DD|xBJP|49tM*RoG@X8J?h%(BK0Km-O=5KfJdsUp`p@#org!r zWW`OBpx~GERmC4V^u!KaX2gn4CT@D&o%BF1-$~8rb#9g4cd%B`{^H$%5sBu6gYMXQ z@(&g1yc4B~y~jtLEZ;+%IEVYblgUWsy)qK^W9<yLYrF~xECx$ngWqF7Q_KQ|=0Wx8 zE61l@cS0uzkgnP$DU(E*rKO2Z4geAeaN6zM56?p7g0lJkqq3y<B2)hHNU1KbM#-n> zD%ESovf6d`@_v2f82`Ora2R)UGoI?TS0(fHv&(H%O~;fV3#o^JZnK@q5B4iW85my& zpRF7|UNtNaxJ9cK-trpx-n(w@?-6v&KTT-Wz=25t40e_0O?s6(EAj=U(S|<?8BTys zWvZlgBcBUPldywuOin?cD$Ql%SP)bEwwmx+8x|%8YGET~o(JHuvS9vxSlrL8*$E@{ zoqy$a6PJ8M+tkWic01%5MDcBszuxP7ncu7P)bEd5=h?E;T^a>AR&~bG<y;Cs>PPhS zO|j&c^|UOG*$vLtqjm^tFl58)od5d;%(-0m?=7NXvmK9!5GI`W=+I?CEcC=#q@`>W zUwYes801b*injjNkgo1^LmpQ|Snfok#cwip3KHr*@3zw^VgY{c+63|g!D^QU_Kpc% z(|sY0aeYyqRi4pid6an7*K70m@*m%rKdXgr7~CL_pGryCSD^I}1WpMommWt4p;<j4 zBCotVBGepAFW9hzrH3B-oMtq|zD%b4_bf!+6x!h?tX5qz!|hzEPEE~Uctu>^tc5Om z%nVjRws&qOzhw{$eI7mXlZzj(D7D(R`a`yHlU{qG9@+OaV0xiJcLGg@sKfN*SdPI& zM^3!Y(4{17WX;^V%LaQ%=wUX-gJS8=LxCLE<$I&oHS2P=;fb^|MYu&7zoiI8{x0pq z=ci$@7!tXHYK$;Cr~zZXca1XjJ0LL0>XfjRAe5Xy@gZ&R>GuAgpB`tK$0A|B)cDo+ zan74xXcidHi>wL``muaCv-(zt7fhRgsHu$8rR7yp0U7Y5GOVvD0i`b8c1cYGD|^C= zyRA`6lMnr}Ud-v@-#G)e6uWa7`%cm9$ql_45=h;_)AeCUfbg9jT`4PWV%U5G(?Pp! zB5csW|BR2<Mcp|LCRKVUuu-%qu=RC(qM-AdgdMNam&_H#JDbyFtOc&?cLxyd4ia97 z$9?L#&FT5dM&lx9<<5}osYR`yD<R1NWU#8%r3LIGch`9@-|cgQEGzv@+*~~$5d*V{ zFs!7~_k+@Mciwfqhd^O^SDX*wpwIK@d0Unb@6*lLNgfaGs_Mbjh4wODs?P&-Yg5!7 z-5<|*4Loz@astqiU4vhvVY@j?(+3BCdt+CrE=>bB;=+TX#!Oe6&hS0ytwm!CAy2Ki za^9qktl+jY*{`5EgpAyx$+4NQbZ|2LtxR<jdRt=iLe1{Orvnkk8vV{d#H148?pd0( zq>3o~bJ=;<R~5$DtEh2}@BH7x^Rm1X?r`!n_PkVVc1Rz5G1^@pi@ac1j9v11&NmW) zDJ7q%3`<ofM%Ma;h+kO;YDqnV{y4<OfCRAeyOoV@-*Z`Q@ugT0ey>$xgAnJchBjbF z6$lE<vQD_c{*`=W+vnW`g*+n~J=<BBhGnDtcIJ*h9Lj+<hEOX@w8}b72dU!~=aPD- zUUb%K?t(V+6nV15YL6nI7O3MEC>+RDnV3>&4I;I5PPfR@m~0!AengC&&SE%uFA#Ch zVDZTc`@3ZHL4|Lzkh;#YS_6h1r=cc7@=NJ?X#<O$a?aqb22zVt%t{8$>`@IvM*mk+ zxBTI`B5S)w>DuYOnl0x%GAaS2FMV1Ii8b|~Dq(^&pWY)=>DO1xdQ@o@08{1PFd-?Q z)}L~Zb0a{3MWOe4v1BoqwO}bOC@?XuCK+&}ouD#CQ$0DAAdwrr&hOqr74xtC(?a?# z>f~k~ZBbe>CUZ0S8{zIQ>BZq%SoxPkfIdT3$f_a)O(tug!1Qc+MqbV&<a0hSvl1t} z+`Oz`&D;0rhr?n{(qx0jI*cC(`p`(&X6nn`^;)jW??vW$BFhLGz3~N9XV8)SRVfF) zRwUk~c4dzzYwtR+Bb)>Sh0QQfF~h_;ped^8v5c;9-hV&Cv<aKyYiGouY==o%-uoK< zpFLtf8=|YNobMiY2HnGg7RkxR$uPSIJGT~JV(I3H)+_?;U)W7Y+o^fWSwbnn=ka9r z_{Zr_bNd<sESN(wWx^u#NW4AK+GYaE@+Td}Dj@*fDIU<Q{sLo=5ft%n*tRPdh6@q` zsD-4f*-GWRW}%^Js`;#^-xhVgv#uE&Y<#@IVJ-S9r3Zo6o}t)(i|lS0u^sTI3#j=C zqspJqnm3FP+UP2Oo3+UB9gsA^H?6K9qrmDg0xOpp81xNZ_{jLmz|i`yVQKO2^$B=K zK%B4-oOvYty8v|=d_)6gjI&tR-N)tT72Dc^+-yU$t;U<Iiv_}0m7c&?KnA;<v#$-j z$QRyf{e@dovTvUzUB`UU1Eg<-V4h3QMslvN!)`~e9r(X_de68L3BQZjH+dI?O$NWH zx0~=M_#?Dix-g1b7)P%Q4R^oot$s+?#d_F*!$-WjF%*hC0r<k$W5xhh16pvFN&i<Y zM4Y=_ilzZHlwPWXLt9*aA`di7F{k6dHw&M!o%foo<rIKfFHbRIVSUZ(IK{jCGUpn+ zXG?4);%r2tDUp`pW=ETqyzz4yCAXNUw5MlfVYXss<ESKaF*W?AVOOU@bcDm->eGyj zch|~Eo0q7zGDiuwL?LJ&O^mR;H`R1>EvEcsxWR<e|H=gC#K7AD8y8W2#PiQ>5<c=c zYLhXFidK4VcTO2F?JxgLeW3RkBy1!N44MBLjQ@E7?{wSrb?bR5t(CRC&mPgLk=Ge? zr%zJFscsml_t1UC<^JTYn;GwN#^pXnds2u)o$(u-s{YpcK8U#tUa^Im7Y5rNvG*WA zkAeB*>g&iOq6s6*L5ZG51~U?#F}HnG{dN{#AG`RFbHj5-T}ysXUa-k+YS1-<XdqGG zS(J~}sKTxv{qRk}v4Db=oKUY6xUw5=bn~NrOmntgrg#Q7l!?6VAME#pqXw1^N(-UC zU+i@R<vTrFPwC>uFVj7Jzy(|dwC!<69@-09yK%H~>oBkuRv981Z7_-LHVymPwTEC{ zHr~iJkgRci0fN6*ntP_WN%}{h{xEBJVTkjAGQWX_t#&W#Z+3B^-P*9l-{7g87-ia7 z(*{g%z0FkQMH*?U>Kq2>+81af#j|Vjk84dat$fLT^G$TKr+XvqIezI+zxG(M8d)$k zXF0UAU@`XwhIaM(to~HWjQ-Tpe~s?|^;bq_6nMuTiIvES-ihX%5eKa$b&vKF>E4`m z11`6bzi|4Ya8m<w;i_@77bc!DaZxgr;#zvNKb>7U|5CdghN|pX^zWQ61R?9(6Rial zZ#DkHzf6pD8uFT`NdkTdNEJtmYe=$6uS9%Ka7*mrn_;jjzLj;&t0_Y6w+X4K$}5x2 zfgjzqAopgC<{WKF-=8kj5oIV`WY>?$%9oL{i1=}v5^?#!8j$>)3sp3}cN0%Z6cSd9 z(p?|t#HHtqo%~B-h{rM5KB9~n2T_S3;<8y$qn0`63>8N;OHdA^KebQWOcju=A;2Nk z`$jTyGihA~<Pa)<8YH&1iKy`T`>H9Aw1SR$t$qzEvfaCxxv^~5@Kxk12V4BD2QOBO zTEX@-l)gHyF?LdI1ZEuHTGYtVuToltB%vj6oo}d)?ZmjClezes|8Q;33{CIL#FOwg z!mi<&Wx0h1vAx`=@MSp(E$eK8Mh1!j!=i*z@J<E%7Z}oI!bWgHEpKea_>}(CJn%H` zfzf2x3e*p*z{;rp?~(MM7x2y&l_5S?(*~as+UCeAoBCLQPHQ9_EF@a}E(;pA1n4-_ zMY3;6EW>ChOu}+y=|E!!efc0|`%ChKYD1i1A{haB<>ou=W^<+4_b5&gs~4Nx0h0Ig zx=o6{?;A&%cD$!qGZ=Zw&la`dm7jvjd}8JZ>2-f9&?{7mrW%6cO9GEz8nmW$bzoo- zjz^!QV^*_hH*_<xY=%yTe){e4Yt!(NvHU5mB5cQ6$IB$@?(2azO*S);9(u_us25N9 zS&FfW9e{BXbA4upL!v4}pYiIBI2?5eUx6s5VngG&!q%5=>urEtORp-+0P6m#9p&$6 z!aN@kRuU~F>{i(_nARs#<yMN+yC7^rpc=q0N8vK^lPR~Z@glX5--PH=yW^vqJ=t5n zv<%+N*f+XK_QUVD^E?ZIac59{Ddgx%AN~EM?Fc7d@DqJZpgy{hugony)3B{7$1Y*g zL+(`#wa*%ZzFn@IGE0gX{VC6GBD&bG{^F#X@@Qpztd#VTY$>N64Wp10dfqd_tU0M7 zlkuD$sGgbW9;F*A;){Ty^8agzP2uIWJ7$b?BsIHra|mp|ANXBwM`=U}A(7(Ux6bA{ z)Zgv8OZGVj*8u`c$hGj^J$i$knMgKl#Hj;O=k>l{5-$3{&O7t*@07O5dcrN-;%#n( z(Tn<1K8D<0_;|Zrjt~{Nw$jR6pwd@+pnBF+&%{;8_doWkG0m!VUr9zoUbz}5!9VIA zWnVn)h7A~CWtj~nvl1*!qJn{D+qhUGtxVPv1M4TsZZ>7A+d@X{a>}cu1kP2<In@G$ zJ<na~wcv{{dmegrE-Kpsf|JY>AjvsyEn01|2wxGP5WMj^uG3aY@^l~y5QMd_w5{H{ zmz3i?cH$;_+!NS@sxDgLExDGvuBPObw9-ORYI`ngir@y|C|+K)w!kJBvQ`iVT#%rw z&c0n#-Nm6BUiF6LVGWtftXzz(m8dTSXSXY_Lsb$;9)Dj;r&e}@U3?_XUr<;W+P%rw zzOG(Yie9LesGc^voS+ih<xY96h8g`iNYQQpwt<-h>W$CDkqjr5!}9NI06XVgk>#=S zmVMW_gN+_O9*I@-b-cjzkIBaP>YK;BVRpaz{O4=iVTk%Yv$e85-e+va|6WNxu=(;8 z&$<39-q#LbR4$zXn<!A5nOpz&Kz2Y5dqLc{tLmMFEv&CwqivWcG&0Rji|knzaGB~J z`fK5B0vKS~>Zz*>>FJUDk{Ov=U1Vndwl>k{;s(iS%Tof=i~GI#8;>aR!ST`uiHe%r zoyWY#wB6B9FR3KHMTr7}2z^ge#{1q2igZ0<#CIh&4M;^!Xk^g{{XeX|Wl&t}wl2Ja z6C8pBcXtRb!AT%U(BK-}gESgw+zIXw2<{%7;O_1;?rx3T$=ct(XP<lY$4Awy^(R$p zcK4j`7~>g_pdl>N6%LXRl*Le6Ee)elw9<Sg@2PG_Zqii0%w;@akp@b-`iUriaMy<C z+m1CCO5#ez^<AXyT^6Y%?8!Hg6bJ1IO5pbqq@Wu{yU?K73y<dp1XNBo|IF<(+9_?z zZY~=mLt>|5ryr}2aDUZfAI)(J`YlDGQrCC$pLl;QEiE@Rcf`b+rZ9q}F(#0^x?5=F z#U5qs1DNjXtw$-&N7|NF<YR)M1CDc2q7;($HEYrPukKxg(R#@?5SA(*CZc(T5+{W( z0|%$>@;EKtL1u`i%O4dw`I|T|%$EmgDhJJ4+V+y?43tN6wb%f^-z?KyWLT9)Y(P6o z<C8FNXyTgN(RR=JQ$G8&_*~pb1vN$Hpc}<e&`vD$PE%8}>+;VVEoq|Y#bjT<GHc&* zPlQ%xo_Y-<kOQM;+3yCTrpm$p`OW3j=h+EqLAiD?U@65f#5a8w*ggH(Ik~Wkj}Cz7 z4srvG_QxBz&F2dzF)_&z{24Oo<psH@Pp`yIudjNSJysg*?+CRP7igSZ##z$W*m%n^ zUv*F<Det6eBo6?N@$+Gc0Du(M|6wuZ!(s@mr%oGF4hy{U6@|7!TV6*NR5JCHo4wBy zTmH?rovaEbR}I)<I4#)80T=DhG1Cc+44wuKglMR?G}}JkY~d;+unI`&@eQhzkt{!J zV*?5|2ED*hs+pW(APZ9HH0ZTF<m=IxH~ygUR}djCEKRUAs=LT;(GyD<^H1*z^hA`A zEQ{QyN~k~dE<HM&6qLFu5pD{lQ?Xgm$1<<;es(B0d_6z1lj2lGOo!iNO44<B&23Vj zM+LByClm7OXvy1{h(cJ#Wz9pxr{BL7=vh?a(qwp_R#zSC66bIWN!yAyeU+vXhO?g% zyCK{b3lUZuwc@SgkpPQn90d4v6lo1f(HYjZ4F1b#^mmW=@Y1==A7}s_;y|Cl|JSm7 zVIxN6d?nFI1#lNSBeY#lQ&-!<cIV*a6+Qtb!iX!auNpv$C;?L4FsK+L!oGvEw*BP6 zx<V&&w|pBoKOCOU)xtHI3kphlVjy%5I~f>SL!}cAiwc)Q8|BaZM_QIvlmKF(jmm1p zzN;`!D};sW=+&_A``V<2X7EkSsKA<hpO_}L3R&d6SnP!@R@kHpKwc>C4}JRV>)kS` ztJwZI;Visy$7{e2MJd^5;9HnttWET)v%#q;{$OIpw@bIn@)jnmnIm=P#lipvU@ba0 zRTPVYqd1ixBgJlgfl4d})nwWavNSE4+F0?VPJKaFX#Yx>jI7d8pRM;<;SbxA`|DSx zg$L6=8tN;`j<t`vRNIsrm6o=yD+lT?-`PK}`HmO>qljzqen-Ha$4xLb0sNg)AEB`G zy%T0REvv1?^?rJimH*gr+XYu-WwpL_6jT##z31kjZKBGo2VG80O0Km1*5$Txc82V% z!>Ewi6#Vdq($i^|fK|WcH%IO6WY@bCzFjS0O<!rbba`l9JaA}cYUYz7B20_A?_nNx z^U4@H*qkMwk3jg3Oh7&yKMBPRDkJ%1u+HLu_QjNO@Hf^MrU+As6LuM_2iH$E5QV#U zm!p>=^%@yA(fVL98Hd*c*T`HjRww;$GMaD?zIF+OPvyaN@k;f9gw<-S7Fc0hsG4X6 z$9Qb<G#j<YUEUEadCdPk(3RpNBWR=!C!qv)S$b7p=+y($2IQaD0D2^FuIC+I75}5{ z`n%hr+Y11T^%|(1fhEM@P$MHF0FDzXhr^k<x%H1n9oWKx+&68Y$4{te5q%P=8=&4E zI!uzp?*N$)F4M;g)8{!yV{C4!8Is6?Ugy}@k^IZztaU-5nH2GP{%+c_RC6=~2_bPN zVT@<S@AUpJ&BU)2*WVAeOON`PLLC}rG{sz5fl0kf&Wfv2=+1AzV?srsvG}04&q7Y- zqa1N#z5tK)$69I5Peuwd0zZbv9alxmb3HYHFZ_*g<?ko5qpF(vG0;t9?X>}JI?hqB zU9k9xWc`J0>CPJ)4RA*b{-7>cZ1$kruS0zZ6lCx#uj9XY-GHx&g=I9_>mgFEXs-86 zZ&Q<W(zvZIJ@ZiV8X~j>GI_(#$thErDNkCZ`{=YfxgivGjZ0wikhQjk#;&QU;|EYJ zxVd=-y<6o~)uz{O>{^0Wo?bB^h3=3L5y`x_r++#|6zz`EyEF5=+`|W0#i0E`?kH0V z_L9=_4{IqZ*EXr}dWk!xK^>q>^ih1xI=FVv$~Ci{M`<*<VmTO*2dfr#0jmz6o0I+; z#L*^nBe`_q*O;%GHCz{!3>=VwDNjY8Hu!a9ocDLO@i$F-K$%#u9R`c$C>ubG#Jqke zkiiIqv(=h@`{eSc_(32=hS20uV8_@M2Tv`vqPc3HW2-5ay_zp)pcR#(_Q|Mt1nP=i zBBx&1mPM%lTQu$F;Pk<l<dw*5TKz$Vv;A?GaIsr60=R6p^)GNOA!_3be4t@{^YVx# zy+DFYzy$ydwKX;2wAUX}w1-~!nr0dp?jBH}FF-7Ib{SoCspBQa$*O*{P7NA3JWTU+ z!K>#Nr|=uz+p_?nH*qZs)q92O9o(6e8#^-3WVWnm22oK_?H(U@?CuuK?S6G0xF;ng z?MZW|bLp-Qb#6RqHxU0ETJly?PcJ080!dX>ZO|J}Q_PU#=_;Zgj(7p39UTEIP6e<` zGRe6F5fjVwdP&MX2nY#RChS2=KN-#QW{Yea-%kG}KdQOK)@FxGU}og_8V4uVv0C6{ zcAejk5vr_~7^^a=Sc;efw=fphs6Qm})lFk55L5vl62TE%h9)!lK6VLTaUr=%t=4Nn zp!5%Fk^5lG943m2VyPuDUPeRUV^T=}2TfJjPA8GytHFkN#wrk5tB=y*3@{OoSDzDL zA&<f3QPN5QY=N|aG%JV9MB+P;ObU7r5)B)eMFhpl<$U<M(paw8f}`?3F(qsq-289{ z#F;pPY*sGODTt|<dN|>yMVR3V5yC#Yd#7$n*8)awaTjYvF`VZ_DsZlw!3c-5Ko9Q@ z9GL&rm^>(Jv}nUqztP^&d3#%3j){jA0R*JDLGL?NHQLtwYw)Y0kmC~Kb1Etb@CNAt z2OQ;}txgX!zgYnDjrOtAQv@VyOIv9zY1+od#(ER7GR*n@)4pKlv+Xq6()6$se*dv? z4hMjbE~)D!tUhav{$~1JZG*!BPb9G*dbx*4!Mo`YNSv`g`0S^Vk;7=OhPAW9*?a=! zqC6Dv3N?m9rKLsrR$Ok^4tQE(5nxm75EB#oC)TrhpDhqMIyAZI5z_f?{9>Bjs{UPz zYu9#z^%JteAtKUhD#o7|L8^&O>b1%^CyelxROt5DjU+>~;=zU#Tx^s!0*xTNb?kV@ z+Gz><0`S{ai*&3XvPI7kql7unxkQc4kaCL5QtL?C%HTZ|H%Ct<`*tOmQzsi+AA}1( zjIhk}kA3YnwjaI*Y8x`pv-$JFZP<3Jot1=_pmYTI#S&zadF(&7PQG6>_2^BMgtEDP zGw^UH!wEBP&<4SqE=z5c?h%_6zi-mlOqcoU!Dx%8Fa9i99#`!oM3TW)A&+c@j8B{y zB4ABcx36)iDx?P-ZUB@7dAUN5GJvx8<^nj|9sX^cR_=01y>W9h;Rakv!|wxWL1ZU~ zhvE_vRJ63Tx#fR$2v;j_S#8Nyt1G{o?v2hRwiy`+c{~R$-YtXLydJ(|nj1P;7%!zR zg3wrV@CfmIfwVcDW_O9ST!f%NQmZhxh)FZHwV8B}4YeLX{4~L_kHA=5)Ri1+{KMsa zUt4qCqptEq_JqMG8Dw$NfQ=|4_AWK??JsK^iu(Fm*R{yRsP|LJZ;N|{o^@FMH0Sgx zLdd*tUwdENzr^_dsQT*lbbWKuKDda%|N9PmXDS(`pp3)DEwXAwOMCG&^XbB;hwbT3 z?N#45PtZ!^oT~vlIXluB@`t39j07T19@bYfk(qxgRm>djm!z6+#?9!NS>qF>f#>5- z8=vzM@bIwn^V0;TrDOzX_}nBA9N*shOVnj5UgPt(mRS8(uYpz6Hje)6`q#F`HbChJ z9pzdPllG{RZzGr_2>j71UTa6{R9v0(C8OsN@nARexTAKW8j0hMV(88Hc0#f@&O3${ zUI!h<cy#)6LKYEXO*3lASB0E|fVZ?i13~|=1AElF6LYlQ^w4V{Ogo`6KmO$U^UdWx zWf9^4mO4>z@-GJaqmK08D|KE?u)PX-)xu}?dRYz7s0IG@$C~9k84{J!me~IRQb7!f zA8K}LcFs2pI=R-;f{OVwA03h0_Op3MWIpfn0s<6TOH26|-`cT>*;%?3S0KEt2k0>2 zNwGu62bmb&k?gSPb_1*b=(IFi^iL`(lc#w*8WP~|bsIlE!@<Gl<krXo!FxHzr)L;{ z#!iFfp9!#V!Nzo>YrFlENMYgPK0cz`+uHzLEBG{6_C?9L57^v>1^<bLCD8ezE9NCZ z<$XE8c6KA`27luT0~CfR?)Yu6jN;nR@30?b;Gn3m8w%lmcXtoqMGs_}wd)Y2hvNs` z{ZSZTVth$@Bj)=?+>yP$ySEqbxUQNf)lCBk_lR4C>@~L@UjxI4X-|*fjpr#Ovn_OX z#SMkS0hti}Slqvgd9)}+E7#P7oSGRGbQb`F(B1m^Glx-SUz7fqFYwLH%`s^|>c)NY zc=vq4GN(t`IefYMLTy=nZU)gL16r}jU#q`xDs%z|&u3m$20#j==9?QD<M^7bH~fOj zu6?GCC)qo0Q`aE7@dKCB`DR`gN?EaqM~wDG68`U&HwP0q9ptr*tuPlran`0}=dts3 z1`&1(zfkd|)IA8XQ;nnNume^KStV!S!*~sEOb`Q0V0V)be;C)n>M|=R!h@&wAJx?I zxCpc>3@jaBm~M0hgDtH(t;6s<jP7jdU|J0i$RDhZe|r5?BfnjEd{-i>gP%%6;EGqT zy&V{19VwyXM17y{z6GBKe$pz#wMqQzqv+(d`n7iyP|;r&e2Kt?yC@9QaA39fQpaB| zVabxUI}4%1f^Kk%IPPl~%yZvYTosMhf_idRnV~n_y~+G~GVgDOfp5!glMm2Tjn7X< z6wL9tfuLExf5QU~$+v9JSBik)KCa2u=FgmJlo(}>(=(Ll<k!G(<FGz~Yhv;hnZWb0 z^akrKPVtDK-=x{dTCeCM-O|}Pp1p>;UP!FjNXU)8m?0L7pExEi$%~%(MVLF%`wp^! zb7)YZ(QeQH#B4oV09-h`Rv>lkc9&~&bGzSx0pRRE6x%uM4ZH&+@N3&gO?+;*1)GIE zszLQ4w{R!kw}k>eoGB?O%7DLfr;LIuAt1H3-`)KnSu_d@mVhBmmdWFwVIh%;Oz%2s zx_NINUv_@Kp0EGWGsCs`>ZGO)Ka<VB?ouH_ohMO4iGFb|ic%&g;s;1dQW_w9vsFs9 zVqlY+0dY;Yg<O5=&OS%mB$afLG{Sbmhk$j}BlRz4>ZY@cJk#tkNR*1d<MY?#uhQm8 zkzs6(_Q4j}bU&F|pnne<^%<`bkj0VYktAkL%2DL^lJ~{M<+x!ik12-J{4ZxD&#;Bk zeK66eYBhDVqYDX<Tik|Rj2y~J7M5oCowMjd{~CB2o&MQk$=>jLmx8a!NXktYfTUVt zmaxMB<UsW%F~{ns?sY%+Zw&q`dK(|K*f1Q<0CV$nQAWgpiW<w*JT;)|gw+*ytgZY{ zkb(raC^$%sqrqp3PV;%YT=jM^KM2+*0NP3h>s55pRJ5v5bDe)Hly{>8lA>{k-un~T z*ZpZ<Q~KuYYV2qUtR^{trU0FgFjYxQ56}y>J@HReR|QCY`}S=EU><IW$zrg<)B?Gw zkOMo6L2q}g`@)Oyn7v|H#IwoFN9@_@-wj{WqRQLei;D~C71xx&087T;kWZMS9bp=_ zef${+2zSkfnP+S3#toiZgbNFcTOL}{ctBPNodxA7^(}PTs!e5<UM6{<H(%cyTnxxR z%F6-i93Vu-tw>792nZcuBo=Ul!B@Ti<m^m9KpbcMT1{lv!1?m_1JL0on%mgO7^_sv zJM2U78}=50v;^@Fs(y8c?<Q%QD#A|$RXK57$A*3U2v6ulB0j?a9X%RA7|m!a*=NBy zDmvybEvD$m5<YabVBZy~d=kb*UC1T$3%Vima;IM__o~8mrkW;5+rw8%n!^UyL`urD zD7=0HponQe%<MA5LNNkZJB-xtQv$-Kv+2(UK@1vUfTcVHXtafZjh!K3k@SHkj$`9Q zbSXf|=M$HIqTd~<ErstV^C>{g1C9XvzvrjK|Dn2=?XH`%kxG(vF9<#}%{*NxTCdkH z!W};kKYy<seR35l30J{5d%Ktwn=m7Svv?3WqVS5r-^uf<RCgC8tq3VznACfUw#mgv zgNmv~JRc|+m?f7ek6M1$swMDIN#^kCX1&H|)5kqwfe{t20o>9DC%&dpUUuzOGwKz% zZFrqczdHM9LKb|`?Xa(q0X~59KD!e8B4_)}`k@e@FmdhpH*N!ovv8LVI=oa&_=)8! zlFrT%%{UX_GpOhtp!sxC($NZpFbcydnQ(Pd;VoTU^|n2(lic3JQ&MtD=aFySSz3NF zr!5JM2(o*R`A1c&mH!)@T?^EAVTI9PKOq#qXrlWqQLPKxjIop4ps_);k#4H|6=09q zr2s%IM!c7jXv(Xi&~u{X^Mt?d`$8s>(4zjdw`X0JX37wv9*YB0lVXX_C<0h<zoQdM zRAGPg#29FX$WhPPbX1d_8q!_(0*iS}8`yh{S34V;>IoSm*C96!-iMm73`Y}+cFo9; zXZ4`0U%f(7$<D1GLx2Zi0IfaGzd{24<&M`qkW>M6Lu~(DiO-pj(b4krhX9{?FE^Qh z`e)&psHeB2Gx<ysm6q%UwQSw3h-TQiy~Mxap);45;gMmn@0M?E4Pn7NQDQL7fJjw; zY+`}dcv1&H^5AB?nw5cHYKBt&!>8DH(Ax(Hf5UTr`Gfd-9e3%35-i!MKdY^0h`>vX z<av319q%M5DJ93}vN7apU~e4J(c3{j`4Q913QV@KSy<iR4Z&+{@}SmF(>q^?5M)o@ z-{c=Uc(@^Z(3v5nCJ+MMGaErRe(4gCki@@36urH5>MA-g-PsI9q!7coARxsGyAgeQ z{i-9ouWuk>@AE3?gz6O-u9(->1Nn4Q%Rl4%!6``7WyBzQG-ITUwKC^`_oS~eQAafT z1lhl8hgIiKeNYrRJ3YZ{*=m<n8&nvb9?HnXmRR|H2LPFxq7lHyI0pb?a})f3f2q@X zpst6v?%A5u6-KIA(WV3v0h{LlU?1c*wD5`#@%1}E5oX>5{Q51{Ha$;p;A?N9%<k0w zR=Z;Td<Vh&`fU(F6e!y*tqCzepPqob-f)EX4O4Bzh{rM`P=sHG>c6jTV3!OT&})0y z+h?F6@Z;`wju0+I2?Sngi+0^d__(<@JVm)hucOJM7b@ItpP&KF-v{gZyW=-lxHRn1 zN>S_)R!Bgqh1|@1aY<3f>O&`!xh2P76Z0%$ZZDo8Nj~D%w{#t%NK%*cvf`EbM_9kp ze4l%KK9|Ed`IfiXcp;fj8~Ufc7^1j5;?j=S4>U_HJ4~d3m%D;4=gh6bp2UEF6J}q$ zlM)V9uh9jEVm)R9w~s-I|8aM2uHiD9k7du-F6w7NgKE37mNZY9PFu)=R_rAMf88nm z^NHxd0yyo*`CNz!Ox7GCL{QZM*>D{c7ngvHIyBH9`*12ye|*2L|9s^21}`D2uuy7- z<>7JhB)jmp)X@n}l$f}jlU1sWFsv?1g0_<r*Uz88%*e{j%t%Dg8#0Lxuu_{tn^wH( z%L{%ul9Il~%1Td53GC~m_Yaa;8<@7by|UBmP1N^cx+>^cL;25sWP4Z2haBER7Zr#l za613<Z3}+{LhijQ$pANXDiY-#Gmf_EsNP4<7)RR^KFonDWP~6D)8XC9GlwfjHJa;g zeSCEEh3$UVL_K<f!`+@Vp9Tc5ZNh)rcF9^5jdE}WA|{Xu5$}0hy0I^z4Sgz>_CC0A zV<~;4u3PkU{Rpt8w4@Pw=xEu?SL$O`Cg6^-dZv;HPov74BWLQ+-==O0vgA{=>_CFF z>rVz!ym}SVrM$&rE+KPznAeX_Hz!o&<fxGS$HeyI(i;)1_kB+oy~rT6MXdl=IBf6J zetbT!;~%*cR7@x#9~jO_y++wEk@ubK?8wf;LLKfFKGh`IU{+NX5&Arwa@`^ZX4RLI z{k{u0RoSEMgS<oHe||dg2{}B3QB|8=8vs0U<8RZxl7&ZKEJQ3?LO(5CTz4m~664X& z&x(>CU~_u$s@)&EgCLJlqP;SiUIBI2=wE_3fQb%pL&CM|FWj?;0wFYNVUP()HZe|4 z)qMLh8RI-NH5GpHc&hfWp|6+7V6mq=z+m;8mG&T#M@~>n?@NLM0h^uZmlH>K_wC;Y zV&D#4S>VkLxhn6-^61)_0+x&~UD2+{OTqoOpA;Tpo*^wM4G<dRi2yoA|3Dd8LXym0 z>(x-Xk3b%+!|grdhx;QVp{IeEwz|zQgnYjB^0%VU4J+Sp+<mFQw(k%&^yOawEIWIT zXxM@iymd~vDB&8F`;RgGWmI?Q{#VQ&(vg*rfEawkD~3Od7<bkL#bD<$#(m%0N-st} zYwP5Rv2Esllss(1KK(9h)k1W2o+ZaPDK$0m%=%AH3|3e((;{T7I(kxkCS`wY!blo$ z43h5cp%$6s{54TibpI5)@0F&KdH!I+^au;<C<1yaxV*l;@+I|FKPXA_F>?GoG5mOF z#&P8T=?6lX)QG9wq=}YVv6_Lt^CvU_%%LH@xk&$UFrU1?-yH@-_uAXb)?F-JDp7}1 zS8O_UsATvakNw=onM5#)Z>dnY^<G?_1V@grexoO%;4@j4r%(OnO?vid572QB`$4O= z0KtnD_$HuT_rQ)<*vR4Uw@D8;Av0fE-G4hJ0sHmQwK4W%x4X4?{RhGmuhYts>UMmp z-W?9X$Om}H^Gyrc2GGGfeR;FUD$>N?Jx~r1wfQ_Z;qIpgyEdn4<O4cfIOupW5Yfq` zsyjK#X2RgW2m3Tb?#qaSzv6*1to(1_B>-U7{p0$3cQC;TY!Q|M{_ee>fqyB(tnjjo zeCe=`V_youKRcy4eFJ+-ORYRb2Bz{j1{uELkQOpCe8I@2o7J{w<A?6mCs-pR2yf($ z0MaV4P<ZC!`7EamvQK~Cg#!y*UD)rtJHQ<RZb%vd%nSU_>VaGM4=Y%JCI|eP0!!Hj zlz;SvFV#dd?WK>X(RjJ7zi+J!0qnu1om%MwtisILC~)$L`}>9#uu=1J^B|eeTlKfY z2R`lfGLii}foV%Ro}e}Ir_*C-!_Gkz(4oZv%4KkrXrfC@m{k)A>Ixk}n#;<ex3H(L zfzomlan|J<2TNkbJ*p~Q?-lqhecH-pW9<#(^O~CvL<BXZcCg?a{W`@AL<9h(=dL(} zaAlAS5==p8v|j!)$k7m7DHJ_o7Ue3R{@Yt`18)5rV8}!%%!M;+x3Yqb+In@2SY>_> z6yy*Ma{Mo%9_v5c5Hmzvfqb$j^ON284i;}$Kqs9fV6h#M^=gDVT%E)Y+#UCsiUrZ& zWFiqw#<KQKl50>VuaJ^2IQH6^G@LX^e}zlXE8lG%yi7C&YEp(>6oLwVABSE1to^vK z<~I&|F?gjal<hsQP+D~Js&07WRnrIJ9Qf4UEHr<C6E3ajN^-TfNYV;v3}Vr>+gUK8 z;xU(2mKk;axX+hG#=ps3rDp$CqcG+LaA3TC9sz^c4M3M<qa%=)SLhDP^!A-=aJ#*| zhg{C1(6g{~R>tPXB&Wl&+37E-8N#BrK95JXlS5AVYlrr5$X;R971y08VqA`Bs`iG5 zM+y_hd2vQI!IZa*O@sT3Z`d`I7-64y6h&^}oASw_%QN%*_4+4Lh#pTsp`f^~1U?R& z0>!|?!hw3-+Xio%wjzc6e6!r)7^qpSn>2EX+_lAAYjvc>XkeNCwnocp|J--?jQ&Tf zu?JvU0<wS#wYO)-oT0lXC6@WyJNTQ)HoFhDQzw4mUZ6iMgKb<hUzq7rdCpq~$-Vfe zpEHsxKc-%BPP6pV33o&E{$^u218gk(mv6%85!gPGR`R{P{np?I;E|U5`?1zVL%7ka z<AH3!fssEjyui_b3sQL1`RfM&7{K9k8|}qs`O5VGD`5{UyTLJFzriA~o<~5%-mQ%4 z?jR5(UaPjE^X+9RG-fu2pzqA5wjZ##+!*ce3lHHZ^gnJA_X-<2!?GTldh6?Gc6V*Y z%>|5K3_i>s%z!e-{T^Ee5F)L?mD0e+A4bR5?5*-T>UZlwCT`G+sf4-S%JuT0jEwO^ zW9_@p7TQa@Az~XEW<x?X6(-_u`@wtlV7S^h!&6QOUGa+JeR>~Yj3nLW`il&RslH3I zabHR-u)QMie_6xL{D-5kYO(;ROD05{z{#ll@17JW0qR$ijfDv9RKJe7Tk40iPDLQK z4|#Bx!24|9?sPKq`E@^oMTMsHIuWM7RsAs#wuB=^L*sUlF1pZepeF*%FmHxgz6LQh zs#{a#<(Z}NHtAy=S)tw^KN5l;YHzD{N+f!3P=Qx`L3mF{%KFA+UuuU0A@0a)Vmo_- z;chv}eH@hE(RKZyoWoq_!(&;gojmLz@`fOR%>qu<paXrKnsl?PMY%ONxwi#9!?E1_ zNOAnZO;K?17rKC(xZA>o{Qt7{A|uTEFfp)5n7cK+M+VONOtR#85TE-wGa(TX4Tf(= zw%bY0sFJn@+g->fS!r3fH}{ALO2(WaiR35C14WEFO(4YdWnpxUZ>4%d1efz3?6-YH zy(-*076xPBEVe~qMObjg`JopoD&fPE>5~w^9ulI8(5nY_$9GFt{tur$5w-QTgXp+N z4i0Wig$9BFmE`H+tP@&y3yK`}C3n6S+h^-y{+=NZ=m(-{F+^j;NM*ZO{qg_gfi^FH zd6xh6c{Y9qDBm^&i`DSXkHe(h{<!{JE{e2u9zY!yYAQ6nBCf~y%ZUu?U-dXX1-f=S z9t%SRv;8q1t0T?H@0%1t5bh_6`y29)13Jc+t}($4wG0&6o8{0!e&k#(i~tf=G{336 zZgW#SseL@i>hwlZ^vL*M77o#lwZRhn*9cHV4ySIEL8&d^&i{X?kT2vv_ZWlIrwaHP zx%$d9{eh96PN?-C{$)oUdIxYe8>(i@flmZ{vVT+!>X)kVQrTwOfP#w^rt;FaT^9k3 z!%OoZxd_b1C#F_^lb3*AnwY5f5Y3JmKOU%%(l4xl|Aq(pA5NbC_(T28*y#2Zn-(Qb zZ9n{^zo`1tWA5KE4mf;XnuqKA>A%^gN^*lvT|7(4D&|LIdjj)1Wvn>=``ZU<=6}k) zdf~@&FKgV~pPnKu2nw(pRT_AD!r;Wlo(nA=)TPftpv17y4~Njf9g-S}Lm9%#VI%L7 znJ@B10*0jev|lf|IWX$nwHNwQy_((&T$M3-edXj1f@sn@Ib8spD{LYMwREfhRZsCh z@tLVYY;MN>(W%uX@}8rFMW@FGU4}DXUS59GHwAzM18r?VEv=zHpXHse%+*$reKW4p z_V$mZog<JZCnpsnpqv)US8oRy8D9)feLKWJbXA(triUiw2fgt|3o<e?K-ywt+LBNu z?>UC0qvOXq8A&NB3XZp*hFLiAx^}RjUr5q*gF_;M%6%{$FJ5;A>YkBSo^)hEDkBZK zY+N2(&Rl`{8F-X3|EEXkzxpk9LySyMxXmqEgp8VoTQ$^O&{v*!xpZ0_P`YVMWfeY1 zsLC@hEC`siEradETB52?&eu95-sW##L_WS1L9}l6Tk$R?st9$sO}9Z_l<%p$LEOBn ztP7NyUZuI+aBfJSm6aQP4+uf;5fIk!3H*TYO-<&6Pm7dI^=~H=Gm_ER$lB1_P0Vnw zA0v(1<f-aoCfP?OGYo&Y;OdWTP35nf*&45=Lxx76Ns6Ss*He|&4a-l<q@ydmHTmhL zT>S3><ke)>v6OcY_x((*Q7g}o3yO+*9v&WWb~etg??GCvdbvm!NLh-R{6Vq#BR>lY zG$-U!Wk2({0qR$sW=Ax@V9XyF?f~;CQ~3h@`GCZFQC6MC&<p(Xg2%+@M2+u-g?pcD zHow%AcaTG>|9l8QL_i3@#z038@hEqa;-W%9345W~5FkYO+velo@VFXi^j}i~RA9j_ zci<KJFUNpD0G50?=>BChu?^6qUg#|^g!<9(&Y%Og3Z5t^6PKobr3elC2r6mWg>X05 zpxnV$tebU2#Cj?f-PsmzP;_i+R16jhKuIVld}rusB%WspfCFNHIQHYmj{!{fT7%b- z3IN(5w`L}I`?Hsng0vR~dJhE_1LUY+FCp=^g+_OfPjaUIdkGb;^VgwWa+S(nv@+{c zF_jKy0L1s+RS^R~G=7Q57@3>zhSUW+B#$&dPy(axMqMJtNLm{;X`l%WAn4n0NGdNA zJK@@nIO3h`VNhWzE>g}tSG6kYtvQVyKG79fEm1EH)u%eMl6Q7w-459S3ZKlo$?P(f zoDnHI-bhF_+32~^N>ZRsyuu&$1i>gfE4#$UA^cu30l}s&aO6q#5bS8bqn)jV+Fh|& ze4|JdQ*l8TkzGqzZ2oA+(gI^|1$v;2G-<~iuG%#$KoI*e3RNqq26ifG!me2Ota@1G zVhcz|Td|{U;*uscjIp+WLoCw6#rbvYxs|o$pAfDc<<&9jEr)|T{6A_-E3K_)2zA;= z+U&sYw|{>c4HuULw(a|SID`OlK5ltia>7*IaJlcYymBke&GSi>PaR@ibTm|SZ2GpI z8Xp9}R&Zdek*18X$Do_aCj&GLQW1SAP^f0zU=tj8U_cdA9Jsq)O?H566a#>qA|XKQ zyvy@42K95Pf;)u&C$Ksw*f>#+N|T_!Ti%uXYhxXTdf+iVv!d#H#{w}o@zlbok;gDJ z<>XUZ)rIKe^v&67>$6IlfQQ&$T7ch>=L3uhSxe2+w#RcY?}{hlDyaZtMQL+0HsFlg z-bCgZdLwd=npmcTA1(F%`r0FtoaUYG+MKYyPxHk}5U}t34DB(nwdqXMCzEtlWJ^g& zb=d5B+x!%CbJ|1p6x`XXnB83UQS)^Rd?ytp&B`=UeYeWUThuBHH6>Ml43RtRwx^@i z+rwhJvyR@(^{M0wn2iCZB5J_o)PtZ61YvpD{yQj|o;-4mla`hAr1hCY<l&MjL_$`U ze(Qm`I?85yYX<_POZSig{*J2!j)0wWtk4%}nV6)clCo#C74;qHVHhk-Ltb7!jlI^4 zM6bd|E$>Q?Xv@9!?H%H4&EdYGKfj83;m#|c<=S0uGB5zAPt%Cr3w$<+SA6EETC1PL z9|vd|ukG^hpdvyXzfQ_iGoMG{hvC&nq_h16xbMIaCnf%w@vGTRS2i;>v(DC$+}Y2? zD5KvK(n$8e{scI(CA`J5!x5*Mh@iI@<%YIau^gu2U;eBU)gQkW`qfGFT(vZ}LrE7F zrYNb1AuN>{#d~DM-WMKJFd*vkKz$5?77@{WfI(}I2|jOberp`p5gS*X1xz!WdESp0 zC*=Grx)bNzXDK&ibMIGN43C~t_IJx=&Bj2P#%aH<q{7+4vt9G#9!cVD>K+dq(-ASq zWGgy4o$s2x2(2CJVjpjz7}ui2P9sc{Lnm@!ihdm@8&#;JKG`P8r?<axnWVP^X@*#| zGquz+Xz`MpI<iVxXLu`@Pxjr~7PoA>0jABrN0$Cu`IUR%ak;!v884Cq|1-QW^0PvK zD9AndnT+G}`KV?_$)8TBr~cVrRO9Y@UC?ANd3PAc0qGHLH)t9Du{*Tg+7_1{_bo0M z_~Np%E0F=j;CA&1;^XgQ5d9tB{R~U#4iPS=HCtI|p-ivsl^vPrm&KFx8&N3(d-{b1 z!PC9`%=OXd4K~xpUOeYxAnZP^$sQR9Y~6rQ$VjFk4S)}jYH)$z9j{#AUY`MkU5+IO z+!NO}9~58^-w;bkO${nXC+_N?>QySANY~d4Cwth$2TIzTUFk8GtNTx9?9Al_ao?7j zjbT3(TI_A0@pExqbX5->FW=lH>O+9N>}Ogy&2Gj=;g*&SvD<0hAo#!uIX2C$<pY4Y zy0~DX+b*cLg7g|$s(L=$2>0gbM&!rnhkgbaqrI{2CpHS^!88@C3E0zi-k@B0O%Wls zd{tqey`pwC_ix_LYudiy3F92$n$iwO+q!Bp6uV>usutEgTg9`cp>vOi9I}<o$v41; zZgUGiSQuLYN9d;5v-+W55=r~<5v;nzb7w1B*VKeaH<~76t>eenA$)RSx#}m@LqbTZ z&RfI=qbh}SJ#NqA6l0>Z(ihTSSVV3Y%0%<wOxx!{R+by*o*T<Nd^9S%fDVI#Rupc= zjjF=s0#nHg!&^=7R0AOX855VEA2cZ0!_Yaly@-AQ9Rkfxge8)gp)1c&PQ)&Oozw^Y zS3q2bAz|5{=vuNnOR`B!B`3uZT-5_`w3ZKbZP!oJnpY^rD4H#8e(_gZy?-9|iQV%A z0G(&})C}CEg0r!Ojq>-DxKc>lr-4TchYZ@CTWxs1maSTv_}NzsCu0IW@uH%+DUflK z!rlgFn8D(qSeB?ESjhIxW&vlgw5bnr;8#Qw=)=CyLeH?Q(qeghT5o?dm@nQ#Gyd`2 zH-#EQThoyQAEByZnYGYpmmD3Q{!FKowy<gtzjfEVQCkjmf{G5b?#M*Y2+Fq@va^G- zxIrn{n3~%`fk9;P?RMl`tEH_`*#*@KrbFYGjzw%KghiIKm$G5zCd7k&ZZm9(Csmh~ zkq8g-0V(u?81?$%TxWGUrgXUib}b(2FM-bUh-!MGz=8Jvj$ALD{J*D5{omOM0i2Ot zJFhO!;~m*L_58hcQ&dcU0NpQiHuL&~V|54UDRAa2o8{#Wq(Zq+@!q`I*uT=IXJv_f zLYl?dOmV*ShFqpS*ajJ-9iO#5uHkaOONH;ltnzX`Wf!@>l+$T(QOW!M0D+RN@034D z3r%xYJ9b^Lkkh{jic0V5&GdqA`%Z4Mv%P*g4gt==#9yOGSidt24eSXbNks!z+pbvA z0ao{T<ypR;*foB!mE~#itT)qNKNgegvc~L*g+YL){Q5*trYEaPprenpIuEN()hCVl zg$3G4#ivEG6<mTZRz)gz8LKs=v1b%&Ut7Ij-F6C+326mYsuEZ2g4`wtF`j3iFNA@0 zHg^5VQfDF}`bISy2GU`!6aJ_jn_xyJsn<hlLX7X^4(n@jTy}FFxn!qw$v>@~GKxPd z3S@g^Zd-eubQ?U?W(bH?O&zWu-~qe$k6x8g@jnt?0a)IfH@H+>g7~Y%A{6f(s)df| zDm_!RwMb#XH7Md^E3MvpOin`?TW>MnP)-(|za8>)IO@I;y(b0=YOGTO5NQH*6#iG2 z*P&Q1)LC%}#WWE!r|NHnm_WKahp&IG39*o)j==_o{@pLmy3ZV!>EGrEdK;|+KsCrx zx*+=9p?OLZsx~SVew(*_7NhUQ5-+^9>Gww@h7D-k^P4zYxI@N4C`I>!)ZVQEGLkmz zWw^TS-49$18ty3TwPL5`m!DS^*Y)~YsNXTc?)5D-d@Z|ATOvj*uOi}0@c2uns!eHK zF6b0qVUxIh>krpy`Kfd+0~~KC>$>zE|C^i5Y@bF9m9W?(HPGhjdNKv70-AyO0x1Fq zz~-Hrcmq&u<(cLCc(l{R!bc;3mj^HGAWTzB-(9GL#n8~1)Og>G|0^U-F|ec1dkS4` z{o|cQS|1ca?-y_M7y-+joOO6d{GaMm#x~!EJlQlB5r4ZX(XREwQ3uv%I2hH|A1NDH zpSPk^KtcA@J1E41cZ7iU;Uh>LL)E~#!w;Yd0qP0+)-||eHC?bp#p_r`(F-Q0-85?% z9VfnHRb6QZ4Uxe7Aa?F|!7<5LLu{kSZdy9JU2{tXDk9cfot)$qM&wI1J6@{YX2}Yy z@&i16Odk2c?E?rS>NIG~K7)ryd8$iWSElo}x*5tl6$TDA%I$^UD-nxfM%)eu7OIGO zIYjfynwTBbsnPQby$oOhnf|q;*^rBu_>98=K_iPU5QwG$BE|DqyFyk<7X6Ue80l;w zJzyZaWnt|Na3C)PtL>Z4IgizKrPO7Ls)}LUG*-K|&SQ5;Pb5M5$gFso(r&t0MWb{l z5kk(M;u74pZ}hv;L*-z~nWi`xN*x+>+_7f|Mt_#%BYin4<V$%@Zj3C-IrijtJu!(Y z>isCmjj*A#Lbt!tmr#Sf9aYd~Ty&U}NN0VaY_LMo;z$CX8U~*XIM#3cepwN`T>so# z{NI7)!sC0)%cAE$EPG-*)*h!n3|$2n@;#SxL_d+SN1us`zPrqSW~+#64czJ{6w=9; z2t*_ZnUSNr7B*?@ALjtoe}$5~B?0d?@cEj1*4x8@b+}G@MW6A31zd*KrMjTwL(S`a zOhP=0l>opL^E^?-0b80x#2dQ}`nbsF^hEj77-CY&=Z@A|RyRHveMH~#I;zlV>FPYt z=Y4d02SDAkMnSNN-t}$9uy1M8f8It*w*B#FDiX%my$x_F;ZY}IV<y8mGlggc>0~Be zq7K<3>?0;U^_skKL?7pit5g@~Zg2CBNa`IvDu~xN-87#y@ve3U49>!1t~5d-+8&{q zzy>fiI5Gt1d+(*|H<yo%C+>Gyzvr7dcm_B(^xfr1MV%2q@BtHc245yGCeRV0GWckA z%yc=B2%5jC2@xYAA_9y~{<Cj2U};u5*PUZw9p$@_$OwFbLGF_qGp<R%*o>caD$9B^ zXS5Al8`AXZLN@Dg>xjuZd~sLp!?v?>R@RsM>9W3qR(No@9ASDw4DxfkfxnD&h#<hj z=-s4s(pgp9U^GTY4(kH=`navnvHEKfysLOQI=UYB0^i@1%h|tIq=iB7OJV7pK+VI? zV}Hkcx7jM=yHf$W9#0+Ewvqa7BAS6&;Pg|Xc^TpBJzOiMd2M8EL7N0j1cBYmdWygH z<75k<I+vU_cw2-(iUDV9WBm0Mt-pY7ED8MqleoIoOKMfLNu-#&2lm7QHr66R<ZTZ< zHE(wzW>{Aq!C51aKc?dFBc^CWlGh)(M$R%)1^P33KErRRjCs@5R1~X9U;=X-GdB`B z-U-WSVdRLwh$c5RHoA0Uw!rhU@PS=_h>Sup?3eOJ9uM8ev>z)%@%5j%qj-!=Tr@Y% z?Q*)QP1~!o%+}_b;Zc|_V1_iK!evf6H!?f{cams8CEJGO+_4doyc`p`REJ7kc*)+y zBNcjMG$||m+P$h9!cxXzL`4>kGyBGOsP2vGb>h#T<*Ej>hWMknE12{dQR5NWRi$#% zEH=P_K#GH6%riQ+0>kh;Rp%!84uMKfH`buUrQ&1cMPUoeP7xUoavJLqgXALpQI7sO zq4~MAQhNP&(;fzsv=z#4iY~8*QnifcF*m=!`OoL;nF`$Jc_^)DgwF-q!)ZW2F898Z zDs+#)3Qqqy5Xgfy`zF$)x`@scQ8|JFsTk!wKz}ky`J?KdqpE9Gc(4tOMp7w7y(QP& z42xr0qHchxx24rjH@FezPd;W%=!Qn#Is<-p3Fklx_N@k>42Ln5@jpr~E$lm%$EiqK zPKb=dQ}Y=<+jmNtQdZDtEJ~Gyo4un?MS0U?d_S@HLuoAi!rx}QkUYyf3;W2p)X2$8 zML0^Fci{_fQAshBsjKdgG`faHCc1CbB}#hd=(>Tp-6;cMmL7ZaP!(MpUi)rtpbvM$ zLbL3!y;Ar^VWao}zc|RrXHnlt=L2`<7;*=otXJ-a21<J!{`*Z>sCWq+6eCT`wErZI zy!`k7A&-zb(&h)eVn+SdzTk{FGomesRA0%h?uTflp3#062M|Z3oAZyNBTf`x5RAzX z!D%x;zDF3G%7}WX<-u(6soL4Gm8~|SkhQ)4;!}~Gjr<z~53?~B1#!2pKUwM2f4W*- z+bo<cC@#h?xH|Chl1SjCx`o8015SnQM}m6v3@owES0ian^c69a;f^g0C+%x2z`JN{ zVDLUKG&hLcR|6m!=s}O#`UyS4d!6BtDfgT;S5J{pz3+w*2&gfs336qq)=kx%(d6?4 z`a5|Mqt?3ixBz%-WjEC(+(D!QbT?i-syh2UWkHL7bRM8G$H#{G=e}vaIu4SRR}eEZ zs~=GOt>dAp%PX@Lb{Keob$xY}lar&8;7>$IDrTt6A@cOVbbE{VaRFIf^nTL`m$wrE zA&TFx<BM-bGwWnu;U_lTR?5kG^kOHD;WWaTMc%&v=4nPrbzjXHsCKoR(*+JpeDRim zAg*d%Alk-;gkH_`<LQ2}b>x<S5GUmdc*yFbmOx`;lWD2=_6-P12?cC>*zD9Buda(z z4c*){QFj^}l~>)MUSAp}NN3TpVEsBzN6TA$Rm|8n7CU=^1SiI_Ms3_5j{)Hr-g&XG zIe-V0_bPV*KH+hGwpOXre$&z1vj6V@5Hhn{MxyWPQD!=1ur?8oLpXonx|02oen{4I z0m8)w;ezbee9NjUC;XKzg<a&%s5GsOtdeOtc7H%Ub(K0*=NR`5?`KId!L+51vt4{w zAz6Whlq<vQ#fzBLlxmRqpp>%Nt0E6Z<?<0z(UY&)HeKP_{S^JZLKnx;y~!Y^1%$<M zjwRpqLM-_ag(A1fB;6!u`dBGVH6f3u<k8`QK>^XQ@jYd33?7Q!w1B<^CCTJJN=4_q z1!b#T3laAsvAOk<zl8Hxi2;H0hV&kM*R)h&^K6!3mfpk=pQ$d5taY&34pL&<+fNpK z@5~m@S7M@mlu??ba!oxc`KQ2OF20Gp%`^86_nvl*UP}6&Hp8rX8TVo7&DC=Kd8AFs z{JFNQ)d%Sk2JCTXLs?td_nx}kfP28YE(oW7u#B=|0I_iR!;h%Bd-@QZ(g|!WqVE-r z?~}b}^6zQI_dR3AQ~Wl{U^bn%fyhu*-RCm5UwJ&#l;t#tc`I6#;P7o#$?0?VGb5sS zMw@PufFJorrYKv>G>mbS<?ply49zW(sEYI&r3J@DmI6CMUR_4q%x~1}WU1JgEnBOT zUB4$H%S?WM;>OmfMlyzHo{y)dp_`+-TlKBqY%or0-j*%}fpDB}?0Y0u>ZfppsBDJ{ zv3OI2*Mg49GrG*PY@hc=oazQ;C^V*h+?`$nQ4a7f6U8y6J)`4SMH&0Xs{H=1<E)9x zWDd+x7@-3Ob=+aIx!sn&7FmSd%kcD{x3nhp12RHv&$!=3am(q5_dZp6%a1NB2=k0# zC)~P9J+WR~Oyqc-gPJNfzM&)3!45_RMBEMvl^fghMLHz5q5Q!6Fqvaj(Mo)6hX)ui zjR}r}Q+ydswF8QX&+C!k3)D=d6i55{7r~a0#;CHRBr*@u^zxk~*X`IF{!9iG6gi^B zRwkzWX2G~oi0_seMpT`RQ4X6BU*0SzQ$<K76=eCq0y9gp<$2{<cDzqKV%mRiFB@x2 zNLH`d^f-Tl7Q`2|HF$PG8JHI|<pp^12E1Jq;SZ8%5zIfStg-#Eq6?>Y077hcD|r#L zp}>Q=itHQ*@#Wxu9U2OrxTL<2G5&caiM|}6|8On>aT4SrA)b^E8+n0<59EvmUt@Hs zjQs;~PJibwCHb284ST^#?+d&AZs6UQ)v}UiVxN1TXMRFLBET#)&HrEx7@I!L^g+HQ z(JAV`2Z4UMb#tH#&wD=%6gN05qJY5?!9ZAuqR$;6t6mH1$)%?PLXh;@fgO5S;u;`L zI6GKv4N-*OjG&}t6+6O6yh0t-Yo=83pHN|?d55C!3DRjk;A=5^uf*>CI1_Yx*>0G| z=Y($83Kc;o76@z%lunL3k0SF9T#q@)!=_<3UO?XcL;2Hwca9;g#s00`;~DH1#ESAZ z{j4qN3N4y8oso!&%iaF!eSw`bz?QdPAnbC&%El^j6fr8T{VKKoEPg=F=ws1)`9E?% zA1CObwROSVs(RN_IACJZ)#H+MqoMDEl1a?xk0A<uEegG|GyTH%ryLeBDho)>+Xs90 zFN$KyD5Qx6F(Pu(`YoOKJkte7MlNdFvc40>Iq|tnW+zXG@nUIxWLCggn3^gX0E`xj z!h2a+DHP`0eZjq5JPm=6tzup!b#>ppy|=2Ws%vDe&r!G4ZTF<DN~W%o@od7>$mJH` z(Ht){1Q6*DLqZI@rH~Nh?CsU(eEkm{bCA~r#AF)0&cv<!*4p2=I2gWcUze5?C8U}$ zjho+2*pR7K2H@Vcsu}zm9vPOaAZu!BH$Lh(YjSF)i>hps6$t8$+NpatZK2z<v_YJ= zoyX20=$m}^c8F~l#blG5sq@c-Upp`QCQi>6{1V0yzb-@a!?(4OA}Q`pNSz8DoWg#4 zcNYBRk7=oaTT&k|^y8B1hR+3!Yu{3daz$=hO*R*vO1|~RZ?OABGI|*$kC8KMB)y~A z6bC-?WFp%8d1KaDfy`iYwoyCFI}X)s{360!Uv{RzjmC?_x1YJ|!F<9jnxH%=TkW>9 z<@_~NM%Trx(GA5Vysr4R6FU)h!7H|f{-=zwHjF%Ynn0zuE~@T>fIzRnn5Bi7ln=wi z0*<NsC-3?p&8fT8F}{iKWt9qdfa{Nc&5R^<DkOR_-uK~{`OI40I#k+GF0F+B!zh17 zy=w`6LWuGZ8zWSWr^7-uRfZ7$R(@D<-unIY^W0EUZmf%hHbHe{znOM=$w!iNy~;%4 z8>}@}*++rJ5~qsAt%00|x|7JoR;VrPtFS@2sJg|CDv=(u-d`(U@y%!=(rDxF9O^y= ziq)o@YKf{A57cZnu`9t$Dn-mmJL|&(^Dm~(QBW2kng1*TRW=6##q?7!J3Ljz08xxH z#fB8oA|}P;RbZ-L7c5ti2%KT<zSA#JN~=yxZL-1*$^NtCQig)P;y0;9`%L(3Q+|=e zHK@EDQ~KbN8CAf}s6%Lp*;MqMAWhwjtwk(@N|?aKCYMbZ&mWJZtzk}siq*v&t{fw> z9Be+^V{*dy((O94j{-<i$+S<OY!#wwD&)@voiOw3daVyvg}SC>3NR!4?v-$PA7bil zhVm%H=Jix{3mEG$`RgBv<z-wv0+(cdsK;kM^^2$$)nJYZ<WVT6x8)XN4tGYt_<n3c ziCOqTE&44<eG8K0x{%MZidPA1Tb(%S+VNFJZXl1~@TRUtMsaR84236E-`(V5F+buj zn~%$`u>)VXr-X++`FrO-thERMLiu#x;f0AKnDYMf+C-Mgp>O;%Xh$hWXp;j1^^1&) zf(v9meeI(q8Y19a^l6UslFxS=)xo_NNm|aM{z(s5sz+2eOpC@V5XgS>wfkz>lC^VN z_4%?$$w1cF1uD{+Cg43*4)6;~onR2~FXYbb0hzYz3k~2OD#m|-<p2VJG&Yp-a*k4! zf^Dtojjx83>94;S6|_M&$GWd{tx*W{g;ql4ut-wB2Nw_k!XD(mYU*(xA7N$X``>#R zhh3U2E(l`bWcugfEoymq>V0#*39cujFwr#O7N62Di|BRrE_)?wMlZk{pP<%fRz_!5 zWF;vpi;93?JxfSP8t5uZ?Frk7nHlHT;7`4Yk0+<~mhK&u2iW)2{Zip@e;+SOLPb}Y z(s+aVQJLdx!&zNjk6|*$F+#H3qm|RyH0dKg1yP6jHyuCwq`6y3-#kx@9F_R_CED4@ z@JKaKNX$7TF8dGza}%(?qY$FhGB5`O3KbOBAbye}A6Ym*Dr2m~B)`)nxEj}Q1N=@G zC|%XcJ-Mn&3ylc<M3W;T(l5QYej80i_=r{szJl|B=}waO3`zE)H@=PUh)qq~%SsNq zrkV|xZn&;&WOjK-4K)JU3yf>m+pcfxlKGCra8!QQtLfMY$OgJjPFDOgz*7Q$LB3wK z?5*40fOB>ZH(_rB1Ojmc$E;1J|0w$fwCz3fC>lyCRxm9?IyC<v^C`?@d0<U)I7aZ7 zy=rKvJ6%$la!3(WkQ-!pmpIEyHn#Pyp=a&SLbH5ZL-Ld^)d9Mr)<Wq_TaNI#z+{*T zcF+M?<VZx8P05$Mw_9&AghZXXdg8zvjXBWV1{dS7Umx-$wp5GKj^n=b{uF^zM$7Ko z>-%oeKv-)0so*U$J^|4(LF8qfi7G}MRkbiK#k(iKBgN0BHc|cyk){WRq4N``FcXFS zM=Kb>Ah6&f0I8X#0j=mtN;W-HK6PMEVGtUo%(A11%{I2%gXw%E|D`j=S7n|@Yk!<` zhCbV2DmRiL=E7;;z(YtODrG*VXo4(v!bmQ*?IVupMM^(QB~1CA@Sd)-T(j11pXzqw zq^~fRdc_vG=tq!kne43)13+yPIG`Uq&?H*UvQrT+<0cU;AEdpaT_TSBy2%M^mN-NZ zS3{NX#I_5#tAsrl{s0HS?*Ka`r|Tjfv#_RZO?l;jT9~FKAZjc9YW(MZzn=70ATs6W zr@SbVO7#A?%2LL38%BGZ;k|B)<Po>(c81Wcr%KTrQY#`-9~Xz$`W6yXkB#1|JbYCk zH*%}E5tdOAhnD_3WB{xmY#C&TNt}8dC0EfA>`K*3`C;KujL4+4ph2%fr>)xhb*h!W ztp0aThuU<c$o&=VB0?Q+b*j!}BOmPGCKBz}UXnU(vMn!?8?v&&g~E|ug+d;)K7rtb zaoS6H=l*YVX?RU$ML1}9I0N@#N>iy(1HPZ{^s=8q(~BC-)`W4@$;fZuPVy&OJyq$U zX6cUZhlKI<v~uZFhL5r=uiYKR=*djZW5~o`Rwch^NNJYg9m^dDwr0-0$}{tAcV-W% zx(AiQat83`XC*l6YS^IIihhw9aXxZCBLv`oJvKsA1%OuQ@?X(_25G-+d<EUuz-7iC zVvG-3(T!sEIJDwHKMP?zi-ftcl0xQs)Kt~~4{2{5RM*#KixS)=KyVN47Tkin2M87* zSP1U!9^BpCf;)uZ?hfbR?(W>p@4Nl(dtLqY?W)IrQ~`&**Ia9jIp&yScK45yBvt9) zpw-jU-QB@vS653fdG_tL?w9}-{d?Et2j8(4@2qE?G_HbL_|s2(Ce$08t_n9<k8rRU z_!yKH$aaV-g_p)j?U4=iB>k-P8fHMv=!KGYd}yqzVp)$(@>qbxJH6FKJ)|*D`Jydd zJJG%t7CLa~siST1hnyp~%^vjo_o~o@Ei!;fT^e;-qD8NpxAM1%JwsbWT(#Sea0>OA zSo6@7zqP!-+4oW<>weW&F*rwHk^NR<=j{qbY$Mdp)W0ji+RPGJ(u)9;It?e5q6%YR z0awu+qc-_D1j>B!wo<(E_rhJ*!uy;<^}I%D-LKv^AoiBZk35)9tlqctnF4vw^|h_L z<zIf7OX8;xN96A(&irs}T>4U0R#rOfF8Lw6a3J?K6B+>>dcB<sS_=IiyWjTPAP6(V z%kc1FND_$mAJ2&gJkHO1$UviqY;0Ef_iSuh!hd~_%yBu$CF2M-c_6U^$fRPluO}I* zua&WMF%`Hh^6v1Drjv8WM$Hs*hC1wi5Si!r?hAU641~nKln8>xF((Thp=M{e7AqnN zPTLZ3c{LYtHIXLPXP&kwZvS|tSsGP5t*BjR4yc;Q(y3o1Ug(wO4TB8K+<a1l@lzOJ zCPQde!%pl#;Hl0L(QUvH<Jrx!cuJDv5cXa{X+OwHj%fWA7gWp9PAe^qh#6O1KPg9N zTyJvrSWX$J%TwMB3e`KvI@<MQ+x=kT$}F>m)?)~EVZQ$5tSYXLx9yP6>Kg8Rs?i@~ zLc40DTY7{<TgIrqC=MH!-*;I<tFef1W6;}?wO=vlp=YZ>oE0u0K6+_G^pmWVH~w?k z6HU$6#a}~wo%Ti5uXIN!H}F#gpS1Xt2B8m}Z|_gM_E84oS0U|O_jQHqXbCK}$Z18w zb{)Q#TCm@Hq(fccxtAIaZpj^;hQez&qD43U9;7*vFMWwp-L|O}RCy*UJi6lTH2t9p z2`-XYG?4eEq~!3Ex>4K4>&s^wx^k1JSuKv(Vrx?-=z3i9lL{gdd1a)CXp^M<NxU6t z{~L#aJb%<#hiRb^`#bkd8q9k7bn4*tGS5=wP{FQ&>vdVpC-_a}Ofv(e00Kp<+(=AA zn@zHw>=$%HCH+6A2&~1_0k|^+pOFtx$Y5b0eT5aXUtpb~USKB8(QWAVyxRsG3H>@R zkPru^my?nVdSicYG-jbNGqA$1ozglomn&;4D~oS$ClH#L4?=G86LzcQtqQcG@o91g z*?<}voI1uOL_$Ok-x<^PWIIAQjv3)8p!?tU7+7&23)KlbH&541IWgva7O7#RCWr{B z<iXD%STAlUxu>Nf8RbAHL%Q8DH!(<c>u)Wwuk&w-Ut3`nD`Xn8<sI`U{dy$bbc-bh zmVmr>yUV=Qm`_58W*rcMeRS3D3Dg=Bx&ZM-wnf=F5K8G%Iq}e0`2h}WDQB$t{z)el zCuN4@NHOu(o{lqY3q4}0W|TL3S2|=~##UQj_M}9_y&T#Z{0!-Ge|ge~G(~>0cObtZ zXxGCRQ7$lWV4RO(`c6Vov{H+uxhKzR97A`k9QvjIuZ*9o^`vZb(LmW99ZZ5N-Z;Z; zE`cM4+N^3<PoiQdz3yO#FzTY|o*l>>QS-)1<Ul7n=Qxj%kW6PuS1d-o>wpjqvGR;6 zsZDk(?o^@C!rJa5L#{%5n9e}^K<KzursB$c(LlnJ%nAwdC8HxnR4zSM_63I8mh1_P zf<wh0Uob8DsrE@pCQi>ow@Mokg=PF~ti(AAS4mTC^wR9<<RV6c26}PGRDh+oX;C=Z zG1_)G8?=Fdm+1_RXmq9O^9<+Qpz*5EWKpEN&G%coWuNCYy^y({>j{Ef2aTOcfAX0} zle+xknyP|AUpFr>{rxUY9k)_~Ja-aGQryF<$+qK=nSzOm!?zzk6l9}M3@f^8&fa>t z=uXT%Yki_2BF{kBb@+fW4V4t(S&O;;6?TZwkiKi;kW1s5ymduINuhG*i{p_vskk1> zz)xTMT;Lc^@cQru;?@y=tHNy)t*cR!AwWaM%2j>fw+KoTd!KmE@h<lLcWn6n+3yEq zSP9NaO7Cxu%R|?UnMDpnD%q5sEX}p%K1MPSW)T%trzq1MGUYb4n1xw4ofpr5v&$Pe z^yaW+LsMk~H?*8T?3k^eJiW(ri%Qxl?<LO8tKbk2dOqV9)M#gyVw+a<D^Bf1+Hi;* zCrH8HpyBCcbH1+J9iV)DFL@m?iodl&aj(`XN%dQCeTKq(04Y;{R=aDsVr*=jAtTD+ zx-$F{ZO}qA={EVO{#Ru^Wp*R>y-`Y@>vac>H|E6R*2+h0z;u49Lvw6~fttbSP;W~L zhfQ&6``4x1gyEv@9@clVMEYL)Ot_$NbVHlG|85VHPGEip=5Yi7Z%ygtGxAqBr6P3y zIjAFbU?+BcgynL_(}{M{{Nm^|xiE1r4hO<fwhEEZ4gc+bYzWqyvG6<fX(z4TWk}pO zPCMA4v=$+KNOU>#N(O%81yqZ+Kp-d@(iIVe?*9O+s^_~->YnUO;5f1V*Q#Q{14}9l zEb$#Oy5WSI@I4Z%8If=X@Nb~~)*yqe2*wG-g%)mD2M<G5YLi@cNWs0MbX<}VwkB00 z3;EQ;R;-E?$HBoq@My;rmF)2q_y6aIh2qYj{MIFdokxof)C`Kh?e~aWx9B$giOOo$ zFg#aknl(e^uF?`ZJ2h{=dMW&GqV(XMJt3wgk~ZQ2579NPiz|eF#7%I4Wmw39f3}`G zVF0cNEIg~3K=u<5sHoWrz*E94lmV~Y|LwK^)Cdt-?mok7poUfM1iZbJbv>c#8|tU) zS~~Y@&lJuK0GfMdYDwl9%6~<P!`C5N3CNsgR;FYCm!+tDW+Aj&B7FRf*HI%g5+?^I zyG)H%D+<zobAn&{Xq{&38pw)hOGtVm>Jk6Z$a}8FyJ04;StRet#s+@vKkX@-mbdu$ zXK{d+WQo>ox{!z~)=wC)rmo>^XX{8oZHj}4jGEK&;$?-tp$oVS?dRsP>$dMm6piWT z1NI{gC~c<;3*3}cMWr0hj>nviBxrP1#dRLYS^9c<BFgi5vzG^p)CzKPDkBk^7Z>Mw zz@<=jKREgkmPAfrx+$D%w9t6kzt>dX<i`B%fOxx?fIwo779xYa4%F%ZZYG3`P>}`t z3mBoF7S&1C;%6I>pF(VvWS6lpCUrBnylo@V{S^`3-g$f{@V?*809Va5ggZ76d~X5h zbBxGp=8h}a?bJh>$;HAVm9vg<b-t^w;_@A<+1+hA(UPw?`*%p8+Hyr<Lt?t_iWxX3 zZ;J$1?h%bB?-?#iMZ*~_PO-pvduwe^&UBnFx-m+Kh~Lf+@T42?vgV1d3h4RHYkvZ{ zy|r`E9;d3Ze^wp0b%-Mj4Dyaj?~GZHGgQLHmv@1g`l-Szkepd9SC0=IeK^2#ApuVa za{=t04whjcMg(%$S!vMUb`RzMFrX-5(^9p$Wo%Bnx}O9b9i68eh+1iADHDJW3>rbQ z7@bvtlH#U-ENxY7$tr+B9~v3U>B=!cL<lEEV<w4wPl$(CkiW(8uolPcGB~mA_~{}c zHdO+LnWywsm9{fG8sH{B1FnWM)6?CKZS}uPde)x!<zXc!M=CzH2yy>R`N8?NQ2+5T zTZ+J8V31ga?(g<FM8sE^6nz2nfBYOG&<_uAQyP3us#dnNz0`$MINA-WPFBQ{=Yc_z zcDy&Z&}S={)QZYVX1KFl9FvPkNwh57>9;fiRTgGu-~U!M)g8suAI4P8%sBAc9w77R zHOsW#9OY5;N80z0<7du6pLCk{WS1Hh{WUb(k>FoFJZ${m>u@85%x3#NC7bN1LMZeJ zr0mJxN(!5>K)@Q}27e5>I{ZPH_&64~E!=vZO<V~6Ukl{!hweWP;eI%<_@2fa2t24_ z)i;cDLk9%j+ZKaZ27>Q~#UtoAMC6-^N#YcH?Owt>DEIZLy|X~*q4R~H{rROxO;=U+ z`T2Q^1YJ(h>)n$A1nfh4l^z9-2AZCP!{Z6m)-g`;l2hNBr>O=`M(dg|;L;McfOe6P zm?)CvRUH8kszSM)FTvh9Pky=BvXParGwO-Qe~U2cNaPGB*hX7&mx3>mI0k#ZL9QMi z?{=*x>{_-Z_hfI|>cTw#y1t>+b7Bn@8QyF`^oW-x+-H*Tw(}icV3U^=0UPnK7Plt% zU@TQ6n)8N9BQJ$p1F>xsEeU)(bVNuFtK309?SNTF!5VZwbzdr(HVx_fjp&baq&4wt zT6*B0y-q;|oR$81ewqbze&t(5Den*e_Bw$3=di3q_iqSZXa6aQoU2z+QML)+LD8?E zB4WT=kY8wjwKl5V=$bMKn34boNYS~izox%p>yb-f+cP8}n-4-I7Le|E*Yz}JCfv~I zJh5N9doMUFEfk3>CtF0+dNX0F|1&QfHEI8nxlvi${n2Pp<W7Mf6U@5=X(l>W+ID&B z2%i~l;r0mY5<SJX+QO#jOjTsY!Q$7KXK3D#mv&NY{bSpefZ5DDZ!|*0Y?xye+L)i4 zzip!_(ytBQn6H@}cxPUc68&awZb6;i!wwleXX>>>NeZ@j+J%Qy1I!0?)z!9C-j2-r zGumL3-y?P}jN_B?ef`u6nU2ibOv42i+to~?mli|Gbl?)&I>{u5?CR`e6nz+yXSQwZ zV?B$iAu{wm=&J}mKE9HcmLQ-7*fsWOZH@HNMLSA7Z4(mRqu(WX^r3K)Bum-rca`_I z*>kczl+)mb$cpQHzHIlpB3RhqCZ?}?FB6<zQNMoCXcMimMn&K|e=t15`_Fg6uOXcf z5sQC7djow>_OZbo@-l`$?k{;6g`7NKQGR@deWc?jkqvfpc~CMo#unF$qT)hR{BgRc z1YXg_$0N$g03YROH*WIqPK<V^d3Zn-*H>gLJBN@z*&WW$OfNr%fV(|aRi^ij?!z7U z1bwRnT<&Oi^c~fAZz>iCS(h{<pd&C~OheAyJt6%n0ln-8wHJozaypmOeLQ@Wu>8<H zZI1ax(aUgWjm*ZFn*6dsT8^!vCd-x$!VKtpb+&R|G5RSEEDxl+7uJQbeqt|zmLdLI zl6CE~+aZUD@8(kwLpc7&JJr~``{V;v2Vgw0u$}F)i!f>Y=~C4GWcXc;%bMO-)geXS z8Sf?DM`J7|{n^6L?ctrrIUcURmA17Gt(~QX;Zrd`=@r(96DakOLi=66dp>JvaeHvv zC0@F?W<o?o*i^TFJAd19WBc<gVeI8dnLB-IYV4D*q^0BvSK2(Ne1AUc`eQNBGExM~ zD$GnS*5`sjtJA~JPb1|FH%WfG!Gr$aO^h$^22a`__fym0mUEj`FsZDH3W}yDH>vm1 zR^4eyS(2%p^7Op){$qEs2*H!VauNRn_xzP}0OZB^hD-i90k}k9M8T&bL<HT~Y%5F9 z-FK`Q)>Ph-2pbD~x>DegmYX9Tu=psJdhT#p3}xcsQ8iG;fGJU!s`9>t<>dHeo5vv_ z5C`Z3q)swHZOl2XMQEP)v#3_lOjXq}Rry<LdL|qi8F0&`#pP<dC;5BHX{B*SM+Qy! z49fwE^kxf&Xkx>gpJHGbDDmZ|xi<Eh65rR$HO3~#_$Br`#y_GRS<ouSGj>qF9Y;WY z8cvu;yKtZCOuf&i(vtx@yaTQe<|xPGRH$j(k*Pz!PM>kw5sLvcyahz;!5#%uav|Pc z*2<XXcb?KKeto>urINch+!-$$D!A~cx)!c)VbV>U)Fs8o8fMc;d6H`{{s71S*Z%Rb zy65IsI$GL1Utf`z7v-WZ61lRDCLTe{y49U#>&R!RQEN0uKOWc+`!(jIM|68My-Vw> zvdMWa1W$g79SMd&aWNHbcR1bVox+#M+{{c#YVd}29nq*!6A8%ZzI9Dk<RoP`rQ_W- zcIN10i0pN}t0JMK*W_-p%5ym&x1?jB+p$H|u@=N}t*WS?(0_$MM2cFoMeMrt@0X#Y z+uvkvh^f*iRU8s@-Qlmg3K(1_g}FpTG#_rrz#=A=ZrsEKpoZ#@L?uC7WtROfwU91T zpEtY-kenii5C>Rq*ry%9J_7$I1dy`~_zcm#%pDx!YijKWdS@f}`P-A%?61#qAA(@L zaL}C={LyPES6ABaUJ0rjo9%rMCzZn#r8e+>0;Ypoo4bnU`ziDX7IiV`^0;`889#pf z;2|O+b#URIVb}ipRbxbOh31cBvODq;AstwC!RX8cxM~T016S>c0PHc$o~h<idL;|r z`VHir0I^iH+ywf~T$Y&EWIsFHkV797+fBB%;8`KxPSS&LOA@0Gf2zCGlBn0atF8~O zYm31B`^|@_V@R(<S9V3aNs&Ddi37ydD7c@I#~0W45(n8e3^AiVTFU&82W}JTd{`wJ z7+g#k8X5++Slosc_1O@QCef~6x3S)F-{fE*FgFlT!-Q{aVD3FpQuCg!^>}VfPO?&2 zwgD$rO=;?KHQ;d9CIHuPEd23#Z4JF@!oPqA=%S+GAP<Yo;FY9S$TGdCDr*gXe4NiS zqT&9iCeRwe#G|`S)0df<8Qd|jw;34|o<?0?*T9Y!Y8-`O9)i${tGai1T68vG##?xD z>@k`V7A9*3sr`Or^eriX2@IMZti`!|;dFFdf$e20b)m-La=y5_*=Cfmsbo#TDXSZ~ z3|KF#v~mEv6nS$%&v%2hSoxrYM?fQ~aTYJk2l<_$Pr6d4T_Pevo`a27W`BryJl&bJ z)90Hyr~(VaHG`0eBCIy<7^0^kI0;Sh$qUpG@IQw#5D^^+v!4LK^@jWDn5)!KMn6?L zp9@M!!a<=-0nOm06=mMss_4S2{2crv{V|}Km672!El5EdIZn;imR8pc4~W}8@1sN1 zdf3aisRMCmNNyyMb1e;+eZZUW^uy3723OP(lt#zzU|H{Uu_DKWUydSh#fSpVF#sIK zGCY40(k1?DT(fWw&YcQY1QB&sQMFowtKh^scLpjVD(P9~hC^9iPG}UF*4ud;6<O>| z1+oMGS<}dM&6P?6^zt3av%{D#&VFw>NK(%<7beD^^zX}w_Y)M6fRmr!=f5d5tr+gZ z$^3`=qGL=}4*NBj#>U2%J<H=AD+<b!tTXfTk~mDf6Q}yGP++8df`MkhE0gVA$fIE_ zLuIGSJz_?uOX#4)8TBM97tKeGU($oa8agxJ^(b_I7w?r#z|LXUJy$+(7eKUUbV4p) zBuqXWgWP@i+dKz#1zx}C@K+&18(z9yp=7QgU7W^FV@FDKJCTM+Ew^}bc{Vu0x!&Kg zlEKsrDz`)3=t74cY<WazV7Kg?h%7rD=V^c+LY^S~0&rI#{(NEuK^o#doPIHmtYKuL z&G8J5oGP(L{ppRNJm37pqD-)Vju7gwF{9N$!l3=ikN#hPfNxcADdB%g3w2O47axDk zGBCL-ADRKVOEACyKzaeNUjRtlK>uO@%t(^)GK&t1y}~?NW1qa##JpY~zMnrAk+(Eg zB)Z8^nHd0l_AMhcqy?lUW3;b0WdXk}_Omq_vo#RsIempx_{7A8zt5y17d~(4>v;)! zo-)6r4stN;F3!x1DEk4)$>j0fM?tT<0b5?Tr}!49fXEE~VfuYw>hNSHa8d4dx&;zu z=(><scnhQRv^HG~@y~K{wyjFY{NT0Tw!Pa!XgP972nv#z6Gt{2@8kk|WnEk%HOs>o zGLzB-P`qEBah&8b++Rf#$^#VbLt}#lQ&V>MgoOF=bq=77?(X3vm09pRk*=#el2-yr zf)5`)OhyVk?jMOuCfwpv5wQB+jqt!l3jc}-rw$(0pb?-5&%+b&^xw;fYKYd)M)xOx ziKo1Y=g&WSN6$oAR2Xx~$>sLguDG{1Xg>o2iR`DOIGhczhBdo}51-wY)LsqlYSWO@ zr9LW-Ly^H|y?a}c|2F05+rjW}F|-fuf&sF5h_?juDf3?$S^K{hX+n@?(LGqkvz7L0 zmJRVv9RcQL2jSxIJ>s1)t9YA+NJR<-hma$Xj&>m-)Cv7=he<@_Y@C3P`juO}XBHXO ztrB^M0*Ti1&A^uSd3szgr}W_Kjpz0CwZqz(?UM5uO`MIM;ETgikNHx^p*9sPM#=8h z(cd2O8?R;WJzUqvhfBM3_gj(;fz3Y=q-+?n!mu#G7qw0&AX&gBHWzRcaK!M>(AB+~ zYb6o<2;17)`UY~IX86{1#NKb*&#wp!DiVz|Gwiqr2Z&G19fK!e;3Th)SnM1x^GRN& zJC)qrU@=t`G%BYmnwHy+XcC2BSX}^LtOF9<e2W7eF?qjuF@j1Zfz6dbS;7JufS81E zgCTRjPHn9r)39^AHEe|#bU&|QuUeQ{T9O#6q|;rDNDIa}Ua3_~*k@{}*d?NW|3BI; z@T^Uf3l4@X{6QvGJ=5(VuVJ+^08@MZ2gO0^{YC)PdH{zY=8YV(--5+3VL|>g3f8kU zcCUwuJZqMqgqqt`uzI{>bzo4M>D~!25?=1i4#F}DZ?;O}JL!ugn3jP&mb_Abx0w5i zF%i=rXnbya{buP26(v7EI}3osqWyvDIKrIZz0XG!A)f0d>dW<wRZ+muOVO&17pe?i z`^=;>Wc6LbFT0(OUDRz@9~0es*fQZzNhp&nPTGEM0dT*Q0ya;B8g^1aj41q*xS};l z`~3-Ad~9mpveQ}z00jY}<Ifhx4Icv|{qC+uhcN*&rxqX<@FiQnx~>#(Se#7x+nq<l zz~I-`_Usa>d=%(%ok~DSMaeFL_?+BBnpgpifL(mCGBq`H8k44xJw^CEstX1B2on&g zaB$kkn*%Perv|Rl2^vyn(AF?e%M~Rhd4F>a=@^(y9xs%)BQGQ#4_!%n*4P2&?(p3< z^FKv~H_Y=#?~OBlV?e*Penws*{ugOlUDA{Nqg75fsvu>LyJdOO^51h9e=|cxzIi<` zUC%2h+s~nM8q;;XMqxURxddMvhdX^Egk*&1jOf6;*@40{d+$$*SbeG|Zj3DndfrNU z72&MEmdMvL#c{oVV8w%Ct1PIGK)@G+Aou+7BbC41W3Tgh6g(AU)IZB}El2vY(O5MJ zm*qA11Rs*MNj=U+7Gg@OaysG)rziLxT4&}O7f2_zw|R-u(_KrH`Eg`Z5;fE=^nx}w zF|Q{3^(Vyp%i`1Y2Zp8m%Eh3K8hO!9gB~|h4_Ud)$3dd;v56u8W|RA1h&K)NQ<qws z<(HQedNHHRhB!cD7{B)iOi3JeR>T3%@NA&TovQf#Y`r52iVWJz+tK41ss`i}ux6sz zwC<3DNK<Ir@UaWpi-<Qspwy9m$57VPQx>1)bZnb3yBgKhQ}|$cxa<1{DE_Z(+#7@3 zSNPulU~&4qgagkm<HHXE2Gt7KH_{KdRsaUc)AtFePBD*MU;_n%1wMdz*+Z>GiwtS} zeFpE3tM+{UXAPbA)Fx$Uh@O_3I_l*v>NSXtuGHlx{&7;>6*8~w1H<u?39}`*IZg0> zgI-2KdfRE``4J7U^T_~oeajA$y|SzJK7aFa_i<j&d7ItuS(yDEeV&%<-ARs>ZXud) zYW1HZ8$6y-uFEbdX768oMc!Ci>T#ZEePMXcPoI=sz9V~H-OsCOs{J*J2)TKj?tEo^ zescECW?Xif(9l2q=!|Afg`qmj*Kyfp>Tq_(>UNu_xE~wGewwo|yV{+<6S?XRtdalI z5n$l01hlJ5!s*#L<SsNVuk@#8W_}r`%Qd1NJ97j~!rlPiQi~_GrTVzr<!aR4=&c|* ziN4TWTKkOu9Q1==hVE*c&T)U1rNN@}8vJUcVm?4QoFL83a!zD=2Jc-m>f7HNcn?4M zeWtR@YD5NjIb`DU8*5tTaM_-<uq($G%yQH{XHG*KftK2t^UW6lgv8yd+ZG%=rQOT- z{!!CiaZ_I&gX3q<_IQ}#e!&9V{~W-9{U2`oGqM%Or!?GjX_=zIZ1}Im<^K)M2^@jK z-$;bhpoe{Obq(v3PbNl280L*}r$rrz`wHaOtfN^-&v|)Ul7Dx(QcYy0rKfK^px=rT zKs<XqeL6&}JGAxGcIT6HGvHCWy0#Sr!Q66XvJamIcXpMmX*H3Y_`3rhOcztJnuhP6 z$!k50JDlxhk>cW-vQmWFz{PIFH?(46WSIuU{mD=daa5*IM5KuCH~ac<p9xWrky&E6 zi!;MLHj>+Cd>eesp`iTz+c+5S>b7r;8G`v_S62meK7^d;SY?ev0p#AL7R!(6kSth_ zXDG+#Rag8pf`XWR1N;;%s)O5!8Pe;BTV=2&IM4hreli6kH8m^$-rac*<nR8(A492Z z9Q+R5N$=p+k1lwS53~H%pnxxaaKz6p@fL?fkKOwgRemI*ilIy$`W9E+&z%XRt$F_B zetSB3w80QD8d=`t^wSB@GcT<^I&0{dh8b7~B{y9C9Z=bZhU*|dMu2RWTTRJ7Z?2|X z&hSo#^^{w|+vV`!;!dhkZiuK!X{&Wx%qCQ_sx9TmNnAyn|1}nAr?t+n<1I;4>rpft z;5jg56I<|-HHX8`X!FK#!U07$C??>-g!&Iw2sjz=(-e@Y3a7t+gB}IAhJWbbwOfD? z#_Y>ZL-%h1cJUkP5GI{K;dg2pDM{&+ufBS>V_Rn=JW!q(IUsVGxg)@k)dii|0{VEA znAJH4PTXB@bbWqSAM1V8YG|C3Vb7|+SV=e?oo8wvzPN-TfKnYdS^R3~`XF(jPj84W z5mv7uJ3K`5=~HAkT~|IuoFZf<pMa2jMjnycoYv=16x%KJjn7`jCs<$ii+Yki%axoJ zWV2Z(2KUd=Hfel!WD(a0R<;x6$F9dwa&)Ft59OZ`m9}72cZydQH^nR9qO9%kl+mSW zqw}ep515Kbl^QZ1%r338qdR3N_;^0&jt2qZ10gLYqE^uF!^Z45#)(*Qp`CM?upx<| zE)ow){>-_mj+TWY;<{2AzT*y~?qcuC$lMwbq<xU?;b;}u+p3|gs~=I5<&>ICV3b>K zU+(|It+Tb$*o#YaXi2uDsX%J~Wb#&DR<0eZVOQBKPzXljPD#xXK1tTtlt-OTV5`A6 zPBE0=KCxL@#)v<`sq{kiHaIhuz)Drq7~(#uRp5eX{WRo2m}z`r2?)aY&$9yRYB@t3 z%DS7ncIx}!@Xw_lv^u(3E6Ku~0zpg~m~BRu)+qJJoRfdlNElbNvro<y9LvLoOFKG) z+^F5moD7^`Qks+@KRXzB8I;6)bN`$5d_>pph`#4^ev-?&zR5$antc81oZy!2q`zO| zu~Dw7uYRXJ^Hd;lICWg_5L^m~i9>hGDJM?rN+(<)ED{_%x0H4HD?mx$;kFxBA9nya zodc<a_giV<U$OKrZNfEh<^Xj?QuI9p%Mg(jcN?=1tG?dpljq>Ud7<dBOy}(6v}(ez z`}{B>D9^lF3bmC4#l|0bv>i-|Q>mD=RTSZ2t(V1SDOe6WDtg)<wi_zsw4>r3aFi6Z zVJVAE#WVt^uZLJLC?{p#>*0$jZ|7jCVvC&DbJ5n*`qnI~CV(zbR_jeBA`YbHRwGIW z3Tk9>({FxZx9G_uw%BRDmY_&uspCBiTw1H^7xg@F51GlYUnVA5dr5vO;G~?cNW~Sr zopeUqbq*^>EKE*?Z_&e)@HN)k(%-0@5}j?W9I2{+IVIHk@Y#UU7fU_oqJ7NyGstRI zOx4ba|B$haQa(YJp(y9q#t$PkJW3T4?XW3x@?JWkzTz6`^6<R$M7bgB@h@QI!YShy ziie)l+%bNj7+;WffWL=uD_h<vU&4qG4^EK})IHF)7|+cQT8@fODENI`{3P}TMp?6f z1T$2SvQp-i7)uLJZau+Q=hOLhb^L7k#YB3?dhBAlg!wfs5Y93w^5*nY6=W$g7fUrI zg`KfqB~wcZt7H<QY*#K;lC+|;t_sy;Dilowv<s^w5tr;sN~_Yp2B#HU)@d!Oe=_F4 zGghC88Gj&1#mhhEGeR>Kn?<lRVvJu-6*pe(onf02LlJdUTZp+`G5*EBUHtgvCQ)@< zY@T9VSGMnjG}bAax1Wz&XaWWV<;{gUJH2)Vkrn)^9YjD|#ktJAi@@705Nm-+TR~<l zp6V4VvB-$0Hl;2_WMs*~D%A0Lylimbh-DUKh9)aPu|QY69DLq9xl0Z=3BvflXkvhR z`d4)SCRn0~i!d2GyAFv+4c(I5nlO*2C-c{wJVGAs#{dz|6)0JJr^rUt=Wm>n?($<L zw^>=+G<n->>_Vb(>~x*%m>i}t4a8lXo3%y<c$RAHO48WvlxmCGbh*$i!ueWS&KT;~ zNve-#^vu*e>_YANhj{pDcm;i3ynyJr;NS(Zpx;Sv2UGUf*8?}ll$=mL%o%o&_;!15 z6o03|?;y8+cb7%E;X`!gC85Rs@ya9Z`xEZO`4n+l`SbW0hqS4$n6syHc^(>k;69Zc zP)=QK^tE;+s3f;40Bk8R#jPnTIt^8hv)5@^7kH{Mt05sTdd(roE}?24CsVYlHm~2? zpB7ADZc~$$gcP{u{`Y!#*!*FP14R)<<x&YM38wN8|1dvP%P3SC{$#nLLo#2xV_sC* zRfKn7B|gz5vQPeJXr4}iHzD)MU>-2VP5l!B-LZL3)4Ut>ma&IC1LaJgvWh2vL3vY> zBQ!>BXoM{m7+u!Zkt_Lo{r06LWvFB0zLO)vt96O{ustKCIKseTzJz1c1A#;9I&80+ z<kay}T6)lC^!MxILJn$v^7Dvlxd)1`xNbp_Mi#jwdaW0_bIc}-=e3NPt%++y)7o(< zTYdQ<`EHx5aW*8|&cvD1u|4|;j*T?;p*QG?jipvJ%k9?zmk#Knx0=`xZnZO~Um-L4 zW_y2wY%L{Z**H;W&uiw!5$pH>%EbjZtc2d`pl9xH<!S)K0*93wAdYxDui&o&J^_{I zx6*2)hE8!sMFjiY)YNAn%_=C~BOsJ-BIf_C$@G(Rof1l2lqAD(Y?*GCKjQ4Y4LQS# zx!R}h$GY`fqpfvLi_nMQi#i7VOA;>u2K%Slwii~zdr;xGPJttR_^uAu#Qijipa?_P z(er&)d|ZNT*6Y{1qdQf^N-ZXff{5vbBN`M16b6hCO1b0@j@!zv`mC7V6zmlAm0FKp z78$iF!e3G4Zd8}0kmSb{`B%{1yZr$Lx>R)x5>vJxVr{OxIKl(%tL8<OlL~qb4%_XY z^v-n9H%TWi`7qYMN{v!P1UR#Tq)FbzfLT{69hTcapC;~pMI1X!-OFC`r5oBGSlcs{ z8`{G^&3-8%2IPaepN+Ffect1XK;XyDVD<0LqkK_)R$|0<+GV@fDXZ-L7<-y&eXxAD z{PfJ-#p5CfqSbC!+<4D(U+mKsW^^&(Qp7D|sgNI=`8iUr=eh|k7hiU^IzACv5DjC7 zbTOu?@_fv3zt}N5C28s;?Q2S5(v?ks$+V4tH+$KkXF)#Q@}Drsrb^qU$Wk~u?hTuB zfF)smE6kI;<>hw*z#0O~=-V3p3&8+&dAH9<hUm9Mcz8L>2%%eBM>5^r-5!MFx7TRd zYtbw}EUdk+$dP;qA#MNmbcr#e9aT}T8fDC|&L|x7OFiG&i)Sjh^|=>Y{tkpIK5sOB zn1+Y-2`&mYvjc^wq-iA=R`VF8wCC)LfZ%Lo3dqPYp!yQ$h0?BSo|9|XJnf+$k7v>5 zl)hN-_Rg-X(d5Mk1r)_SK+9C>Z@c*0dbr|~&u#p$R?-wFZT_HFh_u+hF9ULI`MUN; z){klq{BYuxc&o$Pp~H}M%Pnmwmw#3Gk?vr~)bPFXuL%*7;k@lj?)|I-qh+5AoA}L* zB{r1hRHD4fNv02C8DBx;ONNQ{K-oCov^=dvg4g`JTXH8bI8Rkag4CaJTwhk{)9R*f z88MuBgy(0GhE&M+`|CV1oo6H=K7psW8F=fa-C01V$>d(TNJuTXnZ5e2!rc2U)32-6 zoW9Q}a<^s9hikW8d|x{ryFD1&g~saTywgeb8KH(wBg?(4b@RP*6%$ZAUY(7NN=`Y& z7nkgWSQ@z{k#2Fm$O*dQXSY-QH}bLrsD*N#PZsp9bGCZE-zd5oIuZU6!(4oPEkj|2 zXJh%ql*y#uEYnRv;o)Io67>1+U)RpA!a|eqGUVjbpgvu&3mWBI|5Uw>WJ{v`;+;3P z?O=zTMlIn0&*4-UN6ao}5}{;EtZPiA2_Jt0awg5e*V{IFER_>Ub4F>ki3HhZq7*C* z647^*dRUqV@(2;t0hIUTbda;nV+X0~Cwab2G|jP|F$(#_`Ie1>g;BZsMa<S6vf75= zB1SHtCe-UC2KU}MR&~r3WWicP6=3+IoA&ognnEFQHc_LXuI;GA&=*R+O~E2ryrOt{ zfo<o~qd)4QrX5Iu8qhY-csJOLM`PNec03#O4NR&0mpbv@P5GJkgbnmHDt5@7hl%!q zo%3yDKS4!oWsDKzWuKeM=MI^5?!AlDqgqQh6E9Zza0Mxq?xXI4(%RH^uMb!*m>kq! zdctoh+m#8Vy?RfuRCf9?OvG33%NB-Zx?Ib`WBQ9rSuO@zTXtH#Z(k&c=Xg!!l^-03 zn*Llbg>mSLC1j;tTy`0Ev~3-1UUxLC@y<|d$~3#@hUdj*VVeQ4??1!80YFjm#s|FZ zL*g|zQ`;ENI7XQ3^vDOSHh!NfiO24#PvT-nA`^{ngjb}nvz!!pMh<*rp-g<u6Mxo7 zN%|k}9`03L#IPKloV0Z6XGD}E(t)Nl!N#VF=o>0eiPybX@9Ed8N6ZIba8@NbIFIDv z8Sgm`oHdR1{OXb55VIS1Iw^Gj>K)`Y3#Ol*XMQao+T9iLR<sU$4HEq2<_`*g5@^SN zz<iKiyWKAm0T<18dns5;J|%kLJwaW|WemUn^tgLf_2Mnx3IF6f)ith3Z5`Zch~3o= zR(bVCzvuPDcowvbxDV+FxfgzZe&>nzOy-Go4d*7gzxwsm$FvjUM)u+Mk9W5B&0|&i zYXYxptVh&K?`!82v)@aMo~je%<GoV*ZLqR+W&EM_L~Bgd)ahTZ&#xcu=6|-1-^j(( zZ@ZVzdtblOKO{Wp35B?22)3Z|J)m7b2A7t<7$#VJ((~GTHNBet(^&uxF0rQBz<zKX z5Sos9dX4pZf7N)Uu;o2rE$SsFgnDQ3;JWsj&DHH)@$<PN;ig96KBRXf&(Rah<5-B^ zo$IHwaSO5G!o)b_tHS+@e&xjN!#3JbW4nm^MrSSoXADOzi}7NeCfCG&3%RZ#E}%O( z$~;(~su^KA^hp;O{%C;3+smIC-XwQ>ObeyQw#V3vAtE~LoSgD_T09PegDEn@L(Q1F zYC4~_8;<jv?x0hXlk3`!FutL&F)V%&(v-a?(u$wYY%T;cCZ#X-rLlF`=H%nl&Bu80 zlgV<6jOOL0!&jylE5QY;(ZT(UUKY$lm9_DD(*&`!MVy|-E!Y8$KM%aMcr-I_$%ji_ zHN|b19s^P8bYo6G7BOb9rKv19kt7aMEybQc8Fw;*dZz?4wSKM@MMqr)eY|N^EPZuA z-*OJj%=&B>AAgOdeEOYh)&1-!L#Iyn4L|(kWlc^uK=;Q6_<f}R{sIyfEdbcWHjgLs zM=p0)gh{|daGFxm^vBO1dh^KG`~<^(j4NqHfvr(V&G$U1?d+`^O#_!m$hKbVV_}=* z<FrGSrtzZ+qjFVpn2YEsDQW5*>!_nuu~ZixcBbc%Q>P=qPZ3aTs$vcajg*(YV5aN} zgr2T?+qG`k3a7oXcKti={n5;rT79TIMfViEs*Ou$0SU?fquZ0)kOB%q|49c{M+R%V z3a~t<l(;xqQmkA?w1Q-z2L}Xq;#t=nW$*4a)ll>~ykH2B;xS=`PgZ9;r_C&c(==6y zWhJFVoj1E0%6LQeI1F5ky88U)12i76SFQNJwN975#Pg0eupIFHLq*6Sm$GFr)8!$I zL!^$5)c<>(-9MBBI{XpB61qRcNmN%1Q1(3l%Ca#r7)GS7{`8J*6!^%BOMjHlu6(Q} zMH+$?HccQtg)5Zq&G8CT$H3KzbrPUvMprk{^kgUabfJ{DGXEx819bHNETpCiu#o#a zi6vhE_2)lCCh&P3!Y^<5(BvNRAHn@U8d(7mdc%ptHU9RGrw=t<b(md!P;R~1C4YE* z6w#OKQAgawz7d6h>jko{-!vTn)A9k}SS<fdUQGSR*+`|%W7iJ{;kR^M!NjZRYbF#$ z;v&t_hiBM)%kS=A|2G<ih9rPGCVKmeK%Ald7X>0EAjO8fv#hJq)?POA@bFuvWTsn- znm`lQp42-aaEBiHV7#)lBtuDApr)xQc7GqJrmHIod<{uN%ud6?!WL^5>~3$X%F5Eq z%gb9!=~BKCK>x@ZBYbZFsV2eOhyGWkLqPlJ@UG(+@Ipm|N6cAzaI;EF?s&3I@1bZ5 zG~Qq%J#Mv6BGTXNph^UTdlnXOIk_5@G&Hy#3;Q!#zoc$kcZI_(Y~9^)14X9kSqrPJ zlM`vH6Jjh!fmEQ)yBYZBmL&o8<JQJO<(TcZJw08SnTg3w^YE9DF6uRNuz7`j){P+V zy6wG}PHHIiV*l6vV{FB#B+5*OvAC&Y6vZ-d{aM?5=aDAksbbl3f5s)mD{)*U`<LOH z+PxdwT>)C%v+u&*Y0}7oCCDU6u>7>1jQod=hi`25e<hQyqu}@H^b)>30qHmLH(A34 zkxjsw1tX)POm>J$(r|V96sqgH8`#F=!MVG+m1u_=e_X%1LcYF6Q!+J8dg#o$kHni* zY%GVeL74)CJ9m1(q?b1Sj|WKnn2wYlNSLgu(@%tTP%)sY6EJAfJiW*(&s#%HM<x^0 z@umZqwq+hT8-NEiqh-#=>+WmGw&S!w<uwNc5>${}0;a>%ctmtgV8q@a#*++PY$|r< z|H)Pw8XMF7z{)-H>8p~OT$bgk2$$g|4F!An&5?jqjzJ6u3(MWgb%HLY{{TNe&z?<H zp%fM>A%x%nbvw?~<f5;=aNZ#FNqDs4?<Nz|0b(DIG*tA5TIeg7BJQ$Z@Cq)R-)zu~ z{i7fhFd?5)jRW1+@VHHwiR3YNzCk)z96-Hpa?wGF>Z_K6>wk{5!HxC)iQDG-20{bZ zZ!h(=bl+K>+7&^#!hO@8z&RS7sXA}3vg+Jvw04N^B)D4NJYyU_n#=Tg>Sh*ZzVjM2 zn&_QgI4S2P2|zqCawR_D<1<xLnWz`Ihgl#z&{d@_ta=`cQm8%_7_*Y+%SuoGY74q> zi1#fxfjJD;`qV%LJGm&?Y?o$P@+--k!UV6|<O{*f(}{b=&LzXN{2HGak)hY_4(qh_ z*TYt<{|}jEvTA_#%=!P)%Kim~b_L#x;}fM6cAT&t=oZN-j7&be-;^+S4a3!m3O5i6 zI>iDO&Q0UXlZ(;@7%^AJFQQkYAb%`k*Nj0fdt??y0sqKMeo^4w%N0G~)qfQRs@AID z;XKQxoOJ_4)bMN~_nXPg=U`fvkMXdua3v)LflNAfxo?HrBGSEB)oEIwz?`M4r7K@F zrkAbQxgwEN*`#EtOT-BjevPSd9vb!AK6iQ^GXgsBb@;z8FJ5++XN8u0w-b@8k2Du< z$S9?zzak@5RAlIMJM0ori3LUUPoEO~{JQUSI|XbsGg^~AcT?Oi-l#aYGyK7ombIX7 zf7o#!nyX1(=xxg2G4cEK1(3MT&dw6>hy74=zrUM|NUJ_r0qs^+9mV4Iuq!Qc0mVQw zlOJDsy$OeXqe#eFL4qCeN&0_?F6(_B4Zbmdy7>?&DW(j#TGM~<i>fUy4uOR9PuZ?2 zvtD~=t-WPm>th$xO~*)IL?A=GpQ06oOiXc_TpwMY{>5PgWv;eB&?X8f@5o5Ys~WE< zo)GJ8?QiD|(Fak*>#lo@cmWeja{xD9CduFXQqx2>_{T&<SU6ePba^^@_P7uD%GerT zzI<xdv8O;&v`$PWs0M^0w*!C5U4g&(PDVz73=lV5d$Tz@IMl@XTCol+<YZ#F(?_OP zwy1n@`jlnuN<#Eb_pe}hyydz(6MfaU$6jY`H8r`eA<(y|=x^$i^K)|*zxAzt7E6eT z7?3Ty7#LLBT04Ty2511uc}!Mm_B!T|9~Q3JE5trwD8r;Zs|w$iMYmyfbI9xu=Pd2) zFO{4u_CD+)qoQho{Kf%uVvUG;8oQ32(ku74`T3wI37+h9z(<-E@PoY%Qle8chJ8g{ zJw(prY4~pcdpI(ZX^b7e1+bIe+O)V+?zZY~VyX$y`tVzkq!Xg0aEL<n9&nbLVC+CC zw?GxbM9)8{D?V(i^KD1|Va1WHs*UMyvKkus3z#yQWNW3@C7KHZRt6OH^X<tQxMdx( z9CD+)gwW!(4<JJ|iW$+_LE6&i8L*JqIaW8Za-*q<u`LD8pW7%W5h*k7sx0e@2eE(C zqOk<`;lqiFqZTy}VPxL8*QR{d%aUs7eCkE0%<Eo~9@bW(xW6YRp#TkfTZm6pUt@$D z;)}bPSwN_Wm;o8&#OQQzWEQVz4z4I7YC)B=hp4Ye*NRnF@};SE!(^arBo+aVBA?Sq zKFQM>92TDAbIc14q#k*m;ojDMZQBS%-@B^fmrs3pEQt@p;C2y=m`Q%C@N;@}KA=Xf zrl%~@+Ohg37|Tmf*RfN7zFcm^Y-0B0Ba1;L5l~1XtWq*E!n|a-^E~-WNz4qj?|g9N z8rJ6J&)eFP;(Rx$=-vCB?BT%sJPBN+h#vTX@be=oQQkc0K3PU)xMB_sho68&!SB3z z9{dswdR|in1SL`Xz~GYjw6xdgSBQxqSe)q8L<wMKW_DV&?N7yVZT&?e){sXdbS4V= zaKYybi-)q#R}3OvH$N>m%%B>+{gFlcEB%+@*Flmr9-Gc3zQ1@Y9T`&I7e(MAF7SQG z1yWd3TyT^S2ZNs=_<<8mMZeiCqT}fY^)@!v*;;+1j};4dL<)kn_Zh<T5m(jJ`eqS{ zE&l`xERpx^LD^L>7dZE5r7bw<=K;>HhQ|CwIU;wvwykY^baL_7C?~9vsfnu-OgaxC z_1wAu;FX7F_;<$vhL?03JK;@LaMhjl604BqoZVMwGG}h2idn_gW`C>K;>edtAG6ZN zpSf@7^q1_ewWv{Je<TAi?;M0428y5q(&}H;IXU_NXjY}ar|ZBhRcWoHt0Mw?$O4mV zXO`zX|Gqdv9-+4&AS2{)fnI#PUdq6#nl2v?p+6?1wc_YH*?qyb<mjdTs02*-s&91W zpJA*tea(q<b$gtw(sxhuI*Zef3ob$flYErd6vNU0>47>ts(`_ms$bo9C{A-IvXe>= z@MoiW#H*fO7?<4F;JIR1%v7}0MO!;p6>*q#3wyS{w-Avi0@l80)IV`sIH93l`V}+3 zw6qSP5OZYfs5j4Waa@$A*8P%md)*wTRpJcG-i#7j{=2n4uvgqmun)D@AbtN%!&&j+ zxU6dO^^w~9?=?ECMs+~XNjqF~4uR)p;RTYcQ>)r8W0kuI;sXryq;GdW4F|v9Va2$b zK&S-u0Rw|7gtnml2|4C6Ac6C+j^6Yuf^)EvHJm8Ul8jgEl+N;~j(Em}-_-AXfRm9P zUfazUw6$rI(e42CVp^S591GYa!^Xd|A8h0EdY{)EoTVH2-2*x%2a)H7pTGPDq+}Ed zkWiCog@m#Wi>Id89=0y42J}7CNRul7!P0*T3(ZLOPVtIpULlvx8-zTs;Yc11<iXTK zb6g&K%Zh#_q-80EMJ$6W7nP7*J5a|_sY|?oU^RDX$&L7y+wSNi9F*GpOmtBrS`?-K z_bwMFLql@UYGXb)MM55@uu5+IycRc#uk`eM!Gl8SXkar)M^Z?3g$$nlxbkor8P{JQ zMBvO0@9*Iz3Whbff1p3xu3%!;VBm$k+)QE?x%?!Aj%hwJGgjxk&0r;FXJwVXMnSn6 z*3iqdu+ob!0YTmKcx?cr2S+Y17aLtFAXO{7U>zENIjd#Fs*E&{R*jBCP0FC*;f#2U z^uvKOBcQ0k`V<&?PqQo=so?N%<SH{huyyg--N?!*a&XsK@ZlHe`Ajve!!~>ve=~m^ zs+9s3L&O&iGv*5^e`F?icR;K(@&*ycb}-s0FtjbN5#*W$C*vy3sE+&X2C#bNj)I6| z5NN*|&3wKB!DUU}=z>)M9T#Tk9gAmzC<KNNS~Vcm7xEe*-a7!$!IuY!2Plk$v?RqY zvUQn8REMp*l;9px<PTd@Apib7uBwhMR2FJ3@E!wQKb&c}W7>Mt@G!m02XP$_a>-ub zk3Q9ge;~H&w-2ppSh#0sFt_DIba-@}cJ^S2d=;U<s@0|6tmreE+}vut*3NROYA)=6 zhO^Cgp7HOlU|IuDVj@#uEG>`u;dM*LRy*v`Gy;n6E-sMF=NsV$6xDe(TY1S?13|Cf zk&5i4-0bM$-jUil5PJj=86F@1DklsJ>kz>qBIId-!RG0>q}R|*`Q`UaM)#qGBWX&M zS4x|M3K(fi!$TR-Hhb$W#rbiCzimNdW^NYfp>n7Bbn>Z+FY=OEr{!~S0?8E7?aAtP zpJr+4LV8v6WjS@U_wS;ODYm8dTZpOzlvm{CK(xRyrAZD>E_U;XXxEVdyEeOoP+^95 zzT5AWm?pTSScG0lc^$TTx{W`y&2m9@hQlUT`ec69UXEv);vRB5Y&dSy6XZQ3y>C3e z12{<XJ8iDxIl}VKGgSEfnjhX@W0AKkFw^i!q=wVc-@z_8YH8tRw7R7HrO0zNAxdns zv9;Im8USA;Kp#RZ@ilyeWyP#0&-J7B%|79{A=xX?wmp}F_v4q`{To6)VhYv=TDG&i zIs)x3F1Qi#F{M`o+U}Q+j}JF5fsuLi4Q1QMTw6$GkUDK!1Iqph3A0y^T;Sc5@Q}Ou z9pR>bm(HLeqbVr=G^0KC;PPo`+mYASyUnSH!nS_6xCJ@b$l@m8)J?A-cOVS6B0{I3 z&GeJrvyAzko1j~h!iLMrxctI2dLjMJf`@;tWuT1k{AjJWd0s};+Of_06scfTUshU* ze=^;-8tEN?E9U7_ZC2jm1PEABcmCl1@bsjJ8r;z0b3WZw?;g`@5(B2y8d$xHTb7lP z8SI_)Z?FPO14i<j1+uJ6u!byv%=2+&O&h)auAAif7qzKTIgzrlWK`D9A42pnK<4Xk zABGfl{Ot*6llvTDkiQ{Fx&^)zXxadJWniEh=|A9@{~W=E6DP&$dsfV7ml~_g9I41r zXem0N{<W?~)9Qor_Y(`~whbfONfH@y&&L1)3mPH>d&Dd_4q^>@je&5se`{yI7lT^x z<=s=YU;1Zm7L!3EACl8B4^X7y8X5_KkY7fBf0&J8bW}3-#KgqFZ4eWBHQ==hqet*X zQkZ&z85RLUe4~JHZ=KQ*-F!k;_#3<=9_f2@*@W$_EnD97;-p5B)i%fQDrOp*ol8Jk zDG4Bq3$|R#W1^?7Z_&0udt)0kJ~D<xdSXm(fPaZKkuTbZ;Xf#-D5{@T*n$I$XoZl8 zjdq57yCu!X1=Xc>nnrp)3_v=3Wm!Wa=V#y#OR^go94_*Ni?T89uEm6fP5J12d9_o( zXv;Jeu?TnTN{WM_r$c7mrbGV|i^vyao8-3+ac-c$Xm`_wSTAD#ulm2^IyT>yVe!KI zpnkrWJs^vPP2hS$k}BKSP>D2wVTC6cR}ozIS8TMo6RhkiM_+-n@Di$G_nv&Wnrr35 z2x`KUoL^Z<(eZf!8fx}|IhiUim&k{>!&u~Ym>tk4Tw;1gM2>q%yN&hrm%_9m*3tMA z%!v|W1tB2~@$n8`R1RSg?#fJ3!4-$_dmr3HDBgMDDC?a??EOhfOqg^h@l*g>2aC!| ztM|+B+p!bWe@;)J;b)iy0KH`wKBx{u1Zwopd8o@(KiO$MrXSji5&SjHQ%2Cu6^R%) zu2MJBI=d2XUIg-%_6ZE`Y>(E?8vpR5j0E{6)s|8|5&Uki-=`2J+_~-D;BvP{3dyKT zTZT@rlia)?KKS|M9g1~<H^kgM0c^0B@}&Bjg+%OB)n6fRgoNYC226Br|AckF08CJ# z+Ef5sNEgF@FDhkXS}B+ddKZRDNA4W_-3~Y<bDQ+!a%Y-;;1i{Od0BmNpmO2))*yoJ z7;Ni$<yifoql`cfYvIavKf18Y9~6u*m2C68pLo3DECA^2s0)q4$$s?$;|r&&F^eKX zL>v`uU~omd?uftOb;~I15g4UIZ{-0E510bUPk@o*8!G^`&=#aMZm8+IhaD>Z5-@aR zJaTnWTV0+W4nFmlZXspNv+lXz{t0MS+b<4CT+n)^Y7o7sf0Y<a@k62sBlDJaaP^`6 zM%*~@45lTc4IHGByKu6<%@64LNCvCV!a_1N-@i#sBbnDW<f_t9R)lt4%z-|&kc+rQ z_VZmARVA_BkO^XQ`#t|}0Kh;CeiOS&=^>4uT%GPFVIfyyaS2oUh8(-D1Fz{dk`GrB zvX`K8O=if#$0;D{8QqJPS1SC|m*&q1<j4Tr*|~v!MR&gfYq!O4+Y&x*a`!L9y_1RR z>7$R6=P>(jS9i2jnHveBe|?1h7Uev$Zhx;PF$lHOoStUq-+ZTV7MsV;?C#ptrOcy_ zMpd@<B+Nighv_D+_07|4kD%WH&H>@^FY%9vhzK1rGVuSR?5)G9>b|vM0qO1zLAtv) z4T1<FEeHZq($XE9MoOd`1XQ|f)7{<Oo9?b}q0c$zJ@I?r?_6GP{@F0+8f(r`_Zat} zD#J$i^Yv2GqtI(4`xUBy#Enq``KYfiW^7`@TrWEC32;-lOI)Ft?w_gake1*<Uw(k2 zW@@*d1wp`PqvT}B^K){IFuo$#&-em!e6jCQPl&<Wgyh8>+d*dh0$=*+T9ej-&Dv~` zs7pWAYwdjNX&dLcxe6=(`92lZ8!3;1z%JDjmIVId8FXcG|2dHBIFgVn&&^fx9EzXd zRJLaE6>m=37wx4a#V&;Z{jJ|cTJ8#giHUh;QR02`Ahz%J?}M;FY@aU$KMQipx5%&D zM?8<vudP(V6%^Hgj?)4e%v2#keH!P9W`8!Uj(|XV%Mw{$e{3s39!tw2bWR%1$o#tf z@an}aA<S?y7S_-nq~Ld*u0OYieZ21XF<JCwN-)HOs;CiaA!_7d$b9J@RRLDd(R-NL zibNQn<C0*VlVOql-q{!TqJMU{1$6-2sV@_BHy6#=C&!qZgP42hBV(q6AC_%Mma33r zi011&3GdIQibo&sMux6trrld+o87JGIBT>%$gLfC@uVhv28x86s}~ZhDGjlf{0FU| z8{#vws$1myu4gayU`h<iy+2`c>_Tpnlt#;*-@sf*qqQm%w?{uT61z%Y&*E|l2v<v^ zy{9un!UFH*@2$^Ow%?`J^9&0sv4(q&j~;z8w}$2&s}!6WrQhH{JjC^}+o@n*i2rst zKRc@7p({7cM}My669uFB4C7hMEvJ`XDRg-liK2QIs8j9e^qWpzm#<3jkC<>$Q0Rqr z4(K>AAMhPsBI000e;{%WP!3Uk^@D%$I_iQFH0S}hgY(q}JlsaaqA@#jzVDTG6`_7y z(*HBL&MRGr@^y@p<K<Ck5K(;tdAjrnXDP&k0ZCGEPnIVOJf==#SkeHlI{9oIBGMS~ zvpt`!hHa#Xj=T}ghulXDhyhxS0M|B<V{*^-8YL3!XFJ&2n+tH`+9tL|lz6aQ5ggGC zk$y6qYNrAjQi^%gi64#lICk_uW1ys@r5=KK-d|-)eR}>q9Fv5&i;5f}3)}80{KodU zh57_m#WF#{2Z`lQHq-AIK7mM4VdbqpTI&o~aTNjyLSKl4gB#~GS}V$kFcCC}(@0rl zFUV9#0gW%ITwX#0AX+#U=kJBJ*rq1IsDrx>ZRp^@g+u2@d$~+>7T$pD7$kv~OAI~> zfAiX)%g8cF3#X`|A$1iO*Od+>4zHa*B+=%Ig0qj7hC3O9Qn-a1y9B}<At(qYLi;G{ zXC`QWao5&T?d$sj@CC6t=4(1er0c*T5)<Ma!7XpW9vXb1+WaMQ(uR^jTjXThU&~DU zqhz+Q++?-IO`#&bNAH!;a&vRXbSOP~apBMmZNFv<aOxn(R*Eja;QZNm50vXx8DHs1 z-&Qg-R8>}TuQcfMjt@8c4M+~xxRz8n@%19EQ(h21=hS-lzB9ZkG*p3(Y=nbrC|?b| z2p9@<42tLrK_&OS<Ky=R8mo=M+%R63W*H#g8wO4TD%@ohelSR%Bc2Rl8B<)#=mb6s zu1O_)rca3tGd^1S2XUN7U^q9Y3v=3{*Q06}KXw7)a3*Vr#Q3j-tFxJSomw#LJIv>t zS@I*k5(~bxve+G8FGxA^gwW$ti)?ie9S<Mc=)1ebf|rOdz^`_=w+2<SKF<}%qFK*W z#dzJ`2v->2Qrw)Clg}smekg*!lA*b=N~1@oS}IYQxAEi};^uO{MZB_+Vi_vU{kaA7 z4Ikk<lQdAgb{3y56fj+b-#<JKkuUS~vSKeml`bH0%JlZ)e1WiG;i{`CK^k+bo&RH- z4qBgKWi&`I^&v|CjDI+7vcV%-Y<vm#x8;Cwj{~716<9QTu}X35&DW|^w5fWkeiops zRFN$?ZRBw-V0n($GuC)EjKxeW+lmbfi30PCe<U6~EO`B=$XO-x@81G0@N=pXCDJsV z+OE1uu6}BpNPy?=+R96fAn}li9(4TpwZM{sRN`!9U|I){fI>pzxBh<q%>Yy?*@%^> z>A|wk>5J3(G6M$$^$xw$tmOU7Ih0i71))27)|;>H)JQ!eGqIX8utR?MnI)bWotD7x z_ge4BEkolXzgEe*)Tr)8l26}k#F1j-Nb_YXD=Y0YaOlQ%L2MUgFit)Xi%%`{%JN?& zz^{E53{WX63bE8f?GmDfkrqKnRRN{$(_5!sUQ42}VS|Afi)?%CU|XvG@?cAXTfiU4 ziS)V8ydW9S<-+m{w2_d&CKqr}>%nFd=cp=fItKx5RmT~;2n`ZyFBC9B1HG0#u&x6| zyBz{)Ig#X^OLcw?35zeX#ha`>B{c8T6n5zhAohj*_DpH)C4xjrR<>`avx*&~oGCDc z>M6?%^wy-)d{}<f)k!zLv`y9%um#2F7ylrS7I7-hzDFYDPk|MJ;73x|&N=2}e}ZDR zb<jXo^wo%}IFf0>?dHmUBlwMC_5OUDdzAqe;ZSn}K)G%JjZhDf*!Y^sF0HugoHvTT zcN@drwAm>zgr?CoFcYwZ4YpQ9qb3KfoL_=8@I67AlUksuwz>EVN+A=~rRqejSRysP zun>Zy(qUDg1;uKi3VhGBhd<T4nMrd+@EtDh)iluRWc=w<=>QE5a^Fs!yPMd{hPP}~ z^c*vA;V&0Vy^3VfTD=WdQSad5ME&%|@0||(+TM9O#rHkKCK>hydVS@Pq?{A5z{F;c z_lRPujX65oT;cS+AMZV<g?eEeXIHZ6S)VN3^f!Pk=b6#=Dfst-UkJJgvD^aX>%7x< znY}|snCjDrac_J@4J3!EZ!u1GkhR{;YR8J*QN;b|tOvU+Rhm*(yT)P49#zLD4JK!1 zr!g_HWqV)8FTIWWtl82yNmA2nr>K?q&aqMOV+0?qLkWlz!B9dxYoYm4+b)9FIF2rV z*KGM4Dtdt27!SgOW=i?7Ut%s4jl@+iu_zV@b<$fOtJwADP#U#atN>ckDXuq~#6)DP zEl$}U=LA>$6^tsK0AmEotQ*z&ewK&&!>|~+hH;;NqH$dytgvjw!`Q?Od!Ay_hA>yF zH$auZAoZI9R`BP2weWpvv0r4BqPu}iY}iaB0n+nG#GB<S4G!5*5c5rY0u&PiZY>Ln zXtY{)E&2iniy6`HZCedPR&9yg$IqV;^f6B`rdB<^=6{@rzigRi*+KqVOZ0qMAX49G zLs;5(B^f>{@PWNkV|I4d{Nu-EgOaa_lS@lOU2i~!(hG$76j{m<pbWu?h(<UMqM|m} z<xl-0hIgG>3KLrRN=Jq;#(rhQWMpa=V*^HXM3{c$34EyapGf!jR%0jn%RKd_MLC0m zn@@Mz+B0}0Z8$9`D4AOAG08x(#f`@VfSa2v!Vn1C3Q`QNYcBKCs>8;%6}TLqIX&x` z+&>!<dw}<>f8-c??lX3De4f`AF-UeCm_s(C@34ICb9;-(qA$nXT(bQ03IBhL#@+J= zNC%)vq|^xP$2g7vD@d%~^#bYH`QQR19^Yii;3Li_JP=ZULGx&=IdLLHk*0S!s*aF4 ztb}Z{s3Axog_OlrMFkniV4i++CF-9fr;-QYAO<Rigq|qk^VAm#DCLSJmsHfOG4I|% zeMSl6EZQ%F&8jCSogxbStea*_lcy>XumMQFeIy`y%Eq>fjg~cL@!`r|ubSu*1+_^j zLP^v0gOOTN9o$8jaX@JkL)O&CrX$R$Y?~9DnuKWZ2EN3NppgEW!}QCz!gi`iZR|A> zQ$o!uce1wT8(?@@4?9Of{ENZ<UIW9syBy}Y;tbXdLYCUlSG^FRlSTK~6FiP+fKCo{ z)E>L-1+#Q2GG6KK83%@9#=f}dxt;KawA_mMMTyU3WCn0GclJ?woX{%sHG3s$rhMpb zn+wX$bU@rAP-#nO3G|efxB-}Dj=r@Xhx8Q0;7#}hU<Mx66z|L(q=9ng5Q?fLoWK<w z6j8yU9oI*U-q+4z>JQmrm+T_07qILS$dX9qlj2M=`G=B6XC=IxNGD|H?Z)(`Y;9Fl z7)*sfmmAF2R&mu$f27OnU-6DNl;pI3S999}E7#EY+h9gNyy{v^<K0Pq&UpuPU0nmC z)x4w{zkO2;$A?8hsTowzbi9r$c0TUs=cXPZncfpcIQ;GAG~N+LXSDj%G)2=p>ID!g zbOAl~v`xWR8^BMhrcH6Y-f6v)u*NjO=1lG=u|=@8`D(1_Gsmb8`<y>#)e_dtF|XlW zB3tS&%DSjDklQjKo*wl88{2U+5fpwR@BHF84}c3CIy%CYQtO*_$_j$M%8B|mzPFPn zMx=!g@aKFinUvP{ZJsQ#H?$4CgN5yB841ar`-(9O8}p9nE1U#A7M`><1*pFO@Rrap z){xS7l1MZt5O%-je9Uzz7Qx~KN_SFh`d_|&^JZ&<-T6@T)e?n3|8weKvW3eSaVZoQ zzPQ|Z@Aa3$irG@Z#xSO?lkn6tC2HiYiFRnxXb6dI)WxSPs8PR6*>!V^ojspANPpZ1 zMIK)#!23qXX(&<b&v?JuzIe8Kb*8;{sMU_{xBzo2|3*14YS@-egaeA=hxw^bn9@rj zZ!jghP`J_Hw$m$ts;t!JdvYG8oX<xMRBB()%{r>kZ-?7!P2p}ahV>OLn-b56z+1!Q z(OuUY$(%0wmv*(&fa_Kg-&P~l*6KQp;9dfc5Kz|OnZI|i9Bt%qH%?5K%ds4B^eB3Z zY5$427tslcx?^G?CVToBSzmYekJS20yE7K*h*B~-Bx!dT2V0#PoGgY?U0M?P^YAwi zo+HFw3guKm<wMUiL7A@k^~xokIpOY!wq#O$s%NEVD*!1y%KgDP|Mriv)7KHxkNsgZ z<(v#E>K_Ik3~!4>D>Sd?Q{694>W+Sv0H?hwCnr`Z4b(T1K%^T$-qz{F;MuH9UAH?t z5~Env%1*<CTW6)wlxd@=>$}L|x^`9@b~Xa^0m`HHt4XCIzaxCTv?|<OHVurQU2WKY zu8djXYzd^CT@z}soP~X#Msx_d?cvSA^_JyUrR`#S7*Q>XEu8R@t2rJ3&AKcT^;%9h zAZ3o0Mt^-~cOi$f=ljAg3GrnmmbEYR8Gi*&IvVP<m)l_iGeq20UrtO;*>vw!O?-@R zUxvLPsz`639^R}n_BP=5y?DMO?+$ZEfeVg9CTV?%;CP6eR3c|MoCbUiV7O>Mgxo96 zKI5!OgRv#|DaE>Ca=&(hhizH&E!}~gNDmJVeyK0!9U4*FvgUk*yZZ~y7USlGI8j>! zV{&q_WgiZpB=iO97|iCW6l)3#ttJvDp+C<rF4$P}T;hjXyo3f=S(%Y=*3qxwPa|pH zVTBk7`CME$0p@j+LDC;QE0vAn8U)wj!%6MKLaK3v{spCvHwjxqfKnwcn8*L%MQ}lT zqk*gIwPIO<3POz-f&x}rIBy*GfKSwWcVZFWMuB#%ZA`eXVZ=QYe+x&)%F=nJGhR>& z981XuD9US;B=iqh1zO*D#hRkBOG`w81ufto+d8qe6Jp}KqEocW`pU>0fx0=Yha*g+ zH9k+mtkI9zT5)C4GavNzm!w8X;dX*Oz@qlkO+u87SVD5aV$UuS?v0wNvw+50_TqKk z?EMz{1fQZ3G{0I1?HNiLR+oapYesth_<ih$<_gs-Ea_rjE%9y-R~h(NM(_8(2ZPkq zG)_)Xo*lra`Z4%7A84K)3bmfqm$0+2noa)F!a(Gn9nJ_S_V;>LrL!TGVFRGiem`2z zOOAEV_i0wK8mA$IalZ|2)$0NlOPv}Tt^&dl&di@uHnYY@M$^r8TCp@9@q!!!R>T`y zSc#bK62vrqU6#q2pcbzfs^wEulxdJSi4H6X%PHsrKTZh77E@7CqJpu}$YS7|he~h@ zvJMUyhw~?Ef3O5_Xwp2f!Lr_66h3!JuZ^}J?!U}w1WQ-}6>AOHV6BjLiZX!pNSsOw z-K;6W$a@IZUQvb0W#l6exHcbNB$&Q$j7f~2ShOz`Nth~dOdaGnX>?1|9E;D1h}#|H z8sJ*sdgEM>E2c^rJY6v!vQgXY<`5<^IWf8q6DQ=JBAR3#Nv0!Y{i}ZCvQ)7gF+oWC zjdw0fP-4}rYyS4g;<-hl&Fj>{I1>u4KI6hNjv?)RJ-<9Shic+>Djzr3N;g=Sq!WGC z1%LC``E$wVd$Rut4X0{)svQJywEyUps;B?M>IxKKS)x}s^Mw(O6aVYFFMxYa^q=B5 zfQ9b=U9JUu>52I9Z*^C5r?|TuP#^eI+VwvbJdMMdG{DVO=47P)&IZA=V=m{9JS>A^ zf`iGCEh*fMKp2u}Ph_A^ym3TkhyT3}7}z#IQTwN};wd`r9h>xP7=ZU1fc4;=rA#({ zq^3@0{P#Zr`6%v=h|UX?xe+A4L0jB~as{{iOum~V0Ji&wUht_JkIwETz+Cz7+<QpP zCIh9~I-w07%zlJ~bW0!2lc}le`eN>Y0^~pYLVC`JFe?nSx{3g;t`ISfS4NtmX4C?H zW_BYdX!$?~y62xtza}l90P_EAUWO|DeX<g=zx!)u%e?XAY4UOF;(}lIQ_xW(e{|1( z+2|uqAt1Jss;Tn}-W=6An$j&3QTI(lu?F2_=fW*fO@+v3(Z9v+Myeqa0f=k3JO_Ic zxM?tFAKf~+W|;ofll^<O<3EcZJj*IDvW+gY*fjrqp+b#f4!3s6kavJldB!>lpa6pd zVi_9eMc0m}<Dwpi(G|Bl_wIoQ?_}m+E6<la4SJN$nu6#LR3+A6-wcROm-rfdXGM~O z8t^V(tu7#rbAO3M=sXzn3lEmBaQ>$tQlLs2%?*AOObm39!T@jZzcNz%JwN|;@b*xs z{5tavW90gY?u)&b6c$Tkc(@Fr%%O06=PaXM=_)rv#g3zi%h(gde8JSOIZG#PbS6w( zG`5<l$<fJj9`_5`+?HGu2;y(bT!K_$r!L8ZwueB6nfBOgJe)=^sjVvS5}G-oJJAqU z^6Ky5&#2+o!c4ZkA4`HxNC^`%D*dlL`Yzw6Ro$hEBWY9(#MrF9+)&FDJ?W;^UCepo z%)n)z)Ob#wv-ZNsf!ikVvrvH;R^<PytnvT2Cr<DVr{WRU?crmOC-qJ@+bdp7R;gRQ z-2pOLXTOEdc5&~|<PM!jJ4!esI@`EX3L8$<-ttPhZg8tjIR(+>hC|CXi{O7-CmI`C zm^^@Zod#5`=E%?^mnuCf)O|jeTi8ZN1V)Z{yA{Vgq}#at@@$4emg+BkxR^GfOBiP( z`qtaI%p<rd&Me7-KNgE?Od`Ihd+p3^nJbV7VTk93D=lWXCTaJwB<5!AXp^YA!m9*X zkO*!4`hATNJ2j-@MeE?}i``f4<|ZC5Pg{N_r{n%J!<d#c_*GX+X}wz6nO<zT{aspj zU)DRv$8PE-d#?_jEmOVNjb-SElh+9utc17QU!X|56-z1`*B|64Js+6VFX~0)pttqz z(8I?4%zK#Z-*^9Ok<uZU9>y%~+ug*l*Fz(#omAgWqRC-nC@<0<uAATFlZ&1$IiG)% zx?&g@6w5f-bam^p_xJMj@_D$Qf9AQ&r^rY<#=~cy{;k(!8<GB&vFR9NV`?Fj6-483 zuqkkk!TugJ%OqY2Fe$DR-REJ}Lm#IXrAn{I6cb*gW9(e%@9ktZ&5XU)v{4#{j*IS4 z)b({e-fqL()b<W_DM{g_-aeLJF4{QKnx}2)bJU;qUEEa-9ViW*T^w-N3+KAeYB*0h zs5`4W)(-an2LAOb=EQzyjAuM##gT>eeGdE57Yk?6{Y8NY?8~TUFTSF+epp@=_26%0 zca4SihN0?W9Lgz+K-26G1hMS~!-G$~);zNb`ECVBbYxfJ=fj(37yO(dI>+&IN!TBz z!}!V%NB1QP(01F!D9k$8J!g;ftNGUs2Lve+PDL%O+NM@YIciSK4WcqZq)Bqi@gKX= zx%D?ecVRtJ(q8OI!kyCdbEFFa`pNtH&ns3~mz<8Dv+!Q7?^2OIcnnBkySIHAxcaem zi+h^-=yr|;I=ARbFLDVM<DJs8KCp7#FP9;3W8%WeXAWc{JN(H3byFj%05PGxC#-S@ zK5_983d2#rld=I4GAej7rF2YaOrco}VJJae%iAo%Ai?oK@A=WMaGW20x^npa?!<PH zc3!0yPCCxiWX6zHMg~tEptaC9kV<U1t;EdItVVauC{;Lx_sx2V&5nf*&<mHWxF9xS zl)hN`(saEz4|!WQ_k*K`xg|pE{fz(8b(O)~vC|ovV#etAt-iK>7YlFHxT{JUsvNmV z(tal5=AblIfv6F1Nh`<$->j`jmAuBoVtBlbK`H6W!JCS;y#NqoYb+4Gg-=lP;M|Kv za6a&|Uodj((m?h52gsh`iF-%+7dpr*qV>i)xoo6CXs2%lMN0o}hj3b<md#GGq2()U zb}vXhe7=-bFxx?pvqE*SAEuQNQqNvX9*A|Ca3Uc2QkFyH#1nKq=$&m%B(b^ZW%q+z zAJO^MjLnk!?AzBX<&CarGO^&F9FCk^QbBJcoYjaNL;=ZC0+L0Y69qm){>s-PyEP+= zAv8-1kU74L%6FLnDWb3aWk&kah$2~aC7|}e=`0X-O%++SfvOhbBz$$-tz}*Tr`8CE ze@Tkop2A*d-Pi|eh9bd9wlrtFLI$n-4gNP*bKwI5Nt0m2eS2VB^t9DGqIq(gu5@e5 zXvp0cQ*IO+3@V{w#6H+rTdG{u-tFXRaQ&Mzq>44MLf7DZPNzM^iZw9UO3SSVy@&AT z?X4|d$Y!xz3i(`2-7#XZhKE}Qyyov9X(C<9&*0)2CCt9j`nId-a$*YXR48f64C+~! zzl0mwq7A8(g9PP8m$Q+30eREnf?uTKV%cj%EN|myuQ95PetDDf$<EkK&Uf8xVxxs~ zeBXI?5mZf6SfdGH$ZM;!sE%Aa$W~d8!_2yxq#{X@8yUI%M5Y?*H&0<ybQnJzaFjyv zeV#Z@q>1r=ae_m)TQjy2m5!h3c=iXz(y@w-M%YMNQa`i8sBGD_q<2H0UrwF@S8kqu zqap<feQrAx)hXrrNq#ve%gF}{DX_S|(r@zc5&N*<k6Y_mcCnz_Vja`EoaQ3@VCjy^ zHc!%Cd{Jp=RDx~+JGVkS7VtgH^XPS18+|4zFQ(kGLDl_nXwX&b_v`Xu3=g{sa$)Ma zC_AfX+Qx8Nq>H{o_jAWR$4Zd;CV0wZhn&wywY2xs_#-b~2r|yAyw$-S4kgdEc0)9| zfW6lu(w*>UKBu8xEwxni1jp2<iTWGAAlI_)J#)21^eQbUKBP30sk+8E#g3g~zC73m zTV3_ecV#8Rt@3KS*kd6o%|sdOEV~MR3Le(8PrnTk(6C!XMMHZ-y=^F_l(0qBSVCy} z#U0CFCnM){U;#mJQN>{R{3nN*C!i<TEH8miKqtH>{i`Pc)evg<MvX`fPez3R-(#Ts zQkjZ+3?s4|W1KhAP`QHq(a4_r!NiEWD(R71WTVb#|43*IS?BQi!_fJ_71)_MYv_`y z@h9P~q;ScdneahnL{`2HY`(I?@}U#YTDnu^$IBTuWM#oI-b+1vr(|lh&e`LsIUQqG zQ2I36UB?OH)ySq?aQ$#lzB*Xo`^O+>Lw=I?vUb#Yq+;_ouSA}+FOanUO3_Oh*p*S~ zz`95<dRR}UyGpdztrZO(ptrnQ7Y`fnsWoq1h6=o$;|ZO3D8fe|#^9}VzzW}NoaaBk z?dA(~3&a+k3KP5eja*v0(9OKxb>idXvAz&^W^rF*xFbCJopyKU*(!?LvF2{lvTH8f zK<XOxoi&nGSE+-r<-*!#BV<tkEO(i(MVtE*lh!#<TKtn&f@UDUr>emJLbe!@E!;Cs zWWc0&x;cS&Z0BpHbIHdSiv-%<hZ<{0lkjuPxDz3Rjq-Ocrui;;u5ykoG-bHJD4wJ6 z8D-y}>jxYU)ix9>E*ku;Wo{)WX2MFFX9_<DYJC<q+p1JtI%kg8J`|c&hG#*Xh#tFc zF^`h(S!;B;{d)rquIOg%7=Fo3%COXk20xaS!`~{+*s^p)I7(O{DekrA=zs;jErc~q zkFvZ_k2G~yyZ0Qp>Mw`a%n!0{ba>{jcug~S>AA%z;AazAWpLSp5sw6}HHN#o%w^NA zE^se8cA;K~(6wJk2=lyZn&Mx}nDOHIg(@~T!jhJ3K83KHJQtSkQf>ISjd9gEckG;; zjQxE+pz6Tx`vm5iGtB-~xH9u-@mXfx<9UPNHvZ>BQf~<8h1ghq)UI;l|3*JxgbE49 zyW8<q%e8tfT98ZGzUMbQaT}fxrAzZ^VRVw-ax!wkTWzSKhrp$~@)#L?6!yrfvt=qL zt;CeYwr@oA<s^q5P!f)?ie!^YAg9)73-cRd)DK(~alXX6FtquUx1O$N^>FMoGda&0 z*cqqt67QV)SB-M7lJ9QsK1=~<K2-UVcTL<M-Pk#h!>me-t4nV4>|@WOiO2SgAC@Y+ zmkRk$Ycg5+^NhVd=A2^!v$F8SvZT6K)kYs(OxTCBjl|6dsQ3^o4Fa6G<J|d-66NRo z=AvC4pIPO8UD*_(Jh1Mp^U=6U+4r}xB<%eh)Zb|LLvN=IWl6)Mw23$^ny<oPL>TK^ z@pLPwKycQ2=f~@vHu7Z`xd4eN?!(rM21KVvEQcW3k7ddMEi>W)61%NV$^jA<S*;nB zfQFd;$?+4gWek(je;o%`_-Wx|{E^QM<5%nDeeG65v!JD~jmx5`*Pbd!n$Ap^UBXOg zb!pcG2%KCf_1+A19+3vElPeHsjkgBFb%470@T6`qR$c*1tkyLB^-P|uWQRWr_(|(z zG1#O1{Tr@?2m{TtUJOJZZZVA@IRmbT1sSawj=-k={2j8^Xu#gK+j*=1(OiGL8RS0} z7tmRoM1Sq`0WDgU&|xb6)1tsygn)JnlM;AmFq|ZCPz?O~-={L>%=~Mz8G6aujw4XO zMSyg60W<DsP@mVD!TVPZW>$aoA>Na0{_OPW+k+k`V6qrTN5{<SYGnlQShZ>}bmztk zji3DBkpH?Y*^c7$5V8alz5tht28<(!tUi8ZVGk$>2{HAy$^HMMe(_`yjRY1G*$tw= zz>EV7>5lfyt5%t_Bs<bfq~qllDLj0AamjZmUj|dV+H&P0!)bWFk5BPaU3#SLu{XM4 z7ZxfPXLp^RisTp@o4K96s#3u#D1Zd)+1I*^&r`!LS^_dXf0Aj*f+`>uU`N1XR`C6G z4?}qqH4UQDyZSRiJE`9!@tBYPkdL9W_)!{9BDVFKXx)+Vxrxcf^48-Gsf!z>BzHjF zG4g!V<1_i5Uq?dFFt8Vl!14yzCj-mD$w}Uo0Y$c}v*k7;U9b}q6oj#MmY9(Dl^*Xm zn=ZsZY8bah?>&>_J5?Rjr8XD=9K5fA?w5L6{M-`%*H^(++)9NmGru%8cI$9+Ai|@^ z*&j>y&rZiA<$0JCF-NxhJ&?gPCVP%sO010wCB)vBkzyS{o6Yg^id4_xHN@lUno{D+ zXQA0yt8R9NcCovwErwdX(1QbtAGyZOVpLdLUruGDzea?2%^i9%T&Gx*8P!6qp!@bn zG94XXyh6iw`K@GRHVU=zO99dSUwrOUCNvN`<DXNmMI7P0`JQN<cW!nrIw8fc;Z_ve zjT+7Uf;_4uovb=FB(x>u2ni5d?=j>cg1jD!Fg>{5@m#>+3P~wFL{dkmp6Hp4rn=Go z&A`uE)mwk0-Y=Z<KjSYYP8VGK)6lw#7u_(`Cnn0RpKBTn-#p#B>4RS!W`Ie5dbdwz z9=LRP9l-nkbG<{kx@;lbQ?1KBf~LnEueHjpvt3|JOpJ{pft18T6T6>*^t!fxKzdv^ zy8rd5=w4*T@)`qkar4c<95#U+!u`E!=WVC^W$DrwDWm!JY6#S4Df@Dz4n6tk7Bhai z_+h0>AU^q05>r95lp&xc8k_Si8^P0^{7-%ddO%C*2q#Nmr}iIR^Pg|!pe_oSr)sIr zlN3e{cEtv#I5jqwH4Th+1%)S~zz)GR$oq6B3k@N#<-wA8(WRp2g=@iY*%lI$H)49y z4zqrbh`F+1kJQ6xZiwa3Y!CENu3@Ho_=;T*dHZ@lTsN(jJgQ)mnYyQRc?{%n2y9~^ zh+-H)v;mdq(-bn}6)_JVDtkxHMsT$#K!}Sp4}(o8D@Pd40fn9tgq5axveEam#GRC- zuAV})W;aV^7|?!q{S5*z|3f7OMLV$hOX!8QE|m^*+^wcAwM{)D8LqCDTCdUF9nm*Q z8Y&1Kh$Xz|=2&+jqTuE4bFR@?t=bokj{5FbcY^?&Qkkn?uSDF&Sq#Vbe=`*<)xfJy zIH#e|x;=P+oDOEg*67>pj!*C*r;CWe%8EE62d+j9rN^9%?C{bNjQbbX+;MxZkOzm$ z*Eo?Y$Cn3U-yy<>>0tnId?epcw%NkEZ&7mfu+c&0DQA&FgqwM_W5OryskC&iA(^B` z4A&UfhulTsG|-?zh;gOh!M<~PYGmoV)mx+27ReB)_YDvh3!pIK01Ex?&N7!~m!kfy zVjx#E&(tmc+G68YSiOp3Y|1fnVSb6tK7GSWY)y;#WtXR=&7ruQX?%g4<%D}5k9yVN zL9uyq(~`-+Is@rc>gJWz4IeXIT=>Q(SMoLGEu(o}fDkoIp!ms76oXN~s9(1oe>mna zgsuwdP5x*s81S5VsBKR@$r2%5%&x^lF#kHh&LS(m%qcC?70glNwPtd6rfEaNNK0O5 zW!25hfM+9FBr)xBNP!X>dYpU-YCa$3A|A`hHBGpe=<e?AOr3rI@+IwBU$KT5{;bL* zQU3tAI+8t<O*iU`bVuh(@_BOq)@$ln(XfynWaJ^4w!pTSz_j3qq2h1QN^x++>4o5H zx~7G$p^QH6XK?Dg%HK_^aCYPMQU>P=lgy$v%2yx@p}+JG_P>`7?Sw3<d&K`B#CQbk z#pjSAjy7R#p7K)=@c8jokA}5PXC*t_mvJjTsh8TErKwp9rkoW6Wbuj%OC2M8+yI8+ zy~TTV2Fj<6i?N!JbdTHa_^lTKh^cqC%q&1+Ga!(e!0?4b#vT-yb)e$4mRo?@4zce9 z?WWuU_Obxs7VO(1F)bwYv07}YFZgSua<E-E_aP&iqFCIcsc_H5x2-RWLm(g7^T|m* zhXkid!2-g|%h8z39QBiyc=oM(9yKw`&p6Rt6INWOy?pv6bje=l0q4zSXP2+_j8p0I z+&&UfHSUeN_Y3NhUUEBNn0B2M!^vF(^fBQ}rj$cMm0T=hUA!JuTI}q<AE#RE^tU95 zUgG_%>$mSce@;q5s-meMxjek4b#wmtdt7YT3;!1~L$Y0~$>BQ3Grf*l<zCzvh|haP zHyfH%-8Ds?H7%6D=TBL6Mtr!n<+?jl9m460O|@5=q=dL^myH|byKX`-$veLpFD>t! zpX4>JF<p@^6FDq$OdUoB%UrHteI1_^1pJ2S+|ofp)IrILSYl#5-Aw73gJXUO(ZPXQ zI>bdXL};<WG=JZjf7+BSmlq*VGK6dV$GLYy#AT=^3T^K_E~=Ov8PlPdM_PL6<HTrt zpQ%9|%u*1J<0ItBTj^EQ{@zMe-0VA_DqP9H$co!`s;hhX#-L$@Dlx{Lf&$UwQ=FL8 z(5Fu(*>!cYC%~CEq`f+kKc?QAcTEA|23+~S{2sd$1u_=0tlw5q@ydo(;j{}57744l zk{+Mn4c6+u$Gc_ItvgwGVEB<^D*scuRjH(I6Jr%O2HT10T~Vy&TmQj9wwWOhHH}5C zrSRDEsjK@{)Od=+t*|w8d#eLsVu}};M6Mz-3Yy|lulE;(Vzv}}bkrQk6#A@1jJ1q_ zAXYuWf*W%sZzx!kaLOlpB>Mi9dp){lkzUMw-W$QeL{PxSN6}I;=iQy?l%$CAO_J*T z-y13Odak>C5dkUEEzakygvOyk{LD-nTFOg4^e2Y+0xTaFj%&kshw#!1E55QI|Jp`5 zjz1TUZ_R)swn+5$2xPKZ_{JreW(u<5t*jJq<U>flbfNB^zeL<5{vu-2k4VoD5GeDp zDU#A~4d?jCVC-xRwSY20`RXe#4^J;2$*qmXykA#K_GaCLvS?e8-&EodFM|2iPBsUW z&@UQrUI!7v<cWS%j|>m5;vg8m3p>R?ZS`{PWp+D80`EypU+COl_MO;q4I1Jg%pi`$ z4DMXU2^<`v21t~9wtkHb%Kyv}rDOK4#o3$CWT&@V#zqNrlU{nLuIfGMibkN<>LR!G z{S`D9O7i6G_8I~cF2OOX2x=q+X8k#Nh^Mk}9O1S+qns6e4%zsZV^{rLqjP?@#*=H@ z2xxZ{`W`M6dZssu`e{K+jzqA;3LPq9A`#~@L-=k1M`S!j$LtNyFZbpmd6%rkB@CjG z0_=|E4L@bWxZUt3$_?H*vw?v%C4H=@MSo;x83EQBQ4i5`Wn+zbH+f=m(K9RkBdNT~ z(NQ#&<!l`sqUU|7<E=H>i7+)T!*mhBhZ=@71$PmV74yMzt1b%cnQ4=<!gFDd=gHOk zenZY(d&0{BsD$jVZ+kqR!v`f@T>7{uS<N(}x{E@j$<`|=d8?_Q)8h{`zFj39Df9wo z&p#iGD>08Gw|wE_6_s>QB&#}{N7auR`rv+<EV4&%*txwgd`pAEr-yi7VqcAT7fI0u z(7MVDXI)a=X0@)O=IY(qHSbb+yt%hrXvCe_`jRtLWMd|+s0*u6@Iau?q8~E;RgqI; za$;gltK07GhX0VJdHneP_A*C_S#hath|;qT&--TSVe?i|2We41)UbY_VZ**{%jmYK zK>!y&JP4Cgi1K0f=wUF3Lx4O&G$p*^uBqRjkY;HN!Qx<Rx`R{Jl|m$9E6rMKg)gGg zWNaFU3;vrc?cleT??j4#8am?1LUxf+0-tILlL<u=5j?-19=p{~aPhDe5Q4ydoGU?W zi&ZE4Bs`Tc2e{-wd{LHQYj0pX3L%Tg@}B-=o3V@kZQ+Ntpw|g}@plM9_z%IYFCA@Q z?hVGGiNyxyz_Gm)UEMklb+qCL5Bt)yFhceh%v)rGTZF+QSCjj5?~um~0=T5-CM;-I z3!~jZl>E%h9cFE()0TX?d|SDMiUumB{u`@;0tvYWah;o4t6|-}pdbs{oNtJMdBVhA zD3l&ooOf^$p?NeNJ<OU4?WF$HRH70JulxEhzqFShh1Nb)e#7Au*vXJp`5y3mjs?Xc z4HSfhOB%TuNANv<$){NV*(wfBI##<t^y()q|6oVWBHh@x=_#*{PxYI^%1)9;8!6FI z02j{Ieo&bDn@kt&^8O`A+-OZyTW-Hi8Pt^s)xy5Eo-Yu8dt|pwDl01otomu1aSX)& z3W#-AvXE?AGU;<ZYzZpKrD4NF-YdM9%|!~^qc=h6lgAW8i3#cX1wA-u2@h4cJHVu* zY07|>YEs|w1QLhy7z1X7WQ<?EM>@`OjB<~|HRq$t;i9K>3MdTdBUyR4M;uj774k&T zFV<teoi^?0pl`m7Vs|<?7Jv6SNs-NfsJlcJ`{?LQ;_BDfJ3Tqkw?7vwsvh$mADop< z=BaJ(T0;bY^B91jdhiQbFl2ho?%A*H#KE8@K8VdJE82T(AS{gpqS1i;w7)UwQwkY( zzJK_)Ndd^<u+yJ7o@Je9cr*~38^dYwp|@ud9Gz4Tov*-bGmcGgcpul2`qtnrkk;2$ ztf}M>-~$Eb9t73A)emkbEq%q{Zl}P*xM2btx_ingaN$8iQoMqN(593s>kd%8zY!+0 z#p}~7YUxU6Z9BnV3WbT1HPv|Qvaan>b!NfBc^l`^h(_ssLkq#Y$;`>w2ss^PNR3J{ z94sfWX}Ww7m$1fXv+%;=x}^1+>gdtNiTaht42t$t*W8qTYw5@`5ox^7(%pen!HtP1 z=;{j9qhW{ihv}!)_dSjUI!)M*4`O$6EFT(iv;}Kd3ro7%P~ihQ457J1HBM~=9#=i_ zSEDJnW9)H2nDYi0DdN8J)eDGPoWWKE^siEc$U}g}#_RqbVtK}2(^dA7L~&&IGWPai zE;|8f0_Havj7FSuz%Lg{>UqH&EO<Xlm-5=IH|NUqOI^%iro9UDz6uG}V=XKk)p&>W zNhc*<KR8?B#VbXHI8fM>iJ15r#*|}@c(qMawTq>FlS@+az2{~}x<$22T6mzhI2Aw; z=gc-~(Pp16mL4>g`)EKZ!$7@1cNo%a^kNlTJ^?e-^Q`2I`<@71)H=V|hU2_)==*7? z;-}1?NpWc}sA18g`I(rGs>N<<{ZTe%?Va4Ke_f&?A)!frd7m_{9`Bk+$;IA<X7EP2 z*8TE8KJ~53kEo>YNxou_bChp}s*JHR`9;ZL<Rax=we#?_`7LK)D4Tq?NWps8Uiv_4 z2RK0gO%+w}fpaNINYiJ^MXYzFtEaG!iPZia8<lyuYJ3EGI{M+or3n9C29`Gu&%}hz zO|OoBJwA+{4PkqSVB&g*zrGCR1YPfOBtbeI3`(|D%8Bfyl>I~EDW>{DLf$mrQhG)t z^E2D#*CBX6Wa}?~YS-p$jEjwJ8@rA2$4h2I8x8K`MV+!)q)Q{h+p}IQ_--*Q6{SZ; z=~bsTX$uUzX$>dkw|mL)T%v`(f6L_M%a=wop}b{$8tUrkd{zqq%Tc&BI+{VPt6|fP z?&<!X*)Tu1M`-pI4q5eRVq>c}1c1E3H!(L*(NYaQDZdrI#Ji_3k^qOtpKIJydY$+< z%0#Y1o=}(cXn)D`Wolfu{)(H8*MpobACqijADcfcA-&Su=SIT28}_dtzi98u1vP8U zr<ShogSz{B324+EEXll%C+n{|njQ$m9^$<_zOid&)a7p4d67}ib+XPK^NZafR~^jL zFV14f%5}A?+RjYnAC2iZ((Zf9$#t#W+n~UA0V54lLWA*q$4bL96`T_vd?+^juh)b^ zqD^ZFjIxM`wZ&BL{p0=LHR!#M@c)5#{OOZ^hX^;zuDkb@^aaqPF~>lSK!(n9uQt2p zapY1o<kFdv-b%G{J<~;>`(vf)A^hB5A}-SJgQOwOplp|7NTJExLcQZEXKz&DSet(n zG9>U{X19nkLl2R;8QG-ro!sI7$qQKflcIvxT+Esx*QQ{P?e=Vr^2Z+GY-Ao{<R9<+ z&Xm8*+hlu()`XT5rMvJNlpHU5k{y8<c9|;^VNe5jNE88<Z){3s%rHgE=?a<F;bY_2 z%<Wa&Z!6pTxBeX6@#JaT4C^&W{t+i7-^|tef#gfPejx!I8+swyJppcfFwLgW`FVJE zZ!b3Z<k?gMmm~3O%<%G&)d9Q8^}2#mbu~4Xa{=R{a|t~O-<NKaDMQRr79+O(QF*(Q zHgV^fPx(6I7ZNnuhK8X?hSfHjq8GY--Tj?;`d+k#A=&x{w{L80s!}aAYg4JIrvi+U z-RkS!(|vjS@{>0YEt<w^HKm8;Y`@2G`|Wy|F;^*dQ^_lTfk?PTBqN5A=wI;~X))w_ zG8<Oq_dHdWb0J5;+>zmx{fG^C#I56xW>{7G^Q>D*j2_6!ecj^z=1VhixK)Q;l95JD zdDSi|IC|-R>g$ON+WhgI#GBU!V>FDYQu^F@79Nt{&Qx?MBOD#K20z?g+|%NueeXoK z&L-U0MFJxe5F%^~x~+by@wYM-UHVnn0Wihb4iD*f!g2df0V&phf#@*%w}dQ$94JKm zUYvq*{-!T$5UrWfPHdKJG1++WTIy{v<7mD8bL71_w%fLHjpZo~WP0-i-j?8^lRJ0p z9%&!Vj|H^2UlM&j;K(vurDY-4A7<#}RAUJ5IqTKAnLt6_?jcOt+%!pv`1;NEi%8m* zk<1tIl;n8&%z3sCjns43lHy<|clY`N{hIZ#V%zeQI3m0EIy!cAOau>dk(rs9DiBR* zZMV?c`*&!@3hYE~&%X?a7gcvSBB$dapxTY-GT%A8d^g6`?lNq?^I++~!e{xUy}kz^ z>UT4g1^A~$Ici?74pi~+fnb8}?kJ0AdfSl+&}esdy`?9OHu#FtyP}5g*xEjf)-y8+ zDYXlGRbQ<d-;J>^%9?8n0)r%v!m1a!t+vOLJaw6)_74UZ+;jGG%~UOBRW^OCL@w<I zT0IdVPdURx21%7cPM9ABcY*$qCR)lUpSgXPw)}C5>|D-g9T`cBt39_UklA~^;--uZ z8>tnWfQB4v!N57_d5z~3NIS12cQfL97%b&!PnmAx7kshn4Pn(7Jw>t1*5w#m+wTTx zk>vFvpFG-x8cc3hS?8Qza0obb23kyJ{>!VYOO1AMo_#v3g7)ig+NlN|c()3hbe$uD z!@hHcu{Fb5!K<CHr<WZeFQjd%H9orBU9*os0;PyVh`ruVhG&f=>>?R&t*>?tpdcGn z8x9CvdwWXJfUXT|k-X40dP?kh(72>Y+alim$j0hE85@RJZRwGr3i^ij>fpqMuPm5Z z=0wNsBBq6eMjtuhonufUxAv$;8;<9|oSYG(fZ?Egj)a8iKIJMSKY6w?IbIca()&wb zvSYIX{pZ-9imDyEC{p?cq}*ir;)=DvZ<4F!697mZUq<^?Nw|e2BnaI2fFwfjd3Gv) zm9?g;O%-xy#?bJ2Ryy=URRjj|5XAJ*3-H!xtegNUBhcMC>iA+I_q_@w^<YXzW*ZDO z?7Nq!7@{Hg`BNc2-F#<3V%PYs)Potn>tts?2h!!_k<*(UktJ6jNc)<*GiKoH*951! zi<B(wW>m~=Z19&3R5VN#$A{#(zWc&7S+KxhfmH@hMZN<~iT=>MUh$#QUSgx+Mr`uh zy2Qz?YgnMe)tE+nD=^SM7+q<Sh!~iin@74Ebq-7(HoPfvaY^!e*;MitGTQ*blo!;( zEU?E$VAl(L91(vhhzcGl4dywY7WhC^;Y<T!#D5&pe>gfW)$o%o`u6P|%eBX^0?nR> z)Lpqz_w=kCMFaBR|EkiX6TfwNy_u>mf}(5r{EOlRW(jpFrXA`(m@ck0D;kd5y~_G7 z<+gVY4+qzgq)OBg2_6Dt7N=V3fJqyWU`#<04k(;+Hd^RpE%Tz9!p~}n(0kcqzAfOf z&L9HuSoZA31R9$NhhhUe-D@=7!A9Pvt-`z-YOe8rc|4ne)NMb~tw_V)(dhp}ZI5f| zLz6jk)xmUl6s`e7y`3r=BvD(Cn&chPr-crBx)(xpN+Jl`tdf|WtEhxOC>Tu+F0&9< z;43PsLZ+w!-jKLqx{$XQiQbl)qXx9N5!V0-6c)VPSu;0_jgL!S>>l-AF^Zi6K*9o+ zP+w_g5EyQnSy?zx(RN9@$2B*nzEsrJIWVVqo5+$%I@i=Yr!GM71!tGfGTO<t+MII6 z>D|%l`;wqYvNp~9DT9rq^DCzXI_y5ERrh^Wgc6x3fggLLUigFszhR}Im?##zel4BV zIT#VM1AJ6&%)D`@yL7GvMMfqTRn@{IPfex1DpkJzJn_C3Va_cy?vhd@%FEZFM4KHQ zJ$zNZyAn&&yB_9N6f-NM<dq1ewkaL~V}&)x`DvkG<dlT|e8Eo_h{odgu4Q~iOwnO- zTVeY45@m?<;lmoFG+jYqpjrR^C@(Uy5{Haxr0@Gm3*$wRcD;n;_~!b$Lh2yiA2rd% z>fbVE2faYmFtCk#Lf-x}x>JOO@FobseOeoBQ@`2YwJB+)$K-FK8o=S^P{HN!`mMm+ z8rwD505k`)<pzL9qkz5RC`(&VkopXpM8;fX0OX8;EYJ8w@#EWIB2!9U{_M<Bm9IZ} z4n~9{5oQZgbg}P=NWQ1B7MR7QrxBkitP8y@MV9~h6L9)YWkOb<P~&dMNVn7>{^qaB z$`2x{&>l%Cj@{r^x~cHhoA-BO@V$^b$(K1lUhsjiALRT>hSq%i>ym#>Jta+a+b+r` zD&m1q;VIkIzOFW3nD0(@!o@23TBMkf>V#XgUR6<;%#BNT)~tvd3RRx&Tq>xcwWu3? zEyB<KSUTLgE8;Tk51oDvXs`&?yrED{s;LwEy{%d)CD~;ysC-JO02qjTPg-A1>)FG0 zMrXYBC&P?!x>wr&z`0omtCzmIQG0s5d1F8gJF1$x?*?$Jag}{}Db~+^i>@STAco~} z_v3qZ8^lHUt8GC(iYn$A*^i)gAhmoQk(~xir<I2OAf)bF_L1x>oKqjAEY){zKAfEY z)zPDBN?}t@Qb6h^&lUPCogAgt%F2q3li+9`z*H%mu@G*vzkR$!Z5b~7ywy<d2gHgp z05}F@9UKj=pU{+u+x*sL3&X{!IvOL+gqWa^Nxpw@9(1jW*5y+d4T>wAm~U}xX+CEk zGY#S>@bQ*!Zf?>AoP<RBP8HUEh9$Pz58J+ivw`&JyP5EP`!-f$5gFu9onz<G3O@w| zm@0bxl5W1^)&>s-kA<xnyLN~Mwp6^N7a1IA-xnPgg_-{i;e7&I;{Y6a_YcV1NS8ZU zmyOF2Wrwjgz-oQV^%p@cOL%K=gug5vzpKOPuywI;TVk;UlM?z>(o?Dp;=Wee2({~W zEC*~!d({W`m)<aPelORPi`f4lRtJc;Huwl1B88E_;H|I`a*CYRWflO&{V$L$U>TFx zFJGEwu(QEuqhA_B)+7G9wk&jNL>}-7^o<pqmZVeVpQ~qHTF!Zd`C*V<XFGp<ZISOU zo|CEa^vHOZ{+`u<>3zCW^VAMOsp|jZonpgAK=%}Mg@Uj{qJNY8o814#<tli_5i5Tn z_y0LAW5Y$TKOJ#oKx#AQHxF@G+$Z>tO+J<2{KbF)l=fe^clrJ;nas4%S$8BU!s1_l zAw6rK{{pB}{7+eiO1oGN0gpnwi~Ar>hhP7I{C05vAl-kU=YL!FDp`x=5cPjR(Tj&{ zJFFZIj#@vi7%N8nv;64*1OUe0+IcH07ZY_hDywdN+?jYnkDtKV_ixP=`q!v-U4L>2 z2>;}$5zi11r&+VeQ#o^R>{-gYWL^HW7;?2I@C3m>Vbgz6X%Bl@i{0TyJN_Gc)Z80U z{(l4SIROlqeUMOZD@JJ4=ik9!zt>={M%QQ+2>`<b{=+v<3&`$}aJm$tNzw9iINW!R zFU8YRI2V$o%Eb3?o#b7zud8YJj@)9VqZH=;mvuWy4zsP=l+L!+^ZyZUGw^g_##8|E z1Z3*C$ng<O5ihY2P)9sZzM_DK3?Mn=4R(wr$wE7CdL(V$^SP<&zerrW>DSk>EW0yy z+ervW&_X=^1r1Yfd+EJ9wix+zeg6?HP^M`!=wi=2%YP{2ZNf$uv`}uN`cHbNzn<f& z;K!du4UN@^Qph*;iFhv0Tk+4NI}Mj_&;hm3>W71HJ+EJfa65tvee<g0)xZY;%xL;T zQdB546|R7`^vnJ8lH{AViAo}dn&6`SG@?VO2lbDi|DjDPRSb>>5;@k&FZORo;=lRI zb2p|QBb&nfIn)1Vp$71%<!umDFb~n>cosV=DqQ^Q&30(zuz$-#4c=Kw2(H~BFm$R_ z;Se>(XpP7U5*fie$9a>195dD$^83G+f;56+8azw(Po(wVA<((GZGHNB2E6&F+x~P} z|MNYl3E*(qVspcNH=M2`jv8}{&1Ncr4BmV=^Ow%Jw7eTna<h51#LaTv_;KeGBnN$= zv4yBn{l2~K^*=&J?c)btvatcNI+2G||KXBTGhu_gc|wWvKQH*SKZ_87+%<VfM9~yf zTnJ=ir(Gua9*ytY`YwhDi<9WLj8bmfw{KIcg%6#1i|xZr$+uF5;>k~0^r{};EZ1(T z5_zb&>S-*-?>iqA+z-ueD$L6gDJsf%>tzzPm?Xo~q9u9ER_)0gZfkv=6H2*k#F@lJ zpnvnhp_KwJ!naS8Q;_S%CH~J#Y1wM_<mqNVYTt5ce~C?Kl0uia(-98bi`g1QBA#iH z`(b-NinT&3Yhb*?KjI>&<miUk>v$c%;=Rs*gA-$W-K!1^^2{oLcnl)Krl?PkjQ9ea zdBnjL6&1^i4YyEU$89E;@;n3wd(AH3NFQ`Vp$sIXq|wo_;is<fHO`N|{d-;nmi+k^ z2MG_ENK*<$*@cDocEY`lSxrrp0RbIA-mE1ye`<JUM(}Wd&%f47$G!f{PNstp08*|L z?LzJ)rPf4s*_q{l1RYVPHX)F`j*j)j&=%Z^N{wOOaGySV6~V~L%0E}Lus1X)5iz6x zxp7Fyk+wi8f|-fQrbm>iN#%v)OnC;;_@s54vFhWD;MR!QjdJ7k>BD@3PLN+K^`|6= zN%9xYV?Dm4FEN}khrfRdZzcf1Y2nxb`{Qi=-n*df#F-kehW`1bqV0GuBbS3~ECloJ z&o$1NiJLP+g3c6xtJdD_d>P@oTC)b^%umpC*;#Q%%-uX461M7pRH!xTsH@*AN`wyK zr#B@DJNF;vys0{_e-lZu(%R8?LS{G5O$t<AcP4b16%@f(Em=$HE6~v`9o1H)aG3RS zjZB0ub_<*l>NT%dGy(-sPnH|kWbV`Bvpr5^GqL5ci-twIYnJKpsyWocdXbSTQ=qoS zbuV)TCo9Quz352^>N26Z-DYn&pmioCEu183{G%3F-Wx<A{HnWgvz&*ZV@I2IFu0YU z5R?I|Z`Zw*U2wjp8-3TO$;-N%Fsg|>uMVagC|Owx4cir!huRz5RRjA>kAGh;bMmn1 z@oup*)cG77q|sj-qp=H3onE{dYT{-W&I8;ok7Y9(#^r|72l^v1oabVXG}DLMG^?1G zfG7F?k@eP5ZFOC{cNHmCq{XGUyK5=#6eyJ9UW&U0io3f*vC`sh3GR~OPH=Y%5aipv z@8@~n@0{}=gFyy6YcHAWn!kC?md0+=Qr#L%<W{WJ1&oYUmmPNCohcy8`>g|9g*B`R zS>{=c(R~pMN=#@s7y-}Whw65EPh~*X&o_UsIJ6^+hTK(Jz@AbU7l_zC&h>qa4G)j* ziiakf?9TkQKl0ALfMw=7a^B&6h|ZDR31h!`SU3iSMjkl*ggD7`f3V&5Ay>z5yICTy zv}>cg9X9OzbRsHZ{jC!~Xbw*Kp8&$(7Lh)R2-(%-tugaZnNEWPH~kBp5fJ}h@f)H~ z^-RO}@q4I_#g88}ZGL?pDqWY%?+&qi{LejZD9xp%`i%FZm$n|rx`SDSDrY7MU$S90 z%+FUHf5&h2_Qp0mI&L7YU-{gRnzCq+Y*Zd3zMSvP1^Eow(|JK6LhEvQqxD<WcrVn$ z<D?R`uZ!~Ja<9(Z_cgEW9Hzm!w>q<_2p=~$CH*x&AQ(kRY-JzJ)~IB^+lFL+c+<0y zFWy1l<fFHD2IL@SYeD$Y>0a%RLcZHl{i$o8#y2L<?T+ikz;>AN>?NHmPODtLn}ULH z($3d22Hq0+8Gsn5{ss)%Q5t=3*DIh1_&)=%9B|GElCy`M@)BDMwf@OTrn2c#m1~kS zL*u+OU=Tkpv?6HyaytppQtB$?`;(IvkzL8Pbu2LI?NH9s0KiyQ>+(jGbllIt*kc4` zR_^N#?biQ|<~~2^%)L3<;Fs=`Q6joII(8ZlluO9py2f`~YehN-WN$w=*dxj*s$8SG zP6U7psp|Vk2utid?bN;i0J=s{%PaV?SYow((Vd>{?_W4w6inj6Do0Bx+NXD72A=8Z zWNDMskHsBZt6j9U4GX4XkJFQrE9O!`?wrmpb7M0zF>4AbRpv|PH>@HYYq&Tc4GfMm zK~0u;SOU0@uAg|wQoYCB`|m_NC#^+Q2p8RLh+sQmpKZ$V2}r`9>x)vppSR}axB!xA z5kSlKMNYGgM<GE$axajq{)ofW6i*SRZGKbGGIiIMsPd2EVy(cb@Im>K9H2ZktxzR~ zc*}3wH0#W(RBKBK2=^dH3;nztq8zARaD*(e=lYh?!R5&ZXX{oJaZD29-m@ea1tTry zCZBD(50yf+h|gIfiPtWG?mV7a&_nD8YWLk<lcszpZGngJO?h`H-%HLiV7t&qFEs*X z;kmD0kfJ3CS7Tm}4^;<|iJ{t&i+R@>V~lfgZnoh4+9M_^B(2~9O{Udwe7>Dx)NT4g z2@*n(Unw0BtBsGJ?z=5Zef!o-AglWlK|Xz06}!HrPm}RbjRPc$ugwpoZyqlOgh;Fc zGg110+weO&xG<1%e|vhAThVq=%&QY}F>_EOU!Kwl6?<(5KKBdNbvv3|%X!!oc|k5l z1LvdSBGxYw?NpH(<OlQ>qBcYqT@e}Ez$^vx9k=W8>1(@|w9Zy&A$$Km4<8@ZXe7(k z5u6)zp?BJ=+hWAUSp&^0ta}5T8!NuluPyS|B}tSx;Fke0qCHCI=v6SFtxtO#1vO2v z94JUia&3Z5D>PRWbl2-Rw*F^m`*k>*exk!usSO~99HF&+E-Y5eWve_h#}(WwBk{9| z@4HY%>M!Ekk7{GVLU81qdA@~`LX+;ufL<wr19hVl%VDnZjO3`J6?KA=a+hVq;Adq5 z?mu3RxhIsK%9+@4u`W{o#%Y2aqJCq&h@Hi1gZV#l=eEKF5VNO6VGmCuQ(eFPY(X~R zV(g=kKIY98V|WfaWpEnLX=MNc4`QuX0pOyXeUUdT(E5L)bI!KE89Pw{OdVe##tggM z#_}gpZs!J>duI++c_?p2HVMeQAbx>2hCeHFS5)tIirczRap`*O<)3RSdQ+-#RZH&V zueyx{vlyeade%vz{{qm<H3o+}(?)lVKQ$E#i3z=98VO(Nw;UioK0wuStbHyPxhPKl z=H-5&F-iWZ0RKjyJKKwp?DyuH(UUWP7i*R41h4QnTWMumR^YA(*d(4vE_-y*zZmb| zzZN}rTpc%?%LdZN4>AE#Kl3{>%yO}ldRIplTvsE^m|hd7P*74xwP1<ey+(k&H%NrF z9KSVhqy->uZ;>P=v&BQ_$VfbO(QCP1jJv01q+lIOgqRtuRH+7L39?X9N`z#I;?gX? zDK*$SO%(`6(g(QUn_v(+om67d(vvQlaj({)Wt!#h#Zc*poz3sKr1A=OD{yij#t&(M z-bV+CR~gGz)+Q5N)u(KBV5S72#+*Zzmrd$%aje)*`uNE)xMImZ;NA-!hsq2{5T(w? zj~ty!LP~xS_b9k@{_JglQ*g&tnNVzY8pHFa$OlX}_xIWQC_JDSQuVw3x>{QJ<D8tQ zyD39voe@k1xeVM@Z6d^PsGd<v=mD4aB+qSL83Oq&KnKBwJjkr9x~Sf&5ot3iHch8L zWEf{tNis+AUj&;P$Q&y}F19!E*JQ*H7_52>aZZlb_Pm0c!m<n$=jjTfI<&ao+(|w0 z)eejegw#{vO>dDrp!9)U7yFYIfKT=^lRtf*2x{1Xxgs9x2gaPZ>3ZH$-OH#*yT&No z$WAKdh@Xm7*kFznpE@<kja_)o=AcLpvdVFQ`7pNKkv21sNn?HHX?L7JmIF0$w#K;} z>FQ_N7dLjGi`VC%!E>SHY(bU7Z_b|Q(_y&zK&NcB5Nr7G_=nfu=nYcN$PU2)_Q#zl z<?h*g+HG(}Bu$X`tM7b_V|63y)5OjM3jaA?jOwwpVB#~%c9C4&6f;vgx$N8d%rQ+3 zT%^QNd<{RusPMrYVgIu#fGD@v_zy95K%9`vzufVl@OphG#iJ{hTYlp^a!3!NiuK;A zRK?(Qy!M?5P3tSSAI<mEP9n%@SX*qo@30?l7vOP;@xcHjsoSDS8Qk-|-4Iow9)MI@ z+cwZRQct-o2DvomN<z8T%-s8-<OmV4lx#kGqtUr5N4-PveY}cjmll)eU~^ER8DHhW z{?uB@`_t%bI*uyT&@hGA*Jh%S5E@e=iByOzfoT>yI)Mo3jLoe7POAH9!O4XHbthTR zi2n~`>&5CWT`JAM)-IW6J+HrLd!Q{NV+}$4{XaD68yUTO*!D%(RRMa+fNyZ97dnQt zos_58_|Jx!F7?P7U7-m57d`l_{`;I98;oG2cy%>wM@sbLR=8j3V|U#z*Ms?pDOhRW zzzGU<>M<ue;km3`Ko6(8CSDK7s841!tk;=n1K<+bZK@hd`S?X9ISpQ+DP`RDoPFY) z^H<F^MhyPRv|fl%%(z~_N)Bz8hT6Xh0xYM|eV+PJQjYQ*!1$jA<%tP61WEJT=EJqD zz5U%C1R`Tgk;9FqrrauB5Ud5}rv2~RpGFDebuaTC3Q2sNpkMiJOeMt?{6iQA@6Nr2 zDfR&tNxpO0?-RcaztFI=vEjP6D+si4N%S*gVruitHF-xFC@85?Zq_h<A+gveXrGyo z?a(??xVfozRj`345WVaa_&fukzM9(5afb#(0YRSovPMUES+i6hO>Cl=!?9h_D0@qd zfVz}8I|3f>LXR8IYn|MNPDSLw`-N6Qzz2Ca2P$rmxk63F6+K(+#MU`X5b9xm|B)S5 zW7nqmHXYJS-PY&wZpLQOADUd&UIDvT?Ij=~hy2PjvnKg^NYphFEUsIA<h$v7>Pqfd zUbiMcGJ!B>(&Jz#|A~gY_T{64FG|3a+gM6LXD-*plRaU)P`^7g;Myp3>|@5vfSwM6 z?kssad)GZRVX;v`321GLcv<g7Ng{AuNudr@BT?cag={>CM#xR<MDRn7D?^Pwx3@Xe ztKdZgLRGz6u9y(kl7%tHO=hzbGQXeS`jg*vBhXF<U;HHYuRbj#@TMP)_hGv`VWAx- zhyT%-%(5MHjADN*=VyoTe<BK3S@Vs^$+~^&Lw1VArX&_|{d76%o?-kq4u|OJT?dyz z=huF{%R?k?ZtigLpiUz2%Fb%9Y98JLB~)@iK>)n!FQFnITD6nPZ)&<3D(*bRxBpiK zCU_%e08#6lxIunlyb#SmWvmE9GC{wmOQit#w<N_DxF_MW1p!j5^->iwas2__bEYmU zFtA$gc^YavyC!7N(@HGU00tZx)+J*a#r!G;@8avM7XmD{l2J!YeL>vQJvyq~O+y{_ zd1ii|QChtG@x1NvG-qY64u>ON{UkpGHc*Lz`kaUqH=<@sx0IN6I)RFr7|iYn-Coa> zyLfSZ-gfg2=*aL<YldP~ULrc@b<8tSYV^3<sb4%e@rNF4e{nk}48&9A^c5mkYvSYS znTTTlzszeUl&$^jja4^s`kx(a{2cmhD?VGzwTA#?{*^<M+W@LH%Pog?;x^B15faWr zPLa_qqAT3Y>(h#32pg^dNhgJbxf;Rn@Tf;;|1jvHe_@RpJ-pF;9uWWusd7A&K4o=o z_nEi7t=64*w10*n@u?-g<b1)5MM$|I@6Uh@uX+w(jQrW;miKDN)Z6>RZDqb+Uu<kF zeILg$-le8b%~K%nG{7I&i$48yhF*BbP4Wewt;y#R7vhFBTRm`Q%w><@T0tr@R=Q-O zZiD6TU6_{P{g%!3b&c8Vq0teTP;3})%LTZSAak2@kO;g~jV7o6rh-~wAsJg3y2n2X zi*$*884})qd)RBkPv-ii_f4tA)%nG%jTcNrjpR87fgRq=JeOAukLz^T9V$L7*y!%d zDTH50)}>^OQ_r+i5;*gh%(G`uMbT6abF#X<dATt<sUR3ME&)4jdJ&L7yo)Th_CQOn z%%P3F?%V9!GGDP}*WcU&W>0B3%(FQbFjdf(esu)i+>5v@GSNcpP8yEuH$h9?w;a&J zuj*X$TGrj6mR{Fs3zpVy1oZe4Kj0$1T>w0tIvcDUcB@;ox_hswaNtCGUv&;Cfwj{k zl_%y7?f<l7ny>of3nMW(10m8jK~vqX2@260Z*$MPEhA%weSFV~!VdBJ%)QG2mYSss zwtc_p<!^#+c-hU<A|NpCfv}?E8YSjW!9wg3pfCT-9Q$nOiqjXDucjZ~rgH4IZaR;K zzf%uPs6ooEvY?JDh$gkB^m@ic;kDic+o{h*g+aElPf&%LnB+%2GJRTfO^wCQy*gqV z?_>#r3&CqJyR07@7njai_lYAnS=;#$^7TEk0sv1XeSWz7X1xH>i8Q-FKNa1e`!Ne} zCEdT@z2kV<0w-<%3p4+jQHeAM5=qrpV_Ltb*YZ_T*80SJiG8o{tWLLbL*c?ku^uZF zL3ypyHnWkB$7~On<e)Peh;BU&lO`yDN+!Tmk7ilX^X`7m;PG_}+%uGqs$%~&(k|+I zC_d!ppNY*!9>A2KiR!sX`MZ9?`@XKuktW-BYZuz4W}Dv|0%zQYWsAqsq}wqv8dhrx z4TdfsZ;ySMv+23ZIU~Prq<uwY>9d+}hX5I5RJuKs$09%-&G~a8i)B&2fDmpiAkAcb zB`8U(Epd{sqSOMHt#`nANoU~h?wMa-uKWVWbBgrryleR$x%;{I%^(keVNyxQHwAl` zUiY~viS?YsU_K4GC+_;ZjsI4ZBR}K5x`w?UbVnw_km3!|nC{Lx!r<LB`y61>(I>^7 z9>rto0h%__uwGnePtql4ySPRr;RV&?qc4s=B!DO+bks>!6%CY~Z(Q%+?rsCkV)kDl zNHN=k&{8&tN>d_-nOiSgVm`X&e;DmKMWcGc-pzX~0A}^Q96kQ1r_SUd=q9StiUhiU zzImOt;@<nWn&&zsuai(YY5AUn?j+k4sCmlb(iNn8eTr?A20vP0^Jfl{3|`!9_P;2| zJR>S>*i}wT<p%q@p#)?MyMLX@22rN_0ybrh0HRLtKnl&_imt-!!@Q_9On+6!S9fR@ z1}luqdS`S)?3QW8i(An#IXj6J)2OLix*K$F&eOaeKbDmcg6(w7-|P60y+0lrXknJV zv_DM2VX)<czE~pBhslP%_^Sg&q#j1m=1nU&TqFOy_8jsuqkwGq96-W`wnTQ?JkY3M zghn@BabEj(qzXmYY0i|-x~%Sw@J)?%%zU1!+ju`tAXrjodAU5%5jDj}Y`)KnL_NdV z@+O2{OR(73O^P-u7~kBE%>Ha>@Ew$RR2uqF(Zwp{>3C*@hSFmngQ!0jDa!hhN;$^A z$Q{EfmmiG~qA=+#&NhoEvOQlYd@Z<smR{v#pwA(wB9&iJa1hn^tR`7eA4!qCyPwdy zW|7g3O^yVv_EQr`j3mx<*aV4l3X1yoj(VDE7E@mAeSBdcaMp=utTtO`XqvW;koIK1 z@fYBl5J9qhf2U7yA)(rjx(Lw=>Uxlg%Z_8XDcbHk#We!p0)K#cZS-ou1n_kUP1o|! zV3HR=mz?z_c*@GT8Po8UQ@@D8cW8=FRL=dGjZ-#rleDoKCvc(oGJUpwxtk}$af)hm z^06k=8V_lIZ@-NT-M(^(4ap#wZF61@xai9+TQ-^#sQJXyJP>ML+{cLGuG5I%dXUqj z_x95H?@NAkFK}qd&3*EC1iKePWf^n{-ckBX!{yN_=ha`8mljI05iLtV8XYlu0bx^C z0g7iA-rjC_`nuJ5m+|3&+*gnWeYq^9E^P<g;pCitzLj=9p*jlM<M;_kcU`Wb%O7R@ zqmBBGK#gYeULnYY3tm@P-1=2n8-o7FkVowDq(ph0FIrk`dDJ2xZzY|1&NuRcG)lCZ zwF`o#oiD-7HBOFMY(jLESceCDo#v-s;P@3{<t6>Ge{^*!YD*OdWnzROfR6?Gk>s~q zcqauS>|`R2e+8s^%PIpKZo9LUG{plnF(R`q@0zEGNvEc52hM}U3kN)CWA;)J%R-@n z$#N6#M*GSWvb1Kg9sKV+`@Er{vr9|UD%b{3o_v2ICn>HG;opfiTW^wrVknF+BBA|O z`sE5yWFQ+9f28TLqV1L=p3AX2qip)(@IXS_EfOy_$qADQqRsXJH)={Z!*QfpX%-As zKu6;8a}k*7q{smUa~EGXd>d&v6FwK%F8xY$3GhY3{1l&i`AD4XOV3y1p)>`oVDl!( z`lOQv>;4-x{`>0*HZ#Y`i&NsEBrj2{Wzmq07G|nG6`c_$?N6Vmper>i=k)#*bd(@I z!m!I>XhpL0hA78}E)A645~mSQB0}i|o7cz*I;!@+i4O}KjAE1mMfQb*k&Dm^Ytm-* z$G-AeY=RECUD!MaZGWu4e{%cVibGiP7-xH3Kl@tvkC*#ill{yY#AuKBz1Ms8X50z( zulwB4sl=)*n|bxVFYKp$P&m>n#=Ew2pWig~pYT9Zy6(L`Of<_X$urAJ0bT7}^DgCu zmRA{W+vV--==J>b_S@@krHyZ9eIrthvn{soD>7%v)15B1-gef*Uk49+Jxq$ie-qF9 z(L1W6xGY=LZM?p`5oQan50~?4DH>LJbT!~gJ2h`3M03uH7FRhnL(gy$bs26g!p3eS zS5fo$)`p;k5Obva|E-v9#RRaE_<;e`$Ve;`!%0)y0x0zZ$cKo85KsKn><_(^N_~Z9 zxfE#KEl<*JvD*iFJVS7CiS`=n`BujBfb*Lm<4M7t?HItNo&$a2z3g)?jKLIEgD!*n zkv7BiBE2=ROxOIj0do(Q&&l)Jx`m>^jt#uKhkPiuJ2M`3F?0nC_X%lO_R$f>Ya(0S ztIIe6)+<WHDQeiylBNKFU8OEyh_dF!G|v%=`CA#tvQJ7&O>0uo_Y^2-2(L^_OA0Up z_(TatiPWQcFxHzlNtD*K)HHt!2G$P84F`2KG$MV^?H=g7n+}*VMV;O*BZI2EU_KkE z=Umn#FWVb@&P6-BsJbOqvfyI9r<Ew$cJ>RSOUBNH?>zcO!Pog>w^-G0sYH<60G$l_ z{du-1ikQon>2hQ6$Yje0KvgWNjLGZA{W?D^_Mq~JA#`;hp<yiy7>laf-aZMJ&UD5m zATgdF8#;S7e3EOB>KeaVxXYkaak6;6-J|fczW!|va7xm;X9^GpRcMwb*Ei+mjqMIP zA)kNxrr-RS@_21A3t4y$)K^wHj(oxP{_^FEsghpshr-4#@h5ae*0ZgjrXx2M#M*oh zJqbk(Ao3q{RYLj<T3q#nyJC<?y^NQ{wJ80~l%fV6MN4m8!J+a(O4^MO)-G+X?(K3g zS@jhg81R$}uQZy_Y~rKqR9OeZ#o{Yrc#8DzUm%h99f1)JWd$2y4oFh!R~En9wCRkR z`0N(^T!{H1R(eD;jN~6r>Dt;qI)TdF`X8)!tKT>trEvlaxdOg3Ms<pJ#|D^(&bESz zOVvp%dbuLK+vmpc2H@`QYgh`RDX}Z1F)#r6R+L}kFCG@=%BVkTmsdCFc-L3ea{#lv z_1-2NWXA8qLFe*39jBR7InA`fL_&HwL|ruJKeI;QT)ulm-WRrriKlg$pWljfFK4*} zWlB%Dbowe><S<^w0foI^mAa6oG0wEw8F}(Rn+d~b_Vdv6Oqc*%`ssVg>-uR=sY!fp zyrMqk(l>sl#a335{88Ko@Db=^eBG6eP!3(EtPwwd<BOoI9!Z&$^P`t9b?CIpIKCo1 zAz*I&&dTPiSnDRMvlp$Jno1Vexo5Aqadlsgw%QntEuh=+*@pD?2M)XYY53jI5{911 zf64*C>^p{MJP+VkB*RNBk57uBk_xiSki@ai>FL=FNWDP!7D^E=g-HKs?=h?6@g%0| z`V~L|>$V@RyCT_KxAqLT<1QFC3CJeO1d)?r-&~Vi)UF7s=(rtp0(_){>hwFs*t*DX zBNPmRwEB_)L_gW|TdBDaO%D`N(}0=*oOu+(BSZKVB19h~J}Bo_pVbRo`RT^2u3d9} zuzMSP%-G@}afAWwvHziqh+X|tZot+9^|>cT#gJx0FQ;*l>-4^q^Y~iI%-sC_PzY`~ zRIgVgT3r~qXrQVQNZN8GEAq$dJJP3D2|#~`tvese_v1Bj(v$0Sojn9V%9ayWNowmk z8Tq~4<a3Kr0t#_PgKbKExW0>$(#ZH0fGYO#Oh})(Us(X7Vl+dBW%;<<w1rYO0Li~P z(nIZOTCVNObnl3Tp4-ftK9ncxhQw5~1$LOjrpM<QZ4jUdd{z2<K58*_(}?IO+`L$} z2kZ44-{1uE93IkFet?)5r#g30?o~$yA2D1SPY^lK2)szS8~Otq<^taU)l3PZ;r?&I z%k;K4x9W=vVH=6HeMTA{%yferoA{xx`LjxVnuz1Je9j+`NKoT_U=D!4bb>1ZP=;`5 z<n%B;?wCTb=Wz(AgH5z6B@=PiuJ_XC2givPzV`sGCt7V1k4twANUlAa6UXYYW*RU0 z&V?Dr-ZGjB^&wqQn1*Uue6|y2WuW<#&Gn%pdOZ+eTQ~Jbk~uPG@@B?#C&YyPc77=1 zX<l_q8~3(t51+qH2pkYVVNeWU&Y@}OT}*+Lp6@6?I~orWJPNQE5DVdBkde5_AopkY zr=<I?#-*na>-4jNFT%YR^<L=vHXSVpJU{@^Xx!*|GNfP>dp}U)(}suATr-*nfCIQ9 zDpylz7|Bqx&RnCNPM!(YgAQ4jvV~Ux2LW#**moM+@R0B&JW%@{(Le2muSI9Ll`W@A zqHo}v82`Dy<@*nq(Cv?nWbr(B$28j#W!>e*xHp!ydiKO^Rj8PFJ=ND|OHi0Q_v2m0 z183i0=I!XD0v2=#imt+YPf|+$$}H0ldZNSqO1Uh6>kT9tR*3!GVX;ZClkHMDsE%rw z=~2am4g*|j)yFJ?DtEp#glAByEs8nmp|=%+*s<x@*N1q&6!YB|LA4yLbhuzNy&(yp zknvbh6lKVE%);G@TJQkb=6A#z$r7qX_Kl;p4{gV~Q}A;0$`e|C&bFg!0#3*0kXgZb zhx(7YDp9_3qK@Pk0@gyL%l6ji^rB;2QATOS(AVZMJ@qLt3$Nj00m;IfU>h-sqbi`S z<zedX&|TGtYpBlMaP_&njPJ-}&9S3xi@<u${q1DBR*l`T=`w$^;nI0UzAdTMS{0ik z0pT20wcR>67L^qiMQ56Ou19`bn&REmwgstDSr^E9`BoG;6OZa%^tdhJ1;LgFYmoTb zPZiPM(H1$6+xKpz*!?Egtu@EnPHT7QmDx>ME}g9^20DNI5=Th3QR240KW?A*pAApD z+d%@w1v^aj>Zpoqk(IDUuKD=LMaIHXoyWjl*k~UJxS?`(-bA<O5gfLUKDu|bpS1}{ zep|g?sCFSJYo;>(1pp9c9Ipbb5eCfYTff+^(SunHWVUIsA~lv>t&C#HDxu8{nq7ZQ zK)ktVmB0>b8or*Gw8Mz77M|<>Se%7Pv7nFAo$D!ME-MTT(cm$Hr7|&m?Vb%ub}`Oa z3=WIJ6D@-OMkn;@mC#HpBYO^ZgU*#e!s*Q<SE~CjSVyp$-+v1LkCBIKe1vfVQWV(7 z7Ph^+t&Fsd{;<!9eV?AvVb@g)9yp%<sPT`520unUwt0?-AldU9DRtwHXTh6Jg#cxB zPuuK@Fx>f)rbu8f10<4oS>GTDq-Av)e!H4<Z+}DQ^6o!R4A_|9Gi0u+V4(-guBWS& z|8&a^VDko2kT^w2WNE;IhW`-8KqvzE8x(LK3EC8qg^Wi9BfScExs&!=^#_$Lq&K%D z6V;2aSs2sg=Nd1q`DZ%Kw@8*0hqIeO9j%m#&Ts%GFar1?B%7C89>;>NI1(upMvjI? za1Z7jCcvPf`qeD_X0ZF!8>DUWzWYxA50uM3i=Pnru%URf)Ur{)`d{`6VAd&v;a%9$ zQ%NUzo?8kz{|w|agzSGFuICLnw8d1$Mi2kCX0*eH6UW5nkbn>F-s*KV{=Mw^J*|M9 zTST{g4Z4Q`?E5GSf<sLwW*{z2iKM2M{p${<u;0TZb7Y1^NN4ATeXC|Mtd;MK&w8l^ z0XVNI2OLM+@H!uzcG&l@oF?n_S9${j%Jkb%3@`Ly4FG+bpZ)_xW{4YA=5!hwBlQX$ z{x`kwZ^jxBaQ@@>VgXE3OoIxI%XhzKqkqk_UKE>m*3Te=2?chB26B25k-d5aoJVe5 z%wwye`9h*E^&avFye9nz2!kscn%QS_;==&4BXWA$q_F`@LIhMzo3J_YhyC_oi6CFx zwcnDwr-o3J?+B>9fpH1&MpqS?&q9`h<C=R3Rmf8904JNFz>f5m%8<+t(3@v(kd~hO zJLdK%0nfvB362lc{w>qm0tyBH+IIY}zs)%VpidFMmo&m>{MN$p*_$v+RAy|KIB+z; zuO{7k`Dvxc%iMiJ#3JntSyy|VnqvC4XOZ;#-)I3;g?vVQF4H;jQ4p)m_+Y!mFL))0 zK^OGIPr`Mef<4UMg$0YdgGO9NbRX!(EP*t+up}Zpiomj!t+V4p?>mwvc>)rf#*m}_ z&(6gbVWE2fRrwE{lHR-@BPZ_bP1GwSxKysE^}pO2KZi}uf8o4@H{+B1eIB!yz-t`E z{QD&!VxfP|N$G6}Ub7X4Ekgw4BuKLeYF!M^fQM%Yt9q0Y3)kx2>s$f2xMto%neAS1 z<)H|5iPht(RNTBPBDbsSc;FCmX9&mWeXq&N&R%oLu!MRAJic;Az291j&8}zUve#G9 z7?CR9ywz_;_QSQP;F7e;oG30(gqJAW-{foxT0j**<S8WB@3RV&pnd%BqyJ{_$Ik=I zU;yOc4=>AU4Yf|lf4|3szdR{v*3RJ}F4f?<s*$~Gs)|u%mCcK%X{3$L7KX|*viYZJ zNP|88Q$gaQij+n3EgX~6(c<OmmJd*<rV40;g@EI58_+x8-KaWr!E8yn3v4g)qp&iq ztU7*luE5zf4rFae1tGd2RYfvfn0&&pMqL7H8yN#>%x2}Ro{e0u@b!Nl{Z+sgT*<Bj z(x?$xU;FD{M)AD=|6vqEJaCbeJSREAzei%~{v=9O7n(UasT5X-csYa?Fp>L}`=E&I z18d*n#6!os8w@eO8)4T5Y_QQs0@ez5N7yszppN4_c76>Ia(z+;hjN~ww-ND%9t*k} z{V`kX_tE@54x`=|N1?!;-J8@&v2f7=#0W7eJzv*B4LW@4L4Tbm%{VxYJ6CGr|Jf$O zlm3Nbxq#)F0a6U?So`IpIJ|1F39wNgKH7i%dg>lfy+oXEzflWnT?&2}*1e!9ocOk1 z>t+OuPA#YigSb&I>guwE-RP*U1@&bG;O*p?#@^2%F2`U^7Q!xyb-(Z%D5kD+Xkt$l zph5p1HZ}bJLSwUUaFFD*U@xKz7F6ATW34;!XMXwzJLT_@G!P3ePi%)QAmfUFZSg&i zlz{g<FAA7Z0v3#w3X+CyxUcl_L+m#gjqHtq1zWhevseXVnK7jTUQEUu4${;=q4>uw zz<cWdS-V#l$`sr!&$sO^j15hX(B*-T0Do4TN<!TAwM)d=^Ty<7L8M)!vJA5@uvyo~ zfDBJ*oJj=3!;XU3?gRs=QcwemK;+H((dAKI99KVkSoQ(qDywxhqu6!W52hWJ(Qes8 zy?_6&8QH%R`hQVmVANOs^iz*+xKQR75Z7lnp}GD1hkHE{8onpvJ5ClsW!MpT8!d-6 z2{>C$T)Dt=`FL>k;>J=*xNYXtvD!O3|JesN&PAQ?kp$b8!p4x5fc4Dv&#p^sJj&s` zDR?_x&7lSYp6VZnS>|mSJO8C4C-sdJT-lcuFz*8<M7yi6q5W&XkEL=>usip(74rR% zO{n$Cl~_<vu&0s-th5Ce6cid;9Dmlbi;$L)DF!Xv6A%+i0AO)}g*NOn{vj{%{QN?( znvT#d0zp0#;|;qwFEba~H+e_E<Ix82A|25TLp3Yke?rkiKbwo<;^`E1AqE|kd>`AL za*}4m&wym!P@0O7J&+DZ#czrAEF+E)+%6~m*Rkwc1WzV-h{+IPPE32SWXllUiiu16 zNL6H@jRN|~<Q~V)zdRR|y~FQZY-TwA?6TvmNrQm<@JY`7XcU6o^Vm;E6!D)W8oX*{ zwxclAA^QnHeE*)}oH2jFJ+}EvjHsWjpJ><?om`e?f{;G9c-$0@X(Q7Cf5R0*O1>}! zxE;vK%SnBke}pb)_<l+<yMa84gLTBfLcJ|hi*8nn%`R4^5F%~uG;I$qCMGH_N-8c+ zPAbZL9ziN}ssjY%7@;2rG&%1HCPN@St-^nU{U%9T9PeYL#0%>!og9xvci~(k0!RMN zyJ!w%`z!MtaC6gEI*ey~o?0e)L(auTiA!W?(SCF9)4*&RjpZ&e_0p2piD5a8v2ItM zmfDxS>>6=^GBumnDpS+YAc<G@%sN@R^Zs>G3z!sXW0DGP3?keGd5;8~>Sd?P6%r?F zFXJH&2?Z6f9Z{j*X8zylSsCI0!5s`)))Pzg`uxbI(a|D)btq=9y0HCqxyt--SXW2i zN8Hi%I0;Xyy!Il>vAyoROs8&sbfy9p+iKFM2d3_S3eUt}?cZ#YMu)i9(r~!L(&96) z^in24GX>XbC;cQ;NDFL{1Q%T*4au$EQehpv6Be6KGx&CpQ38(5aNwhGJpp10@2{}- z{W1pqED@B_mFXV!u#R899a#jGZB5Os=4Vq#6OV&%0-~bE-S;AOGwMDQg)2l7Sev*X zT6M3S=RfQ?lq}Wi?V&0Evc;ky;P}c;fF2wjh$)ADM#?YKaNHj|`{p;9U;z17Gj}0A z2b|ls2%_&;6{iXN1Q;w}Ht1>F3{wld1=(j?ze3-g77gVAtM6@&*39Hj2_g$_O!?!- zOFi=C{l{|M?c<_jw>CtZieVEvQa~Qn(|eijSqKPGc&@c%IzE}k=`*fDDCY3X)ckP# zE4%N?#^*Mp2=C=hVnPwuAL!%)n$l5e6vVu?nsl-^L9fz8t)edu#Xc7)W-x#e81X0l zg^u+<udB9ed0HLG2Sq4xi$`OD{^ai|DSsrYp5)-tt2@g7YP~-S;rw>+B8|%~Za7s$ z1(QU$V9Xg`*CVBuIl)gNWGHdB#&kB6<7{(AB#|DY=OwpU%aJv<?C=rzb^w{&ExGc9 zJ`3&s_#F|q&xjzQkWcks=-7I}aeH2TiksrL>ei|emGn^C-A*5;#~@x=#4#FVCbR=T zrk)nbZEGoUr%cNfR+3DmuJc?Pz21O;<C>pB*0z^!a3l$J+<q@|k)>s&;KNb2??5-$ zX}t+eP0L%^7C%XT{pM^7UR#uCkciyiU!30#yy79PjW6AtO=(#s-G&PGF0OFhv_;+e zL79TU+y^e>O3-=`Za|;1?BN!vT8{6B%yF#N#Ugg&KZd~TB}HX>$>IL!ckG*^O(qP& z*%qQeYOsO58#30S=Rk4LblLocmn}+l(-)&4Cuf-ukc!am$9wZeVY$l0@oa;Eft585 z4WE@)fu|R~P9Xx?iX<6=rck<(!-5<fdO^Vf@$xdE&stSK=EX4)Ol%JhZuF1kJDyE4 zfO>7TI$qK#|2lX1^9MUVC8Zd;<}0VIrE1>^>T?iqJ8F#&W-cgkT0S36t08*xQA3wG zWf5!v%}(-$ixw6ZZW(d|5C3-OTp@*###XA8SqIX&JE_zB)TFo6quO!mTa)|gxCxw7 z{KK92iyt*|RZaZHVTzf;UAt4p1O5H+e6WZ2s2@lgz2Jvh)yxbuQ^z_BWQi};>?XEY zaO(L#B=<7U<UDj!h^1j^^;E?9JVzS1IX6=}f3sq7v`SS?&-?)=LA|c^<x9Fk9q6)r zCeSf2@5z>_?0okI9u4>Qr~x(F5(0r6@LCSp$%%&YTZy%c>(lz-rt@txn-4$cQLTRw z^0>AJ2Fb`nA<tbT;;;kqB7pwPg!0HLuw<Hy3R!cq63>9=|E_uTC&X0euQtXJdWC1} zWNJ6s6)9@0Vs{?93E7N(GdiC{ci(-=LJS-#3Hi&jbC5U4qBp~iKk3F>C2S8If{`Vn zsolv`A=4B{cHO0kLLt$Cx`A)l|1&R?V_@@*Blp80VQByVS@BK@eHrYg#aM0-E7Btn zpPfQ!=`jFfBtDxkwHsh<nz%NlBZ94B2Mn!i?r8&@Rc4~L)0P5!`M%H!zC8XLMxj*L z3wrI$q*al`Y5-3xU-BHZFz{gkdex*UX7iqN&~a`G8EDd?b#@j$w!dSPkcmv5vcd_} zZ~`HtCoAdHGYYsj?S)H9vn6Od;rGW;$!xa!Nu0Sf=yTeiu<uUzR!Xwy(e|C8UHJUq z@Oj)4!fB%Z5+>T;>vJ>*yVzkda6LToA!jVHB)-t&<gD4OVS~i8EpCHuF%ngT+~S@d zrGb}@kKJx{Jue6;gg7~cBC)N2)!^42Hd++97s8M(fdAUqernIfv84K}%4?2WO<%wK z{0T1YOcy1k*?i=j<CbDv0sj(W%Rq3K66b^e2u*%(WJ>C$gDvab!^T*|ix>E4OX=xW z)n*H>+KpcwxQt_~SwA;^UPi2#Xs5&$NK$4fUK{L%a~FJ6FroI;&II&!&FwKZ$nBR` zXlUeA3m;GuvHcEoHn+}h%Do?s2ajK8o$G<RE3}z(>TR8zEQ>OyFG?V{O^%!4_4k=K zXr?30F5if~4Ni2akPL+9(+P%|%GHh8*ZZT|hk!;<|C}AxPovxlt1eo(lQX?a3iEzS z?kVRaEd*I)=vndPEiVfDc&+o6WD~PuecSh!#uSYrX!TnCF&UY~;DU_BgnCsSa0XU2 z{v#6O{F`s0L3Zgu;+?h(uy@(D+8dup<}EsV{yh3K78`-X``^9godP<}rPe1eZM9d^ za%~uTn^{wBNo}P*O3U}W+UM!s2@i1x3<JqtPc7&?R)p@tI>N&vs?BYFNcMFPW`hx+ zZ6+F81*p$+9!e~kos44;Ghp?iS$qt@<fgr)O`NuCcu~FQ-=C+|gaGf6KB6=Kf}}`2 zD2T{RR%0dIC5+Q}*qsI*`YMwXRL`tSQUZ4noiY%;(>Pm#V^77oF&jW78C}pCX%tc{ z8TlyH`1cuYdl=I;A7~k0Q#zX6ebdGPE4B%FX+@)xaO%1q3~R_7E_$Guu5Ik5Q`*6^ zL7b+;lDoWy?&>c>I}F*Kx!I9(pQj~IwBp3ao81*KbT)agV6!iSUx#LL@L3KzM^RAo z&}~nH;bX<sTueGHh`}i<jT99prx&{n5~$0%ih&zIn^seCv~G&?RY$%)@1^_(oUcHE zXFT7s=gbI;7dIy<Cy(dNE039^n<>8Kc-X^(%6Q4S);on)&ngzTp*bmUerxhkA*Hp1 zJ-Z)5&XH{(1DReezT`B&quyQq-InR-#G|TqeP)&>V3A#_Z`Ex`pI;W%(TR4619_;n zp2cr<1Ga_x&d`px?NUdpvdBJ=0!P@01ctS)g9+cs7pt)844sUG;LE-1G{?3*#z;Lw zjD}a)(&+K>mwSK0)O_zGW#jc>Z~h4~FyZB}*bKrswT!qo@h+y_hKOD?IwX~xq>Keu z`K481DkKb~fDXg25`o+t?yH_n%uqc_16mM;UHP@j&G{hA`?ozTWdDI;w=W?(jet7* zi3~+Isq#nkh4q&&IV~PO$SJ@mfTfTEd5)~0?*kDuU^Rf0js*fOd_5uw3cSrW@e>aa zr3!qzLSIvXN?f^{AhbR2(Gd|!5zot%;Zkd*srs)Dfd|WAE%oFrGEqi7$nHCG&}|`N zSxTNBCr?e=L;JYLQVw)>hn?Ry6ytju?)>np#rberN3iJ;IPHIk@-ca(AVBf2^H2x} zfS9IEdNiA5-1%F6@o*x%ZI5(ZSr_<zC`f-Ucdp662)X&&sOHljg;EqwNG~wPoObRL zYcGEpG^Jep04kBk!CK_IbHzDDpof!$QD8Td&hL`INZgd`rZ6n`|HI76kQ5$~vZ(8* zzr}*?<fK}V!8F9}8M||9IbmUS;juXAezTLnM$N?~M6Dj?nHwpOUT&R6piA(!WLwd8 zxmsy(N!W`XUODXKbUMCc_Jwu6c|5cu<(Z8<x?oAOk5=dSZ)(={Uz0rCGjwH7|Fc`P zH6Gtqy48bJSDiPO!WGn}oEj&$@%sZPK#xGsG49wT=-$em$&4;HzTe`1R`-gd(WZD& zHwDMB1m6!k;E<RtoxqyDCAUO!-_>;Bejz63czODN!VX|-I6J@1vWnNse8*KoE7x{m zmo0~6y~C<r)7w8VxUM2>6>A|Tjh-!3!5}h2O}#s(&W{?C!~d>kYd8Uen0IeZuHaBO z)$4N<<tUtrF|g)Q7B+C+DuroAHyCR%qE{&!97Dj;6%un6=biG~bhPzlH9NCza1_Zp z#4AYp?i7U2$U3++qC-Q;_$4dZX^UlY`?Ux`n`c>|4C_=)rL3<^dNW+?sMOTR8u0g( zVNNR8(1_xtHnzX(X1MiT#PUl^Or$I<+%72C$c)Sq)LGsi9|{!@==hW8#|mq)%aMNd z!(=Q?pV}os+1H(IG+Ixf#jK-~eX7pPZ1YyY6OS|Qr}(%EVD=+7Ex05$ybDrOU!y?f zQLRr76(5XX*W!S_kA9l@B(}Ifna0>hrdL@c!VVFg<;hl37FJGGeIk|FGV>)j(yu$W zViA4`pR&R=9T+FblBxQ{$f}>%_=d@Bua@wOG7S<_6oEJ4pX6?eMtwJxw`CRaG&<$| zBhAT&fABH>`=Sl?{qWEQId*zSLB&*7MZ+lzE%2cLirr{B>;Oql-fWhn-(7SHCLdAd z7mwTYl~p15%IIsQ1B*0#Gt|T!v%M$tKxDfuvmO-TXmzjI1Qw7ug8jnyKjH}mbpICk zvHWJ4g5!1P(>YJid~1fq6OFkqjUQbz=O~*tkvCn}6Wqv7N%=49ZDFiH0QxO_H0jhT z>Fdz&;5c^<>OhI7=*lXU3{TxxAkMgm6bo&byto7o0aFo^SQOEI$)hd?z)Cia&5_M2 zMLOl;U`od4^0(GTZ&b!NcDuz!ZO@CNift6zC5VK2jrfZAvo(8BXL_x*nbL%c7I^ZZ z5a))sc2lY2aqvW4<w<R}10LAA$$58Puige4yVxZ|O~ay>HJ%BtwStoo3K_)<y4;NN zSm9E0aA?cR%kw-l9qvv$J1Sm^BKRjXeJ!#x`ZOTqHv|Y<D~g^5ZM5iV700a<IBnf& zJJWO<%;-NiJt!!t7BMLX0&1%Z0>QR|0*$x9QvKN=#HH@%(aH=V-?)P%x0N84Yy;}l zM`2_8hfa1HywomwHd6v8^le(@EDo(`{LI({I+cNLh6x(ZZzVWZLR>XTpV#j?*5YyI z8XmuA=L}gCXkvU)wO!6Z>1#skoWxcjCKE_6K0g6YSZ4`LCt3^4iBYP)=#<&UB&BVp z59smInF8~n9e-Zh^a!Ne1PMjK%{qb^wJRa=<;&>G8wwzaK1w9J<-O0q`5|M{_B9uN z$G25$QA^0NV-G8X+LgnoXJ(4}@Avr<nXPTEaRb(7I<bg|3r{BrOUQ&Ij*_%qu+&*v zJ-fg75!Qj=x2ksB*a~+6HUwTGj#=BWU)>b@A^4Xo_)mC?RhRL3T>m*IP|SdH0_8}o zk?s<H)L#~y0i~|PXt;PN_B(Db=@alSn4`8K>L%-kU1&R*UbVg?BE`mjLj@!q_EL4~ z%*%$;g;cgi4d}Tnpb4u~eP0FZ#+|k|LJ_c_h|VcX*$c^oL3cPh4D=isE9qmt^zMhy zC7HgMoLJzlXIZ-n_%PQhfBg<ZV0|kfx0LNu%KQ~QOXcB_UStbs2EZj!Qk$eogS1K} zhBV<_myG4DdS;cRkmK^3kIcXYBc)`*{F}VSU}|b2b4kfA(HreO)2#!k@v@oMe-A;4 zkQcW(vd1rtR?mjC^izDEhx}wU*l=dc5d4!B;uzB(zq6@kzgDnkp=SB8A*P=I*oZU= z7?&?+K#cMewKTpKyt6EpiOR|q^gWv-Br=jQ^`Q4Zo0**4=}kNdrMLWP7n?5=GccaI z>A0d=o^1nsEW!C$M738X{g=~P?z=7*TO(XFV$~lhJ%f#?2`-q%ti=4zjiY%5l4!uJ zp}Yt~*ClDvu)*cs0In;8n?s+P#lLziGN!RxrYmNNE+m1{lOHz*5|rvu;UhY)i%Nh_ zG0xP`1N!b<{5T@?*Dt^5ExC5DfDu^K#`~YrvbJtNeX{Qj#0j`mX#g$Y3gU2}SWaSx zbwqy$?mzF=_s-BZ_8JKtp(DPI4VR=VD%42(N3s$Rs3qA|sz)=rSPcFv9)3Qar`;<` zB_qo$d_UGU#8J!({(GNEO4-z$r)^4#Wp!iIoBr9M8jC!7Wb15lcr``Tv_CBpo9qV< z{9*5EO~3aNIG))9ML%Q{ShE-|+DES(i6FQK{X+c$kez6Z<!Ys$m4B&Gh9bQvxa}5K zOm76l4{D!F!6?=ZMas-DI+Ztr)wBFdd81CDc-ocpx)Z;#<4`<gn?7GVOv82@aJlE# z;<-Yfw8XP&=*D03pdQVe5(=3AI1sT{CH3ci{Rl-2XV$2OM?_o-v^ZW34$qx0-Q&{G zx&~!33YWQzoC`2TxEY;F6)4yq8AW-&BQi&fDr%=BjvKvQ0s^NMEN2dEKj89Udh+Mv z-$?kc0FlimgGc(&>E<P!>E*oSRPnIYP-b+{HL;LUbz}y+lpGc1^RdTaEJ@;MB0f1j zdunQ01#J(H{=LtG&`g;@#c{uCxxlZ^<H{0=9Bq~F%n{LwguLG}BOT0^t8Kh+IeISF z`_K~kNsr2bzN0eL)lUArZenciVq4;rBwW?r+wZbPout<^ds>>RzU=1#1;-06!`BR) zz8?dnKYDJi(Y+wEz%*fM4c%eRo?8wRkCH9BkSZV(@KluZASr~78#HdZ2FOm0AgehX z&K2J6i3oTcdDcJ!Xz;%s4Xpv1w4-s-fv89E+~sua_%9>|urBQ2{EE?nZE@Ta^R87y zUdC$Shk}cm=Havyi>{U32Xaw^Df(?)Vq|?Qn<o3q^29fZ3fkKpY96_&teSN`<V<dd zU%q(A3@*BfO+(()sl~Zy3~oy*NKGx7Kf*a*ZRTgzvTS0uKIpDp=-*5~SK|=+!m3&C z;|0<ZSIKg^7np5sDw>`v66bBIGKwU)^H~`lp2t|&{32PpINF~qguo06K7c08iy&`! zUXe8&`tY`p8LPon_s*mD=c*j}`v1;90(3SJL1vC??iY}!(E@5AHnX+bSLFQEIt_=h zC;c?l#!{`)s={dYf;fPeK{{)GpYkM1N3!dakPkP1_r;SXqrHU_y+6Qk9+2_!Is|X; zoF*g*(=kp6>D(;*aX<FUgiKQG$1KO23C!eZ|8V+7?JZNtys&ilh28Wfd!N&AQ)-t; zn|5NnLiQkg3*P#<iB_TJx0$7;OGc7FqoWd~o4)1hxhs9SNS#d<cgr+cPZA&F{x8Yn z=B@pq!zQto>Ym1&u{oY{kX5>Yozj>mn6@fobl^cnW^n7LUUld0aO!la-QfPbOW1@u z$59J8$fpYgr_m-)pz;a{8miVil!=i#{$aylHv(tFy-fVbu%GWlT?`X0t1)XKk4|X! zT98ftS^k3!`4qeeH+?fbxS7EgUFrE25X|l!9+kn117h#^rh#6;8;(tHtC%HCn>5&> z<9-EwRIA}{4%*M7;gNNZ%odrqFMUr|EB>PS@0jb`0-r{2cW5wLf4jTnTX}Z_j@evi zYnK1)o|rucFWnUnrPgip$^U!ro3@*!o|>BSYajrW$_~RFim%Xa3NxxRZUG*>TTFr< z7W`!V?y|R7L`88;nNBx0jT5u<$npwi`sVfMvda|D6Zk|ksBID-P%Sy>PtGY&U;_Lu z`Q9XTddul4EPBSSgDZ*0Cr8kC-rL^`E&(}bK3{eDA#8U<)GlVtY60W#Z+>4KQ{2xc z@LItVqd2JrKDlC&P@Gb24em6W!sm*%;9x^k=KUw|EUI;NYH#V(0yycOGYDb%_*8CW z2+{gM1j8VOHey>!b`2kFn)<s&e4tiqC-iEwsHyqpQWJY7MvF|4zEJD8|DMENW%tUG z7a%iFA~y+ozbBnuLM3fdXLGZ%8#=U_*~qQ|lP_AXW~<_%bi5wXH2JF!{|UwpfTApu zavrbsM5ulGv^nS^bdp@`|JZjU1D?I{`8#Or&epYV#<7#zRSrz1A0-Rs-)GijCT24J z4ZF`%+c2WwM1?!w@(%cG^y8y@<j$(zK&it#9IDseCl-+C%pH7Dr4~~*cC`W6OKnb* zE>F3E7#klbS|T;wel&B6OGrwCURnTsBHx2*E%i1}8!$;{>W9)n%lc+Ywvp7E9Loz{ z2Ku{$yYuiJN1OGn865+hS8FtE=HUmVb4t_4=qaXfx53ltgpIpT)mO5m)Xm4Xw~L_d z+xsC?twruRp(X=_rPoY-n2brC4Q>qNJLhIa;$&A<d=j^cD}0x+q{$)Pk@2J2D+w8# zSJSd<VJ_mJ3Jthk(<Y*zs9v6od+27;^P07rLaS-hEG0NtXpbKq*slfwD}jQlv6L&( zlP~lU|Lg@&K=EsP{%q(}4|{Ku>z^6-n#2&RR?9fHhF)Nu|C?Ox!SL`nv@#3GCB%_| zk^AFXO`blEaRPx+@e89a@U2C1bFh$O;g<a0jq}oNA80*LR&Y?39GTCf2I~}I`||C0 za<Hsf2bf<eYkTndvi&d6k@7l;Je#<7=t5y5FW3WN9%8App)8R#IR&%5@~|%uCsN6b zO-JP_Jo=i=t@}GUn7EA2tz0iIBME;~*(Hfv*Z2Bn#LBU_R>QW)CXv_*w=l1&JP7VB zuo=yypx1?Fty7G=2S~R=`LqK`ay1)+ksiuhA(0x&SyU;-BbWCgHNZTK+jZJBH%?y^ zF?g$C{sM8h5GpPT@xCv&7VwlXFEWTDv}7wD_NG^lXB)hjl!tpIJ&?K^cwrCnid<GJ zPMv9j@oe~(<ea6WNEjV&w`_;*$Np}v!E_yeZ)N}dNRcms8<3fo3ASoreifn0`P%UL zhqpdgpWFo8ckJ7Jh5aqimRr13%4a6!rRLrUXz4O+%R}@=g%}u`>2EwNHoYy+_d8xM zzq9a7V^$HR3d;yDESQCBuYlOpMzdK{YJ$wl9z^me>Coi$avCS!4SHQ&yqu}u1V*~T z@pi+5w97B2L59Hv)ObDolEKl2#GQp+%~M<632k$y-NW%3Ykl*?u5WO4rgvZ!R+k)^ zB+=#uJH1`c*?zE%B=Pa}&-@Z!y52nJ1&Lg{@Hw4(CE!)GSa*)L2es8rBoU^e+v-#3 zXq06jAZMWa@ZKGZ+Sj@<0R`01?)62&+vPQl*#M!Mj{e+m&Sbj3NSF3N!Hjpj@BT6v z*>U%bbguq9Jdj0zE#im1t@dkuN;E1c52AG`%<6a0$HHogAG7z~0)q-IrOKtTck6rw zd8>2L0-XQMPl*iN0RMmYMs{Zk!{>Ec#+6m4<s<@*8p)D<vh%kXO0I^kSFD8G&IKFf zV%XPgua4f$S1!vPmVIGh`xR?~Q1183lG%8_HD-|zNflq`5JQ4~CoJ0uKbrUa%v-!U zn2aG~>L#6iU}-ks3QMWmtVG;@uCY4&$jM9Qe(i-|W3!2P;j^qy<8=ORV9ud-Zu?Lr zdf<rEquNSGr>=rnNGQwN>6X*A<sdVOQK*<^XmnSujB%luR<^hHJJf<&VXM=5E-7){ ztJsyZ+^wm(&<6DX5%-o+ZG3ItXj=-D;-xKGT#HMCySuee3V{}<xI1ZacXxN!;#wSn zySqCCf}QDK?&m(wdf#tntur6Ugh^)h-kI5Q{p7m8L5%6+4|AETamhOf#ANK#svqjZ zfVoOknU;{vVnW_-5XI5smVLh<t`%kRJ}xY2E)giBaX(Jsy|k%0^L>557<I6r8X4ns zmJ=IX(?i9+Dpm!Z&r%u=uDOYf8M*r+73d1}(hS^KD4-uxvXp#_4~#9VjT!O*!ozc& zbZSAO#R(jbY&d^!QA)VOWYpg=8n>nYj6Iu70O*p#I$*FCjsG20W$IVVX{E7sc64ki z>K!hkeFu}9H@wl~vf*QDD<y6qGgpBVH68Q&I92DooXqT-ny&#Kj(y^E^!1e6K}@xU zSmHKU6VBYOXKsI*l=isqj-1=EHgmS-idKBCb{ptKiG6DM!Aj&*M*Wwtw*5nS888Xc zdxmk4Mb+6WXU#9W8#eoW+-X?oPFr>f5+fhnk03hI2InwWX`d?w>d5&T^W6m0p;Ey( zB}@6kH@a(crM)$b*yY>tuxDj;nIAuCN6pvR%l~Odbh)||u*11p2}iZnqN8Hg#A0F8 z{?UEuuLt*>jm=n8pG_6#D;1z2Q=-+R4MwRLk2lw3U>cv!T}6}%2TEnkFB$ov^M5_+ z$DSY>B<nj48bn|<&idEp%$<{k<F(=0hOy9Okc37I`V!mKC9NWpD3{0vcF@-^biby& zo(~XccW?t?)8N$tOUIVr*Nh7|d?*Z(`zR48_V$i{h7=!GoL{-dWuBqGMCToToGrCJ z3Rx@`PF{YKG}f{E<nHb@(RezKJr{Q|a*b%be5I`&;$dkvaay!#zMASlWpR+1M>@#U zF>^TwGbs=wkxn;1vFSf-m>-*&S($@t13!X1sYAv3@uJ=VsFk=uH(}<_wBuxPG`A~V zK^ye*&z%CyLcX>keJ=wtk8c(Ia#B_+1Ji(AFCwt@x(CMMKNsZZD@mG|{yZ+JztUf; z;O~$%pHgm%-=W1|kuRD5IafPXLa34K@I1Y!X(uc6iN6krmCw9W&h;r=no~%SvY+SK zi0Z|5XnGQz9z{Z#Rsyex^HWM=$pDG0WV6~mTD|E%P5h3~TmcPOvHjh*9^xOS;5{b* zVyP4Ws3-Gqx_$&REjhaAnQF2=S6k^hBzPB~^(5PBpLQRSnGr1cl&2z#OOO97KdZV- zLc^XkGq^sq_K<cnr&!xg-_<V^JPnu(geqxU<+Wr?9@Z)S#2s2xo}2pn!V+i%e>DQj zbNm@z;JRk?E9LuAJH}VGlGfD-rt?>*i8Y-y$6go{Eh5gm3bzi`EeoUl`!}aN&v?Gf z|M_l)Ug!Ue3J9(}|GxRBHb|o7-@4NO8Z?kZ!~e7UcU6IJGmc~g2*1=(Q=<mF&7<^g zZfuV3oayN3B7m#~NSFLi>^l?a)QgHf2tG0r{GKGa8S+B!SIDE~choM`s(<BFzw7d8 z@{Nh{4&JsdFhX6ST;Mm<72kO#Gt)bsPoZ;iE==EK)YeXMGDpYCnCx=r;XwPGVVn)K zNfZ~4i~64rwlI!XwI7fxX<oN$+2$REe$;BpK0f?e^=$+ez24yym^aK1<QY<NzNt*+ z)}(y%2G@T0<);yW845rf_Q|yi;m~!`UG1|ITysG7OF!zHH!50MYkNC;sJxhgj_iM9 zH~+STS38!O-Dll!$AaSGX#d_EHj_CyjaKU@{q2L;U!RH$?tO=Z`L$+kF;d?%L<0s| z{gZnH+%_DFC<8!SATBNKwV`~``tsg%ifY&Oc`XXExzX_)qikkeUEeKmCwB%tX+YPd zw=iguOJ)<52uY&FR+jyAEZmjM<NhUgD3N}hT0R0LDsKP?Oc=C{U{`vK$L89Gk1W>L z`{fud|74p2lL(q5{*B)EqhHt7A{zMY@M&9#=&q~vXBUj;taPE=Cs$g`Hh^=X@r(e> z7$IF+Oh;#;qN+N6fUjPrHUMO((w`gTW^<e1f+gcuj!sJE|1{a3D)}<o`j|eL(yA2O z$d<;hnQhRvGh5}@8f<;YZ?^FnpGDu>Y^kB-_nAnl#ZtrMW!UbYFc7R*yz;S4*c^6@ zYI($|d(!vAPU6`i))V0Y>i1@mv8A`2``a}&burhNP2Dq2%HGGY_WRo-esG#`0pAXh z)Q!<0>cqUfQP?dGY^L^td#^o;X3~<S&2ij=F@?UzC_08saQFWDv7te7bM>Ju<RxF4 z(ev?A$SY$YVEd)ivfuUG(7Xx;YkM4#yq^!XakigUnHLzCF=jjwKo3FxqKz3HVxMt* z|NYWEZHftV!6`jBI*V#Y0v8tC-5tBnFtfsc-xF0Pzn3)U#T@{0`;wHeZEKIiMNM4R zINy*_z}UmI@@|%NZ};Pp509~nBB$TzcG^_;oZ`ac^QF$^Xx5K&Z%-7O!lpr~3Laze zW+>HV5R#KgGW)A*_PJ{C=m$)D+i|1PgC(NEduYvRR5#U0H#H2YwnQ7!He>N3z!`VJ z_IRMt>9}7beuw$dA%joRRI?jClvI}()5jTivug+&^tBvCC_y>2Ee%_2zRd?a89iQY z!rE-EJCoArEYsyT!FiDCO`;95k>Qh}97htuy6W?&E$z0)wQaNmgw)}4$lLq$93Iv` zGZwOsqhtZXEnWflBN?4zRgQuh%^=~I0G3@PE*@Fsm_{_ro7;^g@hs@qULrvV)+ril znH$6R0-=5%+kQ>76a4O19mjY{3>GY}`9yc;YW4}7e~6A~?i>qR3B4N!6cmx~c-g7= z`N2SVnAp|vAq}=RtL^R*N}jsT-nE~T+dbLpDhcr#v?Wk_OO&qd?G2ObU%6k-=^xlr zqI%X1YlU6DyJ`JHaClFx(6Y;~lm&?YyjBgKH`t!XnzNaAn`tD4QvwO=*dX8H#h8P} z<uO00m-Y@9o6R=xi-PKJw+1z7z5K$)mYZ+(;2}wo@yP(t9vTsnRM&qMEZ5gR98p<$ z)>dY!%7UBB=PdJw+O*5QEd|rQr!s<Rr{IQeZ8ky;2iE+lA`@GL1IC=sap|7!9OJDl zdHG>TPmJNw3{bjvQvZQT;pjx<+rx1+J9j4ZCrtHzVGXdLh}Ep_{$LH;=(Zf<v(U|b zBh*fEElrEKP3t24i(k@_73{Lx_L$N4!Bwad!NfEG3RVeZIMet=tEC&usGhWX+fgMB zu#-afr!A!<3~R;2nd9U*Q}K!(+6tmp53&<%-}WxKBPW7k$d_6wD(}C23o;nZWGN^t z9D`Q4`#;luTc@&BZw-ZJ4L_I$HRS4~I_eou?$xlF4N}(I)W9qDFMemPy_$yUF5-b7 zOY}l+$2rZSXV2f7=$AorOKkPl{KXgoVu6sk{_5I`1b#?HGT*1`3Aib`&|9JbK_=Vn zp}60xs}^VrZU;Q(v5p7vp+m*s#z|j{=k~Qm%N`o9Te>d2ZYtYCwgDJpBbq)83bF`a zbiHag>+Y@AI8RPh80W$0U|VNBVVaq41lDs*;07rV505Ii1ST+jmTS4I44Yg0c{spc zSX5+>(35`}Rdcw-jONJ0Bg&*xI(9Y0%teN+Jj!N!<>tt*VmJgQ1w5c9jgWDgZf`2t z0)hi0>utC9@DPIQfLJw~#TnW(Gj4;xFR4hCvFfIWP&G__OW=-I?zzr9p`ESO%PSS- zd!5X+dC|$1aMa&ONv+0tQGJm-g=C)PZVeUKWp_$;g))HGN2&8(D#{)NIvu7}_kHBV ztU#%5K@{;3h=4S;?PgjnhI@mBeV>usfVbgrVSE9HK+A)M7UFulgk&pQ9zD95m6YKW zUgrH*s`~xKh;ZJ^Hn_F9Nadj8dA7{wU!@fle$S?1Sb;`Cfd+k%pA&fOOUG4>k`4+x z#`6rU^{WT1E+y_aabNRInk>(yD^^t)A7sx~@{rZgsfzgQXZ01Bur2Q5&N(i1{GL-} zZshKUpt+h}@^Q9a5OAoCfy}0Y5ih3CP;n=YGSkk3xz9T%m<*OMK~9QygpD>1MInAI zr|@YMqjR~WW}zNJb3&X(5r7#su@;|2q9n1EhIq8}rfBN>6k-`B3lP6)>9{<<Y3Or0 zen1eORwe$ZQ}6t3I8#bv^FZ&8z)cMRkq-qxVdG4;xD2rumxN$$CeS!^hN34K%x!uk zl5ptV&@LgyoQ~|Ywbn~j2pd_vzyamm7?GeLHQD5tFOXBv3`O7HV(uDUY^Ee=VBRj1 zPqze7b=N?@88JVPeG|?qHZ69fZ8v)M%c<U}pt69W+TL?qXF@VwQ@z}`V;J2~<LPEc zT^*NAL9tcs8@TU-49Rk81eG4Of~|vSc9d03ka1qygWS!ij;wBMQAkDaq_WzT#igGz zaW|c5%QrphJ$ti0N_B^==GeXAzoyno&u!IuUwPem<W(pXT7qMOrkVSHo4!%FzR|DW zIBNXjD(>jQk0Q{SuhP*$u{m9Kubi38`a@XV2_9@VTPw=uxS6ktNAvY_mx{d{5npr> z>Qi#WsbAs>7nPr-W#rY@->R7Z!8t>CzGjqLfIVNLI=m1<%U|zN+|k80Vt@wQEjKce zNvA{~>@DpFcL{e&?7~*U<rU=JgPpPjteCzwtri>p3?A)k#BTQ^lW3?;NtoT63Dv1m zNTCjQXAsfIw_J|M=q4V;X<}DU3Qm%$dqUyDHq6+@R^_lu9+t*1(S$fkV8p^~A7uI9 zI8r`3RwF4$XCn~{Ob45Sf2rymzxCra*snd@>lMZo!5U8GUEZxFZ}#({LUq0PmG`L7 zSpO_N%6e)o*g$~pJyWb~9MkTzUyLcA8K368hVvM~h&5WjMWH$HB>!e;j3pBzLn8Y; zGpVJe@}6F8Z7>-Y-<fIzuueyZ$SQ2Ed$H{|Rv7k18`Fm4%r|FDP$nKL^m=N72h@vg zY@M?Yc&&uC=NL<y1#izxVMlovEdz07QqUGJ{eq-v>qMvY#e~~<#eiu2QXL?eDM5NP zlduUhha%Tl1KldpL3j2B2Y?zxx{cKr@!VIq^C?+w56ARvFJDCHw7?~)Fv|_@%;g~` zO`nE|;bF^n+DnhWs&c(IwfEkTn{;fL><!`zniMhwg{U^toZy$Bm=#Bk+3w*TO;wfM z-3#7-mLcYu<4;cmreeL!kV{i{MR+d6`TqE~G|KGRo@v<tp=C2oBkW+@8Vm+!Iee?r zcAq>&E~um>m+1=<$}IfQ-gq<*5$YMW^nfAX6x-r*B9Vmf{$S*F>Q!>yZVsC;G~i#2 znYOKEjqU8n;@FF$J?zb4Eo@axJFU@?mM#EzW~#Z+M0%XO!3lpI_{&Qyxs%j{VMCG+ zhD~>ZkaHN%a{2BE0^#vXRA}#yc8bm8hvi2_j7?&Txo}~iU-Vr`;WPKW*x^)?&amP< zKCpA~M`yS=y+BXOaXo&@W&mffuU+5p@kg!IHoPzI4=DoNA(aH~i>r9zc2fpFR4CxW zc6gOgc3GxjW61)W9%aGW^J64rc79>RG*P*O;np~Ly%YFV3r*`3kEHa}iYZiCDWUN^ zwxRn7dK`1h^MiPDC#6{BI5hH6koz0cBYs^w^xYEl-Lwl~(CDDO3(}Zumbw<?YEHc` zpwdRYdzNiD#8PSzz#j*4hcxYu+8;jGxTG|luWrlqmvbYZt?GaKHZmUv6Zh)|@_+$& zT&$zugNTi7*v_?n%53BES98ueTq9Qzo51Ke9VZTtQS~G4JjJw0c_|XeRZP?Q5lutr zCQp(GX;im-a_%L_WkIZ_@h%|etLL%4!V66zGRu}pX=;Vkj9Rbhqd$bc)2O^c0iU}* zD^t7Bxf4rG&;hZaHmU03%-t!o2aT%Xpz_)e>tUy~!#UTcB)=vjq3c_ybNe$F6hiAv z*SNBl-GZcE(y}LFieVmJ`jzwuhy4o)uGd!MTD!XpnM)Oo)QU+t381ZEOj~pAPxkEC z-*;@_$NAP#27d^|hIjfYsH9__+{e#aFjcG)2xU`vAQTi5iX-x)!HSip7la%ieH?_B zD{sPv<{!+19&&-3u`<|xQ{S!f6fMb%<FcCKA=**!x+`S=ro~8{DF(>FYodc`n6||~ zDvjV55~==hjjK`reW+7jwC>Y(FcG_OpTHCc{rdEt1YLhGughrPg=@d4#)vwWmp-OO zGt=p}4XqM*U%6^;mxvnn6z<G&J7vEyi4;tVGtBK`H*U<R4Qr8BjGODd-woii*7BVb zNRY5TadcKwpg5smch0%-m|uZYy{wuFP%VqlJCl8(%;et#+<*Y)<-Sj&^NUOpHrDvf z4}WI1tG7S6=_vz*VM!B<B~cv%G?X=@!kHKh(fRkinR&e;8ZInyk)dU=19ilQS}&jb z_`2yKZ$lyX4TH%aB(DzV%aA$&_OroH_J=Us*IWoDO-<`)+(a0S3!XlUVA(}o*F^Rm zX?l=hhd$M{)pH<QR6m-L(0X`p{}NT2{ewz_$D%xnq=TLA=cB=e93)y)A~e*o;Sj6t ziLrQtz1Qu%czn9S1o-m~2jS5ZAL&p?k(hb2ET)p3;YSCcZ0yg*>l@rVZ*h?^xB6LN z!K@Y)P2nkt!z`Ok^hC(vo?I?B7UcTWjci1`4syf496dH`1P6GJ-4EU(j2m=zGumwI zu!7?aj{8SBu(|MkxkYhBF=@WUS-MD()R8j0XZy3lZAvCJx7pl$Pi6RdB$Iv=8ni(4 z1E(Ls7IzedeNjuOUwVt@hk^tqvlSe78>um9nu>Y=(D#4b_I*W*fX%^$osb7G(03%! zN~irBnk|UMmaA3~{oW0$IUu?IkOH@U%i>Y?=j&J%ey!X3)XU(Z<9C?&e9Z?dg$ie+ zJjvLko>(T$Qd<V|A*O499viNjr$GPBrhTZM1TqRX9Ma4V9g>8S)+{qpp;{XY(+b5i zy$p+9ppd+8IlK{iWYs;fR-y?%40<bR;%)IL+*!>G=amlN&}d|m)^@$6xH%-WG}TAH z6a#A)7Dq>psn|ETLU{ph_AAs@r%UwRB}o1eK@Ye0xbaDOL<2S|4TfIQ&W}kH=|W6T z#FqWf9zP>5xk4YWNFW>iC;oW)3l*nkZRuoZ{gxi|dSH!flKvm}0bcm7gx7W|HFF+< zH@nacW)ipJ^~48JL0^4}XIC2qs&mt$suS~32b)t*mc_GY4xCY=(@9wLvn{61+ay3Q zBI=f$C6_#4?je~IXm}P3Q5BxAdxPcsQFdR|P)Yy`kNT_P5!(i9Efr2NxwRF6NJly8 zj+0(v_13D%gfNd6Z{zjUiUS?bd$gSI!z>$XgbY<7SC=q?6khH7PR%pzJm`m)$j##1 zq+fUPs!6i(MrWnzEX<W$6!tr0-wZ&3oGw&?fL(Sz_Chi42wA&Bt*9MCi1t?Z5f&%2 z!i1E0f55qOr6xoP*ZJn=3loTGEDjsaX8Mw<Z{Zd)*<80@X3e9$`)JmpK~SZYC24)& z0T)-VQ|k{0Qu-|}u-U`qq~+zOotfg(wFtqG-xxbnMQMBV2gnyZgoak3sx-lzFFd5= zFL0FG%{smz&>i;VjshH{h%Afy-=7W8C7fsdy*<9vxP<IORqY>gJ$enjEKcme<wDE) z4?PhAdQ)0pLSj4jSdj09eiVB43w?rp4Lh92b=wJb<7IDJJ&IRw<BfQ0HoGp`#S<AF zYR?20%8uH>NsUZGLe*q#1<h8n9e4bblxyTI<`?7k0`7z)PxWlnhlzJb8(T|nccTUc zm*VyBM94=Sc9=njYl?Ur`th&64XQ3gMOA6Ol@$AaL)O=9R4uMCDlSH!nbgzhY}j9o zFK>&gtd8|%Mp;|u>H^h+guH3@e8=))cCThX`k*saKkJFcho=0QpP~}INSq+jvjhl@ zVoIs|A_cA&;#>eMS8ImQ?Bxa??$Y^6M(3G4v_;04t!FHD0`VLR{_^Y(J<6VY6OZ4} z^9r?_-2=ayxgps(@aD)7;wHK6JcU0qIg{6Jrq@k#Z#tgK$TmMDK6BVaV;{=YZrP=U zO7=1WYWUoa@Yp}as$&-IdsstlcbeD*WkWr1-dKi^2*9?he!uR6h~>u4X@i0gGEx7K zeZxF175<<++91Wu%T;XIi<u_}#N^*9NM_M~CnAD}xWp7IVza>b)4>{BpFMhreRc-> zho@YGa919KbfVJD{V~mBixi(B%@E0C^9m@z1OAFBGNX#b;!&<)+AyzvZgNYRlhzKo zt!R1+!?^)3D5f#dd%RDhdem@ua~$moOKVa^UiW5C2l&nTUGFgDbqM?s8*KfdpSz&; z-K-KkB0R6(=HgWy<ahSotR<!H;ZeC{s<-}ZP_=Sicv;6nkS_6}@xh_~w{~5s`^48D zPX?_eehbv~Qk_|jMelwWs<_^+vZO#N<I2+id8_?49giN*+%)-gNMZIK8KW<@gxBs{ zA9n!W29_IM0ay3TAxLkEkBaNYVt9a@@r?xBHO(OgMb)TK*q<VrjWN`J%qEr_A>v(n zbP^($!kc7FaN{NGU%l<c$?jCqo>tdWAJtAlPb7X`h_Lsb_SmxXFzQ3tyVig{I)WM@ zZv;BJ9>v5=)RC{e+^L9zod**n+%BFxRa}xtG|TxF>mPn&kJ^d7K_}ssKr|{rMnjT9 z8Z(JB3^b0}tf1dKbQGLO*?drD$#}MuV4On5IxtZ`*g>KaCWTCdz|HNtNR&w1>T>@s zfx|qM$NY<Dovh1US3zWW{6clm!|jg;Ks;l+V%&KuFC(SG9#PUpp<=bMy5Kp|NxVN* zm6mNjlYQxpfKllR=r^uRg}Ueoe_4ISB;XO@R9pSkVIulmZTY9@1WMQE-!1q@-N(k$ z-PO|NLJ9tAObmP_8*PGjCr0wvUcu@_!(8S5T<dO1<bh5Q_;4whJ2E*kH<U?8kSu5I ztGXJwOUv%4`fW2$qt;Ei<t;iv?xW}hg1zwamD%yvAIBXUXitBna2vpbsddBaXqgx% zD_?Q5TkoNT5dW?iCeTw?2<4gNC*SQ1<K0JUv@ad1Ftz@uLR>n&RbaB8G*+{^^IJ_p zL0WzLYk3>ov-i7HMCqcd^i?TX{yxDm$K5}FWkC&ebjo>;ElJhjbRJvs{kE$9q-Ob1 z<JSGl98xo;B-OoN3rU?#O(a`2?{+S`hv#DLla4Scd=P*U^Kx=dOA`O8n5ogU1Z(~- z(<st=6Y1}m?LO-daBD*zhOA$%Q_D@%yumh1WluzpDbBiH4p7ZCNy~K${x;`OnAEC4 zh={X8kUUU~6Ob;fBa9~E5~)@NXsYI@vzH8mUB&oDx%OX0afFbMdU9>frh7FDM~${b zC<~DLUSw6@{%9AC;ebiSz)7$4Fb507F2)?NNQ#sCv_um=lT>2IRypCjnnmhTFx}(N zrbc8GgzuIW+e2ew>h0-G=5TlThJJ0*=p{yn@Zq@n3}JOH7@2=ly?HT&n&|10w;z|S z$82S4(EWk#mEhflES2!swB?9P`r{4g)u;Vj%5L)YIPQ|QHL3vZTCMrY?CgtgA5DDa z?0K+W5mh1*4g~$c1f9%&MU85(|6MrA))&GWhDQlc40)UFU~t6c&?IF3dJe65AouMS z+16SIy8s*EGrV^`Or(LKtz^&0R^uUb6nZ{GJ%h6Cp^@3lqE?jw<h({<ue9_nbvtKL z`ENI)ppAtoFX+&fLj)1s2a-G7ycl0#clo}S5-dxj$;;hG5}k`gC(5RK9EIwPZnHQ@ z(+#-2HPgl7mzf}aD0N1C`(5*_WiST;3DY*sXSWgOQ*SqS8Q+CzzbHv}Y<k!U&><>6 z7dygy$E_B?wq-n}Ji{h1zm*8&#{50IAZpkAnn6h$PP`s#J2E>XO_&LBHX<LD?n#Ml z5oYfZy8901;%{jWnYCAN%*xxT8Z>I+6|A$EC&S5=ixV>vRZinG^{<kY#z|T;L)~J9 z+p%!Q{e^X%xMO#8t?<`dHOs|rOGr;|D4fN8DjT>xg<PvSb3WMLaGWwq_vzez)D1@= z6ftWH$KKQLP$JFQrqDgfX3(ns4B}b;;iwCvR(42(%zly<a5!wa9o;@l;#`j6w^;SU zZqxPu$a+xT`J*Gk>lA8XsFLyZBvJ-Q0*!$>RtDI!v_^&TxR+wQ@^3aGM_P2PnCT|f zQ*JNid)wuF(-ZSVfBFDd$DLv|LUv<hm7)RXU}Uv3QDXIvKI17sD7+%c#(1-Id9s={ zHjTdl=%G|4z3cR%nohXtW;F!tu4&x`c%M2iHHk-z?FL!=fwp|(h#BnaESKr@XylZ) z3n7KvTO}QU)^qvcu<jKh?lJ>9;5MFOA&I+Q+#}p^^wEwDbz4c4lvS$wDK^?c;L*`m zc9_78mJ941={mK-)Pxk5Jl~GU{C!AOgKJsKnRuM#M$%F9CKPg`<fx^JoZ%~^tCT`f z=SEk*e0|=?lLxt!$Gn+~@md^bOhFH*dg6Da#3|U*)%E?gmr;=$$q5<-k|vV9m#FjV z8tr02$i0}(@1p>Xw|ORrvjGBDJ1?XameVi0W(NCaDNs=8zW0n+92_0ZRizcn%Z(0V zWSdd&Gk{1(8O1n+Ih=+qGKGzx2{1e|Vulli=-t-)GDytYnC%y2G0D}B6kyy0dWYks ztgCEXgWt<y-K+YS<Rw-r>)BNOt4YN7(N5`)LM~9-tMw$vjgY|Mf;f<*nLQ7=8-$(H zjx_uUgtwCJg_qqFO=mIrU49bLTBKhN^nFE3-~4A*4EvO^CQ)VU?IOu$&GaK}n%hH; z9kAA#Yb8Scp;vd@_RYm2f7>kAf>e!+!iztZmhku#5Q0+j&P$@5;d^5hY{t9FfN!ys zt1XwGQZdn;17e;(0^;vUmUGpg9#%;_R?J{fj{7|$1xIZDUOunRMA~^ku)}z>QN^4> ze9Qo7(adBtXWy<EBS~q6;bk{3->#@S!@R<BqLPdR4)0<@Ik)7dB#LwV?F<gsdIl3X zTj|U>^~v-SqU|sAdLdh%i6(wrQBHD>YHn^0**mec@kVyX$Y&R$Xwd)eDve%#+y&oU zj}M<cNTxN^Oc>k$8mI12Ot#u@;z)^$*ZGXwU7|I~QCHpqOomW8Ka;8sF+<K9+-7T% z?rs(_?ZHUxN9LX{KzLroS$WXJaWOPvtNoc_9s4L4uRC>$RziYiR;<MSwS?fPub6VC z>F72ef+Re-{+x;(uuHaBY>=;7ak|1e2f$Fn@Y-5dC-qD}QY5klP9Wd*anoaf+eO7b zG|kcas!LtwXp}i&dUqVlOFI^2?maa~q2ukXzc6U662B--GYzt`cYwoLy1ai;pwHX9 z6l-9BgCO=nEL{>U)Dg6Ld$cHEXW)K*2wPm>RiXG+@ol8U3`Ct61G>257ONH!Vbmtd zfa;miP~Ebbj+8ElE(#GvM#eb9#Ano?2;`rots5Md1qzx!(I->DrW#W=nlS|<pe@(^ zHMTU1{w3Hc3J_6vCCZbtK6+*sjC*fOpBFkuImfoCYl4p#f+-J<dh4BvIRn#+ldU4& zV9g&Pd*^Q%zLO)FSl)dZ%$h+uZkm>!_liE$q9G?48xiEERt_VQ9f6#%IQQ(@sXVo1 ze$JinVPtUv*@0nO3}&?-SXfwOKGwp^<^cB>HX>nDPgktST6)}JXFRriJfV?)+Ao1~ z=e|QbH}!}n1=1CI1v%LBAN*$4ViX)HQSZ3?b52p<;Q;krk67!PX7$I+KR~XN23BQP z`mMYL=R%~HfvbVXFxqa^(kGWZ_sfsFQB$9gijSFWEAx7N_QTSkuU??3lMBMpiA#rZ zfB{Mz=to(!9Pv+=@Puqruo1bXzC@WBZJ61kK2xzM_wu(nbA}2aY$Or8FX%FZvk=MF z1+0UXXMTOc94|MIY<0V+C)OP*r678}VL~HvYywaxoi~03hVLHne|9P_3V$I!=9|i; zL5qHg-atAulTa|~pRyq0lTx=)*+b?V0sTPK5gn3}Zxk3fQNDinG8NkKn*u2SPk)<a zx5@wgsNKg>CPOG@lw#|F#UI4pso-bUTDay6Ix$FQO`T>3Lm9%tSx!p*>ZyC)d3?F^ zZ%!T7IIFh|3Z!-%5rWwix(~E;bnXWl89#({v2>&Ay%AZTqu4$%$z)?=GYCy$Z=w~l z^yn6re@W#a%nqyIB9^vm(sW|)@gaB?ApBss&AfqSV9?Sft(c900VX;~9eQ^CP7>2( zKfcTw!)dyO*{K%Wi_xorOxN4?E1z!o;;dM~T69;0LSmr+f^UDK_eRff$HhaMfc*E% znA4(~<^Uqf7_u(x^|y^dQW|Cia71ik*;uA10C+R;mMU-OjfgyMy<cyASs;d=G?O!v z@@-XiLOJxl`v%;SpV?!s1!osi2H?91UoX;|OjFFh;n|W}a`zDMX|U!PZ4~-5*BE*I zPnamNU!Sr0pF`ZbRt$8Ke!;rAflvo|BrPW;wgKvMv0%fs{6dvQ(dt(XoG#AAgzPm0 z{M(9XDi-q1tt<>X3XXGu5BrK}W7J{@l_=O_xRTr(n8O2YcJYIp(_j+6Cs&y<0U{Ll zgf8g(!jI|lWN+$WMP-fjeds^onRZiEo@W+Iwao!|&j)LQMPIm)7dikUfA$Shl+U6! zUJh%EN&do8C(XC|UGb+Pft=(R_W{C0V-cejXTPVLqqB`sg(Lu7brX(x=K4^~yL(TB z1UbgRnum4VlrA`f#qa)HsI084!Pdczi}oA$<cfg>InA@Xw7auUA=PJ!uy=A!`^HpA zn83pYr`D%(e>WDyPD#cRKx|fkIi_AA8jWXn&T*gWFVzdNRDHGa65BAmkgu*p=q4J? z=1|t*-5l;&s$F@ok=^^<=K)hoRYqwN1c;vKcw$H{f0(nSIujpF_arh)W?nqYxX4Po z&$|i`dPl;*@)+?jo`xqzGCn=hD260bU5*kO{fn`g#@uy7DmAh}uSd~=r$uIa)*eN= z0edLuMb6QQBug2q+7}l!79h2h2Ne>TEggk>NUgZeslv6>%*fD|g;{0n(yQC}^R0Zs zeg$WInwgkniUGW@A;NnfrKq<WW4pyoJ0;Ns0`WH9$;!~uW7{~|DUhA_y*6ULF;YB5 z_NEaiH(@vIOV6O7BBPVuUX70Cm|VWl>CdHebUcpHR)Bp{yE)kZ>h(g3|84J3NpxI& zobhI1kwE4v!CAqF@z%X9D2;eY<#3Qjt(HW+M}Ie!*@x8M1OXl*6Le7BaWZ^qm2pl` zNg5E31ay%0-VCfSNRjuCCkJBY22{R)8GfrnWvnm9Hy3JN)!W=Edv`{Sh@oK+P;<ef z@dgLf^+G#+q;Ey%euWTn?z<MPHaJ)DAhCLXer-jJzDqX4=xhCB^*l=FPS^!TnA^BB zgFkx{#B^2#d30RF9>`>+<Nk#18=_fTxcZ?F-_Of8#7zvX!3T5rsfu}d)WyMURl|s1 z+d8kcN5#(Fx+Y{a`SPIYwontW_3r2hzK!i-n675Wiqi2|R0VR5l=6PG4z$ka7{<X6 z-X7TnR6>QUQiVxxEU-`TSn-Gp8b>u|M64v3LZ%kyveU^cpKnKDd;P)c#r4*~lj8># zMlf1zj2#W4Fu7nZ>oY)D<hb<s8Dnc}kFVXW%`<MX#q7PXQRR8;x!BByw8Yrb(e;DS ztp_D9V35P(*M4WiRdsRj9yZJ+E`{D*FX)~!3{!c!xv^n9D+TLLZj*?9b6RxCo}6si zyj&ojN&fmrMCUBO*tshmT#9P0#??}O_qSGBd~`7eQ!)});_RwIYg<LZ##ry@Se7D0 zo?C2#?=TE!<<=9<rdw{H#7WHRKZFn<){IHa`sHC2`H^}+Pu=*-h)y%y*KU7N$D9O$ zOn4Ydas2pD_DG9>90n*AXm{@>InTCuk4>MWVmOd*x?H`>7khS}xi&_H4>HUQ#^WuJ zCmQ+LMcUv~9R%0_1ON;?5oi`Mk6ZTLgW|E2$k@96%i}P2q)}MWa???_RWZ*z<Z58q z_$B9yw?vaO;o3U7W>MXi01Jqq{_Vl(ycik}Z@d(CKWYUGQ}DXtl?;|)n!B)=C9`~M z_bnVDX)bPWATAMGGx;``mp9d>a+I0zUirOp_lMEkzF`bZhu80xNu<zHdF*8b9`9^# zV35apO-pYR5LTuN5;}e<Hu}RkvOA#>HTJyr>u1Wuz_L|Y1o53`>It0DzQ)}(7tPu{ z2q*0UTTV$Eq8pz2TdOno)soLX7PuNB_A&Sh5mJss!0C}m#GjZ48#9^>=k6cU4DZF_ zNdD=t3K&v{M)Ay*xcg7Eb1%PrL5XC7u<VPJmX+Y3ch6;b@SIBOK1=#D4BLR+ba)}R zKDyGjl%Eo3-*jqX5QSK(s9|Erbf@xb7<h!7^}3wIMK5LurD(c%a_Cb?$WWFMF}Bw^ zAO)69OGwy-AD$@7Q)2HAj;_;~_@>ncx2=Nj0)%_N6FpW3al(=XA=(XBJM@H{=B5%3 z2>rK!Frqvxz03&-QZ>6*_OMsBak5HbE<*6szdhz@7wJZ9o+*CVfO;3;^?>lPwR+<5 z#B0!j2T$kaWe0z+S|099e(<|~jWVoOv8gLUj=O+tIhGY*>`;*UvV~h_wzG^Wa3i9t zUTPuX3%|H!Ty*l*?f`8)K6U@%t~*UEo|duP`>f45(|oiWxK(PO8eB>Vf%1Dw0v&5R z@rK4$bkycjp(mH8+~(a9u*ZAm$0PWVct~LiphcS-ZwYalo*drcw94O?ZI`9m3j@aV zcR?k`DIer!`=UE$3_~ybi_S|M;{;?hu8G`v7luS<@Nt<x?b(A^of-nTE&4CG8|XS( zhyF`&NQ%sDd2)2$2xU-|CEI%{8j>=z=yExIe$Vm@lZ5|0U@Q~u97N6`|Dv&58f6WC zUktsRuzY+Qyt!T0L`GtNl<UW(C#4*beLz;p1W3D<ijd7P2^h+n&H|4-CtP1bdsi|j zHbd%`qc|V#QtuA7%Lb51JhHk-cfDAG2NpdeC&;-x{Iw{X5ra&Jl~(Z&<|x4JfIYM* z5U35~;i8dG%DFk56^H7c8nt?WJx=EZ_YyqABe_tybbM?HwB6UuozD+3^}Bjm;l(O6 zB<{1S<XP)>!Xcww_(5V#Rvze^^R*PB0=I=pcNf#GagIFpv2{z29|yv$P5vm82uJoA zETrpUbr9I{=JDe+=BA(Et>Vo#<fg)Hq9BTBkiXa^D%!xyFC@0#FG4@ur)Zk0uRLNs znoEyKc)InW$8i&K7nat~>~WvV5bEvpq9yBIL#W<*1C0PA<5+vfDe**^yX^kR{wAsQ zhVW)QZF#Cb?A^f*Bv}+NbFj1Mcp9X)53_8dc)aUOlZh*HZP?cL)7N-vORX%5EgSpZ zy2Dk(3eZrCdt3~tZ(2TFNuEXNv@j8ZEL91Sj9lYQTLru%^KB{Vc{LvfbRKK&bRhSR zXPs$xG7sB|kE$N0w{`k}5A#o*%ary;Jk}Yue8S{?z?0-d(o=hHy~yf7efyPU8vil~ zd|41+=eT~CK<Fkn57P7VaHl+u&$atf@m@RjcXwafl8dIt1LQ$eTHK#DlY__cMgnku z%GXP=O{L)`w8F<Hp8GMeecyL<q7Hc!m8a_GUDfwuS!v2MuvkDRWR(Fhi!_c7kpR}b z&hmMyFO|waZB^o!G>`UZggF`RrXhC`XoUR2(n<$F%0!NknU#mVL{m2BJr4Uhoc7md zD;z=ofw3qGtt(EQ?Z5Q46#}$gmC#WIG+nQ(LpOs;7wsvCYv~>lzCGC?q38~s{#Zm% zu+fiKryC=HlgD60^$k-r@C*{v5x2u_;FruuWacj3MPf=Z#vtOYq}H2$NeGgP9+wwG zVRYcRQnYMk_VUG0CnVaYqQo;FyzconRKJrA+rH;6b4#w;)u2$K$xhBr(<&(#z3mB3 zqGUH38ud`PFkxxGHJ_`LH-u`g?A-$WFbJP*_(BVIx1K2TnjBsC`F+qZVsg>VdD^4a zTZYg1>j|J>yTb;+XQvM@bL?SdN-05u8x9K%mb33y4Qr<s=4T;xMusyT3cnvU=XH4h zv*UQj#$R+U$=~&Y=RWn4?J}Z*UGo8IxfCAd<HKV)8itQ*HlaUk7)ou<vhfPO8#Db_ zjP1o&#F)NsJnfqtvWx9?i%C4;=<A&)ccC#171MF0e5!EK-Pi&;1J<I>U&G3$h8<03 z1sguRUw>VRm?v;q`~9#1M5fHAB(M~7b!k4aIXJ%f#pAem;Belt_Xk<$1q1s5%>_o1 zDSu}4*_vcJ366oc!q@dz_i9kfM%A+Y$Hxb+wBFMwpFhl@5qvY^R2CB%99AyVB@uT{ zsn-XZ-ABEw0-3~(c+DHwvRKREkmL^m%U=6CLTy_cg3a@NyN4w_N28e8NPfmBSH9?w zgQZ3t=tVY^4TSmYfphkf*Z&LrU0M}cV*T{FZAF4VI_7nQ{|Yfc`A9vLU53HmJIDe) zSiuDAK^J+!qd>eoR{~lmp2qb4J&>KO&(NnuAA#rUnaRgd)eFnaHEOxkLIXXbRNE`W zLsXMNzRoQ>vln}wcJ+`;orf=BjqPcdhIz~eTOBX*mYL|EDx{@xIs5MQ);`&Gs{Hx$ z03{mvk;G+hr$VnVj>}<N&;u6RRVJ$j9-^jT=6Op(5OFtw^8CH+__mH+T!v^*19y`M zM&s2Myj|4KE}4XxHavskkD5f@W_;3MKKScjWb78gzr?#cOr)IO(gakqx9M+^vub3b zV;X<+_;g;ME;IVF`mqNf)WHq}cixK8B&x`f;_)RawBi56!(n&{Xm6>Qp>^wTzOkx* z{FpiV6EDRN<6R66jY3g{RC9{`^fhs6(6^55Y<`L|-fqq|e+qJ|H1?I3i;6bzDz5;? zZhV!srKbpvX}1{3$=MDk{hDiN1Vl@2g5H3R;u2@r%4i!0o1=+ld7sy8wr)NE5ew<S zLj5SKCzQ8yJ3E=nzM+$w9uR=vK5dBhKjnA5${$|XzaS5K)U&AA$w3l7ekHuaO=xXV zKEv-i<;WD7cgy`lfZnuBlZf~ec<!x{13!IyoOQ8C=94XC2Pm<C+h;a*_75sQUu#5% zm{L*Y>~Vc9{kkf9V~e>BMELn_#)c3wl9~M2MidU#F304@O5Ge*MzCk=ZR;L3Ec^Ft z-9AcXRH6I7NGGnWtfb_14I3`tw!q*Ecl~&L({f_=i~=dit07}3GvEXuYLWZ2_<?s* z3c4+BTY$A}|8<RgC=q%07UM;Va9(3anYMepZiN;(d0ek|fa*mP(E&sEQ_9($3LO2r z>;jo=D-g#n!F-2C5AQ0+{iG>m>Qus+3>iNuIwL!KBE~Z651#_M*WT?->vOm9xYL-; z<|`$97Df`s+;|zmIL|&2N-2-0KLju$K9KuyM2qzkTVKw+#o#@Vl~M$)?sA8f4r@qW zFFgiE3XADR#Rm6MbdxL0GA-Wj{aV$obil%ve$HW}g>CPr7i+X?O!Le-yAh;&!9_(% zruHmr-18|A`gr9ETWbCuMjugfUoUgZ^c+j_Nx1fF@b3k7kAAs;z<_br&%EK2-aAJq zM=gM-!28u_?fw|P)#*Ua^k-!K_{w5qV~U<NwP$Ux3NkOLMT$~OxE;7T6~b-RYxT}j zY*$jeCx882rGLf1%|UcZJv9h49AjN=D29IsXP^{K)r2fGI_i|2bM8{%V_y8+EM;b~ z@YEcWOjkTp(DxN~9PJiM9nTeAa1Bl4n+$12k?&zUum&MV9Ef3#dWcCu6C0JFV}zVf z-8qyw4fb3{LXgRokkE*kD0)<utl@G*3+5j<zplTwI*|DMEA^8i;X22*<#Iexo`Jts z<;Or7H`Dp(XuEe%thQj~3l96&TXQK{^mdFM=dluz@CQg_u+3pmXiw=&2f-`k<)Oy0 zmT;14PJCq254Y&VxymX40mvnux+;^-Gr?o`oa%ZHO~q$NZJ0DmfwUL5;p`7^25^N+ z&57z<QYNZ@rk?#!kd~60s&occ2(~l5tck;M=dm=3Nd<aDx6dDi0CJd?#+ki^;kmr} zu*K1+q;ymP`?m3gAn47ZtT*X8$)a3?rBp#Jc^_YROCq~5H%(M6zvcn&dKL6XdqaWC zkMTwT$zDk>lbV+|q?$s;+Aw2-?<J*LBrB?lV6R2@3Lo%Zu#7tHY$%s51cw1;j+Zet z;%Qqc(qt`wAyugG0lgVXHxXM`^ESYdPATOnb`Vz>w!l9yL(1gS-~_T>f%_IJaHw}R zHfC^CbLG{+-o?^#qqI6(v|E%?y8aKff!9NWb2-?F$#n<V8PRo)07zT!yKYCCJNx5j zNpCW*`PDP<e)(yjZP;#Jwte1xIVZ?+wp4&q%8m{DVpmXV`@<Y)Y{-<T6DNMN?QbTg zxE~c#D`sN<IsZ~jprt&*Own;{+pS=k^;S*BAEcSM2bO>%g4p8Bbuzv+;(VfEfo5I3 z>DKK?$a<~6!XW5y@avTrqxdVK#E6F=bPX=oa(+N_YlT{EZ8cIJ5Xey(ZuG>}cSt)M zBr7EsA)i!7M|YjgZvTZ4i#S_dPD;jHW_%NK?}xmu7+2)8AhxQYcSax5ke?aps5yiR zM4eTgqSCy@e$O?>Js<KyeD6m3O4ro01$v8P{ygB-i{5@u&!~m~Vb1ybQz5&nz-#+2 z4s&?qp07uMN#uL$XDFLFS?j58(<Cb6Q>GdPG?_(nnZ?yq*-s)+%{EXEpjt-FVsQR2 zjtq$9LKE2L>YDuXFATDX@Mv-%ub-+4-dU89GIeHsz*q80Gr$ZX#2BXXx)#xxP>O~5 zWjKrr(xRoxR%U$|1K43idR8}wUfw->*LZ=thfo>MQGoRta0&ge`N6$SR9U&AD2)-$ zkV(aD8@-cK`%Djwd}X3Q<j||3qBrK}kVJOA7Yq6K6o(upC&*k7PdYkwWjR(rXMLeo zC+i_$V+cWx_(pM=JC2>+^JScT8c#$x?FQ~BNPB|ZyvO(Xu=oc5Zk9GB{^e7XR-@&X zT=~2Sgixo<?h*M^k{^}8<W^(fY+&l}3-eE&X2u_>252G^X9B@xSv_Ypm?}#{N8s6y za++hHLCNa+yybj$0g6xM>d1H5+EY}N^va^oIS9YZRW=1ng5C=mPjX&wiFQ-aQJCO( z*Vn=4J{+bEvSt%AHY#OYp&3Xhme}jXJtW5`7jCX;_YZdKNm+ungt&-p-E&K}z?g4v znN8+}_Ecci_~+warkV5^0rqvoNZo0p+B^fh>vX4W{Q%Wci1b!M_5_Et)usW?0KSne zivt;~KsFQ)aIwnw6f&oG|GkB0$KF-D&Af)z3>f_xbJHS-(ZORI+8ku$q(8Y!IO83Q z6_0mE<KGW-yI7WHnC&pL>0}%yOk3<C#r(iUc||JXj64_Xn2kQ#nw=EpXg;vT-tV6x zzsRy}u^3U%ka(8LtDbv>Xp4Q1-x%NkOBG+t$<C>-l^?lPNaM~T9G{cnB>WSSIsyMj z*(DnOvstfeCF<wU3vTcA9IE+|?Iwyvb1&_lT^V;3qn|50&~}YZ?*50gb2JB^+hig0 zy3QoWb`+n(1(iL?(sipXcYY<oyF+n~TQ7KL`@n~?2PY;6ZX-(U-(#^6L)t*j0Zkjv z^9QXw_gU}s`}TYJi!62Gvf%9bt_)l$zyXQqI2$?ZkiW-3^M|9M`$hJ{V53Q4z270C zGFp9`k@o${=e#>EZ8}ca^`qdWDEK3|er_w&;jk#Wr%ax-8s7*@rWn83W54FZbW_7B zje%cp-r!I3fyZ=0I_5{*m!tU6aW+neo4YKT_5pX7maZhuAB)VVI0P0SO<orik=V6B zAeR^Kpc#Y2ePe1XNzTZV1s*P4gtlD6yT{s%R)Pj1g12FgC4OJhQqP3yZ*=;1#e@8w zjfhtpsV0o{9W|diV~I=Lx|(m*@SATfhu$3u=JGcyh;c-;Gnz-f05zRD6~)TG4TIwy zb~XJJ*yLHcFTfNy2br-j-OR`B&CzJTeR+JthuRVBaFrG{|FVhKHVYz{traXe_x9cT zvf*4DiN%wXN?9j^%e~&9u5FL*7Dq-aGh{-n$P3Xj^rt;1BNoA${c;~cBhGX8!$#Vz z)Aju^?E@CSA8+=_%RJl~Ze5a~;!EA+kF}TBw=HOh+P-+lwg6FR9Kx{Qw0V@1Qh@n* z_B{>aw5YTFH|kv3_OB?=msIo}(|<IFf7q3O`c_W>2K_5e@c-KV=OCIn;Gy>a`%x;S z{bR1AzJKfSce&A!L{CNnyH@P~Q@}4zKoJsXJW0C+UFnT~p(w!8|KAULt%|Q6EjG#n z567K~Vie6QuW7*64=}d0Ip1P5F*BQBR;hPXobAmE3KRQBXv=C6=4t+MAFXVk(*nd^ z%Gb`Kx;BFUeMa&pl;v+J?tlHgVe$+sI9fHdryL=<WH=Mk9g)P)jx(Ju-M4^g*3?Z@ z<Q%Ny;7bX>z>Os;+fh9KN}?UehigxzS_y|i-z_ltUzHX`Y#mu%5&d)ce_o=D-jcwO zj9}FaYRlA$R7ltYMWqE1MU#{7@3h?y=Tx(Ecilv?ySx70vQzp#nB`iMAx3D{xo@P; zi46<9bPEmm-&<c*K$z<}C<7Uuk@~4629Hv)ptViNBkAO8{6HHs`+rsMf9;wOntae5 z`R8K)hZjn_E`Q<wDg6K1i^3q4Kue7OzioC;0It>l;iUgm4Z$DVi~9e%y%PfJA=YH7 z*g#DCQ$O>c8cVxI8O@??nkT-z)A^?`QK1abhKhv)+JE0~oiR8}jBgZlkwSDDJ?<-@ z2s-Ho)=0!-A{-h2oThJpX@)ly{MuOSCde^uR{Y;mVE+CSx{|amYds(Mx8T`VteSzX z2<G`l5&zz7fbj_6;ijVQD0#F{0F!oao|xuzE8-wUKXQzb__r8w+62V?DZGx!WDc_C zc|z;`IKk{6CM=fDW+fQ;p#Pjbn<4;8{N1VAd5><G0lNBW`+y0GgzA?1Z*?VK)%fiZ zZh&fExh>V1Rl7v;Mc@_JJAg~M>)jg_i20mO;{SOx?DQ8}Av!4^XA~4|jGeny4jUTU zAe?_6+!^^AHCcCPL*w;Ef%$;xukHfSx>77OyQ*_$3Jx(}ayY+abvn$z-l9Na?Ftu) zET{TQj|jM7Lfe+ef;z?>8qVvFY&ieb;;yoWN9-j4c(CLmr(o1yBtoQ4y<d*UsdFkV za%zgTd8w5D>7m>tX<41NLvziYB1QkQB>Zc#82g{WuJioA#Lxe;yY2S}vH0uUP|W{y z4*%87rIP&T5`2Qee@yQFeEbF5e|DWb{<aJU{(W-ykI^F_vh4lO?&{6|w}qIC#@nz= z*2Sf#&so`^<IYf_75`18^2uKO-M6N%`56@z$`6pMCElh$dRf@Re$vgl$MS;f_Hasl zRD-4K9JUbxb$l|FkbkmQFaA?<AaEy!L`F%5#l1~xGtBMPhfm1zwQlhcX`X+7zSUl< zlFR?AwTuuHeyNISo?^TA@o+BhN^>9VdN{AM7y%G_c25bR$nLAI)1lSdxH=wmf)Cev zCz`=VLi3~dhAr1?vi0`+4krV?-|)I+#b`s?1sh;sd%YdNTh3!jHE;85lip_g2z+yX zf6Htz_C~m&Xu5P@lU{xC*W;N<vwBGD-KHSI{mR^j$q|X+mD!CX%Lh_iW^J2C51c!j zQcq&QJ-t?|)mpTyM8^rL2U~3Re6+;#JuSp~ug7w}1lg_Z&K^<86A+|OPX>4b$EG){ zR3VCYtC-7sc<sL|S);f8JcJ<i8yp<0LvhN2$fC=4@O{hbV*sG}2<YF634GUK@GdW8 z%#EoMO&j<iKCBqazcE!1B5bkLL<O;F#nvq)Shq`DpS#>I<gQHRZDqewEXFALis$?7 z?(#$=M7SkcnDYZ8xNst${xXRpu6syRRxV5+lE*so0Fe2>PTA0Vi3Zexw&b?Om3sUs zx^Wgm-)_n3LG>cGY8$|p+fO5-lMC;ApW(C;&jIP}dN!gt5<q%t|2AoUuyFSP1z>ge zqaCF%hk(O`EF`HW_5lG&7GAAEu&|c#XyB<Q;J#Ii7D~e84#yk=v7};pf+RzUqB2k@ z^rw<<+L;NVy)c042(nmupTgr9$EaC{*qU7Ac9@!DIFvx2)kE1os@LPGgMq)Gb4AQW z+oPBoX}QRr4E&@;BEa>J?vobzTQ#8qDm7~SCU}!7iFMzZtr7fslbumNM4+s180c-6 zF0y?e<l<Bti~`I9nm<}ssbj<2eM6F{x-Y%nm742xOTW}uy=FNB6ar-6hTk#?CsK4O zg_(ATqq1tnT2!1Z*CMx!xnJQEyDDYMss+pW39bspRN%$iC>82{fN(bP+Gwp)ZaT~t zgOCo0W;yRHFl+sy1vkPXf!HzNXBqf=53Q5sA<!wKn^dB^4=Na!dXLmE=@U5c>l8=7 z*7Zp!2}o<nW#8OnwlL&iu4X$z!x>8MlzNq#*NidSyiMbX@h54)Qv@Wb&!6Lyxott$ zjI$oYn5mntz-Nq{@1>7Nr@bEIeQ4aLygYtf=YOh;xY$`#G~yZ=R$SA#x~+!&KO|jc zSd{PAR6r0(L8L@by1Tm@q&t_~rMnT4l#ZplVd-uWT)MlZySv}#_kX|bm%VoP+WWcB zIWu!+1dd!i?(c+sg_V5;Z?Slh#@!!pQTK$6v(bt86rzV&(7Qc&O>Eu<vG#d1n=fKo zPr>RxFzcyM6)HmKop)rHfL#RTN`DGgz=uDjcISui^;9+Lf8VbyozVwb&;wYTv&O-r z<OENBL)=`u5%4@5jxu`ii?+O5nq1j<Iq35D{O}`l(fu$%({@4rPw03Tv*qAq=gr>i znssIO<jO@I@Z!XCU~ph^<&XR0S+swg>|X6wq(GJ|e1E#jz8_jdx)^U1a=7ArFr&rh zv#Gs@O+pSVv%K@WE7?h37OD*-M5H9&mvym8Pw#a2a&a<xz4vPL++g(lb2m$bkvMQt zaEgKdmq`OMo20rH%jP`uDMbhbbva%zc<x+nqX(3KQ>6l3H`V`MFoTn_9;4FNd{2M{ z^8(Ng0U?)D#8R8<L(az9%ge&abNxw(C2y}KxQMz<dUUI#66NK}B=fw+@45BlnZ_FM z+^44eWB#|4bCd(^LbmsfvNuWQbJQWQ7#x-An)nqWH_V4Y!D{_OA%#nvNRl&2AnSw6 zi+|w(e&go;blF6?9yjf`s_Y1P1yyohR$86#1UhPrW$zmjp5jDl;D}LT<mwJU%NI%c z+;Kv}@#v{o?MN+0(pf=cEa)4Rq4;*_nL@r&>mku`!HRmjFjfTpgkvCZA#=)lZVJOC z-t~9)rSZ4LL*$Z@Ny~xM`h(4T3^ro~#E%c;4)BEAM8Yz&g-gO3@6nZ^c}w($gh|uj z)s`z_!2Nr^#`eKYP^lyS>DwQdZus!=Rb-9z#m5wO_YQr>lbzP+&-`Bi=}c14FKFL_ zD{;Oh1KijZIIPC#aRx4@V<st`#<2=nyz;dX?=8EmG)L<XYCJDAZ=U$g9<DQLJo;62 zI!p_c?p_9!{l{<gJA5>_Y!|-jHQLUC2DIySh1_lbLX7<G3SdrRK^u#^E6&~Z;xhqh zs;}{vI<OlF%Jr6n`d*#?^QquBX$24KQhzKQ1kXYfvQBG$%;ERSKvnJ&N{-V%?)y(l zEsXd-*QN`&4%GiO#rteVqXPJgY5w-B;5n=I>y}F4^tL0@>nF)WdHC1z-@@lMvpxmd zlHWF?K7(RvBXs2nxo^4EofL<dul8~RPv@j!5b($~FM~7^vRjrJe@R};;D6Z4`F9%@ z&u0HD=6-y!^demF^yA6oKi|4BgU+i|1L=oq;ejt^cs6xde?t9w?rNDwRv$iJyFQdv zn#~>y-jg`*Fpp(JW?*zH@_X;^7l!7Y&8|{ioer*<O$EoW?E1DD7wbkH6bMGzhMw(L z+V<}IJ|1=3)UiHA$5H7-2hNxAFFP_g9QB>mWQhiDQ2Nb<7KMYlffMq#%LDrzD5m<m zV>KwrZlXlq9Yf!90}<=VzWu3TD4A7!^_LLYf$!BqeVmYMYSg4zvL#99Z3iHi*`4bl z=x|^4yJ<G6*+k9E;Ja#$l8wmYcRL;!%H2s<N*CCJH<Z`>jkJd>%>gVQn>#}Qjur~m zXQz9+j{SH~CB3N<J!WI{XMK_%7fa3FjS6Y3YL;^UWwz8N2d=dzi0dA?3<28s45fwq z6U(8e^})216W*&k>uF^x+Gq20i7bb<1eUE&1bH`y%hc`${ASaitq?g@5N0tWt^z=J z1>tX6u!h6@($*K^e0?5{+*2wil|GU02-Et8gDbvkzVksj5p=l?1;6DufGehH5S#mT zwrcLKqZi$<cv(-jx}TVLU|Mn4QRsr_&lA0d`J~(v0Vi_7OQG8lgIRK=3v3Aix5p-! z%Mt6eFSA#us+o=}_JYSwh2TOVk~FXKJ?6<5_KcII2HTKH^$>MwRSU+0oq!JC>qk$W z7M~z+iTSb7y=dW#%S&AUi07H$pgw=@LG6zfAUs|>W$k|b(@i@+X{y115R+j`{w&n* z@4+s6G+^q|d=UQys9ahGLZkJZzGO`cC<WCsJWmq*lT2Z_6ZtOM`PY@kG?jln^A4_? zR^jg&g%|$)oXo6?4FOBK`?5gxMt^-|9>2I6$=V^UxT9*!S+)v9!)Lbe`+WPjTOCG} zyiyJD5s_}&xA@FDRL(n$W2;6p&cUP^hSGWG?R6wiHiI9FXqQ)i4CXl<6)9v?+=Ss! z$`8&tZd|z!_?lKsc0MY#L4IwoRGGvJwPydKm6Rer<KLJ^Ve|q5OE|Rzd5kI^yl$HN zD@ezljapW%X1t?J2&9xy3t0Pj7K*Lu_Xu#yUKGQmI3$<v6_<RQ()M4~y}oH0UG~Ld z?B)rFzg04`Y?H!w$dWp7$QmN5TuTlmVbpC?=U&IZdHF;$?(y=BNRlQSvwCn{?RLaR z(#`XEM7lR|kNwKK+24J4>Te|8#@6=HqL;lS#0+Mb2g2Q0*irh0M4HtLG#9R)EYh_P zI6gRH!6&?L{KomyxvN>>c&0d+pG~rQipD-$5!Qiba1tJWjZ0~u!QZa9?_Y$JX^%nE zTj66CLla#6Ua*L}kfl{C>5-H<{4+_c^`HF9JfT2rUXD(!U*@yp;T)<?omt6Zua6*# zi@D~o)73pjWQZdsBwZ>sCXVX;H28pr)CL`=XgxKXdwv#Y7KH#ph}CTO*(ZuM{s7*b z$6X7471uoeFOAE7`b4x><=x6;=q@VV8<=jpW+leM1}EI1BMl=`M4`@Sx&=q4v*gg> zDah~hSd&)1?orD`n=Ah`9ezf~Z-R)>P=^IJ>_)GDD~$tJ@w2NQ;u8}SOZ5VbKpW?p zWnJkw0ozeJth5?8o#3RBa@Emnw*lJ8wgncdHV075&^FbuUVp)0(p5IW5c?9Db_H0F zP{0Q!<O57<+Fn;-@!<M>`83RN)p`qqy}5!`8+Wnq|K1gjp5v9}>kKxi*4l|%Pr&MZ zH0czI{(x@Z-|B84f2*;hmzqh~E?;Q&or(u6vLXLojaS13mu5;0q|dh>)QLFk3JWM$ zqM()fw50y$VL!D>zhVIWYzszizWUw3#2BhH`WK|JP1;G|S@@YLJ;+6JdC@H$IqiB; z^M|a{{6Mp@QcktbgZ8FldxEAkh`-WHHP4V7@aoPEFNjp~NvvEEc?X4&)gN#}0srs; zOlkxyK9jKqam$k9g`Db#=Tkb)e2@$Ge4Tk(`1_H-kdsibYrb1MT(#0f&0VM<-#hx5 zZssle7u7}A!`ug}>4rS@Qe8D<j7axf2C{AJ+>92N_<?EgUZ*QD2Xw`E<I*I{jq2@0 zp#gP?N^5}OFS=<+S<cPYaGKW<c9|`|t7}$e`D~t2Ak_jBlH#s|`%=9O0jty-EzZX1 z;%(b}r)k}|N(a<EV(#7dAney2E}p>&G<vds=(J`p;4)0=xj!!V`M+T{Lw@e&WyRp( z>2!x>YuXm%KxT2azEW`7hQBL!-WioY5%S#}Zd`F(HIT18;L)s%M%dnYO)cczd{Mp` zMcV>#KT<wgz8mvvyO}DPoN)j5jTTQ+GPT*S=SJPlR>T+z>8-}@gN(NXUe{9%>w^7{ zK23^Crnvj&={6*tCziIN#6TCMu824%A%V#oe9}|4Un#rEudePRQ|0>46Z2KCymYxU z-6*~%u=wPwR~i_Br>w~Ucsh3fc$KY!6cboE%RD^<{df9|3Eagj(T~;Ds+<*+fj+9@ zIb#$T=5e!pV|IVJN2EaZZ+s<kaXS6k`rsL(QeROXE604yjN&gvk&dRY@2gkeU&%^{ zsw+8dc#)JWvdHc8GhUo@dTLD+jg0+6vmn=jE9%va51GM>RPpHg$8JrC9;G+1+e&oH z0zNPrs1Wf*;-0t=Y@D<O{h~06WqYmo4=%~^JME;c2u@b`DFBwjZYeunvsUUnhLymV z%IV&bDX+<+`9m!T&7bP8T*CMb>&JN3WtLJ6vhjiX!GQ>LTJ5Ts56n7J^3<$cru#8( zo>g*iO5<-cC+q}B*Q4U;J|3pIdKC?)li!6y#}&YG%lHf`hHyn0jdD?9X5AVEaI164 zYEwy_^{T88qpGe{l;Tu%ieo03TA3S(29}epTtXQI?&Jig>|4du-D^ob30~i+1>cSu z$VI<eVHqZf{H}CwlqmcrTaso&VjbUi2gvcBV799Q^Hyl!&LZQHLu6v(w#=vI&amq7 zV_?vPII04Ab!bauZhEHe+NGuZU6nB^A-|BMB5&Paz`rQ}<)6uU$AdT8rA@qrSNV+n zQoiro$hjt2`n%nYQH1**15iW>Oq8o`B0OUH_~OWbn0w*^E4r9Td|ld#wEMDWOwv-z zov%s~vNEg9Dbz*ng1z7eY;fP}M)F}SJ923GP71aB`i#;^vOvH%vcu!QP+UIQ$e{N7 z%x1Yk@qFh+o$jZWrh)e45w+2HjcO_QNg+Pb_;au9qKfIS3r1BweIW*YDm@VPh9DQ0 z(#lA@ay|Yiu2o$$TH;j4z`ekw;E!_0z~E9U+esl^aI@P#U7X9u$s2!Y(61z!Eut$5 z09d_Rx)OhJV$iw!OK=FxaW}7BAb;tm>jimsqL=MUZFe$trJyHUcaZHv&-2kre^{r{ ziZp42Ln?Hl{c;uNv*kA%7);f7aIzC0EyP%QgZ(2>AUdI3+W;5+PO%D;W)vTfe%fJY z)d(>#o=Oqw|8TP^v(=}2e~7zRcWhbk$K3Ru;|SN#b5ZD+Fem&#{D{v@(_n~`GW(tl zy2XyZG5rS%RHfEv$fs0Ny>&h4+mSJ;02mx;)>;CwJ;#|V+MJj-wPJq<Wm||l$7Owx zC#&q2uxZt*!4`+b+1{fww)~euBSd4qO9zQd?eS-McHM=lZ8qkLewK&iCgcsPB9W33 zxwA{{&fAC0sa3J5Tsl{1AZDT*eUE(g;t#YSU_tPs^U*y9BdwB6f!dCpYMz^TyKwwm zW(IH5n04)WB19%=U|Iv8B~ufoDh^Vn2p+T8$S~|T5>Ob>EG;}q;V@5)nnwG`H15q; zP3WTS2|yp0C8;7mGmYDnGm@DK2BfXh5d^<gNxadi@+jHQ1^{c>g?w7ob|o^m*`f9# zt38&KwM+cexQ5L2OKSg}iyqB)ALz^OWoI>AcKH76wb0z?nCd3@W0a}NLz@ozZhvT| zZAPJ#cWciTTcfqxscU{UM1}SDy0c4hIq=0cebUWqnzXTsFa}|(mCKKIoSOv15uxST zPgSf+$R(81Tx8fi4ER!K&9|QXX0DbmIvWF*FSD{yIg#MRQ_Jn0ma4WXNFI>_P{e>v z(cg9gf(Nqe*=!SX>q)?S4efr!ugGLio)(wdTGo#P;mJ~Le{*P)!;3e|@*`Qx%RnN; zX>Y<jENqWT<10Dm=dHiyVS?>Xq9%lZz7^gg6+v|40jf$S3<RJp|H%I;Gech28M#Z< zQipE{raGX;i<NUtlS|}NROi9|CMTc86kAW+A7@U_L7c%blWJ#N^hmQMfO=UaWl-jp zu|(Ek2r-W)Ov-m%yE$5A271lJ-I0s}ugFw3`>wCP|1u7LB6DsQDWw(sp(F>@tCxQI zvPJ*1f4yMvDkl>?k7?OUE{{qjRrGJ*U5~6<ew=r5v_o(2E2URq|K6I0Y?yvGX!t75 z?vPBI#HlSd7J+iS{#U*!pt;mSs#;k{+9CWE5!POEf@FA7oUSYJ?wRSr`zq5he`=$! z-@M1m%(&FbiKVV-GJq!uN9`l<);JcQ1P(_8+ev77%SrV{T!s}9{g4R1*JW}88U#Bw zqvx)G-?5iBnlvlFa5kZYowCM@gz!)T*EGzA!$Quqs%H1JZ+wG;K=d@`Rj=Hv9TGx+ zQ-{Y{4SE<U@KdhcdfT;)s0l*<nLLR^NU}Y2mrciiJmM?s?}Ih_1>$TrdY<#e3Y-F^ zw@cW6O%-n$$^Nav;X<v}VySvc4Vw_HtfKkMo89rUO6bII1`3bqS}kE(qZQO1$};t_ zZ_od_qlKV#{c4-VxWaGYFr7iO^0RLuudGA){d*PxREiT3<1A=S$r+*6poocXmnKMx zO1q4UgmNY96v+Lv*3>`XI?A^w=S#I7jS&8!C%9aFw$OZmPgyD+F%=0e7Zv$Ijzoa^ z^O}@JB8bhm*eXm_wa5^0*l@AR`d3dhLHRaKEyw#{!iwmGNLYD+2QM6GdrC$1inR&Z zkukbi44uRv=*JEq8T>HtxF&nqxN0&H68%1>%~t@viNUo)k8tJ#63bo)rIphk^#k<? zB;}!%V=xc47}?M;c&G!2rWStB=Nn+x<Z%{P@w7A)xg4qIHa4%+W_erec;33|Tde!a zIXWEOb=kcp+EwW!hBErvgIf9X-iJwpGB%bC((;&>$sb^J90hTZam_Sr_FI|wp(9a< zj!MK9fw@jszDUxh=Z#>AT4M*e;YjI(f~vskD`qhIQH{;Kljlx`zywTEJ_8l5ARR@> zUZ=Snoa#ApSj-xRJN3=Xr+`lqey^yS?W19fhBIarlG7~>CJXjukpgW3ahBhVgpXbJ zb{#hgjB{9ks$RXHvqml@)2~|Ks+a_#R@&!@?)axK8%{o_Mp=`(kH~@65}RU=M2*@q z<=Bh6;er_a=}3syH<d_!_x&aNeV7|nbJPI#kO%qNUSp>>zaA(?M;8QFjQR6-(26D2 z`RlKABzg{_*YYdz+b-tIQp0A4Kh-gCSk=D&-bt<w<8yW>Yvob_K-&U3PP2ek<Oj@8 zURH(aezU`A6GisIw1&C)&?YemPP(boD>zXBsMTeU3WJz)8sa`@J#0S6zI4Yq^PQ?P zOUPH3h|7*%QO{*61#2iG?}=a1eL5vMp$+!w{hFaYC0Y{0^o3{^=-BmWMbF;|5T?g* zuXuD@uW$1?$NaW(X*4|Vw=}=>;XMw+oxI{^aT@ZN9#U_%T4+m5otY;LMEc6_cDQ%% z{CYY^-Bx7hOhI*05ao0=*%TGEn<7^(KQ(A5SmZ%^rnr3mmIA}HD*lsZy{7mPewM$V zRQyfNI$@xJsaKO_zQH|=%}S<?>X0=r(2~&4UtQY-N@FXp$L%&sk<oIkev!DUTq*Qa z;NyX}$8CH{j`|BpnRi<m&5!Deh+|?SKiX8}CiJK%!0<<8ay5`@sEU~6I%AEJN&NhL z@?!X~$a*vMlThJ1fuQ`uxt^TssQsOs=EP6s%sU{aCjYfn9Ce_j9T9+?>x$W;m#)o} zwx{#DE6Gza>9?u-y=eMPi~Z-#l;T|7{;=2Jx~LS1=YU1g+}nC`Ma-s3r&)Fa4^>GW z5`_hFt|p6)>e^3Q3_%-?@;C!BZr`w``30^Q7?(IGVt;Wgs3k6w`6TUa<NQGOxxu&B z&hsX3oehwvMUrM?$p}<v)<*4ftbgd~`C2xs>07fJIok6!l=AH6a3(F2-957{zOvZm zc)ZuTL9z6g*VP_%&^!F`YZ3bhqWE3-k5V=Yll)BKjgwr-Uj^!WPN`Kr4Cf@RZnC5| z8I*F+IN(@=<A%`PGb!bQWnsjD*R5ya`^qqS#lx_az}?R{A;?XR7#ZC|JPQ0n{Bak` z=>n9t3k~sorPJoslg<?O>~3)ca96>-I<3gm`laM$RS+n`6Nb{flEL1qh+g)S*#Dj< zqdU2uK-tNQq$tw;WWExzA4QzSu%=Pk&r?L|nc-}=&v7jMW>7|oiye8~o{`@rG5TE+ z^ygZhYlNg$T(Z=<psi9H1Cw2-dby?+y?u6oR=FXy`Jd8aXx{zo+u6c`0j&uN=MLQR zp+h-lyS~nyWzHgB8J`XB&`Q;y^^=!1(&_wdK5XY-;Bj4O#oOc@owfOZTbLGOzFy-o z&gaieD43tdZYdMTRRR3oJ|j~$i&uFhQ?OF^M?I&1IcuN0^YD15%_F8qNA><Ipu#c> zGcOUCVMiRXi#$Ko-xR2qye-kLj!R%LkP2@Na7w`CX^UdnJ@{-vUOMZFbxm(J3Wyjd zVo~vqdawh9D2e)pl;kpRovjfEf(>WBBV%R04YjZvJ6x$v3a642DVeicl|?b@2>gz2 zVW2))$d_$`L*kHCsaabQkrk8sAo`8OEw{k8hzUoNSGD%=PUwe%=Pzi|&sdNKi%IC& zHxim6TpM&dryX5gK9|GdU{bL}^b3<bxXfAjG@^OO49r!mFdvuAcly#)`0d}<fx&Vc zACKyNd8Lt0Gkw?df8m-FQOq?OnYP@hcPG3;+>w=)|AB+#;-NfiCpgtZ@k&PLi=4J^ z9CzhqM5ODF!zv5^FA{nU$Jxk~VsS&<70g|qXPCdXHhbHUGP;My<dHNTBS{y_uap|p z;YP^2yd>Bb>*hr@5N=bZ=v@&v#)QScm*VD}s{)2t+HbIQ++Wd9oGpQ!A)~Ja-KF4W zCW%i^;38c6w`&1`A^Qw1wCvk51z=Z-euFZ=$Fon9y6&O*sr|;#P*y1=pUUv&c)6hN zz)0VFz!B*e(SYWc@cw7;609C7y77oDo}8*5n0YMv{qV6)b`Q)`xmqPUd%0#G>M}qf z$q89osBLWEW&cdOCHE@ideQ%1)z4`gvSs~FOMkD(3WNCkDcQJ64@ry$hsWgJcCRa_ zbH(tn?^C1@Pt2Kg%D06Gozme2w}dUivgP_a&CbBzx_MN?=5wbi7U;k&($@2Hg22oM z_{@^FYtj>$W+xNXYE?>F)w*K!C|C|<1j`4OjRluoSncW<ZGu>Co$2$aoD~<(#wW@W z8a<hL@qtSAXEx;bx9b5}XOZErKyj*5u#Kj-9&Ok64{_;yDy!_yGd5wz6N-hy$pyMW zFuP0m>f`XL)t&>6(-?Jvu$XfR5HF{NQ_|Avc#$FCX?}mHqa=JdZAK)={07BB0Cd`p zg=VcOv?V%EO_|Tqo2>G8uGQww$@6B}eL0y&jD2*x04bVfj>U=x#bv%rT)IGRI(4FP z_0=@HS$F64wKvgsQGADtx4&5|Kr=(Bjo;|_+F3nh+K`pU3U=dHpRW}OdBHIKHW%5! z5Y1wOoI>;h?(sY;vV{mb<RKgewVb#JhZAc0PbA@XoBFI$^i7c2c6U;pR=W(M)&vun zN^2%ik!3is3qOQGD5%G!-~Iu}13kF}Q@HWE_|co)aurZqzKu4XtgPbX_1eAZ-1{+8 zC?T?WbO&OA_T5`X;b8du#-$7(<kQg0MWOY^n<Uq~B6^@8NZ@>pdE9a!#O2tQz(iWX zK1P!i#zd73ua0jv9IiLYe=uC|n5|d_uXCBYnO^agHqr_%4DBuFZ_1{!NVl>sM&6jV z*#hBqE1o+f?ij(vcI!7{Y{XOxdiFcxZcBBZwQXnHX>5Ia5`Kt2n0Zlz)wRo=1oOb@ z)ns>kh%zYcW$6pSAZ4uVJOg@o$c>+BjSZeH|2?v83Ky@z%D_bhbhS6#ew)K(EpF?I zTk1u-(XL%Wv$9}Enje=zg8Mp>_!CX}-o1+hzXegCS4Ja*wCgsL!!z8_nJ#Fo;;|yn zfd9f<Cq-1Js_h=ubyxL%u5k3m>4|@i0BNDMxYel6c2-F~%<+@JA`4{0|MBT%hK%g> zIpF#>Q>|7<rG&FoxU>MV_b=}KjYxS@vzZXa*9rot+R}OyblI_2bf~Wd@~V8qI)Erf z+ujP@g&BZU@O)dqDuW<vHhh2n;4TDpaPLGzLyLdR`WQnuz26^<k;W+`ajvEY72aDk z!Zmk0`f5$72zYCBPR5Nbg%AS{_pz>N#r=i7|Duj|&o+qgv%#;n34$?EHA^o#o@Sn2 z%Fl&gSff+bmL3+}kC`%Buf}RJc8MdCnk=7R0QwG^p+xG~MqwLcwvy}0&qx^__0==X zn`MJ27#A7m#-<b*2}1jufWat8BFp@q<j{TOG3dOjD2TFuc{F7$igj+r`lEu>inw<n z-|)@K1F03d(p5Gw8m)H<x4jUdfJZ32DHuI6Nm;1B5=0B4+zFU5Yjs%5jgVE)#C7N? zX4S<^0Fem&iR`rRID*fLu%RC57l1K<<gs+Bw*b2#BK+%GB&)$;N3I6Fe}=-=go$k> z??`6op2nXwgQm)J@Z<DX3erTh%CGDeqvHo*DZOyZJ@O$p1%gDHtPix*gk`nd$owvQ zdmP3F)A8y2UY-t*EVSxu`cMX=f|iK+$RRNa`6IU-NriHUGpEw(urdtig=jeh5N^D> zmWTYcS<u0tg36E<a4jgozG#;!cnT`Pk4Kb9V-D+0r}w2JL{G2#9$K$<=}&z1=d3S} zQ$Kv1uL|<fER5rTEb3s6*(s&P*(BhCuzmkZ1YKu%%RKBJvsvw)ioA@Nx&c=m0W<Y~ z1=9Es2#Maj%Z}l&T^PYQF3%Uq`J-CvW`HM*l+tHLsaNPUotiq`O*VSSjw~yIJuiw> zQaWp!&$-Y(WZn_|5<T~|H8f7fOzmt{_~{a#F-_uk;kPv>rMV+@5K9f1(|WeUce|T| zP-N>5-5|j3vtiI9o1pXee0@!UfPm!4ceBeZ=yj7+A?VeLvSyOXD^EsuV7yBhn5<S^ z-G2Ex!&jB*3!`u9<MptQE`oJEu4ZxwDk|!IdnhfsY|nf4^)?Z7o$0U*YMIKFC-sQ` z9`M1RUnbgK^N(Q$)F{Ag=Ss_Q>88P@)spjQGi2cTmdJcQ6?rZq_KiM*q8_!%4#a%f zMJ6Sr(eaJ-9Makk@5QS^E{?`HH3NOQ?Uu$CF1O02;=ry%d-Y%XcvcM32;9q)|GFJ7 zy)igV*1d&Mce6S&4V|L@D$dyo{>+K|!ce=-7<c}@zRm#q%0<7&6wRFp5rvv3r>Qq= z=7Z-;yeas@gfvy_^`}%8ecgID(|%N61Uf;l9yt#WO7|msvtwcU0Gx~@v;9jQ-O>j4 zmw@9(ul<hX6uGfjHjM-aN=RB9DXsy3+!tv*m6g|%EsCvC$B(@ww<r~Ou1mD?Kawle z^8(Nbg>)Ag97sCH3862SY20-gtjMgv8ymPBw|{nu*;kqEVlLg+p*D64)i2+>!20KU z>x~VTqlVP?ru99=)+_+|o|N>BI55H8>m=}TqEtf<&^RPM3cnorTZhrX8Q}yRVztmC z4z*My68H-$EpGkeMBAM@6;A;XeW5pu>dtIgc!T3Li9+<}YG1(Akus%{b!3b2G=(^@ z_xTe?THaPKs!Wb2kFY{p=hI^6^eg-Te7J5@nc)*M7@bWm(|D@Ec9D{<H1YBnmzq^6 z5L;Ws=uJ}R00;6#Qe5nYs=6~9rt04S$7-3d8ls5Ph_21e54Q3P0fb-*`f5#92NId= zQf*eT7Pq59_N=%#0Klv4%n6{27U6#^_5#BoMFOPTTD1{F?$ZUhs{R#B)gqR5Btl+8 z)&;0w0oK+kjUu}m<1-wLOrL`&+k)25$yB>z$H)4M4F@pxg|W=RBulHk;Ta`)<GNE4 z3LjQ+6R`mb3<*|p=hHDHgQDQ;&HK~4mgG!O4L5Q~v|K_J9q011*8VDwd(Gk<)3v<p zLKJKJX8c$4=@?VruNErTd)#9NtB%L5O5p5T`<K_p9&t4oVgae;O0`bUR<ifvdGK<x zu0MZE4^UM#Ve0a7NhOCacOFN-hLAUXm-S!1iblOqc}SXO&Aw>JH~TtwB&_>9MGCN) zh=uQaRbc7dBO;^6bw0Fqs@X=pe%?;Amor`W;B2{Ntezn+(BWnNQJDZQ>Y+q+`P~tx zR@mgb;|*h-%e#5Sc%(%3o~Od~V0N(c`Ji(kfmY3xq=QdUSFXc!Yx;Yk|2L-cG;g8Z z%?jfC#8*zvMasR;>3vBgI+3DKbYo33pfxJxJ*XEf^oTeBB?nRz4X4!p&Ug_%s6S`B zN>N$Jmn7cXyVrGtlWi>ll8C}lVJm}w?tsSwljliywE6)+s4CE@!(i;v;{`Uf)@qwi z2$Y;}4@6a8YjCS(aBT~Ty)b>Ni_qL3J4ok~O|9RfAzx<yVJQY&Zkq|f@tr>4i1RrX zTI}#eDOt_VdEDrHOw)X7-gH4YBf^2~PeJn8pF^Fif|)=h?6bum|2BPVv?eZDZOiy1 z;6lM(*p=F}`vE7113qT|JypAHWEGG~(7v1qx0Qn!YMX&3uyjj6Ih`lxL{y$pr$P1k zp|g#8XT+$lt<%yB7N?Za3Kkn1PUF^4X1xHCw6rrg=V%Lyz#rp^tpkm>hf<2Rlp1YT zW37?O3&CPx=WKr>je5WYy{6tbt~<EJ*uQxr2>F$S?r(9YU015H$>|>FEhhl|g5tm9 z+0o?}w2xDpheK`-s{zm!zbCY!GpSuR*18}tZ6k)I6yQI=Cyv=Jp*nP`0-BOJ8ZXOo zIhxmk@?ivGicMC-S4D~nk`oCI>Of2XkaP~BvJS$S7`5AAE{-iw)?~-GpqQV2GmoIh z{3$m;LzH5He}GlkQoTPJx5hWf%nzz^zA;oS6IGk9w=~*rj!(X7<-Tb3-zgihnP{tr znovLg^?dK1SA&w+KGJ&7J0v+!nJd}u+7P(rp-rAbXQ)1JY+;>1Jh-te#A0R7Q{JEd z+Tz1VI=d0Z04!UpxK2v!b1mXD!DDgNzBH9V-SH=SUu23tF86X5gzuAKdA*d@c<eJm zBc*2mS9Yrt`!&@}xs3$^!_)>FkDZPaVSy%4EZE|?YbtoH|D&PO+dM#H7!~3nkz*4a z1fP;sVXi9GPCpftuwWR{RCJKnniaaz?8!7(B7_QZ1YbVV2qy~pHcZ)0h#{h)@1{y_ zB9{lD;g@FTs;l8zI9~1Ny-s_;@^e`A)m*6i1*MC{SjzThnIy0|`5b{N_JspAxrzyy z9l6q|vmziEpl1(KolO$UN?AkKx;u6KexnX_q2|?nuuN+>y2{}x%kJX?2!JB1jQ9OQ z%myvFV#V5L;rHJaS^lsi6ArNk6U;qaN5@mC47<dbcMfq-AOh%dLWzh(Y)RH`RMOJ~ z<`7Sh{#KJGR%I>IJ98c0j{^_burA{QTkJPxL26L&`BaK(xe9#=fr>_??$3SmM&Tb= zgBiP@GLji$U5VvA3f?llk?$V_U!}^BQ|r>GNLkF&xtDJ<Yr|F*U$NRwsFy4@1?BUO zOO6F%CRcm4Z~Fg<=xP=oC6Lhm^2p0=!9N94Q;rm=nCuCg%s1N`pp)1oKS-y^cfl{{ zd+SmqWqNZ=5U4Q~Gl;o>#-21`YWAZ@IgK_lwJ~wRoZd=SnQhwC>vE<(vDiM^gBn|! zi%SQDRBJ^&pn2tla1VgSbbhEm%XMoFD;YU=y31yhv<Es{{JIa!ZqKp<XqBXVfV+dT zokuI8%loDS8Wv9_KMCwr2E00xANz$8ynozi_W$UpotQPcs$2r6KP|P({uC|m!KLyR zQDnY;P<YLNHF2CQL1nIPUA){hF;9Cfm$MIm*K|;g#9T_yN?F|GRbq$kq7Mj~jKqHH zhE>eiw)Q3Wgz;gb)b-PP9+gK=nVHQHQPw?NM#kgOv#Vb?U=CrL!T;s{{pBhs3TJ>T z$yq;}qWe`oh|y;gnm%AAgvZuX+I?%Q=#5WQ0G-B*P1yhYCdh@(X|Yw!<@7gnf5Czv z&~wFCOI1}>gWKdjo3teL`<mE;b7nH+y+#@2PJSqS3IL+(-B*&f=n=LHi)hEC)H7t1 zv*;-06O;TL5tpB5_=l?{Io}tsv9|L_qZ7E`1t0#LlEu-?>qw}?u6QtRvv`zN!w${= zq@nGhQDT?}1!G%iRbMOgG6|aZPvDhW1J6X>toba^mMtM7_saK<EUTk9S0I^NjAqhP z$|ZWjSb3am;wnM}g?Ke88Hrc_dWlV1jLcjgj9Y&Hz^ErXXpplvV>@@);a&nwrs(Ur z>LU1*m5@$|jTYPyFdwc&-jt9EWzr~}a(!Y|pqI_IB+n?cZ0EB$wskmagkPZyWQ;R= z`2Bm~4hPfs)9#geHeurH3nvSB2OxLvs@Zow<)BBBR+u;6j(~cj<!?S$pU?20@Yz0# zSfa3iSZHZwarfF%N{0%66HpX~zE_gfRNz)Hx<3Wre_%Kh@`14uOy*T4(BbFU{&5_S zD#4`&e6jtq;Vgyj>VV#%$&6Q#W75s#NP%!z<G3*dN@Y@Qo_2lja|SfER4kyF&D!(h z`M{L-kA}BL?4YEdoCw2)SMl!WKG_8p?ei5r_y^3_$K0o^@PDqH$a;;|WxDfd>K7{& z6~2F#>cK08dDq+s@rSs!aw&7%s}I`4F2|Nbm`bu4G>Xf<p_Z>KMp*K0o|NUS*vMv6 zR?+&Q$%%*E2*JT70$4d5$g);o>}fGw*WGoOUJVjl9r-4?aw#w@r=U<Vi#9T?VeIE; zxVtGWy=KKsbwtz0!5R*$d6m*>`_dcUsAzB2#K19>7Nv43q$?=nvERV$Fst2ihy3$m zXIoyFVQSq($1fs7b-I6<L%dAxE&C_WqYxr?&=BZQ#JT>_6Hb;G#uCv)&8|7>a`QYx z(kgd%`j(K@sjfl|-)HnMFs(JWFzDClbK>-d&-Ce46;HmS<(sLBK;M!K>-R)8>hJGA ze||VQU~m|DEhys}>T`GA|64TBMSzIIqiS<_5dAum=;WVoa?Zmw$yiEcg0cB9WW@QX zC9+hW^$C#j!4B5nPj*Xmo8_ZJ+9u^RAY3u0;qQS6&gta$EwO}NwL45PfIqQpB9Uu- zPa8gczt$-`_jQh}7Q`l0n*>hM&#sSF8p7)N-7(FXyvufO_bl$D)@eQoN6=ZSUgs|u z=-GGDJ%CkM)VPyLRiz!|bw~n5C4q|K?C!QVuP|H)$D2h{_R@ZRMf~bjh0dw#8JS6X z(`Qu01Vm=y&n2?TD_1)g_4`vvpR3zlp2UYl7l;fLAL4x^+f-%=4rj#Wl9(#_LJ2c= zb?e9%LQZy?WRSoMHT8Z`z7BSEXs7`U&g>LIq8qhcaI#);CqY(~b-ST`v(@OyFkoPV z4|bz*BP;X1=HxMHs)%-6!=W0$s_{NCq}8ivgG!PSNXk;p5Wpt0Q@pbc^Im_gsg8@E zG`rZgC6W+riaqM?W}T;{R`#nP<OME@*n&X>$drV%du(bwxa>=?mH2Os%4rc4o1Zw8 zHDAfgkS);pRC>-#=2W1?qIq!LnYXOnHZ4wx;1-mv`&5G-&zk7V5_S!wqvcMc|Mak@ z1F^b%p#c-!?8t!lja=g^q*%l_i$LC$m^112XDnYRazvNvaVIM>Vj$hN%EV7I&zts} zdPBka;e`Ft`RtOOYf{+O2PQ+a$gbGbfP93@T~rLDdUvdgw^T?Zh=RWu$O%x9j^~=y z<Rixm1DrsM6z1z@c<wMYi7^)Yjz_lc_K*w8VyQxQ1i(50m(8Y!#`B`Q(cr<{sw$-Z zGwUd}8$r?#@+WR4$w6TioWxgJ1w<GHwl40rXhV}70NpG6xNVDP7fldY@kC><(xk0} zNB5uYK6!6c|CmW$fx`SdHoreeoWk5T3ot@s)I7K~70G%iWavI56nB8k^<+i-Pd7cp zn3~|80t*iYk(j|3odTG@zRi-!%?$U|*#_!YfqfM0-ajCDBG%QDY+r1{Gx58GB!s&X z7hLyCzWA>9(6)E1X+W&8vAaG@7nV)^0_Xi8>5g{FR@8HiX$lB*B~8enQ_1~F6nJSr zAo=CoGsBgDSE*)YFtIYbRtD8dr4$CKd_G^iId6rqEkua^H^n7?BMcO7K8yQ@iz-i& z3sPri2{!$`$^7kGujEV<2nIL{7}2StnVDV~E60sC<*WN<RO91@#K|4ywSvdK*IxjZ zM;p)lQi03~2sR5f@e=jB6y8g7QQJ8mrXcn^vz(7(;tBO%YttuYA`Gl(;YKs}kj0#` z5xK?Im+uuC7mgju?6U+tOL?CJHok69=;n(~&P*`1q=6%0#^`M9w91(`oMbg$L6KnE zI(f6m7PjAeq6@E{k^o{+R!MfcK?1}U#g16kGWlf}>xaO9YVV%~vTB6~$*oSyRJ7k3 z!d-U96i46J<{K@2Ys64d6#uGS+2L5q-(I6Z;EtqD)A$p$(R7+GB__4vU_U#f@ANr( zwX<SJGCNuanS#ICpR1<J$bUm|x=DSqc)mEu<%Ip4ygp9bm+#3uLb8}7e7ULE8+aJ} zfO1`tM~vCs!S`R*^WAFwrY6g$Np1IX?T}>GLRQ<Rj%Dv&S;3;N@e%b2<BtR>r(sGU zT$vVtN~NGI*t;L2;63YD@x6zXe6tWSLlGVRJE*xYv<tGLtXcPe@b@p0;*>I+tzb>v zh2IANt11)cY2;Fo_w&1twL}wF#PzPD2nHIJs%}n)KR?{vL<>!tv2fT+0RG;voSZSk zAx+!eOYYgE)meCC04K5zS~S@gQ4WM=Bv;yAC6<U%EXo1RsjHwS=&tHaz8QSCloRqp zS@8-~#8iF067t4iF?z;rvk~v3iadD6I^QdTqA5qIQ=>#z5iwm~?S71<Xtc4(xhtO{ zSz1$w9r-TO66HvyWL`<B8<zoYV~#94t$S6E@%B1WDLoq^woepTVQ@!GxtiK|Oh(LY zuc20>HhZjEC}wWp&CU<P&Z{vkcw1sn89)>+5c2AV!~*nKx!OT)?|)hX{d9fFo+YaQ zTmb$tjvH#~mXp%j-3ePrZS@VR6NPdsxp+&}HseTx%f~vP^hGRovSuTozyDuJOvy1U zGhw$LFG+*j&vDllyxiT8%H&R>Qm7aYT!tRyX+(tzzc0%D$xcbyJR3i@?7u66T4rdp zzI^MmxvS!9v`odw89;I*`9~)0I-}l(oQjk+8CZzv_kJT+HuLr4SiT0z9=~zuGq+dp z1M689me8a2Ix<DEow2^vYM2>~oGsOWhP%Iz>wVpbvDp%ZkQikioB5!;%MMct6-ZP2 ze5+hJTt=a?>38ZttBxL@(mQ`m2`CUvMih0PNiiEgw@KM;W@gs(1dadPo-FI39gPyH zUZbwVNIPoN3CPpIHL8t5j;S%Do(*n9xVVu#c*rERyZ2b1O3yyTjmqv9hU26LZCDzq z6)9?w$DW*QL}8TI0dB1%-YGKQd1dv{^D0~)qWI!taW;o6%!a$0L<3!i%n%T2U#U?m zX)Fi7moqN|h#a%F$*gp75%M}9@P>hcP-#{WjGmduRWW>lN1Mk^c!$oHEzvyC(!tc* z=P@@*VtJKX&-u}1?GL4*2r=td6<ql3$`>VEE~|;hFm~hsKScy#T%30wm@2Yz3<SC( zpV(SC8QOAts3XQ89;g3IQaG$@+TwQ%Y?)n{kU<In$45w5l1kr1`sz$MlUfGB6zBCU zifvF-9tXJmK9aLADA;r9>_6PH@-u*23iw{b&|OXQlr}y(o$W+8uqo*zbVx13<M=%H zN4x0dD6XM^I%)vvMx<YCQ1TZvQTEvNXLq@tk+mD{<U#+(9g_^Av%nDaNlpWrCz0*O z#tuw4DrDF7Xhy!awQnlt+8q#d6=<ETq^O<svrr?pi3A|D*y&exJS}&-dDCj<=1$p< zN!>)A?tNaI6?BmX_apz{pFZG@zc%nL3++#(#brTmlsUsm+}!Z_n_s|BIv_4pZ^Ilk zs5Si^;ZK6GRAee=hPLd)kTYlbQqV#(B*}tc5;FnQY66xB)iHNlbYFdvMFGnt2TRYW zi?ffQ4!i@#hK3h13=iyE^2J8m5UUJ4kzjxR8|E%lwrZ9smPv4HK$fbAY(H&_A=c|& z5r>`Jd|(ksN>NYFW%f8VV<uOF-<L#Gx*7HJGwLR%>ybQaxYF^<&Zv<XAki(^k>VnL z%{D_zF(;Z<eLUcO;-A08d$RIyuL5#7V`DrJJq@nM=uaNdYihHwC2wd}N{$&aX3)$@ zMOF=tCZJ3#orWf7wqGudX7Cpg+F%fKO`%sCTY>JG*lB22r~hOKL#P~|)0wdl^nZ0r zBSKEqOQyd)5dJdje6Y42CZiybh{6Bzx4ysy_bGr)<B|JF-jm(JddhxA6WkIK{fvkL zu`sO;=bT>J5Z(NpBd*#mg+Z8<T>Ox~ime_FEMj#=-(d3;pzB1oL@qtIg865_*pxwX zA&?TSNyvX#H**dfG9fc-DNW@HFO_EC{b8EF&PBe9hpSGVO9Ap;KbQTo(86A_Tk7Ty zOd3@xvDuj|ZzjXGP52c>iDqj@R6XKy@}phAdK{N|X{s0jU&2W4Bh54lG9QWRayveW zIIkF-DlSZWH0b*~5I&0(d*1%AfFxz1H|ZYn4!rDMaWk+jOYYZ4M$Q;1LK%hrb#wZX zYEBQUHY$ZkMY(A*9nC2Mm(-JXETP2c3;aY(CHd0~j67X%&4=wzM`C|)WVt3ShsJs+ z7J|@sJuhg*b!6lN;8?_zLFv!?TbDSmv4<4UYaGK$&bQNoTr|lJ?tFE#VUu_pQ|Y4a zN@={(r3VbK=eYfaI3SpUY8K&w!sOP2R<)+aVm_~4TRCdNu<k4jm%707*GHm0(ZN__ zjo!#rah%|T=3Lp7F6oc_><u&_k~)_NZ_r9|{&;3n%lyej|2P|tqi6D-bE~_Jl7f|X zE9V+vl;9C2!CRtJl+!`_6kB9$&B(f6-Y<@$TG^CyZoN0awqOv#s<hL7rRkTF%q3Z| z+?0c=r0;$?zQNX|Q)K_Is#>*8OrvPo+xqSWE_?vTJzj8Nl<C+%T*zzYm+l~M&?3Ey ze!0cUK$TnFaTh%&h6pvDe8}_4bkczhhxYdhy@ggssi_}C>DvCh?oId_BS&mI-=ZP( z01M48QXZcV*kwitVO(sOz1|XpWzF|lThD-U4=G290CvK<p8;p{$J&D1$+VH7KX>J% z5k`GFeU|spkr<c(G!k2`{3?OVZVpyY5v4qHMdN?70AYh%fAXdJqWz02oy9Tkd89UQ zwF(My?ae<ReG1!b0TfFfykV}R<d+$2@9UQ?q--p7JQ1RX#D=x<)2B*jCHA@PDky*I z92CF2n%VG~BG`0a>}}Vax3(ClkNu8vpYFCDk-(rPDwl|nR3NWq#g70WrOq$CFOR~i zYzz}ui4wk<xA`7T5e}UsafJ0fPaRKoS?!m+fvVukfZVOGOM>Q7g|7h+ELIJ`KiD=} zlqR`w@JPyZUy(RIuS?SEHMdYHIA%#A=I0m4AKj~Xomk33!S%$uzC_M)N&b!-av{k= z$IlAWfnU5%e4p}6`TU%X1bpvdF8eE5r8CwP)k@?pk$Y5GKIdVYSYmnDXdv38rkrk0 zmC%iUehTJa;Ww+Dh9ZL&Y!_7?DRip2Hh7#&gjp<mp;)Z!en5U2bkpJ^_d)~(%vd~x z6B!MP9v`j&DeIU1p%ckxmG(N-Y}h@Tr1zY12tgp1FTl6z`69uZosyxZfeQ4#yX0Fz zAvBkQs{26(OeDmoB1EeqyN)YWl1ZmuSTmH~_-(_o&Hm?PdLm>h30S6bMHYm-QWFhN z#${~*@<ZkR$|brM67`=<L#DOVpdL7><)VSXs;zD$vIJaz_I~Hs4M7T#37>v=U=6<I zw1cO{ApYkLInm@ukBU}SLMh72U18l0XH=J0smSQbbuz&#@HoS1kr>H?|066i@ajjo z6N*oDC2qb@`i7xdEb%5q(6S>cGNOb+<;>k>iHAGt#+~nOxbYotWUWHwS(VBL#X#gR zMU}8MoUGrUqMX}#mc@SMk!Zt&s^G>Cr-UUSt#wwI@tZ`YuVvAw|I5Q{20V1eJa+62 z0L2*pBMS=$(^2@ETcV$HFW8h$8or<2m-*Y5x<IACL!<FU49?H^Ymab=>c1(nY#5Cb z`OY309vFf(xLqEAu6vT5+$Eh2UVVN7TbP7@!7NV6Opn_}Uh(7uMbe(=9uL?WEjO1@ zy*Kw^sUsLFA&FvHYxPK`d{yPJX1a{BKC0K29OCi(vR$sX6~q78{<m6fU&budcG0D3 z$6H4x>GxY=gkL9A*?+4|Qe%T6RHVnKnr66r#gOe4sj2cnUZpbR-4Q8)1WJZs-$qEu z!~Y?R;qjnIFzP&_;WN(E^FbN)a$Zv*`<w{Y;pwO|xGB)mARv80`x<ny=E??pTO5-S zO-3<^sun#OZxV#c<9d&7&>6!Sgx&JiE-I2Oy{;?2U(Mx+c*fQ2#mw+!i+SZa8gl(H zWzJ0O{n#lu!nyseNHvyW8oPZ5Bz`#6dg$hBacwyH&*eu+c}jsAsDe^fX)TWGB$ai; z<9zK7hw;Ee`#F}`C@W$dOu)j>u*pMqF~(2gBmTTwI>fDt^q!*jE6ct1WQYPlWhj|_ z7G4#1BNJ7Q3Q4^(YXZ7pPplu29Sz1^UsHDPk>>3aH2x8d!S$Jc*iSX?c+<EKaIQ)J zJ%GIz(d0E}dTTb`i#=9{Rf~rm=Jg3M7V^YgaEyi68(MN9V7>VtCV&Q#q=^u{7Wb5g z3&s6nQTPr3fIUNkaEI%zISDs*q93n@mORm}I@?FHL*DMPKPzOh>h<!-@v}X%bg&A= z!v1Inel~?K?oUHp8nX5U+!=O1!gb(=qde~-eytW0a(E7?r||pdMBQ0Y&amc@jN$LH zz}XN;k<#C~8Be@HVW~?fp8PC1C?%8C^hI}{_$cVJ3AE!T1IeN9S;+s{XIq!{F@wvT zp#VuMm0wz}h`quaLg>Vh{4qf$i_bQ=0Wmd*P&uMK9sfT5-Fw)c028gwf0=LQGULQ; zN)AC6=8D84mUrPAVl2qzJ=H9~Fa-mpMEEC0hPXte9U~4&F@~C}v1XhswgZY#+s_Nt z9e<1H)p?{eYyqNN*J}fo7Wlti3B^w!3lii{x*k^xt_yI(2M0ehXkM(fS_iS<z#(TH z8#@)v!QYlyoqAnO?OtNR;08-y!I=t)6Bg%*Ne_KQ3+6Fl+4D3mUno^)&KJ*&$di1G zv4OhxEx-J>x|~tMjF4t%D*F$1#ZH>u^5qWSk({=tlP9t@L9Hr&S3l>6X;HYaZp2fF z&9vz6)Zx|Q1b#VPu>XyXk*pH+A%nZCp4@lEo9cB1ZdOs7z>z}dj(a!jWY+l4gb)1N z9PO6bDBbqZRq9b<Bokg{zcSc{1P<wHG<+w$eV-y==ezL!4wv0R{wSilfb~LdBywDB z1xhlW1sRKYT*KzwJAvUviqdA6h~rY*BNBRZ5$nntlUUs@vC5o-5G3(tX^+|OZaezC z$Pp6vpGT1dsDTb!;J;?cRbR+ow71n41=DOF^LTG*mC&du$<f}@<|}Gw$|^-iOnOxC z>2Y*R!e9Zt5^1?cMo+EKjX%2ZHF2n`!MFrg>4(_jKMIv;jR-eN3jGG&*QM5f-92yA z>iY5R7)1D4sVe&V#(o|Gjbi_JUa#v=wjjY|C<b=#n$hKmmiuu@$J<~YY*e23Wo<$K zwXcyc4$4K~gjr-j1gIM6d9i&o#H!>vmcg$Z@T2;BKC3CVy20tXRb3!BfKiW<(l#zi zC%q^MY2<<BRwkm5ghpBM4Rc)#H|XL|kUY`omz2mR9MsdE{#%9(G(Lo783fJyDeHTK zPl2-*J?aGF)+j=wM1$&>7bAfXDVs=bbP&Np$cAX1g=Mx0O<Z#KVYBR>T3z_ZSp>u= z?ZQT*b*UjHiZ9WiG$Bx7RIioaN+6RoqVc0eITMt`(1t17azw~<C<pD9iJ6O&mC>M{ z88Bhi@3GV&kLwt^ec*Y5uZ9J^S2Jg{yuX3XU$B*ZLGo`j0X!-8yPGcmM2Omp=bQTu zccVp}3S&K|y{7euxjyaqbt7Q++k5FiCz_VvSqkI{S9UwOxISp7<_ChDB-N_D5D8ga z@{#|wHK1-gKD>Hjx)F@VZ3@`%J@;<g7(Q+l98aB`?eZQ48SN1G3%N(Jv~s-}AR0<; z{Wc&u6{ncSCFnv@)7`3xgjwsY?B);P_BXJd9n{~49;(FM@e4Gq8$%V52oJXfSmrmt zwm$mC){h4l;3C4ZdzfE-aE-vYujS@qbu*KmIb&?~J*S=~5EkO|^8|gn&U&cV-Oov4 zTl2<e(5E}WqoW7ncU0KJM$O$B=HL7g$aIN19Xmtryk%D=+G%(5X_&2jL0*0VxgdBv z^p(D@3DogP;HLkmsW=#a-f>u0)jK)z<d`)W-Wc7RA#8R8neUn%o2|(NvD}kHdkVI8 zn!6u%9T9jSqk6>$!n_A6rKW7M%?%KblAo#rXN7-^o;W@U15&|WNG)uq*WakaOQXyc zyLftst?>;M+o}X2$i{!;=wXX}3lxN5AGxyUZu^ylYT14w<E!5UJ(rAn)w+e&gg#L^ zp8EX%4@j^fuQ}eBpJd*Stx~Y09msAhif>qPt!CWa8zEHxAA4^d71i4I0jnq&1E@&1 zilBfnAl(KiiqhRkcQ>dbAdS*p64Ko@w9?(>&@q6-0K>p{k9wYS&hwu0zH5E|e&1TG zHM5!7v-iI8cg20(A+sYzXvmi1rYo&u=k#doR9!KLYd2NAZ}9f}kEDWmq7LYuZ=v0N zFzX-Io7>}a)~d^MBedpgIf#w{)amjPeKxLB|KX=IBm>34ia5MhAHJRs4(2<%6gpKX zgsc+sHC(BWUOJ3Ion&oxn2m_Wf?Fc<jAwTM>rL35(58!TiPT?DK6cyyt5~G^SneXe z7DVl>+EbROFP53Vy{-zr?Y(`kNtr}aZ>#?L2c@D`N^bMJS4ddh&F+dLNTYn;AU_YL z^W5q@*pq(g*jaH2EJi2HUESh<3td12kJaQz$mhIZz&44>=czsy52VQ`Hoo*j2tQ%u zsrqLVak#GjZT|G?JdG%OgImSQQ5yX<Qm(EMPzb{78up4e!4hY%R{wiRe?#ae^?)-V zJPo;)+G<3uG>m!_j1B)=vjfOsl@N1LXPkE&Vcb6?ACd>RU6Ed&u2p&tb}_|%A&4Bt z=TnwC$*_>Xv^O6Pe?UVhCS%9=)p%L@mf@D#=l3F}C+G9s4>{g!x9z3fSZ!qJcn$xa zrm7jyUkXn<qQ%=xhES~W6e79P#+BQc)(nmD_MfTn9z}p3_iqS*FTp;Igt2s4V8jqI z9~^0fJRU}7h~X&5Vy~XhZva{3h@(BrLJ^F&+ZN8rP9r|idFiArw@DYviu~)#XE||g z30&Tk4j!QLH3YoR?cQ0_Oik;2+Ql~>gmmnP5$@qYBcC)tiz4=GUA5MB$WBj3G;R0a zlDEf<=J=BzQzGn#pZxfC0H{%Ov6IYASaW}ST^}}kHzsc0!;}-NjQQXiU|9$lU}E!Y zfe7{;APcww@W<DN%?^Wq1*o_F@h(oUHR<Pv;@v-oQh?8IMgHgAQWVq(d#3kC+fK3v z;7mwUyZ(RP@%`+b1)tAT{r{K!|1OacY0XD(>y^<I-XDppZ_Nw;l8VYeAsjMKslN(P zC%wzd{Pw~+zvBZ$aQ+J|r=cVbW35}e%kD_xs^D5lC&Y(2LJ`c3<!W)HSRz3hba`rG zbEck+5Uf{U7H+m&@}c$U9XU6Fb4VX+{icTw;tWPMjIkvWE?D-Wr6w@_v<E`Athw%w zg7-8$3q96ocPD5y&=e=yD6h%30Jy{)jmdi&N;Ia)^@RPQ|DMX}*7)#6p?#_lbQCn| z(E}@z?{v}E-Uy2>aqN7f^*ET5z-Uk;dwIB?iqVapXf41wdFB$HqsN7Kc6%Kw8QIgx zEAcjHfc1KX#6;fLil{~JcYKx07(BCUN9?lRIaXYB+Ca}|wG~3e@Amk<R^^Wmiz$R) zG|h7mhsKT39&~Bcx=3{=)@Qa+WX;S5i=5mhq&e0WGfC8DNtt=S>Ox*WOTOXIvx|4` zfP`)r?}!B<xm7!J5&!mrvW+S7yAPDCqU|(lqja7N`3f{)>Mho*%wgQ_SN9mybuYPH z9(En?M<yrxm>xAuJ<2tIL|Bh^luKLhd-}~N(Onz5CXJX{f)QB<_BJ?~Yl<R4r#69D z@3qg|I#*q;Zh*ld64m4f=f%#Bk6dJZi&uz_V4o$Lw5?wf8yV-G?{%!VMV)WX{pjih zk6q%z9qS?0a=Tk>&=U>@13dFtCNrMQNqZrgc(dGMs^7K$2tW2SSPX7F`nmFCqS(<> zrOa*N5rPGMxSB1%VY^QSW*ddL*)Ii6cYwV{0v>Z&vKB9PhR5<)>K`sHHN{RvO?5s& z9MU^Ip)J~G+BHh~hQXsSgFYlgd=%0;#OcZNJlL;RT69J$yd%MI>cv!}RMCqxpk!WY zfgJ}G!J<YF^S<4_^-=FN7%!Eja7LrVV9JF&mRIb%PvD^WZpH0Nm%Xf@(izWF!?9Y& z#1~JWg<2y*^%3b^VB}bB=ZR>J+0;=Whr#6eXHw520jOP_c{rxUOYq&2MB<WAR10eJ zG|AUo+(@T$iHG9b%)4tJ(8)?)Ag|vO9NjhVh&nB?ZaLUH?h9NZp>5abB<rL%fE@Xd z!%lQtlSHgGIrMr(I>~kkkM5r(1}-A9K?rdeD2C(#E6BojzYOSOyaR34hRyjLj^7{e zH#HW4kxz%i#znivoVRtFjmGNqA}1?t+sqvog2fPL+HBGL=Fx)sF?MU$m_aKm*gIP| zlER+iTl2FOn~kHr5R-&A&?xt3a)?767G07rffC6+GiOoHW1-80=g&WM&WC!y1H?|* z^|o5B0?}^t^5-xcR5l(i3uV!$lQrmxX9Shqa0KJ4EIy~jb-tD9>iWly{^0>!rurbr zD$+sRsdPo<1(ky3%pVVQZrwKjapRezs>$dF889@6g9`!;U#!6F`BO4+8gzZ!es-6` z?0`w7)<EMa^I;TpnRZ!$6Bh$+z>{6Cmv%V=&<U^SEVuW<ec)ZMf{&WON5DJLo)hx` zM*t+BCT+$*qtNG(<UN*<qC~d(n!6dor|O0mIUF<rr28??d^5J(A~e7LL_Ec{^;y&6 z#_PH73^vPKI((jUDbxAyP=H`c9QNZ7?COj%vx)nlc`KBDyJ`Mgcs4zc*u@;|ZT-nF z&tHdiWZ2Z&j^#!bevw1d9jTNmyF))a<nSYhjj>e3%E#2U*H*_?&%oAgc^dWf<rXu; z2c%VN*qL<OO%fd?<eXezR*$`|aJVm;tqgc3A+NYA)LYxC6xn%27>d{~=6tTeu)Bt+ z_$9A${=T*jkQH!k!d*G=;4tfn)`H2F05}y8A?z)&fp6iU?*F3D-)QnqIIhCo8o%Cp z+;o5Pu)j)!vCD!nikOn;@hZ15eQN7%Nm-jL2sO9tJ~i)F$l~`?qKX&}`>H|n-p4Nc zNJi`or6!-?1Kl@Htx17<C)&TS=Xs1sTtEEA2n3gu3)i#L!<?^)7E2QY-S6((3z0Va zlYA=WI;kZVw5$@rG;-8su01rqD}8F?4(m6+elh0ogx@?o-WlNU{**LS_J~xa?&hPq zlOt0pE9Ghr*e)fz@_ns37Q=WN(}CnrDn1waJqm`<$b5B%XLpGfcB6~wDYLtP&T!$H zo11B2g$3LQ-q9CBh-xCfP{71Lh)jiJ^e;~0xsL?ECmF4JE(Ss`t_QnApBG5v*A?ln zIUIJolf)O<qG(hS3MC!Z$63LM)=!1N66{uNA5kYDziS1Eebag(pNs9cu~K6Oy0;{U z_Fk<?>Kd?vd!jV2L`Bzk=3Vp%Hs_l$dUfvXE^S_YspT^F_;q*5nY2J)g{%7#W<1dG zW!s@h`ZCNBtCd#8X2M8)m)jje>{$~ftfxN`v*gIxa@`Z%Q{xkSq$ywxQD84o2B{Q3 zF#ilx=F=3&dhap5Ob81r`I?oy*cs23+8Pu|Fy&G`lK-{TbR<nB+o8y$i(iQ3Tx%E! zHJ?ikm9WiY0vf^FICyOxhdZQpTDzgt0*)(?rdtw0Uh89hK#eZuKo1%M#Av1Uww5Xi zukmBaxyiSW76FC<Pf_v#3aZEMgO&Xc4!_lKc3S_JEYR+10JD|YAav^i7_~t4K#A@m zm-xu}+h^8SvcQ<8m2{M6MjtzRT*7gfLwVZL52XWn`7-eyJ_S!G=polXet(0D9F&hv zn@`@7JS1|aJPWN$?0Jo;F5h<SCd7FI!L-%(l*O5sFcx3KuIQTPfoYYqE>W6yxX4g^ zIJs{%1NljB8o>iG<TP3pc2JW0ljae^=%M)PEG0noa$~h=L+-_Le#G80M4~yNWfW@< z(J2^dL>7gA0t4gAo)~l|+ya%4=b-Tw7>8?^<M?Vv`#W>t*-8uhN?Kp50%Ox$VCh8$ zQ@E^z;}uWTR4Pq!@96j8Ic$zG-;&UQ(1hdQ5t^#V2qzAa#X)WN^#a9CTA`Sj!2QQ+ z&bW$OYh?>=kBW&VKF{FR8>n^j1ZG#wTqz#&J-twG0S=MDyDCUic>Y+#PT*J*{|;EU zG}!~5bK{V^<VNB9+4_n7gAHf98wG@b)Ng?60Op_`YYyJh)sw?Kz>#OKV;5Pc`#Ds= z5)bYe;ld*oBCFXDC*WEOB67Z4elTh62lHA?XIJ|dI&3gU_$s9?xpphVayV+!7(YxV zloz;+pcQ&jHjQB}8_*W{aj@iT)qXdkRsW=Wb710PQlYg+@C2DpZ{YdfcUr~Z#`4ap z9(RL1{b+>_`A7HHcsyb|tBzn3*bkk{(I#NbT+<u(UHb`}x~UOi*N&+rz=(S58HRm$ z6sBamr>tSItH3Sc;1wA6S90}|{7Al5WHsuXue_GqwikqkBH#3a$KDV%2Hh1~KInwN z@L0If>k+ke1gO|fy4Hs&ZU}U|Yv~vieLRt)QmzTA;x|XYIzZc{_ym(N#eL*V{w0}! zzOd_3FXKPQupW$k<Q7xA@M1}{XsFn%1kdz2=YIBKOqZ)!2YjDS+KWD<*T9@|HE`zq zS2IoBWOMpa4`eY}Uclk7;iD3vi?<v4%LLmvB7}a>OxV;w7v~tD4r+<+WnG*niFkk` zAO25{RL)QdIjFnr`i5R#xOLlOlt~3lS%@&qiMSGZ-<U7xqaNeaiI>n%9M+#YqlA>M zdj|vMmQ#4)QswU{q{p7_%67QgO^un_nDCVDf(Iy=&QI6oKT0343xwn4K#lmn5t8|v z1&VnlNmXj~w~8-c0{_Ge6E2qr-1Qb=Pj}I`3Y?^uZ>1aAIM_yD=R~Mfy5tT#D(CuW za*zoH!{e&%$O238f`mmig<--Ha9}_J@3Yd(Ns-%vNgyonxunCmiC;JS@$oKsG>5tO zZV=^d$IWSl;sPCcW-w*6jN+|CaC}JfB_jP7x<zh>3z1{{(NtYz$Iy-du~>KFe*CVN zB1Pn|<f-M1lDp8?u?rOAQlVV-tx0=h(iWYaiyjRXO+q{e_7e$uwg?*3x!u8A`D6R- zi&LD%16S*D_jLd~1!f)#lt_UAdg~hbi{yV3dvfNHya_tc;oNcR&Q>3mC#~WanakAD zp%6h#o^8K48^~5Ce~UfEMdL1!rUIaGzo@(-Z-Rg$^4oglPTFZhGDZ72q_J^mdi(o% zc|P~W7wvYz6fhL+GEVq47jR`l4;(Ba`3gy{>LDW}x>)Cd`X*>uFWH;Yd2_Z-dn4BM z{Wr4roQ6G;n`a+=emFGFp}VV0{@}KlU-%^~Y|J7Ma9$02+_w8kbuSe(FNv;<@`-~< ztoN1jH3kYJJ|)YmC*6D$JdL?s_R4d}a5|@b$aEMxf%3Cjux5YRqml@csX9HSr+YkY z@WOGS2fIm1n<5ABVY1z`N6axp$oOQh?d2191ER5FQKsZX6TUu{OJ(BA+LPRDbdG-* zaPS2FfOZYFfHMH+^@=Q418^n$*0ptq64p@nz_c?kmds+hUWF?>(z$yk(JuJMt}4g` zripXv^=e_M0O;TH`&^KQfP$s7^Hnk&b`wGI9QH>qK-J)F%-iYOgpvHI3i>r}voP{v zgAUMsWS?uW>5^ItA|5)>r22`WBch`aJJ4-kTcj;0F~sY6L}gFjz>k#V;pu?G<{6ft z>n?+2HK(vurCTML&;xfm3$yn@Z=g>*#}h6*yo3oW)9>^guh2bqAuBhBbZBT&=xr$g ze29G}>}}#9^?pMqu^BEscGRUw&WC!CI=|m&={NrVE1r8ty0tev2MLm3Pq#Tm4NE|1 z^e#2*phUmyO6&zjNDXqmJ@&oz8$@7F2UrLT$=H6u4bYu3Zz*^tLC8_l0Xfd<LqI#s zWhR*X`D_OmGz!|gKEZqlI>3VdJ%1P;{z)uGdNbbM+T(ED(WZL?eH2J}5BbpRAI-?J zvT>_+9#MrMh^2pB5sOE$Jj9y)DCmLi#Zu@&7D(j$ewZWJubZo9x0oP{sIHOL3t_8w zM>7MQ9p8-a7kLT5FOwH3V8Qx5t?8PyO~b%gh3t`UA@8qoxJ){c3+tA=h~e1yNHKWK zj-8E5PM)og4vIg*X`xfEaf-fOjX{c7HDIi|$lQqkp!I)5VygvE1$d;t;8HB9bXsTq z;Q??TfZeewj1{5ZC|~P0_cEcG_M>d-Az(U+q@-%q`dGCV2^eq)x)<k{3lT2bXhN#6 z=FhLOQCpeC3_aLIdhkg+Scj9w!$&B|5yxvSoBlG9zC+2Ny&OrV_&TTmN$L`-@@)Va zFReUE{|jONLp$spiGdqw{!?bK_tg{tCj|r~{6FaFyps+r`il;Cj}75s!%g9p-pXt$ zUZ?h2hvy`$D))$F?||W@e0iF+cVOPlV2+rA$%(U5>uiZJ=W8%5t16NHQWF@{%4c%f zyGrl3Okr6Hs8^4LSq6|gGLUG~>wWbP8V7jJ{#&;H2n)dB6+~H}PGtwE@(S68Xm9SV z@x1Q)zR=GG)>F0Z_i^FnSJq1<a7d;1M5V0}{P5^(TH!S4C|Nn)&|LI&YSorBiGh$k zkmq+?xzi*O6af;lUvX$@CWZ8R&)rW3nZuuP*Ms&`ThRvO73@ENqW!<-b_I0#LMs8a z@IBUXyRhR69blK;OE%m643&0=MFEtFId|w%f>9^f>Dw%g^MlW-gp+xkv>`Mi9<rd; z|MV+|&W~r)uk`!pzktYiBSTmnm$s+a%uc>8)E%$hU}DGo{E4&C9!-wfl?`!re_Hpg z8FSaVKr|(aV;p6T;s=n#x<^5;7g<R^vz&kX%GT`F;~4YcnFLbcXRy-yOR;`NRaoi0 zW}>&{lStEED>C&lzrKF$<dpD7<S9f@jSNW1Y|}4({fy|bCZYEgHZo~7r);u`1Ot9u za4AoJOoKh|B*7S7ef?GwKHJ5(7t#EW(qG0V<I--Pf;r-^hu14PqztbKgN6PF+e92z z(lzn{Id^RrSwpxtQuKF*&3|<9csi}qHEBirHlj{r0?%-r3&YEq9WisQ!|!`?O0P`; zo}j1xnzq3rtH=rXf5)LW?`%SYd%zo=pWst&E{*WSp=VDo)xF*U@tO2#!NyqG5|{cv z3K^KGqItR)(<#8OK|+(rK#Gh55p+G{%ruy+a{Lu3paRv~()lfz*pM8U0a|cK`um|x zAV2h~Z1%F&MXa8J9eR@Sb_3C!nT5Vs_i%W;YYg%~X}9k%bb#@(dz@g$`%fU=e!Y;z z<uXg7dAILbXhwSupY4Jgs+SXN;^@0E1!`PlJEKVy_zhrwwh9j}YhA49y_+{e*$;#O ztWI^ebhnqNL-UcDs2TBiB!cem>mdXlU1f2WyA!Tjo~gw#m3-sw^p7VJaZYVSwABk; zEtYwXkQ)MygGoi`Mox=P0*~(5L~<BVBHuqP1=h++WCp@*%X`|Jg%D{o-bfCYHtA>^ z_9rA%vALdvEhnd{KnMPJLRf(;pcpxo$P2IyGaVQS`VoxemRwjZ)gKzSw?QF=_&mOU z-bIQ?Q*fCxfO1R3TxzRi-KT{GFy)WXp2V_M`zXw~$ZpIAlZPF6-`=NpTW9nlpWVUJ zb{dXTFuuR0GMA-m08NFVZ$0H}oU;y~-XEPSYePl{En6s|<hus@r}#9~3=|CY7c|r4 zCBMj<M9MNVAa6dA{LF+z*ICk45S=FtIe+op8I;D+;R(U{J7>>LS_Pl~u$fvxf1UKq zTh|_;rAZXul2F!7X=)h$r3xd_v{Ki2NqKhC2o2m~{XnrD88MI4m$Td9E^zg8oYe|T z&x3MT3f6HA8m?FpT!Y=jwZEw=O?wk}@Ar#N8QFULikOEb!Rp2kSuCxWeil)skO~vc zx3+F(V;-CjWDxP@B;m)9)kn=qG~h+7IZ1QW;yLBMfpfNQ2xX$2+RO(|t;Du`545Y> z!FEQw)2GsEZaPbXST$Aa?QdMi$fn@!y-G_NS;gf0jkWlAu<cuAGF(js>1ExgQPw+6 z_IQ?)Rj|hIvQD*iQ%e$8V1qT4pWgT%7TBa!eEKFzXx97~67xzF^5a0VR$Dm7c8CII zV<yE&hdX9)`JW#toowFa9sa(mJVAs`r(RpSs-+c;)IR@>-6dVM=Gpr<abee>nS|G1 zM^~Ylc-LSq;DxkoqTjn#Wx9%NUJv3I7%<v9WG-f@C9=3ESbgDb&m8{ugXZ%tIkl^x z#RKibO>rFW>HpCBWN#ei7632V-o6z6Zm=`SP!#elF#GTW*`WUiF`)z4F(5bnPjb%? z+{AqXS$+JAluv%WG`)aU?3mzjm0gx6RL~w5SP1qp%S#&Q^JX#SFMAyi*LV!Uz4eP* z;8zF%(;95<fn&4sG<?_M8ha?&=sEPQ3+(&CsEW)HFJUqU@D=b)HtaWDezOgxuhcvn z+oomsf$YZ5$HhK1>92<;EN;<md3q_!yJK!wDjTIXq3Sg>7C4`WR-RGyIqVBOc-*Yb z{_8ifB0~pmf+YjIfjdleatB}-Rye6&bbf%>stJORehU@h30?gMF%3WiypFtRT`PN2 z;(;p!ypaY0I}CeHw~y_;KZyOJIpBcHZ$_}oK`b`SlAV|c6>jNUvGOa3n!3AEV~T5? zdOfa~sxe=mcM1Fe*Zw7iSYZTGNTd7`P!zUNreZ$Y-9i0=^@Xm1wIsmun>WCw(u{wS zn&w!{;~K0AJT_s3MsYLUjH!X~<qJ5K?HjJ|tn;Z&)lBcTFA&CFI{WYUW?^~Ll8#a# zH7!cbAzPGWgq&(GVAi5TfyI(V9K$nB{I87aGHHt7iIjd2<r7!OxEx!mYwa$xm+?98 zl6dp%rC$$^r2*cs#&0sSR2%rhBl9(Noh;6v#`6(wd!$YlT@QUyi4_eaC1EYI5wrio zOU{<r)_q)?BMuoMRou-iiM{wTEek!DzPD;pGdeovntSEvufdW5Uwl4@xq}xN5Yd7- zKftF6^Wt&#rTzyr4wG=N7|!<P7vdNruBr?HcFWz|&$N3uH*sHm_;u3X<(j2>(HBmw z92ehC+xf&pi=e2Q29aI#YAg0QQD=0D?aLF?Y(D2>2)G8g81|arbnJ@{kPCNHDPK5+ z>xG)^9Z5*hY4e#8lvHFGOW2zT+G{ZQzuu-nz6*2<a){w;UvYlNxj>76HEJc9l)S)i zyng(FOpyK;8L=8LOZdZ|Y|sl_v0y#sF0N~w1{3Aohl$yE*|E$8&feC!+vl<*zA3EY z!h`F(uR@uAY1nV|!D_96A3SaENjT`5Ng;Sn>AzY?o$9%u=`g3*#L$@!>>`~mJK1kN z`X$hK5pv*IyV_ill6$u3rYD^QzPh1mA9R!H@{w)GP`r1zl&>$|RjAX$UxKU&Uf{Iz zq|nSO*z?A?o<nTjF12o}h7pkJP8uquPS%?4w6A;>O<<4^H`svxvkr5%(2baQm_3ux zwbHj$5o$dd{C;#!c7ir}#Q5XMpy~f6nmL?PCS}_B9q)4ES&1XFN+WO8GFz8>G5|tC z|Aml@xaBg2`J!W1XQ{3b-3w{?xJlg9(r)y;<!IPt`T0xM6Mo<)fMoq*VC`={M%~5D za0|UhEx^fUws|bwlVg$lU8RR;aGW_XaX~Zae>j=93-Z83j#|VHd0Pv?FRS@}({3q^ zQ7Vh*;~Vl@<BBXOsJnW!E9d_KPV}cc`!CB=5TR?8d4^!-y-NT*d)W7mpixEOLLdXf zl?zxm_=o@d!wsgB#chqtj3)ZPRr0!yn@S7nYJ<LglzpxGsjy&fV#jX)T<c%$k20*5 zM8C<4Y-e0P&ZKEsVy{QlI{637B=ddW=e7T(>7F54%}MtG=VAIIFQCf$%v}81Pwo+u z=u~Cvpy=}C<9!T=a);?}{8XFY`tldGvGch21m>KzS)v57K!PTkl(UX8`B`?#C-p8> z*Pvib`7Z_g#Q~g-FIe%P;?Bm;rn>j2Mfws|o%K+nE`J|=4_#f}!S#XX1nn+e9*-dj zSAR(|a11V6c!$8SJ_HO<cIdhed(qb*qq1<TB}&8kOPhm38MJh^fi#UaYd6I$2$+2U z2mX|1z<c4&8?Z-6j~j$_Z_F)tGYMwK#~ue46B#lK!(8<h#j74QYcu_lt^al@0o(!Z zpHu4G7;rO4sYjhlQ_HLqskOq~Kg?C0{Xgj_7?wYLV}4z<kzjQF2a9n2fsnXR)99q{ z$!5)!gEfe43*A}*KH?6;q<v$3n{bB?2P_}o-1D%`t+TUA>YtuLU9I3X9j!yp(_SNo z{lZqz1Qm8FoK8aRbm_~?lYjHeT}Dm7=+b8}ULkH`uww06-_J0J$k=+)vf0NsmBj(Q z2rTJu7X7y+1=i->iMo8iji!u#^}?5)zK05gc@Nxj`*oqEnc8_|ueIK9>Xip4XHS5R zuZr!My(!_L;bCq;8bWUgvt?Q;c@dvb3b#%n9-n7EdX6G9t|lfFp`io}F`cU!zW=ej ze@aq;UKMc~%!@3s9IuuXEvK%w6?}oVVUfCZ`1u#uzZ(ssCPnVZkJI7rWBM+Kdw|`L zq06e#@bW#a6z3X(wF$g7<(F6f?k)<D*MFfcVBlUsEaocr?%orZET}2`OrytWX5HoQ znmf`bFaX=j-3w8Qe|c;#zN;Iu$;q=MCMCsMSMN@`R9E?7!f`XXI9sh&WG!`j``P>g z>G$zKwF<XA+=RvX#<{Khvz3t-QfCxpRJ_7U?Dja(ye~7ZTp}crkyeIebm*&!{3DC) z^@f;e?|bX%a4=~wd1_8*w$NUgkvKIAhLpVf0GIwH*8hz*$jzf1hhXo^Hv4!r{lmM$ zr%nbZ{jTuDvL#JxdX?t3Ln8*>|J^L*RR8`ZWrtEfF$ArInNX=gl6%U9?AUDjeE#@@ zwSZ+Hp0VRqxN)cZ*J%XZ-tPy!oUBBh@jRZl<+8Q1H=jDd&<h`~ziRJv@mq4fa4;g` z;a$=ey;^m-p$FkmZM_M9RC8J_qCI>NZL@_IaNOSUJR*I-T~2er7X(=T+YJB38h4mK z$t(%iHn6_;6oPwDFsqbFncdjms=J&g=ku?IjvP!A_(W75%a&?fBy2=SY)F{y$dII) zDSzpaSo5s!U*ODNjhq?Q$hE8{1Pu>&lZ>~i$?&Pa^Rm|(6&*2d(lUI^1_~tP<b}zl zrPQ`G!6pLK0|N(}ryHl<ixJ$rdxWzcoLUHY(`|$$>lc)%ky#_&>5&B20K2r-_D;o= z?b8>Ma35Ot9NAB@>?a$+9`CMAWUgEX*zbQDB^9ABnbf)O4fXGvd9Fw{@rB_NAIa%? zjd{kft3iHD7J6tYy60oa@1H}D)p1C<GMU+q86r-s_Q+@v9(j1VsqQ3y3mUZfh{18$ zgh7c8=aqq{YcEC1i*aHFM4BrVftzAiOx0Da<MMre?e9=j*%9a>7JvavLXSK4<ws8U z_h+vzueN2s8?Kuhhedq3e}izb3E-;OGEK%4{#!24r=yxbjtgHk){P=M84ti%_@Syd zVQ)z9Yd(~*_C1P)h^x5d94u*_kT{@6Ez~;P&q5E1J>3W+^VHjdJ&xt<>>aaDg2sR& z{;m6ent&~9#}rBHmc8hv3ppHBq0$ewZl>@GN&#m*HbOx~)fl>-aj@U7F!}1Kyb9T< z<;^fJMN?Xi#McP40BY-Iy>u>|ig#wcr?jk;eop2(a{861f)u`ip`jznc1!VzJ{NNG zxu}f^zpCo^?A}qzeKyzMX#aQE^%n}J+BP{2ZVOe#<%;QAnVS+*2XPB-biZ#~Hnhep z4g~$HJ^}ZHRN&!om6gtDo+aFrfQ|~7i@~+?`p(+kDmd6NZq+i3x`lgb#`5d3=g{Er z9O))T_7Yq_H7`FWtJTZ0{*j8am&<gzjD$7G9NRRS%iG%<<qLn`{Q7v)^YHY$=LzbR z=cLK=NNsm-&(t`WTzPDA^0K7iy(qk%=*#aV<Bj<R)3MI%p&BhqxtZK~A!)ioWFU9K z$9`wpK%&xnO%HivsO;NI4dA5@Zt3>khPI>q6Q(sMk5BH`34309l9uKa7wskDcI2g+ z5sl$8FsMk}x)}UPcI&}XrE2+MHS>t$$m={yBl$-54Qp?TB>SCxjn_h5GWHZ5;wGy6 zdFs_)hMiwNlj1xXroIs#_NB*&<ez*icV7UvS;Lr5Nw>tZy~?}qvT74*1+L;BhZBr; zQ{!xU&H^F*9WH{<^l$MJXEU9ITA=gB(>0g(>0hgo`jiCN%KJ7~pS;u}F;G(H;JC8r zC*}|zb*Ch~jE2_ZnEaDm{}8%}+T7f<?eNOXcjsyU5wSpg;^U(QOON1f->*|15Ru^P zFv8F+l*q~5x@sGo8O@q^^+(yXdM+-D4JQOMp~ccncMkbTNZz5}qs#TVBt5!0^N;+W zw{D^nZ%sVop?TF)Lt0#{&iH9bEGXZ3qNu8Vn@rzvnzvQfp=-)XFg{P^!0k%b{Yz79 z6JL2eYuSTW$TLK8?V^_29g0}3SjN)3D9zO#=sVi$<D+6Ml1wtGICr{3_zhI>4~!{I z#bs^J;iexI7TjpFud$9JWA<CD`^o}s?_t;3^<_e-P6)`Ozph`dKia5o9G}QoJ)R$I zdnzd<QJ#iJ6dKBO?|#ds@G)6!Ek-<kp@|2~VYthufS*Z0LnCJ5g4>;NI&8$eI7HFC ziTU9^FX;KUun@YtheGJ+av-NYKio|X)>nkQ_>~{{opJdO81C~{$#4nji+=Z3C0iAF z=L}v>HA(zavwHt$GZjael3Ti4@Bf6|Fp+jPU1?Gv4)7w=2r{ae4GEhzoDyD&cl#SP zlknxqiwXPs+6K-P0{Ndec1Otj{Ke`B(P~3n_PjfL7r+4LvkvGARZ1VY|F-WXxBZNK ztA!(0UMjgL`|6>Kdp&_|qe@o>mE36k;9$eFE%@f?o0GR^cXySy%RNzYpFgKJt=NgL z?-WcUX5`jZ$tXnxB--&j2&WFzhYr=1-7uJ8l#oPOsJY;xU;?B<mLM-@OR#;L%M?*K zZ~x_ewspY5XiW9ClWd-}$3e%B6D;cj+age_#o|Jnr|ze=uLZ_Pcq1f@*jVv|945{6 z+=*sJw8St*_=5fkN&Ma-JDPg^oW271BfT5T&EJ3Ms*gN^s7er>LZ3rUdKuN;xmpki z4QmSabhOc)9NZP((2GilhUWsicKk%RMw6|gF3Ef_$aY5k@iSvSg8<CwkF^<kwtDGj z57Qst&(3^sq(74>JT^Pa?twY+DcIf3g3BM*c(pJ{LI@y`XopP@O=98gf2D0w&6}Kp zRNG=grs9L<D&Nu_wpE0=js;3=ltULwbxb8V8vZMUtxpCyo8Q5q6r_p?cXN{nR&(c$ zJ5hT{dSfB)-CM=l31qL;#sCJ<sy}r-MxP?s6jZanf8q#>P}DYP{UWPA<sR}OSmd-= z73FQZ<mPxt!NE~1?@qkzxa}EaRByi|JdGa0lceKtQjnMRn>j%eGO;rA*t5n>$*1%W zn~KT$uEN|spD|<+65I<xGYe(Pr>N_(+xci;HVK4pEX(>9dG5L+6zsZ7blhEq?8<d* zU2)1|@29$^Q?IElrX>eRbY6#@9d&s5^5sotG<n^@j>_>IqDNlK4<|dypd_h4xuwb} zsM2E=gLlcKORSswcueH<vR6x*<mIB%!O_8-Vbevr?ERqoaYBw~%lc%79JW+*vMnFW zO2)*zHBwZ=fXzr5{L(D-g(V>r7{!k)i%n?PYp_M4t1yEhws*7<a>ICl6Acam<J{{D z+0^oo@RW+v3siNN7;td5mMV$s$x3?UAmXGruB3kt&EMKb{)(nRB1=WLpl1v=*nPf4 zifdzHNj*kfQt935n+=UKM<w6$Ia5esf|d;W#0MED4_{i$@@DvMwF`#C0wgx!$@F{U zbJr-Zhsted<af_Cwmdh{h%GGKPD0Hs%`qKxu2Z?$p9l;EBlK?gyVb`{E}1;8QQ4zx zp|ezU)?<hwoVclyErm_~H6~R}YOTCQ<;C@pu%RZl&j)8-hq>ZT#=TDqyfA58=dG#a z*sML&I*f}fpAucNV^-vO0*PV3b)F{kwCd_kNhy-dy>`&YnD}KcD%ySSsB6MRoDOa7 zme;}M?@D-m>oz+2lds50i>s52a)3T!M;gM-RX*DvZfdfxBVkA?eEK6m61qwXQ_Hk8 zdMFtq!ev}h7K0Kz7IZt2DSvZpc*<dyXx3+bBafNcv>JsjiZ0)FQBhVO-%$H%(B0}P zYCtfzjX%W*Y;uGgjhaDu2#4uO>nKjJ<uameY?x;x#L(TlYR;sBBrfZea<sg#@+>Ol zO7OTv>dct`!d|$0esD%0=Dn5s2L#=pmagE(Rj85hUm$kZRT!1~5b$^}o?mu}(-1=k z(3}y>w)pJ8(7??T5V#STgvq$(8Go&oY<}5lI?8Iks$=2DsioPm2%Yz+(%ek)s2)Fy zLq$`dlR<UI8xpkD+Y@{d#sOVxJ#uJv;vJA(HS-5}9-9XL4eNn?Bc^30Eh~NLdqjIL zqH4`Ry0BK@8nqm&^0e~hDfcY)=a8b@*CIZ5hj2JgsO01hzV~P)*A%ICT?CtB(j*AG z<|Q<l8e18W8TPotzGYDiogTH+AA?R_n<1C?VYXBydqb61qBy1vo4Nf^9(#4FsOZfe zVHGhMD3Vc7wurqX*MR);XR4`kZp2wNI*kHP%pwM@FEG~1(W*zGOetVj5BO5EuU?GI z95xJRb<&Goh$&|&7QR1qb&;=#+EcErFVhgS_m;8!Z9563)uw|NxA=w^Qed^Lq~?6j zJG_<LXPMNQjk}X}y&id8f*a8fWhZb-0?HaT;{DgwV3_m!&r()g{r;AIYp8nu%Zq_L z2Mcuc!Sox(2zh0Vh$MIMHG-KV!TxF*opzx&d%0UHg)fPIBHfC}rX#tJG@g|I6RZOL z>(hkI@8d;n1wL`3O{Zif`}QO@mAsBCaaAu~5wKw0kT^2zdpG91{K)W^H*AKNJ&wmB zneMa7!E!kgUT1TirHoY_JbD+*S-f3-x+OQd;lL%jhO9k5S&2SlP949zMl1SI_j29w zYTUQhhCRm9?RpIEv#(zy&pX(wS#7kMt5-Wqq*(;p<pp<4%cmIBX3YAu4JMxRlnBcG zTF@0KpR(9gDy?GR>;@bcUNFNN4#9K?!bI4t92kAxi$dk7OZ?R4%Ia#;W7+3wI6M}U zvNGJs3p(v|>>nMq{CmXw95OtX^>HB7SIp6~eOgIbT}xEb_1saMVvZ%XlwEDd&)kN# z;GbzrZ0hr8_R<$#Q)k#-mab=D%%8G_80Sm7<z-Yd(pbD6bWk(!MnyNooZ-SuDdh}e z^W=o_AX2M7m=<$jc(5GWqYxwA%CN47sIc4P2cL;>CG2Qz+2-9<mam!cEE_XB_|d6y zyi2+{$0&kkJl-y5>l>XcQoEx%F}}5f@-VJBF+7N+#VB{(DN%NLk-K>yMCKXF>vx#R zh0AlSE6t8gGHFjWx#n7_=)MnaviYXh!L9Wz=dfTZC=cJ&Ybcjk5c%wJb?SxZYbU!i z)vkewBW1(Vf?>h?z8LhYTmMe1M-ZFyuFhJQe&q6jQWpMHU|>A{JZd3DKjm7Tz1A{P z&gWk;y}t1I2p=Z$P9H;tqAdg>P_HzT>>hmgUDLtF?(mL*uUMmtkMTpk*!;Y@vt7C7 zQFgozde@I<R_u}|bY<4?_EChHNB6AioRN%-)b})uY@aiWOKgZ?c)IVC7moGzkNbgu zes9{X^TmJX6mWF63LKvBV#UO`X;dO*{rQH=o9-{%Tmn{>Q;nE78D$|^@5OBwmOS)J z_o=@lJe;-rzmj%8%Np7$TF@W4l{zi6!yQs#1SBagB~-TA$lj?80?&9L5WlK-08;+P zY+Z%LU2>{Dy8g|_e@-Jdm<jipPwIv6TWL|MQSxiNiHabR_ul*t+Q3X?40SnBqQK(2 z2l7fek|VLleez+)InPDYir~#_=2<0Hzq7pH<u65r4Wn`P{dkgLx2MP&ALI&mju|cr z>?#T1L1}8Z5mmmI<&^8{+JBX5{u&=M;68V`A~PE4^K8ax*#a?Xo(k9WH>?ewQy#59 zEt$hs77`4g?Yq^?l+0T!T{;%l1k=R9RY3%oacRTL8E`AToBEHgF9vMRtNhS*#!Q7? z1(MU4owO%JfK6@!R@_pBw!0*L$?uNv$P=&FS%NLw?kCxNh7e6#uh;O`j`)x1vQPhW z(Arm`BPf!`z(`XzBaV83r^utUKc!}gG)rk!e0xuguO>S|P7C#SH1}7F2!U3B1alnH zEPUL=VT`;=?LuC~89UuJ>d{4?hmMK-*isZ6@j;A+Gw8^ct#Vc+Ud~wbp}^60bB*<_ z#=)w}VMZo$riRdj8=b5@_jJfuEuTYN6c?qTs(zExEMbJiU;3g3{N<8g`GW!gP4Gsf znhAS*rs3mKBA=<X-zkX4%(!>EmjWF&nvS(}?L6jt9o~${jXnK%<mtM!ys=yj?%<y@ z-VP7UN10PaxK?6CT^w)XPSuW%YEF={jZTj}98eaz=j{hIoUEp?-}+wd_wSKAaMOX| zW6|ELA1^bQKh*K*;2-c3U{zx09igp45B}sXjRK_TQIA*S#GROsiPQ5fkJx&Jrh*Pj zD=Xqy;UE2kTKBe^KfTKsZp);)3lBfC1tINFmt;u|KD6EbG37ef)72WExak+5%CMcX zN$!WMrEPD@wYy>sQu(P(Ho}XPEmOm$?W@mxTo<giS$`pRfD0z!yD2!EOQ%Fs{zT|^ z%E`QLp`gAcHj_gfJJGi{^PT+P>Qh*WZ;j(MT6NaqZgqDkjgw;Pd_5#I<eTz#$wbsu zu=RT9<G+{v{*ou-ae*FO_Fud5)Y$byRo^|9yu*iyN&Lo`VxfvYyR#Sz(?86YlRUKj zMr@q3L-EOV8a1MiL!o{F-M8gO{S>Z-iw!xwGo_Ue2xGdtyd}jHoiG;vPHf~%*>`z} z%|>r?nQQx|CoZV;DRL`m+pBIBY;XgJ<bzv<05uT_JzGN@JxxjumrvU&hnrTpbLZAu z1CNHg;pTzAaIi#)Zb;*#uw|2y0&P>JpskVn*XW(-*3u#2p@6+y$NXR``F~9u1M@~^ zOA)srVSRdqSl}WL-N)cPKG%-}sK@x<E~Dc9J7@ZrBC-Xz>RA@l4;Bk9O|iAda-)4~ z>!0y^@MmQ)D!WxxkN(h>R6j9XVOInGZQ~WR2}%F=D#GxlH1X)_fX3t{lesM?IN0;E zUm&O``R3qQcEO&_6=pSNCYpAMiN~DWWh0NYY|N%h$maKiW>+8SPEeO7;H;fLN!@bm zL*2N$aY@@G((x>=GB-~-P^U8k$K$o!f~5wt4>uBf^}~Fd_p1xK?P%{of1CFdVi?tN z|7OW|8MJx0!4W%jStDLTc0}kwfI_wySqAa{CO}+dQ$yJiC06&l-pAuVy~g`sA~<LI z0at@ED67GVu$~<_cu=+aTeXW^H^ek`%A_JkR>+$Ob}SHDXYHo2P0nvP{vTtK>IYxg z(|<5QX;pc$Zm|@9O4zBYQ1z|VSJnR3RYG<JCOHcd{F;V?8V#Yc!qT9sK9d`e`OK1< zz8~o=bo#GAbr&><M5$SC3fGks8o7;_2%6Oq@hmN|S0s##A5jeR5NNS&K4@I)UnFlu zAa73^9T3D1m}hk^p_Y<#O2or`Vb}sZa9AG{AvWnwYfhZ9Ro7=VVtr1jZxz=RzBe=F zS7oFqEV||%dktG%{d?ERA2|?-0q<>(0gfrJdurs9%y|X6AwA?f0XHkx-cq!;i&bR+ z-Tb%8zg+XIpPDcHDHxzfg<55)%f?Wc%duwE;Vfc33C$CTjd#53m=~!~PsjlJw$Ysn z1H=ZXv^nveC#B+|f(2!j-Y@c4N72e@HyC0kD2)=0o~VwbK5F?Y<20b#TfO>3c1feu z;``AsAtw!w!*F<wkk*M|4yDMZkmI&rlYno`+~o)3s|7WcH-LHLte%&b-Eu%EASO{d z=rpac2V-k8{PI@IW=y|adazLo-9%B$$ICY$!XmPi4PiAcIBdNOMs2ID)y3Xx*8f`S z3rfdFoM}n-Rh5TCW(FwE;imt9<_)>!u24cMTnRN6XEh}L-2|CHjcVEk8_y$4)WzE5 zO-H%<$u91jJ6Y$R+@xS!E2zUDzvdx+mh=I+Eb7NwD@KA!g3d##z2EH@>S^h-wbTw% z7Vz@J_FHb&5LBdVzkEfqy1`ZbF77ng$$QJbE8`x<pae6=!-{(t&m`cph^vq_az>!F zGFF|eO~R|brDnVKyE$w785TpLY3TgEAlz|C3$#E)R;Zrh=M8H#G?;C^O<5~}u9C|X z>rl3t1pZhOnPKVC2qF+H{M=Y77zpeC2D%Nf3HHaga8*2W%<kW6`*9wY=fNdC&#qjq z7%ZP;@|kMjN|2lyp6~x6WHJz&FF{#zvQCOlo0av)J5b4{O*J+>t2`02_0<RTf5x8x z^ZXTuaW*IE=}<YU<|o<m*O1+Q9_wGB;z^E=;8LFApV|w-!cV6g(2Bz1w_=X8H1^Y8 zw$^uyXkp3S3*=$JH|WbZFGcV~!1miH1UKDZUisl_qZ7S4jlXE)em9-tFlmbxTD|U9 zC31YHg|5D-!1WAy#EntGN9erM$|02+)b}>BxDEe~Ytc2%X7Of?@r!Ft{h@xy6@nLs z>FiLEbc1ziNuTN!!Q9;{7AETZqWL>MrLR@Q*J&tatBBYut+rHY3Yl=7Jx0nDhwLSl zTPyR_W>HQc(iKj*24m4u=RsL3-E3VH4|IUK(cX(vGKo0cnKmo6*bytjXo&x86~b8I zS|k?ZsffpRp|5v01}Y|i9t^+^Y(Kys)o84U3W*OHk(-ErgVKsYeo(n63RX<G;6J*F zt0w`uenNWpO>)BZ(V$COru%!?hNu5&@ho<Ax6)LZS(7(NSh~AE!29<$>?bX&y!<}u zI*kY>SHB|0HX_W*G|&E6<X{<y2VTCdAGAgUs)yzq7is2pJ*Zv2I7KbDRpa=gG^yc; z3Ww9ZI(R4D>){q61vR7^CvYt<ITe^b?Zg4UrP^0>NxPS#iI+7J+uHQ&z4fzpo^tBw z;^UImXY=_R4!;O+@h&W?3kb}WD$OJH+6(u#l)uU`o`k4AH*$l8{C-1-E8VhnI*!Mv zHQv5+tZD}IIv{l9Ft#x$!$6Fniy+CN<a(vMN|Z_zfxF2IY|~etpV(#=-d$=!c0Lh@ z`7^|LYPTJyB{rO%j5drIivnUZ0-ogObl@7|vM;au3>@$!bP#QdsotU3uHd{Bnbm_( zd4+q&O;5%XeDRM?w?9fQZN^OxlgT<M+ZCgo-Atzo_Ab+^O)59UFPfgb&6c1D*+X6@ z(hY$>jqLCOk;{Yf5?wPhHy+Wir_@2lCFPDOrMc9(xf{#t>y7WWr(G>64yV_zzU<+h z+dhB=uD+&BG<I@2KQO?VCHnxIy!_R0_P4%*RAN}o7>zyQtu%!2k$S`jSy-{xr*UNJ zYj)aV<~$IzT*d}1TkO!MV~r2!0uiN5)GQYT5CLL?Co0S~IWL;_VP^hMoN}#h=;_SZ zjxL!2=DBi&PnHvF!B^K?$HCUM$5Q@q<jTu1$b3^}iAYlW$D<h@t%nF@jou^J_L+J$ zRl}RtCy-xMKE~C4s}ou42<*bIP%^MiLsFJnXx5}Z<BCo9<qv}_=yeKE?DASR(3VYm zn_C=+ZeR1)4fzsKZ&~xMnswyk@ZMF&fK%GCDR2AM>q-xSo5pGjL{}zG#Hil)*=F;{ z5$%K@Qs2cUq;3Sy$B!RL{Nmkn@u}k+wx;DK##v)WT+krI{+-5;x1DWxj(-Uiz+Yun zaq$)ZK*&@mHdE~G9*V$rs+<Mk4uV;&<DNp%RUx1ip#Or+tUP$Y+pDpRISqZ5qHV39 zqV8(htJXU7_<mMZ{hFgnVKTBBC9+wvHQYkS#x#*hZ5+&Lbqc>FDER`8a@<TUuh+Wp zU=qnUKfgG)wnO?cE+(jC_r+@=ckgI&!(ORdGlsgl=kyoCjf{=uHib6#QZg%U+%;ZV zUPa1KOBDZd>OXtfez)@#H_sNBUNr{u0ec@y618btMKDl<Zab(*@pGP=xSm&Gf+Q~n zc-CniPbm|}Fj8pmK(b{#I%1zMJ2=qgqOb8&ug`4c<0wy8oTwdIL8~(OA3C<(h?7F; z&XP;|TxKsO6P)-k(l#|7n^~?z$;06<h7Rh8C4t4u<z`acm0RO_KmBHVa&gZvHB&_E zYmXDX)qqp-YCue3m5CeS^{cShST#wPY0Jtl1_a)r!x<kbT)nQt<gSE@1<Pd_IHaW* zraw=c3wX@p2lp<dAbwytp`g$9XuJ2yJZdRrTrqiAG>{Y<IZ{B|=}6FzQL6cRE;_62 z553J_%c6zm@ddf^@((ENbU+;r#^*~$#dGwEP}TD8+t#<#EimWyHTNvsCMpHlw#zjR z1_7okW($laQ;X?^{DQ>>Nx`T)GT(N}@8YUtSfvZ0eLv!AwQ_>2_QG)K{NpdVXOId? zgEy4`#eq_}=v2GMakGS(rzX_QI8d7h-3d2ll1&UinUlxgUAC%u$wNKoobk5&KCY6Y zqKp#g(KYP3dsbOpF(_CVb1s;Z1IopfYk=0Mal_v{U6aVn%u(zyx3HMv(X(o*U_u`q zg@lE@t06tivQ-BAR^)fP!z7-4kra+`4!x_}OGvpMxM9zd!<N|)OpagitHA(-{y!TG zK&ozJ47QsmaZi-y8RUmI=E*51fYl0|>@PWQ;$kqFUAa=>ie=6z1Y;3xPNN=_U*ZWr znsz<xGP-cj?8#GPRq%W_UX+G|cU@rPvQA?qZ*D)D>(yY}E+k=ZgX^`btTdl$*1Ia# zIs34_1i#}tLldd@MgW6_wABtPUg%k!tV6t_Z=O|GRPyfA+TEC-&E9fzQZ%$KJC>_Z z^k8xYI;2-)5clfa&2I%phR$`&@?(<WWN|8&NoI0(zTx`I2+N-zk=;^FcJ+M%J;>p~ zX~7ALCon^rWvO(kJ8}$s$}!++zEVhs%U8^*x%viiHIVFeG({|JOX|LWRenV<3;RJ; zh7fuCM%QIER01xDh-|L|we5+!J1Q`c+OKOXEzu`RlR&v((#;fX(Cj#UgQ}rv{rt!p zTUL)S`-DFL!c`znB0+w1Qqv7doG2B9dHbq92pCFnE+Sx#ecn1mBeJx>u59_TTZ&@C z?5r>|nmVVLM1jO>8Hfh9pZS+)+#pDC>UF@G8rU$Y{P@)U7Qwo!$=Ap6wUT>lh%dRE zvE}!04+ShwScDX7+S%kq*jQM;!<!^3_At~C0)pi1Y)Zo(Pk*+axTefne6G+lXWpE7 z_E=1Qioi!t5)<D`z|*2_nXd&Ac8~B!B5C64x2P><YV$v9Ke)h51w0PbRMWMhj<x3| z%?}2snr-LwW+UZn$<G_$1ynkZk^9@tQG+PxuVw=jnkQgozIW!8IicK^{-b5vixFoL zOSNMKJ;Cph!U`YAKC6#4HjbYgr|zmy^`!B_^gw4<6)0g_U#JPU%{LOQvR4F3HOVh% z^Nwz6`y`wVEt-2s>=dNH)GEtQBrT$uiz~n3HG)5omTo63eU~qFF3?Zxa^l=gdBVu- zu$bgb`Fu`s+S&~<^$X-GL3iBs4$6*twsNW*X=j*m52si-PxjcBl$OG%0=N&zHUU`6 zb`e&uS{~jO54Uk?<|L~!ja0q?dq(Xj169xY#-_s4%?VrKiMgMpQncMTK>|CjwrwqK zcG=RXehK^o<pd>J4E19dr4-MtX6S1Bw>ep^5O4Q<#L1K+sK&#olt53cJpZR)&=Xdw z$U9-^XS^>~lw-VXSE9SK@O6`L`gWEFbagqb?0hPlLguP5f%MlhgCz4TzEu?r$ORn@ zFUsgJw?=;pwyH!5b}M-L<wZLXzX}*+3nCr74a>EEaY)s^`4wK<N_miS{0NkVK|>am zz52F>7ZGQWO+G*%KTA#y=uC5EUM5`HvYChZ*=SJ_IGp&fwnv6rp3LEHz(BJaF>KPQ zaqQzx&lxx06=TM3PuFK@5Ng^)Q9U_M4>VfjyXEOz{dXh=uQlRL<0o0-+Y=Lq_#fbS zmwYQ+Fn8*Xjs)*()?JS6$~*VU0F>+`@GA<w8x(xjPZ48%H}*giuiEgnmNb**Y@ybe z^244N8P>&ageWyjx?+I~-3u$Xf^7oc1G`gjO^pPsyuH_c%1>uC=px3;51b>sh%%S# z^_zU_91{CfFemug-K)``HjEwb;2NSq*5IBzKp0m;cZei!)Gv3)@Kn7$p~p#!i*Jg~ zjC*FSZ0N4j<rDE$TyRB7;qGJI@WWER)cUZ1yp?@-41<Sdo{v@jAud&&Z<2f40zew; z(`}ih3dq<C&zz9z{`VQ2GDsN_;up7U_xV{{m+C$nl;Vyu<2n;iUm-duTZK?Cn)LAv zqz_XI=G4@6gh7^42S=5M9~HiE%(OqkogD7fsBTUntC2=CTA52rQZ}BkoiQV{81}@A zb>*EOE9?LQ6$jM-O^-+)l&0L~IT?R3R8ovocPVf1&JnkSf1igiY-lJDak0e_q;|04 zWKlcW4C3<C?)%clR5<#<#Vq4@Vm}T#_ay4FKGf!l(5{`dEMcW3bx#_uSP=4Q%QR%d zjb&U@pyi~P_9mORrj07SS<88Cq3KDKMuOOsGDDqZMHcQtG4g}b=&SFyVZxEbJ-I3e zdHXC?>EnZR1K&PgM|gZ%*xns7_JP}Uo#0LMJ|%ZK_cLC420)>9wfi793snHlq-kwB zl52ERFcXtS#`46XhbW-lg7YP_r3M5!Mb^N?#L}vT=@Ln*O;xLPurj&`DmWowaem`f zE})prf6{ikcly}zResj|IcbP$ZtcO*arpu5$wqyX)^Up#U6T{FA>Hbm3*lmKQ*qUK zQ=+a6dTMT;y+7pXRjiOmSBa@p9X(8XN0D7X@5t5#vbV|npHg?4?!^b`=?JdA2?oNo zP7K>t-q$4_);W38P$t9HZp}r>4K~=cZ{}Cx<!7nxV9u8D6`KyQ9ducGEkH6t^3ZX! zkw9tf^{t{Rn$(^#$||&c@+#yS>|W+c$x`8U_T+-V%e&VIVWt<YSl@=_;mHJF9=T9Y zC}*dy(jT@3!Dvp3fZSwY>t=w*&{KJv6)8V4kM3A0Qmy>JiqoT|lS+LaiwB<BvKNbA zQ4wz0atDqe3=j;R5?5iXj;Wex1-NI+tD;ki(=hZc`2dn9)NcX|9a?5rJ)lPR4u}XD zMHj>BM~)lU{dKosjh>H(jC@sh!mJt?J8W#$9@$xK4DhbJsyB!77SA_CF3{2^?~$y> z&5<y+Z7@A%#2p!`s%pV;HqjVBP(@lNYG~Mk;1-P=8~4-EsY5HuD*c+vs*Ms&E>RTJ zeR*v!;t_qS6~-RmQ}pER*aJm>N_^7)hpDr0i!$ogzKDP*B_SP3N_U5d(n@#d(A|v) zNSAbXcXxLT-7qwgLk}>-_xQf&oa_4&X0E;Vv(~+S_q~-_lg^&QPmO^~^e!LHR#7ew zoG@dNUjggWt+BK2`o5tD&<d^$vF^{2($&7J_cW?!6>bV<v+ke~k?!_{)W7L9CCS2V zayu>2M(aa_W<MB?J)6|DZ*z_@E!vHq(JqJf0mholZR|h(kQ;@SD!VE&_gNn;zBj(o zTSNOr?rjL>>6mPs3*_~Kzd?py&@SwwK)U%a9*>@(v_<8%^0q(Rvh53Ai_q~DI=%<o z7=v+zd=?1?etrako=-2I?x9Z{?6nzNpyvmcB7h{Dj(w(KCh-qTsT7$Q01pzE*4I0$ zECronu9@PS#w^8zhIX@$F-;f@@%_%bfx8(>1vqwxl#g%w1~t~R5!BQ)gaaZiu(bM{ zdD{OAL{glNN??ykObS&Y*-av;8z<cLM<|~hbhf3RtZj}sDJM0$9ED8o_rVAJ!i-$r zly84`TYM5q>oCVli@ZrozmMon=%;q}iqd2ma=D7IMqPRT2Ai*8&!%j@tpiELMNV<w znrPq1-I<UYG0=?Mvj1V)s}Gj_x&|P^)?<Dldy>x2FMJf{nQx4H^?cK_km15MWSRUk z-0}@Srn=)X9c!QwOjdxb4x)@uuKsJ^%h(1>BSD$#rJ%N7sDG*DH6+J&9y@-+i`X#w z6IXH#HLed7;O>pDsElSguz4s~H3~AcY8ChuvlLyBUgHJy?GgU;oC}Q2emNVZVq`;o zF?QgZ-nI^+Bc)fiqzL@NcBOJxNPmVOqK*@=QrU18oTHa*-}b(AMdXhyqy#_z@!M_1 zsjRYfcj}IHsOUTh$s25+eaPUYc2}=g)FCDciTf>D-;m{s3jm<EWe<p)gpWDf^YAn- zMQ+94Io<oM>^p#<!<w`mDq5kFIPDJU6n6XlptYviy?tXmS7ARX*CZ^%*nO<6Lb*G1 zFuK|v2_W9t`Wt|JF}kT0W)rd9_yIY2J+gML)XGHEIlytsysmypdY!Cs;x`gOldWqV z#Y$|&d{#jghVd^W)@Rc-JSREVqWq*+B33K_q)s2!PuovUQ3*{RnRb*H0Q8Rk@5{@Z zwbhzZYq9jRSJA-rNByUeA-GKgeqlV&*K0_PkGspuEB}-ehT}*}kE#q#blRe9<KUQC zTlb!w{q`6;aoGmwjUh$Dv*0B^v}v~ZzwK~TNlDqD6*us&V5xtb-l@sOyq}btueE2N zK3{HXQTPqBbmsNxQxp-pYe81Kj<*;yG}sNC)x#H<jm(`<(<5DQ(c3=n8<%>>A<K&C z5yz#TdwY)xI}v22@~|uEq7<|%0SFYYXLZ}i#>IU0;BMA92eh9aRP}4A&~k;7-{Vq7 zcQK8!D%{n`*pwiwG)x*dNSMcl2l-V(;-t`gV3@ecU2v>k&mPf0$(xBmsk^XlCV3Wl zw3?@T(Yg`S`2phlz;B`5*F}f#mwq;WUP_1E&&qDak!a16mNC4&&&+jfLl@2^+tp9B zyXJxTI!zSx>F2hSl@i-)!sOo&kk02KKlgMm0DHrx+IGZB1=XevR|~Z-vm9k^y@DI3 z{k`)7XBFeYr;(^`6DuU+^e4oi(O&`Gv8B%Vv#-zF*$<oA%-P(^&xyH+!`Yc|eQc4v zlV~bk3$*g-!n}2hv*_TbCraa@2rxUtXr64elAAH7O)0sQ^OX5D@4|?-j>5|Rji-wv zEf_sp-w}nPANPFU#xGl3khF-(!33`o<%?1K;M{Ndgx-X-2f^zJ1Ut<I>KipU+NtM0 zN8kTM{bHd!ACc;#SlBG9i~rbu^bT0xsIO)<@#?0xUpvTzbUVr9*n3x>2rNT<fI*R9 z%pL&Cxq!upych^ntz(&P*3*AqsXTJbGnL8v>ET$ZLkjn1m(cs&sEsslX#W1=lEsNu zXm74=>gK<Rs#dKDnP=g67wx)zf3ST=+{Tt0+0?VT8b{~(&brHA46gxO5IU-Q^tJ{% zYN@<NO!!zndc7}i!w~uQ!z0Z`Gpwv>PV*RCmTX2)DsCf}poxE#brlS>W>|KB69Qne z{&kmhf;?q<9i?|fy^aC&!gg2zr#@Z~Cl#jicPxa~PG_q96u$db9n%+Y9=0-l10h~# zA9Q!{b2_r$0sP*W=bk~e`uW|G?K$xLboQ8=F2EZ<Z9|?wbq&r%ewr_kx%rG(Hw``l zHh~gkPcyeyK&DRrZYfVV>>$J~?%<hGV&zs_wG8d`8T3*J(^Dw3cZ$SZ2UBIFY`h#@ z#lJN{r>I`7g!pBF4eznHp(#g4r|n!Je<)vjvW0<u!EcY%xH5iPw5P9^I&N)r>TcnB zflHN^+pk?looE%8ab_jc#wv`R_?A_~NbXnN)|8vI?6N0Q3NM=UiXV}^X8?%fe!=;T zcZZe4%FAoQg|1824woB%i!^IU)DKma7M(v)R2^S&SuxC|+>o1VdFBhY5hoqq(6xc! zs2A{8w>9A2kr0BfE8NeU7^Lcbk17}btiamTA-XW59REs@6GJ<Tb$?)l$kQVG<}FD0 zxf&exvHmfJ$eJXq>S1AVM1foTSMfKAzcxrSuhy(Pg7${_<KG|Ukw~D46FY6f?kk?q z?m^}6kXURRLXk5{JB&CXN8h_In%bIwhfm<5HNk({F!qLYzo;;wjTOLFwh*aUb6VaA zpGsS&JzS^J)==LX_qbjtBCqsGUk8a)(;x?jUp`ykX@pNiywYL7?N$nlYf^+oq}1aP zDMx+#ciyX$uYMG9j#T6m=4oQf^Zr4E9=6s_YyjtY6dUC?F447NV7a=EK-qKDTS%cn z4zk8PCs!-o^1a~te~-KVjCy-XEiL{4Y|emjumhdEqxc@Dtb)0$|14ZtR{&Tma2a`z zUG}!jwvo|`hXOtp-IQe*X;d#DAypvJN+Z9K=TbCX1=)X|!`yYXamQ$GPyXwuvE@*} zYLvU%+YhXo`+JsuT3jL;MnBus`vkZ7Tg~Zyb(~noOF}g<c6l9A4x5D5lx)NLVncP+ zFy|X@VD<m$djH3ZC2T?b{tZe%m90<iQ>+CJh76`fCVBVuJye<B`dxsn6q>`OM>`!O zeK(8mrGOW}Ff}=x&{RO$%BMqr{NUe6ai-+ye68p6$OOzyp`!~n8|%GPW5ks!#K#+f zd=}(lKB6($VZZ$A_7YTAn=}1VT96r8QaR{j5<3nFtjMe*MBi7TR+6GxH-@L?u+<aL z5VzHr<2ikA%y#=5sOd<fo23o(0rG$mPE#wtWs#RZlZVP%d>fZJV%FY`*b=RV(L;4# zUN9+ma4p1))23=MME2CpFS-A~yO?Y1oocMCwTggdphUT=f`DoWZMu08dAWiEFd=+) zW3*u;(?nK|es=!sxAmD7pIg;}b>Q8h_NH6Wi`yg49NIw2+KiimTu{qf24WO2vSAf( z`B6-ZlpZEW>)*Pp!*C@gWW5~U5=@QsC>425&2Zg<Lm3m7f}GyJtAih{ln8b<8Kqo* zDaUzXfad2SRJvO|E|Q}p@B)sbZooHPdxXjd!g4P~F7HZMX2Sy{Kb0@KGLp03{~d7E zQ@;#6jpVycZ>rEYT=5Pck3wE!B%?y+8)95ZiM*=#DtkX5=Gu9AmUIqZh*T*`md!4M zmX(T{t29jiuI)Gcj@BSMziqRP&g%t-&UsntuG%yeOk9iR-WUY}9E$s+a@3#PCJ{a0 z)(im4^WKsGWCk>Rh#xR@k#g{^1n2k*Uh%f(zVUG#tNSN`COJflR=7zmSRn%<BI@t% z^aXp|0drog5JEg&H_#%7A78xBS-i-&sa{1%)DkVKnoWG3c~jZE-5yaX@IdTEE9<zS z4HSn9(HeT)=TDc$#=8mFSNT0>4DFoegS;%6i3TlT`AtU8tc5IM`F46snjuS<*dYkH zrwoLQtoKLdhaGm61XL`@86UwZ=<roZ?Yi=o&8#T=M{u;Yd)ZdS6C=s+-xTxSlBvW{ zrzfDx{zpSD+GNfm*-dVCX|rH!+LU&u1OudZVZ62W%vW!1j2hN3KyaEvBhE!2s9#ag zPzfkW(Nr!i$5=}ss}G_Op_q|&(JCt{sZhxQ1L2shFYa{HOt^*Fif?e9(0!WMJAAHh z*9n8BdYm!c9i0~=7#PeWxAX@yE&s|3BN6e;`LppuErepvQ=-)dn9=U*cn^Wu7E`Pi z@lb%djAg^!cm62L?w0s5_kz2R0*s0%0$7NPs?|yqG~(Xk44t}Ltv2lybA9{B)pY)1 zlxWxK+KkU3;)iO#U9q%;Jez#ZOJ3RX53hPJ#~mg)b)y`#V(dAb?3!;-?f`;o16h(6 z2qxg8RO-vR1E?DJasbzgUQaJt_L~#yqd@!>#Uw?4tx|XLO)yss4SC(sa2kUD=1PH= z)neMsN?#u<ARvHV2p&GuTv=h*Dj*^fl}uQek}q?c!Mwm&Pzd^C_6DDqAN81cE>fAY z`R)U<c6)Yf6#8g3u&!=fFSNjQX6!+r0efU*Et#p&;r4VBCv9zdDKAR1weFy;c^gO4 zBY(05OfoU+caqmCn8X^*<fhu{gUur>vzi()x?re^fMUqSW=_*z$Eu8+<K`tNsdC?D z9#-dQ0%JZaSI!T$3`pok&hMfITSxM2zl59&ge+I_Y_}nwa_UN)>-+RMU|AYg4K-o9 zO{be`M?Jwu+gZn#auZq!GKhbNIg(1D)i9Zs@XzxFqyG|Me(J8Wber|TCyisG&l-+d zlI0ONUeD}c)#a-IWQNn!`W&aK$s&Oh36Jko>ah2i9iLU@_%jGk$bF;s9=A{dHRCu` zG#6u6KXam3^$)={(@{11b_xU8h#GrJ01v%{%vlr1!FAW^2BEp9)PbbD^L>wpljYTc zv7@KbwLwi;anhz=fY5%6bBV?r52eC9kF6rw1+P}2hx}q{A-j5DMg3p{ZR0a4zp3V& z?1Q=xOH2_NlM<mAoujL+HGtoX)jmpc+~*&xMNMv*$#3zeIO2m@Ye{J1sEDl17L;2L z8y(M3z`m)u@7@kPe$Bhr{WprX{dW|V(&<{hq1U7#j-G#dok5uwwda@EI=-N?7wAS} zZ1CPz1&};m!eKW4aepD?4`eZ~+aLx}-$w2GC<HV~#xL7=v>r+5tgKem)l0=kW9H+F z6ifOM+ZGpNye3_gkTzel(7Euol8~xviY)!Ud|@jFJV<roP?zn{<OHLM=c!IeW{y|= zLz%T*&(TvO_d!Qk`{!7Na$O{k!Ms2z|K!7+&q{<+vd1245;sr})g2nXsQ8xPuW4)V z{024tm*(ZsWaG7+W&5Oq3V4Z>C!rK>V!gz$>6n+!MxL7I3xs<iq7BJpQ7F#N*SA-- zV0lHwl4)p-aLtu^OiDubyurlK&{vlm4z5Z1I?P@KRg$#$o7<4X(ewJ+Q@&N-q&#Fh z`|oD8_ex=wZ!OBh`{v;QbP{|jj8HlI`fxh?rFQ8vjT1<=Yva4&ijEV%CS^tK;I@|S zYT7^3yvGNxmZ@AjfRNJAY9wOWHk{LluQ+U~y<6mQU;=Mg00OXiOAjneL$2;SXKB5U zsUD2fh_Jw#D1)?XydaF8prsMhEt-Jp*pW?$V&>#BEt4i_4bg2iJ<n!@GG_i<vb=V9 z$#C;G7hBR)e9i=jH)go5N&9D(Z374r9;r^1MzAA<wcs`O_lNk5TX_qZ14mcjl7Y-i z={kRexA%z)#Yi>cPmX7cn8~y|{>D3@NWj+n+q)oi(=a}rCI!wqzm3~UGB%X-5|lQ0 z<ozGa1I@_P;vg{H8m;L*dOi7f&J*zX%ROm3$NM_DhK+B&7-S9W1HBEvTCZIGlQ*U& zE!W`vtGR*8#Rk%Rt4?%+#!dco!UU?Ogf!92nZrZz^RBn*g{`?M-Zub4Pzjk2d5?S3 z{<*W(6|%plABi6zH%Bx?l0{y+PmK+R*T}O*=Ng7Q9A_IwpMNz~fp_e(1HhxIAwSu& zI;q{fQom$K3kKqmu>*x-emTK+qxvMRy}abJ>vmw{e?|gmeJCp@aV;;_^=f^Gn|Kl{ z`O0kjrQ|TRPP5kk@>I+DE?z=Z48?&%zvU0Wj_<4XLXdOCVxf9+=5hwxP!cZp&g=Iy z_yay}2H`!Iy)RH_*jT?~&kVj>c4qzvIWXfPy7)UE=??HWzLD?4!Fs=1rlj3eA9$Xu z?D#Yv#CyNZQ3OZMhT`7}(G}D(9uzgG=mSG614HRykipPeRBQPHtHGc4NsN8-AZI+p zv3POXW5tsNUZwL-1)t<Fi!z(!{_oJ!v)zVr)SI3!#umMI$7rmW)g?ePTAr$j@tVs> zQw4BMKEr4(oXmy!b^qG^j_%Goo@Y{<wU+XEds7i=1_6nD%TBLr{tMlm+^Q-9owYSl zZvvl;B@G_HBiU66c~)yO)uCVWbkIchc>;HeYq;kh?k6Lx=)hv%f<0@M=}Bs*GuU_) zIW!|h;vd<Z7PNvPOQ+SF;xpgs$sF^Rz6oLqap=$yd1b$BN2wP;u?H%^Z{wuElD^+f z^x2V%_mG$8TU$meXgOxmYlXb(_nD^f1cgb;Zj2`U!}O&g(!9|_TL`J^Y>Tv~2-LtZ zdcVQ6j3ilT*upz)=G8(+Hi4BhXW;4bZZelxxq1HuX{0)K0PY$r)i%ec?oAjm9Hr)r zjHR9O-FpzIkF%xvYY~Ch^&z!Y#)tU1J;M_XT=d;<Y2#jsbW6zwK*}<xl>?hEH~8Rf zaz->9g5>r!iV~T}g*&{ZR;iq?R8c@&gSN!;svarcCAt<_Zqz#+AW>fe#!$!a=p7z5 z<3YVga?WE%>9w1m4h3j50YpY|$^N5M;ES{aJqs(6S;s#$kK9=#&=d<LM9ewH7@>uk z$<-I3_m`6gW?_ddZgGaukH0;%xZaex6J(IC^2DWntkb7MDaVPRH+=`3d=lA(!KHCV zwY0erM?89D_l(>2MRL>3pUJN=T06besy(eO#CdBYUfmN)lgM%Bbn7DSf7z+$*alFR zwAwbuN%^{yjnto+|8C6xG(}ZR8`l?gw7nv!H?K|T>$hVL;~dUp@+=Sj!(#E(5P0<` z<C5jh=>1-vl2+L~35+0965P8sDSedq)MSgJOD9{b%&z|XVO^W+I@2yxVBBlaI5h<c z3*A-|tz_yHcxjy3aW8BCyYM-S_MtrG0Vtt>68YSabOaQs2|gKBti@dXeSTYG)uN^? zdDs;74ji2#N_QKO$Ovx#@%EHU4{_nBN&OKIdH)_0P4~?0e*mhgV{ry#zUB0z+U0%g zU}OotEsjDF#C*y-3+@~9e!y5ug&YA(*|zGwN2bcp+gc3_Yx0IiVh32sRO)sHvOIge zuXjeLXQF%Fn|G<2mAC%si4lbzcfKi+sc+}@I`CWmZRgV=^h~f5OSckSCrktZ%?kY_ zv7`wN&PZ)o<XI>?PB9-Of8&#TuVn=>%Gc|Mu5z#cL@94%-K-5hT@TDmT$l>$D5MuM zFk}5&M52P`P{%#<Yi^N#mhororWbD_Xx3^yca=op(Bb>u)lHrl=!M+vfPLqQsdCTd z_wFpM#$23(bj&%|Rjo)2oB%-^p^6{W8di?I*{u-sq`kusY89Q}m<=L%jRej`!Of`e zWxque6ecp@^M4T?4uZ!M;Vaw~Q|W8z{(Hm7)E<WE@@zx(A=CV1CHc4);rzj)pOTnr zlSB39-dPW~9yQuwneLhur!3u6<l*lByZMzz@nq+7^(kOoqIiS;Fcwu1&uQaopK8*@ zJ@ZFEk5sGd&$8{eTm=l297u^`rlMz{8>YzR&JC;bpWTAbWAg+;LC3!G4|r&M$Enxd z&OCgdJH2zLZTcvn`Asxj+t%BcM(}&1<3wDd$Nhn7qIyBuvW!GK)|=#|^iU7aahp^u zdSq1a2b5?Bjm*9qPxU2!9YTVac4v_zW88BM0+z9Z1)eQR{ZX`^o+!0PtN#_koauuR z;fDT)b>eT1-W^i6OaI%}yjXcT&(dc4W#y-&;L*4|#~(s?loM0?(LVxyvoEV;`9|5{ zR|KzwI4v=0t?<IryDhH@+wtJNb7a9&S=BI;0rs*o!pY68b3C#9)+i-m`#$NTB;Q#@ zPC#xOg7#66;D0Vad&AkpSV_{IJn4lJCoYCpS!}~=@w52L#eMBuQJEjX6;w;%s6gG6 zA45kBTGwSczezl(G-{MrCnWy~+m>7V%T(9f+-)KJ`zswcrvGATfDIow<c0VJvt|=) zxfMHTIRW|tNg}ssCcQ|(1%JI0|MfG!|G7Uz-%eb3p0U1bwDGP428biD?~~($-1Dh# zx2^9pN*a7GC*^U_83on@7Gp~S)>C?qd%&BHLIeLS_<xAOyZfBAz5_RZx=Ti~*=5I~ z0#Ak67`0q7zlVqxlo&mS?w-@}Ahm3c<r-RkaPX{APAQc<t7U)Phf9Zcmci*ft<XpD zJipV!n*kG6s3l15xRUK3yv^Hbv_O6bM+@$*|9A<{_)vOuo=r%}yRzXjiWmQbm2(Ks zvdpS1%e`*uR`&poa!vXXd3d&m>hO5ZVsF*FW!S1_VY~RcTDqJtB<S42Sc`p<+`xKD z9pK7x(dfN2?iCg}bAEXr3mz%WMyePX(cXYKeuau2$?{C7`BqHegNEjpOJ@0XOy`qj z@w`h9*a7y;1)8sJtGqNS&;AD!Ttir#2N4!D9DEmGv2iB(0m!CuMadU`yCSe&s8{@q zJgRlxc)BPB2sg5y(L56Nlr_I&u!g!crx*Y?ly`SMt5Lnswk&9_vM!!Zv;&_xS&>`) z*IxC2cXiYD8PL9<P&%RukBBbxJSC^#2FEuj;)xlP61RvX9AsEXsJimE6!5!%C3x8+ zc{hR}m%5kVxy-1Yo?9`F&JbK#E>IjMp{CMH<Regkd6rTt9mTfiHYp1g#dy8dMek#? zvWgYE0;MdcL}mD}U{luP@Xyj~)hKuPa`8P?{hFHjvc^E<G-gSoT5-K=^PV}U^;f}v z{v6;RS{HQq8fH0#y1y0vCQ_akNQZ+EoO0@^Tzp!kd(C|;c8YLK@_t8M7`aWpNuFPV zr{Tv#G|lhaM}Y|O4mV!CkV&(b!obze{&)W97!X$@J;YrYFrPZjR*QtH^Ky+!uCH@e z&yhN{0Hs(86uE`LX$UYa#DVzj;7P$Lw-TlM6O$QU8qyi2rWp@BqYQ;PgW;3$K8{fw zXKRU{gR1}6(|EA`mFIwHJd-E5B%eqImP_VG+cW<FdTlPiRd(4&vFXW7;9UtvjdJOy zfPa#!$=uw^XK?%TCtRx$x~10nNUEqb6R@n&!3_j2vOM?t_B~%_J|T-d7s6OMwmI8> zY-iz0(^cGUX|Qw9jVW6QQ`HN1HHWe&)Bk-3&NAn~mU|O*vK54xXbCQMzQIWS&?`zO zTfZnYW54A`2KrS|LCTDbvVT9jY-qHuz%}BqDo|i5Vsf3A?OnrlMGZfzRB%I`%iR?h zIoKffxXC4JSPh&r=R1da;G8wtaXV&QCu^pDx}u30baHe&`phPYI`YTo`sPsVsLwUz zeu@Ynt_-`Z(bwu_I_=53>m58y0WZO)q|>J?2YA6od$_GP8HoPrHB|FfG>YUvmq#~n zUmOm)S@z=B7XWipf9}^0!EV-O_eaETihO6{!U?nY+^py5U@bW`T$sK(g)26wMa5@N zHf1_7@kZe`g`f}+>$iIR@pK1Yq$$`YWUs0=*`9SvW!G(nL&FBn_fjfXDmmb7qX7FJ zWD;d)=V0?PN)uYNG!bla!5dzjFJ-KuI5(CbhoL%+z7TupP*>{0<8;HlP->J}>(ul) z|Cg}2_a#sj+n8fpUCEIK{aN;87UB$(>{G8@(?Ys|Sh1Ze;~el@EirKf6*B6!awBnN zj!}t;p6{_v@L(?3q(96bM38cX8A{D@V&Gy+ltObfd;Dy-%dgtM5fGUQKsUPOeBPDk zL7Ihv%_si8?PIJE%K6uQ%D_j!|KvtW=LIyICBZkMVVH*N^$8u|WSYY!BZ;=0zo0n> zyhJzveE0AJ&50Fh2H4@=cHF%=ZCw8<uEt&>J`(YA$!e8R_}iBiCOkn@*dsyJmSnVL z1f)n|$8;_Hz%R4jvOp}PyPfBX1YrcJ+}&twLy;H3Lm6;v7=xOvOHk)1O_I(ck+{SK zLCx(;7HreI3rC*@aL_u~xv(x79k2M8{y{uPgBE9tbd`v;-UVw(0}O39biAQKx<$(B zB=$oXbPJ!QY`)7oh)iD(TWbAZkpQ4yvV2;4{92s<0cD=PK3rZzA;Lk64e;Otm@)bj zf3G&1<b&baCZJwG0K0Qf+d`=+>I8545DQ%oROCTDr0cVH|Nc4b-^Q)A^SN4D=)OO5 z0`OHI9XfB`vmmIu@Qwx}c>9<NXA=8jt}U{pUGPTFqzC@Y5Nz7hIjo-CPj}MN*6mq5 zCg7)8wy(ZNTOQO5huU6AsExsiQLK+FiIdVY{k&~(J9d%E$}0i}O{axIE>-lp*r$*n z2${iyJtxk@k3+b*dW!xUkmKqoOwgNqueV95mtO@j58cI^O9(5J1Ayqsch3+uT>%b} zh{NnRf(NpLJ?!ucxZcsTfMt?i+F)|5+@yYKpzSdJohRYS^a%E}mXbWT{IBA>Vk6&5 zp8E5oh}OaA<%HkM*+kFRcOK+X{nOLzZR<$R;k~59!rZ&bU#{(%Q@$VQTn!w%ayhQ3 zuY5I-6Rr{9m=sxqwH%{}1V^Fs!_S*2k*>v)4IUdB$If`iiLSAr)M9$@0-glKR!>|# zHLo{?wx$pt9~;Q@%R}*!nCU01jBntM{l4V^_M?<^70CFB<F@oQk}}30BO3z@mjxhv zFC*8gXsGEFpHg>JV(r!=;W09Vd)2zF!Z^@lm3Q#1Ojr?NvGNgUN$19O?>-5}{$x;h zFe)-jxY@w94Od)MMoTC`6BHyXqZG?@jU7L5-g4RIQfVx`7Z|xLa@7jqr}W2Q8xlsJ zsdZd|FA?x=gEBH<MDV>$Htw79FDM~Z<O598V<4Z5`s-ZSQ6!=W3p{+zYGoyE&iiDP zZj0U6#T+?VXV`)QzCvb(hdMAxbX;$NtYan^P~80qneQl+eK&9S1fW0$C^BH65`sui z_s-jGKgxnFD?(f4Bfwo5Wpd$B8*%b!pgp(MK0ct2BC!p)^1qaqe>5G<WOf6ua;;Tt z4M#=~<NsM*ns?%iR4GLW-u$HrP_FUiCaX5hp)&RR(W~d5ncTl>j5$@4OggBOe0cxA zxHzpd0&YMe{X{7jZ*$*}f<C|LeEmtJX_-yMul9s>=YR2W)W{K`U4RS5z?aO1)2HyF z5)+JG!o|@f-g~lF%ByyrjE(k(J?7U-iHF5K6#L4Pmd?xfo9@*wB=oty&lWrT-|L8k zBNqs6_FQO_B&83Wq{rzD?R*atEwC*ot^68aZl#uiq+b~RiRzOFrZgpC*qLUER<cTj z)xOh_4Xt}6fpPC>%rEWBznW`Ef6(!ZL(l7a9LBs(n&%m}Q%DpSK1StQ+9N}Fv;$2F zW|}^|W|6xU{4be~nXqQoM{)G7BKLYMHhbPks0>}n(5^;&Ty~@7bs?T_dbS$cMZ5dS z)y=h$zMW({W`JhIc=;`mW)n<UXvt&O4>WwYXA0~-m))H?xwUV<k2BpKh&<I|hj>*= zR!tbCk}j%aI;_Ce^;eeGyPH60EFKOD0#>VI756j9LBcF4)<%{%!nwW)(>L-+C;N$l zhtMEl@PjqVL?EHy@hurogpUDOInkbkySlmF|0!8GSEwWGQ)cgNdad+F?+ntivdW^= zM++^!y@b6VPF@818rsJ6OK1E3!~xo}=z4-lm5Q9_P*%6DPRP+Qo6REB$Yu)t8MO9H z=qAf2c3InlIdx#xzdW$)KL~^hh^j{k;ILU;;mKXJU)c&ArbU3GwsdFEuXq{Z8NE*; z0n2TJsB6H>^rMZ3V(IN~KBc3cF?)W$t9ZJs&5g9zf9pN5vZ{_LkG7|g(S2H7T!VX0 z;<J}uPaQvyk<72kqx!QWnBPU5bijz@ewnBBr0B@WX(H(DoSxx&6Uys$<lE*2o`t#3 z!@u@c)LAF)o?s#Q<K%9;9@WID#01U}8;aKpRpS_?5TQ}B?TnjdBCKIYT<bf<>)sOI zX0!t5`%NO&)9eWTYUego6U2V$=x^}G9_vFQ1>czm7)E2Yuu1B;3%4R1e^HlcVPLjR zY`9h2Go4-{8_c)V5U_hqy>nx--Mm|DPli8d1kc15(4um8Q676A03_*A6$%gd!sp<( z<x}|5V7D59p%L?|<Y^A}^FcY`URz&S_LSPakPmEguxd$M<#EvR=$p?6AT$F_`2JQY zFCWyknY0{X%coOIuLJ*=IU@|e01tU8=wG$o$Yh#rc%HX9*u6v18?SsdIv&?$!u13M zN*@VS97|t5$e5OETt<z}2#)s(I)PI%GAd3FPE#sQ>FZ}>rRn3PN0bYuS>uwg$S6=_ zUu?G1wn=nRzC?hwyHX*<n%|^BBGF7Y2GDJ>UYwhMEAD&!b=C8f4}M(R+1{BhbW3Ft z_nXf(+E{MoPc_%hd%1~El65V`(w~TiRr6!b1}T&KD2CViPpyj6=@QG;o{VYykFs|W z%-MzGX?8Oa*!#ow5#{oGbQl?gzpAks+HKFJ)Qxc$Ri=zs;Do$z#NTDVF)8Wx1}~QN z<8P}R%1q^gPao1l`!-I`BL&&^!!D*i#?uy*5A{5y2_CmH8P%<aeuzwCWR+w2-QDN! zP5exu7+v-w_8#Qyn_yo{`zuxQ^AXzb?ux*hDmZoqk2Y3-!pp&pxhaNn3Xi`{3<0I> z^!>m63*Nj%3H3p5XV9UiCbFs>aL*o;d6F!+6J`g!^u4JQbvC}VYF9<O=PukQdyQ6j zy^g!TtS|d`8~Ytik$IK(@=J&)w~F3|SKS162`<W#ItWz|$0?Qr1&F=$!|&*$5Qh%& zW+-WGdt}BN<xBf$kRD<p4dqe{i;^Q5_qBZ{y3N>5;e9VKtI&wpaP*TvmW27=uW2SG zW_z8z&eA_27*AO%kaYV~$Yy9Ud7v1dy_xSxlwYsQh+|6zZR?A|I|ry}%W9LoQYW!* zppSoI$CIk`2z_34^EaFIW?Hoh2q)zEz8%u<i<7lw>w8`bu`v1ILn>O``MhhJpI@$1 zNsU$3p`C?)iFjA1s1_4m8QBgwD$fktMdC$Mj{H-s&5hl7{)FDTFttKioLLgH-y20e zdfe{um&=C~TjUyNPLuFwUi>xmM~C3tgUUAbbWUFut;XQ9W4E4Wzxf_kWXH7-Gz_G0 zz^t(QqEdM)>Y-!l?YR!eJ^9dBLLqt9wXORm16I+k&K=7*0gD?|h7X!U`JPDcfNCC} zO|iBNCoKLi{zBzr9n5ssW?v;)o%j-tX7t9spFj&T17+pCKb4%L%fJ*?b>NxzG!eFK zU;PtvZ7}|mTK;Q4S3K4{IAeMAq0p&Geb^M6^yD=8+(bBrQ_7c*-O+m=Eo=ErtN!DP zaYqx){09rV1%XZvmPY7vva9B}UFT7dM9#!stKu%wYpht-&(5Ott+K#Z)vU6We4OJ0 zVU2T8?dK2@k`olYQG>Qy&yLhfym)?*m6*(zcSknpX0gAph6egBa^&b<aSgvMGe3fd zxNR18JRXqfdqew1A5d17c{OMeW!fzS9))rG-(rPrFuGQ57tqvW2UPhm$Lm&q%;=*~ zxmjvPx&Zs$husT<YaMqRJD+pW&@hV0fsN-U-^Z=khf#@4X><4dKPt*n-&E?YpvM;2 zKjO|SZ3v$a&*MO`2D%G2tJBa;qFc{<5nspM5@VvwW1nTQW`~oYyGZHsFH!geN<L7B z(ZXrrnVoY!^>DTq<a>AWL06SmL!&QZ0}D~-VUwDjJA9euf+HRno4z~~^U@s*GJ6d# zOP_yzVGrpJ*hc*1uuqg(!>?cZ;r?p=hq6W5ewm~dwpOklMU6LAl_!*efNf}<vu&BB zSqy9pbraa#)BD)%`0Wdx;iSiaX_?19siNrwUq;5xlOqLQh$#KH%p0Lt49A-;wWsa} z(9v~}F}l$7uMhW@1^fi?w}!0t`j}xsZ}nM{R#Ic4ET8o!>^DmZrr*CPHrGVNLL8ud znzR!|6Gx>vT$f295!i8o;g$1WuI1f651;v&y>^yc#@w(|DWQ=zG2G5`aiaJ4{3%NN z_u8=L{woXNDMIX3<aQ`p#T9Z{$s3em8f1I<k^z}|5ey*%uWrW^jl(~!qzg_Q!ZKZa z1RBKiB>_uo2e9W;AXg!~yFI|9IJG49=5EgqJsd2c>kE_C+$YNGy3^jjhx_t4rhjH( zM|nkX+{G{QwvQrjY^qglS>}+DZyvu-f2(UXB9MTpw$i#nihh-PFB+f!aw%@<Xg*XF zCWp|C`Hn30ot`0<{5y3G6(;KZ{Cm>H+=O`-_vk0dQUp}RcXbX|R~c0*+L%6C->uX0 zgvR7WB4Gz?ZoBu^?g|SVw553HlB^>n9%YwE?j2uARg=|DW|(5v9hKmJsfofzA12Ch z`D?khv0eth@btfflyiJ{C_N%3)F8w->MHf-ji?Ni6?$yULGXMRo6vnjIZ!mFk>zg= z5sGT}_Y)ref*x0oz`^_hOM&!O<p5I5DbMTq=GS(4q;0=}SE;^~V<D>7$A^L=(nMoZ z#}V5JN4Fkb_5uGcj=V(O7!HZy5cMt%Lwj4@5%IVA4s~A-X)W^%>(o;U&b7p7ji}y2 zeY^@UYOis)TJT?q`7_8^(2R1*EWEK&FH`w?{vsq`sLljKsHXkL#z8g@gKJBL8Tpjm ze3$u8_qS|Ku62~=k*U*M3l%%;lP~i47O<;y_<k)$JnC{|EaZEXeTl;?J2Eo>{Qw5p zx<d1|eO8Es?R9B-rqtzen4i$cZPe39iVrT`9+j&4Mt^<90{3q3&4$1dFHcFE`$XDr z%fqe1)8IaQ=TyiG3p{xxdx1N6xTAaX4U9}Ime+da=e}oi$SA?PwJa>7;{q@f2Mg)< zi3yyif$d+)H>Y<QV$T_61_o)3U0i;!XM#oxGxcJvw*|^8D@R2h4tWN67~=Z-zp2<~ zkzbX!Vva$->h+r<c|JXrdfyeDx~A0a>p-estVEx9eI&hPB8#QRX!wvZd0jV~kE-$3 zw6bm)XCUFy`^heW21x72#FwH*j=ITTOlOGTs7HWT#AwTNYf?>I`}_H5mKZ#{sZ}(1 z7uz?gL46XA--@GQE2S@zs5p3>MAu`lT?&MCfoX=zX45NolAXO)OPSLb(%&*Q{HJC; zWK6F?l8~J3M=m2Z`Ut%hUIqMyhi&tp`#>quU>vVZ*&$F}eNW<e7Eyb~)bMS#D3fu! z%*ZLA*=A6D@$Fox&a+Qr0Evx&+76BThDRCuKo`t)wDQhBL}F&+zF3n`S8C0IDFa=c zIN~PgAkP2esecyIMe*Zk`^61_%=3;>51d=D1$7Y(bxDTX@a{^FCe}1{+7*Pq0vCcw z&OQivQ!AeQ7R_M7&Ox8y8lamWHt>5f<c=J9@RX`koLQ0ZGiIL_LlG-~4##9#c!i|O z;SZbaH$$wCSivdHqh1B{2?NJ#mnRJv#wA<-zSBvq)mQ<%1}3L3&68Mz)oLYvy`f>p zk69})oUG?BbG#D!HE7zGNni}6`pYRI^6W#tD_{!R&#UzP>;b<wad43Kl-1}aQP<Jo z%1oaV9(m!(EyQ8;WUb5Ko^w8POMHXJ-h2>m-w7|^l;bJV*2#Iv_^|gkrO=n71S4d< z6VfK2+7tAd<uSCF*|h?r#LD#CE`uhm%J<jywtYEO4i=|`%>pw8?wd`l04zuM0F4%m zdWTUIlCgA#0Q!&?xW|giLv$kf8bOTSp9kwjIO=1HZJka+OR>etc3LRKYyNs6=EN1@ znEZC6PCjyAXoZYHC#3sd<0X&JH&VU~5lnZqrni2xB>F5p=oO61=NO+9$SaG~<^xkC z6CiV&Z@`?gLb_~r<CL%{9H}3f2%Y_a9F_0gqA@ZU%9-7bF&3X6fTTnFN-$TWboiCZ z_p}A6Uu{#)8gcxnf-m>VJlCjqPYn;WPH$3SQ!kb}99&~E(90o%p3Zx+cZM||Y4Qmf zZb_|VCMG%Viq3iVtUSI2jD-C7x^db=ja+<a$q71`j`7l1Ovb#slkDbrY<W~V!Ec6n z&>nB7=jnHFRJg#!YOv&hP>8bf%F%&=0^z~kc)Ytp-={dv#)bo1Cs)`|r`JRBLzM3W zsM#IXVQjh|I(jk>t|%`bhCE*^Xnq@l8Vnv15$(o-JCvVMKc1;|-nH|lGne!3doYw( z?3@NTU9P)Lw)11_$IaiU?$q9kjBc%KYa8DsWd`n9t@6)cslL>&@qZS*RuG4Je{X1P z6)vfePDf&I<HSuMH$|aFjsamB`EI2wqMUo@F5yS}nT&lvMx?%EK4FsX;7}rwAqCMO z=wI%pXxnz?g}^%SY`TAoPK4ABEPN0S&~r|?p;^+ev}1IYJTUV=Cb^RtneO=n;uX`} zJG3(KQHmj;5)QF=1)_qIP~3%PB=`5;fj(jkJ2p`u<DlG}m^JzFK*+-^y(N9L?qDet zc+6Fwbi%BcdjcHw{;yDzibe%S6;CRnkF(MKRjMZqN2^JTLdzcimIrSr?a+jQ{|EC) z{)Hmr0H|lx5vJ#<G{jI&cFBC07Fn-!JJ6+h&Tr#4Mbx$PC5Rr+SS=^BghCy{w?faP ziF)VQ@6W_Wmti4)zbW5uL@g=ustRLDu+XArD4o*Q_jakdrx~jIm(@s#RtEkG^)a2{ zNqFwJsN8gSkaYOi@sE^evKiU4qw>UEd#Dhfm!}J7-GAnUX5zPOLk53zjZZk35K8I2 zy4%uLlIJ$qf?q<~cf@}Fr)FQdJmEeQ7lIk4@L&5s-2{`reE!aQ)^-4sFhu$)elw{w z%*Ike)%FO*MzC6GsP;W*Z_IZs7RVOG@p(HuN$YK%;sm?{a|YQawmC@_1vlFv&V=(| z2XPtUZLQmV!+V7qOz`!`nF3g-UDz^+`3WP0%C{VjFQ@k*^<3=b@6cZmwa1TEhS8qx z8%>|6PrXFqgOm|q>>URQGEC&71<m3|!h{%XubV`GG=#%WYj1{+lPm$vpP|B)j2i=+ zDYy|kW-C9H72Fp&glA7~Z1as8`dvEQy4@0@@JGUmrN8sm{q*B^K`<xREv%Oy=@ghh zcK)Ln*#Xy}?#CzNTQg~%cad^Ys)`+WV|9A^l)-q)JTzWmtNdnhfjMSYo;}uf;@gtK z5~&{Rd&!aM#+7>R<Hkyg2rWL{kIfUb_VwVRcKyp*Wb&itJ$s1Gfx4MOCQr<N3bR@7 zwSq@ZIeA&X_xhRladYxxR9*QiP|sJt&sU*KwC`d2KbWH5%Nb&w9ao85;V{zC$2^}$ z`9=>Pn4?#S9eR5{COlj;`bI;Z&nYkOYqdKZzB8a4P~T=gAF<SJVG$$WEiBD3L3G=h z#d?2yAG<#elSwh_Co#Y@w}k#F1lNqgdl6t3m2&SV-s*b-Epmew16%^$5($huSN&Iv z<P6FOkZVND^8|X9&mUh52VJp$6J9vwsADd&SCd?@3a&so0`<6m!0~z*n4w(0&$#~9 z!*${;FK&?jFvPm-HF1ihdXtzx8Bb~L7?F{o#+})X`Prx1x<dh0D(SP$srzE?cU?v* zd);!SXDdu-;7|PsQT-FCV^V_*-uAbYwpd2<zcW4v53Q&DF12bq#N@rX%Uf_^)Wk9W zpyb2My`^EOJ{X1-P|Rlf0uL|Nux^3(DgADLyOD|#!yHcIwXf_uuxNRVP+e>Xgt@D^ zv2>9gUg!6C9|*F8g1YB~?{wxw9;o7=&&{7J{o{4bGfEPd%FTrkwIFgQi5Ob4K<Vec zLO%d4fbz^JASCmnWwsKpPLq47#IX}n5F_^8Vl?%Y#DBywLMH-M&#5KG;*^pTR=XMF z^HHf92{3*H5RAE+l=SsQezORIFT2$<C#DE1XjHk8Y2%=?-)R-5at2H!{;j1%MhJ4! zZ<a_^pK06^Lp-uOx=xstO(-qs<Ov|!NRus#l8V<jGA3Pl*Z1IE@-6^~Z+3yTR9e~z z=0g2fPuiHH?x3fZ(!VnCQa%557GwU4zq^g%Eu9#h%;YlVH(&{{#PIUs#14~S>!9u6 z7g6JqXlzOJ8)WAw{Cg=X(m3@9$|wbmv^%@!GD20c{^@kAjTb@Tdh8JT;K8SOsq>ej z1puUWsRcGMz$av|^~3Fa4}k^$e%a~u{-SL$9%SXFrEE!s%QnO%`cm8a`AdN-jdkh7 zSrqUpPFzAV^2(v`Q~8WGgr_`n`83Q$|A)MR_VL)<XVNz~)!10)$PTwxr*|cHIC9Zb z7stTo3I;irT62nj*z6ZkhlIYKS&5S_p^P{}^6xQp%{X82h~M~*G^t1W6HVo(yb67N z9Nsdx_ebF7T<OAcrK6SOS3Q><1NN?mN~c6~If-SAMgL}*s?(!_&Zixv{CVw$m#MAo zjg(5nI1a6sS||Bitxp7&L&xyQ)c=-n(J)12_;_PQ?%INYBA)_dvKXe3G~G4QO`zv} z&N-!h(!R^pP5l+K>yejOF-43p3xtH3?RHrkKabAe#_&bRi|y1YUpamps^=lt2`RLM z(QTj$k?+QwEmpQKmB1Lhy14phYue!suJ<HfQer72oOGx3P@kVqT~4<nup-d1)A@YZ zsChTo`5fH&Xsf8JksFGpYj~E*u<%Eogn;0Wi1$N?tn7~rDrAn#(X*-<m&!fJ@8Dn0 z_qw{B?o4Z!q!&#`{v;%%e>?AdnQIncdjcf`()p%&U*5Y4THqF6<Y2$SM%PLgiDvSw zyt(p_^V#!EUQ3IyaC}7HW|@7i^O57N%*{2b6(LRLL*e`T+yQ;-@cy}WC>b;l$@qFf z;=>B#Y~->$Emq!ruXk{r@!%<3YhOhp2PZeXJ|8tQ1Z_SKn71W?Cw4`@&jaJ2;&YU* zFYHu6XRt4t4@dnu5+8iq|25tT>^zd{YWuJ*62-E|gxW~e^e230IPzS=8v+vCZvau{ zLOV|LffX3^Y;0*k8bcEz5P>gMppv`G-`MQ(V7+&8zG9TTl$O9?ojE)<U-x8t1Sy!7 zDMcD5)ZM0q3QASrSKhWMr>rc#eE4+a*{!?ym^c_r6m>822>nA)=R>nNTEF#Y;GCaO zhfwP7ZvXjS`0$kt3m`Vpb#Y-okZ6-cho7QM+k`taZon}jVVr()0WL#@ZUWHr7lipT zdwyyh3Kjk(aSMNoc^T~Ao9!Ag;(>tCxE?ZfCT6Z@mViinKxrH)Y<Z7wo@x1pKH`rB zMI3o-B`Evyt8K*RzATAO?^Rx%`02)36L0uxK-cToQtDH|_$L0lb=WfHYf|47ffW_; zsz{{>f*t<h(=igaJk8VmiZ2H`$MYlI@JM3mi9F5E+*to3kG*A+?7GU>F5wQMjV5~Y zK&HYgi9fTG{9-Bib<{X7q>rL|eRB@su4T7!S&Z;vSn2$0)sd%A5lt*XpNm>lf>f_@ zL>U>g79<k&_A^yPv~mE`W8Fb!-C%@DEQjXjq#fSOp|Il6q7DwA%pLubT9?n88Oa%! z?;*-$fB3?vJpYCqlf!Af6yAko#!Pv#9G5T;jMuYK0rCcC7Mkcg8Fr12pRcJdN+dHJ ztW`4wwGGcsFl~+P@+F?M;ZYY85`h%#4@Jq>R5$YO8%U^xSZ^iS&q)*!uHss!?&?h& zgFl99?GvkH`_~&Gfh2x{8(TXS^a4ZYW?ndSnN)m?7ol02z_^u9L$9*G+7&1-DYiSu zFtl^k8kk>=or)`{k?FXE!^u092)B<+Piy4{UminOC+c=Egf9Uf)%?4CmIuGW$AcO; zzwi#haD9>SL&ZfWgTgr1WL(K)e7Vr<2{y%6pyY4S-yvHuUMIvUMYYt9IMgWEvPEW? z&fjBxHviyq^jILh^S);3Q4E)SwIzkdF7BoFU8ntFjm76vf4>QlGHjp-h&wg*1W9zp zdFwk{e?MK5i@O?>Qx1BBK5Z_<>Zz9qA%z61EObPR_+BLY*Z|}2)1&W`zzI~A>lHLn zRe89*E|{H*>!eZb(({<(csp$+#sPD`zPj<lH?g}|kt8Mo6akONdGOc!pJ<k;Rx#7v zhW0EGLz|m2o#3?jqx$BC7?VU?j2-ESTE$EV!zhR#ATZ)vu~Wi9Q&}pT2jM|e5~u-` z@jP*wF@e|#^A;ZD0{kt*jpKSn(%`MVpHHn25CBahO1w|^Xh*z!tZ{9Ey0#0()vu|z zFM3M=>u3{8TLES;e@tcWf^D8-iw<B*33GJ6eeC4ugkfUTSAw#dJDTe0u8p{+c{ObZ zD2Hw{bMe8Bj=nNl&O4YwJYdh0K0*DAu?n!XCd~Si)80r&GHs>q{TKqrz6<Qe1O!s^ ze5=*@%&9vlVM_(P6BdecY2skLRk;7Syx5z_-p<VgY4D<%=r>prT7<z@ls&2ks!>)< ztq{kTesxxTdbb!~dttb4Yh`oLVeQSbD6=rS4h)o>L-~7AW6MjgS7<Ky<`vh>>VdTK zqn6s`Rb_9s=PD$sr~a*d9Y8Smv!p^SH7fXiS`UftKXE^j`1q4QdoXyqQ?eJv?kIo3 zn^laAac8U*>nznt#t_X~#1(eXuO44^dsP>PTlfVHBmc;M>eNnLB<(0&&s;x7SOA2y zivJjSpXl-9)z!;)0c6QRNS_GD;|yU^hR0$rU~(Hj#UcH<%|6f%@iN@+9Wy^y;gu~Z zQzA!odX}q1l3tc;-bvpyjy#btOATyzqt9l>KiT}Bb|%Q)u$OxeDKU;ziJZiEg)*Fx z?-9OE=;8ts^BXix(*tl-*#<H&U-^DQrd}M0;Tg(!TA{SPi|g{ei)@z~v6ic4^nuL+ z*C$83A}7bTSf<xCs*&-MCtm}l6~Ber{qBxYDa?bdD>mC(PLf5BNBp8z*<ay)!elpb z#^H+Pc%b_D<yFt74H)l^W%pV2N&A`j<Rn#kl73-HmHNl=4csn7^z-K@hnanP2{LpF zehLc;IdwVREDOIYPq0_v%|~gZNzBWJZQ=u+gExHSE%+;W=6n4iWQ)33t2sGWEAuQn zVFgiR&AHQHA1b0U%vh4l8<9WB<;8wqX^)T{P8?SB;-lYur6_nqf!yfQSobl(;=P_T z8VmAR^QI}BhrNN#&=S8V@aB+nf8e;gFeS<0k$K1=FR>P$F~r_3XmrcOLg9EfA$RzE zD;yFPI$?<4zC&zC@wd0PwGlE3gFJ2+pa%FI;!A>zOOLH&+g{ka)?U^Cy+I(82`MuH zgs-+T_VYQp!?T>@ykPpPKsO+4JDI86`eFghQvCJr&TL&-xviO1SV^>vM5VFdsA5l4 zhkwL(ri)R%N7chaV$RR7vvZ|p=lu-THy_DUZdk>HTb(pqTwP5d+Pxx4;DUUS_lcuS zI%`4>yUR??-Y|bF8)WDT{%v}kKDt7)>_@46uT<J4!6BIVPy20X^caWDL?k_e`D53o zpNF+IF)wS$U6AVLhMIX~=cTicng_n+5aJsu>7qX9`+^<yqu@=O<hv~2J2^bJwfj0Y zhf<U@X;bSQSSNiv9v)+G`5p62md84ISORpFO0ujAG~y<g;O#~|Dkw*6uO<CH7N`j5 z#?(|#fA#Y`NwS?H3s`<R-~dT0bPUdHhXz`CXIjYam8nTz)rod$d0|cjNd|fBzj*#` z0_WIvM6X0Ec;MJuu0ph3_t?>rLky>W3Q+rUBk14G`zO5~(dRYyUdFa&-#PXCbXYNc zaeB#YPKu=UjroU~cW=T0Mf{Cj1bm>`EZt&Se|UcMj96<KVc$Ps?Cdb>rp42o#-9Gi z*BZt(%U(qu4JJxr5fGE7s3{-2gt3VOZ{&O6Lrw^nXm?d1yPgy9<)g`~Id#&TDnYzA zNJC;RWHtL%K@y*JHco^26^*^p+(Mg8l~E35dW^=1CDlpSK9)dGDx94gu-xZc<S3^m zZo6r8gA}6(1Y-}Y9jMzIbv^f(7n<)Zbf|~!-;*F<5*G*kN5cQn_XWUijy4x|pH*lY z2YUXH<sLrebB=dRhQ8aW5dP|BHE>6RynvzV1l{4&ia|4sX3JPgR{a@a0d;$ofq<Y& z)hLs8rzn_4OIa#rnkn?#aw8t<Fy<^ZZWefh6@vRAq#N29x2_SqgK3h&Wd<Gefo;38 zTRgOvo+NTM_qjd@qH<U&vZm#B7*PE`mac*!%C=cccT3j-OM`T`)KY?UcZZa835(J# zUDDkxjYyXu-QC@N9^dc$gJs#dXXdKG4IfZ^N68{Lu9^%`4N3iQ9u<@V8sf;)%6}o< z;+4l4VZlWC)FdtAY-z7x8q17W`;fhZG)FF1>v!gfJTQvL7!MtOZ1lT?&)Q5f3g)J+ zBR7HxQM7e*p9DZuf|3sel4X8rZM+QvCVmQ!6;q@Y!*|;QD%4YUS~Y195SDN!7tO3- z$M@r8^(=&})i#fUO*CTNeYAUFWzv!P(PgFMQ@!<fjSV^{ZbibmR!=S-in}hpD!UIq z%rQl4mv>X^ehow8XJ7UE_+&RZLo~8J=Sy)SQP^J7tO*v6l%<!4`)H(}Z0@Aebf$@j zzb+|O@HQqG-h7QX_aG>HP^oUgXZGy4BB&x^n_e-j5msgfWvYN$mM*mJ)0YBh%h-|~ zcGYrmlueyXO;!D$4_60Xuf$&2E;?TvUt57wA^@+_`Ox`7>2<eN0^BqhzPJcqYR`W6 zdh+_LcQ5vG-5D2f$@lsZ$bDiH5)%43`$0*0xL|m7DBxl7l_}to@>R18q4lQ^Pht1t zax}&LY3GaVedp-v^J2hj_ST_eW%v{iZc_@9tIe^K#);eV#4h;|rfd*GX>)5OQNWr) z6awQsrKxZNf_l>&Gl&(gf*#kuSY^yrL=9!h`UvCYAe|2z+TEXMx2J1A?YCY!FNvx@ z{!5th8K8%O;GRT1^v}wU0iI>%thnj1EF~Ahu-!DRMhsAeXBSW3_$^dvbZWF?ee&2= z5v_(M(;plHj$OAo%Q(DC?_PQh)%fFS3Qk}JJU6SHU^w>nIZ{YAuMduMg!ekAqJLvz z-S4`#GnlC0FK|xZb!hpN26>gb<T|bdFKOW-j4?5UpjUe8qQdJ6S>YTl#+^jW5kCA9 z`@EFUm|FOweeC#400ce7TnqKR=v&gw-vBy}kOE!|(VG;W2aO|6ZnO7EBFbR>f|%Wq z?Q-!1m+<50*h7c?7Lc6Xt%e>N1%;#UF%1-HNF`^Qe#u+__hRFBnoBN<-KXCTY-vOi zq(UuA%E`-hS3k&M|Hy*{jgQb{-tns$!c!xrhVhdrak-^YVR&6p1@iP$p_*2IE??{Q znWGJzQT6EfJleEa=_Q&V-wSd_(2M8WLs4C`-|u|g+wW8r`dEo88OX`3hVfr#&IpYn z@@4E~dSv5l5W%_WCba3@nHTipkmLC-!sxuRUPHW_Ld1QG<fVD?ZpQu-bNqYsv{!pd z2l;#J9R4K14LX{lMD}#Nc)O#+y;8lEHlgU&*P+{}iYNxg_ZzDD$TH})t2p4+MQ4p6 z*>KqSck18OoLvfz6`mc9R{zqlb%J!BV=R{MWa22+h$U+U^ma*T+iyP%h<kHNe5v@< zrdyh^J=63y4#*pn_5HT;8}q(&AFpM_SI6ZcdnoV3=fYulx9>Ed^tlxd#;DTy;p==I zbA_YL*Oko<AxnSmqTUu5ojLkvtncWkZX#D%WhBghRv;@BVIx$SJ05~nHmCBw_`Y+# zR!OCaV7%fVpeC=p%lMX#Uq24hecD&96E*3JCWRZ}BzhW--8AAET=5S`nt}`$#vi6c z9B6`;JQ%R=j`gjRt)Tlb%#8grGNaxD-(_5HLcuLvm`iAv2XUJnniHP8pvLgtj-hi^ zjC^U^vY^0Ajeh$Goy7nStKX)Viyj5WcP%3+t}fLzcAPKDsjU-7G-1a+$Pjejr2r#5 zF{3_b|9I^~ZXchWHOYJF!2neOuTveG)KGekQ02zzYLKL)BquKi<5XRpE>CmI_Wp4} z@54@#qSE-#-d(Wbhw(k1Ba|<GCJqh;bo52v&Gij+&SS~N{KPtzdU|pr({Jw=udgwl zM?C|zF*wr7DvWHmjTwZ<P^w3<oqku64U%}~KdLfiP?eNcWid=O64wb$!JOV-r)y1G zK1KOzCG}$lr=Yoq1`~l9`6_M;g(>~Pd_tWwGu-R)m2fv5CoS$3g_xg}!H#jlnj)Ic zxv*>+5<fb66N}`Obl3OHA|!F3!dAE{aYX&MVf6c%Lv!8@IW;W{C8!nH4MF@CVXflK zsJGD`Pd9w_noKy*N7+`F_Ge}#;abhJ&c~C?&mS7>-ZL^e1n#_N`eG;KU>eISYt-Ne zwlG8JGg^_`KM#Kk6t5G^Tt`U-ikE1L$xTtecV+lr3S1f|me2Afbt+4C?-pl1?h*q9 z@zyxL;<U>4a)a6`n#_;T$Gw>E@MN$HE)ib-_h|GNHhdZOGsp+s?u|fxU)~f;W*(Av z0JP@2*1S%I=&+^lRBL0ZE2s}b`v{T9Ky2>YUPf@$R!A;6c20!5q-N(}_$iSvnk5iR z<)^Sb(=^Vb?W%4uv2_3YYVZx&2ZF#U64$dmog+aYMQ=>&rVEc)kJ78N<^1$&@+i%2 zO6Hh<Uz45_-}ygaZP_|Z0`}7(o7R>GerN1Gkgx}Zhc{T<O)Hoovz)ydm+6si`&Yfm zM7@P$_R4ZKT&qW!bwC=>dvw^FyP<Pa+F5?_ez-gPs}I5OIxCmS?<0|_hc7Lc!ew0Y z-G{P%pnfw36m_4y9OXXM2UI*S!dhs!U_c1{7B9XUMw~wM@on6T?kguO11f+}r!#0O z&Db>6;E@k_80p1-+s5SDNg8nR=KGh=JS0)$nqn76@jO<Vm8i+g%G4X(MhZ-*xqTFl zXU-#~nGy5C3Tnep<C$zdI!%b{l3uH^`N>O*S&eNozoR1BD1bYuMMs6}HQlprzt7{3 zAD=YBR(diicxtT*BcXf}$K%xWxZtx5HUA<%p7R?<?zH2W>IFLDc*0Y$TUh^rkL302 z^VZv62)%1VGvPzbTEI!0=t!u(WKxwo3opuS`)un%wutNANAXo-cOMyfgPkV2Vac&n z6_C@2?t4VH!|iXz-NJVtNv!M&?CZskwX3a&QSK<1-tIq3H6RY+!9}!R{CG^dslZBN z&=52PS?o9>JN0w+t$!Jki+BHrwF~eXcFHC`b*WZq*7`0#g>jGn?Qf^%BI`{cf{%<b z7#Vu#IJNTaezlANVLSSbKX&Y%KELv6#(oZ|9pxL%_F}5t;GTu)6+$VhYgLmEQmde- zGB48>PgE}fo(jB{orfpp!Zy%-YyVL<!wzWM&g$_!mA)x<5Y#Un9*yb6vEFcIC~KwQ zQ=$TTQEg*Di|m3`8V?L}tizV!VxeASb|!u0b<X!l%7-BM&*qu3eEJmg3fWfp;Y8+G z5*AdWbF*>yR`PekgKvh&Fs9hlU9v^SP>%~lH7i9BwOCPnK>)Sm{c7mg34UhA6npIE zI?i;31YWk_=Kg<4Dw)PMNT!zs_@6Tm@;*Kubce6ji47aCMhByT7*S}G9+&wV*qR-W zg!I`9Gt5xG)sD`yjNe7LN<24T{+mWotM}iXj3_)a6pUfOU<t*EbG5k_ZGT)PPb^dX z9YyX%#_coMqc5V}*ioUQVt#=QKF~Qq-%I)Wz8N?Zuk3^=fZRIy;MC=DL~%OsYei@w zp!*$vpC8p!Z+DC#ah$nNCV>-fthl#eN}&X;(JmtqM`)g46Bq0D3AJ(;XvoTwpgS#n zi7Xk{ql0@S<H)0z--+lwW|iZhb75$AMqm-=Pk+wC?E$iAnOXgKv5X4hPOI({Lu_am z+_;#oX>2=M>ud6YnDw?%Gm-;Y#Y=&q7l$G3RC?SnI1B0D7)^_a(j%U=jVUrU;>H{= zjm{}s!kf#<Xy_MI@|i|1xP!t+_667L92Q><fAcrVKzKba4e$}4*pL{4<dzhwMZQ2E z8yc(B53sU_(Qr#ZUt%e42NIE9qQ#A~O;-z6kppwpyDz$P&k-)v#|ALtaG?}W46>^@ zDK}SM-I}V)Dq<R}%MpQ(SU)vG^8IslE#04v!C_%7(D8iBa(%g!DfnB-fTFZ>)I<xc zZ81*7Vy6juj)1$C$a0~E+pXs;Qq|o`%}JPv&MLbarAhLp1}jbBG5z&7Jw^U)GD8{C zRL}lF9O}P$6#pWYcPQbDqrY=twGqpG7_?n%P_Fu0xG%^^GKS`L+FgBVE30coO51<U zz2-_&lj~isSVnG&VJi`iUVb%ne8OyLB(Og|aT-yF;Wv47B0yptBjP%f`w)0Zb=&;t zqJ^3<uX;2zb$jF=A;P$rYLX=pI3)T_m-+mSISzifB=fT~TJcm}g-4EFHis><1<hA^ zZ}6tSFOH+>84KLw6z>$nt&pQNtl`sn-UC)_S4Q&u)CW>XM`8>CBMu=;XasAXBU5}2 z7+K29@gI?mDb<sFus(mx7)2N#DM9@J$WFA!5g8p!KPmWhGs8R*`q0{eBoo-nr4*PX z%or{8FEYocJ*0x>XC2X{(JxNby$LXP<fqIHw9Pyz#AzAi%+)av(^fIm+~m<W)O6)` z0bnIAuzlK3CjwIL595G{Nbop9HuyKc+Q|&H2xe!~1{Vr$x4aUKWlI-LD(^;TWv`9i z)j@o8XL#2|SiDxB($3bO5yoruo=KL^gBOc0-zrCesZXB!WY`cVg3>qXp9%{GIk@Fm z$0oy*s&d=paPcDyeWop1X!q!^zch%6=Imp168Z$0nOEtWCC6$7HxR7zKP!?zRdx0k z{;8Ru-qibaxM&i1Do!H?K;-CE7}kBY_`a4Fj&fNy1z`!}QgMN?_SOY`Gxefq%J|Bj zKtEB&!tZPG{p-W_Zo1kEA(B{!8o-egNzM^2v+V;g$*=6eXj7<t@BTe0GL~4Pf1nc{ zqJeM76fl|Km17kn+$-qt{vT0FBFcx27%8th5Nn_vfeVD9C2}1xjx|UeE>gk^^KL%j zLXl)2b;DkL`6Ux@${_?C0~9Nqv!Lls3(ih1D?cIB30HTuvPvq6BTmmvgaPp&81r{0 zl(I5U{|0Q20kwGSuc%VRQ1GTl(Pj9J_0@6qH!%66GsI*;WYtMxBH~5gp-yS0=>x`{ zQfx=`{f9%n@V!upfa=!^UpqR0o?|5P0Y6Z^cv@<vOeMyDy!r{2*yIjW3W1o7xZ6Jr z&ckG<+lG2?>aCZ7h`j-*MzJz-c+o@bR2JTUKj%Bwm3rMM9>xsv^T&_+dugAv0+@(A zKL8x=g=x8bSe!ay@7hyeTg!@K#az9RhbyKyXKa>NAXxlYcap$3E_RVBTWq%64?^7d zYl0zyudqItei5~!S!Niqn@9k}-rvx4GrXC+k#hK*l-$UsfZ;M2MAhD%g=G#|mTGWF z6ohHtg~ssceC>@i|8fvPsHR~_u92}PM9C_=yNlo6zw-6nvGdhZIidHT7trAK6u{nR zOZXPL*UJy$mCoS8%rL5sBUIW~Y+zbY;$a})mDWtM<I4K~MO~gLq0^YG!DUzn0Ue4n zWF<h|0FMjgY-JH3M{&EH*tk!JVH!fh_oNvrVXD#{+8q|fQFlXOhb?57A3)vp96yrH z^`y?+T5;_29tZm7{-dsd$g%R2p<7Lm-Nr5QlbX4Bj~@3Wb)2~K_HU^II%D9UxoDN% zfW^aPGV{y|{H$h-byt;$bto_Ns<5Gw3UExy?Jt}SK5ZP1K{(KI=`5}4doncr!(SKq zd<LkQ0`WXo5Hf{hnB9II`2SVb+^n}NMGsbt^knw#H_GwU<Y4-(jAPW0F`9pb%@>^X zm8|c5gM`jypVx$dn{s>YRqeXvGSBsaCUguR1=H979avhT4Z`@RMgCB-6TzKyo?$+c zU<GvBugUXoPWdb4e|vsZ?-r<tUCf+elC4cV&VV^VlS{i=S)6SB>qa-g+4s$m4PF=d z=iOKlFrk4|rkW;MwH8EYp<5xaGe7KXw3#v}O0bxni1Z2%c!UT-R#123<bhk7-uyqT zuta8L+B(JvVelRp{b!zJ6P@L^p}~azm6U#15>YS%$jPt$I{nr!w+F^&Xk;cIv5jg& zl00^t2B;}tT&VCkaD|HGulUNK_Z{3IU_~VXTZeIj{gg~I?Vg@5y5n#3PBJ1YkhB;; z4fCB_6Y|N#n;z7TcL$@)Ue7xXnx_at+`uJIIo#CwtXM}~F*K%++X$y(cXDaJZ>tf! z#B&wrt#c@L@(_@!PrkalHjg%5cYlDJ5=tQPbg7)9XYGw33Rbkvw43msa2HRC`GW0N z35SFHi#e-w$@)R15AP1=3Xfd>Ao<tk8JzowGVT>Jr_rZp^#v<EbDsdcj$4$rjXL&M zG<_Z2uVl7kfQkaKyIE*i%iMf^uv1&i1$OR@w~AKO1R-#d@v8o8x%JfC-9ctN9X{#k zh)POqGvlP0RPBp}gQG?o-GRml(*)PACK<Y9sXpuRs4tJ=&LM9Yrxi8zVE^a|8Cp~p z@NOK~39-wnj5E3t$d@o1!3}>A!RU#-bh(+pqiv`#YxAjR^^#0osD()#H($Fuj=OOy z#3dQzIOLH=odFLb3rJ?UBiONNjrz9aRmjA?`9SwB6ws_>^CPR)cs=V(a;`Y_eCs2< zLXzzGWNOE~wh+zNEW4pxPFZF2OPJ1I63cLQ!Fb?q{^BIQUXi!d*Ns~Pw2ljf)F)Uk ziB6S@63!0tHz@PjfdYOXdFAv~m=tP0k4!(W;RdQN5Vp89jOQ9qzb_ScxGGapBDi8u z$!ME#3Y8e>4J@FzFMOU#DZBJ~*ZZc(ozC3UM;#4g?$%wkyotP*&mScB7#K>~Z(20; z813-$dB+XV3KRzm?{`*d<jvS^v#je9C)KYkHr5uG!0$jnrr3JI9X!+k7Lt{p!#_@c zL?)sUmcy@wv^~EDSCU?Nrl)2_e6*O;{KiP}Q<MS_SORb&*(yz8h;vNgF$~B#A<Pw6 zqVWH01aI<$O)=HD&kJ4HDL{8{=QnMyVI%fb&(@B2%4iD{ejbJ{2IKXxAndp<7~K1} zrKm<Sse^d$I3PcqJqwHgjx<kruib>%kp$HqvPnu4KmQ6D#A~fJ+W`{~DM$U0{=}_2 zKWQHHNMb-7*ONmgHoC9U3)+<TdTkhEv#=54XireZS!%kZH?3a%MkMtP5co)!NjddY z7-!{^a6mSfkJkHjKOT~%^yg>`>Hisd3cIc;{@k1p^&je5DW!mJdO2w7yX0yl7va?H z7{9C`s>|>-Z=Uh^i5`_A-B~nDB5Sv2tj+QSXRiA4I9?Y0rT9WHmVk9Ckn7VggjK#4 zkwU*jSRC1N*bqGM0SIwWl#;WPuX%|Ls^GWdBvJ^@S)PMs=73<sUu3Na8eCQYBP*?A z0I~>y@Gzi`nyixwSk4|SS<W*^j0PLyFcA@=n_DNOhRJuI-<EKU%Lxwe1sFd``tXE% z;y~~40g35_?_Q137x*;~LjJn?T=v+g-dons7773{W0a`QDIs19h;Af@;nJolf!NYt zRR=mp6J4ieT2*e5eu3~h<EBMhm_AJEX0*IP5_njnKWXklBUKox5z`9|A<h9E@oFUi z61d-BBX`)sikF_(xH<5w=%7>_T%C#jnSag44U4E^lJ{-k(c2D2&Ypj`+jQ&RSk#y6 zK;>?Ku3FxsTb}_#K=v+|N*XWnuzDWk2cP;95-qf$PvQOQ1CNs0`aBGz#CB7dh~S9_ zOJV{)I{WcV!1F?)W18<%lF<~coDVuf#7O{@6tsc&FQnwjZjOpKsT-2?h+%`(SmKT1 zk*oV{Ok-cE#7Ie$e-Ma|4~@;wq};qZ1&UBr23M)@eB6i=FXEjp&px?#H@IB~3rf^} z{H^#t*Put20eJY|v;Q$#H1P$Wo>uS3XzDMp5k8U}+#VJ(7g!WZ%wWv~Euaj1NegpP zn#`phA1f?^72fT@A9NQ@+&@MUGZH2<jUCFA;~E!pd`SR+IcbB;5n-yv+v43y+$Dq) z`<9ja26?WZBQH1%V_qabD0#wBzyS#1SX1db36xq$&pR4GY33bf5y1BvRQR*ainegP ziy7&_#q{z|GtOE7dZHN9x2eZZgU>5=Y1Y<C>fs2HD2-=qZUGbUNV=3NCUT2m+PE#M z82>BJ=GjcujUhfCRXu&Cr!&0AWN)w)Xez_QLv^OI@89Uwd-pMVBqwpGrYDX}CoCwk z7A_(Ko#8=%e}tM|fk>8FqYrAA%0sHw!K_BghctvqpMWF~va>HZ+8l-mJYDt+ei6$q zCD&&6g~wH#)k}Eixzlt_5t$clV!!O$)MlFSh2b7>EigpX!_pvd6h$x@tmtM1t?G4k zzPld*^S|70rDTDl<;0d_;={#V0z=bkUn0{96F<HF@3s=)m>qjq=}2cL94HA`Kz}k^ zf>Y3sT$;>=x23!TeLwRwS`P7@4!?sb_F|Ca7=K~7K{R1=wsojP-`1I@QQceD5b6ZH zExwXyHd5wT{aHMv==+H^0E0;&3J#`rF0$s=(q~p*x}(wQ1$C`_{F87x6fQ8FvdddP z<LKM9yO6;52gDw;wKO{BxFs>J<hSgqpHQEr$3*%T!rGloWL_~jj5^=?^kOe}*ijHt z?9^rWo{b9k4G6AXytdr6a1`$<X5#h&VQcGdCloN~46W-umNo-V75Spe@aD^7ZMB%A zpB@eyked^nWjzm7)^Pq6e%|nr(GB1Fy&-6=e@0ouCSJPSRfs@b)e6hP+IZ-L2KX)V zg*VjMy%&V!cl%B4W#I~Go<BbTa^?E&El3~^3htta{>(Vr<F(~Au2!KKVIcx0frNdi z{4lh9QpVZDXXaB;4#UJ<<>6o8V)*Axd|FE^Uka_H!m~v7?A1&#9c66JsA4E$hFZys zcoLC~$l+;Rv$}{5b?OdF(X&<V6FFPTDfaZfUsK}WoyR*FTEx+nKF9h|UWLuS5qP*G z#+Fv{+c~mP7>$Njxo>LT%qzK|<CE6Bocu|j)#doWjJZJ=`3eXmdmYAN1}k+RVgb-> zOdTbD&$VKkxn<d1!^dz^mOBm~iY;Ve>6lABCoctteL}pwFTSmO^83BIw_wr<D#<si zl)i;1XJ4+3dptWW*S)hcluCpEO=I9%{3U^5llDUmHkeJ{?)7~FVGz!qr^EK9^rYbI z9$|kC9K2~st9SyYYNFyg1%dl6?fBjLoGtS`0U+DfuBG>zehOnb8JE6BhcQ!jS+=?k zP!-~J+#v{~h_!>*Pswu4h)^4L8O|!H+th{`4AXS^mQt`L1X?`YWIibZg$r1e7OY&{ zJ%(b~H}7+2&Z@HyF7NZf<WgumCFC5C^5Wka$sM2W)yHi|$~+e!edgQ7a-#m{$_EPK zh^BV(vZ(6&tlGHUI<WtnZ2<shkkp^fPeB9*<q9HxfhSTznR3Y~RpnNhB@1*{oFJtj zLo+f&<ZpeFMU-dlNIT-n*|D%@Ec+f!pR0?fZAz*+ytw$8Mn&>aln`>r`Ozozi9?26 zyCTQ{2>;pA)r<EG^2yzo*=<w9Ki!1EnE&UlP!JyUVFc?RJsH-mo{R!(j|;%-4Olx_ zv?_1IiMHmcx#Xme_I~;`K6$~r`Z_s_4;j#sOf&|qSyVYwkQNyYj8wkrce@vmaD)nT zBq9$y-wA377LAgXI3=dKspavqPCGC3f^d5zfm?y0V#>riUs_06IUul*Z!}5=QOhw> zJwctzT#fa%ZBkhnh5fXxhvnU7Tx0Yc1NwUzj&T{?MYU~p=6-r{il%3G%XW1kgk>W7 zVppfr1Sp=Lp36j3x#P**c$C1#$~zBnOHzF#7a>0`foUM$r$*@+Pv{O83Xiw2o4##$ z%i}fqC@yH3{!7%!we#-tk>AS8q&}YC-5Fj6bItVv-A4<5N8OWaeXKG;=09s^NHEvU zryWC^^HhetpfzKotr3k>2BS0Ig(AItPX^_svTnJhn|r#%cUx!Px**ScbC6R1J5AHI z#2Oz>Q|#<-yKysKT4kmi#MU)I4Ld1^u}|F@)j{P)Rwq2k!Y3V7uKbB`$m-}H$|HxT zXB2ABhgJw&#_L+AT$*-E>E24g;S%t3Szro(zdvsIVWqR+DtsIOV%}LIJ#y{t%E}W_ z4&gF)9)I<L=;y*B;YVRGIsGfwJ+4K8StdYoahkx@)(hI5&1;ff@TLg>9~Mc?X9vX8 zN&U7CTk+uBq)=#$DBYWCpCdl^nh*cIEWqfzBX=TR_|Q8-%mgm}9!|#rBJdsP1*tZG zFGQV|COVRugsSAb&s^u{S5b&%!w_QEqwpFm1BPx7@sp8NIHIrK;s0FaHJFrVD@6=T zprRRi30uZ&sGKjF6Dy@xfQx78#s=m*mU3f$y#xy1TkNs;@drw|acWfbtd4y5Gi>Nr z6FexOVp$k=^Y@@D7dhna5*e{qD(mUqpvYety#FWcuHN<#s!YFMQ_;h-Mu!I8Ib5n| z=DRITA29nTh{n39B~A)!ou(>KRnq8yUP^n{<dwx+F;X=FjLqQE!9OvjL<;?)d{1&T z&7!kuOv9Hx{L|#a$v{Tx@Qgsbys`fr4s^u%!KCPRiMPWq8lCm}K5nh%%jtWIb<6(k z+psqncx;jFgMUiT)hX&AT^;0u6nz57a^4Se{buNQm^*8ul~ux<BtIAqgx7<JBC+); z(@}!er_}vMLX3=@s(<*B_n-LBVzm5yisM$CmHJyo^AYrz{bIL0M(uj%_CV~y#-Bp9 z`-RT29wz)Oth*^KM~};puIXpTN4Q8R8A^Tar^?@C_E{z0JV0%n(MaP#qWmje+DvdQ z><=MVV**zTemH4FPu;?;w21|KI(xXQzmB{R1GR#jfP>VQ??!Z2XW)9j%r6~|E@drb zRd-Z0QUM!zE%Q9Mgs2`d5b(_;RhGDW&D=t`Hu9Xy6T+szh+d8G8=ba~+83ERwO*bb z-i|H-0E9V07>}uq)mix><Qlqw(#!@At8+DQFHGrqA#u%h-Sc1hdOFotA=yJ#kF{<N zU<Sd6)BbF?@&&C@+X-gfw>wS~yo+ERxR`hr)|a{{%Zi3E-hbGKW%R<lBcMeZ=}(&X zF3Bucq2$Hg2s)+E-=8(-2XDUF+}FJw7{IZQ#)s%hF1UX{s*F@1fT;e+-bh&CPJF1B z6b_R(X$PRo+6T()I}gYj;K?m;FQwiRc$0|a+enq3UubUPY-K3-V?Q%21quQhD+qax zcbUbg!Jq7Gz87wwn(xE3{IPc8@j@+}i{tEK$7F}VOqV{>dPQp8$u>^Qq>G$;^pOLX z4<7t3bmh>}k1bdPv7>7zw)snDew9}BDJUgVB!enJXj4Lr5M3QUmrQMIC<)5<l2})q zgPtbxqsM?OIa~nNMSUN8>*_(&mqz0K4fo6&5wc;~2zB0x+uH+rTkbzWr=d6gd4V_2 zJhS?unr|;3oLL$ayiZwEIn4&2)e6M3Rr=o-oxBdVRbeF(fhhw2eqMg--&Nao{}S~8 zF8{hogC&v4a7JoyNxq<k8!sF;o_^}i;xSY)=J+{=X8u}r*vNeI`Wwrs&%=~EYket; z7lZhn%BU818lHyolv7wYQ(X~o2p+R3E2_A*oQyf~ce_{TZCh7A6Oq?D@|}*GU$IEi zSvUgD@igTZktMM9Yb?D#@JyXn*gPqLB5yEScZzOHDMz=nKk5=LreIEP16Y*=-T5Yg zg!eraM+F}OTH`d5|3+!pC6>F5w1IHiOnz!E?Cw>$yBHP$m%irVaMA9KqC|B1Qt~Bz zH?CHMbUzYdcx3~Y_KM!M_47c2y9ra6fZ!C*L&bz_TXB}Pd-st>`4y)^rYI-!x!<N* zm3<{@^RJQc0Qq=i!s>b*&vU0@*+{=(3YVBKsvjR2=(8a7ED1601R>JEr^C-DjH+F$ z@)?pRt7@>#&GAEWurMW+G-)vYqF`q5vRf-9xJ|MfGAsB5LdA(_NY-YQX8=XLHSnSh zIJ%8D98wjTgO!=__S#+T$=aPV;X{{8|KKJXfm~^^of_i^tg~%2cZHTt#aLsdlUIDy zGMFX(FZ^>tC3tOk({5S_MB4g(HD@8P_XAdddgls*Y1rO09|x)Jy@7>V*u_LZNkF>L zyOV;rvwv@b0dHnUR+s70dPsWgn0C;3%MC20A};K&6&I@m4*eE7O9s{cTZ&LCN=6RP zD>dvB`Nib4d%{tep;jYn$K_)=VDhE_cG;IuiIWIaUobWhg9}R$8V{tVD<zQPEl9Qb zba2`F^smG?Ld_MfrM%QHMJd)E;)*1AWAf?jHjvH$|JAV{hhahLMchNh5|^LfMM&*c z6C{$zz^zQ(OXl^t8LV(lJ}{AtL74qB;Hwe8@y*@HO9%|<Tz>*lJ}&Hh&ahJUFxJJT ztCUMwj{*33ILE>n8Fk%*PnqZL_<9g_@^WB4haZQ*6aO6`g!E;D`K?U)sv}n+RW7ze znWzu%ftb|e_|fUi+o;3gwd@8+ibYVW_w>3_{dxW@ScwP@X|^(N+pDO|*Gw^6L-P%e zzPB7C_Q$O~r515Ht0+MZpW*Ek7XAM6n}U&&v3}XLIizG1rG&ET4>-;`pdXFNudH9; zcf*CNnB+?pz~G)ty8ke#)nwLdvtry~fw6)0Y?U2U8WQ3cjs2(A5Pec9P{~6G({I?s z*Tq;*2Pu_(1vDzbc+#4_#l4_liPY57E7a!}+i5&kA#cC8oYA;wpbn%Ya+j*zv=gzK z$|7U&z;N4UK*_Bu(*Mvg>n5qUh@M`j6UQuz7`cp5lnd<=DK$c<EH6I#%numf0};mT zH|YsVOM~6)V^v4U{R`z<1K&0i8JVB|BwV;kldPcMAW-7Ep0vu6D}Qv%?c!H-{2(7D zG(M6l-Fb+fOq6e1IeUa(k2uHfO7nO3xGN*q$T_cO-0)T2Z+V)kguG0h_(+|qz`9>W zOp^wW6zsJqrq>sk!cmct63_vmHI22xJYT4r*xz5Y^m2c4G);-|IkkZm=ic(G?P~B2 zlQ{KLbWkisT|OK7Qlp`l#;<O6<Acj7S+Ne#fNkEc#Ui_{kysK9qOaf55TyjR&g@8E zcUgk`gqB1iJyoASM{NmkSv+sc>yIHuGIQ?V*^MS(H2QCQ{Q^WjRCAFGA1x%@N9yT= z-x#CEhEE#$?Zi?Zh$mWCv$I?Ot`xS(wEcekt4jhzMr<xMV3>ErOA@Sv?td%uxcGUr z(e@klhq=tkGLrlqaZ=2Icg!AywvGg~u&^*ifr$APTv4^nga$YEoHageG}%p0ax$a( z4Tz2J@t{OK<aMcT#fAdYxx3`T%4s9G{H$*HL$w(9N^6M%O751kre)U-<V^r|BdgJe zEXQz2C)c;!Wm>i5rvFi(<a<+l5d?k1Nto1=A)Dv<PvoqIb0tP$8%8L+@FO56Z@{VY zC~*6!Lvg({1t-UC9F40;SKTdf&i!bXiTICD!GX?^JQQj~B@OK@gk~R~Q<nfi>Ej14 z;hJ+)Ms^sGV6-gajyKO_56RCyf9TMb#^cYW2N*mmu<nm9yMR25Jv?+ePZ>?G{Kl-+ z_wQQ<Qj5SE)1D>f80JK;KlDjg`)P#m6QL`=KebGGjnZJ?uk-Zbsq0^{dMMniC433i z7X~DQ{!bq)N?%nj6=<;s5G_aKWbD#>bMTRJ`k-qx8s!am8bt&U;#HzVT&<3Bir$q? z2H==t$HJoIec?S<>fe{Nn{j&GYI+IPX=P2|DzPPe;!%bie@ML|U+-5I<+tueCy-64 zOkj6Ls)t5$S>@{QavdAIjp~Fum%4`Sb|~cv3PgoDlJD2I8)%_GzH$fKBnS72Uz%D3 z=!i`Ts}Ztyk-S?X{iV8=JAO3sXt(nU1P+f)MWo$Nnfo8gy}}t}gSBHnd(~aqIkUJG z=Lc{~7^8)qTEXpeFn+C$`Q93lZ`bkMt1J5<S!#Jcvm~YL8%=kJJo-$e)&Sb(RQcrW z-`x&=C~g26J}~~4XaB({0wjjMMm%ZA?6Rqj;Iz0oc$gutjDvrWb`*K5g}qxeoolcw zMmY@fs1=-zytt>P!yPi<a8yFLt~zkW-F))%u*X-0X3*NGnIhQ}ZBe<54Ezq!EmLnX zGlL|Fu~LHGdWsI1CgHdWX<yWJ0|1P-yVSwY!?ckjMHve_Z7#q!3K{q9NAdbFmOt0n zm#W)2)H>B2vyj&`V?c#R`l*xSGN@%qbT*&2d+KCsmqQUaJStrFbC8L)Ux$9iYV>vI zVJa%%J$eGaLvc%h?wW2zukogNCJ+NY^fZ?7AdWqkX!e_l{!{y`br;F|9F!L8hc@rW z?=GK$h$_jL|Cs+$C;x8{KB4H#Fga1%yHxd=?%#d6H^ki!&uk&d%O!m9`e$6Ha)JzS zGm(tp&t5pS?PO9KKiXh_h-sdx{IH&CJfXfbNVYJ*j}DKS%FS2B6=I9rV~wr`ymYG( zd4Jij-;<${r%9ljL`aMr1+aYWlSAKaa>l+ytc^4?pdw|1Lu#5I$S9LMaEeRBkG}NM zUF+~N45)mpuv~1b<8HzXA<CCeT%#9ZM|Kb9O0pteqC}(&%c+f{p0A1P_=S@>pZ49p z4l`}{pxp7Wx!^5_HRcDJvJZo<7Nu|wnRWbl0CEdF)Ca&{0UwDr`4e#NB}H~TUR0k( ze`amGj{-Q~TOvk|ft*s4TurVL*lS^Agc<TNmPBQK*8vB3)V)mEe=-9aL_ix5TAE&t z4ggI+9FPj9Pvb{<nM(~sW|CM5eJ)KZoh#ZEfn_GGoBNR**$wTZ+IC7d4hVtYP8ZD} z@z7s4Qbz5fE2!5@a%FerE|hqZKgzrJ{StUn#iGmOsqJ35)~orG9!x3=wlNh^GG7Kh zJ2D>N7hMY=h&E3b*$F&<7I}6EUlQosEys#5?@8WHQ&_sMwcG3j!^H{~jKLnS6Cb`1 zg%ZKXHUL(ge2Ye^lV|_)=YgMN#NN4K8gcMs9~Xak=x1>$Z10$nL#0eEoz!e)nE?;+ z^N(3b4;E*N-%eKUIoAHH6u#ZqT0*Rx81`aDm0_d?LVKYx(duW?*LHC-*G!mWq_D=P z58Iy3;*(-z6NHId$w%w-@-JQvpx0SR^p4A;(^c}4*JZ)kAJd;3EI+x8HQCsqs@uZi z)P?o5Qm)uGfblUdMT0HD#-mhJrx+)X3juEvDQ;Gh0%=1uBk9m6nK&T7)pvc%iZ1$g z1;9`u^USg1QEO8k4WGan{c;dazGdzEO~Ipt6>X|x+c`K-oL3Y+q7%6@54o8&GbQSn zzbnC#=o{&A;7MUUM<3DLBJye5_F)5<G9OUE6_-Nh0xsR(_Ky;)9J4GKeLIK`V^fG` zB8|MIyE67)8+1AKO;tA4`yX8Zow5y=s0c@yO28)CjqP`gPj0?t7a1L{$&FM6W?%bW zhj{$me4it%zhKG3A|5c;El$DArqb`{O(zk_<D16XQHRxdlg&TE2;_&4T`b+9-(EED zW<{1`9llc9o@5g=y90~T>{K8UQ{qntUks*rIZhV7OO!?Yl8PF*@$8q%@u=q>L~lQ2 z)9>VHqpE_sXNHDX<<Ng!7M(I734QxyqgnIj9NmCfGXB#ONonNfIl9_E6<#oCO?nk2 z&0*Ye)cXSr${YBtHPdYM*5TjnCxRMz1JuTFAw<1h2(J9>iuR4_$A7fvf9{7st3xF& zi815xE#bh!WJv3G%Qc4ds;EYMex;<|#R!Tj4{{zqc|kYaAx$B;S?OlFC;~M+1nk=1 zQP;BOxkXD}+Z|>sA*L!I10&nc;R}wKz^K`$OMZlYlB*3LNs(;xz{B@F$wLK0ai#*o z7xbMin&7Q04AIz!efh&hL3*_aqNfkvk{)?f2G;u@(5VmwMhy1vj{Gh+)MeO#?^5~i z^=7b%*{24-viVbZbd)n~l-oM=oljsuqw8;Lr_WUkSo5EDG{F<Ayf>Ck1zHr>%W>Op z4-R9k|5v>z-mxw60VT##j<sPDLG|#iB^jGv$T9Y;F6zZb+a|X`Qq@9#KzQxG_1e%= z_h3q&KY#wzAPSy%EUOnpSYs(d`VCH><V78y>b|$2t!gp!*53H#w3A^bZQCvqayD={ zm&zlMu?rC{)H89%vWD|<Ag{N1H&a)*M#YO=>En5yxEra7EiBM4U>>PK%lsDLJfiCc z8Q&PeO(Tp(MmrnA(;%iYFTmBZ9sB*@rvPd5H;U%E6ob)Z)L6x(NxJp3q=kJZ`tg{3 zhE)$({@ON)Uy`1Y)v%J+u>yN4@)Zp&l(bXuFPTi@x0Yy5rFzw7IA+Z2@r+?fm|b`) z>7b(DCrc@%*Pe}6j5J<vHhR$fr;~}cq+MuxF(iZs6n0pwK!@`z#A^boHER5O`_gX* zUAOD>EZO!`>^{bSlY+Oiz}<B^qD#CNeia|5TDijpb`~TN|MSdt7njpj1y~@UTB3gH zu+*=RkuG-`bqC`Rj&!6Y{a%wx$G;~V{0`bnH+WFq@3hM~dmQ)f-2^xmvQ7Pq3UM}) zO^q|Y%n$2$Ai(#JuT|k^wuZ7!`z9mY*fCd8wN{y>(G#u?P!3AE6z%r<;>2WL5lq&f zy>JCQabC=4=n>FTzxd^vwrStmZMeoA&5p1RY%y0DrL|N$Z3u><scswa$D38g<?o;E zXym?$`=(2rk5rLSZnlHan7lilazvi3_!bz5vF}~RuzxW>Q-tXc*WL7?2$+2|sY0b@ z<WCM<D9Hw*RnQI}EN3ke@61I^tG>-VaO0pfv{;*NXc}AX4?b1jM)$R~OL3S<O$r_; z)FYvCE1SY}TKE};0iuUyAEGdOb?aYZQZzW^$aARbR1{Y$f32*o#>eph$pdm%u33(n z20TZzTP(j!7W(NqPYjE4jB^KGP1kXk$OhPE|3<kpj0zOm7dV<AK{|K8DTs37M6`8= zj}iC}20sO91xFH0Y8ujX1xfRsNA!se_ng=dtnOAnR9|LUTIbP_AxFYoz-r(XFfHOo zM?c#H9&s}<blN^hxxH4%umA+u8WoZ@XW<d=3FOP=H#LPBp$lU5_>PovPS-(l&Ac=7 zFOQpRV-h)&<&NHo+T9suT>gIp^h%55frjgm4{ZCcvs<vP5YU=6LVBjpC}Xm&7;m)m zK>|l0pckPNp^yD2Ck2XjW+G;cMb)Z7bT1|-<-mL>yl=mR-T5Td`y*AavO!|-AOI{m z3<VNc5XU%?+{_&e6n?>QQIC`UZ+0fz+RJ=jwPlR-(;N>v*9_>!7~OaFVW{UC<^#vs z(xDQ%<4TO&9kTfWl#WhffKNcTTQ+uP(QE+iCihtji3#lm5TpRsJmoe@6wKA5;ey7P z?ZYHx^%G8_by5r>h1w3Z5ZC%-E{W(V*hiGag2X&p7{6>!Jm{Cz4dQq?;GCwC%eSl& zG18$Jgh)y|%-_(Q0+_=^ISByMp~Z-`h`|REO8UYg_W%RV$erT6rnwZ{jdx||kU?qt z9?>*JVxr8Xv~MJv?pjV2$91vppP}WTs1$C@@r{=>zSv~?pSzH`fKXF!j}X+3I;$T; z@jPczkG-<c#(NpBr2wtMCOWtk!ojy`WTs=h;AEN0NSIya2XIZy-GV{KXfn~${i?`0 zM&HNRSB3yS3ebJ2X}&ffMW()J;HQZr%H+CJ-{nK0ci&WeZ@SYZxo_kQA4S-UM@h{w zkmcLDsaH{26Hh$|LWDVo!Bs5lm*4cx+GXLZ;d%GVF3VG5+q$NC)Ng?nL#i$b@?w*2 zH+jR=9lv~xjmZ)E0D}*`vzQ@5H#T@1vt<JtFPk;?N)h^NVwWKG+-a=wPEka`Ky_~@ z$!2r+qP>K7wkF_LpP0l+Dj4pt0~dt@S%sc~VlWXPXlWmWpXqfryKkAj&uXyr9on~K zbN6{jD9cAQ%y-D9gI~tkQpFc2Y+T>Ah2|jq<U^JU%?Yoej<R})HUTKds+IP?TO;d< z5;0Ph+d7xTC)+x&k_N9c`624CvnRBO^9pfNovVJ5%<(mY<$g`~uX47=N;0*ph_LPi zHrxHrxAn<Yr@l@s4a^(^Zt!XJe}c@F-d>)e9oy?t15oh^h0Ic`ckaj?9ftp-3Cjdg zo`54@r_!XwZfzvrMFHAwim_?zbe@R8h^4ae{`7fSFwPqmLCnIhmAjLot7L>`xR|lT zIaKWFIa;|Pzb+<Rvj0XcS`|FPO4i@FNZXE>gk8K?V;-5{KrMv{+;`ObzK-ZS?$BN$ z$*xP>D(&kXoTGi$f%)T8_54jXe2w1;3;o2-+3_P&zXJE;CO4<ZF9ev@!3va?nTLiX z5k2yMx{5_ksl&NKy)4W^I~n78(#U=kPYYg|qWIqpn~8v>Q{AUQBa#rIVp5SVg%2KO zPUR15QP3djI9bZ9%wa)Z%S_X?kZKfBeh6ruiyuYvOmwoCPmqmyJ&}bJkM*Cy^*)jp z!v&{2j1O6R`S@pL2p8rC#Czfp*7T?iAc+_`_P8;HR0;47&Hh!jT?EK>ri*4j(+xbj zFYNHYu6#~2r&sgb9ScOlJX`NxN8gx04qFNW#~@w81y;HA><h+zLi=m}u`ZGuDQ|ES zS35i(PbwlSo40jVB}>wa!@`O|bMY%4YIRA2xgX%UH@ylaIgus<4xU{FjkD>oCD+)o zaV5~b*GsbWCs%^vAQl_hd`dGmiOJKa3Wy!nFL~#!tqv9rG}7#**dP&~S!X35ZfQLP zJA|+xcp|SGpDNgFr1#wog5dj|3?c`iuTnvWsY@yK<zwY<-h6o@CncdCt0?=e@f))H zPGNYJUF9fOcJ3M0{-6PuE|nizyF>sIHUs^h%c}Df4HSR9*(AM;UVh8+)<Von!AM-i zrGpj)JAh+i_q~Zzix%oA7oA+st#5DBJPx!3K4=#bNF6~b7lb6Cdz1Eoiyj#h@uyfR z{z<y4DsjAJRqm-75sE0cX>3jQ5=o+-gO<6A>k=DX9E)izh3;6Q1S7B2u9j$GUCo#K z_ndYoPKELX1kJ+mq<#$>JjtOLQ4THJi!<XE&wwFdJ$oK-k?7M@Xv^z+ZRj^7Yj5A~ zm6XCi+W7=!wu_BN!aPFwyRDTF4UZG(pKVkHRY8CZPoC*M;Br5(rptdKL~GlT^_Cpi zqWFJmA{C6xxjw-1D$fpfxy0PyVI_TkBfwKVNrOf1VgDSGTETc6_aR8T`cw`Pv>JCM z^OJay+}gf+ySN^7-0D~Fw&BVsiT<)D_h!URblQ)|=+Do>`_&}{%T*d$$K!!t>M*n0 zw1|vE5`#fKRKCtTbeH6b0vl?0q-M@x4(yhdi6*qY6}!+Z+69y?o5ysBy)MI<LB=EJ z!%5oe-W6n;ZauZyF{Cx*k7L}W&8%-(DR(d0lF)5Le1x4FfBC_2-9O7Ix>J{B+|+QT z^G}&=yVAX~s}rAls)24E0u_mxdcG=rXYMW~nXe>>=ph}p<?37%{C_#_1-{Va$c{^L z!b2sZ`iI?0h*)PvdQ^Hd$a|;|TV`zPoa<V!f80c4%?_I+MJ^NKX&k&ofX!YOUHA?} zITZvDmwfBWzFyI~*Ny{h$06JKnFOAOo9SjQdB>!iQlM#Upq}nAdAXQqoT_t|jqilW ze9f&pKRrfR7OSpz-BJtvh7ra8vTX8+FlNWv35Ih3mC3p#8EI!h0A7a=v4dthw0q_2 zC$l<31_^tFS;!4B%*A{raVHcArUfo=jp!sRBT@{EMi1-45Gz-^?@?ZATo!qInkfpv z6AHNIPm%o`LnG=3fUY3B_Tfxmdu|+Y9M@irCMqv@&NC<>EsIZ!3l9y_up<5+G;Q1v z;|}5FK%PrTy4lrmiL*^wo?U#%6+=UsFm{jilDk#_xFoI~HJH01_@c{iwcpFZ#tQLh za(qY2<3A~fqi%*d0qe@ODCS+}{4z1d)wj}@FoL@XGS5!i=ZJF3N+#m{!JmFb;|Lpw zuVZ=GINtu!Xs{LVUm=Jn!BV(T@o^;_Q8uP)<R7BsifHpwtXam{;O3i;za0Nk#>j3% z7)-H+DaY6kYdEYaryyUjLSuCKG>GvyqKqE9oYmy|JT2b=?jCma{1wMeS~vv}iSeaK z7*X?MZE1BFR=}HzXuDKa`J)pq<**oWY41tba-cjWcCHW>-fLnpQ;QphdgA5Rt!XQ> zOho<k`yb;7Z~;@?8uasU%IZ54w<~wAs+mMv<qD2mec^Asnu{~bCz#~o9ihvl&xD6` zWniDPFhqIiamjg42;cxT)U_hn3fs&XBGk*4tqqm9+53Z$48(YNmVVFGR@QW4i#17w zVl}+-n<HOxtEOY4f@_Qp?^GWMOYAWMqQ6li)+9!$vDHhD%6Gkaj#(&baV~uK!xZaS z3G&{Ir3d9R*G)Y}PWb)DQ}ULX`+}P<^)z(KaQc*+BiB@7c3O<Yj2y@-Z9aMJfth8{ zcr)K1yJI8yb*CF;_n`oNn7|hxjOhE%ONULbjK}mSu_%_^^i#82D^nOL-9boJJQQN0 z*FS>RH&gRB2ZeaycE$=_?JJRamLtU{$jk&y9hT3w(eo)%Hf0cobTgE%rX-0LS3eop zO?-=Mc5CFhC`Ec%Zd<GQ${N#xYamC8px8)`=D$^62{ct(JxUaIx!pmAH}Us<^XS*1 zulf&F4I%7_&_8#<asR>5ii<`qXa!a^T$LQ)6Uyw8-c&$9T;YwVG<{Em1t>C1j9*8y z{skSqZ2xxL(!vj5Uaj{l3{3Gj?LhN#iXw>o1tc`fGjJ2_n!H-CKEyQ#t(ql#;v5@k zuRM%DX{N!ap|#D3>nq7%h`X7$T=s=ZR6Sa2RkhCZ3$G7QYU2Q0+Kk;FNTXmu2O=u8 ztDAG#d6*14V8*C1=y%7tw?SpSFRf(XIclNYh+o`qU3>I*@Xn>=H|;8LH#dn_BObOg zXNP<*@LB+t7AR)U1=#Ja#c+j+c*s-)dZ?aE#tn4*TFNiOYRo>V4DxmgSv@DH!Hg$& zM9QOo0Pdy+{H^u-Jnr09K3(0GZQJ|9EJtszMJ|AW`L6`m>r3%zmZGDw?RdBPH*$`F zg>Tj!lQ%7v{+REs@1LZP)*j<F+y6cdD$hqjXxU2O(Fd^t$*$@U|4r<mc)JKzmwsp3 zPg|le-`0OiIPkwPedU7RL;(V7vU`pB`7unfod$?k$>LTuo}o_-FEJM-UViGg%JDoY zRDc>821}g`)3$F|BkeL#N#ouo5Y^E9s?tM0zDiFlPHg5yMzrJ+`n3I9jFA!Y+g~_e zU|LrB$EcZd&|LoeR+n=1g<V~4_AaWoDB5pHPXsD$@$qVT!<q>?65ilxtds4*QT^#P z<_3Xc;$>deTq_75h;uYlGbm0g)~)4o-veiq!2pPBF4{&^fu34T^bVM<>*GyKT^0K| zb*m}Tiw<c$_EOPVZu=~RHxPXFg0xnN0vP``$3;c6m|t6asl!xs;D>0kDOSdWG-M}i zCC>P$0p=3etVLKD)!Y#Su$SaGPYnkVAXl1>#wx%1>Smx>9iZZTSAUvAH}$YU_0p+M zk0?MI`RV7Ddq2JiX|SDa9K%tH-Q+dJfg=P{9RHhW(u#k5ST9igrfL+oD^rb#?&JSt zZ@c$^7ZTvB_&+@xp2HRU7EvR{i!IzqG8~aP3|{7-16H7^OW9^*Y4rH-(vDq~_q<A# z)uN<q#ROo@Zvkc=F-PR;FQdmF7CRh~`c@ORIxA~Ou%*w@<NIOz0D_ziyzK>eFyfg> zfSfv!NSsEm$r0H;2XoeHbOw}9tXxqrVG@a-jgDtml8_vqP)5vQv**ez3o86z_0eum z^(kZ1204{XZ!@_7nZ_rw^5MFi&kdx3H?~KltonOU(sIG)`ffLlMp?(`hIkg&vp67! z_*K2+F|R|}NDy}pG08jcBCq^Q%0Y@v;<qr}POvZ&O9@Nnd=5fjyI=TQUw|x$_tkl= zxl!*}b(0)y5^w*wiE?x7bi=-*ahfanST<S6w31Id9#v;Oubo4n8*CRdP7SQs*v6dI z0;``oa}|Z7RQ8RkZ@rbFusN8Oe}aDZ<!;8x?|jUqC2Uc?O;aH4Q8mfmo)}Z4#T=L% zBl8A3)QwVcxwg+Pv#XWqH2pDW4FS%H?)7Uft6K76reQvki0=@OnfG=1rLLx9Fu-;7 z>^zT?zU0#$2Z|y!SHZcmO`B~mK;eI7gs%w-+({Z*3S`aU>I1>0KXJ%d6z(B8ET;=r zbzl2pe%iFmAkmIGD|9^mxfFK(yJrM(_5NI2fge=_bE4dEAnOXqAl^r>JW%3(#w4Qe z>`KjR7prh3T}-s?HNN!?5$wcH({zd=qX2+Zk%ZnKR!S#jzL1}rjlze13>9jlNuA*s zWW}+6+o#>`+yAmR#+yeity}fIo66fcg0_(2a9HoiAL{2V9py6#h?>1|=BPdR&Zhg4 zkFWe^KBEpHW)w)!OHe|GojLz0AdENn`%jS;apIbT>nO~v$5XY7EBlWHY6>a6r?b2j zIHbuH+K?5r+lIC#hC`pjpN9VjV?mt0&k9%*{fj+HMpTWIN|~!rGW#FPgPvQOJ>N1@ zFZVjHOYLZdN?^Z!8{bL-2%+A^Zpj<S(yGNkNJU~0Fh>a(+;&{Gz7=<{hNzgz=uI=P z2bw$=uV6$t#31v%a4;ueU^P-8k<>U@EAu-u*0&-CK_UWl&j!-<%$O>m`hdzWdRT@E zQjv|$*(leQsb`5u&*xo>tx__QD|1`Z>aT?Y0YG}$H;*?&tdc{5Ql-08YR9TV&INpZ z&egS(l_5WC2W)NTV7eKXXMTZa?lRShL4Zs#3Bco9ZV6OXW5v_HwD&S%o;wYW8FQ{E za^*_-DD4^IBQqHY9~?1g?CjT#f0dqvPTL+_$sz_hXD;r~>3L{rcRuX76b!6Mq|ckC z>pTgWVa&B?fc<0dxYmA2)<_}`(~k&649;!OsI>d=+F;OgHN+D=fjtD4|93S^)#}<= z++T>;PO;wbl}KW>Ru!EVu!qPX-Zj1nL41<ytj{)DV$ci52`TuRu?ki92}BSO-=e)u z!*q=;amw5nTJ4kUp$y|ouf}CwM+;3_cYBi7J#xMFW5c}<uJuaUSia2d&7PRg>__k3 z5}UIB+hjpwZ3k<THTcNv4--LE<?1GDG}f~5DsjmHv4VN3#NhFf`rh}tugm=2jE|}c z?X6a&W41-5+oLI#@r4qr#{WQuxO3KDGSjs&EL9VwGQJT%a*S(sEf2}_fhV$g==p6V z;z>fZvs>{5`*t{cm?p>0HcP$n-)V;yt}PE2+$V}T_{|G@b16F?<ADoeFbD_`;FLV> zy)QKr6&?vCvw@JQOz)OA2_yyq2D5LmM-QsOGM8h88XB|M^M|vv40ma7(JX*zmDE8} z>G17=JnX%rdfZk$V>{4S2>DZ*twuv=hD8jTL*vM@OG=#un%#SX7&QOcJ4VC$nG#Qd zDghy+&?Nq-{`WGo8~Wf%Mj55ljZ<~1-<*oz9(YFG^X-z=^VGK2OE|<JP7pz!y76k` zTn$CF!Eh}&r^vKs^gnn%zRWkAGWxin1<!gqZE#I1`zt#Cn`U09G=&vhKiPkU&Riu1 zk@-0B{*S3e4C2_fPq(Tw5@;RbWM&UM)u!E%WK5n0ti&MC47p{RSkqsi)Kw_eZX~bb zH=l5b`pg_f(#9(-##(5fZ<2gxrUs_Z1DLzv6N5kMfsP(Ew>rO-tvHdRIJnbW<MK_% zr=H%Z33fPB0Un!iK@6S-F$g$3cwgARnu#)%XIb6%Cxo#7WPVnUmy2tCIAl;g+29_? zz9st^|49G0*~ea+Ig|?fQ6!)W0Y8QpKE&F=s;UcYzS>kGaBgQtp7o_`h2)c-G~*{b zl<<fLNccfc|B3NgtjRyVNL6A`W#Bj6nzTVr+)#dPETI2%{Y0%~m}h`&Z9EU-;ShtY z$GctEG?O>hfD5P`FV<?hSbA}K!MHE%&T6uyRviRLHb^#BjPG4KQmB1LL;3aCMEAN& zdyQ-5O9bb0{Im9jW+@%bS2?|K=Aj0&r|KntAlv-5C*wX`%<N2>bz35BGfg3bHV}{` zc+w*|n#4p@Bq~M=rDe9x@3*+(vp@_o53&cJs()*}hd_APb4U7DdNlbr9H{?nY%^yj zWd6hbooMhO5rfQGVpAX0{?4317HoaxWD$edeCAT&aE`WjiN_wmJSiV3mfpp#pcQwq zg5M$8J-zR?_NgQJllT?$HrRaxBL?xk*b~cLMe3&n`yLXck)<D58?t?dR7|0)w00O- z4@5xRYk0L^lV;=9`!QBv4~a?LcifPH<$f(Xus+*Zi9x=Pgl8)DX@O+tr!$swJJXtU z203fD5Axhl+Mv1-iR(zmKc0X{D{|*(qEV<AEN7F7u_Bq;oH#M3*pIdkw`Fd7R)&@b z%)UNi?6^NMn`mhKZ+}^GhMyRpe%ki=ctLMt;tYwb8jcmrQ`7@rn{)iet5vz(c~w>p z7f&P%AuCEuwr;8_W~CXPDa04OSC#m-jNkvy-g`esmSuN-f6MAuh4hmm6#7BZ{-RK5 zH#T;)(sFl2t|7T3ceo_SPjh;Dx~HeRtGm4SC<7F{_b7PpfdmrXCcO6^Rc(&$=VT>1 z@$fQ{7Rc(xcX*@NcrTy1@7{CIJ>O#}0aZ3qB)xNyNn1+4;{abYkSosFE_H}?t&ADj zl%Y|Bc5^tAVBHR64TGD(5rRfQQ>4cWHgRmex9cx!Sq?jAf*%Slh(SMar<)p(dren0 zU$4jD%p_37Aj-rb1D(`(WkW@}(T^4w;CP)Scf6G5$wu{b^(?Gxo)8M;D<2>oRREjs z!*33xezIChhVo=?BX|NW5MmJHN3*NZ?X|EzabnyM#*H9rooJHn1AC=Ni9xfO{J7%R zGO<2xJ$wT6Q^!<`)}A5neJp~pxqYyyx$DV?Y)AG4Mht$Zn`hE_P*Wn51;%rYrl~qL zvVH&eZ#8{Q`Dlq2%8iAztSlnbY4Ar3f<v<?7WWrOwlmEv{>R5Q?;C5xVB=Vwrd?a} z5$Vf(&hAdv+~;=DrLhDnF^Gahp7rAXJn5Wi)d<wIu0%8U!&AJxfzpE`K-uW`x!k!g z=^98S#rp6=0An{$VvuvG`GH4feEU<aYZVEwIn$b#t<=06`#~#S9=K&j<cUG93n9d{ zz1K$O5C-?{ZQrR{`TOCan^5vThq1wL`PbKPgT;9hgWf<N=v#0qDa8D?8ftiF9HlBp zN;TzRC<f)T25S*Icr2_(;Q-mddWNyat$)9fnT;vwTkO#+4$s;jY09in^b{-5O{i5y z4HRoMkT+tL7*t(mdsYdvj+vGd<2X{_KBHzHq<1Gu>tvIyaV;DKLJV%~t;>zZ&!ujn zGVF*XvRi1X#L1el>I-@U{c6zc6>wgn-e5EhD1m@)MqM8qDl7U+PnF|DyO2%dQQK{G zE(^xk!n;?Gl}X8P{<{dX(}B^cXSQ1|*M1sw)&R2HK*F+TKJC32E3MN_M%TFO*KqF( zWY@{*O*8xDx)aaHo!<vy5Mqgltm47^fSu_`@k|s{pb<#CKRO)-UQO8-#VGq96ETRa zg`EDl?APvEVm5N+tcXFrF;4Tv1NmmJPm1sG?#H`edrNgBn7Q0;8x0vL=)T8XpEQjR znr~{EkXN10B(o<;<%ymB1o$HcG5q65kNu2(@h1at_%H0s(S5AlIvv;iK<dV;+_fA1 zT*nSXEKO}rXnODUgY~ddV@C?S1K;z0^Hp)qIg}){e%o+~LBLwoI8mo*|Ljs#BJ&9f zAmbQ1jNKZ*bB(PHo3W9h1g_opIV<3Fm1}g;^06YdN%#XPqNzvNHraekAI26-dR5Y! zzj8cNon(w!zGwVite3Ov(~{{-Q61i&-q<|Vpf)+2AFS(atblO}F}Sn8tGbfA#_`;- zJ=fO3mb6VbOU^*LZC%Y^K!2%Yq~oa!EuM@D<QX7S9kY?C?aTO>H~We7-bCH^LPSzh zEW`XacIJ)!jj-mb9r)JhXap$Dug=Tio5NH4?BJUNeV*F!N}d1Fi11}Um}u6AX0y)W zL=x%ujupfdF&Il=5VF8GPd2D$gVjS016%Zo!MAFtBHZ1%&}ls!G2kJU(VMIVfr*6U zJK^1mAR}xxq3ek%OoY#VA0F)+7<AibTcl_pPlgu;t%pxVAjBYxBWYu5Mk_Vi(F&fS zd7?r4jyf84iwXj7aH&^Q*P%e21_Uq!3LZpFQ)g4ZC0H<SS$N8Z3#G6>$2M9rAYzd5 zZ~C}>szn~P-Ew;t1vy9E1mLpNx+*2BFU9Nc5Qb|~F;*fCYJ>^Tpc4tGva&I4#%zw% zPgMC#|G~H_9W0To?z9t8yu4Q|5re!#?P!&z6U7K%2ck=|y3#du8fj8v8OMo1cQh)B zZp~<gOs`K$&3J{TccR`#B)}ZNKnU@OEc7@<Fjj~J!05&LU?*GNH!yYsCkDS&MZLH` zUviz)+Vm}_gMrZ!WXCAbIXxNDHQR10yMAbJ%Z$hogUFrr^$n@2t(X7uvk#=9u}Mij z?z6nIBKbuna{2l#xqj=eBqpcH>e}(BXTZH`>l!3EHC_LIuw(v>dmb<4l~sD3VAld) zh$Ui>`Ht)$VyI*=-{=*`qlOsgI6L7eL!|I&8<kjqn;3lmA_fBi`Wx#PDFiX>K|nx= z0lp32YkPl7&mkIlc4J2BT~(^P;rQlDZq2MuX>Frivx)bISS1F*&f6WWG5DzEjuKL& z*0QD?3K#T04ml5+Z<zIW%6d3#1VRk5Mv#(<w2(wEh6?KT;DMdo6|Wgc0wqdLgX$3P zj6zRa?rZjp^OPIB7d40qda}hi;XGB2dkBcTjaRj#Krlu`-n(M7L{qg{>)`l;rlwsT zlRs4cMl;v=Bj&w9pAsEH-RJSO(Np^v;(@Xt_ZqJUt@rYbbz{{USx(AXYajAo?*lQ2 z12BD%<aqH;*E;=K83#m=+STOp1@1!~@bsQUcMZA#?h^`rOvE62JL=%xtiR+l)#h0d zgMhhCHl<W&LQrGF4(z9h%plUuU&g0M0U3vFixjz?bFI=cRWHeAP9RH`x0h_^us>oD z8_|QtThcw(`Hnz8`Fjsm_Leky+nr+Y_<Q2qezp2@jhYN4Du?IB7Id@WOLb(A1qW>Q zj@WyyH-4s3lKVF6LL`Vm_U+WBd)Ri<)`@{{PJO$g!CZ}I_qB&dZ^AB0;6$W!f>1EC zJ}vouM5tGo`Ec@myg@cDoEu*my=dWGok4<3>c?yJ8bo=ahd?xD_a~pZLpVBwYk6F9 zdouM|{jKpG<~cK*$uha_Kl1iG!8k?6Ztw3%-=b5Vbv)2u8t?gvjh#ME?McuG<#5I} z#2n6DPuuTkih8R~Hf>}XQFGXCBjH@lHE`6XCL}>Pe6RgCdpa+0_*vWvKAPD-zEhds zKGm$HBgkxltRPd2KN*OgQ9e>Eb*{>&l6Kr9qdVCh4IYF({5~P3h(R94*G7;}j52rH z$M#&y8{YR06bwKL;}S7dVY9J03u2JPiyZw7#Fpts>*0t1VLNFA$t%Zmt)+<az`NsR zGi4UVf<cOhC#iA*Sr|wkA~;AXK^jU(34em$4uKGZEd1%+Nm_t)efOkslfD~R=#%8` z7n(!fZUQUOx6oyjlt0SM*6D@~-UFin<OB5+V-bA2je|8Q9L&*;t6d$WK!`!09%p83 zJZ!#c7NQ}aWSK9pSWtAHRJfJnuaExDGwzu0S36N5brTe_w<oR65~{4MjA`1Aj@cHU zN+1eWm5~V$gLvk6gSin)#2^YE2JX(8HlJy%_@?>2*+z+O^qV#<7T`?`($|Y-Y`yG! ztVTB?A{kFFyB=%(m`K2rN;=1xuEr3LlEuAoHfV6w7};DBkh4UrgLev)7-U`}Q;YlZ zrL?cudN?dVzLJKgX1qM?2$1&yZkZ7oVvu!neSKYO>+0px&%c!a>3{n_B{eNw4eZFI z{K69X?9vsvdgHd-d-zyB{p^xdR8}9;VR)XEl@*CkNR;1x_^~XnEX(%xjuaFY%Wr=B zd#S9hQN1owh$UhWpc8aXw;SCeSsJJ6j{eS?h1c%O_Qx85jR)xafFCk3=mBf@@bZ8Z z3}l&2VU-#w$y7l7xs1LfT_2hz>!hGBTaq0wwUPCK^w}Px9mamve8&IC;;`iPWT_1D zhgc>Ck<qMO)KE+8d@k92X{z62m=D)*>jC-&HP9b4-Bd@fNFxwpkf>cE`q^thB>4l; zAH4tM+JvUl;$MD)j><ZIqwXuMFUr0zlHgl#U-a=8?T<oA3l}M{z7knWHj&ZUu~YYR z7;+g$DV_1U2ega6@CBR#-G-VfpZc^F>_=DjmgQda^`MN|Tptn=5}e=rGav!_AN~Dq z<2BvWoz@TDU>u=W*SV@~*8#5dJ`jVDqu3Wn4;o0yVf5eI4Oi7D$v!AhsEuLGrM1AF z3y!Fb{T~Z4$af;@s%oT6>)7qQIXMQwSrUV+1vrUZYxvxj><I?wU__bTH!^_q;gN`X zISp<nqXW%$Mc=Zn<@IK2h6uayY9$7l=R7C23`+83cBSgt;>-ES1MWp#Yf=n)M-p*8 z)}@cje<RLChrIbVaJlOZzQxn_yIONt?RTfl@o0m-<T<{o`$V1c?c{qTh(X3<$857) zHuu@J$smLdgi{3d7u8w8$GwP@BKoRstSTs(7imnbPHJQ+&M&8BZDV~R8Z*i9T=ibY zC297Pj2_6iUfEgFv4$h{lh(UxFY)#M(E!92ndr)l&PGJ7-Td;P+pHHvXlYcZjSV*z zK;I!_H+MIc%u05?R2|ZLuI+!lCyu#RN$QF_HH!E&V6($v;$iEJceS;T_sh)Iv^qex zO*Tb@7-X)HmNu#TrI`=zask-Fn<g66Rx`0?Z(k1;x_2>hG|`M-vKgdxy;9x5wKncI z8XX;`)aZZp6V(CfGogYkgHM|uO1GI~Z@%$A7vERJ1Tlz5`~1R!bagwmEi5b^DKi|$ zTirD=DZPFDGCDSHn+D6R1ntSv(vqyb^MdCb&YhT;kVW(NK#ZOqFgP_mLciNv#OiH( z7Q`SE9&c!rhU#%@8ASeGO!@{Rq;riDgg1Z(7{%E*teVXog;R+b^C!T)!Fen`g!L8< z<(wM&ZpCTDpqu<M3tiVUHPUVFog_36RO73o^0@7`8jb9fOE#0vxlXy={FTgZpH2+Y z{|Mp)#}g$?qa}F40b?F-6;dMAj#b$jfdVB4JsXhnp<*c<$PPNS59gvB^ZRzAq<MoB z5~OUxXaLD_-f$Zi-D%=b7=3vsF^(EswUSNEjK!W~$LRZ5AqM$&D1U_ilM=-j$%5iQ zDz^8D7^H9cUj6euYV^dwi;N*<ZDRNHh(#};{NQny)cI29{ZT6E@7$?#NLBKBs30QB z*Lg$N*&v8PWGl5Xs)qd=&w~X@Z<DKD@&+<vemq73ZkZ7wV$jSBbf@x)O1b~&iTvlk z{2NJ1PEq}7Wp!09UA`t)uiuhBqel&ojLNH|6uENsrW)Wq`V{{zFE7jUm+|te-~86- zTVqn+&?x`-yN@L)HBIK`=Oe1~#2PWk`?F?I+Xbg0HqFd?)`Q}KTuE`n>)4OvMkt#2 zF%W|Q?|LHS$V^Z*R;D$<iO`DgdMsr|{~*O6Uev8dm&ZFFgKAE1x*Fdw_zaqjHtPs7 z$1&X@dCn|x%(Po?a}2Re3<7lctv6fJHQOo0!?~LJ8l8{`ut?~xcnX&c6iAvQQTGaw zMj*r>aL#n8Cp_uCe7zT`h1zGD)yP*lT&9^68pdj*cD!20%9ECRQfkHtWL7j%u3;di z4p*A<l@t%<sbga>x^$!fX<j=~srA4n7AAc5mpAa<roR%po=PyaswOtZrEIuJ+DyL( z`mQ_=5gs^W)s9sdz4fF~*A5uKafXB(43wClZV_p4kp-lr4fGi)O@R{v^2V-)J3ab7 z5QB_KGCw?SyQ_WSYtI9CkHwvNx!-(4CO3ljlxL4eR4?Bs*xv?<9}6+aI9=Oc)l_`# z)2%*<g0mzBu`^J2>7(*r%61Sk!VVAzQ)@1H&~jb(U(p&b{xC4#rw+4CW{%sDdsbqQ z_s4LJ{7LG3Aw6^5vU_-Xlpnff?m(Ip3}zi8sQAXD?V~>jSNhD{JoyZPfOVOAbLpKa z8c~1V@`(3>q`Fk~dv!3d+dD^s7=&;nGX8SyA2j;Z3hW{112{}+#I28ekyaiX6^=)C z&w5rETN+hcbn6tQcl_&Gj5+ijY@@7gIKK4GJLN^&6X~Am)N%Rd+c#?GA{vVb!n1U4 zMqXjNZFJSDV~wxA2ZKxflF;!)$I5xgfk43XBV#vqH>J_^IeQRyD*Er=i@+j?!T8R{ zVb4QsB&0IOA<r(wDx3jM>!kM&HkiW>JUe5EI@zmx%kS>@oz^g{7<#P^T!A`%1p>GH zpU@I+?6pT0h9sjqNz=+I;p(IBke|gvxzapcuhE0v{U1p%|40`L0ZAsw(`5Ynh;t_f z*;r$IN=iwUhfm_<k!eXO=`uSv?+(v357*q<CUMVS%HyZc^xvZ5vJ<s)P{OCDXQZa4 zPC7ardJJP*U0s8uq^66btNT>Lnl~8egppI6dxX?77N+`ah{10)e>}C=N`~_y=&?dX z;DOdn?3(W|Li(k8me?(z4EkUPzV^MDz%{N4O&>zx++Yw0F^FPJ$Tp;{T`EdQJc<iH zY`d-oA8OKAL4S74w8@jUJ4OL@r=ao&zQ>Z;KqYp)Fv34gl`gALK5x+Xc+94BC#Ygz zw=e`s3<5Tbvs+U}NZ)WPayHlV1-=0SzI3q2EFQ;6H1%9I3JLl17q;e)9Xk&wF}QR( zF^F7X%u!#Y+sqwr(D$)I3^Fbua$j~n(Z~s3<UGo5kx@AD;If<lu>@~o5Q6sgV1X*j zc$f0uJqul04*)OENB{)@ukxpD_ml|qj0wIon+fuaLkym#sD}a+b?(8ngO$IbfOiUp z7^EnBet)J>J|i2+&U<6XPIA7G^@GTygpFmmWkzI(L9VsCyQk#f*x0yy_|Yell$5N_ zQ8-n7_QjWynwDX%bD-z7v~|co{QTE4??v`;jpbz{H{uiJ?|%9V$tx(7FTT7ix9;9o zQqb3$94b7-V5F}b4m9=IT$@=#<2BU~DPWw7opQ7OOKG0054c`dEa1mN3?d`Z%^J*R z?UlpqBU)~2jlkvY1toe(Pl&Muoe!fCUY|LAsgi8IvBP|8q6iZ@;xt<TwUTn388W_d zJR^V=u}%znfJ{Y4Zl7t9RHGMS=(1)&2?glY5KMKW)z(81L7>Fo#Ok=DI}^=brrkzw zbJw{h?a_;l#~Pi%UZ>lvF}?F$Y7{FN%2qO&J;ll2M-J#(=o1fGZW#H}uEt4UMviEK z+A!2$p$x;^N?>|7&rQaJPbz<{sqX@PW}Gg`z=3yrpuYhah8o6dbspoG8i@XbAr7(* z(({`D_he*K8Cd9*IHTV&CvY|m#Q2CVh#|jay1_QhAy4~05Q97$hQ4Q=4`gU@K>OVn z(D^89LWBuXG=Y2*Z}5FuXPR`+9yvS4ar{__L5QmP?RiP*idV-LUt{(xi9wzTd(wxm zf1&n=aO5cCV`OPqp0(eX<=w?|v9V0AO-XKFhFKf3Y@N!zi9xqrXi@U}vgK7*oYwUX z#_^+YI8(CwQq<n)`3?U%XB~32?lW<%j|O}#D>N(Cyi*?3-;#;-@iQfW?7(~Agz-iB zN4gf;$%04_gV<@PA$Y6qvdnBwpBRs<gOmX&8!7TxM-t#DMx;D}6K9=r(*tBJ`bybg zu@XXod_!ac0UhW)Sv{%h&{*78ATz7eX6(9uug3=q)SkKX_M{CpTEL0#cGIPRQsnXt zU$#Hd;0|AM?!3V`brXXd8Z=Nem~*1^7=cCIa~=9FOFq&Q2MTOLee*q<u9#G;R@uOJ zZF1E~vkUug__is&7vqiyYHWm_?=`+Ws-?Oh*ds}rlcp2``+3_VSu}EWV|!C;A(sso zD^ci6Pa&|WXed_+*hoH)9d0*%C8Z;UA=Q3;A8_u(;D$Lbx1dOV@^?Rz{DMNs$t#dQ z|Er%!Sw*F+uB}O5|DasIc}ITz!S5s^D@ShJx+{P3r+*<+)6;4w_JF8EJmFu)C+KxB zAh)!%$wwdmQSLo>EW;xs0e?USK3<FDwS_dIxTbaTodq$d{bMF#wX0l;hx3%oiv$p` z2$ZL-_vGWU-^ksjt4g9pQt(2W^)K|Okp-a=3{E8mzrzT$D39818zIncn?s!MMH<Wp ztvA%8!H&>Ec#!He(eYG`N~eKwv3a;DIYwbjG7IgfS+pQ-BiV1^g1$j9r&bE@X7}E~ z5QBiI!N+DpNa}&P%~}0#7M#=G8-*H@CcigFTt=XKgYm()t9MnOngZ%9z`G1C_Dl6> zxq2yC=eq|e+K!nvjW!_vxi{$hSRw|gk@ckg?z=h-_OgRHP}rNRxxyh5V;SDWAj-;9 zBby<n@PbG2C*1_8t5BLE0X7W%3r?-GO1)Xn`hgt4*a$I5%`smdyP<;gUDQ1wT`_em z?L1Z^4c;wKVvxClkvY3JO=Wx}!MhHVKfmLiEbpCvByqIimKl*D20dVYEG@6dhrf3d zgIv3&wqCAWy)NbD6~Zv)JNNYu$bb3C-^<A8*s%-^5Q8sXCdi-s`~M(6`{i%szy0** zGC4Knb3OKkRlhr}QM4W1&0|E3jrOhT6az~bjqj_D^hx<|+#92JktHX86Pu0i%osvX zwI2JSHNFd^-?GsS*+4?P@Hfxi!~46jxyHHSXGMP{zc2fxWcH?N9c{k%jk-&c+nb@r zBXjDB`*@n}0nFHa^Cp(YuJ4A<Ry3R=={>KsMkbD1{GEP5?s{4v<C}BvUQg?OrI8KR z<B@1P@Ad%Jzgyet-m2seW=Y4KJ9>t{d1MiFNiZmUT=8plup~thnE+VBJPjQgJt&a( z4!3bHp65mDW354j^AGo?-_c>L=SQ-Eh0QssAFtL3z?F5kQRB2mzib_B$R}kVh-+<B z^?K_$)&|h+ah#$Si}kp_#y4T_cCG$1qtgu?MWb5J^EB*KM0!$Vlr*V!b_QP!htY}E zF}dGzT@97&i!lK64Ayf!P3umXdv(uu>i4uBr=h2M_5s#ovD9#K$>>eg=mK>@vpI)G zf3!?C$n*AxN2%w6X<QrQF1oZD)A*mwdEwy4LJUHt%xz7}YcmGfi_@2OI7?y>8NvSX zv+|G3oF&z4=mzOGZx5w=${~pzFSO=!G$Oc919~2FJiYgot|3-<6NAX_!R3Cr-*`h@ zE2HoBYxdv74rOBnTGJRg<gq(2->Qbnwf-mR`_M^40dtB{J2-W8&bH{j+1GeE3-De` zTZ{5#<tM76MT;1uABmibYk8pStT(yKb*I-SrDmi;ZDe-DuLn3#Q8zBWBhH<oI*exu z73eEC^HdI%Dq-!*Hf06+1kP}b<A+T*rFXVlws*IWY}V*XS`)ARfjWBxdfjlq*n>3t zqUJ-*WW_$q-*|+~zTa?D`WAZ5#10Y)7!%0Y^__LiPEs*ie1b$~j1e4>-1$;#orlxc zWGR3sj%$9Pk^a^(xxBM1EfbAWKUN(P+Z0X?owIFH*iWX5?+H(A!DPRvo2-(92yqi& z!|3ScMn^6iC{`OBYuL=%jOvn2Q}vp?&~D6(Ee&giioV5ezjHbgc-nDC%0>${qBxvy z8LK!~Vvzq{URjmKrWPqFDN`@Iwe@wmb?3e$CZ%Y)tNel@xpVJ<)YUiWVmvc5E5HB4 zAEmhDwbs7zz(Wki$0tf|UV#jajL64-{7i1%zN^;>MmgfyNMX`B(=KuC_tk^kI{D6o z7!)G!Hl%v2OuZq=QydAf7~?U0tLd_O&(3X5A8o!O1r&g}jakjjO~`F9IE@(8Xux%s z8gcMqw2Rnbva-p0z@jqOV|xzoOxn5>=SwB{0s#W2bGl8QHs6;ot3Ed4<DQZPu>!^^ zUUmtNI4w?YCm@3*2Jyba3l0S$tvg<7#>zGI7Dg6dLXqbnYLs+4uN+S`s>>VSm}?F% z4w%KOMkDp?1pQe8FD$-m(^!MduOEL_Z!oTjAkOd2GW}3_>=^GHYsBEww!1R4G+;aa z0pp`;q+HYdyf4I{ZZ38gv<?kX3mAiW-|3AhO_kO;-Rg5(1`5U_DL1;N+a;mnsoUWA zZjACT=}<@|3PEgD0>cFx&CH$@`LyB#t#jp0OA%e@nim8y*gD;)&xjm|By`;FEShh7 zO=h>kQ=s1mxMfCUi9uZhR#uK82Dx^1ZJm5|`KpwamFscHjQ)Wk`KzD&!!i9xf4jUQ z&tJsLKl>N|xBS2V?$6{e|N1BLy0l!M(;tv|T*uQO{b=5Qvo_=<(TmJ$=C}3ud9`ma z_!v39zPl<tvktA<LE0PZc`@DI8CcNEnb#+d|4`dEU#nhDJ8|q?<Jiu@wlt2{N@Dv9 z)lb-v#&<rE56gZo89j;W0qpM@M7lC|>El=$yT02u(mUsrJfpi%YX;(zHEw%<TXki~ zLLyMM_BYMg_dQ-beOEDDqAAr$t>~|xLTOxYb93t`^2&PL`#hePXbtMVK|*0nP~)O` zx0>-v`CZvBrFx`7>ku<Gtk<Al+|By;D71HdSJay?*Xu7y;XsZ$&r!dW@A2kq-|rf3 z`{tW(rQfXAuLldIX|!IJ*H%uI&qu#Cd#*<fccgu~MV}w}Xg$Zfv5piD<Vo>hff-Zt z=KSMpgTMCHd?UQZ<J+EUswzl7f8&ey8e1BX)UE`rt@E{?@p7^u=if-@RGU0%zNN%h z%Vd+R?X0T3;l%moIR@tY^|^2)JhYi7{Eju7HH|gIdOVierkOj`@2MWDF!N>a#BuLd zF;t@Uo|d*3eas`@jXs&(oY5$Ny76j_TCtwvab5`(KOSQ6tvWIj>F%%Q=UEbi0FsiF z%<LtxpM|=oKX|h*4Wo6E+mm^2b@8w<Q8N(-A-5CF@qLIlF}VNsKzimKnqF>c>!hLn zG}KR2={hjFJaTj&NRYzbJjpV9EPpldP6K@hk?^?fmO67{*guc{^Z-W+NS>QDSJc76 zE(KvEh(SPPcY0@%W_R+A3S|B;ck=o&bpPtl!JX$FT<(=Pvo6f<oDF$q1rbtJBc)m= z5}Rr;&%n3l{YV47|K>pVeXN%rKz5Q9g^W?rj<NzgJ9fj5N`Iw}BB$A%k#kR)?@OPi zt2`Sy7AzR2$bq%(H7V%LHS1gDiC`6+8z4Eeo#~n_!p<%mC>Yl`c4c>E*xH8Z-_|M8 z=9flfzC%714dqF>(Rth{I^7^(iiq3FvDcbDI^6qsFt<G;FFGG88A(5}hOO<aX-T!( zakn0CCmV(q2F)CKEkjFvwsIwsc;UD&<zqz`#GsEB7BL9e^r6U62Zx85n3zyc_ma|5 zjq;0sm8i9F*4EsEf$P3ZaQ|(6{reh39%2xK7U@VYU%R1%;<#%fAY-RDwBY*gn$(Zg zN={GuyFqq7#NgiB9VJ&t+qSwNzTuv0vPcp|M{!S-xo8Y*uBmke#>ynMH^Hw0YsG2A zAi|<+wq0_Z8Se<ElR|>A!lZrK@<bfd?P>tA0`Do(TsBSC2b>q5{v&;i79}<Uk+eYb z8@`F8vj!V@p6!`YWX9J<Z5PvEi9y~O@?~LrUQ_SXyJ|FdIdvBC&bI^ozp=ljQHIog zv!=2{@y2V5)SZyi)>OS`jU!@kyxu5ZzP~3**3eSFDwP=5yumxi5;4fPAsrKubas)W zK+`t4YNcejP?mP>zh$fdnF~3A%q3bk(eYB}xQ<Zf`_8h~kg9c6gr#@^#{BNVjyNVe zB&9Q66<Ckr!@oo+woNxnf!Wa6)tLzuT$?C@o3)qaPQw*>Jy0kMn}MbL3J3IGpv0gS z51*{jI;JSQk%j5oH(Q#%b!gc)q5VUFTV_Oy82tDsl_=4HpM8Ewl2g)DCnKF`r=v^$ z;TOM_g~i1qJw|`4#9)H_?ce=Os%mPbu&7x6kAMCzrKPP+jh(*0dd+cd=2_X?(~kaT zX&C*t&AV-wcV|7~zpdw-cjKA7oAr1sH=cKEdq-B+*8P5mA8$L9-Thrz-CB|TL(@Np z2kJydnnfaLw)b{?esliL+}zw^zvKJ7SZX}u;^vZ6k5<Uz<~#DR{<b8x#>=zDNAg+K zhjORsvOH+MA%%T8GQKpXatZR&I`;Y7{KmXFFIQ7}V{EeCE8NEYR#(@K2o~SX{an<9 z!WoXbD`f*EGBDRKrTxY7)1tpt@^^K6Rlo1a-=BS>|JHXlWN>~!Qo3Hs{pM@x2*ZAC zcwx}!k8TI3NH^|VVf5{c?qunjaF{h>IUu=!PW$HYt<H;7M}o|*&*<N*=N+BT+%WSh z#gV8rC|9>vf}S5dD>bCA)qL@8E^hm{1`)e&q;<SWihA?R{OvX8y|FpZcjLOu_1X0~ zX>~P8Zcm1^PqxVB?xy9QhdPG3rW{f+SSG`ZgR*z9Z~I&uo2+R&=D7K9o8z(2lt|k} z$L*Bo&5!l{d7l&Ky?g$|)|X1auWT>dzO%o^y)c|s4VTH)s?TN7$d7%?z4(rH2ERb@ zV<85yN3pjil8k+{FPt?oh+PB+_Vn%~b(RjrP^@-?gLTR1N|L6rI@?seX9Y4{<rrB& zWU!SyQex1od+bTq_ttfdL6#GN8|-^yiz^>2k=~i^qe$i5x4RPG_DnjaTHM)cg1n|T zuxDbczzI9t`exw*I|P|H9yQ#Rh3y4fJ4vL7LB?Qyf3`eqzNJw{9^iCButIKsy4ooH z-G}FMt&M4<^Uz}diE(fiAkzaiqAN#BWpO*S)VhoxoK#p>y@^2_p4n%obib6fy_Jxz z9SpFeP_OP*%~d7+PP-r95N8ADeAk%}m!W|1gQL{y)~e(<GSvoUjns3Owt^T;Gw;$F z7I8(bYV@kteZ{t!z#t1r?@3xsoc;Zc??5oKIjLDjaG3J`21L`=O;l=K)kvk(Bsvzm z_4AHLN2!%5g@Mx&c49KR5pijqhm7yesSc?bt<-+zI?=?V_8VHe71H#yG5kK_T!}&N z#xWiwB{fZc_R9}scw|f_CMM<PoqO^!A?ZkI#K=Xu(fbb{%i_}aLLFkTx3^Ee`0|SU zFaPU5m4EV2{yRx}M;-<{U?Eec)?GS@J0_2(3-kbmfM>E>*dhrCwf5N-DHzPr2#Ro_ z25+-S5>1wFHi!@+;V2!xz%>yZ{Yy?QI37$6YrdB!jgTD>x+OyeQfq{OuV_8)Pqby4 z*_<{_)So!deSA}i-;t$3IkXY5zTkN`=e#@J1$nz;1rURC+q3HVI=&tmV$cHwAmfGc zQ#V;D363WkJ!gmSiQ!Vo>`Of18N_iy(X;wfOiMnoke)SIs38WMl^FD0Z<6O`L-u;O zKyyrc=dNemvSDV^aNZkF?O&rn#C6=&T>ie4D#iz%g!u!RN+`!NPzKaUY36FtK&})I z<?9><^gZ=xs2Nl}ULH^$et+P4cqUa2m#Oo>@frm<Z0t(sT&v^^q-(m4NC85#ezZo5 zrjtITbf`$v)I?j+QJK5Dr$*k>py}C0izKr*DWDY0!GgJnLkV8AI3xU!p)%tj3dG>@ zihS_l?<FxYNu92^{$KpXU&v4X_U~k9ctqyr7bN~wlGYFOuGj4T?Xtu@e<8p4)dyOC z5c1>Z-TU(EfBc;ijQ(n>sV?Vf=zQM)ZE2j#wNz%l{aUpU>pAD$cqZ>=JszvgqgUg+ z=Bd?o?Au&4(;x4~t7l&#?)n|1AjNr(HPqKP=Y0B<eFw%CU+2YAGykU!SiB$U0Lj?E zItuYJwLWQN?VLIhr8yHMt2b39*Ieo$Oka>?#51Nq-@H@u`ZA?usxd0>&NbNspi6n~ zYdy!^+}{xWhR8DX%`3H^%Fm1cO8%eB|J~>|t!iBMeB+b9Kl|qHuW!}ChA34sJD{Uc zo90ISCDZTYe)r1$n)O~s_x#TN-Kd$B*_R}ftCPA<4@53_uD!4`FAtk;zpD-9PA6(T zN1M2fS7aEW&b-^$9rzgWJS+Ej*m_f5bv&1`<q_R;Ti=yaH}^KQ-Wp@sU32H;HT^a2 z#`SnkqQY|r(zG@c1laNK;C`lzo+K&k&r!!lU*l?T-yNOd>v$|Qo{RUHTAS4L$z<HH zepmN-?)QG)@I(!B=-k$8oz^`4OE#MiO8-{cCR@~@j(Ko&j(+vf5sSoNG;TwmVShj_ z`|Gu{CI;C{x>IW2R9O*<!Ia++#T3_aPr9caa`^WA90RepRgS*ay@=J2&$`6?MkXP1 zh~{dUXbNfHIl4S7r2|DeCs=3c1C09*8g7YeW!&f9Js1ov3}`C888gSNM)k7<`b6DW zr6ji|Xtc83_>B}X2%*|MStmDYFCB}lTHIPt`JLU9aw?(0^O4s2QQIvoA!BDxi#E_- z@&_}d(&#oTyP*?3?6s`Kpw2g=H{ET%D%~^8x9Hm8u$w@f#Wg<GtV?#^-y8ep{3osV zq-Ca2cHf3akVgxA53&wSu1v`DmM5wc+u1Y8#z7P?WqSy;2o=oXg5GQ??=Q3cUdT)$ zV$oB`1{jJQ_QuTSl-Um?N$*1UF*%5=!$wswkgaKZ!@UpKhcj4oY6@kXU-;gv+hiCZ zx|e8l(%)Jg{~`Mco+4uy&JU4z7Jo?VO^~wTLJdrc_Px#nF+mKXxXjJZOLA(O{PH&+ zN_|74rU{*&Uyx_dUrKyJqO7jE1vuACN=ehG!L_yT$sRW`*e_pRxi0_qPydtLfABzl z`pe&F3Q`PXzQ8j<@S#+C+Rpy2^v-q5%Z^7fI5(i)lHSMloZUUy+S*pnd-bZd9FJ7n z+24`2sb)>tGdSNTZx7w+x58<7&*B-G;YdF6c(k4qiAF#6&-Y5EGu2(({LMF}FW<=9 zH|{h{*7Lt_2ePuVs%e_6$N0{K6~AnMDve`x>V;@`E&jH;wJfP^iBdgOE{hwBdT-Bh z-g|C+P7*s`$ne6D?3lrCy@tPrN9@r2pkz7HrD?2Q?`QWMXWe*@p~XSD+i*>mx0a5) zHN6|8AW;PLrS;gUO@Fg5J!XDncBe@FNX?OP%yZ76{(B&kOB3?CuSg2|a*U$4ccOph z)@LNQH(eJnJgBY5&#LkK<X(qpt{$$C>6NKtW5wF=n8pZ^-<Ku32fIi8$iFK8c4$T( z?riED(EhQxPITLWxR%D$8+~x0UxmB%8F((dcAH%dl4<12<m$w^eS_26fi5`6T*k_? zmIs>tX=``u=)IQA9B&-2Q!l}d-OaGxg?Tf(KCAT?i9Dd6j=uB38`<35Pz5*LjNRF_ z8NcHt(8fI%Hy7n)>r;7cHi~0QqmtVFN{R>aR92~svH4vjn#$Gn4R_?0&GGNgd!y?I z(F4?6+cxV{NOHtqGuP{x?Up-r?#&8>*okvupz+>HXoa{@1N=4`#31thtE<-~Gc!wx zA#|+Tcj-ib^}+AtA3yq3zB0N}dxzuL^XqSou9a?n_tlka`v0RN&Cboqr=NW(Y3Z4& ztA+E<-r%_yQy5w<5P4t>z~eI74s1Upuo(@+0Kw*oy+a_*6tM*6F2+GfP@>au^wRXJ z-+iqf{o`vRN{qbhcp@*_pXk4&SS4i;Mo<hBc-!L`6-lQUD?EgDpul-&`*2gHHz(8~ zv1P9IL`tVv1D@nOH~kJViy``X`vVzU9#-N6@A~dJhZ-k6^>`pPFt8HI0+AbN))&ep zs6H{2r}c2y7Y8H9F$$$S6Qz8l#5Pk1*MeLkRaV(hk!DkAnyL%x9##x5_nWczR0&_Q zLqLWo(d+fTQ)?5F=y)bAQw_&b#)Sgr04d_}$RB%GFO2(n4E}r%h{}$cR=+9a-Oes< zk}Z~@Y38;U<g5CRwGO;p%`b5H&4JvnyCrQ#=ViQw3LHf1U6p44{#a8qM$*uZP8KO* z5V<ouJ0}H2C35ZNU3u_0PTD&=Rp$fPwVcjgwOQSJ^i*>53)Nob`QFH`*0y$udyycI zo<5hV>RPpFc#dNqE32%Mhfki#^Y}z*ZfVnJ3v|42-f8EaH8IGXTt8ToKbHSihRnRR zIs*p+WFct}AGBUm!pv#_K1)ywt!27VlDl8ZrX>Q>Lk#jSbyta$99Zz5meLoNx0j`C zut>^>OEj_#xq?&9&BjaWe1HQ<xNxnEsq-(-8p$~qEXW+id#1G~YYK2XJu6bgAhOK0 zJ}RG<e`sXen7N<XW03`Dwq5tHozv|ajdvPmx29EIHH_At%XlOmWWivT)K9nrAwumJ zJ7N&<O$rCH)#k8yuo2aop6F-PJj`;Y9*g4i1=gJ>ZTF<n=n-TRvKuew4Zg?0+d~;y z9F`}|_cTSS-8dlycTr!S(V<?2rI(_g<Gk1Cs<X`mM&0DD`3^~S#>>?DgzA{#V0d*v z?lfPP!DZyz_p*a;L7m7lqYoliBH_Fad5SYUI;N*|8tXQ4cGBp1#Y4GL?J7G#yyC1x zNw8X@TdapMM*>CuV5U@<XPn=}5iPv_jYXV0F~|cfE-tA8cIDblDJ`$k>+tuDjZMkQ z&Xs!)9;uRrGB>}lC|_Q_rsrF04EFZ*%R{52-F@&_Ca0$5-oq#I`#*ece%t2LsKzrP zFj?f?n|!y?m*SmITJEUP_v^P90KPx2UfAZnJVsRSW07rdW&e#)K<BzNvaM~R$zA*A z#CgHAx3aRmqG<*2Iyih|HZc19k$#7_a`5IrCRWFlK%v$I{Yvcy*W$RF(Bt0P_s4(B zx774krjL&wf2*Du6fdS8ME|@~iBr34a4zX*^1E{6bzgy!P+z}AnR6c_b=36Qq}g=F ziEDXW4gc<buwKWmz13%*UY(LcXRZ=t{5?AT7<f}7;NhN<uy^XOswXY|b^Mv!hC)aH z`+hsE@eSVUyHgytuqW5_-QKZl(GT=z-)yfro+mBijVGRibC$N2bZn4%1QPB0?{l_| zXJH(Xzq)3$!p!+u%^7Yz&(lba#Nq{6zw15z%)fK1^U^lnsx=5{&Y3Rv*b1qyLv1L4 zhcdo6Di0cO%IMOtj`x$#;2y^)$V3mb8L`U3SZc^{WIm+d?b<84as2+h-YQ|$GT9*6 zeQ9d=3iVwW5A@&E+LUB<r%ByN&AVgn>v3J`Dp7Y6QhUan7w*_~&zS=mHygNnP1lV~ z8k3E^b+chCmdvgc>7DJ-`@XgLZMe;S-M6~l;*4=}oE$%Q&5VEQITa1$d95EO&keP4 z5Ax`LUi~{IMVW8m_OE?=tJig3JNS3tIfF4=+d(%K5hex!dP!S*rwk1b>vM7+vMo5B zJyKZwTJ@)Pa~uPf72JoqgKg~{`fPyrghcG_cA9Y#d=>^@K+awegV6)u{(=~c4m`vl zef++{&d>s3hzD@bVz(L?AbQ$oo1}WQOcJ}FY2Ed-?gUMN29Xji1WOD8H*xl@8fL=r z%03T3yg|}naL6z^aZ=ZF$#SM>I=gIVs+0{Ct3Ej}*C%D~>N|2Ck%oMWm7OJ@srN8S zk@_fqAWKG9hE#7tmkL&nQ`4FB^sfhtq~AOTDQAf|<K7TU*BU-ko&Ht(GxL0@nik4# zj0Flj;+v=HC8sZ4(t8pm#rZ;V`qMOGRnugd|559kJs8<b!j*Ge4<mA}xnFi~s<ciu zYRdGg(Q=ttpOOd7H_hMEWM*U9=e>B__lX$9!~Uzfk5!+xGc+=vHV)S0Qu)W)$9Crg z3a+(k?uVlRhRr~tMgm17iNS@1MM+7^kdHt8LXuN6G`qqdKD#87lT*4k=r(%PtHfl9 zd;U@q%)htqJ~01(yprvViw=iFo<4h_e|rWw_#{qBO3PHAV$I0O%U8P=zvaBgPoGP5 zO}#prM+#?63<5}}FKRv#vbJm<1fNb6xjnh=k{b<QXx6Ss7@I={BqhdhoDO=8yzwUe zJj5V2E@W?x*>mC)5r~6^71Zh~A1TtzT9hft=}VQT?RV8dWO-*HB(l+pg{^rbw^EG! z?mgG?43MqVG|izt+JSvtvPcnw{GO<zFDgFLH7lbxNizGA->qK-nvyV(??sdz5r#Oq zpGV)}gi|t@r_Ly(n6>M(K)`xrM+`!s5{>n=?T$Ji*^ye20*+{;rNnWFXu#6|=|Wi% zq9s>PiS9fv@JKju;mmxy;fm3nr!5(#++dwWhkMrXAS~yak(B{0&*Yfxu>Dpz5;zwe zN(_##K-ispmOw!oVw~)**MFg9W2__?x+Jv>9a=XrbiDvDuDKqyqRGbRc{kQ=)-W>A z<qa5JxIaZBPrX5X?j*A}X_%~b*QJf}I3W`TWw6i*!|C5x#kmrL{5PpSbMlJh|ND=B zBY*hg7gFEYBwbzIGVXHeF^pSRuHBL^E?<?V<`&7!&XvFY`=4t{D-<jblo;&o(+I<? zoLm)x-OgV5<L6(>o%;`!H1wq)d7h0%hcJN&NlooaRPP<{=l7h$y)opHO2GSAq#Kh1 z4+Ivb2BS!`CRoo2rZI>~xkm^vUP}D!JDc+&(dZ{8Y{Nv2*1UT?RA`i+T)Ex!l{A_O zjCYE)Z#};weHndhJ%-Rj@IG$2t+|gi^~tv<uR%LBN?TdqYsoa5>q#Tjyzhml5F0=Q zEaAX}D*L*QU85f%C`mK;#4M<qE+>fJN2~E(2-ka!*R=pR&*gm_r2yrCIcnGUfi`Mz zHo9shtvgXuQyrV*-{L`YARW`KlGl@^o-}s%+B5<j@3E}zR88Ascg|TfeO@zG$U9#@ zUM-88?uZ!cd3vr{D3J+e!$rDb_<jr<W%h7a3HN+wwiZq&y%7YWZhn1SFItTVfQe<7 zJZik7si~MJ)^ljgwL!BPN$rj|b7bPg@mOjQuWT4FLfxqSQll4;2cCIH8Yl`DV>n{7 z6;7XfHmF1tcTGEv9pkzf9eWnKB&#n)9Sp*`mtOPToEcggkhu1HDmR(SM9if*UK!b6 zEsU{f$Z9q_D7IVMTPho@$E^+0u5`FS%0`N#CihS>lbtWr_=%?&8#wDZ;Wp+7bCbU2 z|F<3wr19Ou0l$w1G03w)@}i@7o>9lG(UUffUWRVwA^oi2w^}D^*RA__pi#TKZY4cZ zxF80jiwj~fdiWs_gT8=VCxsbNAFOq3#OF3=q}<4SqQ)Rhtm*k9jZlce_Z=A4NzaYZ zA)zBqnkO67D2-l<E}hktrvFk0uWPo$e1`^2=?aMjnZ;fU(u#;e5BQEXW95?7lcMRM z@K(pUmD-rWu3zZB)D2JXN>ZmOoZrYCfF5+O<*FL-sLkb=Y1fpa><z-j;`Y4Rk5?$s zNQMy_geivJ_Q_Ux+4@+jM#|jvqCzy9dIO!H$m~;Z)?LzCH}r3=GgA^AkImn{R3n(R zX1X_cg7<|O)F`k0HMvp$h1sVK*`{uUoT1FX2gU!OI=7wCGZ;XCBR?<zcFwfP+WxBk zE!=MxNn&tpd|Z-KQsqT_qB@!P^!Cdye*KT)=<3!?RjKKj=GZHBSRWo9k?fpY`RUJp zEu*7jN)FzC^hBP$OpuOFhd6tB_1v$n+>rJ4b@Tf*`TL*$O48CZWN>ImYdt@D@=Tt@ zy^zJF$kl*j%|B~m5a4`3CY8wzSJ2UEMEp|b=1#*^jT}60qbstfcrah<vQk&xGe*%{ z2^b&(4ab?h{!ERY59Qlo2YEeQXynUNjXb>9cuk6ivc);qp(*mdi$DYyGA+9=RSoWx zf(Uo6g$img67587Vp4tDAyUL3U~gMFTBZ@_)noiyCiSi=b#P$M8%RHphO}j(K~g&t zHOeiJ-$WABT7yvPU+f7f3fBW)#Gu+TcNXP#<Cp6A6D@}-`i-c6GEh*yCeU|D?tWqB zV4jv93WUf#59}W9iesu>uGf8`M6TV~T{rT9RG5$3ZU+=C=?neKJ@TyczRF2E`HC&0 zXR*u7vv?U@9tul_60u2I(tAxeR4=u@7imVZ167Tdo`~WL7W|$FZFI;fvvzn$NY-uE zFvwKwPHE0p+LzwXLVDojt{3L__2xW#hl5C>b*53@8QWAi{Tr(|cVdtTz=Xs!`FDTz z*K+0hElEyI*H&FqC(A1<S}P|vzep}!xh~IN#>>OUPvzm0I5nhs4-ACf*x0xf6clOt zP>!*&tgfz+2M?b}Lu0cpHm8A)Op1&0k=JH(R&G74;I8iC!_luXh*D1RTS&F)kxJ{~ zpn+gz5hx!mmb$Sj>*0uk$%*i%KdZ;fRQW;i`Lh0ZlGgJ|J*@12(oQrcY1)p%ZIJI@ z>QPdL4FyDvFJM@&8LpBf$8)nu9X|Ry&xOK(a)9>=*Rgs+oCe0%IyK0QaJbR*Ma&2X z`Ws^KVe?J(9<~Z{z+4HX#-IoE9~<NL>1OrPU}I?oxuTn=8q6j>Q)brfYcT3UFwv-+ zZb;hm1mmjKRbiCI8d=_6vK~GLNQ-J$xq7{_FapLQwPF%F<D_D^REC#^^jf6kV*H;q zzA?(d$l{<pZ+WEm_ccC|2V-j^`rNMdv8XSo;Xsvx{arJcN9AhmXSxydD2hBc#xTgb zoPO&5og9<F0ENr7HZCvPAIZRcy;V?NQM0ZaoZuGR-QC^Y-66OJcXtTx4#6FQ2M@vB zT|Nlz?sjJWz1CU#X6*~AW>NLs%<9?W?H=9F>-!Pq6-p99dYq7eUR7&4Bnp*n^4+fN zp?$=?8;8({da$GjN=4<n*HqAepmk1l>Uk!k&YHoE57Beb9Rw6&CLzNmf*-u1+p(ih zeAq#(1<WpTxN1y>d)xCA_^l;7Kn1xxG^!5-5?A}b3>E=wQ&_>6L33O&s~6n(5e!&~ zT$%#g_dcku%J)=<VB9p@hoOoqyes7;6w=l*xh$@EPDRKZ3X3u`%C(ZFKefyRBB*zC zP3ih<*h%i%s6^@1fiIN%B*#ViM!bt%VV*`sy0Y=tq)(Rr96^njRX2f|(xWrX2EJ<6 zDceo9i%pDmMn4UUi0t!3UIUZd3$f}9eleoNgK&It8mr-UNc3CMLOwa*Z^L{4b}W7U zjuK)+<<(e3Gfn;Cbh%#~x9yblqA{a1U$G48fPot%k$mp#M&MZWm}osQq|Je?mqrVj z*CE8l##rhvp8P@zMRL>Lp2Nz;#q>ZVkb1iAtD~c%Vo8U;5b!cHq^hPy6&)RYjokAp z?0gN+euKJ){M{(s`o8k$$Qp+L4_J~j`T)x34eRTfw_l50F!APQuIwhZcV4I==UM6% z$Hl_9lDjEBnmI_3WcYD_^N9Hi0zzk5kN{4@meUwPg^PwY;=s%T2?Ld=SJrzNAu!*U zbX<R3H1E+19vp))E55gG-(JYsu&|+#>XYaluo(6F4h)R9SQAMdxA&UdI2r(>+q5%# zV{N2K{lrUw(;K|Y+%|j>dhME!t1#TL4-<F8Z>1-w$pO8A(y!mQmnxB$QAL9OU;)$Y zr}%bbbBizSSQh-yt6aUGB9c~WXxP%v+s*v~Iki+nE$_+YrhGl-rN<lrXN@7Vs&$qd zk23>G2)al!vd1m=ojp?yrVxn>)pUOAo+9$v4bLqHKe>&hnPh4*x$RgSa3WlNj_c05 zL^~A7uU<^YfCDPqQuWTZxCnTm&o)I~6DW1z?cycDEt>;jBk(yEpsg$fbb+EDfHGX> z0w)&0E@8THXK2H3KJEGaZbJ{DmY~+C8>21htkk8Iud1$HP)988{)^9Fec4`FTamhW zkEEk3s1ula!^zd!zn7Kp^yKx1ckm+xmFF-k1D09=g2=0%{uL@@g_hheNLmDA(8@u7 zWjS%NY@Hq<?CnVq@ii#;ge0W`RkZf3@e@2UJPWFTXH1hKSBAm{oyM<IOdVvF!<uPQ za^^uXXl3!IJI5fUfj<!Z=Gg{!*OhdWzH_}rJuVD|F3_0X8b9m_zAXzz2r;)PGix{1 za_RFnqde+>6-fJk4Ar!zVmJGg0$aM?m%pjACZ806J11qP*&V4KwfKzoC|ZT+`le>b z(_b*-&3QUyyF43kCtl!a%_gVK^Bt8O3$a3nE=;Z!>FzJ;jgvC(;hmoAzFJY37*5AM zKcB*Ue~wmJK`7;iU)ji<$g7hMEj{jE6vKLMUMUYQhWq7V!3*zyJO5^lddXxf$W!0T z!;Uu{G(f1p7j{L9|0}AO#cT6^It^P~21l1r`9gcR<MJw(cK1w;LDggz&+yF`ejoEl zNmlZjOfk;MZ}yGcT`;}4Ha+RDC&fl%u7B6cVFX~Do!ZlEQH0NMX+T0FGc(7mQ{;w8 z8hxS8!-PM~a$-|E>ddEuV?Uh_6g7ir0LSO>FHlz!`rGvr3rilqpCOp-#iwFJwhP_Q z4jeilSRWFwJ4i|9%r4{A=fzedR6tmmEV<C&F{a(ONsb)LEX;$%{k$U60S8w9zQ!M9 zGmet|kH<<=2w0o$5V-Brf9^X=dxl7uREvMWh$n9`SE7>c13Um5SCsn{P@bs!Y|#ZG zi`Bb_@jYBcZe6iy<pSsT2oDRU8<l}6CAUFDjCCwA>pt=3T|_gREWHN5&>I3W835i! z>hl;bCpVUqnoK41u>gZB04(f~JrMHGq?|6NuOpqqUmuRBjSP>c(YQ<+y)X4>u_jn< zx8Y`I?H%7`XDy6wAEqe@eW*$f4&FUilW;c<QIUUdETb~9ZRTP*{IDm3H$yeYgoAG2 zU|{um`P`Tuj379X8G8G>MABaaAyu-t#qgJL&b7Qay?}!cIZxM75(vPcEuOrKy6N|m znEJYr8yEJ$;bY)Yo$OW}EA866m01WWHp!a}2Ert~m|6Zwguq+(8Yw|+1d)>|0`sP$ z+H{_@fRUhn>^xh0bLz#%LATtNaZRQ+p4P0TiQ6glz_w11dNWFhDIJ-`w=mAZK)^Cl zaGSuzr(xXJ&wh|kWh{{(AkYWp$JLLgG$Xu_FIkkcSE3^>!7QX4jHFR#T@2QQ4hd2< zT&Y_PbMf_`8wL$(pNdv%MZs80eP&32ArjkFt&Tv<)5L^uGwt8*Uyz)de>EEKO4ggX zQ>`tk4=PF+^r%|~CxveNW`|99EvU_TqsJx2oI<SY|MmbT#L8JShUG3`2cWfTI?w4+ zDi#`}w)#rtG?Djrb;d#(e*@KC-oDD8&g=vIEx))2P|aoc{zHHZopiUqDb#oChfXn8 zA2nB&%dkdLXw+#>YP5I7)j{R1B!1&r6dZApz3n}clkIZwuTg&*J3FZB@OL3<Rlxh; z0IsY93Wm5KFqHYP%|6?)>}FE~K}ml^Z{#YN)SKE})N_Lm%QNKKu{a9L;lS0u2H3Cg zTX)rcT};&64<7P3&XqEJ9jftg6SI6U1mG?p2TXVvXY+p6xC~qBInUX|O;{!dK4#1a zOlWov)d|nGyUWaAi4NcDeg%Z{T5V(^2p2Sm)AlRbk)yF`L}b*Yk+}4v^?TXUrBb)k ziEoqTR3vwplitY?)_Ha*6@R<=@1Czp5ZEj%Tgxa4ZYpT)4}`+~ATm3;I|`1PYZpGQ zwA<+e=|xCJ&!)IJj&In1$j=Zd>0S(9{+$NINN_l6-2Cvo#b+#iqCPcX1IuQRHjW>+ zVnq2XhRH04v2+wUc5*@%7an7#2Tsasx19o_BpkERPGFDPH10tRz`yt>?_y0;qvx3x zctNC}V|=Qs>Ui_5X?Gfj<s4wny+jcHwVVC`#ts6n*ipTW{qdRD2#DuzUwlg52Mj0N zFFQu_N1|JX+C4}IQed`GLhgn2qUM{yLpJ<{l3?%r9v5E35kyZd+)J|F<QMn|ZXVfo z>{j`f7_^RsK39CpVR9rWWW%XT7%eu2ABx10F+urhQ47qPTkC`+1w%;;R8~!lPmZ4X zAU7*EW&`;85wT-L%Cce_mdAsDH&Ai;FLn2E!R*LC`)QhOPA74tm;-HZb5l$shszmp zsJQ0NXvs;<F>xEZ6Wq^(gNpl&esYzs4(xY)wkhwN8K8*ub$7v8d5<Yq8=2T&i!WK2 znO(7xENcQod@ozCi#dz-4tZ9}UT%Lo8|^NhPbZcfnk8*t%o>{%TCCEFUi0vyZ2hTu z1kUZGQVQ$+>eO(1zSb=XZWO9?aF1Vftso!i3gY?M4l1pwNArrtWp!<O-PTN&ZG3Sn zrt;1`+%+|woda12Cd4g0zqnX~+T|3hFKTUcqSHp874W<?Z=$?FoO9L1j5{#`KGY8j zP8Pc;KGLU^QadLbK9?XwvS#3RqBb-+E*|CF6!dF|>v%o-jkESFZ}e5Twr=X}UYS6O z_@Hc&hUm+PC46JoNe?h;R)JGJi>8&3zS?6rb?6lzR3g}>`4Yyrmb#ZShkJk?I189= zi_>swJ}-D9y|Hz&Q+rLyJ1Kf(K8fFl|GwDqW5DMf7F7{Y-OsB_+o@h+0Oo525IYk6 z@v?27>@S|b+F7f^`M8wK#@xMmJp3iBKFVt_sYZ?oAs>=Ur5J}9q*bIghZtb(5b-KN z$s?J*XK>l$V<1xJ(XX_=`ONygyc4PMihlbzSN~u=O2n3r=|{OqO!2U70glzXsOeN3 z_O0jQ0(B;)A}s6lt!q_L*7xKT-vh8`jk(V;m@Bv)z!*yRXEQC3Nh_nS_(vjU1n7y? z(B}uqLbSkj`CD4Jk&kfx-?mUG`!~;6#Z%=^Jj|w?>!0nX*bSz1JZT>x{K8;h{3P-$ zI1Svhe*v2x=N|d0Q?ci5lcqzLCPDnvM6@bONJiNL&(ZT@x<uy88{UuO!bB{2-@{!v z<^u!*S9xM&NGX%YwNLZhh5?q|x^*Y18yQDKTWf{eQpz;f(%5P1LrWCpaN}FUTY*=e zjUJOR@_PSP4CXFjKzw8`AK))HI){hN9ho+9GBq8m&mFi-pnIO&F=1p4M|Z++@+yX3 zSR6wic454Y6Eg1)thmu!Aze1?_{rGEK4_AQd^ERb?Rl+1=&Tr|y=VH%97#`G{=Bb& z89#e}B2-9R;{bgXz*m`uvCFU>qSeR~M?(o2>#&&Q)SqdRz-BGnrtremjs}DPP=)CJ zt3qI0KmK)6fJZTycA*ozqtlga?ObXyB?QCYAwW|W^6<6fjpCTR3Ka*_Af4AzVdw6; z{7!QD9>yp#^o(j25W^%%7k%;XKboOOqzBj-8JNaue2~fMxQuL}P0Fx|JVw&f>twwo zN-V0MrvmOgl5+Kk@$i;^-xoJj>I%O6rOV99OWRIVc{<5|?Mezf^v4e{&}CH0?@A~r zC_uO6cn}vdkyl~kogVb5&3?Hd^x+&TT6x5Tk*p)@i?+4aABB;n=;lzY3?Cc6dP6_( z$+GntKh@jmXW6LmGZpYRjO~74H#>N~4<E0c9JwAGjdBUG_j&R8H;a}UvNvnLnf=j{ zhyIO^-<Qs>(GLyuE-us7NQivyUS9E|PW?m0whurk<Zwr@jCoQ()!i+7wJ@;7orK%6 zi@JKbJoR5$HFG(1FD5C~!N8fjI%&H<_)QT6#5zSA5aNX6HxXL;y2>|IY&cpu{a!!V z<=Jkk*|5KPB~zWxFGIT3>o~`KSiJFVes43RA)=q*_0Q(hql(M-$kxkEU)reK19l#C zRvL(i_<Z<g*_IfLH2r96Q|o^m5VL*(Z}{4&Ig2mT(A;X@LCG8ujulP;$|SoZYvy&& z^jW-7w2g4sl>P9{H*+nKj;>N6j`W1SaM{do(WS&oMX~2RIL&Y}1&jp-**p~7W#h=i zR`nTpB{yt`3xIaQQG2cJUT+?I_REHSJ7esQ8*3wPjdgB*)5b!330oDL4OP9NP}12n z|5AUFoZ$nO%u<us642E<SXo!;-9F0(@K8*S4*Y+Kw<BGz)2r?QO3$YiH7*sJCA{uw zUHI)!tl_VnQ>%nFDQjMV()r`v;uO$Jc7=_dQ6dySI4M)h72Nwz*w7_RH#(PtK<$f? zdtO5jM>@y6&xip4jG^%!=5Nj!91lhY;=-TBU^8yjV$+rdDlm7=tvBM93N_3ncaWt` zH1K`nKD9NLKb2KD{CyMPx5liJ8s*^XV4@ECwLhyaH8P2{*k=j7FVMC_l5*+)IkbQ0 zP0Ky47+B3^)C!<wS|uC-oLE^~*b8}%+9*{*J*8C~z`&pA>-~Pwcw{kEc@!CpA|ZRm z12}H4S!bzm&1b*3Ozq!XMge5DdbYf^hTMdC2~*g8<#jsK*)`Um{{&@YpQd+AXsJgv zj^aJXMmhug9Z&B-rvXg-3k|iSCh_PB`q%HB=UHb7-?!23Hos$(zlo7Ml#=P#NL89$ zI4W6aS#Z2TI^#ej5_l3GREv#D%CfWaWc`SbNP{k(zm0O4K~<T}vSs)-H9|z^Ulm7? z@QBsFwXy?<e#U?5|DPr6zlY1fNzr4$j&^&1xqo{ph!(<7ODS`vIQ`b#T03^L!=4Jt zCe!FE8JhJNz$KMID2#7v_86X6^Vri+{cgB)!%*)c5amNMW!OJ<IUITXO`>jWFW|lG zL;m>%#(Zvlr)q48T#!V@7F{)hnTEsQ{T-`kW;JYVjP5$%ttz0%4Qgp|kpuClD?({6 zKD~DMbUo)RW7}bIIB*X49$l&5ggKE0)&AntU)*EMF=N-aHj!=dhJAL=_Tl}^`a~de zGlSPjv6?;aOK$3{c;0->q6@Hk6Aznt7IIc3ohi)~K0gk9w{c$H$oe9elWx7Kr&sv( zj~~&CWdA(ggtAM;v1!fR@PnXNo?e-WeZ%`(=W6ToX~ZnoMGB-;P8!Wukcj{jZtR7o zpl%_Jpe5(J%EzJ$xHaz}lB1|&GSA{p48(DXfPGJuPWtJ-6}7^__cqO?7_qTaKxkGW zE!eCz^MGod5ND(Q#^`^2Hu3XNC}acZh@f_|#)BoS;&MJCL*-}(hc|I@qF4v1;~U!r zBdatHqaBb|5m7l*lwagEl{m8QRDqEvA?d0t%=Z%SUw;U;am@mNEvO}yITD#%re^-6 zjb&4VLP6%1+pI*{SpSwh7u}m-p<q{HlCq`JxIvhqt>kGfFQ)hGQ}6*XFXxJ9)2GU7 zwWo$-^fvvQsH1eu)PkA3Z2rFW!D`7X6I+jmi;<hjDlg9#$KP&1fOsgIerz_lPBzob z>>yjpAfWNguz+kr?+Kj3bf)00Z5%5=utz?!!Fe#A`DsVc1@E{L6H2%nJyg(MhR${e zcNVfYW^WaD1f9JPIcs@|o<rzIhKJm%#AHov7YpG^pfCK$XL#*uvW&9s9$A3ZTBtm; zB-||-4SB~tZ=oW>^*Qwm13IE?oEIqC-zxXpY|o3_(3rj@!eX43wCV%=T2q51SYF6{ za&eg1x!z|gBV*r0x-N$qng`t4qd(yulS>X86}apKC7E?lemuBOuI|L7?h3ZS%pR{@ z$QMX7&z)!W2z3b0_-yr`%+9VCzo!6;|IQCL=y%fP_98VAqH`cMNftPmVQQNnNfzi5 z{czoHx6j4EUX<*VYwMlkv~X?Bc}J>Iq;PyP*zQw;%DfE4agK+!{}<lJ$!NYC=bpXx z(p=&~V_Qo(Z;OtS)nW6&Sb~Jq;)Zg9I@J{%1!m5u*J`KK=)UnW={SXMqDc~>^mpCb zdEe5z@1h1Uf50w0|1W9&lh6QA>Hn-B1pwgoOy7$%0$LR6g{PfsVVmiT)4lT?9{P3l zTlYA&-vhsEu`ntyd*U3*8yIlI^`Y_p^06I|@^L?BH0<F25cPUC<xg{msL<{~h>f2F z+@R{ePxQR7J+%frA6!!f+#e+6rPh$++uA!A^}Jp$^`!PZR^<-|JdgDx=D+`isHm*q zF#XfTRb5@17;s<rF8=-&@Cg5e_3;cEs}p>b)?HhT8!ZzDUmsF4z3dEP^8hABJ01CF z?fBfMugR<Wl2(%G*T8rK^06M{#OIA+VlWEdx|g+v+f$d)Not!}bT)Jn2hO05GB1+k z+=xcH!mFM=lv=W+LuUEN#zLGM_vu!yk_RF=gpxR*$-s62LyFjZjR0!3>{>QN@Zm#+ zBwnr5>a&{i4F>g}gs3=HH2|?jkqD@Hs&c_BuchyVjzXL~CmktIp3!7MS9Z;&Uu<52 zwpyxcJPfz8#~mhR3)qxs!{y#q*QCR|s_wtvGTq_x1i*wezi}Y;6RV}xRVy0%c^Kdh zohal}StkfnH-sJ7=6;)Yrkl$~7ku$`_S66V;!@)LlYaNqCuJrGy&RiPP(^B1&2;(( z>-I}<iAAeLv1Pqn3li6Oz(wU7Q_2GZ)O<4|5e@1=uvq?H^|_BGAjZ0W2a-83C6exr z-9-R%m3#+!dotM_n;P80zgaV)u-Cmh>;-%&jlUAxB!c&Ql4>J%18b=lh6k0J+u{V( z#=o_%=h<AxTZvFT*7Kh|wQK}6RQuC^^C9XJImjLls|ddV>O1eiSPY>2uygFPXfZR1 zhhvA%uT7KgWOj`VZRWCQ3K*7hK+C8c!o#X@BE%+hBrV^Oo<3(5h{^=lSkn*=&lNWG z`bxnPU|LC|!NFY+5n6qCn-S31R+Kye%#+0ie)vmyIg9lG^>=uU{u=Az&R#BJ-R~G= zZ*pBCT`xLfQN0u9BQ6$lQ7SRg1m35Q9_RDj6d_@fBUZfb^xDK8&sg5Ol6@2uA2nNR zjNcfr&6{N_WANc6R_>tjkW=<CEIYj?yb+jV6lSedEq^*SEM8Yl_fjI_JpOztMa%aL z**1`GkN;cX%Rp_9RWRhDQdKykfp-__GDvv?(~$CK$8%T$4)hnlfrT+3@1|}~azhI^ zDtZ3<g#7=H%KtnNZgQER^jSQ_hyk>@B%Ma&Yfk6-RuN?X&3om>m*u0-`d|R*CV7yT zmGr03B+BPrUv-X+TtGk$Em<uUMck<9?haYn(7%AZtW*xzww8qO?&fOO1$I3eT&~Fl z*z|C-bj|E@UGL6!4t+Ehn)g>TlQMIw4otYBcRiy3!dutsmUA7yT^r2_4AtBM`2Ezp zn%tRpN%*EV^fN8F954}K6(Z=6ZIeWAS!w1<C0F{ywbt~yDNjSdn1V4RH*zvH?`Orn z&-21{N*AAX{<twl!}9B*1#bv0A#u6#oZZj`-O|E$HFn#IxQ0sPn5xpuK7gZI&<Q={ z99D}j!Qc~Vg<q*e<L59JCWnNha+_jxUCJ!cIPL=-ecPR+efbpiIie~b(Aert_?jde z4UYlNEayz&o{=w@=uth+hk?s-kzf7+{Q2$Wf-0x_ZYxvkJy(fWp`n}uDmluicGzVl zF5QCF@8-8J`AdO6DP8pA*u?DrbWh&E44a-FiidtXBO>ytUR&gd@iBG@2^ENU`t4DT zO_N9nn8xbmzp;s<1ItcW8C%#RmRx1yJynZ+Lyl<xk4G9W;r3kINj}VQDXM$#4VTiU znRAs3lpFu29M#8`u>z-RT3NqJWdk6RCx~?J-CIiQuR+8*e#TsLXdz#?#ZzG{b}vRQ zPo8GgVOp~3PMGM6O-LmUMMw2UBHKMuYNc9#iZ$9q6!QiQLsclzY9iuMP)#m3I*Q0q z&tPV@p7`J}zFU;&sJ|{8L834lG@fM*S`5&8AAofWxpaH}5a#M#yz=DjR#W|c;)wuG zC`NBc?>S$beIbJ3OdW0UQ+PW=n<#l~A>X=P?W4cdNAMDnWJ<-;^vm$-KrF?`vGfwG zL6s#L$k^RQS2lJQl7|pU`8)5#uTH~<Hv2Boeca77@kx~rrbiYR{9*Coy#oSdj@Y<^ zY$qaVI#TH9h#Rpqjm>Eu?_Xz>cO7c#ZGYRTKi!<Ts0!-R@0kv?n1LT8p}|TdfUnL# zKNKm_RH(o5;6TDZXS2@*WIavoS@fA_u4N&<;Dv*CUqG<{Czd4qKSAjTJYfCz1M=<R ziv)td<-I_cbKO(cj0jb{02@w9<o)oG%$#mWeGD-CLYN!#926}l*I<VG!AQ?|+`JKf zcfrn^;8M;Lc{Zk}>4bROys)65W2ocx(6KY*G2WP@>FQ>$Fco`?O90^4d>Tg()B8R6 zX9CXdS-5dE<I-ejjBAZ77qx$*vOM0sGz)(H$kxwKs9#V)^xraK1^rCN<N6R%yv^$} zC^pAZ`H`R)l{@vCgj+9-0L~h>cV{l^tyRcNtOtQx2$l1iU3#3tjXdP<3b2Wq0X|hZ z1S@3920kJba7}E3Opb?&G+Ypjrk?U7dl;F#Sh1hsWTK__=QyL-gHrAyX6}MdG_cDm zfZTpgfr`}-3NVUaHlme!m7+8OZ4){Xcx)t=lEmQI1vN7^v<YZs*D@Kye9N^<vaCU7 zujEW|3PC7LqdXF>LEWoftN$796@uWhyxYFdH#sk}RSmpq@7|p+qBUi6%dY_52bF)$ zf`tiZI6eI8IpHPpb2Lk~x5kkos9&XNJ8n8_PQ9==r^y=`uqcROgx70Knk8Vm%2iT= zjS&tEB0{DKB?lkR1$WVT*|Oddqrm!;IUfPSA00mav1nzwjU+)W2^Ry83C7RV@T^)5 z#QJ9w6b^8Q*`N|a8ApCUti|)Zo+=-?%Jt;;!1pVdn5$wtc@N5wTMN%4A~7g9BM|AR zTw*Ef=R3UXb2~+#l|%7Wx`=g+R%z|rcHX%BhD-T}V*=H?NFM>gO&QoV=RzhiU>+b_ zJVdLERH|7@(f!n>o*}U!i@@57)q243;4z~-S#R3=TP&wpfoqJ6t&ZzKsZ@W_skXuu z&uu-ls@UH&MTQh&<SO3oHb!JBhY}#m?Mf$T>7ysrI5JCpkv|^*xqCdAxp=5?K~wZW zwMR9*$9a@uS}u6Dp^eT!Lt57*B8Ne=wO7PoC~sz-e30B2jov*^xfNkWbt7Va%LK^Z z|JtX$|1Hc0{`=vdLM!wUv}&UPZlO&x(WS|<rk}*+By%GCF%U>45WKhQrM$T4UI`%H z<AUbmB0>P%#EO~-=k9~0Oi0zLJH3eZm)irYO8$<W`p}0|Y%?K%7xyu9&%j~TGjp*+ zbCtY&r_VN+U{9Cg8qpn?95CToV#baB21Kwd!mQNtbM2xZr5k!GiIfx+MC1>k^=Waj zhR+G+gv$c^Gg<Vn0#4fnsDPjOvh%EQLX|zE=#HXqWVDFY%u+=T+5EmFt6K+lMH5TL zGqQL5Tph;{`<AZb=brINH9-}^NS;6!ONG0OSuuxc8W-&RZ|IEaC-neyS)s3>3MUO| zNuxqgAlUf=&l+XCF7S>*=A?dt9{DEhG=(%7LZTt4?5zJ!Ae)+7N{o*4Z+E3?!I~Nl z0yYyOOuw*(`T{flMRMl6cr4&`P!TkrzjXx@E31>A0uY#s0?l=h^Kl*rsXsScI8lh_ znbd4vqJ+RWAT3nUbiJekTtXr~DrI5b{ReZIG4+te#+|ix&7lPd)(*N2Z|~_Xy)ARE zo}>2hi}A4L%P@fNPe@aSN(L}DqU3j85NtQSVIjKdE`)#`_(AIyxQRK;&r?r4UDm7( zXogwq%0cMDcb;u2@!gui#OjytnkFc}hK5s0OLEq-K6|}GgPURJX>8o&CM}C#R5?Jc zLtwtCAJ8?@Sk3!x)Wdz(j;D_qE^7DDZA`S1cbNW2c=ybubRQt#!$dd~OL<SLpp3eR zOcuy4*E(-*>mjMx&taeQDA?D02t>pl=pky|Tx@d{M}R}k&E^$sjAg|iUUkLwbiZya zV!xmCAgbv=33-X|`O9uvHeAgnMh8I^(8^Ln?mQtL2AVJ^{w<Knx6)OS)xfQ4m^BV> zWBY2q8IY76n;s*|=Os!a0RPJzaH;;6!(skAb^lKezaRjv?WB00hU`7e&D;)$X`X9l z>YDw%=QuKTJqBJo#Ivw{LH|IQ%Yw^E&rvoqR`Gn>^H29DTmUYW2Dl^99i_q$yVv>a zM**z_!Pzi+wUjiMPWi=@g0*wjVK?xD_;dFcAFACjAz_4cgfCX@!T~{8Kq!L`=Ztq_ zj2tV>BoQq-;nNtCk-nf>i>q}QjXbU%R)DtI0S2w8uSv8i(qCpGM$dTQPM3yobRlxf z@qnWzUz;fX^qsxyJdd|K|AITnm`D$2(Kvf%?`sdIv7O|y=CF;#wCqo8juW<i2A$*A z6}rQ+wh5e(lZn|FyZ6bvu;wm~0&?|D;c$@UfcrwnGQ|`xp_LbIz=Z=0{xdKT$AOkq zZeHb=;>M@B$c((`u64(k02Sw50SL*f`w&v-AVJ$te@_0xHM*{z(V+UA>(t<eqASgV z{;eOj*3L3Z(j}KFlGz12DC_~^2%!@+B!+NKa4z#;7ok%=o7w~mwH_f_6Z#{WH}0_M zPp_TV0&3aQVqaL2ljOG7y0?}9yJB=gG^HozETJfZW?WYbp`a+R=IL=fh$C~MxOiAu zQX`QG%1BP?^`&q|y)NtBu~*~IsPbx^f=FNh765rM`zo>r`J2h0pZF?x&^q#Dfb}*m zk(uCZ>DAfx@V2f@uo5#B^Hwg6gW+eK;s}H<z=u{GP%HIqL4PLaTr+8S4sc$NsNsqu zfQQaIN%+IMl4yLgA|aGK)a6vDNZ4Yq1l#BM8`{*@T%M3~WA@^p3#l*#bs&8uxtc?~ z56U^%RZv_k={d+_+aFsU2?=@LC%PT|b|(r{qmU7+($IwKg<lPh4adLJ=144+*D4ez zeTMqL(ONR%+W~CcQvgV-@>XZ+QI}r;QyJdm^54w#|J~;LAMKuSwy*OdhNFKj<SbZ3 z59J>GYY|4WC)VFR(h*4Fgi_V0A>3$4^beyYm7uY3EdrWbr&CPqv1EKE_H3EzscgVd zd$^U@6jyOW$FIVMu2B+SMJ-wYXEJpeV}%Zx{=O1DsLo-?ZN(Pz)>7_5G~H;HCM`-t zc(M6UuRcRqzrW|lM0dAQ@$c>||M7E$<h9uJR7lb)wO<dgq1gJ%naNG(T7Pp~#La|a zlD+q=GE-tPDneX-U3_v}E!Cfun?<WbUpaj7n*>KO1z2}BAU+0Kv<?WIjq3@1?wawW zFSiCCCh-`E`<VegN+?#E)fY7~3yztKpAiP4@GJ{PXV7TyNAHyo{N?UTNMugDzNper zK3BPw%GfWOG5)+KI&rVA{z^FAT2dr|JJR9ixNi6$_8U0MNGDSO4-c~I8i#^4*gh%N z7FM(*X7Rwgi92Z<pF+v79IA}h`Po8u^$E{``eDk!Pfri653Nnpw1k-ANa5hMdaT+% zRK!#F4R?R_%(+Uy#TrxEu;^dfFRN4V?Uc-Dper8_T5U&S<I^VMY5`ZRsKC*9ovZnv zS#0)0IEmL#_Rcs=9WiYrdQqyh$H+ElYwkxLgWg;_HHv^Yp>d4zP67th_Pj!okX@7r z3CCVq3hrL{hz3){SIP7zhzAofBVJYNIvAzJ?M8?L;oUT3EL+?$ZKxT$dF{8tS9Bea z4qv_IVMlUpzXpjGt{{&)J+`C4zRI0++b(aTojJq9s^D^3X@zdv!fOeyZ72jr>-L^V zlx=qasW~j1_kn(P=~M;_E1Uk-ii~S*cBrdR>*)H1L1QU_O(>yZ$nN!nzsLIru$f6f zS54&phKBevsz3WwVvVe*8XC~1a1Y!%b9Y<1cL&$iX9bI$cLxKw4FAt0OvWP3F6--) zGjq~zc45FEf^R(oL1I@u*sk3VWN!R=NrXa=x>JS`OAR8~3(?lqrboP&?DG|1=tWVz z=*c?KvOW-qW7tQ^WyobRQDcnTJ*#5$fFYb#hYBg6G+G!TJaDjc1;DDqV=Y6+7;!Z_ zd_1)d?PvUK`?@+==A*~+C$ZW<htGZ2%-^BK!U#|M)aQU7Kl30g^eE$7>dBqachi;~ zgU`j6bp!x-FvKNkr-WUJBj=}BC^##(N;D!~oscsPHZfBquDmmUdgV+*V1tPSI`=a& z{hbYi;3@3;qU>{%UyB55E=@zp4f*B!-g;gbBL#F6@tp@zphD${2HjJc`0EomY6OMY zlWKAQ4~Ml7pKnvg9_KmM@WcCuh^LJ6tiA%OKU$1!(|8uov~~LUe>jW{z6acIR#nWn z%3-}qSduD^Pppqm^y`oJw`Fu^#<8)@9JoFRcC=P?*=pH?44TKZ%q>ckq(fwguQq)g zApgh=!;>V4z?unrrt38C`=z0LCc2)L`aXIw;z<@X)#KiEqEnhYUxi-nGCue?PC?yo zob^l51m(9$Q}@B13>cx~6<W?9cxOb@H$;uew3)&W=Zs$`(3Cvd^29SvZ=xm3q03Wi zL~mwVcDf-xV7Ncs<}y(Dw`#MIjNc;pbr}Y|C)Kp;s?0nt-jX*`DEB-A`S>UYkWAs> zl`r5jek^%(&YONa(7viq;4n$};{~1ms(H-e5qZ!O^|59>yx7OXV)T*mEB<dTPeH9V zc7haxo_!MBwEt8YVf~OkWuDPt$4sQi{QUb7;ls}MKyMS~ky(((r?$`6&{wa?Bz(kH zPQ#!4AbcG5Nj<x{H_Lbq=>0Z6XzYnPo4LC@kp@ABvO<1b?;|Cq6`vl$G`1q`1HEGn zX8REM;oHb8H8oO%{IvMF{7g_Df`0@(zBgxG)1`T5ynLluw5#5k&d=OBYb@>e)pq^1 z5#|ss3ibV)_U@fFAGO2Z&64G7?5ceZGB+H(fl}pwlJ`CEZlar&sy5Mjon~p|GK6_B zde*O9y3CnM95;Kru#7I%uP2h8lgg1IVc01!a4ghB7$(Yw)crqvfqy+YqDS>|nbc_t z>h*i&Y%hGuwEgCmC2L`ruwvQCN%m9z2TDkKqHRBMU>DP^mPQuqy?BTs?azy%ya9I` z*2M-^+=4x40wHq~X7B`q<Z~8p<|_h)4)q*H(-UUtfkvgH^G7UTrZ*IBL5arbE|7L# zPFu^v=awz&q5Mc0%^c?Db7xT2Ek{l(`zGKo1<++ZoqzXRJ^7NP+J0Y-B+bJzqh01* zFP2nMnx-;goJ>=s$1y>94s^C>iI@8D-!Bkiz>_4rxG`ank;F=+XO0id@hkNg?ly1` zO_Em(%n&y!pGkFkT=H}acTo@&IKUBK%~M~+JUH&etyL<+7P}gD`{V`*fv^i39^p_K z$Yn=#dofuMN@KSr!nKF3MCU<zj`&6v$q9NNOnvTa&y};q4*a?NcF(vdN7KKxP>2$; z)Z{c!{E6aheU;r$;F`!zf|Nk`D$pBnU$sE)`g>$~G2h4lPA{5<h)&lhvQDB-obvG) zU~8G3_R49)vfzX*P|?7zW6bhrOQL)V#<aD3YSjxfo!93#n=ywvr&|9u8k)kgGl@Kc z_i_j_UZ?gjCJQ!LUUW+g2CVc5+=IM2zC&ZlXHg0ZHBV`1c{KdTqwPm~B&7ry5G>tO z5o*Zx0J2}##BzVX9IYwWmses)@~OCN*#o$hO4`isNEhK>>Qg;``H@HTT{eXI&IryD zz0H%l37%5AqDpL`^2s2NJ|8f@)m(U_$CZGudxW^O->c<U=y_)ZS@!<?>h6|woq;V6 zBK|ZLQbu|0e4oKr9MM+y+Z6Z7Tw~01wvPsE>JR>c?b((x1=a^42{cQ%ND_$fDT!f@ zG?3zA_nO-+o0r>6l2+87QaC_-7<!kr`4Iv5S;)|9cRnuzLnqh?d-F_A1``_#oHW)p zj~EH`7ZYW||B78;m?Gp_gamr8Pk|izYYFH_RWT28WUsxZuLzGmce0d}!)A??WCO78 zFg6SrzBO=lDkY>7&?pH*{&_)=j2{kjyGBy0FAjbP!iF?{t-Knajy@VHHk!Nat{+fl zx)0!-tP1!0o5M}`nV}uKX?ml+Zlv+7U3}5_;unT~F|9q#h0xBC&^K@v*eyKGmeYBY z$^%!a=@YYds=+(hZEfzGj4ms+yYBB10C4rkjQRwjeWZLk&31VGhN->f)U5mk1(fUZ zc9V9<_AMRGqF;O9IZTBHE+U8BE=nCpwDs@Co$`u4$qB(uECkI@$tj;wo*Fp*TQq}c zmV;II%jG*MlJ&n}&wAM)Su!~Yvt@1R@&GB_T%DyRs}oN5U}-33(m*L1(iW1W<fiwZ z-wtMRp||jca>eW*f%m2rEs9{cGx)`=k%ISYQEm8nRksCvu0(z+e}M+&+h3YnmTA=0 zdY}3*+5J58iFOKfV^Wa73Q4LqfA`S`pWo~ai{I*l)U*F9d}V%B@rx-)wMyHSL^`xp zuL1&nb^MszkMmK=yRf~l;IaWxlRdh<i|_r*(PER+M(rLXrGFht6Y3_hVfnHhdPpX? zy%}Ogh4xKKpMpM>vL&(5H6>BPh~*C=auQ}$ut7_Dzy-<Peoc(&ApfQ?&j$b8Ea>GA zX`OVg65h*ZaTr*F{!h=~1h)i})0>k|Q(q$Z*qd1uLGgZr<l76)f2Anh%k)ieBWjIg z;wX^kajJE;a|P=8X$5?dVA*x2ax%2eXZH=CNvz1Gs+KgSr%ifpnFFZ!u_Mx+D>kM{ z1#-|b%rIPxb4!y-a~xzO`N>2P{^ExYmQQ0eE_gy7k|Dxt$QydM)-mncg<^~Q8+2*W z3FNR|)-3-brS|}tP$6pjaN{F!x-Z3E<^V4q$iK)y{uKmbxbt8B1@i0lHYHlfG%`Dm z&7|{wDE9;mM*Bu|jnf@m8)O)EXN*2MG})zHooer`>9ZUN=GO5qrvmR_8>owJ&%e^s zC{kqH;xcZoW@65xFMqb@!I%0z+*{Ev$d7b5FS_94#{qRN6Us{kY8+%3tY+kESl2?Z zM;<Mvp>t)j=b5h>FoR%<(DudFwT`<-x71r>o~Cn>u;N=7Vn?HNDTN$D9A9{aqM8Kn zl&GXP?V$*20U|VIc0^EhE7c-Jg57|N(g=<&u*nyRT41|avOmujRT#9L`}QJ8n>BL4 z!<_mo0aERtET32&ZhsZOi@NFRur<l^6SeKA5P&zu-OTHggOEp|Y{v{2zd6UTVJ8;1 zv1ud!njm0yvU3>psN>DBc&4w{XAB9(>N3kC8PqioyP5qd3|k)kA{ZOeFnR=elm2Hd zt{$PuNziB*8p-e0WV+oLn*O4lS_hwu83$I419_>zmv8VQV2e_k`BC(OWY+zyZaAyf zqddJMj%=BiDn_^$_<n5HpF7lH?tN#~8o*%%q1fViBJiuJHle0Vt4O$-kFtl4xnz*@ z$(ABTFvH!~b53;|c|FX(CZKCyqU`?=&K<(E&m^wijh8^^{u=y}cT5+z5syOa92nJW z4?|@kxg2jMz_tAjvZ^d(l#52$uZzEx!7N`p3$a&S89E%<C729BtI7Y}?j;iQpyd4_ z3{y83ok&D-+|E$L-px-6M5<jJ%IeL)uBk4(S8+r#be<)=?n(pvpHz%T=s2tk{IZ6X zzbVO@Mv!^t*SO{K2IDvOkmVpiZRUS%=C&g;M%D_={G<1q7P*Z(!XBcjx##J=lojHl z+Gy#rvCJSD$hZ)u;MuihYRI9&LJ~XoDvEJ&SI!%-Q#f!e<v@T614%YS6k!w|P!P!f z6vQ`D7Fgn01dQil_%L;{NJdDpYW<ln<iS8vkz_;<u|w$m`d1)a;#3TU^@?(Y`U087 zvxN)eEX1*pxgF8RQo@1mR0H*Iz11^KSB}33vPLWS!)lXH64{gbu`HT5#s;|wN}wWY z`|>wiIzHawBA3XmKru6$!<on2(7e<NQXP}&sD!$9?<{Scod+4{=^arfup(E_2`>-+ z`JUwg>q3-C)mmv|X)X1t012BJt`*Z8f?J$JS@pu&!SrzSeLis?Qg!-Bk<fgF62mwU zel$yHPEf3Gyg2B)KP(!dVLNtm)aDuKj~lRU0n&X-PajHB=&uR!j+Kaml|MIG!dOtE zjG=ZC9FMcIg%Jp6e{{toj=uR+c$scO{?oahT7X^>f#b62zYBAVvanC{##~#@B3=}S zFaeKJfx+ys9%spz{hf-3t?uct>nS=c5sL2iY=E%}GBII7|7`<KjuV+gcwtuh=~Lh* z8F3MH5+5PDo4rqzO`4`;hzv-Pu?u%}89^>4%1h$W-OD`WF0t7C8OY}_e+X3uG0#Cu z3qtiGz-CslRu(A<N(~?iA$_HQEZWo8uv>o|w~Uf`^vM1~qNZ#K`xf*wGM>_*@kg4S zK;a~+Bt`)QF)X>yh8vh09xv=nsC|rKIgw*`h}A`;S3fyq*3`?pte#X<SeeytZqZ3K zPr51Yvf_v-un;J1wbTiW@0<!k4+PbX3F^;_!w0@SR>F)^GE3AOI3K~B?wzxJG7vWy z^o@BHger%kk81O3p}@&|7J<SF>Vwy{Q&(PEMp?w8l{v#N>nb{d?})nplo-KZ%ZxdK z@r$T4lt^KnvPerxjgGc*E;ZR9z`(om^nVXg<sz;`?0!M3=}ozFd#bVv;r^Hf`-j*k z=>OzYU=jUv+xQYT%#AY9dlPDt3aCu-KJOCW2TE(9A$M;eDn2+_?#gY7ap6`~SaHV7 z9^|t*B>+uh0O04E7VQtez^MLn+lDd@VfrXIh9;MB)Y~Q+V0DHLid)Th55$!h5R?e} zkOY`@bq9|&OWc}WGj8C#1k>(EuLMba`}6Jhrr24y#q~ETq&gZHls$wzH|ZO20(XAl zc<>E@9Ix9=sC(v^(xLCro|N}YZYoPAV>m%+*xblVfu3tTvUyG6n*=HclBaL$Em8bh z@dNskN0F}PyP#+yMGd%6+Ow7>r=~8e0+;%42dlW@?vkz?hqEXXCkD5CJir-b{nr33 z1ocPlk7=oH@?XvOYwf1{(qgCXzuHZ~^**nbM7Y2mQmqvtn~QCu)lUbeFFx0D9YwKd z1R7_#C^R~LQr-S5{@wjgD3ym%kVMpi!@kF-ulVphQV6Qt7lA6bQ-4=1cd>%a3&W`O z0+p96tP0)L(0%?|fH%1SK?s7Kt+nZnnMeAfCv5?r_$;<Yw&}Oxi|T_y)o|Sw30tx5 za8jr<rT!=5?-Jtt+2T89?3!eS>pheMV_~ehFI(ts-HVX9!;UMaoxUluZbh%MLe_%L z(YDK$PTUrn#zbnpax$Z_Jq**31GJENi9EaUdxoCcT*!z6H;pT=upbGbcU?`$b=kQ= zW)UM=$YD=;kEV-2R~7@>Qj=OoNmQQ~X*ELR=n42HA_>dSK}3eTl7&*NpxEpchvJSR zqekXveWduZZ}nwoZe;wHz%dTwMTq~(i%kD@vitt3=qK)EJibd6J+CRWFlF$05{~jx z`%f(e6gepN$R6mI-KZg-R%tWugRkBqi}L5sKO?2X_PxTL$+ZhS^PFIr@Gyxx&P9rJ zQWfU)7B!f>Q}`$b&4<K+L)|4-GeU=BOViAnS}RGjmV7sh>McqZvuR{kj#$3vgXLKT z438nu&<)Mc7Ad_;Xo@Yw7_7S*3pX_8?QKm>KKGkcQ@gK#BRtb_loUJ1%<TJLu0HVI z|A^I9h%Gi$q4@F{fmiBK)z}pY;>7Hw&BHRby_!M4^-^07`Y?d43i$doBkx5a>v!{@ z{Lm5lBF_AFKO0P9;UZ2~RDbITJypiJXZB6j4}jU4p0~F*wM?_xfjn`qt@&C`<msUJ zU9jx`$j|PbSUOA_ZK&UW>iSm4$+tBExw(IMu}puBMO%(_S3LhD$`@H>rvQC9+#5=E zur<)%c4}1UY)_&CV6rrcQ6Q1QbNaQ;vWh8sVyVW>LaWq+eEx_b*z0o|VRIR2*8Y3D zFpPVttE#$}R!O?XjX?q`mV1p9b`0>~Jm|*f-z5;hiNE<M7|ff0+jA6H5$!LkoSPN> zl9kThxjuFErUtf2;35@YllwWt4lRA<(<X2YQ;d3oLgf|L+!lz^-q0rB3e$~-hi0lP zN^|T&V^!kq_r+Sv^x(L^ORH61UC2n`7=eE!4kYZEUCc*j)R=C{ZtR=!=H1dR&!t3r zjPjL<IJ9dPQ=i8@5gU^Dm6#lMP_nrtCTdot-d|W)ft=a`J+T`{(QGqOGuseykc6N8 zB0gZuFc`W%@x|m(si`wnhkDx;C65gVI^cW`0^$$`h(oOZs_+HZ)lH^%=hS9S)o62l zdK)m!BzPXiSHgiKdQgEF+JSac%7ob~v}g>!TX5Hp^pA?o(mr<tO-v8^8G!?EBPKlb z%L~Pr6v}H+QVZI8lta-*rF4T9X#(M5*%K7%S2N&Hc%X*NbF9XmM2=1_hal?l+0Pkd zf+dxi+=AsgVv!$o2YyK#K_XMuocWpL(EO4BwgT>&HqpbH^OFFbC~pf6-tsSV`=LHB zmXZqwtQeDxY8`5-SX)M#5bt@^=b;Qg?i?)+#r$8eOQj+0;(o}C50nH?NEeOxO)9<W zgf;dZJi&&n!fPT_9OTbzw*pbSpM1p*t&%0vUMj{`vzYZSB@pP{8Wl(DaRkaJ|05%n z>!i=vD82)4fB}vr0|HM=n|R!_w0vV$y0)f@kP3CyYLEf?S(VBDyenHdeS~zaf6MI? zIt2gZF+f1DIx$BkrB@w3TDJaH@*HMkGrjOLRiSBwNzHhj81fV5ZPJq^ekmEIb0W4l zQCR91OgR*N;?mtbc$99){xh}(F-(0^uQf{eP6vLI1SrNAZVekbZfoZjSQFR;wkkFG z^{oJGxME!oPb}D*$kccq6H^Z{?@>Oe?xEI3k&rY&BoAC$k~mL6L7a}M0<JmKS|?<^ z-{XsOKi{LFI_Cx_d7EwUc(_b?g}caf*pA<39v|@kkl->~R@*VCCD*6j<}F`fAaA!j zMLm^=WyNx&$8GBs_Z#nCpr9p2;^yrWPqBSYO+xw=7w6t@VV%V*&wwLDwAl>n6#NWL zW4k&%m+upx5Z$}HoxUgfzc-WAqC~LK@0WSCCq<VV<Pm??;=!?a0wVHX&xu)(@t9M) z)umxfvUFO7A`9IC5Rt~+;^Ks!67G7s6+^3vU4_IC#up|qKE@(L7;+a7$h|!Ax~Fmu z$!e!}rbpbf5)J*tn+`O_MPPbd6KxAx($uHv+XTyBzZL2@;qQ_-uaGbqK-PDwFbXq) z6+-vpNK>MO0jl5ayYmBCUwB+mlTK%8@vXx26>*GJc132<GKJjYiXZ23y({x}3~b=Y zCqGz4(ucvGuNRZrG@ixFLYt-r*VE2S2Ql)cOu?7poBc35t=u+-tmHJ)<#h!}a%?_W zA;#(8|H*DvmY%R=On<!*>Q=K6{*@?g;s_?6ju)FvRPBle?yA~qmCJr=_6kcO1)YT~ zZ<@6QW=kWO&_F)%{#=%;v-p0r;L7qvNvJ64>44qt!V*X~b4tTa-+nbrTR;eXkNWKO z9G?EXf=9hsoD<zo{0W&tZ(7#;2FfJ4oe@6B8&x}WvERhi;dYJ9Hg)_NZ%~IHXf8@E z{|U~N+C^Y+$u@_uI;l@3A+cir*{t3kNcCkC`KlZ7<S1e=1!7&31o<wr!@9R7kx&q3 z_0u2CBRjA=CbuE-)xX;?BwM<#(3K16jm;YLTRQIsM9n9<<)V7j0y!AO#Uh^}#;SuW ze1QY_N8NAdD$uGz<=$0sC?CyFC!K31)$9Tl*f)cJ(N>5GXpRffN121Rl~1B3^wrko zhFY1-QWCzQ2S+*&7<&rq2M)TMO*2xtA0cb6?Wq5+%!cy9gk90AeN5Tq0r`K07>6+^ z<MjQi6u@V9BeyU9&Wm@cd3t!YE057PyUYj`p_N7t?Fo&8_nC<VgGOi^BIvvIJw`nn zL#e0HF+qWid?0MPQ_gM+0&yk1Acm@+iSjh#;&G$l#@Hp>Yw-jfgnSZvoQ3xL@})dq zxuzv}U(3=;X}&7QPo}4!gx3IS<t<%JdI0>`)^AM?XntC=81LV4(sT4f*Uoq~&6^$* zcWL2i_0T~{R~J6<<2F1r*3#gh!9%_uj($R{yJb1bwtFiO>f34`TOB;A77p#{7RHtf zdJe9w-B_1?e>+5l1KK#NxX^yf)wbVD@praibyZ*|{wLVB$RZ5un&n&<kLcVYd&8sr z<>`5;Grc88VzXrs!oOTNE=eNQLQUc8&7KlmT#+Lo|01-dYR1OpjxlSYLTjpb@W_S6 z@~P1Lpsn4LtwjsbCifN9rd`6AnJtYK___y~cYwrmIMZ*hNK8+_RRO*<L<T)0i2`dR zFGQ+0jn*2g3P$e+o$k&3&8rioAu90U#;$`~>1$&J=LyR>8`xm96zuYbD04$~CQva1 zaS9Vv&Zac%Y14wMLlUDWqO{3s$uiCenSf1+uHTE;B7d10qP+VH_V61TtnKEfQzTRl z)-wF`@+^v5=GqRNUj3(S!hVN{`Y=1elM>Tk0sJPH1`sYm?)GV;2hq#V>;}dVK_`~$ zYU<p%Q=qPvlY$=ikGyez2=wWtsQ<ka14Pd64$VsoaG**6KH~qXc3FW$h169(wcqv9 zoj;fcQ=yzIHAP9}{A=1mj9|j2REH??t%h#bELS4BUJ>&kNIg;ng=liZ{Ft@X(Mmlb zWEyWRBwC&PeuC07ze{ku;tu=*Ja=>Ehy?F7o)KC3$ro&*u2KgXFV=Mw+G5~M?f`BW z$CX?H;m<Ihcf0T1#*MUj)G_KVNsi!;j;JrrY?#v@TLs!BG({!9|MHCBAxGLp{QN;i z2hjv!!qNL85$HoEe~i8$a1S0jN$I&2uk*kxPMhUFx>-RySR?@RpKLWPHQ8Y)-OMUk zRj`n<x<d(+_;ccns3CSiKN}e-|BxK(opv^qhsM{<JYF*^bWl3WZWAGL0c+8}>Y|@5 z=(6DbaNrq4uQqVrM4NiahteiX(X!UKcS}TEh?;S5O%(fufdKjhNgmoDzHY7EI{dlm zGv=u3giDL&i0gUp-BqLry*J6s`Ijs-)@mll>U+CS!OC?t$u+L5T~zeHDRYa|d5lD# zo0s#vmJlZos2~p*)%bL}>cs>lLocBXzAV^`C8Grw2Of|g9M5AUd3Y#zKjK?WZ4?nG z=Dk{7x7<3a!X^)l&40D&{yX4->bW?%jWoeqRv`~tkg-$rS>?)+JXB75`cAK7S$UUK z*`ZAyp^)G!qBi5I*^8%~H@9`@&=#gb+hBibK}G<G7j8BkB0ph}FWWUT&W_}Nm;EFC zTMGXa7V!5UF^0^QiyZ@fdQ(saCIfDO?urA^V(884mF`_bV0L|6F|jTk*7#(0mu*IZ zt4ab5c1VT#-aNN1A5Z1bxU}>C@%5HbaW(6@DDE0GxCHm$?!lelF2OyxyIZi}!6mr6 zJHegc*0_Ym?ar*VzrEMJXP^7G$6yS4cFn5y(MqaqpSQPvx0#^lG3pMxXhC$5xX3<i z%t4bX#hXunGFa56EtI=)Y^MeZR<&OG06>`HMrCJms;krBBxw2!PsKUK!=PPI4$M|0 z{KW*98s&qV>bO<71M)fS;R9GaLqB`RO9tGr<(Kd&72a}<Z6IuZ^f<$j1S}67EvU|= zOLqKfquMHEdc3`raOHE%Rx%s8j}YX_zqH6YqswD>JfbQLLI>P?+ROki|F6YU%0AE( zcH&Uf^nE>m1u8nm)Fw1iT#I(ki>7OIz({}$hU8CyU{;Hy!Xv>)2mJ3^BXHA0RSGn& zg#62OA2uX%#s|o7I~IuXaHa&zS(cZqJbb8-5v=LSS;^AY4#=fPmR(a(SkZQY(H>V= z>!masiwI9pO~k{+cC}0&0R<}#^JqIc@KLG?6QRsA=}wz$6t<`R9gI$UZDOjAQ=hjg zPa(|6{%m^RcPJSFqpTqO3LV=0535%JE((5v>)H1ig50~Y?Uo|Gzk}uSUot|s1Q`MD zZ4J+9ll(B%2D$a3__5+;&NohJvwnFJ*BCFbD-%F-Vs7duvXW0<L{MJwspArPIt?^? z{rUZTr8m+2#dR)-bVUtl`do%wF@qDT`<$00JsAm1Mokh=-a#()jX>o5sfF<)6GNd# z!u9;i1^?AELjPGkNNo4^e?jAa3mr;{_%RUPZm)Ny%k2}GdS9UTrr_Qr?uNUIYWIy- zZhw@$3l`C8CNh~S?TGXSuWxOR9yAZE&+Wgv%;D-zA^CZ(ysa+Oc#<JwPFj!>kd3Bz zr|l2tAaDqCpvMld=Dlr+W=|yG^UDhHoQ^F6)}xbYcsQmP^^!`Xb)S3$=jo&7jFVEE zTF#=oXfmSD6=~EOpGKSU1kFH_J&r_i60Bar()}wCzJz;DGqyrEOs<0;@I=c~CU8F* zzn9Yt|JTy-i5|2lhEypTVM|vutJ4GCl;X%N3OK|?%=SX(*jWYzOjtMQv^a6kbrv|9 z5$KSQzg(ZyBSpb>7=apWd1SU*$TyS(4R_OuM9vs??fz|2fzZO<?}wlS^fA2K0HzxG z=hb!p{@)!NlkU5x*m|m)!N48_3={&i`E_Xi28Vou8B&7-g$V{SIH5AWa=?fwiB9DE z(IUsZv3)V@MF-eXr-ve3xumhp`MBBD@JhrFugAU0cBO1#US!;_9efG<EFE5rE|ED| zPv-5B!6P4xZEWJ0QQlyV*sme>z1D$wR*-z&MLGg|iNz<)SYJxvQRkyQqcpl9L_{jI z;<J8{x6G4F2!Ly{qH9pGsxXfmU8m2tZ})eF4}{n(FRjumCPI{l-6`Oy1WqXPblIp4 z<qu|-$H+qCX1m?_0K)4JR<Orp`LH-}iL1J*#$-``T?Tc@JkWtHFl<|!VC@`*7T-5D z;8U!saJBgyDDvxp3Ch-fm?&7CBWXBth4Ux73F>vCpFYE5YtAr^u6@*RqTHMWhb1ZM z+?-1DTH|0%+L-SmX8pnMm&o_xg61LKBb}v}c48Nz{U^Gl9oT9IAm{(=bbtn|9JS<N z@Ryi7^+jpGuc?Vss=81N$A`HTV#P50qie-$Wcnga-xltU%7fP;u9)F0KONy4AYOp0 z08$w}b+Mk-bUCKrvDs5tFGA;g<#%P~QoCdoVa->7Dcm?}YEmmHi@im2Q?uqD4@<X? z_5DOS3CSfAuUSHNyef~%VHr>C%r4}&H=WklrfGIXh*iv{v+i7*^KRx0ZSF(4s2fG+ z%8$ylMfaurZ;38H=jOlI{*bJ_h`&(;K#3iy_=_&LynGXZuNUw&u|x5HCB`yDkk5q$ zX1Ly%8F1&k5pM(nSR&)bwa~5HS!^^HLHIL38tz4K^+@m<oopwB6HJ|Y9zrziE{|a( z!D>MyuvGT1%EzlW4^U`~Mk*GN@9eApto#rzEc5_u8(U?E^~wms@1i*)$U^i3{s`&z z5J6M1o3u9LA5Xsf^F=7mX;T8&4!c7ed9O~rbT*)RQ&xP!z24W{s)DpnuWSQG50iY+ z+;u9M_8&g1=KE!fbgppQ&6`zFNs+y}oA`*{dvdsiCwY{#6$i{ij7D+EVrQfXGP*DE zT0r#0Q3%-4i!6txFVHpd{1#}johFCvvY|6FcSd>PPPyo7Jp@VH3pK2H4`N{tyQJfX zY?*_P2Ab1qsC3#yp2(EMGHYdG`dJg>x)!W0F<hKDTbqc|SQ6d?60IMQvXXUo5MF<L z6%w5E{}emHG{3qbdQEUv)EWgEnouY&FDshre$;p-+H>{rFiZA(X%|-B>>&#kOOx5R z?D2_8;Qd0*Vagr`vl~xyp}K7*6s{jgwLN$Alw8t9!)11NxH3;kuPR&+)taVG%Vjq; z;6a~GXVgcutwzsTWI}ltgqrn^N_iOgdt5kfDH<u2sX;Gf3V)E}ld5P0p6#Lw>S4jt z@KXbm=Vr^kQFstP<eyL<wi8j&#|TLj6Rb@eo09z24#&3x=d_#&fgeJ;G!J(^Y}~y_ z8)E;Zg8)2>5BFbP`M(uBU%=PME!=g`DJaX>slt`O%!crTY0KCB58q9xbSuOmcAk<X z0p_9l1fdsNY!t2Tv`e}#oU<MOz-wJn6euT05?!`}fxGggbWnLJPZk{woi>FzTF^Sa zF<ZR8?@-X-UJG=9#2MYmN<n;<eOj~tH?5TNOVE_1r$1@A4dyi0IOeyVm8pZ=ynfqv zCk2(CwSn0hg?sP8wiEg17_cZ0jHbTA00*MipFmJCzA8IsztC9nBN0I7->IDe@!*HE zL+b#UO+KG5EJHI7<8D)?fmL*=?$@nY{c3Ps)__tvaPA;k-=!vb;qiO21<CPAukCNW z+y8k!^RtyA?h3^j3TXyV#Idx$M~J<Pf6w^tvQD|3ru0i*atHZT(XQ$qaQ7)M#Mq{v zF3kdqx=Kl_B;%UH=(&8anP-oLA=+y!BjFGOvPN5vHw^_G8})W0vI9rU4;mzL40KEP zkco=Ljvc<SVky)OEOO+qvlsOre!p@uYjMv{AP|&2*tqb2daMmV!@Re!47Er1A2g;> zQ;CoodM5yv8N-R#@3rN#hf1p~oR>0Dl-$oD&I5zEVOzRC9eo8+%^QwE_7iXHJEPK3 zyTm3;AaB=4dHN-MkV-Pil$D9{{LdAKAJ)FwYaQ{C5XPh=eG@}ElerK-NYW&Kr%ZNk zp^8!mIhOx>Q`VA0j+Q$buXa#zS0ZKNC)(52El@tJ3#IqFeR@&XQ~l(Bf4w=f{q@_Y zc9HJ;iEuDjlDv{9O9lkJCvlnIB^q)lksZ$mH~2|*isYI;eA#1pbs4E{ZjGMHvcL!B z`Z?J}ESas=lfAAMdWfIP{?edo8auk_r!k9F=9E{F7WXe(%Q<_y!r2GBd74Fy?B(RG z3%sdxCnc4F+KjP5zTbsEw`a)~3HBc6Jq~zCC8_sNs|J^czQ=J6o1XVP2ca_jksqK3 z;JrWAF;bqzWmT*Uc9v#I8TN@d@A$h<)#tLld5n*Y4X4dlMDfp9={|Q)A*^g3^tX`1 z77}1C&ybFF+5_hihuBB!;|8wWG&DJ^MgoG?Tw?*ps(2p4XGhu%$IXB=j+j0SFKd~G z6!+i#Dq_<bSLaSi|I9al#@Iy&oGj-5xRd^p<V&IQ5xu!kWHi^pr>#oyv|npRy*`DE zh1kP-1vri<*3vXKBuJi0=0QjAlSkTMn3vk;eRGkZXj3ddLCOznjP~sv8H2!5SY|L~ z!902*%Tn~`+3s|TZw<cdJPLCf?0mG-udoTKSFZ3S!E&4C)YVCsP>`JZ0DCLG9(`7^ z+vi)0H{|_8D9@j64tx6a*lZd3<EM4w^~pg_Th6_Fw;9bL;$ZH+ZF-!)(%%TD3C2yB zjIFI_LXLv4(LNpAPN7Q-pdQUZVdrR5i0lnLiJ+L63_wBHq%hYa^d9$gH;Yo<`jyXj z6asY8!V!I`m|EZ7ud!l%Ml?8UcCK$j>@HSWTLza98DC>nZp7#q-tCevMT$a1V=`VC zTd5W>lvE9-R?wbOx^kXn{Fr)t?c2tw=LFUCBnIFQ@b(5nEHHMM{GnTyBoQI@bQdIY z!*S|hZ`I^5>N4sNdJrQpI0q#(T_9<Bh@u7H%bRTlcm;}F@Aq_Hzm;KR9FiiqL|iQ! zm72tNdGtJFhG$kFEIKe2?yV4brjGH7RTe)om}O(%qXDj>W7p3_nS!|67^n!*+bah9 zEmsjBW@5R+Gyq$PMd+hsluarpHffR`6Uw^%N{Y=VJtOW$-Hw^)Ww0ZU<qx;y7QJ7# zr4X<fQ9?0Md_38F&3-+3ds0fi1M|ANxz;w5nKefDJl(%Q(larXZ2E$wZZSIEXydzD zZv>f`R_2aL@#B|KPIE)iC3}ft6jp|A`Rq<@r_WXHwFZ8p!@dYQzuMID2EkJxd<@yl z(ISb|-pY#*$XIqnWx1Pa@x7nDa#BeM$Fq@bZj2o|4b}t5w&hDW+^k?Z!3Hue<J+EZ zv`xuT6Wt@^s&E6|aZ$09eFd`poB1pzBx>#Ir?9*&YgnBT-aVzryoR+Z!9EzCn`UEM zCM>R*zpbQwsvAg{afOyP#y!i)4Fftl?-k^BGPy$J-#5kln)FI%tbvqTzDU{n*gTx^ zeb`wvb&#OjyGD*HH}<4*b7RoV<;a%2Oyg7fbQlU2-RnAP-BX&SQEjG+JY_?odH=wW z5rTsc#iB!uFh-DEpd<l?Z_V%`(DM53#pA1|$jF^I(T>H)#&FJNy*(#ZI~nqofJZis z>5&iCe;^J6VQiDjkN*f*z`^Eu*@YFXYJ?u<Kika&_lt<)-O(p93nkuR?YFUz^_7PR z#sb|X)y}-A&R0O&?KmiDs|o<mRZP_Q*=Y&<^e@vL0jpJ%1Z<NDoY#QP46(fj28DDG z2W~6m;HnaWZ#=V8zpM(x<X~Z%QRUObR>xcl>FY?9ZEf8^UNBtgPg5~JGI8wY?9FRE zax#UxhSqv_Nb3TVniFbp-k0dsPp5P3SbP|39RIv^Y6RHGmp(m4;@?zjz+`Dg3keDi z6s=c67{L(OIhXRv{4}L*d083qfUV9@M5h*`XV%32^Mq%r(J1=heUlbrjb%>~;#l2C zUl<$o6dP@v2#WnVd{KN`2q5u8-7{m*C}%B_#~j=HD2<o}c=z?&^eie&eO5BjnJ9!_ z<HrQ>Ms4U__!yh>;1fFznO?47jWi1=2k`#;lKBi^B2$x*CQ>q{x16j<eV35RVZ#Vk zrE15<$I=x$d@wm=*~1M|rZ@0u!8N-|_6~BH&VI(oMd*FNPrE$*%sZ?~VbQ-7_7|dS zSVu}rg3Ewvb~JCqJ^>s^OqW=j@zKJFMRs>djNRJQ-GLu=bFw%2FUHzwXqT~s{3jgN zq=ojiLI$UJ_pO6^<AjiJoK-2?yt`fzy%$~Wl5e-;hI6~mCF;+0$vJz%C6S|LavxDP z;0KIkju;+?-$1#C{ld@buTR~tK^!{kDG8BfyZN?w2F*1?;&HN{dV_>e@=4&75h{bS z@W?8daocGxKf)a0zB5G*i4h0m@#hi>ZJswrx@au(VWmbb?~zx~9_%*~qlfRoA;QPE zE2^=3V-~CbVlvsqp~nBMpfGP+inZIGh)TG%58J~7<7xa8cWfB};loryCylkJ0U~~4 zMf45&MY`<D#wPV<Q~W3ghVslYioZC%*Ip^fyUZ3L8ybKitn$IDpp;65x0+>B1jrG8 z^b!3nJ-_bxA<f<w8}wFc&W@FXjtUE3a(S}c%b~BUMSRyp*%%k|N)sl2l}r5v^^ts% zov6s#!_CMxi`)*mz8ZcQ*A&O^LrzbRImVT~>lT3q>C-kepZf)(FC+*>G@?6}3Qgs0 zm)pyX^D_F^(IVWSR!~fEFv%RV!5jCVz?)9wNiN+OF1(KjkC|h<vP}UgmEkIzEi?yo z7{Mb;yeH`<;vav#u>VY2<9@7M&yR0%>?8xDBV76EPCf8{Vi+L|QfRfY2^F9Gw7Rd# zYcs5M<4nA~DLj(*^4WTeAbIhKzkJy8?|TcJrc3Dm$zK95HQ<>vat=sjy-mwlc+WKe zeFd+r*A3Typ_Y1Lp}BDSd8vN6_edWt9(|Sqa7N$|nXrUli$lw;6HSyZE%t^G<OhD9 zOq+hWa`5a9So2@rEJVi7rJR6_`H{yS@+wn4m+AC#84?!LJ0X`1%cz_E0DnhXxKzB* zQS<{2ejlYT8ikfIOU$(i<cS+|mGQD>Ur&d7nF2W+l_WWwi{)-q8^6Fh&F&DC;OET( zyknY*U)t+##f3!vOMm@s5K&B<=WsVsgP`BrN@cd^uy*J0T2AQZxY#?1GFXdLRnthH zt)tT+HkPlyF<BPhbaHAui<YfZ;HE_9u+BLCwYY7fr};6`W5sPJcNFGBx+2cvI8|fz zF=zJN%Z1@fw12{`A+u41#8xcDLZEpX_O2+k<y=f!a7fL`zzG?pWf9Ew5|I-n9RA*I zzm=KSAf?EY;EK8L#kl1&@U5Hw3J2LP5}`RdLJwKV2Ei{9G;!Gu^hBzrH8L6VOh*H$ zEoGSMVmCQcv2O&vcsc9<B|*(nxMv`+rGsI8i60WcfHQE({Fj03{@rXlZ(CkKmHXL? zht8-<(r;{g&@aP9$&fGXDZck)cyWRx@(_M0@G^^Bm@UF7CgBGe)ENUqAvgZfc9Kt< zD~N<)i_#hI6`t+Bt}x~`{_T(djQ>OUTjuM&MR(y$ZDdHMD^c`_r-O2cZOFR^-G-o7 zHxG}2x@53cMNpI1^#P}Y4#r18oo$O<0hPkx$*Dl8eDKB&-`=i^9Ul)oYz5DDQJU;~ z!}s};x#>{tLQcTF57+TDT_t0!@0uVQfgRiOq`+qZx=H@#?BQ@ry5VKU0PNhMG+ZL} zJTZG0tGn?mGHg;G^%~4cXW(nx@9p6%!VgcbgUC43pyqn?Tj~s2y#VNzE|&tEos#Uw zZh}x54zRQW`}#2?NS)2ebYMI9p`W_s25Da5Y}WkbxN#IfF0Uhiu>EjD>vaK@^bP%! zLnLTz-$Ib2U1=(m@%NAFd~#iM&g1f<jl<@7rpf*T99QC25?BL|QqZ53v}9qY+%QRf z!j3jMkOafHiLZ+-lS>3)N4JB2_L`U*7lNmkhgHCzfzJEit!pL21@5gz*JzS`uiAut zBIvGOMS_z^kHiSP6UQ7&41G`YmWDGfD$Sp-&3Ab3EG^68eJ%?Hv9F;Xdq#zbBOCX@ zH*|Pl6@hUq9jXOw;m}a_)xDX+yB)!c+-}fd(49ZmRm7M@RO$LOZ5465`dgw`k9+l5 zRb43CpBrXa##3MaT0vtY*dn-%-NU_v>r}I!TYkNj3VeK@y1V%li}uw=sAx)P^e%*7 zFHPquI;j_eynmdd$Dv!w`EW0q0Ij_%7%yL3hb6gGEO2v=RM+#X$d&RYDK_aU`N5PU zu(X!`&v8CwWF?7)csFqgnh4=kzQbnYd0H+t!b-*V{(vsk@wyTI?%YB+Y>7jok6Nue zGP;4qkn!+3VO(Wrf}cCj>|eUcd<j5c{^N-O+^(#iBEms}m}d4PJ*ZxV*&Z@B2q2bQ z9_cCAKa=(KV?#0DP3NfK@@xX^cP7wFY&lBZV&_a3h`m@}i7&9)nJT~9oL<bEm40F2 zEIX(E+;jQP3N_&y+=jPwAT%$DB7}57usZQ_lG(0yfW5x>VEqHw_L!VdwQqZWdw_`^ z+goIhbzp-lrwjI9n@_^<c4+_yCadEM9l*5$v^5}89HW!4Ae0oj$=Ap^7en2-?gDTg zQoifE8O9De?sF}*seI0*;&EMr%8a~F(g$~%u0mWjCT~}TikLc1Vcc|&7MTIEE>Dn0 zIMYFGLSV=!44END@lRQ`p{urf#F$RN|J@M`*Z?anVot(ZqIV<%uwm)KmKT5iX8!1b zr=`uugzHWlwFlL)rK1p0Bsc&y;0k>MJ(c4*Gi=P6@{6LwyH30)15bdNY>f)J!T6H3 z=L1I!{WtPWj5a$_1mTp7Z^@4|Ij~e5Bc&v|-oKnkA>;jLCY4p=D^aLEykmj73xgtF z%IFAVIAQdgGB?4vfH&tn5b9qI0#uR!2y_RT5Dh&vMt#1p$G|0Q0oQ~#1~&36;t){G z2{{}$cXLZt!~^18R#rway5Aj}o$dYPK7I#awBPxr31$&~f=%fv#}-bTg(rOtl>dT^ zrBK_m;lIO!#3iFFwELUFKUxEW+0nuxZB*u{_J03^sLa<cut&1XpcW6WV)d3+Oi_|) z?IOu(Vt=<hWwf}ukeb*O4XIkMnNaLX_)2E{^?bHIR=4yfyC|m3OwJ*<t}gjGzvhK| zW60(k&vA2sg6LPB4(i!TYpXRb`Nj-cuW~PG7V2C_l09byOhVosO47Z>L8zVM^vA%` z;53egWhMO~H~LJvd{Fr?SnMX`LZ#5}Q|u~Vh86+k%P~~?B6ts_VtUcw@VQ;=Nw<M4 z$MB;*+3_Dl@7L>_^2@2x-*rbO)=P~MPcf?3VuCo)PoE|uH7?O`t_RigeChVe_^w_B zcYNDRH4GKesk?UF2<vWhJCg#VjXz6$iH}_5>IB*4=;>xJ9F1OScakLf4=dH9adKK( zTTQZw!-wOeZ@*I7708`Yt<ueE+!K@JPX+|X?7YO}pZC2PGJCkk+N8EgzND2RZ!ErP z6DCjM02BDPRYp{Ghl;}IvUc?d$Nj}BNBp_^+W`_&XY-hbst3nxk2=j?HlH#Yr>3JE z(%tUd^XIpJ*D*bY^>>TO5k@yOHGbmX@tGGd+`1EFgX?!u$#06(XT>umvRNk}XpWC) zJM5et#*D-F?9U99H{eMc-g_HtGdERGqnzMbFasY}&lA2DJ54y1?#>jEXW)R0D<*hL z27TBDUfvB;rwvOGzrX*@IwiZwsj{-ZPU%nf{fA<oDM=J$$gfT>J-WmouQfbhlozIk zW`!;VVY{#<%QsshxhbWtEXQ+-`L{tKUxR`A#paHNJ>3tPhi`;=t~vaD*MVSzR{f7R zHR}IC!~GRc08+Xb9^*?yxhm^Rh?-JI28)Sc2>+hhAB-1l*Gy(M56tN+k>T)CCuwpV zK9pKT#O~kExT|=>ABpj_@P4=lb+w3UN>&f#psFNe7{d<Tu~_8J{5qcZ<Vceb<UnLU zr{nqp&+YVl5aG(iK1sJ|K{f0BFERk-42sw%Ixp_#&r0b}y}$oOq#hL<<X2@{QP0`7 z!0a1eN;pUZJwF6*(E;diH}K?iH`f~_pr1wotfl&o$pU%>#2a02E#p)KZlGaOB;arf zRAvzWY{KyV#!BltSh;@(E&9S1%I-S~+sKA)rknNRl2crnnKXi4Fh)kwq?4D>Lj=;o zLxoY8xe`yI&MC+|iZzi}ocMtH7H+igbq+G8*|$Nmo|zM1Ws+pU(T*4+cwx`lmrc^R zrBVr1$|iS_Z7E^e5pa9!K^7vTzmRWt<Z(Sx`S(mDbLk_MYf8fbbrtt;l`)piQ>;(U zd{t0%{Xk>#JvHp3eHB))4Zpoc+p$0y)?;aVc0oc*SFpI99HM>KTT`Zv{0+xBkz$n3 zBwU|M*z*9{UUF5A!SUf~KIa7=Y;YzA503=t%M}YY0?ZVMp+o6AK4XFK^F#HXbNgt0 z(<lGe6aVSmXNx_z>Y-%MOZQ*CB_-8UV}rTXxJ`8-Vy<riacJ>}uE{QbNU&oC+$Gy> zD}rk{CyD|6zWvt-A1cAmwyDrF{!Q*~o_YmcMZ;=L)r0O^Lig0qXbxNz1qhL7=rkUZ zy@Z-)>P~2a_$7H;79Qjo-0}jqTkD}gvE;|W)p)SD^_Ll5PgzUg15Q6QVRT;@;R4$z z7s@4lW>Fvzd(bxM*|NFlSK+dW>*3vNDP7OUu%SIjs9B&n>;j(Lo?NM?+(%{5W);dC z7;CW~yeUP$MHA#z^kxh>IoR<YVjkY{%nPm@cdY?|E<$OjopsuqiHHpFY$!&`=!Sp0 z!P4;<ofQ*IFpR0K7d?-{YcnWGda`<Dj+GdGGhZJeX)Mj%o!gaz-L)DM5*?hIIvTwf zEYJD*L-ZN~_50)0c#UhqL3Ta%@=bk75txLwWM{+J;g_Rs$nWdq_+EDOsa`=Obx+%e zn9JJ!ra(F0tfz-Dn2Gi;JRcY-8p#L{9(O(0qaGy=muQkotG%DI9lD={1aa<|)0uuS zs}W)akKCwsE6WHTxV`$CcqXGFSp%ca%e3a2l4w|1XH>PHW?|MAp`ocj$P>cter6^) zmu$jqUi1yhJA;dvnfX6FXAx|JIl~sNUu&yo&#KeLR@x<pymY8$-RP3P=HR6xI|>fj zIC4u9+^uQno)j4Paw{x<vT_}tc%fN?%a1qi>b^g%_ttN_d+^?VQ`15BwE80HTc#u> z+g+A+eDIUj&^4DN)>nzC%}eLc%JMk4<IFgqXDYVgZQ)`x6zYK->m~jlC5z+^FsmW| z7hrl6p(m8Mzqh@N&*B<t-!;GR32<%J21bKJ`mMFzqx{Kp_eW^4?_HA_=csp~2Iw=j z2~|j*@VQe0`93%LxAGE5IqIql1JF6a7{&KtBTxOKkX{{T;M9My82#L?y1!e6$&jQ! zZaV+923t|2MfPUK-6i77!z$}gz5&FJv@X2pv9@i&)YF^T-B6dQqz1(r2cTSH_kHZ; zM<Y>^a+p<1FjWyN)%W<4I@vk$s2BW1C62NqDDo%83b}Mfy?<=lqbCTu$h@R6Md<7e zt9_U91BEke<wccKgf?&Bg_tF0NDb!$SO(K&1tur{F~dz!wLLF=bX_7KZMQ*p(D<1r z&g&Z#&N0E5@t%!X+@cwT$r6sU>gxdXtu3%S;EIj*eU$Gk|3=aU<<sAF*o`o#Ocv5& zc1#Y1U=chncsjAWVvs^|z&Gh?g2B6p;iXc~M<bYpfgxessKIMECUN>%mGd+O$)u)# zP4<;Icdtd@A1*z&i?%TL<v9(!k_PY-9VOu3=)%R{>i)=`Fm4Qk?&jt|q2SC-zX4(K zp<1SuBmXDiw?KNE<8<|3xD+tsPYsKvL^S-Q=13KiWbnI{Qt1?b1n<Lnncnqr>hxYk zZ~9%Ws-H-DWrGdrL7!f)o?NtYkuM0v6$65(<SRaKosNE%?#_LC!LJIxZ0=sKm-+o} ze{TYlP>;-=LAu5kXD{#TgT6Y`!O5RiliNM}>BZ^#+F`KI^QiV{L3qFW;qN@96>*#y zt^_PN<7in6!;o=3$j<M60}o*ru|85t!zB7GuaPaASYB+2N)wMd1Q^aZeT?L*0qzd5 z)I_FW=VMsIYs)?F$I>Sd2iYQNMJ_yzcgMNhiTJ8Ri}W$BEVWNAIFoV{)e}Qwp+&f2 zxR#l?OUP^xY0-(c&?qm);l&LseAu1`Yy^kFa+|IXgrsVb_h^+`-0o9-HNg?Q%flPb z&i*wHt7Qgi3J<2}53BH$&4(<V-Fcg?FAVF1uW?Lcm@1Ge0!A;f&W>$TqY@g@Fl*3o z`eh-eX;7{qcPoP4%@VM#(a)@wnd)Sp5H7?|MF|`6kB`nrxWV2t!sozkW{>MG^vc4; z4q<P>U8D~gIkAAkts!VQ85VG8xDC2+<Tq5O6(z^_^v?=cz8_>)8uaZh#yghEvfk-% zh*lLclmO_;)*yx~?gnY_OZVN23OM{}r**qYiqxK)%jrYNL_b;QE8BI`nB;$I27fut zziuY@#ly2*j1%iENcWlVq%RzqfSVV}8Qn}4-%{)~gebn|rc9#X8{xkx#fnG-l7u%S zo^-2>pL#%ysTlF!<}Z9w<NZEzr48k^(+Jp~D|&LU>`Eidh>3Wp+<<Jh)97Buz}0kZ zl9Z(c40}|Kx{~9NgdmNOXg;KGy+hE)c2ACSwv#laqgqUYS!|AP01|*0BOfUK!qRcZ zd1Q<$acMUqtXaTN5Og$5G-fnbwlkfQNR+%}UIb+hc*)|D-m$=^O;#3FLEnE0u{&)1 zIve4dfa4Gu;KP7`^V`^3_WRmB#gFf`kMe)n11e|)nAc!b;Nl7<yVFV<CbF&UggXaY zPu3%gUgWj+I(hA;hALY<x!`(MO)#^oa(iQBO1$ci3&3~KGPJ!4xEYJWccE@<^_JAK zt5o$i$A_?I0@@^67G%%l;(FuTQVIX9?@l4w+t0ShO0viqs)(u}-rS-p(>5>(ru2JH zPEM#;xTAYtLw=uw3zI7IH%jQ(xCyz|t&6}2R&Jmj5vmHt_c4TAA-&#LpoXi5O!&~z z%}<F{|C#l!r{3+Wd4dZo1?VEgGz53?K)Ri49QLHxVt8W6S!>sbF&INQ;@xG_DH|qq zFe!#^q75bNop{brMyMuW;$+(|JD)>#CHWk^`IP1*O=%{VK)<p68BH!Hw)VrzqFgBR zhOj)XY@o|!C0PQ7r1-^xa|_*3^VOXvsAa=oQV66svmlluVR^414n0w+I5%|8ceK=~ zUVEwPb7y+o&@g>kp20LaK?q0AQsH9X?=;Nn&2vz8?i8WUymJR4tKeV(h%&`RRKPud z5qZmF6n_{EyD>=i`N=kW-rfqgD3q^)Y5t3S;U`nn6r1S5&hC<q(RQ%|CcNRykd9dj zS$0>JrYml;)!v8Bc(nUYZ(BPb;Sh)+lEcYEJh0<4njp@Z$7{|^l*b(&a7&9GL%R3m zL-a%EefkV?ceO_~(p44UG*}+TdYDqt41^|4@;r3WebPOB-&tq1>z0mp6SG#IAUfTB z=`(NM`;}zjnBksKcLEhGt0ujP>OXXW&eDUYrery?F%?zPi8X&R=+`;b-Cw^2PmU$7 z9r>JufJl9Y%vwsH%TkPILK>Kia-DOO+_ym^CeiD$uyy?SpDv}pNkIVp|3XP<-J*hf ze?~E_<UkL^))yU81$$>C61g|203YiyL9v^>b28lSV_)N8bQSVnDf-{)gON3aN>FUU zf`cOc(wYJ=HUA=B9^vCkOP4+}Jk7IKcn?sKj#?!jLk#Ph3=lCrFtzR%-4{KQ){L!E zNha})sr;P>$SM>Xay*Z@TpcGHqWj=h=UR#jWGCb_wnSu1F*AKJ#JrcVca)J5s4t_S zjp3Y?n~N3bY`;+V)H4ZDavob-dU?{*^}@;`(I#4dtzJeZ(?+J9@7!n?(Ld18LDDG- zH{Fr~E?z47ZEASQNk)|fxw>}V>}1IrVlM}Wp|}mHEpQ#fCtoS!O)F+KZ)|50-)m_K z<I4@`b{KCX^&uGkqfkq&-1_R(KMQB}A*m%z6Uz=YYj#Mrj`LPSk?P>F2(`_k{zdX3 zToTcr@1w(vfUaHayGi|8_)`0jilA3F{4#U#O;&QW*e{!HZ!!PFnqy#8)%S%kb(O3* zA)FA-gtgpDcM@gqTLNMY^X~KbQaCaKu|PkSSw+eOm9qo?NON~D?<(a~WTSnD4cUuG z#f8_CZ+nN-_pdQC0SVcOZa>sEKV^%BiP-ObvSV_^&m3*3?SYemFM0CO|FDMjdKT@? z)7*CxtbDK6bZUn65#9MqJmZa8dslkrV$_%G$ya~fD2rTYl36bOdJjKtkZH)>YlLq? z0T9yp3dFugPPzYFi0|MdO$Lf8=N3hq_w=m{`}li=^+h+x`Skagu#Cgi|4cGlovuFx zMRb_P>?|YUdAum_4)a=kx!0~8uR*=w7fG6Aqt<gGm8chaG5qi~@58_==NbrY8S7mg zK_7E>Wjoc!CzL|B7?<-&m8(})kC1PO7XvS>dq$ot>0hBC9%&Iwt)BY(vBN$mD8Zku z&(|+2RxwE}f|>5S9v0r)BhxBv!tPPPsYoR#8I@vaOI-<%+qJhFcc#&{?|`Bo$@Ic` zQ7*6)lm<TD8Sva$HLY$_{HKog_dp6*3MW)VIr#Xmg&#(XdWoOqR>o5_tpxiY>%ROx zk%=>LoZ`JhIS>tiQhp6khJHYF_q%S(VJb346lxOqHRfZxYAi{Ns_baQjxvbAb-Mz0 zPzB@;JEASdCnR)%{Ee<Q^d*G;=2eZkdwZFT?H-;sFDxu-2Y6%u6d^R&#U))leesB% z2R(PfVt*Ri8OnFHrmLqlCILkA{>N(~AtLVb<ru%hw~GGygLU?KAiTGbpytIx9cM^7 zt9AoDSqVFSZU$i<-}|$mchI2jA&H8-|19A3$?+78CNWi~MiN>~gY&&j>H6Bp_GqCI zm1G<M4`vE=NT?RZv{a*vAkKL035)pbsJw%yDI%PR>>ybJNOUb`ky^h3hX1N?{1`HA zZHCn=OaF1e*iZXUqOK!AmDm-f5BA#EV{+>J8B6s3e4Di;5hG7aGfCgVnfM;f2g^(b z<Hv|D2+SK2vwQHo9=BDOz~3oPF_=o@7{8<^Of(Cw5Fv-^!`(h#*HuL86*?loF+UZ1 zzu!%`=-Pm*vtRR>hm@-2!r~9|-aBb3i2;a2*mtNGW1zE#`xje3<T4_Mi`YY)h?eb5 zSSO+_<+6knQQGiLPPa&+@L}Q{yixWjiy?Z4P50={JClv9=0<F4I5rkNJr+ve`?Go_ z-WaQh>x!MOKo@OcA}+2q^cfa**gN}~s%086G=hekKO1!EFD4W}s*GzMoX>fE=dT>( zMQFa&?*$MIkeOkXj1MNh%gE5RvSIjed<uW{h8r8Jq%8TF6JXykin7YBk1c)shi#ao z#j+Kp8eL=6KkLgSsaYez1cP3TU0kBo%jwE1%8H6pHFUJ5oxowXW19a2#Q?K~z+c@c zz^@!wsenXNRb?y<3LPOdWe{J=F_BUHE3y9LblL$N@|Y4_tz^;|tB9HU9$2x==(RWO zVqYm5;=O;5YP2Ozu)G?gF|x5)E=y-cvwb!?mrcbV;O^N)OQj~W<*;FY*3=@oV{w~d zzY*W)q14P#zq+gnxa)bU2-72Ka~;^2!UZ98@2%-2o(bkjq-$sAe@h3M-DZ57Ma}mY zQ3=7&)W}VbQ%cfpsBKZIh*3@IO2tB=L8g`BV`R#L#+0ovJBErXqFaKwVRoB7ZqfzT z?xmg3mAyadJ!N=E&_@=MOXkdQNbBj(YVnPN#}9|?yb35${v(}*Zlw~wu5~+%??prx zY)-%A<851FK~9NIRycuJ!`a0xnR)-N&@}wNWk&x~-ar9(cSlHSx{N1!H(V{yTOl`D zd1006<!?6<pZaQ0!=${$&bxHn(!ojRcdgQUMa_5ZdLlb#B%Pf!i+9m?2^%!p16~Hd z-hp^)BYL%3z^6QS6SzL@Cq#>19b40Y`yNDaVFQta<voHoNM5o~;nhY@vYE^>@g<(p znWDiwZ|{8{AC|zs<6kAt7d|h_Ra8_=16K$fjyu?^xY!UjHn=+;-Wait3|oK>%1%Fr z(x!mR^S?zIq^fEvv5b3Zv1jP&9C?aje%4s}Sf&M-lqd^!wNtMCzxx-&i#ouqK?4>t z{r=QoH!g~cX4hJx7q!1U*ME)5v@G^~Z1_Ot>*l<|>9yy4&<$YBRjV6=+bYi*w$k?< zg_kQP+e&!kKa5n`Lc3@JSuP0k7>nNuolS~>^%E#=Fj5+^C=Wly#GR;Tz=JS<7s0hx z4EUE+gY%+fXnc_Sbg~)v>I(4#{=i4Yl$68oq40d8fV;Bwh+DLjlj&~iY1INe)gcnX zqTh?)OkHJpwGuKhwnfH4QKKY!^51PjH!m#1;m$g<gyCD0RUr|6;km@~2xumiMWG1U z$ty{sY13~Y+QH1SXW$X7G5}|_2hvo_!n}#KQ4rtz9<{p9O9OdB(}qcq5th0=eH$sr zIlM>P6@X*qe+dDzvbD6+jQjC3K1jZ$p_)DUt$Zp6@CURr2ypU#W=bw1I)e^VrK}P< zZg!uZc1A~CZs)->tIvzO7VIO#@}yTFgj*Ftn0cKM!V9T3L|e67o{?kEkx@JfxQig# z6y5+uaHr2}Hbsh#n0pmO8en%8c37|+zC8g(qp1E-FU0H(kzD^9VnLrj$*Gz|0Fh>8 zY8t!*?SU-h^(O`?TF%m-fU2`sn#z3t>&cUMLu>13PB7`qQMJ0ht`zPG#l*xEyNs5W zp6tm9;roSN0AKyPE$Y9p)js57*<X)(#qZX(uwL?_;nm(fvivX4>IrHCMiL2#kv6rL zpI*b3&wGDhltPIzV^om^$l#%A1VzB^-}6*(49s9%aqrCrsxoDc{T*HYc1p1LMFFh| z4-m4xd(mu_C%lHzK>5w*U)_6`;?^J-*9_g|Ifb}q4-Y?jVBG}4e}MUO+lYT+W!9P- zHSd|F1i!e{isHkvK^pR<@c?J3(#0m-CEaKjokEI06`_zqK+rV&XB(0hC;IOvTK^p( zPjLfTjoTntrINBx^>I^`>tmU~i(c>M@5iNss=#%Eh@ddErc$rGn1i+B5F|XIH^(F| zyE7opT(sVQ50ZnVc=r&1WI-}l5==THH!=5}7gJQkMu2_3@F&%o_1Apl%^V;ZaQxNY zHSGB2^C?{x%B|8}sr5MyR=zHL<P$`I1uQxg#}QGHo2i;Q1*a}Rdi*a<8_w^7FZi9a z@k3N_P-x{WkVZQ9WsitK0@iJUS9Cz9Le`(+t`eFg_foUl_ke*0LE9#G-&f-3kfV9F z!Sq|hIWO~E9Gx{c_Y4qzFa7$;8_SG3(_s_FvGaXMP>wG?OfOS0t|O7TP5~wVg)FVo zv$Jr9ak~3;r+Lq@?fzn}-UU2Sd2#gg6mR9|5kjr5*0AGPGil;$8tiw~dr?+YnIs-h z!R=!BM)vYD?l7|NJU9C-+wUq9o!5ufOt{0v+k#urTSY?S#AI>N`oDhVzujYh`<nzr zgg*oAHvz8vnoFz+f&JsY^OmK>(=%UW`TF%fO=1LFdL;Ol%Wy7mHcf|3O|gVeki(Rw zF1j5bpXZ(D2oVqw6^oELTFph|I(3ek6{q09zo7=>^sO$3MA^#8F@~4}c;eqhXFzzM z7lD;G*ds`cbo;|+lR2Ftx}r4pDIOLQf$+WWJ^iw;^MF1FKf;;7cgf{l*ruQs=Y-b! zRSm)%0)ca$jqA;Kk@>j(>uiOt!mA)@Gc3}GRtOPi)rlp@gP+bB<0g2pd1_Ho^0=g- zwK*Le26@40Oj@L~&fpxX@lFVRSdg6CJL+b6wJ;nD3g#f>(5C+ul3Hr_hRvt{z#zay zzEAvWiwDR<tqFOcb;yk$AOqzf*f;59<21wt^A**U1)3~^zj~hj8fXQwbf$<>ehFJR ztXgi;4@#P_zIWz<3ejDtoKPD*y-x1rRTKTcW^)4wCIIPA3vszja#MtUSX!+0o`R35 zlIY`RLtaUv9k&#s_U4TvN9u_0uM@aqu4d=+Z0RG8rs?m#S|ho9brpnGfDj&A)i->S zE<NwbnPvJ>A2@|zD7@QXvvXeeH-(H$FkHIaM5WMk_vQ?RAF_m={^+7{>S%$#`S=NY zE-oHgbU!n>-hi)&!nz<j7uTQAM~zr(@U}HKHz=O|e7wV4L+2~6`-9}C*tB>|>UXa~ z4rdO96--ltG)e*ezb%_XAk@&WUh?xtPv9F3yTv{X8^X5<JOm(2=Kt_eeJK3G^2V<) zWwm@gb!M>7iYENjY&h2muXHH$>O$|MElV=BS~Ll@jkKu2*?2R*v^h1^$16EV1s>$y zoVnjK(@4qr?<F)J*&7WwWCeohLHM9vM>}4#%nr)T5g_JACWaQoabkG-;cWxXJ7oPs z57u;T>o4O_k*=Ui;82^HRwYde4)i50BgO)M-68koyu9G3=NrWxiVWaeU)C)FnlS)0 z8x~sq;RN>r=pLKRuAQ>-at%Vm8h{8lF<o&eKvQFntwfPVr-7+zhOL;lwkjO_u^-jE zCwb{^=inGOB&w^l8X0}{>qd~pdj^QyNKACX2NC6=t;WW#y%j)SBmtqSfUFpV&#p>0 zoFGE^FGQ3?w)?j4%Tm>V_@;XWB$i-KheF%uSg_;b|4*3!jLCHS*)}Z|l?#h%)Rsz~ zr;K%)?1-u{#id@nORqGtfj_)hQ}}74f8982cYgxv6M<g1h1hF+7M76Tx8zz)#t~5z zD7?sCX<8KzsJ?4-IMD4r4wu&3R=@D{<stq_#p~}WKS^T!&S&On)3Sv*@YW0#HFl!^ zWps7d{>hA#vbvfzeRVnRFkXJuLa8Z+FQ%Z*$57OhLGEs$-|_TR-mr-yag%!*M{GrA z8mwK@4m18m!^z+L0xP0gQEnIni<7~~QoVeFdE&M4nvb@(a#}1YnKsMu`SO6Fky+=r zwz+|{*JNWFdpaZ59<Y*n5%UOyp6{oK6e0J1C_0!)ejl~X3y;6nF75NB9L;p9j_wge z`Gl2{+L_-Jr-&x?m|ye#f|EyJY+D+*5#qm&w13sge8Gz5j>s_M?l}S~qLhF<Ald0Y zK3bj+R^XlR%XUaN<bxR4hF}u;{;A1u+K^AUkmczu8;H@CeR_)eLw%9B;*h+tgXE~> zJ|d$Krc=J45v9Nkly98i8Yi{IoMl0qo6j)|`XCAipy+U`0pxw>r7G+yswXo;!yr*P zok5%!tu)3t?zZ5z`FC0xV*!yosGSA_Lz%A5pPpdHi>QZp)bQMHoj^5Zhc_VOUFLJz zA0E!^*&&@$HBI)lz>mI~Ju}C$pwCTiH-VGC_jCq7K^`~XDt9(LGG<vmgNlOx?`c|U zj9^xYIWN%R#(lHdW`mPDj6|w*$ZRW!<3y#t*1T$)FH$syls1UcTW}ALk<m|P3JJ-| z&Mx-#0Ys|oVgqlNd(;C57k)JE(6wty2IK$wbMu=9AsSWGO>2om05HVP=1;hpR}Nix zXjpoOb2W>}q-`DGdM<YCp~Iw)B$H^$dcn3S<O=twOei<c?#hl<{7{^6DDu~oeWY5m zcxmX`E)a3N9&;)xG-RDJ5xR`xgb<dCXP|ni!&xo<Jf$-^S^xWY`@F+ZUPt6pp_8R^ z{tG#cNY#kM?Jq==1pVQ`iH;3d>N}zPOH;+=nAq)PuZ&mllF}il`#E5hpRcUBI+sX{ zCkaW33@&zO^;?~$y2|H?oZuFFYp<}|TN4J$xqqU5co&3h9{MB6_|I051_{IR6577h zb?}wg0w}DJI0BbyDNV_36?V4f!9*Ul5KwkA4tdnR#5Y=elIvYEfLhEO0wT|%?O=m` z@;jmfdAuBO0_t7a?|H=o)L_8m(Ze0jgsv9c751xZ`3}M}sv}Y`)orr!8=_-1(9Yzl zlep`&Kg+gP7LvB*awATl|9-0@dd3keLKIA_+rt=A<GqK(hlM1>bN+k%N4ROTOmlHj z(o&i4@9iV<_lvlP=l7bYd9>jw02rVj_>uJwRQx9`6aLphD_u6-qAVVIEbwTyW%{x5 z<XtiANN$JJ&`oHei+~{wKnqLIs>Eccuke<amgfcblIz@f@L(TG`QNi+`Lvtp_0qlo zD8(DTucfoFA*<{0mdnoGj!ovS_d=++P7}whah0dxaJK`L{8kZ=ayRoBqi&HZk9e}s zmnUI*w2}PHAaI!Mv9~P2@b9jm9qG|V@5|K=TStGZv+N=hPe=6<<QMe-F+?bvJK<;C ziT$-)C2n6VUH>!i<U<NVJN>XyA2wV^v3Y?i_&C#@kJ^l2LUN9y*iRakB4pm*-eS5s zUBfdnbPgxi;s1iMfo%e2>epC|;CK?yod4gk-2ZjzjUgrVvoaY*vLqdp32O!@Kj_!r z<ip?R1bI*!AtsWJ*SE!r>=4J>q5MIBK7vuVN6b6}3aHX6VSB=;ypXsv>u+VRq5}$e zrej=G=j}X_GLa0rdC9Xi<rY>ocZ5E3m}@4w_EQMPUXPK3t*VQXxZJ9|HmIP%kelax zK&5Kg;JipCTZD9oN;!M78c=l5x%yF*C!{PC&aM-{Xx(2?Zc5zBdAf@y`AO9*dp7TG z(Va8!B@A`e-P8tc5wUzDBlyU*6C7CNQI3~zRjJ`-ED|&89J%5(#F_p4ntEM;I47_K z<p0klAcoR6jb!|)DXv(K^T%~vjOyXeSEb)`$6;hCtE6rCVVaz9pJ0y?F5c4KK<@1_ z{%ycme5UIBGdd1-+BA#seZuR5|MNFNi0j<WAB{JzIrZ{i*w?$fa|?`oFQ0^G@eDV2 zSPPULckVAoXkYfLbEm5ZZn~d=kj=}B6bC>5RGo6uk3C>pmW>mA=m}~1X3Bcf(^0gC z|4V3ENu?S1)DxIR)0S#rFBlh&$3{=thZIP^CMB>BkRcIl(!!J@up!i;=iw(5dOFPv z$z+!l7azs@KM<nE>Y<@Wx!k}(`~DfV0Otbt{iXZeWX2!WvvlWCdGo}1Gj6q(!U*uE z1cA1HD~39f0VmO~aScn5Ev8(jUGls3A40|}b)B6B^L9<+=|zNG-@dz45owNjt_ZJE zjfW);rNXUPw*u`?SOHHoPvCySS_6dbo6EffpdySulVy*{%SM-EgH_}`L#JVWkg`m{ zEqQr5OG(s9ntB;s-o66Acc_(s3&&Z`hw%H(xivAtVzfUFPje)s!!%MHC#o*rl@pvx zs{19bGMBcWw!3V|(SdvxkoiX-CLzrtm20jDTa&k@i2rg)Ch~X#J6kn5Q&noQ$9CH6 z7Hv`G+$mcfo1C5PSbF}oTle<Q0q27|SE=Oh9#1lrYNyT5MB<i9@CD7%CpmaGKKO4w zGQ7UNV!eXGjgc1QanDyrS4AgRE)vBuKxYw9GwA&HMSvda{);UCfeZFXf1mpdADJf8 z<FD<@S0PHN&IG7WTG4*&g6S<%lhaX*pH$Pun1R_71LYTd*e7kpejTKjc)QU_uhQE| z=3`Q#{QVSGD+Hs7(7eZhdv5@(ST!J@GT|SgPuujKV335pK#QAkq-#5*PP_))0VADf z&%hSj?R_gZnmynDF?OacvCf6XWyCg`X)Wy+_Nw!Z09}NRpHS@W?Gwk(sJn%Gma3~A zrq{jE%ks)XM`IBvc;PNKyhXa7E=@B42i=AJW)TQ>(5#td_*ARKVtMe9rwB^*-KHnt z!cD3Kx2S$(hzO4Da><{!OC3i%J3EVZYE0MY)|BmV?%PA+Dkd|mCea)ltfRpt!f>Rn zPV}nGmM@HDM4dVBWj-S%p3ZU!%ec7#qRa<AKJDJyjj)+v>Y%m$0`Wigo}*2*($`h* zt=Z<Cv<fv$p;IPsREvs}4JPnr`mc27tt|_C+(do}ViZ<yFc@R*j&<*ri0YCqSdPn_ zeJArDQIZNDiT57cRRqnbW_BgUkO&f{>9tD9N(wl5i_aUw|20X+J`z}Saz<Rx6;t~% z;VACU#JEI>dFQp5`q|h--wv)N{B&X75NyFJWESTK#?F2_f;;`ehsho!(9BGWMQit< zps?%;@3Qf?yFsw&>Kd|aTsY#fJXg*CtdIY94JA<hf(cIpZ|lR|_HiK0T(w#hBaodL z2S*;1=)UB#@4+krtc-vx8ht6uPMEua92l1cNn)&3VjJkbsC*(~!#L*9-2`&?sj9)& zm2-yw3n6M_%Uo@IyF~#gbTb_msRgXh*a_YPKSt%dU$^eEOfjNZf4U}?Tc4CQ<was~ z+u-6JPWeA2dp*IYZ$&_e;W9P=gNm}9FSKxTvwwil!?RqugKaTqHTUrkRAFvU7AS0a zC^8sIsO*3J`pP+!N3_RUUh;J5DUO?hdygq3YO-Gdp<Usq_dVnX!76Mk%<FSH=SC8j z`w5w0p3V$knbiePXBf>==c?qki8FvX%_~Tv+_*QLRPvu36nG0mQ$icrVOQxo779;4 zZEBh;bjcz>T_9ThIv^z79^P&NlDkOM;}sGl_wE@oBNm{5XY4e)JhXf=%1mJumwO?q zn}7ecaBmQ|vm@`NKv0$U)6vl$bopZPOqQ+!aq5>x-KC2M{z}3xRsBDg!jK=Czu`(@ zBerVQyx#;EtdUs=U%q=bZ-^lqN?LoOKy$Ajk8xmwvy}(Usc6k<Zhc+FNBevBKOt++ zPlLPZ_H|_b|EAMm{~4lzckpjpu1`zk09k}kf7|Vw#YJb%u5>ErRJWPoo$<u2`LD$< zK)>ACsUZ7ZX;;aV;La^(B~GS7W#b|N45FkZh7Lvub;F{L4e){4(-3G7DzF@iAU{YS z_EE}(Qak;9`un5znD4tU5uz#(i&7J*>mx+u6H^LTf98FD?NGc6hlAsu{WP__+}xPM zwc4bHh(AvQ8T|OcNI4z`<rwo>y$6T~ZZz%lRe0wI&+2?D{lUJpV~c!Cn+Y9!6RYs` z&o9sNpAK7S^+Uy9ypI3GC}y?xL<_cid(5mLr<Ju9rGRX>kw_13P*Jh*iNvbkCEe=< z2M7E75)Y?QSJoO}sp1@fh2%!TE$o*JPAJiWMkj8kI2qH593==|#QSr*P7pjRvK+-) zG;?^$O2zp>*(e=-aK7^gh$?5ERcn0v;V_~{3ZJ{nP3vHX*LX}-f}EOj$VF<n?*!HW zRP*J>@tK!1KZCbp=l9|U_xw!RHYDC-wmSOcl0sYj@}ezS;W4Wz<vRfrufwL0vYZ+X z`8?|^4z$Iea8Z1B>ga+K{|{eZ9Tw%*wGV=TFbF7;QX)!7NY{Y0(l8)h(%qd(3R2SD zE!{aFDGfu{fONyq{d;)N`M%%#p7%TFKjyl)o_S{Nwbov1-S@qBque9xk$`G_PLrxn zq|AsWlSb;)Z?5VK8q(|EagLyXp9`ltgjGlBw~-Z+{+O9y$0-~ahG0dlsI3Th&c|;# z%{au7eeDAw`#wP9_yE`vurI?O_D$Qt0|`fAJ;hhu8?YQJxNxp~D`dj=5dWg^)b6=V zf4Jq;-eu0Q)Mp`$7#|_eCoK7L5718Dx2BU5UnxBVmS)RRqof@z?}{!D9L1^uGqEyt zJ}#QYC+g@tKEalMuj+R>TNcE|+s*=<zJ7$%V7o5BQ_r7u{YAL)u{8JN%m+sZ1kcQ{ zi#-`x3O>p07fj5MNl~IbB~>T}g$iQH%rnF0VO_=Q&(>5Lz!yL%*WxU+p{2!`S>N$^ zHdKBK$=U5I2uVDa8I)_94ZB1#UsR*ACT1@@l3{Mrm_!)@kx}Mm<v71+rITi6VF$M= zd>QKF-!PORx`+|TN^Fy?9^my=PN#89y;!Y}c^F|!k)m(Oo{l(PAJoUu#r^5VO}1R@ zz;^Q>bTBmkTI)`2DV0}TWAnq37GUND0wTYNG&ReD6}1g%EuSvZWcwo4XyONGfLS8{ zu$2m6t9x@#GZ2X;of}#b$(6!$XJkwoNjRYVa&m0XY4aGPX8UVsk$ug9-<JmXdX$e4 z7_hR$0R&hdKeyS~?Xs^Iw)A^U>E{6^B()<#5RDj_&q72@CXPxNUdx;CwKeQ*se-7I zv+|$^ILsSnuZ4L{Sbn1{Br5R#@$G=1Ym8dtHPojq1p^liVzcICVB_D|@V)vKk;m)p z@kgYOTFT4#<-kw1p2w(;>vZW#IRmMZFX^K7r@$!^TBMGN2e`LR#FCm3Br^GQui3RM zGW8iaH+L!EnQntJm36}_-VL`I*VVWL{)z?lp#ga#3?h8s7ySQkzYk@DQox=dGAhGc zrQw9`N|Sdc5}^a(qXR2aN-&*YzC#uY56~th|8Py<|K09Ge=!eu#-o!O#z{SUsY<13 z6>&Tcp)v4C^^&fmd*r}pA@>5%A0}(QH(7wJ7qTzkGu{`tyGEEg$3AUm3?*3OHyMYz zZr;#r8!h0+P)XabMz6l>)Vg4YR(ZXChN!nqPi>M+E^x%(UbBD-AS?Bo_EFv`?vsg8 zB=-KzFB#Ic@v;p|ea0uRn}dTbi(^CTcrozAF|-yoG6iPm2UorNJ{<c}*QWT7ce@Xo zdt+dD(F02?Z#`>A)^#J@euA&GDK*|4^KqN>l7>dtCh5=E0&GA|%zFVi(Wn)YZw>u< zNwbg8k0V0)-y&+3rtFQ}<_)f5CS@FNi5#Zi68RYs8q-#AMrWGq9L)GnL}c{ive5pK zH33h$H_Lsz{z`45+<No_k}`!W0o)}y7a%O5GHb$?@<HG|b!B_gUIn9M>!SMKCS8zk zp4;n2x`MZ#B^xDOHYwfXeA+DkTq_`rxEXA$5VGV;-7i*)IWu4#)6~(#r*sWW-*Y)t z_XF~tkeTSROtDtjvjkN?oE6ugtr|`H(sz^ZR2CfTJ)@Qwz$((3{)gY+zwQ5cfjgS7 z{E!P_v)Q+dAE)fv8a4)2@h=L$Y^u+grgqKS1l9n4@^@h~0fbGxZLekD3%DW|NeWp_ z!E|d4HL~w)^#6~{p5b52y7F<rDn0d>)NZ8|H;6gGhuCx;)siX4`fsXBHDdlM9ZQ+% zR(ExqNx>vDB>F%BA^jj^*aEn4JJeF;@7^2THwr5%dL&WU>S5SQ;$1v{aB@#`>xb(4 zU`|cgMO>^;TlJp{10)CX`|vz30A5SD*>c-)YHswo@=08jc?ODOsPjSk@h;}A0a3?o z-^KTTG&-*NLe)3+(aJA*?A&Z+R@5NMvNCX-_Jp4T++e8T%2KS*Ul^;;evEBI%&@sr zX<TJ6^ylv1dHB9A?lzsXoC6{~?|4GAcfTj|vdt`U*>u+O%nKD%@|T^UO%8C@r@%%~ z+Kz-6Hu|Z1Sh4p)-J@Bzp)Z%CnnmbYa}GNvmsyNJ(eC@!Mb1AH9q<q^s{;ayXp9C( z52)~JLKck0Nw#SjAf-d#O3V^y>fiA|U6T^9MF*(zEP+Yc0Mo!|$2F02_CHx3cj8&H zT4L^|{|EAOKhrTwt-ntzyOVl?BU3xHG-I9e1~Rk*Z%UvY1LDF^#aUH0Fk_t@2Z}P^ zmd4CiRFD+}a7$pV{x_8hY27d2ftEY6FVbh=z^aw#Gat9yhKb#(2SokmLzx``E+mP> zW32mEvM~1ldr#O!V*$!0u2cJ)zkfsUm>ydQOlc}o%1SJpXU3XO-KF`RKe*n3<(46J z(9_oSDl(8S@*n&{Q1<}aS77ps#26M!-Qdh;BFmVCXlWisVs?SrTXjEZHps9m5F3q< zFvr&r%n1ERaU^v_I`!b{!fn^6Dr1pLkwQ&pLsv~zHT~E0hpn`&<;c|7AI23SuhzPG zFeR#72m1Sa&thHyE}ZAAYjb0#X5jgVNb`S63m~$9jS&1L4HGEm#%53D0~uSivuj2O z+`fB<qvc-JaA0q&T%*!(bdR>g57E{z1+R^B-7-`CLzt$vF81`Z@T?1F@BJEd%+q)$ z!q~zY+}kK=>gF}$P$|<RZ#4CasH3_#EO=%{BK%TS@j?*FEn!bOb)4K#3vf>rypyad zjYWLP&u~magFdoBpv-TWE#d{)uwY|66S9ss<4Us{Vve5UMjzoi63tKNuZ8s7_$}OI zIrj3pOr5m7Z&~v-(egDbJm6bgE5Cjst{cum;gOFQOhZg9x@^cDv0cz|H%ocqX2S3i zjJmByV`Jk?9L%EW8~f<$eugJwl1E6jQFE?;31m2J>IX`u(R?jW_7Z3Ke&2z|1nGOd z=u6CH)`?>;m07u@^Nm$R`_J+uj%e3&wAPcUkO=17Gd;73hBU`CyaiH2Qb*}@>9m=b zM4D&(*4VSfj&l1Vg6nZf^=a5}yqL4fqso9=<Y@ieS(!n@cVD-$CPUPoNQch{Tdwgp z81}FonQOR*&H~B&@6r*DBhi`Ez++fcviq38L@&HeV|@4+No+g-*17T&Uz@mJAqeXC zm+*)NBJN*BPn+}ZLr|vgBRp<Z{WATgP%0l9WKh?*n1Y%y-2evBT*Yw9=Js}8Vgs+4 zmF4pGltn#S8BAbQ_@QNe^lY+>D<cm9W8>Q>)@|5@HO$h<bbu$i^p<IlHoGR$2neq9 zjA<dZ_QW_OuZP91^xhTyqe3L6p}$t*odc~5lJn3LCV#!XA_otun}=6eM_Pa7qZ#jn zScSL>7;_lNt6O5OvL~dqy9FE>LDW><ju*Obk?&`>C3sc3lPf~?L=BbYcvfo*m!)rn zSG9!9nPQ3@++H_K3^b;(B{$kP`AYJJ&^G@1u44%G6n_Zx021x?B1Wbb2JP0x))j$> zS-T#_$*=ranimU60VMgh`C%~srP^<?N_NaPibGpy&kENqy}<iz+@+NT4YiUN3rKOH z;ywmuM}0@^0)ky<CT{y>criLKEC@X|ih<@3QjNq~z(5%M+tS=_`z}8cDrEu;^<9|_ zxkkli=P$=2?11ZdjZ`*1tD=B)E?&V;vA2!aq`!GbkGVNx!Z$h|5V6l1eahF!Bg~s< zetjKg8nqT{G!Vv-8X`1nq)U4F2x(RtRu)@eE@X_&tHFurQ*abi6`a~RR}P>{dqvPS z5N4P}oV{0WwUxA?Y&ddhNR&D<_H$(jDzW7z!Bw}A#@1B2ox08#T`3O5O5MlYU<2bI zEL?@=0}azS);eVu)Bn-OW(0}Sx0A0TwgS!Snk&cKRJFDmPsEflE()yOpiksT6m~PY zj&I>A|Bx3JTGRChwa|RGL;F9ms9t^V<&^^bQh~O;m!5v3qrm=nxh?B>7w3xZ<(p)2 z@ovs>aTFp8l9Ie{u?CWslG#<cFGFJI4uYR-|E8-KXtPLaR+oe^1Q%jc?>|uqGjkmD zHZWr9oeNHtf1e{rGtsTs;O}6k&X(_07VGxf-NeNxsLM!VVk*BPV^b(V;bND^sJs39 z)-qkjs{ln3odnCuu%&5|RO^o#Bwz+FVY#DyjV9|j4|K#E$Bdvv-Qc2-Di@g#p@;Y% z$=+G{hpi+Vr&=d*ObVmhF*_-)yB1szUAx)tsXAgW<XEjl8+#SUf`<BO?%slXn`F2k zOe6k&PaSA4-Fb<Q7$1H-FT9e=`(5$?jRuK+V<jL%gd7sOR%KYS5>x%AndicIpw8M% z|Ai}EnggSY^xJB@=;=hWPo~i`$)ZB5Lz||1C~nF2f&0_^6wiC|&J!e;9~#EzL`tgB zr*s5xBi=Z?2;7;aXFT_H<s~H^T#E2i(JG?$S4NY!gFLv3+pKr8dT&q7Ht!9V%BqTe zc{+`rz_n?*fKuuHrM~bAlm0A+J!WXl)MD`9+lxZsO|LaayNZL<@`7y}W)I;*M}=#C zX_${-Q(>sSiDiRjH4S@Lp+oY9&;<5Lsw#JQh4B`D+p3r5VA2}#!q@Mq!sNIt<!ufN z`R#9U)2kYLGUe3T7X=?-zrAmKamH`>e>EFBF&CvND2g<7Hc6>ClJHtbbmLT773e?j zA8k#~dA&tuR}s-Na$x0JVBY3#;P%)zryQ=I)?HZa%g&f$F5*jp52i``ZhV}{>$m+1 zc4SbS2e~s~Mrq!~jEp2=Oq-0fj1>T_V;3B#csam+G<DvALsIA}!7*!bHSjJxmb#CB zX2bg`?~k^o6leiN$O63RK%iqLr<|@Obu?c=E+VUHmL%bJc}9O_RK*&zBDVrgg?$6A z+Tw-Uza&`TDfcDR&Lfa;Wnblr;z7I_df~*b^v^2)5258sFpUpvST$-3(=Uibr<Ju8 zgq-&UUv}pWjx8*F$T56&d{XO9$HO6`wH>`lMc~gdzBX1V#QxPVDXYMB{5&~jr9hi& zVN$!_1}6xC)Q>V&Y2XO~v$Jvm;{?d_luw%rOOrkAT-G23E`qy!IC2KAvLRh)0na!= z8j0Z@*SwKul0GdOhLIceytNJPlK`W)Hi~<r6~1(tN0`e~IAHT_sjiqL6YVwOs|lX) zHlw&)9@Y}$sP&Dsy_~`pJG(w_X;~hep1ciWSB4{H0&U{=k>gYjTu1c&DySo^V>2AD z4nWvLJ=eT7*7na0**MuGv)HKDIKhPrrC(z_54lQ;>Kc98bVZfZE7Fw--j8rBMZ+Uy zWZJQTw$U&ao#jWu6t|{a0wdk=r-4H>5$z_Rxu|G)lB#H2dC}N4b~VB~EN$7&s5F`) zcj(B>qXo0KoYYo&2n#X0CN))Ot(L2{O&cIal;lFcpcnmc9s<+LF}U}QwdfHXS+Nl) zlXVT}D}b|3;Nezg6qY&Bxu`MLYo>l5&mp>Bal}-ojMz1GghlX`PvaEUMU(vPi3VYB zC&Yn*oWPS_an;_WUJ%USBFQz%G`#yws+TEgy19S*_~p7l`%1c<prb|E^vTZHPX-i_ z*R+PHr9R`fK-S(@*}@HmWn@x30uQDNdOtmtfp;{Hg~r%ATAuu)mK0VwgCmbmni$jC z>vXo?QyW&jEMCU?jIVO{UD6%1z_BX9GcY_w_-3T&hc(%@mc7hPuMNKk8ceC#T|5A) zzxCo=y#@=-_W6s&b|q~+62_)H^F4_hLG=r9Z3zO7qO08$)`2K1Wu}wSm1}NGqu}2s z|K0ck(;{tPco)dqkA#-jlQ$JZQiF{4HiB%|1SDxrQbG-7b+*zd<vP&H`x{Tk1~I)E ziqu%r->VW0tfkSDZ0bqWE!-+jpBC%8x*n|%V+nGS5&rZfrP=1I9Rg7w%2vD;!Z2wy z64FCULiC&+Y%u#%Vhb^9?v}Nr|18w*ysIPGEMzcjB>rpYQ0Pd<H7Zw6?Whktd7&h) z!+ce(ViI&(^GYNUDWLAOd_3Svq01Z_aiv>F!QbeAA*M{Bc6Iv(>&zgYcktK?yJouJ zH>c$C`T$Ex%3B=HCnb|*sY^q%zY#Bd{WrES3ipcAKmYRDO54aoHf(3}eSWAk7c+!s z7xX<*Yfg>S+{rf4bMia<=Zu89@&$pgb|=${){1ybCwWnS3ie2x&yPAU3|C(Vym?<b z92smD(z3|OU=Pbwbe0vu-jA9|n}-%KUyu)S(_3!+$_sNdw!;24w<xwP{iIUonV<DF zrtuYdnYGas8l$vDG%<`<#fBVtirVg+<Je<Q<c!UiT%-b;+8gVy-Aw1*#ZzDCl$CBC zCUHCuIh!j>&AZ?^=kJX;J9sK}F~PSWMdnM^l>2qFW;|d$2!VPAB+D$lg9jd-1}W|j z*oDz0Oy%$-BHAm|AudYnZ)&Nd3NQN#?3?D1zK-4InyLcx(%)W2vAOmAwb9P!1Lxmx zV}Sep4ob2X;&nLr&81BH8T(<GmycEV$yoOSIb`uZm;ztWwy#ohR=qUC@{AEpS2t0M z4Kn!+884vV3p^!cD63@pJ6x}UzP(Ci_NhmUReAvR3SXD{uTee}czj~FL#}ygUp$T` zg;j->Dll01>v#!hPo8aAx)-#w;Busoh0InGeXEc0diY#pDyAE5L1aUqzAJa)+eP;Q zUi!C=nd$qhXw1J5Z5XK%bTjM<pw-IO^*(Kk7<iwlGVmNOcf<e@_HVd6VWaz{yv=)} zljGy~=Xy*?gv;!4p}r>s#d_(fTW;nBHMNObm*xXC`#nlCC-ehVYocLpAIyTYR@PkD zq)cVgzl2$6?+?8RO_}mpy5(f8)aPUwWN-|PH^WKUJgm!e+(Vy5E&*L=h$w4q!EYRm zOiPSijCQju=PsHeks^Shr7L6e_+)(~m5%g~u@OPoY*<2$d7E^&+K`QmKbb4O;zct3 zRj*5EbJ{}we&Q<EvD?MGGj=8!BxZ<e6de|vm}(8l9aM(th<(5=-)a0RQ*Jawj7E#m zqd1l#u?h?nMl=wEmy+kT=a;mptkq3<9c6n|aGcwPZ^V1eY8NP(G17u|hu*g}SDV-a z(A4$!3P@irKdV<(B?gkX%80jiocOVl$@jP{1czZ3ZWq0kG8ciIP4P^Wq_$^IJLinp zU<W0FiZIigYTnQ*4vmkECV`N!OcThVkqMZ?+1`b5iD;xB_xg4cxmX<q$+ommHBl(l zdE$Oms@^_=@H|%7eqZTIk}L}@y)l(W88$QOI<Rb+LCd?mP+KCNNXpe)YA$mS43ZU@ zMw-%>CPcy@qe06D4@MSM_X5ICI*_k3Xq`QAv!0O@L7+Q&Ebx&uvl9El+N<fJS+ylE zbl`Z(t#GeE@q}dIVc%@6b1HA)W>Mzc>6;T-sH!s4%O4dky(ThC`AY9v$Gv`P>r;8_ z&W?}$&ZaM-r)y%3e{R2T)2zRF!^PEplhxGjR(61e^fLvRmHgdEE{@m2$~D$ly0ull zpQNhc=EVMIYw=td=$5IiR?))qC64wSy>3;&U%d9;JF`ciPnR9*tK>?fS{*5e_tVPl zpC{i!LnFO)SyBYtKf7IBUN}4WoTH;cbZsyk9i5Y&pU8Hdl9Ke4%PcuiAHA9HuQM63 zqH904n}iL*LXWYoy<wg;=E)x~U3nwR!ew9W<Q;vG*!(nV%f^@I)m^h~rUPTQ7>JOI z`z|+H-$CuLHSX0A;n08WHO4tLUR6nt!(@A_EyZITJ0b4GaP$N`*er8vxDY)WFG`2~ z)%;EU+sN-*PhAFR(W~L1->{5KcQ{3~Q>FL|5eR<&=&PfdP*n=H0w$qph8Veh#JX!a zp_U9-L`6|HA*YhqcV8Pav~Mzp<6uSxZ*KKYOK<(86I}E)Q^m3T6@kQ~E$dPbm6$H% zhVQ}V(}VU69k483LjK?G+Ma3RQfBYdH)aS|6n3ii+#Q&=NlGn=RQL2p@^s6qnR&32 zpI&&9c_p<}o42I6CZw*+FFQVp8aw)2T9PbsLpt=P>V=P0p3ZN(Hfp%P2NpQS^+$Z# zB~<^^vmw&$^JJ25Y3T=JtkVkK<T=c=1IGmIZ6nZrrv9E5g5D{Ia6H4~#yk}n4tdZy zE*5EBzz6PW(L3~PN^ukew+G^@J?rgmE)30^>LqgAnQ_4NbYtEw>}2OO$Fa96tf2C_ z8IBjYl`ZkE{)DGYwUVC-BTJRUAG)$kXclX~479CQ*x%EyZuVT*^OuXkv1Y!jHP_6` zu@uCatTZ=W-}pff5AWDpLi99ga_9Dj=zDnmKuPqwzEy*Jim5Sfj`5CmSiUOhhC7K4 zycv+6e^<RkRu)m~61PY1%|p-__gK{XsD>PB;55ITaOjC$c(6(bac1S!{wJ>n?Y>3= z_kVeN;bimMh=U#6VID3q5peK^nQq{#(x_p)t9a`o0V^Wz?}oSwt-A00xmK%<539`9 zSf#0n??>#OF%QscDU+G6JEeFXPalkra~)XchiJ|5)3Wn_0sLG#j0MAz>KrFip{y%G zViPj+^aN*y;)Y{4yxc&iz*1+VWrI1!(@^h97PMJVmDQ3hG6Ej~^v)Zq{R<pUO4E_? z;3~o1I;bg0jX^-umkg9s`F`i^g(0Eplr!aSDxFC5Z9x=!KHMX0ve0=E!Y5&{lxTAY zzSY8ndC7WVLY)+)tDE$t>FHc53b{_Q%<T_^AlC>Ff2&toMr><Xh;8-XuDk?SS-lcR z?3~5{>(}~Ygw~#u*F5dAon%UrHVJ#V+B%aq#ett%ysxfnkCGS7;iF%tJRR6B<~>Ue zAD&!QXWKjQ-r_?0Vh2&Y!FaW4gRhfSC-X{P>!d>t(N)6Uo)<5HIc2QDzWd}Mvr=)| zDMl0=V05p>A2mxC&qL??E_|=lE$h<yzUFNj>>xXW1K;FUsx=lORt@HmTbN+x^FNk& z<+>?y8NM1TC$3G}F|4w}yA%Af9JnVCl>$D-M77~1uTqz<yFIm~l)RFgcvBcW$ocn+ z*RIS^s)VO)*g7kqtxT^|mnu*$DsIkJ%)JD-zB4E0Krd4dQlHNW-CB3o&3S5QCt1x# zQZ>^mqv!KtIVR>*ES0&zaSDxHIB)Y2hDZEr<_CEU$F$3wrF;pn)DNBW7fNGVtqRqu zQirZz<2s$wwtYGv*G*?jiJ8!OL?!zehx?`^!Zf(>QC0$(H1y4XAvS*&u5rAe?H>JC zf!}KI6Bjw6xb|_HpMs+&zKhZ^KH}Gq)0W^gtj&R~Tu&fnaR7<*zk!_Zz;OS|{>)RN z4uFmpGvpddD=lJ3KmL8Oy6U<&Rl(BxqyZLsI|_h@H4o87HX~zFq~b~!(#sAA*)wKr zZ*EW=PJcfO4v|Zb#Os4dXU-)hC9OL<Sh#(w&|sqlhKhL;szbtkH*|G7JpR$>@s^dT z56k@%;t!pVZg$%^8}gj5{PUZz-yG(Y1sAVcPZdHZxsqYmQ}Y%@`xKGBJB_{lM_t!b zy9pO;UdGl#kY@(d-)4&!mvhf*QWMB)qu3t=dlG+)&pom?dcUr9Ytaq5u)4i+8aO7Y z{gi8Zlbk?qO_vgZSf}^)<o?}vd>%SF^)2CYUEIiR=i>l-^GW|P_dJ18FfaD%&xzjo z;sjGy-LmohMpFR`gN!Y81s!U)A(QhvH-S$q%RJbV`d*=e))~Fp9|ILlYehloe~#Yo zA?y1F8ddNaFnQ0f-OYKd;nLg+X~?U7X$Kwq*ju)D$h5~sr?5*Y-5H)xl4(ZmB6*%t z0&P-$L^PNC2NZXYhuuSQ_XyK81(9a%MuSbSiDYj1G#+dKg!UFYRwse4bbzz=+$(+N zqhW|H?xf$QaM~66=wXyGEj@487vY~9@t4jYOo?^&-Gq+@hQr+3S)4xUeS1aN#W;J- zEkW>D4!Zy<F;E+_+plyRCXEBRa}o5;j<+P7jSL+Mqlc3zx$+y-$XMiY3VCLEYStuK zKb<YWF0#uo&OWoOXVI(CvS{RNTisiH+3T;D7+;%aM@T(4$p-T*wp1;`&DtHYnl1Tz z2nHx-%Vgc0+|ioNf-)3Nb}Ob?L5rn^bqS+Mko*SdmpmLx<vV8vyX=~Bf>f$)PXPA& zTRet~g$R!O!8G*491`0&AIrHk@V|ZywHX<aCGaMhyVDV!jVL*2ZpR9K%V06akY1Kx zUuj%Dng3CORg@s#oo@q1%9~Pn6X)zJZ>LjqSF?Ct=Mjsf>*))MX!qZ>Znx?EDT#@> zX$$dIk%ANFm1SkE2T1S=gt5K16a9Y!d%&t;p42pt=}DEw<yYke0sb*Ee7u-TeX~Yj zOjiny(pCzThslG7l3Cc<-U0VMDY%*U#%AF-1JC+*(C<E`0atq6i0T{4Oy>w@$=^e# zK5OpNEYQmjTZ@|rx4WV-q-C?f0STMTE?&1&XYr@dw(~L64%s}8jvEadr3N%eAMSCH z9*6_(Ptgg&>yNzU#~1<(DW#WwPSIhPid8NDxIiq!^=|23elmP9u`$zy>6W?Xf6X-4 zmY0MT#Z*7CwQ!ZA<5^h%h7z9=HBu{kkqQU?0-6a=|B0XhsAXy^<%<tM$3<7p875(r zur(l8YU<z=&)=t+Z&+n3VLrJLozY(=!(tNMT=h#dmL^=zAp8k*{Ix-`!>6xKQUCm& zy3M9_sQ^xEqFZPU(ZwD4!n`d^TQ?f0E|JKMoyFpB@=1hWOzB~f1F6a3z}05jQ<gfJ zd(-|gpt>jS+yjLb5-joXND<p{1$ORn>xKkUgZ)<BN*|%f_b;D(70uN8Ke!M7yVj-f zH;|+E;D^zd@^FLwp)YNdg3Y;z)K8d22YEX+4vFz>qF5yp2mQFs`Hymc)F|Da4ds|( z|4qD;5{WM77vexTQ6cN9=n@!-R^I4nNvNP;(HHmcvMap#z}F-zgZaT`;;Gz{JL--0 zIcNDxO`Oa2i7WCB&g=}4{~YE0TP6Z#$rR>P`@+s!DH{v8zx9Cow8>m!>J+85PyrA( z->KvKrWFW>!Ci{@F@e+SX#dZuQ#!GS{{SnZ_?jr`^#;_8tZ~4Y-<e43(WCE=8qY2M z%p9(v6-hYM^Jy!t*{x}S-W##zHJ!EC=^Jxjp(dQlt@zPVMPA1({CYQ+BBvSlI{_XZ zv0tW}$L9rDTFs8(Ry=j`=I-6Se15m3b?2gPzgHyB$hqa^-xE2W9sjV{->0C2u77s8 z%b=dGy9#=t|J&5YWU6o<fag8`#H{bhKmP;y-Z#%t%D|k`^;w<5(0P|`nR8QM>N6fi z`5c^aoblng(Vd1?obAWCwJ#wq%9Jsu1S3ByQrkqu|FTB3jejKQ`)!hx$b#4`+lQ6H zjNB@FmCL-1g2V(Mr>toFmi3GR8!ZlhG@xbMp#W}^V&~_VUM`#uL=07tp+iZCh|)*O z-&R`Y{ycp&?)04Kx#n^>cXw59pjV8njt+(M%#Egcqu2r73K^pF`>R*4Oif%wSHoSE zfE#e0cAuS`KuUWX7ZwCg_|bT@g5chcZxGL+as`~%ful8IT3}92SrL(ThFha!j+Lml zP~8pOs+ZZHjfY{dzI|lpCr!n95w-6L;iB_qS3Mi~oz4ceTLYvb#DN|{JjVDfPnWE_ znmXN^-LA}tdFzdpNvL;{x|-VexzV~GV`JO8USzB=___P*EX|A0KCKPge--_-%VSpo z4X&-)>U7l9cH1*qvPZsvGi2s?`(nPWjVeaAwD)Qgr%*FKAz{^BVj6==BLcnXhTyKt zE7p&_Zg6XVy#jg2^rRNK7g=A6D<9<8Y~o(QNhF$}r2>D4Lk`Y`U!OW3uR6Qee`{l7 zU@T3S2$S3V9kcDQv?X(CbobWI?r5a~V;|xrL^WT37i6{2BBxRU|9z90sedsq+@QC_ zym4GF<*<AT=Ji^8prv?~ot>jm9s2tJrZH)6u^dZb4}rc*e8zGNt3E_JIZ>KW6Ar6) z&d<*CF^V&q@c%MQdXfh~YrxLUZC7UhBju{j;lqFKPk%{$CAH>1d_5%R@~z4e33N(R z=*N)I8h_#F+5r1IlWliiL(63Vn%&7k#<%`ABy5}4esa~-wNd*=NWHy-(`5q&{oz&L zq%lK+RvvpV?^9D#M@J<|Fl&n|bcDR`{q<|sCb49ma&C5GgZ3H0Gl-G{=ovmi!rsr6 ziGodHeBx&^9FC3zM}#TUhmXGE<Khb|ynTyLMk+y9u*uHN-Z6`Z9GMVPT+^v2Y7p%v zTR*4Pob)hFSZjV}BG+U0p@d_47KN~}m#JDkV)|Wh*jF}Hi)&0uRmLhvWO%vlqh7A8 z>B2(u^A!KZA(;V2dL13b>=}aF7XYd~Lp-+QL}fjkkQ%4+uH{xpNlCGy0$U)|nIGtI zR-J8+>FE--wn_V2kPp8h$&9jA&{A@LOT+T=MY8;^II7grB_B`!)b0pzF?%F}%bT;* zn6bpHw2;GsgzEP&or#wAH-f<HN->6Xx^V6}F);@NLuB|sFBhPb`@~(02rhgE7rTKd zsEIKvPd_0z{H_QabzN+=y5kgS(Zv;$l;q{%bC)lsS6Paq)sT|xQ%dc!1EWG^Acoc| zDzyo4VWQSAb|?Ob!#GXO4tX8#JIOp*35h)VMi{oCNc0j<21K*UQH>Pz;PkhepraFw zF(KDX<vuP>OIs`I*t67y7R`X_s=`YVFdTgK7$TxiZS9>I)^;j+vNtv{9#vZ^INxyP zvuy{8;C4fR9u~37Gb6F`HVRNr8k2(}7%xVg0AAU@)9kwTF@Ws~;MoBV>eKes-`0-7 zLpZek#OOv*To3v!S1GtOG`HnzDmHSa!uz5MX!3oT;U@KD;0?*l2Jc_0{fwPIQzSrx zvevaRF^kZ|qJX$sKN?^zPEpt6AlgaWTCIm$Dg>~$d$m!PJj&Wi(M%h$7Put^LM#H% zGkx5$iuP@%rk=aFYXpvjPfphQ=wacQG)((EaBx)rwJa>P`6@rRCd?aM^j9BqcG0Bu zNPD-O3ufsNVX3^bBS&0)qcV~bF|nEl{0fJhN=|N{V>VJ}oP)#R@|x!IuoW-(Ye-&a z?J(+U_FHeh#)i#~%_9fV)uOm?3NQ}ncbJK&?I(BJ4|H0Mt`e(7Z1bLr?<TG0a^gm8 z8yh@Bnvgr3?9Z}?S}tR6ZvpuOa5dH}F3rX|g{b=}#S5LgcZ1ACEw<_7t|9E&G_OV? zm-BjN&v>*N+<c#>XkIkydhhML)PW-LI!If3U)6>rt!qS7Hg2;z_@N-updO*O=ab$M z;qapv%MBYr+qLHs3;Z}BqcRmW<?8RH2jJ^a1T`H^jBIMNXqx8{$^|;A?J?yk%5a0r zDjE;JZEoKdS-tHQ6OOE^X4ODJRx!;SUwVF22<gpoW$EV6m~Ewx+HGVL2VBlL)-&Gc z+!u6dC$8kJ2lxjws%s?0g-@xGkbICnhzcoT<5`qg?QNX{XCmh*k?UWIL^py4G-fy2 zs3jWg_o;Xq-G=K!;^?$iOAo<Q#SfZ3NK5DX2Feb5UqSCepJ>zAd1dGWcfAs7HQR=y zHZ(-)yrUHB>{6JWHGNqgK4N{ir(cGY*|}H5e|&J7ll6L#<%d{Gcv#198AcdM<Js`% zwvknHu7ZjNqscfdUIR;(x~i)FD`zLCL&|K5Q?j)l^(F}<*L*>RmJBciKKwpTxgN(R z%v9K$-yvDMaQ#~Ya6*F+PCg-01oH1QG6C_`XEB90>_e1zD2~bF?o-242Gz#l4#0kk z=PRV#y|Z>Y!VEC5{)2D{l-UOlMepkIx#dsZJse;hNDzDQ`!0To9^fq7saKd-I4P0U z);rsISbeLQfzf6ocpSsFLrC!dvjCKL`lA}WCwDQh<llZ9{5BkQ-KaE|B4Hn5o`-mV zu>)`}01kE2M0yF@m@H6s?>q(lo*5Yw>Asr}4Ce`cK$bhe$jAU;`mJA(nfaYbyG8bH z+u8eNV0YM#KDU#H$CoVCZ)g7S2pl}T$YX0W_z?6uw$R+n)j`YM9bF`jw06%%Y4ZI1 zrB<CosKk8MQ)e#Bi_iHn#^l-#R{+oroS)9bWHnV$fy4XC^X7W9b!pim=u{-|S#K{@ z4D&|3C~0+OVX$l}kAwg{PI)Ze%o<y7X&Ls=azPXp_zdUw<GqOqrv|qpa!=?!kz9Gb zhX1LdvvW8iivd)1ce_cbq+}6fBC^h?OP<LMy5Jrh`I0}W06oJ2*&Sne)K(M*3?@V0 zXDs1q-chR`OcmHa*9)RmdB^B89~?}-7yqkwFxUHLK8==+!xw%tPk+w*!sNjhk&WA| z?qqZ%HQcR(khs$C%aGTFg@s1mqUiN+pWt46F;!9UO0VQ+;Z_1;$srJ@S;l**Ahbiv zi%u?g?wQI`Yo9FZgRtag8&?{HeJ$oibV^iT`rgQL-m!d4k<ZmZsoAP`I9UqFFbg3V zU1`GA=Dc*IxmRe1a^>XUG*TY5PDG!XDs((5a`FS0Do6<B!><B|#bE_Ar6Hu2+w-p> z^di?vu5&F#vvnsYC-~Xdr(yKYIAq`t<+ipSP$V9ovQz;8IS9?vjBl$gL`KWJ6&9uG zRXPrIq4n;hL|$RbHGN3i<aNXu_gZFPz(J<@soqi~C>aUW8s0o40J<Zvl$k*D#Z9FP zW1yu~I$5Ifl2v6mD5xau?1ZW(Ob|ufCIr6r_X|)^Q6coKJwu{-`O20oL`WuH9p8tH z1$TebaXicv;2SwTK||=iLKVD2T4)(@H=I$ai6p)rM?#aS{kq#aX5Y#ta$)MSUwf+A zF~KGxz)F<NA^nKSh(=?z63O=*IxX;NmPJG$6KrImpRi9|qvb1q_rhDIcDhjSMiAh3 zDJ`f7vxNlYfTgiVypeMRl%`_kqU@g<&2m+g96oj%#}%QIDZD9X`DK}q_HkTjBH}+e ze0~*De=?x~_V4Gm!|AtnJ&hlD$Tr!VS*ne$5Zi_Tx}1@2r$drT{;AF&1ehG1K0>1! z8`-!PHqg)(%0LEj)*kHBqG+gV5N~8ul1F;skPCdG5NUh_v#ob`QFL{Ect^3BUU_v? z3Vn?uXU*%egE~GofvB0A^U&U27c;<LC-xF?c+R_Ab~<hq)5evX-<%bdC6&tW_=1{9 zR3KdhZ{!t|75Q<`$X9B%!$X|-)$_3g59NidC>k(zREzwD+uEEMP}Xordr!`{o2EmK zsxHCIr9NrYysXPlo{U#E4Wz_3D%YL)cRmcr`$bpqoI_P!N%|ZWd9JY(aWa;GwUZZw z(cut#(5Ss?O;|dg>Hb)br-6fxC)DeX!h6|tk&?f(TP~UQ6=xWA3CsLq3i@M)Z9o9H z(HokCPng!aXv2kd!qI|`eA?XP=^6~i&|O(v(!UpqAdV5MvQE<VfrAXuq^qkj#}{1g zM3^L0l%qlNx1ktZ*8%y1U_tWq-mR?|)>T(s+TCeq1Da8nTFe%#jX>6?FgEHLjD32< zBbWj?pKHM0f_INX$uy6vCV9Ga0QBL*!0%BZe9*HV2_)s^hA%lehC64p)LUEIuDpt^ zw#!wu@x4TmsWc!eXZ!3^P-$F<9g?BG#Sdgqroinoyd~2!P&T45kFcErl<RXx<$Wlt zqq1tGuTQ;>gCM*OU-x_OHs&vvtjT@T|Ks^;ANloM#OcZyyF~KtbYmKDLxqTi3~wu? z-^~#g=Eu)%+RC*J`3P`opF<Ou3zWavgQ$!8=fRP+S2~e$?vnZ<x{Ec^0rKs^9wjUO zK3U5%g|&>`mC}WAgrrDERl#sVIEMHRR88a#-vg7yy{J3bQg;h1(aZ^EE4uAEYgs9- z2JJsUL&;#Cc1W6fEPe5&_wD&k)1?5*EK`k)#@aL;XtfP=ReAwL3rAIfJOSl>h~zTb zYu#|~a~iV0hHV@Nxt&R$sby|nm;#7L#`T0INg8==T{zA1ZLB|W;Z_l4sW4%$=|FaZ zs}IcCF`|6m@b#1bzs>mn1eoI|{{I#(^Pk%;fFjN*{d&-1d{~_V^mqVN$lJ-eZ7PzW z2L&_;;Pifd7C^s^NhGQy`A$Cna+g!C^BEDo@EHZaCLr`?M?DuoB|Vi;!pua{5p7u+ z9DSNMsAy?P7MFv>7F5-$?uP5t-e3r@Y8D-TIoWF<yN*u^zRS>CGH<2yK!d>a=6QeU zpY*pUO<F7uK3j#T!ZA{=Ht<_#tZMh>mVx7i(sg;A4>3!>`t=m9#eTdz4fIt~|JWvv zw~k`daqCmss8y%q(<TQ!4*;|<7qz*$Mu@|s*WIsW`A^{WQYssxG3L)td}Y0~`6MsM z79|~|`EM?9yjM5yF!t#(xsmOby1B_0(Yl2WwB)QQ!dOs|rgnvIAN)o>C2d;-l)G0i zkmxfl0H+nbdQ84-`mS7Xf(CW*r<-e64L<KmyN+myPn-D3x5s4Qw%ZU+<S-FGQXkHV z+nHTfFJ^u4@VG9n{4-x#&D46fd|sOsZaGhkAZ!W?g$sX<qi+htUDsl%6x<%)_XHhQ z^7ku(F&jMhk$r8K1qKF+9X7XQ&_0QK!$Vou7;{ksJlB3=5_L3&nkwUMl8_$?$3)_6 zZ2YPO;f@dvPb>`DJd?+}lOS>FMn?6%I@H<FxrM^z^=uzaswj+<U-S8JpSIryO<j*$ z3$7jYa&=4`x3p4jA)!kckIQ~tTmM!grw@6mRqvz~qchDPv%w|!JRPH&%vZMTqu$RV z172tH$o}TBnF-l6k5i7A=vaGjsPmJPA0cjmcw%OkbM>A%*AH7!;er75_F>n1XQVf< z-A}V@=fnv7o0J0%-~(S74~lLZ{^WV>v{Shyaa<?_RtWYNqPM@tK|5-UL7EEW`8D*n zmJM1KR{^}%s4&}E1)j#bGSP55D+P7^C)hHe1k1qiZ_;5A=ATiXRj<P~#^R?1NQoEb zM+y`KCg{$P$Ir$``{#Cg`7DDl7u37~qVB(|F8;Df04gE3k2~*Occ<A6@jx|-COLwk z{qH-{tup%i2AEy2&@`sQ%Lrn8v!0fsOpNFS5j8ko?e$(#O`&IP+P>kPMUo%XX_8J! zNm0_&{E6plX!xuNOvmG&hbJQ~xpb7_(Rr--I6Tcp;>^k_kg9Zen&^ocs&7C5-wP0v zR{bDE=@X{3v^4=Pe#Liaj|P$Y*g@K02~(;ao`OiwM;`mVClY9xS$6xk)W9{3HBWd@ zSzOi^P|zywy&b|5<ctlWBgri$TOYiIdSB99J?BLRxi<E|SjC&ZW!zmB!Ap^S1zK)< z_p`W#Q6W3Ly!AhPgp!9~eMv7~=-5dd22%P6ohj?$TDt!z4}!lXV?wG4jKgX7UZJj8 zp0z!3$8LQ>e)t&l0AsJ4J#Jut0T&Hcqb-tM<fcvWWueIx9W6vsQsqTmV`F1||GQ)Q zO#ko{{=R+o8sfhBam%qsC&hcYqjjF^0#FoM^fIsQo51jTX*F!L@>kk+X1tS9jas?7 zxB^D~8qrumENBj%hX|T=Vgv!%3^0&!4HIIHU%SYb>=UGn@Z{%Y`pn;aqa-kC*T9`C z9YYAcTh=Ma_~m9k;7L3B2<<J#qcWUb{A3geDYYGe<#UjHD^kAT-HkoKkzQR7rPYYO z$Y*A$E)9%h2HFdW7)O2no1Jj<@Q6nGeU^iQx@^DM5BX4ZG<3$(&%{?<HuFgwqD<)^ z4zKV;E5g}S+gZn7ER0zz#WA~~dX^uX+P{}hR64^o0(zW&X)q-AN=2h@Bz^a&E@`t? zfRh+Mc{^_v5@3ExYnwTiD+&}FYlapdt-J5u&c@g;i4<-yu3`YZ1JY56yNP>V_@p01 zH-QUV_5_9?J<+~1Fv;3e0<c%Qdws_`pb)wj_h1;t{W(em)DsVYW7}~~7##EFCf+2* zYcvwfjBeAI5nE;l6;hHUk*=aeF#-~m9VQkG`M@wE31hCy^*Fel!%Q*X*`{3Db(GiL zVH(}7@(brelD_w2XGkcjz95*^C&_+lpjGz1qqr02vVyO3^soT8e1xO|4Pz2kineOf zBS_=}qA96HTfU#o#b~~D-ao+SH#k?DjTEON#f>;Fy?x<#*IYB+D=*t=wIvLc-ATG@ zYpQKgkr8BgwyJ%YHh7RHzlrtS6snd+@PcVY-*p!jl0+3vdkT{$d=V0L(+z&-(M<_0 zBUp=r@3?L^KArpgu)Q%MI#HbBZawcgT}V|K8LYA8Ti6mi5?ZNE-T`f1sOdTjSWojO zgTPQAB@hsU_M(zhlGwN3ti@Kkfkv9UlnUnT#z=%(N0#mFb;u~lR)r<+T<-$T^Y*As z+ftgL_#)j#+45mY2OB$-pOFQD3&W)lbPQ93vDK5SX)(3dDttV;EUw*G(DqvDZ7?Nz z@8c+ejE>w+B@h=v8uym=$hi@Cz?PXb@>kZU@I11V6#ORyLa72Rngo<^`81)R;nu4m zbHl=Aq?igjyUoRssB3BYJY<a3wP!>N$jTEh0SI>gRs=ZZi5ZTo5zPcp@>BqrOfE1? zh{I)1RcBO?C4Mr!sbC?g5r<sCWD1MC%*Xg^b>!P&WOJ1EWUYBRYPP^WTIs8p=)^&V z=Ad4QS&S<JLJem(?P%;S4m7ceXD_(RV=IsE?va<}Mb5|*zss4ruJj(aFKI8V@#bq{ z+{CJQ0G?qjZ-4C$F%8!%Rn0Cm-7<iV&o7k>pgXaXFRwRu-@MZp8BL<uO4-Ssnt31m z<8j7XhIpb{a_;2EcrvNKA2%+vF8(k>eHiNwk(GKC-l|@lGZ``lP)V1b39$=s8CC9L ziX@ctwd%@hBWty|`655c9g^(pe}QAIlNvk-xw*NRgpa(wJ5S~l9l7`x=G96sP3qTE zkRPv#=)(WD6zQ<LMs_)=<1H$eJkJf5_C==+8szywF)=+YlQ*cH9j7qDkLk^YCGY?@ z=p=-kPhhp-;n~g^e+1v2$WQIzlq;OgG#w9-yg{wsgBIubxm+UM95V`rtIr(vm-NFH zcDA-MqsQ2JZ7-k8H3#9S^QYx8Spii?SIVub$Ro1Ycyi-ftc#E!B@t0wygRfyY>bEb z==uIZ{oY=;9Cz5>oiomi#g^ArnNjDxCrM4Y)Vj|yMau$|B+Yo8Pf^g)kBH14zre)X z-Xs%n3Woi(zr^+H=>wYVWC`5svl0E*0rWw+h50{T;4oXKHax+_bk$T)GYzWn&~R{e zK9g5H$3lZ__(*7M_Z-6bSIs>|P@EUwZj0{-t>>2Lv;|Q!-n={Bhp2F!z8=HtkrI8Q z*UWoPnoBI;eu#;p<dH>9@phHQ1Lsa6;oz4b)POt4G@(59l{04k{#n(rG_S`o>J@u$ zI^P+EZ32qPIaQS{a{SwzlU@?vUW$EwhirLe&U9^It+niHJYS?WB_2#c34Eg7hd%GR zU(0RoHM{RDYiZ3tA8}heu)XDaQEfffBBc$quq!VzkGhH*LBmV%%;FD(M4}WpXe_#J zpY&Zr&p?i-6QZ(8j_5Ud6#9HmcPYlOf+l1X_Nj6c#k6$i1ULmUwr1=kKc28;K0dt0 zn>QbDqQ6!KaFw@2nyqfKqj$!Ze}%h?A7d3BXJlUjx-t?L6kh;pCHgI<gvK@{k_~tA z*LQ!_K%2|?MOz6SqIrW=xW$n5>*6a}OxmC67nphi9L6oYUM3(H-;Y2S8&A>D*5lS_ z0m^ViBFwcnU>s2Fy^Co!#4jPyS+?KjHL#EgVAUt|5mHu(i;ju(U4{g>(4vfxi>{!6 z$aJjs$Q_O*$oAf~JN=*vs4gr^eTL9=aft``G&~L1=AmSQj<LXuxt@*SOj}-6lxu$u zvgz_27`+ED`uLqw&hC=5symh2g5)SyxG3=vm;oLSL_6_~H+)3%1bB;(k?{O|WOw** zo^Q;^IEY>yvYX(L^Vp$tc;Kc)s~!!kCZMmEMoD79f@Zed^<|dG*(&Wf4&I7X(EwVR zJ>cNGmEG;3lsi@QQk@Id7ZpEsnzG~TP{Ue9we@%V0Ag^2&N~*M4YnuG(kJqI9Q2^) zyl3gr1ufL`x+5X=3Y}H-92Y_O6W>*+m#h5rh2Op~1r(SJ0R1kg!o{rRo3z!Lelg=U z8j)c?^^eL&`CEC2$E(O*UB~URTikLna8(G-0iJ^8sZ#i^Xit)dGSjDFKv2X1-R|KN zr3BmAthAxHq(vH&rD*Mx`l!W!C$_06Dhg!MYymFfSz%j^V*=f<tkR4McTg>!Wp&H_ znqDPFQ5AiLm<f++^lnA+bF_K;?gbrP!t8Mm{Pp0_l=VE=vq$ZsB=C~-1oxW25&!x* zY{l+nZY@rVfPL(w_ldG=h~o;bDH;Z<%e~0H4@IAyv3Dhrx4og^U%u;iz%L(5+V|r6 zX6=NO$v9$R%d%@1h{(n+jCJYSWv=HTr&yPfKePxJ6BPy}1%priM4<me-S;t2Y}{|8 zC{Y#QI<ukp5>;u>s|X!dkFc}yH7(mc-mDF+kry{-Z3Yg2NInIT#Q$`y)#3U_98gnJ zI~@?6NSaG$eExOIF$-v&=?dt=es6y=G1IftfN(V75ch(v(0yt58yXsc4aD8BK~mp6 zjGA%GQ+KSXe3=4@EN-~IRwA1AV}SxsaH9zgm+dwz%tG36odQ>Mp#@n<NwarE>~z#< z?yF5&d@9JCjJTQ?ili*nN6-E9p{3$ZS}ua*qLZSyP+RyV9&cv(0^Yl4ymz6NZ{8tg z-|(}<U!#c7zUI=&;i<4InP);m3VXb>yL?9tWig~gN>2*pLQ(M%hz>um`5-O3`W{EL z<poqiT!Ke<c%VH{TU7jzJ)w)^;h|S;b@fr=r;@!?06-dgM=-Q1f`@o?`!$~v!)q1> zlPG^w>Qc<;!ysC-l!Q0St9(m2>SCje|G2kj=F&~JJ>EIq2_Lff)7n?K-Idg*Kk&s0 zc5Y5bVsMZJN2`w-c@SR7f+JEvPd<@$IGz_qnrMB<2D>PVz6~;h-0;CITHWJEq3H;O zq%8W!tlpHx2pc>5v){&k6>B0Fdt$^vK*mKT>6@LH;0z9xqm+ZM<3A@5%E?*fx1>do zZ?ks2ereb8faLi27_kUiDT*r<6vS$}>biWaa5VLp63e^Wq3m=Ft&(OF)%O&@`u~-E zfK>Ds*P$7cQ1EaBorInxo;+j*RH7BBPu!FVrSvPQKhmw^@&v6qG<5OiLx_!ju|E8+ z5h?RU$lM^>>fh)F@_&A~pEh%`1UaIzB|>O(s3I?4xs5UJ&tUs}R`o8+XGmL%vvh6{ z0EPi~y12rgU@l`-pLwNo?3U_F@&oU=#vC{DD}DI*F2l;RFl2*w9sZlNH}h}wZ;NZu z@M`pv@ZyP%tR2~DYLG##lLu#89ufkXFeKLyS*z<SkzX1Zt|%B4ymE`~L0fIkpasD! z9nvEL$H=$g=ira7p`M|xjl6de1YvKTq`ghx*y4AZyHHgWbEiT1)3*Hx?=F6kTV$(J zCaH6>#ld1{Q=&e(Q;jhl3i$+GAVtD6#o?Vdq#n)zla1^p?r6kmC<xG<_-iVU*@FdW zwQ18G76W`+^(i(AI+A86)SH&IWt$c!l+{U-S};PTXqzf}B)?MTh*Ti#=J3tOb&6As zL!H%qicwu;bD-`~ZyWk!RqkQ_dyZ7kusHCS6u`79ek^4!nH^skHwK;%E0xK`)N%pK zxu&Q^&mHs@_RU?3s=_a^XJuhkbSym-Ue<i!(t;Qxb(}vK-a@U~jQG7AmuIF9y7nio z)A9c30RVWGJOlLfj}QOC%5bx?F=y@Qs;MH~x*9)bJ~%w<xaM{x-~o8sPf<iQVlw)t z$aD`p%>s89aMby3OwNa^Vy-XiJ-w~20%?Ce^{g;jP-iQiPcL2xEDsmS{y)h6|8tf^ z^SItKup$0O0#TXvP=3b^vjN5-g-?*@hU_^zE{yNAD(F~(Jc|koU4@LZ_4pr$xDdug zD=ONtmgWEztjWY-7-<_1X;NZAWo3wWyFh#}Cah0aoif{+@!wtXUksx^RPBH(Mk+Wq zbMVukAT7#S=gG=5Vv{72Wi$NBD{%f~9KyF93;Z~+?yw^1e%yDCp&Kc9$UUjU2`Uin z)-`iR_g}2U|2;=SgY4sohIj^&ugIFNjgx8Y3xJ+eL=&33_S&){x$+T9P~ZX#CPGfG zeOv?-mUly?w<%^@B<H_bR0IhM;Ei{<DNqOhQwrX~+m_zfIw={n2U}aDv@EoK0EUI& z`~mUpY#GvN5})R|;HL?${u4O8m+cEDQ2&9V0t_VbHeE|OY%vw6Ja*Y-4A1yFfpI}o zo}X2=X1c#&AB1@(7?7CnA^FBPFwt`ReTR(ZXH`a_n^xC4H=S`lLEw9RJ4qQ1XqA4d zN{DsQlxh<yA|m2}lWTZ3ig+?UF@bQCui+jioyqVOY3FY)ytm8_B)^uWjClKa=2Ku~ zwO;|mHZ4D?7=W<;dKLuW#P^i3aXl^Sl;frzK#$Tp5L=O;p8T@*cX#w;sthay#=xdE z{oTu!B?vwo!9d5D1vDp-%MilLoe=AS^75~Ire$T<z!pr){qo!8<z+!xS^Q%P!4Iw% z4>eth{0qd(-#3Ks+Xsh*W-W~<*c}~YKFu3akW}Z@3E}v2i@diN5J&eX!Ti#0(>;1e zEvAI70BE$DZ#Ox(Dp<fG;!>>OU#{sMoi*F*^AFH$LyZS@viHx((^bCyQYxPLM5Tv^ zA9r9T!?){+v1$#0u<i1>`0-Kn^)i^;9m1j$qKq3Du!fN24^HRNqa$Ogb1R?B4wszZ z{6Eg#GAgbv>lThe2o^#h5Ih8TcPh9A4-(wn-7CQf7Tnz>KycUK?oQ$E4n<M-@I2|h z{dV_z$2Yz)^5-O}ea=~X?X}mMbMAFsWY%7*+$Aps$bqx7bI&fUpOM;n0(9E>qBCOe z<Yd1?#e+31G`9D8lrJz5Ktz1DiOq`@;qSFYSw7B7t7&Ltu17n}IWGfWGqLjyOXC#+ zbP24$x0tG&{!=#wcs~tr!e2Q6RT?c)C4(l$@vW`zve`996{Ez^bM%5j$RpI}C~A*< zy5h_X9HQU7OQK|<Ny6*go)IvXmJ_FRb%QEMNzo927&*nmWK*bk-$xD`PrSrAKOu#; z5`Pa2$u#G%Qj|1%hZ-KD+LWdBH$C#ueB@7R8`b397Mnr-vgk6e@SC0g*V^#5g2yiz z&@=NB%VH=6)nl4`1lBG%SXoEm*OS)UOC4gLAtN7!Iy!(dXz1%Jq8k%*Jp@GI*Lq*y z_K82LZNB$1JKA1vi3r2^t(*H|0^U_kG8buLzEU$f%JlBDrI3k{QLLMrdqGZBsM{o@ zhXG$vQAG;qK-r>Lbj!!l^7a0@hJo@@sl0a*&)0r6F)W2wh$Lf$m-1&tk$KKyU^N*+ z=1Q{-zp#BsNJybMetK9K+OEoU^ELL8E<|8bbi;C23&Swbyc?zuDLJ%S>Ylk2iPlz8 z;i#%E*gM)=bFhJNIss@JFL8-Ffe*yy=onK&UEQ5Wx^Xb7{?_ew8^oX0KYeK1yrQb% zGa&#d+zjRgahFC$#Fv_fcTO;k`xq#*j|1b*uK)=r?}fFw=p-yskYH+LLO9I}#&?5= zpy*^ph`!|HWS{5Vu2=Hb`Tc(`Fj0RJpw$d;oK>BGotsZEwNvKvM2gfn^0;{)^H+hq zJLm-CXQ|-+>RE^prp_yWWPxrKU3nzW8e@>G(B$M|hvK(eKJ@gR9bBQj2uaKoGQw2; z1zY_A8&VvP-E~1I8h$3AQMy{#-aj~)Ui}PoeoA4O0??zsRJ@eIBb<z11x$%m)$>Vf zPsj{QCZ9v9q!>9Z9{EfbwdxEDzkSf8NjQHS1Z&tOoB&G6T%`ltquj()>cZWUg_(M4 z*Tku?Q76ZRTSc49Fa+Q4fgTd@dH6ketSzhCReCFRiT`_0=$5I7JM?x(NrCx61y!Qa zGU(Its0@gZ5gnKm^$r*0i{N!}pr`7BR&jA5`Id1iD>HZ3)3)SP*C=fr%Fz5v!ezZr z8oY)EMz+x+2A@$DYdyoqTL)f0P+p#L>e{m4+kT3Pu{p^)*XQRa6lsc|1S5AgDvnVe zk+<ABF%a<BCYGKJ<P<p^MPT~ytpjJxea(`v&u;YBv?G!lD7ZMeN2`*Y0e|jZ=nZm0 zo@!}68~J9n=q~O|#H&1g9=*WVR5yNAB6&3uRPk9+z5hVzljFNrZ~9C%YVCYVAkD)p z#_0!l%-sI>rQjQ^1CWPz`RC?V5THfr83z5h^3!d_FKGZ2C;aeDvD?i6pY{FIkQ&Wq z+knI*^-tDHKm-wH|1j~KtMJ|>clXU*DEBKxZPG$CsjT#&Ul&dTM!sGol}|4TIfa30 zXQ8^Fuz*ZOX}TpOET^ErbX=(V!+(0!KRRD6?UwyYkUypMlzgr^%UZ=)l}nqX5l<Yj zFh9XQ^{L-_Dh+izE_(2*08VbgBb>WSL{gl6a^q=Z6a-^q6!VvvJ|6%*l-uTtB?3ZG zk@@y+D9<@V%Pk%3Y<#$`BfqLDMSJeTckS_w+^BWuNJ7BqT35tE?tG_{4z=tG2;bAN zUfQCYorh!iDEIoOBkE01_!w7A3uXT&K-%59#NL>zlo0$F*9iE5!SP~HFBLuBr<&Pf zIy%LE9n9w{*K%KEF$NjvNAAXWCoiT))&?3TSe6XELa<O>tH*Jlj@-Kt>AZSXRbCpx z8=Smj)`wdk>ER~SP`vN9%Vo#lCO9aBx+ki&de5Xv!@`sF@SS<>ovB4ceAvkBV#mh; z5M?~@SrJ@4&kAFrn_EPAOHDq*t@TKtO1FaC@S|sog~^;>2VquKS1F1ZV!_JbFT@^C z6;<-XcGYiB=lK8-UXX2fCRoM#xY9gu?o8fND9$)__vj<H-R_eVPO#r__;xc0c41AU zbG!w7j*|g&vlZvCzy9U)^dhez9DnEBj|Syk#8qvzN@=8vnXIs5GOuSCzz)k#6Mh8r zq$4qV9KfgPfELX>JHueO_fWn*pJ(cLihH;Vx4RgHaaiP|ZDb@=me&3`H!aZo<YH8+ z>&=*7KgafAQ$VtefB*^XHVgFN*<7GoC?X|*@d6T#m<3+Xp8oDMKsE9EVB_?=<n1Zc z^L#NpVB7U0BUfGh_$a3F<c;9}b;}c=(_imJ0wT%lg&CsLk+ukYLXc96WATn^xe6+J zc>XQgQv*VqHy;j>JULUByw*#1=`0|<o-hAyVdHz*^7rQbKe~4Tv5tvDhTpIx8j>I# zbruE0I$FmwMkx711F%!P`9_AM=RSaRf%i464ovLo;-(0lM>qpSgaF_m56EoOIw(nT z8eOkQJTGC7B_(Atf>lcFyOBlneolW2c#AvEkx_wbYB6PVRx9hUOQ$wNU(JF?h-(O$ z-L=Pn;ws(*p;LV7_TMJ1c;KK!Qs1GC4pP3w#+I%LwwxZ>7EC~PdHzukAf586#k2I+ z@Fgr53=cEwH>=yQUs9gswukX1CMY_d4rJdt(=Ybdin5&fzYrf^D>kCGWL0%OeH^S0 zLBtV$I*~Wv$Bztu>WP0`dj1Pux*d*(b9tk`T)I8q)1oLT^qvl0>GgDqvlCkCb>{9C z6WxCBUU(=Y3lqc2r2ppnW;|xEW+^s&e0}FYNGvQRGjC{^lAQkAbS!`IeOgqMSW}Z{ zBdQH88()go43reM0fLJXYVk)$seGi6(FQg{8Gcy>Nopb@tfY?+0$zZns|x_&Bv-e= zpI-%pmpoyM;p>Mx9g4PkFF#vrgPLWM5k~`oHZxg8!KCm<Y(<~lF^^EDn)n?zdBuQs zy+<Gr)E|(}xb_YE{x;C!;bEdm@b~wpS{3=Q2;OeJ3fvJ=93P|NW>0N@uQe8`ezm=4 z4VLS6JWH<t6jzz~@tk4Fzdv#@ScB{w&z5^6Dt~H=cH36!h=`EEHjdIpXM<G=N=s#{ zVt9H~s@zBKW46rzNqM<uhLxBD9|G_ZhRR=jCj%Y9(@mCUR<rVW9m>8vV#X@YO4B?i zk&j*>V7Wqhx7q}FLPG!v)xUL&_@~VLCt8wba>nuQqrSbK%+cA=m~WHU?lwkW3AYw< znhOY%VEi>)j1M9TZO@uG9>I~I=;-La6lM&D%vzymDNM#gn&|_ypLi~;4m}+vCMPC- z<mGWZ5Ae5NW8`KJL&N798kg3UekX<gcuR!5b=f0{iT>Furu-rF{Oh@PfWYSv`n~*& z<Gkin<2SqU@#0C4M6bH}{1|f_QXq~my41beGzx|f3yu7o-W1ye<g4uEYE9{+nCp=X zy5HwpMO|aULHdEQ>wV6R%A6?sd!HCjo!oA<jvumJ?1KP)33-ciPHCS${GLCz<vCb5 z&7!>wvXnnebz=;O87`Mf!B>b@aVv?aIwMhSI{Nl3&k+KC<b@MK7!EInnh?*ghMQMk z)b|5N{)5K1cFt#mqm$HIyFqp?f{(woolne$lAed9a0*cjC#xK^#___oqs{cu(9jjd zRXO==Y_e%Mtj0mj*RjnI=RjuW^8D&*D|4N)((o`sP$xm9I37Ga<U2XVTMnz?_jaW8 zZI2L+7ToiP@Y=4v>Hq3VuL!1r`(M0nS*l0_1h&_8SaH!2kh71}R8$-6W>!}8rJkr| zK~#!Ntp|smh!1K-0&b^Be%PvcShO#gfYTx5b(G>w({H!X!1~|kMO~N7%Ow2g=6k!_ zSz!3n#YWd87>19R-r8%<mLFk%rR<9FuX#wAf|3_gZD*}%A!)pho8pf8Z!(&KN$&s^ z+f+(}$M7ozH~qweuj#K2A+!ICXn$(Bf5ToLnsTk@x&Y1cvfB#2VKLSoyM0Ua-|KA} z2@>7%-=QcUbRNxZCDx*M$lkv>c@-Y{l+wb??0q-I5U-^|dy(+z=}At9n5?|B>`|g5 zZp7nl;*%@%br}r8PJYIZhpAc)@HSE{qAs;Pijyk4j*n9fV<hWT?^PI;l6&fzn~lr6 z00l>*XH8dnH1gD&M0HC0t?Yrx_t*5frd7%WH~hZlgh_Zg`SmZT_xrMT{HWfg;$~KE zet21Y%)Yot-f7t7{>aB9ehPKshx0EQO>utFJWTZRlrP<72YRPSr<8WNQ!7Tyjo<dn zHL9-e4>*Vtz6uEaCiZFjA$R!5cZBN2i2n}fFaOtbMmx9(C{2$BMkqrALvl&GW$oXP zOfj6hAIb)VC#OpRTjj0036w<88lbs6!W!?MXpJ8o{~AYNRIR~AxnGGIrgF++LwJlJ zo4is@dioB-mmy1U>bcKX2SCTfBA|ZJ^^@mKX$cWPGWhfs?BH{SUGATKB}VXZsR;qv z?ZNqM?KTPoMyb9RC`D{)!!N0-o}d)+UZ1q;Y7N_1SR|9Ye^364VF)SB^s^~(UuJ!- zKmYAEX}#kfHU#}LTvnX*Yn!7{->V?u_oEC96UdZs&4mS}l-T|Y(!S#?3K{tyg&Y{6 z;oN;oevdg+^^e7rjP)h0Q1(lTine;hQQ8Ig8ZiH}bsO_#0nodR^smZJm)Ob49ASfU zFKIt#8iczWl+bBPeL(*KuE3z>KB8<091%h%A}#R|Y6lKh#Dj$dlp!O(NsE4=g~pW< zg{QIuuyjjaP2Ag~u0+}{NSTqes_+LuQGeiKyEiEqfSmhU-TJ%2=%rsf)^guDZgBC# zX}>?6B`NKho{yUMsuv+Q2cuy%Jn=Dj=Cgu=kcUlP0JPHu8ZA*abuvFIXKP|$d_mWu zl{-3%v5RAwH5GwfU5#3-Uzy^ci^x&+k-c>pes^*7P<fW(?yc9=x*ost=n5176m3aJ zj}G_!q)-MW`>83p*g{E2ay`Xnzi6n5!cU7J<M%?{IQV+f0>wPn^B}KwORjw%wkx2f z>rnL2;~s7T5$1v6S71h*-*1v0@_7`5v(vMkD=;iNa}9I4Qba{v8%#vNW&2%kb#<_9 zIa`-n{v)>r#78WAr{`A`0`V8Qp4P)s__cSU`dlsScH;`9pzs|7CAJXeonJfi2*KVC zMduZZo(^9Zp!hz!FgD(@d$U0Sze_H~wM?_V+|QvHDoF-^{c*U<>wZbF&Tb`PN~Kxr z5jr-;at#4rU+dR<LeaJuxMKk6-Old>`zHt@?<0Er{f-v>bIHTXTI>$U4GZfZ`RLw5 z;1?L^W^MJu4FdV&a!0UX2n=!BaJG=JgS#p4J1fe<{f$!>01W>7F7;4_1BIYIf%2@P z%){w$wkd&92+nR$P#sdAg+xaFoK33L#9vP0k1QxQyBp@{@Qjoj9hH9#_>6=vjInC# zp>wT^?&x-<kulKm^Z|x^08RdJUJr`k;~)4S&^IG8&jCmQGn`@;VEJG0w-Nug{FSz3 zF~<(I<!54&Nb|bK5~2})f`Wp^^acerJUup6`DJ+EdcHsw<Kk&>+^EC$HWDyLZ~$XD z+&VBR!W~OW_e2bjjOu4;|7@me%}q=hFO$sOZG$$=#>x6=+Ileu>~V?ZdB0IZ9s1M( zwyEhKh#D$=zL?t77^W?6%XGlwkvzs+;nMoWUJAL?y=Q9T(;^>VW63g*1XFU6#e5@t zPq>Du6_FQSYBnqrRg}a-MxMu*-f^D$mR>Wfqg!SSe79kJzINXc9g#3-*(MV+bTYoB z{2tWTPeb*ARS>4xc7&b235ZS#iW&kQ8k!B%Nu8gh@n`Vu-UA5Tr+_g0tm<l~63`D2 zSkZO&aA$Bl<~=jUc%j$n@Gy?slKXP&c<E%3`y%)nr*!gJXt=DLogKAilb4icRAmNm z{H-U}?>@9}SFPUjiUiiJ4HwV`Up<{B$q!~%N1XSOLyV14I{umlzk?b(+>a?)vFI$W zq%7tA?>*hZ9aYD5p60MNWyVHy#k5Vvk3J~J6%sK`yj-_nT=fP@j3&O`2#lO<d-0Z7 zD05PV#D4>df1!*2a=vj1*!{WY$CIL{TKEDGa&HSvYrn1TYMoNjxdW|kUBE#5R=Fiq zRMej+ySMOJ_oJi2NLt7lIu7Wp<6NyNH9?DI5vDcl&~i9EnOo0Y#8bpvL`_4GkS(jR z%Gd6G^z);jOZbm6L6=ucKCfQj8vJPMAqnPa-wzsY#?O&^DegP;T=PJ-L^bG;?`pw) z^>@(BuNEFBowqCdLiqalS0!2OF9Hn>H|n{=`O?i>hPh^<UZ_V7pK-{Heedgg*YI^( zwLn!)Zhc68bx0{U4CM9H)24ujqkGkX&VK7Xnfl$-0khk}Gh7ySIni(ehTu3_`z^`| zh#-M`OBzHh7U|w-k{BFSZ+~0NRc@u9?3G)Vui6rD^r$(U@4nG(+4VG)#@gfK`8ubT zemL3Ksv@EWo?x1@XKHJo^VvH;r*^Y}_I5o!bVys3f=VSusn#nkq#+iR4#p*~7g|Pl z)K2=i0t}qCN#8;1N;-|y_G|cSsv;sW+@JB*xTSGCurU!}9lio^M%@vjPdMj@a|K8~ z_ieR{Z@SBZbH0>Qm^T>2Y35@fAeU%I)QPo|o%Gojc%5I~&Us)WFtvWdVSVQf{6Mcg z9pn<G4UE+WjFo=CE~5KG<Iv*xQ=LWvg(=w<Tb0SqU}SwjbIh~bkX!~7K_QEpx3iw0 zEw=j-bCoHt(#sap`R9Hjk%@^VMJx(qV`P&t$qT#(HQH0g_F?7aFB5pc5xKjKp3ouL z=5d}wkqMZlqvhJ%#MM=Q$%RNMuD>phHO0lnK}I0av}yZtMLgp+5jo0j0Pt)&z_X3W zNaLKNcOB$%`ftQ+^n?x2Man0VPlgkjNYF^c{&EkH%ymB)U8}iD^uR)pU@&Y?KPd!; zU;u`A>KO+NqT8Mh9&t*_WW2uaCFN4L5xv~nk3C;KvukFQVAob<P;yS{GAsC)hVial zN3m7CAmBiB1#&(Onm$}7B&Ocn(soK?f&ONEm=;!Y6#n2pLjw0e2kzkmT(<vZbMHc$ zaytE}WCnhkE5gMnysMJ_lRHx8t7`|UCe2LsmByd(j58mBi9!c#-RUa;2Ifs0xD3yw zM&WloRlUMmNc<&_w)sKaP9SoBXpY>uD9Jw%=+-;#?Ulm6B!y~*1DL8SsnfU3m`Qg8 z^k*bs*8Um^rzU9?N7=4IyxPbrx<Vw#4%#N&ThtSA@OE`w^`4{-eyp|F0iphcV?q4K zx(_q|$5LWZSO|+?r7LPS&+Xlcego@EE3QQMkBTAe&jCLHULB>wg!BLP@~88L@5;P# zm$N@F(8p9f&ok8DeMi+SzdhGG4?{`(U*EgkM-1HnEaN@E$t$p#_9D(+CCWF(SNbs~ z0_wv(8Q+B=`3wWk3I5|057q{pqJ`0lqmwCQB14_tB3e-;tS_ce%5L=(V>Y2qg!)Li z`NA^^$p`iK-)rN+#)yBL9B$IrPCTkY)!ECub1aT)uk&?1W^00P{bvr7pFo%NJ_e0) zvFb0q&hIgDR18Uq35?gYe{9uOVb!j)rvJxcA@kC<C=D0Oxxe$Qe`MVtl2uWDLDN_u z&7&u>em%c1f|$fO^XR|LG%X5uk$|MNb`;xIoYZ=+sojCH?V`|<+B+$nb>10LeDlul zSbsgkYibdKQid|ve_7_htN7VAP=)C_6D>t(eo<dHS$fR%qe7iMx2wl5lC{9=v*y)E z4H_l_Uf|!Z+VaO$A3S$7+D|}7wyj?&?n49Z+0S8!R#Mj+gdKYvmOarq^@mq~xIb%W zrqiI*z85A!8sMl80S4r<B#mR?@yC>Z-~WnmDe3_Cr2MQMdp7zt4W6Q2_PMfc)UoX7 z|N8#)DD7Dk;TrQMYHPOoNIrl+1AhiF{^dS@?(pE%x(im{^=CT7Cha)4zS8pIlAOs7 z#6F!xvV#hYA2$Fd{%?y(?iE7PdvcU_e+|uw|Nj^nhvuoQHb03XdT31G+W3c@ZH*{# z(wF2Y?5_X|1924$EV75V`u4BOHV(k|{(fFmiuiV7C%k*4O}Ci=t%8FVJ+#2DzxOU~ zg3dj!Tw6AS|M(0LcplaldZi*Tg3n)5Cc6MOMt)6p*Yi0@T8nI>Mk?u3+LTGVj-~vF zufUq)pNJ6fH45S@(DGn*lLm4WgtU)oz*v9TN>u1YE@NQFzJp%mZ|g<+q#CR7a{Xz- zg8Y0J>Yl}j^K+EhIv#=QgBdCb<6#LyC8bdCUOki`$T-qq?=q#_T-G~Vvo_tUqU#)- z@V~@aZvPFX^}godh7FHVF4yDyo$IN83mVUo?|@hI1Kdeqo=unW8rT>{ULoL?GH5o_ zc6N5!K`#mFcJ@DdL6gdMc(?E&gGcp8<dzH-lQvU}b{7^-0l=$ZSlG-lJNefWbm8=0 z-<31nC;>0#vNL9wz~bB-LdYSJsXry@u+KePW?B&)8@p4LldJiI?Xaq|U%6*Br{D0V zZ_;kNNB#nr3G`{EMmCmSo1|a4IF{@%esYw_Zr34mjw<xw2_K^WAOC-PFabET&Vp5; z{>&hhBj@~*?Fx=plERv@HRRid@F&bXY#c(Zz1HpN+7CgeQWF!wM>c)f<hGp{EwW-O z(y(j4t{mPf&^y>46-6N{VR!0!fbN6jX{|Kr$mMVy_|kK_L^BFA?{!Y?y4s?U`7(?9 z6~PY{3FC_j25)zNOQ{T5W+1YK_<Iv&l$T3(ODU%wfNaDi7hJH+?w@ykQJy5=^AdDA zT=-eEDY<aHfMXBSdFf+ny2I?YFPy+cBUoNuPR79@ORrs5SdH(!HHJlFxU!mKcRquJ zb%)KMNm5}XR%j`&q(tTu5E>aX$`t9rFA`Q)AI|~KpjH1R*#{&(gNR1|Gp2szVFR2E z*g|%3E`8x6RoSLh$p02V9#ga+`FQ=knM&iJ@PFgaGDY@zOj06wi;Dj~&1+H;@H#Tn zot@VWPE8i`)mcV{kJh*jg6~HwkZyq8dD_^m+wGDVez{M3f4Owk+;GTs4ZiFQ9;<NH z)@%!$nYKQ*aTMi8AL8P9FUj83DJ5_F>yiohFxqW5WWVhKpWEZ}<pC&vNaM))3{1ZC zhF@hq_9u+zh8|yvT@(l-tkyUb$8n6gpAwm+{X=l*o-Vpn+3ZN78X;kc9j*Z<ky7jx z^`<PNGf2hymorEW2Cjq3J-cO`Q^-pB2oZqz%l(R#LOO*6!~Sh8I4$ao%JebGv}Q-u zn8MCD_=m+@qlh1Wv;E$sCbP%63l$}8^xd7*SiR6&F4x`41*aA)yeAE$c1wbF@lnvZ zi{%G9<uhKV&pqkzX#IPKCNjwbTnllnI#jm}*u($T-~KgG`P;UnobBlqCF?6|0jJ9= zK53eS{BFw2g~%}c78*X7_uV4sPI8k@l~$z`dt0(8B;*rt+bs6wD)XE~FPZ~Ee~o5^ zbgBxQz>W1y2}!kj?%(-Ohr0J(xTXGZs!Esp9xP>cPftY^(Y~t3SDl%uz+6$zcFGUc z3ws%zjHi1#KHM@Ueh-{x?hDZNU{H77FMytzx6Z~n945c~ei!z=PZDtcI9?Tn#iRbd zp(&`ZbYxr_gOGoCtd{Q6^@##u$3tDjNYM~;E-=56&HV}Yi1kT>>X#phiPX;z-A)&v z+4OACQAU)kn|7QZt6hGs$k-AScOsETUCpFo;r(!^>`~tkAJl;s^|_5C1w~#Ds0sRQ zf1ydtZXv#fnp?DeOL%+NYg49FsLWSDIf#s$y!To{F~_~sYYe<B2TWHPt!A^r@bL9g zhLm%YsKujw3}rN#B;1nf-?R2xhWpnNW@o%EazDL3jD^r^SE(JucO~*9{?BkYfsOEU zkexqdp56cXTWwzf8I$((QMo_c0m<?@hwcaiaD_fsg5=XiF=G5o45tx(+^kVj`$6Ky zc>R-AW^%8)EtfNugk7m-wEN^RT<f;JKdw~VVB*OQ(f#`Hde(4rEFTh-HD-I4jj7yr z+(_VqK2fSw(`}!awg)NR8hd|?_miM64KnqYC{YmbjDni;Hu-$@(Zr;SSQuec25Gn1 zvwJ1yh5QpmgQ;XnEJO?f+t#0uT8TZ2opP+xi>rh6E25g{p+)z8f7GPx#!SI67Upb` z{?ZQ=94cW1AgVO4MbYX5$^5@2m&0=kf3}QDrgPxuVa@r=J&Hy5s~Bu;PX_J!BTlbX z-_OcQy<wUS8%?s!k?`AKXM>4aiO7VQ90)EGhol$0_wDJ8%FhqQa2+n!)5@@?fth;? zM`kBEJj2eGh7&VLV^v~ATrSt*p)Mv5YZde=42_WcLohaj(u%lBbc3zq%&|Q86c#o% z8*WQaFmb4n#vL{;t3&N=01jTMMqQ!%F%!SCrZG`!ecJ8KGhnr9mA(Gq9ut{JkDD-i zSYIvKJ;XOLwVXKTv`applCq5ClZ}a*hDnu&ISj1k2&AZ7Gr2bj&tE0TB=hE6zTTO^ zGd~1o6l?=Ih9@x2)E--mYBj;yLc1O|{`8coVjPR?=J{)kOAVtCR&Ssd`PXdnzwFN| zEq|96(G@BP2fHL`U3U#0m74@AUNG;|u*wimS%iaq5{Wq3hT&=WzzNjWgCrNW3t5Vq zslv;ytkifJ{N3=rovhsM;+<x1jn2vIn+$!G-jBjGr9TLGoO`c>6b<^P4kBYb<;Sg1 ziP)5f^0FmGjbiUc3L5S~WOUMMUMGGYm#$K?&dced^8pul#Wb`tJU`B&yivC_;24-W zB1J=qn|t2S)Zu0qEZyS_=)G;Zo>HYoPtQaQ3IVrc)CJz!`PI)LO^QXg^C_htH*xg+ zGe}W}BR?>9d)5dw>(q#Y<LIL%9F(7Z*mEB>pW1B}ns?fdUGhXU+a2BC$Ny4mQgAn$ z1_R&nUrj!g3#n!Af1J<P&o!r~cOf!cw+SZjTIQC}np1m4ycPI5aOW3nD6_6n<@yMO zu}ZN<PsORd#q9$CbItwz#H|{owKKCk$k@HFo#~!2c*2F=1_eWMod}%p+(pCe**iA+ z<Jf-WE7oY4?dk9^<;_+&r_pLvx6d*qfD(8VC^$uL7Hg~^@?tx~R1-p9T<h);n(p1g zevN&#+bX$bM|#}>F?F1G@w@bdbxBjv(SUQRtv1LytG+y8q1LRkJ&V1EMd=qey18<L zDDrbM%lkq%WV=)AyT6#qjamqmFr>@oLsMhYux~%@Kl2zV)t7p%gi}oOh`*E-D3|_O z-=!Zh|0&b{5hSh|Q20GG=oDp9B0(iLVmA&K);S{Q)*BVuSE=f1wfg*DV0flYD~hv; z?<gk2!3+AO^yiMn@$PVRqTX*l0giURm&8FYecl>QCgLPuaxpMo)&}kD2a4~E)Fp-B z^9T@R3W<e8uoKkxEO|n68XGnIQHZDz(FkJc=AC**R<hRTl@BH)+^&1%aaY3(i9RNq z2``sww#a8La5Y;@YBifo3i!33BjcXVzSV9?WaM-!E>^7)^=pf-+B2N#`IRUW4H;Pa z_;cPd`q>rKW@#uWB$7(eczgFzNm|U~Vh4>nYPH;<7jY2qdEKXYW+5=7OZudq2!oxe zpd)fPPqL_*zyLa=Swe3$1)~CTX;GGMOkZv3&F%%UVt9#Od?Sy_lb&u(%w2u!it)lH zTQY@@B9q*b(rLIHt0*PqRn}3<kRz~D|5oNq^3Y<po-KsFwQmNFO&C{%{BXH?XWEfB zm`v9nTRQf&pgr^b7tFo=AbEYqoE0{eId1g;i<es8FP;a-_6;11^Hn!<FSB__mnvMR z%#h8T52WliTjzZ1oOQpxuRTgdACX*pGe>)Xby9ffuKZ3bK8sPi-u^1J#OWurImZ93 zH~Zlt_}IbmQWjD(b}ZPC!;pS3<vHAUCgsWgyQq~5$d@1eDFPmf>=EZbxi+HZv~Sxo z<~GaSTLdCL*AMC}(-JdbXs3g7;p;^+puXu}MErSo@({SyVmGsmF*;WSSeURsIE^b= z_l3`XOow#|9g2~rLFb~{mukHjErw}s*IkF5sfwBbl<*N&y~Y}oUS`L#QG~bjYVYkg zhoWpT^LpAlr3}ZSO_^<u88tP_TxRVK5Sdvj40?{`nR%Ec^`<R<=w@|E9NqhDEN65? zdaVd{__j?x)_wKMcAz$h`r~QR>{Xf4{AHqO^yD`T`7?vvkOT!ex%3u4C2vi4-`+?= znh$Q$#SVK|<Q1fu>(@b1<+|fW&ifWk!^t!((9B%E#VQ`4ngcRkBb2q<RNtoh)euh; zK2T33ZiRy%LD5jsckOkNG9sJ(xeI2u0adKx^vcT5-RJQ$7xkq_dm6^T9`B{^Boj7} zQuDDL0VU#_kXUak38}$?uS8_r+ZP|WQ{pM3l0#YS23%Cgd*YE4S^2owA01iG;WNMx z&?0n6yT#!G9fs}erte-{aI<S^n&rK=yq<h_kztBU7h{suQ2}&UHNJ4XeX^FPN-Yo8 zp_R8i-nknx(ZCsYM5(ApefMuQ;daBHMeb8=-mY_ubdGzE?BgLj8d_vWY#c6}rcO;0 zh22zH%vGcDy4<<1N}bG6_6k#r8l^;&qF=dWO21C)5=)P4`4%;4-NUR9pU+D}gyWJX z`)HwwLO`wNh_<O6p=W3jsod7S@qBwxrT$`<q5<==aIwl6;<bqQgEd<MC_@(3cg70b zk3=yL^!27CAoo7&zkA0O<mEGH;LX<KS^z-)QyzY{$q;>lNg<_n`&@byIF76BIj!8X zS4#tw2GKqFH<1pP$;|;ze@s5tfcV!A0z``EZi#8pUxn@<P8VU<TVH|8Av$_0`ESSb zQwL;TdzaJL154O59Q!jTm}93&Q+B7HGY?5Tj+aR|V(G_-aTRo%(Ar)jd^_TFy{)9x z#-fWj!|t;)o=@hqE$reNxY~D~uC|+er^!`oKTWtuGX`CXG@t&Aud%*Q>>YGT#7Ubo zRAMp!)ldh}$+x~=q3DGG`uEtvJ$1Cp0~-aXqrIIZ7k3GLri}0ix-9WJyrr^`eUgj$ zb&r)K6@f(i0ju~6@VTY`yNU-8%|u2Vk0+k~@OENew>vqjg$DXgj+Z&xZf#OhC!y~P z>J?Y`uk7#wJfFf(TkcZ!7r`G@p1#3~6L%4grd2D~0Fq-R;&sSJNT4{o<1H;N$h2_a zMH<bx@nFP8k@7TW_e{W6n%7O(`Qf~_%36vPKLZRAE#k@xE(YV+;&kMeck{oPR`Rod zztw+rP~THy$=%C;g%_!)u%Qh;<B{GF`9ACnf%|Z;ZYSBwP0@eacHtXH0cSu%vq=mM zg&NQ_&#wj(G08yZ`wUTe!xIx33ju>C>NzS$j0wz!g$9PkMa173aR_nASd7K`5Q)6r z>vx5!e2Y~2VZTM^aKWSkK~G~*S%Q`2{I*>D3q}xS@jnSWpD3m^PMeQdBj4M-0+!S^ ze?x{|)Cvty*+i~nbxXlXvJBeSahcujK746rRx3So+=8~fL8vn}+j@f~<#qp{SwiN8 z$K-x&upM8V7L-O;|MKN#;Npbe`zWSJg-x2PyZP{9`T;(Y(>?a8%0&Yv6qu3+U?)B0 zA=&;rwf?s%^IY>!-25l9u>8X3*cYD4>!iQmkK7H@a7XD@4nSK}dbZ|9T~x^GI0bZE znsn`2`U;fU#ax7W?~eS$2BMgN)<zQv_6GA|n!B%FBp>;sykk|>)0>vefbzQISbYzU zV$$#0{{GTS<;#~kTGjGT<T45Lin)V!TM5+i$g$#TYW3@y^|U_zp=0_i%q)h#12OE5 zmjxCf3|@Qev!|86t!K%_!c#`D!hseq6MWBruVk_{nO<pm8r~Maz~!+mbl3q24w2=V zz=xc*2YO?kBV+eX6pVlN_+1x3EH23?JhbH5{(;*qREviO?Td(bU*&O+D|Mt7q$FzJ z7P67^`GE#Vq}b1hib$Z7QYntT2cllSZPdnRJ-tx)htD77B&4OHY@>{Oj+rpY%vN4= z*<Thf*j*0pX&E#62`pdnd*S!^D?vOMj=h$fZc7dfk6P&em=<t?f5oNdyULdJ+I61( z!IU2+>05LEAK337_!9;%Q14X2YMGw9_f)JA^5&Q9AlqgiAI)_mDjzLQ?ybS`k7mJl zX_Fo^s;O%s)e;0Qo<-9T1?Lds`r9Kdov72Viu+;r+61VFkQWZNC+ddVflSO6Zw!lb zuMMm_R##tR?GV;Nert)p@QKPB`Vko!nRm50G;WT3=XFbbh}VM1<9C!{P3@dUM)q#2 zeYW4)D6M^sLdLYd78w_}9;^!;;$u6==d%XV3WPOeeFS~4!6h5%-$bX>E@9x32&T$k zB`h=T%ku!B{I8gh4@Vq;d=TN(SfU^U_W2*!-+#t=X>A;fmpVnin157i7axx+$Wa2_ zLxuxm7lH{IE`go4f5P7rYj>`6Ca_pl*aM%>DpGhzo%ksNdYnjb+@3Su1Acho7XN=? z<^M7jr&CBi($D`6UVlN5Ph<cb7S;|-$Uj3FP;zFLG<`{13|f>|Q!ar%oy11L^L7R! z2Y>%Sci;^)`r9cK7)x(Htos{A*Dd@1KKCTc&)EO9w(Z6Y`-J1su-wdQL=-0N^D}?L z)a>(L;Jg;mf1<13oXmi#iqUEw2U}x_TDOeq0SEzbwEqQF1ETc<%#V^q@?VSsxGVC1 zLrWe=fWN`IYomV-u=4+N0HAf}kCRBK@&5(hbLwwbkYU3A8{1*2uWG}jTlVb%90Z>5 z0aSlrB>ui^#s!dklut5*yLH<>e^Utno}TDccjcgU;Q_+v1}Fv6e_MwC<26^2e8?ZM z@<05Y4^9+CmTd-l1NZ?9)+tk1>))*&%{W+M*Ja#hjb(hn`<9pV4_|>rlc7*%t+h_U ztb6<=pTR=(EQ^&Npq28coJ?8iCjeqFG#q%)SFaF=fY6Kicj&!c7R5y1lP{}#;ln9E zD6x1wf^D@}J=Ds;_{zm4w9C&4Pux|}62TjtQ5DEv(JM7?eGy9z-nIF-S=0Vw-s*G# z#?GZWCrAZ2dS>~65I%5r%W=$Uj%z|ZsKvxyI+jz27L!2z4%kf276)Ofmz_T~YL5FU zSm)JR9k405G5TWd)^L5|^+ENd?~1#q3q|*&_k0qCNb2G;g!d07NQ>aX*(&Iy8_hh9 zrI}R9>n%O<6*sx12Py8Vv)a=QE6f~B^3D+f%XKU^Tzx`*!ThkC=k-TGv@!WDCe-rt z`emb|LGd}SFHmna$l=x9I%`7nO`gy&ga(|p4*R8e#s_EuSfINPt`sXd0c!l5IUo_* z)u{~7e8A5Dg0-eQy=t~=YluMo!yAit(m2I*FQ>2(_>7e`G0K|9q8||Z2Z#uPDOEnp z7K!r)NM4X{7aD|H)fU;gD>c2JTZFd(!S)Z9-@ydTR=kheJ9e;+>@Vn5k>+9|y!6)} z-XUPFUB@0r#e2|_BNa1H_ZfPcU)vrD9g14B5jdT$%&ike4cwaFWLY4LM=<yEkjbP# z*i%3;NG)v-xl)r{KUR{p);E$-6_&0Bw16bzV2~{=ERk{*$(j3R-CnKOak(jdKw~nF z?fCH4jf2Z7#L(C50+b!`1CH#uO%FuuGmCyGcH`mG>h5&#t7{<DUGvfIt8Ov#Fv<J& zIkQ@C1brLX;*d~d>78y_L7R0o+7!qnUiz0N>+^gS78BSv{%?AgH7CFLr?s~+6MZ_? zExp+ZUbYiI#=u)$-AF-ij-E81>YugmecfQ?V*&Xkg$mUX<e{ggkIm^<=>ee3$fzjN zU*^+Nmz~J~{fxc9qdcocNBMaLKn^<?wXpEDRZ28s29wqdPbW;7DVQb8D;WGW5D>dI zX;hF6P9k%dP5W4bf;4Hqe3{+-?5PPvG3=685uKZvq2U~Fs{sEQ0BxY3yG|Lw+B_8R zb=5u|pM1f&yD1Px?RY_WIRFsTjbF9{lIj_I*vhz%=aoS{=__j+3pby!Hp9|53!ab+ zN#mz!eJ0!a+Jr=X<O(f!(|_IV3RY|!c7tp*9l-{<Y%g7L@m&~LSfsqRM*wN|*@nX^ zuIXZ3-2N;$f0JW6;L4p-`v9`mB=(Nbn$+Q_nJh&*Hvf4rfY8-!I`S8}kJO$m7gstc zJ`~HIj7D#;v3vcclML^&3trc_2GeR(ey+dVWx%z<Hy==Pw0@l|-PuXTtg|hiu+=qi zfydq(ZOUTUSvVOPlLDL5`-<cP%5l}m!7z&l_c5ZBH4|T7lcU%Pz6^VFoXqPO{5Qd5 z-+>wLO@hMjb*gFwX`$Pw>C4_1-I0>I8Dy?=z4GUEJM{=4KrnC>@!OxN61zVG->4() zfrN)rrDK6q9_ZmkYInCz<w^9-roWn?+@{x#%keg3&VOg;tpTiS@HLZlgmAajk6<n? z>XH+9O2gHz8LxN$B@yoj0HPmjLb!$k)K;>Dq(TNg*G*%wDi&$Q`%Q^~dQ`g=LI08^ zsTLanwaPn%=MDPZvVDjT@U+w613bOVUnZyz%N;B?*pZxVG)yuuGAtSiRCOUC6^boU zB}2w$h|_exEF`_!oqPgNOyBNqo5BsJ_Fr73B<#8@RO`n*ApFWArIQe+;}KnePb-GT zJXoCtaV7l3;=aAJ+^T~dYZ#|nt!i((Yv4fP%NN|&&9A#Q0GM1d2`z=}<Wpo=29NVm zL}1;Sp=>+wi8e=oL;BmM9VbdSjkar>$bLW_{6PL}iUU<`bKbVwxBaGCq1X?iecp;I zB?Deo31DuG*wh3H9+n!(ou85Qc}0&&N`ZQKb+x7kd>afW*L~_QJ~nG_cXNvM1&e@m zkE3g+kneu6Nd=U^ZeLyULHLbExkKHjRCY!3PYEQz!$oY+sPwZr(<l2s^$&>zmd2tn zLvL%Fkm36FuLnR8Xe0c=V@N|{ra`-tou5S>1X6N4CwQ-DEYb)3a$Qg~x6<TBDU|Vg zXWG(WzQng4)Pk@Nq}J~ZXYy^C9lI=o5?jQb_7`&W3<PvD2J9eQwtWr>td6-*IO_u+ zR$>=VNJs@gE`sy|1?mSfwn5J;CDZY`fT<2v=hTAn4mP@lW*DNjyNEAfEDW_WZ?B`~ zr{y}MuQf?}INQ9MFv(@6VjM$^GnN?&@{1?VSF^gMM=XtA)C=Hi?C(F)D$Or;`?<&Z zj~!`+)m`@oMBw+*+38Fq!tR!pZ}2Zyc-vFw<^WJH#{eWdwh-!;-*K_ZTYOYsC6k)_ zy6DLB`28gt-Sqfrx{O(jSUAxRQ%0oShElqfd5z}%Fha*foQkp5_lm7H=<s|rr%f}W znwCSDT(%Qrxvj-~nyyG$YyEg~!b$c>-m=kcqiHiRj&Q`>UQ2VjM4f?c`7Awca>8qP zr-3a;J0<$-;Nwzy^JxM{5pDHxE2756MK@ETy&@3*yasfk%eNlYpRuQ~<!1uZm_GFC z)O4VwKVTGoTiJM)d`e@AaaMA^I4UBuNn~Fw4YTar>wdL-f3bL_^n#X-X8dy|GB$mL zQqlDGl2>pnZQLOrR1yGAqGMvr>njC>L;>QIW*y+ZcHbN<TBj#-xhvCeu1m%ddVV(H z@g$d+gN+C@-)-^)LCU3vKBUCl`}psCf2pjn&F>Fj&;b6Jtyn5Bw%UM}$jwU1qR0<j z-AkiG(Q>Zdo&Gu3A2_<^c+n30uZ^Bj{ab=O88XT6CD?hx_~KOw04oQC<HuxXzMrl4 z%yZbCQV$~nWn58#K-%{Q9m9qb*Hm_=4w)ZPxw5AHAKh|a4Fjv>1)nPJuvX(4Htm$s zc4`wFxb1xcvu7?Ii~U|zTjINCumNZGw&cJ%GnFS#R?0Vu3^sOgsOae2f9|0e>b`P^ zx_Z6Y4RPHbPT9*pI1IZOtC;Sp*x7@o%5_%g?lo)@!DNRady3%&Y|c3o7j-U<L#g3| zNTrKzj-x+Zn>nV7OPikXN5`d*zLAG0{50l2cGf|DTH4rS-Cb1KeS3?FDpRlqHKV`V zPP+Z2wZ{)(*dsm3WA2(%0c(MU8h)xDWuQ#&ULX=KRlu@9G4(lGW}ao(-|lN~cVY=a zz&~p><uBqf1VG1V{REoRM8#Q`PS(0}%F2!@WMuBipYZBh6AyL<d$AnpI;3*iD$r{U z6xco)m6HHF@{>seEC!u%l%gXj^_DwCP7rTuP(2ZQe;|QHK}qRi06HNhBI1`UC8NAG zo5BNfOteKy4%t#vn5=UKi1B2}Ph;YqrKn85dV2coO4H=U-ggJMEk3b~ie?E43bJB$ zt?Ih2-$><Cmcr+^8<>>H<fbUCz1nfu^G<I!prhJJ&UNnZ?RC7`&t1Yhx>>H%dEPCd z0`ygUzWdv2)c(~=8_#p~VmB^;G!m_H6P>}DCMqj^Gkn^s7<A|=Gs>>=Hss};gO*lm z(fs%=*YZM*HN@7EQ*4j-=FXK5^z@7USL2GYi{}!Yv*Qf4UL>!$1`G)Epcx%v7hiI% zzL5O>o>FfDk^e&0Et48STB0rP1OrG~qBQs2^47^6Tjy|2mHV9jIidi!h8!)NJbKOg z58Yt|_G_c4O-3j=#l^vDc@#3ER=5E}MIRO4HoYxD?;RW*UlV_xa4=WM;hN6aOnNy& zlxDw0VrgK%Z4di;i{j6CeQaAbz{&OzIGQhZLwoCa<1u)Y8VdpC-3<|3rB=>g`&r1& zXQqJ(HH3293BmAgSyoTVVGG+{HpeF=8cm|*^jbi+qvwc4c;L&<&Eiq*u!l=NZaftG zfUM7j*z8O?f6YbhRL0?QOWvQ0BHm-Sh?du<=s_oF==1GWG#p;?WNvTjrWTnqZEt>B z&w;6@IZxlTUc{cVK9WJ2D0!~rD{$C$Jh}#y)uZJWoszRDWh!GD@ZW{MQ-I1e<e1A% z^UlixtxMuQx8)tHLpC0deI?=P*%^Rf5wMa`ISRr>)kYut1emuNKP16~A!O<`M?dP$ zuy>O?Diq)Hqf+Xk5xlIXv6ucm{Kbm);!Y5NGgZlol)Lj$;oQ@>-`mq~!;>xJS2sDS zsHhaQv`(V9SL<JV>T^0HE7eLv+-iuAV=x{flaiOW4YkK>$j#3Fu#9YDpR2)PokN%; z$;bArgaD78g++Z!bi&?pMdm=wxPq*~O;)Nw?g&4{J~}3@KcS+RTV<@SDCsQ|93@am zb^^hG>=^0le&_e^Y3$adC)~rO#nJX9DNMuNZ*k)H7UI^|-*DOO`wkJEWWD+=Mlksk z^M2FOBX3}1JXw0U`k7V|>h&v{4TrHWe6q^Q)%=hQ{<j<T2@FQ{TPc_RX=pfNLus<3 zUq$Cw9!Fja@11!rL-oBPS9U|HrEFey+Z^gmqXXx*`;nZ(-b^3T?A38(yqF$Zzyz~d zzJ_`4k()$wbGzMjv-ih;g2>lsHQp1CR!MB86zo*a_9c4x@qb$;I-C!R3W@~pNH$$j zL@&>oWC0oNXgwM7t6@q<Kk!Wmp8YnSX@&7nsv?-+pn`s2w6wOKloX?z1(Y}F?$`~z z8S+~7xHdU)2Qic_x-d1r@WFWl62tL`;t28o{*_s8LiOed(7nGVXcFa-M5jhrF+6O% zIeiPhiHl51QfDEayqd>7zC4&`IBJICC9vz~zIpQofg6iM&Jt25=5;23e3PrhbF*~? zieNM6$d8x1!q0dzEuy*lVIOl!cF7&?c_3%r5^iq;xqmvAyNEtk=!kbfY~CW9vDQg7 zoh1vtIS)R{wy)88V1EI=N|p!h9r4i>3LY_CizywS+f@u%zjA#&te}G@I^A-$HjTwO zwef5Z97QG-yIbLZmKyV>Lo>;1xQb)9$g|sOH-6%h95h8|S+{_yvu}QKvIP-GR%rp? zm2r3Ph}K74!cV&o;viKK!_8JXrlZUYEQ%^0(b;fxUS)}eydwplaeztjI4q}bKlW(t z&Z*>;@HxW6!%`ahTS+?D*a$DaG-zj8rgGUUBcR}XXt|5i?j75pxmXbHIHq;jT@q`) zE%{k_lMyvVgZWD?z^!crQtp^^8P()=B|n-_FKvHN?F?@_9kRZs)NHtlL`$jP`A+(T z11JVyWSCG#ytQ>}I85j!R~qqVDrTcd-(l?I8+olIGF%SR!n?B;Phz6#Yc+oqyq`~$ zJuw^PzSY;QgnXErDqA64FNNP)J5*u840@{+rYS<A7&M4x=c;PZD(BleQdZnDH!HLj z^|A@MoHYDH@%L^gluw27O2(E5WTESI#>qmBI~*qQdr>VhVVZ6a@&jWOm5F<<u3=v{ zbRgIG-tx4r!BAB&zETZd;8AlQ6&G8Lr`{LH2m7$oXS$enEW(&^yNF2C3kYW}IHs`> zFb>6`NE&-UX|ZrUp=dOK5?8gyZc-@!tbL%EHZ)tKj$4gfI)(g!V!)!wVk0nyfAb+5 zo7Ep>`X%^`1B&f+W2uhIWz?7o9=MjEZYAJ#xsCzUmlJ?EqVJFo=`}xa%)se@#K%8u zN@C!WfG35Z_G4+V*HW-uSM`N&+y1_VW9CM~8jyHNuv;&dOiRekOj+`15V0yyggn&m zD{a>;@YcsE!8y4(d#b|eTL34;wphI-WwNV7${6>KThBn6dUSPFFGcaEddo=_!AJ%r z%4+mdoq3JNsVTDQVr~io-y}AFK7%(c<mRH(@b(-xv7oJ}<97Eos4e}MVs_`%!KAY{ z5z5K;I?L;Q@f2sznN=EGR*PtSF1JP_L*x85Sog5)`V7;}{@&DS63C;tEwYjX-C_VX z6uZB?$qAG$(XWlir+%uZE&?)(J^b^2Yjvru5<X1=CujCGE3B>;Pr`yB<Un;4i_U-h z`F@26E`d#_rB<5%irPl=S&Eg7{W<#mb;kMjtdxSnSmomT_-eCxc`x|N*4R>2l@mcj z&%kkDBB0IIc!QUe-VzIe5fgdchdS6fiu=F1#kfUFwckV~s=d4&HP)60=Z95F>)QvI zExPG#ANqQh9M<>pw7}$E20t0TyNP~d3TgIfgeF}r;I||>>?}!biG~nW!4&VHb5#nn z_$`xzM-9DVbS;yxQ6fIK;wOj2ft0568NtAJ+>B49=Vi*Nyn-(iGEgqs<As$Ioxx{# zxQsSGJ^-OLefM{^{n%;}dx`d)0*v5b&N+gtKej)oKMuTll_gED-mK7kSCZl}e~V~3 z|Es84_xx3t!yP4%|3?FT<h8tR4iT9Re&bxQ`IlRNzhU;;LBgcy<vF@jGZGUvgLmJ# zZWOYObxm$L|C~2(nazU`qrShq#Jfz@-@Wlp8kbGLp#SN$oqbR(4J|3RhJ4G@pV3E6 z#bq+e^jI63%=I)@8IG#dMsfsK5jEQC9!once|uag_a5L?S4g{SA2;|FJNi>~VH_u2 zA8>xOrHAdL-4|#yy2vYm-w~tL&ryR)HS6s}5YX}H!6$KGGE55T0{J4_iLfL_iL-4| z;SyCWI)Evf%=eMAEOW5KiD5AsiYRM-n(Yj^x7$~3MM9kbq18~08}~AJ-Kd-1D7ias zK24!O!1IX~o$u~67b_|%(MZv%cc?S+dnvD)r9shPvh(t%?6Wdc7NZ@rIyrx1i;#vx zucrw86lvfbG^OMp4%I-B9cgPNSa;MaP^TuBUKGG6XKMbq`<|}}R9wwzb6xiR84~o; z90qVGF@C{u+kJE);hGJY+3;oEM8QxKT!q@s-zDSX9ddq7oo1Aos!(+N_fHW}BdYeM z%u_JraoNBb#X*`yVaE8q!euV<G~e8seqz2v$Og#9u>y73ow=;IqUjtrl&IhxmX67? zk8&UUMkaPx4D0sZ3KFDR&W%2s23_uO$dSvW7Q#rMLhJ0el%CD(qp+ZSNj=M4jZDNU z3?cng-q<TrL+HsGs_iD~FWg<M6p?IS5lLxufPhsW^zQw4%)GnrN=4?vU2?wlWOL5d z?*52qxa1HIh;k``?hj(`Z--o6kA>!&*Q?a05g$2*kaTgWHhu3AG;lt>`ONV}W=ev| zy<?Q9g4qmUjY~=c$9h!BTR-u?pk&k{m~&$E6{uD86l0beGXy(%e=yjO^c~oWYbwF- zS)wTr>9woj*2!*pLB4kcUwPiXtmfOM<VJg7k+XM{`zfiw{Q{#>@K)~D+uX{cJC)mE zJ<#KBcKY25VYOjk+b~J*=lD5izAJzG?fU1u;l#G>JHNqom(6|3or$WNW_V#c@wZRs zL<vK=%PiS&ls%9tg{AOX)Lb;Be5PqQArF0TZ*eW1eHs@P5I%oo4K3-uPB`kC2h~FO z0DTwV+oIGRC?YzZJj*Q81MIFtOWNZHduAIAd_Si^@iaWGa)Jw|T7dHFcBIQ@@!h45 z>|{+luU#Hq6jJ?c!%?gGJO>af6=b9GWSq}ZmYD4(RJm-HM_Ag4*`K?!-v_|2(+t{w z2PH1yCn>-vO9;)N@nuDq4a4x0EZL1Fh-W(0%+vV_UYqI(_7u`rpZz>p1_$L6_OzNk zZbGlO@BH_=N|^xAE@5|n1Y6sSMq+EIfbOu(Bcdq>0_*4oiYumSB1Zw(xN?E|Zagja zb;fa188rglEBAQ!*215-6-Gr2%bye?5fVIMo;+N8o4Fa(2yzTbH5QXtBPk6Ra^N>b z_cP6_tDPFAXF|qUIWr8&^?IqYJYMI=5exQb9=oyCeaY$O;f%95HD79QR4WY%Ze?U@ z-9%e6tw4ACG_yOm;<vKih)Co{Rq6-EAEH~kYXB+m&bW2iIT7n;e&{aGG`&=nX5>$+ zJKx@NhnttZ{r%I%Z$UM22LWz|Vcy{{(}>O(u)AKNd>szw=5e{s|6~V~PHq5E$fizR zLYkc8vvU!A7v+AREi!c{v+*cASGol;%gY-VD<4c1rCHby(-+N}R9vU?34G`GSv!U@ zLH3qtJ~=h;KL&&J@%&EkTa@?bQ0sFL9E0JdrT>Srw+e`3>Dq8Zf(CcD;BLX)-3cxs zxVyVUaCe6Q!9BRUySvNauBWs2{{C}uuFmZ+Gu=$Bu3EL~eV>(X&|jfb35K6RgQm|H zxybJLKnZ^q8LFA{W)i5LW%5#TFj4e-A{s8T>}OFqM4hkJlg;w$U1rN1!-zl0{@z3F z69QJ=z5i?Y&@t-SGR>R6*E6E|do4AWs+w2GHr76cjS7~h^2LYgNmpBqq~%h1{xBE( zTne~pNzx`i<G&|-j*AjzM-h;L{JIk(Bk}O*+p{r0f<9L%eL6sd_qmwd0&2f)+RdJ# z$|gut)74t?fG<sOrX(VhQBwI^(9bSCOmG!dEn}dTryu?yy!uG+r|2EpcXa6qzE`8| zT{9qyW!Ob9_Wc=}QH0Nk3WXO6C<>Ji@Wwf83d!!beQ}8nL_|vb1O{ZF>V};JV#jmf z=j1R<#fv5~rcMYB;Qa8%JB#8)KZuwf<FhG{6ft_1FsOcK3@t1yY`J$a{V70>B%M?~ zDHt}M+!a}e=-A4=Y7%BCXuIph7fu;)!R52{g*8WbNkZTQpua5|SuR0@q+iB85-<rC zH}~K<g`-rJch4ib2RC7x@&$VSP*)im>&(KOTbfjj>(z;<)o#65&*Y|a<)-E^Kc?(J z4p;g09KR*R1el<FqGVvu%ZFh~k$|y1IKxM;cE35Rtju<H3PkP`@t=rs*9Y`~<|@cX z{RJnB)$z+d%v~hV#6QkF@9|C*ku6Is7NAkEFe3P6`WXd^pV>Lj9$;<*75wyvvrK{P zBhFy_!Z3(h%q;n=jWv-3l~R@6t)$v5DjJXX_-PEQ+ysCs>(_|AB0mm!{b0q|S<^R3 zxMZg(bS0Nyz-u!um$5~)+?m@V0EC!<K)Gx<AOz7{wZwUE%i!*l>m!S@rY8OSO!f;u z4sWDY-=eI$!ZMzj#-X?dGMF|Cs&AOG@nJ9FbE#HkJG1qbqt3U1=Yj!%ZgZ<P-Kgh- z{%CJ+Z!|cme>I}_7QSqS7f>!BLcXn^-TrA$t>aZ>{r!epQev?P^kVaVq(}C=ExLwy zf%$%ch=MyJ)$_b;Vvr4KIg7jee&M5v?E=Ey_m(ECUwU{g3oK;F-dsOfENQv1?Wk>h zrYEOb^txeJE}Bp>IcNZJlMO_s>v=Ay@|itGo=FedoUq##Lmmenr<egd;;(p_Eudw( zwY8Kk^GJqvi*g4#e`H}J`jM0|ZB&!B1HH!ioY`?5kD=p~dRG7W6QrNrL?uQr`{65i z4A&WSP8XZe-9(@tJ;5CHAZ{lR@d>--?47?4#K2J=*@$Dy=X2MOiG#(H`D!UYDWNqN zvkVb__#9ZRxR`SHj<2@Zbd2Xh!@-HRh#s(=HA|8Vd%lL-tQ0d$qC&!;g-2-=(WsNl zXO`S$Q^28!w0ZZCOUlYFJW@Inb`?En%VlvDIuqXqpLd>R@>w)wr9aXRL}Rh&zJ~C? zR<7)Ch?z1Nj#E-m&5aT+zEYKxa5WmzOU<V}SgkmnP4IL|-w$Y`ywC1GSe<dUgHG>A zn3u=6Yqg(EDkRgzs&e^`80O12a#rAM2}>;Y?|Dd|gP_0XLcbw+0$+I)x4ExbmeK57 zE>oH9HsQ!_0_fK2pWf7`g3l2T?|&>*D0iTHfd|p0a)<*8rqv8;VJ~%BIQ0&@czVj5 z@sgR#3Q(n!laAI`MAl+{PdlC+>Eud#uLFD8p5fXEBvDnakBVlscs;c;>4jMhRx7z) z6jc6H55{f{FvI<X;T^l3T>;Hj811%Eec8h<d1RB~o9TP!3gG%tqP#rMOctL64`va) zwyKcx|7zHbD}U%tU<lHF&ZCF!HOlCo_$dztR#i}^k`0DYQN@^(UYmUJjj3TuJU$Z& zXlIBn_zm@!%z{AaFXr_Qy06WQp^A46umL$V@7m%6!}V2fpPXQr;D3J36*hv{DqBKc zw)@e!{k3q%@Z#r6t0*qfLTG*%0ytAJuU%40NQPjxLlrfNc^8V=LSa!>vW3`ERA^dJ zl(kZ85cMoNB1<N5tcl31M<t7?Y!;g^23JqwHwZv*r-HTHb{g(y>UE53_kAm`aJo>7 zp$+PkWw!KO07QAHuHJ@$kDo6me4Gpxuks1llOaeN%m+9J#`YEXjUBn1u9p9R2|zJh z;<LUitUH<1I0>JFD?sbL3=3e>XONq5z{%|Ljm{_h<omQuK<fl@DQ9sXDv=#lRsDEb z;67zjhOu04Vlyb#0_N03iYl!C_<N2T?ka;Msix!GNdta?7l5%w*-(R^f`d2?X=BFT zMK(Fq`Xi22iv?NdAcFISO)|`G{TBO1FqE!r5{J#ajH^r-A8wA@T88M!eBD2&`2c?h zLeZAc=dRFmqoM5=mf1#0DURVROlhd;4pa3>eWruY*Y3$zLMAxBbo3>W&q^Dx7&Xgu zq8-r&67@X}{toR=`lfPNz5vo~O9G4E?q@~9m&K1e?U^UA?QiYnj%Uj~x9~cKtGy*@ z_CXWh6Z-ym_k?R2D2Wyj@1%&4dB3*pTNNP^@1b_#$dDd}n0APSG`!_XuVov&z!moZ z#vJdiMXyuCJT4Ve<p{)xt<X`9?L!i#Nee=sJt;r2yf52Rgg;o^;z{7grKG0$l1(z- zlJDa)dAF7lW^pw>El&73#9WcF@+wn&StMu`%S8U_Fdpev=1jI&Xrj)Kj#3|ZyD&>G zqiugpU-Bs+r)stiwwXB<pxo&?H9&j8dbP@Z&pJaL<LDap>{kcxZGG`Ho5?Mu)xs{1 z9b4o&<y^yiat0E;ojHBlF1L~Oor4UJl%}wMnDCfJsMpY<B-`hTYay{DNc1_~rFjJ* zMXX+gt3a;1jGI8g<M2+4>E$h=$g~at=hNjv1RvS8eM_{!Hisc9&4WdOG6|znbWa^z zHftq)B(2Z5paA*yS#;X<)}N;&Wmn(Zam=lds#=`dcaJfiLD!X9lG97I=E^@T2yAah zTQg-?z(sZ(e_21`E%_qB6dPpB0Ya)cy|?G{y^ts!M++oX;{Cln3vP<RO!tBIUB2&7 zaY{b7X;_&f?PmPNdaitAchGuODc?Lxo-9#5yj+^-mH7PjI;gnXYqWCYw?6*KNx=u8 z#~06AoX=GiV;4%eUzk?o6%G1NcsDl;9fS<QnB!wOO+inx&F(Y+ndO3K+k?EbL#N2P z0zT=e)u*(SUF6+Ht3Hts*!F5o@Ku2NEpP|)q-?<^IB04P`ER`cbo4wSDrHk2q_BA< zRb&WT7LLP|85WYm>iwa}{d_BxIccm`sX-Gyo1O6MTEp`dz&tpZ&9uk!?SO~WM3a=N zrp|0DlX3V$Kx70w?R(x}-avqxx`O>K%Q9+#Ylj-N9>@dx$bn-3=ZQOYIb5&tq@nsg zu2m(nLh|O9OClycK)=Q+N(1)EpkF=-PK`f8-_7k2M?Hc+oG;Zmx+BbSxxm0`r8>RL z=@r&sK36z|Z1@(cXq5%o^c)-mZ)WErK19m)Th(S3tF%6nW0hSbdD(`5tbRy*YFC<< zaVW+B<(EF4mAlL!m1Mt=w=A(9Wz*KLXMNI176nqyccN?W!Q&k44E!-aiR}G6mns~_ zTz;gCYyFXchfO$B*cW6~<fAgL`vRx50bPxy0gFMGm_HzAiX)&zXK17|%VswMe<gzJ z7J!|}lGi18zga8z3<%i@j9Sjp;$rJ)ELINyMO{7dHI5kAvtBh9O+qfcooXCHrxue$ zW;GbgZ7)%;uV@+^WK(PFi$Fqfx+My_1U`_y=tkjlSAf#4FL9>Cs!!OC_x}_K_XylY zAhWR%aO>Atu2gH07z<%9qu!Qwf0y~*N8}{ajFeOoZ%^#}3dw#UpM1FJq#44qQif)% zkjIDoaj7nY3<RcMrAgrmS|bV#?PJ0sD2pRur41H!_^@RL$O|hbw2P6|;ht{twHiUQ zUq2LJl9Dk#$B<?7SKbcQf*?#jIE7qFt&PED=jRc|R;I-bWhT#wbb1va5?fuaze3f? ze>VC9R7hRBiC&5F;N(6m6F5nW;6Y*=5KORvG1WQsvs^@I=B>G|b321drqt@bWd#98 ze8qMncA<<|<9c2j72^y{-Uze$l+Xd2kt76lqHp^+j~$I*9-#XpwNK+Iz9hM95uC02 zOFv)I7kEadaFdHM$?G%1DP_;OA9=INhwOAoUz4m>bkgC`GLRODMS|UFPCgwqlx9D+ zt=sIiv|i79MK26NPgQH#+_!=jOacH-=K|OI-;-?fok-<~pa_awE@NArh{}lih^AA! z1+2h4M(8@2kB+?hF%4awp`(<e@z7acyf%TW6jW6yL>CbZ+U`KSGfkv@bqXG@guW21 z(mxl9BKh!~dC#X(MBRXRAJ8e~$pYRVlE-)`xJJa|b;sIGL<4zlL4@fvhQVesPHKik zbEKK-GwqBFJ{Y>Uv}<;RT|;agj%Vk*>n9w-nn=`51J+&eqxVui20Pn`zQy{LT%+2) z{z+fx7W=-K4(_TI)$`|=EH80|OeO(+yiQ9m<iP#U$)7ffd@(Zou$T-p%{{*;E)a5U zXH)E!`Igu|>6JFV<MS;nAzy!eh(7;pWa?rFc@-$=Da#3%0HO5vHUV`zO8xiP8{Drd z5SCAH$5}yCH*M>!@!rH`dIDI6-hnao+{A(qSW&>XP|eP>of{L>&M5#%s>76mqQ3X@ z+~+z8gi|b+j?ZuQQ$$4Uh?@T^!bjJOy@f2yJhDw1LGew8(CWHJpkV`#&LB#PmePBj znaH|=t|b`H!YR`!Ou$VL&;3J6$pUNjN#Vw%BD3u%>4ID(fBCZ~dI4|8M?R$nvUDbk zs6cqEGT4ukX;?+3`C%0u`%9a(86Km7Lx5t+*O&oLxA`WYac~$7Zy3-IBr<f^RGOh_ zL|&zD_)r3RgOXx&#tZ_BKBky0ZTI)rjggE{i6yi~4WRa1ULq9cDJd<VTpTpAJHm$V zl(U^?r#q2>pDU-9)Dq0>=ol&q7(1SO4!gr%vPvuMb||D|PBpQa%p>+4jnRK$3hr^< zJ@Aw;Viy#<h?J)|;qE1$d*zkrXq40pV&l)U^wR(;2I%{V6IWq$d8`b;QFMmxYZ|mE zq;tvTG$0`7xcMMC^HoUs<W7=EDN4nvz(@f6KCdO^JFOlz9q!IH>lcfz?Lu=jD3J<t zS*Q`QsZiKlzR;t<+?-8p{TQ$iKL2)j<RGvtu451+<y*y?b8ZEr1a6<iL-RKePOts4 zw=rt@-bFBBvB1{FV?nLoKoqeXKq@AJ4R{jqQa&T0v|%v9eRMkULk^Ec=A%%0Mk8-r z{bbc>{iT0@)mOoHigga#6rjO8hdm&f^N_%0K|=?yYx8&Rex?9>rG&H+ne{|haqDny zDSTsEL;`Cf<UoerekgM?Svb(0`-%mdOgopTu!`Ed^yh(Yn#59IE&B(Mhg>TlLvEm% z%g=jN6PWI}=ydpH`k9JvZA8TLH|n{@t_^$0pr{R$%_Jh_p02X~)t@v5DWF5963$^j zrXmu5e%WvIQP4*cwC&6xZpM6&W7q2j3oqY6&!~C@q^s7vJ%*20i8YQ(_CVRb3{uy< z<Pxc0-KnbyCbav^WPOC^^k7*`Hnth40<mFx4w5q&9m8ofS_fD=&HUDZLa!^9uGF0q z-#df<0qd9Wf}0*ad-c$IAzkfH$Md7&XP{>=CszB1hQ7{yM5pJf21bdsRr?0ErQguI z`8wHhGSi#Kk?Gr+bl&;DtS=)s)6=*N8X&c<ar{m!N0|HT9j4*v?LaA(2H>{<*q||} zxtnG7Rhb$t>&>ojvq#*LOsN9EhIQ2bywbjMyU&Q8@!Dw18CwRGS1=B)`s|l2EsrC& zb7qYSIOOkJAbtEH?ch6`1RY>log3w9e3`xKv;)i62G}$mVJYrvD%S`<+)5Kl5&HWC zZH5C!2<6<&?8N@@lKrr@CZ%+}bWa55^$LYR%y{&eOr%7d#~*m~XHY)2E6Y>Sel8oo zTc&;edOO@kL>X*VIT$tYTm2)aT(k4&7i<h<J8M<rnX{pYpvsjN{{9KThsZ8*i-4Px zzJ*fm&p4Q+=LHF`rf54FD0!v!{+dlT4bQ=msaVDYj`SVrG1S~_<_u~L(Ki8%HXMXL zFYmzYP$7G%Rp4;HZofaOQO>B^35U-qhH^(RiNKQp108TcX1?wj3EgQsFtw4dW&;?@ zSK+s>S=qfuD%+ERRlej<2(XfgiVKDann*p-<4w}8kpBcNbUceVz>}y{A6?(!x7@BH zi}u!7-8p&UE+=PCIwOAw`zT^4hSfW1DDV4+PV1XOUsDGay)L<YdSU4=MsnN0X}cdH zWl>Rcg)3OPWjH^sx`;%<p-OekdZ7Jgc{7<}Z3@sN<U5eA@XIOS=(U=pC?w#1%u;!j zP9aW0#q?Wev0OnR5SHYjAsA18nh7@ZfxFQihP^U(kW>9UO&IeQ1cQB~)01h({o^Ju zdv6s1LIrD8?kU}uagfw6au7MiwZ9F%;TZv|tW+0U+;)4$xTHtGeUycH^)??Ivq~ww zI>aL(IhjH6EXxtZqCP<MrBQvW|5q>8aBdU@TvTcOz;|8U_9*2#Q8=1oVo&FSc^Fj2 z<FM+8w=_1%W=wKRT^u#?>i1`#7n-J&^imJgi1e?rGw`So;?HE;ksfBPTsNoy&LO&) z&D3BeZ&lhKtraH!ABET(%5gW57AYh9Km!qDrR)lVxy#PA>D;e2$MZSyyoKJwVXBJQ z9A)s2o}Wni<z~M+hlZn4$S01kxexnx3A3&sISDa(07rYTwAWrr?QYMOK{fcODF>KR z;S3$6$dC&*t7SjF8P4*gvitbmAe=A&xhTsml#<#5=pxyc*1>zseD4g_K^FYOoq@@C zv}+$5X-4=f)Fn3Kgb_C$bJbU<TnNJH7twH5WvJV}?VnE8XA+q+Y(u1CuT_izkEDDw zpRbiXKk+`@sKutuB^q`KIrfPMWe4Tl+blcI-i)j!uX)@_&9(+LQ?t#AVr1ET%vy0g z%yaTP4{+n<n`&+Kyj8AsHwQdvPdMi@#;G$N&j`$F`V|{R=e}vYkZR%srr(`vxG@K2 ztV7^%3t_>9Sa~gUByvaD)c($&dg-yL<>3rS)zA{anQRE8F6vi7UJt)tH)OL%Kq_GP zP+Z+yj_&1qkUl&*I$a*34hrT^eDgI40EI}v$|8AC_O$2VviC{O`uAy4)K%iQIG5`E zJV<veWig&eywmZpaJP7ZU7MJi!{-xx0(<d(csjB;5`N#xUZ0}XpS_+$Hei)y`Dg&h zPH>1AEr76aG8n&1|1_qHMY`4v=&fcIBLzh&EWXGi6)e7+k1gL`y@h0-w3~C3htUg# z@wvZ<5J9ZWl`ad`PQc@TG-F0qR_~YEF$(yRr|DRG5a`t$vO+2wn~XC}h4Vq2MGO^p zInF<}zn>Znj*;N1?fGuR&;*U>L)A?to3Pi9gzcDLZxZ26u1CzW*hrWi-_)9J5BXL4 z0#{!=Q}?-+I4f!kQFDY%ATS><h4Gi^&4d@B;s>eWNXS1xArj8Re87dSWzZ@p#r;Bq z^e*AxJY8=ANenX8p;&1-Cz5UJ{7n-G>inKmdhqw`A_9*^@Pya;T2+<?OZ`Yw_5L6b zukPD1W^Ah0oaqRhDxeQJ1owMgD5lA>cpG~*13E5W2&?QZ^-q4`@EyX+&Ycim{qaRy z$L8nC_IR~!B!sFN8)h*kq|>pVHwFo&qK|)iRYg?TTssl7dHbjn>Ktsvn>v*Y{Y%?3 zLb0#XDWtp7S0u0zKAZxOK{k&P<abM}^=4(h*C!|tcM1<9vKSG?531SRwf+5li#nPe z6j01TruRq1Y`ZA5(E=>vFMXFX%M$$lBz}&ZxL<!;#I{fLe%Sflcd$|FC%TdaT0W^u zt*zG+_$u;!mk*~B@Q&m{+}BzgR{Z<WBG+^ir5Ym;OrtA>l5j|+2J8q667^!T0=f&G zTz_ca9W=1uCaz}|eLZq+Tz)1h$v%HTlzU_@c;_nba7{Drvbib9zCc*%pxMyWRn(1v zJ(?=Clms(+9bWLQWxV--46OY@YH=sNnM}Y*P9y<1d(~>Jy}YyK_uLJAQoaDIu{V7x zTjCm6Z8OOZyd^BVmh-1u+KKIyWL~L434bDk&=nI?9Sm0!$lQGHe4KC)XJqZBDy_LF zVV)3Q<#V{pScg(oCjO%^p|%)cxvReR^znW61wvY-P6%2+#c+MO*}&y?ib#8lfhaUx zK>y5wvPU~+a}6LA9Jk`X&miBQGT1x3pqJ*=w<9Tt^U3N+GJ^S2y{X$2Qd_;M{K$jD zW>e~8IF4YG|EW4I$nSgZz8Xjs7mM!N!CB?}DyGq3UtjR6=)_#*F4_bPvh>4JT+peq zbW+8nv)Fs6Jo-DD;KD=BB1Z9hPHuB((CD{Tm&f66w+69xB^#Z@X~fZd{<C?86WavA z_8%at?15XsSfE!kwS{np<tqUs<mMZpgzno3N7M)?c)l@zyoEE&KZaI@*4CL}*i|!U z38Z2Gr=ndL5huXrbjC7bm7kf%p0-`1_d<c7=OsrmNKhWllDeCyBe$+hTeUn&H6ZMe zP&Z%c>q5tgC2|c?x@oUtdvXSo?~Y@wg>CUF&nTXB?sXqykK}Ni^}4XHq@1MUj=$K1 zk_))_G5Kt+oWhwMJxPQA{6c=`{*oW>irld|2ABGAG8qU<(Xk57TLd{~+G#t%tXakN z@#nX=Jvf##)PdTM@liqdjGYLCe82Gp0okfSg0&qe=9|Arw%d^kwk_5ug?wg3H;fsr zE8h&g_9Qg?GS|TLI&x{z1l)oIt-S?__2uXl-xSX2K;ZYD=NH`SGEOY$I%PPIFJOp- zyuWY#^uw9N%=-X9LNQUdfc%Dlhj9?bC;WZ}A7y!Hs=D_;ujkv=^`8~K94sTZu-7+U zdF^Si1(hPL;*-CuuGmYNuph#Dzx?qJm4@?ygI@j`Y<;@y)SQMD5TX5mC!W)(qL#AD zthnE<)1vw-c;SzvpNX#|G)<2%FVd$!-qXHS_h0;pd{_urO32`0xB0#hi}Ambf{+!R zf-IuT1m@95t^H+o(Y@$Pey08b2tbo;viLPW%s=wbeE1+wDk&nUlHwgyhb4kN(~vHl zeFC^Y{sTW{0Qqh8lw_YoC+QK@BW}yoHTMxUI=g&d$$7b<aA+5kJ2{nP^u4Dhn*aMn z3m}!OkaUT94U3Ub;Ep=4KVd(5A)f6k?d0hYI$$~_6Z49ZO5~(+D#TVDCp+={nWKFC zvx#N}QgPtn-rnJwl#5*Tm9zj$<i$V~FPowT{{H)Vw`enS)SS=*TH44l%`*104{^gZ zB(mnBahMy>ggu_s0X`hkH4{#@?7Puf6o2?m>I?}1^9IEU3e9S3gbegD?Q{xo;@1A{ z#dm$Ux3R0^ltP=^8%ecc%i_Qt23mlqFjBDt+2O$K7hBZcqnPWK&5LH%JK8&SH?`A5 zt-u=jDK`<|I4!{n<_}Ly&&Y4`Wt2sF0+bc3lmP;cH5$m@$MaypsnDBsaQo!jybi44 z|B&0JEtr)S5N5F%G{~g#Nh?FmM3o3*w3=eDt2N$0fvZAcpaizdqqhM@<9M#9sc|eV zb)vqHeqr%N7GBjkLEb6VJiwvVuZfKOWiSIaSk=Q2lhHJ~_@XV3w92E(OSPG-Cd^iT zuYF5+W0O**{^tG7N^46Td8{(B&u@787gm0szhg7P*Yovdb!FavI_I^+7p-TZw>i9> z<IOtBtKCd%>mRh>`GAi8t$91f!L47tcn6-3Bl3hL&{c1SY4!8z<qrH^X#Fg<kHx39 zA>)Y!pwzH9_dI5DoVgQ9{{31F9EWcM?---CIk(Ntsr%h50J@yx$qmx1XBH-t?m5y8 zVZYPDuMHRbg(GP;K=;;;KB~~Vjb~}BBz{kmRqr`dU^p6hJmmeT(MuV@v8r16B6lR1 zv{yp6UC&}&5H?$7IEoi63XR6PW6C2J&c04O1PFBGSdFy>+j-1km&{f#K!M*4!4{wv zo*ha<x8}SdS5nQA?8t3u`+1X1T1($}q$#)DlGm;C!Rqplf~PQr9)7JHQ?y=d&PUeN z_3{wJ4LVCT=@w{?SiUVfFxlImxs&dg&RDiF*ye?eB!+XppZSj-_k%29I2`($K2pk# z$20RB)I$GDNdsZfsoG+aQ`eF#e$|h`QV71D>e0d!wYK5*5n23OFjdw0Q6KZ2FtTG8 zceIP@!wva+iv!pFqB?Ho&rB|4e`*?{e>Pz{EPP?lo`!%Vrf-;s%toHbcNv?-ycNV1 z<`Esd^K_o&Zx`Cn(tX{E?k@7@55Lo1=ool@s4UjvQ`_K$6K!>WnPPv<KS$hv278%Z z2{<Nvr5W8puCaiUZ~#m~79C<O^fZ-|kG$=33Wx~Bu-BHyk}GlGPf*8+Y$MlRukKbG zD&PGe-jEn10`9ObWFcOZPr2nc#xCn$Yg;F31&M?leBI$9dKxXV6rbcOZF=vEZZ93- z^1b5q$2|5qlicBM@FAAlE%?u8>E1Ng53n~0=I2Z!vO~=O{(kcq5PIZr?N+U)>~mi4 zq6?RK3wk$A3j0P2+V`(dZ{CWpZHH|N2W|_aC(xu$8<o$E_sikO=c4C9T<)OTHdXn| z3$?X{o7;m{)eX)x)LR|yN~V+ruS#p3xr&a1IU}!1R?Op;Gqn<m^ZKVVQPBN~$Hc>< zf)8+MdjfLu(0#1g|FiSSwtZh{9X%o(?j~wITt4wQ_!b>P2zqKh4`gHbO;h~rz1GpT zkaZAhJ$_yIRKcmB#$O)&bbM&!bYF8XocdJp=yt2PVX=PV<hIV4279Z+UO9Jw2ncZM zPtM$K)i*50z$zXjGp7#RoOavPk`LBGIz7f3t#RXXVE+f@!?(=qM}Z6jr!Zgrzo=6A zT&ktJEBx8%KeI)Tyg&7SGcHyn1>8prj9AD2`xkPRboYe;0?OYqAzVJ-&-~Y3hir(W z{6D0m_5WU+vi%=s@~>_0|AqJczZA&7%07K8!T!G_iN6($V1UK=&r4!}i|(Imq+vx9 zKuJ}F%n!&fj?h1BtRNBaeNK4trdd#YWxRN27o~&=(1z`}j(Q%OtN2k`TDrH@6XJvj z`}qaqyycNS*jl+tux;ZnWb-RSt6ezntq^ua^oivt)6##}^oQ98Z|Crowec5^&YnHl zF{k|ZR>e?1b|~s9)}tly$F(5h3{Y<3A^y%Dd^PV4dy*eyVn5?as=<xk9dCPzcGGe_ ztRSxZqiJ8EIs$5m&MwJ64v{Y?*v>6D;Ey?JBsBAChY=FR>YY+kQK7wDaUOn5P@$q~ zBIj&!11vv@IpG1bRL&;Wy6vkSK6=*ctz>^aa9R}zdf-Cjd$OEHkLxREx|nDf&0NBR zWgh=HYRnmpnN$<CTgb6fQvps_<&IbJqipMmA<y{WXk2&q7kE<hi2vDVBEI`?oNDfo z4AFT~-V=Pyvg&d>U-fwr>OxlDSCXt(S#zh2L|4Z9&ukO1aFE4<MQ&3Z7a3wBCAcIU zsp_tSG=el;Q=uxoSDvkj1#N}SPB1id(^5lm8AW=)8-v?O$g3ve8ULMDr{h{;zfW!G z&)!~KoQEC(58R6!pr3Ljoq}OEoXlgEY<HE1uVNQ0h7Qo~0VpGT0T_DHe|=zQFK&N9 zRnF&<&`;-Lp{?%q0{bGPiGYQ?DJGWc8aQc~y@_ZJneX%(oR~<aQL8l1?0_NQam_*4 z0?MK{-W^na(|_$H1ffP6OPFL_#^s|~s8q3v*L$-^BJgNta8svNex0}<LogUL$hXe8 zlFQYpOeV-mBd?qP3s<qK&IHPqH+ARCb`oWF6-h;iRHS2qTM`D>6KoEsw?8u)_XRKe z`PqgI-Mj*w9(`j2HMSOscsxF8QojjG2+y7V7qLj}XS}iR79wF0Q57*|G>G0Pih+uL zNY0H<VLwAd-LtAuByzD`U*CE<j|FrM&^M>o*p0^bsU?To-y=2$Oe%FfD8n1{lB*t# zrg?wMh?P+$*FRfJn9a$H_Oin-D7(MLdh)l4ya}I-F#DW1-E)7US@^v+!3(N5k2d(Z z`+jKu?T&0Mw&gy%IJeO0j!3~qH&d~g;oK(b`psKMNDylmXLxvI@0H(^R62<+xw~7a z$@{grpinR7^fa7$R^N0RP2?r-0$bN(Q@7>(t)floT(SJC0ovQ^=|;4<D)BMr>OqO{ z+xcr|QWkx-_g&%c?*1@<BiUQX_9nq?0R4Wg&Gwt1)o53muh^gjkW)uCCFy6Q&*|?T z3#fm7b+Wor3AAmLo%6QsH=E70ZVVmVWXP-UxvGtmXD0*93JFo8!hcyMA5<sl)LUh^ zoX+#RzB=S%4BlIDH`~z)55y|?GFAThqt11INCtor2LYW&f%xKwR32XKX?pEqY<}FI z|9nHRJ~*sQ7)>Ffk@1Vu<4G);eePzLJhQ@9wEF|crbCO@d)QMd530zkUG%C4;-{r% z9)8q;O}!H=Z`mlFKSk$rd>gkb(MC4gRvI-Y!L8hR3d>{7<R^%0asD;O3%ZnmEBfzj z^_AFq-Q$Z7`)`r{@{*Xc(uvOjp{ksHh(?n&_K$rJD36X}CB0k$xdkZ1`(fIDZUlbW zl*Qc_X~(l68`V7D;Fq*MN*G(W5&N*zNxT)@e|vP?XI=8#9-y>_OSJfgXf)M8{n*zD z0OnS-<2IsclJJXGU48CZ%aI5#0=raMq;`EQTqOC@6uyXO+Vwf1wYY~ayD7g-$R+Ph zct8Ypt+-?lrjB|r(>4ovb3dllW#~}Tpju1ql@BPXUrGd*mBr8vSW9!#kh7c({c-wM z^ZlHz$>IGLPexV;tJh4jc@pqG?)f~<ylVm7Gqjd!O)A?PybkD>THbws^{-ab2=0rF z$5-$^5BL9ZrJ|up2Bb)XivYRr+cW)Nsv>8yr*2;jATAtv5tS|g;Fwa<YhIyIv40Dh zzkha&{AHk_02K{<zh^{(1Tf)X>5=p0;VQ))6>w5k?5KVWx(+4%nDfVn()pUX*Wf1s z-*A_MmlM4<hIf#jH%WZaSR6o$;*57xO)6KXTd3Tad`?Krq#IGIm_{?z##~@An8*7| z53D<x6^<g{&V_UWN;);b#&I+%G|n4v!?D()^lG8o9gOR!vA3*w^b|^0wgW0ZsV2wA znsq%@O$0n<p<&s-s~fZ%G%ct7P0Fa=A9bLyN#UJ9A?``{(_J!awFN7k3uPG&X(4-_ z$;7(8Jpg2^WZ~u2spPhHXnhLT1z5%ZEPK!x)NsH9KEtk;%wH*vwd>`7tajE$Mb{&} zcTSNs_@_S6{z$VPvP92uKt8NAE~!>(FQk~67lCl@^2X!p?C{H@9=C4Nxrtu<>%sZ( z7`GptU6t+yFTTS7U|PE7Z)_obXDd)pC0&j2xA#x>d3g)otYvfs>dt=`SW^)sNFOF3 zOc#(LnQ+t}8}~b9kC@v;R6$MjBuNRwl>=x1;eI7fPn#~DO=Y+PjwYe_k5p<)iO}*G zRpP8PqT#{O%Cxm=m?v9>gmHOg!ffj^>qf|AlGcMj&>+JAx-_gw#LMP5VRe%f#6Tmw zJeDQzPMRloi3Fd0hs#*5mXh0b^%)I^wNp?lHxTP?wy~%3%_QF%38fd%6Q?coOGKKv zup7!`xPp@ha~=HV0%Om(R^+JXL(8hqYI~9sN4j>oE3JGXl(AZ`HygC+Cz2CN5B6U6 z92n?I)RajZX~oY}`1$Mi^Tng*a$Qar?H+A>-Wix#c!U5?<nWyTM~yne)G}REnk7z7 zW}B@ak2kfIMk7g9aR846fC>(V0#U~k?ovri8wJ$BuC5tM1Qx3;7yvI>{OG6*U?PZg zJe-wO1}29X;{5)o<edR|2}QdwnG9wf0Mbeh5Ksyn7KzMO=wzm67zTqzX&=kPuz3Du zGV%hWO;APnBgDE*Wp|~XLQu_?^mL7(M?5moqOj0i1`tv(>rbOe^2C$crL#+Cx4!~r zUON;_OolCoeW?L@UX=^OLPL-H=MaE{Y5@?wE|!o{=52iOxa?K>+A~|6M}QsZs>g{@ zPd^9X`bqkwPY1w2cN0_q;$p#n6Y56|q)`ldP3mNNo$2b}c7D%0JlRG`_alI>jE$RI zsZKYu<|=LrmyoP>DCKn;cy)EP&@A_Vk(<THLDKH6AM^bsWv8)l=7l-6y+O#>rX+!% zP?NG6%0^26ktFr<z0oC0QKN7v>z(7MXH}j>kG#xjVKj|<ru2Cq;RNs{l{EjRaonSp z9$_{Sg#ho(Kth(0z);K+5oP{Jjt1_4fi5CoACm)aH*vk;TDG7j0P$J+<z3r@IO96& zBEw<b1i%sAd84EoZH;qX{xm~lsP%Mm4M5WdC0Oj~)R>x-zZh=mT*_Haa}e$`1RnT5 z0L)Jce2e%w{oAW+QS_^`D{2@Cv)Jk1XC}YB;2}N-1R@Cy;`R!36Lm`DT{m)(ea#oG z318_Y`7P*utVG3kRBpfJTtpjfnUKw)I=nuLf&NK3a&FD1U_4UIGrkPkLP4->XX}0( zB7<<0!gbz~oxJkf2xb9V-N@ebxXFC&yp^RF>B5-ZT8SX`kO@ZgXWI;OErKD}a~v49 zsFYfta&SD$SO<1)vbJJjyido5!j|zbnHGu0a1Fe!v?co{b$0WNHYDJaHf*ZWN^;@W zu-!){Cl<Rq@N>*=(vczu4N_hs7xI>MfnDnZZE<?eB0N!LF-=e%@k>Lfko6k<`$cl| zw-ahva}$W*8G|18XG4S=uL8`q2$BvC;p3nX8Q+UgQ>rF6<?e10!@acI>DgIQt9|Y6 zyTI_;(w4wrP$G$VRH~D((b-3FPIp>8YMN5Rt}iB3d=9En2L~en@^P=!hTPm3zXk@A zo8cQ6GFZF8BgHP@7q)*GL{2@c65#{dzp;Q<>bB8mdvTr>1`7IG<rZKwqcc$ukArpR zJYUTzX{}JeeF?B#3y!2VCnqNxI<7w%oA^YHm&NHJ*Ds%uml`+7rd_ot`X%4MNdsUA zyIS2%!Sw*$H#C|(6PKcjl9U1@{y4W4mFXVn-+9RLOY6rV39Tc<NV+hx%D#N`ce2)& z#_QqnB*@a`y5F<~uO)<Kt*_t53y|SIto%YCFmzefhFrt`W%#DHXC$J4Q39GcFlK67 zw3|pOK$rGBZgB?`=h!Oy1)*`+$L=@E`z|?taR+maqOo;(EGSIYo~T0>A@eBacQC;i z0^IrrX@`~~8C4(tQ%k*$D+48+d&_Sv`ef%Y2Z=UtlhuU>T?-FA)h*ixzo0Hvw~Lr= zBH}<KW~If?0c6hD0Z4lYyN$!z8=999=15Q-T#qFVg|3CG$<t{BLv0rdt9NX}ah>jV zH<hRp96LF)jwjlF?NI08;xhReO_!UT2OphAxVbzE^V-EJM8(V$Bfi&UqG4wXs_Mqs z*|Md9PNl43r5m}#4f)#D1cL+}`1CyHB%X#;(FMJJ;E?j<i#n6DkF<7jaVpHxzt2@m zU5(e34jv>f(nbm1*7VnneVn1+uxzEM#vW^K16J(t0JF-DRApk8#yP32M5u|e6Am;n zX{^&D5Jty#<dx_t;qFfHMn|2j<dYI}#joQ7sE-F9y3W_`-}Ab4TR7K`(=^eayGio* z`p42y&}-%Wok8<XT3T9tBOc*0nM_*q))xtrYaV?jBM9gtau`|D*L_!ue?(y5mdVz& z8p3PCTgr4B1zA3|jWt743VE(>VG;oab?K_5yN}{(sW5hzhfGcOx63pz0VxMIE8`!Q zP?5<1zLR^zT{jVU*DXPolBG8}DY^Onr{NHD&bgnT{}N4&IfYh*&1OeWwVkS}+kA7r zYt%ajd}U)BR@?BTsE|%R;ZYJK&aDCJ-AhM=lnXWqDHF^Fe<KcKd4-#lmqhv{KnVPk zF0I|@#tn>CER#By7&9a;j-%~_@l2yuJKS)9FqUEe-%%G@C(S~JRGS;D|6O77<40&} zA-YTG$Qh)DwWXCx{>yBk)C(%^o~~D867(A00x7+pQw?Q~-2Ayk+=xr_*BM6Ud=2TK zeO=ES2ZUw+S0OwRxQago%4aDCrap_@r;iX_h%|5wN?4@8mBdh>1NhnHpobB(|IAU* z;kO<85@{i^jw}S-a$*lrEcbG^Yk2SJlIcR7OE=jOSgSfp8{4&r?A4h&Z!#pU$Dgv` zD`{+yyp`P7s~Eo(LTSFQ68s_OZ9`IGWM8=+IcvN(7t>WRuRR@VX4qOPM~QdPv;GE8 zOZ$qX-o2TuTOX%2OAVn$5E{U_a36Ddphdy=oW8d^Cif_h@Xh|gvL&N^icHUAVYs`~ zJ&eOq&l4rhXg>*O$pnvu<NKZy@0FiP<V$ynQoq&j*n)Txmd0&Un1E<^57M_`0*x3h z4e~x8;*CD9H=*>5i$dC=6o$vf=;8c<*Ht(+qR#swexHIpX1mclKp<wB5f~(X+)SC- zu(lxIS3OvaHN<bHgs`_OWxP+^r0lIG+WgS_y1iQ?Q~O!b8nuuOliqpWK40=V-bt4{ z)c%Y{RXT79vIdg3&C{?tzSNjVx5@oBF;I}Bd&LEEu%fCB=v=N`i&KlZH=OYpzPDVD z>mG>`hcK8{f3kOT#1a=1L#fqZy#UuwaIIm;DVNSE4palnyEm8N(@F0y6w&X`R;kp$ zCcWlx+Qw&$#}h%^FQhCq;z7U;SccR2C2+MxjqZR!`=43L1|~%f(*X_=JZR8v2J?&Y z$5_ki_>q(}1EkBGh*8T_)&)VSku%fmpIwZUoKax>8GpkRn%L%ZL-*#pt_Gb+O#Q-C z=@<Oo)|xZ|d`G^iBGs2suIH$RA#RJ`WQ-+>)F;h*{cH3?+$+SfoaNgD^F;A&7Nk6- zqRXRfdCX~hE;Oa2JFB1ECRwW{KKdbjniC%`&;4iqY!zJfc~F0g)~fg1R3zbpN+UZ{ zPcJ}gklv~fvH!K3`v+=%*z(<dh-Z!X<VV`>Ieuv`Nwui>mqN70@PRjDNuGK76Gh|M z5+x<M`jNJNg0OLmIm>2Q<8(6gO7i_KDnG1b)2O|{M<}Il8zc)6Gx5ezu!XYXsL%nd z`|9hgpm?3d-(<KMGTbkXL5WxE{hFMf6~g1G4G+;QG-T(FEqEO-3U5`pV*3ya(kj-a zWlIaBdhV*yDw^?*SRu6CFQCTK^P!px$keu4IaLuhDL=E5tt3f&I(?*PDvMSi;fyDJ zNT>uyvyATqgB$ryEb1NSNv24xMO{!?^V$i_7DN0m#~Y;ggNkFm#T6y7nQww=#I|h( zBETZ1;`qsyYlfv(Wib3_RN~7^d|YfI9TdVZGgUBfrqP}IXa05?fRQo#2#(C5Q)OHI zvKK#5m~)X(5%2L3N^7U}tD6|SfviH>XuF^!&g9Me^2*5VJ|KE;zg&x&=k0QW(Ch*^ zJK*9WB6X*WNmK-+EXY8{s~k<(y))VbYc%4_kN&7yd5eTu%WnvGwt`u-Cum3ez$4Wu z`(XWhwbf0UUc0pzJ3u4k%F)Nj$VgLYY-mm=pp#PVXHA~Q5)x#zw7Gk562FzHWkT%% z!{84S_E-KOK_O`Dl}(Sgr$}5)IfwMhjypZOOUat=vGE8zf<9arhsS60*BKpy8bUo^ zNGBdXV_l%4)o1~b?7~_HSH3TQKTjMOfx%_l`b(URf^FnoTZL*RE}d-4HJx;~QPOTm zn5y|rAxVsVXdvB!j1kl>vRgmQ-?SE=c^Rqs`Rl7n%PH?H^%kd94bOC0V~Lo*Qd&K! z=~cDrqe&z6l6#e?^~T9OZTHkB6dI%!6&wn1IT9I<1A|9LlK{^{Wtwo|6C)j!fQMf8 zAwN`*{jFT`xF5TbNv5T=WgMA4{8ge_xt+xFuDfWX2u_HRl0=)Fb1QS1uBs^OZ{LNt z9JS08^lI4}jy%OHhW0chh~&aK%N=X0Mq>n_g-!|bgrmUk9BBermX@GIHxHb`3;U3| zI^MWlWqf9uVnNjbc4<+a#?{|vN%y?R1rN)@V+D?k<179#kaT{>rpuP9wrz%)mWjp= z^eZT7hR<&pnFe+pc7WxOxT&fqBl|T=FLDY6Qg9-lz|4D+5`E02u=Qvhw!EfL8)x~X zu&8msTX|w?+Fq~y#W`&pHq@;U7YB!oo>&s$*DvYEwU)S4FNt?V+wpJTzER^mJwZq< zW<kS1FB^GB$MpXq*U?p0CQ&Yui_d%m^S?4jM@L7&!=r`8;E-6#wAVS92PrKDx`71< zJ?+yvIwP~QvuDCy{L)~v$<0?$Q7utmcRAA@0$K_vWkWy&DXF_kmFH9`(Cf74?a7CS zPlPTcfoj;9#qPlZLK_$RbVhRWR!h#uKQuXmnhgnuV?zf`Ur+PJ+cd2m`;MLL8}pR< zx;kSRsc5<AqQ``NWjX^3UTfwS^rCScu%4iT4_|)fw%kixR4~$MEzhpfvP(&Q<%oFp z`<r7$cp>=;Hf$yVW*zJGF|CHwHx@+}r~xT{-!4&nd=P-Aq|egT`Wn7&a)KZ^F*@Jm z8j_AK`8Sn>CeDY@4omw*5XQMmu4i4ASlDvn)cM+c2)UHKL5(TRO?IQ6F0(R6Wbp;p zP*^74^HiC_&h5Uira~O$dd2bLEiF{cZh402nk2p8;z<}CjA7lKQu!6wthJMKdhO&I z6GsJ~TmcM3B*0biPh`RdG9)z8mYya2Jf<%xK8hP&wl1+bn~M9Ca&RuCQKbFnak6ZN zY>#zq-zHk6Sa@lQd;cESg=^2)t!SKXaqU_iKME?d?YL~U>~Pwujd~GsTfJdC{LKn2 z1KUGE6Kqd$g)0V`Tyq^vBVzus`%;LU7+kSTYwkg!BFSlpb%wt1zWKJ6u3TNM4uc0= zu1?S5nWV(}G`9aaI#_4I>uZpp@9Qn!nU|Tz(sz8kp%xdK(Q0Q?BGl_M!9)M=p|qT% zFpI0JSB_p~0kpyi2tl1LMXfKXw49cR+r-S!mu`l4`UE}4$L6p2-w9JpF2sgp$KYt$ zS30kNoPLb2xU6i}X)5^sY%QMJ0Eq_ZVczRh((<RdDLkA>kKd=Z(qno2+x_oCly`Pe zB*;Kc06IQ#U)|ioqOhVut`c*fNMsVjNU_|Y$>eU$>@H-F!+OPt!x@~ve1kY+y<O*5 zp}1YvBH=i8cAvLxJCGXn^CA5vQqQ%dl#<Wq7a<Jb)2E%Xz!NkSST8T0ycX%Oy!Gk{ z`X@^l5<~5cM?$2oJeya~Fmf8><?54HULk4Hn4?pZ;x4TFEj?_}QaDt$VZgUAy-tG; z49Z@n{i|HE3fqlo1`PeM@bxu4WVM>}(VaLA?L`1|fV#jx0rse01<tP3vw?KgIU&A} z7)`+IO+Bkojr7vltR5v0=@lW;Xhd<=R>I@3uKK)~kvwNSZx8L|1)6?0v3lo60kJH1 zZz95s_o+(My{>tu-m*jcMZ3r9Jheiby<<Obly?AK95q&|){1UXA!DoZYLv#aJ1%<Z zhnpNQ9pwb;==h?2b3xVW1B)9Cpy+|OcZMgOiPx#?1M6p>zxe{x-Rqb-c+s3*HkUT# zKM`tJ=O>-p9vOMGkwq&kW0tcRhaN5bGPKTG&1dm`Pl)o*#u|8T;QW6Wh+=u79-{6% z1&V0lmTZ##&3nZ#|9gY5NpFy#1(-<J{5+e}Jg@`++wpsae==5RY%VlNn7^+CkB5E^ zXn_6q%7HhC|1K6Wxx)WGoQ8)HKo!5k_unCsc>n)=9O1tYC<g!gmchw?a}*?IN8M7! z#g!Erpm*f-++6wm+8Q12z6%~&BwJdr$3!!t*;kxMG2Du-Y`<(4BoZ>XFQaoa*N087 zGp2w4kl!{Gtd9Ir=!ZoG5Jx}YSef=e(1Dz=&bXgFV#V^8w32bMQoA~@7Rcd#9B@c* zm@|S8UT|kH<5<<fWlEWtm~410PPs5l|NSyHypz*Y99I3RksBk>X~)0cABh~GH?uUy zTAahK1Z-Faq53HdXnn;z+&nIPFSV9WnEGwZ>XO-cSOVZ{+{=L)Mi;TIF}8Qnl-Nqy z^Eonbhf}6KUq`aB;b;kA^pS`bjeOw#x$!Aen;LlXeZN=~$Z}WT*Eh4Tn(zPIJI+qy z*X0ph;I>^s|NU4ika%^#`~De;bv7}ne+S16_kX|P@c*CR`@b=aVE*6G{%>sl8%*pS z0bl<=fpQxJv=(M&Q*X^3VPThzcX!T5XJ=<FoaArTtX_N#R&)c+AQ1kv{tFoJk5zf$ znK(FHI}G^g*%3rW${J@gP~>g>NlVuP_@>-qr}?$D%FWHqS!am+gek8%yzcJqr+k;@ z5fJ)F2flY*Ov`%lgxnmus|-Mwr)2KEJtZx*qH)FwP9XG{H9NC?89rohhJ1KKePMcF zVEE{@`TAUbJh0^7-VQIApY-fy9EG`33wmzxXuduFlnK=^wb2mLqmBf*6|nvAwA>(C z`OW9;$uys@l9OKhfY4&4P0luQl8;}t19?K!OE|#IdL_Fl`@(0%q^a~|>7Lt@a0`F@ zjT3ZwI&I?d=bS$f>8c@LNZTU4KW{Pw4ozH;Fn}Z7isy5u4c5@P!M_D8Eb#G9Pg~)b z43g%W!5Ora3I{zHtvBdd)N#83CH9~v_@f#(m{X8f8CXVM-jAj1jK2$G7VKRomEWJZ zlsY|gi<7K&8H{W`k`Vi=-FeWoJ`{UZ^wSe0r{_&dASg0s&7*s7bjjf<FWjJa0(Kv7 zXFHzMRW0Q-JTh6U(HimaI~X2nl#7jPLI&Hw=iG^`kbcP)&&Qg-Ht-dg1Lu=FtD({H z*cViY;}f`$Y9hG2$cpsmZGo$<gOAYs#$_fX^n8k{mG=pC5wt^W_b1gmkM{T0eL43C zf5!l&v+|EirDhZ!C$$Gj9w{HOr1g4pAy`hVP@~ODk(b?N+#i08alJo_=Xb;i02xoR z`3SLxB#(^dIcAlrqm&Qpsh38rB5i^&hS!6HjFv~~hwQtC=hLZj$Azv_J#c_^<c23B z8IQ}}eviH7kZZ$h>M2V*&@eK577?1mj1edp>gN^`ozf7==xtWe+uMJ{O2cji$$#SH z_k6wj2y4G>==WLjCbAvM%I`!~MTIoEQXBiOWn3cn(;ZjK?oDbf3IR5qH&_=6HaZEA z195#om~c>5u63*AHqGjj@FIvGz3(o`n^96zd%w{b8y-$ly7u}GpV7Nfsoh3yGWLv_ zb^8XX0^|j5OraLJdXW*!6fHe>Qo)byL6yeuZJz}%0CInvr6+Z7&Lo9FW^*v)Ep`p! z-r@a?#Dmg5&shu;bZ<fYz9`a<w3BIclsXy}ppKVCA5t9L2^_zs8-|s(?BYOudS&-@ z_|V0`%`Wf!1O?>vYB~tWBN!Z%wi3ppS(V*ik_adGsr<TfQE-!qTxfzBYBp1VnoOK* zdgEbJ=f!(?a)oKm$1nQDcE6n;o=Qy*F_B1tYE0?H!bZ4zHsvfeRGjjzONWoRUJ8(Z zg4C4x$?52HfE@Jb5VX=L?X=<T*Y!(B9)1M|7q*)y3KY{Mjsfhw**<bGga$(MY`;}? zy!|2Pu(5FSIav(wKwl9_UxAHj7_n@G7+6AatY%1Y;&*hp9|TP^+u7S;R?@8$=2Q)? zHd4$=0zk=GEA<0EgZhSmiH<{cU^0ur^lLBp^-C4#1z^;2srM!SUQ$7#xJ>iBBq06t zxjEPOGpyqCw&NrkxUX*x&|pG7+l%}|kU^6}kv0MzNAbW1JqpsVqYtWE)ZRJg;Zb4n zKtmRZ!!uqKR9uGVvF3yE%mqG_k$};sE_ps!e%j^JdmzmLaT9a6K3b}6ApPiz$^@fU z>aIMVkoQ3E_h*FO@Ep0ruW<*t!=s7y^_t@feWzonWVD*47+-#pZ)gv#KlRW%SlZ01 zhr3SzISp*k5e^OWbT(X|@}WbH>2K-@D1`o$QV*Umc$LG=9wxA8X?RofdG6!;p7apW zp~08jR%w(e?DOo^JZd@Oomx-$Y_<J9Z*uyoaDr!Nvt%=IMQ6vKn3Oc6>(Ej3%Y^5? zbsOjAhDXT<HP!jf>O+~8KIeSB>s@n>{ZQ>ne~mS_4`#OCme<zHV|dMyN%{j+@L<Ix z?|#!N9e|lj_ynbV%ayd&L?XSqI(8qG?N(txHtc!t>T1Pw&W+vN+S%$rr`MzjCHO<{ z!$4|QY{@4$Mv2ZRZeU!Jp*BuZmqKP2KCjnnSrz%@AXwU1n!7)>Mcp!fHwZgXN^R^E z1kUm*HqM=HZ{w<bIxMf&zu>mD?uiKIBE&VGv*@AU5dIhU;2EE8dNh_e<6>L|FgRPn z4)*DA9;%Z_LQ#@qNUt%thDMjJof0_SATPSrzB$Qnqr{U^8)wU;vre==7)vQT)Pk&U zM3GYiecmpVIh}qlaxnJQScPPRL!M95@^P^CfDOR-F`o}I!7C*uZzR@S*{9ZiuP!Jw zkrWjzgq&$@Jex%RmYA3r=Tjrlp33npx`_W0K`<mdnIU{8uJCGt|2?Sv|7q;3qoQit zwy%IQ2qTOl(k;>qAt~Joib!`2C5)7S$WYQDEe%5mh@>=1cQ*_rE!{OV-}ct~d7t&I z^?vK)4`!`l$8haE*S^l<{GI2)wJ1COV3Q<FMNcZD4rjL3P18r4I(5c<Yv2$mG^}|G zMZ;lSjM-yw9Y1=hLo*Y1PH13LK&YFQnI}yxx-GtH>TRn0;wezmm6}G+u}u;Bxb0}t zlS@5}vRgC#PT!-x%1*(WTRFiGMN47QFi|%7FckZyfa$PQrB$Khi<!GmpS_6m#byy6 zDuh8tILww)$1bxzcLFK>tnc6D!~<+JH^r}I(p?E07#TUFgZq<*P30nZI`lAvVyCZs zKI0msin<u=O!&9S)fRC(z|fnH^%G1LDBG~o{Cwm%`U4O>NzVFvfUjPC-Yfn3_3Ps& z8wd78qxIj4cDe3z8rIRXX%xB9H+{-Q`5HxGVz>HFECgd?V&3i@;SYEb<2(tKF<f1G zq+vcQObFW}glLEpCit#06!7e?YUWUg?>98wuhtLlUk)zvOgjVb#svo}ccY0OZOl;| zu5{)C_AuR@q8CHIX2B+9zLLaI)K1TIEv6jBRO`(z3XTh%4-!bu8?&-MY8Fk=N-i6C z8RCoMHL(_gkveJ%4MqZLS6=Uk;@-ZT8vO1rSno)}`OHD*J0%x-54dYs)8^yp2o@8w zKAU}=y`|F5dJyjIE+0mvSzj@5(!<Bcw=Qk`k`5^d!JRI)kA7z}osXtKyt|ZpYMA|D z;sI#+NOOj!?!C&O6_8V|qI_JZY2xr(eA&oN^J~-so;T0@_iidDUS}qol}`wiJjBo_ z4d1j-H}{Mk2VL)%2)#+xwUH_@Z=V0Y{v)g!(?>DOA<8>tvlC(7qJup;$0jx1)gkiP z=fONo2aBbITUAMSVVZlKkWpap(eB<HQFfvQ*Tg2Fy1E_8lnm(%M>H_C?EAJ{mbB>! z!22ibcjl<@R$RyPL!L2!RfpAgE1$J#pEvS)Sz~*S;^8fh6C`a!#YS?-_wt6UYz-2r z-0i1G-Kgtgao0{hIy!Q=Ji8+*ellGZP75)pvU@6P!H!SxlR@%FQ&|_dkS<kIWYN+> z&BflZ`~`o~&MlksmoEyH041~t+xNNGL5RRgwY5t;0dCrFC9#O0Zsl$z9`|hL>W^n{ zW@6&2-X)mVMc)l51_h<jd<+bZ42sU~7_s72LM7#S02^a`<=1KRov4$}K-)=TU3D}b z>VH<9rcj()T;x;?AtJr_8SRfyIZd<%O1}+G^U(Qm4f^RNEr<j-94<<1IY&tg^_Czu z7$adf;7Q{OB<XRPIakQYFg4o(5$;WV*MC8b+SZJ5Q&(%Su|~(42ui)Xi{OIoge2>$ z>#=fh5M8H1`EXv@_2IrnKCwy!eP&M|#zMjB-87y3wuo~W-;|gf=>!FaMoxY|1CoTq zARi#_QKHSc&8Gga3ID_{1;(Y*G-rBHR9B$!LNht4XM-imgTwWu;*UgjZM5nywbQ3o zo0bFpguaXr557Wxw=FExNFN&<)z9v?hoa5$;D&l1@IF*&LJxoEF3i`3lH9hViyn1^ zy5eV*te-1s=t2XlU3OAin6^p4@+A70gFRJ7KxM`RG~n})Bu{H}gr0`9y8qI-Gm@sy zZ98|;$=lk>?x_7;FT3BF3zHKUTV1Q-pa?juuw0+y&2}KcOm{Te60Hj|4|*@XS3ns; z$3p$=h9z(Plv0HZk6SUHdKiBPa@fm=?#?IqnQh!?jtfqld33?-7lsZx=$&@71pnnc zgFGMG>gFl_`nANN_uON*EBaoXGOv0NjeYlV3M;J?j#opa!`6Eac|uR9*udFGgLSZE zzLd5kq@mH?knh^X$*?d3tHQ62j4XHxd8rPSx;^wM51nf`iZ0UZF8t{RH)Ox?S@Fx# zGzJo;eW?MjUU@zbn-)7vo_!oaRhz<AN&aj}WoM2akEd!ZA2Yy{A*gYpK^k9%Xn#Tc ziT1vkxw*9*9q7!R+x~k6Oorwp0D8plY8$QHMwlq(5^2DMD<)`xt7<LEzeXz|zvw1u z<4gO|*{7e^YMuQkTnZy&nb46hEmShCmq)ft?OlI7DnMlBb1G6ucCpbAp}$^VpjB(& z9FLGJF75x?eBQLLZas=l`e7LqisD!1m7CY4lUp@GeobhTe{1Iu?FGnSEY&Pe!32pe zk-vCc(o5?cC3*^Zu4wuQrD8C#R6K}d`cw0L8|~7*=INZuqm)}22O5syF#dBL6J#yN z6w(Rr#`I3dlJ0kt@;4-isHoUN6FTH*0bndA<G5`ZUA~-9>dQaf{qE69(xN|M)nwVt z*m=Qcy}aHo{#o4`8!HczthwJb!H*mfrDt~HixZAchUrs-2Pu_W!1^&aJP0Bw!iEd^ z!J%t4*{F_|WHE>dYvn@;xuh#4l}qqm()v!CZyPscoCm9)UHR3cfR@O-u5=xXlNryH zRQAY@{0eNO)H(&-u4+7r(t37m7L}zBTU;d-WcluwQ61lsv}@aReJ@hcWTYqcVL~~c z!07LMaW?cZiQhUoJ-b3<QI9M^XdhO8E-}Pgf`Z4pC?>AStC-(+2fWcUVeJ^c3v46- z_JPahVVuY?y6jIRtKiunI&imenb9%YHQium4G64-uU;QGP?MG^UR+>1J?4;$$;mbo zi>|%}#zwU(7}Sg}-1=c?ynE(!>M4HC1mqAakSA-J>#O6SWPkiHYzDKvd%+Eaa;#6{ z7aIdDLw7$02NCr@pV3)ZUM`pt&R9<o_sq{ynitUyyXUcR5-Gbjx0Jz1HfJZ>-XhL) zY^0Cm=NFUAet}}Fzxe4^(rJ;@cWG*fXZa;GmMbv7elv+>A)Z@WEa<cV3B82v8y^;d zE4~<ZTX1_uFp@cDt@b_b<)_3Y2B{9ir%!B;5;Gr?e~`o#$%s*r8uHMtY*dxcV>qca z&Ww|hYqy-c$?%Ekc`B=+JKqa=K@1ox=gW`!8Pn+4afxCk243B62D%1cYg`L53?&xz zIIL?XHh#mmXUOnk+kJ0hJ4cci-a@Zvg5IuuPh(Ve*22QO6h9%Sub<MwxE<H>8T*xn z#v?!39JBPykm`U1zTGJE?Mu&#OZgHGTD25=Kx~2w(vsrbyrsQSL2o4SvN%>z-g(dt zOO)9xunynHDIY<{k3DEy=##nTl9&31MyyRL&TPT>IOMc>jtNNH4y>&XE+*FE(%@#? z2VGPZX_=l4BNDa!J;>d!vwPir`X5(~OtKo!51QAOuj5PLv{V#xHXOe^qjDt6xTflJ z4U6T{n$qF$&Z+4B{8{-I!%-aQIBU+r{r-WqzQ7mDfHLQbN!x8QuH8GpLpFq_Tip0! zyqOiZQCZydf)W0sK)YVWeT>(l&&rt+Bp@kL<FV);d)6YVYPTK}JA*kO9CA1p*7w>& z(Ptd50TXFu9Ol7d2x4rGrmN9o@2HZsi_RJ%b+?ch)StqOzSA^Nb$j;cw$E~12T(@p z+$`5i$?@3Z=iVaIQd4V4e~`@6bHd>Bef4`aY-)aAr<>cCYU`0KLgv?B#^*Lm)TXC3 zE$2$h4#UhzgsZ>jk-i(PQB7N7l;OkG?{JoGg+g|P?Mq$)k@M#dEM4>TrNm}dbX8PB z&PtbVFh`oRcsHVq^7G))g;GR-xtfHp`1^}0jR!)G%b^#CIpV`xX`<Zi)Hrr!Bp|Hs zMS7VCKi5-a=xpE?L(;v2a_PrPygL-6@CWzrJ-R;Y3%Ex`f$5>#szQ~P5Ug=;i1nom zJJ|A?!(P@CUs+4M{9=$4GxD}T;qG0sf)zakQfae&P{jJ3SfTz#Qg-8exm^Mvjqh}N z*u5wG6B{(sh-n3E<616#RwyPORqU|2XFWe(&T0D51GpHdFTtu^^OTj|8fG=o`l{$* z)=HPfm>_LT55L~{N6z;Lm26hpU`9P|O{|l^^kY*z8uwj1uWh3u;xFKK2l`Zy=!mxx zJUpe3oj+`Pv>XnQG|UMY+5J_jYM`MauZ-<7E9(wTWNfa2qIo<cOJlz^D$cJvQAb4) z7#9~^gmJ*`fq>|p^?G~u8hm=RbM;XCx|gO^fL4YzZ(Fo<M!^ecY+kyu^dpy-&j;e2 zRTsIAZZ)2#KRtzcusAMrj5w$-0xiu+0MV6Q{&qW_@om@<bFWt#C6+1_WDSCX7&0?{ ziC>QzcKSY5<s)?%tC^mP!zX9WWP~_loIAe~l`e%xibpDE&P4WS%($n!-#Z1L@^70? zc+yFJFNXkKXC46!)VTo9lc(L^w#V0)xX%|J3kIg$#Cq;_EVrI4L^=^x0*CI}y&?Vf z(e+bo73Xj@^Ooefgs;zTz#KtE_{K;6JU&l(mM~mLcdl0y=W9f{ww6#PK%42dxtwo? zJ=}PiIs<l3$e0%n$2$)98p3L4{BX~J*$kut)xLu+G>nt34lrBWS5M>!ac>Gjl{csy z%e4G6|MJ)jz|yY|@o_zbD>)?O!rHTY6J`05B?!+7W6B;A`*B&~O*PHI0w3Z(5VCrZ zSnhu49li3Tv&1PiG0U%C3imdIAToJcZ;O|1E+%GrXdMCtMF{za@`QS{yGq!V_)2bq zo96Ce)E$o5yR!VhN&wxmcB$ql(qx{T9n2@M09--khHa4Qn9?cpzCfGu*|4L>@WC<D zDeZn7kLVFY!6cW~(JeScnA1M``byxMmIk^1>Eg1;gAK^mt2e5%F|;st1)$usOcLv> z4J+Ope5ybHwZlj+{341lyMy0QJVjtC(Gz-LzDQ0xj2K=nNEG!~dI8A}2@U$R7oDAF zhhsd@>{tpSowuNGz@;OTx?7QXvj;I{mdW%iXhxNWbYb^U2!8}Et^KO3=<;aFlU5da zi3PX}C{$wa;W>Kl%+$*7FeHPK_G-=&(EZgtq`hv1)Fpq6ijU{j6<W2$THXuKqVCD1 z(~K-7!!|4#T*xGoUyYv(Kij-D`6$&ujOS|V=|0-?5`9*Ob<^=!1ng-0X7rpI!;h3b z37thgk$v<$DfibnEx(k4Z^!y(ozB&6<H|y*Q@|$IRxk!HKY;y7%wmhg<CFYxahV;n zHt|+ksVNuiGO*0^m;(`qrTB_VA28vO2$uzeOe@&|)vr?f*<1zuK}w%>u}hPhKC}lH zqdKp9f6C4eg5#|a{*zueRL7eGh-BK&%~JgfEa{Pczg<D=8T!i?--tm4Mm!Mg8CrIr z2XANDEdA+Hh%G#QFMtk*-an*5ie+%RMug)M@65OHvzT#2mdtHBxI$VYc9%`Ry<HTH zNojC%tEjhM-2Eish1&-Yq^-<nocNu$5UoZpDy5Cgra2-SKxzGFVF4%;3%OjYYX$VU zSTfAKi^Uz<yx(7*9RO2qwv@u&+Hil+OW<q2ckf)e{_JCHs-$awt|MiOfKi;b3%mq0 zzbetw!mJhG_d?i#6f67aL&zMt+znH;t=4YMw+}gdABF~_UFE1T2yAU{V>iIpx0Cnn zecl#HIJT6GFN6P$z%8+PQaaO~Zi~E%DgWm)!X(g^k$Idr_TunCC@69-&W%lay7WP3 zr9G<aq$+&$(V^*B$<sZ<j(v7{A$^b614glokQecS+CG5>V!Ck8eMzc29}lE=aT+N@ zCEBl5sPS%+p7=A{hj=l|DG)w{&<8G7PsmTl7Q8|YVQqO=Gf7KhzN7{8fN><cy1E|k zevlx%I^IJQ@y+=jaZkD{%fwWNiIH>tK#a*QTjtv5f13uTK!YFIH>qM=q$aeyXh)Hb zoAWB?|HNWs{Hx%|J-^e2xk@cbs{ZE&_sA<W!NU-N0qn>dSm0+t8R5QZ_fYW<9Vs7( z59df#Y3-~rh)p3s)~jYqdIlo%&^7~&hPy{|_)-wbUEDf#9rFqx(ok0PGYO-#Kxa9u z4Ve)`2b42*+C_Yz#&ab3TGi`o)3#7-ZZ0-{&b^N;j5=rywjaS4X4~F7aH;&gPbMf# z`lfqsHTY5Y_QDM-H{&+iWzxf!hPUfjmsJ6hF2+z^&P>gwrcX4N_}&7MdLSP62GFTl z7I5M++i*5~+TAg<;a}0hCy`CkWwVfIG9#+zd14dqf^7Os=CGlW`;ZI;79(nr53Kl7 z1$goPlKduCs3~C-QO$JjD_NtQ_K<NoO0Vwq$DYI=5`LUhMvM}V$XTCA#J5~x(NJIU z`kL4Hu<KVeomkDUFG@y88xb{;cAYJr*i(C+nUv&D)kt?YpP!v}UtxE?r@>O9@Onls z*mo<8GZ8^*g?u0Qsj5gy%4I|JBL+Lrj=A^z%7O@fCc=c^EhFq&UqIH@xG!0VpC_Co zxX-yJ>}+z@U$3%pj0Sfx7z2{|%t?SH102(&+>*v^gWbRNo=`c(IZfR7w;fdCufT)G zZvmSG!x%hBpWaf9+H#LejDA~qtH;aBhm(31F~YJzXs-q%4guka8v(D$chkAs>B4Rv zU@g2iwo3{*cK}$@V=K3tF`Mmk3MW&*t;`+xj0{~A_oY9~g<8>9W`I0PhVG@#lotxZ zP2zkCIBuB&@@-9g-OY9mVd(PysP$yHvBxy5?({`>S!UPJ6VKCE$+3kIwUL85d{%Aw zr@|fJ2|tYoE-0>!R~bMQitBpY#!pi4F7sk^d`v#9KRySmX-*#Hk=sO@d>8IOchJSk zdqi&xg~fZmzb3jPLc8h<DSSSo%Q!43)O@#$9h%<|s1>SYSZ-dc$8|2rqIN=td$_bj zHK?-EexUrtE8t$_VtrBu><6x(M2Gb5&2|kfHF)0>DR~1)k*2_Lp#Aqox2>t^;;_3; z6Kx3rL#8R;7w@~eY`xsHq`WWJx_?KflFXSHnu?CB^@AwYD7pl+>L^!uTI~^sV&rs` zl8~l4J0b<MC2$M@0)SVlPz|Dc(Qu!fIQYua7p<fu7cKe3%nLTT&_#0kcbQpP`}^>& z5v%<pG<MuQKd%7VuKiEeZ*_6w8(HL_x4a>t*%*A>A(g%k6A5kp9`PU>P-yZMeA>lf zWSYd%0?xO2JABAB#!mF^w1b__ekd1idd<iMRhB#nyK`(6(?p=%`1Q#=i7-i+yE6-r zui5`fi|bQ|XGSaC+*IB#m9P~qPQ#=<5m4qy#R=u)A|#jju3KrFH{ctEidTlv-*^@( zH6o3|McvW9=me{(Z+7?fao3@u&7CBTymw4Ly{|$33J#8Y)2+Fo=e(;PG4EeTrq0aM zySH;ls8)ZsZiH-jrT#juzg&dj>~(L<A>pgphAw)_ze^!S?L~x;iu1^cq80X;N-XS? z6T@@7ZOXXdSO<<ZF;aI&$eS2T#}uwgD012?4{h~S`Ghtg6t%Hgp~qV>w_@TUA5$;$ z3%iGepb?5DKm($tlKW&H`i&)Mx{HsZOpW>2d-Hi<;BGWIMUwiGR2Dc;t^?u|L0ZG6 zWP*aX&U++_S%<x|IUH(Kho<1+DR&Aj2GBf7wLRX*`^B`EYJ~qRsL3!q=;LsdB?+fN ztg<)_tg{oWuBQ9lCacazEdjA`mclAr;7Ds?Af^_;EaKF45#|aae4VRo=vXu-VrQF< zuoH{x5e+<;vfQ@l;SH0t=%DT4MfjalVjhG1)1TeKsc@pGb)uH)8HSrF(l6d6BLlkD zkGiC1w)Y5~f&w$Ud06<*{ZN6XUH#&@V(ec(8QIbMik3OqATy9Bo#=a}7i#CPGTwnA zo(Hac9<f>R7aB`&q&lo_x~fH1Sh1*-F{BEV-Fz0aP^H>);B6ysUD$Hf&~UQ{by9hK zqN`moE&aQd6$>;<y7_ojB$TA5dS0IZTd=nB?6r^)l1a-El5xm>VS4VxQ}QWt9^ana zL>y^bL$g3We^m@xybF9=`Q}2>b|kLd8V_{?QO2}g!RprKFn{F-<{q;|)amZbAdjA8 zUPBRnSL?Mm*2SPd$^M$h@75gO>aY-m#I7sY3En3y_$Y*+?U<8&H#DLxoJf)x$AxZX zjd{Qa%YH{EyF4opZLW~v8YFirr@KoZ8u{XS(eu*heu`LLmIJhGe0{^#U_tb4U5K~A zYeP_suEa~YPrKY{z4&QD&H20q`BLKI9MHR$R6FEGr;f7i>1NVX9;-IwsnPUXD&tVv zyI6=fZ0M&-$f&{)35w*xU?v9hc^ky>#8;^=TOB%rnz#>TW!t}$g6cu_49xsL8=sEZ z?zz8s=&mlOOe!;GR3x(LLCZmyn04wm1}MS&it?HPhg}V@D?4#@5kACaQN5?;bvP|v zm!8mf_D`-M$vw(0w+SXVFx#V+=L-f?b-WVYAsm3(3bYiaJD_@?2=tX4u6XylE!>wY z@W!p}UKZZv;OGq=0Ge&LKE5eqI8EVAm%gxBp+rfk@8pU5nJzHMO#E_c_L-bq7>L^b z=cSKStJwrOLN!!Jz5@G07OB}sA;`QJ!<TGtrqb{`sYzyanISemw=K2{s#)(zNBky` z{jrfZP8j0_T!PN=PTx%GCA>a+7(?vq$xk`kiRDfQs_k)ch_=n?jXVuWQ78RS4EDTp zkZ4T6A1JXKG64S0shj>Gt#reoAm0m-b}i>Wl)<QIHo19=6-wKDhb(-YK<2}$`(DX7 zJ4)2^Ti+4BB+g~tS!A0F<o3@@_wD@ZkTRIO5k!ofr{2FASmUQ{sKoV=oYc&Y)R#Mz zKgDG3xOaPdiGMg<<uZES?d^<F81jZ|uhyv1htAq>O*#>~3>E@aWuJ=blCsHMaUo5Z zuD;G_;<lVb3#6&VBDk7UKlcMAG8Zr?TEyfJ1>bpLPPXjchIF$egb>IiJP$07_aRs? z)H8`XQ`=pnwYl6E@Ek#P#QW6OkD~jhDY{1<!0bm~p^o>&0=q-vsyb_Kg_x!9^=2o! zwVAEqi5lpIu#_&ZyJ3MM=tzSQ47v_&dSFCAC-3spj_ulYRg<gNCt<r}IZ40pQ}kK} zopOU?cAHfS;01fX?BBfEW@|-Eye1YL4C}D*edmwpOQ=_nRXT}zhi*(p#)tF|PM0n| zQX0#b2t<&QE`PHM=}FxAf)MRS5=hZrsE7#EdE6A1&j@ifUm?FfO2LT7+8rd1TR;}7 zD0>e0d#4W+;|hkh^0jwZ1fIPCrlyy6&v+%0^?#ZLO*tf(r-op6<_gCXWiY0y;)H_F zs2fDX_wx<id1adzjzv?1Zze=pQYgc0T`{#7c-Nu!{*lXYt5d5^XV+R%^lor9N{j}R zlFZd^biQqBe8=C{PRMxOrgn@O{e93n#1;J{Gb<qAl=-eyhwjtxwF53%_!XAG>HbFU zI#If(ber%QN6SMl1sZHFR<<20njf8RA+g`uXk8R<)adI>3P7)479L0cp;35f<BJwM zBBLEn6zYOmZb(uJ-YE9>f7B_8RFjWsqHC<-?PzNL@+v<dZsagu+TlPWm$UL#_ulE; zfI{!*CVcj>MrQTUkyeF+`Isc{XrJm=VPqC8g7CP_5alQzt2N<Z()NJ3;juAi7Ira) z$n$#-c^*m9n-H!BYO`&?7@jC|>PbN%u##|8s=!6G;|k}hrB==E)G)@uT^i#PIdTGN z$?8dNj&iwJ8jjMEHv5g?MWv0eC?-Ag^zq?u!hI9UDgRPts;5lzy+%bWvs5+}4?i~+ z^;W>(7O*gR-N_bu(zd$g8WOwuWe1?id$}Y|ACh(vrzhWSvDO3U)1YG5z1=diYBHpD z_{#8~d78s%hSCUlsG$l&*0#w@ySqSz=a0ON3`%CX0vd6ZFWyj`9H>(B`x79m6%>}y zb|jHxdS49dNhga{qMxwS7&2&*Bqwv?cd~u{L*4C(P5H22Nmq;p3q*vL#o(-a@>nO9 z;~pB@#zDb~BPDBq7W3-*mdBkEOgKs~z3$$b?dFsA{3N5ZNt?xuFLs!gfaWw(cwsBK zI3$Zx`9bo~GyGNCn=Mp_34-9mOSF>b>kXH5So?|z(ecK6(l5uwjwkjRgZDv=Y^Sea z^YY%yvFEH}`qd6wR&LF+J)KWjS3<?_41y1C(Bt1RL8o?*3b|Zmv`x|GH^GGLOC-Rr z8L1Q^n*+kv_$mrLXCh4#Q@oU(!#&cGY{#l57u4`uIvMva1R>i#cJamH+U{Rh$8m)0 zGbE-+q$pY6Nt`s>8*vH;1|3hBYK8IacKEhl-k7B_leG^wrqLFve!nB>)#QPSW#+qz z)v)Cf6O%DJ>2Tj6`^c(h{ABOxIb-`K35I>f_GRo9e4d5FFjz^phwWqiykixLyWqw} z``bkqohvbjV{GO#9M&&slQ_O=loZ>aN|%4$=x^{St;Jr}gMQvRV3<0*eAJsCUQGq? z=8Qso`lXhLStq*}tXhSb0UMyy{@t61(ebev7yUUmpxkOkBznz##U|`-dx%+woQQQ8 z;fk@S;qlyQtx+G?gC~_yQKIF{QXnvxj*gD?DgbDT9`u2JIPA{*WYVRCsMsuw|7AJ0 z7Fb=vt~X=xUfB|Cn@tJr<-D`Q3p@Y}2#_QeW;C(89a>w~VN~L|mu$%U$N=4MtfXHJ z(sl0f^di8)b3aHKGY4JL38^^8N&b11!y!I#_FA4W$BwRM9zhB_Ij*SNWEx|u=(@O` zowz9<mA3eU{`V|x!cd)j1ObT+%~Vr}2o@FYV@YU>n4K888dv+|>sX9g!cng{?k-X0 z1f9|Z%@1OnM(52q?e1?q&W{W5f_=(<<r&r1gvWB7NT*>!s@tllE9w_@Bz@av3(i3Z z{|4d`JlYCB`L@7~lF@})L=Ncw-D!!0!BzKf!aUTBR?5VvvB6&Q2yQfSA1hKZZ>xUR zFd%kNB57hKpvn*`SmM#{ZY0z{4l2wl=h)C$JYY`p6*73~{Gjn+46T#i0;hI>1D}`N zZw4M`)n`Cl>TSb~2fE(I4;ug<-3hTX1zi;Qyu5`ZPu#EG+vB*=$KsL_P`H}PkAv)1 z6%p<^b-xHlE!H;NP)B6nC_USaI8Wk_y92r`@qNT`#zc*FY;~l_CvVp;QqnRv)|b?V zjjhaMHxE2I?w<QGN9|Q_!_my2zoonS#A_n_SJJ}uiOAU6LFV_jx>{kEhb#D*3%JXD zPaKzP`m*4jHM_Mom=KzA*fwD@4N_t199tYLTF=z(+rRLx!1UGiVP~~b(irz)ihN}S zIP4Cp@v6Xme!Lj65YJew)6G>+G<j>PWyP*v`g7^64avu?lu1l@cxn>uXU8k>EpT@+ z&BjE#Sn;Op>@DnCYwh+R{zjp>gw})CDFGf#@h+K-71x*LZ-#|xowy`19U`Sb#KG^g zqjD6NYH3ccNj~?3cy?F=+4KMzzW9XPJ}mOYAp1{6A9tgIpj@WkulseKO>;)$hWLWD zO!#<=w3cywg9{|2^gl3i3MzXEIgQ}ZwDTEqz6jF|+$r1P*v8meym0~m1B(L`2q)@f z0Q6Z`K*ltd+?fWJr*x(t8_$LtW4%dWmDk<7r1(v9d0xj<Up@wW&ld;jdKR1A&<>gi zaQM}ivY(T2(9JXc-Dk^%xf^u$gmj(9%dBHu@oE3SX~tYTLD^ME1)rY(kIQ+FRH&4~ z^+kN0;U6H}H<ZT?r@e&BlCY0t<^91jf@9eAbe*pNWt)T;N7WK=osm0TsK^~Jv?@A` zQd;Zli^~2ks92J3nF8A1+<yF+|Mi+SWc$6o$MgnhB80j0^(U*}30aT)f8k9tz=Iw- zmpuPY>RoexUE~sA7$sONt^sleZpjE5lKUV}WP{=RQKiOjpeZi^(oMqpv(AoCSw{S2 zIpy&Fg?a=2qNB|JA$0#ejc}1Vsmy!8hbs<LGeRx)dGH_5)y;7$NE!PY^kPgRG=?j1 z>#dWE^}t58Y5LzkW8d+39=DeTB02={Q{Q@j1otCZ0BwCtj!+tpuDYKsM;40agg7T* z`!)xpJ@^NEeLP8l*tH66yy^3s+y%P2Y;gBcy0iSIb_Xp@+fq+n`ScF+J2V*;y%nEW zacr0T=Xf8FvnQgfMvQz$8?A{!-Y<F-|GdpF2dV2d@@D4(Fx)iQ-*e|rgdlSPGM1yh z*W>y>SZ!@b-24Bavuz50hs2fwpof2>{k}CH{~Zgnzc4I>>i=Wh{vBho|9jj2u+0At z4tLoe=G*w9+OyuN!Rx`_>vrWTWW4Kv0`O<+>)&}d--yaY8YR*exx1U|T-P~7zvm+8 zzNl#1O`rM^6C|S>O&1T&cO9K`BnzDMpZ_GHt_z=#T4dzr8c@1``?kEi{Hcpe<>jN_ zJ4(&G7JkC5by;_&dMq~5Jr2Ox5DM8f(4X;tJXz{*L=1X0ajsn9|LYj_XOQJG%WDS0 zha@%HW?#SYxGMd3NQ-`=Q802iU;G_bhz?`%c4d>!KU_AjSDS&_*}9wI*M3yVwaDM= zK4k;jQPj52U8(%{nmX%DDW1Q=o=9t={j<&APw_u56clpC5x?(@zvuj~7dQOS|82M9 z2c&+I|8tQ4-hGRsIH$>|-3R>hzYg`^8z5{ZRgH^Fb5<z-w~v%`to8@B{ym)kj4ALd c6mcbPG~^+A7%F>u1NeC=`$DEr%IL%Y0bV=B!vFvP literal 0 HcmV?d00001 diff --git a/website/www/site/static/images/blog/apache-beam-flink-and-kubernetes/job-start-activity-diagram.png b/website/www/site/static/images/blog/apache-beam-flink-and-kubernetes/job-start-activity-diagram.png new file mode 100644 index 0000000000000000000000000000000000000000..3e7ede2c59f9b1c814fcc5466eb9c6066b3a80db GIT binary patch literal 177442 zcmeFZXH-+&)&?3P2!aTrfPyp?MG&wMkPf0Ey>|jO6at}#9;%3qrUFWrUK3g<p+rCg zM0!H#MVhpP9v}qn_C4o(?>Fu@&N=?R<1$9bPWGOAtu@!$bIvvA^W>Gjt~x6-4>Je^ zV%4~R#}EW!q62~G>zIxL-%vW~Jb^bl4@33aAXL}61>nOYJ9CYvIyxXx;Ft+S7vTV+ z|9uJY;sIWO=(`snM&O+Tc<HCpfsO$0bidE1ga36U6J7d|e;w1;0p~%gMrs-wz`N07 zcRM?04|^BS4*MfQfZ(8mvAL(Yj<(!m7r4a3CoYfdBz)kmzePX_K61bz+|Ki%pby;1 z*+b4p@%&#`$N|T{zm_~N_}3+#j*92ab@T<*T-@yhr6sOOTsyDCEGQ_b;Qr*PoZ+2& z{}cy4DW13Y^mLVzl=SxYmhiqQ;o|;G^17_7tmL&Dk~eOM16PQ9_&R$&^bvRVxbTmg z{Odk<>^vU3JGgo}xHt>`zVE|FE(lM>^XGpz^gn<9=%<~J!{04Ad;D`)zyKwGpOL&S zaZU1n?hS}4{Qg!>-@(Vu$>NR!98eF?hSK#L*Ka8NCGdYv{oUmM5;gx@^u{$Q+5av2 zzfS$XMNK^H+|^v*K%<^Yf6vW7h5z@-e+nu{{vP}P(#1a%{nxjErj?i#B>!jDl$ccs zrRPB)6_CaqRbwBz<uS&9(}dyHwNv2Fl$?yv44bQiB^?;s5tHRdji|g|oK(k_7kr$J z<Nj)Q>tEMDYz|zh?=eo?eE0k!c|Xl(p#5p(#V0bg?c@NZc9O2*M#|8qA=z%(+y`CB zvYhbY3b(cV^mM_%7^JS0g8R=04|H<%D$c#Q5DWq{3I3OVDkdS}EZoB&Tj_~AX`-T{ z1-dPl>Dg64bpP#Nn|#QnS6~naYj>^q#peIo4$u`TeLCApvTd<&aH3OTTIr>YYvTXC ze}LQ*8$m95m8rnFQi~&hB7hK|?4WKXmdJyO@p@;{LhdwA=T6`J6G^5gx`GY+N&_4! zbEfU@PX2)wR0Q9HmCghU25XC!JN-A6{l4FgtCL4S6^Wv%psADj>oKH?#ID0^%S+RL z)L6O~STxDS(8ea!ljO-leJ}Am_D>Wqs6P*0v)ePV1osU8yqvdju0-Wey2_qsL}xo- zE6$XzOKfWTlQp3tm<d(_sfd*d2CoLcMSTkYvn6<s-gtZBIVkw;V441(bnJ!%yX{<> zsW)it=C1W69#ZSZ>pxiOAn+&d)me%I1KkrP`69aiy1;+W7VH8rTT(7YI1uYq5s{MU zDN3LMnn|$7hyHuKY4zQW@7BJ)cc+K!E2TaMqgXqwjs_JopVI9gRKIGYKc^Z0LB{K7 zrE)7nBK_wyxSW&+!}H)vV@6t9VUCmg*{D#or{U#$(h?yVTsqbr(z{53k_TOex&GE1 zU=I6%x>t?_>F(%-n#SvV*5&t^1offmz8$xT84#Izg53?7JH)~8jp~T5WY<7(i|*Lh zq4W2}*LP$MeLqDlOb*$vYTg#KQRz0Zta4mrPBM+pZHb=${S3s2fpwZs($&1XvK|5g zZ(UAj%<8|ySvyx8cIGn<9JVMaQzR=xFFM+{V26T7+$Q?&W{kEOo|yd^{i+4!<5nYf z^*fodUG|Qq>5Yr}C+WR9KVo~tahh-HB!6_U$68&h`k|9l{oL_EPOydZIP-^2QMbFQ zpBL8a6X(aIMedWU(&DR|1P>Wrzc??%&h#N69-GstImTRA*(M*|_wM^+tVVg$<j1*2 zkMR|i8_Ap@vNyw?@rW~zy4SCh<x@G+<!stR1w|b8vL<a*yJPP&eN>tC$dX=RfhKUS zlV#t5EjNpDeZ^8E-Bm8LGl}B6lCf<F+nO53u~ZLRu<+3f!77SO&&76^6rbh!4IV4v z2(jk+s8sV6vog@fDLpOZAVJREYa0nY`vzL*WrwnJWUj40tW(y4YG{Nz_G7XIzKVfm z#r22F@{^RZ2e+2{MRPYSY(FLO(>J=Y7D*xpl9s@4o=UoTJd4L}LlbBHObcgRzJ1^z z)<I*%`qY4#6$-gq{$?zBAseMuwJ(?ROjq2k(^7^7Jg_b{A>=oHK8BMH**f28xj2lN z+fQ>?J=+M34{AvtvZkQ^y#mo7ZrdGexg=|tiV?}Cwo0d1Rp)g29>lq{T1#(?6q%Gy z?tbT}VpatWJ$VKNW;-XC?dT-ws?N1bYU$UR@}xcbGr(XFPjeTlMOqr(ix;<0p$<O8 z-reX3yF~KIJ1sqhbY_uqG&m#m><m6QOQRVkU*HpKxoO`7DM>tjggx)H{Xp3#t6R#q zLS`&1%ktiWf)z%d4Sh#})z;TO+MEtxI|gbSFlh2yw@9~Gm4`p<X*qSJ96TUuQ$8un zK?nL2-qoMIB>N6!wfjNyO_;>{uy`*fP+M-jsj@GeCz+CXnyb3+QEJ>kH~)iTp#M=S zO@biovHKCx4(#^k;O+Wcmf#l{qa`25?_qOaHH1RaHB!<i-u8e1%~8Q}og<%<s1}{D z6Ji&hlw&nbU#Qp|`!(9gDT($v`$<Rm0rywqQ?rvk4KHP@joJ)R%&y%@;fjx=nQmRT zQ<al6i;a#xBX+L4etrExVXL-BalU$hX(R|73!Yl1U7BKxH9CY`8(VlM9_IFEX4e}q zySomIba*J2ImaJ4AUq}*3@-3wWET*M>0*ZH!RNG|4QYq{k$GESF+K*qA0(Y~=f8Vz z6;O!v$<;W@`=fL_uL5$Wm8ny8=l@Hn@K+~PXiU#L)&ViR-nO~epN;8Wy2h>Z>C!9@ z`2WPZv*!tf?A!=)JSM1cZNDL~tj+Sze4{kjlTn`+GN<|nqZe$YH(p2+3_dq<<q6{- zc!H2&W_EggE7>`<Ke@R&J)Sw3Llq=jrq*`!PgK+)s>1%IjQNK0pIp2pGHFGp&&34J zYvpS!_!HgaX@y*xdKrwhzSeR35Bd)Z`N8}(VYi<{<!QZz+@EyG;*=nhpd-GB=}%M% z+&mQYGl^-nP)BMsdY<zSWa#MK)qof<hIdGa`_G;{E4ok+OsqBFPp+Pc3AWsmR9LC# zmvAB!#Mu?5r>B1}>@E~Qbp!rBplk|~a!Emvd{RZQ7ObTuHrnJni^?<UUq;<xaZFP1 z&8Veq_d4&+`d3Hs$+ryb5X{nQsifM@t%dR^$Dy)y>odIVNuq${cWD1)k3FHX<ooKk z{fCXo*<Fw;p5dK+Z}P<?Y<{BZy{eb+`EUHEBNIj`6G77h)*dNmrgqlQ*a&{?$e2I5 zm{i`abgV838)ru1<V|yNdD@vc#gE)n^6plws&<=BbuWL~=_yv!(~NN*`$n5rR<9=7 z2DPtLPvagZA&5{J*|WvfkGcm-v~j+aF_|!hakBKVy@#2(Ic@M@t(6=~b4ywxx}1`E z*5yP&2f-gvv%JZW8!4FQ9cHPd>gl6Z<9@W!Eu!s$Wrh98YSc&9y`7W7ELu=%SEu#+ zOK}wB+BEOdv)ZFw=pl%H>uModi5zd`%;3GzNyI`1%=@D}pf2M`?*NSQO7021gNCCm zR!j0mBagLxKV6#=CsvX7*4`9v*Hl#C7Xm!`b%@ny+<`*1gqIf->sc(Fa`Z!79K(FE zR?6H0!O<isr><_alJZ_p>3D;D(%r6k#)kcM4^odB#l3c=vdXbPr&wTA#KunAYxx4{ zfJ!d$nELukc|`I4d>3>qH!bkj_L#SN)zH&J&oL}QCVF)@r=)~LZZU+08LoU-FJm?{ zFeMFp6;bQ5Eh%Mh(-e#G7<;pdt8dzV)nV`HrX55i$$Gr--(O=~`M4^Emz{cbnctmF zIUt%!4oY)9b@|}Y{HM82s~wGH$PleL<1Wisd3O{dQ{8rUJYa*f%5%Z6YHtx)@B_`p z!Y$!yW_|s~Hw=8F>s!!9XF}uF(~4qSvKbVCpz=%F)U}Oyjcj=#iEd-rT2+IztumX; z0=uhWM?AF?65?yTQr4G0rt+WbhxaSzWNvFvNF5_Jii?)n8wX2h+^-iW6u<MieO)V6 z)PKLEt%H7MKzS^ur?fK(MIost8hzbs%#||6&$)g<krHeJ-L0H!5cpq56_<;vY|1Cf zvGWJB2?8tLF9IE<K73d{_X)<^kuW!p5F>ca4)5IXk1{qzcgJ4R*~1KPq@KN*!$2t{ z*3@{+-?5=IMVZ+m-CR@ZZdA+2-%|KUK?v&S%-?V9UJme!GUgf;xcc$bx&Gp$Y+-!? z;?VHHx4kW5+On+nj;ygI@25u@dGqp+8khT}J8ivKZvt&aQ*5ikKfLW#-?=kewKm<w zeHy}V;<1%{iP)wb1zU~QHxBG7f*asB+>({<0k-jJHu-nwHd=UqeM`t&X6@AdLdC)T z-Fbn+qWPdhYTJwM6#3fku1bP2$LKg0tr3K~&V5<7qgs-R^P9~&8f;eG%7?~PlyUiL zE0_B5g<JIpdt#vvbHzQRNe$GeV%Kcu7znO(%q2r!BdlX*pB;A-KuQS>4!Ds%7)6y% z9u?PQgXL&{I&D?J+|i}5@qAKfZl&~L$NIf+es0@<cz@~&@x8=7Hk`>X14F}evSUEI zOA_^))*a+`!dj!=6UY^=s~-hq;gv@hL*k<@d>ZqaX6i7i9O(2bU=CPsXH=yluQ(s< z`&R00%GrItBcJ13Jq4qZt~_Wy)JfV_Cop_PTE;(Kt{k~n^G0G1S1o)msBV9KV=R!m z@`MglHG1QQh?K<$ZMC^7_Eg~)+liB$H+HguQyw<!dZ)<eN<8VI+@0X%RG2d~RBewD zim-ImyR;7R-%fKrq<MR_^2xL*Qbx@wyYp#-2fj10{&0loowzs5b4cbim;DkuUz5g? zu0kbY?9tt|x`F5~gQd|&i3KvD(^F!(<V-!kGc~jEOZ3F9#_f{r7Vb2}e2VvLs2NQv zpCqtZPn}mLB3jRR1XF30eAS*l7Sq+Gc{beQPQUjEBYouA4V7yy6N1;Kgp$nW=@`Hw z^{Y)cNuexVOeC|w5|4>*ylCeT37%GnCUZ?(#`Y`aDo5O&<A)#u`2s`sFB^$un0re8 z!q(M;K&SD#8d6E~cBM*zCs;CQXG&fMK1`lkp8HDGo`{G(<MwLA5xpXAew(<)BB^kb z>EhyaYM`HXy>k$jGPUsJMz8FR6yHxdYfL*-^6Exvzn=0q>0u^#;-{VGL67f6qTznM zHEK3`-<TQSim0r_m2TGA?W}sFz+Md4y$))~v?I<m2wB*0K#ex7G;xkwiWUQ>w`v0| z6596xx21w&nlWqhN9Dz`{Ud6E38g2}Gz*T6epXX2Y%K29Q>J|+7@m>(B_i|atM~vo znE9o#<3o%7I5V4H+oK+bc2MoeIkV5rvFfDxO%Nr{Ovw|l2eF-Kp0|pAMW<_u0QXTN ziA%=*I5`?PwI`$m!^f2(s8bLmpMsCQeweR7Ax#;vIFs0w8oMxpadhb*6v9p~nV%;_ z2QSxpt&sMW4B#!w2RofNJmyoPYgVczWF5}}&M;!^o$wnYl-@iSQf7O^Mc;M2M%ljR z!3%1uGS)(8;$buDzAO%gO-2j^jq>|1ClE2&O8cvdl)=XoMutyjIiqvK+OF-w&#WfD z<{5a`gq@IU;`%(GpJ91!t%L2K>x*^$*s1X-@$<b95otP?#k(r=vAQX8wUuy$F8X*; zukW1@IYoaIPX+yInU((vLOf<JQpN`5&ZmPoT;1Q$V=QLeU*`9Hl05P-0=gXHk`Zc1 zIIlDr+-z-N;NhV^_46pN(>Z$a1{Vy|)2?KG>r64Uj6X@X=iAmIVT5GjM<LlWv>##Q z=aCn1k1R-NzT%#Jz{82r(^cAC7Ob8rgKan*w&r@GZ?0_+<~B+m=GIIECUMYz3E&u& ztSag}@BS!Ndpsg&e~oJTC4YjAXzfu5wcVsCt$rCBHBet6?@`iD%I7}e%|wx!<H&g) zPb#Y-wr10U!9%B?`w~oKumn@8`T_ilt!xx7p<{&AVe)Ku!ctLaH0NqQ2n<y-#Qdr( zsx3y5c!AX=J?i@YfqWpdCHNPn$O7Z2vrpc$vk_tWo#0rsP@%Jhu?EM37Jb!A{OH-+ z#6!4k_tA<{L(a8<rtfdF>BeblYL2$WCe)^QkB>SeGppb|3*Rv-5a_c5?F*!<V8#VU zMUI7t&?){f>vvQaKtAfZv;83H4#o@rfn;Jf2ACV3Jkk~X*Xno7-H5ly?V;9fi$lCq z!3)~%{MFoHYgnK)BxEx;r`DuLCSXTaRR^^|%FpF5cR<&Y$Psk~4}De}k1YET!fq1` zw=_C&i?!*)5pt;}r?fCln=0lyafRE<i~@B_m(kmMKW;uA)l)_a;J*qOWV1Is>o$2b z!kwzIYi0w?%&+wJpO+tJZa&t-m;<3qJ&-XK6RX~YY-8%Z$9*$t3;y}=RY;u4&Gni0 z(WSH29-YFKD-DG3!KQwnOiSaiqpVEp*S^BWs1~UK>nA&+uNwE{KK&J7xOBwK#y5mk zJ>WBxKdDE(z55{-wia=OIb%+C?Q3%}gb8O|U+~FZaggA)_A9e{q<NgzrLGl@U5KA* z_er}OlHxTy#;mtVL+(sHd(7~py(~SkLT&(Iz4TF^BCuAhby4mc2h;^`+0v`db{wU6 z;ZdQfj%SM10gZg%H6}No{Mj@;qv<q4E1g`|b!x9cB)fm~^8m26BEEK+CF;#Hr@U!` z44WhwkDSp3JaX);+Pnzdqk3DTjEse4z;@Af>vi(mgJEIQG=Xg>(O0wCb1W_!cE7di zX*#(CHofG3IZd)*2Vb_iQf1rHU>n9L@T9|Rg){pJ(Vsy(Z=i^DB<j{9TbAUKPd^V9 zl8aF<wuhI3%|XGJ4Cq`k(o^y_bjs@N4;KT{J<2O}d<{K5%1wym^MVQIh&ASca_XXY zp<zz+hl@IsoBiEvQE?%SZdgn$3HyD()sxQ*yEVWl4pJd=e*GM}@48V8xCne?<U4G$ zVc!u@kjuj(2Yb1pYRI(haq5@B2dq|ZEpc#uR=lq+oxi4K6d3dYk5H+W4>mEih;;nW zKGTK64qjd(elng<-zg@#h^Ca|Lk$YDOt0|!OzPG)Y_HUY=%C2YyC_pxOK(XG$UbBl zuF9nRs~mU4dla1<_hMrD#-r-W0DQf<+{nk}y|`s11q&NbAgO1Gt#sg>@OKIDx%r4p zV2$nT`TVQ}`~%<FESzBdxS4a%NcKR)kxz>#DQ<Lh5hj%UZuau`dz2$(W>wO(y)CcY zRKFr<bj6P+lqu{yF~EPhv=~}dk#fUW(=+-MBC6u-PJ{}Wj@H#ziW&2guW4pe?#j`e z4>Y0p?sYAkb88dw{}yPqN4Q9BpiVgKDHizaA&H)P@r!|wp?aGPf7#i~7B+qLwp$}D zOJyuGO$gI&c~5x1N9vu0aP9*~oIK%f67JrP1EOd#AnliBEUeBpDku`?(zAL8r|0OQ zV;}|^9zUewPKCu6G^`e4RwxuOcDzM-UUQRLDveuuEH=vuY!zds!Sr-9J15Tj1ILPV z5Z0okyMzYA6&=V>;ap{$=SCi*>>`UvZ;t5X;KPW>l*DX9)fdheO|E6|c*~QjlSW{L zE9D7p10T%5K-?8Zcn3I*fwG2UTNj{^6l0@`ygU(iD`S<s$J#pOdsz{@JUO@B2<*40 z%kP(kaEaO`hxv<=<`Bj>d~{WmhOKLQVo1TmJf8Z+Tq<C_Dj3}wKG{OFipZfb&ZX8E zn87a}0U6R2v0xt{tXZCVAsIvQT}K}@{^a-1ngEBa=dA2*5x;-o#e}9`dTU~2=1U$d zeNPiH0foK`++1Kxt3*&&yxm0cMpGFIamM}z4#k)O%x)&LqEqO_!-_5j7B2G!QPJ;@ zjTVK<w=>^i;e#ICD~}lRU+0;ny%YUKlv+<w4ni@Uyi%ezA6<yT_OPBQ(OKT-fJ8GJ zpMH?;@m5b}%oARcnr%v*AYL;I`UzNSV`Z?Ypph20=ou;zcK{V`n*~Nz5^Apa#mIJC z3AsCaSmc_imxd_f4C&XE?_w^Y`P5(f%YvBN_{OC>97+}~v38<Eah)=@{op=R>(bm2 z1W~bA(pAyNU~23xHxxec`G7{Xd{`MJLzvWwJ3X00ru^#lZh>*{dBoyuU}uNFRgD)6 zbGAk+2f7eVIW--V5CLM@d#J)%{uJKLm}d)phE$uarK)^L8tS2F-2~ucJ2zLGDW!Ic zX;{-<jMZ+RuXc#mV#b|tF~3zpZuhYC3YV3W-8_>3eeALyT0!g#td;K&-UuFUHCO<) zk>!n4va$%wbEf1kWAbY>Jpw5wv$S6cY)Ff0Lt-z8h1L6Qnh%94Rus!cIG45zRF3aB z>OXmU=A#K6r+dLS+@8Y3llS6IV`SW6CA9fHp+|H6;B(L++{qV~a=CY{;FxEWp{u?R z5_b1~Gh*~I?9=^;%g{HRuD_1(T47*(jtt?BAJ&Le+!N{@N+4EuS6Sc|N7};9l6HZI zzE2<`&Z9@Q){1hHQ6RwB#~4GAL+Qyah{$v4DXhI6^hq&m7EiMO>WHqlC*Dww1<qA1 zQHGn@CLV{oE*@3W=-glAS)ENttATnAH8an9Mw#d*8Uhxk0IrRoX?hBzUX>JOlHKg7 zU#`uK&r>?wH(I*%<HlJE83;McK<7A>!uxZQwyUC*uIW;?V03xa)q};gq!(QK=_)2; zG9tnq(cjdE?5jyym-SN5j2?>U>lH^jEpPf;Ug<^dgmb4poPRslnZ!88wduVm@sPSA zQ#Nt#70E55E4Gs3K!{x^efmDM=St)`-w4OPto2fFIZEMPXR#pH*v$n>S)kRTjn)+c z1&VtJDGp^EFn`=B$mA_qYc@cU9HPO>{Oj*f-1FDpk!S_*Glz?J(i_m_a%o~cx)j=h z>|OkeF^_KX?WC=|Ox+SqZDuCQ6$0iw!3THlkSAy&W|@*dm%jkXzWau&@n%NXw!8t( zEw0bEEAAKd<Fa?XcyiG1NSU8Ss&l&TJpzc6({&k)*$cBS=|;=WDXYQ{-xQmMpLamq zl=a&jYmD5Ql8)r(XFXT_h7gQD!XvEsWlJY#7<3>M^CB3H#IAp@@B1j{$6@OlxWATr zOHri1nMH9`UmRcF%?}Izp!On}D`|*$E?_<dUDRXP>Yt`G7A9ar9CSk++nT(s6E2oN zsiXYeILw&AgtiBN%69X#Ldi;{{IJwikmzzhn?58mRoBplaJtxXWL3l0;BynOTwK^Z zAl4zb66b!t5p%qiKj}VNT~++a{P&^_4@@rjwBCGz*NIJG3(9xxfh}&3r@mlPo8jE4 zOLZo8N}TE6x8L4;VIkm2w7xkCzUVbUBZKcF!qNraDbaSO(@c+;v*irf)RqmD@qbtp ziq)I)9k7_!68mBTdXWe>!*#rVgqw|L+@GnNs}AO`re=}Dl$AV}fBBevE{}>(4cjgm zklRe}%e~V&*d@<aqVW?g<Clk4!Xi~un3uL_XXUpEf*+sCC#gdg0xX}UY^}%_B!1c( zd6pJd1nP;7_A?O*c7bO~t3nETi%2S-a9{|I)W8$`0q;0LS$ilkWzh$9+r!_Rc+!+p za!K_R4Qd`(oAaJ%j9xKcvOlTat8)NjwUG7xK+^t1<@;inuP>9tuc=^EOw_a?m*Whf zWQ23SuX;ycv(m)t(9Uyv^WE*tU+&Aaq`4T*+P&8iwF1?wf9DN-5OK;0;xV^>b~HkP z`}DAlV|Y7~igFQ;d)|GxFc-0~sHZZ|0QL72XSvMKa%ECSqcey~_AjF{a-KM#;9Q2o zroJau$ukf)sLQubNK!vXkWR$=dKQ)3nMPGVs^FVu_A=@mD?~T6qtvH%H?LWQ`xL<Q zC(jrj9Lw`6ItbuVKG<mE(mEkK^GlR?RvDS$5hH-U4QFsI0O{&bFT+?{^w7h#*h_hF z7be1A2_!6<%x`RTqw5mzvc`01ESGKIu1GEr3#F`58n9$z)tx*8GX~1r)^A*V4ctbl zaUAf+1^$FPl&SXh3<+6J=)jf^5kBU42|Hjf$57L9xn{u<f>_KN%u_odduf2y8Fb+B zRKKmp)HN-rc#LwnL+?DV_zs4s6g9Ra7Nei)f}{EAGhr=fkkzQ}J;kZo3w<{#N8NcH zAzH-86C4`*icvWigkn%%;8jo;y4!mvvZz*gNvqaWoQncwRBNSSt!_gXbGzSo!W(yd ztZ?t~5Sx}Ku0XKpN%tqu6R)rdT5$=UWHaP}PiK)z4!RusRCMP~qnTVFa#CyGG#~sj zumebUsi~Sp6q?%|r8zTAxLm3=wfD{c7C}4<n1n@&gAV$NC9$;i1m?j}+_-0&$6Z=r zQ7737XJZ3o!$YjO?cXh(z4(`9sHERifwCnVXIkpGQPV*LQ3EWHc_><V3YjR%DgKb7 zy-S^Xo#<}Ds>K0@oLt4|af|QCGuK|)a2%#ll)o}>8aHcTew-ro5G!79o9c`Lu2yl| z)r3t09~#We7ho7<uaxhV@s7OYoProA?{3)DrLynkE*QVRtrIvPy-PZ}IcYv2OUnn{ zX%CXIo2R}q>0s7`2?-{(E6*I0xtS|4IRygEszWjfn8rj$3GTuzzU?<@>f#sj?&Tyl zo2*0HkwPV`Gq$=cc^xl4-4t6U6K&^B+6(!2^5^u3D4n9!yA#|HMxMBgs8VAh@0K_E zOIWAOM8WWRf)kD(Uzjs9pQcx7N{3H;8S3;-SW#*ns+Sj&Q#nCyYHBhsJmR<WsmmL> z+jj)Nb6QF7N{OllL;y$I*783j)lNU2XCv(CCrT}F`$z62EcylYX*{!>`e1p*;8fx( z{xTq4IlzS;Foq==&bCOJA={~010c^pHt){K*0n@sFg5{Ckr^etbw8w0Cd@k&MDf4V zZ;zoqS=gW+-^n+UIp$M&Fx|o>Cc=+x4WbUT+0*9OWvK1eHQ&){?;;a0#wR<zq)CGz zKk;6N;&KV@$W&ARs!|oh?@jg~Mw3{C8CbN+Jf17Z2z6^=Fm21`di&F{NWD03{mfW< z)E2PU5rVy|cyR@!OQ`~~bB&}oTXHVrqv_!}LElg;SeraSXo!sF8s$;o-B*>@$CeUo zqihh-fU~`Ow^O`FTQ$9w+yImst+3AbgyCt-h@bB()x&QW2rU@-oMl!qQF$Kv+Hm&r zr%2Lc0z*7@p%)q6ClItxGP77AOk|4Lw_TptdnJ%Nl}~cpud6%HE}ZW165G}5EjY?) zgRf|Rapi$V_$n6Nw{>fbTx)ssjovGgS${q8jjc^VU6wzN2WPGH?4Dphap*$y=tOFC zUeh~Ee(evZSM#&twFq*bQ9>}9R5AxtY+ow>8+9SW<~>b`unDQp`HIez?yhOV2}~!n zKKtzwg^u-bsHhDmIkAA5E}mWu-&kY&*39;cki79;ui;U+Gwikw@_pf4xeJ#Ovbt5K zb|uLd|GJs&CCAg}Cz-w;E2=QG(=j+?7j*Wt6!71+ZUOeGY>i<%g~ez0E~on35;0F^ zw;1`(E%id+I80cynoUMA&8g*p`oTWY#0=VbZ(*uHGFeXwo-q}4AluKZYCJ8ibXR<X zJp3q*&3$^l9si6oOLR9P%MUGppUR4;U%a6LX>_Tc;?IQ}xvG`<O5tEiX`X$ts@%tv zEkMuNwwWZ8D7%-_c{H0}q&OpY3S3u!bO9^u$y;XWi+h8Sa^DQ0F2)zEF{Hx-oBfrn zfg7hiUAttPi`gfipVF-YzXJA_?Jw?jX4aU9_!6OymxuFsD5CVl=A7XyAgvP7nWGYF zJb~CUD#|jw|1LXT+-le!i4I_00QTvfTVxzOZs;9h{XU}JjI~r|V*dvawE2>rpE^<g z)RtU5-7DHbWF(*r4989KjFUcyN*Z)*vK=(d$LB4aiKl}7MDlaG5owEIW5$!H(8w{b z!sDd;cADBq<;rq$f1#NRi%K7&y=o@YBgXQS1#(L%_L^vx|6)FIOnRh<jGr$hMIa5u zwNe=IhA%A{^gnCuLwfW&9Gkgc$}#50KwH~@sPa@r-MVEFBJmDS!7j=fgzv5ky*z=W zh%NmT?cIF!V#zXxdUtF@!n(TWQva@bv2FPKz{6eYuxHfRs&o!EFq}9&k^4RYNYj<f zwMkY_ZVG}>K|QRUodd8)+ZRDAWBv#h4t?BoioEw)*h+(e0G`i0Bg><yZ15LPwnuE} zG5jOdv~=F13>PkIDczT~mbV&9xU%c!m$t~S-XFt$|6;ncJ+RyUG+7%Mj1kN=73F|V zaELdvQE;*d+qgyQ!8cyX5vM(8VXY}DVR&@OVJLq+Lm|rpmv}YpZHG?CDx__`q@v<0 zi4Dj~TrDIB9R4uuUy#iUTw17l8`>M^8L7|B1<MxhC0__CE;)PtAe%=ZRd|TpKWuH( z)G<IER+wiF`q#rDIKkj~_%mppoj!)_|9&69hggCe@@+Yn3(r1H`xb@7GfVI5p!x6Q zSwIQ5te>Uu4@^|=@uRdEOy;&am?&Y(5o@niu;4wR+#H40DC9+vRmJ|q4a!DA_A!D8 zh3!-zwYbc}Fp|Q?x?fT7&~>flLQ#;;mhj%Nqoldf^SgKNc2Ob<9JVHP!hOm0GUMGa zhNsN8S3Bcc@8Q=YE9VyrB`ZyjIpb)t=5Py6g#rt6LXke}u<yp)MeDG{A7&l&e)dL5 zj4})iI!;6Vxrec?e2D8!6k5vF=z{R`9td=LQ?lm~C46K^D5y)3a@uj~!OrT^;`xrN znfwf%r%)ge!e}b#s_8@PvJYL?xs-#_l$PjKo!Bf?gPCbf`9)6o$g7f)Z&*zq)-|N@ zB#1Cgwq1xYFW-gn>@+fx%S=7+H(Dm<bFY+=#Ek`GuL8-v%hHw%XFB$v(U#Gc(gub~ zO~n1xii6geu9FA+jmpaii9J8`F6TaFGOU}PO2$vhBU5Tun+gjt`=x8&l197(zrNHe zdLuOP`t>%ht5sCWp)afKY3^vIVL{!_(}K*ot>%iz{v4XKjz)wdX=kl&4)tZ+U*G|> zSEO1mJj38h3N>P97wc7g+%QRD*Niut+|bai7Lm%S>=4&2Dq=#n@GK2!ZC8CbnIuN6 z*N|h#4^*=8F#D2r)405gTMmDz3ijF-SBx+2ZNDsVN~-nH?rMoJ%(s=#`!&YhZ46Om zvGEP2bnwBbtJE<vR15KAk>8Mkr(h^e<jRDc4Z@{R@CIK340#VCcnUhtAuiUt>WQ*l z1Ih(+cj=TDFWplG5pDb(p0Ed7G8<~qq3do)3^YZpi{2E5I~w<qSqOC#9^EuYc;;{> z^WNE;bK#;m=>A>?sTmoZ9Qq<)JRow;)AH47#~f$K6R=vuK^n4kt5!oiKR>RuRUjze z^!LUr$0H)JHG2c87mv<!%O`C~>EY|G<$N{f10kY%$2(G0pwMSqwRdoK(nC)oCn@a? z11m=Rrf+#E6|@1P7$3XU-HApsze@$q!qY}OmM4{U)23T<{TkjcQyk!^D{H>rzj>f` z5bI}wx_ujW$oI=LyYH;lUjg6{yrBvk-{u9a&R1-Fejm1Y5kIao4j(ZcTV0%rhn*<E z;jcgDFR_fZG+7!bE>?C^EHUH$s}l9HAW+yxQ2r}_;=m30A1_omLp-`_zA6<C=_ytl zmAqzVCfee1A-#nMfr$Nrvo>39en7Tn@CYl8eG8YL(_-s8>+9%c(@Z-u0+b+HqK1G< zc-eiIyZ5PyxW$_~t>rLHk$g!4_Wtkc$JTY!fg(XU1$+*_KdLJa=Tc}>{AT)j*xb+6 z_mEbljxwaxUdjJfHy1Sbsden6`m!U$@xrs0ym>tDUV9S#W;0i`<MGP}^^|cQs*!O( z#^%m68v~?&{J5n}wWzCrJzK;(27Qs3`wYmmfb|f?7$tJucLAWd`9y;)8BZ?XJBdFc zGEwDU=6fb#1nd`+OUnaWbCQ<i3v=q@HuXny6ZszpDjmxCtZHnVcB&Nw>vb|O)#gPU z9Ng+tI}bgDu730}QsBl6J_{p`?5Bt-7!=>(>j&z#fko1#*z>)VM?O8p!RUuSH!{r# zO1<6&^U@H3A{8bEk*iDIighOrZNyncKI(<A7jR~7iC)uS3q>00To>k?0UBvK&oF+m zcDzthi}S!yK8UvUEJ?ZGTE0o)0O_3;sx(K|d&rJOo=tIgrdf4~-{0aUzHCGM*p>Ug zfaWXg!Vj~*YxxY{ArqKG2eL_T$9?A`MWJVe6C<W-gs$25^m>gEy=Mm>vsEsoN8vuJ zMWfy|8dKO{8uu^=f_nri0vklY*nJ76ImrqHtVZ$W8W68R3-LGNNdlguOJ~+;-xmGy zn=~-T)4`h14iozYtKGbJ8cH+SDiaO7<_(DBI|C-Lav=aIfVoXQdBM3fhoT-;r?BuS z;PXq=NoG+38U5;1P$@+&ZNbNgA_debf>S1%So#&qu<Sjh4HJR#BR;kpwitqmJ#y5I zaYPEJ!Ine0`{4{?b^M1)DNAN}q)$QJFL^YX8X?fYi{DA%6sy}@+Csw~suHhav@;hF z6U7oM6>#j)1HLxW)g3dU#Mr4{mRX)(PqvOY7VHn+aXTEa#ig+t2sMzzyW|SX!Wp10 zalpg9Wp$4@{=3W^*e`;cPxUWLd4D=a<#xIReLze$v%F-@jAbs*p4>pdl1w|vjNtyt zQoPqil6I7^<dB>4Y)EsSYSX`;>w)G*NG6C%E{vlGq;T4{RbT`nWd%T#7*QhS#<xZ! z7Hc9b9s6h#OXwgTg;RT94)B)GQxg<czwni^8r_Y(LuA(Z;1p0WFW(-RHZYgS(c$?^ zCnwD#+_c3C2qOj2^!CRD(O%r2?t%a~)Gr}gJw?d@qQAMS7Nog24_}@9-gJ-9Wu$Xy z%eNpaLJ#)Z8gs7ot#5BAW584j^UQvju5D7!)1Lxh47<M6A<&6OY@`>g^5T{BTLQU( z@^KPa`{xkVdSE7oq9GF%1yRb1g2uSi#SKek(&iT<h-chpj0&xn${2X+Qh=lQi61HQ zBm8F;g+k^GA`6SF#vA8)aZg$iWp06VGUGmIh&n1%Sy9En(4!gl(_|q=EvoW%w~_AI zb9LzB$4gEI(TpYrUc<~nBEViBHdkG@mz7tL1q$~2KmwFc%D}G<(t;*0S54@a`!*j2 zH0B&SQ{5K34T1ve1W;y*&-t%C;N{H{sQsSHO#6_Lqr5YDL42~Ud)m=@Q60;F^={{5 z7M6p2u5$3{3S1^LFU_-VE-H8;?Es?m>{$w;RDKKJ*hqmRsiYKtQnqQ^XBF;4+I`Qr z`NENRuGA?h?%>D`rHy`37{`P;gOK0`<vso}BxTz4S`lONp0mgq=e8JJx8f9Hhny(+ zvG>;2h5UFrpnSMo(x|FRg;dr7#e^TB$N$>4TlPt+ho5G4#sUBaryeEhxzqGu<8ATO z#ajMx37o>=;zkYtbJ`1_ZFyV54H+zc>CMY>dVbfR5a!g+dw~f(vI#Zz<^}T5wtY+W zd(Zi9&Z)AT(|0^_L47>7WLG<dc=|13+&9<q4gdG|MV#k)un67`G%SD2vmf$wBkeHR zV%9olaa^sVP@CaE2NRfy5Xgi#Ooo2nApNtrdtO@6#obH=6G14BM(JRWilpDwXpJH1 zAeD!I7kd`J^<`(9-p||Jf*J#HU97ssgazmL(v(|(sDuQNfF~C>T=4r+l=YYjfO?pi zjTtRc@ppqIKfJU(k>+6pIQHiGZCBmH-ER2GxEkcB$#g(wq;#Qs)SmzF*ZK7C4x+8- znmM@vr#j2~y%50$2H_wS1)WGK>p)fcw*htk=B{pLQ<KTWcjHTS{HZpo<wqDvkEXYv zHHQbe%i~5zj3<j73&8}>QL!r`y5iq{@k`X5<F%Nz-TWO(JDfE5D=$O)?5uJ+udrev z$Li?(YT+_+UwPIlT$UNU+TU@9=z$R6&=t+2P(CN#qby)e#e{*ROp=bwiVIa`?8ub` z{^kkpyl}o0BfwUWtJTEQ_|CHM=+ogJN7p}ql7wqpOXF7ALN0<2<d<zB`6wOHNxa+M zLM|?=&Xz!yV|cr0{n>MSDB)|Uogs3fc?t7L?QnnBG^~l-UqjULluTImcGn@9sSVA* z`B-+4DK$$@vzT_YdiObg-x+=Yg)#0wacfLV^va-lNV1$d^<kh9WsD-|-3e(HzXff6 z2Ng4njP`3dbzpOminJVY)b!c?o@O~v)b1hSlIyGSJ436+CGt@Y0lS+Z5l4&IeV-Pz z#mj$iE|A>11nFH()IF7&N&)Ic&=BD_-$ysEWTO!CTtiT|cw&wKLX>G?pxFOOu{uN` zpLZ)GV~*gS^Yk(y*`->t8FmWNL#H*DfaU5Gv!0(73(}VA=`8AfN?Tjvm`{}U=(OY- zuy1&BYH0`WR=K~fcphCWx~hja%Ynfb9^T%M7An?bn>j7aFL8gMS-LZGHrs<wFP?m7 z8t=5^g3~EQ?1ZkbR=^5|Wm@?7`2*2p$3oNM^EmX>LdJMu?Fkw?zEZvBPUn03k9By) zOnxVrvV*U|$OrpCMSIi6H(io)(LWL)&CGfnI_N3bH$!njk^5hXT=%7sa?^8qRx_rS znb~4adla=5&pF3m{Z}o{SRcuKI?t0wLqBe(4CQgj8S@&j%_Yp9c{8jo*EL*4HlM8} z=l1X3A(r1^%iZ@}s1tpuOVop_-EyeiRK)uQMBaml^T)-{*79Q%SdEmMS-7s2c<AG$ zzL)P$qnFxI6xjMGcg8*On}LIG&-RCI+<h3Ql+@gBPg`S~ZJK^RE35#`pnP}PnCpBj z{`F#_1YtG1#ST?hjRz3^fn%2sHg|6mKdalNn-;)-w5(yH3bxq!G8-IU<po@-4eGg` zKG%<+6|`^JESY+svCmAsrPnkdg-bv8cyOamNE=H-QoO`fH$~Hq^2d8`CO_H;$+uQS z@r_vxv>-AhwhOz4<X%iit21~kja#(t{%m=3Dm4>PoT%t;dac1|22on+3zS1%W7+iL zxyL+2&j=nQCW-{P_d-lVl-4HJ0%?9$;=i;c>$?IU=dqOLeYmP@(C_#WrH5_E(^3>3 zTUxZ94RW~%h<mPJ!tqnMKH)}0VHUnP<{7bA?qb{bjMv<8wGQvKtd}Hef+Tc+oFzHh zGQ^2zztU`CmQUVGcW2}}fl@F13y7&X#yW&M-pm0zcNly<rAwiOMH9M_u@B(d46R9l zhuMRe1xwoa<rFxoU8=@CMF&xQA2ROZA=mBAK~pMT0es5{?&-8_f^C8g_Yd>scrsy> z6O-P!Qo;hjOw$kO0nmv9?$~DSmjeLii%V%tJlO^b`NlyX<4D?lClCrMAm?k-(;3^c zsPSY5^WTAbrzDy;H0PUsA1p-=oIRon>isZu9Q?A2PVHnH9Du^Q`ya;CCb9q+W{;b5 zZOU}Eu9>I+9;@?Ag7W?fX`!I93r&_yK;`y1#GQgHaJ*ZvR#R*Dsr#%d=#$J-R)BkQ zGh{~)bo&Ert`+|;NUh2@2IEZl{*Kw0fqKe=G1eym;YEL=i_;sA<(US);i|D!t8Hv7 zSN*F_b1VyMn#Gf*o}_8|J5zh_a!*As{5tR3f5F>ls=o|7qL%=KWcj&n;i135)@^nn zl!-%$gmU%qzj#RQDuTbuL_6mO>C65?t;eVU5EhOv*UtO_nm8ojH(YsQ`^@g$zx(@d zj|4+Ns-R;J&iqd&{{y2=rQsNmNO19gyZ<LxdZ5pO!SvUT-}|?1|9wJmf}RQ7R;73^ z@J|5W_XSnhRlsKey34;b`|mp)<MO*=Kf-?j_pgclPcj$T0oBnR5Brl|w8;P~Zyg>T zYx@rX+ka+|aWDa@1HEYclU~du0dulj+1e)gC+PHJfa>mpnjid0FV=O^9e}J1M#0fs zRaF%;#S>$nY^$5GyI;4)q9cvmuL$^)*#d=p1y|6=48FMjN27@YM&paAZM*s>qj|*u zjK-<?%pZ+L6Ifw&!_X&O0Hpu)>C?B&VuuhVoBLe$eHU-tyMH|-^h;)_q|T2pnO!0M zlgaZ4>}3<q9!SaUTNmlsU!E>OGtNid=RG=?ui~oDJu`JG7Tn5xT`&4pSqeO4vJ!Pn zrv8%|o36&F+Ot$yTVhN!PI0AI1jn+m6;qfH;E|ut?PUru^BkCGru4?8s@}E($>?w3 zPAh3QyKFmW2MvqJdAk=cB=xaA`Hn*`I_~-CRl%%&#UGWpNWXqpWn#K1hXJJ_#K5#h zE%FtE9_a{keBiq;q68%A1og{G@EB*J6#1%Ug@k8|>bM5eSC$V~`>sRJDBqe_t~yRq ze#7mmJ-;7yK5R=uzpNUMfwhu}+GHk0bx}zc(&sn6Q6C(Z0;~faT#E?^u{6W-j7wb3 zX4vPC<<hpTkTWLp^{>~NiAA!+#J$yLDL{0Rs&b&%qTy2eks06xUg@cS#zJXDYgrp~ z<e)NQ6Hc1!o1DO3tWI$Jflx!7`9RMQ!H1Rd@Qcyivr31z<~Xl4xyuCYoi{Uo-Z^oj z;{dfz&E@fn+`m(?{2@qE=vMBPHpg%it9`|{-X`=|^ynx+qx%YB;eXm^eVa3}D5ysU zH;QUbW-N;mC~aC7RLQx^F4B)OI-mtCR~p7rS8|0-Z}t8{Uuo>DhdElpv+o7^60Sq> z8DB7LB$o|wgjH#q&Q`0GDpIzTx=+5>U?p2BfAy4IqoPa4WR+vB?k<I|o3CZlqYr79 zdcn7Na5p_R;aCA7u}@fQW!zw*B|vM;6b869%pk<^<V=0+#LZn~Yw3#cfgdX=UyD1T zX4Y(KS2Ovz06>#yrDq1P&xXAiN3)K22*ZKd(Np!rU#r!)OVRZgdAXBBHgj_NBc9Fz z^xr4<Rb;&0%q1|eP4Ig3R(S*+82kiylk%L*khzF&ejiOvpMJ@Cawu_?*X-Qr&XV~Q z@!{<Zo=6c^*rV>f;*0ja8P_BQx`_6V%Wsu}6D_N{ap1bE976@3D^^mHFMqj=CVX)! zH?tC$4}o3Wl1tCKdnz|Ea5vf0#j3KxEvR(9+h&og3%V%hCHb97t0`)IR6fU``atV| zQ}cP1fA4-aK)&Q}qV935P892Qa56Ls?;=q}lFe8f7qx0<0!AYzA$PbWt~>Tdea@&& zLr@f6+$RcOU29b2+8MUw)S#?xd1hbYs-)v6CHjC@=COMz!+S55(7^ku-ZvX(8%%mM zd7yNzQoqr@$9^Qs_J3bB*+wa+($5d`pI2s@2@c--<}{{+6L?Kvm>&HU*XdMw4CXIG z5^zC$9x6%bt+RW$dM$;l4PBZ2*%GG;g1yiy7LQ?X%L(qHz~x*=?fMm9(9#VR_TX>6 zf|@TMLOAugl_C`v{4uW1t6y%Ee7SHK{lXMpV><iY<|;tms|x)g5*$Z1MJ;o#$HM7M zk_9IRWe?8*ut@(<iNP}D=>gi_dg(|y)-2EZ=a2jr)tmVY`_dh!uXX(Pq&F_AymWu@ zz~Yb0*bo=sNJf4ZvHgj|KK|RS3Yl}g`A@FtpDs?N5-^Z$mA97ujvoF!k6Ta;aDGpi z;#>a6U;mDz%EY_&{<9<d$Jw?Rz^I=MwJT*346D5V^IyLO@ZTNs6Dz=GJH07a`V#{W zsGMEp|5y3{QThK@)tva$Kl~q$O!^#H0J$0KPp+p}TUdMoQtC-dy_pdsuFb5_kxIB@ zLy6S<nVV1FTEQ`dQ2@=(F57DHEM?Zz{2_yPpT_08QF_uP0D%gCTELy#-wC5k3QYkH zTY`7JKgG+j7!%)FY2SOEvf9j63-ul&d=s`Zm77lW`FV`8GaWs0gSL6<lD6y3r$1bQ z8hNdN@jTjNz6$}=<1(mwOK2!{du0p=<X0*u0{sDcS@n-4O-xX{FT>Tp1KR(4v~Ku2 zSQ&*s;i04FN&0elKvDJvMrR57vboR?-;f8W2okt`cOu#rCpT}7^~|qb|1MU{u4KUe z<?E|c(Ec1u?OUKg6(vt92@qpa2|v(0a`d<>v&7D9g6${(ide>at-zX{W^U{!Df(AZ zry>Lju1>|O^OrX5YBSJjr^pdl&))dDfYD1+A_3$dPd25U^bw!w$T6Ub9p?p#2<4u^ z93D_x*VDQVUg`Yc{SVKly)hUgR_|bZ$LY5%jFAV6vIFd*S4R3@UR_T8g~A>90oi2= z!S|?1_2K?TcP>;PI&%D++t-so9a+~E0AM8XLSP^K=RQ3fJt4n5SvmNKg|gYNN7N7l zy;}DgvP-bgqJQ;GatWZ=H4I<bnfx5V@9tPs`a+!4emTw45*<nhXra2o^fT9~Cmgy% z!^%^Lb^iZgXxG&M)$b4L&2Ru0y*yT7-`fPQ9D3>u5a~S7_1lS7B@aU6=aW4~yX48s z4YX~m+@~b(5x9)^ug?*~)K-4~A3C3Iv=n0(3rN6<sj^&PKBv!pL^RJCVAzOL!`PHc z29D|~(>!JTHtlnV=PYv)XUr0%9NrPeiE<SO7UfSZP8~?!*vU;KRZRreGKf?-^ncqT zPfYOnOopKH^4j#?t(uH6pF`jcNC3OV&81;C$4DG0%pjKsqV=i^AkvWApD=#ulORYj zI$))&L!#6!kTT{?BGr<;ln?jI0aDv+5ktsMlZWz}j?CB-(?ycX*x7*nwN~fRYB%TH z6fbu+h1FoII-gR2@>4&GJJ^yW7Fpa(D!w#A0YH{DK(|J)Jgamd%eHZUGQ5!!#xV8q zanyA&%xby>xMBr%^5I6OOcnBS6#(t;wT9?YU1?jyDr6q=G%o0pUaHf<u<~#q-w|L| zGH0QI3=9n0HjR8^wcg&;<??8ZW_rkUYjo8LQ4Xldo#M@vxD5Q>!@sp*F+pIRb;*vI zzxKN6MMXcDV+bD_Lc!xm3RK2v@EUJUlQ-u~!8K_3FWkXi;Rx`Ek6lg+_<59-XU15i zcy0mG4HO$Vn~hc105~kk)Y3s%Y?SB&u|sEppLQ00q2s(-C=BL&O_d5d2=M+w$5;*2 zB)XX60S?f!5PEKF*O0V}vr7QA6yEQ-G|-9o)zi2x<1((mn-fY)u<>h00=Y+LAj#@3 zx(X<NBLy}fhTI~UktV~~ye5JU>qq>SipP>=J&LU8l*cbI(~No=qm>VSHSGLk;VlS0 z%D|lr&@p}ks@%k_Dy8(!iV!@hPl~c`Q+KjVPkg`^hYb`W6xP0;avnnvF@nLs<hDS4 zroyE=7YotOXwev=l1WdfA$hzWu7nH9(Mfu$%HmSD_5zOEjs&Y!=xDARanr*bRF{bJ z)6vS~mZCikid%T2vA1V$I*kGhTSvO1(iaZ@gO3*^|E`3Pz0RJ!2MYOib@k^PN$jVa z05qze+0SHT;xgyyLem=bTJh~kfRa>m{qeWzrl#96u9GTaU(9s8Inx2c#rM)ViV3JF zovt%8XO3_O2P05Dfdynmhq2_;7xy3xo9VcxRcOzuhm$N(?_i~lXLFO?UmG<B8XG=r zU@kDJMw_yp#+Lvz;}PkCr-8Q|&NfG|RjC4LS33QW8W=J9%=M;$sVb9cy9;cQ(R%)* znp{Gl*;L@IF~C0ztgHC?){Hz&f{#+kIG6B6$e0r6s@U%B{Z%<RZ<3RM9qBiT%mFHp zhX660uUl<m0qPAxmADGHBdR8c;o;80G$oqM@e^RxdMo;GSJ0{eyEvjW_&8e8Nt%xZ zn7v%gemdMpj1@-r`EhHMG_cm??%{Ppu*rf#|8`jis{{fh`<OIpq&2X^yTDn!*{22X zFi9^=9E~{VeqZ}`c(oGf0pP0!0&vX1oS^h9Ut^U+fCphNgk}yg*(m}$GH~aCY&s9s zg&}ZU<56CKgYwN$v4NKkc>q}1t)EShLlb$Z2KrSf$e3lESUKU@uT#8g*)Q57q_q2V z7h*k<#d8Ss7h`!?(2L#&{A+LN5FQ+wNJQu5y=2wex;J)QB%i&`Wujr=XRs2{lZxjE z{f+sW8nAQ}_a`6bQ;!PkWUgsW-KL0FjrN6t@U*7aVkrA*BbpI6$lVAOtOt`-857-6 zVZ=C3Qa)i{=o`KPsq_100T1L>k>g9ivVG$Ns2^N0lL^iL>DLuz5`JeuFt|2g4<+SJ zM_@ObD&2cS(Lgqf?Sh@^CB?-FT5%8eMssv{b3t^%ge)`m;Jue-tbv4&W~zPxePzZU zQ;VW~AUL!yoBVRg&QOY^D@ywOne^d3(dW(pBl>Ev5Dc65P!w;v4n0~_zddaFUQwag z60#a{a%yU!&P;}7ydY;JcYxyq9^M347&AM>Vwp<5KZP`aRE>*mWW+%O1-?Z`?^y$X za9goMO@ofL5FsAP3z*GSypf(P=6O2TS!e9Rr6n%a>EA_ye;3=w*#1sX5f9E+0YALc z#$epTe*$#>Yi}joO6-H#O{!A<y&b?7@rivfRS8afE1>9Cp~K-kKG6+Sz`vM@dvKm8 z<PCA9D@)Vu94`zHv8nePN)t$3H0dx@g1xu#o#h<4hjC6fJrTIKaGBB_ba3-VP(w~V z^A>W-M`ysIf5oWNZ&pKGB-!WZt5PIDXT;S22$BJKX$x0r)4@)_PW;ILF_35$n>Qc3 za=lW=WOjm$J3#5K2xexb0v7Lw`tJQWPwfTsK|o33w`M*)OBG@g<mCAeKls#UKqp7* z?B!Fd1dLFK9jCq-9qj*M?=8ck-1@#@IwceVB?LtUNrM5U5d{G$DQVnD4h=&$f=NiJ zATV@;Gz=voAxOv2jl|FmL%i#B>)!ji?~CiX-f!=5yw5ju;G8qo`CtF_t98}8CtXE@ z)uZjsZ2u)NwT<;Bde$Cm=vd8w@}1JZx+@qhlY{%wuHpPLFz7pT`osge#quVC2gwg+ z1g<w;pa746?>-M#6c?DdS=qUN_QiFt1DBTPs2i&z7t<MiK&kEq@>bWrs?VS-E-MNz z@t3-+{h?zNFXp+Z4jb{@*oA$QQz%txUCz?-gw1<5UrT%ca@}!v;`xw82$<M7voztS zfa>98#X@FQwtAq~;ePpOmHQC?!i9050Ck3HFd^-|-o+iP$`R)Q+WzdSU56yGx|UK8 zu`5U3R6|hV)cx+BtIC%w22TBl#hdRUInXn1w`#G0-b&dB*C1Rp(;S=rMB6v|hbHcd z=gx?!x5fSOVTX3a0@o-zEq~VMJs*6^RsTx{#9#&l(E2=Zo!;PjrrF^W){c&b=g#L= zZEFMXq3x_cMGwX2;c^GIce5Ldvmx;uovY-0l^uSk`K#8aD_E36{VZz7Ku@>~ddiY| ztID+o#n#;{*TW!z!KS5Z?D8_`C+&onio!8Nr8Z9mam5U@C2muJ`U(s;t~^j0!pjfo z_km$W236drEg$~13v}1CL%G0jqjTM#B?IT#N;S4E7K}xKInXNFg6g`Nw4{zz#>HWi zeq#$YJ`{|%ZIQy0FW446T`DrapYu@W!k?L$5vXafs@m$yHw@S{-}_Hkbvd8%b>rrU zSGjr>+KP1n1TRP7UslU@6&?!kdJU?zyo!!{0$xUm5`xr&PyIX~#>y=ZEyO|1+tg=) z@enmOHl{n(QSFIs+SsXd-S~*2dN6yCqdj-!%9Z@9vq?9zW-_k23XRa+Yh{5guD9?x zL+b>nq%s^^Op56~SeaFMQbKlAoTD;A$6>SJ#cH8n^F}SNctEUJo7IT$^0V;hRn3lP zD3Z{3>rqWsec4*MQqL*h4$ZvNQc-#0ORufjvki$|d)57P&*X?(#66_o($`(S>RHiL zjOqD~SQ!ve&3aM}n(jJ00DR1ji+s8{>&rB%{*)(#z~o3{(nN_1Fi`M$?P8wqNgu=2 z3G8(YfAox3z+zW4tXHnsKbLSK_@n)8A4MD+?T;0Q<VCamD33F{Gc5tUF4NN*OG85O z1qwfP=F;rQS06cN*Sw1eI_4;G1wcE-5aUDUukP}ZqZp3n?M%0*i-Ey^Y`){}ed_ux ziUy__&;#@-FBy#M`1Nt|UOv*B&Jk#t>&wwe<Q}lqiwK|5p+PJU-<iM3|ENW_BrlxD zhZiSo{yOD}oxAVBX|phKtt)9i%V&_fgzu@=7{SVDWc7W#<H5%}d_y3;=)}^BV3&}` zL709E``PQ<yle~B8zyGowkrCWo%(CR<R`NgL)sVoAC5pF2Ur(|p5>=T|Dj#_z0zYS z1XEU!>q_C$iD&fp2jIx8K19XB1OEli4SFzuf*6frErn0M*x!ei@IPZVh%5;Vf?|Sh zvR-#xf2ye*>PuMo07ASucydVsor9Sg|5HHT177fT_ACt$W@O!g^dA9K<y%mxxk<B8 z&6&$x4$IA5gO4FM<Oy>cg*>>pz+&+M1|XsH`3%;kOhp6|syOG7#&1dPkwtJ{3t?*e zk7vLG&cK2#72?o%KY`|>;;I4(p4s$s+!){|o~)m-?kas<P$sPraN@(0s5@=eWh)TC z^WVfJA?*Nl*ip74t`l<N0ss1-!w2v!W)i~T3jf8S&4$6`&cA%v-uGW{Rs5^~fMe)r zd#d{147XBfxVaw}Zk_lX{{DdC0W{o-ZVjCM53BlLFrse*boZx5k1a({e#E~IErEZA zl>~{q^iY8C#JBjb5B~3M{#`4e#^?Vtw>eq>Y4zmSkW0U`y3sCPK}vJA4D|h-#Q-VU zS{g!w>?p-t+3*Hhd_mAu37U6Z7BuTXa*C<2jCx_M1TA_voVyhS%X#%s_Y@vk^@Fe` zU32q{A!xxge5+k#>Z;;LvpTT$TGY@(sDn_Vm~s%#p&lcsiZ6DwYveo=sfhx(L|;zX z<SSCmjap)s+X`Iq7ew94K<yab!l3ykHet*cUqkD3nAKp9o^36m_1a|pa<8UAv8Rff znp$_gvE@E+QI<bv@#<efVR9j`uf5;x@%59&u*44tN3XmA|2gc_iG9IZaZC@$LQ8&t zqmF>+7^>y`#tRPh_4U^6qC45onNmvSXypLnfC0vfmE*3Mi$NJhA_l7<aD|qL%Ox>8 zo4YqTe+|Tz6wz8^S#fCHvJCYC7F5#|i|31|enNX$ntA;mA3UZ?rD{o4G5|ao#B`@i zbhAV1Cp-!6f)iiOK>=Y9&@AGs6!y6E16&5fTfO^7=|-Dy1&Fvp`*9YD5k9sA5L&Kj zdmK#vM?o8s^%>|+>-?BO*i};hq}SdIJxHVytN;xV3KyT~Yv2aaUvJ<)#vk;5tj7@R zUgIep{FCo~1OgN*@NghF4(b-sxq8B-wqUK-z-UuuRXQS2KfEQpA3(2fdyE}BUJ#@~ zeA)(GUJK~fJrx6%EP^Jo3)}3$>Y+r{HqfFT?fl5MhHRR)446}9A&8*#>9;e2-bV*Q zErVb=XzWY<PS^e=7l<950d@)BpI{tU_NTkkZf89b{PqQ`2JZzM^!a)p?WI)!iB{~k z((N3uPE)L9D#D}Im^i68g~sD;9iJ5wA8CR?k6wb3To<4-8WlPK6)Ls1y3>oPIX+zB zEZ5+%>)qFI*mKJp9F$>3M5xki16`0qT_A;nKQUE_ALd&)dj+%eaHoZ=GeCf%lJI)L z;%J0yN!@Iy#@jpXu0CudH;Py~<iZl=?Qxv6AM0Rfz_|!kdNRUk#V~~^j(*Up=cJZd zB-MbGz9Q}^?PXc^*r!9b^|YGFx|)$ea#Q|zmm*d|#P*wDz_(=uLW7n=r6@DHbdSN^ z{sG!;VKi6;f~c~?mZf5}e!kx#zsBj9dwP5v=ccB^m7Lz`IepN|q81S|uTUpBgmAn_ z&4qaZJ#W*}Rso%Xr}K-iU%es|H2HpR2VgkSSr$d?@?>W^tCem2KyXBn_~xwYP4;3( z<YDIW6~|K<<NDGCk4lLLH3%I!D42zMV!@KIZ^j2jto%rjkSwsH#2f`J4G}A&BkMPf zo6nLXA>eEnAYH>5X>k@CbMbg&M{6P8WneWl7R<*C9}ET{ca1<z@H^n6`ck#`BSx`h z0Kz0@$Z7YvSfwOy$XQTGjDi@8Gd-MsVe91$4Dfz;LYwrhQ#>+yLQGz}6S)Ab7Et72 zTKrT;NjJU*Ky6fz;AsC65UBTc`S&GxCLkHF1PRX}{WJRqNsdlLU!LAW+96(jJbjn$ zXOwQJJRYZ-I?X0$=oT)3oYU~eAzU-3s7p=}mY_bK@x_*QZb6B#`}qa8blBVKAJK4> zN|JOra8|X@r7=u#j%xyO`m`DxpL0{QW<`zx*a=s(pB#0i$md*l0?1yzAA+`edwFDt z%trmOM=K27FVezO`VnF=Xpd|PzFoDhqY!+<!!>}iOyob#NZkXK22P0PJg0$M5S^wu zVSig8n*STw3OlHqgoA>^wo)#RL1SFh60!jYDzbWXl>m@3xr1rfT8?IHN7z(ug(*Z^ z7?@o;uaD?e*9g}OnC4VdiZE%9TS`@mx5hok`6)y$<0GiK#sPC6j4H0%%W9w?g)qo0 zo@@U7BMDzWsR))J^o>NTPJT`f&90+&O8`AH;W-VEJT9k2PM1kg9P|1(zXCdDw#Tui z3rOP4YnjOAl%Let%uIrM?O1bsR6*#xYxG!>TdtZA2hol+dRTjgj6lWk^?J%mNSGhV zS?X2bP#Clq-<zg^W{>_ORqAJW>IZIdZVT6(L2o`-NoJwU8XT2lP3Kt2_1O2>UoF@F z=rco^akcT9f^WfpHegL*J{CM&8PozF)e5UXG=N267PW4`d(Jtvger?EE@;)L@hQV( z7_DO?3LspC-YAV|Vdq>zqI8oI3-7FkBM`A?P8iFO=i>*+o}pO#=4<<Avc*Zg!<9;z zcdA<jh8|1SSmeb!4Vw4SgG`@z8=lkYbS3+Rn!Z^|8GR|AeI3I~v^;y^o)%;Vvc+A! z{MFi`tFo{L^;?H!Q^5`(>gx^cVxGh^HQ3_e!CGGR76EOX%#}O4CVR8(dXr9vJHg9S zN@LE%7qmaf0I{V3$ER^<nf((vFF~5CUtb|l*}C&l%{C_@UDhi|9xY3_J7~(;gZ5Hq z^$%tv3<sbf;UfD-TR%GU#%Ty@7CYQj#b$e?cNhQ(fj4GXO&)Jh945QqB#C=jY~DqM zoKg-@A2Eal>Al3B?2aOt!Z<T_*rn2)!mrxdtN0%BqyQg`IzvRsWXAKvY(T8zz9cc{ zqGs2PbXI^On>?lt4rKONt);`c$VdnaJ+WCWU;~Y$(?>1f%LIiuil8UEI?O`%c{ivc zCJLKb<3}&pf<OJ(gXss1aW-i_dWRziQhj8&Nj}r-!^^bsT|m*(m6d&o1>h!(LxVMQ zr#n@lTYgb}Mg2NcyxQk%aatT~GJ-ubGR{H|MvA8_NGNAf^*IdyL}N^97-6xDRgDpj zCZ&L-VhX&_ibsiJv&DxNUHa8z9m>%+3cI?+2#(Jzk<Tc|j=**!njTH5a~YTe{B@gQ zi%&0YdueEB(SBj5G_QD*y0|j^iG`KKH~LVlH7OYxS>n|Yrqt*|U~X2p3};3!Fi7$U zakK(~gMRlOJAG>!>*;v*;&TZ=BYD52^Ll*Gpl?|4MP@z_ds#z>%kk-6j`_0+t<{ay z6LY;o&x57F&8g-`KPTn|+0XPhv3?Hm6un)Vsz(<4`ricIbG46vYI8GuAS+rfM6Q}u z6$YX?EAS`*qxQ)nJo9)Hf?5KxcmELY1R%2IVP2LuVrCY=cLmyu+9rt+7J4s#Y}Bcz zXPL}Lp8brRhQGp8jq$P&Bs`hu$<pM*&c%G93X$Kb$~zoQk_l<jQmfEaXah~*+Va<n z>=iA%RU-C6;vKVE)?%MoM+`~4-*^BF&daV>A3l{Va)q4xg~2y~4bM4d1Efkc*StHG z?tP*LE5z>Y%F=jI(tm|tVn$n(OVR<aN)bXhfng6OdvyDSxD(>kl|iv$7i98b0v%nC zsE;;>GC9-21&N{bW$Kf8qYv~zaIw+wXYe`eYdp?3!%v<wXGc{<ON^N%1zkete$l+r z3pBksEATCuHl|Z*NFR&)Sw6bn^OtEugFzz}{fDMEM$W)v?S7<IBunogrC@@7HzMQM zxpl12R7qzW(6mOdRnLo+WS@Jfs=~;|BGovSurZ~$-iI5l-C0*%tiphSx|Z|PsC^4h z7SCjiO?7_iM*HolQ*fth7duz#qK>x62o0;&guyr76sxWKO7E{HXbV!SY@~nk%E~Dz zz&S-|x2wGvt&t;<ZsKY6F|jYwq962IX0RDRfc3e4yYlXK@poho3IWg3S_vT}yvqIf z6YKmXKo4q>zd9%C<`}uOerr#I=nSF-Ie(Tl5r>|oEaAz<Y<nqdz*5POB8%$PB{;$X zX?5ptvl}F?f?`ffM0#R(xB-brM&HAJuJfO283K}ojsSVI*t=On<_dK?5Lw=6pLo<^ zQ$9lnKQB?7r>SEFgTK;m&7wMYomD_DQIPn83zAXL`Xi%&<@>GKT$AqJgMAelW?HS? z%$al?jn-AJ&*NZnDgENL)t<WvU`G^%;Bbc@U7XjxIWSRD3C-Lmtb^&jLoq0S@?96O zo4f~aJ_bxo@fL@~-Kn10iV~f}F7fStqrOAmQyzc>@Y4+S*)P^~CS8!ym&BXDN}HM` z>qf}hBwM_t-mw%jNyXPl%$2@HkaK)U%2KW*9&Z?)%(>Zj?-09FdKUw=c-m&6`F$U_ z=ljozdsX&EzYEePR>quO*wFW$7rLS&BQ&KZQ4O&#H2XIn)z(-HwaEIC3DNcZ@fGlX zraaPS@mJ&*3hv74T~3!L@cz!lD$PS+CX9F}y7B#1ADJ)SDZqh-Tf7#T4wV`Pww?v) zAO_<`-{>99LC{BMDKLslV7lD4M7!*Ar**7f#7DLNUTgoa=@7vO>gz~M!98eE@9RdV z9JG~h)QFm>dtL!%GT|u3-XqP8V@^YL@kP)=`1e*;Zx6z)7<%=$3A^$?+_mvIo?}S< zqS2a$_~_7fM<6mv+<95oeFd&N^t?lyR?NU{K4lr{Pg&rHp;5eDn~~)QA_sZ_x743I z*vFZy`AD*wOf@`*tH{MYYhIn&eXX8;$dt!n{Il=2zCr<*4!w@Q-)b*<5@U_Z@Z5IQ zP+f<(cGyHNsS|L|^%n_9z49(l_bplHrXi$Z0x_GUIuDg|HbAvN1JoFyaeJg<$6^`j z8H}J3wabFYhG8JO(6b7f@Jz7ksuls<YtFbfw%U1RR3K#iOeBcrp&&0(R`AF{vl9ko z7c_G!^2jCIcZrNXT$w`C9^dhD^VIm{>v!dBc4xVRDY={qbseOeC^Fa{Fl8av=qz&M z<Em*(pUL`Bm8$wVpsQra8Uq*+@iZi5?-5!kkdk>Jw``Iu%}AxO5|#YIAWHjz@{%`f zK2N+R<O?po7>l;jZdElt#K;SSSQkq%W_L2PlNrIMGDySbh-<WZ6;8#)0W9NJ-SSB5 zrX~wNSn3`WWkk`!MC+jEYDxNV?@7K+CGz{0ejY+uLL)S%w;)c4PMK!GBDh-q9WPK2 zaM;ghTIHth_nvaI3fq68sW5x5CBz`L4`$lNgw8jTVN4P0r{~kxO=)-JQEhphC#e^X zFs~?jb8qjJ_Ve|8kk(tM{3O>RHLGUd^72yF<m4ZkJw?San@g88dn__AqlNcj{!d*y z1<b|!roHYJV~kU<3F&)@Amc6J>XcdsuaN;s9>We9j_c9hERVQ$ls1ua_1X8K*008P zghr<){n?8`iuIU<9AD%JTMYb#u|J4OcRl#DYqe45dY_dU7eSu5AyEOcmUkk%#kae) z1iD!}9uxi#P}?&5l%*gZg7><E#p+0On{!s%6D?LeR}O0T;3Df)#=Owk2d-u0-|X>x z4XSm|i3mDa_g3f7Jgohxj`r21R=)EhjtyJ0DAL;=z{5`U^W6%1BtbY{-M)Jd2kbQ? z%g>oJk@-?TJo)t}Dx3}vixNHWi3qa~F9V;?Zmp<VY4#L+Y2C^i7mWEv%$>fflyU!c zRINcP*>PgYwO)brsEVUADTCa%It#wk(H<l$v#A+EmPaAlvMgHs;;%>stXMs<-|oaW z{i&TQGHji!FS7+9tNNo3iG#|FA8S)^MzyKglS9M74k?{h(|y)j-~Wa_!6pb~{k?}i zC?r@$6K7-aVI$t4$-kNt8~<bS!>N<XJMt|V;F;J))+_~nVI)0yJ}8`i<9B+Cd0q}* z22a$$Glb#r;PNL|eJeQ$0scqH;<tc6&8=nA|G)_TC13<cmc*ejw<+NH#}Q8Yk|1pW zYD!mO2VDLOP&4pyN6|&AQ$v2gswjodIG_Q8t&pS7#T%swY!U-^0L&yHdw^Y??Y~!f zGPKJ|1>r_@vC>h0-XF75cm^I@8H`Q?Bl!wJeElJOWmMkVD_acW@nYB~N9tUiMPAjl z{=|)aE&oAMwJr;;#bZ!s8pOpP#aCf{Y_d>8w~uy8rWQl%<$RQzM~&&5M<X7Ye6Z<P zcGB6l9EhQ2&A#4UATv-w*qj>iNoL!rWQu36s{L(<ea?fDss7*`tnl9LsO8&J?#q!+ zW1-=4Pfa3dqx#9HU{7g8V)>M+FyY)@x=K=5ES?97M<HbD#73U_aU7l_-s8Ca<llVD zdBQ#eXQLt4cAi_yHd)LImqQ0tBw}&SZt93Hkc2Lml{Dk8o%GxQ&8-0rXT~Kv`Rg|O zW}jY7I^G&`CQzQ(9A0UO{_=?Wq-W}zeWs~3vrG9e;shB)*fVzwgkfb3-UyQ06VGTK z$`etad@XZtz?aXxDZU%`>vxddq`Pf6&F_Wq{@?u-*)TxlO6F5zx=tgz`>4;K#OWsT zNU-~up~X{V{)FFrHAu`faCk^R7vSW4o;nec_jRI0(nwUSpSknv^?aJ-V}jSw4xU&+ zzDvZ3Z5xuDyn_@;u6aT0_xH`PBNcH7PP(DE1VS-5u{c6HL7^<aerN6`9yf7^aptQx zC+UXYo(AXkG&9=o*DpxukY8t%(0P4^<=4snuZIuzf9N1J)FQmY@~a?Sx^^Al#z{01 zm$TyTp2UvCYWpbS#G>4qn0_7CQ5qKkHm9SJFwJIOKM5hYrvv_~GuK=0H=QGY4M~Z! zf9Gg-YM$UzgcH+=q+iDo-riC?!L`3mf8e6+7gF|q;JsZC!bzvWIeA|t-?potAL7L3 zNH<aaK35@J*ntfY8uA)|e&!v9pkw}~Y~aZwlZ4<ssZYya80`Z#WT3YKHgAlKVXuAe zm|xlQ#N93C)Sxk~)+=f9>*%c9<Ba3L)ZD%L!^w5A`%ztuUmTZS(1GUM3z4#CyC<vC zeT}J+G@oyd+j0K7h@yC)qAx>V@La5xCQZL6HheCA-YzNpu9AmT%~<m6QKQX<UFO=D z_%?R;cX9aNwOhiJ9(rVX7uTQvtiKZX2#;)6ZG|!I?0lUMRv6b0SBxjtf)3tn8_?Zv zJJGf__Yk_whsBcmzb?~|6c^TmZP>r!M8w#IAbKzo<>s}p;kZ^nanh`Z<D1I?UdP<W z@M$RF1Dbil@a7pdyRGEoKI5PKjubp*6@1+y-FfoU`)d-Teh~rQr%UQ1XX#Eni@!c# zjUk*P2yHEvANVhheCY}}^#|&L0Vm#}zfYpc1Xl?6sl(DEfs-%l_gDHpAnhP!eblS) z<it7u_pjm-6lj5$Ye2Um;^ce$_tiZD5A1~+LiO=~aT1nvV2lW$d5uphq<^2o|GmZk z_ZI&>)PMc-|F==>OF~k)JNbHhv9O(2r3>hA`vI+1{B_%ru^M1A5GS$-*w6>1?jK4c z-@U8YH2~x$UHujSPX+)qHIQiZ4lPL~x*1BrB33ytExtd;RxuSUyjZ!>R(P6C8k+J3 z7w_<hB2lP8>0l;vNX7~2ke7>qbKr9v;N_yEJD)VFtEu$^HU{iRqk#FuFZ=JdNMenK zs(?wAs>SarzzQem!5s%Zn{J{A1d3JxepB93sWc%ov)Sut&onC|Ll!#k`YAj5<1Yc^ zI%m;fLk=U5-Rbl?SWln0HCDLvn<5S81|WmXfcCo}L#iQJHr#S^uG0w+(ZyO(@qknS zAP9C*vt~bX39v`JqY&>0M2e-wS`6}xD#pes^2ZSL@3;Wb)ut~1a9-Bw6!+K6|Ib;h zqYrP-iuvN&zw%2EoFTeDmjrAe*a3(baCLP}ZKLB^BI!IM*&4N4F?QId2(;JJ6Cgl8 zMV`I<;*4(>kggUF4{y&?iGsDcJ6Nz;0=~#RVjXbS7?bRCA}+Q`OCtYv`GL#<Q$<H# z;@3atP-r6Um?wZ3w|=na%1@XD{$i;&>)lfMY+MO&Ej;~>Nxc7h5pYuiL36yEoCU!2 z&=7okk^~}MLB<+LUKE+!o-YHk$N`uW71QnO=f@z$babCV&=)X7T<x)4V!QKc@w!H+ zC~giAM*{6`;56px3=vtc<&s9p$;oqoTG*PwV`c(efX<~40K-B@>a9Wi;fyonuLzom zPBduY1Y5OOZv##}M!FD)@{dYdcq_`*Id3YMdIPZML+Xx!Q4q`$$l>IZPmkIHuB=Q1 zB*&EJ;`?l0JKkRdsU%Ii0Glh8^d}mC914Jm%Njatj#1K*-SNP5vXAMz&0kS1hnQH? za>H6Y!NcR(cyH@M)AkaO65iT+9eHpq_eot~xz(!&<_93R#Zg|$vTPy{Y?3rK0QhZ7 z(>T}E!@YKIf%8pjl3Q*NmIa&_Rxy?ZFBv=IE!=Af>7Qo1w|cFDQ1agTlmRAw`GBYo zkL~Jdmwv?9I1TkanV_%P4VdqTsLoNBDT;1b(Wo0{V6Bm5<-3~Lj8uE^qF*!XJHQE} z!S)(UZJxlL5ngzIZeIv+n+yY}um83wV-3Swx^-|REaEJ14gH$`kcDJsna2<o*JD-* zX(<B7U-P1x_+vd#)UWFoA&U*gBtvHokyDPSHz1qK>RMV_6j~_KC3V;;Wr7%H7%VXh z3&UPP%9?dW(y>`@UA6ksHIs@N!h2V9J<O7^0D%qtzTKa1q$h=9=q?3v0Boy`SNJf! z*J_`R%>XDWa}1p=&CQFT@{|B7aDX{6&@shsB7h#%Fc%hf22GL^e2R?o9Yb>O+7ke5 z+Yj~Fv8r}F!o6kM<`{%o;D5{e{r!zP1b;^XF?gs|8197g_z_eA-&7lin4QPWfZXCE zMB7>42?$pd`gXQddJSaQL@$>C8R?KVJZI)YR{kN#gDV1Ri7rv}Iwdd%2)4>0r5$Iq zVz2eQ&s_zKWIr%XjTjxK1ygbs3R!a`7h2^mAD+T#j*q&geAM&uUeL}f9f||8h0At; zW<pUiF<KS9#ZcP<xvvfaQH*((pvVYSJYjF}aS14_y*!>s2X&i;xr8#Xah1AhY8%<V zUs7ELs{mc%zpKYv>T}l(CF~=%ucdvQG{G$crEbJ=Jp*T@awC{gB{vu^x=mjaGX!^Y zhXq2`iPXTOI8k?PPtPiNXrEV5e_u7^f`YYWI{2j4q9e8#jK{zx4Mn3>(nnl8u+c0P z9ar+_AZfcJD3!{FT)fZKOWO+;4By6s$-$)wSft;ad&`uP%in7*L9uD^0!`wMkXU8g zg2miTZKD81+d(B$MNC!Mi<ZC369D|FLY?V*5DM-z2D@>yMl(;$37FJS;MUG|NKqju z_L{MvY%ivcj6t@WQxvZ-r8S2+hZuunIo>-+N+OGn8n7bPHKjm=-Tv|3Xc5#C=U7fh zMftm#R;efh;MUXCgWtg^hZIHM=BVyWQwt9H37F9lBGN+-0sOxl+{sPrz26<b2<g0; z)r(Ya(Kg+Ca&)kl8jK5~rJ^7wmKC;+<!D?;!c4<=TYL2Lm9J13@-JZL@1OMKSWD$+ zqQ3uQ4cj5-yZXWr_8r?f%WxRW00aiarPoA)YNX*@Z&<`f4>hFLVQAechMitq2}z2! z1rp?#wuQtz$FpuH#;A31gj`3_esc&zdF1Ti;NVkk2G892%@{Hv{qA;nAbwD`CKE#t z3#CQBrn3J!@CmM%JnV4bd5J_4IkWJiL{&Q;;*qJVtAC)hizVy1A;pa2(b42O5c!VJ zF-A@fRZfZHR<OU2Rbds~n~+k=xJP7fKy~CQKPhV(ld-IGCO-H);8BDfL2-N-mzY9! z_RCE7BCeR308Xnml9I`NsrVh{Y@#C0f?1-~zaN>EU|CX0Q1f?>kII&?Fe;RmPP3Nh z`Dl}_i1t}1-J0uUbB#7=(WhvMwm$kwqvVL`gqVPDox=<nd59U>Xn2H@^v55*>&uJ- zjc?9RNgu)keSuS_oj*}m(tHnTj2BG~DSH@=^G<Iu1msE|rTX#Z1$Wxx$^pGFB5s@F zf=EB}IuKbIi>P&vVik#xSmRBstp3m<tkJ9SeEw>+<i`cLW7QYnOEhj5*?1Fk*Py-l zo|L>)rj;sUtm}5So`IK;_n6&97_Uu|8~PMEXGj#4GP_S5r7(B-cZmg8kzc^aUlkKt zQoo1;SD&ak8}V0!N)xBAFFrA<vm0o*A4Fk*?#-EP<Zy0nI=BO0p(6L%Hf(S(<di{? z*<0a1QX)0%YwCjfAt&BYY6I92t7a_L9V)IWTR^~#z;1$#o4moB5;hd=j?CDG-LLc$ z!ZC|DBIC*)XV?eAb`wE(EhLj+wEQGl;9FIGyAE5k_k&R|vNClT9kCBQseNZCRwt%Z zS+c$}Bpd^9GId!Ft4q-b%zAlC;BG0jT^*<TDCUWBPcL4CP5B9;8zAXW`SyHwBTMze z8g=^UM<8~t#z|f>UB5Tu?pAI`mAmtXJpJ6>mw=b&Nj7h*nq@8{6crVNh1HqaZxYr# z`~qrx`16Am?_-A$<B>3A8>#6>yFf?;XC3Hsd@w7{b0_HEN}n&WhO+qHeDsre`gxaq zx#>EJjv#|<tm}^t))AEq9W20vp!rA6=@tJz6}0y9-rk-y4;J{yP#J5*-ar!L_&mc` zGpWD$M<v4;#2=zr^8){<&$u@9oPyB=lT#WO6J-!fz%O(Z$Y^al>kVdPYf)!~p*%$* zF!dGd(Ca5P#*jf?te9v6whGYOc~?jY4q4tpBeTo+t!h${d|X_&as=~`MAHfa17Ah0 zm&L_rjzE7ho0@nG+@^YIII)`g`lG}1CGBF^e#jXI+NgZUlbK57k)t%G*71;38#NAT zINI#hrFH;|zUT#Y7-(4uy9+@7unoweiiUllaxBR(duK5={VjY6FjTOV6d?_^n%t1b z+&@g{lttn94!QCw|5zwP?pyQnYwmES2{D2wc?WA!{8qUHW6Cd*v8n>2E=K5!2?jM@ z)=>eFeWH!DVzEHJup0|^CE{871Ugw1=B^#~1KyYU)xSpYKbAnC$No-0-RhS1cYaYH z$s#DSPQa~L(b-|K%6#LGXT80><qczCdJ*7EOI;SnWEX;c5J&A*GgyH4)Y==;Y~~IS z6e-O@8|kH9!@$B0Qe&*`yJz9ru0fj`QyAdLF$OblCD?KCbgrZ(Gl9O#1R?zxDBRW- zm6}pZ+-3A4zq0@aOygsjBv(wC%px+zUru&9?nD<D#1aSJ`SJcKehou6%(F7pLmb=) ztKHR!-27e8%iDH7eRb<&pmenXP<ZlM!zO`PC>#iH(#oay_eY7vU+J~ityvd!N;`pO zBU<Q@uIo&s6BsMesd+Y;6MIkxrsE_*F}WU|1kgH5(0fFiXL=!Ub^%en1WD#%xqHq_ zU}?X&+A(VH8tjU+rXt-`Tv^=aE~59OysFs(gox7%11xz_bC)1rsUD|T@F!oT>T(pi z9Ce}h+dTxkU_ejVbb94{w0c#hkzek}!$Q_)@1)*~dsoLC-bHYzmYe?|#QTMv{p+N3 zrioO#v1bSPe$99JDu#mt&AC-s(%hF7t#|pDWL(@+8|B?SQeFBXaI&cQ^E3-*PJxIC z-10Glhh6;p!P1;=RuG!-_z@Cy)0(Cx=Tn>~x#f6h;uR`Ga=h~d{FB<MpR)4#5zNl8 zQE$)zZG$mhVZKx>UaZn5lLZ?fCnGyhE$wqlC0YT!e>o-oD|vV^*d?-5WiT=_l9#tt ztRsv%n9rkfvQKPGB3p!Wxkr(5!l6r4!xRr;IR}~Vb5z!v-BpA3CS0Cof549t&`U|$ z*mRaznDvfS2pVh-x0zF{rradgC)Us_LHmO3f%u#A7}|OVq;4_pB8`gpB~syESm=+K zbF&-~dBGg71S^N{?Rp$PZV^ux^NihH;@QjjK+)rCY`#7c=5pt%Fkm+N`iIZMO1eIs z!Su2536slaP(P-`GgkF;hhN?4vq!Sa5(vdgth*;u__dq5dW7`MbG|rLtZg{lf4_FD z!XzutU~u8~?bA?=LLL(G`=?hsZ;10?1qu!~?vwr9Dv)r8`!L+~ob7vfdj4Dxr48pZ zVVt?LREEml&lrC(!!#!`(7npjd|3-<R~@d#_bK5D$=Ex^5<j0WbZf4WxRv@7zoeIf z>(f{A^H1gEja$@~AFdTUVxc7|*d3r0(WlkOvv%Ec-_afXb7`mM*fWVkTQ$`-uDiEN z$SYZ&D(H-rOCseto~37R-MrvJl*tXX4!`^TzW`xC1~1YW-tk$?QkDA~`VPH>JpJST z{T6{lkt=d>PPn*;kGE$cRCN_s%4aXc2q7|cq9o++{(EYpNdwmeuc<FOMfdM}A&Z+M zqx2%qCv{gGG;qU8n{rAo|85!n#^4e;L8GgZ<Jx>O?E6<s7K#7}Kog2HMdI)8Iq@eI zi1$!~$Jw7m3jE_QpT>e&y&zZa8uh<@<^J)TK!K(P*3Z<v(P_i~^51`*<iB)~TI`|m zh%1S|&o`77&!&2#z;ky%M@xl{%ohhwo}D_RSI5}(`#QNpPWm&i7WuTsPLgE$-I-{{ z0Y#l-u|vg@e{`OI9$Z2L0$Q^&!@vAl8bbB(fJweR7f%IPWz616Wap0B{~^illgPw% ztrgyRhw1Of{4Zxm3b0+<E5FK%MB3SYJM73(2ip!N!j(-Gzyj5Hy^(e?yH){Pi<*Y` znmiU`Utci$z95NE$jL9iWpU||_A4mS`k(blqKNRl&aJ^MrTKsNu>XGSKYuEC05D)J zwGU_i3)cthAcX!n9jb!e7ymI~{bN-B-KY6-p8?OFPW|fre~W<p>)*U6fjT<9<B0zh z{qXB6Szi+XnEigFtjhoK9B>H|c|x80v`qE1wY3+{+S}WQF?_gy@mZ~&CZ(W?9(?DP z9mlt{Ha}^|2q7>R@W_AkXz3|bZ857g!PVVyXiP5?(zRt2e-l^Jp84{=-pHgdt#>f{ zoBFbyg8Fdkfv)k41BUy8;La^~?c!dwhMe4zQ4_`7l7fuhHc;@bRlReI?r&hS#cn$- zf7gz=lfhYLI;FulT-x2BIZ$aBBWB9|QPZ^Be>QUFC4UPjklELIHCU2Yx(+hB=!nd< zs|G3yv}11MTi8X1pE2KF)*N5IzBWXbl@qLEp63xGoYYXSGtARYM&@8O=}ar&?Ub{n zo$#5HuCLslvn_1dRdO_QL@8vHkXLD{M*FGy>&d<(rD|bQZRO<yKV2@)Z^hRny-bK* z7e{J3Qd0PhB=;(|zL(zVrZJC-_@U($`s0}mU3ujW*?Os&WUK<IrTULa!{awEdj+^2 zl$yrlWl^o)n81)_<-X9_8I_Edyc22iL?w=vj@bO@@%gWMjLBMty}moiNbY<TYwLx* zHjjp978tq+DOa^odY%_r?bVXezBO&FtrT6Sn5OmVdQ{t4#WTl9#|ZS*x7y&N*{rR0 z>KbZHF0?a)(<@whRo7aZ@88%FxgejMvH#)xbPYd;gT7aA5MxD?-Yro-e)_hY;tX+h zEq(o9+H0-jLL4z+t6hkjD&`vOT|I~!q7MqoF27CXrvIMnfJnwjx~?KFbM=OMY`%3l z(4<|Co<F_ILz1hLqD381b3HvNLy~!3XyF#JLr?0J<_dlD$|5q@gn;tO$Oe8MlPMCe zeU-Krk%uveFVkaINGHoZws2;i`*MLLH)&k^5%V`W1pnT1GR@;Ee|}HqdjFOZoe&Y{ zs_!I|k37ry{au><G5Y3Ff;}pVQq1%E%F!qj$M|_vhiIyOdyLl9!h?BQqIt#r>MtWQ znMs-BgPn*3tOB~|J0kBHCOvTirR<7eo}FROHScxTQcgn&(xFxMu@B7W_BYYy3br+* z+#W0ls%17khY76NlhwE!x3!(Od2BO!X#cFF@qJ~l?opZAz2#AgWj0M2ik-K)QhM|# z%80jX2E;qohMjDVEb~W7qcYuoa251@!%XzE9_)h|+Z`3^_QbvU6}AzZOtulDBe6%Z z6+#weA!Pp82Z-d#Hi`v>JbLw7PyTCVsfV7I9jYiVV>rs35fq&I-YhdjJ8vtT85h!b z$O4wP8b&I>*XqlURAE+<kh*R;i_AT<XLeLVZ9X8V^w}Kzmw(JJCp~2*M3=6oS7MGx z$7HI5!}9V_&gwD=*atgr^vMy-3rFg)9M4vg2*FpX@2G$qDaK1as_D1_ZUs8I!r33? zu%og&YE>3;0FTvYbX-xiDRVZ?a8K;?<j<NABO9r3<FU`n11GYezLTpj^JQoGcJJGo z5Wk|W{RIQemmO=p)%|R=P}TEq+Grnr%qxb42Rq9L<Jz6DoE-*HO|dO~B<7y^oae)> zJnVy&3^(yE_k@ZNKeS$b%8u5~ex|!%N^rCn)x*lZJv2JSlj`FXxL6Mq?}0l~jAriW zK{YYaj>=x;cr;}$Ify|RayP8=R69K^Gp5dKw-QB^m*-jwcbnSz%-X6^O}tQSt#6w% zx4*OEd$a}HyD+zq4z6K&WOKzg0n0HXYO7c`d$vjw)qZVR^tzp1@@T~^e=NtAXot6= zY&CJ<O49ho<H4QJf!OplcS9>F=Dqf7L1Xv`mU(T`v5F&y=E)riPes|v>V0>?Zg=ED zx^7;c8<iz=@AP)XUJ=IM+0=fwC;G`A{Pmnz7p6=<sV-hMHCWywPFZVHi)-yd|E?HW z4e@^asSGfnL?cJ4SYRF|9iBtO?nv*PZc`lRl~nA<t{!(o+m#A5dR(k|^TM3Cd@8uI zDvrGi=|W_=^9!eTRQ7j1H$a0`3;Ym0UYM+pIWUzyzVLqcW0&oUKqc`$*6iiZ=kdbn z$(_$%;juNm2eT25E56mi2N?M7hS!4Ublz)lPp-wQ-d*wCAG$Q+ZYXQMTAq8oajDz$ z+)-uTPQ{Nge1Cp_wB7NAbad80ttf*R)tkNcuxsG=E2qTe_je?YUA^Z=@#(!s(Nzg( zvEvJK2AG3c%_`yrn|oD#d+w!(72jIVp4AI;m?i`I*BFjn?hhVRr@Bsgj<(y>yN^HX zz+i@1tvrvqXIZQ00>|(VXU`t*XU-B3pkDbLT~(gD_32aA(+?g{Px1C_dV0F;7zGCV z<nVI!wU?I(1AfJJ`Uam>o;rAgOi2Btv%1E{*WOIur=z%9k?x+WPHcr$uch~>AEEJ9 zrkjgtK;~Z&-=+1C!34MRd@oHLcb{w~n?Z@)=DRI$yP-Zj{6*T;-Ah?lUzC2*xpbkF z&y4#aQ_NJb`r!Rq`ZDzeC2D^AfT{eC$j+ND1b0WTzBN%<O{uB3bn5sVX2wt9)I<D| zleo6drBZB{S)^kFsnTeFCL}Wdt5_+QnMa0_2-SKH+i<mE40$b|=#Hepqkwo!zW4IZ zUQ+LiEa_|X43p+f!ElnzS@W6;SkDYsXLAxE60i7{^f>m7@CQxIjFzeoM7GOG1^B0H z;~7We`OTW1ZJWoCABm`&Q6J<l%dQ&SzOYx}aF{_ci*h5GR9uu_++5>lTG&o;n|C2R zv=8wgt8JV<WnL!b?@7@}q*SFTl}|TYbx~#JVxO{%JQJq-RG0iR{L_}}PE;mwd|-<W zbzXZ#^95dK?*4Z})JhhyyDpJc^|M!+i@zeW4q&?Ua=PZb{B<FoULHr26pj0*;1$A3 zF9rIWm1|cvcHWj?8(-o(hPqFeU|qsosn(N28A7~sH_!UEg(vuprHzqIU^x&5H<_Jn z>kyrxCX>VUJL+G)-v97DOR26g3od`r&H%>5cpx{;_cr04$je5d!6kT8R(lB7vu(ny z@FL6GB^l^+H-mVJt<WN?be$_S5gbNdej~gqs9+B>wdK$KGn&HwF|DH3wwl@#1z(Nq zb}A>fZ9|F;YTe7(YrO}T4fN?5FK_dT37ifOR*1gb{qQSh`wGRGwW}0<FNzG-xSBK! z#<`MHCtAY#V_wM2UcK9ldH#^0Ytwbdd>4D;yppZ1)Vj55BDdHI#U91>Q@5aXV-@_@ zA&p}pN9C8cUANHhyq8<%Llmlyh!N$xC|fNDyWNeV`{1q6F}rCtHtE7(smJ|tGaRj~ zT-mVp4AYL5l`1;gs+q`Wa`5mZdU>VGa+R#)`4q*hOb8A3ZVAO2_Y+Ls{MUq9*Vh{r zx8v>wdU$eDu0%dyUzKeLfAK)z2qwFgI`yckNUwOrC0<F~u1XM<`skEL<x7_{v-9$- zgz=j3soasl>&p~jt~-0(l9C-xHSK8=Q)_!`0iDHpie@i6c43mU9C2R{$9JxDtztb! zBgcC9>BOlTX(D)DTyph<O`~<#F}21V^(7a@Z)z%6AGp>QO3!F8$l^PH60Sdd=hnVw zG?V%L%E6gNtS6~|!%W9%-XMaPElQKFU+#u@-tiQ9y)k+_*RgSdi7+yMBFXAzkgb<) z^h+tm8JU=(_L@<Yd)-)wq?TT^SP&7PDRN`@@WTr=^AuqXcXgVV{AW&s!>5O)EA4dX zaq`9?MaU*L4jfiy;-Ba7Ia_~{JU+9w%THrB&lK<#n_iR3nASbFuRST+@$S~?y!QBz zZz&)8X`3s=1wNT^6xN7`Z_zU)I(Uw-Pa?D^>I--GCgDPMae^uyFGVhQIim0T&rEv0 zba7O?)cTpN|HEy*e5E}Sapy(>UHaD{3dt9}Swh;Z7f>WM+a@=a55z7U_lu&dx<Zq? z<de@^nmrg>(YXnR27L2Ef2edF93_8!R|$Pzu_NPiKw!~9Q1LE`#EHNrqK&=jxsY{c z-FF5?Cxe5Xk0GIBzt<@ejHJ^K<3!4<y^oK0g^r06;-0#FM777B+voGpH*;us4)BwP z7yV#yMpBgTBQ2ReyUV~Bg*yG*ib=HdBJ~|yLA=}Nu5{+u<kJ!)J+6>T*M1DYQvXs* z5+xV+jU7)<Pl8Pp#;M45hd5Wer}wqg(doIfc+svBN!pqWAX($m+}RJ8Xh@#=`j~w0 zc*8WTqow@eEUvHd*#}-BMUtzcVJqiCuPU~>spy?Aya;DEnN@HgiJ=D4tms&!SQ8C{ zczB7b`l%1l%H|P4b91vUwt8@MtL`DrH3^Ylf*#-P)3&M+QGtP^B}zBH{!e$1L>lcm z{-Uz(A3<+^5z!LcB&!M@+dWsV{JCC2s*;<%lF7ly|IdXMYbG8o`5z^Qihl%cr86j7 z|5}Qfd4q*yQO%9tdN&DHYH}gm9<@$VpV$9G>Bfq}Gw#l`0`0h9m760i$8S|)Q7KLf zZc*v=wO^}sADSlezI)u<7BxrDy$_~tXz1$xfz}f|WB2D*c9A^<&PKo??@R-OYehRS z8^503i1*t6206infhi(vNE>SinBWqC=l28KD;zKzn(!p^URO4m;3CM}1*qf-jo7ha zS#XqCQ3xRf0)8D86%~i)rEcEbbcNo4kJ^qIJGgrOxo*7ckMqE;d}K;XOPl)<GUIJ6 z^tLeHa@(9kPCfY^NW~H-G7DJRQXnqq18luo41n?=Pxd#@31W~57#tZ9@l0?4xDreq zetgeQ3K1KucL!E&fx(q{CdUrRw!3x!K1>h*F+`>ChBxO0fT(fE(4PYGXcq!U`hfWH zo{~F&E3;)nE|g$$D^N>9yTEqfh<<3jJlQS}xg!86R9*)5FVlr`Z-$MIehQ%GP<Wn1 zXSMaLhV4<-r<;D?Z%174$Q*s;F1`NqwRq|;VGS^ayibR6H|-`}05i}3LE?V>r3z?2 z6tJ&XA=t-s*d<iN=TA!VElqkxMt)w+(I8}c81mR?%>i#y(5dBWthK8D8X!(B@7&=z zwm5?L)fT>*VGu1yhv`lWd96xODhHqV5)jm7*23;QZ1)`gUSq0+s<^n=2}C88HB1^W z5rpRI%E<)vLm7I^wx4lJKy=4m^3cH!WPHXarhMpx=K@S(dgVX!|99*8EJrI{Q-hfn z%GR3LLdrH-T*h(H9m~SF9ssmXCE5pal(ya=Au3&uwfbCKvEtfe0BFN|NRidmqirG^ z53y4$$8(dxjZ@3IqUWz0`V9Rz1{<vBWFwKEm(ayC!no<DAUi(hnX;zltPuVIFcQ|z zhBJdu|25u~1tuhNT@>W(r7ALsrAsw~P{u_7b{7m3nqpfIe)dyyU+_ftS0g>>bHxh& zl#EKeJN<aLs<P$jhu_jz3C*)FdWl+a)f$AkF@+|rp8!{U^;}gW{SW3q#k=4cNzk9} z2du+V;%nXlK;wvhFoF$ct*KKKv3Tu1v3Hlh0C<*TyQG%G7fVqeyXhKr0oj(SvNwSY zoV%u)+7HAT=SPP#8$UE7z)cwfT9v_K{LjKGl|~B@6O0>aim?`umlJYjLkWYL){z_a z^ae$MW#+Tm<K*Ikv@eAq?(WXqITz0)BPbfGo8xdbBwpb&%Px>Olw&47<5O+M4BMp8 ziz$TwLS<IEaqb&Lb_n4N0~(-h0RZ`kwQKEAz*R<gfC+4M0%(X}h(#B`cO6{CcE=x0 zdxbH&{6wfaW(Z9bGfS?CYZNd^yvvg%`vrW>LgK#JTo(IZ0?<B^ul}8m`KUhn*%+IK z!Skj}aKJi0xgNQpvHcm}S<BURzcVtLX!p&nV276=)sMEesLdUaD^(r)U+>UBh!HD> z5Qvam1uYCzi@^Rw{z1)H@ScwXLw@YAhK7a-6bcxVc?8Jw0*F9gQuQw^fj=vyf+#%3 zmq%F!hs{0bnBvY$1T``g*GvR54g#Y+yaIT=vw`9%b{~*CF$@8{kF#$V8w52+w@OTa z;UgIILer>Z8y@Wzq^3zs9J>Re-pZ#o<4rE#Udo=bJgzG#)N>xIWzNdjRHv*@QdtBq zd@k5Q(>V1C&Doog^0%&V1Od^}06)gZuS<OZ;;Hh`Vv!1xs`KDxj&4IT_VaW_2J)9V zmQo=GbS6sWoRDL>@S=@3HU^t>`Z!wK;G($r7;wB%fswW~ywP*aTRwdT3&IMmfU2So zNM3qITE)CUypa_M2*{7r(9RezQl$z_V(PJ*fcV^s*|u1W7j~C_=qoU2z86t1B%X88 zBmT)5`!3R^XTH#?|MGa7E;Q`-ERzN_!4hfdL^q+S6^!U@^0Q}uXqR4a&EcKv&p)PM z5MM4fII^!Y-2dTQEA*9p5Nx{s;YKwebcUC7Ok*kh0>fi1JCOAT1*Re|nSa);x3nEi zdmzrC1=gIEuNO2eW)|%PyNXrT*SF@Y+6;rwX{6!8kX-2rUQl{zQtx3t&<Vvt)Nzhp zo;kJ`Cl{Hq*!w;QHjPjmtM*iYSF->Ine{O+X)XbQ5Q1%>*H?mVy!b+8m-;Hj1bBS2 z*g)IEz8le<5))8d+n)PTK$_L-R&T2qt9>r_^>6HYb%zIg8wx>7ARh2mO2P+1r~W6z z=(6|>-R3mD8ir$9er-KtE>2FesPTcmAd@l6!q$guYxF$YPx&fW{dBw}DMl9vR^Qbe z)FbZMvZw7S#WF(qd!|2}YrssG`yB*%ri}GW1GW`AcGW#4MI?m?QLqcf?6Ma>M#4hb z^fh3W#kE)_4nd%pLR^;O=?uvh60<$r05Fsr?Hy<gs+u$;Z;^8xI!bP(QC@p^7_a!; z3VI$+B-tX_F*((e=ASl|og$t;`i%_Fo;AcxY}4YK2ATc<`1H-DA*rS^NS&tyyN!ez zCQsSq6cxEsD^)Rj?RwRoMTHE2iK+0Jy1srKGookB`i_t2c0b>_{9Wvo;3xQh;>G$B zstbU)j!Atx2GMJb19^d%9`o_TwTZE;TvX-~;89Z>41;zx{A>JJRK)-IN`lUp80vA! zeT&u9wF`g(I;}6)2+?N)MbF@p^h}nDKGs8-n4tA6l4u9Y(*qQ?wYHQ#ydDm8EG6J= z9XPT@m*w8}xIOje)^Hu4*wME?BCh~SDC}vP2!gxg!J(Mve%X#jiUY_HK$$Oq4Jen2 zW*$BFhbZAxX(vL3ZttleL86tG=M|I`gQ*S6V~5MmZTAZCIi4{`q<SFqQOE(Bvj;C+ zt5&#Ld-j!y$yy&Hk;eFyNemq@E`Pr$ya)7OKuVoAC{UhBb>Q$YLAbw(=mQFiWON8s zv%&3QVEE0Ysq?|b*NxrtWSGyRB#u5W$;+^nr^W90k*fN8JSbFySBMVme$6W=Jh!oQ zO>tD;&f67K0Q>CER|0^^Pw-wY&oM_ak^X-3)Cd<WrZlvwEmbd6Z*~>dzB?c$yoEbW zWRNSq5{19oeW~;vTyLvjro3wXe&x@jSg|8dkC9n??0w+^YZe1RGFD*{HC~5dYnG6r zN*cHICi3(?a1`Xt?`|bYTLnNdCu93cY%8?<(Yi_Fy|;(EkST;vGlq5<Ei}z5N;?F6 z6(aU56t63UGsh|(={-pENWdZkmk?%#McQ-jdb%+=by(so94M=+r|8K)1JhQSRdcvs zYAgCX!NR@Mlw%$nL88ENY`uQ~4CFV;<qKKoHhfLwJM4+0)i1)#N(j&-H2hy7Qaqz# zq>FsvL8$e4UuX(*wYDI}xu2|cs_3Dw34g~;iV5VvCAFG`^n|IW`G&QA(-*60WOo@J zEKlS9tV|jt5BD;<8a#_RT=>4dnp7tF)#%P80}^5`nf7PqBzPB^1bm0b$qx<=`r4gu zTlDbMlC;!W&2_*Q>*>b|f%YQDH~p~sPR2(AW`0zU&W_l_xCkpW+YHoYSNbp{$^boV zw8k+sF&5&^<x6M6Biwa}TGghcKvp@tzt3^&Xcly-4nc!}Vh<M2?zB-Ds^OPZUE_Mn zKyeJI(?Edr9K#`K374c(FuRTvLxXX==}EqRCf8OIK!}CtCicMTc<3FQs*o2bJ)Zr* z!v0Vm&+H{GOV8)r`R(oiAWr+LmKs1#k20p~Vh%TtiLwA3W%LLV_5_k$GL!#QD)b+N zJwpXvNi&ldZTg!8C>zx*T!ckLMWtF#{%j~EqDUMuaW*NU^xjvS!v0V>_4|H3h3?l= z+PhAF@+D0gzLkDVTpU^l#z$QdszC%mDxr8i(^|Y@E-}kbC0YkWPf9V|PdFV$PWN5j zy5*a!$M+9+9KLmWfAha7qU=WNki{*&t8X%6I5_>~lGsgS<R7=d0|VVc&nl38z;wre zd3srfZHxA2g3MTz+g1S-Le;o$vpih>t<i0qF{?Z1wvD6V+{@HaO;pXn4Brw^o>?Wu zJ0`VDrCnvAqoa#vfyED>b3<wo@cZdgOiFfpnJG;sFmx6h+@dy{My;Qr<hk}PN=bPa z=upGc_O&Sa4?W>uAfc8bdHVz84AnFOL@vV^XRgLrPNsH2L6{?pMA5yqd<Gqh-)eEA z&r9%v#>+vT(qQr&Vxg(My%cos-<Cj>=$KvpT7uVkpS7t*c4x3FT8n5OH_@*d_3Yy( zDR*`3g2Ki(q@1Ec9{ql|XQ>Sz>&&JT+y9zX`^di<ZE0w#+PL1AH&BD!wJaS8mXvBt zHe4Fxvm7kSkYsCx@g{<X_WJv~d*~9dop6N}m8fGW&|=t?G;>N>Hih>3pN`!j4r0J) zbJLOcVXcl@4&F!cyw|iEai7tusSS{i*J}H@G@N^c2T5n(tG)bK3$>jmL9jmCr0+m& z^-wUjTKZ$i`CASAa)%%;wov^0VPBICA#6Vm7@=~dd~L=N!d(Z>kgBA?nTOaG5M|}k zMnPgV<9_6RJ7DwTkNMMAVZ)e@KvCNOqB1N&6JoAL6l(|43;OTOel(If8uKk#S$?jw zRAVv(x2b%pB(nAl%A2*e!a{;3l=DiQF6B7rh8*C1X<YA8^Qee{!fgs#h(Ot(*m4lq z96K*b%p76Yd~C~<5rPKk#HF&`^H|%({Ki2r;Fp^>AYHE*cF`#Os)C#_jf6`BVWdGU ztD?x-z3whzF|*X?wvCilnj_cJyFaf^)Wv2Dg|3f5xg@}-rJ0li=4+evawYFbb&Y%| z8K`#!)CB8=kG9`$G`??6Y`+2$d8-_RKy?kD-4DKuIwJX5dk3yv2Xq>C=zWb~VWHPG z+COK^*!w<DtYTBHHWB~%Z!O-Ri);<PJ7oDO^<q22EjcI@r&-e&$L{)pp)hul+@I+6 zg<7*JcHOfwOm^8;g{_p5_zi2z<wBIYP6LQqwYG5o&PZV*gT>>7fn<xUKQ#0;%suyK zyK#|PJe1J$MOFLsz4Zo(F8*ysGTfamxm)AKsfXa5onUzG47!?b0p<!YPnLhdzFn6J zJVk1fUEU8{umOc5S8^D$Yjogn*e3u>g?}w)NCXxEWpt+me7OC?tw!dB_KliimBP_y z{z7NWHjhBpX#;X3N`n{$c`JuR`))vUeAQ<2hEvxE;vrh^iXK2d*f7`~c^~h1L+kbr z&A<;AQ{Hd@Y0)F4*iAV#EyQ|GKLH}15^?bErv@IQP(tn?y-iO@Olrb@SK7XYZ>v;@ z-?Dn|hjpe$jS@3Izg0EEEJ&|b0V3RU(I^A9mB?zpb-G}K!rN~^FnI3_WNHl@1cK2C zme{;2tYE)2h3@b88MW^wpxkfAxJhMEH~M3pSzr>#NDV%-587&I8QY^Ga|a7lD`f*$ z_c!OmAIi;VQSyiXY58LnCj9ZPcktsptae>^Kub*SxBog;ik1Dbcdotx4#%#zSquQA zl-lbq{vY%_35`rvR!j08oiynoF$zcdU1W0PImj{uhQ%WK=#E2kwn5ET(|6I)3|Z-q z&t^+i?zU{4zm-auL`}@@yJd_$*mD4~d)hFAHg@O03u^Ds+6dPm3f<y<)Rvjv(cR89 zYFvUS$^VC~?+&E8f8XbXj7Wt_8BNM685wb;D0?4UDP=1h*&GcmR947{?7e3wk;vY= z?7gz*?|$bT&*yo5-~T*4ob!Ia=6zrHbzS!^PiT|U!zqKl$&*vE^FweCN_=y{*W~3I zVs_+Jq2JkPdXz}Mg~jqN&G(N+Bn!RGg^y-thwrf@{x-O@&-kWCD~T7g!R@E?!RD94 zgz~Fg*X$B3^2*<w=^X(KOOCqXd|<Cxiv+8vuY_!MOXGmHij-Fb;&>)+<ST#q+rUFX zD)$d9m3=K9DNzXt=a2CjD@D!RuBxuK0QO{7$>x+*isVzCHpsra?P~bI;Q9Va&BpX; z;P{xDPyb}pIj4E`W!;<5Pb!sUsOp=nt5+JyzAWEx2r7|I(A2y?_g-tc*LFo)LH&A$ z?70h;lPB9uj20JV1Lx&v{<1(vZxC-<q}n~&E@SiZZf4`!x!+E5zB#l{f1lW6O+9FF zK9I~ZUmx|!r$w!q{|4kWUztCS+;eeUuv!qk(c)+~VTM(g;2~+MdnYkBG~C)%RaT%S zp*+lE79i@oqf^&YMmO~wrl;=!SLs1UjE?rR+2>QXBXQ}%*<XZv9>#aAElq_Kn8b%A zUd+#rXc*&IIQC}g0+w}F!hivoM#fHmN6uMWep3w4te%DK;Hqt)9h%w3RYhu17MNDr zG9ieRfc@;R!3>h~;Z-yDJx3FU&g>wgUZe$!zc^MaPKx4%bGRkiUx%cf;tsOy9602h zCzNy!<pu+FsHj_;gR^quz-bMP)hmW?x_6v!ntXp<$ukykXA^Ti<M_Qz&B)$%Bxk?t z>(25;92#0Q`2?WqE6uKs9I8(u4-Rk}tb7~FV>9J<szz-N?oL7N;Qmj(3K>WwLYI9S z^%pjbn4pr|%x^S3KVdOi;B~)XZ0$@G2_eW0BH?x0(-&lYlc3^c<sjkPgH~P^J(eaj zJocHN+J3c&l7st))qGwO(MQ_9f5&wGg6*i1V2_LESsBA*hI->|Na_n(MjmA!0x)as zOhyqzgJ*KmcfQ^Xxk?$RTs`RRyUiW+3i$>J91-C}AdLu{)x6=^SqT|(5fK;KTFG0b zPVcf|xajNWN$?cvjGMQH_=AX9L&kkGc7CB=i`1jVfdc<F?&x(YV?RmY0VQzUnF>Eb zKvH+vHT4vv2^@ygRs#y}`7;P~9<9?L<m60{d%0srfWFTVm*~|6j6aLoh0fX!^7GGj z{~nK@6jJ<ruQs(MpdszB%Py?fM`YUi8Q%zfqr6Dw=dF_-ZKF8Of81|)>d1e^OxL6$ zq{E##&Ls<K6upr1S5?{>gMe2sCIM6;FLwA!v{!K5kneB%zly{>kom9IHu^C-ocO8c z?qE%Emns$6v7IH5c-|l+rWnPz42Mij8g~v9;oo-<X`rbB8z!&BD@w~zhCm!+LS(Jp z@6IyeHzzl(Pg~blA*_ZL^Q%0P^8c{8pC^<?l44;%*Vf`FcaiK-a^iqJqvS@71mwG# zr!rgJt)ph%!-#n2b|%ai4DB`LTlMvZi9R7iv#}5962p0Sz-!3AAnLtfdT>W*4a}uA zkEpoQZAYePrzYGR|IJ5JW?<*scXlLt58981xC##sdXlFdM|)oICCBf)=GJdho^{n0 zYb)n3{Fiz3{8Q<-8IJD03fnh6cNbNScivuD&OL}o_;v7;onL>RgI2gekA9e|%fWuP zQZg%*WLMO_^FG%w5ozpg@+76!L=B(G2(UcU31mo>&QkAuC~ew?$8jB_WXl`7^M)S{ zS}mM}<f~IUc9aB7JQEvy6w`fq8T{vKvoa^<l4hDETi@nvq@g2hB>dYzSzzeT1iZvz z(q7u0SMKldBT0;mB+=b>gz3oC1oeN#H`5S|1{FRI!ao5l@V9jDFMY1qv@o|+cE52@ z2|p6FqeRG@{_lqr6&;@_vHnQUF!W_7X#QuLpih&o?PJf<)DwMY6kCt~?D~huV_Y^< z!B1!-Dj(Q&y_;00EYQFI!*h$|s*l=l@I!i?GGbxp&uw2rX~XvQrTCpeqjkV(T6~rV zhRr?2<QGMxME|_N-=7-j-d{|wg11;~;1rMEEyy2;=(_#=vM=S%ig<hYd0U%KOJv^A z;R{o@2YJYd{<SP{&neX~70CrBr`L!EuQj_H;@2FUGGOYk>2;G(l|kV=zUu1SDO0Kb z@!O1sn~>+?hZ|>1nN)P|1;=GKM(ND0bAO#siBL>ukeNLrFTC^M*K}Fmlb0H1-<uO2 z7Ja=Ejh_40h~O$Izj>6rSj<h|k~+J)Y^IRCy7cahK7L^tpH{*g%bhde$q4dIJZ3RM zfnWH$f1rHey+8_#%5ndB6$ShsX^O$YmFC=i6WmVV^>LLHgTup`K4PmH&0IXzMzN*; z8Cmih5kQ6F;B0Icl_xE=cHljM|9Es4GAvW_Fa9!oA@FS^9~>HKkN8+_vXu5;hYXxC zc~Y98G9RVPIsA*>%BM^_UTQtWhA&P<o<caC9nGdXmvgy=pT>Op?{hpM8Lmmdx*hWr zKtimYnKQOQYG^{|4&d3IXki?()~8=<bA(H?x!a1F|3}YW>56-5W7-2-Mw{Aw=Xi%@ zoaKwu(GGDo5I*J(4)}JEHX1fr?Vk9A-&Tg4a_D4}oP&^%GUpYe-L;0WC#7%t((Fva zfy4EOk4Ilw9X&8t?cV?I)9Wn*`?;D|wyWnv@e>A%U2;yiB1moLt)3U5E#$jhw=%l# z$tAt~KNtJ_o#^$hys$;0NC#piytG#sA%-%M$n)Yyj+J;=WKfuJ2;fKTc>$5zG3iap zh>e}!f)b+5e9r%t5ZT#(X1BEQT;D?&WWu_xq)g&VI-Ow@`wRaCu)}|n)IV(fv~#th zKauh^-7fcdua2oWHTD1RaSkOdW*hi&4B?=1M@y5#5<2B~lki<hDFT+Nt_fK}O=k7~ zd2lqyF^yCcVCu%%6?gw%r>y%BQKxKxJ$_Y1yhQ_@aFI#V`01Gi33HONL-_aJeF^e~ zQJq^oj?Fp>3U^=1Nf1_}5cw?EhBX>E?SJPWlKsswIK6TWKQ4|oEv~WFhKjTyE$Rw4 z=Btq=V3|mUJ^VAM$(+B!@Zw*+H;9lGnb$$+&<QQmf}O+GSebpg5al$IEm;TVT-lf{ z6S(*D{a}vVex#jN;5{uZtr*D+$HET@2?>n;567G11*`{e6oQ6t5^}p-L}i-AbzC1z z5uKo5SQetaY5(;J1yaA&l9wOW;<1$<>r`^#vZxn=f9h!r3s_n~5(@}~v$EFPpZ{|{ z6QL}4>I>a)uw<t0oRuOB$*pvVJs5Om3A*u4WmX7Hn<~^hx`Tp_Np)RZ)sWtVVpKcJ z*6iLwkjGg}wx=%v!rx)Ay@pvwiZouCV(<IrK<=Bf`e%gAM|~i*|0iAZFT?|@6RMaW zM6WT?T7m#;0;ty3Ix%~p5X>!)MUTqeZv1r94pI7gkWzJW=~xFeAzCXb6S;7)9{wsw zlbY-xh3-O>-2e!r0v3Mf$3QbNnu)LT<ObmOcA+%Terq^V7GsB{t<V0rrrjAaF#sYj z?an4$>3|18^f=*HaZNiRF&s{RXH{4#!rU}_y-U-L=its(g9C60*|<(W4ifr3^BsKt zeInUeKUAITrC?`oP<Y&lO&+(^SD@qXf_8JGj9&aNH$df`$(7R{DVdQ%hE&E{!B+1+ z;<+w-x@PIdAQW%;ymG8eZ8tJv)}d8NA^X=$JWmGHq`mixZ`9Ysr8UZM2(>MZ`b7jW z+j{MT-hgNN0*u=B5skY?(on@;Vt!H`JUs`gv^kVZ$L!MoW%nU`Bf|RjPq^zU9Eq5& zDd4yJHI;~on(xPcm`36+U|SdD5IdG(Cs<uGA#CITP2(*3zrC;=ToIl~^8LAj)SsL; zz`42u>guLoP>@Z3_;443+rT0!#7?wwImnr@#2xy2{jlC$m-VG=#2oN?lnGRRT%gNi z2fX$aB1*q60EtRr|4WW{H{-uxs4K2GE~G)#t$YdkvE8cq@hwme@_S1MVz2jY1JnZT zrBL;^++16<$)xKBm5*KJ3C}z!C;thsKCp>u-CRW~P}IjS*CP-rQX=;?sggZ4e*fqD z+ejBeCTJakRJQYN{KcPAap#r3eECw>VX$3IpU><oU>N%ajf%Z{z|Z_c>MQmkaj(dR zbn9(^((Zu#ISU-wKOHv;2n@WB^coB4kqG5bD~4uT)-MLW9&0APR)IJ*iWsb07R^bC zATkcE1>H9HdoSA`erM9#OXT)CCEV5sBSzO8u?@QOf*}gn3sy~vs%_o@k@c=MtPyKn z;7X+@TBCPZ`>celW}G+Do(~Z>Z}x3(hKsbd;e_+{fjXZ?Y~_{?pj|y;J9GoW;gmyv zq)7Wkyss!B(x5d~1d)ibqFz7Yn)&438UBaLhLfjc!g*r_y3*-xwLA|y7xz(*y3ce$ z<IX0aEMttXE-{txb+xP{Ntwz=%><$}K}h)-e5+%~tXcSaKIc4@?ns<-0Nlq;^)rVe zwEvm)EF`m+e^pX-J~6LCU;cpGw*%6eVAH^*n^#{RjqOK7>#z4B%0RP9OjN#99*4fX zQK~n&7|WfC*AM5$kMp-8D#4VGda^8W4URxGcr=E1Z7)Ho_k2fvaxHbxe57a}ETJ4Q zLoFymNCrQ6KZ&hCMQ%daJnr!mk8pa{#mn2e)Jv3L2f7@>yM_2^>q(PfNtEg*_n9sr zg`{p9Y^#!_TdMx&B(UJGRbKoU%@lmTP(gz!HSH@U!UGY!^kvU}(O>;?%i)Y$Q1Eyt z0>)GFp(<P8FK~9h*>r)@_r0Vw?yPRnBaemUDrIY}g~sEex+f32ivN!|7ZgiVB=eNk z5L;8bWOZ>H8t&RZT}8BhUw@szU)@>Z$X#9GrhdD}X{75{%gZjFj{>i?sK-pdwHU9- z-OlZ?%S2k<FvT{E0>+sGWmhvx61Nx|?F>Dyba=pYk*qV+E%6WG^mD^TEdn5W4O$8e z$az&W_M`?;0VRN`aPXb%yQpz!NZ_OzRnnS)d7Sli#&ualaL?7t`dI5caOm$DedMpz z)SGx)->Zu8OsW<wU8)D;JO1-qNT;Vu6dP-8C3aEu{7&wQ18{~J^YYANf?p<nx)x=` z%X4N9QLB+nC+BolVO8p%uYxrN?+2G$I(Np|p#<dx(;ID-%flZ1YW6Vy-9Pt^FMOPZ zvQ*Dfw~eCeUJ|d2wzv%_H@CZjfi&V5-E;9V8FhxOQJXmhrQ6n@@G0;ywf1SDkZki7 z#(bh;EKf}XUkIUBP(tS)c)d4%gu@DfbVIK6Lv6q+OGG#IrOqYdILj;&(%pT4?2t~4 zTl35G8tMmfZy&?%zh`FFmG@%zV4Owi-DeN0yGWSC2wBQOvOY_?z+vs%58GB(8bFUL zIOVA7RhO`jKf&x;VN3;PAVbRB?B${ng=t(blt%UF7<+nknR%oALqZVZQ(#vz4N<Tb zTH;iPkI?Xu<x6lm=gMEAT#1m{YL0kjFF%y}OM1ynil{s>;_bXxEgYpi7EOGML04@Y zlus2iwzW6_^+^8)_x(4FebCUVyOA_l#z2-KZcdwtd_#_AYXzwzVKr7DW-x~v+=B0O ziTK$bD}M)|FWH=$QO_N3k9g)XWFQ_aWk5YvT-;dx)+8kSx5Roj=i_-jb#?!x2Q}>x z3S`hTK>*vWq|{l>_;oSiC>LGxTO1);{Tr%T@F;1%8`%<w*({l|*p1g2s4g5THKbxD z)I>xmjjPJkT^o{_RE^i;`1OjvuFMVidiBP~8#8Vot~K6bYsPRltm{T4Nj*2yAmYD~ zD_~6K0f%Wt^5ImIl@{`pw@b+slE<V96Hk4Ci(QwF(WN(5F<h#_q{?3Ws-9|>ApJ)0 zlCYF4IlJ*u_Zoq&pI4T)`T1W-*f=yaFV24k(oy%mJ@6GNda(J$#s_DRrk__@t;rF2 zbG)j%2eW5ePTP%@SrwA$2!^fLqbz)T-20WJiMSe&7E=YabmJ>06tQ`l0ki`0UL#O7 z=HHB)GZ<2-=bs0<h}L-JR<n%*A>IBvIHA(FekHu{ZV6?R9wfmB3K`0X&z5E)-sumg zzz)et*VlZk<y2W?nBKst=Zx9#l$J;<T|Lmj{*cn=N40^#*9ZkM3Uy{|m}7$KHeCF^ zYsH7$y_dfT(wSvFdiZeI=m3rAELzZ_IHt~RI)7vmj9YBRpe9%l=X~dL*ou^WxYW26 zKQo81#d<~t@3N+YtYD}3@l=sC(ru)JFjI8JvhBlXQSYxf*;xw_iHD$Libh4Ta(QzV z$9jD3@zn3z6KvSG7n@@b4kvn#R}j09n2k0uub*HUG}1G-)&BMC)kh!Piup>wpicEs z%J@~;GsYw(0g{)&(N~Y}ukN|wX?5@9mveiK9riqh{<--^<O`aim-%em@12^xMEPWb zXR!dvZ;wVMbpMqbakAj%9k2|EiMe5$l$tcRR@I2(@t_t3E@h6|`E-KE)t!HM(}PL} zr@NmwTUtLqD#rRN5NTgRn(F2vh9-Pfx;`!T+i(_yNVS<gDGne~Dw_bKq8Y|^grkOe zi?n=0RORx0DP+3*CnrXL_7UBMupLCefKpf5!<K(Wgv!`<#bG4$KGI&~*nw??83HaP z@xS@GiehFldvUOV=a4>{e2)52<t@s$kp?i--;uhU2?Y^W;Rsk}hghJETIg1jDS%l5 zgvJUXBrM74H0Ahv$UWZ>-yBoqmXW`c8gXv5C0&AgUfZoWzDN}4P_=4IR55Zv{k4ir zT&cFrw5|{(6-qfgQko5q`GMK9300f>H8rNc>P*IS8v%uPn~>T{eEiI#Dyf!CTfh+P z2&(KI3o<5(0?1l?16Y-qkmKn`liIR2N}_E9dTqB%V$)&wv?JPf;7;bmOO`U|n7@7f z>VLdXpu&3*Wplq-Q~i237B%{w?j}Jn7|EVyZWAD-{2C9vxuX*i@Y4KOLV2s%)0CFI zbCBWKP|s>h!)*dnl5bYp>bOV=?mCeZtPk36dv-whYX-(Ik4{xUp_V!q0nBZv%fYJN zt`5$Pn8mht#M<{*`BysjII|H=wfQZX+G~^*3m-$8NXAH+)QCL*EiPLi#q0>pkqf2# zlN?s#X%UMPJV2%j^t*QuC@J#UsRx!2cPSCOhRExdB|c?WC^iF43dIKzE+3>-P0bWn z$Hb8SJ-!m03^tx|WgeV-{-0m-)wtO>#3v=$Qg<U_c_@z6`bmw?qZoryBWOr}M|hJ{ z%G8ZqF=rIZiT2^|G++|q6Tw}TkibpYM?713FgvztF;+Fb9K9auSX>eSX@`%Lr<v3f zg!|h3<614mcES1RPz?0dIsRy}IbPLQ!0RMU7-yoosq??QFK|!LeG*Tnt?pzx-q_7G z><(21i8@$+-sa`sczP%1d(};Q1JhbW*Jy6@Fj^$+FQZ7BLT>A0<N4}4ITg>MCGOl% zkUk?TOnyy&x2E)0dtRHVVJOO+i<mkn4?&AWuko9zjb`Va<?B+Ebaalgc~lX+0L)BF zr-4{Gf7xrem2p}OeCtcSSPcRyS|MfQxmcw)wiZp9Wk5Z}xERXq7Z?~akMta{E!4@D zzjT&;e~?t4*;&%nPca>8`giGb%+)J8f1TDcdAa#OUrk@{m^|0iFW*`<yONS**Vsbt zuMf?%P+Tl;gcGl>ydT=Uyedp9_qHlHzPeE0hPVz1Xv*J9F{kh)eJfb3tPU_#rXdL^ zM0fc8>J9$!IlG&~V;^fh1<AX<Qx>$&K6JJoY^-m%Py-d*l*J+!bX+#S%|Gc-;jTj9 z$Gai!0XTJ$&)YY*y&mNF2hZbCsZ#61af!*f@!}Zmo6DjTB`PB(-B#ybGftG5HF(xF z?d9krMtAXF)fN9uJ53ZZ_50c2>_gc<m-Qr_@*ZC;*!;c1@8zPiSN>0GxSWL7&=}*- zkSbXnO{1#1Msv)S*x~5-jF^hh_{sS1ToZqW$3KH<M0GI9$p7LyXb#dCpVW-t@x0`M z)_#Tl<kv5ob{D{F0N>uceYV5HgB==xV!W=$l;os|Df9ER7?aV(Lo0;qTTdFzM6{5b zE~Rm3_cN9b-a7jq7*OfGS((5%vz&CZ@g+sVFB0&Z=da!G5h?8@+7j^oSKZ+fG3Q~s zgFEbZR!7v@yC{Jpb5N}E(f@t!;(wt;ocns=<(01Yga;$f0t*kKPSwCXJc7b{4gn)6 z`qVT#{DKDsn0G-){2S4p!I=ysLO{*Amo{2EHfiUdZHo|?w@Bz5{-(Hz>+U{2(IMFP zQfBn=ksnm<|8=L;T-N{6S6&`W!TO%NPLsz+mQdvAl!hyNYCs16!K`miz6lJ!v&WMz z)_myPf8VCzz^F*v1>vXB1e7tY8#Kyk-J>^m{5^2S6i~zxaB`Ht(0`)Maf!n?XLhD| zJ7#h1<LW@L*YiD)AXHN{w={P#KF%GtUOij(Kl8@NcERdiGVqdz77gO#@+CV6PPpV* zh@)<D%mBf=MUxSME71mawlkr9U?0Sv&2KlIf0#@D;BhD`X`Q@a-X3gMZUE}(Gc7kS z{xh#vLHFFMjlJHjp_yK3tx3&#x=a!e>KMx+h-mcLhlHq3>MxXOqM1H4GICG!Y|)*< z<VRV1XgW?6z4%VfN_zk|UX>@mozHzTI{N)TFQI(!NS;i3x#2@$qHFPu=kV>Kt0mz1 ztBFHT@67r0NO!5rU{N&h!%Ry0R70~;`-O6m2Kyz{;+Me4d$y=wGP^h(ZKW74(sKMr zUq<a*2gY&NLIGg$Sg-#rKOx1^@GXqxrL$_+bv^zAoRVKtzcs(#|Np2z0WJRtuL*w5 zxtqk7br=0|YZV<d{pSh({`5Q!tv~xHeM_L#+yP%@gsJ>YqAkW|<uv3@fDu^_qE?^g z=X4fCzh&omdX0b|;b(v~wDecYt0s$khzR}fhNzE0kkFV9*9f)4Ad>ONXNnKkq-G{u zPnYvdV<K!u{PW<F7~(j6NZpS8dGt^N!~1jxrj9s-;Gu`qP+B{`#!898VjqUA+4$Fl zY8`vpud)l>5uPZ3_C0^J@Jd%+<M>2idpQ{a+KZ?m(WHr}VShou_JR-A7kF-x9oCjO z;6fW>ZTKIrX!H{0!Z_wuy8dE-ukUB0Pj~SzmbL-D4QU$_?#Fjtl$PReeG+cA|9fE2 zsq~bM{~nlq<z}i1Ckf?oIaGS5qu|i*?xS!J{&+t1&yxH5QwJB6NjmgO{B|1C;%ZMK zzRv&R@9bv~)#VdskaLI=+JAKQ%}|T{+&??x?@x{7Y60Jq3QmUI+aE&sU$~q@0P}Te z9Xy1eGrkjV4BnVYwoTp3&*yY*CplC|_*Fs<pn0pj<rUZHDHhu|b^TXW`0!(^ISq9! z&1rQs+0Gw4f978N3y#;~(dSYwsxpKb*qI^R8PbJE>B7Xj79hAKx=M?>$`{873t|?P z&Hr$Hwfefi9mDLtZ&v^Qdt#E4XD+OMms!re$V7ZneF|@DjH5XWcK{6=o5aqqN1lsl zpy_cO)Vh7qg*M88lu!~Nyb`Y8GCv^j@XvR9Nq3j5H@t?iv~DB;St>_4tO_C2-tuGi zgjzXacF^BQ$dXi!ryo?+C5!?yYG1hy^Z2J7FAFpYX@9z7h+t+bLygb%v;{hX!873r zh%-t5)?GAw$abH=6?p=29Yewsyg_r0l3Xy8$zK0@Dl5p7@Z|&?`<m3P-Xv`p2^XT1 zszY>ie@75EbP`rf_0!O6_%kWYJzx(gwc6*YG*>S;X8(WwDW~hL9(*i+Ofhf0()6Bc z_x3;%zNzQ31P7-yu$&hB?zr$f>bv0qLWaEiAo3AeY;K38lWWi6LE~ykC_@Bz3Y*}s zF$r8jr(?*h{L`9XU8XOMs(10D2Kq>leRM!;!`nzJI4@vKLBFsjTTia^Dd1k`Jf;z@ z{A-NI!|*D9qz<~79Q?jUji@Gi06Npq(U}ko>Tk#a7f-Q#uBOL`y(Wl<rsWDV9oG>D zEYP^<MZQKn9D9J%3Gixe2V%3z;yjCs=XQR7@y@e9p?oIq`m*Fo5Nr%dWI|p`Vm}oC zdhXl@N*P%v?TUdY=g)Vq-G)l>Ww^2v;L3imdznw2LrRd%X9yxTO-@=)drejPOIR{i z{4ar-N&j444-h!*fY{#!Un=MI4I;`ML~V8b0_d%l!0t^)UfxTj7_>xJ!o}TOEB>@l z{Hhh10g)0r4^J+1#QDx|x&J8d=>Z4V?@uTWB_$>O6rbWN35ODl6GAlP7e~ZJh*Q?t zkq2Omdw3Fvy;%rYkWw&yUSzRZbqZ-zG}aVbScYU`KNp>}B=z({^=xYRV~SSZq)aB9 zYH4YFhQRTN!R~lo8Xp+oHdcrVRf%m|4bX6gE;?!jXFTMD>Yxu=PriH8;=}yAPO%pU z@OL5kH4XQC&yzZfbjNoVqBngd&(Cyz!Rdzd7w;JHi`-TQ2VB1zW+(vx7O^=A1D;z{ zB&u4r1<tx6oF(HAF4zpqwarC<sJyJvV&VzXkqF_r>PUyYhU}3KRT3HbwZfapY`d=4 zrSztH=r)U64odcb1o|2=@t0@MD-JsCzbG^LvM26zNxPv}&o|A6BkO$!sm|MZaaHt2 zRrFq0k{HXgZP(vgUjO-_dNQMM5>+@|>r9hsbk9;`q^R!mqbi5#eDKlp8X5Zmc5kCN zBBW^*q+X5Eq)H=2D3pd#RZY#}t;e;Yfv#b?{@At~WHPWci<m(4;l;hSLHcJheq=h= z!2I*WsJ7d1B2}8>u+Ql33J_)E=_5FX1JJ4X`uiUQe8tvZ(nLAhXymSzs}tENLm&Kg z=S!MWI4uk^x{y#PEHD$XzJgA0>)~N$#&*~$Fjk(3`8b*j33wMEsYPX4e=q__W6R## zE=ZezHzy>WZ5S!_nL0qVqFuC#(A&fea+#IZfGOotTm{Z$#}G?cE3o_1DB2M0bH0Og z2Jn$d1KCk$<=ayOG>+6k;o;#ki>($^j*H_)h?EX-UH)6(0Fc$?+7qZlm9<TDW+z%b z0&cYHF%Lzt)o;Ss@w>3qLcJLIt8$|g|L|`^r#I_ZdBL7$*X6>#RYz+7>$%ErW{^e( zF(s=tR-j%Hcs22MZ4hA7St5524^)%?cJ^bUuNmiOpv|TwDEy?OEGHs3mUzGnZ9j=} zs6XA^S)jpohB-}-rM565hldpgN6tun>)Z1KVkI97CIa>YfD-OAUk?pZMF4~N{Mk7Q zyAgYfcFHVa;E<x|ZJ@J0f%ZSE@2IB^?>+v&UC(KNzPP*HX*z2}-@d|1_M?D`3KL=~ znR}r=3uz;oW$*@+UYVp-FB6jz5;{8{{rq80Zb}GdYtEPQ%+^#@R}kWVay36phaW|` zz<nvh_TCGj()cs4k`o7>XRGup2v>DzXsAVJ_2EE82Q)J@M+~8JdmXD&YxU`u129|T z-I)7!<Me%C3V8vW6&nL^%=feM!E}mHdrUPibtlfL6`pu6JdaqHroLiW0e}sW*j=)? z4`Ry&cND3RD4_kt#lm^m635MRamXQ?)8F5pXNeT0km6Gc@YPm&7Z{I9IK=&NdBv5J z27QxP{FlZ>G1{Q$+DM0<r#V9bI-L;`k2kQAM#}C_Eg#QTaiq3$7fu>dGAJoYy4?JN z8ZXsg{RV2AiBkg2xk;VUr^xtXqMU%porb7snL(3kZ&Q%`1^oROvEJ&dS;!f{C$|k~ zpBYFovjEBO`kG7%exn*{v0HGmj|>H@LuWRlf=x-6v7q~5b42b4%bxj31oo<Suj@to z`nBTGQrv{Yu{b74m0P&+JV1Z!uhapj`@$<2=*nBTAn|~eM|UjuzWE|TtiH4FB1lP& zP(+Zv`9Zkler9B5nrBZtw5GTpxwf=-XM%XO&`j$x@tFFyrTQsi+cS@jths=@_B_ox z;x#Vv%Z#VbuMMFV8(a}LZ%?zQ^QcHk<;X9)u#Die;w&O|Nr>;H9n^eu>cXzLj5_-p zMD1g*ME_ZaBk%Gn5KysApWeS5j#wTR`RY+|0RXXQ58p}LKHKOL6b0}zC%WqCXBjPi z0Yb~KCz0w*FB*|JD`^_XnV}|K;6V#|jms=GthNQ7i)B&MW>CkP>4F_LK81xP@6@4U zB_5I3Lx21Nw8v(7i=KGKv07#~?xdzgw-DiNO}(5t#r{KsY2?I;gaqll+Gojx)al?< z<+JbrmVeSfBe-^hsPmp_ewiH|-5fMJTO;4|n{o?r;9T^!r)_PzV`rB$C1l9rv@*AD zUTBe<du3VXwJ~kL+c`uc$dg@S0BxW3+nUk1G)^CK);B_^P>t&BC8mU^1Jb;)waq(m z3NeYcKC~GM5M;ye3tkmgHo<DgC5yu4u9`}zsK%Xehwm}klZMGT5fB|}OTExO#G$<} zN>B0|M|_!sk#EZM_7IPoNow(0f`L{_?J)h#0$zjA8qhf9gOT8B)H7eqcgGvPY>G=9 z1Bt$YPL|-a{g4Q)Ddlr})a?q!vpA)(O+b8o@{;`g?LV707YxXc*GGBKn8ptIUytG$ zt+FR&8x<gdUa;tEdSg&cpZ@`K__;4ZVzEX2PkiWD!MJ`dpHd?`1!5B{S1owfM1&Gm zKXsJvyvK&E(=%u%*_^tu4UUfk7J~PjZXcbxLrJ96NRE;Ln|2E?>-BG=;$D0aF7gtj z1$vKBN7TNnDhQqFmD0?K_nD`>nd_s}Sa}dG33}!Om2x;sv*G*q%H#12D)Ov2Xi&vf zC4X;cr0%Am)s3?06?ii3m@+J~`Nvt-1Z|$i;kDX9C+Z6yS*6B3OG@QIikPWN7`9yD zmX^1PAD1xKh2fak44XmPydv<u>rk<<TS%~NcFD7N%m|j;SV=l{T0A`1_Nh}|iL+_! zgK^&Yx~^QS86Nt}7u>d-XO3WEU8X+KEElNUsj^0j;ZP5@giW5z1kp*%?Q#Ap2~(8| zU!ag?SkU7BYzWh6(-SO#nfCNXtd;vbXEFIkjeegJ2k6~rAtdPf8Fa~o?}@=i|6%6$ zv^gaZ*YN{`TVEqr1O|_u@daAqU_ta35HA{(=w0;{)U2bt%@TKC!1n-}^)bnC?fJ}? z8Kx8YG!pYin<v%q#4ZMC${A7q<HrtJ3ip#P?zIOI16_blY)U@#mK^3rC+{5H4~Vf0 zGjNP9eWl`Ap&u)l7dee3F&llP_aZql(<<*bSqZP`&;34(&-9QEty@QgHCr4Ci)=?f zdOChzLojx@*9!LMZzGob!giC!1=a?V17_>vBvzwh^W)~*$n~4Iik!1?L{R1mH6ZP4 zuAN=0N?Pw#{BXGeL<%!*Sgu&sJ=iMG=JpI@5)!X_5v)Yip#~!4Xr}FXH7Dad=V$<^ z-9U<dSl(ME<6Y0ovw9Axx99zByGVv&by(tTT;T@bYl$sZE1X{B!t~jC#mvI@e$Rb) zLr-D-7aZOh-J2&Tooq&G<n=;FU<IK+qh5_*G)<KK7c~o+a!KMa;M2Pk!lq@QURc#| z6w<fu(q98ky$K1lnApdi#@}kX=5JTs!Iz1@_#(V0Hh%ASn>{#@umKm_>g;mCdA;T5 zZJ&quPZu}B*q8zl;<Fd1dJGV2(5<-Ihi7_DxkC3rSJE}mzf3`iHVuhGM_QH*Mk6qL zGd&I<l$j8)8y3gJE`tc&1OonI111BJIP|hVA}x_G8=Nq8tl5v*GW&)i<^BNGo#NNe zvM_IOQk0qvd|-oMpk6k`VcZYqusNBv!qp()2(kAC4M_5SlT-yKz35c$HQ4e-MbIB_ z!)^~11815-sBn1~m0~kR>hIMSF>MXE{i3t(K4h#T>i!71rO!{eo1fR+gOWyk<3acm zn2r~GF&|HGdd$%r9PjNK4OPIiDN*C`?#~5#?aQphKe~MSxEYD5>ut^auHC?nQVdzD zgBU7RgS!a9Q%nJG=;|#wcHgMwLoEkBR6yTSr@U~mG@<#f4t^?>Qr%sD8TY+iDRacV z$7{97DE5IAStM|KKCq-6(XHeSBi0i8(~$>1a7=i+McEm8JFLwvAf^%1J%wE+v7y75 zxe|yX>;~UTuTAZ-bz4Q4Ql14qg2B!QrS_9!EY0SR-v97B^fM(o4;p)1sRPAud7IMK zIEkX3erPjNiKuI<2Pw^ZpM2SG4#Z(WjQ0zKcy|j_^lxH1&2G5$M2QW~Tw+||AvtG! zfK)Q>@!C6C?sSAT)7EG~)cla;9g38WE~0=v{!j784CD3a=;*T?XQ;osfrZ1$pV@E} z2c6sr8)~2MVDOFqeQK}V`wL$t{Iq+UhZxs&y&3&~L4*1i7-eSDz1Vp{KNxybr|1-& zE`wYhd)SMjaVN5FYwE9u95(>r^D!aoLF?sLWi%^~dlpN@<-Tww_@D&*#yMmwJ&|}p zj=1Nf&u*5_my2$NQbuB#ORGxBhvsEnQkhUpp*wX=;^)cv9QC$&IRw7Zmbo?l&5%Q1 z*JO_;_O)yGNzuaS?jLL}q*c`klb)$BlkV7x0KfOpazS!pPIS5Y(!k2eiAJtE$Hf8W z7ucg0sd2W4pV#f_iTZ;!sNFNx+uNLKL_!%G*ikgf{LO_7o90m~UuflI6R3pIGHZ9) z6i{9lB$daY-M9RjKlbL^j4&>TwfWD)GqrSv3D02oOy0ZsN9h#?kAx$(5ovP;^6mlg zR_`5q6wpJuV0LfnXLG&!P^(VfgO6hxlwVHD?f0Sd9_`FF^AbIFUKYqLd-o*0R<Cq2 zVo?{&HZxr}({`!I{*#zEogJvv(Y82s_-Rw*9W%S2#qsd_^ewVQ)TVonCz<yqK?H0e z$-2d4;ye22(}y)T>0sWEl~#K%JkX3CEou`OmpXSz=T_rkS<aWCnZ%NqjV19Qt{YQu zTHpc>uyb-Q%&4vDM~h3LjE_FD-m#8h9UxuklCpGN_<btSK#14%J`o;0q9ev$ZEHKA z=k(qYYu%<0PfQey>B})q#q^nBv)cR}mnO%2RUaQ5*H&OEEnsw3j~&83H?w`x8UHb3 zlyb{IJ0Z01WpPTPC00JsY8~71x!4F}>*%t6y<OkZG6g$UG~RW^JRH`;4Hn{_^QpWg z!9?y0CMu?QWkez!+#u=#R8R2??oATwKbgtf&#rmJ8dR&CB5YdxvsO?p=;uz~%B)DK zv&QdUck~^SJYwZLewt)|b7W!?4ef}6D)WXSYYu&t{KhWBJ%ryRU@KYIac8JFHVPl= zaEk15Yjcoh=3OQnGCQt3=nw`-mx1&wYq2#sFQB)dl8}@TKCSd6<<B7H;XW^{5BA_L z-lU7_risYE+rf?hmuA|^A3rkqTdQ(6mcFsJy8qq<$9Ae5=n8V%skit2hgt&sFY?Dk z;y-=XhMbmU)+WvuFY!6b{)UN^IH{LIe2<xeKmJzO85DTy8|1cQ0sjNLO`>#PwoQJ! zODLoqjC-D9`+mtH9UxvZf^=aQZ#jDIf|JY(H^E^$qN5!fYCDZca14iTopxu4I*XqI zUY!WxwWm?W%F!m*h<0C>@C8BaN-Anb(G4uSoNQwSXbT#w8fbwpTcJ6T%WcmyJ3dS2 zBfhf`e+8VHoj2P~-!JZBBP4qfuu8^LN*XFgL(%H3#do}i1`obzJ8}tBZWmET<qhfT z73nQUI07VCdr0m2eiI#Qe>7^Hn7<T36QfL&VK1VeLQjQ%q0*OBv?OW$c1Z*WTzT61 z{m0ino(f?R4vwgR5q=Z+{fcS_6|E*c*yyTjs4E-g=Xb=Eio$CpF%mOj)Enl2o^yyv z$GKjQKf#<7x4=-pY`FQBfHUHTkwq(n?I1K({e~*IYC0cyy*1t}Wl=xZMKqKR<`L}z zD9D1D$U>#dcu0*c7yx^qe=qQc2qykqyM|$5u&p)u8q5jHjhcc^;gxrC$8jC%qZjw% z<PN+SPH%sC7{i@g&{WK^)J43nGwV<kF(>8~DgF^|U56&6n#w}01jzd=#a(GhB0Ae2 z>2{>LF72l;GvEPF{GOn_aSTV(9n5CY`R?V;IMoNkM;O;6OX3^T-8e<u<Ca%fN@JpS zDuK{Nm9W;)O(c#<Jt-y}X?%p0o`!-&mWwh@BK)rW;kbmPocv4G=5re_V^V)I68@O5 z?LOg3zY5WB^<1V-xJr+IkqCO2tooP7JR^5%NpBQT{>rqaAC4O37&A){jWwq<6Bq@A zznDY)p{j<>i^UGK-NsX?p}rE2Y`l0wPQo_jI33YN45!8Cp+TeM`n9CpiwM6=&^m`T z?<CvByq(WjG3{;*c{jhaQtK0tacQZM{JryqH&}3!lbjRH{yxlOO_9w6bcygZ@hH+z zo8SV<YWzAULdi4yvLli|dxsQ%c4;2Ky_`1<*-(F56Q)vdGBFd+-v4KKEc9<xd^)&o z**(m0l`z8*>d3$(gm`|%4~!J5VUvb~Bf+J;j4y#+edkc$Jps@W4X8E!q>}Mb5#E3H zAUAQvpar*94naT0j~uS%zs@q5$qoGH93nymolaYFWa5o4Xx<6#iAh|~5=)_0f`0wj z>2rvaI~13~i`>mdXBrOue`ml^b|r1yp+2umY2#`HWL{ed@==CwXYs)ouU154WL1m; zXO7UND&8Pm!S0~#494BG(j*qyjlzq$dn$j0x+mag1cglPe;|e=Gapg2wiA14{(VCb zlkloU{26y>{dmV8fT+8R@q`7N^#;l8NaO|ZUsZrsund5(TrcL5lH0#O1h<bt^YeXu zvbHe%6S$*gDISQp2fRCr@1V-78<COGd2dU}@Q>U1x(5^qgjI*1)Hm_KiV0YKG$5BO zzDOuN1RwyV(T5*93mW(Oa2&`38ETT#`pf-~S`ibr31NNSqvX_A`rYT1n0P63tC`Rq zN?Xdki#Y4`rTm>=dozd1cj#z+w27)dOLB2wne;z$BRTrfo|4bdlLiJOf#{z{p@l~| z<M)H0iP+fdZO8PS=oSWFIwYJLmAbehJ7=ES?$+4homb2HKooviyEgd4&IYJ7#?5D( z4_TQr47nJ8I*KWdAU7er#W3Z@iu~KZmrGFLekHu=z4lf2w?x`kgGBM$sFbhqcA<xP zbu}B=+n#?BG65aDL^|KaR(!}t#Qmqao9T|I5GDk+hf3q%<G;g(envGY6}s)>Nq097 z)j~jok%JVQ-~yl#(k-nyZeCUKiV@%z-j@D;Mu5-;EJAA)FsP%d{_$gO2&B1(kS6q# zaZ%nGMMXu76XWH;sQfFFXlzvBfSN&yUSTJK(E&%bm$#bt?}J7xQY3Yxx=e7;ox$^W zaao60$yuj@{U~>~ZoM#RLgG(`8V*6yJ#@AT8<jo6vNG}bHz&gRA6|i;V@3!keG2L< zy}wy<5XN$s*v72nL`P<9P1uz#w*A29<}TT=)`8l>6{24OI+p2@`>$lxvbqD=t`GpG zqzKjnx>{;#cLDb+L>TOL{V&*_ncM%ZD;r0nWEs>njjm^7rvr2bHb43^1R4^G_s8gO ziS`O~KA_tv7vgfxDk&=)Wx8@^lRh1?NFMb&Ojr%Z@6cd>a%H~qlz>Gi-+KQmo&c>! zWz>RM0F}2sCQ%R`JB?I(rl9<!cQ60hStMm#g63;0c4YgG+=9waF0lUvvnc=keRpkK z>4zYrt)BLc7>DEtG#z2u8&zO!G-hbU)9qX}$%?FOY}JlQp;itJH;7IXwkm5VX|qr& z#{xFHIAx>Fe+zqtddLmBdqIb>?%z>+nzMh-$CzBhA)OAv^vE21B6vxU0M}noOuH0e z`bAN_J}%S<<txFyW@sfgmHwSM6Y4$9ft%0^?M{eAPLLiz0XdM*t&i>j339>WL|dJ= z====>DGm*Fb%%(rAiyw}4d6^ic*)>OW&yTRnV^yeb1ePQRB-#!58*X>*+(X^n&7RC zkvIT&XK<iIu6X^8y;EJexsYqD-5@KVz6uU+=3vZ_sT?Jx<?33@Q;jscLKjfjPWW5V z4>c$`?7niuQPLIq;-e2c2xl56eWkYtQNdDWu6N>r$9ZZ`?-R}+Z`Al*%>5-W(HO*x z2m1Ln>&Ro@bKFAbqBP%rQdoeDymk-GvwcSm0a%O%?)Yti+h0k#K!acbowyLY#V*7U z1K>V85bXb@00O%4YA=;ssBlYW3l}9)En+Oc1$vKVWlB1dSuo{T+kj<Q3l50`p?{Dv z++GrP_m5dw&C;w0r(4uG0`W)uLUybj&?zf~1euyRk1%j!91!6LQVYt>G;TPFSVW2{ zUBwaW71*X9v&{Pfy;>saKi)#b!x8f2w&TGQeO{Rvo!1_oJB^P;vXqHkpISvxOQ-Jy z#>QqS`k@wqJ1XheQNKH8Ni8YK^x^Y6WmAY)y$-@MVVQ*TL#`~(=VnlPaIwS~Fb)2e zakluBmRA3GPlXT5Oz2c_4h+-y$A&Q|4CEmCn|pO(Qxl^yTOU3gZ?g5GiQt%7T!U_W zv$YP`*%@%MbbaN4v|2=<WJD3Ld4BWF80=^>B(X11BPzx1Wi}qD@XAE+gsnK<^DNP@ zGP@C@L!NCO*Ey<iJ1|>u`1M#Hq0gz!AeGlY&3vS6>`D@+_T0UHzY7#tBd3Vj(Gp*X zWq@1j>tpy}JU-U(iu1{BWH+0fz93Q{uWxcQIyqCH_z}sCD7)zM_+wiD!f|b#S4zw~ z`=?P2Y5OS?pr&QXyyrl)$stv|a{}#ik}znk`KTSro-TP#)PdVSFV!`Zqm;gXepZKJ zyl~4M7^!Z&Tz!6@6znb5L54Q3;EJuEIKKtux^AfEvAO+)h8xJ{=jV~5be?02-)|f6 zC*!hb)Ksq=zB}8ePZvx$vTY!(nsSfUFTi0c{L{OwiEHmwTK8vL`)@`O8-FAAznOcy z-U3BO65&jLD?*3Z85U)+{kZ=&crebJII3OPgTy&E1aa^BWjMGJGRET!_mRD^OqO-o zIa!Ubph$?b!g3-`?~K5wY<IAfrJ4BVbKIn-N)6Bb_{pH;nVV8N0|uCgYQp>DNWsZ1 zz|-b`&+;pmO(CvDy$4bf7cr;DOt+B29O&`xfQ^-p7aSgsQCZ1e0-cp0wfV&uQd3it zon#EjGPE>I?Qdg8!JU&73T5UTC?>RN3KWet$NP?HkR=4FfI(jwCrhmX57=Z3n8YhK z|499WR`Pm23-*2XLSWM(P~!3h9B!s^&$F|$&z3%E#xCl9Ie;7ltW2bQC^UA#r;GU0 zoe34To$1u}f#4=NmWv%P&o$vzvhZ)<BF@Oj<P=;xg+J<vP-AB!ytgAGPa@aI6jZYp z#-A$8gKbO$4Vb`$$9$X>v75}3Nl5*@1TEptlx(`AsQ854Q+nGfQH!KUkdk6fIfITp zZ7CeG%J*qY0aKR8tU`@uy7xuuS66^e=bn@PKEDN<p-_2IDW9+Z3s`QNB3;Lj7O*); z$4oeElfja7t1XT?9qH;)UqO1FBZ}5NVYKj}D+7+mqKNC982;$jg7C4mTNvMp8pj8T zEQ;u?oBG|*f?@3$>><pvXywH?y!4u!^_I_-rT6>G#VxxpH`TA7*j&mjS>*UFNsE48 ze4d&A?iR3`jX+B@*F`fbK;#IGXh*t0X=u?DtGG<FEX$q$P1ttySJU48%#%^1$R4!K zcDsSdXW%-O7Ql|HIl;l<GEcns_%F=ReipHAX5flX^nNbf%h(M!i#~>w;>laoS36U2 zSEN7FXq3;IF)7p;XFC|r-D-|gMmqU|7Yzb#?$@@5Z@+T+Hne2S7A!MEW|oU2#?4?d zch8ZmvBp1E+G?+52tichK?Ee>-}@8~<+zY$L?v$16V(^9inwsxtii_;tanI(hhPZ4 zqaTqjXZ$E45?)}eH_<PT6Sc7uE$V$NiL_^|-G-XWU(3wL)B&7YhR8u;vOs)BZ&4h| z4t2S3WJ3khF<cCkPKZKnBYz7j$3U)FYGaJq1`o}|K4J7B2!nVF5fHN!or(>4jF>2C z7`R|V_{{>&w%mxvs-@v_T!5S5^vm5+9e{t4l_`pp{9#5__pd<go|u>rAkq#)a<Wf} zXo=Y4)zu?T4-0a57I$K(h-~t^*Bg3TprkrGKid1?M9Z@7AJvhkLuGd)|9s_*_mS5V z!|H`0`9aOO3s}Q+dY+Wm)3<vsu6?u|TTK`GbGHjtgNC-Qk+?Tc26xYV5~jrN4P2{a zl;1{h(+v(0JR`V$-K25Wm&oa$y5o>pl~fl&qfzfrJ}uF=uiVhLh8@iAhM2a&1F@4= zVlUi;F}H}Y3!a}EC}9<wi2$R<Vy?g3HQVid;F_k2ieW*y1Z-<nHo@^#Xv(`GKu-B2 zE|h^=*O-xsRb0GeDM@3q;DNTSdChRUX^Ypj-?YXja4U~K$w&6RwO(P0f7<>dhLq&2 zOZvBj#2T`Fdi=S(rnr;7NS5~IE$Zphr@fx3#g8^G9z;t`lMyhQ07xOGo#JXeH<mhD zg~ore?QV2(zHoQpPEc5C+$SrL5$7jj#h$=^ZOaV{P?i3?rE!{{?s9#!Y#@Ed=?P?! zSt#6+xSg-=r>?b}O>Spl<0Xy5U!{_$>Sy}7AHS-q?VL2U#v*?+^lq)89D50)u5)~6 zcAM4nUPI%Er#c%>_@Ahe=Rbh{b)Qc2LTq=*v56|v-E^^D7xkK?UN^&>5LTL!4-Vez z(B9LjqrhBt@_!MsdP)*zw~aXsCCZoZjO3q6&^}^Qa;Ju(yMqKbIpnGFFjz!6lCeLu z=H1~?LcRbc=%fKt=Y`6jt+B26kO=t%<W&!j)}PzEYo`#=L7K;e&OGG%*Ip2Q<witg zQ71AD_qH`D|1YR)s?czK>rc5x%_b3V#wH=X<T?ZBq3liH{X1V0#7)~Xqg3)xZB9}; zW`?4fg%Cpj&95M<8g)uPn){==o5?*GpXBnQUjOLgtsTVUq7_tw!}mMfKQdkFB__fT zJw7?Z{iGkg6el^X5r+R;cl0YH8!~LZy908d?@_5?g6MhTvp$S2c}`L0?KQkNL*2}b ztDsO*Q`1ydj^@!-Qu^f+3^uyMbzRwJheK5lj`+g85FQX3Y5~#d@`$!VHHS&?ut|GM zf4^aP0I1cV!Q)mD91iC>4mn^)>%U}VWB_HON=8P;*%b^$#}sci@hQgVSNm&w`f~-2 zCkS)241v?A<(g_jwQMUdMq58$#bRRqUG`7^@$-Qj>ITsTRncP~X}FZRr!I9q(zu@Q zVKKhtKki?^^K3IwEOp4AWAVs2Ztj;$JsFi(x+sO&yknc~bUUo0nNMN;&5V3{V(g>2 zbujr^+az87T!%;KYR`3s`)zy(*m!<yDV6?;UD;gR6!(no;Q;xb8J+$Qjom{D6YcU> z_NGpzchRLb8%>-K>^zsXS{3lhMy5+F*4V&A?);15!)6U0waz@2uMbz_&)?FQNcK-} zbB7Qk8Aqb8U-y?^&uJg7m9f3R{}9U{O@!M=XFpxeX5q<XH(R2#3F7^;VM|aS^!$Nj zpDPPMk`oF;=yK)N5L@Mnxzv6Tn#<xy$vdgP1bUN~v7W3rXjI7=hBV}@sAdL&j+hJ3 z@?OM(62)dN1DYXQi@E^sZea|3*I5QkAcnqQGzt1Smo?DA^_xIb)|*9OaA|N{5ov0w zxD9b@zX@V^S2T&J>%)todR*QFowTFvI}YpHp9m8JwPCmzIHlWnLZ{FHqY^Nf4llm@ z<DGpcsPEhCJD`g<XPB5(uV}c~?XOSlJHhL`&b|YfJ`J%xM=qWWTY|5ss0KZE=G?;O zQdeCpGzCqc%YqUmyd<3dVtP2z+kJUp9oTOKw4XzY+t6*?c@6R}7R5rK!1Nn!FT{$k zF7()SafK~Gfg3&ycy)6Nps%fuMVeO5WqC3?vg<E_qI7m_e1l?07(CnijgWrP_G?H< zdky-!A5km>aoT_p<Zrg&#Sk}_H-V-#)V>4MIig`=P^t`f1xZnP6S!li+jk(wv2zP9 z`+IA=g-jGhw!eyhZ$d~r_8sx6vEU5X6bt_i;h;U@?Or#xQ2%2N=pf&ln1PID-*L+O z^4C~kJARqV0<{uM9`v2J5_JI^#JMqq6())_4EGKU6?Mo^3<nLJ=oAc+ci<K12i-pg ztcNv-3;Io<@QE&(y!ie2K-2!qzufH^N9Zoqo_p~t<-ugisL)W|$cN!|Ow0#Wk2av{ z9A#1bp!)N2?A;}yp~SiH@82aQ74Io>EzVSQpG!S)W*C3$msU~Uy>76O=(I5UyG!|+ z#f`-)Z5z!Fw^wJ|>x<Il#_y;GD(VIPElk@`%7tp0m%rk$_?70};Qv^*&Zj1btxF2B z>hl~47QIQl)iZA_^a{?T#3Uyc;|pyDF_@BX^s!j3@4!>OVSszU%$~xT<sj=i`x4Z| z-dmUNRP0l|pV<fuKjYtpKS6stgqeRd)!Fil`q=QFmQ@j>Dmpi$N~{Ap?*^qEHtu{f z%%i~FN?NxRg|vE;B3K8j<a?+Nb7#S<wvVJj8~D~-D+9L_8Yyh$glQ>aDi~V>_M2Bd zx5$o}-@4+ov@kY4srLOrAh(`5v>6qX4-FRIcoWxlFT~<rsK}$G#2$nd65f}H;n9J< zx0|aV1GX67P*c&bx|FR8D)3y_&9&7HIEt{@)MqSIyEIhv3%=?*B<_<L_X`6at2jnW z{ht37+x9B$VVOQKx_w$ZX|OdP=*qv{U;b(etSc5)Gu{Qg6P~6Y1Tvjb;%T!Wj(F;3 z1{u?1*ujM4=6f>nL!Guw@=z_*`1AHi^X*hWZSJXKozi|tAi<K4#_jv)>l%t}O$YCQ z6=zZhc6W(W*O2xCV#$z=7H`bUx)mdVGtwEot|cg?bdTOxK7`>beAr+&U?*)k`^pQX z11+~lnhF@Pg3j8@(>)gO!_vqhHU&Ox3J3DlVvQatk?L#(jo_(Z0R*vyk&n7-DM<<f z(SSsBj)b!bGpJNALO8ry$-u~e8@f<07{$tm+&eL9G+O_$HCab6d~2aZKd1Z;hj~5O zXY*Idf{P}|EfKGu5bkj+OYcmJ!Ronl7p_V0$(#?YViWhr+#JGO@l)ezQN*w(7Rw}* z9QPYM$#TYFtO~RGL;2idJM~bbdMiTks+;L4%tgGv!xKL@TgsS9XODnW_Y{bo7fnp9 zvJ2lW?U~xoctT>`Q{`;9YEF-x4hz#0o2sf%)q*UYuHhQ)%=|Nll0KO$SA0;t${JRC zQMipgc~a=iqV9L0=qVS4-yaUToE;PWHlHEfQTjLH%Gk(@n<!ZozaFLTuinMz(%Hfv zVnUuJjJs*T`9du9Xs*3EmP--`fi*4|wPu+#Gd{;6LBx-<M{FTUAeZ(a@gdJjLXU2! zMj*JdnW80m76WcYwEl|l{)M6M4Xusbeo+=w5M2JbCEDZp9)Kx39hJ4#GfJ1agmWTU ziM*3rTe$tmaN&Q;yh0L0;?f<Jhj9J?65ldf&_tpmV<B9p1QB50Lz6#tKz-b6hB)@r zjHAA?=39#0G^|mNNY4KodcK<mH71p9%cQptZV)0Lv<(s-{43-dUG^&fbAChM82`Ol z0&@?aMZiUNfL&W{tj{YdWcuC>ulTb&F%9{~aa6vzbj%>3t`@|Gl=L<Q*o|j<p9jTW z_4-d)lJ{oBa=}z`!FftzRx(!K;Lw5{2&TRt$;>Yc9o_lQH$kX1+^N)1vJ-}tWk0+3 z#1XhygzU%Ca$)E_IguR+Ap*TeY`M4oXQ?^Cs_Y<Y$Vh!@itUxf@Sw`B5RwpbRo=jz zJM?O^Fx@gpde>ngSQawN;de%b_%TwFL?0H;;;CGsVy>z_SYkJte)ndyy82oD2?wp4 zeB6e92?+=PoUcCghfnX%qU4J$=Z?m28T1Xjj(S>eHFL7qwR7Tl(?gBBf-wi?U$-`x zSs2``lYLcY2Y*y`C{;2{o7*o|bTEk1$%x&_FMM8IMM$*nZb!DazFFn>yvZMq&x|Cm zs55XS(v2Lg()Yx$JE>>A963s=!~Qgbt?1woMNYLnJ*L7T;dQ4%+<kVZ+4Bl>ltyh! zDu0@9?Qi+9l*pr)t$4FB@jUVsm8wP!Ici~Pp`8vfjx9OK&vRn2Wn0lklE+N6UGx)P z4f{)R%PczNTd+G>mnfyZ8i~)4K8H1EF79_|{jFXYRy};wA~y$9dC6#7B`kwWetJ9m zbHMcBmg>n@Scmu`nJ;ERcXGW4*azdKqoW;6OK#-A6E$*hhKxsJ`CZ1+LuRSgBdatt zCfcdH8W!N~TrN&Fb%qT;-8}i*-}pjus>7TZlX<pcTPDvlQPt&g$Io9^Pgb2eUidKV z$7abhj9+})C5K|odnL02arcgKEmE!<Mly|y*&SM>ai|hKUN{u}VzFOTvcqLtCh2gk zNau{dicgQ@=XdLF378`D<js?oQFN)d5;xV)`RR4VDLC>wPo8b1S~qOxF=Z(f?hP=B zQpZlV{*Es2<5UiNvGjWRRXfMSFb$?Vsc8n+x@3D|RV^N4F55j0%K7+d@#>X<9Lrzr zdKDbX-DGvx&Okq(rQ|oUo$@zla)SdMkDC~%zwF!NyMDdryJe3)|MIIr2e!eW1(_OE zqghekk+EOuZ6kia=sVfXV{74=qwJED>-_Erx>}LXEdKbTpf^(R=Kh9Ibghlve7nUj zUCeqrm+Wx-K;+UE{rtYgq;$8Z%WHS99Gizhko-Q~xEaZv>ge6mUL8{3$6=PM?5yCk zL~Lbt;JTY*v0RwESev@e%D&ix-}JM+^=c9=j)gyZ=P-Wdg5T#8JQlJ>`>R_YO`a9= zITN5a+%Es10JHM*colPz>G8s9d;YC<>KW>Urp64OnRbg&-I0*FH76%2p2uyx-z}>g z?mOt4hIh_TXIrR>$2LEj6tGR9`kqpCF!0LOhj+8_>$-u)X`L2yEQRNt76J<t%}Sal zgnfF>ewgsEc>LXR{&-qapx6vsXF-ldk-_QZ>!IsqBLzm|N4@p3v6pAQjVxTdGO(G5 zt2>vD@f+dQ|54}u+rD-Ep>dRb#|nIO$(#G*>4^gh=Z;Iq&rk=aSR`zAW!L2_G1<H> zB);*KP}{9RvM`$;8G87WBJKooLYXD?)H2j0ra_YL9L5|g$L=lceXt~8^yPD*O9%ff z?0G%R^#L<oy~UT^Y%t-OeIfOMwxJphiXUP;w#UJh)3B+8Yi_amZPe7ykkIy&oYXe@ z6Lq7jlPrg-xwSOM{xH;~&SLLhZ^|||4g2W)hw4xlg*c7;c1k+}=S+-wvL?q`pQBe$ zY`ru-S)apawdC_6qgpc6KesW7<)Fj(-cR$R-??f+L{6noZp6_zGP!>Hur$>u`ikL9 zPUZv4$jK9m|Bt=542yDm<3<VDAgJILWD6*4l~yq*$w37L1PK9Y6_6Z|?zBNdDQTn| zL`qr=O1fuA0qO3Xb3ZEB`~81B=UnfHeO<H1VP>uMthm>G|6=1|9Q|5$_%8qbHBvTI z$0Y_58-U`Dm1`u|P~o<zk;;)>e&V3gEOv&OxzPQo1}405ob?fRqj!{lrGUbo*n8$r z(OJAt;xT%dTv4Gb2V1MmUtf7N@gXcIL04J7S)cOwbK5gVsACz){)Qg;Bn*wS?&R2D zNTA=BZKM}XbZ_TA%M{j1LyYw!f$Vg>j+IJoQrpo7N@&TstMsw&=Jx53+t#{cr01yG zqv-bu)+Kz9nA2qr*c~=)7Iv2PMcBs*)#~?aqFK1nY^S3smp>iImCfog?0$OhkzH-5 znvp#7(W<nZ`a)Y-4GFvEv3lEtp)<~x1`@ej4JULZ`gj|KWDiVu2UB|;nc%lOxae`9 zVe{CT*xPja^jCvJ1iU_K@u*x`OKbd?fR>xvwRVq2Uf9X7J9OmH>^&`>bySL$-m3MZ z2|d-*Z7<@k@(MPz&Nh-)Vrm<?C7RFB6<{<ud*&WvuGS6JUbfrt6R?Wzzgp=XEnaEI zv_v+ME|J-g&z9>n+%lKvX<$2&H!Tpo@R3_$lDbb#N%1Lfo0O+l^npr&H!o;&Wlvuj zR64*FG881xcHEp!w0tsa;Fa_c(Q#V(g35lDtcGSi4ZS0;3_9LbZX6CB^V1bQU00Up z`BbA)O9{<+oUey9u`4TVrBaJ0RP(%y%O`y-1dw&m#J=?;*c^Bponm9mT?2LxDGkP( zr^uOF2pD#|oqgnJCq28KRQo7HbDfjxMVQY}<6JVv{E<m~)=^ojuztU?7pei{cO&>- zR9Ng{x!13EV^&Rx_xO#EL;1sW$+LGi4rj^odfggTqee@z3Y9R;?HW?@7Ztayl{i$; zx1rZ?ddij0YM<@<q5O(-IxM7UO67ap?mga78$w6@71k06NVB>dmrAv1MPF)dY-s_s zfOTeoe275@Z{2MtDXU2vTE_^zv8M@iD%?VCqH9kz40}k+WeYX$E|m^AJ$Jmimh^N{ zJvDyrWn-sKH=(4Fn3=`%kJ{mV?HjDNbd24EzGy!Iud=l?FG0qG9(+vx7LVELU=9Wc z!n^jpe!~KECB*mV3}+pALdVv5lN_C-z}wW(F297BX?E9?>)cbc->OrL7qdTeA@Tb) zi+PO%%GqW!J6vOCUwasRt!3pBDopjwkl#nfM>%`YG3J@A3wx^I_t$9(EsT}ih2t;O zou{n{@#ocT-3*OdD%J6herr{>qA(>|?I0~aNj+mqd62qKFyrWEsN9_3$*d;<P4;n5 zyU#16W6Ga&4=reX)TrFJDKcvw)}8EcG2Xgq6f<|XF%%A)p|q6YH>l;5BwKJ$o#iD( z$GBV!i}%xQkAGu-eR(~pj%8-en72)|JXp@b$#XVk<1Axn)4~C9)<SwwsRLlm8b0>4 z{7G#a+xg|tsJff0m*cKZnNp$^hxk*er>HFpiAT>wJ>@ei3h(1}KeWJSd<1t)dCXj5 z`Pf;ml~>ZXOm*oqx|O_jcg087WNpZsozi3KYOCa@IQrD6y9qs>u8G2%XJx8K&)C`W zTKPt3o~IpAOX55}XfDB9a=-4WDXnPv&E-NE!kMAAfuRlFD`A?xL-}X>=A1gk9tkXj z-5PnJI=4>KO}J1@G25)G#W)d|q_li&WT;J>_MQMNxh!b7T=4j{HeIrVgc5xlEDq%l z!T;5voaM4pdHq_<teTw@EHUkHNi2&g-MyvK%;`f3T0AStrya^kBF4&02c`=81W!6h z@9a7CbSVEogMCBufc4|2x6>q@q#p^Fh?Pe@t)iLTHPm)HEkuy<MbGHE${MuZ>zs@6 zN5PXXaLJG1S1%1|5Bh>#>$)1X?ZRkVck6-9fAbk#IBJnf?B}#0>LtlQs1-)vd8GYC zw27Lg#<`@aSD)f*$*Lce-~V(^M~*%u-(fWWvp7evJWA7SK}#i<YmQc6ic9xEJ2!zo zGsOl|6#Yk1zCjhI2SzxNQ#>5ir1=Baq@@ECM1`3KbE_Bklv^D%ONlhMnh!LzZQ$HM zg$73GfG(HqP_mQu6a_loQN#;3>>sApdP;?x8KWU8YGEmwwc&Q+m6QJiJA&-6^<j6i z4U&)$-Edt+$~}qn|5#FqJ&<nrD{6M6-)AT&T_8ESFj_##L-FBLCyN%di6{Yu@X$_{ zIu`HH;czdzFcx!84i#zfYPKsIEb(>*9pee%{<15Wi*tMzRz@E%`bS5adku=&CJC%e z`0HlP3@iPc%FfZkJdilcIQzKjkiY1d0e=$e!SJhH3X5t>?z11fW>ZomY#8U@g<oJ* ztbiq#dsTc*f~R&!#Ti*E<s?r7du7G4RxC@V0<EQc;lMM!XgZGQ%gtgKui2}r1%b2k zJ$--MG8G=iKI|+yM=QGE7hk2QNoyNE>2DN0Y%6`bu3qegQmQ}~d)AY}C~i3?E-VXM z0fr^BW@+Obn$lKu<=BG6IJGHlVt6IDS?6$x+kjKwS`>!GU~D{LnhV3i7J*4%!1NS( zW;w&Qw#y**$>LhDEtA*5x+W}3{3p|^(Vl;o%T9313HEc(x}OmAYQEJi*034iKc6(k z+h_&<RwlW!ER@=R3ES`@c|)>XcD|svFl%b$f%Sxvdj24z_mP7IPn5EH-j~bHwzq6f zmCI&@6{?_PY!lG~I^OP?&63f6N4=u!#roP?P7Rem!LZO>lnqZxu9`&lye#s>PA;!# zz3{liU3{9u*AxUdc-_UsWDlh9@m*^cYjBe7R|{+uo0U5ds2{B--14e<?lN6rv6Qga zQ(LRYvzKj-3%0#d@<Tsh!CeS{?r9J{<`-YMzD|Q-8B~j(xIi_%I6s$+yirTtU`mN$ z!AO6ESnG(be#tOp$9uHj{Ora#4OZJ$<LG5x_nem>UZq*lieh&T<*zTEb^2f%-r#IN zXmVu2gLO?5{?O=%!WrA~s_LbY{BCcDl1rgu%}h>d9#zzsOKVCW+gpUfGZq9C>W0^o zq$_XJweYqv){Rsb(kF&@i%G(<lYhH=Cv3icIpE70X3m%1{gB(eR<m17eJ&v9NjGhj zqtb-Z8+L=WJWshohmwb0w!&7?cK4CfGVsvFhF2#}`xms%ER`lqn<aFK^*KDTqkFgT zxQFnK?tQhMsjJ<44o3I0oNiiSm<jsV)TzaE(pGFx-YF4%RZFvZh_dAjM(pDT3no5V zD|gW}W(j3EY6Az5=j;-EqXVgXv(?{V^_+fhBYAPiFBUxzabR6g#+Ip8(<zM%P3f<$ zKeEDAbp&oXB^KSX9UCtdKF)b9>iY{eh*i!RW*K>!=NG~%Y+nE)><D>3-9~l?=|^?x zrs1ohPhl?-?5|^$U4awDRx#E|THNkl9c$(;h@~&ec4+5ig+Ho$`bdi>OEv(e=W~dL z1*{<O=WSaVdXeBcK!v$9q~F6~QT%Z6%3z{-*rMn})978LV!M(mkM4Lm>?NHgEuPhs z<O2pra36Zh8ye@lBBGncDwm>ZG-SguCL6q@Qx~C8QM1h}%5rE*#?*q#w4#>cg;Qc^ z$|CQI1hMdJA;}X>Vhv@95sdQKOV}AL9*nMGj%<5;pc8jvCZCr<uI%{;#%%HyA+L|z z?p|}tp)h*`*VlvB$JN_R{fv$j4Gr^Dy)}h#8fr^$;Ipz!Sw5>CAwNKMJlWQXP82)e z4^vMs`ZnD?Xn#6Q`IAL`$AHH|L;AGcVwjRBG&_!Fs7<e@(UvFMGr*So@gkXj(I?$X zZQ6GcjPUTY>nxGwLoF14cWLiI^Ubr2Npq*6N6K8ama-lDhWKpnnX5NSa(UL5>3$m3 zbH3uxT{qW%$<=#ig$pA+YD(!Ga2b8XHa6>t!#Ub}OI;WCG0vuRiEX};E|=wvf57cN z`>@2<mR6MC1vWYHp_|S<p4{U-r{s#BmNy*U@S-g0v6^@i*p7qgXs*6)E4<<~K>JB} z2j(G)<gRLwk`pcKsdq?e>a;vCuq%YAM_)e2Nrmxc=K~-n?fK6CtZha<v{c`H>h`sZ zxW1CJb2p)?nQ;}R^k|*8xWf6;L`!;C;{kAmaDBWNAwA5cC;{o=K-2WqEM+1KuGF-} zai|kD2l)**>$UFc;3YV4II=;6la#Dn@jfSS%Y{h%tqNom!@qwMj2u2W;=y=mo%6ch z$D557iPQ^7!KqOyG9x2Hi~>bjC$*hE*-8~{<ygLl4Z2zYxRBwX$`ih?hv(PA+tyk7 z<Ns1XYO2(Dt2p=j&wM?Dg8>rn!?atR>%V?|t3?bXE{@;Ov44K$$_WWK1}U1S2LDTI zQ51M<Keqo5E!RH`H_>3uqb=W$%vRfee$RygZ}Ct6_Cw1XFF@9cQ~y%Z7naihp(P@C z<G)ZJe%HyEN07nPZ}hpkg$Di4@pZ#%C;rb5e7$>L+6P(E#+ZK-x3xI_FjSwP`40RH z^53=X=l6V=Aa@ueeqL_N_~7TU{9(LE2(Hur=Lf!agh6Tt3Be`b`!T;@)_?weqz^D* zh5oBATay?SOfnEBE|~Cj0BTN51MCa~oG?QG<F>xgx;WNTQi6Up7|@z+*mkSVUO{IX zTyooi6^>Lgk_~*ha?HULvjae=H;)XygxW~UHKv1_R}~aGfCH<I07exQ6*0Z9@1Vi6 zr4_)b3L0IXjNd3|T<88ln0u>1$b6}fLXxj6JVBppMp-(pNTH><vg@Vp71ctU7oeT3 z3_5k{W|fSHKQm{NfqdNM7r@W$s^#t9HP#p_8QrR^K&Ei@YCB@J1iWkAmw(|`#H!{R zb6ZcU(E?F>H!b%E5J%<)P_43pg1#m86_^E0fbnV8Zl2D1N{VTKkW@P@Bx`c9ksvN$ zFA)v$O`t-zCA5J`^eC2ZIP%WNk>d^bO>YVO40GRpyn4P!KPqoUe^H_qj(6eHZCL@u zzaA{$FZG}h7c2;sL`28=c<wcXyDSJ*N<=M$!@5JE2nF1v4e;fC0ix^aqf%|{-B95( zLF}LrA}^v4F-3<1$y)H{;!b{{mR|^Jm^p1T2s9YMcO}Q_d+wgKLDV&&tgC$gyz4%I zaH@KaBuDM0K6dRophi?YCuV=ntZy%J21jJVMdZXVeSS8_kDa_;iif~yq?8Q2K~3;8 zCh#@LNAw^0i{;Yw8m_ka8+|Q7$!!RTmcty1n=1+*5vgBda!}JlaDlprd+*2!INRMs zaA$~J@Pj~M7y&ehH*f=My@c^KI9m>m>FVfoJY|#|hf$hNo@&8QkNLO%<;$0d*<LDw zKUP#%pNJ+OmM}wKjK&{c{EPHbJ=IuM0eBWJc3Ep`3UP4M<;rWSJA5+RlxNPXN@O2U zFebv+v%vGb8!0^n1kaUJ0z_a0-~!yHA1@$ED&v49eNkNWp#7odgnrQ~dhpdS_by<g zX22=fGT;Q@DlF8=r{+L-;TRTb<7g;=FSda~$WrgF<Iw^LJ-DY>jM)bCoKuz3HLO|` zlx%(6Fz4|}f!>${Qudf97Y~gb5#($>l%1UVgxoKy%ZF3tf<YsiVBn-->oLStF3=f} zR^7&+zYtKsa`M7!#?NM@P?i&t8cVEY3mJ{xdo;N5zKY^nGnBXcCXxlHGG^<~lPj++ zPHf1ou)8p}oXj5oL49%iX(E$SPnDnpot)?Ql`7H77`aE^a#<Sam1K)Rkj311aPU() zqPZdeAd7Dm5yqM7ZJCuA4ebVRQO95n!vmJ<X~3e*0R<7;2TOY|()At89W6zOxJ6p3 zVMBl$2un|nbUUT>-iCP!&{rSB-i=ZyF=z65N=itKcbeB_14vm#*=Fcv&KW>wI}7U8 z(oH=R0?AdR6qvK;AfL@46|uzU`O5^=U|ri!dRj(K9Wl3kKxg)F1Jv$wKx@7%-A{>l z1H{+|T=$IjI)PxR(NLJxi%MbXtdTQutN92HI?U$dBW~MSZ7fvSCt$&5LJ71JCJui@ zI-V<ZtRZ4dcrcldr582g!BGqrdPMX_P=74m?I{O@R^oQVuAs<%DJRSdQ^|R(B<-n^ zL4u%zs~>K4VQgrv)RU)s_~1C;tys-#5PV%vINv}^Ls}%^#1T?E8J~xX^p=~|ERO>+ zpd`A&4b}SoCh(>lNT)*8`*Am;cV=B-p`Wk^S&2wAEJ+Pr|D_1rdi>NCwh*^-+ZC~2 zPENEYfX*>av~q;V@#Pw!j8bwjT(*uyoTd3KEg7AprM<$yAkoQk1&|VG#QPml+_tdH zO>5WG^5k+jXn8X^S~aY%&%la^Vz_D=)^1bt>-3mCH_#WAa1UZ?`w_D5MWT9z?tCs+ zz6F2pw2ZcVOwOc2dS)jmy5DG7mJNFiD$9#&b;)Pefdd@DiW_WU>_(+B=LaMEnm7L^ zE;%>87q}kWnsfK5beD(4j%%wbqz*DcSGkFp6MS*Q(O%i=xkAl+mNhdv7Lxet@8~bX zKwL*W103cFg1ER7-)M&QN%TqB`-|i6lmI$=sPKt&K|gB)Is1e?G&|27Io}Bw@Iwz@ z08V>Zj}T!Z7pb3Dw#4FPdl<7n$1+uDxRQ|?k%-$?Xkssgb;F1YMeaRB$r*K>BKE+U zTQ(GK;q151rLlPvpuli~GiWO_|NRm4`RoMLCxEv4cVGHqO5^0@dh)G3unQak7U`s5 zJuIU*VnI~EXKN|ytUdxyVYU)csqp?MlA6X#LmyDMn+dq)Q=v3?k@|>H7%|@6A-i2m zQEkx##7?Ut%u;DQ_txHwE#iad5LXvL<SENU^#C7Nf;|@R)&OrjdVcW%_ULCw@ulHi zZDbv+H6c88{Lxecx649%u*D@pwK?r(dEU1oYYn<uQ6f|lqj`!Hu4yqQMsh*t@LXDD zl_^ns(+aP2No!$`SRN#DW1;9pc<=~JoYVb|F&70!Hkz8w{KKil<aR)m7#3;-QCns| z$6U;{ktw(zgB2LQvKD3R7>T{CXK}wF`r_L0!)0ay+Zgqq$16)6qA83qR@UtE{8VH( z30MkxB9v0C;j$y+?xFPL7I}GN<H^$pmv0o`k5g)-nj3AtB6Z`MjTG6zqQtqtB>{_2 z4wF|WHJ2VK2`Xltz<8x;2m?qx)vzti3f4L1Or1c#QrTE5$*4`6|4_E?{Ujy4GDLC$ zm-k6!t=gX%vopFbi%@K0#G^IM9snF4Pq%g}s;5)&`UGpIMY`lW&|nWGVw=p-lCCuQ zRBsR!p=e$`O&K|vw+Oi1`OxMV^wTC^g8n?MK{WUQoFod=%OaV$Lqqk^+j%{D%7e|< zSRt~3g7Vo+tEx!{3}>-Kb=mtr2Ae;Koj@E9OAFC>Zxd5W8mqFm<Q~&l7`>jFeCO=d z&rqugyk`8g81`H)I_HU)wd`CA0}1s}_7G#|X*lPxh|bE=b0yTBwC*Vg%Wy5(-P~9s ztq`NaR|nS$v?iXHPv&Te(T}?lB71k?GGII}D~-V<VqrkN>3k5gQZ0$7?jXd9IO#A0 zm<Lv#?*iKnaqUm*zgHC<gSVH6kcAa5DlMl)Alv@NjpyX$5q2%bd=r5@?sr;ESoAND zb~$}SwkrNncN}ngI5kAkW^nK?N4OD5pB?n0Il}aoZ1yMm;dVHLimZXSwEkAGyW8hE z1Xu)OdWU*8cJ_^=Q82At6U4&UIC6yZsRIs63(QoCH8;}!?dka{{l8v323pHYgYP|Q zb-|^)1cW@X5Cx4Y835#^#tG{pa_6rOL<yRv%n5=`Vb$zFrNBX|%OVtrOcX*O5xTGm zULxaQnVe0=QCMHYi34AiT-bS9foAgY7VK;}ukUbYh)V7k(>N$dwmUUlm5ikqp;G05 zXj5A}Jsb|3d7vJFz{~ou;OY*sZ`Z8*r}Mq+B)x!~UflZ+pY`PUk^P~t4@(()gW2R- z32sfeNsj`70r4f_(R4fqt=t>{n%kGU8XuUh*4%8QtdERzxgD~@lVTbq@Pik3X1_A- zi<BirKqTo+@R&ff!32TF_~Fdh12bKYQC=_u-Q{+`_6T0m0WA`yc*LFk4moA$NjO$D zlb@IR`_;^D3GVzdIe#LwW}R@o?VrOp)*_pnT*;M{pQb@Bez0j&y9hDh0jS=5rK-`3 zd)BK~0I0NEnw&U42ymr|ZsipAcc4ap6M?mXboW|_=95%^&L|f?AAgHj8}-D*rZLR@ zxL9N6X4$4dCF<iZ3u9^>iSQHdz)&tNuCg+Ggn(c{!Lnrokb#J>orI)h{}ZbFyD;ru zxUNUVkB#Xglh<1GQ-!<aP_K#LXo{^<I(aW`xQ(+05k7MuS(iy~=DRlwHYM*7)g`P> zp!%oH;Ivx1E>e^Z9N1;tWzl@|Ejjs7LNO2mcbJexwjo+(b%<8e$Q7Ndy$=Y)n@7@m z@jVSF+^8v!KJO(bl0x`b2%~8E76KHsi6B|ITP~wiB%PX0K%7NyfCVNH7f~w9NDkh+ zckd2`M()MqyfSE%rb=lOxBlRlL;LF{S$~*7xxxEWOCNDhGriB4eVSZsZL)gR{ZB0# zR<_tAL)$pyu5+C6D^5Ji^$nj`n1Q8rS<Y4Eiuj40!=tw?_l#*QTB@t7BS7vjmo(HB zseF}!{CvBhk;Lk(`MJRs9nq@jS}S$Wv?mmDzI^YgYdJE@=wGHt<_e6pq^k?4&Qb1S zXv!Bd(F>t*7tY=%pz+F;dG1rqoa));7HKJca-!42+~hR7Bh%3O#eCfgoB2<xILKec zFri<<25f13#`0BuID5u9#n&qkG-W_og}Fly!r<hZqAR0|<VSa?_@eYp{Wl=*lW*Z& zOXwIFPOKt-fR&5@Efsm&E~q9%;no*<i?is|Wj|_HMfT4?qg&=i*_gkgu9IX%qSpBv znHr`pon7~8cAM93`9R=XeC_WIMIG+zdN<#`zg6p1^BIbq66s=mHo3>`4gC@P6o#Y8 zU%-X+J}!JwmR!z`;s=Ye`R%if@**4@$2RC-`UO(4$~%VDjtX$F)CX2Y>2N38j_yk5 zXvwWv4IpoV+ug#nMNoNM)&o8aS0hN9&7IeEP_J%Q+638@wrrc|xQ%{#+{25ZG96{Z zl(a?}B85(lEAPn=jeuX4*XIo{QVPK{#RYqmZJ(E()XU&)GI2x><}2+Xi%hsIcOYie zAO|Mh&r|mF=ks*?UXPZ5lnQr?+E>w2B$6*I&A9RAvhz0W?%O8&<(2D)y%n{gNpkXk zXx<N>Z&ZcvpEuSt_~Z2bmP+sufjh0XjF!B@|M{Cilb}LsYE+j0>qmhI1vjdbX>yKr z{X_<W9k9|2FE@-~nhcJWlKuJXFwY?OgHCzP@>Q=Q=<xJPAU)Q%&iU4E%Xu&p(rmLp zGaIoB{8in2-KBZxh7ib)eQWWLud=9+AVoq__Nx!zX_;QIg!V{pRvKC#Ziv>pm(RD_ zA0%ZHCEzD_XU9C>0M#Kb&p$?pE+|7pT`D`5V(^Q~0mFHs`^KD|K(@MbE8THIdXr)o z#UZMrkvaw(ZR12oK1c4qpE8OlcnDM|Evl`-(sXSG{?uOt6uW$i(b8M|nI9e;at*Km z-VH;Tif{hcE|;=Vk>KOg5TQiy`daZHlV(T+b^P-c9rw$ql(JD)Xxzizl;0XK6o&M! zDuJGbOAl^yNniB}+m?{?)(5sXxcVF<IL1y~9ND5c5E01{;+G}h9e*aM{1xrH{poKH zAwvxwK!D`i?IxAVfd)@6L{kf@gnCzh)$RE9%c<fpM6Jy<ep_-2n7b|n`19p@c<88C zEL%*=pF8Ozi_F-o=JLv4NA{Wm0Vx5)pIcaufy9R%`uI89uS*-^7QohXH(mXzwfpM> ztoR2ybviCFk>zW*QVj6L2(Pt78Ik|`;V;>~>pC!g*OW!Jw6$CxQ`WQKK})rp<n{ls z7=9L9fayn>2R>4dRq?al2Ry(0ZK!^F^+W1Xib4j@WfR!I6V6iC{6&94M1HrO=~+Rq zg2SHe`uZM&`VJDjOMw9~hkp9{t^Y->k`Ba~v?LtZdbk+|J2>Z^^N8?{;HA*W=WNMv zY<+%vdejTRGvf4SSN+P<apk1AG_j6^brUc#xCEbm*Q%eArFyCGr&+B5&$lcCu8-a| zsg$-JjHgUnZ8PV?#iIUL63}Bu{?<PrD)dW>w4ascJ($c`qc1*_md)2MPe@=;;ze!r z*0;VRU9_by-H*@H%bB@P_z#Z<&y|}6QtGruobF#7i&0D@bXL<Za{AL$TZTiGE;<_M zIy&x2d`)m8XtkBB+91I>cA77x{K(dDeOo14uh7Vp&uLci`zmI)YoE*mhx1{xi3isb z>&9*-8LI9^7o?$2Zu>v|G&-<Bw4Q)A8JhfCwY9r1kj7Yjj2B9`Cc5?WTG{$pAas2q z=+M^?PTKmTUmn@FkGg<95L7)*NlE?3T-t6|qdN4+!sN#2|M1N|r;w$4SzccDPb2<q zb%CfBY=m8^&sP7uvbWo;{TPmv15M3W{}>~G+h;+|2lAjx`{SJcYYe_l<pYR9(|b-X zs%^sApJ(nRYgk^5MfVba(xBg772C}PQsDm`;;qs6zeBt=LH_?fiy3&j*e`8-zknFd z>?p=<+S>!8WC994*&vIJt$%)gLaAicUq?6IWPA?f#3!JHmDBg;0Twbt8qv=|iC8~G z*tz(i>D#?R8<1GbJS1p`sAL-)Qv&Lrx6o<=2v#S(pGQ>FxjVG?A%Z4X+7=-<I1U8| zwCCWc-3;*1+d+<83v#~2P7y+m2+jN@VulNMgLuT0Ril;okWFO67^lwbcOKMO#Fmg& z&@KxOXJaX4f1O_sSy#m)KE);XvuSRU{%|bT9KQ|sV+^QbXt$}Edpd%AMfWME*F@Cq zpbXQ}shr-t*keB>STI|~u4F@t2v7ha?*n4P#LmX%o(q@Po_55Rb}EZ}pUJDi{r2F1 z_NIDy1}xArfCV<i?YKH`6U?`Uq*z8jkRvL~r!32Om=Gc~x7GMHdk~Q?l#&LP8rB)N zfpCY6K8!A$I0FQ`5MI-pMRQ74_eWE&KNa=ZpE@3Q?MK$==Usle1LCLUj^WBL9x`9I zXvKRjX}3#2Rlx{!OI|2x*uJyyUk0uJ$*^9e{*qJ?)sZs;8G#N6JWKpB#eD5RRW^hN z9bpm=G7$XTmJ5z~`I;j5{pH?E((6y3KBZZKnv(VAD)jhqyrD8$-kaT*v&;oE1adC( z6NzEJ&7{|`mrmdXw*{ttp|pP4R8VKg)IjA!QAXxEV)JD=-riu|)NkT67eaH^u-)F$ zuLN#vaYHntn~0b`TnR(&x0FnmkvaZj<H?3(Ipy(QgK#>Mrz{S;-`dtPf;gxiG^_X* zq0?jFdO4^xWIfheGg*Rja!ysH9EX}BH_+ZW(xJ3K{E80?^6A<UR_~~kY&3!q!{3m! zgej0>K!a^&_jAx<*Nm37Ye?4=<_BRLWKPY^8XVoBZknFF_GT8WGVhvbMiV7$CcfT_ zQ2EhG0+3kC5XK><qTWbdQ8?{`bJ7xoQs~bTDf%Bm&o<rJ%UM+lRW^rq$F7@^P_v4A zvbR|bl7ndo;Vm2uiEc!FJ_o3ZEy}yH{ZlA2A}v9GB0bM3|6p03X}`mTI~d!0o<%^a z6f(Q}A-Dae$H}0ts?Ogp+Q=FZGew1<`AC288yqc+;6?{eK&=^snH||iFBL<+$q3<? zA$k@3PTss9N!U-ESQNIbNIp~alQ4^J;VT&D>6y{Xin>msqW2K?SPnd-khD@~n8@hf zQr9>a7odiI;2+ACwY;`8-qFNA1QxfFSXjCOKs0#01DR+=@+#*y*-Lv7a%;8R+M`LW zbQB5Eh2bkJQ@|E1e#W;+Z#%G;{B}DS&q=*mjgPyXjYFhtU+UH^D$~e{h>PGQ)m6n+ zRY{QR$sEWo0y1DVC0(SI%yb#tCLLFyP~Iu)O8GDANt7!YF!{)RKbC`+pe?opMp8r5 za)8|h^-02mnj?*eqee-v<<@B=w>AL`)K=|nI{Mbh2bJ+SB2Y4y;-SB{t;h?a3@?cw zojZy*+I~f9^xBEz$FoyXj;#TIbV}*{*na%cOKmO75~D|zh%@fLrUyU2EBugg7>~TO zXLbnmZl~E7uJBKAB3tUle!_*+lJ_)*%X}~eBdpJ@c?_3mZj359L6xI~>vGBcNyOZC z!j0_2^b?Z5$PXEf#t4I(_Zv1R#zMZANYZ%>o11<`Y1)mi_KM)bRdTwA)Y3u?diX#z zj_V;1@#S21x`bc?gu9@eB(+qibo&n98kI}r>BMNwBX|PfV%Ob~CQHnj`5IAyM+^?d zeVIBy;DnmHEv6hY!ri1~WL?)R<$T`3M4SklA~lxT#4K{Psr5&5q`=_kfJt_U@sZn5 zuT=%Ta{apmU6|{@4HS|?8Q?=H{E%~E11;xa@JR^monYqhJW(&B5gh-7T03Qfk(s%? z*~#F{?}j}9x$9P^UhMd8AFlwt=rneaTxpI;#<V<1^%~OhI{W@Xf1AVZn)F(bdYVd) z=^FuIkAGG*o49CP(kR<*&(m~kCKfNio9gFV*_Tny&(COJWxH){ZSC?eufks8paSYK z`5)EAy+;ev*+&+M;Ua$?L;lZ6291HQ$C~O?cah+<hzQ>3HKkrxH+;&wk*kJz4n!;o zXVgYO#bS;D&yzAEN(2wivqTfgXomG3FkRLkAY$xtL+U?^!r4!r9-`V2S!?fqKKERG z1rZHZFS+yb9b$`RH&$Gt(X0#}>Jl!Zh~M?4T{n(*Q(rUQQC2qbp`ygyhm?=x9u3^8 z{JXKH=5P0y-Oq^_&xw05KKC;wE1S3ya`=hynnrX@bJkMCQee+3UFo3@XV27J9{NyN z?0wO^c4ecuWNK>7-pSk8x;SELC^>}`5p-7VqNk>&p4A{l)eD;Bk1+L|c|fx-9j)=* zSbx1exL!kGTl>Wx32i6UASOr2;7b`L+w2?CU`*Yi#D)Vot-SLXZj@;9IxMWoZhPT# zQn%UFi`<z!-l$JWH9|qT*|4=p8Z@`>?>RWCn4K&f?9jF&8?#Fx%e%;pHtTQUjO8nn z#NkazwhKUgB`B5;KH5gFBRDNLo{T0s$3k+wB}pMIjj6|93Kg_>J_V`hV2e@g(?H%; z(p=o3`DSl%5O~b!zjN*`v~^TsnR8)U2;tPf2}kcKK@)g>j!eOCmFivO=GbZPQ~@h? zqM#e+nJKs<*uFmw<>5COyOyM<`q#E*%(o4|#~;qt3W?~-m)~98d}?;g4p!?*g>YLY zk5hx32PC;gWbjR9)v$y+R8yy<J`xx4j$BtsYr9ZFfX9j7lWnNfmc<mi;jtQRu_Dow zF77}@#K__yGFau`MUPuebVMld`I1)S?R{?K^CBV?V{Pfy9lNs*Yl^auEL_AV$hQVA zbLUJjmmwxor@C`3YhA{E6CcmuJeFqIro?IEa{rKy@-!y!jvE)a%8Z<*(xb&QQQc7o z-VKB&dPT~qu022Y*F91yGpP>|!adpPD7VdxwFiZPJS#XTFEQ7M?9qs@{mm6BO1w9{ z=d`H2_b}NHcV-__e3LSj<Z6F>l>U<ctkZ{Qr-~I32b@4|jUar#M$LWiFyx5TcrhPq zk$Ck&aIcbq6VX3<nU3&)P81bS_DiL(>xqmkmwVHjc<euJrn=7psvVZ1$^BffQ?phO z+YBmp#hB)I`)U)rU{89*aQB5)dGOK@Kml%BW=m+I*-AzBEvz;b+L~KSC(kG(DWy*` zjbt;rxfAU76D`GEF<4VbRy@A+p=5p5qx$qTSauZTWZ%1^q>`oA*S!fYua!*0;A25& zy$-@e<|CVcl@VJ~R-|cys@AJb01~85+DI(1X%-jK(mQI-t6$5w?mA~iJ^LF#+`}9A zr}5uT_cmC5e{DXQE3pn$5Yy!1?5a-o;LcWev)J-BQZp$!!-W+9XsJ`BQZDsKb{Evz z*Z8vsF14nfzA|DbwM@)TdifvF)Y6wzN$vqn0TJqi$VVIC3-fBHUF<vLKq*l3?MPM0 zHgjxdTMQSp&+oU}&%x!Cv1U_lHdH%N%@S#!pawqD-TF&ORMRTz>BsQjxl_FaD8*VW zd$OBihvB_q_gsZCP^4Bb0C$00S3=^|(J?7th^uF~;6zN`Ih~{y>;Vv%9<b%D0y_cJ zF=>g((zk*$t6(WOa_RKpigI?+V7-pa*lL5@P>{?kt{w+xj7*@7=NlkOc7XYLwbVD+ z0z~~>DpxngEcJ8IVxFCr8n~jh*)M9!Teqsu3+wGulowd09^z2X&l-D)<Ip=-q$G+n zcO?&Ld{vs?hapvr&_cGJTRV-^g7hBkyqIdxbxyowe$S@I#)na|B@qVxophH4ZjCu( zO++OIUS#@j2H&lcuB&@_*u^DiYRdJgxHI|H%UwN$DlgJnMq?%X`TpHqq#NSn)4Leq z?cp)ECw7!8yUkD)Z$dbobDa&7EOhErgF7Yb#{$?s?A4u%HpYwg@ug|GvMv<Ojcg>L zclwD$-tH?K>6+g;QF-C7;M;)>fVR8rdVp=T91MPMBKlUgK?LixJ|3&XQmtclMB;jv zH}3Eh8p`Ddj_L`_4peH9MpYy+I?ery&7Sa4NKh;#eZRp`%9t#~QFjk|`qo5Z)x?k= zht>(;fer=}Ih0&X@}0MjQCmO68eHyP94mdjCeu+Ir*Q>yICW7IYZ5zYVDLH4)(&uS zqK~t+o4V4vvHRk!M`m9&O234O4{7kF0t~5aB?h&b^W^P|HtPbZ@gkk{GnJj!J)BXf zqvac8F+=uJN%4y-jdspD|4zF<F2r*Iv5`ih&1EgsBD-usfC*^$Yp1(v<ad`8Y5V&M zEl13i1S(zWILz{E+6JY#w<1X4(A}Z!tkl<rIJ&}t)SWs^f9mFi%v92wxJny>@7K_m z&oh|ssMz6iwzhLC5B)XTWgu{c=`SYZ@m0|7yDb`a@aOPH6cYFQe{IeWGw;jvLrNZ} zxUXAm2@rk#g>PA@?_RmeK$h(8(>~rGw#08|#t*MfF+qar@$P@RezUv&@*@m>(2VEW zyg$V4e`r7I@y-FS7n_Y+yw(aP0?pd68>%0bbbR9nPwhpOYW5C4>0cbGJra<Ml4k!W zuo_1>1;!no1h9N2dN;Kc^R5|Jhw)sbQSCdugMx&TCA0)g#Z9rLZybIUik41<X2ttO zgnYfjp(F@E<5GNta|nWA4^ldQyUslad2gLHAId0Pxug*w`XH#hO563w18?k;^X12` zY=#D~Q;XmQeW;tr%s@romwGwfG^6)@zbKMnNZ*cDG<>xo%*;dizOPArGr^-h6F&ex ziww_7U%%DL+05NgjL|;Cu6jx{gN+VA?pI0Qnh0qZ<Q&W-Z6Wo!am3&1>O}@d*JUYj z92za`Cr4HLV|Fs=L}_O5W#*y#-~aj#5A3^t_>%AGmq`BZy&vb;cmJTB4}%mWUAv_~ z_1%Q|mNeaZ^`(CC-e2+8m}~1J-<C>+1ybw?3#I)&2>W#oe`(p56z*g8)c@(jBS^lN z?<~(3m8}14GKxV935EVqQu&`|auY=G6&7a3e*_(WH|SrJt!Y`mIT(H!Z&VzJVEc>= zbbj%d`E|Iy4ICunA0tS9Ej9fu`RZSm+;@F;y^LhEn<`)Z`@bygFKN-dMz;Uu!$C+* zAkNqP{I8qacMbU3m(ya9v43nD`#)5Cfk1+!_y6B8p<O!<G#&fntYd~<!s-Kp0R|<- zED#P<foL=Zf*QqK<F0+|@iP?=QKy0l8u#^5JmkLI*e)?#ji9EmIcWap=Yme_jDw5W zOMtAlLz3p!an&3P%f+c46Hv3s+J9W-+AFg`>AVxuDEPV*s|gIEm-SaT>Dmh9)3W4Z zC6qzyx2&~=cOA5d7r|ETZmQ&#7CX$6c4U@Vy>}1)?V_QWBha95IdlOZFN7OyR@a=R zFK-VLt0{2$GjBX5-b0tslB|S%e*S@Ue(Lqz4ggln0nJ)=*{BPdP1y!5X8Shsm*AXo z7-~&X)<V#-{wLHiD%l2u{D($s@-4>>lkO+6+5lxF9e~L{hT~I6sM~_$+hqjRij<rU zJ(;h35w`wm>7y=#0B&>TA&3<D9hNg3r(Y9uTMWzaBqDa|&xV~L*=jW2{(^aa0a9kn zbjivu;~>qXs~_Gj{9I(uck9c?4R$cr-rJ<VUsI<Q8Los2ARGmGQs>yr5>WP4<6DCW zF&|9`nvowf3BmuTBDSM0geP6~9&WN=3IhN-9mw2p$6`T6%wu8^Yr+#ot_VrwAq?V< zGFTm0ccN%sE-=#>f~?P}J0x-f&9X;Ra<LU{cQ@hbjq?Wt@KljT;t~5pe1!ol85%(Z z=?E<YIv@Soji@zJgwPP1ld6!W*3mMDdv5d>0=lC+YnFF;C(Mdj;e(4CAU4%$j)UI^ zzHWR6A~p;;!tAMh^#UtpxIT5};346fxs3C7!OIJFpEFhk(sjSzq8XA&IE_2o1*;&6 zo&cF!3*UyOI4Sde1ETJfFA(-Zxb3F{N<$DtXMhVKB06SdIofP0f_LPiSC)|D>W)EP zyZHprb9LlKlu1dQVrL)-x*M{7LOC;LqW#4iP+b!2$(9&QSJ>pEPD-dnbj_U4#kMGw zioCe_PR*I>c4SoQ4wyc-RCA0>Uz0%EJ@H<Z%QU2PgL3HeAh&lc8y^o=&r1;IZig!~ zdRF1OC>Z6bD-uk-qC`DEPBYq)tZP10n+=)tQUt6$#hL!?Ks&{PVsiP1jx+yDY4ztx zr?`(rqcG3@sMJ-%OpSC9P|tCvV$bgfz+f&QI#uB=;pn{y#$I`pYl!=jljPrqTUr3B zjKUxa2D}lEslp{OU#!YbgjInBCP?odau9rCy8P@qOsgDN#z9P29+Pe(m?+l1eM!wJ z%4gCa?bF~vV+NR-ivSZib5|khYUSFneX7iJH!{6vU|lu!tXmDN)fPdCJYP$v%I^ho zpSyk+SSo9yp7g@qNb0VZ)#l&qci1!({+`j09L+qCACn3Qzq2|-q*Rdxw@oHm9gWCu zE8}e(UJ%|*^UmR)REWNgiLwN;Vg<eBT9JNC7BR*1TaFHFvaq|jE9TlW>ek{m?Bqu_ z6k5|Yr@|JC($rox7A;*dcLv7BE3GPlUa3VS%Qx8$)dG`b&5~j=l4-8Lr^S3{F)F2N z(I~fON}VIq5(}d(U8)Q3*dbq;y8FQCw>rl!jomFIpr^dvOCNe0iU|35WIC!O$Exxo zQwg*acHvWQpEa)Slnnb&Mqac<KgM-iVYK8<MA%mF3al0*cyNjt*F_5Yz4)bV-m@tr z<#DdMGLurW^zJz*<j_7lR2MNm4|`qbP+lHyk7>HssNrk@)FzDWT@_PRIsk8U9f16c zEj7bDnA501P$Dc=MfVhUH3zcF$F(Cwrz$H7@w2ykU2G=~3fR=U?>`<qei}C<ZS`@A zd^YyGLC=BR=!Kxu{Fb8A4@bA_GwS*(=h&jQ;suEZ&LEK;q&TGn8^Fe+0sNtwkfEe7 zKzx5nHS5VWiQtd(TVg}f09;1x^+lS%XtGyu^au5_5V=LT@cDz>Vp&n#;zcjIvz?60 z1y42hlV0#C2ld(;km1P&4D{UHNxf`mz<sI<65$OO!{yOzA<MrE5jx(3Bvr~k)kRDZ z99uuLcbO;OVZ|)a5he1^kHb&Gz+LXC@qG<sdqGh&A<^`Uo%@*DlN6F6Cn;pUuZ~=B z-p4);PP@FFwo-1^P#`fItkRKaK(yUL0{Jwl{EJKAj?HbqXcPjXf|7lj*#-(cBdK$+ z=X2}quivo>Wjb_w#tq}A4%Gq27a^Ca0$TKtH84mrz#Kv)&XufuqRka_$s~-4O-M(u zzm7nqNfX8>G8}&wQqg8ba^}{D49F5;0m1vhnQTWkKL|tSin@hn-E*lTqE_qGKr$xO zI~dB5q9C%r1P_!zHKWsZqdiOCm#kFgJ>J>T6~J_!>ap$q?zx7S;<#(vi;A1HwkPNh z2hPJ&_$IOIuyGDEWk8i=f@zgmDp*_PRhk*(lRq=tz^JzsxM@hG4LyO$&e5$4*S@j_ zHqO_egw=J|2{H|cY3pR6ViA%q&oS=eTLZmCwR{{@DMK3jmxdeOAUgLzv9n6vEZWpF zD}p_GaurUr6CI8O!U<_iYv-wAC)*^}fLfEs=DKf*Z@7zRx~QOZs<3fMcxAvh$6`bd zqJdy*%u>>jk|wg*^E^tAbVbrMbEGH${N$(%!wzRIchY6lCKdu8WJBWP&^ZSp%)}{h z?sV&amvFr)!%z+|Yj5^gKZyiqn^k)9l%W&zYkRM?O<k?eO-!ZF+PepdO%5<*nCe6^ zGSOuxS&}h|cHsQ0I0%i;`;stFFx);l^DGsIE}>$5QeLIFYN+|ah0Wmtp6js_y=(T7 z+KoMgR4euj9^^dUf#L)+A><ut=m#n1vI!JVo|unOV6uFGW$@A2@sWpdBR4w?mvh<< z1$%*EFbQ&=Pbo^6ruVuaa_$E?)beg;&K0|ho9v+%GP0c=m^mnCZy<^OYroWeLshr4 z=AVuhFlA`OMEC~_zib?ZlO0RMzk`gI*J(YEEb^g;hvfdk$VhhrBGPMMW!z<}Lr7xf z7*iJ#Hn!sAeU*7+>C&rZK{>0<6WZCjnlFta7&oYHDj49hV=pP%3xyNIT&Z9qn3<U| zPSZ4>T76@Mr>vuG`-|nv=iMSR5+$L(vOrcVV~@3`goG*I{*)rfE4G8)xVnt1;s;`l z{%n6t+XpZUErIiEf2O?|sg;KBrUok<Zr}{OJwsjpLGA|ciG3!2yM}$bb`-XZc8^sG zmAK*Mps}~$XxfoDePzGz*thUO`W5VG@eU-z+e`nuI}p4VBNnOO6qf^GWwYATn_aM- z25riGx9e4q5}bWmc7A%-w?q0rUR@WX!0Y)<GUeC%M&?6km$Ej4?XK<a{;(jwzfj@B z9k7@G$98{P+uy%JA>P{z9=ncu|1ZR3lrhP7J^k(%872tGEvIf8wxvqj6RLsC;3<yj zOKru9**7PiIqx~N)tk)wNFTK5Y)OC92Uq!{^$87wRyw}#<A*0ZztHivx_Ak>nwgOc z{g@~CI<P)$t|7*S??~SdO0;lY6~l8)Vf?xPYgEYJHGS<b>I3P(<Bsqq)8G8|*3W$F z(j_D#bb61__k;C^8NonNfaFCcp*;S-eDza+K>IwtL;fGLV*AOxO+$`XeyjN}F7m%N z^IL=Af}{=2INIM!_`lqBU$8^}#=Q;-epfQxdQ87OV}>ATMkVj^cDw##R8Zrj6gwyi zv@ZT3B>(Y8{_ik<_XPicH_UJ2N`Y;KaN-fAV0%E}pn?CFW4|<M;rI-(9tsDaAq(^w z6<R1zPZ>e|BM}^=jk}D%av?FFsfe9k(60Hj+by4^L2!ulrAxQbAHoIsYtXm9Wq(Q& z!L+Hjxx-xdx2vyFsu!+xS#aIKrYOHO>dZ4mq7uWFB!5R}&xn;b@CUIFW;Uf3eK10t zPFlek8Vmnj$US)0EJi9^;A~XBMy%yntFnVb;nIv>;UQtCPQ`Q$wEg0^_}qr~LH<ve z<+(cHrcN<2^=BB<4+&27TON`O6g99WFU$?s^Zd4Pa;zuy8;|nL;E>1<x{rXgxTU&v zmuBG$vA{l@fB^9Zs~>(Zm_xlp2>*NNcqww6tdb**2&?wrW|J^@KXY~7+kQJG(!k)u zPs1ld|H~TwyzUqriIT1+-h`{UrG6Ju)kADI3S8w1ogvo%Cn9&UN_J8gRFV{n>@2ON z7obvX46(xts2Hh0Ks^q3^IRG(gS+FM!09w0<!#S1jUEX%m<lzo{Tv!2=^-F;1su7! z$9kj6pM3(VxY@u;>){Ma?gu`$xhl0Ei)<Nilf;y!5C(GSS#u0rDX)Vv^qhKnrL7Ks zy~Gk?pQFR4hsjDzpR!BeM6NwFoO;*M*qm(m&Vp4i0x?~H7*1zpcJL)Ad6!iMa!h#< zZ7##bF7^6rw;X4j6Sa&fke{GnOrTcB4=63&1YLx=g=C$I`Rt;v-+^*kelbF`kL%r4 z*N0Gdpk|vtq##`Mca<4rVsdO|ZrZO51|EkCOWG<_0uk30i@GSnS&bZ{AX@WuL0p^P zd-a8R0lPG~2lMgc4E+UffQix~u;&V#lF4aI79_R&!~2)v@9tVO=EdKl+lX*o+S>L} zR)52F&x!^e{=2LvgX&H_x#G(~rx#)L3WmaYO`}5ALG=Bhx>bh`64tJj3vb>C4pmYH zz4#(nJ;ns8zSv-1MT%0O{%TqaO+O$Y%D0*@0Ps~du>TsRt|HeHh*orv97+gpt6*W& zBW)?}WX<6EgPeLkA=)O5F^@v7PBuh85CrNum^u%=qFTl@_QIvX65N!lInvVY0wDaw z0@ADNr48#^%aU<den-G@)$vA24lb#IWI-ya;jwTwxC6gLh=d2ti3yzaK5Fj6A_dCV z<iX2}9Hepe+J&RpdJSbn1jDw3)!=_T#8gAY5GigxWg8C`C`ejqZM~F`ydCmLow2OS zY|(EJyCDY!W$YfkMrV-TpHv}Y(laa6YlxaO@AbajzX|XSKK@u-+tSU^=ETZinUimT z9e4}UbQb+lQ=)JIP>a03q=vCNEcm{`kmH%W!TVDkgwtEDWfT>+ko@{f{2|kCQQ)`b zlNsou7-|A{<E&s+-!{l-X2F1TZLZWgW8l$Doy?Nxg0gXo_9{ZdWqM<&7PQ07|Hhm9 zL5kCcjcWTa8uo~B6wI&ousJ5b?7G74<PDI>F8IWmh}_i*An!5(A_%7CSq7Vs-g)`& z`TKcw!E{QtBe8*Ms5_IfP#nIOwu^{VW!jx!uZF{nDD|_*-W^gbQ;>er*`2pOTdmHe zLn{+`4iZj<)lZ8I+*AjXlG9G>?3bx`TfaK5;oC(@VFS>miQ1SHVnl2r+qer8w&YYH zS(n5>j5|h}*=T9k4HD*gqof)#J9QKJfcWLFp~gg9WY-OsUQ_nvjWhd)3nlVQg&}oG zRwxtsI>e-SWHl}lKb<Av9FyHAuy4;Zt<Tdh*pVKy#CzXNZt=mT@`f?m?MxNQit;+Y z%Q)|-z?t7fC#amg(~X<<$u0hh#q#memEq1nz$>nkcr)b>quL}@Xy0a6zF^l|imzJY zR3zCk6`RmDkjmuiugqXQ*=by+>Dvw11#bM<U<5$sSfe+FANxtE!QTI{sQa=8`_+VA zrWt<~O)Rlq<_fTzO@|iFA0Wv(jDx(#<g>{sDiaUF7FMWfTl0=5%Y6iB;EH0J+Sx$U zUQkLg#Zs}{Mqz0a^xoMg%z|55ekA@vELgr(CQ-4-lr9x&+#(=iyZh4bNsgF|A^{N( zi{5$PKGI7v5w~hXM+i=gwx$}A=1g;Rx8I{#f;pLII&*seunL;Ek4oo&%uF$L_ceP} z+F><2Z!H$)E{|`hR|*ZN8BB{ce-|Kpi=7ylh&W#-D~wA-q+gw4lUPEaut)8~{g^62 zf`6PT(O2DPUWC0ufk3`~tqsAKc2-%ZUe|na^Ik}TViN9>2Txy$xS2mdRa9VG_Ek^2 zPKev7KNSNR10oWea(TO!QLBw7ZEymqD-a#htHvk7KYH|No))8Iz^;zRg{s*GIug3+ z$B0xG=u>R!;>@31(*R?t6NE0O53@`^x1Q>5EwZ=HJ?`R$j~w4Tvh@G#pfK*tPIYrv z>>y?btYAJ?LIk_JLPdl)zr^|E#f-6aZ`-=y0Mq%j1rzV>O%jiQ6{#a-lI#9%X;M5C z$Co;0NLApj_{7jM+nvcVJ?|swl2E62s%mb&-zpp`<_g&~ogr7fJUq<V=fIKXqUIUC zc=4Miv4xog&FPvwscLQR^N_$<R`gpt;oj?+NvNk&gpr|mQ+at8Zl;`G5su49Yp@Ku z(q)SNw>K8k#%8S}P;ZW;vBTPO@JMfyC{m&#ciIst>Luf$StB2lvie^-HkjbDgNQl{ zjcrBiGC(%hK0mJL)GWbaPT6$1L^;MK8iUTFV-#Rf;oPaXH_fcn8*<I<Ft;Xicx0^Z zgRg<If!o=_Pp1v(iq>Yf>uuUfuDHR)sx$%tp5H7YJObrQDBm_ntC+FkvI#WV+Y#Ul z&jANSBjE9YGvmlv3@(THb_hQ?GrldRSgzbKa|JvfEr%dKKBSRdvH=O$2?P!kfmoRZ z#%aF(3lVdXx@5ngAdXK0d2Vt2)LT~!8@TZ>(!JnDby}*7ov}FQz@^KVA18>%E-M`n zb}F>Luj#zO)nPdrfoVRv391{8&YVxu59cPQ<yR3M*##;?aJU>?`T&k7p&W)Tqqg{e zVa!DKeF7w?@V}xZqeKC>14te8y7IFNE+?)jaE5p9-_^ZHoD{FCfO-P~Wto>focyVf zGDb3M_E5?}Dh#!+yLzYSR}Upfbr=eeQL#~S^54aAJ@3|?=e?iLQNA^ST*r?>%6yy? zBf$QfK|}?BwxZGJ9KS;v#6p%Mjqgsp8u+{Cj4(`s6GvPco5#Er0}OAft`*xSk7ZwU zM3Pxtd^pGm=C`c#MT!sWR0njp;1i}mTG9%1I`Tf?b;rCKT^envK2bP*u1AoT%V40J zo#MHKKU;Bsx4UJJYe~;tzO+&|h{`&q``)0T0F{=*l%8GIRc+SmoQktoIH8mlDp`|` zw=;Itq#^3w!UT3{svx)~=uCML`DOET&J`p_<~xd{Ba1dy$7(OsvIWz%aT96GOajK= z$sVx32nriY3|Wz?`)%5Wx&D?+U2MxC)eW-@DBy&G(TRd{y#d4+H<arEiN5q*1by$- z_f$c4outD?>yu-=U(NNhyE~B-U6Ue-omEG5nY{*1w_#1D{Eeqp!dT(r8_#GZ@9jhk z13%dSgogd5CUqP*=j-?EIF12I$c8nHEQK~L_4>kDgg>brQ2O5gQNXj?%}K4t{rr+n zUcA*6u*TpsL9~bI(94CAiQL1Cv)4K#Rx*o-@kq2U;`j6rGWu>FN~q`|?3m#%JriGo zTK{xD#}NB(vP`NA)#GaaO`b^69hr%!^|%;ZPEPqJXVnLL4w|tK19mOgXg;f|0M2FO zdyePMxvA=er<C>3f9&sfWbd}TB(eiTikcwWbdpJ2;Cg}J*%9@iJ6wTDT9`SFeYpHU z>O7y|zp^-rawKF(H+@;F=puC&2q>iEj^hmFGn=(bLj4tc*l8q#ScVxg61+)2OiLt~ zE+IfJ0y489>0+zPOU<Ls94v@z))}4iP-o-l%pvYxNVN3~Fug1lA)hq1CMXwnG#)_> zF|yab3n&fNSvxk8ekfV4K*q>2v>S==J1(PeLTx2dMZ%RhuE3g;Hfq1URb)n^0Kg~Z ze2ywiK6pjuouutlSev#35+8UMw;?y7=0tgQ)$Z_xVqblS=tskTtNYWGr2>dUPLf-z z_uAqr7fQ{U=g#`YVwyQDHwf`8j>Vep8rRA*CEs5k>GBBb16WGRsLK?ja`jdeP`xo# z#&3R*<)?R`JZnD8a4U9l)uXOBFmW(uTZ{L{{E5$5f?${0K#q;mGYIfLHX2b(x?=EA z$UHo}V8_0C>nI2X6k)fd<uMMe0YIcNpkSMx*#+Udsh6MH?zpQwGpuKM-MX-?75+n( z<RP1Dr7v^5p)-?o=(DtmhHIb5^rAGeXNSqZ!IT)d5fBn>{{@cUe#K7*FbR@5xptWC zKXM@39i^Z}GDGiq8zi^e`$Ow~{OU3!a}w~7bnA++{WZVIyS%-Rnrt6tD*7!Zgn^If z6UTD*EY<Z?DGRSXMdtvUwP?hXtn$L@Q|IG??UvtwJRu3;+usvw<^KLh_(sY`p0=!n z0x!@KmDM6~P3gC0m=UDUm)m3j{Qbk&hN$f@@B0?ZGS}zUp+L+)sd>R;&%c1^-)W@b z$S`O8K5To2oP^=Ii~n)U6?eN~pGSaoC*^UK52Ww+?(liKcZi~5lwhYRvR`l}3p3Si z@gz?jF1@95>a>i-zW*55?cckN{NBEgKUrOW-|BB;e+m(Te7uXY>$gMvEfo7Q6rqqI z>$}VQqovD_{n`3z)MFrqQXC-^<@ukc8#6S6_utyzX0(5r7!*g4=~jVikoupd8zdqr zq#w?0wd?o8@a13MBH8EvJIr4m=>Io|*@s_jRs2s=&h;6^wQK|b8rX<)j5=;Y^(6~& zaFz%>;V7|M45^T&i0iSiAGek`J68vCL<r8j-B-*;?>PSi@Y^-D!Z?SX7cwRYa!44r z{;~*Yc+ZpW3Tz^@J3ut20%zslF&kzCzF_SQOoVHRa<PSP_7>H`whp+Ufk}nu=ZFv$ z#Z-uW_o_=WZp~kY{eVz=_)a#u?)S5l!k+*<U~>Wd8^RO<4KrC)13WqP4xeYw^bp0s zkh{fTVJDr0dy*52=knypMHPq?k2`O!S5Deu(YYO;YC=*GE<-=mm0y6f-fIA!@0F!M zL8=e#5Vh8h2Hg&jN8;A44Snc_hd`N<mC|#dd@JVQ0iEhV>bDRXO|DS+oC3YVH3aPf zDaO6Iz6xx{?HR8rL2W(STjj}N(ab8X8M?2Xpg`1M8CdB4M8R)794WpDdA84$__JmM z8IV9(Wl~$vD7W0ymNdWy1C1htgxtEP^OZRkOn`5gnws_{iEY?QEwkgHg8s2v9M=b! zKqf$J+Tk{PV8OBO;T|@vJ8p7-L<!hi)3jv_b>RstYgx{JEWRCvKWuC_RZ;By{t}w_ zcx;Rkez((A{L`A(*hndtA>17YB54jh@gTXbv6ne}6VzU1hGZ3wXI3AB!OAfkl(2_^ zn*hIi(|vjHWO5-3+X?BheObq?PkIVBUrha5iQ5InzrY9&@bj)y%9M06IY!Eea1s*Q zi%iWzt+*YMa%%JnJX|_G#g4|Yt3YJ!+E0XVcvw|M!Ub#{Y6<}sl5_JNDRoRe08b2? z;ylxxizS>(4gdsyu1%2`iiWb>J8+LlSi9s^N%T~S(e%@R+6i*tJYi?$*qv{gY%b^H zsw}NqeDhHe-|G2-(WGRf#qo}~aiaW1B;N@luK8>B$ukh1@bFJ$HI0MfWiFyF80PU- zr4Bq8^#Z9x+HwtAs)D)EqiIvX^|DXDObjHl+Z#rJcv_QY^>AAX8)-=nG$A9&em+58 zyt!)Fc0U{uage$-AU(ZBR=6p_pnG{CpmsZ%`_r74Jr?<B1i^QM9BB4F#ZdbLc!=<3 z0dOw6_T$@8#=b-H$el5%ZX{rMxVNdwkt$Usg2yCWwJ5c6#p6VN1)75*>A1D554}V| z`cCwQa;CPp4~_G|B)I7m^u$SpbmB`LBB$Z3K+K4dOk9YA6BNRY+S1gSvrd7HGA)3g zk^?b#c(E%bVE?8T>?9cC#@wt3=&Yw*x*MwtX^v=-lec?OL!n4`-CP1J164pa_!8}1 zZrg2L2fB4{5OBJ61Yo^_Gmj=*3FEUg+qX>G{*Qo7#%F9pN0&V$Q`KRwIY_-DGW8!s zC$iQAn^Qw{T(7JM{3%aJ-aSppf_H&2bOFZVCT_W0cpS3P)@?`)o3i&L2wYkhx$V!Y za)uv{!y{ct2A1SxXwIDkx!C`Qz4r`jYTdeqQR#}PfNTqPsRAk>3Q`q8I?|*BY$(Nm z0-*?@33fpRL_nl>k|08WP(@LSN(UiAlp-aD&`TiXn`@V|_dbtjf9L!6e(&}C*j~ye zD{I~BE_04K#+di~><M-%^UStG*d_!@n@6>K@r>OU&i^Vo{jaxhCP<&{(6OtYTYnx= zq?o8j;;j$i&AX6x&GBhy$jEFu2<F{A>(vFSt)<6>3>bc+WBejYgBBhx{q|kHT3tEe zmYPLF3H<&Drw@C((0#dl`qMOau1|cr8v0|7Sj73gpUL4_1eBkQ+fmhjPbv@W4>NeK zxN(rimH~ap@DsnNQRYSS6gPK(dVEI5By`LyYz$4hukaf0xdQvcTv7T}UfzQx(6XEc z^BePmbY-+P9gCScCGbrWp=J?Y2b{cN<ZvO&u7cF~CFwd98e%Z1_AGY;Kw3L-3^)Jt zHWX<)pKlyekZw{bAgmq|?J4sdy#VY@PfV0n5->DBS$FTc#lLgh?|W1p(r8@DG~<Cy zDd7X>Rw;)q&EC2cuIOLU<}qr;m}C&_kRmdU-vB%I#+r-XhR<i;^L@R8SEgBRtSpxf zpwe&q`~{mJmO=z=GW9RO&tK>M2CdZy)DL>ecTt<|rB<MTL<?DUW~?8WBIwq%)2{o= zEyz+2%Pk%*2^&-<>cf`crtdU2I}A{~k+~@cui&*+Y@|9H1H3uauH~dV%;#S1^kE~t z79yAA3VEGRFvw*155#3vB1qYJbn1q)?N_2<_hf%Dg$<v10hJk#E(k{3D>h4<dE0dg zfF-QJXrBh_v?s34(u8vj$1%qXcB|OzOitZFw(5dYp#>J#>VSxGm4}DN3viB-tLzOr zyS6RvgxZn9C|WI<Kd#-9=lihXKIWt3!>h%))or9lRR3Tj-N7PKrIlZ%*#9~Te+GLS z99HXcu|UvW!t6jgzZTy+ZYC@wpAYA>X&e(Y?MFxURVc0!$$9ue2`@DAS{Fz~29cB2 zzIed}I8L}Nq`MCpG=nDYv&gh);y{1DKL#jc^gbz_J!egn0)%jO{0FS>5>f;i$-36# zTRvNq49|pYoLPskf~&9Vk8WY&D$a3oh!na}RIL;X4K(rt#CK`cuHB&H)k4`BC80s) zd?2FS3Fg9nYz{9c_&vG3MBe|AG>|wT+3FoQLH{Ev|NAoAb&5m&7#Lb#S3HK-o)C2| z>MFMk00kjZA+&Y&PVep-qgCI%K)|zH@`=k$+6=qgPN-Kj>Ev?ZGNT7|%?Wv2J)-BJ z5}-i^YSn8f?F}mzW0$;Qj@WZa0u&s+WYi;dP;&@Yl{X<+t+2pO%deYjvjJX;$po?( z8eVq2p<6Gh18dl*5~(2gnhQ-$O1WaJ7L7dzhft*ykn89c{Y!+e^9eJ4si%EAOp<EB zWbbmrx^GOU_N>SDA1B<HRe?D;ucC_Jcg9=RuFzR4vi^JjKyrh#HbdpdjGsT@H+QVR zowzV4!Dlx$tIh$}sR(#rr5(&TK{u#Y`my5-yWUPmNtoJNV{zv!q1a+ZuzK~*AVo@C zT`QY%BKjO!qR8J_517PXUQ8wWPu)#dh^|^KQaDx)utz6I@DyY7?v7U9r6FxYA#ed1 zS;(rVGN>`4x^W^@Nya`Q2H@!1`3{Yniywv@US|fOU3XR=Se6ft?@fg?+6GD)CpdY& zfCWMZdc%0dyq)*XSOgqJ*s=3st$t&b*%A%QN%|XrPwa)w1Dd2c&bk9B_X$TOo#VlM zXTjl#hP5nH5x9@$vW?Bb+6N(N17=3_oxZGio&xdZ^x^es?Vn)j(hVbk3OcJrG3D6j zD8-A_hVS+E?rSuBbjIfS?_(D{vKa_0r1!V`eo74qQeVd-UR&OpE0s>t1sKTo9@vbI zF`+YYUHTHJTSe=O&Qt(@4XIBHZ@&f<MJuR3{{Y<o5N=Z!yI=mM99%W2Z)L{rdr<W> zgoMQ3;k^*fRiUI$E@Bh@z`yE!TgISd8to*+`E{H`%r<aPFdBa)I#JO+XFaOwzi_Rs zlN;2M*M?<DBYWn%%d`=ue4}jDOt<9M>fHDrrvd|yg2}moK;PJH|BBXZQ!(Y~4|1rl znk9Z?Tj?u(vnz>Air+Be`6*nN)WmzMIj!Ozh@9Dygywng7`II4s!6`?aTi-z0ePFU zq7y;230m!npqrC|X>ckEvwx-B0vyp|L+I)~NRSROvp2=y;cMU)_j(&{5ozD)VEhM_ zR{iR4;N2u(!DX2fENzXf`J9nJnTbhLCE>G*wl=TFJl4UQtve9>P)b1h(6=meoK2nk z_YaS)iicFWFZl)Ey!-wt(`^Z-bN9FfvDJP___CiM8&YPgtm4uYWvUoIuHq`_o|rD# zvTsAx;v>)j)Jd92o#Pk?p$|RRD9*5AFDU#S!<(j9Jl!!Uuln3#L2Ae!$eq0>J<iD? z!8-6)tpU+`5->c`6^`TG(fnc(qF_7&$dJ9I*K!MP776)O>8QSc|K0|YjXl{nS}{@I z)N`9>#m+u{_-Rqi@4|(YQ7br9nsz{e=#f)dn9lv<7<;D7@z3=9o4s<aM02Am?YQZV z1Aem~D}OEOuE36YbVLrR|Et`5vELHk&Y~Ao+Q%()5jQ%Ou7HZyYhmI%#oK_{^>oAG z*~$o|QyS{4U7pfjr4jPXvr$Oe56fG8b92?ujN5j)(I89rUjOYBm!I$QLF$Fvj(dvy z7pHcYrPfoc%@%_r`>mrzPV;nH?Cav_tj}X(>$Qv=kPrgXv;!-n;|hxjJ$ss>^x^J{ z<|oFeG4$X8YC>D?Cn>glI4uexs)xS|A0oB9=71SxttyYV^ziQh1OQ+wymAj8R<`~F zPL*<I*Ol<9fZk8ejI8ZDl#q71AoV07ss|fBMgIH$2|jqVaW$W8NzHdwel_~oqm8tG zB_F#sAd0oex+ee84el>HW3V_v?Q7t=Gyd~+{`YVHKjzAHB16nnzn{L}lSS&RVC(_! z7u!vGwUq$R+9H+DYD)E*nYb=xd%Sq)E8*sY+KgF!p_1h8UW@Xx+4>b?iofo9M}0wR zX4v;=^UqGuf|u1$pnnp=rc)C8uxj;jrPW^N?9XhdZW-{}$ECsMFY^0`zP3YU?b_J; zpdCL$sov&S>)pI%^}59N&@#KH$SQV2i*o}FC-wA?I-64k3!azia9k^RVY^f1*Xcd* zJ~dNq=2zFxU58g`M(y=T+5D5@%HQ7+gfk|xfNiz&Zxs2jAhyHs)O-H-%D-c;|2rCg z0ncI9{og(}9x0Fc>2UoNF}`CGjL0?s+xY?YLkmdib0O75bQB<8j~q3cfkHMp?R4bT z{<n9xxYI*J*~@@Lp9i9%&O)a)L3%O?F&shN7)|#A@_xqId9xS({ddLk;N*emx%g4< zQ6L^c&I9?h&1FbJ3p_p@dk>8)4kJOz3A@NR0VFX%a#zHLEcfiolvz53QbBon@pjSG z?!yWO4n8Ps*YJnfiz3Qig@C7AUSO82`qu`behpq;NZGZYx()_MZhi&~erfAMh!L{R zL73`J12-hAIZ#kpfis5@iP<in#4{ExWts8)iCKq=*N<m7yNaF_ND1=0vSkq=7X&ie zm<>UqMIOWJnNsZ#IrKnZ6IVcJ@((*SoLyO7{FRmnk-!o(mf6y|`w1^1mGbWMeLr44 zZ76&$$wg58C5(pYW_B<b=7b7vlnP@c)gn!2Bjut$l&uigDg-#ngA})Np#0C?H+8nb z&8(1*UjnZEoB&vmrvAI~FXOa3?tAChlvtvFE9CDLf@05J?K`#EFRRPW?;{7>%y=&u zQ6s)^g?RI++!Y3`h5@`MYmkm1rCrqu;Is9>fBuPUDB|47K#mH&-9`fzA?qpxduqE+ zXhJD84^&e!eww7-2e<w_I?hPLCY)1W@|VN*&x|a%eXZk#g(rG4F9QK9$T<DQt#a|5 z!LxC4aT6}b?ss)ydR-N`Z+&r|LK1XW==oB{7|ZSDi*^SvceXP{Ywf(1z^95Evxhsn za%K_9-+;aUWLUJcO(&8WxHKne)P1@kMfxkiHs1>jP4gq?YuJnXZKb|M->8Iq2$4M5 z`vEoR6$qG`y~e-9_GA5pLwzD!E5diEIQN!$d(T^_3F#yU^nqPey|t+d+V>#zHX;K{ z5#`yU)k(s*m~Oar@9<&Aeny-N{w1JERuBdjIkmk5Cd_y83k@Oy8jq^>5WfN0G9Q#a zqGb{D=iw_e%o$s^$+D>rg8<>X`E`QCmOzkj?tZUt#rfjKn8Xq+F_c>2{@383_XBZ1 zVdh%k;@ma`00sk)PhvN$FjvH7ooWeKSsr|GXR0=Tu}W|;Ca6-5*V_1UFf3orqK@eY zkn+qVw9lK8l}G%t=urhKz}<MK5?|@Z_#~OH70Xk|3~NA)NrLXLwlCxeBXS!nBl=ac z7Q(EY@gKWQj4)Pz@cg*m7D<~>e*|(DHcOJ;Tz=)6;y1287oAS<%-Yj@zH1GCcKq)O z3KC0FXA?5;cj;%qB|IWq<MKKOvs$xVRu`LPpUJ>d6gG|t8I-teIvbfQ33QJlCYZHY zr0WVP9#eK~-le!P*UzzFE|_b_wa9EpUkETyJtDt4pNOoDtD}RJV7$HsqVXu1k7Kv2 z5-dTSUzXyUc`*k1Qp_q*N9a;9rX16wDA7v=l2*pKH3q-PS{1wnO#Bj{^yiyx9Q$v; zb|mviGx9yJVDn(XJ<%Bj6Z~C_!6{*2Wff%;&CQJbx?TEhB3no?`kHX7xjw75ydDda z>-;5bs~pEeM=G}vI^Gy~uJe_!wCBLuxvOxiy_U`KFJPdT;fHL8GHmJ^+inn2l6W{2 zeH>UT%ul9Gn;VYo-u)W!2_`CLfl9Xmz$Vv%6;SQUaG#_oMkCsym&WJr5IKy`g&nV; z55tb}WqRmid>6pk$AvfS8i0lhBvRimzGTgEsFS!-*A(FntmUn1_Mukt_WzNEL9BFA z6bjpQ&$E7uwEr!5QYUfyNYyGx;LNGkm=#FD%c46;!4@Vi9F-P2F12N%r-areRg4xr zL1_t;#f>Dw9%>UzV2+r}TOkVDvW{*`+v{NxDrKTZMNgFT9q+Fe!qs%H(2({!z2Go> z1d`S+VcavzZE;^aE=}%@IDX;P<&W-I6<`1&{7{RR7qk>Lo=aKE#1z<drKP8N$`UGg zg(Q)Z1=Z#lU1Z*_g9|qSrY#<~%df7AYgZe1AXo0}>b?l2122J&8>$>8;@tfp@>Mi= zB-N~GwNd^hUY|F)0YRa&K}Hvr(@zX;t~Y+C6w~?)q2Z{!60M1*X=ruxMQLfi7M8Sj zA{=U+Kb1QLT%W<EPigIn-jfuykZ#Gk69P+H{W$h!VL5c4`+kr00v0d%NB66Qh!Ar# z!Dq=|*&SqzTf8kaw(vd~0kmEj;A<@6w)mx};cQZhJ_n;O+i}|(4$k8sOOv^#aRiz_ z-Cp~7V)a^Et$;sQOPcs2x!PCHe;&F9dkRfh#JGf5<d&E(o6fn^79tX4Y~(y53|-6C ze2I?W8=?V4Q#Jy4*x_-x1d#sW@17HKD<WFNgS(Kb@kG#SHX!NR&NjuLvv?G8rXn{m z&Zl!<DVSAl$-6-+7NwZO`97c!nbBoDKee@O%-^kS4gKubTt=Ec50Is2(ZU;c?o7<H zZb{MVy@D$wpvs7iK1plh{GZ}hvzd9yHnoayFC@P-ZpoQ5-oI&}{ee~f(sm`!I+4!l ztTc!1FqCbtfyn!nXaq{Rd-J`Roi)p6UtNBfn~WAJPoZ&{EmuX_89xY5ggzUtQ}#`1 z2iB7)pbhU2LDf90z-RtBR?GXL|00F`kE-6^j+(<J)ix5ij;=m-eDMubud$@JkO5I@ zwoKFzd^rQlgvP87;e^KG9tA?b5?dAAWqynD`o(j@p8AnpeW4|m?^%5aKApP)fVii( ztiih+M~9NRXv;~_552d|mYZK?iyM^d>dw0hJqci+>rfuAaE@3+`j!YDyp8~~<|BQj z#)OroTsE$Ur_*Y4bMLMg?|**%E9Tp~1NoJzQul)yD<9;VO1^GLpR{Z~m1f1(ekut^ z+=>A0y_<Q#ylfIT*!5Z4Is{u>F*)~6EnnL0xaUsvp=5d(F}3-XQ!ejcOac&z&dKj5 zrGGjShSuu2twmL&*5@uwUT%99!F6-8r~*LUdAmp;d1m6HR&><}LgUsrFhANExcoAn z&7xsW=U(xdWxyWrq#k2YghaL|lL*GHdeWewy}nJ!Gs~q40t5@#Q7=U{^bN8CSEhGT ziX<vpE-fG~WtWP^oTS*J!Sa%n6cRN)BzbCL0Kx`usrN-iAs@i7YsYKzK=YO{Uk5E? z-esin>6uW#Exs*i4ripJGl5Fw`2j_qvL=ZwROLrIw6Yfo<x{%x$2>B~oa$tbb-x^} z?(+Or6NZM}y0XdcDHkPpmz$tY(iJdIOC)=xz)II}X1Z;^nPA3t|K7V^Dd5>f`cYvl z2a@e7(A=&SvlhidwGfA%FdzEU%O@xipo4&4%75x|va5na{`xV99Tc<i_4)?irD`i% zt3i(5Za`cQ2&aL%3qOW-n$W?55W@r2!cNcy-wE5xYzE&Ej=ab;g|}Kn=i7VeLNV~E z5(vdIy;a2&ER>fFB3U;%`3j(#63@tLr&=4}rMaRu2LRd1)$N8t(rJ{z#8~-U81RRG zWhA-^-N+)lT(DgtyPR1BR^=fTf-XdRHGNWM@xkw-b?qD=I*{otf1&LFBQ&n?B?8n5 zrMoz2RxIFZmR@N)8z#B63bOe!9u{Ju301lN!$*!o<mm-#fkZrWczuVfb41g#LvLn_ zp^u~MY-KAI4%hcx4=OJ<Y)`(rsmJ{rv~QeASUZ~m)PFpS*KiSbTp9$9Ugi$DDQE$+ z8EnU)4xR%&;hM9BarP7^tfd{4b>^{dHr{Hve|P}^Bk|m<fGS8`It%%NLoSqG*U`%e z71;XyUx(5oKA-~Cao+r}_h-e+=U`>^*~Q+&;Ii``)KaQZoP52(NUpYZE%3>&#l%%X zfery+i;bAS>H*%HYfHV81=eM;B1tt@-mULTjKNpJ-ay)UdIkU74WK%oh1o=d&T7R# ze6)rj(_j@_Ty+<Fg{U()rMV$B#TY4(@-rlW#x>UV+)~ed1KG#Gr+FH=3(*}Q)zKxy z1~%Mp3)k{el04n{#1_6nT`XKmW<))=-TVy6GR^OGBBTHrr@akUT*#r2x_mZJ>>V5* zb=c8du}q$(=dPCWOc4Db#C;IU@HAIsDUAnFB7kFAQq75u*cD+{aU=H3sFYtw2z$+z zrTJPycfK2wV0vmb9IB^6vQ0C4o*laQ#lw$mSa<Rwp;AH*@m{lCa(+`K!ipzCIavm1 zy3{YFNc&K<>tXv~3B<ahqp%~0X!q!GrEuJcCC^niuymD4(z=*3Y5A0cBU!xdWPZO| zeIl?Qc&{1ng!eYUup@bno{PDTK(@a_2kO7cnEhBjQ!H)kta(x&%U0b_;>CYvXU+sJ zcwFj4m-4Ufw)=Xn|Bwxf93XXOPee2>N|x-$76kPE3BYv^kJ5whEvq~dsakW~gR%B6 zrcY-9Ng54YV|{X0Ja|DNj%j`jW^uX-t>_A=SeB%g`w7|x?O*DR57#0Ish_in^Pd+j z1_ME<_Z^k2eR=qnV+Pm!B8feXsK{3<^Bq^x+nN%Ti&^N9N;OFMvMJWM9cTBHT7Fe{ zttzEZwY`El+`&56wE2g%NSmDGLb7qPW^xX@3jQ2_?5!I_<98a{^oBH8^biJt$KT;d z^V>2@v-g2tDh4Df8VoPuhFceY^b|F6B>HqYy72TGRRX1(%Hi^c)g6)$@$_OPWpBr6 zhRPY4fCxe90ZlbSyWb4nnDOS>38X3kqm@sEzt-v=ZC!;{&N${zH(Cgc&CI}z{cLRj zh8oAdEY`VZ8@HFt=a<#g5bXML!z%H0N3P>sFrOe+xMDCzjR<VHH?O0Yb=n=Zv%X?a zX+#XFI(OSxclmTM;q{9{`G)yT4WBpUq+QxR7N--@1T-Fm0|H)sL?j-aStMEHvY2{b zOO_JX&h}9Zn>H*w;62N;xp32GyR_NWgtp<r8RfIe;MMi}EGdX&Bd@xY+49}$r-+04 zE<GOb`DjC5{52m2&bLi8VP**^&mYe;)xY`k;6EvY#IU$5kOco61dUclEQVkMFzFCe z`qMe*nln@huQ@YIm4E)ZKfeL_+bAN|C7HBt_-}qW(*nphrqrh1pFi0jBk<c|(NP3! zqV?gEzvRn*X3qcqx1&XHx<9uL9RIJJ;Yd7#hQ&`Ex&O5;|M=z<>!hW<{5}0Ug|#p{ zN4Kwjx$oNY!u3WVFCD<dR_+N>=j7SP#5|gr*Wgqze8v4-Gx+gMF{{Z7>4xX|OMW#! zQMZQOSm*xMuD=fx_z&D%-F2j@&(*P4lGD#HX5$^h;82t{n$ow|qQ9{Gso2g}fi1^$ zXLzAe(&tiIzKH6z4-FO{Vg&gf9h)DN`Qr_$i^HI3)?fQ~QF8DB(QBAjF1T~H_ycoh z$JejUG~5-?2EK_*#5XaBMKtU^c<l;%iss*;<U6bH+|x)F7~PY9xm(pVW9_fulKNGi z!3VhUR+~R|tv^QR-v|BIJ&haUOLg%r;r|s$@*+Z#{2bKshbP}(aK!)lC7K9H^8ado zLU_K5YfgFpH>=@)hxspa<^Q7%<o_YVJh&bc@{eYl>Zf=R6S&Q?c2}V`iclSYb@KvG zngW1H(tR4i6{Q<#TMe9&S5Qk%Kdaapxin8NzAGSR4Hfo0+fw((c2D!{KxI(@Ig%IX z7<)>GU&Q~g{d)U>&#h{|v>9In+w=IYPcCq{Z=QFy1;GrZplO3{Od7Qm1d&Y$L(UIy z)i*?_iZ_wyZ^1=sBhmGg#aVDQ#lrtU_=WfH-+#r)x3%PD=e&Y7<JsdTBQor8HmfhK zzJr@gDgI-(`kyDyC7xhVJAQ`v(G4Lc0(#Yf(4LfiCeD4nqU<^H$Vt;eV1Cs64<|Tj z5Fjg*0`N=iEi0z=mMQkzI9>>ZW}1jN_jZM-bQ3~1B~&tUpauECt4k8*h{aWr8>R2^ zhiB)Ny?$K7S{MiETE9?E_X|@JlQPKAK63;Nvmj)DnC5(|!=(Bx5M_TX11-q~q14WT z^9|^{zmCV+VLu#Tt-G0Wo_zz@!5&_9@}qI{#{aw{7Tm4|n;#mO6TMO%FL{HAR7tlx zUxEtq`KBO1I7ur$R%`()i&xq%l@{=w)ax$w$V9}$77%$xj<tfh-B6FQ4KRcb7Ew#J z2Rstn-(<Y01<btwBy38rfJS09*<XwL&dVVRY$$n=@rf$Y305KpP<g%%!jW_Im>T}4 z=2gG#tKb9ble+++JGHC;q|KG{)(+TihZrmPN$*QQZ_A<=A>ht5&oX{3vNT`CqFG#$ z@~eh+=Z@aJm>8i0XPXlcCw<iQS|J(i5}^7xfZIEPED;YJM=YrMElS-TAlFUk$TofN zAoL65zGZm$_futW*KKl_A1r;zwUo7&WPnP&?Be5<W-l(t`wGAc=QH$U&VvN0v()1X zI0A+e?nb?MZ7DJOuxAW9$||Rw_UABmbRZE=+!<6|LDvGM01#JOwxDo7rioWr_3-%T zx!qDG<%QUVZ?}*98{lkzum&V+w$*RJj{hR?XXG3XI08vk!KL#9g5J2|CIbdo-DUho zOLIu9fhwAGw4l!F-TOi%6@t_S(^MnOIM{bQ3gX`%a_7#;RZhI$H8z}NPt!4t3)-bM zp`qTxGC4G}XTvV_O*(WkndRS<;NgAwaQqKf;)hTEpQPd?=08c$J;q~|=Rx=SP_ai( zw&D`f3`1^1=<#e<I*BTy_9pl_(jPw1FJzFov6C+XAyoRmh)RUsv++H(n{BRs6V4qM zy4~6GVSOBtg;?ohcBm3;jO(sJ;Y3x9u?%Fp9h)<cJLiYiKws1!Tx=u0CMG6d*3T-b zd;x>*40ohnGnt|%Q~EYD`G)DX_FV!)_UL)Qi|1dROIxCD=63de`0!z$MPi#r5gAr` z3y?#<UomT5w-KmEg%PH5F3$mBs{C@j|F{NQaJj%&FE(GxA;G^jQu$k05L_wLeA+i* z22lee;_69D3$bAxv1^EiZ$UcYWG9b5s%S@u>+U3hEM6tw9GkTxp(P=P`c(EjPHxA8 zuPG~Yit&dX?p5(Kp@3$vW{qc!ecKfE#rTO^>n0~%7=e8In(sP<iFki$$=t{(yr8JO zkci(3=4Fkz!oqeUCBRQ@*VlN;^5q6;{!X)7GnZ=Cu>nXm<GU63v-atGNB6x?kDLa} zvIoWXFlEcdO-maaC70b3(hw$_ffoogQ;OT$XL-SYVNR?aia7pi&B-Y#HT<8Au@dqM z3X8A#?_Gp0DAV-p08NCrkK~9J(6=Y<@wbe4*C{I1BQ}I{Z`QcHGz-YW4uHwrt5^8a zs(#&t^Y$Z(8gtYrU~FYW7t+yPsT9A^aio27I5h0>ax<EM77~lAIFk)ReL82dY&1ta z>c2s%TsBG+a5p8l<9_j?V7(Ev)Zgy#;%%?YWjk1$#FkUreE0G-933wiSPoj-k(_6K zIr{WXdHJuQ5S}keSL?GaqF7NsHHrH*{tdPhZkC@>npB2Py=~ECXi0MaSEqxar#4<k zWsyc1v?|t$7YG^2(12#!&iAm#xd4h+9C1SIj!<!kVffg-8GttEnU$2w@#y8#rU~kO z&`o0?H>+rzbKBK(aL2btov)*k0l>ai(VLwlP_9c4TC@JnG~&QvZ&zCet`mz3offwq zIi5eCTos41OsB)KMT9;=3tBFE!(d6U&g5-GoP);A{qol|#Ue2wR*N6Z#;x4UZCr8} zbDP0pbY$S@NYc&aT|v-LeIc@S_XWO@&(K)K|4UrhH>5#yQ!b}(IqbL4ZOGH!-rBxx zL=C{4l~j;gB=3w$ve{HfdMjV|4b=NnVvUO3mm)V{m$NkZ<0`&_u3+ABa3`kSMAh!2 z=Cu|r;l)|Ac?7}_#J?8Z`e3&d-$Dsg6PW7+&ZBytXrfQOIx@n`7))Aryv5G<d<36F zwiC~yNz%8Lr!dUx+Q;R0om#UqigHlyR!@*218LiHeX0E_7YV%z@kMyVc9YGK>p!2M zyPB!1<Hk{IR>`9(GX}b%KPnySjm~|hAXLG*Y2=azh`lpPEnmx{Nj0E^T~0NWm)h5K z;45V$gXLLU9K!iaXg*BX#Z0B_?Leb*xbea4*5FX=3xafleqzhrYYE1kQB0-e0I*Ap z!@uj2pQUje>qUw(WZ}>)TRm-p2a^Prc$JcI8k@mD_C+A>^=e)cMuWTIgnM)Jv@;iD zEhgptj0PJ9gHdHdL+*BdHskF%!_b-_!?r%rcK&=^;K%JrHY(zd6SrOYA8%KMAY(9Q zrIa&w#M!^MKdCIFCwN6(M?+3SKS8MjVw?`wnYccCjTNLg5{S21E&d^1QFaG=Qm`-r zY474b$h}Kv;-5T}D<qRZEqu?>Acm0Jf^qP$<XknfFFyqTWn8DZzIS~6x3_s2tS&UC zU`@yL4g9=N&5aeU*1!bmo2ot0o|;ccgk`8HI=<iT@c<Ft(AKZc!dp40ExfK@nd1{n z@O)zM?BHvo#GU>1Gg;Y8OpTpJ@mkmbdqsDd`oOWs?3mbEejVH^A>L|St`615Z23_W zbaO)@r+4T~jo-IJ>y8@xf1BeX@u@S^A6)Ybn<$;gTD4bcBcXd=&90FAJgJC#C*&+g zj&Q`OXcS*=UMEB=^6O2})VL`RtwQ3Sq)i<OT=8ewkxn<<D+hvGxxk(8!u4erOwax2 zP-R2i=HP2#0VPf4cCfGR4vj4ShGC@Q_gVe|YR|wQc$VktvI=8+dpD#Y+tJ>>U|vIt z!<?HLX<1Q`=Klu6F=3>@#Af6Z4~GV?-7Qkv67Bh<t&_(o;VjZNB9byEHvVQ?Q{_Tv zl@beS&9g?KCVKY#V$Tk9J5&b?JXtQ;n!nN*ab7~sCD_yEJI0LV)`l{=l6Y~558=D( zPO*lq-f82CeLfdi$N1!7YSjlqh=kdGWlEs;hQ=bl<+s}oah>+X=EgdoQtd3w&0{a` z;BHhnd*YRJXws_PO4%$hX=y3m*?V~Rv3*^WP{?HN=?HWWnDvKF9Q8W~jsaY;VeLTy zb^X@@|1eS`e5(?+tAJAtQ)K|tG>7PD#I|LmQNR|38QmHeU}qWa(!-dUh^bnxH44KE z(3E0xazDPov9)&5eo&#q`|H3#=m=i`Hoaf$YpY;O%{ZAV)rj;rLR=|n1&uPQ{MP4l zam1-&5##OiUpykJgo39saw{;aZRTV6EhoG)*2NvR6T@O^&Whb~$nxxpGSs~#Ad9pT zvEPC6#XC*f2P`@$4?a4v+&bef8`pb5;2nBA2GOEhKwobD;}b!FIHwh8=QVgTkES-F zItK;({3;S_8)e~EB8dfI!vnP>|Ko7_A(ES=?$vlGB2?**jyhv~`>wwinQK15zi9Kz z2ytWF>#tC`pDeanKw3F<srBF4+kceo2aSC|6BAR9Do#b5r<a3P^YH?2#QI$X(%~L~ z%X<Lyb_L4Itm&dl(7mZ^;G?+|xj5?k1sdJ^$HFYLs9FJW_BQ^_ElFW;x{y8f))b+` zCZ3HGUa#I73Z1SL7v@q|iN(8;vJ+L*oo?M|e0IGLoEytGpWC2rXQ}qr=2QQMBP7jp zc($6iy`hGBawGOTRnkFr)VE7ue)I!o$V;{BD@6qCM~Z-oEgu1slbcJ6GBs>54NO^p zR|@j#>3S+7`pJr(eTGzN<W0j8D*HT=FK~&ESa2Xmd8Fl}m@8uTfbcG8gg|C)Q>LNd zZ7aSEqr`Ew)OkVA)%I;w)P|*lVM{#?kDez@EmzR7O$dWW4O_{%`8Z;gnc?|FZRZOY zwvddJD(GFtw2>rRT}c>9%#87i=$$X7Klgk=h%jECaI&j@8PXm#hj&4j+AbOOj(vlW zWgdeo9~&$(biFM=QF@G%S~`Jxajk|&RThd(i8RRG<U1-jZ!G_~qBeb#EnIEIM@_C8 zX$IJq$Z3(4;HZ9*yWaP70DdU0MICLlX6;O76ZNE@hsUeYIl)nl#rAWLAZ~3}8!HXr zd|s!yTP4DNxnH;<*04`1OGQd#2&tX~u)T*wVI+nIXZGfX=i@tj5_%q9*5xMOA5;C{ z{K!sqrvz(vkb2)*!4lb)&TEO0gW)L^!<+Q%p30PDIE+&9(s+I9^bS~&%{Hq&E_Yhf zaxTZ`qoU(KWiQb;cG?W>5@Bb=t7Fmku?*LTeAa}wos^QiDu={n0OS<8-!iKYcJV(* z-#Nm0CPlq^Gq&@b-mv25X9iMv?qA2aj_lTC3&!&th@%p599L~nZ3Cv<gHP4{w>#N* zZ_mCnpp%e|N_f=SJ{-rltBvPuFF#7GBhpBcdI`t5(Xj`M+Jc+jMC)~{l=H1G#E;7n zmYQ9*GUh_?b(X5P*KK)nJzD>Ip57KuH^w8o0=*2+!<XwSvX6r$Xw$Z*(YTc|skcQV zk(wR?J3cR<bCx~llza(l938>7hdbi!chl6;vn3|E__jt7_ua@|DAadoJ+M|UMrQVv za%ePkg&MpPz)f(-${hoeSNXeXi}TDG?V^gF9&6U(j03I0!lwDOf01Rc?rxQ|VUuv@ zMB0|zyw`Ha@Lv=A#s|gXXXYc-8o?N`&S=%F+L&7nWlY5^fMo3aPaB@Yssy%<jwV`| zzSsjytJNAdK~D_Eh!e%J1Ao>{RwSVg?Zy_Q8YvNuk+^B7O4Xg1c_fiAgQ;;D@`VH9 z+mLg_ZoHO-i0oRq?V*aDX5aG-r&3!#m?fT17+mn&BvtVM46ylg8QNP!14SLqbJ^|V zcO0o63lQpEf8M}#)YsOOY(*~6Z)~F`r7m7pfqPdAZeK)C5tgI=bI$Q6j2rxXBkTyD z%=UBq$piZfZuIxPq0W01=c+-!wMonW5N-WhO@Kd+?XBJw^gLVl@4Yd9{=rA|)$3N* z&2Ls;{4bC__fttnujp!O{4I<6^ND`{+YN(XJEb{=X;<h3^x2&fHHeEU&>^%@^sIt~ zf4z%@P4MKIC1<z%kI(+)%5}0}*GZ}uKKlGCe?&cRtYQC^8%BwIiSUof=k5&>EBn{1 zd1CH^fasovK^klc(-+YHFk94j5!mIqcRg?Jx_Hz?HgsZz6ta(hJ@?J^f>(5;j^kaF z{E@Vyjtd#*6Vj%q{^?Rl_g90V@<_j!^2*CWDb^bv*1=!YBN%N<=dIpfyo{*ocd^)c zrwMicFB4%GnWN@(AfINl2Os-qr_6-qi^SoCWrVqom4H`R*)H~vPsA2n#Ot_$?dGb^ zhPQ|tM3C^g4Qzb>GA#S)WJ<E<PUGkF0j|rER%JcX_PseCX26I6y7fQFjDLG4XRoZ{ z<9sIXw5gc$?3L4dAbztIgtsWuDe3T!A9T0<inJ8n{~Wv-!(*cQk*7-#fKLvWpWXIP z?;nas?DE^#irMR)H0;_2O|=z}nHra~Ywyoz@*lH5VPo?yHV)3!o~IsSphHd<$1m)U zs0#Xvf0<hUV`PH23vu6Mb1(D}Tno1}@&xi4(%W~p|LG&E*csMektEvc+(C8wF80yA zLFz|%AjFS%zFGe71(iSU?XP#9zaF0FEZ1?>e;GIZ^i$1>t3kh7-~{F+hR4#~4y)(D zMbw}2!Z+zj-MaeEGKWIhgCNIMV}s>E+x8Ne_fw`xN7e2dWdHMGXQZk>Tcz%Mg-ty$ zPh$#t2B=8GbbPT{G~l0B1zRv?Gq)IP_9$e^@!++s^!_v=OjMUa?p4?&`tU#A>#;qL z0;UkL;tafqG#^1kXFK8rtI<<gV&l&%qQ677rF%Ev%jvjH5-MTHhEd~zT5PIk_2sN; z1sG^W2JU>t?&jc6dhPx>4MsGAq0;{DCG}GIiY9l{n~OeGdn3NFS9NwwPYA2#9>w(A zCreG<_VGbzTvG-xoQoC%QUBOqY9v(s=*FFfJ(+4mVe<@u`ERdA6M3k&3Tvn{vpx@z zThB+HF0c2F!aq&3;Bp~uvB5VNl}=9GWeyIHQb&vWbt1%=n2e@LBn$~se-9J9`MOl+ z&lzTatS}t`9o4&u%g50QcE1kh%oFwVvgs!d|8++GY?l7|v*tI*7IFSu%75bmoK(yX zH)PYA_Gg*)KVCRUoqz2??xfqncmFfy^T!jwj?;&n8*Zl~*CP-9KLPu{|4iioy!joR zqBs6=&i;P&8+!0gEA|^G{TCPJp2>&(;l?k|M*evf{q|Z%q~R^kJe42)``!Kgd;Whq z(8ivih(y_%n-4`wh>PEebfK48=-6!z+%<X0ps}^9gZc@dD6zf|T6~4C&9m*f#D|f# zurP5n;FBggTSXR;B7|cEkw`ih)qQopLzso}*>h5x6|GH8bE8RJ4E@<DQ7LiB{FMIH zta3c5TUY8{UV3SVMX8UC^#@(uwK#l6x3h)3jmha?A<}I0<%`(+(NBe~Q}tf;q&wlu zOyU$|RMK<hg)C&83NFWF*eSM?FqZO*N29(=##HvpcsMzWBxW?08?_B-T^A}nij9+b zanvA&I5n=cJ?65;^Ngv!o}R7gqYX*&R8*Y8UHRM?Mdj2H=2XKT7iUkWhQ@TO9}QO2 z)(7SHp2u|a!Y>XtZfW%lua%)s86S(Df4Vs)L(a+7%hyO(ZV=@pzOyo1GcLEo<gub{ z>D!LSHrh_Nlq3&GJ{YS@x5|`9^_gqyzQTp4xaX|b6e%jYpY{B;l&kQNjDfv&Zt?w^ z3o#kFF0t7t@sGyZINYbC!s-sx9)l~EBerMATDfk*&fYoHW*Ob+jA@HwH!U57wPXy2 zP?-2W8Ee!UQ&g9EnY`tZMaG%9N&`jflyKWP=eAKNhS$dQE#;bJ+)jAg9><56PrYdE zk9U>Mgo%G7dOa=|DV0Z|at(;ZB1I&c_9Xu4Wm|2TkBlt*37d78w8QVnk6niQ8Wnu2 zMD`O=Rz+xSlcjrPtsXDI;<dv&GDnFVMISp_J<&!EM)*`lPZs~3HnVuIS;CRtPMI^s zd(ozq;hUVC<6DTmW5gI^?J@rLjE=sKMY&P(0!+hX7d7Ym*<_RNl`@8=SX7T!p>>=L zJtzJFHbKv0mL~224^nmtSLupox=eWq_GpdRpXf6!PW`}iDHPeyAQi4C6pn_^5U$5< zVRU^Jaw~Hgu^*7fmfd00mKYGuD?97An=rhhvIF_-oV~}|OqaR}<pn5w*X@Ui=0(#A zXzdR8+(hq&)^+3-kjiMob%?1YMn<k}w5!^q%-q)YdCQrK_?{{vTAMach$W`_kTEvp z4V$NS^k}tLiMD!T&D~jBGh>qj`Or#*Ym1zsQOu4ht%m_ivfhc)%v|zHSwZ5HzN#@l zNr%MU0|C12nYWMNR?MnwE|H&)*uU{U@uh2#p4wjd33DFRXC!f*rb%m{Z%e1irL~)+ zph9Wlg|!n{!7h_q%qSl+sx3hGg0k*u50N=j&A4QgCq}KdC9Sab$$DC53H@;)GpgA> z)N5n;;aS3NdNL~QM`mGL72a2Ezw`a;aST^(vg_w7btQ?LW=G+FOW&%*S1<+9Hj<gd zZ&MB7^vr$qs&`Hqvjo+TJ9><G&`S8!x0Nk%j4mIF_zBildZuSmAPPs&8?)cmIwoE? znnyM|%`oUBR_%mu|0s@8o}vVsRJEluqe=qNWW{@dkww$W2jP|-+p?K1Gfa`nh}Na9 zL7KRs+_I?N60J0Y5`12HYU#;c%8T4tf)hnsPguT$5{x_ISh49bgI@h;UVlmA<G11~ zFxhBTc(j^>LDd?8s*bMw#3z+a8?bcIR(&#spQctC_%JXsjPi@fqR6wA_ybPLPvXP* z^IirT_;Y2j*6a7d@1#gotr<WozJa^BpNv|db6U>nSFL&DeUoy9yGo^3^~8+w!Iy4P z3kKh-wtboFCtv8HS2uU1r;YX$FrGGVCI>|B7cn|b4&?LYM<42;*iQ%OlD*pJln?&! zD^8DHpq1j6B<|A0(_#x)J%+5fWvpP&1*~uY*VCw?Dv>baFFj!_gU(Nf2#@CTSvi!l zM&UBL`+Qltv++v`S9@hHF<pVV{;KkeR)Hmn1eVBWqR9R^(=XmP)o!)bQ-UW|PxK$P zacqlabS?PkTC=u_#H=Hmj7>)AGhLovwq}tsV9W{BYI-${?sg-dC5cITb>YQr0b~QJ zeOF;(Yb+%gdE*&ZwVjwQj%^9OMm)^io&rr@^T#X$i|3BQ^;K(5^tBc5osCsGl12HP z9Ntd}KD{LI@a~Gjc7nx?9=DHQsz{U<GW^%$;8l(P65i<EaOk<rkIXEiE_(RjKu7FS z*M}vEDHAnKB<#AOmz0t5Zf8~Np##j(Iu*!SrcR!F{o_VF>vRGue%9h4cFGPnT2Z>Q zJ=QL8&Lkm_IjO8L*Am~cvRH048_@kDwuSVn_e$aT@J?;&ryIFFs*Bza?JX0#3Xa{y zsQD^dd#_|-jx?p`Ixxyx+m%e>TBs4I?XUU_-^xk7blvm9H?ebRtMzj1)J~PIRjrHT z%;<5hry+*b)S`eSN%4<bu2>G*b3k_4B@QG|zS=7)7A-^$SQD=XE6e=|>1=VqiiT== zr1%IHKON<IQNtTU1fZ@e<Z84w2knlF6;7EgHGkjZjP({yT-H#wzB<upQ7TY%1GkgF zyB2kWVcPsD@2OL!xrW%u8B+G4s8@ZpB10-^**gVz@2h#+xuTf6S|W^9?8m)Sh5F^G zDe}S}yzD$oD`wv4YaC%*eWho*=9d|Nr#%L=JJce>50WRHIv0oC@3uZ@$WFjkd@|Ek zYEE`wXrymwHM_w$pYD*O$bn1kQYuZ(cXYie?TOaa;JR3JwYJ}}cWB&%FuO)m18wAM zmW2!NZ{gn^wtsq)ovs`{H8kdx@o~)fATgvhDpoTqwOQt=tbSBmkqO`7mf@%STJv~l zdnN}Y3fG!H88f5igbN8K+DQe3g?ZS<-StW)rZ^;rKb5`L*%7=UL4MweG$kH-1UHwQ zKj?9#zb7YTJ;|w8=&rnFsJsBVC8V3rRr%h5*);~#v7qRh){|w;5jiE{Usf4&Fn{SH zF&=~vwa}5gTg{Rj7^~8g&}Rt)Uf-sO2R0ggwh&AV=}GA9h<GPj%pCLm9PW9>f`GYZ zd_LKwiPrHrAk+y}OQ4--5>^%t%NOAwq)(?{Fvgl|R8!>NcnOZ_sPUwt75l<#h}(#J ziz}$F0>~J(fZ5|%WnL#qS!?_TB5L+JuX|fTv&`l!!dfz38HRT2gVbHx6?52xi!B}< z@tT+mWIl4s1<R(jU);k=2zqH-<_Y);@hllKG4*7b4%NP6y^ZbD=*uJa9d@%yuEIHc zyYNDqaY~NK8yHGOCLD?LzS$T-#!T&9bk~0P?4(!WfgH7j2@~V^W@1_rF~#Qui*(>6 zt>O69viohVc_^VG!r+R+IJF<GsB6l6971msz_dT6IiB8QfOhGVGBlmtLhL;m_;{qv z^1!_@rK{RGjXq?98A2|)Y`wnN?x<T0_zet8k?J!OWRpD~O9*RedtQoYp_Ot{XJ|T< zjl6oO*r;$gOR1W1&IIetJ%z2dN5v#FRdeaLqwzb0G|QS#FAytba*VL^5-;*a>P-l} zbP<+F((v^%ZY*lVzSsLE0~40t1Ya_Hh>=I9+OEvniunol!`Z4Gptk>;5IvKf@RqoZ zF6XFxZ>cLwmCPWe$)6C}-`ei|bZWy`-8|tgJvCn>4E{(=IfDB-Z`#Tr8Eb!A5_eG6 z{gGK`@AQsI-8yIbbbn8Zo3L&hRtBrKzujbn+F72ssi?M{8I|?1eIG-q<96$4UBEr! z{TZ70xV->PlVaM}*kSTH#uk3xMEf+^1kPBmv?lQbGq61eh)wn;(RYV6XLQ&J7FK<+ zW5tyxrl8dw){|Stn7MPNDXJB!kH0s(oqycgq8d%7ZbB>d8Szl~&a3UeS_Tj1%$h`k zkk8>ykkPC0in^}m5D-<>obhJfqydiRDw(uwIyYtW3D#Icmg<S!X{FZo!#TTuWQM|n zHYL4I;e`kSj&le%T-{mLbXCIgTc^ukjP8yprF9!`>%dF}=zdn@rD*rqTbB2P_cNp3 zdf%MNb2zVT{p7Pz%Z<?`iH^paN|QYhEt;_6`eZb19?U&VVMa}03wu@qQc!iUBZpFc zInRPv#`!k<dNQgI4reT9j8?SNRqe?Yh>wbzzCM)-Wp?GuZh<4er}p?P;clg&Sh<kf z$A&EY8@hVYU7tX;AKSv{y>L{)@-<l&8Da2*HCI6l(j4P#^_gOsAQigO(Ph%Fc*?O2 zLZ2mx7S$6eRXRU1N8UzjmM8Yl1!IoX_@y&N*Dl_HQ$K(!gO(X8k7dOzsPa;Rug9-v zy0p?SM{DoyqZqZDjI9I=?g)siC7Yy#o5f5Lx53}54nYuO5_g@obxdc8;8ay#Lo7SK zqI$2Vs=R*%V?Kl0I=BCg>diUn8hXUcioyyG@3k=?GJTftP4O1-{zRD-ws_w8iken* zY^7|e<V)JvrDq{M=Z~sxV31(mQC{>I-JO59iRr=!xW}+$dKq(*P2k=zf^+uKml*-N zUt~666F9AiBu|HJ0aZ>S73(R%JuAavV>(s!MF(I~yytonkr~R2I%zN5<GCR6`MH@k zyd@#J9&r|Py-Wwf5-e&epkZuj=`LCkqNmn%SkzJ1GzQPox-omf3xo;fX0$d=+6f70 z&btV_zXoqH>tpDO0!-W=a$r=Omd(`ce&=)xU0hmgub=FoC$O~i&Ce2jS_AfyuI8>< z*qv=EBp_ySQrnMZnbY=M)jpr-YrbIcRHuzj7n`dXkQczBY($P}&l~w#lFhj(+Pgmv zQZ8*R$-oRK@j8hHa4}XCu9hXE-d1&_(ormtU*;Za+6&J;+_bc=p|L!|1shOP>8&FN zyJw|It9R0z#}le!cZj1+%Mrmj)7A*zFs5Kt({qUb7^Fv+Gnj%RnbS@gOp7jW*Q~v> zyLrzm^Hv=i6mrvs>Gl4+U_Vo^m=(8DcI<~-F)i~XmQ=as#Lf-n@MG`bhapBDMN0R2 zoE(LFxLm3BC(3kYl)sf+T2{_ay0|3q4PrBqdQDhJSznyH3iK&Pr^jLW2{J5cne2)A zP5pbns{ss<6vOe1Cn@Ij>Gi>tg3S>fPws8nRuYgN%_shHDmdLjwT@Z(ez42ohRD;> zq58&t%<M5~Jv+s>k!OiLB$<y#`wB6QoR<t=6&8Hn>Ku2q?#?D}j9aq#x@(p)r6!qi z#V7Zoq_{RO<|ZhRP<y4MI0YA_+%X*IJE||V>0C`<Yp(m)u&UOTJsZC%yx;hESlBqP z*ZhnA7RR%=+?JssnHL8Is`FdYEgtKbPKxuXkRnLASzk!wx;Y0)l?}D2O)l-Z{khi& z(HT#v9;$R1zK54^o_pUYIGgXynT_R*QA54{(KxD_S)yRwm^o@k&2hx#9`)}NNVSdj zjy7%dkCRCdH<-T8kgBjfK6PeCRK}E;Y^^~0=#+5g7T-m8hJ5Pd%7(<&N9Z5XQ5k&^ z4OY*^uS8u=&%IrjD{X+Y9O107SnrPB^LYN8)V7a#8+^4e{S~$={vxFgeR{t0PNEIj zN>40Pp6@m|CMh$78f{Q<7JeKv)#WNu-`J6=n%R^k66!wYQ5kMqemzbGJ%qwrTPEHw zKhtKWY!MqJHe?XUTN!R<*zve0W$MKpZOd~Oq}det6Y*XcoRa(KNhe9>$v9_w=eBW- zqj1Chsp0AIe)Fv@UV<Hum-i6F4e9A~LsQ8Q4HwVYorr6QUn%_1_1NaAt#$*URlm=x z2w%QT%q=flhCZQF;|(b^yVMV}l|*df<yJ2<lXUm`gW1v~pGb08|6Qt%6RBKEt)Be2 z-#j)|=^ha^r0p;P$-Ajc&v$dfS?Nk_seB4rF^77Ipeal+YFqS`6()@71nTZh8-)S! z6nb<(j;J-9c0^!a*jZSHj;2ZF?Cr>G&C_viQ#C-BNoJOF?(Zq|!?i5j)Alv#o;iji z97#Ry65BtG?nxh=9&gwlV~S1aH?be?G;nQG-4V01BC}G9D$YkAYBgbs@4(7fE5hFI z_3h~*@mYWL=kQIKG}U@P3HrG29c}uee6)8=Rdc0gobHM1Qa-ZLrs$T6^rM0?4S7;+ zMXv6F8~yfA;gw9cN{}Bn+UvBzFe#Q&c$~)MbhWSV3ycsqpvZ*8G$s=E##9*_CVJ2N z7ENY_?;*Gsqs2eAE4kD3kUTa<E&9!HZ~C`lRPg~_Q#1*OVqfE%{agZ$Es@gUN&_*5 zrBw3+wt@!`;7Hha-Ia;}ScSiS$gD5bT}ieBCzZH<E~>O?oru+UyIj^iVk*D_t?lL( zyXlqBd1O1h9qlV4bf=yewcDrpW}_Va^k;+{ZW1kv)(**zQ#Iq)lU<)@{5Ugj8fDq> z*r=`6ezd>UgFf_S2!}pF&7znrnfGCB%&?@e`XvdtP|n_p_1FaBbe6PePhn$XQ<<J( zT5M%viu)p3vA$6&(oy(4*}$jthDqN{ro6dgC90&{CEj#N+<oz3c@Lb$kDL6y*k>ic zho^=c;F*io3MY<PlOMYY-<5BhH!UPQ?a9m~c#!c75ljA@i?yj<8;Ph=ZDKtce_tCT zn7L(01*5h<N4s_}p~PTG$z62JzKw`V7XUQC)WIYUk57o467C73Q@^;U6K2g})Qbz; z+Rl?dp4F~jdLq+z^rN^M#HEv`oHj28`zW0~=HqkMoX^Xy0OQ19c+zwi^<awI`gTDi zY?-lQP7TAP!pi9+ZkN9QWpCM|jE0ymGBW{Vzb4w)gk?<xG$C#xqU^PMyrR8_ldJM+ z&atg#TMt{<r^+kbxA^Q8wo5klEv>PZ&?HIdvFo5iK0JSXOg4qS_v5pS*~7F<asFhA zN!)qmCov&CH&wT#Z$XzeB#!Ny;-&3r$t=Tkj0+k?W!!&1q)n6SGddmBHr$I7W_Y4) zAmq>$ySgn|`W@t#VRNQY8RfR+Jy*D~L3Hn^`83G=+tR5^Y%v+F%}NEc6pCwXYddlK zoM}!?z`a*<OA@Yrcgj;#4`0=;loo9@!#0>Sn$fkw_Y@gu&r+9^gb7m_m9>F^VsL|j zIeTjd8b|wiDP_q7Q!9TF3A&6yk*x3NrQ4j-j-o+_yWVM52{dd;-?Bw~XM=sAeP*R( zj$cQ5|Cy~@$YD}3^H0*HUX97YL(?R6<%*2vX=*QNj3WtWwC}1`r;~|^PXH(g--IQ} zX4=RW);cQd&JbQ&lHJx7c^}J8$oj;bFD9fC(UwbT_1J_5PhelkdEVD3jl3#2{;=K+ zE)SO^K6MPM^mz&RMl{Cu$vi)OBkuaKR0jaW(AJA4zE~SifcqP^mp^G74ZOGzkclRn zlkZnn3E1KBsVjR5N3&3lWb^U>Wp}3$IbBnWN`v=KqHsE+mFC<V2PzGj;dJVm;@&}~ zAokO|u*}0jv?bac-BK(ooY~{Mu-<H2Co$gEu&BaV+vJ!4vK!nH>+vc=13)y388gC` zU2~FDt3=i5kstfvypmyyDeZ!t^-0@<sU5x&f$+6dM!@@yFoyn6_i%4*rspzI%BG87 zimrRY=@wlb7Y}h>mB%JBo|AioePzPLp-(yuo}&ai=%}VO1iP{THlZ<M?6Az0TS{kE zqU<2H9kEX>*LG;}h&CV1`~k6N*wZjt79dJj%f?^9VlpzFELkps@I5@l@^g5hiYSDB zqm`7?Gnm~5!j>ac|9FdI(U((TiJj>)PrM%Ysj+q&EpuP1XJfP?Qv$(s_yg=)ad}E- zZ0?kATAXHTr5Kbk40?@D_j;K!lwwXzOwk&LngSyhlyt9_$pkvf_BD2y)JQ$f=x;k* zMI#+pLU*xLb$ul0t|I6z=Qi|`M1x;pgmkDQ8GS{qHhmjC(?_;jw9-2vy*2MnV5HWf zo}uZYo<wFNa0O2s{Z`S(Iqs9+|5(Gi%y>mGam;d3uyp92DfJdpa8aTXki*8FF{PJ* z-&AADX~bFO7O&J2z@w&!)M-3gaojt_>l-=7NX}V!%*H?NlU}1gYpmQq?ygenREadn zp~uS<&`59bOmd9bacc^0jCz((M045@$w}egy7}YgL7<``H8qXknOHVnXt39&1a>ru zX_e)gj`Y%}VzO(gx8EyBuchu&^YUU2Hcf$UOGzxrqy|&u%5_!8X;}61ZI|MbmR1El z>eT^hNk0|gIGj3{r|4D7p}?cFH#l`R1Rob_77|LVv>D(}!&hCmdLe0im3_))*ErXg z*H32OxAG5qyClu@?oX0hhl%b6*30ti?(usuJ{y4E@KH2X%C4I@Rk7bVfc-<$jnRm` zXlnT0(#SjP?Cx6Xa<9kktxS?n7Ds=AhRv6b4W`H1$mi?a+|3P@6m>W51K|9V_@=AW zn<+GZDkTltly>hn{*G_R)zfFa5IMHzH{!uht1g4#LHJ^kqfu>crKYJ-?7=G%Uqy9x z!&NJ}4`Al^v|%40-|?TyP0h^JN}Q~EWrZX30n&KH^IxDS6ucfvo=G>2^KPlja2Fnl zk0;fC)GB;BF_z0JsE;frf(Oz?X_tb3Vu?KAh}&v|D|Mp1?~hb_1H>5hBxR(~mk{?h zO#SKFykZTf+vBG#Y-}z9w>7+@OYY2}#@whMf0kHh)&zIIx<Ur?Xb~23I*~O~y7KiV z{|}(KTjAI(?R*_>bJFp1uf6(wfGtgXp|*@qNT<%t0a^MAV*6MnGS(%L0y+z^xyK!3 zx2g)m$;tD7siZM<ld2PZ5$e;!?!pv80T_({0pSpL(5WEx9X#?!h$T1g`kI})_m|?f z`etr<dHH9C?nXdE4OeA2A~w-Y9ri4S<<ckb%1HMbR<Adu2qD#h`SXBl<07JkP)i7~ z+fHdSQlqs!q@2AihO{DNg7&~}co|w2e<5G3T>RdbgAZR-Pc@eI|I-{QiJ!xBT_us! zy>+Za(QT*2n$vq9%_9tgB~dnYRh|R(?T)0KCYCkOD)2|m+9o0}gfDUJSlAE#Oo_4U zFW)-o(#hr3<7B#wW`}!a^BYO_d$Xr(@w+{_UW^sf8?eL&1pczmkF{4aRVJ!YlkPA6 zGDjZqufDzA{SNBdYmOrU%RcsB5O*e6au;u``*v6C#e@-viDoA{mDKt`$v#)pMtC=g z*vZVbn;O*;x`yiKE9X_>6<nT<4q`29fQ_41TH1XoeETRm{Mh$r_>*t0{H0`Ew{=zT z6Yoz;MZ|M+q>oRd6*%?)L!wz0nJu!p5B2vpD+Et9{6CC+bwHHq_x&(*hlGU;pduik zQcBHWONfHBgi5yvB2qIVq8OlnsECY8h{PZ*HCUv;NOvemclYnUuKQWnUH1F?4~&`j zed0d%IrrRi4vO#UU$%~X@?@BDXfADtM6Q9y7ullX9TPb@<-OD~cXIlEWt3to82Y_X zUY(B~PBc@l<K6veOluM3L?wLRI!O>!slYUl*(sz*AGC{1GoAgGB4_hD+v^rSk24}m zW4~0t@}F7&KkKTlhs{CEy>PqVVr}$SF6C*;q16RYe&!-)gBCkhu?XbGhSkj$AFK>p zJ5sww`^*|D4#b3N=+6nb6j5cBZ1ZZ0^61#a-c+x<c#PjY%*5k^u1evOz`Xy!AoOMV zb&~vg?jI{2=%1z0iPM;<ofJ>;8gtyz<ke{6(X2|(Z-F#Vtn~S<#sk~)^5voA>?d?` zPhnacnwt8*9RQNFCiNyn9GzSRjYK}R!*qhXSXr-A!q(!Sx&|S=1hXFmR`=}w&iV6w z;4KNhQm4Au-m`BrJx_gudu}Q16bhm$8L5zy-H%mq_Gc8~-|dK7;&hi&n)vu(^y7;7 z<(Yc2yELlJ#-nY5sCdgSuHA1f%BiKAeFVBF#!suv)RBjmr{eRd{iX8630Z0CQjU?X zZqxm&BgI?B(!dn3t>db_(|OE0+NesVp<lXJLpGqF=!!q>Z#>}@@aZ;RD<Z=ZO`fh% zUp-CjI@e+^l?P;h$M~qrllq@Q;eR16bc6{J#xiz#`^k-V1*o0psp(i!zui-;ioV%6 zsVUd|vBP^xGjc+WyYuK4M+>CmfOYGB3TRUaw8B!&d{X&LxfMKeY&M6u^gj=50gJ*B z^6C)TyUl179KxEReZhLErLoWoz9H5Jw7{}~g53l?Q@yozwzuN8))Hx0F_aOO=~wyu zzVqEd?+1s5+KF9d2(x-2VGSf=4uBEes%7V5L-w-k>i-(efl<Xr<0jZqdbb?z-4D+0 zVzWeMLab;{jIYr}i!Fo{h9sko9w&OrLIMhG!&sx+ws{S25CH`{vn+Uv7ON`nMs^O$ ziB(N`$4ImrNV;SKS)}-(l-YNzj<)t@zpR{`rs|F3OP!s=Y@1Wu=*8%ZE3W)yvxmp2 zFyuqeVd0~`r<d*_O(&~|S0FBqbVA?Tlh($<cW&obKvsX&pVoJ&wX;}c)!lss<T*y@ znoPIqC<AqNwcN|csx(B<H`g!fbK`ERV5P~M5@%EENO?ni9v<mlj$>LKYhCo%n^?b3 zJYR}Tqrw{NRv{G6=?@_S6Bl!h7Jjf}sNd6fUio(SO4q?FTGTY#yvJ47w5i!nf~*bF z1UU?XK_2NhBvPR#s*c^D$(g+3083xGeX)z#Z2sW<{@`EEvT1uN4|$Y_I9@?qLOkR7 zP&Pl`SYv<?SiMljXmVXNUC83Xo4_dhI?hBh-IhKkT~&X07t1!K?W;DPp3qVv{>aYW z-m#xD^VRaI_UY3DX7X}!G5G?f{Y*OoI6SUkSKc@s@edE@izQdu|9W<0;`VoAftcjy zPChw1)HAP7Ues*`p1LC_^{oziMOo-mbe9CZ7b?tVwL;^Y&72Et{PI&%DVN*b^%1*> z4}<(y>yl^`&XK54g1ZD1Y{?FWGO7Aw0#~{S=Pu?2LwNJe*Pv$O4nvT;U1CAAW9sTJ zsb7>!|3Z!<oaMA}r|;I^?UXjiQ~6eVdpRTxE2r>gEi0)trTfq(JR84`9y()?GYjbw z^Gko%Ts5<EZ8;OG6X74ZUtFD-BV)%p!F1}H=A+cd1uL^#MsdANYtL4}XfA$*XZ1s1 z@SRR|-^uY0b4$J87p&#0=-bg!V<)RV*oMNRTg}fJE~{w$8pmk~$SI*!MiQNo<zIz> zWd%c$&F|TBf0fBQ{PfUu)%4t4?De3aEULHN1wAF@<u*8{NiTb^6hgr{jH-eB#|<Yt zhE~87Bj6MNRTj^B0)^z=E50fn|Lp2(TPuI*s{J3xD)U|birjqSe<cbSiU5rpi|_FF z@YRQ?6+v<j#5z|Upu@a1B!w{%?w~*Ia0-<&SUQ7(u7>2bWhuV{?N*hGMQkeb41Vud zzuu02n;5!k19qq3dD(sLv!A3+W+i8}6rbFG?zP!kMr~XZ1P;r-r&A^#y<f<~t(_{; zRV5%_XaOfVab;{x#SU5(9!0Xb-Sg>Fv@WR#jh2zkXnK7lQ_kFlQ&FvKkJ4YPqtZ=O zCZ1T}mx;T`v#b=$7j}~t^}E>=`Y}l3rT_-fhiiof3ZKW|IuM&P{6x>}9QdwGctv@> zkO9k?OZHAqdHZbBE9EX_FI@egmt?c0X$e%(mb~w5ee7xOY+X~A3vL9NkdrjV8M8h0 zo-E5M+Zj1;Om*Gxy}}lOzs25Uv}JgJo>7%(jzhA*WB>#R3U%)f>L?fbHkOqA@OvgZ zdgFeV5YUK|8u`H9IVUnITfaulu4D@>N26k~!jO3QnS)tKV&2t0_ElnUPnp>pvz0+! z4=IZEFf5!)V`f8agFcN_N>gpzOOOM-gYG+yrKxCvwAzzP1fMZ4teRm8A8V$jFREz# zbgZ14MVVcN=bsi8$YHfANlg5@HWBRC#HKjhbvq40{bK0K+JwN`xetNIq%B84wl8kO z+Oe${uVXo}&{ntfO<>JN-^4xK1RD0_z$K#<8s%~izk#*6(8{_z89~lKEV)%yrgdr^ z8MzS)5H01IFrT@+&KZ%RYNd2XbLR*y^+gK!3-o_mnaw7T(D9m$(Q!NXPJOAX3(Mfu z`h_;t--%?;@9YyDMnAW#nsB&+GuDa`F~_X$*=HbB#?u6;F(uVE_$s1~yiS~-;fq_X zJAp`3DSKckv3h!XRy;fjI(Hz{`?c;Z@veBd+t2}Y^WJ>@S~qI>?QTh)|D&;1P?l^X zRNkNP!G$uP%1-<8#BPh@^t@~0t01QnA^A`HehG*@*NWifzH9k9=Q+wyN2GklLZ_le zHXXw`XeQ5w!>v!wt^X8al-TZIRI)@xWmE`CZ40gR{g)PX7=o{w8az_yz^=?Z58O6L zH@3}bvjNcsl8!}Vt~S>S2`9|h<5}oBTAGvP=xV$4l29;kah|8m?{SA#mHOlqn8hx< zW0r4o@fVmTcCtV-%Cjwa?*t^-Wjj6jOWzo7jn^fw{|PNIN5-9;@5dL^21DhyVb>#R z>ytw_IfcCoL;ap+&z%+DDWxto{^VgSDK9H$Uvf1FOjFzm;p$Hu7SgN|H=b>nIi5RS z2KjdV>TI?M$UR{l=4vx5hLVY~pP+4_EvVf*z=7dX+wsa4-AcK0d14+e(u$K|tdc!c zglA9<b$p}Mnfyj|C_G2yxyMrCGs!Iz@E|2LUwzFV4bd0Zu)As8%}g`I=@&WY4#ZuC z7PHn9B2i&`6J=#(eThkQZCQK|CGXPp=;_6=0at&eXO8dJh6r<+E;$|VC$Z+xOb)tF zgVbK3--ZS+#U>C~1-C+aI1BoxZw*UdbTT2GLuBRmpn$nN!u9^Dl2Ax8`u6u+M|;)5 ztng&d9cJ+2>F9}zF^=sQV}$Mr&m_scnTsn*%6?rrBGdxS4p*fO|DZqj#~@bp%Llxp z5*y71V{@bK+^Ofp5&anJ8s&^vUhlMj?VLAf?HopoeO|8?r0&>z?aAfP#EqwHhMdoQ z8Fl6T<qkmUq+RV?Tk`YOO9nfa5|Dz7(tdjHX_J<U=d;Of$-+|ZXg2Qa#$0ZDTP|Z~ zq0_AC<E;&jFzd4-i|$*lA%^8Am1oXi$Gbo@bS@ibBai;u$7|8(?Y7MTQe5KZkG46! z+Ljg@2v0W<=gQuS#o`z;q)w%$HM?P7+|7RF_ENrmywa43Trjhk8FxmzxbC?9k@K|w zJCO;#qNmPqn|LD5ijk|&;A!iDvRHA=eP8X902gr9LHeqicSM{$kF>{{KnK-j(EV*8 zudO&k`?K2C3yUM4zS2U|_5YG&Oa-lxk&)x7$c)}z;!RbQp{c$FJK-$S0wvifX?X~v zX<k~&W$$DM_v@N43k~&$#UFmvl)rk|9PF_Q6d1CPF@0gZ&$dmesY-e7>{`UOb_iqm z(+W(azk<kOEbh5g<7S;fC$EYC=an>cnGE`X5RgB&CL9kU;EvI_`8lHhgE2ZCg3nuZ z>MyO?7WC8xxxL?G+{2P^;bc+(?j*zRM<&kD=hJi8a2L(`pV7RNGU!B4{5H<q*I;ej zP|A*G=(&%Qrs_FusdI*A@O5NzVHPdJ{|5Yj<Mh7~>!%{<@mY_E@dx%j%vUZ_4}%q} zijbc<g6cU+X$h<%dOZjC?xJ4*5e!9785o#v$VxuXg41hYdwdY%N0Y;kTdkAbaiqwo zt?R=>E6gw1#$Vj$KYk*{)3|MYc40y}Ry`lSn-lr&!?)ZyN;wJDDoL4!&iZ}FZFkYF z|NYH7C^a>M!5^xZawPNi9d&|?orFAqsjhrLr3ay$a70hLxGCD(nLcb?DmKxYhKY&E zp(y0M$L>O+H5Wz?hKxv!T&+-CiI@;^^oNqLUbWM^{t~GD2P|VK(p8LVdZk&OE@@@~ zs!5aTc=c;jiG`w$&zO-*|81;u=_T5r^&J>-)eIR~Pq1v{^Cxtwx8suFc)-yklxC{! zda4+GXd*E$Q-UTDH6LxZ^=6bX%w^;L66Lnl_tV$n;qr}d5kUejM36wYom;Mx_Naq_ zRb{2q#urQ0ZTEI{MpF0wmE-(2GyPTsv<DV{`ddZvtDFAg?HcNCFh7jQ{Ge~^a`um* zd8mUHy*Nv;>(7m$I5H;qcTasQT9||`F!bC)WmA`i2E{#MIBq@~m?<}uY*(v%>px+x z0<?saoGE|cnzx?G+))h&WLQLyVPV{@cZ5i0&dACzvS;={_m9mSU;S%y{<h@O&J=0V z!ak80y`qHV;{D=O--QRnJ!_5-*=gjS8Z^@<JV2UK)`8m=1W)x#lKIJwy1v{GwV|c1 zfeHFz8}wK=9U?B#4%!(dUA9cm<3|Z|<e5bvNZdbn+K)fm<F~c2utAd^hy1p99Fkoh z8m%;j>42hZT0;(1^qG2Hal%6DJq7aWb4>kmC<QjM%SoTVMgx|+NU8x3Bwc%43eG3m znDavGLGb#m2N*?KH^z<-&K<Bd+;f@@5A`q+mN_6u`qoLu9xh^r@>tM+L>-3d5S;2c z!ri9Sq`>ncM#x^dfMz)~_-KQ@ARq&<(o;PJ8higgLznt@S&RRLE`QYIt29wDbbUpn zLvPK_|BM{;G;W7rVUP@)+o<1c*a5Rg*-HgC@x3Aq2QeZvM;*1#NjPx}{c`pC$E@U< z5h?9#=EJ<<dS}j1R_<>QMn2`&0Qglq{Tg~Fsw^NkhGpI5(JXpO@7=}u;iOHj0k~BL z3UjZPiH<gn!>m`P+*G~p@&B4ob_N?{c}Gj9XVIS*QTt;sKhymA7HSpD?HVYn<mBYh z`w2H)UucW2J2AKhhW;imPs``w4=?gNHJoAEq>5ojx{w0HPu`!>H}}8TyI_1pZGH7O zXo~1x%)2W`8ty)G6KRYvNJo7LX&C;XP2;L&UO&0hxG-mLeEH#XziqBxel6vtlIJ90 z*~~`nm27}NCln&B6J}+hl4u`HYR?p%@wlyzK#azNb;pvSrx5Q{lK5?0ozF$pbhJ@! z$6<^YEv)d_2G#?Ih^@3YQq1fsH6_}&N&Q!Q4&IQECs1^gRXsT-g8_+`#A+mHSWj=i zgBxRJ*;>A{b7?T<m*n>E0X;+t?Zkp--E{hi)Th{|&;%BXZe#nS@rkklJ%vq3ZysxW z^ndQvjNGy-?puA|IXG3e7NmUyU-2l;uC%Tyn>Ui^m9xactWVSN12_MV_?O&Z=U%@x zaLiQqJajTtArOE1`J`YRmbb}fO6^iDYWf>ji2J`B34|&J@98hcX5TYQH6hkvAjX+B zRi)SpqA|%yNuR6se`7%|S;IY{TB90a{6-!oUkkxD(P4hPI}0Eul5BFMb73<A%;kN5 zV&9PKWbiyIt06KUj8OG-buB^jH4*7&KDQUac8U}jxp{e;;#zf+b+&pVB?be1*o~<v z*Sxn8e795Tn$Bz9UD@@)g6AK*5GJPr4c+%K2<9*go_{Tf31c2&6&Q$JeqLW+zpx*> z(6hlf!{&6vH?vy$#vP-efs;K8xKoPg;+4mfqzr}4Pa-2nz|0hR!)jj1vo>pUerwjC z)S|vT5n+0@%okd{OGt{77GR!vd1fZU$i1@sUQ!h_-h>RHM-`O#1D&19XRc|hKr^Nl zl@+@5dyAkmgIju0pFsUWE0spuIh=fn2AqNaJR+L6BQpMeFT6mbAPw%(`p9LimT??B z0h&PyKV3Aj;7LT^Gp{O(myf9teHwXNEBkWXP4aTD6i5J(+{Lt>-0EuPTW!3jFdcT% z=UzDU56aUToh46BDAq$JTM8%C{m6irGw`r8NG)MeOvs<XEgDx(JTETcJW$ARi}7DZ z6(ddy*GRT#5+J!v>?q3Z7cYbWuh3BQb>Fdq=nWv6U_JxtW<RtS<eJR=uS5Z6Q@+i2 z{#+zzG$XPdC--;Lw1VLJiYu!arEVqniD^Z4(|P741Osv2%W<sh8PryqrUz=1iNqqr zXg4~zrx&0h@e$dPHAHNE>M})@z8M5!1pFL8Pxx}#j)?hIv_r<1qpNCu?XwsigE!{` z{JcKi<Xk=N$y6<S>6#N~szn)>U*h%ha<x1#Y;|Z(v%HJ|>u}FX%FS&-<e(G*(RW;# zX_Rkw>IO-Mxr-}{-_;#T{lhO;9q5~HTpUh_e`beZ!G0^{;-q=^aQ*fts?yTeH?fCx zUbuIlxV)lbC3hKIjaKxDMV3BYl=I@EwbDqo4t<YpY((d_6FdYJ&W&=pb#fa^*l(%4 z6FL3Q9Y*A%b(76ISq?sMzIPJ_%dcqOCIRX(9*Av%eX)h)2XOEFh89gt&DmZlzh+av z6?etirVEIjhi{Z^ZCo=Lx!J;{^l}ju{X78$hMnh@7sh-1R{Q)w!D!nQHH{3%b7@t7 z=EBt((o$$I$V|m!RY?iUvU>z+VmoVU&V!EH%LFBbiImWV$M2!2vD}JX{1QA2lY7$e zh`#{TZfq46$Q6=#^Ya<+!7pMstJP^rq(2*Q{k(y=$YD@&*!$L|-8{e7s<jy$0KRtC znp3gfqo4vfWSIRsYKaV$`&~Zq=+9Fz06sO-D=HzunuuTmD+WYjYLP&ofZXdOfrx8K z2ok_&zVnZkM*`emjGOvLkJQ<HXg5dw+a(C+#AFlXW%3-sF_JU!h&l##Q!)=ItvUj{ zs!C;PX}L;+=%6_8nYi5#?wkJ>oDKRHt=q12QPna6*zDI4&C<wtk+za6WE+$eUIwcS zT5q8wGb(KH1xRQiczJb>{G$j^I2APp<+5$Iu!N$>o6VSI^HfW}LD%#5sVnum`U{U) zYiesXs1$DbeSd7-QkS5Elx)(YmwQCAI<5{i!+_aGI%Yi<s2i_z_LW}kzp=&11}X2b zf)Q<}iaD6p#Ldeddbcyu;m0-Aw)ncwoYKdgKAuTDygXMv?7zmM<))6NWqnQg%(VIA zDT7j3uT8J6FcW-5x^m5pmp%Q%!+4vzx_7Kdu313XbUlPXpr?bw&;&NdB!f1anM?_| zhlKF{3Z4ELgI08KI&DKtGc7GGnY=g=@*#@uC=(bVq&;{5gx9##wTA3M9=(ZRC~!S4 zfWQbMLbMs+P|;0moN7+MzDqG?b=%R;i^^3nqedO1=q@5*Kl$w2efd1t!NUbkJj8?* z*+clPHU3sCSK#iw?6!=!0SsIx$9*TgeywIE>$?J5yA>&uuDRu*XqD|aaQg4TSHos= z6Hyhl`S4!La&9MD3eRyBe;hB94W^-Hf%WIz+yDzohu(-7L1CRt@8KfGJ58@X8$B^@ zziD}C&fNcpU)OJmG)AA+Ei?CIoqhS@*RS{Z=^DBS9Pb$~x6_$Jt%=EbsRYcsn~=ip zns-y`Injy-W;UoI!N9HOk)lUCD77FGT_RFWbDkz9QPH<>;&qLUtq`Jh8YHBGojjVG znjTI1Ug3=EV<$-~4_+r*Nm6uIk-`gNqmc!!L{>)iQcDQ7XuUB9jzLExG=UD(q9N5j zUm~@$T|)q(coO%RO-7pF#O?yX#!@q#!uUl!xQOlM-Q|`|y;1pjTTojP29ahWl4u^8 zy(0CT;3KGTym}>;_FDB53#y;m>%qS?)n3&xhiRP9e~kqwU9@cQV}9H-Gh+t*j-l&w z`h}C>Vc5!zd%);YAYk-jrzu5(+AVDvq)jrmIOl(<-V<VLVlqN9=GbL}!l!fzsCei2 zfftbhqJ*!jufNo|I2IdWI;lS|QhlC7%1ujVUbSLhB|0Sk*|Wg?GI@k+bO(Xyu<KJl z5<f1QxR&k60->-H%Q=DLi_XoHHaAcOJXSo9uztSzM>iU9cARR-m3SxRJHd>g+#?{+ z=4J2^a_i|!RQ!Qb9wpxx;T*-8F}gW0*qHU+a|5W$nD)ExsUb+;9lnwG6s^1%Xn1w& zFH!Kn)&b@=4L~t_@?bTO0O~H^?&q5~l9di3bZyah5Cy%FQsLmU7o21n%t-~allren z<kdxZ>bzz!#U3PGL##$@L*9WHn=^T_o}?j?OCAr9nopPvucD&g+Vzc1WYS%c%4EP7 z5e5Yv8;5{^gvPjb5ZF;+)2tvFGwizFOwWd0cEM(n={9Ty$ulGkSqxo9gs#lA))Ksf zRTf`G@9lyUC=CfyVI>;ps-7O04UJy@SU8(WG6dlm`5JQq)ZH6NDQ-y6NBpiFkmLcn z*XPqjtAMA`Dkdg23`f`W5QvQqsYELdr~3P?%<Igxj^AZU%75`W3o@J9J2Z3c9?>A$ zUk*;?)}Xg`<Y_F3yEOsU-({^vNe0C(7WxS_M3>N46mSwd0kVl>Z}<_Xqxc|SL%My? z558gs@n(h0CQ1eJ2re%lBQx{&To`_RD?!{)4$3(j#;(>yzX$wtL~As#Mx+ys$>-Z9 zmD7dxt1;GD7}64{aGx|2sRpe%8}dS>^D?75E8QtdnM)-X$HAM%o(Dj@Nz<Qwb-54Q z3ZZ=~L?)!(tr@M-E(y+~YexQbm)kBLTk{6fWM4Td6Nxpj(OQ_3*@zpuP7Iw<c$cz? zh8qDV6+u_&{nxM(s68(kZ@->_oO2N-mkpIz$yQP!_vz$8JB{FEOQhl`kH6+5OeCh+ zJfx25!49Tw7-nbhaFSYHIe}&P8joE`AeWoTiDlDmc-6K!2@?U{39V4Uv_NE#Jaiw! zVa+Zeg(&l|G4$_O3jBJBC?tyhku==(cBof=%KiRi^OI-sQnf=pAzP14E{!OU4Ff9q zVF_8^9?YK~8`@_kK5=}1S=Z^hf|o=4h`KjJhR7+i*7Lwty5Qd+{$@sA(%xCN;~0vt zmG|Zn;arNqMs;k+2VKoCS5KRNv1ZM3^YsU+%X=~UeG;}(HKAwjG5@L4bHKFn!0G(r z;-lF1!jo6{_&io~Q*Yd$OH59l(QmOve<}0e@2l6H9O7~tB0zO6SxyK4)xptmMx;u1 zp7`q3NS=mBZFov~Y#)>^s^%A_y^{^zyg!oyI;yc-`PheSJ#B4m%j*})7Ib{Uhj(-5 zNnFYOhY|@{cjcrq-yNb0uvxxPs$U*?i!b*G-ibd%Me$&qyC*xLkmJ#eUS{j+=VE7_ zdcLyIod~Xiqo;LK6dnl<AXkk|xbG3vgR9K&1`6Ydo<~JClLtkMn)BaRp42Ij=mMGF zQCC&>eP{hfXe&7={O6#aJ#zFVb<5+2Lot2~U`EcW)wuZT9i2VX<A);gq~#>Ya70gs zg+I*O#HEKl{p4v$7<&%i<~jgg2yGi)8GdmHaLU*A)NLikwu$f90rE!<yvGX`wB`E3 z@&(~8!@T>cj`JmKk|nB0L7nR_Gq`{3j6~*)jEu!^<4^A1Mz+m#IH0f7;YtHS6neCO zV1$9r%gDT2|J+c2ZL)UBE^DS{VqK8>qkGw{7jJO@`wdYLVAFXc1e+7*kNVH+p)@*| zXDs#fNaw*=YwaBfZi&&ww^GGi9cLmu<u#DW?of%=ji|MZarx^jfB$JZ9A|%f;oJCa zXtYY)9U=+yW&W84Cyc`c)7a_B8T%x+QUfpTAMgI>=isuLGmH{JpVk5(!CYl0A>{N@ z5QCSvr}kzW`T%B+o%h?!tnWD0UH_H}{`_3SU8lN6=~E~4!u1ULN16Wtk>M_0e7=RI zXcxXPRRoc``Rhpd<tIWLA0=V!>-jnC1(Yi}O;iwVhz#cLl&xtukp1XOF79MA|H?+4 zt7!N27naA5kH9$77S65II`qEo^)&Wn0-(g|?u<mk*U|TeMyBSuOpoxm6b6l@s%ppn zknXqIPfbOS-Ol3}`yl^1IsY|9t|ci=zh*n4e~dAKm^vVoe-sl)TQ0`j&VBg1HFa0X zaW;LKBfouVzb-){yQy5Qyp7!n2B#;vsC>uWhPo!ZLq{Lxs-iqAaPK&AMT|MmaeCc! zs4zY5ZeR;A0F%-mwRq>*b<Tbt!c57-Io}umz%p?6n7zFQ2lNhvP_ICw;ASVtGW{<e zW$Ee}eB6bXJ^v=vjPWz{v6`G|Af>hSJ(h6!hL*RW!>f?!_W=`;dC%bkoyMr%YH_IF zMLIOUFU}W}4)|AFn;%06A{;U{2TQv3CF}>2x1XR#U;sq6B=P<@9r~?!Dg-l$r#jJj zv@sTcuAYA`tLf*s#4QL~3vJia=OmAB1~lCS7WEGuD$gBz9L^-j%J22iG1XOA|55s7 z0lz*L0)u0Cnz6BacC!K_Y-~HYJsaG;F3(YFLU{eoVe+kUUf9278GpPCiI%Bt7hd&) z+M>di7(nhlC4u*&<c#9Rl4-vCoV7+`c&-TNe<F<f#Msz+fxzfB(d_RdriMQt90nId z7wZfIK7y|#<y7N&$nfL-4-{wxy}Dm1qWdy-_g+As0}+nhIWfJ%#5LwS?Jph}q89yR zwbAQtirqZBcJEHr%oZs6bj8pJYr%}9-CMPPrrk0P*+z#<q<Pn$5DeuIT7ix9ZQ^Ir z3g50j8FL#az~HTt*wB9aRTjJU&J*i>J=|%`eh$JMW69S`<Sf{h&EiEPsI+XS(oN`g z$y2g9P32-nxyq(LLdQCLo9a?F)vi?4As%Df1C8A^+aP})Lohv$*_b;x5;Myd-IgKS z2N>kE`RAHK{%k=ADMu`BiRe?mC@O+xut)%SGcDtMDKA=CFx5@<rNmR?&JCwGz+bKR z5uQ>%3vLy>a^DMRQGgeCAY~#&FYTkmMp}^i$gp(6p#F13m=WubBzzkek8s^xxa58d zXfF^eM^Fl4pRO|W;ENV7R3v}8dvN=Dj{whxn**J|B`SygZ{7z5wfRsXCj?FaM4*Vl zT_<?q0N#~oF&SLw@*i<>nZxA*D(O7#^#2MPE=Tv+C*HUt2*lz`9i%txod)$QoIe2Z zb%v9{;K!+P3H$2E578uPL~v%4GNUkM2_5h$K(9XzW7m5RRS>KdwElPs!Ztth39wZu zvB-l$e}`FLmPE`Q&iJ^VISVpCCy7NSzIkO0x(3K^K4PHc>{G7wm0>ysS6RN(I5R21 zW5W*F^;hQPVVv~D;C}Cs*WZ;y_kIGsOb2d2F&vV3Ca>ezK@qXl9CoMN#Xu_MF!Y!8 z<e(Nr#(_cGtT>i`Au}@*(G>kw5}ZxVw!cVCU44nC7y(`&Y0fMQ6={74s1!5{Uo75~ zRVmln1pZg~4elS9z?vWp@h9!L@{JYPu{l07iDsE_{ZQIHIv1Sm#9h>vKA80Ll<ABJ zM8kxyAv$PlN%F3DaS=3h>{4&O@04e`zushFD30jOG(<A^w6t8KW%kaOZ+8ntt~GL( zhnt+#<E1%maqodzYupO>hXY)v2PwxQBuxgVl+J&=$;OJF2b9>M?$v4jAY*ZajEO)} z#+H_`2wE}=8k<W5uFW)*@MW<o1Z@R$mfRfdJ1tx=Jay{SDf16U!;LPUP>t>jV1XxJ zMU=VxhMqm>dyc20;#hCL)8v|&gA^@7T`~6YDFAx#x7qaQmCw(-Cr==B-CXP&wvRc0 z73KlE2)REE7PS<l2U%@8bp_bD4v-^usF^aERF0XgPVxc5)abpnc|3XxwfT@Eg=~c{ zrIVk8(P|fm1YvY$yBtlLk$LyitM(2ynP0wDq$@;Qu>A|DikuAnBsAbtm%9k`i>fF{ zeh*%_@Uq>rFKV^iL%9>A&Jjt`3`nD!!HpFNw$<5i6TUqQEX^O>1@c*)Vf$R~fWbjK z;B`kj0#O+dB$G9OC;FU~_ki)l(hSMZ5fPUc@FvQ*>n@LEcaCBcn^7|RyWw)1AqAN2 zYC+ZV+@kSXn!U5vN5B)mIbQ-rQ;{YR#OFNreEAt@+WRg534g?yb<fr5{wto7p>VfZ zGDOj_+;lQbOzS7b#(t74nARrSUG)XO+au9_&3CrQSFtHipIQR46<m7*d_w{r+M@_g zn#4_Xh#KqIGN_C<A@4H(kOjGT0wCvg8T`&3eO?8!Pz%r>N9j+zDug>+Nl(({g|~TI z@MM0Rts#L`K&PSi$~+RAQzg*>LzCxJZo}16op@O{U02F^(CqQf^$am^SS}koyL~26 zDXD-sER`VU&L9`8&}#GYu7*gt`#~d-30d00)zz`J)oQa$QfQyT1ZhJZLd(`KmzS4s z1w?NJgEpxrBWNCS?T<L?dMnrqp?pp$Fs5Y4KKj8>RiZ9Jcrb8_+^ad!**s3tH_eDP z@yzmBo+!xz(5o5f<L5h<zss+f2XI8t@gqjK5hK7op7G~=FTwd<q5JeUfN~MuHl=ej zCZANY=Yr~0Um)IL*A}!|VRbly(dDeaz%pjG-Q1sbSz@Exq?sj)$Yfu2^q6K3=QTQ% z4M%_y2)a9X0TSr|{S~v+)G|fS^A7%z4Q19>w)5<mKmuJ9F9fNRp%YL<CKW+Hi^fZi zf4x2Zh`a6LNO=MPgb%}j)6H;>fk{FZ6jxfYs}t-1DfbP(xGkCH&#uxWg>bD`!iS{t z$FCv#dbI$Mo2A)$l5aiM4`|oR8fb)br+em169Vm%%o}ybnGnt%B7c6lTr?h#cP(3l zH7rVUS472T!pnRd^Y~sCH3zRu&zcF!yq!ToLBp?J-kY?nNum=Aeh<6CDLxv=$th`r zZ)>6eo$-@eAS1C8*xQ!iG5MBkWJb-D;d}Aon`*zf<-r(odF=F-&6^#7x;-N@&l6#F z22rtJE<%5*zYmz>YjIXPO7dozZ`Z7Z`cn{cN~HJmayR22Ff>IU4Yk#u$%{?AKYy?a zUSyrjrPPzk<gPDNFBTHN>Nfo#;emB9b2bg-#FDemknyUP{-iiatyg0Qu~9v2=Kx=x z(Q>CPj7!m{gpUh4fKKsh!o$GvaFX{9rICjvxw#5U4jm4W87rd~J7k#OHNCm;O(|9+ zM#(NpzSCRU=k&5s_6{U+EoZE5)^LG|DX!S3{AFZ=OeWdt5Q8*hzuH>LjhY&5MF@nN z(k!rAoEVA5%x>0Q2HqZaTznu%dIbL%mHVRDnxjAdhfFXC0&BJj=5}ghCFq*h^FB#C zY2HtH4i>{Joe1#~DN&3d{?ka=NH58td+K#)r;*43o53ZG=lS_<WD_$p0kc>Ym#(5K z^{Hn?N)K&Ry5e%BrMHA5`eayOAN6ax*pKq&MW|X7K|_^#lU%dK2EW_>8WkVBaz`Lq zoHbWFmd?`8O9glN{lr#|r&k9PE`yL8c&(T;biC&GUCZlSNznus#+1TEJ4I-gSi#X$ z+EDnPLnf_qHNB-gk;(C8rC6S@+E?Tjs31WpI25T6r6d5C`ws41B(bz0g5%2*UefDp zi%zPV11&Hc@udfWZ7V4Vgi6vlLY~rD{UMRHyPKqU3g1$RPn{^<Vr<X{;rCu+{fUbK ztnzJpe5F=y)#w<2dFkSd`AoV^`7;g16DQ_VN5N`N!=J6ZLVTosuJdLiIA_F0HzRbi z39m9Onf+azYpdCs^J~NHjNWJJ3yLPzK3ixpaAc(lCo%;w9Bg+<cFdHl7fW`xPsGpY zn)y8JOAJajdX|}7ufm7BnB4H9Zl>8TnO#CFnf=V*wep?z7)FkR56a}E$5z=3K3E(t z)!0`N^_E=A{yEC4TDfBGV9?xh^?{d1k31V4HyHX7BQBKDXewb3fVC@BJ<77lD~_jI z`_1VKNU$!q?tb3O!^bxicsOO_DIJ{)Rn6pq*Jv%JiR)Xh_CHUv`69IBc2H$T*;|!! z?CXYWCj=qy-9e>??&sPfFBtL|y%PA{+UkSt4|lhJ$-P5V(YcNJbX3a#{6zpi*I;pO zOe&WaUv))5`D#W!0Gw9i3rXo!OmlWnBGfiDl?3CX2q;CWxp@(!{1z&{=T;ezA^caa z@bK`Yf#vMu{TE+|4nZppdjzNF9+xiM1{_g+EO_gI8kPrIQ3|sAZeKqr2sR|X=6X^T zd-aFA_cpqlpRF}Cixj*a?bDQGM>OVEf^GDDS)RP0Ih8<0sw0zdUOpzw^O<>n$+!ro za`(`EdZdA<2Nr-Ffdg!O9zti323SB_wZuDq!3UPko`e!<+bTGymo3f=S$+!Q*|`g+ z)lgr5D@AP&R6abxYV`y4qS1|lsQ2#!n0p}?kasVHRn?6T0$l!d@jZTVNEIW%T`1C3 zsI@uj0;y5>&{4k2@eHHLB|BFF$AQac6h(pK-K+L5LWG8p>1`nq2kyqju9H&ZZZP(v z3fW|aCTDV^SHM6;7?7~UHaLFbL?CCedgy17?N+=5Sh(HE0b+sEj#Eat9-^zzey8td zxC~}PkA@UnQaasv^OgNo(snC1Of{apbk|VD56b2*+*^8EJ>887aL{+`zP*Mh`qONK z!=Arebs}Ahd<vRsKARV?HQAF9%dA^0J1Ma%P-C{!iD{UCvc3^e_s#Qy<f3!6i=h3j z7pjzUeX=3lNZ_fwcx<G8(RV7c^HCS7T}5@MCYjeQ^E%kl8X_g=sRUK7C*P~0#5bRv zcqy<_N<(KuzNav_D}C8dS2tWnUjB9po@i-PBlk+@g?4ER8!_(n5m#5`nwPufPk+)z zbUQgB@FhVROfy4`)ot7he%$>0_Qj?4Uo!dojodn_%I&1EeQ=>1LT|x3hV1k^RjeU4 za8*z;_tKOdd434J&+hyio9UfUW!DxaV&kX#bQ5L-PEg-Szjnf|E`gTHn(?|H+w>b> zHkNOF`1EIN$z^8<gUOX?_Fm~MSytP5-W)NAQ#sl)f+Gw{G;}7btvVeT|3cYLqb<qd z8|;=dv#qSeD<%w!XL)_fnlyCB!^E!hWUj@^ChMny&^NWg<b}#&Zj=_*-rjzPlvK4* z92qk?O%-fxY+UJ6aCIiF<5Ct`-EGfCmcCoN9afIb=y@eMGO|f<v@_u%z6Nilf1Q*h zleo3ND3@)#Rh9X^Wrj!691kyV*2UDROvr7zIxg4p@MK8cXB%m@A!nFu&gY0i6Yz|5 zHe)b2R|@HqG5AdJp;%&*_@R<fGiN2s`-leZO<i2}kuTBJvr>BKo4WVB87v66XjM-x z77Nv5{p%rpAp~@;_X<JVy72<PMR1a^h)78in!{6a(SW{A_Bion&(uOoK}zdZo0=k; z6(0MvF?%ULuaUcw<O0IdtV0I41J_^8)Z#a>Cz-r?Hf7}R(ksmLG2#&JRFYK>V9Fnv zIcP$z=%^hodV~HHyBzK@{Neezf8Q%zYE)iU1#G^dX}F@zOJiRmtsm2BJ&?qNd_gaB zm-m+(`^I@w%X~<IoFNln(e&;}gDYmDUEOKD%Oretf6_c8*&>DJY8tn3yLP2bq|=p2 zO+VqGs;OZRgZ<nH57)JbMWb>~U?^N^*HG7JP&ot7YxURt%rM652faE!ENvvHKR&<% z0M;scnW3fgx-^4DauMa?5yMt_1z;8Z`wQZ6xwQEz!FN<%Y;Uh5f`+ooTYO{hO8gQ- zjvBF9Agp*1pSY-Dy*^#>csM?VhB($8xh2_$>NtD}c)=PlVl>Ck+{P);WI5V|?G)!Q z|GfPLM{T!O)Ort*1!e3vLT$077nWjtoWNCVHeo#8r)rhOth9?-oDL`FGa7O2Jc)(B zVlaQx@-ABi^d=<i9mT3_4#s(-cNAQTA>Qo&Fke^tL%#T5F1R_yMcYant4qJ#JJYhI zE_<xjODI^lg0eL1WnJ}jlc(r~d&=8_r-ws@-_q_bj@rw9_l8hN0N89=bRUXUXK>qo zN)AJMss40bE-a2{LNH9XdOS6YD)9ZHdUYvprFpb+@6$Uhk_I_0&)<Gn%@`VR;@>MH zENGat5jB8K!U$TPW`Jp-=z<q%u2_i=g9XrI94l*TwuVtHQ*Wn^-ln-;I#wf`3RZ?O zdP&sFvlwy#8qdGcoSwSLb9m^4<0*Nj#u1d5ysVmOq5f6ry`(J<+Mw0Bozq<6!D#<Q zDfFhDr}S&ZG8K)alj>#G6^br$L@V@@s8=`R%}I1bDk``waVqX9y0ueKoC(+Yp>gE9 zN&bdtcE^K766hqFu7@I;Sk6?M@?+e`_m~F=K3Mm{&QZ!i<uyz8wL{_gY0RTNwB_9t z@y{<Xe;P_>`?-F=`E`R*y-)}nAs@}Vc05qgeLYl#Vf)ddZS##IXg6KFg|jY)bE2{y zH%|49R7YzmLZjY(2G9BKx~uj^bQN#YyrF;60<o(3ZjAdn>k$6ORveC+ig&jGCfBkk z$uONQM0zH|P1e+hoosm*I}@&hus_GOV8-TO;_<g>T&I*CM2_*;s|#_hx5H4dR*COW zxkC5?WxTM$g%LDt7oWEd^j$amRE0TQ62PZ8XZl^OqX!d<(kn_7e|+P*k7p=C6sKw; zpiT}vZ5+Da9mstk_CN+8YiGrZ)KHQ><8G}{Xx2=s=JfRXsHp$`90DqLGycMZOfFJk zAc@p#l7HbDBdcl=vgUKfpVwN>F`aRCeBiicb2~Gxdfok>rUum~nFBjGFg7D4iDTU! zPWDw|)`$)A;~hnpQIBWxZ6wg`T>bS}eYAw$MxlS>p#1>Sf&0LbO20N@j(77Y8=5%Y zsp|P4v_z@R4a+=utf&%ss)d1Ck4DC@hrPKe)o$ww6aW)+7=tRArU=(~;htT1a_A5T zt|$t9UGVMH+gkewrZY}<4=NK^BuA9n*V|njrE-J-X1;PDMDo`rpW0*5@j>6w)>R|Y z2KH-?7Cs!+Hurhp&28GvL)0}r?70T`lwTU3{~mlSZX?U+a5k0884i5F5Q+Ktu8$4* z&69n1d@&)nkO1>l=Vc{>+iCOCM+b)lwwdHVZe4%xxZv?lR77!9e;?p;TPZJmG{>(h z@0(0LKVcBP9Zm?03EF`^MACEc2!<7{YR`!R>t~c!_`2(fk(2Vy&x8sWN{8xKL!DG# z+-90hi$NMN2e{04;N8k}q|mk;O}P!uS=1Y)6SlKv>L4WQzrW)Dew|=$l3trRsoc=J z7L`GZvk1P{(6zQ0Joraaz<b2>D>7@xAEQ)JqA%lwXlkTR8GH?Ws>F*ru@*|CUH5b8 zxhU^^HD}Y-X7BjLSFL6*eVw=xlVg3ahi_c6L#}%RWL#2k8Vpt@a1t^Yr?)(a&dN<# ze}O0z<z4$z3qYC{=@Q(WLOn5Sp8xFTYDay?5w>~zO|u@#4GK?H#z`X?p*++W|D(K3 zW(!EovPKBL$*3uTn=s8#H)?ag;bBNvcwcNhRoyRg5`RB!LoTB4tB(z#e0ngz8W5tU z8J?PbJ3D$y#k#Lv)lYNQBQHr~o+aS$Wk+YZ#L0&b_p(GVR^O2m3b~73o-QknSBvK8 z$06D3lV#OwD24HEnf0asj{f_qs4x2L+}xBevR*D`SmEH86;&U{AZD8%JCe@L?G=E* zYj|JF*hCoSz%XDGp9vS>Fgpo_{!=?ProQJmvF;o&Y{y|-F<7ckmfoD(?6G(PJ<m<4 zk`_Mvb$HpB6L<_>=ADiNdVpisV7L0Q%)0dvdAv9c9Dt$&pT;A_x5-5vE_#PHrO&<_ z2Mmg9v{@-r6^p?`N9MK*{n7^f`+CEjr#+~;cdT2yL4IeJxf)V0{ffK}+wL@>kOA~N z{G$<+4A*_+(f+qMfBWecAH~u0kgC<U9i|VtfJ>`K)gXLJHwZ~ki$eAW7*d#XfB)9M zJ`FETtJE;vdO9#-fBMA?CWybn5PwrV>)g1h?ib&{!)>79GU_m9vTyw=58*nq0jU%2 z%b}hBGK#O-nzl#kFMaypiyG*PKe=x5x0{c_D0F;U*}FHcc=woSI}C*gb1FYRG@8xv z5at~^*79vl&B5t&*9=A1L#hEE5$<~9I|d=zb!z(NkNG?X^EsWPBaXJ^r1)J4SwA4* zFbB}vNXHggtF%sbhap?w<1UE%dk@}cB{*<2o&K`DRX&yGw49Rq;=04xr9?1CV(eI3 z!*G7_7a>SO??Q<9Ih>2_R0-|POtkAyCBya;NJ7#$DXDTIpBq5qy$q@RIF{oUsu*LM za(1zh5T1bJ|6v3JS^1UZ5!4O-v%3~gApfog@nX;IN#G;MF!c0#?KsSW+uW1DwGPsc zg9ONvR%L>Ny6a&HBtb4QaG>v>d_&nq^FTOtqL$8ZO4Y=bDgMlQlL*U4k|RU!Tm9U4 zq>8i4Tbg!Y`nWNra^Vza61EVynR|E)E@JhjpQq5_xX%YoB&<7=MLg(*Bc^-DzL|V6 zR)8INk{iQNbW1BhPB3IUVUm;fgxKTr>mKEBOb*SIrd~XPL-vd<kGUc~V^Ze;I)Ud^ zYdexI>KKb=uD>a}xCkMc+(<H!-BA(>D~8|(#`AxkMT+82ESHLlW$HB6Ka3bG$VgiU zl0-OHtya|>f`yQTjOFYX${rdnsZ)B-L<o9AW*Fq8`y*-i=ad}J!Iylyzj|ouQ^dCu zSDLwz_KlBXFrP`<J%=#NG-_8T^i`O*37f3@2`;bj5b>3lzAI$7%Yf%jn-DB$C(oMc z4OHspOE-=-Pro&D)B3M`k72tWhMRk8TcPU)<Efo`4{P9H-bNKiTk`UVDb2pK#4-Wy zmDmc-`|H024)X>r$avy{Pmx%dXU?tgSQ)rVBb)L{M(J(b4Q7^2#*ey{i3`64wO@YS zuaC^;t6QHmiZ6`{zp!M;If&o0U0=yc;Z~6;!94M}y|gja_<zNx7zP7U7sMyxr+<l0 zO}SN4Tl8dOUfK=*3lH_@i(!UnuRRa(WH*~Qa-m@^oJ|OizeL-_R`j;|#zVmn7s3ne z=$`KzmO@<&nAZQ0BCc_RHSvLPVNcrp+kNcN+7zROM5ksQX|yO`zv3=zue-M3`q9CO zbKKwpC>w~msCx`R*?#{SJO&l0vhhg(ZXb=^YojEen+I#GPnqruir4skU2zE^stxMu z>bIXl=dok?5;!(W_4f9<)|?Q@JRmR>0rC)Mf%hf`rP-jFhfaOWbpDf2#VxZZ73?J7 za0qs19p(P8r*FfnPRFxMf2_G80=mHom6ra(&~;L&CfzxO+W0tZm?Bnm(vhz>;xz;~ z=i_lB_jsG%7Hnew{gJjxi~~Yg@nDDgto=9e!`$thKT+)4cR~4$eWRpd!BuySW_ib! z?O+ob4vg<dlOACMeb}|PpPwlkgLmZyLvUIK=RtBvT-?skf<$+vsn5~?mVUkWx<~2f z7{;8&SRA41_nL2slCX84)fL9$TWJ~4E=$p<#W67gY%R<2R|mfwKm5-dkK68XsMQ3e z9H+Fk&6qDjbH*3^4KO}D11bo67uv^st;=^yNXYVdWZpyaYXn8T`0eeVN;b9|AesHH z>`fL-S`5t!#C>AGU>xX+1kf~Rx1TySeCf3FfY{&X87IfQ+|BM65-zS^4joICIIrqw z%Dq4*jv)Dr*mpb0%gB^MqTB=8UEv^Ma6(sCciu=-^Dc-VZAkN8a7`eQB6PHr>m9Ir zpcY}6?G&XnSt`793_yJIhoe9W9<rnmRjne?{Ga`x7VycebNkDqcOt||xd(%;PP0ci z3fQbX$<R=o86(S^9doxaHKA42S8B@lOxK8%(}XsYtp}h9MUxYu%n!G{t#yzhHU{ti z%COKeS17`+JRM>aol-DslqdEUDg>5L2XI9yrL%UY{5zyUgXX|k{*=Xd*Sqfgtv}Nk z+*eSY@Z^i$uc>wr>=+r~jL@&kp(n8LG^G_w4XGsYo!fzD@wq1Rn|)akFGY1)-^@q1 z!*9*!G5CUegKUy(t)_gLz}oz=(AdmOU8}06wsea?Ot!h?1R(Y7h_=4INNxd6Rh0mB zOjEbZ1>YN&pfG3AQ+j{r7}6|3S2^=^Co3I=F#r5eV0q4)K;R-{#q~cIYApMb-)>uI z9q;~dgnk7)sJD3n0e(9N!*wZvS8S{&U%h%20woo_hZ4p+kxFZS1Jx72(l4mEqs-{1 zkp7NB5QOTR1pe=%3rpERbvAAwnK_TvccECX?f?GhLJttp_pM}1P5hZ<-rB%GcL<iw zr`+O?JZeZ5i+c*F1c%7RX3kOzLdd2S5i}+=TF~(~{H95SNIeWy%*X>5q2lpa7xle5 zS_;~*QZ~7*!7{S!$oF(@v5kMMlxtK3r7oL$;8iIEZ1LRwn&n&%5?cxgkVRP)Eff9O z9t_mRM>fwYTqtzzEd2fq3Y;aMDP$<#_kNbH4jYuHbZyGH&@-FYSBmtdxhU^UpNTB_ zwwFK$CDveYv~{|F<Th^^lo2EmdA6{Kh-~A)jzn=*u!H=-Jq6YNb{eYwAll7Wh&~HY zZM7`V>3?xs<5`t2NS(+G6oBbp2#D(4Zmeh{R<-5)(ZPvpq@69iJGQveh76P&5;sQ| zF3O`f?bu^6-C@YQsorKY<V(LC8!&M6=(lg*c8jY-Yayt*CoGgB5#5cdmo!WGJ<jge z^7U;k@N}PDMp`JQ)FYZ2W1@l7VY&$1kIw_3ol}#vp8^QhPQTj+vK?EqW}WsL!*f#` zRfA~D4%qQ;n(fR%_h#gKV`vLq@;7L2x_@_z{1b0z;p6{4HvAqc>__}ofL;)$o2XX? zcV`ReNsMYQ{<C|r{~8{kkNbh)ZrvxH5M2i@5jfqy?iOXFDq)7i`-M6u&rInXE8+cx zZ)Jcf`|BZF^a*|Uu44p+mCm2bNAgKL=r%pIb8>pfuXy>mq2c4Ez86!V{1s?*bQw4| zyOB1H*toc(P*bqiqrF*W??*~bmJQ@P>q3|wZ8x^_$WU7u)f3(FNMr+>fVFk&m<~0! z8}BOTv8{qgwc1dY_!5Ik{uU&jvZs~dtnWhaKD)YI3m_+$ZuWTMX=z#6eW-PtJp)~_ zbXh_Q-UW))CyqJZjCYv)33tJDZYDGN>x^eiYE$;wkt!qzh;o>>lejTDwCIzZuUgu+ z8#l^vp}NB84=VopqA)BiJRA3z<CBcDPs%woaRB*YX)JQns_?OUEYY`Bt^m^|=GCj? zS7!!y<hGHkUYVs^X}ZZxnOOJPcYv0XE{OSU+_d-n;k*TzM@1(ux95Id198AzCEm8V z8jszcZuM(p+;?T%T+ui%ddChgYg(t4j0DYRr3$832X@qsB3^eiEO)kK`?(O<2EkT3 zQ}ks{jzQu4S7z~ftWA|Y=CFy0$$l+U(`Pp0@8K-F^9|(TI0}pZWKnNFi4*xc;59!e zp?OO@3;`1f`)L7IKDj4NIHU|6>*E@mX*pJb*BGV$eM^@*QJFNXLEY$~z)JtI$z*4r z3Kk<N)BGm4#iy657(IN!^yEc)i~Ff%@N5o)BPzYFz-m7ngT2r(CZE0pT_WC_<QyLD zHqP4ZO{%(v1peM{!C1LO+U3PYvRtEH%2}uhYurB(y7TcEQvZAkd;NufKhmb~IU3vf z`9-4y{o>5#bM<Tb2V4iwxvtVKwOH!Epq>P2%h={PqTkwToYqxM`fswC`Xxx`QHrgi zTb@5c?ecCMVRZ{YX+jM#saeb%DxGm-wQ-uENNibEfwRPFU=3-%%&}cn_L?nqZ!{4B zQJ^etZti{n%opA}|J=$Ud#G4>hm|pKqk9|}x#tWMoEAB>-cZLG(UR3R2wgQ#(>Yf1 zWnp-E*1E@M_ARwfapW4#@**o04cqQh<`T*=##ABaH@s?Z7E7)a6cw+wpPJg6l~)#k z0_`z|j2jjn?zemR7Gq-gu<5yjr2t*0;pY2r_h&8<MX>by77S}H_c6!}GakR-gHx-x zvT5LH{6TfL?L_HO_f2caIkXC$xB4AEb0%fg)e+GkvFJMfjy}@Z9cMO|-8iJ+WorG- z%<an&E8^LRPAMrVdG$g2O`CTS#hI@ZnA+OeJ%C_r0M6}V_KhjB6+KvxmzzWks2ym> zgZ-8dS$%#asC;#LPbbo6d9&(C;}SGKkxGGyBatgcpxd}S<F~rKUQx_)d48fndo6IR zG1esSav3cA9Y!}&w29(d&ic&M$8D`_Gv!C3b)_1LU1grA1g%4H?1<RoTQh%7w;8la zFX)!4aIl`52Nlz;rhCojM%yQ$4Y9%xI*TK5bje!!TpQ0NyN*LCUDJ1Msj~v1nQaIo zj`Z6kUSCmo&j>`ZGnnOVws<J-wXi?Ej>}@GJCpS&WjbrsA@uwP%5ym%b~E!8c?kqK zg<1=p=unQ=d?U2-99mhNk)AomR_fuFv`LtBM1C39_+sj^o}Smj0cct^;d|B5;=pQ0 zAM5?G<JDS9M9)1%dD84{vLxo^PF`cFp!r~V4<+}j12hCEWy<-snA3^Uu~d5;47H(# z6x#r^e5uIjlTFbcU$&S>(SvDcaBIfqjk8vplyWIo1<NW*cl&%cY?g>RK~nD>BKjH{ z8b+!>PwHxXK2*tHP|xQMlT8~{SNmfu`_txQV`INWe4cyqWb-A>G1h>CFR174yUWVS z0rm0rrqtlwxQe!OEA4hE)kD)`4<#!}E0lUvhSZmY0)-536Ca$PZOvsn3a)3xD9*`; z!}DZC&D^EUGMrJ<PiAW+r56=Cp*L&@-92|a(0J^Al26eI20&c_cT8UM<KE=i)KM1G zTtl*$6}FHl&hxqO(vXo@jk_lDz*e&vxMo_!+}x{XDEv$(wMJsI^#@M0p%mp*tJtD; zf%oAC59HSJw-eHWfM+vk4b@i|y!U;eHQV`EL?zMV7!~SjDBFwH6ZF->)Trrv$!KS( z=gxh#)ejzU+3s$@;J=sFj%*tuE)@}|TMObuS`!ad1Z&(M$*7(is4Y=p@ylw|Tlz{! zY4)7|c@rXxGG5=l-Rp9un(7;U26X?k9Ujm>+X#txe=fg=5MYXcNj@=T-PGu^8C+2R zeuRhNt9+fEv%Jp875?w(MR_R1_nd*eZ)A_F)h7U<{RMp1P2O%rXn*KRUo8%_Sc~OM z&#7>O81U;+f>Eah?qF!F@=N1CgG)17aH4m#Jwh>kNp%RYvryWNte<=0kF|2%)G@E^ z^EJ#%ldQ@sU0+24g?__u{owh6KvG0y;+FGeC%3^@VniWh3*pvdI#To)U<?&|Mj?Ay z)R38A?sm@g%Ld5#Z=Otrq5Whkx@&qz$|*>U0SyG;<Ilr_`bMv{>y)kH-^b|JA-l&* zxpn1G3Gad2yE1^Sq?zG?cfPi%_7Kf`G|a$pP2EfLiu8XQ;$QzeblmT!YG*?*sQQeZ z|9b+!&-cG8y@%V$JlxVDq`69*9*l@6YbTEU{y`Ski+}{Gp*W(o!sWTOnNb<*IoUIw zP<lZ*q4Q;Rfb^w57vMck%E*;OIr&HlDz&S7S^6H)Lvh39F$(K*cD86BC6Bhed}{l@ z!?+=P2y<8iK>^YtMyaT?1Lx;OJ%7JXD0s#<k7cRWDd~Iq)X{CNjXxl6<g}e;x}qeM zD@&3pQTR~~u;qdQ{aMt#sh|<4@Ow$lBCkj|Sdxq6>?}1xSSTed9-hW3?7{5YZw{*m z>~;QJJHAxGJhZ1;MC4akp9A9GWe26Svn%rh`K>>S#Se>#fNgBweD|M#9buHYZ6$Ik z=2Z#yB`VkKY~GHhceMBUMvCxomke4>qox|JF3sQivl4@|qq+7j#KCYRBL5&MQwXPO zCl<?#tTHu7f_HHg3WZdnD_4m!hGYg*>Gybz&R^F&te`d_b9%adBAJfLCJj4c@AQTk z{l_cpKS6vWxa<94B@tz=mTUMS8n_y_3&($O;@v2ha?|fh&H_8i=eKJ{sS#50WLrI! z!eSm>-EyyX?;l%i>&~v3s}`SQxbBC1*nIR7U;w&kQ~jBTRYF~^goob?#Hn(g5=MWo zErnzPOFPy49XeeW&(9K)85Mis=cAp$>G#ynP5i6`!EX9l3E~;rQh6D*lW8yO%)iuc zVvZJ>QbE<TX){Lld)*(maXb@^zhLvctv{3uEc*@Y-A$3(eh3}RRt^4SI75s&5h=j% zaQQrv^c(!W*#{GjzKlWYUKR_Jy$MNAa{m}ymrPvnz9AEDmD?Bd7TKwyj$ta%%QR|J z)WoYa+M*O|^Jih~QpYXB>mbeeZIjq?Q*==>(0nL#H<@Vn$LO^Q!{~kXnopZlnsa;d z4A7--$Yit$KUZbgu4S%onSRF6NzIft?DzPHLf_*`%|-(@dOA3u_{Y`S;tdWcLC6+9 zlWB2LYG5+}MJ50g#bX6q0&!^C*<puXfo+V_F}wcw?%!6+k6(po4>s_zqMFigWAMlM z?m=M8g!*1gLu3ob@x&sOy*OU*0#j`Zo!D=y1GZZoQz{eAt{~w@>@8|Xn+BKU2aaL4 zceZm(gUU{gSV(9?O9H6gYm0@L9dFZ8dxu2uzdZy&hPjCfl`DbzGgfM2x;RC5Ps9?1 zUL&}Z0<Zbu)C*fuFd^<}fSE%w*IrfU`hF4o`=`K%+L)PlzYlmpg`%O#82e!O$KP{F za4-dV{wRWfIiiEOJ9DI?#Hl`N963s}@BcCO)nQR>-`~TKA}Jz*5`rK|N;}l3q)00% zp|q4pHzOuWNjFGKcZY#=$IvL!-N;byK6<_3{c_*m{o_9HIGj0WpS9QCdwpW**UfCa z=%9KNSj-kw%=kZ&b%7=l8)nb!4VW*cKL-~Y^md9>t&>>J*?$Hs7%cfXS8`dOx&)P{ z=INoA|5d97frVJL!H2cRT;&VBqXy`#(1miSRy8di2AtJjbj62(DP9eMpn6^pWEfK5 zhZWwveMPB7ikE*=)j^|vV)n}sh|*h6PoCi*dkj-GRZ{pFcj?|apFkmc%R}_XzAGF& z>L0g;?3-pWU&l9YO@71Q3ns#k==t0K0*O$Kkw@A!B>Q~d<B80he=g?<cBnv*_VQ~~ z2z{%0?JZSigaRQ!jO=3objTr7WN~6HcNXgB6bb6wd-3n`wroV*9(@+5nZ!rsts~fF z@ec&%vb+>99)KzH7C^4SzmTpuidAP(RZu4{cZu5+cpOEJtlI)HSauiygIS-N9lV5A z7){Cj{ic&8zXZupH3>*yu9b1oz6dz~I^dA5eV_>?Hjq2GIdEw0HRQnaef-45@SiV4 zHW8|jd*Dr{<VnASmfLMA1_A>wigpH&nks0K@@mR>ewSN^#Jm5<bAJV@D(Ax;o8{m; z@m^$f$!H7{{t{^~og2e=yCLc7@Hmsik^x+IUIJ6o=>BIS$KYg*!jkXD>aP=4bvr%u z#L!%C`{%;3?|=nf$!Nb7=3m8(ZrnvI%=Tqpx&iP=E=q1wJYVS9K3(}4H|=K#v{YbQ z#aJhB2BOe|O1}LU0!P7c8SQMOI1v<)pB1f8zWxbTbmi;G-}6z^nHO}Q0uJLAF>LgI zOhna7ARSF6tkqAfrRFF;fX3y|E!H51!IISEma!fF+_~Q`_1D{&qx>S~?s||xi-nW5 zh6I|IIx)D8`kwKU#QbQ_J=bFaW^W`RE?DfK>6tC@v{;oAd_|`^peYQwt`eiN&akp> z`*JsEll;JH{}!vi_7y+wK#m+Bx4~8L>8~uf;nzX(Vgb4S{(VLY(5<4=%j}w?{S&?8 z{uaCrBo?5h`p8fr@{IYxoEf=@Jo_d(PSC@lx-Kp@yCAb*aEk<X{ND$jMg>Ofthzat zTugB$gw~V$<Y32fhIySKkQq(`Q?3TUHHS-#{of3P{}}f_f~)Qx{JnBx)$`m87J|Ba z0nerxzFfvC!G75HEw#{^YL<^x37-9*1Qf87<odGZY*e@E-qd7FXZ^vJ4U~`Wp8>aF za+#MgaQ6pUuwy?>mh`W9{_Usp>>%fIwm4XBn4JYwksAPVaS}Cked$e}t~t(nPQC9{ zG0p$TkAGd|6bBjMGq9#Dor^kkm$x2*+wxEHa0)xJKVl4AgJ*MIFgGWB`TxC00)z=y zabJq~^5zHb`vR<G-UHc<FOi>tbDvJcnnwcmUvc;QPr`TV7B(7ZnQq1mi4K9an2X#% z1Qsr8R`JrCkbl>wdjPq4!HedvoBsVL*=NW`ZHOc$Bm^*7>Y5Nxear`7AXzj;Iawni zYUvz}agQmnRu>flQY!!NsPOK=u@`{>-W52p^+ZX#f!QDh^7@yz&lNPtSpgn|Ti;)c z{I#$AM=1Ys#lVMH0S!z5$h&DRzVVR4Umo>i)w%||q83DsYNcXj?`YXq?8N@p3iu~9 zc>c0MJxul}ahJr3whc_^QUbDPlYcPX09zxdSwZ24sPLFaK%V;J5i$#yPo$__1dmD< z4HAC-)u5B8EV6;ZOesf=4bp%t=LmcFKcg8;fImjH&KCX$<E{^|U2I98<6Zc|2nBZ6 z$J9&$qO*Doi3ya?{=HrP^8&yl|KzMA44iL#pPQGRC`~~EEb(vgFiEwZ>;Ky8h5cXO z`P)x`Crigpn2<0qGqByo15nxLFs8@L&h4A0u$c1Z>}mn4j)eHv{r>j1(66$OkvwF) zD_={q)-EO5`$9|NeF|_q8i;9pic}|<oOHby@q<Ix#r*O=U#Kva?g<98D`|Xii|ELw z3{|iv{lpHczg71qWf**L@#t;`zRymZJLYwOP!ravGxR$kl}K*<$pQpcX-_T~A26l* z5G+yv*XU5OO6{H4FTk7i_n7`OCZOXcv^O}njdMzag&CTmwh&eyV_+l$PQs3sR$-u^ z;=0MzG-=)Gc>HzFvyl~Te;eJFN&;K#lyrP{xbXUgE>@ho3`jEXlXSegl67ZhM<tZB zb+UQ>2h{f0q`7#v4&uwb-+l9u;O+;jA4!Yqi;VK3`lKYoH(b{c0C<0+XQch}34VXH zhdJId$xcUiXG#LXE_efa&RKH%dc)LUf<?;okzQ6#g$tnQLw4$qN0Cja>(ADi0_J7G z07IPX%M1ry7c^OJMKJi_5#Kc6w0h+VjG~i)^AoU!Z!MuZ++6}D(cvlb;myFldZ?PN zqp2@bqXkGCE<jQ_g(DLC70^az5|qfYM?hDB?oyj^Y-3{m-=I`!T7GGga~{49vEeND zE|UEA47}GF&JxMhS4cOpZ{C!!kb?a<t&l*cl|;o@z)woZ7iU#CG>CeT26>T!04&fj z7%s6|2ky~wDk?$1pzetX!nFN-`IpBhKa5kjKw@2h0l0VHBdg2Nt(3%Hp$LL}<s*UT z9n}0`nV$f;S8F|r;ESNQ7zOgZ4yb$_euGr(Hb27`Aod8wUlAW0sWI<iaRD08$%=)X z;O_432;a@UgLP0eY=&x)po;u*j*j<tPDLOq>X{hULaohUl#ohX2;fq#0a7?wEwvLv zU_Lwx7MLI4|Ln)1(C9@S1Q>w|F`v&Dgqa6GXc|GoioOW^+;4N!RW2LJ`T1Akbk0YX z78G>j!0v*^1M86Zu*i1f^U}n)ilx8A{^|{v-Kp@Y`OIAZ@&gg4<q)Vl2M*;YcM9s? zx8JhfKcN?`BLxeqlSpWX^Eax@i6^%<FC1ZHDJ1%S6rIYD;8@TUam8>_WPI2E1iW() z1Nkjpa&d7{Raa->uKE6!=qwM?6O5*>Sc|UdP<VKF3uv(7aCNDuM=~6Pp4xZ#;s!ub zX$AZe;L^?lSp!W0OL&TEQviy6eQz6!ss*QD7a%h=2C8F|OmErcu&^@_&V0$HW;b<q z4S>;HKvT7oq7*M4G<YA8lbb74xDn2+&$YX|`v~~wH3JNQqr7}CY(*cCw9QQe=)_tH z;p5*w3w`=Fcr?zi-F)J~{WQ-5Ow(st;(~iUuE2d#rgaW<ZF%480K5gy)iTS6MQ{=d z+KgGkxdGrCvw9N_;fduk8rYqbc=iI%fbLk)-uM|n5X7rouLJP8HC99elFD8MP_v&O z-g&b~wKT<0P&MtaTrwzqrSyBsxKG}>vr5N)_9xR$zAie)Hv_P?Rk`zw{q+02s%&0P z^p<+l{Crst1L`1eB19(kkZYFw&;qwE+n3~E#WzdCrLRa{bL&>zhCscpYlo0|-B;r^ zFgUf*hB}J?h9egfy7vKxn%^uC9Eqgu!>UsI`_&_X`4@zio1AbLgZd&J5L}`atp#|t zF^~iDg8orjRnGa;W$2WRZ*qX}VA2+43@5bT^(3||12&<PKrLmZ*817++*E|2e-QCG z!52B&4}s%hQMt7jHIT4xu8fc0^#b7VkR_AyZOw}Sp5oVX2-L%$c4)y>7()B{5>j~Q zSq}1+Pri}ArTkQ{$I-mofT&s~9>qeO)9E-UhfM}FyPSr(Sk#c~+=;kON2tfp=f3uq ze_1r-698Xi44vmf#rH!uk+6a$FtY$R2r&2ZZw8S9{xK`Znw%n6*K@5^*0~?r6z?0j z_TXl!9j0sB%h2y<fpy#yojo82jHEtty{-Z8Oy)auVAy|`nZwK1?~rf^T)x6KIwZ!4 zBo`jQaB*?h0frn+y@fE;Gp)ARb=zLNVm$k$JclU3XQ|?icWRuNThtx=O-=k-&^3p+ z9AL#C3%>}gB3llJ#}Cic$ZR-<6`jwP*I@2+d{6-bWC~;?eiXs>0pgtgFv4xyvV6jq zy>)#b>PYkfd)-l7m-Y(?`7nU#H-MfFrHvrrPKL#|1v64}9biZT0gBWDm?<VEQ#(tb zEl?ZIh1tv4QTDjF#PH706a0<WG~_#%U`3GO*3Iv6*tD}HI8nVekoJ63kNZjwKJPbB z7Wr%80fw%JaLjmleTsM!`<Gg_9rRZs03F=)jz?y^Nu|R%&(LWjTSG*1XVyEc6X%b# ztA;d91076i1%0|8X3W+8B`kh(q&L^D%_Uk{H1#SC%-~XNaSl6t_Dc<k(ZVf<*r^=$ z6SG8)w30U5b6-kXbl1leD{j@s0>eZFgz^!X?CAaO!M&&wt6>vy!vm|W`978f6&YH} znyD8|{yYtkaQ%A>*ze9khF!ZAj!W8m_R&hjXF8EE;`^Q1gf(8fDFr46?*v`m)}{wr zdnG8+z!EZm;t`?KI(|tph7}|4!{-S!eo>(`_iG|w|6>F|1nsNU`C{>`eBC1%+Zmtw z0*z0`8L+##{Ys4<Zhm^8I!Ox731MjRhV1FUrv@xPs71=}kYd`T3DX!iU$7gLk&)5V z+8-dA#&;+%hb-eVh8oEfGJA6!<&HZ9R#X=mHjor@c-_0V_udoOM@*NT<SIkD0<<ZW zfU~qPd8rHV1unI<wM`GiJ<Zi;9Jb<;J}}9BMn;5}l(V?^?)@pmNg$Ah8Zo)pOzAwD z4EeZY{6e_<aCRO}BWVlre@nVb5|pPECH=pX+C6$S3E<)%ezcYftC)q{Xe$!?JzQu} z{l`W-pYX|b#>x^QdBr%Hn=OwFtNq-;Cy>`6s5UX?^fT{5Tt|)KMt|RG?&a7~Rc(Mv zLH}+-ILlC8ZR%kvb`}7wvsoDM-@|^e_M=B^@QG$yk08be=$FC>d_7f!^*7%KUbq3$ zvPFN^%NnB^NWtko|K<tkLUBJ@ANZ1ut<c39+vL(EP%Tk!(;y9IjsOJraR+i9nARNb z((ArM%5NSoBG1gs%+j*m%xP8dAi*d02U4v+`UZ)O2lA7tK5zVxl0wkVtP^MD+qc(A z6$(n#@@f}0dXOi!`tO{zkN)@5uMs@SVuem9m992O57C`Bkd{U^1k{Qy-hA_jtPvRM z^{?MRdH=OLM-`ofz~XhV*fx)p+qKbc%7=3aV6PwwM-@RkEg`=BbLY-Yfvrce(#nnt zQ20ktVi#ro$S5TXf^bK~REV0<;r_0PNhOIjSgW+{Q!A}i5{A!6UZQ1%@}6;jmX=A_ z5nqR>p*nLX_0f<&aA1~SuGzEZQ_gXr;?}cNXX|`~24osvH_Tkx6x82tu>A|bOp{`O zH$=Bypmz?*!>Le>s0BnJ1fmP>#5=AFwWj4fOOPwoR_3EkmzWH^(OM1|S2wM#bIkVm z=Sm1FuGaxvOCj$u$%F;$Nq?dar`fPW>Jj9IfJyPl3&z8_3mW!30!)(U8iIKm!g(93 zzlb}(0t#sgr-=|>O@OaWEfOoQa7duRdWJG001Q&1JI#`HiSVal+skr`ozq<$KQjrh zdR?5$@>WT0r@GtBC8an^IL!VeN@&da<E95lS?eG%2rVXHk0AKO$OvR1)(XxNLZFZ< z(d(O^P%~)V#|BiBd_H>!O>5HoR8&;a%*!^_Tk63wD_wnf&q(QWp6{iseQRiWN~OmN z%7Z*;dxirr6NDN7nE418FEj(^6%M=U)+;lR*tcfbs{6e=F)yUeZ{5oE+{*k~H7)tc z2G~8k9X@L|ytL`V?#22|n%<;l-n)imQDU_Yn+1t9Jv>@8l>n{3oJMS)H`xJNVdNKl zV$sgQBM;1GkVQ_pv~}%)W2Bf{$y_GhT{E)x0Qj8(?y@}WT!74Qbsdmyd`l|dy=%rf zX~yuBg;=``m?5`bsr`LRNtWr&rKr*U>T9WHbt9OCO@I{1{r)-xA*T#;FKx;LG61^O zkGxiDjLNp+<45NWCrvUcAa+$lWm_C{kDLwp`uvQJSA$5~GKL7^rR!4#s!+VDU%+kM z@?wC=Mk0eHSb;2-$1Rok?3iOGpRr#8BaSL-LJQ9FShWst7r7uleu(7;Y>ra}x~{>N z1XN6`&_DxC((4qmS<#=d<q!tUxuv`Y<u_$!Aes%|#*zVc_43Xh;HLG3<Q{axp2Wno z>wB7sA-r?3aI)LtswMC-lg-}u+`LF;9}sYIa#{^#dvqD#AEd}(6Ee~k(++*st9I2o z(y!r0-smuoXY7cCXN?!piP?mk$InopZv8;Wx0z`{mCu)eq<ynNZjbh&a&<kX^3EW~ z{^+C58;NY-=m6mPxrYm4Zvu8?&0UFZN8_Nj==*ezh}nSdX5wN<T&&cy$Bdw~Bkvj1 zkwp*~Tb--{>S6p?qwQ)9qf7d+wSIQmq-LP)ioF)H6b{G}1hClx2BDw8Y@jpN!Siww zRi}{q)cd6?eeV~rabadP{&<^9yo?e|_ybHG!wpM)DTSD&-a=EQy}TZ!Y<bfIiwBeo zesZ_Fkv@%Jl+KTao5>K1cT$qRRdZV^*^orv0}uGy85X693uFR;*)E1tjPyVmcMVj8 zE+(j6gey{>MU*sVC?x6rnv6fP^gsS*ROQ{E{ldEU)0@{T_~0aoYCSVh%3@-&<I@jN zX(D*VrxGq3CFnr#+rjMDuc(Uzn!}f7-JCt>e|+q3=LQ1H@lMtX3F=?C#jij4`?XGS zlMy`yWwB-jzn9!GK(7HqoyC1*Pue}(J9bq>Nd}ET*o>#e=TE+D?EuByi|Pq7E9cob z^8ZuO!T<_Hv<?e>+<+ysL+>T;Z2W^QZqE?d^WH@fJl7dQNJ;;+`mOx&BO`pH?%S4Z z?S<WTGooy8utXCA^@Rn!7_C!MoxiWiU`>9LpKe;m@fWzLTQ@{pSM_|nJp1aLg9^le z>I?!!BHE^hbSRF~>El1R_XT;T_QwwpYV+jJrAS|YIO_0oC$M>Ym<`Dax4U|5g$P6F zZ-XMyIk&ND@gu<fB_kacav%+=B!6*(p_Ar~m_YWM@RiWFbTz`u7ukX3*(+ow;PLp! zV-Kzi&pp3^L$l!1JxeNUo1ye*BZBlxV8eC30%Ss#ePE3JX1P`;#h*f%m#0>>hp~u6 z%4Dnd3q@3tp`)uV(B&kYG#h!0OeLK4reX6$30SB%cu_ckeLy%azor~Mo4a_xB;~4I zW=r763nbo2oa}gPh5imy%=uiv4u2W`KHyENh-^af)r*{**EWu_F-Q3TA(4GuSJdQl zN{uY#k1YvO{RF6FDY9qMvBq9wzy{=o#=7Uy!Q`{=>OL!-%c7dUM$MOzoGDedUV~5P z-03?sDV4!aFCMG-%KI>;J6;1QWq1voJ++6BDFEoyL#=k8#KB$M+dnV6(vz@lNe=il z9AK~0@}=u(R#7k~X{e|0^#s>fTTYp84m@esc+CFjx<Nt?xTpn=#??Qd(*efkMZP8s zfI|&olfx8gwK-?<OnbF;PUTlsXC7VbbUAj)7y&mI*!JFcqrPk9Ab#pl;0nTlBxtfm zUP#QL)z4G`onZdCCHwWOw;hiCoxywfg>Y}XI0DXVKmuubbE17lF<1BUAM)S3Lv9tv z$)&;Dc?950)vC(uoz<;MonRgE6sY8Y4_@aNnDE`}48u5%))rw)=b{U;bv7Xr$7AYC zG6b`ZRiXWE-(!cPX}joXK*e|YG^i&#i95M2siMZPA2B0W6Xk)w@3C@B*%C<8;dx`7 z`4T#K@f|CKxXdbp{6%73=8fDwu{j$X?GgJ{=GU|Tdr*WW!5%$O@4%UAC9vTd-#=!@ z>?ujGqWvEgXPJV90yg)J*#J;GK@K+gf9Nsz1#yyrszcZRhkWXPzY!SxB_)BJEC8!m zzA=Y|LZT2wP@tN?VG@Bya_|^gCic0*VZ6K4-N#oAX9#C6R;f_PXEiyuTQPA$2Dwx~ zr?cqwd6Nv4hs?WkvGabml;nRU;$#Alg~#eIJq?3E9%%^VdBS)OKFE;g5UXGFJ^8}9 zitF^py-bi-Vvww-K`ocP#?1ciuo)RZIb;A`W@IC4jt#DQdp>x?y`Nzs;5cgs5JF94 ztC~9NVVmo6dMIw{dNs)Vmn;}$yen6NP5@aVB<T=AT<89edq@DQk&oM#YfnimsTIJ( z3Y`E+_b1wfRJijB2$vsKW5bRQhl@9U=G@%Tt~akI*r28I1tejNh-iF*bR?LSKsZ{> z(9;dB@y{KrI|KN{b@kKUHP6Qt$h>>#h?lMb@&>>s-Zdzgj&cFS421)~22>IO4NEke zzc9joKNIQ`p>s7IHUB%LyEuT}eu;%ZTy!x4sXmvoa@Nu87kWDC!?Wh%$4+)1Twi%= zPrKD`iRO^drwjQ@=)-BTUvQY@N^ZgYqa^x|)7#(12MCtV5=HK;>YxSwm!J}{oeTJ& z|GxP@XFX5>Ry|kpXHNmnql*AR{bVhrh{LOA&^UXY+C?+}McSV3V3B!n6)17+QEme7 zrglPp*}KW_fgK_fC<IP;Ze85_G~btLyjB>{99J<Kt6mJ`EKo7l2)Gf&PW!dyfpdE~ za3nKc9w~<ks5gO@W6~IqERNZ8@ZJLYCx9pdmYk7hi#aeXHiBYoaFc)TBs`XYzaKw2 zQ7|yD4|WJDU@f4EI%ZOL1X!hk(18gUt0T<i7Q^FR=h=31k1)Bb#85j24!tVjeK9Tk zcu!1*X!-YR>;wrVJ(t10)p)eRp}VuQ(_ByFV+(gJ2Smn)YQiR~Uf{I$_8L|cP(;ja ztWUKpEne|RZDvXR9C+Mn{xAc?l#L}pMbVBuP;taC1b%xW-H8WQJU`@A>M!nrwZ?Z) zZ<Yt<inB9iw%=~KfU`^3Zm%7v!OuI=YS;a!{o0O?$(^L)3IP%qfeXfIg=De2kVRWF zXoPW@E<na{d4vrdW>V!`vu3tjMcH&LC+7HEjvLWSCm=>pK|eQO+EIfIqntt#C)npQ zf&_ARGY=?pevq=SUg>%s@4o&LOSi(_T$tbs)TgBNYOItO=nN6{4TEuos^OBR#WOQA zEueh~J0u90oStq51UO04PhL~(`sAb1NYgs6Z|BAXf7m)<fN$I-?f%)WB&bFm23&~H zL}9g{Cr*LOhRIc6@;>P{z89AvQ%c-P3E$jr3}TupXyFQmyz1`ugyO8PuP1{>MwXmB z26eDq&|<|rryirjbM|zs`4K2K-zid}V7V0-$!8kHyTq&KGDYbE&c!Ji8FHzLu@8pS zZ}%QbtiQ|5RH)^LA%ja4g8S`&`DLUb@o8tEP-#=g1||2HL2G2~f-e@6^y@%Sp4Rb~ zICNfft)?9k1%z!?H8m!vGRdaqRS85sZG1hPME1izplsX-{EZYk-$VXAGITCR!raF* z4D(OUij0pP*f9&m*OwV;FoDwMMQu=KdwC{B6z4ogXQqD6CT-04(wUN?qKNk{oU-_9 z0!!fB1*v`PfH@(3bO(}w0W|&K#ok9`>f%>880h0C3M>uee<XSYb?i1#uFlI-hRl<; zec=^Rbx{HqlR#X?zuN%Ga_~v%Eb#^fgpoZEYIC&CeZ*Kn`yTX5y$H_1ad!bv`|H$} z1ZI|r;zh!H0tk(8Y4x++cLas`7Fwr~dyh}>e@9}&i%4i!OM8IP>8;AmJ*EIr#zNB$ zK_ATaR-+QJy)&q$PXR4N5a6{->JjF`5)S$d9)rqz<N}RR;xG@?Md9r=oTF`<Ba;$} z&R4X9v+s!LL5K0^b<iHhN~_khdUm6Hxnkj#h@$we(Jm;(-XfEVZgu}#>(76tiWgeQ z8IMCk+=^db06=dL=V519^BgMiA>ZPDxCYy{?42b<An0QQRzR$~1HUE()qKzKZj7$g z+B4#Ft=WiSNKasrpogC(0JXd@Q7>;dL%(;4ZSsQDRZ5=g6nuM!pt}}73L9ZKu81JZ zDV$%aSm5f4+;G$aZ9^2D1#73p0OE0r;XR}U!26P>6mRX@iRDVn)2a0xk_?D$^{P!* z<#3TjLNdDY`FEL<pspMsIyz?yjIE&R{It049VC9c12dqZUejRC>w)^-H&bfdy`bH( zQ+wpm)pyE1<;iw;%vHNdphn_<4@c~ck)(EN9Y9*A@~I~jiaZBJr8=G0yohnW2ONDT z?YaZ+Z6=ABZA`7MSh54AupKs_i_IfYDU3Go%k?@~*fthFE>i&ZErHfBq#&fFY_lDc zgSImaw(zeE-k&JtfHv#PZPc?Nh?V77>9WwD&1XDoP)t&@U~mLliLo`Dw|gB~KGxZ3 zfEVtN;i7=B8&yQS(hF`MDpdpGzP3aXZ+gdpdnVEq9@&K*ps~SVnQfBRnemy8$i<;l zV5l0Ivb?nC!2Ec9{n4Jq-FSqBUQc+<X3DAu@GNU8A<+LmgR-_wkn>>}lCKc>02Og6 z?$)+Xpef1hMSQTCby3$e7&`kXd$nT$5b2qqL*lGZDq99>YYGM^ynC}@FbvW07_}}1 zfmZ39o3od0f(7nFh@KV*!;*G8<t<B2VB}ku5cHIHKX7{|Js^#=!m_IUI=%42gvd51 zJ|c|g9A54L=iR8c_SBf^>FOYVpkN@N(g)7C{tn2>Y9oD{6>Ybf#Yc-sxQLRRoLrfw zSQ-jz?PBFf)re?*0MlL~_H5LmMVGM!+2d4Y*`p@3En<K5qw)L4xSyAlyfA7F)Y<Gb z{12hp70~jSsk3x?zg$kGKU{sj4)BJ4%bA2sDh{oJ<VvYW$2IKyqK%bXOb-?)dY-C1 zosEh!LH^*JTne$$1ux(-1YU*LoDUk=Up^{E4=z<)thJ_kG@p?crFHb~AxJ~=M~4Rr z-gO`YK&nQ(8GwnAF#RagIs+r#uXbzaCnqOgaCHDEAOh|EWM|xKT$~v~P2|@C&R$`3 z&nfhY6>*AkTZF`7(|IxhalF87({3MNse|ED0QGo&bvd1$1}Bj%OwcSIl*c(CdC1Pd zaNZ)wbnfW!UOTXxFcI+q9>&9Ob#Nr{#U+PlJ?iW0P5N+0<E{ncne_JcMH=o2f$3B# z(%E<F45zkH{Pgdol)tt;dRQG}JF8dAN4==%ZTco5Vy7jRg=`?;<5`mr0HHXr?0}A) zExP@YOQ4NQB2=|YeGM&6MB59kJD@-nX;*O@um$x&@^}R7L7RY9)ha9}H$9z4er#c@ zD@o>6b*rbS-&Nqa3F@+XK>$eiC2>^(`ckf%@Avtl^-PNv-W&u-9G0=IN00BDv`SeF z+nc-VKG4@->NMUV75o0Mse?Vxhh&~9m_^|Y1c!E~DbW|cs{;$G>0?s+Eo`@XA85A% zkaztq1FTq;+)Praz5G(iSJ3Y3w5|fA*L#a1XIIY(ENn-#O!iTp;rG6-q1hQqX%Exi zFz>!|wpU@pLfM;zwig{rI#d`b?olCnw7+38jOE}30-&{!M@YMN6f88nprH`8glCl& zF|y^t?^Vh_cc=CWU_0706uPAf{9yoTwo;wI3HtCTp!Fw2dyfE8{fS9R*ZOZ;5zO;T zgVSpz`leqO^foI3s#!Q^Uh}mFf@S}q3L<?Ubo=vd??z88I8g8XbS^Z@9cNky5cMAm z7GIA#lTi2t0C)hM=bM@{s3aL%SUdvStJeMQP%OxwFcRVLu8Bf7^(ysNG|6Lq=dVv$ zB9H)v+FM*%$-YtB5iO)-T|RZD9Dux=nbgU(Af2`n9Rjd5+HPxWix>Db+hrBbdjZQr zS%BRfHTKk@z(*A6TqenbA%ST)faChlTg2y<Pt%kiY!4d)3q_WQFMS&N0v_GGV8#96 zjaV20waK%1OgvPmvI1~GC7@EZY3?`N>OvF|LnT$B>hx=HZSLc?l*j}NINh4;wS1*@ zLd_Ei7LP{&sp}=#D(ITWoXn@egg=Bp^b-ONkzhj7#@h7H8k*5FB(SWWMOL<3uCVs? z_X9+rMVwqNycr#e?=G%iR)%CSc#LHqxv%NT^3>+Fa3{+4Acb$@ZyKY6!v2Q^Z?=np zi!_e#K&g2(D2AJa!iWPqD@15Ov9n7l_G_)xu%Mm3Cr~HeC@v}StuU8ZLgKl7(ZlB8 zH}`Pm>@F5NXH{B~S?yYMKtq)99_u355GSyQS?(8I1Ax<@NZSe0CxU>N{%5&n@5sxD zCQ34tz^Dqj)o(IbT&;x{z!mbAZtsvdc8xX(n;=>1kE%_rj=3k%?Er5koDVBOB&B0a zyW2G~k0d{WLeA)E%_Eg+fsEEDeg{okA^C|PM{<-A=&XxIkQg1^XZALFpe6_>p-9g( z2dz<LUANm5%vu=n6GSeTfo?BJ^16uJmUWp+!~}Vz7hlwzS#TmY5?1E%*%qj%)y^G_ zbH?j-SoH+ra#}$JkZCw^8P)9fhB7;><tIJ5to*A|muzxl8byDRVE<jwu;)s+MgdGY zK81e^dZS15(Sb{v1e_6`!7PF<-U#I-f0zz=$*GoSG$m#yY5nb}_gitos07*Q3kM7X zfjgX$qvYDaHI14RNuc`6TF-dO#e~`W`!b`@afeTEZ=V<qvOjYIhu2r!;n~;juQ7sZ z_eGgHAXpxQIr@s0bU;Gw!3%ZWHwM?v|HqM<57$11vrA_kQJ4m~>cnsW=M6<q5acMF zla|As0Aa8o8x8*d)mR`U`2Mz^-%HL2P-FqhQJeviU)DfBB_E7ujv6<)x`5r`amUB= z%$>btr*oagFOT>?IPV8Gks1W&c?cL>Excz}a>(8W6Bq^60k4z~jIbt`1dthL^S(4% z1Yc4jnRz8j5<mIRt>)jpg~1Tle${J}ocTHd^0WAL8h=25gD%YODtifRlQ@8OnqLQN zk}{7%_AK*fpGK6sSV`@0{FO63_XI&O`M`lrfAF3-*Ke=!uV00A5>m`pe5=ZLt6n*6 z2fuY7Fv>+nGkE8;Zmu+o4YtU2kUcCx)?=g#Y3m~7nA@$>KWYm(%&S;w8<F-e$kFBn z?B~q9tz9T@JWVTX;05twnoPWg-ov&{ji`^0M`0x>Wk0(^{71@;MI*JrDwspoc#mwH zvLjNF(E!KSZCcjz9+{so?sBbH&0uXt;rj#4SAJX#ht{Oq7RpE?18+Rs0rX;r<^lv! z{Pj$K{|Wg37tH)8cVMa(idGQMIRU6Mt+<O1r1OwEq%{|SsTyFW8{aSN88e|xMtj7l zh8EQ{*1?G3dDz&SYPsY@f$!3;i5++KML34brEc}{fDHpgAPK}lk@&#QZYn(%lwu4t zK}EegV{S!q6R_>ftxxndFd2}3A4b8thH*csojLaCjPvFaSwRUla6{s_>Owcy7x2=h zDLB;#hfFvS`TQCTzv86@_@+oWn9j*fN@PeAaj`4xumN_QflO=;%{&IR${Ur(Y_?HZ zV8<yobZ3z9DYG~`0mDLt%ySS%H*l(`yA`9rtUEzAE5WZ1QeYt15>j7pzKx|zZ2(j` zCa=7KU8V5@A+zHqHqeRul%X{ER%z{6&XK_*F9NP5ix}sSFH#7X(APt?mq}(|IVn1j zq+C}6tCJJHxP*1Sn<MHtQg4|?9TPBV@Fc(SaQWZ{a21rTMSj1eJegoHX^V-LJqd1H z1rBPl5aC@?lYl`b*_>0R6b~3=0!PRN!-Pl2%vT0Lnuf$F4%rg!{t%s)p$|H3+}60| z0mju8D|bt1Y+-{8JJ1!A+?S4mMcMx(2>!X>_(yzU08Uo*oV;vPf<PNsJmNmOKj$XH zj%{x*9>xn%hBBzm^1tKR3RN{!1x|zB$?Dg?9urK!i3(DfT>JLQuvwl3x-TvGrwicd z|A_oyf;Uj&ZdGD~iea5;&nQd&DWnr_`Ehl^*%}~E^4A#uxUlE^gWAZ2MS)ZI`p|w2 z81w;yrXNNCT6<(Jsyf`o%u0}eCv_DRHCS>_-8{Ce{zC2#qZe>YyE1i&0t$B`R7453 ztaLp@+0LMv9Td{|Kd|N#C~WqKT7zTm0ynku;c!q^HbEUchG|kVn1}%4ye*KcA)D#2 zba9bhoFJUt5K<t>|7V6o5uM4q4@=Gx<@Rwy_J9kW!>NiT#yB~Oog<sYhW6Bzw8$i# zis-(BBA_Tb=MCDWm^$dsoj)d91hPR^JHU(0ufULf7|W=cFDaJ5C<z%I3Y(x_4G^%& z+ASat7ry}DS}$EB*sP}syxRVBArCO!`1bTA1Yk(7<6W9!E|3+xh0Qya-sJgWC+%Yf zO2uWW37{IxjyuZ_=Q;(#1j$br6ygF8@;O3m=$FC_aJ^gI{ELpUCM?i_ESp}eNxl8$ z<@l!{${RfssvOAb&EM&iag+9Yz<&J~ukf?w$8GpzGraL&&M&ZJGp>eS@J8yOu<@|$ zi$c6)W`@tS34tPhRNZvPxmHJ*Axo%mwz<D`N2o@-HQ&Nm9LmOv<=t8k_MxlgCA=f6 z6UQF%O#x2#(iu3;gnu{{PeiRYyZE^q0s3NVqmFD=#J^l*n?DRdKN}u>_3(t`+sD3y zzW4FyZ@Wv<Q(l3I<B+4&R8Cy>eJrcmBqt>)Nk(~^@>Ya04(^q&^q*vyfnE4raW3nz z19Yj0XSG=MD4&>HVI^N>yleHci6i1pZDqx~ip&bcJPz{X;S;gXi)Z0M$c7>l^vLy{ zj|3NI!l`P%*MEKK-bQB=Q(B+R=j}1HvZkc)iu>GA&hDa(P6Mz6y54}jCsM;z^Sie4 z&y>)c=Dl2(iH+RroyNRo$iEIl$z33GfHzi_+Mj7;uXiWtjjvM|vG3JSreTn7IA3Tm zuovBHW<$I)^mKB3Tj$Zv8$Zl;sQ~8E%#6)3X$yw>Om4QG3pc>={(M#M(`nSp%YlO@ zygwhlbZs2WZQ{uDbckPApnX_aY8~B5BrxbCFs6g;si59?^O)I#g+rrn&{>|Q4j2_| z3f7+&TTRM7k1K71sWz|8@LKA+kC!ez6XSDa8%{sRiY<HiX0N43)_;-RpW)!);g9-W zsu)#ccYE$R35l0@KEpF~17}b3e3n7AUEq<1+r^J9<6+!|B>w~SA?UrE0QRC{Rt+TR z2^ly8lySc?ci!|la_%bB#rdpffXNEWA7|D1sh9g-$-$t2;CsnYq@8N~`&bV6B3PY! zQAlT6<d}2kJ%-%V8yA%0L5KLEDY`v0%Zv;Y<KL6|Z~JGYA1)pq-bat`DFF_B&TUzc zp<W|5PZqnX`VFuC6|LxZj)z%Wt3Wc|tC)~$+PURODl9?spH18zaDJixurN38#F;(v zB3FQO0bq0c@N6H^Q!)Clu9BNQL@mYh6~Eu{o;^0?7?f+_ZJY|rTHH}uwd5B~z)-W{ zd{}^GF~O8plza`_6&3HJ;&QOi#Tj0wPUbBfyXp|U&hcLUxZo)Yxlf+uMs=P*o-6f- znUg2lA)^RuFLXM?<*6LYrH<w-t6oqT9v7Y<oR9_v20SCr^=r~AvF^*CCPx~)B`g~9 z&q&m##WGtJA{Qr*aE9}hSUN2A%z6l~`E$7aQX>BA=9Ed{4@{#4O8bf(ih6I@zW^>1 zw^UK-6y8e`v{<(uE8(!j)y1S0D^$ha4>&fy$<isX#_{W{s&B}|etKpGKyf2Vf?RCY zolUQDb91w(+9Bia{*N{U49adOffm=1bt&Cq{VNv72NeuUV!p(V{ftwkK#N-^v?zU6 zV6cbFTI(i3O{WYdPGsr#%=G6hMmvjqraQJ^!Jv!}e_-hLKq_~*tfZFY{*BY%Zwm;E z!rUIzjubT!e)eSQY=rBq41C4IpPlS!YzQnnHVk@my@&csytZ$nLojEp4?agn08jT% zr_9=hXt!>^r9g&})G~EOz?alA9Qdb$nEL-U(!X+;AH$bDSegGErytp!3E+cBp~+Av zrA!NR=B&dJUtb(bKCQV~$FkE|eU4?Srm*MO>kzyysY4zal#$kcf3J{K&DIY&Ya0<J z`{VGuA%{s9+%n0E0N<I!*ojCPB>s*j+eL}*%hD$moKQbrvRIc)sqCh;aQY~hbK~lW zoW(~_NUR$;k?aV6VH1aje-ilak2E}Kx$)Zn^BC5B=o|m{d`8lq<likhij5Ph_BWL6 z1;fruK(z1-G=vHMnpYSQe8=Ehg?Oo_!no0>$M$coXn)Hxs4mJ?rnPzT{nRZ_O+6@) z@hHNXV_^jdm>Iu!R)TRsSc~>EU^HUjj-$>wXA9<!U&rUc-~8zQewP$?4eClW=EV2s zQ^Y9Tl?KId$kI(wzPZ@xGmR8r4Ng}yXxUY}tFP~v7B}DjdE}VqgDy!%Ww!3s>gF14 z5&6~zB2t5!c&i8tjhjZB!9u60<8=)(!QM^7pnrtT#nf2*_z6vxu9}ch-qbc$k=>jU z0d%Hh>DeHM*2$jRk*T<A<%9wTpGZ9v<*Uk33Q~vZ*x23Ury??e5QRcrE1mqxmA3bz z?V~{zPE@9OBe*GWlV*2O?ZgMxJ($DKqgeNk_X0u2p0vD+o~aPQpBD|2B*nsEDL`%_ zydO9fM73l0<wWm-F}KGl88x1t&59`g$(s8W;r|#jZ;S0*Z0G6ao>&Vwq!K+Vfm#LE z5mLhUeo!AlkHU}}AOkR0&wTn{UE_a*0+{Vj_Ew2<#MVeaB12-IxIKEpn*>H|Z~yW# z`~>O}-MoFkSnZWHcCY`q4J0R6GoiKR68+M<`gu)!D3%ye<h3LQk~)SLz<p_;GWfLb zTIlq7czqUVw*;#GV+I)|P2TyjsK=zZY@1}cUa0vBQyjr9Nz^+87V9^fO~VKtnmWl_ zgo`~!WD&Kxdqx8a+n`b8ao4gXUsW)-9*#}(Zz&mM1vg~?FR#2%f9|l4+#8Ii-=#dq zJGna5MQLk{im#73ZGx+t!LW)GK~z`tRTGoxyoJ)r9R{aM-Y*I3JTLG>;`HatI%LKr zQ`8wDrWXO5?0?T%Fw<wRYL7nGFMVgTDA*hZZW{FjnS;jqB!eZ((E+SBI+5^=QY)u} zgV&Q?$mg%qj~#LTK<}a65*O*`7Uo+kxIZ@*7XvdXklG{sF(+cNyN$jRqcD^ah{6<4 zCYjb_h6j4GhtSf}Ht^_En+<WzjwCpX%}XUrEWpfWbHi^)iINStqnJsZDrAnzyUc|d z|8ujWZt$?K{OM6?6Q}OLongvCLGW6@bSa31&BBzQUMytnguz`Y>yYu%Yi8b!r=*U1 zYhVdhPN;RN+uC#cHN&(8vb?ktuzxP(1Mp!9H*LceSpL5}CH{<r1b^J9{coirwGX9f z8v@h8Vq!!lOA7^d@Yha|AypD8?dlw?XX9U2wu>LTDlzcP!(F%4cfMANlv7E}4_Ja3 z!;AaO0S;~@m6C$Y%o$}I;80#q>L|GHnfD*!;@%{0=`Z~r-zi_Y&0lf-1dcE)FOe4R z5_2#z^cN90g_6PaqPmcSG=?3?GB#H5AEd6=*+{iuJDpYw2dgDc!^(RlP?wWWIM*-1 zwr)PKrW0gET0;Iub&_Ne)eggw1H{|^;l^EYx)Zxp$7)<9+?Gid%vcwk>n##Yuv>B1 z?Qx3Ms5?sA!qumsiE>Ot!{EhKs~w}y_Es&z))3x93kA|(Ps!d{b(I#z{}}4_N+V<N zZAiS-+Cl%DWJJWV@eD=&1Cbt&?|uuIbFBT6lwg3OkSzdhx5pHj_l(GjV86C6e~H(0 z$k0gu?=dMU%Td6#K8O+*vS>ezF3un4bV+W>Q0=}_U7xMp{Y_oGYbmwu!CEeX`J-2d zg-s3^7Zc0Dc|p;>nJ})!N1WLTIoU~9-Sw-B8fu5ld>7{E9x93U&S6~fV@vW)Tmp3R z(}#<dENrvYqDCh0MLw?c9SDjDZB|}75Tr!xZ&G@?4J~hSei)V;d5e$G-AO%M-Eq}d z>2Ac;eqORWnWTGf^j@@Xz18-vRr~7j-Tno2j^*vWuEiM}65H{jn(?-n(Rp?q@nz3J zmRhAHwdhf>V^d6GG#je(wtICT=(dkZ?Rel|p|Mpl72s6Kw$yeZ$3x5~rzcltcyams zLwUMod+Y-r>v!X++xo-3PUWJ@)o-nGmPn;tdv5OahUF;^o74sr-f}&tnA+;~rK3+< z&I?c*it<U>uHHQB!tM21oojBpAw6yK?3{2aEz=;If8n&^iH7Nyd2Lv=ZR)iOwebk) zu_S@~Hid<~#}4+d_r>SWwy|E79&0(O_H@XVkRCM-2-?<=p5N6LmVRoVXqfHK#%HQ@ z!WvQ2Udzp%d~!Hd$*p}y=;Ykn!a%Nx2H6JlCZ<^HCBl`d9W`^)rjNQ^SKgF&+uOfB zq?M3<%AZ|C*Zovtx$DaMmK`lq%2FVg`kC&h-HHA|-KdE~tN9qhx%+i1rcGBKY|R7( z?0F$RmWMgM4huWa6rXw^7&=Wj_jdCv=k`oQ%1M!cUMlY7leGct4s^SA8dC-Bqg*zL zhN&C6=84RR0K|byua2<Os~MH;D?^h3Dqlrahx)H{i?rEWFK9lq{~8c9k+>l{^P;;R zAx~v>II`6s+afl#@$}&`#wBjmvqV88U~$A_gq|tIJeHm_yX{{2bl8V0+S_GHJqRZc zz56pNyN0)AsR{_@-a5bL&~3aLQ|TGaOWe?yCO75Kb{J}fa0=hGR~ay|5Ei_UTWC1B zlkt#lmw##Jj``<x4H4;^wJG4U<~C=$r9NNj-sbdL@>@rLD9<$1Xw*OHqATiEY^PzK z;Gu0dcjfZ50~oM5jlI;(tc_VYFFLV1x2VF<%mG2&rQJc@;U$#qx?We#bw8bQ5Z4b{ zA&RdhVc9@`a!Iz&v%a%_w)`PowrfEIdhYD*x7>Y)?^iA>P!&!WRUD3eC}%G<7gs8+ zYH!!2mC$`cwXWf{j5#RbRd$gQ4hTU@HJBwh(}hkutQm&r;*m&5t5nExVplI=NAtEI z>-*YAt>fu0E9{tt$UHR+xr0YC<<P|YFgE36JPYA&ri7CRb@9`6^i5*}@I-ST=83w> zEnH&TN5@c-)S8V8L+&o8xv?Gg`!W1I!+d1(7oYO-c!?RO6K&4qC{eBPcFwb1wjtaK zQK*TVsCK6*sN&MX3+q?QBFeg}l)qt?PQ=l#)*OPmu@%AwKCwVVy%qA+*uY$@e!XJd z*x;sE%601WjmGAr#0sN=xVOA{nsmm!YTeC9+LfDIr$qvuH6!D>+z(`p6Rsz=2XX{N zC9TUgNa%&1_k5iNeY_~{O)e2f>Z=gmhv{(>8wh=&n<CO-X!GHx>*@TtYKWyB4_9`y z8iyC{yEIn){wqBaeKyZRQB9bd*vfbE7{g#ayfgD}5{Wij*u9DkL)7uq-=%%pK#wiI zpAe9~pqoXf9{w7wM!1FUHN2fqbpBnMrk?5C{q#81TU0fA;dh`%^>xwB7ghC4wRoK_ zdRYhW%tl2KZ=Un(XX_gJN-7=_GSp?@#A5BSes<SR+U4FZZytXxyO?ynJ-IHv30m!~ zy~@aRr-MPuYDIFW%$loj?n*;TlDlz2Ez3zXs=g}c7BtwFH0pTTx~emQV4z%mI~Lr< zq>TMj*$&u=Lt`o(>7Md-=3vWjR2Wffy3%TvscX<OFKBxl&}624ui~01#wb$@+h@n+ z6193|$B72KU1#+i6<yR8tBiZ&NSj`vCrjn1VB)}}Pk)EzRf*OkT20q%x<2atEAty$ zA=hIQH>0xluiOkn-PlnJ8CS0%*>us3v~Y;644r3VM$a4PG_8lDKIZndOy!#Is69ig zJ-VB@XHgJ&h~d@3yL@yzGBTHFaUDJXbdRATK4niyB{tDnLdw`cO#nUrvAOP3aYYXH ztK9`P@m$p*4eE2tq~G_1)RpAG_o3U=m8dINf-{;&XQ81DD_5si$HRz@>T4*iZH0Vr zFmR+cbJ5)weRmgpPQBEk01QA`Ja1JTBX{Pxrxsh24hFaJ&aB&K%3mf`rsb`UJ5{OP z0bW{d{2TDz*Ec7@cP|NItmY4;*0nuc_C1^P3_~7eTf6A4MP<2DYo^z}`&zC_on8^P zjCCnMWvJM-mU|)enFBF(bJE?dkO_|_XEA1#aitFZlj!+(!x}khn$((Y2lF8oTN7g3 z!>OYim+#_->1E0|Ji9L@g`P+31aF<zMb8d7N5>+r$0ayWKzq)vGXq8DYK^&}rVILs zeO5)K%^2V%$%S+KJ(^hbT>Xtu>Ct56Z`F@4w_$XI7B`i~9TL+LTsBL&kf(2}+-e{0 z{~Esae)^7K`C~4<-m~u##*1ROawzRp5^7J{J|G!)$WQK^DWxo-&Q2T8&CF9AHR>wz zH?RMG`kT1&;$3R}_UStxeID2A3iVU+=h9%E#`7I6Dq8?P<L@WL9iCYgl1b<)U$);n zN9t4Wu)RaMEiYtF*(;m8L~whLresk4Mb_nR%y3D@(jekac|Sj`5Uv2(g3VI!<C;u6 zpU3`whb4x+OX#Fp%G2B)8dKWW7J9}_63*#VXyx(<cb-nws>ri?px2f(WnU<tIKFaS zQ>m-^q3B63GfxGJc=gwAwi0c!ewx#9uPbSptUmh3r<~j<I9S{=|Ii)$y|~SPuB7dQ z=aAr%=9vk-7y9uM>V#sW2lob7Wo*R|rl&a+7GhL71ubKJ>iNamgm~E`^`g*e>}?h^ zL9uo!Bb>V5v90&6E{a~3?AZFgm3~k4efRw$I(aHjJb8V`cD7M=lAFVX%q8^)mhsvH zvNpV&j+kgFHNqat=rAr01@`RA+C$H1UQjNc)r#uR4}0}EWiF%Vq*pXrIck}d7!`%? z&hAe$utl$w#6NJ?$m2h4)syD6XKfp=eK#P*t6ul$wA0BOw%q&OI-&u$%?qp)&AD4` z=021pZ5lTWb$jJbiPYOEE|@)fTE&G3I5m^gZ)nTQ?NFusVnXkA0Zqu_i1K0D)S~-~ zb&Zu2cgFKc_YWV;<fP2e#iv<#;@Yj`85+&KeR%eRE1tV}V#0>Zf?xOjD+(6x>POq{ zHhibI_A;DVH8JtptqZ*;Bp3P+A$6N#1%6e8YXaO&Bp6C_y-}D(pwvCFr;Y5KLobcy zuG8Y5ertuX(`aC^ER5ievf@=DHE&$6m~b&NXN^lO^e;{^w;JybZ#!3CZI<kkWO(~l zsmkF47gvq|LPyVPm62EGS3H-sF&d^kxFAWVP8qvAcL9lDO_P00-SnZj6}#l{>-E?V zww+aBpb@pv%(8*4goEfu#K$PYg#v!7xmsl<wq)|rx;s3%tO`^eNe`E|FLg^@@sQfU zROOVXkYR~9aptmToZDSu)g8D>m2&jf4Vdt!egBqhU_R8I#OhJsuIu0m2oR|IvG2Q| zPB7lJ>S<3h8#9Z(u_gQp1ei!1LXuEhY0aWYK?I*u_lc{ol4z8M&~wVxx^AhjXP?S- zx#Sp9#Fr{dhsDX9aq_r4P@cIm9hw$*-@L0kHFUf_`Rs)COi)UFq6@yei?(al@GFO= z#uk5-mu~MR9n^4J^tVPQ=u%F$tZ*vqZE3m|xL<jfHc*vs^&pl|-865L-l|9D+8&L+ z%Ao7bdt%c9oPuW~x;0lXPwR)(vIGQ$qM`;%sQbW*lRqVJ^9V71bHZ9rMNG<;JK5Z! zL@m!wD6Xc+h7h~nP9s);l7sZc^wx$z=^6i*v=)UW{MpXz+S+Ld!slD3zgfE=O9Lu7 zg>K`qbQh^l-e{!4zu7Um>3Y*Px@lvi`)Nm#r344Df_yKvy`n|t`y}MKsoWCFqIgg7 z7VNCUDDXnjDpJD}`RLtjo{+<XGdhG@r(5)cLF{HsaVV5hwjf2Mhc4mPSEo2<S&6Ac zaZUGFcdo=-BHqvzOL2ZpxV0G;sLx7&C$GqCH?w~9R>i*Hu<tP1MmsljoOwp2t8N`_ zCRT48s-fv=JjcGio`iW&&8tySx_eqe#A!*W<>;g>_b1bvZeP7qQ;t69ooSY^`q-UX zFLqH%A~<z$D|$PZCznxwwR?QcCA-FEZ*Zl9`#rmYPMo>9E$i})shtD~#e2~&4f#Xx zzwh_O%M(tI1(lH$8742#>{)L}qvyl8F&;&Ru?~KT{Mif3d2JJGi!XBeIG67|dg1mp zx_R^i|FDvL&r^)dJqeJ}*=6kgte$41c!@}_Ezq<qJd2g7Ul(cCKWJ`ccvq_(r!^oz z@1*-YJWTgv$y{~FyAM}x#@Lm_-@I|zjgn&->@b>LSJD$UZCp~rxRY}y_(hxql8)qh z7DWr((;nV+*WT>OS}gi@^|lh*dHh%z!l$mT-)Tlo^U6qRnQEIR@=L1PyNgoluZ!zy zrs-zth}HWCmCRK>cv6NhT&p##G@RPkaGL2=WhkoRbC2h+;M9YLq4|7uLCLL<0oTPC zwNa%7wq&o^N}sMP1y<T(^_EK`Y-Lp2J>7xz9*y!nC(RLv0N=Us3A*MwkYE<__wS(P zh7?7W0=7HW(X%v_fr=q490$Y2uDrQDE=CJQR$!Tw)0;8OU6R>4$zGe?e)ra^g6_Bn zdLq&{b|^xpqi-A<E#z<7%mhvKBpzyW1c3Bep?ozV!7zKzl<>@sky!n<!rb`$^OW5p zZYK}jTWiGRJC|~6hO2JaqV?wVLTgDl!q8!d*<6vLPQ`I3u(Ixk9r{B-`%&-!!C4!x z%E7ZF>N`!mz-Bo<UCe807VYb4D?wPVZNwOTedLu({?W&IwlK6-lpy!3=&(Zfp=Bmw zS|*9E8QUmy`&AKXkg3oa+5z<*V4wHG#yxu4da86vE9}fiS9iu9!6X+Zos`_`<rcQZ zcIbAd6zZ1TBCa3B+J3kK;9;MO5wg*{ouGaWiyfJ%>kmqYprz^UYx&V$#V_t;qcsE- z-)Uf)r_C(Yl=w5kH`=*l=td1x?!L96EY%hCyvi~U&rsE)vEpmMthc@~^>CR8HPvxc z2{w%`?+-UjHPbntbY9ytOhq)HfKhq*?A*jnCipB{dbX2ghqhE-t;}$iYtts~dRA^w z2a`Nimnq_gf>)cNb{pQt_nFAF3akE-Tbz|k2oa~2&jN<`!@NRBzh81I8LEkOz4=Uk zsH4k8k~-ou-+t){ce;8f^||XxX!F~?R!)&L(zL!xx!Z0b?adq!4xZs!7d^Uhgr(<1 z)G-!xJ>AUBN1>)3e(tZ|V&we5-ahJShE^&~hJuo&hMV=&S_hv}T-mUw)}BF+1?DPN z`>5z)v0(NVrkPI7Uqve}>*bWciaq@t?xDEq+Siv^Tay?M`1vH!J*{^Bj`b#&^(^j= z-PpS}^W(vjXT@BPK0lP$jI2~rGEU!a5IuC6&M>KT>6Q{$U>C8uq8k$X{3R%m)Ugs| z-7V)W63D2QJ0jE_khymw3$)Lokx2-ng%zQT_4Kux?%VX_kC+r?Pn5KL=5F|O6a0g( zS@hNZaMLi+g%*lj(cyx)Etir=3y-TNz9scnniHo?BVjWtakPHjQgM-Xt-;VU(`i+_ z5Vwo4T;^~A?%mRKN1_|IOWPhykF~m7sw<CfsNJ5s&o`Q>6=QhYPo3W;&M@xMk|aCF z&QrtNikIwY8d!r(K2fYks28q_E?7)%u)d2w-SPE-r8L5ZuIE92Y=cCHXrEPKRJo9y z-a48bs~-^?>W9;i&Y{#V>8}#j+US*AY#z7O$Ke$Vo9Su{7-<Y|{T6Q;_uziuCEeSF zH3b{~_YRURMI%6+G+57D5f&_d0`^};q(7=lJSsS|I6Nx;7x#|i()_Ki$}@)xK0Fr; z`wzw2pC6}?GAKk~^<t_yC~!j${x=Kb&(h(~s<<eQU?xSKvt)$T@kOh@JrlA5Ig3+A zaWE&me?0v2R~-$>bb{v;l8Kgc;{Wt%YSi@m_wRXwBaa0d|8^^&tj$O!4NqV~CY|S+ z_}z2juY36GCu9lIPEAcM;--wZ<;B!LNA1|6A1G84!0%o3mVUu%`(G&Ip9NmjIs_~6 z)4-VEA^3*>fgk<#m0OpUyH6xU$qrfIywiGiJj?9wrEP;v990o401(Wg3Qhj}@_)YA zzrkNvIMVV70v7f21a&=!T8MrBx=&QILjx=Nq`Yf8g(Gw%n^dq0L87wMUM{t%usGSC zsJnZ;Ea;JL{vgM9se9dMky+OYcLXY(C8v56`<|FbHFl2O4KIcMrkgPq*4s1dMRKu` zQ;K*YsPYVHE9n+|OcTCzTUiukX|Ygc7@MDNql43SB}z^&d2{RZe2G|Wua@?3)JRCl zXD*66JBusPlT?;{Mn*}(Ha1F$%gtMLtc)7@4@MizR?PMvW%}yoL~B(F5|_P4HN|pE z?F0!bZtuw+ipC6$tnLJrah-40!@I5`)F>9kC>LB?+x`9^G!f?mgu&^cQjJ@}h})xu zxcMHN(_(ddw%o-6C&Pq9a0JxFXqH-7Tg7b>M>(#g`i>e|kMt>ojOBO=IYkSWPVKW1 zhsHH97G%aP+AgSXm8*{<uc!NYTc6jubP*BX`}wv7^=gj7q5If$ByG*tr2KOm57JaC zQn|bemz$pI*5n8C=G`_~JfhhYinO%nT}S2bC|T!s2t8~YDd=k(F5$eIv3#WNvHVh| z;yS)}Q(Rb<b*7KtAZJLbsb$y1v<+Hs$6T2DT3=9@=*S!fBkV61y@rXlv#6T<e~o>4 zJe2L*w>4y|ltQRmSt~7;>?D+8?6S=$m7VMiV<}6eA=xU8!Pxig##V%oeeA|$X9i;% zj4|fDx}W#`-S_jn&;9%551$!h&g;Cc^E{99IFI9de81(@6BAfs)%zQ*-7x5EqSaWS zX_GNpj!e13PK8~d{R~}0-e0gKhbh|D-Myb)e6FF9@hNIb2frgInwU*e?|Tdy^i|by zOkoLUc(rD=EYu~ZR7v5IzTEch7b5cN8iSO1F$ecvfgpr-U5DZOYY9QRvd4RZVXw}o zanT7{Rn$ocn%IiA9WUwz=aw}+4aiARM1AOVed9el9$fvEmQvYWF<KI&l@`p5mfxv; z0>pJ!eV21_cVg^*B&iP1XS?0yyBQM|syFiaOPi^UTZ=KwEt1I5cjw_)hR}#|){x#7 zT!V>ehC#sGuhyr4GFhkqPs~o1#`_j$4RycQ`xqk(Y7}{Ua<rywuu|T(HgPgmdrxUn z=i@w~PbOTpd78icW`117GvU08Z;oDlDp342(%WWKuDrWa&$UqT7N;-5)Z~2M#Mo(} z%n1+kH8!)aZH2&Xw=tX&wyF?&)^-!@0^mj3G|q8AmI9>)>21%h6N2LL{W!UzAy-St z8GKm}O6h6;L}^uTrQYI^mG@!g>D{~z$SJ(e&Uy>WX^3ChnA9hAslGuWJ=GEv$0v1` zM4P<4P&_?;Z`dK-3x*`TziWYYXm`CdGqFZ^h94bpc!(+q@%{0Kp8U9nNTZ$f)r+5I z;pT4e2F!x%=XHJ#NK`iCmjw;|FIgm}hP7lMpZefQ5%`|m&81m=mQc3$lFoJajBoKf zdaAUVTMqlK7XIQH&xNbE>eE$@aX4$xg=rsS)zkuQrcdxs--KJ8rLJF-I~b{HP4AH_ zOkI8R*4UmUV#(xW-`hz2*&8QxnQkKQ-P~ggd&q!}E7;2Hf8GGj?w`>MYFGa<0uXS; zX|u!qc;!A`-9U&lp!EgPj-4U3<OL--&luXfjdJb{mQHtCCYHaJrw^NCFzN9$z9z!J z`ldm>SfB3Ak*hl_k(YmGqcJG@^@XA01i;PvZ>H?;cJQwcY*(1`@l0RV_3c7JNB{We zG|pdkH-OpGkKtz`UBKNBKwA2LZ}qSC>B2USf5)<^qv-mM+Xiu+u=<~*_<#G?f3<cl zR((yMF2Jl!`LE0Vn^#wZ8EO^j9Zu5y@_GN;KKH-A#Z>=#LZk-vZ~v$m`rkTr!QeBG zy64HzuU|Fh{^RZbx`#9_W=UU$JJB})%W1OrKZHa6Thsrqxv-5>;uCD=6pnt`(@re< z=WOHOZcbex?1w&`H=THU8=!*upNAR`I#<r4o|U_)8>)Nj{y*nM|LUKu3-iZ8{w+Rc zTNiEytN)KD{$Hc@*EiC58AOEG(s%{z1^?G4|33%EKnVb0om1tfL(yTll4buHNA%Zy zUrl1TC_-;CL4R&4!1j_zchzs)I)DtRQOB^QjdRhpn72y*R@bhkGNkj+3DUJ~9V>PH zo#*Q$(&r*bcV@gvF0GLHe|a{4J*EG86}EBWC79(Z__%G|Z*|B(AS@|fUvYWcaWFfg zax9l_s1SOg%cMMbL5nWzGtUx1&|6@Hv!=?i$gaBo+W0_*R%|UG_ZaP#;#%bcloY?J z+V2gST5KeDOJR%a))un$FV98v-)>A`-6nC=m?34dyImy6u<NwX(|M|kAFZsr5`tiO zbY0z~)@j5lpjNUNyjusUWf9s-4qWr{i)JO(%*w9;+<1|7JL4s2pa8u+uo^AsJ1X&U z$^l|lWI5eP!MdXXvaV*VXIHn|P=4atG6*-Uhg5BW&57usC)_BBfh_`4XIF=T!mhQ) z6<A*-?I!q6GC(zeZQoGbeV1H6P+7qNngAprg+r8!y#v?C*LN|-&Xa(SR9Ex0v4BPU zSYOQbk&>G>K10cq?2wwAy6pik8y%e}KkOB+!@2NOc!OJXt+BP2m$BzS=AlwT7@<Il zF)<P-Jww*zsH`rpNkMAAt5C*zIhU>Hy8$h}Yb(f@!6YlLM2)-85I02h>|S?a{b*CG zjTE7dAudW?b@hluQ|-YfJC;Vp!5XbU5diKnH5XNC9}Y*{D(Y3ZTt2AZ%dy45!4Xbv zSWIo&^IANW*EJT~bV%x*44m5X-A$^lZ|Q!2Z7?|XB=UOQa<;%EAXs*4vl*zcyMWOE zs?CYN$QNq-8VH<gr%L=QC!7k3Ug#({IXdRSYzmF|OqSC)wz<(?59#H-d{<}994At4 zz`D!8Bi9OCz09uVcjdpaZVbw^s`ZSZ01XKagu5C9G;;^93;F@o<TU`rDa#r_GQdf` z9~c<beL}TsNLbEwd<o!kOqo30<eyqc-LbNPK#+mM<-V6Kz#%S^HD3{4J&q5@e|#u! z-Uh0igCSEv*IXbtB>9JZ#8yqYSA5-O=RHkwvA2r*l$jwF>eZF|@VG#$a6&c<l!7f( z;7rO+rpTz=2>Gb5Bns$>>3yFV?3A3RRAYDU`5kQN)QZF&0C1*!3H$uZy#si=PO&EH zY5|tl%4K=0BS&lF4rO!Aj}{xa-5-So#M<KJ0&dZEC^(v&b#HzTbqU2CLEx})zMo)l zCAP34mJtTGa5Eh*fu>9euZfUi39!s|VaUyu;NI7#x`JS&N7cRnOmH4NxdCX#ZqLuv zHoEyb9N0IGsHZ`8;|`wqiePo{yQCMKAhL`KO!rFIhPb^B6khpkx@B7R{(-7VF{1FQ z@i-OXgy%eEhE%lhF&}aV$Qj!W%<mv17GoVpNND#&v27Q;*Ntiy9{~Ca>6mQ=_Oh${ z3@LK1o=(-(A!N(T+G=2zKcT3mVSvVEq(Dln@}@zD?;(yxao&Tij>}$b+G1fqolnz7 z0M(5xDu;WhHGszW!(c?pbGNGCg&V>?ME`aI*1QRtXRO!&%=`m);fWQ8pjiP38)sWe zLx!Yz8kgKJBY?dtCP+BkAE~1@PR-~mZ@llT)hHN%e{78wjvdZ#D+2}1Xda3#jAY>C z7Nhx90E+6zyHRSBX1=m$GV*@h79HL76L)WGm=tNprlzi8$N-Ta65R`1OE`urF21sh zq#Yn9N*}`T^>!qbn}dX4$4VEZ@!U7Z!m4gxUFNmg#oorSU1vAyzR&dgC9jQ{_FEtK z2a<cZ7gH5oMw@`MMpE&)jgf`S&j2~%4}?dX7?1OP+7>%x?~T&>7b@WBOBL6AlmDoU zQ`!63WNdHtqN(S&?DYo*QHVi(N==GVwWr97p3?|<C;!5nw8(dMu`tlm<PH7}%krm{ zN=s?-Xzrk&S9i;i*txYjV(POXpon$6R6#bQQ;TzpcZLk7Rw*Ez&X_eioiE@hTMmke z{h}ui8%Z{8+FG7Q>Mx?0v2P}wORzy>wdW=asQ_IDUSEau&7gQ(IT6pTNCZ>yr)%tD z0;&TyJH&!=vLNA+5G&QgyzLyX2}LHLz_<FKtd0`HhGXlJ6gJSIi+~)=w6B#TRXRq@ zg*XCm0N-R`U%%F%I4aWsa(?ywgQ?<KrzZ(8{Qez;kx6oNlvC#ir>^qHFc&Pka_yza z)Nw_l`DK6d&lfb0t|S${ST~y>;Z^7!^XX6XQ-?b+O?$S)*&O5{8!pbld>18*y;)2_ zRtu$yy|V~-ociR0{U+-Js#0*%;;b;uHn%0)C+J?MIL$!S$o;%UazXXpapdza{t4wG z4Q&I|B!Ii4b!fk`uY;oWE-!n^;L7DEi4y3f0iU9t_-;;MuBX)LC;jp#09e>(bp1#) z|HglLBeqNo&@zYyh{3&fer27h`tYznw()eT(an`o)Fkt1>*LyV28?m3mZOu;)@pO= z7`bjbc$pgyZ8~}T<0xF-=&(RSQHV$2%Er6W`c&IV--%V@5!ol5C;eztkc%(3n^o<V zZ_;%ulMWL*ft5&h-m^rcZLq!nJ|M}-Yg)IL`$NrnnD=4BK8B}N_g?tM@we0ivXzpO zDAdwpdEnyt>wr}2?qnmS(c{M-JC!9iM1b2?$_*3{-2-RjR{pW5G(wJJH>+7xwl7%E zBs%6Gj6DKWmOVE+ggx8?C&D<X6&OIaP-nM+XtOAAy|<{QSLDNVVY%ulMSW}Ev7xoM znlT&&eGcM|A~B=;v;Q^6pwAE1>S)J`{wUyh{q$#74Xd~?P>xibqF@{VO}B$@9{0hF zd9>T_$I%6^rqT{hhsVe?QLrWVI{S4=QNSwG3EmFXnFMs0XTY?5`3QkV3(hR)>pd({ zwHe^QWnq!M(zv80O+xU}PNtrFQ0rMMjbfkgre`^OHXGp5=t(aw+v+LT)C44io5VV~ zB~Wb_6|&qgow*fup3Q8?O~gG_H5Uaz+T-t5Jr|M%I&e9Noom&PJ09Wa7la^?RK<6u znAYnk3vL>~jBt|wrVsVmbOeNRKDvhp?1t4fs2IiXs|~xa#X)?|Z!CveP7y1VJqvfE zR%KH|70dE?76KMwW2<FdUi|JXudnG3&-450i28pM$ogcuXy>zM!orYvK2h6yVzR}o z0IF&BVH2R``$24P3BC}Dkdx0(FT4vLao^Fk_{9cjAja^9$v}+dxdXsT0%JYcOjcu) z*8}gORa)XtqCN8a1_x)a)XuT5!QF$h24!SJ9hS_PW&NF4b!{B*j&3T3d1C5pteI>} zPa7k_O8ZOPl=Ee$!!5QkF_8l*#QH8x7CLI~h_S;<4`$g^DL6Keh-=2}gIcoj)c{ey zf=YMRbUW$7+e61CH`#3jY$t~rZ{3CMmJgBs5xCK+DIY)zpH7cj6CNmVJBgPtW#>OM zU3{LZY62aOe*l%dFCQ*zQk$Ae-9i$}#-d<{d+qY*DB<8QYOg{x?us15D5;8!H|^be z-82J?j1J{|bjTUKf!5U2M_7C0>HWw(+ELEroo|ghy95Z$b1!Ghh++$pBy@b|(Z4BL zxY*3zdz9jax^i2pcS8$g2zX_Os$A`nIcu(Q0I@~nF=30`bWls5Mc)$RVk%|qnMcub znQ;p;CSpbcq|z!(G<aL}BTDUH>MDPDkvWoI)KcCz@e1J2A)`_nfJraAHZP5nQWe0p z8`w5cr}+6n-iP22DhPF?%+s(hzCxFvJEF@2rRsQkzbceTGM#d$cnz>CjV7y{zMV20 zztm|pb+FK&SxZ_52@CU^VBTkpoLKr~uA083b-wpm)c7Hbn!kSn#8F}XRA=-IsDj{V zv}&jPDddpvyig~qqIV+))QKQKPnZ&#NW;iFfXl@IIfdlu@tHT?*PAcCL<;xzlwm}) zK=~nyh#){nx+zMW3b+ezdP<ntx!4FJgHlK5I|F^*_ZgSxvz!f>tGm_ys?=acZGH?o z)CbJ%wOq!~r-qE~W;uw8gKTSS@1R6sarfay34MMpV>0|XSTjwmJ@IrGC?NBeCErQW zbzBWw(7`Nn!*SjgslV%S8e)BhQ)j-W8Q8B^`{>niL$5SB$s5yFQbWSq`~u7kT=z6u z=q<5-92gvcIHMGiQyVcfc<``0z@nu2KwC``X&j}(T%;pNcXAsax<{Z(-FRaoj(UEd zdSD)H_wJIt<h({l!-#C+<|VkTiG+j6Yj+rdkRF7E?VgSp^U)V?ie*W>k<V^vpfZN9 z;!k=I;Gvw$VF1p(u0eiu;7G1OKS+gLG7nBHh)Rx*?o@$iW|iqt_S)5^j^p4<bzsd* zfS5!~NASn?$Efli^wopn7?p@+orwO=Cc%z~>wN{hGwIer11=`QZyUhQ<%_9Gb+N(Q z)O6(m#jc?i)5Q@_z(hK0)@D%?(r=jwid~N7U|$zz;A24d7>z@*gdHy;jL6b#9e^*$ zUtIoVEab5_&2oDrsp^C0Vx3LSGStKegmBJ}H_b0eDj+-=nm>v3^ysjHR8nWWwwUqh zpN6PhiLU70_*20f(a3OuenUogCu@qiqbV6?W?|@vE^1eeCmo@t`oI+5UDra<IfST@ zJGd!v`G*fvI4SNv^of2Q6l;r-SdrS|Sh>#AzfSEvK)%3Wk>%wQfCV09b8xUV<t1$= zrcPnrHt@SyX(Z}kzxUS}3{9ADo&~6W?ja(xj5H%9-#MM>=kas*Z?T#Tp^h}#o%X20 z7%)&)w)WQ(CcRR(>et<F*<=##im6D94-Y4yrJc^1@DKJC@_Kt|Hsu*wgII&`#_bU) z2mIGfiJ&3MbJASe^QqY)8AnbQ=ye0o8T>17Lk^`!$(>t%#^pVP^IoBs2Iv&w(H)SX zcmobNTBXy^fI`qWCMQde1VqQf#^J5i4VP)GOs`XR?5CS=wF|(Xr=5R8$}FBy4cz(k z%rKve^(cC2kQuq-gM;DbFu6HxS^<(v%UThODn#ao<O=QBtC}BZ72SX^LAXCB7uRhs zyl8n0>N$Ag*%$hfXnvz16quM`qS>?%N8BelYl{8hvtn@QF%^(gN8Jt4OK8fxj5>14 z=c<8_-SlRc^yb7eJBT*Hvj9cbXfvLB6C29^U1lJHzwMCHgtH##nD0>vInDpbhY>{F zta?VH(xheFR?q~uhbO~gVJnR3^b^^H8&dEmiSkM->l|toG_~de`HFh}Lom5>F_Oy> zSTiUJlBRI3p50m=jPdtt<6DgKav(PNal^-S-J#E`jSN`i5jeo<8tNa6NIGp1kmutd z973ZEn0o+ZWQ^%9rl^V?i)E|yf!g8`lDK=HUTS0G)HCpE?P6EU`trUWd>0sT8q?rz z<1XSMVjomuW;j;lFClE)ez76rHe3q5QAsH7<=$$<o6=5gvJ#jKi|frl4e{YQ-Tl3m zE~VLh;yony2DbFF2zoylpTpn=Hd8!345!jkV4W$6O%3Z2fsrGe)}D9M%SWmv){ZF# zIzSSmfYu^*AB%ORVZN`=zCW59XAC+ZAQq(q#|oOOXxlkz;n(XYc`O6)+<`>u$5K^z zPkbU+_Qj^S57%pV5Q=r#FN;zb_sUx}?~+-RCSmlZ>-e_A$&;PAU)0uUiAtjRl)O)6 zA;TUQ_?90l7)x-!mM$@$KkND3s)ZoJrLvv7m-!HvSRAXgmiv}h?O-}SW4Y6ORwTut z86XPkGDsS`FNsX3CO}sK{%<z;Gbm@|^GLEOA25QuX!(X^*K`m4fI`JAiOiu^$yK|o zO@M<IeC3K0Yd~+{lu(Zf{5CNCLbyxV*yHXW>;g7i*87X$tg&+a!F*tpT9Mr+Jc*w^ zx2|rY;U}{+7eW|gXi7Q~;knVjEh9iJhAsmE4)T7iBHpngE?LnAB=vZx1*AsJd~07( zT}>I`U0K~mhr@PZcWV~D=g(%&$2}A|h*EzYIuHle$a&Ur(Y$tHq1lzFOft3cD&UF6 z-$#RoE3@#HM-zMeY+9VaK0^LG<w%0kbfXF;(dRG`AlduV5=pjRbce~VoS_Zfwyp27 zm|w&@<UpISId>*L1F?=jtjmRbFj38#_wmQv(vALQiXuHnaxAmQuV<l9EjW>n0>wN& z@PfCCs^-Jw2Wx26!cR0Hjum!<*TMB?4Uf++Yn094^wPBNSWf%dt>d%`+K1`9qohpJ zC!5ibgUFHxP4YibDLO2|2ifaPnQndJOGk=1Kkx_tRAcgF9UaV3S}8S-sU5Wc9JvLY z`)}t6JV6Jf3kX4vl8ljxsxn3ZV_JJg#A*k?G<jg}P+<ba`HU1%BSs?5Bb>FdHGKxt zN1gl!wLYYtGu0eQ4Q%{l@`u4<*hu3J=fl*!5T}(v=G;i_UY?WYoLU9_a+k!fIGF%i z-@>f&UZgt38Hdyxy_b}yGb6jK+{l_|M9tuzpfk5l=kRAAycFRL7f4k!-^I~vs>btG zhn{60iK!zV!a6*i)Du?$I6v={hTn9AeH7OTU`sP#%mlwkW53JJtx}<=OkXHq+#3=+ zo%aMQKBV^t`^FRCOmNs2TEKONzsiPaV<x>LRlJwf-D8hnUy7__MsO#oCio_Rhn5Pr zpN&3rXGi%gX=g<g%IvhsuL(cBn<S6UEAjKT(nRnRd=4kQ2i=40KaNB^*}i~?!1_*z zd_qC58`Gx9J({-|)AT?T6OmAO)wOr}vd|2R5=O%@G(fx=zUB4x`Rpx5^(-%&4MWCR zt<$IYoi7D<<`g&VL`{QaZS$`g36f>Tze?nMI@aluh&|daxO-Ra?8>Lo2KFm^x=Ux$ za@0xT?^8dS<eE%Lxf?g4e3qXg;TgwxW2p84Ws0_QoE-<*M%Lh+vS!wEfVXxLH#V*L z@CHbwa(qNHctOBb`Y5~uAVShPlXoGNwg~6mf+IsWnG;ncyJj>HGni>Tjh2uJGPw9K zt?eri{MmVCSF=918r-N%dwQpR!zW}>3AavAR&=`5U*Xk5espIRzmhHH*Q-X`G!SY{ z9_)TBJG1O~&nM=*OloLq%#a#8z{(H7zq|;ztKN4N?6&OZHRNpI<M=OmM{aViB4zns zcbDY&nTk3jhy+Hw(ks}`J2<$3l)zj}jLL5*=$w%bn!cXX|I9@H0oI(PFB=Z=9Sdme z**kO?+CaabDSTTQn~2Z1Agz0P^h+t2Tqy`Lt}Q}8HM2mGF&&rr=CaH1uf&96MFoTR zuJse#RX5{6d)hsVWayz*`V{1fs)d~824JA7bBjx2!Z#0Egx$8_X?2;yEvBjjXfs$R z(`Th}#JM@wLNeC%!n?<UI$OtqiWETIAvu!kmLkh0IC>nNxKlXg-zkA)C$$-as16P? zMc$1QiYDfj6)sIrhRlc93MOHq^XDCXYIXl~OZGMt&<rS>p8Ovsi#$G!M*Ed9(j5&0 zjOWlvKxB-#XD+BJqWmL`5a|yqx3)+pnSK<)JuRCuSl+J~zpga_rBPzJ5qvIn0YtAX z8?~6Z{o$%Vg?bE#4BaFljhSBxA#V~x-Es+$@u|7P29v`Q>Yzk%Yme5}hf7a6afRtc zCwZlX(6t%q)}-)@^kGg+E6am=DUr~J=H>MZDobfCnlDmQpA8r<Sb&zC!K9tx_!{@1 zSu0dZ0s3ynaVE@?tp-uEJS~Uk)QSy|dE9AUuja(@9loUIkhd>HL>`=VEFIxoYWv)y zKs9E3`5ci_R=)eq=`k}i{Mgl`uWp;)qCN~V``0su&Y;9lB&QdPs=JQ;#yF3rx$FFV z8lcBcH?5fi-%av|wZtBnk1#-=v~(uEn8r0n5Oinq1%jcQ_J;=xeR~;OS}kGO5fw6K zdG-C2wV{oli6sXm4+}7*RJ37<*hONUyWyEr>Zn(_@p`psDcU=DHratU+Xrrrj<2)@ z`?`CT7tu3W0=XT>dUG(JH8en~hZNjy6cG^G&`Ahc%Hxh`7Y>mL%BGs<)vK-5fWq4( z;6@qeWy;;kx&iUrLNF@tjf~0b=pu9CyXv%lx`d{7NVo0a9#D=rbkXOq4I+yg1w<;E zn!hVMVS?u3qB6|=P4d;mg#q80H>%B0MWM)CXQ{)5T~gjOQUdadC9wqkA*k!JU!)<M z%uJ$o^J^B_OedkD-IpSfip{Ny(}6H{Kw=~(qfGb7@K$Pdpy~1A1QsCLYOg7htbJ8i z^>b=f`YnE}bS4Rqg*)w7KDVzY_|v3`N`Mf#NRME}R@K9?ecSk4YlDFFNN=Bd!#l&R zE}9mU_dv*rv}EkF?djp@?946LjbYm^a>uqiZeLDJanzvXXd@o?9p)O|0o}2DS)wUu zFm(PF;<aRJ&L=qjGn6?AJcxouG!JBf#zmuGmV!2j$V;P3Vsd&-=J~UuWkG+u6-`sV zUDDI9a4h5$c)Wf-*f{E<Y;yrPYx5?NTpaiH-T8vXkJZJ=){v~4xuO!$kuDrc<m&kR z3T4XAphhi&B}rz}*32LZ#0<C2?`JcObl4vB|1K%zlBzrC+E~#G?3X~<`t^&o@VLpv z*AO+)k#-!AC;4*wgzbgUSKcyXfXn{<WN90D4!*I+3FTY_7CLYc#ma<#oYE^6L%Ke= z-_W9>uu{djmCM=ge(Z^MI2-E)rYc<HdS)(`>CXKg(jsukZUKUM-1`C>i%}1kLe^gp zU@bG)Jdl-zrrkkXnp{RCWv_2*Ue0CFcv!hNXyMAkcW%X&vq8i47ELAKhU+rVmtIXg zzGJo6H~1(*ptj?QHo;vsQ*u<#2sXm(qrCLA1k9Hv_O)p{%lXre9zvk=GHRYQx8>2^ z(A)IK^mHG6vtr=4k?!3#o<1I3Al9<6&IOSz#s+QqQF-{1$%1!5b&igOMgb_XW)g<l z7jSJZv;R6~`R5=~RY`^+Xh~C;oAZP@1QxJ&*_H2dk#<KxN#nHyICKUK|A`)xc~{zX zrI#%_PecRQoQM);ec~79n=yIDEsqiAJK@{Gl!jIyTH}0=S86vwCu<W7dWi-8wzh>- z8-guq>V!Xpl%Vl-7YQT6P_(v&hdFBC&aJGP%%VZJgVwF23i4jjA+xu!ktVVr)d_*c z{49=ySJ!6&OdB8fTYyx-tyCZtq7=n-M>5&vg7CZszuufh@3`zKZMe5QE+{3i+j-|1 zbn!F3g?Bq4Ym**+GQ*G_v-Iu~qbpcdtWn)kC{x=vD~?do)D~O6IpN^_@e-#Bxp;(b zNo5PK957--<0z@={HR9oOMH?qo$prXG23=EWz!BGe2N==SPpeftpEZPy7T5iN;@hb zYkj5FPgl>9?bRqAIC?jC8vn<Va6@Mo!JXQzp5CJfrtPVXTCc^i-t?(lwQa@H!ZhWF zUM?ou64FNY4Dbh~1tFX-6h)HRhwmyJdRKuV*|vk*4MxD`aqQUl6MfDOjie%BiWe38 zkVye<6ldD_q^Kph+F5_7KoUXx6~b$Iq*&b<;Sck<;G<GhXJmacN|`j|x$)Q%cJy1< zx#))gf+trvHDIA*I`42!<z9CVB@I11SouZ~C6)W8PwJA4?!94BT;i#RVF@%zy;y0Z zwSDgkPjrK@`>O9K)*D;bYE=(^Q5uQ9qeIY%4)i~U0K7`ZUn2KSTN)!DRy`Zk!*Oy7 zm@N;%Svd;|#JKz>Tqo8WJsJXQj(yZks`UAWd*RZw@k$*`Z}Z(zyf?)S{Z)Q1_z#Xn z_b#VQHWfoRd7h+85*+vPmwgAnEDZ4+u}=-q@NdeVy0{O-Oa9={8XX>9Sb^mN;PffX z`&}TC@Gv^^$PJ%c0H^`QCUTHLRiGW*Xvt-QP{uj?2g{uNF{P!@vRk0OVyj(~1zwrQ zjoPApdMR;}HI7bBTm*1qkmG`!)3vWtRGTX?(4$%l(1z#yUYovqea;*hwx0>c`Wzx> zKf!idD_>qGoYA4j-Zo*OYshYnykur)UHeuHelZzvYOz6LRmAW}I@$eJ&5#hto|kQ! z+P(#RLs3okGJxI#jFHP3Ify1Bz<g>RW#}byz185b;wH2qg8xEi1d!7dyKlzWwvbA2 zoap~9di1qU{g2pQXSiyoNMVevd?4bUG_kT?kp`nosO_{9QHP%zM-jpM<QDBi|K)oS zz8*6Bm77xakG^Xg_HUR67iz}?XO4J;Cx|_9=244D&SmE&N8#klf!8@M3Wj*LXKz_F zF#su@x|RY5Of3kW;dmZIJzqO{K#tINisrq{5%0Z52vR-Vi;pi)Iw*=Xl6^Y`ge7%@ ztJPd;Jcsl)WirP-gN92&=lP&zJ^X^&C#*PIOPr+|hL^&5Vm4YfaKb(=cdc+Ns?Q#s znV*@7F5zeqaNgVF0UdaYJ7a219L;?0$T}=_92TbeOEu)X73e?KP3DJbgVdiUdRY)U z_f<gT#KNdWc(mfdlI%4+)3bgp<h@m<w7BOoMqi4mr%u32Q8k_SH7Fl>y<zEMMDCkE z(~eIpEgSmm*KV{y60GY+cLwm&!y5X)e&V)s+bh-Z4;O6E9|rnA_h!}jdOU389WThj zH}|MSn$vCIXjU#k?DMfs-D<F|AILV%CpdnnKwGEQ%;lA2x4y!)>8F{^7Z%_1zPweI ztTF*X23@M(k*GagJ-LZ`R+OVZ6VbD%i74>~nUo~nkMy^Q`;^@jKPZL8N%;MW3ti;P z`CQYRI4K(P3vmJe!=OfUO8oQ1)L;7yE3@~Y#%T+F{R2QX?I;nhxbU9FDgVH9dZ_!d zibC#g))bJ{X*P}<$80YTwotzLw17+KYGz*F3o5}|Q8~x#TJ#2wYe%&n6F<JUozr_X z#Ca5itueMgC3_M`zQ#O!32q>oFPpcl;{|g7ep}O;^oLUY6nWW-y`t(vWe%kh+-f9{ zbZ(T$9J6w(RoLD(<8}-L%4+T#0|Dd{N1@41mz#wG>o3&XSrd0Llx56Ujh0@3uV3l_ zK55r1x-FCBW)1ekju^@3?bb;wy$ZNYn-vbh2?DXs=J?7y_6xxno_F}=;?+(aip3R& zQL1wZ)oB~lFjL+P9L0eY!1!wLT*OvA<?Hl<K`qxw{GH6mMj!$2RdXJO&x?q5Sq-U8 z6VQhe%cn$g1q^oFc0IeU7Bw<^i-6h<IcuCIM-&Ru8%}u}Pu{9@`~f79nAv04A+pMu z{ChHmr9E2M?P~ALl-rU#y$XvJSE-|RYL+{W_XPIAa51Q)q2oS~`@6p_^BJs_u{GmD zv!oAf{x*XGslYP43<FXkxy^J~eGEn?y;K@GDaCO|FdsmI=eJGXR29VBMIU$30KJyT zJr7yT;E%>^WLpQ6_*vv{Kq$4(rbU48R$C?jp{V;6*5v3WXUjK8&S?VC4N|aoY<1Mb z+$Z)TY<cm0K)Cwk@F`k}k@d{hQ`m?&>v$5_-g^<zBoz>0IDGv8EX)uk=<6-M>r-4$ zqeew~8Y~lmnR4ZoZPxw^?RWiyX#QHw`E-1Vq}2|N_N6oxpc3wRt3TLzHPMux?;eRI zD=_PPpnGBGJz0IH$8@p=`IpR5S@2E66L$|C?u23eLJVJ44}`*tufG%gAnzXRP|@`B zxm0r>CE0p2qO9Pv_VL8yc$0iW34j`q2{I4P-p(EwJPX^d(>&ybe87K)MuZORhp{wX zyI(CvBUPaJi`3S`G+UGb5m)nDweS5-7Vi131g36E6$ybQt|%`0G}7j!{m5d$=34ON zDjN!hm0hx2<~n@AyuqrZVz4of<1GIB+O0)MqD?+e?j@s}o8P%3dv!P1Zd^Q^u!~h` zk#_oKj-8o{XBWAYTW@-&Ac|nA{e%eQrX6dt4TC@m^WksB@-!Lf!z@`Jh~1O*P4yYp z(g!U`6^$*)0}=-OwYO18OXK%O()JHV+x0p7JG~ms1!&4X4Zwo`(}gq>fGm_0sTXyQ z0gS~84ULA&HY2`{yDBo4@%Rt`NZ=O&TI!ufYDf1`-vOb6e24b!v^!=N=^r}vO1wjI zCP~Y8IupbPK1-uuNU16bfC7Pzf%9UY6X>gqN#UzYzfwwBp>6|^z*X)R9&t1|J>)2~ zcj7s7M4Pq>B1i1Qc8@VLHfX?L4%)fVCJ@4>$o*gz!9Ta6Ek<_uIn&gA%)tF{!mUnh zvPNmOB38?JfHJ#1r5|ylr{)}o2JB}TB_di@z|wRXB+p6$gBrFhPy3;#^TfPQdB>|0 zQPl#$emE-e>dcq%O6O)yP3(of>(Ch|Bn&K^)E_cG;8W{Z^&C)s7{7Y9mjBS3QA?wR z)A^-&1RzJCC<9IPpJAXi^j$l5U;Hq~&#J(@j$aw`P)9?ad!*1pu+jYWi)^Gv8#(yp z0C!~io=*>ODC%l1+n^Ob9PDvT$=+4KBlW6rq`Z_q3X%44J^qUCDUj^GL}oW2j6q}< zjB()$oH07RjQ*TZb^pN!8VGc)0H~z&v$R8Xx7$-$M>i^ZAO8RZ2JY|n$6i@qoDiCx zRGLrdjnL-u@&3FeA_aH6053d-OLaQ{kT>$=)q1rA8%u3Y2E`C>AsC=(!Ohu%{+j2d ziI|R2@f9b&a&PI8b$;J3)1YG{!nkyxI`fjbF`e36+L+>g2wr+zIsz_X)4Vqh#oS5n z(MtwH_4k$$MCEUOOyc!c^#ip+IB6>+%yM$u@1X9maeJU7D(CJ{$1PeyZ(h1_q+&8g zJzcDDNI`uu16&QreBv%yO%V{K@>6@OeIDv4AP(DM`k}>M5@Kv$`0Mz8reD+JU8N$+ zeWa?$H*-*)O)xp)>w<_d`Sx9~LEc(A>C~-w7~js6!;oD?FOuqV9@nr^R)7U`3;4Qw zllM>c4YojwVmn0>>G6FTX)>AUyat^*BdP|<ZJR`j)V_7O#LKXxwC|<zQAPV9|B{sU zjy^0_u*!9@sOLBqI3$e+1KAPS3;e_L`cSP<S=|dGEjW^dZNO7y_t^GrKrz)dH*o2d zhKk1_0GZ0T5X=8jUWr@shw=Nv$;CC^Ncae=ge2oT5pfH!JJNDz3)v)tIsa#<RIQ*q zU$IzXscw_BcYv1(T>t8RbkL_5kfrQSU&|B5{46Rn6&y-fJB5+Z7ir7oIt>B6f3OMJ zS2dGts(4)i^{=g2@?VlYhtDlo-%KXXZ>?`N@6Lt1uIBf`uzgube%0yKpn5?FW8h9Q z=te=&olZdfwxcY7+_lTqu3o}oWFXjrXXbyX6QFCU0_X+HY>rVf^ckH(_osG)kArK? zi(`1?nnPF<j+c6ks>&90ipoBdL>y9yv1WxOc(lHtcA~jBkUp{3LOcQd?2XmP2s@fZ zZ~<}#w}l~<;XUVz;|uai+A=yu_}NE43pIFF8OV+KdiPF83KX-AgoGT-HhsdsJBoKH zjQP}wT9uy|0#H}S{)|8NEbkXt1MyoZfb-c4Fa+G~Ou*fyzD8;f7cQtKk2zZWLYrQV zV|qun8U!S4SQdK_0z20fuJy*mKK`Igcp?OF8U=<G>rvgvGhlg`Tl=Vc5Jz=xw4!Xd zOXbjT9!SM$?~W;EVthQRRkuao=_YQkjw7mDMk8d&5=T%RXTeUo9zD&&<1Jq>UX;aN z_sJPClYQSc&#Bh)h7?Q^;ac8Zby1$pg->?RzCp?u<h1<s*5;!-UHHgP&{ypO?|jPj zx*T9A(^Tit;fwcpqg)?o)~-gY%9>PiwhGcz4s8|*0C9>@@5o+|BaVdTvSD??4O$N* zb?obbp)4pj7yL~}m?I^jvffRmU|NeDy%}d(U>4pMUq~j^MSjMxvf?j10<kBgsP$Hm zy5wqCkt#!#6<k<MPx<6+px_rJ&4@gca^0XE;@B}YmU~=rTPDNphr2%iJ%Gf*-pb`# z-P~j~IRTlg+=W(nvNcaO5mQV=jz{_T1urXOd{<s+b6!s6m0Pq1Ss{Z510&^KMvrZ# zRRo;0CyRBFJm(uCm8-Qe6+WUNR-Yez=#{qAof<of)R_QHB}9XyPbcpMS>R6Lc6N3g z!rvE?C8<**SmF_MAXcu|p%6OLvg_(?<z}^%Yw9RW@IZGO!i6$qoJx}JMBCTyzZeLO z1eudf^#V#<oomd8K9Ces9{0r$Rf<F634#Sp6zCCaT$@n!>J<RkMw=eo9&8uYlUmJn zvxxT0;B7Yzs2X^7T_yiUHNHo85rIH_2&m}H2L%4e%U(uZt#NIijmU{++|s8{4=D`s zowLY(+6uZNX*bj_77#<MUF?~bSOTzBGtc1Z<}b5=6>~QQ(v%luM9whBOXL<8)OQxk zm*J-^p141_%Sj`wZqA#5N&=xT60Kw}oQXH}ap!;5#Y?JL-omR%cjuSj5BkO579Ea* ziC2&+fo=<xc3E4n3rmT3sD)205M#YQgfZ7SK~(PYc3<*Yb%px~=j8-`Q6W22r(6s1 z$BOl;n%@95B$Ow1z6L@w$gN3S2U=Co@x!Bus$Rf-eljtz7<mfemVy|!v0`C#Yq!}0 zSj}d|N}Y!COFe8(hlhAGoN0RTHR*V+d{_riu`@L(a8VWbTX3c}is#FE(R$b8lYQ{q zXzAf&T>!2k(Q{C*DUXj4K{p`LUdE#i<=8tFw$Gc!qYe`meq`PN9B=dG?OwI<ltv0P zblR6OE7HH|KJK2WPiH66Y&OH-&G6D0L)_JUqLsBws<TH4-vIuh@tPHoUcocH1DK{V zOc26e|AJVfuB3^Lul<6u=Au+jg5|H`1s?y6rin*i9W?LfjiK?3^KhjK{!)<RzmxVW zuLEUVl{EYXW*0@M6EY|4eE&sXo9h*}s77O6+nFF1Z9O;vFx>rLbaoh&cW)4RXx!T` z;s%^C=ooy?8RQk_$~18KhKc`)jsC_oQ^IO@A@mFf<af~f4v*;->18(iMb`do?mztc z1{BX7Yx(()>|=kGdjW@`OMS{WZ=RVE4$)B;As+MiExdTy(eOBcmh;PFE!(Ew?0(?7 z8dQ)@ke;9X?1bOH9@M{qPk)M5q-RBq$IuPO)7|-<FC04)@(W}>^&aA?6S{$8e)JcX z{NKHDqDb?x(Vh6MD*&8+3wz>pRRV90*q#Xbog$6Co4<ha=Zdz}XIJE|{Z2^)K*IOf zm%&yh^p#)fn8<(bR@ke{Z}%1HCY~rVyk`Q4<Nn>f{@14%Bv9l)`w>Uisiji;&U5Sf z?-VWo6+hZ(kHkXf9Dq`UGbxNMX8&)p|C=#9_Fd$ddwa<!+qvGOAO4L-=HKqJ4RX%> z&#TM|yZ#zMl<3u)PR>tTdDPSL92r(_U-qMaYw1*Wg{jS4oh<O==bjK_6?lZ<#o@uv zlX(Qq+jIH62Cv<|m|#N<i}(9V=@?{Byi{X0MVXd;%v9(3VDv7$=^pb<?t3bG?0<GV z?DMhGO?lvwpI;uE08GcmX$R!^3^Z^btFCt1V>UzvFMR}tg%#UD?@_f}&f3ZNK+6dP zyg>VzQb(qI0FBSb%3e`1fD)4!Q5{0|CWPJPJ=8vTVu^tF8+<AQR<YI9DJQJ1#O}P1 zW_=$z(26;7e=t9G*Cpf7u{{pNRK^5VqD*3zF#+X4PNcw^Sj;L>gybQdXO_+5sFJT% z1#&K2h^pFoI)an`xgW?r(!;atUn*UBN<1@OV`QO3o)s8V^@gJr3-I^aj1F$1nq35d zoLMF#RLCiHcU!I+>X_r4qT0Ig)=NaFb-djIbVw>-40S)3B3HiQb~pll>;Sj=kuoD! zxayf==2w>_*ZKWCU*iz!OIlM@e`RC4ifhge^vzUF`(2gCrbmi1IL_|cX)xhIX*j=g zPCKA|SA{0j;UDJbwXIwy7Lt?)R>z^1HeT#-jZ-zE-E+_ELTtZu2aIDPCl<-7deXtA zH)QnRW(%yXAD_N<Vrf@UiGAK|7BOpg+hnlYrP8L(>ht4x=Mh-u)6VgaGji*l03tX~ z&uzc3tu9t3WWu*P$RmgFeypvb|A_tz6d&npn8rv$IuKYV3;zOG=bF=F2`-kOAGQ7j zq`oBEuKl<lZL1~rRNeLu)@^Uj8n`p%Q<Jo|`Z?=$@B7XkgZ%5wu7zB<{hWPkkeJwy z2La`XIJSztwh}(8D)g|N+mA#Z6TX_S$zgaI4Pb6O57q88IPA|&$EhJ|g7uIg`O!Hi z1xROx6{nz&oEKKw^aj!TR@M2*GB$+pulu5J)KSS+aQ}0;n({M-0u46{1T{qHWFE25 z885F2c?+cC=cjVR)-C?%TcK3lM-Fw0(_NH201BQ-=Sra@CtT{R0BscqsN}KO_omuh zbF)<csw#Em3&-!Q@pD)68Jvd;*$CA@O__~y{$$}l(`^2=fPwOo??Y^%Ikfh*(9(;4 zTb2K~*7J|QtX^KkGZ}PuemMPCL;vPgo7opd29p&g+Z!>i+oSw|YyF=smWkA#m01xl zUcBf)?)@{1?cXokW}(hamQM~GP;=%Nq;DgC|NM_i!+-x;y?U9An%9NUxi<hh?C%@p zKlD?b%Jywy*I0@En|*H663=fKY~X5lc<!t+y_K`uln1lDJvsf41nGb8Sewolp&j=s zX<j(Mudei4hW?xZgkeK27zS)`=Ttv8X6U*6<2e5PgS2<*<(ve7Hdfc<x5~%T4w+_F zsX(z724eE+_-_qpCflec?c-BH`j4<@Fc{2nYwH)Y`hR}jn7q*O{h|$A#u;sfrL{HR z4Iu_Oz>5C8_kg4JzrM{?qp#d3C{h!`oCgT54Tzy|0eSS)6j+h*Tf{F%_CH=;U63tz z1L8_?nk5p6+}EOGkM!k3wqJ-ki85b)EW(%o(6@}+%XWQRtegk%p+q_NgX%dyYSA%d z_4>Vice!rW!K8A>-*Fm%{keWaNy^u8%VhtuDz<SVr^*gDs47L}MW!C49!yl3F8if` zkVPNI_`eiT-J}LKtNL|uvxAjEW_MDgSF86n9R~-4?_EJPT|V4!4Eha^E;l4hceTNB xFuv&tRgi{lG~ePwRuA~yolq7b7e6^vGcGp%S-qa4NeBGg)zZIRe9QLv{{iX$c02$8 literal 0 HcmV?d00001 diff --git a/website/www/site/static/images/blog/apache-beam-flink-and-kubernetes/latency-graph.png b/website/www/site/static/images/blog/apache-beam-flink-and-kubernetes/latency-graph.png new file mode 100644 index 0000000000000000000000000000000000000000..3257f9b66df1647096a87bd7a18410926fb773a7 GIT binary patch literal 81918 zcmaHSbzGF)7Ouk30z;=TbT<e{cY{iobazOHLw9#~cZYO$NlBN0(k*!hKRxIC?mg%J z!^_OPvG;n{Uh7%UdUo(Tc?o1he8d+oULZ?Jihg+U0tyKEzZ)JF@)y_a&hN#G&o87z zg_K-A9<-S`kW08xa&?$kjXDXG9VR}sTf8Y?;^EO4GA1W5b8-?lnK1pzP@kt#O5wzr zDiJ$I87hXY{-q?uF#6LPT*<o^h}6s1=}qVD6I^HAwT~_!AyCu{R#v`fus*LiCngyw z*;<(G_4VoXnKuk75wFY1dY3O;;En1P^xqGt16^dmzaIup;QiqKoX67M2*^>Q)pK`W z@>)>lLT3HY{-HUnWMqe4vEO|5?^n|Ko3~z4c#F%g_+{E1ID2BIRwkmO&ZCl;R>BhB z_a8HWsj*io(-eHe>QQ!``c!F~b+{Fim1vI=PTA4m8|nDlf6fSjhHbBx3ev_gb*MU7 zdbn4{Y-w{PKw4olAjsh>WPCr?(cn2CV|&Y`Jf8kp5swdQLq81`a>zy@0>6Wc6f6Qi zpIB^v1V|g+LxacJVjqMYK0ROJ0uV;#g`6lK??AP#BpxZ)gj*;CITZQ+b8mCVy+7HO zr188gWOV~{_d#ar1`vMzd<o><(9b6-?kpp4#h<<auOojbuDfqE!$w1qxcYRaUmf(0 z%i~}qMBKn0*osvs#GU!S2IcZu(8FPe_VIw}sr7*Cxb2Cg{r(3le}}D^^BLduDb>yG zS>w$~u7VSSu+U7DY=JJKJz8kr&S}Of|Mt17VXeCBxE02@*}L)oId&otu*G`4o^v&Z za~i;M^L>~@tLbLvXnVw<@p!psU{_#?0!WUZ>E$Ja9?51R&>NoGc+w@Y%fk9L!uvK+ zXI;nZkv1X0qavkK!PlPu|6>G@Bxu2oU2b4+fos*{?)9%Hjt?vxe7-tukKY-=T-T?C zgNCdwcj?!_@V@-M%@sCcT~imp^^pLkx5U)awa2wvyJ?H?!XD+m=*TV-^Q-etJI;E@ zo7PJi#93xC$<C6pw-h)k=s*4Lg!e0hrf_ME)5x%v*jaI_)wrV$a-2aH(MXb;SvgL9 zm^z$@VYQcUBzD%cP!j)f-U8UX<Xnp>I}dO4@||dJqAo`yM@I^kxfyFU*{*I}b#+bz zdN@8RZri(lAsg@83Vsb`r-B|77}>(i`nCb$8ZQg=K6&FCUeE)7=&FO3*~EVcRY>}W zS$ijBzz8GXWT=;{uGB_R6vjR~ljEl~1T|Ld!$4FqZ73l+XrX>g1#+iy7$wXAU7}z^ z`?bWqmUe^m$(>Mgiyamel#c@))@)4ausa@*Cp>#5<Ev*6#eY6>@d0w|@P48P9Yf3i zf6PBc24w6vi-H`Um&89;L!32@c2~Z$IE)Bv%c3;6@U1X1@PDtHI@(-;_<^1G#FP!2 z%uDRa|J~W2A0%GN%CCd4v3o6UTVaEJ`bAHm9PnqkJ+Gb4=RqiOJ44AEL1q&=6%wYN z8$=cA#hbJwnh8SZ8+j>3S|Yc4%nVt9|7AJ-enKs43@@)HB5BO*J8X;>QX;GLdWK|( zI`qCd-NCc}dy*;9EV($Rjx3+6K&Va@B4V<zNE?{2Dm{|sl!U7-jDObde~Ak{#bHA} zU@sw*2z%udFPbqRIjLX0D0V(JR>J?b7*}+3-g=*gT2#;$0}`*?e7_}gaP?S`qbuQg zw8+N)4m07y)I&YyYWI8@yazDlnAx%~Cla7pcVhFpY5w;JbJ6hj8?tt~Xk3fHnb`ye z`g8#Z-deDlZ+}O){~ihnsmP>yPQ)?yDTjJKPU2U`50899i!SpredYi9f4?xhxG-{5 zgH#s7OZ}1UY}nNSU`PUXc(EX%#{SLF*#9;@N^CR>s(~hpA@04)iQ`Dtr5;~&tf>CG zx*byuVCZdZ_5W*0`7Ydsco*22E&%Q9_thjth@-<hC5^0o?E5~4y%`K>Z?Z)wP><Us z;{VY-km(WNK%5z<NBEKGm)*SLo<!O)JBs4CwJ;yveg*DY*pM6QA!g=UYy9X&dSPr0 z;gvMK;(Ymkc@JbDGs*3-bQm4LlfARr`26xy5gz)6Qhy6T-^>ipMBpBtEZ1PqYtC2a z@dvVL3grY=s94^#*_YdIsrDZzVFLqBc2pB7nz^19aTXR9MAy!}ZXRB0wR%D8Xx*bX zd*~gjIuc&S)h_8KvLwv2)DfX=XI~}-igzM69gUZ5`L5!rvvrdJ5B>kK$L~DDCCz<K zh~t00K|cE0f(hXTLx&m5T4i5t5&~tOa+5pFXMP<?Yvsuf^!_{n#)&{S{3F@Qty&x; zR_cr;oQoTuq*uN9vn^=Ckau_aE>W;*#*xpJ76HIBVYBA$ohS%Yg`vUCOj=GC9Z%y6 zybMRazajPt>FcA=YPyK$eZ$L2JJgfjdd<SlOA5!KhkRr&ky^0-p?D#(wOE}sVY`DY z846a#M{=7z?uXV=g>lF@<^&cq`CDt!%GX>9zZ_Q<;%CakS!3?MK`y`xpkeU5+Ojqm zJIz3ziSs_(UH&RdLY7(h(3FaQM^@hS{2K?Jg8jHz!R&XEJ3CH};X&>duQUca%=^nz ztoxZm5|3{Jkpm6{d)jBY5_K%qrdsH|q;caUg~|5vt4mzi#UnQnH%|dBCcORLKlS+$ z^6YW7u>1`4klIFs=*6Z72?8hB0*6d(wh=xd08Z=flF5PO2*(-Rp*|p>zwpbHiY-Vs z_%?9G|MrQdW|4mNcC0n15B=u8o8lK<Ou!u~u@vcZMoJVs3AG5{iRYQx-X`6yhNPI% zA>1Ks!<|7xq2r)Y_E;-&`y>SNC=Av^9;WSVVZGgY?{T^Cki|>Z5b%VZ(Q~#k8N@=Q z0^V_Ra>!@J&3G-#g~#%*HA761q9m>F;gGg8(Og_DU{6%@oXD2PxIK@nW4N%TaZYrd zpj8Jvrj2-8@Z#^VX7MZa=u}ZwYoq%o)URoCLEpPBMOA*pQX!?f93kn;j2_ls%itls zEysd8Hn7=-4MY#}_ZKU(v6P;fjV)Y}ESzxx4R(Iv;_s-cn(&jd7fiTq`iXG%i}w}| zcIq;Zpw!}LQhIvNw_DhRghXv4;f7YYW!M%ZY?z;Ut|7(o2}?N+XJ@Tu!b_6h-z^&$ zUu)t10L7o73^~!|Nofx5U774RGk^VjoJ^kMz@}gjWuHNc1G|E-d{Ct`df4+5t(uum zBcxs5%4<RX<ni9ii<kk;A)DL{CVEd}p?fx1`OB59(vNg;TWW9>Jzx&C_!qB*G1A?3 z^*l${Px92oWW%dBoLz?i(uA4Qt5^g)%v$`BL7Y{yDhCIN3Dcw2w`kANEL5;bQ=DIf z304?c9toTkcI35GNCoRtdN=DL&WM;u23gv3`^|0Q)QHA-oP~2Iv8hb9-cCJU>e__T zppQ;X>4MBj-#wnvUImdP+lCgsy1-i*f+G39BJ$^1FHe0E$-Yr6fnbqV24s05vpRje z^De-JzQMyZBsoIEb>FCHS+}MSTTDCeP_ob<UXQJVsFUW{AjuH_Ps)QO0-GX}Ba)-| z{zSXUF*6O=f4f<QLpa|rKNKJiN#HnO8XjK@wBoEGwH0bIp=3i|QBo6_Uy!R2{*6D` z;VQP(D+^7ysT<*oU<BVvMIQjTD$<_;ZX&2O(J+E(3(1_HrE#bUhn%bD=X*ca6j>bm zR{N`ej{_y064;WYqn5k7v7F#|%1UiVXT|`y?hoaD{})d~0cK#-amkDE?nX7OW2%X2 z#L;pI-uKqP%+s0t#n+IWFewGq1(V5?(naSiaq`3Wy?SqO@v)Pt|345JlG9(qhPs6v zTjpBf3^1?~!Agrek61EqxDcGg2K>bgkc`P?2G#YL$tIn2Mr9}_83Q|~;__u(Vk7E> zXyI@(#&7ul7Eq)3{o8n;;`F_;LX7;#0z8*36=$wmsL9oApe+H!J`n5v#{NB0KzJut zV=7yntW~2@FA-jwLDxtXcy-N<AnyLZH9-){=lbzvxKR&X@qMBI`QZg>>P+jDny(y4 zxeH@X*3a4XH=uzG&XfvF9c&`Vc|NHD@NCagmaVjmhNbH1{MW58V1H6EcTy%z85$nF z--`>A7yQL#t1cBD#+#pfC8PE7KkSv2B4|s}+w5k^Mv3Fj)<te_Tx@MAlKZcEV-tXe zP3OqEkT<8sXiH`27=jO87wriPQ)ii6#zloBwclmWvpZ>!0MkNaVKC0K^R!+th)?ff z<mow*QAWnHOIr9o_@Zq5g<`!3;I0_i^C3i)rM>Hny09@_Ch)qT7O*2$bA)IoWgqY# zrW2t9_$(T7QqFHTywpKq(&I25(=iHU4B7e@uoF$K=2VO66HD{hk{Vh-yh4Zfah;Go zbkyYa`%jYv2u_v&$4%`qis`oKT%H!$DSNe6DqKhYWwSRV<d}Px!5q!p2r|iUYDsMx z!)SG>fB>@i=6#hM-6rGB)Bo^Sd3ZoD^h)_dDM^@>xY0c}8_l=2>iB;-nH3bY&BtcW zu*l2_k`+T;y;dV7up7Wf_Y2&Jwu5cUONe9qF5Dm^%tnI}^6A7J1<zL9kCa2>uGxO* z82e{b3z%S2!!y(E6#1O=;bc3M$nTALmwM`sN-3AALOe!F{yp%3NTvYAR_OYf-#Gmb zj+H=zFBoHzV0EckAJ8YK*v*xOyRH0ZY}F)CESw#m=T|*sP=>)czAn2fY<uqfw%dNo zkm%c@w}9FTd-!3T9JP+Wik@Rkzx-zA8%3@(p*RS>#=XGl%$1okC;fZZ*57*f`NjW0 zgU8;}j3{9F(k1M?xZ<Ob?(^@5@$dv@cHE|T2p!*Fy@KR8uZJreXP&1OcU0TQnTeBA zs?3e&3e4OMw()p1aH7!%Rp4#pTKhFW`cPgeWx5l!iTHVhrFirlfy@jq%3;)djtEj9 z;32G10Kca%o1*uu<>4Y2Qi1<QFVE%lS9rjVVX#^M8{3Udh2yo!lGs1K{6-YSg{$Lz zZe^Mha4o<^_4G8l=+KHCfI6NoL5<nP^wIOR#ZLlANjiUae=<m2q16^arO<k+Z>#1N zp&Lk#9s1hpm&eNj*AtgohLoXPn?<H4dx1|_*Nm9#xB;%~5K!T5BE+LLJ}Zfyc`>nh zHb^R5ybM04nk_RiBjh?NC@_2LxU#}}#%IyuwRh(vv*oaT{#-8dAVa=$@JRhM-khI* zhW|0hf7o{s+3fV-<3g`FyZ;xL_JC|-MwKbqnK-wB=fpix!h$FBA(K-S1dk3+>(#>Q zL08)Pr0`rf8rE`uc^G#O8o+o45ntTD(+31d*gkELNs+tVWgMdiwSia;M7MlNmuiS@ z3Sa;uhd7prT``6&SLb7I+Q}<k91N_?w$}<0@|u*t!u`^#G#3t#>9oIeJ85y^)BrVY zKG(Umh6khPW_8NjA$9i86Uhy;-_hDl)5Krn_>Cy=*J`yE=ec(S`X8GJrM7UcB~JTu z?YJi-Lw(COvaBIfp^i0D=64zAvvQZ<{1$m^!@({Kl7w|RpS}F#;l6{fqB6hzgb}Ma z6B8d677;uf3{GC;4vUZjVImq>S-sA-#ntqBI=Zy}Oncp#pr$;dlrSqtalM43%-Vo4 z(9CSUo@K1QU@Ct+Xmh58YrYOa1YHh#?5N{ZKZEJr5`1u1gR-2tsef`~5+<!F;LEQ- zKT|dYl)(CkaTGDd(bXZBN(&#T!7j~e6&m-{)GT4{%1|thcv3|y%{EoOqNwT>(>Ot3 z8N-B?O{$!Q70ka^a~fDX!sslS=+Tk}#P&taA;2ARs;1bEsKI+s?5`Wjf<C^fD@ki6 zBe?c)*3h0t(1<@JU-uI{DYTtrBfZU2<blvee^LiKtg-y0{m=@tPFa9aIG+1dp_@9c z^&h?mlFZ~j$5RaXEJ+assTlIoaFGB3a1RSAS}Hy~LvghKj`9+|D*npV3w!D)!wUhU z@_mDDcv#Pa{!J*obou2$?O!bT<!~EpD&8L(pa>;|*;~8ig%^B3G>h=>sAmT}P*KnI z+ZTjtYX^F~w;=euMJo=e&XJ!ftsvMgnSn-q`&M&oYP0*;F-ResF31M(>7b6I(IrWH zTV3Aq?`f&^eNXD^PY>B&HN^kEt|2;(+GoGzpZOg3pKEnd==D$ZvNeP<JzQN<L&0I` z!jKzf;l-4OmB%UX<*)oZ*1gPQ$50(mJsDon)56n?emOUW>2(GP(_HZo;|ydqyuN-b zlbvzm_q8Z2U9=W*6SEj~`m%S6N16M7GM7gtg5R<<PwCJtw{yY#>Fi+4a$(M*^$z6c z-&r2zB@a2zuRY_7UX>vyLaHshL8|Vg7H72gYwWc7R|g@Bh+t3*j0ie=!jm=CpTre- zOa@-}q0ivlj!4qtwLJzgjvlCE$v-E{-}n3>h_TD~igEKH!;Bt5Wf|d2kdR_48#o_J zpXXz-_iGO;?XKquxzYQSYEZo4K$TXg6yTGo6Ow229#@|9(C1HDJMkn19x#<bvtdc< z>W^qJ+kRA@4gZ%3UyIt~fAnMy@os0M=QObU5vk~(Ygh)}9TS$<z{7U8*1Pz|ExUUA zs{PSj?X`y=!PA8VF2+EZ<?!Qs3e$K&V45UXv0tr!1E0koH`=%p1T82`*L>C@+)7nJ z<BspmhJF6&ne5bqMk2ht^<mxep%G25;Rgl|6&az!kG|5s+L-qt^ax9a)6w(HBS=08 z0AZ-dy+V%936^dSDj)I!7zb`wI|;&oL8imoYzhs<()oX9SAzkc5MSUPV~4UKo6Yba z537cA)c&G|M3ymcWMb1tHBUQS0FI_3*58~n3A!nu5K3xse~G~<(03n82R{}mEYtVy z>-naJb_+wpPwGKo){vc5o=KQ}Q#d6w_;$qp6ne24t74PmKko1)4B#`*GUllswiO+| zXZ_Z=l|GaMlH~sJ@z>1hMH4tNLY{|52jQR{oh}^jEARD#<A({{wzHS|r4Ex0G-gtu z=Ar~t^engaPY&8^h@@!MnIE8(0Y2*vCee%?AoNjGRfNAbH6j7>4pfhL(qNO);DLtv zI`x#vmVU#tXS0$BhKgK&5L6N8Ek^Oi92Qe-p~^^ihY``~2-RZ$Y@Tn)Fq7KZJ!3IA z&?D#lNL@q@1kiaL6*zrUXn66jCxC!yJT|fw8=+ld5F$&TJ0%)BD7InspLKiFMg@#6 zqom8Mw~&x4WVSckFA2Iej*{tn|CNJ8)k3;&-kZ3%vqVf7Q-lM&PO#q3J=5hE_Du!m z@9o(!;M=`h@VT8IHy^3J&J91PBrs$$+ZFxJ$V&a503YIyAFoI-5s!%s;55(&x;KJm z%@AQm$H#<bF8MNfgX%9osiLCrlcx+*-P@_lH?ysH3B+jf%dWKrIqJ%_5FJ=27^7(? zM|Z`EF_;L<vMOP~`>23Ygy&<qjrv;uF1Q&<kg2-vYq+_A>{6dpZ5HIzZ7^e~{J%z_ zzupIGzrEGGzddx08Zz9j&fY*<Zu$gQ*?v_(j-08}<acDkce8h2Xdi`M?1=B>Ef^De zd5Qds^jEVMW1t#6<3zJnw>lz~(aNXBC^%R#MyEAFMr_efyUit}K-qIG;5B2O{*@v_ z?rVTgRB^h`@|191clq`or%I!Q5+d!0AI<eHe1U1uXWo}-ULN}##JJ+^h`~uI8C?_J zzwjS#Z5~l>BmA?~==E()0p90Z-ce8EyXQ?8vpb%EO<#vyyTe<#`^KkJGOgP)3#|@2 z86pCL%+AVH7bKgGb4G0JL@gGu!>Ym1*Nu8+L`D$mXquYqDXvUgV|M=59V-pQ;a3eR zej=V(!RJ}xfI&%Y{1lP^Xf4fW0_b;b;RMyCDm#B9`?)wtZ!}PmReVT#z;XUg;1J+* z-i{XmS~(|-*IM<u?X`u_J5`Z_T94mD>KMN~b#c=(FpQ)c)u{JwB0QsxH*YwjrSq$6 zT4;m1Dj{&MIZ^;^zW21h(i_?v6vV{nMN{!4s6k@A+Ty$W0KMnF+9AVr!`*KA2GKpn z%A77lodvYHPyedcW&K|24MSpI?;%sw+v#viR+Jg`>vSeec01Yduo={Lanh#j1Fw2x z#1HC|?7012i^}ZnnOH(s*ggn(XXSiIax+V$xx?akpH{eRpOI4(Ys-WB2XTIdXSSzt zxG2<>I%#;b&m0I%Cq(bRIf(6O+Ojx}YiVNOz3pN4n>SHa5LX<`(9|CI#<bdb(eZ$) z_OMtQ#Oo5~vi9Ob?93qJuXiVK*%UF%kx9fZnLZ$rFNqIYB65R;CN+|>W8IcnIv+BR z$$)%qp3gmplZncEbO_1u;TP%`+-{pS9=b@1Uev4|@SJ}wu~(f8JpbLjPn~QU_g!`q zB?$v2uK@4cfB|iVd<q}(949^@d;6I31{&<fFb)~4U#y4oX@)bkQnD*;)E2zV95Y@S z;n>6#HwOc$W&{R&LFEjCuTbxrRW!$-^aLt$SklkuF5Zq`jp{}VY4DU17lyOWsf=4& z)x|Vv_YX%%wQ3)0D(7Ab-5&i+bCB6;{m#=!e5{n4s?U}z@?<WF{mG!rml4zd{lS6O zqU2u#iQ^SJmuPWG=udwGwz^?dTQo*DUBgFLR_g9gJmCkoqUr^7PbPQ{DvX2JnEjNT zGkJ+xy$Gg8=7FvDB9Gx-?ysk8$~vM)RI5)ipP(GPFQ_D!z5PZ*^&}WpmLkLFSNlhg zA0%8ATJNvK56FNXFBdIU8fx~IQ^XrCR#^C7^mCuA8Ghh6dAl05>w>&O9Ag$g*fRQB z#<wNNt#Cb$vx%sSb*LvoJ@!?}6;K|3e;vQyCQZV_V1H1qLTPxAeY`%KafhOuMyFK^ zm&1eYdyl{>QE%<eXZGwj&=daIuo*}quiqYaGxsK~uVO;j8biTtq$%82$!yZCn&$KR zpym4pnP5y(9c`h#TA8A4q^p^9m`oCe(01j#mKL;TdyLam=jeS;6a!CatRJ%3Or-%N zkNDIkei>MAG069Fo~#|LcZ5F>)>cftN4wtQ`{mq-;Av52qrVgQ^!b+n1kndd=-om` zq*OOVoZ#r_e_wmZCVvyYp0BXAjhFxypsRjZvMW^@mtW7skIFA(i!L{qp;aCpJWlbp zS9TRDjs*c_e;O+Dz_z_f{q`60IK~Lte$3;z@_6Ef6sbNuYy>sHxVn?W<|B3>yp<h- z!_%XH-qrOJd~|5y`>+yJYylLdgXBjca$?8qwDBzF#?QSga39cS?4w>2f~zUw>bR&7 zd_jOEcltBh&b#=%@9+T>rwiX(9lz4XOm3^p<%K~8OaVMNq0k_j>xMU~gLc;_QW_7> z(kLGsD^GAc7L*bHnULNfQXe*R%{WAl8qb9Q)pfky+dwE~pe*aA#m)UUMA=RQd@94R zd9mHy+R<YeR~FxWIC|)@EOfJF1216P@6(kAX|;q+;^>fg{KBvEr^w5=fba7ZX@Qrn zZc>V{g3eTULpSzR+wn9I)9c6I_WAznE9w1<2>-6CGWZ+?dwCnWn;1>pRKoeNd{ay& z3NXAOViWI=p2Pi5etn%qiTZgO+{FGRdwUL%jGByoS|uCD$_%;3zxeX*&5@+j0pKEV zDW$v{*fINvhY#^jUC<+i1|M^rh09jA&e8zBv@u`ABvh1$PjjLJXO)uju#*mUv&F^G z#NpXyjL6h_D^m*^v*HqHo_XB&zs}3J432j#+EEt3@!FAv_1{+~Kjss@|7Jx_>_l>> zFA59{J?Zf5za=vETgadpkX=F$4Yj4+l92)pR(Ly^z!URj@;Dk$&JH&d6%{oH&-NZp zpnXgcm15$$m7iq|z@ai!Y-Zb)f{-T14wfUX#edOW_6mU1CXSgvcK8Oi|IOsXKzMmt zluaWVeDcvA>M$2n#Tg!spqMsOc&wpNOt8nt+nbq*vMjNf5PkBN<8LA)M{;z+UiSE> zn%DHKXHA+d)WbvMB|c5>0<1J}aWg>``N}y&f0}|!ufdX-ubM0S9-8IUDGsZ;ax}P? zs`<Z#k9a@W*j=l7M=T5<M0dpm3e|^c&@zr_lrATq=OF+x#l!L|pb_#e`{k_uSXzV! z9#D<|b|ez9K*Ph78u5K!!rm6&kO~jbe)%hyCRq=pjo}V2!Ut82<7mKq^f>_uJ?f&C zzqevVxTF=QKS4(W*}f-t+P>OCuc^a8ujv|ykKoHSzAh$(d$UT&|Lz~|Z%alDG>4(y z{Mvbrx_hBG#tB!6Zntec%l&b`()PRa6NmNVxp&Q*8gr-;BSY7NMTx{{1$HV0zBG0S z%(XUpflC10f&Z7!o1_I(hpZ*dg-*W~V&%h-)ivdV(qZ>4<N8>NxrnGG1S<NP!-AKU zM35dOy<G7nk__lBL?=ErCNu)j2GV)oDULo0JfZ!p{1UL8TZpYES@3R(-AsMKcv(5r zbM+q@>1G-TpZShb@zGB?r3HLlrolZg$k}(Xf@GgR@mb1xWDGaRcoGF~m&iNRvbXY} zJskuDs4CS7+t$wx&aM45Erf{k%*bC|^WX(f=+F-?z`_dG^oJ|c^*siEpEQFKDizW* zMmhc2sAn1C`V+YG-B0iUh5=(o3BQ`4vHHfun?iEfz-<4W5bnXBLO8<>s%w@SZbFNi zrZL*;BA!lleDmY7ki-4$fTT5Jw&9PWxY2+b1{7a=JQx)?Vmjo!157YTEyP2gj!_$b zfRFMI2F=EU>Ke#k39)Z)<){ysCl25EaTB7&^%@&~5%H}!h^rF5FXQMa77xXdk1$?t ziXI#VToY>)<xX!fXRbr|M|IJ!C<jslg<QUz+gx0MdLJ&FZ*T0;vss&}_M@3%1-c}z zRvOPDsB(;qnXkywWLzE{o*!>6NSF4T-Q_be|GQeJ0{B$G8^<As{<7AmiX}C{Hma9w zv0S=*^HvCiyOhOLIVdRQ=~_>czlsj#BKac}QD6Xr^mHF<1kwfGZDpeseHah3!R`2k zAdF<djHFI+T_zMC=KlWcnf?5Pa7h>SwgH6S7P6fcyN(YF<jUdyOL0!ZgLPk!-@RcQ zX-dX2czxCX=7gi}Tn^%Kx>~|crb@8f2_Nv16BgnBsSkPM0Y1eZp8A4fFl08pPui&E zv?eW-ZwB0$J+Db?Hp<iKL%$!ZqLvS<;cR-<qIO{SQW*-&V7`@uFiw*F9n3_VRaq>5 zO8{mlp%0^$tS!3%^SKL>-^sa;dKr_fsDPW-%-=t|i|mzJNG>;{T6w*P<(78(3i>mG zPXM1s>l!=j)8&wfN~|5r?dhauZ-k$@6Nb`;bap5b*;sgy-5&GGrKI^j?yX=!U=4q- zv1|+bkH1`B6$U&ypoMZTAsvAdZRp1|)YJi~6wEisdj!zlFH$sni>yCH8aDX9(shJ? z9;zaq0v1@;f2fPYLcmXjs;VHawu!&J;d8KNCP{m{E|#Ia8!DWsbU$nr$EjAouu{mt z9d@LzCOjY$JygbSQ_cd;JMkZ!Y~I<Yz+0)umk}LGg6-yZgfzrD6K2L$@^)J)CmrBZ zkBR6Is`hXyFvL1JRatR@bePBi_%i|+u<oulq>yibQB9%khw#OgFw(2t!q=agN#aWc zhjQ1I`o*I|0}a)!w7bUBtVw`=%EY9ajep;>IS|M2`dcrp*QG8dC#5PHdMH_&zmLM9 zuqM3U4OlkaFiqsgsUofg!K5c$jlN;+KQ+dw5)l5#d!1eGoQv!I_@e0xNOde`>V;{{ zKXbCH;g5lrE*O)2l<EUtPb9eyDOJAy5Mgd=_(DYlX%0hZ)u1^+XTY`n{<*)cohcge z#rU8bN?_lSBYp`2V1@#Tt=2wJ+H=TcRtvZ`RYI`ve2r2+G=LNrajPqLKr(-(bzp9u z^W@~zU>gZWR#tuk(lh5hJmNm{KW))J-TT_I)!8MU2tz0PRu01O4$znwD`6rdr($@{ zT`gsgH2|cE{I?7(VH_gch4|;(K>17XfUaSWIu$X8Z%0edWRD9VhuaozamN(@flhC~ zKzAje%$ihrGIf}ghlZLAxXCY;tr>GoJYrQz*PNz|qJj303oc85q&NNBpSmO<9qHaS zIMkgnN9_T1J4-`H*_Z%9=AgcOqx<s-9sgz#;2h_hNdac~wPD&8cem(q!xMd5++b03 z3=BnFT$hc#o$0{z=~u~S)$x{$5Gxbrry&*NRy=H@p6qzvy&e^4Mr3$xlT5<tn`>x^ zHR9<UwX9*bh_xjS)kP(aaq7rS@-XseC5QVe1f6QcdX@HGsL)akS`FDNuImEx39@|z z2fvQu^wbzSvJjdTRro6YLrOG73ifW<-S0l#ns7N>i7dA}D6Co<7+`px?Fw|-tn&FD znY7igU0(1HcnUF_W1IyNdy~TV0KM*=pQtPLyW%Z4Pc|enbuB%hh|})iaTvXVlarGJ zC~aYZZA*OqI?i&s5bcDK`Vu%d&Vh1s0D?%@?2ZKPD}(425Y8r(gZHTMG{p>E(ZP7w zaptSXl1<Tqz_37FFiWP|<xqJjgz|$Za=3^N+(%*cRe&?0&DmH6_6LiYOg!K$e<7Su z>1K!TqiM^kIoe3S1BxQvHCm7%t3T(en*@5WGeB=02?5~aay_cvtIy&ZW*t1g=0Wg_ z?<LbN04zx!>#{Z9Aw-SYzl*~-H1P^Ka{X$>efsUx^%or!bD!5S*T%V3?o0Mu!8)tN z@e~?l$-J^luF4D(+KZ!`;4L?j$f4H6s^zB_;9qv8o6g@Vxr<fSiLO5xDoxd#!ehH1 zlVBhbbM59$Au0ngd`VQWt{l>>VOBbOjWI2zFl(BjUiEmg9$BRnh?vNGq&rxfc?87$ zR(wsnnG39rC(OTEoDWbDNm>peNgF4lir615DROAj!#zu1g6I=Y`79uf_tnNsp1NS} zQqYC?MVA4&`(3twE;G^>iF6Arj^UwS)6ayz!!NU_=$0w3FCTOH*CaqL>!G1Kn>`N1 zTSGI}%ibsO#Y%|8nNOG|0=$7;U3#NqOp>botFz1X1dZFi%@?yqc{6xsZ8txD+OV+k z1VWlVXKZ!iUsY5Mxi9hA7+~kS;f)bvx@wi`AMlmKQA5gRy*z##iTOdWVTkc3ll<9Y z<DrEN6T$gDt$D2EXgz`5y!Y*!5is3e<H+HZTXxmxX-K`%CPQm)=bQ4Uq`Q@Cp9K~J z<Tx*^J=f~{cL#UP1S)E#raL?87|Z%=SwDt$tvYHl8<(!$*EFX4g_Xbg0Wi}mbEx>3 z8&AT??|*X$H>o;bUS6q?Rr#**UGXBsR(7Md&muZSgG?(W@l8P>1q0H)qehKE4AkSE z79*7}Edw?!Y~Wa+Simp=>tM`$tTGNFcIZZZLjK0dXK5#S6>ZIsyu9YcA7xjiUvuUb zuk=3^Awf!H>L31|13dn2tR;TS$!z$b+}ccY+_;Hw#xVSwkpNComr8d;Nb@mjkO0y) zeXa*l4juJuz4X=hLq%j*v1+|J$V<hrdzbrek;C~@61)Aonl*SW`Z)&6tB(c5tqryM z`PelAG+=mf0w81$M4-uKmSkkC;RjJb<+}L1w$8N2_U7z&t7ch6s*SRc$}SOYMd%&w z6d`|;_>P8iRqw!-2LgHWcH553(vCA@cPds8A(=#k2LZ5EcQkIjPHcm``7_#x9H}3C z6Ci%V3WT4tRrB8AxbeC{m0WI*{6J4K*t-$h=k$jOi$I6BesvCMPOM;!z3Nn{QsurY zysiG~ERj&;gyaf>WX~l3u9CPit0{VUXt3^Ao`)$mFqg3pLh*~`pcz3BqBz!+qtxRk zwr@h9rcOQG>p^i@6<#A9{r!F8fxTi%;3j8cEKP64YMe4T&SOW1sI_Ixcf7lJDyV-F zAdfUC2`7C@b7>|={cD`Va0e}C;`E2=0xBOTNX3z(#a_MOP@3=ap%9Yr`#z|D-((X1 zFiqMOQef@7ga8n8=c?E$#8Gr%F6g0dsI+N7gy5KSMJ~lC0!>K6nL0^@mJ)}r9P{&A z$oE@(8G=9TI5+2Qa+!G|yO*I5*&JS1^L~BqT-wqaNFexK;Cw^P0NzA>0CS8Av;}yu zk`kreQ64#Az)xudTw5%y%4W$S0*<15JM0nV(ZCpcBwK5xZOW#2!FLk<tjb}xAvfE0 zMBY!Xh|U4^i5Y@bHnkb>Gb|=znb|wiMsg{NVxV4~7AiHbx31SDkXn>(Tqw-5U$BTC zarV`73W}RzCrGT$1?0p^1~f3+UlP_-3ohBmYKq+<0Gz*n@sV{Y-9P5Gg0zkOcvWKv zg450r3(GqppCgU=KJ=fvV)TrN0no9?Ym8ov{!Cgj%xieSIC}0C7=rWkXrrPSRjx2* zJFX$w|CI337LdHl6m@hMSltem&Yt`lk$ebAIT{kpHQ_9o!EDZKYAA0gcdugvnZ+C~ zZU?5Jx?XOoqyfMu3LQ~>HN(aaR#lNbW)OzCTt^8g+u?u|Y>Nb*9JLYxO7A7JFt@$O z%*&=NN5%{{OFK`6O@NYRY4D!rGffbjr0wGuDusq_{q=NcU)_={IGNK~7Sfwb73BO$ z6N~)QgD4wa8iaiu^KDDra(Bx0YqT*u^+m;KnqE_*+`Ntuj0=M@Ci*D8z&Zcns!M8} z9A1<kt?S##Oo8)v-f~Dw#~ybrM8;oeUc7J$qW$=YjU%dOxY&En4NaGgUW0GpQMKCT zJb#LJyCp`C2-S6N?XL*c6+WMo%~pdJ-`dQY{$&NcVoeULqsO)QS>tAS7M+9x2xhM3 zp4Tw5fB-0m$)GxHnp}l2=Y}0Y1cE8nqsZ@Q@*HoR+EoEQeX-(|UO@{#2pv;4BHoQp zl|?88Lh)Ka=!^F>$@Zc4=)uOeDF}cr-Qnu;q~*5S*yEHv02su*5TqVE2+@XuMb(Yf z;%F}|s!#B+!&tHJ#e~S0^Y5>6-a~b*;i4I&_kz3XOST9jeCXi)Twc^n#=5BV_hWB1 z9+S<^B0JE1mKMSYrN`zhjrHGPh2S8TDvL^_7{Q<V$7M9wP!xIR%4VPF<IRkg#T(vP zawUIv7XfieHbGO!T^Ysh_>`M#UUrUs8Rt#y=TRsGj{N*YH67at5vx3Mp$BQDf#b$I ze%4{YKk6TaabX3K#nCv%iPoh;8kPHfR!R9*+<K$d2`84{I!AB4IQ!mL`TU?wT4T+R z?=I&G5u7~h#T=D5*ThvrGI)~|S5KH!u(<aV#~eqQ8U9Nb>Q|vc1X1pI@ryHMNUXYs z_jZyxict^iE9awg(rRrLiffplmB>YX1iAn=ZL3195Ng&0Y&Q2XS0atJ9W79(6Rw#s z(}r)_K)}%|H9VIWPhVe1*9_-fM*W~M(WXv;h$%p*u`YK*d12iZ4vWF-%y`Pa6O>qL zR^}$-Gx8vK{*5#w$>(6})8SWpZSQjJgdIucScV7{MWaNKZR*(9Wg@xrWsj*FXWP9| zX9I;$3`DAqoG6ia3egntYG3qF^MpEp%qB~?dxtd#m1E>$gH+%F9nm~vG55B$jD<xZ zFiO^zP;>&f2*zI5sA!LyGD@GeK9kyTf)^NnFslZ>+w1sjZ#m}GqUQqWcKhVoMvDVY zm^kGK!-Oq+F^->v9qOl)F()!IZHxQQJll7#tKQz=iXFDn#yahcq70tZEjw|>%mA78 z^z9KObxaF&6RWj@=;cV5d6vJ9r!hf;K{rXYZtr0y{8{P8tUGcZOd+H{H#{XBEAx+m z%>0w_@-lX9qVEM<h#XL8vT=RV{UNcoWDt2po!~{N7}Ur+p7-tXhpb`dFa4!qi|qj$ z)3nn&MK0}Qe7=ymuF$8IdU<&RRQpx8y$Hi~HAiV~&k)g_iAm3esgD+64B-NetODI2 zTBU`GAb#95iL2FA$&WWOraf!ln9OjAz@G_2JK8h4=yVG0oJD)$K**<-VjB2OlBR}b zJn^UbRKSgVsZsu0ShqLLk=pZdY^W^XZhuL`rw)CgM<=`91kPUs8)(4I#LI^Wo-EP} zbRaRjuFZUl&4dJ*chQ!sNDO?zQT@}<aiBSrkTVQ)?p{ICDm6(;vGkLPWK?jj3W-Fv zIEX!$R4Pbw3-ml|R#w}DJ5@quc<K8On2pG$vt{JSZXAKr4K$Ie4Kim>iz#A|KJ-~Y zZOU@f&!bUTeovN2IEPef;u#yYZ7Iz&64zt28WFOZ5JQSopJvvu?scrwC$BpR@;D55 zp@tF-*!*%D+*)nw3Y6?74rHrs+OXt6YJcorip#a7_c#0)@bVVXjxY*^j#5NkZZs6- zzVeXXBLp=-s8i)Bge{A6wI6NTjl<-`A5=JZ1PxCx>GZ$lR!}`4?04JhuPOz{k^;j} zlaE_MJa{byKGA=KlyZY5zWP4KkKQ@9H3T-rq(H6UuuAOw{TMq$4o;X>bSlik*x`@Q ziW#ko#43EJf)@&i5b;n<FNYB5U`~vYKzd0`2=T1dudD3G^qJtaZlo)yIZMnZ(-QW# z0BWl`iG#gOG1VvZ^<xB$_p;Y7#)eR_kxfu=Pf;DalD)c8%u8IPmBHm-@mdeNtrOXp zr@=q#Ju_9sL9`U7>0U`l6WiC*+V5xC78voS(W|Y@i?Cy;t>wG52(Sn6-5EJNh!J|7 zd%L1ek?%QIu*>GVQ&7er`v@S!>_Q5D=WOQv%a+x<yOqO|YvGiqxo&;t{_Uf%K&)Ur z$2UWLQes9+b{_yfN#6Eyz@VsylWLbmNbNZEHf4sHSYBG-Tq+w2!d(F6RCh$r^rwak zvIOE1VB@umAf%CArQdK32aUa^9VU1|)`;ab3MrGMa5x}}d2(7;i6agsxJUx`fZ8Th zIisn)Mf=G;LWB(915T%R>1X{$|B%V%)PT6=TFfFu-sIop8S#w*#hC~+vjpv_2e=(o ztRD6T7(CKQ>(A=QTjn(JBU)YHOM~&^x<{C?-TOEq^(tOv#~@~Du%=Q0h5Jh~MC=S+ z-CjVSIzz>msf{N4zxL5>Ep5u!mEI7;mGLaX57%^VR{#h{T*_j#u?5JsMi|ph%8h*M zw4NGvai%p;1az}L5Hsx9LFa0(o3}iLa-OU{JqT>{Pdws^^S8N2q+4<%#M5XJRM)A2 z)s8}XI>FKc28Gf@ndD$d2N5_lW~B6buHOX@0U(6l^;uf{XC`K_DMV4%BTm=J#*!>s zHnnS(Z^->cuBQmnRCn%Io)i<#*?a?0jnN7(+8FiG@>hr1njt4kTMjTl_Msp^d&?LY zM9jy0uHx{`g$L|rK(;tAsr1xbyq%HxX%sw(Sj|{rpiA9lRbSuCEz0ESv(UFjI`0m1 ztCR?WcPKK>UBH%Br?nM&_09<4M~>Em4Bh8@XCdkbdfH_85Y1R07ha5OcTxuEc&#ra zBb`q9)I-4}{1%9$?;^^jSp_YiwQ(g3G>1t9Iu^*vvBTFP$_B0i-{25(WHth^JS64) z0?WP}(?CVM*BC*X&PSW)NQozPcZ<;&_TEn~UY#3KuHp}iL9?ND^h28O&vJwJYopf2 zFX0wK>#OL|W7(u9k<Zn3Mr%h0so8O}^6;soKuv7B{c4f*WW~^5$^br~;H=;zN`oW9 zC2MwJ&^DsT{Z5E)^sJ*IGd4+d?KHEpJmqCoq9Ails`V=kp5*|1a8?YJP!^AmN<dXY zr!;Fw6gI{-31ggqfp&_w=@-s&?6E8&u&xK}-7-X$|1|x)0NYDJhx7T9q&0lI)r;2* z34tw9t{yPel}w}$2Q;r(uH!dfn!U`@^pXDMveO2)`KWY*r2zzN&<=3E(<1OE8T4yj z2-Vk}BZX0BsX#2&wil*@W+3dlWKnXl3{2w+0CC1!iW-~7-Lt6-2tYBhfkCJrks+#^ z{<p!<+Hv8qVsHI_+pp2<obh=lnTqPXQ~c7{k~4YnGf|QDaAFV^5x$jbQ>kxfhhQ?{ z>F2!WPCT{O#RZ7Zf+FmAMBB7*U|q^7!XbCn&CZ${E0T5A_YC|7`!Sr><Q6BH%quQG zN840M94l)_?-5}Ol-A55?;|+*r9@#RmQq<FcAbb_1O%TF(0q!$qCGMdR)#mlNGJiy zm`HYc`iPyFfn``fIEH)P+$^N#P8sXAAOpHdCkI-WzzMMm;)rI`$Jx+Ote`uxO|A$w z0#<Nj(1Zvt<X9BwL%4IecXWKkK!|eDQ4ldK)nX-=gk4v+u64gj)$wdwsmsZ%7U4de zj_!km_vgOMN0{#dE~xvfL~%LqmDC5o38FCRKF6tBsR@CHF(6IxkLa)(Sh8y`>NZLx zb)J$fZ*Le}wKTZ#Yu*t0rhgvE0Rg|?4n~V^8$lo{<E@VoFX0SJv&u5g$X7;)%83Px zo0`;la6q`~dOgbM&JWq$wy?b1ZDDhdFEcW7bN18b<PmO{nU^)hrt;%J?tlU_7dt%_ zkx%B<2!tCcXBU~Ty0|AQApR)c-GL(VDx=&`yukF59`4kG`*XP{thx|X(}>!kgTD{u zV~5MS*Hr`Fgf1gcLa0peaQE8^e*AmoRM(sEWy4Xg`yylPKEv|&^8OPN_r`3QU^I0I zWx~fg`Lk9)tYQ~qFIAs)D8H9uyj@|&5jUWA;w%2zLp)1n8NuW(5~#2mxY+em!%AWZ zS3c$DMi1NjmmZYPwINzi>f}Z2Pl9^!(wqthdi6EF#%u#U8Jvgp_4-3|<M-5uIV4Xt zptSd2v*XJ<2P|2F%aYN1X+yuLiF!Dvn;H{YWMhrq5s|juA{k<Py<oW7>wHLO*DgCk z>Z1>fspXHx@-($1^I4j)@L~5m5w6RB2UYxqa@FF2{~dd;jTCHF<ttLoN8HC+8-uf_ z``2NQ>w!rw{60!)Tf$bEy53(MUwe$V4*Ytpzq*4v+v_5`_J}pz=!GyQz!bT3JoRBq zD@tbMcnbSY*Eklg5-t28`aB=Pj(oFY#jYb2M!;wmMw9V^UQxssPe2SRC^Mj9>yJpV zv&rRmmIbvp+ap7mJeM7uj&Bq|;`ArXMN1Tj(1U36)UQtb|J8QFTU<`N%}gVNt=)^N zBz|R+Vm|j7xKkCrNo?)mS`(o1B~RAYd!pB&T&#a+YMb&HRDOA0cDNg1g$&tuG*=W> z@=f>}IZuSZ-k0UBxd(s8pm584*sEenh2vA)rX~ck>WvEc@?Qh`?K{|G6BNCYHrM=d zbfjxdS(s5pWXW$w4SqFZYKzqm>g&KAw>Hkc{qo?FTdo>4r6Z8hS)2d)f@f)>)Rg?| z^g;V>lCcs?o3*~~YG<!G+7_J{Aoi6O@6E4R(*%ponz)mv<o==bJ482w;|>k9kT-0! zn0lzReZAoGfyNRLqM2yuPss$~!9;)i;_Y{3_^g^a9~g49q`L+ggk>(ef&@MjwUT|) z&g;4E72bW>-NlI6Fb)rOiZkhxnQIC7;h|s<&Vn`iRkt>ir^Jfl{N|t!%FERo6Lpu< zD6>YN5ar{@taoy+Ub|);5hf)ZT8ABUG!qlqO_%G9jX-A!h*)!Ixoe_f;if&bj(Luo zO0$_-8!KK`O@jqBO?9f5P!UVC3RBa!eKORZ(taKPYCz+YUgMkb__UeBfkm$I<4E(H z#Y7s9mDI%4*(qf`%+f%CyEp6__9(GatMCH=QXqWJo%#enPXF?~Pue|YVF8Q*M^#mm zU-3SkXwG{}t6%wdq_}UBkf-QfGS?4)ls4WGj<(tf1r5AC*wa2kd}(Vj??(c6Totp~ zX(t?XByTCxv(lD@^)?&cxv(WXRU>dkzlTvPW~*anbf&Yyj60^rfu$Q`X>3g~d7&be z!Z~vK3GYEpr({R*P-sYi4!RMyQ;e;6r?DKyEqMXER03~X8R-K3x_BG?Z1q(Gt1+>R zH;r$lA#KWGGsn;FBJgphnd7TYBU>Y$)!~mguyXOstf>`<*2QBk&@HrT_r?C1Q><4l z%<!;w>X#rCSo#ng#z(eyScq+9l{s;*{n6SY4yGfcG9C=Fd)F`!A0tei^~i}3Kb!*k zzOrG?Ou!7`#?2?5%}`@~e>az)8<H?%92MjNUO3Ys!Qh1>n?WV}WxC7ntV?+`;+~X@ zfXN{gAR{$p!kS}*CNH)T8dPoh-LFOlDIZf@Ovg9*hY$-7J*M*3gBO!@>wcPqZzO## zj*%?#9U63<d%^>&RTI%3EVX}PSrSw~Hhn0}RQZx5KhSf981q3V>}9nrokhz*>?p1x za{KXCEpr2ESfkTt(UKUHP?$ak?xSljlL2j9rINH%OS3BXuIH`eD8|xLST~O=gkBd? zAF#ojYR3SS*i&XMO5njecDt`-5LznwG}QUgzUDj2M*|Lv>G$BAi~+6r#+n*K61;Lv zN56*<Y-o4wz&6{QG>Bx*Y^aC;jzmRN$vR$3mUTM^_e6{Pr*JV#r&Ow|i(%ngM0rCc zum>r9CS5sq_$C)rS64>+m7?KsJBsK1o6-E;=|Xj>Kyvjcuv+W;jmu*@>YD?K?7<OT z7Fg~0-rB@o!^dFy4~p`ZL(T?eRo_<KMBZ-)f^B?GnJ<)U1&Uy)%xl%6{q$_7;^K+$ z%;cs8l|Be%OZM-CR4P&vXfn@07X@{Akk(P(yb)2;*1<&d7Ny>@F76Fa${!u;B@=?b zeK#zJh+a*K)qvJa%J3q8x#`_@4k|p1f|w3N|2q!<51i1OkEzsV{HFeRh7LW{K?!nK zq}a{=y+7zAWgj)S=XAuHL4#&A?K0?wrI_y+&2m^yE?ZA)X;k)PqAwop=q2UrN#m+O zG~~=!BwMQ!>^>Txohy>et@(TyyCk%SfUf9Qdz2)za>i8)oCiw@fD|lzGgpIX3I|WT z<P0Li+grU57<GyX_Q8j9qT?Yib>|54mje(P1PjS61&%Dk053Zf%i8f=s2bk4*AuTH zJJ{b8zp#(=3O(p)M17qJDbzKPN3(vUd}V1=7N%E>f6C;PU*{%*Y7HCcVgc`wvvz&j zuL-&>xQ@@ahK2OLoIWH6(!;{Q?Rrp?s0gaZ`EQm+18BQ*bi4BxO6uL_Sm%b-%Lq$^ zi>sMKtKZOL8L`z6QRSK|NsH0*x9*#oj6>iw1IjBeS)E@9h)j$_ab;~ZhQlNXFy&aA zJQrc?r7`a?VHrs;FV{fxp9(hWs(LyRQsQM2lVU5GvtlZ_lc@D{R}qu+5x#kLMQ;*+ zo8*Ml>@Vwrm(DmdxElPYk#vO3Z`a7E;+oJ}Y2uCy%6?_&8yF}7{XQv$M0=J#T4kpd zPd1jmDlyO{E=Pxv!f*KXw6>zfo?@Y{NfIf{9V{$H9TI};Ak1xY$=sc6w0#i;Vj&o$ z$RJ6_$C?P*nMaL4bH}H`!6@S$8Vzt;sKBPMqbYEM*}wy`zWF%UMxP;5z8?3QBeci4 zH@n6HtKLL!Fa_bK!8ENZ3Ijq-RfN(!MqUQ5Z*?<;^FSR&Abt{sQYl6@I^>nbT-^tO z{4g4~I}`x=^lF!+3ENO&jG~ZjOPgNGLf_Pue+Xt7$0Lc3rO0%Ts8Aw?>WvB1<FAxE zsCb5d<@BZ+po*MNJ+>H5Ry3kljONjwyDC~>xNx$?=X?Jp4zf!LbNM8=)vQKCKVK@b z^CP4O*bFro`BCJf>_UOsshK8QQYyY5_9_XY7RSu-IIX#*7L4l12k>EQB$vw6bd2F( zkzpX0bpMsEiOD#1?Q8tIRmkohIIeQ!{B@rlVhTJ9I5RFpM{oEV(wUW(Rf&o_!iymu z;R0@%mz*~QhgqyS%}K*Szi!Jvlaf0qzhX$n)t&=N#p^CePYw^+J0SW$h7K-2#?#-) zlnxK&(5}G4Ji!ezKBa?D8v*YnC4GQlMjJ>$EV;3uDq=48in6XwSp>N#7-PAr3SqH1 zBi*6Bns*U)qUSG9wVI~XmAxeL^+CCPQ5-)~4<YK%dfue&bl#EzxFI=isufD(8YC=F zthfrIq~Lex(jN<$YKZ(ml43%n@a>AU7x;F%vy!=nedQj`z9C^2%#FtRo11ESDkAD- zhN00fSlc`ytqW8uJ%o7P6=%4ip^wZ4JxoCiGLEIa*#IADt<l);bL@z*&e?J+SfM4& z1>c*$>o&hCbHm_cmihv94)<Mm5y#?^u{TXFrxP)0R#=$d!Z<vtZl_g1_{Z5A>Du)o z3d<h4ck50^K80jhhrOfN_!H}h-*tAW^8HeaiTrSPv;Q1f!v6PqccD;U0b^a=uHNN> z`bgQ<TH3VDd@`WF5@xJ(`)6<CcPVqqG+!aDQI1SOuN#7)GP>rX0~g<sR95^&Qp8IE zRh%4=#2KNf<~JxcO9l|p>|r;gxs!KK*Luko0g)kf4b?ZX2HAVaX#mn_F=w-qYm=}r z#QB3uK#r_1;j!KQN%!6o#}07zGN#!esC^%;2X<&|-INt-{jaKVEPzE}g#0!?KTio+ z4K1U1GX4=-N-t>DkE7v<HJL9=`KXNs3@wGFmIQ}@<ZctgnWuO%T`~0Zp9m!p4I4x$ z7!gE-*yl_aWbn-^K7Ys$G9>X4B?;5t(OpRh>ypmC430||JmF(`eO;AQbPQ^5Gtbe# zVVIW*Fp9s{Bx@qgH)8m;^CWn5+IseY`6e*>JQCFLHF8~v6B~|L&)@)I`rLCe5l*+_ z%ffuvqQw6JV?dn0$Ub-*sjq*8!T_yvTVlLKkQ<*sy1P#jhjJz>6mny827eV}Dd;9q ziFyQrZsSTmj$(LG5?NYXWlT~wBCWHPeEp5N+5$1QStN_A|J0R#l&Pe<)`5j?%R<1v z;vI3!o4yy;{u`SUQqh+p%QBvV8+oO^`LTR2{4H{kSZmU1YL(nRauG!GBJ>02Jh~fA zF$YSQqj-n3C_bCkF)**m1p^leZp2GLAn3cSicTAxAHnp!i&CMLO|pA+5B)IZ%$Hvj zEK;tTwajjZ;Yxxv^qcW9l60`6vXa}89ngluWNas*E3*jBjWoP7r=37lm^HD~CnrRw zgsGkQE{Jkl%Eh<|5*$u09OxuHPKgA~)2vkKbmOgB&=ku~TLF?)SuAF65GC{cxgzBR zRLyav&u3|U3;)ekE8R2gCK0qUNa7#}1YHtUTGjRhQOaw?KS?0gj@=hyTLv4(OMY1e zGFM6MkOtN}*#88LvM$D3NLnAt)$e}$UTDRfJYT9Dv?|yM1mhoQ;%^u`)`)Up9%k|> zN~uYqaAsjFvbtg~w6W{R-hNdm9xgIs$Y#Ts_xmujvWjFP0wXJkUOpw?Prb&yC<;U0 zB7OZA&?dW(9U4ab);$#F7EsR8!s&eMSY@qK4UnY9VJ8r*1;JP%x^1hB?b<t^F+PDC zWr=fKjXNDr&4Mupx$X~<x&M|`hk|=-)GPx#)&fZaYE8PCb<5qk6PPWlZ;FN9c2SXQ zkk5ja?BFUEp5Djsun*y68JRpY9PAgn2dADu=(!Yb{L>%d+rRsLS=fkyl1wX9je5$N z9g3?vnz-zyuy97hVk9jKQ8fceOju4<bgD*sP(2yj4MRMC?#x+8)nD1!v6n!U1xd0h z&_yH@cN6MmVaVb*41g6AN0BL{LNU;+DXmNmfWMuqKDslGt5g|IRq$X;$DLtKR+Qr& z5;S4f0wf7e)MZ0M(LXK8Lq%qk=m+{Du}dQS?IICX8T4qTo}sKQODhDGxIGJ_1_pex ztaUvfF0Bf5QPA>WRP)fQbX8<z!KZ`RM#0P?mJcFlGT`y>v0tguy^*Nv#@I_B7_i84 z&rY`PwpNh=K`x&|pS=cc@dEVC>nLZ|<QPt+5?Eea!}3-D)BYGH{9yzI(aVeXRpCS6 z^EslGVljlze}bj|`X>my{~@x&V<;t3a@Wje!3w){({Q1q3cCbC^g&v{xK9NaS5{_O zow<o|Na7iA9C6WPSs>dCT0{zInCV!<W4A68&9%G@3%rf3m>8NbAQ6fnvp588c@SnH zW53*0IQ0Y~Fg=L(fA?SGgWvyO5L=xCbAa0cyK<(loPm*yS*SN*)7L#L_r++wgOcSI zgN%o0Y*5){JPGi5&px`0Tr6lmRTq*#6pJt!6}>(x-9Po7U#C%zK<t7(Qn4ywW(duj zmIcKx3C3~YYLBLIPTb7=sRvpd9#81_#|tUEbv=vOP3HXXUvu-ho6$AstZflmv;uFB znGC-p5o+BTG~t(Wmdm1QGX46y3FR;Xu@x^%62W~qYN%~_ZN)WML57SMs5QF+qLRw! zNknr&XvMr0?{ExiZVbBxV=sYV@FL&QajDo=VBWH2>SnoS?*EVqigzK1jFAw}WDaJc z3wmZ211o-6SnR9eC28H@zB)$5st3^sbl)o0e)@CF{JTFu_=`*8I~Ziz+F0k7-qKYQ zRi~u*zDq_yZuc!~u5bZ#OuyuNW#6P_hHegN-D@*i3Zhg1#xl%q7ST6nV0hU?_q2hH zXjzU2K122`T4f3|U2>7y>bq_iPA!4pL*|A~bC18p>QEOpC;Ct>Hl}Es!CvLe&R*P_ z5-c`(sPf~L{c>N7#vkqhi0_lpyfU5e_?1Q{2n4}kRT)Ux0Jsv-mIkca_%hq1{`2cJ zjwBE@><|gM4ETFkUxEtdxWKStxr(`9Qf6oDO&w-B*3^1Y22F7T99ka+mrUgKebm89 zOB2LJUmZayeZ=an4pdtrm=bBqv`fonK~w&|^M??KoNmQIgi>Xh)>x=ou?^fFdv9FB zVW68LQHV&lMUkfE2K#-Xm=gpdj-qWm?IjQ_utfJo?UsC>BnR}5Gsu@UNf`7-|G3=v zNxYBUa?E>7^hNRi8{rj9%-_QB_`A6E?AI7sc(S{wXlP#yAp|ojVplICbnbOTzIX%q z(MOiMw87te6iAl1J7=TM??`aI63X8Q%t-<v7d}qzD_t6CNj&&@N1t_wP7(<IEEZJz z=P%RvnP}V%mlg>7mf9SvMk*}bl8l?7YsGCil>|ad$1!pHGx(<mp=VPvov`9?S4gMW zRn9`;nIsR)-908gf%PC--WNlBhfMpFX8xnUuTunKW9%6c8;kNi90i$a!GJJ7T{F#R zHJk(*A`mr*-yu_R84St-J`2xe&>Vf%<;V(WEBhxTv1HVJFjGJzW$l<ZEXV>^Ay!2> zn&!9nMP186L9AQB=zB(RYkAS{k6YaxgqoNLW?A?SxGd~Aj6jq`UtUk-u%1fEI0srE z>cuwHgHh6?%YsDUuy}{VKsQE4PatP=W7G%)SLL)2ut^BH%R-@LAH9RrrT;D~oYLAP zQ?ThDXROJd-^SmWGmFxFIlK{tFP1_o7eU&80fCYK3(2+L$XXTz34|&4#W>hM_?xO) z4*Bs>B<|jWzBrF!EC?g!mz5P6e8{wK8CTE(Sf7M0HKWz+Aoi(r2qp=ZjKOi<FD0Z) zLz5L{OHtpkWV~5)f<VxU$haR?iDfN^uX_vltSgUrrr!JCQO>OLNH<qHD7P%<!nNsV z+XkFE0uh)U!qi;`O0J@0s11}n3Q{?XNslR(94L*vM8-jILnWq6wE`<pbKXjAJVNPq zia?};YuFs`J#uGGx+-LvCKZ>SZ(KQd2#yFu4K*SmlybiNLnJ@@6B%`H%siCDiP76M z)5>%*vDz42&Eea@2%azG<hHxCKrqJPz#F6zOKeD22_wPf>XYS)_XJToMJpu2%CZ=( z>$0#z`ox8-HTQz$rpQY-#e6V_m56bO0<DK_<jssA4*?nPKr+w_>Sbfp2m}c~-An0$ zz8m0cEcVK1{leha(p^-~Bv02AgJZd9nAnO-*8n%XV!3Jlb{3v0_|MA5XGa1?7Y$t) z+_-Bms6fq)StRdYL-N4`6uGhBPNBUrl^!i$qMy+MT8ge%7Rn|SEjQs}yvJeHEsqCl zBIIQJ0!eK}EDTCwA#tD&JA##R$wiusF{7ndF;0p=Fg}%a8%BJl41%WHj{i@K16xKc z77m+c_9^F@btW()EuGT!+i8g3W@;t^Gh1Rv9P%S<=qQ0$8-9wlp|1UblHwtgiB}Ts zt!ySYV-k#5=Jht1EkINC;9BHZUuq+$8d<A8MDg&6qE!MRSGX)AeS~YK9_Tun@4ABA zAv<$2R)CocGEP8D4E8imH=-#5v5Re)->pyFwiGneDaB;Sm27%(1oJ&lu-x+yOTCXU z+I0uMu|DL3KB*8avu{<t3#Mesv`nUVt&LcHk3nWT@3<8LK^#XSsGzB+8DCKAS9Doe zl_JM&E(=;pxW5I9I|=0|COldP(nbvayblQ@gnT6}10z}gpemH-W~?Ol#b9~sqrkw( zQne^EC}?@mYxNzPgmh!bY>e+LW)^`cBb-QJd1eUt*(X*?9RmPa7G0(u<`0a0rsuMj z!q9qDuH+MffYhsMyG_>&D`=Ww6f-X@@>MCwbX3-akmR%G_ZEw!#NC_7zIY+aPuGfk z*b7P3`iZQlDu|5?=xtzOvlq0EhBE~$Mx4hT$n?w{(=(T>zFXM|j;>b+q8U(W>RH)G zWy$##-nkvE5Qwr_hJSnmgP-*wwOL=0(}Kv%DHea?o-_=K&KvtJipSx=>b|x>5pxNx zkov?;Xp>h(AFV&{wgXl6K|>JVoPoqz3{T&Ffb+li9>y;Z+Apn(juMF6@-Xr<Pvm6E z!OgsU?rFfPfvIo*!Ag5<^zAr-xTg^m1C&Kz91hUuE1^S!<s4yqPdvmSg|S*C5L|7g zul^(QPtMk_vUVu8G5`3=!9Yr`oLPmF<bg#}xN;)lb2)+Fq?$H_V&i5-_LB{Q&rYU) z@vpdFB3s79ase-<Q&?F~A)gkCG@pf$O<-kp32O`E$ghmZfIPZDSPe6M^(V;QdsDha zSWCc8>W(82Oj~4}02Qtsv~zG{r&6(iLU<m9%`s@1AaX`pW-9P=Ysm$SZau?fpdXvb zB_xuAh=;Br7<r07Y5~E_y4A892#FTXZi=>raL#I>+$MtB3jF30g8C}Ld0uCICY0F_ zigz88n=4ofi*xexTZrWRhzgPrFNBfQqB6!|F*=EHe;@qmRYA&)mijnRb7SmgV~FQk z%EnPx9z<~T0fI~8a)ozTMoUF}e^N7K4Fzu7PIm`WbQw5(yt|UC>#QJP6LS9!vlN4_ zg)K`$*eA%t1d{h3K$~B%0wwq00D-8X7St=|ha!mt(+iJrckTtP4|Jy;C%J`fOOc81 zE5!x*jBo6FPD%*`A-`)+mPV777CU6Hrvn19EeMB^(Ggt=WA^SOuKd*{y#H_B#QlG{ zfyhD#MX?g+P-MkO21RmDngzdA{7E;r<+=U$Gf~RVh<-Q3$zuP>9ascH%=HW=uClI# zRgsk?+)Jrh<Ys?`)aV~T^Ifoiv>{Z4UJ6WXVx^bc)K}%JEwfyDJcs0Z9DZ?b>B$`C z?oVO%&IBf}j9}{e7-q!37aq^Z&-I>tg0sK#Ym9$0WWRk~bd*4_;0S#|WZac(Og5$h zroQ@XBtHLBxl*fVUTg_ectQ#4F~NYFh_h&=9>f7K9wM@GXgoyo1j4f7&_tX5=Eo@X zej<Z*xM`JDSml`&xlhYPo$n=<pWl-kmF=ClWa;j$Nz2@}bNn<Ah?PPSR|07~i{+6i z)&IT6f%O^&77UE~Osi_7uP8GQXtuPMoYqYfvs+>KqoP0SR#6%T?U6jN(j6^0^-zyM zkQlItQr1XfJ~D=Z^@s9rKqz0SxObGjRQN*4vtnZ`vx(%^7?LZ`kxr~iHvw1aT<uQ> z2QVLcfq3*0w8T>s^-Wn0nza<z_+7Psb4AZ)R5awuDe>$Q$P_k^OwJ&$g`^e1y2iTR zESp#e6tEmDNMc}?bwPGmzPSLcl$YCHlR!)bU*N^sBS|{yICz=@pHC_umce?`MIp#@ z$}fnVUyzvO{$gQtLDH=;x*3(R4P$~dObLa{07UMJ$@9*J-L_N)#l#@88yAs_52BO| zAvZpT)Z-^GgCW$b>u@N6kP!D2b07PT%z&U(Hs6@(pKcpkAda(?%3x9!QJQ%KbJN;m z#lDzngg|f(a5nz=yqKk76{8}V;BRX-<Bo-_452GDLDnVtu|o)75`-|i9LD0~IZR$1 z!Su~BsYo|R))4bWu=r#i4_~{B<?aP653S?wrz5!a!3%tQeiBPFabyb>X>B+R6*DMz zcTLaz7wDnSVRCc0_-*m`f%waPVIIg$;|dY}`yq<lGe$JIaU5H!2l=(vkQn<n$c+7! z461HOKS`2MMrtb|e%mZQ`TaBa_;=sMi%)yRZycAuap~zi)`tYCp9>(lA%06JjY1+1 zEh)xjN{r96CKf(A%)BX&`)4+xXBu;Xb(BCzmP<uM)7Fl;j~zMl<b7lwy(1MM{mo6W z)Os@jl^rXOiItM6eP_mX0>(kmwApMu2s>o@XmGCXei_aQgk{xW304vWNy_9N9cgJ! zl@-=lH(pCgA3QBTwW@7gDYK#*3mDO?&e)X5^0chr$j`|NZvN>5nHFv(5PWL6a#fOt zo_GNtt;O-xR!Xd#OGi$IcF@|)4Xr+XoWl=4N#d)XJhoz1de&iaYP>&7Cv$Tn7Wa|I z=mQqSOS%?pq?k&yT;(P@Suy#?q@w5QKO`*xn_~4Ik_vJwy@Kh`pd=0~w6tH@TMEC- zRuz+TNt_o-hXKN6nxn-#Gjmf&th|DR@24<Qy;ktwA*&_Xmv-YDxLvkbkqI(T7{89h z<sTy7|Ama-pk<5o5?CRV{bYGP1K+0TXY`v_t@;7%BT~t1QyH1agyxXg91-M2O}Zhi zIr*&##3ye;6BDmoh-g{(V@dd;=>w_PjA!_AcnPmP9K+*<ExFMZm93)!OtccP2ocTv z;+@L<al*4OlVd0Z?jSvQ9hv?YLOG^z*!~qq6Nnw~+3`EXxZ}J<g)jTi5%U5qC$yq4 zsC5^VSM&)k8p_E{xrox%$B-Vsg-qY)$oK{Yc`AxlwwfWD7%MrkSjv!C6@+r3g2@`9 z>9T2JE(~rZaj$n74_>Sy8O@4!T9gV+u7Hn&beY7UFLcYwzXs<ZT5ecRM*L33iLrJL zQ{Uy^(9ZwMb31>TEwwg^%yU891}?$mA}5<cY&nAEXA4q^PKcHE?43z$j;>1=g_fe? zEJzCkiABCpM%PFVuY3~3_1+A6#zjB&<&cWx5cI{d+_Na}Gjn?a8-s$7$MgGQ1h&O^ zqYE&*{5Qz0{TvnD2kr$UcYNLRw^B;M)Z_BsEK+9e6GJze)hcFKkP5NTi#--(=z>`2 zJ+&4a;$2t7!iS2UziX|3SrCJ1OQu`n(t@%+xFX#!k6*uw>p%QTTo<;;fp|woP7p<H z=e-EIX?;0t6682FjmEPdJ4zrp>Bxa6ON#pBN}NQZ$POf!Da~p6To($%-|jrf20EpH znO;V0{B>yI0r_%kkdf}JR7U2#v*&p|@HkgW49v0@vSAPvx8XgO-HUtZV%673V3=H? zif3SF&$a#S8v!R6S-FtDb*2K6bRGIxYf|FGD+vW#J}#933qdglKKA@F=D&Fpk#B#7 z%&nixLQC`;+wa*3foSwM9ENBv0AGAYnyC3cWFzUUT7G#dPAXG2O-$+m-Nd+9b&e;K z*vO}`oLooO`T+U_0phW!7)Uf*9|oSv2-3iM4bSIIjQEOpG_K?GCwWY)(X@T|XKSE^ ziiL|71=*xIoW+sW5|*2QzauI77JnB>LNL3IaBfqqfCETt3AyLPcjUk#1pLkX%T>LV zsc<~(F%jJODn0lAiiz%T<$e$KXOZpQG)#>XUPi3w*M^`?+(KdWI&x1xkY$5&g5-1i zbjH|}#B-G{NYO_^6VI_R)PwA7uiT?xm!K{@lZ|mCnY*teb>%0rd@%bPKTm~L7UNy+ z(n_Kw`lK8Wbu45g#r!_`y*vHY5dTiceaL35d4e$zD}p@G!oXk2KA0AyZa$Qf<M;mT z25wBPVl~c9z3ks`4Co?>`UIJ}DU{zcF{i9sSsY@_8C&Z}^jt%7;v&rG=#g_)Lj<CR zZMsT|)*c+J1R3XdvOYy&ztw=_LL7^SH?g`nC6%lxlx;q+fJk5&Yrzf7g+rK*B`_S0 zqub}h!>JMUZU!-%)G!;%V>+6{SSX7ze+r}PaSX1;@InyqeqRjz3t=%P!gw|p!qb@m z?o4js%ybYRjBTP@tlsAz%t%E$EfgCYSNl`fM}?xBkcy5g;`s;D(v-f^w}jQ68LV8t zhQN&vv2p({LgS0r8l1z{(1K`-*ch0H@5$tDTYNBv*;_+cxHp2ON8{rEM=|pGW8C_| z+j#WjcQAVX7NWxgSnqm<=*$udaciYb<x8?)CpOfjEW!TX?JWN4<0P&P6n2B3t;boF zD|0I2d@P6Xd{D|hd2Ix9;&=GRSD+_mp~t$AkE|fQxx#_B?>lN>A3Q9>;s2-ZJ(%m- zt~0?O(^Jz^HTKk0wL4DrxO%ER<<M=ryFJ}KvMpP-9AsIto+P(|_2d`yp7e4c`A9xy zkOV;j1d(&jzy)w~&N=6N0q~pk?Sl&fAV7lDbIG%+HZOR?Is2Tm_gde|bTxPu9T<PR z7rrF}qMH+l&c6>&&rJk}`(fx=fo<G`z=juTPg2c=Ac)t+L6-}U!D}rg+2Qu|b@wdV z6M$b&1#xO5=rMW<Ku~CL^PTpdLMV;r=V@1~)#rXkH8o+rwQ5Y&P#OzKRS2Yj+qX>6 z%~6%jS~ZSp`N+@l<>y`Z5E%Y#B-UOPO-MuG^doBELCf?ws@-cE5}7r9#$@K}fhnxX z@Mj$dnjo1!{kCIPi@fjO_Ynt`ARyqNrkIkU^4U|fNIHLjRJ2Pp*sur<c5l>1S%gWg zRaT~5H3Vo>vjJf^g1(z&Wg4acaVBNcq)bX^oJOj8W>Bvz2r4j16?!?_!#_gQ6!-o7 zDhj)phNbs2gs*)axx25ZAi|T5`#b<a&DoyVRwj4)@OizweijTOKx3Z+O_0(I&wCtr z;)j>NBl0)><%`~7&2(SzsqgVO)YSPJFNWdDg|U<fV^e^GtC&Mjz-Y9Tkmr$5&v4a0 zDGYH5)|3;vbOPo~1mQwZt9U2^y|GhQnir$0>F;DaZMrQOVa;9-#iTr2w=jzQ={fea zBv8xdv!mM}QHqJMazGb!!5r1g946#i7Swg|9@KYa<L}n@!9o-DW*w~^T1ZYl{>xAs zMAQS*s3)}c+O3L39{15T#mtF@Xho&G2POY9Dz-^f^!=#ldK8E!z$L(E+)y(ULMz9n zPnxDk%ZiE*k-b{Xi}vQrAs!duM)yq{peoB(K?O&%_XFKI4fnDF0}RGxaH;A|5F!}S z3lQEoDCk7s(Qa%+7aT!M_U^X?#9YMAe$g!oZ^l7r*%*%ngtg#K<dmx-Rj#Sn24+++ zK7eitW>j#TIipIRD^2BW56^p1lerWZZ>U$k&t_LF4wajlMbTpwZQZSzrVFd`+mrd# z3jv5Wai6kErfN~YDw=yYs+LKmku&A89CaWO^}y)&V9pc3b|MW!Dhq3-0Dqx`WVMEZ z@V<%&7>-oE34f`Ad{gEx|7V;KM^!Z5YLkJ2EflJCxmP>dkpFYA>SeLkLbB4prU=*R zcnRx~Jgm_Iiu)SA8Z?v3sAS7Z=oRAy6ry?LBUxmF83CIf%)Zx$&M$w6#q%Qw>jMZI z0xAxH@d+FWhwxNyQht*|KActN`hxt;W6>NRF$6LC#zXXf?=BKHSxc3&faNqAF%KH; z8WtxAby^Wwr-ggn9%?D~WTyD^_u9mLza#glUaG0<DMXWqY)>P&d>3msM$z@Po4EHE zH<WvXt`J%Y7++xM-PFL8fR63n{kQ`GS;oGt{r5Lj%t7yWI&tpT-oeQ0y@(p!C`I(h z+dEMapq^bFLD}b2mIPWOPUAs84-ts6*<h~whiI^)=o#n;fY{nLpySS6ocrw3!NRs! zZMfNS2OocWK>>;GhmX*8zZ<=M0|!Eb?^k7EGHt5Dma}qk%PXr2L=bH7vlEHv{46(u zo0l4tb5TIVh{~O1aM9D|bXoMV1r*#55Ho&)l>U7*MZ4Q6#1vxRk&RB@EI~rDNHcLr zG4r+$IbDvcfHN|GdvaTlx=#JwX7Ft-cQe4&!Xf}#?6Ox8LBLT}f0t`=jO?l`_@5%J z|6OF9-$$Y3QCcSl8Jrq5ZBT%w1bn1K2<F3j+585?^8_kBtclQD_Kk{`WL4i`^gN^C zwOg{QvH+;%F>XZb+wn9rdg+BGzq3#GAZJ}xjhrYTPNI_l1g$PwIpUZCLA(Zn00ILx zM`HASQ2+$*Edd4ZBZ~*MeelNzgYQB)w*npQ@9}vQn+>>1Wo%`Ouoo-B3`KL6HP~PT z{vJ)zjGo`oj!{mk=T4wdmibx>71)KLM&vne3ZkDCjnokE!4meu7WIkX5pY<D%Q<su zeiesk)r&V#^KYV2BM2<0=STpe0iSj}f(A;sj_C6*)~hf^5~2ym5zKkjXPg6p#&Mu< zQVhh;O$@9o7-il)3PiLrZYA(2WEHqzj6ySuQpO<+Y8h4c5^Am`)J3?{Wkv9=K)wR> zdkAg$v2uN0UVQ@28(L}}`@HPO@i3j$5qU;m3@Qc6;<X>t$n{Kxf$jh*5mYAj!AQH8 z!Eq2mcUo!kJZk2NBUZb^*%%3VzVptoa#66b65|4x;&^5^1vUP5Mg)Np!411Ha^AGu z5e%t*IeF{~pDP<o$j>dJ8BYj5lr_CqQl@K~www87Wzs%QF9RSr`<=(EP0>IsU`A;E zHay!CFeFkirSq_53kVd;@a9S|$Fqpg)gb4o$#ZG$(Jgg!)$w-lc&FW!=fyx|;xY8! zyoQI@t{{~!AR?f`RH!Q}-|;vx@(P;IIjGrFOQuzTpn&VClqjMlz=VT>nn8gFLxiUK z&7sjg7Qb6e7LW~R)c3TL5ok5lT31cJXencz4O)9!kKvB)DC?rui-zvV$46tCo<om! zMYiQc#wDiFEN2DqL=}Lz@ukmHFd{(`M;ky9OfkNoAoDx`SBF4WC0$ZLX!+_K;?}6T z2YXBpuLV($_lo?|s|H{M)-UDKe-_b|l#i(nq~*D*$aZ`kHBr{jLPr2ZCX<D3a~rR} z`Sw9XYc8J`V4=s~zy1@{>ve>}VN6U;;qsMh2Udx$2R$+#;%D!EfE%}NV_{(lw{G7R z@L|Py0VC%wTvWe1F3j3#TzqA)6`4JKrpybcpgErAqauYPOs5+VR2dSD-Gyq%pcP`F z5O#GU8(T&=smHpfA2$D(0Ez2}yQdII+m*FI6|9l-<%JQ{nukrr4b68<Um>+f6$Ey2 zgm^Sxg_C_TR8&6~aZ^R$)-$6rCS)#V>{La#r-o38T#-!@kmI&glstvn;lXpZ$F5u- zucuMTsPo4QVI%}t5IB%I(z2m~<zfb;O@pH8ENhu0%7pp!Jc;6`-W!2q%A!nMFG`-( z31s{%D0O;AO=Xc1L0+m;(~BxV#$Y}c1**xsQ?WQ^x`*M=n^3J)_U|#7i<VEXn93=~ zZWF)PQf7Rn6h210z5{Ei0f&qPwIX3e4Kk%?9H!CK^6p90Jj|gc#vK*(T8q$!s>sN( zs&XB5#-!{pAh4$HJ3&h->qW%W5C6(_#LPoT1lEu(1P=fZpH&;|v|SF`HEJM%+L|en z7D1MhzfWFMq%Rym@V=FdN`URwBBHS_qcC>~#pPQln}$&guAp2F3dql)k?{+_*wkVs z*gdvVKqV>H?v5+}s(nH;O%MFw0nUq-L_aGvD(935p>-QF7Bi|r4b-c?7*X!!;}iN~ zh^fyEx}$aS@VewZPl#r|6%1h5?8kzemchD;fjBJ8=nIY-LvYVKlqT6GT0pg04BmZ# z<5D$Zm~yRv4O4F0E)^vV1uzNEEUv7hxT=%y78J;0<xQ6E-gzv5Xt%Qe3>F1&9Swqf z9)RF_uE=*{vfiELCd|b;j3S)*%C)~OfMM9}$9gn@B7NlY-TFRIU?C~C_h!rapNF0Y zARM}Nyz=k=PyCPH`Za9NPAf3NKxJ>G35(3L(0cxQ=6={?K+$O@IRi0`vw30`2A|W8 zCOe85q(zH&7_2avBPdjMRhfB$s<f;LE8{!PVER`9v$CfYFx>CdO5l?5rx8~f9Ga^{ z#Tp#`-Q&|;7^GcVehRgt?g#S8W*p7j0-D(^?1&;vOB^evo<?WgcgmH|LV0pLR%0cI zU|?&Fg#oFYpMj152rd+t%Z;DB{f+_-?f1N1AKv=uyGW(e!ibw#Tw2D5AD=s5$e&-m zjzB1cpS|}1obvm-om~PPE@OQ|hwg{Hn4Fry%If;T_jybLlxn?#q9FN9DJfca1d*Iy zv^1A8^>XlUx)Z3~QTWq*9nH89@-HK`*^QuU7U7r~G227rE%JY37ozfiPtuG<*8oh8 zL1bMsNLp{Bm=B{;E6X`W<7(7UDwl<EHq`@t(NV6}vE~UNPv2L10=3krAjwVC%V9KF zR#!BqrU;{k2vxpUNesyQvIv^bKOejJv2<)TpT*|GRTu_0QDhzeifB2xBvR``2+#Jx zwYY$-nOQ84PC+-ajO|D3*m}5z?T72iG)CX!$QymQ@Eh-=^Utqi<Ib{rUyc4P<(Jgc zoF;#phxM>cn&p^w_%=K$3ODYG!7)FNzQ6kzV{dmOr1vAopg><zeHJkI;M7^LHMflA z@foZR&tYps&Oapc=v%{Px6J*K4ioPT;_`p_2=~8n3%afqxR)FVTSCanu_~e^*5qD3 z>p4*HZwf*mj1{q#slbq{!d0juCBG}yoANAA|809X!gbu7EaJgp88gNjmaR27LN&y4 zb$GH>=+hO%MMLGej-I=LS~-WZXrIZ55di|XYyjny4;9hQYuTtg<Jwc_J3e^abzdBp z7R>0?dl1PMpZX3*tyU9|-gxT$ml`5m>$wflZdOpQQY)^ZDBv!U+(0z3hGf!;yl7-; z%No3+qcGeX!S=lw7`m6C?_N{u!a8n*ds%y4DPIEFNJiF#6>}FSVUf?$!4#6-1Oj>= z>{A9fWF1Bp7qQ+ukI9=WFprxN*!CeKnqjq6Rl!*tNjX-`5k<9FeR|$UMpIcZ`I~yJ zq1H;ZRDo^6grQd-Vyk;y%_$Q~Bjk)?W5t1i;Z=-{${5pQa#RoNlo?j}zIj9sL*J$X zJ)512SUq<iOYdHTrFR+yuMHKS0NkML=ePj>d=iavL4dqyt2I8~V{1%6V0L*8zSrJH z=-g%GbldWqf{LqBpSPM?)=8_R{Z_QzdeJ7lVp?A9>N{2Wu1BtaOSIC2ti7yUU$9hz zO_URp2oZt=ll*NH4b?B-OUkup<U4HIR;V#6hjlarKv!x4)}LOx9HESW;~<~?)1RGn z)M-Ih_u%Qzav~4rdb{!ID__TrH-3zfo7b>3*oS;h_KeJrd&(-;V3+G*HP7R}|Dp!( z%XG4Yv3LP7bua7cp2h?`hUA_GiZwWehb;Mv=-x_V*&}PkVTZ?KRPLVbbOHU5JZ^YX za*Ps;`5N@XXL$cja{q0NWnzyJ8NQsX`Q!q8u@yvxSMwS(qMSwLby9vi8ZD^#@cAAe z6<NoHa^<P_Iruz{a?U>ay<6sd^z*s^{TPbLX_V#M<+Okwhk$?UEb6jG+uzH{d1n$u zj7M`Q)K9tJa;$nXfQtOKYMBy+)vB71alNbBNU~llC9c~OetT%Nt<FaP#LVnG9`z64 z&9{DrNF*vke+^5^D|p!3cko`volXo6jo{t)KRf^gSFYVaNlyCFr{|GQXO#7URtExz zvGED$Hn%XpuyiDjqCFJCWlwjy2pGW8sw$tkExxHVz$cpBA+vUiXflbSWfWPxfP<hQ z-->MdVjR_|6$Se&;>$OXH4Mo6IW$FU&%3)t^T;a=L21o$!+Gw*b9vs_W6D~EK{Jyu zSUOP3>ag2Lie{0Q&uU5|IsO}-k7lS^Y6!bc*5iN9Ir;rU9Er_MbbR?dF8}VwSo~}p zrt5c*nCXXmWEQT8O?ZV_1`J+=Z9$|1Nu;AmghF0;d{(%976gKBxGWa*Ubv0TiDl$s z*(c_-Lw-pP`Vm<~glSGOOIVqyy(?}3BTKmU`>)`_Z@q?z3xil5n#RiL3`~o1{tYL> zrT{`FA3|0?V$LwKp^WlV(%0Ee6AEnl(e;CFEM8ww0Z1uNLcmB+i;x~RV54^vOAppD z+p~eCMHdXgEVktsi-P3mtWm6b)41bE;A49N%fckGqA4_XS=i`=dw0U9Huf;AtK$1N zGdMR`Q08@}UN%~+7{}~~J!G4j_7Shq5_BkQpdkW|t}hmiB3j>NZa5>%TgtrNOb5_l z8g8wmnNv?Ve|s>Pk^tiRvJG=`uhs-m7{Xf3fl{5>6Ivd-()LdH{aJ@9`Xqv@lAA>? zGc6#=1BcIz%@qNaYfJFQw&08EV6bmv$!Nj&)CNXI*0Ho|gVh;9ScH96m{5g*X9Cn3 z!WdhPmeP*6UmC)s%Ccryv5s06-*3tM=@+gGAZ!-$$a>xK-1V|P1S}b~XUzaQg4flX ziz?^7<Bu+5{``ars#E@80W1aqwTB0z%vCVn;NjOTjJ*B`=l=aWc<`+|n0v5>Q31_M zV+Pn(T(GQK1xV<zv?kXhz$fERDStct<z>;v%ZW7db`w&w_mLR7gxuyNGIr5Q$8-uj z83eRj1$g+^J;($^VX8Eq;u9@Vu4313!|<E`1^&0+M>Cav>N&8ORBe~>OpGhoLtd1Y zs3<_8!~y~-1J=_eJP79Sxi6!Hmb27Q8A7?{uChAN?NC)Q35==H);p~z$Irv}PI{k1 zwf7dF-q`n<zsw-dre=Tq9K~z~y2){@jSV9ci=vQDso^o|G1YY!TeH&wi0f+piF!*3 z<v@j2-skYYGY;O5GP6x8O{`{^WtEkYQA&ne?xm;PQ0H^y8<=%hG3BVhn{NuBkbBIy z4grBRQ79Y56tY1xeELba*ZM>ecB%W`ju9!y^T-K*NYxqdkwqjigG^43K)I;S?UnPm z1*k9%<f_c!vRCFHd&egH<mn;6kbvZ=_v;EY_6vCU)SbqVC`-pbw+V=J<b-#obfWx> z!j_D~8+0MIJqnZCqJ1Zmhh<|F?)5>0a$2mG8gkFcqZ%PmdIU5u<&~BO6@y5(;i3Dg z_V|8`<5|WL0Fle(VYS)ea(mQ4B2ifwg`#@T;c_VkM^}W^YC|{@Q3oIeLm|cZd_KQI zFyU|nKEDWU*=VVhEUbc{vh3->XUFNleeevMR;%bo-gX;R`zY#;nSFoio@SN_n^By< zii&Ae`BiBKs%I@I1xMsh&3{A`^{n|YuXnG-&XPL<0K8~q<;G{mluvWF5rf|EMz+Lu z!_PN)@%-#B(9AC=^VhS8rZCs61<psq(7*KqY`pqiM29+2avFtk2}3JZRMjxLB<Lrj ze^B`Z2|QFr0?qGr<?`^PESNJ7K<AxRF$Ah8&{HFbSU!Mg){jWWhj7X(OwNN~%nhgC zi0%)6i0&)r;IJ8y&7@G0^-wGCNB!>mE9;^G(GPiGwmWoRowBiR=tCHAB?akMV3ZBM zEdpvioX5I=g-sa*4K~pLLbCDF)g}`1>a)dQS$0D|VaBc3`|#c?o#^{$9NRrQSVxR1 z5;`Q{gaV?PWe<s?!B46?UZ90#TU~^u6>CeM!eJj4rt!2dnp375#4d~{SZyhHh^pe5 z38-MTlVVGhX;h_VYPsyH!r_VE(}<~c4B%sVWV$p68t9h~h?Z!Kq*Z3XhJXrJLTe3v zlCPe|ah6HuYnx<cDRX&wr-(>71xL~jv*`hrrtV^TbRKI{+psR#1Z=vI)Gr~s@c;$u zgb35HvZNJcK3Q6HWNrjAj2GAnCb1^-(Frg$hSLWvqSZABNYIB;DxLXcqB*(+uRjg9 z*@;@zi(0OrBE{+NWzPN7`y=T3%bPF{8Bk1!U{4mG<fR`T$9=^z2wrC9toX@0efa$L zJXS41L=$<Gna3_`r(UhW9u8vD<3Xuhlf5eIw7{;A?)_{p7xLk4I1EG3uRK85IVB24 zQsu#`<EP6C1k?TJTXn0oMymxuj(gsY=%NA1VNoD@N72_c1l_7$G~T8X{uEq>R2eIt z6jlTzZurAkbtPmjDHsD8I7OHzGC|~$i-?OLFv|U&PL$y1ymucf^ZyyR$(e~kqTUp1 zZ=4c?b_P1hg;5gWv^_fo!@Ov*tgLy6TFr(iGZ9$VRxsb&4cGRT@bev&QXP`(G#Bd1 zUGOqQS3^bwI^8#nvv7*=rhA8=i-Mfij<!3qA_{^#(ZY;bKvb9-KHZ?2o0IJ6;Ckr) z7bUamSU_R^8p@VQEvAR*>{JN(mB);;$fA-OLA`7dP*qpQ;-I^T=i_;Ol@@F=H(o1a zPL2-(6J<3$=`-P-j2oegD%GGn>gn&)<vYz*S=PD{`Q#YtjOokAkh4r9tLs;(xVe>1 zWaqCVY+pr5*1U4{GfRnX4|Y?PYh`@VVIF;$C=)3g&q_xCg!bbn85iwW9&l&>Z!W0g z>4nF?Pv0K@=6BlY{j@0k7bP<3N^A+`@Dz3`F)cDW?vxuecaR^gm_PH=50!9c_se?L zEWZ^YPM`HN0K{%HrgD0Vj=w=I@j!uwy=FoMMLpits<crUu*2V(>p=+PZxy#x#X43k zJNlU>jIs_lEVm!R|L6fS(|3`){?|xc{&O_Ex`XVIvl1;G%zE%;ZK}8sEjveR20VXM zs|5^w@*Rxcd;{6!$)#s`503{?lDM(L!c=Ed6+z`$48@CxGJ>1Pepee>aMu&Rtzw{$ zwe>@-m)aj0yJy*j<&Gr;w*x9gi58ARG^Zl8X)R%)6y|xWh~~a+tafdp_tFMNJ54Yz zI}o#66XE#>a(`b%J?BusryZoF<wa|2l=BD(fbivuqOr2T7aOc;%#P=!1%*Q92r($| zpjnVdq56|HC;g%^Yg7rRb}AVe6U(DBxR0PBE5baoFT^Rl*}RE0N2p0Q$b@&ZNVbBi zjC3>^hS}rBl4#G1Za<b>L9Cf{a9MkhjV_~*6X2dNqmpN3$_T0fE1F5a$|6t&gAQdf zwV5O{5}2OMEQOK`WhNJ(7!%Ool(o*nOLx|tcx!Z1xnlS}Ehi_DE7F83P&%kMsI_^g z6>m>Hzg$=}V*w>?%gytM^J~}ExW6HD7+6E!mPgI+aC`&#gC`b-$)-mwWYN+?RkN(T z<_E7AMUY$2W!}J5habakGxQNBETREgiw)(sH;cAx6tKH3nxHP26JVFd@|GK`-QzGl znt<NyM!zkLPmCel-U?u0&5JF0wV5IacvDCT2+5Ty$X1KU)#U!y>9W{SmH=8?cUyVX z3+r+Zu8a2Ghi3ic!eT!!5ri_8nDJUKOUD63IiG`ZaSoQ1CDbb?D?_R^eYJ{sz>m$z zaV$LQK_VDXmcyLf1J-6aOS}QkNgfdrK0@HZXXBUO=3APTh61%60LrynZO~EzWXpRh zqr%{Asv0iODgdR3u~o8fS&4JEnp6IN3SL!bf+KBF@oYO4+ta-J5DEKGpyD)oo^a+e zCLcWg2+#meO%%!KA}Zdis0Wv{RBF1tH+xYQ@KBuo3>5;R^=^cW(+C+RQTK1{^RuIh z)^U2?9FNmeX`KI<X}6I>(btb!N-uvCEbM~=YJ&Swq>LKp`>KNCF~&p?QA=&15}ikb zX6N?j3{+{v&?>C%R}dT4s*2hEN?SC$)V7W;b}rcU=k>)ne*V3>)+)KMg`JdJ<+9hE zONfs3ATrW}l6eIsVQ7tXOkEGHD8-3SQD>WB(S~0bL_kYTW$@+;lib6s*M!b*{Rdq6 z%D<L{xQv5-C5OL8!E*>esNzlKrb5z;$mV7zkf4^p&HggPpf>^u3VdF^0;FUf@_!0u zepbp<V0RR-w2@V>5pNV>!vvC+ZrF!zV*dSaWBu|sV19T@z{Z-&MW<G&a@JXpWr(j^ z(D$vuJT;GX(JXc2Mp%Z8u*w+tVYjOAHuKRq?2{H{mOmJ*UN>ceV|WGO`FTXwbnq@% zk9=q8(hTl=<qG<~cVDG!QM0v;nqZ%>z&37H;Ai{c2DbY)u^}4oV&@8G?=E0wU>R#> zD{PSf%yAdYSvi)|iaDbj=3oM`JX2!p$QS&`#pgxSw+iNAne4|((@?FHSu;Qwqyi0U zUp$*}A12;wnw{Arf*LK;A-gtRm~)Ai8K(s)sVo-mWI+k@w&ld1)KX&Wj3yT#${G(^ zOXwGiAm{#KAmFBeDT@FrR*Os*N(jW0$P`Oz-O<&wlZz^_P!3yRbqbjD*-&G~M2jVZ zWgffQIyA=punM!a<x6UfM)L_|<68o_e45+f*dUn72(S;2P9eQ*LRIcJ0fB43fYOF3 zjDmoVD#5O-#nyfiCB=xj-%o7sE%&tMLD~OP<gH&7;naWp3tt%Vei3xNEFA_A1P|t= z1=u!L<yud=uA@YLQ7IM>bl9-e-;0%@0lCjv1=Q0G&5}S~L8XFxgG+}zBGY8hXhVSp zeuqKR{G88&0`|C!CO~jUTi9^aRG*4fdjpC5t|~cwW{yMNq=c(tFj@u&6@zjR)ld%_ zkp(m(MwKeBvY1|im%XTPR>Zj|1tp>~50w>--JHt_nVTE6_@KxrznU-iSg!NiD#e*r z2P)9jN)hf%7RCf?M+8(+F4Y9^m=f{ibT2l?|K9+Jdd7&7ZvZWMpnE$fv>K)+SjnN` zTtJJF<fqhG-`lAv>qSuzT(h9lN{zC#sDOvvT0priXldA~iMCerff*L3(W0BFl_vuT zraqQuE+15d<h<&(Y1Ax3s5md95tu?szXjv{8F*&|)WjI=&3tm!O5anXk!Rk0LZu$@ zfvd`Fafg<&`2s@VMTo8-T5q^mYcUse_s`?ekNyC`g&wq6pz0WcZ3Bp;FjbpqaMa>x ziX&4X8s}5uhJW$xb~fU%*)n^{D%xY9iHtD#a(W5X<PaJKlRU#DDw#Aqi+aS&7UaDX zD2CTj%|59}caUb4%n5MRqZ08WmMbauN+2Td#R^!yGKbp&05&^UwQ{iW{L_Php~*DL zK@W14ZR9*Y^&UTS&DzlY&D$7%dqCy7xAWMkjdL)CEf>omn@k~-P9T|zB9)3EC-W>- z8F<x@`NYt*WfkBW6)lS?#LVOo24D54DF5B|dS!a1()``~-9n!<?D~0$=gd`D7oe&B zCa9pxC6X(PpvoxA2|<P-RDi)>Q;b{zde$x|rZgrf`;)+ig8w8UV4;O?(;vr%Hw?dk zg<LL!N+FG=9HTD4yQJ$ymC@r-t7dJK_qk8fxf~qPID*+I3iX`QRH?OcUHS_dBx4&W zrmY9&=e@lh<=(8smlba~OpI;FFRq}lvaZrI_Zm;M&nB%#r8?>YN={qcEXY+q5kNq> zGV9qnQHb6Z05KrvKI#hiqKM7FYy$LNw!L>7K;+XYY)wzVw!S9H*U8s(8nNRjV<vnS z6J{RV!`kSu%IG5?2ny&LiI>nBVyXGczLlY+EESrSL9<Mra*GwQr2J32^LYC~9vjYv zigDWgx!L2MvN|`?W3sP8B1`sDiv^TYfU9*!S4ZVo)3Rsmvu6rtT0trq0V-w%_7%U6 zw*iAs?c;-*N$qSPrTE5Gfw-Nr35`-7Yv;d*rMLeS&0JhNtPEOelJzF^nIggkEghN) zYP*P|0`~jG$h0}0Ze=-6zfb^C5v`FL<Z%<`;r~AtZK@O+Lp32<crB%F+UfLI-Q?QV zBdrI-@@@5h^@(T$WU!6wj0nbV5hS0Z5*<?ILp9d~Q6<4Z``mmzeKC-X;mpy#7-W>S zlB$r${u9}oUqfN|9j!$4ei>sfj#AVE|MX|Dk4z%tPaK$XkCSp0wOs=$6{)5`hAL@G z(=40p9Iit5lBCktigDEeP;Cm%RhT=ET`%Skw{OEfbsq({PR{F6bKpj4$zaTY%8U?= zft#K#iDy-L{Gy1bpm1Ul7@8_$X%XPbWz>>`YBRAx4WBumA=gt)1TNhoWYntrWMK`e z`(*oga;+?kMAN0RXvuGw8pq(IkO-infD9I}VPN5b+vCJR?WHA*rGSP2ux83HYpg2I zIFGbbgi*+j9j0<pbKWD!qEA%7%T58sW)8KTvMM31(p*6wET6x2KM7VT5csOyVqvZo zyD$p@5A`4j9{B9E^uwsq6>+9m004jhNkl<ZT%KpLd{N?@+wRFW@lU!k*kq1?g02Qn zs)#XD93!SE%mOUpjNK8&)XGJXH_5tRxsR%2UMo*5OspXz<uSl%T0_{eg<{YwK+Pie z+>UD8t*X-U*{EI@o;8$K?x=nj%CgcfLMIam3Gk!IK2pkJiv`2v`B8(VuqtQm8c=9G zN!=vvv6x#*FG>olbISi2F>kRWH#vi<lR~2MWFPW#Igck%!oXG-h9Ik>p1j^$kDCfW zoGkuWo|ybEuB@ERqV69AL=fZ!vL%($`7*@kosP$pD}rtdx+Y$hTEfqAi8!2_I?Q%= z!MeVxG6(49WDw+fwtA|S;W+VeWQu#L(ba^$hAZPG8AZeiSbtt(y1nct0=wOMME(DO z7#E@Hm9UC$UX=w`ZVpirYoZ`lt&>XlD-cjli{MYH?wYiWw=zN05?-xo89|6D#72-{ zRll#AM^RetP=(v50H^|Gtch|^Wbx^nvbVc~O4f(4xf_<*%lPYm{mXde-~R9FSsf<D z&-SguraYAo)?KM-buvn6WxZ=eEvS3fQQq!XmX@>7FBCw~r_2T*G#_(#mYnLn`SX)l zy|I95$}I@JPc(7I{-%HW2A)JU&mgi>{|))s@1nHzKC1rPC>cLOJv55ls$b<}*UM2k z_9IpH`dPtm&!U(J6Gt>9i^8YM__AUi3w`V|_)eHz%`~RKgVx>du`;zYpWZfRMAm0T z!8;(rgyzMQuj}ME4GJi20jmNjs=$p~%iDbstArevmW+1JyIQbBxU||F6FLlv^Q?>^ zW?2^i7E#sif{8Hfg$hDdEscuR>z=pizX;;(NDJfG_B3G*G!YlzP!Qliiv!Cu?=_Rq z{FFy0bGV&1YtS=?yH#Mrnj&abVf=yy9-g3Y9)S#vIqfv3GRU&n+B@ZG9V3}2?<F&8 zXbGr=`%rLv2PNNI0zzcIt(Y7;uLR7_V*-{Pm4!qf`RPP3q?yOb)LLcHf+%<xpCinS znhMRB%HMy2dGd>(lK{kV;zx|_Fa>Kc2CAwOBzscOGO^?gpmTm5w<hM`^m$RMmX$kV z%M*mr7eZ9D{zk2=%*R@V+PcO-x#n4HYCi);W71snPv$`;X%g@->Jx1_oq;=-KtVtY zHQ3!&5w(m#_Pdiad@@-@0)|UL0V=DgdW@<dRB?S%?q^!&aHvJNEbzRn1vk_6^buSb zcc5o20DH86oV<S=NVMxg&=PSHwd+6-=<ZY<{~#cO&xzI?mhe7KJF>512|w(v4V20! z<o(NUQ{fQidLF>GcCy<-)ww7eWgTwgq-u}HNolwOpp&5>^m8+q>3@U~y&X4gF?b5k z5B6k<>0pW$$GWVUCL)<#c_zCEXPfXOYd@FeLEBfNOw|L*GQHQJJNy~fq%3PqjTg&4 zFRyhe0KjYq#zd(mVX9GNQuP?xk-?P7T~&gdP4?JUp6i^|f6I!GSIy&E#bmXAkFF=q zuUaZ1=&)mRYC=7ise5;@)c+8Mxj{rNT^Rb{U*X}~{}zwVzXQ|K0(KA88R60DvWC;m zU090D`q__dQJ|=%_EXKdfa2U0Rlfc#^a}+Lty)W%DNV>_loCuOOu{~Hz-Pbt9zOex z_ptEEHEef{%6r=?y^s;xZU5raM8R5Ai@EH37un7Kh`jmhXl6H65uH{>HfT@;pnVo~ z85P$m8VM_k-d?ms3pl&*mGiLFbJ&EEWd^q8Ayk5FT-9z`#l|Qt*(5c%)|XJurIqhK zu<lj<_LwE2BF-t4=_aXX#&J@WI?~Ai;t8pe-(0*^O?c2Mp?q7<07S}HCR!ky(kNeA zC8sfKF4z;!rP8<LwO%e@d!h&HE*I>X9MY*U3Q?!LpI4TW<3W^PB+(=p%kCnU+g1Ke zS{)+Eq5y|g)Y4Om`R=tCHGP6>Yo87S2$ELdXH$U0irhE7+VF?L!(pNsbxbr)rf4mu zS@!qrV2RJ7RSF1Ly?~76kCF4eA<uslwd@qC=|Rza2bBp@6&sRuxU1$sMy6R#w86@I zyDAv%{CB>I&50oqhFTZD-8wr^2KOV!+wO{gxMy(|0MRD8mRwQU<4u;y-c|7g1QOSl zEO>2j4fl0UYz9&)Q-D?rT2|=3I1?#mug!t(2*%s2CQDcfXW-3MP-xVV678?u;=56B zp_bf~=YH}t-j(&8T^K~7_a<syuc|J|;}bNn$e2Swf;p6jSJpwgBmz(XrAL57I8#yo z_sF@;;T+tFB8s&DRzRu0C!mAi;p8}oCjf#((7RJ{qFLDZ!a)Qxf7q)0IKXMt;z+nJ zBCz=vh`O!{2-sAv-1aEN#^G4}1p2YJPz(>KEVMe;tGqU|)2b|bEqW6L+h3yS{-Mm@ zp>`kl2A@f}wTm<}RLXUDS<^!Jo<a8Qmhf?%%xB3H!N8^!=jIJ~Yr%%k7xb7hyRaD( zu*vVl`C3%3o3R3By*c!mv$(yH!lWyQ)lfn8?bwVL&>`~SwmGcM5s~K+uQb$1$jCAV ztFDbaGoYI7vfX>AGpJkUX-XE*@9@K)w8?&)-Os=~<5~pyNlP`AQ^J4WdaJ6pl1Gyt zQ%K9-thJzyduaWt-|x{vmV$F_0nJ)P?Xz0B1i#gc_0eHWciqLr-CLM{*bU>tJR)u< zGSMjVsT7LYtO6FziXZXy|BUGBe?rXBudEOwH?ATkK>aZB`KYWI2B!*;V1|b_iO0%v zQVUzOVDK~0FBCx3(%UG9CQ%Fw$rwf6-G`*13sbNCDJEY13Q`UOD(RvsbsCe!!OC&g zVS@^GQ(ef0R*`Y7q2L=p#@>xmXhZ;n6V;Rr=KDWH&NIF5*L<u!K)L|-%i*$NNW-^@ zvR;0-aZ~<YQy@T<)s;J(vip#g1<nd-O6W2w*Sn!wVYAihE`w=A`05;PR4S|ku#25m z6O}?ir7`tiI*)Jv+OOiHzx*pKUL8Zk7*f7&1_U0+xW^PO2HC7@&;m7&((?es0TBuB zo7HH#KJW{(^B(6(pI1rqKhspD`P0hkH>Yh9GMl1Y`#4Rtu_9J7RczTT*jgAxRu=j$ z&Gd2~dCUg2URAB~WIEF|JcGKIDtN6khpzbw+_Dz3i476H(*hjaa^4)YIF)Cg<2ZmI zJ>Ds!5uZjqG9n-|!FP^)ewdgx_9!aMELT&3T@1jZzm^sl#(NwF6ogz%uVgvnQduW} zVH}~YJMgVuhQ9wDEM9+8*6&4Rj6+E12j#tsn7;NZW;^~C_WAdabliq_{UVm`{}@xZ z-o(VMPceP(CO-e(pWuz(`z1Vh`#(W9_<^jc8B|5vY0-zt)T}SqU2zh%ed5RO<w?W{ zcmjx2K?7XO>R@{4$I}M<sDBl`danxdBe-~xCBpknz(ChTKs0|^h8Q4e6JB9Wuv|H_ z9d?l|Yh~l<`XqR07JbJLC2L?8^*{*u*##6<=TLM^qYyJ9p0BCYVrw{$AVFh=#d@?% zigrzgojna;aPWQhib%^`+_GmZ@;hI$j5G`V>{ly1Ow34dWJ*|;^_<xU98W;R7i)=N zGdE_@@cthVW2{OR;hWz_Wb;e#o6o5lH;ntD**@-B$4cK7glr2U*nG;(%e_YHEnk^Q zQmtp;i*I3N>kU}8zXhx30k-04CHx5#?4lSFID`ejP?%>$co*sBIy6#+66SOUT<jjg z@aP1_db;uO+EonQyNlJ?Iar-R%$SN8H&)<`H{^cGZ_08Uxz~9HxXQIT1T<U|{(5OU z0JHGeumA{-K%oXtp7kH9YPbbhTGM3(GVC%e@?Bjxg(aI6%bTCWzWvv5IzLB}7K2mx z;xmcAd6x3_`1BYE%8mOUzKd`EyMKdA-}?^6Z{Nhs{Z5rC8})hwC?t_fCQ!_z6|i7j z$zit|Nd@UsF^P4co*qRpI*vrZh1KCf1wa^wa`0Vf86OO8r<;$d#2Th~ti`yTaYOz> z0R-I=%&ge0XP>;vqRe?_kabQT1U1p>py~}2%P2)H$hn8)^KN8aM8$e!ye7D3R$&`m zhN*8AlkW}S<KKJ*kG_8k))9j$<$QctMz&!eUWaX936`EwY=8a<jGuoFU(a1+mj)5* zcomVGKSpNmAsn|mpuch#mIss2bqr(T{SFMha~-|!e1?lZ`4|t*U&Qd0D;U0X6%XG% zj|Z=x#|QuI8~DTj>3_y|fA!bV@%L|GV{9DdJPS!3eK36JG(+!{&5D5>rZWSGL!>pt zs;IWTb<c`2t5r>-s`RN!PZD%oX*N~!9F~D*z!HntFn^hWW)zehOtaGG>$u_1;gT<_ zqN3UE*S$K9l-Ht_x29$+psbz_i6C~J7~ILgsatRUJ(dR^p;|n_OnL^=ZI20a`4IZ; zhbSah_Jdv9z~O9k96(U3*UXH5v<llw`(m5j(}=m=^dYv>PMLzn|5J!D{(~+KSGkH5 z%U{cJDy<|+nHjh@u3(|(0v?@z7w5nEC+K|p?_gV=LoO0Popn9(q9sRMNITX9JdY!9 z9Y@8xfRc9}8RrC&P5~Lt9wZ!H@GX4<j*)+d(CUv6Uhjr)c?_$MIx%_sx(aZkS$OA= z$^DBaGOVrXN!)zn$Eps%Nm)6EiM<6(ff{CQ8R&v3n4%e&-2q(u_*1<0y&s@&azWLY zIIGoxE(&J#&<b?bpT$s2tD!(ZL1yU76jf?9cMPp0#cEyvggl4j1}d>dv^(?CEm5`E zk(-*+hJc5x%*z)@!0On9-8zF%BB=x~K?B_}XWfSc6Dngx_!TV^flMAQVg5d4Woe=$ z#|sr}SQjn(mMBFZZFn&k%p9cuY3>Z?7wXO+pbF*6GM6XuV8!$%Oj}oBvkZ%H+JfG^ z3WF#JES8rDdX-r`U^Sspuc{zv&W?jt1qx-oXzV7rCR;j(<$xQ@UL#Uz9kMA|yY-UN z>gm=bP+~DHf-PR+@w>=1#a>C79BX+LQ4(!B5sAs3_u+o`Djs$`#Jx*5(SNfOx|vyI z<FSKh!G8(lcn(&C4Cc$qqQDF}4yM@iGY$rk7D{Ef?GCI=O=9f+T@2j(6ulR}E&J!| za4-J_;v2t>wV}Vk)$5|^+x@C~<(bp;&p;;u2);+bPV3^iSN<A*^?&>d`Y&HlV1b3X zSZCv~8{*mYSUualoucB8tuniWjvya1Amp&CxRdp<VWsG{iO+=lhw1AE;bAPQ$10wO z6{2U+FBCwWOpldscSq2$+y&8+bBeh)()*nR<?r1_Sj(1Zv9ctK7-^O8SgeNbf&Py# z<17EmFXOfU`2S+|ZVy&^hn^Y>-TjCDKY;m;&oTYkkFnBq8|(M3!Sl%<!~5ayWBkLv zN8kJJqWh=s;L*qDF?RJ9W;?rKn4X76Z-mic#oD$NiEsqjSR5Hy{FzW30fP&J?{r{m za1n7|0A90Ep}^^Tca#r{0YwK_yjw*xH>ZL;X_aWx%K!)xf6>?#!4`7Ex6&hXoP%B% zqFE3*GbIQHe4<fNYh%$8GGzt~su~UZL#43Ert@Z8@MQhM3^Z`Vpi0|R4@#+sb}pG; zt5K7=7gZcVQ-De#5r%VX1B<dLKKt&s@YR3zOZeV@_`flH<BGa=24bD<x*j7bh&yC@ z9g&UyjAG~(cIxy?o^?Y#p(8O6`{04WP0F{<HB_;y8XeTM&Pu$3rf3|6or(a5JhH8# z+|MeUDJ%5WE<76g41+^G!nCZ2N29`|t?>H%sMl(0E~>5utw#)y-D#p#D~kqLRy`h6 zPX-mBWkw^kA!7|C%X`Rfd<kXeB{W4VWck^WFdtgH{1!7d#z!%6=a!0ex9iqa!Jp$= z%2|ldh8hk5Z`kQjS`baAJobBk_Q&}0|N2kx&Yyl66QU)}_1u4IOmubP?$ztKarr7P zpSyu;7eoNuzKxO2yXZdm3BLb3zlpE>vtP!0-}(kR?{#CaZ&3Mi$2)GQ{V~g9$ZNM~ zqU^IXv2DhJJA{Fyb=>Oe#s@d<;_vPa;Og)s>~h`o%`;e#>7ERtBoRO~n!*rs4gpq^ zB9Nb$aqGb_LLZ!i`OAL+!)Kx#m;!Kyb4UmSEs1uQkLpm2>(01FPbLD1f@sJdna{TD zy(QrZ)7})O-AOEX(r^o(S`4JoDVnoRz=TmWc525IymYgCk(LMwS=Q`emdsu=jY?_; zCjEU(Esdip>upzfMLd<p^0onMtE*TW9YWA<Q#^`-n1SNd+UY9b9Am;iXaQiy&bAAn z++?*<#zUBtOO68UEJQ(7sP3t>Us|kep%&)Nb(sAP#04m3Q(0u<2?dVb`fa66Pu}fN z>&3V@uY@eijYoZ6xXmW@%y##)K0GG5o{=H7t|so>#C%T&bR(BxU-$$e!{-9ZA0QK4 zMlt0?qbPhSGlxP-*8AET?qB&F_dC0=?9{9pe3n1Qa!_;_Kyd%Ff@gcr@O~$Q0i|#e zME(3kFwsz18gxfh6T`^GL-5Ky8NPl6j?IlD>ze>PS!-ZK-tV~fAq-y6p<|s+|L6b& z16K-;Z=hiRI%>&>`{`-^eZY`YzU$KhTgWVZvWTwz_=ou8|LvdRlW+W$2(nG}aE=d; z$%ugM$^Qd3GranF*rqSSxHN^U9m4!Begu!js8DY<9!D-As89qN87s{^WU@6`5Ue72 zydFeKg!7m)0k>=dg@_vs<*#B;5`%}d;Fn&ti1z4S5v{e68NqHnbZ~ty0wAcJv55_f zw9vH9>;b!I3RA(TYUi74Xf1r(#ha*6Q|qDoiz0?+IesQKamjmLL)P&MS{0iTcHBh? zrvcuzNfeS%nSTtX<s~c)^kVw%4H!myU?0B^&-$Ef2*2D{t8(Y84iBjNOkqUR9KqDq z)CA&w-w9f?Xe-Saa<OrQe51n5$5BgkE0@K~fCo|=KyZyP8v0310`00b^RZnv+qyE- z9w%l)Ff)Sg3DvUuaUIpY5^6Hk-_;`Argd0XZXx7+h?JoIXd(i4G6QR_f;m|ScY-;L z#tAY`zJ?>sv^ZB<lNdzE7!?5~b>AXNYnM?Etf-);HfbQPq1Gie7Z&ZY>()i9d8mZ` z+Q^{Xdn?irtwLZx{$|BM2D8yL$~=AJ!W>ox`?Y5y+S%*_VV;A1FbR_mSS|9~uv-7D z12AyqBDQ8GRYX14G)G!Kb9e$+(AzP+X~nk9gHSk%ZA%nerYKVJq_+O$II{zN=)ERP zY<mmANF3{SA2#Is45D21gs>=M!5zegCkSUmgh!E0%^JuQN=PJ=SezKg`8Qre@AYfg z6m61o9lZYl{R1P=Z|Kw>rLVaqnpPpHLnb^enl;Pw0+sp>LK}M6|Nd3jzWr51Ci|7O zpvF2Sds@s%s}d95vmvh<NIr>ZiuV_)%7ji0o4$3L%*|;6hN3dz+cGq#*O0E4k!^?& zq90!PLX^E00$Ds*58&Fo2V3$t{ql?zu*mPtqS>b^^l3jEMDUqY*qkM;G9$pq9@s=B zIS!X=4l^rrO6zYE*Ph92#<eR~u)4aYfEWcQ!Hp$ZK(g9Ub0la`_vz5FpOB^jbDuVH z^BOOE4iPl;tyS@QXC8Ow%J3yLAW>`bSv>h3_b%6KDVI}goP3m3E|&%#q5G2$@Pps} zb$nmOqt8B-^>9yFJS}pKVcWd$t3JeS9Z0zzpb(iCWiu#BXj)ku58R&$$a9To=OJHq z8b#RNyNAi)5v<F4<TLe&lKQe%39c=suW~OD<WC3?^2^@hdZjE;-+2kZ|4ia_$@<|l zVKJ;)&WvO@p!o2Z04vU=O^sRwBbVRAr(gONOy7F%&_Pb8e<T26Z$I*NQZACw>VG5S zo2uA~O0|1@vft=1#r$67H=>}5dfbXpl96&TC?tJsRLT{#0gt>dnshn7E!yK0-kICP zPiFL3iszL;rg{J5QAeLUp_;E8et#lS#zk)i_rnF{FAr3k$O@BcvbajAh`caDk1S9Y zSK(r^t*@Xg+I~Gfroa*x_45D(Q6o#}5-f0z%obpoSXvSY6liKEs32fC{U|393*Goo ziS@(3@Q3iu{a0k&7g36i!lR$W!oz#$`RU(c=)!w)zx!d^SW|&I6eyKk2Gwi=)u>bJ z6H#Y1#GKL+Xo_R74+RM~D79vq#A#Y1pfD|oMv-6;(~M|IQ_z_$h$KYA$g%*60edxS zH7|B0ZBhVYm)#S!bU-D5kKJ1ZBFa`3lcGI?Csf|eA>E3gu$xxELF=~ILOkTd>ZoYY zcRt1T$lFMH$5b%n-i}rwksyut-z{2Ox2%P}Xc6nVnsOPjn9kwiT1NoH38X9&j9+<1 zDOwVlEv-DoOh(=3h})&A2X<Y&fVbZK2|j%LZS-FL4D$~kC=JcLyr`Z<BH&j*nV>=o z&SDd+lG-P2#@3MiR1_xO;PaGgV}=FOG+9uJ>65IQ$*P*|xQ1*=i-X9NwBK_*PK469 zZ4YBn1g&0#>0%^@*+2>o0a~icAhp{4^?OncfLgh%tS*JLXwyY)Jz3m-tm-YWtgR~G zqgnf|W3lfuSPT;gB(o}>!L(t8v->8ZeQzSY^f5|q2C<q&pgaMMv<R_kZ8fomw%g+{ z@tNvG8~;S-LzgY8g8b-Hr{A4^ehS)hi-CH(3S2bySP$Y!)kfa8tgMFxTLQNiyqK5Q zNLGslp*EiqVeQP8VGz*ejp<R#GqCb#90kqL>_hSD@BI$${Ol*{e(;%`v@V!+RLrcx z9~gznxrs>ufg#cAsn`&-&=q0#2Jo<N5TExB<2w7@MAKT#0;`hN8!wcat<s5we>4dq zYELXbClS|0=X?ck+{<IuR99}Vvyjr}>ov{NpkKkY@Ba>NeE&CKT^f=-IEj?|9txp8 z)UpgTep27(q@sW)68A1$y|Nd=9uMY+hcMjJjWvsN9~^4dqBe1_7pg4*yC=tjaZjeB zQ33QFnCiT(6kQ&>4IJ9^EPnqyQgLgXL*eQAM~|5r-jruv+EVcmm0VcK$z=ia)1owR z?J%%Eu(g1}x4#De*8K59oJhY=08y-^P|3`oD#(ZzrfM>@7zYtjdAQHjHsyw8Me`ve zWMh}O2r*VmW6VUo#LR%xAI5RvOs3#1)-V%HV8Cw2drMn*ciAKwd<qM~;FhC#)fq1- z0-NUkgN8RU)`a;*O6*6ld+;8MDb0)4$p9}hMgE^|0}2kt7^omT(NNs+lxk9D-B=f< zcN?=^pCW3$h23UW`2?v+wdbi8$i9_<fiDCO1Q1LuB-3Sz&WmjH!$bkW41i|Qtjv3c z$#WRF`Ziwu-G7en{9peR_kQ+W7-sGvW_y5``6|N39<1O01W~71%~RV5?Q=6DMypBC z2yCHZ98qne8JEG}O=hbwrHQc<#gGGO=N4j?Nx0`egk$t|*hW5uV|he0h()9vdX)7; zC@tSc%|3gOJwgkDN|9pCiv0pyjB-|igvagywZrvD7EZqh)ujKK0=Nw#2x4{`k$p=; z5wU6jfwCR>W)j6>Mu8m)sg3br__jsEvEN5IsaK7xp4^8g#P!91C9VZVK(l5B{f-2F z49ZM1GMt5>o&g{foigv!NHJP@HmQ686Lv2?TC?E7stL<>kJ9FgxvUb%1PLmp;Tgy0 zM}}}3of}S!T7t^*#!}l_e-9RxVzA`V-;>AMBQ0!)*>tFh4FOLC)z?Mq))%;E_C(<B z!jUS#mdHJ2$vX@2l&p2&N@P`eWoqTANJv2WFh+YWp|j&Ax~^Qr@Wsnm{P<mL-?@OL z`<F3(<3q$fMiFvJH1jlJ8|2xCl?!37SyWYP>sjl;dH4(yBHSr($9*Xk!!Q#}EAw_6 zEF8an5`iYub(z-LCVsCZN<~66+PtiTxbOp}hsN?OTdw)+86@bF=VJ_PAP}ArFz8Wc zdz!py#=rc7@8YY!^2>PZ%l}<~(!&EFPV1&-=D!Wy=q<QcE?~9)Lkti0;PVHMpfi|| zt_c5;dw$On!p9q4^jZS2hhm853fQ!}FuA5feZS%0@m?5bBDrSv6Jn+iW4x60@)!{; zc;Y))um~T^?4VMA;j@5ytx>cgxcJ{?{8!;8cU24tv$FPfXt8eB_j!4u;zM^}Sb+Fc z&ja+{xDJ!mp)5rN6Je1Xbboy~=)?yD4)~LmgNm$eR|RDhpN-q7*MP&m)s^dj-{tyY zT**CKL<M_tEgc?*19sJTtOfG62j3%bc)#C{pZ40Y?ajd}N^B@oMKoJe!M!;Ft>sEV zK(`OY_#kSzEo5ab9sx6x0wQ$NQvxL8xI4UoX!2~X++QewDCg`5rCiX*LfB5EML<^( zukBztQ-LnW{CK%B?APAhI~dK{E*g~`6w@WlbarFt@=drltw{KiqQTc?G~r%KWAJqg z%p)-o0#7eoQe7S(H3bIB5FBhI3UFiz5SmDembx+VifCtliIBZVn2`>tOb|g?L^Mn4 znTjMD2(=FS13hW2c+Fwc7S|7fM7tUzuZ3;{Ew*4!F-{5;#v#~6bJ>hDtto_AvlWx` zo!DBsggaAX=$c!BRT$&CKZ?~r4918c;%q@Qn2L&GZXf?7DhZgPUp(gXsZBs3L9I06 zaVZPU>d2sCJk$4XVfOA%;MbiKLD&QPN}r(mW%#Wo+<5KpaN#@OgneU8z*$!AXHIn> zZI#T38LlAh=oAgfse-SbA`#}zh*=f=sfdteb_laVDEyf%vXd79oz$VmQnBfI)G}59 zlODJn0qC{^*qmF$Lib$^{Peq+{OQ-Ra_2n!+p8!<M7zlgcqk;a62|F(ioamJ1*S7H zowX5jpcu9x9B^X7;lrmo4{mO|VHZu5#f}&pSCV_r==!v14KZ170Ren|`grpxB-5*i zgnMBP8nNLH<I4MQ;`Oim4z9laBa8}jwyZ8A;kO|h7?U|Ks0M2%X{)CZSBh%-v%d}) z3}|Gkn=m7SiPOq{D=>NTM+d~;5~%Q)w2%>`P<Ru}(lR!eDyucu2D4hZ;MQ|3Rb*6C z_Ap)_T1TJ3hXmt}R5i?_zv8uVopWvT{8Ro4R>L)Afg^Zg@GkqJ@H=g)$$Yto7{_pu z5PlZX;=$h$$W-Lm0l7Yl08g)N0_nI{gs%s-<uL>X9>Oy*gqb_H@X??BEnfYtKf$fn z-xB3xMAkc7gF8`83wTWH(JHu67eG;sub^B`A}G&=aSASB?$pc+avr7ys+fi26FNp( zJ&5{K^&nKwmwn;9!@yiQrpo>ZW#RIM;nHtoUNpqMTc2aL|2)=KJ1{&qC7=0~;>75D zTK{|&v(m(AiJ^I)BOCJ}=AMAhdJiW13VKIJ@XqJA@YCB5F=?=4$>hNO`#pH$y-)Ci z&pOaEJfSQSy!GX36<bz2R`do19$pGSkXQ_@nG&UBL-heVTqlB9HJaJQrd#-M^>iL~ zI_)WClMM>#>MjRAM#=x7a(ACp2j+_)@=#j6=$4&&(1iybotQNRaepI>dut(lvtt|I zyt$1Vi+<c&5205VMBj`7=Rd!P-Z`D{cDL~Ggi2%OAo$?mx?+6E1A7#2uek8gA$(rI ziItfm$|a_Km+mpszFTS5Co;|&N0%;)L46F9<^;NA4u8|3$6sIH!rcv#CC(JK0y&t2 z1(*VPZ2NN1d9qk@Cb1>}$mLyx&)tXlt!)g<nsNKlBHr!R<J&j3(YyM5_Y>(C3LxV4 z6dWt`tER9y702pC46_?4eEWd`-y5*t(Lx-vqfuC9lkl!);FDpQln3*(Uaa+OW4%|8 ztpOX{i);~{N7#@@a65<f&LFP;(He%{w4z>o!qmx_=(D76Z_J-Uk358xWC=kA1qst; zKoB<&TXSdZH<319LS2MoBgYDME64@Lk@O59<{1{DJc&ru08cUky=Y;5f*jYPIc0ie zcRp&poAEr7mDA0C<hyi3ka4%owq#DBTn7H&1m?ES;o){S7R^TGhnpE1#B|>y%sqOD znfv!d<G8O-Eq#8Iowt=KitZ5pO)cb-Xei(K)qjPzzw-MSymCpQU<M&E27~UCkkgL1 zb3(NLt4MkJQ7ee>tz|@uD=OCy>pQI2JXp25WRo8)@}hj5^)x(7orqgHMYA8#nvSXJ zj!&*vH85gjL{(X>q6IDPF!RKWdZJ%6{YC7Gc27T@YLr&rh0fJLIK{s7W#u32JNF^p z`qJ-W@X|SKj18*@@u|)ZjCWka)ZLp{?CpYeX%;cYT%^)4L=vj(vpy8Zx-W*#MJvA3 zxr%oOw?qI&un<UN$R5LjCnZ{I5fK57p==%|j}3G4V;JeVjgh;bW8%(RSbX?>_4>vi z{4T!yOaBB{U;P?Zr|!aT9zfVPAexCu7z^EOC&%TSg}4p~CaB4#<@r<aw+mNM=&Cer z8C&BzbiQ$0uGMm+`zQfJySD%V0f7O_H#62kS<9%=?mp=n*tS+tfGcZ=oP%kRJRdD% z{5`uI^81I0|6?#7dtb1!;bB3~IkHnC19S;kn3=*16}l|!@g{5$S}3&d^WS5KR_`&* zkOhxkv}z*1zv@ZA5Xiz2D<YmPAsyDC5M7n~R7JtLhUoZxlpHoRE25Q|!+7-iJTCrz zKW45j!f&$(AY2kaJ%G6fvXA81Pu}_nivyowVWk&y`ZahmMHM(m9>KX2$Z2k!llS&< zI$jTg`^&D|z(RkYa<xW%9#LAxFy46wL$_~Z=zcfGM`xhZ8Ig=c1R%tbOBvwxPs0+A z!od0n{sJa-8QA<~rM#qM-MEDRSmHkTBNOmP=U|Hm;Sl9)T4%uZfpIJeI8T)-C|0US zM59<389~P<AHyzSrA@rHT(O8Xg9+<K^UDJW5;L`S3d+LV$WE&mHZGfFeizHOhKhH1 zv18M^nNvaMjr53$TcUenw~<0a)$@7zefT^g$ctvH!V?o1Ub16!-iWEGWjwff7fU)5 zK3MYL&1pOO)-1T$aUa9-{G0(1*5&}F%`psaM=`h+#<VpdyeTgLysolZINC8V8=}lI z<(0sc@g`If`8{K}fGJB7qlP#}jd9F7($EL<qVQDYc{WrbvzYANX?F^}rYL*@bX}r| z>jjuB3m{z#W(34%VHJ7D<5794)k~<BWj%z)kcuuK5>Mj#^&WiXTOZ->oxbCje<J-t z0fa$r>cqSsed8`X7_{S7j|Csx-oiis(|h<Af7XTJl@P3<5;lS*EP67S5v}Ler4ATZ z3@FHrs1+Khms&D*lvScx(O!L9A$<CW*D(6&D)f_4xRYgNH6W5(h{=%OZpi;5)rNX_ zPiuR^%=1j4NeP%>F_4BV($bMs(^ffxO4^8Qa25#x6~n8e@YshC@Gpr5VZ)%qkB>Gy zxND6lt%Zx>G$3)9_zixqQFSA@`E__U{urs?L*!!HD5irbW>QM1YJO@hH<<Ah3~<x_ z7VRSuSAm}!V}1D1Z~sf&`N{VYa#)pyLi28=kVQ447f>`P;Almzhh|%v`VL!A9>ZU# z;hHgsbDKWQhce3ILQO2m)VUVpIrapR$6y}sKp|pMEz7muA`|;AC@ny#-NflJTccMQ z=P|4*LDE%4|7?5HlQD0(8}P<kM|`2m59xBM*K#4R`S#@+1Hlls93G6U=`kc@$?nDA z#2l_%x{i*^*KqFR%eZ!D2tW9A0>ApT`}oq2`q9-pgQ=lmbltd#>v!*9TE78Z$Sa84 zh>ev6VWc`lJOZp-YXXe=ka0gk%6S)_tx;^xOd{$sq7a`I;WU7_fG(?f5=NO@AXQLS zjnf`6Ym3w?byZILIPo*SUl!q<iD%)l1+lSc!Nl+e?sd-M!L3O=?3lvO{`@0+^Iv=& zKl|FJ=pI<dtV=WxVG5(+95&MOY$}Yi(PCgI{JD-!na&a7eWxW$m3x-owi}?eEffS6 zv6`lB)ADv#1TevILC(cq1_V1CjPp@F40g0Y-FE!K;W!8d;)Ovbi}ZINwGQ$#S`p|L z$&{b=<C1v1n5@A~VTv}{%dU74bE2`W`;y9<#9+u;A%Jph6}7AlCEGMoqj!<mUP7Um zL`wjwUzf$ihYk$9W`^#50BI4#bmP!Wosqrd^an9-vSQh4gKm2bW1XL1;KC0vdi6a_ z-oA>J!9EOp{uw^_+Mi<J@@Fa#m@z;MRAr`sZGBZ0@gY!W#lSTICR+<**jgUNbmt8@ z#uYrgauu(>aSwm=t^0WEM<2-lKgP_+AXZlNSYCHwem#IG(PF#j0_d6zVtOlunavoc zR@X4QHjW1iBCPHO@TE8H_}i;qjF{6RFbasKtw_hm1bjPCZ<bWg5}PlG$(40PQd#&j zB~_VH-OI-;@m^MZgMj8R(e%#DjCtJ_>|UQ{`Fd$052$D*yYoxP<a0_%U@7{uA6BzC zv<fQ%9%hAi_|^a0soZo|Gu`y(qob4Iz9`jpU<((qY)=b+Nvpq;B6sL^D$05<GO$+w z;F2h2fAxF6hj0Ahmod<B6R~huJ`>=?R8|56wjD1w1vrVom*2Z%WvsihSg@wBU`t`u znZbZAig)_#m@y~ivkYeCbzVN-^w1q4zbg`OwN$J{nE=1Vo*z%<%zH^!2*C*7Cx5Q) z|DOsnmD-T(FV??MRuI|CmXs*n*%%51E6T}H;cpX|ANUl>XyDMDIGs)e5GJz)F1P1k z(O7ME^bZapo0A8>xhcqM48tR1N<%%3Hn+CXVAZW>(CowluAje-(Sb?$L_j4nRRo10 zyn1mF|KpQ+3@!R_cie{See1aP*)^DK4jBDm7}7;e)1vA5kfub2$vo?U%R_kOU;Yi= z_<!EUh4%(<=i)r(w)|KS6lNEpsL~c|PnVu;Q$~*c&G)A9t@mdT&uEDGIME!(Iu*Ok zf&vv=KEG_Z04iCR2>T5fwr}J5%mWO}UW9I|3+wt`tk@StYjDDrOd+0&p<IfhRf#KL zvbS?IyQGka;quqNg9~5%Dx%hTGz*)`FJBkb&0-;}pvcHt#!9eIPOB1Cg>q;uQ5{EP zi-_tz#^#M*hjr|Sh{pV~8ER;j?dta|V8xh|!@z-;Nz*I=55X4AXT#nk2HXh*3RP8A zE+)*MMOg?Y=nLKyjd5&p6mzp8V1<#|5+Tv*T?j^(5swU^oEk$laz%9<WYaR1yxna? zjsS-?*~2Y0s>j?;3|N}-r{ECnigTkmPoL0SNSv2T{@##zP8!4LoG{_S-346en8U?8 zbLf@-U+Ngc-+t7K@4eTB_bxpYz&eek&2_lg^r~Dyu2zLxnCptmgB7P8cHcH~88>$7 z47@5SvuHJX7X|M}s6_6ld&hLRRwIu}F)l*i44-RB)~5~@e@Iz07!PoG**Mm0xcZ$- z=z8leRt8qEG_;1vo<$7bok9QYX$%Tb80lWX%+NYEmn^Uuyzsgr2!&*=#DwvOGq^pv zg}?i#AJ=+UF*&8jXxAJDA1-2X(+#6M2bwr(`lLye{?v9R0J8-cMbG*J3j#R!S-T=1 zEnd`k0`eK%c{H0Eg~4nHb6FB?Xg-p|v<U0zP)-Fq-nK;W=CTX7EMXNJ)cy`%|3B*9 z{K>KGy7T*2{Ih>?IPCG*9(ycLjACYD?D0q%4M(CxjYWzQ+3aSs$!4$BOLf=2W^Gwj zx$pbF0Z8n70!ZxpzAp(Nk$cwG{P;W+*`&ypM+*O-@Qb*COyB{yxcA(1KHqomJ<loc z@kn3%>XNH+xwTuT`VofI0gn7(ciwnDTA|-70W#GP5ZK3=Z+~4(={@EJT9-majOo|g z9yQ5&_guaFulzmYFLJ|?W-L&|DP!7IXiEPbcAe(y1(0D5^P3!mRJ1e;9QizSIyDOU z5?N=fd$qDtIpd&uPBU}Ty}FqXcPJjbo~BUjttstwqU<^l+h}GTlzp?*6Bf$BO_KI` zMn3xq-}$fqE57{mZ((0wAz<7lXxbuTStnweC1#!?ZW$+Kn;^3@OEIFNBI2T)uoLln zMKJkz^M;o{cz>5mPj}JT66^%41f-oJxnp9ved3ui;aHqd*h?(3gU@fl@6*U<%u$F= zpwqAO#e)?djHywZO&BbTxZMlgfPa3jY?c0sQmrn);i6J+pxk(^B9I!b{pJ?SU%ZYy z-*70&xU!`(I)RUBqxst}|AP-@YT@KCA$F}2f1g!08vXjUzoQP_D;Up}H7d$l3KvR} ztL|3o-&ohsz1yk;!Q)z?`<wrvhmt<NTr1-Y257eTC@V?xGGg!1CT<Dd%9@6@1b-!P zD}egjaYKPEMc{n?XaAA+{;z-2-6Hu-iPpO7k)A8}xlt02ef7&J)E8Z_$kmSN-%Rqu zCmx11DTxoK-Kz@9>k7muu>ITL`wu;ow<}Pg2!UTgMN8^XQ0~|&{p&4;$wjZw*!?rk z>hr(-_HR6X2Ov6~4l}cJ%r7o?fAGoaDTBjf1Og$PJFf1+t8_Zcm1{S@28ehffm)-* zFXybSZ?e~HVzJt3NT9UJqefo)CYU(v98xb;@L9}kPfcK2oD(Ho!m{#+9rX|nt&Xvt z2WT~R#ux02t@%+o^W94@Z6(mjfZrDNwUTRM5XU%Sj_~BkCQ~CizW+zJxc6v_B}alS zU6_S?0wjY5j0;vQ^Jc7zX2O;bjhsT?&*itgB9avfG^H6?_3_XDa)kfsJA*tD$3C?w z16~uu9;i@npMK-wOy$yUva#RYd;aqIG0wn_fYvRvW@Y7W5NkMsB^(vnOp02n;mV6@ z%4e`fC71+M(!Qc*b%R)jZ($mnLG^He4}R}&_{sn9f3tG)0Xf?M`|%MDQsYuaIn2mo zVS!qHjZ~mloZ@GsJYTZw9w8pLP!eM(xi?78ULmo3k%V`GN?}t1l9O;YPB32~B!NN6 zn3rlC5fsO%WW`hHR7wA;T&`7UwkW^1eDHc<mtw%mVjd+ii*iM!Ylw1Qv)#tMH7!9j zBI@#thLrq%lXyK(Av;Dh_@Al9-X~J9<FAR6Y;;}+y5A^Dn%a#>8(X3CdS381UK<S- zxavr<DUhH9L1kJfE}&^kHkdS|xI5>kcQL@EI*!g=?Dk`2@0dLqgsmfijJ+JeYM4+Z zOtw~}*l6L90JW%(qjBV@NS%(FA=-7Hs3Qs5QunZ&;}w;imd`krK0V6crtSYjfua8H zYUZ!iO7{}&8OJC46l-ZhNhik8E~~*@ci+D;R%UI(Mel`Se)`Yf;fwEF7hsk?h!x3- z?UhSS>H-MOdWS}>O}lv@V0p&5vOTc7bL4q<hrsz^hEPJ!`pz=`=r(&<3yp+>ywAq2 zN`rQM75kQhVquTNU&6t!GWAt|nMzofZ+5RHDOZ)O($@-rDL|GzP_hM{cl%vAznk*+ z+3T!<T}Sgw`K;Fp##SuQ=ksCA=V=`uic^(x+~(k@PUoP^{(gaGGcCXxC7JcH8+Bmu z8l{~zSa!8IBNm3-UT)YzY{t{XWXvR_9!mP@thBZAsN_hn2{78F<dsZ@(&v@KbE;Cu zvixlkyY2TE*h~w6RQ56D8eNc8?8aTlV2c>=IyMO}Es!$V=<FYnkL?lAXQ`(Tcy_A% zrq^wwIO$NXgU(gMx!aI>pZxN^F25e*uWz>H;k3RZeSgGxCr`>UfV$^Tam`*}FQH~X zXXc=|!;!$`Nh`ydw8gU{34A9lUYzYq+w9{>R9UkNc==i+3IgegBXqm_<m=BkK6}-j zzB?0mK37tIOS?Z4|N8vdxs(&}QAZpdb;wkz_(a@gvsN0hUQ+fi$$KBLmzknb&{E1v zn@0+0oMmQIKE{_e@HkgFk-DEZcg5ejrC)YMd{m^=IB)E5QdsBZ$v4k=D|Lv-_vr;f z4vxNgJL%u|P=rCOD1I`R=A>!qE_Qq=0<gSW#yhnu{_H@w;1^==ucSQh*6FmeCSy;_ zStBiw(2{b(>E7wDizxGUq(*$wBgI8K=;Ea45W(W+xpME5mXBvmkNofIE|OLLS6)-v zV^7*5EkI{=8F{c|;r$UOgIgYa0h^4YFs*itBLT@5%GCC6|Crx8ltq!sZYO0o({EqR z`PK2OtB~ida}INdw8AZ#p(cC5J#srawnhxxy|96OE<iO<r|PYW?`(2ZJQ9KU|Mlw1 zHy?_fcdt|^(<RSNIjXO+AO6ekmE^zA@jC#a%y&hFddDZHy8w|d6!_6wKcRnMn9VJ9 z_jRq#fL3q#nz9ClM+k+&e17pVF1LrJ<y8iTM)3LkjE+sRu(Z;>s`2YYKFV!$zv>nx z;mc{KAr8??%CW#w&j)<?*MG))-}w)0KK_J^UrV`BXIGSlE?#5Sm1D_U;O<6<d1sD< zvJfSdXQ6)!)v^PVsFQ2+Y5v_i4&LdD6RC)SDGDsV&(5rLuyjhR(x#lR<FrJ1(6hze z%L`21o8{goeat+VW$EEV-uPGV@Z<mEUEF#V>9Cn}#Dv3PU_)SNdd<xEnwv#k6q`3e zE}s@y%HvDN*+1MTn5&V>Cy055Wx#qlI62`^4Ds;voX+u?1Tay|!8}uQRvzD*XZ*no zPwz}H(7(vws;Fjb2=z{YtHbNu=%2&w^s+s>$x81e=4}h^P!w-GA!;Q>vY4k@&vM*J zbJPe?iyJ7|r>XfTX!tHuw||Ga;|CP&Gvq}j<%~8m_ADW537fBi+EYSj&S5v_(d)A4 z4Oy(VB0H`!zTGOjb8&WO<0PVc6huiDn<o@ouZyx`Tv_y07Ka~~`Z%}dWZ>nyD}L+Z z@D<RD6%sgh9^wnHQ4+duNHF{DQfQx}4IL3KpHh?e?8$TcCkM3K65LB3dY3o&VedK< zn`<nn_1qe@aJ}D5|Ku*~8&W@8ftb5YI#{7psL?uX3e`7h?$xPhD-;4*a;dls?j(Eh zx;U>>4#XyE)jR>eon^I&dDRZyNVc1;PDx61^jyiC|6GFCsQ}Lbs*py4_dU#_Lfz#( z+!C;qoc&7K70?I>07M#Xfyf5&Vi}`G$5Q_U%l%`luj;uo<mS(Rw8hW9u(4&Yk*S`t z*HIRbNuYS9tS1pjIFP{83U#j_ohge;mC_W4w;$*EafySPD8a-$2XzlGP8;G>>YVN8 z>6F4Wk~?JFR(4i(7{}%bY|YaUTR5t@x=E2*6&IP*HhT9GHor2}W}{Z|QWGay6R509 z*(*6|N@<mq1%*58SKLxu-J<;8D($eatl^!{#Ad}F!(maxAv@lXgJ@!xLLnp|l;C(j z$Jt?t)58i!2Q>+v4WijPApyXArNq+}GoKGE<MNv6RNeA>ymD?q+TmD0^<{TEqjHOI zWqW99?|>KLBa{Vx${jzIM<r=dLmaWvKS?DcqMUblCN?AAThHsLMvav9dh#Z_)T@Sh zr^fb932iu!S?s}9YB3Wn^Td}&oo!GMNNFo@l0GD9-J_m5>Mn5n+DCn*q<t=SgU<7l zYInENX(Py9XqbcCcDL_d3Roz@;nj;Xd9Og<SFh7v=WFK}d>z!THm+n#fc!vgQxvl@ z)uMpjiE;zAUw_yCTTwE0DtA0>%lODutE3{sw7h>P!s26TbFrJV15VFQ=p3Dr5dXHS zm;T!JuxZUo$w|MLNaTvd^Wvvk>Fym!JBe{xzN;Lj=ep^>&&B>;9OXpmE4vuZyW2$% zkB{+7Owd>zQs-yCQ!k}`ezU#4sVGpCsXoUQJLv-5VJ%55s>Z*uO5YEA`Rw;UX65pd z_$(1=o?Z65)7@Jfw4+}3{9&3RPWOCn>TWZ&9V2@I5Bnlq4+I{o!3KuG0Gg*>D$Wcw zZ-$~HMcSIgCqCM;9mZsjV{oTX86s?M2G~>uv4|)M#R|leC04hSeExJ7y(2H6))M<I z3amt>pFOX~qm096?JzHo3*8+$%GCC6{yyctPJ8}6sasXX>T{m$3lx@)X~)_${0$o3 z26geLb%|Y_L{Xq9!;y@a!@Qej$|z%ZjnNNo@b3Tm4ohG3be})2rfJv8<P&M49zRJ} zOkk_j{k%@PBXv6K0+@1l;II2a$!Bu*I_J$lw7mN2mB4kIy}Xx~&pN;J`fnE{j{goo zD2KAGPyq-f=|x2h_2%s!QA>I5KYHAK?sm_8Y2;ti=#xHC$NT$SyK$RZt<K2U1ot2I zN)U3hy1s$UE-J)o>%xL^D31z=obIP+S3NW;yHv#SBuXhZ9Xot9tKq%zb!OJ|(#c^w z-YkBbk%Y@ZVmC)zh}CY&F+R4##h*{{=?ANPc3pz;lK?Z5Y3@IW@yG84dGAVusfi2| zlNlB_ODu0y80$|+;0ZE1?`36O=yQ96sB=?-j0E0Th)Oz7Q19fu|Ltvl@=t$=>ggiJ z$qft>Yitd#u>5pc0>cKYBO2BwjjS))nVzvQIIiLP(`m-$<@JpXHWtUZ^}&~H&ks@x zX^H3-u})9OXDwr#R$*UU7Uj4|T$EDYw@uo+O+26@95EA(+wjHh+_n0+W|DgDx|uT? znNb_)AKzrKcNM*;+E}_yCL*fbSEJ^u3(eL!NY^+nmnE=N#Q3%BNB)|l+#OC^Sz5K4 zK%E|w%Zw*kAm2QqA<nb8cfx+{SOUi(d!hmwxdUpceM(}e{=NX7dtMT{0x4&On5|6M zTqI;F5R%8RfJnenz$g7-7+u1;q9QDgIT~*g&$Mwm3+Trk=r=>6p!dYF?GsP6y03@B z4TAm}MuRwaRUW6ni^m{?TIVAxpyS`PGBar4FW#TyU;TKJkMC@;y5z;N9mi`*5OM}c z1;lA&qEcsfcb&*_N6GZq;ao|)bauws@hK<mWA;*Y_C>iK*A9g06IASeaYkNRnXK4s zlY#)jPPodvHo>|%O|GnbhT`a7bdwnAb7lIyVeG}4K+hTb0)xs8pp?gsJhtU~65$-i zB`c-{v$TtkSRjSo7-V+S%Y!L9e|^cy`}a%&pawKX6Ka!(1$~l{tpr1Beuh>(jBW(w zxdduQRusFmYpg=1*rF<ZrIjG~XfA|X6qlk5U%YrGLE}^chV)xondYeBl^|(#L|9)o zu`sy8V&5WL0;R5iz;x6hpw&-G;4+^N%5N@_s|a|>cu$wBzg48Kv-17Hcz0fYWyQ+9 zo;7~@CpY=-Klw55eg6qhroE^I;=QpF{&*2rNP?6vgTW=^*pgt?7-z*0XGtHYe<jMD z=@8dP+<fc(Wq$9kM!E1n!_20e*$q2$Yun7NY_X`avtf?0EuXt;OS5E4vJ<UHJv&tP z4r%V6NFc~_ZP?juqf~vLRIMTQo)X)iryZE0ZPwAU?NIcKt(pYZ!uvD@bc-TXa&7rt z0(||>G#9lIbjeyb8Jyyeo+*o;Yp3KqEwb(=C!O=JcOkrZe#S||$5F-F{riR3*YndR zFU}9XKBs%0l)3=1HP}J5(~`O$3PhZ-x3564k~`q2d)~1Ci^Gv)aWN#&yZ^U7<G)mt z?>IR<5nr`M!tf`gcdvAzRk?RZOv<kKg=O<D<69=o;W8%i3)|u&7VQ-VHF=(FWEj_G zq<vG&Y+0CCT*12Q7sV|=daQt{G99WY{*a6p7YAj#fVc&VWd(!y;b-T{Z+P8ijbek1 zCu^+tuE;p7P|a4!hOz`r0e04%IM*DQ=gq7P=oo*fV`ET{_OS--0~I@S+jv%H8GH8) zKKd8mW%TEtqrQIw>+lV-)^ULYl>|~7CuN_&O+xBZl74P*vfre=*P<lg7|XS=dW#HC znV6h5uxiSnb(JuADmVk;UlJ|S;?qj?V;UlQYL!E(;@`>%@o(WK)lgIVy3W*af}eij zM>Uh8F6zJTD^rtLQ}v{~rRGUdb494Tf&w`4E(j>kSL|L&7CS|wkH~<V=#Y!DIYBEF zV?X8-$k~x#?_fV=rIWJJ$+|eJhj}J`>hvHd;?s>&yM=aSiII<<@c!?;!`Ou<=tt)8 zZ7fqy7-{FtG}0#O1s}y?jI01)A)BV2tkR4$yFgJF|KAV+)wG*wJM^4&UfbE<rzrMx zC_r9TrlW^d48DC!`p)q?=l!iBF<gO&Zv=>Dv&FVnkHKgbFz86f%yyx`<MDRaMwl(u z?wfygDC;3k#aS;%P&_z1M6<2!UWJIo<G6(;6Nw~&VCd`jbXPD-2}X<9TyoBC8n3@V zR2*Ye0^*ni%|U~YWUN71sJ$RY;1)96QfF9N$uc?|;MRxBEZ*zGw7!miH^H(l#k~<P zeF6td^D(~nV>OdQK~$P74~Ko+>T@%`B0#gE<LRiGFE8)%@S&Hfezyd@FnLksb+6E` zw@%)WXa1cf=H8qmWlTxuH3W3pbi`S;#AutPz3$XUAzC2fjA2}J(f4SRN4Mt~x--ku z>r)JV-h=tk4VJIoV(`W|*Dg<U|M3=mV-`kcUCge!(OD!INRUZniqd&CQN?XePlPN_ zJJOE(JlqKJXwy$x0z*mJ*5wSdyJo^I&|;r8uspoM$m4l7)GExoJJ=&J+}Ro-age#r zi74Vab3P0DR0LP0iC<K5zTK2yQ>2z&=V;F_YU-?eY3&z^f>4Tb`BvGQr(D8rHtFl> z;mOUL6cZWF56^|npLG|no=Q<1>~Z@{+T}=8<Oyw21MayA(l(!{i#@6`K;xTPTmm58 zA*GEg-REktmb_jQ<q#&lW0v2g7uC2+OPpXS+eBwfFfs3DdfA88l0hd5L?=qbmrqe{ z*4RI6aeRCr?Q|s0?3m}zPsEY$i#jZK@49&23Un_ypY6H3f1mGp>BQ$b%x-X6vB~o( zQT5W+qHYh;?e1m5rqrt`j;1QVC7y4ikpMQMiFbpwvR{8+f=f|?kTN}>T=x6bRW!ve zd|EG7agqVOU!b9Y$x)WTSw-t<6U`rW*U^-6L41~BBCc8LWpOV1E%rNyH1<#BH=TB` z?xf56a(;)PI5<zLgEb=c@HSc7X)tf8@$)BX{{6>cKJ3rZzg1$|RK_9>Bvv{g+c@g3 zz1TZE>#FHJfu=@npK_+gt|^Fi+Q{<51s=V5nfY5ogdHK`-jsk}0Na`q{j8a70b<Rh zIN=e!^!X+qf3VDt-k#$}KbhfYA20Ip<qiJk&mZu||Lxnn`=__L`2Gx6uc{c5@zyh& z=Eg*v1w#RUvPq^SHdr~JSUaRFb<0(cu!cGqJ$v*Idiea3A`S}F6{jw6RI0TplqJBI z<-MYkBOwnCo0_$471P^p)~sm(hFo`w@#Dune*bL?e|_Ca&$OB0B^~n{MyAFm*m(Gu zt*2}BKXuSI<YQtX!h$-%ij3n~ZHNi=F1-u8{O3z*{`Vix^WH-X3%Udg0*=b#x_rLQ zl1IIo!MK{En0>tvuDj*(tR+zH?p_h<NE@e$0w(*f)3TpEdnJ8$!ltv0!Q1KnU1^>R z_?=1cIudXeJ4s4Vi{;R{b3B>%^PP`2`2Fum+kUZ4f1it?0S|poT|9i?;PFET{f~B} zIO%)nWazPr=|L|uLtdtbc9|L86=Ao<AH8vh_b*PNU5(*$l*q*Pu)8BH=`3P*9gfeG z1$W9k`it&7eod5dQ0!douVVIASz9q-Se#>7<&(PY;m__75FZlER!GWttnZ2aAJk=R ziHfc!G1)ZOEK3|$b|~y>aZJr(>bZ;U(H#u^eR!6q2&(2tnpep0tW)%=NZM66rH?%G z9%AbjQrqif9V%iX+=2oM{>>lYpZ^1*h6@6>R)Jg(>KQe2z4KBx9cFt3bFe7mp@ucu zz$W$F5i#P63m`_*WCAYInlbXa35o%``0EhOT$KH?sQiX<C7~uj)Z*o-^re7TyL~ER z<`AQ+#fGWDqNYOcu$SIP8tx8+SkVc9ZP(<nCXZD%3^nw&1}<-dsC>TyO-J>XfPj)P zFUxVpPbXxgzA?()))*b1#KWSO=k1)p*dDLWm96grn$MK!vyS*G@d>#C+J$vg<Eunu zEVh-nQrsi!OLpT+vsfqVkFg{1$~?7BP-mc14scWx7|Czb%o+tCofN}H0Tnmxusk1h z(%2rNts5k3H{jEo1Wu$FjQFfdyE<s5<TLVNd7lTbJh!o|mT|Mj^w=7scSg}kUnVB* zlG3Waj?;fvQI2l}2!$|?j*h$QA(X2H?e;!ZQLoB6hBHOgDH+dx<2XD#>N?X-=b(GO za&-VtK(N2!;NXDsa|tx3r@wsP@6HPyRPr<vLcZxR^;m#XL>?nP3Pw>#hDlr&Kbspq z0g@;#XOc{!h-J~p_Jo$CH$l6Q<zz3-N!{Bew0u}cD&l5lPRr>1A#U86N2fQC3>wKr zWk3~zbXr1btwY?~73|9;60r_$|2}H-9;-U#vR|DodHm#(hflxokPEkI3#~T&&2FIC zkGDAyr}*OJM2vD@I#pEKjz>D)L(*jxRjwmsP-C5XLUr>2iKWM+E!*tq^->oTDx(vv zC_0rjAe5X$FKSJ@lV(fQz=|#@kRwWM(8q_50{o}X{QSx1L7ooCd8#7GNQ*sDVkZ(P zj@yUb75gz?g3aM&bhAsucg@{7&Pr*AY|h3;z{5p#kfEKRs8~O3X{Y0cx0|N-q@?CB zJ;h;Wj?=QXJ1=-tbBoe25%+Ik(T^}Od6z%=(GU1{-~BE_Ltm2e&5Gl&%el&&qH=!{ zFV9**3B)0u?JH?_b2O6<;@UYn#SAaRhO*LcV{Z;)>DSRtMZ0R^MCy22@lnpDa0knT zVnqy&IJ2s7SN;1Fjjq#GRO9jK3tG<`RG()#dL>Hj7teY1i<iopY-MftZ&=go1$4i7 zb>5w`R(8#N_2RI*e&DQTk->GFgTx3gL|Juj>8uO_Wv7BT{ujqDr2gj|NU%`mB<sOF zQ7A2n5||Z$STkq1+-KvBi<@kjMTt}d4D3<ti&i{p4`EME9BWA+wv1X-XEc9GeP7h< zu`-YS%dNpLP9^xHMR~kVRP^;4ieLVvob&vJa@qU!f+=OOma@|N;6xn%k#a}N=V-lc z?k*M3?6yf&PVuIW&<Lcg*qYs<@$BPIOF(Wv?_M(Qey@zn^W!rP1eP@8IzIT<AMoD4 zy}-STGxR+Xm>w{3zu(GdJ!XD-)xxLu-3-jd+0aU$GiN0j3E0PL<O*#{)egGVFn1;J zspoy<<5hg-I4XhSxkuY<%!%U{g*(4elz?7F?`#ncw5fyy(!wnw<~qL3BA&qz<~trT za;`ELtXztHaVpUI^7X26MctmB3IOdji6+v_>%H`^`<Yfn7~6>R!Q%-3>StcQ{iT~L zV{S$^VvH?&80uYQu20Y0XpEJ4QNaSLh1#(M(o^>K&#1Qr(yC<wk&FauFQ0s($0$m? zzIQ@Hf{L=fsVH_A%eC1F)Ve#wCT7C)4f>f^D}WL2PG3$esk!mk#CPAC=d(Lj#?>Y6 zOegtbFu=W;5Cf}GCO6~ENxf#~g3M3&SruVpR3!+8@?^`(lunB$3jyA|u4Bz8N>)+E zl_M(ULrP+IWhs@?A;m(6RH{W%pe&tki5P1V4A=1ns`SnJ`IkSK<&8@k)W!@O`V{@M zb}l_$=E<Cs8Fh?_RcW)u5R=Pc7FAJ<j)aK0D7zkaHz`$cW0JCFfQVj#lmu?8$;tG* zi7^2)v(ZDsA0Qp@65MfO-BPpEJIK_vn;2(DN!WEnwQFQn9+O*pglVcD!@?91;~KSq zxw}3`x2a>~={kLnwT#cY>6_Z&)}WP79+`Rfu2Qx|RNWJQ`~xPJrWo1E@_0GN6%ia? zJaY5!iG{wKB9i*&2wQFV^+p^<hm1KNnMj0uB23V>DYpF>>)arQ<;R%RlW5iJB1Cq@ zmxq{|(J(nSg??>@lxve(NKf8tAmNre1_gwqo*Ba;dDReU^Cs1JNJ`|Fk6|jY5T#g{ zb|%7MF(LjW#c4fDMal5$jf-FRi(nRCurx+uW0*yWKg+9jiCqr1)jQ%#MWpIo>>6xD zw>5Y-x7i+?B5X8J^zCw3Nb#~Q;-}!_*t^E_N|YC^qCiNTcEm${*Gj>nrLv<VyfTA( zMgb)kUIBuQwP_h|0?e};JRY6l(=VTL>*j#;xdn@eTemk!(3c`+jgvJ+sW=l9opE;6 zPA2ZHa{2v9zWj8Kfe9N!<3`rTCaI|Vh&%p&kcrZtXrpsh*szq*2|ZgpWeTFSkK97I z?s*QXB?*`n@{ttV{foqnHulp|I#QlDvb<Ch2-OojtAseod#HxC@d|}H7JJEWJi<Ny zkd$qK<Fu9Yq69pJ3Tv_MEvOGfm6uE&D()cr**J}CQdD`en{+9vkl*qgrrBB4>kbkw zKgC@iA(Kn$;G`7}igJySG4GN$3iwE;)YCcaPA5-h7ueKkWq>+yX%@QoM|dXmd){f# zDJ3a{90biOESpPcH&&SFU*gkiEBySfnwuj5t`E3*|IQ{qxiQ1Peru9{`NlY(KUkAN zr{(g1nNJ>D`CQI<AgXWPP{Fj@!s}P&LksMznlMjqk`rZreAr}pC&3#}3|wCnQms|G zNuZRn->wK?#If%f*iwrD&fQ^T_)8x5e!%eHMK;&_@CU3UrJcWgJjR8dK>;5v8vPop zTMKB-8Zy}!=O@kXmeyy-%5IWMchQpq5(&dNQOyY6<uOLzzsbA*>@BYS$B)tWPm<bk z31JtACG+$zM??)q*tF%k3m?mkV_E_YXU|^#twn5|lLPXN5+~>X<(&V&bNu2fULI9B ziq3OZa&+P1n~uK)6P<H9nSDw-C8D|%0c(mc-h9fR{L`Ou;d}QPd9caNyE;C-Yv<N* zm`f56Z%>68+e)GmJ60690zjRE^X}Ce1x~X5diQFVekDdbf5<a={`t}K?&7Fd=gL(d zQMk(Xt5?6fdiz&LyL?Q_U8h}po#{?l$$fD0iq_!^ng=gL@jfF~IK>h?U|Ls|pj%){ zS7BsR6z^=93)l4gr#~I$uf8|IgU1er7ZThbj`5&Z>OL4ly_CVVT_k1`U~ttb?FzX0 z8tesL!*f#}V+%>l*IY#A65Tmp<<e=%StM)C5Zn%;>eI1!SHq4bPCR*l)8AoJ-(XE! zXLhTB!O}uApJS;{9GXvnymU<3-@>XciyA7jt|&h#3o9iyG*#@oO?=`fOyLSUqTVaT z7#{mN^D_fXt*_&V<prAeNN$_4^l30k@Nw+4G1&Idh;m$8FSEE%U|mxa_}gP)Ez8hw zln<_i`Kyn8qWBY{-V3PLim22TROSYob^$Vz)K~r=n#*zTiL}!|O4=~PgRuagU)kWT zZ{Oz6|C_hC@Sl}M-tpH{c429soeEJ)v0D=2+p@t1wfq6C`U!^uF-ND$G{$jv-9xK$ zO1^Ty$=Tn%Ku#G3-OPipUavDy-uL{4a-*<UUFh+P5>FQQIXHesG+kmzYsDAKQrkNe z(A}eyOp*2MlC(HT+MMKe{UT@rI97LA9J0}KYnkh}#2$LpY;T8ANm<oISXZXej44J& z99;PMG@tx*l#3tF)7xucWJ1TIkwu=)n&=nba`WjLed8O9EIHXS3aBi3nVWPox9Fv3 zY=<xI>FFOaW6<v4wmLCttmsr$oTeRoQm2?{iK6Zyal3{>BqX9ODDCDaD}u5Zi&BnB z(2J$mb%t3UnB(S8FZ1!+_jv!KKHk2n=6j#5^QRx0_}=GsKI^seXvV|DMo7RcC4C-a zRc&KwUBl|yS~nZ#`tmr-a|0~S^szMC$IQqbh97*wdjA*9i%@czXK>lKQEAOAsl6gB zLo8|n0$>7?DtQ!9HnSOEW-BP7C&Kh*M1alDn_sGUQ$$A3xPX!X_QNR;cZVHZxwg*j zPp5h3ji=%_7U`Xou{XQRx;Bo<ne8UcbMM568MkQa`fzKcuG<Nw*A-EfV$G0eb|b;P zQ4b&A)bhpUH3sjlbM?b97DiRX;(5xoDzVru*B*@W(<{?NysIRm5)XH`u$hc(j7+iK zGr;D9S!R3n%nQgperVv*M`K*OdWW8#9v(ay<ntR3c>Cikj4!PHwkdw&@sA1+ib880 zzU(@`eNkBl%Hkp=eQPR5xlrV!b0{!i#yGJp&PK@t(30RMB%V!+GIF63RllNEi__6` zi_vPq<8=}CIVG45;MA|Mso7v<Qxub4M_j1#^svoWXGaqB_9aks=;Zc@Y@|4;x4UVP zm2%d|)3Hs_GLO+L=H)$|?yacv#WrrgI5%yU!L<}ai%I$Z9DTzECWJZ-n{GB2+~Pda z<QxT@Lmsl09Q(<AQMNLeU1eNi6!XJAMh9g8uX+hPT=>mqLJoU(1#c;v?_SDHWQ$zs z>E)wqcX0-UvJ*KhyD>J*K4w%7#?~ziE*WG1n|L%Qs$<a3Pp@maaNEM?m$td{<qGc# zUH;j(?(*Gt288$*xHG=Rt+7oRTxK>cJ~j*vQ5$0n58Y&Z`~fo)x1^nn*nJss+D*Ko zz8#y(m{%6qwj@|K<r$YC**ov${d-%Y5-b7%A%>QN3{CqPerjZ?e~X2C(>(h0F;5<> z3lVSg7k}`W@BZZ^Z+)!h!k1be-qi5LM+@Bga+Ur^dZy-Ou$u&iB-nTo#{@Gc#EWO- zYUiTlU(mZGAau*coK9IZDoS|ojN;xi(iLTq=4tnQU+Nfl{0K+*5VOCHT>|#5zd$^g zlptrKV;$jOXMw|*lhb^N<7`ks&`rT%C8plSJ-3QwXco)J0_MR5OoK~Uhu3jVX+&9= z(BFH?_T?`axjMzuK_4Tca#afwa8=UJTRC<%Gcxcq*mM~TrYu@(jt!Hj=dCysq5wa< zZs7O-a+*JWW06}uR;EUMtjvbcEJm?zB=L$(#ViH#5`d~+2|7|5zYN^1V5=Lf-8140 zZ;8`g6kt_$4D8m4nhWR`V@!)p4G-@!Fc}cLj4-j5U`>_5VXa_s)LGP4_(THXzkk=m zcRuwqv{n+EI;2oIp;|nlRXy!yay{=T>CB)1x+st%<MMTRPAAn7@Gf;1hy4nGUY@?> zg|fx%@QhAvpGv$;&=3#+x8P8TQcUhq6S(oEP8r<V<A*ns{N)!>9?X^5mSAu5cJN0I z$mXRk^>b>Cb1L~`iqU<F{wC#JCE;8drz1-uBICobi)&~N)2b*sWx?E{0nfE%c15+O zwo4Rt>f}5Pa?)m{!~xa9F%9|5M(a75lE8&4$Aw2xzI`DmW2h{_|4?B0NWTA=eCdQ_ zq9wi~$?AZO{tGKy`;i3DH>UXT$Fp2~U;h7Kg)4v4&qx3E0@uHDk0r5Zb)S}nyAnjN ztT6US$Gij(l{t$eR3|I|p=<?I@&_nehEHA|cXNhB3I~`Yjqal3fAAq>@#<us^HP}S zZSj={O$i!y0(Ld0g%B@9B)w?nc&TJ-J?=<f9da(^^iUvO+2s_gvn6(F*v_C^6Q48} zV`(AA%%F#nOI!2_EZn-d#5MVC_nz1pobl5;7v}bOlxt5ze0k5wTQ`indEd{sF1q=P zFFkzlP~d7P#^tdjHzzXOn=7(nZ4eO{uQiT1K04vq$uVcgN1PogHw!!Dq*<c8vrb(C zXU%Vr=Q_L;|E#2^e|3H+Hd&w;H;I3o#Up{;Ygr*8;w-$Ikg;Q>|CaQ{`xAWo-`wQ; ze{`Rr86PGQqK;q<Pqe{qw1zue>iV>8TZr)$y#)0IKD#%>rxH*vKN{lR&?pyf-QwGC ziqLs@Uj(3v+4WWWW)>u9&q}~w<j%k#cl!IdH#{zCUxeS9mQ{ljt;3JQAIIp5@X^gl zKDs_ZAYLL7EwehRXK`Q~i^@wd7)NV&uoK8)@mJ6YbT1jwjBi94UI{a_6kuxBjd^{C zVS%sSxggV;6gqcReBLqH>KUGR2fYCGri|%zT|t6;oT+K?<Kre~pKQ}}dz<fmZ-l@2 zw$$^fnJZr`@a6mc+`TB^a`m==^EgA}GmK0t>kD;gL_qI&!elZTGU6v=0$aL1Y17Xa z*cAZ#`yBtM0O9h7g~Cj13h5cGE}^9`JGLNSeCGxi|NJ^PKYYZ)hf|DBEwilAqqf+) z+ljo<7^!@bdaKP|TbYOKbc?dh=tP-fQEsxbzsJ$x0qxE~_ndOQflHM6x>16s$%0Xo zaJg3RZkIcd&Pf=Rd95?Pl7ap5=$w<749aMp7os8+g{Y*_es=y9zI2-r)vk2#He+ih zoS|ZOE^$GhXJNa{lf^76+Nw}`jyFCHNwA1@)zpe8AeAT}r+1%p=9rp%S6h^>a+9eS zXQC+2Px$KepqsJrMI*ymK?r<%fRySfNBID!wKT`|jMO#FekDt@oTXmLQ!N$A<Vt9r zqL9Q0XVh^CA|5W^)N|$C0cJkA#`Pau<;UND!Vi8l%3JUDbLq-;Za*C6>b)f{-JGUJ z+IV<)k>y!|1&#cs?Nx$yBRK)FhH0F-eu${mh(T>(a#Bst{dq24o8rPJBV2j!9uGf# zD32F->qqbMqYv)!>8&NM-(BMF)kl1AbweO%hqu18NZnGzQZ0<m24-iKTzZd~Kf|Ud znY&+3aP`6%Pj9br>C0&@-d<!-U`%DNaDBkX&#xMIG#zK#v4>%&%7#A5)RLK}L-Ra+ zJkH$k9GZz$^kY(n7BP>mVxLm8vtYzKw?$;KpVZ1CdtoJ!m2!*tL!K$?Aez$e)e;>6 z#a22cB}pq0rIm~d_(ddON=XK3+T^|JUP_jAN&zo*QAztT0g-f!lYE+!Qh^h(r=vYl zgZsyH4$it)Qxuh0+dIMPDRSw7frp~VeBqivaGOYcPavm?%~eDz!DdDJZf-3iCCc<l zgeh?>lS^SHq$q!1zi;B_pRIA@ftj)SAam<cw)81nyJeEnuI2Kf1Yl*c*s0i-s9j~f ziUjyG3GfF3>)zxsmnRE6T&W0ropq;IzIyd~D(CBY&F*~4Z<OCUUW)R4CTjQPv%mZP z0!o(5)5Fv5nxSH>z^>YZeaR|8AR&sd#Lw^edGDqRy{kg|=p~1zueUEMnf|&}P|7PM zh5f77aH0qcft8YSm&;g<z-k=B1E&Cr8-L#hyVvK*Te1?2&UtqD8hD;bP=0ozB!K&> z`}?}-4wS1lXU`-6#d-MAFdzS?JB)m~%+?bl^Y_#Y+*lL1UFZH?6;Jz(%*=V%+=^gT z?wD!M5w_=XF2!-qM#$JaB-LxA4OWUfIf0fUS$REf$lzHGvpr~KQ9fs;XNwte?u(OF zwzfjpJUP6nGUi~FHEl+ev9yUmqmrb^qO4ztH*qR!zEbUO+F@UIUrL8)Q2?vbrnRTs zj=RiBA;!K>%b|dA#W*dHzsRaUuu<nmFYvah_OY}nMHOU86A_;j>z1|cIO|(6*2Q*~ zrUe#0o8|hS-el;LS!V8TvDI%PY>ZP$)+px%-pa3oe_L#+DF9Rx$SjNRDprmqXr}n^ z(KdhliI4AGiZE}e3iu!6%^WgsuQO+>6Db|N&SUW6CHn`5-MkW>P(=K%jBT@(`OPiy zLwES;hd-0YyUeZ5Gc?i5<l;CJ3llsUxX$=YA8VRb)XLQGj*Va}O0HNYpD&Y57O-#a z@Xqi3j5q)1A9Mf1ex6(#qvzrPBX{Rm68jjqJ;jv^kGS$_FZ~Z@1l%R4>pl29F(Tm% z{y>U>-evy!k3V8XptLUjp{CrUs8Xj`s!}XgNF|a4cOBRbD`=LUGCOdIvB#e?F>r&0 znQ^8^miYKvpY!hj`2ho$$Jid$VO+M8*v(MQ)W~N<%tXaD+;Pn2AUcDeHFc0>RRoJI zOHlkk)KjAx=nxb5jfVF~icLhqQip7dX62A}Lm<B+V_8I1xvJa%O$46UvQ9u|c3va? zcAUTb@89OT|J|GX?E5#lc595m@in$I7Q7Psa+QKWYnYJ4lXOMm$f2^#;5n~eyxzI- z{NOq5kO%<jyN-bO-{<&81&Gag12?`H<<6(Kn7w<Gt;fA=^$zp)|M(_v{*xckb90pk z;|}_k{ETjeSu`in`3i(`P4*5>yO;C6>G<lo1m<>yS7%DHqSszfnG05OyeptlZ?uTU zGMLP}ScOWH(F$f&2)`(+a`cd+_H&A&h?KwEd*`I{hb+o~T-ma-tWC(@VRpTFDpHR_ z>D0r$7c{HSY1Ez(%^r%1ZVOqK`NNMReE1+vC|adf6(iW|2-qA8Wh+qfTqygiZh3Y3 z%OcM7g+Rr5<A7GC#DQa#1M33Ku#baOoLV$N+#AJZkBGALV>Nrx>xITWHsbDW63z|E zK9xALRU+C+qKgkHuihgxc^&I`5AKx_!n#=^`Waj@P&fOgn7B5~lt9GPoq1NCtf3iN z#XLUD&eRp6>LFG>zr^w<Ut*nDKs&O=a?b=CkEihK9b|R`<Qxuh-X@aUE7)frifX&e z%<uzd$4AhNKE=DGA+fzhSd6@pu5q?^NM4-eUUrYeT!l&~CgkhFwXT;sZ4=ru$#*TY za<`w2`-8X_Hi@p!5nmf-cYaRlwJcz;jCFbi^Xxk2MKw;9nb=N%k_^zMd}gOq=diLT zu+{1=cnUeA1SE(#RZeu<4piGV);3LSsjS^LOoXzuM46tq1m;A6D}C@%0mnlr(pO)d zin7iXX!rs&3q_7);Jpwf^-7e~EBX6nr_GDK8qcdmo)t5k=MtP}qMT<V^1m`=mf#?^ zLrynG*{S8ET9H5}{Vaj*R8;+uDEIySL+bSw)oN30rAbxlUnwhjBC3?dffe)0t+>VM zNqa_IKJwWj)pCudfU$DVlV>MNrqolheSzQ?$~q7wWw>&KF(s4gO9`H@Xti4ecHIQT zf!8|6U*DSfA95T?5U+(Q95?>UIRTX0)Sn(5P|K!Bc|6_zowzo`hu^x*M}K$`)!-bZ zu$x-a$6hW>yPRUbnxWHBwk4P4y$u1Nb~g`#k|0PK?>QM8He(pw@HX~`GbElo6o;qf z#p&NpYWI4l*Q;(`foIaE^-vw}qL;0ED(0??qwDFx*?XI%2P4dlt5L5QuxTYY*<Ibb z7J^=?ap~`rfMUwWVcf-W%ta@#L&I*RX_m2R8J00R$)SIn<7j}Bq=00m%we)hD^#Ku zC{YnmDEKPGEE#sz{OBevtPcoS+*zjYXMH^U;a#S0te~G$GG2<f85d<Zslhm>!m+Bt zr_mB~+Q|6aBpfyoIz~d9+jy5Yu{<4S`|>3=uHR&}e-h`q3D35uO7T_pbsv_cT^S!z zmUppj$a8AHhyy=68b8Jj4>SGS-1*);e)iAaX7Sb{%~Fd_^?<Xsl4@SsPs)n}2`oo1 zy6yNHC>7vTrre&f->l)<aj>p-v%2ZPW18tMiZpMJ(%XNF+fVNDXkvoF#T5o;M!5g@ z5~CxRSY3L+_Qn(RTaVF*e;Ro36E1!BUHb07h1oDKzST-l3D~I|&R`U)D13u2iE1Y% zF+h~M#>MEmvXx%PgtXDp(iVoHZPxFvF*Cl%z~~|qvl?bsY|L-$3XHqxpEA)uX<~d% zeE6yd_4+QBtzCAuJZL8k-2Cose)vz`;NDx0FwI%eO`BL8H8Ap2OYeOZkM69qIH<uU zG0Ql*DRF8+AaNPpxB$iUCgVMGy#3F<&yWA<U(xr)$7mNG<2KIW^KRhuS=e%fS+S;B zG>OXJNeDzGad^Y%1w^%G2U?>AzlhrXWDdIs+zpM6EuD!~wLu;&m~1X|R-XvqkO-&( zvBVzP%pOJYsmXAjh(C?r7G?JHSq9(k$FB<tyeo_4tF#N^BP#oxHTF5KH94v^I1oTO zZFV>oSZdbFoGUl*68JvNpNSodZ>s*?Ng{vq_(uhZl6?pN#4P@yexk!SNQ~Vec>gn2 z|HZdh{uh6OeflOo>kR2|fJ`JwCKM+VPLPOX1OSSJV<oDEJx&@O2~5fsy*+V4b<RWy zo@C;jl}plSEpd7Ww5vzroI2eGtmj1irb>ju+3vPCU$o4~nv+i+EOGhv93xBHY-|}= zSkp7Kq~g=33tXSjqH#&b$LiQbH5(ieG}4(`dm6hpEf5l<kqt^`rAQ`=QqMiq>IT6b zfsE3gKtYRV(pk@29gZ5x!l*sm!3suq5r<!bL#j!sa=>Bxfaix2%-Ri3WB{GS)AAdn zg9V5V_o@V<SyoL^ww*~lff#0+Spwk#D~kiH&E95v{xY_eE9e$);@1w-+F79LS*4oX z<)D(_q@I??H1$kE>KG>M^5b_0aX3V^=y%!HyVzLLGbPUb(d9ng`~A0h_ka0o7VbX4 zpxSty1H|UV5y}%t*NG*Q5`b-_y<6gZ7jfuk(WvK`o$D7$|AO6_OYCVLlGhAjTc5xp zAmHA3L`Z#~pz2=tE`i%KSJ|An$m-CiXh$!iAH0ON?-s_v`|L~$tW1i1O!VNN9>%{m zM?{=V(y>g&y+tW#qnQcIfXH!Jm%);+(@50Ban~rgs^V}<1h?YkT|sfi0kqRWwx@!q zC;X_!ysSJmFx{hK@a_tueXA@ih%?gJL}~cD+eKaatN=)XTBOBc;!uJ~i+ZfuE!A*Y zo)_qh<*268v<gY-|Cj{Bkl2`?YT8Fx$&V0oQ}U{5nMbH;$4IQs3f#?*TA!hyTcGA# zWlvsj25hv$yApuHbmB<~LK$ki1@hLCKv<sihy>gRmr0B+2~0&Lh-C1sqzSI22rb75 zEd+@!`$&mn%!|UR+IMLRFdRgjoFr@<_%~=-M`>6mDetJs?^s1i+sWu{<g|A3Iu`|b zUqSDFU5s9e;;@Tm|LbB2P&I1V(@s&=>&P0EDT63UO`M!r>gp@g7RT1iwd6O~2s`ua zh{`suBv4I-SyrZguBsS$w}-Lszt8ZWf5gli*I2ypl%*^4sIIT$yt__VRAXvFO<_q# zS-`Tqu}yhXMR`k2a%BU**ka)ApAxvyLrkM=!8Wp&@zc(EsV7VnykZ;nDN^P!;+jzc zixYUKrwPnW6I_}gusldX)lXylDZ%wVjO&wFwHxf{m+@_n5SKt$2|8${lG4AT)*J%C zrd6^IHKl-+N@RzqegVtqC2V5@*v2jF%uC>2a$#SvW0})oSyW+L)8du@5Hg!dyDSuU zOyrGvX)_J6ja9-E1B8d~vO7A2cX<u>>>9fgkbF~{(w{owdOK0O7r(MwOB)q12;&oY z3fm$?j8WD`O?>k1Q-&mXg^V6Nnh@rNFov0s0EDvZCqzPggp&WH;7yXUCJC#Ocvq5G z=M!v-{cjFvQ1#8DpAcWXT_L0`%Wo@@bBbLD%2dVAmZDMeK0CQxr~IE07)ucm_3T_s zU|kY0*Kgr5Zt_H+e^|uG(qpv*TNMkFYPPm)7?m4IZQ9rvQ=<}S(e#<*95eF|O=t#e zGNxT5#lBpNX3P^o?29QJTSe@eGG?iZ!B|9RDWTEj(5ljy))JVOqZnu9w@!s{%n1;? z>r`D00`j_hD^18!B<?9o+^CaP5>3fC3X9SYhye6$M)9pnJE|ij3}U}V`Oe7=CO){0 zchw-GCQZ?s75`Eo?kEy-<OrGKB+N0w+6abaX}4*Q^luRFb{Oks0R4gs$LuPW@g=Nc zv#KpUn*x8T%~_1AV`!HaF$(bP=ynC-)^TY^acM-TZ7$$k6+2mX5Hp$R2&~4;PVrSD zXyvng8l{B9E+gz}qGC^RDM{QaiA7ROOJOM?EX&dk>mf?w=W-S=PIZX&1uv`fHdd8I zoinQ}F6vlZGP1ZV?Js3}YX_(NcDKPzNQ89Om&NH2yETjx5W(~JIsQ=rVtaL*p?g;t zdDzS3z&tY}n@ruG$M{Fz#_$J!#@dW1UbUHZ-44s^DkkRU7@F=AioDC*;#FqmE-^db zgJye};O;ufpq{ALMmcPt6J4MYQIQZw5DA!whm8Uv7BT@RDYr`orn9@MSGfsO#JY)V zdkXz}FY3iRY_B~cCr&9BvSU(Burm9At;Ieu+%cS6;%qkhu&eqo%X#Lt2N)M5XwP53 zx$r5b=?fAJKf$YcLRFMW+_uQ({B0KJ@36Sg!<ziPws4oF`FqUHN}0aJ;><0!R-d5T z7{ID}ic2ky%`{0XZWn`c<JOA<+@2(2TF0x^vLi~`BgW*H0Owrw3Neea7WHD9Gl@|f zCFogjTJ1u=7IA_$3K=)SVi~uPs5M(-OMq!z6p}h512j^_7^<T3mAZ+!=4=@zbqVe- z1-aO7WTNLGQ@3t0J-5o>nwba7R&LB$xwYcs-d2K9Ly9?boQ<70D^lj1F{Y$U+dV8= zl^cNFrDS@5ef9!FeV_1j@G|{lR~ekV#>nh-#^$dxzHp7ng-cA%elG3tF;i2Q8JT>- z$W$-W(q;=iKVkjK518w_z{2t^W>+7|^G{e(%1jT7lb2bY{EUsskJ%XcIqPGe2~6E* zZDmp%n2KHVwy5S!QahWdmIV?f7nmEKVESo4V?Fm66b1M6@|W~p`iv)6K4Rq7M@-*& zkJ(%AF!sd{8T{;TBtSf+<}u-#>S6lI8$A8&4SGKO5w|~ni=NNk;^CEdcyjywZn<~k zXWY8_4mYp<jO$n5<IAfbap&TjsPF%Txts5B=d-uD@yVN9{pbf={O|{S`q7(w^8Syw z@bkC%{QY;h_R-tizwj0VpZ{3C>pfO)eTw1!WzrKjX|LQNGkFJ3-%V`2*U;X-#8%Hm zmTrH><jv0*x$zl8w=XjB_y+T%52bA$us+wr*1{bDw;rmdM-upZ*;=~C&{z+>1CO}h z`<RChdU<^B0Z(t=VC41}Oy9r3V(%kXpURl%7gaX$6vyN+*^Mze#$l4HLm0<~S?hbs z#J#%=+_@oR`6iRScbMtxk#dijC-<4}>t$u2kFB9WG@~ODAcly~_mZ2v!*1_Y^fxY{ zxqS)MovWyB+(di%8kTF<aXh$-t?xc2`7G_@I~bncmcM0eKDv&#?<#v!*C;GLBBPt7 zU|%M(JuAWdG3p1mSbO>q<MKlHj<ad6UxI)SzsZfq;OLHx-R`u5hkbv5nAoYw=tOVb zMXNQSURB2HlsMff=4Qqj9q(gs=pMs^cNiL!-#Pq%L21{CF&We2mlzwmDQbIwzR_V( z*tfWK_Y-d1{(#%}KIT#X4F)E98JfH=wtSn3={wBNN?UG?k`}OUSVw3Zo{%$*${3$R zzc$JG%+UWs+<P!LmR@O^|7P9H>_pG@c56oJmQ*TV=_-9Dliqtz5S9Sp3GY2X7y<;n z_nt|n_gR^xGLlNJrB?UOPk8n)t7l?Ep@`ks*jORr#6?~J_kQ0ouR&CSOczp&r$U7M zqd4t-m~Fj`*fgje7BpTLDyNSgeT-Y}Zhm&h%&k@n4y%o%<bNXNrJQ#%JM9ps?`CHu z#_3L(7x%vZ1A4!C^Nk4X-}2AmTz~WPZ}|54fe3;%e)r}XfB5Ft{PFD@zCK*%UfIvK z%<0~IKn#hC$7LhW#L&H-P4IRm%v(8ryR*%2B*(vfR^m60@;sgwj9NaF{(8%AAODI! zy!k!<^39+4)0h9k@1OjUZ+69KiTi&xAtY56p}z5yqoc3+rI7W1{qhfdEk<l_`-F}8 zWtIzBX47#dlL_*%IK@PYQamO7lww|DS#rJ{@v-XDau|2;-OdERy({D$GI0>p$vH0v z(su{azemy!_tIezY7xFD$N2edl5dtL#MI35yHnZ!(}(=-=^DR29p?{Emie`qpl?^w ze7l_GtGOJnr*j-<Q*5VGY>L@kNXfe5Q4z!;F(h%p{0R4_6a4mciQhfh7A%f2C4G{S z{?GZ{l$>@JJz|ssL3U-%wggW%Wk$C>!|b>R+4U&d4e7)%>!mN_+?^Z|gEh&o?jQ2I z#}DM(UEUlW@N{=u`s9fF(#MA)^mjHNvblQ1-r6pwo7=qD-Q-_|_x?rV#glb0gBxsZ zZLzu`hIntA<-14B@9!}!=Spjv6gPHANrXf|XSX7ZuO&&vTPX)W=V;6O=XCuSAAeN= z@k~f^r#QmNgo=k{1FywBKP{PfGAAH0ZQ;>Of>-kie!iTMNbzP}+=Do_N5^wKl;L}| z?cj?=9j_Kg_-55B?!?Odd?yE4BfmI_b0m#ED<E4AshEyySj^g4olJ@voM&flnw@f* z?FkPDWvhT|fTx=go`@JZSxWQAmnHu6a*~7bJgeax>vBAtj*6oR@%5GnjJ*h7tp_Ch zyjpVed{u^Q(<jO=!0wg^fYUU8dYu>LpX2bj%<f)+ZE-iN>lxP8GOVnPv$T?7c_}HP zBFySSlC`-cixLYnX_gmcf6o^9`o#=Klk>8#P07tZPfABTo<8B>^d|S`S2$i=7InWO z*OrD61-o{7!ouk$(|2dtevs$&8&SK@3Tz08@9g@;$ppnc1o-*Jzr~IW)lOLQ8sp{e zIA88h^Y#7$Z}*mYwKvCi53>CFQHHnow|H`Vmj{P;xwn7Jy~9)Pi&B1k@=(s7^88?n z$Fiq~i+&zVyLeD^aX+c#)r^imK8o=3Eei+!7PiM4*tFNl05!7aZDTdm!&11PrL>xb zoPni+#DsylaWmzJl~Two!)X-PW@k66XC|m;F{EW9qF^VdVxyp9C)LMcvW>%Z8~d>~ zj^iDirn}fn4KW`X;qI(U20JB#80P7+S{zFccT4RY6q?!3wQ`i}=5DH;(?m1(V~sqH z)bdo~d7_<X@j4zStN3zlnBU%a39xps6}%+C^$FANk10FOGc|TWoY$v9+@DbNd`8K0 zk&NpG5l1clv1UB>E+(T2UTutVD*2cQwUYDIGUu;m(O)G3sgh}51+#%l79y3b#w*!N z+>!OQayYBtbYYnLi)wzk@8ZAS7sA<f@?b&Fu?W`v8HqU&t`f&{IsyAp9;{k<ylQ;6 z*N0P`+|AZXUas@7SjDf8M)+z+#eTM$wL}%0**3NdLox?ii4pdSDvnG2;>bEVEwpny z-pyW8UXSY~=Vo3l2l&?~8J;irWbI~_d~!Z8%wDvQ<3tB1sWwj1t(<0BIL)?7--`H} zXcGeJ<mprwU(65j`&|eBdN;%`m$kggHS>kM{^eXhU$1KUO4j^pQNfcnm56ky)!O&M zd?&1;tmz=rUdghin->W?2M!gpBEt4Wm^~>fWj3^eB?fV@c5!Te5sY!Zdy(b0Urh4r z*9-jo$tq6|rH^;F_|>CTneP?ep6qdN_n1egr#yOa$m!{t#5~U)&53)T=bPt~eEmd- z<Z+6hKZx+=UVxV;9x)((xkijHPQ2XR9p(Pc2+vmrc{<a|lgT=s7OHq&ls+oda95sh zXK%5VzQRWO62~(&Jl~Ugo?7|+{V@?XMwx#VTjdsxrki-OuHePCj>qdm?9TR)&1#8d z9TaCAB(oMGJ~d@`KbvVCJ0hkxMT{@|`pI~-_(g2xh1^$mv&?U$+1w6tv|$oguHyBc zo?o0g`NQJ?e|j3`_fJFo?s<a$<4u--dM<r+-_36y`1y_O@0&e~%&dto<+ERHn)t=0 zg>P5Qe6wKU+eH(<ToD1c?%~&)4*u!BS;*7Di&-OYq%VGZkl>#L|Ni(S&+ng2@QbHK zzI!&wZyuNU`n15~!#wv+W_bMUkk`*2@Y|O={L2^1{OZXeUmTr^+dq)}Z17-fiF@0# z9Bh?1*v_-JljdMwaD6{4&N;xNU903t%iDd2+$+XEKNDfFXXWj-T4Iz}>lU6Y8F;dw z<JqEymy+YxOG>_wd3qzqw@br(v7zPRzMa>y*WaJG1kcqxULN3RZHVKoVIJ>}@NCb_ z<83n!cZ4T)$9S}5<@u(OFZS$wedOgknakhEy#486T=JR`(II&#sd*xX=veY`D8}%C znA6uwUcQ~t^Ua)<uNTaGy=;;YY+Ez&ja>8Ba{qsR5$CHTo9xLf!q~&t2R2@<Dfntj z&+Ba)F9k<mFKBq3@8MabnTM`wPK-A>)>rb-+05@xNBF}-J1;U#oO)`w@2%ruxQXZa zK3*>BdAV-o<(`W-(zCC2jcn~(S&^CBnG+nIR0-Z_`Sz}zZ=X*6IbHw7$6r-IyeZju zV7tQudj*dIGB}ZT?q`)`UOs;phe8skhAQq3UFW3#Di75)JQ@`+vJdds*)P${V_O&Z zZT;M}2*??`STVIQJ5<e<;!{ck4NNEo$#is(Y;Ggl*+Y3y$-F|zsz$~3s0{7M0Go<d zc62@Ln};Nn9GSXg2;9<8ULLu-IZ)JcDEB@!*Yjkop1Z?0*ls(|X2&_Uy3VoL`~jOS zKVzfkLst7QGTU{YZN+E2Ot)~)c7w%%%gpp%VS3;uQ=QkDY`@K9O9iFo8j4M|6q*_+ zv@|l&)x^Y57t>}9b1@g0Cr`<~eaqavhav#>*xp}ZZEKnNwM9zH^P(W9C?1rUJf39g zbcXWDA`?f;WOtTGtu2r~J|OwU&zU&c64k6^A=Jx!*d(MG;Cac*)3B0LOEdRI8@Z<! zupjB>bfll7(QdXaEo>QUcpR$dd00qww3Ua-P7cKx@2T3@Q#Nxl(#b=&My_M#Y0Ad4 zaRI%&fd7<^M~gCat0qzYZM+bYIu2E`7P-wrvXaT*WpZO5ku-cjUh^4c;|*~b9n6M= zoHdf$&W~AZ{0S>H|Hw-H`@GKe3rKloC<eJ}YL+@Q^T5==Lvx+vw^8a<&)%q9-*Alu z8Ky~lGf`t3E=4oW_8O)%*Tu0nia5B*(%Ab<kNzVi{g22Se?rdwAqDq&ih(OkrYc#S zXlJ*qV0T{4(b9;B4dpvA@{It@pPq+#Ijd&hSI@Q#?wY$sghV5&{$?T5CJ_LQtRy?x zn;2lL(8pq-m)Vkt7`evrqE-OG%j*pb^FFDw;v%KipD`)>oN4(7R@(obJ;e`Yf1h&d zxWRqbZBD&49LalkoOP^OZ*$jsi&x1yc70bU>)&TmEqfV$j|J67Y>eF$0no#0{_NWt z$;HqmHu|Lx2R~(3b&dmr5SQf>jt%c|V)=*%zFRyFH1OD6%OiUg4{bFvi)T?K^H|9P z>n(|!GJjV&(tOUI_6lqIn>?9NNG$?9TNdC6_pt13Vm;g}&P<5B*de05i{trT_7`Oy z=hUoAFDw?c%%oLJC6p}2dbpQt<$0!2go+5H)nQK7M10J*ad$?^{TU~ZXM8-9<BPHY zUVe}lv36O@ZT9+q#%kl=k-YK=ruS<JU%kab^QWw|zRzOYdz3nUO0nY~m~Z{h%ryKX z6Ad3wX#9|g=1<5qpC{FLiB!YqGKaT$QPK!e1~{0qGnKc9^B-Y7+R6G@6<g-(9N2Dh zc$QP+IkruoacI59J$E&aWfq=i+ISLc;!&i5Cn*sO`7WN71&<0HBGm4%;Fg{?zt7Cb zIhJ%cWRC8zW4_6*`7#IgbDVfS=TWAXyNMdM4Igvlyuz-vn&sioDRh2@zx@hh1C==S z_4r0wNNd|!9BpA;`g~T~$<&Cr*7zB@yZPNcF)+&>=Ep1)%x1E7Ba?m&g@BeZlL5WF z*D~6GP2Wb?Ho#11NCc{g^?;69w}L$pv(LucSeMxwck9J$jj}N=?s!6QV#+0=)X&}J z7<bna9InMU+>qxx2{9099v#Jbuq>k0@d<a`m$<j0=5)))gAIomG9g?M8E0bdpk!rB zuwmZcLD^YJ(RGCh=QU<b6>OTDcqzi~H>Y8qrH9$=mpt`dW2fg5yIr5K-~TcD10Qgx z{D6JchwN%TVO#wv8@kJ^>#ne_7rYp~&2#@9UL>12lK$NmjNW!Ou;r1y^fs{Vu431D zgMG&(4ty6mbbi8t{-^9K{(=3z|IBLlKM-!afU>2YQrE}q^!|v=?*Girzz^9U{xOHT z_t-Ul$cF7xx!xtVL)S#$-<F=QU^`vGQTjGd;uZX6*TO#?J9+97zS4Zik@jQubTWUs z^AeZX*IwdK@a|B1L7sok&d3cGw09^eZjkSl>zXd}O0e?4azS|HV-B<z+0vgEyt+kM zFl{b2#9FSMeK7)eOU-<B=;FIaNq+Gl#y9ukBD(yzM`|%DTX?cL!ncp&d@G{&?XiQ0 z+bY(l>sim=WMkqshs*7p92!JGjImiBV8?fd-O<l^p6wPfndT4AXaAh8|Kj7XDj?SL zaY>9gN|%!bua7B@Tbx-C_x>+%AAQICcwF42k4>MK6^n~`jZ?zSg3c>P7c;{)rWI#l zr#xh((5)rkqb1SUfab<$1ZuAk>Fpxi(@m&Lj@{ivdwYrXb`$UGCehzZyt{{JOBd1B zZendc?;_bM<f%H(o)mjtiZP?;WY*Bhf~%jEXur7NR#K)a8SXon`>x~cxkGfIg-L^k z6`O-ycaTHBksalEwhT=KTY53yZpV6~8PkPoi8>4yrO-DTG1oT>SoGjG^%1l*5!Tlc z7?M!+U|v7QdiX>ftC~e)3(NWz<_&F38#|b@_c1RbYBe}4PQI5_S05{0Ejv*g`vM>v z78%x*o51N4!ZR~Cx_a=6kjm&Z%s4G9d95ru^sM_vILeyXlL4P`b`c*IVD73X&>-us zsAjJ5$1FGgfW(z6cs{+w*rzwKeR2)!$Ct667ofXx9mkEE1S%^?H&jq;yTe3VCE@xC zoYfU18ZNTj{v&4Ee@14wf{>~LbMs|JtFL0Ly1`iWZ9MhW_}ZF@w6!qa)W~>sHL2V3 z+3PomU%SDIvX0}i8m2W3_<EZ#_0(e<Y$C2xP%>JXwRl;y1z8>Q322BT3#3@{r6ea2 zA%uVki*`2q-eXgJ1^~U%PX^W<CYB9m=7;sHss;tvm8@ATEa=^=TZ3#m!$Qho*3AJS z6u+qX0TD`d-1jL2@a+OZE)K?A?Al!HSig_$v*VbD1CO60uN<vLPF*6fG8S>kN?C6k z1)Yc_+lUZ`jZK}Ab(IXU#=yQo&#`q>NJ>Pt-^HV_M?lFZ!ptw9_wy|0V_HO>yHUiB z2%|N<o@JGhr9lHL3URO+tMrDI37t;z+)Y;5O2ODj&QT?VQp;YXms49ko1H&mM{@Dh z-z)Ce%wA5(O1g(d5pW9<ONmYi>2VR4<1QTm-3T6q8n3b&ucngn$Q2F^7bvN&VC$~K z)YXqgHG<8k!J=!&G*pRgpaOe;HP-H0EbaA}8X7RxG-9f0#w)X#X}v_L?S1n7l_b@J zBs2rWH8PVUT_nxDq@CT2``Q_obtlKJ5|#UfRi6_cxFWrI4O`_E2G3m-qOD*p*v@LG zjX7TfMRz4R=S}jCYm8g3kW_v~y7ebw9)3ii_a}r$FOw0`H|eNlLqzML{-O|zh_rmS z0CF!o8JTY}F$<n1W-P6gM6Axbn^})_vgYq$&e$ztw2KK<C)(CFE?jM)wXO%x@Ca^= zg|R^+7QrB6n}X5iL78m@_BthQ2~XV+&X!heeYda-U%@(DMXQ*a^B3!Azf(b9OAQ^} z9rX08P%DNR)%0K(X%hj^fkEAY%h1h)%zED2he>$@_231WrO)1xQ*rnlL+zh3AO=RM z_z=yogyNhWuVJ@U6Yw|S^0uRQ3^8O@(68uWpuHJYYdwPva^Kn-f?6>$(E$Knp?m-U zKdnhbK~%DVKGLpEVzxG-V{ODe?Ic58WD>IPY(HfYbfugK<Khs@B^9fY4%T`_0Qdcn zxoABT<DJ9<ZTO7s*m_$rHML<E{BucOTs><1Lu%3noro+Odw#)v<0$G*1@#>|#`={q zcUCstBD(d0C8`!e{cYIVZ{br^k=J)pRx8CAswoa=<fx?BH_SwP8;h-iQz9VqolWEh zRbotZ5?UrkwIc4%zO_{2({?c0*CBmai}vz$=^=>=*BHEZjdSPP_>Uj-aP6}OMy^$2 zxLt>_LgM;uyti*k&x$A)F}LsN;-0626Mw6Slqwda*Cwr>GHbfXxVny@x}L0v|7fU= zyyRkeO2x*MmW7EyvdKoGVcA!_nWfwy8wI6^ZZ!wFArT=1B2uoi)%^pubnmk-!gPCl znB`OtMG?^HNE0a$t@%_lGkL*~87-&l9yX)~DG}FBbrm{QJAZ$nm;dxx4_)0&Xoi{? zQMY3eQ)tump%so&b=0D0sbWyf>u_f?%FZ^F%?%6)ek&?&(OKU@ZLf)3GV|wj{TCm9 zRROWPd_XBY!*p^{9Lg3G$sMBZBfJBT@eMpdp)Ap_3Dd83F{~Oxt90Ntxykta%tRy1 z$Kul95p?PREiGOeE8TS7adP={Er0V-4?jKE&&O94d~j36#cB;#WQcFK3{cV5M?+s9 z)gmr#cC>S|t({x#9o&|<)6vPz#ttrjdWGwsTtTC4CKT!>lNn%k*2w0TxR#?V8#@`6 zR?-y9F-qkOlT$e+%LVe;af%YYXr5cu3e2_yiPR+N)HKQX43W?zUe5$hR~~<0j!=97 zcWf4SVg`dJi^>$2{f22D4DypkFF$JV&_C?KZV2Nw$MIT|_{K7nBSj9f(>$JC;;=N$ zUZEsPzs%j)Nsgzo92Da0E-aIpSfV_=iAm=}Gib#+;uf&?;<g6~dCz{UAtC!q;POYX ziijJsnrKn0=+vswn)Ue1R(wKMF%fs;@etWmghD>TWHI*cM_86-k}ORpSe}WqF&AfV zL1HS%O5DP<h?8<OE<hA#DqdzQS7I}lWh$H&p*T*;Um)o(67>{hPbtQXAsnM2Y4#9i zbx?>YfK#3i4SJ|;)-$4Vpy}4o-_%LB)Jmz<ptsvGd)?TBvZtsQU(!b;7bH;(lb=a2 zwUj|`)l+@#I&G~DaxTij=^`)h@AGzZgBP>Ye6_#LA6`G>k6*mypI*M<pI*G?4=-Nv z`)4ot-4lsN&v>(Yz?0=g4i*<#nww`{MA_W*46~Ed%#@}i&W`^rW|=N5GL=~o5w}7v zwnXnxh#y~8a<0P2z;GVJ=mI)@S^6u3Mi)b?kD@a~88JoCTVjmbVi+ZFCRYNpCy7Pk zM)N3t`*All9S-TMlJw959`_O+*D`MB5{|J2Y_=I}_A+kQq};bmxsWF}nPXvohVjHW zEzJs)B08q>Ip*g|6c?r`EN_rrIKnx;MX#+$ohnYp$T&K=j@3Df%RNsxv`8W<Yu5Uy ztsTIub&?54-R$F7&GKGjmiFNU*V}?r^=25hEMajhNw4KaXs1XR(h?aQa_x$$5zb#5 zkbVphu$FO|CvjOMtR-3d1V(#7_MOM*Nu#j`7&OZKo27q6*bV7@GRGtQr;ob1)-=kf zJBc+QYY1kf{u%u76rM<eKuqQ?o*?Fzxv{zNyBwtBezJ)WDQ}$2V1#^=mz+MvL`<+` zJSCB1Dm~7u#A3R@bfQF7`Zf?LF(Px^+CRpP1}%5mRa|Zv;oqOra=FQlTkh>k6!4}> zc#}mu^4uBC;*4b8g-iO~6-ncW#IQyK*km2XK#*%4WBkoU9d&~)Ofr8eN0>oN6t#p| z#IH9IAu=g*nYPpK)6pBWqKZVZBxi7sZ(xtCp*K!pk@~nY+qlvz*wb^U!YMkPHd;j# z4+I2*(mFYB#o+g#8j~E322p4OG<S~ilPfCz!zTlLRx>O$x1bhW7<FYBb&g|lX7Ps# zOih$I+?eKce}=>TDK>T|Se01bEwQmB=jZ3RJ5yw9XP%{vGz*IfCQBJo2?^PE#y3IQ zJ%wB9s~;_(9EqXPd*~jtQP*zdqdP|ao#gV9Dl0XeF6ufhG<O;3>M+t!t>WsnKF(e3 z<HO5?{N%iXAD<iMrx%C$=&Fh<x6M@6jB)4oAUCe}(ApB9qT0!&+a|8p8|e_v=o<+# zs7cb?9iy%zCRh@tr87ov?>K|Kap_+>Ds!GIJyJW*vWUz{;;tf*u^g_^1SWNe{vIc7 z9TsXEM`&&urlFyqy80d(TL$Us*U>#-;vYXB<VTl>=+*ntd(+s%Il}2OneizC@f>Df z1ij0TNjPiltY@M*CW=LtS0#66dE8jx!DN<)JIfs1UuN%gh3(^cHutAl*_b3*Ok<UN zn*4%kt}w0o5gJ8k){BT565})KOrmx!)1oU$4WkU3><ru87-a#0IDW?lLb(_gF;;3@ zL}p3)Y&5`t!B3yeQ@7Sb!;nqp*22fNCjQ-dEu(?|mw(Rt|IddtvPG?Wfm+opP3l?d z2a7ax7dd|^!1*f?u63um+Ml9ARi;8Y#m(L%)!nk$E(`So9-6fU8b;>0K9J+nmJsht zBfVFl;YT-3{LKeS{_c{M4{nWeuEEM}8Pd8DA9d2W)gun73}ajyws5X{l=qu;{IE*R z-`*JJKU^5#fBvMG|M*@v|Lae?_`&C`ynm;SkE*)3*f7Y|Rt<N0joj%oQ>U;<*l3Wb z>9bNR=W6>*bQzr7?sf2kb87jVMF`nNzYwF+oJD7wMrD*i8Y`oB7U&m;TdUM@UL3)D z^|$%3@eUWdYGrL*{Ks1j{JU#)T<mJ&v+gcgXA2)RHSm5zE$3R>xzVGeqTeCMfaEI0 z<@N-hw+1*@XXE2a10Tx0e|W<v>z6#ppuI17`RI<kR^^cQc)8RR<VtIl%dIg!Z%c5w zBgv(Xgj_qy=Up*wD{@@!%5h_$%y4i6b#x15WD`w%3td8<hc_7vY{+Zt@1n=GLYIAs zP64$xU4h2ooaCXvxk?XL+QzxrljnM;5Mg_qTQbOZx`k|df+FOi-04YB(Uaj;x4fsj zAhj;YG0E+|81LP-^S^(l<@4q-Znast)nMRSje+ZpR<6sr8*Of`wYm7bWsGx8Cf=*l z@M8hQA6H9MDfrvZ`}yBK=;m)e>Efpq{X)bdChJu~=6!s2qk{_-JzTCC<VvlAD>VwP zRSj^xvX5(bzK=VVJ=~Q0-EMGksY=89x4U`oPA~6QDfy&UFZZ=`qr*)_myhZm9}Pl` z&B`Ecnjr0(0DYzyE!qU{-7xV<je{m#f^JJ1t#cZ0Xh(qL7*p^FjrWjY_rbd#1nV%b z(yX7SMl;1Nh2&!}C;c7fZ!cK+xGu`=ff7|ilT<6Fs8LK)Idt~#BK2~t8IrsYq^VS- zsaB<_QfH{qoz*hRfB9U`d$;V;!wKqSy_Jn-?$leTsJC-n0Q|xYgMjiVm30=X>&(>G znP`>wv`Vh)>a2Wn-N4^{rk1>Esj1gdU8@ox6;PErG`F6SQi5UK3|7|$4$s;<apQDv z2+{A;XI|rWf0n<yY36_WOeum&g#C4yClOU0a=&H){43Q)KE7j?Fi8&!xl~*DsMf~$ zhB2<Uc&O-z@M%+;zpY5}etnXf-ZU+;*N&lai9B^ZDK0ezr7i(JsSojCt)CC1=Rd8| zaY+osrJ5ej-|ppSmo)tByq=3!d%1d}k!#Wy*BT@b-F}(p06i)fN~MLN0R!D4T3g!{ z)HU{VM?Qb2u8*6Y0WK<bx!RHD*6mL2+!6Cp+4<k2wz`AX`tPHzrk#pA?cDxvQPD<K zoq}7fA=y`!tIZ<3nta^u2ukf-w3)RGg@({XGz{7-3<x3hDSZqoZH#EFIK&K$siU~Z zrpS~|NW~XP>doW~TJk0pX=6WbZ5Ly@Ara*!5tc42P8XUn1O1jh+I98R3|4Zft%e`e zRPyg{-Qd4e3dU4xxmu?ekz|&z$()&~YLXrhp;O!KmD~jRygk52QmYRuWq$5-aY^EO zt(BY7D;FvU_(W#vqiPX|O>S;=MQG{`(jn2+8>X?{#hnHtS1XlVtT;1wT~dcWskKMy zmF9e7m`n9GF1I-a)8zAgGTYr2xu=Z_GWVag*!W2L;HOnKet6r+k3=N=w93g(Ykd6e zIX!=WPRD0WAucq@8ifl!sj+Zb`u0lG7*`v_Sk_6@e;-$6maob2x}2}-@=~W<pmK1X zX7xO+BG5bAM(Jrc&?jPMxZ8@d%P3(+(_?0&XM~aVAuK(8I0yUj4fkT`XyFGJ8u;P) zPSib>SO?nC_H?5j7-VQjD+0zQJtdsgCAq5X=lb<#Zr*C4p=}VgR*%u-KyNG1C9`W7 zE20pi_j!vDdZUe-eSW&EIXWGaG?@#+V^ONrW^ReWx~2A0A-zziPt$Cipjo#`i)KS! zAE#22;)XIQ?@P=4XSl7*aAiRDJrLrSaOTZ^!S=p96@Bu&H!B>H<4W&>^kfcq;n|<l z^<RAaRRu&YyDkna%#gxOcaN0@ahVk|=odaxaPi||>YHSEg_PBzkgVdwyuK_EajY?a ziAbPKG<@ca#DV$FevWk(qmZ-4I7wr3h)Z&f>o@JR4QJ@m<>@gLXqBWk_2+5oE53^c zA*z<1B%MNFT>|{w;z~M);?n3oxxa#Q9}c1zv|t()$0shxY4b8BPRL>mU^T|Dn-daA zIhVw4lA|f{?)jK)oSsfMmp@U^Rc~e{mS;Vi6<6aYBhEY+(}+vy!5OQ=7Px}deF2a6 zI<aUA<5?Z!6F~yGGS|g@Ur>gnai$seW*PAca0Jx215KhfuhDM2%nj9bKJKsNy0(ob z%OEv|Zf>jFsni)c-)rZ8xv1yECMz8xn5u?^3>7^z83ah&KDrz}>Wx+|4-Rvoub=ls zpuFGQ#s|&AT<SE6uyAwXa}i({4P2B){<Oitr$RIrnk-yuwsKyS{-<|jpVe-z*0?!$ zONREkolBK=A>uJPKPEy@*74CWpPkci{<@xPLJC!qlZIvs9pZon`U4CPMo}r#m@Ne? z_DP8Xqt-l1)Ax0|anr*24~M93_R+4+(xaWEPXuS5411pl(2jv=YFqO{hT~K;$gp(e zXcmHP>dsQv9Ft)cVw5#pJg?z)ou8W4Bo$3@ZV16#6N0&1;o|ZQ8<(z{-mUlIl~Jx< z7I!IP^3%^c`02TBKK!Vk^B)ZI*(J$MwT{X*JDq(YMux*UL@0QqR=$xietnq0XoR3O zh0l?D_ia(sC9YxEPvsRY?X_n7)))zYmPBZp@t8QC*c##BDgpm2N&h%mUyPI|M$k1* z$Qc(W<ilqR2pL(WFI6-)_|fQ!jOdGu>av)$<1$ADag{}ewIw-D2w)W$_-`Tq9+2}5 ztzj;UySQ^xk9OFFZ`4jmZy-D}LSV#*OXtKh5+Gy@5)v2ew<ZbO;<A^x^mPc6!9}yU z|BD|D($_miWIV&f!VK#hn;fsK^KfmO7pIT;>djaD^4nkW-7kK{Z-4n6-$}gr>MQmi zp5mJt=lY12A9r=pU~}Nit&^F0$n4@PW|v-4ntx6{cSO!LNzsvEGB!u{?2mhmAL5Vf z;Ph|N@7U#*>8=Ra2TU$LWo7FX+lN1A`{-*{_r79b^DWb>Ur=0nLw@n>?_|6tH}{OB z+$UT(LGN4U!+H<@<7Yy2?P<Dg2lTog((ixCQ1}hQ(Ko1LuhB(cq76Ss6@1EY;4y># zM-2MpeclH&+D^EpKcvcZM2q!Ufc-9w{}v6hwuaFYnj~7}d57f)m1PaPt%T89pw}`< z%V?SEMn5+%o2U^otLZUvv%Q~Ar52ZSjB%ffyw6M2;=!j8%us6>HxDu89bz%8WI16G zryN8fWE+(pd$qs8o5Mxk9?tQ_{v<E9b3EHf@nR{!i;|s(IUlFxASbJ19B*6M*>sVe z5i#rN=YzpIZkw9v@u=u?4>I8DW5Cvj!a9t`VZt1UVo%KCNpBHM@8eA#VvFoy3am3Y zQjnffapk<4+cjt39;9gKE73AECwrNtSCyw<m101h77>)9TOmSEjAGkhiPnK4jh$H$ zkREP)ewK$AZHm&nZ?Xn-F@`M@44L!vjwTrtd{tQzs03e!jR}Ti)&`}Ahlbs>cR9HD zsjT6Gn)d!E2Eh!IF2(3@fKd_hT4jWRA-Q&kpLW5tei0(Wf;|c$QsrogA*n@uQ-oVr ztb+Y=ee*O*F?lMTIC`ZQMYoxOb_o$Nnm!xqK?hxe?~QE^+C&`nD1tP#31@xU$;D4P zM2KJEX8ldB*Vl5hse@ZBBC<sUwR8jokL-duVi+q1`0Q4fh}l-IR5Wt_Mh{m6AFh5T z7$qXTrO{4nTSz_=qrNrBosJL{BEoKVi2;*yH{{=Ux<ot+pS2B7GHhF8V04jNS8Y^W z6KqmWGbBCNZ(5>bI8SY-hzv0h?ZcAS(G*=GKKe%z=*$yXZAEPM5~j1hY>seSdNL&D z@c+r<uPPvXr6~r?vRUm2EyF5W6-wF%&FD;C0YWLzP*O<GOXaYGY9YXSlK`06Pp3W2 zfJ1=NHZMt<q~9onJ({6oBt@Gzy>?X;lYnlIGC-RlL8H+@oyAUtQ9|$Jj^0MSL0pdz z>5x%_%A`hPR-m=1P+QgX380Rc-6*v#Nw5y1Ly5^bjK!_Q>{7|U)e<^PXECOiFbJVb zL)ybMjYPyXiW3uMI4WxxP<t4Xb*s!qxt9{PfPmF8h|{aZ<{v}nmxi{u-hEB0HU~s~ z#xXlH^4T2v@flRfIfjK)2Q5?Ye&bwaIcs{I6u*}?RX6nm{nSYAdgR(Vr$PpPkS=i< zbp~-vLOKJ{JT0Cyb#^zmjd}qJEw#2$I@~sS-NqfGI8CFOZqwP<eIh^x4dP~9@4m*a zlVR73xKoP@sBU(MT6c?6l4FaRx(+SX?cyey1o&FD;tZAC5|?&UqM}Q|o$g_2cms`t zZbmF#0TQRUV-XG-12syesNX?u^$$}$sF8hW={JnX5a{U}leI}S$r|gdqf}W&=@#PB z8b!V9#W{<ks<N1=u}D9Nm}@deXf}ul6rkx5fIf>parXW0S&112-16KkxwHl8mzp#T zNqvT*)GDIVa}gPSsZYO^VG*hWA{M*VB09BZngp=xhQ)CXb%|>46Y(kFG3uc~W#F1p z!40j7D#Iwv;vBlIJ~ZwO)=(K+u#81S?x=qPwTQj}M}%IhQ}!ng!{U}}I5C(kGO%Xs z)-e&H0gT4Dh~FR<mz?)Vzm1us4}B;s0$}E-47gv!YY2l7yJpmjb~G$5KTOM@i|Q^r zRh?4Dt})5GLtLUbs$Q@3MT8b{9Bl&Ht;!G$>LAS`bW|pBvf^%fG)_k3x+ZrRgExjQ zkVYLW2x*ovCTAIm&Z6~BNlz7JZ&{S?1pUsitSNxjaQ0Pr6lJ7<K0Sjuzl=S<il?+h zaBh{*(hkwpLx}?di>p{mSu|+@;)Ij#kmNfd1dz(m9t_bIaMBs@qK@aW<=5Z+-il>> z4M%ED$f`s%J3%l%i7P80m&{=dN>9dT&=oe&r57+te~-npI1@RX^7>djhh4%J&0&fb zWFJM8u?hO48G6EE6y%=mJ^@m1f>xK1orwL`F$sH?7F(JIYewc{oNBvZk3B-A-A9GZ z$t{_c8&(t7t>VzlBQ$y>m*Wex_+wlYr++~^$VIJ!Ya<$JO?uj;e|v2f$z2F{YKH98 zCZ*|JF$dcOk}H_KGqiMygA{Sxr3ee2_@wTmVkkU9&{M=D52^AY;ruR+$Rdv5G`{#8 zp4g1^*91Bdr`>%)ZV7R>xA^cRw~0>fk(@msG`Ws{Vwpfu3|4WH$V7?I#4MqyMI!Ty z#Fv*zEUn_5TEdW%`exH2pqx@$1!}<%orn~Je-M38fzq#{OR%cRW1`t(r_1F=;f+gP z=dfpYuw@qUrIWam2|BcbzaxUhLh!?OFPbsCtkZ_x?UCzA4|%5O8Jnd|2)bTKzFJ&! zwJuKANC=$>rG^0^zX3HBf(?~}gH#VIsZt1@3=PN}_RI5rs#QZY8uYXnM`Y$Tf^{0f zE<M9~=}D85N~M`*nYTfI25o3k)?W}zNYQDCq0|R4%6!{ZCaf}#2I*(5)KBFNiC7W5 z7K7NK_0!@=(IWWUW|KY<>}omV4}+ch5trb*iz=0w%l%5O4I60_9vX~nq6jWCY>?S7 zdub33Xpp(BHORF_y!7eZ3~M~-M4anPGP~A@TsJK5aSN}BSyA?Jp}&_#S^KEXO>OTe zH;0G0qtS3jtL2tj%@viJ&(%sUsYc$#=gJXo4C%R}6ued&sTWRb)d>fU_ylMC4BCt6 z;>!$0=IQl`D05_K8%xpQ6n=DsgiCGI=#A7#G#X6bhu%V`&V$Y6CtX<mbGrVEkH4yb zSU6cAwj3m~;3GWmC9x7Ay_RC(!6Xa!%1rL0<liCUs{!I`{&&aVLXhBMgz%C$v&E!* zHcntRh@<3WG@}<#8^Sx|V)0>#%tnIIxRQ}bANpt?hG@S4&H(1fAoj=r9ytc&Tu@$% zrUppm6eRP5#7io<e}Vba68@4whGZC1N+rOi!J5-xm)G61!?@-(IA^sIIz00x{3{N; z%T9dD4zjyZmL5$qv7aHdY?E~x2rTGj&3Y2c2C{2L%3CHTw=EPlEKF>;DQ<WvZTjTA zkLkT6OHXE5cs$3{ewNAI3`L2Fl_1%YiA-UH@!}|%X%nd_J>xT4ro>gu>;>c+9>(X* zOl(G2d@{}CVV=yagJfCWEAP+Dn#eAg$t+sQ$l7x2b|yC@)?^)P4hriYS&yHo#Sjl; z0UiYc%uM;nEqlnVxaH_zVp-O`Y$dxSv22#}CW%pUtNQ;-7%1%pS$bGx<|r-ek<V=E zWe+-v^Ezf_&8t&7Hm7y0&1zVf(=a(N`&uv&n=ule7$KNdQ`(NR^q@>;!9!5;9~d8& zql!>oEdZ$|GN~ad*9uAa<a*B12=<~5OF@e%ua$oBGJ9Gex1GeC)1yfYp-T>999Lt@ zsd376Tm?B!Xrzy{QeUZaNr|O6jJc@5G%-wOTXHFDh^=|Cm9<#QYTPphJkwU`zcErv zlKTxOYvS(Kwnmv=lYJ~|NX#pV%?*;1{4U(fNiEVOmn_7jZz3}$f@M3wDJQ|{F+#Hz z0@HdtlRCy^?Y6wEBdfwD`?X3Brq@I7){s~UV9JQ-jSZlR^$Yp-p-v2<PRhBo9^LpT z`m9OF!OCdPB4I+GAH^qmm_Er<K1z|@7?Zr3DQy`jtf{4rD$>hpQfoTKr5|(a(u+%B z^3zeNLlAdf_MLZ<Sc|_~LwYBRZz)Y|wJ3F;keWq^&YktSjqqBU)aC-og%XjXm*j+s ziK!kkWtp>S1I5D(vyY}ot~>Eh_TruBB~Tb5CVBT4G?>R#7^LoFQjbtcC)XIkozsf2 zQR7G|DF_ZMKAUHLKgncSPdYb@KRJXeF0-GK-pK2Sm5rq4ETm-aVhbw4i6J4+0a@c9 zDVe|Wagy1GWpbN-Qqni6nGurGhl%MC;xlLcXD2ylr?6mUa?wO#$@uPb@fCS}MNeTr zC~KIL+4bX(HW7-}6N)vxi+HG>OrV}@yq!#@hs1a<;bb?iSSRLaJL*6a{oY2Jo!!*f z6tsIf7!EX}@KrP5twJHMYvaArpJG}vO00rWc3&56PY+&~<jSKEhiJg)u?U%&=pQwb z4r(dq>@08QSUZ?vb+^RI`Z%jAXa7yYIN6C9fuIY^m`%)%2UjvlVzwl`FvH^0c_t1g zq#hv>>p|jY$Bi(_t%yXN@J1Z}T7rPoHN2K3yqqVrI7w(>p49px%TMN5emo;sGDUo< zM5-t_H5p@SGQd<p<|5}{uHa^7!pmgdL4MrAc*;yVD;QL=QkMB%yk8{0<d<9x5g#8U zCdc5o0!KpnUuM;wQeaLEVoVRd%gabsgicOGPEP7RWnku{z|3(DchQ1Ldc|C}GFBeL zRy3kbD1_wu(aEg1Q+*_JeZ&gABC7iE6?&w{6co0CEZi5NH6K6~QK3&t&rL|p=Z(Z= zhNJVcu344fr_9Nuf^=z!{EVKNRX6i{lH<E&*~66J)Fh>yJjL~d^p4==w2Sd_i1En? z*|U&&%Fc#JOJrx<<V$DYI4cCtRg^a(f|Vt~Q$KFmyJu2GbmIF6PpAn_=>9E6WL|_% z3T8rrACY;ttXuG7+Rog)$#?u2l{y7C@_5$^xP&Xm=KMHjUAX7%1eWBziy~I12N)^! zp(+eYE+vmLo8!AN)}G98@9n)mr|ZA?_^S$tr8yrycN5`YJ3dz(0e2k^SCxRWPDH>6 zzF-Yr&mDZeS|ZXok*r3*#3H0&B~o@1EV(3UA}Z!01ZN`BJYGD~tg*O>q7YpwW55^D z;SL&bMJzJFb{tVNV}5aZJ_A;-hy%A8yF-b~Ie^d8PsrOrz}m?~I7BfXA)k?k5u%C4 zq>xc9@q`F~v^0L!&bWYTChH?J?x!FFIx~}@JYAqz&cFM;hX*e*?C%K?$44l~1hf*P z|2?K8W)cRuj}U{ehjh4y@pw1cOc#Y>4>OAjPM-vL_&mb)o{_ap-Md&_)3CBU%*v9) zvXYe*CCe)cmX{TBPtEu5TNh$pRk5`^!pUBU{WZG?0F_+3lcWGsRE8-bU>r*biN#0p z`bQ*ZcH(gnKG`7Ybdbqnn#p*Khju^D#0_s1L{QC6FgKHDZaT|UK19+fpl|6SIW|DX zE!PhS5e0_G_`Ap@luQ&xSy{7k?^%q46E8EfB5Ddoig^<yA+_?j`Q2~IPbDpslR{<^ z`NWt2uaiv7Njf5;C*XSbLtLl3L6*}ZK*S}DkM@zZ4UlyVGVT;Ya19W*^$<065;1oW zv2+l!w7;vP&(%X<tc`qhh-9!AyQ^PZw+I8D5OmmxH{&K!65%?VA-9;PFqNQ`4=^z< zbxnz=jk$;g?c|DQB1_g)u#<||$i&AeWZg`cLd=#EOcj%qClZv;{sYrYfI`eo+HWV} zG7}#&FrT(@dJ^SuC&+BfNY0@mGp3XfP_ip!{R(0(5gT3sD|asum+ZscM#$BM-_e4{ z*+N1@O=)M0=@m0eB3d_QbZkv)*cCCdKR?RBqLag=0DB95HfMbz&OFQ)T}&68vNkuF z8K01Zm+>_ZGeY$1BL244+-wS2Z0)4kJ}9z#GRwh(b&j4Ma{A&C_g=r|@Wo5EpFU>k z{sEDV5cWkO<jpA4$GdDCpR&ER&(7)wdmHQAJ>KK#qx-zMcaJaS-*2b$d{qeZP1ebq zxP@(>hD>e*zkq&je1x5%ljBwCv%MTo5Ar-cEb#cCz>}j1UY?YAdvA)L3oyUDd-huZ zGGDW9e3K&X<_5$;4-*%%7@x4q%=ws~3$wAD;BX_$>2{uz-8@IT;~ea!*xgI8xff$y zV&fph@x!ppXN>*(F*c9K**Yn*eQ%cC2TL42TIcZo8V5&<9PG`ozcs<mN{ThP{?eqI z1?hp=w1LTpf~ky&g;J2!=@9FJLmOvnm<fs~4oD3`Yzc60&be8ecCb8YWobgsQdUV( zh|CnMqsMg%Pq>lkWUpMapG6^xRT23c3l6rHyzGd<+FO<xUW#%sALno;&XM%!(PWa1 zu<ZNnfAJIyJ6(wKPz3VR!yK=KOkbU3d3`r25$Da_2+#KY+}m<+xNKx^PWoj|&7p|g zlM|<4OqAneJNw%v4tK2_9Y{@%yxhMR;n9Nxj~{1w^0dISrzKuLnUt93`NJ8GwilQv z7VypqAxS?a;$mFFZpMRN(t#jJuTT2lPtp|@j7yR6rO7yD)=Wa2Ch23Fg%yW^!=zxu zVvtAKF`kute6f_^^-7kfOGWN4PIJ0E$I01WFJ75uxfEqGE!Zo#m6;5aT`7`Y&EuXP z6@%$ya%-IJv;P3`@fs)3HaK~@#?j+ti4`8cSm)`hb)LUk<MjSCtIJUm2_rgdJF0NA z5UrL$f9ty+ev5@{l%iwI2rf=b-_9hg%#EMbUc{@!QbCN-M408G^n_sMQp(BwqY(EW z##u?p=X~-RkCAD&ktvUr>9B|Cm|yA~kiHS@2#V?QiTL#>NrXhep8d{>n9qkV5^QbD z=hss*TWj2XxWnnQ15TeGbNu`sr!VjG;Ppf9zkV#U`J9JezT(N(U-0<r*FyTQS$cZR z(zAUY{QL#4zIno%Z%+8~>pi~wYLl<t&hyO|C4Tj`#CLBCeDyrW<3k4rOQX!@1&iVx zSkm=`x6G99rv99+|Kj8SQvq>!bSw>3=8ymU&)@%Hp8uQ2x8J_t>C=6lJwK4+?z=cT zSwxrWL=|rlQ6NrYUR3{rQCydR@v0EYy8HVOx1JC%Oo#(WZU!WR@?4G*G4mL)N-cpZ zJwd&=BBQ7@hntwkC*bTQ>X6TQy`(~ZQSK4}AK9>v;#qN1ev1A8(c3PfH=S$=u^fsz z-#_?14g_Qm4kzVYSz=mZ=3N{e&2jhcybR(3hZ0BYYrNgq<JsCC-=6)wh({0j<%0+O z;@*9}y?gpDzCJkR=X=Mz5deO(y~pc~9bRwl@P#~owX-9lKvc1~#+L^he06xp*GJ#S zw<i*(NBrX9F~4|p%&(r@<yTKn`0lyH%Ts>+>K?y&bDw9&E6mS3*xgL?%Wt3Xo8P|S zyYJrc)wj?2QsU*84|(?H0r#ID3YZ>q_sOmd+&+8v*9Az|S(=|BTO&k%$;M*7#QM<! zTc<Ld^4i|LRW=W%+1Q<A=Wve0lNFBcud;VA$+{^2C2{%7X(iJWA};3pD9jA7ERJDK zT*N}Oi?X<^vaf+Dabi>6CMMkV<Xp96T@4bAq?`@J9rYwU?cy$5@J~oT1Zs$wZW1!x zAUJjhue%b5I5tbL7OOZvTf7xVT10~=>S$R@Ox$Gh?3*F&9qb?d4UX&oKrq}V#500x zQjJYquPxgrpKZYr7WWuxB^B-zan(nnC~kDtCJsb|=!}=hu!%^mj!>(fP~RvK#VE-U z8(FJcgpBmyn2)$i+_qQx$r})0ke(6{Ek&Y~BQXkQ55*yIT^c=}fH<mjAGTC4qvBEx zu{KP}POO<;9C-z<f;eh%<7eN(B<HOX0+>r8bVO9UMO>!D4Hq|D;>4|T-XLo{`!+*? zx77RYb#G~aKzW$Rv;fn*h5WLc(prE85eW|-u5x^LniX;Js|!vEhXAQpL}y4M%$kUa zwaqwdTS<AI5K)>CS3AzaR)MMY9QoA{xpkM^Kg#T8o+WWPYwHtiZcehXF(IGHv%Osu zA}R~<7T8%&b0ALmcq7N59OuO}*u;?!C#1f!CL#54_Kx!GihJBXNVB`26w*$zCr)U4 zD<r)e;9%3wt_Yo#1uHXBQ(LN$TK_pT@oEZ_T4v@1<QK*1u6UUjfwQy|V{s?K;<oH% zCo1<4P?qcM$+=^BZGR`k%$m5Z#X;ug)k6FR7MILSua3#SBv;!3%3EQk#f{I28$A0( z$E7J73q=$2A_5ks1c!#D?|$$<5U#%@0@BB#kkRUDlFh9g>)U5P;*?~1E5@{}XL8v? ze$h^L&LZy9C@xe_RNSRC+lnq-g+1Rvv?TQ^4il5}@hR!uvW}?qV63DkUK%A;k~$V0 zOrD9CoQ?U6h}yUYm#<pPO&hb~9_JQK%q*FhS{4JfYLYsc7~hoGGRo0FdQ&gY#kB00 z#c>-+Z%98ci5r)gJM{9!SAs)dAMpC?eV%{0#fvYudGU6er*F1+{AQg;U##)ui*=s7 zUgQ3YGoiJ>)OL!Ii7pJqF7hiP@^^9)1*yS=<X>W|C{cPB3mZk|7c*i&qLRlTi)FtE z+%YB-BE)>1Os7>MZlvCe5s4^!i*a@r6YMS}-^G>)i}l$cD`gjp^0~!nuhb?iqTh}( z-het$PbxPowX(1}=M?<(QC@a2DLgT;<zr&oPj1aYZox#htRX3`JM@3Hcb&~`Wy#e) z$iKmV!QbsKTQ0B5wJxvA-kRO<j(2u6Bh6^2FrqZdS)ww7M2Qrsz??Gx0w9Q-bB-iP zANRQ+C=ifDQWD>cRp_d&hnM^H?Qpu^CEvyWz_FI!;;Y~P2EY54f5aF6^so5t4}Zju zUw(z-Uy6QZSz-D8=fB|OSAX5&Z*cnS!#F4T<?Po|)c#Qb&2L4wzdwwtU$o%PVJSe5 zoJ0GM*U;H=9o@%n2w1v~-lLb%d#Dx9zWE8=Umr);S4YwD<+phBrTFx}Nip?bN6;&= z!Zy?n*LV;75|4wr5d`$3!s7@oPa`A%A}C-YFh7of<bc4$5CY>v@Q#hb@X`cbp9#U) zIYg&M5E*|B-$WnW60fZjT`)~NfPV5G<|o@QH7*72@CA$vi+%?#p|Ag(`tQCcq`;Y) z8rYev-$9=YK;&{csnl;_*=WN3hwUhrD_`py9UDjY(;g&}$p%o74P_Cx#Syo@;gB&T z2zwAi_g@J@T7koCgL9v3d~QJR7hRY<H-^=*71*X$VV|{1vbAlqZDv)Hu3gxn*uCh0 zN9R=Rnzdv0>@=oZB@xd%5L)#iY~4GDgxwQv*t@K7_BoL-$B=d;k#(hz^`wv^`_jn! zg@YLsL=VNVa3qIfERRw=kGyQNjjRa3Fe2?uq8P7iFGh3fIG#i1`T0NwId58+*H0tk zOd)Mg!Z+oGu4`V5(SlN@pgu^nDZH^+MtNf$YvmG3!o{_Mu-vnhMNT+VNFkF?$(Y5U z?^(b?yBMt7hh#R6R4##ZJ}I24EOSXDGjTay?qAFxBkQR|0Ev(jQI84E`C;hAkoB(~ z!y-Y%Ht-NO0UWl`CvZ&kz%}y%?)d?DmPX(a2l1I_;a^=uz+ptdX+pqd!Tj`VNdS*v zkYl{I1-LC!a4ZkQve1X+$u2BP<vRP~GRAvOV^o6mShw(#lNgr7JJfj;Pmlfv*Z%%* z(e}@OhrTOc;C1^E3`^o1lM0XL=|=CvJl`v}BM#|aMj&WKFzQ4w?m;La<C*Y_J@(_) zm*+8fwGZZ(23QA8@*HAYqZZi4_?-a@tmE=~QfzID9n)52OYT{R*rX0G&URz)X1`R2 zi&)Vw!z^r)>Yi-Y8(>^A$kK?_WizbH7T8VJE&I0%JD?kw!|20T&<)PPZjgJJR;8j? zkz>uU7>rxj(!s1VsOzs9L`I_p4!Mp?t`V?Uk#bs*bXpLxuOMt&5nDA^c-4%cMfANQ zY&ODgT83}g0H0nDugK<+>$#Se;Wg?ITwRc*LDp9VwAy6s?C|OwYUy6IR<`Nvirrl5 z9q<~QifywNsj7P=2|vK{(h_3vFyiqLVzD5i(EwsmKVlKG57Dp}(U1o*;b_o}NYII} z-;Rm7ekm%Bqi?(&L0RW95#g{LAC~hXa*bfvgJ8%l*OY6?b>p&3$aPb)jQd?;KTGPf z&_0_%wmak)Cn7NyLUFgSTr2KX92JhmM24taWU6o^;J|cu2d@14@9?a(6;6p$9;4{M z<bdDo5*v0&h38N@cbR1Tj8^pxAN%r(`ks$X*gR(tVAU^4^e(1fjlwc-6nmDj*O_)O z$ChJkmvQ8E+-57hD^|IO6^4l!JbUy2V*~vN1?>n&T?q3TgoAMx{N#jNSY+B_t_PD| z#jdCggQjP=J$eQ@_Y4XW0~D8X%9oWti=Wf)D={It&8Zw?K@7o!7t^c57&E;@DCw7D zqgx!SVOh^ak;p`lNQEV?1QC}Q5s&*3_N~G=+lQfN=drptfSA{|D@VOni7i_!zg;FB zp0{`6{_Pv+x$^|wZC!ZE5AxmZM(5op=)B*J4&ip;NB15}QPYX`w)P6&Zb!$BN4S6L z7S8;s1<$WMg=tXya_IeR7G4=P!7^-Ctbe|M_9M4(_b->Q^lSl+G4a`P8UG2f1!2#G z<d$(K+#@!4N39he5gxX{Gh~MAl?jVir_ul4kFk7j9!Z0{!bTrbCjT2Y2V^O16kRV% zOg0GXy-4ajh%b2H=yt*W*ooMjY}3m=oey!n*spLzFSe~0+t54V7oYU$#WxK$@pD-h z_Q^Ke>s8F&n-y^4-;t}|MV|~nxIG>`>gYnLRKls#XR*G%uD1WsazX)!WHPl4Ao!I$ z?k}cF$VUo0I2*_zG8aR5CWd@m5?i*kZ_dSX@QVTxZb`iAtA~xpq~jSBGjHF=?QvTI zvDG*V$q$nu<CEu@jw7)wU?C{UIV#T=E2uBsDNa<lm@HLTbWu!~)aQuF#e`hLl0(`g zKp`r}Rkx?Nj;)?2tgcnPZmd|jm*_I>%pxi>B+Y4*a%)QG`{i6Jhb+G!T6*{S`FsIu zYmNKY#1``5G}6{IQmav<Jt1TQVf8_?Y$%RgB!OHsCGw_4b^%gxdDf(Wq*P&tE>Z%# zBBltk0<zYMCE)@}c>z*cIWL_+J|X)g0i`8*q$P2r$dWvgl5paZU>u{5G5F0tVEW>p z;WthqAqjx{(~@v_UOp95*IFy&vHq6lAV(}wIEEb+IfA(cyWNdb&J6rB0fhAtdG36J zy^ym>3EZi-?<;1DNO^*YNb=#bn10uB@8*Jl&%F3V-oua2iT)CjEaEjBt8%Ww(Knor z3RsN@^Q(TYF!DYbyKuhYId|i%KZlSZ0{?6PX?JFOjQ7g4i-Z7b@3JI!XH@K^@(J&H z&WqpX5`~8CWco?gokPYZHpefpzGb}M%i;^ka11WJ1tGil-ErhS%-F_nM0lqqPFYiL zbVT2+Vfyp7Z`a0-IPD=Lm8bwPdqRBT?RNV<rq8k<2ntBf7BUU{$t8JK`gi^I+G}L< zEF4k|`}V}8qS(9?j{*Ss<<@nH8T)0%jNoDzo@oL55@&*r$i6sWi@>|&m*Pf1nD}?S z&F+QMz6?B4usKJZGA`+c{bc$}%AaZ2uJ%ceh?~T&2gDxC$%g%V;anh(*g_J~nFKQ4 z{l}<WD39c78j%SBjIM0sSk*8a1pLDFCj$`4R0?O$Uxdr;!KEwLwr}vf|0Qm<3Cf<C z!^YO<w+ST3o3g%HZsamQYn?8u{ncal!@O^~v{AYKX5)Txu~<~c)o-sdzmZOU-!bpw z&Gk+7Dco|NpT2E;#=S6uT{&A8IU4uvg*VnJGHg(W57gyuO#Lv3@bk&_`_z7sqns64 z3Y#CShmE3)MOL0co@c+~q2!YQ7k8PF5ztUxD>WRm7p4rf$&JEBgM52oH5QpIdw9oK zP>=MR`t9UR${>!rA>e7VyuNQesB2ITxxcc94;+hfQG8PpV7X9k*taK^`<8PXN*5dS z(fa*+;Eht{8XLvNeN|pBNpdT$H*8nDUa`k=t}-t5dfE$f3{{LMfB5<9xv~@hYwCF# zWZVO9>wz};0rs#LE{k86#4Zo&`ea^vbK}FGLFrQBB#W*Oj+OLp#_RXk)cctC<#?(X z@eUbiFU3NsVf*%e%8)FpW&OTAvGT3=%di`l<=%NISZQ1JdT!)T*n!wkVBv-8LRf9K z?VAh@kKpF*JMehDYJE?{!}fUShY}OsfzwAm0|D2bb@ndOK$lg=e<yvc3?F+w_8{Y4 z=p#?x?KwV)jJwe;GVDewU~hr)52lZlLD4SPsK{9<Sl&vzoUaA_?`9?WeH0z=bL^_e zUkNEWKZPzohISbX%D8LZdC&QKqFrRz3vHBlk+Ft88Gs<A(;0+9;qCbkB)+=fq{`ZD zO=?n;n$)BwHR=C1eNq7NYa=4QOeU+AY!64H(H%gB^EG0zI1-7ZLWTr8(P&J)-uS2r zYoA0JtH_k{<@5P%)K57mUpAXn+qhrJSgN$*wIb?@T)k&Do7<sF>MWi}Y`5W)$)xBu zhPCy|To|pWG9ROIPwKQ#DD03kU#QI2pe{MFx2jhyiT5uSOWUvXC?mI3iSHP>pK@n1 z8D+P8F0IGA_B9`Wb6b_To=T<FHXg&`fn0qiEr|D}o~pXz`}#uRjqh^&)1tqsf8}y{ zrAv;vR=VVUs=6et%a~He+Sj@Ccg7x$H#wC~E4j6#_4P{Jp*+=iS(Ul6e_hE)9ai<q z*vWIWBI>NxpSffl+IbAeIj3xcx@Jt`Ia(2QM_o}*B+60MS?w`Zo8WfpF&>Yr^Ry!B zCY!71j`C1vRT;TXA7P%U`U=NBl}goKLeXanAo$P&14C#%eO3ZnT218adESrSzJ3(8 z0K&lFFrGYphKY$Ob<Fh}w`Bhd`27LR0^#zq2}fH_z~}QTnFfbOaOd8G?Y~-Cwc612 zq#IYRT}L1gL|^|4S$~S|XT2(bYe6*b+iiF8>h&NpVo-b?b>{4OI2_JxXL#1ri`#eZ zVRC93^9zf3EHb`$`C2)LR>9hCcc}R{3yVuCxXjMY<KBaZaJk(|2B*`d^f))afSK7j z(P0lBJnDeURoSlvG4MW>`#zQNi$tOd9NxTr7h@BX%7F<OdY<>;N%u3DS61*`?6ITs zF`O>fjv#c9#4&4^d-nA9DH{s}L!!q=7#p8ZdZLUYqhq*ns|^#AQ<#(Sej@(GbBsoF z{pAjX7?_uIIyJv)NvBsfNO>Q1bZ!>_l;u%-r})fEWebDDBWitR#iE%m9Zo0O#4q?N zmoiU_-BIRHIILv!`TgjB@d{m!yG5ttFqzG`dFu{(C04oJp8Crj2r)6x<|r4z4P~P| zFJBF){Y=1mgF!vVv)<=eu~^XA^+d%_yTiHTS_ef0hxF^K*KVq~lM~>=SjoTqBT<*s zFJ<E-e#R)qA;zT6$4@i?5tJbwPpH2_WL)98*XzTHpITK;sm2)27i2sg92!PgV!+Sm zE@EVCT*Z7Xh@jwMdxsit$`loUKk`Ehygr{=r~fc+GJf-~D)|gouisQMhr^K_*ZOP$ z1St>*;P{DC3LH4akvqib@7;fh)m1BuCbJ})>o|Gpj3mG3DuI9X^&zyLIR}r&r&(1> zgFAKlXEk^}F)}*tPpv<zziMHUyZ@kFoOf9z9VR{oZ3e(wl63gGNDHDt(b!+Se5H~e zliT6%e^kM7YI+7bodJC>UaCM&aPvSMh)&G8FM(iCyW}k6{7@WsaZ#rXogju2do7k# zEH3F_6)@J`(ItUpLG5Ew;j_?5v`Q$Pkj6krebF&T$0pEn;wJ^Xbb5mVj7*FK4*b6= z7cN~98$B&R)2NwE5@7S*9Ph9slW_qz$4;J7pn(C}EcVAF(=Yo628YzQFt}u*CZN!Q zXj7Ddpn>-!*g7qswp0AaYO^aJVbC_3Oezs_yl;s;9zWTN>6uy0bjhv^@6Yk{N{nG~ zz@$c>wyds7tnrHd8C6`O?aqqNzm~#>_AoR&T7S6%A$A)WYbXQ3BmMZ|<*N#eQHKlS zJHb#0Po+q>e)G2Sn;(yz5dUsdaa0SUe-cm;s2W6fE|*6V?*%FPZVL!cU}|bcCBnO+ z-}4u*h%f0?aY&!)7hkhUaiIkf$WnjjFJ4wan~D02$Z0aKh}=fy^K+s<CS=YRq@Dif zAO3{4yZ05a(1O_MFpE6QeT?lLU60jo`p}Z-%VxLZuZIrfg6NUHLi^=%TncT1##;KU z0R)2#jf_<vQ5T(r6>20JRokd2R;LWo3`QJeDvAcD1!5&p#%nij3V`t`G|g)B(6>ia za%Sa!SDczbhQXUb<%%G1Ch@jA_f=)6MQWyu40uHRy}kXoAwlGS{(1-tg7BGORwO|) zf%v7G>g?)PK!+7GlQokK+oM_#<z!MbNU~+6dE;gqT8^Jo37f4<It#lfdU29!CHnGJ z1!ZeBM~5=<FO@i+Oc1Q<zWMgBsyO`ucv)EyOb(BXsS29i5tUFTr{CH_JP4wU47{UK z5feBvxc$&_ToTB0<vVPB5^z^RBkh8=Nid`pfFN-^**&E0m{`7+pvUCwaJtlM3$|q` z7j1&_5|me-X+(8?v>+xv+8;ZH>_U8Z<fuv>j75w|d<8(T+ar}d6Y`A2A0}w}8#@zP z5kc2u0n;2$#z@AZE7xu)091X|M-a}+n*L?6SSwvT(Ysa!0_C9J5Zpc#|84#GoD^GU z5R*8`LWO!e^xgNWqGzX>@rPYocFs(umHNva7%`Vn27>-?zB{6RkB?8PZk|*8x#ieN z+!3ETcj1zXOVl}K<o`_4idby1Gs0q;@u&67S*2UbNWWoBWHCb-ImR}Ea*6XSwrlCL z1`s4xpftP(kJ?pRhR#8yF?q11$@bQ<6Q>l?X0pC={f2V%+V{N&O>|ZQ2nLUt*?ARY zST)bi&sSb)+<kyktv{;@n!tgT!i`&Z1aZgeFLhu<^h{aU79_~}{>NkL_r%nc0%2`; z@2P;pN{jo6{^{Io$!b;Ilh}4+fMd{ROPJ`N9S2tRoXpE)OaMWUNPD<^^}2Elt%4wl ziHy!l840#nt?<<p10Y+ktWrh<KrG6Bb^*?uy`XH8j;$3jfic-~Nx7&ezII{t$AG6m zh1kHD1V+ln#C=VwLE5iYL?4|Lz(P=HkbuXI7b{TemBH_|fZ8KRTa;f@W+oo$nn6%2 zU9!``#7ls{Aj~c(bwyd2w7HL6N5&<_D8>L5J$xO+1f&(QLMMQrFSFysj?Kl(*OaZ& zpUIbG{pPJU1svG9V-n<XS`h&>{f4qsLD?xud~PYQ2wW*Ufh0T2!*U!eXLeEu(%GfZ ziWp~@c$rh!A)_8yxicyAHS6nvLG{Xr$FMlOeC4|8DD#=LB6dgE*<=@+okPmPB9O6@ zGO>_lXM%puZKnmS@i%i&Eq(R?g4G^dHgp7b99W64`@l(ltmNs`tXLR~91f=f7c^8Y z6N=aCSCb-HQE|eJ-R@LX9;-ddLb)jy_XmSv`E66rq2;6-R&;#b!pVGO&L`kDb|BcT zVbWtFCa9?(ZI}9UUIuC{b;<j4o(6SQov%@y@I^Vfr1LPK@ck{vUdxGKB+AD7Q#X`} zfreESW#gnU?q_AfDuLt7RyVs*TD9Ft99zmr`IL-O$<ZDO#yL5bdSm;V?{H}s9B=BA z6{}W6ebA42fBG*g`Cu@#HQ_6*s!s-O#sKP*ZFr0L5s%YSm$X0Xj;{*X`Jjv(PfnJk zj;pbWzC?egjBM9)K7v+6pXU83BY)G6JmS-|QOeFSXAEL;q1;?D8FHeXR>UqL^+ms^ z`UrK$iP0QWHPKV_N1x&Nb3fbi9yvzK@um(b7v-(`Fm06bFxF6~T&I4iYmT?W;nXw^ zQ8vc#%0%s~ir-2`u}jLw@nvpgXNUezJrb1HPWC2!_5h-qytrf~%w?5H)omminU;oD zrTPS0D>I4zW1}h;H`U6l71ipjx?aEhwdnF)a@I1bR<Byq`(&)&Mj305*Mi=r$6BUL zoM>E??bk5g^)ixcWvt3s`&?QO@3yNh>ty_;bXmV%FXx*$^vnEzh^j2L>$S`JZNEgU zB-P7PFL!l2>DMscwK7(@wsZV089#df(WE9dsYy+0Qj<PCH33AEn$)BwHK|EW+BY=; zM3b7-q$V|~Nln@}{SE(XUz3~M<R&+{$xUwZe)#_Y$BO#J%WG1f00000NkvXXu0mjf DkIw9D literal 0 HcmV?d00001 diff --git a/website/www/site/static/images/blog/apache-beam-flink-and-kubernetes/stream-service-changes.png b/website/www/site/static/images/blog/apache-beam-flink-and-kubernetes/stream-service-changes.png new file mode 100644 index 0000000000000000000000000000000000000000..5431792d56be64acad8feaa156dd56ba280a8a03 GIT binary patch literal 1187027 zcmeEuWmJ{h+BP6ah@c24jnXa6qPx2r1s5GsQqm{_(%ndRFFHgi7u_97w@UXn*LlxA z<NSEPckeUC{&{}saNt_cGv__;`>IKZs<O;|EMhDqB&7RtvXbgZNVt?pNSKNkcfr44 zF{6G4e;_-n%ZMYD50Y*nAyFd9Ns4K@8}Fv0H{eU1)b}BMAfgIICZGx}55;uW?tk(! zKKIE(cDnw6Ec(0HSvGg^-wa@F2Vh_%#S^TQV~!I{wLCd%_8F>ZPjB}2D&|)jQXEP@ z6&&)Ohc=azES8s-7YnyL6JA`OF8T64L8U}O#t=n9dEkfi_x#nYUmPQ93;l2Vy#IMP zQGX<qR~IOM&m;cpeTPMnkU7$k|Gt<1$0u7-`XMC`{>{sZ!Sz7YPtoG<+ND31fEa^P zG@tYD+<(L;-$#8wIY#t1SC;=>6i`^$e|KTsK}PL$x%<a@{7uiCg(z44JO3@$zp1JG zu|5AS*Z-F5f4l2{yX$|u>;DM`Qn3ArIL)72e^VCNtEC-dB}VzrF<v1x7>x0`Mm<w0 zBED@z5T{&^gVxjjx;cBB2*Ka7HJjOClk5*uaF8*^Igue>B1w1$nd_!a^JOB*^NpI{ z9DfX~llUjBRFy_4?Sf+@W<N#pH^h6wW1G~t-?T%#ee(GZZjxQy3~i!op89`2*Zl<_ ziTb5&G>-*s^SJ+f;e0s2%Uf$1O!nrv+k)%kXREX>kyco!ln)|5w|E@(Cm7hzY%Z(i zNGB_&^LH`%<@{^Zi=u$X!~6V06+<*j?t!lJ05=Itf+*_p^k9j4R+{QxKlj&XSv<8L zQe#Z*&Y3=Vnk0{3U5`l~Mo~68Zw$IBzRL&8axDAe;%Lj5(7nB^^<>t3G*|BULm<z; z*2^5RUf4-Lz<SL;1n<~vApY(@evB+>OiI!65fUAn3E>|fsQNz=aT05J^+r9FzKswA zJ5xx9!f7$c6-?pdo<qx4)rn_*vO9Ae<X<QL$F@BMkHLtV(gzS-<5NE*2ycBr=zk0q zMZ>`^8LseFY8T)3h}?`QCUfMexuRn~c@ZIWGOcfgavTH}HVV@}(ZpxQWotCAuryNS zhA;Lv1M0tU`59&<VU7s+>SJjnWT)1JU+{VzF~Xj35(`2DTlTRW8QigQqh`2H0Mj4O z9tH77r+7PjnJ`mlXE4_6=1{X+-DmD|9}Dqie+tc2Z9t2ink6e4xgdA$XE*=7`Yqhr z_hPFc)#v(R!ZNIo2YiUX3#zf#9=G$b7&<}v)y?HW>iR%x0aIMfzdqkD5XpwU=`Jdz zssrNl=Ey2+y*36?E$Zf-`lKI|aa+f2jTdP^nZ8W?@n3r5K06NySv0vFEUK>m?qhSv z@Z4so`3+~X@Fc?u{0|=;Z0DF_8ef&#UZlW2pY!*3k0<}_(S5I7otOR53DZk`9VUvk zn#3XZ{<YqG7+}3^{;}T6AE+Xp!&MC`$CL#Ijt-XE7oQI=ay(2D_HKgw?xiA$Z#$W_ zNtKJGjo%8zzxU5JBT9o|=zD#<K5J89=fZ#)!`8f8EpM2C_{2Xy=DjSS@iew_E0vgD z$X0BA_lcF*jCv}OqAg){AF=EhDCO9_WBE#)C1rKQSAVRWpJ+a+ZWuH)_6ztCVoJ(i zzg(w`HbSE6XUs_sL7y2HEh{oDdyoWi&WBw6{7Rzb7DAKoyvAavJf2on<qv$xjB4NV zOB!^5DXxpP<v?7XukY^%KK=_8gXG^t3(kDGgYOzKMZa(k(?R``kNMfp@9`z5Isu3R zzk|YK0oB^sD#+y7sc84Xv6dyE9CpFHIvP_k{}P00zCYh+zLD<6vo}}o&=j5rMJ&{f zf*<UO5#XnVIw)wM@ar<)BnE%(V{5l<a@kVXJnNzGHO#oD&iKaz5<Ja1i0{by)_l;` z+_(}U!06=7`j=WiWd$hEjpISCDo-+(VKHevk?Zc@OWHTFRyd%6hJ6NQs#I^k5E|jV zTh*f&j6-G`-y!zEPt*xT<mzYO>2B+<Zj$?SOeNo529qVUJRd~RR8<MEArr#`*~S0Z z^C&Nr*`@JjZp(xA+cvAk7LWZM6||@SJc^Z%8jdFo2C~EgiLV<bv<;H0%z6T_|3b0& zi5el=%a=Wq$`}3~KReI|;utjg9onzWdFE_65uKr?6Hg&d)p|@22HNksgYTt5PWK*c z%GkW~!$YN-j#@16Eq;NYJ3}5R4P^+0`H+eDw9LHzdjI=fWQ@3w&s~pAY{9T9&mOfH zNP#98dv0+U#hW6YTQd{MrfPfXj`jb%gHrHku&?$wf<HfJ-`!d=fLTwt5X&q4vp!v= zU##{<uZJ>~TaH`rP1!7w|FzRR1oow%<x=G4EC&Lbc&H?d#G$3i<LrB|QjPh5dcr)H zibkCo6y?|4r(kmH_c&y;TqKGmh%e>CP)vH>akMp7usQWDmM*o<Ztlx{ln3;W{K?)- zcf9@mjZP|$N|g1{>(F;ds7?}oFkO3lGy;D_kMXA!{Tz?%F<Wb84x7-nL1D?o{GVg% ze?k3v&aOV${|-8j^%Y}Dsi}E?l3J#yCHEf{_<tR5|64pT%c_|Soxahm4+<RAW?cUP z0HEKL4!Ke{H<HXE*ufm|7_)sH4%;rs^x>fJK8j9k^$O>nEUcQ<Fv1uo^p~OV9zO!d zq;Qn2?JQf*V=dmMgq>S^?)I|%_OQF@9ie_N$qojgx*2a}=eB+Ai1hl>?Twon!#iO$ z(_d*g{?AbQD>m^xo_^LZed`v`D0M!q&VhLA7Q8>YLs;=CIF_l$Qzjx66ZysuWEGcN zPZv42r<lrWV-k;>rNRh9`j$}<T>~b$+)d)ZHD&0ml*71_I<q@dV^c&lPVkp%cO9|2 zl1-<JUa6p6%z2*lMpF%Bc<m=wujYHquBSTK-yB!?a$C=b&}CQ-3Z2g9&ny#LPwdq% zEy`l}tS4LOcSUoLQx{|T8$Nm>@TPRcnUJfxGU2qY_2-v+>(geDb`P@lYsZw0DWjzb z^>SMt=N}n9I0M388BsT4nGd&1D%P_jG_$Q9zVMl--M1*Ke-G33*r#+3`~5mk6Y+i? zbkAM4HOI;f>a(qdPo~ZCC4zB0Duj;%MVV0|1%H2l9FJ>|L@a;<Gtl6;(jTM1O0@G9 z2hs6V!%#szTH7~#f1h3Kd-baaHch&X{a1h}B_~jNG_tNv<{Sp{Q|-3}W3hX?EN%7B zRYa~`#wtwO@!kA1s-OK_a9h$#r|OBMa9GuS4<JqVy^O5gH+oRIHn5@t?;acL2Au@r z3R6-!5D6G=45kk(-JH}*>oi>b?i*n5A=T(ES+CecyGQtHa!B~<rv&HQv)7ysi-I-Z zXj78_o=DNsG2-?O=jncwVZ(}78_v5pE%>YRQDZ4*e6bOzA?wq{wNLBB6>T>cHeeo+ z)PO~|P^XzRO@6`57}O}`(eRJX>~D>Cww7qpk1KNZ!sq%}XRKI@X{;LBm%R;!^T-h> zn9Zdbg_8C3m03N!BmuYb8T;nF&``m>+A+o3o9mhud?*60^%D4(lsCZvbYPKk757Uk znUjS6Yxp5f?f<^A{)|93Ohai3Vx&uVi8)jx9Sj{QZCp>0x}`^6bnGEZ{DJdYUK^F# z{_HEojjCo^br8;z=aeOVs>;%+_HF0u(kSvF<VV5r%FCcaIT3xxY9#v|qJIZ2%VaMv zyqOzl7Cqfx7?65w^or2PWh91`&uLZGxr;ytKi;28Lr#}UWzqX|Vc_zheQC1+4CN`o z%iWsjjmZn-lzPGQ)mU?Ygfp^yz%=IoxIIzNG=xT-cdrATg3BV^ReR(de$muJ?v=}0 zZ%B7@n0b3NOIDliwm35nK6I^#d2Nybd0ofVbLA1wCHRWdb^H`J{B=T+>!Vo6*>d2( zoI|@Ez*`DA>{>qx)Wc!AMW}llQ-|}}f|3k~Ln?ZXDXV+F-+Ykbw&>CG_Gm<AeN07! z+w;~s{pcHACcf4SFf4e`NniUxR-+^ozkBn){DL}GppuzwMyHZtI}70Djr40YjJRaK zow=Sr&MAjUIsorrMnrywCK{RVH>^ZR6X^P!uMLz`wCT6XJ3<j>>;Gz&|JY3k&^jPp zN~*cH?N@7Tg8;6lS?Sl?r)Puq5Xuo;UNguq6;8tUI%{jP%wWvcCA=0K=r#@m+bStF zyKflD?Ie7z+vC|AQrW>-5{ft08B4?rB_oAuIsFp2LaUhG0kn?AlHW3YZ#=%L0Hkvm zB$hV~%G|OX6o|BxEC#>V$*y|n+B^md0B~XF0KTia#8p3IgN>^zC1hUISL37E^(V2L zgPEW-zrW;ryMRr`KVC@Dtmxf{@96wPzKgGeLh-Zj8Gr@aM}pn-5fYRS<o(VG$U3D( zZJwb8HXshFfkW#W)ciCxEk#G)Do@%uhAM*Z3pNo)E_GIEm;9H<o=&ZUEO42)7{rRc zQV~6v+sN=5q__E%D;F#H$&q;M!hCnSD&PC^WPL=2f*btF!nh9RHWyPxi&M%E2s%pR zv*(^imL<by0ryxdLl2y_JQy`<P}lpDSD;xYX5rjU^#z(WszvUmD8w6}6xyUWid2-7 zIcOa9(o_{ikuQ(Rqz>~4v~`gs|8&bhMcqeGwx8zT{F_0ql)_oLF^!YTf%YX}D%dv# z4`qrh*@BiRb-!P{cgm$mD<4|^G3vuprmrI3wBlH<06w&LftNS$Tkp3V4pMi$E@fXA zxjm2c09d^rv=MUtgt-@cjT;%#L?0E{Lm~rz3}|WVb%9;Ti8F=p2KXsZA5FF2Ow`&; zmCOCskgiC7Gy83p2YRh_vP7K^8TLcIBlz7XsVGumI+e}~)ONoZI+>;>4F})a7Atpp z)AM#~{!@;U1bWf<>aH1-mqA7n&cAt=-qt*oa7wwTBvg=KS%e(()qjwA{|^SO8R<94 zJe!yf_<$NEZ%AYf2WFIJMu2*T$9l3<*I){FA9L%`sGd=T6QwFQO8vr{RJG;dtTo3n z6K+{UK28)oatbgTq>)jeSp7%i8Y&NmpK^Y|j|=)tWRdo=JDg-{6F=aX_Nk@F2?7ml zWcvE#d|t$QQw17NyfGimUpYktPr)gAY7VXWfH(0L%ppt#yV+ph(*?H-O94wN>5tFB zGU$x7--2<ouHt(Ui$GUGJ&zz~$6I4Pq~1p(N+g;D`URFNbG9|%$8fz<vk(gaVtt16 zG$JJN0@7V4^*$V%&7Ye1oOhMhFM3MJyxE(bz$WIRm&PPjr~LUnn8Fh!OTDu#?{4z2 zP%{9!Bn$XZ1D}%_i!xsMndJB_eZ5D}bbeQkL}Sk)J=Y1X#wtvV-O8^x>#-`r4E$zS zXTK&Xi7-P4_demxEcsmSO<o8c_H#};c1<LK!#k7axm|?d5Yc-J0QvU+{Qi|HvSbx` zNaVJ~mE3U#nwY%5(5zF<Xjp=Sq3PtKh0bTw6aru72BHbuj-!6;p>t0J*;s~sOnH~O zNg<9(X-}gR*`j$07}l$4pDi-I*Z0V_4*(kCWoy53`XOp2m!kWC+r<6*JE;P1fVuPa zYRu(A`v$7?53JbUAczd;{zsFtW%@9<(si-g!#$N-z>jIr0vN^(zMO$_9$GZ6KXJlW zeGMcfo-pQWS?&}<4@7UAop%{ilImf=$li_+qf|ht^Bx^c7f4NYXoK|iscJu>p+vq^ zS73oA=$R)g>bsB1(IwMmxM!>B8Q8(%i3Hx*(ymR}Jbgr`%q8S?VmtoUn<|aRj(7=u z2c|J>|7ShdKZOcZ%6Aq4FqH;YB1QHE6)2QUypJb>@_6>X-U29=JixouQW)*jxdcXM z*o`+(Q?$R2B1WGln1Qt~wHjQVhNF^3%;9N`!{Ya#>Qo^Sb|`GnSS{^vfJgXZcep0R z<5*OPCIj-Bo4`~N)v$ZBz88~Y(e86C&ssnJk&Q=flj7zl`p`peOQ)L!vq+&((!zLl zy&2K%g{OSq1~e<tLQn!}_kS#nD<;zb!R=nbp*P;s*0j8M2EL?s3th~S2R0B~c*t{0 zth<#%!k1jIITf#B#Vqy15=P$GKA8uqNYVBSO+;ny<g36G7zZ7j%k`n$lxY#0W-#64 z+<Z2U1H00DzzL*9O^0V{EWBpIVW-8PP4n6;VtE|laSX<)`Ap>v*_-2~cg|_b3>bQr zz$lj9X$Y_iFI5~mt}{6W%{R$wzp<w;;~IeDB$@=>FNOf7uLDM{9PS3N!=k>Xn9_BA zX(Sd;{n?MO0@h|jI)FPquRN&DZ}L@Vjff+O^kZ_*==Sio`AYc{5G!IdxUOwgSBogc zI|uEX)|;1K-^X)lKc-ipW~wS(!gm)#eo}daT3hgT-m!;(;5mRGISxU8#odnb1z_{h zd=8jILyrR9&AcbEw~SMIcuC>!3*d&d+Lyrmeq00lMre-j8%XbC9AZxMk27!P>^f2( zB12?mC6^hEmD8PS`@Xzc@^(7}Wo{704-Ti&Qk!>8gJLX;gtD*G&ApT~I~vKZY|LKA zI^Wxy5%g|8@j(Z+mY)F|M{(V&<4haI?F@kkoTQ^^sC(BS-T75SjPV~?fGHz4XiPYr zl7RIdDcYAxNWqTrE*K85$+X^!o9hE#H*;Ej(8C76JbczSj0}#KZ~u%%CYiukaA%9t zg~H|A&SK5bK<P;GJ*Wt7O7~q%7(2!*IYdb??<nfc;6<G^C?v8bLtl(vsPFW1CA?xu z$iMOwE?8VAJbqhYB>=#@5egpZj?o$56k3L_No^ZZdlBeg?FC1>)ve;CCCnN$(w{+f zJ|(<rQ339I?YB2lmx+0=+JM%AJD?^*2EG5DcG4ov@jZetI8WJW(j6aWDP;)G)Y{GQ zxE<$-!<3c*d8i5%NZs}^vRf*HPu%<ih~Zk@{zkOHrph*^Hh|p2Z;WPKbii?sVJu=d zxV-TJ2=BTSIaZ-bH9fu5WBykUg~_-zT1)NT)Fcsk${&){|ACNG5Gbp|&Iz(~7N$y= zi%#xNDORL(I}8p_KO?aZV9H9DCwnBw?mK1m{cV@~J3-Ha+bl9Xx@)b*^0xv@MDML( zY~U>xQIv>0-UX5j7aa9BnAF6WL`S!YhE8$^6=3);E)|!Q>r;QwdY?kS!9^PS#o+`D z-d8|Dgh`{9Rdo|fM^xx?A)^WKT_26*;3~eBWFRtTYgi6=R`m<~5Onn?Nv*K=h5Jmr zVLmheMw+v7K+4p?C)Q}y;qN+n+((NS0Q$95?SoFq&Z1xYT!94^mzXT;+sE2mst15H zM<1R^L^DtlE0QE|Af5WPCG9gfIFC9mBX!<0qs2{wsnR!+7Sq?Vm7m%_p<`mC8PTm4 zUho99np!pFTr1%?uhbUPQpkDwWiXUhSbGn@Uxw)SsBR2QH|6)vd75Xc!T4co{vLoo zt{C)wag&Qc#C10(P!+9G;+g5R1uWb=@IKFTn=)8}p66Cxt4P^-oSvFk618;vvRwB3 zm&HEdRji^RHv>6w07111m9cONC>>$eJjhNlu4ZL*b0!XR#^F9oU_cDyL<*-gga8GE zLx$8*ax+c!S{e{}LOp87^EJUwUZ?x(03k6obppc2djRR6jG{p{$dZotozT+pK=CPU z!??O!x;<H}0i4F`$9t2vzq;~uDvU#h4l`d|;^b7l1$~_+-r#^v1n@oSCH8<f0DYy9 z*f)Gy{59<JXJZ67BB(&ZWOs<Iu802;fNu{#m_mbn62E3}xZYJ7>5G3i;&8Yz6xqL@ z=S=Ro`BVkzeY`X!vqXLx2MzTlz@Pm9g6{NBJ}mWrmcZoHxMUka(s8(|%1EbL%I^OU zl_MxZY-bi$Ld0eHpy3$?{jgZ2$XyjRX)y?$>*jC=z<z`M$(%iDZ)UBevK?+cu-^at z5ys|;a+UjLQqLk8DC3nQR(PStfNx1jF_fqeS3%r|Kl|T7XnZl+Ir|ZbfsJX?w)t-L ziU$|IS1S}%)0WBz<!6oO4lKZx)N8kDvaZuLjYRmZdv0KQ23ok3)04iMMviUJmIREE z?qj3q;T4WwG4}CmX!D>_3m?okIu{O0uH>&1!Q0B+Vo^+b8-2zL)jq|$1F#hyAVdwL zwyDa6PF?;w=BY0{y;Qmc-Ul2n3PX2i^~Li#E|}`FP}$O(c}JO=^%NUgR`Cug>i3I) zD7PGEPKp4~Ws~DHY2;>fxLY%%-v^4hzE$+uY-SXoKi>>Ny%|4fZ%L|FG<NJF2rtgL z*4bN_3}l!#1-Pe7Nh$VRJ==6|uB!?_%-PcRn+r#rvT1e%=kr(z=M61C1wBQl6vwd8 z2cx0$7VCpsKYPzOE{48!zH<6D8t>KB;gH72u$Sl2=19>XOIq%6O90DR8PM>JMe&)1 zkkNi`0!>#9`K^k|P#otUlIQ<F$)jooCX(3$ox3`a`v@T?T83#Z9@T9676U;~rS-%E zc%0W2$C_MsvVBDkS`MF){RA7S;|$c}fJRglr*+NMUmD7gydd@60=V1&+k?ax*(U8i zpQfUptbo3BQXOV`OGOz>;!C^zg$?d$6Gn{NeETq}vXYY7)iHV{ta^^uVX<Z8Dg4cN zj!fh*wsc2-qvOm@i{nZ+H~-aUP9&?-$pkV$ZW+;jOEhUL_%YgX`1OM@pR^qzP3z`W zNpd<qxuPjK=QhcGyyiK@mA0MwFTZdFfBHcH^QY>uLqq+RT2os#N|*W8No}30)y;e; zBmFMhhUjOxte8V4dx<uTO(Pb6d0omdpm~iwVkxWqf+ZE7h;a@~JVze+L$UAyIhHMb zW~=jK&$mK@EZfJ%V*uS{@uO1DE_fZZY8V+JA5&A#0`A>EY2aYTbucta9eU5)Tr=uH zSzjQJ+rD89&=l~?w&KrEiV89~5J<J!x~Q)SFw1^GkwOP`UXX>9r#3M(Bn33jC|)Lm z((trJ#nLV~xZ|kkt?}%VkVU@}(qh^}A5|DIeO8u%x<znj;GH1KtXFlJ^HKO_&?99( zb+QM;!f@~u3ExhIp$;aX`>AV;30I(KUMMZ$Qh)3oeKdSig;Is4v=q*b-tqs_aQ;o% z@xKCcH|Yv_qj1EmvJlV}`T?wbk~4k*>0C>@28tojsP*zOf;igbJIU7lDq`dOsg+Jf zE}lB{1MF@J1Raut0|2LZ;F?d>@~7o?4Olg10Gcz+`EmO@vE3a-0UGW&1H{f9Xl}=i zjOeK~0Cz0n9JbTXodMDhgo`Z$4tA3EHV`NF=IcS*rG1W~ENg63A;1sP5kou*J8!KX zQ6ebaIye>e#xjG2o)xGyY-hOTZZh<(ym4hlD{0*V5DzBj=KY0Iy&CbJbanD)(AHT% zGvpbvZ8svGu>S)rPX3P9G9U^!%*R$t845}!&0BlXzUa9VUv^)QkQx0oB;ZnMW(a71 znQKt`5@n24D<3VoUYqRX>&sK#l!GS4!8-nKD3)j5=`}!A8-m?DJrM!1E-Zvv#;sR_ z^V!DxTzj_Ls6fAdd4F?ad~N}Mw!q7+^Px<qeJE-xqUQ~O?qbd}{o+myK=NI}yj%Y8 zfWm9kcDYI`ui&{(_`5@wK_j<?_<Q~pOh^-}-(r>K>g-~l6*sOBB<)AbkXKC_T#nUw zL*_bWzmCmvGJY`pJ`e0?52C6=a}v8+H8m1LZ$Ssw0%4ep=$AoYco1qC`mbiAtV|f& z6gfuub@spm%6>h+r-2lMpg*o^SjKqRjQKu=Hy`roer|}|&Jg#TjVbYR!y`T-1j7G+ zL-`+R@SG07oFC-gpf{Q~@70Cv@lgk3Vs=!5qmQ9D(-x2~`=g^Y=f>;?z{iwmDj82- zo>)uSA5Tk@{De1|oaGFI>zRq<;{Z2ONl9QaAS5U$t#-1Gvs6j;4y5hrH;aJK_HB>9 z)x*nVykkGyIg&usxK5I$c4w8cSTTS#mXDKBpn5R<`O5%cNkrD9;R>I9=CYY&Rx7hh z){R%9V^(NLtIF^`(Q^jxL<D_;n<821S-w3$4M)h%Wh-(tx~u0x_z7~HrHhU=+!Okf zb_RC~zy@%~?lQ)2+89z%p#J7>hoh%oly?E;#}o&$`hYH|L9Dp#Vw<U^i_n0tFWT%z z+rTa!<IOBgH#>~$Q(&?I*QZoDjqMD-&Jw&TxUR^QvD+Dt{s({-K%wW@>WeFMr0U@N z3drXng2G{UsN=9g!(r`N-tV)lM8qz!SOvCnWBVZ+X|Yl;c}58$s%EzXV@+wz<+>Qw zl$=-^<4g`{C`{UM>YBpm>{FtLz@Tv{>JXr&TzxF{h7t5vx5e}ref&G68k!YfaYg10 z0oiOUVIwH^MkyCnCZ4)Zj`qFW<IOpc0^&kmGOt6%$gErkZGyfv1KU_@n-sln8Oj>X zLP?o-C~O90&y9(4V{;^j)9y21ul77MuVOW!vnROm=`$x0imv|rLR<foRTE&x!1_GN z_#VLESCw)2<*@BV2g(6t>ac9%ZB-ZH;i85DTdPa@Xzt9Pmeqqo8~}D6&JTfBElKhD z{D~3vI8Bx&&m!Lmc%vs!GtKT-(4W>oG~USKe!VP;P|153vdbf)MXN?}Ug8DxHCliv zd%`h3&0%oq5Rf(#CnaTiKjfJU#oCixW)9#yl$Tr*OiVvEcJCl{9aobI7P`GUGzT8- zM}DE-rzXDF6JF)`GbUolM|1_aP&Y%+Wm^<^;dsoSjLVWpHTmD{ZkCp3;6jwU*+bUj zg}y<7JW$JT{yv^1m<Dosp+tn6!!bx0F_pw`H(UGuGMBvyXlGSvH7m7^+7Tn1uffpO zsco_;t$II2fxsASCD9qtEh*K`XJp*h$8W(PY;m(1j$_x?YtJFgw_!JEwQCu_v<4W` zqxIc5lLe~vOP_eoSA*NVdWMG)?YGwl(*A;6C$g1QfX;XwXogoUG`aTWQGT7IhbS@W zeq<5-5HWQr?i|^W{Ny^IaryWzF1anp-1JJ;H;p3-SL=LP{gTi*jB!bn71tjd6Ptvm z-w;56`!si@6p4&Ov*EIoily0oi@vwlT6*XTqB%|HO5&EjnM_x2z_KL)3z0o8=7g8~ zv?$*y@=Xt!8*{OE>#j3+lf}ifadV9VWp1Fwcy8`R%WeaU?jv?Y9l)ht9lx~=9lNX- z_lIqoQGU={W&~`!e9)UBs(aT7?XL05cLX21>8CuIbTbi(WS?alBDY|q1@BAxmy0^A zPOvPoQo^xT-n3n8{5l!HlrGua?P`QbIS-N#Mb^#d>#2|9(^n0|i^C>6<jlH1I^H!L zA&+Wf0y@xC3lPtuFGtY~QJ)kCMo2vRQ!wQu1p*_zd#^EXzwBoTr<`t2FzN6%&c;_c zYD(IKkXG4Z0_^|T95B1Vx?xc(r}_8>=b8Z?PB0?dqBVxH!&vnvZ6=++<25ic$q!S{ z%cU-C0CawcTB#!VOVW~-{Fa*r=U}1PO=&*#`S&%1{*SNwEaV;ZOXKy)+yJ@PZgei6 z%JMnLa`5ugg=f)^NTn`!)P2O)<I(UZh=Zy}Sb$j&O?Orw@}{Q6JOL^<T)s>u7ORn1 zZH2Bx)V(_Vc>QyM^XSnz^IUv>>G|OX$-+=p9ue-Sp~%q2rgzqPQd&PitTs7WY*@du zUr;(tPGqKe?<^#l#82MwxxIGFDXk`sjT5>C96E$q)$|owiCRKx%N&q?UIE6%Wy5_; z9?e(makyTU>If24Bm?(s33+3yZ|Cz{2>Y5G*y}$mD23r%0bJeUhms<*nV!w8g{Nax zR_GS8$OQW-SU~qV*@Nl<_Bxm@4sP^5-paz~uW}UMi^V=r59qa?{+EtBiWT9CaD}vF zIx_JKsTq4?*JXWy>r^~KdXNIVgH~>XS1FA$6U1s?-q{7zh&y)lf9&Y9RLb)vM?1R5 ziI&rRG^${Rp2zSBLX%!B8Vv*{3pFAgV~@O%n<sbunK3PGHKSz99@o@*5;a%9(%!1h z4*0lv`3Z%G^z~<CDJ_lXT4*;Ib6*vilwqy7_)+$^*QXU+*|g3=XjBMcfvRg=j-<48 z@z+WZolyA7pOzMIjN^cREAUyaH2Y&F!bBN)7}{O>G54|IYq1drxoX|~77K2-lptlX z@7dL|z)#4two^05g4Ue@oF>@A7GkHZZ$*PhE~OBT<v$z<UO>g1h`ZLljYm?UIDSZ? zQ#UueUzDc4mX^}#bM;H+5Xj@n05gxuj;D3YKNKoxG){4*_L&A^R9OMzSpSEIRCxF5 z4;of*a!N&b6k_l9Cg{<rL@0XR%8}Ck`al{)O+1ps(o@lr#NooU<Ec&iDV5ar-pss! z%Vt!`NxoHq(q_3+T!@BV!&Sc9C(?^-Nq;yYGrLULOd7;lu|PRJwAgzPvXK<4%+HWG zz;NnZNQn<jGYd7IHzXfHqGGPEfzd(%np)%<JFKs#wOUvVQA*{p>$ff`Gf&~RS+~{K zNHdXZC~}Rsp-vfoQqgZud^mQO4-+Mg+a}@huB0>JTnb|`+r*b9`I$(!IXhmq9Oq%` z7R}4!2szUqbwK#mQ&Bsi8-T{>l24h+Z953mk$RT^#x?rU!|@=pI~p9!xsPN5JPDSP zM7ua(a_9K8ti~vGw_^8*Q4)YDRaCk5o4QrO&iM@*c`RQevH8D2RqYM%3Y;lZL72N2 zCwn%7f`|RqTs_VCC=adL1uXUHq@l0N_47EJDi<y6=IUSxgfQQ0AW(T^&%cOjeDM;m zP~mN!&}PdVNL;iYn+#jCKhay%o07P$H6A_7hHL?)!RAOw+Mj6AV}mRQ;yo^`sV^<$ zdGr(YR#zCa5H|?zXq@;0_*cg#)`lmi?Y9e;K0DR5-k)hjZ^kljR?9Bgd6CfoU8v4* zy=LHx&u|{_J|e3l&A)~t!dE$#@V}C5V@4Eg)4OWzkED>%i;`^TzHjphQZ;edy?9=& zK!`Q&MunL7T6vejCVzLjuo%=a8=z|&tYR1Jz#qzsV~146edoD{{Dmq~&{SFIcwG88 zQ6n`CgURXwqr7(mD5w=$)T)JHq&;z6ibSc_1J(Gw*ra^JE<2Mfrx-pr=N5;6jU@vj zGTNWYJLK%53!1NxSmXYu8zz}A^7jg8<~6;kpD_sz;T==MiZluc7to<fj+mx_YkeIO z*T&*C>Pl&ctb`2+jR@Eg4tB)><h>qTMc-a_uf3v1A4OnLwP<)70lukR%hM`fn4)rx z)2_ORBLpoZy&Dx%H#W2=rj}{2CN@u2R?j#1K8ZP@vHN%GiT>-VI-h)b^vlw;vdg0_ zvw?(Nhup<5z$J#fo9zhR0)SFVr1ASCqh#Fqte<lz3Fz`X6RmOdKLE|cAlRX0;2S*5 z$Q7Kba&jUE0x?~jGqhLjHEyfP>5K(M&9rG4H)AowKm}KK*jE71(k;704|u|<<a(sh z1l1_UE+vPI8EC7bQP5UrY!7S2y&W39-({<Ls~#KMHYs1H7}CUxkT|lR9aa-6Xj=7k ztnC{pS=p;?Cp|tjW}up3sh`I?{rP?c|EDM@?X?a{RaPJ&^hwqIURzi5v+aYE#z*TN zxS^q`{bZjbWf%|;?rZ*zN6F+aIo-nE$FEt!tTU&TS$UZa>jx9;Nn?Wc3!;I0HPdDl zs5yWq_oq3*8Ke?N)6|!+o?~=Ar9{rohy(%SPKe;48NR#9Wg;kH%l4NbCOHYWD$8<f zZwr%7KH(!E?J&1mGQs*nVi;y4@{QmpS8;^Wxmzpxak#vf;>2^bdSZSTOK#Qy?mus~ zs)?X<4eT$0=BhYVZd67lt_E9EWqbG=h?`n2ip1<GsIhngxD>+kGnny?KM>hH;8`n} z&)`uPH-i9IKLQcAmZO?Cm8Wsf);nZgAU6W~8d@z?;EE(`fC;^b{m`-A>%gQDfdeka zwMRe6Gt9?5s%Cyyg@_={SQVsm6jjf*gQ;$zt_{IbJ30b-h{urWxc9`hW<HCMi0Au0 zmTWD2-(%j8i)-4KW55-P+ssg)$9EZwbjnsId?B?yRg+G;lC_iU4hHR`iB{56p;ARm znWO&Mi$Ky;F5v?1#3W87Y28$+*b66wC8y1H>_HrSq&(J4xefVk@@hCsZ1D8eHvGv7 zR4n9jw_#sybmw$uYN%JQg+WjEZovza=6?zp;;9v%NwNCX<?Hf6z$FRTm5a<Z{Tu`o zGB+1f?J35so&+$p>zgPEoT7`eidE+)JU!v}#Ko*Tm2X+PVgwJYDsuYUhzqzMhuhRa zUT7qlFbG6(9LOn>P4O(aXdI9|q}vRtCKWjvmZ++>?rCi!)#G-9*O0l*J8F-FaQu>e z6{l^PEvmy)6*(D*l@i@4Zxm+blf`bh;>K{`Pwjd_VeqX@YsWdPw7Z4xRDR|;jFzE` z-)-Nvyydq%2*{MEhkiH2&;<czdw#O7G@_FkrApL#Aqh4<+1}FaRvCF7Vx$@mWC!{z zhBEAGBS*5O?m2iH4G8fiV(1)-p)uLOc`OD=$BH!QY#zn$@jubjK(`(!6NO-%JvN`s ziRUW^c!xVsRh(nb=&q7C4>O6?09RCKLb_i@QBDMZvF%K_-TmPdt;@2V(wd0+$VJW0 zYyh%KMtp)$Fm?n2Z?Fz<FkJ{-w2)c8av&#Ozo%>YBS4sL5l$f@eBu*}x}S15qv$1I z8b3a1=sb|Ekz$JUUi~s50ELM%>_mtW+Ef5gXU1s(thvp|;@l3VzPHT(7uc2-IN5jH z_QB{m4hrRYnjrjI8RYy*0m((AAf)w{fwL&;RQbV%wSau;9B&F_Wjql0Ho_|;d*#US z445o(aUJ4=`|!7xmq-*^51~?ckK03j0rZfq%yRAq3EJX^J0*3;v3LMtpy+I!<BWf7 zq8tl<T_$`25;0t_mNnYd&F4WJqT`W!DZugO07$8@_#~LU0G?NwE5gMhfoq$;fWl%W zB3MXXz)+*rD%V`7o?7W|-2EkdbG9-7=A0eS@)`^KJldJUlH6MWy)b_N8p`Gv^4r97 zi$;Zc1<x7YAw6Vb82emFxzOjSW71gWMPk6+S2TjG<D87CD)T3ChkQB-ghtLF$C2;4 zGg-vOhMdYG+ld&!7Chw)JJBqG5_lWM4O5}~%7Zpd6;3|NW}c539$6>FAI?O(|H=x) zsTQ_ziqW2w6A}PoMkvCV5PDhD{XI)-%u(q6Dc^>y-}om9^qo(5PYKg%Y>{yS=t6Qj z^R1*@7*$qIfW|ZPoK-HVK$Lo0Vhso5(+*Q_80xrKth@Z6^+c__VDP(>Exjqu*myb8 z;$<8(F;ht9DVJ*ky`)(h(~6^JatAgqjK7@`W8Q8^6#ve3`G%I5)t5zAVyc)>)130M z^OXHh>m?rCXm1@MbvD<Zj+ON_=Inm#aIGa&6W-$w-nni99V`X7Lezs1>F=fTF11;P zSgOenzwGOzEz>w2)K9j<?-qL?Bfv2#T5m!6o0R&3cN65uiAaqp{<$DSY-$WLNz%{J zamjbfJgg_+G@{zW13<kSF-Y9dqCNuC+T)NU)rtkUDPDxvtFu(UDG>2oqE=3WI$J%B z1#x8QT5X(JpyRc)#(Q#mo4ErK`r{;$zS*{lx4zfKtiyPd)U6RPZnp{4B$^4L#FjMS zP-2x@oH^4lmVVk)+aG&1Jb7(JLP7(3$2Vj3y?psu$gN(d5zdxck!mlP!i#3_)g+^2 z(LB^(`Vib%$m(@{JoN}oz$w1KwW@OuX0LZrR(bGT@AYU#R6yV+WXZN>K=1}3Xc9{4 zv|<aPW0jYBYqlzI>F8S*<+Ys3eOEb%YZ*)}#)-oE)9i?5IJjp-aREq|SMpEl-m2uz zFQN3TmwhHTrmn^x;fnh7I?7B=%CQpe#ACtk5W?vo+m~x{_#Pl5O$+x~XH$v5cA6bW zA89amhR29jkAIX?n@Swk|3SM@qOO?y3P9X?O{utN7kog^5c*_G96PO^JVyN_mb3n5 zH7TbbDmHdq%N{c<{VcF0DPsu4q$Uul`eZG+kH><Nm^<SpG~}nL0S|hUhaH?UqWK_< zv9{_KDF8QlisQzYTzdWpq2ni##pdz&y{h?foE(ivRb+&d^<@@m!_h5`apaG<?QEh9 zmkzYYx;2L8?2GhFejGo)ohv9OI(bG{vRFyqUyy*VV$2>6iDWW<me7BzhxsrbQ<liz zP<!9P0>nIwCqDs~Z`xUw@uMpiJ{}iTA#s||QjS9_Z_Q}ykc*IE#cP;n(S7Ca!#dJ5 zYH2M|>_i5G*hT+5KEW-a!lvnRL_XVjD=#r2XajVXqjfHLCw1^G(97!z1&`IxqA^ky z8GvP=A~(ghwkj_aT%5!4;M+IO?d!N&p_It&)>1dA#>`V{*Vvkxj6HRhv$k^Bd3=UQ zYe`&L0e);?eRcB;%EMM5Llkj_Q~-}P<7-28ruE|Y+c8qj9~*+H3URDrd6IHiM;f74 z_E}X5X79RR4(K(z?j$ty*4nU7HA@oEyIyH7wSX*ri`!Q~+xF4!W4758Pt;pG+N+hb z4^7Tw_*`sF6A8z<Xb&*JSXqu6<(!lv>}F8|h<<@PAN$vDFr(t__`=SOi8<iqMDqgL z&(Y8xm1^~=`;}(tV7<r@!uNenxyNLMGXhSL;mgsCRGSFsTuI2I4`vIlQ{0FU!tqD{ z+dr}ZWtWwW)-8D)KrXN2t~ms^car)}Yul>mF>-FG%VRB-yu|Swp;eJZ77_ur1J^64 zUYZYd_F(js#tdEz9-DNtVN-9f5h1~p_OPPK*QTWAb(Ht>YGp2Y(2%ol4DLV74(PK^ zSACGdBwBRxshW5QzwkT4A(>r?ndT0+x=xVFRB_&G*a!;I9`m_3b}FxvrT@zXEgCj) z4vcu14=xjTrIn%Z8eh4gw}RLv{&Qv~4QVrPi;+|NJ#?$vq6a^pm)J<g=Qt{Nc++4{ zo)xq*hI`DeeETANwtUBdJlBT^CQ{;92Nbx)OUcXgzmVYgEsBKv+xH+KO-mWFhJSdr z5@~RtFkGditO>7s$mQ)GpCcD1_e8*Yys+@AxbKvVEGE|KIDXG+P)=!u1Gw#x_BFkS z{@x*&T8{;6z60~8<s+_7?48z|Rzup2ECf;tksdQtp-7^!mrv;sZBYUG1}79A$CY?2 zY~S}QcYfw6`hDV6?=zf~i-vtyaz(XUnwIdD&G_0jC;O<HAf}q7f8IaS(=4qnl0VaZ zj=8w@U>WPhGMlKkyz)o6{W4O@S1$)`Ug{ax8e+|4#cxRslEL<$SU96-H!Bc)qu$ea z+2adzs!8mVina@D^Fd~{V=a+r)#vaLn&PxC2doz5ErJ5?Jnz=<5#9)u1sWI%Pw)+{ ziLz7`qQ*OA$LR9f3RixAPvZHdo_Zy&tPJn_xASpxf!)eLZc9-^b0;K0#FZGFL|Jv$ zghz!-)P8kPouYi%EcKz6{(<XM<q&(r#qt$uya_K&lrLFY7e+-p%>nmwtt@6CS>;-L z=M$hxDI7AMk?SQ@pEF+C593qP*lW|Km%LEMO2}@EXL+tn=vt;p1ZONObr<V<)*zl> zC)-R2?Bj-L_Zf4%d`m+)8$8FeKs5%x50bSvjg~A&WrF=IjF5GZhEOq9jdm*iHl9e$ zI&zsi$sv{Rjl)-Qg^HDp5kgeE-(wx?r{MQ?Tumk!1f+`@!_|GUBHrC(#cLJC9c1J6 zTB)e7)SYmwys~4~@+<Xrg{HLZ)y=iEDkd@@#jneIa7>Z6pY+Fl8m1tnQhvZt_O$lI zX^(TgXs@mb5zdxHVju)s`JmPW?6!Er9(cTio>umsV8IzPTw=;c*jraGRN;J-)KA9V z>Uc5f>7`f*Y{4eXL&{)otFtSUy!S#*VD*-d)}a8!k`IB1PB9N}9EKwZY#NIkqHFx* zoT!fl@jS1?>D;9V&4?#V(~0hh*0UCjuDBKB{F3Ph+a$9(o*?V5?wvaPB<nz$heX8Z zZMKCf(V$(2^f9X}UKAu$g2so<Z3zx~iWfz~x2Cj~TDSZI7tg!L_eae=(XA<roQE2g zegv)|3}oY<!c)3<lhS+j#(;hkT)xp%UW~i{Zyw|S`d8ICz`t%!Z{9)KqH;b2wSkO@ zWy4(gM;*sb=8GtXo*-sLWpa2YaZj>hgohO;EBX%Oh+mSFRLhhHoa=B{0+*Xn{D&R% zRTu~6`%eAn6TlQ?*t|^Q7@oBTO*&!s!dD+u?DTMZt0sI+Q2L2qk4;7wOb*=@T=u&Y zla{cldprZ0R?{Xa#6cE23dRuA!okW!B@Z08R$KdW%k>LNwvA48&QmFv*tIS)a-7st zOV)=_v0fd%8wB88Wu`7b4q2_?N;I?oNo-xwP;7GE17qG%->NoDey4>CW!8Q(D44{Q zx-4+e@@Pwy?qO4!Zu~>lFW6`Uva{uGOWsrWVjjx8Wv1hY@`pHmgaG0*F#Jiv`F6Jj zsFKpI_fx9(wuC0)`nd~61lB5iuS*i87UIbyNx7tDH79^oA5+z^$j;cHiTwMa4C%|b zg+T|96l#esY<cn0EO7lP8168{{n{3f_lqh^Zx5BeW?r;aXZCHm=EZCOuHL2B4eY)} z+(+R~l@|^!ewD_Yos!7-<Q<0;0jW0kx}`9u_k|1Fqc_vKS^d;-N%C6xb|!8URl)&! z4T`BNW0Z6H*s%uEl34w`4V{|{IizPJMe7x6+P&+~*rnpE?&ny1cJPe&%2D(<BHm9g zV3W@#q0n=6%?&N1%CRpNNv?FXyld64gi?`*Ny?AeKel^u*-&z&A2_6wuN(C@4JI-S zG(G#rjK!0tRSp!6Nfu;UIUk$6b)*Wm$J9G2pO}{zl1bX~b^%$`wKetvau5^3!Du*v zASfT=mVu&mp-07{0-TgcrUf=4O+wJxgO5>U))SFDPG|QNkIV*_{Ryc#xFYZyrRE*< zrvj5`2B$?t;+D={BqyCF8NMHjO1>Wl7V{&ny^xEpGK893&qhZ0L1G7c&p)x+SX0P_ zjUb6;u+3|d*{<XIx64v(X6f!mxkqt$aJ7}ZVFGp7S^(O;k0?KK2c!)OgHOot8w%pG zBJ%|294~{r#n{z<M(}yPf9FKtuAs+%o$p8=CaQaJllv=gly(uTGZ(4FQE8}{qrC@n zoiVy(Qh0V{fUz>N{1zaFgz=#tb=-T7Fa0%+g{QoP!WlSWJ0uCHyLh}k+<Q|qZ6e(R ztYxss69VqU!fxI~`4blJrLhc8Lc)z^S;8sIgipVWmNtL%n_z8Nvw1Fm_7BP67TUNV z*MJ|ETNo~x;O`>+`BgGe82CLeHx9Xukf{d6i0Y2sZdP<Ys3!)K0>vkwcHAm<RRIK< zqch!e1k^bWXBj?7<u`xbh_08Dz|nm6DK<Ms0g1re$#e|ew|y)uO=q=op99j@y9H4; zM<JU69BGI_A7|v{;ILfUZq1-|VO;{JS*=1L?S>OOe3bgbh>Vl(U9`t@K*Jqy*&b&Y z3HI>*umpm8f={c0oZ&o7-{T~A)$*Nb)*S$c%#oN72{0ugTXqYGaek?eoRdLe#HJ8N zGQSLLlv56^-p0D3B#yo5YKOaI(j^PVkiadn2V5anhT>OWs+t@Sp{}n*a$TuT38RM( z*JbIy_j+ukZ$P}iLDa8J^vvVKsh-GYt3FJBgzvvy-y$SH>}Gplah!_Ccg-46jqNBj zI0Ndd+c$n4>f{br7o=k^Wcnb<kc|wvQxAYiVYpb2CA4(JJr!m(m`90&mw%U&HQ1<R zbm-IeGm9pmHudSTK<?WEBlXjVP6K~_gnvgS(YZjYTE5z>o;<0{_dRI%O`$Me;p15* zwKR^h!lCBQUME%;sQ>+DU78J@4o}pj$90~?2iE)Qo9$Q)+2=8+Ps-WFiR@Rows?im zU(SrXmagl1s(_%5;Eqg{^_rD(i6}o$Ny1_2J?U9kl9J(CW-L94A@Lah0PXfv`+3c? zDe<M^_4#aKvy5trW)#~kp)5)|5PQMeBg7(n14xy?k^&q{t5-iY-`5db`W!w2=hGy% z{1|3(212b~z7%hN`ai~3+4%|XF;$LSX|E)A=A_WgoKP6;%;)M6uFMx9%qf^JxHV|} zlO0VOaX+W5z<MFan;bS|j`zY^Wg`9)X9$uWb3BV`hyCaWP8eHtKSy7um};~cHT9^E z&v321s$WUK@!lM-ldqbMiNWiy)b|^K*4yd^eM#O5Ze_V2tG+I9O5HDGc@Y&rs0199 z`9xfncFHJNNY{#{qK!MI<jSl3Nf^GM7Rm^-V)i(2I!ErE(Uh##kQLmdEXmQ6(b313 z3GPu{O#MLIWjbG>xV2<a@lsDY*sJOxH(u5nAagaLX2jJWbre=M$cY$PY|@oPGjoa0 z`S`64yXoBuxLBm&z=98Bd#p9Id{A6-VVj|JI!#Ee)bTw7T&mR6gWubOm6652Gc<F; z`_<0mm&vkh90fZ`<kD6FY1d0RZ=;p^j7aKdo3*EvquvT$&t@HJy){_g=4fO|3>4v~ z$vd;5<=5yL`n&1#bR6Qp5<Q5i^~W2>3Ey&*Tqjn*e*ha7op{G%8L2*B@-T~<=_c;Z z|2nY$?LL@lGwQanAM?`$RK0RsKPT~+J37OLIIr=p8qzv6-&d>Bd-@p028M3aS=2l) zVD;`y(D=CX)r|CMK)bRNYV_A=_)(Zt;BmwXC38V&7NzcSh?9Ojq(CNw=lHwsmyqXU zu4}SS`aQE`r|=49Iz1dYd3p#ac5)6qcgi*(LR%G4Mrvx4$1fMnLFK$IrN3~z)c($= zQ|pA;NrOK$Y8YV)*KQqGSKw}nll<-^9ctsLE&cJycW4oI%6<_Ctt{W8b<x}n8E@fD zd{gJTsbd$F)mPhYvL6jh$O=l?<f<Jc<5}8@Z8XPE#C>D$C?;uWjvxO%YGw)xAYBA< zB79y`OWX(&ipml}70dGIQ&0!E#-a4;GOox1Xcahnx5On~@HN4lT~c_O9p|rlXefNF zCTBxDdoSzTQ>BBW{8yes1f+{Gbxb>ckL#Gb-tjZUX78jw);g&4yLYo8r=0GX5nZ_R z?%L;gLMIhO9t5jDSPKPPxA!%2%lQ#-s~)IgehpC0@l2GHe2}K1vu(s6V+GB*UsKsw z5RhTH_^6+-;5YT;hey5C?tAis6<1T*Sy-?MXv&T(DGPhu!0HDE3weV~2nd?AFn9Zm z8PZ4O)TD(Xn%u$kavIz3@87nN;a6m}f*&{oT;*#Bod;&_L7DhM0<Lq}r#zKxEPa1| zM2zSZ15LAjXBcc49bcD0?lUgt@x<c*Q`%OQW8*GF@(dqw;0fr}k*SRG-IBYB%Q|vS zG>xGB8A8pLnOc_$SfdZQFrDn21J(C)B@6L#E?DS~#vJg3tTNTSY+lm(XARctks?=D zD_T1!b%c+8IOM{OFVwl@^5EEeIEPYxe#hA=B3sntWxk&hR<3U7J80ZnK>bU-G8`*T zLEzX!PlpC);-;R(tWfeC{Mr%?wdh$oE!q+5<O~Z;W}}XQACn4~pJrvkJhfNentbq5 zx^;Z_85aMxE@zv_!k9?wy^qe%M$SRLobRgP3O@?7Ao?05?OQ4<7SEA#P{b^UI4toN zn3$aN6fa46p1O?glnj>pA?NK*S(TTym9#!CD}v#cEaP9?>MN`X4HurgvRh)t4}ueB zF=D#4#>Q0{uf(-MLxV5O#%su1-rH5M+uwE9SwCRz9s-wNJPsJpT*o}W(OQ;EFU4T! z4TW>))?1Brdvza|$z3YdA-RAASEK9A12EEKZXUcTy$C%iHbOS|VJvg4xAX~UX<LF) z2drqq;VGG2t>)kP_5`eqvk&_I0{dl@%i2fly?9Wz+rgueuN`7$tH)qOq>@v8SDy4C zZ^~P6r(LKZSUk=i(kn~<gCp_D^qS;4qs&Mu0k>g(WbU22*gvZm8=2+ZsS5J5K0vlf zjcm5Zi?n~$SI131OWetEOgb}wG6tqAUFle+rfuSj@mo4}Hdvb_vKpn$NUd4&S`JgP zmDKEft<7$2=8&Kpcth+)=?HO_*JSGX%&ashLt<MgrWRed8|H)$!p69SS(oM5`D)Sa zuHHg2+vw%Yl{|wyzY<6pA_VpuRDV^GY9f5X4H)&0QRmb=2ImG+l@)XK;hOSGX1W%A zp2XPPeKo>CGrcM`^9I1pxfxl@sbaemU%VgWn^|?OQebVQ9=B472yj}*?)$P!%%s_- z;Fnfl(G!oUlg)>QXDLsZ@acJJx>Ho#p=Qi3rP@4~@LXCGmoXDYAk!Tk%|<7C4Ibmv zbI#B{xy5x|GA*CcB_m{;t2NCPNZg}Lj=UMgA6}R?V|+#Mmnw(km=W<1or5<a#(@VJ z@-0l<I$D((m3|O4<bY0@LnBP*zyxhS!*AzWp9a;gjSf5qw!T&$ST1HChlUp@`IXwt zk!~6dMJ;+fHgbvH2qHQel)J#zUU5%7v?X6+u$naW?4A%#l_P!8=Y)SsF6;8DFUuk| zaHG{h%gkP_j9n#wbAZ538~)bdz;@sfwO}-*HMn1QD9MN@8-Eo4O4q_kVvsXSdpxal z8$T`egBGoJO#dtF;89|Ec^RZ;sqO5?Jsb1Y(mVZIDRr*tcJ`h?3rqr1Jezh6zKaSG zQQ-Ql0HML2z=wB1dU9pp_a|gbYmw5RD%70FdJg3FG@oqWoB8<l;>CA;y}g6Rb-AX9 z#jbKnamQIwZ0o6ZUtc!Sf+fSt(FyG8oYEDnfH)ovXjJ$~3c=@5s6KkpP(t50A(}nr z-2=v{8nVfV27k?!YSmjSfB@&el0;}cD8(V;$4}SUZlaQt8gHkj9QeXz-v;7iQkO>M z%gN<7)=P>@7WfSgz^#V6RIrA<p=DSB!JRjECt0`~#A7@q$!tgta!@h~758%J$13^^ zmpF5_bC!<Z%8u`3j-j@TP025frhTUEE{g-O<k*hS_V)Rp5Cdt>XKIk>am&bVOkTHK zdmfGp%-0g<p?cD(T*bc-{)Zvy4U&WVA^EV!z3jIDR@9igjANGXnj(C~#DzX{wdCBZ z4cXjc+z(<s%$QOjZm_f4SckDw0n9Jk=$y{SaGxh!m}dbq`&T`i#7w1?0Wjmb&2`=d zm)IE)>5|A4h0<@$!fZy^!A0cL>bvMl5gK^aO^RQBWc_4wFBM*O15Qw{96P+o^XBS2 zLw7RXLt9xA?(~4>cguEhiQq_cA9aGxbv2hxCRX3an3D@u3+#|!8p&U&LA8z?!JBRJ zrJ0hQzrn8=@FZ$ja$mV`qj5D9{j9MbT-9-mb*<p;E*!8<9r&dcQ1tz4FdeJFHY#Qm zBB0i**n8uI$p8F5#@;e4%C_zM1sqTWj6q6JP=^kYMoA^4bEskH?(RlKP>>u_K#>|? zK$@XLX@(p+q(-_K>2-3i_gU-1de`&3-~4oKZeDX9$A0|dx9^;|h<9p?7a#9{TP7J^ ztV}%pW@glP-0;hDxso(iP>!QaI)z;+>@=-cV7u$SU>_i&+N!xY^Xpz59B&2y#YpVu zd6gG8%K3UAp4H<A&pHdr`Y05#a(HjPITAf;ti%EZY`sal96q7E#-=SX7;ki&>zLmi zc&njyX<4zDLY?-3uQ%}*4f^HS{JJ{67xl`X$tkmfSHl5MxN^#hW`KM4w&$Qo$=;Mz z+9!@B{Rw}gr4Lv_H$-*nT)toFl=1-`u}`EuniVX@)@~3lYB;mc>6)SP;D<!ML!@uH zfc?)he3MM823;q7YdlTC-V%nDNp2Cx1QtDX|6~{K+tX6$?_5#V7$PQ4HcAZeTU3Z# zJF-@6Qn7W{-Nb$%4x`;56P|Vhq#}Lz9jiG-gCNjb%8L<SNz`iCOyY4TM(p1-)a2zR z9bHU2PN(wVuu%xtm4mB&w=6k9z`Z65-V|Nd+g}tn_5jUq|4Ej1<+DT74w=Hk8X(9C zM2cSF*mG~iUiZZv<<&(>L%vN_)THvo6lGmgjO5OcfW#1uH!e9lVl_=(ZJVkvw#`jy zklf;DW75%Xd8?aKr880!RptzS?|#&$_>Yctx2WR~>@mf~@u>g{1(!KC9$}5HoPbd0 z3Ggcsqc*t7W>aFXqEkBu6pS1mbg|g1_*BCkfb2N@IyT-p^=f!Z#h+6n6+?B;{qNv6 z1$3i#@kV;;>6g>MI+a$E2PrRlM!!(GaL_Ugdu4JLInUrzQP<9ZA7R*7b6VJS;&b)7 zd0?kF72>>NW=2NvD4$VLrsQ&mj~HeEG*Pvnu-5NmS?AO<y{M!nD)M(4HVH(XiS?!S znE<>(7>AdVG#1A_O#WUD+4!2X*W%?&WvHaz+kT9rB7pmI-d-<IUfm}iJd<g4G6K*@ z&RG&1{`;SvH`yGj{vYJ>)k%pwIi+HH5Jg-`rmB)W5e$(HhcDoa>_<a`&Zn{00D}&6 zsVU<f_@XtH1Ld@W&wbW}lJ(-ZN7Z-pP8T@xcS$K(^fsQGeqiu$u<E_TLinzn{(Y*F zh$H2n^*1Pn--fF_cN||CnMp)8U$8_-?760XhjZ3v_8)%JkI{537n=eRUGgQVXvf^s zPgDBH$c$Fix)~@{V>zq!fl$<lgq!0NuFSMu%(_tE7{GtfTT=gwj7=ieZsSVjBzrR< zxDvn=!EfmuSC=8x!ycerZB6=qnK!iqA~tNCs3)P2nk0f1Bq?|V*39+X??!J|Z@Rix zFi-LQVXN2x;w*29e1TdRU89yE%icIpLvu>hm-)`rJW?r>07l!)sJv`i-I{c@?2hS` z+-Ooq?Z<;&?Txom&EzEdq50k*l|=`;Q~9Fn7-*n6gWddv1o|<6{IP5@0WdW%gsNND zX^CeQQEP#v-pGFB5yMmlD^#-a6LUeA;y?Wo+$@5WrOP;VFY%|`Rnn5jliDw<kOUT$ zeWH$c(Q#=9@{xBO@C`t%1a6Ee3;R;5j5nn7U0&e)(+$L~M_dZ#zNOWCe9xH@bLWox z==hG?@pYvJY?&}*_;z3pE$Q3$kw!?)mb{|>8T0wLzu<F(6*u*2i(${%YxBzA!Rgl( zM!G}R(!io&;LDHky-l0#`qZZ9-;XL@03allZ3tS7vMWb7VQ=&Z6-s$mD`y(eOb~IL zz7I#CZFaQ%(ZnN<P4re5=NkR#BPGIF{$wB1GLwu2Sr0F0ffY!Izv+|4R>hzfqc~sV z)hp1t`iak^b5|9GD0A4V?MktRpLozV-D4Ubl(*#3Tz1Yci;08?(_`+UvY3w>0n=8| zNvVt=uyjzE(vn)WT#_ch%7JRc0CtcFxVZgi`h>Ts$Q56o0Zs9lDeCc{a?V=QZ}&>{ zwVkx-4=LVaLRV{qMuf>>gWui6rmHtNW5c%P^{%i%NURsydwsq~hWknVX}zOr!N%VV zAP@9oUGlxS$5nUIt?lb7KOM0t1wSBFRaXPyd!Gnf%tfko=8K-!8{lV3KOU10<{o+I zRgb=YG`ho_SX_1FO8Z7!*pDSFZ-K(J6`|#s2UaZ)^d4*S9jz6y>Z5(GRlfn(RE*_I zwi#~@0>1HFYZZ*);P<9q$u3HbT7uZMX;1B4f+P9<E5vo_QY?Dqwh$dP?^M+lrC>E1 ze|eZakV$f)3Yo0Z%@0(W#$yZIP9FWwwF%o>qqzk7smaT{+fgQ5dxhWcPAd;ZFb8Om zFeOte{SlL$YrcjVmb-D}l&e_KHN<oTyZs~OPrp~<(vb1zd=Xc$`Ro0`@+~Imu--O? z%=Yg|(|ih_9}05t)t7J<ee73Y<7&_9uyp$z*(0NAgD-8qTql)V31F3{iG;REEPk*| z(P4}o(JIm@d76a*UVYQ<9Z`Id6wELCdbBh75<n`9)6eUc<BNGl)MH8{Co<eT!jvYy z%`aM1#vmIw?Nhhw-zB>po&dI3qr)Rtl|^$=mVXs2^&LN7>P*V@*t!`dBxomw%a+{% zF1v3^l~6JkK>T|*S?=0F=gcuCm;OdGL-{r`OXc7Fl^Gf#A@o%7Wog$5xcmd@(D%7V zzW!d<#-05FS@ez@)|MmBK?WSpDi|$)TMta_@XR(9^+IRmul{vLHG$d|2piC-71(T~ zuh+kKFnb?{>r15+oge&}hF54{_u|31T5`p;8%(&H(mNwh7|?t=R*wdM7-0PM<WzNW zn#SW|`Ug`5<f<HR?u^=oxoP_ajpZsOckK98DIJ*i*X3|=smEOzHokgb^B1^=K68{0 z?L7QMwON2D7gG=^anP3@IYR^O>&PkNf$)*yUv1?fwiL5~H`f96D%sMG{q3pCo8R_+ zf1-v%yry6^1^3W=pXK4(nHQn}fvrnC)a`a`74>z%pzm7QAdVdgkkN;=E4#nhN(QMM z0B2$3N66~P%#77k0w<{R;ELzZ&*L9SjxPoRg!HOGI;{k$xxjzniv8zzEL5LZG-@&R z*UoLX@0;Pa${T8yXzp9aBZ+?`JI87~KA+}+3^dwwbstsE7YE|90j!R!+-rQ8!aXXM zsOMGDz)SXxQ-(vdDWM+p?p`vRI_778<=nJ=X=CxcbLDnfBym;NMp~!+Ble-fOZxE# zQc|f7;r_R#tg-mq?4+g1J4SB^MX5Zv9X4YNU>l;ka+)RhQJ!?*QioA2Hr8f%WN_QW zU&7B0Oj8#V44?TpWZ)#Z{-VOtzWOXiVnZ6Moh=P8Y!h~)Mn%{)KN?$Ql?rE_+X-%8 zixv>42+B&$FiJ!YYTP)u5kEWb{7@dybME_z5{GzNUhy!SXz7`B0_MkzLwPrgQKMb1 zGyAPQ0Mn$`p=lNrt+c|YE0x@K^nh0F>^{)_sMe9z|48+@)e42KpF;9M?8f=;<G)_l zRh}<|o?7v`bO6A&&#NZ?Cr@SR>}^mN&`hrv1zbh((CmIlb;6Bbz~Q$^s(zGzw-NmH zZ5IyA9Wl#g$s3#UD@jn)$4T#asqLrfWMcT_`*xX(_xp=cx7)t^-h7XFU+#E+Gl@zy zm8iWhNkt70V5x$lRXt@j#-*bMc20dVe!ZbaL_bjg*(IBsmc^Vh@o2c~XzZ>OTT(V~ zCe(2A1s3)>s5+yH<4{eU@(iTt@-E&*@Ih~(vT8w(1C6j!e#U&F?96isp56Eq-Iz#e z&d+yq-6(QiJskb7EWj({MMF_<wu9H2IBn9`556`5hRI5?8H;5-UsczdQffpx5UlR@ zRAlb}Chu;uV*5Kl7`IoIPN4Cwp=zkxkEA#7I<7uaPG?LEXIFTJ$8zzAwgaHw<^dvi zVOq`$W%KuQ{Q7!&$EMK{J-^dNKip+BN|eo0|G*utmvYMh*Bu9&TwJ;h99cmB0H2eA zaj<gdDqeMGi*OSyB<ALrHW)tBjK+y`!Sshu>tl3vFd}qO4me%6ASwiv7$-IO3PMN! zfZP;_H5#dBL0BR#fsp1L*{y}U1&KdCD?Zu)%UE*Du~&%$Php-tS$6%dHX@s^fya}p zKnE>mY*g&1L~(!{DZZ+}WlE;Qv=<{I_MfM(+x;$skm8n4Q^~`%*Q`xuRJBZYqu-I8 zhtbS}TeZoZ+qc*V)B6vD5~W0K%)Yn0H9dP}fJ4hT0cncb8u_i7N+OFY_*o^`5@(8G zUZa5OnD&d0=uKrCpzKwEuw5#}3#U98*yJg-*>wKs&sKcxX$}#4(nuvqFwkaxgF6LY z$`lk$g@LBTSGUP%k<PCSK25(H=OHX7D*BuLz8|7G^4L~0;VrbyZ0m|cJ@nY-sM#^} zbI+c`%?<ja3u$S@p>hN<_!fQg2ykG?o6=Vp&Zo!EMJ()r9XQA;OM)N7M7v|!kG~&X zu-M#=&tJH&5gJW~TT6dO^eK-tN{1td3p4JXi{rGZ4x!aI61LT^Zp2@={>k_rSxvoi zBOV?0$tUQv<>K1?iQRl(Ji}2>eTvxdB@M!;w)c3dj`|z5S#Ocqm$qA)`u75VZO(!) z)P3Apa|sX7-hQF8;nv7XrLOLtVsQIBex)&&>wkI_hxSkwyaJ>_`d+#wO%{1B%g0aw zb5dG>bXPa1)Y<>8@b!aeBSy~f)h~TK)=>aYhqb&eYP-|{u95d80}zlxk9Y(7sz`kt zRCjt7er}-ZKNt<HFy($}lYS9h{&bpw5W=Qg9qN{r@x`Id1D0o{E8h0qW_m!Q(q5Fu z%9JPc^Xa(YxFMx4=}~056T17;acN&-&%R?Xi56Se5d&Me-k|;vvlJplH?^^XH#s~Z zbF2z$(Ccrt6!oQzZ)*-{6k}i72j~Ii@d_J73%ZAQSyVIrS`PXGNY$rb@Qh4x#TmvU zFZ&>9y`J!2!$*c)-FkOCTxCD77v1ws+wx7w=8Tb2nxuB1V}{2lyS<$R{N!gFiN^$@ zs|AvEL}JUpbR79^@#Uva2@I|P)fPw09GCN*L3qiE^KGx_ns1L87p|j{n)>Y%+f>uk zmmQi*ir`N@R>l5LL(Ave*0Yy}LF}YtL^)j4egkAWTe!ibZ%}t$m{1i2-yZBBnCY!c z^0m_OR%Sl_YbL(3Rh2`u*^OAVJL-L}w4~~ZTh!<QPsFfE9-{0(klI$YP*gAFnN7ya z#Eq2eBp_LVg?(~Z*2i>^>R2`K@a5GA1wC-{P%*nbxCGWMWDxc92IBn+ahdIoRcAWP zIO?&-2M(!~p6vj5n)cmwF_n!bRFk-C>R<@{Mjw*I&bu{(8k8XI0c4Nqf;!i5gGo@= z?Z+!q`P-8f`8BI|(pKe64>-0`6%&E_`CVtVDX%6hdr+&QCSmz{cB2I`t-{OaY`Vq@ zTBubpc$P(Q3&X|)Lr_yMX!`pA1wIAOx-vmqbrCAW&lh9A9sv&3@kPXk5@!JncFOEM zm5z&!0;UrXWj1Ec5-0i){lmwf*Pl++XjKmt%NPN2@<nW-iryIuPd%WNeCDf)NDb?* zTMcsUd<{EEqGpQdFA;3Ab>?=sYdBwW?dv(!tx$9F5JwL3JMx#9KXBpXpe(;KB(~Sw z{Ztl|wZfm~sjnt}lt{B6U~@aDh$-0j1&sQih_oMk{^pS~n~CAn&o}0kL}XLSp#;gm zyIxLmrO|ovACl0ui7J2XnIDafgDbJakg)QW?7Myn=40p}fi9iU3)cx@kF8HXQoVoJ zO5*;<9HSJIJ&`By`yN3sh;dk+hOpmP|MO-S0HjjV&5EJ!sW`@JeeCaUy}HlOabO%O z8-F*;ZSbe%rMs+uwJ2vN6%PGW_GZOz#eibI@YSB;bSPsAP(+Qh5d)%rT5lg9->$no z{AdsEV>h>^4kL#hQ;)rgp+t3UUHVOxYkY{bgAX_e-OmMN*rXi)l#j)Oord>M!|{ql zL88X!&1<R@KwL2y!hkI*o#!RCOE8-vk*h1pxeoaXAnA3Q9zAu(C#7WXdLz|3H+(f% zmhG_c$xo2w<MeVpAVl+Qe-2%KAN`wt{MBok;rcg|bc<Fqw{%qRXuDy_$Xw!Ttgrie z{EuFGl)33pr?FL#&dA3w*g~8z3GYR@J4^D*Z*|vIk9sTX-FwhT2I2W^-}4Z1(=7c) z&^L0J-1fB_E)edY;qP3O(<>_UPfYhr)Pwj!y%~Dd--Fz1s9()VQ|g@*7QwRE7J;UO zXI#SdfuL!82KMiSVt;AV#WCy|Ye0IpM)h^WZq*(2!7k{UU>P6I&2}R-Bc{s82Vk(7 zxsQqb@Qvsw1X4901C4U;jJiYeZIge&4-zYyfXmM`6#?nnnfPhG$<eFf{YCuvR{!wd zyV+Fr3s;Zz`kSELb+Uq;4P||i*O?0qmcbwR*#Gp`-rax8A%!(mKcfb+d5yyI!-i~1 zBbi}g=_e}w<hOfhEx{Xh&oRzm`Fg$q^zX~EjeoLYrmZPq`m(^3<gb?fXgq$?xRbl$ zVJCaG885wuws>Vte${I%OPoIMLLN#Zsz7Vfi^^n}MP_^oAZ-*zztdG9?pnVIu*~+4 zCk?9XELSmEJCPiW{+lm0YHmpKZ+%&VDGcVRWo<|D!}BhpfdBjR{!LY=qa7vN-~#|* z%Lgdfu!7o(dmel~6uK=`Ti-0{7E0qW)y>f;lj`^857Zq2n@t$}(VqP1CQYo5{#l3t zSuK}pp58bBUzVCu)1B`b7KqSQk1D%xX_S930krAv`2zLrJ!E!cA<&fa4eX<)tyNcx z=IjyL;Uj&({7&s9!p2xRxC(K9lB!a5UM+&0E`DW;yO0Y;?O~Y`efzK$VCeYh+lJ@i zo48YKG8C_kJ}V#~P<cF3*3#;`4MTAqtn|DK1Wa^#VmGhUIcYzH0L_R*w(&x#Gs~$L zHMKH^YE(N=I<U<uiCm5$Kg*RIQ0V^$*EarM0X<=PhnE{|!MpBAnb$I+?Em@^fjv5m zutU3PS8#-)6tzI`@#or9e7s1^(e~_g8dUENrBG;Y$*dbvQ&eLWoGk$mx~+gZtGx(3 zMhbiaQu$6SieEE#5mm-3dXlFpeSRdSx>yV#TQHQNEP|q9iuW{;b$s64b#mVq*eXPC zy|;Xkprye3CKqK&NtLV>!WHpEG<1TG@GL7rM1N$Zm?`32navtcsCvdGSOcG(I6roC zB}3Tb@1TsC_Uv;}vE1kdW5uuS0~C}=8)DI<xAkH@^U`X!TuGA_KQh4w462p$-1wj_ zAecvxs7kl*I^dG+$=C|d`IyhF`**JSfwj4T(#kj7%HE`FV}#%x9>B5qwl(Arf9lda z$+yiCrvD*4sf5M$gF8&H+%8?IpvYPy=9k7rfx2)qe|7VA>t_3@N}GTvYyI_Ddkr&I z@?c&5!Lj$XS0AXZ;{J$wSh|!)R#8gdVlZ68T>w}iKJ>g0CrID~C=h)H5&LyAzpyM3 zU;50iI1r;-c#@=aUSh4>$_;R!ZJ#g2Fg$l`yFo+$^?AN6%&H6`=Ge*csR2vT`a-bS zu)lvf`;oUe)b7zzX6Xy7tia=+FX?G|ViQWj#FNroUpRdY>&(^v^q%-<d2|&DZJROb z&(1Lz&qwgy@@>rYvE5;bV&C;_``*%JIOts%+~m3&(j&{H9KvO4^(6cUpBu+N#X8(= zpdtO@8V8lLYPZF0>Xyh}rsU%EkUF-Hxyv0c|1=c<C6)k(tM?>Prh*jf6YJKtF!?%E zvH+XHgDksqRJJU*4Y)%i*lB0l$ew2$YUB2%7#|DTX%Fq>g2pRJSIKC~w)#8&d=vND zV!y_ESpf{Cs%rUGrOL~}%k*ADOtd`(J&6G(uA^Lt&}3IEGM9>+&u6D?_0<&MA8eHH z*N+3YDp?X@!ch8ad<ym*z-A~P`ge~N=FBdAYPjf3EZ6gLqZAamo~59CVRWVoeqPV_ z?X-8KmR=rm4$zSN7O@a}Pq@5!{hWX6&KHF#?QZibNy-38DikC6M~!Yj;!@A`!u*GU z-iI*lLH$P?x5eN+_*cn#y5m}dT?N@sd^cf2O?F>E7r-3nomXQtFMdA%W$6Lxg&GEZ z?6laRCO|{WYa*pFWV+<YTjrJF_kZ$v?!V6F0d)c7sP{Bmk})|lg~D=->_8g{uC1Ix zskCb%`ze{0w<ofj`S)J%KjlTkr=|rNG~fHEhNf36%-k<OrMv3KJh<i=Yb1r4K}myt zwOfIQuP04r%b)HqTuEbB4rvh-Dl6T-@~XF_^FOd%Ue0IvuF_Qi3>0S3+`kjRt841h zGGnC(osi*m3sIz%QfFFu5}e4+Y4e}!cIJPD>+_syIy$2yc6xR<OPkk`FYA>t4Gh97 zu~e2q)<~cRKI%VL<VgIa93<`9sTdjW5l{sZ<gw!UhPv`O;uif_Tsf>rCA$N(1&ac- z^ZLs|z|uG>-MnW<x|VnMtV81uW&rTR5^1a9fS_BG#_FA;ae#jL%M8(JEg$;!K`g`u z+=gbbChMHa=#B^Nd?e-$4>wdzw~gXqerEZ{4@KT^Dz3_MFVbg)mSf02u`qZ;|MozZ z;SM+n=SEFducP-{>#CmrF6J9&h2{*?GmSp7X5CX?Z&D41Hak+h2PQFae1(lGA07$0 z`|@vgUl>-~E+`j>FYIPd%7nUwQIkdc@(?!iHGh21hdt#t`n5%-5Hs>h{<~RZ!BR@5 zE2+Ih>hNNQG0Oa!21^XN?JhHZX12S~DtA><34ETik+BaNI;h`u&kG&BAUzd){)sOd zx=yCA&kW4EtJkw`Te%r9$d31F{%5HM8x7$&tzL@v##iQ|(WZB+aOs~4der8UG{hST zfGcwQ2ll({r|-dPPxHLCy6v_trX?#PBjBw$hIpMFsqrR*y3;q<mAw1))a%*ehtlCB zf);YzRZghQcH8i9Sxes}5&fqo%?jFqfU_c+{yoqk@(T?>`_=3BBv#_cI3HGeE;yB} zwE)>FOdb))n^;{orE9M;zSdtYbV?K@r;)W-bZO(UftBMeYHhn%ewu_-e(N;ALbBFz zE}4W7JkMvQ7|aQL9yxR5$P-gLY3v={ej4+Q$=6qSTv#^nYNVz~EJe?5r_0%3!=|6o z3YAR+q#zqHKUk#Eb80-FIL-QD^W}R`dMYt>Y3_0X;!&--6}~{up;z|Z%*kA68NFSG zEsa&~Z09;!4W9q$5|kQwHn<Fg8?wEZ<K$U-Ht8Dr-Mn;VY-PK42ET<TE9?M}L%}G7 z6My=Wo*p$jjIQSnSA@`bXBRMQ`O=9ws*(n=E&#N6f&CkGSEoIY2B0qW(jQ&Up_pjG z9~l5dkUPI;IoyGO^&>2$*&Zx{p`fn@KtKEI;Aa3kjxNMW<#M^`r@afPF2<mL17^_Q z2e}-tlD|(!SQaMQ6amDzxeJ#_neuyPUH44^6S~|?L%jXnF6xpACl~q75wU!uAwzAn z)Bzzh5|t?{I;sNjz70^B?vZU|16iF6?w?b|e9J4QJMNo2P}N}q9g#J^x2*3jFa6D} zG*Ayrc`2-1F_dfYFl|u}8Diy+{ujkDWWD=~%^!Y)8=nbyLEB6xlj<Z=0=s@c>7&z} zgaNhId6%?|+bf2&0@0jq@7yQPPY$BzOu=hH_SnAOk3S^S`Bg?6dzLDe!0mFDG1pk~ z;?~E<Gwy0?eaSSO5zlhVrIVf>fAXVF5?3Y?OlJ=-p1YyI*ZP@JaxT5}L`C&a&)^1M zWIDj^S<tJ|z!>NuRIkoifUmQ+RV&8rx|>ie)G1mpjH%~bDhGWwrSm`fZ1Y(Qv942M z;JcK*T6mdF?m9(}O95tM!_D^9Z#{3XVO%+%MXvca08TMlm`0oF!!^lI^?*-Y4*v!} zirDM-`nK@u`AE+XF6E^phIerH<JUjISKa_*w>OP}$_HWzjZ#&rrzY#I{qc$I#`5)w zqLS5jq?8RNGOmyQR^_D%+Uu-dl!60`z<@{d#TP)jCTV$4dc(7z!FddTNplHApJD}Y z<1@{}01}{+LRrwP<JNOpqrt~P|NeY$2?3T|yxDAFzD7&+4BGII2Nlk7r+xXyZKvwY zIbTI%xONtQ?3YWj%ONQBy?G8LzFSLI9Zqf-szlZ~8D@Z9&vst~O;^s|49p+Ilcp2D zZ$wG`u-$Rq726UA@u$$lkoFED0~dd!3cx*4zbhV55Au>#r}f%rH9q)OrUc}zjTu_K zo=hN)O$Au&v#je@5S(_IoV<3_4B8*C{-;)PR$uh}_ZP3bcOQcI%g3Ohyos$d)D}WT z)ky{e01j9@uB!h5Lm8k@rUut58Q_4*AZ>}|1vP7IIcJFc#MvZN_~-_4=HTP-enY~t z_NlbwrWG*L#w-7n1@xd0aeq>Q-cs1HAQizB7DvOSXL2YmVi(L}d$r408ZrSS<}0c` z4&}(F0|1xt)AVCD-v34q`WL)8OIl`O=_|Zz-op$wbfFo2@-gK5TWuGqci?9zHroCL z>KqmxX^Y>)6sY?@mCbxRZ4B=Thv~4`RgOj}DfSK*=yJ1wW(ZoD9=ejseUkZ}8+0T& zKYnm@C~iN$-83uSTw~X>lDT#9H5aolfq4@`Ku83lhME-Kh^@Z^Vx03m2}w7JCg<oj zsn(V$LOP^<sV0>}oqp68baBWCeN5Mv5#^Q>N<jKR1OSzegZS>8#zm20f{rVCuO_wW zYPYjha$AutRiJ4WrfMf9U$~Ubm<b9Mo(~4;Lmvq-9?5g^X%l<j!M_O2Bo*twd9GlY zdU=U&Hl|NWI<jo(V!F2uY{lWec?VMa^hR--0g^DELy^dAiZ9bz6ken|UQ)rM<WKaO zGEs93+z@YG$svLl-|<y*1(1W<1IdKao=Mg6H!?Rkl&xC>!%N-ou#^c&AFDVD+^a;c zubEr|WQOu$^}!<_zkYiot-8yUWuH_JCE~-oLA&8ACLe(w8)}>>({Kr#n0ioJ^3*L^ zksx6kmb_wV+}WVio1FxDYF*a*bteGlhR{Ye15D9grV<b;zs>ZL|H>F?W6d=EXve2> z5$KJxM>_`{q}QN0!rhCIFD*<INj?cy<=MT<NwQy$SA&?oMio`q9QbUH5^IfJY*S=_ zM^k!|a=62nGzo2t4B##HW5>)sFY&MFipK<)ILD_9U){vX&Ns0Xh&SI<k+wO3tAaz$ zjaCi&ob6-QP^}-^rxO)t6{2`&;+3BSI@&pQN(D7vGpF2`Y_;egONsB`Ou-b&9J^jv zoIl(Wm62bEz$m5vx`56~jn8CVPp<sAUC!oVs_r`}ipstufBn8BQamDOQCSSnLu_bn zyCO*vF_rDlIPKA$B~vJ|dPtnoHfur`bMt}kKyRS@B^&^AuZYWUO9F6q^rDq;y}oa= z&-DhS$sIU;rxs&Y$Tt6AFiS#wg<Xxk$Z<vZ=^3;17TxGo9VQTkN4<4Jf+}>Xd?l)_ zT%CUojW^_3*og+!>%7<rR8I*7ex$M*qr^<!6sKPW%(k__%Yrj#7DR8w>##UE80vu3 z@1c`IK`LIvksJTV_ogn*CSraNGE3(Ojl8ijSG3y16Dd+;fcR=sq9CqYdE^}*oTW`v zDi*vd>d{!T_DE;@CtByxfR~|5D{&Pb8opJmI5+m7G60}~Qju!zOsB=?+>wm98V|nE zrQ_8Z1D|J@JBU-&Q2xmB7V#u~kUp0`3RT&*k6RO7oI&|Be%<PJO_q7@dzW8TN$_;$ z>dV=HP1!(%*z%Jk!en2@Bxe_u66BOOk%4^Cm^GbmQ@8toO|P?J%+An;KVw1wru)^n z0wk^HEX{EGVD_8ZJxkQMuLPY+-*`UbNSe?^SA{t|I4?=6RxQ}=YgbL;AH<HHXXr@y z+B(>086G@$g13b?Sr@Z@^nCWEM7Q~LY(*F|gm4$cIlz<z*Zz1@oOl^~2Jfe00`ktC z^vCvL2z$S|`Onk)LI0%h`l)Yiu9=@2jA1Kwe$N^m+-H*d&k=F!-v9RxDTyt^BuuO} z{ER{A5`&UBQ4IS{8|H%m66KZl0FFoA{py)gRY+t2G`zOy->>H}RQKXSXXr#+cKqk- z-`-Qh>I(n0cTo=jDxA-^T_Rp3Uh&`XUFQ%Kek8N^wnbf;k3fRaBAw&js<|@>4|H;j z*w_$oNK3GV#ZYLfNhOin>Hg0zfyHmBx2h3@%mC-yOw>j40_MDe)h=?HVx5ejgJ7$O z^hjv)IC^$l%C!x;_NGR`pk&)qkRLe$-z6_m{<dt#rnhsSoB(M@2TI@=$l<<d1UC2C z;T@0tNtU&%kau@Ir-f1^$K=YRN$u2rVJo#cRS4usdRYlr!;_}MZ1|0##JhPMWYF-V zv||k<EkxbCvI^?p@G7t9GE*uE^y|)Dre47pq0-k7eVi-{Ez1pOP3g)tS|?=Xgs-Jm z3UB0XzLXj@B?61_1JAM<@>55$l%7h`!7iz?A;iV-$|+R$t99i=3z7n6I$3D%Nh(vG zZH=#(H#j5Zz_D_rA>bFMk%mHi%LsbIWZu4|vfmwrg*H7bel`9#14AmOwh>><d;nqc znIZh_2n}c5(u6(|Op4)aDK1=@!vU{&o}%L)7zKiZgsk%)T`g5S%};V#F$XS>v9sa7 zo`fIs>G^;ifg^)R7#UyZq2j_HDdDaEn`^st{di9f@|KpOt9vgg<Z}~rG$EFywV>{* z#2(}Go1bInP5oxFt?EJU^T0xKhrj$GlH=(T;XOB}vzA|8zccN<u?JjO9nn(H6>*lX z3*Ag}=YS>p(p3A*FD3djFo#3I=gpzMUMRas=U3j_n?Lha6hjC?gNP2OL(Z#f8cUQH z1*XuT#8Kjo<wyEp+B-Eo-MMae;~;*}NdAaqDAj~;T1GMZkkzsDocSo!TzSE5`{cJT z<K&J_%I2u?Nsn%i1#f~46q6m8JpG^`T`{V#Z8CSI(SFXqX4W`HhjE~Me)og$gVMk) zdrsXE=3Aw9zw?m!qydjWkz<X|dtEbkz`~7dD!s_$6YS7ow3!DzUvLbjy5psS%i%j# zv2W!&2@}oX#F*2W>lH=tCjL@COe+4=7uDnc1>|jgg}1Hq${pT?%)QWoX7*Qm6qXwu zZRf6^$!P83nZCJjnC7>w+uQ>*n>kV7e#6*bV8^O6hMMor^|DIaWLft?nw6z+4e;ax zr=RR`=5w!B;Vrd0jAy4Ys#*B<lOX<?eaC9<6KQx`0ayOI0j%b$CtfUP<O|E*qx^Y# zSBGorMN%Z;_5RDA9p?56q!P~w=IR{t3xIA{(n}BgLW1H?bP(@m*ZhOr_x*=)4L$x# z0lGG?Bo4$SdQo=pZJN%>tkoy~|7QaeFIi|Vxs+3ni<ne@NEC>;IDgEtT6HBafQ`|K zjw!+6B8QuibA!Ysy23Fh=`DW?OE`-hj4w);I3uQFpfBNWbzzY9n5K$oxV;=Kd7aSn zjBgSJAjdansZ;NE(zOgeDCR{|(0$%7ix79vE%TIkdN)k3-e7cV8g+12E&0oWvu`hq z<K~<&1ra^hGbn`Lhx113StXijoxJSry?qxGUm}=R5?|;sl>3Y+l*W^r@QRd((UDZM z5Q#YMRtqZsHzIj4qB`4ssccVgq3g+D!PE_K+N-R@J*A=0PLpXlEcBo&s5hiRp&Cu- zDEeC~`bjtSX{Gj!0fOj0*a}Qn|2yVoo}eSi%JS5Nb;zQ~^sps?!feQI?VNt8)oLBx zLi5TfuaO-SzTI&OqysECs~NLYmvA69-Zg}<U(J^|Dl`LBvB{%+^5MQC@$g$kF(<yo z|B?^*!@(O<Kr!cEIT3!rKs1F1Cl`;btRmOMEG$4|*QUW1lB%*x$Ri8-%+t|)tPZFM zEy4OkHfZGZQo>`)8u1lE-JaCE`bW&+w$#8L-z}M)DGYkeDzGX4RC#vCYHXqX=-7GY z#M(6&O0%4?VGtkgp;9d1#2A6=txH`Bs$6-QM$prgwCDCb!QKEaI17rNSrw{D?Z{K} zpsO42ci#Ge^7W)ow$22~9u1z;;|E-brsYgiO-wZ1yR-E9^E@Gaxmo^GF-BNLZOz1A zdm~tG69hI5n0~M?vXsCzW5N)MHpyt1e!II>41*K6aoANu8<qEC8^Vk2augocL-aSn zDaoG9`hWh`4-bLBo~>D_lK4c0ZQbN$hq4&ZIhUpqv<GyIaN5xNuPlJ-_m!`!@{*@R zp2TL=#CfY1GFprIhtGC$!E7_k*&LyVIrAK8nWhJipX}J&R8Xt0WlpsVGv3H-kD%ns zBeDt`k(+jj-3NYDHxceH@Jyb)sw+dC;@hs4Nj*rAaLz`d(!<m>bf!IA*(%WKYT>42 z!MRn!#kB*vef$}r*fx2SG)1}07}{I)ta}{X5DW}PUTbdowwg%-P+56cva^rpPP!Z= zFZLo+n;G4I;T-dGvlI<UQs_!I=fc1On}2z{rc1NA$CZW`frFPVu?=qq4l4p!2Xl=T z+y<#PGx7gUu!AczTjc|&5p7~7+$$Z}>-M~;8c|igt7s>WoZtO__f`DMLev5lDSthQ zU>L4~MUV9FPaw|*##dg!;9;u}7djQMjAeZ&uG@w$@=!eT(?NgH`kCi~TS=eb)8>*B z>OtmIPpz?&&%}Qd7<`_-ntc7g8w&nwOi+GnvPKkJZxK1L5DM}y$-h|FZRRLq(X9&5 zeq8bqh9?wdoGt|xDJ`x%4?gh|%FhrU`?demqt?x7wsQc~y5da7P&MZTD3{d0S_HW* zGd>fh8<TP{qyP);w?a6uBQNO;$<rlx#th{uBH{AfvL>No@ca8sHmve4(A+P7Xa;MM z8C!UV+MriXW|}4sD58H5-+D@?o<Qj4$c^e|$Xt(x8)zYp$sb(rGwDUPU2_TP^fZNQ z?Q1z?NBoKaF^Kpecb@cRmr18><ut&`mqm)spA2}T3<Egcv#<qJM#J`Mcy29k7+;;H zWw#t1)|A+Jk?dLAv&-V3qH{?K^@rXlMe7h4sbyjLox>EgET4=|*@Bq7wt}??<db_k z>gIIdHCo@yMSNdh;%_kiYnhra;4=SLV<ukhc`&e4H=?b<HXf#}tp@!Hs4KM$eRBdf zHSBx81;pyPD!4UdwL;XlVPLMw+}wT5`Ov941nUz;xL_{NM!x{vVdeo7)Nq~Xc@f8~ z#av#L9DLkZ5FQ_q;XWX0PeM7+?mO`yM6A2xeucz*dKmHVlD?`;^K7R%{TX1QNhGgk z9V>9VD_n$_Z$ocbWv0hBT<^=<suES98quVxarRs>;xxu})Hw(JAv4x9^qREXFw{KS zoIhU`enrt`gsITOX0*hK>3)`&KVH*D^`204wJd;ymg$O+P>M^8$3cKY|KglkPN}cL zYQj7kjhJuQd==f&KnjuK?j}sI;o8Zn{Pm!~ACo?hXpm3!EK(AS!*!zZ9pR3}WT3uz z`EaxS#~k<x&l07w0h2ihfr4Pu5CWI;YHlw^7W{8{G8@G#@^Xhpi(OD!&6Ar{G-20n zD%2-o3g=1qWr;_!IZ0IFH^p)9tF5!youDfG2&vnE|HtXsZ7+MFz2xR|%ew76b1RQL z5GxCTPi-#PNh>TJqS;?xHT1&@Y9)C#iRpRg<k|g>53}p5mHfAA?ZSSJKL8kPIrlFX zA|%gEe*#@zs^+u$W*Cdy+No+l*%okOclMe$Aeme3!<<-$tbCoU{WplF4^WL9((~D0 zd`W77s(i0?YJ+p%5jU`X!GA}Aw^&~ePJrt*Ixfe0Llp{ilRVr6G%UVexG)7kar=+O zsP&D&G{2v86HGPHsNAr-$D7)+h-TXRbHt)9*6Ey_-EHgOJjUlfw<;=%IZ)vqPjv{s z921vR^-C{`?}}#GEHJQ!wdFy$`}TB`{)u$zHjU)PdygaLOAYy1C#_r^#^7Jj+NlF) zB||^v*RK9vXw7<`Rd)!k1rHRSXF~4p0M%FH%#9lKScxA$?#zQIr^ukGWYt^o*?%(_ zlzk*pq6(Q2h_x?s2iy1nFZW`9W6Rw!8>^K+?_+dOZ!LdmZ_qoQ?4A!sNL2P!o^M)9 z`fqOabnfwE#N7Lt^0nuC2x5j1(sXpZadwK(;2rdnlfUVNRw^Yt_xQqZY1S3n4r)rA zv+#NfEWDWhN|1m=NRt;Lun2`ldq#eEn2Xll3+e7chIDD;c5b?=1bzP*?5`KSG4e#s z@!Sn1M|8I`3Gb*RDNkVq|M`<V^eWUbq~b~-NF|39*wM)#E@mxFRK^|qJxDDubt3{k z9GzN3Rv&IsKyE#2Wmu|2M+qmO7L!|PM0KVN!ie2e`b^SbGmF%}`T;>;_!(jSnm@-p zGrvE{xn-(nYrSK*$W*ntPOuu)Isg)C_UU{zrvXIucd9#h#)p7SZ{Iz!DJ=hx9Av>@ za<PL{jDw*Zlt6K;WdlOqC3S}8ctmHDW^XZ1m|F6PA<g75=BweG$Gagn57*>YDo-v- z>EBjvzb`c>EbK|xaDi^{Nl7Xw^VL=;^@bF?hE4Dn2#?P-MH0l%FyFu>Fj)-p-4mh; zC_*GjJJ#hzS8GT2_+q^;JXu7O-7p%^B+JJG?Pw(9np)5_*h4Z>XyZV<S?R8C*Bd+G zz<A*H#sKevcn{XGw)Lq9iNkM;_@Es;b|iY>VD)s$s}$k?mr_mar}BBcbzEyR`Nx4q z`O6hyJ}PhbCLIb0ql5ai)ZPHS#4l3!olNU*Gu}V=7JnFI5dx_OYI+Cf)3~z_X#b<U zpcNy2%#ZdP-Wy?FMPOgc$^Z8x(KQbAbe%t&tML;4a_2O2)YVB8gnbsfu=K}U4o)do zV*PdXYmAiE_ihDNw~gU&v^7esD1XsTt9Qp1zF!aWT%BFYX8xlECu$R$X7;IF_szGv zP*Hf1Fv_0vY5wXGuj}Lo_FM^2>^(~qBO7nD?h=@+ZkRtCPHxgZ?+iYdKe-gZD79l; z;ThI1RNyTef6*iPv|BTE6@qcrnWbQ##CQ5jmEMgT@Ms0a3Q6#-bQ!Im{o)?r=#@t< z(}6y-KVLk&mGlEq7y=vLcZl=8=}AgKS(|i*nGqIZ_Ux^&o5N5T%vRWgHCgUCOqBp* zKm6&aYqc!`=T%+0-Slcl6WsCgIrn&r$d_tuT2&V-c<+H3QLH=+m9M|*Qlxb6I63$v z`4@YB2A_MBVW;H3*1L$vpzB^C-xD+7n@@lw_Z?LMqF0mYGVHfll|wMmWj#w>kGmO1 zwdCUcFKo@G<W2g^-Yi#5z?q5YzO_Gp@Q8iv13w(~=me4M%m^p?ognZHIpvz{oqfWB zJ?!ThA82^fhyMkMe;XSd@3)-(dfq#+?lRxbA1dX3e9-V1M!Hz4s3;szs^OU-G#k7O zo8GNyJm1!4M)vN-?l_N}Q}az6qmi2@XAP{w>G>P)9RHgaM~Y-<?rkY2`j$9pGt#GD zWqNPUX*ZVG=d`{%NjX2vmkjJ*E>w*r$Jo-@WFff%NywT1J`caOzlTPSLEXF1V4D;~ zD=n$v--B<j!owE7+O;}z6Mps_@uXXb<9I~+@}U!*NnGm<l`S&kf3IDE4wjln*_BCP zA=}=+PRXE}F`T(K$!h_M4*J<S3qXcA92o}*2p@4mx%q6C%^A=1%&+QcDX0gjA!F5F z4)El!#qcxkuL3(tk*_CphM2~N`FOdLc260Ka;7s|N$84`Tb5w+r&u^z=Z;->T_RH^ zr~{WW14Ni#_QM1<i2jazk?zfo)4rgsQB)jzdCd!tguwnzbsv5!L@JE0^x2`wJrt9& zAi*}s_1Q{sI$nW-jzOHy+qO_7K_){woyfR$(s!ht2|Ou;p|q}Pv?}8UJY1!7_Bo(Q zLjF~5olcI<Z|0M;(dQGj)yBYa_P5o0m>U0VsE+YfG;6|Z?aF5V9(ocs!K($w^U2RQ zH76zT^o4y13l&Iv3+T~j@_Wrd7191CO&*^@%mvRz>4qA~221xF4k^K^Xo<V&BxPzZ zO&kDaVI(m4xSOe4E!o`7V5D#Hk^}#qHN)lg*1?;Bh>Veng0kUiHI}W6kO@#fm!gvF zGRPhB0)SP^!GlQ%S?S(oUCmXM;PoAR#C@B3j_P|4)`5TQiTz17yb`p8V@(zw_QhB7 zT2mad;@z@0nhiNMH0MG8{T?aN=^<XNBA>00jZ~bnVU`K|^kncsNJa}9J)dfO#6x9M z1RnNOU_&^ffvG^}MAiOqu3@c)dFQ1TVV!-E-<Me5n^fv*FV}vQE80QnFB}IhlwnVM zhE|pXHMWD%pkU9f5lEg=h~~7o{#*@foyHuaSIJcBSymsvYOK~AbY)T;b6jDg5S8Zt z@eda94>Gcfmy}-d{oRFs4_WSUS6J;APT=A0><GR2xYm9c1l4WS!uXdUTSj>{{_?AA zk5C4{$MzqKLM{H+D3Km-w(=Oo6B_^y+^YDDmLXLkeSVN}06dS2`3Pz_Ots)HfzUFy zEJsG`DIg+>A{TqyM8eGeI!NiX!F)jQnRMTY&e)*}vM~Oi$h2gnZ+tORBr;yRM6K6< zQeu9-d=nD)_VrA)PI7FHl35K%IzQ^7*Z{6s4k)&$#^b4}caRen&7%??h4uY@F@>jc zu%MiT6sKB2=3D97Zu`}N88R48s*-w^?@q9YX3GD1CH!wx#fe*?xujA~$ft3ks)a9; zlqVDH9I0ZpeqJvTuGdL6$w&4e=zPAml7u6I&M)S#0_hg?@K40U1NL9Mx{(&&SkJDv z1-vPI;}#NzgbbJ=`aceSRu?{WBSA~<hTi4kqrIlzZp)rr%Sb+BMMhOtnE{_2i_RG$ z0+b(<cfI9xmk{wL#rEA?2Q6ChzM0N0mw418(Y4TEp~dKQy~tXhs-iGVibTPpnYpW| zbZOTWfEO~bLb<~U4{9W+gxz8+LS{s`>1O!#e%R+-l20kg?*5UsEu8oX_^Z_@iEu5j zNE*{>hBf@yp5zCM4|iyqXwYeGrPClSC8Rx&#MNrrJ+NnC2)Vgq&gEC553V)#{#@)m znR8%$PmN@a%dFQ^I2__m&f+VBWVV?9Y!vR!@W`#}i6!;L`G#AAj<3b8b4(}FK7gdV zl0viJ^W}J@bnlU1t91@HPM<`<Dpnt^R250yQiPnF@-)7@ewH;|Y&j>7u>>9FFjD<z zwsq&Cddngn>k7sV0Luyu@BDp~zjFGuRnbv8e~HUq^I*A#x!FI5yEz?*g!^pR#V<VR zRDtCVXjdx!iaDwz_flDjfb9Nfo(=sMKHmj;oF9dM1Zcci=<;KhZtZw=rX*vx<~ZpB zNABh&Qq^v$aTIsAe7t&|Jugl!E5h}y%WI_U-5T;Tm86R3OK}G`=U4OY6#cHliHVRi zVM3~nHER;*pbgkON735(GOHvzUzce3rhDm_SEK5cm22=3u%snO#Vo^gdZxYkq~F=d zn|V3I-j_sCaCGKiHIqwPpsrM<cSj2B?olVa;Wr?tBCK{}_3W56@}AqHeqb>1SC*gU zn~-R_-k+u)D1>KPdfa<Gh-|@v@LGi&&4P8o4-Jjo)Y8w=#qw7r3VX$+PBX3NAsam> ziMVtmF6aL~bgqpI5FPVd+e+?zeyRBb?BOS7CIa7|$GRg}A!4RxeB;hn+cLH43XX3^ zmses5IDB=<%7xWfg$|PZE-P)J(s}Gzcmw6ezN&{0?D2kVL@}vde}(QG4TIN&X%&8y z6y105$HiW$N8d7bVL*YI*pYvG9}nuibF(>SAOYV5jZ{9kWTWD9-x(ftGfwQtU234| zzJRX6)b@Y1R_8MM>I~Gci}~wTA@M&IXkR`Y9s5<aApoUnPtao{)smlH5ZUtY^PWYt zdU$pd@8hf2GXk-Kz3#89JVP5NR~ujfZIqdV)(81Qjw2Ty&|T|HX>g*Rr+B|>){@_$ zuZ6&qzb@8paPhy{;d$8RA!1$TkzRqi-?(~B*y8Na%F3kJJRQIDSk;o+<+552sN-N_ zKM?nse;{Su{~cOJH|KF78~h0Vd|vN>yQqYjR;+UcV$f8xQtuShTr~G{lsWt>-rW9Q z`bpUcP@U7)`SsYzmqgVi5QTwq?|}EWeQH<!Vf+)3SY}Zi<QNVf8+CM_<IGok2w0x~ zf6_&$>;t_tf@Kg$n-~juE1cTSziTpFVmtU6nQGP2k{t+aJ~BQa6b!b1$d$}gd+qUA z^6GMMTMf!caCy`9iR<lf8tp~7#SYoj18n`M`OB?&qk}7JCMdFPd51Kun<)#Vzudm- zcViSB@)a2uPMsy(v=?2aUm79IGx<}?l{54%5Xtb!5Bc%ugujgI4ALs(7~X+?%&J0I z;i`O3o21<$L~7G4h3Q|4D$;V2t}>}rM)W412s7b}aO6}VHi07$$X*(jwmudu1$(3+ z*#mEeG5b8Z!9_VpnM{tP&9`KFok7X;0M8}NJzWhj>63A$<1K$(vA{Z4Q3R!-pwm73 z@uO9dex^)@FC%u7!E}QYxG4c(3(>NhyO^aUbN3rmxJviqQ<QM48J&lS7@LE7U_oy= zxGZ%(f??^}NVT_K)2w3|{{eDfDfnS;qD4~8toI6<rs;{abD6Fr(qwVn9)*hAmR~JN zUM`gz{DzeNfggL$+m<`?sf*kYaDUWivQN&orA5azJ_a!Mi%AV0IVcg8xf@q-aT6Yj z>bj|}7ze3oy0YL6ftBXLCud);GNn5MgE!17hj6CH!C@{DH@UnD=lJfP)ocYw0^R6d zyJ8jYR;)r<JO~1Z4TOS{mroH=y+~}8TTe0IWyIelclPHPUd>z(s;N`ajp7p#8f!av zWlVK1me{iF-+~cm^G&+Sv*ViP#)Fr?Q+Y9_b*kK2y%oCgiOY4CfJ#fZ;irX*J6k#u z`_vzP(D46An_(SlqAQ8;h&S7jVBXj>jcbMDmQ-eqIN#;50AOb}GSUZc+2i^eHRirT zt9ZIK*0(3)cUMV-Yg<vjSZH|&&2vh42=!r%#%ELG)XqNb^jKxYJOXsCEk1(t<=fdp z&J{ABXu)20|4GX2ae_YmRK1mnolgg+T76X^eC=!0$FGAFtgfQX8{Tno@tqbs@sFtp z7}3MjH;C2XW~Jsn2@!v5+D|Iwn%}rn^Y6!($*Tk>(oK|NTMC{VO2t!gX*dD^J(m@n zdz^ahkq<IRy|-sX<M7fIFKcejXSg{kyq-iN@O26t+DFkg$O$SkjCa|PRNJ3!@V_h> z7ln^W)<=0id`T|>qhSPF%sj7dboiYH>cx#g`2Nm-_WqpHjuRfgI&Z|(A2)RR42D0} zPTl1r{k|T5uMde)+2|II4`5Y(jkK*qo11ySJtJ?OYIi%K=e)LH@ktQEw?S{xnxt`! z0bp5*8sNl@e_OU)!#ywP{s?&71gWz)oXy}Qbd?a!9isU~21KzN@jlBJI&(*=6@rcQ zU7j`D;!6TG-Q}eJn<?bK1{CGjCbmSe2XTJ2PfbbM^Y(Ero_`cVD4svN5u+Jr;?eSk zgo;_uOGJ-5w1!lO-jQx5K1yfA4%jHF<GNjkL+J;Q*ng&!shHd9i<k$sKvJGdkI@hD zedM6Z=2z!xO(g|w!l2*nvA{;Wc|Yu=XNXMLd;WXe`t9SCDNUJ<g3jKdhDDl!-VeuO z`FDkGm^t}RPkYTXX(5OT2Qvu;>n<%8dMi=gp3OIM>i={-`@(e-WF4y-q!w|r&hj3N zq-Av~)lR;Btaro98i~q7<Hf4U9r})nkhhO^k5}<x<7n-GsgamHNR`jFdlsy&RBNcp ziC{x)d&hSo4;a(6s)X?wqR0GYUWlZ?ytOQUw7mGbMhR7Y1x0x1=enlcsmX;)=Frqa ziFA8Ij$al`CYKc@%puR9Xmll@Bvw&OJf0f>r9x~~P<YZZ#`DYbMVe-|>(@#-GBAuH zzA|5F@C{Tm6SM#!)rJWk@dUMxSP(cN?{dR-U5z7L^RptXf4bHL^3DT~MZhjdiLf_9 zB`KK2AzBK2)%wckmio^>aRN80D}MGb!WgjcR{{;8(;ty&1={$9ZlwQ@nKQ^Ad99+` zA-xwWnN{-BHoKm4zjD!L;~l2$y}+%V`A4pZtqB+rcsMya-72ey9mbfai`Rq0(E9_X znFd-R!}dl8X^XC3{0+BOSA9h>MDXpTX@(7*cnjs{KCNV4N~@D&&#6QxH-p;g=Um3Q z3WNNBj9iJn4BT>*fnUKWkzh=rk^x8h5p|}M;AoE=e7OZNQ0cx?zEyYg>!^Rd{@3Zd z{uAv{kPbyD<Jq%P(96|Hp-3yC@)Zeuih0ah5^a^8e?_V0Bp?&2!rW>hJWBhm`jwy< zi4A>>9UL03`kNU)Q&ankyT{pX4K~M)!?W%94VNB@_7<Q0PLcsB47}ReisD4EC9vzl zi+;}@LoCFAM0C`(*je()#;<f3_b_q!!z4h1%0es&Q~d>2H+8@(EVEYe1sqQdp_=i) zm-Sm=25LD`De3n7_}X)immp=~-x0_%-w1N9kiLGe#a-n6cToX7nd_ncP7p~mE2kR? z26tmeK=SDwFE0`X=A2U|dUuIcYu<hrUXMJLs_<*;%RVN~_AbKzsK^*j@;IhiTNHcQ z+;SJ#hg3dn&LNp!ZbM=*O3|l4^*KOB)w9FWflmGCKTrISEQaaAjO0)SQk@}cwS`W} zOIJ}n!EhfmdA!5ADasY#&RAKF#s5ms-{u}LX(w-30}F_8#Yw9bdK^|{T@-uAf8est zyb)Uw2zW(bqW&-T-aD+xF6$Qt1f+-{2q-G$5kaYn0@4z|f>=PLca+`*q!R)*tbiiD zL`6lqv>*f$5ETSzVhAntBE1umkZ^XMndi(qGeLddbH4ANGyiybk^A0d-D~|;-Fxi_ z5z2oP8}4jp%Jd9e&$dv+8YthiE1vQ3-oXUBukZUvo#k@#-Xs%m3xSx`mAc^f!qwx! zG^;M2J@V|WBR!o-6SsFX%&QI9Rlt~HkJAEVSJ<U=`AEoG6^(*3#tgnlF<h)szV7o& z)`}&N-V_JD?9xOnTu+AV(Zc#pnK@a3v6OOWkryFQZYHMKiL3G#o%St7D`SUqdO&AO zGthPl5@&7@^Exjt?-{?jj^&(u+|2cpVq-r}Kq-^@59@ZSd6aLG3GhwA!*UE|Hr_H; zG}-8FGsPn3)ElNB@AIXO=t(isF1mHk*(<3jS*zx0JQp{pz?kQ$Wx^hvTK>u=n2>y1 z%~)2e+fz4nIDx?GnJ-sU?up*eV7!~XCoSR!ax^*E3EtC@VS;Y$qTK{517XYn?LbK> z5yOd9qle;k4YyCj38(>UlR=Q>w1)f6NbDgUiwVAH&kL@KUrKbeGr<=<qaL~*s4<}# z2Ml-W@^C)^9q0(<V@_fCtcx^H>%Ap~FgIO;SvUDC7bjMGJjL&GH}}-~VJ$!cwPTo* z!}kulNnT<Zt<1?zb~VBF#H~w5ekyURc%y6_myQj%VXYr7y}VNlb4qtbU9V|X9G7l% z$BSewY#q=qZ*2Tte+h*lreY?mj_4+(y9XA$@RS|E`wC~=_@*0=K@s(m(T{9I?4JeV z>OA{n@{y+*JWYt9EhBFjsN&7%zx7on+?K1@c~QE^7cDQzc-pv00}rYxM&<A)xPE46 zR`8kSFpNI_-Ygow<H5dyJt|ci>>DACh8e|9d}N0f;Vd|@A>B<N^AdwE%IVP+M!9WY z4E=}CDpwkq=Z}C2Ggm}#ZC1ncQs%c3mZ8*8R-K)!va&%q59;Q=nxx35plm`$T6`3P zV-a_buw|8-1D7M5VU>2|F{QF=TIHQOQX2zbbB3FktN}gDfIai}a&iPp9MND`U^PH1 zV7+f#mK<NUDswkt`FaT1^>TM`^%uoTudCtv3_s0+3yWt}FR0)FKY!-Zp2I$QasT*b zCFAyM-@ZRBJwn)@eJyFK5rnVG-EQsijd?<jHadLXv9dW;Ig#)Z*fw0B@fVk6i)z$` zEj!TVX3yOFc|`&$kqk`4{Jr4<x{7V$BHsiI2WejXVnJZFd--GiL0zWHN-qx`onqt; z(sUgDz`VW8nD^4l2JuJxOLI8xc1)S}bBf<PPVizs7xwvhh5@>@Mdj%xRCYd`IwV4U zu^ftE23cP}>rah&(zPc?jJz@uP9<0hir&~*tWV@Wi7KUmi{rI#Z`NK(YGl*m0$4|% zvW}1IATOu+Bw~@)@DZn0Wwdtj%n&#SQ+ZC)xZk?GtT1d$`cw1)wFm0T;<a70OrQeM z%CLz9PR(!l_7p@m1nUL0_+=QHEX`P`tS=)r!=8LZ_{+Ak6I-633$n&E;5R2gUGUU! zihIG4C2Psn1GKp8g9!+s!d-xpL8x+mQw#c%<WKbp%ScAfz}KVTW{?<cnHA@fT17@g zl`TlCj+H-~4=13;2+zb%d<{lKv{ZUK)%C_jG&7HN9=hY{r+e)l=x$fxQ!GOU(gGjx zNgEUAxuJo}qlJOO#xE^BS1){Z#fR5b13+H?>b@t-lm3OF)|TZGON6wCt)czP62*Du z^5PTD=&Ry9Dg#HCWS+t6a4$ih*b!mE;Y~aNO_1Fzv&K!4o`^r{-prK6b-x*0tpxtv zQC#cF)di|bZ|c0Xih$t@bsPdYBC9I|A%4*pU@z8xSq>#-LUXFQQwa9bU&g*A`dE<C zMf;TsD;<vA@|Vz5!M?$fA4}I?W6FpdR3${I5aP@vFvh{&C;N&A;@Ip2?swD|DRTcz z%#;`z8$7ken{`Y6ss-R9g&N1-mr>mouVCh4A<Jk7Czr!fPK!*jS%<tTm<3Elw}mKl zNi#zBmUcM~4iB#dwDv9;aniDTmQ|pREyHc%c#=eXvVx20K~1|r#)+Yo=zBXL&9Hfw z>y58)Ns%krmlHSBa0w5TLF)Q*SNzS&HR<>o1-r_81snX5f~Ll{qT`|v>|4KqcEyt~ zrj=5>T6cHxMa$x<G?u$Mr^A_U)!Y&<K6lxeYG`>eQrt(E$Crs_pOK%%j|dYRbi7=g zc@HNuqzo~<^)S78t+JFEBQa?^m+7mon43_O#C!_B7#(QkH?Rt-MBeY%bMf^sa#X>% zH+68jFDC2S*z-dtm4GuOcRWTm(y1HG{L-tZG<%0o$c(*6Z|<gwTxvf!RaO($^Euc# ziJ^wM{q+OKo6>yO5q3F75_(IGC&$;Pc&}TORc9xpG3#8CB_D28E3atwR1H*Nyv9DL zCrZFjdNIZ=iD*Z-#9_}i67#FT{-h%pyKZ<n`&;KkJ3%GPs4uu-F7DR{o^u8#rafPS zld|Aa7(Zi{w*>a^OG~!|s|H!BZ~LYbm!X@>ExPu-E!}zQ7)AJ^Al#sjG8Z^sCCl!m zX9wRu#j6?3S^{5uXe$#l<|<)`YO<cYB1{;`h)ZX5Qos6i<4|&qP3Wv~->8Ig@{b-q z^^}i7w-Wpg)|(Y*Jp49J>^gs)N=v-kJPY1Vc$!%CYB{Vz>5%!wx!{x@5a<6;OuB)a zw+qHCU<<>`)a&`|tjadTznps>#H+T$ma~{r5%_(pKmv^?WpW}IM)%HE(CVYlNC`he z#)OR?bUah&T82Zfm3`XRzi?Oap<sl%tWVP+&4`*XPVwWLmi<c!!yqS3IIVo**3uPM zajd6;o%un{=QcILC#y3LYBB5HJUaPces55QW+L-FhluH_a%PnCAT_mr<&Ez<;wy`r z2Ex>Zs6-seD2tRoK+4yZ?eS&f*>kE!(R;^<5+2ca@0i$0F1d2*O)X>L()@;e*Ar_l z+&7{YhD)t{7T59~k;q9N3s3u4c|}8&-$kcal<iAVyR}MdV>VmQ>qGCQ`SlGoPDHkD zn=o3{yYj|p#M;_g7P5F>{|3jx#hQLJdCszBc~T!<x#JZC+}GB4`f*cR3%E>b7<4f8 zoNWYcfy`OO>J^QPqqdpyIi%Xnf$QL>UmH-%$|S)kpTMCNyplPQAf6Tk;00%vuhKDA zu|l&KR6LPI`c10Bt8aDUwQhUL@;~%bcRB!0M-LWov=cSX58pH!4OgERSwaB)J$aVT zMLcLD-y1e<lp~ha2%ve@HdMbCX)i_1cu=QX92+QHAy8`Ecfg+C5a9^Q4lCz7GY<nc z0O=fbo&cU8d=y8`0#;OhOJHS3dZW~0lyHbka)T&l+pPd_C|Q7*knw!dNM5h_PR?o0 zI_WP<3Z4au!SaEi=0`yHSuFhLQ6>SN2=&bBL`F8r^c&w&%3wcEiAX8zakZEQ1sd7= z2gMkIErt^=fSSioJnzR-+#4S>u<N<+J86i`WZo|+O2d0w9{|mMIMJ$4KdSS^kBTt0 z)J@JtHOXVD{!Aj9yc5cDd_D(`@>|-B@TwQR)!LRVmlnCQw{3$~`Kx?}H6Tmjl8nNF zDt21HXV7y#IyJK9Cj0ww*{+=>6}0%+yBkcw$P0Jy2<l3f<C#I`qhr=zUP5#{Mt7d) z@0L^_Vy{*n0wEEYioWHirb-rWrr6NJAT6hJpSqkO%gO0C??$1XIe(zBrQ@9ypMH@w zcFmR$zV&3O@7cnT9^u7%I=k69^o|a5fw)G|7@rwwJA7e#bxeF(ulfA@dNX5&rRRLv z+MB?We|HjGI;P6<RoNfH*g-SliTew8CE@I)>&f777H;ObxYoI{NpQ&T#e3k%p$iTq z`N`F`z<LH#5*-9|E8EJsm}1E-&LLZe{TP{+<$xP`HuUf<p=b!{hoU>VW;^6E2p<rV zdrZV@h@j7gZOMia!{s6mw;6$p;KSV6bx!d%O|n1nn_3x|%$;LsqDi&&Eso<qB={fm z-2frlsQ7Iz%cyv*K)?)fyRHlCeJ*l4vezY<aYh#%H*HDfGF5F<l&fXjzW=&rFiZ`! z8F?6rpSHP>Y&2BdBqtp(*Jp0R)T9XARNGW>sVH!63ztIgkh!vYzU|{mwEhK&qppm& zD&2<`!hh3%?{+bbu!_Z?@`HshzHU?(k@LI2<D283b!|Sc$CiS%<)>YxTV2V#YzykA z#!AAZ+Q!pI0X||wRs4^?>IpzjtAPTij|ttS&-Ysbbw_TL%Dp73`;m6Ac&VEL%ynPl zz%<fl0djEOgKs4W^n-u4$y&R9d@?Cs<`K=D3l>S4`Vb8Dk4x=)b+J=p(t9VZuD9v@ zV2PXV(|5(i3b|PGpQ`I*T6}DmKWpIr8*N(doe7N|*-}dR312B$pvz*)CtGmU)2C*y zfD+gIPT#HLHq|V&7{f3KB_BpF+>u|&C~w_C3r#-6RnPf<0;M&0XXKsqH5`WOzX*Jd z@*to=)UgoplS+|raQD)N5w0ZTPS#65rdI>21%dg_$MTmYYCw_%9-6%F(YFTdxT<x; zCJ`|VB6V6mnfvf>H?npm-LA>F?vY<mAV3QYyveveNXtU_fD*RzRE_)FIuw4QD#rCR z8J2bg>Pp>nyj*uZwEdxtkvj12QvmOU-Y||>29Z*z0I;*Fvye8N5?fZ+JUSR_*KH&b z-jUjG1n54n$9Y%$viQlhA#Qk(59oI#0FKpt72Ye8z)?f1k9`n`A>;Y{ns3elJP`!F zbA4Swe5pU+Wm87y2kNTrX4#H}j2n(Wm~Ej;Phw<vM<HLPnmc^oSur)q#4=F9$@s#P zv`_-59NtVbGd1*4$5o|G(@Gp9kyf04Y`IJx(WBnLoppY%qwsbt7uc{~?q8$6%=VV+ zdrF6sIuHU;2LifM#w1I~39^=Ne!Am4ok3*`@zUZa`S7A3kPxy68$XVitbPknQg@mL zezczGpTfzrrdIPIxasON$Fw1utu6hBWJBnQV6Q}~#Nf$DB=Aou4}hhc@#!&OSh<qs zu@i~5;laR;s>-DGAFJ;{(+&rzf4=6Ei0XU&9#nNhJAJ>Nj;L8-4hPILbKb)TbnjTT zF<^?;2mK3{)S=*Vfrp@eH;bU~^(TbTwA;nPEZTa#P-E6r-!kqDMK7?@a1oq|@yf9* z0w?|yo8ZHI2(vUWlA(3sovjn65@N9tMd@mR0-#W_{2^%Z_OfBK1-`RK7JTSG7_UX` z_dU0lClp-3Bp8V5#WUYR4%&jA=%1WHzVF-8z$RtjA@fN1th;S5|FbmFIinML3jw_S zpVxpRqQ7h2(H%-%jCly^N8m?jm&Sh++q}!iS$_?jvz6U>axlWWz6%Ie;hmjm8zk&4 zl(~obY1c3fE_}eb)@Qit4O(;Rhx71FGRmphTel#Xrb|IgtuK@2yFZ{11~8uORJ|=v znt-C<BY%>q9KG^dUKT{OsubO6i9hrGVGFf}y-&%YK<85+dO01RfY9QlT^JpyY-m_+ z_cO%bG6(dRJ5h6u1Ax&`I87!@s-}z!ElNI5KsSk>K(|w+GTC6@t6vEN>}pf80YW~o zPr$TMDO<HT`q%rkqjf)#`7M5`8Kyzt5Ck=Nlk)+sWZAtlsD^R-GsX`%RR8LmVntfg z=^?Ia9Nb<;0VJph8k4qFiTAfS{6CuRe=qW{-2zfl28KcR#~iB>nmQN;cOxPiJAtEX zdqj9SsoqFVO{<+&^7p3Lh7*{r|9gP{!GFxufp>Pi-x=toBPOP%xV%-G%ezQ}D=Eum z8qA&?jXfom2>W{-@Moh=_ZR;In*P_NJ(L8B{xf-IW4Tq{2E?OH3sw(`5YPDlZGFk{ zA+O6udrWw#Z#PZ+eJInmaH;9C%m0(__>VawKk))<Z-nAcGaG4~KV|^}1vo*P>1Txe zL6eiku6hX4w;Q+|f)79sUGM)(lU6a3L5tQFsTOb&P|+O%k}wZw3~bo_l2)nv>6Y-y z^I>3USe9(iN>!z!HP*Va^KjZ*E7sqly}$*a%vfg%=V9}qjz<?j@DV<bT%$?nr^R{^ zXqhZHmjE$VU(MFHF4>?#R(aK!?9Ub3Pck(B_{nTP;HG`0g>vr)Zw~9~hkPp#9i>H5 zK22sJAglQ*z=(X12moX(Bzz(;JSzw&$dlOT6WM_c9CT#y5nCQm<*CZv<J!N^<ByI2 z=DPS^_iF&S1xJwmG<w>4-eUjt8pb#?hb%{$N<Lihe)z#K?XqT2Lx#{>PP+FGM)^yC zYyM#K7(iE>nVTO`_6MDgnVtw<&FVh;yIh#02I$~de!+w1uL1w-6eDb}mKK3t4QNi0 zYGBvyq_g#Zko2!K*q?s^=dXYr9h{u<Ux)iUM?|Xd?<ojZ@A?StFY(nY?{xZEHsGH} zCGrF5z>q&D(0irrlqQ2Ow@e)GKfOb5Uw<ux6>NSdH$7S<Xal~yGIKWlx5rZ90ZZGi zRx3e|BJaY%Ow-Dx%=G4>OI055Li9Fzk0)r``TLi36+1S0!74yU&cv7As#?Fzc1|>| zOX%r5L+6beBNioa@0&Z(vfN7+39hKeu9~0x3H|>0C$qC)$xZJ~SMUG(H?I+v0T%t0 zKMkz$=2U_mi?!r$tAi0vHcjDymbBjj6uQLx+f)fO!};6nl><MsHh*VTN6{cwvpD-z zmktg8eeYZU#cq<BSyUb`hZUpuoqw{EfBWe_B>JVC{)a^WA<_T0g7!7JW@^f<=r5cD z;id_XX^)-%jFLb32xdC~fIcijcpRel$^qs-u{j51#st*@W_}z^)crRY*b*TluhP`X zc6u<Ape>4xDfj3uwzTDqNYiw9yft3^E8P76aj*(h{1JLKV3IwU(XYzS<7ctWf7++a zGk~ghdaRWa=&T@8m9_ycdK;iig`<1H^Km|<yXn25WJWVO=NWQ;R|iGD-o+D9bpmxo zht6a(-N4i#%uosmTay2u_6Z^yDzwFwl#0_|oE@!t*|Rp@)TW}=(qZ*?N7b}jq^XmB zNXaL9uY6|%iT~Zb<=*m}powbdHPG-ikIruBSua2*MrMFbhTcq0qw{V4Wp_W-pfw6W zncIw4=X%py;}(<Cogf4_+;i+szVvAL`}<-9;VEDcB@+tJ^i0a1?&V)d@dH>^6_QC` z=t;v*r2gN2k|_al;Mk8Mo&8sN`?p7bPOs-#V4w^;iQnaNx?~2VeD@W6`@~?-K6$Gr zev?f9VWNLwnScMF|6!t^^Ys5uNVIk|5`>rqrQO%35P#JINJ~2wg1%iQ9{5UsZoW3_ z-v*waI)CG*{|5J8Xq6#Ax3_-j?B4PFL+=5iDQFMvK2PrzRkla0qFGG7Qey7EkG7hD zKsn*_FVb^vYSI9DXDlCZSpHpvwG*T1q>o<3Pw1`S22iVgXTF@E_ZwY4cy6xAP<-$< z9WnZ;il#f3-8K*=_7+7O(+iXzHK(cQ)zTC6=t^(%UB|&_^T}lELOS>_^STB^Q66Bi zSZf%H=CT(fROvb3boK=x_YKWuPpM3}L`NUgd}T2)F=-SX_3TCRg8)<pk(3`xe;7Ke zmZ=09;>^G;B;vPUp-ak$8fLJ2eV&e&bS9zy#ugL)XGOo(rT?MPe`xgopsYpWnL*y? z;l_pDN`L7Czl+PFnm;px?ZR1*-UBwkGn(nCIiMD>n+Bl+Tu8^?)<||cY01VRy&_$D zuRM}`2NJV*r;98!fPP$l)bF>wj7SY5;H3&zj`*kkJ66^x0Pn}<b9IhBy;<oY?ooH3 z&`Jc#A@m6MMgnLUw^CZI%I}%@p|L=5Xt+2}Oa1%vk&c4ES)K_s+x%T+U?%{q80Qxr z2k33)>uErSi%I$Pl>Wa@4*rs01zvbMZ-=LIv&cJ1b`a@!ZDwta$Bu!XSOUNc?u}zx ziT}-gGqdGcg3&A4W$95WGI-ay1)rO-9ZKH)R-kdLy^T_H0KJbv7r#K25TM&pRO4NG z35+&l(Z~jN&Hqjch6<=kPEEe`+k2zG)S7QJUhA>WwWs%a=)XgMN)oUh=e(VE(qqBj z<rNy;FVK$hxCZ7&|F$sEWfV4^AM(Iu$0V!#cP6Jt1etsw4zKZLs{h9>I|d9ybsGP7 z$MC=Rmv^N!mpuSCwP%Z+|L$;+|6!tEQSkpsCW_QB{WA**dOl`wKA)f`6m*fn?f_s+ zT%@xJJq#Gvw`)p}J;M~+5irvw#uH2pcq03|xw=Rdpmo$I)EdU=fkK<L9eHi`Q)6Re z!QLw@Dq0P{9hK%0ZL<bx<v?jedZ>JKf~gK0Q<4qNTZKh+tJB*QJ#0z30F<#`g{8-C zddxM#2J)Rc&Fa5TRGP~k%|>tbgtI_heVaLd;3v1?{1X;La_t6MD01C|uD9txbMrbo zzR}y%KMnKuec83rMsuDEtfo_l`0g1~CS$|jbIr(SFzgX}JD@w&QCYC#j;Cc-^xkli zdskU$A<h=u1}w8MK{GLWM5aevnMi=Xhq~@p($mj1n}W0=i_Lm)wWqO1f7RZfo!I)9 z8QTm23+W$?qZetQHy3^N$;M;m1h(B&6}bA}NYc>zED@UeU?7d!i$5!)qi5m#%f2-2 zk^w`=FZW}W>3s;gpKgo&YJh<V%*R24SbATEE|$<+V5B3cmaqC6YC7=WnVjB~D}ZzR z_`Z(P`|Y=v19p=DLRL!(^VTpr=I9o*pg-k*n20Vb|5sW04-@@LdVa=4M+ca4b8~@` zJn!?}K!+aybNc+_GJpEqNQZTQ<qCuA^}rn+Gbx~_Of&5u<Y&vIir<#f8eu75^mF4E zE9odEyS;z9@g`7VFH@V~F@*qQ#P1LM9Ym=0A9voQ6Sw6_Dgzq9{Gp}D@4}KyCV0WI zFPC)s-_gmA8&t1VMa>1_=_o@w^aQz%`@r?eIGlBdo=x-a2{5IUuMAYuTR1&Yu`{Q| z;+rzQ)2(>p0HttDzha;Ny~<zZH2z+pR*>Ia>F-_yVjn8qs;R$Ur6vGM`IfHDNLtd% zLTKtl#>dBR*7Z~B%u;NKFwEqR_+~q`qoa4(+0U!j)#zTzIkbOom)$KTw&P}dc(&C( zw2ZvTzklydrM&a^?;LF0Ep=J*>cPkhhgd5aPBBKY?$BbrbMu`RUq^R{gknoRq$8^} z4c;2+tv`@MvWeIefF(obm%}ok<xdAfcgDxc%gOB=vM0)mF!N|KF#i5ONt`^I8J0eW znAk)k82-=yiLr@bSjt3PW4-r(VKA_y8Y?hf-vjAgTW@NRonvEY8b22Ufy6{!u(g3z z_iB*W!$>#IC;JsA_)z9@$?dSU$s;<!5YI2`Esh?Q$kx};tc*jQYuRU(*FIW6CVai% z@PiY&za&A8D+`+_qBE}ZCp3P(>wR^#=VLiJxgPfombd^(TvvcCI;?_Rr;Ngjz)Jhl z2?4iObRg0&((pt157hm9ns@~rBwsrPLzO3(hEcyP4%Hgoj(Yugpt^q+LLOazK6ixV z*LcRiu}BvaZGOQQbNj466}~};L2TMm>pQ-tuFWPL&5gd)6I*nEta8ST{A}5KVz#R& zeK>qFW>5OHfq+hI-~&oJbUs=Gsy}pk-!J#6DXGa&!8c6!$%AbiE^X&GrE9Ip=3G=H z_vlSY`8JmI%I5(Bjcf0+ICfqbbIbqgmb*c!R!HwMWFAq=bhRC=G%Q<%mDhEbDl}P< z(abZ!Z=B;c^3*2!P2nzrzhqia4BU{n?I)~lCJ@_}v7!GVj(I(S_@4340VAJ8+>T{G z6!Z2y+pi|jlhI@#etdo+YD?CwvFCet)qd~3$dP)bx<}h-0}hI@cU@iz$H327XBGa0 zUVjSLLIg9LDD?T4m$_R%8D`65DYgHC=y|Fnfx4h$I}=V^&dscp_WUw2PdVjEc!+mK z^ebmUH}R$@{JhP40Cc^#7he)Hty4`3DUX{TJ2>00y5vo)poV@k_nd&jiA%jrn?FWq z0+IsG9xKAvoyy5|I0BUH4ebuc^pW8h8*iQYB1%kn!fIZ~8>-yC-WxOB{-kC8P&K8= z#IC}=!kqyga00>$C#jb~X-1V(sO7yq-7D9<v<E|Z*N0-{(0)o&J&26twGb6@`3#H* z`L=8ey1j3PMB8Ef!dFv!)6+IX0)7OJRh0!6eivPAN|X;wOiWZG7KRrT7S=a6RxgP? zQkNLf{>j(<69M+H@n{N*`mk?}RQhK$KNy*sntI0Db2;e#(4A{<`+nZvVV_<Jc7K_t zBJTO3Goml*&R8Zij;AYTlj;$$B3xlH7srs}>DSpvf&D3k-W3A`_yuT{u7lBI9g>I! zPUwE_$uo$#;P@vL>(xhY)UwSuGE9v$A75tA_2z6c#v0WMspM`iTzTEYz4W!=<6M(m zP$4EU7Com7rRrX{?R^sJX^9~<H;pY2$ZRp*?35-m4+iVjr>;pR^PAp~&1;g3X0R7p zPP4en0Wf9f0PfR%&aGB>%l4F+HCU<stMpcAh3$wmL5IAS<kySX?3m|LlPGIPupyL% z|EdKzGl8K%o=qc}pTUy4;Pn9mCAXhs4UAiD&Cq}6^mGQsQN4Qu2e;G!3k@t1FYMXh zy1vy_k<fCiHmUn8bb^Z7oEqlQ-uMDd89MvoQiWd#2J-1{8?sNH(8Z9y5vq7Q?M_sd z*ms$a%3O}lLn94`g?i85YlFJG2lYahv8j#i(nBMYT@tyAiJOb~T$AyEXj|Oe`R6K? za7T|0#i!Qg3WRU+YDzw;8#AkG8kzRi$I43_C%i7~3)oOih(9jc6WXEjWWr@L)322^ zN^!xxzQEntw`DyzPL$M0X;1mSurNvt7s>cTeWsO)C6@ciM=oL$FK=LN2j;xIZH062 zg}0xssu@z<Zw(8bpX>SKk-7@Dg7sH0p+JKnuB{n0<E^HkdS_eFhG`*xP66rV-B%}0 zuHJ%Q_hCEGsH1Ibx$PB1Gr)tvn9OG!+?;TmJ0jq!%|MH|`u;oH%hKf--0B_~sNpH3 zG?~Nd)dSJJLf!uFhUI3@ylBAYt7Y3+lub&@7k-5s6N43yx??|b4v<m0*Ejn<)CJ6$ z)MW-xTzE{vk-T>wcTyetZ)h01JMwBHj3~&n)Udn{P3_>CijHbEb2VhOVbw})d69y5 zz?>&r_}E6wc=(tj2e#sf{L*?|YNFHzNf0;tB!T6@>K#!mp`;HrFyHHZopEcq`#YjN zXJU>u=iD0J)Q?3<1<gnw(9hZLK4|vg5>h+UQ3FYORXQfkUC0^>V<^epBP9FqF!M>S zv4D6zRD^3FW=)y%cmeBWSdCuKz?Eu~IW;Bqkf1eFLTh5H8$<h=7VdF6oM=w?!t|<x zr<hf<WWhtwWo9mB+!le&&-a+AXN8toi)`57QAkq32Ar68JC-7B4NKN4tCyViEe_1( zSz39tEAegPvUt`0{7B=hw~y_mKlfH7CQ7-CT@CJZIB;O!{`nacWdJ`<XqbnGqR6eb z=spS_{`A3@^0A^)#YY!;D%56)8o#2>TLwmlPNuD&(myvyB85o4xVBkTJL3AotWMwj zM;FxowZ&Tu?rxU{ONT4lF;2SzX8T+oTS0{qu}ZlaA(kGKx;pb!3!n7cYk3&O9^Q>r z#*ea|X10^5tWUD+zi8i#Yb=SHsAh*cPv?_if>(^Zi%3hWu&#cHdk6+Oa;nHu8)abG zL2<&xa1FRYM+wrN-BZ<_F2xDO70Kw`R~6M~2wgWGcl$fRD$9)1<CoMl<V~ye%Qh($ zva^k?63cI3_oS5dTW8(o^!WEDhC9h0H%nK8%DIwBq^ZHxCIk-qWR*pfN;X5cj)g)= zJg398;Ty+{ww9njtt%xC9?gEXde5J*l6Yj^)M7YhK$k5$=Yh<~22eyh>nLBK6@B0z z^hxHICJ?22t5d0Do|G^N#&2&GJOsW!B7HZ8r!!|tY8A`OAbBC3)ZPhTE_z~H?<XRM zZef;|WyMa5{4vAyJzyue)emC9&8<^_PwRo)@WVC31K}r}5ik;Rdk5JQ;W=m)E*Ci+ z-!?uZY<MhWx~~Z)#DEo@J`WAM9)1B)Jc!|PcOt(mnJ3xfy7j3Bu;DZD;r??b8<~l+ zd2_vnx2dy2#qzMEjNX9R(^pCNzl^@-Hdl-D!?j|pb);*3CU0?z>EE1gf3Sr8K<S(2 z)GnTC;MJr8t1$pdlqBP&gP+u<mfRCvYP6NHdbp+0^7kL4G#QeC-?$a?)f^OPxcSLq z(JXv(X{_lKLpOXyi2Vv>P^3R9+Vo~!kIinT8EphoUMS*`c2Hd7J1hkqv*f**Cil^W zEOe)3g(y9A)wP6EM=xRaRiiU5det&pedbiG(4bW^S&Bi3rGsMj#xHTzH3Va;BlOL> z8ftNYR3gW)uQ{_No$967Fh#(fKYX0?e&KC<HH@3=C8oE-yev_Qr4^Ndvk!O8gymI= z@R4F#N`F{QufZIk`RX#Brq0WwJCf37yNSp#7M6e*57^`_XvynlS#p4dKbnJIB!g1Y zgtw(GQ`8L8@o~nN=Q`OjI=^s^BI{*=73k9{pLuGlCxbpP;cz%L_I<<{=EuctMOL}B z_QXQ&2t94YTCg#HgJDz~lJ!{`#vm!5Q}b+7Q~5!s+#!2cLKn}q-FGdyv`rYl=g5jo zt&AYPQcj#sV&ONPXd&p15AKeSGL>T03DtokEAP(qcL;7jXW5|`OU!SvUa8XcpPSq` zLrK0HgH@V}L0`W*1vH_MXfrw^=km&^^en8Ia#cU6u_yU)A!}cHTvpKJEy$@Ji(rGB zB5||RMShqus*CJ-L=L5JV0kHB1U9M$#~`*pp_Iib)+=r;V}ElB7fu4d2ig+sXhio( zbyvphQK?#82%EZN<l(T5IuZOlXgIKj!bXox@T$`M95fzk%y6Sj)zO5dIHygGy=Zd3 zDOS}~+s?ox@Iu5&*TTC;P0_YJapA!Kp}<l4qpHcrd*?bCrcS7Lv+Ok`rLw#>9k=;R zg-9h8utve49%GA>K{F>E<}XdW>d|lSiA9`&r5GJ3cem>8M7|laAF!Q`VZCgc)9NJV zkc`2`6slMI-d$f<X;g8+MOzdYr6rIhhi2X8NoA(p)iH^X^|BM`Y8Zm)_*ckCQqBlf zjex~@*M0A{DQQh0Zw}k`V{VM%Fza4>f`v{;7o@8C9@t8d{*o|tIbbKmxI^}G(S1_i zX7JNPK{a;KrYMF_ake}41#0HDX|-vh=eRiSa}V?)^ra4%GPb*}S_C`oj(&zHyvqhK z%2@<I0Oj&<vQI>KyVjX#;=kQ3w_aS)Qr-6<?e1O_gNuNs6<4Lxw8x>aJim!tv=<jo z05%nMYpkoW<B6wWClXeS>seSnaj&Pj8ZvF<Cpa5#Xq3^<yOB_o?lwP##`#^OYD~tl zn$4DVr;*B%?nGr?_wN*hg$wo<e5mUvIy<qMu&h89CeCztm7~dIdti3VHU!?K>`V=h zI$aBcXE=Bg=kINfOc1n2-%9ze5PVhwZ&}rI!a6&7Ot-c#|K=Ta(Yzz%UPpEMTy{0` z?eH`nkLMG(`a}PcCevqN<qzCl$X&dkI0~gA<O1PKG4cTh=iDu7zu$$FUHJjj`D*j_ z&CwKHyH;cw@Atg0X!e&xh6%SMB~JE-(}{)i+j`B}nxn9PVC24@<a!=5?JP17rM!Ba z3r}4AgW$~-)2sNM<!}ixrgNyX$Sh8GpYhnt7xYvo#`=^h_3#hX$=rVL>Qam4MoJWs zI)y)9Xt{>&8zIadn~1U&=~%om<l|c6c0m>e+gElKr&GEnDK<JqY!AObp1;rjd$et$ z;rz@D4pCTjy9QHJgFKe4E|R{C4=<$@q9rB7MuF#-stJh`Ew-929C3u`&&un5{sqj} zplIgTF1qv+U2vh%HznIabRV;Lm<<08R7`$Q8cTx7wh~BNj@WluF$EE)@0b0Q<tP;K zh~6^C`c0raYYM|8M!tMN<-X{bI+@tqdVZ{5=JtM;M(ILBR-W?PiJk**_``D-gDg?P zC}Jzqhq%FaD_;mu&dbh>tFl*h%HHu?nrf$ZqkDO&5aTfyF0J08OHxB5j$)xT1k#YG z&-5zp_{)&9Mk?Kf{+43qxU_=lFUy(Leh*-RcQR2FC-T$k6|KU_n?E{@JLje>@>-g| zpwq%8Ox#_3;R2_TaK3wjVxx2N%;7PANgn+`R!s-5Xv9Y8cxw9}^t9l@TN=boO*&q` zQzITWbLwo5l|Y{bmzW+?0AAebYNwp~c>N!+#0T+~H`@%@T1`JTlR8+|-(V6KZ`iMS z5*Y@>t0r#wAT8$4M|^MXo$6U!&f1*Yu@O?t_|?=FlmAxDGmA5(X;}(a;1(pvYsIcc zW~f;4^sm;5&v>04U(q`5RZQL3Ag&(9AWS~Yt<AoIwr>O<)UX`Itl5(@@;BXL0-R5+ z$s8~(N6t_T*n-{1rt0>yU~?t=k^0^J>7{XZ;8_E@;qsGxCRb-&AUS@8A?&rC6^W28 zv|iq5hRi(==>tszyM5C)OrcB4ADf#m5foeIe-c*vQ~M1*Vdv2dci?NhPB-Ee)CkxO zu`~n47wLbfCsV>DhhEO`u{0oW^MLzC(ejYzhcJl?FTyLY$5-2uA^GaLt#5DIStof4 z{gHZm49V<dnwJMVLFkycpk^?3g-ATVFRDUG?5t&NKu>gmMkq``=Fys#GTgf6MDF<+ zM7(vm<NRsAav^Fo6SCguKq4&1y{NaOU3Dcs2?6^a6`MsyPU-UxmW3+i+?|_vVOfH1 zBs)9`KX8WZ_(_dZtgxjTzp1oCnv%Yj)r&G#>kgi3U6~2}7}XDZ?E!@l%V%2~_m1@P zPS5q-Z^^OGJ>#Bg@Oq>`NI`5gi(D>o@aLvOP$i8PbA88s=^D1e>6*JgR-sU#H%1&Z z*~}sZ<!!1AU$;@c4#|`rVsJ%Z@2upQH{JZuk%L~VJiMWty;GI_rlZ)X9P>oIS7GA$ zX24n}O0zsk43kroZ+e0S&Es+$AG#M7O6A)({A)FPvvh-u3R+3W(=jqGZWDIAeS}ua zz3#|_m5I0svQjB2j~aI9<9?Q-na>@|HZ!Z8ACKB_QAw2QO7F~~{8BOZA6R7q<;kXZ z?p;&&ywvhZo1)n~RWjtpP|sXy=RDjxRC+qNGBC5TV@(BHNg$ixYDy8w9YZr=%TFK0 z6V=ZAu+5n5oxC~UEwq8hRP<1mCUEl`Jtn8tw8r8w(v^xIf6Z(gwCL*Q{`OOJDaf-2 zcm%@QKHZtS?&X9x4Tka8dDR#k!p3VF?;cDuxgC#oWl#up5{!<Pav>el&v~VHGtq#} zT-zls)Dsb{h(ktmz0g&&)c$H5RFo<YRqo!<XSx};;9|@kDs=5TMCa;fnf;ctC>N?J zuC248gi=g4su#n=Fd@1#o+8NA(w;CP$@BWj-d-ecVIg{YqB%+<3mECP0>T5|?&g2< zxMb4u)$prz>0J};x8|$Eld7e)t1^2E+>hksRRJSbb=pRC2<K*!tzm&`M?Y>+tM>Bm zr}PzmH4oUCzxHV!l_#{++}!-aRl5OiYp@~w^DsnYy)-aW7h7Zx(lt{Dbve4+-q8fo zxNlu_VGm33!Xh{PMiH5?xdu6Lg18==mH$J9-K5(tp-_9TvCb+mo+J}O!{TkS_seKi zOj3|s``%e{Jtv`m1R;$wR_z~XhhX>jJTH8t3$aLEEzy@O+3duc`pu5p)J0h=cG*JN zxi4^|h`Bw<%zKQ*IK&F??#V>1IH#MQ*aum!uDrWJYG+tHp*cnI*kjr|m2<Y~eNIu} zv?k6#DPW(&8aB%G)~0b#n45R=2_9Ki!(}%v|AYW8Vud9a<$|S~{6utL;Rw&N7w9kH z_`*B%uHM?E-A)5ng1bOGsDC3PUdd$J$9mR!N}$Qgl}B?Mt!vi1Pb6S6$6ia^j&&Z$ zwz~5R)nLyDR7023v&Ah`hnV;x=pwS8i7$*N=n<3o`nn{vcOF9SAh%J(cgNq^miwTn z(~oyBjrknvT`J@_Cy$=C2bwE?A*U8~-&84p0)WlnOiR3v!w^=k&Apz7B?$Jq_3-Z5 z+;K0iY(272{)_m9q5~zc@dO@wq=4zTUe7)4d(xf`<i>)BZW>PEO1EVwkC<jVmc@#T z?bD3pa7r^(2R3X5Hk}+>9p|!K?5`YCwTN1p0h;DOj8#WI=rigkxYm+UD8H+%<Z^~B zQSU=N7Sc|f{yd36l^eOmpf^3o#_tCP&J~spO*9mY-Pc*W5);qkTR#8Nt>Fs0IrR;? zx*GrGVff0yA&1~@?q6n=f~o-eiOlyYDA1Q(O@>IF!-Ly2RZnxVUDB@gZ9U(1g7a+p zADU9IWD}8i_e`E8x3R(acP7Z&FV83W&WfKF=+>7#xJ~4F+HJoQp3G=v&SJf~`kv^* zZ@GS0jRBO%zOJ&_eCETM#2C^axJ$QIAYlcJZIh(Qn{PS>oGE2~%QKneqd5iB*aaWt z&M;J5X3tuaNoc3rk2s~5l;hnc$k~}QY#qdhYn8Tq=7^jJ%i$jQhm0HZl)kf!6fe&+ zemvD4S1V1!LY363RK34-9-Bb+iQGxFg&ogTN6e;Yy9VM%L-RRBas&Cb*Dwcft7`<m z-}wunjXDB^_G-(%>a$z=mM@vEz%FYyke=k+=4B}6$4H84?R%BORHJm=?G`#NHaGnb zk<(a)A-7cnS3lm;sf7AtT1q0KR{39D3(ME|IwrevkskZvDg-YiXUW@QxO#L?`Uw4z z$CO%syR|&;z}|<eh-fXGVQKlsdQP~bR)?I`+tyYle0a6<?4?@+=4|GtvU9z%`peNb ziRbrVqt3nd?pQoM<X0WI#2e{md$#t*rZ3AZ`Bi1vqbAf1=ST3!JNV@&l`zBLkSRyp zSWzu}bLm!m@G=>aqn<yJ_DGhr?$>LP(=2{c*t%nO^GjezY5k14zu$cQ{_2_2b)O-( z^W-%r(Sg4XovEt>cZ%D)<;O9)?i8?iYxwxzqtX4k3p!9Tg~|hIftZWR<$9%-+1D8; zy)WY>#UV{^rdZpKc3T8rDZpx86A&($J)?!3G%M!8o!HaYt$(XdA)}zxpzDT(m}T`# zpZwb~B>|%oxs1byH!8}wiq!JHC~HmW^5b*d)^AWf*RfG2QL!)JX9fG$UVNr`4uxYK zcxl}F;?#w0ZwNY}K~qNfT>eadivCSZj}2idLiZgGGLRDBJaU2|g=N(A^IjHgR;!@2 z<q?{*d8>Zvq&YNTXQosGp6EDFfZ0;yCYgw9<_4HzKQ#_9ogoiA0h?3;37EJ!AZ`B1 zl}LP0QHtW#MgybbB|Q0<$5xW}ccv6B2T}fd4a}o0xpeReTi)Fm|1L;rr?JJ?_tES- z-BRN?5ZD`&6a!<O>OxL4;^n#X?{hg9{29BmVY~PDPtDh7@n2MvRO5!6koGKKg}<)G zo^MBTLr&&vSyVWwCbX38i}Sx%FgxB51Lg2>5_84`R-$vP%Oy}F$2g8Rf+iq3esAU3 z+ccLQsKpF987%nzXW^%pz2P?LVn~=s4Z+=T8!Esmj%qFq^@iNQH(Q=4h_Ajz$c0!U zDjer#32iUDIpd$jBrDJJ<d@0q$EO32&fM0`&NgUB+JYRsGEv6$UVrVr=X}y~=X|f= z=Ey@lh~ScEW6F4!9tz6u6muk6Sy|1z;#K%H#(}!NSP@*9g3wl!1{n8FF)VMlQVn3Y z<!4H5r9yX~B0boeP2@9}dpwRdZC6GOW(CH=$x4+|FFP2nI?1+Lv2T(%-XKXcIn{mS zv1#%H<dS&>R7l@wsH8W*Jl6kP;#;>8??U(b5R++1_pDs21Nq0?t1XJK^NQALxyzC` z^5x}MJtgV-7=g0pmcBxEuIssQ!I_me)L=-G+vaS&S@Y&q@9<Txcw4dzdH5CxA=pl| z%Ao4Q{5`9k{Rd718?$DrxttYOc0lABzL@&5#J^IcGI+Wra#E;dKwmXLEe8L7mT-%U z%RML*kldoj7h}Dx{%-8-d+H}BY@bqs^EqH<)$?k1#xTc@Ifv1875+>9esB>a@xs;p zzMR|gN#Pd}+#*dp^K)|vIw2lGU3V9v*^AE_YdIdZPBI`P?!=>biYYMM-T4bf!4)YC z`)-4**WD<fM9#-HulY1>=)`N3HG83}V(#jXLH4z!ey#4`*WBx1CG97&>b$f=%?2Y- z{ptJM?jil)fbT8hEAoW1_C>W{u|?|>B*nRHNb3b?F2rNPM`5$wY}O(kmZdf9_O;%K z^tk1j{kB)u{sU9{mkAcPCRL1@2adFO%)(1`P-Xm&HF{BT{&U+FI}!yP{OqI)bg<u0 zP^OpjGhc#pXGx0<t)8FkiRH}Eg#`s~Z}wec{Z%w_kP%d8gxAFB;I?$sgv&K|L;B|l zQH{5x5r@wqEp$6f<Y}6*<fXeLzZ7cfquZ0meoH$%>!J#B8k?~DPirAvU0MmBAkG(f zWI1>PmlN;@VeM{J9g7aj&f7xFE#oQ*nLbFnD^><L&UOW>z8Nm@tiQ{X)xQ=l!e9-0 zYS}{rWI;VeVBjSaE@ckQv!QZ1v7Wf*&c=~7bBp`PUS(hX6%QhDwBKcJrJIC`514tl z&m`x$>9lDp!=$P4B@}1O1vPV)5my)WrKgDCl6b%4s;C(D8Sms!5%qy$!farG;My8F zCuH(2w^$RWK?W2~_2gS3+}}w5@Mp?>E8R(BG2tj0$Q_G$#I%(v9^A#zbvb6|y!<8} z?*+s&^kqaM>(ZOq(_)u<z4)V3Znx#cPK<^_R}DUyRyYausYygQk%y(PKj=Pj&Z#cZ zQu^=(+m}4f&OLW7no2zk5c?)|Tb&(P0m!=^V`GOmrf?vMX_*7lfqg?<!aYw{>#dh1 zkV7S2OigB}uPpnc3s7Zm>~mYPU4y<CCVn}YS@>8cObw3;pBqiXF3jJc<_L;iY%O-# zym;v)%(b_~_-p3;6R!cwz{RzpAMtHL0pnAxSp$CE7FKvvzZs!n3S=F=5rE+bo2<Jb zqZA}JtwgpIRppbE$G?N80%+TB^WcwK2g*4|2cpWaA#R-pan$_^Oh17*9%Cr=OY0h{ zmAZl^EuQ$g+gRN?Tdw<#rY9M)-`Ln-#ZcgmMb^+K7ZU7V&>{81z_JMG{??@ZC<MkY zUgG6jy+Kam$t)c@m1eH_j#I>ald_(}b8RJTdVHes?%bQ{zSd+ee17Q*N{W$lLG*#? zw^z?shkrx7y*7FNa0zFG9@A&63ST~7MSsC9SuzfzZYT79Wm)2LA-ZBfZAryxv+%2B zY4{CvmasX)363eJ7o;h3e78wHtheBHY^GhACtOHaP#H>ygAEhxZp;<ry_oBpBaIUB z=h(S+1x_DLO(m?R&w}hMUrb9yZ*OlPQ$4R{IB{uAFI-N9UE~5=GWS*{$X{-TdKk@E zf$!YAWxllLAm2S&#Dsd;@iN5vu$hY9UWL4z`m?RcMg}V#0(Z(QYfKbZEhr|1lOODn z`G*-)v!6?QuJfecUbxXxX_v<(0ww=IFTI{6ocMT>J&)*qLR_-7P4a2lU$p=o99i*j ze(el(%u>QY{#Vn|aeeIPDWR|9ZLCjni_S!POr8?l|9D3ZgUfCb!bB<2nRqlO&qoou zfqwf)9baBw!06KfJ7LWoV5Ri2YAoD8>29nfW1HN5sLH{Y73F?8%ZAlvn)Rhu_EwN{ zkj`sU=r|-L+4k|lsfX3J^$@PJI^>R?9~QBj0rDRAIw=mD`s4krCH^+qd*Gl5FAih8 zaxl_9M>J46?I%?Se~St)(4v-ADw*~4qnhBM7^^|z^2EU2ArKr^gHCHVNMAkV+O<8S zk|PIwiyAD1w91AY@g!ZIPGERhYQGI*!D1W^VJ|bTZPaB`$>lg;{J3nS1ccPd3q>h` zej~lvq_kjn9?vVa-%Yy{!V8S1S4Y0VMbrpAEsa8t)K8ov6D0}LS>!GrM0%a>+iH{q zXY|>-aXvaP;GXUZgmBq_p{^FW-txGj)2x+uJyr*G*DVX${bl^TT4N_EmGe~wWpI?A zhES$P#~a4)bzSHXl>VwmpK6O2K43W`y;}h}iC&(FMW4PJlCe&nt=Mbc9d0~OZmy{o zQgnL7+G94S1m-xGOSDx9Jaj$JIOg@GecEq0mK?Bt^ivK0pxcBz@IN*t?g`TkfzrsW zE*4opDtg_(q!YJLgydsMnW4n-yhQpx;IY_!&x-xDuB}JxMUqHaGK<i`4%eV|C@NQ2 z3fCKv!)mIyaFSh>q`7*2{bY)A@Ea4DH9L$X_2S#wR8{)5xf0FUSMuh2dXe49>kOU( zGp3>Y-lffnz|+@UuV^T%o~OJYOYur;S9#^$USRL)Oj-=}p4zo~1~+h?WpRpy(SQLP zlg|-BYV$c^+Ogo{-(O&o4spV)FM6d1hI&m2HFgFe6N(q!X`phCXLE>zdha_|jvXGW zoM92{m&$!q6{2@Z+TGbJ&L2N(OaAbzsi_GMYLGBohc1Yk-)#7`$ub0DPvP(VbXvDo z#K}MrCy0@z>I1rr-+5k{HSH<BdQzky#b~39($15z;lcZ+m_S;{+wdxT>Lnnss#W?H zZtQra0?o;$cWc0)&ESB<kbSk1kULHX<2VB%nx%7|y=rero}DrA{-YW<T1sQ^A$gs1 z4qo{%lj@#%<z8j$>9}Lg@hOsov-NMdV*^dlTmvH3^&Q6#bI#MSPUXScjg{lG7}gJ; z>IfPf({7hhYHB;3yHg+3^%Q8tnXJKXP<s*iRVO~IHDzdZ48A`Q_sF^FqylQPuXnn5 zRVm-AFFzD&gW;A6#e9z8+9V|&xJa;AJ8)D(l0pH+OtW*Xn=_xvp$RobCTHGo_}=JQ zI=+?O_{#|a*JzDKDcY_GY^|R;tAZAXdk3R;`e6p<U**`Xe$h6N)+)^-O>+c(k8wRL zmD{Jr9_}V0iF10PL|)dtm&YL)Af_}~SoRd&w7z=0!nW}7hmj64Y{7fjG=KZP!T4$9 z+9?fjyDWs)0Y@03U^<@TZF`r;`^&dJVpSjH>?~}7fB*cwuO*W^dfLLuvuS&cEes~& zQf{1)xf*vTW>#o(X|-2_7j&K5m?a;gEKYwi+1cs8zZt=eYV>;zjZ+~^Thuivom1ta zTGYE<E-xlvER#bAHg)%?sg`8s^x~<j_J}lmz^2}ZrhW-vBjyZsjxX!a*}A<z#Lv4| z*U#Z)3B6R0V8%QBo2G!Ia|BGa#*HS{*s%d4tpc<n@mJZ#-w<rZNLg%*>_hHrmp)ux zqrSa4d45mJ_8?fOH9V)W8X_)HE6cviX=MI`pMjnJw%gXwvE(d{xwBW#x?-j=13UXe zcu$w`+d6(#Vh-6ek~;X~23Hr)ORi%7+N-^R-Xi&5T;_5YvMr1mM|9WCdhcgA&f38I zM4qMR9dzw56;4208gDHCfFG?>J1eWT(xBP7SQ0aES0>ZHn%5*>$|F|`8Tis)Al*PH zrC?k>cMmju`3F@t{#IsLJ~TB=teyIlY|8W_fO_H)1>8sO>GEF5@BYo9Zc|vPPI=e_ zF?41*x;s#MBA_!$?yX37n%n$zm+2asvXdAJs&(Q7-<}r95rsxV?e_lhQ`KFQDF6b2 zZ?h6>#pvaT{shR~krS;5cvf!QRDDRtG2^4JxwsTu-<h6riMwnqARebw+4y>CMwg{c z$^*%p(pWV*5!cqAJ(n1;k@V4~4pqPM8dc6Aav`XVkZ-vS5;iyAs@)J5`_lH*H{`s& zAOx%OkSVX;M6;~6z3#njVSeV}h`wfo#^IMoM^>nA0~fV7!{xgsNP--v<@dZ>hH}Br zPDErJi+hK%$%Q@6f$DF(^jE29ZAmY*LGgZ$o;ft5q2fJe6UzF-T;$GW>5iE*JEo52 zpV#njvgx}1pai8hvXSUl>J6Zm%OcdXWBY0rtR2#@itJe*eZ9+gJ(L8?nJ8E@Wojrd zCm6?4qNEzQ$K(x2_y00<wuqMof<S(W6n*b*(188i`q0$W)X_Kc=yg!`s0U$7Lur~& zf`l|7$fpHdhOS927U{mcCOJ}H&kkWd&@W@lta7mrC5PmD*NzQca;=YQdn49(3V{jP zXM)YCSn^}foJ^9aRfp;eot~De&57qeXo=8u9CD4l)Pon=ADst_xN*%9pQ)(CMV+iR zz)rYAE72D_eFYe>{;#esx#YzSuD<%<!Jks*ZbiDb=_!4%uu_L?^7$kl9?)OvRybFC z9~E+zlGHz+%ROCLlwnXd|6^mx-RnwwFZ%i)IbHZ>Sb&$#bZu0t<iOsJ`G_wT1kYgd z504zG(*gyzY+-x0t<1FghA79B4(xt`+E^g6@Z0RE=Jw^WdG8!i1lcxqg?lSB{F|>6 z=@0D5X{WwO`gC`e8O&|y$K%s|yeEa22uud6_C%rs#YwE)HP+RFMUlg#fceWy_pHbx zD7}{pNz`y5&F|ACq9fv`g;wo-5$bAyA#0jUoHeWnl?88;iR8hoXq@T(1Voc+wssuy zTdv4}12wbl{cVG+>5i6a?m5>%Zy{3NI82c_^y@~i;&WJWt)JR>o~<>5QBQ-_Mw$^> zg0t~FIgaBq2*%XEpUP++ni6^^=D<#xg2W!X)Kfh2ty&d?Y@Bj&XX9_7h={K~0+*3k zROY)70qr@dL>)1Q>DTg)k}?vpL}+sEB%%Hl6rb#l;4c%rQ&$oZ(21A{ivd07yIM!T zMz>hG>TFxYE|0NjE@{XAI8F?esCpu@rG@i%maW;&4@6yy7<`gGBBWgr%|QqF`IVsF zzdt!<@4^`w$KimDfp2tByfBzPW8;gj4=SBg&c5QJ24z3!h`Yhhdr-VLAzwg+J)F1} zw#Q$w;m0XobajEgg_aTN5Kixuzrpo(h%~1XY0)2hKa=PAntAWHWBN1E+1I8{<ykKj z^?+KKjor1&UT9B*2iAqFuyH=?YJ<!(S<llWxi<&e`y6ZA-%`Hx_3~5}1ga#0PJ83c z+XXyVt>I3WxN-LlaU%Qd5|1^irc~_~<FM<YNVH-b8{3HoD$aO1iBWRgf&<XgJ&yoJ zfE&Zo{5wzG%8hIMir?t<!`Y~r*RPSk6^yuID(9Q#UNd9;il2ug3AxKsWUn8jfmuO~ zk(@GFoP)g4(EOs8|8nRrw0w&pIJFVn!db0N*H@1OH)h5vKlf?cE(kRJC%w*s&5c#~ zd$rpZxz7!DE3D2s;PP%i*(hs;pt&K_&W9K*?#P=;9L%|^#&ulpA}}MLe~0w44pio# zt=G-!y$ku=E%%z&-4yzmn#X<MA#*+Qc1&jYrERzn@7>xNEs3^*xmSHc;l^5_LfR&w z=7fM2>?@8_dYy27hnbrb;RP!L3Q8F7tP!kI9@*_)UraAL!cIu|ap%)VLm(#_X=^<p z6!f@-Ph{nh&EVy|I$Bq~<?KV(+9?;<i>K<0&k$$sFc%Y0Q+3@$5%Y@^aQCiW0yf<F zBEpkAi$e(YD%#x1mSnVZ%}q(23Ki>-?8(?DS36$9Qyk81>@J^T8>-Y@jrr7H5IJ{- z)12QScyH|OSR#F3{fGG1)X}OL-i<%(r_bn`VxZh_Y_W~`rJ}Tb*E|6QEMD(ipMc1> zay1jVA!riGbI<t<Wp*Ai!&6qON!>+*_r=o~1nw|fHtvR}Kfhk)yL8VIPA_LMhkYR} zAMOXFLBu`loQ2+}b?H#7xr)2PFXi4j8|uT0o?W|s0_v+T_#n=GqlXVlbUAvN3U_wP zJv$Bgw%VT0rIz(A5d_uSDmjmxbw8{y`#HI}?mW-2-f~qhOY`8@P=odGVr1CZpciDr zCd24|vG(rqO!xo)I0_*bNhz+PB3+0M4k05Qq(aKbSvky^9OgKa96C64xg_V1XtNDt zvSm{$=Sj?A!ziaPbK1t{@O$-szpv}_{d{lr?vLO1AGfjH{PEcH^>{uW_s9Kye;ma5 z&E)h{nXw&1%xXA<V0C+8<lDP`%2a*ufDWN65ldhd=IgWdeW;jLd?y>@Y>%X8L4O~9 zh7ES6P0wS9s%2B-e~{<PMNjdM&(l(V)m;V_07tPa*lBvtkJEs!q(bZ){uFyqc_WL( zGDiq<(}1dM3Y6{!Ke`2%z0e+)+r!%ko}jB1ovR;7{NO9tn~6ghq4Yd9sCEX{ys#7D zu&uvLCvMBt{B^<=ykE=i0d&3;ffP@*xbGP=QYR!?3kON-I~rO(s19B^8Z*y`o~{MM zHD9ar+mTGeDQLKy1AA6M>?CyoF`s}s_c0S@1HL&Dv>M94QDG*}u7%jx=nmobgVp-z zVauMXHoWHnA!M#@e*?XfRyE}iHVJUWL;Nw(0dBbO=k1S3uY38(`JIc)vW|u*f3=N@ z$u6d|v(}qgWat?vX8P=4FnnT-Dibvy>Z6lmvelp)!T%A!$)QU+^;?$(-9!#ApZno6 z{XZAjy`&>RmpRt4$Kc`*KY0(ZBRO?-yh!8%SipT}SXdZy=ZB<PMPH>tih`Y$W$29$ zH}rQW*eXeCZ469ZmGri2I2B2h7=@0V_d0^e%V8Oa`p?wv=CpX<SR3Fdi=8$XXwi22 z;~ZYYnVxGgpW+bR5+L>QDYV$A6mw@7RibwGV8VLw+M*9#cyT=%GK&<IH<`6U_X7yj zAMz1>SJDH0(k*$xb41S9>T}%If`w)I&Qgi(CbJr1A~YXtaHBQ)=`1UO4>~*;;|}h! znd~IA2bXbdG6QYc0ejaCUuvjE^^=p}Yv(2nvb(%t1I&-6;e&O17{DIyy5^eQur1$+ zh<NAwpA+A77<U!7hJ}Iep5>`0ZywBo>@q5N@av{s2?*z#$@?Qe&IyfA1c);^5<v$C z&(<9YJax~M<;trxZN$wCRb^hsp2$2KEQP?HI|he=8*Vg@-}kk2sXb-)s@nTN37?J3 z(-dxnp}mO2jKQfeS)fceoxa*c%7$5wfoXa1<yaP=Ikfr<7yRWsDMOb6vKm|T_p?B1 zM?vyc<;S10^+?Cg5S5IMeKkDVvM<3UR2=mtx-r#c>Tb|caM5j&GSX+`>OKUYyYyyV zQ2@-}N?hBnr-!vWp9HE<WrX=O&qu)q<H^~m7Fs^Ra`o2yBE+=zx9Saj5jNE)q{H&l zS^Zq)+4fwMdsD|DBQxzM5iu8L@8E@2hI+7`6Hy)TDgRe591Ji~hm2O@_PmNwJLCrX zAFSiYuNDHoqe*%7#brN!|AgdgJf3P=dDHkeK%bPfwRCRucfsnJ??_!Gw4_V1JU2y4 z^j&LhhJux}Hb5*ngBESNcutev<?WkMnTwMVY#tAj8puzka8!{ObVY!vUDvUd0aVwq z2ax*@VjMu@Wv&kdYBMCgWj2A$0fgr@vKRWzjKeHP`>j>`*vdW%IXVp%0{2u81&K?d z&(hJta!Wm;RNS9IS_Q{O8*VgcQ*Xw^Af9_j+mYD~66I~7+=|w<ZiYQ-9dW2b{SDGw zThobA?<rxzgR;~fee}P-G`LPPkOw;zK7;azqC0FD7RD|1@0#_5EnVQ%Ovdj8#gVo2 zWnw4V0?MR~MvhPQJ5L&BhcVCSx)!&!Z6+8|)is1>h>O3R&`$mLWBMIi;=c#Lsrm82 z#TI+Xi?<c)>%>lY@oWVx<f_{RY8@pj#4ZMioHZrEZdo^1hjyqIS;DNPy=#N56ZTOq zq8Gu}l#85Usho^L+KEu>i+y@tJNgm}2{W04%UB|$XsT5&fBXKj<t*@e{z?<sueZ#G zJ_p79=^tjbrv1|O!CdHk_4C@oER`CPU3ZuiA7`qu?@Gg|RA0IoCuSTn<xMq*g3Qm( z?Hk|uX%s8|A#7uMQg7Bo22?P|Fb!R9vR$Fjyyx)EE6E79gLHANVUH*i$+JeMImvbl zlCcdM!7q4D=x@|-D#~H>vKasBwr<MWFl@?v5-Jsx+V_Opayn+qeLXDI%4l-JXt+-J zm&2cE;HVaMIegmvvuf*}@?ixqzSOR6>+&&^ce<Njn=WPWJ;8Mg79b>ptnYPmG{g>t zoh$dmGYsrtR+w3VC@^U}%rU$I?=0}Vr!7bOP3f4{H$vjf%dahygGo(@i0X5msJz+! zOM8<Y^GUA8OOEMxOM^ndZ^I+sUpREBK1?rc#JE4m>W><luq9;-$hgACS*}x+qVg79 zf(`ysxK8TO7k3yTOU^3Ti}S#tPS%q_CaOPwRRq@$!<pH4>P4>ImpIYBekpht>wKXf zQYTE~w}s$RziWL_A?oInpcw)kAezD=&=@`RtXt3~k13xo$6Kz*SC7tP>^A)9>$gVQ zHb*kt8s98E&~uu=l`9t2fkdf`wzp>=0G}golCujsf30_;V*nJ2*`HhX(>7nBvR$AB z;7^4`M69KzqddIsdxI8A_!XmqWk7vCC4reRw9o;hWVr;$lqSoy7k^@QpGdhum3`Es z?I%eVA{V<L#J^n}bB)@%@*FG5&rN^Ez>sh50ih?IV(O%Vy84C`-`qmkYz!-&8K}kL zg-{!P3kdOi9$%E!ORonfMC~x~Y{GJp*x)5zfK&{bje?K#@l^+u(Xd2A+fO~1DYbH2 z#$CfYJZ?R*h5BGpQ7<@zVb<5c%z$~ltFa^K6rANO99VoASM9`XYEsoWn9iQd>8{+W z5rFLlrIOpGeX%emnK;dBwQeJ?N1t$PR@<tLB1I9m3VNmnV^CCkxF3_vCV<mL6qZd+ z{J1#!=R%SI_Ixfi<!D6yl%Bk>V>>X%JD6&WkiUASIYzTrI^kaQv?>z!M}O$O7svJ~ zA3BWc)fKttQABTTIQU+;8^1~8!NRS9U>(l4BCL#Ad@BSsZ^7li<XSzeP`NpK%TNE5 zhn!Kal_87KLZS>>Tc)Vgc+gG+Whd^FhzSsQNZr(IDepJAeRKAd?PKlN8#A+E{i#}y zcH;RWWz?O-V|ZFFJk7z=wmTxOuHGtu`Le!wZnAdfBr8YWzAwHEor-1ELHu!$4cns3 zS4bjPmwk^uuOtZGVLRDqO)Hom_F|3r(jh$mrS-X}^UYEX4kgqmuD(9p@!2rvkQm*A z*DqYC{B`eX=T?Vihv5KbMksskYZ%Se;57C`O$~Nw5%Oe(u?2{4<&2(c1A}hsoJqT% zJ?MWISF5o=wQ8sUO8Yr~C`Z7SZ!jqcuaGmd7MQv25C<g+_>>(;*j1bF^<ZUy53zDm z^kiEXu;#s99r1gf;PD1i8OhpyleZ}K=n9U_p>b2A!MY<`eV=ly*Bt=1{3EKYXTkPw z#V1WYuk%jk>v%zG?NIp=av%!MSLqgN+>zA`P)3eZ6^I}W?<S%1jDsBl)7?2}gcF3P zW0wRB+!JEbvf%__6^!p$OV;AIf&~rVPdv@=gWwrTEE1zWyX~2+JT5Vev4);?=YHX% zXWfjyY(oYahOrZN5k)yyrs-%Ik-nMzpKHgO<sA!CFp*D8YA(U_Wfih~Y=Ax?5mWP| zA=+i)O$s-#W8cakTWwu_lfWbfCh!Gu`uF|6NV5uoFjxZtkLQvv!<Vod^BP-g6Lp4# ziD|4v#Nhy$k?MTimrIp3UzyJ0r!Ss61@KIHj;|4j--i;?>$Wdc&*VEAx~mZ*5w|WH zB5?@^FqpWri%Wh$ZP1FV*+^AMGe27_Sp$lyPW279iCf0c`(3x7wyVU^KIO$HVo&(= zD$B(bSk0yMQD+MJV=fd?O)0_pm~K5XqVdw~A<-#+kxA=wRCNX4rms+qwOSc)L8ec% z(iiW%B|`a^^_c>!k9nb??d<0Ir>TQ6n_S{lhC!KYGP|Wl9_->k!*D-$M%aMy`u*5Y zH08bx(Ye~7%+Pu8blK`i3K1AB%O2i1Gx_0=?Fr`?^+Ncfh4Y+$KmdKvTt`QT@c8lL zRvWSYe=@<3tE$*W;qSoeJ>}TupK3tP9>1p+yZ*IsJ`BKE+RvAx#2no3XkXAoz3`${ z^S2y7gqvQYZYpkf1Xr#2nPf&_B`XfNTJy6g{mp`$^}K6Ma9kHIX!?D-usUp&D(p?j z$4QN2{X&brsYn-~3{kRkQ;tXqw`x@Z$EiD1e@tNx^#%pfurs6<IF#(kQeKu9L@RUL zUF!_{Yc<86v^B(RmT^4zFL?DN!I0X@MZUCx+ML9+5|#&Cdy#X_(?H+1bk0<tJ2buX zR&5q;F53{`Ms%ea^T2ynyqOl{49`xl(x50IkFoXE)9W$1lcl=`>y-@)^9H;3&vmIc zytF?*9A+CvDug@NM+d<<H-P26D>@S2yVhsF>{<c8)buWh`t(nA@Qu4A!#CF>&a`dK zZq@5s9qR2;T>mPk_mW#F?sA17YxH!>P^4*1GC#g5+6W=)X%^=O-d8L3Hd57j@$Ta& zD^?!}qL-9W1=;9_3=juq>cD4uao?H>@k02M*M|KJizV;wxxo&e=<n{;5L@R~LEkdU z<Ipu#QXm`~8jajzbwn;Ib$YWij%I5(*Y%y3A0?w$7yPx~K^jzRxV#ME81E@C7~)U} z5XZWa(;88i%sF|uiDv&U`ojj;-FpmpUFw7`IJSmHvve6MyqOumB6yNF!c(&euJb8< zDf4jH-FGV%`wd?nv-%x1Rxwl0KHq<xGkgFAhMQYQUEZ23?|Wrs7z3b7`ilR)#&zAl z=>0#)uaN?ld~096+)>`cuY3=EO%35x(GShsU$RgoH0T?6!lyZ@-dD0)xU@Kyeq~>W zvnosv%=)5U;m!a)F{B}n8vSDnF6HU4pH#D7P)uvbQ8xIxMoyFOaOk-<3M)rYmM}GN za`yM~S%jmsF>YB?#Lrv4Sd!N!`{-kB5WGevMEnvMjI5pBRv6>k-H<U;2JN!ips;dT zCv&oVH@xV|xBcm5-wG%&SwGF!e{%uQw*w3Vpk`rX0s7F0#rri%98X$i0}4OKxdgSI z4h2u!W4^KH-m6kel86oakt;{+IF368iHfd4WNPg`#_)X9sZ-sHw8fC3FuHj!XTDU7 z&9PJpbnbku&nap64C6q+uS!Uk>vRLIPFD|4EPdBcpYneviiKa#^d+pmOQFL;Ls|JF zSG5}Dw^o*DMt#_{`6k<*>zU$H4cti(AChv5bvwg_j!k2p57?xtc-ke1GK{c|z231_ z)MVrPOUtja7+Dxu1HU7p;@%4#9E@ltSMqs2sXWI+`n{FC+^OZF5jEd|1WBxddzdrs z;)b1G$J8tjN#JuCM~r<?d;g5pB)sk$9*q|&4t=d<G<SamxffjN9Akq*s)aZvq+6Wa z!b55U@*Ve7^hdFB$Xx|TEFajMZ^Wyw>n(jvBhQ)|(UQ)<Mmg>M0bRjS2M5})pA%C_ zqKrto`U=XG4Wll-OPcF=3b9&tN-^Q}qvQ<3a2mMCm2v$STw>0f%Wre7A!h7~IfACQ zJ*j77pxPzEJyS4W8C+N@P;F)02P6z(u8G?4Pb-3d5%0qP1QJH!c+F|gpFINiBo3tu zrhkXG=hNVk5z~KKAI?k6_C01XpQVtO$ixR?BvuAh@FU8ihr$><?dZU^#FBBU*TV7% zGG;``W`d49R8ua_2<T5Mr`BK;_6yDzUw@H9+k|8)m5sR`FsVuOrP-cA<wS|w+E8o> z4O+DisjbE>P-aM179EnAUN+l8P7v}en{CDz5Jrx^ZOnwp<V$ofSKozn<s{lD2fwa{ zmuybg*HN4~XiT3|zNu~CYlKH`!@%3Ep`_f<Xrf93(d{sjnAeH5vLVNH4vyGX*?PHF z2W-^$gN4y-l7u{M`I>4!WQwFyZoEa&J+<NAMVzWBRP7oFn&&Zh4NJrrn3L`0jh;r$ z&(9yN3;GqCVn?(KFqV-I7YIN5v--HPXV(McGpD$u^{}ucKGgOm-w&PY$pZ;ur3^Kr z^4o~awjzK6o#`{7>*#3%(G^QKzh4Ng^+h?&IYzT=*I<3}a^s&zy{!GTM5&j+iY1V> zbUSHKYRqKVO{)Y7&H(lM?}E^}4%?dFtZ9qDT@`#|v5&ecS&xdRq9#1)D61O`gs{o& zE#IvWSgp^nH8+#@8!750j4C{h0^=acU1W2PtCNDu6rEIgJ7{lc=0jv#*XPYPF@g0r zuLj+i)TwEQZ+aOl^r0`F^Tljse(DTupv$(_qZ*2v2L=yoY-JvuL#N+l=m+SCY+aiA z(+@Hu153L`zGirkKv(9y(BG!{#$Zj%93kIMH)NmZi#YsCxp8#(_prwNHvMz<)b3p^ z0xes-t%$H4*Dn>SNo$G7`$Pd^pSUDY^jQ&UXc4v5>XP3z0XH0@TzqX~X5U<J$@LfF z3HHlCl?zLWBY@U*#ZQ>*F;o7UB7o6;V<7%+O>$%fL^@jRjmR*{uW`M<Y$p<;$D~S# zpF<Ef?VGkf`%l_22&UyYF?o}yz9ef_PU{el?YqMkt}_^w0U*{f(4(zgWj;oIXo=+Z zh@%Jb8mhUbtjW0#;*Nm>q%#V7*cxWW+GFzX1ao9P(#ft84;kmWI;|>jXfO}Xe2s?X ziRi0)!Xie-J5rqXo^&j;ZE+M=jzV*Zx$}FzC83}wjo+V*6tZr!Hjl+d|9diZOIQ2# zPni^dqpzS?pxc#*B_H`IcctKE<MLD&>GXp9=JXv7bBkdn^5uMsEPiQ^nITUz;HW6< zN~{$eB+}$H3wnT)y}&wGO*wIMvS=gMx-9wQ@4ADct$<46&R+25Uz^Z#no;yCtX@wg zEB;2b{AZ`VZ4SeoAU^N&xkS;|eze52Lf8r}lIb?Wl9g{;$pj<U^5ZuN&*v4_XLu@G zPh$ojeMLrHqCV(jyNP?li(&SiHaPY5P0FWACuh_}xr14=6Ljt(CB0-L8?2GamF;xj zW6oLL3$9H==!Ef_*{-EG>V~nPaH3S4HAGyTw_M2`dDvHto;trZVl~=V=<XNFL*(>R z+Pa3hc`@|0Eb(ja7BfQ2`zxZiF8E9j@NCGb&Kr|Z&J>*OFX$%FGEAQOZ%$a>Spj+9 zX{XE2gd=}4q1~aduzh<mAK(*3cXV>Xm2eKJh%0z&eX5A7_B$h{(5BUM9c&d5uCy~7 zto{^Kk>~=8Z4*|6MfJ>JKW7~~@dhB32Y*VKUO)T*v4Iv6hUG#X@E)4)Z3RF3((ATM z*2>#n|CF>@tWYe@^!{LP&kru_2J3Xj4QVG=@IxB<$O-ZO?!OP>yNu`T9)`G++@hd( zXRKDKVIf*sz8A+ZXQP}78E;Kr58I&Gyc(qB2oTxd#-`uoB<mnzb5UX_&peoj6=YKJ z!VI&2)52gqyJ#bS#JQ8<HiA`ny+wY&#ma)n<e7QjSH#AWtjjgds{!f{qQ`BJnaMCd z+pfA+TWQ8%YXpO{?!3-n5@QqimO}Of{)*NjZw=rINvx@FKWQuY1OUgBXMEMCyms6? z?;`bEPU0@C0!=Yqu`BJ}bzGdgOvmgytEFT_is^K#aFU-<Kq=Q(bHwY`Y5ULFa7_=n z@Ju>SVK{{B=ZvUy2A|Umt}E&_<s%Y^InK@v5n9pnO%H9uPl+C)A+TT2v*kJW)y|w9 z_JNMzT{hZw$uiQK?BFnGcU{pKT0|ANhY_rN?pA#ao;*&?G`g#qsB%3fIh$$U_q2jv zJf9}oXCJo}M6&(_Ol5HlglLuXWA>YG#_xx-2|@5lZINTM2YWYq1P4PI(@y#1!Id`- zwubp*E4;&0iAZJ}=cTyDV^^zJ_B50e_2hvpGC%)#Cb+5gr-AwR8+mEp&t<k)WZeIZ z4*$<r%2I$?-EiyHi)uCb?c|F_edEBahuD*!sS>OGrtDC$*Fu-@2gGxKH*FD28!SVz zgyeriut%r}FA3=_NaVb9`rLf%#E^l$eE@N5DS)@Q@%htJp>~}0QZHn+a^QBfdWdX- z2qE28MkTetSum_*Y7pY-qAq$CGTV~Yo<!Kla}Rbu|J-rFiRLg20rbM*$4PX&5K<Ca zb8!7&rE97{-|$GaYpcL)_wQ&lMXkL40}3HaPtmd4W*Z3dZb#p=ud?n+HB|~CJuS6Z z!nU;VLZawQ#$1g2!GdU#=P_&56bKI_T4OTB4nY?$x3R<UOqf1<?%9g_QvNjNA$M<Q zk@!bimg)sgm<FR<pzV$e^68caoc(2?u0F<kldYRY`Z_KAv+Mmop|r3Bj_J+3(2vRg zXKereY5-@I@46&MN5=z~!tdU#%81<t#QL0qm;!xxI0X-Bx*v2Ob#_QFr<(62eBtRr zCC)N|0@&54yZETunj^Lj${AzF4EL9a@5BfJeMShyP>%ZfN<SN7w-!Q8L)(@6DceIR zvRKz|#(^1p&eD81exg^b?WCD%7xg4!;;HCIst5RP-#pT_@AjULnUHI08=D=*#a~`b zU9oZROAU~%ED)kdz*c95mpgM!&P5oSJ7O{vAW%M9Xn9}HqKA(H)ucOoZfzN0y`D4I z*T8YB$o9QZ=}!z}g7&~!N9N|va2AUu^ZRrdILzM$+%yq37@4*{$N8|5mrQIAS?3Bo zewF0doKacls4*Niwn9}m^5?vWX&xk6Q`$BP9ul{$ziAP>wL>9obLi{gpB1S8!NAB@ zfLPm_3FrTK40+F0@Vj2Xe7Dstee1t?7D_Hni_}R?Mfn6{KbPy&*>_EBjx704qgyhc z>_yIByr)d#s6TjW(4M%7i&v)*#Jp!i9Vaa7%~eCZjb7U++fLZnKfJ;0fAp87WJl_7 zVzc7&TdWeq9b3U>>mhtPwx;$}81YKkkaL103NGzJSt!}`|13jGMM>*9nhb7GVkmd5 zc-3Ojyjo98n(=e!aj#NG+9jP4^}h&I=X@Wp{&}%%EIP8*LB5kT<a3^o!54uGz%O3% z=`QHckgSG{#p<orWBI>x!ASN82%_oMMN^se7<PVvX%7tP7eQ|we9~~{94&8-<H>NN z)DR3TQ9a&<tIS5za4J0QGQ9v6EpNvp=cvV3-23jUa9Wjn@JOt~MrI@P$Mmefiw4Ov z&OjA5rLVKH^KbO=U!BB1KeTtO{(YbMdp@+6JcBZlHriaY;1D)8gKQGUsu!w8Zi01_ zjXf;+dW6}<qUFUWMUJd(dLA>MFBIzb<@39`a1){~df5D<%l#<r-Yh{m<N{OiEMr@k z?ZrDJ^P_LcYU{o>brKS^g&YSRapoEnP^{;N-7_H34$o$47xQ7*kk6^Z`r655{KM!J zm1K-7!%6nKqYWd<N7wcUUdPYHa>}pT`8PD$yJ|!SMl9Sy%lg}x_0PXLc7fXIY_p*y z7vN?HBL?&&zEq~b88JP<*c_7Sj{0CU+fUj!KN{&j%o~LDoc3I*PWeW}YOFo@n8y&a zueY7IUrdXmWo-EH?vMg(3YlH}TfPD<8I-PHIqBpTe$U6byCdU&lqvWGfK9GOAb~V{ zI`Vrk|0A+ECC>I%Kp+eDg9}qsPF%Wo`asCuzD(%KY4P(H$1Ftdg%%m=^Q(d{l{{SL z*RVGOqq_uPwyU=acL^R*KKEJueQrYYctj>mF;*Q`*fHD(3g5cfnU7gI-j$6m@>J{L zm3$(J-@Kwvi6kS=$<JSn!^g;@w_nEI1#diHU$c4=<-Iqh6XdEF5R_&}K*YvCJ$VWc zD}y(*I>$n!jMOx<n9q|%n4S>piCJx{%YX&+<6(=9eG3au>#KwJWzR6nev53#Usk_P zyb<wQ>=12+LTUdaTAy2dzkI95(EJU3e5l``Y|lHEZN@@1(`IEDqd{JPZyO#OngdK! zU!MQ^YuLu<J1j1FHti>b1I~9s!p7^*Gm8R5Zeh(a{yh2h`;P;V@44xS_-V!Mny>KQ z+pKVpiiFY;3stFPp|8FRw_@mmo(<^|e9*CzaJWowg~xk4Pm=Q6+dePArwPxi*-dE} zu~&neHuD*hw^k}pnGnXCZv)Cc%DG#nTZ{(nXz%_8htnvxoCKB96trP?12HFnW6Y;D zCxC_y-2tW%b#FZm_{b=97=HxJq2NL-B*NWGQ77!$^XmhYMFY|CRM>!~XrafUQA+JW zL5~lpT+;Mf^ML!?;nw<G!B>pP6-s{z&<*!70CJo}(gx$*nyQ9E%SS`;0*v)RGMS@3 zeRiz?@Eov&alrhe^O$EXf$mIh!`J@T5(-QXZ+K$Dz5n>HOjGy-<Zl7_R=9xg;7{yT z$^aT&w4H^WKHzU*1V~Z!&nk#^a8LAW?r^&wDZBm%aR_omUproraZv6;`@7!HY{eap zf<{<{5^)PJrM7un+W{Y7{xYQ^rU~V5?C_P{#4f+|X&NC&b*wCtJ`3mTS7kxx@P)6r zF_+#h`yO!54D;JDfYV@H{?yCvPIdg44y%sw`e3yNtJ|#cv#$qcYWLHVs@{mia<dLz z_*!B95#=+0etYbUw{wL;|D{<K`?4oVhGANP`UXBI5~dQQi!3%ENa0z*qg^rib*-C^ zi8^yR)6@rKV|&YpNVwG+t&TC(%v5p~!HXT@G2;O(;4%nidATOQ`W>qW2H~{18Q8m$ zx{h)naMuASe4(z6V7P^qq5pP|Z|U16@rBYaF0_#{ASG~KU8Vh9FZ}P!u+RWV05?Zk zBSx=37ge~r+j8G=QP<G4$8j#Q<qX8DD>TaipVGswA*57`O`2f$xU7B34elJRLf}ru zsJqrjFCeyukZ*MN`$$vU;$H2pJNBSoFstOQTV??4V!P*YuFp}9r#@~8CB$3LK^eDL zu3FLlf+C(f15?62JL%rclNIy*p6J??*664$*SKWB^uerd?8@3!Aw0;iZ|-YTa)#(U zd#=jWrbO+ze8M=k(X9sL8*UHGgi|4|Y{oS<JY+r!=>_O17h(coj)jDn;Z<T-)WD__ zcWi@jotXc%A}A!pqh0T>G;uf&)&nN>=?>tTsk*@>Oxid6;_L}MnGL%1AU+jQYC`v+ zaL_Rj6S%7<kF>>L%wsn91AuwJnO~jb#(KcV22{7d$neKe4*aVTSc5sw<#K7-HSgvu zQ-V$lC`&Hzj|20KVZfCm%x=VP?Uw8#1a@ZIQSgD1Pu*ACn^hh>1rMz7IVzbq#FTQq zG+~y{{bc`IVfejTwM$6TYOIqz#=8C5bm9uZe|#(Kp1FQA=z(m)Ef{Z5uV(D)_u=^x z)~Ce9QM$Lca1eUxrrh(~cu>QA@pI;Jn^5t=#)kE1y?|~!?rSc)i45#ueb;AeM6GjI zeQ&LCs-@$@$0(~Pn`?->HnZ<BTNGDAeM6+t@_2zdpW4O>nC`d0t`v4G^*>-Q?SL{S z1UZBA_3R1ksLld8_V0h;%6Lw^03^%9r&`W}m4C{1Tg-3D^x92_UYN#LXM=6Ux-}c* zIf^}}ob%Y2)jcpb=-BXV)P__fu;gZN_gBE!)KMVc4td0H{&+0Z*ey_3SC^!%INB<F zFL>9p?dc^asNN4gB|dmr<b3jxiY^H7V+?IY<B{TQGsG=x{yX0+I8pFUMa`LhW1F>L zcSePEax(CI?rA;d3kZ??c35)J%hv_J;1}$q%^4~l*5Lc4&bBRD$TxYN=l6U<_nmh` zylu8|=In8;HEgy*_Gzz+39n4qt0-X5Z<}P-W*B;{1zyd61efvOg~RHst4}NDB2@;S z4O+Y_v<-Lo*aH?L_Z1(wKO7Ya>($tj5|BTP>p~c}*itwZ7St^7`4Zhlo>TK5{&u=V zF0q;i=D9SVtxY2{)N7c{UpZ}0H#|r>jK_M_OpGtck1|a;5gfAD#7C2w8<fwriOHmB zMyqxGN$Q+nDr^9S@Txv+0<#nS<IC)yVr;Yts1-I(%%9!!&!7IO-TqyqjXo6lr$mcE zN`f&cl|(xt0#SyN%uY!5l_Vz?jqfK?E37?}5iV2(yS>|MZXcx;no54=m~1x4+JSWp zJ!W-9PI4#Y628>H0pH7cilrw~G{`KjRJ#SiKS9>a6*I)Cjqgo>AQhbVIE?6}S+>MT zT}lHeMk+2*u2s20e_$1AxS06Eu*U26dk1A#qFzNkMJ391g7?HHtGsF($-45|b~Nf# z(y2s>%9wX<f(r6^06SC50Nzjq6u;uTAhXTFZ$DH9SCi75j#;A8)c2`#yg$$oL3dXi zOF1)~&J}_=je3d+^=CzOuu-&agY~bu1|E(jS)r>WK?JkJH1>?$GW(*y5^r8#gQ}ql z$#=NQxAt(OyFKuyaM-^p<AcC9n{kuVnLD@tG`@L)c)n_(hi*G%J#*s6qW759)3++b zG}lhpNMCAu_fpmyyGMbbkR?I%Ug$|}$Z0?JX~3YTZb~(POUKkqov)Nt0oWq;;9FIZ z9@55r00Z;Sg!GM^XI9iy6%^WP&{~4@6qlG*YCs4G*|^q}Y_IZ)8v1AFLx-M(blASn z%m|pd%^+)`;>Dx8GDaAcc@JI}nqS}CahW1ZMi}r;LW}Gew5mPj*xK2qA=&DbYe%EH z5<@7>#K6X+O5`F9i$Pw&Y_MQ8U+4R5db6fWrB$cgL!X)vn@V^SL{s_d(ID%K_!$fK z=Lg>O!TL|k8H65m<-nIN%QepUIvbLJ>q$u4(5pVWJGf!<Kf@XZ10Zx}e4G5KMZ({~ zTy6Ybvn_022>;prc(3fx=(d*s44Iook&+OZL@Zbne2SYbF?(^0f__vg?)7~PG_6($ z*ypfMiUA}Xt9<)fL@{3l>Nf<({2kbKKur{HN&r366Sr?`upIqt{hB-%Ex&QAjKAjm z7CqjP@=+*O*~!~y%H`_BZT+p!aQj7@n0{1Ag{-$sOYf%19(YNhf3K4bKZTisumDfo zCV9-Q`-(Vl8+`76T_Uuuo6I{J8PeLHa6*`uAlVF)sTn^kYi~kmXcskfmVwW*7+5Z~ zpM}yR+bOykAl`B?9#V~kPi&S;VPIoTGgp7F7;dx8OGqm@>0LPY_}o#56FZ+V6+E%d zrst}%yTd-YZIp}fkC!dhsh3>%zaH@a{cCtM5K(!0-l?Cq(DzPD1F*Sc=8gSVWP)n> zU#Rb!KstLrxK(gkWcT1`fe=aax=uaaJVoLgI=-m-1p1bhJRQ#KDDU)oejJZRuGC9~ zBZmM2*$&2lcNKdLbiw<6d4_JyxwCS2n3L)=_d^R4iyES8ego)Ppj+}VL?FVCR~ZJl z-+B6Ts#omKuXhh!^ijt9-}o@Vu_+|rosYnoJ*e~Wf<bJP^-X4<vGnuQcdvjvGrT{U z?TC{>KF3kRTvJtW7EaXkg!HnnD@VHC0S*h2318eDkP!IZb^2%FEh+H>0U<&cA*oIQ zQvpFoEznPJ8$>Ml9YORPRfhz{0pK!Bzp7swFSEmn-T=(a7bI@tf6{;bT^%$U0aY*G z!>d2<k<VB+3a10KozE_JSJdP^{`A3F_b}qBhHo)8WQ}sSTfhj~ClI9E#3J8MR2x#K zz&+5}OqMRfY3_b<#^~zp)Df$leT&~tmmR|Ss^jA8kdJbP^B}<xamFpIZsj?n`b+(d zZQO@*)h=%&s<}}NlAjWIe}J}-J-@bS8&%t{)=RE0nT^qz7g$7slQ-Ih@#z6|N24d$ zUX_FNvb{4s;FZp}dP+=Bl@(8&2h7-u(}UfnI28^(`C(A;)hl%EWNPh^E(D_5g6+IF z8a>&hcOlA*>*wtD1l2k}HzUAf=RLu#=sFXL5~3#H*Mymgj0g_@7fK_J>u0MUBhF>2 zMQNM$0WDz}{T@{UK*=Ar{mS9(jKz0IL04l%^zSV251R0wKm&e*z?J2^l|%@*Z{sLv zR$6X+OZ8r#nzX(W265Z`d<tmiA6<RkuO0V!-`MEQy);&*GJR`5YCER55~mmTwPW%K z(nD<whX`Um{S?#-$kiOd!cTFUJC3ozS~I?L-Fh6uWe|REHD7*yl6v$=LR#&2&!ZSj zQG4q;Aa(F+wy~K9L<Ns~8uV<p1*M3$c(`KKp*!(g1cizN=W*~Or)hOF`)!|T&v4u} z*CRjiA+#9eE%gT=5k5#!!2bK<#pcc<&u4F)e?8nT3|RQq_d(41CSKqpCSSY_*W$QV z5H04i@k<AMrt0)>;$cXV;iRBI=HmE8q0PCKS-Z_)#_-k}ykTU6D`S&n5A1Y-0{O3} zDEL=E`HpC#?=v&WLX#g&-l4CjW&Y*@{Hr88dw?Hcoh@`0t_J`piSp-%?=Y%w%TZ%s z##QSNK~i^?{aQ%ag(=0xTVN$%6)nhTA4+Va6|kLL8#KW_$`2xm`wjAXZ!0yHbKEj3 z8FmjpPWE@+j>a&VIkzqZEAwA>E&>~SE<H5%v!r-yO$#JLLB-ysPf8^msg3@%km|-! z#dYtgx5Bjyru7sE5v#%orNm(k>fGl`aK%@u0aD{W<ihKr9`Mb=id_VY8a&F-L?5dj zKi7qtw^tJ@nRQy+6OLsgyvH)l*zdN}G3BzA!(q^vm9yC3yoP86Jz;EZk!u~~PkOWC z#F!^0Br~a`)~mT^xUe%DcKaUD#g)}jOIO;|&A`&&Qzo+{u|u=e#|2aQ#WHf2d0X76 z?db$;+%H);sX)T*x@$eX`)BxYW6AD+L%+i|TQop~1F#r^`UPK|HQwWyFC?pU*$o-! ztz`|frkG2Q*HRpmfQ1PIH5+KEJBw%fNl#SVr!hb&dX6UtR!7ApTYk7q*NONTGE!q` zdEM9U5SUsCR>8X5!L3!NyY4;Pf%tfZCl$+b#UPYz#wcx{l6EPBBPBL?Gk!+)!<t5C zL^dwY4&%|}sJ=4OJzr-^1wYTXg;zJ$*SPk-Bv+y9e3|C?HvI^?2A7Gxf^?Ee>EVQa zne5^h51Ogh6+dKwn7+g-3tqnM11Da95HSvO{nY_YNrBi}X4lDjGT?ATcC4{E3h|RZ zn*|zFh>~%tiyESOjnZT2=3TKZ7iqI>45`izJ9w~GBlQzYcEhjOJf>h}leG4M##g;A z9s)3Jr#H#2Mt|qv|J?`wqsKja2{`F#cdhsTJnDlY1AI#sF*rJmU6$28c2iu!XZlm( z=c8}Sk0rl)YqypkQ|w|TeUx=hQ~aPiMH&{XF^iQr1Xn*-gZcW&TeI6%K}WQ~eD4+M z-{;>2T6(G?@92vcgTX4W>*gPo7;Vjols6GF*R2X$LoK(~VbnsE`04X=me2ILw%S>; zT&vyoBK0|2H-$d3bgTV~XR7S`Z`VPO7Ws!}zma?^<XJLX5`0+SrY7K&ko~#gB+U)x z(T!dRbZouN01>OYtUUFkAS;vPT(WuXne_nc?ZEuSot6~9i%+Sz^7Z_lyMJzS`Fv#5 z>cAPSi;|S}i_4|Knro?#$;g?g&Mx@MpMvn1YkhRZ*9OC3?5HOA6x7w8!JDz01ps+A zeH|I*{LYv!B53B9vJ?65o1Bsgw?+OF`Tx6{`##bXZRt^XvEVkQWuF~(WB58I4bert zlE&uPD%s1uUwV67eqSo$jP*J6@giqJviYuqwTz2nV`{xN32C9JNB5YOEQY+D@*JX? z<3RYpyL;}i?E8hKV9%nZjB93x6D|Gtx|0^MN8Ss~)WlUe0O~(9`KBhsGV^Y|zx$R+ zD!0Pd?AjzMm**)FQUviq-}doprcX_MN}PtVXw#dm{g<RaI{K+bb)RP$^T9nJ8rL5< z{hki~qs|8^Yg~{@v598W1u5-v#cgkajUpDq`0m+a$)y+6ni1#xd&6sg_+5Cr;5D(* zM2xv<Fihlomj=T#=cq`4Ze5SBqw0?Y=mlhxa(JYBI}|L;PoRH1k@)AT8*TcI<SYam zN&K{`D2Q$RuC@B!9Y2}p@bcpWd;_l{ysunW=B`#|i(fW9h7voC+Wsio(-fIzu2RR9 z%uSs4y=?bUM_d!NLL*#!+hn?0u~bKDj~lW)QHhj7Fg!M}zh#X2N;l35M1ybDA|q!z zqo@0mObJBKC`+vyYmU+j1#@!t;vv<0tGz2SqOhXnI6QX6_IwLvGi}o2RC7;+y@*P{ zAR!VxxDcV0q&<M75R@ke_dkcRJ$kf}D%FD{0779EkmeNlslnl}*rvaMj^DmWRtGx@ z-0VB$76d)OexJ*SX$T7)9&BsZ-j4AzCoAv{9PXzP-8Zu1(`qG#iL=E1BS=q_1*0NN zlbvbUGZ%R@_RZnuUye6_0jC3F_Z@=1d~1`F9$|Oy-reWEx7sRBEqbmg*!Gs!)qTgs zV!_}OwIt%BZ?}^%bo1#e`&2|@&F48|u5vGP6Mt(Ut=ktIzvpXzzzVKT$+LOaY{}!E zmP?)OGpPYrkNV2smjN0z5xJD(2bOoC-7r0a7|j+0!3@_!LDC0oY1+|euHRo}=b(#g zP_eR|uh7c-%c85<)e3j_7>`IPbWBEi&rdZ9h4~2%s=Wzd*16_7&^xz6NNdmvz(K)Q z$xeQ=LS^(4&m%O2%9JqZFTS#@nu#kl(+|jC`FF)QwnwT{vG5@FB_BH)lN>`ZXVuF_ zI-wB1H(WSJUTqn|pfV#k^sLSOY@U)nfHXB&M=?8&&IX<e7#yh$=ICtAd?(n2u0A2g zE6MFP{O|t8fP?4qE&%HV1)sC{=>a}iWwh7m=^;p_BtZIdUuLv))g{8yUK#(<m7*fq z-bIk~)qI-ioo72r-HQI4h7fSKs<PtxZ*%S$^^iWBmw0pIY3sY@$YI5FoRXR^uL!oV z{Bknmda#vgh+d*vbcJi0agOYI55#jvPa^Z^p^sjIR#ExlwY;@&aqle)eOWfBAw2`T zjpaG2gIBvX9}ZmT3=4*jgce;QaaPo3=k*;pFXQ}iePtTj<xa=0H}DyE@iUg)y1uRZ z>V{!Iq37!|b(7y!+{mYqi|DFQzl(40%7HM%ZPl>-;GRyhfxRECaM6CM!A@B3;08Hk z%ICvGW<(v2Z0%cs+l9K<>a>^|gSv%1PS0k45GV0;`U%dPd3;MNYLma3VURBZ*|xr8 z`^NuW5&zTUh4%`8`D}#_CZsEE|E6jWEjny(*&PSvM64fk(HB4OWlv1Wyz-JbgOs=F zm60E37{`I1w~<XeY{1VM#_1z8wqyT&C+wGbq>xEwlrm!wdi3pS{e$`XuRz`@)hdmk zPx}VVctAt%XILWIzQj{s$rY^eTZoi0UV*XR`udb*I+UD0MO&sAP;Sc!ezBp>2;@2% zQIpbYm+&j~$Vq|}4i(3yY}R(A2ci?5HX>)ZIi01}yPQO@&?2sBx~87t1hKq27+|%| zI(Jp7HOlAzfm;f)cDJKuYNMi7hGsed7^fS*G)dL)g0pz+N*sOrRH$PGZWj!K$s6wL zCi=QmL!-A!Wd2y6Ep%qaY={AM09V3M=)bSGS~&LI*uBlm{rXP}i>adv>ubvagphxe zs&){jRfTDiQ&$vP6YH(#Gt37U@=?##NV)MgesX%38BH&^exZQ*9TxB|GmLIgl}(d) z3La}$D{oMW4)@*Dj<_*>1^o3m_!gz#ng|0%<BjF(BZ>Pr^m>e3Z9=W7=@!xz%lvrL z9YaZ%BsL;pTgjObS)nh2K_c=zRyps8=$85Uc-7#c{4~&Z2XI5JpZnydpqTbPdBGlr z?bmH2*-tF>o<?Y|^G0?@3Hshl#d6U#LSJYK)aO{vHuz?{R)&nuwaF173N{uak;EJI zt1X+mdL%?+)j;FxLo#R1osZ(M_Go5z*-VsC|M+;qZYbm8SEpBd(Xb`Deh*9@Mz^c` z7KVe}jd#xN7P~PA{6FaW2ht0Hps@L+UrPVKb0KueqOc!J|AE^uJVc;}uX^WD;wR0H z*HVX5RU|C7S!%nmXL9R`EafiD3z%6dHG3&_qPZ2T1MFIEInGcd4SHzYtJYC2oHB*X zUI{Msu22t`H_j*w8yQiIg6jCwQ!Dn1Tbpxt0q2)qGFl{l=F(e7c@?zrU)n}l;N|4s zo*oM{5ZQ9)psbBphq_Gn^`sq{X?G)6--k+hc5)?AHmFJeuG6pX01ljWkdE6M4I*$G zep`59d`TJDu58&*Hvl%_we@EL7)~-Q#wyHYNj#<juB@q$Oxy<tW-SC*gHRxX(*T_r z2AknvmCGab5c+Y|F;~ZJg9YCZDL#B7k>A92ykN8rj`iQx><3h^;5)cOtEK=KrH|QL zo(3h&P;-RJFc?gjjg6I2ObC4Fl)}%Dv452m_e{Q<d39Ty-1b9QX!M&1V54*=EGmjX zc=Kk6!Qqt&H$LdP;Ua64s`4CKq^3W5%neaQRNv>`)cpD7P#I^n(sA!2)@?=k3y)PA zyw60I#(7C!$N;r36YkE0$|`2SeGF~h6<Ut=kv}M&nUnF^`v}<=ODdCvkH6fs&1^T- zav(qvSBfnK!ScuIZbxXbi;U!pSNQRW97u@K>P)v(P-k2?B_@N8nzQj!6;m*$E-jZt z+0Y7VcBQmhZ^K0?E+JLEa_blOr1-ToM5apYr=1vINp2_pj(}OhidGFu_?_f>^YgZW z;{6N1HH|Lm4}TpTshIp)2_oVR>3$RQGwUl~d$l~A`HEoZcVD3a0a}Lc12I1G>sM8* zoq9jz8>;fmsh_dr`2)^R&LA$JHXb<bv_wTjHK5!^@^=Bf+>*sjy<d2R8v(xS^-5oU z;*S`9SwMSKVS`v-Kj`S-aQ4NaPyHq;F56lFP;~5A316kt84=2*G1pf&&WNN_A5~c@ z*)dFHdXwK>kHkm}4#L~D>}v0xvaSb-7Mn3CQ-hM#*4=;6LyO*)-+7TbY~!AaNWUbe zuw2#8U33q2@+;}+v}Uis$nRRA2Yas}hdsb`C4p`={&+-QU%OBjf%nF2uV-FG>GTMt z5kr^9Z~lf0P!WK@vg1FN9{ARyXmy~|OpDXKZckU+G28SW&(_n2)2rTE&};}oWgd}f zs92w_RP`NmT~!rawDT<0(BO2*G9n5GAB+?F6EFu6E1Vh3?sT>)qB8SaV;&QedJpRd z;0^y+f78x#bY8!@TXPf{g|lx4HdI{Iz{8aNLNq$2RkFi;6tT;DFv|37pv}%~`TvU= zX^Re|muVi_lV1AW!x!`-)@uA|+Yq-^phb4*ui5D5rZLtDcL)mJfEi+9TJ4r?U`J9$ ziP1BOV$)npr7x|~7c<WhMgx2OWj-%)E)=0u6wZj0tSwa4;l`%Rxn6ak^?lWk2$_Yb zIEJb%PxNySU!k~9yMZ0Wx|!BlCjPE<19=Z$QH_i=W9K=P074aFYaUJa;pBO0?Vd|V zZpy{7-gZ-BW85KABRK=uCMKuVVfFNRYVLrah><nqN^gNvA!?#RfLkL_SrahjiHEF5 z=31Gg^)1fS+vJ;a32H_tx>pRGrkmYC(7|%TAQv^ay6cA-FakQ1P9s7f5v=u@b+U>y zPcme0BY85S+7M}Qv%yyppT@T|j*<OkTeALLW=zlfbmYhB=8Q3r(07FoaYwcv<#&^p zs=i&La#k&W9?+0iRveHE%&~f#T;R2>8Ymeac&HY5@3E*4kUiXv=tI}+s&DDC-Ub7; zxHsiME#3K>)Z?ECj^JLeT_h^Sha-&a>kbmW+i>Thiz?ogOKc0HxJ&THSR44N;KPPY z<5E3&TqJJj>-i+h9o<Qol5J{Y+K}55<RQbGJRqNG4o&yR+L_P!I@fqd^>4L!R>QH^ ztJWQOzFQ=wNlh~$xM2pl0YIN6M!tJr7ocsTNGqj?_h)s3=MExNLU{&E&QROBiBFBb z1j8VPPE+}6`2oVLJhc_GWzp2k?U;5yU{6%VvhiBSIJ=Bu0^oN(|4IkT1Ou^(nkc>d z<4~>=1z)sP#2c`J0U98ozJ85Jc)KhaZt`4Afp@}Ey0Puum5j<4jAQ^N6HT=X9ZcGd z{nlaJh&-h2MYm?ny}M>PO0|AOVITdvz}7aJY2GE+hj>2m;>xUI`O&XcZ}rzWf4pnP zIqEm(lN@}u`GMkf@1_+TAHfh~bUJcb9|-odDvh4a-_Da_@_8p`UKdLivx8+<YrCqu zs?Lr0CPOjT1hO0p=a@(tVb8*i&1K#N+HBtw!=9B1VW>vfa)7G4d|&S@&&j;&dFv+S zu8%WfjWbJSHx)Ke`^kUklURAVZ#0YbrjBOfN<ae(%$1!BfSV(IUM<<_{c|v!m9Y17 zS8GA}qyabPNkag$TMN<kh+6;_=lHmNrq5N3RQE9)%sCGmnt9idl-)*?60Sdv<3Ba> z!I!{{E=f0FMfRUA`=5IG`-h|es7Ng(Si179+&6%1Ib1jF@G5Q}*q0(%s`*;V4O?U) zE@^iEdf$MJbsp}CU>VZo*;yU2B(K8t*P74wqQvOJBF-LTZIp#7kcMdIV8rLO+z!XR zs|E3wN)~C?vDWF7S;S^}E(WBx|NQvjF{0;)@7CN|Zhd9H*9n4HpBX}!$2$5IT&5m4 z2)>ch=hHjoTM@a{HUxov)*8rm=Z(;n8KF}~A@gf%sR$5Fw;OWd?Wfj=fqbp*2-l{k zpm=Xs^$a!0Y_D%&blBjs6Q;0$4kS~9?0Qi9JPhu!DzoDP1jB&ubx7g56sp3Wz<Gh3 zEDRlX^1RL6m>b_l<YxEFpcd5v9l#n-*%nnv%0A7QhE`i9t&<2RgdSl4yT8d&{XbX1 z{|oyGbo!2S`hcc?GWY-e4`5FBZ;@Su8*(VM@h+f2jsP~089YE_w=waJh4Sa5Jsd@h zO{44q1M)!aUN2QM?#WO<7E~f(=W$L`E)m;*q3elC3C=KM0Z^GBpCKq^HgSX)U;9a? z`6}1mh%^;@>qEx%0au@$!ajaCfV03+Ut!M5f)6tgQ;(Y5I!)AXxZ8eoVi45h&kk>O zb9Fzw@sYDvE#!&5*1qhNkd3^{TkCr?K;#BOwWmWJ6+R2uA+w}c8}1B;Hdy-{n`wS( zaHNaHSwk=*M?&7(^Bg5v4b@muyNb1ZN8FX(?$_9?ovWY$Y!PC0h*6&jj6=g-fJ%aK zQ)$CZ-bc(;i?+%Qf!(~6GFDXd(t%#s7L@*w%?#euHk=Q*GXLMTSw^EgX+Rtc^Ilf_ z@o^7l0`aHopOhF-07?lN85u%uZti(isevbwi1__V*L5}qt~`$HQoQ`Czu@prCBHTA zr%R_%Myi0&aGf>Qf6Vh3Xz9slRt&zeR^sX!G2~FA%JT*(r{oTa;H{(2bpXFI?c&g- zEo_+jLQd+$Ch7di7u&pw4~LwPGWEJ-rualLf1k>_PsWv7M;`q~wZCH8DV=asv8el= zZ`rU{822>c&&#iTZ=NQ6p7Z#Nl=$1t&BEDyv{pa9FI2uD43!-L>EvugbKgf6PJyDC zv&6!LH4sTwA}Vc9_t~tRagPao$BfGtIw?WsHy*#gw9Bscr7!+fzK^*8s&wk<%xT%E z_oWFD=2V}|v|t~4P9*JA*3*a!p}x+073fn^e!;a_NLb;)3f@I@+Iwq5+_C+(KBlv# zU4QlkpO)IP_c<oj0?WZ2aQ5pwtCKs4n!=EdhE4V>!gvi#Wu4#F>N89H>|Xbo{^dCz zVW(?5Nx9)1d312<`d?@0D)Vf@e7CMmMoBQ_CB~M}KaR04Jj9-}t|Ha`=CgEnKDsj4 z*hn39f_{!qKr)tZy8<8o4x=BxbO{tuP>`6Xo;dZhPdt|pSf1>>sxuPqDmB*gr~sz< z;ke{J9RXc_+foGjz{OK;$oLmQg_-ZX?>K9MRBzhet2$~s$P8XWl1u6F?gyB+CM@Wr zRHV|H=AAywdlg`R$+wO5!qyIzA<BC_jj&5!T;C7(KOTbUu3F6=18Xb$N7-Vu4%Pce zdD(pHM~~h5v#ezMMBcF+eX+wuPD;l0Ww?2wqhaA*5#2iV<31W2b-hmS9lR}i-uVQx z9^#dD)nsO)>{W4nAMZ?k0Tz~q7riHy`ZLbLs__R%==&YRWmkE!tq*&+Lyy?6r-tny z_0)0$3p>tZR8b$Kw8qc5`Zmw|)dA<n^$?HtWqRt7NKoITbVK@ECe1-AQZX*;MpKD_ zvY|ekb^7zV{Uqz<Re_*(M6mdOi}Jr?FLa+zV9BLoTgiX<O74kVQ&3hwTJ;`2{YSy& z3w~xjwTG$WrtK5p!ZT_cy>2h_uGz?z>_FD`-ma%#wO=}|aW5so#?Y$)r^(t>cdg_^ ziJ%mhDdCY$OB~&Y*Jy1@+e~O0fBM&I2Q6XDOlw<5P)k*X*hkr*L-ktXl=k4`EQalc zpfabMG}U^oxl^f^rT^lA1$s5<XxFN!bIsYv*}Q09pURK#`!1Qf)T|%za-J%<n!*vG zYcp!y(^gq^#W0cvypj_!I1poVMG=DvP~1t&x;pJlgI<}P>F>jVucW4av_xX2qi0xK zT`$7dX7`#Bt@X7MyKVj4{dlme_VrojHN+FfT;#*{+S3g+AM3NsrXT+?xArQkOiAl3 z;+`&GCS%fN8zc17Q~dYYzIRSJHumha>igCIcH~o(Pq=<eR{S=!RMk4<xcgl2!#Br8 zvOm{looO(ld^Z2<z3ju-O?=FI=X#TiUdQ9mQaMBYX_~jLEX_f7LVqtvxB`8$zF%Yt zRQvj`7;ha5GponicD?Wb^-E41&P8K)MYXUe@}&|QhIQ!kk5>!y-3~t?KPDVMTs#cN zJw?4ZkW%z?=WKTG$1fKb{y)~<Je=)yYaicK2OV~ctxAhlgVxxZT2(_^YDiH^5JX$E z2$htU5VW;R%|TLgizp@LAc%A{mkN?f46PzHgbI?1>6^aiocH>@XP>k8{^R$5T(0X` z&$`#W?sY%wgCV7+wZklFZMSI8Om5bx*!^;N;msUXKDggN@dV?7Uue~*CPLh6om3*V z*wCSHQwEPP2UiT08R`Mwpd^ScJXq-3WvgSALoW;~gGV^q&Qq)3<^yNO7wq$Y8}%Fw zPAmxakKn9hjV3u(0SBPo<w<8PUSdcJh7E+l8cu&p%(|~><q!Q>p-IfZW9Xlp**_2X z%n_kODZAaJkL>)D+kWllPi%NGqVzi*@VSCrWhtwQo5Z{;IV?$4bB`Z+>dZNnS*^XQ z<2SGmBrl?hu9th>6u-xtc~UfUgY+a_>CSp^^-LO}$ODPDZa6H1m&@{|RhoUn=}D3^ zXyEp*Qy<1$Fe=yyziX3VH&Z;N#=Cfv-c%}|7KSnCbSA5tM}S&aSEu_Dr3P}$T!5Dd z!mmQC2zF3UY2g`sdqlv&y1dM-5$LT7nlU>xU8bC7>TPj157X<ojot%mj;|Z-%Om6v zdJERN?}Gt@-#jHl7BRiOBTjQTwu@j0OVUnn$9#)oVB_UCt1cFE3sAEygD_Oa>?cG? z5EtsGcM$GPig^7XBiVV1<+HG-1FtAU`x`j?eGeap$Fk1I{Wl!uhv4Z=AwR*IYYhrB zgrdeUryFVT<|l6nT2_*QWeLB@E7!+16VI9!xZqvnUfV9j8P}o1RS*mKA+)$Ywdf4V zR^BR-B>qv$^#-qcB0Is&KwJ1M8pV3*OuN>eBX$Tezj5Km(zhn8XfW((ePs5<m&-mP zq7h*g-!ay(Q9pKR(g5vVg)2D{c3_H=mf?*+w$Y&G7kY+n2B^JpBAPB!mP~>r&8yr? z4RyOWBz?u=B9t2OC%db)FA;+`f!<KnDILZ9daB*bNVj<hu8G@sP<(SWW*1RH-uP>* z2QoV^BI3)xyZ{P0jE9)!1rekb?detKW9x_1Wn~N5%FgyJ|3PxHAEPpQJ}v9p=%QwG zcO$Pf8^x{gG-SS$^xaZj4jWC2a#_d6ES~u9RqtQd`TWo8?5%o4_|LBMbwQAkTHHBR z!5~Yang^JSNU)eYOLUTQ1?v7$2QHcuKCbj;c*%zhns@@xKqh08ac-6})VaGVsYjee z(W_fh$zU+la%FYqH4ZjGbDNq1&%Ov8sEwuC(p8rFIx~sJu6n!r^c7|30AKR2sjnDv zXX-6ti5L13-%?4^_v}mw5~AsjCfa2xeiH<&8v<#+eu_Y2Qy#gfG9g-@Y1Ru4Hah4! zm^9<j5@ynATP`X~r~=VX0x`IxYITSK%2^XM#%c<LWn=J`1l3#z8+f&>8w}Fl#6Yf0 z?bR^QJKPLn=T=~!bktM*+)u@|Vt}VJNcG+uYhJXjNQqT<E@(#bI<obLzQ-DkgRI-0 z2OeEn>-XH>KP>;V4({m?TXAqH-uYMXeC9QuYgm8Z=@s@Tq@;=`>`xd1y?t0Bza1*< z6ksP%W-rKwyBo5re{D?v?j$#7qGKMp0(aNSH;YQMGgHqY6peTNDDROF_P?-OGG0RR zq;8cm*7hjIEq%kc9b|Dok^0;o@PYp!yGNvi5%fnX^A-x0dOJR9AuS1iJwGj+RHPPh zRhXn3-QR0J=|`+E5&XbP8v*W2N7gCM=LL%IiO6*lxFP26+?Pzx3oM;6b6liH#x?Wz zkc$g1)h8hm<{LM(Rcpd4w<DjtLm3i!hB>}zt?RlCtR|r9y1x4jD#F{MTQ|2X+LpAA z+>o)Oh8;FFXW|I99Gv&V$kq;h8;fvDgI@jJb^6(NAjn=5ZxGo#G603|JJ_tC6HV`0 z$M;1<h7ej~Jy*Be+V=c4PX8hR{6!#>KOg^-dtL4!=qltP+2trN=X+9gLQbSsXZ7&# zV1o$m*Z4h(`B@`gF}KtMGSIFkJIgSyq|PegPSB=@HO52(_3}`YR~R97SJ)BpA7`hU z&EBY(zwE!T<}vM(<)3~T|Fxfo?@Z?NgZOX!iKdIQyz$gRW?gY_k_m(W4Vb!-dCwBn zJ9Jx1CL`K;(w$6If&(M9PtZ87dX;c=Ge-pPW`3FK0m;8PQsdng7SOO~o)Bcru3pg8 zTO3<{jZ<5f3Md^t2J}@{OlB@rGhMAh5+PEDh6)$H9IG4|{$}zSj9034L8`9i0e=r5 zo}-gP7I-Z%n6ly<1`0e-TR3+p1xki3>Ho>${F5#3O8d#!D)tpV`0pnfEo}U(EK22u zx^SMh7HA@!Otmh%j-zpvuVMfbl#%i?BT7}noPxX8)>^d|V{0WufGvo)52^*lfenBv z>g1JiP`UygjmiR^PUJ1VsQ@4u)yQF8MFHe6h*4tB)Sb3H*=0H%Hiw>47yQ<xpXJ@% z8c#J)^u^Bh!&B#)7PigoRyT*H3!&e*chrKxj6h56hOeDNWqa}cj=+KtiGKd~7ni`1 z&i63DB$EDe@Mr{!3h-Db6_4|@Cg_>+<22E?`rCBiX^S~l-!?|mYR!zy5tBbb%Oh}* zhRITo=?GBkB7AGIw~OAoydN1ED6~oq9rm_LZ%l|9!DTfb{;MA27r}@8Pl1Nsp*~3c zFI~t_Jboo4|IyxvBV3>)LVaiW-g~E!Wm?^3C&2E|6-beoh+lt1_j|zFWE({(3gf}Z zY83Az=?2xbsVF*iAA<^)V=}QX5U2BqqTCr=Snl2^`RP_m^&FE<jQrVJps3HgzDl># zr-zc>?z^iU$Jkd^ZxCaSPfE|ued{Q1>{~0F*%;$Kw{!$&c`xJ8YJ!h=d$50qY<kgZ z$(Em>d-qtXLGXUuYQr)GS+|L9aRKjO6k8suV2zGM<hl7C*v!lqb71LehEmweE^RL( zStj9Y74-*6&**X=rh7**`py@&q2(;z5IV(~>uk5q*$4<7qhmulNF4b+1n;N%2sjNx ztx_m%$njNW?h}O5o=k-O2@m*ZUhm%}AT;^farae$KS5rxr$!C}!-}zCm+<Kcep7nN zufsZP04AZj<=Re-_6Ag3uepz_kdAVBMw@hogB2J9bv8afUc5-Qzscz=q;Qk6)g(?& z4VYZ*j`K>re*_PeSUca4)Sq?#Y%i_QQZ^t}hOlTRS0`G{PSZi;m;CBnL@pua3_41g zF2wAhqq<R5?lX!R425sJW%>3``6nQ=AI>%qN+Jk*5^$!EWZmP9j|EXARPOS%ku^G8 zW~!HSEZHZL*}HM9lCn>2QYITU$8BVa4qb(kt`@Y`(cX;FQUc37_Z!GVm~=|y%>1HU zK2Q&}IqzhXMn=vN&O>TPjv$z*02Z)sPOVQtXi|6ssCMJwe^20l9NyE9d@i84!q(9J zlY>7Z#K+7@moRUKOv`Ckv;hKb4>SZ_Sbp<zg<<lUJ(CHKjK8Qra}u_E<eDGl?Dqn| z_Wf`-^YMt7d5T8G;Wcdw8U9E)_EYvUI-Gac(s;S1aPMsEC|IvQk3HujKU+bd61Gn* z(JD=W8YS>=whu8eZjmA%^|7|>O1M<nA^dGm!0xO5Kz23$HQI4_YUl2d#o5q)?XwqO zUNYR}+D8RUR!o#6ow#stwb`u3e&)hU3B5{sx6piMznlx!ZwTeUP{zElTp(=t2}V}5 zDF&Wrzu%|o{X)NTGyW)?%cu*+2X#N|9+oBDFQ{ASmtyFC8y;e7#8!BFE|~n*MXC=V zN~qL*zicG)X#A?=j=u)bCNU$Uh28|cKP_3gd+PRf*l3inPGS9(Xmzn~;jUKHmGRDT ztOvlCNtN$r)J{0X#}*-!aPO(cB^aH_EVIj0R3X4j{lqjqcJFa)Nv7Cy{14qwTZmdf zYKpT-oh`2vK)>qm_dZ~q3UEhSpBARS@Z$Ork2_%-nSdUQmXrARCbM*u3aG6P3ZZ9* zY+-IdXRBndXgRnY-OA|?GM$@TOnl_$T1(IT(oVMrs|`IL1%_dU&{c&^tD^V2bv?Kd z=_<0FB~9bORz%GLg%i#Ss_q~7Lc&~Y2+C}opas=a%s!77ZOTZ2B5M3q-%^g0hpQ~t z)4J}f#M8UaY|t?rQlt*1H-?@^=-ZGgq_hs4H9v4KG^C-$@ijL-$_0sL|NmE_UhVBb zFncX;ys(ni)>A&2@yL>RRuSe`sCpR<yCSAL<<0ZWNNtI=6XUP_Gb`q>{kks}>5pT2 zO?Ub-tkOAB<|x;l%IYnIJzpS>Y9DAxHq+$=IXwi03RCCP7I~O0ShK40ncFlC?%1)z zmEWwXI4EnFbY0AJ^E}=MHs#(d(Nx6d>Cc!9#>U~f=u?5T=XNLMXEYSG{1`Udu9FVF zFM`Q_`S@Up?PHx)C%@N8`l8^%MbfJ=xmB76W5k7(<LB*FM{a+Zo8-#G`@SYLsgw4+ z?W56*0Z-;O5H-k(3OPwSgdz78DLIT#9SZK2GFBH&SyO8YTpM->^TGl=A{Gb+FJPp# zvok!}?8o~4n6$CSQ6q44(BF}6_-{<vpYg&c|C9(^eWq}nX?qNKL{7l0Vo^>2>`PJH zrUHCslr1<TK?76fPYBjlpXQaeEDUkTmPZ(|Ubz6gduQAed7HlONYyz^T8efIcxqS* zxe392KvBmk$g;@zoWo~pzxrkZ)%4FRdN2&|K6uuTtU!_Ln+86q9h<ztcVx^3e@UqM zWsZ9ZL4u#ana>doP=NREtsBogW$&X<)}|318*pT=f)TdJ+AWQEp_@fEy9yIGgSk;! zt@E(?L$e)8%2*!sMj-;)FE=*byE}4vaJFK~|ATN3YCnm5Ze~{dbt}N5!R(Vr;JT(V zirK5T-5hPOILX&B+d01Ncw_<8Ipi9TvX@m@7s*<@bmhtw-|3!Ae6mu-Um`ERnt@L8 zE7{PV{E9#6*>19q9}@16TQakFDe(Y2>59_<316qj(S1aQ;isQayjUppGlZh|JgD3* z-O%1Hq{afeYFfEQfiYzK!In~M?E)J<P)|NJOu!u=DkvWBx~D4YV>wm}eeO;PDg4dP zFb^y}RJ1w5<uK)XrYG&)+*CEZBm(HIgGTXnZZF@0574u|vmDo2>I_;d;2JO*(b1Qv zZFxS+8WxdWTU-D<N?Z*TAw)<M7Mj1cz)Fw$8d_|uF8ld*_gde$9}}amsAw&9dhRL{ z2E1Lv0*aBL%sh$aLSFBX#a-A+ODQW8+*I>q5YO~na4HLSd?~AkQF{|OSggMYwtWih zyMiZtyB7DzPiT^Q<H6jY!s-7sS7#i5wzg3HBY$?euNetKjK;j$OZAufyCSuH&%K#Q zvzf{|EbyUzNa^4(5<ZixUF}HsA-1_GM#|#!r|C+<P`VZ1^|Xb$Vk8xy!^m?h!>n{u zmj)jhj3*^ILwgn8PQQEWv$>gNe6avrJ@YmMfAYENOD0w7$>bJ_lOQ3NVKR5#V%?Gq z9p3JT46V+hfA^F8AyxIj{dE7Qy-!<PEfA%oJI}lG{75?7nn#q@C8&&Wsm3BEIDYoM zCRDfK<V<tfD{0}^A-|?RDkpf(^--N+(!A7)h};qt<s3VYfYe^k0Mv-8cTTyr)n)2d zvq4}xI!ZNSEuu^#0?7Kb9uqfpTAf>hztvr<=O7E7<9Dy$vChRpf5M^u=R~VOhm4FM z%9u>^%m0)Q{!L%F$vSx^aI8fucy^EwIMH+e^ByzR-lu2X@4N7kmo?;N$S%lr0R+!* z@t18Y66u?`m3B&t-vhK;i%+-#$#LPd`AeNRngv<zVQ{JrqP(f@)=mv+pgCJb96{E| zlK4=5r{tQ&1Blg#ny^1S*)lI9bjY>@3J(n!J?RCi!c_mbJAF8A5ROiF%a);oceHMK zs&H;F?!jRdzdBFfTug?bZZgye-cai{|FlM^cS@c#jyU~@|G^;Wtzxe4xDK(QE(G;q z3T()A(Lm0qbj!C>r*F4c622~;W?fl#tvrl*=k0A(UyoojK|nHr{=H|O_=W4AXm3u% zqQLwIJV!}&&!9)^rJK1t-S&BX)rqQ6ZU~ukLyxc&w!k@d$w;Qr&06x%U$cfRz7}@w z*2I66!fsm1eID=<Nu?DHW#4y)iKKW^wV!vPJIxIs=}P90S`$I9yMc9-$J>t4s)`K4 zlZMngzB?-pM}K7F<*P7CXQY6%(7Gg&=P`)fYAL2QF(>>O-bU@Y6K$Ldf@al;)9^xI zpvKT@+O~vm%^WIqt}xxZyV7%e%I_?Fhbr<-60mad-Z51{I6zqmOWkw}dmUvMx#8mP zrXAD(xYDqvE04@_hL*@jkL7!Y1!ARCNcT^OdKQK~>E(=|lBq;{15kjxu&x8NzbV1~ zDHlnzSSMt039}|^^v(L;<%Ef=^U2en<tMWaN#pZJ+}(apfxe|Z%y~<I7CxGzW+mG$ zPK{yb`HRjd&!0cnLQMC>aren;{z+;2=iPoK1mMf#0umXg|BE1b6Q*zm$(h%JQp;}% z$;kpIRAl#*tNoO7+X~}qH=l%3uByGi&O9odn0mbE@-vCP^8HND8g!JsD7Purt;_s~ zNj~*#$#rqu7pjLDBM>k+G%eQY&fmShX|o;TdNQsK6Hq$$wKD$QmA>9&hxZc{u%m9j z#Ut%6RGOdNGJzz$n)}tEEXMYYcfc{6zC71heMd>++pZBDsbV|>5R^a5I68ptN8dI# zprYWSaIB^k;|X9AG_2p?eK!DX3bObxRABwRz9xjmOXQfbuALq-wQ8$3lrXpd(QTlH z%Qjs-8rOiI8KPq&GcOquK8ua7hF-s*7je5T=}O!9s>v>Qky6AQvrk9-$?DX8l&s?d zN}aB+JcozfUvt#0&lzemEw379*Tnd2clB9~9TgyQ$|sP2(lGq1Q<LM@w@WVKZT}Tz zRxEF}B}ke-vfey?oSTR~+8$?QZdC7TT$+K}BOi3$4;bn79!J$_`Hu6=rhAoXG2}Cd zzmGK70~758KJ4|TJZ67WdSjyD|L#-o+r4R^tu&~AmfyZU+eBm08uk*GV3q?J?ERo= zVa%C(n)cameq}xDmGjG^jD`%$2v*&^;Y@oxSzEw<p*lEQY#W?*2Zq!!X=<wh#F^f% zmT+WkResRTiZ8##YW0yusB8EPtCRHW@2?r8wFYnEW9O!LBS%O@st@j4Z6(fvP*nL; zNZ;v5E90-rP|vwHt&NxZUaQT07F%={nx+8vr(0rc&nbS#(7!X$`h9i<FzG7**@hx- z2CtS~8%Z>pk#y3-&L5j!Q$VZ;57R;2V{v)GuA=#y7rMt!9;cz+e)&5`uJE%tc9FRH z4_R`D#j7m=j)3%p-_gVVprc_Iet!ivF`9f%n5xzJ5Das=F)jQ*Nm7=?Cwed4Ev;cf zEm18uUq6^n)PDY<D_J{EMNx~c<0lxXwpvYX$bAMYq`D>gU<V!4pPBf+q_|mruW@I; z0jsfw#!kBT$|qycmiqBv*bm~$7OL|JNM#-K+cidB2CLw=UW~CFGEu|V@^X@`W3$=a zR1dJl4_L<aj3D1F$Ru~Bm`Exx<Sr-75<rjT%}45)OfmCVJhCzVx+{Zg5Z&m>+C){c ztM`^{UHkr~IH|j<4Dw2q)vX3qkH~=BIX;e^?TM!=5kn$;uguJY;#xr5{)+|LP+cU< z8T3*$?klm@5FG_|kL~r*+cx2<yESATQKO}EtV;9gdCTZcJP?!INxcTk>zA2lH(w4o z$lzTus>r<?e(<j<<S)^$f8xv2xi=-pKVG{D6Wvmvy9Bl@wM_>@Zm8w>K<kR{c<1~> z`;ERNcP4gXrOtO?*RKBA&$SnL^LsDy^fkE6xZJ+1GYu6gjWU9Vj>rn`>G=9K71Wk& zBq87w_93puM$j9UF8>^_pzX`hCRS_hQkSIx%59wu;s(0P2wKj{!WOcSBhb`ZxNoL> z+@~=y;mQk#^KECvG@M1_GhEYuD2=ZTGC}-bV==CkpzTbRx%Q)p@AswZ#Mo6P0}S$I z{6ib|WJ2r)Ly^@s??2~U&tgR4=)Tfr_;oWLEfnKHbCJeOel4j8mu($YJ6+<Q>bB|h zF$3p@brGzSxEJAS+7rRBQC&qqn7uB|kx=heeZ;)Wil-1>JvsKi^I@gi{7<!d<|2bG z*)0--hJQoqU$L0DAFyeGJk{`cb6GQN?J{dh6rOC+MIec>HmS_JD+vQlzPe=*t|om9 zDt#%qoD2J@kdN*^Z~&L7=}qm*{2Q~5U!nz$^MmtC{726{yxmn(+cd3Ek+ap^76{e< zTacr*u}0?mA;a7L-_}6_uBfD{mTBs5Cm$O*@MGZ`7qr=>z}M+8OOr+C*DA_{09<3? z8f{671?A2Z?Ev{`%LSPD&CT;IW?t>L#56X8G0p(7hK^iaPU=P}B{RW%=U35XbXAEc zL|M~NMqHp!=p*F<SixYk4iuj{l9hG~3a_lh9+n*9WB}_4olU7OC7ZLGL|rUkNa{r_ zA5!dQ3>-}RBXC;?+X}aPdTrEY#dDYn<oH75Of#1&zBEdMn7^SCO(rUG8@y%AJF+e^ z<i;=@*OiG_&_aKx?~O3%Je-M7YEU@AkT_hYpuiORJOY#j=cf`wNUv1p#9g6cQBdLb z3kt;EVpv(p=;APcl6vx1Q`!B6NVik^rp|fNAsc%-2;*5l_&4-VC3e2r`OuuNe7>T} zD;}wwH+-gl^T?53`Ntg{Egw~fc9Z(F-+x%+%${qf1fc_q@1X-Oigu|QnbC^J$+e&C z{NGJRyJ_s&6WzM!zCdO=q4f-W;JoNJQzPm^JKDhi#m8i`dCy`sTUYZ#uvAloJ68i_ z+1MlOLqV;$nqMoEx0ycTEaG${1TiHZ*Qm<&7|gRgVK3YS6eM8JHxFJv?bKj0QD0i1 z(SY8Nh!~fXb@odA_Q_t55Z*MO;i5W8>4wU=1M;EtyIM>aWNObR#yRmdhl-`RS8UB} zcCPb4tch>_G^a?s4Co!T*as;?s3vNQ_O+wK%fD7nlu5gnB}fewR3H)`yEY$l+kUV= z!v!{q3?QKNyV2pthN@Z4TVd^%Senb+F@WoE(xtgZP+1<f6=1Xo9aeAsb(7anfiyR& z&?gs+>%#jOVCVkU6plzr`;r7O>u*pUB|a3j?7d~`+9`W6TN?^mR2F(T)N<r5AH~Y; z1jQ~{5em92Vobg|35q>Wavamne?{NYV<es%Yn)b}IO1JZPkVJ_DM(L<7uGws_zP+~ zk%_9iT@(8xoF6^y>?o|hND6I!HD^zuY<$$bNsSqx{H&6ED(0Pn@+GG)B?`Y2_W0vP z;oTdE+vS$edxGMN8qH3X2|G0H(mv6|-$EH7(SMGXC;qDZ5KUgdi8|iL=*rNsA6{PV zY|>t<dRJ*zXd`VR>Kp&cL5$*=Kp4D34pH5t`W5734;jZXL0SP$$u4A7?ksP8?Ora> z%422yT`#^Yd|~b0=-ZHltdKGor*=!^q<A=Pl<wp~Gj&6KUhE876@-?K0VnVYwJVe; z4E9i6kG(Eq(6f1$HV96R%p>gGs1Jav@9PLU#R|<oj;Gc4$crup@3+08<i&(M>Sw#C zvCPTp5qyq9vqH8${wqy2>eWqAe$VuNu0|=wUD<cWt<oYi;Lv3H(#G3wu0YRV&S-GC z%l|Da3F^`IOOyor{`RgP-W?gb0qcFYczs2+-hB(uAbT@~6GZj!5uxbl6UWuJ(rW!Z z^YnLUI_=W<D8v&_X_tGWlwuZ}zF7WIA#(A?b&?R%qyYWMW9Lis8P*Lg`Dh9XJo$Cx z;wn>>0t!&WTv83L*GZzT7lweR4+Y)m>;rl>ZQ()?rQJjis@e5GOPAH`hO(k8zDHkx z6fY#z*&Py2s4oz*b)&idT9)(45!{>;uD03rBY3kpc2uc`m-VF026^dHS&=l(2N_bY zTH0X3#hdvAWO~;f?%(&DAL0pR!u}C{C_(yRWd!;0JfT1HcBb#Pj^ugtN7mYSU#3Lo z0)ACeUv?vlYWp>Uv@Z}(`!;+SAL=`PPq(>nxw?BZbtJzmZiHC6nPJTFx#pVJVDQC- zR#G3rhi0qDikQES$mgG3Y$vq3=8?O)1jF#RUfB|t&UpT{BwvvKMsPKf?bC3|ojaem z{7m&<ycVR!k4c|^a&vOViRP<=4xEdi1&e_&vd|=Vw%V*y)4O+GIAPG5to$nCCmvD* z1UL6~C~21Vc$$<+pL5Oga^$6ZbmY{Xt|^ibLI~Oejt&U{3?Y*3c%UtMyP+UY*3ELF zraXKi5a?HjO>vet*DDoG(D;`ZAiuh+%*L7Ht-T8l1628<tHUdEkTfsRvj}$ec4+$G zj2On50%sjVyn)5P^NrCXLYTCgE!BDah?Mzzhzp@*JtGB%uBvb<RcNV$a<Qb!c>a4m zE>tqtRNKa({bLQe4iBEad$;#=8K!B;+x9NzK5@9xd!ZbUdykpzUF1?S@vKi_sXW>m z6T(RWo?UpOYO$ZgBzor{t?mogQ<Ez1ZP)CK=)c+>zH!l`$5r%e;d=1cX5J`q6OFuO zwnEt*A@Z=!bIQNF=;wcCYfr5&A>VDwXlZGsPv%#z{mD@5f8+VknqNoc%Tq1dzset& zl?|Y7AL^D3fe6Q6P2-QwEELZ`yeBMVeV#wObZE|jxv~5CV%&Aphp$r4CM5EKt@+bS zQRDV{3uW#CfwT*x<_doN>ZSb0%JlN>aJ5LEtb_lc0MBBK3<Ya4PUE8-fCuWoR{JLC z3Ut2!(Ce=GmMH|7ym>l8lXDra4h!l}B2h6C?h$)Ep%3m@eNPJlXR3n{%5UP1skyUN zobPjZ;ENaD+pnuspTc=l-_?P^)%C1(?o6SE)zm2Z=A08anz9$B9nyb(GB2wI3<hqh ziLMqzh_Hr1C(cQk-VM~UR?o!P#ue%gw?gF#Dyz+I;&UbG{JC(i`FG-SeOY4`v|W?S z#OLkItL9J7L{T#?{vtI~Tj)pr?Rja7h}CaXo1j|;4#}>)*QN>{kf-eEhW%@zq>CrR zUYpcy5w(4W--fuAtdx6WtPiTMj`D8*f|Qwt?ElPolBcXA;@Up6+T(9oZfew<dtu|) z-zf7x!;?cbiz#uc^}?a1PKirSZrjfr1sr*?vq17+iS)m+ol#8Gh9)W8t1oY~$HK$I zc+$(oWe=ZT-^Bl%f%f9AD7|v=tJ*nYa;jVYT+(B+PoDjSL4$4AEj*=xqBRK&i{Ej8 z8|rU-C#n<{>-T~w-h<YsJL%L%7Jc+UOR2;4a%(T2VS(!j@4T8Th`~2RD(!u0<GOSR zji7FkNNG1bs(`m5*z6)^-cfHbe*}RJ!B(fwen2U@52QPaOP@3+$?MZ9-2+DDgomlu zvG07xU!2LkSgN*Ry6QdPI<-!18mPDE;cxP1yim}0VX7H&<+c~M7N=&g=OPthoKK2H zsvI7+?n_(pkyYErptbY&TIZbPq2<WN6-vxV=5TP|r-&EccuSc;*LM2aQ@zNRqJYX} z-<BpXqW)9lE-IwwW}m-vq`}!{zzOVZZ@zUus`_+4-a8l|!{D*lBN!;m?|#7c$>aBM zU8{dd!44fc#%HeXF7eex(hLuG<F9Acgc`2=LBnfHHFA)Y6bU`{W#G4;6iwLd8Gi_q zmaLkW<L|-nvD{8Xr^t6YEL28-g@(aLc@aRrT8J}{&kC@6;RG#D_57<7r7MAqYPp=Q zSY2WJcA2<47<Yg*#@I!=VT@*CoK`oGv*8Riopzu55D(KH>pF@3!J>h;#%gO3$<&tk zA4kt7iC!<99QW}@ZK$qSPwkHdc(`Fh`i4Dx^FJ!hEpqR`M*X*yvM_$lb=wF*ByjKI z{Z3}ce$@$rH8Y92^`@*fb8GeK+s7|jXNt3n@8v|g^5%X|dknp=;_VG6wD6{QVOML} zcLEL7B}wfCtF@F;mgdVu@F-0P6!~k$q`ULE)bk5{4}A{I$4jp8zNyq1#(y$V@l)3< z(N``E#;ykFmTiW@)@R{X^!54Lw_P7mGJGPOG4*%aaW#HBZa}??b0-p@^y=6)_G9$a z!@DE9!vmuK7v%mG%JSQSu0ONFzUBPg-9OXfz=@Aa#~WfFqq=Zf-B~L*n8{2D&ow^X zW2()T;tnIqiJVoW3{ASTy1PHgU+MxjrhfEvni5rqx+i_2hXC2nuo3({)@Z6P?Pj@e z76l`2M2b}!Zy2<Oj+)*kiHk^#p0mwMaLn+lx*LH%44$TI^kJYD>pBy;Y1#y7%MYv? z<`me?P;Q~#A_XN47R^F%zv)U0UGA3r6cY-_>1r%M8Swg(i<84APoDfe;>VZAFLosB z2RpJ8J-*X*eaT$)q?AQJcJ)|3^7*B1DQeL`&NJCWJQZWu9bwyy{Rr&qfQ(ALZc>P1 z=4oo|3cd8WxED*wL-ZY2uI6S_Vcha!cmp8Cw2!Vv5Yu;q%_8b#52ZBE`~RJae^Z1% zgRPR;ERs%xj|iN>{!zZ(l@ry`!KYXiNunaeC$al};#9f4Q2G6n+r9S}Mu=~hX)ESy z(H+;=zNtE@uX3&p%4vrRba%%RmW*erR0Xrue^HO@;wmwxM3wE~FIQ=D=jcTi%6F#d z9Clt(LwyKwr7PAwAT7OUR%+jPaqeUB4i6da=c=OH{z%Ag3DcT~%^exFrQNu@1Ly?0 zMp90KPrD6}1xw9=A}i{-tc!@%6g^G9Rp@#zPT?N@_1KtG%WX2lz~XB^xI3;-^>$Eq z6Sh}rP0fMnog2)b$M<NOj%=~9<Q8t9C=v0j`(b~c6GxXK=W6O*kns?~yj2GxYNHLb zM8*spEQDVX=y7Oz1P?irQ|%p-U~bT*){gfar`Z+))z^H)+kzn>7kO_EZdT-bx8lE` z`tp1%ajJUfgeKLRv!Q=7vHF+G@aHJzD2;D9mi*w`o&~@5Cg`M#+8@Vae984tDy?d# zd*jbM0BJ^}qsrOozgEZp;`3rE?Tj-{VVu0@n(CX1l~K7;LdP9QiM_z57YE8lPP+$W zBxl(8T4~#yMNe3W-ocOXQdJ|eMQXvKK3Zzztv3+dC1NX}E_6v<(bz)IMtbFBB)w^1 ztuD3~<dQyVn{hA#NeD}%Rsf7ldkUKt9-9E<!2PvtIEClb*~av6$c3Zf&8-3YU%@YH zty}9^%-%!9;uTll1ynd>goc>?D!YCE!q)`Nt6Hlo%=wJz@YxL!WO;p`g*E_sOg_0= zALktvxn#7$jz5(_@`P>NXLXC)JzbT%!>2}MiOsd!J0~#AT++BEbd+lVYo0?ORpN%~ zy;%1PHAnISQTp?%x+nMxLd{q$PdJAK;Vk37jW}KOj_i)9gW@y;xuf0N#g^IB^+<z% z!OZ`McU=n?jO`GMOw2s-6Z)5qUk4h2*2)wvO|_xZ<e<Lg+P=e)vhrnJPvPj7;;k(! zrHCbWE-W<Fe5Z~$)g!^R(F8#e>OsFd+zFNQow?tXaA|sMF>9)^)SvEotn0GVRbL#% z?cMt^<%p~35cB#nVSii%R;&>^lb`AXDI@zFGnjH(8=^QzID>k6a1k_4xSA#ZeScGc zWkmhv;N%sej>SC3uq<wq{$g*cjbJ52>_e^nM!k7Uy|nUmT5p%nxRZd6s9&ExcjPW_ z=0&pO8^!r5=!Myn`3JEClLA5y-+QoWJ$7fUX^^VXpw^#7U#4}jD5k*N`vI6Vl*9*y zp<auJ0fgO{^n7ZhCs)ta>He+r_u6=JagZ~q$eAkvFh6hXiW6V8!qzgVgOu$&?4;*7 zPf|+Cg3td|q*ud#0`(t?>i;sHe_;InJ^U)rbNR=)vrKAy-B*E9kKh03U2;qm0vmg* z?Vz%E<Nv{pU(rpU0giLiPWcM+e!HRk9wSsuMV*}Kdg7z)TcI6oH$9m3sdSkw+=9;- zs@#}k7QAsYWLG8-#)BODhk%dqw$CJDGjcUDbmO#S<O{?4>j7g|ALJxdm)$A*x*V~l z)t6=Fmx+8{1M1fJ9qd<?TWJ~mVaUsBkIE2yUm!BlPAu<XujQihkPYDFG*!`}pWxm- zbgRy=<<3a6mwEG^Y<@>Ql#oQLB<)?Sca=^Xq~f;?6eCt1zDRWJ!slO<Cw-ZpdmQQI zR&-r&$Erg1w5mHBI`w|rY^--RqYQ7G?b_#>PswE{_Sag{Uaqc*Cw>@r77KtK?O*Hf z+vLb(?{$6o-esbtA}}$|?d>;DS)S;V5#7%*D}1wFJ7Fa$bf&IQgzqfd_ji2eDg4=5 z^H7T69)(CB$1Owd3D*4-i>K9p4Bf8BM)9#1?=6e~wgaO7(IAxT3&90zuq2DyZpLLM zA76GprUY1mSy?sT+OyJ9nmY;&g8LOt@E`7ApD5v$r|EH%H;5qP3TmALjG>W{@4FWP z)0ewcrI5F_HVL>w?gdMv-YgwADb2xc-5upWG8<|x&fsJpKNdHx%C1i6r1=95hLpxs z2*MK^#-SQs>NC8xxe@2t5`f`4y`E`tw?5dnn|g)+;>qp`<Ty;^6hXR~(+VlT3Wl#Q zGjVg%Es^ie$ZVB-7%<0H{8&uZzKZ&_bsEu0N<C9fxXwTlCH8-+9;CD(yh|#%X3y%m zJcOIjX!XT&g`{z?b-i0e^aJ*yZ8ziy$DE$}Xas$}vQH%YL2g<akm(BWnKP(HJ_8;w z%XZspYtdpF^UDGG#hrf>Q~y&=?Bc)UXB#>c*k8od$!(>4f0nNP^^Y>*eue0)*lSXJ z_vz99ai3Pz>@gOEtNPX`?Cf@g3Ra3+*dO;x-GA~|-JmjUef~G)M9rp{1R@v&Cv~~u zDJA*VRsglc_wo5UryYkelswiE6<~hBOPa|wxwXL>-&7=&T`fVCi~f*=*N(Y>{sf&{ zpjq;|gd<Z+9O-=_i<1DWNOFcEjAa6csjf9Rw8xt&Tmrh%+HAi{3%^WLR9$R;0q#Bm zmxI|oRz>5+By#oTQ%U@~x(Rl-*K2PY38Wb~G=%h`Wwy4Gb22KPBBo<+l^gmNu2i~h z`7ZVh)T&sVQX`#zqY#S)eEilEpc@E-(eHApSimHeQdw9BgKDB*vvifYOv>-hewtyN zVeMGzTUxE_IYSKPh##|JB`rJK(9KW%L{%S{5im16K~s4to$WHZI;`w&u(nw{KvT=p z9vS-h@j5zEVS2}Z3dO&}lK~%|(kb)9waC(TV)CB$*!G8RJqd^Y(Ept=6G}<h?W=uc z-_LR}1U!z7D!-f|>l%M;NbVhfsp}F#MKqH$jQ;fW<d}UFXjn<~R`Z@m7i5%QhYDC# zbjri&LS1}m=+K42IiNXJ{w$%bt^0*dZo2VEmJk!yp7Vp1mg?-OrQWmdF-B2K{K0Hf zW~KX`v>;au*;?LN;9ZU`(O<8PEw#S1+5$ga+VyrJV$I32Xo%x%-0#EkQ#bMv|A-KT zLq}&m@kClTCBa8)bC(Zb!_VhN_O@SrPUIEuG<O@hPJgR=mI&lE8{Y5>4%mHEU+)o7 zf1AG%2NP<^FBSVvhr_PkCU5mDFw9!i^v^{cnEJ$_B0D=sBDoI<657Tr=BIUUbVTtF zT|!5@4QZ~=Z#COFa(!71_D$2Cx-chk@_6OgxbckiAI|UJoD;tYZi+e1?VOa<^a8BC zb?e`~)VTd-!s<WN6Whgc-3~JHRD{`;V%wk5;}>pS0ixD2`r43^EOb5f)z3yNq}=g^ zT%l#Vzj%Mkpn~~xB`wp7@X`j-Syd;Lsf0*1#^o69I%bllW*Z2+47WPc(}}|s+1>hT zLL_f(WY0WyTeofEjv)u?83EbFtt;PG$a99R=2r<b`As0a2Kcm1E%j=AcurZc+fmeF z?4D4*H+v#9z<QA1fHd1*4oH2+qIbH{CcnCj>xMS8U2K5h*Drz(Dp&H>GI}p7k2%MY zfJamLFV9+8L>f-r6ScV}Yc`&R>@6u(b*#GiuFp@jCNG#cI7t^9)EofJQ;1Ng6{BIj zFeX>pK~;X_#oTOO-8zl@%1AzdVg0TKAiGYQ^F)Rkbbey>A=C}N&&(SHwWNcjmy&uv z(RB}1$5Jann*H=({pG2!?yYvWoLn{iWIkTa6aS8P`|?e^c*@klaC1=`D+3uRsac%N z@*jIPegScpH<DsI3d0S5CIw$3k3110@ey8ov{J~%cO2-WE!mZHTFNwC&RHzF(AaWD zQSl1(-2h7d{LDbIwx~AgyCT=5l!O<pskmgwelazOelR0XbCxhnX@&PTQ_fXx%$LB| zKaVUHJ$w-$iFwYH8!4}H7VALv9Ii}xAHgdP0jEVacVIQVqzk7&qLDCt2HhtkIasD2 zQoX<2C$nsa`b*|)FW2d%GI<-gZpBFC7{iiHP0|_8HCR9O%cta%p>BjtJJD_6c(5zd zkbBbVqISe8d)aj3$F1ZMt<4{%hR0$`r^+jbF1;HormhRkQ@~dRn<_JHvyiN{TjU0K zO|jm3D?Ip+@KPQQziCSvUN4I52J&n~bY}-@Kc@TSde61SHLeSN!_>p)B=pz$n){Dh z2E>7Z=(4_1F~zHl3<+5zch5cE{?$#c@7{mlssC1{kEjSjaZwkMoQ~}dU+kmp&$^B~ zyL|o))%brS7W0cJ+t|r``X}3ZrWcLc=`qu3VUwnq_?vE_P>0XmRFx6AL*~w)ayF$# zI=c(ZpM{01h}r{1{VgxT%*E@EO;rK~9g;EtQy8JTtotwLC|&xZ9V?KZ)Z{9o&AJsC z!J98Njls%BZY<3-eofj(z7cA@^CGE35i?D(bH=&B&Y>gmbva^tu&T}&)M}X<Znyro zOTOKz4c88$zCV0?;~@^FH<mT8LR+t6QiGL;z_dv=W)8t_cD?2XB<OoCvP@lihl;Fa zD@jJr<U?EPE5Uuc^FQi=)*RbbiyK-sUDsn$-)t*n2WA?qtSzqe9^YQNIEfTL4_!`p zx~_Rz@xeON%uXF{SwSth>n@!PD_fz&Q)&UTG>9xo=!D{DDx~F><gn%y=8dND)*nWk zr+Ya4M9}&Hxv*U`j^N%C<X#;T5h%UpUB9_`rLSXT@pymh@-{4QTgI(S=<f_eJ|A+N z6Z#2qk4oG5MqHM>O!xMG&LriBQ4$}+W!Z}glLy`Z@ssE{lUMQQ@WLGyClntMu1m#- zpJF)+Xrr&;j}^~kK+x>V=^2f3{eAULVv!k_%7kcgcS5~D#?aJn9@jucgwsW;aj#KU zA-TaSs;8gA^EnY8LSNt(Q9)LY8m7mBjKjFi<NC9D%25_%6y5`-<$HdDE*Erl;(3py zm@)pY>W+jT<9FU3FrTivJ3|>e7BU{ez569bqBxl5dhgxS;0JhU35{kOcCrNWYT&{G z#VawNWAkz*$M}=n35LG>*ZRwz-3B!-;i8pe`M`dHcNB-U?yoBWWIk{^$9U08$R{i& z-V3sLk+VWwd0UCPWqghSBVV}?&e102n;_4N-o*FNyGVe0$k#*lPVRJ*%N4@Yd1Smu zhAVIarW!rmhg<EblMCvO-v<u{7}C6p>xOGWwl3Cj8wLwB;_{rfbVtGhejsR6Ypvp( zmE@=q_R*uXe@mu6axkNtXN1eO1#310+{~rW+uVF2f+;6+&?`bi=?}H()g!=liz`<z z=cNF9i$FUBHo4IQ#g=oM&4!SRjZ%lB)gB8cG(?Rk*=BfX<R?k0I+V3|Gi-x;x-4}} z2=RAmO0sjG3YVyWUY{}IZ#{>jdw@YolzN3-bNA)WI*_3v{gYtWm8azJZ3&#)Hkb9K zB0)5T<m2;R$M@adU=c2L(n-(iOPOf%hD#_?gl>JCyvYGJjD|2`WyxI9qUB@hkWZ%s zh%vbmTF}x`T%j7a1pfdUpl&KyA>Hhne9DprA6oKtBh*jv=#ZMZ8+x-5swSdPfL?<c zrN3J+-E?lvv)xbzTIt@jQg`l!Y$THCU!JeHSO;yIMC7YKS%>2-<1vH<xtjV|a)0e} z2TcV+1IBBSYis9#51#1LLmp1&lv_(vjUk4HlBVQ7%Vv~nU(~nE0o!<#^*VNUomo$7 zK!Gj|J>>k3HKL6}$O;fcZ2y)69{UP>;`sV@Non_q$Zyvt-TzaAZ*<dM%;+*?=-o^p zOz!h`#PMyJJ23F(>zp}9p-HpLXea)|0x%j^XCpcUH^=I}0Ub|+wrJ5byz+l5pJ~ER zJk3qZkZ|TvL!E@LdHU%wrS6e*L?sb5Io?uy@-Rs8v7G1W^A_VN70nA-qsNryW_QZ3 zIZpLv*}ar=QC4T<S)`~6zDx+MBylY6jRdvFuIbO1TU%futAsqWIYYu(r(3N+*5ogy zf~eCZZEz&nAvkby@*90C86x%3U9!lU5jsR)-4AD)tsv9`IGklP&xx7lI%Ocf<@^Yt z=hX~yU0n5^HCaTt@`fWL+=bArJd*c?4Uz=sJd^VFWGIM63{f*>Ls>mZt(712x%1z^ zvb^$?h?fyp`&#Jt{e1Z|kcad!b@M^h)gz;eO!36m)G9ZqYrd^iol6=}Qk_4QT+_j% zPl9dF3BoxdJraMTJlz)*gBbO8w)5@a=Atn_{!_w551ITT92%VZVm!|3IPczreTP+^ zejD&Y?YQ>LO+eQ|%-<Ztk`pmu4FMXC&APy=T-?OSlTzxauD+&@_6rp#z<H8`yjtoj zUIrPb8d+&ss*1ZsoxIWAin;!B+zquhHS<o)x@RG!w$DJ1Mc&>bEfx}J!AmtztIG=x zTPy4$@l@?$t*m#;`P&;>ky{%!ZY1Ut24&Y#*vLAu>Z37iZ9V6sk4a$m1!uA2h&xci zH|X<p|9M6iE0MYxC_l*`aH^e|eNpGOc^>ZX$BbpUO^f0qB^GSxcd;veWDZYs=dD7* z^{}4%3vL&uQdS;@o=a$)idk<3uymKdzSSOPAg<Qb_Y`P-ACbw(Z@74L<HyNirwZRM zAfy4I+|_O|&wVab^uv2<kZR0%^y5w7p^Y%7Jnhm}0Czq8Ubo4twP!G^ua+ezYuaZX zX;5!ai(p5lcO^x+T+sjV>t8ePhn6=l@9WTAbF@vCUw!n?LE!(@aU4ha-Hroy@5gQ9 zpZ+cphu{x+Hw*>g<_cwTUYkYtIVaDUl-<68VsW(vhM$l@Lgk~_qYpDqJ9}L4ZMV1? zzu<ggS6Gn!^d}{sX<KCkycipSKv(uM;(ctIO=UtocwG*y=uu;=3*0<x0g->Ss~wWC zoOnNOu)<aO9!!(p*fxU?ObShA=JUeBRxsAbg2<KD%1hG}cUEK5o~2g(%=nIC-O!H; zvD2Rj^9P!@`3@iT10|ylX(<t|lTXzB+7*(M3BUU-e+)8#x>=NRmg9;R@5wIJ++F@~ zF1hh8UznXq(Cj^804@LQ*S8m+;j;Jsy9VfN<F7sBAD7ztm0TH!8*MKsb-gC7>a5If zQ@ePYMC-nw=5uh?kjJ*n!26(=_2MYPmOsmJj7J)6$AxkGpw0E{+cDn~5*vRk+tAGO z?VIxO+JT`z!WJl3=k^^u#CMO3yG#D%1D{uP%bRcZcJ#|?{5Hc6QED=6p}?E-ht!L{ zbn-enB{Y@(41f5mlNev)sT3}&3rReycs#xi0EebmI02zO<{bv_H7(?N`eoxU`Y~2! z$ZS`0`p_Kga*_C_Hn}S^jeZj);sbW)>o>374%Vtri=#k(SH|6>2Pg2Ky3J&9$0H-@ z0nMx@M*}$Cl~&d{>^c$ia`+Y1*Px>xa3x(_?S<9@$}aRvSc=^X>-@aKYA)yflG^i- z5%hJZ8@B0r^SIl}OD1z!X=#X?2pS`wImyhr=|n%O`k+P@0XRgr8yh)RxN-x3L)_F^ z=r-BgpB)H%5^(L(8v9>f0Q|2|rl~KC)v8Vq3mOk3Z&F|lqdJ1)<c)4}(G4qI9Jlrq z%W>LS>}E!_p*K|hYAiMABBvrZeuGfqWcfMi9fQ(mVT-^5AdwYob+;t1*xgGl52OL7 zAusQ(Ut2uuTyxu`plJHrv8saFg!V{3!kFz2SB}fOQEC+bajeWc!>{}e2)i|%|4{z_ zZHC2fe_lQMEoJJ!1b=tV(P|Ao0P3$j{g2so#8^;XIP~>{oe$4@wDFxS^86I50@hN0 zRZ%?7C^A((L7Q~nR|C){bmh|_hMk=#8-#6+i-dV1XxB3BZic}WN^iG1zmu}cXedg| zc_faz?q`Ven02}ArA8RA2ud~16WhJn*hl<aJF{b((>pPyw^XoLWT`U_f0yGt<qrE) znPHh&F?0(MIftr3yQ*c{LoOistS&L@>tppM4U%7SE`L6Gq+Y?f0pcJgnPG{i7OsA0 z8k5y!`Fle~P>DCE(PhxN3fS68@ETEkc6G5(LfhvkG2Lm?paG)2fJ(5@)15*7tMKjd zXg`LycHzq-vEL&P=VIBO<_{XQr)U)>Kax1*8$EklwF7X7@nrq5Jd;(I3kW$U2Ip(O zs}AnG1tN)F8`rIy1Ty2cfM@mjyJhZm6SMicW9-|I%>qP~r&=p#BX5<dEyxD1E1wbj zOMBAOUxXBLq7uR{3cN(0{$p*T+6w9lJH3-KkDK6&M|J!uTEN$L_GL*%<}FKbP<+&+ zsf=)xs$eRtUBQ~J^+}C&KhtU<QOI2!XTh#c`p_WR=@ACA5>mlE4I8r&e8FVJsEXqQ zTq&3p6_!(lL81eFP9w=hZozGvN2^<|EiN|P>^Fw(^4wlfA4BVm{1AD|e+Vq464#=g zImX4wJ>MIZ>d%C-CHwAQ1=o1298+)lR#4Y2gZ%AW)3}~?bge>p*kRSUk|vbS=5W^| z+-gTBnGouDies{;iHmUWe4W5C2J*u0?lO~A!eWZHe0~L4th%#r`@>Wb>jt#Pf!k|+ zk*F-g$gNnfF;0C0t}xjk>|I~^MJTyMz8v{w0A8j)YT+MlIb<8GZyMVF2x|@Xj_A%z zOG`n-4HJaC^a5^`GVgV3s?+ajTM@mpT0LIp?XdAfuC!DP0cjUb;mR0%W5TD_<HzaF zTjX*6mJVm>zt=WBy)Fn}*}c!)Kqlw5JKzuCC$MKwcoHzlx56r{agY5p=v{~oU7nT~ zkzRRAK3T5$^+^Ws8D@hi-wjGNH+2<b`PIfxR#w0v=4Lg^10ELAX+8D>aD;G@&E+zQ zI&hnF8i|m2SU^?O-P4Njn}z!O((k_S)i>ubL7zA+uQ++u;c5}Qhk=!<kr!BWy@Bb2 z&g#Q;U#L<EjkNP~QFrI*4pvR|<x>!1Uy?fC%DIXUwB(0r{`-bW#}+FvGW)PPB<HXy zIQ_61?+R}bH}~Rz&d7Ov56bT|Cyx3*4WY9|e8*kuEf0o$d{{FAxAnK|ikGUJAuG*a zUB}ruKz{cxJt_@U%>`A6J-0I;gsh7@;OFPJ!Q4=D5A_Hp%~+QgF(3!zI;ucYCj*9! z*FAYFGrnKm#GJ%z4&Jb~d@PIl@x2c|(}%-K{?Ijdm&<7|YYF+4vtKRu1>6-YG?Viq zTd&Z$-F2c8v7}9BvxS?D@XWlC!ih#pXdrRiuoA!Ff<!s)ayKtm8@j}DqO&h@So^R8 zNqMQ0Tm;`7=pg&S<qs0yAHv2r@twKFC*KCshLB&c<(@tCA3*iOq3^;rVanSTg*%MZ zT<3&Kdens{<i7vxn_tb!T^CF7bODXcp&?S8J`{-MVY|cfYh~IIcJ!Llc{byzr<A#I zPmAI*<%O#N`a_RSH!p;=){m=+=6HKE0!2P<gfWnECkS>85U&B~Ij2SS8Q5-C-hxxO z17W_h)%TL9z*+3^T<9}9*}NCqNYXirw@DO3peQ@!M!hw6yg}>Ow88rP!1|i>y^*s@ zi8|*hy$u5^@6N<9ajR5k_3qv$i5GS^?*`94TbU4WTO?|p0B=)CZld>C{DsSccga-h z3looI<FQjtjjP`BK?4=`b<bRTor&btn1iWDwx1u2I3<1e<0aCqAFJG%S`rlVjvZ+A ztRmMP&-UrQ2xJBTwyx4>ZAbAA3q8t15i__wUBrB*S2_ejUnfQk@%C3Ui(7l%#&@{( z%CH)=rCWfTKcun{6A4<|hlM6*cVzDSqfqqkK{^v9r10n`L`G%2Px%i&KOz>5uc!MB zZpvMM@~CI`|IkgUilDZDu4q44OqaBB%0sKVPC;%M1>`#xO=c?XT@oo~_8|X^Un1OG zr_1c!ch;+QKDQZK8+0HRx4BltO^%{dq*hr|`g)i^moN}-_C;TLK)R`ajc?P`wCu0R z?ZA{&|Cfn=tRafcDpGHZ|AN8$V+?spGp8tmTS@8My$H$mrtM4G{Jb`U-@Ry0!jG@^ z3DZGEc$TvI?GUHt!%GAC2Z@N|=Okj7Q=hZK=eeaM!=?s#ZEF4vx@lQCY~O1{A3;R8 z!#h7BgdVxQX<8&c2l*JH;;lbEFQbgh<5p~YTOi)OH&6#RnrJJ0^^eds`G1Ukc~sJS z*KgTqbu#ss9ZSvmlmks_jx=bdq?|N0=gQ2?)SMLrY&6rvAvH%T2XZ9m5v<H92NY5h z2XaJIBo#qH;Ql=Ceeb&WIlcFJ*5WS(mkYk^@7|wj2j$Z<D_xyh$8~R2GVd`eV@<gQ z{U%<_xw_x8q6K+-`vQ+T`enfZq_#r4hGwzq1Voai-ElMN<z|wR9{91)Ga>=24?pkt z*Z2RQ-=af*iFh||ej;y@TR!~y6WOeNfE!reJU~`PM#g4%WZ&Q1M^Or3DPotumLl9! zyk_;6hKkKxSIM~nMd8eomK1`94;&$V^VHxoL8B^Wh{sF5gy=Uh&(smGP-T9tGHyAQ z7z~N7aFt7{<Fn%2^2M5b(fNv)Qyv!e0`7d2WO2y>jo+rGDhG0B5;Kpl;%5mFUyE2x z_lL@Z08%=)fk`mnkBgx$*9ez&>GQgLL%y$oLDvo4ILoJ_A?Y(itM3LIc_Wq0EWoGv zu5#FRF4n}vt1WxSGAAJGM163_a#N4_`;aV0wmYDqY}^Q+8dO(n82Dmazy~HU!SW;0 z*vp=a)R4~gyp<(*)3nFd>X3lwm8%8`krAJENava{S&znVyMyH6izHv1PD=5QYZfd0 z6`xCE9vo~9S#;^?EPK9qDr6W}YMV-PP7SQl0l|GuSMA|LmpTJaz;;ft8xhfu^Ihg) zy!02PRm5kr;Zvxn=dL2wu=UHSFkDvIGO=ozSi?<7z1PuM^~+KBUm2*!&>Jrg#^->a ztE0Bq3yx~Z|DpgM{g5yD8j!C`cy*)Jwheo8zn($@f+!!Qza-*Kk1vz1LF#j*!|^89 zB*cAq+8slsw&p3m;_JnYc@tU551c5yOJNBg01W(4Znu_1j<fudPx9eaeN0$7<b|e6 zJ8UasphYYfG8TN7+reofS9Y6hF<9=ZYc4NhC1+rC?$X9ScMo(w2q^o*rE(E~lbrM? zo-2`DLPz`0pqB`<3jlGwIC|RxH)QQ&N^&cSvsI4$KFuab%OjTSN*<He6C|HOHa5a< zs@%48LLZu*GWBpwLI+<#8soVe*ictVtDKciVpX*(8`f!Lp;0~mfzKYs=0Q^9W*1p2 z#;K$MGVD}cPfp4?Z(M)thEc+y!zwECd`;fKenZDjJ9fewDo`S-mU)(Ezk5c<p04pM z*<L(o8u(rXL7Q>8^*bY6c;tik`>MAH2Y*Ua>>ImDNA(av09v(ou*V9F)8e&Mzl?NN z>gN0h-#cd!@5;ypp}WfR(vO2OO8?aFoO=Tn5l~T=)fM+f|DRg8PXv(nPst@4^Y7nN zeC13{m%Equ*72)*^gZzByvo$1qHFncE-S=WP|Jm`mpRmHmq7HFwzpE2OhvyOuQ8pU z%sUNgnZE7rRrKmmF{4YknC#V4(5CGd>qVrh;aOAAXNf+_<?IaSxu-rZ%;)1ZO?*bs z?As_E;!Q)Xk$_H3a>%exYBnoZEpMBc;Rzr7>cHFEzPx47)(-)}ntJ+b+GdYDUfpw& z_%k@x+PjHIrjp$S?BU9>(a?P67|fH~#|X`BklxT34dpON?()3vm5l1){HzelvCX@Q z{z(k^Ks?V#cY)hKg=!+Q8p%&+BkQRKVyiTr^LU+S)_ha#Lk~Z!{N$U8{IqlO(JoBW z#jH+O0;YI*D`Q+ord~L>9;BAaf%mL`>V$d8)FAB-HyJUnIdeGG#%K%0V(Fk9?J9`} z%86$1VKq{xws!dN%0P%u_qW7X47sw#`w}6kw4APH3!V$x^$doBjA><Wr1o51@l)6u zZkTOf`A>bxzv8?PZ7pxSl#1uef4^Yt+Mjn3^^3z$yyx!_R7C|Iu`BI#(gyFqe}RlA z*0#4UGb`}#r*^cY?|zk}>^*Tkx==dt-UT~ld3YJ*iMlvC&u4vVW?JQpy24k4=4N`W z<4LPUGSb_*KHmm1tgoAKq{zA;?&2ZGhrJfx`!ands*qqh2tqOt%Qik9TveGe%&ZR8 zpDfnvP_3cNE73oE(Hpc^E>%B_nbkvZS(OjQKPOk0)gYH&u^PP^@;wbF|1h_{XZpJf zioe<{2r!>O8rSc#SRJ1Zq~5fON4v_s;aq|<{ZKro`lZ&xWYk6*Di?&LzX%c{lx)Is zq8F<MUFjPuMBFK~36X^C-`NJab<O{pDZiB0SFX`9R#)X^<euHQ<VH%uY)oA>M1zHg zy@*)9;iU84o-fB1y#JC!0V?Iv=v1VWO-#;(_@GrBLg-0HHwk^XWck$GJd|_dfI}#r z^-b-HRf948B)t!f41#sJ%O&Z2zwvmCaTPzvwuv)m?0Y3x5G=(iir1@t7-n0TQ^nQc zDf68o8>ev|rGoVV5B@tNETd{5+vnn<Pk6_LFBd7{)V|&A{o7Lhliudd_JPY=hlDpZ ztx-EG>kl0SXv=>s(UTW$3=}(<3lPtR8UQuxOu>r(U8JwqZEsaP+S#rsZh>?KDw>^( zNrO!EpNLxJ(?d_(l2bQUjcx61baFy`0z6(>9h)Rpxk-F<G`ZyE#KVa?Cq&M)9TbIy z96=%i5%WYpL*DD4ep~yekTotDWF?(xsBgRGK;~R=TU&Tq<sqLUszj*Dz+azNvF0JH zMv7Mxcu2CPt|f5<!K@U_*pQ~w4E2l<SXnY8yZkD4<{rHcnN?g_^7)3=(l=(GHG|#r zNJagbug4dU&J=+2;WW_aif~qgu5ceUQ5UCIl!P-=ls1cY#jmOC`FEx<q$*)BoJ+?E z<K+l~$0VwnPomD)RG(Kx8~CIVVj1SJWF+%A72{+*%0RxD>ejmD4Pngd`&BNW>EET% zch<+%1?v3ZoVGKaZR`zu)l+^YJJ#oAbr45K0k#M)91SEI(Q)%WuaI>;ljPP6E35h9 zip3LOOP34A!kBnYJ}1-Pj!UVs&x0SN+Nz^jfRNMB2tiEETg_e@sU+vv7yFwZU=#I( z7q-I}Q<X>Fiy8QJb~kN*XFBU^IiG^OkN--Almt6G&OCSD&G@QQ>}-n|um`0&4c>Rz zN|E@|bT}<gQ6#_z{~sCnEN;A%jL-QdFZjTClOjF&3+vg9#@(0qlyAVrc?vti8)r#> zJN%L%H;P5dV||My{DDn2N5s`1Menvv6wCW98EpbPo`-}jRlho@!GJ{{SCequ3Jjf+ z=NV!6?VeA|4~iBmqmo;1?y$@9jLQ*hD1=66<5y>uK6}8|6V!6m4B^x>^UtMYxk%v6 zvI5-y$9y9%>-!~B12^!`v4+*dYaseG`I1^2yW%3}J13I$>9i--Z{tqZ_^`LvXdO~< z=T%t&AlbZJLq)>5qfBHffnOoCY+9$b7>Nc=TIxWc!}ls!#Xh=_;rkVq>1Vu15A3rO z<S$+``^pq*3S78kG29im+9|Tp)?f`xj8^@hjIuVl7S+b<^8GZOq?CMsCd-cj5zcmF zz~JGr3b$nnE5$vEw31!!5d0n4WWs9`7OTn+TjB2)td81Nf2-gc)ChGDKdZ=>y627T zM^dUs?oW<&Dmfa5?6+=UA6e^{t-;0^2VUh0s!}~NC^xwTuQ@?f-Zpk-ddSHAthsgC z;d-HfM^+cp)8J3dFst^3e$gKGt+~iuTR2UAb<75~ToP+!@urloq4s%Z=WXw6rB|19 zIU@UnndLMBbc?TxZWHvH^FY}*O0?^ZK2ujaow|yg0hBM;gpu;(jsaRvEq24xK~HIF z$0UF+{A-o?$A$6mliG(?(RL@mr@Omwll%~T(EMe0cOpU51muylzI>}Xx^b@SHz4En z&+T^40G=>%PcuMEVE~)zoc{G=9c(`pC3PL(BC!t0V>V+WwCYiHm+cORzLGh{dERIb zl%Rcm!PTQ$65d)Z)-%?V7)Zow0Og>VhDBQwj9Pg+I=YQdyRGU|dsUxa&cv>!Xjw2L zR#(h?3V@CagqV=TwEJpH?UE1sA@5ZbER0w6VfC_B;Dfl*$2#vn!@hvkl9998HssMJ z`HU;-{H7mOZF%_p2FNg97xtC?mtTF<92Br?nk?;I!#(O3$~#G)pYAO(OMOlNg?-vf za)mRJ3E|9x`1!!#X%6l(Wu?>*%;!}NVljfXlzVO^(w764Z7|$#3)4u98A|wczA0^n z!0e_Acqux;4T9X4Bpwm3yop6-@K$->vemDOEK*Z7AyI}Zo5MEltSZsrDUp&UsSRt^ z1Vf`5gO~u3!{n~ht8DCYB?gar;fE;r#a*HKVt9@Vy+?lZuLi^#PYoQ5W!1%it|acx zHozg=QM_R)NdX7H$Q6VV`bMBZ<5$~`V5%dQ`sgPQ<8&<=wW6m!yXD>VsR<SvuDb7H z20o|`hX|KNe)ED@H$lwX>jEt)cN3m7_!Qj08v%kb)Yss<^y1`v1-TR&=V56)pi_Lr z4T4Uo=x(~QcWd~S;TIYIUO@iALbUj*$a2_*LW!+KYR|9E-^a5On^Qs}3VaKkdlap7 z<8O=Xhs52nt5hG3k`4V@qw<;XW~qXf0Lz$m<w1vI1*(Umzu%UOH62NW7S$e(cGJgb zNmSs?a!-2IOe5<+Y7eXSJRm)@tX>tlK7Tf#pp&~|IlzPHb4pH=x^<~X?3g!&?1G?P zi0f+|0U1YHY+xGCr5At<k9PCFwUsA>M5d^M02FtbZkCtfyJp9%H?tfwG=c%`y|k=< z03{IXlL&n=&>9p=kXu+yDCShj3ly{9Bb_A7w&+!V=+H6QwdHLj0^C^Fhp;|FzbOX4 z4qM5fW|r2?q+bpE&I?w{r^zp;j1isY8-uooorDhj975OXW(5y8&>L5J=6S5|9?Wgz zVKm$w#@-J-(*F$VDi{W~-Wl`!{4O`GeOj(M1Q`AkUKrwg^wml4e1_Tk%6ZpOgrVGB zq>Xvwv@xN8!_m&21PA!*FD4TY2m6&$XCspIotVT>EYuZgSwG&&%mZt1F+m)cy=07_ zc4G_WZq7^6R7VmyM{f0f_Vb38gHE}84i(I3T-md(2XnXxt_lRM?mYX(38|c!75=dF z@6)#M$D?3t@6HrWh_q|iO?^fsnR{#Fdr7sq+uDc<KAiU@9Q4h?%`c+UpI$WRGHUN8 z$k)U!WNKvv+QBv^&CI}N7g*N^6*#1ql)Cvmmj<@>9YNh&S|+s_2$_K`eoytkeyl+y z{Ca604Yz=HN-YjcFbmR}afS5-Xk7w{#Pe8bbhi_6k%n4MF_(UIy8d5s<AcTS;*%of z!flQhqMl%Et(AgA{#g(|ZV~@Uzq(<NxYhB<v(4Why?8@Iq+F_?_`z+J(abH!oBOs~ zg4Fw;+%iZWIkXXkXQ?3W1^7$*+f%ypbgQdvEH3zJe!FX|GC*+25U5kvPxgOl&D`gS zLN4;47Stwo`2!31k%%Du@h>&2+s89dWhd|pPswI3oLFhXaoMnms3?__@P~063Of5W zA+UQoXA$vjvfA7a{{wunPFIZ-jsy#q(B#a{(#t4kc6=pg>#n>(W6zB2BkND)H&hW` z1H)u_RLxN%->+I;8CYgY^{AI4Y|KU;Y!OD-Snec8t7V_P7r~QVEoF_^%MJ0uUj~<` zZXgp(W@A82r(Ad)<Yzku9S>JA#SDLb4ElNu%gVjC9_D{_<^eyCmRkYdc<5*|&auhA zVm|Cs;{yBnObikuA9mdlmGhEZouq38P21=_1N8T-b~EEsjW|_TtwtV^{#_DkrHf)f z5HBHqo9upWP2=2-=0~NYs5}H{lJ^ytl6SbY@q|mDYp5PamKkQIKleW8S)1_y0`wb# zTj-b1ITE<GTTTo<l}~R2b8kv2fo5V3L~JazVL_<%PD>6=D=~Ff1brnisr^mfcBh0) z8ZhwW6W(Q*ena)exSp9)ZyMl=?8Sdj-9NmO5hKrPk!156fee4+H6uFl_hH<(BIH8g zQCRdtS8wLvl;^M+OEHnuc`krRbw-3Dj2-gXX}4{P7;6XO+YknQM}N_D|I2l1_s2u+ zOXt|an*n$4{%QVM?3Wkm2OaU(x_+(_qC?2b@s1hCBkKU0legkl&`}8GedcHtTI!3U z*H_3r0zN%=L{XvUYa6A;$0kY6DQ|pajZgGuOixXxht2So-r1G7!JD9<)l&to8Ha6N z?JG7<3BWyfsc{w#v3f!Yr!|z!_XyTJ39qsRuUBDIHvylTE9q7>{N5Yebo&x)smzmw z^rd0>kcq`(Jg|yWA+uqzr?RJh{^fxP#&;EegH%JrCM4!X&d&poj{P;@pczxInTK%J zRJ)@ic8vAXFG2~w7>qlt-HJd*n8C&#s(&Gs&f|WAE&0tijip)X({ovPCIVOe)Bpu4 z=-I?6VKb*OdKa#zmNCoL%PS}24B~#+&i{zNxZhk+2fSJUwLE<P!>8w+kau~m*AuTz zr^A_eUA$lwCog?a4DltO?juc&&n|1(X+96+tSSp2va$$PAKieuZuOhl8b8I!Hx1CW z)U$e{j4L?^*774Qk%ki)!5!Lk5_!z<U6YyLYHrY{R6>g7KF(@B;_vPf7%+#Ep0g*H z!dq9^=Jw7T9pm{MffMXuKQWc@rCBFReBI@Hb;7Bh%lJ}{-x{|sNFsbNec^IO2Jhw# zD>5S%Fx##oI>zj-QiN4fY13n~&asJ6KKBC5t?lHv#fzUQI5*1?vvYDdSrdbQIWfPe zVb@o;5A5C=u0cOgBzMf{pB6ZPG)jhi#2&4AabG?3TIK$~E1KS>kzMoA8>#%_;#a&n z`J7bs9G$7`gX@seq)IdO(=Ve~#~Akvv<oOP4S8QWtXnR5C6ydeJUNno19r9ZbYaK+ zQ*mvha3_6MGS371fJ-AJyBaY>@yw^6WcM;H<il3KT^)pu3VVa5hEuVv7EN}}Nf@I! z4i#!fn6Vu~BG)_J@;5v8G5jg9j1h-Kbl#^5;IhgJpGi}qj7>xv<hmRCTlyWbtJUi7 z<1`8RtU~*ja+BV1>1q4Jn|t4fn6Q)BKo?yiAOLG2bcO!D`m`LSwh@xmp!a^%Fx{&G z?`jk=T!cv^ZO(nbXH5C_ZLcfbS%0NOsOD4Nq1rOf&=dDR#;DmuaAje$Z|Q(2+FQn^ z(-a5hPq1=bvUMZ87$fWbIAnhkkw(N&NfB(Bgm@KG`247=zB4G`;Ma{+o5L}#YH+?Z z^>v2U#>e4HtQtNUleXVzGN#$n;mjda2v8!o(ScDUw0><Orf6mO_KIfG<&nL9O?5Da z6KLa(e%RbV4k_P5s`d1<|JDM;G4%j7BKbG!*XlB|mBFg(GuDD9*uZE)9DP|=faF)+ zrBlg0VtjYUGeRsqDb+Q>SI}es!u>2QGmsSHwlQ&aa7uZA(y3FapuTbMMR}={rB*ef z6Is;33GthCP~p(AXJ$qqrz&Ue5iN=4TtJJBX$J7Bdl!d~i3;nUnw~55HPHH3H0c*t zp$z24%LDNYr-b!wRo?eYPCobr&f6Nj<yiaV_QX4zXSY~!6E##;6VGe>>F$y3xLlgL z+qN;0?x1j7cl1yCVD;6cx0R#Ll-otMSjL+26{&P>SyGG#%uP%CJJ#Uqs!D!Bm|ap| zfO|Z~?AU9SKde-I)gL<@TSpxf9z!fx^70#)j)%1957m&$S?ekGYyufs9d_fYegvj( zc@82L?vl=~N68)(&y(l3*Yl^mG4m=ZmoP7ND1Y)<Sg>p#dgEnj=o!D^xZIU7Kopvj zZahw=T4eZ^IcKM!OWjK20-{TH9eHq$`@PLrr|aXuc^{}lh<@DR9z6v<PYS~fy0-wx zvFHv&m>uO#hEJmMs({AvwWQWA76N}?tDt(Num<@jqnvoYx#~Cv*ssH=0C)ki0&^n_ z)^%}^G3vyd>Pb(GXawj5<@v!x5j;_SzxiR`I(JG02}~kU1&S^~eWpVSj1_K8p3#0| zSlB3JQrH1Sk|3egqLiw-0Rn1KJ?=6Ozci}UvkNolDwE_2?`%RHM=Ta|;%0q0qoGhW z<kV@SPY%RW>YvR%&%vp@F;)Q-QFwL%?+UvCYkhc;o{%Iz*x<|Op)kHewk>@*W+oSa zETqQZy~Cc^EVrM_R|{ymBTGD%;4vnR;kPM)0LFE50Pe72rjHD<H(Op8_?Q=68vNXJ zMhr1~k^biKewjY#tH}GD2+Ipv9c`2ZANM}Kx@-Ln$r}F9UzA;Sc$>pTMs6E>4W3*~ zUFo3uxrtnrjLu2D85Dm0G3_4Fd)>PMsSM<^$p1&&y2K$MTZlFJNmK69FV^dPoKrft zXL-&YO5Cil^%8%IwkM2yGc_J4Gg-Ea`H~BGa}GF7PdWy8cpD()j-MF9bT&3VS+IGG z-(r_EtQc2!`&OM!;!P!bvC0neK+@OSNXGp~%ZJf_SPZ%Zn_horuqa5(j<}Z;JB^d) zbIuxZ+Lh2({LTtoloDO@Ze}&?Zj=}NjA|zA2XFignk&Gv37zJXbut=x>)8Q<elOA_ zX29FN?dn7=fPrrIN>|-SZbr$Hv<IEZyt)!%dhY^ma5a0Wk&)qHpHKm+n@X%)=`j#P zL;f(DQ|KhsuQVl(g|?cJfc>l3T^QCUXs)hoqavM~n`C8?>Js9xx|`z5lO?g?O*)L_ zcaKS9P9;v35!De93=@e-FKpI;<W&f!fG3|4_lJxH5iyb9yphL<qSl<i&zgevI9LR% zxJJo>NA6nz@C+JL!7L&z8i4hD%32Q61uv^FL*&ZC)7iOZ*j*q&`0J1AvTi@?)X97E zORnp#r$MdJbCiME1vK&uf^k~@kB4TrSy1G1lG44a!_n46Q^SK>^*9MKyuk|Nq!c6H zrzh@v^Prruq&Qf=@!KaGGrCQBjODo+F>H3HOy?T-GzUUoK7Hqzi*G$WpI+p~zXwFG zz`v9xWJNvykm|Nl6||}+)S2@Tt+>-|feUL<zr@P%%v|JM&h5_H#-`!RG^2@k!M@4y z!AI*>3I}iq)ynoZ3I^d7m7P;U{5qRS3UQ*e8?7*$3wK#QsNQq2;4v_19B9lV9Ef+i z8}R>&q5a>|o1IA?&+Tn*Z+q-;CU(z1g^LgSp^?8&^*8N|-Bc%@QwQwP8a^h=uQnRJ z13si`IFX=?BZ>+IrndmSDJ4YuEydz4Qs8=Tu8ilI{$7=1F;QXhppUN@F@$bveetZ0 zHJf;5Or?xmikD4FZM$Dit0`TN*pwxGCT-q6WE!qff3#_=8SZ@lH$G9g0o{})8q67s zi>r*ERhxBPinbxI(=hVv(CK9R{d=oZqS-EqKQ`aot0M10mRpD1ES!9rI^R#ZXUO<? z^VFIgU%58&W`k(QZ1QvGaZSjug{yNxm2CkM8|>j4(RMU!M2B99&cd)qJu=1#N}pH{ zcgbDGa3}1^SYtr~t6{w+D%SKAeiO292EG+z9F$-ML%wPzn@m{m_S2CCI~n>%vC7?{ z5D>DbC&zjKvF)nNy`X1+*jUy)@c{(m4%Jg)AU?*5E54r=&f`$}c$yLOkNz)<r^jCH zH<-yClN&(0DVaASr`6rkl0Q8iqGK`Ha=b1)86-I4WkUH7H!kGQQi8a{W4~LSzs)8h z*%dhi^{hblzErMo087P`HnAGTeV#v1pD2llcvrvphfyFB7};|H8M<an7$CNUdb&LD zs&^li($3Ee(m*?Ni1f#PFbD25=+a=yA{^r=e0mysSr?&VwVo1qZqP_=h|3D8TTGz} zjR3xwNMfQ8p4?8RbZ&{RzzhbMNV0SQh;n>A{F9TO34Bc06=#|h>n-9Nbg*)4QqSpT z)AaOCefr&fb?G36Hm~bGp32djv>R`z`mL^qDJtx$^j=AHsg^nUaucyEu)*qNxjX-l z5uN{+Laf+TWSizuUX6m--w}M1x_&s<Sk`|vjsMb%+8J%Nj<+`oggpVK=w7kSf#fit zaX!>{&2qwrZf}~aCGpB{u=9$xsJ=v+qE}xJXfQvy*W%RcWEYPOyJ{ct-o}Oe$ucl5 zdN5?)Rg8=E2?)KbI%}0FFolipb$hM(w8A(P0i#zQ%ej3kt+S+NES*h739oYG0GT<x z%FhsaO{Cm4Y~U(9k1AM>yUOvYn@*t*Qw1S_yh!d$=6>&_g4Jguj&8t(Nq=QyB}!ZV zZ~dA1Z5?7?=K|@YG-_>_^>FaY;r<lMa|0mF`Ma2@Dfc?J+jYQDP*f`=cJ@bP(Dz~R zSq;LR>j%*R!b5pJlS<}|t(3Cx-k{%xk>rfASIX<x!HNp<fn6u*>Tc^6on^fZGjzCI zeB9Z&ZZBcMuoTm42)j(B6Ig&JZdh;m2`4vHc7;2}(Q{ydI3P>UMy>;$kOwiVtRQ|e z%ZDO=y4-wM{h^Ub;3X=HLgs>5LqJI;h10T9O?s2jP^(n!)RTzWSg~BcZX-2=C{V=s z-p!t4;5{=Z(2;nwAlUL7|I-fU6}OF%%hBt-7sV_SkNAM=PMalm1kS4mHvV|Dmt>z} zc(3+Y6OmB!z)PS0S_!x+>J>64)r^;-E2+rNq$<WCE#$B|8H!>(egJYotoGyi*#^T4 zF%nNIkoRsi2*)sZBynD5t`5YX-N0{gIUX|XJAb-@mFXKE{?zl<v<G;AQZZPX<?J1P zU6n9O0ybJP6$Hh_d1jbKKxn`6u>0x1oTy(+H-}D$Y*Rm)gfzU7{Q1GJet%Zv>Yj3G z%1+<jeY-`9;v*NtN&R*T59+^4+!c?XzN3C1)DpxlXmr_rUA8tA&7$dvf6cF_Y@ax? z8^c|%loB{=oA=eELD1I{G*VriGm0XpipY*&*BfN*`LD)pqZa<dzF}_ALz!R{@;T>C z4bdt}mbHA_<8H>?_1>pctF|Nlv+v<5Dr-BUWHCBp_u49lZB((c@>Li9Oo;(+aB0|< zCb?0Xw;PYbB5?4aE6jsP{kZ3}iIy1hN;aqQ#HWx{yx0|-{&?4I4;>;I@}))yh<(z8 zo}351I$Iq;AIvF)){;o3N|<=QX=V@NNk;)TJAD8UDu_>%ID-%~+l|&2i}nf=O7(fK z&M>K<{TWR_CqRbHm*{R}<*GqMRy&wsp7laPf`lLa&usPvf@Bbff>40sY36dU|1EVd zBvs2YaeEeTp^mCfdWyLYZ0z8qFMnHYShT@n{oIg?&%HhHxvI%SK!9omnOKsVnXc9F zr`6OLKzfl|q}?(%xlM5&?o^OoXf{8SP7iI0Q@K;;fDDa)GBBGonY&z8yz&P&Gu~LZ zo?!kt(U~6?+URbQWanr*kQj#e`DvHuAAS|xI`jl5V-=rjhLsm@lx+n!MhuS!c{^a# zOKab$-J~h|cwLvMtQmvNUwy|Z01X=#W0Qzy)$snUF%`>Yr;hd4LNRF9>wnd|d4Ezn z&=~;(Ijnh>d~=l>pSKngY%j#+`U^EQ@aLMdJxMjsoBJRqrW!~Uf5OD0dd}0PnCR>w zm2m^zXQUXJ0Qj@*Q>2cr|HP(v|9>CP?@ya&e|0?fi5Ty2xGnmm^V9Xim&#zG*T-#0 zGEvW-*)bz$5hbd<!2ZVzKK2P;Ay8vPW#w+;^>2G}1KgCat$&$LS-N!UhUV7Oy1r5$ zg6_NQSUfmhZdYq}-4@8Kww36;pON>(t&<OUK(3^^A;ZsRrphhXvz|!`g3oRxv-zD= z;jt{#s?bu%d9inMou?GGqVLk-yd<cxp1E~V-26pcTKb}ZKl_b3i3S@p*3&1SP3x6N zgBrt6y6$}NSu|(*itKkMV`Q*?1QENsW}3M$@`^9i+F<@o(WPfRPZzFZglSn7(9b{? zPU?5!*~wVe5*lTAA8H-G8P`$Sqd&h7MNL%Ike)Xb5bitZht8;za42Nwd*f0(o5K2r zH4gqgNywd+C!r8*7AslZ-9blpoleGLbqPx^>3YO)D^R}<OfQn^Mj8lh;*GM>2OCxI zhs>)%%{SZZO#%-ndCC*>%nffY7v-EJXk?95ZjvgoMxj<j3Nqs=XYTC54OV{8Vz2sV zf=U9fu&C5xGt8)Hp5NQx2W>mD@s2WJwb>F=0Kxs|`Nl~4biX^=NCyEx6mtH!4LQXo z^sJi1)x500XRc|0p|kJUm%OcziHZkA{F{$syw^VYxd3MFL-DEzzLYBuUVqEM+Akhd zyWKS2oQh3dTv`}I5{*0^T}OU!GG*;kdD=_%MA77LPTA*4Sa4`g#Zj2(WmaZ2y*c2H zNtIkl9m3%Wqe$gxt@25l1hynXWAYgO_w$6BD+}qNfa!Np(8S~Lh{cW0?_ojyTmL=C z*qZh6+@AJac{ER-{3}dS@>L}Cyp~F}<zL&c>=nt`9lzh|>OPr==|J}9xV7~MOh)oc zAasq5uS}IxQR?Wis9r{R-^-P+Nj~iJ*~=bq^X0g-Ns9VwDrR=kfPKcQy&+O90xjiz z=CRy;#6t(({dr;c)~%o3{5Y`n`~kD)@1MWl{U{cC?)HIQx2<;~_C1O`CHg)*@@P)@ z(aVY#PVCvaZ|5VCozd?uFVn<-G|f5B<@=7#d2t9A7x`IL*OfU7i=~44dX9Mj!NsxE z<0Q1nTI#hnC%#gh&-ZWj2ofws?Ib%EF1vEtu(N4c;PfMxMIEOxduKsnM(0AQh|r~w zHDFr4kl|D9pawGLE?Uk~q3Ipsea2Dp5O2b~WY(T6g#@de1MB=3PeVyj;EnMcSM0}@ z)L`UVk7VTP)9Q)JscttKF^|k%)O=O`3aGO+muT{Z#*FEkSI!ZpQ=De2a0nlCS%Arf z3=~u_v2U~WhP7;T`ynf$*1U=P2+Sqzz!_Fbs{58&%1=OOdr=5$hSOEl<7RFB`?KZk zT_g4*CJ$2|G;FNmP?k8rQvA4iD9{o-byQ^`@JO3HFSy=^%NZ3oobentT%AA?)>qwU z+3Gz|7YhIkJ#Thz=EVTpgLnz)MtnHE=#9fyo1H$$<P7^Y^$?4<sDm?*%^-SXGu-|- zbf?pUPhP3c7h~*)-|!Y@@-vwr!>ah&7nK}+`Ubr+=%uBBAHAYG!|eDg$}aFM`v|0T zmYHSdxM6AD?)amMbN=T0r*{k1G(4kyh+=ZPrx6Pa&pX;e;+EkE%qutAh&TQ$3$Mj- z?eMQuI6A+}Z+!ehLDn<Q`7PVe{oBVIBaz3}=c<Lf>wvIw_MO=5U%%wWbMcSVynSUm zT7n8i!$bne7T*1z-fTN?^xf;o`=%Fu>nf5fq7tQWw=M)0qT}yk^Ts7aJ2EgE8a0@| zRxJ|h0`kt{tfy)dzPa>=F5eqxRk>f+8z{;yfUn%vLAuelxvdbxo8C@x_?Md&bAa(C z8z5ziB<7sPs(~e+ATf>%YJPP8I~R`r2PG|w?ABcG?7_^oR8ks`aTyhUqtpPCkaLNQ z2oxU3DzZncybTYe@3k@=E^}|&OyfveI4kf*E-Tf~w_<wVnaSmLFux~Q@n=r5=2Eqa z?sR26*?v1E3<or$a+6O2uF-0{!FR$bWoEsm(k|cBXOgV+qeF%~QhSK*o~@g(v=vWY zxYpJYKEKyh52?xR5=8aVu#8?vZNLB63fYD}!OS8RF##emYIB||K6x=2Yr{hoh5A^E z5ix__`;6VdYoEmKyxhd`d-oeG#0P7-1ub`|HGGbk-MS}WCjuB1mB!4ttAL{-d$XN2 z9rU9t!b!y$e(iBM*5^=UkUV!nA*%1Qii^~Pb(Il-cHas(u;KO+H19Y=nuUzfxMkKn zgy?YCy(IDo?5>HYpN6B3Jo;sF>cujzo4R&G#=%0Cob@MK?uMc2R_r45ZQ;r(Y;Us^ zE1fn4eUJHeaoc^W&+hhWF}ZDu0X@ejWDacE`tKj_l}^lVzkLl{>hxFT&R;$M5ORBv z_Mh2CCY|G{+g|%aMGnL#h(CTAsdlu}q1*Prx>Ta>MTr3O<lEync_c*cZAB<|0%=3K zaaYr@;24m}Ar)Hkp>xt7yjME3<F1~R>bi_Lx(-qJBH%sD%iQC#-zZ}Qxx{PgrG<{A zWIHSeHRZB0r~v#cSeuOYuntEoCt;LJ7rotZD6*4RYclw&1U;8wPe7>he;Blx(Gp=t zOY9Q@OOgcu8d&>w*v|@OFI=|LpMR5Wn-*#?bA*DTRA)KI+h0DHtsi_m3neIjQLnJo zl?h%X&vbsV*I-<G<5KxPw1w^`{JDqQZlJq(Img;TTQwq3UPHkxVyT7ovl+i*xmWQg zMIHddH^q?bT9cU|@>6UtNXxLwJ}|Dq1eUT<bISW})27)&sRzpAvL2fC+VAH&4+yT; z2OaZiB1g@lIWzXrccQb3?xN+4>f!s;<yZc=JGhbxIFnuA{X*9B7%+|*G8_FVklvw& z$px_2_N>nR2Fy!P<C;_ad)zvQz*Q~%zfp6{*~rCpIJC_5bzl`sEaJyDUm-8c|2I5M z^KcN+rl<WIqB&&7sP{r?M_uYHUPVH??Zf!@(m(_DPf??gxV4NznM|AL*R*_Q7I*Ab zVAI|XN<rbu=QFki2D)jD$Xhh@px<GU<zv`PQuvz~MA?%rP>+A(udB>Isue};?*p~q zg-YmC+8w#&ACHf{sy9430Zb%F`F7*E8?_RE`_0)N`LoAB`6wFM{3<RsMIxc`1-4+W zT|vgxwADGftH?RMkTu_O(?eAgmZyB?ptOps`hw%J3_w!nuw7e_ChKN$T>bV6Q#0dr zL|0kNq1MSKGZKZ!g1jj#23c<t9=zz><oF<Ci12CW;a&hPytG*o;KA@O;z(0h!p}}m zOYBK+zjT$iZh_s}l|*Bnax)U>zUe9;`mxIvN;t(oapHkv{$x>KYM0-)KRv{YigKGe zp@z>F30^QX`!K=XjI#Qi4DZr3JGbAqdVlwMTE`|<SWr8UkINPqP$E8eHsnozOMA=v z0iV!?o{gUABAo7U)B5q^GvE`t4_ack(J<zNS;J0>p663dT@HrWfOx%IhSXpP#s;ue z>!Jap(gs%w4qgzWD-Gk5ZuNhy6Yp$;%SW6Y4V#`W_zA<KfPa|0b=Q&J^}3fX7%_c; zyj<0;QO^7yM%;-#JxrrU$WitHddDINxfFwMrG5Po<$5N@4ngm{JLb}q>yVmt3AN@U za9pN5*x^&UabLZvGrWD}`f}Uw>|Q|U&`DFL@iUG2i{@FJ08>|-lHTN+5%0$?ymVtV zccEXT^jg5d{4sv{nLEr+2ge-YQr#Ch`_d{h9SSm~b}B@_Mr9{PUQDjH;W%`HuLbLe z?jLbGEK>euhj`uEp^Mx8eVGDE*?Ywk@Bgk;j*v6Iu2ikt?@LWxYiYU}yT?X;=XPMb z_G44mp8P|S@ea8pTf2)k<O{#M*&IJfN0zjwCBD9GSkPF=J1&)0Tawb7i?A~FRA0RJ zheKPginr$Wm^b-t9FucU3MrsWEz4oQ!g)hm+0@DKmgmA81ZOnWw|4v)vV)c3TVG(- z?`|~9A#jYi00K*IT`Clv?;RvIJ*$TEkN2JRBdp({0I#T+3E-~|ZjPUW^N#A{4nse? z@HP2SG~#@}FYpOha_5RT+`6%CZK(Tu_$$$EFy{*HVV$F;+^-;q;fn;Aq6>12j^|Dk zegd4DvV6YqQcz)5>k}mI(P>D8y-TnZ0}%IV<*W`+BaOqq67N2hs{gW}3XdBeMe-P| zeo1H!Z!tT$>LAee=(xb>d1rh_YJKUfY;V+MX-q)nFmm#xZvI|SV?5xEmRz8IeH#LP z(({l>JrPA9c=~M(|H=!=;V?Nb{|rn)bM$`Ps29wsSIjV6f`HeNPU9ln&9-<7tnrlW ztW)yPb19j7N~i!*X)5jf;5=?Tl)~|@Y~eMq(UsFgJ%Ekd?~hz*-qSefMCIC1fN|Li z=sVvgD4DSn(WWEG>mN>Z`@?Zi`A`MK-Qk*06F;0tP~7uE6+66H<1Xv(yPUoomPt)9 z!i!d7<&70MrHOdSLgTIoWwC|E5|gF}fY(2`WhQY~=##=r*cl1fjmO1)g?MM0QJ6`9 z8fRR^zUQ?mk9o4h?A}sy`Py3tOSi^d*G*=G!_vnUb@N_?9Ni-F?;k%TBO_zGetpM2 zR5%B$U#xPQlM}go;_1yRA0^`zro|?o9NKOpCfc62Y$@sgJp*}pd(7K%ACq$vI(3SY zZif$N@HR#j-Ab#M{olX%(DiALw?e^L<sPrMmS0XT8rbfzIsMJO{6UJ6eABCg<S{tY zvSuUzVmsqo*>_1cA+_i*+R^*esFBdWezph`$QzlTk|%rCP)DT3fqP2<)@Cp!9_umj zs)RmihB0M^k6Pd<Ajit*abQ7M{l@Y}Va28^l@18JZ{wD3k<JG4arjUy4;2`Q3&FYu zX(JvKbr>N_pAzzfoPDxYlZW9|!(l}}Ua2`fbb%g@XE`W{MFuTMJ0EmD&wa@p#wyJ{ zE0mm{YcEH+mVLfePkSNECqtQ<I-Tb-_d@Zks!@`62Xf_+(aM~`ES83hn&~EJr@3p` zV}kpj^f(luv4Q}Rt3wk`FpNHhWQIz+1c~DpJV5ioFO8cjYcmRe_Q>)#*i_OVJUmsF z^HYBGJm|1N=}NW-QL3tGV;U4XHQyUEy|!8{JR@7t2B)W~5V|^TpM#hBS7!M(*!av{ zWx;|eIn3L-3g5YtQxLb4DO&uNg&S3Odf<N4&JDw%m8a6KZ-BUhCyPx&Bn%MDc)ipf zcryqnHDu|?7AgA$IjUVK?fYtYCxS?r*4O7h$2(uy6)~+0TOAIePemfvOlQyUt(H_T zbMzlhy3e?{Q<-_u6LsYd2oGIaJ^4^qb&OK?U?fX$FnAPfAt!fBz0fJ{ZN*X6o5#Jx zi*ThH%yrX{#m76Q0{45_pG$dtu|IwtaWG+1T{NW^wf8@~kSDh9OpY|X`T2=l-vs30 zWw&1!+jgZJPa!Fk>la%0Zhxk}Rdc!`;%I;GGj*{p_zh83a^#4K$+rs$?SEZSQ#oMX z0wttOpM>^U*alxQzguo`%-_B>uB8j{RQWa%6qV%*TB(uBu;J%E0U5QXD`n?G5py;{ z0KqD)vH&<jOgCptv$L+DqrqO?+Jg%U@&TS2Kg%N{n7C4AKZgxRri~F+T+geLL#Ro2 zMvs)XHV%Aw>9sr@dQ2E{;lH&2Z#aHc;+B~|?>RPxo%C06FU>wu*dvqv!_m9M^z@ys zH_3L61w4r?U4=2s5Hlu-YztoOXH<f+E;TSV9?FrhDQSKL*1mc{qa>`#;78=PyUkf* zi|vn=xo9b&Jb%zCV>_*Ka?Ffuzke-OJ;3Rhe;!sYZx<4w<Pya8VlmaEcMS3xsC+H_ zVu?Ypy)$EKHAZoADp{Ij-^A^=lIjCsc}fZEz5$R%5JdEwWkU$2O--j=#n)H;VCo3= z7x`d&0e<#F7JaONxx3X6xe>N$w7T0S8=5O+cl~W(R>?l^BEs;Csttl~vs7W7brr|a zXC#dK-JXbD@`f%n@}DIL^Q8{<<UzFRp5#L@a-)#Vh^Z<1S9RQ#RN;h`{uEh0&b3L{ zr$$=S#HtX{#Dyxc(I6N59X|Y9SHl4NL2%m-#KzBNF%FCw1Fn6lN*J%oa_%ZQ>uu1d z=Qr&(<c1P<CkK(lc@1T<5kEhjp<mopd-91~@ln~K(<efF;g3qD;0p@-Lp1c%Sl<gO z9HGucuOr}yA4H?FTyFU!Ebwwk-dGVgyNEe_B>AN&Qw8K1z#J`rOQW;&GxE^ufn2L| zDaj9?A2F)`srrvb@t>{ZmfhFH51kPy=ZhR9{C>Tv^zOc2)#(%aMDLDEh%BGDvgJyh z^wt#d>hHH=;xeB~@7|}p6?#sKZ}7nBll7POSG)6QjE?Ka&a11Uhd5!y@y>RNZ~egq zUt@rM1E*4c{^>(x)czK8ds+DEoT~d%WlhaG{9N7BO)M{SraMFWO8=(*rgA`ko;<oD zyaYn^hrw382Xm?hs83v1zFw7Ge482zUNKW6+tM%t^W^}HmXa@vh;vFya^IMi7hF(V zWLG1HFVyb|uy~O4#_H_&_RUCTSW7Lr6yAFfeuS(^rwimVz^Q}^PRL>-pw-b&KkI)> z>{A@5k?~n_HhnJ~t(5g)_Xc3Ie0^P2_Vk@Av^D^=&SO%`o(rn(Di{S3Wm!C@QlQPq zS6SI)?^q9s^8HzX)+J8MGs}|ua~n`g;L)W`!+PH4%<Q_7K3E$_@_b(Um5b)xA+6Hn z%mDp@Xz9{cwQMsSlH6c$M7W&cZD3n(11cr(0puR7>QgOcWC&9%`S2O?PE5ad$=?{{ zJW@qnYk@P&+9I9-9g?T*P1)w-N@iq>qGB1T`qfqEAs|hjxTvsQ$T7VsYEMB%Fdx&u zgBVmzd-xH-e3KvspuI6H*!&Ci(l@|{p6{kL9jX}z?0>7lWwFmj65w5*%%csegzNn8 z>k*$k$n<4Nv4Do58Gb|WHw1L9^oKWZk9m6@&6qPkYF`<dq2GXqM_ntis_LH{_OANU zJ(yQ0HgD8Ah%H1q{j4WdEbM?550r(8b$|G*=Y}hC>lBbU2jJ9L|GkK3y<k8Sqd`HT z2?de&>&}~4Wp#zg{|7M5mhE3f_sMJ>vIA@Jmu=U-^@;rYMbCiTfuk#3wIZj!biS=> zef`_+$dAkBZ+-(zgld{1(QmFP+MYZQ$4g0;%>MZe?{Da!aV_EG0J^5!&Uo6}WJ}7S zz*Fz?<&D58N84UHtD5E?KM{EUvz>j(w_7qot_Ku!ln@1Ge|P^xUm)-nD~WI0dSlI8 zNeNqmkBmG=RReROaNZt$y!>=eE`WCv1`oNn#pMlEcybC4mAIqkdV~ss&<q{ddCRfa zS`D+iDgkD<E^y{}J3u&ii;161o}^{NwGa$%w?G=33Kr}zs$_xdr#k>$bN0;WI;5WU z9dWl~0t>Giidg9e1gHXk&KityUD4Qgu)E?!EHDZy)xfO13Lh(>4HNSt9uODrwb`A( z@V^WDc43RjMTg~9fN(*jlzv910Z17v2%K7BwRfr0-vwHQs;t>tUXO}vSj&(5`1OLe z{TKZ5S9KfmED|&Qqog_<glCRAg?!m`w>6k0d)M{8NJt#sSZNwvf#|RQ7%gv$by)-S zok;*5uOXusa2wYdXE@#vH2w$mMO<@#mL4^X8HU>&6<Jx#2<)I!gWOLkeZdcCgHH&> zOU+({c5<<Ha=*;UTPA(kKh6j2KoZ!mR!~>4ax12~)u}k0Y%K!P4x1P@hg1-|(_ZlA z@S_D=`n10dEs}+$Y0`o&C$f7W261BsiY*Y|IeY=}iXc3;DLV(BdUyT$K%enX#n#JD z=D!(zmHc8Fk`f4_oj+`MVZb&gZJI3BH8(QJD#8gf_lVae=SXDzV$l9){X7@-@y&sF z&+a+C7#d)tKT16O=X3u4$?pd~pxQ`-ZVOtwe@lG?{OIEc)dLdoHDipUx30EX<Q%lS zl1WP5bJ6yJjcuHVp{9e^!{=0DLFAt+z3;Buh;~bc+W3c=A1ovls}%jN>(pV5mYlrh zaR%K|D11FAUU4gNjs_Nn>o0t0o;5=`zdc<kH5X*)c(&rv@Hp?OdV6_f*E{$3k5p-` zB{d^gz-Mo93h|B8ItIN%*enx`ui0S}(VLm)4Sd9OBZgdvOZyfMGShFID=fMTgy048 zWWY#6pY?~(K9a5LCG^G;CrBtOga{)PHQ*Hy8<gL4nC*|VuY|AUdG@Z-vi@pO7+8Dm zhq06oW5QvvemE4X&&(@EvwC+(^CJLiT279e?F0=CXJ7d<oC~-l`?TAT>Ap>?cWS7+ z<ig%K$siZ&XtS@~<KFJos(o{ES82SV7fE^37jU=vH{D!dx=FQj4WE{KR5UGY&SAgd zA&qjDUXMK3lmvqibcpQRK{)>B=e?DKpQ6aCWYnu8G(ZzJH#G_rzweWj_%4H+qzo^n z4RX#-aSTW&cx<gpy=V-;ks+<SML)jfU0Rgm&N-x#!`oyfGAlC0$M}k@-aU0Xo?`P4 zt7_H3MDd&Jl<yZ$-r`upT{~;5U#ba~>kjKPgSGt(b~&wPP;12Y{D?w7nOSU6)Jsf6 zGk{byS5U%h{GP9_iL-UzgE@p<y>2M0r_VDDTOVxwkM3z@(S3Wi4k6d~^rMsJpZyQ5 z>PM9~MMncy-30-<uKzV}ZG#+DJ_;4IpO^JIX13T}_sr8PFUQ%za`*JLGwi!_{z20w zg1W{4mCD59iDk)E{V{bHPIS6#lv|fg&qUv|o8^zb)1qsf0lBnTPu<rXu!K#o1by=~ z{$Xd794;HsuE2T<ig@c}5mSPj>1s-`$daIP=W$Rdc(h>x<Ys{222p~@mIf~0!(SA4 z^NU$u5MM5^Hr7{SbfoM*8vU4+yh}ynaSgI<<@2Qt^($1y>%hDG8sEdeGZ%gwAvTPy zmcD!47=yLSvY=s5#^x>lnpDwL8pg>YRjvli7o-keLt_or07<n8lmG2JtHyS4PT+r8 z0N)%f0G<~TCj=0LG@b9aytyclOO~CGD0~x&<5|d@zOxUwUiYwo(?>o{G`xu7k}<Ob zZq&a!-6rEBuJ)y$4S<1u@ub7IX^SQ*e=#)4f6DdaQXFhFpOqf0`|hPGG@mAPFInUR z*YdJ~Tz*mr^>|H#>tI=Id!`MJs^x4ECBGKVZD#1^YUKf1&5SXCh7WRU3lEqdF{3|$ zu02|#K^q<oq}Tf60w-Baw{vPp{`((c_Rju0e1MIgqfEF|$#nNfl-2A!XdUA{o9MSN z8As%CW~du+cJgE!id9xpz(C5i;z7iV?M1}u>u#>LH#?6hE3~m2jV4zggQw)wI7XUB z=0o1TXse&AVJR>g;m=*(<a6$iLf@wclz!1bIEXq>#Bw7w%7ANF&ZG7p;NvexqDR|z zKK-u&Xe)cr2Av@N>)HOqx83K?$=y?bu@2PIeLF<5BQ?aL+Y{OYj)R{)Ih{}#z&$WP zdWOQN|F%-)y2FHCkt=^waXBweO*cP&cMIcUa<)qA>uYL{UVr%{^z0k#AxgI0x1~{g zY9DDC2qJI1kcWL6^r8}x<z}|Gpail=2II;Ecd$udnxqEZA6G>jB<407uaEWcj=l9+ znrI0E4h2~Zwbnevc4laMlT+lm`w$T{vECVPeC6P(^ZL4y{Q70k+i(oi)!e(3m;Y{K z7|{6BDc(5C*5B(R)z59Fx3Y#+ih=`%l9=^HJHR!sW=1wn*RPYv^ramuGU$%vH-8&K z0Sz@oQq5|CmDJ$5zAt>C?12ic_52PQ!4HZ9t8}%E1LFW1ek?%#rT~>LyE-Ztbj1Hi z$x<A8wf*r5(jSE2{TK!4LancIOrK%YlQ`k>*2)3L&Z)(}+&hG2>P8T%9t>Ph8wj)7 zl*^35A<V43V*C^Q@tRP!%#5WC6-p+svU-TCb^hacBB=l~oRYm319})(>I?v~tolKG zX#?@qmdwCK3wggI_$S77jQqumdsOiZBUUsnld&i&+gvNHzBXr0`*L(}c>UG23vovG zm9Dj1UyK<Gmn}*;VxV@BvmK7!H+uShKlAnWV&*XtSlBd$^qWkqdQWd2Q^O!{99+U& zUG;uL*t<nskN5q{(_bHxe}RU2EpL3>ahn-q|Np&D&m6s>er25LAal+{a&!K^n|AKX zxtX_yq7I$c>TGwHJkjV^f|$H}6_Nh9HL3Lius0KRiqJLm;^s2%TV`eD!qVPoUyY|_ ze--mMCTH+>40nyHVn3_mn4aK@UEOtvXECgwNj>lIp+m;U<mX#c8nT;7d63Pd<$ZgK z4CDh&`IL7@8LXrf_M8F<n%d&02LTK3b6^*lB@pljF1yl|I7La)&8Zrzlq$iGA3P&i zoewQ!v<9pVd14sPn^K+GlH6|Dru<_Dh{S4MS1RmFqRshLVEQp)0KJ*sG0MD53K~RM zy9JxdlYG2<Qn&mTZ!gFYimK;@z?Q|Da(SM@<B-oDqA5k%en23_7l|ccNF9+y_xKeQ z%iIbDbb@5SO*nR!koVFR?B&i~IA*{7>4VbzF2zONruA;Syt_Tvy?*r>i>Q0aa_&_2 zM=E_}r3d?mbZB(rSj=G1@{AKH!?+go63(d4f>?VNj5Rc3F;4Zb1V}RdZ=TNme%WZ$ zY-JKO-#^EGHsNqYoCbeol=V<7;*EWSE;L*nvE~h)6X?0Ad#+*@1yTs%2lNiwna!~r z*ewLVc444D4Ct@z)(t~08Lv+zWqlnC?c%9-19}p#g%^<7(To!4UTLk|$705{gKN~Y zE4O68;hi^HLYah5lMq8aiGa%zGuo4>*D`5t@W7<g;4^mbQEA2YC1<5-bo~}V{aV+O z9eUiHM28@|HRbnuY6|if6G+%WbenSUo$v69J1eY42i8l(uBgbal^<XbF;g#Tts;%1 zKl|qK?8j|lS1kWvKK>uNx&Nm({P;m(lb{joG7>}?{g0$hy7=o8H!p-XKP=Sh*9fQ- z0hqyuZ_SUY99=0ciB~>lYw^0`OXKk`dCNG~u~S+?W0h<4v-ewWg@d$H<B#5H8@QD; zh&|}<?%($0%*}&Ej|!-hdgUM94|#bF6n&@zMu@(VI$-6`xVP^2%sBnU+Z0%{cFdGJ zUD-}kU$s~+c)|S4y$Too(G#cB&?!oRfk&06^9HCzjah{k?g6o_2rnSt%yd@=&zO6R zJNK4H&H=>+7db#Y9luT1eGfl{IjOFZ5AnleQZ{%P)@Pt)do7hpnkobJZmL{>wM=id z2mTjh-vQ0`|L@(Z4qb}UrS>Q>imC=d(Nd$-h#hKFiBV!lj8^Tvw`yy}9zo1%?LA6} zptVA+iV?(kzV3bQd7gWp|9$T7oSfr)bC7RxK6%gA(Ek8C`lDCB*+vp+UH=`#ntR%R zH~>h=&)_83A@CyUnXD3g-9eNFO6kC`M(L4hDr!z+Y1-7pR5Bb1340#~g-xLzemmW1 zk8{(GXh-#aG3`f%Lt(C5_LW1_cyE$qE0r^ER95hY`W>CETkQI^m3F>(tn6naUr#YX z)B$VN^7A;GKi645pCEsXWh{E%-*7@gD+GE+zb~|j-1pi0eF|N7&)WAIPFJu-S<OWD z-bXr76lfvwupP7>7O_1)62#(_AIB?uw5(y%pAPo9etb;cF0K^*%<4l9SSL$N+VZ28 z^0%aj@850qQ#JZ()4nVFK+HVaWxulfl(9fer3ddut2!AgmRI0cv^Ct_o<lnkO{t17 zy-m!luVV$D*vkYtYn6r_HMDH>rGmry!Ipn7b&ArK1ojFCcNQ~`E>ZvMruIK2VzWyO zx8I#L@MTW~&#9IxJehnb{;!6R{yyvv?eSGFt#}VjRGWXbvz-RuXs&2ZHUw|=fsIP; z<!ePb`2_x8fZlD>1=W42`*{b=x`GbR8RgT)iJe2Kmv{@TN-t|3FiW7}>gHF0z4S}* zJ^&Gz-~hA*>sH<y#K^C-KC>fLYXOY+x;kJx#y!NPFHyOtl!HbsVmin%oA-HBb;rO? zLnHqb?0A9@XWK2}^orONk>qR2K`nEz^d*D8(>MaWdzbuYvucu8=4D*X^hXv{S8b7P zPo^RZ`jCG#Om)FGZR5b7s(kAgaNa&!M1pz18}qNNg!E4nIWh8(X$^Cfo8fK_e_dGJ z?7WcUGP>sb$&}#3+m|I4$8xw<dfG9E$tVB&!g?RlzBuDSj625O=i+8Z>Y72O-Hr)D zx+zH@Z&^|&=XPvTL%<l2DPKN&=efM4vU`vCm&wR3rp*SRpM36$;~hCIa>cSz<b(~J z6!WG)N{6h)3ck4)S-s1)tlytPH}s(zjr}M4Y6mO9qajSapM%!ZheF6-SdSh|kp~j$ z7Pelp2>kI}_oB#!l|m0kodQ~q(<`$mT?I@<9b3*%voz{bQQ=Ab=dpTAD~R=(Y;;Qj zczJ6)Ou|LBW-DYTjslh^zEgH<a&A8zq_ps|U5QE#?$!O{Df^kj`Oo$@%0J1q3Mt<{ zF+|@I(0>^3Y2om%|6-6xUFFwnn!0$g>G%168xB5vN9}yxGV)=~!Rxn=+1X>;J`MIj z)vM`Wl~4n6+*fW1$BoL<+dAy1-fK?yvX3f|Pv`V~;al#rmm05VJGtViq6U-RA`axi zOR#~*e+NyMuNnEiZ!d+EGia&EY>kOPC$tadT~VhkAOL2)d-Mx{1G)TQG$6}vD;s*w z(Yl6>X>~o)@s!lNYUz;q>we+ej6YIJ`6mG0;JLNW4qK`OWsD{Nc@&HK#*C+)*M7Uy zaOz59NVNg{V}CkgH=UOix&P&&nl1}wO#v}5<*S+~kPRRm{9npYIJ}lJrO?-<`>&Mn zG`3XN4-<Inzd}Jrcn@vQAE=_IQ6s^;i$eez9UzGyg{L0P`7U=gQ!4g4WOhYPNy2Y` z=D(Ukl*fvdW|cJDvv3#y-*1kb>c(Uvl$^`AvoDr6#K<_-uRT84>yU*t1PYtld*}7Z zW_c9V3QFt^w~5*G>_B-po-!PS-;r>hz?KY{bTPzKG+q>0GM+WPf<(evS=?T4h=Ymm zFU&&pIiYP5@#jRF(}QHkWvV>-niCO#aCYpGHp%HhvpEq<kXUOJ;ua+?%BcQ5Bx)bs z4X5Rb3hDG_Azk5c{_^E~#D9t@|4r$ih@d*h&o(J+^7~48kLP1{?|&I+o<5cH?YieX zBl;kb`sr!4kD$U*IippqFMr|HUC#PA-(Vc_fXB8Sm^)C1UDY)_8W#N<utL9|OsPy@ zkzk(W&glkm$J&+;bz2aW1XxN*=<qc9tOd#Y%?=EODs?W)S_i<O>**;DbVBf#vk|Cf zL8VjVl^~Z;h1t@A-Gc#D=M>!Q=^s%;=&<#?BuIVmn2{sI@N^;48cg@KaLjgDgG&yy z5~L`1KQGa2pNN&WED|kRh}QmChZH_RofS;+y=a1*G^zSEEwx<G47pn*bJ)EJAF3TS zRru0g@|1zsQL)~U%w!0dBi4a-yr`-oq!`m``iD0@E8QAQ&SUvHG;kP^=}mfJXqBM* z7O1~AiVT*lJ`SYwuMm=o^?UaPX|Fz@J5eCMRDtd87Nyg1_>(rk7f+tlrv#+eX#`9Y zZE>cM4O_g^)KH7issUd@Q)A_D4-F_fEMEXAr0WE`n7d(AfPmIqf!usbVjI<4%49@$ zuVtKM%UG^~Z|yX8#l<QNmu140>&UkQ))Cnk<Kxq(dcyn!R@pctAkPM`|C7X$Pk&p7 zN>o-{Ka)*b$KT?APZ9o4Xn07V`J1GHfMIb?rhsW(?{Y-~T+69^)qsLG4->4YyA+{I zuR!+BM8&_EI*?WT@;HKYwXNMEF+5HDsqxQbzoS_4`}0@sl0OFi`~TdbmOh(;93>(O z2Cct@>b{5wk>{qTi&}p12|VkfEI&%0kWk^=HMxmsGl8%swzfCwU(;(}$a>)|LNfu? zSlBfTzS@=V)>MAEn4O7_54YQOw1i#iZqh2ohq^rWNh{=DDV>;qXq!qT2yuU&49&~D zn$F3J6h*gYXvawCK^J8#K07KaCOu{_Jr+>c-{&)QYGbqk*hsmu$4JIHGN5Ow>nR=& zcUw(&#i1rDb6;Sbr?SX$0r8gv*NPj^WbY!9!&{jGG^WlZa*%JI63XqHds<JN=10O@ z*^q18^ezj2KRV7PD#A3?ZX<&)`-llIJFOt>!ac8ZBO8*0-<pQBXJ$KJPYnK1j$TiU zSmY)1gMJ3_E2)}tsOGUmw&0AGVPU%&$X}ng4GIQWkiM!S#FZq8vj!w#ZjLW^oo(}G zQUB^&`Cq_i{@L<oa_8r17<jgS0W|LXuZ)XBrqo)q3O3C4+qkS*rRBp!@(O}G{xNYP zi|U1M<R}m4_P>YxGgRl)<$MF<Op#e#tJXBAjReym8QMS}7RoBF-MYvBjnvQfnyi`w z+XxTlE(ztXb7{{GHcW)-S4A()o~viReV<y+ESW)>rZY3TwbR+EF4<dP2{KfxBeQ!j zOwZTM8W$r?lQJ2ssLS!mv%GV_`QuBC$gSkov8PuWzRRO|#jf~#<a#qX24(GENU7_; z{MNa#HFxf6hzmQ-gu`08?ZC8?CPxGk`9&vil!N0051n5Waps@JJFWQ)?%NXu_cJ-^ z{#1*+l6(YHzKe@R_1_(b#L@p+OD}Fnt=F@c^^);Ak^~<qX6o2*&&TRZpM^Si8nNDQ zE=jb<O(QS0eAScD&@)AeXE?Rrc?7umjjiW?QHsh1>_GfsVhgA${8vGnG)$f4cDkPO znO$c9jv+{AYv14H$>tQ)|7q}?kU9Bz8p8ogLbuq4otexofp)DRIv^b(<O$E;%Ankq z5OvJ*u+~^^bn|44w#pRX*kW#IR``9fP(iUDEdCDIU;K}K$TyOjnI=tVetV^K-tb^B zgm`7I9PYd@neG&L;Z-jc^@Tt=BT>ZUu)C2HW^~|N(e>x&|Lu6>vwzFt%%Qp>qdgX_ zIO)&!&9wJpG@*Gu53u<>;<mZedDO%5<#ed+#~n={T^#*&5~(X=e}es7!Sp#*Cu?-h z3$>sle>DV7uMrZytV<8f3rj+y4*82^3ZFfso3(t&Vxb|oMJ;b)FtN)k(ysVMBfK}J zU%~UUFH2n1f+QsIjUoS$iZ5(uBG$F$NO^tf1mRn*r^fa3yPKK{;NjRmv>q{@IRJjD zfCQ7zo{?6HQQyaL9~K`LQo6wB1kz&5Q`FUR6aTbfo%0#n5#}K@g;4owVzk~q&d712 z4p^G_@6CJywrU$gfZ|Si91o<#z-s<&Drwh!gi`H^B>KOC7nNU_s(kx#^>XdBC!6e? z=cL$3y{Y~trZO|EuB5jsV;pOJ-UxZY1No8|()wM*p>xZCz?hbP7KYi=>E(MR6`z+~ zJ3pOx7pEGT{3t6;O?|*yL+-eLKWT`P_p<B0m!@%tFKb5@vEi2D{L^y9OaIN<>OW=U zPf?ey0;uBVcWtKfi^4x<#T#hmm2YCG-u2)9fn<D$1iHuH&p!!fW9?rXjb#__IpZ+R z9X%0Oa5*$fNa-<=zAJhHtkhq>U-s?tc;XYehDl128N<BeY$qFjTcr$EM$tnk=CYs& zUPGIm{-j!jX~XrF=kb33fd!ayIzM|S6wZxU6dn!q_{dLXP_4y&NB6yRWy!v)p<Me+ z{Uc&<;%NvUyx^bgmrlRnahM#C&iP;k>_M;fazVSNCA9j`$3YhQi{WhXSE>Tu1H=7j zi_0N~)p+M3l{{e6F<eqyW|L#p2(7b)3tX(y;dQoT0G2cZh21uwPA_n2j`8no5Vl=^ z<+!0ChK!ikxR+*?8W$u&_v%WlXkwB=W&r$SM|4pV<(uEni4nyMb_c)wZn*bJS;Mj$ zvp_n3b~CS*4%`6oPeMNB7(ki{{L|UCzhUC`k-7n6gvQyBenKL7kh`YuOAxU?mpe*U zdEm%irY}t*Y7r1%fJB_aruR1IY=Gg)mr+<VU2*MV+S?smsm5u^PfkfjY-~DfZXl6N z?Oc`{gbJXN`Q<lw_v8F-yte@PI;HG9_T*2c9Ls|kWHV_UE6FO4D7X7+?zC!rIpfa3 zG#5BNEhrnQ^n$qudET;z5vbYM1O5?MJ@78zQ^B@%D{4Qi*F&(GdOK-#0k<2rtgsoj zcMfZA-=k~4oh0aUm8*8aWtCnex0r92n7KAkj-UPDBlk&=%09i>zD;zT!Uq8b6gSCc zA1s=gTD9`(MyhVzpAVGqXCcSlYFpw<d&zbe`$FP2imBVW$Jl0kI6<}<Y0XbgnUqCy zknN}c2zx2Wts9Lk0ipd2wHCEJP&C9N)~?m_pHX2P(lpI{%S^wjwfWXxK<{$}%5Xn^ zW+3KLajgxdZ~4RISN@Q|dB_x$x;;7AdIq$d)KRgvCb_8W=T;)pC5+k4JYyVtF7t1i z_A`Uo3SE_FJ`SS1f};yMjk{y8Es~|OBdWnkfrQfjErp3Z^ilh+k-<sw>B6ht6An_@ zr6*gI4uEClN#(5z89~(%^M1w00>H+<0iiq~)!G-vu6s?Ydh9VTR#ceGTl!Oh)urT5 zwbnkE+_1A->QG1&{9Lp)xfl_djew$Uf3U<cx0qyMB|Fv>n9PYKsPXr$JmA#Ok+3O1 zvfo)|J37di=^#BieL==`VIy7aTf6Y*+Ie$yR5lM=&s`vVW3_+(6WF+aww&2FYD<%k zK~Lr7SPqxDr~-~f!AjReK6|e@KdY>^nkG1Di`nL6`5uKWSqZCN8<p7D8rOh4t7^ZC z8u3Nr{8>(7l<GK`^Y_U=_J$BP>x2@l^}OliFuCZS<Epvr8#R8_&l_`u;_pH2Y=a;C zbbbfKhqvCoN{zo6C6$FslzDC_w<BW|D1&pIf!7-TovnYnXhO^2gfY|g9G{MQ=iA-$ z`Nr0ImT=0{)DV+4*;Pv2hsD(U)=+GH9X^r}|Mq9DLCb{fO8J8X3OzemXEeDvxu;Y! zD?c<{wQg1NQm0kJJMq&zw#|JSFB)nJzf~O;6td@XJ%UJpesm$ZN39{*LV2O(m)=%? ze%LXnV;(`{k||)<CT4^g?>RRWI743iqdOkoROC{;Ae>UK^XuwDHg82P`lA;rWbloJ zGL2<6Gl{={QHOU2t&*hN={03%HK-{>?3^~-yYd2)_}X@(3ghN=t>`L<%5B|e5S5^u zUw*lKI(1QpJL<&|y|u%r<GrkTY@(ahz^2R?0ykSwBfNvXC50&7E&{P8t%@Oi#*fmw z>jx(mA-x>L<0LfHZl-3qv_x{F?N@PzRdiZmIV1_O=-$82o0zdvZRi%-xL7c7A3QgI zH0~`=i!4L(x3$hWfSfx>qC20?H44icm!yk??ao*jMm?IlSQhVn+|@PYUw46nBp1_B z`+th=|1u%}zb=oS0mtUwO1k7ZpsqfE_uuAy`i&uioq^5%AWkmJciC^jf&*_pX-`2I z_OzRyb??2MI_z?I*E4OBv?nmbbg6R+t_9NA8b4%>GI{7D(O~Gg-$V)axV}=H#LY%l zx#7@!@TU(?swHHara7|oHZFNm%-82TN4UE(T-$r_mD2a4Yn}=4-?@i-|ILguntthm zQA4ox+Z*|GQ41`^g+p%IbTk+_IoFVifWy+l`f2{vLL7vL;}OzAY2%eJbRU3Wo4Ytc zgY&0SW-4r3vVGpu(s(&JR1N=m#Lafyx}&`T+nfO12n{(gJs4hTUo)+?u%Vs8xvy`k zvaU#b+m2-{%hu=hYpo0Gc&1DSaxJzq++Mu4Gt>NQ0~Y8hYIXp8jOSE737$>ieZ4Tl z!*P?$Ijn__Ejw@BCL}1E)B3oep;H8@1&su<7wAv2>epKR*l{04uh9e@gq3KfZ)BC# z9G^*{Ci2cqQx>OE`&brbi~rl7tQA7sCIX#@*MEk#SmcOxYd84mnqv~iysxjvh!L&g zePq}onkefwvsl)@ta0#yrY_qRRPeQX^X<A}YR!=c{7-RM-`S1Sp9F!t!JTZ|3NWR} zr8$s4B57W2i8;*ar+q^Z$M~NqKF?7j)?)+ZZ`b?MCjTPqoJ;gRPC!`~nXhE<6{OXX zKZQGesq3FkKeFBsFW=(68Rot|Kc79Vq2*+7+Y6xF(4q#vmnW=a(j8?rzRp_+%i6K_ z`kmq*@6=IMyz>iA$0z(F?2VZx_d_<S3bGZ*o5sydOSx%KcvwF%fQd#o?2A?l8n?h9 zN4m$rdjWX`%3|die_}x+r1k94lcZaKimLs^Hz^PK|9264^@;y*kb+CeK!f!~E19i9 z>Z1uZt`ZE*<<XeQuOUs+R5`Qdi4m3LugaYXRJOc6G3Hme=V1s({fwG{gCWG6X=VPT z;@(2FJ;~uiTw*kwn7i0P@0`c~SntvCwD6aE(mb51ZC0iHqKfmo{px~jv^o6RYjpA& z3MqaB_`((s%e#m1ddR;6#p;01Rsb-hP$TpucxjXPN&||I`QDc#{@#-W`P;lI92xjL z9jlL~vVOo|htU^o3AxGZw7A`ZTawu$i^L45_=-L+X|ThpP>ek_Yj6qDYs_C~bk4tq z_}FDAl6njz1n)SXh`btsUbsf%S{~ZqmH{ysQNd&mCZmmuCvZ9$W1H>#n_lweZ&sZ8 z7^Sr<DiQ<{{Z}BRgEH4%3>3tfc7Uuyr;+8CXCLqKAp5)11lAUV|4uhFcsR3on2{?U z4wZkkB=C@#>-Dzp6o5#99^;VA&>_)Eu;WaWZdo-kr#~Y-$g)#^noy*_x{we7(Y2Ck z9jwpu0$Xh95W^F5w8{525CPi|-~PWhqOBn89C<IT8)*L({_p<^IHy6Wtby+KO)61e z&&+1(FufUuC1vc%!L1DEb!<kcwmM9kN1fx?G0eVtx#WF{pxn~sPwM>9Z=g4>^{b*w zop)ZcG;b$>eWLAO?@Ykx-Eun_Q^!toloyD@K`Ex;f#U$+8?5DA%W_&+1fKVYk$;2- z!So+O4m`GptcY%l6H^L{8B#Tcs9*l9T@><tcNdJK^T49pu#DD@RPyZ<vay3ORoeFG zEoWzRDzQxYomz!172qS;b0FPM`C05>Odr8y0G~Hxsq8~{tG;p9-7ls~H7X)aLL@S+ z)zwff^*lU)V^ps?(Qp1&r@GD9Mke=hWf(90icZ9fw&`0g?CeQ)+kY`Pq9=a`!xA;9 zBu_jqc$}FfVIvi|ZI2O`0VV8G;Ea2Wa$^VLfv|EruY@m#Mj?Td`=#I~8=lOE+*?Mq z9tN&P#|A2FTL+`ETWH62lKK_2mdNns9QLO@4C~u<IbRh)u5c!>IV>_b5nXgY_J+cD z&wP4YwSy`u{*&o`VF_A`*a{Qk?UUrrWW2j{WF`Z#sN^rd)BH9kF3@8Iyo_B@nPSnH z6bAIA>~4_<r^8?15_6btBNlD;J6U?V5RNBa9Cqu|1AUl5EFIb<4dji4m$0&Z+dx{s zRQWTy65XB01NfWQ9IA~YMaIP=Lw3t64Q2oOca&Sd;!m<eM){|=WQ)xhouh~eH+#Ue zhD^|9NXCx~cPGNKn^XgA`X(0*LJc11<!~G=DZLpwMqUM!;2sOJbI>MeY$jj)CrvGY zt->Azdl#^%4h#9?FL%^+-&K0Nz*xF;sTU}GOimIS@bOV;!I~U^B8Ix+0|z}$bBy>& zF3F|XTW~F{2vP}pyK$L7fH_CqiEmQ;KDTFT$68dewlgJ1wK^0r(zL{W<{DQ!%N(xC z_YdED7g*}MzEVd8Gkg0r?3echCyS)r!cy3EZqndKvo>Rh%5Mc$Niw(u4=P<)BDnl! zO0RdIP}bo-1H|j_NgM&OklEON_tDCL@4B+np-PAo_Qhp(p`@E7TbBY_8BoQb`;|W> zZATV0C-ZYbU&zlAW(@hDXfZ^%c&w{YI{r@(@e5_PyO&dfuXK=rqZjSipS|0TzN>oq zWB<@#62fWy@yr21OZM+49p}P(cZ#2ynbCNVjaU8sP2zsbcr78=PNw$8>hJ$-iPL^D z3Z)wTuBo;j=f)TEfQwy%Zd2aAL8Veswn|ytK}RpglG06gCr4e4M|Cq=#;(^eHeN=4 z^tD1l7B;HCE^5t?qQj_)5<D^7is<81lBwQ`pP3wA9T1$;4?l!OK3b=#<7f-rP6FjD zUJB>iPn)Zzhrd{`oj_)*RDFNQ3#1*Izq@oNYGI^u@k?F#;Ba{hV1&20EHcn_e?!J% zfZZrT%6enTC#`kBzNjpf-q)%(UCcnWtN_8qBapZ2h^8u8a2$BxBg@rd#Iozpz-@t~ zt-ZA1pCp!jx$nT|Ya1+0VK?*309=apUQ*FhWtnQvY}VVqrV!kV#y&7<EcwNT58IaK zf&Vn^>pL&Ade}LTQrBe9{g;8*zq-`V1TZVGOejbUDtP}EO{8$Bo`7`O_^2vZA!Eo_ z6oY3ibb6HJ6t|fI@W`HhwdMairV~??P@3dUQ|h*6J4wq#Q}$vVw+^u}1@%izNt|cY znc}Q=Tf0z)>cdX9WA&tb`mi0FxGG1G`vrb79N`$8g{(7oR>xRga!O%+>0}^rHu2ZK zYEZV`c8gkA|GF)d6K6|<f$psp@;8fb!Br3L+a3L_t6{%UGZgd^Hp$v-P2%U~+yZy| zu2=c)DvHm$(Y_R9n>+h%t>T&EhqN=)^DT`sKSb&PJ+#6*Ug28rK{cCPUzlmMqpWj8 zN&?Z`zv%CDL3QSxoEOoc0Q@H68GlM#!jOv9AAN4sJDgl}lhx4Y7&w|}4|4a~s!o7d zB#5d`tYBX-_Gm_SVZ3BArC}9r)$hx1X{f1;(wnJxg|{uxc#1l_)i7jXx504TQU&yz zC?pEACzPn8R-u+LH~oW7t3jTq$-g0AzI4P3UhS$Q$W*!SH4;>a^hzps6UBa3RMTcF zLoB~VGT`UHRM?2N^_kel$q<0SYTekHgX1Gg$GIl$g2^l!XK!WeHyq2;h7Tr8y0WZ{ z@4?!HeN4$200CJ94Bl9Xl95=LPWho;8iD#H-;aGw{#0_@2Zs>OTmC*r2X-a?vsKxL zgl~SAIuCkEufBN#S*R6e@NaHyKk=AQluQ)%xj0wI7-P#-Wq7vY+P=r9XC*D)OJ6K3 zEttScOepwW4APOUqZBaFUa*oGwB^qqtGKtc+k;sDiyuu#+E+Ppx<83r*d#D75elZH zt|63EyK)j@**VAyyoBr__dkd_9qIL*Gi;Ef>iOT*2fxSh^sbYug~iD(vdpI1B6Y<c zT(?|uh&=Szgh1?P(ShRSFp80Ay=ru#-lu_KVZ6M&L*0YM$)|D-QQTgJO{eF(X$<~E z@g76;{=iMBG)7H}DvlBL%(ADr!lt9rys*KPLv$JZ0%N+)t7i%4w_3*P-x1{c{L!3L z8^q6VL}T#v*}i8!sq#Lj4xJd3ebUJHKCw2&bGD|e@HJ~_7NtLCVCTRS%w?>tm9Ezo zPf2tufVANZLku>~B~qRwYRZC-+>;v~*s9EMGum2M=kdNHu0z&wD#6^0!4}|jPIja6 zvA7^@^0h{{ibQGObwGA2v$eoC*Xs9-;Ggc0ad0AiTcg;7h55j*8z(x)^NZPkt5#sn zZ&X>!(OcdTkb7%}-P1c<u*M+!`grfvsaqD-R2_76M4OW7A54~w&U;AmWGu3HY6Y6? z2;a!vP48nrI`d&4@n#4dq_#5D_U_n0Zh0nZzM{kRpY&jCgL>2#6y-7>zNR{kDbHb4 z@4A}(x`OMXjR1$n2!EtF$B1Th6#N-FbvM;C?s5f&<Yq*28X>^phdiquCspD;Ylms= zrj_oV$Je;O^$%%-gjY7xdVOn4P5Q835Y04{mcb?-@??YKXlG9kh|&^`O#c8c8uI+q z1E>m1oeZFAuPDwZ%KW-s?qIGP6($pR<rbZ0T4!Er@%BidUpH#A@NsVS{9D1JvEI_+ z&=&hX)v;1HAK1cd?t4t1YKDcscQpU6=^rc&h@Udwu3Fpgvlk4o%ee1_O?&u%J5@sR zd{RUriFd^N^;W*n?NjUekZt>e;Epa<8j1Vt{46)w56S_Z;19k3GK{on(Iy<ir*n2x zv^ys1V_GBh%4>m!T&^n3B7(1waQ*R>B$uvO=_Y_JT8S(gPri4NY>(>;_2u5rH*lWG zwr%%%3C#DBa}PGDp#|^x8Cw)Y5w?wjC&`Mb=-8FwoEkSir?K6+7qvN_77G;3oOe^D z?$>j_!i0rll(?-|4JW+E=*vp>?4OYbf3TQadxRHvv|cXen~>2dAItm!Ub!_12>Puh z9}xNqLLICOQP9@i_ccUw%EKxpE!yzE&IyxC*on)yf=cavRl0ers}8)hlFH>c@HeuJ zhKt6|P&@W_*EV4zL5z(Jc?cBD74-iIB6O9B>Vv!-tjw&%>~}e#bx2V<Q`Cn3zRUK| z&EldrG*=n-YoV>?LgF0bax`DHS}MB^G2X_LK8-uU$xpiqs}I)9=#=4zylPY&W=;go z{q3%QTt{hvjL|*c;=>~H%Q(cBVghSE1~|;uTn^RSS(gBb;RGoRxyavP#&S)xKO*^I zo}=T9ITeRpg%f<VH$ho?tX5I381UN-kalL^NOVnWaA73ilRXX}bFEbK-zI+yFy?&L zb5JEBYoeY)DK?SHNCMhzbSU$(Hp{lzGfNpI&!{EY8X&Y(h$LbB{fs+ki<3+NC4;0H z%sq4r@lvO>J4k<Xz^<+T_Ls+KJ(Ux2l;^0mL2fLVbfTIg>F@=_-~P2EdT<4^sn`=; z%d<r?PkO9{8ir=8y{Pe0Yg}hGMHd&PFm@`+ERqomo8t$0ighmuxt~kzzkovq?{(Xh zQaPc$Y(rdbTg1|{=iDt}RJIbr!kG-EY&LX1L}Z+0Pr>R~j?FDDnyKBaUZH1rZdJO1 zo~hf=VQC-h&#d$k?ofQ!M@X~gGhH9(PTjq_`NRXxDp{zyem@(PWo|z=L~8C8;X<k! zdCKxCnG+g<R9kmnFBQdR<*mQU-g>zk7QC86440Oc-}^dSM*J$nnN#}H^Je!E{k|-8 zD*gQY5gAyv4NlyrkJ;Z)+H?Kof3ZCh)o}42^jBx7P%qC=IcIS?xcg~s)^gt8Xb{^x z!KRuLn+Zvm0=!WSAtle{>?+?|E;oGCFdFquBi&Oz>r$rdf?2(QMy2gfDo&U5W%xUo zBHaF_;#y7X-R!F!o!7^DO)3i+mEmOA8Wy6t26wL6CD^N`J6f#4`->O2+~tSUkAu(C zfyw8HgN;J`$=<))giRhZ`rJwD%*2QmG8s&w-y7YF%Va&NvoW^s`oa7m(G{*ZRn4zq z^!$**<Fu0#&?_Q{1>MnG4XZv-t&WaP%St+smOnqg$UNk{&t*1nU7Ffy-?$^2-nGO} zU~5#~K>KVsh2I`DXLRR9se+273e8(D2ln#rxN8_f>t-;)it@Mhxu9Z@bGM2&yNwXf zhf!#D6~UFe=O$TmU7696?i*q^enEH?p$)dO!VF5<42eR2j}%@-Kif@5kGeHw|LHxb zi~nW|+<C#FkILQcv6ezfA>6{Hc|QNgda)59<?J@nc^lDY-Ec{V`3lJWZr2`O;?px= zl;=v4q?7`ZYqY<l-YPaO3s!PDOjCTGhXN~o@ls~8dya$@c2^)L6D~688%!3)r&lsf zW3xbnE82~3dToBs{yhTTkX>_P(U(Ob+UJ9#aTez9h~bZEBg&9Gd3Wn%#2Uq@XfJe6 z_$%Is*?RJu|DWt{k9h|-0IHtSbMJj|$Dy#cIX>;BPbO~JeDIP9ak+x<oP{|aj#m)@ zx}f7f<Eli#9+AOOUpV*o)b3Gxm3P42OsBcHFAJzg4s8*RWjb>9b-Uu$7dK|`f9l2O za3zqAndj>gRx^p;y0_js&*Yob;}Ml^huP6m>bR1}Wca`o9#hzxq208L>CX$&9)RGE zG=o$aJ+oYc@Ve8tKu8^k!<X;E3UF1Hv7<*Z`a4wBjBy#JBenSW=*fJ^-mT$bM{TTO za*asyadfnSiiXa_b=1&qMUT)*Ng$w~b2P0frG!t9g2b_zRlm4}y5>gZT&-;7x^MS* zeD1loLY!QIpRv~32>poY&E_OfiHfXv!VLw{48}oc{v2m%_B#>8M?h<H{uM+?L#A5G zSl~z1wyvRi90jw=U7f4@;Mu`Qg-MZZnch=zrSBtZ^4KK$S@|`2bM!HmSHg?F42CeV zNu5N(9dUSu9eMi7D)lv1NQq85<7?oywn-vwcz9gwCD*iMf!nkB017|H%E<&f-foMS zXwQf%EkLE#yLo5yiGkfcu4$Kh(A!5oV6lh=Q4p-zn9G|X8K~@|ds2msh#yeusI{{j z4WTa<q%(bF<2%5#>A#v9F0V|Zc!SbPJW74<OUkYi8jX7<Mh?^%HYe`MTASv{2nTo& z30Z7xNvP_N!N%=Z^!^DpbsOS;_Et3ZGhNf+E!dLQW-{{6!zrsIAvz5hcfuq1`RTi- z>r!+?h9`zkOPucPB)zhy5ci4$ar*1!H-b`;*~N!8-<zPv3wzJl>_vS9n_D=dAE!uF z7RH~cpG26dQ;(+-3QZ{0&dps-2^7VML+L@R5z?U=<DN<?1xw(&Gjb`pkI9q3&s8H} zXJ>ldLM=a-K=v)WBfuFjZ^6WkY&Zat2>-IP8H$@7*ZT8~v0+7V!Jr-YeC`%JPC~U) z^$o-3$^Ix;ZZ%-YSpssY^yoblv)Jm_LIv8=+@XEcBel=QMs)hFds?au;($d|g}ZF! z_c!^&rMD`X{9Sgs<CS?)IRSaex9!j8#qX_z*t;DjhrcOIByBu9!}vl%NoAC|c&`gZ z3Ztlp<Y!)Lx3{+%j*o7)w-l;Fshk-_BqUoo)ZKj}fYUxy*qF>h(&y@Zj6(n-l(s?# zhV{915gDp>5e?pIU1bI0Nt^Zy@MBN(pSN9-OVWSw6W!my6q63SXXIFPi&qJF&dqtN zEwm62B(Tl&THie^183E{N%~A61vK+^(`t6>P|J`ul^<R*VB#mp4+PivAl8xWh_1sx ztX2C>kc)x0@wew+Q|~^dE}%rhZVB?|t)r&BytaPEmJmKzB;Xg0S?F;VPRpC(wAhmg zvCej&<;*UGbj%|!b>{HO>xo<XWO)54XuLb;pR^$T-w`PP^Cmh{uK7ymA*~fD>J76b zwWaUbdtn_H?^mhK#+9gAelzs1dH$;~K^AUt3ntE?U##3F(z%yIdRS9TSL?c}{;cG# zo8)n9dV*_VU)JNX5}uREU&3<|F^=$kjKC)24Bi%33iT;5pgcx2cVS!RfNa$R%R;K3 z=BA<u(~S(CLpc8!uEb=msr0^SsYx878OrPke2oSIf{ndHX;Ust-{Sw`J59$|{XE#p ztM0M`!@dsmlR95@UsILY<k+D{zj`5U4fjfQS1>SMfyZjQ!PE~1>kU*pd+X%tG+uNv z@Te`Z(9n8xO390h>B#k41QwpY|4;T*+*B3Hub`Lblg+g<m=YAJLur1vMPjKEd9y<U zEDKF@x#aF;l^myNgGVAe5ETfGHa2E+Dn~S}O<|ib)7)MgLdA)O#s|u#s>?7YpI1!Z zS3ducktG}6C`}h4<bJhBdmxT-<F^zK(K7+AErMKmOSJgdo?a=Z8gKaBo`YRtf}xt^ z=D-MZEMg8BH!4kcOU+4O-9Z){y&)GJHg)0K`WuZm6FY?Ie6+1gS2>SmE32)2mH%D~ z`9`0m2Z8Jio(&P-F7vy)Kh9vbz;dJeEJy%RLhOHvz<sWGS1s=?kk=?aXx^Akan5{U zS~mp<X_13;a$JO!cEH1ilm7pK1?c+jo&N`6hq7zi<sm)?sFxcGa{bu2J)E4PL;rQb zf1<F4=R9*33rQ^AvE;U&647h@<(ofR%A)ll(Pr;6eRdvH5luIazgzFp?>1W1YRn%y zs$$37BX{|$*Sb2_x6~aX7I*0rW)(iUvbT@R!TzvUoOZxB1^TWX!q4}nVv-=nVkh`b zcza3TQEJfjOZE7DoQ%h(MAJTO!N623CC93x8Hh${GTYg2)@;d*Z*Y%}{b7{HXdk6T zmbmCUtO3|D8A)1nWYw<@e=CSx7c|srxpKKs?(fy>uR!I0A}K0mR?m%fYiJ6aqJup9 z)!UYu%(+%H`3+XQ<3qc1P)p1=z%a`<%d#Jm$mgxVQfo-#*Xbuk;d;q_GFjiQq)?p} zedm<E`M~Ct?at3fm%S}SRlNIZT5g*Gc81z;D^Buu#Kzhw%v+$^psGDE?+hl~9vQ3~ zmU9BkU?390nRxzqLSCmurVDW);3&nxk!X(_yT?gP)ofbMA_jlq^VISnc6tIp;tF>= z`!z`~U44XALv(ok?{OhM@kcZILY|ZtCA3|WBp;PGRde{GB&m&b1h*^RYu2rx%B0eb zJw=A6vu9!@qpLc!9FnIE@88%0l!EoeVpM9%(RE&gamfi(R-9CJ(D^0w5SS>J|5-%r zMvBzWsC0M38<A7z6u`n;3&-+6kw0faDtBj%r^ICH_3tN2BCLvZMv3vsnVe5M`g_xZ zXyxQyW&|#if$r(f@5}$B>}leB_=`%kX%CNZg6t<Z2UG)wP}}*;eZ_mk!}F`_!iHgL zt)f>?>J%@b8U5f~`S|ilw6`(nS7@iscN5_gQg2!Hd+R;|oVyOc+YpGMe`|jo_jhg& zPOa5f8scYQ1WuVNjkyi)VC~$+c;+E$Lc9wnXj`$kH;3wrrpOJnNpmV{W*3W_&%S~o zN2H=mkq%1;u{pwrV1m1E^c_3M<iMr@sa)-|I*l%QREK$#F#&FAnwhm0W`Xv^w4h75 z=5vei$#HKCrDO%!d+xru^rOXU6#Q-FWYt1Zb^Q;0rx@uqzRCu|@|~N$;uy<VW2<GC zUkxzqbo7|E0HTQg^rf%V^YQt~XrCTw`R`B_rk{*D2ultYP#(WU|J{V{fjSM)S+`lk zpRUg$sq={0r8Nlx`;t*%CY8hd@vsETC<6zKdcoUeUCC=*F*yJ*!Wt(l;rM7Pd7IeP zni$b<1e1+9P_gt#X)ZBKle*<-@<_v|O*%2I+_ruiHu631T^F6Q;E%^+vPnrSQan+4 z{qoCs5tz!=?P(lEx#OmkwH9sI;8k;~GRAV(=ER!CUu!5w=12uaV(iv%<`KOrf8gM> zI?ayAkReMo+e!=~6HmC@RiBM~V>dAAI)r@$l-@{fm9pzo5{-@g2#u)?yW`GtQYyIV zdNOhQ>{&2zL~~laycxdOEd2Y{Ne0*dgG&7`3#xy=9OUu$xfQ*Z6vWQvF35W_2RiwZ zUE(0{zPust)->bYs_-A?T7fr5mMaBAY}u%V1RWkT{=%DSYJ&};WCyRNg`0w6MwMN9 zOQ44(_NMi|7Yt=e683E87x3L(UK@`lPB3${mK1zLp_TPOAAu6l6f?ShRE@_i0{wWU z+mUT6xq4wP6+`rX%OPBY*;OBlv~bXLy+`ky=IebY9i`jVA)51;`)Bn*Y-l&Lg?5U; zBJPc;eQzmrIKk5f3-uXO9J8+^J?eNR)%=u)eG;L=Whv;CUEzJ`9pESZWP>&MG61UF zqYI|##D2f8&G1@<Ukq=X!d}51G6cQRSL}zj7_!J!d~h#IuO+^;D~1n30}azHka4yY z6+|uDCSy6$dL^JAMT1gq751Nx6ytvHMPHHNfk0PRm+xlY$m5<RVE^6)bSJpUPWR}J zoVjI<AckWg3{L^%9=&p-=LB<4j56k7pGe;9wTvs%huSDlc#@$oq=|Pqa92ryDec57 ze0&;E&y|#!(pb8Rv9qWsYkG_!%hNt7Q?oliGiZn$b{`=Om9L=tz~m^Mgq8Srj;s&O zh&TSo##>|?Z_bv{dMbzDs%Z_<VvjiN?WPwqbqjW33B>dPx`ykA-JVMqspofPw{jIN zk0YOw&-G5SvOrFm_5ai3su<NdAvR&aiZ+>j17fnl3Nv_^i0;UYC#<ku!%%lSzbeuB z!kvGY2IMXcIWjy8FYD|$XV6yRc7%^L^)!KVRq<duEXG1-?7K#QEVb%y2#o%Z#YxWg z3zlzSm1JA|SY+QCd~v2)&XCh@Y|$qU^<^JdO>9yjn)WpBPU~^9t{^Ixv6Ik4AfDhc zj`*fo0^oZXPEEULAakN`CflQ@Yi_Sl#29YBGYXYu#+j}3#08}VkD=Jv#Y%a~m+UR) z_!++dvm0}7z0uiW89V9IHD+64cZWc(b^mDQ7t%K&pU#wqcaebuuP$}=5Fr;{0qw$7 z2o?L;{mv2Ll}plm%Nh(_jK2li3}55I{1=L61Ucw8Yb;ry!S^Njn}r4AoTTW=pXJ6T z%Wrzj>Z|comfO+^a(}s8qsgf2Bg?J2B@TP0F8DcdyX9!PntNpoDz2_U%~-;r<M^Z+ z^zw*=PPBa-8^+f)aJi^gj;MwH>=9~E6j}@e2_o95`wh<yWnL`b)1m}YNQ#3x*S$#P zQ>+yZ<tRqO`t-^N+{ez+FmCuUwn}{gsv>_~my0%I!Z06at&#uUap^jKd;0g9EX+4X zLv^6+=QK@7JuJs7q%W>bVRR5NVCOin4B<btpgi7=j<|WQJ}?$3otV(o7{oMBINA43 zKw6J9-Yc9I5|NWzuOF!5qq{R2O^Jkf^>db;(hL9@g%@E=f}a5)nvIMTVOT@$hzG#x zc;vRK?**7TOzfY4V4IguVd7ho?f=$v{LA3;A5R{s0F#{N*>y;Aabmk(uy;|ja-S@1 z_f<prc~hc_IEG{!ai|Y^XtNprJN~j|#Z8P;)o4@5rCjb>o~!uKj1C75)|I`qqO5*@ zo;h5#qqe+V#W9B7+<s^JGEJ#UlfA`8HP#JFES5Qjx0l+0M1FP-5R{CpmEsz3r@!y) zX5*&bUTURR?}nkZ&yQ#>-c{c<NyBQ39P{O6s^r0ZJhw-9`8}@x4cfZ!W6Rp@=-a3B zl~MT&N*B|<_NZO3P-9MFvePrncstLdcz6=4XGD4!RZu9swUO3YhZ!!DaKEo-qn0Rg z(o{uzFRt~+QPHEOFe4t}QhE|FoUSLEjh;KBLV5#kV4@;9*S@`U2Z&k9^na=6F%qfr z$V?Z57;k}h&gUOr5mQ=K5tLItPH^J`N0o-rcokqhNy)s7f&8xn6)glkJg*8!F|{~9 zd%Qfbn_U>Rc*zF(%edAu<*?5pem7>aJ<PNe!NA@hN8vh-K67`@9z@fDmd{gEGqN$} zTFPCypPIz<Mw{x>l*Az=y^*QMDowb+wztl_q9ieclRfvVF+|og+ONbxFxF}v?}^Qj zG!g)z0y|+0cb@2i#};df97k>!B@31>&Z{pSc7cT-#c4ABPDd!pa(#N9%y``UYD|vH zP(zdHl_VD@ae8B3a+z1@cz3;~&iS2?FNCRs9Wj6!mtXqLxto0jJo-sNq=Vl`LGGoM z!4c&~%*vh5L2j4I-bq{cGuQ1erkyg*b3C^e8C|8FW)-~#kkbkq-b5)IK>H#f+E|hH z*8kA@%gxaF`4Qah;vh(ph@Rc7Z}5WegP+2Qu?EV^JHzx3o4c9^V!*`>hWziwI-@XD zu?bPC7Po>WuOAy+muNhL*@_l_%Teyc>`o{bNY%Pv-$i78%0O4B9#Yb^hZ%L8jK7gK z4-;$#s%&X*KcW$*FrW`YQ<GbqUM_u-i!}vm@p1SBM&ty4k+y20scYgD=ih4AfnTl< zAX~1};`XL#zL>4jeL^;(W*GvsdD?3l*hA;p0l)HFT=VlCOHUF0N7B9%&+-Q2BcSW_ z7H}?g!NUqo(5QJ@Z@NVcx>6i<4N8MW_}15jftypE9Tiv>H^q66Q4At=!d&yIJ6l|9 zsm8-_o|T-O`B!Pr#io1;5>Kv1@OLi%I7{W6N>_3!G3bBH2I#LuJvpj)xicB7r))bc zZ&oC06jQEfUPYCf=mx=#K}#2ErDJgI_t5cwx_%h;S0qG1UjJn108(h=tFZn$ej6dm zjW5X!ITNj8O}SCbAtKz-izUsyO8S9Zb})EjW(FZd=L)UvrA0TGt4;`~?y^39>*U!X zXri*5S*`68PUcZ_>`@`vRL$q7bpWF^0zU!Q!ZM`tMip~gHog3+dhFVdH+cPMtY1oE z&9e(oy`~{mH}=2wDXscb3&0eoQ$zaJ#<7mLPgS-kWtl{=OiGVIls?AnUb7xVKfpYV zJiVVcDdA}ru<GfV?9b$5u{BCjyRtI(TuADN7BQtPzmXUifC2>st+Qq$o$N}-rTzSj zNi?PFhQXb#hl6!@`~lUCOhCVxr|$S*MQV6+1Q7Np;vBKvUK*<Uab0Q<5HZl$@WRyt z+L+L{Ii@cE*695Ic4nHVgKcmn)NowIMysGp<?10KQFHI45tiwl&Z#3fKq^lNOjFjs zS|NCWpm!Zn?}4EzNc^SJ!TNSH)gV}U$`Z)I<EbSXky*YboFx(wuOI$}^KXuJccl8- ztmCSS86SEc=59G!te&=jwV&KJY9^XqtXH|NKc14R*hp)IAsxnKT(TegPTm}VuS=<q zBc>%uRpe(6kBjmOmw-vs0;hYHo4ITa*~3iJU+vD-;m{%h_9Q5IGy~pJZDT4<UzqVW z4dparzipi&k&t2!tYG7s@HYy@$%f@7v%jTp{;szXS{7$`OmBJtv@tN^9QgTig)bV& zH6un;k{3@8p518{mLi%w$CCJIbFJ7wpBI9IGrx5oE$Wz8am4ek<)Y&fmC>1dmW4Nw z&eo9+@;?4F-q*ikDDa&G3c_-|P(Bl{-+*1B9wA|Z?cCP1Pyy!33v&6Xg;Qs+D}^a5 ze{wR-(v3u*SNb@F;J-(@i^?x|O{aDZs8Fj+M(MRZmIx|3EJ65n@Wpf@cI%66Gk&OM zy{euqD(egx=_bxSckP-vNJv$9fJ1l&?G`5vAaYK4Xi;UziS-uxlnI7!+M^}nD}yO$ z9Gj9^p8@iYUY5I|WYpS^=%PZRR>kq|bTnw#$jLaWz+!bZsZ)V5m4Ijr5`0TzNGv^) zrt|Vs>T2u{?IY}_7g`TV>}HHvoREIB5_~EB$2uRcSWvTZaLOKpeFMbZo&(x_%u|5X z{d%+jokr@TkW-fSP_5J4Tg)hQ<*=iaH`C+u)(l{iy}a4Mns;i*`9RnaQ15hUrBfO- znkFS4r`~Np`>${wry<}!9-61ZwsMl=!m43Kzq&dOY0K7Cdpa!IU0&E@&BII5(<Yz4 z&!@A7pML=APi$ZB^Y?|~HSW=82HS~q=-96knp_U?;`1VDmy-$AZio0?$zz_wqCHqD z@<WJefk!c}w5!4DMCJHV%oDhCR7WYD9Gqw>=Gaq0<C|8mKGvLgB3E4smb#E?qQ3gr zeUF?2pUG`7_8OS=fG-g7-92=kcpsYZ=64+@6Ms7QM5oVq>klw%`MPbev#L_4QbeCv z8p#($g!pE$d)7IPsua~}n;6u*$T7QTW~$t&4m$d?V|t*!g&w!>eSxT0k#LR^EjPi# zX|OgzPR4FoH&^c+5p5|qE~aNRx(id!_x`w_bjtUUV`cbDZOKBay~kwP)l8qM9Bg>l zouFVrb&t%-aA3FaIwo8?TR<1^;*F)hv@fZV+)wAG@I5`lT4g*zA7WEv$&!jv!-ut5 zoPWD+{X4eAwx~`+<II+RXmD=ZOGkC*t)7a@n?s2By|q-Y?=S%+3(<zfR#bpx!ysKW z=zebG4dEn2;aj6S`eQ<Rdt)o69A@}q!~z^-b~%myY{Fz`1I7-7T4<%0lGR$VBCEYE zcIKKNOt~G}Qn>9fSDui8Km+dLHH@hiL@Z-><L2r_oDm;>?i1_d-?}ILDZ;=y1Dj5X ze6yBOOkY})WKK+jaAF&!mA~yKaH2Z;jbnCqrZ;!hiiW-8F4~V*2Kvgv7X>7VH!gL8 ztzS-_j=s_!957G`T*(3qOk4n6knQ&sfa=+x+oV0Nz_54rZ!jHa#Wz8dy7=`bibJZ1 z#RY&h@A}C1_z<?13&<G--FLfP+;IX6xb;6ZEOO0sOYQ*I|CR3(D9>eGl`yDg<uaRS z*d-5+wgu&<;*88e@4b5kCe6R079Q5D=y48>23umN-22AAn|)gBpR1wsRcE#D>^NvS zClHg^fRFv_(aS?`roJkw(bw#<F{%;(u!{QjFcvF^jtVjk!%AxhH9!ekI}Mn}Z!~H% z)I)00)=@`;al@z{pzApiUxR~$ty>31B@I5s-O_{j{Qh{0jXjAlb79GlGJ$G&f8K}d zyr|VX{tvI@$7Ua1glV5onyh)1VQ{9JUAacLMZc$xvz<W@TbQyp)t>J1eiVvxP%Wv3 zL-lDUGPAuxe{62t)#_lUp1;Zd?`YH`1wg{e6oq^L^1NbVz+>jgos&TjcU=yD+n>wm z>_1=G!KyuG;NcQpLP&4xR!_pWJwuS6Xx_)LxdcvEDM3#fBa>?qjA5CjiFHq|$GB^2 z<?%+U3b>g-nxISDUu&ZuSc9y#YN3z97b24+3QXc+2f<_gcfq5wu0~BScY}hx+#1a( z@|8bD@2k3FMePct?^?{!G!`7i$A%+jN2shLYe9(4Whqx!7o`A|IH~c%_wjK$Vl1s5 zBCK7JpBcvcG_PN78j&f`3lO?1<8P)@{`w6b*KoKfhrjHraan7;czK+ESR~NUn25kb z6*bxzq(1CS<ZSSA9CY8^M3y7kp`XSW+ZP&t9^}?;QRwylG;FonEH}^h{@ooI+0r}< zb{$w<0S0Ph@!!8kPEOUVI+zhxO`N;98+jT}jwvGm5mO#u+rI(Hu>iuwb=$*${h8Rn z>gWt05%W^%zU$56s$rGl@Q*M5)8PMK5yEGIP)W83tO5X+4Tevcun&}M(R7YGwK*gM z`<)+#PSDcwH=6vPuT{wzYI|Z6Ujv7*`8iwB{pM=OKsI!KgdS$IzPzbQX00in`e5JH zqrPar737<Rm|jMiata}24+w8LG{?eCrK<Zgau4Af_r(F*Nb=gWph+_xB^$kJYvDLY z7}*k@s__D7@MPOboV3fc{;b|0hjC%wr}FC^v_wV)?n2g;CQLy0XNNePS$woqmPpAu zdbOuCmA{tI)CV{OM?27`!6YK1C`HRblxTE|I+`B`6{_~s1I55;QYa4mdaB0PK<ph& za$)?PkXt5t;TO_)E{fn!M+<UN)Z3m#$oY$S{ol?c+r|#H7oo!(KlSq8^Ve+gbE=Li z`<Md9aklhUSmrC0<BCd0j=yalUlr{kKFTsoGT`^7D$K-9`B5SJk7|MiA7|qaH_?I4 zX*YbMRB1r<4%wlTBK!IWWeEn){3;oth~1C8Lb(A>X|l|foy0?!1G0|`+BVt`stS^) z&t5YsbsFKXf*ulm%=ayDreQ0GFcrV68C5V@Fi~@f&>6z5Q*OL5%1kpM#mcp-$>K>8 zG==BP>wF}nS9VD$Y~NkEsG)rI@FYIk7NInwA-}S^1_t+R8%x2<-Wmw+N>|dtqe1ow zJ38mB9BDuDSE|13I>$HPLQ{aukRjS7INkvf!$*j@C2xggHQMMT9aA%1y|Vt65(l21 zpVyft<ACOd61F|aOazSHKLJKeVe1WB{VWWJt-LZ%1z4l*Nt9^t949p&er!5ga~#=T z9zkw50}arKfpx|+i;ku}oiv#{_cQ*-q0Ymz08cgkB9`P-p%vFDAV8Ad(Mh_Q!nyb} zR5^uST{s;xYf=29@LUBdA@r>wTi<Jk*FTj##LID_RkN;q5{rXlgtSZXAD+sL(X(tl zVBEAt6G<N8^ebj@QL5z3|Har_hDF)7@7_aqN=b*3BA|eDONt=E&?P7_bax9VsR$?_ z9TGE04Bbdb4h=)+Ae}?QdwH+*ylZ_~|NHsEFx$o#<~rs$kJ!)sXLhHvC*hr5X6$%n z<XhzxkS<xUb9%0ccG{h>M48hE@6L>*1k6R4I_|)CkPhYzPHy|q`h6D`k>n(XHCQsa z0r6@oEI#8pc3LaTPKel(SNAyebd9<!sLayto97y*DZBlje)V3>2Vm(!8twXW(G`34 zO}~tD8+f2A3p}j)?YOQ2`qJ)653;YzY-mEzzO<w2*MznL=?4?BXa0}H0D_nz?bP3% zP;QYN7zawdxwp(uRh*o7A;ItPw~1Ck^0tp^!3L~RU9AGt&!jO7Hoo6eQ3u7IT9#@X z-PbxM`_RBvF>c(LsNFmD5z*aB`a3wMLY<dxO1;R-4Ez9HQX6?BM&)g%XU4}Gvzgk* zqIF%>VNqgVKfs?7WhUZQZtux!#>~h0DHXqM!S~B?Huksjz$N+Orkzs`*?x+riV9vZ zZI)64r(u!7iM{qY#u_(=9}4w1zN)<d<m2U@$Iuy9$~>RL`dlL2?yh+wO1(#*_~0>$ zqrb2C>G-|0Q!H79O6Sd;v%U0sRE+l>Vw-bRv}$18W;kTj-)Sd@Vj=vMPL}*yJBf`s z2}ZrMjG^d0aHZ1Ss}sHsfC5yqF2{>DD|(o>m45=Tkx{^#=p|s<@6~Ei57>D8VOTuJ z{cGtzzZ%CPoB6nI3DXZ?B25bjS&;vq8u9-x7XR;G=wN@-VQ1fPnn-DsoEsqVq13Of zI#M_y8f`!d^$fbbj{o|ak6k1=L0p-_#~z1wkIx2{b6QR5R2(g4p!6iuTx|!;{MZz4 zw#C}PFoxc@#%tdGmm@>#b&GCW%A6&7ph`Faa{gPqeKlnxexO#lqck~KJdz%x0o_gn z-<|!~{JX`<3;s7x!hvz(73Wz3vMLYikK1GBw=v%B<X&_#U|$uORB?J<a3%$$OPMp> z*z)Jt7cXpO6**=3(48nFVtHY}$=r$fs9q)}CPllR(v5-g3E5n=06~MYT(S(Z%xvzR zIitB_aE*@O*^KTj&Ez9}fxt3Dy>-zN$=L{~r8VDE_W;d{&C|~^WnUPl9sJC<G%A_N z6XBt$cQLH20(X$;`#|C(Cc)&vFnniMx^B<*NY7IziBRt@RxP^XBVbdaG4PO`s+K@6 zYmnj@l$fTJb^GlHlVKg&gd#r0!jglnnt?|uEkqv{AC7vKjdBeOJ^W<P_6c>}_}tX3 zvB_<+T=zh8S6hJgwQGKf)3%J&i+q;y2HPKL9$ZuAR6-gh&wIpdDJJfL-)4-KOe}oO zyiQ+nydof|6Hj(joS1ay4ec2cY_@qT5W&mVlYjL^^*e*jW)4L?N*A`nbGV_~>ck6+ zssrv3ihA!Ta;y=nuV??G8T3e~QV8BG1GZ|SL*p~fER}6iI_{@v$IWFj%{`Wyy9^%% zn}m~OhN#+i=zXzztsoG`ohg?YmWmwn&)!A-1G8&W+}lPifY<^6BB}vu$qC4H?UTxA zrvKi|z6<bE{Bp9k<+)9Ut8zlcu<TtxnuMo)Yfm`FzieDB>d1ld4;de{m3ZEOyCV5o zDPHY^nj?X==7a3Cl2W*y|7pV{T95Cu9f1(sOq2x*GN~jT1^?3DYxD>(J+1$MZOB1B z+1KOPI|dIN-^gTcjg&19RNGT!o>cXLSGW^95;SRUnDMc=Uu&-Kigv>+dn%^X%i!DQ z?oN{%FVnADK7fol;-@mxy>=n1<um@8-8F})3&$Q;oE7@<ET7|chY!2TgU%0_@8`Zc zwVKaLQAr1%!}VP1?+OFe97Hhf(*u11Cwj{q(M8W4|I~iC2F4+KHor~Rw|ji6t-Vs) z=DIK4SwzIiKUYIa?P*nebNZyu6^<*h@G+0t7Gwn_{+87Ep(YsL%BS9Z>CNLg9OszN zCw!}#X>}o{FZbD?OuI{V>WM=<F|=n~kK3L1Tb~HKQ%^|)vi?sE4~8ai`K)dfUNzSZ zxYH$97poRNY#NuS5yG>~eeTqJTS51RJr|Y@xSh&KKW6)cOj<dmH@uMY_*kF>woC2P z<5P$%IZMeX6wh_}(fTE%boE}n`6RoDlll&Ni1hZGnR#P^q)nkjmI9`SZI<7J*K2>J zv-LxD*8}A7@J?NQlf&D+Irqu_&@a;r{fg=RUSttN<vJH)U%0N_`7KkyI!+y}zp-d^ z<~+Nu=1fk8Bxe}B$b47*M_crz&3K>oY%D>m0$mJ76mQf?WlfA<fY{xHDlI3LA{m@E zAJV7xwE!@lNAo_baVG%3`s3|2N_Grz*i+?Dj?FI6NF8NZIKcIr-!_}_-9)89U~0xr z^$nXZHUFapcnEDe>Qn3fYmfG4DV%K2bsq364EzT`@96rkMz(U|p%~POQJu1nBOl<D z_nEp}lm$2hv)CK(C9cP>#+4V$>$p~ek>iKsXubZQ;Epj&mk-1RJ=7TW9!&OK+w_3! zv1#M9+;LHv1ztZ@C};FpvNG6y?GQQr5Z6>X?fBW5o@5XMrps&J#<+^v4B4jw(vo#? z8K#=Znwb{eH&AIwWRuO8y;<aRvwOMjlH+y`SHbqXVQ8VR@o{}rPg(tb^K{T8$umT? zRs0{BT_w?)H%t%K<`@*NrbJ*t4vHg`TR$}Wv8WAoOSr2*64MXXj9jPHp_<m_nv)-A zH7@~C_nNusM5>iFK|Mh^AbX9X6Zy^cAqvGFkmj=SRT#3zkw;m{<&oodU5OJJViGi- zi}N>DPjEshF}?vpj_D>J$b(AlhriMY2S!EpKTcK(4G$-USYiHXHq$6zKS8R0lOpZc z)2Vp+pt9hILnY+JY0g{0mJ}<~x_phLp-+j57kR;ZA4YPTKGAWOr+jQeT|FTyW{2`o z82jp>0@b-E_lvN#1-#0iJUJ5wv~n<!Pq*rCYN(o{NP9)Zx0$xIg3Khc#!u0)2fX`{ zzpfJVc`G7+xk{=u?|q*$9G84%WdnY3A|Sr)u;5qmYf?~n5g1GjBQvG1&Gx=oqT2oC z^|iI6Rpo~}S80vBwa$kF@k0i*^lKA#83ZB`-@bM(?imNRB<~nYrwp<D5tvA&jf(CN z&wBWVOzmK(aJrd;iVqvu1*7Cl27~bi$vW5WnrLrd`muuFTJ$viJ*%7d1D(CeS9=nQ z0B7ish4b!b?{Ls3V_IW?=gOE}mF*|@Brm^ZA%J!7@)tX62QWoq;{km3b)*+SWL$Aw z^PKmb8u|6%Ue!wH^Z#j3`Op650yy4LRlF^ueLd_Zvso&oPk^I*wq$8q3A2R4rL1Yf zI>3;JU;9bIXbbYkgBjpr-f)gGD8==;%wy^q`|gct$ggeD2MvWQH4Ez^#3wE5=BXP% zIXMbnt$|ZQ;sOR%r`DS1Td%KlKEU4v@a?BY4~wEl;iaa}?za|SJGe&$n@dC?=Nzk~ z&-8c^H5L1BnbePKlo^2@$YI-EaqMC$!0YGPcTQhp|GkN_tOb8anS7B+kocj^bKl=6 zB|f&i5}QMn<1O4~=10Qb60h$08nFK+=gpIk$)44xbi2ovg5_k)==8c04!rpdE}tFk zY1lBHC3CkT_-0<v0n2<nCNlN`{2J(q>|zoIA8s7&SdDxfsd>{Rtx`%By558ZEEg{@ z*{|&}1C$RG2@WtrLpikpcflPV47K6m(rn%%`<LdA;<mObN4=eh!9S`hb<_o`k6g+y zE4cK1SH99%`Z}TM6YdYE7j)109-n45eg7=(=;Y^CCiv*nibrb&VI@1YzYfn?&Uy<a zjo-YIwY8AY7ZzYQbU4%3STp*xmf&z|_0LVa`^(=cX^yH92?R$AzSgLsQt4C|vfr`A zz6HN51m2aWdDv$RwfMdHCRk=zG=$5bI99JN>(!Jot$0Q`o4RKDsJt1E!{x4i#g3co zR6MK1*3r<1>|G*6N5}VLR_z|z^4`OviVUrW75A;6w^oVAnooztuRJrJv`8<y4F#pM zm@#)fIQkl}vtA#^qn9(kr!PS3Hc5=>wz)<ZAX^(u=ExBdEp<LwCcyp=^rmsQW}7Bz z$_T>Wu=0`5c4xZ6u+uep6(HFr0T_%_&*tXnhbw<g4lhvWfBh&q)M6qAFpnqn%{5D* z;PYP@%CB-M{$*}N%mS$KuUw$bgm!E^GbDM%0R!JB@V|;szKoNzSuf%oEerv!M7FJG zrA7nORETdAJ%P4{O97+Dgu#Y3@0$sHAE|vKf>KUz5${{iKXVU*?ddiHuf|`SN~yb* zi+UqV<oaB{5PpGI5Z$G@=i5zKr#BW{4RxC#Tiy*_;*L&Ur|)H@xhM26_7YY*?)V$V zaN1~*{8bdA-4&rA(__R((LG6#EPv}k2y?*FT5}4e$LSDa`08NwQ3DwJ1a*$pDi}w4 zdBTwPp(xGXnDWUPkDd+W;kYO-jxj;~hwN?Fm8=Q$gtCJf&MK$tAKq4uXKY4K8owdt zY9h++PYlG&PO2iC7DQ=o@Ao4^>!gZ_$Q(A2FUS*JH5g#nCpQe{Uszy<f^{3F?5q$r zHQvf22e*exrqcSgzbV+I?R1k~bS3hUN5Y7H8$6^|;xA2XW4zazh94pY`#Ck?@!mXs ze9N^WIl)T73YIu1WC(=ivrFWh)X5PyXIi0;aXY`>>DLR5=W`jnb(fcnc8JW!DUG@_ z8{2MI{qy%Kk_MBa6+3G8xE^4mGIHzn2CW8~ZbrYP$0kXXt~D^qh%-9KO6)86PxjST z$hb|KqPgy}j%8KCHXysVm8}W$lGS2aopXECPUXY6{SRywp1}0Q7fO-LeqZI|WBCrJ z@jN4jY7Lm9#QK8})IXZ5o;{HFA87Jd{2d@|*qU!Ig}s*X+S<hwi^A6hsuHYRW&Qi} z&?3j+x2Hu<cQ!}z{@BP~-EY1<EgvRat(kSo8wO~c(!BR<L_8i}0zA%_Nt~J=MN#U# zEEl~h4duQ-f_;%{;zj_i?BmspdM^1dZ4(GX5!eE;6eyJw+7Z!sC?DlpKf~rnsN=j$ zF?P$Zq5q(+Ay+VJ9Ve&HM{FEj0~7(OGm!x%93wM_8k0F-8JMA>VK`6>aC-X!)Al)) zxVF+`pMkHi7WL=9(&j2CE!6g=>r;2WiB6hb{`f6FU*oYS)siBXKG6E-T*Q6S#(eCs zKEWtd7X?_n&ce$bhi{V6ud4QE>uiYE>SwxV4Ef}ydK>-rQ(*oKsLT!hr!8aP;CEDp z-Qcqb-6<88HLUq9kv~5dT?!skRPry^+Q}Lrs0hBHw2jpChZ{-MYTbkD8#Gwg_3wSs z>(Mp-HU)z?Iv6oC3$^$@-VE{?OjG~k7`-Zrmqs`;myk(&*J4*-`$26o<~Z6MgJ7mK ztc+z1(OT5KL{EZwv}8o4G!~3qwiz<4b<@Mlaq-uCQ;T(t*;8+hq~~#|wu6cW!Sl?B z2Z6R~@xh!E!Ru3Qrm9blOYr@VQkk@ZoF9&VwS}OVK0kXsA#2s)3<L30_*f+ie)v6G z<D8)RBA>Wo{DM<YN^v{A530*NV{*WITlovsEX6Z2^t)8XlsB+QX><Al{Q3q7Gqn|1 z>y2g1ONLqb8gQ^g<^P&)nIe{ccNFx3aVl_4^(x1cC@za*v~a_6sxNal2am({L&{b3 z_p|R+hI&bpR!C3NIx&+GQ){D{;EJ1WE5a<D3xltJyd*XZ;8%!m0u`H2b(m$ctp@A$ zH->t^#b-wkRUD==UfWc+XdOw|U3^`vVy{g-UI<#>cH8%qicHRRKLhH8>3J)cRy-h* z%eBmvdhVEo0xwVYU;<4C3Vv6!i<u%+`Z5auqB8eLh-h%NJ1+0y7{zt9*4J+`a=tDQ z?0aH#TXmZg#f%Ib1b9HesxdgAGT7ul`i=i*jgY7U#4vNwN~v23i-6B7buUTU`zjA4 zJ|*esn8x*jUx)sDXM(HEL2Mi%Lg~c7ZWt$L-y`se6=GFRabQh?{JiO=bcJX*9!{@D zd5Ii(rvrD2&!K-STCY73O4Fq2xBQSHivJzDQ|Z;t5<v`CLooBCbGTgJO~Ias*WCm- z(Gie#>!q~<`E~h~M2%S^r@soyF$wLsa|1Ax$LF4%EI2;7OB<F(&z0PlMN9@&XjtFB zaaZ;1Vi{q0h9INeYB$x$66q+1M)b40{?WkJ<Y${wZ-sw;Je;HHTDJ@QdNYG&H%{FM z8_cs|i2+}b6e=69)G^D&wC0(80Z4XZ-JQGkZcnO?s=2(L-|c*t%v!$&%zpkU3c9gb zC4`luA_JA$8ntGIPmO|()*20{)k!5eX}9fWwhdgy+>DCKpG)&S_7BTXer`qWp=h9Z zlhWJcwf6)Jdr{J5WS2Ax?pE(g|LhIL-yyK{BA$<&O6*BY<yt6Q^Kwe36nHI8^LbiP z!9G5lDT(sbhn;Cl<i+oFv%N2{=Am86V&q1=X(Pf9kR4q1-NvX*iIS!38l;Lf*B3rO zBg8HErZ%Eia{>hd-gH2J<?yAr#}^J62n3a<!D1b!c+rU%EntmmoeQbzh3n(H;e|$! zzY&%9U3eHp4Zgha_>qyp^AEnSny_OraLQE$kNl##rigebv=2JcSh=&CKH$dj(K`b- z;zGz3UkI{4pcBev*S-tkWT#v=FU<~BnE5gDD54c|u75X#o{<Ukdw2Q7CKxrUMcpOd zdOYxAxYpk<QyO*gVoJ{Jx602Nq1xJP^FhB(llIaoNUfF{p55Y~U52maMv&)y5V+KP zy}XLBdfZ1&>jOpwZ?_<GD)MLa_>V$67n=K$^6FTGhq0P8Wl|yfJuA%u@@rk&>7P>O z0e;Z>TLJ$Wht&O`v&Q~yvonba@^n#0-oHrGegJ<dvmX=^pCEgC?gk7dQYK3EwSsPe zwl5{?dW~z{fAPxfB1!flY35^p(;@H;G8&THZ>j{m$VGO~J7ups-P&<z2NsU&hXY|! z>pWVoW-^}@U>3{qCK3XMaR1|raVT~oEvo`|w#%p*7q!^K6n&-c;jU$hZu#gJMAgbm z(t;5Qr4#SQ>%=ngGCON1@Kdoen7?vkjy!iLTJ4Z3t5E?-!J6^f@<0qHsP`-*G!TDV z+R)Y<Y3LF`prL29Zh66K=hS3{-H!3B&Tg5jiaRm99XZ^~xc048zty(Oo@d{82emz% z7M@ojS=sD)`^i(b(!N?`<4-1uxj*u_3H8Tn99ea#`7sc(a0W9@t8Ky~lVbrOF$V6* zF&zX{>WL*B5SG~^nJmn4+T+Ys9&2$-SVq})rwuQku>fI=Q;VdTCrvjC26@6J7M#xX zPwJV?DO<T5iI-o=&1g{D9X^}!D#m4dUl(Fhf`w<JN$*>Q0AXdP&4T@+Ff&7PVgYZj z;bHX`>_jMm$iDew!Uvl_O{D4rHrj(#h@Y;4Dhn^IND{Y0b4-34t?Se!boNYx!{W&g zb}(xAwMJ`BT9}Dr&PLS}8Gr5SdF4A&Wd0Hrv8XM4>s*$$kHGyZpim9pDwjr(Izaw( z=e#9USD0O3<UEw;g>{_a#~3YJ*=k^DXGp>q8#u+jsO%=aNkqP|G-=Y$!%I+<iSK5a zj$G`&(Oi}%{DbBVe4TC&F*k<OU6#nlPD(xKN&>qRn4#o6`LZU2s}y?S;k5QGxi_y_ zR#2bKA=_#3P>f-oh}iA#;Gy+HPU!u?91Li{aL>;@COD&kv1EXwntQ%X`yAi;9kZO- zSPYn?P-bZ&-B_=+z|**&f&NC?pWm^QjAV0n>cV8<sBiSrFsxUj-<$@5fgjYxXZGqZ zj$EyZwt4gUI$+2BqakMOr6kk@Se6~{1p|mQ6Pb(OS}Eptr{?A8(|+^Y%Ij0$O$KlG z$p(OU_#t&QGwg({1k_o-_gfLSQf+#>4NXGSsyI5Kx~%`>%O-Ck$W5PpmgM^36Q7a$ zkI${}%rwuxc6QIw%hV{yy*lT{oOhd2y3A!w#9P8KW+ylO<weiI@N7s<lZKoRb18>R zIc$pJoMNW}kdlV)L$^b~2iW{&tmfEnu>3iwRrAQr9_;<0FQoQqK7^byoHZ9U{}akB zXen1oc61)KwMtDwnPftc@H?oL`!3WMvX(<Zr$NEMetcHe_#+6B$xM$SYEZs;OcuX5 z@?FsbA2EOaEco?g)2?F`n9IP%Gy5j!@^ij{TCzeG#gaSrzza6(uUj|Idf3Ko3oL$G zJ&2HM7<Hynh`fj!{f@5c-1-)@T-iMly*1%cti)ywJH*Go&lZSrz9ArrB8nnloqZZy zWAJ|>cLXa@WQn$<9Yg$_aar@c0Y~W7a|rbd{<a4vQ3$S}K|YS!CT@&2pI|cEjw|wr zm8GwREQEEVgdI(loF@|{yk!%nVGQj-2=d2-Sd$*B78%q_t3fKQmJ`EBlVWMgSkfDH z7gI-O#>cQOPXSVUZPw}VCG*nHJa_kz&sU9Je@YGi;bE%23Xy+F5RrfID1L0Lcte)| zKx1~S{s>KjUr(bVo$jY+;(vKUN{bkhuR`0`Ahz21StdhlC2BZjAU^}4A6@vIA*aiM zCrjqg#S1nH_;k5^BU~f_eWXZzE`zsdJjpOOOom)KsOXB)oyvTd*5vp-;R}a<AVRT% z^J@K&E4gZ+zJ~hUMb&edTv%_+oIL4q%fWkrFRSsAPBrVNQ|9KkQAhh+i+8tY&*jdG z0SYL#`T{WL(4+YaAYKQcYN#gv1i(}ulRDmis`a|R^+lUqlkk#T`S7*xf01{AKl$<* zbhN~sEg;HnYf@_V%z2aI;IJklq-lxir4$gLhq=@t6xf(I7G_`|C`X=`*s25n4VprR zj)28jzM2B9auOw$pWL|&se1(4?e(;tAajV5^Lgb$KP80oqxo?An?Dt6{%vinETIhZ zeMv#ue#WKS&GV7-yj|$_<1|TSuw(5nsR4H&HtTSqEeS~SnP+vl;MwNw6d7+Ncf9#$ zOMofD2yHzlXnw{<YsF&ln@tLXbF=5DM0-Q#$BL}is`;g!RHN9hRoVu|NjKevBZve2 zN`w<Y--;zw6lyqF*IqTe#`i6{L3IosG^6PW0z1D>XVEej?y?OvyDB;67?G={w@%NQ zHX3QB9t0X<_!3A8Xp->&UtZM~hb9nVHOQJMAhvL#jhEzCM4&1w?{yiTp!+#7A07nh zX2&D-Xs{U1cY+mc8x_J5%X6wY%l?D#($zM3OX;du@5%1_1YxZRQ~%@Li&jy6r+c*+ zfoAEY!d!OD*({sL@D=;yI4HHUNJx(EpOmnDN<l3Qz1aN}XM0wA5@sA}NV2=Xk%mjY zsFVHS*fA~qL--e+q2>Aly#2Adi84N?@Gn)iX76v86LG(Cg`u0pkENm}79NuLU;3|v z+b>9X<UXqzCwsrWj7qsyoQhmWVA>6;$zb{$2^G9gfLAPFRoOinBiv327e&x8<I$xW zvDhD(I^GqpPlDs^*V(HL{Sz_oyf4|x#fwc2GO-i=hlw<nQ3UJ)I0C1R5nJuU!MDf# zm#5oY&sc)5583few(WM2@b#m9fvmIDasX-g79h0kF#KGPzq{SPdt4r{&PT*Jq+Sk` z4A42C7<v15FoO16`M{OJSDqn0?z-Q6_80_~J;@>4t?6g@*A)9t7S4vX>0+#{qbUW0 zLy1fB)TlpME*w7=2pEtTB|67fwgoQ1h3739Qr+ePC1Bf>i=l@cc`p%5rw>JR5_qIK zMAu_sQ$x#td~}p%RtYq2ORO}z%Gh*yd>X!JY}ui@qfH_eMTVHhYvk9bYD;?4OBv4L zo~R=EGYi}T&IHMP^LnuHBSE=~^u%t2+3{c~*u#HwxN=XKNh0L<XRPLNq{<nEuK=>A z7oE~9k@nQbUu!^-CC@FvHGpuaO!}x7Giq+`ypx5-$Ca$SEX-l%m&2Jl9<hjjHlhe` zv};H_!v3j0-y=e8P7@FtmRQ4T(9^p!6Ao5Wvu3eeiUG?9*8PshAvKf{J0ssacg*Sn zg-@8`oS7Au6d|5Av8H<h=K&g0hlEE5dC8ev_vFwInjNYbsZ>F};y$jvgE_dYS+|G% zoE8~0a=9g{6N>!WR2|x&eM1Y?pE<ZAe%&h5#EwfYCglxpOT)NuTDV<QbDm-+^PF(X zFH>312NvB&&^#tOiMu^+n+$las-1^Dd*nswV^)ffjOnK*p9-7yZwnl>31DWV+@OB` z(11~PRj;bRful>c^z)OGkt^};>_RuEe&^~@563wpTqs7qG%M}pSX+;n6VZ#E&<%d> z(Km`DE;tPSc>HbBB8f;sM|)DCgEF`o0}rn4sgaZ*XlLv^Ysk1n%)F`yk45(I)6*3D z>v;dzjqGr6%X@$;=#2Zqd0jSbyFylT=j4d`)?~ZW2sUwY&(L}FX_V;jNDZT0w_;Q3 zRMJ9UrY2B1%-(^>u#||WO7u$m0JhlaUr=2MCl2DsJ@Yuj_bd~z1YbWb4{k0Y^BdFC zH`N26vk|_#u$IfMN?_CCakYvLILTxA{+haeR(|2%E8l;T^zq*ZVxf(Y&0Jx5sn>=X z6++=OBIinFeXO9J=34i%`i%>=NkmW_`4Ho4ARmhd>{L9&Tpr;~qmflwUgZwKduc#= zP@x|z5sa-RPX@VGdK0A>0!~b*sjZ2_^r?&HAV2P9wEix8+WK~0L6Y8m(kJJ)GqzNG z)MEpb9BTP{_}M|srwXh<LlpCSXATILgy_Jtw5vQ(B{YdsD;GL9Iki2KqQ1vc67LUr zZr*Ta|E@(=Xu@U=tPbtcmbgWa)8&`*TKIF#qUGs}ho*H+ZCup_)M*@n{JN(ry>s~& zn&_BLM+BOC9ztG89lL=j_mq*^%sZ5(RLI;R(;xcv(9C$F$K2^9$am1;^lLKgJ#PX^ zvtJn_lW5%kqVg3hSLHO-4~$-c+s=O7rBq=S(v@~;@YfIe*En=UFy>g!_F>RB!yXiZ zyQ3+<smYwQqJ|e)<M6(I668YgWjE9WS(jw-gs6AUei18x$6+Dt!0)Eb<e_a7wPdr5 zxNfYUo8x(4z(C4Axy#fhKSmjxcN#ifuE#J1qovnpM0E33r?or?(R~f|seHNj-H^}6 zt~=ac++i?uh`Cz*IISm=zOMLCb|@JVuUFyk`?)r%{!NQKldaxHW}(MpB+zsqq}2GP zAiS(&6_D+U8oa}gD8R3-9of{p0m8_5#zZ8OckHzLO(%XtS1}b0k}=<9BN}Doegrcl zOY=X!bi(L{(m8h7VD=)5!V7}Ufgl}=0u)~_>NnCLoZVT#{qFTR=YTvvcF;M@xYlXm zW)awv5b8dzG5gEUFw8uN;2MCS<Gq{mnwCkNwLS3;j=N~h0oOF|Ae48XFY0@ded>SN zsfCouVfKczD~8?6|JZvRx%#sLM<zbRfdZckv*r0R7cc9CJ?gtnhyl^~J)GjU6nrHV zj%wa9^K!VIf`_nIJ88_Y-8JGAE6bR5oEsz~;nfsH7Q!@HL$Ub!hU4Jye+B7xy;jvB zn4QfBV`lHnkB1}EOf$@vm9RUCZRS8rCI+N$DL7%eeDj=nH`iZ9*js5<h166oYwY|( znY2ci2jkV^EwoG=J?*v%t$isYKV65*N#@ry=%mhb;=4fXtHP#v0zzB8f9dOw?VgvM zT(L+2I^>RPD;!A<5+Aq6Nv~gQYB&yRN;|Y~pS(DTU4mN;wW8?67%S95%BGt9b|1ND zxTDlHokm0L`1E2j&iY=1%tE5J0fJbDl*)QC;fvQ8x5Q*MJEv<{eHAQqFGzNEu6zqr z&+m7H8by=rtxg1smpzd86ZxuxRiU{vhxw$2mv-#RWdVO~V#%tm9Q;iu3@D5_-y?n= zalUurr_OR^hN`DXFS?rgJfF`jg!eUy2-e{czvd?A`>?_&4KnU8M+kJYce8Y4h_q_d z9B-AUWyo2DbQ7Dp)6?l1J<C)tq0Hv#;>j{5Vtp^u3b*9Tn8AFJW3HJodRWj&xrCCQ z-!y?Ik37tp1ik5($``l2j%vs&6j7~0b#ki?vKcU|&<%g)gcA-yr3p&ujhC!=-krB} zTtT4A;PrJdtRqWpq6VUuWbCDFVHDP8ye-#|)on^7IN8xNy6}LhI@?eHZWE22DAG81 zRF9YG>s+@S`FkG>q1Byzk2Qas)`AIe?H4opI_|EA@5*37#FH#u)0XSN7&`swC+-s> z>Tml1GEqE`*#jZJ{y{4sS6$Ey=K=u7JNW|VCd@wlzYN{B031p=W-4l`m)iQWDHc`{ z#(+^j_Cg-HmNO%a?7efDV~&7P#f*{f=UD}LSOF;u1fDNc)x-RBLKi9#e;`kldki*P z2W5Ef(J3-+jBSza@&phsNuIY6vMZB<qQjy;&0!oT=sGIj6th5OV%M^{<<xG-%>7qz zk%p`9H(FG24F4Qp_A_o?LxWZ!0?36tj;I8F_XJpDFY>;SC=*KydQ<C+6hf@BY<YRT zR3TmIlpokBao214Q1F)K#tC4Q_ovl1JBo6s>Lpg?u4m)<kH($EI@7;b$l_R14@vZ% z8wSVG62tUH<EE9a-k0Zv2%iXzW~#M31wY;xTGuX}kU=s#Ty)JB%mr5kW>P6A-}#4Z zyqYL|RYXDVrfaqJbaE<g(bpOS|GvO68)jiw%eiyjafZczv;aVQKD^@Qe*v_<$vJW) za<{#A%?_BPXL3>{lu-#3C!=khRDBUL1zWBBwjUI~G=&Kv4Y87|4}{RK!}o0O03DvO zP#RXaLFi|2?zpm0c?r)S6jboTajASSyl&ZOs)S%{7~&csgmYIupKz!y0?|`iM+n1c zZOL|xk{urU(1j|W-z9maS5$e!<qT{g7_a^q&}S#iMhbF33yIRuAS(m~Z1O8*w$pn$ zh>>~}$QI+)?D<LNo$ZA4IEc&~aT$V=(Hw$dT%V9*(T95w(@%{eccCbwIg2tKI+ku6 zCYPI|`jM|mlLGMtp=taRF&Cj2P%Mgky2S3!q?oo$0L^AJU?jY&m>;?|I7h>i>ME;t zrV1e<)7aeoZSt&T(T;R$DNq%%-pmE#*}hxvo-saj{?iOkmo-NcE?H^=g|m}v&>szR z-g5u>ZBAvGT{|tltcwAgaok0qsayXx9I(-J1t@>6-8OTgO0Eo}8OUVLm)ua_y<ax8 zZ05%FRb}1myr<*5`uhz2W&EIu1CGEXU?p=qwqid1I+F>y4N3idkYIm+b-vo)L}bZa zmRrleWF?5%18>tuJSIHQKJUjOe}!R(qYI%nF_WLN>K6v-5W2_@9Gs8*I>Kl1!O$g? z?Eva$Ue!@Ff~KXmS|yNOkVRG)6krjTrAj@-E!Rl5WR(_PCPY3cPZ9DNB}WuOH(~nG zwEuis#?2P%d#lLC!z*b>YENrHLT>|0g^f?CSAv8G8>V^qU9NwW_hQ@h{a3zpmq%sk zz)|PeY*p2q)nTPWy_QTmWyk!!?CiiMz<WX|;(2r8W4a+r@Mgc;1n(cC93z=RZC@`; z4r3SvD_?wVVselrajDsOCVs@g6-J!60c;{b>C~PP60xtW(3=#@<p3>f+Njg$i+kpH zQ(Jew?*?_!xNgBtOs&Hjy9Pd115rX%&=Z-S*I$eAl+f8vNu*xLSJ4L)VZ`|wlM=W^ z$AMu5ZPsL<HN<0V96WDha^5)sy-)c!Czwz@PZ}h;yGV4&Km<|yE)^GCim~7kMBnzU zvz{`<IEI4<*ElhdNJhC45{Q0-htz-+;`-IB$YIjLTgqpgKgidmcE&PBlnrVSCAu|p zm+fNg6=#}78%RXRRRONPq#I-VT*vd-rJDn68=3p|KOZr%lunLa9GFyHy?h#!+P%Gt zXM$z+-cX_pB+@?J-8Y>E@9tR@g^<N$yKVwCN8EFFO(H~ch?%VhDsQINmw0@`y!3^D zo&}8QtF~lW9&M4%&~Y1DM|QmMSnDOI{w$*+BRG@wXa&7zSM(kHxPI;PhfhUWL4Lsb zYBhoT7oz;QDAR8%<4PZDq?i5isFN8j=qXF&x(X<I^@e~2x7t+enSZPP?eN`&ERoO) z^tXSV1p^gv8UEi?z4XrLK8ZP9-p??fk58lGYN-Qu-iPFZn4+>CEzzA(e8-x88@g;u ztpG8u=D<u;W)&lhU-X<J4uORB=7H!378ztIr5q~0>kq((OpllX-eGiz1(caq2?rGs zAe5(Eup5f#efa29c8gYxuH$R``Ra5U`RhNTnqM9FY^{Pxaw{}WtI7h^g3kv=RUYgg zh2N8sIoZ2EqR61UCY^RxAsd7K6sTCaC{g1*KY3K+Ie!q?c|7o&G;aAPW|&7Ot6@T1 zf1ReL^Vt`b@H8+v4RN`AFNI(lI%Y-5AzUUGEyqHHGvl|LM;((x_9MapYOBddm)AX4 zgak2BB#6=M_HRp)viV<Gin@Y)AL@V90vR@n9ku-Pt)~Hp(u5Fm)Rv^?zGVc0jttt~ z2|{2rSr^-?VsI0No$@l~4dPdJoFQlxYA4lGns_;&3S}6pJ`)C8j+$5W)BK^o;i@a3 zJYA7mTq^q|E5=&f=JsI`cc^^Xfp|xBv)E^gI-_I@$LG4mA012S`KGX1RY*1igd61i z$s$aSztb>7FZ9w_Fbk>N;NZ*ki)e*j<j1Z6lJTWpGipnM2qkh|EVij5;+udci=+_) z7_S_#?ov0h@6*1ir4Db#VtQ~_%X=}&n|B-}Xv1+*HpnoJnp<2d_21jx9ypU2PqEKA z^d~u4uUGEd9<L49V|?e#WUVbpoSOnM5-L0m{gUMeDXTF*M8z#Eb2kSHV643l>^9+T z#)_URi6lNY0Q~Plz4?xiqJ&cCOW|+tZq5W4L>;~o`wWUqeQ7>hFqS?C>W94}RPZgz zeJ^+<#!Oe{awGwKHC^Xwbup3<qzhb>XgPwkq9JdmOuhLVcPce}Hx*e*nDyN`Zfd8T z8h61?Ef*VqcOQ0X8`A&ta^pX9xVFDto}8-7G~KPsNLkPeHiHnuYE~<%{N+`96f@4E z*!5eS=pWx1StGOTEZ-r$R+S~&h^(HoHmIH&(Q{Zx4s>BE#2tjv%B{OX;7lF~x=r2Z zALODDcu=ujLyeJBhGIE-rdcBGBgWOihYMbx_xZOYky@pAiY#dmssP4QbB`OPBj&BO zkp%l4oL-J?89x`y>3y79+RhxJxuElfhM=aa?r1oJ>9c@a@Oc1sfQk4?IHvXalN8<i zhDn7-nKhUIM!;{&s_!kTxq<431q8aM5p^FUa8$jH0zr$RRa^<<5>wV?E(|nSHu=3l z;b+y#nD~AedML-X#snUg8yyMqLN1y!`D2Sc`m_2zipPTF#_x(vbp9OPPbAmt^Drb} zRf+I0QFypvSoD8?rtfebfWD#DnFl@|e4*i90Wr|U3Fgq$MTqK%^wLCg>pkjr#ZZ8E z^jKiw<t#sSd=55q9VeU~xd^{uRLDcyEj_7g#Tjv(rr#?#fOdOB?Z488Xkl^`$cRjQ zajj52v**h->gg8p^?aMKzxf75UX9!5eDU%K{>8)(QmCI}eDA4;34wJP##*!Wbhs4c zBKDGWN)pF#`Kjjh#N`yQ%-E_H`I*tfxHWc^LzRk%?|!Lqhg`6HZ&47=>?h%V2Pq}4 zUBM;m2L)?fedyl5&NPDAt*etD=KGSRH`5{7RnHrPr12i81|?3KIWK;`lpnMt3A!AX zF@ar-N4<0I4^BltK^@)t0>j~+*1syu{@0mDUp~l}>X$!0*r)D)ekU5bn;rH<SEq35 zFX!29NF4PAm{`CZ0E3wYa&3`|L`6~_;70jzACcKX7~4cr@L`Pk7l2<`QDqfMlHl`; z<Iz~Z|LcFRCV+y&A_gcZ@DxMC2oTh|xe%}{x}ygnj%fnM$3^;usM*ru_$Y^zZ|334 zy^zoT?wvW(iKU>zD&atMTp?$Yw`|ovZQ`%BGg5~mYtWO>kt7wr?7uCQD-#XNQ^j#5 zeWEaD{8&Klch7;suLUnDFX^Bj8CzoeQ-*I>?7mSfNQ=0@hBJey$5TM*oF>V0z4cpK z*L%2~;M4TF?^u=j^CigTTu4y$rGTG{yizNLYyasOvsakkh$7!3aecoH%P3}5J>Mw( zt<&iy;#mahI%%Q0uq<#sB{f+JAuwS>MWta04$<bPjf&6Ny;<bOkpbA+O6!)$DC(ni ziP#7SacdGQ!&3$SpWmkvnpgzB4z+JZFXUa)PiXNZrE)GmS8W0x5e*q)%zS=uyzsx$ z^h*qq5aPTkxhI6}hH`QQwxgH>KO(8+BE3ZL@0W0os`t`&J=7*g+F95;;eBt^HBAnm zfD@)tzL+lc`{+K=X8nqsq<aM<F+jpc2XV*_)P{O5+7&w28QrLYq?#9rSRmQv4JkfN z#sMm;Rp1<|4c*cwCgJ*atvYanpWn)5<A%9=B@CmU;*jVimlta?R(GGigN8h6?=gTk zJags=dPR$@!TAt24-HT|#sWtc9gZ?Ybrk)0YX7ol7h_G>#?4kQ{h1$ipNQyzbo7wg zy<ivF_ffvgqbKsmm^CU{C!~Ki*O!pzlYQ-@Vx#i3RM~*xXx}W-i;BLRciBVJ_$su< z#{kC^FAeKazRyOmmig~HnW7-WKjvJ~QaAW@H|SZeB>|HtmLQ)CfRF0*Wa}^?_*yFa zyn5^<A8-?RE<bBOaNmDD_UVvd=paZzk;Ou*`pqGq`9+`G&FPeuLj~Adl;G2|qJLQ~ z|C4|FO@I}1IB6ypXNJe<S_13DRu2_67Csu*c%tXS6Jq<`f}I*)CB*qh1+`gjOAfLf zMML)@EJ3<PmzSbMq1U)s^ojF0!DaM0D`vLxvL)$)-!-l~N-*m!$@66^iI$#Q?%u6X z8zD3HmefgsfOjLTgib0L3JsBumYl|IbQ>vdu?8!)8XuzD<5hv6F}#*};??F%u0nG+ z;%nt!dPCtOn)b|h&Tnb^(}5dRW{v&uO_8*4NH^1<^g+ERsmEdnU8UJaMJsNl$O)-} z;W9<kjXU*M?;nW6;YaJY={8Q43PmP!ue2XUY4gAQ!pov^nP-*ob&*?}VTT^S`X#Gl z1l)CK(q~D+-!m2I8?r$GbbUlHxL&%EEQVkjY#obKEL(2D<bytDFW<<vIO=Z|97|}6 zVg2*FzHY=Yx&2s&?2ph|H-}Q`+cFysl3X|l0Vm9~#p?K>lT{7H#(Bdcf*8SSJl&a4 zu%N@kzw9_msd15wQXMge+n2&UJD0t8Xm>D7{gGHdW0#%|VOcdiE25Ws8TdM;h-S&* zeIkUl1pB8t$j&*FF{B#&vY;g?+x+15Fm4O;CQ>1Lq|yuGo?A1ymy`T@nfW&wd0ECp zuN~vOrD)<UsD=G`xoV-w_@^*3Z^T@pE)*QNHzQ=Ie>Q|S4AJ}cP&5C01g77zk$+)Z zLTd%<FzIA-<B9TJUPa1)j5<T_p9q5NzK>k}@Up0)xn!;|*SNUNysGIA^CBD3{kYdv zDeO$jdbhu-#cR{-zWXM?L?gX0TaHXAauLPovlNQccG}d<>jHRne94QKupj*+)Wi6C z;S}awj5qz$QgOz)^=4z9Z2P7_An*_N`Fok>DGC+FSHRWbvP!^F!KaZMG6myLH4nPT zm%ioinl}{v?|QqRe^c>yE%SFyVA+iWp*9{DyX+KvY`S7OI*j4m1Ikkks*X5t1-3Dj zpti`DA2c-hk0tggWTJnWQRT*f1)fK_ND4zpXVteSR}?pG(zb1Xwc`llk%4+eBKD!k zxF47CUIy2XXJC%9Z|2ojjoX)PZ=*MwKkY&*6v@7a)EqQV29G<Pgp1sAECmQVuLR_< z2m}nemwV1DFG1{EC-wv84R#CD#Wt9=Ze!L0=!2QAc&bALzn>T$2zft$hO+40=~v^( zF(9q<!_cU<i;`9km3c%9$B=&|K-}N6PJR@Bm5eEB1pDNdfN3=nWK?iprYfvv&e?4K z;hp~#{i|mBYPy2c>>zHRbkI>;68@0->lCFgXN7-LfuI@2<(0LjnUlOD{q_S8?d&Ot z9Z*DsQbmlNC|W*poG_9oryn4y8RKoDjCe8o#uG^+I0$#rgC{$b>1h_6*$by$qEF;e zif|M)zrW~VN!m2+zDn4E&*JB9fJXZfMIR7*cC=MB-kP@6R~=loGkZzg?btqf?*6m? z-aaBz{{;+V0eU(-#RTm@Vi_1O3BaM==OMT<g+ck*(i5b6f4aEg_jhtb0{B{fL^8hX z#v-zpNj*s|F*R;8->CwNq<6m~icPL|W@<x>VgyAVNCcV2uvM3*Se4;PoJ<N19*)1< za_0w`NIt^4bnKGIy0<4nSDSL$f1^AV>NT%^Rf+v&YVtk!c(S3)ZYX%ls~7T;m$r!u z!+Nl4PQ9TBj~(j;p?+BJYAzr~hH*Ow9j9|KWxDn8tVi5?sqK9m&T^eCwQV?@=6zR? zYG%{|7$c<uMj)p1`n%o0)sZ?Amv)?W{h)PylR%8N007(l5*f5|FN<3sV42AD#yanx zv+rN#6BY%0z=D+(^PELvIl;`}#%CyIkSY>sm2jVxqK)#rLI1u0=8-mjAEtRIC&f~V zK52D_@i^DWRu_i_EShFP{l}{VPi*Y_p_|GW4rU>wR@}K*XOi8Yaj**{3`$*;9f}}y zEj6&a=>3Azm@LC;1{|%<=}gA8(0!E+nc;J(J<A0;A}|Sig~sYIQRvZm?JrVUj}FSa z5l>n}$&m{AGwuM**-l{AQJpZ5h1~yrf21kOYIJVZ^x34>Xl4Q8y8_WYG4x@AK!Wee zJ)0^)#@M7(a}2ZOxAB=AC09c;0_BgAN*Y`Y*}ywom~2{;i!=;k5j&x2f98}y`5bI( zpMViXW<U_Q`Ntv%aoaWjRIi5PhBaa@?!zm;gac_?sWzLz(;cmQ8nCxv(;7o>?OcFE z?2#3YSjb%IuZN)#M!vQani07_-|(YH!qSXDIk=C%ED6%l=9Dn&PKPD%NOgs($>0W* zLvMNIZKWl1f{Ry5sB6}P*7aV9PZwf%=V;l>meZGn9&;x~>c#rb9oYYZe}r2k(3BJ{ zaoU?;VTjYk=*c#>B$d$T1^YY%UuTpP4APfE-!ezRg$mH01rX-d=4u)wKe+X#l6SHv z3`Bww--P!#^PSl_Os&0S&iblFuDyQ|8;&wMpuJi@wuQoxeoNGS70yJ$F9SK(-JEEL zAMzihVXY=HHW%1e!H>N8m_;t);x#=h!K51qu|3Iy;N8phDCLmWN68Du{-y5%RrLxA ze4OnTjwJ8W{g`75eyj4GWnzD-{k|{zSHUBDnJ1e@d>%>`kj?13_Gzmf;I7xWO<P6{ zHnLobU#ume0$c1SN{+^hWZ+V_e;-}I8b?1_@OkGEAfqUn^V^2L1)xzAuI4ve`l)9C zLDs%;GY;^>3GxNX=cgC=8yCo_mdOb%QT6|7_}1_@L9>Un^b!Yi&jStPJB+m%*$j0M zl&1G%TaTasp(R^%;WuY0shBp&aT$WS8ae4|Lbd_5Qb{;lZWsPJ-9}81Zky`8F>&la z-@LKe8E{xS=bkxbd`s}!`@DbfRAa~GoUZjOB$oLmkM+$1Z>50YV<IT~)(GikRdv$J z;)-|mV&0-cdk^?3FG@57_fwO~tp7FF1%eXmlOk7ceav3tq~~x?%_Wo7vnO}24xUV} zz!%e>D*s~I@05rFhR#xZ$wFG@9TH0fTHLAMecu_nZ+u(tHg^85q(t@xAw>yW^WSuD zF%U}bOkUXP;y%LPOhPYL6ydUk$7SB;*SFg+_xeT1xZi#k0$suehf8R29OP{x67z0A zzjpqK^RPtum=Mj0j~jfCem@Lp>R`+q(*!=+=g<%wxfD5tuO?*diWndTELMWYn;Ipg zF=y%_7EnSi>N+p7s6?@GF1<&EH%I7Z-drl%&Wnyain>Y6M1(Syy_)=FN%qGY+tP~3 zHg}TinZ_PWw%6x$p;E<ZPqx@X-|O-bqQlIf=a8=Q{MIRFIstrrt1?x&ig&we6$>R` zV+{l55ZcR?iQwW5M%(=3<rX{WeaF~P>E=GDP5@%K`hc@~WUXug_f@Gv=}35Il~@X= zlOSmYrEvMt%d70<R)}8rVNeVwKM0Py-YLd-zP}F1v=J98#fhqa=Mg-P9_eF9$rzOo zRH$-3BYpQnj|GM2QoT~`=@JbVi2Y+E0vV9i8Q=<;>zBE3R}01dL@V=n0T2Lb@nUs) z0^S=bD4DbREqP}^)^>E1+G^q{EB9N4(e>9_AbpVILQ{Q)`;hqC`3BF4mZpgqSwOxv z1Re<LH(t;Qx;qOtf9t&P<-Zxh<zu>2Pj7&p++gLMKe?kkA`c9u4Y2}IAEZG)cU!XZ zf1!F0;tALeO|WEtwK+z(@3Q#%fTm^A9`8n*oSTrvzxn_(A70X^PG|S>R72}j{X||k zsOdoPyKF}fvL4vgwFQWrPg4$m+Je7IoWL5^*=@q3JviN#j<)!6W0BBhW82DgrX0xG z1?#!p%pno>?wHDitH2Mcqg;6+><`)E#7lz$h7Z;P8y1auGZR2r?F;w;Qn`V`-ahWQ z3C0=&<CB@|zTvX&2ATEjzBU@O;(Ce0GK~S7<+mkmRiY+_&mW)@2!jv^BpibB7Ndkk zEMSGKm${C}{FGMcpo%{Y>9m~=;jMA!T{hyw)>tN=_=r|3CGjsYR7}5W6?(l(>#<;Y zI=AnQG^a&;ZuMD2wkYde#)Chkctj2pAASWL#YPjbZl}CjNdEh2x7BEeVD(nDs?gw; zheo(S$ZW^t)cEWAFrjLK>>YU)TzKiiA)G@@ONt}|xM>F$+4(f*b(0ANWOhLa=Tcrl z&tMXnZhv8F*<~{_K{d-)#aqSLs5g&vWYyfEdSc1f`5(;fnvGU!CNev-T*`6tp?dO@ zWrp@-IPw)?k7F0HzCl2Cr3%jmJSoo@YQS7m18|zC0495kH}ouGC^JTl?MQvrNrk&< zTAWtb03-R#GyQ%o>Fh5(<|LbCfuy_sQ3R@O!TbWYC5vvx_IfjRw@$mII%NiO)5sh$ zp93X<cw;%(7iY*J_quwM_u2P%*W(}G!W12YQ4rE8sAu_Va*Fq&>$@9cwD{cE=A6+s z_MR4~9!`H`BaK6;w8yh{CytXS(6E_%v|}S{JU;`UGBb_`RBanK2Qmme+4J_JO%>UO zSpi+$$i>diy9+of(?3NZ_@e*q)o$%RTG6>~tE9A#fGb1r0{=F+kEiG>5;geW4dn#$ z0QHcyVD@R)#2atQKttkNI?UkfFbK_gbekMSVvq5A0mArKE(jpPZSuT8uE2G|7t|*F z1EQF3;%t}SQ}9UPyTdO(#S>i`3+G)XL?pmv!Vmuh9Qu=~(u5A@$z&ggl=B})UFA)b z<{iLjv)8t#AW^kU@hlj0^|vY0G4U~I|0SCeY1wLwpm*Y`?4QpN@x2Q;fj;mP=F2ws zgtH|qiI(Tb6iZKsB@db(1{Lqsc`G4}mmsmgjwGQvxx045Y1@ondd~066gNq}%c>3} z7!E0BIeEGpjCT+osRX(oYodxjLo}nDV@b-Uh3@624ktkrZ7S~I1#Rxepec$625wLj z3|#7L*5|~QZ}<=R{_ISc+H+mNYY}2%dlSZCz-F<W3Nhy=-fnxER)egH0G^&NfjFP$ z-*o-9JcLFL)^5?<m&cl2YtK<#Ajlq6j&c6rT)!mt1L;NxPrwkL*UFjESCY%6gY37Y zCXx};h<E%MdRT%Y7EE|6!mRWPUeYe#1Yq>!#5b>49dk8sUan?8s(r5?u}-gE@sXUN z{3B9>lxDheLZ8HX>KA3XQLMH=H5_wu(!Qbj{Bz01WJ!%tjwO#bLS+7!>Ag=en5rcb zuHE=!|4$Aceb$JKcw2nM@D%po2c4kLZz^mw$Uu9$1Zlh-aKd$y*38xia?GpL!IP2? zuzIo~-3=QT22#Q36$9h9P4FT&(07!jlyEjmc2}lK8_&`9>=}>fBRpAzO}3O&MeXPj z>N6$*dqP5tR?tz>dcy5y0w+^vP6p8IX-tF%0JFA7<-s@GrDyw9!!qlAeC5LOo!*Oq zXKT(3qwB?C*MOw6BqQkZPtx9y*lKKH^VSDc+<yW$L=i(uX>P=$J?)WldRbEQawJ2U z=F#mx<#hw$-`;cH|0wbgulAMqst43aJg?}MuV+ep>_7E)2vA~IKIQazXmhFY?Kg{N zf|MKXk6-<m#|Zmqa3U9Ghf<pmX|N&a(Ox)yG%q|(E}N!xSxRa`QYECEVu{J|tlR9D zji6VxG%`VAlyJc4&Yk68U!<O$F!6{Gmf}hE{vhREz>YMiJrCpbo3}|c^i{>Gcy%c1 zvDQMk3?}*dFKlOG6<+)?&z9q%r=kWZt=-8DhRn03rKh_+L}V&UjxsH$-yW`<QEW#h zkS!Wt@6T|Zn^BrPdv-gx_(jCoe>PwLe4SaY5}&p6|D)^8qoEG}zyD!uQI@hL3~3SB z%93TQSxP0cjy+}H6~<r&iLzHH`<ituWynqnk)5Hj@4^@iW-P<+t<U%KJ-_>$`~ItQ zI>$eb>wUeh=kj<t=<a7hdaCQ=NxzH4gZ?}pNzS+?8~r+7+mI=vGyx=iv$dEXW#oOH zUI!{5f^TE`nd3HrOgJV_D6!re{Zn$N1BnH7<ckbgq5tyeht8i7aG+fXpSb;n@b7n3 z%Ge)Nv^OyYK7--#%toFwGz6yvj0o*d$or3!6zeRTg*_6Jyp*ZU#Pi`uYg|a_`fd@G zpwr1PJ#^YYpp1SKqxy&6KVr{oGv0(f{4gL<>PDaz@P@S)%YYe$$!synRtu=QL53a9 ze;gyG)6D9h>wl#C+_|dAq-caX-hlO;v4bZ0E0<p&e)z~?fKT+ZU2i<rS}Yh?+G51K zfkry@+~2?3rDu?~1M1-q{5>i<niTdEZMq2G0Avc|X7|Nv<E8?R2uv1q=X%KR&Dv|o zk-gU$*gGuvphlTU3aZD@uV~u~-%ro0QEy*}#|2rvg-^x3#1p^0A|U$Coe?n4<RKoX zoc<X{#WYjKo5ehW*35w?p)by9@!Ndt6Te{KtCwh!Yr>}g-z-4ms?o?{_Q7V;;YP^@ zi-*#3norl^6sZZ0w>p~RN<N*fKOcjN+b>i1a~^bx{tL<{mbIv>7BCGgtB2(?*%I%^ zq0^B6^+x^$<yxwZ1Z+3mN=PdM+bvN8Z^ciJlg3l5-HB$$Y<F(A(Zr``d`6beSWb9R zcD=~l3YeA%&t@+E@M<>DYZEsN$kd{NA(F>Q7a!?3i=KMZf6I|xAcZ5;ylZod0mOR5 zg^w_f3qK~48ZD$41%7(nT#Gbmu2x4`p}tfH&-yS$Um9`$LB%liD$a?ydgb*&TxSq4 zG?O4q4fp0d*4i9tZ38x6%Mb?PZKFxGahn%M`3|buSt4r0d}fF>%3ts9<jPMMLt_+= zkDh)yVIbe}c04;kkkaOrQWPx=Ju5LZ19>oec%%*U0B4@~1oX2W8Pa;|a#&H7MOz4E z>*--^8KdVc^7@o~c&R9Z%1Xps-?&$u^1MvbqJn?S-aE5;S3$T@2mRE@xc<~$7vf}- z$bd}qOPeK{cp%rASM2)!-=v{_qaPfP5y++ytSOe>TM}w5g>f?;?(OJ@#Z$f7Zs}mq zAH1Jp12N7)Q~9zRFR&)jnBLDy%4?SP?ddQUU_QX&u-hsadaB0(NETt2KM#@Eu<^@} z(+$J1j9GJOwS1Du(x))6qD%y}s>;f3C%0XYgX4Hu8U}Pj1qgtz?$>c@CroI}nw_y) z3>vzVqnwmw1lz5ZsYyB^E3{37!W>z+3uCf-_Lcg!8UHAeoR*^~&RN#0NdBlD)-w-z zxBW+rg)$a~5sP7Ln?Br-)>SMltf^E=Z~0+%kVR&L3Ul(4@yhRVRU;cU89~Od8tDz2 z^qG<Fp#}*BNRL%2$eZAF6X~uG3XWj1Ip6SAzka6W(YKjRa>FxP;k$uHTOG-MO~iEP zqNe>GkEG;=g{yOv>uH|q@{1py3<nKdf&iaY?@qyp4?wla=h2~2iE*L_@^E}nkOdxD zXN@YB04(rd6yz2cQa@uW+M#fK=|A7=n=-V*kE5)gM9PJX=PwOZ$^GBEA*L4@XAm2_ zzdQPfQ1Z_U^k)PVgIcZG+uR>1Y<GU)R0D9KfU{1=FlfQX5U(z6%plCz(*L%Z)A60l zd8#j$-HQ0w7~&54s6%EyGQ@l*aWaU&8qY5gVxWxW@RA4vGETv7wVpvoCkWIuKkb4g z6pn>d-aL*t^x8=J)+P@RvNmrf*Is3b`2uO*TM&q&v$8RMdo{o_L{qNLSI~YF0c1x> zsLoj~r4!RY9sHglYb^hSs$EMCe5Zn@$rs8o4PU=I^E|50@d!bBu-@k52;x(rh72{h zSbm*AW;FC1(R0M#@Js`Ju^)1uZe!swpz7?u8euwn+augW+Uv@WQ@F|BG{<rMkqB+o zW}zK5w6;-3`?0WUY^K>9&|TB_{CE)GJ3{Rd*N+esTxhpgse5g$(@J$t6ra=FW?IBk zDxJ9;2)I=?TO$2kzpPl#*++2*do95}o*~2;OC<_iL5uOerZE>*J^G<tP5GrJLet`G zgQhVRJTe}N35|Aqc<$9*WY*-1V1wr-jNMz+DLS#J`gbS#%^BDFjkgs?;?={^wf7$L z(IU+ic2r+l@l}s-#c6`#vY{3ee!L*Jp13lJ8`UzKQaCX*mviPd`x*X?ON<rHph4q} zii5J6DR{8D^$c%7|1+jn%K_EQH^|u67grz>jH%lq6Yn5*UMZPrT(aM&0<~b$OiHxG z|9n@2MfF|n=W6e7XN~i#dyU}ugxK^#2gvlsE%QWe6z1lozhI!Y{%X)$u@Lx+z#W$a z%~WymXJL)cFk5l(_XNi@EAc)^%_b^GQJYT6sy&NxHYYR!)T=I_0xP>e1IW%!7gm;p z`m2u|PL&NN=!+KT$xjX$X5tN--{C!F&Bq7ZDw`fTZy&`$_r|taPBvNIE0+IHoLLyQ zDE4n0xfx75<I(2NtsVL^YTDY#_W$^9|Np9eKR8f+{A>ojVcPQn0RcLoKe@0QBPt&) zd;#qMbzP~|I-GTV3-p40ZkW1H(wJR|XneF-rwy9ZF?ZI!%Onx4(?Nsc+No;M0lvc8 zsxR1ayCBb%W(J*3TRT18yQ<#`S^n}=EinT&;iyL*;{gMi6JM#CHD*r}=sIHArI<0H za+w65YqW2s|FP4Kls?&<u#4d%;NIY3aN37&zK>7s>#i{?e{Fws#bYxF+*Gq(@7cT@ z|5ks^%HKD4Bw!X|@_2;GYAH;V>h^S4FoWR-foDuJH$FW$+)@3}?~vy$6>h|qwM$(Z z7#wv@o$gS}Fe9U9)TGyNpSHR6WcJGsFe%l9lyc%##-Cd1ligIndMhyaq({~g=4JSa zsW0)2K=}iP@fy)*CI1GGcF`891-ns8<E+*74MjIvJ~7qdNasl24HqNw2li}L?%ptC zM7XTjo%<=^ssY2#AByi8Q0|=lX=}h8VN&P{w+sy`A~^db7H{3}uc5shR_Tm0J1qLC z%5*kSRkc`djGl6t7dB+JEx4=3Bt8;ae_ptQGC{5HV)VcX0`0jP$){$yVts*dho6LC zX<77HG>RBtex))S?3UT!jlUvZql)2~&UWLWcv7AQ$BSGW#Y#jVXHg)Q=q?|Sb9nXc zc1@K%1~ybX5bXKD-5fowvoY%u+ZUnUc^-uejq{q$3L2C*ow0L5Z#MSiB18$!bLoVb zerpN)m5)0WY_E!?_N3+WjAha~UOk6y!$*(ki9kZ)$mjXqHzO*s$TW$oJuxYWjoJbw zx9(|2P%o{H2|_J=4$z=Y8-geYyNlT@L50;NaFS)<*3VO$@FQveG?KqJ>1X}2xewm! z!#GAg`cQ7tkbk#OyXb#E)tt1flI@M|uof@lOR$@43BET!P+y#9%6HvM{f{(KA;vM2 zC{(jJCXXF7I3+2W!srSxMJ<d*Hn3*!W*77v;>L%wz<Ne}!_*K)7i3m2lveL=Bz@4P zs7$<w*fn4zm<bAVPng4BUh!Y5ycC`*z+o6&t@<sYde|$0F@ra2=zF9EawnJbM~-Y5 zvoVSh_=GpI)jdH0*1f&!fU^maZ^!nH<_5Sgh5Mo+=lEY-L^&F#Nu|CDhS5&miqPMb z9(#I`%Z=nXLu;Sk*nV5NhHnWk-vByz^F!Q?@J1yAc~hD9%ya;=GIxSpBgcG^dpr?J zi5~2myUlZw0A>PYzP|L$M&IfDd4zW_!=dO}lJR*IS78NCgSs=8o?>wAU>34v$#eP2 zYWSn8U*!v3it?f;nj&1ftlL+vyHUA{fkZPQJ3%LMfhh64MOq&h!CTY^OLbvdsf3JA z1)37~5OEta5Cd-i0?Ny%*LyP5Z>NJ_@~UtA0P^ELEc!l+vX(CvEdS@J!$^B4(5KS{ z+xtDBe*%2hYvlp>w*C{RLs)n_O!JA=?t7Dv4_PV-vTk}MJ<cLdgVp`l!!7fr9$S|_ zW%LPWviZgDs(DyK<>#qap2^u+9Ma+JYYLnR+xwK)*2MdR2G8}Kn#3}FP!&rWjNh>@ zM!t94ndW3KpbBuAxw7N2l}=Nk4#B?c23&R@`tV98jF$yu6M1Ghc}}Z~@4d`Rf-yra zhfvDP-VQ$fS}r?AH7B=?{H-p`&|b*F;QcPx^vJesyTFnvIb%uQN;AV)E&|LBBi)ye zSiVHg*wav(=2V@4wJE)Ju7$m)zp_%FT;eh=&&T5S9f1gT#q;s_UvzC+4u+TKMlkBk z$ZaGvVl2&vFi^~AJF)3&D2u4lqrg1sO~j<J%rkB#=B)>;aIM{*RJr8q(7*RiHS70; z*~KB~OwhTXFM+62^2r%wrac$vJBU%c>@*6}uxeDEGWzq>%VAbn5ilC{oAA<xFWt_R zuk{-#Hd;(FcR!;xP^?D${O_YgCoG9xZ6}9{A0gW$kjDI#)8)w>P}lP9PP|`FvL8KJ zrvCrYel`9=&Hm_fguo0zP0JaK1fxVxm?aHHJ~f?J^>u`S<MmEwoV~pF_NsTf7ilAH z&jMQ3I6TV;$KSN4ey2Yv^zkOW2-B<X@yj$$RBy-@m)*o*!!0Iacklo6V%$jTFC;n9 z-_9w2jg>HRAL&gmLo>XBo>!(zE#31ON3S{3G&sNc1A9a?R*M_!qtYtzl_r@=n?`R+ z{hoY(xMERS+g1XJ%NG8|d}y-J9};sZh#rzPiKSc^59w2yxxLzPSos6Vt{l65X(G5A z*H$g<j+QrBZA+<d?6bJzYuMagmoGtRRY_yb2+y9qn%dyX_#1w4`=Lu0?)}LREY(C0 z)kD!2A~APzV4m#5i7h59Eh0T}#{7Hg=&^_zC?-myQ*lG?$*(aVP0{1~!Ne6SVsk<H zFJcQPh_5MDbt_9$y;FbMXvURMpnP~jhouCVcl?t)s?GQrXQnbFK1r}3#UspseQ)Ig zUAC3jhN~5f8-$R@V-8HRgR1827C)O{BdqeJG<f+Hksgy2mhw4V%5C%p14+MSggP$A z?VT(x_zs9Ms9NNlZh)hJwPQiZdf~M5OgYWu$U#!^-Jy@4g9Jfuji6Ue*_O*`voLG< zbmJ~+s*L5%n5C^JU>QVhY)4a`zNq^hvTn)kT=B-P8ELxwvGSjV3rMEGp3#hJ?M0j^ z4aP&(T_(d3Jmz_;ZByjGMlFefV<mU3y6w<y;C|+h^mvG^8O;Z$d=Hj8zuks*LYndp zOJz-ncV_6_bt?0IL0&Xce~WqS$%35@>evgph}Ph=c`%$s3hY-@MNzM#mpORW?$iKf z#6M7^vH_rv?>c{PPLyaq+P)VC!09N9g}Jc?a<3UXAD{VTCE?%7)CJTBN;WhC?Q0j4 zyvef>eTM_ITo#X+<ri3urmNm=pZf7wwE6#*U?k50vkv!o&UY#GcJgKM--#x42U90C zO@XA4-9@?J=jCTmX(s=#0Vt1pT%C$cVfg+lx8+%Be46AoH_J8DdVj)k$OqZx`6P4u z;(k~e!|ypaT`F!Ce@-g;T3PnATn;Z81p@}GxmvQbZTP1K)@SU2`S?Z<A2~$bmD5of zR%;p>CkEnckIxH}&8P)_ab;$0Y^<>zu^i8$OIw9O+L$fetQ)<ZHfBD^(az&aAnz7c zoKaQxyXMPq77C#UGB(9jHUw6**?^t2eiLF=ZrMV+B#pbd^}W=H{BsMr5{B~#uMs-b z8YT=QH!&}#0;gLDVif<(TF2h#($wC%#YCvBsgl?#u)VeWNs-K<Lajq@cYKG+*HY-Q z4aDMxTBrWEcY9}K%|-ss@p7G4kL~rqC1$`&qbukSqk+`51Gll40wePfxwWZOIgbZE zmk=Apdh7QHSlS2u%i_FhhB?*6ye_avbgITw(=(2l2!^Gv1Mglgez}#_e)T8;Y;|y# z%pm&ISwNOmvl#a_sI?D4*%1&@cun(1-;+bs5!-U@jPZvS6J`}G=pnPs-qKsOD<P@4 zY9&@gmYb+&zDODDV;Oz$_LyU<uUN&$al1Fk{6D&uL6s8wA#EvB#UsO9gJWc$hc|WB z9+|5p;neKdl*^r3<_qaU_m))01-ZLgr&IcB#Z$N!ygGwSN)~W1IfDjgZ$82hIQKsN zP5;2o*@q8*n%xL?0R;i45Ay_UD(~Oj?D13h;36U9<Is|V1=snk@vmwymz~=^u=2$~ z<%oNw9%BK1Ojp|L$=Mgqb3a-h3yZ6qhGlH<2|p$ZT3cg-@4PzOr#c$As_}ky^I{=p ziRe(&0)(UQZj_J#E>ly(--jM46AYdbs!h1Ha~}R0`-jaZhw?S2j;Oeag;#hr(^k`| zFMq1zq>{b@1%um*PF*A_t3%vq(rF*}y)r10PEw)#->085m70NnNJLf0b>grlICp)o z|5MjbdVc?zJY8lU-Fbn)vu>G9L&ri7egOv$_z6=iqbqq)?>aH8;nV+|h?Y4M(6;Z= z`cFbZl_c;6t`?ERH(*TL3v6y3p?@K+Mz0&WyVTRf^YPpV-J%g{_~BURNNb4dxM`Ay z=Ge*i)d#gRJ+?&-GlQJOB+c@~Vj&@*Y!<!U-h@r5F`I$tg8}oGip7WOEggknoGl*& zZs{(%@v<C*bW>NfQrHE45)zTZ>CtkGQ!!=M&Jp?Lk(9tmv-bQg5424^Fp4T5BQLVB zUhF~GgvlYZSaK@E663^<Aqfa!X>^aAp@d^t#B)=u(rC|Z<c3_|)NFY9)n;>|y|l;X zgj3IC2BtmpZ_b()l?1UX$x&ThHi>@y6Jps&HF(m_A$TT4)P>bMFJ)t4Kg2L&cTDqe zvnsj8gz=r$l3dQ<ZM+cmm(WK-SsrIK&Q#OtU~{-el^L0zoD9{%UPP&P-VsN?I-6qV zI-_eDNtJTl<Z@stM?C&et=g2IadUdMHW*o>l-Fr?^D+>Kz1WkR^{8Aumqtzf(l=?< zRJv=4>)-o?m1r}m0!P4g&LfvASKXP?aucP!l&bRO#Qju8we1uM8YxTwOa0PBTQ&?w zcZlA-EuwXeXR3b0Lk2R4aLn&D{yClmsf1k{Enn1tr6;#T&~?t1a|RqRH(swz-fRSu z;XTUPYE%YKONA`msLt1<fCyU8mXRk?b)1BhS}ZoL%BspGZduz+>0X@=B<Gx1e1l04 z_krV2TNc*xpAnqg1@I7@y69xv@)5#`GO|;7(BwPzs{J)C@TLpL@@qGnNgYJe3sdT) zdR!mu_(xz#&$(gxKxDz8L&;MwmrUZA+Mx!}pLqf(mZXvfWE!AYj$)4f^aUG(c-q%a z5gy&KiW`OJPxMZ9^t%5zWI_p3kfRy;)0+8{UBSs6pgs1~Am)>`^}%3rmmQ>jk+d5# zCFee23cMJm|0ucx7sPf+3V3sp)1mF(f!JY;ejV#%G`~iuuLT93-K^j!MJ2z@Zsbz7 zIxBHCjH{H0MU(|CR|(!lb=YnUrKtR1Htfl0Cuc#8@u&Y#8dH%9nEm-ys*IN^myf-U zPB274p}N~7s!Dn($(SuE5Al(=TTy}@dUice;~yf9D}4D$6^*~8RC1_(^Bt#R$sgqk zUXJ~G$_+WXBSuka-b(qEArrGzkS^!dx7~}x!R@Zbso)*(PRl(`aV#zlDR39u2^O_g zKq+lB?1)=JPc_TT_#Jz~Ll#~o1nDIKdee*dAQs}i9yZh+gVwPpV(*=w(UP1Gv}`09 zw^UE~;~8-`Y%-kv{fqs)GtEP;F_vvKd__)8jYW~qQeHK^C5z>R8U7usyOsmGZmH3q zO#3Wa-9L9!LY!6X!9`4FnIdp;^5LArg&PxlU49A8g{@G5Y@V=(RxOE@fg@_smEai> z<KWTin1!=dnK-vMip3^*PaB8he2<k*)U7%FQ`^r4x?MCG^tjsAVLS<+T6NT(<snC~ zWqfc8)_SRQNuuW3jv12LlhC|;c9SWY9vkk<a29NHrs8>?1bZ5Fl5u1rY4%exji7dk zsICvunIrtElTBO0#z{N(wAbTHcHPwDSb=91@`Ef@1wNP`nuZ?CD!Egfm;7%%-6DSs zvXVt~+*e8y1b_n7=8J(<j8{;lMk0jqB*x$JoA$c2&lG~0o_;?hL(l7sg~p3X@?c`Z zk{Og-L~GQo2aiCYGO8Mb5o71~oIGeJ?UGRE8QT@@PzR@7-Ie8g^*z7u)j94OaYM$v z<2i8N(}UGAs1LDb;1NJi+Ix~(P7{hf(l4>lb;I{)7xVC$)n|x&3&d|0FY;7W((!$_ z@HaC2m5jcVh;95iI<b5wiRS(Ntv}oJtCJ~qXlTzROmR=4HPeG+W~;1}ds09QNNP$( z6;bY8`Yd(fgonE~NP9F#W?9!O7N?V#<q}+i=5i}rv>Vy}<QvFyN%}J);$lB{mW`x` zi6R4PIgpXcN6~p7TQ_=-VmdPqJ03lVJ-kA-(CMD&rY#yT@TGL+bZ$D%6hpRI)VN!Q z5M)MGC_^fVCJl2OMECG!y;7@2*ig8CAK<WeFD6?n6C9=izI7L{n_>`8DnDK=@A`E; zQ>m5!@TJ57nBcvdhySU|x2TIqo>%zXB87J5r4nJKqdM2XI(`2&NC+3Ulxx+M!TI0< ztqlObk>0yJuiZ)!{>>KCDEz8sxFcc=zOYwcC#ZxX&uoJPhhA@fh6YRfCIw7IFQHeC zq<o=>VD`|cc7A?^+3@7t%9>%~wf2nb9%@Zd>ZpOSC{>hDAPaqlKPIYde1(WLfFM)g zw+|Y)dQu0*()Y+i!Ba6yXs4wfw#u3h#LqH&xstbzlzFBweI@0<qc$jJX_AdDzER); zEEJSV8yZ@z?-~6a$qzb%k3*FL5C8XU41G@;--nf-XBdC`lMTcdLq*2TwH0nuL0g8h z0Kw|*N<T2<sRGwIpXf(edyFvz(ikjdg@Fq@tI(TZ<_BLs?`h;!-uqGd!LM|UWyAUb z(D|n?XAxrcxZ<!0WooJ^{o^?0ivN^Ej$7#<o$LPgE*LxZhMx?Qy^F1US)R1##Y0L8 zGTB!7VLczJVgc@KDW{3Yvm8E69A>i-Q||;Bx;G){cA{GnB{2yjq0nj_*6n9_x1#>_ zzZWcJbGA1bng>{U^x-@Qm#D_KyE>+7u_D5wf#WcPhB>*G{-mh2?WlU)+LI-+!k@Vs znGQ<>_lCc*Z#2&cB!4+yqH6wKieRY>oT!Gtz}htXL2$fn8I+-x_O?Fy3U>mj$F?a0 z%d7tSP)zlFM2=@e5+6p^Sb(Wr9WU(vp!qcdCg9p5cOEqcZrfT0635bW4$#M6mK2NT zZmG|UeT?1Hp4G2NHF=6vU3mwwb+>7&k^Q~%XS;|U0d=hX{ASohBEuOaOs{Ls>Wz0W zI2snA>|sl308~t}f#i8qzz&-;Zb-LFlL2DxO}8@?W+h|e%(RK7JlN(sxStaCLI?b# zknTYQ9II-pY~M|XW7$;<jHTyy1Y(<Lm*tSA@<n%Gr%q*6gyd#r{!0zQfHBReq195v z)y*(Qf2%oPRYZUpmd9;042L)j>WRZP=mza`EeCx&`kC32rjyzHbvQtPU@i=db{Qq> znp~EKrxD_Z$`im=tgcfI5dZOJgx}(H1KbBb_&&o>)(t1UJXO<(cUwez1i92SzGyq` zHSJ9!Pv9z{!1K?-OS7i$kkH($G=H&A__X~raoB-8>R2=?4=}2+{O?h_0ZR1W)HLnm zmYMmu_dG}{xjKKdlB<w=%RqK)A>>WY<_-g^h=IJqoMOLgZB?fIx@PfmEU4(RQOE5) zY<-Ep)@W~J1AXOk3`K`_NMACF)E`#3C)yaq{ynPQeW~&9V!BfYa0YYm$r|f$pJU-^ z1T!D^e#2ir6&}Te26;TvC5zD57nS?eA%9~(_i11KUb4nis_}XYu8}N+RNfznz$kc! zc2cG-uA(MJ9eep?zG+al#c6Go1md{TV!n46dzOE8q3L{EmA`klYSWl-*O6mZ=r~A% z`Nqy$qgRRogAL-1%F(O6RN^0_IMmyI^{-<3%W3HrQV?@3FsR8}#iQR(4|V)U-~071 z>B!g&Lk_;E`vW6mmBu)9OV4eu-St`E6z4MDOe4d;{Ac+HvOjJ#V&F)FWqp%3P!(VO z!*baD7pZW*x~fePd1C|ZqLM=Dr?ZgQgzgl@Ob={UUMJ-ocO5-JXoaiX2RkMwTT!v8 zW=t~GpfHcU#u!Z=&Bdc?H-R>U_mx-cpG?jf?t)6+r3&w?RmQC~4IL$q?hLeMd6YvX zXBYsYTLg4dE=CZh_Ge84wuR`R+^0&vIZBOwFH>%o3c9rwcHqa&Ilb6e)N<KSfNDW= z<%6`7w<oJobK6Lm#}(8!qd%talEFGv2IX9r)g$9$3#jt$JpN2uWXsDU*|i%l5`RBY zEZfpeSu7P{^cW%e@fjpoOJ>wj>?d`^97rU*FF;a&R-Y}ett*qE%gJ?4RRB_BB$#x3 zc1dgA^tSAXMseQnl6Y@QK+}As*zRbUR{ot1XI$Y|-=6z063F9*ecs3R!_w-<<d5z8 zXoEYkLnXY?B#giU&zSD=@1ST;<rs+=7NLIpm1;sdWV+#g16^q1K?viw;Q}#;mR(^Y z0U>1=7ME{v1&!pd!NIqiDj>lOudeO69E|dZIn!&lPylFyyl9=@CdSUJUOGcxgK>H% zBV;>4iet=}@k*pp?hddW!~_t*vdkPw07YB^Si(@+bq*vBZuwna9=uzksW8?&`nRYc z1Evggt@`R8GRb86O~Drd3^^gMk@oY(pF|bC)A(mgu_}$y>t@1t-Ez+Xk&bug29mqB z@|T<%xq0D52pT0mc+sUJHYZ3pD7VS_FJ5Y|p%U_D(y8N7`_uuo(aqeZxN6dO;fsji zY4lD5SNh4*NN8s!f20)A>L%f#qVMO6h@q*kxw!J3y@9be)sQWZaHr+QSD7S{IK-G0 z)|&n|KRf0_CMhXkT^}!l1kXHr#9GV0_k=z^8MY|+m+RHFiZ&+JFqTHpry9QYhwQ3z z6p+$$bf_ZFlEHT+pQPBDfa_=r#`$dmV*Gr}#R@^^dfL#OI`bep;v6XX<%VHuBy*re zUy5X?P0v^*=v6-(t*zK`E*#uc9oud99%p+$J%#+=EWqC=W9!&4LVDQs2>n!B3$ZF; zv_l`+V2Y+^Ks*Wx)U%|X?LvCMcdomw-z6$rk~T3TaaPTm_@uzN;C|<h%!`-t+NFvs zZ9IxX#nnWht|j}}pKs4{Z7~M&_fL1ll)Mdi@>9kWG|}QU$tT{NW=EJ+*Y+WrpMCX} z?5PWrT1w<G(=-`Op#B2-8jxiy@nHOGM=lrsHX|@+UEJCFZL~_iUb<h+MUW}19c4!H zk7#m7EOqAZAsEoE=tDA7<{5#w$1;bejHh=KdqdT>YZa(7E0$*J`!eT>0rgy|>IGVY z@3fbwKmoe6s};(~arpr*<pM3zX?SPGtf9uJ_WemIos#pc$Hr|d+gcgDUytzL8b=Jl z6Vjd;hMzu#t)Sr}s>aII(kEDlI}Z4O8J7@aO|)7kl>D)SyFHKQ#AVJX^d{Pce`DE2 z>$b1hYe(y0LZj06<;zZXZ&X~Oi5Pmk*a+4oz9FAc1~)PEwij)r-)M4<=g>**o59FS zg<!Gk&UTSu^#;7d#nhx3JBu~+ODFjW>Ltq|@WHIqT8|75-nn0H%<&E6M)E-0%}vmo zsepIV)$7{t_|6EB4Er+mo>2AZ>U2}`b5AL9Hw)6dyMCN2F-D8XO?Ybjd7f-p=U%&u zHSZ=IZxDLx38$?>E(=e!8cG^;CCR)CSg|8sy?+`<-NThozEAfB?|t3gDNqTjzDxuj zVwP6N+sUb}ovHtdhLIv5q|2)yT9(YAy@-A<mOGy*f{#I&aGNw?)J0mJoO$ipzrv4M zfzwm)l?OKmD*8;D6^k-!ko+Ya*{2vRO(@;jOK6})Jj@F^y;^R2v~S!0APtO#?0qCP z2#iV-?c1$x?&ZJ{l|Ras&7j1oxyW=U6@u8$200PmMw~XWy=q)f(L=L6&iI`53iLv- z2{<`lWZU~B6}9JQyaSK`Vv6s9z8rk8XW5C|gO3j~hSH&)8z%{?KKVLrKy`<))%(&6 zqJ*<yyM*)U+t6~}EMiN?u&+eJQb?Sd1EZ(g#2xT}xoi9!{XwN+<$_DclaawE(ByIE zrm#x1GA}J5;6=fH0?ziY%PU{Ui02kglP=>vWYhYoe9Dz~8u0aiJt~DJ8A5Ad!+x0a zEY;{=h43T09zH|S-xoPm0+ykS%CWsFIOX#-O71fVJg)8An+50;{gPGGxrDY`Hb;fQ zAq_XEI&BQp?GNaK83p8K-55<az9Da5i|fw6;w}2j^n?brbL{c^2qLbBlwmLuvMqmG z*1@kJUcTcRo|i2l@j$c(d3Vui-6YgaEI_~6Rw+PW;U4pKqdKXwMoF(1>33h+2B(4r zGD)l6Ek1fxCJxxraLoWc>dvTPDJN{GXiH|?aC4ivXSX*hu@uFjN{7B8e2<AW8_TOz z)$*X9K76XeS)4w>C6UbqaMJt1O7i`pMmDkP(Yzn#SS|Xr_=ng$(_*1pC$G_sF0*p( zdQ1Z4KMk00pNU}ZuSWSvvFChkuTJs!i}zTk)L^~~H(S5G7{F4#?MlT^=`B^px1-Hs z@$>Ks!=E`sk+<xQ+h4F-0xF;|Z@lEe14?955{i&w3FN==f&;}p(sYG=m>G|Ua^#5= zKY5*~ebsqw9`K9F5W=^Zs%F^6Q0TJKbj`uMuhXnul{IyQ7|K=5BaELkq4K_@#ep;- z_dY_VjtEU%#dZk}283*7ohQ}Ey7r_FBL@|-wz`Su`aIC9^I|%dUm@6az}~#7LM4%2 zTmFm{K#KM(0wP|pS#vPuhpk88{3RwsA*{uw2mbW*5Z!}Ld+Afe3{s)$(mx+(QOx5L z`uhV##qCa^ZhGa-kg^*N=`W4tNfSQTNqAuXTU;$E7a#X%4*K5#kAYTcW^1Z>_tJC1 z^g-t0iUWn<o^R)F$8Hnzy?G${OYXd0eb9?zHvfQO-9P%E7dU*fNW7$oZvj=VDu4&w zQd1eCsZ4SPKHQJ0f@rj|E9c)#>PzajTa~_@ufp~k^rq$tG^}JOicM=0dks8PL_8kZ z_3?7lUqm!DDE2gPQT*)xO@9GXkvJ6*-v$@Ccckj)>sKRr55-KWJGCfRDm09szreBC zZV9W21B}6AQ`8%t4~2l`K<=ELpKxp___axYP@S+6H|<Kx1{*6Mph;1Yg<E78m0!z; zh{F`~UOJ!!F1_JXP!8k^i9@?cGPaCh<DN6KU2>cvqQ<Al1+p3toay_$cWgxMX2f2+ z&iY(n4g$qxQGnpunSG<!&+PO1Ii(+%Nk<Wm;8DK0J_Tgm@<wIH0|enn48Ov&x&G<s z6MYjYwr{jDAJPZQ^&xH4k7-v>Vt}l~^Pb9%-fkTr>kT1~fNs6G7%qC1;o`wA3ga{9 z$)x%=)9f%;;44SjhS5MwuLsS}TDVibWDYSr(|k84C^nLwnFSHza>go2z@cXlc6aSr zZ&a@b><KpGTYIoS%Dy*h2f!|J71<4z(wczwd<$_j@>%e2ouTIMz<Jc>99hlQ*!HXR z*R$os7-&iN@v}Z+XFEzbo<zP?zt3KkvuBGyWe1=xW}2Vl8KF)wplydK0qTdCYWraf zYW}>1;@0A8G>5#{+S~8%8HQ{TXWqKBX0l7({m7m54R(pSV(?puS}9M<x%K4=CpSyS z8M|Q(m;?`*d)&v9MNMn1ZUmHPaQ3mv%oqmqCrWUGK3Z+Kc3Fkl+k2*xPCfwW#b)x{ zar<<8q67^zBhW^dYJQ+T%va&BHaiB=R*3RTQms|dvDB<rEFq#FMsnO}c6zEty&;CU zvTG@^gxG2$-7OYs9|DL!Z$vYERsymES|~gWW?*iYjlV{nOMM&`F1QR8hbyRwsH~&? zrv^6pp`F&`?_6B~OGj?)Et5n;g%YW@4_~(gaH?<8Ljo)Or%%ZZKt(=ndmIQl&h?$F zH6NFhn%9)2A&>V*yqbUw2+7(i2Llfk%Yf~L-a7-@scwWv;vWA2P3EGwl#VAaSXqEn zW1Gg-oFG9(aCXpICEo;FP*K3}QGqS8YW3x?pFV@9;UP`sQ^_cLP`*pwxH9#Ww(SSU zL#O4KeC1#otvBubJ*gREG3}44D8kd_!wnaM>?40{2*)#GTckn5`jyK;R<PB1z1w>N z_MgpUlnxqdhl3y(q~W^%sMQsMrkzBcgVwlqP|e`Dww>IzN4U<q(=oFuQtSgqg6=)g zA99Jo8PyGSzJC%QcdG=Gzs7E4bw`u1v}=(FH&yaqrBKN}nyFCbnL%gy^UANi#QMe3 z>*@J<vkwbJvJ!%?M4nX{QqN~YeOP5s4k2LXbNXxpbtke3LcYYbPBmpWj>*V#hKd#l zoL4@1R6UQ)F=37Kmtx?#0F;hG)71^F4Jt!DG>grV&LT;tcLtI4)4r}xXvf7#;xBV> z{hGM>PZEG1<|oAzQii%Lk2b0CO%I7>`RA+;*(jIcc2SfIp#9&s0WMiFbGMZ81&Pt| zhoD=LlIP1NY<IDY?QHRh<FwO=gZCn{&U_M>7*x-Ti2>Fx+NtnE<&Y*JivA|o9My}e zgSfUfcJ(PQm7t%~FxuyO7XGu(hd=eT=IY+KKxWmeFn`1GSRR#z2$RiT4Eh&Dh&E|q z6c`0rS3Cflj@XE#nU%Ag34LGu&EVU)gLdbHB$0Q4WmVsA?fg=^cBbls2wdIVZPASR zk>Cz(kg0&-H|ydb#rUwA#9~czDsQV_7bvePQIsvl&w(5^M#g7^mn#J5!RCq4fBUXP zmScEsxOT}T*=$&G=$AFi1{cOOoEI$%pGrw3FHWH)uNiX|WZCxwJ3Y*@Mkm@9xf>4g z6vh$N@iT}NCRobQQLCV-2j?EB(@+HlH%|2!Ft!kCcv~$)@a{La(_ZVR?V)1*DB%Ln zqAj>j&o?~RXj}v<eup-@jU|`;Ond<fu@BrpG&$^g*U_x`pv~8cXaVxo9<zTI{F0sN zU<sC`y*Y3E{)u0$T;}uozc*A_s+`R#)2tK%He2bt^|kqoY&QIZPQ&eSh>7{SSk$yk zBe2l3e5&TOm@$3SGkx=Lqu%4s%TdejQ%iOm?}4nJcro<wU&GE9bPW=)lHIcfRHOG! zvzEW7h@}qD5(^;3W0^{P{ZDz%VeDT{wX*hP;9I{4;%kel_0+UxBvnCn4W!2iA-2Gb zXzd&u3vq)~BH8cKHxX6+0fyDryGxMpWbIZ-U5>dZN0z2c-$R{sF)r@sMeS3C4S#=` zOMCe5JxC!bA*`x^(J>dF+ho3q*ItC`t;de}2*G`ZGyYoo=|X)!yCK;HVyfbn(t%U! z(mdRe={ptmW2%ZY@jG>`+Ye%@k~r-ZeNfX?pd6sTU#}NfMk-1F)t5Xp4K}=BFLxE? zY8ca8HI6o53yEu$^`Nz5snEhi;$ni}B{Q+0&3-zTS5a<I)<wX^;^*!H$H(;t(SR5Z zv=5wGJ78karjBpf3hGx4S9BG{(*Xjy6Fo_rO7;fJi~}rQCWo&F0aAsvx@91$SQmB3 z0El8urRoyJ34i?^iG0v@|9cW;5HNe8;8}B*3AiYv?Pw+B0?tdD7bk%Kgx!B~c%z{- zo!+h7QcmgGWfzsfB*Gt7aDV1={(<GsYzLZHv0s!p`MF4ViHzXwAnM!T&i-9t26zP2 zz8EsvqDPUo-{3v78jywKDO7kGEF{9Boo%`i=g4b=w1BW|b{`fRect$Khz|WS8gYj! zdUuMvv)tgldrH(;dLG%6c%0)y-I;{?ALfXuDXrt9jbr!P51|4q)F=ZZORI-$4v)ja zs#NWn>^~UhklOmF)L(Z~U#7mRnQhHmVDcx)<c4M>8_q?|lDCAe-D%c#rGPfp<v^`j z!<tuV#IRsN>M=2doe3?{%`W+aF#{t~RJAJJ9zvJ4*c2uu6TIM)Frz1?Q)nrKb`?ac zJd`nEj=#S(2Cns@Y#lm3mdd2hRH3hca~PEtKP;)9qoU^Tk#Wh(htdzS&AI=%yx)F- z9&^79CII`I;e1d9z!0zZ@9L0v#-2D{eO4yZGgNpZqQ)HWgtXvFGbaep;38<wnG=tn z^|(@<+&EkM3+l*}9wi-&zUAu?SIV~;a{P7uOye?GEbt~ixgqrH2ZG5#b2r!CncE0! z9okJ*#mHGK#==So2cQJtvrN{ez;^MCUoWV5n}F{2wSG^+!3@2Msq%J0+>ZN`6JFHP z1wExo_z7SFHz#v7lYi0{OSWMGc2~FkkR-oFwmAP+0croi*Hgt&!^qsNc4QUW?9Wv1 zG_Bvp8gHB5lvvQyKtxtX9!F6r^V44Ox6NaSXC0B41Ijc`h;rYk=Xf|ij?BNK!O#be ze>6_YZ9$ap>B{?RwR8XydUf0n@t7bMQ2mFXX724JgQ?rOpp~j|hxK-WmdM}%r088c z2OPrpZvD^r8sLn*=!>q7Td&Q1>UUMSTy0&V8v4YX?bC5t2T?IUf8rX|voMlyJgO%p z@qG@5Fcfryn={-|tF;5Tn2aCB{_ZFd9ARt1en(CiGk@Nl6U(`XAD`E-avRT8bOX<b zt$0wicrvr0{AdDs!~JjEl@%k8<<_%nkU#mJyNhX)iQ266ok+p+kiDe?A1s6B&Bap= zv-ko%cAsA%3`{uiaV`Ja9VVj)mgJy3w2W`zByfbjcMKd!U`BhvJ9{mon3oPQ=>wZv z?J?C|0~wUqau>a$e&x$+Nl=e6fR83Wa>nK(P5kbgk1B*yykA{Ado0bjK?5gFt(mL> zuFVDPqf6>#J$A!Ktd8MuM3%ARGF709e(O94<=rQ9nICw*P~m&uJmeX*f_TSsAAN50 z=WuiMJLz#|faf=_02yo9{{Qm8tf-h!7T^$R4ju%@GlWe#hqBh9FesuQa{L>7S$*6} zk=ba_3*qL%znsM#n&{a~=b9_PgJyrwQL-zf=6ZhE6wFr(aC1^=zTL0Bw^_O1=U5Uh zR>Xf^HJbIddM6Zl-$m#=5{h@;5^5!%FZ@LXU1egXU{xPhWXtf^RDQlqx*BX9H3WNU zZ1L5Uw$a2b1=qo1vH5I)lYEYHZ&MtNi{7hdHta|33_=@Uw$9qQn{@l}GIoHeubh)F zwCEmhwCEUDi#8WE)pbeYx9TrZp6VKD3{=C+!bpK5Fq1~F{3gVj=3DLlGqzqmR=?bq zT)0cBTuAckaS-0Yl#|n&>UzQR)Svk+_o);9xA?69vMQ`wc&FRNthnC)g3<7?6<}2D zk98<>`tq+(z9cu6{^>j2e6qc2(u^~zly|l9J?){08dH(YFE0qg#H`M11YG2>c7A;* zBpFYi71I>YBAZLe;L6o*?BDpLIQEFjkmc2#)qL+bLN2Ri6?BX@=!VkjNhdkihC7bn zxUOk8`gLQ4Jmju|-q*JIuu>QIy-k`BuoqoO83+=p+sMTWA+V8{3&6!h2ndAqjO$OA z?~DP;IdD7eT~2%t$b?Hhg}(ZSxh~h8PpZbXpI%QP0jTjYsUD>0gEhDluJSg1OQ<hu zU>Tzi7vfPI*c&>oX;+Ufr%g!i&dy7+-UhCs$GX(<sjBqo1qR93#qIBs>?k;H=%!Hb z#3i>lAuUxAkCA7}=H;&~=*4udwzl(LbA?|4j?(BFU`<(gbWhZZS8dnOK1~Xoc@w6? z(Z!&u8&nXZSj<Pu09}I|=15)w$a?bnig6V7H2O<7=eH-o(=IdJfb#7}326-M17bu? zJ3kI1j|BgEyh+zN&Rq+Rg9g$AI7!o#5>Nn!|BmF-zC1)y&4CYl#^PG%g|A8evxxQ; z4bf6Py(#kSF=^ppiemM`17e6gc3+%$I0?G7avt5)-TK>l0tz+>Xoc-?)VR<HxV;JE zf5n?_&3^p4*54L(kDfUQ{K_@NwX!dP_14H3*kTkgt3<Al=7eBkDF;mL2I`%n6wPHI zNEY(xKccS9_>C;u_!}RtQkk+JQLj;fa{A-w7T&jrp_UvM@40Y1sR$1p8sk?csF~ei zdG)Z!B)~1d>sNwE@p%C)QDXIakx}a~@T^jLqfjMMg%z1}rozE)&Wu6FE^d^xHgl@< zclGfG505?mn#-kPVaH$oZVyvBuzjKY*&|H0a$!g&hWmT*2yPdqh(dL!v8XPtL8(uk zOu~%5ng4-(4bFnb-HTNG#kKR*cv^ZRpb|UOWI|;MD|x?FW0*EdegM1fC|CY!g8EbK z$=_Kp)bpc@)-p<Y!|7f}M7h^S#@iNYx)ocdegfc&P1`x~?g6}p(&}P)4{cIVU5zuz z#0js1D9|FMxD6iUkOq;-;CL|7u1}Q1^etoQhvc|2fHv*@TdL!C-O8zB3ppWobf0>; zFUNc2rY&X!kfrx0+*I9BN1p8$9X<d$d(EHqWt9{7>9RPBrtQS?-&217ZLcTIZvTgn z^?*?a8L`MWTs$+Q+XE!=7sqsHM}-5KWVLOQ^d&xo@v#Yja0s<B9nG7?%_Agz|2sH- z=}U{vPL0F-A*{IvF`=0S`ICUFG`dl^g<fM`=Uc4TnR>He$u0M;Mk~aEAFsWTUg$d% z4Xs-aAeBZ^^g%v`fSA#4(3d)7r3j=N)B_Xty07{cEFS7iMIM_ArjR~2I!GipB*T5m zx4%W;pC3?K3deSxn@zVfMvwJ3ch#33;-{&MVKNIe@#l)v4>X@*L|`=3M1Zt@H;XpB zsI*_&`6AV?9M+pPtze?Rz~eK47|@CkNa16>99>Qw@gwGUdbt{ov+1=i`0G983gfKT zm6u0cgd|17gP~rR&{p>;C^$&@t7;1yNFj&Mngpxg|HwaC<UX~lJu1E$6XYb|e1mj3 z(s^utVjtYG8M>AL4J-HRYo)jHmrjw>UjL-zyNT$o+wHZ6`E7o>YM%>daoaRbC#|yu zK8zXe1JW6ShL0YB{=6eEAa1bcl%b+6H3gqB2V8!2mV!{j{!t2AJbNW(H1;t0%g}K^ z>*Yijmn`QzJ?H%@QK%1x_82$_JJ(9>rptpsxmX2p+X2-$eb!r!l@w==(oE2{-O^oW zqNGfaeW&98P~~_T9YN3BuIgGXatl7w<!FQL>Y#e7B!v<~zIQVSISZPuL{Jx2@f|$? zYkGNd<D5d2(&+Kv_PbE8BTj~Kq9v2kkg98g_l>dNAtncn)jUp2v3o5wg;&wfaKtdf za15<+jmR26AZvf5hrEZ#pmscJ3;g@jZa(ACK1-kG5gH{k!nr(Fk|TmPI*K;l$2zx< zo~<cb_Hnh2vh6;|Kfq@B@h{szzUMRib`H|98~UQPl9Gr$gA0+e0K@Yw3VtQvKK?kG z+|K?$z$m+f91OQ((*j(K>pu&7Mz6h#GhZul2;2FnD(F$C=Ug04yi-%1ygpExj-6*7 z&3cIfq5-k=9j;#rIr^JWRUb+O(tt+8V(Hi3l-)L9=eL<SP<#I}YDMAN26A>G-gwMt zl4#kt91Hkj;zR2)7^avPcq_es+i@D*s}Htr4i1t(Sjk-m^#Z$J6uY-NAk%KCa{u;g z%^0DSyN3@7DN{jL0$qb@WvRvIm4pOF3zOfra<ZJgK&xXr!Ep9Au!9Ka9&>4jqH_YD z+4@P{=D&04OxpvE`h+G@>0}`RcoiA(3e6~%HWyL=m3>}&2@vj-)?$|$h0o6OsjSvJ z)x?pN>m2N-EUcRjTLMPaWQ2&X<@R#n*S8z)5tAJH(tOuryHW=lmfaw`meK^x_EY~# zXUgIxoCS+=z~4O;F5=$u#SwD)5Z$y5qZ@EsGZ+XpNL^5sJ+e<Yxm6jU@Rdt|&fJmq zF-X6PE8PF)3>)4WWm?tNH>JG{)QPL_e_&&X;o&J#?V?gRM`pD1nZfda*;B&Gp#pha zp*g`d2WOW6&v$L&>a^Nf4H^9vbEaqaX0kD%u2+0laJ2Pe$_riv>9?J^sI2!d7qC{I z>V==FAU-`}K8fdZ{U`$seY1AD;-n3}JIM(MhMgh|fP8h@XavNI-YlHC>ykXuaM4ch zIA)dWjd@w7k;b=7e&4J~4;OhRYz4-<j^4B!>fvY?EQn*WBZ?$Hb^=*>cj*?WGN`^5 z6fQh7ZLVg`9I#OVP&8p{f1~u|8QTp6umG0o74P5oy%Gv?E8XS;CX8s|=-8lG)Zo<^ zDC_pZA0$)fBMM<RyN1uXt;AZ$#yVv8g6boXSq;<wG%0R2P(wkr^cCJrfCfnz_>Mf! z9wm^?;n*hfP7mky`TQ*_9)?fUFO#}*@6gek-Z)#7u6^#Bi&_&yjFVbb(v)O!2HGH6 z$EQHq(MZUxH8bf>Ve3eh_lxo`66pvQa>?H-t;0}D49635)?4`8&vIy4#=R$Q0kVS_ z>(tzQob=9hBNfaT73u!Wo!{0>NbHOyTCeb2VZybkq6}&A8_^<0CyN!Im$T@8*%u@F z=rV{ih&X~RZ$KwCCZwa7rsMu;TlxvH_Bq~}=*Gfappr>`t^uVdHr<qZtE@5o)O~mT zN#G=GTN?Q~9%Zk4^Iqb(d7so70f{3Y`&dan+A0YFF-_L}R(i-y$l+K=SKZ4s`=7L? z<OX7|C_Uecc-SVON|YY~ED4d*fNU2)8=U{*HQ)Y)*sGXq={Jd42Od|?Pj}D&8hT@K z^Qj}ec$%vHQ{^<YPPv{L=q<nX<3hLZxLd)hkjFm`P)l)3ii`^*FEy|~2pI*0CzkV% zJ9W2&X66}K*_@bQ+B23rBhrE`krcM(`3WIOy3uK(0fN0SlKRZRzBZuWNl<%hj!x-h zsU8}r^pxvQ1OFEUaclm7j|1evcaWa?j0TR8;G&wi1wPZZgfU1ZAqG^d)aU%7&ter1 zl~3(O!tezb>m^S*GgR;m@*)J_Q8u+D)MPFWXIYw+ziD68(OX4Ycbs~=4i~Z;y`VTS z{R9{tS%R<AAn6?`?hI9yZ;{}N<Jg<PbL%OuDkn08J4E6J$ez~Aakp7#QI|zEjAy&n zyd{I-+k+tP5P@t6qlS}Uzi=Qgpvb#RC~<FH<l~y{(K*sqRVbSK^6!Wh;x}b1l4RE3 zC^l`(9CjXr@A1-Ldj+Ybv|8MTTX;^`Q-D6x@f?7g-H$`XH4-aUuj5nMEk@gItF{qU z)Rm(i2^N}|C}WS<)#dc&stu)mhr1)b0VulP+LsVE<y9Ws8D}OBHwq^v|9I>}S}&+p z->LZ^iBhV6r1R5Ov{Vt^`!uV7CK&fw)`!IKc+`a{!YMmIY_vO<V^50uVApsDy-$q^ z%|tBAGTvVfCQ^^e`2V@YHRYRiV^{vi1@+EQ>=}XUsP%iF{#^Ul;|+g|f^fe8n3NAd z%7tHaLk9R?b-7)#+~+zV;ZTW&`CVoG5+Ey2{grZ)kV^VgFUb$VEOxdpSB%5&;kgT= zXv)l9L>WpZw#eZi=ye?RfQ&#Er`EhtggpB))my~gh{%=sgDiy>2TX1M+q?#EDQlPW zp7bQgZ2f@pf-X5++ivv_D^a8)1!~q!a;KGbT72ndw9L<2Or<1GE|d3HD?VbbRa?`T zn##F2q8va5_X#GYJ)E62-Y7wv7dcnZef}v=Do(9g!gh}+HM!8!14-!hC)695;{w>A zNNd9W%+zZ{$yK^MFZ0ccod>Vrg^5vrQ!M_Th+9W4cdh$)-Ka=YTPP`S3b1-Pt3nP_ z5k6f%@&`bISHBSgc@VRO{+k7Os(f-p{1?4-P$Hi#zP|h){EhmJ7G#`8N67c9o8C8$ ztm{^yDmQ)lx@)JB=d@8u30#O=&@-ATP3ya=!Ye}C1_~q7U?Dg~IbdVzTexq6d^9zW z5>7ja1)p2VMRi^)<bl<+woe~0_NnFRHxXl1MEtZmNY!yIlDUD9m`TT}$ZC=^BqrT) zU3>5MN`};B!qfGDmR)^%_ZB6kdEj{MPM#XTzPa<-+t(}=Gms9^Z-JjmF+MCL<Wxf7 z#-HI5tBCSrq3zFpeZ0HBXVMYFfW4f0swQfkq(gelg`O^JKn^_rDS!K-g^SFB*HNzI zbI-U`V~w-!GUzik<=_XpezG~TyUhqB(yqTzub7}CPf-hc@rmJU@y23TQN62338hXq z4Ed}~ZhY;_jTd-Ll(LgWi5MGeT<mi+?ktag%fn&ur*eHkyvo>i+aGlI6vcKP;A{9> zR6{&gv=sr)>t>nCifD>T7K=iYAAjy1YlXJ?Jy@s*#;w+u+<2AnWU^sCK!lU{?Pq1~ z{wNF&1svlO^S$4l!RFm&Gk&JJfSbJYc^)zEFid>=G)@=vXA>ncao_sNxApHe$zbMS zryJ}aFfa6GpR9mIzQV+r0|H90+fwqc)+>9MVyXoe*oaZaS#;=}M7VhXLU7?%lM3j5 z`Oo2J5ETo^nv69(XeB&b-$h|#ss8_?>AmCG-2eA+qrpLq(^@rK9S&+#QDU`}o(@Ir z6&0)Yj1ha3Dxy_KQKQv?STSqUmZ+_jh!CM>$v}u0;dl4_`TidFzsci~NACN1KVQ%5 zx}MkN89mVG{99R@miHbQ8tT{Yng4e^_y6n~v$=6X*(^)vcs+j;EVL}vTd>aTw3~4_ zDGGWJ-N^k9sM8B$!{>4fQoWACX{HpFlYhLn?P~;`_hD7xyB|zY+!PsBXA}5A=ZXm- zM$JRlAChW0aa)n|pB<I?)Lcm|2h^Jbxc?5{aeV&z=6!lgr<R2uU~jis4eJcdd?IB3 zfcIa4i}+7Z9<E-fju~g;Gs>1{fja!9_jI2aLYy_^*o6D%_3z!VMiyKHp_&e+R;skh zS^Vq=I;x-b;aRx~n*Y|Bt+ap)!6%6Y>&xmTd)OP9<nQ?D;Ix!WSS{Ooi+}H3^owg$ z&YTPF{ONd=Y!etTl!etX6CcsB4;jBT&?&$Aqw8**XFlg;`{M2Gxep(Ef<Z3=I(M<# z1yda#eQ|Y{1oJF_p3|~TZ|Uv?<h8YCHY&{rA%FN|wUYh&`#qd`DxM@Dtu^lt=S2dX zX0S@M|MQ8N<ZhIg-YfRJ5lxD06@w@?cfFPbvD_@ZWA*NigI=?V=9=t*1#Ia<aHT%Z zHHEDs7TuH8EG?)n$Sh;=JN4$%mZwvAc^q(NVie;LUD^7sWTW_G^R7KF5lpNbd`*Gv zys6KzV4&2BM8c+Vui{*oJ{gx;!+f$*Ef{&l8~9=w9w3-4+s(<HJ^_=ApQD99Z$gPX z!4)K<5&h0fc9GxIOhS(UsAS&#V0b>#`5bF$EU;(Q*X~~iU>o!oxM#=U5Vu1?)X#$$ z!Ee8Vp8{qV)+hF;SC``@Dx7&WB3SF>cB<lSE&q<ZIVmWhc+M+D;P*Ej0r#^~83rwr zu!kfgzfY<><FfHB*!7$nscC%A|J+(z_04aRQ>VLH;`4qau56;3yVICw4M2tSaDId5 zTeT_t|7@OKjOu@XE{ev7aN=94{9CyKU$ZMlovM@$7=xPdWShu$n*?(Zl^cVSHVSKk z+Ne*MDSNk(nkIa;X<r(ypTM^tUJST%_;hzZRd7rfNF4DSE+d;W9QJ3Ybg!e&H7Jmd zwOF<zYn3>4c2zZ0ayI%NLwJ_{iowL2^IyO3w+QPUd)I8|`hi2m<mTk`uCcrsylMLu z_9dSspa#tGyv%VP9uJxO2uf**wx{KyFQ3~ewp}7m__G18qBla>SO1Zx{;_AWbU80H zAD7W%cwVljA|bUrQ{rO1wNX_s`iAjX@wv2tg+8ecp2a6Dw<fvI|C{{sL$gK7#5u0N z7JXc<amW3r#1u5^G{2W`#>RRZoXfBRhOy(SgiQ65#vX{*BO18&6~R8-GQ|GvXW{lA z9hYpQU5Ua^KXL$j_2*jsA3LiHJhG?$-GSJ90!>@ZB81OYrfS62oyNRj$Cs<k8OGIM zoPP9|>DTt$I~-r<puNq7$QmemyJ}?^*z<RWrJ)k`F!Kviy!RCfL%OlQ&MJ(Rc{F|f z=RNmsle+(wPzjhYBcRv3b7lGO?=U?jT`vQCZInyMY2F%L_DSDg^m{&LiK+18+~7-N zAgWf-gVwEn@hdI#_@b0(;i0Za34R}Xf=|5w&VbT+nq6C&%O4B<3uG6a;bbdQ-QCIU zj#b?N(KO!qE%hnKM{|I=&3}I^K}I7U!tp=<tQ3M-OKJZ*0=m3C>(zN551!8f@ImG8 zQO>4W1|5Pd0y`P;j^^2yuW9suk(2qt^R@KY1%cUP+|q2^?Kmr01mF4Ke@@?-dBMQY zjj)L;d3tPKD#I4|<w(|m)ldI4Q910jn<ZrjXk?w{D5!4T`8LmmF2V2MaT7Q4ZD?Lr z03oD2wzye#Mj3<4)C+mg5VMnizCcrWh%teKRHDSc18CxHnsew8P<yB|6R}2fnOXXk zot|T##tP@?f{aK@QWVtFa;^x=n9vO<K-qwOKkgl#EOVcKC~4q)H_l~A1^oshXI%L^ z^06rQR)mbU@@9V=d;?Tr`~@w=C9=AZ@b=y2JXbs)X|E><uy=3#%C`NMew||v2lddy z$G#6sVm^6%A3?n|^Y-uc=41pA&g}HnnQ_CL9TZ-T|JTQcoi&(YD}JS^+cpv_y>n^V z*$3p%cHBubOU%IN<F;@U6vH`Pc3!a;;3pDZe$HvkL3o6rk^Z;}+n9xP&lEmdaylTA z#n`pyu;aGh<~PLTT&3hxR}O9dwjQJGhV#AOk?jQ`%?tc1ECYGb<r14qsn0N$4=xAc zrtCp2p!3wvC0*WEH473yG1O}Pq{^+?&zr1>q;%gr0|=l%16=Tng*qB{{d9W5PCL_a zMynhU2dJQWZ56P4y2>`ts`<J;V(!iVJa^21ovcU<l>fCscER1wSy8?{mcWaw@6`oz z6Me^RstbH;WB99Xwfk%;9XYZ~(_%Ldr7n&v=y4aCbK$rRe=)e0?N)jh{!JD|J*<|J zVsT<)eeC#`udh`UCNZLec<#=Wy{`@|u+z)0l(x9&)wDv6%69hCizd9I`k2Wf;1%{) zr`~c}VqHu!izT|MtM%>AhrQDu3b7_U;We{uYdNlv?txWidUVYq8WUA^+%wtv_P2Fr z2Yo^nivcxw{_oFMH?0TRU2;MhNgt!M2s7^omvm456{uH-nf<ii4Sq=B6z^(!`^tLR zx3cx4g?3Y|@TZr~KOT#}3}D1umHe>!9IrP4H~WH%M89=7al$9TqGDFY9wmHuZ$Q0$ z@y(D$<oVxu-AC^GN2~e}z$9uf7f9EDXDL_%nK90QV8hg4=@XD`uxzssaQ^9r{_&#z z|0M5SJ0FyDku`0Vf3gfrH-Ipi5H{`Zd62Rkf8>B6ukLE~|5$q<8n745DG}K7KK=Aj zH*6v=kxDrF$);Rsf3u$}Sv=++&1<l;r2wj-!pT5af*YS%qu#TicKiJCSlX?a#%<+c z{R4}L$)|UYj9o=mx@_m&eP&&h(!ESpr6q~&^mlX<mzU-$shR=6m?oD!=yM{+)Io_j z?Ulxy)hqTOSscn9MB;yR2xdwK%)JH#@=XBiEN7%ysk<f+xw6&oKY6@H8*KzkR%3S} zpP;;?N5B&uB7W3NW*XH*PuX<nc%WFC8UCxWNqV<qZPy9-5T5LiI<gbH*KHZQ8n9n& z$2P#x#84Kc0y~`fSi#c184WhmUvFeF;wXM)u5YY5wR~kesj-a$C2q*u_i8<KtaX91 zWW3&coDVKyN3h=0Z;D9JYc!j>&Way6kTmZ+%QQHwuVVcy!u)_KJSa|VcS(YgC-i_= z+lZSRc&VF*KYPX+rKL7-dBRUKS9F2({N@bt%s<$@AL~Jjgl#+wK0ondGsx|DmdfaK zXGW|*9KttSFVSTql@EqhT}L*<pc78%`p;qG#VbC21<Negjszu~*N5(X>XBgCuOISo zzOWgeAN_ek;2KE3vvkGR`3fd+LCq(w@$VWT(VB$HvK0}G<c_FSZazH)@6gl_la+_w zqiuhS8nJOE7&D3wV^VboQ%|TT4)b|>fz?Kg|5loFD---h2HO8`F*O3jpuT>1!f<!T z%yAA^Sl)ZvX>E18t{GN)YW;Da3wP4vU6-}zU+?OwJY+*g$%tjh_PUjKfaJ^O>pm8p z+K=pc+?dPu%{EtLm-JhP_4!LBe5@RcDRcOy$vHL025T#~zkJ^?g759RFzkI4`p@lD zJTbAk)Rg3twP#1l{^eW<QY_V{o`H(Qj3Y(wPi3BxhMrht_{b+Y7+7<^ZSJ|}f2I~U zOM5iJ-`O#_Esf8j0Xy=~|9$dcs{>mg4)IhMwM+MuI_VhfcvUzqd^LKQ$w7RiQoM4X zVaGpP-8)?!ExSLSBmE&Z;^^0rxBU-2?~vbpMm{5VHMXPgsZ*=PV{sfs5Pq}TYDwVd zXHp~M#>Z4SFU408pxa%`6$VuIi*I%<BLDfFz$qn{=N$FKuEn3AV<x&93<x~((E7() z%P$boZjqB^QtB198CK6VeQ{><-6cm$SC;4C)T_k((eS~kF1q@}J`1qbE=iR$kqon( z1}umE^Ne4r`g81<kTXj{sK?oqDhqv0{(}>SfM&Vw3&m7uSuizz2sfTPEe(fcWLcsd z6*p0wORUnRfcGQY!8T6d8PV(LHdb<2lY7x<#WZF$^Kgw(wnM{9C;V&ND>~0P_LkH9 z<Tih-?Vy0ub;=rkx8FIGY~-rx9e15>ax3|Tu56p<gCSsGdj<$NGRvvwVX~aX8R<>| zbH(&x>8+US0Y=bNO5}?$=@=})KLZ*M(rq-N8S*R=;0BDF5X~jlq|AeGkI>^e9HwWi zlX$3{OWgB`Q_W+kx2~|boV&!b7VjgxB+5E!&)Ow-<F7Mmmb}Iz9Wod|&S8D}F`tyA zL+(8xgK-I40?&8?S&9*qOp(2*W50#5xY^OXq8JnZ^e3BF1yv_V996gq=9aa0(Z-K* zPh>ttsKb;mMpxPGS%yn}D%Sn>^C{jXr6qDUqBo3Me3PbLBlfV*i=PGjYn<cB*v`^P zT5L3Nyh@xB-j$_@k{*HUa8y~Xmic%z#tM@=#uPrB<t_jSz=(NI{76Oruj;kCsfaIi zW%1_!zBF&XN$igLTrD4R&w*cHXjP<0e8=RlZWJ-|RpFsB2Uq^Vi1^0K+6!%`>CDs< zlFzbX93Asa^SUzk^fmOaZZPXoAGPd%8{XDgY{2a@bPlo5`u)BsC8;047!xhIpca?3 z6mKJ|*v5@moxQFkxT)h1U!b<n8aa*4XXGxX^i&DYrNlOtUP371`C^|6RB^b}Jb7LM zmDVLTRercj`(as9>{b7vO_39>URov0@oHW})M4ZBoYxEzd#@=~uWntA`Q{exD%?0e z<L^LQ<hdbeose`>dM|)<uGF!PSG3Hn9SJxtMR_g6IDaTa=VPfBj)o9yn8}nLB)rop z`S5;vWR2t)Qhod^n$nbd=(W3rZ_C+MvR1^_v{*|7vUCA&f-i}e@r)NoDXNc<UzDaV zRNHkt6i=E2&~ylrV;BgYB7XK)Y`!I49MDAeIZRvbNwc4;uK{;R@0JN=d-Cs7uFe5D z>~*|CYn|E2kDTR|FEfE2s4N1B4%(kx)wA&j6i{!aW;{z`2Jbm<zx?08$CCX2+$DIb zds6j}7N(Q#|F#V|cD7Y@MfITP@`4}Rb0tL_q{Ux6>%jun5g>92JfOJ-UFG=Az-k%! zA(}biWhxIRpK{$llY#)+UI-%5QIPy!8?}NdRqAp!ng<qPAnzse9eh0i-j7IhD|1y| z2agpCWflpNKm8?bLgUNqd_tOj{(*bhBC~Vgu>yce(&h$mnJXiADCzy9*YPGLdF9kk z6}q^EEke2O&%<GXOs4smjuhFLl>ak&<uz<B{TS>A)DkUgmpH(nQa+q*?<DT~#MU;B zchk;c;(&&B>G#m%9FhP2c#@vljn_FlU(XiJB@@-3vY&M-sE32}2+Cg){!N%a`5w~` zq;-jbb3I90FZ$z6A?*6A`CiEH>Xa8UQYk8${*$$31q5OJ9vxYht63nKi#TXP8n7Pv z&7T6n=Y{j(8DCMd_$|G8_=C}Z84)$Bx}3_5t|p7@T~u2-uSB7BX0wz=5A7p%MKD!k z6M#ytV(_i8f)RG@{-4KWH{=TTdW6@X>(XC;UvSo^)3txEN<|dW&-hWJDF9%yNXpl6 z1-X$%c|0sak_W5KxY)q|HCW|p2W0quF9w^cy<25hG}!=_DdO$pb<b12fD>Mvqo(%5 zt8!QON06<PqtKshRjsG8TdWJ%B5f`tS3hWog#6E3gr}M%F25-^$DyuJ7`8M%sk=93 z(9)6a&U5OO%MBn=UgiCJPaYOBYwS1#=~~V)o(_;`F{iaDR^O5%Cw)KYA%QUM^4Eqn zSJ%XRQ9bd_a#;s_i;0Qx!d&@yUdYepYSGGS&N&kW{zezSe>A)xSVpmzapir(j`L}X zKzKPcDDYLkcxpP_3cDZG`?~Sqx%BIC_FZ`ER|_F)cIm|k^)%Z>@LEzWBp!`wQyLYc z8T$}0r4A3Wa#q+%de5Jk<rpbWg>B#MIx^##6Pe1ek;byDSx{cmFZxN)wC2xk^fg>| z)?y&FR19)_yt+suIfup6-RjOy?$0@DYMapNsAGt~;%Z5U2fZ0ck3}j|?I_$k&Ue4; zD+>vTk(e!Uhb{l3%Y15DKOQaWAD0cKq)G(ei9Dv2K#F&(JaYj=r2SyP_<yIwh_h)u z=%bWC7I2mFnz2m(+P&|kBGq#~DbI{qLtW6<<4=idZd-u=B(kP4itwi)CJFIReoJQ( zxo+@>NP))SI}k+y$u$?JJ=tH%^^|uT8r;wf7SubGzWiPk(0kzD_KwE6?~V4YCJg`s znE3SEK@}?$C?-JK>qRBByz>bM=B>t0T_x6s06>{Yd$DEGUvq_en-)8HESD@IslVNE zb_6clw8tpcQ&w8;SdGC-=W2H1m6vtFZJxJhOiro|X{OidpQd9S`lZgcf#w=vSM$1` z1Mhc6bs3k8Av`(obj*k~q=_xMF-oqn$=TZM(Yij{kEq^9{w9rypE#G^vq<2e6EBl| zrQ)-)o&|k+rpp)eP+X60y(IyXHgQeG^Fh9T&^3s+$;32&#UXG6d;R~kjAXQHia=4v zC)4n5GeQHci|Y3RBs-omA$lI9BV(WYw{QB(FLKF_8t+B9Q{3c6G8OLJ{mCPFMd?&j z&N%%`%J6HfNVCdWH{RgD+Ed?C2!$LvA5dw_zxvOfT6Fnq;f0X4Z4!!WVfs)fr0Y;? z%zk5EkJvQUzNfX3@|ZqH4XEBdf^_fcZqVS4vBJd!R-H03>~}!q&?G{XU2<kxqKfIU zTT*dzu=w#t`Jpj7VDB&KhF$%B?iprl*@+xQtQ-9r>ir-?w=tjT2qbpLY9jM6Vd0q5 z87K0Ar+pS<Hh#ws6I33ux)pkJ3|i-<t5jtozr2I_afHGWCgehrvxTN@%jsR`mkRMa zyAFrb&vTXWU23r&R7$j1@)59{vg@{zA(`5a(mOXr_L|L=#T^u#?02_!FuP;PE48&3 zqT$*yHat766x;bf*YQ;W{>B>0l-hU%NXx@sMrT=|9)iB*qUV1KJz8}m>3jX?3&G}d z2~$YQE4dst`iQ}t;Ewq2H9Kp8$Jk}f21NMnk@;3$i_z#&mS<{pVY#}pq7yq`C^fke zjyK0;ZO`5WjCDo%8+Of4s-T~zU0#(~#S5+W)Qox&C-4yugbjHcpNF^?)hb+LNC*x$ zb{7_P=IqK#lMcA(+VP!ihHCA#uCb?_Y?eapQ1@EC9}uKxzbxsY?>;SWV0I<CHArL> zQ51Hs&^o`9dsOj{-A=6dtV8oLVH~%db?Y8CeS!r~G2OK-7cU=-#Y~ONR5{y_q?Xe* zQTpH9=mMuo*xeZ;wOt|mnYCRpH0=7EXH&_WJp{=4G7s+Ud*VSgu?^>$jM-3482C6; za`e5#@pzv9cz*lO>T=#5u-&|E61QWbdD3!1+CA8)g~XE3v80s~I`Na<Zp9Gdgn@wQ zZ9RKfbw6lMU-$-a;rtVdHc`?)++Fl^Y=_zYS%2%IcKHjM)awGe_{=Aqk3*rEW?;eB zWWsnovEUw$7<tN_{qBnj_YK4rdZj4S>anO}$p}~+`9(&4Jbr5iRU+U0vZa#ZJzI7{ z^7QD&nYXCWUl%O<AS02C<MX}fqR%G8aNllO*d(Q8u&cYwpMndqB<4%MYO7x9)OOD7 z^vSeY@N_5Q>@>lsGK#}`#3gF@WT#|gKyWD3W+Kjn@W>V7bW#A<n+rgOwQs<i{N>;^ z1uauG4pYZf$k_l&xkCg@y1OUJ?%0pSGVpEcWrT~3b{$FPG*3>sWSn6CM$eE3lIzxO zxp0q%r_xB;92dSsTBX&n2OGO&Byh^I41e*gf!k0&OSQ*N=1%DL8;a=BBUHs@8pj5I zY;|tL``Asb$PFpdWuu4nA`C!W<{EE?j-7LFzqpB#92^a0C7hNHrzs-~CFQPR1zo8J zYZm>n(?_E0Rsw44gCwyD*fF>v9JVOD?6z^g8PeK`Yc-bHc|PFqQx>b#PuG|u4@?$_ z!asY)9L_kB-50OcWFFW+V=V*hW)`}p#Ne&7I{$v>Q=Z;%-=F{SPfU^O2l#Onk7~Ou z$9c87wFkv|DNkZ%FPqU?O$VC)lXt-AHTbvgy(J=|>B7ez&UZhp`buVe%F@yB^l0JR z6u(KY88OlN@#6c=Om;{_{G9`xv2TOyZGdu~e?ZLS6*K{wU^B<I_5D-s;V$p7s-d;P zgX<9-bJtWXG>Tb~9+GEe=W|&f%QA3bM)ScJZBC`hX6XL(<dAOXmDLrPgZdUHG|$E0 zwE>i3!8rf)&h53IyS4p)f63tLs}4OEcG?(jb$*Fg$EZ&2;EVfn$q3I@)$}Cp>)tfu zz#qr|eZQfQHWAJjSK}+WI-A>q%JEj;E2Dy&pUJpqObN=YFJ5zj3DLxo9a)toj%Cfd z&Y5Ti(T)j9xcg+28#iD-=rzN+qTgzIVKF-~$=Nph&g6GRx+*UJwb=QWDwf3hy-AhD z<tHE>Xb54$nXa9cB>jA4waCiWLd-RI;8$*3zwR`)C2xP_WPdkY2}{N}=fX>Dj-PIc z&gVA7ucC*zE_4Tv>TpZ*Nbu-r^SEuI2(`E*Y0?QYPg-6hz6x$Iy;|2KD&U8H>$dC? zXai~a=SvyEaKUM&l{cMF`Aqa|Q#Tt6atj$xTZwDRDP=3e9Slb78Zb>oQ>q<cF{E?d zl?_`lmje8hKyDT>BG6j)cNGT;i<l0p7=(KFqn63t@B~}QY1MGjswqh>j1$@-dy>WZ zRBjuILh5bamh8TFoJF-HE;5vi${De#N9zacU1eD|OE&#oyv@WWPgux9RCiQ&>fZe# zUC*iFnQHO%gLG>6tg?I;@df-Y!MqEawg)bnoet~9X6jCzoE-F>(6%ur%W-f@*(ek~ z;6=A110{9^=dRf}=%^UI6aqV6>8mpniCHykV^m72Of+*FO|hC;vDlF3ez%41ST8KD zg`9M<8<L+Q?Cf4iSh#7PJVyIrVd=A|%eNVy<)2?L^k|pYdA6-aAgcP4J>g*+#V_MD zG?4qgoDRt{;O4KQB!gGwsWWA+58cPonrTF&s?SvT!h{q5kcyaPww)LeqdyZs4#*&5 z+C->MzJ3}UxKzJyQ5AYOZZ&lujK9`D`dXOvSHKCrmf!{B+qLrlQR0-eNk400B=a&Q zBt|RQ__U5Y7>n1Db(x%lw#l2Qky+liJLP-EQ#k~7jWM|2tx%-W{lXNt<ZQ^{mt|Mg z-VRt;e^zYa%>}i6<Jb<wXF1$o{DPh7VOuN%DKc7l62S8z!P6nb6UyRtCi6G_hK2hi zrh)r<7K;b_V;hwjUOiZ(tcyf~3g?>j{Cv6WDz{3AmWM#0Q$u2lhe{X*yY_XMf{np0 zbl6?N<J7S+DQ5V=Vx@bhrhN%=4jNNj#u>4QqQ`1r1XjHFH?L9KAj;|!;dQuRfqP8J zK4HqX&*jyngBtmJ909*(6|_7g%O;97)%n`kP4)$&rTct1((yMYb5s7cQ8k~|7^ib> z*Z8J}5kmtbuv3o9SJ&6+?$LRfCi_s{h27)qQ>U6u!X{R})3c}9+077&V}W`I$sG6H z9*#oI=R@9IE|S+;+xE5DqV2dMtqtY1t-xlUs5iRT=WXJCi^U(w>z%E#P##rBvw0-2 zUnsfprv=>*jz9^g5hrl#!`;#@n)2YKgTuIkt8ER|BJXB{N*%RCtJty!@luzZX7(eN z&9v78Xeyj#Rs_t}#}kT{wpn6uxpOWA)awY_HPb$$Db-R*(m=Sv#SsEt=sm_fNAtQq zOw_2F(C!^@g%0cIqt#_rt%NVzxKi;fax2azaq86Y<vI2B%l-bJJSrmL<gH37EK^fI zG&pYRz?&3?Mz)5JZ1KB~nfnnXYx|4Y0)H{1KCK|({iC|wkHW*Tejg}~3b4rzn-Cj2 z2=mQHO1|h}5$=(+g-|zCH0)L^Nno2b``JTOgv;LF$9~uTe=Pv8kSc4e*STZAiTcrc zrpZVk9k2%6wJw$MAgsORGZJ(61m{r_5#T7as_#)>7P5-s6$dgHTk>%K&6rz`vzuS^ zzw+oqY-x$!{&xq_ycE$9{<Ur)M!FB)_T!`7^2+dS{$=ALClQW|6rE_9p>C(mAS)QC z*b0zq6OKHTjj5mY|B(#E9MonY_PYiP73o{Oe`TEZQ(At@1NK&KC%PX%_f|{^U$&~4 zRt!eP36W`Yvc8QSI~vY}4eP(6X?|8Frd^Bot)#9eZ^$4Eb3g^2J#t{1(BDea+C?!y zO2#zvYrsyt-=y-b0ZG7??v!W6nYUEFvoQ_)eA-a819C>iFv5vT>u0U)Q@;44e#v`9 z?8Lqvt-Q|b!gXJhbF(ffXMQnx!RePk3iR1+_ekE9-#mBj!WRR&b28q;V20PNzSRS| zh*xvbV6O#QJeP53C-llsySxdpWOZ&0M}3%`b3)Se8`;C!mGsS`k<N7wdpqeVr<()0 z{<VTKqb@`t-Zpl;cy(_g(*LU|35OHZ8Dy)vVY4!Ich0q_ZWr~a37r#md&F6R=ow|K za*eH{tjWwKP9|}h7)D|U_|$BmC!Ile`fPU!S_fJ4f==-llI7`dG7+lkm7r;28QIvt z@8h926p`IoO(o1p>NRV596oF;IT_99_1|M;KlPr?Shu}A<2zcV6t~A2-u|Rk#ti|a z?3En|??vb39CZ0DZ_<uYJqj+Y>@Ub6Nz9fgT{`<c1?K1>n(fcE_auR&vIZcHO+WV1 zxTI%#m-r^%ON<i2N(rPPDQ_G8>k3hDo$?Slrx%-MD5h`idN1r%dq`=F3Fmjo)HbU_ z2+gcrzCB9LDZ$Q?<%g>kN;iu-$tOW%WqXy%K^k|ej`5~cY#<lI`%Hmi>iG*cwjCo& znISP#-gTdA_db#jLu0McNPgbwrdUnbBwQ)1<8Qe49J1L)x&Y<F9Mu)+?_$nYIJGZP z`)lgEqPokly6P|##gh6n2x8`&s_u~4SwpSeuanq~4M0cKwik6+>0P1V)G)Z2TyPO? zX3Gu(oRcNp$7Z+e6b_xePMx>uhPzVjAm4U54K7zHvJN=CHcB{g)D&`Vh-{OYG)zE% zVZ$t!IgB6mTJ1ruz$R-(CRRC0P)-g`$L1oF+}Y}h1EOh82?(or7#i<^<~w@YRLvtj zBOygoj4#J4^>;I!Ss~_glAA$SPVQrQUs4TTI5Ap%x<6lmK|i&7;tu4aGj@^ZG=rRm zv_6g+wf}QqVs{k&kx%Bx{(IqzqvO51oujT<FT~u<=wqhEBRkZ!coq$xxLHjcYN3lt zh@%B4(HHuhfnV>>P|g!?-|wxtR@zc_#{X1|or@;BcKCq%%JWix8;!!}Dc+u?>wHWY zP(Mu9hzpK>RMqj#qQh?1<Y9(V60%)m?5NyeAQQ!D(8VKD(wQ(LK8iuiuzhaPUr`Q{ zZV}1a@k-z|sQGpomb5dPZdLkEQo{YqREuXt>Yh%Xr*GA_c0fOTPp*BTlS}z<<6&}v zz<l+3bQ$q*EVPm6MCXz|#EUKYZ0sKl<3D$eAzaRZMZSblI19r$vOP9EKZ$Q}tH5Sl zb-wxcE0}2?3~9gCRuoBi(hhU$76nVK(&JiY2I+PtR);g*j@G|*=!89md496MW9U>$ zLiR+sod_ZSDs<%8L9GGS8|^jxsOq+`8rOYma>>nVNyd$pvAe{z6Wo5G7(!Jv#1nMB zc7DOO@ST&n|Ik<Ssd-*KGA{PNhTOIU7z7jO(IpgJSpEF)HmD~{RgJBFdc#8<++gk@ zk})L|pV_JhH()&$hwmRYhwwxy1}X;r6rAq*S=qst<_NrdSN?DE%m-}}d5>yysh%W! zgd8r;lkbLt!295G;pLW+>7+jSjM6o7EMC<_#K`dVmn9St85_s7iTYk8wQfzWm8c{( z!LHDu6ia4Sn9YP!icJH)kMdU~6NAS@jvH2q`;I(cY*$GhDO;;$6KwZx@$MdtXad|h zz>&xUz72oQ#77}E72wY>O**FSZK7r6@!dH}nxuP-I@K!PlF(vFBv%#QC9eY-182$F z#6@pXISM&gi!Ji`I@V6w6NQGG5Ul3ioBYY=0$^*iG&vMPeF{sRrIojqk@DAz)!T`~ zqa4$_i2d1>Kh5ZeABz_3<t41|*SM~y+_lp>P6A$>r#7NO&l^aO-NK3WeCzQ7vuHPV zG95uZFr4oC;F{DTc;k&zLZz{Q*w35I?8RkAqME*YbE$07g?e1F*JMzKMPg@s5Si-w zp)z{dr*Z+|4|eo_Mv9@yi3qFgVA+9)GG|OGLH8?GRJX3w0+1d}Q!znc>Hpqh<H9=d zF`|o4QIBUED!2aL%ea5C?&v*A8c(QpC@XjbXx8*`wM1qo5zKNeWbE`*Gs{xpIpO%R z(F4J!O<JNr)fkJHddMFrP5TB5R~G%QGwjAizRn#By>%GcZ&F5mt&IQpvs!F<)0Tvf z1_mXU#+me2i!1}229#IJYAag~GEx_#hatc8Vgn>jkFzm!8<;U+J?dXVd~Gm-4B_*d zRN-d^_8d^{9v|wy=oBL5tmg~WM0d$}y3(fxtqZ<QY!&sU6B=<KiOgUl#!s>}NG(Qh zr~B(TZXt{#(?&b%0!!OEZ)c)`npu=O2U!?;4MuA8QK0qtmkXwOtgbQi4<bE0TUfu8 z@GZWp%FXHxsuAtB%Rp?yKz<0<H=tPAh$s9eJvMz1Rmjm^E}tD!bXksFP`qb7D5-0Y z^G=<UyZNYKvSUiBUyUGXeZCrV_bAfRQ_YO&5xe=gFRtv3J5>tEd^+Kc&N$xV46BlI zL`A9e$e$oLgZ?AhWr~u&AMfRNZgo}?uGNV_b>?zvDRA5Fk_`ZP0P#o&>|1SaPi*}u zliT((NeCqWzhb@UVOBp-|8%5We#$eL<ccPw!3Q2K*G$uTmGqs9ia)<xNi_G*DGQOA zooe1HVkc;r>P}YUA<rN}w`MQ3Povhut9%9%Vt1T8{lsf)x`$$&_S^YS<Eq;W;7&{- zQ4f*isoWTwxp1G7dcFRKHgT6=7=~3C5d*TQE}YtCHi6URj~~&1_B-Nvchxg&?8`2A z&#|4Zvm3j2ZV_`B(-`A9W;sP%1~*Ycse2e4(Y>2e{AsdU5?<5RwXGGDM{SF&#l~30 z9=)l+I=M${!g6(yf43`(Cr7g^{nC6ru>~ldor5m+#3|lS6fhUkyIo=FHMO#y=mwUt zow|)lN~yDoP+9li^yHi;^(28wT~NL~rxu<yZoQf>UIa?#W&Qrl?_U{atmD3Zw<)&y zby=ZdYk>0Xj8FUPJvzLP_y>wRK%4%SlmlYOkSA7$?qGXPH!?-6Iku{E_$x<S(3w3g zgdIJBJtzh-Ey{xgr~01Msbb(PO>`s1=a=w&a=v|%#xvC(Qi=Ddi()9v7}b3ZCJ+}v zjWQ^-+9t$Ev%2A7l09nasUbKx2l|@92CpdDpjZyVaX#D~+Mmy>Hbbd99C6+`d!ePo zZr`k*!Z25qqustE@_0jjHVZndbn)o$h4&hAsw=S7_P+eFY`+xCKFh$Ye_&0lonri9 z+ixQ(S^4?-EGhm`HdPJJR7VpT;08OhhHwdbvIxPHH4Hsis9bKY36*Q*zR^|zCCDG0 znmhtRPoKMdG)(0Sv0@z&3|yzRl4S4fUU825NT^bi9~5^V#=gE}m7Hxd@9fS;0&D?o z42;|P7B40iSCP$4h5HLx=cgTu1L$+*G{wXWzJEMbHY#Zh`2f`7{&1@7e8ajzOTf-F z_vFYg&KPkpszyprz^DJt@m;5{l<ykK2iMr<RrxCU<p#88z&qgQ(v1#PkBY$>bJ9xY z&mxnv7n6jI4~=(-ik|roqCsD3R?>IRBQsS!Q0)vwib;r-t9%X^&kIhoS$F5CpSh7y zHkhPWU*J>!c=UEp>VYOb_vfGES?z03rA&R8VRK&0=#APxNXnmU^{Rk0?^FtcnXMc$ zV3i<;VaMFHdFPZmO%RvQ9@l+cjT5+ekwY&FMwKYH<_|3-V6@B-S<GBR@?BB^#oI(d z$iw`m66mQ4)BM<HsrY5MX#4%@UHh_cPK{1i!h^C*=gJSL>C75ZI#tzwgijJkt|+5w zqgp~bM*3E2&K)*?Wb2|<7EaMS&mB&Q*52}3gKJPa=<<jEwRv_ahBf5utryp>T*W(A zOlv&+M5)|Hwl;(?P5+z$5BrDeSLIQkut)xL)HdQq4WdL%=&+)dXy-ssdwtkBv3G#j zO6XLUFwihu6p6$rZ`CkUyiR2gje0ED*jR$h1EYl;(~T*KZn9|^tohf#bpoX$6A5`j z5h`qEgwbB`{37pr=Sx3K8KMilK#unA?Moc6UNm<P;u>y@Ub!N8P{Y3PPcR}Fcj7$q z*7T_FO;~mgs%WeiR%2cIUGd6UCUjIG68B8{XKE91kR!LFFUn^sZab$%K))<$W;D2( zJqJb~IV_L#S8hH=9-071Y3EB5Z^GBlyRRRUrk>^5b-Mk6p<`x7-=svxU~aZ!u8xsn z{xWEOrq=UPh-<-A{BR*4#DK&Lm_h#ReQ2#T$&bEP=YQG3DvP&<j%wJeq?s_adz#-t zF0aOf-l7bOwnvAbUm|ST?J$wfB8v%??2oUdEI1y%z8c7~geG<7Incd*YqT#H<X+nx zszw_;;Od-?pdO@^Ztxn3-%QgG;khtZb#h$ITUPT@=uW^XR+}YrnP%RXSw%6Y*X<MP z)WqHDR;t)Nn}z$Q9>%U~cn{Z5)S8o?2!}hJKz*U=<**Ad25gcEb3d#)gBn7mmtbbg zI*hz9OqE2BMF~OCDwvAovY(Ki)c)+TGbk3?gm4K}e>&?)Dd8QzEu%l<ZrwGsBgsnP z`sEQ+s4MZ#V#<tC#iPr^?+M#B-4J4khV;e44$-MLRV(RQ4kNo>{HdLGqM8)2s6rVh z<n!=xX|k1Ho%%0Fu0flA&!CwRu}Q7bp@ElVlZm(pBF%Ob)G-_W&ZIrBmyOdXR;(i; z^bZ>F`9ILFujZA+Tu;tQ$>9nq0fdTG*qiCQx&r<oE}Tvr+-CgOsVsp)Gm%yQr2g76 zYZ|f9os|fy&H8#|#vckjQmIEvkozAxmOO)tx68)Xg*5f)(JUTkIY}CUQD^+dmSTp* zPl|R+UWUuXnVb~Ohj}t3={7QY4iK?g3OJn#9iieNc5&#gcAuD>lo6*U3MsvaQd~9{ zPD&?akukc!oN|devFAHW{!GLJXF1A0PAXRMy9EF;_h`>IEM_<Mw7$YWaMKpb!a>pO zqg|Vjha>GyVeO0UMW~^<E7Z?6vtsR2X}}zW!I|&f*Pu60caX$?RZAXI5Ap0WB3EbG zvfB-YP1DM`vu=Y!7odfltaCM|eWgwXSI%EtpGZK+ZhjlkXPdU22FqAn{)CUrnTWAD z%5ZsLEew+BYvO&Qkzc8o!2%JohaCb16FnbSi)H(8EZ1zfuM1PSrZDbieFEMWVWtMl zS~^SzXoo3`4&jt@D%f-SCET^A3U~?rzwIcNzxw+LSE6UD?Udl*9lxyrU`=^CN^sBn zu&5f|<)O);oqba2T(2z`_NYRu0Uy%pXzKam?!-v#3xF(Y<}xz<V3fTNEm;7lA&D6L z#1!j-Qp7n>U*wT4$E{H}WScLxH#{$Pt(&jTFF2Z!IYYrnrfK0X3_^>ZPup<tpc&=E zVyox11@twUih4*we$0<VDr9$blhdH)6r**;z0p8uJ6bGEUt8Cg=Nq-4A<ovI)DyJr zE)bj?py1>s*cHMua<w%PO={!CyUSdL@FZyDfs;_4buJu{{r+RJqCuy^t$k7Jzr<L- zzTi_}nQ^a(_f!MByA=IkMxLU;T)y1r<6QLI(y3Sd`PX#st+CFkM_$7ElJV+^i&l&f ztSU}g!t8tZ^jCK4CQb!EjB&&Io8CIjrsdyXxja}<`@At3dR2@M!6SDd5Y17+MY;82 z-HUI3HQab;%_*-wvuf_GiPs%X5@@bgfBEE9LxZZFgu8(X#l1n>a!Y^kH!VvlxUP-o zncl|az>5={=%Ba3a&a$Kx))6k|M^|Y|M5v&2LENq{$@v<;-;Bmputk=uV=DoR1c~L ziZVFbdz91uBmN@Tay<?y^}|Z$?ygz&@97j<D^Y`qnMH53MEdEVs_zzL-zIiFYks%F z)8WGtnbHoBlqUkw6a$bPm<Lq8g=n>@7P5Vq`#H0jiZe|yu-czyXXcYh0g^M+&CUXN z`mrcnk_b6tWU<J%;cT*Jt2y0<g2r4ro-9BPB94)iye%LJApzFs-vTQ=F2V!ov+t(^ zDRg*DZ75;xeD_SPY<B_t?i`#iY~i=0j{^L^wyri}C42=6fn+-E#*R%<mk3#H4GJ)@ zI;5%dT>I3hu6p<+t{uyWcv^kXIcaz>M_vb(YTq_LVaZQP)sa7&*0*AYo{`|9MbjG` zMTU4~)<gfcOps_a6uM~P(&*$=0*><B6#A+4_hNsqI%w?{xZcT3&8r~y_IFlLhY2zN z0dhR}ze5u4Gcxq;tj{#x@*b{w@^o_8$Z@VJQt~qo)kB?lV!eB%->k|1c8utud-IyX zj(PZp;|N_>8NBJN{H`AULG{TxgHI((`y0Lw=Ufd}O68WG(ILDs4>KbZtF?ltcaC5z zOIE$o-Q6IA=HKD48uc{Hc{gD%0;t&Ia!C9n5$aCcB7H6#Q)I2_dNgyHYGugI^n}{V z2)mQAOzdQL2rsTArGT9>!fy=()rh$`ZxJpk{SQLp(fg0;D(|m9CBZYS2(+0q()<{< zt#*$b8UQ!8%hn`WTBRwnPy5~y_8v62Uie8qm8}-9t+sj2JtXp)O5|kqkJCCE*9sDb zc;9Gl+|jkyU(deTp;qe4QEyeMouk0M;W#Istj&&|wlyje*ldWDxCN^_edfyZ)I!yV zU~jbTXbsx*TXS=_wPcuy4`Mx=Z3a+c5>*8N5eNkH6W5Ea@pKy?scs||PxoxxTKSqE zXG`42H{ey$%e>GxOw@_*c#E$b?|F~(pG&Co?+ee*!tIN?i(&{b$Tnd*ctIa!m}}O4 z3E4B_B%fJYpazrN*fV!>WtL2TvaeNR+}JQUZtp%GUABqaA|r;)S);9+!PNgTGy#7A z=kospoXhQG^vY>NkmK#;?>Sd~_D(0Pjh<gtGO9?t{_zbP0(O%_)7vH>G+jH@)o^F` zQ2*%o^nHhf3FpfUTV2tbZ`pshtJ1+{@jtZ0rl<zl>j@D;a%NE5@8AZfPoq%;Tm>8* zF4xVR%{ScbzUG|r_-wWpXLMCnz_G=yjjJZ5s0e57O&OhSXq7*7@A&Hx3*M>g5v>{Q zT<zpdBA<&pO@xvv1s0o@Y+NCotDijxD$}{UssWb7aBqO=1aJ78#Hzz44|R9qR-K)k zAgvG!zfb*L_Ir~!D}C8Ye#Pn5)o_Z<<lzW9HVfcqny50$XX9s65Q2%n<3lAQ)YnUT z(Xa;~(#`mRejHU&H@fQkmw2<9d!L^B?ufB>R&I(O^!V9gz&SaY-K3~y_>;!Yp|Q}% zQa7IuOGpZ+<`9lOLadlBQ|)i)YHFiX^}E_oC|~#j^h2!NyvE>|&W(A!d8o#TaS%e! z-L)55&aH@uCRVqr!sLU$oqOMdCb9F}KN;;3bN5(}fsMthr&o8|f_+Qe(a24oDkoon z$_g+6m`}gZvBfygZS{W0uJb=LG`sdgiRU<7mf)?+`mH1_mVKMKF0$)Pq)=PE(x(ya z^D{CW1|u>*`63Gb62D2P@g?18kNx<Sk9lEd2o`B9MqV9dta?v9`rHO4`g2V$1c4;x zPJic*j$>J3mr>pzx6XfP?17&AqJIb-G+&qH`DR$lwPe}b<Tc8h;rjMt)tPtIXA%HH zQBY>bN0ck-1Gq-;yqm)q@K@k`IL6&wD7+ECa8YCPM%u5X9`2~$(9YWY@m^mn7xOKl zlgIr-q5EaSK5Ez8K)9Wa<teyLkZ=7T$hQ}|r^=S4kQTTEk+69dmK>&r^IwKfYQh5z zKu3<#zCtHAq<Vv>Z%9;|^H@91nY%~$D6c6JD|<;S*^bvm2klsZ5!h*BC*5z^ucEkb zF-NEb>~K2sS496wdHZ*d9*r*R4NrcsL3V!iTQ6c2&z_>GaikjDC3)lA)QGFUn<w_f zuBd8qN)){QRlCD_Un+r4zfu0}@AV`C&M%ha@}^Jk;PQqkwVC<<bP0Ow*^BHpqty$4 zI*l^`4n)@Kj`{upqm}bOzq*UM?l5c<<if)e)dK2Me}pFp6gBA{oCTC9gS~F2f_!U; zCqDyn!4;L`jZvjDOEFv_a;onKYqM;$uaq&TJDzuKNTjuBot>G9P=A_>|6mX(e?Oru zJX&PK)RRx>sX1CfMih_8BuH+e>bt1m^y9mv{+QY`U;uL7h7#7|-2e*n*qdyIMmCu= z5o)$RqSd1jyGhl$K}7OE?YFrsU5BlbAG>y)#G8cnO%<$U-(FDx+>ar)VLTP<O_JY( zubr6Ay*)14k~gTtn@ZMVb={don}Gm2fHU0B#hKGPYEK&Cg7^1Jj$6Tyw&|~1BzPry zQ=27t;+~*^RIFe{{C&P3ukQ=ZYYtLm%tYK!`vUJ>f$iW2s^eahWjw5S9j-E~c`b1@ zwm@X;8qD8vceL$8{U$yOczRODmRUs3=$HBAYn^3?y6?SqF}ObL<RQ^t!W_=ZRDB<M zjSO4(CVDOY6{y9DbETJ6P_JfXy=tzZ^~|y{F>J+R-8rMon`{UVg_<M;G{huLcf)$n zT~yVLcZ5mK0J%lXXy->xtda)eRa@RNKTj!A>&z~75^}0;x&tcpS*6nHIlLM;fd*{n z3{hAH;<hq}rec2wc+ao#acPZiEd&tnj<QrMv)$4V#ki3N`?DH0AgBJQe4?TvqsePe zGf|bJdA(mGUBn-+0FZ&g+c24C5%z&HX_Pp<YG_x!?@SC|JOHm);oj^)OnF{)kHNJt z`u4hkJVpj{-2tOLw#GuI%#rgYV<oJ^^x|GCzDX<a7CwyYYqh~;hfiz;tyNoit8o1A z(+RA9J9p?RZNJH@U1&#yTy(R_zdM~@jcfRE;~?Y3$!RXfo1GKVvV^iY<EarB_g8(7 zEfEhnUorQ@=HK+$C2z+5+Kt4z&j-5uy%Aa*1CL!h9$cyKo|f=0;f@zwyyD$e0JUs* zmsy{MQH>?-eRjsWV>X^0>aILiPeqwp(KPU$JM$^I1m9w=KKMx0w*<s(uj_2@^L5Q* zk&$*CRomxjk@Ob3n)7QN^-N#Lx4qd|+1FZ0*)hX+IT{?At=$8jtEwf&D#Y%!)^`j@ z4fZZ2PIcNzj(T_L3%I(4cMvNK^oDiETIm4j=HmOOb*=qBu5^x#A@f<27)^8KPju)! zGE<Q9Xz5l>brXumqG5b}1^(Re74f{y&F@+cL8~slS+QiPs?qsH`PZfEHH?uk;$KF% zHvlZ4`W~Q%{L=JDOGG6C*av!S1=YabQr~-p|L{eITPMAF<6Fm!`HP#N3zJ)f0<jGZ z@pjZAYznBiiq^7zCCZE>(~4Lze^Ap|TXv{KoYo<<RV;PNcdvup$F5CVF``xve97k2 zydVHrvt$aLAo(54H;UViYYv<I39XlpB=iG1y%_e@4&Y#uX<BZoCHa+8wfN<GpD6$A z@opuk{vl9p>j4D=d3a^87!gJzGFM8RNW`(tl+!%cL>ng>6L?UoA$oN<P=FXiDaS4) z&(HMK+b$;A;dKk!q)}Q%f^nIWWFY^l%lScHUE^4glwqr2`Sn5@XLD054BU35{mosG zwHLD)df<5|y8xJ_nr-19r_=_~o1$5)@=lx{8MRbGJs3bVMNb}AO*6cU3YIOeocMMr z8s~oo1n?|(UVV+NnIu_}ESJrsy6T|BwVsk71kltP9VCV*C+kUI?SnEtK|K%G)bG>g zlz$Gww|$rN58>)7CUhtoxfi`vJmMX`4ZJ#*J><Jrv}TwT<=HQvcu9p1s(W5(K$49m z)7_T}!n$v7HbC{>%q)`!JfQhOuo%#-xepK%y;mr(-ocXIdAELRvEkXw-J+1UBPHu0 ze;Qi&-5D`jux$^Y^UTI`$kDGpK9S-nRF5Ib0Q(k*Kuf8OIYqXfZz{h>D)*w=tctOA zaAi+NbNIN2c(C|$tW@^^<cMZ{N1RQu(Dl6iuAed2H{$cd4y(SMyVV5i3|FM@TN6S( zH8yt@G>UpxQHO!aaOC(A_usXQ3$PxT%3N5rK{67D+EbueE=l(FRMLC_Edu=1T9l5J z)VX<;a)<+Yv~4@4&lo^y_?Eb8Tr(ZMCwAHPxn2q<bj>ATS$s0kR&(x>tzusAQ~c2A zG4GyAxZkV3_9N>BC<M-NSq**`J9u9>m0{DGL%yc~=_^?9Ip5=)FMGg&1{QA>`Ire} zs^Mh>6OF7eSy)i<vJy<JHuTVVe3fMYvmRUn7RGr3RYh%B&j0-f*A8-!Ii7(mn|Z=Y z^L3Z146+B`+SC4H3zU~JbcCbddKWf?)V2D21vqq053{2cyK^_YvdEK3%XfLn_A{i3 zXZn`{RFA<vNCA-=;<N78-N4*`vy#-ykC7}dr0I3zK|fZ^S8$-OuO%c)Z|zF9oAehL zaGn0~|7!u<Redzpz5$&G+^>KV4?$GV02+i_$wl~2uprA_n{8oQS6TZx_*8RITR=sZ z+TlMkxEfsuAq2Bk)K;Nack7RIr(G&)93{EZ379i)GS%tP>fyDzu_k-7K9fh1B~7_w zPqsol00G3ZKl<2vgos*2ANG+t%$6TVV#ZAN!U(L{&;QGW7=Jvug9qhXW_xlDB&r~$ z-{bp2IgM8I)^5e?6_cNqvwim78!c~PI+LTTzo46~lb6qyB4<EhoYAwt6VoH9_2SW< zFn1ZpOX1;P(FD_vcSTHxwF*;FPQoWbY5daH%&H$XXagD@r1Uh=8lLeo+!^V|2e%^S zegB!TCd7!?>0#rDznza<bS01|(_VW3>nynl2HZ`cvol1b#;j1tU}sW+8aUL<yZf33 zcI5Ubn*Ry_)X!C3DB3Hrmq}?+3v7X^a;k~-sMRC|e(LljWQ)0kW0}|$D0u2!G*@4g zrkk+_sIq9EYnAXpx-4(z&V=&w7D?J|+t1<~K~kVwS%ER1z#*=*@Hp!3kV72p`v@3A z;V8NuR}S4A-$QyQrLGss0>y;tLHZ;xdhX4X@ERQNTv5uyVH#XNa4A1F!EuYG6y+@L za%JZ$3f?MCcqGJ?&&pYLRp726SH!8ip3O$MU{LKogBMqg({j*fd(GD}-IPnojd;LO zX1y9T?W_0Pkm@ZKXCfXJok}0ozEit&TU3bE;3ItPfrLH?Cpd4|sem#Ag_RKn=R@K# zSY*%V8#Ww*9$I<R9w1H|BBYP$D`gGI(4;zwgIQ|ZY}%RiH>xJ>JNj39{1+IppyT4O z=;qe?8#`epn-!UbH?gX8lh&LO)yk9m)dgftHR;mXx#dDKgG^Jq=Elz(7z5QU=p)L{ zsmaAOdyX6-)RnN4LeX#fK941ohA%sRU_YpfjHvk|DJ{2~j~`G67a_AiSs-U;o;v>9 z-}#IhQr`<@zj9KD@<-#rd|#=rzlLr|_)_o!kGa8<zMSuosfCytR60?GW70oxy*C*2 zG<{Qb1CDvR{&vTS^8l%;ITR!8(%1@{?pBBsaZv{vj4kNxJEk@O+_m`_#uEyBU?H87 z1|BpiKVqitrEt|_?Rqrs@v50`fDiQ)5lja_I1D5O0U&Pu$mU0^7EN^dw!`C3lgB=k zeK4K3?_?0?bvPQ^GDN>Tq`p3?6#rwX`y+F5%o~6@W*gc!sO92>CQMt)Mr`dF_?DV0 znxj3qyP?^;T@qCogMGG|;#)MxPOD*({}hrG;N9*dV+=B%1T;!YjL$youX<x^Zhc;B zAH{RWBYPg@&(d4uS%jx31_&^V$5a{51xh^P0SAc<I8(8WSGKMOhnEs;TiR6KZf_=5 zLA8D2gunEgj%YOV%(gMlG8_C`lpq+sqPCPSPjVhPyy%8;Yc;Lu_Xy+NVNc?40HEN; zcC2Y)5$g=XW)S9SP%Q;pBN*@e|IzgB@k}@F|M-x@8#n51avI8gmrzI{3zM=w-5n^5 zm_rV8J0`XzXDSk#?h?`*au+$za@rh=tsGOv9A?Tfc5ujs8NZkJ`}ck9-~IJ^ysp=E zUC-0?G`VhFZw_h^dDjlesc$i4EP+0C<!#T@dJ7hW2`&_)V1OL8C)s@3zk0!_Nn(V2 zB3yG<T|e(9%Fr1Ax$T-Pki|af7e0-}d}_A0qn<22;}{h@DI+()Hve`e=>xRg8!zTp zRkGDA9AHtJ*Bjzy?UE(5+g_S_-B@>z-Xq=(mX8=@iju{x%u~gnaQknvn7fwx+8$De zbvFDWoV=i(CP^jwso`_a<hA?Be*O`<+Qd`JexK)z7w$-5Gk)k`&eWm(Gsk34b#2P` zuZLzM0Yxc}_|xd89_~x%f&iRqmZtSef^xNZQm4=;nWevqpPGIe;t$hFDkUcWJ-nak zJ*<~tk%kY9Yr3ZTDUf&gg1cmj!%_C5-XW!4Ew;Z~R);U#n(B_E4_%JQm@C~1b!0KQ z{?&maQ!0Pu>oa4!*E;>YH8t%MUn8RLy`Y$=YB=+yHGPTowHMLWR7UskyY~6g_y;dn z4$EO6!!1vlw;?W^>*fQSad*KP^CF+bLqTH6nH)b>e6`2*{0Xbh0nP4skN>H8@J#WK z@d9;S{;3Yy+anUEj!%_VzWGoPt{`lQXl#_zxu`@dY+UTvmI<vj>cCZX`5S^$?dR<i zUIf?<3(9_@ORVU9dedHFjLm5l+$bGu%fq&|iffe{r5tF;-76I@m1j1O3y^%D2<4@W zk2Jzr07-khV~lkbIgJ`)EW*pVLOuESJp#7q#tQ+vRV#J?hv*3ZWl6zQjMJn4Y+iR3 zTzRjRXJ)F*8F?B%ZwZ`Lcs}94!`5K1zK9V7O~3wT5GK7PXuSHFkg^zhER5capp{2g z-7>>4d1%48wh>3`;__(<v3NqioJdmB($d&c>3^>YQso66wWb;V!831=>ntU>fn}&@ z&Ig(->>#HCmSBpY?Z~E^Ceg}3;ea5)1N1ob5hnrf1caxKWE4CdI9FB1Ux>8nyS)np zFCKEslZ1y<DYQS*uI$QKO#@zk%w-X5w~52pFe4E2h@UN^0re?yr?wVFh!_ne6UboB zM}{Hk1Qm7$XR^^~G3>iwRpW(XItEt{4<M&=V|B_$cNzj)+C?R9g5Xzod4(G%o@fcx z^Muev{}F74kPy*c-PUwX@d$gtJm@tiM>_FJ-ZzWGteJ)D$dgj_&K|DWW1;h7jrjw~ z@zvdr5>Ctgtb#2<wu*LL@+g*Htehz6F=|D;{+RvC$kFgWdTbwN8fN!Uc4>z0lUqLv zC{MSIV_a|kpZDC$Z(_N`;s5Rgz{t|&sy#fvap9ZnOheL3Ab>~BRq%iDZQ{Fr%-g@g ze5XhY1R1;U?i(J%7f-jcIBFS6|9(agA1*yjWb!@sT@3FB$FO~fCaAd**OZ}N-bvei z@j4r~`n17QxP-TrJ&~={s<)Gt>eIebkL|P~gU=>gBcg)H&K6ia?5}4d)0nJyS6owz z-trNut<=TpVf@PT#Q7_eew(BuI2?_V`hk!6Q~?+(RIv9bKa%DDsQy5!ie{^@we_d1 zXTA56%om5;0XRO0V;OJU&imA`|7G=1Z*Wy8ZJn`VUXCp%XY5p4Eg!ZD=+d^6RzoZi z02c&Kk~<=9kQ$>sBV4ED0(Tep>>}-v)|0<!<zad|?ykcczBV~aE<1uog5QW|n!}}! z+JIICyT9Db?p*GQcK|54^q8(NbG)oC!(Fz$0t5+!pP)a22G6IAx$YJ-DbkBhoAimB zLD>Lmch+&dYmB~91nAtXDPAvelZ>e0yNxC(ffQu4CmgdzxM2o}3c%1-MKsJ*QfdrL zVYj?oqUdADOZxS;EG*w(Jx*@A^>?$zI5UCLzdnBU-^2doS?-W`SWWZj;OeSsoS6_E z(%{q9#s_4kWoSg~ass*a?m!{8X)&J|F~FUvT2&2)b4y!k(LiV!BbyxeC;x}*Zpx%2 zK*4wEs)>J+RVt1Q<s|qxjGR1I2D;!NRhNtcY6bK*{UQ^<xyfZ;d`hBgmj)&G!-HyT z*`N;jRv)ojX9SK@n$rtofOzTlDI=xgo&gvGX^C2I0l@9EwBf_1!zPvhusvOTRew2l z=;qIjiN}O%{k*c#lZI5vA{G+^--WD;qXnCKEB|yM8-(k)SC5UW+iCKVfb@vHNs6wg zOQ`4L{(q=0c=L~6kmZ_6ijU?4%lH|S20}2WD^j7~qDPPEKls)PZ__FDzP9#wCRhIS zWRd5CIMqB#2Bx!}Frebe8$H{dOpZ+82oo|-Jn{hK)~hI1(?gCrpZakC5WS%-Fr#cl zpULXps{s69L|Aj!0KZiAf?MLjtF1CxXSe@_IXLj-m-~XB*_412PUh)jFz<sjx@~1f z3Qt7)DOHOge)tHS#cOImNp-4W@$JhF8YAm8=CSF*(lKkaT)y3cB87P59BS)gKp)Mn z@Xk@gq>7Wa@Bcn_=*W<>*jniJ_KpdIA?nBY@#|vyTwnPTpCx9K`Wtj3BzInQ{gADK zd-^WjPiLrQO=Kk0{SZ2sF)wfXk{OM9N{_=ed_wq>pYru(u=|uuzHk-pnU_nXD-LoQ zmCeQQ;X1YHATYi=S?-Pj7h1ekIfQPw+6(b-`<S!+E2!a2_108E&UFV|7J-z5R~&cw zd7#Y`3cCBIn-By{UKNrGsoS?XvChyvN);#hVs}pZHt^L1%hcZ0Mf-JyB94F`*o(P; z`%BV(S_6={?7a=ZCVGdjz@$qs-wGA9Z3(tw7G&bm*`jm=Mf;`vWr0WCTII{OKyFK_ z#AfkW$p?+VUf!n`B!B5#q<6MRukO^A(Qtpi1_gHXS*3(GccKC4EUEH7*+?igb|mgH z=cbtR89}wB2a&KS1aLO4t|GgA9g0>IutNdp4O;)!qID_$Oeip<@euZLbU`3<DV@ex zN@o?d9f)4ce?lCHyY_J)3dz!st|!lzO#@A9%M-c>e1GR5mx-H2zf<UpCyKvf7v^s% zCNythxIcx*6SlLeComDdHV1+8T1xwUiA@&TN*~S@Z2uv>A0zg<cNi)2Wo{AlfBQFX z&oTbvLAuY9jtMrs;Hez(eXdsbIyyhMMK$M^*a~^dlKN`=vKWIw9JEI2BE-HQ9533| z6_&-J*I#v!XlioTP*I=G{~NHI#npkt^z~2vF3=LT9C)I$(Fm+6^5YSV7Sgjd=y#vG z9okB819rVbdZjh-vB<oBaZRPoWtw9tY&k;1y>j?95NQ330%!qIxzXSnp;%_FdLCtj zD%|<?_y(nY5VCfGaZ}8WIBxXe<FG@6@cd@n2Eh&XA`uQ&?}JSvw+{c&w=m0h{M8fl zx<7j~7=GnAAIHqO%9DB!BMsYm66W5@r#Bd8Q=f6j@9uEIi67s722g{b=-G<+|ITQ4 znmXgJsL<!-O)HXWL+53*4qxho`I>s}dnpQ|pL`A})qiqG`cgRaDY7Q7vTs%id33G2 z-k|fvXDjglh&T{rRNSYu{Sv*nUS|?8z`D6jjnu$l7BC<CV1xTh&BR~A+cExs{i4g7 zw>+*ax$|D6L8++CJdGjtThwD>=UctBFQ_NCH;O!vTCrO5H3Gf6g<1yJYoor*8Bf|p zJsGKN4>eW3eQx8H{+);y`W_U0(<kfJ!?Dphuo$n7PoXP`GU>Y}6smhU?9=Gv_w^X? z6MYtr|4CsPTGo(0P&*c4Ve!B{)+Kqs<i1kZqKCm8@)hds&29PnR`FF8v4AnIF<(;t zLOrij-*Cw{&f*Bd09|!<&!(WOT#B*oS8ybyRDe_QNPSYZIFW0f&OL>O9UNlCIV(Do zPE8egeJhsJ!g`5!g@s0}+c)(6?hWJLK_7ZVtaDP=o%Wfmy<fZy0hD*|WDi~tjZTb9 zis{vxXw{t6@@XZt_7#nJuD6N%mKvqkh80bkUZ?GJZ$V>6o-%{5hf*D4Q&w0{J)2mM z0lm)7nY*43%>lJ2yP=Hd!(lVxv$v1_g&J@cl>m5pbyDyODM<|r2t^g69e6jLHnEdI z<!F2h;z=})9CwVj64zwn!4wP$E0G0R(0V*NyD~bETTZN=<DO(;tqs|?+7^ZPCU@0Y zTZhb88Rt4jfW^k+|HNk6gxD~+iFO*2CHBt#8vE6RWMZ&_gtwK@RbiiP!Y2zFhEnj- z3MeWnCgETCgX|=dL1gNV&-$d8B^Q)~l$N}pF*-h9(MkHpS{7D*{vP1{xpv$LBG-nB z6IeJWN*A+xKORzge}+I=*6jY$$Am<sqN@)qjqY?Z4AohWA>h_NO_%nKS_u~QOde<+ zezbFpo~v)=)gkbDl1$e>zrnVI|2pd<yw=?NV>nPsu8_i^ob_3qiv8WKVQ7pOWP*XK zW4PFSoOQ~!^XD^hZTqyx?>soNf5I;tcgIgUbz(4FS<GSSbdZH#yw5smDLfw^`zQqt zF0Ge6OzBUHGv)gwWtwOc1{HnAwJ|)8iM~Voiu#8C*wYP~<v-y@fSR8zqk8uM0%Cid z98gM^v=V+Vv`fjn@)8^g{<FXEy5yq>)#l=$i<)ne-z<@mLQ&f928jQl&%mWT{FL1G zb#1Fjj5nA29qp0dUagxxLz208X~ehYkX!*xq0JO7o%%8Ai^9oWnZP{syUO#Q${C-7 z)qF6L2mFm&R|UOT*U73MX8qY>lwJ+j?V2;XR7K1;!sk=nk^);P#1VJ5L^r0#0;OWD zZud86K(+Ry3eVtNV9;^JjP@LOUS2=RrT=cCq${evpigW1FjZ^#WQ<{Q8A36u-TiTE zEistze)ES%`;1!mGx|SHqQ^~IKIB^s#qrx%w_S?$J-#;nO8z&|y<O3dTtG2;(FK45 z3_o<EGWOJOhyE6OD<o>^Q6*YNS^lfo!<WEQCb{IGX~tp^BS7NKTE)Oar5B^c5k@U? zks2EtQ5!jR@rNEgCXS2gVG=JKl`C(^aEL@q4dy@tcV2Q%2wX~Se%@~C!9bW!Djzw# zEx`HK>0yS!cAO6GEo5=L6(Qrf{cKIkG0O9epWV2+k8%DXr5o1()cN~#m53~C`B^pB zYN6@V@H^mlbBxuR=Busp#5=BJ<F+reIq$9WpnBUBTZ~^@IvPG|;yT_%9(*?n<%}@b z=wsiBETTw3l(kMD7z>=_jgcs=h++J)(b6uYiSk#RGyh&SYt+PIDmvJGxnRd^cXT#> zBYJnr<~$%fJ{d>Z)JWK#-`$=K+Fj6MiZ@GTIeO|nSWtRaNjrb~q>gya(d*zeo5ClO zsjp+nN?t=nvVXh-RgL$yS1ZKK-6awJA}zU&bHtBb-K!_Gx&JSRRcR>h<w=8c)u7<C z3D*!C*Rsu^wMNAp<3S$VR?a?7DnE8%IlM%{2-#}VX2MtJ{2#&GnLMelXwc3e6_kfl z)UEobyq}fGiqlrEXp}DN&$ZAlIC0JEqX&%B64Nx;642U41TXLTih2G0wgYIdAAhC; zGv4~#_U2MIsP?yu?W=E{MMEbVds;E=y~K~fCDXzC?5<|B!z$u1mghUI3?*K36EaEb z7|@i1wbWO%o1IV>S%U5MF?|BFC@vTkd<;9c6eH3pXmXeTgF7cDK5Hs1c9<oWxkq-~ z87co|0m=0IGfWG)w+U$GNgheqp7Sl}CR^X9O_WW7VHi^c`$$^CKN)W5&b}(adRZm{ zOzx|S2&f^y_!zMNA7qCw>Eu&JQ&XTJ`@`eq8d38<lA?DB6*`_2)>XT=#brISpqTVN znYv1goUs@pv|&6=vL4%ctrsCUJYD^i$2e=ZT+`>78?^5TIi^>m(O>fG1uxdN`JxBm z=R#q1#=Awe=}(bQ2-WaYzI7&x0V9qPT~%FbGb`)}N7!7zp&`^<o2rHeklvhL$WU0e zUIbY$*_{)oT{W#}Dz`LQJ0!O~Idoy+L4T(vRcWhTYqg`lPJ9s_{att7MA!P+@yYR5 zDV|Qeul;D6xZ=k>$(%}@`Heku;llqK(`i!5D=^4KY@_!wKmq}YZND&z@QucZ#)fif zO{$r8$K1Pw|Ew5c;Ql~wg09;)_7#8dB_z{zHeF?|*A=O?$@V7JVih~FloH%sd{%KD zRt1|C^MsPPh|Rk$2Ec7>tSLqddV$7$YCK=*-@5ye#~RWy<px)Aw+69<u}X{wPH0-* z_7AoZb4*|a_)>AAA!$S1El+h#DucuD91R;3K-#*55BoK`^M5u`<StMiZBw>KMkAKv zQkd04z`8jZ=O2`Rzo2Zm1p<1rC=8mce!XiEMJnd&qO}AkO!=6Fs_v@EHN65bCZ^v} z%keGadL3agUNQ#H>aiY*cLOHi?dKL7mYjrMO&7b&Qi)Y|f(kJ76>yvq6KWM+(20X@ z4IZe|NU$4HvZub3%N8_kS4@^o)N^rXzw98D!e>mVikG*GA@cdC?>>^Sn`?0gcL7h& z&RAXWcE+{`M@@6qpPdlyZH*23t@{f-{$TMHX}9<BCH(1o!X_yE7%r<yAS24_4gE{1 z^MV)o)U{Yfr!FfdL|H0h)yR}F7^@N+N_Glf0Py~vfHpynADhNjERH@4RCvv_d)jZ9 z5465l_`=cBC%0CdH?qj-;+cJ?WL(C0@~<{0;x^aEH`(;WT)5IH6VH8iEjlKc0#4-L z7`L-AX=DU|;vdEb6K$=*M<ylTQ$n0-ERzRNkT<9-aSlc9ys7!ih0<8i%r~*ktg&8P z?H=lc?-pPA`>hFR^xzZ71%#2)=|^8B(fbmPOi?3$of$oKDME_={g!+T>dSKpcNro& zVN^Ca_veex_PKIm7xwKk`d_&oyNbtWX{e^B#z(-u&6P(wuA@moBo&keW2}y~J$WKs zgF-#kj!_;l0iDu67u>*bSm`eOlr<1jCI8?;&R|uZ*rv`@ftUg6&O%vLRNpE4*!5*G z_haXUPSHh*uXnFVNm<!3Pc365>-|lEp>EDixgqD6sp8nN^DGzOTmvflYqzJd9jS_1 zpO1CLg2uIODXsmv6xl3Zs2}aSo(OjB&)qiT3q5}0<0Cp(@+n#OJZ71J*9B+%<9dTN zc|on)vxc7e=b~0ixqY~ry=T&(Y_7ff_`zPsq6Ld>H?)ol3{>SiG$Wl98ft?s7EFy9 z^uA8gPMZY8PMO)32esC#G2bSRlSUJ>fEZedRQn<Y$v<u?*69<&OdY2Yp9=uB!fjz` z{4oWvD5NE@wd-YrgtntiMS!((0H6u;K`^o8WkYKE=ohjUsor)R8#G^c+iYCm0&!#= zH%kB9CYMBNs`?(i-XS6T=^>yYt?mM#piA^d8o7_k!8Lbn+#GI*Iw-_axxYM*VU5C| zQ6*N9HptyEG+a`bF04ZWjdA=l1=3VKeA18QI5K&rHKL$um;fojV!J|*@rR;kA;j`? zzWs~bLskN<*k8sAMfsc7fYwI++s1o;b-SCh3OCT&Dw3^{ko|L;Q^{`Tb>cejAO?ff zQaJckFV)28+P<TzrGIC0gXeGT8<@D3J`rkiQIMk!6x1JP#foFa&#XhGpJ2Zw=h}X@ zlp<FS9eLhWRXtOUESZ69iN;w~Bfkx$5G6kRFooA$>tm1nbY=&kRzAR<2gJV(MRUw* zB>;GChjO-H6!#fBk|=3zVohoVn2HJoV7pS2mfcocn~bZjQm37o#*E_Y(8>a$N3B5P zG6zBR7X;5q@YfA@XJ<3owh+AZ`=wfTE_9F1_0tLL^Jl&OZqrw)6(1cN+ppI0uID<m z|I}T-!oLq$W#A&*qK^4e{bd~~i<oBHHkrfGf~mk|JDJ=S*Lo@O*(<ca3-l$vLYZ|T ztpT2rTm2S+hQa7P`W5kQ*&X}8Rq8iDo0<<7@LW>{WFO2}9M<)p8pp=pk-In=bY?)t zaka_8ROt4=ryTTbAUiS2AV$NF#De_cljzJVyhQ|4?WIJgl!E5g6kKu>)49>v*n<)# ztc#bhQnZVz{>y$3)XKxt>fv{wPd;C+LV-NbD44K_E7PZLOCyACbWM{>jpDo2k(H5- zo;I1T`S%Tuu$R`;-Z@#B$z|RS_8R#$Dxq9^uX}35sf5?7={ixNX}5nm%oj+&+09FC z)z7^5;+THC-xd`&>p|(m=m(5fWgv^ocLu25v?zWeSk)f+RFcEl%tveHkJdlK-t)9i z#FJY3siL{+<_L)sTL8U~#hi1GFj%fRu_sXF6zDL4sTdZTcv&#bTsDrbiJmW-;f}G* z8~FXUEmkFW8*?D)A)b<pKfKvEk>j0V&KvRlrue%??uh|(TTb{nIiGhcOViLD#P^$P zdAEh;lW~Z(6lO}<rt&<~8w!4q#`g(Hi5gKYAM+Y%{n(w~L1SwGypD?o>oXEyl75g@ z(smtLqg%4_#av_>7aCIygm^rZ$s$*#cQr0iEva!R1;2rG0f8`q2NVljRR$5t5B?K7 zBqR{f#naGPD|8Bnz7BM|Sle}06`8&Q=u<1X(TZYL+@UM$aW)yT>tares4IX%uX!}8 zjnN08cnbH#LOr{J_}!g8)5bve_BiEnw1_U|d-H)*6!!1yy*d(~WDXoxO8?x8YcN*) z&8jNt*+Q{G=E3hK{t95th*8%K2K8bS6!hkGbnkEMvA>teB9Fwcc6|T!j+3Rq+gM7R z&m?0299ztMo}!Sxr}ysu_So9fR?OaECvfC$-A%OfxBYSci~YKGxoQW3!laM*PH_hF zhZFG**O%1#lZ1fa+n8V0)Ib@h+}fqDC`tkzrp4m>v*@(X>T_e4d&kdQ;g;9!9&V>y z_q#V;y}J%o24o8&sx>CCC|F>B*jM5@v}Qi`{3mmPVq1jVHH5cY-Oc4*WvI5Vla<^E zk~mX@Uvq+9^BKF4clbdsY<Ry1xARR8n1UZ?9Wy={AvBIrbf|V;(qC!M@)VVgdWF1+ z==$FVwQgYFB%Pt{#s1YKUTkM{L-Izqg@*rdH~GC<@a&gK`Q~owk$^{cab!>2IhBxE z<O8C_g8WRH{maUm4;R^E`yVtfPH9jwUuR6b&^P)rjIX~q>XR~3iWvUSnX$7}xYS_) z{Y+!I<OyHpa=@%j2Dlv;XXDZ+t5AHkR)LY#TnQ@8jXlS+r#ba^nRkP}rZM3=<j*>< zlPkU&tA!NU%z!P8c<>Q~g*3&Wc&w>xY0{E0m%C{+d}&{-{+Xl0XXg&f1@IhBtOS+S zH!Mv{m|b~08su19)_%`F*=0IBWB;WLV>8H)mDc~K1(29TDN2_Pr5N-V6ZF@u`+e4T z|BBz_eyr}F?_Xki=4at0F>b(CMLBVQ?!Vxkp<Ji*ZzzEdKwjL3xhIAXNzj4uZ?8Xb zBZgRHFN^{TyPiszC~HkkA%b)J6bLJ!*5K5dsDswrB=0K<e6RYkP1nKk87GTWJ<OA) zZ$!uz^T)u9H^;Lt+aZ?sfo968;uf1C>iXf9j-@t<-dB{RFS)i|WaaI!%a@`n=+*IT zc$kHj{wkJAeOUr~z+BJT=5etLJ^)~CDV=8x6q-*%o=^`|k&Y0vt57F2A|g<5HYVu$ z1PQ3+m_sAHVO%H^A<&<|j=sW&e;Y(*c~(LfTk6R>*8uSlK!X!mqUPcmOR6V4t}gW5 zl%Pu3(vNxwG}bNGzld!wu~6Pw3t|ddz}h5OQ??1at8F_A&NvH7|KyBWHPFch?;J>| z042y2<W?A5@bOWaTVd%hzm~Nx3fT+)`Q!MizK>H22}dA4ThR+G-Zof%vWB(k%pFjP zWt=S48O^O?<(Tm!{`7o(-JN@~J^SbP`10=JggZ=f5koA*-`I^F@j{WQ&^T&duRt6@ z-7gJ)U~JE{eLZodbxcjKaq{svfjLRvY72^K;9n>(i5g0z`%F+pk3LJxjX6x}Z_&mE zhI!Z6c9xFn?Hgu0Up>htLfdWaS~8(&SP0(m(aReX8#%@r{LkjcunEa%=$(*ZPW`x; znVP~ZQDC*rv3L+XqMym_Ri^y9Ib$LkQc)Ql-|=|Z5{?TXF+#>+f^NC>Qz}~ntsjL} z&w6%E7k&X~#W%601NrkW_vRd*#2{B~?oD76)cH~>i>~temo8+)0+u{ax8o9+kRjRI z?}JWh^S>1|iYP65OhbncrFjyC=GTYc49%x5&v!fLqRF7zaZAkM^)q@9J=8LB;P=@c z@&%JtlP30Td+DymtOxun^Pbm~Z2?6bj-N@5IyCLO@32nIPa}S5@sjNM0-Hx9X$fuf z1&BM`O2ekjwW>Gf1-)3|4xy{~XZwcrx6i)3-ozF5R`ueS-mrj=^5=1_Zs^{OGoP2< zNHox^I`*G8g_JL^l)i9Wv{SwruH_X(D}7seH@3dNY&sI`STz^&X55sTYyIF~61=Nr zWUZ55=(K$?&yF!4X<fyR+<OkRJx!3cRRQENV0=9zH)Z*GA+A$lO_&$}Ee(V{7q#7o z315P0#J$S2pWH8G+}^MFylrm@Vh|qQCQDPCHW)=$zu7)*GMQX+kv#B4+Ui*OUxyw! zDAoZ13Z2zoZQchpAX^a~@scvBO}m@;c-+`aVBAM)!}=`gacRGi&zMj(gP;{nd*5H~ zno6Obj9&y=RI;To;xEeP9D)ds*In<+Lq-MF+jD3*me`Hsim|CXsMzYsh#y<XRENY% zPQvo(tO=)S4<?Dej-9L~0*|@X#A5~N-uAmNko^vgh;kXgH6!@J=m96KTZI#b>{KMH zI;83v`#9_Ah0W1{5)~|Dlx}&wGH>+WjI-A5L%=+9$hyB?%ol9e(O)fpVdg&i@Oeq? z?%>7yx)w&-nc`D`VtjF6k&p;CfSt`t=4ehx?wLP}@n0q1s|)9>!5==ZRP_1mM0#6D zZXtqGR)y!XCR(m<<Wxcd<(=dfR^4jfM$TtMFN73T5^)zb(2Nh|X_|0pf$|sINZ~ak zevWn7T^bqHh7CIOVJdy+N{b4Uf?o1#My;|`)gy14Ep{;UNn(s;X1u7{5W`EW-qWyd z8D>9`*B-yIw>v=?CjH+CKEqAJQv==lOBnUTU-e?tMm{!K^K;#l(4fq>uk|uRCC}Gi zghG;$E*D#o#tDzc|HRXL4xOiln`XDTfMWm67Fj+nlF-%yHQnIuMcC~9M&84=cSd>N z5y(pjrMqCe?kN)zO}?KnD%YVatzEv%lCg5LiFe4CSE_^+E4UV4Rr<GlY-q+NVZi&! z^N4?+jcX^B7x&Y2rUnoDyG8i#$}>pDPWJ}U=VQZu?M@=A9eqF{^GrvCh3`u0a`WdT zNE50y-6Gyj+XOR+*)TcJkm17Jsx69MdXN|c7JpDCLn%&NUq~{P&;7t(fT9&N!tU?s z%%^L_9sR>KY-Sv`(-*0w9@P6k!pY(^gy|ZWMt?Of7Ls?xB|@*hpZyOf$m6<mgkAJ) zNcU=jSSzZo?N$(yH7N1h73<{398p%4N3lo@!dM|MVqZxFh16jRcaOvb^9WqpNHghN zke1v;8$dn9?3!BX??Ds443??t1FdocT(}ao2F%7rJ9WrPf}9o4=m(W+rvg<J+$>>t z^VhO19gyCgeZ`&DfK^e`;59$YdXpe9yI(qT&XJS=JM~C~N7F_3oVgGydstN!vsUFc z8s9YEW)iOPJ+|J$1Q-CFy%oR@&I?BeAi{DZd@Yt;t5t4BP0y&)SeR*1i>AN~oa57{ zA<M?ksz+M~pL#M4w>5iWzPNgh@<Kw+2<R(;?Bi3WV;&iVQ07fd#(ohkDWJ4XFa8+d z(}Rmf*XfN;!;=Io!P2%)go_TD%@3o^pYV$XYdQzv0phj|&9Q{<Vd&Em>Q|KOxm1Rq zKyy6~Y<;Y@me)Lj3aV$3{0g$_UwNXe-|rs*UZa2t<-MMCG+W$r?A|_C`F$ovZ^U%~ zbpjIKJvOKX$uvp!EQg%KF7_`HKKEhHPly>&6o0roDTPbf8*k|vWs85YOUMxW(u*-~ z{YKP?(8j<}z*!ZRNgKtgmmIDsjP<hxtMx9L{~lx>2iJVfawg;Xh1WPYeU2^mj~PXK z*+^Oq+<vPT0dP`aO;gJs9ygiMw>e(yIsTLNq+FdjuLjJ;c*k|tQ<KXrozX`dj`>Kq z=MkfOcgtG*&+z{4%g}<c0Uq|`<ga6(yA-#cSlB1|oF{tlFZB|NaIas#$A7P<)Qa`< z7sbDtnqo5Qqc~2I@!(R-FlWQ+*AnCSScxnS|G|TJXiE<LxG?(6?44H`O>h~2-Udv5 zk7oa0Cgd$;6|@b;JWxsrr#<6Z(sgA(<}kv-sGt4N>t7@-(%28`Vt3Y=jPhES$tTzZ z)pKPpycZCGK$RoY8}fJiyx!kK?lWxLY;nsQ@V!9s109hdZn~?+cw`;UlYkP(gkar) zDs`I)E1H2O<Y;e<76%v9w5`AKEb30XtY-ZgAMVRgNP^K4%;V>Mt0y`~A1O(#ZzY=| zKc5o6K)U|};^V_n>7cb@-XvG{xvsdSV|QDop?}7s6cxP4Dn5OJv1wQ(*4KkWGyy?^ z{u;K7nc2-9P|`3Rb>8w1svzs&Bkd?Y4R=z$ARu33Jz|u^Z|cN)Qbm#>PL*{hrIW1q z4Sd1h<=SWe=VLP8%VX$hnvsb#SibD$MHi_kV#bOvU@L!xoU)X_KB8<w(~=SSbBCsG zy)p28eJ|z{t)87Y53V_^N%cn~DE#~9d7vYXITdn)U5q-`X!3UoSp@^Y6k3vx+9!ww zqm4lN8zmtf#|~sVddm-3$H*koL34FCrQht-`1r7E?oUf-Eu*Lyy_gENs;RTsJZY?N z1J(z1ibJ=xuj@yLq{}(Q@72@ZAI_HX!eg*3huOCJT|iv1!W2sVm*9nFmVt`(b`>@d zqh=T@gOZ4WhtWq#p57v7AJ3>eiK{irb#tX>tcQ`qJ3wyTJaRqXyJWk9gNYel9A@9E z&Y3;2lQ;hC*!9qzK2A}yLW29R3vw`jF~{c@6AfcE_bHVx6EiL*EF>!~PTlqYOWEt= zS*3+{Nif%j?vv)_kw<+0OGo8=<Y;kR=<~#nwEbS?RKj=&e9}&9gC{>UTxq#5e@!sf ztDvuL6}i!PQ@ZR7>isS|fzh%vt_IFQ#BQ7y04zzbqmO0P=n8mC`$GYjrguS8)ZdvX z3({XE%5v?r?l&86Q8<Y8+n50=HRe@0)M~6+%`iNc>Jc8i+Mc)9rIw|$FA8Hf+hUhi zXHfA~W<O8LsM@$Z`&V}E_0azF+z)lC0b>uZJ6JuZ!GcmU%iF8O=~W4A$ky@IT*7Sw z^ldL+UReg|-g({DpkwJ*-I6&;mS1~z6jmX7ISno6<Ln^7M;M5oa{+LO!rcBu)i0^A zZ?Y2+SDlv4`lq##b!!H<&Lw%ve{%UbaV|L%iMyQbPBBfoC39m;NPM;|0|R8fxrE=G zdn`szVQy;M{bI@=ardLkoJ#kakkPK&gzr={jwSWvHs_`XH`@h#5uIMDwIDGSSezW4 zB`=Q*n|*DNv<pg2FY%-3+|d08`auiwAJPfcEv>g{oWMSGfgQydMCu!?WD>`*FLc1J z&R%hTxJFY90h2kpHok+8sM54}N`DwYN=qLczMR50&jzxz(h=IHhm(3k=^JY~>+8;n za20xC+pcv>#3tP{M%X{zYuOv<Y5u>0QP?Y(@Qt6Bq70U6-9U-~RO#11Sfh<G=F7Jo z&nUE*7hsM3{1%kc?S0QpeVwbZ*mbsdvTe776`1&(Y0Du-gn%E}PSuZh5Y<go@X!Q# z@B?Rsb>6w5NwHB!E#1vx%}cZi-U4o#($pX!^)tap@6sEbK%;ZRZ;5pcV6ckFJl$o( zVfS)@>7gKKI<udlA}rY5nAaQMhL7lO3IAZKIL4xu^y}bE!#lCU+}NoHi<BxRGYrbE zK38Kq8u5i6+-E;+I^4_M0s@7k>)EyO*a7Zj*t_LwH7GH(CaP`P1Mx0;#3LTX<ji>p zBAhEABQBnjsad$Sz`nF-e_9+x^SMuL%@Qva*|oEn-?o%*Te+!2(Vu@*s?MTz0iSTS z6B3yvgc*&aWl4^@(n*&HE2i1W2Bwqk&t(%HP0NdqNorvu7ch9!1euZYB+^^T*QQfz z02=|BTE?Aq-y@-HeABfKrZ$YPkE7wB49V{Yn@AQ1BN}Dv&_UOKMK}C@>={M3JfnQB zw<6Sp_+mDsBC7&IUT)~A)xo(-<&ka)9j*^QD(?SG*I10i11YL_i{l8i%4p&3ct9x8 zh>g~8T9V|(UTN;saQuacIJ(Gsr$cx(N(}34uI~KHdG{ZjV6PKRDV~g0fbERO&#rdn zYhSeL;}`$bF(2dH4u)}4SW_MDL%_v$zK670-`wY&;c&_BwU4}sMK?*`tZN<m+R-U- z`)mE7s&J*2!0cSDiAh3ZZ#$=o03>j|4Lcww4YLB>E%}3B{J-JPa7;YB?RJ4bSwizP zfnxt+Tt6wm6*~;hOid`keto}*n6}^PuOo2rG79%;ms5^i&37$9YAk8z^<>ocdYyis zWz)p)&gFRJcG=UpIfnb+Ubqw{wOQ0(-v6!Mqs}(GclL@E`@Q>Pf{MOMNjg*K&P3E5 z*o%gAd2>xI<ta=#L#I1|(8g)WIwunsEeI8aRUCFjF=V$X;$K1(q>-em>0<Nq$nyFR zO+#ar-Qx~8-=@VgQi($@<95h&z%0v^F1I)ko4j)1`;Y7KKU7PXjDOs%{&8j0?MtZa zFUo2x_yIqvX6?O%Fw0y6{a8TEl8(xe$1t;ZR+$5NCBl^_t;y$>Aadt4<WwF68!o|a zMG*_IdJ*}6{s>SYA?12^ers2xj&)p%R#ocKJm3WKnguGYyrMGcAdu&naj&q&CpxdB z9IkQ5d&Yr!A>f5&dgWH%?3maEhP%x7T4FWqc_7@NPMx6E#%%nUQ&-@sbZqV#RssAP z;dM}Q%F#OsqV4?auv|wOEG<w1&H9-%@8Z)ZwJ34Sg@O|H`y}Cs9#}pk-K~CjoFQ`n zJ!j9CpMXNQc%Nb=XVRjr-GtzT8%>DS=252xi`Ma{6w1`}wFw?WKxr!e(3T@aGP6>^ z9FPiS3gaxWSShb)n>I<l1${1Tfz-j9Sk8j0PFQp^kf2(CCP(8&JWcu7nX22?bj%(Y zAt=vk&R_v&wdjYoou&ZogcPmhkFCrio&DNA+S+;M7#m;Ow1$Ef!uu7HdA(R(>JG7b z-&@yf#^Rs!VEC_12PA!B+axsl$#J!y!;4=42_I9PR5hf7(p$D1`%WrodAM9o3|rL& zjVS<U6q9NhELWMY)S5a8QB~xRin-7Sc<3pb`65*mpb?b9-KwJD%olr$B3lreE>rh) zHas7X_x2m;=FjHMt1iJMTbYBgjc$fR<U`ZLf?L$M-ny*V;+t@bv(vS@Lh7fuo;<>x z4$zbp=v2@r%18+$bGJOcTA{<g2QD*O^%GNr_8OX70F_dvz9ZLq*?wE2Jr+Nl(N0jw zfduwR_a_#1na#gT2g+Z_tG0j*+>k8_7<m(PObcc7i)OV}@%w6!XCueAw^jVXHka2D zN0ffDx#YA&y%1Z{F}CIY|BI3gYUqIVt@ppyH<pcwA=tMEWa-u>ZZe9_;?`9S1oS%` z^3pPs==Im0_=Gd3ejm^?T8i-34&RrCSQ;0Ws$-HNiSbtRpTZ!liyOhS)uo6B7a^$; z5IY!O){2V$GN}Sj-^I2MGyZdk5<5`1W}rYu!edW5WgCQz;>i2|<ttQVNvjpIT7}=& zARmeK_wv1@*?PLUv*q)t)j=!1XfMfH|N83T|4UEo?N=Znd{w&ob>X#jkb8aP`06=T z`u0<tj(V5cOPjgD0hvj1pB^E3T0#c1(f7-*<9jgAd)`L9u}{y+wS#8diW1h@JZf8^ zqnLQbnBIvxLJ$B+?AdKMXV*D{Y?rCSZtdK^ns+**4}d@1RA|rxM!(7Ku|@AtzYsk5 z`FD|4+^$sUq9)E?@giJQ&bKde{f3nE48Z}hkHrRjs89T#QjXBF<+>x#IZbt}aTrwk zE4DW=W$eTYm(tRghNG<s#GcN)TWht=-Xr+Bo;Y4-0+3d$R7mDIY7HsH;*EWH=<fi) zXY`1XgN=3H4Q1=@+>9;3B&I8}yh85uk%M+@W&X*s<Yp6AkO`-Ut^7FGT2>i|sh~4q zgF6ri1~8NG%`7d1tvAdLZX6O^t@Fq5dmsYx<w<0;2vVS`r!>trX?{r=(X3|Yb-UF0 zcSSqbafxWS=#{ch!%gewfJ)E|Ghv9)8vZzpJhwD@*6ego{A|mfEf8T<@kIMCuv(-l zCentDIoP7O!T43y?mH+=2u(d1FRguZ%J?~9-Eyn@Fh~BVM_0f;o<+ol8o*ZVdum)8 zNiKXH@<qWQy%gqqg~rM<<cKzul4N08G-6%Jk|30OyV<1#O**U};hy)n&LFIq;A-?w zOWYrAqL#%qKy^F?XN|`gG_qp7Y%0$3qyLmT1tk@3^zK$@(M1y9mvAh~`iIz%D3Z>c zrfiJ+<D$XVzA?{HGj}P|NxQQJ=YoM`x~$)@F?|uB)$F#IsMUsDS?Lb{Z@?KA%vEq% z#@b<*(Oe0e>%0$nq7c7s<eKl2E+m|Ot-~at-lrrtj%jZ1<E4C4_8@1a^!%sLL&*ai z`qR(UM8c$Ko`;;=a%)?CItQp{CqN;5NqQ7Su9>Y|)528Hid%W&+06L<am_;c-h#4y zfxp^e*n>q4nOCFQnEl8OlzFzqza{~pLs7%?xnlqBx3>iiQW1saPr0h+K7)vjR#gs? z;}cw~VQ*)tfG66o6Yn``V`res=@4{TALv_vy7G|>E6PfTV6g@`*3OHgh`0JQzaCJf zOg94Y=q!}g2L)oe%B!xZ;VjcR<a?j`Ndugbsk2q@+=?;SWzXX~?jP~QHrfd`Z1AWo zu7~Q+96;UgBTH3p9|*FQ(n5vfb}EOq=^D8ZJqU{={ikDafn`O}sZeF8Lgo0G@|YgJ zMY^~2JN-2;c{t>bwFW1vl^R%}(h?SrUnc%sEsGtFTM%TyTLCb14`3hRx4Z7TOwnsW z`AfE-sAHM!n(_c{QOsr+O(IhUPo%lDSgxK@sf;!!m$&tJSWY!h8}V@xgU|$-wd4c4 zX_{paEDq%(&3T*7d<tcrHJbcQyzY>Mqtlw-`<_xz5F$*yB+11%%wqleJCW>zN43}) zXR7t-jhu`q{A?(D>t0uU;H399DY`NvW7!y*@|toWdJxNp>PPoRqtE5V%fkMy)rZWR z2#tjg$tWt2N8DOIY4VPb;AVe`C$xmoB*G4+UZX<<!KPst_6Xn*T=bv`pi9m)KJ>09 zQ}BGHn*kwvnqZcr#h(>V$5&Vq$dxgOqG3V~l${U6f<wlMicdPIlJ$Jsv-44)nZaY< zmj$O1+&MeBmE|7{n7G&b6CAuwJ}`Dw`V{;l$b=9C23=8oV&!{yNd5&I^gOrPKuT5E zq=}S?1po`;18q%pXk&jvTc23vWx-!0!ar+7g-Gx7%5gOG{z)?xMjXoKTwb;8SMOKj zjQ~uumH3$8(C>u)e3uqH67?>8C=p+rAI!1uKw}1awm)p?JrOCK%j-Jl_R&>@rOQ2M z#&Kt&-&#-00g$>C01x)M2?<OSdaMM8Rz>(+aE?6-`b0Rt9ehPr+d67h-uL~r3I$H> zUzW-;n@7#!%7)RVOZnxILl^V>A>pc{&-d!dbZM)HnuM%A-&yoA40;lYamr^{lI0Hj zWC!$^2?;CFza%%7G7PuzS}npdNZQHxlEKG7prp%GqT5uF6?io<kCTe^lZ&Ki>)e?E zB3F%S5t6+cVuemJO=CptkZYZ|5}y3-2djKan;f;T|8~ka_+QhnLEvivSYCUuk4oXG z<i)Z(xs|i6JwOy?kgx5Cbt@j3<ho&=_pB~=V1Lm)p|E=d^V^5)9g4_$tmOx-;+cN` znY1F+gfw2CZ>l2X({P&Ud7<xeoGDFS(|_^c+#glNfUpbS=ee?b6&bth+TdH=1a51Z zkRiaQPH&k$JAQttd>cZ408U6FzmB87<ZZpa=Ahj6_IC7piUh@pQruVbEmm@)s~NxQ zyaoz6n<l($`Kp7K+rq3G909XvE)7jF6Nor|WluxgJQIIoCDPi1v$5xVLX3z@9T~Tk zE2}<sJ^z(-s#69U@ZkdLbRr02UC$B;cV(8x#2ld7Zn4Kx+_G%zjurs!CGE7=4dWQ; z^hbA%q?4x;R~N<BWLor6-u;)xYJ<QYi?8UV{K!K4u*Z4q4sN5>@6h`EuPq3oTQ32} z4X`HVVIe3la$sxvV!)xP3*+br6n3(Pn8HV(7t&dEyqf#GQh_nXdUs{OsbLXcH7>co zcB{BhGr0mFigV><H>!q1>iF-r%A2Yu1-jE&#6}d+I3>J%2DeCopl>E#!j864Q2I^f zRLiQqiDPWCr-PUskQr7}K=VMc?5&o%gic-Y%(0>gL*A$8**K?4H|9X83l<ysiZ?{q zXn%y;owk%vk53Z~H9sym<2s6+sICQI{dvZ&Xm`W9F)=fjSGSP2_#pvkZ0Ww)&lQ$1 z7J{uUz22`J-TR(^Avxf@x+C-$d{Y$^d?+T^%6H1<L6*aHd_fiWsI<iPZ8l2#bxYen zeurYxwKXL&U#9CO?|Kky`zQ)V)@5grIk<-`cpP^8c$1shPa_1mCgQ!f8JBcz)IjCu z2qIEz(<4JCJ@N7+wb{6UWXiUxDWuH%)e+X|6?4YXDSgk71E7gO&d*`FUoE1EjPX)y zAM36#*KG!i0T`<q#+GUsKP#5jjQHJ)Qr*@J9)Q#|cWAjQb5KkhPV#tP&>@#6UKM@g zKW}Ty{!AEDz4Qbi)*iN(;`g?*s}F9s=!81$*>%y-Rzs21${}<8i!z&ahC}TugL-K# zMn`>)&0LE)(!LFBaY>(?+E81fUC*}WmEYs%i?Ki=(w+Qtnxv>{%O!yXI~NmB>{+)x zZ-A+a(^Uo>WBpUi3R3CM95D}lnZG3wePonBrjLx@LhEN5NvJ|5D%*n`@)^g!LJNPm z>uU%9J0KhQ1cNV}4{TmavL}2N0k>Zs_M`4#yRiB)QTraj-B5+^Z4;c3)32OhfQ9!# z>V991+5hzKFY*RTr%m3G=kq$lAM@Dg&1WOh^<KP_`?S03q%q+|NR`3$D`7ES=il6< z^nszTQRM2ttxNgc?xVJf&gfHs+03mjD8~6&Ous89uVc4-%sNf%&oGmPwtpgYgid1o zLE8g4=W^(IX}hG@sNpj)bcg2J`L0`FXb3{{pUhcaSll|HCoE>2gP*uNFC;mw{?AP1 z;kq|YW}!t6+b&a+ClXj<Tec2Ewm-YSJ4?<O`5X@#zsI-Nx448<pKbWK!LL=QV0XmS zn`AB(exPA8M~0mheXvxDf?no+pHFcp4ogyR85U2&K`AOPCkeK*%x#CDYroW&NRXxm zxldkxEHSNerQFf$Kw)dmS;_V58^O~dQafEQgU|z!3q#hDe~b~J={t?_uXl6Cim4et z+91M*eMGr9|LM3A9yIz5c?TC;Ec}vkk2N&d7F}&h=J~E3rJ!!`H8B;{tQOC-RniiV zvG$x?UbP2^5Jk9kEZy?SLjdsph2W_U4djhH$fjE&M6d7wBqF%X{chX6-I|@zBBP~W z_oQ-jCE|!ag^Vf}xG{xq(j_*i4Le<8nSF)$?^CP;Fy{m{@P$Rp!3<FP>*TX{4<+<{ zC|1xkh7AA^4;?BYI=C?at-^>#9)3YB8Q1C}VE`R3``UXFt=+6(gfGZnMm=yjfm4lQ zY^<S865a<=u3&`YvOjFU4F6(R913C3t${f4M>!g7y5HaO|4$3Bp1Vso2Rw{#;xG|b zPatcDB__u-*!fr%;MVC}$o%|qXqiw)^F^&*5(_hq(`hc~*e2aW1-ie2CsrGW#ZF%C z?_@Nvz>o=#-mzTy-&rWFbJwK*89chZ%_;%aBI7)j?n-}2U!CYg5hH-3A&YbBlu_p^ zIp}6i@9!gN_NV2F_3Sy_EGR%K_uPghMuxO1Jiu8!2{)2xLWI|;o~I71<G)9zW<})) zvRQu{J=@z-G@nWWg_rsYeId4A{GNHd<hpi^xr98t#tRHn?{K28IDD4hDL-F2%8*Sx z^Q?o0XP-?PI%V{2I9&Jszuzi;_wxhF3+6tYi;?<xs#Se-!W)%c@*_n?+gYq}q0M!E z{<5ph##j}0V}e!{lWmi;KL9KAwde=Uf_qg$%q0Y*CKVRNl3ZxHbp&I?O{locNhRmt ziEE85VW&hhC84_o0C{1yv<fYUW-;(z^oON|P5(p;`YRg-$~*dM&3X&G<;=*|oO2lw zC`FHUfr*GDKt+GiI~LNKyKBtS{}Su0W&bFRHEp$DE3$p-i%9vqkK25Vz-1KuKUOew zFRsO9FRq0VBDTFVuBjKyD&7!t;(L{UehUleE}Ckr7xi~nC+GqSjhp8f!P5UwjBd@_ z$FjHGIr}m|k2=2cZ!4arePpojHL!3|L&@Kn3i$zH!X_UY5-A@N!iP0yIC%YYt@ri` zN1M~{cbBw;4r<#h$>c~jE)ErxKWbk|apm|$xJcdrW;~boT1ym}>ZqNX`tsq$K_ET> zDht#>C>2-6JGk35;jrW{eGakb%%ZPcW*_&l<UWsXh18LR?YMgbj3yTIxb-uVOUm<T zspkcKkTap!;oqG372HUEY!@HOs{=AMN8$&!%F&1xPv;TD)Zgd<W^OcrTvCNcX7?!u zv)>gMXT=(RpjEI}JUl-$`(5g|t?R>XhoA#o!`=R%$gu4rJ8>4k(ss_WqNn{uxFz^P z?WTsWvC5;wEhUv3kw&UyrNRt9@6+nVAg{~w)Tg^;AQ!23#QK3M)chSwvNj)Z*iQ?u zVDQCdT0_nHkp%`XUZ99<36;{yJgjO=Z-siSS6emA0-(a=$1;Tb4aO4fuU3KYS;{_w zgm;Q>Ghb=Z<)xZA!$O$`jHrqW2~93F(ZS1t;8U+Yk2mu5Q1Ebd;BFD9d$LdUV*IbR zC1)m(UalAAS%>hCe?1*5HTKFjVV1BtCy^I>XAL16UD{rH9k&(}X$TYVG*w7+5VlAZ z`fG<s9jv~RoAI;v*<8m+0lK43P^o^`X@qf{0plzNNzQP;gw;_m4?~QVM(n2+WgvK% zAj^Yq)xS<0xD%(irUlxpveEpIeIrZB9EkVRRW=9J-fl7Bk13}L-dxW=*OVAtn~k-B zbX?m1s?Wm6`S8CCn_6*)zTkJDf3K7<t~>&N$`YgL1Tf~|_d<q3zEu%AXp>?X)5#Gx z^lpa5n0ZPtbP6?R_&!8Bb!~;N(z&jrZKp&Quf-><f>m<G=7~XJ`y2<kp~X(y-me$~ z#f(~k9tOYSd&-%8Fya{C4@UM(Yse;OiVdhkV|kc|f%1N5z67r@6~n}C#AmP<&tU!A z7A54$0NqNw4~<>n@5FD>h12KE8*T`EG}Lu>fb6oZkfHya?LN0!z6wilNJq6-e-2)8 zAR=Eh5H%~dm7y&<k)H1$R&70Zr>3*I2L5{mBORcddtH|Pn=fEg3()ZwVmC^^y233F z23zQ9=gT^LA2WdY*TPeK=Y8fEVUnut&aPS^{2j%%_+p9Z!YEo|0EwlbeYpEW(&6r+ zxOIEV*Nl^k%(E(0GuMy#6W?+YcK0XvQ*gW_8w(6@83rxnlu}HykI>E}Dm%q%u`0pb z2+0Q(LyM!xOz~a)sA_7VnptC5<Xy$surmo6iH%{z=oaxKU3Ji{@}+4@HyUQKwTN1E z3+lM(BrM(@c^{7o`4j8d0vz4TD=8M@qM6**x+M=c*WEEexl03M02)59G=ywsEv6&5 zh%Dm9*^xNzrz+q!8cH&z{dr&4;+?A-0Kfa8n@glj#}RJ|h5$#6PQtd5`QFIbR@?`g zQJ0SU`B%g82??>ymwKtbYmv~jBcQbo=>{bui64bbl?}Zyid><ccBSmE-gZCb)VrhQ ztt>GGAg3et3n#&HX{)>Q{D|PFb=7nmV%khR7}+*XMkXk(X<NYxEspstxwhWt)?l<@ zjZj5ZBiFLlm-Kn$mQanZ_G3gJa>e)&IJ#yz+-!b2e|E<wWcYKGB){(m{9!IQ$`Pa@ zqG2xcBD?$9B@5Ma>7S<~7z6Ycv+{=ENlvY2LueQI<`)s*cUIs2?01i6%{DC&FK)H^ zki|F4YmlYCR)B(rm`bNG=t;4kFI&y=qEMx~b<vUgT1P_%O|*f4*;YH?8j`Wn+HJVt zM;;KKv0*CO#(pDib$f~I@_}>PRUL{6?!UE1Fn#LV#WJ->0vy{v@t(GIa_0W_|BtBm zjA!%l{)b~Lt<lz&M5{w<mZHSg;?srJ-nDDiN)Uq5(wbdn?P@8p_X<^eZ(_u#Q4%o| z72$ui-~0EvpXOPv*LALQ&ilNFpx<U?ofb!VlW|zqkT4ghVxyJ$@=+}dZ-K8m+V@63 z-Z4bepA+`Y5&qLpEB(5YrH<2rMV`R7?r5-6fAQ3rw1Xzf^*!@;ClwT<v~gJN+3h}j zdizq}>iRNoyc<3@W_XLOzIY$3ekw8Jax8`b3=k_Xxkjd(J+!))boK;I`!$1|EyR&s z&*ZlpZM2gYcS8aqDDohx-~&qhM->F=cGB6%=Vxs_9zvf@RcDrSG}N@6YdO4!nf<j* z_E>9cEG&Ab%$v0<6_s*o7h}A*tlwlYeWS;gG+<l$9vB$#WXx%1qnFd0MY3a|+W+ol zf~=u``5QkL|LLhgsMDHdAx9pNkvdBS4ebkjs~Oi_{RJ2qs$p-*i!RirpSrmztY)K# z4~;j))Dgxs=Z2Z!cd+}<LZuj9&jjbEznk#|y!~blJ!TiDw!-$4DMed%=9dN+?2Hn@ zOuMkMk})N~)S+)9h7xx<(u~5ztY-;BzR+|~ja&YBTrjqg@{omodK|j)`$+d7-3&S8 zWF&)cpMU`AUw)f=&{K=6`zrNXucQnU9-wFL&wfWH;)uy-=dcAm@f5fdF^c?fZi=ua z(9>qNG~ce4<uY4XmaFo@sk5ormXf}Suyq0)j!mu6K$K1XrYNZFud3K?9<h<Jn<aJN zgN>UqRB;s{|3)om;mz;QO}~?`KAAk7R;@ce);*ooCabeA;(kHsLyvjcn%DqoBx%wE zmTZ)*jyPpYW}q?ox>rW)*<l{c{iA&c%-i2V1Z4?U+tVl}WsW16?tBgd1UF-ZP%uq1 z?<iMo<tEacm9X8d>%HBWig-Ia-Dyv+L!i2VpDjxsbuG$feyf+*Snw*{MHD?{c;WFU zR>SyIzqvGEjGzSGjHEV~<GOg$P4KHheXwYr3CCpQIB(QJ0BJkiVJHe%)S!NjZXB+< zoy56FF1(5PxOG3@EM;!eZO%4~I!ra|M5It}-in^gI)25ZWuMh=SBrD(_s*P50z%7d z<RjeF@|uzdnuj^qO0hVB&{c2?`GW3>I`cu5<SZ?wkcT#x?FhcwJ<%;>-%w0@Q9I#N z)$rNdMmCmXG8J<@^M}p+bY9~3GQ_k$4)o-k3wN-G22IvEdD@)cw4j*$^F!aL6P7m< zQW?LvNoL83Nfc~NfP?8SPaklqIRJjXxe_fB<g?|ev1m0va1_VNGsW6C8}slrz~?$W z&J$0r+;rvA>IL0R`=+SU@f82nEd4s1sbip>3+GfO^VTn#VN>qy3)J{3U)0-{07Gkw z_*a)KOTcqLwqBL%9V{R^TF@8VT*jZW(HIA*_L|cZoV;Q|6`J46!Vj1=iaXdkmV^Jq zb<dfeHQH2#4_5_dwmUjNyXt|MqpzW|*UecEe)5~X5#KirQe#+EW+iC*PcM4HA!AKu z$QLDWWHz<pD0wa{o6Y*qbd3b=C>=qx_zLG$LInTskm&q(V)XwzF|M0{F!K5|za%0W zmi5ZLOO3)XeXBR`+WQFIbtKuqN^=D9*?8tsLwwl6G%)rJCpKQIU}@H(%n*xCGURt& z1n{^3aJ}X2hGk23EsvO{V#3WG>F<y3^8j{0l=N}s)eJg^voU#sUB;jRiQ3=r85-O6 zV>>%zH1+$ri^+7GJQpO%G|&oo{H-CHvX4^-2Ms60xHiQIAE}cH1wF)RI$GHUzm*U3 zU^hCgMIbsv$&Y|;hBsXgq11hVZ5-8l`gyBre7TQrb_PWHaV(adv+@nMN+KDdq9|h` z&p1WakNSB2W@g1w#!UANcvF<pQB1jNNG>Ke;Pii{j+&~jqOPA@#1)VQoWtYr0Qyo4 z-Xw+WKu^{XJ*28rOTd=0-SRd=0%kw2_XB1}ULt7_MXJ64-P?F6)vSg8V!~hxfa<eh z!w(S*pr<Vk`{LMUTD3WzprRK4IkqWbn>0YOV9dZdcTyX-0-gyLY;ZCPlHWzj;v8kw z`=6yXfU@T})oJv+a&A66l>PyjTBQET{4J>bHj^!Awm%?AJr>;8bbr<n4pBbNDkpzG zu%1T3K`lU0CA$xi=CFC|)@UBGzGMaR+5{u{8RmZ9O)@wMP!@I-z01Tj1BAq(Igawy zCMYEaFg=p$p{@S<lyMrq-`BB#yV<hcmCor%8mQ+A9tYYNb3hI1YskHC&evnY9?P#x z2Y~}F`xdEnabCW@4+HI5c0xd=ILU#t>^NFtGi9rt?b6W$<%q*=S(Wq6G2BsJ!2BaS zyC}>AI%j{h6<K3r%jKfM(f{n`heE~qVsKbc;a2W=nG7?KfvNm8r;)Q6-D1yq+*4mN zV0(v-k1uUzR)mgDO}p;nz<BOJJ0Rt97d&$PV$Ma|4`lOoYGJggq6x6LRzFV1B2b!& zZbYOhA7F=x!{U+yK5mtAhIaai+T4!Q2FV>cGFCVq04snn&aPiDmT$!~uNX~|ta6ot z)4^w)q_%a~tW^vB3`70lbKgaS4UJ+hjJCV7n2XvqRpL^D{9uk%@6(;e<^s4P%jRPt z8qyz_sJm&_!E3t$*1?p=Q6gmF!erHdY>!6lU+q>Q;WU?826Cl2p0}=3)n}`@eAo?H zmk<6_g$NV778#h2?U;EG7C#-{IvEkq<vhC_q^(`Rt63z8_?fxc){%gXWeaZUV8cG* zbKNTyikW#(rrZ->*V&a5YR}SAuz`N5*MKhpaovpgUgUm+fU(=v1YAXZEOXhqdE``; zhKAl`&if%%+gxs4fqc{lgJVi}0KlE54n9{gt;BMSElZ!V9=7f`mqm;tuQ3Q31G?j+ zoyE9XV~AO(ED*u*9cM5@Gy$4Y!<KOa`LoN`)=^{kPi55gEFjp)?iOjfNu|MheZ3@T zK8F3x#(L-`f+TSjJ~qf&2KOadu4FtKo?B)jHaMMQm9n-P`%BO>3!I|))BR!T><d#S zNSmR-i{S-<O~KwDX~KLa348q8Wwg54^CHN5#IreeHkQ^fI%2&P1m0Bjr_f7klbb+9 zxBOsULKnwRzxtQOF_5ix3d<S>EA(|hiwkJ;RgAe>+Bfk|9mcuevEw1eYs=raDlD_H z3W2w^olezoG73a#z<;Z55}h{Zaqu<V>~z_``n6~k(+VO)3WTnQJF04(f)z%a*51hW zqMV?CWuquOHsv^i$R65<%jo317>W<)${JaV8p>}iqPGapX6I4^t5<`SD{iF+<mB2q z&9h^?SR;&SO!J;@7tNG)xAeS!MUY6CT{8;xOtVW;+JatXO#Pd~NMua)=6z67>GT?q zqdU>4)*i2>d0GMTKku^BOu*I84!{XRJ-yug8Yw*>J;pv?j{Zw(X@k~WU9MFVXD7_n zfwH16G+K6&iv?Y^>Rs=TYugn5sYG{6#<F}XX+EqrC%j)_v%`%jx=m&ovuJ-jra&9d zcq-k3?oSHCaV-R0jSfFG<h`XqlnWVmTCdT+PW&V&<L~<kF|EleEf6R4^fShm@^He? z&Ed~`Kgp6J=-OM3o69v~YiE_v8W?6aCk`(MNKy#5aAANEv7w*iJo(57yy;`F(7PI3 ziF8t$C`>ccfTKH)!f!n-2;WH^P9CRIP;!($JXD(chU9^F{3V1*0aYSxhxQ==CWg<$ z4u`5ecz9P{h(dY@f-Z&=M$c%0Bk~=kA&<z^eh7Zi<zsl+Go^>t$`&O*i~(nu83O2X z&c7Qn!%G8z+`e6gJ@<k~trvtbDH!Md;a?MAnoyc5nLC%qTKYYZ2r7gBDm_{_Q=-Xo zc7)Hh9`Q_Yzix2!oqDV?*JXT%#Ht2@NW?-Wjcj;h%zLjoVx;&!Dg$167vy;x@K!uT z)8boJvE~kg-c&?OrzNhvzIT?`KDBx1xi;Un8aspe0MrRrFkwJBnpSNq!5UCx{lopj zAD`rJR{(C9M~&0N;7zUTM1#3;q;jRDcF0wdK7tZ6WUBh3(~YVJvKVU;M$Y<}On;)R z)H?Z1bQ}Z+(I<>oXl%$w{Vkps0->{Eoq8P-QQ;g3EKtNHwaM^T!)~8x6K>eq1xal$ zFgMz^E}m`N{Nb<v^}Co%RME``5%Hozm$A^vKcpH-9)R@_nTm+P8O!FSwaIxJv+ZZD zU7#rs{&F6ETW%)aidh6J8&hKBgQXZ^fXqGn!I$sw2PqIpn<Yot-u7#cfrjJh`PQFo ztS=nlMs6iP+NQDN_0Q$(58}OY$5^Yj9FQfG+4s+iqM~;IMs!)^?#v)piZ<CCU$4Tx z>UQvqFU|fAXxBx?3WK9f@Fb;|;Al%Nou^h?*y`otsi+H`s(vuZy|)S1V4chw8Gay0 zcM<?oTv&^@Oe9d*LmHSaRiJ<XPa}}|qz}*j=APjq{v8ySUg}l6?gL!6{{r$4_kMx8 z5MU(8{%ckSfCB!d02O<N9kR1BjtLyMH9Kb6FbsghoZS#TT%k#=FEdYmiXXd_@X_Dd z_-Z!n=`hHAo?pnA#-H=VdwXocY?@}J+eU}+ttM`|(c{5~Rx8F%^e?`!&|&Ko3ohp^ ztvGUKSfF!m@##{Km|7rID)+ZT*|KG(F+5WpS6JpeipR3d)@?ja`*ZG@VI*%;v=m)= zpj&;vTO6Q++4T#yXm>=Ru>pZsv<+VaWGcnd-RSsslU_ILuwKz!qYXM3`%$s9T=c!q zb`01CfKPqvKrTG~5vt^r%hFNS$A}%sO&gQ#`Q^9P()sDF&p&sVTF<{|db4=Ysw%gC zBX>*41%ir@Luad1h0U5xL3uU0poM+|Ta8j0bKfmxifD-3SjMUhZoH^xvCf_+?YT6W z!s|}){1YB3o)Mv@0rebT0`r6AgZfr8X`kZz(ml=sF?tPNlMHtdE%YqRD*+X(%=!eU z-BOMWS|5AAhCn8c;Q0b_&r#WG0Y$=GbojAl)YF6X?krFrjD%rROCcB=RZJ(PQd|f| z9in#qj{$1%Jx0Oai6YCoX!1k1lkcDcMTk$*ocvgAXVo+^4@_>{XexF=`UM|1ws%-o zT07yM9N@}`c%YmLz;BOxrq&zc@X0{_2Qhz|yMlPSO_4`3r=7x73kzHk_>w8zCTju; z6Gpzrrv!yIBS61;SH7CJX%bb+8Pv3RqZpovm0xZ#O&5lX7SZPBHXKXPKi~7SDH6eq z1i#}GI6zE`3XM2}CrGeojdzR<6Z!6TfE>ZN2KI}gMz6LFoG&ixgq=w_?42-xWf}tr z?2K!~cs_779x&}zoecRH=?1p(g^oYk4`#ZH_hl!p1|=fWnT$*8YAWVa*lsF)ZvU^! zW9f+G<A<3ylpLd+Yg|*CE<7R*@O_BvraFgE(gD3Rsja?U>Gq{nYi)Ax%I<*bab!|W z$e$#l!I#b#80DMUlZVJg^0$oSI7>g%g~?CtZ1rqlne^acXU-9Onlb%vIJP^f{o^Uu zSy>{23BimSadSC59GmGY#&dCq^%~YAzt(A9lj$@n)8!btii6DH$UUB(@ai3m?k+Iq zBGLm$uU+v{PW}HAXf-n|sftZK6pIkd{$hIFK27z9j%cVa!zDYOvHNTDXKQ8azJpBh zB-g#^Xa>DGSDCJA+P9b1XTml(mTn7Vp+wUR>a9~VD%e^@^<?*wdaK^1C-d<|%>K5* z@7a<USPtcP6?&#$adLhs0lm37W?Dw=s}b$ua)1BlTo}dW#BoS|xhCawDnJSD_8CzM zgrERWdc*TUE8&LX`T@t?_@#5y@X0Kk=v=josc4x@uVqmH-153p(8+h4w9V{Ehgz46 zxq+=eJ=A<{BTbP4(Dr1I25#}*f#_I@n*-!yHvtcjrT;Ix1Lm>yQke?N`af&8l)son z&R<MI$P5IXn5fg$XZw27?h(6RBkiSv$Z36)Y7CCyJWOhoS;a~n3|lA!>?5Yx!`^?I z&FwS3wBTVZX;I^oZ%NSl28q(9R4^?eR^z&}Tg2ohtmXzfVM|+kP|T<7UF<s;OiipR zTzaFmL$`>EKeTkM@ZxdxsF`<?x_1?R%rAR7cJQgmoFThBIH_!IpdP4&n=Rh~+Qssx zsi%`hB(ZqTd#}@l3=z1eTPYWDb<~eYk3LH|1RNi5f9~nhwfRjzjTBY7r*P@0KZICR za;P$42_KZN<D!0wB;cchtKcG(uhK?M#=T4N2Tkvh`INtFUfST9o+E&rF<Y|LB~Y~` z>i4vNW}0dRe;lPaI52yMyI2{Vz^YaLz#ddLS|{DJs+UPk4@)UE2V@=m(|N4w*dVc( zRq3A|AI7&n2p`6jrT4akRtwj@OQ9pY8sdU=>YQvSl_&-kY2+qr&oMCri5E{Aa!$Y3 zU0BnC6{#uGS3h#%=I`6zm`K-e0u%tkjJy=IOGy)ESDjmC{4ac7<G{6Kze$Y_ji07U zcAT+mJ1h<RK9@tKBCf^5TPmhijK)K*2jSUo*^mh)o}vmLrZX&g_G-*)+~Nl4m|hTo zSoYrj(dH1s<d(rlx)ejX0jEhFQ2C>9HCG;u<~Nbwz@N_X0_2(ToF3RE`v@Ol9kbs< zpJVyd2R_eaOoIuA1eYM|xtv=NgKgvA`GV?}hTMiFQ_5yx33db1#PCL7U7Gu{A=qU* zy8Hk02`6c%!zec+ENJw=anc3CLY*29<(O#g#DZcHqlLT}tVlbY_7-7)KqD6L#ziqs z=gb4C8xji*w1F}LcWuzpojnLXF&>)U9!23a*kNZ&q?|G1ZN!3C1XWO@q$MV0P+WT| znZNaQvR!U*eyx!shBJsixOF3XQaChP*7l?5wGRk*+@!&;sgwr3uD-EQeXuWIVftq$ zaI1DeYe!12%U%HbS$$gC<6HB6Nil=75-v3)kq5K7jxbkJDGg&s#)?HAo1<*Vcz{yJ z@OQ`DlVnfkZgaH#YPTCo5lU=|gjM_<_V86LH1EjRy*{|ucCvU@&{~P$Lqj%(v?hcf z&i;}#*{^039Uq^5SRd&Ni#)2R3d-O!35_g!gyN>XHHQFw1cXHC$`|8q*#{mH`;_Lw z7uP9U+3-ApxyMm){m?wepR;8+*sUm%{2y^rN`L`xUfSl(eSy8#2jDHAxveUF#%s8s zU<cy*s+&bxmF0fj>A*$_{Dt9Oi9iD{%<{CLxoygh8fk|%!cU!=M0xT=IM$0CAv#6L zZZ5N_eU=L$+H?BBJj4LQ)9E0$W4|rg3gz%=S8c!Wg0#874`?*DDVM?)RZ7YFY)QuU zG<)?^#*+~bX9;hiY1(1mJ_TRsEI+WFq<(#m=~?T?`|zio0zn^?jH86_hn7r*$VR8~ z*=N=p85Q4QJ4zJUfY3rv*CD-z&w%VXy>o1QYFhru4DTN5;32LU<2AK~jc|64OE!%5 z8gzz>xqbOSUHRO#DPl<|NEhX5nww-+DyyD3vs@=WOEfeE;tMTtY3zE8r%n8ro_gTy z*Vp2joAXjOw0rS8W~tuGLKd;4#QXz|-{xJg>tKu762=sI@ot{>Fx~PLv!3F8Vi!4X z8fGce07TgSttw`yaZn;b0eom=iL4|W{Z(eIt5Q-5k|b0oW;QZ7H=CbO3Q4K$C78*W zOvK5KD!^AixEAsv9l#wLnJaIWUWX4=1vGG!vt()6T!hI8@Vb_K&oI@-Sq{M+B|o5P zUlM7nWMKH$MTIs8K4iFVHoU{SBIwBhmSMW{v%vZ{L9O$>J>!wRxaSs@;llqc&H$Ny zsJRfT#v{a9y-V~EF%~QHGR=@foUT@o<tgY%jd~R&-x^0iZ7ypI32DHEnWx=1!O>12 zFNV8)c=_<9?;!potYSmzEJ?Z!LkVw4w-yEYGos)@p3+&TCOCk+Iu|6Yrm(@dbxWF7 zUwhjt1`RNV0mkn4WWP2mbh_cVb0#ORWkHw72TN--R0Sb8IB;tt!aUvSBTQHd?0$X{ z(ylV_>ej_h<Luv4ujcK`i~C~;h2<r-YAM(0l0{L4V;V#=<(GwJ^Y-xg7SD)X;5d~+ zkO_bRI@>q0Et_)7$j`u_SIGQ7h(U?|AKe`(<a$tbG;5H)_Rs2LTT1N3p4#fSs+KFZ z4H(}`7JsSk1)*YcC_fRu74V9bY7bFL!#HE0*6ks~>o%_mh8`<?*uFn+8kmThrU49T z{qr$1G?S~(*d0YTv4PAX<)|sI8k5)A|HlOoX}W;ef-!uFsvF#ZIFue1hLTO#4HAl6 z8-RGRin5ZHD<%!FfErgq@qkLP0G&xiebrMMML>mt?Vr*Q%U{B>&^t>bI|C<}-m+~d zf1tG`)UN%QB8<8F;Q$MoDj*LLMtXdZ@n6&0+!<+Uj{FVzgT0r5gN3v)BA6#+CB~H0 zrra$n3Q89B`0%(>wdTnGa2A**^<5&pG2ijV8%?Q!m{PmJC#v7-sP*h(`lMrUKzOF? zIJR|ObP=1L`V>906El?N*&NWeV6_bRq~wfOwADLwUQ+6NdF7flkI0jSerFzrug{*{ zwuTjVciC)n!X@lGa<<&Acb@gIpByNT=kU7ap}LW?UY*|~y|EOtV-i+h-yFYrNc<#g zH{h#ik~5qBIriiYJ4gONcSmN40M+;a$!$C?kpV=u1j|OR6rTtO^TF_-8=ns2{f1B_ zQw9f3iW%bpOk#p|I=?MGkKeiGQBUzpY!C}7EG9^cZ2OTwaT}<W$v=W@HpZI{A2F#i zrp42^DL-wpMA1pDwSl)vgHsuu{ONwC)%?kdb#uj$o&N&6YcZt}#Tx2HJAXZ*kxtEF zt*d1*-i6oKwOEBg48EV<KX^w8Zt4j7ZOP&7IK%m~f1P{zJFQXYAAM~(Mrl(+f2Ht_ z25<3WU}$b&QvxSU|AI|I|LM2@t%W_#aFA`(cmc;5@i{D(>p_0HcJR3f5W$mi^iLt} z$}?j<BC6MS3ElEKSE6A4&t$`sxQSRxw{sx4jt9Mn7Svv7hDFJ`e{s9Ox^t?(BOi2g zV6oahNR?Jcl*5}|p<)12IAs&e1Dh-_gV?onRxBX!Hho9bYq~g0S|LfEF(QV;u^oCV zL%7!CAZMhNdj;CvNIPcCA=;DL{YB|EO=meAQ71iPxY2Aavk@9o<t}GEi)iHT+b=kT zb0t0e>~O$9rtQVt-})J70G2Pr)shYB6SXnHD^9rjpiRVlQ^n6e_%8dF|M^_s`^&_Y z#4b5C*@Z>g(a<MMQ$9%~Wc_G!Y}&%gakDcZ6nFE57<xP-y&ED&Vaw%PDW3wi`6Eks znSc7u)9B@#Dn-{vez^7ep`0Ux!QNn^eaee!s%o=lujOQ2a$Yi*<=ZdxV1e)aJxa=4 zACWObM$W*7qaEIG>?pV1LO*;;f(fXy=yCd--Yc0SC<z7^{|v>p8orq;J~3gfM`E6M zR@6E?<K0i`@0>U*3wLh><nSE!vkj({_TcD(nmdX8f51qiM#S&2$5)9<&68zeEM;4b zPL9DPzXmkJ<4xxbG52Q50+56Q1fX`of3GXVlh-ki=S3;6153Wf8SkPx*B%}C@Xt$5 z50Y&6bl)f+e;$hkuk2wZ{F_9P2Pj#Uh?2gtIF(;4A%ATy{M~^d#IhVE%dB-Eg^lZ% zeFvaL1kgR|FzHj=DBqWkP}gcW<8r^vg5mMyDC18K)ezz6^OV|w>#E&;TUsy1@K1N< zx`<-OpZ?HYGW&p5bmD0?q<J#72zb8~u>$~1_1Cq><JQAu-pDxxMTN2MfRC1Z@ruTq z!4a=g_&=<qnVkX=th+z4qdQ2R=DD)!X`jydnc<n>0=Pc$M+4abGY$b(#-8}*ip<m5 zOdusdM%5>P40vxd09c>GFCf_Ml!}u~x@jPgRIkfCJ+T4GXFI|A(<xh1eh^LzOPJXx zcj*5sB=vN}td{gRP2LkBwRu`dFoKNhZx_zxIH?u)_jgH;n8p?|Z@`zB9RL;opH8cZ z-wmvnbpI6;O*^E=_R04Dmq!3(5@5JXDZ$8|=Rpn}<!z6+OyudlUrcsqa4s(T*nMH` z^KY(B8M{Z-T=r4B+RO?s2Squ=H~rtHM~a?kzPK1>$*2}Pr_A}%VfEI`&#5wupO68x z?I*B>3<OR5zE9YTvG1w38(ad1na3+*nwKc=l=@6QvLDzsJRVeJFWGIe=k*IkgT5AP zM&NG2jrR;}ubX!+SMCkHKIEVcF`@k+ISf}}5}=!^qNd>*(BUN8RX)FEbe~KGbixq- zbHLYpV8OX!FP562vXJ9~h#A+#z+v;Qo>wO-0_{WWT*W(zRPVgB#7$6pCUH&28<I^_ z!Jwe#rxtVQ&;K0NA#Z`r>R3<7>la{f>&4x&CE16V>&f;|;_ZcW$@YqXa#+|4BVF`S zk(rO6+%CVTPA|1~&S|#)CAo)5$J4eIL}GJi#fDziwRH$$(aV^}eI57nA>RkpG_-ga znDZe~gWpdjnu50;&&39`P4!P23iU`dy0{WUpIN{=;D8UOrO?ys#QxV{s-2kdUF)i4 zHrx$Mp<ABTr&ICsEuC(7Q6%HMOzCC^KxCU%)@)jPviX`;y#Q%cWDcej1VjU27RGgx zZ<;rM#_?0;j*W?*q!5a5<3-bP&+o+ICY3D(l20d7g9_1a<8OMROOiLHe$(nvmE|P4 zhfMD*KSP|JYC|%X2)hvsg1(`w^jx)T94V+ml+n)vV{*#$pDgk`Qj)?8Cx!stg2O+> zL3MQT*-8NU9XOPwS+I0<#~6b|pquHeg+uN!bk&sS)oe3W{`B<<2|uw=b*0fkREb&` zC-dEzzIlOm4~Kr|5PXj5r<t|wG7@a2^-2G9zP5bi^r66?e1TIrYd<&oBG^gBbnMO| zKx<;LiYLXp0uTH%Lixop6ZFLbra!FCxWJf`+tCKsM`~EK8!O0R-@t{v-~whsSZ_cF z)Ma4dXlo{<cRsCU)6#%^dVzUL4aAvXsCSX$i;xI!^?fhVe4(lyR<X<XO2y1xh!U*7 zt8VU|zi7aXelUIWIN^TuBQ>*Lc7{tL^ly086VBTwmEKmfe|K!XR!5%wjLeXcaZ0&D zzP1<KwA<4(xR;(Hld0h2n6WlMK^E}`E+a$_r*e{hVXP>1@Z9=g)ml3rYH61DsA}L0 zN2t=;>0Vocp3}vFyL$nSIOZ<?{{4pALiQ}518p2t`jm2|8`*6N88u!b74pv|6qCL? z_;{Q%b*0=T#2=|W@)VEA*z@eOAiT<b6);KjYbV@M;bWd36=iLlB|5~l&LRe5@aFK| zfBx-!SHyy;!~^f*$tt%GSF5}^E`J;jZM_<$m$ovJ`syP#W(Y#kXZ`5);(U+mvpa5u zj3zFPAF+$wUsLGHv)#|}CJCsXi*V~c?xV&`mhj1UL(xLHzb4JlhyE^9a~aQDNcq`9 z2EDw1wlGJQ_$RRS;3r6a+=P;IC>_Q*z8&%9k5+5QX5&iAH}pyXra&B9R;awM{#2Ju zs5;KF_|$j6R?C5XU%Vab+<2gJNP~^-Gf(!p9bWH*mu&Ke9}O)oL*35L={irOqICcL zH4}aoP}EtzOJYg;Lf0SsL3skdmIhti4#Ghd#=r^}&$8SDEigx1bQjQ0Y7@!VCM1X* zu$7z$3XO)|(zm%l@CX*---v~{MQW}rTu5wNc^5vuuUA!NxfPOeGI}d?mpe0;V7|YE zB0e|%-X=rRx1Q)rvbdP3vt7m&k+4rbRHM(2YU#8I`QmOip93Z*MX}!vitl9GY$TI1 z2N?#j&^I+xDEqlg=4~%~d@jcoFmf=pEqIW>rU{uoi|VcDB^|o=l7#v1cB+=+;^LHD z)@d%8|LB}oW(}Uw_(@D6H||wKW9QJMLK}?3oQEtzcr(}+^QhN6xMT&@U7;I6GAEFg znjcJ>hsG=KWp)&G(LGQ)8lGCaF}PgqwB>Irmx-{N$%eRD+co&gIS1UL0jn8p4fTP% z>95eF<t$tViI~K5EvY}c%yES&u#)516{IV@#VX3C8)fz~UR>&0<XS$|`JfG@mnIZm z%Dwc-twr$xQJ<Q_7xf&rEn~rd$i9_3EsRdB3e_rLn^!pNrw0-U(m~#%dD0ysm8c^p z=6*PJzQKFD>_8o4wq1Yw+Qlmjs+Cp{>o4rqEp;|;REL?2P1Z6l>Q>(R`YU=}#)#K| z9}%tmuCMByY-Xo&@8@?*T;iU9^1FuMd>RlCjjI`n?{Fss8k$Lp`Pq}wz-rZu^NW<a z*8l}rI<)YGY{VtY#3#&rZf`>C^+B?ea!h%$)1UH+THiIt&SVZ)yr=S7ku|z9b|k|( z#E*>W(p?{OAJ7z>FkAFzFLvyCM7y5`^)0O&Tz%-SlQ=;cLGXjstjt`<feo~lVZ|-w zPlcE+g)_X<+`p>#Fb(_>=M+{bBA864_JG6tj8(^QLF)@{<nO4xzRtr8hHJmJJ=rGu z0#X@A&;d7I>D-w5`a6;lBB9<T3oWEscWz*WFoLh>L98ZQ5=W+x9TvL}bd|@**-FlN zG|hRWveT-%_oJbdw;xvo+(fIy|2#kAIMLF170D?$dGy$0_V!)`SzOxF$3fiAI2C~n zbRj>V;2z5R3MGd*cQ`9@50}ONWJ`Tx3iJG3`;5^sxxu%2e4k#2z2bEaH?_jn{S%&u zV5q14pdQ9{S@*U6h8&#1mc@kT7jptfe+2h)SRk*88APk#p4>yphp!_4?5Gv5z6iQF zb$8H>2I`a0fxM&OU903Rl2PMYrJ5Qm!Sc)b_$tqv%X{3LZd@=>2Z+#mUED^Co;E_o zgze{3a4t2D*7^O(b}E-`tmExmsU*gmS^Qcu`GE@#&zt>an6-l?6;BVbi_>2I)TvzM z8p$Wx7q7hC5-=MOc+ltyMr5nFQuHygMW*6MP%y0CDIyc?CCLvY0C87EAE}HjznL3G z<x=Nm)3Agc3O91DYpTjQb1R@|FM|?1MiwTRlYhS_ny-lyFg4V!j7#NKfoo#c6~dIJ zDB{yO2x@DXhS?pg%3qPyIDUGW{j>8qsJZ*MW#SEi->}o>))5ue(dNd1W;n?7&+i{+ zh0H-l9^vu=P=;c;>AURdtGBl`609h#Y%EHqAZfOE)1KWEb7_TDn|tE5uiTRs>_o@f z-A5etB*K!+L`GRN9w^Qttnce(a$kv~4SlE38!SQ#H9wB8-h4ISQ#~o9qS%?m)@!n2 z8^_exUVXJ|o}Z}_Wo0IB_M@}vmKx04#|nnXX|{V6pIhoV;H02bIIg>UAzD(bgefkR zZMc@n?`}t1&Qp*1yGvKApqn)<4dVN`%US6@Pw*X+Vty|OIyhKp(d+_`#|71*Cda*Z zF|{uO%KJ4GSd2&9Fh(D4__A>9R*+}1%&=A$)%w*dXSL2nh_zM<)>gZ`B0q&K%7V<R z>`E(d=cp_m+`wDYALZ3+ADi!r>S9_TeOs|8x{`nmTcDWIRbh@RyGQ3cHh&CzgVo}0 zPWAqM<t)SA6srBUY2wkC(6CofyHA&u7%D=qBMTB?dL?gRcjdzs3)uv*NnOdHv+hq` z@Sq}9`Yfer!^efBYBM!cw^P3g28xVBtoH1eR;ZndrLoLP7Ib%Aw{NAt89B@*@Z-A2 zUoC4bnz+-KI`$7ttH~t01Pi4$B_2U0OSmsEHj}o`hlu`(x{VPl{c>BkWu7V1Y^2%Y ztgA`^AJQt48?v&;-O0aJ{1pO`@#pKE`egeEoq?rH89y(oa(Lh~asK*qWhF=RjcmS( zVacjJ!?)8RjVE1&2JuhCa@hsDl-)v|a=!&fMkcpFXivTwN5;!mG~-5rF8uHsLy?el z(Ce41OV2R>dkUanp_v#yne#z7>-x)TI=}R!rY?#@l?}k%>zag4&9V%-C9AA(vL)SW zW!LeP?d%Zrci296<ZiZA*Fy&di^uMzpyXjq0(#M|x7Do}b5By`WTJcu?z^>D$GGgg zlP#jN%5W!FN?nvAdeRQuDPq785UO5D(E0pT1^Da@es<>!QKqgTnTf{-Es>3cu6o_x zA7;<q@F{r&_GksmOM99fv5h$#SeF3lJE_W#-JKw1mlWaR``?y54^)X(A6&35gJ=C* z$mZyYMsBaRW%8KkqSFIh9Z|A`lsy-P8m3BALU)?DeI^^`!wK_|4|k~LegmV)%Qt3R zFu2Rhjh*D*R!pi;tqFA0IX$`ZLDI)HAi50(4$+;?$7y5NoUlogxIM*Ci<wWc&7QOg zO`uEImqJ&LJ;HRKH-Pe-7e)UI*Ma7c>^k(^C#=qdmeaQ6hcSHZnRD$<Jwv#EV1mzD zUSr@i2by#w0$;~=*f4Pl(B&`kf9e3OLPKK{1EySNn@)6T7S8{a=UcGjz0hlMmmt2} zZg>4all897X4z2=VqncSV;oWBQ{vFr+pgPNJ6iin#WN+c@)Ux5(^4~BolWFCtY7)O z^Yek?$@*Yoz@!G|Gy6?)qTQzIe2L}W#b6?+9L}*zcF5%B>X&1E<UKW-rmw(s$u-*y zOU*ko71&~8rPULM=W{%x@jCFf-OzpE=Rt{#_DvnGc$Vxfy^&?@#&+k|P8&>vbC@g9 zRo^_r{X8A9OY!mGigLOxVWt~9dU<ot0^by?{hNxCby&vO#?962&)9qtsudJ6q|<Rt zusxI5^L$`YwRVs;Yow@M;PIY%!_YMb(P5*A=}H^1A1`iSxHgs-wseI+bH4&KRLO3j zIbZW$cgzj#;5x3Blwb1kOw~m5FYU&V48=QV^TG19y0lV0ivN*Ds{=aR#Ap8tW>?q2 z8l5>`-0$ikQ}es^UZ76q#8rUwalGa;sYo*5VT*HJ;*gAA{`N{wmTy!RkYpnitVfP` z<G?kYds+1PqO`|*0ov62n{-nuaj@8#RK@$74v(Im3+h|GQuSX?;8}8sTjq!BBUZ&T zBY%h5U<Q}Wlv)PxU8^Iz5%YDHyBD)4DOmFCyS8l31iB+0aVSH|<;dR!RL(=*jh(4W zH1<^wKVLYf@|9^sooTbxP2>XM16i-pH-FAH<hFn%YjPXw)4OZpnG}e~%W&t3=DUk_ zk}Z6%;-5E$W-6<Jip4KB)=XRoaK3k&RO2#(b%I#usx0gv`S%)^iB)M^|25tcQY3!y zn=CiZ31&Lvl@O<YL%2%<ql4rXN%OnDbLB@RlTPc06Qnz`)=^U&)1tYS`k-WDideIN zds?ybL|NwvlB!y6v{Q}t!nC6p2uyQI)5fi@nS*9*O9+2HUyf~%C*8$*-a;ffQ8@h- z>sJHU+@h%(zFUuOA5QTj57(4>o%)PiAO8$7D^hqzy{S4{93W)v&b}Ccq3gOoJ_ho> z*tO%h+y<v<4pUqxx2O4{cz0nrpURALrjb6AF_O4q=NUHOonk2Nr$!Nfreu{gk!ZAC z_P@H}{#j0&Q0GN%32&6x;MuNlfVgxPuGUzcMg~@y4&J!JN*#+cyfGgrqqi1n8qGEq z+2~cl5ad?X23zwIG;8hE%j{q>>u4}L9($QTt1Gsb(AH*z%cd|8+gQ|?Jn=3xP;*<0 zUjH+G!%Wv&h}c`-n#1KxJ0;`+flW`V`z)$oaEY)))R<-yGH29mmS+`OCP4BN<;Gbw zAcv!US8c8xv!w(x?JT+ra}<!4<V<PC&fndn(GN)B-j;V(Cw#815HIu-Wj5u9>UKCk zcgvNM85}vsZ+aXxGu8v?Ld}9SzGiM3SBH-))0NBdk|=*a(8Vpk471=_85&(6iT^^g z=`}wI-O0GcC6h3rCdVeaRcdu#?<=EZk`|E2w_7h(`puq6Qdg!t2O`UC`m)58gCF~3 zH13B>KH@*RGoL?#%o|JSTkZ$yOYzS$NwEq2OeKz0dg?>mG}FEWJSnH4QH@r0S<rKP z`jY=Sw-FoE=d|mfm-ghT4HR3k|H*|X-N(fEA|yU^*@o_wo_@#!Zoguu$HLF?b+7Y^ zi0?CpCtdiKhb64o8@CvuWVd}Kwnn|ek(H*aK0dZ>vWZGXG+PDj=M_B~-5)7^u}d0l zz9`_ox{5t~hIE~W!GH*i@YnWeb=U_4VoaRK;tRCtHkL6J`JMCNyIWRcT*=8SCLnua zHfg<e#Q9HGm>X|ZSGJMJAdQZvqsm}5yqm-y+I>KBM7h!Ey+mDH+NI^wV?VH7|F9BT z!=v?9W6)Xthlo+~H-;em!RdNqo)1&*H-?<|s~J_2`n>H}2&sbMAxA&zZS~!aodJRW zn0kZqyaZ6B7T)=?<(+Z!ctm;@>>M$5&US9u&gPA!xcaq1+;+3-@ocUHhtx=<kr&gE z?XYDkB4gLL=~bq<S3`Tf{><_ieED6|17G}5HMbAXW_gF|;QxBoIX|;iPkZ!Iv*+`g z@y11@W&F8!k&sGfRoY9{QzcqKHSMzp#pEE&3Lm8_wW4v=O~S*30(~<*<}{gE?@^9Y z`GRC5I#qSmPuk88$<9=)YOgAv1~ngSm`^0GXOnaH;#4jC^B5DZw?X9$fwEH}z5IAF z*{t>jy-Qi>ov|kIZLUmiE6_M=>%D~=9t{ro#A|>JS_5~EzXIV-^~?CCW>#vB+yc>z zLv%#K9e54{e2Q|+ZMJf{?&;W_o<HInD`=*TaTR}?=PjKRMVNd|HtFj1A%4M$BM%R; zs%G9-@lChr2-O2k6Wd<Vhw*ZYN(V==@V+lq6CeJ_v#8{P1?MM8nJ+7_yq(mhd)$7N z(2eRoSaD_zyIGU>X3Zq(YV9n|FKkD_35rYY^-U!`!cY+JtH?z^kQiNxB~!|$?6`5c z$Ipiat*cG)>3Va}+zA2C)w?dfj{AZ0#V<ImXWlCjiLjt8C^+E^iE@kqICzJlx0}qS z=}=N!#}qAc5Fe9J$g<>UOifTVgu^HSrWo4z@~k~k>t6@c6VZd|g>*4J)`=AKJLU+r zc**;K<6`o!urB9xJ*Ix4f>$a9g8Ip~o-`XIhGVh}Zy-zett2#WlC0^nel%T8uA@&) z@_kL_fV|Fh<q$+XJ2T68PK>97u4heX^9hC7f8to#m<b9KPZp&q;>LCCR=G9Qp~e^d zo{L>^o|HM^)-x^-@8sOqc0aoqToV^e4_WmYVaX0$V|&$qYtOeIafOh(P=g?J{dU~3 za~JDV?@B$$*&kv$*^V>oTV`k8j-leGD@Jj_Ta~_R{S}?(t_wZ}m9m`>jsJ&QOkd0o z_E-w^<p^-tOPI&KDF#ttR==O+zSEq$oy!x-F;U1B_3V{Y%fNcJSw0>98&jZi${(dj z*F5Rez;zQz&%@gB)(vKp#aAN0VDrw+SIMXRKuz@k7@E2oUscypGyX&$(SW|~1oNlm zNWHC6c5C=scBYUt;e)%OEE~w}3Sh^9Yacgr1;uPK9us$vB0Q!QP0eX(s@wJ48nJ2y zL2UQb__q~mFH605ItEs-=&gKkd;``i*G_j*Jo;3N0!e9ja;kiC!IEs&Z;6kWg=>L~ zAT{E4lIRfbnS#N94)G1xuwO&Fl|1b`2_ngS%eZM7YATa^av`1kZi>&+Czpa=M@g1D zRvmuUB0o56Ip15)$JfBU_xdQqX^`U$Ods>Dc{!bIbK2)Bp=+&h{Hx6UAyb0xov;wd zXFZHD%`cIkTD|7wd|C^SieJ!wef~@pbeCIzL*-}RtHkn3BKIM0&wqTzX!o_PK?fgn zKcm@W*9X5<See}!IIj|(;TK=lR|=Vav2<hI2<>If8gBM)wJ1D&mZl$iTb}I>_xnMw z7rHQ`tEYMF5~cQ+`u~(noVH$L*$tqaKhWHY=7*WZG+lRGxMN0{<yT2uDs`AGSk14w zx_X0VsA#69>toy}q@D4Vy$X4A1+IvRXKyGb+WTSGP2wG!KiOv2C{^Lg*Or@&MP~k2 zW6dmTuU&GcEtxj;g_%?6hB?(zvGj6gHd9ZoS;t$QZ{+_iz^2L+w&*85#oy2UsMB(! zn!z62&kJTi|GDkz9zQOrO#tdCle%-;F4Jz4<{3j>E?t*smm10A?}yxyCXcqNZGU7j zAcFS+==qHYHxrY}eG~ggExnB@EE%oH3AfLJTACudqdaVCiPB>)-Jg85epLNXBrP1g z<9F^@Gz@W-SC=1l<|u-q#3B4KH$_P{^=aDr)ZW>zvPhTGf@`8L-0?2k{%36RsK5Af z1B~3GJu0p5Sc|O;OR&@5hy@4tyD-H=6Y=ADWur0C*TT`<zUV(U(@(QFV|_B}co;C* z{i&XliK4}^a&+}tqh_QfY@lJFe<-6$;wtk2!Hn`AuCR_-g@GV#86aS(Vv2W#U~zxj zl+iU{3QVu%2ByH2#$2-BWl=Q@F+(+^+1Z6&D@QMM1+eftufPFOzBos#vZ|JPT_&?L zP9*5;0`nA0lHcr|E*tT9I~;9PXnj8Fg~^P!j>3{A=Q<xg&Jm@>!Lm`lpR(y07HZaK zZpCyADWeJE>zHFXUmc}v)L`C1Kjx&oEYXkAQ7v-#{rTX~!+t;mIp5c&v|4Q<uMC35 z2TVkw{3(O-?6*&xDV$)~7Hpy_GLCP|A(pQ(IG(5tnh>EsLqc}M#syztb^k&%K*n$R zZrO!YqGr73_V8pgLQ0O1sqaakdLuV-K&LTopaDQOnD5p%&a6=&gUwer8s1T@eEZ!w z8yd?-VQXgi3weq~cF<rw&W+cZ0Zjx=1q;o@@%rJVJz89aiF^D{@R34Id#d{!akyDk zdM$gOok(r)nwgqK*anr%?2>Op^GI6FmTNq+JXSmXG8ih@&p1Up&#<V>!OoV<^YSO0 z^H&6}isKo5PISxDvH#F^*BvaXUV17Fi6`|}$rU&&bcLdK!uj34F=V5Ka=s{o4AR`B z--j8q`V4zjw3yH077^}gd0yKu>nfqx06O^HRW@SeL`YNc*oGs`jq5Byjg6pHW~H;s zi*NYu#7)#)Z%;KlF8isJd};<=-OC&ZcJGV+(P<Md|1>bV`gbZJtfVSsk$6qv|F{4n z5A4>1O*C(#YTiE6KV5-4+d(st+sJ2g>mYCOgfW~kU6-EW?FP}3Hd@MVab-k;Fz7p@ z2IEXsir-cLDuxK9DdSk$g)Eqb8FNZvyVnqji70uHeeX1f?~|}fLYkv~ts-sbw~rk( zT|AbRPaFcis~Dgm{1h-SD*j!)q`EJHrkF`pd90u>bMP7W1kQjhTFVKO8@*Rv9f0mv zs73IH){HTAL)=$Q^W~X_$6KCM{>M7F3)j1zugyAtZYvllp>qp-F#=<T;Y0ba--~ZI zJIpSeaP#S1&428wEwNX@onPz#`5IglwjxU{F!C>9v{`8$T+|+{^->+#=y!h;gZW5f zL`Quh;LU;l3VAf~LS@o95j%p+yQPU1UwIyU!QnWM_enU)hnlde`CMguVJ@|r|8&78 zg7ipWuMD%lNF#NU$Ye3L8uIwM$>}1_Z_-kp9-^84VbOE<gIn)-KjClDaWE)G{cG19 z_jA?Gsd@N`-~#ix7_$s#GQ!cr_-{5L(Qe5Q-g~rNx6MZA4wKuv6+MRA7VhkrfjDO) zi&6?MOFg<B?OS>Jw{zK4cUXI?OQ)*88vq|p%Y+nAL5!h~5(6(wz2#YOqq$HCDqC+1 z%S5X&f>one*LB6qrvs?3tKCy%f|~t<h3dC<$}0A04zz_jpK@<*tL4A*7yT3GM@R)g z_bXd>Cu0)5CS^`@Hnr^gmzvoz+Q&|t<J#p;<|WK``M{?lIuo`hfkHhZ$E<c2-%Jks z!4y4|R?R18SLRfvbfJMDjQM1SQM2oKTnQ|r={#7~Gk{pn2_SDPb;Q7qjH5We<8>$B zLAn|<d)>Gqq<BR$ast18h-}6!xZY&w;-sA{Zcy6V+sot;kK8K((Dyj2ol+0_u0^4@ z+!|j%eP*t|vqmkf3(jFlWfGD)bSq69;K+VGR(74Ya?T*3Gv0flIz!hY9nm20BorUX z8l+G6E71lf&P-rf8-{;xn(z>DWe}>v)r4w2Pv^|<Uka&wq^Nag3vxPn1P)fCS^u_E zX6APAV^bjw?Im=+rj8w1xz=2S*5pU_`s-0!82jm1*G-=Ib1y&kD7xA*e?i8lnVnia zas5xc-0H9&D{3Mx#a<O#jJA8;+8HD+(LclObM3t3u=P&Zbu+@D#a<jZlRQB^UxRaD zVwx1C5<%YXoZm43RFkd9;u1)+#O<eNZZu<0zkOg2)_Riza$%D@cv;ritb@xJ(3|41 zVq)eyD6D&6koyE`b|6Vh-E|wZI{m7sg<sq|hmxPVyGGhQr2V@3_`x2PO-S;QA9C(r zvDw&)W-05ir!<1@k5~w9=Yfi@xr$Y{$_&}A+KP7>y^N5ZR&XVcsTsL)>u0nlx>(Yg z{BBA-`#Eo?5<9s0_D6ysq$HNf6Z5gEK@QG+X#8@vmD|l41|Mk*3#?ozJLPUskv|E- zUzW2m@^pT{Dzhdu!mo7mxlom9p*9Tu9y%WMU_?otTopL1hz7ZN%#S?m!6rxh?d;^1 z`9~f+$ti{3@OXhKj5YU=NW^CGo3AR5b+4V-?|tt8<Ea6<#&p|3uLDdA7C2$o&-U#f zKIqFZBII+NggJJox~ME%ZuT8zXPlKnyC3FS+8IVX`pre7m?Nd}g|MD&G9=`k&(-5Y zQKs$)ekcFv98-}DdgvFPiH>X?Ca-7PyLK#nKK-$o(Eo&$OCMUUL1Xa1`GN$i9w&pw zouDFuj0;bgmzki5Cm3cUVcsz$V{GosLvzEqPu;v|d0-8C;;~~Xa)IWi|JgE9D_!Yu zGH17*?lO47Zd_ssIm=Xp9WLAZ_~}@?fw9u0qJZc;^J!K5W0guNXyXm(k~B%5`F&gy zf)MwU!0azds8LF@XTLodJimb$K`~eXJQ`=lX#kzy4kp$QpI52g8uiC0?|lU(MQOM= zHol{29VI(*%3an*89EPvi6IScc=+4u`eGthnyR&-CR}f(eM^`!M_agEXY-EK!4tQl zijchmDslLl^~Sv;HcOeJ`n{=G%08P=yjEiOcI68G+1yjzDmn{ZeFeQQVK!xAFGWCE zCN$Fg!)U(H=hMtO5f0gd*rKK(IGai}$bX87_2H**$kpW9{Wn?n7#)f-XTJ0=n7$+w zsolidGL+??0}fwU#Zs}4Wqo|L7+R>LS*3`jS3B45c_i@>lGua@951voJPG1Sq2peE z2S6XD!3h*y_pOK54r?cP24@EYK>bT*bY4DwLwDH98sPzJjTdDWTDiElp&O+G%H5q+ zUiU{d-s*CnCfPW+_bpRvT1sY#MZh9gop<nI!aO>UJKY=AXZOaitxHpKh}a_M+K_i> zo>qY-J&Y?F22NsU<AO@~fRWM+^izBztNH~q-~68A?|TJjOV_e=&d43u|LJ!KAnnJp zVdyWpZn~Yhq&pKpkLoDoiog%2ZfMG|yDs8K3}Uctvc}l4zxU+J;DNW=Lspvq))xTD zzRMsMxh^i`E%D(C_xZM6X8j$vuB#&N=aF4<0pky*C1g{i0!TfoRv}-+*(0KicKgbV z4%s)}{Ajaq%^#)B9=QcsFSF_9bzZ1{AFnh0@(;5Q?wyj5yq0WNnE_ASCx*&z2lra7 zw~*Tn<00M)7{}TB%_ZYs;MVRzl|GV)%npRN-D(*6x#e(xsQT%AK(AfOM7;O1v4Fh$ z?5~sGRUalbmp|T<fp{wWJf0zsoLnM<=(fAB6O)xgS3p1C6|+Hff2Nz(6adHZ<g+X$ zDw^scicv~@T2(W(IU*qTGocDBJgT0&Vb9FK%w)*JJfTc35S@7A^p_z?vzE^Jvwcd) zJjQEzBn_w%Q$c~pcEy^S)W_(Tw?}yWkA59wi*$kiHVAc5PeucFsX@Evg;QZCP7L>j zz2aS*82By=F;vdj&gT7bB4`8m<BkN8L$?F2?+x@&WYu2B!1{4?F>;n&oEr5~@10*- z$-4<AZkw~wEhfWE@|8jUVGgs0ft#k3T$BJ%jWv};2n-OnR1CWVNeuS%?T@9+Z#*@d zF-Hfyi#&I46+Gwa4>7oKkj71Tii_?zr;Cp|vWlMK+{s)=-LMVuZHnKj@t-X}(!JtQ zb=dcS!j9rC!!;Q1|D<rc!tF6DvE<_l+p&<mfKA28fpe@U^D;uB3>RsB88_Xu+%M4m z^NgG2=hy<&<s04IZun?gnC$!5cP}4uGZ%lXd%{|h!@-?=6H&q?WSP2MHj&7^?M45Y ziCT~W%G3-tJFTHjnB8F=`t#!z>nL)5FT;1QUq1fR(Gm|PdH$dn(LH*dcGV0xiVgHW z?@TAFrO77FGpV*Yq`Lx^G%4}fs4;ar0kFK}orjdH=TCQqdg(Y08)-sE6iNF2(~9IP z|BtZu3~I9N+JyrFf^-$>O|hVW^dcQWPys<yO6WJ8&^v^LCW2B`L;<BqQ@Rj(6$O-D z1EB<@_ZE_n5cn?d=h^Sfp8d{zKY<y3Fs!U&9qlBS%D)fGPHMW2)LvMmKY7^AN_L2} z{en;Pm7~=76fb|&BtprRpjM~F-Lw$hqv~3VQKN=fJ`2K?MK2mOAdvBYJV97$9f9Av z)S8JF$FCSw8lQXiN;S6@<j;i8yK}Q+(`5SU<lE?P*QWF;zd*E?jXW7=JCINb9BYn6 zy=>^c%S*<BL^ETJsXsdcWv-MN1#vZ@DQ;A(vooF5E%^XY$cctOXi7Sg)biGf3MAdW zN*=#tHg&0zFE92^^)x^WRbLUFdDlyC9-X4cmPaQ!<|G*-RWx;nN+ntkEchh+7gZh9 z<?X~c!|9RJqF7`|#_3X*NoIQ83a@|PytC)qLY0kvgsTxjM67JRWC={nK&hlI+Iy8n z4>U0siLb|V2o!AV&4>4AVP;ZSpq>Jk&2Ig4WLB{Tn$U##!VS#ToL6t}Ik)|6Bz8i? zgv8JeopL&VKFI1`HTbe?%eQoxzoLt(H;+p3v-KG;8RPAGg&)u3PCR-YGX1B6N){&j zLujZmO`OML1w-Y{1fSNcXudrgQq9D#HN^sS*EY4zo&814G1P2c6-_&hS?3#KKC~p2 zYiQ9v1WC(+WBu!_5qeMJUu$(tUgzr~eCK0dsutKN*tTH3`1wAc?HH*2O}l_lJlH>@ z)`IUxVPnUj^>%TK5$j2L%;Vx0HCabR<uJOhEo$uJZ@$xH^$m(g@UcGs)s2<e4d%}} z7Nk8BmV`|1j^EDsReBHq<fFQH#I54qMeuq0rb|8cFRd<=?$A!QEPKyDBhAuI4+G+p zA{|d_LFrc{AI9s)y;jryS05{#o%8V;u60U&GTpv{FU55{$2k*K^d!QP&-(dJIqplW zdu{AN{daVvdvvB>oZf4P@6P4^w2m#G%Btr~4;ryXH#+HbmER*jra%*NSN$%WmD07r z)togN{KEY^Y)`+3Lev~u+G9P>raYn#_RN3!ssfxR5QdU@UsPIT<q})#Q3eK@Ncdln zN=ce+=P~bpuL#^?^zcCS%13|mn)bs+I%GjSQ75HNiK=09W{Y&VI~SB)`f4#diG+rh z!Xk?CjBhrqPQUec3xG~v-K6R<@;^nPlR!Nay^QSla&$dEUyA5FD4Vw*GRS1ErV)hA z%V2`U{b?Pc9o@#N9Lf*zu+5vZMP=ZvIEOd++c~Y!oB_u2yufz}sO*!a8>4qtKg!6y z4F@`O^HEeMd7KbGaih$8u>N`rN_LW6nOKrsloyH&^BGcpam^M(jjE-&nOkA4S{kU` zacHufJD72dk)B)rOR?m#Aln=TjN4)w#r8x|f~&DQTFq!>XE#_@?U}eO9K)gJ|2@xc zDVNi~Vwqc6jU}yW(NT3z?#RNv>wrZF&5=(c=d!wq+nTXdL#sLLIj>nX58a4W&1Gi( z5MS#e<iO}{Aa@t>a4K!HQ_U9=n<73G$bR<)-VXtyh}SNWc>9ZIek(NCm~~|l`fZ;; zQo7^OR1O}tYXpNp+O=G5Jty?9xn0k2)13SWTJ0C1Y&tK_);quvd7!)~J?d9}={j;C z*K_`*jz;fDw;hv9I|7>pSaQuT>y#$n4a>P~3AIqv{?I#Q$rH}-{W8th7%<G<6nx~i z{AKnHtJ!aRtEa+Y;!1%=$(No#mo9GM6zD$qa5Vi!?qjXQly#=59FgnHh=rh9FLDi( zpPLOz{(dz7M7UkH**YrOC*gLBFCH}cf^jGIz`xbD1?6JcyI?t*RuC{6DvS4Ps#M{J zZ&{SF?y0Tanrd4Sbz;Kl7%7-?CNa)+v%Vge=0eZ=Dq*imLSHGPkbbt)fTyH+aW9~s zPRVPnCGB`{F-?B%A2{~o9k^uI8RFFS-aOZCZ0uS`ZpY~!fBzaD{;!%6@rbIgh5;+# zQjvAuyyR(Hg{ZA*&Jp1RXH-I~aktksWH55iWUrcFR6_GQ<lC=M+gs<5r6_({C!d+T zMtfCu2P&IAzS0+sE>6NjKc%yJ8Py6kdK_LPi$h1_ezJ~Rtg1tXOEsL?scUL)^aJhU z7_W`py=v2N5nigVc~U~gS!qW<m<A_N&yzp1Egw98a8<pwlyctteckpxqS5opm_u22 zlT!rcF*Z@HS%{i7nsTxLNh4lLKMWhRbv1<sfvBHvlEO**^Jf>@ov^NT;vH?-+Vtlw zB9GrUDY6?}RucuC3p9#j`={dQYvl79wXL8Mh-dI0HO$5Ipkc_;eX?T-o_*#B^g%Xo zIJn}Ig8xiz`Gts#l8cwk7}^hF!Ko+bSD6P8CA)aCDIcw5t!>!Ss(ij27=JFSZ~1jV z1#OL?x{Mj#4bUZS^3Ilw)Z5P%x#Ht-C%iLlDg#Kso5F4S4RmmF{F|r8!#bz<T3BgJ za{gQh)l&%?&VI`WWZU(i9cw-Qp<4eod9XNIu=g@e0jl6q*cR>TUum3U<zRq&%#bQ< z2Zgp?-)Y(vI3QjIr6=*vc>J0#<(KfJm00~NJ49uQ%anb7WOT!l!<KhvP_?){4-B9l zMJ9Pygn~laUL)Q2ge28uIgb9d=5c(8IUrG})jUrwYwcWcjK>aro46B=<TO{e=u5xf zyyT;`8LhIjxI0m`cx2Y~Xat*a&zic98Ib~7I?hrpLA-DJo=Q*EW3MLZcXhgwvs6~& z-Lr)|V@j_tFLhW0YG31cd0K>=N4zs15`UV?F<T5SFK{3J5ZO2(7}h4MCVm+fKQV^q z$O$M{+n)3je<zPp`;gB+^3ppS_o?fEc){(8%dm&pG5h=}mT*rrn-wPjJP8!lWA<c6 zvV}(EipGofd|IJ*(i_m<hgmQgTv`~NrhI)jxYMU+)a@d~Xn2B0<!ehtzSqs@TjOF7 zH(2jzqeyY=e4%bou?)jyW6(uSGx6v25LS^7@fza&U1FM5xOQq<v*Q-}qPaVSM%*9Q z5LTqia1VS$6$9o}Ei7+laBo;-8V(}bNwUC%uZnl9%H6L`3uut%37c~h#JAf6O6MWJ zX6QDL(W@^F^@)Dx?G*ZFwx*I`SwsI)qK(uZ?0Zo?d){%#0D%47X3xPMp@><@0>25M z%j|sOn(S|=Uu(4(0q<l-99+;|`4zWOrFH8G44&7~&8`g=Z;6d}ZCh~ez3f`SdN1;g zi+m$T?1(B-mm+|~Cvihwsj%IQ*)BumZo@IPwXypp_pVpd%jz!{qY18CI8wW|%k-Zn z<5B`i1DDI2)36$OX$B%rS>3D%I@MXQ*np=gN*wW9A4UWNE7b1enig-~vZhRi)a(Ui zVLyTHirVg*+o=mp9qDMxET_R%&y>g-oBz^lLzi+G<{Ox4-mI&!QU&)ma+)7ngbjQq z-y=?uRGCp7Q2CLdR%d`WmBk*mw}J-R{-FLl7SS10J=Q62qD20B7K05ovM<f$0anFv zYHc6-hm4wcr}@-yn2HFa?#2x8^S_z_+6h^Rfopz$)6c;c`mMA3-JNQH&?~r+9p>xO zH3N`huD?@n>RGRH%b_$rj0aWz6p45U&qrORHh3tt8h)lAemY(;;ulWTf_iK`wb<er zhY#!7Wb8mIg~}RMGF$<qJLr5v;-^Zmp|W+efY(+U4vks(a2U@a{&2q9UTN6uGk^Hs z^M|h0omo(Kerg6_L7Uf9a$qbqFmsuZ>A9o1CgezIXGYavAGrg}nuCRv$w@nx1%#kK ztxej6i=AF^o$tS5xO!o1bEq8L^P3sDAb*-KqA2uSaKZp3EXEL{VtZ{+$^DmNAM>&i z?CQna+nhD_RV8EP3UGhX_6(TLW(=W*?OUoiM(H#oVVGk!1ltU?SA95ds%n#m*U)0f zREkhRYNvI-H*(8fQwXHxx!K+p7UL$-xR#f7zWjA-OU5N#szJ?A47@LesiI>LlzG|g z<<OeY#f_TROvNS<HP2QSpn-FkIAb%W9Tff=ByU{0H_xgeVT6bj*ZRU?JTU$?I@_;t zADL??l#nHQqa%=S4wV`M_cQkvH{{k0CJbacqb&>Cuz(pv)(A9l@net?f7Bsga2G1J zLQf@a{GY1|civxuW_EDx_LhYZZK;W8FQ+SbWh){RhaWw!TOfbs>6x`v488Q<|1S;p zJ(Zqwe*z${n|K3gF(N}3G?F3|hcfB)FZBP3U2chmb|AAnCc*irTTe=2sr^A&=9*Tx z*5Vyw)x8YP7jIc&VXP0}k;Nim`Ut@j0Jx-gh6@OmAM4%5)F>LwUCF$h$X{GE5u1nD zb3;>Kk8`lgx8FEI6n}n)G<rWCBpj#(&#HPmzj&W+u}4e6WowAkB)elW8=?Y`n!^-o zes!Wb@9J>|xUu6>OmZ2i?xTWj8*S8|!-*$Zq^yAcB>b#$(1P+^;*9HYd1e12c$oO! zr^SO>J6Bu@dJ6a>`O~Jopk|X1PoHaGw%9)St$D1wqM}H|l80VWlJ3}68X|UMS^T!S zT~_wXOa-UlF6A8AoW!hOjm~j9Q_YA~xcnf}u*rE1bWn`h0aJLqJpBzXf~FWNX{+ZG z-?lU441ecqt^qwZ<M?gn9_LV=U;E_4L;E@!dW)?WwSEMPc^C%5dcVg#_+-C(^rE54 z>j`W4b`1x}4$6kvY_e6StuJMR=36Pz?fFruSTzUxgs$t^D6xxASa`L_IuCrjRspZ3 zhF_M4L}W0BK+iltJF56e`}J=1Q=i6_4A=z~AAIZa;pi*@t1h`M!Kx&Jn6F>d5&DCX z`%0)ol16dKMIWUD3A)Kjw-9><ULzd|j|KD8nA9z(9@b=FawBv2?a#`*5TcYt_0z=* zpXLayxQ@EkfFA_T+claACbQ7a?jc4Am0r0b`@+N!|NgxMWq=P}vDk|3vdM4wf#%WU zyZfp6lm#EWcn$9ed}_>{DnumPo%syC)-9ZBJ6Tp7blIobgHBq%p7Wv;iWXsig#&%= zrM^q2+!7(i(p=#6^x_vQ!!#_g&;p#G63r}Z-{eaIXvPX#!P^GYAz>yiUK}vB$Er|! z{`t)Kl>`UUi{3xrhKTc3kd4yGzf8^dAW+CXXhW!tFM~g*fvjWNUarBe)5T%;DZO>v z)1I!`y_?%!J^YWA9&!R2w<k8^Mf)LFpXJ54Q{}sMIdMh(>DAiB1!fxes&JJ~ai`f- z={RHNo5kcMem#vTW4xs+Q;tASAtUGA@Pe&|0>x7PMI?5~L<pks5cH+6$P6pasI$`* z%iTUWT~YMKZb?AQuy;ra#{F4p8Z~x(;%)Ug63cQ2-dOpLvfT19m86m(yle9sGV1hs zhI>P&=Q-)uqwyrkAAL6z_{~dvVM#b6dL;p_WV=t1=>15@Yc2SPb*`<Ap(-^NobrDh z*(1g|(LH*4Q`TOlLKPrNxde$4h_iJQC31zrXkSB}a(;kY1S<Q%ntIz~7*ATkC$m>T z7Ms!I0y$#T(j!}+K~?6cIN-y6FT48;c53`8cNn~E6eJrhfl2mbb^n&oanK_${e3lr zf1*Bikp~f{@dI)N5!YwF*2KxP!kQ7$SgIC_(G6qW*em6ptgSm!5eQ!K^51g7E;viv zUo;oZ4*_ht!_M=7HqWsiUC#TMw{%%pDmtm9aJGP%e=+DKPWfJcw7@#o$fWA#jrXkW z@;KPKT?OOm-lUM9jCJK)$E9OrO}NPhgP16-i2f_U>wk@y-Zp@Jzmpr`R;GdwT)o{V zS`41E@EYfQ<(s<W`s}m}zLhzVtt4GkaKvE^4t|FP`+$K2MXb-guNJqGF)kDC^^kAW zouOh@D2nxZb=->Py-H3h1I5W+f>OP6WPQBU+b;89gNu4~7{^Rptt#P(I{RznM6c`B z^}_WgryaL0NmIjou`%x1O=IR#vTCFAP?&taB@EXj)nC5{52bS`aI*tVG5F<LoezFX zNOZEN$@%5$BQi{_ZD%e@(;X?#sPdEiLh@ag=Q*~C@0la@kITxAY!PGE#qamv8dtO6 zBbnHfl`63|Uav~c&~Xlne17M?`|z!5LyS*1|LD`6N12Db!Lx5VF76b`yMOydpHf&R zoDn%Hxhixh<)E>G#!q`?JN0MC9r)7pjo(ea(?;u}_U8=R#)#W*Sk2-Eb#vN;`D6g` z9J$7i;it#zhNnx02ZTh-0TBZ$;-5Q6%OxYdcN$!$TL`!7E*@Rp=-jL3?7~gjg}H!L zENx@?15*MpZlb(l`rG4u=mIK1T|u?WQw||S@PPSlJzNIsyJueQ^q^W}qePy?&b*>6 zTGg7!LH;KWyVxigXQw3%Jt?U1goYKL4^si9s2_b{5i&K+J7CrV9r!>BE+(lXRX}d8 z!VH&LC-T!hGz?kFb)*Lm5~`t+oFuL3cR~)Vw!PJ&XbtCWu7_*~wq7T-jfNuM4mkQn z5)H4$%X!E$P>iuPM2+&r$MBrFzK@%rgW%!!z^cZ$qIWkLYK(KhJ!Pp|N2R3<LXx!P z#ex7=s#^?FQQ~qRlBatw-|rV(PBY1dw}@s_o2cua&G~k7yG2}j8Uxk>n^6|_dB85e zas$;EW^_ORMZ`trqB^8LX4ru+R8@fm-;NI+`0Q)miQ;yb+6-FPq;2T?Q<Dgt@G8Gl zqbum0b2FJoEHPn!^2~B(DYkd7^^z1!@zagofB4tUlT2HW@(A^RY1xkqfIG%F$)YaQ z_tp@DMv{PjA}Hm=wtUsrO%{Bu4m+C7mtcG##u~<SG<^C!*S5ZN?+SVrDnK5m&W~y= z=Gbexmgz{-o<HBL+v{?6^8`YvC56-2CJ2Z?;HrC9K3xoBF%xuPTaV_PPe{6R6g2i- zX{X`X`BS%b4mBqE$Bql#vt6wlSI@qY_|Vqi#}79nocnW*?hBy;ZY-LURUPZCUhfx4 zfo<8j)R|MS`~<E|8RYBw(JRQBS4~-`y8qDGZ0d1ewTS9wmqrc`@Ur}ap2p7v(fezW zAF>G~V`T$jCp^1UZEppFzoWHJtwd8h&>}7`D=z;q2P?L4YnMqCV%32+aFa)fsa-5G z?f-aOaEJ$autda=&L^#=i1xjPT6W4oFfSU3tp&Ad6B9|8Xy%#-n5W=VjDib3x`0ya zBw;;bI$bvtijwVx;o+#keiXzh_4jDYqyFwS_s(iQNr?e<%Ey01lop;}d?n=TF#8!% zxAQE&r8QtM&;0$vKiZ_EqluNkFNjU+25`?!=8xPTL$BCe22FvpbGg`Qs3m_g(x4(Z zhp)#oTYu1r>aN*i54y^ze~n64U>oXS)P*z>2SP)1e;nn(e4*`BkbDbIU}f|NViAOz zE*@CEI_b0eLs)r(Q|Rf-8zX`oxAjZPzdeL{B+8Ar@w%s&m^;3!+4x#gc=Jgi)2R*O zXnd5>dM`>-$C0RjAmKDah=eHRo%<4Ih$EbCMVMn~hgbMCtD5ou@d9L8*Y9_^VoGeR zvNudXZ_q9u{aDqAwwPx<2`xZ$5qMcuRe??rfD_gOn`e65YIKzpIF>x8QAEqAGx~T; zSFvS3(MunDimzK%e%YU_pP3ejaV^Rv&z+qTirzj_x`}-=4Ec8Hamom3p00E(!yqk5 zbu9P8VY13WT<ItroB1e|Jbg5;UZcA(1<fuXuDwk62j#J5$59W@^)-df(gpL@4Bh=P zT-rO5LA5DIO6lV8Xcp=BAyr;<c>=D|TV$;;$McrqK8KoPR!J5FeM|U<SgLIn(#ZQ4 zS>C8C&KiY=xTwKP;^mh?$KdcckBIHoRZJBZ&)+SqEaoIFMTj#+(DlWTU235>;pLFZ z<sqpWFd|y?P40UP^;DdAfX-!yE%4e;a;2=$X5W+3SAOxB%Z~1~kMvn12Bz6oEdLT* zV^j_$_IUD*&(!=HSqYyfJhF-z_O=pxshcy3q5e%Kp0rm#aB1kBdsXL@xI(b#9eus; zB;aWto0`bEfmYKH8VPqPGEpO`xxFhXGEvWd@VJFIIU92LMr=)#Q2L3lr3gYJMUKNS z(7yc~JNx@z6Gi18(IL&HOW_G{*7t4HxZR++>KDV&Ur<`za_`(?-@uk(kQLZ+10_Ri zt@)9mli(Nr#5CNVoKw?5dpx`CK`MlKEfU_R<n)UnM#)$RVqM+d{S?00;rUeoq<g{W z@j<r8eGc*6{byUrbZJ8a{PlyOLRlo%v~i+3>Q|_`_~v-|z3Vb+J9EdJR1sfJ_o^>h zzL1`q<WH|we32FV=>GgGbZql7@gy3ny66?@?R!{H7yV#a#W6p$S&|{UPWIwI;@0~| zbjZ4L?y%YF@}ROs-K#sHBtrso)`!QL3gS8Tr1`sR!W4ljy7h&uulX=f7MB&eBb&;^ z-8*iiF--AxER0Nnk|ouMIUEUA{&Nd(Nr|=2JLeN(;8d;eq?^CPYnp2Y3)2h4cc1KW z{C+3hcUQl?Z)1|KU#uy(dz?<f@2tYrlvkN~=G9PR5oK-i3H|wvKYsZWXHc>nR($<4 zg~<e?Feb3}I7&#4{GgxsaPzT&LV|Du#K#41kyiB<PevX5_oISkz<#qa)}Gw+F_vz% zOG0^bWGj&BgMpb$en4ln&1VE*2L-G8Tua#Bsmmkt*ObWD>EU6k@@0SBXW$6;$cEw| z@3?O8cMSyl@8KvT7u~CwwtB=#Mjxo{*O?Z8>jU)Fa`mK^2@a=DfOE=ZQ6tVnPpq!* zYH_T*HB`G(2IvCE_+~#km@IX6(2S36XeaNSq{CnnAQ0j11uIe$>8#U2T=vynckC^! z&(Kjhd^05)UVqPM^^*lw$W&4|?z3RS$qK3hu??~mX#C*Bs(#`f+7Gjq@t>~W8hb;V z24P(a)zExOWn>yK;}%PE`L5PGe6^BCgJZ@Ksv=RNGNX6Ic|&1L&Ew;nt(R(HcMn2( zRq|n3mrkFbYD=H#RNvMjpC@MQ+^ISJ-knm*s%{~dux9vIPI}dr5z>ive?Eu`-19m7 z5tb%L%3%!;E}%|1j*gn|l;RspXz$SiNvlY%$Cc?sOmVP@5)<pPID}+sBqB?!*PcYB zh*Ly61_jzpe)DY6mb&uIvmNB!3C)gkh}YruU3}7ey)4mDVBTT`Vlx`Gc88pY_g2rw zXZQ~-+(`b}9e=ygW#+vrH2wM}Vnk~kxYo^etEW94Yfz_{E<vizYSsyXw9uPF9ZMz; z@uB|`v>K+i_e^uNx<pCzgwwy=kyYe*!09uink!m~V|Sqvs#E;nrplD?YK-L?IyIip zEvB(RYtSU;fuG_>f|gQg`JMJh<oNbdU2^e`rSkK;DjFBI6>zlf)wq{;Q#ewRlT>lt z)T*PW_SY3fdpKf}J6`ZezRpy?mXasGx`%+qqIO4SN~oA-%0)D`hs)(u=n>zDO6Yb* z&=U&4M>V^iZ~8nnrm|PIWMh)O*t*N#B%=QZl}So=J`J5vx|tP>-sTH)KeEaAIm*DM z`{&UnabaJ5uwG9J`!&R~?H(mMTX(el9=qag)fyv4$u$4|eyZ)PS37lfgPH-Q$0p*Q zE-TM|5#)SgFzpB-Q|l)5mnj?gn)VK&A8;0?w*Q%lc5}mwcX_(jvHE?<ulR{{#Ttv+ zSSY*MS44)x;OWM^DAjxCqQnyUlRu4EH^}e8vWH>Yqli2lciDv6(WjBgRW-<1m^y=? z*4cuf%F~=$Y!3J80{KC`?d<E@`T|W@DRQzSb6q){Ppvi$(&Xk>RQqUR#pFv^@8GK} z3d5vHJoybpmTK@ok$lH-|Ig2O7<iO~z_=Dqlb?Q{2z<BU=d6Ku8cc|2YZewrq1}j^ zj9}ver2^iD8Uuz9=)4prNUs5yyX|}+>nkFtAdzx)GHfnn^<DMP+r7VNY*w~f;f;IW z8JA7OLHSRkFA?=`&iaoUtn=>%cN=-=3WmBU`J7R7g_K~|4dc(=olID;j=cn)fA0W~ zsV_L!(@Y*6uVx!6%mv#wF1^tkq2;t>!*lg0GJ_85_10jwmE|1PvJ||mr($dNvY#?0 zhEG5gY!MSxhXco{#wJCsqa!LGE@s3Js2Tw4(j_q+^}(YoxpOd0Dz3d^SWYG*;7%j5 z_C+#qjXQ>?;i@&xp0;ru&K=~Ob8TJJ5|u^F!)f9N4{8?(kEf5VdFP<7H8W?__(|I_ zua7xf?FGl68l~(U1>A9RRXn>3Q%+PkoaKf3jTp7hKzJ9K7V(ck_;GLu6%50H-kKsm zf%<=D5lME4U$K`se#Cf&17V+Lb4+CwkedByH+jZaaz3A4>u<F?ogW9BCO_^E<pMP2 zB`c~%g$yRFX8GFUFuCqPq+RCfXi>zIOZoc0M;6ZY(YZd}>nJiN{p`Hi;^U|HWdZJ2 z3v+96)O-ls%MGK*LG8i2s>9pWqH}tGjk{3V=5)B<)DPFDxyFX=kw$7*S3LT@Yo}a| zDt~LmI|Pz3^w+;T69CL2e*tq|f_9z#J!2t;zmvKV$!f=RMTkj!S9J_ZJt7wST$RU{ z_!umx8)=RsW9+>9Pjc|zDSJ)#WH5BJt(u(i>cgG0Y5333kpr9t+(7xnI?j3Vt@O_L zMqWBkZZke3q<=YjKT7PmWRRF+6|-kJ%b#8Z$C6(KUBzCjYsAYp0&<gAP1B<IIMm;* z2sE!GS{a!zHa|eT?25P_viotF2U;WgGgMDieBzUc{y(2YQTwq36l+pkMAPg2&Tsy+ zxDB@bwhLijUbcxZ#@;yfW*e<`3SBgDO<w#J$^>Ji{-=IGgSX+Rl*ZVVkT){(6UBB~ z;i@Ir3?ht9J^{D5`Dj(M7PNZx&-D_vFk5v{V!eDej569@Fcb@YF?Hc7^Kt=Xd<-v) zs)`PU?a2a9){g~PZI4p!o{Q)&i$O`1Ch~jOxP30P5#N~z`&{dZ-l|BnVNMa7vWbqB zLx^rW!#1tTC>ZN%jykt0lq^1D&=pnnd3UiBoKJbQuKbW#T7#8><kxLCcyahnY*m-S z4T|3;B|G<5^p}ma7Rc|)!m}{Mv5g^O+f#@t7_vy{45ti2pBP}X*+2&lintA+IkzuA z(#YFIcwQoi6_RkL_=+1?!<L@Pwc1Wc#&o3B0+cn=lBQJb#AlfYbt#hYvViHWwEopu z*Hg4uhHG2(ccM%CGNVjJ>N{XOpf?s@$6vl9?<a?UWKqQp<s!pLal#{ZyGR24eaQPj z0p)G3-_{#t5O+Y+QLz^t<!_$=e%7?Ej6HtTcy>z8JnmXA%Fe<=>Uk@eidj8YTJta2 zDS6F!tjEd<xY*s_{V0ecpH6!j%FGdUe8Qd%T^=_8+uUX(PHvpdTqg-J3BY`-z&7ZC zBaY45XK>flYh^Vg)Qj!mGjXYp&USUF1wg!PTcm6uqgWSJ+m6;dr$+bIR35}lViJ6_ z(2E(N3VQx};S6IpBo@Jxh<ag``^d1+j0Ny3-MJPat7^(J7DEy{@~y{YNHrFg0T?%6 z0yDG*q2x^^=O%F6E#$GYNOK<^gIvS94!60adCcj;&WK#vxh|!<UwB6~W3z^+9P)M@ zh_`iQu8~!(s1m7({)Q*QWrg}}MgKNu0F-GqHHhl_QTL{VATOYz2GVOJK$-W#Xd)77 zvAq>s3}dZiGpW!i`0&33>))06vO)v$`o<|&0OO5h>hqw6@s6kB&DzhFA(*lNZFh+B zRjw{SqX|Dn6D2Y0-mb5A;>z|h^osX4SItjUASJnKUumgPP?6uGoW!O()LQIo!43mj z;QDBr2FRW!f8E)ZeBu&CAeI_6xI+(rJPEj8be^+$pZKl#@D8c(BVj8~aY15+xrfm3 zhQzm*)RL%~v$(|H?T@{+P$=E8>UA>vItWuJ8)eK*o4;2^2RMw%FL8C2Z9e*OcExcW z3Q^b9xH5)mbUsK@u30sQH5lH9^N9v02yy+;69`|A)(@U8sH)z&cT<CD+s{Dnv}Zb( zFcRuZ&c1L`F%_Gpq1_Ai)|hr=7DWHCrco>OdrOGlxBzTNR?nU(>{rT#tI+sgI0Vhf z`b<e8^LQPNdMCnIzoefzareD@&7?&CE|gp_`DD|=!~J1v?_!{M`O=)kPh=UTL9T>G zebk4ljp|dw*;rRnYj&^Y0Q%KPP^xzt!v!Uma^q%nYB&4}`F7&jlDliwiY}(;6gIo6 zIh92fA}cecJ=kbZm(rra8tYxNmm#ZFvv*#C7ZrKemM~j17{&GLuR8i`slH~xUYzFP zHi7y<%svud-ZY||taxQMteZcrzV+#D%b!=GBcfE&oxRG=!6Drr)8$|$N-H1iUy#U0 z$zfNw$twz*2rjq1fhM5{rGelwW?JuEg|=e@(uzV``ruQm{Uo^}1HD*5A0WlqMi~Kk z?N7JcP7eIn3*Hwd%40+*KW$F#0Y~Piv@xTe3&=rcg?!by<408}00xU1q)QikVjbQG zy5odAZ{pn$RUMO98JG!1YtK&7>;4VT>Pr!c?d5N*#Rb$lJia+(t+LKk0?t9_Ql!@O z1WC`czi=5PCBzfd!SVG73QPQ<u{qOmB)Cz%2J|V6jr@_gMiHkJ!NoyyDEsT%K{Ra^ z+*A&?n5lKV>2v)%VpsSQENN15jlI(5FAHX*Uue(p4iPfQ@|XZiC-=uZwirsfJF)OH zp0lXIZ?zCh*$G-7uRsPs!x1{PLLvzPrr*$Ngc-l}Z@ss9w=N0P?+)PJB$^spcvWj* zb|Zwlb~(S`Ia-LpC;|0K+S?H+a_ONr+Y@-qNnpo^=D8LcJ#(d_lY;-wujNaO3{)^N zSrzO%7-4C7m6Z8E`ru!U<G)DuKg~D)`SJxrD$8*P_EtWm6v}cYCyO#2zx1J!A-HW( zg8zO+fIP#Wz32w$Jp1;m$)o^puEM|+^N}A<bfS`BQ)aHpwy~*^W%5pY$awHiBg32V z*<CDZN#6DK6B4Zg?HvI<ON^X+58EitbxE*BT!d$QB4w_;-Qk%KL{7Ll(XVJiogq;8 zN2H_o&csFpp-N2Sc~$}z*FiN@PKQbRr-rWfE$Q<wcC*o3I?;E(;S;=H)G}gB83V_@ ziA<0T(Ao51CFY{Owb~PHP`W<o*=Kkby-GrDh#-`0II+%v5PGs@{y>W+c_F->m{lcP z@zCCx`#n#X<-sOTf(cxtW2Dcr^d*dk*>CT566{enPrgY!oTr}MBSj5X6A_+!Is>M1 z1KIsL#Y7`z?M+F|PQwCfL3>hTr3D3pu#8FL4Bm{Rr;CCG0Y>1u)!*+|3_J{m!v@B2 z%waOlHU{c$TAo{eN9ccG=d4#K)Pj{H>utYk9Gh?>9pX=Xw*G21a$x8i4f&DEld>sT zDz0Li_X@eZhafA5EuAtTzzZhpWeH6{#maf0cVtNGGM_{Ojld?Ac@AHU_g;|R_4{5N z8Y~;NmD%6RM#KiFbSG+IY}K@d)XF#G%1{OVSQmA{Q2dnt*v+E+OwYy#7kS@rs4DN& z{|A1KeR-mbsRB%fO3=AK6`zkRgBN1fe_S-=+c>-ijF@%xlvC)30miD3TtC;;Qnl!S z1nOu*m3{3f)>e=p{5B|A9XAkE*ru{mFN;`A40yJrK>-ZDhpT-L!hL6a{!u+#>DT*9 zU-1gMEyOmnBudnBkuAIo4!zo}ik`Ucxa17~0tspf4i~xEw+q(i4oA4%7YwEfFrV1w z*psr6Au3rzA+FdmN+C~s1}&wwy{<+D!4~zCg+<Wf_d!UzU_@TLrErj)xeJ!s=CD3i zX##)S#o}kUNgyblC6`v`g)fo4;#LHcf<oVKC6MXDfkp=3UHn*N-h=tCDFMXFP8K~= ztAO#GF%t_9xrkU(fI-`@j-9661^N|*lU1ZCtKp|bie&3E5}LJjl8dl+fmNfgdvl7c z8?g{&_0?NLE}A+`=}z%Bcax>b+Dj0?MPH+yg3WKeSyMU0@c;oxw;UbM1A#}<vQl<l z?*zV1$OsuuE^NH`&*AuAT>Ae^WN8jVs)hrRHwNcD&T^$9XJDww(WL#5|Lj2JgWE5z z8!yjS@oz4+dog>9OxZ<#$qzJnO01^?32P*9yE_AZ03n%v7mJck74y*lWNzBoha&pS z^6AR%|GCm`u;9U14}^PrSETE-tk_J?FNIBKkrQROQ8^hkW`1&#&jzH{8`FEAoB6ph zuceg;Gc1m2lxA!Xt6m)21w6ZmA`f92$SH(u(oc&d&at*gsJ3;w+1IVk+t+JGY50Nc znHdW4`*$KJ-{G=*QZvGiDJYGI=3Ti6_N|g*qzCKQ$Yee5D?6LPU2dM4G!Qg?rQj5{ zV)X@>uRvL!szAeh3KTz;(Te(@@6qd+{O()E`xWL{Ec(Rw6@SeRra6});b=YKApU;m z%klM^-IyX6Qj2Wzjd=$~C8$@08dETE?@ts&%W-B`QlK-S>0_-X@14{xfsMbPKLvoC zdo`-3VfV5e0$qy8IDR}W(?OnqG^5N8<*L5qM><)Ryi?z<ktpgNX!Q4<{`ZWO%CCTS zamHRr)9o&^GI7>O0;!fsAG=a%KX`suXY8v}ja9ul;~#BlQ{T^QG+#B_%H|hMj%?LM zQfj<&A9Z4aFIH{<5K*B0ilOS!80Z2Oi0ibhapSA&hJA9umt?h!!tzEpXu#)09pKw; zP{()A=+3mJo;<p9jk2bz&f^=in4}`@A?^5(^wTRmbEQ8!+cQf+ViTCYkD(i1TzkcJ zvo<E#)yfm$U0-De6zzu;kXDE;Zr8(4K>}1MFwQ|I^Cm&;s@hV_hc9VzY{@?}R)Ix2 zd(wn0D5+>w`V=%5;@S~K3*GGAnRcu0A9+mP&7K1u1#;y{DASP<v5H9UP&i#ZULBoF zXI->6-_#)={>Zzx9yxBJ$6Amo5VqGrBYA{dABCJ18f3~7>waC&%ul=~sdS;~*zEf* z)CQ;G9g#HPf_<ixm<F~I2%e9`4K;_Xci5aMsqT}pMeCXFZwlL%t~q`J)Z-B%`VTij ze;w(uatxRRR&Sj!nKB3KkdH$v#JkCZWVe@s>>tMm(3WJ7kk&M6@-44=@T_qy;Nv>t zym{*%bJ$r(Ce(IrUY%bf?Y$IZ0#kT#PWW0s(%LokT>xL-R;1B;`RDq!6E>%Vf&~;j z?-?WmVA0G^4?aeHPuCwDR6h#~o$9qVnT2mP?Y6En+7RzToj(QW>>PpS!<tQ|gK1J? zdOs^e4eV@+{f0ljX8M_-WRYOSB9D*2jND%mTHPp4oYNrFvmMlY(v`jkuZU31>U>}= za{~$}C_ESY_dNeo%l7{a{r_xbTiU09UMo7EgTE~vK)XhmIb0nyMVa>2rj=X`m6T%` z(VmV)?sxWQFj-r&4lbCjSB7qb;Ea0=?nRB}5$l$13r6ym-jcjBtU+dGa!0Qe#x?G< zp2P*5lG4w(YnT_?=Dte!bH@h+Z<_DLc6WiqDkIOVWp0hcT3X1p-Rtj=$B{EWQ7Gl@ zukzcG-F%X3EI;0KT8GCqz|M2URJ@j(H;}$feV~Duqg<Hv%CW*EjMfNdC5+cJ*PV&u znZr(M%##-uo`80u2j7uwHZ7QaM-%iIiCkU$LW=bwSFNV?(nzMFy%Y(C$7w?LEXXJG zp9tJ?72B&YiYQeL54{oi>`(nZaK$}d^LhYxH5^!QUeAJBju2TUuT#Ck`3;kvp!tcO zEGL6hNf^AJRF$ln95w>^?wb}&jqnarj0lXzx1_%nZ+%8x;)m2-ylVjeSkh4S9)k`W zsQBqvs!&vYeNlBRvbC3NKCQdwPH5P@ap$={LML?cdEntw3-rSy?cX{CWfxO%dDkrh z(lnCNwpF^DOw}`#g(pjGbB*t8f_x2fhY8PO1B5{qFyiE!nDcUbQL;urR-7pyMCLqH z=*kyn#`>6y2S;X9N~MgwRQ1I%$uN1C++(LdCmW`18LhYi3FiKcV@?~DH&lLa4d^u1 z|DYxwvR1Sq8-e$@MpOwiGQ{vQ;9CH!-Mp~J<@T~Konu)Se(=Q*aqANTKcz)!Jo+s< zTJyfmlJi>4o^nNEw6pbDQM#EPx9l&ov(t3vkp_S*ZuK3wv;ONw_Sn(e*&_gA-)Zsc zN~j^@r;7l|h$wWIMc_i)X}?=Z0q`qzYdl(xCiLK~D{|}mSjz3$K*H=g@|q&^RC%%* zX>C8CHbSZ_VCeYVMEkA)z2B(8sMpG%A(=1C+qy`Vpr5RM8r-;DJ3T@nU@LeweU!u* ziS-<V_2Z8-y_<8a<9B578Z(_()X*JBHfq_oDmo3_71lh6@p1!~fxEsp?&fSJ{Y**& zuHBYGvVcTb0-_d^Gb0fQY7khdk(8cz8fI%8U0NC)D3bq$$}~jbrX}NU$VR);HSYi% zs<#4popOXt2>0$?h!xDTRljD<o`l#zp6{E5RkV!E24SuE>-!b3VU^DCpQ6@Mz7?FH zRS*p1!Y`8nbNcdDt0zmp<Idy<x71%6R+q9!Mcwi{*gg0U4ruo9pmKzY3Z|E&C-ScC zDD!{8&i~gQ7kx<$1g#&?H3n6`q&5MWzm>b?Y_?={bL>Gb&EUd_3wgxOspi4&k2lth zq6B;!L+0&6H3V1;qdy7sw_hLYhYHN=6Jw`jVU<o7V?8a;QP;E26{w?gZ2GJjKy)#W z0mFZ9$3kGeTYwR^TP)6m%>Zn<lpo(0QYxbqbj&vd;;T`Atrm<L)zH5d6m$HYqVt>j zTtKSy#CI~ikv-g^;`EyjYf^;)FK(9QF*<c)7tJ0K9P8~Ojvp85G`3LD8Iwjm47hh0 z!{Gzj{?--N>U@w|B-Ph=>PKI)!g+#FL-G*0PR`}r#+4Yatnj!zi=L5P{5~Ba%{J=8 zAB*+b71>U1r>kFUOg6XeP;j>PJDx|}6jtgFO_Jody0M0Q?vz%rXw>8^y|Z45R5ai1 z7CTx~Zx%#BMpC=f2m-Ua#%=e`w0MUUjyF#brC`EuqS|-j8aa%gitL@W-w?U0IW>c1 zu)i@exC-gh8&V)N7=r8IGB@5&q(b$hPhHwwwiLo1{rb5)6KVvGgJnm;So?amDR<Zf zVv_x`GhyU7<FkiC2f2bO?puCJ0}|T=2LA+EwUl_**A$LfuV6fn%aan-w{m_mHHHY6 zcMXF{dK{?I!N1PohJ&vl2Z^CkhQg}v7B#!)d0m=dWd*$$Mh{(S(*vJskjjOe<>5zP zyZ?#JR|h@FxjbeS-QXq<^LPy8v3E>c58i3w{|PN?=;CpmYgkPJg>le}GYCeXH<_1_ z!6pMfEm^-67ktw&oQ^^oukb(rU76r#E?K6LCr_5<-dyFBc+`!cY`RXoGu<Hd?1Y|? zvDtIvIYOa0hd2+_F96&=H%=e+L7bv0462>_A9nKxeeCTr&IM}I@3id2d^T(Z3MEB4 zrVAL#)HDPYK}T*YVb`5h)DfW>0r;u)`q4)gN=j58S@0uOUw>w2Tr&>))=n@cV?M-& z!cREJF^h?9uvK^2P3c;|n2cf-6-m78fxSRzpiI;cKWv*3$X}^@a|)zDe5#3JDDG{M zAC@daoD1e2ZKm^NV17>{sj}|gwC0$XExIwt&X`hobXPg&<M{zTbZV*{5ulpd5W6$} zCpT{M2W<kr>a5$%d*7Q2*pogna7TIZIcKiu1()&(E%i#YsTgFdAs)a5wnHZ7EjjGs zPx<f&{Aq5hQ^_Km^o&c4zn^(-OOCw4#M^+3iQy~L)lajjUhRN@r;~1PO5}<?`HG$X zA0q1i8r}c7k<GZ-iz%O|Pj6FUKOe?B3!8J-Y#=N{E`fqUt&6dGhjoUcINf)Ne79mk z75ki3KTLl`gjRBRj0pH2Irt{Dcjxe~Jq2DD%AjJO|Kge?$?yYfV#`3ws3^Ig_env_ zY3UvpHAy!YOyuu~<^ky<Si1x|&TW4fJ)l2tu30Y91O2^Z5|4ZwmoR7r!#13Pw9OV% zj*wqpULXCGZ|C2a|6x^i%m#edO-+OR&py1A6G1=k$S{Y-6!>wR(2;DwWg(wF9PvcJ z`e}3UTT^J}KMVvgQ5CYWyiu-cEtlyA%BfXBS-c97y|<r#@PE7j*7hr^#ZAxnB&N)% z-?XqYzXY<XXeNe;G+UH0Q``@gQnU;~Sjbq`(*n<YZMza{{Ef|;MGf8Itl0dJQV}^m z6Yecwv<8&A&MLXw;=GvlbLLC`*(nl;Lst6UjF@j&&85zu=y(`vZKnLOFJs;&PpY%D z2;xavblHR~tQ{)1Zviz(jjR5KYiKZOtu=iSuzxZ-$p7A)v&FB?p5;E7zxr&l^STOr zdo5#k)=N_m8fKCZX1I8ajM1f2TEUT}>b=afDO;d#(w?jGvG!o6nEREpCX$I98mT1S z$mCo#hDhK0;zeGve%H#-d59hBteDyrPGePTp=5MH%-%zStK5Qp+h+OCtZV=gujjvg z#1k|ax>@>m#nP{zUgMl;K~mNC$H9{fVH~0Oaxlw%b;|Kz_iPog@(Kk{<{@&1DVn6v zFAKgHsl|ZN2Ae1$^U?+&@H(pO3wqXdif+ZDqSe<<rm`vKP+m|7gWW{Lao<NoDCKDM zbF!Kek<cTYoPUhA$@Xm7uEq621oahW{NL(4K&vXucxz#7qnXnZqJIX3#7|WD^&J;w zBST+4YC8?cIP0}3wBQuG-`2cYPJ=LqqM1#t-pLbZce~WtI(RpSdgxW+1NMl{+~7_v zPHn53l`K%RcJlp+<u^#oY29C~h%W>b$<h8oE%<1ad`++K<-pWTR_Dl>NSOQI^*bea zY5&*6h^7D(#-Za$Rpm<`f*H6;W$W4i)$F;8pE-liagJcK{3jG)6Tn|e)@!c$TTHW! zkpfkI1#so~@Bid1;`M5lnhigr{A^Ui<+2tehb;*&?(Vw`R@1>Z=F<)mCim39<<+0w z3OR^sTSq0T0@E9K#c_jf+BZF0<_cigH2dI4xt4k@^$7m2r|~}}e*Y_m7Jd77|EB9S z5yrnp&}x+PH;bdJe%I$$rFVu+L^ljaR|(LYZM;A7uAotO>xy)8`Ofs!q`&<XW9L>6 z38P;urLEVbXE{@23K+Mw&G2YUi+Qza7%39Z@zksMBCe+0wnoLtp*q4{T*CyXtk<?q zgoBjE)Rlr7+7>?!C27jygXa+}C!si))Q%!ufmjZT$V<y}#bZ=I7!@mcpHkUHU)%}) zv?M$BW8=-~u)BX2{#Z+{UU0;{Ot!k^8vZ>=e~>Z6j@{!rP<{qSGqnan5p@A4F6YC= zV6jfvoGFRB3E=?S9Zk9St1WoKzFd8`t>(I&Q_xIrI~UG<7sWmD--N?O8>$ocm#k%^ zKHamyl;OYTCdyNf3Vni1^+5B}F#0QZdoY6vH5dJ{^X7*wf2)RK!%c9wPdl?M%g3uR zEJC8$`t{94;pa6bqSTFVK-=#dRiq}G3Pah|3fYR~8Et_<+J1kJ%oA2hTkLz}(y{P{ zg7FcXEgFB`ssHNq3>{7}jq0nzVZs2UIDU3AC{Hh*;bStbyUaJa*$y6^WmQw$*#U6p zjlXzynWs5WTT|BU9mi5()%5~gw64GdL1H^ZmZ3m6N%0DQ;G~_a<@I0*ZAIt~9H39% zK2nuaJvSBbejl}*W>MICyGhIdzV!M$*M-|LXYo!@L<H6*{SYuK5$J}_d`H(uvmcED z)~J0}j=djB>=n4ZbK6|r{gP>}8n%mCoG-dmzBw7&+Sr#Yd0(Tr1sG4GpnkkA<UN5C z&oA-i$|eyXoIg#dZCyb_VB|@gH5r%0Hnjih@;p-Xhu8Du++Qq0-JBe!m&=|m4!wPp zsF81`5OaTl4tCvAWb{g=G>>lk_+guZ1me%dY%Xc?s`^<7G-!6;aFW0AC4teC?Hl?w z5RgY~!))|yj(?ke8yWnhtWZ`xptOi3fPScyNH~ehJB(jS%&(M~0kM56eDpwA^f}Y^ zvaNlxPIL5Q`g9CL9Zu5qkalqTyXWcxL3di6a6JEiP*wa=ab(Y4(7E*p_+Z?OFMFuf zMFXnfV95dIEY&0oPb6_I*HSbNEU&HEEW)%H6LD;2!Ac07E{JuJiY{K&@LZRayW1_d z#w6p@|DnnvD!W6V=aFOo6$YpdX*ouyWlze9tzg;0;n75raKEkFkuq+&r}=LktvDNR z>gyE{C=b(l^t%2h;?ZXHnA2DX^~bun5MX`~+0(g4s;;vUXut=&cY|M9ujxXiJMG-@ z$uQ}bjYQgwu!IaBHWQg}XUfun>|sVPWH!WMAUc=nH)!#r_xJti(kToK2<q_k2V+Ho zclWdTWIe}+F4n==On@{VvZN0Y=&e)X|A0JUeAc19peY36w5MBS;X&*CCny1wx2VwW z=}U*H!Dk_Oo~<Vfe^@hKeeX67A@1CJIPbp(!M?w$(O8IFid20)_;7gFY|WW@5ZT>? zeXn+7c#sjWfo*{f{^mm!c|b9Pw;8}8zwHro`jxU5ue6JsaK%R5hZ4EmlK}TW5464Q zuUMJ0ZNgUl6zGy1L12{lBrkpl>z7046Pj057ga)f`xOk&cJ*vV2B|g(0ulPt-558z z-Q<nRpG|Xe{t97|D#uH24^czMU{Ds`?VKPCuuu@JkAM3QFTXEJ4nJ9ZRzjtO4c#p3 z#&%!U7mrZ<O5m<`DtIz*rw9|+%PaI*4rfo^t`iJfvsrz4BMnIJ6Jba97ALcPy13U< zpFDECsf8-{QQB_yg90p2H@FZJd-cBfy?OJ}WV9XV4(BNey&mq_EJaxXmK{G~E)z)s zA_o(I@$gB}BZG?IpQ0Bc)*%AHwxN!mYP$1_9GeL3wgOr~i<P9sTuyt>t3N6`vVMjF zw(rek8XFi%B9RxbYai{`vIF`ye&@yRs^G{xSXE%{3nh|=h1fS~hmy7L9@n{rZ|^ip zavF)js-`&U(TCKOds+K?k@`4ap!XRinkBaZ3QKAKd=jk?Cn<e+y)e`BA>h$a{-LCf z7$J|^?W(&HRarp;)pS~AAST@|*5AgHL8wIC99xf1vhe}_Ck?y$?DP$^JqZFvzfbg= z^T9oxPYS_3jg{lO8~?k8@;_iw3`Xk<Idu#4XKN@l3J;gQl|z+sjoxS)z1A@X9JsXF zpDBy&pCR{4_^s1$OX-Soi}SQjKyR9MeoT@S@8wDI2}=pH?eO1p_5h#0$2S~fz}>xG zMs0IU-_$GaWT1M5D^Y;HE}~R6`U^xOR@#h(<XFYMQ`D$=wm&kU&R}@h;QY&FGg0+| ziYJxOO-0x^Ye4K-S>pq#=WaQ7CX?-1DMtgmHJuoF5=)vcc9>+~$>P?{(w(@G|FYrD z&YZ6;7{}?}PHZbBNl=Nfy9{GqN83uqdu;2c<loq8TcDn-2^{94BIFf9)pMknkP~=| z131jQ=KVmp*U3G2>2mHz=b6uqnjs9$XDicn395WI6jnH`{D<dD(`5<!auFW}$s0Y~ zV#&M1jr)<e_AnNNNd*8$dFHv!`^h2~)j(wWR(d$~dKBD;IS(1(Sm_yep1yZDbd9_R zil7Y-b}v>FX##jG_UUf&f<f3C9R}=(r1YIJ{q|@f_3AG#Al(=}1g*1G$&FQ{wN5o7 zaB$+rWmUH4bMfznDbI0gEv(@?8lj$DN9Y?R)Lcyu4_^$1oba#qn>021vR%xdeEf>n zXfQqyx$iQ=m4x6r^q$i!jey+KUa{NBjmp@MhGc7~pQkSGXh602t{<x74nJs}hW3bU z1|_|Q;I#$hpKUhj?A|9EwGl2`xLCM`xfuU)MBx2?1BN6{7>#?#pu_Vk-?&pGS|?d{ zqCT+GeT!=Imn;SGr4<ZuoG`tod){$SGaJKrH033nF6!FGJY>$nQ|pFi_~!-En~bVW z-Kln<3UHch1<b#6>E7$8;}^VzC1=~%G>7M6wDBS|t1(DFc?zy*{Yz2yqiNuIDVA9u z+^}vD5Gi_mdmSYVF?17>_k20-TzMiQ7DhQj7X`JEXUB(H$4x8>4hNJaJih(9E=-_X zScGJSTyIIz+<GRQpwPbJ{@yC^K?U-tWK@3I{jL05@FJCiQXuNNCTc%(aPT)#b*~+~ z9#8@QRJB?=9k28tr)pyLs!TrMQW~W4odf*D>#^!5cBJH8gv6|5Q2E0gJ&6|odF93O zWoAJ=lB>NMt(j`oiwbjC*)n{OAtnyRDxUIBQXQ)iXE`DbFHDipT;$;e`gsx+fDT%> za<u0gX`xY8H;lT<$Gu4P2Qt=8AsMP(;Bi47?Iw{po|u$eDhW1b$LvNa#A5b<em3PG zoHd<PwTOM2q`crK=lZ_cP}45>6393={A^$F>~#_#jpc^cVSFjLPIK&i*F-c^ooi2= za{2$q*Lz2^9smF0vA5{537r%*Ym3#=R?*VdtQC8Y+AAp9npL%DYqj>KMvU4c_6Rj< z#0s%PM1Jpn?!DjpJNNhd-hZ6q@Q1@W56{<QiVU$o4>6D1)fbJ7XBp}{@hrMk1Hl&D z4HGNBO=an?<YZER^;}5yvBnb<b7Vg)&v;5VqStuJRAn>l5u*Bk8YTYwf5@PJ9zsHn zKs@EFw>?|@6}8GxLBP=~kN-xQ^Bu<S1ljrv0EXaUTThl(OPgRem2o|wWJq8?b2f*8 zMwZOS$SJ>pxR7_PQvIibUIyp$lu}Z5;;*otw2cErEEKItD5}yWPt6n8;y3w~k*5-A zPd#EgRZxB-K^p?uWNE&S+J(p~1>K^!El+Wdz=5~KzG45-_d0h()f-UC79HjrFhnxY zK42Er<^4m^ZmwG<G$-0~$K8P<ws0z_f=o#nnf8>x6~d`Oyi#*4|4xZiNienAE~877 zZ%A(5<H;=1-a(aSa9Ol7L5OQEW`|=Vt4qvFXAYZo$7`8$30q9jT;R{`a1pl=LFm~h zv(*o+!a@UjX(Hp4iT_F-^u1sLZ4vPc(QHlmfOlw*75i&fOPHGqk<=aD^Vax_1Rjp+ zl-tNiw742MveEnq^DL;ZhpB%=-;k`OfjEAE6E_{_n~yXMX`NuPdoz|{`V3EIFKj8@ zHS@dn%aoHN1a0oDke5w^V?FfKKpj?gp1c&sI!r`{qNTg9Ge^oOTJ`tNj80by>qK`w zq}pp=%za=y=P@GJ!qQE0`rt(@>Z%i@zY~3h22LTuy)GPjUZXf;Due@{KO;;hqA@F- z#;z_r3O?{*-A!dOEc8h8074U+f5;TjQsbTUS2C&R#2&1g3vjEUmnx7?a1Xiq^?mx^ zFQ~fTz0Y2*E?PIgn(~L0`#r^UVtO{)MY$08v?GWtz#IWPVL7d=eQ5e!p+whICO|s% zJF`kGU}Y@vL|n1cV-I$~zldD-$i2SjgKwMx+G>FwefdqRo2CY!vrhtQRjP(%AITwM ze?SekN~}BIS)h!TWt<7gHZO-av&p`eE*Ta3jcC~;rkEK5N<zNykz12DmNn!4zQd_2 z%4%Vbr`}2pot+`K#d84STgbrdEW_%%?0mKDOVmos9@npJ7vhP_xM&ZjxN#b(d6I8u zU@z&YhX5cWul4!-YlO<){90ST;J2NR4BYkD)SPoleiOJ9a%a!*!e#<p9=O~14%<#O zO?d1}1seNT8Wl>c2P$94*#dgnyJav-o>7Dv?(G>FZfRFc?R4Fw9iUw;$YB!eP(k>b zKivL_o}VmHLopn2vxbnQDK1v22C><cJwVkVuRxt5h=g|oLiKzKc70BGAT=cmd`NEo z$BV&!e5{^##{JPLTX!*p081WWLefDw3LEB`atogKNP`YPPg_SP%Xg0lZz|tlMgU_> zZkY6FFkV!9DuN;pOc;X{p0RYAg|5?)+5hA?in9K+#^V4gBGgA|Y4!wP2&mqwIOP`B zMh6i(Z^5fizPUa~3@PaxtOGJ#4={iv{g1Ba{CWIok2}?*5mUS-X-8?N5$x+_Mteu2 zqPn1*^Q3lyU%7YOi#z0VtsY}<ftSHHXZ@QY&XAu9aEXadJZs2mZfj^Omy#B2cHWKp zkK5DjU_$GE9Vh=KH2uHrAOF7kkGOWe2`jfu_;;@?!_mF%rZTYfiDOi*ag4Fg2AjDW z19`-8oiyu!`Q4&)n>%I4s<NCW0=H@fTGX~bh889`yN(d6svq1SI@1L&c6I(4Tl5<> z;gkbMQ%F92kw^I#9yWON0>QkRhZU&-A1j&GtjV}qY$C|#!x*r?*3BeO&#2$iU<`Xo zD77aqvXa#GKiE`Gwdpe5vu}HENf*gi0`z>T>EILgT=j1i@`mT+M3cWy{3|7_4J+@a z<iE@*eY4o*&P=!k(+i6eItd$^_jU+Eu5ez&E~TjXH)NNAlYAb}b^A~SSr{n#&R2Ca zf};gViZ~zrC{3+pHQQH=9)DwNYS^i8iJYsCRerZPVS!NZaX<?hxN}JAOB{-32?!|M zz9U7o30?Ot_p@;2;PU4FvdQ`53>*jvQ5!mc45OY1@sDHY>t1wWfH)`#|8jo8$^v7* z&_3ON_h;F)qULnqtk%-VS@Z9{e7)C*0(tj4XwbyonaOBX7Jxb_v6pH+33J*IFJ&Ns z3VPA!?b}@Fz!ZB6BF8N*eOoKAo;k{+RbbM9bk8jSmooL&oMz^+kc3Av^0jR8!)&9H z6SPI>5!2mG@o4KkRq-kf{1v<MkJ~)MG)Fg}USso}k&8JkjV<ztnSf0dru1vw{4VE4 zxU7~OW+04}j%~&PTmnCE<qE#ohb_0%=sgI$EZhqrwxcn@K6{G{LIMw|XRGv9zN+`v z=^uXgOgtJYJsZmskdpFsF9oifY8$1g=x^~)8KY9EqRH-<#E+3yVxc#mkclMTfJXev zkx#6(R2H-<t5M1r38IwcVDC>FsUVm4?}4Mhv)><z=#>fy#dGytoFj<oWz9>W?{H{E zMXjxe)5hXM=~CtLUq~^~gWg-4b-*Q$NSBjRz-@fr{INMc{)@vYamqB9a0K*XNjuey zssTtKsXM$nI`=_AYSB#{Mcj*EQ3E+Ic8VYSEyFUTu!_UJM`RB4@i~$|PHA2D(j3az zc45kWrIK@knb=+L!up+G4r#y6)_;{uI}V%p4+ssl8M;M3%Nil=&hxbG8QyOTNdg_9 zV`cBRpn;UM@^=j~=(=bVGZM#o-;q?BLGl&xK$w^SIM>{b<juVI8dhS-e0*Ig#6}pc z=%&x<>P-v}Yea!^acXCy$=sy}(cw4pPP&61)9ZS7ZF$sKXcLVrFA7E^XJGRJr9d$& z<M~>-srMgGRc8x|h{>RWX1B}o2cN}H?EAHusL#%~%eDV`QXfa{H81at|15r9X=PBr zv@;R&nEGr67N;q0Gt6m!sH%<-2wd$gRX0H@7jyE}*~Tc97BVg{M#8gOOm~WHxvAc1 z5QdJ0letn_xk-Y7`u|$8bF+}{e5g%#30W_f($stS{~Q(j|7G6(L!X=7{1;97C@=e` zE<Ybc5l_->`+4KGU(xmRev&r;e6Ww6)GqN&>yfhCM{am{>N{d<1VWhK_|Q74|5;*^ zg!hb@?-;Kuqr8JEP-DS{;GvE#aK6T2h;>ug>GG1iG3q2=(8nj<gbZ=mdBluMIhQ}t zBolfHjVrgqGn}e}v<$Os=EhMk4$K*FkM{c7Rq|QcF~JVNd%N~`v+<NmQ6V#(bbu=U z+rB^TXJsR48wscJ2GyQI>+0(+--5K~CUBf}EHMa~7h2yHvi&*#7GpSW!#}nbq#|A} z7K-j%Grd?f!;G3xqp|xPi$WfJq$}{2tVnKG$hXR11FHu~jw8`cH7EN0Pd!-Im!3vm zz7;i~=;(2U4v}(oE>NuZ{8Q9FlpeM7zuSZV(!f?ys{B@<8h$(k<zXSqd63pF!J^x^ z_c!RHy)iq733vZ!B#-T1-pQ!lw?#tlpktn7(<GN|SNJrhuRjf@Q5o3m8$eG5Zh`X& zk9OO^V<sY}A~gKFBgdSl5o0rlC!?lCga+{NnX}yve}G_@j%<XF@VVT1x<tMEwiHF| z9h`m@FAKgFPs8~6)(!*40qX5atloXx2)#J`Vm++;);&Ay8MgdrgyNFvo}J>&A;>u> zY0M;Ev4Pwv2Yl9lrsX_H@;4CRUMA^#DqP?=FgP<ulroTc`s)~*Z4qqx`$@m`>*T)) zH8y!}lBWu{1DDk7cJ)u_KkwPywE95+${0~CRe5ji-U^cRinK0mqaw-ePHT=@0q*hj zfVEPVqNzdWh7&bed%0SQWl1hG4+Aj#CK!<<_~DMv;Lni!y{k(<hS;B~zYawRdX4<D zmGNnz0NBVX{y{5>-nOaiONqBKHnChvUP{@ZYowg`KYKw^K;*etwnuU?;PB`_didpW zxi)YUG(pLnpKT)cehCmJd!UjNl!8gYm*`;VrKby}CR7dJ^xuq%3bsd6y6$}vD0`yL z<@xG0%&&7j-8n5*_Ix7*3d&&-med2(`m%QE+J@G{%)LB7#n&~x5r_+Y=%G+|Gnxx0 zU~I8&<)=7Lju80kteXi0B!OGD)zi$F<lMcImt(CO2~D3eu*sCL>Q7v*Y?Bq8GJ?5M ztTqF|e##M1TxBAg?jEPcZoq)iLP9GLy%Y?I<2C89@-?HPN-lMH+rBsfURGK>{O$iE zun1{5{fQLPi36nX58w$U_dc7_AnhK;{`(e`r^jm)Jju|R^@T4y<~zuPh;RkbyjSbr zf@Oq;W_FiZ^`9!g(yp7Oj7Nu3(2Wht?Fo7vQ;8$B=bYWQd;OW=uR`OW5_uj3pHyda z*?au+3@A38;O!>w33K?m*eUhums79~%aBIGm!ZAqC2SfrDs{Hel51x68gJH(iSDUM zLSJYw73c925hK%#Jl>$$zp;QH^#8)?1IW_$=ne3lOpF3kjJL*0_^vg^{i!Ph-LH@U zS<-_28Ew?emD0K--9;wp@<Qq^UikCl`C`M#eJ9$$=dJ4%0ey$sEeC^4KJ=c4orfko zA8jljXZOe4Qz#l<=3sb#x4kFjOhea%R4WIJx<vOoC_QJKH$tto(jpvCc|pIy-;LT` zXe^XD_}*v98M{zGahw%xDq|NPy9G?RQ6o4`lnU={pCMWhddVt=fzza*Qg20_Q-uUy z>k-k-G0|Z(#j$;71n^7nuXj2~drP3kFK=OO_PJ;RzL(q8M0fOza1T0gt6FFJ+KpLC zJmMm`udw`nh01es>bV`GI>hkm?BSyos$pMp6QRo<!iX$!DafJj$oPVy3CJ7b$%>@C zNscJcOM>V>bwC}#e+bH@;E@kVAsY8D7FATx?x4i}XWNjZ6(%#9wyxSaT8PJDVViJx zb}dMp;)MoQ<d{SD!7G1T#HwikVOjKZ_bYHf^jd!FxM%Y2jVosP(2TkIXQC<i@01;R zBlqg!R$c%by$u?Hkx9r>b@>4?(@4HS0nDNg{*?=@BKo|IR~s3mLh8j2jFUNZgB7<` z0&}>AKqhld-+}?Z0}c23Npe*X`UF}__(0ACV=KBAsKegG3AnwkAH&WSLMd?(`=P&p z<n?^Jen#-Q%-;l3JY~i4n!2N+Pj;+$od-ejZKe>|Dkl~ha73g>wHF_qWp@JyUis6; zJsw{%J$^4s7OfMt>pZdWPTOagcJlnzkyeM({u-#=pLo;&NpF@yftTY(-~feEfhldM zXld5-jVnq<1vK#I)w42Y2_dC#K7@OXfbM+y<<tMYr3N9i^vCdS59dF$`AGVt&aVeP zMe#pRmeQn`IG<NY0W?xO5=c_2?tUI3--X95lg&K6mzn5{Ab4SpIK3flz5g`Aak+*4 z*qZaL;<$2E?6QlAm%;g<4ODz$>OT>Z#b6@P-?9w*58N=@RSSDyUtV!=Zr<-E4>oJY z?>1?skp%>7<VN285C;Us*IZ|xoANdZMt^?ypB%PJFn(?vxaA~zU0eFez<GZdrFn#a zBAMJd;;||5AqMc3%cgU*r^L<syQCc#o91UPcp05zT!Q6B=#>N;l;Riwp)o#_174&# zWg^!S7|AuJ6?|TJi*4nhN7m6g9-Z+8itE$fdIwk(Yk-GX%&ZC>VLQ1s(d#S<!MD8% zk1KCJ=zoV>j*Oe6OAR}nb3u@dwD#XGj+QqIQM7Dl4?82|4Hr8j6#l6>zxJi(J=-Z* z^8OXnST>tMm-E9v7`6Arin3MT7%&Vmw?l$NEtgd|;0(t;%PUW&?4ZL7J{p+%6G&jv zc~sXP2cWYPl-0RYt%(Q8uun?Sk-{>yEYNUKZ^T-Zj#K(I40O|VIC|PyRK%k@iz+rX z$}mxa?esmyHc8($HLNJBq*l4tqX|=Y4RQg>K4g@~wk67o2!wl(^oB1k8{(Zd6NQ8C zWdbDlw{&B#S*y+0<$kYQG=D1C7Bsd+JoZp?B$;7(AS7b^GkXj6LwC&frYg{b(h%pr ztwfk@7)KsY%@nz%e)Izq{(kJK-?3@@h)Q+-AL}On{rLnG<ZFe@^D5!yUSQw>K+BT2 zCNqU0SuZDd#THYz8z0k}3=mp_`$Cd$1Pt=NO_S;WDQB6L$Y|#v#tk#NO1(wW1_<4p zu$IRCRhSy@lNj(5D3v7W)whsFz+72WSl~p{06$0z>I{@&YWu6mdAp;^kmL3TiV%h; ztlYZ)%M0M6O$N0K0WXyEi9llX-i*Y|A;J4bBumD1@-cfMLf1{ysOe0mlhB}IO25Yp zw-=|o(>ZM&Na%|8Y|XwW?7f_MR-Ad=rPia265?R-L|~CBebn1$(F5<w4{cuv09XsX z9w$Tf)!aLrAF6(9aR=AHwC8%ys-}R@fFb1Dw!}qArhfv49j8ABX#kt8ti&nu3N5(R zDvQ47_u%1<k(){xBrzdxuich5CECi&N3L4@l!*HwJUwN>t*Q(OfKyQ1*H{tQ3_nFH zJ*3?UW7s(56#VW$dn#;~E!1WCw8?m<$~rDT=T2mo14;vi4v8C8Z(~095IPini;$`K z8IqhYny9p4{CuDt`A%3bO;vsu7#tF19%+d=0`cz#vw}w#?Xw{`8>?34)%5^I?x8@& zzvqWO;fEPX<y~A`a<+x5k&V!iK*&~Us{e(!i!=Ck`v9)x?QZ*BY&<RK<8s1Di0fUZ znPOw#ncz0>U_cH!4H)zDKVpGO_^`6J@KL34sAAaJNIu*G8!6)j@s10HeDa!s^>x#q zL6*B=hC>Rco*;uChO9~_q0$qr?9ql>QB>Ro$G-h~VtR~X=XSz$j}n`v(>x{h;wWx+ zT<-s>@l4zux9{jFOKSNXM?*4Oc0L+6W}c0oA1Sf|mwqSEQl9{N35R7}x{OD`l)$G` z5)fc6E-YvPdLUMzw5uU5v7x?@vtIS9@RE!3^JzrZNq9+RP02UN%9a5;)@_I~&FT*b z0_d2w@lTm0#j>7P#rvKI?)jVgji)apMr!v3G+ZJs1u}6Mm%`VkKuN`vJ&p<_WI4vw zqz3Dy0U~q{&i3OXyQ!6|;U>i_nPhK>kkX6Bradj2;BBK#adeiEQ|u!4G?;k0Nd|!2 zK~)`LwAD0t`t9cL@<5@|^LY=uQ%s#k6U1k!LhCz2$KCw|jR%$6tGzRR=uGbdMT!c; z21*Z`NUn%wm=%3Q^w_`Nit#^ymYivNKK8B+uu~P3HSH!)xjO!`^RvkG90NYvI(QB0 zGCln@&0w|<r0?YbOQvBMu~0iZ%j6&-nSP5aEY6(KKzw9OMnAn=#s!#YQ~hITE6Z}~ zmPzR^=;;gbQx3ec?1IVYJ8oy9>iL;-zQUfvrqS2)HAvC9R9RKR$@)BSq!7AB8Gjae z&11xfl442KVK-qR?6}Kaab93kgveqO8&Xt_hRW%Sdn~!_$IS~>{g&*tl&7Rojf{y$ z13ARucbyDU^nHFuc<}RkSx0N1r#(`=G@v&@bzu1#5zOJROToQt<J(E{+a=XwMvLcD zpHJ<Vs`dO)EVfiUmJRY)sguyDMpXB$I*08%P$TT3I|kgg01!-l9Xni7P_b7w-(6x} zPkv)8JGM{H3u9E2`A?`f6A1NE1C3fbFShpn?}3JQ6nEnQaw$t5HYrpq0CdK!W8rtV zg=?j0SDsLv3w?S*rJ+D&RT}A^FDZf1SxT{vt=fnMk0f16`j0+lVMtiCEu*II*-ox9 zTjQZ*<Q)mau%~E{zHG~4mxoJgfP__8*?FSs<CY@3y(&V;GO&i27oNjjbQd2zpJ0p} zdqbYoxIDyi?sg6OCqYNfVn<S}2P^jEO$<;GvL!i2&RIF-`hh^?y#Bs6K7Z0QG^RNy z&dIQi)X#+++T9pBlI^vik&{^~ILwh1qAwoo0m`;6&1uypX-yz7Go^+;V)wrZ7W%m3 z+W<CNiM*~_b9k?w(iY9+GN%*UgM5P(RqAnVJw4-k!!e*n!;YSM)#{Dywxq%38^G;O zhK=b9+wKi(Wc4+WFFkVr)xBot*u@_ygK7c+#R31uuSn^E7nUHm+i9Ee&B;|iIawW) zj4YmS9w@buvCSky1T;T$dlkCjPFOzC$l<G}IL*mVd3I{z3Mo^5wPNgRFR&4vEyHxT zr&6F+8)B?F*VDp@;_Z6Ozx&L8R7XBfcC4~<r}26;`mjzS=uVUGjw+;`@F0dv2T_X^ zJ<T_6Ru3$}iY_{g?H3%r*$N|vpiyX#xS;hROgms@#bx`grEUVAFG+ITH?@jjA3!kg zhvKLm75+si=7PPpbGU<))hvCOKCR;OBZbTrijaDYioQib?=x<JpMM1X=1c=3#5jeq zU&W=s@LfhaM*_ozBi?`|Ag}@fANVyrcJg^aiWg)lV*3r^$3W66v;<@7Bdr{H38=q1 zzT}lPBH(&Gx<PmCx<n%sw$g^aK@%nWKfGKnB)w*wwsLeV1rXCfPuW+!%s@!dJ8$w0 z2aPc-Sqvu976EXs&gVykikOe)lVYH*ET5f9JJeh67^NP1d$JE#&7K7*Cn@y)mSZ%g zB-H2$;8KPO?9poCb@pifZE)LvDzeT2Klgp>@lwy=Ij4F}2)-Ec4DVR^IIU{d+HOOc z%y99J!3#y=pS^-|(}4XK<HXB7BD6QrxOs!K;+$yP6}RZ;AbSFXXy_p`jka5Hm+KrZ z84rjZEMxB~ZgfBP7>xw;!MXv*$Qe0S81YF~W(ihA927@B2YNQEX1>G$r(Gdz5gYNj zvJfN?{dg!+_gQ@4MU?5qU;RIz!tCHF1H1c{k0HknAJ~XkFfr$)yDJh%JAI9nKj^KQ zHu2Bblw_kQanHZya5ppTQ|qU@&mZAt##qwYXV@c~4GJ6MCnyElcEaOstx%rWa3|z? zxi@@1^1n&iI|d0nbC7wVkEeotG5MPQ8033-A%X7~AGfnHk)RYgu%FPLb|%rQgUA2i z0_EHnCxZmxmN)wa$~kr1282u!FBieBU&q+L&xXb@9f%|4zi|6)<Q^?$Cg;rs2RfYh zW%P^(n;usmt$xwAQKz4JUr9CG_7)gWa%Ps6q@p+4T59<)(I;-s(d{xAox%K&;o+Ck z@Hjbwu7pcXx#?wL@wgpP;=n8&z$m_lqll={2BZJVEC2U0oc=c@nnw{Oe2tOCe?!{K ztn|!o{-;{2191cr3nfb;%NjiaZq%gC(k1YM7k^&V$cG!G-(cE-U#tzsaE^>0+63#; zWU<^DZ-KcD%5>jugEW#d%1|I1CG_Dg35y~16~|;H5K@0%%#h<5NVqlm&@PK)kc{bY za46}l(}{|qd7t)pd`qoItDp17K19H^qeq&`!4BxKx{X79Q;fW<LAl|g>kJ0!V%S}& zNl8ks+CDj6VLK?nR!+3m5gR?n5@d`%8KGLoD+@<5rd(=xs=OAtW~Cl2!CJlaS0W`M z1rH;473BBtedAa>EfG7NCBL9HaUH&$Y^7tE4U(ta*shp{h$TJc*eh=DdF9Q2ck)8) z6vSAhXOu*iES<cWu?iEgYOC3$l77^iJBc!RGe&nlWUl`UY&?Y(&DOify(O*oPN8si z;}*8^?xC=YsUdQ+^=7@I8ayeq?GAB;C%4Nx4{5*AiAovfeZpw1!=(4+%&-b^9c%|_ zQ?#pt4Q3UG6z1=D%zT*xAI$K_ymeNH=XUN=q}p#CP2umlzHJhZ0@saGRVW$Z&fad3 zWfLVFq>s6xZ%KHKAe%?Z$PdR(^7qk!6!9rG>}oA}3gnO`0~?e3Wv@Y^2q$D<k3w72 zc>tp`8dl%MY(y{Qn+@GN+<YvD@WKA_974C=EG=xD28w!GZ7KsyRMPDobV?QT8+pI3 z+04E4FW!CF<i6((TwWO*N(D^O6idrq{3JEGyN^PcvtnM*T{CZ@zVHPW?fTc+)P%4f z{?`7a+FChr=&7`<u2Y;iZDTGbisD$$-G_kW*D0s(YJ57tYzr&D8Lf;ue@lWQ0D0iA zP~b8v0G@Y`Wv&I*J2iDNZ-(k>Jrxe!`IR$JWJ>DT1hnja{2GMoglLufpAC$cyZqu; zxBwhCl@F(%{|jo&ASmdVZ^JGBsq_DX60c~d+_Vz*H$d5B2aZXO7seqk?e|ya=Yj5q z>ykDYC>|)DCpP^3^zQsla-LkS^GfYFkPs2h@Uur*r#nRuzD=?4DOdGoxgAe#CIXS= z-cZ0`;~Fm`<NweH*ZCIktTGbQ5l_7rf)Ou6_Um0*ff7X6y{wcI;6>}FAfY8_ES>RG zT9-`eurR~@ljc4CfnPVZxPai;Cd#4whR_*~H!=?vCpSz15jJiRX<-!OP&}S!_(ePe zTzf7vUUQZ7WA`%O>B#OiAO19SK0Zh;)DB)34=WkfEqMN8L6I`#Zzj9xnnUEhg`_En z)1b@J-U_!Yi6npSzFWr_{RlITg)j`5!Q%u`W`P@R2N7}Yi*1w`lc<w)j1>4BNI`LN zSE_h54z{9pItIvC%d}nRxRRca*l(nyFHxBLDoAjYQALz0WoauCj?9pU6S~@WK8#BN zy)S0Oq$Wm1pX8c8jXDkCl*@y5GW#GRqP~!7H$8bba4GeuE%bHwCj~%U+yn>XI&&6! zAi?s4%Tt86RUPpR8Jsos7BB*?+k#Mrb(TNE^o2WGvt)f*KFi9yr}y-)64M|odCeA# z5RnV1ZFm`DB9Z5DUXqSR(S#29OS#&}Z%GqnrFSf%uHswwo)vGD$K+Yi-Skp^RFt2q z?3jO2^HSn=`=V9}FLC;8!QXjUVbQ~Sc<}Ic)!?C@ZyzBjJ5}Z|RorI1@zfV-*Vx1e zNj9kTty_PU-pUOop7yNJ!7r+_YmIKv!ZWs>tX+OndYQxb;yNGgb&5}<k3;^?Purq% zm8gLu&$sIAZ!^n>6wz0FZ-xbFCrEhohMslif+lVRYx%ZU-=`0A>Q7GBtujv#&TE}F zqzc&aH8451=i7GMT0|6El0F{VU-R_qn-3L`h&O1Z*OWCgRS?Od>@YKMS%)=C9o-C~ zZ%yL4jU?al6_ul49kuu>xM%le`dVn5MIai$$7ciG1(SDbZ#C&yL{L#ySTu8<-rxU( z!s1G9*sSXR?oLpp7CX$b*%8FCVh*E2*5&vK$*t)+Q>0E-7vR!8G5oCsj`57GFAn+l zbnnzZJ`8jr<}Y5A5LEsh54MG&F-;k#*Ii&a1<c+Nrx5#4)px;;$hTIDFY*N!T~>cT zXgu+mmfsxbA8EXSZ)vqT6FHB4UxMWl-B!A10uJfH1=E!8<MU$JpIDXJWYPaLEctL& zL+N(l7CrRzQUR61uA5KnZ_%bds+wA2g8#%cicIO(=?4j&&Q{4&@vQmXo@QiJ((!+_ zaKOP(p=I#J@<fM*!Z63wHn@bOc`BEwDoD2NgDPUGd0lb@)x{gkI9^bPJPmziL){E+ zM{wg(){E-0X}TWDNYyl=Wu(>RY<X(9pV+3Cld(S$24d=;YFTRjChrK;g&eyx&2ELg zNy?h52(T5te`<re%(O_k=Zy8=UyK8@(uAFxOkVFl<5s~dP@T{KFV2F4H#_4~uXp`X zO_L1^Tu`j^v`1O3v;WDaXW5xMG64l$6kpqs@xk-2<zE)bg|20xZ#$XBpBs?GXMVCG z>m_erZM$um&H_yc{EQU|$|faSCUD+9NSfZ6<dKRyS#bAM?qudmlcSsua^cFw>nOc} zdhs)H#YT~sE1Yz5V6;5G@jha*0!Sstvr!rdY|4BPn&IxbIYv=8@2;T_1Z&N*eAh!N zVLcsWV0p99_-5z#@h<X*&r^bj9?n&bv=UC{DzvsTTv?l+qlYX0AcGm3xZg2dP*;Oi z^jCj_3X?QPEbqU&hyShA?Q%~$0)e=cH3rO63`=~+9fMN}{wMEeuPhyRM9b6JPnWq4 zVd}|93lGi-d%<Apj-PrABI{AN)FirH|Dv-Cz2=$$Km4g4e(Vho3!EJ8aCyKYecVAu zkhROHB!hNSGR>>QO`C?Ht4uT7GqAZHG8=k73mi6L{Qy<#@7<8&vRKy`H48*sJ$Bfk z0n#M^^;S8DntSFg=j6<Cij7QkpRXEpoQS!<CB5^$UU9Ga@~j`vvtA~*QZKK>tEXBM z?)aoM>=aUM2~)^wC@q~`IOfK@X2j>HiUq_9bR#CM8(lOBWF<|QRGb<Z0D{^edUfuK zD;JK{VI)jXhFZ4X>yJkKWD*0(pB!9uJ#-3=ymCto#QjUG#l#0Y&=x}Gc_<GmrFxZA z>#@ONzF2+_6mW@@|MO$_zNbBt`whsFRsP$?AT4Otm;6+Tzd3=7mVXSv!OEK-RD&4Q zx&1GXwiaqpU9hx^Fnu7!{Pou>$;|so!atYWANedV6dC`2@#lY|ll2V>+7I%@(MN-S zkg_04ApzY__E$Ha`6^F@{&Ytr&x%ws3LD?arPjvhrIQOQqz<SUtgARhR&lQkjQyap za4;y67e*Tj@52xlOkw1FKRbFZ)=pWaAEBNhRt(zY31}u-q#+tWpBs`1mpRJPmSH_E zLiz)8E)PG1pJP(h#XLk&J5C&X$8De_M)&#B=hndHcHA_&0mB?{?GPg4J;H)LK;<aE zn26J#OFQ`iOD6RTzsSNz^`MI$FhEa^()$m6K4#~0`oK@8zn&SC$z-`bAt{w0qARW; z#nOIt>-;G<z6_AiEu-MoupPZtT|1L~)jJPtFzX8w)t76&*%efxNZH``S5IrP5xmmO zHgRg-X7eL4Dy(O<ULx62VN8wT=77vdhas6l<qIRC4}I*}U5Fgbhz>rt^7HY4^y<e* zF-E(U9IM?s*@eeyhgNi;sVT*6B`Tq;NeajS!LC2;I=kG&fb8H$$dPJJ0lBS=l4I#p z_nkYBI8k)(r-G`&ZLUt8Br;z_2adt5*{0>O<<qAx_g>*r%FT=1w$k{rzutKgbsi~p zdJxF&V#KY&GyO`az}2)9e}`2V_JacQ1!)%e=3?|@{<^mXwlZ41s!ZRPP0IGt9iW1? z&92R>3pSb7t6ib-_gI`%vL1|R-gS|(PgISmY4PG)w|Be)EnM_jiP^Q%9d4ROx6D68 zZ1H-!FEq+6_`#fg&z$`c+YdsxIUF-l0Vf0sC9h}66g-EHK_P+_%GY#wm`$!w*<n4B zjwff=_?f^ruu3s$Em}FHeGgn3Wpm9Le1^HgG6V}e+j}@ki%HFCs4;+N=lxr+*?hYq z?%)eIeaFhK0{AN*b@Cl9c^{FCcVkbTz2^b<<MPqCb|kR&Q~iTJ^tYa|8wEY92#@~a zPk~Z$DMmI@E;%p9c`5Ub_p7GQni!m0_T$vKC=eaV848Dydcz^j@kySc{xd;2Q1W<S z4obEGFf}JwZnQ2(j1b{uIpg1=>Nc2uU3c~Ajpv1@-}dEtH=^v?L32-H`VCq1W_X+% zR1m1rslfe>>ii1{qRs@i7xXQogq9X>sdq^F@0Fe$0j1E9lX|8tXHB$N(b&28?efT5 zSH^B6|J0?hbsx)nTCr0#koJA%I=ETXc3xT<-F&!ICAa7J)Blh`>b(FZ*cpFuJZ&1O zDLH$4D-y6*aSArBCdABCPUvXNo_fqc2t8z3I!^O8y8UhrC7y#G)aneE3nJk*3@07+ zZw3!yeQ&QnqS9Z8Z!*y!DE#u6^9|Nzq#a!k*aC)>AWrsgg@H?CN5P3g<(}P&CZ>wZ zLt2~)w`%QYHmhHMGc_94wOk$CQGYEvl<r)Snou@TJ+Ho<5+U_&x0#wAa#_zGxn%3) z|86KGUQ}QYtzB=4>#FQ2zW;tuoKO%t<f+=rc|FPNb$I;CVq<tigv>Of(j}(bb+t0M z$P3;kaddWH^9HE#lVASMU=Dnq;|4a!nsst4^8dVp{x=Tk+~DJJ|7o#yx%o8YC#Wjv zD}R2l;oV!p&u=YL2>1TRjGJ6&_C<X3Tx*`cKZmt6o&v!WSwc&_C3H=#?-Ja5v{c9^ zyi>0{=g_9hQcjm=%BQ5qChsXg^IAaJq~Gk@KCwKS*wNl6VIap2OYLsE#{$EpYY*WI z0DIroU9+V){eX&lb^lYR3U~JS-{)KUhFI<489OHR!MIk9#ag&V9P)Gdp}(#;3yh0> zXLT|w85|;rLJ3SirNR4K*i2S_0nsW)nE1?vI`8?d?ozhNuJ;9}&oMT(+KD82BXwAw zsABSeKe+!Ty;1+X(q_K4<!Vq8OzPB}-8+>aEx~vMoh2Hbjn~bo7(6QU#<9w^S06w7 z@M2uLhPaPQd~Fi8RpRsDRCpEKtQcT&*zlC_wL)l8dC{Mz<vLE9T&cfK2(g^=eWLi& ziGI@27^6o5lJa8d+jWqn#0e)i1E%S$HRfDU*;e@C`K1^zzO^gNIgbvcE$umw5%$y^ z3@`MRL$&iBFp>U1)Ga&};-ixU)@x421D5@HxO-MFM%TF)6?K@coD>r9;u?pEs&qBS zy2EnYK*Hr{miyB0!W5NtCbak&iQ-gAO8o)ze<9F*jkGt5tn=rN_s_&T#2+5W)xbAn z0a8}wnrmwzP9{Sw_`aDy3Z+>7iSs})_)a*LDh3@Yp??qAoIwUbE5+P^*WG2)5o4XA zU~p&?4^iM3x#3mdj7ynj8<>&%x&)ITA(;TQv0S?bwOCG?x9ch2DNMQsNM10oOHgac ztX-d`m)|P-z66*A!)C41g?to4dy`H4_>F_k*46`ixcvn=-Ftas_IhGCsJ6=ytpgGF zr850db`?EA__N}rL*YEDVfo|VoWFXoc;KN(z1w`K)c`oi%?49M%YsVBOF4`^$2CWO z=-$Hz-E!CmUM3Vki?rSS)49dVIyQ@PlJ0=BABARUg;NP}2n+%k1De$&okM5WG9W?Q zrLOS_F|2ZxqiP)O08>8Ks`z?oWeU3Z(Ej>S<HolJtsAh1)*Z&Mn1>T?k7h3H$-{MF zHroXMEN|}Ufb-3QUk`^aff7E6DhGez_cNn5sP1UmXK-<fn`MBt%0V)2Z$VZ1Kc~z? zfSJ$<F5jKv7NRn7Hi#dqw$WI$+I~zD-}2eKKTs6Qy7x-_=``q3oxw}>Is=+-cSEyW z5J8ZV>b}cOQmvJ%_?<dQ&MALY{fa|1&|wW}TM`2|JSnW7YZ^=TgVUt@<cdmJ!Wzr= z5?kdDE%4GX{;8o)|I<H$iW%Hup5IGvppA7YEeAu>=w55X5anYT<nfgcF6hhc2pUEA z34B&jq%yR&h@%HV+21Ysb^rU<(qInA3fr;xwV#fdCF^@&DXyFTHP^{{B>%|#TrY<f zUedyvIkxf&9y{%%{8|R%Fe}nxmKebQXS_9F8Q1haZW1w0DH~FEwFE94SlHouRX;fp z+`f45*jhrDRGyb0q#tI|dK%q$)bxt!H7Z`mq*`$;f9w7sgJ-SS<G<cnES+ci1JR>d zGHoQWoS>R#L|yZ1n~_P2J^_r|36)c%VnW1Z9+J(-Q-8-Lw^%U|9V2emmxo;1GWOrg zDhGxF24Rw2nb+|TaWir*I9y@qc2f2e1=Yj)t(jZ`CzhsNC5dQ1!UXx!bpGecV8nB# zDtU6H4yDRtBQkQB)fU$L=PfCZN~^GY&I2>qRqL)!tF7uC3cOr)+24+{gPBorPeAeb zJ1%qS%CD=Xu4~k?s(3#6AVR)m!Nx$Ij}&rELHd?tPnpq-`|&(q@HR=V2`S%RRhK}! z%H!BOPTiRw^*McSWVJUf%S`j)gim&7ks(%9uQF?on#6J)zdqD1N#6T-W^$u;EiK%! z;$%3H=IhB(8qppv*~)0n2K<wgW4<UkbE9r9+;cyH$;*k`CDTI2(=vbEUGErZ&IWMB zRx#376)Q}PEr$gN7ul6#cx}PI2}*~Gr6up<WG+gqG?QV&bBWkr#>gLmKUK?akSneP zR_);!CyHKvPdm9meyt|}ok*t@sT^2tU;n(<yGS0P*an=vg~YIB!k(>OGJ2`vKGx-H z)%**AGT#%;G@^h1sKV8nLLYSLcTjKXGtIqO^^&1H`+62`df%rva4DTns0;Hts%2h$ zPudr*2lKC~jJPsBN=omFkF#QY!n*BopDFw#FIFyz*OhkQzQ_b%H=+Yb>$hLfmrlJ* z8|On(LfLXlu;zbB90$03E}b$`puBm^^D1Tey`&aqaT(#N-kTy`J2RB@vO*6A|2D0s zT~VTB5apS_%W=miNqiw~@o)3fUQpDF^B6M9wqx%_uCG_5ZcqzJk>q?10&~e;|JSft za1I96dZhP%Uz-ayxzc^HQ;l3=`P0-nQ@{8IUk$*Kq?nWU?!0gRsf2anAKxQe9AP9E zJFOh{Todp-FzfqcYK>aa6%VYrv`LgwA@qTpcchPw#ye$SzmKat0DR&sey6*oCm3Y# zrT9(-z`_UG#wcV5p51Zu!7an#)y+&wikG;Rbf|WCJd2?%uk`uGcXL1?i~wdY{6C$n zp3bQ3#8XwNlzm-ruVOI_h%2RerT#eB<yGh0`RW&D%?!8Zx4sv=cNkyy$qGGO_3WxD zi9G7L^j9(y7;xJ%jPV;~&9m)Up*&Vk6b8A!dul(w6cX#JdAH=%l@E?KPF$X^;qm(- z$)4^jsoE#cyqyS#Wv<V&HJ9!c@BV?3xzWCNRBq&cVNksg8n3HlZQydFB2~%!r#O|$ zx*_&JoaFsHqZ0HlbsNjNb$BtTx4MHb8EQy-9n^?Z`@gX>*1(I%xY<HYmxo2E?A(Gc zc%Rwys%E(1RzGP9UEZ?vVX{#RPh1k_1A?`@W$t#`;Q#Uh@C*!$49xNS8T6U!EW;Q` zSJ^NYtw__$q!9`#SqtIzf#luFKdg`JNXYc3(gRud)`F60O&vzar#4C08o_x=(-L=I zgS0lc-7{dQal%!V2{^R__r8zviO;+0^K7wTcQckBpDFu%G1C@K&n~K$Ajsx+|J%1e z(<<A0lz+NxwDOe^Vl>(>z8^4nd}{+)+%lm^{^=>c1?y&eSrGarErL_2FC^|^@OU?} z->}z7HekuEVjd+R0X-<2ePzO(b;W28clvro_>}CzyPvbdB1g(uFTAc1bN#Rx(6zol zfiq*0vkm;8GK5x%of(2$HbAj6vSrE{5)atl^9PDw@BEP!uWefix8d!PrKXN8ZAYDJ zTb-p992?9vxdF$_%Q>5Y6f-NJgqo|aeT9i6B`wGVH%(VMP&u{qjPd2c$wS+uxNxg5 zU{{Cp2VlHvs%eDP3X^C#?DiAXhtu-7K%px!lJR?fVp0RoCj%IJ+Jb4$d%CrTeM?O7 zgR<9Eqmm_%G!0)pluvlcA>X|&3?2<b43b)b(f5<mlFjDDW=%FJ%lW_i>cDsi@zYru z1<}Hn^mA<sxg8M)|E$R*#wEKE64w|Jynqg1JzPc1g)hF)`EVcP1|Un@uO--Cb%Jra z1;U61vs_b|J@r)j8Kouq!5jn6bbMen1LZRk+aC}atjIJ^ee)m=9bTAs$-gEz(vu$e z!v5|u<fkG<$I^+vvI?E$r&$}j$ZRx@5h)u(c(LUJ1pm#`o$Bn;9)@Wv+>pH3)c&$m z&AJqd-7l?QpcN=<_~wyRw=gA3`MoDLJX1^N@sLGSYVwtw=M?dy1-k$u7&??vXb4u4 z4+0i&IKjtxg0JDEZZ6d8=AZ|#4i||}xT@HTtn7Z}DkQP@Lb16yXz}N1gUmXdOiPZz z1ZXlegPbk~2TLDM=*}%(3n_b94~vy4_+i4?>19yk02f8d{-ti{VAbGM9i0CI(yKk| zTfa~%y|nr`>eUgIL6k1zKTJM8TSsSuw_lM3L=!5-bXL55eh&Y2X-!>MEe&i~y2R)1 z!AX8vf)$RK1ZyXBh8BU#%SC}GpCRhXGyf;83wce1=~4Id4TOQSOo~leqoIU0i3?e( zjRL0lveWz(nB3^p{GRRgDH%Z4+&#tZC8k_6#5cQXq$4JX`tr3?j(eBkonx%x@~~2w z*Om-?Aa|d;-EFYwQJhkY?12C-=M9zg^O(t($?xMcnYMm^yv9LgxkN-1)13x2AUirO zT=&B9rB+D>lmN~A7B~8>x}GX-^vR&?&W>r*P6u;M2HETZeLt6M{mgqk3(768GzX@t zdM%mDMuXIcLJe<4v2*U<WCP|y^KQi&fh=w1=6_f!U6y?_w(~X=cmZ0qEA_D}|L+-L zwx=PaH<3@iKXjN&BjeEvlzAP-uWOd`3ZxZV5g^Kj&?XtfUS<r+8gk@DBbvD`tD0G$ zv;>Ug4XB#Y?V~445mc76Ru5KZi6tyCzIAfViWF@nbkMpSmbkrVOc6^~$f!wn(8rbK z(M-JL5Sezm`><=vnfJmk>1K5-SvzB;3{NgFcVRsI`eIft*mCyOQpUSy6n`38*#k1J z5>>FDZFVg2@<y3@>ZQaineS&0f|87q`t9}?Q53e=3U#TU+FOMoez{yv9$YxL_e`Oa z7~B^MkGQs1LC+>j)+W3hFG|RyM!4U{T~Rsd)^g%(xx6>9-VJu*UPZ-MQ`%BR=Ix}N zogZJ0?}4C@(WdMA;8xWL79xFc-4g5*2~eBRMa6LRET-=@AkKLwjAWrjZZux*(|-%# zRDtrIYH&ik&n<G2m?JBkXvp2+qq44#%1m?VOHfcFxxuh^BVt7dEzAo&%dpGp31nz+ z{$;MGh=kWwXi{r21}~wX+1*ZRcD4`&f%A&@*{`iFU#X(sOr}+YDOl2-<}&$cFZMm; zuLD<}AoWcL_iAn4vdJ}L8wcIFKXtL&a#lS?nE^T{b0zx0fhcq#VBs&?UGr<?!#>AO z_Xi0?gEkqi;r#Y-l;5y;@&K-@r{e%3Lpgo1wskkfCmlN1DbtfN>r;@U!QGr=CCu19 zP{n=jyUc=EJj!~wYqN`3Gx3zEYkP-831!zSSgk5>x1oO4Z|$X2EXXdQmkE(|1M-=N z{$DsTMb80lc&m}bGD|th?4}Jz%$$XYeNmaQPs-NdP|<y5GvW8Q@CiD@zoydz<56YB z+$F+*-PN7f3|0Je@_EMa^TdBDi&>OVo$Wyf9V3+0>kllBnQ6+GccyCWnLw&HY|3a* zGX34hp!w6kQ58fs8;RGz4D$u)gO(#59{^>{!x$t#NLC(C8Fjh@9dgXLb~5CuMlxdd zT$~PTT@c|du?lr04~?dQy-OBnB*#BM%Dt}tT0FXk^OLd(=p_z~&JJb-+FE3Ja;)A` zUj;nj9=h*VTOE$pfY?UlH7)7fAN@f81=VoQekn=!1v;L$;>%}>a*gB%aOhbW<oRrL zZTjH`zI5gEp!u)^knV3B3+w<9k_R(aPp@P=k1-`}VBC5)@p8!EW6S>Iqg5cm3h!i; zdyZL5k=uJOgTQrwE9+3$Evdz}0>oL2A$mRCg%O~h9v^z&-2Fq`?{S@X>-oW!s2RL7 zbA@r<Q*Q?LMbnH|ey5G2b4wbOL(60mhNwRTVo?@YAr`T<HgXwIp6Pv@(Dt&V&q`9= zW#N&jQFV*Q$DX+^7H#Bpt-8a9ZLH#F16y{19WNz7K~z#p!l9dS(}suST;syA`&keo zt_R~Bv(=m1Z@l!yaPYmr9NXY_n{Y<F^G43c#9X6b!m&A!;0X(7F<}LMT4+je8O_L# zP6v_Gi{kdx#y1fzt!I?2mq#)<>%Vj&Mv>v>`)k4DL75B>yW91f!6ByQ>)h}ukGLA` z*x5W95H3~MyS9nbu{*kd#-&-_tonU?%UT|98neu~{_$O0!Otsap&l1>aAVMFqV${H zOquI^po2lt*nF1%o@0LgmO_PX7CSQ4_vIehWzg#;K8^l~+KLHiu|PZkb-tBf5E)UZ z6(qk%{E3BEKkF0;1m^#P{Z)9J%)9N7xSGYq1T7ii(Py^g+bZ#$;$G|`cI`KKsY&O4 zx1vc?CW0B=d_rV5z4@`VNi>7~CwBu4S_iYzlwY;_>|uoKRl{nr{w{AX9>1I2W5ty# zTZ=9|vgZrAfxO6_dDLe6Kt6>o-U4q}Qr&h{h;oVnqFX2JI3P!vw&!CJv7>k_TrVix z)1Ej2>^bf&1UanedG7k5jj6@7WEzxX-#%OPC3^fVT_fTeZ7x)p+>I<a(A~;%<ZT8u zn4GJOyS^*n+T3<sPM=S+3AC9;m5XwxKE5=Nfk!udgrVCbxtgz%5Q^5vrlig&6P>Ue zcZ;f8O275I_tIxj@>987nZ6z^YsWR&$nI`ZFFQgk#>!&}w%vBcj*iNrmhiqtNP^Lr z%n5mC?wf+(rRkOpMq77jWn{(<TltkFX=lRhIEC2PB8Jo#AiU_l!e}-S7%x@efUO3$ zlUr(-y>+K=(=|LU-4yvc0)3;B;r_s9*`wcHQ#N7B(3hcg0$L5y-o)1|=o{(>R;=Hr z3K4=NtlU0c_lVmrSIG8YP0E5ayQzk?-uYeLrRB62<Mo8+>y&UnL4k&xkFuN(8CYer zwr>}+v^Ik(#yHe?m!37Ruua1JDK4_&-fo#Zo~zEA+JD>G(;S%EE{pcQ$30Z)PYd69 zg#_4T`YMwTPwTSNT9W(q!s9BlICGw<NHu~UQmJ3LN&A+0V}$~qXMJ_yX(`qA7pvwK zR#L_muii>Mo)Vx3op~2Q3}O|vAeTQJ6dx%ylG+&kPUQ$B-9&nTI_18$51z|bmY@IS z_|3m)O4_k!USpJ{X4C<Vj)+8nRsmv~0;p3{LGL)WX!aG!*u^|Fo3%=I`bot*1Mo*Q z-Ek1`c4>XqETi?@_96n28Ez0giih>6AWd49!u3pFZ|_VNvt)Qq8Tk1x;9lO4M3;ID z>4>{gUL4j5l)t|<EI&TsiCt=ENNK{fQT!UmHUXW_1u_m|prf7HYr_EF&3_2qej3hX zATN!d_gFqKFExv+;6^h7Lw$Gf$n^s@mbc_&RE*K*F7<0kVY%oGG7p{)ia`N}lqS1W zrN2^T3Fp(U!w;HjA}2jy4?*;7VtM;fH+((QY`A?kw~7gDjj0j-+`;nEoH9C-^tpe- zz|aPQx7vJX!&tqGDQ?@8$UbSh06+9V5KbIO8Po5(w`GrZZ->U|!p)tpyJSyQB>s}d zrTbBnQlTre620;(7F0PbWfp>y8k?^uswCH0`%l{o&B@e#O{<;t1p!o3A$^|IoG4Y? zk9T75Xo@>W+a!-wgOb0*=uN46E}=h)2)4dB!u&2YJFp?X?_1Q0=;M;@tY2LGx&Lhx z_|usK>xd$y#6{xe^aU4{{Y#E!X}<JtdR%Ows(z~~bpe9~l<zOj#)B#hRQ4N{YRGf1 zoJ@9${Q`aB*aBg$1r~vNDU`H4$K8q1<w_*+HW#9T3jdubuk_Lk#s5NS)vnOfj$f1H z@>Cm~a*d1<R%&E0YO#jJU&jsqj6=NAe!&!!M5@;XXPEVRa5U(YpS;$>6o26_IGcD# zS{lH{v3+2j`&Ibc%0TaYB=rW?GB-i^i$6$TD9q*)VQjB?iDI*~-IUz;x5;4AERos- zdt`))Q7pts&q6}uN}~PeiNJ5$H&gTvu7#a?Ipp-V{Cm$gGJ+wF2eEIdrE`C&$Uect z+9GaLV?BhnzujY8g9q^<4((Y&cdah!r)YjCb~h!SeQzJg%ePtRZTPJ7CRCL(sKZae zGe6WsN1ef|e=;svgAD;~g3KhoPTo1<#(ocZlGQ@nMTat4{S6k1@JEB{MJZ&bE$Vwr zB{XrZ=#LuelFHplvAj5amFi|jx|?188%-HxLO<Jj0=Qjqy4#VGbpbm*-BT|RsxUsV z@&^pMR{GJU5}sA2Jl|w0CA{h$jKe*<G$ymI%pI|^AA3GbqCZqGgkxQ)tIK9Mg9fV4 zn)>>`aeQjGNY{2i#?;XAMt)fm>!zk~7Q{2rN})L35P33&x>Qe(%nhVo^tUjo2|nrQ z;U%gvw4jWZ8d#v+bt*hH^Hc{ZyenBpfh9(B>h_UPEE+77WZ*BA>Lgvm@-;HbHU3Td zKpN37=i!jTMev3QtCNYlX<OBQz4i*OV(#U&j6wJC>+ZWFS);Q0(FnysjJEHa1Uv(h z+cYW=)lRaY@MV3Q-^v%_e)3Fgm&bJ&tloD<c>0rYaY@P}HyK6RYXZ76VPb!x@iVHk ziMk=LLrFjG7M<OfO~dS#C82V`J7MN;fnuA*wdn6919GtwNtDn1Fs|06fF0@ka*~m= z9eYR2lmXS{M3a;tpj3mh1Fll={8Q+dU0)fu_@71qb=O@o>4whf($@4PHD~MgsX)`C z@lcJUq?zSD7jd)jzFG5#3SIlSwju)nh>|7O;E4YvwH;YA5Q#_*mp0}Ur#FpE3u;el zHm{~S4h6}Y9p))7VY#O;6eOybv*&*_k@93cS^j1A!$2(oS>(yqWCGz!8A#eO<Ujin z#&90Nki!T<O6u9)DB%bM0po&QOE&G`jpQW7XbCvf`W~*thKZK9Oj(L|6S;u3wSKhw zusASWx<m%pl$`!_x{m%R>=-t3QCMRFA(wB=gHxOfwEi@aS@XM4$=<J&{i*x^F!t6# zQ8sMf@Ggy{A{`4C0-|&`2$z5Yq9WZa-66S4mr5)aDJ3PbbV`GSuyp6rNH4vx#C!V8 zchC3FcYn|OCo{~>2{Y%h$MH+2vi+aybbq`~MjRn(x7P>5uR?EBc6u;&nUUd?GQH!w z08bbxFVrbx8t;z*XvMa@Iy9xenLk&PBMXUzcs*bPl7w56y72kUEnfV?oI0`$-9<>+ z{%)Bfg}|s>-fbbg+N<0~Cw&Baho-i_Id6^=Q$kjO%G#|!_P4~vjF|36OTylgWJ=u~ zDJjufDLWwfVp}_jzOw4k=ONLG_EX10is)YW{9A`MVaG(g$=7)-_ioHKc^Sjxvuf1z zhvbbm#ieGUpVtZI3#dn5j6Cm;_y#2_i|*Up?!^7ZZJ?}5`f_Iq<_$n=c<e(V8b-nm zM1*UL>hceAqI{my{`ZyYKA|lgq3088nch@8!xTd?-fOHYtvWaX8LP3#_RDOLVj`!R zdJjzMpYGk(BW$5^mQ9Ie*umhnPPcr$7T#PpBn=$yU(HB@)f}qg{LGbS`1mo5JL(RZ z-4S-wCmrs(mx#UR4AhFpVI}kCIzWjDzJ()q2>^hwJWg2`kU*5w>X?0?3jL97pd$&_ z!~k@?9L7VFm^3>Vi*U}|;B%mJ0#uQB5dBa$@ukeAxa6T%w`*7$>`J6`ZqOPoBl+oJ zXm^bBQDV)~>iq;gW?47%#k9usZ{z6k%#}pnD{NrFncPT3LJcZZP9WDlzw7fO9P0p8 zlS0;>f0GmUSLCd3%fS-^GPLlq2^?Y7<)-aC!v4@ni}WH-w>zhR$iqt7h}eqhxB->@ ztQ6YIV|^_`*|niVqL45Cq}1Vx5!~+-$VKt6)N$u~wbp*;j}i$=5$Gf0^sOK=rM11j zX2(8D?#by_<~8COJufJcuNf)YYT%M<brs@I$CTz&R_!v*xa?m<`$uGOS*1i*?6F{{ zVIeQc6(~<lnT`FaKk1s)#+)B|6W~g^B$yx+CST8wo1gOG%DxHSvralxi<uEa*rQ6C zf4~){;K_B6aKx7T<NbpxQDjr?c{%$ZfCm6$gC3anBb)Io^q`fmCbhjGqb{wrBD2|U z+1}vR&Tw6+tJtr1L99n{)0fP~2&;AftD_QxQQ+v5yvq>`d-?gYDdx1dp0i={8#+7p zEEH`HvMvaC0@SO=M9m&47gth}W(BY_8LGWd)kO|91hP{LBsP4xG*aufqWCh%s%Dl{ zM@ihq82Ka}a|raM6Yi8YmTm<m*PQtFu#vR_C()nbJPoo4M-$++4E}q8I9>V_r2BYg zf+ilyxw2@&*J%wL%9_U$_v=?F4P0s_zEyp0cYpJJB1vd$A8`APztKM5Us&EQE{pQb zfKg7~tP2kKpprg`|B1c0Rh@GsgZ9L`6xDfc4h~Qi!*3!*EM>AZ5Eg(i+7iDz9g}(6 z7UnfXA?wmCUh=}L55F}kM0R;9S&L58t-n}1Q^EZzKkdbZl*l>YZ~;&Iq65>mXW*R_ z%O5(a+%Dum;b%KjRD+7QS3c{R0o%L=001^eX^sv4%-wF(xHmc84Ga@<VixzdFGKj4 z!`!Fzgx|=96mTVW51G#SPQn3)V(S&53`h0Oh$5&hRR&o@XA;VoLW(D9<^7Upd!z@_ zsd!M2?;}FV(iTqnx0c@hHYKx3@_at7lV}lkAYX-=vCE`@y()3<mM<a~eH+gtUIe}Y znrs*LJ=&Q^g73z<VxKw9e`wzyA*#Okj9;<QG1P?YqIUzo*Ey$P35!F{nNTx!dg9u8 z{;8pCI14!EpG?(ShZ@HE%lb{L^@UR9W=9}612c8OYcg5T{n{Cjr!k6U5|4BBD?Z#4 zT1{R(lEvxzgGRZ#XmbY)+w;YfI~i6f;_}0&=|8Tj|9T<8vsx8lp|gzvReRgom#z@s z8&#c9ye$~2dvM>pizlz;UcBl%sm+y?pE@FzPR9?1Tk+WGLDD_)8YztC;G2%!>YJ{e z*fO)NC;uI3ud2i&{5RvPTJYsIu=%ruJ%u(`T6{8)zG``HSY#f~|HnpOkbxFoqPj(W zK+ZdY{av!zBrG{&ce2uzG!Qrsx7OWK?h2mWlZz%AR;B9YwTxrI8iX_ID7g=~0lA$0 z*h0`xB1-?PR|+?DZ7U-gLxvOzlUeY^TEdWgke{v<QS?*3WOgyC^_VLHS^rf{@|O2o z?8`iO3UW~Ta0JxF>5xhB_)90Q1&Bm&S)xuL*n}QVjSl6=Pw-oExfzlreD;tdu(dx{ z;PNqOIY6q!x1yF9jyYZS*4hv(iaXcqMM8roKqqjZ`XD-aaT3lBNlf_%dK}%|DWk4* ztW9>Z;?aX=AWxlR>!q1Q{p<Zs$ZoU=bMEacmX%sfEo^enlNs(CO(&))M5j5hdTl${ zl;+<ULJN?OJr|!@h!6-J3FkBy5x$JRQdQ1^@7jM8AAY02jIoRUQqM9PQ(d?6q}Q#G ze8$GXqGpK7id0_1ZwZ}na6HXX#rpXmU?ZL==5tW?VG>S~f@P}D3Bx7$cr|KQ<q@0( z&#y2OemfW#J>+%~JJ5e*(EjA@R&XJvVsjX;0iiaqJ`H%grFQu_0%XQjai(m_z1SxH zbHnj7oOUaI!JfC3@fZPB&!kzZex;o5TXgc>cGMv65Z`#~bb!kiy)mRFz<(a2@c1wQ zP>UI93V=L|JH}vVcWIP@PDbbw9?&i=XRJr+<>@mUn$NL|4(;}ZG8J39eUb1NWtdE< zsDDnU(jOG@xas99`=(}grrCDOT+q9dqkh58|ABm>1T^@-JuJa+PMA|PCF?j2`4fd| zG2%xCe#2gDa)-#h{qxemdqOuyLef=8#1XLaHI^TCGZ!0P0KQg$Y+DLnj@irzr}eYd z9a#aYZSWKUm3iu30!Ws5>^&LZFh~gL9I|JX{_%HhVS8#@b6Lyf9l-ON#I590bvhMI zCeXqlB>EO}=b`{h***lUBxaEw=W`xvQZDBU#;Km5h(t89<?6^CcMwF61UUXldpr|n zL+?gkxBiT)T@$Z%U^|)`4<wklYv9%><=)%Py1WWx(cN|Zxy7A{u565-1%Udiah=um ze`b8O(tmn~&mH_M&btA*J8b{j?ENz$bW<~B(Tq>^W-qQAZ-a*>$>bjJk#!Ht0|9|k zwYB<-2t-~eB#w;&zpNhb=8KSOx{P<2hCl;>gsPI<CkI)XkjbkN%3s5ea&SevlGspy z(x{NMeTH-TU>WDQGcCHDx`*ufiBLd1nMQ-*u_EH&-Xv2Z#W~OZA8CWVk_IsET^a?c zRea$a#q}a`b|{pks01(5%Cv!<@1PnLm3AlK!%-_li0($lBH_^vVCODnV;|kOrJ&oG z3EwmvleB)FR>rf|-jK*fC&XG6k3Z-qIH|+cFL=&-!qe15U3sE*B|Swat=e#@e!YF> zVKWFQjX=E>4#q8l|3z;7|H_O~Y0H(YujcxJh)-f{5NN1eS^P_EZ&%v(>ozj>vHo<k zgm-iLghb>v)hCYBVTk7I!;sEVww*l3q5gwKP1B}<VYIgn8iK?_9|f(pien{n?a0Go zo4=K`GZ}ExW^D?d6oma4jrwxLMDPhO;aSMY&gz;59XuA@Q{4lTvM34V&h{X&F+FLy zZ_c&&hS6)cF~Uuq>VQF=giHL!f2rW*{8~s<;u8T0A~vPrF@=TCcp~nto9nNw>WCnl z(;>4T4+>DGdrJh`+>%sqmb6ygp3?Ym{0DM~S^pmG7};q%vg!7N>Tv0j-P}T491A30 zBvEj4cIo|`NKhVDzAmQwWk=5s?}s!ZifcwYt()$6F1rs<PByI+dxp>y$(qWerTb09 zn<A_am2LPy^sQ6dSwlzzzA1LbXt}IJBCVrj)xh?ldbGR6D|{puHD93StSz!&IzP5} zG9zY4#c6jv`ymH*sw%iZcm*RIS4EWCcUFUMUysP^Xl-()gln$GU1lQ_iRj>b%R0sJ zMHu1P-(Uw<!63DnB&4`*IQx2#jMz=XnbTfh4fS3@r*(;cCp0`Za-QA`@cPHYwE&f4 zuH?+WUfTOAYnQHjDkVoEHx?#eaDFUq6hae|{EP+unqwP-y!78F|7xyX9a?}_8!Y<I zDuX*r-S?tV@hW{$BCu8zY*qLX?#9ZjhGCr3G_6~|WSCnuh9ywXv(DAIGdB%ApA1OX zSv-ovW3Mavc1e~y9Wr&@Ov;XiduixY?D{4Gc(v-|A<G8@X;##+C7i?xlc1t`h1@S= zmE`$tKgWHGwZ=tgw^zI^uQOt!DimN(#DFMX+vRy>>M0n16-tCc-R>{nhxbv9g7r?T zG(hzi=>c0+J;1BOwW`lNtda!EKYdJL3~dDnKpC@4aE)amda)VSNt0XfW2-OddTBF; zZxs8s5fDk*nFl;`aRL<5(K=iVkH9a4^MT|^#&2GO(vE?qKi`Ia<W#GL(}zBrd-Q|E z)KeuR_{JXqR-zr5<7Ky7I_^B4{ux~vSt%{%F#160&mX{MqPw%*u%77t;$ZQWd-9!D zpt-Xg$YGG2n6MdnYPsf8M<_~+fGD;c5^MXj0#EF|ayC~7pRai=hnsbJ6f$3BH^jT@ z5E4NGfq0L<3Q1}7Dg$OLRib&2f1}hnKK`tP_dl&<zaTn15vSVD@+0{*x1SS@DHAEI zA0Eurl@;0Kc!}NWn=uN}oB`+*!&h>ur+)yb4L3|!qnwY9&J4Pvk6-}fyEaeAJye?X zcwS~U%)I$xTc-W-=t`p`z98bF0Y|Yoi9O_3^6hJBXRgZidqY{%DN$_SyHi<N<VZ=~ zYr=+-V*|e9i64%GA$A>eTA(9FyS5>pMgQmACx1|Bhpq|b?gDR7Ff+1{SX#XSIaJ(= z_d4FdJ>?h3==S_Ns$ejs;#2cmj1Z&T8}0yywhA86iW25b5BPty00y=z1Ca?T0lC)t z+_-vElB?HxyKk_@$M}>SFiR`x8-vnSCVnB$zPz|s@d}j#GH?^y7+ZE2IDstW+selJ z;(u34{>M+!j7RrQGs^?~F{Ve8pYS!v8h()&f?p=j@^HA4yKB3U?ALF?(P0Iuat@qj z^P9a;de0@Ov-u50qVtM{wx-*q*)|)@BeTx|!`FNjckceMYxEz2C*5WOlV)a{hQn6t zHi{U#fUQNcdq$wL%`SH>#nAy#pL;Av)xUQr+I!6Q?61WB7%5*=YB%u((TACe$!~`t z3it0{KVVn8KHt`(RY+t9pFQb+<8bj2PO2z=<`|OYJD8=V3)jbg;kSDhn<H^9x2~>D z8eCvM<g&iH3RdG^STj){I7=Np|G5lxoFjwe*s{oRzKpTr2`j6g8WG3{O_=(;(m4C` ziIdu+VdzCA&y7>iTAZRMx%^UN&A~^E@ct9PgeB2t$GY=Rch`0+kAgZ$pA70(6&0tf zv>5X-DOnhg`I4~VG2$ipdC9ZMc409p?75yZ>7Kb*NYc=&3}LgdNftBm(}g}6zGVQH z;60#-v%GV)bIME_-|wfo`UF}89xB8%(1x$W$NuhD6-}rIC0$}$jca5qOXPcbDwu|z zbHL}JpmONHc?ctofz1xpn1>724Dq$;;H&?=rCk$jjW!(jRg4US{3sy7M_hYLcqhF3 z(O9<+G)%qP(0?||Y7aQ57e1ixo#ju{h9e4*J;brjVF`Ch_A8br+cv?Ol1Jx31k}dF zk1^9Zdv??c`@sub6iYKOKyco(w4Wfa>+)l#jbT9Y0VIpnV0A)fvt@>R)h)Q-Ft5Ey zR4n8h9>yIB6>_vUi7hF6rt*6k#qh5REwM-|5eJ)t!E^LZpMi7D7Qi4^bPnx=Aewz1 zC|hv+5nB@^$z2K8HCA0iC`Kt}1j(4lS*z;^qwZaP0w2~krr0!P0Ew4?j3})gC`+4t z1F>Cc24;sPU^1!J63Z&F%kU?nl-(6#ko74ss?{$ubEmUKKV^4$&$ZO#RO%2?HyL^8 z2-uFDFN`h|u>M%gGJlFuoD*<XI{K??F?N3QcdWksGn`$XhL><W2NV=}O#|>2sA_re zZ+cYAor}#sBbb3`NJsEyD%p*=Ejc-}1UX=p<<N=d0B~|A*IGkrmW@aN*fur@*vgx@ z1P(LtWPnf{sAUU08Bt9%^-l*frphIi_mLF`=C_ynoX{#Cj^36(xzBVqq|=Ve0#9lG z3e-dP4CpWCfyI$*&bfxxV#95(L;CIdSN+);>EBd%6*51@I1z2TB<W1+TIqvwG-trc z$BkCpz;rHe%(+S8fJ0PLEF6tgIH@~aUH(-y=-Bt%MK@|zF8FvrsOQ&v{yM*@$3+R9 z1z5Kx00>mL!9wddQk|IGtG-rzn|pingt)y-TdIB63q$5|v-ays=8QQLF0!`_Wf6^q z0Sr0ia)I)+G`qVgM))kfE4HNhHBr+70lWFzS7ae_H0Jd!qtCub&%}#6tU+24QfpIy zYI}3D&_MiH$gL*cxk{JAx_Wcf5`!Ak?x1X*&HU1>)fKLf0VV#S4NuSk8B(@ebtZF6 zlg$KxX1|dSAOmV_l)5uLSag=%1Rm6dpVPE&Om{6?;(hmW9ucA2+|D_rQ0%XW*?#TJ z3-SafK*w_6=Kodd`JYYg*IzlfNsgyl^Bq6%+Bd!R{$Ij=Dz&d@e|;3>hF<e3(JUT- zC2|+&;4It0x>&9LcM(48Z9PYk7GW=}guYD>Q#`LwqNKE3k!N`j1NLB&`i$>&9DwaV zXJgj@+$8mrCM%bR^w?%l3i0rr0VoA~9uCl41;Vc$$g!lsj3%GDHF}n=4DlZ3JI+mK zpuR+B`tNN9sT^zKAvwy~2}=E)6EI$myGUk*K~`C_7csXP4bkTosN+A}*BL;DCgZ{U zgjG^O3E{lc%LE>%<2JkiC(W|{TgVBS0w2tcg(yyklrfs9I0eVKSamB$XspSp&z!r1 z_&3XGTNJZ>piZvpJzub#*IYHGt~kt%ilyHz?|V9XDl686j&t78%-@v#L;km11YDt- zeFQ)-!0TueAOfZlBW}u^1`JPE*#UgYEvINwLyT})%uYdQF?^`&T)SG;DRA?rli~Ng z%wvD#1<_C<6JFBAPkySN_i^-fzpHK>U-C}2L+ObjHn@rlKB}M*D}8c>#2^iq?#X}2 z{J!Gcb_|s}8fSz4sbzB-g8bk(_{`%~O%&5p4P22shX0nfGr<c3ZIK#BhLIVsumPG- zH5knW%jY`~r{hex*0xJ17T%n7St6gy<7yC1*K6<{=`=atCF3xl#B6Q2<<`m8!i-s> z2r|7rN?77}e$~JVwgOPVa9T0+RK|q^WOb(Q4WVx#g|x4{OMhj>1#MAPDXKr!4UqKd zUNF)r-Jow@UwQO3^MXp9IYgPw8XfP6{HgsKV;$s8_VH7@(3c7Z<`ZRg;e>~T+kL;` zEd7(;@V`qx5g-t2^gY;TWAo=!<}5}wiNJ+iSz_V)(p4EFIch3B@vM73UxZ0)$hu4V zNUQ}~>OaM4p+cX<@Dm~8Sm*qJpvu`!8MoU`h{e8Ot26#Gb;#n|F|F{+D?m>X#X4!$ zC~rS_d3*%GLI6TeSqB*G$}>1rqS{jX`KLbprY&PoPZ8DTC+m(2?>T6s3i7_(ay2(_ zLe^N+Oz4F_-OOO#xY`v>z9v=-nV~nKZUJ^NqVgH+0id|pO>^t`e0zZ)TbU06;4Q(; z%!4SzZv%qwXh7V_J$Lt;X&6ZHkUyg=6jMf|LZxu}W;y|64N4ZW3?seuF|)qQ0K_<y zuMjbVP_lyXOOz<)=oT03zQBn9#XI~_?hR@S0Sj#wmv!i9%YkM4V`sl(E;cjK@Kjbs z_kTW}ddC3|c&=bCZ5nNkI6I)yu$Z)ZTAcMFoVYWJBsfNe&a8f!X!;Dh8XkxS{L~ge z_HdL<k|~``;3e-B?mf>^(Opi9+>a@8Z)P$n<?s7+Xzl|{<i&aSUxLEe84RF~0`LPE zI_))UjT&{qyYo!=5vd7qyJMzWZ^XAddDE68@T*UI8j01VOW1|Z=ens54q1}hclv5V z%U}lPj@>q{_|XWRNvD$4#t_5>c2L$WhGJ0M_>{SXO7I|qIP0z&Nq5(G!<HR0e%nRn zG|(E_6|nAk5P<sgQ^#>KS!f?GsDb#qui+>IH=;immwznFe+yDd*t97oEP?q-HTr)% zu3~~!op@qSVCl@|Z&!~_M(~FN_D5s}`n6JDx?Gh9l`<;Nj%#cC*qG7n^bvj+XIIdN zlp3<8B;=_cpN9N}&1Ps3eOmv2X#fALd?6a4)bC;wegT?jrq-md^fsE^^f&#w_c0II zZ*L4YVNqB>s(PVF=vbk2Upw;;n`5R~eLOe;0Lz}*w?*-B&S8|^T(JchDiF4f-J47> zEnMX_@7}<;|F{M2Qy*%Sno=a;z1}3Oj!3zY;MMV&d`fo74Gcbo=O+*4z2-j6+Xl|H z?Obp*)}Q=}-JI-+RYe0&=oCu`V(wt%+iq#j-Xl;I5qBP_jSb6tU((;0rQ^E@8+z*U zfH&I3-b?bYlA`K))a4NP4ppvw7yN1-?^JH@7$)Z>Ack(L5G6kO6u7)p@O5u!E_l(c zNOFTrt`?*D-proZr8Dq!CVWt%k6{q^%LT(WefNm-Y7jrmNn+TJ4YC5`y7YBygi{)0 zD&}*RMDGMgwFiDUv8&`<+w|O9_Anm=4`t(dQe=k{l&2>{j{I^4T8F<?TR14RRETTE zw9g}~3?JnUvaoTpz6KQCNf1>FR!#Qv8GBL1f0YJlT=5YL)pl*ThGrJk#Fo9tU=^=t z%2_N%;}&}S-5xJLwItyx5-4WbxO16r2W6T<&w{YG)AP$1(JjGcC@?PPZgWr4dmyAu z-%Hmt5`&oDZM~NkIU6?Nd?#0%0VLRwVXBv)3PTRNy>$Fh&(IvVUtx>`#1Bs5PmM8E z5^gmO?A~Oo_TOzNH+LWht+l28%TU8f89z(Z=i8{i9PT7x#D}4u8%^CtuBKN&3FVY2 zp1j2LPY<VEUt%LVYquz~hDAThsnOk}p<P>^uYzK?+PvC31ZP|}WzYfRy>n(lC0AXi z0AkJ(^RT=dypoZH`yl-QXI6{{rRD?6LWY-41Jz$zroL845kY#`UoIw@{7|=zuiX|? z>hlRwv7u4Q{aeX(rzf9C&Hong3WF3cerumN#tJKC-EV)r(}5~Te#KCA<g+(R3+(2# z-B%YUU-g&L%UYwR%>4=X{VyFD&M=*?+^MS`p-0Oge4;%p`~$+*mnU2JK#H&a%P<=m z-zeIjK9OCCMt5Yq|A4&U7%&{*8Zk>%L#Z=~{*kRf1>h1;-ogor=1>vh%`7NfO1&$o zU~bQ0P^CtdjGeL5jg~EzbM%`tzriDWfE5t0267?V0ZISdxIjm<u}jTY$DyV6aORo! z>l1}gHNX8uSax`C@4J*t;bWg($maeqcoFx9Q~y?S#+s)7WK_e_m7u^wuF)q_mWV2< zGw2p)agh@*Q}nSb?P?)i`fL?p*~sb}VQ222XIOAN7>*P=uJcTh@!lpFTJWNJ>$ihE zD~CTG_Mc7Qa6dhrYOMYdFs6JwM1?rnA!e1F{ZUvX4DA-PA~0o;d?B?=@Morq2X%H3 zTo+RImT+m}uN|qhe&tRZK5<F0ah<C>Ajl`Ga;<wTo736Da;0VX_*Dc6HhqnT;BLWe z7ILOSrsO)#7tPrtSAAprk3DhvgRm)`V=BxusAM?0#8zqXoJZl^#5QgmxG+$s)-4me z-%DDt>BPe}-WC!Ux_X9PEDmz-pi6C`LE6J2+y5+GUW{#pltfs_&&2(HO{G)}9?QsZ zOo^fKcAJVN@D^UJ^5ye4Jjpg8VVwJX86XzBr7Iah4!ZSSe~c-JadWQz=9L<RTH88> z-So>}<*}K)ec5$#)0n)aJ7%T--`RryXFdu&{W}F)W8!Qy1=u|g^ONy(b`CP{MX84x z@lkV4(%;Aj7qXf2#eoxy7AoTnaKLO^SH^rGD*a_aU-=Y9n$n_i4U0P!@!4HBqsUmF z$tjm>@1gsblY;=hWfE>yEu0KQBTe~oi9Hd-^mM=8rpJ%WCE_iQSns53y3Y(N(#8*M zT56RD5}tm3rZNi|GXb<4UKV}EZ2c1y5MzsHsB@cciJ|G={yfW!hy$nV^Dmt@_Rvh@ z#H8bHSH@$Z{akE@UIv`^v<;sLJ?`;&@9>ah!p4S0GxU|;uCi>=wpwalyg5*Xl_{Bh z>iJh63cytwM8VPzBV3k;%pj91FF)MA^yiPs>$$bQdR~(@WJ#uwn7WF)56F_tRxbzz z=L?)+m@}91M+dETE@L2hX=1sc!mk{G(4X$X6K4Y1_*?*4>rBOH+zQ2+^HW08vhiuH zC%wp|*1hd3z8?5im$brfjHo%|1P2t4vWG(njCQ|YdRhIbxiLynZQkoy2^Bv}biMNG zcPrG=)~wrLQbdJsf86;<1nIx|H3`*Ha~{HntOg#HL$pyND-ywj?;RlF8RG8gJ?xAJ zTe71sF8|r$;Ls=(N8|V7ldMfU3}Nr%lwq$ZN+#e^KFKjYejv2U$05UGwfzOZ-H*{c zn40l0#35TX>6B~+sWbuM>NNoEQFP+aBSPu>aqT_Tu9>w{??Z+~Tjfw(!FpQ$lgoSd zUUlglp^)v~78c_%<yybxP3n+XZ@+T`Gt;%eDy{DE=rQc5#p2d=-qbA6RND4E?)M2E zB*#TsN$WXL=g%xl4L7&5X9|=}0;s;70egl%*WDVG_`rYOogDrd87>PB#-f3BZr2<R z9%{XO6dQ=!GqMBh?co+s)XPU@>JWe}+*LiZ7>cyuuwBq<24o%5^_Nt^AzUpN`FTHv z-L9~Q6d}R|t_XeuNn8CczIqoSYm$zytLfm~+8HjTT38f$DB8?Dv}f1o;7Q2aKmMET zh}s2-;>GK;Y5e_Q>~enP4}RrdR++c<xcwNxmJnc7q8tSB8ETL@XJ`nezR2C%RmiQq zQn9y5viB7b?-v?>(5lF_%BKg=vnS053iV5)8sP&ce@5^JW&W@myt6n=rmF5PxE5%l z-VffbxO{urB{<vJgUVD$aZ>=2(dFL=$*HLj1bb*$U3)YfTx@pw0F{b56D0n(`@XGC zm+-&j$MN|LF`b!3XySp@l7kaPGDRa=db2Id5;-ICbRfAQ5b!z2`g*8FBPyb+CuM7# zXd><QRMDFJLla0Eg?QEt>e(FsGkH&x%1-P06F_^OjUK<mhtD6{mCB}q);QxsID;F7 zL3kOuA?TkR)+vNjX<W%vNd&Lbg+#me@)C{uSg?cXvUUC4dq>p#IRbPju#rv57a6$_ zj|uVtF+JTj^Kn(j#80Lv_={pnfLtuk_v=LmPA@M*Z|=seO&vQH5-b*wdZ`@KA6zh& zb|*wGol;Y{_`l2c{|BWL`utZlmg1~bkh8BD*BYz`ey3x0Q)`_o^C!=FsK2SQ^bN@y zr*N|)iuT7$AnI__dgReT>51t!+7W=v8eHX|bb+PKyUsOnf$!V>^oX?inEPkDy{p-? z3~+Cv2}6H%zrNHeLQe>rTVY=FnH<u$+ni}{XXfP)Ds;VFm00g)CcU&uFT`Zowochq zCoSn*eqV@zZly^2AqYqwz<0OQOs@+LPvT*{2hA2?<BH;5NWRR}Ye{|CM?GK;Qd~}k zu)Bt=5C^fsM)|9fftRRLHUlMZgC#?h8}eFg5(W@>&t5(wQT@=RHA8hNZQH6>Cb?R} zw?pV*1>=v%l6u>94)H{Uh1^H`-oqkK6%!}S!n3SR3c@v&E4otq?REIL4TIFZo(y_5 z5RZz8Ycx04It`ogu{pvX8mT<w?(-jbqSmMM3?@JklzY?Lvf0tYbTTB)IXo)!OO;ua zLV>bNl;U#g<WtCFb68EMe{O38U_;b4iJ4y1U|UpzYOlV(tz&=cRC>3KH~THc+8U15 zMUV;a!G+O7QEk{Hv;qBM#7y}MlU!MxDJ2}$GaQEC=P$X=s*PTm>EJ!fR;`2>8SVqh zjQko?*o*j!@(p1AJztQI{@Cj`wDT@>Xtte<0HV|MG1>!&q57zWBB}#Vy})~axM*{} zGnw3>H(yZcxzHsdtoLQ8VtSU&_?#d8VK2~cWA}@E!MteRu4K)%o^zQExHlVZ_v<p_ zC=k%;I~wwk`P*OBa!Z9UhyTtsmGcssqzSjL4qCT<GXW}M$s*e0@Z%vG`t;&n{b9$? zrUNFSFFI5++CO1R5BO*t`<;l-nM*`z|6SRGA3Y-j&DA_^I_{#!0BLt^09Oc}G7k>4 z!JYyA-cejhCY8;b(wEqFVn`cc^z&+S?$udUraoEutR3aAcvYSkprTj}vF!UP=nMh> zZdeSg>w;kJ41|6t_m{;<Yf2t8xv=lLXv<kDXLz4Oc8mir)|bnGu!VHV=`1dvJhRpA z_TP5jz*CE`G9ynQ^fL?)d));HP7R=w;+B+30h_fol)jNd%m%yiNFFMOW(J;xwgR$| zLb}?<R^NlgIPxY|>Z6{+T+{|&E&&ik_m%fN5*wzwGDrHYVJg}Rj2rClCLawmeSXoh zN?Ct4>r}imUPvQ1Sq4E(HF(9eG93cGNQH0}DtWPFX_-kiIfErYl17C3^xp7Y@a0%$ z(bFKnry#cRsp+K#!C(zpm1!oxT9nV3!NhHvB=(pjDKi29m55>Jcm=ubmmczo_ToEm z6RsU0#yr;Kk3W6f-${Kzn%;TUW6Arb07~(yheMLAqNeq57iOI%T{}epfzDreA1q>~ zV+3n|f37x<)0#E>1G~tYJR`H`*KjKm&>>S+m8E7Yy~DvXUiH`XJzW9SCHxQH^LoOe zA=73&@VpAr`PTrEPM-{7ZMoDp%C{xGx2e?2A@3<F=2f0}zX)&o&H$6{KYRy~@BO)* z`O=(X2v%A%hON}Xv<oR!&emrmzXr*(F%idnCGVwpuR*ldH2mf{?f+*0)qildzwuNa zfZvg2>c{-Ye5D1DCkc6sp<W)}1hSn@uG1YwbfrM#+$*EZm=1UV+k-3Ut^DWoM(LY) z)(@r|Fv#)WOp26-{p2P`Pu;5*!pxL^j>Vu26divwi$R!Rq%wEktp0YlZdN*A)YwmJ zV$;?ux^K|2-DuNz-sMUuZ^3+Tsq4nj+t8(;>3$}&=mV+Fs#E{&Z#|@?E=8^edi_4p znXqipqE1=2+&#LG`(sa-6%j)~+I-^rdQ*k5vQlpP)VcNiQQM1o?UHbsrc|%CXsgJR zE$Zp)P_xArr<>1r-1^0NfU+hVBl0d-!$CGo{AF*ItNW6GL==Z2W)N&f4AF4A8tE45 za{X{xJ@}8((*rektik@PKeCiu=2soGE(^=oh|?IH>_zFfq_rdtQx#akv>(Q=^IQPy z_AN<7wNE<V2J!V~!<6_hztTU8!|b-h75|}1`3MuB3_EJ+4eB-Er}}rvW?jlXB<?Ke zD9CA7-0xL*7u4(7B@a6MwcKh}E$wQFJo{-6{;iSlBVY`#GR{UvoW<XJKDGPKhAn<C z6wCBa73-ME2@1w!*T4;1_HtA?Hh0-ozMi~#dG@56Ja$fReif&&&pF#-*wHgB&;?TP zt*MkdJ8UNHdg5rT5M!+H@3|9{`UCT*Z_VM<8Jm&ZNS|W`Rn5rdVc+_Y<YMOd8J%a- zV(?{pcQ(u0<y3}n#Npyh#x)b=;%b}EnBo@?Lt{}D9P^$mMV5A9IIe6{T7NSA@*AqZ zV73(8Yeyv`xaD4T-({^CNQf!YCkxpH8;HTSLlDx#YmrGcnjz9k6lcHF+RF!HyHXb$ z7=QSiuJ+VDX?l|E836neYG4WKAgLTtx8P&U2`wf}h{*2)(0Rh--}MV$vra#2E08ws zR>pWqT)K9~!T^qd#kp?MXm}Y}XLRQ&NDdVOTM8xCi>mnD;ZnCi_qPw7*U%j*p*}4~ zv=f~xv;+;a1d<-6dRgG76Z$xUfu@~zroNFXKC`4P=c^EHu!qjJW4fAT90R!L>>%^? z>*jNy%o9uyfh4<zJSmzdyG{5B#KqSr!a<KYsT*ijGsGP3vHXpUDFf>spWXfW6g$~u z0ROp_0b2rAr2+9)<UB9%_|3r=8Xy@yPU@j@!9d)ZOC1N$HGVAv_?njgcKaEB%$8gv ziDvUP(V<QLeRYF7aJ#&!9(xgUDXcsQM8eCL0Hx67OJQUZH^vsBgchX@8)1N-C;y}W z1yEHHxv3f-(Zd#ae?Rc(Sru7xz-bszgWJFV-RE7DB-=*G#OOUBAE^}ghCce84y9%a zkUU+wcmJRnCb*+8)KcJzWuS^xg1yf6_#jTgp6Q_qghGTqbk8JZhcZ6aaC_c(PYH>q zXVu8!c&1!1x=1d>L{&WlsPV@$(4x-NCG~yXyhURBUG-OhH>>kAVxC;(fQ6D10z)23 zcOPru*zlQ4C3q<kz79)0sbCvuHjHN!qBt(uI;bKDuCNyPckmh%2r9czVi+G>%I_Hj zFY_XMnPChmG{nCex()dfP}CCqvY=HK7-Cy)njfD2Z|GQxDfntm+U%-Jbz8KZhT|op z`HNq-eY*FR2)79myeyTfRr<LT@BFgI7?K1{$hDD`A6AMm$MeJ>|K~7taATc8>_j0* z!#)a*_Lvb&vG)%5{~mSCSQnyIn6!{PFfb4thV^-&AAneUrt$F{_wrj%{5wma&jkXM zi%y_2db!P@y^*;6p*Tss*5Ya{iM&5iHhfqq>lnJn{04mYSAPWZhnw)74Qc{4HZu`E z#CoV$uv63Ex7|+oK!QY%lA25NN~XV6yu^VYO1)4e{AH6<8J+L2MA^;sV0rX`Yf?p$ z)3SV%N<SWRn&fDirzpL#@Gwnqc?m(*>%DH7-m_b?L%d-x(x4ZY4#c@jvJU`Brt#eM zUj-?PLixJ9-X1%6yWoUho9@+6Hb{p|K^1n9DuCnZ2^r!%2<UYw2sg#H^PU9So7qz% zBQC|U2`W(p!AG8zik|YHy-erAKT?hcNgu3z8&z_aHh&;i)V`A4VBOv`9q+>)aTPmT zcNUU}Ha<2pF0&$TwTHP@n)lD5=1BHncdg@C3GVmH#kxdJ((HilE*2!9ov3q6KFdd| z{q~4O7T!}&@=9JIS}f>_1+uJ;uQ_XCZhI=oyKJ^pfEG-O_2ZBYmCB}X;T83v=h2lr zea)ha2Z1-MF30%#$}r0d94+ca(-KomHEU&8&HDM7+*>!P&1BKE{F+)<r|3d^!-O4! zV;!*B47k+0FX&YatZ?1SIcat@h-=;ORm{NR&|l9{5s*MbXpswWMIFJbU-9r?1%9cJ z%xgj&?S#6E$?Nn@e65&vq}kDqOu<fb{CcdUJQ1lJcot@FAPlK{L(Pkq<><rtPSSHt zok7G;Xnq;QSwSiGPFCPQS^$(nKpU924cJl^Tb{cm{+WDDlJd7E!S+mV^G&4d!cXxV z45rxp`p(A9b}`*MGQol%dt$sez$%?k_d6d){r2Db`&BnJHtq2<FU7}ds>AG9)F{YX zGo?JNI7OF%DvIIWDAS(EtUtBINFta^4J}{9o=lCKW17R!dIwo&fb4+@ug;k*vm1Gm zxi>8e(DS#uwn#ZMoXBlG@H@ys%zLsc0Vid&N|J!75#>Yn0mF^0TFqW$O%ULcirZ}m z^s~O@>6V>Q?T?B}zN0y<C~QN`+|s?5<omFD`|V$tC5`P^>#u-pi1@F1j)%;4<iS9J z>M{^V9Hnm_C_cLlq{H6<BLAF0?OIbb+HDi(^Y{g(!s{neO~j%Ps92=Rw}3RQ*5>T% z>Yr!xUfNioe5gRuuFKa=4rN%J$5m~+Y!-Su_n7+cAbXM!owqg!tFC$k(sGdvoI3H) z_Ltts5b@-zjvXk3(U#8L(`YM0$Q13Zsb2Nr`t8cnP>;?D!PfUVB#-&&U-pL?6DlC_ zV+TVL{v!^aYv`uHKZ`h>yz#i7-xbcMaD`D}Pz;j00^CqxKnf{~YAk~#8%foqkyAS| z{DmXf=`Q<zeU#v8)i@DS>o;l#9II3L*sI2G(xRHC%;X(Qzo+bv%*9JFk{M$?BI3dn zwN8VOLJ|&o5aX=b^kc>Z=%C*Jg;s6*W!9s2vFu6>AMm`vy$wR_aZ0A_l)=JpjXHWn z@H2Ui>4N=bPa`LPRzmve!{jjn#BErzW0vR3O3@RNHzF;lRC-v00nCA=e<@~Jgv`x8 zaf;%#m2pNzHE<&s4s0X?p2Ok(+r^*OlR^}4hPR9GTyzvYR|m!;5b{4A1ltQ(TBrey zjD$LgrD_;LMaF+w_XGkYlCwWm@U7;1k4>QuTHn)Y&Zuk(Hi!>@!i(|v>KDED+~Nae z5at*Eb{SE2uQ=EJDruIUDv5w4V@ey!ZJ0RuL9m*RP6l6Z!(p@}H-~Cp!uG<~{8-tt zDoX12W#&$NRYu0AR|x@oA@|{}@du{0TUGMTV|I-`$2|E+VugI^l}KtXb^2pF7lltG zOw$hF`fBq8kcBhE5(yPab9#R3&o^F4e5_!6eh>cK<H=4mL5>PWIB|+ym27$=NC4;B z9>c`PW~7E>=yjRcNRIX3fZT~|tqHz!YPz4shfO}nMoL@M2vR{hr1pMWq*ZYnI{9rk z6?h~n0Lc*R92zFUTC$7UCdF%JVYAgsZUY5tc5?9%)rhV~4Lu1?iWfW!YKq6FQYU-@ zxcGGyjEEarraBrYuvoT7ytWul`JRN;h<$D&y0uq@yW0P8a8DSqD|_#AVhM{tdz>j^ zo+8jE^V5KV@~_FC$ciKpGEY_~g)=QSmZgPa_1pLI1@@;KpkK2={Z;6EPp0`m;51M0 zve5XT_tp(KDvNaSfS>MH#w>9VyrTAe78+{Jimj?^BFOi`CA=;zjqTD)lcts`BRfq? z#0byC>}`aY5L8+kA{N#Jk%2eXp@d6|o0-wCa#Y)$%9@D;fl5AEYJ#d>t6GbUhH4j? zb^8S)Y`X8F1E_*Z@ydNwk-!?N56Ji}$Fw9PGjFdp%hx_>)XoovF21=|MMT+U1l+kf z*l%2JkA}pHqtCAQ8xIykDgYbhR^pJ<zw}kv+kRe;8*T_JWGMx(K-}N5xk_|Le~wxw zG5S`a36PFZEYaWIj;KMN5EJ}71RRV?2KF}>o6#2=X;d4)b3{SdsjCNNcG{@Wfpt%B zP0)EQ?hrF8()y#m$IB?uH0>9M<ipzwKikEp%+ckNel_tkIN+1Xo3^;IF?da{FCyuT zNF?UnRiv;>;g*rKddaW(S>IBJAhvABE6iJi+<``gLR(7-n=49fJ*S`529mBUCjK)v z?zV0FFK?M;SiQ(mbyr9H9z0x8_+pi$=->%mq1{m8rS-bSFyY&4x10un7=b%!uGM#) z%hwYr{tyPLHBZa=1s++~TmB-!0$E{`A>FSEd}QkMLYVe}{*bs%n*1(tKg?2heQ^>I z6Ks_O0bZ;zVrrjbQd9^;zodtM!ykHW?zj75goB=-77i@^4K7CjitKm&G+v&#t<eX0 zN`P??F3X>i_zv<+LsL41e)QtdDp9j-N50l)wRlSu5qrpR?%~G|<P!}qSHX<lwinkZ zk}xR({{jq9_1gS#89N|}M#-40<t3Sgz3>Y9lBAoYQl6Z0eq3<9x(T+I){GTuIA0mY z<dO!Zm0UKkvX*Be+iH9AWjL%!ZD3-t4tgAOlDE*0S9y9RGmICwPhfvlwq`5X|AOsX zY`n}0=@spf3es=>fv@xe{HHodkI+NRkC#wgCGEleWk#x*sDc4%^$my5`-Pmi3RM16 zj5>M5CyP*&dQWIP!>=~cyjN%|3?dO+CV-vBc~%ly{&L=INxUO}SUIo%+I(Zuxqu!c ze+He)-p$PR&k7|r$gDGMl9U9LxT2$tMb}0?wO=~v*P88<h+-Tn{tc&o4J{*#|78^= z;mW&O8rnK;d#9-nd}mak>%9Jk#prPG$pA6bM#V|B&!iXJTn{;{wQ7s6Ni<}SB`p3O zP_kPfOtH?S9kP~rE)D6HyEx$Xsv!@1OM`x#+p+m3nqW~5BPJ<Av=wl2Ug|x`u++r3 z7;JAW0Y%OFt$kDzYkRa4a-SgKVsU+izEnL7F+xZB=UL{8@W`m{rN~)VpUflN&?<$X z+999M4;2!olikn~6>4B3WJda1&R7gMv4u;8^oVPOB7RRYXcn_Nt}O5zR{+7ro=u)W zI#F!y(Cb+$WX$A^isScFj<vdRTW2kDBv|foi8uB$e6Y{(vluqn;zecq&N<1w{9mt0 z@51&r&qrD!Hz*wip4gLRY_{Bu;S@P@@K9m9vS$5|hxBjaOI-a+TCSUr3_u%rsy3() zNpky6tAd%nTS08cR&8>bkf!FB^NG$6hQ4bM|9)u-SL{I7#kSWSor9a47cd@!E!mXJ zR)#YwH@|umY|8S$y<;NJ{)QeG2TJ_CDfyVUd%2VGX`ZEd;n*#bv&M~>c~im)j~LXy z;E4hZOirBKV;7PLO|ZVWC09LLxOrjxyA6`;AmHr708S-{jL+^5Du>e6hc#M3$69`? zDRxVzz{~m!4aZ0Gl`E|(M%=v{&~+;`w-4xNXG~I^LRR;iBd*Kj&lsAne$fbBhErLb z)O%pM=*`;zgZWtR+eRR;Q=Coe5S=dT8%4<~qv^m!7*Z;;*yN@Q`bCZ1h+Ns;ju8n; zUhoC%=AooC+<s~_GTyr+Y7lqOiOg>K*=o*27V6@!ngq(D7510;n|<2aBdpe{0&3X$ zr;&8gJ1p7Y{4@~P5spx#gurj4ODbm_LokOx-ot*T9r6rX`$x-Xj$f<bMf%boKm}|D zXxmwZVnk=n)yQ7}w|cye<h{)~ghJSkU<*T#q&>Cf)s<IWaPHzesONGdy@VaS=R>u$ z2>+(mU}_g(Z+as<wnE`*#$ze8uvyBlO<c8sy}ZWkRbWNUXZ%gx7J+lVwS$J8g3;Qm zr(4zVdiS^)a?j7qO5HbWwk_*M<D?vt8_>rLafV^?!-XvLDe2P0A8vLtF<Xs@4%YJ~ z5?@6rHdTcq=k<|;NpIo$)idKPl^(9+s+awT&lwemnptj6ZS6OLBA`+Fy5;HNKMi=m z%NcxgMn3y2kc8`wT6v~!IW8Jyz%~LLhd|g!*&db4)faT~?mF&&7tKC=sS76fc*x3k zrET_QgyNtby=jEjugU(Bd3YZfI*Ed3`PR`N1>Ipv{}fh6B+0Q0m2nbU<akPBgx#Xe z7GH#-f9%OrLW~8=dQOroLeGvAz^s!oeU&bcof0L|O6YG6%`#}kBxv|CUB|%u3H38W zZ3K0;(<316x9vB}Lm8f_WlooVJD)HnBodhFJ(r~I=U|`0SV9;7Zd6gY!ujmb@-xb8 zYI4S^ryU*lK%`x!6c*}{?EI}m>7o@kp6F$0eCDylR%0?N%lF^!TlMk>4Nza4hz|5i zoSg?SgM@+)X+NZ~fSwJo#GM*Ze;_P=X6(1CX(;+o&q@|>(u&>$|IL<BSLP7hDff0z z`9NhHM12g_h#tpsZn`<QL`<5KLROmzT#Ruva5ucq1S8qc42Zzi+pWa;NsJ6!)LU`y z;fq<6yzVq;H^*abw%2V`b|meii-%t-68fIK8zHkCU3`zItH~d0W|8{kLy}M44u_Xu zImGo)g?NzTmV&m{?3(#1w;H7<Y^xZy7JZ7nf?SEo>7L2^g`tz&5MviXOXTe+z4dK% zeSr<A8)pUMWeaXVTQxAiOX9Qy;LaCiwyhBek9d0zm5zZ_*XqUImR{+Sfk4Kj2XGWs z_{t8`IqPikku*d`{TYm|?(CJ`l}0223T5YTYDr3P`o7_2V8&)deQf$$ZN-vx=f)?e zmaq3d)nDw(2X_|^ktc$zp4UfzBrdd=k!dq;@YPt59JlQB{pt_6OpsL~Z~sBmglP8< zRe{z_5(s3wpRiv^6Sdo0t+o9-y1|kaHFpr&%a`7?M0W~&Qh|LkzGqkU(OCgr#8kpR zQ@&PbFm4mt%%s!JGyI+9lz(-U>VIBkcJ@?ACx~hzD|!#-rmvP2Xz{Vdm!;KDM<}bf z0q6DO?YLsWR=M4ZF1lgpW_tE&WxwiyThbQ2D18o9SNGtNAwZDA%5FW}8T+FJTr%Dd z&C4KLWOgfh*oFazQPwK#pvK&(+PQ_-C-2vnqLwkoojC{`v}qD3P9C`S5u41jSOGAW zZM+5FzK8F|3k#<ea}vEt!Py)1g_1ULGQnA5XgC&hHo#@kxFTQHI}2)tD3xgaUU7Nv zmNn(E)S%~cK+3|mrS83FAd$(BX%~rgnIHY7rU92??E+8&nsQk6FNk(32M|e+17g@5 zBck={&xjB+*|!mtUZ};z#BU`(b(obmKxM>OuE=cwA$nm3iHF!KclSH#m|8RuL;*fX zu91RS>DNRNK=uqC`_#SI=Uo3ADD+*)hLEq9;V~c-u<^J1d8Y`kSJzBH%l~5T-XfkA zRIpMf@M_)cAf`rl8QPbcgfE!Apj!pe*1|HhM^aNW^v%*TR~1q~U{8d|@ZMh_PMArb zM27VPJh9h18kZFX7W`ZBh1Z)(j@@SCZ}8`uW?C+i?azAPiG5y;zlZ7-d)^L*Z~#3F zq+6CaPXT|pIlh|Q`b&JIvvZ66?6O_cz#G4>cF#}Z)2kJEaBh?!mT*KfaqwC-KlE<& zro)T8KX!XGe~yzOhtpOzM%60?C&84jmJJ*eNGG{Izp|J3wVNoRyKwBsZmL1d*7_F( zvVm{nsDmCh<`obm)B$(XH!oC*%S%7L+T)sNkP>AJQ0UDVBOD(<b-Z*K)t4<HMz0(# z6Jg=NR5k7$()SEIotjn%s-_M1qq80j?Ri?Q(XXUww0tX#<q6aaN7!5^X7;zhmdyp> zd{n|&o;u75@u6w+R7h-GP{*}iyjt&V$u-qEAr)M+<<U+Il7|gDU6k_zYk*gmTSE-* z+T?Uum}IwM^D3lB)wu$)k&?t@a}RnO{#`AVZyd3E6V<s0SLC`bqXdqyW-)tt0>#g^ zdZ^FG2Bq(T^ziY&;h#}|(>~Kuqp9R51Dxx<$J!0yK%4XNTxL*Ng@U0MO%L6L1}Xzp zb~9x<`Y=Y{*|fr{@T7e_P~|I{>Z2C$DHYi74q4ZwHb&gMz!|QR@w-KXD4`h?I~hca z`Lx{+P8Cbj31xE54A&H<E(CT+yams04*In_4(kB>-%M}zWi@{3f#WB<mjP9x#$MfX zYD3?0etKTEi^pC->hc6`dt{zzT>09Q7b_<vUSp*v_agUIpqs2yKQC7pS^;klC8g9- zNIrRMKVKF3?#~6E@;&V!JVmp&BUd+@zvN5DcrvR|iokmujIi06v`QCNeUk38KwAsQ zl@b-P_Up~W+`ZGCfrG_4u`siRA1*;lprFZ`$Fcr6mwMnRS_Zs3k?(1E0=~!l4*`+` zQl4+_czjNwyH$_iu^)cNq8F2CW^uWb?SWsNKOh4+q!w3kM{L@Ri<>gb(40LycKy^d zm&ngxYrfy1>ViwW17E2<U5TJYb*^bFXK>tJ9R@g`*oep(UiMqOJy>J~POlPP!AJIO z9xwDvN#hEX))rx2OMkjWY4KYR?V|UeR5Y-D=FYXz2mY`IR(m1kUfoM>@Q(i^c^{C7 z_xC0K@_+LZyWIJuWR1c|IFA4LbhkmG%MSIS2=w<&M&4P;nw4fRXW<fTQ(gC*%k}ZA zl@|=@aVbr`o(^D?J@YJ`aI4h9lbVg`yN9O^{teLWDu(lNfFr3Y>-n`9yZHSlQa~2- zR?3t^=vYNXQj0cw*jKJNJ9utoqARbL(q%|N3DK^HMHG94!O7&V_5{V;@DlW<!8Ur= z_YUYW4w1V4YY~uJ3-(g?Q8MB$=Al=HQV+Vj&AK%OvB(<1)w~G_;@dxMcPRRD6db(b z%RV-xzhQ{%>>=>aAGZwkFI?RWjT4h^xF&#j@$>tH@&t}qJSYRcl9%EjZ(8oacOxhG zgTFv+nk9J@dp60(>C380rKw)b?fk0czBKhEJ(;%H=>|yW!PbI$LEKtO5(mRTs){GY z=NWAWfnEiBL%)kF9+<xz71Cjrjbw*Y@xuTQS;)qov6IXY!%$Tw6vw2_w@DP;`y-a{ zzLQ713L@K2UVI5EeNb|m)WdkhrNC`Kc~Ei1bzcWrv1w#*_%D8W1LR1!`byjQcHNTg z4We2F_F_C_diW|wMr>(9^nmMEawYMnvoWdeYQ(AWps_+yP4Yu3#ss6@qik=rK|>bb zJwt=ibo(l}>#B$5F)X0;13<Xnj&i3+r{YS9L7NrCxBOI`ADxgV9;699XXaR-zB3|% zSLe{vsSHHIq0xA8VEu=!L@h0CzO2r`b<8JQSWn`<nmD?!()`(spl2ht34t7{RMZVp zeCiDtLv`OX3ge5oDva#s_mk~9h~`parFcGbd|h7Be)KZ@`dKKVMxr8~<yvf*zkrRB zkiVr~k<3y3aT%*FN7JKW*FgiX+tag=sz>gS4jhy{#g-Vz{;8M%TYO?=0VFzKbS=F! zf>Tu%Z>A_9P+Z}X+-VmWGKT!jtrhmSI|cCBfp4$JGH>oD8o9S8N~f(z-XlPzkTpcp zVf;2n_`wR4+tuT*b(#v|r_F6{#%a%gsC66w%>x?$hp_JqWc&Zxj;&&L*o5lP8Zl}U ztLU%VstzLv6|*slsy$LgZ5^mls;br6dj&!55tP^xdrPdu3QzjGpXYgX|L^;i7vAwX z-*e7&o$I=MsfnU^sx~&_FaAhHOs@KJ$ZJTLQfGXd)!dW_>i&VE5t!$1q6$M=P@lLa z>l|Jl@E#|ndw=R4)aLrXg>FoB=a^rOnzlo%9d(Q<$0prKZOMIwta5yc?l14VQ)|%J zX|qMK&*d{WPv3Ztc7(gG57ItgvV7e7Jye5wu&wylE?p(LMF%KFFHu-=zJMC(S@IWf z`ITS9!kcqH&X;leCa0Y9$v%BZvJH6ZGn{842iYVvnzzN2Xa1DAHrmKjqeS?1V*Xy= zIe*Nr?RRI!=i%f3if<o*U?&y0f`2Sn{&$9256OS=Ewb~&btAWm*G<cRZ=Z{rV;{A> z{1qD5Jd{42+uif|sTTixSswbO*2FaPNZn`ZF-{w8mZ#?x>_LX%DFbdUA#wI5k2Sh> zm7=IVCWsSTKx|T_1j*Brs*+(by&B`n3Ml(tv#8`7((6MMW}_vdhti7wIQ|s57Q&Sg z`0n_H_`0`rMc*bzvsLy$_v8{E=Fw7?;+AYyl1Frc_g(9S25w0QXAy0m?uU2C0VkNW z5Mj`8w|^yvwoa)TeATEVmI>COTEjLKqF&dvb0e*Xj;W@p&F22|4po=tNuR<q`VHUp zJRk&%mQOhI<6s}qy5JvqBROiKywZ?nM%`;%Amrd^*2>-Pn4jLxUNbpscbVl)p>MMO z1fsoS7H7!J;o63Ik+E79_-EqGw_;(NadH-rwfvB&mfeFhbB(2~ZEU9X0*BwV$Ew33 z6`4+vnGbP=k|k*gRJS<#=t>H6mQTepbDsh*unnV2beBxt)yQE0As{z9-VokZcvDS! z&ixD<iA=<Ge|mnHRe3i1P?Zc6PN_V|0)jrcd*DB{60!aw%`e!*t1D7S2x2L>rY&Cz zoLE{x<y<`3+W?k7L5G~051UmN-gJ8&m<fw$d6+^7X2&9{t#WQXhhlK?Hkkqg0lHeB zd9v5$t!byCp$VOR)BS1V9AShHGm~)lAmYRH<UgVuw`&i=B_`OV=alXSEvSvriv*A- zv2|JH?un6B-&$IS98t&j*TO8<H3IYs&(5#42cVk7b~k`<@<-NA*>Gsn=mV}tqHk`B zMA3sEYYVOcn`>??ma;NwJVvGsSFQJ&70`s$&ZEl#@CM-n@8^JwpL|-iY*Z~4iqmP! z?9Zp==Y@S5FrMJ74|?1^xd*|F!_E<G?2jW{61_CMQMEMQ=;E|OSW_NL;v;L+M7XVD zWK+W_h68*cJqu6ZNDLaT{^eE7jw^m-Fron7MYEr@?)`wbqetCQG+wW$=gr`TTQ66J z+G+)Cs-Geg{BRoU_atH8|Ee11I6WkftGInu%0GURIBwy<be19~Nd49l^3~Uux&qYg z?LrPAbR9G4H>QUA2B_fK#?W=on%jz@u_M<U?4L>virBGFet+frT&pu$<HEIjPJ(mz z2F1TRI;vbHn|ewd7bdcl6%y<|cq3FA^JTs7Ua!UMe7KD_9Hd&XcqZG|0JplN(+;t< z@ExJafj1F#4wSG4L%$wmEzG~c%dT1(;Ar`h97hh!0Cpi}H+;DN*V7YRA=Lup)NMZt zW`Ute%tuYBG@v_#ZNd^#1AjsTx}XB^nFkM+p2C*3LzGz~$W6N5xxAq+%}TNdI)DD5 z<rC&rV<ejGhP4{eZ{mCKf?u{9r#8X|b>_vJVpdI+0ymJk-{M5yj_gw9^R(M36P3Nr zc3}(;a#0sSbi<Okck}lLv|W0AgOVx2(-q38Lp+<XbB*EHq*X2m={abqOQ1gYRCoY< zi}dp}hX$Q_x_fxr=IUYka9f%IcVLDZLD}WwBLhhZCuY`noGHo9vI}PuQ<Zzwqz>lv zCw&m>j!c7gIinM8*IyeWC8#RQEa43?C^FELo<pXE+(OjheO_BxU$tqSXDKNhfqqcH z6ZDSVJ3DLj3jiT)=&ShM-idIzToj%@)KNf3rOIs05Q%SibxXr1wLE%hnM*kY`SpYB zuRTN=^aKjN4*khG14qX4|Gx7D;Zzhk(Z&mJZYuQ^<X21GOiE~AFnWIadQp7<HF73M z4K`yb1edDXO)jYMPxTQNOD}Y2?=BAbU)I)V-wL2kI44^3xf)Go6L*g*>0%eg+VJp+ zR*OilFH&-Uiif3nAWOUPCrKDX35(L$RJm6Jyh|+|Lre=N?}Ee9yyA4=TDvhIG@5c} zOS@vMBg><<*jFCwMWIH(O;oW%qtXtJhbcw%^!_d--fBFH{)ue-VLy}D+Ki8+7t%9# z0$m=FVxCl2QM1ts;h5HS<!_V0_xrd3<7(&s5FtFFmX{dCm>G_ffV<R<`#jY@;Q7GP zN6F~(-&%K6Hlg-c9&T=5Dx$MDj?<2%x?fS-xayZ_%aHuFT~AA8t!SLzmwQ`kDqaRH z)+)iDp6d#ES@rYv4y}bRmAW`>p!lEcTk)S|6l{v(wvX7dsH(N0^I?ycXh%8sjc#Nm zZq^PyWh?4s;Q6OsHBhrU{#WCNb0DQU7}G|Pef)!{NOeYE%F`ot>t)Zt?3)LMq;I`^ z<4h3l(*5L=B-w5t$b#%=3lk!iy5JDixfWW-=9fx#UC58Ne$tCn7kacpzZg@;ZnlNQ zYr&?u(l-eV2&vL5Z*Cq@1ri*s!T}pxAN{i#mc!)}CIuBI@z)1_?mjr=3o5;IuIzkX zI$cG#WL<wjU0lEaRv$9Q4^;cB7IIZ#eL#gxX9=w^{s9s5W{xYOU7{M&1N!7v{yb0; z;CXisO#a`?^iLQqw-d4i0J|!-vnO-+Pk+P{K<!%0Pq~<PJi=G?3V1#|M&ARY+YOMq z&kj^h1rd+wOl{F&3Oi#nZW3GHf<M!O3=^-=hdQe|i#)k!#7=YP({CV%7{8l>ffEz= z1Yik1lD2?{PF0>w&9xtB2c7XRwgN>38U`**7t@G9lW*PcxmjmEpRDE}_-Wgr^%t!X z0&mUoHV^41C>+%=44zC$Jyuu!4#@CiA2!FXyH7FJKJc@-3WKEwm^n%=?W$fWSh&$M z&wxz2g_SjvA65)r=G%RU`c1G6{!y5EU@orq^^&H7&D+sS(e)~)KkTU&RpG#*(mU`e zkCVr@t{guKRVp-st*GB(^4e%Ws^}#_N$}H5=Hyn5E|>B?#sd^`lzOG_ev-xhu$fS; z;d}gwGoRs|Coai@t!bXmxbXam<^Q_{s2mf%d0}^%Gf*;H-8J4E@zw{#ZK12dG<)Y$ z4Bg~{JQjIb@L`)LH0x(PCw;FxXOD(RpHv^?Io0R8y6sQFCj#jgxT%36iB<g;7;!=D zBBdm7fIk3+g+8ufgJNOb8~jinoyHz`Hn&&_jNNky-JyQ^LK6?+g+5kzE4wJd<mVf) z#4$0}<vGRf2DZi>G{S_rMJmOl4$c<6kX5Gm()aD3A%93_)R>_%&j&kYLp%jx4{JC& z*f|EUO=H;?&WDcJW-6W)Qg}jM{jqg$+FUHnteCfvINf3!)vYUO3VNl<`IqZxH#q+Y zd%{on!`mz1j*`96%r!3393Zs?&FTtEsG!r<*>{#5nGF7%nlh+a!P;4y=6{SPP?z%0 zziI(?sU!X7KJ$v~uq6lGUS!l7;Q2r>2TIp}ph2+i?hWwzJHP|>f!0X_&cP3U)l_f0 z<42eD7x{|40G&69X7N}o3QtsDBUI+IprAZ~S+uuyYPU9XzNfwsdQ8RS#aF|{^J%{g z|F}7S9Zqw)G4c^(9sO*kY}jpU%5bxs8}X~ZK{c@HuF%t^W-8dTTREZTzTi)tbG$Y? z63B(->hVt)LP`l^%TpAViI!#UX!hD;ZbTnxL(_9|A<Z-uJTay>H&Vbdmg~}>K|b|g zj+-t0aXR`c-@DATqG+nO`iU$P<Xll9t&`%Cjh+ltC$Bg2FE$W-*kjFT&quYcl`J1) zlEO>Co|etn`bhyxmE=n+p2VAWANXx84!GWJTBjmp7{Q2Z%qod0HO^XHkt;ElSs&+b zIqP^2K)!4j@PmM%f^ox(5$3mPGJrLiDgW~cyG^_CU*@gf?EZbj$24|_kNl!tZ-7G( z`%4a@OWyw|qv1`AAh?PVN$j*Nkkz-uMLRuGBT(tYFKc(wtr^!bN3h>!q3N9HnH2V& z_|O&Y=(TJ2aMy10EBg-VhQ+1e4te2ryy!0pCVgO18$liJ+}t6HYbJC#oR&55RI!g| zvJ82A(xLUuuDwY2Lw#48VQgecIVgF;v8buykNg%ExCb>+yC2D`e!g5Ap?T?bK|ud+ zm#V;IIyK=j_I0Z5L$E$Eg9*ZcG@YRX+%mhN7;-pq)Ad0r5IPB?<Pr>tJvmtgVI9su zlMlOE?LtR07j`ShR*_Bi=NVis$dgG;Tkj*zeAh~jw6i(_wUS3|k&djtwAAhBtecoG zzyBg@Pd8k4c^AW}CA_<^qL{~`Kq39WA=ZobFlOAw=USkrQphVPaPt1s_RDO7m5_P3 zShH?pLUzMLCm56{e3*S*_^#%c8rwcZ<M%(=Y=IQ2bkJ4v_p*p{zcy-M!lf?=sK4ZO zlm6@xZu_jXR`7L5C7q~~pzh}Gz24!F1ROLV*YbdOGFGUQD=e$aO-jF$lAolZSar=g zSR|<PgUW;|TC5`Z#aZQsQFy{(V^;BC+S!iE)#Z|2TNitdrO>iBr+z1M!B<8f{PbDj zd$7wO1;0gJesVB&fqWG$wLdg>iEIwhJdk(mbUXcAWtu~wE2JscI93oubs$~an4e#z za6y(Z#})k%SnqSv@I@8EGsDoCXw8Y`r(~h0>^1oUkapifp}UU(3l|eU>67(o7H-P$ zG9f*mWeR@EWB<aqZ$y)&FmRn=>g#qq@^-Dvh5hS{mm@-RKb}58QhmWKokr;_+m=kI zsJQnsZMdL~;b%4PV@=Wx2VIst!6EA-vo*6fHum67%S#zQPCef8EL*K5<v0rwWibbt zw5Ra1M@Q-k`1$`T**Tb70UmMbRE>LHH#o14iy+v$ZR%_|(mOpBF;#hf9W8McQMzH! ztzLo+CO{pgm(7dq&5cqAKFzAmR$5>`43Pd7uC!?BF`_H<UL2Lu%H4TPr&HAW9h2pT z)Z2c}@X$JF|LNa}`MzKM%T?Oy8Gi{$Dw+OR#LNVuDBVm@C{bmqsuHo|__B^n^qO_r z2(-e3vAKsyKN5zDSC>DQd5YA!Q3Lc?<1?Sm!)X#F^pYR>UVCU`^7&Z4c_2ypzfTPQ zuY+x;$J9bCuaf@;=u{y}|IA00{7m&wMlB%03!O`2;2tyZ)NWO06?MZ6^z-vQ%ib(w znj<)@eogsMcsjJKzHZL?=4<d!5I?h?Of-W|XG+wK$D$J#!Z<26Qg+&2B%C>k=&?dX z8>qcl-19|K^RX`u&M5>oT(q^fg(u*CyskNKw*co(Gvr2{7w{TqlDc3i3GJzRd>p%P zF~3!c^FNnKzPb=n=E?@xwv$@+lHA8L6`swgJicBdIe9PK)N9UN9G>OF?k|7U(;7R+ zKvn{yV3#0OwpAVE%F3DU0{-Y}=*eyYA62=MC%)x}_K16i-l4+1MKn9UYXj{ci<>eP zdM4!kI<#IF|9k~M#%jy+JH?WBgRX2E9H>3Z6J`1(9XuPY__HFoi#3+}j(~u5>bXBa zP!A=yiaVU$I}whag=5k#M(b8w&mn2>y}VjuFlT|Nn<^g&8v9O<w9i-yGO#!_+X;Hu z88N0>-1n37q^Egx49fl5{U0W_j~^g#4K*jq5``^Jqsd5soQQkGnp=85JWFNZF6R6g zj5)L~JfZfLpvWSiV3bg~(UWa<lQLTWW0vfU|HY`Y3vnWTKe?xNF_ge`ctG_JTd{7L z-{DI@^q$>E@U)eU?S@wI%$`bgpwl!18;LdF;<97^$i<tlhgY5rbw>soOg4Rh-3oE) zJv@4Q3{?LZR%}EyN-f<?lzju+mbr+aTt4=Q%kAy@mcD;XBh`KvM{a-CNZ#2Rsi@1h zgC!$^x?w=q!QdZH01H^-wB%v$=s$xE+JMwBr=I|;X<{kQwO@33+8&S4j=-92+JsxY z^!PiWl1wXIG+b~%VzuT|5%nSs`>%Vn-Z#)S3g(^_@gsV;fMX=ti|t<?%i4j4ef%Ml z?;U;*kfBh_e|6d;>HRReJ>L39#e(;}mQWQjnCxN6k{j#cid8@7UB@5nN^)Z=WPi$q z{3`JCY-|e+aSKRlW%1RR5Zvxh7Yt#G0~ONUkd>z{9l@_R=P)yrh$Y<{yP@67I)rSM z3l_zdhuT#Eqk`k3Q!2edV}q}dE{}LFQcj^Xb}pBdc3tazv1zrRAo;X`57^QhtCa&~ zmzM?_0(|~@AgFNu1>&cuK@}w{Ut_Qwq6k0<6H7Aou9lkp3F?IpyP#ADTS$K&B-Wlw z=UQThoXh<3%to&zk7T6xOUs<av!6Vm>yj~iW%+w?lUOL`-^QD<7Q1+wdY|U7XLOa~ zlPI5lnydLgF;#0%MCdQWBc7fJgV`aEjtzwU?8JbGS_gFo6u^@mW%!(GQDsOR1?IeC z^9J<XAj*bpPXgKhA<U-jro1~0G9kg)c)zEN!bOCVD&-<Qco|$vZ5)8=QLE3iUyhjw zrQU-R6{_ZGLJP+GJL1B?z99t!eqSyL_+gwwR<(3r2=hzkzFg|Y8&6|k`Qs*07TenF zM|+0S`tru0)0%lAI?jsKruoR$3%Azf*zW`<jKCz)>56wYTtDuEzo%P@tvIqzw7o3H z)(xu^w<&|)5*Ts|YOc=uMTfyC#Na-r!{yRC9lW{ZE;uRa%SWG$f&M<9@v?JxZQrWV zYxPPt(u7OVS(&PyJw1o(snyv*k;9gBaSGymn}qUXq$~9AMKA9#`Ir9Vyz4Dbuv3ZD z00Rx@_{3=w&>>12gjSCBZVOE8@F=iI56RXQvO|sH=gllD@ESXdW3VUvx<6BgVRXbh z*5@SehRh!Amd>RaI+<hp;BUN(W2t*HoLJ2BpqvD@1oK^Y{=AI2vg3Ed9@0_P4qf+U z2^8GXEDSmavcQC)vc3VeKTLc=lny}YYOH<n&UM%I0-QCU-^}(a+%!|^<7UI_X*jvq zdYP5$3Qt{GeqW(WyC1muk=muc_EHRX!w1xd<{I~`YU8t|)Fay3Lva{|ffHCvkQoe8 zZ@zws{LD|BA%Rkcvb*MNbwyxN?QH;Trhf3r3ObUU70mP(^ndlI`*<dKZ+40j?-(?) z)Zo1m<=Av(^yuQ`&)`j;yMKlJ9qYF<o74Eop8z|0Je68CHvpV`;_@m>Xmu5tcZ^NH zMFj5;f2@+iw*%d_NrNWmG8d@=VG&za>@MjC%bQ5(9qQU^%_8G=|C?YT+<TBb6O5+_ zzP_KT=6%@oMpReviI97db<)O<?9$zMgjLWiC&7mXgrT}>aO7$g1a4&)+k%Lm=+L-h z6u$-e`$vwpja3@RgTc6?mU6cr$v?_|&i8p2(7rL^Aj&3~-zeo)z>i0Jk<~p3A8f-c z0~)+$&&tV)=sMWb&&wCC=Y>A`BbURdFTP5fo}8V2Wh`5tDd5d8r%bk>YXmRty@NC$ zv#HF}RK|}ST^C88h=Y8$F26T>$#f?zfmi3l&kFj)glonu`m%4^`MTaRu-=Z90;NK! zrz;B~r;ddUqZ7v}<QcCm|9o-J8S;2QF$G7xdc={paInH<|8;L5B)&+68lTQFIDsZ@ z<d$8(X;V$|3AF<3ta3n$>UQ4S*N#%7b@{aH>g4)EB`)25W}IKD;BuLH|3g;5z|Nnu zo$<b+f37W5Q@zo9sLXnqridN#`Qp{1o_wZuedHpSGIXLEZM_{fm~wPUr?Ua_+`^?N z*2vcCx#2K-KH;P5(Gaf2`IwkGHFUj6H-ovDHqq#faS^2IA|7Pj(CU_Jd*dO?N7`Ji z|LzC+)cA|m0!7{gVa87tcC;?NBgs>7$fFvHaA5O(!8s6_ee|pA4o+j>%7mlf^+O-t zL-IL4^QNvUc6YLkBzEGL;g@eSy!dCf{We(IE>4l(_bi4+S*eKH$A;5&Uti8U#p>%k zeU>*jp?EzEPTjit<R+BobzqtvTe;J7?%VwZtInC9=~!Rr$@xM(^qb$YzHy8Tc%r0E zPq1fqKQsagua4U|E1nN9C?iJ~o5MjuKKTVnd@`!!-@@UJUls50F|y_0tNRszby4vG zwDKA<U1^hIu3j44>VL{X>oyOc%m=<a3-w@n_Tw$v6zR^AFJFkcsqEw8J#fI*GrJi- z)n8yT9)VydEgAMCNcM@1C)jK5yjlKSZl0fYMg&%+CmKdC)HAenNb%m&ocFL7?_C$e z*!K!fSQx!<4+DH<V7$vT$QW<W+O(Saz4F>Vetss<R*QT2J^R~BaNZqV+-|5&G2v0l z^LP=lqYTHeq^qexNtAMivwlmw&8mHK?}~g~vhV3owByWJK;7(dGNV(|R<;G=tIr`v z`{U0SrZ|8$3zR3|<N$;gJtX<GQtk)Y)|umAzuQYC-k9&eY$9=Np#Ej^(7PtJZ4^A@ zx9UNeIAe;&TOWEz)d{BF=AMEnE-s$tNL2@of>HmW^HZhL#wG2nA?_axRbYN?ZRz{B zVZMyUkPnrQDZ0y*l?2c~g3S=Yad$Jty`}2P<wWv}88X(fNQ@uXP^AEi$tnrSNCcEI z><SYAWvrJuuzZ?3Vm(irPI}-(4@TIYI)eThlrE1eK+ZJ)egFLT@lKnhsMmM$#3rtY z*j~(gj+_`g?WN7gp{8M*U(B?{2@Fo`TGKx)H@7o*2I)CnsqxU{jBr7|ET50t6uxy} zcEla-V;sqKUxnSAhp)oK<B*lI9IxhSZX=Jbyd>|qJj0Kt^q$PtB*kg_&W6V7eLq>T zXGq4(S|-GIy=|(pEk3ek#3#uya~Y1W0OgZn+#M!2SI5f3<j?d)XU2g%z6}ZS)g;F_ zl^@=0cCj-wB0a(TI7)pXkY_FF5t=?RWFA$^T5L<T@GW*oz0~Q+IAr<fyVy+}F>9;d zlDJ{*@?4s@{WsxTUCS1T{QCHd(ps$f*PPU-!RHl*+61SiIj1b_>%_J#1)G=Kx$ZNK z3fGLAS{i=^50-6QMPPHiL44(gA?sc}SKX&x$CsaRrm%=~jJGm~fE?)_EeU!vuxLTn z-kXN~_XYg7ss$AoSZ+g}KLY)UxE%R(@kJg=F7nZ$9|x@Or7t%mxvFgZeQlpGF`M%2 zz-$JCPJnWN+7}JWsLqSa?)95hLY`IBS38Rr51jhbO6%O#Da^W}{qm6_)t%u}j&j54 zu5wuxsXL%CZMB~1vi?#m6B5c<Y6@!Y*l3j^PK8y;dxZvxJi^kRejRT#W&$>@9u_Rh z!<LBLoSnowV1&SgtFt1lwm;DryoEuZnwWFkLyR{A!hgrCj7250CafHdr0)ur%WpHz zhwB)cIA|CCNe7xt_3lltg<NyU3t|oFI&P@c>G216+zw%s-kyujB<vhaA4OrdXcG&) zt9Uf`ci&PhdBX6<BG<jf!|g-d3tklG@nqp=Zm*r73zgg)S?vMlOAF?a{WG_wUnh9< zX=m-AAzOyPD;wJ5+=&q+(%i3VMq71v>NTvrbCzg(vh}PU_l}_B7ro7GPhp0n5fhFt z`+)-B&+i;xVj{F;NBd-H!TF^kkm1*(u-+QP_{>nRMQe=)@YOE<pQusw%)*+DxFEax z+h6?lt=<fX4K!BWAgw{g7u65O>H=SrFEl80L0d;o@7yyu`%sg0csgFcg5FWvv!VoY zLzW5#AAvB`uaT6dF0qOMWPr8@&Q)S#-g4|hB9=J_Z88!2ST;lOzC9@*HfSQ^xf)A< zKInFtL%vl_(P?^hgR|dU$nilO;jT5YTq*ddudhe?i76yz+EB20SGvy8_KeU+=*+ya z6U4EeTKajE0|C1f8hAT5^MYSkz%85-%|Z_t_u8Wj{Dwzf<WiM9;F}8_iiKj2<agDi zlQSRKx>FBj+>fSK^d$Ag&_2zfq=pmqv!_{Qwhlcj65iFK0m0WKR>cER*M_6C;J|i^ z`L8r17xq)gUk6Z-bq$cvs5x#ixDzK<%AwGAz((64HgL#)WSAv}-j_V+^m-8&bKPN$ zx8YXvPHF+pW>V~QsV{2<HOtG%7b2|bg1F^H?hK|mE>bL{TCy$`R`l#vdUL&<7@rWh z-5x&~`?1+09Z6j?v&P)1@1wY0x=eLc{u^2vU|yEn9cSee?77~>Hz3dQy-J@Udqh(5 z`AGfu`vz-g!^>yQI~zQfWmCb&n{Gx_uZm+ScmF@>$)FM5s8nA^xud%=CfsvQ4s%xP ztCecVDb`zpG|2{dr&14BGoItC`!P+9TDJWIpKq)?7ZJqnLC%*pzS{I$6)1*EAMSIw z-Q@`l?9z{<5G{H;Uznyo0$dUY)ShHt6lb`LeDvkN`4LvEh3sSsh!q3u<k2GNWCPLa zwx+;O72WgMb}A)PVV)K2z`D(5=|a+?yhdzKVnXtKNaQE#okBs<CBq4XkWELh2s(N| zm++#DY0z1SFA;I^+sEKNfCy$;y=idSf+=5vS)3))Spb`Tc#qpgQ0?ajd7a879WVlJ zS5!uNci0L^0O+(kI*2@2=M3a{qO0LRP1MxX(V5V{&6I!k&T}Y&VOp>KL3bIhx23~) z026J#ku!Lj;FJ=cprO&=n?$36&1;3G=OBJCagwpD;H|FX!P5CpU{`7yDr!MDGwMoc z55OI!zHrDX)G;m@%6nYs9r<Bywe5P1lHVFIv+-ni`Q+f$D5n-R15Jhc?Ftf_|BB*S zg##g5hi|BY?8wU=BNlYIR=tcHG_ZyS13O4I+b_POG$xRI9*R3~gQjh5w1zyW5xJ$r z<-doOo5HR8zYkAnP>7wsgZONUdyYic{8&B&(I{!0xgHjhvQxMk7Xw({8(LxYv=Hcy zLc=S=B<xA$oax@NBRymbS<GGYi{OE~)#$&@(4j(q81QPvcOLcUAa|F}o9fOb9n;vI zvTk-<iG?*KA<gD4A7}qQCujiU*!CNYcxKOGS&fH}yyDJqr1%F1tV}ISdu~Ue7GiVl zRJo7875f%JXYJxX_?FVIbvN-;^t(d6suh9{_>SFQ7SC-q<ly)!PN>>k#Qn<iGzR3? z#%uwY&o4)*iSi=>PMes*ctWIZaXr8e$yqefo%zvUZ??@OS4@VB4rTsDf3vHi!KB@` z2g^n6r=cXn1}@&`iXXmKr~zNthLrP#$-^BEr}MB4b>Coqdf6yr>R4e0T<aYx(O-JH zCxpb50p_;x^=nLgo6yFbEIkV2`~Xj5VvnVyYa9%U-i6SjJ$o(TTOL)*@5YK>*@KI1 z07}4;esoXyX}g!vEG91)VX0N}YH5oeU5DuFju+~GCK$7VYTJh{KiJy=a6F2Sora-1 z-n}aPqy%*{=Wev}cWO$Tm=f-doCY@_LhEjhOS-55W75{8kX7TedT1Z>WVOcqlEp<H z>vBu4K83zC*vT9ZnCjyXeE>OJsBfM%$3KC7+^*YQ`b;FQ@XEc+Kt_n6cf<u7#)}2{ zl%D!Ap0ovj4!kR@a}9Kdrg!72Wy-;C4WsGq+Yn;io+-G1wbL=9RGn(+4_lDQM3GwE zW$ARS^zI^sST2TIvK#;Bi8?Af^mOaSS$k`7Yb1wzFvq@@y&o(~MRs_?x?xHJph6o{ z@o_FJo9p29x&TeGtc7RAeIQ_^SsI3`3{sDcfnSM?x)k3+EoBDTTr2e{H{?SNjRP5Z z2jp|12P8ZbgVR@$G$&PBx;3*sz?bAEXmAe3K#m})EU{P?SzD;P`-nsLHw+{RxWW=v zh0b?UEadV{=(&UYz2s80k$mRNsAZ%?t>^Dx?z~ul?)t;Vk@yrO_&j1e)#)S+Ng0hu z;M$96VJ&E)?HxP~fbe!<U9=HHjf)pdg5-lIWBiOMGE4q1;DpNz;NaV~_}H^;|7=-a zq{zrmCJ~Eu_hOv-bwBkSxD0TPxX5QxfZnyLBiKC9le;ySM!l9kiRsH?1_7P+foGK6 zSGNXf)uhA=m~8JZqB2QK@>{OT5&HOf?98lLb4MAw-0v7jc)<Ns17E+eq=wN+jPJ3v zAB-GY7e5#LyUtgy;{jRvs9L?}SZgN1pXlzs(Qpv9sfV9W=J-dmEalE^>^v;J4m8wY z>>$)!`EdD7CC@U!9BaAK=DB|Uf+fZRM^mEV%G6OorXV*REedutR``SZX`SSWr3yZ% zeF23(j>80^8cT7|8c5wyHdyANYazKX=o&_spaz+UlkM%XiS>w*=LiTchR26`DZ*>3 zL$n0^iKXfih_9CZU%@nqSqD@&^du^PYbuPWci})IeAJ#pD-)T}rntAh0%M@S9dL4w zlb7R_^o2tMZ(%~XL#Lv}_r`Amc3?Ed4Z$+M<yb21y3H{T-H0rdoH6IQN);30w&a^g z&a^#)(|C03&W2>5r1x#P2c*D<ng-&gGchE%!&Wb?{YK1=568c)*Ey1+DbHPQu2mW| z@@3Q=!-4H=(Ri;S42IRr$6GErZ2}zvQ;xp!+hmCa_rn_)15ptK%rFwpJ<^xv{!S&b zQRxD>LU;tU*&o|=4>wMxpAa(N)yG<9kUg~xNG-s4nsGmTA%*%_FnSflC7WTN(Fw9y zsb*cfuwM~i;hRnWe4j<NK8;Df)Ef;0^j#qLX9gd6o5GRn{oMhWKgklsr%lNzjeEI$ z&F?_mu(P94Fz$?kb9V%1@<D-sc)M#$iW<!3nd|m#@mihojg_YO;)4%kNys#*7rwW0 z6h<!%s>&2g(0S5y%8TZ9Z-J=;^NjWUQ*82o*m<>X$mCO7YdMQ>$Re3mtGzY5p%s^W znf3P`+B84@og6oMAdCk~6}Q@?b=*15H)mdS2Oqm)f^Wo7ssPILFQEHB*_g}jPzqbS zG!D$V$#{m5dFp#DEb!5wJe8-{qn2Squ%nixaJ_h-I-To0D$XH)x4ZM$-1kX$uU5v4 z=z)`Uh5Ce0c!FPlkFTsUPgH}vmjmOei*^;4$omfV=Jc5iL>q8Fhgto3bB9KsN3U#t ze*d93%B@nW%HVirhw@If2FI%Arv~ar=VA2z7U4f?+_xkNL5E9VX|U$mSD|a<eux>* zE4x<##w!`?84Ml|!?3C`sl1oa&GObJ5%BS9nZT1$>oa`(x0g~Sk4H~CgWqQzf{jii zCy5A5mS1t_cD>Kfu<8gAdLf!YUl$9sHHKrsD`NY_gV{_yQBA8zNjy|@SHxyZqT1bx z)pj$8vufD)J4Nd(wxy%N7sW#?)oMh6vXBW4kd3Q>1*hT1(Z&ZgJUxZ7S|QJan9#Ky z<9=cYl_Jd=;rWxYxyxHAwSsF4$gfxiD?wlz!I%jgAk%+8HlPMXY@GMAKUA{>bDINH zo_`8YSoDh(H94^ZB+pYbvC3?-F6}~tJC3iDMl%W9W=$JEd0ed}uLZ=JTKR2X3%2$d z4@h2@4j}6Tr^Cz|Ti0pM973Bet(0#Zr&8|zj-Y<kh+cM@1M5ISYvHm5VCPWtY_D%> zYTcBk^IZ53ngxZ>JZ7!dAgBODxQ`bVu{lvK{`L*j=6DCUvu1DWGM=%Dk$n4o@e*SU zI(DcjHg3j}cd*Vhg(Sw}EsEye)<jG_EjPaZ;*{iK1~0OhPvMK<7Uv5WcWm|(ilnkp zTweq_p!=~$f*ommS#cH4hxUw$)k{;X9sAyT|L+z+xnRGq03-%)-(16Su@#o3dgf&- z7<}E;UrSxBuk4Sr+rpNvw^5@tMC(kdNROC_2J$iqM?x@V5y}$p@J{475I}v-@{JD8 zXDK1IfqR-Kk&9i_eLdYMKvV8|Rm=6MocjR3$);=N3<wSRRQFtV`iMn$q~78PovlTV zD&oU%oVg#tZ(@glJL5^;pe-D2AOsN>6^i}(JW1n@VBEG(wTnIO^CoaTR>YN_#UYGu zpkJ%ixF&!;Uf=p*y8M6)`GdCSp*mf4acaX@nf(`cHEAbhigwRa_u3bTOhF>g>A{tC z9eXVZo(?uWnG0~NEDioGJ^mji<A29x_Doua%b*NdD*-JBUNM%0uV2N?-<F;XzD_7) zb<-GY+ZE)1EQOek>>Rj{m|Qc~K+Vvtnn?(2N)_|%ppMiNb8-!UUmtUpGlBDjx#@?) zjG*?5<#+sCmEK={*CE_tDVknz=1bvr&G9xr%}L#XdO1rS_%6sCzilc#l_v$@tt-H+ zAbIqFfLi`%N%~q+#_IS+pArzZ*{8wC`)!N$1tG3s_<~irB*-Nx&a!|DIr_J?Dn`d= zue{;6ZQ^^igH_L@JF@)EnD2sd9An4wkM^?mpo2NB%29`$tD5UsoAp&fLK4HeXF|L< z-2t;e?S3QFK6$1p!+T}p{^PVou5u<Q=XtvD9oE+s*po86YcFbYg6=2Vui8tdgGTCr zlpx@|AHoGyTVQ`VRJmA>A2@hz?<aLGwz3=S)|v`SP-rw$H2Wg96<!eZrP{I5#l3AF z%Rlhi++wq*@n>aLi|ls0a8?w$1D4*<J6U)d!=`>63MdG47#WSRhY1=aD$=?m)Uae8 zj*AQXJ!QD(>CN{!-u7;A9hwWNA+|cl@Gza_?@XZTlO$0`$_(5dl8h;c(L*)VRA-VK zz4Y<$_&>fLMhFnnbv|=2I<?SysvdQF(ycA92OLapB~03>VMhftDN91@-%pS>a{Wf@ z0c;Krk}<LYWhZe*dgOYPqtiHk^KdT+71+|<y_K}Al2-Y7`2oTmPt#d9y4qHi$@mTf z)QZo0xsduZ_67nFB_=Z{`7r0mZlX#ggX>efggkAcEnnAb{w@(gWP+H@=-*N<2iQ6E zqWntqNp>VriW&=}9Q|Zidnp)k+=QJjdlpwKy%|gE`y390#f26knmcSU%%N}RmE|I_ zp$Q(6ZJBwP?()=Hcv~8)i(4Pg{4909nY9E%!mKo6Ps*3>*v))#Gtb(#O~IyaYYvNE zkoTBQdL@RiM}U%2nNTK9AvuUfvZdYGU|JX#1v^2`Tl$)gy6md9yFBEBg4Zr=y5j<& z08iU;ql5^6-Hi(jcLTnQ6sQ19+`hn(CW}`WFHH&kG<5x3d2avRSz`w!g?!TC4hH)+ z1O1;%CIl{srI@1TF7;>zbt;Xn0?FsJC(N16<xLH;p9$u3UxC>2AcAj`Z*+2NdydGm zR9oO*R1_>;buk=j^09w=_SvxN%NpJ3O~7F&#|v-pw@_ilO8{$<!T!zvw8m14RoUYr z9=KMr9LWKivCQQ6Tl#Ihaj?Fc2Jl=WR{x-H;bO_MWB0EL{ipao;h`tNM$+*`&sbDd zg$oH_$KWS<@XyQ#Q^E_k1rcQjuiZ5Pnl2JQB!162(^*@DkFtQSU6bA?h05%CNu}(A zI7Ir;9o(qgU|jd@^=1kApts5PSsjvY8hF_9H0$qGdxB2=Wx(q_9?@*^H^Rt-0M2hp z!5UdS6n!nRk?EUiw8gd?Np+*I69Vd-H6>TB(H7)co5^a<Y03v9;+pxXQ3+~wr*mG+ z9D25Z<jv-v4M7osMdRikedyRGXrwodmqf7stALzEjTNb(YsW8)ycj)hGfkzaFBog0 zP!FZQTPL^CDp8R)440Mz^SNU?tCeScU<4*=o!FW2B&!7dKmu5ttJ0rua*s;V%}r!y z8HnDC3wOE@UtBpl;<&HESmd2$*XNa?X<I#vJo1{JIm!;W#sK2d+1p6jZayUn7buWs zjv^twr6%pAK2f({LJaWm3Wkmkt|g(%!hur=O{&R$ui*FvU!1^Y#Crx0hsF?+En7Qa zSh&)yO}qy}3Ie9JYdoWeU~8R`C+eC9?hzT4S!O)#m)FapvFc^F_Lex>&&58&PW+yL z%3>x$0&u&9I}Jf19e3!fa(EsP51O|4il;4yMZxAq_EwagK5I)@@TuOUbe>IFwL^ZR zS;|FyKxz<<Zm-b0Zx^v{PSDuTnQ08`@#<ZRdp82X4ocu?6FES^gG0wb;yF=d7iuTI z6#4%<@vNMf_nps9A>duA4n(h6Y9s9FHI~cIG{lJ8{NwPp2(A}HWmdQtS|{oTF9r?R zb!HU5%@%OF99Qo!Ex4`A=wYE)AUm4Nu@N#}>eQTXT;i>WsE%)A4qfR(7M!iu#883R z3B`U<zJyMDI<cUtc15)rbpnYN(q90Qr?Pa_v0L(~L&my@_}j~U{IWLOnFc~~vKKlI zAmnlAary3dp0I#`B4eCuUn*=Ws{w#c7g&=wf2+KPNKUrt$sM(BQn?~ZC20cUmqXkr zy0z)D9&s+sw%wA%9?l&lcU*Tw#+zF{d2-MGI6`()7sEg&_#K0+Q{v7@ap1E<Y;|M9 z&!g`Nr{=yk3!n$Ela}3N!r4}WTT<$3Ox5$aQRX{t$lu0PMdpm3S4iO3{nD?jrw(#F z7CB)@v36sb^pRFs_l|x`0}g-Xw=~vdQ5Qu&%>>vu>_0BF|HT_2Zg3)F#V`f-Mk5zY zZKMExajn@iG_xz(M(P8?3|vy7*-Q!1b)Og{=mK{-_d=zFK~i9*=qB<oVe*|2Cr2@r zNAj!}Rvzq0TxqsxXl>(0FKb_yawlm;!_F!n00HkY{)-lPLrjR|LD7$tWu=2gCQ08T zSwPDxi}C(%(j2OvR&}W#q8}`x`grrw5XQORWqCZ4BVqee1Gin)S8b4y#w-6&Y#jQ{ z;J95M*U_5@ZP$j{$Wqht)BTt0`x<h~7L=9B1h1~>Lu?In7aNm4s4&zsHChgG-3FY@ zK--UQ9K0TQRt(}Te<|(Q9zUN_?Eqa)RgSTFHM8xzV|KekgdYK2cX(OwjeS^$#+JT9 zMNDMEXWE`%UR1v^J^4K^f>l(O_gjFZl#?@jH3d0ds1han$-BK~m{yq!k7$zvd*vms zBB5NIU?3<UCBQex)E7q`R${b$G1>Dy138yQ1BmWd0s;|dui@@NS;kqz`_i<J{kIEJ z6tFSQ?-X+-9lWaNLYb(>)P;s}W}1}DwgEqIvY&@KEy!Zs=o+`<a$MyWrlq5x^e8jl zN@8o<2O}OB+nBQ()3bxCONiSr=WMxj>}Nf;Yvm_gu3Rl*YXr`>+A}|m?@HmeVI^XX z8pR`yr=t&52?e}6NfwPYpBc>Y^F1@9?`#XvO*G7(PJV*|P{5|=l<SmPrQ~JG)HGj` z_p}3!uvz^uEVT?f13nUnOO9(NV;g^v0~nL{0SiAG@vA-g);rNA^7%m~08gE6Q2?Lz z7rsOTnl#pXzX$r5SwGOE`g1;Le+5MIA$Rn(r3d@zaQ-qMj)qy^T=$OEsn1CTdq~GR zZT9632Hy$F0gxpheh=!hqk#c~Gr|)r;5-bI56P^6Zio!9B>|n0hocah7InuVUm~b8 zdbG})zrDEko*$y0N*yWp1}7k}ccyw)zPpMYWC|%srAv**1P#ij#vSsK4Q3B})A~+x zc42!T{2fS=VqPI&{5-5T{d3sN$xx#mNuy>d&dCTKvg%GTq3Mj$Xe2waXeHmhrZp|3 zD04fo_|Hy5^F7ofXqhM`w4P3JnyO&xfrF06P@4~;merS4dN7(4(H?T(sOi$t*L@3s zj80YBNX=-Ni3V4nnE8jlXtIFopWt3QkUp<Jy<3OMWzy}v?LAOIK476R=EfDe0@xE| z^#N(B0RsLX8t%WCGG0Xd-;x!imm(R0uvD(3AcK#M7+U_Z$FAbz^5F}wZ?3@0ap;#C zFXV@+)Wi0DBRw<#hji65H9$!J%C$(pLS5n`CP#-I8(NObE6q$Dz`zY{5sl+R8Az+( zyWRli#{CI+NSTN*kmP(QN@{-n6csih6)NuaMV-ac79`nu8(A(~5xs9ly#JDQz>&9q zK+%8vCSS#Xr-lG^RLU)+LI%M+d_F)3317o~v3dO-M@5TAhr;XZcbO!pQ85Uc(OP?D zb7uleD^v<$Ojs!I^j|y-R&;RE$f)HWEz68&P0%Y3h6~i`+jm3fN9%|!uc;wpuWKCi zM}vX%3B5U$VE-t<9q!QB<MUnJodJx4$XU3b(fdbVK!~itAG|EPfzik-(`UEaThl%$ zXF;{#7a?wa3=;8oj*SmKfm;%*pqzP?O-tU9t~=7VnoC=HG$my)b>DQe?GZ)ziML1F zr?4}u<5!%f`t-PC!O?*ZH-`=3(3%PrAEGui3bu7Z|J2jxjs!D8Oa%Ni2QBDO1hy59 zLRDW%m*qkUV_@vMK7!r+SRg2~gJa#Lut(!yENtEb7n#CHgfMq#!Hqc}*qScy(z9p1 zNpif!1whuCK)a1^u;MA|AVO%ScB2p<t)@%qZN>j`)sb3c0pIZsiQ*bxzX^E1LN%Z7 zvv9iy$(~2=<ztKMcIGy5to~jf1$Yk|vA@Lp%|Bwf;s}KlQ-ThM^g2za!S0D8P3NG{ ze%Xe|ey*9*7tR$fPWy!sI;R5F+TJ@t=DrhD+7{1EdR7?wC1+f4v{P6O_%%+a-b@+@ zS>=t5O<f+%p(GyZ;~(D`rR^?DvnvD4mm8an46$irHq0tA6R-;F^&18|<hQ~(uT~am zs14d@s&vuFxT#Agg>_%QBTor;5te~W;F^mo0;%J2>YPJ315?Oyi=`WHo1XP*AIPg- zn<8@wlt^`HNd|Sl&dcHJAX6K2FU9ypu;S9Tz1ZM(du7CN>d%~Na@Pi*b7p2<sFi+M zXSp-L{N^-}+3%K@RLfV}Nq$bxD{umrN66-Au&dwnI*jL?>Pc^^V`i=8Y29)^nZd#i zFxLHCurzVK3*GxpvLlC6?l||Pf$=~h_(riy#HRL(F+tk?9l@2+D1HzXd$Kz}D*vAe z+JCPvUVQzp7MkHyClqs9L?eQ6Xy;=<J_SQM1qR<C8Nrhhqiy)_ftecjB<vr`7o&Ut zCS_v(T)Prfsddv3sPQ%?5`Q>Q?@h&XT3jCZbJjEcHV3?A`n=cfK_zzlsUvG`CiBbR zCEE1raO>rJ9xQYEfrGSK1@|hZQn=B4&Zfs`2<MH-*RFDrtA!qf!-9!8d%k{jt&?_8 z!-kiJfMg#uF+<5m{h*2Ivj2b;{mGVg{X4T(*$bjoZ_n=dNx-^}jn7v1b1ordAAe(5 zbRHK*#k=kWMr>}|!k%cxrgIDO4tm)2;Y$<jx+>aR=su~}H@tZ+w%c!^woh5%?RB7u z?N_Ma@#Yl2OS(!`T{}F#ZABmIe|X5V$<K&V);sRCdH5}X^~yiG#v%<2^H7`X$B~tU z0p;_h&0#itbEK2^@X{6p%!QYVDebd%uz=4JgV%*0ZY)ulCqm7Cc}Q22kTsg__K#fK zzHlqYFG`NZf$dhCKh9(pF=&xVoWi#f7KhKu*5AepIgs{AAvF$AEf<7l4_-akUF&Ss zKHg!V>c{86{uJxm<v0@BbR_EdyDIUM7uJas@aD+lfyH_}L;^D#9NL80QLdDxL2pwL zyzw*WuHM9lWv)9*JFMu-P6_-TDzTS_9G!K&`5Bg<f@0wfL02Y<6`vpPRJUVBHRdaO z3bPJ2z8kWzfmF%0lLWtexeTLob#YC6w@J=YO)*<fQL2}5d?g$F#O=<;9{@YvaL!)D zW?jNuz$$C!-;q|N$BA1T`)Ia?8DMXJY|;9v)S3S42T!M_vGc1P)KH4CcMOA6I#yD~ zt?#p>)_7V-hKBNoq?go2)_DLAksD9h9%sGPb(DS9<(RtBP%e^Lp#)5P9^#;VVEpOA z{)e;v{-!}gpjmxq#<H3GdGKIJfihno)a9^&)EvEuMPclK?rp*pN%;f^3R{OeJ!GlT z(PddhY*QjIBPx|2tS`rtP^8k}5xXX{R$thlZ3pS=*G4pPI#Y#AuRageu@X;iGi+c_ zp)&L$F<c1k&h#K+rA6oul)l_3l(Q@}XK<s@p;y!PF;i?Ku?5I;=n>L4N(VYXFMdyV ztFxFlR!)yA1K{sC%)vDHKB>_1I<D=y{2|ko6sf&(TJQR?fP4{==-9!v{Lhb~3kneK zP?W=!Uv>Wzl0^L-{>u%#cDrZE&R|0D-ZgDKWM{{#VMm$;I2$O&-#5{eoyP$7iRRNR zZ83_$*pB1R-D%&44Nqs(00G!+ujF#}&keHjKv1>KM;J+dIGm%R$5BJX?f!x9#cvnY z0sx6hr&2_{juyJ>>eKS4VVn6N(Txi~duX*T6J`75-$X1>?$rNUy~Bv9#KYaE#F*9! zm3jJ59rzdrw4U!b`!I%*JD<%Uw3lN0`FaRn=iRieO&rjfl!6eBia8aRw)~llHdQK` z5+TgX68*X$6X&q>nF-{$RwBY4A5~cCb1{^JZqCG%NHJ?lpof2jk@d^m8#Y|sCz5Jq zy^wFleyiT@u;KpAhkWrHHaxq8q4MBEAr!+N@05lBv0~2~>Y6HaIqET-A5*O4hdQ`p zxu?TAYve=U5_6B(j}6x6&NB>MPDw0rk8kN<E`Ij8F4GK<3F$c2z+4N$g(Y}M4N<C5 zj&Y^K;fabx&saovcy~^xdeo-`j=7bKjujD~m;J&*EI9M4lQHOy21T(HIp0mhI5m&u zSabg=Lb`$^%XLtr5aFb04dXQLu`tIr>}+tXG+8oIVwOSe7f5{JNQ(eJL@^prQ1uhD zc2fSAi1li^6a(dK;*oC%N`zo!W9NTVp6Qv5BT~rMc$LO{0s_v#y!MV$MyslQ7HY$A zxZGHD2d(J(dR$iUt|xV<AFrH&?t6h`yvm9i8iIHDWp;w%)=O+}YR*$M9-FKSPLnT& zvgUZ=0e1BMmu1oxv#DhXC3;+o`D(w+UzLANp7S%n(pY5O7ocpAJx#SrD1Z+;diUxd zi!K_>!f*7L*BG40AX;M^lnHzeJqHvjyE>6#&inG>YyF2LpmnFc^pcqvr_J6FEh&Zh zWvxku{@JsTC<iIGLz|bU1#BQkkzR!*+y;Mg*GIR)ebym|B?zWI_SH&#ory!X;|8Wv z)lU1jF>e25^np}$_N&|bYMFTWk0$DEA`gX@H^_}wE18k=;%6F-+#>TYe6te)8D$I# zJqUUylWp4ZAPXjsuW&R*V>cCJ2_u!IxHbrth>pZF05Dpe%q{xsQG5RcO1z8mk7Nu; z3NkxN>w6PR;S_Uh_T8|c(}5UM)e4<IogQ+e{6_-j--AT-xnH4VcY(jCkoY5!ol>t@ zK7b$TMIMvE?NF_!z1=5&9$!E*_Nfntg-V9r3ixEYVar>Z<Trez^z^4L%>woIjiXsi z=JoiQzDQbfgz(9iNJ0RSIewLyHE+@4)m|CeA0IO;G#JB?P7aKx@Y1ksPr58Q0Rd4h zU&;YLNLSxamnmUcdhM2QtiPSxYQ+O3gPM7B=f9U~oodRvOTLhn^Kn?Bs{a+g25~^* zTk;D+==XF%Y5P|BAdNmgf19_`YgAN-N{OZ(%xeoy6wjn-Oe9cCcn`@OC^;dD77GD} z2F|!c5C&#j`|0o+hWGnep;*y|%J#|Q+sphZ%U+yAXUm!=uQxlcIJb~9kAmvUI4p?l zx(|iN#@EEcY%<QtpEp!^wGFjHB9#>j8^YSZiEBlv$`eO_g}EmCbI!Hx|E42WxJpcx z7ZtjMRiCwy3ja1kcz8+fM8QMR{+}j-i-yWjuXjk@{HzU&Im65Zf6k)ikW7MEFusiy zu{l~h)PvfrObEDf@&KK3lSq~<<Y-==@wcVv2~53DKjTnha%-iqgZC^~y>-RJ@3aUT z%`v7_iLsdXV<(e#3KaU~_b(B2nDQmJG|bOt^q7Tz7r*i)mMB>tsBxuK1IDO-36dGw zW8r-iHqF%;bhs6XNV{Zc+?1lR|5Xob<kvuy;&J<8*GLQ_ze+yaIbFn^4l1M*_y<j2 zCQSSS?*<qckz+Txg8Wr|0c9I>qYt<j$32IrM-Senn1M>oYSrK`fCY1a5q$#Smbwge z=TEJ2Y?j&#V>-L7eP!78O8PuMU|I6b^&qXp`={}*(Wx#YGjFh}mjU)*aT}8R;8p1L zcq{HV>3t4AgSy8`7*Cw~$gwz#15F$0X_)w%j$j#D5nG`gFXrW9()CFu(zugfh05Hr zAVOarmE(puIPE=p*Tub6Xd%{E-F{NOhbchH2aQ6+?xd$43zmO{^ep}bg(=-*&Fc@I z1Q`9eb2zTzRH>nwV9>+ReoAeG{*xazmLCIlNWx0VQZ<N4Y72awYAjM^&pTm<?hW}r zguQoA(`(!P8#)`%fFeao5bTstM0x^>z}^<bf)IK~0tTdp5UL6aB)C<o0)p7+CG--c z1Pu~E2vs_v1PC>h5IA3-cYZVH%$)c4{O@E0y>j38wLWXDHSvg2{e{2H5r94l^%;2l z<H9E;2iEXFa&ZKk{##{(ixSeVk?gRS=YY7SyYQh7jyS%Vj3|Uz552@tZRGrb1c&75 z9yMF3E0z)n&F`Gs-jtJ$^~276fjRA_2eDc@O6TD7-|3zQp#BWqE%g5%BK-g1hDkAh zR2OirZ`Bd1&Q%?&3r(nP`|{P&K0MzO(=rJ>krc9{B~WH9e2tq}e#tnlf=?C=Ds#Q| zFBi!<#<N>53>xIvqzp{%HuKLS7Z<#vode-~lD1T?4Em&XFb@|qexCMAK9Eor{^eb= z1I2K}5;D`!v0plDcO2F=97!=X^4gSoXCvf{b`j^WGA7G%FDyvtbTf8;$2Cpr$8)|c zU-c8Lf3X>HWM?osSmB_&z24>Xt-a&#B6NdAoUi<p_{wAoi4}+unmIG&*G~Ev9jRB! zPhJ*Ub>wwGH}Z}8y~|x)Te26#R*NeRx{t<mw4KZ0aQFNVN61xDz~}HtVfN2%Qoa<3 zy}(Q(7sP*ZGNKCRW9N~wk>owqB-+9V*Fu916#<{s02C;;W-(hf?vv{2F8_)nlh{Y& zh>YD{(S9kiGw;34_hS2GA5)@#e;FTyo<1Cd&)uj2K9#6N8WSVx0E=2F{>wK0_(?nD z&4uMW{T(asZp{mvC=I!MEYNMnRQ|Un4NC7}Am*>?n%3`l7p@Ifu8arX>d>wP+4H{f z+efIKz$QSwi90zy7>q{6yJk0}0jkYb@u(H7^&}!Kqwq0xTl3kwwr6AZ(bJWD-Zm^g zG86z95)?dC!v7Fh_51trmv2rCT`d?#1n+Fej6ACQIg*pcFlBx&6aZcvnzEk~V#oDO z$Qwfso_@pAb}5`m$8)=+ZXHjDS+1&GRTAMRE!HK+6SMy4B(a8Oqck%}cX*F$1?20^ zzGoJ`$4DTf%V;>K#t(mBG&7ruEn=w#!(ghafW!5=azjac)xqzRF`5&mWaeke-$T*( zU|hpu8i$;pP{63htj6jgBK*Zh2(L7A%C?=)12p=QEMHKQE(z`HN{36z1tdyDs?ueG zXVg(PYYMfWn$?nnG{mo2`6Lva%AKg6%X(Q_{^P66OHC3|?{4g3EYkji54EA(LC=Dt zZeBDFRwWQwZ(WQx>{uU1Hovj%xI0V}Y66Dyj1^zP0Zpr^rUTE-c?3Se0k&$S*EJ&V zX4D_{7l#MZBOLYz*WVm&C(mAY*d3WAJLsWzrv_H4_II-mch`Zs*YMr^roH^18$c)D z*kd3**k@5B#QNyLANDYQG;ixFc&|9wA&kaVSs>l>l;vUcQvqQu=l3y_-|_7iFzb2c zcUEFT^ZVUs80^mY@a+os-OIF5;3X<(cohBE;NE}P<>nEO{sD$z($1oWnL1Wq4ItmI zBz(2L{lNS_eHF!+sPFqOc|*ZX$)=1LAZXYXc`qbe`n_8h_jAK)w?ls60Z!DWDq3_V zo#TRyuo?J=m&{3&jAK=RxLx^yi&3_<uoke8#7atTWHOBR7*Sd4;oy@cQ`{q>rOjN| zC0bVvEjOh)lO&@IfprJxcVjR^Q^Nz$r`up}Jw!s@3`l-yi4F+GRyn4Py`2jb7>Og^ zD!dUba+nqs)KAr<74a3D42-9RhxkD`aW*IC)4w;)?}Ce)_~lHeOcjq_BWo<BEYwxP zAJ@bu1K97_fOi6?Xci!uE|YI-A4ThT^gP2$elDL1Mc%4p9_1Ga^=k$OR#%LpRnoT4 zDh~Dx)?cg21I_4J*9~%x=i#6?UYjnqcaE9OD{$j#ci^oQdD31cxucyPb<g{nO~8?f zHG;R-t~b(rF3|vI1Fl9jlNzqS4qZn69vg#9SIw()LNFVAq;FC!Vy$0Jnvlk$SMrV` z$J7Is?QRJVDImHl>PeU@D}3$BrFXX*Ya@)CkUzzdjPFQU5&%O8No>>uB_9Td|Az&L z>l9CLQeL%V4aKADX$rn!_1{R;?dqz7v5_DU$#VVprXK3|?r(4tN+)2&zbaNYwC1ho z#MarzlftZfv@!rqys*atQKS4&AL{2Z-_N8xVgz#Q0&>8r6RJ4Owtk!^$)h~tAE*XM z^=q`GsDdyq@BFuW1`_UKqu-c?-qDhvTXbDZAg%|_H91+0bSPW(<E~)3_rwA4;O548 ziH=pjio?(EDhbaHuQw=5QB&#3%R{9^G~VS{zvB^K`w<xq;X-XW2EY(*-7&~@(|P2I z6`R9;_$juNx)$8GWYt2dc6W8@PGA&j9)=tk&mAExc>~l@YlO*ua6nTe`*o1V<h1u_ z8SPdDt|1z);NOk38OQha0C%e3x~Nh(FuI?g^vqduCFL~_3AKA|dK(C}g-<ixYBpZ^ zPplN&EWq#o1`*(^QD7xX7)>KsXgc41>+9_PIV_{h?2sCnqy<zhY8P#(5oP>NmzrvD z<NJ<t0MbWDrMh;#m>hK3?LAHNl&ds+7q$9)$YxW%C0ri{oZxHP7v1jumjiT3oChIS zcDgtzWji74ET6_!4ZC-4`g@lJNJp^uah9?w)Zgrq0Nkk#a7S?6v+(Vp`}1bb2qoP4 zWC$+~|E_qM@5$^5js=9ZmE@z2E0>fF{wSf$CLX8F{iEz>?=vh}VITxakq(zNOy}^c zIa+DNj=?@W5@~U1T`tj5X{Pg&fPpK`97=c1$1?uqp|)m3Sk^Z)L&F{2AoRpsasM5* zS!YRmqepqA%bPZI@>9GBf;{Zltlpqg%q~$s)Afxca%i;L5uGTHiTg8)e#Qe1wt8(e z{t(LW#?CcT*9*NgL1?QbN0jPFao=vcd6Nz{$~b5Yv2p(Rb~oI8+e=7peD68j{t>&? zL+fUT{md6=$T06jxqW90Kj@5b)!+xVQ+Cf+axXM}R~z^q0e$7H8Poswzy$OUy68=t z9_K%6dlv#9(_Fw~vmTP#{o(U&mt?~_v)}fgBR>NYFCw$AJW@{mciYio=N{{epBd|+ z-RArBhrg8}|9aS5w$epcPDsC~c<%T`6P2@CS|yu1{yo#l)toq<^#@&dS#DM@eSPZY zmtsv+L7>ClPZmhfwkphll-BIS#3<6W+mDR0LRlPZ8n`cFGfT@d73RrM--LF8H0yKV zc32AZ)Vy7oYjew3{YK-QjaqqCN(nP!m^8m_FME89{1skiZg4m$?hSsk-+qQ(QqTS5 zKkx8^gK^I7Vs<wxXvrZ5wBE6;$yDUsF=(3J*4U`KKX-m#De&u&0Xt?UXVQIFjSSy_ zd-z^|JCM$+zkZeJaKW9>+#$2rT|GhLQ)XgOo(0|!vO+_^SuhH?ZmgimFAtL+6hn}% z`1=icD(Gv5##EO}f}Xq*+wG3VbbGr!v`M^0Bu4qOhTzq&#U68MbLBbrgOrqf9q9PZ zHt#H(jU6i@Njrk!pA|;4TA*~~@qE4NUSDAK>ts_$dL?gIXOJ5;+l2=bf4Ll~-0ow0 zPJV-zZ9@7I>=dh+S-!e|gK67mzSPm`g<}6`{q{!}tr;7%cGK{$29^|>z|KgUnTWl` z(ecu7IHH|93BJv;ah&Zy9=ZKs_*$H{Ao?+yLu2-g@Vc4Z*O)h&+W8j3h7AXM^prn( zN}X2|(DRmZNQ1E1`nMA`HAdqvyF1!}-IFUrw~Rs4fOn4dy>@%)VWQ%|o2?67o%({W zr2GLYu@J&8u;DMiTbcRabrj>ffJM$lV@d&}E_)vGy*-xiiH2d)MbW2Ow=Do(0h$kn zWI~=snrH&xhz{Q#C2M2c$FXRP!>b3xDvEu#WvJ&t`w=Z_AknQ$TsAsaC^=zd`0}7i zE+Lcs`5h0WYS?4mW)DCY1jjm}s|_G4&FA=EbZ#2`U~XDMO3E}nT-p$Syi$Vq=xtK_ z^`BziBx+z($LSHST~^`ZTfb#Lh$fB;2+NhcByey@_rrT55;A)HkVgJORDySDnXEev zr>{578XOl(=-Y6rIK`O(RHmM~{j?~@g>=$&K$^fNYeo3!9e+Q(9c6>E4Q25z3F5C& zW6Q(oZSNC)TJdppq_eFu7apz+X6g7#Stif%yxn6etduq!EyEkj{ge62z+mn15#9v} zAyKIwQYh%tVqdzTZ0kp&HTbo2MxJs*XRFuvFn!&hy7wb(wy+TiM`NeRw+jqDwX2TC z{J@Mpg5nV3{#F=M``G&HO4YF6bMGCTf+jk3M>!=l&mz>61${%sUxch`6rCFj?OZK- z?bJm7X@DA+wZ1#>1aAtmOym(#riY-D+@bAOz|qv^5(oO!1qH*}C1M+MQv|FwxW<3M zu1&b)->PiTbh!6m^GqgJL1Ft}6*WA{<?Ta^M~n2Y!Aw2j6}vpt5?SCa9oDvfp=QSz zy5|Vmtsf7gn!FGTEASIDqVY$1@1iIXJN^X<kB43+SE<BE3{`d7)<$p4O*i@dqH!9d z@AJ-xH+cH<Geg3ywZ=M#KK2@_&tF5HK9_j0V5FT)9P<8G?2<m`ub-So!zn%9xbh*F zJhmi1o=hIMh`S(|U^AT=RUCTUUo@w^=G6k+D>glG$4RWU|FSw!hOQ+r@z9{KG~O%w z02)}-lhUJWLhC-cT1tJtS~V}s^LM#_tjo>H{o#Y$t><>yk9xLZ<}i#hzRHO9ZKm^q zll0=2TO~ufZj<4;L@IrCxH3c@&J9=RD_eHPDx^3jv^S=udPHrd=~Q11d;2hh1m%ZZ zF9lEqsg%LFvyovNP3h#4I&tB3cjX9Y*#wjGmW#st=!&9ctd83e+~3&kyBz#Ox~}pk z-eOj2$UydFSa9Wn(*EMm4`0T4@1Os3fz=l<z23xC8yovC{p8w(&Q5(j+@(2{rx2&s z3gCI6JW62iJGvQ;QF0$t{<j6v+DCxnH4Qc{1FfGo%Wpsc#t@k>(-VbULk1!L{LnCd z`SAG2T+F_}{c9%al)L|=J+Sv%TW|Oj?!ZsmG`!MtnhS!F^P~Hg+;5LDL?Al91;}#u zZgB8wD0z~ABXlMuW+F7nz+Ym7i=-h}q<P?Z6%4Z&v?iU?<CBk+#Iu3ld*8yY!%Q!? z64I^i)gC7HP$b$g^8Ket`)?$JEK+q!Y1*_N?n0&HQ@llt#l%=NPM&Rgq&~ca5=u<% z4_2lRMXw5Qs){tdsci@u+R#JE=DZuFSdPjr^Kq$uH40ejJIk#%YrrdGFk;cf*y+<s zzB&gBCgl!ofYZ~|%XwEN+=)nJu_ou*ynfPwBLA78_^qa=bm8ppnD5fzf0UK;tZ!Ae z)o5E*OLp>oajqQQ<UQ}rW5T?D6($rV7u#IE78MF6x=s{VC4W9*dD#-$SawTHUy0)B zFv$Dt0lMikGCQUDm66}*^$>AR->}K1LPF0;2so1_L8K>i9)EJq5Aq)9nf-2gBN5o$ zY7*BiMz2oWu1{6XHSv#^%ZS+`n>!0@=GxZg(_es8uD)Hw*xR~Hyqb%C(8$;L_10Kp z@|+$X5^=c3To;0Dz(zAR_$i9#J6DxhSLv8+o;3w-pIu1!_GD^4Wm?*6SC&;fJQjnu z;QP`P=oF&~9|Wr+gHVvC8<{T8_6sZLQxSgmhm#s&Z4J*tZG&eSf81}s8IZ+kC}#0m z{C<~0u`DpZ#dSnINW#HbsaTja8`p!-3P?yha$5$2$trL!oHMTlDw0<a<=@QgsNfiu z8rOPd3x#<4Vk)P#WNO?<Vsxy&S(^91NTU`@gS4fLVLrhB4Z5FcDJf+_{JSq)bhz#D z%SObjJ9LSsCIj=aeQLm7H(O}gRHE$D+{uaU8&vDQ&N9uDT&mL*eUdz=p%izvXTZ_P zr{FuLRJ<uv*^r&<v!2>nBwl`y>(UW)4eKFBe7DvhGSR<b{TAM%_TApH`$Yo)BdmxS z<TTDC+PO}*tyvzdwAEQQD3m4z;)<#OTe^!@ou#xn@RB!mZVJkA-$}R?JMT{oP9EU~ za*J-bQwegerCdfoMKXk6G>|TkAJ6@?kxVvSyyK)$;*Av6wf2Q?*HDz<yRYCi`2Z8b z2qyz&FZEjoLJ7zr_kNOfq5{s%O#??gzHpb<%N>K);+I?&;or^tL_39H{#}zjSp}=I zU2m^De1Cxj_I>FDlg+vS+6|F@Q}*Tm$LjX~_P;U9IoIpxjKc#4N`;q}a(W`k-*<i8 z(QQ{KYlMPWA}mV+pA&_@_DZPZx!fJmE9pFWfur?G);)d%AgyH5rC=c1T#^<q4g84d zW2d}}6J%ddh^}Hg!pZ>oRz(!=sqTk)U8F2dZV4H???SHVy{uINag)(5c&s=}M7yI? z4|ZGj0+Iu)0Gm$4wDR)uw+;*y$*Q(Tp5PQxma3NFL6wBHZkwtIC6!k0tDHm?>M)|z zmF0S4;PSBX9|z(+M>L66!xu+lXa%|rdlLG0q~yc%DP23TVjHp7gYoP;r1SetfV?5O zdWqMRl(j9GYIJ+?$J+=;8F`qJk8yc@C@Zd`c5EcZQzrGfNI9r7L^QtuP$%xFg-Jct zR+dn%J?hWfX=dxQ43u{ii<9+tQjI3Y5INJQFHgIjU59zax^%G#a#faL)7l@FeZAcy zdu`wnFcU*zsHNkM_#tg=)j5X;Vw0pvI{B0<Pv7J8eRc?RacCm30Vin>YTXd#xRsc< zbAvmaK>z?H$*~iMmDIeB^uZTuFVT34F~Ng{RtaGRGkr$u{3R>e>aw4?2(^%adaz~9 zz)S+IUh|fa%0>v_hs!h->*$1a|G|w|z0vsl{-Z2sB~eU0`-S~pls_VCd6}zTVvdlQ z?rcr}_pfC=l7g)N+6AazQpGmva6bKJ!c>Z8FpsmPFB5nq?5!`R)PH-MObrJ2RaZGA zQ9QyKB`Ps-6%QB2rCf^b-wo&n@yNy8F@06rb+jIaayL9^hKxp^_b|+Sx%55Pn!9Er zt#bPqOsq~@%EL8V!Z;z}s-V@2=n0`oH;yigBp=biYs#kOJ~q>8!{9H-k5HuB>uVBa zd>lAXa6uYR%eC^Z9QWj`Z9{#68xVxI_}%vH#E{6ltNcHHM35B`mLoK!5(`#2XbItW zlFiRPBj8&G`D!=J_{`|+49?tG555<yl5rsxn`PbN&Qb}Q2>E?@02t;23hLDcwiS?h zRlK=kJ)t-rU|~+N1p%kL9wR6H=hDD=uCjZ9BdqZzrK#XDYQJJZ<X>`!J(|HD0+~?( zBBFLWEp+Fwm#Coh_k-t!N51w&xdEed7{2>wJvB<V-(JMWa6K4D&%GnVlL^F--yze9 z<UdSdp*fYiArB_0!CRr6T|J;7*o~vQ3^@|Cbf3xW)C~}1vi$AVor1blMGgL4kV@Pe zCXn+`^<FhI1Mq70BN}Al4jbMj3oH(_rIOS?{g<IWn!w2^LC!avX^`qYADN(vymVXv z;bW9&Az`}M979NZENj+tCxB{Mh@q(#^*JRbg7i}wmTz8%=Dz1~a%7(A>^u<_VpS|; z{5-ZW7gSxTC2;4TU0G1yItRBy&gzBA=b*3kSvtZ#Mz4jSuI)`c<5odxk)L$kXzyN^ z8jdB?WQplpl7O_zwDG^3o+e1T8Sv(S&Jq5!I#-hG+9&V!wjrJAW*6(<eMdKy`#)-d zfu((5quYBQsQae(BP0L%AcAugAH;MMmDsR%NA&F4#Qi7nh`_|ICP40~aM;Uy4xxr~ zl<)k5?b!fZijnO{Y1Km;28x+Q91xezt02jdjj{_<+TlwBC`3_M3koU#kfhXT;UUPT zynte3$<YE1uHUlRBN?Xr-Pe3fM-W5^%wt)nFJ(>V86+=!C4&ciU8WW~5pE1bQp25p z>LmsXjHz|Rv&?YN!W&-;Q#-P~z%Vf>wekw!ZwWoVI+Z%hr`$v!4q5M5v4&rWR3A!3 z;q#8rd*zmp0V}Gll0)%1YzznRXFr!ZClp;9t9>$Feuj5i|Cdfa>x%5v`hcq=MuREX zLh<-em-*%vOQBxvNVNpA4Qq)X@DGT&*$sBvn4?z~50s~Zslj}7c~P9{C@R4R&rZ`w zv4O7**^%oBd~0Qo6^#E#<d;F}jB(t#o$3=(1#DcS7E9_4p-CAeDLS44TAxjUdW>#2 znnGaiJ8oiTFNmrd6JlLT;L+#q<E^z4^3uz(R7rx9*o13RA#iKJY7mR4Q9k%9VU~95 z_7#xzB~O~o1wF3U?+;z&)-@|qDkU%D_N2XngeRvzL!G5-r*X>J8i?e%nw|XD_|_vA zGT!L{KYDqS>qH<<VVLnB1*c2zYg%;;?>@Zl*`ynfMb^VYY+4N?(GEG_)uy|(d!wLo z*vxhEw`f%bo;i+DnX0nA%yksWPPRm#Xj3saMv`f1WM|y#@i!d1GK@G<-DlK05Tzzo z@)l;olIDa>yz-L&)g$H|bK5g5$dl`@D<vZnIZKt{+IL7Y5gS=4bogd4d1QQRZFbt! zx#K^y1JLhlBfF7GQknxArq*S~*JOfjzU6sv36VVew|QvqW3qD9(!~wMrgcR!Fx<i0 zEacS=<in@FML5h^{{Yu8c}mE%R)#f(bTVXFtiG_=nqS8zb1!6}*xpJqiK6ZD>i*S7 z$6pVG?<mu_tsZ}k|1b1X3FSm8&&%0h8}vik3|58om1cA3PD)!gnRK{xPvh8kOsbR8 zuw>Vb;}<++XgK08s?iF+_LIxjX>szGr!|^bliSL%UDy?zb74VDf!0A{Y^jKMLNv%W zg)f_s?VxibQ^yuCjro0Gi|rhXBH%T{lw8_YBY9#Fh!9~{#srKNB-U89-_98u>hH;` zyX}p0RdD0KLN<;(weX7P?FC|V;HLUysBkbe9Gw=UFumgTwokidIK-u6Posh=#j+k> z-fHn`;`#Y7Az_Ddz9Rne&z(l<bkxJWaPTvl<^y_SvP+`EfyYC+;psD=p+HEpGOw=! zs7x~p<9~ZkErZ8b?3;yhkB1Id1<YlE=)Y4-9xLY{Qgj2D%rnv=Qr7T<sNV1Z8I#Ck zWCHDj7G8q4)4eJ2+kjZyxk`<MPyRn_FduG+!76D`kH`0AM3zv65NVQfcl`K<BW-N3 zs?3tUU+$<^3SqVQsp@7Kj5kZn?iTUA_8RK2&l!N>^4$2sw?!J1BIes%Q9|<Lr@F&= zn+ak0*ikO4AylCOl%$l29Lyw`R?zbKAT|??1m<G!nZ|&qx%v}#MQxKqx2B{}c3V`8 zG|*2%m7WPLU_d_8f&7MA)xV(q>&}+QBbBs}&0|tzXo198Ci0y#N)VQ;HbFK=AfY(P zqML8cuBjYtey|yxM?JvMwvSR9wq;0VT=dinl!K?<k^9W(lv5_ms0N18OUnOTp<n1% z(=`5Ep(nc)KeEIOE?3R$zlu=Lys5(3ZS_rIUmam8STuQA+1I_oPB1k-+RRhD#al$K zz|CKD#acno{VctRi9S93VOG!Dw?K1E#N_o{j;bZ^{w#AKT317L2C=#(r2RX3AjjVi z*n~{O>G8Qi1so83rLwMZc`mtJ|7`i{`}=b@MgeCJpbSa=9!QNSM-G~G(rRMp%#$wS zzhA$H{)q5!4n!V`So`PI=V2=j0q^e#@myl~=`nW|7%^4S?pz`J)-maE9=V7a1sl-g zhC2e6P<I_P?k*up=9Ik2AaCBdz}Hwtf3Z#Gp<olWj}7!wI;b@6OwGOrV?VQ{uXb$f z2ksUs7CYFly?C$`deEssu~h+@8{RRdt!{qHA31>`2A$0aRLm{A)W_AU5a~A~i3*V3 z(bIgs>D1-=Rwnku+(^+*$cWEXyJ6JRi(KpBQl({8|K+@8<^wWqG5oOw$QnF{nRrMp z^Q5U3ABBlRETX_&ro_8nQpj6JM=+JXVx@6EcM_!Mv2;d*kI2qn4kDl$J}bGw`6z1z z)eOEP3mnXOi3x4T@!VM(VH1Y$W5KZJ#9`pqwAudyRuD_gZS2@cBrmvu^qBxufqN4u z(KLHV4UvJas>Mi*g*7WxC*wpMZ)x^=)MSdpX{dWh$lf-TpF)XvK46LbOvm}3caE8F zPciwN{PKWY&fk5i;#XM>&(90<$JVl5T9G#%{?xV70N+87dC9RKR^|t-{Y=|`#`{|; zT{*OHyyGIw)nQL_ARk3T7h8s^zq1WxFmzk@CU=ewvH0HT5^m%0`QU@@{{%X5l*l75 zv|r_JmUolvZ1M9n8Y^tz`sQM<#0=Z@Y$j3dZ6*<Ycd^o}D2zkm^APJJ?f1SMXAh55 znWc#h#$ybqaocwqYWjvF9#gIQay+I1wtm7L1qOU{J=>w6a;>U7TO<QN4Gnc_%4L;O z_buw`Yks|e)N7UPbqMG1T=BP^X~6RwwV9}$WzA=WHAl5WpXQ*2{Pxl8(tsBB+2po{ zj5h`n1NTvObAA({p)4*sE4E7wW#`dQ(D_r@F|#?;yI5Sy_mHHfLJzil`ykF1HeF4( zd-C38bvC1^Ix1<m4I|w5c23;h)xB`!rQl-(i)Yt+bNeX0>ctZ3M!EP5fmy{P*V4hT z0qrJegdQLxWS{cZR3snYxuzcZW86a6jD96sk+a3OvdJQ~EBd)ffJD>~V%gZoN(z>| z7B`PyKFjezX65+P0tMMb!%Th1(^9JzPo0M@!18z+US3k@WxcwVd<>u^NcLE#vnzkg zq^+mblqY9R>Cfoqf;{qZo}(=_DgB9LcXF}Xq5Gsuf_UzlI0@<cyDQ-YPKRkX=Wv!5 z%EHf&ruB@`{f!0bY4dLZ10n#5uffaOaVrDaE5}cikUON%m4C@0c}Rl#1<cMXxmNa| zLDXoY5)kf^&y*Fuo@0;Q>L)|@zd?;ZjOi;-1QWa)`5oEc1|kB`oA@jj7{KLMsf_Qw z8?V{Wc!JeP@+I$ym0&4%r-usqV?}@b1pN{*(_c>*Fz1bZ1uMz@<B+?U@pLK0&7Ynj zd+_f?JoP7P@_C6V7{CinzgZIg%i(!jKwF>#)sy<18}iDVl)f;CDN>goFSOz|%=Bw@ z`3vG-`Q9F$lWkNi+x4v<Gqz_7-c;ZJ#g&9vEtURAYN1<>7{r~m;njcv2K&f9GcmFl zNQ6flU<#L3-1GqtgmT;+thT`PR@mSV*m<lCm*Y&R{KPxH5-+i;j#SDpBPyccxz@qw z3oZ5l`n>9AejT+hFXt?l+OE@?wlbsf)+4jYNv5DF>{&Mg>TU(XB|it;bp&<GswtkU zf?};iP|7TsAA4$NR0?Gd4};&0<twh75TZOY`*=$^e);jow)w%o?S$UP9d=+EQt_oN zT(S<#zId^ra|`hlxS8HMyi?*U#e^g`rPzSZhDk4!KS&`&Xm2$idJjz3gG<%ryzHvN zr^++p?{(=0be%o^7^ytHMf?FIN%7dr6G3@Ar!jKx@^1EhDh&C&oLxC4p(fOBc)IwU zSlnl(&1wlxGn)R%%YExF_OK2763+jy)LRkfwcBCq+(d|UK%!caH_5RGB$|u@NENqp zSH4hnr<R8-^nf++)xopw|Grr_U=b3jV|7~3sImoBDO*&CB1@TZy+WyoHI*_&-ut#Q zu`E(Z7TPiB1U<5AjD&qExkxfVHmU{CqsY<})s07ryv_wBmnI3n5QiVayQVrD=N9@$ z$X2XJ$Rqf$;4)`hiU)e!sZec?ePp?Ju0CGX*Ze#ML5Fg7WTF;#Y#)Em`wwpUOUB*I zCdF@)DYdsd&yY#m{axlJ8eX}dGO_?TT0JONep&d;*8GTkdz;)Dsn02A3OY$HBAlw{ zk9$HaCqBqsE$yEZkx^g$>{Tsx`_e0pd)`hX3VYXee1b*8TPn*Z?vcS;<(_arZO>cw zynKD>Dm6lV;*0q8K}vZzMI2<jANr9}n5zBC2qOw>-^FCHIvr}|*yPTb&JgQT_g3@5 zHL352rQLr%nrnbo=2PQ;13<C_T=SM2wWKF?IWen!s$m_TS*Pt_s-jrB_tH9ca631A z&OltUZm6OC26c9<Q8@!E!G_j$MIQlJ#FRm5tn3kvj*Qg6<ud+D7mK~!A5^&VR6o8e z;}QS!&s_iTf*fbFx?^D#`=b=Lt(gaQer&sO`*F(=EhV6Zn7#a%y9CB`urD3mt2x9B zl>d~n^D8K;wX#$zNkq6UR;i!RNL~8!{W8DfPw0~*4d04BfBmZ+T&lbOfhBG|%?+*o zFGuM=QT?Llh0*pu8n}~`Q_uL{))0Wd7YI!q;ZoLbqv{Hg`F2fB_`#MNlhf3ue(O+0 z;qS4THc5BD+NFE0Zc=1iRG%K#ol3a`+1F1S-HqT41wOqGl*Dy5ak@o+0jy7)Ru(l< z4!r~ntfu|Yb*>M;MI7x3RYV;!xM2;*PZ)Y6=qPLxiU{@pIJa@9>OFJD@Bgp>bHV9b z#V48ybWm=6-a{17Mfdpl!ARIo>y1Ba=A0xa!SwR7Gyg~>D<eDgP)itC%d}T>e$Q60 z`vE#FKiB0GAn5`Ra3bs<yVPY1P1lbc9Ip9cqJwuw{DjKsyxZ{Jxh2_oBXDSUkvzW~ z5MFfQbINBk$?uV)=>zt(+nU)`s8UA<y1^H)QMH)9WTK#@ekYM+Ss5@Ezh!9L|9o3v z9ri6!jJts1801p<9LiftgH~z?KsICxPEZiUqP=(mbuo-otA-4v=|%CWGMU63Y+D^M z8_0Ho1Bl+tx+HjPi>j*l487JS)ndPYyJ=G;*9r7$y4pi;ud<c1PZdyzfw-?*n`$zq zyuLb^5J6ZaRZ6gStKD!eCh)Hl+p*0~)KWe=eS<&He;=3dZdS`eMxOrVL={kPNI$`_ z+wB#UA&Klr5MGh>W{!cw78oEP(}Sr2sI9Wc4J)Y`)bMd&W4?{M;q5c@gl;~=;-sD! z8m|P+fJZ}J7IQunRPKcV{?u7Zj-(UkzZ>d1D_qg!oc!zbt3n0A0Bbo4{|zlc?gfeW zLA@!ShXxtBs;?^|3MO4joj``eW+@9EC>UU5D=;S+jb-qx<_sy7vXz8%bRU%d1U3t@ zEdIE2L9VRPlZVz4n1KDfx1C#nF-bG7J@I8t%b$51*n|`cF2C42%5<PVcl93d@W7&| z<EoE}foL>mv0`h1O!re0omOk3ZSjg9GDTQGDO+$KpGkQmfLebQXKI!opf~!zR#jQz zTEWN_kAdr{5h;y}_kk$F2v$(xxqqHfI=Wpm|KjSnvw%pV*&B5zI9}o^v5*V0PCjR_ z@Hg-V9sTzuW@D)Za<F>y><r^zg;7c7lx8;_LYk`T57xoUp+&*@dnc^}f;C?Zk50b^ zC=8miuzqeG&nX**^$*8q9>q6#$;!8fq|}}bYp|zu2fCgpvCRzodE`*PrO&5n%xoV) zuicDZcy0su-LN^41@us^YWI|(b01UE{%@x)pJ5c|OA?=7F=)nx0}?MFAlwn*p&{qq ze9BUWr7ysF>a<*JY}R^7)>u!%HtLtij`rgTD%W$HSWJVu^N!isyHw2_H$YEn4l&rN z=DQU)?#rx*`#Eebn6nh6n8`R*@5o^}Dh{4VkRCOa(od>7kv51b!%}N)tj}RyifBw+ zm^hYbryODdgdcjk-EfUGT8!IM*)D%@D-)n%=c{dqnLGqQGqILZ0-e)63}vs<BwMCr zyU)!k+X#>KD0#hHW74D>A^qlLD`CEtYlY}bx`(R%nMDKEN4oLK-JjgF_TVj++L^ME z2`3-!Z%-9_m|mh9?}?;sSm>3g2pYD>-`3`mZ@>KHQ|MemXqu`G8)?VgENO@SLp=f4 z_llSy6Pw1L$oH?ysiut+#247FuXvyv;hlBdPn@j2+?@>j{a|A6Q&!?Pk~B(RqJl7I zi9NB|T1|ILZnt_4I~{mRd+5a%S>k!OOGR5$5OUd6xgpSaxmA-}^NvH@H_|AoP?0Or zM(yMMOpR0oY{AVSi^uBzgEfb~Zq5ON^Fy_+kpuD%O@|%f#KAAVO;wPR#!n^q3hMJ@ zsFmS)`*x>Y2Srvb4Da8;S~*1tWzRR0LM@{7Nhx9#w1mah?m&0wGqtDF`UlC`7_Kpm z^-y9H<g+;fRJMOq7|2(`;Sbwy#_yY)g&I_OHg718m%^mj4KB(%+UuBS1E1y(DtY9Z z)_$QLg_FpO3kD8=G4(su$LPJ6Nt6lj@a4^wOVY374FhD_LLLbI$`hokhVgt5_teR! zLFdXLkHQ}J%_+soF30w13i#Rpu;uw*;jT_*OGx+3_m8|i#LjweX@rV^t~5!mCX*tJ z{R_BAa6oMr&g1ONi8%Yn9rzz;8ye5Yx=xiX6kqPGk&xudWDE%kyN98Q%NkARe%>g! z``fSnl9&e-oM1g9QSppD3f|RQ<jVCwb!<al%6;I4{m4t5UoQ^d-<*0GTgKANUx6P? z&-MaKU(ig#N>}*>koBS!SHVo^zrM3YMV;;*?(^E9*6^Jch<%sA_UNQY$WNsGFG}V< z0l+6VpW~s2(}(+1yJdYE5V%}aJ@q1pg^U^-6hOWLRaWIU-)Kw6>Spg2tt<1Rxp8A- zcZ**yXGGDWZnWp7H3IQw=fyAG<DHpIy2k^K;|$w1ROBLYEIba({l6Tp6WaFyu413K z#1pA6*e@qB<JM3X`Xja`*ANGUeeMkEEsfo&3Y5^+y(rfGd5DW}ls<01Lu~Xq8(N{> zU2117VgAF0ue#v|Mp?$a%=4dyLZJZVkaVzoRW_ueiKo+e(zFjeIU<>KYteOeOz(38 zAi3<BPMT8DtsWnKTT;J|)lUnBEK%00#IF|AyeL7K^9gE`WjnLdG^m5y-j)U!KY0P( zY+iK_QOoD8<6`?ViL8e3kTnZ>mE5xq31^*m#p{O_Y=Y1?125#qqd2(4xk91mg_`2< z_`!)Ncu%Q|gLGn=E{TU}<jKD+({X8{ZF;dO)!;5Yu7HDDTiERk4YFoa+(DZ(4SW=j z93A##(#2NrLng^!#=$!{k4LT?UJ>a_{m6qu0AuT2-*8nFRyh?n7{@B<K;*E?yen0- z|GZoGE*`YI(A}(PY^AFWV`CgfK$6IdQo(8Nb}6J!1&xaWf$}<Hd(Ya75mRu!&j#&n zneDK$tY#QSuKG&0z5ZmQHOLKh7~A;p=TplYkD6I8HNOO+IlA~1Ro3&g*7dKJD@WdE z6*l@1NJ(D^rZc;X%_L>eKv`SHhQP4;62oX-yHPBz%O%dwT+?a9MjW=e=ln3vFLzc; z_DP}t$!TNM5gNc@{2GZ}9lL*N`UcfmaOKF3yb#!v#E8B_B89U*&i>HaXX1oE4+R$n zK5d>*9ABl%fY$Si>O)BS3k$AoV?E<~1*ryQsORCN`9i4%#z_NE%>!619%=9ZnB2At zAjznI&$lX6s@>Dz<hX7iV1Ettneg7pTSL%kapyCeljCn0ks}01E`VD2aMzT9j-{kD zDeU>zcuF#7HRTY6{=A+U9MzH%(oa=^%~JYA70!;zuwf^(gPw%DNtX*N;MsA9*JW~p zvU%6X6_))+URFnKK!?xb2R>(gr2%ZSw`pH+84*Uxw3rAp#vqSWZNd>)60BoCrSo@h zP+v*~+yZ0c2QQ~t>Ea?wa{b?*P*6b3Yn+@@2l7rs;`bh6htx<0<DG*tJySr;OD<@p zRdyV6>4@C*Yk);g$WtI8{kv8CyY#0%l92>CYfnZSG<<EnE4cC3>n)E6rbi0NV8xF- z*#fm~JlN9v=^?wG6sx2)>U%Yqo3S%>liUIc94nez!7?z=tldtJ<X5+Jzu1uqR4kV> zEz=U3Ek91-XnHrEgUAZL6_<fh+f;(}qHI(Wx{D`31yrLh7aVilz*p|X#Q4U=jqNCD z+Ty+copfaV!vQu<9xsw*&znLyuXK<xE31<&8I-LWZ2#_uQY2RgNdRWzImD@~Z!Lj< zrN{d9oE7}+5=mk$p8$VXFjNzSY#XXGKvlA588TDU14`uUJFzKIMFR)t!<2l!(sH!i z&FKV5Nq_X&be-XoT$@ge?90q|PQ^>oC5B^`66-{YXBJ^aY^R@VcPFXwig;my&uCGC z<~KP@UW4m=%m>_7`ux3L&j|`zGZn&)>)aTR3CeW@c@eu20ZRywJxX$E3rHhrQrgiL zcwm+NqC&jKVR=3Kd*|p@`ww#R!ozQjZEx+&Nf)HF5=c_Bcc<82tBlted*+y_!ZI4L zyGuIkeLLh;3iRVOk@4@p=DqhP1LZ?Oqdo}D3v-9i8j@DPeZ1fgE?8ks<O03W2=T1E zj!8>8@k{CUkpbBisgar8yx^f1TC&6Eki{1-Pn<a>wC?3yKIJaxzUllo@HNngU%e*2 zbkzbh9Pg5I4Ol`&)AGW&!?)Y5$?-uA?5yF7!g3Mf-zfDvlZ99J!svS3!^H=a9BYMa zyr#JR2)ap$f}Z%olZH4r-RS8C+Kvo)_^PRRG1GKANy5LLjO!yQIZAAm(Fy~H_*w4& z7u_zNl3Qf3I$WN$eRWKK*)6^*EOxF)KX=%pX?tq4IgqPK3X$@U?~Qv|Zp$GuZzyM2 zDqa4|xmyjq1B_Vx#n=?_#@TN^e}`fSnL%klQfu*Kn`9Vd?kMBF{OnpChEZoCXxO;B z5mEB|7lwTir1O4NHBw&y67R#`Z!9OB4@anQmUX#{v@$g}J{v~ty>QQN4K<Ow0zUsE zGF#}kJ0Rf3p?C-H|8@_<VdsG`zFg}|LkB{jx@Sy{)!f0c=*gVhGI(pzdY?or^{^C* zK=mEv-f2xSHEgsvQkLdn8bziK$23NeWNcs}&&RFiN~7_nfgTZ(Yg(uSSE^xsfn0t~ z1qIpJnV#Ykk%(?VR`1IYVz4UZW1+g&oMnqoqy;IgT|QamyFc4x8m8@Uop9nRkxvBx z6_=%QW_pgFY4LEjfrJL|8z!d@JW0~DDeRMqRQ!zt9=Wnsibsw~v!9&PciP?j1aBaV za>N%m9qi4{yfU|kZGACUhHu@4&!#KRG89s@1MEL#?!^JNg1yC}PuzwFnT*{``?Ys2 zngYbfiu{7(8+~rO?aKhran64_R=5nOhjRj(;X!1P+e3GKsaYlVoxKLSynkT?41u|Z zNGv{&`E1D1!RL1`-~4~SPD=~%<V5;y*2KL3(FJ741J3o$@}m@!OLJEzd#;w|&UT-L zb;B}#nl73MA1?<3m#{#yF-~2;mM11mVp}GuaXEd#<;vx~uFC!k-4q-rO-97F*vuL@ zo?Q&?tqhEV|Emecg6A^w>UdaYyPA{JOK%C0g(6vN#Nv*qP-K@$lp#3eYFv(6%n5&` zxEj7(L^rne5V2~-P;Jl(65t3qXjgl?^&sS&GraN?#Wd2d7%N>)4PzMjf0b%iD~8(F zzt|~cw$%Ba0ybA4(Tu8gVK{wg!U*%Sw2EriW6zem6FY}l6a=c+QQ4e5uu1fscX3ax zeZG%~l<)G(LtJv{9{i3a{W+%*wTZ(;>{@>UmTyw#;V;f48(}w;%y#6Z{(IA3wJwjH zj8=6N&2BC2Q8SVl85Tz%IasfN$1K*eer<Cm+*i7nf(~DtJ#Da)+N{@aA0{*>>S2j$ zPEa1FbvAjQTPY3jmf#na8j7`oLo?BSH?Wm~f88OYbh<)MhBU_ee>_2!OsJyfjmZw% z#LCm72&m1-5bBd{c0G92`6B7Sa;cIT_j32Ea5xf*pO(i!!l*CMmO;Kx@PX#2g}xYk zL%d_CS~(9kw+Pt$z%t4$ucI@A$J{K!IPSR`5ZXmJ*2VgU*9CD=0v{FPaHVG3(zUU( za{)^Q<DR01?To*YeR?zb<nT)}zWZgfyO?l>9&VO0NP4;6pqVyeyj)bf_*F*ytF-d1 zY>+GY_}lGg95WW##zx1Y9NbS;oF{6%x+WZ0?zkWlnI83(pTD2;MULvAkntzogh+3o zQK$FdGo_TFVJx$y@P&`{?}b>mz<LGYbWN1TwD!NtzkUg_GS*o9LIW9G8boa-AN@Nh z1Yv4|$+uhh)Y{<L?oSCqH+_;!?yot%uwmaqavm^aj!M><Oi`Co{p;gA$I}_wV`+nG zuWHh!8;PINYA7e&!;)P-Zt6(+TT_H8eA<?TxR(}0CH3fOOS-6sG7m@An{Vu|UV_Nz zCDg$8as>}>hgLLNov!R>P4zf|tV2n{W%UCwCI<I~yKM!p$Z{A+dTGiHp|EtDKZJ#2 z`MKh-S;dr32Fnjevh09{Y#;}R?X@w4EUIzWS+4k18@~SflfK(-hbgR#UW3Ojxsbv8 zL~sBZC2OTIQc>A4+`-~J<q<e{qb*gldQMQp5v|OxGXj;B%!DfVpW#0Rbew`7E2;L~ z<+~XLBxViASw{TtK6mX90b`Eod4ImY-)nNXDjrAOQ`8?Ej!7-KMOXFX=9E>A^`S_& z_lGKhxWG*L$WD+sUARd$Cz-2CLHH(8NFr=js5UL;J21xc1DtW8X)h7h+w!ZW!B+ZH z3~H*!q|3ukDatq}2;papoW^wfp6QRu>@+iUrba>Fhog)n7}w3@Lr?21ymz}Hls2zT zkTYA#*YpT25ylO~F%T{!pU?wZ8<P97yvv*$=j%O%*;$<eg@4yn1Rx&!9<8W>RcuiV zsh9_-7Pn;=Xz^;S)e=@Nw)V&G*4vvxe-b0O41kbu@}G?Vos9bWB_K^O@_lhfcv~-A zeJ(es!+$KDa>!e7a-x2E@O;)5$W<_ss`Dji8+4m@srw`KDX4*9M(U4wTMvMU7#Gnf zNxvJ2?yo%H{G*oeY9D1rVl4A}U3Ke=X-lPM1M!xow3<ttkm<0k?9TaM&ak6SfeSH3 zlLo%2Y-6?^xr%xTx$iqR@l-ciMrd+!v|sbi0X&rBrMOF|zc>r9HRLDSQq%o`Z8T5U zIu9Q|L{aoN!UfH?kg*76s9)fA*KyB*63QIQ&w7DV=CHuU-yQOz6@!xb){wWwzEI1z zFeKcdALqg!5%DX;{ozypUM}LAB-qfzrh%d99LpoO&Ge$a&D?E>ne(J+$r9W)wdBbW zxfqqcl{XJ;>uq<RjsK4U<Gw(WQp&gGEg)X#$+xSZ$6`|!7~$}j+wSU)CQf49nR%V( zBU`pUvxi^_k*}43?bV<s&3Q9VB*{m4C!}w#pe9n)xexDKPw2XF)=aiVsN)&7V~<ks zsC=jexIK3P_JM{Stqdbf-_fPH(nx%++LeI2iSDP+WRJXC-}lZlJCN=N##qzg=4#;l z@`_NO9f-r0Z{V?)b8DEA>^Z;)-?EWbdZnqkF&~AyGO7Ty?QpIPO>XJrJl(nxaI|U> z4~%VBnkM($SRCX1p={6Fi1P*r2c=^pi`cb!Q&g=xdL11bJec0w8=Zvd{`~2NaM6;0 zo?z(XAO{X}j(^$tdI$7bQyIAX`5j<57uVxV^}k)(B@|<qaU(VL;RXye#ipo%Icx_R z--*gk=7V2ab4x!p)8prSU05!eB6=mcvm?iD#?9!P&{@ouRod&_A+ySvIENn^3KJz% z1A4|CFmoU(M-Vt|b<~#^(_%jTIc-rl@=gCQr|rEzr!8FvIQ4pw03-uW#YwBoCfJH3 z<`O{zng|@-$hU+KZp|(&L$5~#f(<Ky><3tDqu7Vm*wX%X>ZB_)=1uf*$@3REHABO@ zfnhxu&?TxYZ)Gbyqo`w<>ec_evq-w-y~H6U#QKcm&q^TrBh2Z<I!*U6t9V&ScId<d z#Yey<S11P&R6$D=_m^;9iQ>rh|Hy;iIy#N<`^M!a`opWuPLQA87N&SS`}OlZu$SDd z=;*^J%ff_rU!}MnDu=vd4N9TH!rQkAveMTTG4_;93n|2!ZpQbc+X4bNw-y~kOg7`% z;k-c7-`Fsh2^r%u_^X!S;aPaM0qt4{Dbxh?FXbI^(tAF+4W5%dHO(w(#iA-XMlw_$ zwD_=hM9Ybt>;N3FEeSbKD=A2DF}wHK{ZCy-!1wMp^qWY71wXW*W3B0dDj|-=VJnH& zVl!2H7b+)g%QBs$FmyCn&T_fiArv!_JpK>1map#+=lYYT^D745&-;uj`2iQ5YG3et z-lGja7TPtAaa|H25}2H6gN$X7U3D%&n5B)VF-^i}8QvY?+qLo5=x~eZ)_-(cbID)V ze$?Ea5}NYLs$0v!xDD1vu3Xw};B&$kG#6@~Tii$<UmU$Qq0I&Hm!32@eQ;Ec{hxd9 z&~{E%$ez>Ce=f^^BG>Y&=Bj90fC$vU`1+FlLSJKDN`!?8qCCZ9{kcoe0sx8~=y3oM z*XI8agcdciJ`NLm9|}s+FTcF0lOA=4FNoXGo8zs;`G-e#Bmi2fZtzh>g63Z)%@^`j zxN~MU&WpC4G}*_K?ARVnM3OU{Wfi2v&8%#PAt)VhOWU!Rk8ggK(WY{ayF-<T+8@~` zqZ}372BD^G(XBZ@T)R?1%yQh`QKWbeu?%axZ5{FJvJQROb37M$mLek!jhZVz$S6%2 zX;0X`V}6$Q*v`u|KxSBwU;EGCevsbVyHURtK<<*s#3;DTf%bXiiZ^aLoir5Fknn!9 zZ8B-eeaS<;tLs~(m3`DNg-Yx!93HUifp63!tKnL+4i>Uh&SizI<C@n%QyvC%Ycg;h z?p-mo_ig4OAJG~uq1PPl6dk(lzK4X5mW}3TT|=8r63TA6<narSPrp`nw6T}88i;<P zF%*v>t_wZUr<*5R?v2gc_8k1sKQ&kfn&C{Ui^`MLP6D$2!{lNKy7&KAM=deRQcgg^ z6kwB`3pwO$RJ~1(cXgb&fOJ!TVMr7+)O#!;4(?}8oGXThW{YWf5RS^y-MU0}2Dxtc zh|yJp7?CO-S;q}T-o|;U3EX+X#sH05WtG;%8x<61J<de^gLU)=l_zHRA~q*w10tu- z_%PewvbBJMa3VF1vPmqq99=C7E`_4^Y$OLu8i3D@`4?DHg+taP$piA!D-RzT`B)qQ zvSFYuXh6}ciESK+zJ-v~1049nC5uu|gL=R~M+aGMu1%2H(z6>oiy+>2{Y`3!=CMKq zx`Hm{>mNwX-*s5soZZig7-qVOqzQ`3XNQ_|y(P)ckLN2$1@DYU^0cC-@CM;h&g<S& zj~}BclPxxXT`a&FpU$i#T7ZShsqL92h1K1|T3gI-V~ua<eT|!j*4(!U=JsI(=Q!rC zli-9xYTFr0Jxe8^@mJ_^<x~wH#6=Pr0j*X2kdU%rJQt%GB!F2=pBjPy*<eb9Msf&1 z4cKi}ED^)iD{y`~wKxnwg~1*zM`5}P2nXxKElCA@Mbj**8?VsmX~Yvb_2}Ps@cTb6 zf5wev3a}p>>K$wPz@X-t)E~8GW-EVv@Ab{_%c2S+7wSCfc0^~SY}>%~=FVW*NV)iy z8{}I-@4B=DHL~<6yS1)@_f#kn&hUHllHtrRcfH+ySLPJ!B)dW3qz#6duy67q&}6%+ zp<%oUJ(7{E=x{g^zwdsw_}=3}dp@bG2fQ;+LN0)G1ityQcp&3PuI67B$@WNGD(xM+ z!F&4xWz~&)*v(PvPW$?ZMOOKuGq8ZT_7gO!At%K=B}jCSs!O*2dN)W(fAA8*ypmE_ z9jlt#+0cn>Kgv^LIyp_i7WboMq0byN(w#=TtpFGHbpuaeFQxtHTJzoI>4X(e|I=NV z+AiXu1}vZ1Su~=dX2w#PKI@{E>q5n8$~jhQO;x?cj;3GDEUsV*<cj}g?9E*Ao0#<F ze+b&&$FG{w<F5@evFq7^sZM6syMA1Zz(>D9x;{U__IPh`Ap-CaY<W5yFkg|i4r#~9 z@X$WM7uvMl!1(=UZRZRW{e2w=1Q-u&m}7ppBtJMnJqR$Ey<m?1vHlVt<~raH31w^~ zL&AtyQh>eU$R$KuTCR27Bl8ru(lT=o;-qYeH_(MwSGmnS@Mt%t-h^%QqYRIyKHUN> z0pEih9g<Z=9$*}2VT3VEyP+WSe=EHEPhXbt|2${epG6!Rm9U1lKMZ`5_3crFhnBo9 zCjdUNQnw0erb{K|j<{?AHD-j`?4qx)beOycSck@gXx9u*cky{Etd<5!)bX8OZw`Fw zkSmw!rOkCmJuH6u7-9U!jWb(7o=}4~*%N|`zarp#-2}c80lh1U#4UgKF!&jA5Gq%! zMKRdX{^junBN{$aChOaKG6Edl539Pg8Cky69ER#L-lxQn<i(+zDi`9USGMJ{&+8{J z<7uhr%YebJ%A(uL2Y5t|bk^H&!P*Jn5qDxK!G1fe`67AE*>$PHo(z`Pb*AYvv&_)@ zTDQN;?kt;*x=881*z?`YiAg?q8Q4%Y{-a&y3w}EeQoYsgvr|oom@Y`H1^(qldVnBq z{%DQbf3W(O4I(v;Q3|oI`YTVQ6foT=q(H=KF3(ANrg$@)TiOOGQ^cl*SgoqL2N8w= zQbz*S3ak7*LMk6eSi2VzEGPdlziTHY@4|31UtXS+n3!qwWloNrIlnl<uRN7m$p<Ni zUK=n*$-L;aL;gREoo86n>$dGfX9-f4N|CBqP)aBwoq$qYOBAsnMFL0(5~@fI9cj{+ zN-rv+C<KJi34|V`1r$P0=s_TK5+EeGfA=~2KKIKx`+nhhfR8XU-#O<Pzd;;lo!3kO zj$-v2YmC8Nbsm+aov(5o)s3I36)oo0=a6*JpiHSJxLbyMEpi3WI-FS;;mb=DdAzrK zD;KXNK6<W-vksrP!myp%Zu!|0HhGkS&G`nVQVe&F-}@Fe<p(`p1>(-8{GW6#J(H&B zEIB}$$tpm+63}F25GVWUWQfXWP2~p`tDVh-$BgqB4j_(W%2~1l*8%v;lB$e3hup9K z#RH7ASJ>JIBW`JYDpJaV`MSvCV^fQU6l=v4I(DUbQfih3Qw_Q=aYz?4c)4Mbb}|Ak zLb07KF2^%FR%EYk(nraPLUR$tDH47>)x{%I3$|{__Q8<Yr(Zp>{e$EOO*;6M!zl7g zcA1rExMnbwC+fRLH!ZVgI@5~);=ib^7kc}rmMT3v!Qd3iQS0AY0Qh)CXj|iXW#SJ_ ztxPoO{1R-^D7<E#-ebfnnA@DC%_DWYIhjZ1gg%tS)8Pi%@t<bbe?U;JLdDA4@eECu z5nm6<!tnGdGezcnc;NER;ZzH{RPG=0K|2q4dYgeNq^~d0H^Z6bu<)hu(BtO>T;62= zpT}gca6}MrRB8#Kuh|hFF&3Dl3V9-vpXt9w7mV;jy8m<)Wvi4CP`S|#3Ihj(sVtl6 z+E09iEWR<3wB{ica4e3xbO63)05ehlLKoqEI<27s(wK4_80!6|<bN&V^~e8R#2-6& zBXok59_@?jiubk}UtlY#*0koszUJpQk=6s&1Zh=VKhAaf?vj}HV&hx03-O=uZA{7d zC4;L!)HTy2I9Oo%f4_Vz&>|ofOi^W@cNSl)yrHXUV^?vftTgkx-2KeSs4do`7Ig~V zAgog1xi@ilCcgQ8h93p=1=>Fn{B6^m_6QN87px(Pz2kUmgp=}^e}3iJFM$56nG~b~ z5e!c4L=rA?q+0@nI7EwFXl{b00=eGK0wTkRUJ_7AI703Xi_uIaR=B3U*tKkYzi~io zd*{%jHRu1VXx7L}meQb}F&;S-8{Mje_%2^RmYE1&A{wZ!Nv1lCcp>*?R7lO+@dss( zGN|#akoR7}*R4Q1rwRfFDIBc4P|9)pt1h>Sx5H-~@IoOvJoeA03MA1&!4rvU*XtBp zzA2h-FB|h@?b>V1JhtcUC@cP<=kmBlQDwINJI>qmmo2EsF&QqJQK|1a7*clqS;I<< zp~dloa1Yy0?V*(J8)IB%Z0^M9$IX_5j{P*tQMD6KgSU&%nE6P|@>VrdCc=Uw#~y$^ zm1N5pUx^aln@M4<EvbklKyWc8-CX?#nX1Tct}jLM^C~R!vubH=`RpsQ=1%5EZzhWi zqmy=_kl-!l_L1z{Jk@sifUyGHFOmi!WR7i92HpHMK#(2!WcKhIKIq>NW-*lHPKgxs zU!j(pGxZEB#3wjpuVu*(7f7a<OsGC7!UQC3sG&oNJuR1g8&cFOyf+_%cI=PVp+nZY zzU*nXu);`l0mkcK6@h&JYq`yDB69_^JUO}BS)OJoOfo0dE1!1Ab6-0VHE;vxPLnk( zT~M0jzXJIwJP;-~D@M7(w&P5{%rkA(_L79)3<GY!4{)-9eL$(&T9GR~UXn~y>pyH7 zEp*2hRvPolz=XvmNyEym&;K#7-iu#R@||d1&$C<j@iO~RoyQEtTp<=I%dyg}ca4dC z)qP>FtPR(*tah8ng_46pCT_V5R#Vhv5QU;r1v9?0$G*$KEIktu?hr559*hs+d3P#v zuWEX&y3+Xc_?d&iOE&A)5uf&+3Y{(PZxn7>JnVb48CdZJw-4aRb=DpDrx@$&C>)fr z#n1E}Lp{cj)%t!h54bJYL7m|{m&2=cHhfZ5rw&mN8fr<Kv611-w%iLBx{;wJowzYR z`(XK5<zLXJj(9yz4DhJV?{C#U>fDEE0T1hOyu38nPw*jq*eul1_f%fXp42-P+v!x? zQ=QjD6+S6FH1Z}T4ox3tRyCZy$Kj;70zqe9{hx<pps%-vJ^ZgHH~-lr)}FxD7wjbl zSt>crmV4qcLB@S^A}NyHHLI@9eeEQ{`Skwi=iava07*wsdPvKVz*ebq%ZMLPZgK%< zrSZqZaVS6|*eiK3Kam<d*owus?X4V?f@ed984Kv$Zi4n{u%BUDC_sw}vQbWXpD(Y| zL+&L?hL9&BuUpzDyy;7VbU+kNisl&4%AYU@o_*ppXtibORc@2Bh7h_njACn(WQU%I z8}|hu$UA5QW)Bcf&}kp3BOY$OeN}7IddR7o?}=&yaX6_FI=unxq<x1gQQb<YWhI>p z;jo3cJqZ;5iHP6a?c)@``ZH`nyfR_9eIxun0qV=i9sCQ!DE`;+^wS@@)FIS&Egt`W z11Y~?qf%`u6Q(-qpE|!-(k=EAj7=Jl59#+nw0N{teuxI5Jf9ZUv|OEy-|lI0kD>bU z{;*z)tK(GNay^X8l&G%Zw5i0ih5?M8#utWI+2C{O!p!qD(UGoKBthmbg;<g++*f%g z_JOiX^)eJL@k7fRNE~(f^=m+^wvd!W(mn=4YwgSty59L<|8`QBP<{nE{BcvgRK3?& zU=b~#&wpQd;A)ayH^MCCaD5Do4@4-#!!K{vDC*yv^3g%JIMR7jY{C}0f+xJZ<=4mt zL;RK)0yK-te<UGr?m4m}am?blO|yvTsK7KKGXI@2ko$pEx=*ij$C;q|J0X;htp})i zOTEIw<T%=p*k$-;^yGd>=alyMgT&<z@6kZCDmUmk=Sv4%YgN&f&vnEVHuWNkYNt3y zf@QQ|+Ve}VKh&m5-(#P2Y%(#ZRRevL*H|{X5ERa-SMaNEGOQ%jy={ELbS}d)W7Ap1 zwRYL&bnMM73wLpjA(+0_GhvcvH(zY3$+8s9<6*RW-p=fT=+-H^^EC#5$}IA^O=_3j zORjWrLTh>!F@U14BG`+IjM}bcCCtF}Y2%p_iMLf+;K}AM6bGG}JiA7c==cL8$CMh9 zNV1d&RooQ~7+zy9GqQQ43mNZplwJ*-|3({_ssnwOk7y*Pz7|cYDVRvY41I<iyk<Vt z1bjrM?=L!THxcO@|GF(Pjw+bJnt)+pfb_336LVIw5YF0(IY`9Rqv>QX;v?uP`e?PZ zj$l`6H`%+QK|W%5J59StJF#Na?-;f@Y&SqTT}l}%dse4zKugPICPVi4F_k)^p2KpW z!;SFxL_ROw$N%Hx{C~M}-tK1v>MfzH%R9Ff--Qe(LaT$ckESchcQE^cpltpCIny6r z9&3N-yvxzd>g5_MQo`|u6osDEO-jrOly>7OFal*WXE^3!B?VqK7Tsy?XqNo=0g#a> zP4M2jLUFUxgg^cms(VY3D_!AU*AEPbg2|*4CV`a)FJ8qrTyzrp>2io2D%_1e+~9Z^ zzNlZvk?j;38!E&S6;HRwd(~PrUd|==T~$F$V@<-vEb&H~s&6;jd1I{Nyw`e@5NPt} zK#qFR79?8WegLOlcm0C%ZTCnSAt%voo6!h|-1LSuWKH^hyN&93$CXG(_V}*><}C@K z>bM0#Q7+_774EAqKL~hq#=4e0Bj<it0O+jb733ir*0A``?K%{=mq7&v?;{rpk5R{b zj#I9G&t3_#Qpy+3wPYK-(Yqaq#YaCFPPz5OJ{fop*D&8SqRUps?Z6bH2%BdeQ=41I zBlO7CS)^1+<ZAy`X^qq~;MPPI&3QQUEvdM|4etCrgAUzmdMS(rNVIG1j*-^EXT53n zW8Zi2hV;|^ta)Dp-ctPMTSz2{d$VoivP>v;|M8wsa%TqAYEwJ0Cd`_+CQsBq$ihNN zuIzf<q4iXNM%6g@v{08D7~y^UJIlK8EhXyAZCX;d`LQDIthK)h5bsya(X5763kZh4 zTM~9l+>-zs+$(gsvlzPhLuRCiOR5iL*RYhBn|N<A22}M($U}-F4aumZh#ZM-HXyVR zeotxXoAOWiqmCztpFb+Xf4A^3t~3}(FE*j`wsB;JY=4!IDa=4ul-m>r#9=gC57GJ1 z^u{1fp%n8bHKX#pwUGGkhk1b3DkMANf9Bm;mbQ;(>yT#o0#u`wzwvvA%7EX{|3ZXg zp@g>AhB0}Z=RjlfdOKi8X|h0f0{LYPfTOUmr<=7_ZXsI6=Qdd?re&CFvVc?%R|%D} z^7396$@(3U=QwnBw|l3fWOLw&$Xv3yAtswg%6i-1e;}b-d{o>K=h}Y~dSDKEzF6wa z6g{OM))&(!%G{H*iGkTae5qYro-SWM)l$PHP&JZ2a8MfB2t@f@&S&QS7F|Al6+#Fi zDO7~*mu3buuh~v<*>ANK9wZc65{~e7p3O!RaJAh%7=$*l1o(}vt%Wmo3o+bzX!?x^ zzfX47bN%&2T^anhT-=9skE*2Scr~UGwbY=#L92tQA8s*0)ZyH6tAgWo;zW7#?`2rg zPh^4If6N;QeDJTftW7xjOmSjydTABR6@k=Cm6WfMjuxsf^+?UQIdv%Q%bDY%En-OC zZv*zb?;KBm*ydZIRHYh8%l~2m1-qo{dj5HR5PQhP?E?+g@^_o>{UuxbCb(+QTploz z!p#fKY%@Bf&H}3g5w(Pa{D6pp$V+gB@>X};YF9?a$@PkzvKv=Lxqq<UQw@blpI6x+ zj63x29uFjDbGti{Gq)3xL@U%ZVowy+%yXn_c1wV5K8$5%_x)KdC0QmkjI*W*%zrzY zv?*2Ht>7R#<L{Oh-0CL<7EgdxmATb|Cmw*U7u{c@+4c284vcPl&1Aq#j`2A+vz6<* zm2=hpp_6|50s++yrxQzpv#ndCL5NTiN=nY~i$_r9_r$KV(SW6cqy1NbA^qH+!j_Md zej)c;C&PPpfz4fFs73yLDQd?Q&DtLQ2H12VEr0aJ_J^jH6@9G$qK2?Y(?Nh(5gCr$ zHW{#9ORa7I0tB|crufNa$S5UTzZE{rJTLsk(&wU9Ky$-a+6}Ket)Xvp7MyoH$=cQo z7qTE?_l$w71Y2caE~1`luySojC#pxXwbbhEk`+$4zEBH--WOPNpugxpxFPd|=e#JY zSP}|3Ks@eCA>_E)Hb%nl_K|GD9LPk!<L281*1>^vd;8-_{$?-j*Ojb2fNc6&ca04r zW8X%95}%0iH3md7VU69!Exz9rP#9&OX}CZBU}%b#e-Bi(5(UvS%D*f}nZ!X3znc*V zWt-j<wOPl=aP}LgG%oT~J5AZ~e9Dbc@{160N!%kw$9~8@T})2*pZFr+l*szTAh2W& zGm$g~jZw|MHwF@XRA(+A&tryLb(zSn3_S@<*D$a3$^!2Ash#jcd2O!6ph^Si1O;}j zr}eA&0~N=apSDL8m9k$Yuy+(PfBn2Qos%$TF>x!eZZFq103T1|?ss`H#z^Xx-ZMJf zOb?3AaS08&#8x?G0Ti*aoeUw^2doVXLJoB65b9n`f<X;Ui6<bfM61QyQ=+N$*qQ7g zRI6+)nls|-K9U2WQkEOzk@|)?bQx27JdGmEt{#`cbm0v^h+fn2Y6WAp97Z@C3{XrB z9x3u~G0P4G*Uv^R#fo_<tzkAA*D|CI)1@X3MWgz6{t_x*-m$XPxd@ts-qmyNtLXf^ z5wwbDnm{e%+WSPk>#6R*5-l0kXtQfX$ju561Mo#1Q)<Ey;<R(%`B(d<|FdfA-%sw= zO7s6J<Lk{^HUpp@6xU6YzEqb##TDY*d?aKh0?VRDn%PF&qB*1;L%FX9bHyl>+*{q0 z;4@CUTVUdP&JFWMjN?kNJJ0k=a@PurzZ|N1{Le_@i|I1u`vZsxnPFz|P)sJBkKZ9R zC3k!-5)C!n23fFQJg+>oI}uL|y6$@k0&l5~P_NNuJ6c2ONH&LA0{d(Tj)Wj@Tgq%j z?e_1gi$CBGO!$rKO+9P>2?LFNG%~h&nXNK|8JKy|@TmN3`AI3qJTDeF(hodOTfb>( z*BQ`E5zI>L4*Aj-g*m_Q?(8pT!GD78sRETc+Yz^^$1~TZ!Wn<QwMs3ns<Bdc{dTij z;TZT(?A)cu!r$|xn1_w9`E0h<%T$HtV32HdrB*HP%5!3w+RM4Vj22|#1A!3A>Nk61 z&QHZ)Xa*GCT(#;ni_eL3mD#u=a-;L0uT>64sCoJac3@an1Sd5U4w&Eg-PXHQy_oN5 z`d<P%y@0G?WgHJY&qwOA-l@n`(TBznRiiDWn;|~>W^7*NZs7=cmCof|YjFC|b9~-H zXxvef8Xhk>eALq87vFq9e^+A$BqI#tG;b@j^=k<!*sVPK!TKI)S-dd8I*vSOXAD#Z zLx2HidXr4G2FQ1w_UjLbnfrUchvV9$-#b9t%y*W`H}RfykH1ftfSe>S>x5`9Az?$! zn$|KL`oO6WMmEy;bRHMn&en5&d^_x5j6F@YRSv-sr3_rMM<gPx_*g^Aa#TZJi}G+r z8r#<k#`DkB4X?+k;EO<FH0iap%4eL5j*$K&31FT5-X&7o3_Nn>0j#ueIJhdb)<PzT z9~;)mG$HGs97%Df6?vrgOT4^O<hb}uann%(;L+fp4EAe_Yb#_{^3?RPzESm+q$^C7 zuRa?n9B#RtFXJz*M2i70f=_-Q?+g3bBD<jX+$!OWb`cO@ivqG2Yg(YwKr0VS-hS<~ zGU3^kwm7W-YB&?`Tc{j?rbpA=Eo{{~qSBU7f~|fExh{ZlYKb`tC)6DeYJ1)NBkbtc zXYgaY&>Gw2TTBL>#w=Vi5|;R$W@@H)(8dq-L~k26p~wAz9k|t76EzC~>t2|FgV`Uk z*wGf=3(0=2nmnc{<MuVVU;5nG{QCA1*Q{BpNe6*wCTDt%KH!EY?sViIcN87BoU7>J z4}wpa@BeQb?En7VdOqv(_y_j3YZoy*eHh>Ga7<*6P{caD7KrRd3${krXLr0;71)+% zzYyBiF`~-lhU~Wz^l}DZlTvlkIrurKjwMq1m1G1?+*Fd|iczi*`wy{1p@Sb)n77Ie zT$o>hlv&3h#rwi=>~1&Py)8O`J=KOGS|5MJ>1=S@8qUYURHs}fs>Ze|GksDPB1lwS zx=7wgW$MauO?J#<-Pwyz0sCNLw>ba1-Fc3>1jbm@j5VWULG+h4Hl=2@wYH{07hyb> zWP#o|RHS5*z5!P!IE=q`b>C)nvDTUIkAv4NX$tiURrobk*PF(5Q1OJl!i9=A2Ju&d zHw>t6bCBk%X=A@79}&6D<DD&T&n72{RxR=H1h=JMQ@L#Q;G|C0WIIU#f;B?q{G%yB z@9Oz#J(z7-L`zhdgR@s1e%1;=Qs89Z@f=fs6)MRoV<{^E2}0LOTBf206<nd~j`1Un zol#RT%v+V6q|8WBDod6q@XW8Be++zS;o7SzxSb0)au#ZjSO?G0TTaEKinbcrc=LXf zA~4wYcPS57Fz%4G3ccZ-7<_QalCpmg(MH3fy5)hlX*Ud6IYcenw4Jv@`t0FH@Wqdi zYC}6bA*s|&4V*xK-TW74xYtbU!C-dMA@3*QsRL?0;b>qB)ty9O#2BovK%>{X5_3|{ zgd|nwvp8v<`ij{$0vG*yHby)C*aWnVa>7l-wx-l_=Jgb62gY^RB|@>@o=5WaV|Iq~ zIgU$z9+*H`PdPitzBE6t#M5De8oE5G&AODN|571W@U*)o*|ggZ8NhyFnUQjkR1Nu1 z^O@_$h`GQ?kD*{026pm2(-6x?OW&6Y)%$7Y?j;lk%pPIJ*lR%JorXCKiE3Zl_#;Ga z-)Vu+f1JEp)_BhS>Sl6ldj{_BLLdNvQe;|aJfCr9cd>cB{>vt;?|qm~xG_!dDFh3p zYyT`vYSsxD6+WXkSl#F6?D*a};F=2}G{sf<l&{TxMFTa#GtER%ejDF3JHE-92F8SF zF7&Gw361%_i{O#&Q$<Zg)kwr@Uzzf9)5ljm<KRB@^27uO9W(cG`|OUFn8F+FO@#gO zCE)J#k#>9rbGW+XT+uj9Ki&tZ>dyx?N!3#~Mf}<Kx_5s3xFwMhV_o3_un>Aik|Sy| zP_0g865_t4Gp4o{VwQS8!R0(;@t&7YKs!R-fEr6fel+^uo%H_>xc^E!_<z1z*XH0m z8!yh0w%v-@yLXjOG3sP?ba9b@vXnTbimguEyV|3a_@iY+nB!M9-!b*!_u!X92K?Nc z2G^iOg|uxz-YRr$M0*<0$JX`kN5Dq^c`vMUPlDsuM@7A%ncj!7yMNFkL`Q;itXk5t zQIa8fzfgcHu<0=nYo4KS@TexCl}E=+{@&L!%=6OU)GUDDm@5pKik^DkTp?Kx7MQyU zyEwdSN=+rz24oB0c$)h(chn4z{fZN-GeuKZ<CD%|r6dtN)s*gb*1%vOKsE(0m7ITs z_B%BC1mrONHHlph|B#(T?89MP#q>8=bWk+Dq;RVlZyl7+=Ou{C{f5CoTk2b{<GoyU zvlIHoTb0M;w%7HXE7NmoEbsr8>o%HO09vZ%P;_-!;8z+1;9eqixkcx#oqoCQ+tQv5 znDg8<)}OYHMkexVcv9@UZ!OzAvjJl&74P3uK$E<~%Mf1))<CppaY{OmsR*cc)wKo; za7rA*EEp*ppV5BYpV7jpqU;&XcU^R!8K5Th*1ES@R09xe?^fRvc@CQhOdNrxPli6) zZ-Jb?p<iXF{0)Krb#BeDK*r@m#!|!NUyJP${>kUfzn_pfSsSFVx~jYnHQ~i6|J0xl zn<^xkcT`1AKJq&8qxD9~o4vCV1CBTDD;Yj~I24KvA?Vs+XicOi@WGle8hBSUx-G1b zREN)5Yg*2M>uxsb!Bs+%;3t@3RNtzwcM$6>jibo0%Z8A#Aaac5bYW2C8<H0M1H)nB zY1h?JWk5v236rx}L%&gWBDzQfBS2++Kgu5gQ3T%8@vT?|fyTNx!vo7PT%Fwxp6q#! z%gxsEU06O#+AvH7k$RIwk3zw6NdGj|S^+Iy)_tbKbI#>ke#};sQm;S$35%Txi?|%k zoBhIkiRKBV;Gt7dQ78NmrPDdmY1M)(31dC=@*d7@%4TdqQhxE2n+3Tn`U;r7q<qTd zHON7T6Iy=8W!7XQ*MYop{GHOEMp0E-Ns@r;zj15Y1D~aNZy5G%)f-0MHlx<8b{syZ zk$vJ3z+NqTcZFt7Rye777hpdFtP<^~T(zJ5TvoFaW>kI4YbeG~-xvE{a{&yDPpPi` z<iprWsb7!^O&MX!{tZoU#>FY(!hD{Dwzb^N&}G>n_O#-D>sIj1o-EPB6_dqom_qJ# zT1y`$LpcycL2nm%JsH%ZVqQuw`r>p0g0>Uw)+ze{!F_+{)njx!{YjM+27uo|H-c~o z%6?9a!`O%jvdagv_DO+^`liUXuKO)()Qx4!a_rWn1*@4+3(mz;wbQn$oxYrleTq%) zb4l#<ga>B-ADvv>^o00T>E{5CT{L&N?7D^Vhj@vQh_DNZv#wLFQmg_e*j+r@FzVd> zMR6s-2ss{Ubu>~=IOnPNvW#o-OkTny1W=FUb60qCA`9r;SS3SgKWbI^Kr1{@;K_>{ zn9N$6p)*Ixqree5n@b9g*kKm9;PZO>t$1<_xMh5^Bv|w-Z=Y-J;b09EfEK1k1#*7~ zrf<93ix|9D48y4Nse-Of7P4SvGEY0rIdXcdIKtZe&6y4YA4oLuo5-->?4`29-fzPI zTi_F4(&0LMatwM5$Uh5C^5zdX*}??m|5Vj=RX{w81!R)0XIawrqlDN^5!Q9{o+Bir z)(7QOAUx-tj_L{NzVSa^lx<aVy{eLc$}iWu4S0e57wW4iEIU?kzCD288p<z?ubFBJ zG$21D-UzR4od%}H82(~1;O<h6T)5Qn1kFz-w~a`lZzB%=0dag%UiNXv`oOzX;kdFe ze4}a1dv~52V|3F#=C@F&nf0vPM(%`Hpu7zoh^~Mh%6ER>)dM`mEzyj3yX}c|i{<Bi z>RS!6A?}e`f<SbRFqLfKZgOCWA#MeF^R{au{jBvd_yv{)Bu}zPps`)u<OsDa@kA#d z+VFE->O$i<QD$iZT&7W;^M0V<5;Djslz-(gOpZ+@Rd_?&n=B-OE^98rY>Ldl1AQ@d zWA5Yt<6gH=Erz(=3H?i5{92&HxbQ%^L}&g$XIxH9f^k)*TkZ*nG0XW{TU6%Xx;v|p zdib)lF%vqus-f8&k%8;yv%^hjQ#-l58qUdWw;<I8=Aah}2O?|f;nWEE5r_&fo2)xp zMT84=+5kKiufW4d(3|Wr)Abu5-wO-clZs5qNJx=$AX-KF)0y3o$0loHG4`<Krajn; z!PIg===#s?zA|l{fZ**9XTW|fPT-T-4Il@XK_9!S%Ap^cUz>$xik3_wccJc**@zMw ze{$D$Vz-pII)&&&<B*04X)q@gBjk<7qRd4%(7T_U8x|&=MM2@}vtJ`V9`)WRll_BY z=B~L4NKK<Ea+%nsOjP0aZr@2M+L&ho3a5uSY$&8PEQL{<h!oVA9nh?|yM)1ypZFM| zj*By)jXIRdRB~-4y;YvK315%u;x3X3YtP_!!s1xRRkLdlppZ|B72!K!wRVUy4^R)+ zIbgR{ry>oE$NXpZo&Mu&%+<uor5%s1V_D=zyBva~TvPPj@AFH=ORxkP^DjJwl-=x= zshuT0ARedJxq~L-KV_FWy47(=^}&Kyblm{Mf--Dql_d`~+s7LsuXS$k!_YncvkpWP z*GGCk`|WeS^ot=v?0T;pC_R_ep0{&Vzo~6LltfDb#Zhb-QY?oqfXwr)Jok4LOXyvX z3>aXdwb=xIR&Cd9>%MAz^hN34T7ZEwHIKl?`c2>5JB6)9xm6wGCb&lnva^2vu=fIV z@!@!iXoTnjKh*~N4VFrZMk+5>VuQrC4d4TvYM&niR)`aRMhjd=n%#H5FsWhlb;6o# z=fHAoRb2UQ{V5whs+oF*3%AT5aw)qx=^Sapsi*Za{24py#5$c-^ybX^Wi<T>kGOMM zCT-ckF+0={&Syx85ppj@u#8T!IU`A@RYs}^lV@3$Kl1gLec5hhTa}N04OaPv=VJdF zB(-{2RO3~rNS-?q=y~`q*Tan^c!h~Cuu}V}`}WMuAmT*ytnSIOKH(vl&g{bi2ci?E zEsR1@S%LFzm@Jq3nMW+qTu<4By$eRr{tU$wi%z-OO16f@33F|a*HLyvnw;rF;}u6i z@T&3$8WlE2rFRtUT15bFgby)-W`UQ$M*G!BkMwkz>zE0$c7;A$)aGTjfatZ42|GM3 zi=ee3`7r_%Yjg9(`qmHGf~E|q9#zSRe&te*p}%#7BE+r6%IL-v?YmyMA7YnFb+FKi zWIPQ`YK3Xe@iJQaqZ>@zUEgZDlLqs($I7qWmja&WDqIo~I{*CTc`{2jxA}!amgi~i z8*w5WS70o6D;+!pWKW<5Z{VIHgv8-@+C@ctK7sT6R8D-bV#o-*cRgPrnf5jxT$ayK z=3F&L38~{UEN8jrRtQ*gs*Zedo+CcV^>KbqlN&4vIs|yv?J{AyGkZkm!CRoJQ5CrU zL(%Zv5FrEqX6~Xhn|nW!dYm~dbc=@S060E8R-I%FvgqTN*$1S+KR%t2DPpwZj7Z*_ z4X(6xuF1^o?L=44L#aVbBt>d33+q1CSkYo>*up06j(Z7m`gOXaL}uvQ$bv!7j+MZ- z={t`7_Bw2c=gfa<r-K&|8NP$`n584;X5mdm@86^K<vj29=cA7vD@Xx;h(_l@UDEfy zL_7b#s13T3_Vw0BZ-bC(8`GDj)c*M+8|7A1Xbec5QvC<@dnU^Ik;;EXEph;5oR>Yn z@BSalIPM=-F1dKL`oGKg5^#+R*@CbUOC*Zod@7}cc(tA;j(q2`m&vJBISBGiVFAVX z%eNM(c>zihouRVq{%T}=m?{u1h8r^ec&En81Ar{1a`iTM;1BfHgn(BNF<Fw6s!zSO ze81E-h<~n4^4b+#5$@OSu;madVMGtd$)jg{i_f6~uhHKRMEbd4@3JJU%!LgKm2Cax zvhK1h6)muOlQ-BqY3z`^|9b8czj&L^V5eM=(3H!#+y^P@_vq;0%KqK9f|JnhK_~(p zYe}ZA+8<aJWGdf0D1||~`Nd71P@#c?Mv4*BtPoY&6abVhP4!6VX89r<n(ugK!$2N1 zmAJrqCx5Re&Xa){Z4d@u*4>%9E}GtKInHW=*ixn%I7S>pa?+dBxCHo8Kp)d7%KoKv z5r};J!!hI9$Tk=~kNV@Rulzw)&3&5}=<}J^E!WJ!m#T=bJc83X2kp>KhP{>%*Jx=R z*KLH^s`2)tU19;#0lg@EjJ#xg@3sv+R<*P?;sK$lPix8fxo6i+FA3}+`7JE|wL|ci zjzkk60%MbQpmHWj6;IXbQx7ay7)LnTM`mf^%9`W%(dD*ZYmTmCL-q|QnxQFJ15&y= zm(A{IqC-KsRyR7e640%$`E1mMbldof9A}U|AiQdRHZFp|*mTBR!0Ej;;1rG!VK6?? z-}Gq{_Ew2A8?_g1Hf&zO28`8SKHh|UoAw?ssq5RGpbQfOfW*DT$n@s3DcWwC3Vl9f ziDU)PVIE1(H)vuvJp1#?Dg^lFOM9X9dityY^uMmipg~!I&ZCr8cR)eMN=a%@bYseD z=>D+7yI)wSA&sf(X|Q=$uvFYGJk9<9K(0AClX%u#4xyaV`#=@<zL!|J3FPL~ZzP90 zwFK&Av0!RaL<@IU<K5DRttRJ5CJV)CVe_SR#1=on8~(gQDUPW#y-y`#zK!)j<>?8v zSjP;7xXFR?|LWCC)XN1~KPBjYF?6+-wMZ^6-cVF4y#X5GQ>Pb-14%)0*G;CV6#1eB zb(MgT*lc=7IYV#di<uwh<HcG`0L`QWR3+P`wA#&e*UXuRAS6CFD0BC~`uYO5MsLye z?6$zB#_vg(Li*m@z6OZj@J=M=@be@3Y8!KRyKf2{y8L`Zy`WLv1ajU^9hNwlGLf^8 zf=AizSP8kCOjX-8uXh7r^^Jj%E*Tkm(?X-dir!R3Mry$FEtVb*X;QS|+keGbzx&@% z0z>y(+0p~n|L#1;rN49$_x(1=KI}W{XqsRF^J3`<)K@zp00c3Cyg1)2vh7JmEI-pa zq30M1Z^s@7WCqM^6%0f!2xTof`>g#cnRLhilOOZ5|0KES_X-(`v3xOZ6&5cQJs21B z9WGXfX823)c^Ggh%v8ly`5Xr`o(?<;0`rvl>T*L3j|ZbtiwAGXueAMr-^{7pp-}LC z`i_bo(&rnf8``w4ssod8jPq${ZMfHA5z32}Wq%{8&2t?8#)N1f{qYf=RTmjF8l9?e zpd$vh@pD=(SNAHqx8^!s8zQ^fgwILXmicqH*)&!1<1-}9VtrX;*%_oK06^3V(=}SR z$99?kvTm^K-bY}dmTw(Qkr4J|n<upVMe*P@{$;&-D7=ZNqGeefWc5)MAI1am)OyeS zA5A~7!6n4vFN)VoRfZIj;yJE5&IpBMjuwo$#dr-1TM>c7l#Q?pkR8i$so5_Q6CnL^ zob9ep{<7bjbNC*pvE#Q5+aW2mS_0KxC`5igaGSgBIj2wa;ox>8Di6`(7<u?3)~}6) z4U5-fAqm(`I!&bD*1EVqB)W_~3KYhsgwFJhPrl05wNYwW&AQ#h+gZE(v8Kg(t-fy4 z<)D5^70qRwD;VKPE|Nb0YIvj=$fWp#F&%|PWD3s`;{4uJJ<pC~4BpZU|1M_N!jfiF zX>`13f}vk&dK3pz>A(#We*)r<i3)2+_4|^lE;gBeTQbHceJycypv9kq5LD3nzboh~ zBo1klw;#l4Ke88%SuCo-xrgRc54~MZw`Q|M4TZgY_CaZM5}3J8#Rv~m1b_RCYhLuR z-dmqim0sa(Z5Cp$U=-;?%~H+7AJ13|K-rzv+pvOmc^tA0;Nc)_%WDn{(jTTe<d!c; z-k&i2<aVWK;(2kh6*K*)I)%<5RRL1+{MdhVvO6(7Fmdvi4p!~j{mbQ<_iKg0<0MgI z+|iHzF2QmCYd;Zi@axD_1y_hF@CR`&b$>{01~5NI9^}!z=6Fmkeywb?{aAag3k)SN z1H$XR=r()ZSx@EPH+cZ6ijTB%<S2VH#(hnGNuy)kw0q(^Z}-%K&L8K9el$A;4DE>c z1bSpgXnwVp7qaucZ}WI?Q=VUAHAf-M?;-r9gF4*qPw_hYIddC|<!eRaYHkZVUU&74 z#<-`m*oL#ZvDiJaQZrQnm7~M>4ObX{24Ua*clB2M8S)i~TmN55_xJDkSU6Oqt2oXj zveQ@%31Xls$$QEAh8iOlm#SCVen=|eeXn?GpV+i-1`786rdWCXi=g43;7QMUwK0q$ zyWw*?L>%H#lf1^${rstvsF&ROj@>uDRHzFnq;<ok<{-my4vpy@<J|JHw*<4<LIj?u zx}VeEG~hbWbBS0>)=YiPwG+JCt(64C_#qqAPeA_87t)vBH0gfs)r!puQerg--ty^$ z=@h0WX|KgfC#Ae4d1+*b`<`h{=syIM)H@oC9i4UvVY(*bNogVPd+$(%6^NVm(Uta} zJ7J&rya?8zHOq6dAg@TR`{BSH%8$)&r7Hyd({Y)VefqPKYmDP8OAi!3L4bBlsXBp8 z+k!`g$l(|Gym|L~4M9%0RQ9}J+C{IKtkwL|wJuh)cxR57P~PNO9DXvca63%)#5a{| zy}MB~IN4TnszP18e$%z+ZGIk$DIE9ttR)UOh4~VTJzA=}R%LGXu7*q6pOw%iYqw`M zS3z^vZ{v@bHC~04R#w*tEf8#1v4tio;Wk6h;CEIHD!v+A@T~$OX=dv0aEAM!gVVlH z?$G&2S6Q*hP}WU|!1?axXfDs^8BbeYa)Bkb08yI8R^#;1;LBa&?l~}FVcZJA!XS)R z71hO7o)n$cn-~5e+!7Hv#<v#5$zZIK6VN<Q<e6CA_>i!+HM?&;OOk}?k;X#b9hASS z)z~Z@IO5T=>E-^h3s4ta46Oh<r3<oBkv0i8d0S&%2nHwt;(S>j|A6d|JXk0OVqnU2 z0hO~Adw`3<S$2CBQTrNx8PFObB2{(k!=xVTcJkiX&L>_C{7$zhsGx0$*R)0`BroQ( z2|b^FhOYMjCb#SPg5gA4PiEd)Z*8oat;=vO?7LBRpC6u<jWN#;D!{e2Tl5P_0oe-% zH-2xhT@>}Rh9$7Ns?d^rHDG<MxbNbZsWMJTR`+zq4(^$Z5z}i;BE#i4`@4~*L2DDP zLeE>2VzB!2Cg5%>Ym+Hbqbj|2xw$2CTNI9YvoDoHg#m`c#0dW#fPm2AOuGCNRP`{z z`}465Ter%zCL7{zd&ktIl6x+P$0pk6kQ5%cyCYW-hTF-g3v~_jF4%f`Ssp`yV`{~H z`;~zP^3J;AQ4D%%*B$ebH$w{9Q8mCig-W}|wr=6Sh<B|H@v@+mDHP<ha0ARB*0i|j zyJzJ8qsT+x(X2jK`WR>0w*9kBkTPGxaq?U#zxLDDHFu!dRg&>pK34`mW$8YA1$tR6 zHW0ElAsgCw{53$nLW??2HX;0;j9*kj^Ek{3fBn*6-F!|f63E4Od^?H9)UDUevU=13 zXD8nhi5vI1?;EbJM!HMBm<0m$ZjK_i6W`Rln*-X{`$Wkk(!iml&ec#a7O$^&ZgcS; zRlTdUfRt&9v1l}F=1OQM`{WgCvQ`AQ-tOk)nSS+#q&yNR$D}Ujj()}?ke0Jur6uJ1 z)U$qpz7gl7yplzer|`$P?z=$+=R`s)gm-mal~n@vMLF$H?^G#<&#K-jJm|da-?XT| z+MlFbA(gI4oa}R2;Hx8|Agt0hmAruWxkKx5-Sv>N(1i9(q-KOqmsk<Ox^)+bS%8ZP z&I&ocTcY9vOHykU@69%riF{Xbf%8x7kbBO4z|AtE7TB@cl0PN&-3JsnS`f~<2O?Be zb7EyL0DGs1JsCsz=7uJp8?*?LgiNy)ebPO-WF<%Q*sNmVDcEeoW_Zo&jOntkDmv@P z64(xc`jMJ(vsJ5IdJ!$26Ky`NC@!iIaGkAwX6xCy8$Y(q8zOFHc8GY;Ns@1NO7$c^ z3l`?F+<u7V@u;MHJk~V|-<<oc%(3w`)I-ewl}hxmd~s6pFf@+T(Y)zq5z@tjty+>H znpx9|R~VM7t}H#YSEpn_4Qe@i69BQ}pF`m($?umEwnoTxUXdY1(tSAqDOis6Tkz05 z&J#~T&+Q`h0erjK#)OY!Arwe-b?tFf<B58P2@dXOty2}%>FQ?tLvPR}*_jAr<`frL z#XmoZAs@b?2xFfR#1b+I)A?V__A|3IPs|#t*(>ak`tz_pxY|;(#EGqKpiRP^bsu9+ z*`>5I^E@<8p<uI)`(o9qjd*H7N7?c)ov?CN<rJ#yFf;$AG2Y`is_ys1>a3SlAjX<j z9L?<{8P;Ak?L!tBdgXUTK${J*p}=ibwR1v1ER@{)mBt^=Y+D+GGUt|%nHaG3b2f3v zZ+mb7z|&ETlL|Z7Sh|;Mnqgmd5mb3S+4i=hC;Qr&W|LO`*m&f64|1?3#H$-Tj9Vhg zY}HN9{Ce=aKgLDh(Y5?f3dz<P(@3Fs`M<QTna5G{dTP{F&R6^)o!XAwsT-pe>_jY8 z@8ZS-jTW~m8_n1?)*Eu(MwUP3gxKxcmyZClvHix}fW8R%+1LAk{<I*4JHSbWv+5+? zU`zJllCyi_L7S|OM;piFXUXI4u+L{5^K*vf?+bT0B2AqG?}^dMMQ6|FSTSB}@@`0s zkdj_DA3Cx`nA4iqt=w~qnS`iRU4en04#_-h!Py249(~<HmJ5+Ag5LdW0k@g1{aYX{ zuND9yXAvr?AzmY~pbZ~M96LY*8b8&*h8u9nTInvrfmNZ&e0~N>UunAwDp9cctl9je zrla>7D~I0C3&N4oN~b-3PN@cFzS?_QvUJ<r#+ngL-kIQ^`26=%%@Dw>1$-`#*0h?F zxl&H8IU$X9lEzQ%bHO*;1?)Jb-%^_WO6n-wB`#gAzj$ybdBh)uS}RF(bKzg9q4beL zA?_?C%{=E`42<bngTA!4-p6b_*w(E1Yw}e)Rx+@hVvo1-lE{xtZ3yKI9~KUlR-`4b zUi3F%@ea)}=;79p#{sJm$@r<9O@QbDu1jkKfCze@Y3KXEG9XBW<FPK5EoOM^h4{p! zH3SGJQ^w`dCNgpsh3DW)?{CIyrO^Hv-t?RVdojzj3@PP=7=BBW&3`fqz|}3$mcDlv zKsaq4{fwINTp4SERw}(@S2D9@#bbcz)9(a^^5_YXWiq3cBaUk^f!ODa2^^vReU0#| z^@%FnX#f!G&aGcC@G^0!NLqwd6D!%oQEtR?VYGuaHTYd>@sjJMJM5d)1o_j|7Wik+ zAYX~D;Z{xVb~a%;Upq`Wp?!}<JQMqy%TA(N$!EhhBR>ouz9yqSFSuWt6xGH@$inE? zPi!Iu!9KkBsB9B(dX|;13rnx)mWA5lxKuvy)?WCucX5>so<cNTFUa;KoISs3qDY^u z2Rg3!v%;yAl%UN9-fA48Fn*@waDCc<Ik2uN-_*9rqo3k_?Fh#U04Hmr(pVF(zRctB z8#M%<+UbVfwG;ZH4rI;SqjQ;dEu&^Z2`8;q0v=ls#JmtgJ1NxTpFkf(J2wE?fHMWh zduGhN^UN8N9(5bhWPE+5=C^5|1XY=qFN8VR%Dq;GnRp)FziqdNEVKJN7u8rg(*z3O z`2j$2K#Q&=Wu*TZr19U=XnoVwA=Y$`p`o!B*7V;>7-^XlrWvyr=h=Z}ozYvM4e|jR zSiX(6+ZM&Y<uPCA7Vl)(iYo5}wpvuDVQ#PWv8D;0AJjLuyvrviQ~kO!{90n^-XFAA zT!bZ*<gHPyTf{01hZJ>k#Ao*AP~bc3f&>=~luZv$IiFK=4EYhTrli*{Gkk~zRkrt; zV^H&^?0TNR`yu<dgJn9&-t0Dyz-tQPsFI!?2xo!cFXRd_0`a|c{ms{CeL$Ac2X))9 zBXtH8;s>Kd&d=tkdp><I;$M+cfSaA(2ZaQEdaHbD?c4fF^0^ELdqqb`%KPJ~Xis#F zxnq8Ji;&LrtDy1nZfUsqq3KU}dH9TfjqX4h#RTm3>{ogM)BM14b)BV0a{Z{?S8?)9 zmoRG&(y}aP{s@pLi8`$(Cv(lJW{pMnD1}*pDm(9!E4Yy7ol5o{LTcu(xs`^&2h2y_ z`_1tj-Mip9uZoUyN5b5TIL<iE+|N9A@A@MpfY-#@I~-V|$u_-hsOFF=Z7$#hFr6HE zf9ID-(ULh-Y?OPZ_-9ds<4SyCzq62d__Y)pM%kwFz3C<(??7nCr;{r0ss$W+X#9Yd zDy9Zfebhg>#xN~hQp-?l({vp;<Hd|8Kj&S#Zi4_N*`UKagRbJf;{xC7m%4k+IMSM% z;p~5t{6?DI-I!|C4P4ZAzm37`0k|9A0$Tr)Gmg15^U(luN@}sjMpBP_tQw~nys-Io zf?ja8_quiEcZ8Rao(1L<WF>U$zL@tWwOovVdJ;COo$)vyvAw*&nx<M*ms)U#tw&Wl zYwvrH%NY48o2JYJ^J6-oX-*c8@lkM<WxM&FzOyOHeWw?&{;Gj4JmdFEjBr%=EHcod z`!jQb_tQG$17<{2lq<%f=$oEffNUG~2YB=;o{92xJsJNe{kmf~|E<M_r(L&$TwV~~ z^XCW7k1z3yuzlP-@->XhR)e_=q6XXo0@^-s85=x=f>nLQ?qtPa`A;8cAPysHmW{zl zTli&Hd5eK-E+)UX!UR%=my2JAmND3Li*h?#<fZ>lt%UG+;$u7cs-Fz|z_1hg8GPr{ zzF5K4eXjm-^0w%p(o}?e)1=Bv9k!l9m*b1ll-emz8_(kdX4CXR>SJ(S7^P>4!Cd~M z?{^@|K)8ui=!-gdE*BnT1zY=OKbP)9k^}(wup+=+2&`|`zs&Ti04Ho1N~KQcMw{Cf zAPoUNPdL(pzy62$6R7oBeU|hw)-)5;<ZR?La4r7~EUtp!2lkgG`BEfhjc|Pt8U@(- zHn*6-BBirq-?`k0-d)85ygk2jdhY>a>5tDnmiBha(G_ntB|JSN!GaOhnYzd``8x4G zqGQmFWsD+XU>tHg^otFDGxZv3y+?ed_MHS+#sS=D(^~r`p<j&W@7;Ic>JHYgB8u@x zDQ+p*TtxzlvJt%$*gP4SVSLsR?+TM{@-tF8dB4ON#6!`u?yUfLkXypYgF5eL2-$N6 z%elYML+Fj?grhBLRW2BlAbMkoc9PYx0>Kx;wnXA7tAzw08Gi9rZkfsHKfQj5Bm`L; z3ZXr9iL7Y~kR289-!t6Oj(F?5u|_^1y$cTLuJ%eGa7>r5$j6$Hb~hqhyr3RF$TL}@ zeCh1Zy=F<k5oqn53*5JMRl<v<4t5IOQjz0Pm2X)ddg=KRk+%`v$a_O0tXC&KafkG` zbSvMYen9$+wFDu~kB%l@4*4aBy$_!ry$j~E_8o&`xb$z^BGnSc3i~e_3es#Qd}W5N zP8I>Sz&3rA)$&Y%Z&IqQxGf)^@ui?-pE%M}IZjv2*s>r<Uu_R5q=AhF={`qc@sgUb z!TJ<@El>BU4w-!AQ&QwN7x6h0P5e<-)8ZiRAT?C!DA4GP@RIte{WzI+cJ$qESdLAJ zE-mDfJU*I^+sG|rSTTl!!Mpi?tr^TL&5Jgi-Y_x)^i$g{^XymIo<IHg0Hks=P&CWX zqnGvNiTAQ4*Z-lLO-gb^-Rm>Y8i<OeTn>>BeU&uh8a}}s@-mjTF7pDS>Fuv_ju{^d z(DfM-KpY9GV$ccH71Z(0_4Lb|kW^OST%I%hlvBFkN@-}D*7+fY%*2d2AJOdxa?1%7 z8~g`Zj`yyI_4gl^q}JhW%4Hs09+Y@?Gc2XkLk{Zl-V5N`PgW><jiju1UCbEFZm^H+ z9o{K&But%h3gHU1zvh_&-E&CJMrUqI6;Kzc9MYGsA10en4?Y~{p)w?xSbVbioG{zl zfG16luJI)QEgGA)-JE`OdT3-YiM2}ZW{;S!i>hb^OVx*yGHKT{g>#L&zdx_F8c+yT z0g^xmX5O+)tv<L^szUYREGIR(4xavvrPA;p`_DqCH>_eD6!myOCoRrV7}&Edts}Yq zh7=s9eJFs*-0$&K)#IJks-G@pLm3-6Tr=a*kyZ>37<Sj*(Q(l?9uS;O7U%0u8sPn4 zNTy2H-U_O=J$f56ApjUe7YS|46W9IF-#khO;~9Zj)~icNrb6;^xS-G;)+Ecv)n%D; ziaBJf{hQx-QRB|X#;ZSkT(Y`<78*P(g$NQ2^#7W~GwM`I+=$JZr?y3Lo54P~K1w{m zsESqo`ql%i;?HZF@<h_XdDHzF*27Ior{vZVd=xu$b@fkC?SiJfD_}OqcSK`F`B^`g z3!b~<@cx}WHxF{~z0^k;z-HA@W=jblw|xrixzeS?ub*<DONFsVZFegL|6<9Y&iZ-v z>wqTCnmoB}srto^#i=0EQaB1tm~=95M5nqYC#!Qg=^0Vp0v8!Y%SxMlWAC$A)Y~&6 z*xu+=zm(@LPSd43S!*c5fqmhYDB&ice2Uo!%tPM3q=DZG;Nx+}A4Jfr@u5hY(4Hzt zaAZQ%#{$lEJ9L28MC(?0dmet;19_X5u*0E$-;TYYo5xbeZiga(k70o}@t*F(9l04) zePyA=s^QE7UOH_f&TX=J(<|{eJRh&mDc{_BLp@U<u_F%uksFOX>}6E=The$&^56f$ z6aIqS)V5v_nc(uKAwwVX|D8!0$_%ER_BvU4tnXM!$)ta(OHF?cIU<`)-1!sT9}PbJ z^GF92giYbeBv3g$`gabqiEpc)CaNu$*}7CD?I$x*PEeJ-7(F%_h>)#8qbl>hQmXdV z3djVNNB^#DEL$u_X{zXzm4NA0aTW)qDWB7zrwb;L(&aFAm)&>oT`w6`I-G_+q?uqn zKVr(Pfcxs9BAbWkyJy#E@<$%mb1zc5qW(ZoQdwb>8vvbSuL;OfBPW2y2nRUVNYN)K zfxWW=7xC%C@3<}m``(!nNzSwc<IgcO##&z5D|7BI^Q<YWl3>So$XT`l-py|stLRvY zpPq?<KQzF2%0fYfY^^Nq>+@Yc#C;%l<;i?)k#9Y9x&)Gvp`AAC*g-v^f7g9SEOmoL zx{zA$jExai>>Ez)QERnw*sS-I)864ua*J%R@;;3$<gltKp9q$(4?Zd2$NBPYi_Zme zF#o@`0Nr!)9|4t?q-a3jg5|E@t(^VJ?)OV%1!C?fp~pTiH?XN7R<D`aL7ie0Cx!%1 zIA?l<9X>C?YzYS*^?lmfROj>a`5GP|2RL)5C6<q8-0qd4{x^Xm&5$KBZRuio^x{~G zg!c8>*cKgoD?JiSrW?oFQ^cj+bv@Rh-U}LjARV@+gj{uyxRvS}gG*}v4AmC;*nY3J z;Pc+>0=MZa(3|lnr6;FLa#*g|NPBmO$t4!Tek&*>3rr3k`X#l-2fmw%O7e!zI>6=R z-W}bi+`FD)_pywFY$^H&!Z)7eaS5S%(`{1k2Cwc4HmYW18Ct8~KOQUn*WONKwx;8u zty6hmHeZ8$$AnIb&?3@NI6;)R+DIBw+ZGjJGV93i|C$mEN~UP8(KG^zCD>u-kG=>A z8OjtIjm-*5xjXkIb!x*0-cUbmx%22z=0Bf_8w!8H`(dF6mZKO*=gIhqA^qhLA4iq2 zn)wF72O367^FzJ5JP1;~#0a~iPnnywUf=NN`1QkfaCr@(&!X3-s<xCXj6^a=Q!-iz z;<I!^FNL;R`I39ok0P{>ycLhRMbD$%47Q=QcfC?}f=LF2Pc!%BQ}|bvs|wwa=jyo} z4#r=d0xYa@jD3nXpnD8vU@LjS_HC8VScdBF&QUKvd2oL11HOaEg7&)?DvZ)zleDBE z9hZ>LT)~v)VzJFLTeM#8kR)MoS3A+Y(d*4TdZ)+n4C|b^956Ejwm;81!b4@Nm#mp} zB$(`q>>Sw8Op*>Qc)OI{+iH=O3qb%C`o75~f_o}r((%|0N!Pe*5!#qyO|O}Cs%^{F zCS25S<aZ-GK17l7j>~2mKY^-%EkFl;(|<5%Lp3u~mzimEn?ZqTglTOCFPt8QIJK5H zSCwaHH>=>KW~6Mc&*?2UrqTzVcc@7zfHbuNX$uWoEHk14rdroo+okt>X=GsIY4FL4 z+my>0DDWfknCyG%hXI^WL)clC1d}7UrZ*=bp@<MY2s{m{Iri^NEtVWL=)Px<k{nW> zOmWlTnF}Jr)^A`=HC4^Jm4b)u6DYkHRsVIf1Ch{6Uvmeog7j4H7G#Kg_CzZ$*F6z- z`8IZVI(bf2*>(8`Cp+G`AO-C*CP8lMPAqY8lVe57WS7GB7I?tMV+v4&%vsB@!fYPZ zC1jPwCFj0EYeZBM-ID6Ai7c(NsKJLG%By&;*BzlA0PnN(chxXaRkoc}=9IFx<yRBw z2VoiRJu*sH*#=k(2Ay~dV&Wb(P6VU%`u`7M?-|wPw)T66j)IgWf^?Ln;-Z8iQbQ30 z-KdBzX`z<{>4-=PEg(fH0hNwaJ49;e2_PjvfCxe$fC;_#5=toNW<C2k=Xu9F&OToq zI)*Zw_ng=DFFz2;VbncPL}TCoMYngu-$m}CIYuQ6mnyW_FNyngmB)r*Y|wLX3~{8t zZ{bLMuk{(p_=rM#46qgM_BGgfy!W=mykn$WGt_RFfSdw<ThAVc3Sc?^liaF0%{MGA zvB?=8|InB(qEC0rE?w-UCWrWz{dlq-+N}jIHvAW;95&k5s8TtOqD+kQdcV9zW^euE z*Zl*J-sA;XwfXF=oyT2*LKpsu9dKr-ku}#*vUqha9V2w<Q}Ia^S22`fP$M3^<D{b= z?G=>4*vAj_@e?H_+_+8O1YIUGDnAzef#<w}5V*9Qp!O?kX<li>5Xr6ys-7{-Mw?fx z+-`6D81|uI*b~Wv-iI-;@jLZsb-G<7wOCv(tg4pdiX|b1N*fI(&T>aAki#8OP51w# z1p1m&*9M!nSe48SChUzgUJeHx2?*x|^iTP7Bs7LDc*%U$rrrH|1b?0#l%n|jV#0|H z=L!bS&|!>NF<v!^Un%`ZPV`j+M!l*j4~`ENlUXbY3Io+D6ONv?qm%LFfH->W3c*i( zUAnhaz?{fH5_vLeiMT4NzhtoxnX=>moO{9fw+AwjKe*ds8LQB_44omH@;}Gvv_Oea z#2FhmO&}on>RWE&1)D$l65#0lcHGbY5|;AarVx(So#&2-z;iUk?Q(5L>ht2V`+cT< zb7iwfhRj+dtibm)hax{D$kw&Lyv34GsZFx7A1zx`#R1+8ahLrd0=yw4x3X<&7}{c) zmRC8OUF}tV@Bufy^7eAZG`2R5`Z$1lFap*0fLmv_CVa~E$dgU!S~V%uZEX~AIB5-3 zR1cMRP3_toYK@In&(A5*<;srUv7N-Me41|fobv9fD`DO@g+VRfie1Nm$DP?gkii~S zq4Juwkj!q;$a|5aSp(^^jVYO!`bSy4y<(tf1I1}DMLZ)*bwAxF#sVc)7U|xVqJy4Z zfDj<_o;19Kko0L0iqv->4W|@fK}nmh!>N`hLOe6^YtEC*i+`%4Y)IiF1>AQ-1Xs-U z`m{Nk6I-olVf$(vjgvKShp5l*TaTB6b6)wo#zd4%a;viy$nUt54lr_J9la-l0mE+6 z>Mu;)<9jDaz8UF22dvAO6IJ-ts3d|bftcxR9qJiQpijl!{`*JdLM;R|%>lN0k-b|Z z>mBgt=rT+Md}2v++z+gjlh%-r_-@gpe1<!0Rw>uwpM;<I;W%5`DJ8|GS9K~Lo$ow+ z^Xso(FtsLlS<lP-c*a({S|pIVX1nS;rk6?wNJoSa+-f5@CV_F(!R%&5$;_88CZ@I` zh+>T_6LL9$F}q?%QOaF+72P?yiN#1v_vFlMYw@g*_oLj>b{p4ZStN3U7FlteN1;=8 zXNR5LuK#@4bih>E9T}9u_@8d^k4@=}^gr~)Sdw<^;M|wX`R1GD119p{4c+SqNca0- zEzL3NLYuZ}FQ4x?btAVqZk?_$&i(7oQO0sx;W#_Zk@bT+GN2y-25!Y34PTA?S3it- z^4(miNAEonL9FPYY|Ro>;%`1e5V`4_r}2m+<i-8mWnZHOh|(*JbjKW^Ie0YL89h7} zn#dOgRrRFEagt&`d0yG(5O-%^Z=)vMqqJnkr3;V)uh#Fhhsj7tJ<6FA14P&;0g{NT z?(Ibm5su=36r>HthFQNL?;`OEYuUh%jLy%j7*6elfGScWzv|ms?^Z^`Hk#=nM((W1 zIz)-fbSxoNpdQ?~C`u^m#RjNYRHVrhb>xTT;U@{L4B2|g3h<K&NXK1N@cd=LybWja z1Ve|$3#*z=k>3%Q&<DksJ2_MFKgg=x@*f4#G=C04V*<L-bHy72#{u3~P?v^~xz)6L z>Y>2Yj^2@};Ga0YwR?t>%@x{r)KKk}U?uSfS+6t|y(gP{>7~@cR?Zv3l$L&oO~qnY z#ka@PkGMRvGm*ou#W%&iJ7*kkW$!pV;?|-y?VOsEaRri*^H?>fr@eH8vL{1wgRet$ zc5eoBvf_2YFva;|k8O9e6>P{|X<slca+k3ITxLfC7JVY6G`ZwW60Q?!$mfX}>GyG2 z5JaK^ZK`qFrSJK}4}pzybq1Z1seD!fsB>K%szu4_jEoj7(gQkg(RObL34BcK@Rud< zmx=m;LHiONb-y5aaXMY>@rrNmKRXBXlfAb{GSt+{>?FELpH;dGbqm*#<e_rjcx7_w z*N~}G7IFi2!-;N4e;b7|Jo;NXJ8Hdwru3pGK4V7DBKGgabRff_dF!3hid&dsnG(lp z2`?C1TT~_`L@yQV?hxr=CX|6kSKSY9d7CxwP<lDEbAj+~x(0jYUq1z-z5<oRFO6@f z#ZxPKGtxDLcVUFW5+m(`b)OPc22kLxbJG*OfXF!dqup&q4^9F6TW9`tTyZtL!4Yp& zHPI6w1e7~#GnzSA8)`cgDUlGocO!T<0mh29oc5-qcKO)zU-z>tQEId4GpF#QA<H|U zG7bM#E3M-V6nMzWx98c;pc&ZACUEAI)SFJqIb0KK(A+>Y2CA)DoQysjU<#G$FZ`XG z(fJ2jx#v~~`>nVKn?lG8G-U}`X_12j`>P~&a~5;f&}S~e=`1%L-QQJLgoQ8po#-5m zS)Q+Jq^dsDc>E%Tk?HKZebZxq<Q~(LC&~X0s|ARK#uD4zNPRntXq;+^>@<meTn`wI zcGNg-g?3atpqO48{mvKfbk9RAkL6>6Sa-^cAhwWw(8lS4ceb>wK~X-7<)^E5Aq@rl z`R+xKgg5tx%qSG^$j{K-5vMTcJ-+-18|%~;#r14o17mgzr=pLu)qZ-24&d)S1s2sj z;RhbAp<o36D5)!Tf#G_A5g}U4n;&@*3Nkd-_Ubiw?l^pZ&W>M;7M>P+PS#ucEOWpX zCCdvq(b%EaEdMA-L&D52gF^93D3-o?r9J|;Wl}*kCGcmCCN<9A{Ii*H5f_I{_xHf! z<YN)6j_wL>A`|t1p~KR+-K$t(Lr!nrBOQADOIkX27vu!yHwRuSw$n9<j#)sh<f2>K z2NH#4l-3>_VxGH2$~RqpOKZ4CK^t8zKFX{%@TEC$+-y^IvUX}JYkS?6T+3&nSL!MF zIEtZ7Zkz2rQvp?Iy=?Cu$HS!r$@ZQ2H3@kU3^IgE`4M%U#z(S=W|(9T@Cw<T4?&Hu zAvh;3SAw)0F>M0(VjInmFtdS6o7WC0lRKkAzVW+fV>U;ke9}UGN94`yr(Ru*p&a+R ziSKTaa35HXyS7YEpz9kQLY^zuL0a&Iwlo`aziz8p9_#7=@&3)Q!^Dl&w_=T>&(H$M zN)q;D>%vK3;IE=E|LSL0&JNRT)bcJAk$QUSA?3Esa9H`ZA~0Y0y>A>*Arg(5Y|jB7 z#j;E)it(Xa^K;JC?r_ci&z#*Y@^9X)Aq~C=!)-;8H3sdTfCs7_QkbE9*F1(ZUiFuN zSP^IN(tcxucv)DhF{Z~N_QB%D2x<hw+8Vv(JP=bpu97IM_pWa^%1@$6L~xgMv5G$l zGx>24m#Ehsie_Wz7WRsVKNNcfb+13pPRO!Y9~BbFzDBOe4-v>sE0@Hfj^*Ff1{p+R zW<y^b6Wg(vuA;`<LkARJlUVYohSOnKLRnTueP?m!X_G8EB5b2!)IJ)c^EdK3O4Xi2 zD0PV4>ZxvSVH5mX<c96ZjxAW5=BQm24Af#!?>lYPi!hYs9&|TaQ2cw~{DG#&^kKYK zRq(W=_+4(2nsG?$W_etwH|KDUs7l$3DlU<%dd~w(19z5=U$1qL68ft*?R7^i@_uh& z$ua{Kbzs6H(B=@A^M!CtrM_>=&7V#ypGRX;u>a}CWy@k@yv)JwfW4tn8k!nt+7%Bv z9hWQQIY+Kq>`wngHz<PMEh+*A7YOIa1d-w=2$`lN?&nH}zQ6IuFeV$P@hSd8VGNZi zpy1(!A2PvTu<%H>ORp2r<?I6T+dMnsTFk$h_#0{pLTKTlYo(Z>-5gk|Xa<0tA9iAR zgf{QWv-mSomIQYuI*eMfojm!>4oX~6)OTpsMt3`!b~hg&j`5BuotL8zVt=2_%_r&y zUt&xQbfhv?7bvvUE$3gLqqQv7ZZ|1ML&|a?-r0cE`!1tiGd2C`&#uL>%-uZ;FV5oq zk{JXpZYAEv66wbL!<KiPcNCr{6@+A@g|X^XHT5QAKeG(nR%C?jLQ_2&v|BklWNm4n z>CY$2O4SwoO2YzCH8obmTuAA?gCfLNDV@KZ|6~nVddO{=qGgGw0jAfRL2i`?QQemU zZ5W13Mf%~&IB$1Btj^VX;Ia)LGOOw&Xo|i9`UL^mw)itf&iimGPU^$GrFy<aycT}l zW$h6cMu4z0c^%IkXruJpD1^MJO0Cp{6Frbsq#W^~v)uRQUYHogU%5yuK4L@JO6!%4 zba_Cx9#+ULRfw1gmba@VBz@}*S*rgy_`O%fO~!KE%MBb(-7#pa-=RHB6%W|5afGdD z=t$=TA^K9e(2#e%&FoEb#IdPJf6q|kNU%kW?PhzLSEzsk_QTN@F_d<hE)a|XITht* z9Q{=lNzK<3#$q2%(_fjHQc<fY$jb8vm-nm%lelL&JCx*K1)NDGhRZX0p9%1rd<C^& zHk6QI^*7KFtJHRk95m&=F=?TE7VHX<bgX6#Z=KR&^eo^nRPWw)z8U#dnRU~Kc~WHK zu?I0iv?6p4BaU%1@E7rCCK?6<-MTppLm2M6>7MmV<c%M5D98&<n{wjol0~b;Jz0E- z3Hys1v;U$irCqQa6IuYH>c9K<1xC`0=mn7t<v4AQ!l{O2Ehihdo}*<U#nY>JD?6cg zg5rxMV#@?O$Fc3hJ>AyvY0X$uhK`KR1?^FmR34HjLyX`+cSMd$9ZN_rC^!55?vk-B z<<QVfrg5F4(eCW5%~cE`+N^OEoN9J>fG`tG|9t%OZJOC(f7tLhH~n3*&)XO9rKU2q z!<VN)qcWWN>)u-(rqpts`ZSNA_G@N8|0%aIJ9Fzvr|xGRe~9*c(BG_S|M6kW_K8uj zM)km*UnfUxID=uqiFtdC>+}1t*=HmBaCg=j(vqi5&PE9QGpoxv9<~a+YiiYUv!(Kg zx`d~wK^4e}8FV=&azkfN>ZOPCk40VWks*Hev5GLhoGKlA%;%v`$_D8W0u#Sxj8kLs z9FOS+7Zd?pq_jdvM$0%!H-y`1<^HYgwawxmHoURFx=11n8!`}b7!(&b(`PgA@W$eP zBb)bjJ8Gr+`TBEsNRanyHiaObhb2S@fl=ZypCIBZzDsBMDIA%TDh>?fCVQzNSA-GM zfzlO)0$Byec|=Tmc)o5A>AEulM|ferWKhCoQ+UHuf5M)`eo1W*qnxN2BLk{UyPbpw zFRGAK4IFGrtc<rzMRq~cHt)s$>&3X?5$!0Opy|Rn^oNacEf{bR7qz%j8Dj@&V!_?x zmA%)ueWKYh+G4@(x_LDx%|``K1lH!TJ8$tuqY2?9$#L?3M|NahK%#uk+2`_P?K~2a z=BJV5%S`5K4ynJw!H=5aB7a7U1rW9H67F8s1II>?MZh@04g3gcHv3|C+*hsy^h|aC z9y8u%8st%~UN(^Zs1@~%EXLgO@x9o(N88iH$C{q=OQxJZeH=UL{9c`3i)v85=rM(> zrh(>-)Va4I@mDr;7>}_&%V^tFPl~^nS0lOK9Z3Ao+AsrrQ4tZ11hAqCl_sz@+|;`F z2WbCy*!Bb81rp8HZ?z2p>X_5q%^;lmui+U(XKAd#W6&(<6lK)5k|v`BQVKJheW^r` zJg@ps3J5Anf#zCd)i6Pr6g|zs^~>eA74ueDeG@<y?2*pnj?7il)Xhv}X)7%`K9O@; z-y^-}gVd2|l)&IuL!CT}0HUjg(kkht$e?`{*bUa(>!c{k(yL#8dI7JGpl-c$({W#9 zljM0n6S-+~H5A<!W6~%p7=|s@owdB7hpFDn=RVUdLvyNhJgVSPB96LrJLh&L1`>9a zgjvKI=Nl&Wi?rcWb27p4xZJ_egup^eI1QMS2(b6Ffp0Tn*o1;L%ZDZc1(?JAvxExi zS14<L(h#T9@l_EK8Dcc@qjK|v%={~dDha2+Rp9uhN<N4uG#1%zCTwWi9-hc})hsi? z$Zw6c0s}`!DYGPcni<Y#f5O-1qF43S^l7d8%z|;)fhoxa>1+thx0Z?)HX}G7Bp|GB z)~B*{>yB9$isXn<<G2Jpa0>YqS-EjkZ0uPQ<>&~2;ec<+bxWd{`q`I5TY8@fgKz)i z^VqKOFP^Blwhl@KQ1h|Fx;>NT*XuoN2WS1D2VIw68wKzGnpsI}@S7+mqMOHC27q)E z%X58Q^{NDAeVz1NXV5-iNA{#AcQ9CUA18#r97$q(WRUmGf+zhF2=jH&pL;B&oVRCM zO{>*wQ1B}U!9tJ=%b8&#ei=}euC5sdlbHB#NdEoj)~Cv-E$dLzy{nTd!R%IDhB}%? z#)L44Xv0Q-d`dxMq9fmA>&AS7EnzLF4ro)mV_=$?#*W%P{QNo0X}QmzM{_~fls|8X zG4=BJoexK*!CuZ6-O(1DeR-aJ%EOU%1?#d*W)SMkalkm7fP$|3T&1fsjrVcz$&5ee z)A4k^ncyiVsX+Y3K~EM{`A!7t6>7ItQf)NLhc967z*x0R?H;bgA5>_ea6gM$B1qn{ zEs!XIdLN=g%HXbdw&uEpd-bexJB}0u?oYjN4H7QosPLl*8)E)K)|KHk<jbh=35i7l zZC)@k6=04i$YB*Bc*{10=yPjT<?YPpL@>D6pgxrF1_&uBPiby;`5xLgrsdZD2GTm= z6|IsS_u%IOa2C|Yy?zPSUbRn_Kog#?D8qAUKnTmc<i;`wIoXoSe%;>4uQ|FIxQ38j zle|oYAZ}LD@>MB|jv^cn=YXK_28xa>6Q6Sg#%Bi<i@3B**#46>bLOk<Y2GP6%r-fI z3PmT2NEV&hoh<O-tN6E=-6mQgJp_$EpujPJG+d{>O8&0Z4k~|FK_QG}1RjS2G17d> zJ#N|rFhCYR>}3zXT7`Ni-jn&tGJ#RooQ+A%gvTAgFzDv+uJqBQ?%d#5{`SJ07`vj% z$|T5J9fyM5Pd!WO&VJwNl!>Gi4>$2x^Z(ca-p^4hosES!_>MDk!LJ~gPDe($DLq3i z&jbGa@q1?eo~G}Vkq8AuSPQ-{y(mO53imO101tHoHtU8VppZ~~I>6E@rnz+s58SNq zPftK-dMV=a6pZnzmQ?~-g*&C%iSMRrWc_-7%n6>@Q}4cW?pW1noQI2hE<C>fG5ljj z_jk@0FkR#2lqWB)Le~VoTT>43URI$rfs@w3)d&tb^PJs=x!PP?#0rM+IA`<SI~t`0 z4w+FPocL`Br%5KNIxHm%qDfL<Kf9`VnTK(Xk?(s+j%T97D}%KdT@knphVQaJ5$D(g z?UqjoDS;_Ef2W8%Kf8WObwOt(fve75CejG`e?PQbw^)L_%bXT}nJrlqEgfp$l(}BO z>@R8#qu0wugpf)5^4{Ee70>5&KL}Ra--)Uq`dEvFi}dwHnk%uz8A{w-0m^)rkvHh) zwM_S}7dEmr2Fia)dG)UwJ@LXQ<e`Wfr>Cw)`bRX|d!Kqv#BFz-++X{y3}UIWoNvvK z0e+Y>=&jcEU{R#RKN@kR_IWSKrnK!sKQc177g&lF3!X<f4n?9QfmQMVTj4qw|Io`u zfv5f0QMc$(T0=n<)Gey_RrkmgS!m}KulRJCKz+qg^<*M$=vNX_vHRZFaEB7EZTpw) zmle7rv_9fmj%mF!@KE>N<w#(VD@;R=A9(FMZg29vpObWB9bNxt>Z@vH$_-fi0V;QL zZy4VJ4{Juty8(gE9Tf7%9e)4H1YJwwXBn4-3Aijl;WNvh-@r5)i+e??n{%nhN^PYU z_suE?BM#WAH9L^uCK=OiOSeGqBS6e#DySFt3)(Wkjqpi!%&7W$6gQklEFk8u#hyb4 z%v;Zfhr|ML3RjYXM;pLWzPNi;M7($>Z70R261P;wTJy<8#4dKkvV~U*b7LyqsqM_m zMjy}0;QSfGRJmjV+eKQnoXUhxyZJy}&24SRaACqOO}Bx&ueN<7Zb*1P!`@@*M>OR& z_g>)5<3I$vXc1|bEioowqX|9qyP;uzLBMawp7a3ZgtK}8nm(azHC92bhmEamgQlDo zyn2k5)RrIvo*z8ED*bbEeSaAJ%M|ztUC`38WtVWjxo8~JZ)z|orP+2Q#J~<1oXig* zKyG}oW#hSGF0FMvKv!!O5mY}M17rs>IlkzzcDz#`V&;V-<;o0$^XBv<SMoyngEKp$ z>-hmkj5EG_k#J{OH&%s1w}inYvDcP)w{V>*<GtAC-ZKm-&}Phd6vhg`X-2caZtXq1 z>!WVk90jhBkzJD&xRM#_0Nt)q7M4(p+R&H`bHihc!FIHbZcFOxo<g@~7d6c75HHwj z=bm$yar+o!($M~`WWc+8w_81n<qkA&&gAp7q5wjE+pe8Dx2Ed9-%@>8E4`RvW>Y(Q z-|}!gL4{JIdC-S?z}=;Q7pVv)R0k72t;Rr>?FkFod%p=P;`P!TN{bpyVN*2;pt=!h z5qfgabN3s1JNIC4(@Z5<6wK9hKA@Edk+)F-l-p=W0p|aL8;Qn#Yr2#VA|6r1sobv~ zm|gqN21=$oNG!u;rEnQGCrJ<wAwuFlUtXU&&xd^iwc#1Bgn+Fqdj|K1U68t)>d;Hn z$53gFCq|C}H<!+BS)ME3S^z;wsA_~bLd=*7YD!K~kn6un&lZVj50!8DnRz_FC8jyo zJZD3iTh3p0=fCIG@G&<dkQnn*c?tS~OC~N<2-$0HT%-+gd7WP>!>pzR{}~Dd@kYvn z6Z6?#zLTfW<TS>Ecn0F)tvQYp-=*Oco|Ct*Qo-GJtR;H<%hx@2CBPymWryzr0Eq>` zbI*-aQ+#oOYN@nQVRMLS$1^?)5|A+8@?zEE@0p%FPqv*X+EMu_$ZjyI!}Ext*yYgh z<W=g^wv>OAP_0BzM*%|wfm7U#pLmaEX@x?Bl3%x^M-611x5{Z&_q0_O5C)9wW0K$I zIn}nzEp*B>9i5>)O?Y$w32HdlPO*HMcCYl4CaOVU1Lw6)uK{l_qpa+HO!d3wg6$MU z=STkA3$Q@8TOqG&Bn7WNef<I$fg8dcy{iO_HyC-Nt+9awTEW`ph`nwi$sf|qCsVg| z{QLDU(k_;ATgs(=<Td>XjX^}a%%}^xB)R)|u2SU~@ONzWZBCGm8)vpCnlrVse%m-i z6^L@LduYEW#Nqw^n=^;Oe;x;B^dq)%z`rNvylQ3(3H$K1&@*RCT8iHn-wr+#>(YS` zAj~1A21dZ!?O)Aa0H^f)?mC&OWT9|-Q3kZH%RS=Q!1al<LjdR|2W#XU4EXE?Fu$~5 z`s4_5Y2^~E<qZeaDMg((l^RUHzu^;ff|W9&`+gbb`AU~7yyIh0geyCR3GU>&^OK8n zOJS++Gu&$cJMM5KsQxfpP-&U?Wmc7Tw%=J|URg1m*ljxq{>RuX``fS*xN#+hlhcb< zzB3SE!5>vl%C$7l*n7&U@*I8O#*1Y|;?!UUdA6(Cy+?wgJ}U{s_H8v8HE9VgW2W0b zVp0<<gaf#oc5+#$@L>N>%ilu<!reXcHrGBfI6ZY>)pt&Mq3FBu43B}1D>8m&vTy{~ z%@@q{8-bNs-gUPh8lGEf*4}@sZ4pYZ4nyqs3wS}fw3m$t*j2yQXJaa2*x+9qcsacG zIxpd<aiN)T%4Ybd7~|RD2Hxv9Y&zL+)gt9{Cp%vphIp&H!VxvY(w+ir!==&*UHMv@ zWGRUf=b016`|!ohb{C#_>Z*goKbNpt;~#&&(!0%&adP?$p1sAn-{z0~&xiSu9aVZ9 z*}=HjCH<B+gJC+EGTz)P!|8~;VR|CNu9m5Q8N9XxddBymVi5ysu?b<9kZ%3;%jX=A zC+mmG5q!&hA+1`X$}|_TE}VEFF}VTd4Z3B(+n5Z$;VXoyV2iZO`0^nqS&z)V-V=@x zJld1k1vC%mOXo#3p+NtIC_L)E@@Sb$>OU88A90IHu_~L)X4Esk$$0y+Cenf%p>TG` zvFZMt4)J7Kt<vP^s`?xz)Gcwdru38Syi~t`nJCgE`g?(29&2${+F-`i&k-Q-=ikTb z#Z`GYEF{j+v9SLTOt&j)VBuP**|;K6hw6u<&}>JB;+Pa2J<(S<MGdC3T?J*Q(9W)U z{5qz>rDLZX^G_LKv2DIO9&d49s<7$(BlXMo_sn--dpwu)qB?h;#fb6Dp(DS<4e%nl z1-rIFXU%1_mz*!2Z3+q(u;nWVnmXJqFD&2}Sqr#VI9{{Jw^83A#<)~5`O5j)D)MiF z7^tVI{;+tdZuKK}e+5~jN|ZdvhlxlwY-S@(E524e-=f8nLWztlvOQ1W@fT`TN@;q4 zLbOwhd10U1$L9S&e*L?1kmI~A8Li;fd<<D1imNRy#H20u2wG|P*;>-dwG9q$b(td^ zijB0z_q96ZeQ(ez56znQRjhm#E|cRofOclTz@+&hrX+M|;<9yln)Sw>Gp@*zo|3WL zDLYySD*qlwv2mpxwY*pztM#pVlf0SABsvdL^c*%mN?bFSqep5FGiH{T$OM1fal6{g z8-8{vDU9P#)fF$*4Qn<uk+|!ggA~A=NLCS>aPz|C?&KWXGKh)9TjjsCV-MCW*EiUI zrfi&FCCUUZGk9g}N3MSvP7_oj#9UstH;NhcLOH*0SBEKuOEzhqWiX(JNc_MTT(QxS zxCx_6`sowQ6~IOH1Ia9n<TV%=+f;Mi&zbJEkke=H)T47QU=?Mf8NQ)W(Cj`owP~@Q zMlfuiGqvg&$(xdY0}(xf-CU{~;T2SIqa=3It#=wu-mh8p9(3l8(K?imLE^s^MgB!R z$Ix-PDHG^#Rypf(xmY7vVeb?yxY?hE8xR@M4x1B~;SO|BtXCNMh=J)sjou2{W#MHx z=3TX6?P4Co0zA)5eT}HaUNq2V@%_}*gfi*VaDv{`gm1Y2-6$o?;zX@zcS8qF{}CIs zV$CxBCINdiBAgbmoS?p2#q}iY|JR1mEuF-Wyuao*QpX|-muZQ7Z*lPxOTWIv^quwr zQ(KaHdXa}q2z$Ri*MVZV3|xE0u59;~w_dYA$j$y7%~7E7jpyQGv>^V2O&NNQjncfv z&pxzRQ8^l^`b3;Vi?oSdF4OGYtVp}w8>x$Jq4)Xd<?3`?Q-_(`G%v)5&Z}UHeU=+3 zm%P9KV$9WKkDyO2j5j9=vcH+@mhvv~HE>_+xeypVm3`}9=A<nZg6%Yxz1TU$nnSBS zJ`PCO9ys-whzXDeGJS2Ln@K^<c|kX4pN;Lv$;`JYazre&arcgYNpga%R+AD^#i3J? zd0!74!O<I5O?#t}Ul5tMPqCj}%PGbC(`a!UETV<=X&i%7AW=g&X6vq!gzfr)fXuVN z*G_MaNhS2unNDrrS(ry9O@Ty-%&$w;t_B6iOY?#rhuk?Fg2t$Fm4Q2hF?P>bRxEX_ zfcX{VP)6<wE+#4tyIH%-Gg9x?5BoF2uo(7ySfVmJ%m94ebp`q)jZ#zbiDb1kdDo4b z$KtV1BU0=0mj0z`l2+!?yGEwU)8#YmBc-l`N;VjG?5&E&MTX`%{)|v5E5QQ3AuW^- z=;%`UiwP|rJTEw4gZ5{GK#Ouu)?@Vknl(=TtT-Z#d3=nx9H|*zUCjm{D5d$$GbaKp z$rG{=G-TLi`RRViK~3u|3k_;$C)A(AkX^!edGP2FhfY;6uG(yyCYG_5Wre?Ifi@b7 z>+rwo-pKaA5Hzh4HZ7Y)_1LD(7SeUE1G|-5*x#V($HvyvX|RPMsh8@E9_2plpflm= z4C9sGO)k3KJTD1oMGUHj?zzYilM^6&qmbPhtNJVL=#02wv$C6_r96~XBvoB@Pwg?j zJKaL-9dfu*GXk-hbC{i>S0UI0pY<|uTge)hR_}ECvp2BL$a=HO<+Gncut=1hAy6Gz zGJ};lR9D*dmZ>Zum$lB#P)&Ng56ySux#p`Sw>+=T29;?$;N8a@f2we-Cw>=zoLt1l zIxrXHCX6ZxPQgc`)3=U`N^E+AHsY$WUqRE`-$NeTUDQeap(J0-!o}?*M|obrc8Q=Q zI{Z&8LF{A?53U}2M)16)HC4Q3X|uU?@>J8P<VTUE8^6}hjm}$T!$lcBz?)&I$-O%! zw>0I9n*4)HVECR!(LOApF_=(Nxg6_d>1?ogH_`f^?BDEN$lw2+{ktdilj#MPhzR#A zp`3YJ<Z|8np|m`Qd<lamaYP<^oF~?#i2MJf7{)zIbeQ#NAg2pneEMQNuICw^7Kc7{ z;9ryX91vW$!@w#;AnDQ`iDKI+&j1W@iKn<`I}_f<mS^lDvJ}ep6~S|{MKuHH$bK+l zyyguSkC?lc+Q-zHvm$7h&2UB<n#UFy1Zs_?KDCCdb155!ZI0)Y4kseV($PdhYV3|h z%umF+6g__LGt1-38~SMP25(Jy$?}iLK^WL5pY&_9o^+e(QBJdov}xdR^wD}#VeFdE z5&xe9arNeGjCZh{D3{zpqF4>AEfAxu=)Y|NE`;lA)0p<jgx4c+&iqZkmJB%#7BF-B zJorDne!DPOL534TUekH8I{5%lTLS{h$Zx#Uur>RV<<A|Gb>fs@W5x1#&1NC>_#H&7 zqPKlRgl}&F2u4~x&^U?r2wwy+R#vLMmC9_}kZ2U*c-gr;u?cS~k`@x9f*Z3X0N8pR z?@G2=hep3KG$ezC4_f+lAm+G9&}<(**wha>iwi163(U0{#486UGsE9`THpGS_}6hk z-FA~bx7O^vVqQ>k^DoLSyNDxAE#6r($A+^#bScwe#SmSKcP#O+Bo|{hIsbqvbdLJF zyl*(p6E8JzEj+Ycak@j^GIA4dn!;-hF&d)N|JwP=4TLW!+7`Y+08(LNJkYM~-g2a} z7vOzO)Fs%KC6Kf}T-ZWWFPQrPIZ8vYZz)Zm;tvzqN<EpfiE&I>><6gijce;5r&?zH z#-aIugjM0aNoFrEN38gMXn&CxtKDi!0;r>;_Sml;T?RoIYMmzOGgVwB{xY~Uplrv- zjQBIMT*ty9-Hq>MMP32t@J%N0;&}WpE_$_Ji=@OT=PFKO&QyHS4U@;T>rjMQ77EUL z^8a}ILPopCX~JqWrrg&Q^9W>ogUDDr90OEl*!Aw&aNns6wokaXv0PCwdWYol-B9VP zfuG1Ls7*u1e4>xEqA^xY&3Az%@#qV!!f(96Bo87Sw!JSXs)UG9p3LdJv>_Pi45DCJ z#NjkfRRi`sL%jD*&yq4zxFZXF=6h)4$=h73-XO|WmG-GS6gg1~wyMwe>rI?F^*z$E zlC`tf`riD=G|aA%F??Sxvw70tcAwp1B&+-4VW)BEvs2ziYSH&PSGB|SmNfC8O@A3; z=s_o8Vz&Wr>Sdr~dOb!k(r|GjZ%{D(NoC_y-S11I9qAiNN)nZ9mr#q#<eLR*iKBpL zr27nV935mv`bSzawEEWd@2grQ`|Ni3Z$>0{ua9gIu+*|MW+8#nhd^b0<;w0|@jEzA zup<-#{qOI=KYxnnaURQM0zBD_Rex1<d3$8Z`9t$F(_9-sAq4M2X7+j<3)x&N#@&tT z1^3Lk*r%GOC{z>Pv4`td3M13b6tNxM5e}9R<-S6IW6NBc!)1B67=$=(5t^W`k_X?n z(QJ8(d%mCaNFKI7@D7i`l5g31X?B7Hs!#7Sc+Y&<K|HpJgHt-_VYv!XaQtCj-rUwQ zcSd?`^~6+1cJ*P#rF=@=WDmkvqo{`$`0bnJ`~<kk9>1d;EO<EqviJiOmvRAPmdnz- z-TspauIR?v9xAq!+QToBSY}#LXkcf<!CIk+tOzmbu>@`Avx-BH$|SC83FDU<U5<V= zZdTMc<2M1?JR%8w@G{Zb?f#bCv*MPl^J0}(*BoabdNi_$OS;4_=}PMX&F&sn=e1y^ zX<?o#&6WzuA)owtSy5lv{_dhA45>+r1<*}@GbuQEwArxMVf_e}0>$R-WEgaVbEk#F ztCcl67n|O7Gd9;!z|njwCf`g{d$1{Tc4rHal6q=w1Xr&eAP+z#)g(=P;9oE92v)L% zh)oy=+S;Ny+v-fqjkjv-Ih*_v8*|6~*?~x+burGu&09LUs)=gajZu6j=^cmrMdt#_ zIx1=M6lawNal@WsbNgoeqVyLycL5%b5D*_9ep<|<{W#a4$!77Oh#Ew7ILeu|R%w7E zLLBH6KxZ=V1yCM{mx@@vR;EqtFdR!1xEB9zWyJ&P*9+?A^;QR{M;Et)H4EcTdsoh| z?3X?}JKX`}`MaJ+^No7JxDbsHftuO6ceC4s$Mf99vlDH+v2Lw23(zaPacD1NxN5W; z=)R?`sb05w2GUDbyj$mq<oflc#E>2m9Fn_mVyX4+>95ytCGxLYZb>ao^{|SmD<R@N zitY<Hvy?~TCi;fse6@sk;geCrDg_X?*rqRu_bbJ}V6D=Vqh^6W0#fP~X7MScM`32Q zXp)mEZr`ibhm^)k60_QWe-YowSqfQAAlv~uVy)_6udPjT;kg$yE;~|Q?{f!_mop?0 z)!pb{2XM-ZYcU7D{*;>R9Z79PZi%=D>|_|jBk-uqt*pMX_3&YnhcEcWOzYuGy5?rO zrUN1R($}@WaU{mTp-$ZwZ`m@vu5y37TORuA;5)-5ugTd3X!vnFY(yKreD)tc_s(L* zzn9HSh72!XZcwvjYuJRvRUv;F{d)jjeQ-9w?gk@u^B%*1|Fx1YxYYln78^5g>sNK1 zk-5Bnt}#ervpfH%JISe=Qq1PoMiC31S!j)+87?Is0PWYBe~JP~o2u66`3VZJTP<qm z6@EUp!Bh3l3mGov{eGdE?GI;ecp+8FiI3q79DnX8U6+63H;F*t7i9U4@#Gj@O?aPr zd)Tb`4-TnmCW>i1(3QY@_VGApewNHO|3+gfb%(10da#~up(8<$U}L!LU}JnD)#Rqz z1nKBv+_r3b^F-V>DZJ_7z>9&r6#G(THnr6%JIxX??(wHPk@>pJ-!E?nd2CF^&fx(@ zDI>S<;m=%z1snDm8Uj!s2y5v!rwInu&jv^BLijt~ODTjzsI-{N;i=nd^~Kc(gmtoN zv?jJY0qCOFwdX|MJ|b_jgb?Fq8eW(u6;=UglaG+-hdU6C%dKU09fH~|<>(`By3%O( zdYR@LaDtaCoMbm?s#adcW?qL_&hO}Z75!D1=)kmBl{cCcYrK9c0xnS^ky7`C=-BAT zE#+IxW-NI_FyS*K$$W7_ZAiN1XnM}sdZKukbLn!^*BuLqx}A^`7V&D5+~B>DmRj*B zlmhD|oy48R%r3fy8<)9`H~73fJ$pt;M57_jN|Ami!R!lmM#D9iWeoUs8hz1JCD-c^ z`f-?;CcpOjy`~YB0^<4==1C5cnEuF{<$6>1H#g<}Z5vIGd7eyto4qv#f3DQBW|Bnd z2&J}8l6G-6h{OHvwTz!M(r!I_cmr4!$(VGWq7ZB@XkW_N6eCZRivIz6z<TwOhk<U5 zvb|K<6^`D2306WZL-{Ec&*Q3A_9YqbXVk&yKaL~hpFLlV2U$ywUtCHYH>=WG+jlt` zV&*CIu$(Gu1gzA1wg*V9^HJJ9-3C)vTvgvHTQ2ElDe}k=N4uSQ`)L4nYy>{ZNf)C* zfuZFZlS8j7YQ-&A@ksZme56Ee_w>>wjgpTT+rI%7%hshl=+85e<ZgWCb(_WRMD2qI z?Q=sl>pF`y8|~;2;`geUU3A~hk>{p}*x4kxUC&j|u5uL#r>76|d*;)N-zx;19?IGr znH%vlCg<0VEU*PLGqo1NJfBhyR%>2xUaWG@*)UqpN*rR>ba&1c`x--s?;ZSBzG>MQ z{zDkK+Xl{6eEca#_P@GmY7p1+)=Lj<n<p7xui0ol#Qyu&hac&LuKn%(EKu)%j=a#b z`Z`WWzh6ir$_K7Ie1(H?rNR`$Csn(s!fS{wrkiBiw%;(@n$?4_<+qPo5~>Ku=?(_h z5(Z5XH)o2gV06p}fDbFgQ<Qc_n(@h{nuv_Fk0@TLF2ZFYhD07d)BBPhT@|wIH<Vz2 ztXPGR|B~k8-MS`or*OuR6d)<p7iovyPt4f59O~G1mSf!xDg%*a|5eeP3Fgc;l(^&7 zyM2gSA&PvM3k`C->RUcz$e9Og?=4qZ3Y*mCWT>9J?bZ!!zY&i3P~yflC4VSRZ08JN zt*!kIkdiDAyooiRbbjyA#w{axU|;-1yStUU^1f~kNfoBN1(>F=v}@H_iNm9#((~r} zAsqxz#>n<^P`Yy8@p>&cS{PX2Cw=le)6fkXa89$X!m}Xxw>x$0UuYuMMO|A%40ER; zx4GZ!&?FzB{O>Z4{76y?jSlULM18aZLo3*n9ARYBdx^!jDPl283G=RP??|{Oy9KiA zr2eK%?VJ-InO{Eb!;3w0VzI(s^>Uhl-HgMuB5U+#-t0kzO}P)vdye~94A<>)*(p^h z{p`S4#HWo-kf;hrO*PJcJZQMCMjta_^v6XV{GNRqh+_N&hFvpZudbry2cL;>T|y&G zWy&3{EOpV@<F&DV#E}?tT;z@&aFLTo1fCu0LIA81oy%bd?YDuvzztn5qoG+Oe1o5& z8y(ivh#|9^<#Q(h0@sD1Cb=wbHG@1<NEabQ&mY%3shV@#to1v7JQ8}^e)B!ZHk&~f zL<Sh-A(|iDJk)y}(o>Yq{<w3Saf@C5GMM3CitFX6$yFB`)Q88XOae^U?D?|AqCx`K zc?$^f8beA<=X<xE`3>HN#_Qw*H>zuUlP9;gBNj3)s~&bytVhR%G7O*7axL+Lfx49? zy88At-pGdXZSPE;aMtu}7(OR9;f3g!oaJF+Yn;jOdB17(=UN7Ig`y+Ckr674AzS)R zW4p5}f)(nb%@yP{Qf(SgwC1h=(d(mic9%xk2EComSx82YkDZ4WtZxblHU|D?po)%E zU)^nwIhZP9?6^$gT<(<r$l_tZ_JQpZv8mBPuc~3B2k<`UG88K~%EPvID8xnO5n#l| zZ}x)&&%D%h<fe)GiGO{oq)ZL`l^R0caBR$GsWz#$KAP0lw@@J){GPN6`VN|od|KH> z`sY67@?degKSIs(xk{Ms@pn(aZx|*d#JwY?-ll&z?6EaZokH~vN%EsAQ4s8LDmwRp zmM3*53{87dFE|Rvq-_o!lVV}u3&Y~R+#z0{jxx&Cy_8p-{DiZ<K3TLg$xCZnvs*AE zTl~+rvnkM+)6FK0kSKQ=Gzqwb_~OTE1xqr@6F}PjnBzKQkkAw(`+g&-EaFCAOBLsh zhV|^Ys>Uq~0X4+BPbFjl&lwyav!{%3XZ>)Ct`O(KteS~LL3scl9L>&Dvvk$@N4X*w zto?pSl5Uci=2!%`Voaw`@8u?;uS_0I=FK|sg90QaCUFEPJW(yOBPTT{IvEEo)%%dm z7F<R&FFn^T9gX6pEg|30nY8#v1G^z#a_OJs-_XQEOu-pR#A#}>LJ}OCE}grx-QqKD zzUHj5P0HjIcM7Ro+dOeO85CRlgNf8I1dp0gj@Dl;JrJwaY5rBL>=~SM_rSMbQoPOz zSY^4_@s|w=>upahf4ruC&49#N`I+ShV0<2Ydr7|RF3tWBVMcjtq4lTVC1znVM=6fo z4>S9bEz839ts~b`bQ7>;mT8<VpEo4IRol^fh3ebo%7vdOV#PC~sR@UNlxP%m!`OWR zS&{mQq`7m5a=gBV<O2q4F#ud0VtC`7*`K~DYcYOY`r}V?eEVVG9EZbZTm^T*Cr+d4 z*+6cfNg$zA&h_im=tzZx{1{MsiBLVSv>1MXIZR#}gg;T?{xEZ|_KOlzGbwnN^rsi4 zB5_YCatRxfRGqeSFgB_e;A2I=XjS7{Xv;3n)<bK}nt)Y{`-tM!WBdC&V6}@scQMTi zyE;2_n%+m*r&eU4)Yp8nAYNYCGU8>kx`ea_tO3nmUo%ybo>RO6c!5891?|JE1A9A( zBddTBv9FIBL;R?l2moGmQ)Sq%WJ)#3I2-EbB;QMN`|+?J#op?)-xRx2Nam<L0`^-$ z2A3Xuh2Q5<I7=-!k`xp*9DnToc4Z8J<Me7^o}SwaXgz_0CdtL*{#}#C9;e_Z32@|N zuinn#R}jeqC{7&BHs^736}2cQ<TUsCf-f8_o4x^EjdZd}Jt(Ln?x9_g0$SbOA@NiQ z9qmI%yn=$hXbdO96>X|XDaGeRxToi&Kn(_*8RUu{*x_pH@;1&-eDy1L>L(f~sc}h{ z;OrGNQ!aw#U=&^G$mH3)dTfo8VGO_QEnHT_cF9OoaL*sOrn7iqgS&ZN0PP!|R*2FH zIvt0PY=Sc5U9ZTZGm^~~35fzD;XIoz*2Q8=Ie?!q4!I}!V0g2~MEem><^(KIn;uby zIH86Hsx3X$sB)761sc5$*&uvgE*;$ZxBodNnsL*RN9|C$L@<df-|A2W{z`vPYrBN7 znQskRA#?XEeVu9Cs&U@$r6;6?{Wz&);ep$_VSvh;GQY6ulT~BS=Aps^+k-yY-?vYa z)83^G#fCam4{N*9El0$QSBQy6#i1j|^J9aXN;st@L?KSQ=y(33P^aY^p12(%wK4cu z{5u5~d<#su_(~70n0{TbVjunc!QlTyl{O&@!dZsDcsw)VSbL?(^N+Rbu?*tN1@0&q z;a>=Hd~<O<i(UM1{*ER2>*XDbb&<q#OdVq5$!h5ujAc)?WqAzpV`9&V?ft=!D6^;9 z|7m!&8E=VW^L};$&6sTfIGseY9@(u453Jkha@vy17myo2Vc7y=Nw2mB301F%mF_=l zUwLye&hfJMM_#*pJjXznb$NUMW8ga*l1G&1=_pHVR$xX2Il^@t$xKJXX=3YzRa@!e zUF&7v;Sk%`pm1!N;Spjb(CsRYTGV1mRpod=WK#%*(m!{MG|vQi06|k5GD(d4$eezG ze66G6qI88D>&O5wCe{Mac7~^QY?>#_a1{n84lS5>fRiS4U6cl45}(p~ykqKZ)9hod z5d;)8Tlk%OGTa?8TI}BbD6v_)xQK=G+10<96`n{MBRCqW#sj@jkvl<%vYc}Opj0{0 z##&PjLhUQlk(FF8i)UFg`XIX_3BFp{G$^Lo?YyxmKK3h6;l0vNgK}7PS80g_{jAuv z?i+EAA6GiE7n<<^vh@&$d68f8joXeh>7ezh5dpB2Iv1~5HMB1q?WK>p_6TBrjn;3^ zHUx`pa2PA0#kGNH11k7=Fjq#roTd&eDi*gv(@R=372(Ey0h_blC~it^2PMk(zdgL1 z*4b~4a0@HS1$2=r(NW?m!T?t!AAVRlbUba~)nIkSx9_)DByVt`q6Ax1Ki;xYb+quC zGDzraC7Xmrh)$_txfe`V=BqE?%c=Ni3sIh1!cjE_?(jX?v*5C0NwNT1A-m5$p?uMw zyzv$n4}0DXGrIj-Mnh@5l5NHLKs%G+*H(UVE$aAMNLO!u2Jo>!39{ehSVbLV2<*Zc z%+daQQrUf;pNfZncj*gTTm$B_#g>ZDxR9iq6>gMbpgAcdTaR~z)Vl}d8&lCH+OC+l zEMEq-4dvv`B^~}&``?U!=<e18ay(UZFFlN02L@{JezY~y;?GE&kbbp~n=EVIN@4t4 z%&*Jz=4c0jF$R)eV@3u1U#kaW$^^5ujr#;%{QFvw7sK|@D+Wre{^JulYNNT2AsYU2 zgbPEWRfp9>Sj1LL_*9~XXW&*AwLx<`|J<7oZ00CXlUq~ihn4%)?`sZ^*G`T%^SMJ@ z_N%<;RWl|4|8TPg59wtq7JQk5{Dx-eJD9GJc`4&;zROF$*e@QHQ5IIe7<|H}RhGT; zxUu@^E<*?O5ns4S;)5N#f4}+-k%Hsq%?|q~ktaumDxhWK_o0hC4Mm_l;9T<lrLS^! z<9fKd<?%!09!&sL&#$9Csi97rL`d=gOZ}8JcWX`W(u16cHG0b~&3|l!y0R8mTPWaT z6|ys!HTe9#YHj%RNWuTfPB!I#*Wu71;dXVLI$5PG7ywNr*ORNt0Q@XN?dM-r6ZUq} zQ>YZ>i}+uWE(i-JNNy8TPiBVnvtg3@xSYrM@H*A1n6vKDRX|!ht&e2Om&{0Mh}`BK z?*NLVwSj%Ss==aOkQA29d$DH8S9WhWLj0kBbZEg;G%l?(;%R~kqpb;`hpR-WU<0Rm z{Il_p`P+W3&dHUtpV>4lA3nRW<-<y$*~R3n^8e}L{B-xQXHy{Iyinm~31{i8=N~i= zLis`gQBC)VKl`N}D=G5$uai||#W&j<g>bLN@wOl<Vo+IA1RvCKTsP9yjmU;peoH5p zXW-c;O2uUGE8bK|U!Exp>_SSAWatt@N1D}vuo0Mo2}bIEqNKZ7b8hD6?gp29{P;17 zO`>rrdUrY1o>ceukN*3_fgRH2vI=D|B-VpZYj!6$B+`)!C3+x#%h@z|qWnueOSJ_# zz75A8yo|&Sn{vS1pAy~tfKO3g_EMX28`BRm9R5!!^+?WVhH8`+;ImU`Uos!Z>1&xm zW=?E^Cn4#>Jjie$OpBaD`}K{UeD##wdo*`}kyxzM>)~eODyqt6Zq&^svJ>b9aY0+p zOV@qRBQCLk1@|r<b=Fz6(~k>jj#j01CnTZ1h>{&8H}IekopO+nYlu6UP_^*o1#gu( z+@4m9q4@Kxo=?2Xr)7GQGUsz_IfJG4jqmH>-nB-ivglCCrhepOIYQ04RzqN2-za+; zsBRpbSu6FX&xvFs^$pec)ya6kGRK2F(DZcgO->2;B{7U6tJ8$~pNLth;P@9#<=%?x z8G}gsOkVbY))yqmFZgzT1gd)ABJx}L`qz&dj0j-+yI9^-8-MbBMUQtcX-Z&3`Xtp1 z#?|B0lUn6{3~QcC-VkRALDw`IEQ{`hTvuUg>a|i0zlmyxuo@_BvC_Yfdy~B2&vOTJ z2n1rGdO3fA;gWM(jT9(;>&e_=cg-twUwXaDxMMu9=nE`u!6Vb#8r@}UWNH_vJ9}cF z&99&3(y+%Ygi|+5Qz<A@3H%bGq*I)dP@SB=i#pjurAe1GJWF0~&IP(MQ%61PJ!*GI z&@`wgIpq(o0}1c7Q!|=K;B+zpi>h-|TkLpG=-b7eJS%Y&EA7+_9xyTVI61M|ts9z! z@2>%==-p4*n!eq=+%kruKBLdN`wV3a#drJ&SlosQ0kU?~k@UzPz$0&hPd33>C&$NL zC*HNIB_+1s4+tpm$<*UqGWrmFJPDSYd+%>k-wPBNC`K20--I5J327yv3y%pF#N)R! zC0gqfzqLZvBgGpI`@u*3n=h|vHm`~&Vw#rxDU~yC1;t&jtGxXPU9@JpH5=dc#xIx| z)*ea;oZ39rKG|VRx{_%9g1ni>262JDx0Jj!@m1`v|Jq=9^F5qb&78QX>)2x#Z;=3S zK3<{>C0Z>&&q>E*+R$Ix1@CJ*aN&EFZvi0vs^ttE)NkP(Qgzs@PNR?R!bWp<uz~>= zvjajFjfNq{Be|U-AyTba+oM4<xohIper|_CU=Ex`A>~`)|5OZ*KI$;*n2r`>f1Va- zpMnm&#;5*^3#L!IuN3nLt@n{%9>8_*Kjde4)8JDMYPl048_>&?gk64S68jW@T-@l3 z+=2+71{&M|%@%kmD1o7w$|O!6%e*_l|L8iUb-B?uoe>KRw)`C<uk|>2;45ZXvQqk^ z7`r<pU*LeK>N7BL|7$g&{-FL)n`2DU{pS{x18BnAXcFS))CpT)IG<BuvF^_|-lm`r z>Ws7n)_iWQnU*UCc~3}9_3M}Zn8L_$z$S@7TK2lHu?nT(&K}i6VgTS=7&x^PrOT@3 z>z`82h<MQK2Ip%m=!KNFt5~en)zyFl!<Dravm2=v7Q>L(g;SrtaN@RH3Luh^r4=>Z zkV!UHE(xF6sF*{0J}m?46PLe-m~7w;`&3J|kMS%=6F3HXl;5Rcf_(YAr_J~}@$$py zp(M)fi?TllM&mGtKCHkethGq`msSg&G!m~q*^stRiU<V$`ml$qZI3-sAz#@ZqMGp7 zn}In~mQnO)gUJkIB$3W0Jmik~2p(g!qAUJT>e24Yvr)d{YA{P*p`igv@Syobj|vzT zw4q1!vUaj_?pf9S3M5qV%Fi}XiC+h=4HWOFEq1PlZcj20#GxRS-X6N>al)uT0Kmz5 z`k76_*NR^hi$UB7<ivlch~Zx&lo0$&^^UEqOSdBLs0Bs?(8!bPQ%_EhuYvLym{_)z z#pYMRgQ9(S+bK6-?+!idRH5T47$s0B!JYa`kt`EEb^^!-z}izs^ZLrn1n(#&Sun}( z)3S{I7aH)dEA(yO)L6rkBrM0R+CO1Jr2ZFE=NZrT+qeJNYPYplqB`tZt7d6wOLeHd z_XxF$A|Xc6nyp=mR+ZYZ_h?B}5R{lTYVX(~!vFjG-S_>!uGjYgPdpHD=5v0I<9Hvk zJBlH+dP0(1*f?-PA0Ljj`j&IYN*qo$WcdVS#VhqeZZfNR$UA!|2QHaU?6E-JzPBo; zoJME?WP?NV%?Lt>|C!gHw+S3>QdfOVF>(GzeecLSCb#<<;>$T&)to?XBc99T6o*-x z1Z)>+BIBwUc(OW5UNK-NS`Tzmg=wQUNdT3KtqE;TgeMimTTVb{2`B{#N!UC+b4(bk z#>qze`G!Kr9umN#p0sQ9S)cvoNYC<Y3p_a}RB!z!9LhWqgyM1*0}A0vh}a+-`E|7q zYXR(0smj6U%i@^OJHF9V(b4K>^E@mk?-Os&c?_URS@yWqgxTXy3yz$`9|SRGaIr2& zsOGnKtt-!j9a0{dU#yyd_QK?}9UuQYu>iTxkG0RL6puRW-^4CGl?+E2<zAivj1$iD z3#PA4PH$6?4QRfqK7ta56Einkt3ew=U5fOGfwWw*2hQl1b!VkJWriWRfbH@nXV{Zv zq8<)!I-7jtygXyproa>6gsI+X+e@vO7E7)+{ouHjYJ9tSwzCc_9Q+?+{oHmwBA~(W zSP6Guf{tV1?o2)D)VyEUT7@O>K;wHHgH|3b_MX6^-tzAp1rwSs!DtU)p;qGx;5~ad zToY@5hTQKBrkR(fu-RZ~O21!2HE7F)d@tL2pX?6xQ*DGdeTU=JF%__``n}Qa$71b{ z(!#5HX`Dq8%R!pVAuqtOfDUhM1}Vnz{mTCKG$REmQ(1Rv+xbYaL-mJdtM^sq%cxxx z(D|G_1#lK!j9+cGW-Phd9R3QzlGhBSx4zDwyeE4R#G!W2?pxjHyTkX=2X+CT4BDT; zR{LgnYf9b^r;4(UH!0-mXCs!RoCL1l+;K*Ecz`C>b*`Vw^l%6-Y?+%8Z;373)%!qZ z3e=!@r@fR6T)-%#kK)@RZ0n5N52`}#`O0>ePCAkV2MYOX-a+4;XN_@ro||u(5Ro(I z(WK1kvv*eGMv^y=?5b+ec?IX2-GQHoINyewZWrP_DlyO3N8q;1{O+BxMXj|GCeV2D zo%bZx0VT}J?TDA6YNq?_E7lt_QT(*i%_)%>9ngqXmdb#lg7~-sIaqXXVn2{N`h=W% z!|rMzv!qfbj_stSUfz{KYj!lSn}hq%#uRSVoA0suvrHAS;>iH6|EukwdU(V<>A6A9 zFtb0Gv3IAVtt;0&uZb<#j03V8B;sdwj*^}=zeU&sJ&6Erdt3B4$|NQKKqB1+X?3Q~ zaK(jB7Vya0f4WFH-jN4Y1;py@uslcIJwjEV>2|0NzYt`-hPPgRgSTRjeDBZX1jxVW z%@QuJ=W_gE1SilnUh`7tF8>ndJ*#i!rhv;+dw(;L;LZF1+AmPzUe`8p2Y2*2#;Nqu z4+KyG^S4XQL6)OLJbzz7*@|;R8)5K786H}5ksl&;rG5lAq06IP<4j@+Vndj$$DQIo za=GYK-NHI-YOB9^{Do9!XXdnH6K~G0mcs@3<{es}wyJ)rg3tNg%2y}Hd~x=lNgN26 zDe12fCv9o`(?KUce_s7h2b~j{k-I<ZkwcHcoB_z7{S`J^eD-t)Yo-%NQOeGh6)V}M z=#*NJoRZz|wCQM)U|M@NkxMdFkKL~lQ^!-NSGLSiNt)>2S^KIA)n+kMOV#TE@D0U{ zN6*?vf=sMyC9}R~qIVKD>QEBq%%p4DeZrtXXeGdp^cu$5|L-CsOWlo@bu>KTvLgGh zFYP<IfG)O4<7vtc<mj~_U71Ita!!yAxyjtHT<0lkdU95Aow<E3#Is2Y%=u;DfBbl5 zyr29l)K!7rD<N&{JTc80bF2+_zPX*ZlUS#|&|JNRMU~aULgioHhdserwjSgM>_*pH zzI8Y*DbxRNk%Xq0ymj-=X*1o7=qLa4OpNPFE|O;&0#8{nAFNW54a#H16zAZ=Y%$b* zX22qJkXwtsmrp6lBh?mkvZhY#V?=E5b?uyRg2~aPdNcFRWxv;N)p4xJYR#5Ok6@*; zas!G~COvB}e`XXe8PuT6Nm~g{x(uCn<rs=1rcn*4*oY20yH5@4sU>P;i|4(Va&ct- ziRJtjgZRl1DfHlPyx`FkMFSV>S~vA>SziJD_!DNYsw(rNv-_l0n(`oT?yYxmz&Eek z09~(JO?02<Xf{@1PmdCWmR}6al%;#;xcrj-PJH<r{*K@4XfvQ)^5T9+X!a~8RCZ#l z%um!OES!yaEhJGbk|SY*m=A=hZxb36m{npPVpVJYX4Y2xlcX`x4_ef!U%H=ETg$ZS zoTk2SB~-=l0&iOB)O~#W8-O8iiPRd4Ffy|7P@n^Vs53xCax=1@LC_xFaP*n~&P_!c zPFU@-6*P3!(YIR;DR(Xo4DH(SEr-nZhVp_fRA5>B%;b~LA?zlx^1;%h;cH@Z%`~11 zRzG6<TbQ@Qx^USWskX`|fmF)kJWHw`$#4MB2n>HIN}Y3+wTXAZ^PGZP`)5OUf*}e! zxn{Gj*V36y7T!(*1>TZYL8c>q+vdo=Ns0xv;nNnj3ROf3vC9`loQ_Q_BdlyS%ixi< z`d*v+N5jMCC(i1@jsFO_0=X8HUj~)jAdXfiz1Eg9fL-udC|R}n7ZPIz4qC{W5Y_n$ zOkeF-4sHkc^rnAo3ga%4lGFQa2pGay<0W~|L|pr-VKr#(GbPv=a&6R$@~^qzLQRc& z)?EXlls%S#;l>vBhs$$x3?tWSLVl$jkj!2RBL`!<n1=4#YP^w>^^HM`XWdOlhfsdR zc7!TA?Rf^qkH`5M-(4PR(uU>Q>u6JaqJQmrAIPa!$5r5Yvn>Oh&eDj0L6@t#)>)Dv z^X*tp!sUysc+=9X^Xy%U+Pk+g8Y?n0Il0dE3Kb+SqIYX@sLre7BPh_+;<33BgT1-J zdtT*tz|pr^9)Qlx0#(t<(7SvAD6-x+Ey9u%Hu{z@q3!cpm{3@+C+`QX<A1miN-L%p zfu=R>tyV#IwuaqUUH%+}k#Qci?!b5U;+$3TollNdo&-<+PNN43xRuTce>|=J^hW_( z(>T(yoXu~3RLsk(9Tlg4>)Q6BG=?Yi`b9hYQprxB?TBKAp84;_%>SAUyy}f1AGS=| zd{ySN`7nj;m1554XoK{f7HfU^zLu;vb1abGM%5F+&I0nT0o0ln`6Mpd%Tw5ONqFuX z0&r;<;z#SBe>(g$flS|nI_C0`)jPvI{WqQGmrRNGFSro?xbs?AfWyqT=d5KuoUeK4 z9Gj9GZ(iLNJaj?oT^gP}8=bam-dO8>r*hq#?SHQ_|DfVXr5el{M*A%5Y%%w2t|+k* zi+v@n2$}#!!-_qo)D&L@Ng6>%hneG3frg(?<d31t_m$>U?4sRd=)lJ^0d6ECY6s!i z%!V(~!HxY4uxVvlFtW3E<R&Q&^orr)VdI0WRaqnIQllf0oHSZlv!wSl5l&=%Mv6T@ z(Cjco-o4wg-vA#RM8CF?%JTlaJw3Gx-?TB1v8)A1yAV~A?<=AF9XL_vZlyPtm775E zrpLkDJpXNeu3%RkS+8pi!x5!o)~<JN8o^<|C^dE7OV5peR=a+IE#)dRX$SaiQPhH5 z@@3<)04yAiGl@kQbwvY2RsPT|bL@_3L@l6mF{<H6SwQ&%2U${Z8xL8M=?24K`990S zBWg8Vom5+iI?ylJFt9o~X;xi|eM&(Yv%5zT=g2SR(TN7sl-k^?WouDgQPjh7g)b$w zw{~$h(;Q~VuDzl6x?E;bZl3C+6yBqW!L#=Vsr&M#DXhv&`b=Tw=}I<%^j)<_S6bEf zRe&ClFUXXkSn^jnB_3whm=!iP?bg;EFbw&YN=p#I57g2xcPaJzzBvxakTZqfUlB;S z6;VXUtZg&j5ZGAlj_*<Hp?E!-=291&&~MhZq+2As!DX7LI#oMcOb>M-Ss~gzbvmrz z+97B=vVWa}JY1!uy?-Lnt{^2M{3;1^Ix}|q5(<|$K#Kt{u{8iQmj|{R#$-Ls+AfLi zKPI02(ZQOxt4!T4lNuGG{dVMRdnc0PiHbM<&Hf+m`7C8r`c8hFVbR{DIv%<Gie>VU zd$%CNtI&}gQx+y$86><bl;t{m@U`=3e`h%VN>*AwxqCHawsd>_RyAH)Gv&E4byQY= zo+zC2Q@h}TI{nV>w8K^vl831NA)adm=Fen<TI<Wm@1O7~vc3{IEhsiX5I?r1T;%N^ z7x$=;J8mq`x#&lU;U0$$5IR>IN@mi4BclG!?;a}lYH0_DGW%e^T!`Nx3g$BaU|pI7 zrdlmHG}6#Iowk^Jkz!PXD(fknHX9Cmts6(}W^F5>fjFPvh3d@-5Sj<#Skl`a)13WM zw8Zh9J-muTELTKgNY_}<?c{z=*@H*R^K%;pxu+w08EBQ8go82uWcQwbI8$xBek0G? zNLK2<%eF+M#?f2LPpt!eDK$|iH@#|c$)K`-BeSpQUNrtIvTm2ty{_*m;opm9RN^Gg zMSp4v6B0D#HO15deh^w;YR%;HKFvsCGYQxooT^iJ*_~XD@iGcNXumw0N%hpZHdk1k zUSl9lXG^DJ$8`4n9&}B??xJ+3)G)+hrr+(>lC$g+Y}_C2BP?nwb5HRId@2b3xVB{} zBIxtv^J&X}HVsVIslI@!CdD**+kxME`mBe>9#CP!G0%@AL-*krOwI+yR`|=Hgi#NR zjFZPUA7O_}o(`<r%5eKNq{uJI=isSSJy)1yHx4Lv9_dvU!v+Q8<5d*MS>B--z6$Y9 zV9wpi8lG{cR9Czuy^kQ4>iAEa(_G}AzGZw(t3v+%%O95r{^7(vd;0R)(}hm@nQ7K3 zF_wv}Q2x@9-`EJgprxmtqtE8;kGSaTc59pwg)|{%b4Iv}E_QlyOA0k@^U}shHB6Ge z1A`Z@*s2H}vJR#S7;4WEA@e8V8?toZ1<o&?wYS|n(HUjZGEBJa9kS4z*<VD}_Ig|% zm~>Q_VNTXhK0S5Ii=ls0`6=eT+|O8=A-NIaRH~Ded;Dj^G+Braymy~s05xIG{CZo; z#bxWVVp<+EFeucmw$#kQYllwzA`+KO|0uY!>jYNN=$T7>DZh1j%QOJGZ3-=8%YXg& zEeK(CHq=<!WD){k8(+@^wZ*CZu%C^+-{Mr9xXS<3KI}~F;t0nUwN^9x{&nxsf?1w- z2WF;kH+m(D&7hGX{HM`0@`%k{b}TFtBPUUZ;<<Y`bN{+K5Qbk+JhDTC0K<hYN2GZx zT<GY{hRfE)=r`J$9m~bbrcca?NmhaMkgaZpoqd4p@eX^(lg$au*|*@n#WP`S&TV5I z1d^!`9v-kJ;fU-Tdr2<Ti|<KoOB+M2GU$a>pnWg2e>{O$YhKYD_-~8b?Ps7pi2`PA zP+%}#DnAF|GQ{CL(AOa}N$tEJ+mY)&D|8RT#aMj@XwY<NpUhcP_~D%CQ<{ZKCTkFz z%?P`s)9Hw$ur0Mu{1lNWM=T$iiV(g$mvtCOr4DE+R4tBoZARdz=&BC2Q>@Ff<@W`R zu*Cd@#09}N5qNH9fa1!%#3DX`a-Xpq!3FZa`O=9r{m!`BJ3?S!Q*q#wd|g3iTXuKS zJUG~YQ=Fi<P@TUpLmme-E=<rfydq%;{OGATf9Q$W4-DDVZel9{x@?l3-EU+;h;`$l zlH?XjM?N7$6*4`kw4-El8`oMf;Fc3cjAitvN?z}Sh_rtq^8ZMz3ZGi{2bAkCh`q*b zRivHf?I!5S9*0^j>?9%7YJ2*hA5m@(sdsKac*B!kF>~0x#IG~$e6sFyu=hq1X1hP@ zQU9k72GrMEs9l_36JxhiRF*UB(*S*C$65k5kO*9D5baZ>?{V6SRu(U*tL}qYXq7TD zCzNmhzKU;HNH8Bz-f3BilN~V8SDcS5Lr<J@6-~<6XoKr^@K^s~cicFb%+ED@GD!1} zQ?gN^etrLcosvKRsXs*>^y}iJU=Vp`ZJiHV2RLXN*u5(m9+Y3R-i$j5bkt6Q*P2)5 zr_RsC)JsV2)W0W=oafg}x12!_c0XnARnHxBw<(ugxJ+N2awfL7!#$0Z4{|TR4#pij zZ{>P~euL3ZdphEwTb^|H%KupZ=jOu8Kxe{r-u05Xa_Hx$*NVx+wXhyK348yX)f{2E zttxRk0a4(yNSECqu*_T0h<QneOn^RU_bifZ@zzCO<`m=+<=k-4iAH;iCtpt8CLS~? z^x%yR$cZu5LAi#>TZi~3LL(Sd=r-S3N_!FLH&4uN6~D?;1u>(ajy?T89*s9-SJTeA z{3@o;9v)S#{Ke^V5}Pbh_`#Jh|F55b29(}w0EAY0X51X~Cfu2`B&ZEDM?<dCdh(m1 zK4?j4`2Yb9nF$xhha&)0azTIOi2X%+n90ys;0!X+Ty6gDz4zn7hXSQV>c<l18p@?g zgRx-T8ESKve-lNj2j%hOqordvViQ!+Bp^pD=G&xA=3o)cdLFy2EVbe2G#zB7T?_0t zf3=L42vR_FfMK^AZwXYsLT=2+n%GBkXs+`2^g^b{<5XvH(l5KgITZZk8=dF|nyad+ zTY^ZMLt90*4L&kZu&=4wPFS^((^II7j$GxOha~@BPF*ERQzSnhHs-=k^*vmVRqf~$ zv5O8*G*tj-6L;KC%$U{EWN4W;u*C(MQ)Q~LhkQTwi8-qLh#4Mw!1;l6CV7WOr!&|5 z&6S%qpOwA_)H>L0ZvmMlV~=+z82lwGFj6&~fn}qQ|M7MlOQ^YYYn6~@0yt}T|0yKz z_r2c0<QsR0+onbIAu-LXs6EM@U~3GAIWaXS*LwnhRCDp1@TH$IX<Jc2cAlTwkR|O7 z?m6MlH$wR$=5Zj<WdxhzO^U9fX#1Ow!t%aIM#Ue~z*Ov%weG2!;C|a*Cw?&;W9XAN zNb9V)%)<UzWG(A#nqqCSk;}_`3+G>@QE&*WOQ<C~igDUhmIaa-VQ0#qs;&7ZS(eQn z!orP9)%e{~e-UcU)jg}*X?>td*{0ZdH{TH>>`s@nwuE@bFGz{Bwj>WH*^u+0Idt3k z?;JN?8r5+1NuRa6H>4B2iXIw&d!&2f&TyBsIGcIWb`vJHpCB3Hx;arg^)27LC1@K; z7fI$ITy-<D<!xZIjonEAOZl|P%e#fN3EI^n!G7OW_!<rql()w<e&rIUSL(<d19$c> z_GXq6uzy-$Cj%8?0W*iGOQOeT%S*b}@QyvkSv(`AtbV3xdV&9V{44-S5KEBrf#C;3 zm<WJdyClFI*t1OP*M&eMYpS_{?q%95o>Mm#(kfTd%wL*X9em*sPq%I>neL!GE8jUS zU+R;^TI2x5c@np#b!x~ej|Z|))r{tKeXI8U$)>bi9t?fS{}i<FXV@8j{dat5P9^7m zE!Iwve!)*#)$u$}taz5>My;kz*e25v$3(xJXvZU0k?I0X@B4n+?Td9O-Z@%Vm*w(u z_Tb<-x}^~;MUnL7Pv1D!Nk7K}3=a)O5UWYKZ_=G*nX#u5>*~pVuveQ-_ULo5++Dtb zFB}tn_db7oV0ozbp93X@94d+cRh?OBO6pY)aGBAxniB=5X%0J<a5=&hTV|&%^Y3)i zi8(%JK+%_jZQ4Z@R|e85$|v|~TH02pue<WO_~E~;&?AAf&4Je^b5t-anLn6S5ww_V z4-j)tn7{66eC(O`rRR+SRvK1{6ovpFmjRGC$5i{rX$X+eiWVwH$Za;N2?C@(yb_2& ztIhE~WBCxqiD8d|nnTj%##G0pDehK9@eA42dUgt4Zm$K=i|Ph4yVOA-T*`s%3jRho z<a-JdZMo>HiQx|NVsVXhxxbAaTSpxb=ozpNeWG4kv9zIz*qf8O{eN5li}hVl&V$Vt z%2Gx;FqW<3xqPED<Y^xlYd*IOh&40fvWrYr=#L;LGKWZlgI+K+Vba+V5jQ^b?(LrG zi_pNjRI)~z8{Vd^pr|eAi6d0SPu1=4?Mf0=^;)pe<^ocSz%iypA6a{3S)92v1sFf- zv6*iY^O5MLY!o+h+!AimZHvOpUecpRLMFz-=1U0Yn?$8F5!m)tp;x=WLPFcu-PVB5 z<Q1*#UA)Ly^1{oTwRU&V;8u;o0`n3*dZQzVK@`9+g(fbYw4BnR_;h7Anlq38JZz7s zm4H;X?cZ<vU2_D45jZZ)2ECx0udQ9T=7W5x<(4tVufXHx5v3$2NhhzCA94j$zQMle zX4r!>z~{QF%p5X19AZDQ`SB1yt9VL~cADl@=_`ckOmjnzs?&MOPgcM2o?G1_2LhDu zwt-f(!~*eA^=jkyK7zA`rK%ODSbl~LpiY4wJ&W0-=RdCe1n^<wn0|c3nr7x4-H#qy zr2w0qcY)96IXe_65+CWX6ul3{h|(<-u?T`B>&r4dWi^;JZ3G8zPeLvde!ODfcRGCj z2N#~`Y}3HS$mN3hvGC=m5KQIc?(m*+pd?h8B4a}DBa}EOb|%X0u#pk3S~TG@mFdG- z_;pLql<5?}4(D^FjD6e}1Po?$P&5E7aY8kQ0^EG*>)g$O{lmb@^k>{r%d!KG54v#q z4ZaL>S=&y?O6vVD_KG^e18%<Kv`-Vd;HN)}{9CMURnvGXC^g|e`*Rn{60aX9{ki2j z8?jG)!2&+{P>13_d0Hw+1&IZbqeKuQTwXV-H~OHgZgq1uHR8$MbyVDmPUGDnZ`H`W zMp7;dY`qU^iEAqwjq%XFxt_xVJN=z&rG^h<7BjZ+?xxn}@4rUEN2|k5te+6umM$fp z?(OAyo}<J)mJ0HA9)yk){@uCGHou${^bL4YQPJ%RI$Y`>X!$oUUFkXcKSeat{j4sI zDlbU`A*}ZR)iGq;+Z?h;8zWEg^j&ais%k)&|I1Lykesm40@4t2?7JMrvw=wo`Bg$< z#}qMm6SyE9l7Lxd5`YHh`+9~Rh2HdEpEkgU+<Y15rl1T!o!jludKU#}3@WP6Ltbq= z+;t#xTfEV(TWb8@=L^jT@@GKXds{6~mG+Ow0IK8{N_Aekkwn+dc`pa3O-({&S&^b1 zAKCYJsQ5f%;7>@^ntN6$N)L{eAk@F0di`bVVIcF@INr@e(4-Ygp6|&H1WuV(&9tuD zs!A}L!byKv5ll*u1Mh;`Qs-F?m(&8`i9eiOnE4kKs?$iMS;-yPigNlwqI$?lfI6Xa zX(5{Q`{kd@Q({TAY!QQ=ENY2__z@@{B$1$w&H!@t{rwD-zkI3ZKjmBmR%%sbkA7S2 zNivkA*5x`%2dWgVU;PMdUb#)+RUk|HQlrXGE7yL84Q0GVYNGf)mlW-odvmOKI4}hA z`jywYS3-CrLu8QXh>Y{lqXCM|;=iv6g4dpRk;}gr$uprfvG1~~{DPzpM;`OX$#Kh} zib4;o?;*QpDz9@R^NQZu-Hi+XB9s<$Lpz`LGMg0ZNlv5GSm4UQ_P{p($L&CV{+ZyN zi9ZvW6BDY(+n#9SX!*7RNt3NPigwMA>kDRaf4**2meQXi9W)Z&DqC!e%rA`mG#~Wm zwfnngdOASnpn_kD%zE<O*jA~li>h<4%H%V)jhki1?8ze+!^woiyC0ueg1!UO%bfh4 zt0=7Jvut3|JGQP@@%N2+cO6}qyoUq8WsD7yt_u@vp@b$WO6KRKdt{jsd=0pMq535V z<+gH7$cIWdp>NO-H~3MKV{PeJo7`tHtgZ1_MzVA)j_<21lbbI%`5beo35Cpjlh-a) z+xcLryoj!6T7<eWWj*4wEYE!%TC*awL0CCjRNH))32Ql-gI)@U&NVrz`H{7JeqDCb z!YuJE<QD~B&FkKIcrNdEq)x))a$iwcU+$*|+BJT;a>q%SkCV`k(X}fHqD)N>sFHSr zBcb}Fd^51;Y?ILfTR}KxauWU5xg~q)WO%~Wdye^p+4`r~%{)-gwTSR1NZ)<g^QC3g zF>B^O^-~i1*~J<{ly0<(@7}mJGH7QM;2vIL>;CanX5UwD>g!Sp!xzJ_XkicN%=-lj zB|MU{9mvGOAO3ofHJP>?K(G>f`uz{s0Np|f7EFq&i=!g=o-z65P=E`^+l#$1@EDjP zwbhcn;wZDvt6bS-%Y|Dnac&<~m7Jz~jto{xd0A9-jz8{n4b*?Uh&;p6fYEWmOq*Z$ z?Rt9^iI>7tqx6oiH{;N}1V#BMJn>h8K&!9$?rR-sg>Nk*aJ>n-<#lohnp%l;?gHW} zdC%gbhq8Ulz!Z$q=$D|)l%YW*iX@(LEFHp^Wg{Y!>`ze}{blHMzB^>s%VX1?`JW!F z;U}RLogKvT=-{1D@|*hE9~K8c8Ch{3tv24=Y|)h@{Q9UInro<xIx0^KWM3aSZAkC| z4jr%%04GdJrZx~hh!_NQre?I(g4on)lhgEHd?f?F=DpJ+#1`I1wt5{F1+k>l_(;L> zW!>Pel#IE=J&t551K(6$9vk@gx^xnH3<<G+Ioo(f2%w8Jxxc(P9%8(tY&aot|9sl} zr$@TXc1b0?{Ja?dDaBjsTni~llk*tadN7-E$A_7}N9jQ0RKNU8c*z$=gPvJiv8H}9 zGebMjkakr#N%y{P^7F?-8vpemipl5^dRYkZEX^A+dUIk<3$90#BR6NkQ5p`BZ(R3- zd!tM?EBj!N@Y2G9PH>dM>6KTuWX+4;mRXV>RJnqx3RJ~*rXKM1TJ<RW!1Qmg(hI21 zLBO5OLdn>ua+$C6(DugoO0oK6$zV1xFE>KeF(52bA_nqR*261m4jD#4^J5ncww7!< za}(8QpVE&=i>)bc)j`}mnkrVm^w?{>_k*hG@7RO;Y+rL^zL7N?hFLtfZn1pFmBJXO zF*6uO#{D{S%j5t(wNk0|VCPDu7r;sMGCZa4-Nvx~@+)%WT{h*h_rQL>k<Rpd-X4sX zoY&aQzcJGrL8mx!Mb=-=>+RpN?kLu3MHTn>PvtlCduS|?iS3?Bis9Rdd;x7o;xs}J z1~b=Vkv#FfA0>;#tij$&a<u6XEsyis-oNAy25A-FpE9K`uox8W3`eC~`YSOs1=(zQ zL?&L=L}Bcnv^>#te1ZFLV)9%4_+fD@sz3$bL6v7`QFs_ff3G$!v|RL3@0XD4VudHw zbpt~-_atOJ$JnkqPA~Whr>=MqH4M0RzCMO;fYx>_Bn7-OP-e6}H62*qAB&;QcEyp2 zOPq_|f7q_paORc$(%RccqToIC5=kCVd=)Gt$+0M8tfhNQ*vDdce9_OvMlAmFHvQye z7T9ENk0;QDUiDVefCJxcWBkv+uj1Do;^A|7YPk738Ih+0_Hpz!D_d^T$%%sfvoM<5 zrtXsH?;DK<C<|*Pi9#54Qczm1`CnI|5Emp`!3hTVeplN35KvNx`<T|q8h8xC4U6+Q z^Kbu6cKofU<N^V;U3u*PVq>AMx->KJh6T;%9gY#yL;yRk_R47TDTyw1#XdFh7dhm3 z`_J6HeO+pY+b`LLt=MP{*<idB&zF9ni&V&scXu-CTwfHdR*1xS`*)y`QW=}NVjcam zrl=nl4iT&@Qnljl2BJLXx&EOH_e^V<yAdwK$HGL>Hn9Cg3R2K{bh>m+`dokS+Yg+D ze<1$uyYog#VE76GPN85Itxc05qvx<v6M+}S(Ge*gQv<wMqgGdM@NBGbZhmn8vhaSI ze3Y<Y1$3#mn*BFtOuv;F`l(xAOh13$3fF`>l-ki$0+-l0D_%d{M<HtWm$@!iX6&JP zVOIs+oIPtksHeP>?YNCH30bq$)yEyFmnX^o?w4ZeTOl%CC|p8O@rEywN;6@rWYgF^ zaM<vrYTM?ZaMK5$0`evhek;@((azY$QQiz!jN?7D&8Dbs9^1xo9j(`@Z#InXRle9b z9<J8cz2tKc;kUa-06?3CtGW?MY=m}D50!>K|Gk1aSOr<*!^`avV#Y%qiIx;TR@dRL zc4qjz>_yLanx2zQ^5PA{L%%~TB{X90{37f05xC>7PLnr)B&+-67Q9!GeN+;aXaMOG zd6i2JHu_Gqc&%V~p6&gUm>60(E-@DB16M}}6i3z@Y^F~I)NmaNbYJ+)&s&g|y2RI- zg-<Jn19DnOGI9~Q(6-#%kopT`#m1wHSn7BG`-H4=SAu#PkV@<z?FPd86>ynzTmIu3 zLMyG+8AOZm-c7H}18}ndJLZ>|e*^OH{W%2>&LgchlNw~wAwdsY+mk*w7tvIqaUZH@ zN<BUE0p1@YIgY-Sk!|Ag*TM82BlQPa4%d}lQx_+lUHLYcH>yMij@GCNy&%q3?xJ`U ziYPwAd6i#Zmb$twTM;>xKtY?ftDrn@Z++$+|ClKo3Srx+mv<^X^$jx;Xf>a=$B?l| zyzabGe*=NGzje+B4c=Ce#v`E?CmLrH-d<$U>zeWhCoGcCH;I<WDd&ihxCoioU;B6_ z&roPfy(2%ynR{GA{OsygKfaeM-J{qp165JCclTw!_7qx1m)<w{%&V5eqJe|loDGq1 zOJ@q2-%#Xbc6cHh)9u%@@UEg|Ihpl0K9r*-W-dhxBhX<&Fo8x-Jq@pV-EMK=k|+D? zLA2%^Au%jb3oo`tdyyXPNjS?!3vum;MZax8*aF3yDlTsl<@j9orPT*De@Ba+V@eCT z^@{n{1(*@@YQMt%vb2<it$g0)2v3yzL!7hs*n-I3&>c_5Jb;>fr+|qw>&<)X21j|F z#NP*6=vJR{%ub_nRpkOLpcH`w1bb-DN&_<i2eL?<w!Q)Dvreu+_t~lu#(}?0rF<fk zcRXFKqKxCV^?cK%Ze}I44`O`f0_Ogq|2oV)FiOCDH_&Hl!2G+x#nl}(Lv@=6<R_DA z?xkIamNa7pOTmod+ufz9mQd{Y9f!Vz)K567NdW^1(8OU#->KO043$$~hP2=}4Z~wM zeTS2((&8iEjrqL=C$W80w*^bFTaqaC*AHBb_O_50cRE!F0eVvom8>K~D*k1MkR_({ z<X^Yt_C;#30-AlAcM3H$m^wfIv6rxBe<~We3gnsm`sne}V-)My4{vHj+LO;CDhxVS zI0=bU3>OK)qNI{y&ZKpkMx@}%U{s2t_#t|u18cXS{iyC$e}J!seyL=4bnzbFaMo!0 z(mbsN)5iFMzwWO)pWFgwJDH@T)f!+OGSNMOf>c4}<ZAk=V&Rx9#i$z(q}0oh_Gz@f zf342Fw6`)r4o#UdEq^yyi#`N=(5jL?^p;AFzzEM3F^@d%k16T=Dcn^2h2I$WcSH>9 zQkY%kaPd92;{d?HkR8V(=+6lZdfAG?7*hQ-Om*Bon$0$=Od&#^@2I!9#7MC_{dI|j zPQUV3t)44q75x{=6gK`_rp9DR^oZ;od_mO7Gm44*le6))YraDo8WfWal?b$nhe%gC zh1WwDMYe0@<X;3?++S+ED!LeH)14~N@M^^0KK=x}z?Uk&9jMcg_Z_xc{4pL4dG}}V z*yk0R#DAfWv@B<6JjK4{X~es~obK6Deynpl#J)Pgc<3T~{Jm_zuZGwPdt$DvKddR4 z>*A?=q`MaeRDPMjmh4d>*71KQ$p3tGN~abB$<*~%6HOgIKmKcWQxzQ3*Xig^kJ)K) z#oiqBc|bk3>>$R`Tu~)LQ1tf|Pye0X7_;$%W-ziU<8<zd`_F6bk!=s<XHj!csfBg5 zZ*btROll1H-I5i!9JAW{0=485mc;()p>-X?B~fLMmKRv1G5)lf((;eAByJf}e0cgT z<EW9Z(vJL8EUQTzkGvUMknOn8772ZD&Qha=eC$+My^dW85NJ=ep(J~0@#M;tp}#rI z<aC`sxnaEYf?cO=*yUjz7+!i^li$Cin$7HtePghyXP(IZI%{)2%hPAxUUzfk?3&YN z^ISXU;BPr<asR<(aSs)jUGA-W3yQdxFLL`V-tFwenF&)&(cULH-vas6>Q-~nclUxh zx)vv~-EdkmhZejBLkr%oJ*1t^$ze?64o3>-yEipX@1KIZ?<q;tO6&PTdgmyZ<p}z9 zKbMvk-u+CGnfXQW^lP8la?lueUwu?^P@y7s?{IP1PH|MC@b6OUx3)wxSiEXVX8<Q{ zH>)wxg8Iz-2l}hvp1^F&HRij6cCqZqawkh+46w(n$=w-?>Cm?<uuTf>8`%%r9?JMw zr$Qkf0G2f@?b7%MaWmAVHvdI67BhoKMVVliYv{a99=@)7yC&fq)9DV>0pk3p%UWE@ z7Bo7hxVzIF1n#~i<%6~ZbK|}gHQ@8fK1)D7N!l#D^~`rGn9LQt9a~=8Fv4ejYy+17 zofJP)z~$~N3V2I{s+IzOFPY<N%Su!dKe4MPA9|5cr2*ght(TqX5iPUcG(Q4wkp{+E zH?hmYAg(7&UnO^{g5|SBlX%kmA>?;&2t!zpzKbN$Z&V@DzoJE|b_+UwPA7b40Y@J{ zKa}tT_gbv&sb8w@))@be7yPBUgzQ(k1m0IeAEYm)W!|>8727(`xKm8dQW9SIOcDuX zue8=V=gv9o(JzR}e+*7V(O`8#-n(-&3Y*=K>+7Nwm)mS-Bog`Pa7iQEB)V^o2dd70 zQ8p(rMkNWBb$$d##f8p>UBV$1>gVipu?460aodEJu+aQ|gS-{q_Wn`TbPi(Dheocv zx@SrH^C-8zjgg6#kQ{JH1*J(bx8`zYYFS=6J+FS4J)A{`TZu|L)k_X%zr_<EHUln| zE#OZ-lSrESJFevT{oK@R%Q?4&vCtB)az<L4PfLq9-fYE~RT8bDJEO7=BA5HKXm$c- zlK$#`+`_9;Mj_jU)jHrw>&`5|H3>LPwigQ)&q~7{E$^S0lHEW0LiU+#3;ii$XURaL zoEHJL3QzRaXyN5cx>OPl%pJoo@Y&7a6Q(wWTLLQXlExQGAW!{eW8vUO%d@=gsaxnt zM$Ae_y-6D#NRH-jUw>Ox{)L50?vy0ebh+KMFF59Kw$X!VLE7IdiXJ~-iQ_BT<c=v7 z&!&sCNK3v?@t?=noWcmCWTf9HJzA?atl`l1Jw;~i88PiJnVC92sFbCO4U}{`gg~$) z^=x$bCvFE{Zy^p()z*tg+hOEuHv=V<jmW?Td_ELSRCa=j5g6w`PdCZm7PU~X5jK~| z8DLUE*g8F_62lPJ0CF7|Pt{J-hJ@l)&2R3$M&iTPJapL9*i549tYfGy-}E+Rqxsn^ z@OD-gn|QLTM&4x+3SGcbS=sTGh&5a*9r7`B!Ifs-aW4)}{GKS2e536N&!RX!obpg} z&i26t`xN>IdXm%UE`xxSX1Kq~wuyrdsp(9OItqW%L<L##d7x9>r7w2%s;`zW8~mk5 zmK|fXpC-Oiba}VVRfUKWuY2h6Y=Y&LBoRxBORoj>e7xxDYaTu_YG>?UVCU(*v#7#F z0U`bsD!0Y6aP^JHq(k|>-fbCf!|8p5ohNHRaPwxeUQ@PwukcKK%by)pM;E`1m%vdT zeYTG>11;;8K6=*Z6-jRh3+MY2qCDX}#D!sK57@Fp>gcgDZPfbj<pdK#2DX}2H&E{q zpZ2vCVY5(tLm%@)TF1Pb;KabAIjo+KM8}wmpa=Y}+Q3+;MaZx(?aeetSVs{x*q7^} zC#8>XBm8K&?T5X&ka-PN)bhD5eAG_vnCdaVDawxkI_6VKtKD7x$ZWCuTywfji(Yg8 zJNts>X(6_8w#m*(1SYmLd<J=ie@ETCuVh}{sI{NqxpmiZ$+r(_$^5K8c3*g);A0%{ zcBw)6pRkSH?K3mQp%1}(vZnH{6yHDbch&NEYzG*K_*<cj8s7-~Zn8`J<d8y_$Jvlb zLzA|9&<`X9s>oh4yQe)b5FD82Q$hiB_fqJ=`3$^mb>tnqGe7S<KNuk9Qsgalpc`4- zCGN%Cc|b*Gu&KT9Jlr1HmQD}$ZU5PBuvOvco$yvw$=>=@zn3%PHy+7bZ?yXSw<{ao zdw{U8g<e;?^>X=?7rf8vX)g;dL@`v^8$xd3(*J5r`+v8$9%vN+XK>8fZ!3FDP0Ctf znFxwYAPP0lue^2deaCgSEksvT+K%33%BEaJ(j$1kYxoB{c3h!(YVi5dO#F)?Qw%<f z4&78mm7o2cMJBognZs)7ouS0$;P(r=k&s#@0w%26X2_q|E3UPC#*fiNe^ss*nrI{0 z4HjZUDmTiTf4G&Dr5Sk^OFn~|QQOxDkar=|`(jivR#bp6?Nch6N(|-XE^=49ur10z z_^VS&mQCX=D6W19JbD>eDrvx!=A1|I(nktZd0jqAEOrS@J&wL0lTtyN^Rg@C_e|~X z_OP}uX#Ju4+xSDv5acQb6Cz0%c{YZa-#_6+T$!r)aa>G(yb^W(F!VdJq`0k+P#$L~ ze5$L24^LdkMv5*ty<=)1*%5QE?%yEremuV2^KIuc$!5Hg!=|st)-Tmgp6^V&#o|9_ z$UootYEe{EX!w&kj;n)X;%4*lj|&!bU`)<ip-si?95G@7CbP>XjSr2>F?Jx3u!Fu6 zzgxPe3e(o7SDk1fV;B1h`fG7y-y6yumTOtOuFJ-M?ju;Oya^%Ch3!3L1X%{H71(xC zak%5DIPjWTI7^-$%cH+Mz3@tqB(_?}bk256F`Z`_?hm*QHs;Ruv7SZiJ^P7o%9~5G zD8ahp81kQu@=4KWvov4!^c!vX`kj>?{H+$O054@r+f$!y^fTKY?B~x#jG;Qxb3-pJ zsPWP#-_#G+|8!^cirV4k#6hL2aJ%I4LqS)YNM<!AvT>7k%Gb-3u(&O^xp!F4>x(X7 ziM5;(emw$wv9n(`)ruroWq*A$n_kLX7LNuufO0rN8N`g*ix=7yN6$5$WZoUR7EKL$ zgHTVIN-dLKJZ?h@VWL0|+v>b+C^ge%%a-V4b(+QZenY%&st5N5lE)Q7Ky+hSb?OO3 z78=MUHPl`k+^<c#W9(|1_xFA!?yuK@<l)j+op-mUId)|)bwkf}L%zb7!xML{+Wa9P zBhA1UR<T*-av4|-F(jG}+`N2m?-L7yF#0C;dTI9+H!Ph))8F?8mVtR;hM`xqQ+%&X zDo}#$&2NVeeuTWrh=6XsSM%=cVfgODcn?aeJL}kYts!%rp%cxS?2u;H>+{&hi_a!l z!Ww__4fB%wl`z@Mbp9Gn4boBA>PNB3c?xb7{r$J4f|Tb?%aAeIWZuxlSha@X<7qCD zo!EYM>%XV8-TY$>zpES#vU|f4#ce8}jF{Y?_#HLq_UFuR^0`e8HK3kp|Fl|ej&RIN zOHt^jIhufOjh1R2HEmOIkWb$Xcr+$Tfga)h^uq?b8cR6p5f!0mmlqPX{($Q4cMr-2 z{JVZsI>pfYP2P^{N}6@acXY`HE8|l(V?VkpW#0Df^?}gYR+$hV@<CXf`ZXW<>xLaS z%-RMc@knYL6z-W^3wJoPT>G!EMAw!t68J}46iN#U-RFi~cZ{d?hSYJXcg}tOh9t}= z)nrbtSR{Q&N4J+rN4TTWl5HGWhsrvkD$Db;!n?XhKV{B~6{YU}HZMJcmD#)*aHfIv zp>yQ?cQds7<JLl%Zs*dy#P~7&5?$#S{HOyP2#sVee<rwbj{IXKT={|x@kskY*%OLr zgHAL0NM8CRp9XFsH~py{7o$Twu+|+K8}ponJRGeq^&R%gP}_A>-1so>h5i#E3Qn_Z z_NRj3{<s!?%7YV+yblXcZ+PDS@pDzH440M3rL_DZv(~(f`KMKu*#?iLdK#y2D5C4J z#u+hg@LL{dZF+Dl_40kj7k6^8v3z1OY>b?@^WrJQ*kx(HBHWec%G@R8V%?b(^@_{B zfK80W5Z`KZ={uKFo}{+Z(Mr~OjMy^4K5eLv9J98r5crOijgLey(Vz#$oF#M`kAI!^ z!*2omlPxs*@w<z!D7~ut5(F$<>V9Y=KB)X+xl4RASpURgBCxOeG=X~jm07d$W|iUb zWE6FXY1Ii=+Rdf3FJb@v2r<53gqYt|@%R)u;Sj{%s{1&kO#nwFz5s_Y7Laj%?ng?{ zPEaYDA#$?&kuU=<285WxVE#*&g^+&^qU9fV#GbN_DLZZ@%ctnCiTuP^zMg;M)h{<T z5?kJSG&xwH%c1cUmLD%4zN}m4@?lh*Qm~ntnkHtKUwr?YqwJl-KVM{pW2bw-<ZJrv zcm__dem+LaJA3O_ZDPyninPQ&+dve0uTnbxNh<e%)Dzk%Wr5Dr&1h<6A6hBOEgLv< zJQqWvx)f*B%rOIOch{t%cyMLAWnbTArX_GoO+tNb&@O=S4&HI})x_P1(MElk_oLV$ zVJS_(wTe!n{+$1^w#|Ei@XF=ToRPmkUei7ibB&B$ZPtWIy%K~dAAh);Y1b-ewSRh> zz^VR(N>v;$Rl5_B2)F;0S~lbs?50k6=>+7^rxt^J?!*zF8Q^%K1!c2FeDLdD{Xw;8 zhqI%MS-Nr8j%&x2X&x_)n<q~Kx=K7PqHN?j8cI6RXLD?4*%Y=G6>UnQw{zd*S+!9k zdHroopPc-qz0ljpSD6d1_+3;ge`_yD?&4@-rj|Wl`X=)ho-7j=_<}soOud%S&8+_3 zTZ#2*<1}wecMxq~{G0j{i)9RCi@W)S)%s&)sm_4$IClG`?4R@Y>#}7_z?IDKrtY=> z^8(cRC8R`0EU2&(a47-=lgY#iM#4OW4{uXWp%@-1xnCU@Jvn{lqgD;)$~(cfajmV$ zWhqAgZ@YhERnsD99;j6%Zv-;{7b#;?=w=>~;0x-OcgLoanFbqmpVm!4mHoEAkwk<y z;QuaW3YhULFEPg=IKM2<Pm#H%?fD`gpPd=g33kn>jH|iOtk<LdC6u2l!#FddC?AoL zA>AcfK28NlXfbpcv2~fS!LpMd$`-np13Z2k_-j*}N|W()W`0xo8(E@vUs2kidVj)& zP{$4|NnGOg;Rfa46#`Q|{L%Eow@QtqQCQ4OHKEig>jmF(S6g<~cr<VB&%H*-4KOq# zOrD)zo9j8mq{<o)Pq>NFbw%WA%UknzI_a2w+gBEFbP37rF3{xhhK&zzs=SDQoec+7 zP{fP~`3K)bU`9XC!W*rod#oTOr>eu*5^`O^fsD-ea_muE-xN)_=$GBTe8a8u;<Lzc zr&WA~PF<s*krc6pD^3ZcO3_I-Kv^G_GTk%2IJ;gv{5jwJR`TcjLN)S;3;E%-p7xV| z&$*M8+-L!<;i?;+t%tb5d>!W1=|<UCo8w68>yj<_ALn5PR4)B~iHGxsoeezlJ|YKd zt6!SVs5Eji`$eeLN+f=yeSAl~oJ~!KP+56pY4%Y;E-#*JIw>m4dN&l4`r)lKP1|o9 zXYyh@4U|O;y<hV0M3JicW!!r~^&@o-%|2Iks-X093pUCOil=5|8WO?fe^&fYL{L_# zAUC~fi1H#KBw=>nJ36$Gz+Zony{)_^>3n9CdZJ_)s^qOTz(cr$qHeUmWqkA-+s;63 zF~+Kh!(F1%tSimwRsHX|$j~kC5om%c<z(?jL&i@uZTanB50SYYhO@n}9%JSP_BM0q zXKu0Zbhftc;2V+uUcjSt+?Oezz(|+wJ{FkTecU+ZKT{miyi-p-MtwR+?Nd5U<vWBT zval|ZyNXRHJE9a*qU90l=Y$2Xaogy?(b`QBOlK8Fk{(lIM)$4$BiYrg$5G3_KX%H9 zr-gg6XS^WRJ7D}P9b)|}JILur*3Dc#zsYmNqzx7uin>hIiR-z`aiMB6K<ci)FSDC9 zP@NS{a0~{%*RKqkGITo@0~oUdj2Ma4TVG`-z7L{;1J&z@#MvBr>@?{`M2J4O7d3DF zuXzE55_k?h7|PQV_({Me%F`+xQiu(G-AqYlXPL%O`up4Ch{D_u?(((;ZqD-&%uUsy z$IPk+ETy$Lv&U5lmsMnL&D%2?-Rcp#y$sxj<v^%HiYDu7C9q_b)g?u*@hKV_GrQYs zA}M84hUwJLhtBN7cr8GQH~c^7UCD@jt11!mi!SJqN%T)M75W;((6l{YIyLP~IB3r> z|79uGP-ffYGw-%KPKMnw$_`ZVlL{OyBl(<9W@f+d^Ofvj(4q*i03H~^*-oC0tUNiB zVw(MY{VRQW02j||_k&LK6{Gk!iO<+`q$lZS+~9K_(N)t?h(}-sVfe=iddocwg&s`f z8`SE9qvGr0HLYG@=CiypUK8fg{1ajIk^7ps$?*a|*~wVz&kV!mhq+Ij)Qs&&HQOCB z>?WpJvw?hOD9=4tt9DNY*Q${*FKcyEqQQV=3s&W7qhq{)t-ccVZfzZEC?~Xmmsbt< zH`+7)Dg_v=aoqBFZ2pECX~DVN*&#57-CBz<XK%gvHZ1X%J}@Mq{1-nayFZqWL9=im z;-D;Lp+$++N9up*A^U;YDkH(>6$GZuFB+Q;mzLD_-wsY&kz>?Ao#zJrnvhyu50gVK zcUg^pAe&g{>xIKECxT0?zwYX#2tyvSX;_JXzIK+3KZ9yNo;iMjNIssOe93m_XUJaA z^ZEOxPfD2=5wCc!S-fn_d}<d*!q1F|ef_S6KD^)bC@k>?f=e{<;PVS&p+*BM)Q6;q z4rtig-FeX-FPuJQ&7WUV2gdPT{Rs$G2JB0HyGp|X;VI+ReHr}pNH-RjO~S8Vr=LpN zCje;!b=MFtJ*oxrD6(uUpZA%Y#hrwoQEipR|1?pm353)bmC&i(^qQIs_)tQZTgIZ6 z=5;r$SCeV+V)_!q9yuP@G?Ee-Wgye+gS=4+eRJCq<Uwb$A(~8=-y1Tzyg-RpuIHC% zcb`qap|ltz9F*p#&&^~FKUaCR6L<sIE*E^hANfWnj`LY?Xm5(J)T`B}oQhtM>Cg+3 zz*<Xgz3XpkoQ3aWlfb1^=*3Rm)XUm_Mn+=^m$C^7-)Y9+6RlM^;kd92`&QDjD#h~I z{0FkeYxmRKmdzP^c6KTS%aco2ru5l6W2t$$ld=qU>&8zcsiJzrc5aU=-H3HhVE8(~ z8<U*=xWDb4&!PjOfAlWh>jBI6_280->AGhL+2<?c{8IyA#=cWIQ%k-F`sc}40vdL_ z`M!eYl+M#jE<_G=yafw)gy&@I$$D9ox0f3F--9<Et!b5?2L`4Hd}D6(Sowjzy>hm7 zgqiy_m8IG9+U8RE_Y*_NV%3=#x6uE8GLpHAu4tq^Mbv^}T2*}?3(Mkfvv4Ua{Jo#` z-5$Yz!dF;|p*t{{)6Lai{ATl#XKZKKM=MlliraIRYIaanfUl|Q?uq=dnal%QV(@n3 zt(G$%UWnUaeFl-;vz>C&e|@HMXB&=icJ(1+5%}clfd(agZ8z5^U2Qq*iDevRJR@pV z4v0XhnuQcT_^$q_`ffh;RT4gwzB@<*H3jK0(c3CZ2xYSF0!Hx;3T72hz${!Ljz>YF z2)q?VMnT>2oaa^1T?O9l&R{M7F`HXrw>stN8$<*k;dxr#y?z?^#Nr?VHp8j(A)}{X z9ch@t2rc}VXv`4gDF*8rm~1z1rv4mOu$X+8{PkB)_S2zft}IN^1Da|*#3c%K%Zb&7 z@*2IcGbuz8Q>~V4jow_N4AjR&;3Es7Z$ZDWC03^3^b-!;FCjo$SZVWq3t!k^wgQvR zHz*W2IF;BECXRb<+V#xF!$EDL`-Piho4^tbo^C03QQ&RB2J0hHZp~-;!lN4rue7D3 zHFWud2|+U}6&4b`7MQ7#<r~&#L(BKAa<i3}sbOL(dP>xR7IhkPub8dTgukDu_>b14 zUsz>5Sqi%Z97X;$YCaGiQg1!=Uq6>~2P%SPRZj*6ef5Oh+F&oDRFPF=#fcu)x8k_f z7zS<;2y<?%^Xb;xcrB>hi1Sh-x!OO?au(YNn9z4&N|sK=eN<sjif>e{t&J14ibTKz zr9id$ottSCDGz91*>2HR_=ui{w%pgp-)~~S!t~q!=EY`%WcWw4cZHWbxJ#}1n`}&- z8<uki_aC{;yi5H=Y+2$mrbt3wM3jdB8me$quDK)Oro>_F?VQWl51oktkrXY25C)LW zp%>)gkGpCG4<RBq@^2pUO(B2$wD`sfGFca+f@yL+pTr$xQv6&(cwL!%j07J=yK_?1 z6rA#oDc5K9Iq-6C4hO8T!+594F~_?k6ZL^z<qv{#{srUh?@V-junC+jSA%{3Rnil% z@6Vsh%su*3_yZ$zggOmL*siU!?{xKGh(?wBcWA!j^Cb(Jani%NjZJ^>@rtylk9<}5 zCvi}vH$}z?*_zP?-6$RJ`!$SH3d*k|(;%;=D!(ModzM=Q+FK=(-DJ)+mAV>LTAxFX z^BGdQ&LjB$sCw_9CL6BXJJis@Mv;=Bw<yvCq(f-8q9ULmy%U=BUK0p4C{>UqNJNk( zz1Pq?2qL`&rPqWW`gi#}?>XOlCVw!Knaq&MWM}WSe{1bz+AnHBaAm9OW}`M4wbyoP z_qkG}9i_J#R{cr`S3dvpD7N|)UG-V@@<0194pi!v{9DO%@5`&!!y1W8RBTNn5L{FB zYv@GG_0PYok8NLVgSQ?6pjPdl_o&>)elI)6_<4&8jf9qw8u+;(1O9ot`DDd03#iWj zc>4F-R=mhwn+>n=<3~epQO?B3Frn4DA2);EQ%2ne8ccPPbvMvM`cu{3uc>!x-q;H& zyaYkV*%L0}`#+GCtxS?wEi7v-m5{cn2C+O3l)4x?#N5Q(t8<_EArR{}ptfdwYC_l8 z+pVsRj5xe7x>zL&95%nN{Xn{7SH?FUP<K(HdBwY{($&N&w{p}(LrgjQ(Gke-T~2h5 zHuDpksP!n;Kl{5vPuCOux+2>z|MvkkB-1A8x&6xenz~coc+0KAZtxw9%RcY0Z^)QE zyVGvqEcAVvp%tYSMwXntx4zIUjRJd6RpJ-}T)*LE0**U}h!eQ_SIlOk_$r#h4(?Sv zar;b&J~giQAUPj&G^yKZIyRYfX{WdUQ@a3<{9zerq4;<NH>P6D;0!zS3-NmzeOtSK zi=)1lQ*z5{YSa;sIM^m@1)rPHHlmJQB++EGl|hiRpfAD~){%a_O@eicBDQruzE+jB zDt>1o>fb}~Ur?N7TdjUXdbW0#rR|APvCg=yL&*nS6*|Cw_Wl0hh}bqfA5T;py^S|2 z`<0mblVZ<8u6Qe`I3TnoD`hf4#MyQ~%{pON&n4Oy@RL-CbseN_*@W35Kzk9wCF4*o z)oOhy=6mO>`w6cGUKo1=%|m%6qe9ZQTZ5UeQDiT^X=z@Xz9ku2WCeqA_9}A|4LWFw z?4Ttd6*px_)TSrD3U8BniVf?BY}WE)+l`Ebe_Bb@IW0Qw(^c)k)>TGwn2E%uYxuUF z`zi{X`2yx=+tp|aaa;YNSDUwG-Z(!HG!PNmKP0cpErQ?I$S|5I>GXV{s6)>)W+t)C zMDA$J7H^vH`OM|nm4WY|x8=ZBPw*uqIMKLNs6;uq?7)D_t`o_HSCn5)-aHBE7g{7A zyh0cU;mr+UfTF%QS<Iusk9b*5@C}>D?0h*NI5_V%*KX?mYQCOfPfo^Qu}Ubb!icQ5 z>iT*+i@%W@aU6!NdK`-$S<RN$_k?!md#!a6uVE%uU;FwLrEB=&e8w#SN7IZT%4<`= z*qyE4Ao`41o`c(S@N1Ebl0ApxAS<6tY|VK2b%l9C#Gt)!CaCA)RNp~-XtrI#?YCJA zV43HmxbsJD5#&twQrR;bh0?30@k#vhPoWUSpBEIpwLc08@dr7xg=3mmL}?vn>V!_x zSq_@BrG!-R=qlwGPgRN=CXXHy9x|u@2&;4BhP^%0Yk>Qvc0Tsn%8lWKF=$q-<#cgy z30a)=FK2K9L%K6(;D5LDvnSzaTvkERRTSv)*BN?OLu`R{^~u;9?9H{1u1O`XQYI~( zG<&y|G(FJ7J$r_K+1Zt|>7>9q1CB&iM;5+j_>T{9iX?IST<mW^C+&>QV3o2{y|~1G z<EM<ZHs6Y{!01pd)=@?r!OK%AS__k}Qkk*`PDPNX9&6)8h6#`zNbRA5Ud_zjTj|wu zbD6wcz0VCx8iRXLZnX!H-}S4X{iTfp7eQ4W@K`yWe$B1=t|&ILW1PPjU9AmM6gvj{ zoK$1U#Cy*3(Ao{^Snxr?+K<z|L4VmY90j2b)f0iL(B-+={|ESMVf>eDT3Xy{K9XX} zO=SMMWLIX%%{WmOg}W>Zk97G%d-}UbeMSS+DaScv%+9o}F`cKH+>J(a<}<uZ+^yev zt+WtH=+EiZWf>d<=Kk&&UqOFC1zx03mvb$L_J4Svtz;Rex=T%u<Qwjf<s%Zg0V;?w zx<~4<;;EDRY~&a_q$Xc@NaDz9981c5rNI3D8~gu18}4UP7J%ZkBoe>YTZ(JUrbOA_ zYb25gWE)W5>8I~RxIajGCp8(iaVL)L${8ROp*{RorA{v2V7-I8ACZwWT{TlF4WB7` z{tRc%qGCMtH(abVjt=LvDQ7+|uO!m-WY55mr!8SDCX@e!Y{U;H=aJbmf9@r;Y;%#; zl=0&t0(W7nt>~<I=LpH93tEFDxj{+;$$gS<9)S-LTfc%RS8dse%_-7cX2h$7-wlw^ z))25R^9Nm1dwR(OjqWz-?Z9br7{iot|3;+JbK6M@s(psPr4MgAwQlU%{p~Tm<Y6Z* zrsP@ETj(WmckwrXn1-ovA<1EvYjC}9hBm}yUT}-%rdOU#ea)!=L|(Wh!g-(%n?SD& z`yCvMEc*rK3Y%U;5=bNk^Zrf4NghCg(a7z6GJd6DrdG2zm`?n+3(FNvQ`AE9)i09! z9`e?Z^oa%dX1#Kx*tDJ}Nl!VR1GXamartC6JDP<0>h#>8OFLMWnxUel`?~ciwntHu zA<h5<-_f$Xk?#)6sH%Rfr+?aT_=qK5Wh3esqZ~MGr0Q9_(>}@VxDmo#7bD4!#zT;U zB@3e&(NPI2S=idVXfO`3^`4K43*y>w?U7{<liu60MDu>-njm_w`MhfDL^RWIdJCh{ z2jMOn2ztTXPkq6j2+WrGHG0HJ(zB7<5kr}7Q9a6J2>7#o4Zoo~jPjvq2<1XW)&P}2 z>l-Ke37@BIWv17S?oau07JG(xm2m9KdJVU}9`zbPYyg7YzVNXx1hKU*x5Ecaj|^^m zMaWk?h02>D_H(NC8U}`cu99q6>tiz@Cn<^UtwqX|E&68|6VVSqVTWN&ay>W9(aBk4 z;}+vldar+Y%erxN70+bPSy*)+`5xK5Ps3J00=kYthq8XUYrY49Rrsd_8J^RahgKS| z@{CVTNJR;?f9hPcKLnWzM$8Ro&$bv11*el<+nsg~l4Wq~41O~zuH?-NwNRhQ3waaA zuWOTiH9Al-8z9A$7fR6FMlVyrVu0=50(>j5U}uGbSZn7arn8#YY($jw+_J^8(_(UB z&Wr+A%XM8riZQbJm$AqmsY-5UfY;T4+5E=&=(pMn2EMHJ&>iSFhIca#jmMM;9Tl{B z6IrW!gc$XnmBAS*2Ulk%3&*fx>Vp5;AIkA6{|LHeDHA!@Q{!{SOAy(3*R5Y5<n=?> zp|$y+&o>xvl(jnsG~j@lP({+?{gkm(qy4t6-HL_j1;<gP(kCN^jOeLDY!>qT!*<{e zlkl({EJ-9k_Ijn!R~gVoxam_Ww&n#KVt+90=3L>K#zM^Zvmc0|X^A371mkkSb*19Y zF@2Cvp#vLgXEa~SJ~wrddTQG9Y`aEc?RA6mbmiMe2k%sW3z1>MgJQAfhYP33WD`H$ zg{QQuiGPyI%W1w`kV1#f_eucz-)mR%zx|CFb_7)&?kFkwYdA|UP6T{eKm+S?S|&}= zQ&H#X1!$<G^sV+;20c@LS>t=;5!>A@9(4<%)%UsE04@NvS_Xg0(4Jmhhs#0zs#|uy zcseZz6gVz-hqxJZdI;F<B570TiTAz?f`8F#t9Gc@D9UkP3V^@ZLh8`gF3qa%PMC4L zOy%Z3-#GF9-$C%f?Ue1?Jrr^7;yK~s^k@akS*AX2=m7J-X`bKyC_&39A*QawPpa$? zR$@$D^{uxyj}%dYASsGFcaZ}?K<GqqbtVzOq|xmRG7i;e%~e64y{4&)v8Nl-mhI-X zAL)ONglEzf?_IikNz!A9J`Oi_Wjr^?HRKLI)~E{B_Kbj?l!MD3e^CjJf-xo1ZjJ=; zRFFI7sHIYRUmgnjoOaz`SUkS660=V@6QB2iD-0tl2E{|3C-Afaw}q+j%~vpwsY3R` zS$8qL24j|mFNsfVu0L)UVLi6gr)OGeJAr<Q`&;Oct}0qw=s<#daE6ieu$j?>5%*cR zbcTo(%~k^*OJKy}9Fojd=aJl%oAqzD6FVt=AQL3$E(<d%VEWPrk`ET;CMrrGFhb9C zV0mh2wvzPd0wcqwO9aoyTN=Mi^BQR{-<73_AHUUq{tc%JVw3Myeojz|0|(yt%AcAE z&}Q9}YV5K3uVQY&y2OqDt8PUurGbq><<$Yd3C=zDcijaHr2{qkrK80ETy{*vjPhVv z$RPa=>BYi>iw#D{FM3{gJ$H0kKu6eK#9cmEefu%tl&x>%1=4M<`|!na{kb7o_&rqP z^zHAh3o2mc3hTj*jnS2u$=VcBNET?q*qbvw@bW}0{5Pec168i^IXkiPkfrLlZs=9v z^6BYL1Abwm`BJR$@^o{!VDV1r`39#se(ork%iiJjaqqA>-KX<CCUV+m&#^IAXhe6- zPRpL#HaGshZ>=f8s}94&8tJz`#Xd9Y;M~j18INx7#)gP3oxZc7PTuXOP*+_`flxs= zUF7!lyV<9DJ*hrg!RUJz4AG|+72g?)rPNO;n}^0?hea#wFSh+m?wmafq>*bU=)GtW zH6{Bn^m?3t#`xb{^QZEqqTyW2<g{HlH33fu$T-)-x23xL-VPzh%RL~;KmA_ObiT$~ ztLaO8vvx`Xxd9AW!Md9D(=I&PdQ}^=L{rHJ?vP71k-qT{VEuT;OntLc!Awu(AyrTR z9Z-mZngeq`2O;ScL%O@X5jCw&Qv8k4Ro?;$m7u*3{?L;ltdMvBK{yIvvSe5yY#h@L zy6~|<3qUzcQJy49m<U5}<CP0>Z_DoU>||SGe}vsXnOp>wj5#CV5;|8L)3Fq0?iS(` z$f!XdoV(%+>L1-3rl*R9S8g-8YhP$^zsr7&1X4%mSkf2YaoVkQ{ESBuT=~gg2CxEs z9Tp+-c2m{_>o`NKV!*g-F2DnZRSWU9@Z{)J2%nyAtJ}PO7>h#jnTyi6`TU<oa&8T- zpH$0V5q)Pq@qgMG<>|6iVv8@>rQ9qN96BARD^Za{(`;VT`<4=+QB03Whx+baq&l%3 zt+@`Hr#`X49(?q^t-ZnS{E{NP>L?oJ8G?9++BpO~I6#JJMnyv=UC$?u8!&$^E~J;p zEb@w$(8pJj5<dCad!9PYsXqT|z5^zn|M-T|Wm#7*AVw~AE$94q>jw#?tUToN8i>~q z<%2S#if+TkQI?TGR2}bwnk}0EL_%FZ{w}yJ5qRqnDYR|o5Gqi2Gkm&4$-C}i=qHka z_`bd`NZ0q|4Q`0!)%Q~4M9=Is=*01N0_S@Y-qd9Gb0P8h1`F2$;C3~*H{P9~>@>UJ z%7|p#tRNk(LnaekWALF}YC&-V-{agg{+HXNWouM{0p&1A%<C;tv?&g-gNj&XXNn-; z1xD~)Kd}_O5Y2yXW+qm&B*SLjvZGHfZleknjXo;!k!$ZpcgWg*%27_9EtW>Ghak64 z`{t0xb1obCxj)p8B_@#V^b!hbYSUvuqNEa<;{U3$hl^X<r>$yC6Ih8}e`?g0R9ub# z>$J;JZ&m922zcjvW$*l-E7(Fb?Zya+P>B_t@{|ge_|V7$h`>7mV^X(!oq@}c0`~ep zfbaEO_}+D|h^YZVZnmwgBr7$t&^jnAuX{trKpf_9x9dpf*&|9fkJh^>O{au*M&0eH z>bb8j)5z79Azi%*a!n#Mo~cod)K)ku`5yb}2Cd?wOy3dkgD?)vOlWt2eHUymv0wpS zpkYnAdNJGuLvro;=Q9@je;>Su+S^fe9wS+DcWwYp7u8fsEM_{>b%PYJSiSB?W8Cm) z%F~}s+#ohMm_*Bf<0r^+g7wQ}gd^Ha(sBDi#Rm({+j-({v+To8ZC@Xiomdk6THg&j zv|1CIpI?Qnm`a3bbOq8BH5;+cUko)64v8h1a~)62+d`=cW4flTwI$u{q3>>(^)2y= z9ah?oky<0pIA5OnT^~TP4O@H&%WN;`Z{Pb~1WMbkxaBarzIr&hBz=zB)CeBP*1-L~ zCc;HTv{G#5v~-JL?d>%k1vcUpi>t~Y2*#w9EEwwuseFIk0E;f+ReYv0T0C^Js@jBw zq2Uj!Ue@wM&$wh1g0~&;uG83j_S8}4Y;IU#91f_kvWG#WqZIo;sofe$aSi$?LMBkv zNf9q{HqNOh_>ot?EfsqhBL1{iwpe(nB8~ZeHiNNWi=VN3LSeFv#(38kFCvPg0r0Hx znbcb}F-s`9e|2tri*32+V@X?wVeqb>Pl4DsjZL%{k+kGj)g?n~w<Ov<(5>is_Rdl! z=aS8myKIFtkWxq%XN=>n#ZIitFBnYvCU7eEcb=|lw{+|2zo4`*5%l_ecS0s<l_UQq zi*4aHtr*|q-(958`w!HZKTM^|pYLOA=Hp3~mMw7|;_f0}R@%M;FXd&jFaFW<g?H-y zH_;`9FLjUfi!Vxr@)d0P&pQu!DquOI_e7s}DY<BaBI+*NMQgtbb+mNn4RL^UreW3F z4;ko771D}*jGt`vvIRC?Ox9xfWY%Hh)*?HZms$!hs3F~L?V9+qWp@!3)#j2mu)v<Z zu;5AydAYIE_{vR}S@4VQE@M~*{32zO&+A7X4J>Bxc2Ts{L7m=-m7GNruQ}O(%e!2I zBtBy@9#CC1<xptc_ks}b#E?(IUyAu6@r9>3ReIo^zyq>cLL2R8eeXKfAvq5^52Gru z!scPx?K^SCzQbP>U4Z0gF;@I>vmD(K4!NF#-!B~0ad%HMPi5u&eSt0PJV?#_0hBlD zOCYdyO#L~aC8@i3;J#&U-=R(WVk`UF7vFU3byTaC7G_Q)eE7Sk`>ta>-woXmzS1YX z^uv$2tjdL$%@W_)$q}=>uP!glKRjWkt`SY<ULu7i)Ag9&7x2DG|MxM_eA@-kkA^6C zp`41j+^FYv6BM_#kU9~xn}dT~k)BHmaZ*;#h3Y)B+nOf>1W~+>sFI?i?_al#A|9z6 zOA(3>8aPmoo~|PJS3r4npl=6FldNB(IltpxPZ?eH?|wFFUcOBD+VIh*WH3G-{dCFN zVJLb=4`A{3YE=kjhYMk|SXJWWw9}8tWYkOC-P6|91nZWB8dH-J@sDNBm&R7NdjEfi z46xp<rgQ@>y&6~7#aS%CO~A!zgqjFIaUn~&V6C_|k~3>vqE@6Bx)*S+uF_zO?37^J zkJ7EV*iDLi{t)|rSO9|@_JCfo)TA(>u8cSHRZ=>nJ#Kf!aL>4=6?RC6pUpEJgd793 zA>x-<)!V6?QD8Af-+d{f$MkBcX4<VUi*5a_xwz=Cs}?=Iy5+r6&HfldZ`9BN4u_<a ze>-(ey2>CwNs95xd<wV{P~7?tUs1itfpjBd<8Eu(qn*K_g{&I+P!0KuH)6(V;*NIB z`A&fy>SEBK&DpP_`uWvtbKtDE>{rRCC+4`~=w|byjWJq5EYYG0GV8@rUpfim8x@Z; zJjd8nwfjA0ZMjbdxm)=fa(FqZ5{Qm9J!g6flOoLs{4<qeuG?JGJwMaNi_Rdom%MIX ztH2jICq_&uU)n9(AOoGGv^SR-kKVE(r=3#&F5|Hll=Do$KBL5uda?fW0Wxi~lfvNx zVoqF-b_e1DskHssX74{=P5!lz>3e7mK}?99wC5XVT}x63d(V-Mi=Gx$^j-UNH%>vQ zXh}F?SYic95borlzgamQeb92jpHoQw%4Iv~iqQ5lxb}!@u%Y}aY4W$bZGR?X@t*Ks zVuUR2=Z?>b!-IbL-5o7@Hr=uqBch(6#)$u4&X@lHPqFY#0^NQQD-RNqJ69Spx@H&= zAo`D#<eR69`lPvnwdyjmcdX@>Ndt`!<Bju&+@kDL?dk$5g?}fQ14bILJeCRud}%ww z;UOtRf|pR{VEPNg%nmqPsQ|^+_0!cbj+0u>&rvra!QwmfjR`|%xz+h*_@&34t?DtM zjm>^mpEUAC{h+D<0ue$BQLSa0p=<5C{^KCpo&KQmG>0cLs_yzq@}kC%Zfq?NT+OyH zwB>C)MtQ&hXUxFbM^yDZ>3V6a?m^O6Ol+}QUW~D%9yBku$sTS67YEqtPawgvU(T6{ zdSu_feXhjD7x^|Wz`C!Vg4m{>H@Y}W7s(UExZV#<tjIbM>XJkHh5hwwIr%p^l4H9# zrkD0UfArK^n|k;-7ZBvV@jMJ@ns%WnYwy0*ICD){a#{*rYL#ORyP5%<b?r9NMpg1c zp?xW|B`c_*?5vw8(OW5rUeDheRtSLGEEGF9QurqVX2E%;I;=vsGU5Z0X)K<v2`~pQ zg0d*MMz=E3!jk$?hlZXS<!`9XyMhaO(UeOq*y6IZuM#G@8^lH72Rhz$1lRXFz8H7K zm$0K<=SIyxTro0zr<__a-N3r#=gGoTkySoy>=mVRv$cuR-t`LW>Aq0FIEZEX)Ljj? z;4!`5nY3J=E5(=Nn14UxIooEL2^Vfm-L_K4aOanp8D(6V&zRXa5)=AqHS5T#;QANL zI6&xcS!6UOV=-})i#l;Zl)kP1y>hA2I?i78E66~_%ZmSOA%w9&zSg;{li6ja>OSZt z*ZiCCN&c=8>o;lIDn9b)JgYO9g0GIDLDDJ%r*sRKjAw{vJwK)K#2oXqQ1d;0fI&IL ze&r<ef9i$=a$Ew*4L;j0T_t`kzt6lesaCfycvt=~d44}Mo331cbl`kpK8%|Wey+tF z#K3ocQyLgX*_|7O0iH!T$GUmGQ0$&HgV0~QbdjLW4&Q&q<P_ZFFGYN%jAgi*T$E3~ zKHArigEjlqLSbNbt1OR?;ZJ8ULJD-}#Nho-9cIN61Yl6W&SWzCl2A1U>_K=qmlQ&! zj#Mdxp6{erf159oUJXbf`vS1|?zr}5`0w8jJ>vXO!45KBTW?4p4S({i7#0CR6J7Jc z%{H_W`JS$B5#2pFSzFTC#0psu=Bvgx;vT5HY%th*_trEdt~&~m*^9prPJ6<SHQ4#z zgZ+cs3iR2eJ<Mw_em%mCek#v@<FxjK>g0WhV1dLp&e=bG5ij*Wx{8XRV_yq4R=QR4 z{1Vsc_W5X~RZJeY{M2r`g=1PF#_cWWcCtiet0V<%g}tjxfMHm_MeJdFMOR<cORQSg zUp_=zNB>(Zze;+*V~-i{3m&G<rQQVd;WLf4Ge|RIWV9pMqxA*@{qv%oENjfDB3~cE zC=4I&u@LNOdby-6iHr=OzmWbNdSjvEhK7TWsnU(IskEvxVAE5Seqc`9^P-e#1MyRs zw*C1kh9(OGxg|QgaV$9QBs{r?a?D&ck!fY%bJ#8lfs`^OP`@O5P)WQ#4J{$5&s3(( zQ!cJATF&R9haH_@E9;Wz**24q3BnF7IDFVIM(HVg4A4GTc|9ZN5?@HO@9E~B+Nkui z-12CBT^vGgM9=`zI~Q#pnCu7Gigyet>v!ifP;U};^IF|B(Z#;~e^jsoaY>*!(%FPS z6R)Y|zw||wHNpFfN=9I11_>L<5?S?{VF&{<IqJ8t*Z#?!0On8fKE7;>MvL*ut|UF) z(OCJEqFb9EmP>AX!j+SW)3oxH)=bG}b1ab9uyuR@?Jw8~r17$qoiN0BNxe7^p@m<G zj{o6F#D##9&zG0M3r*{gv8_;+!3JVYsJ5}oXTI^!^5Q<CfXrHfF4qVY$xr^qPPmih z>Xjh90_<>9EA#jo(hfB@8yx$(I@dISl3~(aJ~yz^>DTQazJIR19DXgbKemNmzhULE zGdrcPf{y8LWsAMU9@VW}!F)dckvxsvd<4Sk{3^%wN%Aw!ejy{Zy%0Rh>(tu@F-uE6 zskjfhTL;*~{w;Q|JfuuUgWOD+H<%Fohif5vC$oLs<zo~1jYFmY*k{G?6J#RTY&F+D zvnwZKi+1?=n||6I+WXs}-xdss1MV*`0mADyWqMeSFRF`M-s}lGS?!8sJ^n+LQy*QA zB;&7?j{$MCQ1>YToKS7B;Tl`Fwt5$Tzx1z~`FEceUISYH-JNPng`ElFvT|YYYS)D< zt;onZ(aL1>UQnZ696ftvn9s8oBOgzv)^2UZC@B!Pgc}3-r-UF0mOCA=Y-`~`lA3!F ziKL~yB4u#Y3=Bv(z38;YkDiA^2Ut_0XBe*3-?9JYimi=IiQW!@bvp&+rW?e55<P%V zZphsbl5lcy5xM6sf~n}q0Sv&2m!;e3We|6wf;Pdm7`NKx|2B72GVV`z-R3QrikWd) zb#i?qTi~Y*O%YRnglTYj*%!sp`a=~D_nW9FzYqFo!=hw{fcOk8;Js!2`zqmd@2Fd~ zRZ9p#$tX)u*uJ?lxe1&V2ocl$^!Ns=mtxzC@nv9eQNqBGqbop8iJUFWYH2|FT45*b zPa+xj>gZzM?QOD;Bnu;|LScNxwQbp&=hr|QrIjXBBJyHunzmx_@z@#O`kzbQhhq;} z=emFIb4+H~l#7pQfq+)EyWXy@`9CXLvw)wd>Zv=+sZeGmwZZfaQd}OEHRU0Tf-<{m z3LhnqDv?|l*twCB44^I#|5X=RYH^Xe$T@O{wsFiYw_rx_76_AHXX%1Mzo%GHAI_Y1 zev{OJ{MA<u3J~DK4;IZFC`n!CPxNpk^msUQc9Kt&s=IGU6q5ymJEZ(NPWLg?-dboz z!#@-KC9O`f#A!lhn+u_l>Hlka6HBQHS{haRrrjdxro8*t9h1mO`oUFHNa<gyM~h)4 zRnZ-{LlHZUED8;WNyUj34hu=728XNtsIC5Lo*8#5OxBqadsMFzdB*EHrcl+?J<Be! z??05-Y?b}er9B@O)^8NhUzQE&pS5zIand0c8v&{qOra#55FK6~5Vq4jMD68&QS{p7 za>qvOfR<7Ba#<*FY6(e=?jRlgWWtZ88kRH2(YroOYUYM9bREzv?H6qLhhieJ{*wGi z-XTyAxmiqDCN-7%ty(ZkVGy=e+}f>|8ycd7<-4Gm`9{~5xw}{}<F-bkTTBT%>1rGo zMAzm!6*+S(b@)Av3A}L%Ilo%Ckm&N+My{mVw3(wh$6aDSc9IU*(?9o-V1{{FF;=eq z9NUd!40njYoNWw?q`8AIW&*S~zgls9liUbaGj8$d!HXd~NsslwQx3ypjN?FdP2tj9 z@_a!F%PBxcdd*vQoAH-hKBv<W*$jX4pj%q-8;2k$RfG+_5_oyRTvX%3etqEZ<R6HN zYKui_hf0z8aAb)2WPkg@Hp|oYiN)sf0@s*1?;lw#Ej-`7N~%_YI4t+=Qd+)<_1i2- zA&kV^w42-yZdk=1?n!aVuO8bSFZy#6Sr6U%!n<3u-z9vSv;qdNaDgB5nMYim)Q|}F zKJxwvj#*~Gy7fy@rz_4`*A+Dr%(|zN>EwAS*AJVZ+<9MShS}=%`j<LBt#SQkXEBCi z9JbI2gL5WG{O5DkeKP97KCHc80@#yjc5CF8{1>wB(rU_)@gK5+*N9xf30CLB3%ant z+NIx{X0!<YiJxvS-Q|}9@G=Ngd+w;vp65h;&@m2trV8|qLjwEx_^F@e0m-lgQb<41 z7MbMMNjq9*AMNlC?e?+`<S?bX$HoUZRAkR*vku9||3SS3``HG>kwJ60rtiQFfa-SZ z&9^y=@dm_J*!cb8qpCv)j`j^`2?9bzb;X{#R?)<X-vaf84VPAsA9MuEK7Jwj@<sxX zX#nSieBLKTq?3(ko6D~U>d5T52`@l50{>=Iz=0TjtM}VvC$C=|X$~ImHUV{QQ}`Qy zQa&lH`ZRKUb+R1j9<_#MQL*j^#nF(7_wC29*qH|!QF!fZ-fmr{>H{AadAhwz1vh__ z8RX1Nrf7$4&3X}Qs;&g}wN9b%ps`~gLm;if$X{=H)&5qW5r>w}226a#?non5h7>EL zs6P;AP>i7em#k+pdq6s!G(hJydJ6^?qZd(2c|^h>=K~>j_dE0Ol>;USLTunU*(V9- z%O(>e$H>Bdcvn<VQT6l-#|BEVk~-^OFYC#f*iXk_@eUnDLhf*|8^*?q)z8O}P0KCC zDKGTi$GjXbDOpgHm?M|-JILdiDgB6>n3$Y*X?_{DeUBJ>C|OJjWdQ*<N%}ATTRiqd zC@aP6X#nSQ=8C2ISUJZK5XcJ7{#($T|5MPy2%6k(@!a6>{`H#g7jJxIDY-hT-4`_Y zF=Qn_XqVl~YwW5c$K=NpVIMVN_rsTC8q%ZYv=5qS2$(bT!b^9M(K<j3SkKxo!<Se0 z$2J$O4LTkwtK;q+q{mm)(GWQ<R!<ZB+fzQYZ~VG;$rHmavHn6j&p$xV;}+ZhGkbrc zv>^Fg;U-&+SXKH9!qQ$Ev&EFDu|za(WX{=1tMufp{%jb%%pLqgoMUYEx%eo?Ca*EI za63z$8fwpeW}Jq39-#l!>0d_2on`j&GIdg@64G1IGSkfn52G4Q-i@9pvrZOcUr4$; zB+sCZJI+3JOd4F{_{y;Ul6{~IlwG}xF`MlzK~UVsPo#vdg|j=M{^r9z-{)vqB)q$K z!-fjh70%B#Nx-t=mbdo?T2(ka$QvIoJ#jB|n98~5fpgTFTws|qZ#eYN)BjP`=W|As zS{JZ926RV<Ulb%^U#=w!xuZr&V_qzLJXjhD8W)5#b37Uc>9Um&E`_g7KwQLjIl@SS zQ(6h&DXR*ZKUtQp1>vdJV<#5(_3b&ltr$I`<ps!~e+j>}1Pp9{T_MECc=5x`s6>oU zg8>i0DAkyRo4GdkB)a7VH8d8o%eG)b)yTi4<S=kdA-*1&L>yc!P6`rPQU%IdOKuz; zheB7o3DcK9*cSNrq5HXNX_SGt1Na&x{uukLLY!Oe0|Pr{!2^Tr&WJ4?J@2Y2b_HWL zbfqW51US}a?-2{1&*tqj4n%c_RlI6f71Md&HtN`I;h+r}0KTgd<6u#sUGEtl;Qd)g z{sA|+rGW?--Y`D!^8wxTa*k>-`o0rlbn|I{aXE+45kXr_e>rUNsFgdmB@u1%r)crL z{4shiVBUjqd@DTmHI$8PNz{M}R^5U{q-*)RU48^7+Pah<aBokcH={ywraAuIXCdh! zHuEvK_87z{=Bqef^kiy%8d?rg&rXXgvG@9AAs5SjyO!&0%l4;yvT~Qy`jA^=Pl^)! zQ@6_P<YJ$v_=W{|UrwPFZSv+*oFad@vzrU>jFWpGe+*U;E#I>>@IUTCzal?+1+SCu zcz@2q?iw#usmdpj{&|ZjtH^a3Zccj=u_Gy!?;)o{6M=wv;9LI3=-2b9&I|z$Bt%p$ zYCqAmyuNY+L4$Ltddh+oM#YomD7li#7dKf!A8XiYuf3kBWBM|V*1K4?E?)v^@kfR2 zsj?2*p*iMu#ec5sRawohkPb)@?j{R~FNz59S@rCGo-B7e?@H2;d#rFIbq`pOehcX6 zU`VF7@Q7QynuK-EeVk0Zf&VtARpHdnUoKp)%;tuWXMrVw1+z~Xj@oJB?1Tk7W3)R2 z0$jjHts<0xRDemI?9y?&A$|4jKu3&m>Uu5B8DDh*qb)Jat!_^&jvNZF&<1sAk8*3b zY|??MS&Tid9pG{Ds_$5gTu2VKXO7RLl@=b71>F}KB^z&%EHauf4U}`Jne)O0wsGqq zB$&BY#FgXfe8GWT*2y}E3oRIsqr49~(>(O-<IQaW&f4uGt7K<f-aFL><RN;Jgg?J0 zsM=I@Z7u2#Z3`nmb?=O+*DuN!=m&(Rch|q^_&?KP$~O|JnhzEhjQ_k3d*$BJ<Vy{E ziu@Lgy51N)nDM;&{g|Bhr4gIkR)pI#Dj^{Go|E7jMK2l3a#v#AbRTYS_6hfUkprc< zf2K#&nudNu#LzEgQs@xg^hK9wAsG9YQ4>;DRm2<LuQ=x5XHw{=gxV*_=ZTR1u?=>x zS+txKawsSxzXZ^kWsf+$U*S)?bk5Z{GT9iolUHAE7QXtQIsCu$a;YtpQlVN&*Pvm% zP*E^d0F-s>-y|1)obCJV#Zy`#hCA>%{7c_vO1{Og)C7&$PZuVZue`Qa^qc@&__IMh z1?dT%trTe^Xc;4HuZlf5>6!RMj{9_F3<fPOk;MVtsf3)LeU44jTLm^+g*GFHCzVO9 zQ(wK;Y06Uh0GZCFF~Fa|?y^<ZE$%bh5=3VH*>gO<d5ENOa%+&8I2j*AANJwWtoO1{ zwEHLS%&CD9aoK9q`eFQSw6@vz_L>kU!(y^Q!^b5j)QWex&+1xO?4@AU1ZQgLXFZCW zQ9|yY7`eR01Txd(NX3Z6W=ss)3&T&^FEpOH0;lCKRcN~Xm(<Z0S$O`)b}8?}BVxG9 zLSn1X_pkf-4Il+cO7h?Onn@-KPoU%lVYmqomO_lAidh{74I(^yUj}jeOio}qhG0G~ zx#F+=a*pS=D!OxcH6NX2hfZ6#CkjPl6MBvDL=QGnG*UX^XC$LSb&rDoJz}Q719tGM z^{N%dtf$ez{`e-Aos0gaN4|fSUQK>lg+UHxfz$1~`u-1WIQ5iE?{;3N*ZFw4BmHk^ zSKd~w{Z_wka!Hv>2B1RTMLzASVkPQ@Yy+o^{vqyTu4c`eTb-{s$8GCgY$GQW8HlOQ zoL>jK&*fM`pCd1S?dqnYj=7b?TYyhvHk<LL#$Tfyo#dHLNnjr`P`kbRdgFN4=P3Qk z0N3c(0rFrOW9j?yANF%)^E$x67IVyN^Y<7hloQ0Nz)2rpiF3ZnJo!)^u(Y2_c}Ml) z6ftK-n=d-i`WowFuP-0k*8qjHB`mpr*>n*$-Eu9WMc703-WKt%1NX%v(c@Ca8z^~y zg4oTfp!x+6tY$9I^n@d3$KP`BF>!{86BS&tD?>6NZ8!4+cmK|x<sVJkq_#ElBO9%+ z^=$ew1+XXGr;V|mdf3HXil39K97Tfjf#)U><NqmkMJH?uU4jJqMxu0OT#2U)OU~E5 zvPX2z4euRLyiB?eJ9KY#^!&CTRqRWIHy11{zBw}wl<J8tDe@X1an9#&)0zeIL(>lo zH)J+OXcTomxEdwg?Fw`Qy>z(mx#hy9<=A6Ya~c`tDU8ioPgMRWEJCQ)>~<>Nac6Ms zd3QW|Y=rCYO)oqN@|=CqtdY838uJiw-|Kuog~c#wONT=8^L2dHKvO<44rW`G8R}WI zP~*Ue+}<p)=^9F@ZS{;J6;k#6j*7i?a_=9R=8}AP&gc}<Sp*dotWCnC`x^Zw{@Ri3 z0*Y*5Jr@(B=?ci}X22S`*lK>x<ChwVsf3a2OleYn5)wO~WYbpbdgYk+r?BD&=>XU4 zQhdEq+++f^?lx<1J(cKU;-}YB6Y?b6fDm{Sh(o;b4UQX3aXsCt7zziZ9PQ?4U=*Hn z!6Y$>j8E7#rAn(dMFp%EnVh2HU)O<p?vc3_L?*fYPk&cHxpe&Vfz8&w0zXqCS<w%B zg6qR)sR@V4XPPmB?p6_14IE_T<=hJISg_T>DN+|aQf_BMt?u7vN4*$YQfXnjRuVwS zo6kEd&F-jvyq0B4y$_FGs64!wgL9>=#3=BAdm(-oo4Y31?JdUM)fblq*fkG_sfuL^ z_;doej+5Ss-GEh(fB5Ci00|WLwQj6Z8k%*=ZYXH+K0!o-2BO$RLCvaJd`Ai&Q_75X z1pXJXLv($MQs$}M85TGC`m}I3F-LB*%Q$%jE|GJpu+;DrXSzv?LtH+)3Q`+4_y-LB z@Qtq9OD?;bd(zKE41LPa_H6l6Tsgy62WDu06?NLFb^cxEFYWcnZG$iOOwhD%;-^Ls zFz@9{miC#u^LB3iViQJWa{fn!{RQhvWF-R93FV>U^XQ#bjMiDrydU2JzHZO&@?_OK z?3PYm)1br-;O7_tLtbs{LsaYD5LqQQG3CEq>8A?6G5BrO3v79r;CWE|R-Ckl#vv;F zi)#G_-${m&ST_JsiUDUGK#Oo-3mkEVU3u}-Nv`VnxGoIW7aUK*8EX$lRPCjYri?ab zBp4B}Wm+FXkxJ&ax1(9vJr;C+BvLohUH0r?N#lN7f(DG5iYJCsrn(w%Y#mNGmzOB^ z^VpSfAK#t7Z-?=4F5@0we*YOLTtO<_xUr>J5?`zI_*Bxe00Db5TWt4`02pK|om-xC zGBOruuP+r|HoL7m|A`#BTI^kk?HHKxDOt${*u|o{(sRGhSowiqoXv%xU0PFh$Pf0y z*YcHNGrtn8b~j6!i>)3v1E&_u57vGP?ppVsaY}ojfd8xQAPgIVXC~3By!bj?4a7}U z1Ajfs#>~sYn8Sv0V#73Y0!{2O4CZ>xRyEVdo~D5k*XBQ+YuL(V-E(N_TOq7(n2s+m zl|@;+qt#i0ZAybE`KnPhvXq3A_FMOt&-syE<;>ok*$SSjKwC*GL~fMOy9w;g!|o;N zwXvxhyBa5a_>*kKvAf5DcN0WNuC#w?etWjr(~YS+r>;E?Z)oy_h9YNOyT^8Zj^B_C z5in)3-{fTrw8V)uo%DrHM+C)|Ibs7QEUsS}rKu=S6`ycFnVmOcyiQ)NFo>YD56BMs zWsqcrYuqxBAfD?)9?HT+k3SD*s!hh=*?#DnoEAm7kAm51jD(+@hT{ph3uf;5+YW^O z;Vzljqia8VfwtPUA>DhzAd_GsNrl{uBXQ|eWEm?+0F{d~CZrQD$c-CksU<L^00ESB z!>GD=*Qhn>47KdrZdH?VSR9oqyuBCjEO60(iWViSs#TVj6)d{Sf#voSb$<v_x#$b} z>7yTNUkHtJU^ndsS<dqCf?RH~L0^ZzA;$M(;LS!6(h_NRCrA!^?YeLJyf~$A)hdIg z4*YYQPc<mk#^8>Qvt{!eV)er&)#j?x!gN}<g}-rNG|pM)yrtL@9B-Kni^&fGOoCMX zL)<$~`B9tqzWHz=mUQ*8rh!@ZnLrGq0mJ8%8G>+MtaOXm_2_n1AN@XzMIikAV_e}O z?U+>Kn+&0}hV%1_h4Rgew|D$M21pIRugXSY%40_9svB8E3=2s&{;&X-s1T0;ThtHx z1|a{Ub-uxo&hXs9T+jV8XQqA{eOonyVo$~C>={AEY_dPvH06&qO~6<GCPnHmf50@M z8@J+YXmi-%+g;0X+xyrW1I7%0SGx0~n&vQJcvtA*N-bII;th3462HR@iXwxmy@|Wi zvEkuXCkBXg3dhw`>sw6am6+kc15y8S?Ukb7dt`f!R<U>fC&g9!_Ao>LyNduh=7_%D zj2@B#C)FRLr0BXrChlaZTVLC%;b-bz381gHU<qCtj;4JvC2*^rw`r$}Q1<H2G{3)r zJhTrm%3sPvmO$yi-U<kdS){*GXOjVb^La&{opqBo)?qPOA+u4-jrcNnMzY#iLsU@N z;hSp>--qQp3-wOQuCzOvCcRE(^()}y2Lr2|0Eq9pxGzUN2%QYZ|9p2WT&k+QnWg#v zP(X}YK*mz4xJ{R3rDF7Qn<Vo>s});a&WOnFE}fE~%gF}GM4Pw0z#C^u33A|Ub|e30 z`A4-nAmXoFc!Km%tw{6h`$+aWW&OoG2JFxIdR{#z(xfl(p}LP}v^Xa(AxE(9l>B?N zqEP4ZU)Ex!onT#;69K>Ea@$uB5^7jo)sW9wvXL9L8D#13bhD)WYoy>bIZdZO)z``c z#Spe^`L~C|a_6FhXr(FBdxg*T;7#{0${|sR-7z!h{6)6O@M6oh{qb%HB?5*I=2VjY z)pa}RCn6FtaZ4G=^vd?U$5q(8jht0>&-u}5Ma{zIF0SPct~zuq$>3I!v6sG{b*h!! zRg#h0cydq0;{vO}hV_Ywwun|Cu{Dm&vTeWZ;b9GyrL{CL_H0LHvJ9_I7^ig}pP4!& z$eGP7dKWY?OrdDPpz&W8C0mfiXDP(HGy8|_U@rZ$q1729BJ1WzOX^48yB;9WF<WZB zsArQoNl_2X0MR}q*s$fs8lXmY_y4c}9)~T{j%S?SP$0s}@DL9-diTq1LT1*+Z5*~2 z2ZS%NIu3F32qs&}vs51w++&qi>2<&Z|4i<$8QW?16i$y|V&pG<0B2{;kJV96K=>r( ziM9RtHH-%LlSn&gBj^&Z9C1_CXp>_HkKk<QRvL{Iht#&1`pP?}l`t+L-2|X#_}lQ} zkeiPu8D>04dVD53CYPHFY^<w0wA7xGUgb3kW$f*JW-(x^Hf=)p(gO+om{x%K(rr$* z8IjHICL#P(f%tAJ5?Q$|G4v-!Yto@;;I80{vpwl&P_~NUV%h$ZsMgH0liTXmxr0Av zxr(LIE$L6>#yXM1x!{ZsI>m#&@@2-RD{v_tBQW?`gjA);iu&T7VI7Zam1F7+<00ff zXlK}p^nIDPV1F06h80ORh`@e<{nj&#%Spw`eIKC`2?|(_u!4(F^m7M|Q%KFd%V?Y^ z6DrD`7l}uC_V}i3_E12NAua-2jlD7SWf4PS<I0AWs1<3w4wiGbaj7@XFB*(^9Fc&` zINy|(J^M7Q(<x45FvFd|bJBTRe0!Dy(77x+Ii9@>GA}I*jop=_rTVviAdz$Pc@R;P z<l#D*)PmP7>7GH$UXq-T4l%&$T3#nklp-TuI;+~^H@pzt?!sG6p!+foYdF9Et+5ix zPsB)j`9R#x<(G#+P7>!bPn@-*WQt<+F7~&YS-pYAikPzbNPNCO0FH5I`20-3IrPEj zhl9Nw0&tUCFx&hr8sf*;lhNG-X>U7W11{2QqLdaB0(D?@mx}~Oy{)dZkH7_JvDKfF zKpy+G#qg^&SaV4(eAZ5V*8ik4x5Y|#?pHQ%Kg+lpjje(`f1XqWx8+Se5>GHA7abZB zP^&ct(BF`B_a9QjjlTPMQJCRb*Vz&(RAYStyi<2|_Urv~rh&akmrB34Oy$}<CfruE zEBMZqfyTgc<Z{K^UpcMKmxn@<8+lou(v#{>R?^5WL1zFuJz-9`m2AqPW8iyF&nZmI zY`bcLQnpiyifcb$DHv;jduGa8U#ux!(EqdhKah~S#m)oA|Dl9<=?SYKJ5lSRPo$!5 z#P?7O)j%;KYyWGc4(kh2#JHPg*Bho{AWibg@hBP?;7MM73{em#Uw_5LN2mW}2Jq<7 zm__XdmK&>&FB|#jPY=?ZCv0g?7Qf$3OOTVXnRUJRjnlzQ93*RU^}qRWD8(Ew_Q&60 zcBNbk3H-*{Y}v8RxHCW$s3r^gIhspoU_aO-W&;G9@q+<)l~Rg8mhR&<H%Gs!LdgFc zweW;2n}iRQ{#Ce+4Mr~1E6%@=^+b?75|_Fcknj>g)d@Aq-j-W}ia4Ei7Akr~Lw2R2 zAiMerQN2GdA}eTiqn^S*iN{aK;sn@x${4WMWGVm6Ix<ch7FY;(VBD-E_28x&C4K$e z_RF!5Uu%PN<F5|2A5BZg!%pbsl?Mopk)Ry+jBnA-H1f+IwMGYK{zByE?$cngg-LJ_ zu`7zQ`dNx{44k$y%JCFxfDK9)-?%}gY}S%^Q4$Omy5ic~J%ne;iHDeSHOmj^mFo{* z84;fKSR)EQ6fx-ydpt91tskbqo*e5EI$2CXmjTgG!h61Asw<PV3Z5^LXq1whi|y}2 z8<n;}2^<{&9C<u?{6}%^Y^#D1xCh>h3QK-F21*Y&8?=<nKP$771I~Da-?;QM*cKJY zx_^9aXU2f8flzz}o=89H4);=TT=bZ9ZQmYsaNM)zVpjs)O6)BB<`Z!O4C&w$-@|5F zbrf~v%TRSrnbuEyMI?B2Wb-@xfFO}L_#EF-DlF8R3~vq%Yc{6B8s0SPjZtmAqgtsA z|JYx8TOsDSOF{JdICX2g5i>oAh7;6cmGs4os^0qq1R&v1fKOu$u?5)R1(<Mp!n%Uw zi80-H$kkdYc+-fp{2r3)JxL?idTx?$LUJxKkp{ZItKM?6ILlJUQ7ZQ3r^nQQ&wenj zw-MvLF-xtZ7kE#v!jMJJYuCyg5h-MznIr%#hg90{>dJjdLT(n-*!c%lN=?#0&RU*J zh5x5;pL6Z;VuS|X5+}vFjKI7^2Rvz<5T^@8rJnt@F>9>yD<2YSgaMp8>FhjV$Q&nt zy}A@zN)c*47vZlO)%28at4<=8ifsY#QRY{dj%z9{&H?*rJoeuD5the;%Ch@KQMr1U z(KJdgJ1B2&JQ>7(!5_Gt^FF9p`MvFq`+6%4#QoK%7-d!AkLd~)c{4b$Q-Pr*A7jE7 zyiI*!J10o24f^44cx9v)fn<GB%=$6rA9nlHN@MBaQ>E$^duX8{*#}p_rSez$=h)wc z7aAq+mdN6ky#@ZJh;YGtC@r@ovH=M^U@iEtfomI-s|S+)X3S-1d?5tdV`Nfz+P)7l z=)HEE@~L18JMFGZ?5N0U7sxT6?NuEeFTs^ObkX>l>s+_OMB@kt=h6|}1QiQk;2kl% z1?58jW@_w`hFkV>dQT>E_^|Zv^XCYcCvXADbb|7ioSRp?i_A0mq^C*c-AbX^@=}+8 zBzi-}4Jo93T#m~pl!`ldtQ8m3qA%X1mRO#*PUK_Tk;9tkDBt`Ar(%3I5ba39n2&_Y zba0u)xi9<+MEw=)7-5wbC9SszQtgZ0BAfaoaZP-!Ym8NeZo;or5-9Hte<HQpleVbX zvuk2R4f=?PWxOgf1|M9qrMn-!RTJexi?H{<CuQ0LzEX6$#-7^B`?kq*-ncnO4MA{q zknVfz_q=ZK%KIU8*okHB@w|W9!6x9SLoFv679Vk4_C!pOn#z-+E?%tP7|WpLhRc;Z zzp;ri(i1w4<dD}r)%&~DR?#5;_TT#hEwTkc56b~4HBK->GW(9MRB&%0Qwq68(Q-B- zAAsW8J(+2tZc>_xJO8Pv4jAP)cUl+^VTuM6zvB*_esh&u?OrbE-q3xdZ!gb$Iq;Lc z%CwU>xQjDF$*6#@E2zEWY>eSJ`_`o<6>hPS-*zPBnih+hjM_G6cI98RDpOV;-vpvE zR*4xUg@_+dODUS0TdHV`g`zJ@DIJr)|6jB8Ut1AWo2=r7!b;k2xJ05wpwdEC)sUO} zBXDFXgOSbeux5&VTE!(c`<gt*^Ug@+ej;155tG}-iXOc}wPnZ>d+TzSFYV1lQiy$8 z#A^v6_Ex%^RbxCTt^_6sCJ_ST{h)sn_tGW{@OTDP^zm#XgZc;>aaY8OqjD`uH)hHR z;=lcz^<;PRk@*%m(Qj6_)Fbj<rYd^O-i78;nUDYpD0DnuU5KhYQv`F8|IIW{JIGf{ z6DPEMXS8YUp5{TWYfN{!rA|$ZK_+rMU{9$sx-R?a3co)#ye25SA<HUa8R5Csess1& zWdqqa)Z;o1eUQx8pVhf@z1q#n2jSS3=5kDS+(Jv15hm>1l-pOfM5Ef8(^QS(Jf5ZY z(lg)m+oFwzz?@(ldt4lTYnj$_1a^PSFpD_Mg;uYAS@SR|0T+GA9D_S!wLu0Vw*4kO zKG1LI=k%J$j`#cAL9%&P96xPnues6eXuC?#iVYQw$)GkR3bM%{K)1&CfyH9W-=*R) zE->%{<)$ClFD_ty1!oLRV34}TY~u5q$0b6HOm-}PifOS!bV!y=cg{oX@9wM0zi7HL z7<+aQ2Dm>#q`1t;p&D6uf&IJ-Cm}Psd-!}i(!99r4d5q-Hy?gsWW!OJbYHn*n=kqd z?d*<8dPceh%Gz!E(?fJE=_VM^gsQ>A52R34V<;+W^FDdL_;j8>GGOGKnvrbM?mNCU zKdyOkl~qDw%1&$2cRyqJ#O8Z@Uf8^&B!UcYe~hah_O39Y*VbGyz@;yfC6p;aI(SGB z6Nts&SgWT5rchp)JjED;91VRT%rD<7VW*4ZU%SIQ2~W(n&M>WjgNl~Q<!V*_D(Hy} zKjpTL360Li>5WO-V>PqZuul^mzqjA+VUU+nF8!Vo5|b-y_GgO8Y7OFz&SQ!IVnqV0 zoWXE4uocUl(nPY4@3b~~Z*Cn`oN|-1dI$RVPPxn!vi|u_MY1FiSk)vtBbr2S#h6_7 zGI&!&@GUr`RzcIjROZzSC5)_=OV6<q#vNJbAc!eh((|+u?5d|GPu#ByEnGJNyrMt7 zi6B!lf}am3%$BvkcI5^TZ@6LufCT-uBFT$3*uC2wF;<8n20>y9;-w6+RieZW4C#^1 zWAvG%;~GDhoe^sJ?i?Fhg2RCD&2p4ELXw<G+m#&rO?%}olk;HZV?lZ6uQ!qz5a<-! zqlrJ6lo0sTw|Twn0IDLu#)+#U;5~h$diC98^bj4l+OvfWLUInI`n@?CcSiBLc?|P` zQ(h!-fnPx7Nfd0nNDhuFKCsEXq4!}vc^qi;vn0I~zADFf?pq6wl_cjE;?jcNObdab zo%Q@o?0BZBSgcFiWykmjnm5a9G5yhdIhKMeR@2p=sjIryF#Qm%m`(XH@BEnbm!I@h z7(jf0N&4(&TCX@9sr?JtSNr>rAbCA2<l(uz_R*U;dsj&J9TQ>DKal2pwuw4?cTDp` zlG%rdVa@JFQb=wvP7t%GQ0jS4a1rw)Q17XXOPfJbiKh!TCBglkX|Aj3|Hsr@2SwS2 zeZxy5jna)MAl*oJs30KS9ZHMz(xnoMBHbX}-JQ}%=Puo_xWobryx09a-#gzs%<S+F z!_Mqp=W!mt)Sw>vJOPg}PQB!!j*D$Lf@}wp%Q6JW09yMQA*MP+VQ^Y)TuOO6Pl?## zm`&?`UUW{-;{#y<Jo?n%MT|U6!<SqZJ<mkC%!x?#`ae1u`uK27cmnl1s;CA|^jl{p z=6M~9WQB$qX?2XC0E^S|$%or*QFxg)O&1dbzZGV^E3~tws;0WBu!4ihW~qr|Ti($A zS-9kJLy5Wozp$^K2JIX|>D7Z2)@3SVmY%?eaY{Qbd5Zs0-p>l>RA`;|rPzlUPzjKC zeo2h3;1xaMe#MJ}y(RuGLRO0?OX{@<K6f(OQ7y;~yibD6;E-vewF5Pb#ACRAzVW!t zW>F~b8gWZZXi2<_(S>$iS$Audvr(d67?uWKdvxC~rVM!+M4V{`yaKU9g2b+(xjFCI zZRr5y!eg^BriDL>b=VifS)caV7Kp2L3~lm8j@YpwTg>%${|Oy*XA8R7Bzeb>G+?g= zh6tRY46-^#KR5h8Bf;Y8qlFjRFj&FuxbVX<H*Yq^g=#U`k>7c%eHz`N{Bk0bK(d_t zXqwWC@Ql`1lt-PO!@+*LEMO_Cv9gBP{myqTn91<B!$hwDcxnqBDi42g`4Th4=z@I% zx11I&jMMMYTmgIA%>Lo&dkN%k*4u68-!?Uh#pfY748am-t<M18_!ca90BYrbv5&Um zR7XhQdS{n#_h({}%~|`k-7Kw?-<Je|ljWu`k0=upe~7(VT^zmo5KYc=35w99^T2FI z2fsbT@Fz^Ufk0kU4fYZM_6rqT2lg@eAEW&TczrpVuXrQI?LogzBB3W#3=*U>|I+u; z?TFD;56xH~hv<IoNA*M&E#^i*Wm^qz03M(oiT(Sn-M3&)KeUsQj$pK^QGPtisAG-5 zYsFdAKGR$3+J3QlYx2nA=Y{YSJ5_V#8&?K5{Qp>{hS+z!9b{Ll9va$X38|+-U44Ck z?i)MQYQD^Iq0(*sa<6yK`9A7ZvsIDE0~{V)Hz`dkeY?+6M%4g?PRcxj9oE6DvwJjA z0!LN^LVa~`Evb1VGa3^V4ev!8X68Y_QG1UkQ=NqoSsB`g_AEg9MTc!FRjrqZFFW5o zr+JL~C#m&n+NwDzc5SXR<~m@0RiR_<QX<un0Ji)Q^>q@KEK0XKJ{U`K1Ux2GABd0Z zXWmyY4?6Z*9W_L$>z$ZQ#!u`$N2TA*44=F9zM&OiC=5oN0oK_?5cCIeB+(4KXDCg^ z5fMw?Pav1_!M)P&Nv(IRXr)T~-qI7^UJ2d~LY8bgG+r5jF`;D7mCw$vH%qs|0Ey@I ztV{-x#wyYwY)u%nL=FM2fqieyZjs~$&6mAXcIQg%fkl8&4Z5{J9Ewxb%)1Zy(ids$ z6?I=JUVim-i>G&4!l^|HNX%Tcpzk0$!bs1qjzx58UcpJgG31Em{yQ~s&ye3Qe(}!m zU8W9ZVlU>&{KH__CkdUR`g{|B+r8sKl0C`JQma>WBo+#2iJOIs%^edCnT>S_8Aopz zSL()!Xu2I!A*Rw>h?locfAte!TTRlTSe+R}CtZeg(&=!q81rq;{l#1sr<P4t`1n)k z{Jg2p%{+4&QGo`pNTYC!?=HR1er^;<FwwXYae5<TfGT=7?{Wz&&%YwCXXM`QYp~-d zCF5|sk)<(5*n812Hktc6o?J7wyAk#h=NI@I+*J+$k=lTQ%hTRhdf}pi9!7Q>7q*bF z%ri$CT7@5s!oZ53Mr${W_dr|EP)m{Y=Rg<d2Qhe+mmZhZfQ-ngb;>|pS~xaci_&iY zE#mRtpO36!mVOp+NtggY0h0N45aZsVY$&wGninTniFSK98_!H1Eg=861;i1{%%Ea* z0f-k@n-c`gw!gEpo$9+g4E%R~_d9Vv<eedTxZk<qH5*4ji|AIFiDTlo*zArqmkeYI z?`pV5yj(K{_jXMV{^~kC+OGC3Fmxmz4<!wqw)^XVeEX^AY_pWkKM1tzp$WX3W9K{c z-OI!Qv<d-XPf79Cfh;w&*G|g-{}#{_-IuR7Y!ni&A-41DjQF$F2Lc93s(IXjZ9lpo zpF+tmhC3S_yZ27(XcqtAK8$eLVaCpTT&K>pX+l3U_W|;aCHrho33t3>ZpU{uiKE`b z$z}2H>axilt}omdhfAjJa-t8tQ3fJ%fON~1hrQr`by2&*gu@D7HKkGRQ6f2!+p$<b zKjaAh%H;qM$Puj9HF0z#JUHN;t74AmSMwjyU_LAD#5J(m`aSyu`@@;%z~SA3N)B78 zA+aP<Oh0^_g{%VZ`?^&ZWE0l&FE^Z>*_)!evF3jjoESd?0o~po1ieO^$yWdlbq36= z7ij{Q#M#_=&=C2WC?kyTJ^@y+ee#->elV`f<nEoAydyp;<547Ta(V6Q7-u1Swau)a z4s1JZ)l&5T9O1Gv!KgrgAhq^W?b6(Bz}Iq#`WitSLVP4uDTd5~0OH@6|G#R=(r0%a z{jKj5{?n~?DA7ikUV9kg%1<oOMbmmri|AKFP2{7t{t!ahpG^i+jJW&`5iwA~l}$+h z`Vphuc1Gf094Ex+a@bPjP-0}t^?J)<wpdGjb!S*GE@XySrr?1AC+CkV{4st4&M;Q! zxs#~h>$zZR@BZd8>$J@oeTA-_g=?im;lv}Gi1d@GZK5vCVUJq;o}`!MxPHqTH=hXy zxIA%2wqJA5p!3|!j+8Yg;(7~M02pC%P@hGw<4b`2Z@-d20cS8yUlF8q0GH_6;Fu=_ zhfsRZDNE&A0$aIjnT)4t#DD)!RG%wEs?S$q;l5v#)kn_KG5rg4y=#e*M;b*8AEb4i zwUAl31bigDKz4~~)b;PG8-c}qR6g7NvxjO#DtxI+T07m1;f3^(3uEw$t;yeS5@6d2 z8j%31_gas>fvYTe?LlR+eP-|zkRqGGXNeAbhzIZ!Q6p@w>iese1uH^3o84r5tA7I) zgq=ILFGvQTVl?nGa%kAT;3F4wjVJn03;TGVEO4%%A_Srq#Q!`s_f$eYGP$+B-?e;( zGMZcDmvp}%@20or7Z5{ZL4}fub?}$3t>V{>_@597_yV_9%RY<t*#V$BR!99JVZD&_ z$_~;(9+`Y*z-ue5Y>rw8dL%4z0-Go_0C#PP3-O>X)>QkOG!;9C`rh7IlCJF0ueU#` z4FafpJOK9{Z-`qOcom{&==Ku1SKYKR5rP4mV~b4Bu`XHVJZr<AH*h4hFEngtyp({~ zcBO!}Nx+iMBSKXC$d^7RJ*4nMe3rvauFkQiZdYfMky4FN(@Re2Q}^=t@hXWc2>f=2 zMM0tQ&mJXl8=fCm$H(!Z$=0K5x1*1Z!Xu4Q$GB}*EAO1&8Jjn6c46mya{qQGJXs!d z{i&1H^LIrE`)*P1(9;<2KfOQ*X;BGXa+ItlBUtT0b>}=NRVi)AnlNK`ia%+{YbPyZ zvxd{|^U9}lXv6872%zn@NbSL#FzxX+y2@e+D=G3)-+<Q5+!jz1WE>H3^L)DMlZNsk zDSZ68p|Ej#bK<y)0(~=Vk#!G`*WhafSP}h#Q!}*TO0Gldx`z*m2nz}kWuRSh!_}>} z1nsxp7`biFvH5IRx5P1J|G;T>^T{3xpag4Jn=ynwN3&V*<_1IXv%3zOSh&+$l-T|; ze3H443gKc~FseC{KDtJ)ls3FI2Nbz{!N+5vs5a<0CRznk>_s)%*N8*aaWLm-k-SL> zi8(omH4mn$IE=|u4O@#V8$ZDvF8MOu`V*_#uMWI@DXbS;e);?gvYF(eqgX+Iu4+#H z)fmq$_K8>AC)Kb3U-8-U(Z5iuA^blw5`!)=L>?=XK+Fm`YPWH6yfu^7`}f%pQqE;; zyYXFSlV&oF=CSeg+r<?7e@bFp`jyZ>Vr+$v>|O0L1;n3EEKIMr8D95T3y;vxc)Pg% zgmN^Mb-g-j{3@2;J(Pr&7zBAHJ6NhMtM~K`dOtFd%LKr~w<D6m5D&;wJf>`)&2MhK zPuRwe32_W;8R=4EGm|XiPy4AIWs`YKXN{r$2VWdI`+1-klg!%hOqtn7Lk={3Q5g#% zJK6#qoZVL!2Q#_)Rc428yoTsJ2$;f8I!cGFOrk^T00jq=*&)t-@ZlIeXYFZ*`}JHO z-*E(G&h>`(tgaDg+fvx{S1c@vN%Q|RJEe#M_AHG(OfX%#?=%)qL3aiqBy;G{A$i|B zM6s`jpdbnxnE!#lp^H`Xk06I49-`n=Bi-+rG81Gdg1xzAs0xVk@=+}y*eX|cx@Xd{ zK7iEJD8vzYeCf6wbPv{#IRi2S>!cAi_S9va)I2X;tOHsaC#kQ&r~-5tAg)#uA4?Vd zV8wf#P1Tjy73*4OF0;QJsC<8M$2f|(Zm+T8`tZMK{eQ2hY9T%ekd65+OZ3%tq!qqo z{v2&Ws%Geb6l9(JzxeZt<#QJ4D2zsjG(#cG=PV9k@XyDEMoy4h2uU2dY*M#~b1hqk z!FGHOVd5p4X5~1PKmCU=JI)HLW=5^DyPL^!I);Nl-!ed`p|hxnM5w+80Or^KhFOm0 zhQ~K%J9%lJ*){dW4B?Uidg6)A<sN?$Md@QvmEP@n?Tk)D4-R8Dg|XALCZ)@4*XFci zTCh%1&9hL5X>0W_#7$=7xml<P^VjpZ<kJX=s^)P_0o8~Gz&#{#QumV1r#qy2w)Wke z9FSdC(>+wx{_pKeSvWv&5CgnN6|K{_=fW+5LO6v+P(k0<8OV|r9=@aOLSlh$4~jm` z;jZzv-hA-<nhv2Pg#Ui4`Vb_|Ps4NO{mlqK^Ues9P4a>wu}lnS&ILazjJ*#80+fa6 z6iM={u1;1K1|>pYqyEV$f>D++-C+ak01=mBM|-;FR5$+V?wwJ?C*3<PK)4A?mLy>M z^<zSS!G`UFjn^w~EX9q^1B^wIteqts_YOrSo`)-$t0JfnDJL0ld~2BAA+2hZSSw6T zf^xbxZPDTzE-`SLxuJ%Z^ib3_?S4>y0lZ37ro0S)udyR;8Rz{f;x)2#N%T_frxCVp zmfX)twrA3hPbU^z`IAm-&s?h44!RtyXp9g@q8XTYGVFKwBoLp}^BZWNYsjjNVWDON z96RmEsttHg{#iGRx8!!r9kWc~WL0~M3|F<>yFF%+-_YbLFp*2k>2BmFTWlN3n46JB zE%;{lo+EX17sF#B2SaZ{UleiGkb3Kz(1r`Y+xi2yUhyd7HlwbyCjAV;s8cUDZpVw7 z6$Ui(KoP+($NkS}!$3PArfG>fHAb`fXrDgN?={b6X`AM1yQ@co0R}3jZjoY)3YlpY zxZVM9eI)d^>&A*}Csz2NgeUUNy%><GAP-@tq8S4Hd6gSv#(Rx*LlAL&eq%!WNC(>G z4_;=YhRZVGgUc(Rw&i<U=}mnr|85xOM0ZwR@qwqJ;TQB8y<N~71MX1AY~X{VbaNJ| zJi+uF=gL;P@t_^?USkpM^&B##rXf1qW6S*V;iSbmH-e%xpmIQlu8G_1jtF~I43tHW zQu`pG<4{m36GHpo_bi-9v-fU#eOs{JuJ>Wn$GxYqGG%h8T@o__V_2j8?iZrvB_`9T z)9$P9o3~92zw(yGYn9JqAIX-u{|8;Cl2tC!s$&9J#xGEZ{8SRl<3#itO<GdEXbwFN zNwvfRJM;071s|2qAS3lCeh^F2B-Viq_KX43+uj_0S-rT^=E<K7RyN5nb1qD_J9<aa z+5mF#oK~w_^N2GG`jUTpG5%S;!n$Yd{_h9fosS3iWhl`zSK#h0tCO)3V(_5K)0Q(B zlpiLTEn&6CL!EDBP0^6)XA!)Mcy*oyS>l*yn0E)(EB>p5Zm=4(;Pl+n$+v`4U*se7 zmiO~21xU<`KhvuVkLeK+ky_1aS_9)toMGBFX#OmsFs#3hqA~frqd+*aVwdg=18z@F zx1IPRS2hp26#_>OUD7v%sKZ~JG%ib4h*P(f9Vxp`#(Z9)#}CTe1}pB1X)mI&ys)YR zMl&IOz~y{xYEs16sLx|kiuh!?0bjyZm*lg{uGR6%HmUl?u>tPd+{*Xc*T+kDx!1gR zbH+8O6f%+S_8^r%!IEby#i|BEn!u06ob3Cntwpsgu_eETnAle&v^c-GmgWDi7GUuM z6USdi4%}q?Zmm0A`S=<>%l9knGEsk$KVrdeUmLTUrlJX9TWxJI4`F|Zr25MH@~-IJ zsX@A>REyUr(2yz$6zmSU>cRrc2|s`74u~4mj5+($oXFzuWGF^i5Er((n3*7#0%1Is z8RTlWvLb1K>?D7bdREwYC~#Sg|LezO<|_|Y$db)yO5i)_yL00w%+~YN5B|PlsFBu9 z&TI1jfsOw6h<77If^1N6jDpX%9(|@Cr3sB{N01{$=&y}3pMI_BPQj(^5Zt*J2dk0r z*@g~=9U==thB0}ETuKN0d><=70t;eLO!d!3|9iY2l;Z(PS<M>fpQ-lBUN^C;@w}rB zAORTyqtSlqzf1oyd_|dFG^nsMXeoRBZ0)6~JI`^=v>M3FEDW14G(S$fbmN|gJ~`|i zV^=8=Q`Bwu{di&r!C8$g$zSjD24eXpAQL2tm}2AMdZYMyW6Ah|sE>g*%_gFPc#)`4 z%0~sc^!ezuB}N5=hA!MN?0VUsH$}*O{@celPXo+xPwr>3AL<_>FT5q)Bwm(PDV6^x zK>`K{fqj;5p`V1^)3as!4s2NRit;V<K$+J#MO1>fM9-5pNx+C2eB!8207e1n&a<_p zdhV*-z;6Wich|1Ze;&cGrCs<C{L*a@2C3z|E11+?mOl@jTQ}Mppx_0!WiTJVof*vo zw#6a92xMI^5>skM1tf#;aN-Z*f28=%+Z(7SGv7y9`Hayq-7zmjb`lIP(|x~k{;xZl zL@>~Prxi(KvqL@F2eeG$Jepddm)B@f?l&rnF53Lap8Gy>v(^-uBZm3-IGLic(QZn! zwdfUf6>b{XU~kGr)1(I|Pks-8^eGG$+6RfXdVF1h)XQM%fMn;6ddl8kx4#fP3(45) z);Ev$K<7NVQ)yoGR@5)$cx8E#U+?8)<>eJge(@Z*e2k~~-PR_7`2+(KQ|SkfmS#2S zkEgHXP+1=4c486|9%ucR3L6s$<Zzq6{Vs65sA)r_KaQvKIo+xv;rHCZjwR+Ldm|5U zpT>&`s;y@yO~TfC-9SzqOI9vOpo_RA;7)|=)`Gg+uz`jsz(?a`ar_F9bjdb+y1q?| z6JzM_aJ!(pC|lEBA=fR{30dx==^dP-wb05I8@U-32_!W;_`q`POV@>p5WheR%q$?` zlgZMm$KV}b@;Gu^WcU$BIt|{~Wf^;SEsGRAVWV&*>)oCU%ADkqVGUQVNblA#e<Ya? zG}*enJi-C~FG0t|N|>uAx!*bi1E!s`%>zkK`SW<=dX?Qf9B0e7qZf~P!tz4)e+Q#) zL!sAGH&ZrJN&vfl)qp|onpXS%-A?=aM{Lb<<v-N{(!{!mt-!%~U%ub&+b!xla_E5c z%GQGi>)Vf;AAFvN+BjX2kYv}Wd}HF9#%mtE%GX?}o*F(%B3A~hb+2s#<o3v>3&`@K ztB^=fjCeNsSaA)mR*RhXjit;UzF6B5E!GjgT6C|d@*Z*nD*mnEarS-5W*Mo%keU&h zYM8IM7XbM_sJzQD4>NoJ;}KF<J^e7a)*YbwTvlJDVnaHmT(>|KiyEKr)_sl`^wqEm z54~p3nMskWkL*QV$by5$5^{o=agpP-W3Os2Q-L|_AxKxGvrf0BKT`9h*0Xp>Y1iSP zl*jK^?a?>RcN1TZr<nK}poK0ns5s|pG#@2veNk*DCQa=fhrkJ;h$d(2$um^zNj28w zM}ot7ZW<X5TR5WN0QHgfx{%WQ>#?viM&_f=0$nH1nppPkP>}UstV!Piw@lRY0;eBo z^(*gxb_w-aPvF6<E1X;A`s6`3ZD)C)ci2W&gyKwG4g2Oq%)CYojO#u}rn$u4r>oTN zVJ@mKQ?8@X!cC4E7B=cSu3d<1m0{c?KIxxTYxIY%$>ayo)pcSrFK;W2_%363#!z?I z$n&->&GNr5a)WTZg3xG$U9d>1f{h5Ys||<Pzz2cBo#3FOt1iHqyf_&2{xJxAkT)Ol zc&v>UjGDX^Wxwy*LZidyh$4BE;Kt($Sd_HL4=l$ITQ0W$PDZQFBtpAmL_=H1R7i+C zRJk{ZtRO6OF1lhNiGmjsfn#o7h1;XoOkbt5r!0eFgsQ%sGzNBM-#DXKqy52an4$IO zVd`eIjBt-4j_S!O`$G6DdjHby@)e^`dEI6HOA*C^;z9epN3-7cRVt==If8LFmv6u1 zMvmEwz=SU^mmCk1r8JG5x_$MC>iUEN`5xmN8(_1!lNrxRLwoK;bqm}jrf#ZFPC~D+ z8E_>@e3;;}u#j$yPYiAt@wP9k&p@Q_2Nlt5$#tm>TN-h+Kek9p=K0AcHRYO46k6`q zN>M#Zv19zGlSGhx$ylLhZyr+^Ej>}$vGp1zN5HUz^7_GXI2q?m0zp*Jqu?5X7hYk> zNXZL$+hMN7k5H;c$lPdI!MEiJ42L{PUfl}Zp(}dA(YPr)pvgP}lx^JRy4H#X<TrTe z+Ue5zu4@nin)<_eLjCnY6TpNP{wATnE9lDbniU29rxGma%{0$Ay|FnE?RN0@WwYBx zD1tFGz8EuwRh>pP1EtlRxJ|VEDxNYe*>Ssl0t3@U_B(BTjMvDf?C;xwu=W&la%sD) zegQ9V1O657ErAG6bUSEuWee7?IGV)W7fhK18KA*>vgl(ywV-bvfK}CbTl$mc{F$#n zkE;xdRoBVpQm@L4UKzr0g8GBOUc7J6a40n;JF6V-W9E18QZ<=}{)K9Mx9@hDBec87 zJcx+$`Vp^ex8h3!8Z%7s&C{Nq034R@Dg&pYDnvwKjk+-Fi-i!#RnJ#Fh48h%m8KH^ zMIJBuH|ses)nYxM5E{c;!o2;X8N(r&Q*2b6DL%+k_(mTQ__#>?Vme=i8OR}fA5a@j zyT91{dFOjR&2U+DG)>mQ!5GO_)qWPopW?xGn}xTpeUIvA?0)p-t$_cmZ|Ux)hGsu$ ze#_exzf*(|J`&XGDv8k*XDAuTGd7#XV&e)_434J-=v&Fk>o4OUYK#labx^!%#BgDH z_7&$&eXMxfJDiM1r9HOx<w*<!q$Dv`U6(DqX8(CoR~$4rPIy^a$GpW_*eCbqo#j3I zGPIg%YtJH($ilsUUS?W4D!WL!rIqsPbOj$8=<~D_wabY-i89|<aGi{F8aU4q_xz-q z#6|r=Ba7~*t@;xUy+1^de*YKcQs9;^GB48ETvwaXV$($<O(n*7&f*8aSMp*xm*ore zZW*saYwd5S=tZ1TID^%OFiMxIOn}gxv26V0iLw&>IzM;kv$eJ+L=mk)^rd=+<db#k zrez0J;uSH5NS%YDmd2Uh>=GSJ8X`vuzi$nOUQdX-Cajc2iq>0w@#9V;(PaX`cs%z$ z(>KR%z$nyQ`oo|>^e5qE)HdX0lmtX6tO(COOn^6uQ0@UNQ^T)n{dL^G-pmfMRA+BF z_V)?=@>@bYrzsJ;PA)DgQOu8u{m*jZ;b{L<jwQw`q2*?GYbrjAVC$#EQk#w(aQD2G zCl<hS0>6yo-58wIXF-J|U-!SE!Fl@P4KDQu``}+#GxF=7v6zf597qvuPN?X4%hthZ zs-zRz6sYcoHFf{7*A4NB@hUH`pS4JaG+9H?U-hall|)+-$|vgl)6rDO$yJ1m=_kyK zQK!=BHilot2ua0g3c}7sU(CXKh9bw#J;b{tt`Mt%?@E$v%PMd<zv+--HpDjB3!c5v z0kV<SOL5+!g}8M_!Jf6fR`P@}d;I+44S;m#De{x`gzcLh#w1@sHuKoGYw^cz)RQ1> zFw7i_NyZ^6NGZO-i<Vj^AMboB?)usDnTYEY`a;l)2s`zo#&h~QsB#Aq3S!jGdCL~* z1m42|t8LU}2gRW;d2V(WLXn*dCCNK;n18|%4TPX*tmW>o!mWAYF71_bC<nY~I7Fl| zr4dTJ;(WM!G}Ps}wJ_<~_d8Gq&19`{=JnqLoGzGky08B1b*c4`D>7vwh=Zi6Vp*!; zaU`+$Qoi9NuVLM0-SWJ5xeQ;^Iy3D^xXSNJBgjG%WpXYn&59MT;3$ozrJk#S7nhe- z`Gjng6LDDj@4MgG{3LvQ<3_XfP{<OR#B0s~G^~@}WD$rIz+G2SAcy=H0gw0Bn(aRB z8*Kn#7XiNcr_^DO-uU6`+$^--MGw2PkQze)$20U}9W>6t9<UTy9t~=d9L+PI2*h?f zt)nx_LN%e)20{m%T9^jQ6rka2msWh8VuoQ`PKRI7IQwaFQE(mNIjr}Dl5p35!Gyu( z7-7rZL(d&1h>-?34EX_ec)d%ZU1Qj^=Kq*dUxi0CE(Y5;;AoTTHiXFgGzQq0S{z2< zm3%uX<R7vc#LgHHa5`!_PcJ4!rd2R{Ze0i71`kMw?Rq*F1r&>~?`PQKL%#27WQqVI z>kV|~2IFKpWZ{5j8~H)2CbKaeeMlU1bsq_-A#B`Qzy=r9_+M_@^EEqH#2cUczq>Up zoK{+UEF?6vc=z8N3c*1q#smfvJm@<3Nidh-Pc$)~ee%RJgAZDoKsu3brZpZTLIj3( zi1ln&jN>c#X#mc$y-pDOs=(p3C~5QYRc6V)1rPf0aV<5M_C>m|GbcLl1*mxh&H5QK z<@%YFNd>;2HWt4VOPwKB-X@xm7DIi5a%>uWyz`l*>0-H|VDVhOSxP8Hg*k;I(0J$x zqs$ENx{*eV3C5*yqIXLWfeDO7ob*DqNkl3Q>+B3X0U|d73!h>?CFjDaC3!^dwpK%a zy;chr!GYN`grT=ffzy7N%n0X?mc#2FOBW?BR7=g9^}5yU3B{t15JpL*zQoJqvz2gS zi3kIB6|Ca;BkZ>4iu1+B+N!3k7>W>5`r7^x+$b3t0?wqWU@2N@^5yToDYxvzESG0c z#_G)4db@ENc1<RffXgz)*!5AY2mE}5zS<qNB9Du^(R7=ihK^1MmUr8$gPbj??>!^M z?AtBls>uqbm`lT(6RDAIFQyOiNdgG606>hcw0{=<=2_1&@x0v6-*4nWPt6%aKLPb# zm*J4D0SL~^!{kV(gY3u`K`%o!q;D{*r45Zb{DduHipMgk@Cj*zf;;@LTL&!*uBzk_ ztm90se={aLikD*o*gjy{ViAIaUXzXMusacHYth@^EKv=ld<cp9h@t2_R8s_GI!O<4 zdX}0Q^*AAefF&>)FSxvRqSdpL@Yhx$_Oa(V?(%k{xh7h?t#$A~5?=6F*e|v7M2Pmo zXHt43)!qJ{DO93#16(?vFCzJ9)bh0h$N#;qR!R~l`tOnt0<qWJ=1x(}9nf%XoX&Zp zG6wZj%vSCr{BH?tL_7IMm18nUzKWoWZGJw>exHigrM{1K(nKF?3&VWAZOX5PpEJD= zZH{L-n1Ops2KP_C2t&Q{h%$Y@G3FeU^*x2HmH6Pvs6AI~H5z%mIkz^ofRY9=W55L| z6Uo<P&6M9iU%u=8O3K0n>@}7iqQvcj2Ca^XMDUq!j1juy*)oiU8rei`UN<<-_B`?A zX4%lb639Tt@l%G@y-mxzd<mkyO=*wBtK@q3g13q2PLwi3o6zSeKdX`_!V*in2vn}< zh!ATjFSl}yYV<P}4BGxa{3JNkE!iqDknlb=VFyC%96GH$&%du9qZ1mP4`>8MkV-{< zx?}M_H<_~gRP;4~&zRS}{13<?FgCyQ$9Sp(V#gzyd18{EsfW=$V-y>n$Ap61#$+2K zLAoIj{=g(~<5T51a&u6=V5jCHHkONnIsERCzQ{`s@iw-BEJ@dC*zmdN?BR79#vQcT zPvPaeMM3w?p^#%Nk8bUcwY}!&kCBe<uHp!TT1AN5!Pr1<JWQ?Ap=0-25M@tu)u|7y zNwHG462oQM+9Op%(xcx?TOs~9_l^efnL`)+BBVv9l((*hkeM-u1>DbzN;z<6+P_RH z%P5q1r7^;5PCI`<A9bCKU)Fmij{XA=Uj(JG;eu@D*!g|l7lxJ-Fh`SJ>HrH0lQWrS zYOFYj(m9zCue6bga@iV#@85%dNHw6Y!qvoKu-k##7NZ*m7B>mNA*EDhgFF#`3}p6$ z!$s}~@7@=gElV2*uYWm4T6TS03q)ozdYAH#4kg8Mi+Dt-chsL0y4y>ftxQNXUaoLZ zuG~UxP)I_?e8ScvtX=*~sZkZ{iQRwbpP~S~<tW@`t?>v;W)B7}^%~t&Y9f42D`~}~ zDWkWhi9}?u1;@<6Wy^3(I2&U0=jI~5Dv{KdaQ6@I#+pYGH%11b4XvNf6cG!#u1nUw zArn!ywMZ(C<@w(8jO{(y`dF+C7`Rl=wY=i_rPi&tu5XaSDtSDsanu>0jL*>Ju@{Hf zg5DtZSWuL${Hwp6XL;7B@{$ZfJ1rl2WU9MdGtrtmdGVx++;b&NxSjkB&}^m&eb9o- zX?=PSkDkPsl+CCGf4*u)sqP`b$op>#Eh0}xRzN#g9h>K%Sj8$G6&j>Y{_DP!aYf!z z5%c`ZPe!~PgX|~$s?I8HHS{Suva)MF&u+rykC2^}p(U||h3AvUHk|_JZQNkBT+f7C zXVl>BG~|4Xt-0!Hoe;J&7{&%J7hf+v8M|f;n1m=iBbg&cE*27qyY6!|)?KBSlB8FP zeyJR|6titFzM20`>}-`>*}-v#Io&=!L~!5!4L15;cc8dI9*P#|$f1^`{l6>Ff1^io z>c2h6-8JNKLy#aiL2Ro)L>E>dlBh7!W%j=Bb84u7v~jW0dHnZ$!l^Bsv9K#RPH}24 zKa>aWd7VPbG|^gU<E7CPQzrCeC0XA2$;Y^nAdLaeW@gmDFFsvxG5XXunjVFBsEtom zoqK3xW^evz07sY2r#g4H<2n{gyAG2kykBRgVro)41NOjW@fwrHPzhI)$E+Q>J}{}+ zb24G>4I45ovOMS?fcH^a**fYG$=8!}>JcnG0-$Mko>FxCsuSvAMyXce@drk(L#sTg zExbD&SK14~BGnvs{#Lh5mr3=3M1p3xTZPmor2D$n^N7yk4oJr{lJ1G_C1uzP0dm-f z*l1mmX9oJ#Aauz*Xn|pY@Tb{e<@>ar0|zb5XT(rOV?R(K5h<rJql6a$k;>ypO~iyc z7CvRTM_0zPM*O)9t%Gz8NjI9>owrsvUksYvEDe(S-#qJ!r?<Aab=-mthrr##i5XtI zfqnwJ8n$Ok&k%CGwZOHnR&L{Rc$40oGSm6LMo*hhKUr)zFY!xX<bSSDK6JYa$z;+l zC}OtHE0Slz8C)DuFSAruaW{vhyw_q^@87{1gOH(|mg`G6`I343u^q%pq-@@2zQGO8 zSr(wbtGD)he}>`vcAHpRCMl%7*7FA=gu^4pX{_#?3*A`J`fG#I<zKuj$qI9F6N-lH zkV=Br{hx3CwS#Po<A-)P$r;{nCyI5jO5nwOyIsSVr5GPTQNhvUJ7CsJ5ija1+O2-$ zZuatjgQn-wh-Z_?y3Caq2tFwJKc~qSP_i^)turkA^>#G9!HcMU$vsD3{|&TCFo&6} z%&gZJ*hC8_*E$1eKW|UuVx7&EXyas6X5c@yRY$L3_(pF>OT2zg((d3IRTWDmf-W){ zD4Lf!mp&Uld#)`Gt65p*_)KEP6q?+`q?p2OETS3?r0Ijl`i8gyn;4eR6w0nBZ|Sc+ zqlg(JUvU=p?VJ5b{+!~)ioO#jLq{G8S-a|SVpP6X^AS1Cba%8E%}|voT@H8iPU13r z4sw3@01`za>$0!pEAr-^mpaW95(2(q79Fp_t{M3#8t0AKWT;PCRPxUluxmTU*Y7av zc~>2Gs{8Zv{)%cVyBCY6h8t@*b{!JLBeq>QG_A2geou((>>H;#;vE88fYMZ_ECkQ7 zNu&B}BE<F|jLA7pEW=UTsLgNq66YJ({Qeb1uX!sLuSH@Ug-je*+isi=TT|nD8ytwk z8?PJ|=&!oEROU{MCSR(hkyM&A=Vh?<B~Wj)U>Cp+^2A_&>Qgg)6U?J*@9?Jm)bG+7 z%<G$gmU`PM@wKq_`A$-<44<=J=G#mV@xu3T(`jkUc!}gVlu6E^Y~th|*f8%t`*1xr z(KfGH?F81vYk4Q#<fYB{Z)Ey{>&FQ7bKS@9S@<9u+B4ss`+M(V-8K=h*Rer=Le?iv zRubZh0gBBI^TV=0kWD>-Y5dolsd*y2oWo_=9uL&@BVe<Sm2eWRTTn#%OTk)qB6=J| z3nE9X#7RB#iH!@Oy;DCb3>5vQ-!({|WbNS_8hhq+Sm5prY%B~G!Urm}@O7{CE9>K< zD~zBwUm3SlygMwchrX=mdNjJ+Dfu%|V3eNZ9ATDH<`e4<1)uo!Oj077KCPiL2L7e? z2k+W4zvU}mco6b2AH~SSj|hLuP4=~)M$JNxjptjQK;10XEV)P|uY#zIe(?&i3euq~ z@E#PD@m*D5z&bpr>MXgpb#EdQqkTi`%iZT}Gx{9gGM-b<gW!tBDN!T&6>33GGK;dQ zN2K*0XO-ew-Y^^L!)y1=wx*B~3zsU@LjI35813RsajvNW^#m9AtQVxL+HHJQxz1?8 zbe?xifsCY}VY4uHpGUcW!GMdONm$%r6k`uf`_kGp=Lfo)I4p}mZ$=B!{~m#1MwXag z+WQ7gI($iuiH`zY&n@2Bk7`FRF1=4#ppY+kP#`%u_Vff9h_2gUB=#(*-5cP1t*hzb zMN+L4%-9EB&{m~X7<X~E4>zGb&L4=<;FmwD4`#<LOE3lnvM=7|f}Nhqh#&~haAaWC z37t)ukIuV<f$JU*dO8}7A%(u?M`nvXw-a8dv<tCVTCKpq$5?2AmTkK%)=kJtDK;VA z|BtXe7UQgfY_0-CT|v8YDPaZAg0j3|)@sE+zu^swf9n}d!I=vbQ8(PP*<-fleQ8UJ zT*M!=Joyna{0>s(PQ+i=cV4?^Aa-HGg5`H$=_v|Il~IfKj3#Oyz3m~B-X$NZ1~0o* z1t!ahm9KF`rQ161F>~xLBsx>RKJ*nL_q`O`hwJT;v=`;kMBeD4*|cByUL7yR9rCN! zSDjz}o@R^)Fuv}teG>iqlRs`d%*VRRx$$(JLWCO%+xb*>J$>5XxUhaFl~_El)g9sr zq3u?^OB&$NKG~JCA?;@$nFM07$`?Yeo-j@Zo0fTZ7dd)y++j0shYut~$Yl{2bc^tr z6HxJiGz&D6G+$O)t?0wHD`qBGxYqY`ZxhlX81mwILiUr-aXNE5WxgEp=>$I5ZXm=w zcEi2beQ~V%zmi^cs@6at0hk?ej50*U`=~iywv8WZ=ed9=2Y&P2G#D<Cttrf}e=f=V z(n`yGir;@@4>5HXaVp;yO~H-#E?F5};lFsc^Q}HzafD<tahitX-mg9dQ3K`4EUAKf zQqe#YDiK$*{$UC>wT<|}#o<PUjWA;Auk6;t<s0$HB^%NzL?9T5^!u;v_yet`WzC;r zy<iS-?V=L7y`L5icTAH4Hp3q#{b^eeN2yO9zs68qaZmf9n7=ZXPvK5#F|4F_==tNc zFxB-dwreo;GL$L)Z?%o-PD}qd@0a(QuQco5|4UYAPVB4N$_w<yh}ZJ{QDUmS4>{8~ zlzFI?T;^@x<Xq+fQV6By^%z8U9}BIYCXSx}MVIcFuj40hk?MXXT_`;y;LvvCvwJPZ z#T#AI_4q)G9mk02F*Kod;Id%<U5A^9zU@osa|q>^ERVW`v+P)?OC-<~{%UEe##+T; zwwM%^1p8TkGRB+d&W*mET5&X3`4oNlZKOM~t)IAH481fq$tuCKH73gLYUI<928X@f zZ?u+c7s9=VT1st!*~)IPRjqxq`i^hk{z^V0o(vuNHu^+2&yY3nIIknftm_HFe<D<Y z@wBNH=Q@qhSR}Ok(nDcvkzc_&9(~fOYjQJN|9TBEgSKzV@i`+m)V=B;KmjrMOO&l= z1z(SI#kj+s-eIXe0-gtICGJO`W-RmPjnEJ&JG)ej%#kr_aycpA@^@u?VhU1_ajj`d zFKPY9G3MmFw~)IMv>r+Su1P%GTX)t!`WFu#qG&wqEbsmWU64Z*B{&8ki7rGg%4W1n zCIB?12Qbc@J$a<8{6J<#`&USwn`Q-ddP;1ecAr~$Z?e;%&uo3b1vgl>i5L7R;v?XN z-91X$d4FZRS2vj<<{{|}$9sVZS-Cm_xg%cUjpIA5_3;n<Oeek>Z9;km$6^I`^;Oxn zSD=(|YtPW!h<iI3djzPEZ>^&T(g|eOgAQ{LH+?Tkz|wo8iKejM)?wc^1seJ=r3X>7 zQMfMpewFh3d3hjNk#9Lc9~n~d1k#T1n2+Qo>+c(xfRsdzx*oDk1cp^TmIDB|Fy+R6 zFJdIgVI3nA`Qp^=f%o$s<?)MAmsv=tmx2|+5~yaK3<*H3V8SlS9v;2d$RvfeTnp|Y zkMj|r)JNyRN^X#&pqE;XQsB5CmZqT?fua(l)!!49^jqfzuJl04ip<jUq2-uPffEIp zc`oh8^JK-v37J-ZoOH45Ug(o&hasc&llQ^9_u&w7D~P^i!hP`r?;6D;+xTDfBvfep zE4T0QFIlm2gvWX$4a(B9k1BwoxITbeEC^uNmG{z?Oe((HBu=Ikbs?L-yt$)ivc3HP zIzJ69zj&n$OF9#^XVic7wm@HaTDnhi(S<lWUuL=r{5JbaYw>2fZR7H8S|pvy;yhfM zUw5wXo}SZ_%*plXN_(F}Ip>8jhzSQ&dst1B<zFqC(~u=&piXRr0Qq(+-WyQ%|DHVR zjuvoxL+MZ(F=^n9U@GLmRXI_D6D07w9&_Tme7QI7efNL00K0q44!U>zRovlkkB0m* zy9k*i>C<m4Kji~?PL=pohl8B1O#l^yBjlppb!x(#W~4rp`9EWZF#Kb_#nI3JmPqO3 ze?un7E^316D*w!RFUZdqG+`*Rm0dzohU^}637nYjs8*<7-)~Ge_i#H|P#1c*L51Me zR*Hu0Vf+tZ_P<OHS}Kkj$i|2QD-=$05^>)Z+4INj@Q(n}pkavWcg72IJk>NACH8cp z#ou_0+Ue{TW6QyCq50@8e(Qgs@#oaoJax4P8}`(}$AC{r%#5i}wdX0qSFZVae6*Zu z`(NN)v%gTXRCQ_^*|1}fwPpfAp*khyj&&2&o=5Y130VO(?6ZvIo@n*@Z6B3mngs19 znj`HKIdm<M@C_1Cm(}V$2mf<IY%bU%zRXX`!7IiS_WN&7<w<aLHtU?*>ccdk4+frJ zsbr$Qcj6{nJ-sBP7e&T!fYr$BO1|@jffUa<#rxlr_qBK9(-1>VQD{k3#!n=YofGAu zvmd$H8lyhgW)p{8AsJ%t0JcaLKudlHGKe@)krnH`CZOpN`KxdVb$O%#5<7XgSUg*W z;Dc?h%1G~7oM#qRi0NKQTm=@mLINIGEqm_{Jeho#<9Aa^dfD=dbY`(9h{Z>=Yn))Q z4dIcyrWW!esBW@2u-eT^hpV=qjOC`e&GU$YjMr?{Gh@QMgQQaO(tdFf;&7HF757~r z+vxovsO9a>6aUm~1^9EHdbz^oZ&Ma0fmL!bDg3{uNYG&hUF9*%#wrS%WDs$$eOW=B zgTy%IDo>0f(pX>dEG5lv?5FL!_)8Xlb$b#_-}bJ&NN-yOZU1>!C|6)b9<!&vp3L!Z zofh4(D;Fr~R}qj09jlIcqhcS$#F~+Ad;ba}{y@AEpK+!oqk^$KkRlo>3Kh}e0Voh} z#h?wb@>2EEDASzoCKVj%!mV&Ky&kLMs;-C7s}70YLEEDSYaCuqlAiCyB+-<l51Gu$ z4$i&3YcMyb1P(>xTzRY%NnZglV!^w!Z^)s@9P~MXXFS*Yh=GJtk1N;p05D+;aVhLl z9pu{8d~)0AT|=yyyy@GE4Mk$*gE~Fel6!X5(;IGUPGTnbS_Y^rv$*{md!Z0z{1x*l zoCEM+&WdA~xyQ)esl)6JtN6k6(9iAygw|Gaq1sJWc#Q{}?D`9_=euCMn;GAP*n{8G z3=yqj&&lIG$cCR(>~)<UTYFVOu>!<Z%<MMp{44E0_eJU+vGCx*Hmom(j`b2Y%KFR? zn|pmlEdMI@2CA&2t2tw|B-nUl^|=p_L6*`~t`X%{sfQ=i^ih^`#_wQFjFTUi4BEj@ ziiKM(i(S$TcLdK4dFK_*4oyurwg9<<LX}xZ4nUQV2->O1zC6%ntcoBX)$h(CyyjuZ z_uMjb+8J_`cVq1wvhv)SdI_K2BrZ6iYXU#-A0wUgHJ`^`oryK&L$iZq<zl{%<}hvz zq|ON1{It(KFG|MYWh`@Z_C(RWdk{Moi$U7Xy^mmH!y0UQB_*3=pM_@{9Qrs2oV;+M zYyw#k-6Pxkg9dw80B=~0E(TNsX<!?p+2(^~<F_%UfW{Q9LyJte(UJHEeaLX?&ROd< z&hlhtI0PBv&QQn-j87`mwZdr;A%vJ;Z5KnG3Yn<tvgN_Zv`R)F8IEqXT=+T5X|=SP zI7ia#8SX?{w~b_T#gS;|nfkvw;VZS<%qDXjl-+&ARr;(m#%dj`j^4X*3rZmk>?|p= zTY0{Hzv%+kYrZjNibX-B*4hKdmd2u5LHzy<0TE9E95($|gh#)OZ>)@#i5<C=e;EoU z1jjULbsbcS=DAxx_1BRy5)DKy4s?+_!LAd~z~!{Xc9V`6?<B{*cNc#9#1$bt{Y6Q4 zty=FEOA+Jsr+(X!U+(wJ<t}Tjd=67N!@O#1WDqf2-+}MsQx(RdmiEGq^8~kEV|zr` z|30-n*EG~=Wt<YN8jcQXyLdEq?ZBET(tuR6npf6Wdj;mJ1}`-_8zc~ht8M+Mr(+Pe zhgW~}9rLFN_WGZZvh;~$`?kto)?ezeO7wO0V+bPn+ieZc*N=Eiw^)n_Q2G<a2zpn% z-Wsq0FQ*DlZ=sz1;XAHAY`Z>`2nk086kA&4M8c@++xK!tPzjzJdzyt)x4z8ncwyjL zi@ZqB-$lV#G}`$2N%tbbs{^{>*<Mdx9weZe;-c`po%}M9IG&1ZJ$+q&g7E{d<~y;| zu4Cu>8GI6_U8z>P(t3TNtB;S-o6?$cfw0Eka$)~Vw){6cx&siw+yGOAX=$30;`jM4 z>XpJ2Uny(8U+tSb62aXfjuE72)$cL0zDJgNt`Gk{)L@`l&trGKXHDD0W=41+qOxs4 z0+(XTZ<wEvgTCok_Y1TWkt3KWB@`<>a*{(8E~bP+^`Co6YB7EVJze)=c%fLGpf51~ zQ<U?I$!uSV{DA#fQsDh<M2^XifXkx=@4(5OM0>#2o2`B6Ah{3O;WcU2qzVMeF$vrK z10=XkCG_!Sm4#P<2dyXHkRgooysF2#i7{eD1{x0hc^^QCn<~&xuGRAQa;~-XsMl+f zwh#36kttHh#XQH7rnocXSDj<~xN6yH`p!!hXg!jOx-(uG>K6Qn5Su)o3Me?_?fW4W zRbWfTtwoks^qo-b`(9|#cbljWZ=%C<LLAs!`x*z*E$2#g|Dqzyc+eFF+sU!LVFt_9 z|L6@?UCk7wb*hwMS*0@HHz&ETl)7)&^snUsD~}S0NrQ5X=w#Q^{TOx^Dv!T^mw!eV z{;`7%Z)JqZ|3RJ0dUN{?e4<&kq2C+g8hLzP3xr!Cw;;J@9ey8zu4v~OKj^$YYo9E8 z_Ti2+Nx3>B?gtsQw)|w`J+s%UO7F<thL(<_@|>NaywCRZuQ$1hHJh5(5k~P`OHxX< z?J$t#8=d<;!54YqIkQvr))CT0lhHg2IRQsl#L<KN{;{EgwPHFH-e%T%vv#H=1MhAO zZ3#Uv;v>{T_tc(Vg3c)CgjI!jba_rK><;ERG2+iR$x1Iu(B1M02?95BdGVgsU9Of@ zIPusai0;$loae95>Y}<=<piJm6>C>bz->C{EUTjD#Yx$fhtS)?mFg#R;qH4`rWPmP zn@m|=8aFykEY;M1oRLox`UrFS8BN0NOYk+Fk74ZiwPEoW<7yvf2cd=nK^%lzg_GIf z!rnzXvrgt7;n+_WPO@?>%<nn@M0qL@2EX)qwiT6shpVW?daU?pU0$@yWzpd8)T3|i zN6dX#K0F<4bXskGGU!S3ii^z*<->QeRYSs}r)Cdr_BbNiUn~#XEXEl&f1q{{XeEzv zeofc8<o*0Pbi1jj4{CV&ig<JvN&*JolzKe*Lu?M-`=oHS*^(olOUdLnvgjtb<gEIW z<CSCr599XRUxYT+bj-7V)y=*JFD|9^Tnk_E`=6x#jDY>8aag)g%u?gCo^yI#Z!7b~ znZCh%N}bcYi_m_{95-6Z@1VP3RNwMbw7c8w;;V%`MA9(vM~|YY3rJCv+Q;l*8HP4A zNFkWaO6n=?@=i=^%2$o?)`nn_ij~*0fXe>W*D)Sz3y-H0PlV>rP@Xt?5QAO3?N;in z<MVAkcf^+>wKSdinfosKNbH@%H$8q_RY1@De?7!L+dus8Xj|>*{s-kveIazdSYsW} zB!|8ywY)2B+uB=D_ak0Hdg1pw!C`3g*My-HLH4>0NVLAi)^E>hj~kJ-sSusi8QI$U zKRHH|;&78B`^@|L!yRjsNl%#x!jVES?lO9%z#MD{HvcR26<5}dQuO==g&BVn-K8@J zd|G}#JPRk*m36R#<gyNxK9+MGDg<^>f`*qB;LvQjMyk9WOEe4gn(E!hWHi-M`C|O8 znyVvp&AjTGSAArdTPOBGN4GuL(pM~}Y8%#0RorU3IG9Jthp!t|C@mVXESz<8oc}Ek zN=ADc<31bZr*CV7*Bun#I|u@^!0zFjFXL!^sy7*Er26B+{B=E49cI2BReCjZ^ORSO zF9M<tJQ~-fjn-ye8`f)=eH7Amfql|LRDVCVRh%I&m$79Rbj){y+5<FsPi{m{_hW-! z{>ZC*V(}(-F{R{8BD~!dDqU<O{BPc3*_A3ifmJ;ez}{87iU>qHRwg{tgCdEfcJca{ z0#V2bOGy8>XBeu#68aWW2;Xw_Tlx~{#IVE^R0{`a=Zr<;NX(ZTKGRa{1m9SkgxY<+ zQ_nHg_DOA;(7;VCc?A`>rdV-lY++}{0Ra)!kF*PE-GxEx7|KUCGOl`>3NSQV;{8%a zgzQgBHG8~MEr;Y%2EbWCU}gg=;praA_=s-x*XwsoStXM5P*&UA%20%lc31SHgfoV& zwC6FurnE@JqQjjb!TzrTT15RC>-k&6%5CQEHBDJwN?i$*!i+3d0z7Vmx)t_P+t8h- z00e&Ux!=S~s3L7q8T<WR&*+z#B8#DWjfd0q#9wx~=L5CLt<rC9vqK@Cw}dNaSj7I$ z9~=UkTGw>qso|Tt`tZR2S06y)(~T`Vj}=<n<VzBXZX#J6bF=7p{8-K<8NcH0a;-Tn ziF-WX*M9pi;@&=2^+$2vR0`Z%45g98Zqnp`OVka`xCy@#z+6IJV{=o{6fVYiS>Gci z%d|0>w6m>EOG()4qV-K}s@l*r1A#h5EIG4?5p}Ge5K%nIYOk+ETusg^6e3QGB=f=q zFm0?ZgI!V<<=1b6^K?4FzA1qCoGK%1QtR>CY}J15{z{XpL>DEa_!suP@0O0r3ZdnY z-ouA?mMe$cl?jv_&l{bVo|*;T*y30`Ezu9~-J+G%{5<o^^KWhOC;B>VGU9y|t)D$Z zTwbqcUxu+OMlF5}Y*N^%ZO->RS?{1F2~2!7Ze@Cc;}tsMS*F-Ml<w7zQ(^9SuuO?c z$K`*innjX#kL7k1<)*5lGu2(<DxaTB6TWWsu=v`SzFp!n^D7t1kP-kl_4W#-9W7>- zV@W-)ph7~Jo^We^GQ{3P=xL&OPea+l^MCe#ehyIQyKnYC@rR&cs?48;9Rw6AZYwOl z1{h|`Di&n4JNMhUqq*Qu0f(=H325(Jk{L6k{3#lpSFn>^%p$X`v%~0KepYB#^mbir zH%60*GHLfA{kJ<_M*hr44`Yang9r#1*5_D*VHGcTB>g;mo%AwPTz7~>CTau9u{94| zU?~+hLKt~vRd>&V&}Db;^UxvF((yetW^S=O;26(Vv|)l!4~(JJD?S5hU0k~WrndNg zXCC{Su@TtNi`~o)HzF=tGT)XVPSa1aOM|?T^0K?;sa8DBT}~Z+ojm~-fVi73PASRl zWT=7Nao#_f1Mc4_m*}U`omt)<@@_p;dEgKYt9=b-UGI1|V9NJ@sCvt=D8q(ZduXHv zq;m#Qlr9CyAw{JdrMtTYh86?~gOWx%q(QnvK)P#&?yeyQ_U-%Z=X>|}1O5&N4EHtH zTIX7;39Wb3J;Y)-9&-wB*rUzV&bW@_^P<@L+(NbAt!qNjXK9BKvN0o&2XY1QtW5PA z9cKB2y%@$yS^tRRC&96!f}t3CQ}!hJBz|0+1o6AO#I_Vayu9x2gsZt{Cn$GvaL#o{ zhmRqK3@6(^(#SlXLl9JNTUA-Jx!m_E2lU{-QRIsDg8h^d@Om??XD2&uN2b?f$_8R3 zD4^uE11NT3vFA9@z=glWTw#saZBpThUhWkud%M)2IgY<|R(Pu-vzbsPdZYfX<Hb== zH2qkV5FL>Nt~WDbEABF)z;ACIOkvB0+NpBp9?eyyJrkG!uz=<0P0_~d)#EOk)XT#D z()pJ3=;#bM4C}N(2~QIysPsld9QRw@Um~oy@g+qJ+Mh5IU>7@{D(YCkyp43GMfa{A zm;fRfsduy5?tU@T6EzPH5_Fiu-!%d88~6s?EbGvyW+fi^G>v4U#~g<?pHQNbsDgLa zV&U}EG5&50EyBJ8svf?_n#bKr+5N|Ep6R$@$XH!9U$wdyN%yxK`ud?(B8LaXGlpLc z9lH|Q8qT7`SI=n;aiOact#>6G>NjZ2xI-boJrCg@oLdgdW}J2Q=hV`X;~0T{)8k-% zMP`w`K1|LDM}vs)SI=<j6(vh<9!RQtPhdZmhRvX2uD(bs5wWWqs5wX}GBE)vnGcRJ zBWw=zy%tJ)Jq;K+(FV35F$i6I)T~znddMCCq?JyAqXu@M=y%vV^TGj^=5>T%1yvBA z4NW_lhWyRN@k6uj#zY@p)ysGdZS<BEp>N-J&SlQu*V-j)hq2g~8n6D{z{LDklQ`vs z=`?;$FrdCK)R2&J9TQ8tA@hwU^M)x*Fk)h8=5JKd{CA|V8<+-5fdXYdIWOI(8xvmi zt2|eNo!NJ>Z(ygFCmjf;(mi-Yh%~w7_Z?lR%N%a&qZfbYgEA9_KOlR{PFmS7(9~H_ zMj9s<ka9*J68er-7pz0F#M|O#wMJd%bFvl$=<&Ub)q*xDDGzIgNRP3kH-fB75VGs@ zrT34~0E-5o0pthmRp1!B&(o)Py*N?|cb!ZlTCgdtpVb(owBD~wWXO}gVSFbNn+z+^ zz5EfF)O1q7?#@BL)~cykVQ-K$g1gjp{VX)?rs7%BI~-6gMnEChfkF|Djp30`px2tx zRnq5F%NKG5h-eT#<5l?|I!`Pu?#9ovxKRDWdiRHX!@UiE7ykAGT={lW$bC<a#zzOp zQKEIhb=$D8e2wzEfQo64RlL)R7Yput#lfeB?0xNCsSO+|M}Gf%z<9qDHzt7YVoFUk ziZSF62ARII7D{k?>Q$mTN_}xp7tH3PntW(L04Xlu5qOq_(f{To*BNB`;U(2%4Bsb7 z^Y^VOrf&JlX$<(u@tyr$twI?KRG83aiUZlvJv_g`eyy;)BOf08?|7QJjIcWt3t(CQ z!2k^-euISj*MfH9i=`^g!el@Xq8Z;QdTx)&74Y8N=|DQhUD7A$P>f3-7^jqCn?SUL zS6e@0p@6Mnvi=7u=J$G|lT9P94S<4K_KFtEFrNFD@)7x&LH(Lc8z&q?@kvjNdGqnh zb3FZPFUWN#tYWI0EHX+y{<8y?0P@(TS#doIkGew9P%VkBqqCRjp(`*~Jno&jDz|ot zfZq^1?|p*n^_-kg^%4Hvr~}g<&a~K;X@`x6HwCGcLRrAn96AgMgYPLmgk1-0M*J=J zYp;yRj-uuviK3EQ&qCI}PKzDhqvF`AGouF!!&fxWwZz7+FNI`y#$AsP0ZS-y%jZ8p zMoi+|?l|n<VL&3s<ZzV#7vL8Imw|_0baJHU0Www|<A(}<(`P-22g;pb6SG*IO$ypA z+_)p;X5SZ<G{W`!!qCL0XNWVZH6Kp-BzD2l*l<gltxv0OaF`O}!^(<3OTAl2ZhM?P zK;Gmxfc0?(Zg51sS~gs8mnDw#YaVcB*l^UF>>vKkDB3Nsb{L-x*{)5ik;8?mh=_<z z=+`O8pb$$lOt=D36k)Wo{hF_45w%Brzo;9v4+Ec|H|5>OIns({W?3myr)>g`+_LOR zNDB+h-rdrLxoKS<TyhUA6Vr-$^EO?~*_rU1qCdu2TxRd*W@%Sjt69Su?eJCCdg&no zdxv^L_o4NRmTk+Wc$ytEK6%F79lSZ6(BE-)wliTQ+4UjUTGasJ8AnLLfo~zA8F(g~ zeWB;b_u9V?>rhP7cWaB8M%yfN0*`USJtQ%zl_D?w8c>={0L3XCKA;cfXp(3RKV>jv z2&RrWdcYG;Gw&Jt9JiqPpF#7FF`P<yX=7^5|LcYMP{tE!vXqN%Ak&&<()m$+p3ImV z(PY}COS#{e>~w6Ja#8=o=Q<<+>ajCX?z;XKPx6<3Zsk$-!#pV057C9&AmdXYR?X~^ z-YcCbcs#;8=XO<vKt@}t`9FTU728m{6OPfQ82v+>C(=Vxf(c=sqk8^cQzaGU!a%t) zK+PTAoVHv@G~(a^+pPKL;>ZwsbuLt>`CbA}3ubz3$-636z#L@x4qgp74RYYk-hkRP zek7*H1%8?bz8P?3c_V;C@UdPJBk*3=08m;PUn>z6`1v+<jC9?znjd3it_gknO;6}s z9TSs`B}94;W-3@r@7xKc7v#0<F9{~f)TY%ish^BCya_THWNDD?bZ9Yi*`yL0q$&y6 zn{mk!$1e-~oLKhff|MdX*I^u;*0+`|48PBBJQSEcJx2F(EH!oqmmJPdI8cK6&J>=x zPlOOb&2G%sTE;IJgpIEPF)ZK0bpxY=%QM)&H;`sQNPiMb4!kN*qOEl}A~){7*gC@~ z)uYb*38kHive=&Zdtxk+V1Kf^Vc{`X5O;#Cdmy<-9cnWMAFsXPvl_Y60UhNKmj3;B z(m)zn(T*B*)_YrX&cIzIg%EOk1WRaN(S32}?VVWDK5@xvmM|B6uU(Ty3}s5F2|{`> z+b$s*kO!?~5`v#FltAyX2Sy+s9<WgC;a%T0I$cvJd+{8$Wo|hu&U*t*LzegbGJU!v z8$z#vd%FwLG(Obm1R3k#?dI`}EO!~gy%_{(7H<MLyTt${KDgd(^VLERqq?-GT@+hE z6#CV-H?5-$P8pv9lkFx7Ffzu6oxOHDd1x*Hto58)%VOhZAje@y#8roJ?h&rEr-=`1 zi%Cna?GTDix=+wkE#r+nA!rRGsh!*yY{jGq9yweKMo)yy_JVDT-7(CIUTxz%-tOq1 z=p{<cL4JMky1TR2P;lUSmO^?hK2d2`zERVJvxV3XHY52x??NL<A3GuH_5ja2ay=3% ztpy!o{aARyu63lHreL4>ZqD(x?oI3L#PE&MB%;a5UQ$szAp_Y$(zGMLXWT06UsyoJ zeVTEQRCk;_uRHZghG5t%nU$|=xK+errlZ$=sOPtuLF+#yj-*H=gCluo&}~>dK_(xe z%QO|h>Qh0qc<T;{!J|922ZcDuhF45BlZny(|EM%hCSHs<2XOcpRzK?z-!8pSywJtx z*GgAi8Trcqot7rAi8Pa-dL}8>a*Ur(B=+qOat$bPow(A&esp@o$HerCG-!4^FYxmb z3Da8lz2=<qzNcs(mxeM$;cNa$9-$gxwPrEiHWH-)XwhnKY-beDlE#x9i(1+xvPzvn zMs?oq%LU_-1OvTJUk)KTA{$Lw_}Qp@e7MLIpKI4^l(D7xJ0S2D0)*muN;sDSP^_YU z2cFOypwbB+Bd5BJTc%N_T_-)P5bJ42jKzGYCreu>lw9G_Ir|d#UeqUpo>vA)&ewKR zB|(@tk1!<sG_@*z1Ysu0pE;=!MVcls&Rt?&x&vI%8bHfS)h8MTLR9XdXPwzB99iDk z(pi^zaEV?@ZFLgPKOruwy<z)$buvYog{e;qVCCp<4KX1eyN{Uj#&Q)zThSU2@4}K} zw<7$aPKgHh?T1JUgr77Huw4q4-kYgld<mq;Pzz3VJZ<yMi5*kt5CT1R%)=w84}9%k zu;epMYMft!GqwTOBsAN!#&**9aH*NhL(4&v@vT=WDXx7VH!L&Rf?6oT^azOV#02~@ zY_^?8kKH=tc?RnYXsZRsR**97<RoHj6>aPQ!$QVC3A2EGCc)|7=hQf%nj1zURDw!A zPqFWjhv~nSQ{V_w0t{aR0_0?P_SF&x_%w(*aIYp{HNe*lS&0udU;PC6#t-m=UyIbT z|K29$NZjx3$C{`i>d97%Oy}8Evpg(xP&2zHRHGkcR?z_Uh#4AUVFO?vs=$@AyiRTx zv82>!V(3-Bg-tc$V(8VUepHCBr6v)BI#HWgIoYV<tv=Lyf2xztf0d;d&aEnT=q7yw z&~Ae{gwVPl3mC2mpWlZsn8q0i9Q)K#jn#ktOPI3cM=!{=IaIOepcVO<&1CsTVR`#y za_7{_Am5lw6?^n@J~^ym=t@bx?xWxOPj1&Q2G{DvQVul78_fe&-jphq_gUb}&cT>( zVdq({K+OgkI=lKuhL0-*|CnL<>l0URI;oi_kkZi5PvoJutc_>kvk4oo2nyaGS(KiG zhYy>9Y8>Y7bE)Ew?R(rBEd7$_TV%z(B5x<%RaQ5^3n|E@+gC;NCO)VRKIP4ejO`up z!Z+dtG0c@Gk(=iEk}qNRsvy^)g5+|ai^g9PQV=8^d6hY)<V&jQ#uOo^j(H>d(8UZt zTrYSlu`tI*h-R|7==z@F86PJDm2k$WI9b$v1#x&|$M6${gi!bGG2zWu8NLU?5rBcc zC9rX6<cHUr2njte?#=*@P-!~$A|9c9P9{W+I+B)HSmvV53J<TDW9BODZzCmOe2^-_ zFqC*VP!5dAbyFP_lJb)xQtRP$GZ0HqZS-`Apynwu;yXTNnft`oYqZvBg!{^2WgxD- zH@lf^K;aT97AR)%=lQY(>8ZMR!nG8hWNlFF?461iqSuHf=*ok<%)2ZlW45gY(S93* zd1AmLu?6B6)TlkS2r@r2<Iox#$H^*RxxtZ~3vRS-B=+k|J-`c%vPf<-uO)>}*%d(@ ziv2#hgdEWq=>pjj-H)HX=4Kq5jd{86C0(={krsVIHyl9?_GePF2w`LVw+xM}o-@@5 z!3+DHSCF~j!`f34qt=6UoLYaCeW7=!1z-=;aLg)JWTBBARoJAv-|%gZpXD6x97$Sz z_++@??vzsRU_6JKnPlCeLoYE{_4H2jN;cDH`85bq{PH{_podP<b0f8}?Wq7Y#=!5Z zpOOlbvXgf2Bn&(Zzpz)4bSc`Sfp(=-`Zdjf)DHUn{_kw7epDSip~@L3?3R}eU_pB- z#G}EY4jxU3+asCRUu93)7l}tK!V1J#cJKh>NqU~*VUI{ezY}nSGG6V8?!cd^vI6J4 z{_<!?yod61^}j5DEes;$cv#;}g)>GBR4j&DX4VovSM;agIB(`x+Rv8g3~&o{{||al zO)qw7`~ehT+$Q6xNW9i=Z!uDY_ZKVs@T&!^W2>O>8VA8=9jWtGa~puq<Vt!qyJvY& zv<{gT+ifKiPP!y_rC4e|f~Ttko+-jne&2}=p_xjzBMlDB(u~OTDZ7Vdj~b6(kOvOq zGHsYG;TpcEF?xQWcq_U3x{gs0)TDGfHuHvdnhB_K+WG9`w{%vCHQH1{u++bTc>d9T z!^qaH-&^gp%<*>A=Wy>u4ZK>-p31;G5M4ljDOfRZJ(4qQaazm%avzsSoM;nM^Fp>= z;J?|Oe|Az}BOY)He|V-#iJwh15|U*|nw7HtC`=~4g{qK;Me_3quL1@C%0L%ZTY*UE zCnjaB!jz5f&0}Mi&aWQ1i2RcylAEEo)8W8rH`jZU#Mq?0@5O^ZK}`1#bms)Dpq;5r zj`)vK;<%XXPWAC2hvz#J<QI+$jA~Rxt>VXB<m?92MXPDY#bF@+5D6O-qKBLDQwt>K z2!OFApk&A19L|U>-~DUR34JrPR(?|2eKh*@b(qfJ(MF1sG*DC^?UPLGUh)a)5gI#R zN6o<jKxa&mt@>l1@O~_`K|VM$g7;66j}lh%Ob2`S+PQ9)Zy1d_yI}$0`EC-#&2;Wq z*b1_rqFtse=fI9gv8nAm3NK*H{dF8PNaIQEFfseKs0m1sfuD&xt+{~0=vIbpJU2A= zG*KD|6Q<R9#}QyiYjStdY>X4~G=VzvqVYolXJ<T8+!otl1f1HXdc+t04<DfZ4NmP$ zmD2wZ(p`WgwbEP0QpmVj?*b!D45V_^%H=dEzJ6>(-W67|dSpq-Fqr<NCN7l4IwBBk z7d7A1&OQc&Ibu_?&HqZAFK^-DA)|(4X}hs4ZG|huzq8-?;j?cLK9mwT>mWFLsv&g| zth&eziu-FK=BI4JIO!2m^9Ng&tOnv>>_!|i)nspiKA;Y?$HF)zSbPMX<;ryD;mES6 z$LM3f5o!oGxq&dEl4|2SdN}VE=6ZTDi@LMlvS+-V;vz1+t|g?`nAkMDc?xGDV=K}r zHp03%xYeIg634p;o{D}S*8H@XJWC^#Yw@AR+`5_m1Z#tk3EJm)MXs%$ftEf^6A02- zPZm*902I9Pk_cl(%fMWy$6iVm;YLh-e$l>`(^SW7X6N3oo+G}5VWYUXlN~eFpGO>H zMf2x}_=`W>V{`$Z*Po1pL3pc(H%f9F36_`stjm{n(t5r35)jzfv$>m^f(@`cIY#)m zF0SM4M(XT^e!HB19GP-dxRLy*uM&zh+wg)9^)y@lTBr|w+gXjo-KUqzO6X2Y+)AQ0 zAPPK$xw7JZf2P(^Oek}>>Lb^*#%k8{?1f#bZ$Qflp*!0IYc`Rlr(un+1I_EtcUwBO zqWY~L40N9wjy>e@eWbH)RxPS=b`~e<<v4trJUKu4QN6T3Y~QBBl-BziP#}48&tK{U zXqM=b0byifyt}(cic0!|mMi+v!~pg7a4zy~Pl5H|O0QCwt|dkA)PrwNgGK8O+niwO zr+Wj*0#?#J>M9@439sEdb7}%JAgHb>%(`LHN8LVfmEmC*`^A>?Z3u7qCg``jq5pk! z##86JBAqg(wx6S&5{E~b!i<DXX}9a*e)Dn`$NQOP2x7^l%AF8=L?rUv^~jwDUYhRt zTlO2X7NQ?`n{xKy(@&mM6OD$wGxY4YZ=3emv0NV!Jo7J2JnNus5E=_=@R$!7mcOAd z!>b3s$yqw$y&ZWLOyhmI>JWqqQfHeD*1aSQoX`YOTz61qc1t_l%I{7G&FGqW)4au7 zGqp(S=uJZ4iskR*&X5?)JH&H3V5rVrHq_oM`@(kWf^^Y+Q4S+PkZou!tyc*{3co!_ zSMS`49#K`R0n!qJqm#U>(G8>pO0CBLN<A;<R%T12VGN97eN&)zr%aS1%m73=zlCkg zw(R5Ls)Jcj*MyUG_yZu;ujt=9d_^O`x~fYqF{LG}F(-fvruAu~2GQE(Wj+f|NmBf7 zCKp-Ju#U>=Cjz0QDC6T)(#34n^x_rm62T#U!&5E+--FBfttTmdugCu!eQ3a&aN$2V z!Hn-SQV<<j7Z#~L{mY;;KsMiD?M5#bo&DB&-rX!BL4Nhkv``7d5>=3C_1mUUahBtE z|8u^0a^leb$KH3&5O0HI$FX_uw)M_DCUD#VsXlW4gS~N^k2K?PPVjiD<s^Hq&ImbT zmYUCg4|BqsD#N@RYfx&pQ==7m`wI?owhrOydt+vpe_w!ZK!<tGOC#VNz1Fd8O_4qx zSxQu*@MxpCaFPA;XsP*`cJBdn5-qB~H_nmhhoL9*egDDqf&)3M>2Fcwex*+=#vr>J zzJXAM*klX&RWjy?Zr$6;XkR+tS~J9B*y1qK@FUy){~RgTLf)+ZDJS^P!Qy-g0z+4m z8-VUC;bhsH2c9!i8^%u)N-@j&IEBJuE5r=WEzZCp--ho!9uOeJBUHdR#XU3{_nn)C zCzXnda@f!)N!hyVW+d%9!xko7$HI$7B8C%n3Cjzl)R7m$Lx-r-s(7jp*N2WWxMaIZ zR?=ti<5ikvTRPVl5WmL7`3`J{Pz&@98)+(Th*x=BewV~&7Rnku^I}xZaGvi}Mwid9 z<bAlA7obNd!@qopCo;?s^KpF!F}li>3W3^8R-5oNMjL$yOz9!k=(d0+6Pf0;5nU_f zIEI(E#WBkufhjcNs;KbO0)jMpzrhnU>W>Z0)Jb~qb1pxCPY5h0n!8*-cgj3GT?`GB zIc4Hv9<~cZ?&QLdM_-%Ey(1ZNs6E_=9@d-u$&rhD>0-L+Fq@PZRJ%Tq7@jCU?S2vo z5>QJMvxkO&M^sogqJu0#sPN6)$7*Rof6PL6no-+jBXvGJ%T-$gTza*VV>xd+mo|5M zByP7a1x9L|^-P5Su@_yRX>mMFq(S#TV<jTJ1)aX!mRlaP0|Ly)-CP{bpMMr+Z_lZY z+J9WK><k%-QD?fkg#FGnUhj`5`R=wcNLyC;^h|;sSCDZWcg)gXzet2TL|@MBOJZ>I zYkyU_QDx8V;GG8Mlja@6phKQP`S^!gz@5lxr4<nEw_9Z7$y9SjqAdy&vV7k?fW!f{ zCqODt9mnxK?qa9vx!IRTqOb!CH{pcXwgkO-*<;_65HmoyUYon#i;^?BI?H9QDjcS5 z8oCMV&KZ0~cbIWs39+RhG0NBV`R7`VI)#N4RQhbL@K7@xX+DWg<0C0bXw}ItP@-s~ zjIMYG;R?iI*=X~US1Hz{{lGdto}6VqUa4B8PTa@D6Ls1^4>(X63w{i$`qEF8;JzZ7 z{rB0tgsqk<GS4>(9*SHr-}$?bN?55dEBp)MRJja?_h&)HFD_I0s*Hi(cE?`4sCEeT zk5-Bgha2GOOa+|dFdifSb4tJie)^~-866H20||&k?%Ie|KA|X3?IV|XC8O>59>+k> zOU?BPS^6{6xdF3dKs3qe9TBkThtIC_t3;FgU=^@eD1u|R8F(xARZ+;0*KMvHS;?g0 zug+AjTK&%ih^*&zbP>RdWa)OVFeiO)CBhfrYgBtu4`A+ZJyq_kS<RK)z`s2-n$t$y zDN<*+sG|#wS|x}WUXn`OpT<4(`;2>6x%ZriiE}CcpCA1yhJp6FM6_Xv^ripSclAa^ zrVg~_^c!&&q*)U(M<zqL+;MrHtXsn1v8bXx@4JSN5!`SS^yByB6`1fCQTJs+?|>d$ zm314nlw<YCO(PDxKV!MhD<K&$Z#`XF`=!lI(IUUW#3ys~qv&8FNh*Js%&x(Ny(K=N zzIA!b*uhvEtJ}CF%&8xG8@PA&=h)Tmq{r6DU*3?tgKNX!={<*myf_4XUd`v_w<E7b zAt^EYmhEkr@(-kI!+4cIbab<VS4xx&Hs}%}15ix731NpCKDEQh8@ObkAleBleZHg9 zZ{7nccvis$inN&N;{F?ZFBk+x7Xghp2Cb+2Ek>(DQzZ02tM7%~-qsMUJ-njc9W+oL zlxTBvjA-C97Tvt&yRf0x+|OT)@6Px$(`aNj)%rj}@EF4A#)_=Va1-qL%og~I?*bF0 zYbA3&4BG|jH}V<^mThagVqLw<r${B`aLjK_?RO>$-z%y5c;EIZAH7XR*=E+#XAZDy z_`EWw!HGEbH(0kx2rt4xT+>=|iIX_R=(Bjp0ZE4}xhSfk*w0zA7-!x%2hGZql^)k) zJovM*uBbb}%<+^EUC@HI?k0jHNkHcDS5N=@2)?HR7r+Haa31-rztK8U$@@`27&z)` zk2KPti7JEK=uP7J*`ppnc0mS$J7p!rc{`mA<&E}Y6~Cn-RCL-WD4=qB*3Cso$f3YR zo#O%!ge}I58#IJn$FSG0&}d=^$?+Kjpm}`RD9H|1TE%e4u5Qemd55|Er+$My!OHaZ zZ6>ocW676WsO}0Y%;S^Yq@8Pvq8ZNMSS88`B_0y8zgI>5-R^slnyDTa3TIv}cS0_- zgV`GJBm4g~b%0g@V_!);NkO$Plcg$g8g_Ph$_^7B2NMRSEpsa}+RVMY)XznK3a&#x zb@Za@_QamvFH%awh@@3*8Xri8UyS%EvONIf?J^ML#Zi<=7C)5dYOD^v)b_#ZD{(p| zg-H)=kL3~)>uMj)eC%9K@qhXV_XH5AGxh6Te_^FwW)VpaeSJa*Z3~1x7H^6U)1XwX z3qd%tcWqZ%1s)7sOIL4=LYG^TVy;V{F^>d|^wdxUG_aG7wKRF|(@}8hsutZ&+1TrN zSL)ZQdK4Y;X|<W|{K1c@$8dk-?c)+Yd$c`e&~f2;xU^V5LLbDa$iM!!4C^xKuy+;F z8TyJPH!`r%>+mDuo1xq7^-Qj;>SjG1nP7Jm1iUmSOl8^=LmP}ObYF=z#bt<vH+2~H zPh_*l>-gPWln}VULmgNo%n?S>zW}7=ZI=MLj-`L#KkOb0k%VUoPg5K0J;wF+HDo%= zDfyQ)PR~C8DD93Is?qXvIkuTPOUF!@x6b?z@H_j)N{b&oTs<(Wxh2XVRKV~@{pUus z$vmEEm@|Wrhp&)3g^1QlQrXBrtVenon<mj-eI;yy^T@WqNX_3BeSIGD;WP*fFLXbW z97ZCh^9y^e;45`HXWS(1P36mk_TGsMd$A9N_hT+yL_drC`RSkSsced+d4&8;v~N-U z%Mlz)=C>SVUFH+W(+dVAhZkzzs!W^gVA1+LA%MaeFN24!_;~~BIfn_&G_;Om_e~6K zHE@8h+1Z<Ne^RPFPv!@99M@5`IP+5N#uBf**LA5hD!kC^sl+SfivgdY>~}su9l?NJ zg$-q<D_;h`STflX;)tm$nqKHE0YBB!iQC_w$*<BUzZ{052DnnNUY%o@ZiQ)>K`j(m z7d88QebdO2Z?bvfa5Ouo@_ok=oJG9Y;1R;PUfDTa*GdTp|EmnBeh$}j6E#Z!thFE+ z{fP)slRL$;tFq>{I&%5dS}FbK>yzc%t5a!FV3mL{8cz^BA0QxSI)$4$4!jy8#@`JN zBT_mNhl9Nby6hC9;P3r9(!YV?08cLcfE{IPM4wDmaX3$i1EG-o%)1<0mja;zwXUQN zyel8@Fq8^@@b0(kvRAGIm(jjl5}#$mX~~u=a|1W`Tocif^^{DZeghtI`cCb%r=#oc zOMknQ0|e{SS2=^B;1-CeH<A61R*7th+w`X15kqEF66WQeiKWmr7G;wKdYIbqWO7V( z_3wwQL<{8hVXMK%rCbpO|4|^FB>6E~hbTXTdFy@ub*0-k<#PWUPm}s*UFw-OYWW$? zDxxJllDRFsKKL-F!4FPMsVc5KF<Yri#TqIzjeI93`5n3GVU(xc1iRV8r6n0NzJ6lk zGg+&Q_t9p)$3LF%`VQHn&gZM%|1HtIu_zcXV$y%>`Qm%5QPm86C5^F5cx}0G$U52I z=3BTFrR2tRpP+}{1B|uY-f3C=DY-Q10ky^A5#gW=YAwd03)>=Ji5C6&!z9#zZ22un zl1Yf9Cx(%JV>cW}asvwlB(*cHc&_!C&FN7T96+4O{$#!gpmYRDp$@RSdF;nO9aObO zaz-5q$>yHDo^LAy494p~X<by^uX}so(q@Ku?rGl>Q;McQPMkOHr-sztV8qQymOL+W zym8oM1g<jmoweYtCNsj!-Jz7)+8Z*_YVbYcs3Ll${O^zMV($5pG*&))LZ?%^7{`TK zR1CyRSBOJpnk4##5hbrGnv_D!>JY;S!~D5?&PRF<rSC<*lpjj<=_&)|HK>62HS*BU zTK>T7;81i_w%(nlVivIY&<u2Tg)j)ahTbT2%kxVnYG<AhIaNalrYIib20FEfgL8|O z-8{f>2_^u8GViDGa7+7UjnKfqI`FM4+Y2%(lfWqCPl!TE>**%5C2^gSwfw@88t8B5 z=QTiB=`bZoUEfx9)=Br{qoz7f7_kkEl3erV6V)451-&Kydq=pV1xH9gj>4b1FPmuu zD&~DJBn(tL+S&8)pETOBA;%vbzIHHHABVSHqy$e-cwN#<)T(9H29?<~H-MN9y$nz~ zUL_}0kq&5p_;P1z1Dff=%K2jd&Q#chLn=bhfW|iQa>AiesnN!8@q--4_wE0I48m<% z1-iW$QM9KTDZ_?XZG@ced&qVpEy&SGgL0Y}(*9pxO%f1_lCVNM&<7+1EKUs@Th{rq zhw0YtC0qnp>QH{C`+v=z+eGn90%)=1Pkz`86OqOG9Oy+Y0C<=$UZ=4?efl@!;nxin zvJa_v+nRvOmAU7^tBYId9W(&^&=g;#b-yuY%BRSM`}x3D1c+0fhqBCrZNOaEa-^77 ze6zy&+}I#otv#;LC%|V=5+Kr<FvWbb4ABb+NoVYGmIL^wO#3S8PKpx--50MO_EOEZ z+hPZ*34^sFEDG=Qvf{-22{w`5X9x^{+T!r1qQsvcJS6h@Y8k6q(y90y7k-cSlJ{}m zR2H6@@aiOeoy-@nkwH-XhO*P!`Ac^QK6Xw;kOyBzP4OgM%iMi#bu<3qBi)4B1#%FZ z=}U?1z#sJP-d86Uy4P3iFa40D_Nh(E@9{d`_*)X=OD*ce8c<<s6r^hZ^rA?T))GlZ zl$WWbzQlN{lJ1V=y^W^37iB;f<9paM<*Z_KY-R*7DW^qh)+!`DCmTeW8`7Y-Q+Q-g z8;D5rjk^k;7_}wUy;4AjK`&~Gy;>I@icJDYk|gRQs}PC?JzcFRgSxvpWO*wS=7_uY zbNJ&hBhCOq|8Y&J?EdC*bdW!C`=ts&#utNywrkDr4^A0c?_Rj~EsCu;Xz~~~`6O%Z zgWF&1$!oSW!O<eOUv^l#-asJyd3~f);f`i1?)4w>B`z?XN4=Q>)q-LzBm{!qxJyyc z`VC@@8rDY4mHS>R>%MeJAI}%_&2*imKUAQ+H#5j~VGAiUKYmSV)O<mh{es3JQR<!@ z#2?cw@$*_#E(!>SvwsVk60`rjUQzq$`g@T)KZ+3D;xf=UQYV)~=R1@tYy#-kCKCA` zs)&(m*+hep%h}sz_>8T0z-S&XHSAjV;Y_WCUFfxFO`7Fz)Sq`-^Obe(%H++XtpNev zzu<-CQLndVq#+;Ld#e{H0v-pcEG=OGbO)7TB~*ZfC%0gsf_;rUjc1Rn0Zf)g+ZFC| z@R^aT_H2dZ*8g%+r*ks*iqOgX+y$3$+}{QWl&20r&25fF6_!o&e0P)WRlH6&l&r^_ z8SUTQ1YymYr6^2sk>||}Ts*}>(BMPLB0^u!UpRngZRPe4Wh0wI*w14?Y^R9<xI17+ zqC)Z<*9(}~=RWR-ru%lHnOH#W8{#nHw%nu>w=bSUXG<Ng1aA}H5bbq$5>{R?%rLBL zu~{g4$~br5z63TgPkSA2)HJwyY`-njUwz&`8FM5LSk?@b$Pnj+cQV^@ZwGm8g^C^k z@y_va%WKI?N;K5}f{6LI9yT#cHP5LW;HRZ3M48_(k?}^C83?<Q6=|2U*LeKj<nYi9 zkfZ&|4*CfqIY<Nk4nHRyFL=|1_hF|>ztb-~pTSPqQf@43l8NxV_F|vi0~>*)*Lt<F znTe`LZd8Xbk)>^`p?_IkW%%6_20b5A2|uSO?G!cQdi`;NBia7XWUGXPD?GWzVdKr@ zumr6QLqA*$^Pa$!)Zj#K<7@<d1ULJ`H2-!#M>`=dd5^LcJin3~Gy%e=7IA3o<1TKs zjt4rCA%U7ffrX{t$Gv)ZE`b$~Zyyd*byWg~6X>lTJ~5*i)`%$_hR!Fb7)_SLl70A# z^^#B`4@!SW|Bbur&3w$22il6hkrl+ak3Em7p%s08p@dYM^E7Ckj{z_>+>G50U(Qw| zhY^R?(nG%1jGa2xyQ*X%>wkBm^BXdlAd)i*vdF)cn#gN1G52%c6Nr<h_=MNcZrY~Q zjGOPxUdPtXKR&QuO*j+<E)Yc4XBk^q3H^y39nI25*-lKPn0*2}omJh%m`6wXwxtxe za=J7as)`LGtYM~PVwMDrBI_u8W;1eN<{r1oQZe*}-K5iPo@22tH$73nF7fN6Y*ce% zVKhFoFUFK(AUB8BI&BM#tzh6<sdqXHJ!l}eq-%q`dh5SD;YBURg4deUIUIwM;aU1h ziykh6ZtRmgKfM@t#!3$)q~?AJ!-WXDV*hK@%5Op)uS(BW@<;-Wl0S_d>&}b<<CG-* z=QbMs?`D?{Gqs}tRT39;Vil7ZeBR?mgZqE8oX8e$sLV$qlk=0^1a8G%j>H{FO(J+U zGKIaVn@(!E`@75eGT#mnaHvr<P<lv&NqO8xSfEpotKW~aGbJdW{iVvZfKx~zizIU$ z2ct}1)wSx<Qg}~?-CH*mpnk*(>uoKRwU|j5g9?8^P^jsB!W7S$&!QsJd??+zefVG6 zhV#e`9s~S@%d|H{<051XJvkx=HVejsF9Y;CdW$5EI2=c;iabvIEwLmu5XoosFiWMC z*aPt&5?)S5A$avOPRrnd&6AtTWFmIr|E)j(N8ka6mOZz4U+J75PlR#}!<}*E&ay5i z;=-9-T-P!C(Q7H_U@Trlp)Ef_0kS1p<@Ru%NK)ITk;vQy>x(xsfidtBv#wx~;%oL1 zU)4MJzQheHLC&oggc3&gg|^<ZVu@9<juDhXK+7SDSYO0fO_u}*Dhoc6kcH*6e#1T4 z9Ols?`~@_GAx%8uJ2znWw*vM{O|(6+ki(*;DQ|OyXR2AMG@PM)RY0j)VSp-**LkoO zAf~O9qk4&Gu<%8G>D3KgEg*5BVt{^8oif{9*vmy?9zMbc!AdK5#1HQ31MKnGw}yKo z%{r3MnfO*~ctnZ>M%NosYW3bX4+>46;Dv(A{}#B1-!psvivlpWaTJ+t^wc{Gbf(3D z&21-fIH*3p>eLIB8Yvoq9U3v^Aon|jWv0AVFI2#6-2EGbWNXhfbk{dW+yinwC!(|{ zu?qJKo=24He5T=(=J9RsEwCKOz_LnK!)bJwuN}%z%&MQS+SixCnG_%s`Pu*H_I*1w za$P>v>iYXOof=Qi+MW1K#~g>ikgVJc@E>hPirz|)<t@{J!<NnF;PExuxR3$E#2SMM z+>(Iht}AAgJhkLhP0;KHE~L1mThRB3C02^BJ;<CcQSmD2R}_E2=h}>JCcS_yM7B@- z)Ej2C+si#JLSxq`J~HM&=;9uDGkl6os9T@vn|r!>U^Tu(Pi0RH6Gmg5d>XaMC#m7H z7fjk#hjOAWgbl4zat)ht-Omj%NAJ;DaWD(V%@ZuRBiWUTV8Zb`dkB{DPe{u;DBbt+ zz#~m%D`i-d9kb@985gJs?ea-JkfY|*1^wGudsb{cyNT?kqs%lK5Xw1+c75V~$_l?j zZoaCewyFq1`<QQLjo?sazjcnbZuu?KrRNA*HmbW1rr|p*b>bA*Fgf=4F#mvN5$-Kp zZ$JD5Q}x(nG)eYlW&iFZV@9O0?E62c?r+A5hhO^9JxrXM1*8C6XSJ10QQK;T?lav) z-Y)Xmtr1P1xEuyUeFu~HWNtL>q=W4e%cL(&zrg>M0BQcxdXmG+ae`+kt(h$&%;u-K z3LbV|W|9`N02<-)xKa*2b2S857>g#<-J}_mmtWeOjC#8a1@9{$-K!xiAZ_5>Id(R= zFB{2uL%yvaXa*md`a=2?&VafZLd0BGzFu$eI$Ro}?FG*3j=k9`la;Pd@Bi``G(4}S zKNh)V*qko+<D{Co<|`Z3TxL-F7h>RJ5qo9s25wgE1Kt&@Y4?MlK2}aU-@|~)WqcPM zp{JQDR>~<43e)|?+%a$^4sju?H)p4<^A?(w&C@d#tH$Hs3kEQq7s>}LXPv|=j~y|g zyM}kcf`$6F%NshQaQUY+%^RZemF5|~K#uu7pH!*3^_YCm|8+L>!wlSfEeLlC-b5KF z-8Um{f*0%nl^~CZ=m}SX6XVO%eH8>za!m9D2<{Z?ww8kJq0)M&cpdJku$MkbgRHHN zhG)%6d)K30K&;0BZh-nqBAwDy!Z6Qo&#VpWX?}o;-mi0L%s9oJ`|0Zv=zy->uluXg zTSg-GcDLE+s6uI>3f_5j(og9YNQbt<9GOQA0|jF}Uhb`dt4kD;It-$}AzjUo8)#1v z*tq=W2t!UamqL;|NeV)fw|;7(CZ+IS7J!$dCA@4Kvy?*m)ywHui;1)Ob|Jw14Shy# zuP^>^>E0Q<Cyx4QWwqjfm@L)_znukTXRUC7Q+6;b*#j#85_>&0vf{M%0gZ3nLC8~b zsYO;aa%VCnU82$_6X>4o;OS=RHwTytR&5&aOz|_FD7vSZ=N?)uSFUCIA;X@(9)Q-A zfodc8Ev(hM@)H8;^dBjVNgEu|BAOpg;vuC7%Tys?+78wP>>O71Y~VZ1H=?fRIL{m? zB}Z`=8~DIw=6{Rb`g`ahPQX48m&OFRNHT>Ozkt5138EP2rHu;)r=W&p_fsaFu=dqq zypZyl7oT9?DmZNMTJl!qQ)aS?W_8QBQT=;$X#$G9VvvCbvv%;LU5d*g>bA}oF0_pW zU(z@ag1B^U2u+Gupt}vH8NJ-^Mrj4IuN3MW=e&sE35-2GR$3D_OjF_}BM1Lm5lZR} zN=2x;@B}>=o~$^mn?;~fij-C+`xN^tF>G&vc!i?@rIZHf)75KiXffzDe_S`AenPvb z;Rs#TG%-Uq=yM7qr{T&SDftg=YwL&H{XPIRhDpw<NGidjj$r|pTweod>est}wAjHE zMfw!!+&{4X<tP|bhbhFHvFy@;Xp6-Ct()YFdMrE=@45F4?S!hpf-XMl5;*jaR-$+s zHew+#3G@DR+J`v`M91X4;WI8~GDWT5I_r$mK9=g$eVB>{pZQqv=JbCTQCwI4{-tn( zH7fZrw(GJRkwo4-RX2G+1S5E-sxpRq5j_5}Z!#;M&h%KdOsbjLVRFz!0+5YxfZ<6c z-JJ_8QKWWoYiex@P%;x2Tt+12O$!o3Hk-r&VLx?i{T$g}1dysby#{x-v%Oi3fhzn( zsC(?a)u%c+K`XDVk@ldCt1gPTLUaf&QE(06TA&2SPy5+Q)nEF;czG@9C)zZe(z{>& zHd4S}-t6-nSw7)=7G6FHENpD<aFE9(ZjiC{e~1>f9RYe?J$SB>JcG#j;kk_(%gEcI zlaUuQ#F9NTomY|3#N#yxC)7;wes*9Esa2Z2X|7V)VUjGc?n1eL#P#|IPnZDDr#1t6 zl<rh45lXD$=B1*w2z&M){Qc+|@zIJmKyY`q?<V=$`Sk#XvwQDy#E0XD031H{vTUH| zZr^M=Q=|0ay&I?hZ@(I9lj|)T=aZ@Ji27*6FyJ(wB+YT%rEJXC*d@9J5d=_aSn{&d zd8LjsvYT^~tLJt{N&Rc>UbcY7?>1u&d%?+oB4`dRgPt05Ky!3{arIY~e)JU_7A2^5 zZnjq?KhzZnfF!>=7Wd`7KIZq!e^TQORK;5Yx<G1m`u)~jBnx!DPwg3ig;o)+d$|vG zDWl(<?KVPM@cx*x5@TUrHBa>kn;DO_9rqk-1~`*j>+Zfl@_fs(L)~;Df_zT1HCN>x zT;|T#I)r>@)A$(FNc6lLZdT~KeBr#mBff6q*WAT1EQ9!hmi8TReU~0vuyOo!Oe)sv zTBh21G?aa8PauNGPsd}VK<#HH5bI>lIHd-siJ)G5E`({m*9YQA##^DwapHzlo3wDq zAirhI7mSONg?0TOD+%m10hEUJ@Ba9#os=aaqsP<|@kT@MVBSa2t{BF7AKG4Ykej^8 zW@Tx)7uxZq+x4&C;6!lmyT3nPqzUA|N%tP4HG0mMG3KvZnrPSY$L#4N`@TdTNQptC z;Mp|{2NK0?mDQu);Qptc$ZGD)hRlWa)NRtSCeyu>I<;;aNGTC@O!HL?gWQ+Yl0|0( zbXnP<jTn9W*1FVyaL=6QqX{qv5|fLk$5-gz$RXlDZJfBLJjuB6ky@&0g4G#7A)7<u zRf-gzbPtc3=yIX}{^XXV=7c$UL|^PJJW#S0#8umgYp#L)DTmAy3v8A(5ss=Ok=>$h z>Zi*m1Oac)q>JWX`%Q;RP)$7mw<@P?6hyW_R)_N7RBnQx3ZvDf$Da1{2b)bHEfWqQ z4rsv^dEY~w+tCj$w?97n&bQ>R8hZWB_Yi>XB((RWzk!_GjTfdE?JD}I_*?zRsG&bi z=?Yw#dBDAP>}7;j`Ob<Rm&g-tRdsKPOLY25-IxX;gVLr3<vDYAq}L5?CCUs)irRxT zf&J7Q<U%Z@y@F@BV|L)paw4HiDufPrTy6Q;gd4r5W>d#Lipo)S)exJ)DA59$F(I{y z9~lbIVSjGc>-m)NRc8Mc8qkCXifZV^!;QW!Ct9Sx(Q$dN2?coRXF3D@CjCpATnp7= zfzLqoce`10ObH4jC<qh_H0$;l`~SO@_)4C=3t~(i35%r-G|y&W3(ixNWG6UI#`K$E zW97}|YU;tm!r>$1)=;V?$P^^ytonl4JIKWIKmBWW<=c4lq$7_FZRqXMjz3dBd(KQ8 zQolR^oTHzZ4&)*5?ihi4>!O3omz8M(<qYJgZ;&Wfvv%$Cxr>trwpB5vqIN(jsJ-hu z>_e+1+{G6m;!Y2-O75FBe3F&&kYWi>gEM@b@iWZAK_O8zS}ZKy-2YVufnLv4Ce%}a zM?>jQFIhAnI|aoyBMl@zC}KQ*PZIEFaO2y&bl^A6_rP`or^5GP!t@+L@Ah86AG*cZ z`g1rGSh`y$`L&h2eNsUO!-N_}9)NF1<tC;dN?Jk0u)Htxo6Rxx_p7%RWuB!EEa>7B zrE?6S)um-3g9ilf2<6!b8={R2-+f_f*}Ufpj+N{GOkoax^j^dpn6E@`jsC1v8XJo3 zZ<u|*84LPU=Y-(aQrdf2AYfidhr?O_m%frO>w`@&ec*z%EMdW=E#tOdJI9xl8P|&q z;R3Z9TtBa~>PjQEnnuYWOp{>^y@iRvfB6TEeC6l@!pWPj@k+aGikwRqx9*@kp%f6p zA6;7MfbX)-9uJr%D2kjvU#Y(4!HKfvVmFFzm9oCk>o`eHcc{G`^aGB=hl=T(;af@@ z4rG3u8jmg;WP{5x3_;KkT{bakDd98^ndh5DB{lb8V1PygBkrogtl=Vx8z_nrN(TF6 zB(?Q6?Xwc}Rq8S!ys?Q6MEdHtdWuUau&ILD*^fBwV0uO^OTL&rF96d{?w(7mn&GY3 zwLMMs+g{$NHvlfiHlu7OV2BT`vz;XPaK7_o!FTa$mxMw0TzJsmEpPWWj9Q7|T?<%y z*nrLd*V%vtavfumd5T4?Ez8<q#Vp)ORHKnHTIuXhp21;z3@xOiMWu=IivZxqOqKcI zJ$$N61QZfEPpQ$i4F>ir7~SIf+g`-gl4)(&ofQ-M^sjkM>EqU!;ff?yV3s?r+&(1V znJmUb1l%`QFMQut@1Up0l{upsP4fsWOfKqzaD9)J+mL4<v>HnNRM2{ex{Iq+ypEWz zSR$h3$KRKen|?XWN>Uo)J_J<1BeI4g;Qws!v0BT2wQ7c9BqBoF`1#<puM*Qc(Y0|^ z%J3{(p4#H~Xu5K*&CRjf--bHB^k;oCN;4kSyx*&<qazAr@$FbfuN8ikx##MXJ|DOp zwS6o9?8@&Cegyr^qseIVz$4y{{L1fwyQ?EMaB$L?ZAmeV?#vc5Y=@FOc&H$Bu~iZL z+g*a;alXgOlV@D-s{;qNe|7#1t?TR;zFd6w@_b3B+~yDN@Gxn?h^Y1H>xjf>=^A1e z%Ud609{mVkCFXHx?~3*PTm~@t?Es-*ym@TkOu{WWHTZ5hevd~g82f>=#i&0xO$BvT zVOT$Rv|DGzVlSEG1E4I@^jkFD0aXg0t;LG<->>38sy>WtgtRh-F%HzWZ@fzZUAh<+ zE@i$NDNyblx%<(&7w8YqqNd^vICU$qKJ4c9Yx4x6zKolm)G=Nq3V4!JvsS9D6+m-! zlq0+v0D<fQ>!oRq5!AQZBi9!S;|veDLQV$7@g43erAR-GrDqAn>cW1MjWfGub(G15 zzlIlTY1aDa_2aapTj1jhvos8;HT80_;U7EZAL}`PCcaS@Z8U%Fa566$H*qP_Cd!B} zcwXgtpmXEU?lNCHeyy~jtapUCSizt7xq)|QWE@ohX^|duE3w4XIR^dV7T#i0)1l(& zd{1CU!xIg0xb9_#Atdf_^F|$XHEV_Otcgx2fTG&&>$7K301;KLoZKYM3sl2cQWf63 z4@_s)QO(qvWU?j9kjLS)j4uZD<e^`mW;G8rX}#egIY>B0&(5fpbyJ}3!uxoBD@wop zZncq<e$+8gIvFXjEui*?h^F^a^Ttq<WdC&DsstG@YvuH!_tq<e0JwNLB5)G${`Ku2 zeF(fa`RcFDi*ZZ}m5yMy&=(Ip0K)jm53?N)f3$rU94YRT0J5z=9^;IZ>)-0(;R|4t z`P|TAe>zWZ&Z8m}JYp?H<Uzb~Oc?QFCwJusw<iYXl$TA??kukg?>%;fk~x}8nGn-8 zA@Ogd?9(dD$V2@0wz`6p2GwDAiOXg%pq;LK2Xo<;E~jXQ`vlgL%8oy1L}fxJRR2}( zUB|8U!%M-))w30_Wz<lE-M5SwarnWvLMR^M)6SXl|LgL5qr{F>F#66^O9@mTpNIv# ze4o!imYO@>SVWo`xyTA7P0*{oCUkZk*2!XT(!~4s$k$I$p`^RSl1Px!pYSr$7sZ$- zISefB<?d_&OXz4XemnO~6Y&6Lz51cK%%>^&3WUmd%hgjCvS4vi-w_Kuu{}X_TIW)X zdh<hDM6;1QQq_tW3VFet!BDx1UJ}H3i+pZno>EH3Qu9*aH)YYqbVh!2eDk6oe}$35 z!{SB(=oi38yjMx$S{u(t%;b!oCFi_;z5XR&1RJ$J0JKpTzxah=hAUg-Z%W%VR#C`> zpOp%vL1|FxJWmpblLel3ST%7G*^^0?07aL<LQ4^vG)y(toLzrXo?$}-o;_MW?`8O{ zBxj>P6{tVF_OZi<K<Z$jo*8(c#PONEieM6NIQ{+FrbTbr(;2<vVd4Vx)_?hCL`B&2 zA)CH$AeLTG=ae9l-7eOeh<|1X=!5CCOpaBWdcSl@A>yvVmMa*MsnF7F2>RC;)B$3f zJP)}H!|$VPW<AZyIO03cc<nH`fAa)zRWA7p9z`eq_B9BU<jHMo$!$xaZ|lAkkN)O# zkr7%!+%fIDF$^SYItyiCa&3{Q7JLcIz~``6a;kSHLe%X;>+_)oI>mB~-nABQBF<lP z{&w3W(MMg0G2E{`v(Kob9r}=JXA3Llpy1M`Am|>#0<Kpd!4fD*5wO*t6iT{SJDg8n zpTx{)jLb+)PH1%|N&|&iAN8M{wpGy*idaNKB^$iVS1kcT=4hj@-P!t5YWhVMFo;m@ zzO9bBn0LxD|MPXsik`%dgh#s&dDaee6q+0%+kdT7ai4Q`eWoNNITT$JyHw1z%6b=@ z;JkdOkmWb1t*_v$<f2g4JV`;I!@Yz0tb?`y)I$fqSAbmN`;zKNokW7NM3;t+*oo|d zKdKX0GE&Ie&ikn+6CF7T)mz{^0&FaiqOk%JajOFcoiLry551*H*wo+dFm$+YNmgx% zL#nLSHh0*k;IynnwEr_}Hm{P&V=JaZ$(=@tRp;$897)N+U||a1r<O`(`4}Dew+az` zG`zMikTpiG#$IiB*QdeVT4jm2I~D9g3$rr#9D<-F{^vVdHq)>l{11o>1sR|~1&@cY z$XVeO;}F_a`P#37?n%}L{W<j2AdSheF~TXM4#%0N@ascXCG5hgoUmP;Ul<*|?7W;D zm+{;$GVNEXWhVE0FWr*n9X`m`{M&YWK+X!>*fzB4fT^&#ft8d4mf72xd_^4_&Qvj< zhrhah(`Drh_I$RFIS-EY;+(mLT@2yq@3P&j7UNH5s6PHe#`oo6*vH78%s7P}V^XsS zefmU~Rto<5HEsT1Hn`+w6i+N!6`$3SKKP=?lGW!fwRK-|S#7vm;<bNs=-#l0I6ZS8 z@(J)gtZDV#uerQwX&p8^x(?Sc86QgNOkk`HcDQrmSd|iUZJt|e%yUK@`I=1ZgN1AI zLk!oghzsy+yUcJ)6<h<B-Qpr205#t(c9NjkYLzpbI0MpO8+A>e*|_I!)9f~GghNVV zZ%q2EWHe({TKzHBen^}NSxH`eN*_R`14WmKeeecr0eei%x3h7>FFO&Hnxx#he(a$M z1Q^+G*)H;>y$ea1=TtzPcMJp|Y7D2Z`C;Z2UHtstEZ$y?RGPc>&zLca5whAsKICo< z&&O^DyhR90Qt!B_-{|6(3*HQG8bn^O`Kw?c))xpFJ12f85`d|<1DIFk(LMauScX|g z{<b|nEtcIV@HKIQp6`B<MCe&YZ(R~f?eTB=pW@Tpmd)Sidd(#+l0@@_)L%4jgAH0v z&{_n*20&BRGgXa{xQaVfYmI>dh97^EWPb7qS`T%45LEPjKYocT{i;zhuzc{*KH)bv zd|IFuQ@YpQhB3pR52F0i?s*e`_dSIr!e_30`3JAdAlpd6_Ad}|QO2Bo$xmy9@_Fu~ z|HIT<hBf(zedD7$6ci~328fi@=mBDr3J6Gdmo#iJKtLrGq!Cd8k?!u4?ie*X$3|{2 zVE^m)-2dk}?l-*HORnQMzvubM%C{TGe-Gn5DdXXG{eJ_OfC>$eR!SzIfZtAhbAjt( zGfNboB4Rt35P9&G7Cw!%*7MyGrjwmXBk!(0#bH{ZyA<~j(OF~|f8CZhl$!N0#52as zm0`a#Wo-o~@enn!SqcBt*{HvuM_H#i17eTvM-Cz(#i6Z_mBrH;pva@cr^<f1W}kz2 z4NG-ewH9%@@d;rCysP>^Au6px^m&f*=F>P6TJlfuPZVW3C{HNyz;0=#r>f`dv7xV3 zs}q!wS1b%CpKHS20BbWcEvClH<S^=NtT1z*okv3V$R0U4CAp<ntIET6QCDmU@YL={ zwoOyrN>fDV)C37cV>IMzU2387TOPsq8@;rJ*0dAk2re?#mxekB4pWE}=P#z#`!1VI z-aUyu=h_|3&oj0dc-R=g1n5YC&HNKWRpgk4p$r^lg|=AE*U2M&90QC(b8MmWOewFz z`PQ$IavK0O$j=`1Wxi5GSB{$lsC1jl9Ie;>?2Vw=8hOcCYqHg_^+tuqtUZ^@ekn5h zvyb;$g&bQNHlEDn9{J>)!zA!;D1|~JU&~+dG>gL2QNMB(gOWo7mI8K*q|zAb0F&DN z5r?a!C_Vfw?=xV4TI6ZUb4tv4Z#T=&E3Ro8xK!;Z(<VuUglBtrP7gJ-55(F2T@9MG zNp2JC;nF!s8#Z2rtPCR!+?BDZG{dd&EX5V)Sxm6KK3m(+X3>CS+O6^v(N{i}-1Ce2 zEg7_%-(i|GJAy0iwf~8QW(rNz>}Tq)RnDB+l^P~!s9AkRUu{<kq9%iz3n7y~pVIl^ zmm}4dD8{;a!Zw(jY)K{5sXVdthhDcpL(k`ILfuW)WgiSxfC<H%<wV$^drSflh|10p zSl&ZfjR)**c?`<?!NbbRpwuZj7WK6qre=|ld1iddx#nf;tfzgDeP6X>z-b2s_*W*L zdwvLBuNzgs_al0vj6w3NSc3V3N)zPKLgE=V@~)<S<Z1*}M5|-b^k2Wh%D6A9;S^F2 z=!jRN>EQMsss-Y)X(Gm$mJ|m{E}OdMj`KE1Llw!P$2=8Rz*hE@_qDlOUOiRV{w#f} zptB%vd|gdPRGg-ebPYU@8}&min5@OVPKTm15B+b;4bzCd8C}1xg2yr(c^cO8!AyNc zE)G&EqU$r@#Bg`KW@MtJASn`leq0th_3r6&(<}Lz^;Q!#(StgVy|SKm0M?9gx%(kt zb~QMZVcMDWLi<Kt@|Rh2|M0aAz8Q2$@TFiz@YUI0hH0(l8*k05PG}pbuKbzl8!SIg zW~E6v<=VcTqAhB9ZB@J6oNlp5sb_{Yrzaj>%H{}lL0+&);I2;pT3NKcThi^9bLmTq z^^W6|W&~oLO<an9d`Gr1)KWGbWC<YHJJ-XhK<EZcbB@xIM-TUUbTi*;cOeb!N}({^ zDREc%Ps8h`_osV#d5Uz?TtRg=UM;xdd`1Xb{Gk2px{~GQlis^=AAkbaAvs$CQOk6F zsN$_DgWWu#O^nhZ#5iz~rwaLBj>bd)CW*-yb<|$X#1Il!wtQ57EJ%VzN?Ej%r_8Q- zN~ubnf4FH?cKgcL!6}m|7<8;w&mrjy+RO~R&E-Au`u<j?8Pbj(FL%DsUc%O%aQBKP zZb2L=Bjkdt3~?d`-BmX;zOiaw+~4$t2D$-XUcWzb(VoShm@u+}RHM@5MMMWOyTnDp z8?b3==6fTrZ=4>54d6rAi<|D~by~4+-zeWKZ9_ME@*FpPEu5x;c8l|Y41=IL<3f*t zp{Z!aKF`l<?OeVFnsX)#qoPObK94OdZKrz@8}iXlO<+9=(fg(xXQ}jt#NJVfa^)c@ zbYqrBmA5^yOf5X}b)aitbWlFl^%W5)eICvR=Ajr|OV}(u$gH!m0DYB*H@yaOd<YbW z3V%FkNj%kA4RX0H*Ou*8dDnJvdLyl?hXEbOPME2Cz8Zm*xQf=NZp=(8YxtNVOkk!% z{1X0rnyYHF#oA?4Iys>6<N=C2zHI2HiUz5d2DYdG%$Fj2PhJ$mQq4v?=?Hm6JuBT& zK?6Dc=c?|52X$8$_Cp43S}b|ia!xPttJnRGv&K<<yF`Wx+g-3vU4&U+)A9Fmm!`ja zNM0+$O1Hf8ugCo#&nsooa$l659jzCsQ^qIPTr-63wMFt`pLi;G%Se)u1j$xTn+B@X zxGtD)X8J||C9$I5%)YnQ6mPFnRH{aaa2-n9&p0N(XM&saF<cxAM8>?(7NoR&<&ec> zL;vByvJ@CQumKf#<Usp@jm)$^s1wxsl%z19S%yl#%&Zyf?VRg4CPH!5CHLmyX%f@6 z^dhH(ndFkzKhpl)sL4B3V98CImE#zA;}SVI5`jqrieiO(L~mXtuCk|v)O<V}uGm~@ zW9Q{aL#;9^uBF|#upZph5ucnxt=?*QEqX|`RLc(+{cYkh6cNoXis{QJY~s&5l)6;z zd{wS7u+U3uO;Jv27*g2!Fkg8T^nNESTax-hWivj~sH{=g=(|(=nd-l6wB>?;PlC(u zWriNcL(+IFKt;Dv9Us&mojw!!bJ>b5T56?<3YGJmzs-|L12GJJWan}h4uqRAO`G^; z-hg%onxv*K8q09+gT0k5c29ejTQDMusUz#=eY@05m2;j{GWZ$QAInKGeZIBG9!`(F zrSlC%Kc_V(cbvEHyO3+?4*1L`IF{2$2%$z)l^&urVlK!4g{aJ5wD~d)UHh=TLl737 zrx8^UzxJI-s2GJO1?-&{47sn2@+&8cm^=ir`->up$t_EC9m7q74+L$UoAE1Af~GZX zPOz(5<LyF=%{DSJjGh~(ypND`gNq)e6T3cagFi0&y^y4V{a8W*>L<%X3EsP?U_jm2 zans0o*@kY>I?NytSckt*5Xu~gp)MSi;<kDVmQ{nOljnTN;z_H^txwH1q3$%5PUFS8 zXd_IE%OG`38I?rvif@oYv4tTxe%k9%Bn?m1$s{FsZY3+}Y*xxOjaw{`Vh|_erzuyj zY)`{*4wNWt4!O5HS;ASKwq5${no>&mXYx{I^{W4vp*l|&WIPN!*o^n7s^zI*?^A`9 zMN{&mpYkXK-c3S};y_!YHFi?W6Xb~pg;Vd06R8H7?>+4QBI2DVBtr5O89uAlz_nO} zRLhmgk4dJ1=K6aWmsll;mw9$*as|j$ZjV*p(S+g-=-eL<uSU0mr$VS6%)56m+<y^g zm)$k}NOg~0{y0CLinqCd(v%`acCs=FwUomobml?RDCu)uA!yD3f83<?aA(1}KT9R= zu3X3E<?}yQp|j>aO2m84wa*d`cmMMa@yKwIdb;AGd`32h^YjK{R^yZwX(fo&M6r== z)X;r1DH{Z&>#=xB7#11ic-*jxnlO9o7`v|3gjS!aABRkC&^jvkpR(dtUFBZ5Md6T6 z)HzGCE1G!wu;Evto}4`zeq85VhKh6*g;u|WArV*7ri|%sRsJvUHmZZfAJ?YP7_<kE zx!Hq1c?)OMfQDvIDrv`Yb8bqv#DCMaUmf;~S^%G&30)tcwBxaynXEFdA^i~i@bT`U z{IFISXIw)!l5r=53h#_Nf=srK3_}U@K1*<1`|01Yk7nG{+>BekFV2Q*8DY3~;tzmr z|J#&@Ba&6h)j9_E0+DoA`bL4W3YsTaZ|q93UB|qvryi$fT`u?nmIawKO^DR&_0H%q zt^5XyMK)V*DdUf_BJ?UFhu;CEdeo2^^RPpw!yk`eAz;!l?_$L~sIkYYvftX`t9Uky z(sH-X50{$L9J$fALRyid*vkwE&ZgUq6tTX^cGs1|l|23+@b-kdse$yq4pxHPf7CM_ zrT{->H_&<r%Q!!ytx-8Ib_2P?zqjs=f~?U;scBcu!eqV8zZ#<V$9dMSTVL#&C7a(w zdrJ;d-(jOL?^agl;`=NR$E_9uDg*A7R3wzjT~%4=u!c2W)Ej6??w&Z%s%rUjD05Sc z+442IXCPA-1ewG*vG536F6PCWyaqiyz0c>$N^#ZGnw)Qi`Jy;g$JNt)c?UEzQK$1{ zfIr02k}69ndtC?Rk`K3^v-kar=~b1syjTM5r?*;NpjdEj_~7OwGYX)aNwURWJAr2{ zUcJTmnQ5`91n5KR!RK%-)&8`I#IRPsI8j!Y(3rY?sP~!j0u<MK^)IMvr>YVS)}N!! z)qvml{#}Obd)6VtfFH31JNg_<p~b2&yZ^VKI22u7AV<--m&Y5_O1Sw(^Bre8h0%Dx z=o-+0<V7*_jb!cbc%a9nSmtN8J^Q*=+MHIW5<q*Ra7e9ig<pN`(KSIgi_iPNS^&cf zad)6RBmzKxl{$I_6Q5gKCF@g2g8We|;H1HmkXg$gI*!Vr@5EBAb&$!c?GmVSkluCE zBa&g~W@b8V%S-_tvsZ>YTJ-9vv_bEMW%1m!A!_)U9!!~wYl|DZ>(L=4IPy6APnp_3 zQm3&oyM4#zuwXh0UaRb2mP!sf#7}hDB_p~*AmViv?DEf6Ela}L_nyxa9ir7vhdNx< zF%)LfX;W0n_DJ8oO8RjzA14StHvQ&N875nQXPH3+pRy-rqf)$0b?5O(FV60NUX%aM zlhTU^+V>E~ogF!U)M-1N`ja+)$x%6Kum@_Kq}+aR>X5v@%tS!|_40k3p?b1nsBTrx z-v>M>RF2b|yk@EJm;JBS{yrz=F>ATJ4?A3=!cG~vwE+h9SH!LM0jJFjPex$zyggZN z;ebt~GheN~YcFw7w}}e-glQ{CFb#=Yy4d!7Ha=qo(XCu1`nb>+$vHD<?7E>Scn;5h zo2C@GVzHd&or4`ksYpL=>W`Lun?<<An~Q|yJPaV_vnq^2aWR&RhT)N6Xbz!I=$H6G zsbTkMsF?!WcmGXUKVlVo>CpT+7TxHEiV|%`02xL}S3T;J1tq56o?`nk8!AmwyDF>C zVk};gA!4Q$n3+QzCz@%zoN5ATEW7%@aYvb&m0Q!$%bv>l?dC|w=x|9uR(vD8dJ7p$ zSC1m~Ke8YHClgGO_2h$Hoo!-Irp-Dcmxe#QZJEF6y6p+4xL5?@65qUa|9+rx`VUe9 zlZgEpBjdV8<`iVWcI7=>`xS2Q32?+DynB=kN#9AUD|H~Cf8O|zi`xN9RsroJ{jLv7 z*d^Vn#EgLsHUX=1z+=jN{c9%f4!vJnQF2m6{i5gULe4;4ZUKPfyYGM7><EmpO<KOj zR`}Tapv=&1h^Vrl74%^m>QA<qene78=peQYH&lm68A+Btp4UzO;11s~-?LzQ6P3~w zN`v^i8{Z!(X+J`asukzK?FT%`cY62MmVy4OoPE&WF6snP4-ME9Y+MHGKX7scBpOkr z@*tvxC{3t&aJ#<G*pUzt8(yqf7w@~s2e)6R+x$TT-x|2_^>g-W&G@(AlbHUJe}z+! zN19>ff9`yKUeb6-%jJg(88WbJAd#r}E_n<zU7qAl{!nSQEjqB^mRTLqF<%b->s*~% z034(vxWjStgAJvHgZ08X_tBCeNRkTT@T;0g$g^}k+cfDeih*~qO@W%$0%B_6RA6si zWJj3(LSFL};dn`fsAT-+fMee?2tJej$osR|TLMGy6sxP>Ueagv?3x81PUE1nb;mhs zB_{mz<T^Z+j%VWGGPLj;|KOFEeK7E;dS17L9*II*6!dGmb)Gf;2~FZ<tezYyvX~h$ zLJdkS3EoJ6tju#sI|`Y<b)K&3Sma(VjfxAUi~+LvrWL7)^+#o&?}#>HNe7JFRHRaj zqP8oUe$z&9`3tKDR=tlq0g@|{5CM%twcF149;t7i_)UC{reDXc7AszRC%LE_{=!3n zC~=-Y=4oC`N8<3I2-w;d4vVbJ;G`mQXBR)zH;a9?PL6KZ)2F#rz1-ISte0|cIl1K` z=N(1pSTL9WwMxUy%}Md}^hM%%d(UAL!J>i7QsFpfZZxp1ZS%s5=K~#d!y}22d-<|! zT2ztv#_u#FZxZ^eBm+o{%#lKcV@w5GmfGvLu(S_*nX>$x$5`AKUXxAA;S|@Ew}Mu# zCdCosG1UV-Z?urX*p_i*wqpOA!?T>>yRSO_#MUJ!g+;{`E3i_NSU1}?EbhL!6eNR1 z!^Lc-rYH@dyLDHLoJp(GPoHP%L!)zZTip6V<!<+kBXAB(Pd!_2nPmZn=&p2?#z;Q* zZL078m`M6Y1aGeV^m!vrWyPJ(%tMuC$<sV>m1ixoaoLllYEh^=7Zn<5<-7@xBWe9C z6z5%J4<wOfkhbOyHB6l9Yz94qWg>}4a&4W}XV*OWrS{o!%<T6bkRmirGH<X$HN$Ik zm>N+Y{9aAJ73s_>R4=J^wNX|qe$IgvH|q8Janp^Yiy-l|H>`gIDo#Yc>wT-cnQgsP z*k&L1!N;l<nb!NkI3VkAOAC}Qs?bO1@TQ(zcphnhO~=s@Uh?>VEI6!g$<oWwx#YHZ zdC6_Q*yx`l`t%@d7wGZqs96f41NIV=ZE(=*&qQp$HGUs%kKCf0@sGTzoi`e!=QDES zt8Chw20Ut;WwN8^;Y3zW#Fyq&k~MmsNcEm?+orIO+i6b4A!1hDw;ICDwp=PlpN&*H z=-2};<?Y{tQq#s27d-Ue5ucgjP|2i#QufWR!+fpBHrZhqquig1k%7M7R{Lsi%?$;3 z+A@;RJ|U|mS4R2%;9D<4wWvMM+MUY_Dc{z4J-80tVE!vg{|w9FC>3aN>jfX7J4AEo zG(VO#zJxknk}@|DeSRLZ!NPU%T|AGL{`A1CQ1ER-;z7HQ4Kew?U)q@4n38RC3hZ_d zl(losI|o51OD)ln6xu<B{ygX22}E+yhnt^Su>5&Ga525of@$o5Vr#FlwAf_9rQ+Eu z;H&W0Sh3Sg=hkXfkbT`oXLNdoJg*hs9By=ooG1OZrUzR7a2Gh5{vQf=LCZ?Rlh@kt zTQV3lO>zC-=u3epZx5d2@@r=Si0u%|r9;20!d20hmZA%UV6$dw)E~`Ml6W2nnE55y zv&Y99Ad$$;EfvROigfNqg%FNEEkagdWtGqu@$@9RA71YQ^l}V`R47o$`Ihs#&P{Dj zm`JuL4U_BP<ufb*pd~l^c4BUhQeFgk6je=yR!y4%!7py6=f$mwio<qXzk}w3n~jcz zVNxz469jh}^>hfs6=8|k?pRPniN9<|3UjDr#_YDm3fDv~RB!2yb&|8$H}|*c1=yQ~ zMv;S^n5vH=XJ{nF+T7j|bQdZ9{42j+EnSsQ64EHLEowC6%t+X)i1SYSwNEWqwO1{r zZg$Zuw#tRO1PsM(z}rZj^3M+_hIHI$3?p1|_{dn>)ox$~Q3bPRNiP3^Z6p%tzLdVp zzGLQQ;Rwf|P?wZ)Rb;o8En8P0{Ngqy{ro`E&wND*Md(`UprFcP0GB`t(in<W>D)~h zWp`X}U5?nN9P~Bt^1f^Gy^P^dB?dD*oQt!HjHF9x-u(3{;QJ^YqwIz@KH%hRJ-~%S zL4x?y@8m>(IVR6EEcy0#@~q%<|8T|h8Yi0IAZ)Co_7~gAXho=J=>z;NG%xYUu%NQ) zi`Upjm9USK*fb?-24o$o`CCLxnQ3<Kvl2|mYVI3>iqZTf5qP%l_YQ5Af5^jczNt3Q z1X^O)tg9U|j^-olG%NW_Ra(=lX1Ru%iS2L1bTTHopBXXbyX3C}fHi$bBvagHX`{W` z8y7*?mk9I1LVcw7LV_wP!s?-fSzpAp$Bph2h{LQ+IFw0bW|c{k@UD94y3hU;jhxHW zHDEJR{4;0oonne1LjZIn6Oby3woJLa67nNX;>?jBq|K)Y-)7_TzXWv<slx(^`9C!V z>J=PTeve%}X}CIF8g(D8t7+pTrmj@7wT%VGa_M_4JwLK-FOj=^qgY5`fm~o*Xle)l zxvnF$W~Bvn2CC-1zr?8{TIlN=+MA#4N{9Xv)f7>ryMxP+FB)a6$INjr`tNev5vw?T zi0JRw9=hT164dVXX!}EfXq(GOV-hAa0-}POJU8(2;E}zan`bTY?=^Se-TBs^5wMd# za#R_Xi51JjNwO`$k4$8?5vzj8p3Bh)PFDlu^Wal7+oP?KE9{c5tsHv7b!PJqJiEt` zs-4nFvf8$^%j1@~^Pl9~et6m;|3>=@$d#>X?MZ05+DrP+^;q-Je&VH$t?j#DtZWZZ ztIke>5?Sz#Y>1pU3~)oRDfNbxWRFd204)ksk>30Bvnhqk)0^BRCcy+29v8Ax|LAd) z)#nZ-<1>A~^TmU=4r0}nn9*wxry~jbsakgDvEQ--4%*%_qj5?X)VotARFKRi)~l1g z1nhk0rg<M;Kl;R5@h2yJ^F{sF;oiM}_=f5hv7))$CSPjAz!KY3W_fL*&{I>q?Bgvv zDOhHQL5LN)!7T4o<7!==>5bCTGIYP;`RY;$LzC~Vi<xRufAwUWJ8AShgn*aA^LcQZ zeidsYh_xgx+U(+I-_9*O<i?61?VTu!Pj4#RZ_;3ML4`6w*&M%P7Y<3mdFQV5$`LL% zCy5iMR{ePcO}Z`5FW(?H&wEZ=sPS^uXy6fxIyNa3<a5o5g*%?js7hOJ%#Eb<SH%fL zt?QKIZdwGt0>4>CPox+<nSU7`_jIo^q(^qwL)iHbOFBJie7-Pa_PO=?r1KN)g!0Ba zKho2>Qp~kNQ{mZWl^UVn1waomPU=(koX&e>H+2Y5MA@(+a^__d_Pe$c%Eii`e1xo< zzD4F@C#ojfFpTZ5P@;{h#B;R;%k$2v&m<%8%2VthJ&zqo*2B<X`SV>#7;u)iDdOgJ zEaaiOV@-j+TR;L9>UOxM<;nJ%Jf@Kh2)>kz27qHg&6)H0PSLxn(4%3c7G2-w{}3>D zinTD7z}w!GT#I?&N*7~_53?dPs@GuaWED3^z(=Q}rXT)&Mr6Yfd=vL?m?j|4Z9Nm$ z-zYcV0op&owLi*Pu?bG`q}e^~Ju^@${Nsw?R|_WZQoFf``@1ELu@fF<rpv1%XQDky z)?AkG9!ayuJz^Ay%~iHByYT=N-QpYnAf@&azq3Nlr-Olo{OefyqQ(-+GWH7p0p21@ zMl{NaZHTS#?gS2>7IwuFkb^i%J1st~!mS(dk@bqK|F7rB!28AnT4hamE$efhb&~PK z@)=05A2N|)bnSjV5A&2@+cTJQHRc-sI}&S_)}8VeV9bG)H<(PuRhx?GV;f5#=CZ8N zQ}~~hxL-*W6O^T?Kimo>)<X7JQ{s{*#qF|%iw8Qc``y(rhNGB@`=b*?K7LW^)P!Ol zcJzE!rN;XnDkcv;ZR?rg!?BU_Q8M15Ud8yg*L~!9Qaud~Y}<E{C$n9BR{ME}uryy{ zu`3E88`zNsIxWVOro$1ep>Lu#L_@W}m!J*!xgL$o4Dd$vJ(apOEm`uG9B*LjdKcS4 zRm}I=PpQQOFAT$<)BCv{diiaJ*O;G2xiwgq_+9>}U<_}DYyPFYykqXwPQx#=>sbDe z5&)7Gew+;(DQ~(2lQdj9C%<P`5aPDDrcmiOFmAQH?V%+rkuD9yCZ!w?JaK9e<MDsh z__g~(aaCH=trqoQoJk(svYI^J-`0sMU7Pba)B6^aB&=ARO=YZ;#{>nBQy{Zx9E~z? zr5GfU*`{gaLL4-H3?$&?mslDl33^*q<0O9r&2Op=*!~L<6PB;2d|n(`?aj(`XFKpf z0E;&htmsIC`Gmzl5V!5PK=M9I>^sG8r)j_mgBAS2q-3$py&P24b4}dGb!U~1IC}i# z*`vTIXS?}_JCx-7kzAhK1Lw7JzhQuGu-z?~U{uAvvl`&d(bn&)Km|<I%*l);F8D2a z4sGuDx!!9h{AS@sT21-y&Q1e;5lZxOoEt@uB|4tO7*DVd$wxm-C(i6c6i)6rx+ClA zm>)k`f)i;{%8okq(SjYa((lRhY%w!@=waIOr9R%O^v#z;W0|4S+wm#5eK-rHnSYU< z-G{gXWrhq-KhzL{gbc)_I_r>^iE1wpjnGF8mVZCIjP>yY5$7tT<01(WmsRsYfItQH zk3Z`JyHb$uB31EO%jB<PnUjq1AB#SEsfh<G2Vo`GV4_kR#iI`;hbSK7Z}D2QPrpt- z+HWRb3_Rmb!7Yg{Con-t=7|n$P`k>mn?|jC>CSHgZ0(0Ka?Bo<r8|Hhta4w)o@WJ} zZ+Cm#eUTPcaDf$eS=hwPmRepcRQfg&_8Zg;$E$YdsYF6F-qk#4TB8H6)(_)QGZ0)3 z8#t33z>bUlVAl%mS~>#Tw=W5Ltv!-%Yk_0h_*)EEiuiTH<8fbAoJQaaMX%NgLk8%n zqD;h&`X%f|Glo%-hY?LZYxZR!Ls;>$*z<>8Pf<UnNCL5!al2rMfx|kQSscm*?BwDh zz4&)6tFd7wr^o_WNzMDG%CIt?w+}n@Sz*zZU9EEhhC*e7y0;2GmZ{2Hc8gN{OaSf> zp0e1)t2`u_PI;_TT{uhJ_f4BOG)SzuqrZo=RRHAVNYtqm1@N`*n4c?ecV>E$wAfoF z_8Oid2m-0Ube3lm>lzeVd!7z8rOZ@RUcS`HYe#wL<&7<K-uJh`&ID4`&oA7f;IR_@ zOe%C=my>k%Yg%=5A9eS+aU)~B66@bd<foAMx@Nx8DJnqK7Hw5QarFKVw<#3j5#z^Q zAbk`Of6z~Ip-d9enj~!0ZX(UipyX6tTYfn*FlueIm-6pG^zZHM^w@zucI6>ZxmK@{ z;l9ci{%6#~mR5Nbf(!+8Tt&LnEL7h~f2_WAKu4jl9)xNQxsf-&dV34#wimL|8`1{= zCsYXn=6;8~mp7i);(-~SR(O1WJ@tbLO5!B8&zx{!M_kljU?)?(!2s?dHOc1zZ%4>| zcx-RLcQZE;>mfvLXaxKf5`nV?Goj_Fv{XZkptja(TeU}L*T<VDcpzj2Q_0L9rEc<U zh~1?`t(85z=XzM1L<&9%iah^*(@2%jipgqvAY>$4l_id;lKNO;1Ko{`4s7NrjSl4X zj4P3?eAZnUC;+O=@Z>#;@zOXjk;TE5?Z#F4@QWl8$7_QBhtU83=@^9W4{FftG4}mO zqyWg{m0{(VWTtR~C`8;S<(0OSiimg_?J&YQG|uGIi77EsLA|hz#$IVwy<~3hM>7&( zGuP4n4s3im1$*63>KGQcE^{Df!YtGxO|g*x;)~6UV{+vAD+E~P*m|Sjl{OvUN|S_- za=A(j@CIb$H14aH-Zq0z1Fa*ZcSLPru(iD}rX!Cq`j-(W7K211y5`iDUYaDwD>c-o zL7a8&>MguM`(xO=F;vw@4GwS<o{?enX)Se(N{<E=KU)|TR<jRe$g*JLQ8kMJbh+xO zq{9jXdLiRdZIdD9dbtA}d?LET&+kBL`XVuP#7?m@EPC3|jvK$<fQe3}b2;Hj`Hxp! z6-H)krVdo$^K3qCiY?`tc`;?FLdDWC!Xu1_HQHExPG4}#Dpo75nW}Xr`&Zw><>oQJ zVC&uK^EMnO@Ek!e<E_@xWs#IT1pB`vy@c;D<tp#c5;_3Cta<r^=Ss_vT~^`GcHr#y z&9~N!J@h{}iNBO24f0gjMGiyaaw2}0IVkj=It*)s_qdkXnYmS=-;V5#n3j%SDTPf= z8#L)w73z2E%H9UIobg6CShD^X<9|cPt0=;$O`s-|M#*We{DNbOh&doe6r3C17SH*G z>2gNvF9tIVz_wLuw>ppdoKRpQ{y9(zRPwH5zTg)nlhPUWMG=u2Z)t1u%&+iCSn8SF z;W0M2V?Ei7aj*tSUeC425G(sr8aTLA^5OpRQ7)G#t9#QXJJGO`$3IpWo?p@Aw!BLn zyhSqM*<(z3LM#c&r~RC&JABjIsC754`Hf||?O>V>7u^79)23Kp2PCtmAW6XZJh8eZ z4}#(LRx<YAzVT3~VH=X(_OIqNtiy=wHxd!hJ5c9!cS*}WSd~-yEa=x*$i>Wa<JIEY z71|m5WR1UFW<rFH70hz@bH4Tz-e3RSc^oG*KFR1GP8hStn+pwz=IT&i2lcg#Ut&48 zB>d;;u&v*4`2j$;TW(^l`4IBx;c4f)dF6&_q9eCGemyN`^D{OTJg^@<Te5wAQWDG{ zirvH{H@nY!^Q#b=<o;0c@b1C3vLpYl|5;QG4!*Vb-csMdsz~bZSwr={6J{3M>=3mW zJ`7D^I8iE~atf;AfL)$UYv*Zl0}^{GF8{SJj{sqVqg4nGtR-+rc)?2k!;5C7iBqPF zCJ9?grK8%QDMf9s);F|^;wx|Q*_T>Y)`4190a+X87wv$=_;frM6X6=P8vCQ89RKbk z22(0L&1+};a4yh4t1Ld~)>w6PLw4YOn*W1(3dAmdBeBjJH45J^vytnj8rdobCVrQ2 zZqX)=^ZvaQ0ZNbtMcEq?^*vqMUOm<ikv^RWucs;>#9TZPqLl3`$eu*4@Zhpb)+G%W zB?`ooIh}q$3?_hrAhVD`JM&sDN;$-*T}{YX&Y7TB^KqZ#pPp+*U*?NRdoLc=-uMNs zQR3Ur-N_<#V=LSEJ=FB%l27q~oWA+$e9VWI#2VcWXWMVt8cET!C)AI5NpUDYqxf&; zKiDN3a=hERYy;!2*jc$jdfl^BuCnl-pryC?HuKLoCi9Fm;aedFjm(KL;qhe(Q(@jR z<rlH+61m98MLfw$3cyp6Y~1>z=riz}iB&Hy!|mqM0O&nsynY%*-`F*UkEk&<(Qu$A zOShrGb8`q`77a{$AE`SXp%io|0N*H)8DwigJT_wX`=2@gmdB=O1!OQzsW*#&TRi-b zD^vR?L{DT_3M50B8x<m;vn)L;?W#&W(v>Y;$P&ynis*1J<8h$>m`NJ+9{r}9w<|&y zFLOW8{UPlC=(WU-o~HNbGvAy1N4UO15Lg{%qH8^%PL&6KACEid9)nD=c+P|h-z0?A z=u)+`IKV*zL0salteX$!6epJ(PTd*)s6S91++^-@0yEb>XyL`n*e<A-sSMSPcqFW= zc1QPh7k@KY{Fc*Qwb@(pzbO(&0Un8qW#1CK`F&GpWQx1;13p29>eT?yG~|wejn3Ru zx!)P+MY+;-=ihSbp!Y(pj->x!;H{iPJafrueDiQvw>ZmJO8SWQo@}NY{T@9_@SS)R zi0TInuH>^f@(j)EX$DSIj^3aHD05|HMv<Rz2;N|PhypsM0NiWqG7j$XG`GG&;!CuH z4}!S}L_)L5HeogRa@VZindHocXoX7Y2rR$~a`0{@IhfX2{u#I)d)ZYF@FSH^cls@E zFV%HeQH6ZwWzG}0s@ov`Dfx~t#K1{Cl~T!$pak0gP^ZFKcVYWEm%yzN=fGcA2ADzV zC3<}zy-@t<Qy3Gpmi(84K0R!pCw7`${4`f~?VX^k`OElQ!_`}y!Ej~6anLNtn%sZu z&``;be)IgSFnuD{tr0gXakgc5?(ER|G2{j|i?Y|$Z}9%)50?+MfyZb=Q#TC)iG~(S z<G<Bw4g>w3O6U9GL=_;AzyFZuV_^3}u#nA4qI(|GExW#?hbPl8_vJ>4(yflO(y5+; zZ-9%$s;LUKtI-@XWpu$+?J)EA-IS3A6Qg(EN$S-7f$Pz;GvL5IF~75K$re@KJ1XhL zF3W69^~%2(ri(UODszhz1BGiVh*@gv`c8-K^nW7v{j~G5l&-LMFP8G0+TSu5wdbb0 z?nIj@SUp_4=)ftpa{e+4A(RlE*r{DcN=b}iqh0PZ7LnhSpV$8$e)%U8*88nVm*b_C zwH&$X{?ZG#3!)NEiRP{-F$#%sZNi0y4PV9VWU;DVle|10*INGPz=HbbYbI>dzh614 zSsK*Oht*4<SikbjN)j`kv-4<C4JPcTW6ZB+hAobjz~R12sf?Ue&XjY+zhrwbW&W*t z(>(Ev9{D>bD2%TpK4r?PF6&~apu~K+J-qawn^T{w{%*Jv0NL0d+k{=m%XxRFW80V{ zvb(y4Q3!NE>jmFZ`*9a0ISMQD!b%Gg-<;b)sU7!;Ns$brmBL7b<E&=h_X}qO&is)A zeNh8#x7S_Jm+5)s!s8iEO&U%oJc%JhcISmib=a!EjPsIek&Zw5WI7~@(wIDQpRoJ2 z#6Ix8fA55?k}JJi;{~OH+{QJ(vof{`RWD6>bmbeqqwiFwg-M2&11+W?KTwEP^tYqW zmYir32RG%TlOAyJ2Vl=HA<tj+)RXdFHg$?q3slamSL(W1sTT;ocXH1N&bZPh<<+U- ztM3&C{yWTcz>_6K((C8`*$H#s>ON8pTQ=M#rrZ2+I1*2Yi?7aMXbzMVktEXF_BM5x zx0o~}n+<{9YYZ0NXfzdNKHDv^z#>T8FMGdqFIrOCH0s`xrC3oLREU0dZGbhw|5iWS zHxenzcERiB8YH8Zeecm$ypS<y6|C@5pgDa<z7ZCUalf?coAz1KnbQiF(U}u{=-Sy` zl&H#*lw8XqkI^9+ORP(#1?N?22zq_S@U>UmXZh@?#OIn<%%0!Ok`HdC`N`B-rQRc1 z$;;mN7K`3~<#zR(hN;ahwh$>l9uD+XXCM9qsR7Ad&$>XJ)a_GFZ@=ZUmoPs{!U+H) z-pcQ$5tX@75iQ$?J)LBx55`Zb%}GD1R(20L&gL-~25k;!F%d8=N`OlASDN~-=W!7} zMa?Cl@lqO(f4KauvtJTvPga0QQ0%<>LI$i{bV@7+NnnMdUSyGd^>Dg0lP2_R%`wle z+HvwP`8`baoNNDY`7HRUp}=k#`Hx5j&bQ~5cg+B#9brO(?lkzW*sc4H)QETX?tRH` zq_25fKd?P^&VR%*wMs^GM0r0T@AcOB_w3b(kvgSe>ZkEM%oy0xypI*DGtNNJ1T|u0 z+&{AI9t-OqR*9}$#w7P-S-~JTYwx4q{P2|Z5rcL}{n%^cJ@*8Eq^Q$@X@_0Gy-`OL zglq5Z!}6D!UD%cbkpU>O#l!akVo^KUa?FVP%(K!g#ThMPN3ymIt7uffdc$IhrDpVk z0HhaxjisS?(}S@qQOi)U%!TH!b{6bZ!Cy2LQ0@L-&DGyLWm?46C;FZLs|6SU&*Q=k za@s}kgOEE=CJeHVZ4y#_?Z{m)sbcmANv{A?sgq#>9K5y8%oOsgaS6)&jphOyPVK_f z6**wc(CMiX&w=RAmdQHRz?O*)@Sk)*ss?7qZqguT#M2#s?J$Zdd%Si$>=YkJ4IAih zI3K!<EjPCG*Owi%jegC+#rjo>&E=8J_7m*#Vjw1*PNcTAhx;c+#eP4ffS_Dr#%)hd zvj&qK)Y_*@%N4g}HV^40bThIb);Lr}m}|0*nL=a$ahR~}IU;@D=VJdNy5hX8Fiz>y zZO^PwyG-x5+XS_NANEFe|AN5r#=A;W-1Un+g;nCK_aYhLJv<c9?x4QQYgd|VB)!c+ zLK0J!ipV8rD>MrReySiDXHxByC0w;rE0qy6EjyVeMpAAS8CwjBC!EOtVy(HC(aEb+ zcUz5^Efon%nIfOiLAEm~wFX=rU`DZ1o;qr&;l9Wg7U#;Y?sf5EnGT$@0JhAy<2M-V zQQc(&<@a-*GIyWQZ%}9Eff7FqLkChTj>Al{c-B0$Dh8z^I;f5#?gk&Ei(8|SO6lEZ z5FEPQjYLAJ#-&<#uS{?Avw{6Pw*1>>2<>V$tF?xqLq*A{D*PM<RepPuv*PsXvg`GT zMF+ay6WICkWZMbd#BQtF>XuCxtP~d}2hLMs6vp8K#L%xp)x1X=#m_!CfL+R;4h6f) zn?1*;*ay-jIHm6;Mi!^+`J6$k38RgvBX#KF8+_(d`wczOrKCABlr;=ol>cFEvhu!2 zhrBV*XJ#(V)Bt7cARwp^qq9ns3MHxs-(<GGGH-9dUg#uboLHP%k%v4V3LxjZTxG|< zpEBO^a<{Xuw$U4yJbjO>|APh_2Y#NduuB$Cx8f2!mph%YT22n$A~Bmht2eIt=cW-E zQG2>rIYoaP2CtGz<oIp3QH2Zlit|By#NZwmlhGKb-8?vp-*$)S&mGqfZ(gsicODK9 z>EBe|6>7W7=eRs}JNHmFwe+UC$$pwwXfO5eq6$V7=YP4zNsG=Y#g32O(|-KG@LSo- z0D3=AIH^Q~kjzH?h>~;_xjZb!Z1bAO;O}MGx(-Npzabyp)7n#&DJuiWjF?1TWa9GI zbynZLZf2tc<#nQU-4&`}JM}8?xJi*o-#5TOJw@sIr6+5B3M_Ma+nS~kU&iDWO)!|( z_YceOUtpBr{4MZk`+yE5&9AF8C`zr|{YsW~o2)Y};&EtszP3<EX+FFA=%V(Yx4U;w z?MBxgqHc<sl_JTRBLSZH9TFKc`s!5ht*B$VvTgP#lhOr<Kf^sV;Qa}M^gM{_wx7AT zS>k1MNW2wXJ2qGdQGE7=>@o0c3?UlDOMEabL?{>hc=SKb(YFTR->-E`(w=#u#4I?~ z7Y@*=BY9d`j_B|H8>#L)`FU^U2YpjHlQ;)xg*It>hX>${(yvHr-}HS=6fkr<bALfS z$)EnKQi-)WVL$$BjO6{WLUQ2TaV=LQ2WgThRh8ZSC~VCoqx@NEb)C-Ra4);L^=^&4 zO=H<@#`*4O!!KA5b;1l4G}2b|Zyyp}5Pa=jpFf!NajrE+7+_Nks3j(Gl{rGk*pG=R zPtiw^yaAdcaB7h(<;pksX};=_|I*p7Nz8bzu<3Z?QI(1#Slq6b8+M_(YFD^)bXsLc zj`d!m(nvZ&8$(3@7yc1?<0D1<Sw}@PPnzVyCW*0*uTGz3W^rV)iFjV9G8B>mvKHzG z|HSzt6mN8ni6j0+8$je?h!@ix?cyv2S}p4`j&il;atyZX%;2lN`Zd)pF4Ol_ao$c$ zsRV_LA_tD2cSW@fOJp0LX_wMij~{Q0q!eKw5Bm9yB#_S_p4w4BuGa8B8H>=kGkoyj zKYsU1Vd>VT{ko;~n<^!4^@Fw}9xW#3J#e4UOvnN+JG8soCH3R_%u7Z1#Udr_46p_O zvdlx0>05_Eui9Kdxf>1HM@cJySWY7>`3iFFcAclLtg77ziqqA!W;?+BNl6DuOM&x3 zeN{m6b}I?#bq&vd<mzX!oa`(WQ++292;IKM7;mc<*b0cPi(>v2i3#Za3pi4Rs(4^` z{&tq1+mV29QbYF`aE(q`l%Jd#@$2?4e#9QO+=l%`Ua8quRRiHq(jlP=w^LD~3>lIk z9pbU}R=*rRrbQM8OFo<T^;b_AX*u>NS0O!<<ab`>z3d<dkHd8~nrMQ-I3~ZdkeAr= zD4nm5Z`vAOp5#ZWG=#||iYtsG_tI2F@G`9A$)%6H`zV|H6ccXhIPb=1G+ak@od;!{ zG8p@;bUnnz%>HSa!#`hD5zXD}<0@UJs_pe3miW`xVJ*r01-C>&+mEBsO~R0EyA_9B zbEf)a&OLT>?jrA!Zt5TW0XY<MRvdluJNsBK8IvG_YdK`TIB4B@3CC6~Ww^`G*P|Y; zHr|uJ8QgGdU+KtI)ILQ&B66Xy=@eOt4W|25PIDL0h@qfLo^o^<!V2vl*U#gCNzqsw zX&Ow<r+6WxAaN)lrRMoUo6z;Wp>2H=7YT+G>7W&Bg?&<{TyE=B!s$<pZm+1Eu6PRn z2OIrkOaOP0_C9UzDOQYlNlby?EfG9+XLF#cUc!B3u>fd@WYe$1r2sEwy;1lRcZWKR zNy}*#S4rPfO{n{D87p5*E?O}A#pPWdLf62}C0Ue~RGmWGs*f#)J64sRx?823)lW7T z(;tRm%?WlU>Xb+1A-~fqt>V7`J0V+vi6hae0p>_K3%It#ee$j1)^+x@CX1_knjiSd zQ8pL5;A^Ru$`H89nszc{XZBbJwZ!bzZ^3uJXON`c`>Oq0)L8KT)DPq>_ld;UvgG$p zdm)_byyQqr3$sl1QV(jL-9WeK#Tm9m(F;Y&XPc!`Qy~yh?g}-LI`aKyf~(#k2^nSL zkIpfh-8?ObeGvuY^VO6tQ!y(y$EfM93pui0cI8;_;q@ji=%2W}AQ6MoAI^@{dr|yL zTB+g`rhdy5et?>7d0a~=9LOnB`SmOY1+0&e!Zq{P-&H%);4Rz%Cg2AUPgZ($zAJCz z4*)gUfMRyCIsWm3vx}o)-z4F4$_yfY)*w$a^OYV|@V;K9N?mgyLWSgP<O35Tx|H8W zh=C;gb3~lLuFFIE@G(5XVC+g8ugOS4#j8q)Cq2wf%z>s<mWKT1T8tS)d_t>zeB{Mt zy<wMC*5gy>+J8OeTiPln`|1uFcj8}!o;j^!Q1GlPxq@AFwpR+ImXux2>@@+%0Nw1< zWBG1h=W6_vz9dW`&_a*zy%|w}6dG&T@rH5W4-LLnl4<ipPkemU;(9~&fy5NXX5f}o z(DiwrQ@4i$bkJi_%!7*mr1$@$A~Cqn3d}qgLtoejHHwcesEmTkiGhCrV~5;4axbFf zv6>w@)mgSeFZ&0eW&wpH7(b&Im-I=mzt)yBkcHlfOu&Ds%#~3PPddUOd5!dulX+Ae z(SH50m!#qcO)NcB(v56OvV|tZKtnqT9pa<Ir^F@CY4GdpYpxiTxARpbZjksMI>4^T z62ZvLLE#3Vag>k+7f-<1Fg4O_5}0CEvSY0Rwx8N`2kq1G^#bY}u(VlA2@O7_!}m`Z z6HEyH{e11$-8g<OgTwHLwMr|)OTjor^`MKVek-SC*zLe&_Al3D_UotjOV>4xIwH{N zZ6%QXcZXEtIbFB2WDQt=h1L1~9AEIb_?`4~*Uxpu>bN&O30Ge4WC=V$arY#62wb<@ zrk(etIpUo;o)>>AiER`SP9oq->6EM6iVcV#(<73brE}@Bh@6is&Dj^fG8b`y>$KJY z)u7kyUw-OSDAiPtNml-<GBJRiY~G-P7>un^)jsnu(2D&6iM`&JwEI=~%0tmcmfh$; z_S5KaOvooujSo0ef41gxui|T49h+Q`S!))9hFQ7NeTas6KX*+;Ott-6IfYEsVG%P( zNEK|r=9!U8?9o|Ah_CF@W0cE7!3yoU<4xs9U^_>)EbeX3?w8n|&VM%JfIi%}c7JGB zjS!?EntyFk759dd>+~D+K2soy!yWu1pw6}oD`pl}Sti+~$4TuP7AHXI*=o4IdFFT6 z6h2Re^{5E>C0^ax-XjBD&B9C1`<%yrUbg@WMn-bf*t}=s3Dbi~l9A1D!ry;yoiXWe zObH#|>{R=|E0w&ZI4gCy{3W(=L@PL=>@#he*wYLopp&|(2a)kL9@94x#^$g~HFg|K zVGe*P(7E|+qjc!m?&FTZsff4S)e+qCUI<0$J!5o}Ji2Eao*k{2i`%)tAms*nG=VW` zdAbz{I^LSAI4*Dyjw6ZJ=flrbLK<LclC{e-D(uP|oHDE;cFI0=TI;*H&b?lFG|#GS zo^BLr`*U$xJ!?TymN_<DvQjg4X$oY~B64lBh5o`^$bO44-v&erD}iEI*S{EZq7KPK z;$B<+>GbK$jmaRFGuK!1D;}cVzB-$Whu3S~^e}pV92NE6|7|_E;c*wT&$b>onBMU_ zPD#YYA6CwJ)bu(cwgXon_@0L0ZFJDBuVGmaUQzG$Kb-)L-E!AW`b6U7MEiK-<b0=7 zf}vlWH#G{rA2UkZqrDRzd=y^zkIHXuY4{;Y7f#`V-FSisS=9|hg;1IE`)kMNXYr=~ z@{cN8|2+u|#htorku!^_zws6?p+CYgN6G8|d<75XO7F~E{U_)pq2dBb*bY2Mmj~>Z z*W>r<nMG~`4wQMn)fiw$^O{Canmz}&5A0I&&xu?--l+w6LFq-mU$D9V3SlB|2iR)| z=1cbrITf_eYfTCzmvB?8$*m&;uRn<ZT_Z7pg3=nLDQ&;nJT6~+<t_VR$10fKZ-e7C zqlD{pr#DWkBoEXnVDpbO;U<JxyS<0Jlz=<WlRvACJbx`jo|*<vNRHbMe2C!hSztpx znbZ-dJUY-I5vvNox@Et6g;yZ|pnk&rSt7f&F~b+%pf&LGJ?!G~iTL2~h`)a4h1Ln! z1x-78*OuW$nc=v>;|b`2L>Hyb0M21r(oWSfE?S8<$nCQt0bOetepU<z*o33Y|2GQ( zUn>`FC#V-XlB|}=ikP7sszYi}Wa+5zR1ViL>QGvfiWp-foTDl|6e|a%Btz`+s3tm& z4fGu1g}&k#Wmb@y(xVVR$VXjhD>%(N%{d+vFKm=lnXcM+?A{PZto>o*+_OQ&7oTj$ z%4g7b3R0wMHlDrdza-^HZ)WxuTXvHl#$*z2-X*;RBg4gkX-d?C8KA^1iIk%@Htazq zQ@^%&JEKNABz$8qJ()4jWxAN#YiopLv6!gxbVFvMm3%XPP_nq%zYE?=UU%RZqNIkH zyXoYDEpxm0(k!~`)x}UbOujS=`^S3Jn+!*9o>~NQ89M@)S_%%y_CJcgz|LSy<B{ez z%Cepr;@bSub8E!*`OJ@#XCtNP`ck6;lUJgI*l-^dqM)&e$QudmP1z;eO+#66-rHo^ zbvf!@0|XePmB4}ga3Qpn=`y-S>-)jq8&|mjE@d-4$-nuaa);1V%er9$B*=QuXIB5L z>ELzVZ8CY>wy}6z2^A8Sc-659K6`YcvrnVWe51$=cA4PugltukA>YMCqr9z$uT|&t zA0^oH)<kvI$cDJmMJGkody}gAAcG4<p)?|cq@fjjrs(x^1J6OJY}bYQ`wo=^6RYAg z;h1ma=Jz`tk!*ccu~QVX7?&&O&(}oo2K$rLx@%4$6Zj&x#V}v9AgUesJ1WRKIN8>G zokZhI>CPFG%YHxnlaDqTdeRzv#v#c2N;diag^?s|0GEVpyIx&24<X9NYwVWNkmt$r zdyjT6RbjgDeK>1{Q+)Ar>z)=&5Z9H*U@xu2BJ4<`_3~HaS|Ho!lN2EftfF*#O!9@- zgAe~|bgp6{nezC@1{gNZx4L2Vsf+R{r<eIyF{R%jCG43y7*NRooNUn)d%MuoP3lk0 z6o)%1AW*+k^ib3#`0=WveCPA1GZ@3loqvA!nGSvg(kNG>c_l$-S>stk+n7T$s2hyQ z^-Cl#eoqVrm+l0sGWvcwIRt`EtzQ-LwP=lhXJ9B3TU(S<OA<1?{^rM@z;#N--9ga% z#(MJMnN~|0mS1FVB9yX(fW<vgIQN(UnQIl<Ev93pyMBXWx#%yNjl_LLIak@aY~5i` zhqn<a((?1%)TO;+OG1E|mn6|ZOMMEh3##~2`wd>>h^B6ObF38}rvvA&g1O9@>CT%* zcqE)kS8j$7FPZo>GC9_~y+CyPzoH|{$R|_WOuw*dldQGSN0`Wgxt17IZKqJQ_mw%w z5PjlNb|gXpvO@AD<ONpZ1jo!ciq{(M?$nZdo^JHVF_#lO)@HqFUY0Jz&9hne2@HXA zArFXP#l{6+{PME2>$2J52ahu=%K;pb#nZ{VC#HV^hyO5wo`>rB{CYKrw38YK1&ZHo zIKe@9i`F_!{eKCIGV6&tePGS{ed(s4HVU>@Ob|}q0DdwVqX#rWGRMm8Jn-AjN$$fb z0QE*AWmY{iwqboY0)D*E?mycoRIVlE-PNT9Wst^Lmb|H0d~fsEV(1ojK<YYqM>we- zV5`w+@2L*Lmi;QDCsC?P;#&Mo|EHOw`!v`lEDnCMaunOm^ScnT>9?_(QonX90{<E) za^8FX7o%;a+GaBLNYd5=EpQsqT3wC5fs9N{xlaJ^DqTO{cvLR<_UItb<5(6J*3*=G z(oxRnImkSdBX~R#!^-*lwCpNjrGe{v+$T2JUT+Xu<@Laf>XHosuiDK2zfNR5)evn^ z9W}6iJ!xjD9<!fmmCd@g$}}1ossKgh^+zy2P}|?$YO`?cunDI3>PJh5*dL?HcX`cd zn9~p{Swi*mBKj@sw>r0YNOT#a@yb@#s@H(D#i#Jo0~MN|mi{a=c8B_mRm^)IagIuJ z;{0PBXS~=l$~cQTkp-z2U*^<(>Mc_s&vQi6y~cMKQ;*(dth%IvoLyW~H@?wmik*GR z%@!Vd$>5Bx^SZY*?N!yqP$UYKOS3fX%_e!j-#-uuQJSh0Z%Y)Zy-Nv&gd`N#<;C1J zbAp$n(=2uE_r}Vt&nruc;J^ffkoH0&A2|i>epafMlBon6uni;k>53B=kejdY>PsVm zk$SDu?^^D)yg%afP5k>TcttsXN;FmRy2~aHdnEN4^y^$3@58>+{^{%N(i&D_9FENC z4yGyQFU{~X{~uLn9o6LjzyED?W6-f-fJjOAC<6r~m5?4i5D+QZV3Z)OD4mLcN{#N2 zE<u`sba##z@w<J0zMuE^`#Xn!Jcn~UXY796ujli+uE#|jHZXnLYK^Mz$#KHAK;^Wy z-(!6?t^8pNoZD}sfXf<`VZG+ez7ZRhKBL2`n_XaMi19kvNsN=zM{Ej|z2zwO>YLj& zsy-8N0Dg_*4Bc)zPU7NvK_kJ_Fo4IFs}Hi?yDNUvZyr|F$)mnrJQNn4T$)OI?_N{; zYlUcvtJia;(X?jd@}-IenYDdKM^JEI#M!F!?OLXE-Ji>!92eDHiQxpWDf<OKH55Ks z{)x&rA9(G!^ed4yN%&L*ORxKyXit%DB;(In&(?9#s(wu#ze7&GS}XHh4&pa({><EV zQ>Dx9o55wH;-uFJ{-9S4zP1>9Xz00ot^1<rXeB8!M7*O+s{joD)p*2OaQ^LidLQ9M zJV!m*1N*}r;fI$+x04O6!B*FQ+3v5i<~}=j4|=~l`-Dj^g6(C%JBnN-7>m#}b*B!} zl&AjT;rGU;l=t6$D#XC_#OuP={P;Vb=`nWec~wb(20p_Y4K|TJ8~O)$-A133palY7 z@%GoB-9Nr>Gcp`mqCyC!>XSV?uJ(U~tOaalWH#OHE|u5wkaSiPd~<2Rcu^K9(_znH zEu7{&8&Jl$+4tx<xP!(cf+_b7xgjL0oWGG9Y5mZ;mX6D+R(VEuDyP4gd%{v_2#1MT z-_{~|NiZf+?3hMLU)ieO?{gWBoX;1K0V}YnZpP9Wm2=A4h(Dtn=|<O1wH1wf@aQ9* zo*h%)0EjG!34BZP@uqKhC*4O)tmmDpSGiZlOL9s?p{N}&J8Gxtp6xI8<vO_J29T}* zqOp<&Tnj?=$B-Sk3>SwcGx}_|-C|IKrE_-BphX^-jp+rLzA=zw72;^uBRb!xrG73Z z*9lhsqW9=-BfnGAG#bNA0T21<iEB)n5sAOHQ*M@iE$+N5l)qfq$Zp5gj#M2GD(Efw z6_Y3%AOBIo7;bn{*79@{PUVhNl01d2zop%PsphFUu*^Ne&i9Gm$6RsRArTIjsf4wX zNxcT54@Pp7y^Y=C33a(o8nk3}M31{%$$J$Ax(RkMbNkt+Dw|7(rB2kCMKoH!ZL<Fr zc0K<<I!pnMEOW|l9qrf_!EE5{7e@cj+zKNB?qnz6muI5BG>+8IL~i?F*Jn-Q1!|p` z&S2{UoOUC{ud5u%{figUqGmW8o2?s2F+o<eaW0U<ZYDUM<c5<=FeyjnY5wYVa#dvf zFiZmV=Ws^0u$frnc&Mq{0Yh1iAbRPTNM+(X1M$W>UF&FT=h$wZMpD7Kk0aKd0%kN2 zef|`&OL`-ug`Fpe@N2@pDu{=0izNMW=I>y}a_q<{Y?RzICwe-zkDlKW@*G8{(C;WV zJ{z!s+H99tWWOOB$AHRHs(~ZEwbsS>>MGopTMN)$c!MAsb9A?#vCTb4arUL28J-ws zKL<v>RiSzl58lyzx(Chendkjy_euc)Bw`#?R`wK99!O?pMp~)au+FYXbNl`}5!5a8 zgQ8Uc*xMU=oKWaz)TZ%Gd&L<4Zkl`S=>(!fyqk=4*l{N0&RB7;19rF+Dzf?DHcY_Z zA|BdxM1ixQBtF35o3Dn((u<zN2>5)nr@>m2e_H2Xb&`ot=JdKbi(uPv=>Fvpb~1UH zCDc**_2fg})L?n~%z7r$1h}NV(&Tok=rz%oc#RA1*gf)(_TIR?Gj2-<JNj@yh-^0@ z+LlwuPvU*V>3z%CwUxnvy-gvhSlO+2=~pV1WK<=~R?T7IPH%v-<AlzkIB`^s`#NdO zRyAKKa+G#$w6M?yo>RFrAAM#{xfW>eYxnS^u^r{B&4J?hJ(bffANfN_B(4VD(2wsj zcw3lR+fYtq^G%><ru&HBfK%a7%>HO;TTt!vQ?Q?Xfvc+AL56T%^8shiD0UGS4_;U= zB<sR;KX~orizewidIr8mz7W*d!e08=-_t?u5~j7{clkraZL@5?=kQKn$pd{PG1!1V z+Uce$^P2~8LhC>!lTC9uLL??TuD!egh@Gqs^A%hYXS69|zKy%N=FwaBfb3GXvwrsR zCdRsMgkj<l?P(~o%R-vz-Wl|>$?)JpU33UaffT+}e@(l-G)?oxsZF8#dOrghV41l; zoD>|toW$in^UAwBsV47;1J$RPC?0^n()45r!uJ{S5LDG{1Ijc+eg&})sduvGHI>6z zP+#TQCGtn9%>li@MKmk@Y90+Q@)bRWDj{H+yGY`e8@8_wJ04)YWVg`}VhY^<c)>p8 z99LUxKWq)@Y6h45lIfN|bQ~{1puPcT9;W#=48CaOFlr;KcUk22_q-zN;EgIdi{p@f zbt?spAGapbl-(4Q+V2rfB?Zc7m;qEKNnss6X@81Q!001vk%Wp-YFqwnr+F3#H5X4w zI^o2|Y}3(QN2LH)lM{WQZGg-#OF7J$f|YbFZ8&@jTiO%P#7z3&&aNxL0%DY5lX_m% zgwzj*aqEnfd##{~0@pc-AdD9;tdLj|VNuLXo<d|ru8|X9Wl(P)crQVa5v~y&{hLXu zkqy#gIj-gRsh(-V`Flw75A1xKc;iL8VvLn0KXvNoz@UrYjOpHdbS&iW5>f0`bZ>vX zfcs|F=d=pHR0}0tkp)l~xF&b1)FTvat4+p;xq13Vr^r@tL~7ikRGXidd}d;-wk&)q zbDq1qy@hT-`o8So<SZVUUNuf@=9<5Ha`<m_3&41ok|=|$^uSQPNU$?DkM!B;-SszK zqt7&-oP3^{*8<^)VFbDBdHx3VbO+VN=BIKCd?m3CTH*O=pJxy>#;#2+tTQUU+sW7o zGG&pIyj7*w(3yt;xdXIJD1IeHTG}c_OM&@he&@j{V(TjN+r2jYfkrzQr;Y909ia9u ziEZ_Ib}Gv!(?#~M<+x5peQGi5gx4q3<981u0kn)DZKQonJm19_=5}(FM8~hiKvb>e zQoV!(4z-6>N#6f>KHMJdWro7V*0ZkdzpBH6-Mg>C+sfPI2+Rlab1gbdD!_dA$i%nX zeP~90wAz}L3^IdS7A|ZjQ!GFHB%+EaZ{tM!Kp=nWYB;q?51j5naZRJ_Op3?=y4#>! zjseUYWQYTQ;f^qwp~{S5K@QE?QlY)i<Gm{{IE+2h{D&BH7YM|xYz~9OySOvxa+JLp z4`BvkdRx(X5W4N&)Q;+%Cs3!Ky8<2`I*Rvy6;~Df?5n=*E`v>+X4g>!&UgIvF(|5{ zoOZxpiWL=c2ppK(72lkCJ<HJ6c9Slf&|m8VbQh8pZywG9tLH{Q6EhLKJ4(G3e<rR` z(mL%+Hkg)5nU3D!9Xelllpu}Zu7w}E8<d)Bf4exPS#QR%<q@q*BB?hg%0lElRw#k~ zQ%qYVn?4)kgwLZ|?Qra5pEN4sNEdqSPg;It{;KEXY+g9L&4^8Z6D?CXicfnu^{D(w zP3h6xUJ>!#K#*`i%btk*#LsCu3+YY@^|W1wR$D35*4qu;j%)yhlHMnk#VPEuFH;J) z0!b5f;Hq^Iem9v}*vP>jkUbg3uFuV-pJzM-i_y9Kx!XIimx{;nJyi;V#S0&sN(_PI z_nKD!>jn7bvz|&>pA%<B`70jOGZsihj)~;e1lc<fu7K`xGRM5O1IvK%9739yvZTNl zxTCkvksU{TiZ2FxA`%Ew=<smTANP4V&F=gV_L!92%DZ}Z@hR^5JsWk$7ln7#ipcIc zJTrjQaIu7Vx3U8MzP=~(w;Yp_z)b6!!{qBTiK!~)PS*^OL@zax#&D6HavGZ*J--?2 zvY)0>o)eA=k6{n@vD24ZE|n`+WyuZlQ)mvN06DYCHDA>D%b2wCp&*Jy0*Q7hdoF^Z z#_)ib%UoNH=o;FH>O983Qj-F_gQ<ItWA}1jR1}olB&V1LEJ15%pj|i!0LL@=UhJo< z>@W3pvqWA|_62^|yd#Ed2RH&*CYXgSiC?b{v^q3by3S{4ro$I~1Z#Jcr2wu%sz=UL z40u70#O=g76Xch{I3bFD-0c5@p8hTar%H0EjHP)tb4;|#3%H?>38GD@Ln|ZPF_S-! zYf-xLoy2KlV6hN8q<s{ns6!i*=_@?XjW)00ex_9@^s!B;KW3wBsQr44E@+C`ZxIof zw3|fuUO19;>DJJcu6s_`9dfL#z2i!*$cG>~=d_fx?_7S_QL7O=YJzwNx2Oq|@={)7 zru@zL-ruNvH8$Ab6xDI|dyUS}&0gJ0UUx~asf7#q5O5=UWLXJ44URV1?3RB+8UvV# z&ffeF=nXihG)R=e3kIKV>Cr=kYZ|5FH2_o-$iKgynN18dPU39-g7x0jJ;lNv(-3!^ z8aUm_Dd>Iqo6?Xm^r^tPA`!hYNZ8LSIRl{^Cl@=N*hppuQiq3btq1COYCJLvjhmqg z2t}D+p{a)LbPRiL(dF_vPm>vO-+707e~&tyiJNAD{jjWgp*s_nO@S&tsdt>P3a271 zw@x^auWTfSRkfw{-J}3_$e8T5lST*stQg2KUXO^_Kl4o{&ED`G@|>1#nOKfV)AZJ; zh;jW=-<Ibl@-m~r-c^cNZVnmOSLwRhq3L4$RsrqIa?#YMdo22P?}Uug4?(K{lSR+h z<(p-m+Peb;S+j2Hurs&YH1+dt*TGg1i{wy*eBbmbl=+@z2IXga!mY3-l;GSn`K_!n zC7cC>f)|fmkBp$xQ5opC{zx6${+gVWgVOv*JoL^_r|d)R0&8wi%o5S6JyF96GIRH1 zpEYZa0Wb?ntJiO_QdK#BeIwQL>xilRVlUw{N3IHxVBeX`_*|Mk;Y7q~f`L2mft@He zyDQ(Vhf#bUQoR4N#(w+OVGMy{i%5rUl=(EGjw)qoH?U9EiT&2h%jER(V_P;i-SpCv z@Wgk0xRZsj8++rT0!Sf-58J$1g<bGHgJuh-X*9Lu2!2XwYuPa|#Ds7~+Ee`4fRnRv zxwQPk4kqiT*@6hMhw^N`;=sWsT2y2O;it7>yQ67Nvre<{C$fqy<<7z!EweSFE_DqO zK#p5}vCdcB5WZE6B8?HnIPRZY@3Dq_*j=WU5x6$)cJu$NUGLw8&EC220kpNQGMmu5 zIy1(3%3wf0?G^+HV+ylfwG9Ksb0xU1<p*gXW(Gf>-Z~5eYiqz90ogElQHpe|4gRE# zctZ}+7BRr31BiQjNndRbR;g`M`bpNz%k8Eke(RNdqZ9VUxa*+Da!zGIGnc~p5khRZ z_jFQ#U`o^>END1kT~LQNssyZtYBuI^mO*dWBpg@7*2itrgKtJjDc8}7X5VD4ZDU0m ztOIEw6mN};a_62hhQ+K<Hz%qrN-2=^letZ8|4X@{9PKX;pAvLmc7R@my{XzL)=`9X z;J{Z{RhOvruH3<uW1&TN2?9+&=y&u+TEV8oSmSu&CLLT`xlZwFHl~(6;DM|%Ol;r| z=`BrcVszvI7z&E%LK=QSb3Jw7A!TjFBd@{R@Y0oQvSy23fM#fYGWJzDCP|)ao@;hL zq1b>_XnNz1h?+}R6A6|j0ttcB`kQ;FCyyMBMiHWh1gxgw+t$O^Hs6kEdUQyj*Jh4_ z6xZc!c9=H@vyHne|CBJbt(xhbg};MN`l08`9up0w^nr?zWFO7?{_7x9d=6qTU<8Ty z-H{*;dO$FqNkLvmVD6aOt_5NZ^z&%;W@r7L`du|Pq!-8BSjaS{F{Sm7Mq^Z&?%xr8 z5B$G~oYff!uXtmKIk8#8FlPXdLB}v0S72?ix2vR~`|S&WZ)^n1+eNd^^j-ZvzkiDa z2<DTO1-cU;Wb9&%@Xu2QIdQGOncz!_V~!U&;5))@X`?EQvLm|0z?*zbq+=X4GP>;q z`Vc*htlxHv0Us%`p=|eW>-JD5R=nz2thO4&^yv!70uPZOhqr>1dB{2WrUGu;K@%@* zlR7xVzDE+KAoxOORIa`cmf8{Tg;!$3<bex~6*d)#2d8cy%s>EQNF}puT!PYXVi*$J zM+!+BSgO=vTpiq&c@&8p@GCM~sCmsC${Sn38+fzN^7|&@&adthz}TU&-$7dN0iW&u zt8`@dxV`CKVp&|9S-2mom!a+smR+ICqwDAzJmg(}^`m*s^H}p-D&wzMk;q7X=NAgc z%;We%RO%;;^dVrBP`eyLY1GHxXy|V<<P$owPZD4qa-0b7kpJPN&?MepJc~mmwo6NU z;LapDj1-0a*T0;QA4v|@E|U@O$YTl%yT$~LHjYn2oNMXyp&s6?uXX`Dd=n{~qt<B+ zDOMM}DNize?McJJg}`!#6EIrI;-+LfytDFs%6-T2zU*(}&PDSJaTl!%-AM{|JEeUV zH&FRaRXm%DAQlCxn0VrtvOYs&P_e`a*LH<>a;6XUYI->{0>{@W%|xU)G3WiCL!r*d zM?<;8$mWrEG{r^J(;$eVFny^f{&A**`U_!e2d8^(&Ho-02DA)2!J(Z8{dFy`#ugn1 zDDyB3ifz8ZRs&_LTGxw^0S+on1{C`3jX4hvAFnY|eNGbF8E#`Ti2D8HbYKAPC+196 zxf<8d)bajio$c>uOn6MQAHfH+ZTrkRDymb_X-$Rywb`}(TiIr6&w4<6Ed7OM4GuY! zOVk}xx{g(qyMOHish(3+j@|v4Di#3J!X1YP@JY>p12`+7S#r{a=pXir%*tDRQ}0%< z%P}m%V#2ASY^2P*39<tGOFD4>F;6Pssxgj@KZE%^lR{;1ANIJ9MnD8X(xh^?)pSME zvoSe30<?Q4AdRbXRKe4wzr%V-4eyic7xNG|W{C*T$9!;4K-vb7qhSs}){i%2(~W>O zcbA^}2C~a-z@2fJ?eBU^PaV`0+Hlh-gH4m;xbA)F0deRub_YR9v&XK8kmf4!H>LqA znGH&Rb0FH~efj%RnafC*bd4Sw3Qj0%mO(8?XS;@O8Py857E4>OG*{t;*MkrKXf-Tf ztp7;my%^t!_c=%bSJ~hbsz&}DY4TPmhr}>>^4HgY7&<k#0~P|jC~4Q-22FXZLWiB` zb<t5h`tIjxZ8sP~OXWAh&*SK@A#&6j)%NEu{vXe{T=GINJQT$^l)lf{Br%o!p&jn{ zIO7L<`6AWtSB>F;;S!i%3zGHdWq0JgY&E{AsJN!?MK>J#ut?iuS-^K;eGc!>?VANn zhTOUZYZtur;%kcY03+&;__ne8@j>(wpMSNm%$`WD>@bT%L!A8Y=OKKAuiNY&#*gm2 zyb|FYprY=Q=G;)0ur4Fjcs<88Pnv#Bl$6_r|N4Dl>uo$Z#vAp!t^1?EvvA}j6a|D? zD;;;oeOxt!19!(Vhv{~5`aYHVx^h%$UhRkg?fqR2vuFLBXaNSG(2;vIMz8&=0VNCR zUtVC(`r#$6w#fg-nXkYXX$}dCux|FO*lN`T{R!DCj*stM7Gt7MX32o;*x5bEgzrO$ zYxZ&WYDQB!s(z<u!^m6O^Mv!mny^wK4#$gDu801Vxt-BR=OTP^9SYs!+S)(cVJMy; z@j>GErMum;u%)e%69%d;j?daXj^UWXMedBTg`2N7tT#$IVm@=AX2R-N8^f*PZ#GPr zU*j7cT7`ETlb!1gPm?$0vnk*bb$tUEy%!r_MwMp~Rh}bL%Rwj)7@hs+Dz5X<tWS8) z&GVF1E!JyyHf7|VwDDC%Hy<K;N}`pfZ$8a)fzrZ|>7Qn%cVxM|G34l|j7awm!e-2_ z@QH8N^376(tP@Tm#hNKLo4;@1g344gvv=1+!SV684I*{g$11PVXm34vp3%l&FwE7J z6eg@Qn&gnv#$iQb*SOGJ9;gbak=5IUAeCf|(tmiJj{Nj|6N>=nBo1T{SMqC5M!Cs= zHra9s*IEK%tTX=9<dJ%QcJ>g@o0XRzY|v|Bo*%sgZ+w+IGBo;i(f8Pu!dUuv+J#`2 zel+O4Y674!LjHIf5prCO>I{eb;O8F%63yog5~MtO)IDzZy#I#)U}q^1BAM>?OB&NJ zW%`$*3T(89*GBT!OAo%~qy<dX`RGrM0z?323CDzz+S*wlff0Z#sLKA#Zu97`s+4+o zz9{795BQxkP*7je;@!6Qh6Abj&(KRsIWAqXlJa58FNz`E_rVmf?hpxtD07aLe@oOg z0Qj#>9r^(B7NWu1|HHoDvgC0=*A6y_Tm=o}Mco|my?P)qiFJV!(<g1nRc9<HgVeIE zF@hJr6S2Ia#s#~jzz-vy`HQ77fQAX!zHp2iuAP_2vf24GzGg3nMrYhP9vfX2k3@*r zG6`=hx4$@jy7H#Gn`yS89&ZcsPIMIhKeso7S%%!lAnC(dowR(u>vujVsy#W033*ke zl63$j`0oB*_9p7!CLj7FoVZcuZ_M4Vv;KpI`L+B8pHt{>`G;gDJ08?>7^9t#UX!92 zYIds%DlOqp^G3mI9TqPV%lbcR8=TUF$V!PVLRh3*l!|VRIPtZgzYlaUcnWdgf<M9~ zXV84BBb{k&;YbsIj}c=_L3D{Y1a^yQh%4-pv@INu-a>6$^K)#}AxE*vIBS!!VOHGC ziAFM8HG0#_XhKHdBg+Bt8Xh5SCaaGMq@&UITJNxgql(MjG%Pbrq@$(P7a%d>2A?JX z=1TUtvbutf|BcmdyJjlCG*BQJUBQyaG3thma9oI+Du+G4bW0vb7|(eOty2Gt(rR)x zd~i~@S74IKem5Cfs=OZ}<GlAkcVo7EuY5s?qS*8rv0Y&3ZBR#92{=8;4~J17WmPeH z?1C%5j3Wq9H15@#pJ!Kb`Af+}+;3B7lFMr~NMjiL3MNCHm|k7~TGl}#=TP5<_yQ=? zBfVUn1HFvSC&(e8)jq&so}vQZ!BVs=x{a4!Do$tsb)fnK@!P<Z@Bsd>0g2#Rjx485 zK)GL>o$LT1o1b}xCFb1%6+!sT3H`hf6sN`6M`57P;1{M3G(Lq$8sQwoA;V;owzA#T z9MpZ@6mX?)v#*U>mi=%-nFL&l>{pgcir1OV`Ni)P*(+GH5{+8i5HZy@JAM<WK<9UA z+DIg~(3;--fsFfo8xDNqSe+V*<Wxf`@VqbIH~7-3x#=vt{mDu`+iMvSh);VF*Znu8 zheq*E$1)gfF@y%6TY{0-Z<Y4~5uS6H712L307k2kj4}HL(1Y>4Ur42S{A)4f8KoBx znL%i~+#Y#=;k!tIY&g(6+L6AXrN8-Fon8d@w&4pJ&a|y-XQ+FQO{!@$f*OpH-J=~& zI<4BVW=5TxS6+-zz`NprV%bVzpuKc1C*lig(j`Jil&f1f)^~QD42$6Y6mz~$G8aD* z`xj5;qgk-H^YCn@Of6mzNIumanT`Q^O&$M(d&mA{oVk^rdkdXkrLMja0A%66+-U8M z7!_K70LwUH_y_1#$b3=p@*lhPmi%7F&Dk)%0++aMNnqx-A#qHsrdUuSzv6$d76mE* zR(@U2SE;Vs&tIn$YsJ_KUWS(Wc>{9r>3bQxu^KJSX=!gaBN*D=Sdn?bjpiYlYOdBn zJ+z_52lHs4ReMn>mYS47scW*|LbzK_=|}lIHXIRECfD?{2)szq^5_{4)i#Uo@(iCf zf*Y;#cESF-juNdBYax9}-OIjRwN#NMfS3$i?tS~5GGtI^PdV#9h{9@?J?n4&D-Hkn z3=t=RO>jowe@m0V;}q5GY*KD;MhuIvoE)%iXW|!|o3p7qfL#IPeIn-CKyTc6+=QBw zLt_TjjgJy7AOmjtK2nQr*ExFS(6`wkGr(Me9tAPPGA3nnS2iyApwcYRSB-Clwv+y$ z*9&lR`MWJ(v$JL`xiO1yiaKpe1wNu{j)-tWqd4u}M~*R=C7!=lSuBs=M|uotFTI{s zz)i=-4-~9ot#SggSpqNJ{1&i*O97qHWry0ybZ{AmacV5E3gX`mCw+Qa+P>aDqs<gy z8K!N-zxO+OVxCB0pP&I_`6KGNXZtl!aUnQ8pPE^?C=Z)mK>->(d+cx(N2hCiAdd-h zoP+yIL{!>)k(|9XjJTpW0uwAwZqwPaDDBI#N+!<#ZM<-~&upF~B*uuM?J64}h3>16 zt1=DN-fQL5V5=|JsgpQP2&^oxw8C9>Zd8i+;_&uK(bXf&1q_k?eeft~qd#kAHmXNj zYXwJTQ8T1T@n#-bkDfXA+_YaHkPJOj9<*Y*FbO}u`cuNXA};3MY71T1`}(d?gW^bl z_5Ortyv%)+L9)tbQb?V}YSCA5UqQEFWjkqe$gEyM4cI}LS=SVxg`#2^eKhJNUM;9r z9t@~iGzE|>X5$s+Kx6B*=a~S(XgDdSyiaal-D{_b>_?pkAUHN>F0*SJ`IXkar6>@q z>k~Ot`tjn0!q=DOf#F^oiy_37iWceax>2`#JqLZ>4A#!qTP0s+_ep07B^AGSLzg*r z9!vU?Q;veJ+0VSWOQ$}E9sn%>V`MRIs{io9bZsl_8BVIRfnk>=y!<VpH1bH5d`b@h zqkTCC)GxZfmLrDz)hu>*z`s)5QsMSsil;#@t6Bs4kYU`T;6IdumOC+!dpV#{E94V` z!6Szg=H6F{d5zx-FmxTrnn!_wk9E_l_1(|iE=wTaD7}a}#^^(CpkdFGUD=0P2$t(5 zN*(owmJN&=7ZjHHqj{7c@B*6WAJLTH1P!BBhnHP*Y!I3d#SXg$TR))48L&=e23D*~ zAc33;K*sON%S27)mNRGk$;JW}=JD7u;lb<gdfHO{bB+`(?zG8nRT7;ZZJQP6{bgVF z$sc*Wc<1v-4*SuJ95fA_D^u<rY)4}VIw176Mnzn`Vx^n@q&<R#2=KEA{J|Ajs3eG3 zQ|8s5uE8Jd%NzQC(g9xtgZ;+*1LergXBUaA_Ig@Nn4VdS(2~5D^7;h19}LkDr+^Yb z+LKLL7hH?FFuh_fZbA|1?8$M21Xf2RqYdAndMs&Gn|qmgao0V3Q*v0PQSyRQ#?~5I zgaUHViVhKs4;tIYUfP|#OZ9xv8f5^TdDFU*->U@Oc^+;^3<bWJJ9)i+$|!9hD!d-z z(a@9a2<XK#g;a2SWO{Adm^Tw-+u`EsdX0hp6GH*t(-m$7&u-A(ld9I<)UJQm4aE>~ zFf0e2@g1DV#qmiooGNtAooUycf22K=ayn1hAgW@U2!`y>r#nzWk95r=uig>?YwOjk z9y3qS1-Bt)5RUC+uvaH(crAsdT0xL{;C(b&D<Cm*2kW7L|J3;4qGW@NOiQZLf$vMN zD8=_mkSdMN5bnVL(ko>qam;xC(T^H(`q%?(Q%E-d7dR%+{#Ol%ulEwzFd(VF!C98Y zO-y2xRLk${_fQ2#Tc;e!*D3}+SHJg_&lhmYCcUhmx7oiLU-v4_uK&u2b4hENP1Ri4 zfdWul3!C?RvUh_#zWF=%%Fx%xcTqj(F`&GD>_;S5D=bo>sh$o1ZfP~BV?j>+#h$M` zT8ZBTH@3?Kej)wwq<^GTqGMx1Y>8Vyd`JJZK%!9NaL-2?<yfLp;@>W8lw5ZHk+@HN zKMWV|-a#Zsj$c5`;MzNh6prf+10QJ|CKeNQ%Q&uQF-jUEzeFsulP``(moYaVv0S~e zP~I_$I#N744<g-VB7q`>Q}I@lo*FDxW#<@nswy7QR`xX>XMHv@&TBiSMm2v`W`pp$ z?2|nJnbJ(blae8*2}Bk^uNA)8o`_FP1DYTiP&f5Fe=?D%&%gI*y81x!T0&%@tKVPO zqth>b;M6%4BJ_&?)o-OW>Kz9SUl2@O)1p(2$y}YjHjIN74n4<;Sk?Ivd%(p@%N5hn z>pZEAXBoXPl0mN7%F1zT|H%x=-;uGzOQU{O<3>PM0~kMeEZ9msZ&AVKf8fAQceTWN zjEOTg2fo-Pn{0NDB5TgIe8TJf44aCvn`4S*8sTFeVI$(B@jzB`C=a6amxrkfTqaT} zx!jKRlv`;$UQ_W}3^E^=-YGINQY#Mvm1koq{f*EV>psUOFG1!sV$Z*z_Jh^IKl9jw zC_wIP{Np)I^TH7uZ2Hh9y{$&hV#R?WscnL+WrRPRMKnJxH19w3nIqyCw9X4q?i*4O zAZ%Z7&Zn0py%G!+X#lF!W$qo%sb1z(pxMKj_^W3Q)AjOlfOu-^4u}uYUsafvdJ=W3 zFSq?JefqhmO(PeQiy?e55LDD)Jml&{^VTs)-V3S}L7W-fh%iV~yyehG;+L33OOg}@ zA+M63d`nLs9q32wM~I6ix$`y3*b^EmbekyGF_{3WXM|w8E(oV)xecnxy5Rnyzd?XB z;W2&DSW@D~J?fE6F5!{1>l~GIgR>Oy{es-FCMUY!P-(d>L`k1lP(*rm>UNS+xhjVW z!f#E=uCa_b$q7DP?)ofmFI9oNae`;cVV$~+ElE1DuqM-&r=8aCXw&bCb(sw6T#%>( zt_=gOfB%M7|7i&z`pz1|BJu<mSX;m7M0b}k;(z4IW=vPP_r{vgH!TBxIQiQ1gfa1T z8*xNs2HjM)097{C+(&b|ZQ1P#f#wgS{XupQ(Z3aIwRXBc+W(fo0~cipeH2;?F7+~i zO1*sW+$dX#>GhW;wgQo)bS5y=?KncwuUF_R?W-tD7RWQl3;n?z?56*lYxk_WKEFec zti>jd>;M<WRX2lxAxZ9~eGxgtZ8GQFhD^f<jq|pK8dwc|2G`OKga*F!=4$OSA|$1` z9*B8W!8ga3Vw}CVe2hGlC&tFx`YO<CffVLnM!&%~4Y_MWY{7Af*(U4RPQ7%SWM*y% z(3@`|LuN!VCrus_$IPQ<`Er{G^&Md-kpSYmV=Rhtp4hr>(KTpWK^<{3^XV$tz)?7` z5mEny53<>Mpim4>XZ5m??NJJEx;pld4Qm2i(BV<YP@m_6%rW@vQ*}%MyXb_>x6Rlh zA>74JF<Vt<nt;W`SJ+x1R1ZQ9Cv^UcalZ`BYHuE}S4EpNg`0ZP_L??BS1wcjtx%7U z_>iK}VAB=U)<D8eWp;kWt1|%95ZmW~vlyA<N%Q)7>@m0E-w=|D-`4Augr6Vij(IPq z@^zV$zA+9|H<w_Xk+9nh6QhYl8&n8fB(^2Bn^uSP1;?PXB<Q5<o}T9%YKL>}?GTL% z4bI-cp+&w_+MoBI=Avzy2>Q!qtuA}ceCeX=|LML#DQ-Fa<_?Nq1m6XSSh)*qT!iau zbkQdTN25t!x?SD!uapC4#FGWO+)tVltQhtAR)zFVP_b~{BGW&06#pfw{!da`v6q*g z8Wzhd4%WB)^Nn6)q-l0g%S&$xYQ#BF$V)y>Mml2rfC#Cg_-g9qIR}V70Phs?C&Vy< zaJbI2*pncBa6#A(DR-J!Tday4t1vJViS&$ss+xuRfBshCPY=vEq8N7Z-*JJ;X1*u4 z9UIQ^)@nQVf0NPx%U}ujH9de6^Qg5BC`y98-WdD+gLX69wC+ulnoynI=2!I5A0{9& zvu3g!%w$S_^I^oDKYtU0&C<&$^ClA02@D3~*JqH|(Y%eb4fHTx#*14Tx4jsMbSzBR zNe}%qt!m`L_dPV`9;P%0)hk{vyFUZ9b~zMChY0SQzO{iaZA#?R?VP5#kZbJw_p7I2 zcieLu>^R>;Fa;4iSgOg=-tUVfaD3w5cm8!u>iQz0MDsosOT!IX=)G3o>5Cy^6hB>a zO!aQ%qaD|dH{ZZhz`HgmL8UG7v>Dx}?_BWU$(ed2T6kT)B6ZvE6{%HvN6dETf8Y0w zKtTOBs8}F2UhR+Gu|Z&svD3ghuDOOQG4@;97~GCduY1Gat6xx|!z3^S&H4TI3n#;S zXwIi`=2#jBY!fvqLU}{-u=!IHWA|gP=(RoGACcl00@uys=-ddW8SHE{-%FKNfCU?I z5->&_YE@Dw?+EaSkIZ<m?fh!z^oyf^eFqic9>fWx?h89B7a`}RQrY%hPK%h}!fBw> zDmL;}kMVAM7FXmqil}}|)0Ns!@Rs(j>@O5W??GSt3@Vq3zFj80gl|6fqdxSO*4*(- zb5B|GYz0pwE!K<>6wz7fG{6dny#3?EhTE~fec$X_&4@xT9g6nKjQcoVLjp8@J+@LI z5o9&ajk_uO$pIv=s%Eq+Pv&1}z|`t`A;TfU<u1{Bb+C4)YP$4S;@q`pKUVsc*s9Uj z384`rz$fj*ktr|IOOys4BhBqZiztBZ@4GQo-aT(31}EcR7HbSvTN&AYy>!@(*dFav za9U5%O!@5LL>3vt_-*HFW!?0Dy#T`H*0}xsi!N`+673giSfN_ldLB3bK^%LG+b@!8 zq>+Q`p99XH3qRz2@IgRXO>eb2hv%QiF0O@c-A90PlwA<$I|&nshfB~WIaXVgQMYHG z2QwC0p*|)}jNBOH5|m(`Vx$u;|N66?T`XreyO~H;ZtD>Ws)YTtZWS0^m5++-pK9)A zzB9ewd0r`J1rkq{Jt?6KWhUIGuO5mn^ut|{(1Bbx3&nXfP8`YBV|`<br{D2*BO;x> zDPWg>Ii-E=|2*K^Te8!C-|GK;uK!iHDfM6gZHP@-WEJNqNIB+ieRkN(S$jjVBiEL^ z-vMHWE|K%GktS6``)pO&YAqpFMaa44137@Le(IJ6yK<Xs{}OPcCw{{O=4u~PsiVQP ziDG~oJtY2}wb3x}{LTXQ0fDl2iP6GD#oNIwG1I`&7dA0bYEJ|-!|ms1<Io9pUi-b0 zg?HE}uS7!?IHTn8V><GGEeSm7D3Pu-<2!ssB6ncm1^MVTeTeGrp1<egz>hs&3?AK8 zY-<mI@k0q3APaI;?sFK)%G^D!5jIFrYe09B$C#jL!A7$<3H0VM2V}6-rclFwS+(Li zNQ|K|cDzgFT%rs;fNB1iNq_sOVT`{KyHgb*|8-g(eEQikguOXN8cKzn7%7O^<%7WE z=$%vvINTLCOpx5h&vw$SeXjA(yak_AQF&<G!hG8~X^Ck_Tzj?7$X)3p6@(_`YfYz| zwry%yA$85zWXQh?0dS{!00CeUx!H})xmph1i40=mHPPEd1$H&7O~w@4E+A<jR?Yb& zo~6|h9YY_ES1eiQvZc@pv3ppejBh5(p7*%20^_0?%^T#D7%di0TKl~e7ESj=L!7yh z^!F=_n$3pFxf9D}k&wtb*F~g=W_78gb(-6!!M1Q)^`S;cdS2C4c=<_sXS|A%`n6id zfpRC)agmRJTF7)3zi--oV>wBdo4lE<>jT!beDPX|7YoGM7~au+3<^ymO?u9%xY}i2 z=&uQP(<+h8nl|b=6cCzt=tCiMK+>oSXoVw5#iU&qKt^1Se4%3^y0Am>UH#^dcbc^o zy|;CZU-v4+3L)LviDAH<-e7fx<ZkmQ*H(kk$=?x|nt4w=WJAQ?k^7i_dl_UzjpFxz zR{9`01nxE+XE98=zbD;9xP>SwUvk65_>SO;HwVQUu)|FN5b4U1!qtw^wU0jTwIM_~ zf?ft~TuILL$RDfl@9}GcicVE#%Nd`Y1m>`61#U*QGZIO(4zNFt@CQbFK4N#w3f|!e z(809yz6q^kgHw?sUfhelhD@4Qwcb!^P+`*!D=?AtUazq>rUPn>g^YfJoqS<EUOmF? zAI%qm_4bch)J}JE#`>F@kEI!E6NE%dlBmLObOncMvd3Md8+!?efB>}Uy2JJ->ob^{ z<Yw>5<OeOdy**Adk@nqyWtg$T!i8qs`~8$t3tS)!Gg-5K<#z1JEE}o-X>ET_<K57U zzI}kXwwiwIqdJl81ugDt@-az&WOwKaN)&k-yNR>4&6GHqGPCUj>KIYZf_w1TeXiED z!hgSV|L1`(lDh-5c;Nk}Vz{lbC$Ftq%J`im7r$e7Ia2gVpt*{LJw;}l6?Eh@sNZFc z;fMDX^L)CZJWMrf%V?$w+26I^EO3_(9yRmhfS<11%mEI9Xm;tV^mp`x3TRHX?gm9f z+Od?OEj&a-JbFr>zOjP#{oP`w#@b8Q&6W;64Fp^PpYqtGTwv_3_lL<U%G~SD_PSkw z?TNlvG}M9fb-&V-&42fV^_QzT>2K4|H*BXC_alK({xP`zp^4OjbAQpW;Mc7a0Qug$ zM-C-4YF$M&UjzkMB?$$A`fE{PQA6C;8ni45(ouRNkEo{fi6zmmN5#}SF<tFf*5l06 zo$7U<h=ZSR07sFYm!SB(JO2jXA$4zp6D!^7Wj##rC{+s@L1HI9CMtoD?MvI7Sk{|% z1J+Kc(BY^G6WtwGRrU{L9LJd2+MbWn7Q&CZ^YZc^6rdko*%$Q4zcC>Nil#5!zMV;k zDXxvwBOI-quqEu`vUEO41IORWBH`alR*?g*)^$t@1R7$cFz}Fv0ZtoC9$wx=d;@sv zvB_=B*Ol+V=~laZ9JRt%l@JuMU*%$Y2<(t;3+V&-Y803HCYP7Mi!b*4rB!qDfl0r( z=q1U%<ZfSWECs)2t!-+$8IKnk<XfGO%{PP?5)17Ky8&7LYwT}}PSSaQ*xOBFt)oOa zykMffxKq18|HYK${>Z}V<QW9}qtUTJ1GVX+cehcT<1@l+Qiw2bu6@SMqO$E+`DEk+ z_g(1)R;nquf|x((`k+(-s?T%Mj#|vFt!$}7WG{m#tO@RlsbHwKaiLMQWl{J_Q%>DM z4%#JXG=9#G22O(2u4Ro}IwZn=;MIS1GMg|`v;DDOnNk`J$<viz^q{@^RhLbp^qe0h zb~k?1UcOei(6km?(P$&Ph6y>e+wT_(IX-7aITCG@`KlJhufKD$N7j2Jgc|EQ62y8( z9e0H7HRkVc)+}Wii=K>Wr?PWP4)gd+=^Do^Iw3laZkAW^FAF-rGVQmJIOApf6SLSt zdopqJ2B`?-i<=q%3uboaM!LcPRDE6IPdnHP=w*0rrkUEV_b@Q}TZ9c2^qReFx+r&s z!TuHLmpE1{Na11=nc%b^zyyg<#3E@H3kxS+%X`fvhG6=f2MNFr5hs<D*smj=0{8HS zmk<x@p5Mio34DHSWKsYyqkOHmGqNM2IA(_|hUud;Z=t72VcQiijvPWHRiR?}4T5IF zbML6sD^6V2eA@w^w&zsk6kF2TEJ$SQ$Yomv<N+uJ!+l5A|Jwlcx1jVt&poRij6(5Y zjRT}0>!QsjVz_wtMntt;vBJl@B^QEe0DW!oZnttV-L-pRi^@&U1}obgUP7=c#e|27 z9d1iK{u~}-nMTgE(hKyJ09Lcxk1!KRHzRa3NqIDN3F9}|N&P*4Mfq=~fho<>504V! zhTT?Xg}u&P4oGKvRAP1zQSLuS0Mey1_xJasaa(ADgp<*?NTBGE_HR+bm(B6J=?Z@E z*8n4K&V@KaiJW`1&`hC2q6ZU^7_(cwSw3M7_@*iEraS+l%%i645PD&ng6D#%50|?# zsIK|AVt$bs&mU$(>}myR?^R>CX^ts7E0F`-jG;zTGYF`_vHY>9DG;0bqf=~_V`aY7 zFJX<sA9~jc0mP#{IfT19ftRK5YeQ=U+bC!mz3SPw(WJ9%aH;9vhd}3aj9p1?k{Uij zEoC|hwm9&tt9Pb2v5}eiMZJ@kYYRE8rFe;#-tb3O00`enGL=SePQuP|e{q(I!&%#& zehP32iV+u@PX70A@l4zQ254a{ME0@s^u{CHu_}yDOjh&G{gq;_g(lOyZoGK`DXg%e z`Ry>LRGW-44&Vo9jvuLO5gksDF~2?uB+Ux8Nf|kv4LG{BB3M#(f}5dIA4j#4<aQJ% zWoOb$hFm9SpEFE$>esm2*tx#cA5K^L?jn+DyvHwFm}*x#NvPg6B+&U4=kYIvYsXeT zxzlmGgT(Qf@>f6qUP_MR=r6xWT#}D-?UkJ#n+>2g-btV^7jKmdNLAh(eZ9AfDELIm zSv@DUt<ZP6LX_cPNQEZkpVV$wCR@sLyv7`wH=qu0zOXcBW>a)tk49~Qx1E3%=Ef{a z;(Bcbw3vWH-;Mdt+ounbfNCpy??1;}F3s4$?^Sa6)aFs{V;ZVN&>e@^LW^?#B-93Z z$HMutt3lwD`d>}gY1cSeWx)A3>1UIk210;Ti@=HzI0}C)lpi9dD&dmQU=!e>!K&Ku z)v+D-#&Lq6S|gpIQE$&i_A(@nV=Oz+p88uvkFvn>#{>O=Mq*coZ6U`UtlTYsg3QS^ zT<+%MBD(TA@gvE^cO#FSDURGyrNy@PEbiuPnrRTOvebOLL~(pXG%mA~DXunB(F!Eb zH2}Me{a<Asy6MsKi=Ou`-uod3n+eYHz9rB}5j)irU)?kDI)xcuSi8V$jwOUsI94j! z5mw*#=A{we0!p$jNAlbRXN}Xe$0L1oR;dcSZqFh#sGR5mkoO8&3o0*vJonMt&nINd z>%FxzdCKGD`DZ(f0H4nrwvd@=Y3GC=sVG1{fcBu(sm3owfHl$Dvd_P7mH&P6^Au_L zVyIzX(BdS(=q|i-r{ZxOoxjuVQsY7v5Tr8G!xdfRNZ#nk>URyczQ+VD8u|2X{E<qW ztM}<fz6G#)&$D5%QG6I<U<w$51C`>_sr^`_o5o^+mEM_xk1b^Jnd`>ob8V=lSQv%h z!R1iGj{fK0#POkUedEuSJWCIZW9|TjYaC2h<3nE@%!Uyl!a{pu4wDbWT)TCZv?rB6 zzrWVHXpu%Wr@Y{|79~D3r04$Zs7{o+|A;-hXWO%;=5%6s7;U?i!#4>`3>GJg@uNO! zP0p2kOc!w&ByEn$pEN$to~-JdZUvwY5=DF^Lcex-V*0`(K^*xU)bY2DI>nj0{%Fc) zHY@fi<<X>@rYzgT2<9ua@XAg5K~5E+s)Y|OQ{KzZDm;QWesLyu%RlhXW|$F@tpx}u zof<>EpP!VxU9$4OAE;|_hq#{Jch9P>w52ouCR07J&)UaD9G9G+^~wS|2aWdeFe245 z(l-IgOYV!}(m(Ki!kI9EoFHjLoA>73DO=)`?WLB8>f$KCryt*dz(buSnb|{1q6S1~ zDW8cV!R9sXOj*psEw!4fZYI;Qa|T_g4Aw>q2GdQi8h0)u+RZm}08&ERr|(pkXT4;J zC#{)Ntj2Z98znj7iF6uVf}_&rYFre8w{(|&F#6RMNee_v$VuT9eg<Kgo|}E5?)YY^ zwIxJ(9YD%H@c-QSIaARDG!D7sXF%+PJuBypA;ay~EB<nt(?U`bTaEXD7Y3c#89<8J z`eA>bGpf@#G~&g`wF8XudfW|c!_Ft8^f;tSV-Q4s6dY^)=3HBR1KpE2wcXM@Vw*O_ zYxYwvpuhnA>(%c;#1vDi{?lSJT%@w9iMPrm@|jzjG<sa#?r1swdZJY>?tJ@PnmXX9 zrMTXbrILqqxedd;#UnPK?zv^Z55N8LjpZYLg2bi*<i4J5Z~UzeK~fn9<Wp+jsMR}> zG{ZmH6vU3lbWQi<x5t2b7+upY168G4l+pHXTC561yy9-6)3&O+TE*b4c@SNxreB3+ zD|Ah_R^i)u|D0u4J<$f>r?w%oBlXjb1z80Z=GP-ZA|`H)X=5ovOH`-uNR>%WUSD0B zO#VooBs1i0@>Cz)aj3OO+-Z=E-TKuEe}{)$rnH8S5m~)_rAj{nx2s&J`HLd9W3ONq z*RLpN8KGG-x<$vIES3qkzzqSS$Wap`Emqm2lJT*R?7Fu<WJB-ffo?9F*aupt&`pXm zZ;MTT1XAIbtPI_*D5aVYUImMOXDxe(mWe-+QiTT``DCDZ@!4Dcre`aCvll<MX+2He z)RYh3Q)t+#UjGR+(U|;sG%74(>o$YfhQDnGy$QYI+kW&z_;{=&t=-(6wWM^JcYArI z8=JSF$0N{OTIxK9<UHiqNM}6a2T3nAgyg#^vZV)<vEhYA7<>wfx}KaLMY!=l_oDTj zT}1S0q`|d7Cj-fCK>&Cwv&;Y-BTpIU^UoJbQ9I*~*7ZC8{eCHu@%58qs3knC$rGD% zh@<k&&DZDXzw=PtD~wg?s4^EXlx%0rRk{1f9a~J`R_y4LyL>q2v~%dhzYQSQ66Cy# zzn>wl6;cJhJySmS{oDTucnPgh3zbVVWX~J2Sd%+)HP#<&x@FqssU^CRC<=m#83wB7 zj{edsR|a!5ty$0feg;vbIxM9~yZr=;3x5LSTXw<fl^fVi4T<qv#y*>6o#WA&BDqHZ z*W<^U{89qq=aon7zrUUHMKX|72S5(2L)eeAUot+UjR0%KQR?_d0T2SR`d46OnYUH$ zL*UT$j}Io?L@*=%JC)O;ofsn~IO&U5pvr2H-S>u9y=xJ&A!xT#&1Lq+kSh9Hcs|L% zH(SRlHC5v^;w$0|KYI={CtqO?+lZzH_QZ(pDNegvdK6p^Mb64?d5h&<J1RZoa{n`P zlw<RB$?Zs%@9XycwPW*_ZO-=jy^+&Hezxa}nqbo;44*CC#{yGc?f?Dc8w0WxIY5R! zC*wbqQuo+jJ|XcgW%^K8-i&5cr~lDSB>W1LXe(C?=mNg{RN2I9{d???P$jJCvDC0{ zfi!N}e^w6kUK~GZJ9EK)Yyi=-G;+BoN1CLHj@im7Z$DLPPqoR}$EU{SjE`6%Mt<*L zCASQAgzaYD$tG=k7x!}MLEKx4?E(ahbtyot!aF@IxuT@7uOt3=HN_MLztU!+EZ=$d zewZxX@snlkYdgYyoQaT~K!K|(bBTc;NT*y?XvAX1!s6w0QNWmWH+0T_MeLZ_l9S9U zWBL)t@SY%sWabI-j&B?;I(r7e8qM!=S=YH8^qvrDCpvWOb6a!zHpqMLN8fWFo+Rvb zluiJnJsY{zK$XgJ1L@s8_#S`j2wXY*=0FqQ^|vkOVWcIyJ@LY_rQQ()ItoY@;dZr0 z%Uoh<n?VTFYeb2)A#=JiEi>{u?O|iPo-74qX}wtDt)5#SoO~-2QL9tIfIAUxxw?50 zaGDlp!g3x~=na&&<Q*C`kj;D&33TxBeJsj4XzqxBH?`j&fk*W|VAVY4Oy;aqY1f)i zXi4*2N$^RxsQlySXaA;hGhlaik!RooS01*Fb(o;0#*e+v2K{m&xEj90_e-|d`K!2m zc2%4}0=z`k&EMB1KX70_{G+RJvK!8=`pEm~4}BT>fL}NBeSHmrY~`cC9oS{aj*l-U z4B*n8I&PjMO>V39L3ua#oA0`$s(wQxL!Fqi<qw}c3tE2;TDX3@yAF6S{Qr;f19dRy zJ&b6!CsFjpWgyuJfKt4dS32-y*pIKE((>?cZioq}W5zyE&s?gWpjG7bv>5X$k>6v@ zU@=V&;%iEpJiU|kSd<Q#j+9@fzk6~LD2@IJtOJ|LpeXt{BZqJI0B}pu>DrHq!f#(- zR&><-hiXJf;qDUP7DDvO@509Gz(2fsDr9o#S3ta(uQcVJ{7FY;(K4LpN4=^hv!3&3 zi2%Q!)`xuA3`fnzAi3eoQrbQKKizpyN{lgRp%w)9Q>!~!O3w7Vy!dn_*{R%+S@NZE z2+Ql)V5V{#gA0Jz+zhyo7_&9gIh~xt1<J|wja?rwqNo`(Q>28cSwylN5P%#(Wt-yq zFyydMFC`mv2A*=m*%{?8zI^O21A6s`|2uoulU~TX^GwCBWRN_ng2yu1R&-w%rKyu3 zd%q=wuL(A0@#;Ut&&tqqRzyaTpyEF$Nmg9C&ra+4&7%t@d^wJp9Xo^3_6ns{FL<u; z4s9nk*gfLl$UWyBuSpK$r~>gE)_pXvXeCagfqN(n&r~IPC-$u?mr4T`rZXl4-O7&> zm(<D-VEtGlh)OB3St@uoP^*?$s0AP0{)$zLWSaCXt^38<k)*#!_&<MMMp}|rU@M`P zoNsy+a>2=YObJyq2a87AR0}!^d3g_>Pw)El`qFMIk}km63Je(n<B{?^H_N${Vg}Ss zh3Uf`S~Cs!24jfpvc{s%i$tD2PVxcWKuWl!H!&T{-%CG$)Q*x#C-BxqK)Zc;|Incy zxV*ba9#!F@rYvad(&Di9<|n$W5SvfDCG8j_bn*gD=Cm76;5}yCSD?+9Nl~dJ^~JZO zvQN)DArm_hF^c+u(Mm!Bu_YwXb`gDt`Ya_6;4mH&;FvT&!HO-rfpvHZ>9({_>TP@F zImz5+5e}kceb~s!h;ZI<-xK4#Hcgx}jUuPlN`WDnmO=fO{0FZa!1%9Gw};lj;kh;- z*u1}o9n+coegyW5F*+W!K$b24kp>zl#MMZNw+mra-AyhalVkgS)PKFx)Bc56V4>JU zG17kDmZczxXjht1m?CS<icwChXvFWK%Uj3HrC&f_%4qk{B1ed=ZEVD(o-WO;XltQQ zOL6YZ#Ob2B6-z3(si-ZarH{a72C2Z(KC{Nr>hrVMwv;xC6eb<Et1Ybi4><i?E|mYg z7My4Hc5FAe%kffWnRRRW__$r9ePH`m`XOAl4Uf^h98Lcjdc}#(tA9=b<qHrHf&iLd zu?Bn2#jlAEY2!OZA=T?eOVO_t9vw}b(KFt0Zl(REup(rq(8u4?{vhB&Tq=POSZ?|R z@m*;~uD-wdM(w(ANNDD^7c9|Z41&Kc&hnAevHk3W-|`>Z*0rG#mqLyax8P?$?Mbsf zCFX4LU7KDJunhb&P>3@7-<QyTjuC^b>&JjNPjm*k4oa>0e`LLdTayplKfKWmUqwJ_ zU;)xd=MWVHky50)yK}&3lvEI;K}l&C-5?C4yGx{N)EJC-FP^xc`~JQE!H(nFbzbNB z$%B^0pcoCF9z~-2YCyEpF^DbpYhM|(F?^Bp%o-e*q`hQF+#LVB2jxw%`|;G-NTgQU z=I(%Gv0v$_9z0IwX1SN|@1}3Z!&qf&ZhYRz-4^V=3pW!vYktry{fk|ETP7B6{DT5^ zN~x1LF+)he=0*9`wG3@_Nd0viE@cRifz;*qb(ZL&t!~3CtHuaDE-vxF7W0KOe4@qG zib0AdbV*i$*Qy=z*GNCG&o#x!pp7#(MSZz`BZCxZFLQmr!0fyG59rQ*0V!x%=5mFy zbldl2Q_Y-{!0R^T#^dW}2Oso|WoK#QzK-A8@8ygIb&k#V@S7E;YpYl)$;(D)ufgN1 zvk|TIl&jdFs6y>`J2+;DaI0{_Fq=w!MBwwlLeP^5s_o;%{3|$F4u|>Uy)1o7vJZqE zs|FnHNP~h(V$Ad^@BBKjU+9C%DBMW>tOc;_+u_2w-JtStG*SZH#CVw|kXSE0{}0Mp zG4tFQe^JZ0gg}s?_$#>x?R=i_xRlp-2eKpqCS4X@`)5LPwTap@cO6X|auf59pL87* zkh@bD7BFBjO)AO9(}ct=w>uT<@3VjVD*OfYU`z_m@a;+k{KSL^pXWMhJPqkI%o(zV zT*%og6b83`WAFv1-cbB=J&EdSPy%Evr+<VeBniH2^PVflu?192;mAIw1Gv%MNW4GV zeDtU9FaG#^I&jGfu|UNOAl3G`F?9PAmq;AGVrf*1hEq?F!y~WYq%-)CPK_e>kqhP< zj-@&c@*4rL(Z>6I>rP~(dPb08HG9ZEW$?Y01<d?V){&#LK$rsdR(J39>G1WBMIB*B z?wE}>q){Z^%vu{#2Rqr&nN3S9iCE`7=}hTNgp4>>*d=!JS(dE%Qyy~TD~v23zQdkN zciG}P8sfyANx^PVSHZ%rvTb4&!Lgy#v?bMSH3yY&-Ae<Uh!sa$te(`n{+_|Prkx7k zj&fPTTK}kq6+-H;c&IocxSL$OHdl@uA*H-+0F7LtU+!&x_u8}&BKa`S@*u22(u3*~ z0$4Z-5l_@xe>ib4|1utU-9i9g;DcCYZneD8O%iJ=lI`b*tA28Hl9|Ri?GpI-T)S>6 zC|0aI+BHFLdX!EH{-h>yr#43fqA{;>B&2s_`TAfw!!V^*kAbtOK{FiB$tFehCYwcY zLLPU#!jIFHtF%NCZQ?!TnrX@Hd)ru+9nMS-$dYKTLVw8;GEu)3-+FXP^^uNxC!Ke; zigZcd>bjsEIm-HoqaqkfcgmX9I$Ec295bZG9d@)Koz{2hdM!D4_WK>@rAT;Q$*I<C zFbFQ`6)F3EzC$VHxcvIT3(~W$MeY8X4J7)$sKcZy@T1lq&W3zsLQ|!i8Tx-TmGtSU ze+r*Z4zSg=+n=v@l?oH_i$#D#glR&P0#YX{$3Oh%U@1p&fQAo&k`hgD2#8(Sy`*9L zr!kVrldN&KQb`L~>{EYwWS8VK!C?!B#;d1^{X?Qriohh7$q!0?XHOwGRj3kdqai@_ z%;>D&kde8xVLwLr+WD}H_QQ3)L6eI6Np`fTk<mzAgHZ|CMFSO66Pu5x6K^}dB_GN^ zb@{ewKvzD_2+g=q-I9li6SfdT6S<d69wgZ#bg2Ca)j(#pH11aiKu*>QpBHNR_b|}2 zA-eTzWPC9f)FX3!7~2UHr!xRy5MO8Cq+)Nb&sxKkMHUbjyS__UgX$ZOsmu3JY%_PQ z4F<I`A&;+EDq~O&a$$<A;DNsKO(z`!Tjd8D7ybm6jIVacp(YSpHN)?Yp~XsdtSWjh zRt^*=M&P~{l=>yZU0`$ld7LuHNpw;SWVnEK&9)tHb--@g@@<sgh*9O3QNq?wSH#8t zUho}QI_xfkyh&c@ZL|?`=jBEjmU<xe8Y)`%MHN;ptlRpQMZJ)38<uS1V?XFGY|e)U z_}b@tjEljtd#7=Q0diGNKmA<&#rER&`G5ZiOsTTmgVq>n6n(gpVDUA3$yb2D`t&fV z-IYbFK%$n?07SBvB*0QOeXzr`NxV~l3z1{O+zH^)txU|mXTJ=85wl-7JL3Gsh?Wun zSLyx3T!sxPB^6FPX$zaBdsv?mn?;-k4gzLJD)jf^XmpTBxQHN?R*Egco}A+I?-A)S zF5^uZbsB*G3G+~Au_8D4F`T-Qi+QrVI=uo?*A%gCIkfv~(7NGDAAr7S#uIR2;hB`N z-dD@~7#)Z=)5RiV-;4r9sB06;E{R0`uNJ^ROWLkWi5Wm%d^WPEIh$5rVddOU5;s!R zhPx#Q?nv$@yUt1n;tVH}>q*oFl(qOG7}lA>_vq15Fei5^g<TsQ>B&DPW7sFw(jAtP zSd&bE#;xSlslfZac)lqGW?^pPvwhfY&SCm=$#k?X5X6~~@5DwpuMT}z2%#6%B9u<N z2)vH)B^?MqbR`b@x=s*d&L#2lo(e}3;n@SLBvOc+x89WV<@wN^g6^x)#u}S5<JJ<d zpMjrXi`KppC!99ni6!rF1kV>A3lCb|WpZp7{FQ<06@a<;HKnm##mtafop!-LXpYCh z?KF!Ev2EjK@?~z!wbbQ6yZBa%?tWy|@<8v-fHS%P#qC2)!h_Q&h>SF>93y{^E4E=3 z_OX4f9!s}JMN2xphP98f=Wr@IX_BwfGrbXmadk<wcxL-*m)|@2>lzuGE|_HfCeJx? ze_(5c;GGD2WQEGyoGQbk{{t8%3=!UPtU7goa$Y_({n|cUcv(PGop#>drFC_|^@!M1 z>VwtsJ(>I#k^Q!!akFRX=e0FBEV@55TmFS}{qH^F_8kYHG76c8ha?Upg~pr#uA#HG z<+KD<UFOFP0~&WIIZF`gT0ja<XMUzX{UV?%+HM6@hipx!le)&g6TFixy+v5&midv4 zcSX(SJ}&GXDOcmh1z_jO4;z%i_!`%e*rLt(N^w8noQV0lD6Q)>x$KNVN2uW^N<^pa z@_C_|%-*hz{6N5{-Ph>3Cia~4lzOcqNs%IlyGA8AUci+xXZ#~t*v66{3J1=OFpe=G znHe&v@2P*-0=e3hU-{5_@l{*iT~5ApmqYpm_uGMAOgnqX_zs(0N@oE7xF6?t*X(-n zCD=^V`UF_I4)}uAL1uzDEaMMjB?Y)@rcJaxMnvlteRe`6_N{!KUTWs%s@mhTj0s<= zah!gi`4TI>T`>M9a`#FM<f8wL2wwaI{E9N_5!kShCHe0o4v--Ckn9Uy*lXiDg9Upl zE!W-gs0_q{yKjxz=s=)TdxrRZRo9hO@ni`#Ig$lmBx&EIeJh2Vl2CYn?|S6Ah}&S! z^925zv)LDOZw4;YNr#2+Nq7;gQbT^O`ex9@eL1f?Vp}c8DI92^vcJ0;m(rEeiPX$9 zTXfvIe&?0Gl=<wS;2;;nv`2}*tO2rUm|QEpuiT}wNm<+c%<z+znMg|R>c18ht+sm# z;JB^6TpHMe-Y=y4Ci?EnrUk(*3Yoa=&zb7|dc671!er*DU+s&0f3FDlT<$V<6wYoy zRS?o8RPK&cY@nV$Rp=H9&y?bQV!FgJ4)^dz8U%&@13wyPZSLO#YfGUjCZzp(nJ+%$ z@GGJS5aG+gLT9=Cu9M$Jq&3);N9T^jnAzBKsE%d!y4<%rnQu;U)ZH?yP*AQ_W3FJx zWb`V9jIMOBd0mS-jhOAsDjY%ZSvz?t$HSy0r%UpLqt7$9I&6-rD7c#`<LZt-x|@>2 ze}xgT{soW18(VU^v?_5&5bxBuMib@S=RwST)-PtwlRdOL3kjC0Kj<71^?ZaO1pVz( zsq0B0uorO7M+cYsQBNO7;S2RwIesyLbU#`Q0L8IJJ}51jG@$GYwveE(eVqkm75b^f zMe&{2*qeTe!>qI&$(R<^^}?EB1Y0d9JS^NIA3rJ5AhKfmv_109G;Of@mghL(Blx>2 z+Wp*R#%{H~r23)P&LHU@;Xi%>**wF!JB-|A?U`OOle7|KLHpUNxTQ?dmZQfncfAA4 z?rC2BZR2XqyogKus5z#ji}hv@LsgvY%54ao`H4Rn4W!x>8K>!;+B%ze;0SXuj9V`j zZq21DU<$b-f%tP<W5t#pVB@u!NL=#%`Dk+0khBEQa{j|z>a)p{YH;gNXzb&;t(1Z; zEsj?sesS_LeTG&{f^TiZxrdu6Gk>6}5Nw~fqoq`i&vD%!%=eoXY<P#*vh2xvJuwyE za2~=HPHQ-b(2Sswp7M`XrIikAkQ%D3P>y#ag4g#x5D_GX1k1>xtuU$ddU^+gm%6&( zIE|*PdENh9R8*-qS@3}{FNXcro9k`x6_6WjKSk2UIOou4e_O-EiJA5tND<<c1SHHW z?SznGI5Q^MkdV;{3d$<99LiX<p-slV0C}%Q&vm5Km-7~UBmJfL5E`hDk)GlqeIC}3 zG*|eT6yee=VQMpN(zb6rSv)kmaM)2r8D{>$>4I;l-xjgs(k~7z!uzMHcu0)_&;I6o zC?HH|QtO9)1+!E`IFL8R!&N=5Zc+OxhVeej9Dl11T2J~-#$}pz%K$mkmOO`!b5S+f zf_1a4=kjxK94S0)^ZD@_8sKhucAJg|-_PA#AKWkxN_j4R>{n)RTa6Y8YrWcYd|!PF z#GW3|8p711sPX5*$p%_6UA!HCB6&0HM*DbM8EcmOu|Ma9=j`s5)~~1h6;}+5c@Hu& zt6NT*#E|YK&~PN^RX*<for}}szG8}33Bq|^&_fCtWzK<WjWz=#DaYUw`IkM6w&Kk{ zIKcv9R4j<zy&C9l!+ogfYT|ce9+*C@!H<;pJ6)|cZBvUy4p{eR1|ne4bHlB)L$m(L zlND6+)Q#(k>;Cb?&|7#-;WQv3a?Gqh$W4emxBT7IhhJ-r1!x!R5{2lMTdaW>e71OA zX8mRg${993LB2C2#U7lb5*?u2m60bfq-2V$TIVJi+Du&fdJuU_u>ALj$Epa9$I^MA zkekHAjpEI-S42N7WTt#^*H-4<1H0L<aaz~xR!Lu$A+Ns^3<sUGB~rZSK5*F`b3@_+ zRL7WA6&^IlEd2!vbmJKKY`PQ=VoD%G38KOMC!3se=iXIDr>Z7I6#7O88u8D?1KE%C zuw8t%Eu~$-#%#jbzpC_`c<u^L9|`qBeI<FZauTy?3egfA+6SALUn}o*FAGq@ioxO) zb?qC1(WwRzUxUe--f~_K*9XJtt7qa*=?{j$B0^S#+ff64Ix&|0WAMId@lNo|A;yF@ zR+Xmj9>3az2Rt8H5p5WdA#zS%S0F#Rf01JJ-w5Zo@<B=Abn~X*Q_ckk2d*qz9b|Ok zl$rX+{gG&57UxMJpM||x5WU0;2*9J0Zxr1nxt}A2;vZ)VjAyiw%Z!6_E+VzKMqo<8 z2g8n!fZ)E0QP`iWh`rZ2LXasr`Z<%Vszjy<9ok1)VqJtn7`pAYP9mz!=)F)c$%*bS zv8H6w-7Rvk7^xvuNY?6$GeIMXZtt!02Q{8q!a;^Bi(fAO!KdTKXw5!!Mm<{>f$aw} zjb5i)7AK;uBzaJlkZmMv7xN9}-F9}YN}GN>I=cr1_cgi=-KqKz8K_BCVX~oHHb|7& zgfX5rJ=X7hT2!pQz{7<1gXtZ}RmKzYW2Ur=^XI(<$FVDQl6ggBT1Vr2K((b0fIun6 zFcmzS=$lfL9RJvDxX3ded>Ac_9KMP`bM+zHaN<mDS+D#krXOB0%la5ZJ2-^!diNtT z#!P+h9aM$Y4CY;DfVV&U%jaJ4knQt=Tz-ZpVE^sE_TStKX*mv%X-*W*<e}wtv6Vk} zf&*W^`&KkN$ff0CqHJ=X6&?fP*Y`hPTLJJB+Yd{CQ2MU_QGDjOgQf9LD1Kaayx_4D z&o1eyATg_}k%)+EIl(9Km4$4%a80OGD}lojub!|?xS(jtSz*lNc$Qw)dmKce!2<T> z%I@UhlPfw=C!=rT?luz5VNB)vqQ-yUuP0nw9RwJ5QEgouCoF~Kwl-8~%cnAkkH&68 zdpN{7rPf`2)V%7&*6>RN<vBb(6LB1y4qr{1#Vh|@UuL<p)fO02V}84G3h-xo%vyud z+@ttZM}RIY!;XY8p%T#BzFz~R;?%FA>A&j9a}3v$wDFc@dj0wMb0m$cSG5${Ymel@ zhnAA6D=yo<ef1)RoCf}V`muTHo4^A^gX>ODC~rGONDuVc<_C(XZvdze<U2HAXFW%{ zqIR`y0qyFaDNN4XX|pv7(BI5>zN(q0z3-_1kMt|U{^Z7b76%>}Ma{a!LE^gCepCMa zD2;XdTk`gU;-wvqGywy9kFE$VUwXU&x2uT?)AC|aZ1Z7NjDslJePvY-^4C^hanfwN z*1a;o*A94~8~dwDw*Q(PT2%KG&aZ@<a!7db9;zmc8dBirCaghUY)6(9Qe~Uzri-hu zSlaCX><6M3-v12sOEAG;UVdS@k&bHO{fg;k8DU;LRG2*#r|OVD_=quV;v_aOB9RGZ zhl$+OggmnP^_xBFDh?jXh$51GSUiPm?|}y+<IVPMC*-(m@^=2SfTa#9n_$7(DOr%x zr2OOBGuY*!Kl$H?uJo9&SRupFwudRrW4Mq1ZnIlImVGm+g>ok_rAY6bz2KbL|Cr<^ znTuH&Q5HAqB)xu)oyFvqp3dK3be2P73l{}RnQhF$Jz^Dd^x+AZzKvW$=`aEH-5>mg zOY}?j8t!D!^;II45!MaHk(6^fwBQxik*hS98AC_*d0DNHayr0=2<-986^_4$aAP6D zx5+M<&Zgo3=+$G6kPZBFAg*aA9zmGZ#=0L+TpcB+=e88$`Wq+9TQUpIv!ttlwwfv0 z^2ZBYnr4MK$xMLuHn8+tGc1JCd450Ok7Bhe=myUA7dvecQZnhhh?{XXQ`?l9*hF;u zJb^Vc3imZIl5mlUtz8+O!vQ6JzINoNyTSECuFAQVB<w$=AgWmjw(B#o$JH+)CEiI; zK#qeS6gS6CLP(ldy`}<Vw|uBlb**?~t*t0S&CahWDI>FO@uV(^r}(XTx%Wa}QHqJY z@!kF^T)TRJi|`Ick1QQR@h{JSDzVo!)|sz&{1v^$(S^R6;-1B)92W5>5<hWRZnjk3 zPty2vlknc;i96M|{s$AQX0-C&=yw8#F%aWsFF=ak9C_Hr8+Hb$IER~k&nUJob6&@H zXj`UFQ1}q&z7<<b7%tJRrLgvtEfOSQF||ZQwR5EBwpsEB_U~oyZJ1Hy2_GK)<3%fA ze+zM#9NeJ7WkM!Cq|M?u=EHw6;j?bUq~ndKi_VRvehrWMk)Pr7vtZ^u?e6`WxwTF9 z@i$fBM~cji$3Q#Khn=}E)#7V^QslIY4Arl`Kyt0RcB0O6e*~0@?G;$cHarGq0)C-k z<d+iAO=<O{&if=RdXE9HmoJ`AUa81j3F-x0olY=6Z)g)P=U@B26LuPYnBddboL&T7 z;OaHn`%d=wSO)RSU;Zsvi}Z5MAS<w}P*t5ovKB~yc5gFhj2sL@6;O^3{7oh`oI2Dl zU&1Z#dSCz`C0KG{=V=l5Qfq%x^ZuBvK=T!5UP+R($=*xlz0=v2@($!U8&^ONe(gEt zxW99ezs(18RY)b;x`Ye$#Dk^Tc<L`$T`DHqx?Q@gLSm&1mUZSt+BOQ^z3l(%EuZ<% zw|p~d$<db;`nt7&x278kI`Eq4T}&f+Fp~6fCDat2i8{jRQAZ1B6b|5=WBR5>Bd-l2 zkjH<-Ad!0C-u5kiHIdUm@!Ra4%5sx3*qENm;%KldYy>)p7K815+ibUMso#B^;WGa+ z4wtz8B6UVfToVsO#kgkdP@Mh!#WZl%G?V$yazvn=5kqF9Bxwfz{I5U7co1Fx@Kd!- ze7l5hGNCVm+1D)Q(&%o6awMsO8h6gO)ns4E)W-;A?0&kOPNI~Gcdy4m-QnT!N%HoS zb|~Xz^vw6Ij1+v8;H8pl6sG@l{XX7`@3p69ad5|%X@{q(Ke}F061_!+c7)S-DYtGM z#<iYhgL@<?4#|qkGtUn=ujyi%0LLC7uENYi@5>|sn<Mu(pO)Uq4Tp5cCy3&hZ(V&c zv<=5q;D^V=;<uVG24l1INeYhYS-W&N@0NdkPm7H?246!`<6)U^rk+U>1(hn(bI}xq z$3Cu=*9D|t5nUQBhVtcU7It~ktb2RK8~7v*YSHD6{FraC9H*?5ZJbP-S>U=m@VBP) zR(oPS5oJi&+mWk8#7&lKSZ9~$HG+~+tE(Q8>Nt8F!=@y%q-1TITup<__(yd96JZ<> z6>Yb;QQWCBpUt1xSq)<>xz>O{mCv<IFGmog>*>`kN?3>uZ+e+}@y2Q}K!H~lM?XAe z9SC>?y1(c+vd*JFWD2T8KG|PQ9LnGK%xV#DefWJfqrvi5i`CF4Hz0nQ?R$XjPSr}v zz3FJzu~Dv5Y!`55NQhLc-aR{(VK+&~hdMl8P?!HtO-5T#R-3-(!o4tZhRN=6ISR2y zw+Y(OW|B36aCvZ#y6<|jW!TnOe(hpQ7maR6ByWo-SL@}G<~`D@GY2hxTC-ddm*r=K z=wBC)yae$vT|OrFV?(8Z(*}FXuL8%1_8x3C?g|@zS4EU3#C$R{=~S#MuatyUPQ|%c zz0xH9x?jfxN{0$=o+$oK6GApL0z~Z8i$ZRku;t|!t);_9`ae~Kw&bhWYVn6sgW@z` z>><k;W29<sJ5jU<ng{g_b{~d4SK__^SA}!S95FWJcu1x-ok?Q1T#oLU#hWjFz%pR} zRr`MFvq8~^w~64d*EAf$mgGAiS$z|SB!ZtG-!Ea5hIyT;&^3m-Up9eUNHBf!+PH=q zf#0dUDC`OV_I5Fa<g+nTeX?P;<OK6}VZ1j@pU=g}JD_HK*%ZE+k-0_qfdg*Qo>);4 zcHjbP5qBk+Eyjgt+=K_@4+@PxWHYUyS+jmJMhz+bA@>%&|AdkU^*m0ezyDOqo{aB_ z&h@Ywia*?@gbCcbbhp+jee{Y3q|df*jn17NsG`Z=el-=HHvO_M|H3G}`YnAU;57wC zMx_vG-@nr^ZYO|`(}_<Q=g#n#qo91+OGFckh_CC@Up>_gE0iMDz3^QzJ$0YJ`-Vh` zP^MW=3aTU8exPkv(i<yw|6>mArmKMwvc>!NpXSA+iN~3rFQaj}bzZ5El&ir<<_JqA z2<x+$$sgsdzBP{4y5H-VHE<4w1ani@1Sbs_G8=%j`sla3vkh<wC1=*5B7is2ufC`Q zslFT0l}X1(v|2P^OkR_XN+6u?5A`TGcs(5LN#P9XmiGOv)RJ`PI4hcNnno7_;ILf} z?8#i#mATM1pj1NfHU7Kyi6Ao{I-=-I#&;*bC>9|NODM)`mMz&8!2{!mvloNdGC<V8 zH4TE`zIgmK*O!Ez@Smn!Tb4{poi{7=CQP&bFOKDiQ8{QUR0ZDWG3TQ;q?62ZySjr= zpK4rouO-Brwl4&uD}*OhgowvAH|tbM!UU=x><oL(=5(YqEsUgj5$7d)S5#)XpeOo~ zhz-(|^Ynu=Q7a^t;B0KSP|`us^-W6c7a1iiOG{=F`tBVVxo}<^(v8P;X^KVB`jTC* zpAxfzRoP*o+0uS>By^y&{gHKXgJ6KITIy*Geze5NaOgDir;FH}dg@j6Wz(T&Sts*7 zdmG`tDCf@^W6eXZvwC^_<QsuCPgHMy+|=wQcBhnn$O^_|k>T6R79Mh&tvbiQ%Ukk$ zwd)z1uSsJK;hUKqZFjGY<Ql`@Gq%o48~mfBEVjPirx(T68gW?i3*PT@|4WedKeIqE z3j}Ya)pt{?TMi=CvBk~W1k$HTQXLwsak+Ee9|9;uNGF8^7t}+iX;RBH8U!5{X*rBW z{HCdFj5m78!RhdmnfiA~<wqy;)O9b-wejLkgaG#{|DNlazwxDgY^Z6hRE>UFd+W`~ z@V9ck+9X|pa~z^nTzieS+EVH9-}4n?pgo1KP2oRrM=Fx7ZkXQX&p(lA9+w`*H>`xU zp!wZ~gBEPZkU#oa#qPsa8NuR)-yb&olD0MG*>*O0qU%3!E;iCJLXtgSyNbJCdyHTu zF7QiuX7@q|i%V)|pEdzvlJ~6{fYrXdgT#=Ti!akh+;T9j9Lek(!l99er~<Lv=210r zGl;pE)vbgv`~(Uf2qNmY`ba&67aR)9N^q3TUU$>e3bQA-va>RczGf9C)mv@(Kr5Yl z>b)Yyu0g98zaGctnkRhb@~l<eiF{eNPThDmjlTeQ#hH9lYkArWJg$dnh#3lsrvBas z#-xN~wr9L!VBbyEp+WKj^I`o)GPqVGX1w<@eYKMbAL4t(bn?UREr(?ZmeD49fj4lj z&^f>K#wt0o`I<|%m8M6j??flhoawM8zl5?7<*dI@K)H6I!Xrv)zftNw4%tCuY6zo8 zLxjrygRAAbBysx-%n7Xt^{M!Fm7-2ke;r}3s)8dQOl19JU&Sct0?OCu;!im~%dxav z`r?~+>NKnwCFAre#DDsF6@InfAByaqCEd+PT5{gSC~fkyC&bn!`1S1s)4+{S1LX** z8IPWeT&{U<W)k}Us4=~dX%(^%?WQh&t`Lg;fTLZHW~>PoAo}SAUXFpsn$n~Ar=a58 zMGQ7=fsb};*Pdx*>=XskRfxXsTZYLu=8%QniJ}jlFcDXb(|-yh^BXgg>WdXqeRY~} zsXeCl+hQ|0xISD+RpQa+8wvKn#8v*J;2TZG9{d_aHI3gQX%`Ap41U!~z{Cj3%B6iS z$v-@P#MDb$Afp|ZX?b88ICbYuSDzQ?uNIA6`Ej{BMM3{s7Da3@AEh&+cREbk3j0q+ z?ze5~zNW#`1VsCE$CMT^b*QiC3vzp)Kr4k5xGuW{nQgP9wVoKg#^tykCCf<U9YbgN zOA}BN;A4xU=BG1+6X}2b93VcBCNiaYo;2^!lDWRx>mK(fcT%M9m5#_MtL;s}789=1 zh(q~DxmLl>BD@?WtO5CN^z<Myq?D6p({Vs#TV4`vWBwS@r{5=vQJK=W=Y1lzGi-OP zn!jFKSYz71cKo%QvG?s|e--7|N!x8#(M79BK^}Di{T9#l9YULIsr(UXuVqu6))U9= zS{UP_n`~Fd`M?kO7(Y-mPqI(+IgIl3mS@@nne|MQrgYhjniS^fI!H|(P;)VCo4pNd zDBK@Sp~mXY>c=kJHF6~S0FEp9{B()-zg5Hk%cfA{)9~T5tn#&WdrgG;wPOB6YO)h= z>)BOw;fxdCoEd=br2AiNm4K)caIH>&Dr@qh(uSY4DWUr<W?G-0w-(M=6#5XEcUTs- zv6tlkSW&ZzMDSw@d^<V=(&h&#Da1FkoJL=>RoB-B#mQv;)3N*dILoYz(7BF&`={^3 zeF@_4%HZNwE*Ui1vy&rVzYuLU#WYD_nc-ogRXWvVhjeT?x0trhPw(hYHXP+EKDosk z8v?G4BGF!J80ynrasQb|A!HtpaQb^)8%hn`1CN-_gEVP37ay47FbaUe$=mQ}!aZlq zlg<~<fHLGTR=F@l)hdu$=K?9f39l?}y}l^nCqpv<fiw0^j+e2+CwQoC2F`m=I8?L# z($sT->UknwS1IQOX}LkUowz(4nVh~&+L{lJGxCWXb588+&2Xz0)6S2dcwtQ?BAu9s zVr|=y+%M${Z<vV$2%)PUKhg~_afxN5GE;#L?J=bk5MQBz&>`7iMhxBWLpcf;Q+b{j zZ&vMjB%gL=Crqct3#HU*(|(^=HZ6N1#!^px*0iU#l+FKiB4B&_m7h$Phq%8xto)X$ z^)|(y&6GGGfu|w-ctrm7RE<7sWc~1OO+Uh^eAIx4@FHYGtJl_2)bZ9*UYh@ThoS3h z6**_}Brw~<w*Z++@}8@+P`B%**bm_bxkL<Y*9z^Un(<pu&VP89e8E;*k9|nwwEa7) z9Gwe>QbPp@XcdGN?hm25+)1sn&9fcG2T9-Bpu&YLsB5~CmW|UM#VKAyZMPY+l;J{- z`KvB~n&?>NVy>D_*+;$dImHgsG_juru?XbBd0LH;W$#T6Ek^8({q3?rH>MV}%g(YH zXqBx(Uu@a_j32Iayzwo}pziY|YhYhS)KG=EWXr=6XSaRriqz5)RSMprw6S#={48#n z;ViaU=|MscEX(CpyqqL;isBw&^kZttqo~k4c|eysZV&F^2a*52IlC7NNazg>lV8g+ zNe5WX$H6`_M`U+VljDWdR10ZI`~s~Kp1miPo`k+Sah1-ST_m@wfhmgDJ$R36!#m)y zjX!Q;%hwI&aOI8-g}tfv7bJ(+W<8iintnvX{t2EbZqfD?((&AdCr-P<soAXbVceft zG`UwrCI{v=_{8<&O~W&xexp7;Q(vdsA6wncbR4ZBV}g)m`l6{uSy?78!m;*grl@Dt z8gr4aS0kO3=a63L?=YTjlQf$ATwG}Jsgh3kmNX3zf*E(vCrI?wKW43-27ovN#l|Uf zo58iQL#8&h!W3U_+M3-;!JEIqW<795-xh%QaGFsz?V#P^(I)pEln&j0?{a1jYmINP zw6BD{37=jXwX)d4VL2vJ_viV4w+yZ}0EL6a%3Nb6nSrT5b58*QxI8USNtRdRRiwp0 zyK-fQ*GEal088+?^thYBkqp2IY;5bL{VqZPc7&S?kaG08bCcm$8g=#e7w?_ekUS6j zBqsSuJ3@ny183se^dcbaz`-L(YxP#w-EwnvPz#f<qQfO3{cxAc<;<U(#Wg)LM*k_4 z$2uN|HNg8InN|h@C_N~uX`6kyX<x-)v^TK}=^Yv>Qt!>xGx<%!yly<^>_raBaY?i~ z7sRMs?z%(gxLiGi96qu*r&6$)l@F$jxx;=YMGemWuNHtbVXuG|(Ey#I;ke5Z!|XZl z{65oj!F|eU<RxuA_6D%Yh20iL4qV|9kP#IXSAJ@w*m@-jy1kb6TEusXt2%($xc3vA zn`A&DgK4My>FbN$z5hfxCBY5;spvBmP9|1Un8LAnt!4clT;`qR;jg>zyHJkT;gRDs zD@i+>`F#8Or<I_;3B}1xvD9YA@UoOw94I|~C*>+ue0ZL_)3<Tl;pgXiZ~<Wg6w}E= zg1m`PR8Gso7ym_&EC4Cd9_t{OayxQZ7!EHK796=ri<+67?>cCPvNGL__b=7E${!6; z8!=;hHekQA!GgYaiuKbMUl?SHL<OHgaJnkqT4y3#4vNP|tHzy|&vb=R2O4u@Cc27J z%56w$W^}30xTlpjMlv@hZ)J6S`U7EFp6{uPs_KV@5}L+;*#`J-s5c{)9LJJAmC(lJ zW+*|GK0T*q@3n@^erm_#*L7iewH4|B<Nvn5hFGe%;pOvPF}=nS34bh!a=y!wb|TPl zYEp@}{Ge_bKkIRw9F8x-dE;WO134<H<F|}@GLR6#Qcs|+h<ZgL%JH?<z$yt!nX>g6 zJw=bp%w`0ma{Dxs-EJ$HTE6CM0R40kO|YPV?ymH8&g>$yA5L6*W<@#mp-;Wwycw_8 zD(kB-Grb_LeK*L69MN%j*6n9c23ReA=6`*BM<~~4q<BF!X^Nn@ufY(g$ZbV><_J-2 zWZ++S_^J1Y4V6r{?`_x$!c+ozlqs##KUb2k!s03^I}YA2>SKTWew6W8uo~jl3b=M} z6>JbG>=O>O^&!>VN8mJ1h3T<NhTd*007PMVGqF3d?$TfDb>B|u_x#%_8WOp&!>p^U zmWVUdd2SbfL!vbr*{y!`PcLr;1qI`SON;Z$8iFxVxa;CG3PuxzKz;Y0+70MEio2`g zihD>Oqg~0i?*s~~8|s(Yhw*Y{m;Mbk*2z}Rm$`>O{NmtBdcNoFR8~J|b6zy(2#I-M zh#fP#Kg`y+IT4@HN-s45?AY>xoB|=8)-l0${#rpgb6Ywxe%>$2OkzT2y-)I0puxC@ zEidcdLoNVcA&oXvM_cWb6Iqjay@}i-7hI;3MsD!hI(8zV;!*IjcGZoYvercl-p8kq z?V#ZPEJ*`-15cQ{*C(WWuKS}6PsZ6rZp)$HXG)W<DdU7g2E@OuK?1*+y~;=^@&k)N zJ+q|<n8E)`#A>xs!=t7B;Pe1LfluYWGs#`V2HcTck`D8A9&53HmEGy@XL*YW()%>5 zV-E~;9h=?p`PXNY&0o-Ryd%{t#(igU!+}c1jZ$Y4lO$QneO{ybDo`pD*m6*vkZV=O zdVon4E(UH6`_>;9Tr-XMQ03~6XB_!5U%Ki|*{tJ1$e@~=v{N@xbQH3)q*g}Hmj;wJ zZ}BDt6yeQO&)0~zoe!GRW3nXJ(Q$1+NAzcsi2uPaIUkFE**H<V=u>pc63#5;69VwA zgVj6%fY207ad%}3U_qD(D4F$?-t`#VUOu%&t3a?OF_1R&y=WiUdL(aqh$fl=B}stj z9%4*yP2KS+z#Ez9?_9qGD*_e)8pJ(@l04`nP<-gCk@cZOrk=`!Cqe-{Y_M?50^L;F zrgOR~U*e#mw<(xd3w-Jd9(}SrCysJnuv>A@(iLAsIr47y`DFkx*Z1|T{pN3)h1WVZ zQignIzpi_|-R(&~Hj=;>l{)@arL*2Wq}a!U1IdrUZCx6LW4&yAb*?y0QR2f-Vh_GU zhB5IwGWF8aFLq0Sh{?*rD**LiOG)@YZ&CT?Q)qx?Q21AcOz(S&-eB#YV_1^{3rhT- z#k5(~;-b)a9B}UQD1Hpx?lGu`{?az^rD`i7-J9O$r0(lY(_FImGiW0@1V3gGT`t)M zpH#}s{yBtf;d_ZQ(-a5t8UX@Pg#D~%@iSqEJ5AYvZKIMS%duMBqT9BrY|r)7HJjgI zrm>YW+x^I{XBO`W$uZ@Z6m!`#I^2ByIA7SaRu8o|38#|KOSp@``mJm`&QCzL-iFll zc(sqLK%0syx*8nsX_5Uf&*}eB;T&go7Ny%C&b`!C35)(g$i#x6sR~8zrU}U%$@FVc zp0=Nj(kY^%@aRU)S<NAmSKVsHUH*n;llMuKy@h~26r9*5EYDEO>sd+#D6^vmZk%<h zI$CKwC{bQj9*52Prk<Q}alB2Gf~6a#lzOu%@{ZwzgF?=BwMSOGm%@4=NiYJ42$Ya! zW@K5bP{7$MkkM@^2E-n8)fD*Irzm&+M6k{%rCXjf$n?HMe;`s|KHjMDl8=G1T)}<a z_N6{XOCk$@%56J#7Gn%4=W&R~^BcQLLF64)ETi~iyI_vLGmS+-gB!7eo`p-+#~HPp zC3C96?vR)6XV9+|8}F=n^SVfUdJ+(tzb1|MP_*3L);em<b&g$=yy8jh>`~^Tzxt3b zXT#g2F3<H-Z$>jRk%CdH@8mCbg9&L=b<dAK^h0tiJ*`Ve&ZyLYQV8dvAMHrd4_gQS zfS`+8OCAN4Y#Vy8d6OZS`qe0yh{MCnzwDnPm01a!GbNBf4W-dp&(*7UF99usK+xLa zzj-45`-?>Tez|(efE2_ylp<7A>6Gq1NxX`TrR?LXdXv}%^bX2?eSMzcyZ=w$6gV!s zgWt3}1c<b#p7aB1Aw%QeigWZ$;@7L%YgJIFe(X>y7obJ|bvMp4f<s-~ELb&A0C~aC zrjoO>mSjk=(CF!Sb8mu><?l?quE3-G>2bVwocHx<pNEP66XDTO@%27Lg~V0b=V_YL zy4*^u!;J>|1UN}rK6HJ1oTOK2&!i0crOs4;zo355eLV_yx=bp+O5W`H9vA57a;#-5 z(HO$-zCHePHlg3>>f)62ndP9J>028b^FrR6%MF=MdRxTJW2fI4#<dF`q_-KS@2`Mx z{?7GO$3=wQdp~^WK9%-{Z7wN8Lcd``o0x;FZiTD@zF~EPy4@x_Mzo7yHdc?S;^DGs z&jXuwCwErK!UHit1co)Fy~f9GeapqMbDzt<BG6slpk_z`fwJ{8D(?Be9eD8x9XzzM zhR@-O%#48`CT#CZ%!$lC8F0SjGA9$o+uD0JJ@Nvr#VlVznQ_<`dTpKRzT!^op%UE+ zkf!5k1Xg2vWi^5HYJ)i;(Dd6*uprtCrmTD5S`{!A8hdUawI*iS;_yo8|KwooByK4< z$~Zm84AN+*Xkxkhc26kXR$m_@eL2XC_<3&ftR#ye+5T&=tgJ4n&gYXP5Ydp*0Ao;e zyX<V;ad8<3uZXJ?!7S<mj&GzTuE;_<AV|v~LcEyO7Ex<Y@zV?bnqnr<LNYZ+h+vd4 zcjj9)?T*m&poGPy96zcT*GN~#)(9T~2@WlGmc!P!FixWw>)KA&-F1u}#_GcSZ9s7Z z%&+~aBTv<(yaLFizbq3I_g8cWp);RzF-Pd6wq`i5O2cJz(SQ74rK*5Dex%K?P||-L zT(haA2qGr*29U&p)NY>rzB`w}RFwG28d%v4S0;>>_={j{i!h|5^JNw@F;v=gw_wul zc$AfXJW~`-Hk+N<cEf+Y%9!ULNND>DEm)!T{k&TE=Fp#5`c<16{RUlH{$6BGe-h(* z?QBOikGp=>*_hwmn839@7<t|y?FQZg#H%~7EV80Lx1`;OFf@#xv>G9j%^z&jwkJO) z)%G0_tH3^^x9PCnWM3wIh=q|uj|{({WnqTwdk@b8NiwRx+5%cH2zGjf&F#{PEdJu{ z73q|8$o}Jiv5Ud%fCC-&E-Kqal=B245Q~u|`odg~{iY?xVK9OT<NtKPq-q%Jf5%IS zk^wH9#hljqj<S?hf9Kpl6POoz>sXQ#;}U;8N(^I&mRzQCwIz=AzO}3JZ1ClJuI}7k ztzcc!HM8+j=NY9Xb81pSP^|8Dw#aZRwL<!Vsmai*OM^Yj>wX9)5^N?;`Cmtl?*#AJ zz5f>jX(+&{vK*oS;%ZI3PA5KalHP9b`+zE{{OC8QrS|3J@ld#NXILmesT8g4;Pk`+ z_KrG0PO3XcC*|V?QnmVAFP%ZHxzi}0x^K};l>7z!;cgOOjxA>!SJ~Sl7(U1mT&A_h zQ~d_&Ty^l{NP`{qnot_}Y&~YG*|_)%Z>5ePAXNvgf0NtjC|CC|+^XvF!VQ7_F7WMi zbg%~>tMl~4h#Pm{Np(9lgA`Ow3-bhQG%|lV_f0Tu3%K!sj?|0+iMjnXOVYh5b;ip1 zxo<e-6mk}>b?AB5A*bu}jmGWu0C-n(&)?p7{k7^;Bg3i*P2Fnj1(4-MPRQ()r7gmH z^P96QB;Y(LfWL2TUKRA0ae}5vfK(uX3*2K{+~|z|Ypv}CW78k#4^KB&5D&>^Zo-t^ z?~;cA3NkJG427fh6^GwMrfvan+^ubjuAv#WSIjkbo=apcEs_;eL;DW3ZA305@Wo#a z6fS^Z`Fg<4;o$3VQ+Qx}Bjx|&^lrQPX6jTPAYsQj;m3%&@2kNDa_uxwqD-XXY{g{j zOkl|Iw?E00jKJ_<y7Z^)(Pi5g0z;I9*Ud7@NFf^+EQUK=hGUm3wEm^IR~ovC1ZG2I z+-Lt|k%y`ojl9nSGX2-^%}PZHA4r0pmdFL&(U2E<{EVVZ;+{!@!Dga?K`gijK|zuk zWKW}&BO7tD=4>mOOIx5(_4GAwhp&OnWTmIKfj_e1hr5=lRli~G8y9{FE>P#T92tGZ zg&OU00*nH2XclFUBOTKoPIJjR@G-5f(-?9R%?Of6{Q6~zHux1+rYu9Xw_`NxX#g{= zSGwlq2#jb>Au!f@z3Cl!&T0Ekxjw){8+TOw3HKX6gGn+~^Nzm5eBz{*KT!zVzOxgT zd&vf>af+bjl~Q@_UtmnQZUJ}w&79MKCJ`;-tXS7Aiv6!5>&ZB<TRa>f;6Q7KM`o2Y z-GG2xzSH5!(q-5LMPQb1KVSsB4}z}mP>0W!yU>s2N-JdXC7wdd+$1-z5WL@5Q#*!7 z7vqDY_$4scp%w!<$8x&31tpSCw3f^8A-c9;!(iuK^$SXS6mI15v*FNgH_RKIsp_jt zygN!eA9KKtw6m|$bEj$NBv37C!^~&f-_o^5Q)8{kq$gpoeP4aP2=Uvq!id(~WT9<r zLux#sbO|xZ!t$&wiWMI?IZKLW4f=I(O@<0*J?ZtB=O}&9t4)RlVz+R~(^&h_hQdAh zM*%OL82eNsl442sJjtZPyq_>dJlFzOpV1~cUnl)G?<jlY#tJDgM8X!rxhD$%rpk@D ztq7s?()L^HF|+CRn%Lj2pJNy4&x@M~KIqzy9-KduVb;r<kPWywYD0w2GA805C<c}~ z-2;i4Q~&<|8)|t9+RT&mX$Aq+m1dtq<6aGD=Cwu{z*I`;xsALup09JZ7-;|r2%>t~ zGD9FY7$hJ)=Xmc661uI)n7E<+xN|JhX|uGwlvgy9WuQo)r>sW-Gx1nRYhQ-C+DsbH zt|zsjHg~S+dFn7SFKzRrg?M0VJE>cV>O}XU(?gbj14oE><&>1=dc<b(AAg%A)e0aU zU6eRnb2=G1ps2DK@Q6MEs*k7!0d%5#J;k>BARi>Qn7pk4tQifiRLwkfWt)Jp7Kf$r zi8^ib7Ld7FKq1&pDll}mH#dbqM?D)P`e5ki-iTeMjRV9b!hN<Pmw>~C@>Y)4SO%o{ z%<>tvfvOBd->d;fxu5%Uw<s+Df7m4UHV1a5*SG!}v~NvF4IpuQlt9*67k-mcCkQ&P zPv&N^`2Cd>d?c(c;3@oBNMqad>17yT!vD_xf&f)QJD-Ez0JTvI`e9eSmmLpKGVhB+ zijQ##Dae|APCqrsy`4imUku>=^^wXr#$ec=2Nqs46?)^6W!t;tf>jaf>D3uX59$a( zrPFSf{s_kVqO&JV4B5a*aQ9P5jVqA<M8Ki_!@Kb^>*G5wDIwR28jGwOpJh|-eQydX zD;()C>G28WltvC)k8})j#k6lqu+e3MVuJE|PoMYE(v0=hgpSPMb+hhI)Y|pGZJRW7 z_$fGXsXi6sK}3&@O?ln($wJs7e@?U@{jV#9ZP=QdN}ENa+=u%OF#<LCG$WNd7P32` zM|u&_+idM_6|@@_#pplH_>`kH=SsWOR23jT(pMj6jKOq(xN9NXkU8RWjKl!7d>``S zt^G#UC$=5F(A^wi?#y5Too*Z3#n-Muw$|+rhg%4wz2TH}x*^q%e1*z!W~0qv2w2!q zD&0HtE+rL!)P)R$m+P1C>fCtKh?iZB`X$Ejqh=+-tz_7_Hn<N4+zPki7-}C+50YL7 zaqIM>fBDYR+LLkJtN<p&%g@V@5Vi7!aeqO8=Ypqb?N-5<FOb++Dt;J}aw%?m6?9}S z5?yH4DMUI+v8PICt0FgrOHyIIBg+-DO#kG(Ix(4XM=8)Tx4tb<;%5m>>!GW%#4Q2i zgql1u|0#BoD%VK^8rXZ48R+wv=-b?nl6FCYUjn_yT46?F1Fy*3yCz=)QJM!(%Aak1 zxJyNHGFB4mX{$@RSou1`StQEk4&p7AvSzg6)XX^-%TQ>@JeQqDD7`*YvE{ERF&+xx zgJKJ!gAp<7rRKopvL~^>GaWvEWwbl<5n}Rk%3fUlTxVyd_LwejPlaz^E#iZO9V@zC z_(mvwt17_G5qfoZm2*!f@#~-3yg;F;tZP!<%UGu~^V-_a<Y*T4Zub@fxpU)S3W~7| zpWLUJ$NQT5EuZPr{;<u<U;-&<MC}H2hgt}?vksbYP|8!|I4npD=kwD4RCi^umxDO? z0CUT2O_ye*d^G(%Ko0qLA8hu(f4_dCgUNHAwL?zlr81~TRRlT|x8MW_HQs9=8C~Cq z{)u=vkLKMJ5hmeO-xQ%8wG$ZQEh8o1B&-Z|0Zxx3Jr_06gVFFV*4CT#%uu1$<6x>p zZ9`AujCz{={@%g)kimIH3Weaei%P%Oi_9=r0OE;bCK`If3rNb1f!u%N(|h@0XNy-# zqMrTZsMJ)5s_tIBNi3hsui^5Y4Eaz%X3}rF&=fVT^#Ca%#+a#rO!9vhEv5&Odl+HR z$y-k;B%XiubZAOq3$lcw^&C~wJ)Y<pt<wcZke)3C1i0S-i6>0JAQ7+H1z=@u^Ec~L zw=Uh=PSUC-x6a*}Do#53O1k4dTwxiYhxzu<vE}vw8p}cfH5q3gjSA~UpJv_7>6GCa zHRl;u4vE`K_WIt_j}{JZ@Q0mxINEH%%1q>t4N$s!RhG&6Qd&(tGq95fqY_ij6{i7@ zkZxwzGE2eXDl!~68Fkar#q7sEvAbn_5lZT?O>9*N(+KDTPfOAxJG2*~rpA|CE+}gz z@R7OBzlNGp);l(a-%=<v@O2^$STY7@04vSrx4WL(9i8rdN~XQ-2GWE(W`2{)>*HFg zwM(P($_wIaz~7)9S-&^tO?iXvzCS?u%I*W`@4F?Av;U)V)YP+xLG^{SO0u4${y-Us zZ=r-aqbqp(UBqWw^Y)HA@Hec`46zY<QjSQfA+B%ggjRO#nWLh>3u>#~Xkt1ft5QeH zO3%r_Hk5$R3s-W=7;gd29^RM8tba(LuTNDolOce-;%Qgq#p??-I$RcppN%3|zLxT# zjS{x~U(4j=ZR@5$t!L8ZCmSg1GxFg>3@jeSk4OS+8^ys^p8YqEdtz{!CbPlAuXqr+ z=0fnkAbHF;C!^7nT0|4UL+>du4u7q~7&Oz|iYuk`8Dm4+-bufhPFOb|`%b6{h8et_ zgDMhiLFjX1vyhI}71{9bo_oDB*zwTVY-Mujdc})la7*~iLB!QuaqVO_g{VJ$gyJge zshTbM=WdlF4pQmFENy$kM<uL+42X3NL61=}hCV#50sD-Z`x`tR;gifE9W$z}i5#<^ zh+(Y**^c9ZvGp|s8Vfn+(M!i#aMM1&9v!wVpPci@;lf+S_oqv9L}a)E540c62{t39 zpD>vPN`uV#ifrv1d8K@<!$q}y4aNPbd0yF<GzSYU6$}bK>1J`_@c+9DKBs>Q{yuuS zYClpjOu^Kn9iA&!+-AY-`luC=aWD6jK_>o>aBOIkOiF+@2nad<`8fJzMKkz<PFiNf zk7@zgWs)hJH|q*s3^+6Qd;vDIXG9&3$PEq<#5H5N?8{8|%1qW!jq^)GR?bVuO(1hI z_?ygUr10JQ^)4#^sSE>FU9c+Lf@v2>&Vk#7E;P8iYEz&NV*O8k@cCW1O<kSj<$=r! zj&G$RmKoj_wx{t=lu2$KbCt@ZCV?J&J$0m=-Kl$@m+n!_bi?&ZzCK|!vkJB#?-tg0 zwh~6=FM&>Ty5vausF`_woEhrpd6mM;$v>b+S`Cg%A;~%-8yW^0N7A__)rAcSSOb9D z<5sM?@>!nQx|;gh3TK!Yw3yW?ZwQl+#3~X2rn{w%CBQk0S_y}Ql?7bw7&o6zlHSzp zW3%VxYyIe5xdB);wE%Z_6ngRw(k3mimx|Z4-SE!xD<HqL<X8~fdy^{6Z3k2fltg<A zgvd_zlZ(vrwPmwkir-uwzZ9L>9=JtTZ;dG)wX(um<-HA{2m(HO9*oHJ$Ovu6B+$X^ z$YpCUKc}~SpogD=7jMi10M~213G?F@FII|bEyF=^nop?{a3FFw>?kssd>o@ub`1uC zlH_&5`eeq(doD)m9He%3WLAxW*sa9ojdwFEhCW-l!Z>#2?F1`{x(@%m%(nNfj2%@h zV2V4x&l8KLHs~8E;=qU!&gNWy(#aW2YCu&BT5k8VQatv-#*=2cVv8~@XPxD@A6^7a zSuj4E-Jq7~;kJDF=m2vlW&>{2X#`LdgO<0uZT(%MyptF^9n4jL0b~(&Jn+<4LUPXk z2D{`~_4cosUm`-Wy0NGo$<d;;`ULZGJMHuOTN|c&+mf37VX_{v`~}_P%74##XbUO< zqQq{mq}PShk(Y^;GziPD$OBhH2jx(@bJ7Loy-m#WL73Rd3VvW#D>bacuGlDvEuM9| z%h3PKEN0F0Gw#@H-shc>SPl?^K-p`1QvowQCLOL?64g;4b}E4(^Us^vF0K*6bOwFh zR66~(5Es-#q+soI@AN)JwA}p=;ij2zA<XnP;q+(R3@Z>}?bIhzx6;2r9wSAO6o-S$ zm4<0JK2=vYG-#J<y%Ls$eoQ!<ZK`7bA|Q_{;~t@ynSA@pKEj>nEO%h`Gn`$%lw5@a zx)=2mhWo()jg_*fc?nQ3g;#L@k*inR7oiObJKyrOw`D1t09YlAtNR2MJuUecbW}@m z`F0`%uhMjG(m&(^7st4%IFIluZI*k0zb>hBd$i?qM=Gj0Cma#x<V5~kQTj31rs7T8 z=zN~PJa`)5*pV?<{%Pev15R6>rCJ-R^1sKU^yW<KMz;-!4E8bB@N$GE>RYDG!+mXp zB_U^?;t<`uW8FTV!?!?9-G`6t5gY6d!ERIpuvfBLn;%GEQPLbGQb4D>-^o5J0{2a^ z5Ik|!UX97LA_A?Fdd4qd5?;298S}I5UmAWpbX+9t>3q|DMY~!;O(=9z2{I{AV!)jx zoFB7Ug|0Fz5PGke?t$pG;;5LF7E>?Z!rc=LzJX_gVzb}y!1X+rIJ#bNcp#<wuCAi~ z`cCStc?^4gdeVAfXB!oS1=I%OU;bnD{U4z99f+l!yJq_PL?sZyBTPoe_y3V~7H(0; z+t!}}0coT=ln`m@8U|2lK|#8tJEUc30hJadrKF`>y1N?&q#Fi?9{T-p?mg!{=iGn5 z=TT?Av-jF-eO8M-yBXQ1y>|E{Agnk6w7<~%9SGrIO$@zVi&br7O$ORdxe<Wn^d2Sb za&9W!R7QvQVIA_`)L)_Q+%~|(g$D;PCH9PPV}!c!5B<6!4N2k8#6VLibXn>6LI&k9 zuq`Zmg}NEc4K=6m>%}5b_fBuCBYHj&1AbStt3>CokCuQ<>%H-EShG~B@t)gW+Rt5I z3Cl85aZqn@iQ1?DBv+M*0HEBf-0XndZ<tQbWZLG&clH&Supv==uwJnijnU&rjMY#i ziK+zO^@ebB{<BZ$4}H(;7lT>iv>(GkdlW;EE4xE_5!!6W#U^g_&4EM`Um$=S8uLW% z>$%Z~UY7TpDISqiozr2!9p`aj!-ojLNpr8-{HM_+pg~dsKEd;^(!Ibm=T@#T24R|6 z4~P+I`GMoFpqqm>QW-g~90`mF`8vj;C@lDsPne-^>?=oKA((Y7Q{37B_wVGs3xvFq zyWi5~Dy=}csymroa|b=x0_^*`aR*v6qDgX7f*WOXOPJ=p!Hd9}kHVtxTyin2zMKVg z6X(am?oLZ^Tj2<P_NCl)Q$rwo)vAr#bTvN^hyLU(({0`Dli6f68Ku<_x7%UAJMQb9 ziA=bH&`JZ}``S}?tJC4{+893$wL|f<W(>51cdlx0+nH8q7gZ8BVT?N<6e(l~Ar-ag z%Af%<;~4w?|ATa;TkL9B(AU7#=dIA$<7P`{D<K6{R=NDw1VUc$VDArx1KC7hw8RM* z(NRGy4+FDE9~}9^s7v?CzBFH_CUp{5_6I~|$Jb<V<EZMHQUtL|+${0PxB#W}NJz1G z)pQ_O(5eIN3k!<UurwZOtRlr_Yr+{u>XT0|O*rry3o&wQGuu2u4{<=&VkLdMPK-+2 z3_bz8IJ=Ttw*T}3*nQuqUwcwUvFEkOzGZQNw<CwJ!hyW1=TwWHOAlbv#I)W?t|TbI zq^>k=mE-dd3)I!Tjs_@wv}?DPvvx&Q$VPLXsn%&LvGb+HvWraVU#X&US1rpP5-@-C zpoYaU>cRJKaUmgM`>>N#oTZFu7WI-ZX9i)}6KRzLb)vfoQ|EPnXGKYx21P|EQ&!pT zGZsv7D`$Jwbq^nKr>$06_j9Fn=E{Yvg9|*H1s7M+8D#n?rIJ2HU0kvvD9X}OFPOnL zKy6vAN>zu@_D8!xg<;wbzVJ@Arp2?ahxYh6eE#%3ZPO$Kj4O?m5~tX>+TTqW&Zjlf zR!o1>jXbtyiby-L%iNAQ?y1O8xU$OUI>AC3?U0N~GoCm-Eat-cyX=jf^vYkyRx1g8 zOn-8v#H>cpF`ZjCa$gH&s1f$k_4V}Mu%o7o$OeX!^k?DYdE%(g<jE|EEH7w<zEYVQ zdR1-8Si+Ha<)*-X#kJ_CgCE-;@r8nnJ}#T?V~OPC*%ulS_3wY&OXam2X|-STxjCKI z>kioMv=7DWLlH!tJIY9CuI-6e{Yf9>yY>2-rr_4K<QIj1;%J>N+=%19_oj4S{lBlq zTPrxUfU9E}IEfS&hMWH$t9yhVEj(ve0RlZRd%V$tHZjfYk9QgYAP{K1_sWWLAV8A$ z{f~E=+bd7h78`)M$gMjxVp^{}d*ubRCPmQA_)TG9uW~c-N%BZHU{>d{t6h)#y#=`A zZ2p{Rg@(E{Uo>-_YeRXf*eIa(VtN8I_-;d~KQXLp`uA~>7eLxS=={zSwH?D%7HDYN z9f=PZ=r0<gvXuK8)aaUR&=@z|)-1R~d<B;Q=5l-!AjoF6A(z_=VjnKYt2%Mv5V_w> zrrS;;fwk$~>?%@hEm{^0+FP~@G&&bOXH!js7&O=eYuoaOEFnHeVQHZ5Apfug$$ch> z7wBO*qzA1VFILQ$gyp+IF(pb(fkkpsl76l}?y@uK7c>k_#ayY2QlA3qfCG6QAT8(w z_-ter70UuHO}UjPzq?n`tG-e#W(eH@oIlN<L$z9cuR>@XtoX@lEvVDk1{ddVg~5v( zk%A{d)oD2PV@IIBIcBg|_{m#-e+MeQ-L+pU^}>dhkEbmfD5i8xUTxLumh5=omvKyP zW^i-Wd*MhtivR;i6LbRKkNrBkMmVsaAGHGnJ(!N26t7?e$bn8n71SRfIXmnlj<?Ws z1I8KI-`TYz<*t1X^A}P#`C-8iOMR<Ka{T((v|n!PJkQ|k!B|sUb7Lr;Q~iflA9`<> zX>SFoRA<aoo&I<K{!syu^<dl!Ii!2YhK@nG@W3~Q4<n|fjly5&KfyA|BKmSRR<`!z zEv6a1E3(Cy97gNiD}qHEX}wS9&F&rly^2GqfCWkHcGBWW%J_n}BepSFR`;z@;aN5X z%6daNDdJr+8|e(lXu90Nt51*Tmu)EtM$9Afvc08JTnqLJnA}HaNy~)|G=_zX{otso zJ^v}iu2U7sN+wJi>#~kjS<fge&CJt7PFGT~vC1W5ia-k|KhAw62y;Y%$L)tX(FB&+ z;hW(kZ$RU7PL7hCh;YUd2%nW%rJodcE6Nozo`3mQ<b~puL#J{BMODl-=!zGSuLTG9 zB>0RZ`F&Fng@m8VAHUHv&7Y#z*51VpO(TNlL?;OlmZ|us?Kkb6{r(7w4kdIHRenUM zo^BgUre5hut4CP*{2KXSPb#>Nvo!WnC@#wUPp<t)m2!pASLu1Itqjt*M$)lUGY-u9 zN)hNy!U2y-8Qv-;La&gpOfZ4|#V_;Un0i}va)L_`c~iVIX*5|bW3{hR0=`G+u_)@7 z&AGM&6JV)h!5H-Ev5JVwIOI&Ykd3o0+NH_x8<?Zw^kc|DwyZg-mCl#}NCoM6)sLl; z?vzhCG`-fYb7{`KS+`dhCj3@n>x?s3e)a&q%ppMj<}T(;#Faa%kOek8DD4vN4U<g2 zEx9QU@TC`rG3M~wut|oFB!JJPjYg(Nnvq0_GnYXwaYKTs(uq4f+up4mN^=Nl#wR>o zX!v$=-qw%F^qZ?r>8}*eb>gBF+(ye(FK%$ASc-i3lCuv4P>bq>mCg2PUS9@q{FOCU zO*|j`+hO9LsMR_^KxA?P9B1|zb`HP}R0=?g*%I@?Js>pn(s3Zh_q%Bb6rCY;4;%m` zFNxipjJGef2E|@hFfMSmG468Dev6?fegM_lYsPw7^=XypL(1O3M5;rh-euy5oPXav z7OD!>(@ywM>gF(Mxj=HEX7|?fEov@}?bD{=1#@8oyzBknNke=kuBuP?JTACN1LP5I z)4e7teNE!Flck;H$)P$G5Qp43T%HD^I0N_D>NKUq?_RxPM9)L3T|g#=STmO3&7^6y zv<HjDvDiAF<HOKkde0g4q+MIDeF31uav(0Zl^}hW7m!zsat}#+FzC1s)IrTScheR4 zU|eGzZr(S(kxd)H4m*(m&avQBb{4?3@k6`35olYeP^s(p_jtD7dLMz?93#yUKv9{* za3T6S`4XH18sI}kStwV(Xp_oAO6HztY2~F^r0-4sjStW$=fJ@H0R?(ul*d^*F?G8G z2^f?<wq+0klD(dI{y#5dHNFl31AGiG7QvtKxmC~LJ56_qScoZ%r4OKLF&)6gBCuLQ z@yG7>-P?^B+K2JopuFh<_R|zqsYJS=>kEbDMC_XCx&cSu<K)s_*5uFJ9XCU^^`vYc zOt7$?s$cU`T!#;`qz+EHIPNsWn$Vw#3kFq^y1WHjKpZNs^@b&g2qciuHzDt)w@B!{ ztYnA<1~$3S#><AYyOKpjG99kL^;Ve6q>F=Ga#eZgOUGXE#KNPPD$M{QN4)FE=U);4 zxjn`Qz(hz9&Eh6Wq++Kb@_j8}#M8%ZC!MG2qyICzur~bE561j{&Xms?)tv_U$**4; z-nuB#<z-I${;q#u-ci0~qH~MZ7+0lqoJa}V+YWu8jT}hVm3q1~1$Yd7q8dep-7;H3 z*xKph;*B5fzI4St8S&@d#c-i#Ye1H53@`At-zu$QEUw$6OUu0V;nVI}tuSVr7Cs%p z9BwEx!3uuCtgVVp!n|0~0<|q#UhH0al5dpc*lj||CCk61zzeD2VKeNQ5=C)OdX(Y~ zNfKJgR|Or3>@QSjqhwndeg>(P2!!YhKUE^MTRI&5z4}jJ?2qc<5Ze&!J2$_vF1~IL zliT!8My{MLg8b4mT?ur7>qZ2?gM6l9>hkbh!LQN6CV4Lmo$zlJIcxyjkzi!&OnSqP z+0$ukh_m?i#*@mLz{O~4iWI27&a)ZhX$pCqg}Y9KI-zazr#2O(4!gLOfkz!nekY~K zb`CFUo$;`hdMaH)EWs5K_Q3C`wi~R_wBfw<ZFlfWzIhlp?)gwRJd<v4wW|`0U@#{3 zj$Bd8t;`wM9_luNi>Cu#8PXzyor+pprXjasUGh0c#YJmxdP-as@zQQ_zibf(^lj^K ze?J$IMf4v>g`<DSGr5SPm3M!nHl}l)Mo_}+whIGVNMW0&>UT0AkqtuFo(YogB*M}p zL})1`0Z@_b`9C##ZBN-ac1{~tM!~-)6~zy^*r%D>`xl}s7W#x%T2{cbfd2`<0(#nt z_9@Zy(<wAYlF55y8y6e<e+LFQQ8*)5*=|!hZ1X^aOmE6B(1&{h6OcC24-hxH=+lBv zOi}@NuZP~Jt5H%-HZ;$9qiBS=YgKV|!Gbh&!B7&ALD?OB`UG*LWy)Z%WCk690J<^` z(QkCSroHn1HsGFSs-x#4<=VsjK6@R4<5FKlT<2!Q#aY%t{2m~?5#Ar_@4Y*y<Fq?k zP(y;Ki$(G!!Hi{r(^m9BNGgXTIhq;KnW0J~RkaQl<6ZgQ$SVEAb19Try&)d02(N$Y z%6e~`ez%a-cbPe)OQh<d*oRFVhI&zc-REkSGaAC}S<`>(#_l~cikYK7fpA{WJxh;O zIblS18z9w@H=Q>gyrcnOlHR{>UW`uzj80;q8DsP+I8PJmm(4lAVo)ltu@+bIc^SY; z+JbjPD$q6jcGnRN`YM5dzc!~u*~$^ebLtJUOyfWbV7Tyl?9WdH#WVXfCekuF$P@-@ z*8-k{(9=8nU1X=O)WbW=Sr!>OCvmclwe<9yoBJ)g6v$Oo?>_Gsd-O!t@Y#yFdKVvs zz9TVky~5MwcbobI^+^r<oSO9EYH$<h`X~Q%DLuj72J}xQw}0ev;H_6Ww74Br&lW=S znr`fsjL6e9fMSPxZ(uqWJ%3+<81rF=guQa|e{N2j1QMxD%QkDzzA4&34!BB<G3p+` zz8SAiIDc;K=4BfOPK!>RWLlrqKLETdoF_aCX%ilUO{De7y7eCj`|@aHKM=yxgVZed z1@2Rax(!c3<(3#6|8bRP#DY4kbiP%aKPDDtNSwhvSa?&YF88cMUK$t2`4G=42|){8 z*xbn|0%c*`NvF`7#H5qowz()DvvzT16L5KJeLX-@lyf0}oEaUFl@J>1z@+s*QZ1tM z{E{S1Dl4a)#t;#7nm`wKQX}2loW(BUi(b;&?)+%IpteQRyh-%U^7imBuucpekdpss zd{$3BAHi~*lG>9Rs+yR|xFCxBVj*X+*m)^hC<nIU@I@xbUvXbltc6lD%Sx}!EK|+j zHLS9pO0jHwuzF;offN0>)6RCm>Bi((3X@h5S(a5p=4a)1cQ_#aekR?^@A%Y===)SO ztv~&4#?4q|C4phjDMKol8f;c7du)Vb9npvi(FA$jyIB^ctSWx%kp`|oc@LhPMWEZ2 zB^0&-^modpOsod0)z6TZ&G9BUY3<ibM1#J)Qi7$1$#Vec+mhgY7c8(FOnr`Ati01T zp#2>DJddx<=Nkdw@bi<LoyivTw}qZo_`m$4g~sfxf02ZhmZ@rOTk<`FY=0fXNY~MR z&PR_*Xg^-~^CCmpO8AV{ytR9Z2@Sp={grA0%M@954x+Ih8XuOhO{@%2+$|>$%Tran zVzMQCc_Rl;mgkd&uNv!=#_nwDC^7qGd-FnsQP(5aVF&mEYVWAgAyP}0h}lnFR83ep zSfZWR0Z}P}(*k$vw!Aa$y%V<Z8-n2BH?+;)@p>vF^#Pw|A?pR}<N$iDo}0F}k74Ri zkT#~-Q@O=2h@PyLu)$zq&yDtMa|3qJ5sWf?%<Nykga4ySt>TPOaR1W~%vy5@gcCG) z;tv4d-uo7R@J`ry3I^kF0w2Ious{3=DAnx%$SHol1S20xs+px@=+df4nu#qEC)?v@ z_sa(zMd=&TpGVRXss37&1zKGg+3@j1TTHoOJss(%Iah8g-9sRsyz`;kFM=SHfuARc z{brx=+<;os$Ts&+xF-2Vos%~SWby@F9vXs2rss>V;s365K__FHaznFkJ-VMFN97pa z&GPXbOZdw;En(B@64!&!LT)gmmYBNx-f0;K>fdWhz;qZ!9%`3zpPk;7Obs6tJ^TI~ z=(-?<rc>VDb&#lPOm4_@@g*ZYmTRe^AFY8=L@mEaiAvm5@63htRHr6b%PkZ?j(ltu z+AMOq$2`d~_y{N`K4*c3oZd${OyWEoS-U+JeT4C7gg7NG%;sc`nlApTv;l`P68gI| za~tc>Iz7f5fRZ1<m>M*AEq1XM+YaPBD|7+fA`!h<y;1U(n8p@0gx+@~w$tJVV4vW3 zKWjsL%9aMdi*%X#%c(1b>I^S33`tm6)#sGd?ffqN>poZIo4V3oS)OAerw%etIb40b ztY2Q~2$c0Lz314sI(vMbhIS}pkQzi-4Yw%(VeKePH`|Nz;z{g3_ZUC`F2<_Gk$ADg zP#dhf*N3gFOQtq{DQD{aZy7#s%cZHR8^x)7N&F9wD7iAmTd>(}pq*lgDUZ4LPQD&J zJi3Kdb<T+}l#2878}x^74S#3z5Xf$HJ}D7zBJ(fp3Pe{RU6Dpk{+9h10FSLouu0Mj z1>AgnxSiAj%lWcL!X<>d$j_5a7-I@NLzx?H;MX?d77Cy0Bun3p1C;~SoG95N%i>VW zTk9on0{<d&2&PJr-Dg*N9Y}k3nk)&jv4()Kk|B&Pi7+D~iH_LyRmGYYV$;+iKuKeh zvg5(Iv++w{7K6Uz-}35rxjR173j;445pVSceAhO(HmvOC-$WW(N1PQik#1zB(?u8% z&$CBJ4N<V8^Ii}3x{L~n#O06v4!aB@56upGQFv-#8$T?YggtVZLX0&FoLZ&Y{>J|` z6vP-I-T6wybIO)as3AMw{-$1V@f$&Oy8WRjW*uOb_}Q*$7dJ&z`s3<3gKDpoI$`z* zqa@CC*9IStD&cUem;~RXdKCGSN<qrG?7R506<9MtwLF!U=`QnhN+j70LD!E<_Q1F; zbvT>@Jba>qZa)tDGHm2)^?6O$l|ToYRhf!Z{xcu-=g#KJbI{Mq3+Gbrwrg&ESPJ0a zZYa9zU8#z>k0sa+6T+8Ykbf$X46{x$P4kk<b0-zZ@sSTuEIKPQyo5>L$fIqKI<^w% z8d8%eSq>g#C$z4jb0y<2jEnS{#$h1F1t~A@0H+TconPJNnwvLx?R#UYkYC}=JG%wz zRfoZMmyPFC2ESvw@o;&on^eoh-5tTv$#2Ui{*SiAhyx8-0<3{rY4gA`!<wKj*D!<@ z-TJDhPYf{a_RZfLi|?DaBDW2i3b8pps0lxU+CPDkKwV->*(l?|ur?|!Aavlq&Oqly z_hD&&x#Rt!P{x+K5E5@V`en`j5AbEvzoQv+Rd;Brivs>klstcqS=0L&KaLfx`Q2vH zi4uGf@8RsvaW|g~+&CVp==!B}0W0D?L0@DLLuD7Se^d#OcPyRKwSB1<V0XOQ6;bfo zJ#Hnsy91q2g13c<1B7KmVa9`jO0aO10tBVDFKnq>*1rHTZOaKm3|F8lvIUpOiRgj; ziE$b&DADrC09eMFPLmaW|Dd3M74pAHE*22KhshOTaV*_L^SNB34iR6Ups<Xrkc;<p zexP)$1Qg1yFkKxKg)K$(@x?pHdS5SE7?RA&zSYqwJ&KfrDW|w<GS$93RM+j$_sv7R z=cONEp+RYJ%^h80LWX60@(8vP4y?KyH)F3TL*0ATb9-Jv)E`QHC(1KVb00W$w>TC| zws1I$lmloglyaLe?v}*2Ltn8C8GqP}*d~%k3wB3&ZrB}A8cXe5GB~R0QZQ~f+^cu4 zGzSfiBI^{En5t(V@_q9trOw06)orzYPwD&5?;XO5hQBl#D><AwE;Pv<YhO?IQGjN@ z3jG_4pA(Lg1?o|0)ZX#>k{qR&60c}&kk6B0s&i}dfg85hOZyU$jco;jVj{7POsdOC zMwKVk*g|bNMS5|4IxV6fK(*o0KGe`HjBP8rEywHim4v_Fn!S8R?it$hNnT8^E{Vlt zcny0#*}PhoM9rgfjD}lK3jk;WY5@XwPp~nJZ!xfunPH?QcV<4j2}9SKG5IvGajJTq zU*a|3mh9e8v7;6Ae`p_m`Z5Spa-@W`V3{9R{}&`(TG`kIk3Pdq9%(d0d$PrHQ9>#% z$w#rXGAnYJjvmED?fSiLV|1@wV%&n0+&O`8UuCd?8!Z18)0oF!dD^&iSUhU`&qsj9 z+lrfSuNZ1HqU_HH(m|UAG^0r1%h9fGOqdDQhx>lAT+D%6HL-NQ!G&NN#hhsnzj$pQ zeez=iZKiLNq{e72v%^^5+QB6{Jg6p46)E}<(0%BZrBx+18M&@$#phq_@<hmL8y<n1 zv@)v|M=~)u6`kegjM^Kw<hIn~qDBsiZk|l=oF=0GA|_ICJ1@4@ps~~Pjy#%))Lju7 z*_5_D@ZS|Z_p+9fA>@}2u*}{B!V|V!VUka*hNH{cUuE4yS20Xaz2B1sEQLP&QXS|J zrBPLH_f=;a>6MxJmE#b2z-@aPHfKuyqSEo}uZW3eC!<H+^rNXM+2}%f%?elU6QRYe zNUFg<N^`NPF%qB;B*Ry!o8s2PU(?oaeKGsJ(?$hpP$+HxvQr<xj=l)E3ttv}aQv^q z<xNXQi_zWXbSYpl*7896j-D78uWf=Y*V~{cMgj%WL#_bwAuhb=zQPxSjB6mCM)QRl zMMn=<Apf(brk;`$&QGEDgcy*FrrB71pN5k$rFLLnvDHbwxp@@2M-?e*!c!w@PrU-Q zBS`4yQXMeCCmIBnfVtv3;EyKsyBv>wU+XY_?bBY(q*sNhF@0FT3R+>^fmT`hE%PJ2 zHXB@qx9x`aK4Red&!2W>S%)!ZvK988)B#`bc7HJXKn~;@*?uo(1>a|`@3$P*@P<}Q znosuCvpIdg8dtt@yxIjol5+H3W(mCSWp;f^rRsptlenvqnS0%QX%9P;p90;LP)p8? zKO~;U<doh`Ly7$3wj!X-hC~$&q!_yxAXRpRvC^69d1xbBczVhSLd8b=(bZdBC_4&f zq;dO8Cd4I<*Gv_h+1YS8G49eC$o)NmRpnE~kXS(qd1gri6RhDft_W%Jj8yIb3q{FI zIP&WB423J);BdAy;d^wpYk`}4bG}7(?C}7(bHYx6ljSS#qt^eYakscWB#gT*Z-;>E z>a05h|3L{Cr%GG5E8g3szxv2=UCzeUX;Roc71F72_^sbX9cX=-@n;$k)3A9bC5rs) zG?d@1eWbalbZV<suJK;$Q<RGj#brE+cZR+L=K8!ydv_Z@P{**BM7*B!FN+~(IgTfI zEys{^Fd#~F<VR8f1(xtSi_H#EKGBm<`5h}cu-OAYiPOWTBMRFhu@9~9Au(J1dkbaL zhQ{Y{STtD_<&S=q8B6poR>s4<{J22vXnyGs9A>92tfpyWWw;QA1JTb)c#bbFLd{<% z+2w7VD8b{*1$xlWc6ZX5BMalHCDVetdB?38UGwehF-CNb$ZQ#l`SZ#@=;u0TrUqNG zuEp6M1o#szm!|GkBaEnWk9{TQ{?%^Rz0p^LHKvC_)lo|>mTlb)cOyKGXUjA56)^dr zR8HftE2lkVKmtCkM9fN7)kbGa(RaL-G2bVhp;e8cLLUD(X$9GjH5UQ7z{&WwOI5=Q z`6>&9h+RC0_XxQx_fE=oiOIlLM1J>6FRUm;l&LD%Eg;GU$u}>X@f4?|%|r7{h5CGi ztx9`_6viq2Q<K&3H%D3!?PU;T^|pm%m<yFK{T}??Ny?%Ns0H%m=>o#YzAXWm`Q1=? z9Q(&hO7?ctkhq=LRD0%!bkj2SQ8@_9WtRzEK*KqFryIr_GCt7t2cnZM1<wKJj?KK; zlJX#2o~=hVW@26(L6DM!T=6cH5jmoCT4I){ghN>gF*Hbyi>&Ur;N7n@+hZx9NT>*V zg4<rT0AZhnE|C}6@bhe-XO+~i83Swvw;ar|_pLPW57JorHg!-8V%P+p5TE3O#-dG_ z{egmDCCGHwd*RjoFa~t!UGaC0|Bh}y#gRa^po7cN+AwCT8ENd~3{$WfRe?bt&{??O zd0d`sk7GXD>46ND)WsTi0zMx?HJdMEkIq1A*zqJXO#sKb{qv(I%ISkUUr%lr4k*br zcPj=ZyiJfr8q(wGLX|Pmoga})t;>LS$H8(|I!Ks+EyWY|u)k1>$E7saF_m*bvwCd? zDgY*O8I-&kvs+0lOd<L(_H)R2fLl8U)MMbJCf36InT<$m%ME};e3`{F9c%^){=ntW z+WqdXn(t`tb>>naZmv~&m}FZ_6{llC6@QBCUq>epNmn5sN7O7f_5GpkPuTn2Ft0%h zKza-9x)N$BZx@^W?lf&%npr?p74+lb-Icf}FIBE3Fgb+r8NAD>rZ5(>m@;qj$_f2W zUe7qS<9Ck!1?<acv<^Y>SWjVlW!-J*(n0#i>$g`2B_==56*2fkVJy;a4T$3*ai04n z452J`U0&brF>O$fHu@GSk@p#Xc(xaJM4vv0?yUz0ekxN3aw{Q(4dk|ImN>7$pnM0U zW0WDEU-w0vnEN;Sk*Mzj-DQVZEqBOvLPtxn!oT;WXf%uY0*O?CWa3%Q{Rj^S@)>>P z*pX$4>e21Tjiw3rzm$8u`g@78t^>6y+Vs9yX8;_$U^0SMv>g6KH(pa}klArE;Nxn; zk1qU`yL&U$&TIp~9{xut3@BC5-vV$s?;1S{1@%5ubPMC(1^Td;m296CRpa{GdAVFP z{mYFSR*l9Edfq@3hLrwlLu;p4l<A;JTRIsSRe3w03X~kx{j(q==^PlKH9%yJV}{SV zDQqQmkpp^$fUCJjT6Y+?l^TY3`q4hxuqTd@%YLA?*=u=~5`<&#1$wys(&<5@DP!3K zLT@IL32C$<PNyF`iWN7`Edu~J!Z{P^JV4N!=3<yP-I7a-FF}Qqf7hALQL?D6HWQe_ zvUom_ti4wXdOS?|ww`F?Xn<zFU1ag86Iqzl+vY61ujFd|6;(&*4@^QW*HE03N`X<F z-WN$=DRpD&W>q@zLbml%qB=wW;Y@nkj^(EQRsKQpD1_djT%e})qV-wp(8jvmKfM4m zi$6;7BEk~3N5!M6_+l!<J0Tnn3VcwsktGN&G!h}o8)?dFJ@6^~qZJ<uIZ0KM^=i-D zFH_vRDT33_;1PgiS^T%`f|*YL(Q!W|te?r?M7p01MGbAW(+&L*vk<r=Kv&6^%LnQU zR8{a<-ldP^3XO7>Qkd%xj;f8)#(HgKuX`oOm7{pvy;=j8W-mjn0^UdhUlbAj1OHX5 zyIjaXg`Lu2Z=i4n-XwZAL$}`Vaa~zSTOB;#wcY1_1eplapF4Ockqt-6If>z~-<s1c zu3JL%Q3>#j)^2@Ed!Oauz_;O@x3=DJZQ38t<@IcTx^Cg`8;lw;gLJ}JCUVo8pPbWV zp!F>HXxi<Mh)E1z!0Cr{(*VDvsKubietm<_{c?0(?f6#<WTuqyZ>Na=kzn31z6k(` zRQZA2%YlXOybl^ZOz^F1JXrlD0F)DQ(9yTzIT7d=pk(k3&S32An`!g$;W}s#01wf& z0@IpUDO;s%#ujkIVl>OE@NSN~<Df8WY1sK)(B=qO2YF4q@cK=r&!Z(hGu`46EYYjv z`ay-0^UQhp_9BdHe%GrE%k_&l#a5$(ro#^xjjm$7sC=w_G;=KE=89Ig1}NvddU8M} z`|%-!`LA0M0W=Z5EW_6Ea_)#^9RK;M?p%f4`?Hzv<#?JVlGo75>Y;xR5QT6kXRj&- z#NNkH^3Ho6@8B{?G&a>OvJrsiczmVHjtSNaxWJ}XBI1jv(D1^Q=<;R(>4@jk^)aep z+*SM2_maHjADzOOZV<Af^2?Cuh|Y|H)cx*5XHZI9uF?Sy3)}}dwE+*I0ZA@5tPps1 z4gt1>;JoK*1j0^iTPls9zT~`W$Au&8IhS&G+1MyJ`r~Q!r*?>5%Wn<B;7|@V$-`ea zuX|D9gHCGn8~Quh=8=Pm%?(luVp)f|yr&0%g~=vmV;ZP?dlXr})TQaST@^ljAYjFh z)w0f*MS*r;gDmsdKjvEkd`M+uH|*8&BY!k+?_;i@qHb=HLI4bQ_IV-ml&{mlw`78F z8^pkC2VB!|OIaI{3oAbkyf+gk*y}93ed1;NMayEl9Z{k+H}ki-9R~o(teErP{9~ND zIP#$K6Y&H6cg%TQ+1#lST-lrp$w$P3MMnK*q!nO9Ht5~HQkC&}O4I@g09381>Q%}y zBpI~$9l&4}B!_!@R8L`&S+_RPdYY4X)*vK*8<OU=vveDGWWtXj#vI*+HsVQ@B>ltU z`*%ZRT#MYF;Xq}l0cPXQBfZPVQZg0-$;^ln4F6+_?N;uj&e`Mus=oCBJ$zWXK(sN1 zV~7lW+L9%$%k|J|g%i&tg<sdD1Qcqw;As*H)R!@uD0mZ$co7thq|@OV$1Pz4la9NZ z2(AwWf7D0&7U!D%_$kH(zkUyvyG-Oyj9VEmf)eADg=!X>5u*$#SBU&!>=l8`Zo`c> zZMbx*HlfdXD_XQ<>G!IrL;sg!y8AkdRn?~UUB0;;(n8kzj4(WV^ncXxOwQ;U$C(uX zklQj1^xL=4H@yV-AzA0Yiip3nBI2|5vKHK=^}?($jfa`=M(F25ZN(xwT~&V-^-82F z9)saatCV^{+lJ7fG|>lNu^x?32HxfHg_OjWd8H#ttmoSs!!Yu?O}<yNei+QIQW}rh zXE&j0zM%hFa@SxqwoEXlt4z2GWoSRufcqUa5=|@F04|`0BJI5&duN5_uD;`LigRI# zrljbxlHQ`eR-^}P2&}3wNP!8JFX(esXt-p(g3A-3yQ)9Nowjg4L@C!TXOAhS%+ZWd zA<4P(x4&;`bI)~bX%Rerk?jY}pHaZw?$@w9W%Ke~Xc7NGy2+&>UKPG+;pxx5o3B^j zs%n-Or)MjS4dAWp^m--;Jz%kF%=H<T1R8|%H1^s5dn96h%KPzT%c&KELW6}-FZvYC zLS}NM2pEfk!2g@8F0{MoH@YQ`1AgV}47SSGD~vlM1EXF1->Ah|7FRJ&%~(0LkTIJ5 z`B@#%3^F?g1dU+foMxg0*y<^AXkfSvi($y!Tp?u7qdk<6aPe}gMi)N0r+_exH4V9B zGUM=H1ZcOesAWRGr_XNoDuzSI;(Xoc_3?cU{W~j#?DS~j#S)`4QdnEgSZ|&)iMvew z+%JFBX9@JP`iOCHdPrlg0<_O81M(?ZLdnX$5WLxN@tuz+qjK$eZ!+5^^3~@Az*G`I z+sS66P4z^5^fM?wHhSTM`0oaP9L}@tx^#0)7UD4J+kK|O=qrm!LA4I7aKLRglY&k( zc55R<>)i2Z`E1USBJ|{_pNA00!)D?_e1VF^HXtUy4cF7Wsol;g-Ofp_(`P4JDMBZ( z`FL%;Q;>146(>&}s!{(2k8sZ(J03?#vFR;^FS;iW%c_gqxfI5)i4*_f_uTj&nD@`Z z?|_rp7|zAG(A^?@H<*cHVKfOb!+%%B*)4iA6dWbmvW>yhc)NF~@cZ4qM4mV31%N5| zdQsK{CAaU%p{u|5a+eMY&E?fto#t9?1>d$3h_uMn5kQV+O5rVWg-fYktdlVn7C<XF zAJ9bVhXJkrrTU-44IMV4H)B%5<yik|e&)Me0$l3lYI!1K0(s2SUw;o&d~^E@KT()L zt6K>1DNh=<IKT$%I-L0ks|t!=Ydj;~#uQ?h5?6FHh;9BU>8nVd_ewlfN|2}0S|<=C z^kVVQI}=dK14sWG@)Hig`CoB86_{e>$0hFhGkx|k7Ye;%gNb*9Y<V?-u*;aK-~e{5 zc03_VlO-}}A7{+4v-tMR?y?b65FpgVt-lVt<D)Z$20u`>YC`lH4zyZR;{It6GQNAu z6b(>JQ}YGMu~7Q5bB(?noeVg3x>#!Rwq(N%1oj;(ghu=$2kEqkPzvbKA>ICXs!uGb zdZFmftZl0#pq@6XkCj`!)yCWTB&z|E3Be_P*0PAj#j8{<Oa|$?%60?qOdD&~Id&+2 zFFVx-sCo45V$4@8bbkn~e0w*Ef+egu%T0(Jqilli&i3$8^k2`orm(ntb6bXC&5@=` zbHNZNRDz8xV(HjFDwfnQPg5eqWhO=@1rB~(mhcRrRprAx)!zESUL|u^^sk&!S-^pc zsVO_%vsj=x%CFMhYwxgFt`^D1)qQ?JOqq}H${Uh6)46g+Ajt;(Oollo<O<!(d{YtB z%YM}STbA(ouY<MP4UyZ0l7#9E-w4EF2)=*0_(m#TDaeh9INmfY3@Xkj_1y<Du!@Gg ztWER^pyyTKoq)8WO^yBd%9M$6H3|but{#KAX3sMvO&xoEu9lfgNcE=T1yDxLZu7(Y z|G@bPFkVA0)_4Kt%+3&im=U}u$wj`ssBsnYmvCBY(fsqUJnVF5l4B9YZTOr<GYgva zM9E1bgmr1=0d$;}*#EtyszZPY*eQkIbfGz3;l%CRJFk5$%#eE)HVn8$Edw<uxGs>N zv>QV}$dS*(;bFf6N!<QnNd_>{BL*|i{HKuo`mdkXRkg9YZcL39jxzV(U$*oD{KqhK zGV44J7RYJk(&1=f*1_av%Jxcw)c6Qs6BFK!AF3<gtQ6&|+s6lP+;yhoZ{nnBZ#sE= zN9!nB-5LAYXje*sB`nlDu2+6Jk5a<C@B4kNdN3f?WDU|0RX75yO=QXeEH|@GcgNr% z>6XoGxa7CMLC{MjXr=HjpM5tjqcAuq$LyCqnttrbgxw6FZNobsaT0pxiW=?eng9Bv z)Q^cBI^R;!SEy4lkny0!1Bi&c2h<sXgJ2p7BDSJD0$_941I|s$husX0rgK_adsI0R z{Iz{ByyzsXonO3GCHpDK7a|yxYe7(^23i!4TZ8A7%78F-N9RXL({6jTbg~vLS~*oZ zLpq11iw!40m;d7O{`v_|NQ5S0bjRc7;{cwE6pXC_*m0LrJQng(7n_cLlzajby+W{Z z&vWGU)g5uu-_7Lrn~prwx<;>?MPWa`VF<D97Lq-D3wti8UN=#VIoj5kD*EVHsw7Tg zs$)ZdR!9g)#42@7QCu|mY#ahr&z;H0Vgr}WA^d`prjM27H-28SBCb(f1zzzVf&Rh~ z%u|B{k<JxMyTR}uK@jihL~Y4b5sdZi{$I#r0GyQi%<Dh+@4?yuk!aDo4Wu|9#*ajK zs)5@Y$RL$sW;?9wp|h9K!p455(d)B@HEQ&L2FBtC8m@r5#VB1hI7(U;KS`cSDlatX zCo*xf)8$VnI!se%Ki5-g@D&;Q7IQ)?Zlb(>yJg&>O*9?QOAvpC2Dxl_5OM50(N>a$ zi^WtJ4;TFWTnk+_L?-{BA&NdNdzG!#AvIw$_)HQcG2`#6bd}lZh~E)&{j~al3o-~T zsH{phknIELZBaPZuefU-x5fgJ8tX1eN-5Q;_msGG3u=vxzG18E2ojO#4Y1wFIj2?h z)gy$|nbOw)dXQ+hM`e{1d1zI_&Y=77+}iDvPJDc5SJ^Kr{tGeGqLu`4mm$o@w!*0t z=_|KqbV1Fahr{fQ)C%k6TY9wO3|!i3bduVwZ1JpqtOy&$vVST;uz2Ei3sSB={W~u2 z{C!xO+RvJ9YQg_tcG;KKR5vP4I)1K;f5Yac+kM1nYazx0AkuAvv1U@^yfph>4ZeT% zQ4J4+Gn?|YDgNJh<OSzHuH<9KF&GT$yu#$U?4G?D+VnIidgbIWQe>9B((&KbW8Hd* zGtcQKy$-g7J)6%Q?*l*Pw~EJl6=4uIV^IP}@3Y773Yw|j&9drO?Kc2N%Qr8ux;y7E zAgE|8N#UI8&zVwUEQ=MSI^<}kq=-TT%brjU0dZS=RCLgPGV3(xwC2_wHk{p1+?fNT zC-{4{q#4VH{#;6+M_kKl(W@}}TJ<U%zU`%XSZKiBT(SgtJjrnN7x%>HzQ>Myx2L%R zke)j${%^1y9Ex^dTXZkWx^1=-O3?z$U`G+f^~JGSKoZE4a$f1^Gxi5xo*|MQVH7+% zxSFnLG2Ga1`Ii)b2w4+OsJ_=mQI#|0$~3F(uPVGBl2Z+KxI6NbWsG*14F2w`z9dBP zbz{jB5T2`Y-F&f$+rb!4Y5}f|fJlAZ0?of8*H-4J+P9~c8KS-G^Fa3HqxXx!8m4}_ zt4tMpsDq>H7ZP0KUfze+fEa|Pf8p+RV7TKyYM6lhQ#$~$_-RJ$|4Qu~n2$*J0W%Je zdOHzXzmxK|L~}o_yg4LOU@^^BerYD3(Hi|i{uz_H77I3&Oc0Zce9ns`7mj?5987+y z#|8%K=rJ$0>fZEmY4~ZOq;{tbaskhi_hrg)Q=|1=YIo-OPszfYzqW+U=YP{0iCnuL zdQ#{+wT9r5t&orzkA^*}P7$=#{H2ibg!;Ir^Woh1D$VH3<eOm3sC)<msFsMAxm67k zBQEIAfe(EuSOp##OcutP$Fd%t(6Vd`lsZ>zdJfikD4TkZB?I~$bn1_KOq1K9u?~X! zesI*+NgPW~XDWKhLzR;wiLaH=SLpq&4@*4{S2$^C1gt<cQWEUoGM`of?4rxH)!U6# zo}_4vSMY15=&HiBlkVyjl3x@#gb}30Rn0_5HlFw>BhI;82HB<KZU#Tv;^y0rZpssd zIwWU>Iy6nXM~>RnC=%_ODSVWvG%4TXh`UX6hFlCO0}`}=4DR$Xip%QBP2AC>zNeq= zf?K++<huLo+LC#;-|W?<{PB=LEd{wKZ?4A4b}`+O)VWtV{H5L)NHj%UZl?J`P|s5; zZQYq*!f6ltXnkM3(Qk4{zOvh`isM1+DW-br%Ru%S=LN<uv^N1<dI7&c(SmtTUzj7D zeB^zVlU1qUt<HKkM@|(SD+L8y#LP$FX!a<~w96N-2Cz&r&$77MF^FnU+747kJ`HEs z0Niw3hSg<D?5ICG(@ReYxL3}LpGN3281;OLdPESqv{~K0#M(n}Ooj$Ii&#){w~C1y zC1>z_Fs@*n$jF-ZxvI(dQ!n{8$3zr&TFk17%o|iAp-9^tPydn-3WOXB)wkz+i8p!O zW4WIVAq;qLPeG1?KYC7I=`PQJMAX1G^&ysH3GquMgXCN*zYY?{0S&tmfrc~vQvsBW zJ&}FsMWgS+xX8u??Y|T1+8m3%m|iD@)0*k_kME1F!2v=rK6L7rm0}BtpRe;)MX!1L ziQ364lFv3tN_HRRxvDO3$DN>Wj<xpr;pRB-`N9Yc0<&2AG2cNc!9{nl6RjOfPuvEN zO$GMAH{|oQ^R2}ODdpW?oT1z%&aI(2xX>Pl1*{ymH!=&_IrxTCXXi=#S&7&dYe+tL z$a>PrhT8{xl>>tLG!@_W-EWK~n1`2z*5yUj4|B)d$04aTFN2~E`4@&+wP!02mR)`o zFjR_8o7Lcct!&&>Ov^fde)OnTsyqUHPHb;XKay&}Qs7lB7i<AJ8`g7KRjcQl6L6zl zQaFqiJ$VT&j)6-Q3{t`A6BasQ3*ZY0NAJyzcz0wY`noW90EYhx<hAu_**K`o+BbTa z#~0$<y0&IQFb7eh;kd4VEt$?Wto+s*LPxo5EyPYc(ro4|Hb&ned&V9?&k(f<+|P*q z^^wYhYA`-iLnFNh>VY!BE62dJ0T9#e!mrgyH^_wj=K-tX8ZRFNV1|X7MvCS<O?vmZ zZ-7Sg1oSwFnwl7_#qnq44A=&Dk3U7<otF5qjKBeNbTk!*;O_&p(8&e#A;}tiTCcr} z*{Q4QdLqCm?K_I|6+%1l=5g8QVhQ68Wlv-))+F)fa9Y|6qW+I&R|dv>!*-wBek!!h zCFc|cBiZx4AVlSvg$>=V5Z}@-N5J2m1lvgFcjuM^aLO0EcVLP#(j<PS_(JQo7K0sL z101JVM>h_e?~RI@MY<RqnuY#9IJDXbrLIAg2Bez~<Kwk1mRn=lVID6=i65&AWI}PA z(i3j2jI4SnRanv~?-onnw|891NOvWX3Lz=SZ9~OmnBBlV*3zsFvJdfeA3CYnHur}B zrME0D5ohy<1G#EB^X{*<Jb+=>X-dbk_1DX=!m%_@O*zWb7CN=x8tez$a8e{6wN`;< z(NFoIh~%okxM_mJK-Drl0@#yJyXfdjo`Y;-ze!`S?2@j7{iD3Hj2X1!YXTS>Mqs*B z6SqX4-|i%0Lb{$mgh*xmqDwMr_LZFX+Dt)iWzu9ydJ+;T$;=zg-_7)UftR#l@=~N| zOA|?JG?Gn3NN1)E<8tAmyWm$&eqLSmPS&A9h7SCIYB=H>`ttVc2!~`6d5)qTvaB~W z=XDAW5SFfhM^Nl8nM2n5uY#pt*IT$Oo+cKkyvmJ8|LLB}H>kz5O5Xn&R6dRT^5y<f z`o|}zwDXyo6-X!WA=4x4Qj(+K__A;qL5&SoxBbu?E0lKN3&<OHl(aE{FH-iP8okcC z^?k7t{5cMop&x<-7x%gkF2a09FrrZYS+vkKWD>{MvK+mK*A{3+Fk>ui=QnFoB>E;v z=*)JU%M{~(T)-F!!&s>!EqrMWGnOK$58cZG?6gzeTmEVQ_+O;g+n^|){%BBukb^+c z`b_KP!-*vyiwfDWqr8il;rTFY=8m<o7Wel7KV=PkpGvzbJJt#K$Fumfm{(NF2Knhf zYOeAkJWg+Hf_{DQBVU=K)LfTGfFTDmxj}$UWD^72qd3l~grHyTs=+<5`A?Uny$s67 z@#O0>lbW%2id%z1mWrT@A?I^5nEquwt==^rMo#ut#F&wlVYVb~&0IEC&N}pI1UkM@ z_Qy5t<(s_OC0qvEn3vf%IS_t>&+n6H*0jGb71_T&3<$YsusJ65N<=>g6U6sDSKdvh zjSUWrvTu2OK~tg>Xp<CWoD6LECF(&uD#vNn6wcsyS8(VjPoF7fL!n5Ox%!_cM&po$ z2PDVWu!ENIdhUnmRRK|!+o9DrLHby*4I29!0qLd>1_B1J*8-6$SC+9G8YL4^wm?_r zHGf*;=K@~{Tdd0bo&7D#rr;p`xHolkIPPv|`5$tCj=hzzy!uWrx0l?mCcJXMC(6jM z6|u4Vv^b+gQj)8`QRQIi!%e4h&}LL0F64FnX1d#t4W+bm0eigu>X1FjGKu{st~x=Q zQJ9R!)oB{XDCO#hjP(|CC?DWG`j6)+{Q-Lmcn#PINs%okZy=rcjORO(xLx5?u>#n4 zi$L4cFS`X+u6o8&6Q*TFjVFtFRWGeoxBm)3dr(h#casEdje+|duj`hYZL^`*_Uy<_ zo%j1>;0T{EL{YbX87*n;uNmk&8YpmKj!Ts%?zuPZbSJy|4j)jxG4F@KeUI`RCf}v6 z%!tbcK^)w5_27wlHBITRN~I{`GoJ_1O4^}zk6nnklb)O7D5pwL0c&DPpl(3v5tZdn zaEh2)pyc^{MY?%+n_>>y12PJ80aH6K=b)}6IGh3clnyQzf3!BPE(A<NG1{Vd>m-A` zMouc@Z5Rxz=GefgpyrwB`rm@q$*3m#PnOd~dRRc1HkBHYLg6`)Pq%a^?Sa|aP3(L8 z#b42y=o&X*^G^}Lj`jm*4)tv&REV|X3;#uv5X<X57wHJQ>*~fpS3=En8=2iSgbQ38 zgWX2S2?#@9LAtJ)V%mHv<)V4hgt8+Za9-U_yV)*fDc%Jz_+9f|901ih{=<+fJmk?6 z^|eFThTuTG+$puw`sZzFP}xZ<5fMKZYTG8)@$QEJDxoW_<R&Xf`5>v=smaFky0>&? zZtpXK)nIcd1s902xJ*swC}9aaovrS2+M2BS`$u?xJU8kP4X@A<z(qL$XU95y7XB%| z_qzA5j)9A+)lD)r<dyfjZ^2W-S0duiS^qf<0q;(F2RrcUh@OB%D42~r6Z;fxgCr96 z0&A5+kv;^DwvonYy7q6}&FHbG{}908kDD*VY~jxZ{397f;?K;I-HG52D0|PWZY{(n z9Qu@eQZQM+KN1mb2sy2N%#a(PT&hoEwGQ};0zAh=j8K$_n9;{ROjcAG(U(1&YW2P6 z`lece@2Htq>p-oTyC_s6dmBylqIol|u$iB8TQ{}qD<ZADn+Xd7zS_QMmk@I4QL;BG zLnH#4_COC36RUQI$Iv9KhoUXII(AT_Mnx+2;4)nFSy~h!6=K*t=z*eOqk&U6{?F%C zvU6;`H<%Zt{T_C<&A`{EUxBx%ph(E?TfET>cUTNky=GK$(yA><625q6!IPQ{QaedV zcW01%iXTucKu~$}Gbf|k@%w0$0f_co4}X6Zp2RI5a|I@dwb@0RPLfk_f2kaYShp4P z?Y>Dk!>vB#3OKGrmVJs8OrIABJyPOTj(?MI=1!e8bgqzveVb<dsu<T`Vd*GF4W%7t z`BCcIqtfv`+*%>C+&!&Y55_BH5Z3|L0{MQ;2gLHtE(zbyC`;cjA;XZqN&?hALuIW( z9D7w()3RGuZFpC8NQQwBigNSu_VGnq_lLc2_yVEUx`_+pJ`35(2Rri&C{Gr@5!YA> z(9-Z)e09~lYe1-VL>WR{fz8)Bq%gT8^JqTYUdkFR+TOOneldDDHR$re+%B~xD~$~I z>(M^mL0es2Y{vfT<j2f&CrZZ#`BI0j!>pc_LGP?E>EX}%K9eP<=a=>U<2UU8zdBsX z0f4*Zp`kaTDr*wpjDBi7Ghx#I3JqJqjOHt6dMO11P((`ro0vUN^GEn&05SA9z1Bep zbjy$i3(%x@6H=MbwH_9#cLaruL#R(U;9p|+=PIpD!&}%3iVX@d!zl&n-2rY-gi)|e zPXftY2tp;Z_Om|T`flz!E7_*IALkttwIdZZ^wOk+QXX3)(j{^--5LM;2u)tffI1pf zf{&Co!Z2vYxw#Za&L2U!rBmL>D7VUI{E-3*c7>M!3R&Nm(rr@<;0JvffDOFnW)cOA z5JCQsv?J4`Zy8W$|1K8r^Qj5<fNsf;Cr@V%O6@@(nqstNmrWJZ$o0o7Ps)~o?IZ+V zWm^zwRPT&y<+tlZmfG1*#@&4Vp#@jpZkVaNDzKT5>S4Wov)jnh3U8UJhGF8Xw)v(X z%@^C}BW0a2qP;vj&bTK#3$r?SSO;-Mb0C?kmKN(LmLGy(jP~m2umEZGTY*PuUOL=r z3+xhWZC%`+<C?5BSpA0>)C!@j^(CQt4!igzs&FSSZD*;|*J3mu7q9#&cvY*jBhd7| zEv-IVM?A{4ziHW|Ri+SQm05mot(0`FwJ7<3RdRlMuCeL*Na?9TohxgjzDE|%pN(Rf z7vWuDoR?7_(dZc@X*Ekw+__woO!sihnrc4#6Hl*R|ANNi3zFvZ6plYpZ)!7Q!q|=S zO8VSifM&fPc7FQbd54H><}uk*W`EibrRpT&Tp|Pg_is<jF7Wmc8N+3(j*L0(66I!> zHzDTqUdUO@xpa#oMn`HVoPKO@VqQrl`ZzJ;2Yh)1V`af$gYvdxjx2)JNU+rfBGy+2 z10vuau2ytDks!3RI+oHvgv$k2)RuXWlZq4Bv|1Z?Lg*#L2sJZFQ2E3N`KkRiT;hFl zx3b&S2@OL<(!oWgL3P?*{G&O$=m1<s91Lx24V+2PoD9(u23(=&Z+4q*)=l<6vs7Tl zfr(essH1)9XWOCi4g2c>3Q!yo{l<E!xz*yu09}L1hryW9TRRF`l}UwvdI8iD>^6n! zHyCLCnA_H41N21vm!~cWIx~jk7I}Lk(;!M8Sm;<nx{^B8wTB{C{Z9hGOov5l3ENB9 zz(XAP!x&;QA#uBG5x89C%n@#L3oRBwLOiojp*i24wX}@;_lVnQsU_Ezd;KQr)&@<~ zAU(l)@matN(P}%y`pB<*x(@@<nA9ENkoVN4)y~Fn)z~3g4cFFgI4w)79OMl6r<<NR zoTadVqk!k+PSt+Nn|UF0{?58_VuLpA^Rk(6&bqgJ+Ql@c%G?<te4VP=gBts4EHyYN zXMVBPmsMQ3+J&wbGw1w;gHlg@4Q;*kl4-o(EH}f*8PN>6PL8!|K?MIGkAX{`&~Wk; z^rCiKGV&@+e>W}df*5bBa`_;L7V3^*yI2T@zkU?$1NA}t9n6lfW{5dE=Wl2bXm*S? zoj5o6lIG>(HhN-KGB8IEUW+!cRHpgQeT<0gsWi?!;K|_yl%|ef`Rci7GGZ?;NQM1j zz<Oc+IQ+h5n}&$1FT^3R+{L(aRB}0wq*&e&8%rPzRE#I23|B8+z%&*fY|mVJJfPg- zR(n}`9p4D@3n=w0GtyxG-P8rCZy%3M#R@M@^nyLP>&t7<hEfmW;xpY<lP-8JGi^;E zu-~my3)+AJZ^)i$jCf1SbSL0kJz`*>{EOjS-{f)eiV4WOhB($MyMMVY@Eu9iVXyh5 zH1gPg?*p9c%KOtfZ`;gQGLCOOjwrut$BU!?N7h*eH2uF{e>8}cARsjm1%qw`MoCEv z3erf&06}`dXcZ6;q*J7nj?rvLN=Vm64C#`Nk@v@6eE;|TzaQ{G9{GU1cU{*x=XI8M zfotv$zNRgFU=`C^vAhV%fA;y4Cs;Oi&|ES2H#}Om_|pxhKC`DXXD2&UU&*i8FgmKc zDZOH*W4MdRdAT&wGJTf^aoh{JFw46&?Ie({w?O4KuZZKADyu%XXEX7i)?6oi{XLSM z7cD?+I8(Ri#MO-@noac9@l3JCP|mWzUR5o2i@Z3Qs-omh7&T^UJsXw7$LrDBCCbYA z3zJo&Ybbe|N-wtvs?bifo%;y8j9Kk|LyM19Gxt>bAhup!E03Sws#PSv1o6^sy?lHD zK5SXI;p5#E5jFf_aT=gsT3lbAM7B6?QQ4fx+|Y9LWfH;IaoTTG1)sAQ966M4H4O!x z+)*d_vuj?odocO4X$x2rKV3tB6ti`94KQD!j$fQae5g%FN%CsLfhAtjF6`rb*rj$K zFiUaupE0>epGX)?+6D}AV$#rS7%vKl7JzvTsP!_Udc2Ax0{TlicueZ4fn><zDcf{t zjzPJZPOak$E2&ZCSpFNQj4_AdDK?i5t5-07SJXv)Yl~P}kn$oUo}|5DXU9M9+G*co zod=*-1WgJZk2~7*nMLfXmDsRH!3{7J^Zao-JLQvmeb-6Pf;<FSA0>l6ZUfyjJyD1= zmYHPe4Wd*5q_o^GOCs=g<;{V4i8winFU3?6_3|JBJnORdaTbjQEWyNbZy%H!vsTeU zrK%AhsiyKxn+<W|P{&_WTsuj(J!%t40k6S*BX((^oO5DIL2*w8zoj@GfsKpBPGeQf zk%jwjN}f?q5dVs!pCP`JIcWC>`@|wu=y}n=wdN38n^o%p70H@p#}bo&bKp01f*q59 zHcM8ZTQI1igqm4LHhEqRJb~q$7y63qxq9pfU{4k|iAcJgQSlkVnibxpur2(1`G3kp zQ(#leQdK6jj6X*m&eLlAS*J-_Tk|qQhbD*yu0=u5{#g0=<NS&S9881<7?w_0<Z6Sz zaK&jU7!~+xMDaY@o@BRo;3v6NE2*0vs))5}ARb~<R>*7J6+CJ{C_8#&kXbB2*@K4A z{2V$)1g9(tO#9@NUwPBAbz0n6lXLhZV@=_W+aZMM{Ck4AcwZ2gVLPol7{!zodT8MC zf#m#P@8!b)?ny1Qut&Cd+oXxzD5S&t>hlf+;#5fn6KxsAPOa`doZ+rL0gd#CayubB z%*o!@4Df^9#=qC<r3)`hh?znx&ZG$;V%x8lrJXS1sX0EFKz;um(HQ1!4@s9L8$u&6 zpI?wAC4B3E(i%T5-~@wiQnWYwvjYiS@e4Y-Q(n;WQuPuxA2}ojX?DE}cit#I_5J_0 zx3I^+*g=It1ryLd2Kz42NC34jQDeV>Ns3wA<)(TPj~Y5GQm8M9*ZAu;CNX>!)Dt92 z_0e&PnjNfDN4n6fPvb}KK;=ar`z1=2@0uhdu<fbg#rX9XSrRY%?a<>8ZtfO*pB$M3 z)v0M>O^+TX@&|a-`vA4|^HT~@nelHgLV>!gl)R{eV?FEDxV28co~fgj?^IFv3R<*n zX<OFE`sLmRNk=qt5ri{$1%}3J_s)w@D<z=dd<tf<us^d+uD>+ZZ2W<ZjdWwATC!gd zI0#{aA?=vNZ?rd4H9-3^qpiU(OZJkH!e|j)ep99}&`>sPpRC3?m3-zk>j@Pb9#jjO zq-Q$c8jx`LX?wcqm}VE%Zk5}XwQRET_rvU75F8t@AUW`W@Fhcl1OZz^rQ)y2JeAl8 zz*p5al~l9)M%6s^(O}^Khi@{U=?wr4ptLd)4Lp7nfxHHH0{NPlMW)JcT$trPx@aCh zo&&KVx)~DdBz2j5M)lc99tg&`5<0oGM>`D4&oh_b+&_=&ICXisx~p7q5O}mO#Yq{O zu(6$Pq93B2)-&%QuaQ+Ax4TzT{<3|{U@mO%dJn%pYwts{2x;+4SFd>Qpgo0y%+P}) zSqijgTlYcWDM6$87(M;qx!p&|8rW%~MAwtELIl{B$R|NT7d#yd`{8yCtgTLE8Qu!} zYPtukQbjLR!#s^ITa$@*g?diRz{wQs-QN1;4%}QH{&VFnLeb~Rae%BTh$(0U<}VBl zjM!B|Elt_MvNO@1=WshNnXj`(Qm6CJ*S+n<Kxh?mUl95P3vrF4lFqM92j_^lS&LmS z1Spf_2B0-;o&{A06OB;zKF%xCeKOhRyp57jh<^J3O}zVzGy;B;!g1vAAl$SDvVw{s zWAvWpxD5YtU>Q)RxKL(qTXq5a7>BOkPlNMIYEQw2h{<suWs3Pp-UQrDby$3Xb`F)$ z%pqP+rGl}1!^7Pq$$?sFD|p*~cSX7fEm$y-6t2Gp8Oz#ZGqsU>XZK-C`=-3*VXGD9 zis!%V`g`AS<+3tezkNlB&cy{g{rc+Xd=MK(4@ISmdGCw*^n@)`TrZH^+il5UF!hfd zI?_wmGs&riZfgJ+InbSJdf(N0f1&Nw%OYI$L;QWdBd>u56iPI?9wr|+h^NmCa~{$7 z^$;OCmR|}mBC%Ow^hG<Gn}UdTvE*mWf-Q$uBR^IZLV$tEjA5YgFtwN!O+<GKRnQQ2 zym}f$VKKfTwEeT8^lQ49E1HjZ;hb^4@7|J)B%<Z!d}QOX6|k+L4YB9G>rw94-(nho zZ>#xQozQ}S{oX!d``I5}Q(qPwH`5(t*BVhzt1~wl@tYP;e^l-mT^GaF%bQJMb6o%z z-!U{AXVf<T^Ij1YeXVLgd034&_Q(`P-b_`LwEsh3+=ocpN?N?=p6<(?#EEmc?}|=& z`bK&E3ON4V@DsLn?nV)CvVk`1UY=K~^j>%pF~N^#J&g?V1>NZ!WqEL=JM6vI@%EoD zwtwFZ?!O4GwCes*S~r)L5=pYS7rXecxk}?`A@_Oo#eqQ}z>erXYuBxP>1<!6UGpkI z;?~&!N!ggi*pSx4uQG<>1cv1e4%E^zHhx7)uhz9C*V!4LY{k#4^|uc3#->I4<*y?Z z<Z4S^Xr_y4ezBX*a+z&x1HjUxQrk5NSsFFho4CA%yx5htKjV~<iXSCNdJ`+3o+ZR5 z-tRC~C6$bsOHom-{N<fdl1?$cX{{nKe=CrLhLe3&^s=Krh=ynh9Dm(wg_Mw69JrrW z8GGJ{TKw|CuOB)}DTzJV;Xmc#-YC~j_qo?4vMV?dFXs3dn1$Q>Q-yPZazDod+y0bz zAj@&pYBORzxglzaO;H<i%VrI*?)yp>?sNYs1ODi<cGA;x=pVYZap99MVLubN6#-A! z_;DCoGo0=Q<ZAUdICmuInnJ4FoGf2GIYQ=mir_Ck<ep#A_FnL{0Qx9J&<a~zm=%IM zSd}`jZqrp!Y!b6;2LRwVfjM&b8<-Xs%}^>(IyicUM<V|{kszhXSw{cr$h=}SD7g(- zHvi({h?`4_UUz<YRnT*lJp45Ci{ve=$vXxgi;nrfie}(99oCg{p3ixy<VLmYD)Xsl zc$ip*+)~3z{E2dND$JH45&h?H#5TRq`l4R(jqk<9K<zX6-vp4rQ3HeJim~h0Pg`Zl z42sno#v}u?Y2{&ZHQ~FmlB-P@3`nq~m$U|}Tf(=|o_J2Yc^gsEJ9tY905`Eb{0waV zeFRlKbt*EvcwZ!AXJ(N}sw$!W%HeL8)20e~Po=f)B0n-G=sJcb?z-3iNIyZnpYET* zp4TK8hG!iZhbA(2e6G$YqUjxmDy)KSW9tWq&63HR-_$CFJT36rMdtrLw*oFY2?TR2 zA+@Zplf3MywMiwymm4gY^)R(P+AOG}@tdQ#r_QytrpoYT8#L^5yR3BPrUk+ao(;aL z!{cPC$T>o)$<p%tCeS!Tb(aY~uuP^tow_~HPr~-b!F6PS5C&+JO!iNYa6nlnbN*z` zy=-6d(W#Xi%%E~5{=>Z>+L}mT#V!cHit=1_j0}VQvQs26*yCOIB=HE(+ZGX}0gr*y zQ=ol)E9fGXjXBYB-_#tL|D@(9Bs7fc)Ub&??2FbxLmv_5r#im|Jh5Pz{Mv`Cv8IXV z;KdG*o&QmdLv`Q!FtTDfnz2F9gqh&K;G^sC-HqW$U5gT>l4YL`wK&H~yL_0yAg>j7 ze-GJJX?zi?Ax%peqQ@nl^KPiNJ_zWng`}~|Ju?4j2jx)(O@1w%NOsq70~s)z!;q@F zLCa<xBuoXO@In7W*KE)C7|lTS<lWtOhg^)>X9b9Cp@IDUTLeb3s#5#h>*+Iv*2sfK z1*{d1UbSA-%l=i1UQYgmp{{$b<;~729|``{tvxQQeSvoHXGl2s?y%)XZ8G{f)?4ye zbUxp&HY%ci@T^P)&Gy5NuLiiBOO3)%s}C{Fi~jFFh#}P}YSh1-p01qXwv=kf<Lp`U z0ow6SNbK^(ugj17PK(bz22fm#ue`!YEc!ob>@U#S6%6vuIU$-pK@(9XzWifMBUd22 z=YA=ItC(rZX+4!9yhzvSu}T|-^*&hI@fq1N-7n6*{<A!;G%DYEq6C`mCU~9pzwTK@ za%B2g;1PoJnKs@_^87P%=Wo{YZf%tW-jYpUQX7p5OMFeCatxC|fM1}6L;MXr_zw?| zbu^t35*_gXdAdaKl%DDAKBK5+n7Olh?)Iy5j^_Q}g!M~lh2vQ$#RDIKR-iNp9vNpI zba}LdDt`EUBy3?jcleE+yjG%}xefx*{qQR{ZCycDdp`*jz)#*jm;ti&W@aA_RDm@R zv_MYy1Tp95H0Aic)B-qI?T+>iE*yk|+cUJ2+Iv5LukLMH`x1ry@VKl8k51^l$?lWM zn;51uy52$`x=mPDhC2;9$*n>XzbE%2^XT>M+gOs@l>rWc?@d3weMs4p^7hL$r%#vc zV>Q38wMu71{Dl%It8asZ?0ShkamV2yNQW<Gjay{3czx+lJRR}B|0Lm;R>0Do$Q%L> zCy7y=3sf}kht_S&p|I73uuyOB)byi(2OY+Mpr>Fm7!Ae)%oRT?FUJkc!!J+fWmi4O zO7H5d5Y#L)%h1nLkVx-KNFSUsp4yB*d$HN*xv9|oku_H-ie3ldJY6Evx;qaT*2vo< zD9AU|HKMN$u%T262P*^P%?_~J9OJ-FhZhK-{|G}&&2e(4Z(pqQw3zM|lyZ;?^BEtI z@Lc;4Hy?J`8=hXJJGwh#fPf=86oUxA#x!v0WLv$yIJr=xYCDY-{{d_JU3qh4=LTNv z^F2Ol+IerFQ~DKZZJwN-M<>0q&t?1*`OPP_Qr#DGRy3f#PH+4Y&iU2iI@w^(Z_~Nn zW-7W*`6!QM$+jE!7f*y`jc=SF<}3K)1_4$B1ueH$m{F}`@y@YN1M$SeJi4ZhPVa^3 zYCG;^>_Eyj#f7e|zr-18HutCQkp*z{9FQB}VCQKo=kTq$Y9EBHm}qpbX2!vu+f7ah z;)EuBr=MJtZX(VCFnoR3R`G=$8o6KISMPCo|CJhd4PLSX?hzf`H#+9o1Vcu$OCI(5 zQbW$0PQLZ8*gjJ2(UO7tfQ#aVYcb1Jn+S`Ln|w}6>(FlH;6i|7l-Fq*xc_807R*?u zlvRbh5m?yP4cYxjjIi+K1N|TziQh>OgV4zR3<UQ)-SoKVA9tNeuFqD>UR+KW83O6d z6KwP4pvEcp2`J<af2vo2Nm_wtqr@Da+B%`nZmB*VD{x|knR<g)KgPz~Q5?+FtVLUs zEAQ`s(qx8hjSLeSg-=;BQ4zL=|L893yI~!)BJ6t5T>t9~jzrhz4#K)9>dZr9@Aq31 z@Zo$oRNw6|m~3Vdc96#HJCGEx9!!MYghe50w%nf_u2n9))?-*IjOhc0F9v&P8p+q| zu&ufZtqcEkxK=>I?>AIJba`^g)1ay*>>IX*_^oy!>ZO$c`Rj?UhzymQqF7kgQFyE- zsOC_mRd#WRHMT+nDy*crV)G1K5!~d?<KB}58>%IVx6&~l4d|*vtR_2sMIbAVz^+ZD ztZ1g3w!W{`ha_VIVzFsht3OT``^3!m974+1J$OHr@}dzyaq&_eAF>eGFZ1>Qa5z7# zANqTHwQ=Nvcl9ANOcS~)S6DuuKEdJg-HCT%g1<q0(Q1CF@FGAdy)dB7fVccme9rDL z3pJ@*mi_%Q<>lG;m7rUEtI3XTnmz$Avraz^r}q;i`?Ka45_>=v;lVW%?BfOw=gI=g zf>#UnzHLwA?l2gI)?QMd#P;oMrg`90)~0zS4N{I7(W?c4+Qt!LveUi2C*JyesHvj> zRVk>0`sMqe+S#Yt(eUQ?VX@=N|MJ}Y^OVvR<NLrrQ%YLB{Q@+cG^uVr)&lvCd{uJ; z%96OZ0jQ)Nh@%&|pNHS0!djdAvvt@TihujY(AOst0!C$KE*%yIcDIMv2;Wo5HbVf< zCtu;jhuhx3gq0l*yZ}<0O+eaj9zIm!Ro}eQ`eI?ZTS|WYMkE8hf)+uId*;Xzi7K3^ zIRivx$FLmg_Ji5A8SlmSm!bw%2g@*yB?ToxKKRN5xTgk#jf(S+Do%JaS!e8vWJV$} z15g7nEh3+bSeZu(;~^7~_56jv;aWrBGylzdd1evHv11}sz}v?ts6gUEW}W8;xUIPD zCra+X07hLG03Dn0+Qp_k9NOq1fSKM^<6c-iX?;cE`6xMp^BeSC*&{HStCK)oIo_@2 zro*nLg=I3TRqO7PXE0<URpgpTg?6>#4=0XcS>PB&0nDJ`Ky!V6-Fvc|(t#H~a_o1& zlnc{Xq<ko`JTmaId2{&IZ~sjI>-+gB)e04^Y0w7;JgJs|eoHpg4-T6x545;xGFGGY z5j<W@XDfylwmM_p{at~<M&_$G3ya%diPs-SG%Zv-$``g+)!xyL0Ak`k$(HJ;({|bN z4erZTXS{T5M|<*fRHtQOauwSlk4pQ2{5T0PcCI>~jO45TWwZwas>j@uYzgw$WWAr~ zDOZ9nbloiz2NU>V(Vk(-d`VvU!Lh5n=2Y6p8=!-^AQdPV{vzZ3CbJg53U{01PC^N) z8KLFTd*EyjlY`ahJHnrQE#{hi9W%!r$HE9c%CP=&Jr|caMwtdI4L;?YV_8_Nq4vnd zE{Y_J_h`jm@t-JkwlEyHr=>jjzhrRD0y!g)<a}smxI~I$sF!I7n)0)gujb!34O%Mg zTQpDatA901#JPv9&TsGLxOA05eKa~_k=7;$Tj6ebtzy4W+?${*jf5wNx%G-D*O%*s zsb3(87aFm$C7}ed5@r&xBBiT)O(t@91qXQR_rl;f9R>q4u<COK>gReCBJMb5{&qqo zOQ!jTZenb2k2B&ym~<mY-LRO~Js9inrV*-LyX>O(Gh&!<q9qI;T#~SBvMaGbUbVwl zlH?R#o#wU1UvKLMA6ZdM+ld&95vyM@#Ta56Ex`^Ibxv2qE7~JnqE`-oV1TRZ6}5Kc zcRu(*la~e&xUW{Ox~-<{_6^8!rg(j6{>mk>LCriP5flG$ZLnH1g>k^4F_$|h7);Bc zp#_USff`YpmX*V1KRHEvJq&SzjCSFFj+!6ltR24)rhxHG-fd-TnxG}Cegl=zQ3pmj z&50Zdhved$l1XG42~Hmz_$G3+it#3cPm&1@i_1k+Uq-3+a^^iZck)U*#z%<GEQ>d5 z7PPRqQe9y@ABNP79`mhQHJKb}08Np*(9jpSMtPe(gh#e#>1>%y*yA8l7|$v}pf0HF z1mWAgdZ~ypwztYIrxl5Gg8WVs7unVj>j}7HK3U^S&^LTo5%buEe}^HEENsPp#+D4+ zG{KfYJO&x{Z@S2cl|%>aov1aGk4(J0@@AFx7=ymm=B&`Xgt!c)G=r=suX>lLs)7`c zSpYSOGk5va>}`H6!FkTMyq#_9NXQ*Py;{NBYx?Ig`?sGEm<wNmc20`{JJCwT{U1S$ z@%cqrzX<`2Q&`!ji-ZcE<?%~WX}si?ZW-;%!<belr<kz7QDuRGCqBLocMs`F8CivG zZys)pt3z>{34Gy~j+q(cw+K)`n+X-KF`{3xB$`NOtd2QjxF&?n7tN6PdAmJiRIY;v z^!iO1>YWP^58{6w1Z-!03jjK|V$$-!rtB3{_K!4jn$CJqnpF4O_~VX(htl{x>B=XD zPCuXLd?o0i5#eJClFy{E<!rGj1(0q0j1>$cUlj0q2tTO_3|W)6$r743e9=Cuc{oP= z=-^M&PDCX57m<r=<2qM8umcLeF!vmPp(dh_@#uSJm*Y|04(lKI%8_ydPZf4{5=3nl z<Se)EO1QEyR-7>P<_8(~y=;dHmJoO=+GFwSpQ44!$+sbca{-)%6hOaJ$gKC^N1(O9 zC>pkD6&e<+2x=CehY2^tJqrnM4FUSX7vVJH0RRH~-zSF+;~G)Z&JVX~;f%RnFf-GA z?*5EDdY*u5rK#e60IA^%s5|nhLN@Nb@{IJBW(hmFvCTlBQYUb-CBX{^8mRj8%Dglp z@Z>O<%FI{jg@3w;%pI*Q2&vb+gE8u=<0KmJ08{K{YC!B=0jF8c1qW|#FN8#HmGg1C zIv9p$aJ6k+`=>LCp&cM5XTrFH4K8>FW-O!TJ{;x7TGkwh491AhktUlW_J&e<76JH% z!M^`Zj1s{5R!Xl`{bTeh%{KQ*#E<rok3*z?M0Q$ms$iexJ(_P%Go`$>_@;lT5XAK* z%~QIhpE>-_lQ0<0!`%*y%#m<B1A4)Iy~)a!|7KjBk(b9^(vXAL(jPU$i^Z9Sh(Qx* z6^Ewe{wInLX>5MF3c9H7sb8e*{!Km_jQ_JfKM$?}xkGf{XpnqTd;$!BPs~pxvK1Sq zl*w*20DdU%<msqYE*txbFf!)^_M}a0%LK!%jLNn=X(aKG<tq+>^fBPpJ#jHGNn$PI zV4?7e!BH`e^t$e072l)~9ro*5SnE)3aSP#%RP3#Ud9^9S5P|~19Rlz<i?TQ_>v1_D zv55I{V16*L-wu0kO=H3HLQB%6G0YfP7jh9qbCf<n$?KTK`>AGyoXKZJPdwYMct_;j zPf?OLvRDMBi`O-B3{^K@baNom1Y44nJje_=ONC8}x@a7<ntV4Ao8oguRqehhZ5R8i z>p~Fo0jE$#@wH)%QEEBJ{z#YnCWFlcN_G8&X>7^yy93*2koLRly(bJUNb1HK%xRNF zGhrw3ljl;y279jI^@B@PKlB(ohu>C)JKS_&Z$u=GHrj>6YSLJX28i?|ji^;5m4_BU z7JW5zt!m$NSeoKoN6)@*PuGjN%Px9w;MJOjZVb7+ide^|2F`xg*Ns@4(Yaq^2JV_m zvR3}Pc8wB5L3|=Gp1$g7Z4dDLV7w;8=@a7muBpjxGg;2jdzc2o;UMw5S${jgjF_qQ z!c6Gs$)-*48-sOU5gs21`9LHmOq|%w7c~x=#^ioUj;#91o|X^+$4cu2`US($baRcc z)@5KvhJ0LM>VG_Wcfdu`6*xI^kxm@W@yXT&PoRzX=Tq$|8}sL(uLSn6xV-3OYTY{q z6Ri&l(_w>aUuQjZ!f_~M)y?d^N6cx1J4Hp?CP-3t;qb-Mu7Y*`K(>GWu>SQVR!BaC zNAh*YFej%(Q*$YM@_s9<2Q2~H?Cs<Eplc1`xuENSDNh6Bnvl}?@fW^H@|Qe>VI}gF z4)WS`_sw<m(6ppViyW7LK~`C2Y<98NBlr9^6f_ZFPZ&hwyav?JYMqEVj1YF++-&41 zx78zs4FnsOIMB-X7#_uOD1Lh2u5|Qv8+Oyg<MQ&)yD71E>E}<GN^dMAS4GPqJGRWM z8jGI1rv1!}x1@OwR)To?Q0EqC>91yb$Se4zvTxc1E9e;E#^JE~sC;NX9kdiTpAZ>> z%*Cc^r%T4&zNN{~2RZrlM_}{wL&<S=>G$WdFksgw0%F{VH21v#TI_@eGcF}vt=;ms z;v@VSSwA7^X@gi6^&MFUPL*=Q%De9y&w%=EIbbjKc2Lv%uNT1b{4$o9GB`fsB-5Dc z<z))1Bpt;Su)pTGFZbTTj6SVSj28x!FBtNEy@0B9e7*7fu+C8hZAs3b8i!GkWtW1i zX_HrY!RiTowE#)NS=I+Jc?*P;@Dv93<w=dsGYY@ISk<wg_ZHHhZGDSx5Fss$G$fPZ zyI|(Iw$)WdWYmamibkj?$G{z%oZr2SrWQJ$9?Dg;1cKSBSf?=L-wc3o(q}a}zpC4- zasF2?sRGZm%jSo{0=1;;xkEhY`vGyGDWt4*F0)VgSp|fT4@4(T@{&CMLJ)y8+vjf| zLpRy*Y#tj#wmjM)#k`ned-?er=={Yr9_5^siaol<fp`pYyUf&r`dePJ+>hJxhUf%a z3Y4$wA7;IU_x?@A1B%EWo>6z@oIpa#H-j#Yrct>D1LYMV{KyL-yv|V*FL<1c8EQON zk>tUPZ~93sxkl+Vd4mCyQjXj!eL!p5vc)UNk;iq>bIPiT%%NbN2R_DTW7K#{Ls{?E z|JjBsY#zoD%l%rlu;aj5ZE=j|`Wo!0<qQxFKa=2?S8NbMl0&+Vdyq}k&7p3wRPVma zr<8fAvo3RNh)}a6sR92PX}GVmfFr3Y7gdhcbeZDm1>A=zMdvJ?zr+4XjZ4(lgL922 zw7paL;e9f*z2@A&$jhfWAhg4#cquZUs_87|oMQbh=Sb?x4Kvv`tu!QNg`Kv3Ve_QG zx4YY}*p!8u^AgudR8V>Uz(Xh0$##>V^pfxm&d~yY5GE3r^Z92T^nh`4(0j%`VC1xf z7wI&k8ngl;4OD5}8W~&ya;_8B{tsY0=b6q_4Hqb4t%em!$jR$;!4K?ajq2Kce1(d@ zk0fVN-<3dcxO650++#OrNnj22zi-A}D#_B-CZqb@$_P%}+AF3CN<CBu00ZVjBDLGa zA}J|v-SFImGD~GOH}k^Yklhm<c;5_O!zVR3G#tn-f|Y5e*{=b2N|7Cl7Ka6^;@}zW zw6muhTIc4z@86L@xJpeRKz)Ry3smQ;iN`X__P#OnW6gl9NOJnza*d=4fxYIg;ChiM ztZjQAXxti%0A4S9bS?2z7sA0bIiRDv2EpK6+RlZ+1tuZ8Q|EXEWwM=2j~rW7ZB%zn z&r;U+@rvsPn-a%Tkvq?sYfd<T`ydl8`i@5XrCC(YR^!_rghrSNzduse2f((E$B&;& z!jCsRVczPxOaptEO$FzfXLa@`+HQgcsytnO1AY@@Flxe(M%A})MT=*>%%Jn&c~9Fn zoz?gM^QZJ*|66B1NLl6&kfVOi`r~H(;J@F(RAD=X^W*IVTNb~y++S-}O5=0Cc_x7z z05BB_v*8VTR^#nnAATj05e#aj7E0paFx7r=2H+BYeR$cOemR^ziTe2BiX_62-Fmcb zx4~oe$1e=u94SZg_hgER2zZsy*;}R-@?2q%G*D~@A8rJ?81&p2viF=Bidt8H1;*8q z|K`e8&3lo1@%-a$ZDhyBa>_AI>1CVL*A((-kh>r`+xN%uH|V(GCM-E`!lEx{O1_46 zk9~mQE$QrvQ~0br&%jXQ*Su~>sq6DCwfq?}V_#S=`(&%4{l8~X!vXuzYFU%X_5PBg zG`^-ie@fa#B)DNg+EC;iD!K{Sj4fV#CcfiF%OOODkKS;`Ab0`BpH%8SO{qiBOcZ;x z%JLR&)qTxvf5A}@U=#D{ml}!U_$Xlgn5b*Y_haXO3x9y9!YXI6FonW9D>E1g)0XZ_ z%_iXWkpSjIa<<&0N8e}biS*0#X$rJ`c?z$zAvnTE`bBX#mOM7AC202Q7@s!^yUD++ zmZlaOAKPSpDt;Qc7m@=j=pbEpVvyWBymk6jP~d7q&*JI=C#l2<FOmO}JM{Rr^M|<) zXihhyq7#njX8}Om!mgHN(|)=tzixuG>h+;Vf3<C70Cx06e!`oC=(f+6R;q|>)UXcw z%L1zHND*A+^=hC9nAj881SmcIfNKZsP$t{V_nrr%pTBJpgIiAqHJJ1;3_ON$Pwx#* zGVs-7@#}$1sW7#B(L`0H1HW6`klWf5HLtG^3m;$?Ye2P{*T8hAtuAQ!k68erN~D0d z1hZhItB)hmQ>04X56y3-W+BlzfHsk-y45NY#7VJD^6eRr(LLi%iku3-@b)c41_#KC z@SfepyVq)C4~V@v&?@sY<9Ae37OY@a8&j6oUqcTHTE&%&Chpog&?ZxDz22~rUK{l6 z4qVzEV1NK6z{AS4yO!w{nxHREB@U?MmG21OetmKDCix$y1IiWeY3(#O<{~Xg7w3ai zVn?@@(I4HdOg&MLd=70Urxu~fIk&Gg1X7`tpQs-VRJYP(jGppLPST4G+U5PYq#a!) z2cMs@PUM@uv`p9b@pty%Y7+y--mt(TN&~4H^5<z;vpnP%@^Hjpt@k~TFFRHQ=q01h zOQCET6(hLHgqo2%qa7du4}H^(%2-L?%9pv>)}2BV{D?4BGvG@M2iyAd(J>Op#92R4 z(zEL~M}_pkug(y_m2E+u><@Z^cJ3OcnVcM4KXDt=`gySLx#D!t1wM5)PSsK4AZ1Lh z2`krB0zZp$;^L<n<}+P5>W%WRsf`#6F6d(r8PJ3P(^mjr{&25Y%!_RgiX5dFUlB)W zV8PLdUBAPT<GCPD-^r!CF^R$is9|zw&Y@yf?9{L<0OAx6h|T8@Mj7)PO=y~JuI)E# z$suC$U1h%Vn^e<*v;zF|ku@#ip9fDUTF)h*6q>>=8VvVSRvwZ)8TI>#XU%(;zpgas zI<o36mmaw37u0cwvh|!rBY7}O$v+5q!xDBPRu&PX7qI68i^84Vq&*tzr0kJ5-=~QE zU)K^~q|5pGsKRU8NMbD~RA|NW>14q1bfIcLFCXX*C#ekJxKd*V8WhM3Ox`E~!pFVa z8x-t$!+dU|b(Z1p`qe$dQT1Hs&*d7|z)Wvrhl!fZM|`Xftx6KU+#!#fA{0E?oaC2G z$%3PcZbO)EZj$#F*qeXUpGl(A^(~f;ju3y{@$RXVuOVhKz`C)JZIeJn<WB_+*9l&v zDSCzZDBUS5(#(PVoRr;L`wPr55;1aS*w1i#iiHlvjG}SQ={{0-&K6S|no`NvOtAI9 zi&e{nfq-W$S?(Ub_Ca`M1;tDRb<XG2?b$~96617#vcA9G8Y}`3oL|~K$(?=Sb|gA! zt2BPN2cdisSaS>8lpPpqZ1l&j-|w0;!QwKDLI~^fE|8SUX*)&6UySFn`Zclb#bBZE zr=m=BC{6D5=^|AMZa8^&+50+pf775{`};M4dUAQ#k!*RG==Y#7A)DTNY1@sPWjcU% zvy^!V(3Ul>1{P}LS{0~SIC~q}!l)}D(PLYFLjHdD7l0@g7TEtp>4T{k1xwLC%#O6f z=t&GB9B=n=H?r4X;8wHS0W2zU(dTV>DH``gD*SGrX?n<CW6<3f=L&Jlbr?#>)mokg zp<yy{RPyyr)?ZFk^L!br&(hW!126pyYkJu3Hvnq!`zvE;?PsHc!&eaEM$<%!9z8rg z+W^Y!%VsEL1s`Xy6f<7ja%f7HTz@z@!w(>xv5<e%N-`P#VxIqva+l2cR`o(K30sd) z->MKnKfDzN(85pv5RLe6u9kk+JpUrX>ua#U{T^1K$X&^V*!j3Xfwpc##KTW~=;o!! zY)k9Oq}`n$Ufm{N;9lvrQCw&nFj@a*(fa0IAs~<J9%GpKd<1G)IlZ%i3#))@mcPmA z=}aesdB#(hY-Cm9Z6@<t$J;=HMDl%eGr%4oSV<U8kd)q$irg3dblx1hEgi$tT^LBI zVRcqBB)!9*UOt6h!e8cuvuw}^{!eHI!9%0~T7pp}(c?q3oI=;-s!Qu6`$N`uS4AR? z1j{LY4}D>L)1(pPy@&Vae$NKE-2T}psyoG8*<1s-6~40eR=xIk09S!`?!!2gc|+14 z^Y6Hh#J7v{Sv>v0W^|~)d|@jU=DH8C4e5$gK(y0l41`ep$O@OqQw}NXpoH17i?rib zU*9oEdN~>}%zQjm5JopI=CRi5g??Ewu?&^)J+IK9jB4om`13@e8xlL0P7mQS7sMeP z=EkvB&ULPYb`;Iz?<KPTzGNI^@bHjDj=)HVTF;~jElNtq%{W34&A2$=WN0Co?PU)n zGdnowc(?(Lm#ZRfu#@lfSFK9iL8Epi+K7D{YgK3#(=PN%<f?<%HaFrHL^;LrcgWA5 z^;N*yN_k9$b0yVrFs9W2UUv%g#>BBYld4u=&#L~UVUA}CLAD4u^%&2c0_)V6F&eN! zNH+31i~2OBybS?M{1~f5Y2iU(4_R=lWm$FSTM4*3`IH$Jz1$o4X_N3M+{8Fw#beZG zd_u1w`^`Wkhm{hBp?>1+=#|dq<59Ezm*t1GLjiQ=-mv387cEN14fMKMgZ}Q>oRcDx z`2=5Mk10`j7MT0DtB}^R^~ZNd%<u1(mojtp7;cnK7nDMc#w>B?1fy9=nC|W0{?7vA z!zb7pY<n8!$ILLDBqEVh6Mm3#_0m%!_7p7uUPaYS!9;tLJ$*$aLAk&avnn&7;++i$ zBzX!4sb}fef9~V|>lHnxeE+|bp*de4yZyYMjCx$&{3&Jn+l$AHLe{GBDJCo+A+}## z?URQ<@hAnQ1QveRL0j#S2&tTMJ2@SN_;(hbNfM@7G8F+AqbXtPky3;5EI(lHGy^6k zX3rm9(CYLXx76QdrzHy~Ip<XJKX{Bue#uQ4OY(Giyz$T9v{6v2Bz~*W#qEpp^YMze zxSa}EC;u)`>*dSZuafC!GI#SB1D%c_9k!1xXS3Gfs1e2Gk4cxB1zYjq_W)T4H7AXl z{8Kut+#}_sQNEu;CGX?j>>cUmItsNl>j$pkMJr6(Jm%(wv)lmZQLyNHw8=rB{Dd%* zKa-z_F30>FJxTEzv4a~V;_%?SY<T;(#A89p1m!69=J_-Jg^VO_#=dp23mb{`-O#GL z^yhLGhFAcV2+B3ay+yWw^(D_TZqBTes}hUSgDA`V3AZq7`bFL&uZAoJx0@}ulg<3d z03WoFlP3j}2pj;zk!?)QyRnnVTSP~EpspZO?jl$Ml#&;Hoon4u4E!&V7>DE!L;f?H z6}IV_!n@y+c7cF;7#;yJ3Sg#vOY-R9h(rPO`ox3TFf!U4K!@Kw7?l0Xy8W(9h<Lwn zeo|U)P!bIM4%~|CpmZ_^LM=mpqg+FOWp)0XB@5;!)O^$-kay?bIT!TfReys+k1e|` zSKJ{Z?n#EBH<BlN`0m{%*a%(A?Q^DiQ)jDy?q^N%QNP3iICeJv!12RO&?@Os-ke^6 zhVJ({Aa)C%c=HU-(B5$OsTCT?e9@xoT&aMS+LQ7DGya8p`&Zjsj|$?gh{MpA*^gz% zar*=YZN}x&-#4O@LmfQQiP<NMbO{Qyvt7bsD?=vDgIM{12JB=hqJSXf$X{+xBaNen zBoqdDjcR%t6XnbrlUMK~MTROSDr%PF1_*)V*2-{}$T`I{C6jo;+8~V_rTcn{#6RPK zaRz=aNazOLjdqJHYPYd5GF)OVt3AR~th|pXx-ml;%{oE}+xY?cO?7p3o0H_Xr6tA^ z=KF3Ia2S@Whu$&W?%eU6y~Vz_2QHnXp7jX~A$7J0p0K3=4C#EVwGSM8u$h!f7&r~Z zaojLDIR)6mv@+nWgn6j4=}=Rcsr{BkLz&`wm9x2F@JS{a$ZI^tMrZ7q@mrB}Dm2zA z`)pbIXYGNE&JH<qDoiKp7I$~k`Yi%l`_6z~yDiL@6EvTWTkY&5rI{?4;8YUQDrR9c zAGpYr{w4dozlkN|X|`gz&7n+b2COUQr&mdKd1lJ58r#$yZb=t~6)>e%LsvE#q^m(E zM3y7yS|~WY;E3y9Z?#i4-A2$Y{6r5{*=r6_fq|$)T$EyVc#^4>x<=s1LY?JI4n>f5 zMR5NZB+!>+!)o(!X*QI)Pz>U0OiGsoDpNBakj<w8yf+wi)zc)D6WA@eoTFxC-#7M3 z6Ac)UL{?KLh-ss_FB5hMOj{qyYd3J`R@sQy&1C~Q)UZ3XH71XNQJd<MW&!!rdRY%r zeT8p_nX&T`{gHg{ZfzgDf6#ZX`w?9B7%at-H4giwUp^DswDM*or98CRWs*n((U-I& zMRA-E1_8Ps&uak#%_K_1?qF|VmBQe>fvN{LSu;1)C0jn|Ej#EbhI0n)3H%Ih+kGbf zYOS&36A<XuenT(eqrT&Ph`m1$WH3bq8*a0@$Q8DtIJxv}N3S}?3RBO_Fx~YvwV&^M z9_$%Bip79GQ5~eC+3>7ELvm#E1Wb^Rw4?tORsLCP#PabERtF>hmStGsD}My`&6qlV z`d-9uP~2V|AKOG{;RDMJ1&8NF?jd;AB~!O$CMN3@<SX^fTWQi{IF^8TJJUyiKbG-Q z42q@$PIk3>9*?LN45I@9Kr`_+qjQWdB0K0RNcZD)GOc`Ktn8J9<jpK!XA9nP5)Ok= zy$3A>!G?uW50)43*8koz1JAd`Pl1B5^rwpLgS+ey2xC5ZUiSUit&{my3}diSL#=2x z8yGr@zYi<1=<Eb>sDOM4;9F<;iKF~hAw?+LZy>gJ?F4T7?{oHS(YgIw12ocH2cw`K z2b6hl=q?q+fz#*Htb=n-GfUX_`qz{-z;64Q*2UOGit%hr#NNnmFz140sXdzRw)+4> zL!@mn_L=_#d|1ysQt^?iP57!t1lC5&sAKkJ@=-|N6Z6DiDM9=lpz`DPSXNxeXYX=i z&Irmdz(Vt+dLVW24>O@C&DZfP;6BR=B+~Z#^G&2Fl<#yQdPz77?$+D;{b9z@UFdGV z%dl@QILr<>z5}3sxoJH=kFW+D4)6Yw+F7z@mJNh`p+NlP{#4DA12BrzYF<o^Ubr%} zg#@+QYP@Xn+FqI^%X`%)7%8xjoIMcfvMqj97R(Ta^#<50RlQ!oLUszw!J*vQFrn@O z3`tC|Rw0yUkf_u#MhGv_r{Na5(S7+@9{o{zCP8V_98rFs^LK^SSm}M$Tcw8n06Qz; zW^>H^qCYMEA0eUg{-|9cWg6tx2D9b33xzP&F%7}((f?s)t(t?qI@-+j(ddeI6~w@} zXJyA0f=A$gq94`ut_!#(y5ABsf-u3O<C}Mawrs5KnZdH5z;^${APpTcJiXV#7%;h_ zlIhY6c0rcJ7~MSU1duT42dr0GuIJlzgZOR%{F$oc6zZHKNqTg<m97{YW|P*4bx!z} zk?8M}fBdskRkum;;zl2JQiV2s;ZNNl4c4}~gATSeHnuh`Beg<Qndro->ru#+`JUpD z!MDg`Em&JgP8wt&yNR|)X&+aDC$$&3@OYlkNp0*q3Z5Z!20f&!O2j9E(gp*Mk5k;_ zn_c%eKsu33`gTYfe-(8Iop6=jNTGmyw$dWFW$F=0_WDqgx?D*%@wjm%rzYM&P30AY ze{fZ<Wv5;cmqP_}4%s0m0w2^WDDTt;UzZ{Bx*)nh_28Y&&Soy3q%-YNl;^8pXz7j2 zx@BNBW}`l(!~YmL5wl+FIbmm_Hv8ap-b24H4qC!^{$+cv2bo|y{z$Xc`IK?ZR!|k8 z8Z<u8CVk|O<UXE{PbS)=`{cvLEyvO3B6&FJU8KZeYIN2ot6_Fg;DIzZzfT8D7FMqc ziV$yql7OYkuHFXQLj>}j*gRv@B}sOHcX7K~eCSw5D5S_XYD?Ca3{Kcs8xTz>BMM62 zSV?}ndsWl+u`0U;co>-lHXGUHF|=;k374Al4pxy);p-;5l~A+Mm8X+L0*1)29#K@{ ziR5x+TU~))Kv$F#xF7nav=ec{y3Tov>+?A|j&g}y-pMIt{cem7AT9+qMD(?e)*#xt z3rKenk{X)?z$^*P`kaiI;=Aj%O=3N-Yd&hjhhE0x%1IflUE<`z6x|&44||ATg>dbz z()#8LE9i4;DG)Z|^3pf!%{i)D7cNU3Z2I6d^ZNvH^=x^!f$tu8r7n1~tU9Y~M^U6E zpZAD6>-pSMZNfP5_}xnM{}Q2J>w%~g@A^b*t9G<d6GSn|B?U<<Q~Ls+STeyk)tc!y z<O(LxU%Jm8-OV-ZPo^K~XvY1hWTO^RmTCgDQ4b@ZL75Xd;G<e4y9#=QP)Ah)|7J1q zUH&X-V>xy&IxF$$6kf%5ImCvaTf8gcxB9)>`AzHco)-sC^k5^ExU#YrE|22(F;O$r zbGK*M!O~Vc<}REC&pNrOd*>Q`<LS_8`6N=hv3BCgOweK{pzfJHA0s7-8-*ip1(M^< zi)4;W$t_c>M#|Pnfo}P1kEru&j<cRfUr5wA%?UgxCxevfk=@cc{-n;Q+t*ZaKHz2? ztIjUs_>ev#C}bLl_>a2}HXi!<>=y8)?)-<qLGGxR*5UJ08^8xdzoO}Bk17<gPZQ!` zk4nBdoy9jfdxKwEQjGgOnOv5L!!475qjAT0x=s<g5XDny8-uxG!!=Kc>`NFC@G7di zrw)fwK~nSv0gP1!90}m@vMOJlvc3K=6ph^gY)Co!#m~Dg0>J)vB4(VAeF@zoDOuBq zJf9&0{M(+_??}&{+=p)wfb%Nr=KLN<v!-JVNKdnTe!o}q7m1r#4iba!1E|bMpu>_m zN+LJFz~1j_{C!qIPcfos;g($22LPaMZ0~zEUE$a1%{Q8`6YK29baTyv7FHElXMK^r zC2;#CU3`?{xjcSO*~Q@{<#+lYx1niwJb#>cMst<PcrvpP$zH{kzA1eYsSrt$C8~V4 zYi*X%7roj>Ubp)L6qdG!1IX(jeh?~(R$~y*m9TIE${rk39s=xx4VKK-28ay3`sLZ6 z(bLmkG7h<x4svLB30Si;>(_l_$ebeD#W7U9iIn}^CqRStA5D<gAovS$9J#N+IFX)X zi@lV-i#k6QK_r`FewE}HCsp*h4=Z|*>Xu%X0fVNv@)txK1H|)nn$B#I39Z}`#v5&k zrJfaW>tc{A^wZd_)MmCel@587<Ql>*x{IO{RNqvjFxi)}kKO(q|4h_khA{Pw@jd7M z1x04*E&LlM+w9C=j|rt3C_B!h84S*&Zb6Py{Y^l8ZDV7Ur5YQu7nB%IaQb!M@qyry zu*+!%CkOY%uNTDAW%BP+<^kC#p#R`>3D{<Ma)r&2D(Cm$@|P;Rp9UiDHHR*TkHxG5 zS4<-_=uBf*wph>^Lh+YzGasSt`N>YUC>>x>FZ(V=k_sFa$TgkniGR|lZW^8VW)^rp zBZ$*<628vBN`=Jg%3iSEhFmyjvnaCQCo#OI!-pUSf(hI5Vj#&W_5lxrZ31&c93YDa zj4R0=xjXs7FwNC#F4>A0S14~h1DU-?)kF}B^ii<Da%%$C!}ku+I;xxTC;?!v5UdMB ze|7HIUyU7#1^bPV)X3nSNIGx_bob>p#KoFt2jm{gFhBh{Y35jt$Qp({yR-)e_UdU7 zhQ7;p#Cm5X*XZ*|vKgTH3v>QatbAYc`5!6`^j<@U4Z!O9TDSe_#=z=NKJ!Zrry_@K z0aNO`n+Mb89^fS$?(}KU=*lbRCH`*lB&a_)4)NS*TY(z7J-U&wV{pi4oo;7y?Zl8v zt;)k6fjSeADVsb5^kKkw*>G|=>*0p18kjC?ArH)sy+lN1nZPSI{lS=k5x{h#lApr$ z&+74?uhO{Z!9dS7@kby#q2VHJ;tzOCzVc`ezDxvk!qO*;(ZYZIVrt8bWDNkolBgDW zk0FTx9!*g#U)f4A$-Yz*$5lsmoms$gr<R_RvAX96qc3Tm-A@uK60k?VU_+@`Tjlab zJGc2~&AaSBN!o6TI;K-ZP`uaR73Jrk-jAJcp~wzuz)!lUAh?*{d68gm=7Vjt9Z1t@ zeCtrB@+PQb%z@o{!1V?mhd+MDFXdbPJQtt#l|X>GS_}MvOw6C?u66UYcdh}!)eWqm zs0g9l%5E9?O78xB`xV;vLla~U&e&y|3k~Y|j`y^GT|K)S-dSrOnyx8jFxf1LgT!Df zdBy?nrJi`lR7PHJK&*v}?20MGp|j{8f2w$Z?@Btra1^@BX2w$sV5o{-$egX^xnZDn z#q*XmdLQVVt7vmKF=sO*R?3um2^L|NW5y6tA}U4!xdD#@TChU<<s+LfO6T_0_`E97 zjPXyEDFdh}UA@)3>(3@2yRkp>#<sY6XlRjjKza3(TEUB|XR)VIL9mwdKTXxg$1Hxb zU+<ziUPW0hiQ2VmUk~@m`7}QJ{OCROR!UuP;a_Wy^zoFHB=pO?Wqr;I&6~_}IDnVM z9oMwpkh9w2CO3-sIaz?CtW{pq_ZN7wgK<S>9^C#FoE>!J%ILA!uzOjj-fWulv%vt> z$zVzvb5g#PsWQr=uOhh`wgPzdCTCoD54YSt&>)b2goFihH7{!An2pEFvkpv;LYYxE zm)!T0we6`wP7KQi^Deb4fE@TxfhIGmrxP6--$`SbJa~OMe_d>`@99kuW=bh5qt@z# zqu6cOR3oS8bH^E_7X-2hr<22?x1&H+ebxW(bobyHK%pVgTLvQOb2q~6CE*`XS>%51 z(?4?AG`=h8RDG5|?&(oU>89)KIjN{Conti(OZ~4G0C@X&UWwy@_<Z8_ok*%Q;R&`# zTW?k4UG_R++1I8JXq@LV0XPyrgX!s-$B<cwG@;pa4y&d=88e2`^WXv^7fc0zoX7Q= zbxo?Q8M4^99LnX^-fcGonDO`rZDbuBLLc0RZH0Ujfj=OGtXMtJ4;9(1JQOYCCfN+& z@EJ>l+g!OC6S`8M5c$bgs_9bD<KBzrYlpd95izh(@ByE_IdeNK>+p_24VOFW!(b|5 zRsF<54T_=MJjB~jJkU!WiK)7K{F)66O5Oan?wY1&0u?RA;6T#NTIopjf6>-9vLfnY zRxH_YDeXEe0EW)+^(!Wx4g5z?ff5IqI(Y;_^3|p=864Pms{$n%BIMfL=MvE9t~v+a zh1SC9ADh_S`omYFuny6JmOQA8K#ZUR$v|F#XHxr0QIs>%3HjTo{h_Qmx#r>vW~t^Y zom0kx*Xce1RXh27fC}xt9gpyk&}9~f=pr;dEYJZd+pO}&-S!_vwmoWE>Ni<`SU0Um z!DdJJR7e|Y;Jx1C{G!Q==KkJ!4cX?lR%z`^Q>zt%Nml#*11@Cy9@Mi4hEFzXP#ZI{ z`%a6>W>=^_0n@3k4lUy<NE^J)6yhL~-EI!dJuBnZx;-S`9=WUlIIHv|byQ<9I>=?i zJG)srX6L@8QG>&Y28<;>R?P=Z#MsnIHQts8!S4xwE|l~?U#9;V_XHlGsHzq%q0;x7 zfZAAkjq)#qaXZNCKt^dFx2_2)$SLTNL79mFM*Xckk~e(g+(zZI&}G{U`lCX7s{Sh` zpOpO-a{pe{Hd6nj@5xWcSRpV8iyA8&mgCxXpno`_$^zQo{72CccW|t;Q*XlFNh)jw znG>U6IgZiTrrDleM{*2R=U4nyp8q;-8WjDh6<QZRN@06AW8bDF<{yYP;pMQMo+?am zU0RDc9ljspHb*924P1O`{Qz<&j8mKyD&TJbrosB9V@+M7V*~tHHEvpX2`fDZH6SK( z!QYQWhBz6Sf1+p=pyyW%&E7L5n3q(@%F}wnn#Cw7a8I57-K%_-do;ZHA6xv6p200} z)IBN0b5}q?Fg8*6SFrYKV4NPGa$o1W?2EgpI-omHr?Da$7OyGGpZ-iCNcChsAixvE ze1$7$^}eSlFj7Ge^#yoyLlu^aaw&s{7E4ymge2Fo3<Q=>En&O(-+?Z^cB~~;dj?g% zZdm0Y4P7GD*=6EGJ$Hz!kDp3?gTYP+s@qVawH)iO-s3U;!k`Uc4X4YkeeS!&^Y)h3 z4Hwo-{<v`JZ;t}`K%Id&cohMaSgtt>usr|lwE6t#MX%gW_s%ZsjEI}`g;g)-?&gyg zS8ASY8Go;WS1e|pN;_Q$nWHg1$xrNw=Cyy+Hc~3rSMdc0C0ALTzOG*~vyey`%#x>N zP7$K(B!~f^XL2#+f=$>GppH1?k=bSW+G^2JLrDJA%{JZ}IYb3pdi(^$lAEXC;{<IW z97g+uBoc5UvdyneCdQuk^bx!bLe>1uz5o-o0TeQUTc~rwB1WB-VMu7hcFrnqT|1eI zI4GlO>An#l`w>~Z^_mPlv+8i!epUHe$^HG@r~6M#t^nY>-4`ysz>GS9ss>n^XCviD zSAU!RO0u*S-+LlkO)vZWd}Xo7fK6}Hs)dmkXLwf`2#VZR;0!cB_kR6<Xm~b{a`1Xi z&pAG1wd2K8hXb6uCX{Tt3<vgM66a>`s?Osi?t@5W(WT=WnYI3m0!QTJet4^MW4??| z3Eo@7m#X(Pc|XLLyQIx>Kv0_nMgA<&9&KS&d}C82uivs7oeU4&cF;bqJM<)j06CK8 zZPL0^{2KIW5_AvG?S6BPlG_9<YyZuZS}3G;%}zgyY-V>|9Mc>+kNeSd>uO?!@2_c* zJUq7cjW*4_0MR(BNDOvF?u==pi{G!}D054Z?D9a+gas(;v>5@RW&$14|3jx5xGqKp zOmbi{)(Hj@D7dhF<CM&xNZ;XD;<p>*QFG=gq5<_F>ug*sUlaB~6D^~X3q(^&22Vab z=;<Iz7*}VjISNy;UIGavwaY(})K))bMAj+gkHJD>ACER~kc0d7tT%+o^oKKSnPOqQ zUp#{`+iB`Hgg%g+@RDM8Dy~V@5rbdaRg|p-Baj;hz)M?16I@czD!je@l@-8GaZ?;B zVJfmE4u4$^k4^p`vfeTv>iFCCrbE&|kQy*Rq`QVzQ0Y**q`P~DMpR-@ly0P?V}@>} z5u|%SN_xl{7#N-(=idLh=icXq@1^#9_g-s#){&QsndpnkZ@6Oo1_&S8Fi}Kcz794u zXWjHw(OB0%xJG^6={%Ong|g`~Yg<TRoE=M=7}v50iJv%2HDWRek{p*(JAF-Mo!vj^ z2jP<*y)@bpAGh<oOwWOi+u#*mz-6Axq6FJbIuIa%^4yAZQ^HR+?Vm}&<qNg*J<N{f znQmJ$XRI>?OBD|=DOhq%{%K=sw&Qb|wJxT{-%UNK!}V%jjWD^wMPbWX+0!{ZtD%q^ zQQP)))qHO-QUpMCzIj#{^Pf!o4W{$Ppz~eNVtwG-C?G5j;k!l>bE>Bikv*dS%k>OU zwg*szTfn(o#(Qh5fR@8e%w~-RKh<bE{2lb(N+@UeH;k7ottb1kLAZ85j>V`GN)xY2 z?KxQFG+p67EIK`7Jn-nmwU+6!dRua)sB=*=Amj5-Q<vj7HB&OO2jzax*F`_p1YcdO zS%`Pd%hH73+na-Xajt8D!&iyeu@{N2+P{#o=QC1P#NW)zMZ(90PaSUE6Sa?RUoMQ! zi$5B9oyKZhBg3JUEaJgZ@w8;_oS=^=FRDYHS(Y*?Or*7s<nlLZ&tIFA-2)&z=9%#5 zL;;WL5yBNd`Sy1i^M(*0w)Wlllkak<^>H11coZLM)}3{WM4&n+nl+Q?%B|T5emiUv z;C4OT0TjwV$)+?25dI*r{7P_g!O^vT$WXz0-<tcPZcE@9K-aAqo5`b1zomro0fb(n z$mGMT>C@$f3j7QJseFIAK>tWG{}_%M<<)QeGz>X7^1JKo^aHQ+<;ffBNLg7=wwqQl zNkfqa#=QV^>2*h@9cW{bM9lzk>8lmo`>6^#JBAw8kI4*>OyOF2qkrNGF%Q~Ewe#F= z*uq6yVwKWK!#-5}D<@m3cSA1CvS#6hgoJ<2JY?x7N<KO{BH&5Xyg|9R0-p8B180)I zono<Vv21M+k(v4R#F|(w{|4IgewBgXckw@i|K&z^C2Go&13}JV2lT?hbT@^tT%;RL zkk~90{Aacu$WyZ}gB$h`^BfK^=ZP8aXQHs9@;66oOrS2GOiz3=wV?HH`ZwOB<iv7- zmnxaeCjumCBzCsjUI8B}-rc7O!6|&hPgwM1<4k!7q5VZT#m-)PC0Y77i{P3k&m#bC zV+;5HLn$Um5KR$z?qK|rPhV1&RMqBzH9zUoan>KAEVO?3+6x@i2`^Ek<F5ov2=QZf z<g0!C(?gle6Es-+dc-cedQ+zE!$&4zjG?cVW4QUJhT+NDe^5aLpG7}tvw}Bo1B6=& zDl_-<2WVa<o_}8Wp^<w6Fbc#&xn$Zb{)WEZj~i%?&<~sWi2i@kg#ilV6MzhT(OSX} zxs1oMo`0IMLnvKbO7jL<KJ6HI^zy=<ZF9R0B{*9W<_8=Mwd4qjo_uhQfQ@6yhC>de zr!kX9$s(8;;?u$4>5W&}=e<Px4LvZbmQ}XD-{7^^6`{U!;ar$qnK~a{P{Mm9vkKpI z2q|=dcxMpF?ZdQqR|+sLHCjiqj@#Tn(qU553`yolc>TMT5wiQ~bw8R)k2Tsr6e<z< zgrBAQw5VSWSkLX<o;D25t2*JK`P*2njIz&7wtr8*bO{Fj1cb6F+BmSlQGtZ>NNC+e ztu62L?w*Xq^VePphy$UpK+groAc9%fEl^!s%JQ4BU-0CwZ3m3pa~b)wZj5p!`qZsy zJ%UizQvHkA5!NmuQ1jUcI9si>b(|KWcBMd1_a`XnjCF<4D{T)ELu!J%@97W*@mo2D z?JD7F7qaaST*v6uq3rYg`9A4q?+_k^rhFFZXyD_`2}?U3MZ%5}bO7dF=V|?<Ag@re zD5|<^#1Uw`!(CxHP5VOJEYilEp(E{sXg~1SwFv;w_y)U@#fp^OWO0Mr<YMt^DmVXs z#0KrKJ3l(G2kkUB^}sz%z^}UG$A&d8i>1l$01SI`D3!3>xSsD$km(-RD<nOoe2+5S zv!>F>U}&y3&=A+pkZVgMsszH{qGmj%J7G$oaAos{{fV{ynxC;v#B97X%28!ITxr-Z zAY=+0^%s)n@>uA_Bbmu^S355Nk}>U4;0~{!wBfDTNC?oF({TJUXlj@`V$S7rD>Pp% zy=)CpaH`s!67yd(FXB<*%spV{S;x9OEM~ZdeU>@5OV1mw)(9cS{a6;{S|(&G0?I2P z4bJ7Phtuu!Y=maO%E9m{*@f^xt)m3y;@?5XZ;|`2xe#n37YiJ;^JrfyRcXlnZ(uMv zfk1WaO#&pImyt1l@^672pdk1C*q^n_CXTIh3XFFgqa;B$OMmor4wN9$$3>sJk3VsV z{SzBa#YA$G$GI1Uat9utkTuQBLg2PAT5Y8wdJQbFmCAI~jg*6=C%Y3RcUyptcrzFI zzLO+l*M51Qwd?;#SVfd4k0P13*n>7?gXO@7eHN8c<Bt`|AOMP0D_1cpw%pq?O!H)4 zSlSFQ1)l3@OuD$SQmPwgq{Rkh_i_eu<0|k&HnsF<aBb^=2o<COO5t<wc)2cdcF<VK zUv}|_YSgP)@D>K>pCE|_1^XX)06~z7f0H`BA}FErm#1)w9MheRo{FvMN^_y>GSlY! z!RPDr1_0rTAd=tU<Ou}qmgUhVO!sniz4eLHaz;<1cA|}}@rjk1s!K>gh`#@Efk_T) z2!EJ%!uohifR{m)g~C*C(6lT82`pfM`$(!ihmeB2^?Ss#Bp_E;@F*%0cHC$mi47?x z0W1YSW(AV|52daBJ;4inHu}F7j%1L5pf2Cyg$XO~l!<!|T;Ge-Nq;JJu#qQtJNOxD z+*1xtFksE0mg!U7Q=NCa6Z$apC;j);=rHx$JI+vxPjk@$CPwyng%nw<2z$JT_`P7G zG?EA>lwj9W=J)mRc~alc=b{p?oGNK9?ZU~bHImy04>L*Dz6L1fcn~CvzI8685}2BM z7j9MsHg6XXd&jp|thwIs=VE9Iwt%iydvG-)<|~3+?TCVrm?)96p2r*SJEdi{Et9`! zS!eV7KqV{7-<xMr3h4-N*<VR!9YxFL<kt009|&ZVU~ljK3%>|33$P^Y^v^w}af>Ns z;lqosG(UA*GxG+Jo6wvj#qM5?ls|C0&y1MedWq+;>olnX%toS)pG#h+!_J#}M_@Vc z8_++CHdaXSy7Y`dM?$9Z1slnH?aQ=42qJdfu>)~}pQfws^^uvvo-mrs^!U?6G)zIV z2s=c6^pIkxo4*in^Egm(JT-N6Oed6dekw{91uH3^1Lr9mnQ-eesVDrfO?O}wi_>?J zS+{98`;zjO+c<s4ek5WmAYFv%78U3?#=Pz+8bb58HacN%1GgTYY%}4WrLVkSF!;(o zBzkV;5mFy=m9_o%bktyz5ha$g#0zg+g8Qw=E*o)U&}~*qZS(d}cJyf7+XO^$;|3am z&eFn2crVPHQG1r>O8)YGMc|khHmFBEZPXmO3t$+8y@WNn^FgbIyaWkc9ifPd_n0sT zvsbz#>wtmL3~?q+h-`wMIp`(bppO9yW8073|Cdsu9X9+ttSBq+mcZB@8ME7#GIMY3 z@dAKn-RO@urD|(G6jhwh>~%iA`a;NdeFc{IahQq+utRIBi`mv7Z0gSB(P@Tt1_#PD zzA~q~D(-uGBPE76Iw8<n!?f@VSJ`sfevOZ14c64SI9%=R7VKqS<s;Rdk+))cKB&6+ z8G3kJH|i&Uz1q1CBy-Q<+TXj#%mCX;i8MdyXtg?~3_IEm6)To9Co@#O)FAA&3VJTG zyG?4N(luw2Z__00XLSoV2zIR8dfSrVX$9y#jKmM^GnC=H;xrc&xWlkZ<!9`VN}MZR zH{w}?Qgjfl2?BSL)Yn(qDab)q0Y(%332i@P^ic%&$5#PQ8V+!d(DlT#r0eS4+`tm5 z6I<9s24d3|C!6Vq0&QdeAOPCef8}(_4$S>mh;P8|X+=hHX#!W1$>sDH?*6eIqOset ze=^~u=#2)IqD_EW5CypUc)wGGk4=>u(*p$)4pvOm;9|f}+00R-MihFR$XBK<ha<e3 zjJ7!ou-P#7#4(Xly#4n3W9Sv|QKSS2GoH(?`-#??zl7Qq>XV{>xOmMjV`WUozTTJJ z3t6=hxZX3tJI~l*jSSY$S{ET?pXiJr?Jjj23aycT8#!LNO*l#YBVE`Ia@5ahINDsh z+(E>^^<d=)`0;W`Li4NNH3bY#cQ<JwLpNDLR6xv_v)J~8F7``n4rK8$2*0>Zwtthl z<!M+%5ilVav}UhmPsdxd>o0Hp^VZ<T0HMe5xh&1Uz*Uom+BJ}`1p|{5sN3mLMXA|n zWJuiBa@t#QxwiSkBY|G<^7@h+0=d{KlFVK$AOjB5w!b$4#u%}4?XcJs$7|p>J@YG6 zQ;uF7k}+bWDVA9I-Vw;4K$NwI{>M0MBZva@5)T1UUb4~`x}K+O_0yXP?d9Ncd=Gls zUxO374@OJes6%4|$RfW)&nQtJ=`1>kCXW&(*~MVEPsgbhqcOCkAIR8>2zP-9N((FR zc9yCN)#V0lE^pp<@~I>IPA#Q*c1R*jhn0FR-NH#<M2#G#)4qCbirXjPGC5b@Lzgg( zyPJ!j{}?1)PkA&9&3#}N{J~cbCgxfvo2XrA6HE<#)h>A6PGX+^aB_A-l@)H%HG-%v z!Y4}0$^~qM0}9rAV|eNzmqVh{FIm(W=kj=ug`80c8vCbvhY!385nk>e2o2^8M)Rl~ ztwu#@(n4kQrJeCUpR52n@rZnl2|w>xCe1|N<Q}0>^Tx5(b)rvGUE#%((6Sf0-|Oc5 z6N1zU5(Gzpq?*w1S6ZGp)yoAj{&YxJ9bd{+NF!_%GC=Z07s?EX7sBI=`%J`U75G5l z?YnUKXTFrN0&#e^7>6X$K>qE))6fG$>yX#d{!qQP$mw&NCy=XeUiY&rtwBepZ_`KC z8J2@==rO7hokW%clAUmlBcB)PUM6i%v091?044Q!X8T)CvlDC4g&JOw6R797Klt0# z&lM|I&dE-sg8yUt=`8s<zFsD`2&WOAFsME@z<l%xKB6^EEZQ+hON|rSZynRpa0~kN zfE<qYm~MNu&t)2Xk6>^9jM_wc+{|D*$}8hlkVC!iTNZq4mjiF$uxhc8qjEP9HT1Az z-Wx|6BHO-0+%HXQu5SuF1(OdJ*aKXGULJ+$2P}}91iPklny%hG9;nUoq~@-u%_+bE zWL`Pbu6vW{wBuCQ6|QGY?;8KxlJFlC_+P!{|CRF%me75|de-xSzoOVwjERDrjo+j} z;G^;jQjJ6R4L}{V0JLEqjnvm_rq4nba+>6FO5o4&J_mFUk%6pIG2g?Y`+M*t-tOgG zuV)K733@lY%d4h=uHZY5t6VeV7iFPeY8ZAN#EFecJQflJ8XajBZ!ShFk#{xvoR}@@ z9Ecvauu?l%y{9EJR2*XdjgAv?5f(C+$Qr<zOHS@AetTm~lz?9+ryO{$u6B>E>ZyFY zKw&z!U<vs{^+(pRw!nxS=7iDsF97sEHNBnTBAccHfDG>t%_=#XU)uVD9E9!AT9Bm& zn-<$Q0~u>M;49)#VWeh#7#R)YeYWm$^}aJdV*kyP9`yGbo57|v&9>eq<<=v1%7ZNE z;m4s^ZHdDgPTWnT*ic^JT0`0A+qwfvu&~olI!)Xr3*j=~)rtnf4%M8Q1V~60#l}>H zDbsJc>r0|4HS99Zcex|13J7+TzJS#I)s%qambU%KUH~IE>h}+*f_m&xEY2<nW$F)f zYF9yap7nSXztC|7KoYSGf)BaUcF%oQ(%rFd0w*?aEP=G(5xY!gNzca_-ea#MKf_F| zXkEUGfWW@uYK+Z?estf}VN9-3`^o0J6F9&&8UY|Tq!V2MXxYl={>t@G>_0b#0OO!J zq!eWqXo~4=<xA3zx212XX!DQ&SV-9Fc5f8dw&ZMw&kvnorJrbIGH*CPk|F!KvRN$$ z{U>p9&rxmll_{o*C}kfSoqYEB$cc+V&Op&`T$WZBvtnrN%o|K=!_EK#NXvjEuuZh& zzVj~P4M4Ss#k7*9TF*4>K#mZNT$nTz2|E+Yb;fQ_maYImaP7hFLDDsFWP5UVTha)S zp~C0a5RhS>`z4aQTz;Z(F>BYl?jrmBFG;)4d6T9zWP560g+6$qWNmk#j0mtQj&m2r z|L5e>kyI$pj&Dt0x0&xEYlG_kDNCwrW6Jk$46xIrq!qG&X;|8Nq}rK9<?eQVm=6sZ z#I7c|)KCjz>x>bP;>r!)N3x=7fyI`kfx0?sT9X!EZepKMFbz))#IlYb=q%0Rh)^(P z+zmrqAQp*YW|m)=i5+nA8o?=ECZZmp-%N*v(7#+6fFaU5+_DMRFHzv=U*{?ArxE0V zht-)dCTe<sZhhvo8m}_E5$v?B0#HyhERI8X9nJsEY|=`Qo>nm%FSZ=mto759rA0Up zg<G^<B01Q)M1rAH;;-`^{sEuu4><hiH<TD#B5ckt%Eiu}R`O%=hf_HkhSx|P=5xb@ z3j=OryH3QPoQsD@f*h~x({$D7i5eguu{j4VomfpKX};@jseqqy1PSXIpHA(a_>oR$ ztU;OtuXqDP*l#d7z(1qH{)vNyqEWH=ma@24&$N!+%#uiMd3ETRJ(mv<l~3roI~-kj zK14L;VGQBFlW@#x4l`XfR`mFqJf(d1x$}5>cLGA}B?fjDKNl$P4maV1Q&9E6)-O0+ zl7i&!3Y4Da@)$QKJdZKs-ka{BJ})0J8><6V#nIK6Dj%$SQIoo;9}5TbDR?+Y^Xs;m zr}pF=1X_>%{)RludJ2s^AP4$8;)Hci-brSspP}ab6QK@GKfA?{{TD8u{orsF$7jeJ zJ>pq+-Fv6y$_%f9_6D{P@?XuD^2SQ)^x^ia{S(n5=ybjZ3018V#<j-MrSZwW_E!({ zk6yogTTxokN^+f5KkK8ATw*exv1g@fgtcjvJ@eYTDGCUkZ!w1>v(M|;MbrU>gD&R( zp9KtfS_Pc^9(&$ZUdt_V7%vn6+Wm@wZSTjI+c{W3x~Mr41MX=OKSpmN9Q3B%+HWNp zGPi*~cTJmVb9P<o&`X05iS1D#rch)Z-ZSz<w@3q?=m<E*tId}VPZ5Vp7hmbyAS_=z zkou{b3WA_It)2Tevn$WS7S9f7H{Jo0mgv^z`u8~SUNOz1X!e(hBjbS8`f-y<iB7@g zr$20K31_JmtBUe{><JC!a3{G)2K+{8wmN;2`lWM8b2WJ5Gur$IZWS{6o_kAU9ah`q z!3IP$%A+$sJpO5qp5r)c@+>Y+SP5WcZ0i({EKQlqZV`Wh<i)pL<FM{$vvydK!BmpY z#hW8yVtfQ-aLOe#+{J>SFvNMR7KK^VFY)Pr*~t-6mG$ueLatOPMTgV;xHfpK4!BOX zrz^AHObiMyMX5@!bxov@g~{CgK7f5qU{7{0%O{DoGf8Q9d?T1DQ&UdEg>erl$oO>s zO_5sa-`%@ec==N1Uc;h2eZBVAy9GNr(<P)o*3b4Ar5m3Po?*h%F#9b4FIr*`Bo_?c z^<Ng8QpE#?Zemzmi9AuV;K0ESGU@dtb+1MxQ$^TprEI3I9dO#f3wwPj(+RX_6JK4> zgwhROuFL~`?+zd`eK$cyea$M%?stq;;?w1$Mxa}K8R+zdZ^Qi$-`#{0y~g^UB8z3( zitJH-fbF}pdeh_<72=u(ENyv@5;#H3s(|qc_9e3S8&>1{dzssO;-H@-04B;t(crBr zw^8Jc7bmB$!~gK@c@I73ffXW#rkKj;`#baOLLqxx@7dqgfe~zo2{P-IcTHaKv$tg_ zCSB29V?R!9ghqGkE3q%k+jnaQJs&nX&8;lX2OX4~+Y2~N@ytNci?wNii!9KUewFuY zB1qq`G=RY|{ptsgy5tio*|Wyo4utDj$UpWJl>4>Xmu1z>Pjent=l|pV;H*P@i<r&H zUDLkXWI4OM1>m95dv7^XkbWGiQ`%TLi)9i~CQf+Luqx$V4rd33YCO1QIDfiPo+CAT zw~(+)cCRW-*r1K(=Lgb{0Y8PtdVxgV@`Irfg?Hu5d!tgY{Y}SaZi19jSb(0OTFm<F z!*Yjp5C5+!yk24Md<k9OvYl6|XgW1y14Qv&UHotYr*v@Zw}0MifslVXXZ|p4p<*ne zq5E7V@u0aUo=9IY$6TH8LBN9O*%~pSi6#$nlYnTy9SkE`RlT0>$sV*dd|#M5lJRQY z9jo+E#VH}95;<m~@c9^JH?Tv2L;0rnlsjEowi-#BteVGIZy&Kx$!}<f6mW1bskyvv z-c0aH7-?aLa;|kzw3(S3N@FvaU|lcU>=|jm;hw|YG%q_=sql`JF$Nd+uAGl4#xr7l zouFOp!fm+0^~_Ynf4l%63hT0t4|fRum}r%X#oP1zG5Oh><RX)Nz6Qu9XS%6#MtKUE zVwi;O(T4%Ru~&rt<z6uqlDT%}KsakH*Y9g7X#g6wODmV9_eTxN`J1OMd++NmC`-cV zMImZusW}>q^q!Y9&KrB;FX)m^FJkGZ8BoMa=itIPK}pz4jKMztn$1%P<;)q;9K^;k zg5>9bDj34go8W|3sXLw`%Pj(Aa``v?(vipZTg4P;b)*a#{MC*Vk7je^km#mYCWg5J z9|wT{T|nVGo1^62VFyfqe&en{;q=k6i7C&6=z`5WZem2z=6!9N<2>FYgIB6C5SL>) zTqXp4GR(OtX^s`$^Irm7WoPng^#AQD``<$N%+S4#f0AUOtw#gg4+a5bGWiYcx%KP< z`RdO*=ew*!%(4*ww`<&BkvP)SL-~`W%Qd_{O>i}PAVX(2duSntbuRnbSaZ~i%1P$| zdPSTIm-qGr{*phIB7Y|*MCK+x9qbl}(h6MdiSx=+e^b`*Xejt}fNECpk!-WP6%zN9 zC^GhpX}g<(=su4@&g%%$vk{5a*tIeI*fNB5|DF8zh5xjAC>cjt|E?Ck9FKaRa2$HQ zo)UjM!HQ^&Hz4lVB}hA>2mFk4ANEi1#m)&_x^NzHZ$_unYMY%Y6mp!vdB=VV%|1hE zDUN(uRairT53-sSL%II_`7zYOjh!_Xasj;QIZ>937zX^GLdfATCn<zj8c<0VTc!%G zi;O4?A$(MEm99^uAE__x-PfjPcamYgd^W|T;Alv?e=X3(7xK7;Lv%XtL?<}bG4{59 zbu78AStvQQX-z~mrseW<+MF}!4{)#OXL!YczF@Fy^x2sT9nIZhPjJN|L&CX@C$tkA z*=7>@(&a8uN(7yASrw+uiqV6W9?JJb&ru)jmcr@Q&oq{;h?Xm*iv%w*ru%IG^m>NW zE4e)S8iHYNIXYoTk#mc8eAuRHnY|3U4U_mwb_1{Ue7JYf5h^qcJj}lgj;%#}juX@= z-+OSG!rFR2q&Lh$Kn4Jvt2!r*Al4BP##vbu!%kmc@o9CCKAMSVk-kqmq0(U(G+HTe zA!%k^1t7j{p5#i7l7Lj*CrNhie2CR<aXGM9_#oArMGyiKqV+}Q)_J8RdLcAHM~Ve| z>?oj#?!C>GY@dTOD>=|m!)iZWVX8ymCQPtYTt4zBzt;)c3OV;&wF&gv7z#7>oqeK9 z=XixGR#|DxtzXxJ;bMY$vTu?!V!?)_7x|h*Ru5+9Ca1;Jj}3Q97PQR2ompJB%I>rK zyO<CqKriJAKmMPmYB50UUQ?$6IgzHhK)Gmqu=1LSBZ=bBE$gq$$>y`mR^WRs7mN2< z75BxO>~tWqq@PStNp<FglF$yhXDU1Vx@%nO()0MfAYSXb-?X9lgUP2}?w=o?I$T+X zM^~$vUp<CwZcA?&BVHEBuF5NO)xP+&2IU-_szU*|GK~6M^O2*hQ#j0&E%)RVvf7Su z(;#ak%%4f1$EE5tpUaaE-amC;y_*)fZq-YaJ}Yk$&`$(%VlGR*t*Rc}=Yn=nVZ<Os z{N9HibNtunyiEfb&g)DGRUkdz#O&_47rh`qC8lqLRveBzGD|$G(@5tV8O@`9ePb7z z)tUef#w`j`htN+hnP0AcPjBF7Uf7y0eZtaQNY_$0f58*lGhJJ-3Ri(n>pdG;L(VZj zRgiGen7YtW{J?B4c3#aKb0kDI-|#hvM)KfQA5x4y?V;K+F4<)?$DH?8#er1Urb57j zSQ0(25OMe<(0$?QERen>tvkyxtd{c#q?@8o0IY>XsG5E|XRQ5jh<dV)MK%y8YCJ=o z4NCUcp(zf7T&O4)qi2`y!hK@r?3=7#jQ@rwC!RLUv4t6I9m^ia!F=Zc?T4>mPhtI{ zVm<#zG<!YU<KMi<R0+>s7n$*58Wp(JcO_ITOKnuPeN>+CvJvEfs(1J;g5P&j_%`nH zeFX>&4deCs%hp<UzwbxZALb(IF>RIZ*#f7{{cdX6gXsE%7(d98ruu1NRMt5w?IVrB zMqU8g%1S$~@_(}ypA(BcA`ZU3^6U6l!65UA75p+*ThqAC{xkDKzIZ#}4%%<|4qT>5 z!@AT<m~@2a<}sr?wk<%=XK&w@!zVgL<nsQ5cGpAp_M^Yg8`eCf)xvS{p=I|inP>`c z)>VNJ+$A%e5!{|giR6^fKwKL7(~il0mtyb;r^2mdX|Hr0ay%@4(?#~96}C)_Q!Mn} zq9UC+qy>j_&xP4vw!JuBwZwG8-usV{sB42Dw6Z<fN^^Ff-kQ9WQwh>h1R{vQwCpDN zne;79eie5m&WaO%F{hSKd6i3l4Qn^(;Z*E#_<HHI{s$+o!2A4K{^?caKBpL7z(p<Y z&r_vb<8;n-!pDa#Vknv&6_gL)l_d6ZYyp?3Ahl6jKgJ%Q5+9CEm3-Il#{aqMcg+I4 z;;dsmKOMYcC_p$){*a6Uc62xQRl@w`5=*py{pMF)jaPioylA!>!9~rm5!vBIQe7$5 zH*Xc)O}rqRKhcB0kL!U;*8z=GuKJyIzn;YIm4eTWXZdmVrDqmb*Cs7@lAu?TU$3^9 zifgBThiV6SjJzpSd%(~8)Nu!~^R^>^(?%t-kl;7?SWPtTPB7?JJ3HiqR^66@J0pHC zXZu3cCPLxl()|v9<?ty@=ef)8m&>aMA?GZvu4a#lq$V7Ir)&S_oyZsJ&yoyju$Ksi z`+O#3J5%MVd5!O~UVp2}G~AfX>l_#m9ad`+`CmaOn+b}+u7KF?h_4399O9FIP$%*3 zUX@E0j5lV6s1dnq=KH)xMDxzI>Uo4X{ai)jNj?yp4B#C6wk$p4H6oMTQNtmSR-_`q z*qgfN2fo8bb-FcrsTaKWDWt2^iuHsfL|r2kstrKY9^bmU_z~SPSB{9V!}>&pTyxVm z_1e#2%(z{EQuXwe>R#64G+~q<`3xy=ZLJg>5LO=D2cG^$tY&9#+Ac}PGV3t@?}PCP z3p*+31-^*NM7@U{<`HB>K$orR!ko~52B?msomwX&m)X49{-i@)l6|#|WwY)*VhfTC z0ggofIdSpo-QdG{Yb=Vto-^~nS)Pf{cGCE4>WU`;N$FierSzq^>R|A_^bL8z>0x3w z4;FaD1!I}T*phcbs<-#-y9fb5Ji0HW^A>0znL5AT%_MJG>OQ^Jq;>c*cjtS!4&96V zXYikG^tZM3oMH-|Qhl037Z|C+e^k>ckU5sFyOk82B^l*SjZVyEKYy8rh%`D1$vp6> z@EY;@J(PMLT<Lt=VrJ%faGaScoGs~Po3;5nh`O`;5;@oDNX}zm5mAsy!WUqvnpFa` zV}2|xa~zt*F7P6p5EHal!16a4dKl_`$E*WUwQpd!)Hal%eAa{KzlAQrD=CxPWNVAe zs}rP%d&F-}%BVE#Px6YtZ@?wLQSqX8!-cYCTxj~0`NW)NYQ&hGm6-9~_(5yapW}t# zRLMDtOONz-)k$)t=cS1I_8yw?nMn>(PjD|yt&YvZUqJZoh<uNW0#q2)=_j=OtJ!qA zU?|lu?SqUzQ+xM9ijE+zhgT2VV6=zJa4|3Q9QH6)N5ak$)2T)LKq`X<CDZB#XN6FM z$_=FF)n*V0P0rKJ2O|SpVfgSEyebV=Sb0Fn9<;8N$AK}8e#)o|s+@k;1W3*s>5}KE z;}@48bqL31)2wt<QeE^2)?0!hV%{E~MCH~GL{s#d$)2ZI3Z-*Tv-I-%Dug;SVZYu2 z87J$EN*E<8^}*BU`8J~q3w+2w*~!%4EzKBhr_Mt%Fc&{ewQ_&mW>u}jl+UtWT(VE| z?$qj+nKp)uI*$i)io#`^9g^?-Jj1`2y>3SqOA07#>f%5-=OH2Qnb$RZLzm~hI5{Xk zj_C@86*&7tFGYD98SM@|+iMTPdik>^Wyd@aaaNT8oG((=>Sq_>EoEKC8uJnUSCez? z-{l7lRDRp>I;J1!?=uH!#`qG;|2~Y=DSWDR@9C|3#GeRWgyqR==R6fi<N}JV#5-`< z17znm;0)*nn5%{?!3SmZws*VAdM`Vq5m689Zhy)O!JSK&Zl(TMVhfFxabQ&Jk@8Y_ zOU10-Z^o^nJQeLLX-;NgQOKrg`*g@4>OmiBRLJ%e3J4k85+LCyr^>|mnS1Z|tBB>% z_;?`L$Owj+Ag+~GXuA0KjG$Z-E~~=5L!r0c8w!}q-_U41G6*-^1CGeDZz}(Uv@-<q zU%Z$S+@$p#I*N;`sx<pIJs>j7Zp|!r0+f>vOX}mT5wI0CEtH)>?CgY<TW{HeO9Rw5 zPdL2=JbVFShaoN+?{bwAw-U86_D264Z;fMRRZ3pxSwrlsU%3n=8XP%HE%lF$0cb~y zj!6_bFj*yS?%F41p@9Z^mxNX@=sl^8$6b`EF2mX#hQnXE@YeJ_eA6=tgud|z`pM{I zPezplE*FkhPW4ZD%5x39fmlzMI8ahgO}m#}*~-W#bxvE$u<fzvGu_Ea!K;7z*=$08 zLDvdlON3$#+O4*bdE^>EW`;vTCWT?9-uFz<MQ`y?6S|sFfJ>;NAWN!E@tcR%Ntod3 z@<iqwUvUEeP+APC9#RmVW?+V;{Jrz5tVj30c{lOtjd8xJ>sGm8I_Z@=Cxh?L${&Y@ zYvYGiJGO+z&b_;>FxiWzLz0^rEJB@yw!+^=dTX+c(-sd$^esntjrkQ7v+MaDcQ)Y; z2SQecWW+-KkhZ@u0d7F}YnD0zr7pgx*I9G~U$Qms>N|$ax>=Qenda20Z<oWcsrh39 z4T9mPIoSHCs5eO|(~&tg&14U=u|?@Cd9}f%w)Gn$cfCtzHO}tHZ^N^<zl`(_jJKFp z%@?6MKCG7VjB{14CmB;GncJ|Wczs^C*!z|grB1>_yx8PZ`eWHe&SR&*eMqpkF8(xd zr*kw>88KA3=X=Ek_8!@D)pf@)hn%gWf-BdJ{(^gY%`gAb+nVyXVcz+a(8DbG<xT}M zLfq%ecaBa0eW0Fjt%XvS-q?!gkpv!W#Gn10ERs{rfiT(2dHJd@Dq1F)C6eT8?{$0p z1vFZf17k^7ultoW=0AThTK7_H=!z-W28Y5+6kin!7C`=KHNMewy)R!b8_=*s*x4Ug z6-y+w&K`U&0}8HJ$$m7__7}|1vSG_r<I}gy6jAU0>nKOk=5q|o#SOUCOnDG&<MZ?@ zoS%e9EFjc8+yw9&#Z;5<59~(eDp0a?O%E7aNXwLl37LKPOekbvJ`}BJxKIUgUO+t{ z2N@fx7TgC0t)U>iZB3DX;XcK^;}e#h&mDTtAVb-11hUGR{=1)ub2wJ4h-e$KvM_@A z9lgwRF~dw14i&*k2ZeRKwS4ak+gy#c+B53!UvA@jmIwdb&?F00S{^p4EpIRX<HuFq z+8cWdR8%Gj-cM-n8;sYjd6oxaX+FM9rxqnJkz=X!jJ^m+vYO_~d?N|}lR^H=gX-cK z@6*`yW9AsS=&4$UPzGaF$+`hy%)qI{TiAlvU}Nk$+()MwuYei|e`_ZM-E~s?9)Cyq zyT@NI+MU9|*JS&@kgxttetL0)o-+tsv7#`0g8aDm1P+ymPd+u|sncI}qRgg;;3B73 z&6Ws3#!z&bN)57QkG5qub$$W;G9)W^c8>qJRnMMx%z0C*CBsb+lLGs;d7KOnHo_uC zFE<a?yLyFef1Qz}We1?y^xk4Gqk~>K{dRKFq?Fx|?cfhs_A;`ip@{=cRbP1;*Hhlz zQ)<A0v4X)Wp_^uyKpF3L)0OWLnjc&yjhB*>-#pgOfdo^SW<1$R-g)yHTMacT;|t!P z3v)exhUcl)c@zwc+W%|vTHM#htHs*`qQ3$TI=zI=fRw?<?Agb@9k#Jh@n|C8d_7qo z^zZTVN7_|DA79Ke6|IjmEHSw!-}8Hl=;8dSV;(D!9G9>F=z~e|gPk1bPRv1MAZ_6c zVAo;vi|z%r(cRC$y1AS_q>ZU8fniq+TtAa4p2;tmJmUY({q1}qiGmMHn&I7J%G+d_ z@o%y;`N!tXEz;PzI_hHSy#0#ZS;(F=lr>PSaT(78D*GUtg#Ea&Q*_1MGhEK6YMda0 zW$EqB)RkVaPV)a=tFTJ&VC0Ugkf~Hy+ebqLZ;FdAk;XRQD8*wBjjS|bARnB<|Ke$( z7yATguBW+pXuBi>=yVpjQTM}vGj^wCG|StouxBT8konxIo_)@aPZYxsBFW)pxz`ye z@>mHAftJfkLGuCTSwIK3$WHw21QH)gk=}0ybsqSdu`FdR-17F!zu;R0UGOwgo@CV1 z^DH%U@?gG=ECc6<Xdw8r*fJEIEVnWtY_b1s4)^i{&g)Mlb)?+&`3Ub-&tN$1ztzf& z!z4N{)ScdUN0{kl@#eDf?f5Eof`6BEguLbW&zE%wA03|G*ant3GcrYBqbk_}g;qV= zFOm&zowY7!y`IiGXd?nz726!CgT=1hKf>o$x+k0q$D*xJHN=#8NwBL;s05A_`F8Jl z0?TI3#C6HX>`wI}5FQOOz)Dpzb%hDkaIsxldLU6pA6%H~)+WXvi(Wq^pV&IBmy4`L zo6vBy9KN9JQoS87@%O{oZUeh?aryctmZ(S8T_yu?F7P6^P)9(To$MxA|F7E&Tk=H@ z!!QW9nZ_}=O0Kds9QI6k<Tu?GSpvudE5MhoDatp<Q>*54*j1NB@#ca5le%Xv=e=?` zZPTfo)?g@h5S2;1X%8Fum+ZeX#XSQ40zMh6sJi3a@SCw}=3PuDAO+LhVDd}VW~V6H z^KYu;O6$p|dPDe8OEFbwv5Q}(Cn^wf&}MjZi%Z!fI$76ut(#C9n}B-nHKR3a^CaBE z+KHifY^U`m|NnTAFmdIuRIQ<?Sga=>mjnrjCN1)bP)d@j@{;k2=2Oo-q2Uc;lbFrE z$6&O-Ra`NOvA<aVr+kXq_46M!8nk}ED9pQ)K`NUsrDTY0VPhOIi8&o`Hcq-S5hv%c zNTr21vp~=UJB(mjLNUeE+cJi24b&}ADrU0bY=)P&fke-A9|wUr*yaf;Z|i<qGh>HZ z@M`}|^oZjzbI3up`59<_VuNzFWU@ay8KpEtJ&g7z=}!T9V$WN00~p5TWAwQ^bP=#U z%K!?#Prrsz%&Lzj++ePH^yZuaNs}qO_r33pFQj3Q@+etVMROkC6Xvq@Kf}B4e}5-h zOtaxY3-Q8*=drzN+Oi*4`a!KtP><0d32))|GM_0kcCn3`4CnFYjlPB#%PmB;w;5$O zKON^8()CkE!shYTfad0%^-#KYtta=AmE<#l6if(0C`c!x$iEY6Qt?p|Rn1(QAm@cr zufVUnUf#)vb9gleOqn>un9TEFQj#gcyoKUb(@R(%Sbc9Vl8+P`=Zj{pHrwKyx{Nj5 z_zTyrSf9fr@+I*bbQFo|*UA2}(5q0j-&;8gT3&71`TT7UT$f~&%zD^3424xz&Jn9l znLPvYuR(&`^CG0y@K$0RAA)8bkJF|ev7LuhRkyj!-CVO%<lVQI9o!7Livm#6I}aZ^ z{f?e;+d_-PGeUUKsa${$xTlu!7~1z+4hf$d9E&0BI}y)jkY_rMkP5p0!}vG$Yip%R z<-4;Z^g2$MIIz&5)uWyNH`6`5)RLI!=zp|U136#q{>O*+|9boi*AkEsntT6w;rL#6 zs7%y;G-rGC2;GL3`3H0l=;#CFVSSs9CIBe8QqgSGsk{(k)w52gZz5Q-G`nI6q(2qi z{bvZvjQ)z^vs*0^{mkPfbNb6oAN%b}wb%~uG(-ZFkbFV?2SBmTjh^mustuBi%*dA+ z8Q`u3A0K|`tWl3${Mx(nRX2oi)9gr#bhxe?t1|&qBX$6O+S$BQSvf;O>P5e+%fn4t zp{cA}K~5K*TC$zlQr_{?p9S`b<@xYDt?(U4ia!u$ka>>{5=eWg@#T2LI~K$i%HzXa z+JOhkF=e<9u+%1me%C90Myh;;S%q0{6(oyAb@H?)oBA4_o=3L}?EC|OaL@Y0PA=x& znMb$!BB>CwwoG)6lwm<$t)hj`IQg9iQWw9L10Dr_vCge-Wf>wT{^|N3Sv~G!!kJ#F zG^ds`VbNWG)Aw|*Ud{?@O7DqC0O+<!*S~hJh-kOI?QrXv4bEMaJ5+W|rI$Gk#HJQv z&#p|0KDrncTs(Ule1o&{=-o1`P`PX;T^V+b;k3E;YlM^D#9bj~(azk9lhVv5bpV!R z9vlEScM1s+pp?Sw{*aNyaq}>s5hj>>P8%KU(O>+`oyg&qmu(i6t!IuQG=mGMlgAAQ zUlLb3jg5tMS0h3$#<ML>C#7xcs7-ZFcg(LDeF3zwe{URT3yM!h9F5^z&M^%h)tT2( z<s}ZmbOYi-rvn?>>ROzRuf_lILeMXe#r9VkqwPg2n2c2D!PwG-bB*LGlEh3isyzcC zet8uZq+WhFhJP9Sny|}Zct!TR0|4%W0c7|3rSSMT+~Gx##W=pI&j(Js192kULDbYp zEzsmfLGX)<I^vK4e;wfNwTBmLdwmL1^$nr{N_UYuEWsc&(ABBezG>t6j^>TVDJ^8T z-(pBq0}!)yTR@U1f$4`uEgSsA@*0P0N6!EK!myPi_QYRnBGo+)Tb{vplzf(}v_3`V z8Khf_Hy~w$^s0Lq^$(LpT}!$Ys4*X<!i85uavUI&1(L5M1X%^W?P*Hjy`+^H>G){V z!j?7!`GA+k8I<B1niaa5!r$LLJ&{FkFr5=;5F&~kv*^^rXcB-0rON^oSDCqCBMukq z@~aC64cWgoT&YjqO~TrXVgs_;260{(<3Yuu^BUrzggEx==?=|0b|P1!YNH64m6s+V z**`y%n1m!DSFI^q>W9%CL=^tVJ?f2b1E9&%{0dSdmn9TI4qh<A7^3ewZ?!K{ofEGR zLP<fP1}@Y+z+<(gHk7lX(PyRiV{#T{D+!N@j8tuB?~-UVtaW#lH>8>B{a%$HDSEnt znV0160S7F&)>Q6OzX9lK_zL(26GawzuB^@wQ-ZCy_N0~Qx9o9TXl-z!8|BYUkL2#q zEC%I3Z4&cF*4{J41=l%%i8V;zWhFQ=*K1rIdeB?!O07SLGpIJ0@9m3Gc^5?Q1Y+Bn zntI+AA2mh6&7lh*+FwC52xVIlPJ%2UoakT758+d+`xIk-@WU#$83X1b-^X|dhJ*!X z3v!}tZz5pT)X#c-G-d}KP;JgJQ<{=VX_;W4uSP@mej}>-zIfI3k#MfhR@jN*k=Ies z*cYtgKQ9=Z|635P3r)eVR1^QK5Nd>TFLK=O#lG+YHE)H^wWQuqcDJbKmdsoGYdXXu zOT*xguLsaXdL;|ey?@)!+L%jI+K;6CXTN$HWvE`of4um;azMaV`-cn9^SAd?oBuu* z{I3_REgdn3Mg|$s7Eww4!igU&&<L3FI{}~uIbhIt|8|>B`1T6^0I+Em4>Uwvw*qH5 z6i{%0Z=3l*Sb1bAid0f?Rd7n*tck!C??KFQSsBKQ(g6Ov^7A5G>6=gO^lPjTp7Bsp z-IS4&=!O~4dSkwh-Q%Z<AO0gt#8?!o@?KddBO*kQvZ8T6iMky(I}-d~{&X!B@Z;-w zBo48j3ZFK)txN@4IK!IEBrE+~+u`ytyzu-0=2Q}%>#__YPvHJUhu4VL7f1Y^zdG;t zZ`)W?_ECoG(ASlo3d_k7eYNho#Nc=uR~EcfHde=9go1|mJ=nOOFukLO#EP-Kyc~85 z9=~TF&Ei`u^vSW3M((nTfKHIxX*Vi68+S3?-t{nsXAD<Nx^NTZ!Q*ca*n>~MQ%^at zBKZUKcaYNBQ-&o6fqWlQ;{NdQ8u}f2l3!Yxj91=ltyQCiSp7-Q>_$M09G2m{;360R zb|$**XyP&GX=`#swmNW3h0Itds!LgKYaG6D1oiq3mLuL@f1(_hoAIDAa7niM0u*4~ z<;N<Tf9B8X1mMyX_U+3(Sc}iUlGFy8b`LEbVBc6$Ej%YYf&Z2ml+{v)=o6<}3ZD4V z6%B)Q6V<?knNwLcNse6qbUEybDHt*Q-toa@@ZXJexA&Ehb?$yX&A^{IS{3uqkjC4= zUK1n32wDc*`AA5GVuOirS?FI2G>oH@-*3@iICAJZMYeT$^;!)As*v-b_xG%0k@ncV z!N+yXaG8UhBpu`UDK1$B-#rg1v^1m{wqey&X04J0Uz_x*{Uy6L1M2JE_^tucA=fOw zHU92&50l%1Yi(@U9M}{n4OvNZs@S4MSD8@z51u}p2k-bMq6CrR{&Q2$(Ooz$8;Veq z`Bqip|9$8J_Vxn0KKul$(JYm!dko4`Dx_&vcue0C&jNEeOKI-wZOOcZojQ`vTCJa3 zz8jMIU`VvJE=ln$H}z}TNDU8yX#m3wAE&M=!X$vnG6`~y2{HEcxS=qLkxuJ?(5#3- zcJnXM|7?8QXHuL<_pEFhDwyqagT3URiP89DXDWzD6=ki5?;gb&$k~_qzpwmS!|E=w z5%G3P1-3T@?nM^$@bW0F7TT=V_52~Q#avRJw?~eHmpLkp35cwv!vt~>1qa54a!OX6 zI|+vA{gsMh2FKI2ZHBm=Qx-?1aPKV1oL^Kt$zzD!<`cQ?Y~uI!g1U%<Swj2eD1KAz z|9Alm%RrtBUQ;FkQ(@k6JE^xP<#?IuQu6Qv%s_uvm*bcg&^pu1C3LVu#;*}0(PyAJ z=Dc&5`y}7azSgTSq~V{P-___8Yz)IsuIb0JtYGZ-7NQs9VoAihiseQ;)$&r*ia{$@ z4?@lZnbVPTIL&n9MwAB`?JCxXfL|1oV*ZRB`^bvX;eNPjKpEwpsbWs1iLt1KF~N^H z8~zq5^6;b9H}>&P9d`CKk6fl9f<z!Mfov={%rvbP05_FCSWvD+9*dPpf{aNU%yMJf z>NxW~RxC>Hb4q}jWBR8qUP!q&*Hy3KI5ySFu4njV(TaFNlP;YNp0a^*nvSxizX+Eq zE+M^g&Gy-w@7#YdM~s>oF%dc~p$s^nnb@x;oPG7Y&5cwI%13JqdP(My;q5^lKZW;3 zR*-{&(no)aK-0n653ZS-><&7yyXUdalUw}>EZjSX$675P&ZC{Pu`_3bP>(||{Df4U zEdESz{M#(9|9NY@8J|N#x0ZA5ha@6M-ETj2J%aBy?YvF3N6fsVNCAdJdRbsnDL1)% z&yp&cB=OkSZBcx?sbN6e7{GHDWHJbv`m08>1(fW3d!uICQTk&Qy4&=UDdAJ4-kw!X zwI5hW<ZsR8Ns*jO|3AM`1=2UsNTtdfyFRU65<ihS?r75-Zo}83uZUQt_IZWWC?6aM z9O~t#SJdd!Grd{n?cV3|7Z}(@^gKyFOcshQxLt@R=l?WHA>{dTYv{fUDT6(36=X~% z5x00bXNdB6ruQwKoPG~!TfV-MUs9f`_{UsBa{b;6uwD*Jo&u^@H<0Z&3BB=bQUYi* z^EdjpFlgj8j-Fa_jTZ1GNq^ZCi(i+;?h*nTkYCU&(APo2*LpIR)Wn1G0SX;Zz%KfS zs5&u@p!lM8HwpW;<9p-?>hBNW4mnYF2H1Q<52%*l1U84?wz+k#w>Mvz&j4X5w+=x0 zw?)9Pi&1Tdd$)*&T(X@pJO8TPmvF|ueR@GBqWPA~I<1|F&=jVRXP3A>OVr5~3;*po z;B+#FKLoVDF}XJHiNW1c%7b9SuI}}Vj{$g(b?jgW?ieT$D^*Hma&5qWYvfQs-LBM} z&2W7?qiK8_g$p@7kPt&cj<*#NUk|u2PsrHWOcgWqm)yBcaD!7Lnl;B{t*Zq_nACgx zp+PI%=PMDCZMO)q5R%q*Au0gLEYq--C5Y-tQCa?obrn_cYWc976}0aV{q6iRQQpfQ z&$`fBbHjQYYn7Z$IUNTcDGSbz6Z=t<<3!s)lKFnFXQWbmX(s7VtqtYLd$~Js<+fQa z?$QW`iF6ypj#!=j&7hB81WI&E%cj`z^lJl3%Msy=FdQ^!p_D1;Y4AH4R=kv*r6QGi zVp_q`r7?TuFqkNoA~a)lwm|LwJS{=QI{0f0b(`A130VzK0TXOiM74LySL*612jebk zNh5nLi(Oi2$4+6IF*Ni9s=v!qjIL@ZY~+a<Sp&R9BwS60XYewpAc&2DU=Ij(ZCVJS z#mSSt+d6!uwV@=K#?#_I8C<d2mOiMoV|B+Xc!il+J+QAOG4~%3!v}3D@{2_CV5Y6) zrgDc3sYDPxe;l(%4im0b?Sbn;kMHA-=MT2sS>Ml1)fBEASaUQoq{s*Mmkno8{+`41 zzKe^^E;-gus?UL^(5zciZnzj9tHc)-v`5C*sU2sDb`XRpJ}8Bk1<aX{oyVVw*DG`* zA{Dydr_hrbKtGwDSnT;&U+Z4N>bL$F#TYX}Vw+fmPd=Qr!!0Fu+?>aPsYkVKRh|vD z*D7YxfKhl=7Vp&gUBKlGPtojikWXRucb<*^P4x*QBh6Lvl0=}7x=fEA-kx}qZp@Am zsH;R*JYx1-W3*Ug?)sHoz~{*?FBsbE&tvtAFRMQp`{th~UL)dtsLSy5ldr`J(-WFM z=F?V7P}OxZ8!=Psw3r~ESj6MY*?{jk_~-&^&zpyS=hS}I7&b8A@s$>F8QTqw?Cv$X zwn7+p)Zceov!j{o9fy2`23PU%S8sTZqXa>OTbmeR2ZzpHYfc#Po#OcOFa+$yVx6cV z6QRqJ`+TE|klA>f%7vQ!<yzV=U?=yOzy7OK>O;S$M1}G4w|xh1oeE<B@67u7%RVwb zwHM^Bi$|d`X2KhrS*@k@rU3zQXIBiBF$c!)klp?+qggvbRGQUlMajNQns&avQa-gJ z#{ai8rFF0GEnfW&uuq%#XwlKM$pL`Iejm#XMsQy<keuoQKK{u8dzGdqORZCwHC}4u zLj$@H4Pwj2nUAxklOBI(oi>fWQ#~)Yx*ILSBEW51KkqrnyOL@rWh;DiF<}<8_okBx zFcxH+Mxt-mE(cm#miK5G^7Al7q!MPTJYCx^t=vcb<eRUg;LIAkRf>vsZNKuB_cv@4 zm^b(7Q;Ii6<GToWZpUE7l730)Y>$bYY%|tOlC#s_JS|K3kSE99|8DIRq_dwNq}f-7 z_8>kL`{0|UqiWn)h`}1;1~~RmWO2o!$tSqiGIa=GO-p*i{PH#NYcJsyopkZcOI`ne zS@Unk_gjuN3~W<v!rz%)FOR!-#7E?ZTXE<{hKPppk>FM~s|Wd-%LwwyDG}E4XjU1^ z5gFggj;%JRG1{<!A1F$?Dpf7SIE`w;-dhIk0fnF`mS|w~mVU$co~g2|0&-WKx*%SY za~N?IJ43Ne#giaE@CNu_`<bnwY_H`n*+uEDUcGBz5S0)hF{x)aRpLRtBL!XK+MJk6 z{Cz@GPFb3nrJ`{>v6(9d1GoGGeJ!7Xm~HI7{!(PQm?@2H#u2d=YzJ(18O7gXLCH^0 zYj$%(TyP)$HVQNzSsVjrsc+?aSSY0;+Fr?>D<rA6EMy5vXyWtQn8bAt4hUM^fT+LY z@FVd3hWpcC2inbygG*BCMPDEW>;wAg+%~<4lc$h(v;nY$rGy}UiPJYTUV}bEu59)} zfcn^N;x#;Z6uYjJi}{FDFht_BWwh^&Xzse><9Cf%^~Udw`ue0!nd5Y>To;<NJ90TQ z2xv@#6vnic(*kEKqu=03njh3LV1OQ_E`a?ylZ%s~k-DEWXN^8Xu2<B9RsEil9QW8g z!ASUO%i|{`E;NBn{`<FhRwL{lo+Pn#b!3^Y3ZN2AvX1n;DRL>3qu*wq_s25RQsKWb zNE+Cizm91Y&bt17(4d&S>k!@1s=-QV@$R(m$Ui~i3DbA6rQ9;xv8+-2%b8A}3Td3N z?Tc0U?_joa7NgDCE3k_(>1ONk>ob`ohv6}eaI7Fko8UU_K5K$T1jC;m149b=zgPKo zWo`EE_s>H&1zN;LO0H6_Fc~DV4ms6yBZ?iAv$u~F)RlOWJr6@8-@KC3Wbxb$m|tq> zpWj@UxvJXW$DgQfh(0L#ZEna3G6op%ke3l_@4VioGmU9lgR3u9D#XD-xd>{b<5%es zS-~$ywIZ~v4(WwS&BQLhBv~p8`2GuSuCseI?OiM1rk9{{CRYH4(dGI)Tk~Rnsf(Te ze#w+B7G2!hpO1{R@{`SL4d96h(Wkx|=F`g-%X&-s6(z`MX>$edXfVN~L^xHS%I{}d zFC@C>!K0mT0sdY?b!qM-+P(q)+-1S{=I#$S8)F^rUGvO*bDqpmsYr|*Ry;P0DP05Q zmX%$~T-HM@&FkQcW1lbXUGU_$7MZ?R`2W~?%c!UtwrzNZZm?(=&`XgVT5>2A6$AuC zy1PSqfB{5QVo;=GKtu$op*y8xP`W#ZMus8h-QM4OU(d7N_5I`z3)b5E+~;|uVkv3n zp?pcznt?&Y=1HoMNHcxWqd%t3978bQ?n+fGJ!;(e27i?M{d-1Orc(r?kwEe-m&z6! zC#dC)MbWzK4Opk->8e_T0y>IJ3o1RoI`{KO=!L-YsrpxwwYDy0TLo1ksS2da!o?cY ziO{-E4HG+T;$P^W)zGQ;9TU4#Fu&|2rv}6cB+)`9GcTolC>7ikji3TSD$Vo*t@=T= z^TXZp%Fd{~Np5JxeUK;!c&gUQRxkfU;Qwz{NtINX?GVT!DJblAYdqFFC-7)y(f|x< zIFMDu%xLW1N=H~!dbPvg@Jq5-Q4$gTa=ib|v}?wVLK@s#;yOGRQ5l`zL}5H}6&*<8 zm`nC-8;IpK=9uUTZ!1n3*<3z2E%l~&_dL`_40H(Klq`d?C02R`3C;D)#=gIVKoqao zhDu4f?i7Y5G3!E$uU)Z!$LwfsQ0IF;HBX-&yUFf0qVspltY3OcOrPM~N&9ZX{g&C7 z3z9ZXUgu?GFtR0sY&0pl3#YXP>Sn(f7<luJakS@cx62c7lC;+G5;dmnN|wg)nq4@p zBm#(~m)aocV)#RvDX;wCMm|<piPsxi_|uO(qeKJ-SQD^VFT~dGh1;PBOzNBj!_gVh zdDCl*W)AWb_8Ai9X~%j6u=ozs^|_1tGycmj_qXEd+rrd@$j>=L?m43`8d@#?Qco9~ zozn}0ADhG96Hc@X-){xYkXw#v&AeK+fx#P!)X{ipFW`)(Ya1iKJUl6#AA%?*37HuC zET~+Eamw$iB=60JsGQK7lK0!j7+3>Mw!wH@bu(?6OO=6IV{*#t4hY8xGrYf?y^HJh zk9g6XUD70Dst0rm-vZdny{zO`Q0)hHz3#=M?-7T<<fZZpjl&(b<>7<CL>RytdM}#g zgsws6RAs_*IL5p@p?T7@i1hOkF(b(?i}Rn#8-Pl=<G~ZF!kmyjhy6a(B?YJd%C+)m z4R5H*MY<5qA5>Ux0uvI|29(LsU>fit`z_6LVa1R}>L_?6HQYPF#r%E5`VaFXF^t@e z;FJ1!|2n;MmOqP+$RN6-Do0RpI>Yemda<WMC4@yR6cMi>^K_>RAT+7TIbJgWTs^zs zo{?l{%chp(PRv_pYPQ|LfXFDCF(CRnQH@a#=z6Ky{FcU-_Ps<c<nXq!-&#Hb6;kvH zC>}0Q8>*UimDRV;jwtBzyI8hr!M0HI1RutuT4sIcXRok-Vdx|1j8FE@KVe7_(~eG= zl`QYEfPy6Ak$ffzcl^W@^d_j*=N<2@f0ZugiF#b5ifh7dvG;4q)zC%h#6`7Sv18SB z-lGx7U-#sq<p6~Z-_d@S2!W>1q>P6?J8(gDtDdB%!ucgPq_ndyjaHOw@FyKpU(>jB z3yKJma}V%pHJ`L3{T@HCb2F3BPZqVlrnQz5etl?l|9N-hKgOBVp3mmFwK|CxJIa)O zB;f+wM060p?GkE;7h#!;1ob0-U!>JSMB%HX)P<Ur>Jd~g@(uE?-}-w9<Lj-A@aHx{ z_v9^u5<NaoL&YN26gAP*Lf?a^7xon|r&ym?9VLIf9`aQPeXM2+7+1cKo0gZ+-Tsra z>2};R{K|#)c#DHCEnqRiTjU)skdiuwLzFu+x}D_LuyI^S30A_~g1M@FZs(lsV(`X^ z6#iKRvWR(tWgjh3Eoh!#cF%mk#)WoBsJS(`^J0?dPz4)0G989&#8Q|=uB1Nnb`Sa0 zqF9zR7w?r(g1z`n>;%wy(clf%{uC-mJj82O-b;749j$IppVPs<8Ks9a;B2=7fB?g9 zjrUIEk@H?I${-he4trMEn2`4SogTrEk0NnhRV<XSHLvzE{+qtmgj!U*&u5P0z2_Cl zyzBM8ko=&lWv@t#zthg}_Q#{7?d+hYi^7cv<Y{WeRg4#T>yi<s7HyN#<?R80oZo|y zn8Q8y{ZF5P?fv!rD9myIYoIq7oZSdj*y!dcU2NtYY$#*Tqy#)9pl-Ck&K&grJ=1TJ zek&)c1X)+eix`zo&fep)PDx-A<6h=xQtl{|V{`q=**|wVVTEi3%#%9cLu_F=UH|;f zt&PJ)odg!I{$HeZfAsUu=;>3PB2hWJT<7mdOHtCLm1I4iz}Y8XKk{OTboWJ+yALJS z+#;2cyCwtMqsbC1<ve|_$U!XgE3Au{+LjWs%Dj<W@Y#FA{3`}90f4hWEB5S9;aP~z z_hHZTx>k$Ed8-{Yc;f8}Y^nW>-D-f-OrK1(M8X^Kb*8Y#P_pa9ESPAQ{lLj@Ip#ru z!V?R|exMDaD$K4ZssEo-WSVL#ZHQDa{3?uE2=QJG-W;I7bdl35{rZ8$hs=%#;$+6* zc|4W!4y`S^^yBB{7+3_Sn>;m2!w+l<sCIGD*#JKVg)g(sBkm>{m;LrQ=wK>okqvvJ z)9F0&{e}C;i>gff@m~HC8lH4_qCb1{^Q)fUr~8(9T2%=F46aup4_cEW_q6*=fZPz_ z^>i=21+K<@<u6CBHIOU$;~$k^MM{QFuHIyN%OHFkdV3x#mb`xXYSGE;Je6v&NyeYg z8GUfJKlt-NX+YSB-xi!GTn{i#>k#OrLxKaYxPCXxskoM`6X{kIlAE9x1|03#17q(k znlDe93ABHIax&h`&P_C=iiMroSF$%3BL18hYeg)_wg?byvThf*cIo74L;zJDH(eW# z$Eb`)%M#s6mobYYIrIf(h-GYcBxjfkz5Gqnz@i=P&_O?LhVvr@WAZ)BBpJ+V8toD1 z^TaIXw78If$k7PHs(SZxMwiNl9sDS1s+)45Z0t+EN#uzNvLSM=+&tU=H3BUQ(>YC$ zoMH|3-(uk?YXWL2J6Rg=N?0tAzCu9&aAiJwmSzP&s`2X95hI|}NTkL~fubbe2n{H@ zbDT&J{1(0OHNFb^@W}+ea%@|desJ4JRUfmZh}jk=8#WGe*tBk{zf{|B+nTteAnq-x z;%;=e&X&M`pZu1>90ZeMXM4V%{RFkx(+E!s_Cg+VSQVGJCfRV@&h>XkeqnM80_EHK z-rdBlT{QAvE-jTwg7J?cu7iHlh3epHlLrn#GMojS?X`0%t*qjqE}f!{A5EUOcU?!k zKSlg`P!Y9YaphLTDR|`Y0i#d8ZUwt7w*2de{pLZx5}-0QWF)rF-y?bVBu-4vGz}}# zSSy4sjN{&TQWc}nr~U!$AHL_MrQTK{(!A{Tu+4MM<8z8*?trO%^chO_L>kaW#hCj` zyU88o2*C>n%rG?{2Z3Whmfnni<2SjBkdf2pi$=Rai<H4Iq`U!H?ZAI1cgAVF@t-j4 z0|mddRPWMxzWD+!quRHI`=5Bm91p%~Fe=R1_NLxhW~}b}{XCO&F@!$R18FHWui*ZU zrEmB3VrGjNaO!$YTkb>2?ecdRtZiagFo<Ud3_I^CF)j%VB`+m?z@$6v`L3Fhubte? zk3&nQSN0}~RRQ^bQj%7;U+3jr5{Qx0ZODUQ1Ilr}#G+T7ZnSzyK2GRV<PXvkOL^cE zk=Tk}yW;LLJ(_2rVGDcbFz6`K3rL>M889~N+0^6>Oz!%Uj+bbJGmyUeT?eNn-c{&4 zEZX$8#R;=sqUDd#KdeN+HFoK>7pz7;oPVm!qf@#%Y)qzi@IJtZHo%=Pt1N385Qxr{ zvjXr1cm3%cV-W4mR5f9DDh{mFDa9l_ED}tQfS$=aHvCmWC5;O44D%%>z5E`d8q8aX zG+}N35s#ABnL-BPqDF0sF8@PL_xtvq{|mhTD>!X=t(OI?owx@;rW;lrEMNPU0;R!m z`H0LKP9KtFgxWUKkk>|~%X~#KFvoZ=Mscm|68F=z75s>3*}HjPZqeX^_!=i7spJSF zPbz7!(^zH3&uTDPZwRt3NN!)QP4zNQ7Uj+x>A3vZ3o!F)42@mu!C7$n#YO`+naD4O z177=|&e+bi3-rRv%`Gwq5}E`?MH|L8Wv*JUo8d#k6gPamw*+mbZ^nT{uLjRMe0~WR zB1j%@Y<@bJ*pmi$(*A|yPyWe9BRApcMAjZzTe6MWciGUdL`>-uTTw3}RrhL1-_o*; zY14)!wFid?s$bX_JMSa;T&~0JF$hY&_3YOX=c(7}ItP~L{IbZ~Bdmcgz-Sq*tDN!d zYXy?(XX)l{|BTd<wq0t(&G9Ut$#=uXJLRtZpnFRXF}0!Z^d#vomh-#cfOFWRno=KV zcnS-%;_1sH%F@=Pg9LVxayP7B1<M-yTxjOVz%Q|8k&w#`)t8Sq0A!Of^Sr{XzT^bI z9aM7$DQTm&BG5wQ2%wpuuA<iH(Fx|Uzg%-o_Bc@M)W{k&;S~Q|GL*w-BD*&=W}yZi z0pCk%zPM`)41V-Ib0D6yh-(F?6Ky&a%Nz(ikiMsc^fe0wf@F((4g18#4R5)X&uZ+~ z>x7Ihy1X$U;JXxjo=IBwJ731%2_ast`{NL)uM?yRgY;e$p2{T>+70YlIhY9F8s`-* z7U>rO;yluSEv1W{SP)x0?o)A45>iWAER`=IWW{)l+G}_j>2q-S9w>45!E@~1jfI^T zQ0<@ei~P7FM_ir9NhuK6Ywu!&z16yh7G>UR2#DMQz_10c!q|y`+yvw)UjjgDir-8m z|CiPTJ|7FDaoBgCsh7>u#AM;oK|@(TnYD65?_Zy4t=kNIO}9M)l*TsV-FiD85@Ju4 z1OoN8_oc?%Y^vK&ELSVrx5gGjilExQ@J?!gn^=|KPy6FtCg_7mv%5>rF>=zcWNO-H z$VeoTU)Znz`0`y$+IGl&HI$@lX80cWz8NurLgsx8crr(cgcFspdRKTY0{Jw&j)ngh zXPJ`nYW8VKYxCT}_>!EPY)6RnpWB3;1B;RYZR+PCC41&3qPUUJ%+jU_*7lvGq3r&O zp7Ia9!I75tHeEtt4MNm%Q5|%&#=rh9h}1S5BV`nac(tGde1Qbgl4YjI6HtgyZ_|Sb zhqCvkLiB+QXVsKP{)8y{2s7tW`tIiCAY4ZAXhR&7I;`B1Ye1#8IxC>(<ZH>n)qZ2# zYCI=si1>?x6KLmAw=Gg(x%|2@ZN~#aBUcW5vU3&4*(t<r$7W?_$s@wj-g(dR6j5lz zrbca@Cx5%6e&`Wbro_|fb}PdD97h4C@=8=RIO{rd!qFfw+q{GsyYdD7J!0(SamE5! z9U3FyLbpnB27gYutCEJtpv8I72LG9Gsa=_uNtJ}0vg|1Diat;w=5=-u$f3iEv6}z& zFVJ``C0>f;-09rNX-<cICAclyWb21yq})(lv=JjB+G!Oq5)emAh@Kw|UHXKo4u)iL zJj31lP-<z}J--s@-Q*jGl3#rU<(6GZO(<Jm^U!9?nK00-HQOu{i!}KL6g&#r2&cmJ zIM=%BR%3jhSH0Te`|sEEueAQZmpLz$I1!gs_;aPIkK4t&O%4cGDk(m;0;g9xXYTdi zHMU=kKbk6&Smv)pfs-p$Bp3DXIq<xJ1B6MtX2=xP-G$KYYj+;QKYFJg6LWvush;Nq z&bW_^Oj(653f#oO+<no#(yT{lW&PxFmSKsNKQkDG$T&5!l}}*l716tC=)hYko16k5 z&vlxVZ%$z3aHR$~PWoX2F;TX6zA)e3i;OaM>mTt0AGeecQ43aKwf6wb0ASrAK+K0M zVp0ca8K(9Us=6(AGGANPs<5RH*}}(hBeW*{@VTOUtrhMJThgv)<VJjBj=~O`z=^B_ z@Y)$iMpMi-7sy~M6G)Yi_vU)$AQnSO^gbw{BJ>ZZ)W`D+$XoizzCr2PJF4emvvntL zPS?|zV5j&HVTa%A`Q<kp6+x$U(~u`b1fX;&pry}gg*w!{U;FBEF9fm%T+zcHlJ|h* zW7SfinXx|0V(!Ub<a3wvsEzQgfm$c8!gf5Sr)*e5wzVX9^qS%~(HlG(B-<ysNv4Lf zcSPHMt8)7PBT#xF!?E;;Pm2$A@_8Mm2?KUn(DGjMmn|k!R$)L9mZfW$j;n}CI4nDv zgG;1)_VVro*Z-pT4X4&rD8BpS_*<GQRWeIjb5nI^-G2Kr)O$eWhnOMRu2z@LjLU7D zdGvlU3w*mF!06J!60^`_%04`I;|?`~J<9lhAvgVVRIeow#^jxp9DLP}i|F3n!G5j^ zRMIv@k}3PsEqNhA?K0MuBT&2t=PJN87ne_7WrJD;zwYf^iI|j<Q?${BXF1@R&v_OZ z`g~WE0(%udEZ^$8@36VsGxY&yA30~0dQ@j)`%p6xS{zP#ZAnZdBqmfAeP~EPNBhpm zk<VL^Z`+UwR!~X+6Uf27wfe7CC(qVtc~ubq&*w@FVzbhDUN19j2*vBgrmil{FifWU z#|DCDv7`G(NdmTWApO*-!_=)x_wYs7Z@(bZ=5|p-oD?jRn?d2GShF);-!ypPm}4$d z%vYAS6LzK?6=mNL!8olmRyQ=2CJKc9?xo&Yr;R+ax{L#fjCn8R@~D)6sb9@5F+I_D zq99~`P~<g*+GaeSc-Ex7Xtl|tZKTPl&PgxJ2J5`)l|bM`zDAJSx7!s_D~ZzrJ8!l; z2}DKkPB~dfZ<8ae2dwD-B6`hN1SJ1iFtzhhr`(0Kb#ru|c%rGom-RwwU2#7c$%|g} z`juv&sY{G{DALB?O)eg`ao`JoYsb5eO0r}kS}32mH~x<=l&95!g;<(7|1`US<V`pM z9dsX}veQOZoKu<URV3b>cwu%#yF&yy>wkC@XUD_SqRR$xl=aqRByP#2e>>(TZTD2n zl$;PfsKDQ=&oLEf)YJ)8Ztd@^&h)aXf9f0Cm=}gzoy;!JWO>nX4ps}QzBJJM#2jn^ zLdi}^88r`gD!iVgFX34a<mB5s-DcWSbw&YT8t{(WaQ^@38y|&kW+_^MM5pF%TC)*Z z>WD85@g%kjwC+epoBme|knb+!=Jxn)Ts@9dG43T;d-hXAZvfUinnU@gUzS<mtp9pO zX}@S6d{J$-nH=SFF7!A2{BJVxcbi~bIjR*XdU5qF->rEuz;4?)pL=#41CLHAICwPV z$3bDxGVpf|6s^+($IO0oOY|=Vb!UiuK1A;O^#$0?ewmb(PZn`JCh8VR#tgI3>G>nL zOzkpo($Tn)uX^qiR<f{mCD9A%u}F4z&GNhCoMRPzJ;`dvKhFv#P3#)b+m%RUI5pj= z2mtJ*`o43NA1}AVIBG^2gdB7P2{F}_5WnahT(LbEyRhqzv1`q>3;7sRmFOPZ7G5>O zIySc+{Z-)i#QgNtB{%$Rta<S`sjH1pH%Kh&T60~XUVqO%3cjrj;!*BSuDt;6$PZ(a zBgA<bK86YC+ke^u)Aj=YIMm=WfBofZ&D)|h(-cL|1WkQPH@-yarF?s3V3V-b{D{%u z>)k*XKCRRn?O!oYpkQ^i0gaNKF|QE#hTw+uJW=WJp;!I@0OG9)8jLqr_6F=vA3biP zqV5zJo5vVbOH~+=!@hZB^@+-fq$6kF@CMr(V1xMlI11XW>IL?WnfL(DlN<z^`#(!e z{r#a7P#F)_;)xv{lOJ7qZ#Z7MMR-z&eCyX%%e;T+vlMTn(?8A7pC=pVes8`q09Tt@ z;Dv!L_wDZzK+$d}a#d;Lb9m1oNL=9pA_E%q{y53z&;Y!3f`Jy@A%Y<B&!it9nO!h< zijODbvgDQOruuHq+i3`7B|2UUx!TP6g!>k3O*oWicZG;wRAT|dZLi%|^})vPT+RqG zF7Wx5y0U_)yUrA5ogKV_uI1^PBifT|;i&9T5~O5u2>vJNzHND7&QrYp7{^vM;{!Rb z#A<Gr^+|2st2c^pe%nL;WrN7d!lb#0_VdDl4t;q!oXd<`ki^KsD(948YpbwZ;VA16 zmpdj^@I=mx46YtXcRC9Nw|jCLo*^6dsHA+>?@ZM{jeD^3R01ST2GN<xes;J2+vJzi zyf+8-1V$$m(xCSdQFbE}`KjfRQ8<PCfgqV<=TyguG>Kzqp$LA0Xi{7&nix@G7B`f` zRIu*!<JPOY{m+-vh#+0KPkMyL1@U=${R5R<(tmyyB~UB%kq91M%0`+uV)IwS(Ri02 z)3VNwXR7-mSL&(yfwlS&are}&+k(p_WaVE3c|F3<+SW5kxgy8BfXZ$rZ3EO({Mi6B zH2Z3SunG&S=3W29WBXJgA&{aXe91>h4kIzVCf$T=R?YAWv^D~SX*NYsTJFyquv4-H zTQ<1mcsdwZhz(XIe=Y*Mb0VlMR_x`tW^S6q9)elFw^@(&=Pa8ohZ(G@?7Rh8kGM|I z1y-Bs52rj0RU@;;xYo$&{L5nf%QEyQQdNMu*Y`+>$LClVj+_g6zH~dI^W*MGygOFs zm`&uBtfnhe3y-55i+Q~Zz(s#U!q@=W7%bVT7(;)}D|ys_Ug8K)I<%Tpn^9w69&7>) z3$HBRcQ7GBiC=}EGM>LE>*t{A?r#VDfssXda&($<o;MoAgH4k|7Wed6nm3wH>*OfB zo~JFaAwi{bt}``!cM@=h0M$_-(48=+!x_mN4|!{)Q4c9@Jh0Ho3EZtEPc$spQ4=_N zX*Hu>bP)3pvJrQhd@)7s8+bNe2^B!M!8kJi%oM-Efb&8e=uEXktg4ruhy1`ADMqH2 zeiIv%$|;B=6uVNB7%UMS1{wFuiVHGcH37pcPjKMJ65*6J5VXSn?wR3FixJ)L1sdnx zizZv-)!+~&;xGSP{9JtT#-OpJ4tL~p{`!R2hMiN51G;U*0hls$&IxH}U^(9b2lvtM zY7L)k<Ri~t&aD4#sw}>=f5`Dp8~Y})BQVk}qMq##5{-&v_k6OWUN6#055t>pR35;F za{JgB%9FO1g4M28B3c^vMJgju{3SI@R8m{tb1Y5rk6~eLf1}f#OIyi)1yuO`BD2<v z@UK@7rA^wP`fIAYDmV4E(gOTfeJj>!<7+Vwf(?iAyJG*VL01}lIW`9>=J57--B3a9 zIVR}e6XDy`jmraqCx?9}BXtqUNqJ?wPOrdm<ueU8|6P1roO2ZIz}?>F>^D*Eq;rLG z?6;qQJyy)srmYM-#qD~)*`=J6U%H9s<y5$W&E7FFUZ((DIEQ8Nvsq%*(?qTfkzpze zQF(O!45{tJ=yt*ceTsZEv2bSiDC-l|owx8Fnfs_Ev8mS6A+W1K^|mYu%85s|3vwe= z&-`g%xac%?S!m6NRXKxd%jK*X6jSip(aSdSxu(GNjMZ?!QvgoX!pTt%zR8NwN^d{0 z-`*z}R>|@Y{@%Nbk8VbqOtl1o)}_baXI%_kV8AAR3WbG%yr9QIH8x6`b5%C;J(?AT zUko$25%0tg3)^ci<Cs3L6xKG-*m8Vr_C69xTxYLH*0yWD<NasXcWYaxu!cD*$(ZR> z{sy>RkIdJlxmAh3Hw^w~v#S*-fct8zFlL#;NX*NbJ%4z5w=%}2b>%>q6qd+ULH_Vg zn$65n8{ny99yje&VIi-Of=ss?obynsc+M|aJ#=vlWN~qQn$`ou{{9kYrUs!fdE0XG ztVZ&{qjE{=Sd0lqHz{=DI~DNrgxK?{`$``I7qmABtM}XsJ@&`VC(?J~UNCa8g(EBF zsZPWwhw`?$@upACY9tfKICaw?Q+Ru99$X0UZ}|A1#2t#=m{Njen~b&Yd@Y-bc!-vG z;n>n@zNwOZY&85lm@}@<oSmyCRj47m#_lwx>eZsl-gL+b4ssv{+flQJ-o{wH(6hb@ z(%6Q7^Mw~BbDj8^hO8bNF!BGKs#ppsWN+M?Eitf@&kt$}0Z^7#mxspx>wNkbMzYrm zWdxlrsaPz0FbRrt=%z~%lzh<6KN?RJ;5qHLHd)K)5bkwwt8^0>F-2utu@Hr(-ow|} zqUtueqyAw%4dt=erddD`Aq?<@FVWf4!Sru0FZ4IgkN4J)h5d5T5;oTGPWtso&VcU@ zs+i2#Y&TS9=>}knfJOZYFA3UlDjia$Zf&E6(g4DRx2BB&F9|5$Isa^GqLmM{$u898 zO+kRv%~96RJQ&ahBSfq^TO?i~!^W?~S}W}SmeBuAFnb-(Qpd6UO-}&ZbMTy!;{gRX zRAk{5M}KTk>*@`+&R|D9@;EigWtM4z?M*BdLM|uzAzDR8k*FI_&X&nyo4-d>Xs!kC z$eNC8BhmZ@Qqni$Q}1^9PQ9mX;=O29nCHqk1e2f2n$f-dZo*!l$8*}Lg?m6A*2m#C z@~kiY^ydvt-F@5P6h{xYsD;IQY0gCQ0e`-~Z75%`f<DY2$VnHN<0PO&ZK#X`Hgnyt zPS@MNK8<Ik{*)Z#3kb#vDuOP5^u0_C09)=7^G+JAPAp=0U6vlH|1C9$W|uA#;VAv$ zCNvH4negEU9Abi5nkZOt-Cg0dE!o4<x>b2$d%9AAj?$@vpPD}kyL>8^$rbQm^Yq1% z7|E&cb;(@E3FWRmHEB!rQ+2A%M}^aw`irCHwm|Q+8>u2L*4@AO-=n%bk0^tbE3idX z?SMhpGGUPc%qi5N(weO@oc(L9Di0*BYez>BVbe$+rjoR?6LOF(>{m3<LY>{?qXT|e zBv-W~zu<J7rottHP_jj?>?69!zwWXd(|}*IuEeFv>k%vJELa(2I<1BhzJ?sBJEBzx z?D%o65BmP>fzm$T9cW~-{TYeNExpEx6f8?$+Gj2Ynx5Ykwgo@q#4}Il_OHuTMVx)% zSjr2KW9?QWq8Ql+N~5-IX5P0N{w7~(IGV^((#-U0J$~LOr&uh@SiyLdZEXMQOz2X5 zc>sDv5sEjKC5o2ILXL)9o$Jr%0_C^^YgoEU9*n@p=NIYWKr{kcZ`$h#ME#(7Cg{Ux zq<+I^T;E>G{$+WI6n&z>s!?kcv7c<bYV%}pC{?1u!SDF&aYTfyc!W6~@av9TvUhO# zgGo-s#>a`=0O?Sfh6}_JjG6$`%^NPW8WI09;j}|r!(ll8RLen?BKWH)`V=7B926AK z`3*{{l0Gu2cVFJVD(nzyI-xkNOs~$a*$M@y%fS<3H0I%ye%?f^OV9Cr639_qWn=iN zn6?;<+(tgr?^{v(NEOpX!uc)Q+v`plGwbdO%PAiZYPBHZ)3x;W=#CE$R`_oMY7${5 z0+*FUGyCH{s=-U$LG<UV<#Li0Pc}cZm8OrDcVjD$Q=Xar?7)t7g0NIJRVVnUs*;bW zTbuP{ur*fqqRA*5e1zs7nquF-=hg9@UHN@>YK6Wq!WvVqzLw|xbM#3)nG43!0b;~s zZIXXZ%U2TQ`t-C(#Ym?x*l`|taF~Ou$!ec?t#|!BaAK{!|6dpT|1io`FSAG$o3{9v z@fpPi`;JlXu8;H&SPO_<+}=+)TVn78f(|I)OKNCJF<loR>U$pmcq^;R#!UlTfry1k z^h-Je)iZ$Su5SfMZH|h7YLj4+NOm7N^R<F)CUOJxbrM7dV*4-oLrg{^+`6EfRth;9 z9OkI=J*NrylMZS6ox=pbIOuw*w+Z}C!AhCqQzO?}QouU2SMoy){>f02G__XK(jc=) z_5tzf=Prx)t5v&Ms^B=Vvf1HjM&1!hAj@OjNshm|Ma<1y!7j2^fgibEI9AP!xcE&( z{6i>^>z*1L-REyDh0+X6lS4PP5~v_fSty?KZzHkOC~AmKJqaXmVf7xD6l^Y^!Gy*% z5MTfozeEXz0j>8o&pUt$6Q(f(I}UvyJ2Bg=Iaqky*sF4+J$vUCqq^HSx`+9diOYTY zS4Yi+OL|uz=voRO0nN83&^;NZrI_18ZVM+C;of6!Tpk$TV$?SeEue#5^sx2!%zapu zmWox#JL8-b_CH+VyCdpoVbXi^Iz{O6%dsiYnu*Wz3G3rq7ONd~Je&C))bxlxwvU*+ z&zR68;8+oSGIc`%KnlLk3e|5D`Tb%EDFUigdlUl(N$K*j(LjIB5p*l%`jJOQw~5nA zosk_TF1}IUNx}Py2_(&{o)G!3mr!_`cZdb#DHI*Pw9XzqKpI@xI;LogQF1TfA7`Ze z@`i5Y@REbWFL&=mgnH3rd~OkEMOkd#sz3I^wARL#rYbIA5%YRzw$~A;^OTKNXLw^b z^YxP5cqOv0@*plwg%6${5NF6aLD%Y(**?o|%C&Jju{Np`(b*kFW;s-oj+MKUk>^a) z|0yWG$w~>ziE6D=o;;Krib?lnba#re`X01}O(KQp1Wu5L5v*i-58^$#ZYUJ$@4UB= z&WiezJ<>-k09l!&++>0JbyS{!2%d9`7{*fj7;cm-z}eYrI$uR{aY1W@T6K5AY<?$R zb-E37%v@^R5AI^zp9!LhBt1<{^t7Ou^y!;-4Y!cpo^92gL2d)2*xFs1)VFX26@$<D z+P%V3|JNzMF_d_VxKcfp(a$N=<2tO$vW^V?`-~x~eQNH#+153q<r?P0B?gxuEDe6* z1F3d6Gu(NAbc<<HC{&HrNgB0!r5(O(dCtqpb_!o+(d4~aJU3*|^J9#<c6gg{+s&Rn zuQ9Jkb<7mlO65x!y;fMcrdPd&uqO>1PLQ`4zt&d{Ib<J?YQOK$*PyO(p)83@P=C}T zkhMab$zoqK3f1b>AMS=)OFHcmQr+xPW>1W+;1syK?n#}7=jy<7e}uU{6KRR99#e(U ziJIz|Nv~*n>o@U_8_<4UpG1<=PM2{!?yU2GTG~kCfUQpG$Ge>emdkqwuUzKx660ZM zhMX>OPS3vY9iEH$zX0!sNU@s&hl3{H|Ld*#|IT?@QeBY$LJQRV2~UG#>JUqdoltjw zImw;<_bdSizF%Pd{SjP3Aw@MyAKZtZKdAH>rsUoOj;#Ry7go)=Op(;8(P023FB+mN z-Tk3bN))yXxE68De<S2icL2+!$n@m?V*9XB0!v}qK2O>0N+Fm+sX)vtM|pKp={!4f z3J=>}gfZ9hA6}zp_*+><4}e5}?~OvJ;pNz8o+u)-`wT>JNZ2{w#-y7$pzEMw_{XA+ z-JK<r5AH8Uc-Gt<+Q#r_dFT}#f>){GreEAy%XG(89>QK%mG1Ri3ZOJby}!ZJ286ik zUs_ztw!YX)vCDJ_`3kID`=g}iZI)^Rq-?at*P6~}_w-QQb-wiNS?p7;K<}>NXG7!O zP$U+QVS}iLU#pfUf;3za^<U!DMsqba6;7bZ7h`yzBjNf3U<gsoU*W1PB5vs+fGlI8 z8}-o1>pY%t1-rqI7KUrvGH6aHq7I0aU5xz&ZRIa%BGTz>rYO0<PWs$ufxd2w?)=_y z`?87H3)*zsxeC`?+W)1oye#1r+R@Uf;n^|ADcsSY(@icV9vCU;J;U^zIu$3?8~Y+v zA^63}_JmialdU#mfW-J~ADt%`UIvDVNf3d|M-`=M*QZxLE|$iZ+ci>l$Rc;Lk7aC1 zl#T%<ibjQi&HoG+EdP)=f?d`gqOBYDC_B7MmtH|Px>+sME>}?Z72lI*$M%ZqG(X$R zSl?a${nV7qfAH42Hz>3r=}Q3jlRidDlzTS`Ha0XFT^NKuoC4!z_C-!m$xsKEz{LmB zYk04IP0Lbs8lO_Q39(;8oe@ie<*Eeg#JVuSi!+2S^bx*^$0?We#oIM=y3#gm&Y`i& zWD*17XWxU5r&1lhXb8JsyMH<nq|c9M8du0I$7a*pkSk)(=Or@Al-B$^$(m7-h;Sec z3>Iv(QvKA-qK9gf-Bd2M=XQlZ+o4e(-C2-;&l@;-7ecXxIa-kWbB?A0oe*2_)4tDn zVsv^axuDUdcdz!e4Cs0SE}$eVfrt<X>cqv;n~lwcj>tBs<V0S_Yc>8Rl+prKImhyk zzIaGGOg0!SWft*NV4r4k9m-Gk+qhKm&H$Yg^rm5&#L$a2`>`7PKnSg8<(QO^P<4A9 z!^5fUP-A@_qzW(kok~a|ncSMAc~*3;C&$9_V<%2bu+YwGX<0Nm`tl~kw`t7&K&XN- z5T<5-zHxK<m!4pzJ(oa;o-)p}t#iQ&KDQP4OzyX3`pMzvdq7etT;<4gI4!e_(1*z~ zoEa5TT$K4acJyu0iV~lPn?KAQ!fT~x*=a*x{}L}9d+4+iwQ0ZE)@49+!9Bl<EvyRO z-|m_qBaidgX)k{TWR3)lB{6+}(?q=BC3)9N_(U_`$a&s2A?8#X+SN4__#!b{5MCU3 zCe#dxUtO)u8I)mqCCcUW<Kd|Za77LAQVm%gYUINi^DFGdqiC#oymQJWi8HKsX^dQc zdA#LFwbXxM{#LhzB<|PLJyH6;fc<sj8tt}JX#>k^u+;^Tsc?i)xx-A2$0!rUfAVwM z*#Ui%Oe|e}anqTj)JmL+F>I**1n$0`aiX|4hq?h6PM5OZL}PS4dq4A*0!BtVo5fxc zv_^^qSurTdIDq3dc1*CDl}C=#1u-7KJWq#;o@(K|4>|<9FNBi&)PxQNPVhWhpC~64 zkLA_<k2K4chg2DKnAu!QMOF+e?fob9XvzE~6~kv@kB>pRRj!h%4&mgT30Y=zi1(KZ z4ElU>?^gZmZ+xq=c^k7iPeQ`fXUcxaZ7RuRx@@@Ed~l0+zvj8OsN1ty>4tPH`SX^p z{Hr_QpZcfY7Bq_?G4JgLf0YAYz_PhGLH6I58gBMCVH(FAl-r%1nuAYI;WSTH4;X`B zqo(;{NS(|&0mUgDJzNms2IdXh<BQ_<k|ZNnX?&mA)spGe6R5z#!Th}?*V$xEpKoVW zOu2&*+@3p~C@X9NJHuR)s;`UUIONdEZ?h+IdOnpyh{{XW$G`O7Bj*wLsNR`YJ+>zS zV5S@M8HAwmR7>`k->F59+R?mRx~`J9uY#C3wK>l>j{YWu6%;?!>SBH1pDanL{0YPX z=H!KO(JSZS6;1H^Td}Oyq*@3Z;rsiO3=dfkB?6#iPTlQC+<pT&3Q!D~WlL?tysA)v zeC6{p`*51m_g0yUpp@)q^*<5Q>BNvD|MtcC38_yRx|s+o+RuOE-!uW$OP8{bKFd$- zwWhJA2S;eh;D()>ganUytZIH%RC>Gj!iz$FU`cG{ca3MdinG#m@tBCYcuMSN)1N-K zFVe|NyYYULdt82-(0MubD(UIPc`%LX;b5R+3v1-W)w5lHPbh8N5Rx39RP}<TSF3-% z>*z}j`LCt21hjh&Bp7m5ROT!TSGVduk!~<$3I8t0XBYZd5017hlc@TeeK6XqFEGBW z|5mj&F<$fM7_p#N`5R;-x|<l{)d<L8%0e+pAF{I>smC9k)sw^S%@#!o*(0q_xp#6c zoj#ErT9|p5E%r?HTM^A`1_R>AQD03t8gLD7aMwuzck)Ob*Q4_prfG7gs-s2<$OEUH zAzYT|TLWyUafuc7?Kd72(lfT>tYjyo8&fOD*itMXU&nJUx@ff{CKHQFJdj(dhiBwU z3v~&7zSM{H5sH@wg43xM9GmW5DpbucdD~-O_J?c+ZQkMC>1~V$Qbt9M$N9T1YNx0V zMT9_QzJGeXV)$FYPp<a<A{Ht}jQ8_HO<1#rY8D5nY!uQvY~OWJrOT+$sEB;|&jZz8 zO4VQJew<XPS&9Qqo%L3ZZ0DTV=?&QF^JIc)se!akD26eOn$AjI^oFOatqu7$_ESpM zi@R6>+eO2W%Q1!53+&kG>JrnSpT(LAmp6+WafVWW@Noess*Os;w-=2?&R;bhe;c<~ zx<b5(i;2cGbOV<SZT(J_k+zr_uARSlh`v~EoqWivbrkH$_4@gp3zPV(44tIJS{bX% zo1oJ^(m2^0;%iPKMWZ$*S7moJ97=${++I0UpKL{*JjBB*;8|cT@QwXQ<pJo*LJ`-$ z9fHu$Te})E^X{E4Sipp&_dV#G1%tAAfh3gYp>KO)(}SJrR;%X!p?EL3fAwc`G19On zL`y{zr(dj8K<`TikM^Ri{L?Qjjl+8h!M}G%LVxyh&oY-fb!Zg{x}8t4cZY`B1d4FI zJZs=FDtl$;HZRfWwcUafs@r~sKiZ>r6S-FY10ykR{A(3@E6j}wwsyHQ+$a%Vr3$29 z8k%?u(R_6B9b|VUf_0+LM`5Iju`>St|04`2M-wsHr&9qahKSl4J4I6O*EUot?F>%$ zd*oI;%L}W8cLM9u6&RN!gSg1uYNtRlRa(%e%eBaf11>kJH`W|L?gN;X=*Sj21`?vK zDTT+jWAN$#x=b$?cw7z|(xLzwbOJKZV^l*ZHx9#r#q0BNo-(n0If+)|LV-Y(f)n|W z)nwy8ejh*ir%WRZS6jL={h3%*X|EQ1elEq5m}##!k(tFani9C@9|^06i^_pMMW350 z6j0Om9f~wo`WLlE;f<-JG>*S1GQ+p;0xu%-7JcMBiAKSR4^HNxb|Uj1SHJEpu(#M* z2H5W}ve;d4y2`J2CT8DmQ4K$mM@v@Wes<0nmOuTGu15S7Hs>8R-L~Gc^2@CaEerYe z`z5~-I`KejkoQe0d5HvtR!A+Tm3l!$MHHOMYVQ;ivx<VlJe$klq9iho*s`0dZG(&? z?^l&sZr5^n!Xmfh4peOgVo!6OlW>}4sMP1i!xIq{tZ;RYNTDY!O-}+>IU~)KuZJOP z>3-AWIX^B`#zEY<MzN}8nj-Fo(>DyV1Yv!bsMjNwpGjjK1D_kr)pL%S5ZlO#;F(>z zB>ijI0owgX>$$Ne%+(N{5V$a51D-m|A27Hw-muJ&*+)y-ZHkGxDUd+=`n&Q2%Z50W zZBdWYEUg&Qh!4dc(fMfxeNMW!U!^jM%#C4(0arsJHnviM^0H}gl~;QHC{9rOXd+mo z%J&eL5kl_+*hCC*8vqK%XOQ_L7sNsK?aw-?d(mAx-v99%eQ9z24!ZjIIb4%8IJGjJ zi;7FfSGqKfI_Mb%3pUeN7^IxYTk<X2o{rmhhnf&z#WDd1gN+30E)M%FiYmv-1XnMq z5jj9NK>?RF$tTqqR-3Vtaic4c+R41=qm(hbTs{1oZoB~XqD$F>Ilns%=ezZzA%9E5 zr<)1qWDWACg2{j8k$_d|+gLe&`Jcb&B2b#fCor^=TNP{#suO%J+a>5&_GeFqK~(Z> zQ2T+}wiP%ojU0IEMLV;B{F>K33I9=Lr5J}YAhri6?%e%OobyfDxWR?aY%L6-4<XNz z=lc4l+S5Xa7N!B_!&>*><<ObR5%umgQufPFf&O}%9_-bePu!;?pW3{77D{p-?TUnW z5>kXa$Zz+Gd_6!;x;CPyLtygxw49xU<v!}{@q<RKjo4#U?^vU-;u+Lt5Lbng4c6<| z=y}^01hbpL;!KFYdTnzRDX@DzNG}b++L!??UGxH#BigMAuu=N7nm=G#;3~;|oZoaW z;(NJmA3ml0O6y$Ork38+{RueE?r_Ihr1{@(dPafb4Ui`XGtY?@pAFY<y2Lz+F%oz` zz0?hcYecHuO?Bj9B2HX3@|}C9rzt;&d`+*1twiQjMx(Vgg8@M3+f4|C5*IqWV+w3v zp(*R<dI}!T2!Abtp8<#wB9Etj+-p1yuG{pdOjkzX{+6J3WTt&};<HlwNRS4fIKul7 zO&uzljoXx`{kdXC`Cl!7c(l#Qrv+U=x78a^aVO3H+`gIY`CBdxnA?GLN~oYD&X7!m zXvpe_?!0APko>C1IR?&NsYdNLohj|y;kd^dw_M&pKTHjK%w|3F<UsZV8;Q(E)V?%F z8ns^kO3I3*qO%ZO;2guvNx#9fjv~3^Kc)nm^E16m0`*^0x8G6cg^u@smgq7}(;0IX zeESh{(A@aT%dpB)(6&#ygIi##oy#jU?p7nN8L!3MFlM_&$^ckjqkGVP8=*)Q5B=SY z3T9@*<q8jg#a033oZKd9daI^@Z0+*YaALaMEzth@X&A}A;CF+wG-IR6KJ$(1;S*CM z;(*X^fy=Jnr-?Hj2W){rAj{>G?|W!!t`)<dm-nhdJ{q)F)@JQ+Y<{{$3rmCn83U|R z&yTI98wi~H37B^MX}EyBLecR9;1!n_9zaiY$dAPOgvQLIy;-nIZ5L<E4-|D8eD*zM z`#3uER((y0Ax?X8Xu=>%HgR<s$>cV)ehN5@3+L*cYcd5WPVV790Ttiz{>>|P!2@n` z&BF`cn$nzmR>4A8wc~3);EwhB^n~DTAb|nVpKc7Y-rGo8pP{-rF(LT!f3J){tg9d? z)n{Y1CYFLyHpRz3{NIilQ=|~1zdoe<-h+KgA}&jV_~eb-6j`>|k7cTi9tQFSX#uL# z&hO%%H>}`70^m@3=^YGE8G^^Vb+df#Z7=Gpl%~ARLZdNHUzQj4>eRdHhxWwtXukbR z@XiHa3MGpl5X=wHsat;f*0ue7e`68A)WROj!-o@&rLDV=!GG>d0EIN>K>F0xl1Q;P zUH2fHI{Sm+F~C;g5}Sqeg)D>}uR>A_q4l3SgUCXXLt0fYpAXQ5nh0ba(7=W|8UPO3 z{M*Yc2axowpG$3{;FMGl_!pxq@fABU_uYmB;uZ^8Z|*5}hUd6(;@W4b>&qF2O_UTR z#jf5L2^B+%-7;Dba#<?_`<}NlGGB@QBqG=GFlcyCE2@zpfMcGT)Vbin8pgPqQfJsX z4<|989mL~(Q7F9~f$^WoLLZ*-x*1{f#v==H+R)4mVYf~N+Hmsf;3^0VE=9v4`tx;4 zI`gN1o`_MYNH@wQo_Di+?Gn$9B@FB`B#b&S?1&m*ADM<Zpdh}87q*d@O|hNMN@ml4 z33foLV*-*Uw_Kc!Rtiu3=Rynp;3!)UDQ65J{$3#?ejnpSfr1poD1Fzf2ZZWu3)waS z9u$+kVa`CrQG4wac;zM}ZYPguqbz8<o*eN{jMNj=g)bzA6$e9fP0Lbwe@uuKRa&Y^ zGc3$J7p!38Qrk-`fo_azS`_j7M2=eyY}k>q>Tf@<!RECW?VMH9);-ykgQ^#-CPVni zIu@*)WpDbfa0gR*^oAdoK^ubLNJT+R1T|WX*q$zsi;hTXC6`{rFJDNmu(do>3gcbT zQZ&Inl$ECs(33-Z!c7js@uLt*LH}}^XOP~V$XiDMVq!C<RL0UQX5dodZCJ(^DMazk zW!{?ROLQLEvuxnAL|*x#RfUNy@3$*1#TloTwzX9*R{hugLw81^bXO+hZ$Ra13(~q4 z;)FyP2A0)C$*iVXyPbB{l}p7s6n_u0^#Rlky&Ojrzaiu6&1Oz~GRA@O#DfhV9J9w% z*cYhGm1}HNbV*@`tf+i?MEn~O{&Ot#Yf)s))whY_BGv0dYs<>sUix}bvNOIqi_6J- z`$4*eJ3~{gF43nIKEJ3T3THHmUeY(?E30ZU8wiF1zj0izdEcqZN;8Sm_K@$(^M_Y$ z|2Pq=!JS6HLc9p;flq6F7nm{_R@@Rs&+>M-1r~tR5$yGw17s{#0%p`Z^0l-VVoAob zD=_OnYJ@j{`0Ldy>(IQ{N*zGYqRU3)`5$D_!MBe^CM{f$4CjSy*e8?x4whY@<%x52 zW5&s)&@doIT??w&Twk#WE>w8%LuI&XfbV$Gx--pZ%UUwg;6h3(T!z*-*dO7)E-~%* zx1!uLmTeptJpZ%bRe!DVWa_Gqx33Q)Mi075I!hgx4#5w~PT%bN75k^es40HBSj<;@ zHeb%{KaVVMjDIR8Sx)^?$dVZ#ocQ1x1}*4_RMDUFkOiWs!3z2b<Ndjc*AdYsRv<?~ z;Y5V~Cw_9}&N5xi8w`i*aqG+U>EuRxl4UkrBImn6>=^`NP8s+3%=Xd(#1;S?Gs-6= zuLt^W3JSh7$MV$|-*9#*i9JUMI4N$S;D?k8F77}ClAIh6GAA5-;g^hSCbZb0G(sKr zZ*7R4SiU?4QdI1Y&48wjY&!r@EOHB?9y9iOzTp(a>Bz?>d9qUn<Ly!ye^5K9&(p-7 z<qeKoX8C({2u(&WbC=1z0RF}PV5;t@+xYdQk-c6oMFH)G_hl5<7W=3zdwyvQbpB{H z$_E+S?EQo~52BkXfj2v*bxa#d$&i=0&R|Lhf4s9E8_0pWBbeITRxr9gU}@YHB7lI2 zb0#8>XjU?x6r8%y!PA%P6N7+jWiY)%6!`MM>A3t-sXni@^p&a8!3&RP0U3FadZo&k zm_-|jJ`i<eWK5jB0!V6%yi?dIV8@@R`(EX9lB~Xv)ubtkzbjv18Zb3>wwU4eJShPy zG?+1Y;8n+?k!1I;I1IR=bh79W#wYG}gW0;YB>mF5{Pg2_);Ir|@2PwgcAuN#<&n;u z5V_CF{N(EVpb98@U8_V+J=R7WArQQe0zDyOwgPc)nTOirGs%|{FT2jfvtF>#`|op1 zs~v>+$H}afm$}kTaL?}}j>eSmH)o|Qp1B0}I$0$V$5_d;e!s>>UK~UR?-%R%LTfME z)DV;uCqK%4ZrQ;#r{I_teor68q<@E2kWKmEZSQpiYhS9GcLYc~OjOy>^U{4T*Hke( z^iHOq63+J?oc)M<z=yu@pYi)=DHm_rU9-0jJa_Gr$o$3zjq&=$(lc(2FoDZRg0qc6 zr?YRWP^Laxn#$bqhprQbuMRslMSm3;oaZ(TznBw(f554QjXdt#>KI+HTwUHW`Re&! zF2j}8_8q&p^OHatx*Zh&DsX~1is*+U8yOUkfz*SQh|~cCHuBwiUW12%ezDcpq8@qD zEAM~Ri>!K8Vfm>tscb>47Wo+)z2^3tywLBb)rTs{gDM-sfj^NUFj9u>xCcUP#tf+2 zRsu_y;Q~Rgs@S&b?VeMeX897@P*JDiQ~#XYKNJIz+p7c;IdObD`|F;B#|+5A-Wv8V zii}LfKRwScnZqnh8LtFdOtBqeK8otSNMBBpw?CLS#B2fPuafi!4T3H-r-m*GF(y*2 z(7RbUywuV;1=pRBcTl;#xm>j*9I7^kU3AUKb-PWZ@D46z_%}xg^dxkyPsh&yiSlBz zZaDbK`~&`>5uVW|Pi~VtHhAq8NP0zLwK0o`gkxDHzmzsq0<g0M$eLS8;&$sn6!go4 zgm7@NMZnc*)nT8Iky8Its6gZ@nmti7XjG7Jh6Um0zeH7C0yAc|Z+K#T>s6Ppd*N1M z$jZm6YnX$}x~BE}ko5D7Kz;JT4CDEZLw!l_xi7XOP4^*<g@`rgP(m#e`A{yocOQ=_ zd<b@r?BOOAw=N@OFmw-F5#y40=+geP;-sZy$7=fX-n}!<DkN4Wnq}Cw*R!v7rf4-A z(?>t!QvK&Jw&qr0g{yzn8tei^03jbpyNfnQlPWy4;Lzo1BSteNK!rWk^I4WJtwAgU zu6#$%It-pSaHTS)sed^1LzW@cGz>&&3d~XxT`|+18Hw_oZJNshkW%54k87BBCQfs~ zpW1~QKS>+!1uqsyRe)#ko`@ojXzI{r-juT1$N7@(;eKz|XSQeb&2Gza#_l|*9#t(# zzv_wZ)qnMq-nJ`z08P_<ID~Y%dd$g}<#B^AQReTfJoyOBl6MMm*A@kNGE3ZgTtFN) z779IO&TH$se$ig14VDrZesd|!W#|~P3Xj`IGqL9eY}pR>M-=B@%3MA3*5|UOdovgW zuIvv*p8v5wYSmve?F*&{IiC8(?!{{`L$!ryzheXSoY#-vnxEZU6PV(<{F$RrZ3074 zLDpOPq+80qRZFVMf$B%QW}1#_!;(CMj?(6JXNjR(TXT(pktfzP$HpyOi_0T5yz4@( zU76AfIc;4NJV>9hfDFULF2myKZM<}l&*XYyFeB-IrxWGhxXPRRy5+-R6v{zM3=}XX zqCCr~Aq!i|3V(|QV<Tnr$Njc&v#oD7N=qS{U48xs9rWROwhF5|OU<WB=Fqr~b3yv% zK)QQIT&z>e()xm}#6&20F0>pRRe&P>t-mY?7NO>BIpoA^p3z_|nProUPNl1-i9LJK zciu_n6_Tu4c|7bh&H!MLE@{dS{@W*QU5q!s+Y}@)Jlil2reHr{-$+^jJFLevbK{fC zV4Z6Un7JngN8&Y4(lnp9v|pDy*ywaYw-o6&3vTpy8N7*}l+)-{|DYFAfliV%3jT47 zA_$H74g_Z|4r4>`>?aWCq~NMD$c>q;MlP-@S0`+wFRFD}_T(~|UVnLs1v&|lj;uJ= z3toTdogUQTFYGrkAfwBRjhE?@r#1b0I52zO{(n&tFo~p9u3XxK%fi!4(xj)%8c~~Q zX7z&!F{D&<C$ya_O$GgE>)v^+xBbzxGZd*v09f7slyp+V#VJl){>GG=$8?c={7ly` zIjYdl0MoJKV59yq&}f&O6&`cefonHp{56#R^#q?D&t*#iTl2n=UeNJFzmWQPww%+^ zDWz+@2&2ZGHO{K(1Z6y@u_$VvVzrLRO>{{+gPdPn{~e2wr^6c>yekS)e70ssuLw{^ zdAp?duZCv2s>Fq74>ywuP9mMG;S&)zti!$i!`~o(NnZ39n{2dlTLsTW^mYduc=1%3 zQI6&mp|j6wXfrWto*4^P9Os3-J8?;aE4Wi2LxUgW*@H;mf-SClm2uEt;Hno}tdK=~ z8e!ilNZP_`K@KLMYC6T<of6_WOG2LAsWHB;7$+8ZD$^8hWm+xQ+9$mrUo}s1k&dkw zJD<R8<AA}``G2T-%eW@v_x*dc(xQUoKtd%&1Vm{RR6rCYM<^jNMoNwb1*Jt=N~9S* zx{;2d!sr|&Fq&<!0r%ziSO5EUJ>dzw*tK2fKIeHJ$NQKQ^1CIwbDt`1tyZZz_Ux!O z(6^_hu{GgkX6VlwLJD^!u{j$Dtc<Qc+14ag77HqMCr^u6YnU<U9>?hGHRH=hYt}1I z;NA)i**ok0r*8%>-kcx<atHht{PF)tnSa6L1!jQUf$^K~W|7#jQ`IjAmvehg|9#?Y z>wcOIHKr(crbnv4iH@Y*{}nrSTiBPDz*Qnj#=~SFc=kvG^Z1gF=c7q2HrVKJSY?j3 zPn&Yj9f!pK;v5Gd{xe`mk-8NsaGZaK`DKVTPY?&$Lq>^L+;^0Xui~!0R%b4tc|7>H z{3!e3R3spVu%iqAtD6nkq?y|NrJ1gvZSxj^8Jzw2nqU+P9O4S8*b6$d{7AZ0bpw@M zSqkpsoEQDKRNd=8$%&k?Nw*sc*N<df+*@4DU$h#Il?TBAnv{e1(s+i~Jp0}ws*{Jn zyyWd@whFdwBzQ4BqR9?~;)wF}wlvBx;zRx$n0SXP6_lnnU&z_0t=iV1{?6AMWs3Yi zPActSE>Oe29tzL%d|de{s>2KR%{_+)&{-q_(-K5iGXa13w5chPD%H06$B>ixnP`*3 zo`{3clC47)Z_HN;g8U#C8v8d?{^bi}lsvC%0;r&O{)Q+<&JN|D*%kYhOhVs_;X%8M zCFUdQ>B9cI;qd{JJ=fbYc8lS?wola<Wv)EHVjn5?4q{N#d6)S>pSqSGI00%qv65UQ zYkrWd2>{HonaL!){cCIAj~(UB`&zk<(KyaeV(L(vFO40mr5THdeyy1yC$12l8EH1% z=Uu5w;yxlyHkMC4GydaBKPn;b`}A&nsImbL12{l_Rh4<4a->sa5>Q)xa|K4k>TmuK zdR8I0;C-#O<W%;S<9!7N^oN&(U>Y6}&=NwZcrkQKyhmh1Rib?NTLboh_a)6z_jdlE z;}StMT=B0iH3RJg?!^H)$k}nJRkKnp1jh1vd8@59;N{G`k#$FJ4Rt|#FH4WmjCA_G z0u;=VmYxHwJ=HEd4<Mg~j-Hr4Are}3o8BmMmRy@@&>1wkN{&LDrLTIWX}f{HR0(}R zw~zWdM0)`;blicD7BSpN>%|m>vDKX%_tw}b>@C&leaVVNE0OsD6xH-}-bp&3D>XSZ z>R*54EMX%N<S}Y{DXlFN_1GLd<~Je?jm|K&MeG+Gkanek8^Y{YYE~gJ(J=WzaDE@} zxAq4YaA&o~C|x&`XyG6yI*1ejRS~|#Q5$N~z~o+?_(MKg&5rp>abv>Dm6HMZe{iHk zCJhtn>7J{2m?6!N<uer7({#}+3Z<(*4R~9dx7)OBoHFNI)BgmJ|19WP&-ZioJf6$< zlep!y6omeiZWQ3#iPAnXA8GDr)p7Rr_N`M=@Gl@+fD-_~OwM{H#m3A8g8;^y4sS|w zV<T@<eJPbB!g~-^*M*{vD-zMR&Rne!wuwT}IjnuP_2;Pnb>(hAXOkGkN5$5014tgj z4~h!p`Q0Ffv%{tMiuT!y)9=F%{c}woD95DwDDg1eJ>HkUG({dLxDn;jaO`Hc!lFRs z7o$H}qdjK6Ly&vBW_<ic%Kuio#GkzY9cIo=a*p0>!n<t$`EvE8y-&t_sFg1wx=~la z*f)0VQVPX~XGY>Q?WO4*Z2D_hOKM}k?kbhFBf}Uw?x6aSlmu=-W^t5`51s-4(L7ws zSAJUjs5e>Ag(EcFMoD0sgF-od+npO+ih<|!{Zt7M`M4UDFcD~Z3UOy%()P<!CHRpP z_Dk*%jG!U~{;(bIiA-j@06+uQ0T6S2r;IPo&zAmiWW~)IUuzebx4-ONKbdq#2x(EZ zKVYWt&TB2oZbB=bmAxSSe5fnIy!bFbsT=p;fw)JyzET0v%X+o{f6WNjs8nfzAkIor ziyWK){boO9AJY#L8bnL(5Ys2C)G^V4hT1U&=2DgBW|RT7s^8jAO`VVC6s%?0#n(Vr z22GTo(Z+Q%onursu7MX(EY^Hmho_uM@)E#)G`u4E^;vTTcuIn)Z15m66lY3TB!$-5 zQ`>$c1GZqae|0A5LY|cnexkRQF0+%<>JtPeuV_xK5q&9|&pl<#rkI2U9wt`r<>)!q z<}GeD==^W4e7u?oIym;%i_|;6G1W=2zeqY{a@N}dd*ChF;~DZ&%WTFer|yieK&w#d zX&-d!9J2Vr5m0Rp(r1~289TxOmN<pW)SjkE5+k`8lKxB4IpzZld8`@PmWfk~t2C5# zicP8<H*rQNTvEzS*3AlwyK-NMkYZcjv%tsHH31q{LbgN=#fTB#A;<_^CpPqtF_pQO z36lfGcP6A^Jijty@KRLd$f}Ls-`3ay_I%2QsangkCwqFMEqvc}6y`j*mUbau{SJsm zfqpU#y_583e&DxmkLtGbXRMo2_n8hvrv@cA-JFPXkO4?Fj@q;xop%B>_8+h*qH!#h z(q>S)-Yn7wwFR`%fIo9$G!hYAClfHsqYYTV|N1*6$Zp9^i73(8EjCfl0+rWMg88?O zgB`*jtr)fz*GveSfSy45yiOj9U<(MEjrOTgs}Iv?!B(7ivf>fP%_HZ6cw)@#u3?Qr zq-~o3Q5>uIZsQ7esIX`DpmI}7sBRF*|I9b07@umBV2?Yz{(>y_@<XTaH=995uUWOT z%z4pAANMxj@n&S=4)@bD_YSsc<6byls)*SgxxFag1H>;qYFRdzt8^B2TEVr6E1MLM zqL$%;POE0qqhEJ*oL$UobnZ&pB+_Fy?iZ$r$5ysikl$S2jenGD4Fj=9JKQ`JrO~Jy znT={@P|1a#TSMwG<*_wSTY<=z&dayH#VKB!6{mE)a%xlF3l8L}o%e+z=K&W!ckP^C zk&JSRc>{BaV=%=E!CWYew6Krw<PL10&h%ZnYUB|;{I)NS4<`0(RPKsuPwb&Y%MnZS zF&Z+8fXJo*-m+}Na`W+vzb>AUcBarVxv_VS0szcHWd6MMDo+f0z0mfZ;^b^<oTLLp z8B-YkU%Qv@$_2MyvMJ+&;Au-NeV<icH^&Gqnw9Fjt2_c}Tgy&u1+QCQND~!|MST=J zl<yb!{;{x!bAz;Ee7L<CV0d!xW+tvT+WT_FxzCu{wubt(9Dp(9IWTn9#u>kJ*52=U z*vdwmnRx^KdgxaTpun@?E?PY~EYzeoPdZP!dibOKs_1r6$+L^dXpfEM%6NW_PTwuW zM5aGCab<r2Km@%wz7CtqTkyd|>xXMRf(;*TIw6`2PEt?YeG42JHS_-0E^wRn{Y&2K z$~bF8jlc^s9Uty{6lNnGx}`dgzGmqfW^`40Fj4jngyV>pHom!c`vip(^!Nk<g3Ll6 zTPkM`Dcc7Nq@nsKMkiZKCGKaezP0pA3UUYVB~02B-*}WTR#zXAu)!As6(hiiQ0#zL z2~RdAW{`=J_EpT61#7@&5qO1mrVe!@4f>QEPf@#$KVI_}%)HBFan||TRTA&Qhe$)! z7QNX_j%u9lyEhyv^c`djMGWAul=;2N{I?^Muh>y3OVtLmyb6LtCEYqaR_PyOtpY?m zy4D*QDn)bM&Yv%GFRbqg?<#fns^MqyLb_}ec$(BCMe~}e!SE^tD<b+_)OI0>oX%>d zj)ax@$9vlR66G>tyYScKq`-@z0~b4Ya_I(f1Bia%$^uA9ueVTQwhNPd4VS}fO6vzy zp7SsvDwhs}Y{gc&>7(gT93oPLgNK%_IoQ;i8zLW)jiJ)sGF0SwlOi*}sw4-!^Md0t zMZ4~^?Hm7<%{<l%75CAKXZ8^eiuCA43&<Hv-kOf>zQ^`w{x!OzSJ&ap2Ivo^pY8DX zd8r->QA!MWaBpFtX`U*HtU6YbeIc=~&HZfl5`U>IP87!;ts~_`Fif@O;|1N$yS6UF z3ogw_f5AG99^t6;Nt&{BFP&VhP3iWpdC3aPx>fhIwhuioX4ZAn58NZdVv*auz^rY+ z*4m@;oRn-b#HgV2FaF7>b;nD=gDBo+`-!Zl60hLvlL@#_H1gQD+2uOK4o%vv{3;mx zr1{%>Jyh}=R|{k*jd}wV1LSxoqR1t3!KfIJLq?^>aN5Bf*Fp#Ga=^$Pi(5`_XFQP5 zVn4GWq!Z9-)EAwv*ILkg%X;NkzB8%4d(dDd+u~o~M7-?!Ct>ae-sa!<MDwGSkJQbD ziR;WhJ(FRw04u;StxsO$EecrMsFkN?@7yW)Mh>twoCbLrOnolbZ|VdNgqjG-4=hgR z#yq4|kLJjG*bU^!k@a%9YiXpgWesgZH2JxBdu6Yv8t~c=T3Pt4UY`V*#&1Z3{c(N; zxM%2krWJd-t9*Wpr^@P|e0fHR*aixov-QBm9ANo=#?(TYAmpFvq;n(4_`A%!x7EMx zO*S0A!@U4&r<oS{8ZhOrJWp^>-;4r=ADQfoGG0P<T5*zXl$dctiwsF}r^^49oaEDv z0mSvY$$~|cfqE)JnH~42Q9iJ}_Ltf=WC&;=FbJi9Xks@(*&adeRceFE>C2Biwlz2f zB8sL~hdT$h_!kb}39T|Oh1~(AoK0ieCf^AwF9?wCEMcx~wC+qAyS7ubTTV5)CqaQY zxtZ2gXpyqL@SeE;d>TdNs@_&X!L9*d*YKVu0!Wd!$Z-QbC5%(eFLjspBL3o&j}oYF zP+G+ivc;{CO*bvQYQ@0=KZXOl-oc4GgH|xR*>@jNe=B_q{(p3^!)w4w*PWNON*@2R zI83>jH|BMzxL?%qGl*q)UqAKb)*mtX_?NKl-Q|+=_gGrLy&#I_HiEM-CfK4V@L&qi zVTQ{zHy(EVw7c5T`?Y&8p__K_m;0UbEMIR38kUIPEI-b`3$BD()6um4^@+v#mG$1M zvG+7GzP4|y!xbNpuktnLT$T!)FmICNL6_2gugafMe34jr<95y`quM7$AE<YKfBS?K z%ekWH+~Ph-epkBwDW`^Z+vJnW&6$m>Hrp#6y=ywcd!1V`9lbovS)l`I^L%ZIt`;%c zrMtSa19qBFRuy=qFM;5?P;gT3+IBqtHxLjXJT!f|7>%sF!CGy2s34r+_BH~kxO&8I zgAgMALIdu81ow@#vrf&IcS9?^7?uMfnBCg+lMstmVZE<7!;m#+jF}h2wVgAFr9hTp zrgxuo0&pY95p-guIDwZpA`5s3Ml-1X2VefhYStOKKU|UExKwl^d>UJ8F^X%ORVp1q z{XihW`|Z7G^B_{nt_V_E9imwnC4;jTiDl0@3*v(FWck#Z$vIWQ#MFPD@<8;E$f)g0 z{?ze{iiw{(dMO(YTAwL>GF$lxoDQSJ#o8C#iO3rWvgY?Es*WRvl0WWEbMT){On2EO zyM!H0EJ9C=c_~N!%#MmP@^LCazmw>JHpy?Hop}AgLX@A(twF|hVc+yyj(@N6+3J3> zxY#0**%Y-``sf_{vF&zAnKipyN@+BFsCSHknV82~ZDq3}e;g%Qo)D;GsdCK6>4uXt zA<pjHc*6x>C^6n{Am{D9mKHqEG)MjPUGb>ou9|pX_tGCz=FhyHvj-mcWOJ!4ZbtlQ z)Skm_eO(9HGCoBMG|Nj-6ngl}Y7m9|Whc%lFX<egHLkj2SSQ5mc0DkxS#XbbpKZ;I zgFl1#&7Wf1?q&-`daXtE#uNC>82`D02HXTU@vm9y$KBWQDo0fRqXoDN5JdNgCFL?+ z@r3T1t$U#yEu~8J4yY03nLksVUwc5-uS-4w$jaZ3FR=!`2zon1qmAaj$Jk4a2UOSh zq_~yvi+rQi_)X9@HQLDkcbWYk(EK+7k34R6)&;d-@d2Q+^^|_+R?fe58!CDJEjJqd zg_;lJ9#7w?Bfn3%4Zw;20ps5n9*_UhKqjqDCBglefFuYHZ$KRiT6#%hlH!hKm+BD{ zRCoXw`5yfJkzg_?7=-$zPO;}*Q|cEKYr!2cTWG8PaFS#b)I;DVt5t(7O@~4wJ?un$ znxFqy?CHyEg$QNwpzYv>`<Ts;3NBxDW>G5372KXyJ<pBjxa8BNJ0-}-Rk!VRtYwQi z;cJ(bT$*c~IMVMbb0#&oz|Mm04HgK^syPbz0qk#nKpYt%yAvOjE-xOvYcB)3BM|$R zQX=f;+6sC7?0F*_Z{GpWqS<L)LDZP(AzL}l1nzMr;rXl`kl`;K{n2rD&7dNRlGB*s zT#n&7+eJ<2crxJ6nztZMzBHOU9N%|+`$+>oTf6GzeAIx%36;G5AGQo|+G!Wf0l;l# z@cKXSz4XXp72bLJd;<jnoZ(UwL%uxMmck+@xu7L%nOM{~*!F8rh8e^GNGSA$RwhL# zTfG@sHJkK;g#thilO$hk(S;ijQsBzZ=WFGgcmFmSDE_D$8~D#br-B=?OMu5>t?3+I zOsV&K>iGC2Q3#mZeVvZr_KWQx{6^`nH&JtYBQb}G123r`Fh0xc?4@*zQ_le^6P?6S z$=}#E%+!{=q>d`TX&}nTsGYe(Ns{yao(8!?W<$#6;j}OR%%@>6L&~SbC77LSIFUn< zvoT0@gd3I-MOMktypo~BTJxkm#&)iDO=&6!BqG=r2gUqLrZ{=@H_W$(BvykPCGDSP zAsM~J=__%Ui-<JrUCn2=7PbyndMi*dzO2?mZ~TA{(PKP#x|IprsnvJs0Va|q+RheK znc>bvR3nbMWbOUzVt(4g7XURa#cj6tay*XU9&@Zf-`y$6skbC+GLWakVhQk^makM< z_QdtkOp`K1WosPcwEVASNeed1tyIxi*st@eg99F>Gi-Lr&qo84In_eyFZUqP4|ilu znkrj+71fL?<!f1ulBJEI{3=C-d8X;!?2My5mTA-v56|vSm{I&y^PlK3i-{jHeX*+< zp+5r|AZTMhtQC$co@MrGLAB|yP5l;X?KL$j0gQ|AH-GWdH;jnP^S4a4)Y!Q8yuk;Y zJW2$v2}>c|YSj||658F#F_3dYszIXvL#%@0yKbt#N_U7lqMSs(cSw<m(lJkWoZKv{ z$j;AyD`mb6z4`PqP!%qZMb7s5t#`+^``LG2SZ3xt#mUj~SOuG=<p0}x-ioMTd>nqf z0xH!G*78s(1u743XFhMEDR%2!Ic(s(aoXx)i{Ru$9J}~dsga&OV45)#;53`VYgj-! z`WWsrTXMCoB9rSZ{WDw+=Dsi}W?{Qs;6?Pnm;+^MCSa~Xszri6HQ4VW3dLtlNn4Xl zdo#=XCMn=&L{hJ&%*x+7j5ArBMg|!GBY3#L%jh6OW%xz<$_)eQG+u2=kBn64LI4r= z004I{0SFJ>5>mHfde_4eaZ0ncX>jAnAva;W8dXfMo<c^5mYoL;IfRu1qSl&IroaFo zy;t}t9++MhwL`*SEfS>Cnq#!3RvM*cKXYa`2T~QgXA^}1JRX+;b6)*)G_5jvAYi^{ zFJ7YSudVq~A}?VUu=SPjEILUDw6AZgrw_b|qV6jl_!Q&^2Y>Zl!84=9{H7j8gjldz zN69iRN8aOdmlBu+;xzNs>&E^IypZ;amqvLlRZEfrHc;|{^$FuE?Ud=Hco{(o@u^f= zy?@~w5h5z;HNA9im#gJ?Ig&7E+OVrx$spIy!$9R|7?qCpG9#ow`_9B@!UWPE82$Y~ zv86m&1oF4F$<)rMlcM#D!o(aFj*#r{9ac`*u5j^k%C|T9_2{WX=mZ{bXUH1Bi<sXR z8~nozmsSl3Z&rDBMx#0`Ti^y;FE#{$6%R6GLi;RFQ^b<FY)lq;QDvbGXkEDbnA`>q zCb>l{Kr3G}OnqwqSNB$(Ma~2A0R#@Ej7j-Cw1U#^jCltXc}476C$|r}j!iy_q9O*n zDk~|rUl+)OcI_jT65xgCkhqq>@?VT4@Fl|<i)n1X$sn-m$-=<VZ{gZ{iso)ZR-lY9 z8u=*}q-5?wcaTp}A{DLoaUEkPe65Vy{L1~sW1q`01<HPsPj9kns4_pMPwCh%rqCHU z;F~K1|CYV~=96C*=?*3TzyZ&s_)63LwWJoe9pl5KP(z*wwc0@`ru_B^^Q^ppzI`1r zlvP$3gViC5yi`75aHc!Zy$MnJ-EG^O$a?!nuwoStWA-0-WveF9C*#gurK#<MOJT<V zQH|27rBbE_*XJ66I0M8j$OT`SJVqEQq%9I*G3^U(q!jEy*bx=0unTO83?x%ttgYw{ z1<K(K+B)Q#LhBDi@NBHwUcr$SIm&h3uKcUfN0h_e;wUB)u6S7Qa#|GT9>sn1tq!yE zrZ@?EMfT9x2Jk^k1~qOF`MH;0L<lO=uh16t>>5N=3|YCn{}j@TY$>YA!m*oZtrGcR zL!}B~R6EaGY9wR2m3<c|vIbiC$-bkf3vDI^%^J>Gvb9sa#%xJ#&7dxd*5wDT-ZGx4 zVc+OXEm{kHL^W3`{Md#&iY4ty$EKpwSf{-boEqR{Q>`_CiMYl;V(o&%TNr<Z%Z=&e z(bT(ct^=yeV5rCMyvlj0WIuXKMb_!aKL2d|!L?RgH=1oKU<WVamhlW`2q{?c`~0sX zQ!^=O1~50*kojHJIocAbS7%5Ds4IPmv?^}iSGM}e?)ng=q{70_3Za1+2SrxlI2o#4 zOEWp<w{y74xrbBE=^-0wH;CUF{`@$T!^pPqwLSznn1QODW|vpD>;PyeEohp0VcpKc zYdh_z#gv#Hk>221UR1i!;17&?R!bWSjkEn5uRITp-yeKJB6nL{j;tH6+$<hnl|om` zyC1xt%pCF5uKkBZ8KdT1x5f(|rwzytF1;<iY9aTs`b+8wz<CTN?zVydrGgpUz1!KL zm#_6tIcIoxKkCSLC0^s7KOQ0U?2?=Yr*X-c!{$b<=ve8_zeHTuUT*xw)FU?Uxya%j zrernpFSMOk%$<w8pImAV`kbVG<LPsbH@A7&vkPw<5l$`bRUJOE+$`kdmUt>*B1B~# zap?*L!`;_o-^X%Wn1u@Y-tdIs58*P<YTQHUd9^qoB}w*C1S@Wz$zP-z*BMKp6c01> z6$u?o#qXx=3o-_%*i7%z+>NZY+xwuY^qNtA1>clP0N??lfY)H-eIQp^OlRwjx0Srs zN8DuFzg{T!(QxlCQ6<BTnQCgds$_zCq}geLW)J2GenxNpUG`!CVNnWLNt0HS{!)SA z!Ric2hvY7!x#1+1H%`HJex_A<Y?Dc#$^!pb+U!>wX-C-0sAqHhuAqjS`cZ5Yie+Rz zdO?M3s>1}Go>&^ctbt-Q7uss%UGc>iM&s<)<b|wYbg>wGtu4yqC#QLmk6yyBn@zP& zHH&)e63+fxkU{tYah}R~cP}Dipk$=J{k=V9n;k{QW2@w@x`<GN5me9T8nswbKXGq5 zht1D%zXKGP>?!e39n5;;zb~U1u{UIKhGnL5q2LYWdW`~lN-HvTld93TZ=h7#Q(4qE zy|cNF*yNaJ$;BLeyL7;$ZMnll@>O?)W&@P@cCaIYqCai6`Fk=_VI?UXi0LTk>NDM8 zRi5BnbUg8=RZI#mf-K(O7KDK-n`-1vkv!W;l~;p%`gf%(2FOW^?9Qt;Q6(yL26H}e z#+|+tM0YlI+ZV?B;RTEFcf1>@GWI5OD(_$N7QWC?xh-ZyXXSP#c;ws=q#F8F^dtnh zo#4T5GoN!Z0o#pG3-pZzEE=;5Q321ZmvBw}Hj7l?*(FTro_NzLex&CN^wB5d=e01= zpGQA@1%V@vyBckmPv)v^O|%I0$N-@8@$T?p97cNwY!3BZBWYII*^(*>d=BDuED*&4 zJ@Dhtn86qw-?5OE$6`VSJ$0pbrss>OJGG|_)^7S57$iG)^l~UqPsfV0`k6M-t3m3r zlc~q{@invJ>tYjm)L<R`i}CGnyeJ#Hee3eV6KnHPx0tVi4mw>k{1z<P!tVMdzT4Nl zxrbL-3H8)3(P;HnaXTVK6>=8kEAA4%pi++&U4+tQB1^l{E*ywCMG9|V7Q;km<OIp0 z<ZhXN?B0$VWY2w=W?C`qXytt3;2D<Xy8#mZTYJQKddBThn_EFLx|X>b?`pvP<dwCg ze#n=C<sM6`t@G~cK&RC;bPCs4Y!tL#JB=RMtcI!uVbJRdF`j-8K-S@v=#|{0;$GyZ zrnGNdH*uf}V4+g8s%rg)S6q!!eISZ~+c!G4_2z-Cig?KAP2O?cuC5AW36M0!1#Zvm z{Z8C@Jg&u3mgSM(j;KET^H~J8s<QVuc8FNITjj$~*J?Y14=Gp*@0&%Haw;6W6WTqF znA;|#Hq^;<zYy?xU|B%R2KHXurkim@x!_B18Uti}<qa7p4xj_zE#YYrjSOzp-R4QO zy<A2Kik!e2c3WG;t<_yX!`%pj3roeS&1>>iE&^Oyion5Xt4o49Oq9&+Egm32mIJOo zN&QswvrWTfAi2wF$9<=*DF;_l*648jc|?T*!$cQ&q!IW!NeqJCQ6c*^6D_Rs(h-B9 zC(KoUIi9=6{eER?^|OfK2;B}<+~xcEk%FUFTG(nQ<`{~6CC?>4o0~kb+I>Kwj5BeR zR}#GoTov!c7Rllei<$W<)Yb#-$)>Cleo?h7|H{TjE=}JC)h|N5btr4I&{+~ygk0D< z73dkOlQU`iZ}{M@eL;6rjhO(WwdhblTM@(c1A=jDH&?skc(`F~0O@S~KK~T-?i&e0 z0tyU0HmGWeQjj2+41RB3xO1bb_uKeEIPz2?SxHwceR@K9xyY2+UMlhM#0aP=C6il- zm2~5B7e0<w*{s0x6RGi?i}vuu<Zm_KxoU`Nw(icGf+8atR>_00^_4?DpRY&xU|_W* z-9Lg=<^!$XVDm@T(*wt-dpJ<s#P+;<l+1r#>wjKvnQ=QUug1}5)f#khv{))3$FEq` zHrS!iBA|VWyrbWfpy0;)z(F=jgJ!o5-~;PlDl~}5Gboy*wSE@Cd|-Dy-yt9Y#1tIq zJ-;mLb17|*VJcs_b{=G_Y0l*5SlJfE9>Cn{k)2NJVIp1S4MmwWwk!SiQdrmAa%nPv zjgpHUJSbJ66PtMjHaS+Yuj9BI{~@-ZSj>cRZDP?(<dSiE9m_yUm0)-^!SsVaq;)L& zisq2p#bB`qB9g_L5hJs>Xfk-7hK0ZglT`|TVIN`zl^?T3yPS!+GJlAb1NG&|{fwb9 zy#?tTWL;_GzYtQzp6@gU-sW1VSuuf~6`$lFHI|^(48I)_$!@V%iWW^n%?nntK1a%a zBlR3C{ZY)8nHY(7NrkXU*sT*U3A8O99j-;jEh@ZLR$qlqZY)X675S!8gR?+-EQ++? z3G`P3r{9bEm#G0M^G3FQ2a-GmiMs~9=e*~agU~SHNv;rCRe*s97XG}q!xmUO@e%tY z%^Xf5EXkloTla1Z8~O#lJj|f3|6RYrge8XI6h8Ed|DA7q-TA#q5NwCiLfE@qDB~*I zdreGTA%E5pyOxN`dJ^ki;(8Q%((OVN+e@}H-RkSWbb~#syv=SQB=+wk4o7gy&!;;% zgonJn<tNIA{8?%yj;|Dd;Z*f%OIE*4cQbi|p|4<O0&=b&<Fn#_5~V*@;xgu{b}P2| z#m;JY%?Hmvtl2|hEB^L@&J}%{sIddaT`StVJp=-&G4M3m_^MtoBsOS!ij0?rp)O@O z-)Xv|esV)MuQ{LVRUFJeC>9>9b=QsZ<~G@P8)fm3KHfIA&MiJTtL30-1p7?KyHwim z3e%62YsU|i-_Pr5G`Gllj%uL2ruO08=GOD>XZ+Zc*~s=8h2mY^Z4JMy4_Vc%vF7!B zR!Wxeqe}5!bEXriQ`?DcN20mUAGuwGehf|6?CieyCPfKZ-+CqlH|b-{b(HuUS13Qj zt~$RZyI@#GT*I@Cmrae-0SxSeeA$*FE@04uEn}rn?J2z8bg=^_Jvo`|RWwMXOUz^Y znHUrY5MfWJhF#|;k|0h70ZS^7<ZyZG@Tx;6l=#1pL7znm@W$518DR=WcjDr@mPJO1 zF!+66r8dY1R`zb&ePi|yg>sA$tcLwR-|I&1tZ_aZL{Q_046t`!yD5~@kABNUSn|2i z2wnTzZ#=wKl#de<BPZ_r4e`m-Jt%nMCqcO34*}i3qb9-p<up3zNS9jk*oZ?X)C>Z$ z!yZ1}*2wnxqM#?{FAR56uBQH?P8$1c5p_K{mVdkodR|9xkzB&`5vFJP;sORxT=!## zfS5c0pOoYkxC|b|2a=R*dT+_+l9mr_@}z5rntoL_AY7gW+YB#PhO8^FT_1A;w>TS1 z6rAprDJd*jZB1@2%;P4lhG9KDPrJzf?}8$!Sm~5wRBH@eifvwFU$(BhKx_PQ$}V&@ zDZkUZ^t_#lS*kpZ3`$a<R8{vI6qp?uEW>zMBOp8>`NA%&g(7q=4p>3!&NCr0Ngun+ z&@@(p&UB_>d-cy|%5y#u?(cOqsP<kx-m5DWTuu|kGVgd_93(#R66Y^pwgCcF%tIVl z&K<e~5!2thsW~d%rb=1!UyknEj7uWrkg0K5?r%5`v-v4LCqm9%_}hVM1V7ezvS!)W zuKQ|cy_kwM{4DV1S9+ksXl!>3N4<{B;`o->7EfkJ%DoG2|GmMn$PvLYtA1n}7Y}?8 z=cna|4#C<C^?QG6@7<`@H}iV#1Vg9=>Lv1Y*t}F>X7bZ=)5@TbU(pXA#wiy1(JyW# z*nX3RQuDIAtB5@A%=vAiST?K;L$PIC<!GKJ&$}PTcIviKo5%)*J^(;Lm!LB7l|Q<X zLQ*P2^0)YEP2kHMm!3X!XHH*Ng~68LZ|s#aAtu*f*)x4u&p-H#OO47XqU|&}k5dfg z5hJ9mgBY?MR}ZF5)MZM|6m=tEu=75Jj@PEof;=4<#(F{+R?O9Rj=ksqph2IT(C2wc zBDX*G1tO1gsK9l&YbMPd3y583zsx^})8C-cIt`!2FJl1ZPHgdT!JS9w#Q>fOznugv z=>=qKSN}1~L;ro0VjYK(JhD6o)y($M_;dy3-PVSlu3a{T+BVGQ=A&LeIZ0wLfeItD z$m1l>i^ALdL49IY<*`|Z6t=obY~E2luTs=IdtaoyGvA?Cp1Rr@sr>%cjB9}n8JZrL zNn1Hz?=8i?rQlGa-M8WUskQn>Jn%kFsuklddoKVT*aCX7u@tS4T6Yth{2OO_{&d@Y zA*cN5&Ss+5&D!uG_B|y|_*CRo-_?x1=HU^d<M#0lrAPRCK*(o{<F@tE`o;OK+rXz= zNlE~@-f(96K*@1g?Iip;e%{4@R=%7pqd1%w9@Hfe7&}mHnyFovMe!pU`*HxayI%bp zW|&y8dpxuH=wdg#)f2k+H<u~ULGE<iSsXIwtU)YP;4@GxCg*EA7fb%?etvLR?1%y) zJ&%><BfL+4-co!u0$}0}l$1_hO})>(xPxz;sHz(FY<tO5ai%utIRi7So%5WT3Ix(a z&TQK-7fSf*@{yAa^F0O2cH(mFKv36><X%q!o`&s=V&Oxjo^1GvZjgDhMZk$;&HQ*& zQ6R_QRtyh{3DgjNT?71I4{D?>8IZ6U+;C6-np{Pl-=Vj3@1<*Kil19#_tceZME#=d z20_^h!a?skVCMdE<YeMPevSxB(cp1bdC}^yutcEZ=wwJqrhIBC(t37gQgZQKLi2+I zOl#QC{34#=O56Qg3U}M;+}vWn$yQI{4eKBxI0tP0eAWSy>b^6gcs^3<o+QfV(~9DJ zPnw0pDZWr~2bqztG+*7>r7#Wrr_%gA>;G3$IBTg`DU{d3A0e(kUiv`a*VWR;bQ>;Q zsS75#NY4LZ+=ISK03LnYqnBb6sfIDLgu#+`7f$(SGxSQ6rQ9-jkTJ_zMNsbVc|r!G zq~;|XhotAqRBc!LK6r{1mQcHVM+gSA?6HPttSmVP@XlLar0vWm0pE*cLFklZng%i4 z+JVc=GYG^oqE{@OS2W;{8)N@F1uc|{@W6E_i(+PR?Tz4%-}fYkuk*|#5fEf4=m$VP z{T+hE2vx&cjib<vW>BUY6wz8kTZE7?8Ch}nr`=AdI&3IMB=B7Hw~F~bH7;TrGNY@H zS+^nS%`RTVy{%lr&^PNrs(qxVOm@oj-wMKw4zu2Mp-J{CBLZD*Ef*q(;}MdKBkp_t zmoml^l1Gf(?{0ey+zEWhbVKh^@xLGeXN9$s;>#y;Dm>5{uWWUMbT<XK4t<T`vqP}% zEZbLj&1Sa&NUy}47UB}&;xL?a_^QNm1Y>UQ+wQK*L`XSEn`N?ZK(L07)gp!Zk(BN( z!r=~C_IK@I68VS>`N_)ywXb7gf9D4-%M10HuR!QZ74*+imt%#O(c-t_abGKc2!<4C zDzA7+iVz0e)#+Lhh+tDR+D6oG#MQ-Q1fm_tFum%kxpn(=RYDGBpShm`?O1PCx-2yz z{;C+Ekj69pP<OE4_eaF8(zIe-twC2{{UtR6=`vu0UN=$<S#FZ1R>TJc<|zUhV~dI_ z?v067H3TT42$AB9EGO%;CX(?&ZaON!-_a_TB?_0P@zVY&h3!dR8&dYZ5)m`QOMXRl z%^c)JX+7Bf$JqONqb*-+6v>Yif%PSb(=8p7<8vZq&vSi3Szgti=@J#K;FG+Nvt-av z#IE<Yx|LE%pb)OQG64phM5G!Io3ER>R~^{;XTcV>eqVAP$(>Ck!&e`gw+~KhM>H?9 zKjs;an;@|T^{d4xw7=CY(pWWSto&{f^ut?DWz1rRm^?6Xu{r~tpS1i5tDVYs1hP47 z0KSqKQy`&hu<Q{GlG%p$A4KElEO+`ZaSigJMVUqX!_Q^rTAP8I@b&6~YrI1n5KQNt zA9-5beU&Bz(z<PoO?A9pDP{O_VC?r>;e&@mS1<G0l5@*(lq1RRW&XAT^|4FD)@~(G z?Zx&^YA*dI!?ON8em0Ra*k}{X8ss<q+1>W8u(^4h9=Y+n@Mp?+GS$x^L03|WX(A>l zGGl`5bQdnQCz2_{f&1BVQaacd8{ivKZxV<W?(vQZ(xYoHnhIKTEb-ULD3T~X-3opu zwYdCx4cXqSu;wTh06Z;g&Xl*04PGQwz(vd*YSG2=kkd`}Kvs)71AAvKTp#?N=exFr zv5zi!*yc(hScimjL@5|plGoy=%?2{<l|U=#(i>S!boBpyIZm=}F!0tDCytk5$qMr2 z4Jf8d!y!`PR@F2ixmYO8?n%eRblV(Ab>6oQ`z}u0e5{8&D<1q~R?JTQRHw7Y{Lvn6 zV}=533wjZihs^q{a~VC(7zn|lU{4w`7l_btZ*_{`{9^30mKC!1PVhm$guT1rT%wW! zS^MTPFGE>HYcaU=;$<7MxV|zj(+;?S_Xj0S3=;H`q1irIeD;Fl!EVt*aZGHegTd{$ zMN3v2yd5G{uS7o9OhQkhW-JCd-4u1YnJk>8;dQ)vJ4&f+CXG7CRgm8xkK6em$uKm4 z(Pvz9;8`0n{Lm8Oj(vSzy=J<p7GsaV4NY*_gGFkFk1JM;Ve=D~nw#Bt?gv}J7kkbg zM<EiPN}GSHk3Sn0Bi9sbnoc%%hs8TI+6sM1o7a0$jTiXp)`!FkGp{-18?7>?XcQ)9 z-752sKv?g(Qa`7ep7$d}`SiCh*Wt)bq%q-!FU#nTakVtTsA*PXM^1Xu_>W4Di3oT! z^Ktra+5*A65Lp+Gg|PV;*%Yo`$HHTVbf$SarOKoW?6>Y>yG^cJuilX6O)=z0qeWUX zk;N}2(63<g2_%C(eR7f>n~GgB%~No#@XzNnR?S6p<Ta)v3kg0iA)y6p>?hS!S%;D~ zFP4&(_Xw4zL3ji*S<-NDN_xSb&A={M@eNn0c9uLw-q1CP)q4mJLU`y7UY4GL;9w>Y z=SS!MEgo_yLT&h63S>%MAA+pUkF=n)a(vi|&@Xm6X+Jno_j=1#{a6AUs}WhW%~CYQ zmBgrw78Yw*vWj|oH<|`;euHgA$lev>Rq^zFW4Yw^2IusBiBu1&LDmwN7z3h|zp~G~ z9Wp=Aum%*^Gwv5pPB*BQp7a1S2lbw{TKtMu=3E2m%B3&s(GiR-pvk_;smbBf%yOSA zC)GX)(;8L<hV%Vv!=<DvGqzR*8pQ1nd{YC%i3*x@_niFaXZ>PmOj+%CLbgGVN9e|K z4@$Q&F`R44)-;cOr)-iF*;+Q9h(8+V2{^b4hVWE%<*&Bl=W1tdo6H9NWiDPdObE$- zjh2DiolFTa36~fkgq4U>sCO3&Z^OCQF4UeZXv<7lA3MQc-`g=xOct>AKerpt9hAcR z7&rV~gRu1l0(rEFvTY~Be>^_~)LFDNq8xUiEyrPTJ|EV_GMQsPb;n)){a`sQEEd$a zMQdm3J_?G%<UBGdvtV6>Ha4%=oc(@pS6ybDO~tFUJG?&fdZ4p+9!L4=%Fhj|R9<(= z9y7?I)faA+Z*M67M+@M);7$A{F2?!XUt#f9w-`-*GZv^cv*&UB<cs@q$z-G6Lr>?# znDb=tx%yEpprUsaX;NZS@XjnKzu)X{i_;bjq(^4CC`RW5MO&%|j-OnzvSY6FFgrAu z+fi6=u0<amti|#8I45lxt=d6%wiv*+63#}3OIDLjCv~!00yj_$|65w<=}MFT6&JYh z#21y5C0DL{F+9cNjH?jFRJY!N=Wo!s=8U|M9Qyp*l!7zbDo>}|g5BSQzd!E#9inW) zTU?WF>p1z6!X+o1x^2uc1}}JIwco@xUwlG~fnHxUxd$Xk`Xnmf8GQX4vzKt4BhlTQ z6QyfhLakDh{RHM$W;M7beS!=<(qk?N<GCx2R}-Nbz#$wWf5&p%w@#HBi!C+M7>&4Y zx#rGC4u;~Os~k?Dj_^$Q4;V!92blx{2zKjUXRNcO`?<b_)ddV&Ni{u;M@}6dx#*W_ z)jVeL%{yj?&OQ_ANnp|R$EY;e+^xq)du0~#Ytj!i6Ua)bN1{iew5mewYz@}Ju@eWI zgBO4mY4;Z@gP~jG>ay2s`AqWo1yWAkh`41=vLLGuW1_yA0(9WDc$P`*s=0n-OOeM9 z&)|q{4f4)3EmIl}J%hL>O&oqbGn$jegVFb>^WWQJ5e}f6e?ML51iP`M!MbBu+c>-< zU8OJVlLcI}_L~RR<7J#Q@#r*QsU!BrU<X_uzzPKN_8loi)?p4Skl|md)MOaA>O#r^ zyyM#4s7T3;C+E<qDj=p8`vF-zer?N|&gZWxY;N^lf?|?_Hk5mPzreo^G0d5o>#%Ug zdj6(TO|j?TZp!cQiL*a#L+w_ig~O`5Q6^<aQ7R~Pbf5$08Jy-~<ny=bv|QctKE?3U zMaehZhU9(kjXAqL5vp!K!Fn+2pxK%4!RISVr3d1w%AP;SVL-#)c{S>4k!kZ+=FG5I ze&)9laSWbE(-Mz7C0sbFG?Z(AT;L^46zNq&`K@|zyb@i-yJ=gpvC`e@>snV1+0Uwd zBuXxH>rg?vv$*(%2@%{)D>a;G{k-ORUs|Y4%i`A4k!7IT95$?*xmO=%4q8k(*YH|l z$vH`H-CiHkIreaBYyGEfF^zFtHNIT(eRrnZts~6>2=yD(0b)(v*2S_!ybz}W4el7k zV8)=Qs28ltqXC$@+6Gd#ttLSRfY8Hfw=E#$1fsjj4?cN1<&IL&j86oN;)_YP(z5%{ zd&n+%1Z!=`>m??KS-I}brSlk#cmrPEs=;!xbf?-UJ{``jh0WVG66yNxGA$00Z=#|h z7h6>aEf@8r@*9D3b&iewJ>m=HLg4b!mJ_GE?BDJy;}kpvu2-3T7?jN?_%L^p=|OTV z6}iKw@5mIHp4KIb7?wFZF#4lk(P@$OG0oV-y8X=hHbZHVw;5mD)f`LzvyZsn`I=?z zbBH%kLw`LlWtqkqd`sfZDDY>xBvCATGE2vZUN-3L{q-QiATb*B<2SW2;lX#QEG5ce zh}CV1R-HU^k+^A0Ck0G&iy(dBQ!;h1Uz(V(dc%A%)jIji$g&U@m4`TLnn0V(dSmXo z^?#6Y62KqVi>g7q_RUv1nQC$;_sz}Y$&w|-op-s@9p9YJ%sgyFaa#W0d4-;CMQrq{ za?I=e)Z&P>UoRegb(RAS_eON~PEmCHQbYczrr|16>9hTII3LnAUxm5@e%jDNlO}hm zw<O7!iRN&h@0Q9<);L{?ck%(F)A)+<Yc&JJiTAzhNhuR=o@~zi=0P1a{JyHFzJSoA zcNh)P$8pp!+oMVZo*RW9#Z_cd$Gjwc0p~AXnQGOCz3Q<+xVdQ07vWfHmk%qbCHj3( z9W12Tv<uw}^r3FR@VVG%@p!n$xm|m*^v>OnNmdHbLh9MReMsVasl)Jf9`pTLDy)42 z)2{1Ny0+ZqMKIT?uk>{aMfFuH#zvE6?JYn{p+o8?%~yGdtsi<HmqNy}yv~;80*i(7 zSqE8Edr}`MAAl9H6o;D~vf<5}6;tQA(%>5T?&GPAH2u}<627S%6gN2GyeGbRLsVKI zog}48l&}=EAvjipSHSO%g_v2s+sT-3+3FSTr6StWX{wvjCj~f9UDl|gZVy<X80;Cb zuE;1(^5q|!$|b8-3%<Z#97cn5CM35TU%S3d+o*GGCrQ%yBR8wf36o#`)Obu*v3*@s z?EXe+DcCnsLOiHn!(~UlG&z_+zlB|VKhX>wx#Lu@Cid5)NnW`VgFFTj#rEhNO?+%O z$<xwENdLO)DVfu1uGxm1ZM823Qi`x$HM`eLNH12o8b$^Nuif>$CSUtnA&J<=`ld`g z$g7Rr_+oWFN4nSHd@Wf_8=<3zZ6SK379cPOQswWTUCx3~C})1II^L)*>Ouv@&XqqE z?9SYC%|G4KB`;)AjA_?0K9S<(06!w{JSOM8QgLO*b&mf4PkuZ1KqEU)u0d)sKn8bj zPxPp-HukmSEusxs-%xYHy~mbHkG!f+@oA5)%)x)Y@Z7OkT<`ORq~os;ZI&>O+qN?j z9@{o&bstwIc&);IHP~hv<0j#|#*h00!?>BN;P_d`#fOykVQT52%VdoH<T26B_40PR zVR7r>D;4D+4~x^^$w`sojZYJIzPXRw(Sc9@SyN1ilj)~OkuJ8VJ$;WVZk{D?*)}gx zoGpA#%EUJpG-F}T>Cb8~T{{*p<n|j@0di6+Q&`lC>IzR*o?8$aKI;>1KefV;>EcrM z(-}2^ThTII<eGNDWV{kN`@N&ubTyzAud@^27xB)X)r({yaeMh}*2^!i7cT$<!W#|D z<^&^(`J24vOmXq2LD<skK@F`HY9%IBM}?FT3Rrl1ZyXJdTAsB?d5ek$8Bx}eDZu8q z?DIr<`RVo7U;61rC0I%Jh)J>4{006)v%|>iPZJNt$nWBUVspo<q%R;#81w7zrO(6; z^OLMv8YVWkicUn{E*o&+3LoekcuOFRv<MGaE=EJkRm5hVDLwA6Il>%6#0<CbVd0{L z+wbM-Y=dv082{hHhaT$&6|V;GjT_H-e#USWW68i8IrkYd_b8ReoLFw}9p$R@#j#&7 zp6?~^j8TI2OS#Go-U7kAPO>`V_cmi=C4#L!7fthFMB9tVWc3u2Yl}PN&4Hu5-jfGW z&T`CU!-=eWH$4(0cY_J}%0j73_X+vH*&~RqER<TzdKwCYwueyzq3w)Tz!1oE(A|L9 zZQz`BFOarfAjN@}aWwmiPPfnSxp_*k^<o9H-{j{Ujw5UCHtOiH;?Dy<A|mbf71|<< z_x382;C)W_X_{ricJDYT`t&Re6*D@N8<sz4-L6|#tk#s-u%X&10E$3$r)a8lQ*A38 z?cd<}dzba3pr^8r7taNXkqmqg){IHu(KmPBuC!?@!0MU3KoUw3caZ)u+gz2wmHS^L z!=%J@d<OJr^_|w>V8wNQ=yRx8A!1$H<;A8nGR6fIkK&sTZ=O^ibk(If`he70OPX%X z#d1RS1+0R$I>}5${6r5QVDs#prIF=dtm@G1ckZV5e9(6Ad18*{JA1<Hu%&w$#(?p( zTct!OOa%;>BX>1fBJvdLTdx_)m;H6WT;Yp*sRxZ)lVCs8MWZ<kLSt>V(ZW9+-0uL1 zbY?5HU*RmNRzBbKnqQdpNFhIdX>nj<8b1h*V&}+ZCCZB<TPDYNUJPWPFz@XHBhr>e zPkXRY2WGkv!?C>+-m+kEaszoX^CfhK33anpA#iKyq^Ek}R>O*~M)GLQ%9^ctrA|;A zfk2_`WbC2<!2C&<kfN&t`p5jtmO6w76qv`>>*{?`(E*oKyHHn{MMad7J07(j<<xz~ z=yvt~J~uyslcgy5fs4|jUE#sR4QSCe8(D9|4q*z~Ch9lPiywJ3pp-hBkkMp(R;^zL zA8xVRwkFd}^NQcEw!%lWM3GXZX<!q#2oLVzr4D&;bT!WUYCOyb%K?%uWPR0gZs%N% zoJ9j7`tgaO{E4=5rJv8q_;SI{f-|(+#1XI7>VLlE;;SzdPgdO8$!J7LT0pYK<7*BK zs)dKDVyt1MQwl6srI}mE`7D4$(A5b37^7iu;C?GxVu9jc!1QS$yqK<;_@TE*!W=tu zLxVy&V0asx`v&;fQG(#EhM#^B8!drSQ7)1>eRp-+|5X^E3o)1wF7#D4+7!{foV+6B zG63<pW`zg2@`KzxXSImt%;uDhXi_x3d5?2?jKuwkVrn6}Zc6#!>9L7D1tUpE<(uC1 zcP$S!itE12PC5KVmJXK`>I7p*a(yhp64DUg;7L2=cG`s)$XOyn|Iav!TFctE@d?EB zOp9-CjlBSfQG9ck`-kJsPmZNeA05><2jkyXawqdqi06XK7aUI80(QxG>(=`IN6_+5 z)S_0a3=B!B=*PB2m@4ZSohD9;^bM*{k<=PB(;7?|-L_OUBv)1RBsjch!R3I{kfIhu z-LmenoA%l)A>A+=%FUeQnOkggW6bTgaSTf3o`mu`f=9$EPq6$*n;|J=hn2T1|Iu_^ z_7z3y4n{8DikjrsYUSK(T4GOIq`?Bp_C>X~D2#!b$D+(}u6|5Hg*vrj=0LEXK3??U z*a26IbF|H*YKhUzXAAMRZ(fw6-W@qfL$eo=m%(-0<-tvv$1%j1*i~bqfWV#Jx>9}) zJ3t3A!hPA~%FtL<coIB!&lQ?shc(`C*CFjo+7CnjM!7l!R1!2dbI5YVZrD0-ZRN4Y zsbw~(JyE$vA3y4i;EfRK9rJzOsMl>p;>_`T+gAOrKWR~*s?dA%?PSuoZnfHC|L|?w z`#t&kUScgF5ha+Z^D7oe4Rh~**jmHhMp9J`6C=8DJHU&I&X8e@5J|Chm^Pb}l9)Qu zZ%D!t*tWVk4p<>ujOa4gK`tc&4`^OMw<?F4#%|TobmYo=n#gINCg8ANBcWJm<?GN` zAJI4TKmT5PJtD$4;g%-!C_GvEss4yf1N8yjSCd0?&MZ<dUK!~xy}*BUMW;3=AYc4` zJO;98->{ocN4KY#B{ls)X!_y9@hlG=-Rj(-^l2|z_6yVSBF)&RwRDOXMN=lVKp|9U z^Lz{-r`je5x{zm|{T7I4iFL^fXC_2yhnciqu_?#`9R#~7cm+)tI9g5_7Yj#=u+qzd zR+5e<_EAk;z46Z@fPR_2O`UAu(&MmwU-J4ZanXZ|<0_|jDO`MeKh7$&L!AQ4NiCq~ z8!W$+?(JId&<5EZnjIardCfn&u$_Tpv#5Bb4r90~Y5{PQGp|ye#ZRv(oN<tBI^27H ze>00a9mVZDtG8{lvU`P?TD(5hoB_)^ws<#x%YPvxPVl9y@P{K})gpDA<onpo?jNd5 zFB7_o2WzuJf9xNSSAG^{Gj&msJKf0D4+Z<}b*utZrLE1Z<1ZT3uPDpb##8w{i#~A1 zhHBTf(VXMR+nd3RF?5$sfc@;9rrlO}1K>zBOxTD@urSFyETx$}47f%}N-VAOR`sj0 zS#njZvdW|f`GETNv$@44Z0LZW&2aOl5^x=m9QpZ|50{I{!&VX511;C$RYF|G!fAed zVRojpjtXjzmzgzZ0Uf_CUE}2n=n)}b>Mx$)nK=d-K`ZP87~h0Gy^f*xD<!KC1HC54 zU%QI%G*Le?a9;JdF3y{%l8XIQ<o(r^?b8AeAc<N%-|HNV7w*wy0N1U3lV;i{vGpt5 z+u3ij*>>#q;gJ18d|<j0eO=CW<#fsT@CT#8j4OLRH@}swxO^~|-+AkM6j04B9?j=0 zRkZ*2r94Y+{GaL|kRHM!KfPD1jr{yd@2sC<&LBk?&YJ@6tPRRC4Zq#@*nf{=(!zTo z?+Jyw19Kd;yUV2!I<Q(kYkj*%a(H}8ZQQ6%i6u99hR*pJ#GYoiD3Ff%mQ@dA{8{LA zvBg{cx2@*?B$2d6L!z$Ed%+Z4`Z7Q{ta~}#F+()y6We}cy_1(bVc}d{=l>s7=l#$2 z-?smlp=JkaQ*CLB)UJ=&D%#T0+Pkg2i`GiSYS9{@idvyn)JV;kF-q+bYLAM&iJ62D z-*>O;zV7?_KJpjjhrDv0$9Wvj!)5K``#))$qbFA3sLDt0<Zg%3+~aG$s^mZGTIn46 z>^Jr|T!5V_$cy)tDRUP3<%4@Vqi?k0dJe08<@7@2#@jD|(I3Jk{?xKgd&uqRSd_^5 z^BqNkj6v#q?+U9xy7o)9(Ld6d|DI%L-aN7>3g|xwX)1>tHzax1;9Rg*0H-{2s#nBY z@;d$pInw**+6t-)oIqylbDUAR+`f|X;}z)+b^K3N3?jXRG9xZ<-fxEkv^44JBH;u? zZ&-7J855oyp{pW%6bnv=;hM|Ww(gyCpK(nQ1)iRO8)_-)d`6u~ghVb|C(sW}RPLM9 z2u`R0sQ4N@pJ2fAOItO(5#3{2>E+zNH=ZyXAmM?QD_tf}&O~?QSLHp(HUa8p8%V_F z$vVk>h2tbLmwAs%nb^|OK+r8VmxV;;MW*pQPB70M9r#L}DNGxyj=M3{^(KWJ(P86m zQ_^(>QuGnG_Wf}m*uZzu^;<97rG<eMEO(&IwrE1nZp(h>7NBV;H~<I3?@`f~f@Vs} z0FfU!>;E{inbI&LHb3RA#4HU8@B3YFk1byZ4DOEBdTJ{Tf5)qeW(<w`4-c==Q#Kc% zZQ;_FbB)HIodJVQgOF7fwP<9`6+i>uVNbJ;XL0{}JCSc{9h!E}j3F8!SjIobuLfPh zQpcMejmeY7U7L`g?|w6cgCM(2VDH!$y+EEgnGzvL8&$ucG55Jbz;i=qYEsHplA0p7 zDmIKEWKB0P8O5^W<=dc1?wyXUZ2O!!yJnzzy0(`O&<&e}mxHJIfUyFm$3WG|o~wQM z@>v&^i7`s(%c@hZgK$riRdCkct){(O>!m<9js4L|FlD8D!XX_+sV$!6P13FybVK45 zOt!}MuW%2vGdYg6C&WvDW};wxCrfowO_W{e4++N%uZWvniYplgNCHW)*JvfrM608? zNw&$pP^L#w;mZ^#9vrgykZED!u!EXL-8vXntLhRKe0e@;t*irKScnK|vcu}q8Eq#< zF{`L!1dI$tg2b=EnCmYxfS+GhjRDzBQ)rPEcUzd1AA1cky&XQb0K8(RlhlQ|%*pF$ z<?W??39&MI?&?YBBgnI|^y99U@HA!R>v{7}qw61>JhFIVl>7rFh+%U@yGi)ngEOhm zh^)Lg5rh44?KcbhV_w~`p~dFJC9Mha-?lk;(FKt^g<8*qNQuw-x;Mo2s+8cq@!Oq5 zoDoLPmF^z>@ACmXBY`hMnL{j<?o3PT(l5>h?eD69vkMSF1HlVRo|J0hz1Jt|C1B)| zZw}4M>u~{&%s5q&ec{xOpDgCpa~=qz*$&N-1pDKP^x_Z&4yC=@<-}JNZ{ZlizVWk< z>^i7NubD8y=}sU~)e64V(Ph5L?m}-lvJNg_a(<$~@=GZ7GgfCP-kwIz>E}OZ;)<@_ zhzq9kBNrr8hhOUT+eYJoqRyHjby?up3cs`1V%x!^jaqL&$#p3C*$4CZYho*3ey2A> z+qJKtb=bsr#l{pFC!Dj!K(Rc4GHlf!6n2eq)Fl+2#*z)r7QFO%pxL%k`1Y{*)3PNd zM1yOYE!Cm_)Q!&Zm(|9s?mzFK0^Dn5!+Suf*NW*La4CZ+r>vC45H8hLitWaqDhu~a zm!D;6YPEec0lRB5{>&u|>g~47Y8FMJiI(+(hcN;EEIw5K++#h__G)<}B$hyP;xqEO zI(>}Fow0c6V8zI1m+z+95bRr705m|69={K-_fSwH@MRDo6^;Jn9!L{R?oOIye)GxO z&C;np6WpDRv=#ct@S~$>T=1_=sr7JGoF_JDCEcf<YV=q1OiuUMGx*0`2b~{E2Q$DK ze3?8v=_GBssr_h&aFw?!ZdX`st>3oeE70eD%l8522SyL2u;*XuIqP3fFdWZf2QYb! z0h>dVCmZRLa{-JX*9C*K*MsM?h~+K{eGr~3B3Na6I$#3k_H+C7dDfY4OmZ$|IzhiP zu0db<^Wo<d*VBobd%&N9W$T$UOGe$vdveHu_Vg9j0ydP&-P3P^lY5wE+mbg&mySbD zos=5|=4AqYMzn#y)%T`bKG{w=3!FwcUe9e{$3j6dyYs*jrGs&NhnTxHA&3e<H!hKp z_)dc$Ey(P9N<%C8)Uq3Sq{Mjq5m#$Y=68p|UwmlKWCcy|v~7(07?DW5GCcxNJpWKT zQ>izJ@wV=zZ=sblWwj9vLYVI-6YFII?}U_3_(E&H3Yi0&(aR>}?3-a<ZqT#iekS3s zLG&#a>$QpRuqky7#1^_#GnoLUAp1fHu^T!VFeKtDQ<fE*DGtiuR#TGCB9at+UT6-V zO^jo!u*+M_ce1Ri%Tug(-iU6HMb!cb9gpEdGG`a=2rZ8L1{NBvv)k!0)B-o)8`whL z<-)zdyN09KLi_qp+RBvXLPnEf)2GP8V#l>C9WnEJUDket?{?D_Hz8fvC2Bhz_R_KK zilELTpRItYsUJ7h>UQ_-S8U+gBQLqE{a*wO=Ln$LV>*V;b31*qPHN1>UD>Jm<b1E1 zN{9_Kk#63;zY$4yXZ9fv%Cgq`RNup?tluIV9MXF?o71g;J0|-kNY|#ei%t3q_FqpH z!yR#VcB;X}8fy}hv|S<Aq1oWK8dIpl;);}rI~SD~n&M&<QG0fc){*K#0NMO-+g$k| zOifthI(_-hbBSH^&6HRro_a8>*FLo^UgWH6yzH0zx5s^>);kTvjD@6X!Qiv;XPdFD zo4IVM)777MyzEYQCgY%DL&iUd=O>PU&RU;!PtUR)UQ`6#|9wV<nTCP$@6YZVcN`v_ z>T(FoK-8umfxlXS#67V~`K{zp0-&KLyI7^MTx0(f8qdJD5ag1?@Q^h`9mMnZ9J-&R zEio%LrZL%P3l6xP$~tWEMRMzQIXHVyz#%zktdOHx2vVL!|7yO10j6vur59aqy(4v; z-2Aoehjc%~uhRZ89>CYiZ&%g)H?;`j`~7piRd=%+rZbx_w=E<H=JIb-X2e(CWb3Z| z&?o`IytP5Wb4;s#eRk}d2Ta?R5(IPi%8agsx2-3%S4L(9w;fUCU#|fgNn_O$4K8kN z+UqXx_?<f;8W}6Ut?!JzS(SpMhCV63lB+ppOsVJ{n^bLs&S|;_5sNLnPfNRTleh#4 zRGF@HD?O>Y&VVhLOl@#+S_Hc`C)`}vAwW)R+1J^UMocO%%P-s(XuHyO(XImIgxM<t z3O~KiH%C;3C^oRa_C3)h>sstN-gorS_$d669K#Iaol^>I04(WGB>2}IC%UIFLB1zZ zr{z^E1W_7c!C}e7S*=9^c75ADD;D}Y{CKxNpwGkl%nSE3nS^zL{!p{)iaG6)>G#3% zJ`P5(+4moILrx=c=-st><~qahTk8d|7+=_*O3)1I5L-vthb!ZT3ixqq?B8&O?2`6~ z7@eSl$^HFWDUjXr7ekKOfngh;*vi(p%PP4QKnj*o(DCMxJ=G*Lh#|*OSpDE<k*MRW z?ZMK!1nJp|;Xg|9i%SdIvwxUrJAq&Wlns}F7uk_-x3O4$@(XUpkVsa2*~EO(y#lv$ zGnf1ds?30%=l*J&xM>z5JEW?FuQ2M7S#Vsi{-LCkpwK|3t5xZ)!Zx?6?7<!WbZ;xx zoS5M%2yzT&+X@W|hN#-i-8z|P$Lw~%5;JW{o2^yU*8QlCRv1dQJyj;GiKbY1o(hf> zX!Cj94`}6`(f9L*h?!TMpNR%lUAQ*`QwWx`e{9RjhVyD&7b<&WE|yJb<D%Fro<xlq zLrK3=8VvzB%SJ7>7XFlX@IiwR^w-dq$;zzT<Lhl_H0lDN@;8OCE2C;mJEw_6wzZ8c zxd<)SM7na2bjjDxp@@PH9<<j#bPilpz5a>^erW6+(;Q#!ZQs5fKQYw^ew#uC5%#GH z^w4AH+Uul<3yDv9MGaBM?@iyu8uX$kiNlICGmdku=9EP)?%|N(nUT?e#`5EgMZ6q% zHrRE^PQWw4rSEDz7i?@f$^m@qf!ppI*UZEA7YK~#)@|}1P-ai<2I`}v`{1o$hcQHF zApS6&6Fh18_-qTs@oHiJMd<&1-b_+I1(_MQpm{=D-DYSCWVM(I%R5`BIQfzf!(^=) z;_N;M6ejH$(%$<R<fNWV5AuS}H~(|9mr6)@XX+(WK{XeBkD-1aC}ZFc^s`xunVkNV z3xMig+#R=S2?sRKg%LN;0x_eZKQ1QDP?oP5Y4{7xxsDQ2x9or6&HMM7J33|fzPl}{ z{|KfP7;V3jZt5oV^GJ-OXS6zyB}y-`L;M5)Xv{{zfq9&2B#gZp%d|UsuH4xuT`s$t ze%?nv)x_*uz*Lwnnndgs6mY~G7jjsx)PmJ=Ca*N-@(6thPNaqDO0|dbyZ#t?DPpdu zAGcO}`&(6`<7kjh??*~JIyoIvwYUK`LZ7sdC9U+?X&0W{?IiJ~WH)!ftW71qd`D%i z`nce<_4nwq&W5S)NHbmW2AwD0fJBYE!dCI`b^_zJLcvFvUg}bA(7_)Y6<6y^w`y(> zv9!0Lr*ZI_Q;`MYUY3G~?AahxK&6v{l$U)n>MQFD`O~nAy9gox=V1(SQb@_sesjT3 zj~S341FOd<mh`<=o0?Z;^&C6U)P*xL2Cy?HvlSPzIi6_Z&Ng|Bywb4@<;ONg9>1q# z-rqKB@ZLW~#FSA9$w2a-Pn&=HaSh)Stp3h)rmJ3^1HxQ+c}I$NkGdG}lh`^GiUG>G zj1ceQ%2vbe3*f8<kS(GRtHLi-dG;SKz)#nblTs^A^{pFC3Iu0rgOk81pI6*!&=nib zuf0{+7Q7h_>XTRHd*8nE_=onU<LnD$buzW(%JG7u=S=UiTUtdz23-KOKQrpImNJ78 z87tN$M{Q1wCzf4IcyZ(BnbxMMu-&okih21p>y=-t-VEiyI^j_(Fd!QFVo>3x@WK*G zm5f_SX+RR&z5ggQl^pv5es9Pb32|l$KuQ+s@VP+`1R$6Cw1rN0tU2$+jpyb`L`FWu z#=!ilPp6trr}nx{8p~Z4JcG$o_BpZgbt^uM!j5i#E|%j*#(rh+g7$z+M(sN%hUpRN z1H>6ggY3`3R4l>SN!^J#xAhaT=R88Q`tkM;O?t)|O?BlNdh-Q82#oKYaMv=eyT$8e zWP&fT62B(X*#z_94KYc?G3h0ifU>))7TF+|xKKBHgzqrM&j78MOL0ouVm-+c2BJw^ z**3g$3sHC%?mRjhohI%1xc1R@bFSIH*NcWSaJ_`qNx+Kp^~i%x=x^2v9L-Ld!suc~ zS4&3%liK3+PyG2aFSYyo+7g`gCtH2hbUb?hecCM6;EV!wo@b!Ho}g~LsPE23GBqBq zZ0X=z{a`4v7p8Pc>%yhz1KUtUz<8CqopQ9spF6YA-+0QL-7jcIa^?ugR7bN*ym?Ip zV>QI_j~8iRy%o??ekPW3&s2xSPm)Hf%Dtt*$@!`xXF5|0K{{=gik^1Lnv2?klP?H2 zq6~Wcq=~vq>Zk@e4<Y;|n498fHT&RbhzscY-Ej%-WFFOI`9-hc4`0M##Y_AGuJQKk z^UR7(l>tN48nXO$0_l%VXv;np5073@nkLANN4DpE2q!h=CDK*?SevUA4<QQM;wl^q z`@FEzW8O}#jZf2Wm^Dc6tN{OO-2vub6#wx9uW~`z8U6Y93R8OL1=v&Vah&fExe_}{ zi_81x)KC5n@_66lYOgI^=OAkdf7Zcj>gGY1o}HZ2u`bOPl<M2%R;Ke`S?+#5@1I<K z)<RYV;tKhmy@t6rw<r02IhJzoNmze(W4(N#Dw5gG<d{!!_M}Hu4!0toK26+~g5QB6 znsvtc*_9hSjwhy9{Iez>1Z+>*SC~ODB$9c$c`$;M*@5+A;vH^Z-0IT0N**Rn)*K++ z^mY$B``}-aDc9X2;XtSRUTU99V=MV&ALVD!C`36Q{#j#DdorurmlQ<IK}zAF%~|N0 z!DSY(lZcnzh)<3e5W@f+049_8%mc(?tN;)~&JLi|II?1w#?5<Q^k0#W4|d&=-UA~v z7X?1Zbe0}^O!e)tdPl~{yu7re_^@+fN$zN-r@VjUdMV-5&%3;WEx-bcb0JZvTpL%y z6qxaUY+nUc;&qLK))cMZ@Oa&W3xK36zA0okJTWa?fll+4s{68jGkq?^Ncke8Q=j%z zI*B=eqlKZDgAVjXhmE9Y@l%fGtC#=iMymkOa^XLAOAU#q)Bjq+@HUyM!&&n1v(jgU zC=tlpRMd2aGh_v6LMX;V4R6PGH_t7gs?wruODNn^#0;f`fJBrf^N~!gs1kgXkeUG* zLv^O5#-6&(MECP{r%n)3$LGqp-k%EiU_oj<!Irsdf!)NhF%?4E_>`X#ZPzV~R)|4? z+h4h}w5JL}Hy~;K<cL*;G}i5?c6L~()JSTBBro-(RR0-a`jD4kcg?nW1ZjeO^b=;e z&NYC~o5KFYJxRyb*`C7YHl~tk*p&50SlMd=r+y5xp&CH{#Et`x%lWCRDbJ)}piUs& z_APXG^WQ6eTB$m{`m#_WSK}7lYfU>)v(C;lTGa@ftF)UVS;V^>pCr^d$GC$0JCcA` z1#8%_RHj?CzxsF!3UZo08}^iCtr4X1woo^%oMWGBI$dGw2?+Lcu03*1Njjrola`J* zN+7tSem{q=4ZwGh)aHL<7_bvAsO))9QA>xArVEInl)0ck{`G7a{9Jvqt^vnu886WV z1lFhsi&jncl;UC%_$>WQF?DhPmQUKZR0SP1lts#;@AR~1q0P%<xbe9hdjUGg(Ro?# zx%;#P<fMY#N_OQt$J1GN5&2%r3@N$bwF-}5sKrIzYN_`BmNj^)0bWU*0BaIhLnP8g z4pc-J_Aq>}HTOLG#nS-Rh2Ny7^<Pv<v@Wk{SCvtlaWbSHv^ITNc&9ObVnpMOx_!wL z8){3kDtg`3kqnYNSuV>W?)W}7>HM`Y>=OLg@4z0N`PNF9#wfcIbRetRUNLUbFBe9& z#AgN6bJ`^w>!?+JMCF=8S>`{Lbrd!Jt}#%2RoHNCWn|Bt>7S&d4{i}>7k{l&o!u}- z-~4VMCAL)Ej=A1cEpaG3b}q2VKA7di6FumC9btl$sdQ;aV@&E_Wz+p~n6vHkrDZgi zJpY~QyLbOyi(T52QOlyGFfo{TM@+Gzw6$`O-E#BGpB?D#Lpg>cAg$({*Z$W*US#yB zoeWBlVq1jgR1vdJ>@!>apM*Dy1qXURa+gFO$8UiKs6uj7KtgxavuS%>9Fs^;Gk-;< z7thEHpmUiOLOFCGlvQzDl^rz9LD+yLY>8c@jqf0z$oLP=7LngOZ`#N(ny-@nHZpi8 z7thgq1LF^!Va*!Egj-)#`#BU6c_X><L>qse_Q~wYM3c|*@WOu9L*7@s^TOp+%Xd+Q zDsPexgYp8U5a|y_ECuSnRbxzFbZ1Yjw+<uQf8i6>f5s;)|HuVVsb~zSWWVS(3OVu; zK98^}R`Gvf(@7p{?AYmv>GYi>vTRX*C}=R<nrE;4*+!2p>^L~(!led?8)(`My{#n+ zBZ(?xEk!N*_E_oxYv`C=+$t)_vCt|`e;<sizQTHxItC0fMJU9p`+lU$ggryhGH?jg zYSt8(;0BpeS6VYbXs$d2NmF~<->PYedrG!1<RXS3Dmd;}?D(}4OQD>6*mbOZREQg2 z*jtH)AUol1Hz<5J#Dw6sG%p^KddOdb<lVI@Sw|;ibi6@Stu!JmVF{^FX}o{Oifgi8 z!`C-5Y5N61?QoSjM?4T+Nk?uCj@5}ZiQo&__o%?1!hujMg~@KQCub#k`S@hhB~l2h zn!CAsumUu6hjhMjR8fD-D(t!YR*`}Gb^%RPo1la$Wfx&GQgiTIC%V%1aE?*z_yeTm zW;L*5Y$E;v^wQiuH#3IRaP;ND7by>^6yL^xrJj5{xBKT5wxOhuVl#85vL0W*UFUiP zq*uN~iyZc>{`<N4CRKg7T%Qzf4Rbq2xK_<tK#}voiOp)hDoZ!?Qcs_rD=AYgR)O<; z{Oj6wtKp2I?&PE2Yq{T`KFR<SFVfahS9a0hh-x?%nY<EY_hm~L%G&b9Mbz5bU#d`{ zuOHA!gN6$uFJ!EbI2V5Kk|<I~xHkG}n;m+lBsHB)A(F6G{zOgPi_WX9f|OSQ?J~$O z)cpqKfMH|p9pbP5+_$QwDdFgP$4Gm5oL-Pu@Fu1&?H%HVp60Dpm%6IuzL&qUz@wut zBkkH9IYW}wvbd+1noEGxz~&M<GX{DhQ1$|GA|+>nrpeq`Fbiez%U6>9znFffMKjG` zo5lk%wiYecl&^yD=5cDNha2~oq6FG=cNIIGsnlwx`50QLEJ|ixi=|9WpA-JbC&13_ zzIt0egif+#j8%^G`R4bko433|KG9?+lZIxO3MP!hF@x8E-bfNx5IXxBx6c9elByQe zwar|@L(ZZ{W5!^Am_}j`F1y;}q6CaoZ28X{%s0J)o!IGu`lq1o`r!zRfdWpY$#K-l z&y~2h6jz~_J$^_XL5z<{`k1oG`3Kilyd?x*;moFn-Snj9YwzUHZN_`ZfmE3UU8^d7 z-KO}4qUlRG<F(E>^|GJ2d%lc48nyoE^Wt2VAX3F(%S=xYed3v_X0YHyX4nXfTFE0w zOyY_`r<n)!!FOy_4p~H-%3!2Qwqp6K9Bc(gY+l-2*G+vO$RVN9(Yo9Tjry48@;Fzh zQJ4;D<1&_}l477r{kGf2Q0}u`qP|Rtr4Go&HExem0lI-k7-s_k0<?dB*wm_mjOwH# zXwnVJ+8o$y)OU7(%+CUHT+PH{+&5s+S2+?0dND&ChH7w9`BpgQvw$_S?3fOPlqEZZ zWQ~g2DptW*`xmCJ{cflr)U*|Eq$K_;)2AytQf_f*?Fw#+I~{~kU_k}0*gb6CrPec< zeq0y?F6}&@^q0|Kg{zVD&P2mHQmO_h{W74%XPo-uc9n;_f*eaP$Olw5Y`w$0s{QKO zW+Bot^3>u6DwhHGGDZNmtNW2#iHY*)j|e;0@hVleR!4u>0ypU6$>;aDn6uooE9~}F z)Gm3{&YY^tWht>)3?Wmu@OKeWc+v^6rTVhWlq%DX!-@b_$(n4xfGklO9PgRL>Z|y9 z(_}ekbp<GUYA$D1IY1YhQche4h(&ih!2+fP|DR0DR5G7)3E(F^T(vAmsUx68HRvv1 z2(zie@x1!C>YF8e6@VJaDBOeN4TLY<vbjOt$~WOs1sZm{ABtu!R(zt?JljwCA&ymg z-Bv8v+!1_647Smq7(4iw1Q#)aGxoNJGzi(TQ0U%bjy!yBmmi|nkG%m)H3SiKob_D- z*)X=fgzZ0eYN$rCOKW+x>lu}bEB7lo1pRL|KRAs7_c$E7pb8wPugOv_2mTA*(fmA= z0Z6GK6yqbZcK7aOIbBS+Oa0(Hdx>c0gY7a<FSozxBvX?``_-cw?2Z7uIs__xB0U>m z9NhVA-|s4G&Epr*yq>q{p8NLv;0jt&>$ZU%lyoPp1ec!dEZ0BuXjT3L;3c~)^psrl zXasYvL0Q<{Lvx;edvItxqkYLTLz)F4O3EgzpI72}u<;HdELg2{>XxYkQ+8civ}P~L z3%*4N>{l!*u`qaVV?LNxWC=gvqjG#Gz>`wgSbhAF5ci9xS`*vDi`f7p$g6?a{m3>? zG&+CO7u1PLl~Mh_m*X;LDRt+ani!47nea>%)`%7z0g%qV3ma*?NNF+tHd1u{j#zRi z;wW@RES|2`hSJfYD*zvRtuYNary470w$2V~1hE2T&6a!W|3ub_|MP%EzYsX<&KUp1 z=-KZ=x6#70s@ca94mFYjQweJ9)tozt$P{oHqi$(TtZJUR{Euk*w%sG@^y<O}^W}e# zyL>Jy$77RbZx&~-5j!ry#BIl(d719>XSo_jJ#7JdyY)=xTuvEix<)Ve9LwY9y>03f z8zby-D~wVsH5?Y4zt<EKR9D<mLFjU{KdG7*I;T52&xZG5Fzj9kjVp_oRjZr57k6XK zQK#5^FH)59l`EnWB#xaCdB!+EEokB6bm{(Y$$vsi)~;M6oimbk6_Qp39~ez3>5D&E zvTlU$t|%JfYwv4WYd!B0nY2#w&L0DRqA95RKH5l^;r!fK!tpnm*Q&y$vF?VqSRwvO z1S72RWUqF;elmLYVdpt^o{<5ngn6T)MxB)VPVvv_vzOdOcEs%Z?-g^0a4rxFEw@4= zYHeyM{<tAMU@FG0qW}6VdIaTOnG_kuR)MxxU<zRT*Hd7HGmSO7l$Xr~4%ijwYF!Y- z4-tiH*_LslJA7SERy1~$*?VK7((d>d6JQgTmPT-;x3TM1j+D#s*sN`!Ai+Hd|519D zXx?Vqslk=)G%Ka5b}E9GAaB)TKLr?*W5~_xt}4Nl=LF)%<Ui)_D$@7IU>}9+8lOP} zUKqH>rqqp%Hd6_C3W3Ut;Tfvan{=l3OOGKTZ6g2BEF4gaRS2=E=RKfXMY5N0<#mt$ z$e)g)E7`oHro))+a8p}@;zsxMV9<73e={&B9!RllK}iDgG0T#k{hx~b^Nvg3BE~B5 z*-R)ZzzBLv+tF6k0vI?qHODf=m4Vb$Beor$7i3HnCn5XD4U5G7fNr(-wIXM$@1uiu zI;1wW*U+lggoR@VK7W5XP(a(RJ;ExvkLs(n{wD->uA70qhrIqY!1&(alLyB@DaDOQ ze&2CwCj99wJ<0qgqKJJ(pZvasbxbhn1s1|(<Bb&*LGEX)e3I$E(>hb~HeQ6?3-FCv zdRuewHfC(q6Zh|S<{B313hqp2_H+fG<jKoog<U(@%g(PY#bgzVDDwSr?q>l77^+u^ z#_cZt&lXhncNYk0vb;$B{(E34N8>`!eY;hv`9l!5NYMSs;mvo%1`rpFK3D=bO+Vq1 zkY13TF(SA(3~Ce`UkSbI*>p<q9@OMY;#h$dTOJ~^SRQX$`9fa?%uP{TsAwlXHi*93 zLd|q19o);}q%jK}W<Gf04J18k)f!=mY>UKSyj_1pR-W2iF!gK`m^}QTrg|o_@vbsb z4onE~t>quJC*3Ju{`lwU1VMtNtBKdX1a&4CD<S`l4)_ZO8ZVW2P8VNL{&CFwLf*ar zbDRHD|Ic=@RagOr<t-KVi(>)jW?YWK1e=qC0#BRS0)pp)QY1;l-DJKKAJ(g5FCPhL z5$;(@M3rAFQE({ePNt>LQRf5SQS5#G{>u&juHV+}7&@LMLdHK2Jg8Eyl2~jiLG<ph zHCr7zAY|*bmG@kQ8@igPQH;_!oxDrpRK8m`+T6<dx)43K7nduTrc^#JG4bM2Fbmv$ zLg#pbX`_dr4BzE0flq%BRwOEWMVI++4w)j+(LJoS(4QUgsN8yaza4=t^*VRxR)gb6 ztXF%t`CSfIdiBZ$<l%>rXMRl9d+msh8T5FWA#~Xt%_J^Hz#QqdCup9HVycz}Z&VQX z7&^Ux2yCIXw|8fewRBOGE`-H656gbb|F8GWT)%6^mZ>s-B|z77nW{?|A~L)Gr3yT_ ze0xGln(b&w#9Xz|Do}k7J!_|mA8hJXvM$x7p>jUWm~3*Ank>9uf=*C;FZ6V^YB$AX zWLh4YMQ|CnXmvagn05UiY8H?dKF){9t9?5id6ow<Wpu+HEm~Aam36d};swdRl=*?; zUTqjQZ1<XS5qB8gsEa--QpwL+{_IXkW6#!Mr>L3>nkmdBz^j{4I#{o-)_kVN^2*m( ztk3&i7U{g&e%OqKH&<44TCDhv)a0O#tQv-V`9{Yu#vO(Grv3QL+++E)s<HJ|x^3K8 z_RBQAnoHp}87=OywR>q}eD!v~{<#YezIOCCD_cddO#T+@dK7olR;dtjD1&YXei}3Y zUy6paU&XJ=Kf>BN>Z$Qb>^(#FQ6+hk+cX1vWzZ1iRjg~|40}obIonZR%&{YILWyD` z^sB#a5Uz~DH@G{q9(-21%zHY^5t|it!pPup9iSX+Jq?n#g?469y*dICrb?>BD~;GH zJh#%noE{@;x9a)SsDmBtJE<#EYc=+-moMwj^oHzDypi!)Q}+b6eyHCj|Lb4pmp-_b zWIoj3!tFo=)_PvuOCNtzpHP?8A^6p!i8o@4%DLr!D!lU-pIvP8ARmDiRKFepJ&*;J z{Z%2lfsSws5AxhtH-T7RpuEN<XqbF@k~lT%Dsg{1!t{*LQj%ms!8ciN6_mgai|mU~ zRpsE%#Rt4)0snqt<VbP6k3x%|93hW-9g{ub`{_j!3tf6Jx1;f_(+HFrqYKS(tnu15 z<Al$hN%rWGY|}SEimUG+d>rvvd!NG4p(PxI_p(Sn;3xa{v+7O-0nV+zI&e5q7V!H4 zX=)Ko_3r{g)_M%dvm3@7y5nL@`WKWRBFkgC8YU~|Wk48SPhx1wGdM)K&s}WFT<DQR zi@g#mZ3n<an*FF-FWlQj1Y-bmy<+40uga&LR!B?MvqVIn36p2nldK-aQI~wiwD+N% zFn8|gj$ekXUH$wYMW}I)_-p>;_=Erpbl@B8xw;EitA6Q2W^wBq#=ZUvrCcz+TK~|g zDRZ-iykP9fdnwG(n5$obHZYbR3&Qk5BS(7WfD0@$&wmAQnVH0|hq#4unG1A+?yf2g zH5#{7U6TnCaL-a1h~Bk7?BKMM%{ADHCtjkpL!YO(XR0%t=q6RI4Jf7X22~Ya27<TT zkYtTBIVI?vT6qd^eU*t8H%EI#_tlj6_XbWq2{li)6q<VqWNhCxy6i+A>esuKpwrW| zpK1RSqepN*(yvI!|4?L1*DHg!zg8<o5)4JW$l<&_+I<7Sm9U;YpEhQ8E^~~vK$aYt z^2C@HHFlrs?pzkcT>zV-jia|G|A{8*$6ng|`f--S#8Q>~XaCC1<uhWhCSWb&OKrI4 z(3i|*+IgxW8)tqLK}H!_m2;d;BJ<J4K7y~vR8^le)px|3A%@6rT%ns7)2M@6&+UG( z5jauZZ9E}uwk<B<lQ5@H*d$+sjW|ehO=67Qyj5hUjP5~tw-Tn|TEr3tJ!x$rV!>Wu zZ-p{4&g|xg378R-e|y5VzG3s`wR+xIdeo`6z6@53WE87jJTGCKceBibaEOd%6RP(O zWt<^eUKz4w5vm^{1R0cd5YJ@s5`>Z^zK<_%`Z<%YQ;?#*-}nIN`0~QecsWpwmJS*y zvHB2pm_sLEiz+LYF{kV-rWL3d(zh>_IW}bLE@w?z=Uw+%{wUAwIvVhGOKAr=3I}!a zf<nZL1Z@-FcL@XrytrZ(AVXaipAFb5!Y~rqS0VAAxOu(p?p{3@yf}Xt;$${)Eo7U@ zL~bg;E~QMP32N8boOQVbe5e0bltwWA7R|gDDE?$__bw#lysh?LWDM>*Uc`2vbhym7 zZM)ZG5>bp#u3MLP8^}3!m-StoKK#|tYxNYTX`iwHwa0ctrl8AK2)V41)boET%2N8s z6K^7W1P9f8oN4w?IdG>QimNpX%N5+GUlPw2FeeN7egRVh$P#O@;UXT75_qT71m(^* zCXnw0A$J;%7K}<tat&wSFljtsvu`@J`5N&jH^tf<b*#jl&*$*wUSkFP+P8G7m$lh^ z_u9!J)BJl4vvd$K6YGE_mqM(hfT`DsWiV&9X?VPddVOV*2G>aN#CT)90j1+I1}$va z+k7S;+N3jE+6#(Ts2p)yRr*UBlebW(2KYqK?J$-8?xb5QZK9VunS{TDBSDbkitf^? zpB;vD7yr?vdP-7eV)J1*uLhBWDMyE=Z#K5>(^>?=`d2Le!hk-VKD@Kv&g&^I2*68v zv{m;KyjximUZ)>JPSo;~#sv1JQ`!E7>;SxYpF7^tsS<h>`pkYenaeFEC)TQAeAMGi zif;DWuQCd1?>S;gJ(m8$E5&k=2c|`&v=!hfAae!H`gzVQmkrzs{;F5P2ju0ZX3}o$ zOoQSr%-mqA|H~2szLP0PTCMXBHN?dl5O=h*c;cN^OnDV!dJdzi##j%&+F*rucsdhQ zvMrQ!S=qTuZm@UC41Iqj&5FngoPtBQWn5>d)5<~l;LD-P2g(!gXu|n?%c;G1K81P} z!xuop%Nf<w_NYLVeZMaM)#`Dh7A-@5?75(k*~A-D564479Lp8mI&Ch~c+Uk<zgzV= zW^Nh+xDpsOqq_;}j4o`rLFi&nf#PsQLvh-}+SiX*Z`_*b`pt=bbyUkz9H23jRqf7E zF;7_-5InxKnZk-Z8m@aXr|3Gpr`$62uoO%EB_U=Ymws`0x!f2vk_MF|o#Z?0Q?p~= zoE^lbreaE^COF|~MU%C#WU{ODaG|Za59+|)p(H*O;d(obF8<R$rhCP`j+gT}Evzjb z`S;i1i(>cp)Ks-l5yhEpu2vEExH9!&&OT<wCzaHt2JmdGdDZN0ug`U=Z^^J{;fTSk zs$jRf5h$kDvZ=Kh!!siF#%}^$SjJxFA2$5FFor>l&RA@e7@4JY=(^*yH?o(0Gj?x0 z^TV4P(o|cFtUH&q61C8$mvGvH0!G^fPG?}89;#tF&bvv|HS&#M2N6VK{hd~UwtVDc zY*C;0Qjews1wEUjR4)ayn8q%vbXzP&-k5ORsD8tbG~Zwd%RHuN?4Cp<o@LjWd_lhi zZkuUVQD6F~lxcQ86K<j`zo3*FW^M7ukQG)p!PtPs@14k8bi1L#pLuUZ#Po1}o3#We zpaeG!RfWG={(h)p`zWgbD_;i`)NFr{@2^S(2M^+2-b_y#8?pw<;(FMk1CGr>gXSU0 z;x*mxdJMCd*k>fFd#Sz%T)xRf{g5=l^z|pJS(L)L3jnz0M)frh7kCHx{FFW547vH- zL0EPTsyFSe^pSOjz|*|GdzVu+u3cgcxf|r`)*0VW6GFz>`l-rK4&srjf$Mngln(d2 zV|RIi5V#32muS$>ZVE%I5HTmpl${C%Jkd$dRnN{%G1!SUKwdX>+-$uzrMB{JS>_4N zb;6kcdAH=rlt5THQrUto{!7-|H-PJ0cf<O0U=@&K!(KD;EWIi=(p%Nsk#05-{q+H_ z2lNW-{WYu`)mtB=XeaRp?_TrKM-h|I32$Flyh0uDuPuIm3;T6m`x6rc^GZ(VFRy+H z0N>YvGYIOP{+!aY9CRpXyWgm1x26+~`iw2X@AEFt|BM}`|1g<S*;zlnG@G>o+>+;S z|KL#F29}*q{2(st_?~B{sk2grpWU2WPd^si`D1}!N+-|RN;Yx9!}$Zl;?09GXA<xO z{8FzBRy=r@D|jxIXCLcPyOg}N)5-=RZ`XSUEQ$!4sU8;SCH+M<y7}V3pCVNk6nYx< zrqJ2*BmfN{!|@i{9G)gDks@SQ$g(e$ivrMQ%x3;*TP9eHe1p(vZUjQ2(c`gvqlc;Y z1He4u)3)i?+Q#&I23<W9@gFZh@}n_>W5*(}kA<mQejh)^(W=CCtHIVl4N2k#dAnYZ z#YX;@Qw*R6Su>OFcsCnTr=rgJUt*DqALfzZY`(jpBhVekw`w)CQ#PEV5YtN*z0Hk( z82^AqRT*B&r?p%@ycy$&NzxwF3fBkiI+qr4D$%+PIMO~Ho$P5n;1kjme0`Zek|a^G z4<Rp}*z)Y$6&!tjfz^HIocGiC!7Kaz4yMl^n`YBp)8E9Gbr?9L(ChYTq9D}pS5Lz+ z-@ecvziP19k-~BqzDb^}ZkCM>D)=q62!1fu-PSN?+xgvyt-4ckjH*1B?tUQ(k9?!E zZMIX-VV+)idp#)>pQ<cm(3dhTFBmum9`1Sk{Q<vZME<Tqox=$8CGw85;OEzZ=-i;U zZ=1g+am#^*%l(#iWG1kHgdAYcVZ{O(EkwE~e&N)a_T*VzsSVL?4;jw&b#6%uLFCUm zcL#k6J|Z%)w(b$pu!l(=&bW$KXZx`?Lv(h35)EO3a)c+z%rj@)#g6FH9|zp$%xBvG z448%`{26r6TIigh>xLV>>hgw@D+>VeNq*z+5-bGyIqUs=dyqI(5X59<{4_l7i>T3k zdjHJ=eMnF$y1I&DG0H0$xl&Fy6GtAXw^3I^%DwFNunrnbISh-s0pk8g5CE?M;d-@q zKWYW8eP{mx-+Y(K$)rkXxsTb~Ft+HU3Y}361bzPw=PAI%=y$1qy5y2@u5j9y#0_C< zPKyl{*zquq&^<I7kER(<I9Fo{_niKS&k;h(`zf<uC)(U6x{-4UDuOr}PtqPc-K&b( zJ9eGMf8x5I96L@r&Y3Ctsd$wvxgpLfyb!w;AXneM-EFSLWU+Jr9@cuga#-J6OH%nr zzOwNKff2R^HvyV^naWxD-B<j<o`!d;f+3%2;|Cu@YRXO2fTB2kpT^&w^!Jc$M@Jq} zhaGAZuZiHZ(;mQ_k^hfty8E_CBWW!q>9NAZxY0HLyN=UZoMI-AT7n|5fp<I017+a~ zoqi{~9YsQG@)n+EKPdB>K0!N;7V1(Hq9Bc;CyDy?K|GQ$Y3`{U>b>tFd@K!v9{=Vb zOGyZ^4RzzjG=Uq|JjjGrpx0RFdX}d8OS1s^bAlH!o1x9pvo{b1)(yh|VWa8-t(#GF z@_x=T{sCV$o~M@*)fa3jc`}gIKfBP$G`lO+!DnE7GS8Lrmu}2dw?kd366!j`1&d4> z0dv3d>W+{gnIFH06Z`oqhuX`jo4;w~?MWTaF8uLNXL*?<{gPD+^u%*>#7+JXK01av zy1wss3%<0E%qUortjt;y3_&yi(4U0NaOtzaWqrb4sN436Pse&Ikc4v>`tKy1X|_U$ zIGcurI^P=Ig!W>{4Y0WKRtmrpW+?{;E(t$pf*Ge%NiY=4`Fua+njzMf_gyD$8S|U* za-A7B0m`1$55VOx(Sf-jb{6!=fw8&jV1sG~-Toi4&_QJg{cVG74c6Rq?U#MZK=L2w z25haL{WU@Bs^~2lu<Z2}zZq<}x7-rXQ+kmq;0jlxu9e4pQmC4qP-gBr3Vc;`ho5ev zlB-Z3=t**Q5Ip}j7>J+r!AsC{ry!pDC${)ke|;|}*1ccBhZliTvLyr7>?(CtpoFuR zmUcnrF=ce7e?ItzOv=!2<l^Yd*-Wl1)h2|<XLE8KNDT)&Bs!$X&uW%wV_V`#%tFF~ zo-TTK2yawO(g(+$nX1!dGrqnaK(hQ97!wdt8>RS;Fp6nCtf0<CqqU3Li~Gpt@9dFd zt9yBZur=t&dYkl_1wolsVBu^u`hts8DPYuU-#h8Mc9VuE+!ib9pzQj#GRx9Y#Z~rr z?<BU6UMP_{Uw}dYqSu-qH>C24;g}~a9k=gaYVV@{bXkcedvAA;3r@WE%`8f~P{WuY z;O%(x!99F2GdIqseqjahT)ZRIuwdA^v;3mW##^Sq#4K?0x5X6ataoD-jacKyD~<c@ ztYDDW+Pz<Xd>IImq{WfqkoB?Z6}2%OFOl27I&w2rE^+k6sve|-Vn$M&g1d(yUTEOc zvielaPGvT>e_@g$i6Fj(=;iNWo473nD!cWMv)$^sIJ#%Ma8Q(kwMw>w+0kc>0_2Bw zR;Pn}wNq!s_3t^|ZCo~%BNV-vy${~A(Q!>+kGPhm@{e9j-6L1$L*5w188?wI=Pf*; zQ&Zj0nSFr~2VCR&+w$`_O%|GR0W8jm-MrHIslU!l@9U_zs{di`*@xZ$9c48(degVe zErrJKz#!Sttu>?TJrg&xfhmdcf<?%8H;#T?C@Rp6X?kd0`m4fWVkquA!u)W+$>{0K zc9th52evL)a-$k{>a#QyeDuY8=&|3$x;Vx<fQ+0hf>16$IW9R1oPh23nYb2El^T>A zV7f^kKx3)rTC7)xT%>`&+D7St`WQ729SD2(mOuf8KQtvb79*WVk#-Q5GK99TD+quS z<P8)%%Y6^ci>|=&OV#F-gJn-=CTv~Pr`i4C6Za)<0L)SAZC^C;|NRo5)0C@wf@+P- z`d&^tExlZ~5YR^pbjAv_CoKeqIJ%^(4Cq9<tg*kM?{uNMDIqv2<YBFHQH^ap$G&U# z=%smx5H(MZQKlK$(ztR#Na=XVU6!<O6MT#c;1H~xfP6}a6Izefhd{)9>jUvv4Xi=d zDt6MEw|)fiDM<~H^@aS2)e>G1-iDVuMOT^sNNAkjs=;X?%l$QiBVHm4&&}Tm)T7qn z%+9BWJT}_T{HR)I62DOd{%+<%`rMusW`Mf=^pa|Xo~%vP<^v9KSAD6_U-*DO^_(>g zBTw2Hj=y|Zyi{S3)Qi6PEjcJVrU|%L6Q?m(N-Qf#{vm$XW_50Jzs~D!kUn$HhBVcv z7xn58_-kd6Iw|E#!)+X}ESV)JWy0`qc~Vt($i|{of-JjUbbTml2wE%=1bN}0{2tbI zi)zf<-u<)+wlYirFXjWI#qbH8VFToe@akv0)sD=TVzKKrgiUl~@{!~iKU)lMQfSFw ze1hc3cca3Tz{*j7EW9rVa@cd$IP3x-@GL3+B3uP`+yQEkSBO=dxB%qDDll2s^$caN zRP4<{j;(d=q1~z@DQ_>`JJVlBT}^$bntPA?a%6Z<d=Jwz`?~@kGYbW^J4Dm{r*f<= zEiR1cQN0al-#$+>(f!j5B)36ke%pB(k)=|#vM-uLQ<A28`zv^Hk+M(iKOj5>PZ0XY zV7s6C)9*8Z`u%8{=YHeD=Wc7nMd80OU(YFvlupW7!XP^FR{J&Z2~6vP{1v4vCF`a{ zfz;X|5rT{>{Q8Rhs7|?+?^aKDRwh)Xnf$({Rq*DKph6k89eSYY2mHBS(~${@+2)Ji zT3mhFSv~5^>5@F)bcrH64j$NjFN!p6JnHi_etb}BGydh|k2c$_^qGy&Tu_+x8;u*) zHXJ4D;=+pz_uU5iIKAd?ctFSbOU_+oAzfxKCmc_5;KeLFeD_u4@7Yw_sDMSP4@PcE zS5NBRn38FvDN_M0QVB&a&pXNfYxOPd<orw>j|-gRd%F-C|G+YgYPX&aGcp_l@a{q+ zd4LSo&8V+dw6t-RbG|*5*pU1*X2YNf+oqV4s~Rkr80yqv53GCeGh^6n_>xGlugg)^ zlqXAHcLQ;z9<z82azig@-JnN{*Zmt|4%IkSG3g^uPs%_nGWQ7?3(3(m4Ao4Uj_wv9 z7~U54W#txT-?~fWtZcfxyVAvNAZ2R>4P1oAXmaGE25Ms@Jc}RxsYp?<Q+Jz@pKt0N zH`pKh_#GVLR`&bv(E6WGYn3{Xf(^`00+^?l{Bh^l3<L+)?=;LV#PrgJ`?DRml<pe7 zYb}VXR)21=<+`(L$jf@2#*()8H<|q?LZ!!K3<(MaOoOUqc909xour0Wpf^n_M6(^K zyZZi#>eneCaOSI5{7nmLC%POgYezf)b163b>ZCuAlRxTycH@(}Oe)m;c~~(i0=|Q! zvn=W}*ubQ0y%0qDiJuPNtg<M~eq`}9!*njk4cX#<Rc-!Vhii|g)5b2EUS`l=c}9&i z;p6b)5&M<B**efYgEmc2IzWXGsJPRTk(n0B^R}g^#>L%r_QN$}i+vn#Fy>qZ`1=|l ze$3_oPR^8MJN0kC?>mSTk%%)@g-wF5r}(lq<}~v+yvk$S7z5<N?HSjB!23c1@JQA2 zJtvmgYuqHBC?be<;T^Fcu*5LVWPj<jbETc)yz2+sQNUvPB2XyXOG<m?tVrTf8~+8u z;w#0rb9)ltG3Upr)XQD}z}PfT(b>wil)-%iXs{|y7~<B}vAZnb&e9G=qw9U|(nUUK zz|~i(FrSnH2Z%nfV4oKT@*+4UvVp4YTptwf+;fSc;fMNfW!BxG@D=$w+6P6L-osYx zxXBfpAxkse(}xZ>?_H~SkDl~##l4ttePS=orrSe)znvxHs+v`ijKA~D6N{{OkkLtU zpE0KHw7^&WaSPVj8@qXxXdMKM-xFW03I%mO=h3y;&)sqkXO8+cb&m}alSCI92$bAG zpi7YwIYKsC1o05MT1DD-I19kkr`7&t%-0O(=w3RUgiwURdL(?j7DI()*wbUC7+{&9 zV>!*Lh@ZMMACIA%l5-l{+Pzdgw*X9UH+F)cV#`L@8yCbn&i)a9oz-=8iPwz0tD|<? zuc4%Ox@&n$90PbGpkuenrLvS}<P3h)TXXY4Stz+-9!)$(@j(~c=!9}Mtc7mkMV}>^ z6sB$g+xkvESPM120>}&Lp&ePLn~tLVm3{J4pPDEeO_FRAFD5JxZ86WY#NYM`TO6zn zzQ4M?L{7u&Q*oD^lZEvwu6!D6q8t{U{-|9}^Tzd2g5Ky)t!uD~4>81xnzz;DqcZ;e zERc;ucMtw^WmE@P*g$L~M3U9OtmqCQ0x?ImF+Oc^K3@L!b*1ZQ?7hTcw(2vJZAoS~ zv<=`~&<~F(@Pu5_+47w{?)P5jwKkJHKA`*Zu=Xq(p3mnu{6!Pql$hy!+fj#LJ%X%| zYF>d|b##hAQ>UuTF5H-c9tfKa(EBvg*b=_O+|a2Go{GWZ1Mm`3^dh<7^G`?4rbE*r z|LwA4t9FZa&&y4Yl*-AtsVp1S=Y2<L0%y}hTp|T|q!k1pw<?Qcwf>92`-v`)3V7v8 z?8n8Fbx*aX0M5kk8cf5|j%u|AK^=1Z@wVQj&U-egg-^JZ9;nDZpu%=)tQ=&#1<JEW z3eJ6}8BIl|%EiA6Nk1h_``ra&?tOjBvU-<M%%`-Srhg|?C2~^Y$mQH<VreY#6-K@h zkn-jARD`a2LVh~xsG&b>agt2K5_i6cdOGaOVdX&Tc9<%V@0zQ`b|p*$Kj^Q+a@ps( zcZaA-AiYs%G_9RO^Yn1@t?@ycfgFV5z&5->SPbdnQzKdvy^GmTn)<>?Mx6_h<G-?F z<w^Qna+<7TBTeS)V@=L5m{PEMbsSL9BLhHc0#Fjmn?Ap8&u5N8PM)}PU38IQR1h$C z6lbT3Kk}nL>HYcQ*lqm+D}3-AvczZy%`-S9pk7(kw19s&kgS@REz#r@QMl4wC8uJ? z;Su-*C<|S&bxFc3Eh$Xw<{x~;`ZL=d<lkIeAjilZbE%Gn#jQEx&D*rs%KVmaUz{j` zcb7YfupiZqC8Ar)N67a5T;fL|ZaN^zvIW=@W%>g=YkgMiD?FSLvY^FrON%{YKK1rK z*&65O!@ZVYW~&D0QmCjDquv7U8IYn`{0*inUhL<9Tz1F4B|Y1H(K#D7P<If0Z3BDa zCtg6E&%Zp+o;~K9S~Ijr9B12t8nX9=yjCVW6hyF^g~&~*(v&C()wxLafj*g?8BB|e zX?mNE65`dUReWq6K%?Ty>3vhFyE5)w@WG4{G-A;M?nSp(MNH~yNX-ykv9jyyYJY;A z9@2p835M1zoZW^L2`%{bsC^7*bqxxfVG+_evVLU`=<>pUb$K0q*~2v|-&8(@Ka?o% zNisy@mT|FD1k3CKB52cgX%0Z+s$jiCIPqUEcwdT1ibrXS_O(A%l-1cO{7JG5jfd3C zhIt<hY8L_5%Jm|qO$EVpONCo+m3we2*w{Xsr|PP_2upfxn|}@IG7~Z2f9DI6o2d;z zdq*ZHaYR#-o*3&r&F4Zf?c~yXIVyz@9d12%Gy(jv=gse2YL+eeQ@+J0AmZPDr6T5o z?%}k+jwQ=%bXXAP4?pho+25_+>f(!(KS)!7-QMtS8R9W|Zv%n0x+M0ZKY3H7NX7Bx zbAZS{YPot|iAJ?ibmjLnU<J6YQ9@c6@kAY2r^hIBiYw~2KcUT7@-@9ls?k22?d4l` z7oqgT^o>m$|M-4wXJ4}ME1wVPE|V%SkLTXPAtL`<{TlUTIsnB>+_vfY0C)qA`#_or ztu>ZePUOu{>rSB;3%zrzoV25R!X;uZD7!~NTG^5XMc~<kU*g$U%>v@Wv3$PCMsHo! zFS5q5>1T619m?_v`G_2ze`0*TnCka1xTHJJ=*L3REk@(hl8#e<>v2xz>^9`eZ;(4O zaJiv)#<<ANlIEuCjxh5;;g#<{J0zf8Vx5Lr8aCf$dy8T&FucXiVka|l*K;txuT2VR zjdH$9Sf;g`cl_kUs%~QFHDt}nkkKI-YC!vGr?45B+V_+FaRi!CH7=}LuJaDHU~3g- z?~6U9axYx;>{Tk&CVE*JLHC$-9s3o0{82tf^}Nzweaykf%RLo=47iQt|Bn<3V;WFh zlPgiIdd^aTTuJ6)3%Hj1Mu~dPWkYPH;z#;keN}#@Wpx8orO~r;Q>OOEo>I8B$9l(p zP!^&Oi)jpO+okhrT@nP9`_UI)!;VpQoG)OR1ak~ngS2L^Nt)8%pP%I~tiG4RpnGvd zm}$;C-lYh2GUE86L+sa0dqhaS)5Xw6VL#6Kdx96#t<K>L#G#v_r<}1(x*WZ4_*MIV z>4CmlNc=yh-aDMl!0Z1Hiki{ZYK`d7VOH&tglcJvYRwu&Y3-fX7F5xpqFQ@)p!N*4 zW2+T=1PLXgR$?dS@9yXGJkR&{pFdoe>&_+TKIeVj=k-q1%Cerk&uuf7ZxZ;Y<XX_` z`NS2-=+niRE^6|`C;v`WBzVf<Aa*}cYBhFuuc{9?gX#=A`C}%1-KW9@iJBgESy#WN zf5`L#PW)oXD)qbB1=R^H77WDD5wjE<oYwcCyVpZwI}6;yxc)WJ>I*-P0(UgSx0m$k z3+PVMv<WAP-eU(3C324^NxE};v&`^cSuNyn;q9!^Ow#<kA)q+lZGOTzX@7AcTF}67 zT@}+uf4dPd6!%<=68whS>J~4Nm;#cYryT_Cb67isA)C9WE^DuzGE>?XT5CQ3pV_$l z#^30u|9<VpN*QwiknWpdhNbsF%bb6I_YI2fwwZ>lR2`ZFd}b1e*j2B^kJ=@Az)WeQ z^q|`&EZDmu<?X<4%{Cpa>L(8CD|O4sX6}7UOc)n_^@Mh)FMKNyzs$yhr~(aSs5sbS z*n-WYyX<TUM7YEWFeIBJ_lGkRuE93m5v>5?eh?pY*6!sZW0SnlFZj37CK+JYFT=cB zwWxtg(+;>J<?*vUGuyd5p^1_q@1Q$L!0DrcP-T~pq>Zr@QUa}|0IcnrR3mqV4eT)G zF!ySS6GZDJ2!P5($2w&He=pH|#sE;)V^E@NMp)FK_F||rul<Lr@mVwS*XNVkY`0S0 zZ>%;i{}d82JhhxJ^tMXSeOFkk>Neiwz-7OMbJOM($G1y>&RtO`M#?*^R|xwo;Cvi` zKczPB!1fEMrUslpcF+D=tco86Nsh9(tT|2Rbw>`B7#cIPP~6&I=fH7=9s^(B*#2}D zU)AH1*%X3~cf&6n>Q%)o=+Rd{=ICcA^**V>e8E$^fQ=P!ndqY&$+r-;Rv&+8?$u0Z zrE@%Y&|pO=b0E*x+@vE>uPYXM(7wiB>--;9zd$~tqxGGGc1^3DxJh4$)pWI$ghSN` zhoV1KSSw2QQ9~QsyjCO0q0yzM3HxL>`IZ$8dCfM^--+|-g%`<uXVvil%Yd~h$qCfO zD_;Bp`vVw0XXkZCA3N2F4BT3|%J#@!<0=fv-9j|JauLPMbjd<1|E;}<n7d-vY(!%X z`Cimp{Q0*x8P>gNbes-Hk;j*~Ah^5Y1Dg|!)EJ~1Y>7oZpd;JwHx(vcw;l3hUwkj! z^aM=@>u(uP-klCP;|VWc+;}A}i^NWZ>*6N{y-WC7`U)q#gUAz(Sn&|;UEA}1rWOZk zHL3w-Z+SzYLFB)eZydZ9>jJQd50zI6X(0>z#<95!%IqTFEm)%68>cK^S&=`O3isbm zF(`HnIs1h8X!%VB9B|+FFvBQe-W}&1zF2rPj6Q$&55Rp$%RKR)3|J{z!6N(<+~B5p zd?kg*rSFe_Q={OA&y<g;^A4>VV^4i1v(?r=O?^3V-JT3w_}G0>ez8nFb7p2wG||$4 zQ+4&`sFpP#&Gu{(d&sotPWvAz^3{7nia&Xh_4>1;)QppIRTO&T|3d`eNu8hj@A<9# z^DrxrH#ke2Rn|VN7)YPt{vkTaV!mq>yTEm+g|R6R=OBK~@a=8pE;10ckkN{u&akcy zO+Ym)w{ILUi0u7FWm+6Bry~N_^Tyvnj_bUaWQEO&*ezUE7}W!p>lIo%6|MK1H?Lhg zYObI3nfB`3G!5A?CBpV<u=Sa0rA_uBOD83S{g4-lMaOk&&!>vI!RtHfeCpD6iyc$J zTd~-=^y)Hz#)G$Md@f?*vFe2QA<vh)$X-&<F#+HFQ&Y|8j-WXeo~{(wQDv?ijAB&C zEWgh?+lT2K+%fN9LsnDuDj3>P4l|$UL$*@rM6?%Y9A*D+ZC!fNOoUCu0X!-t8Z=Fh zF!$nelNP7moh@n)MSb;H3@zQeDcA7<h@Ii#vH<Zg4@hi|txAAzTi7)Nfls#j!a2Jp zSqoW89i@V8q*>l(tkRp0WgGNL0mE3NQr&5g^n}*y0l$;i*brwc$^485SVS=M^|5ZX z`7QJe{aN9h8oIaCPpL;Ox`+Q}=2Y1vouu)f#Y{=2fg5R2*bP<eXwidOHS$JHRHK@I zY{s=%!CQ~+dg{O}tH<tg$)^|AnBdCzg4jbwLv8zf(L2zJb&I31_iQDSwEZwNhV|5F z$Hv4N6-4UYKb-Tyo6${XrCLraAA?X>QVrYX#WB^x=wF=$#&;_eqP~89wLY>dZ8Z3_ z0}@qTIX#pQo9R}~P*X!EwphIme`=mFtKi)3wt{Hft86PBgY3$_WMSCPQW|h=+0!e% zK@9FDqN{L7fJEYDZs#Hsw)d;SNm`Mp-*)zCK=_eRADvrMC;YOI%fW4H_Fxxur^WnM z!_t>tNo79^b1Z39sXAlW#W_0U=x^F#xt`0avH-rKa}jH3jsD;;s~E5mT=$|ECp$ti z3Mm~-SCuv#Kiz#`NKlj^^*6b$tA;au9Ey;)^pIrU(t(+;JkukL-Mt}cvAK(eS`~mZ zDkv4A9viUBw?-OWEXKv7Gi$QkZvgAIer$aPvOll~sXyfqmdX_B`VNO)J%6l?9j{-M zCr;R*&^k*+&n23hPOIM;dZC^Jz#yK#TT;Kt2@<XsK~-l8Q@l>){Hgk$KWOsyLBYgo zSW=hCDc5;lj{{eT;E&CHb~}W?Ea~*A-xrVHvd3Qj6X>p5UdOl814^1kQk8A%*n`sE zEG?1(wp-|u59-TG!sdkqypN!3)c0{=QD|yKi3wXljy7?x*gs``JHufeIGC>3NKIC~ zMGmkt?LWGAuoS^I>3?ktCgnycrkN3TQdB?b$f|T9yHg34$J>P!_8AyHi&evoh;Sst zICkaTO<*(z>XVknNJhMEwj~Fq%2|lzqT(#}p9PMHVF1g8ck=HkWBnfjjRyX)0wHK? z*I88j!2YWV=imNa-Ge+3Efv3=Q*PV-esm#lqE*-=WBUX6dJ}cxAYsNg(9@r4w=4DH z!?nttQnls+xp~?P)h{UIe${4R(_Wyn%b{l5?vzIpMYhj+@IhCqJKWS?<pLpAx1iUx zEMm9!4VgJq4Yc6+<fG63DCbULjC@S#f$okaQgc(M+1m~%pl5<hwHpf7J5MLj%t6t} zOCn`&;0-?Wl#JZvo8tmiyC$y0jU>7!%4$)thW&Aj&U;aP!TYJ8$REdw=Cp{-CX`aC zG6!9k@J=U0tYEz%+>9?zQ)d@I>N2U9gnjUxd1XrLP;7i$3}k)*BD-SW@zl#<JqNzC z&{;b0^(nXNnbni}xH4XS`2OJ}(JjQU#aST)zm;JcLk;)svsC~vYEixo2Q$PLo{(Gc z{+K#wL;dcWLqf2RTo5xirj`%3=+7WhR@i3Arqr7%;L%W^w!Q1|<)nUYf0Gtcm*@I& zMUObRS3Pw;SUdmvUNk)A1n)1l)`F1s?iwjDZ6x_55jizSeeoUUe{uObr)K*sFO(}_ zI+Lk$2Olj;>b|9Izkf_uQ-k^VKZDJ`K1;!d?Kx<nE6l`wnVsv-(ZHO^;kq)nK;Egg z7u6t6bm^Y~J*L~2f@RCMRi3PsK_|1X1-S|l5d7vidA2r5`5?ivYSBdtk&FAW*jXss zVc|!8GF@aDi070T*WqJ^XQwH@Xv+elc|CM4YM6`MUAL@OA{jyZn~C+6>fukg1dLws zyS<o3&uS)<l&PK1@Hj@+!*Ae9BCVf(v@0L|qe;|b^OZS$=O_O{A_E6(WY*|IG5UIR zi-fljrEn6BRxY=<@M|CD{$DS^X5mOMD`lq}19%!rEGiloWl=)@y<CNVpTf;=y6b>z zJetYENrfiLwM<|g{{>O{q6aaSgc}1M&WDcZm6X*}eP*OG-tR@f+=8X-@|a+sIpxi2 z;5mOntF>2Kq<Q@w2_&vW!frB~d)Z93qHV3~99hc;%)6$vcXqneU1q1P+}AG&D#M95 z`*$C1oUg1A>CC+FeViJ5ruGxwFj4C67<&Q4JTMA}v-=I0<P+>PuJ!<1o%7_oQcK;) z&ap9}63)lxe}OM=^hirM5zWr~TH2G%#rKjP6VU&HJ&S6d9|e7{^StovofRO~w>>cx zFWp$~9IzvH$NLlnh@Y3{4O(UVR8EnRxWkpgEHa-!G>*EcZxnT!{f;+9tYb{9Fl9|m zVJTbj?<`sX_u^vhy>`dAYTq5w+v8~_{BHSjHQ&<bQM;nq+DgdLI`G2t4csP!1|{D@ zvGsE`mv4E<XMQFZD)jEh^l(Lle6Mxcg4u?UQ*vn^@mtZE5xX+>3dlec<ZiF3+qyPR zT9nEuRL&Rqd@NnJpH!xKz){clTi{HS%p?RLxo*j_2D;P7juWB3f?I_`ZgGg%IdX#M z_t<~jw|T*RAt7g0#D%dmy)uKTP>e$Q$EM#L$%Vc0q0a0F(0Z(41rqfe9Re1q5``)p z!6q}Y40fV#3aRT|mr)X#fH61amV477siz`j=Pxl4NmT4vuOnMk0l|Lu0-x8&$fc(a zPYv;2`!n&GcQp^^cQ{pz@;!7y{MBvDhyrC9?>vfbAx0kXBmvDi5YZPo;;8svRX>S$ ziB4ootm;gX1S`ZLOp4{9_VXx@OCoN(Z+b$Trrpmt#{H4`u9+j*kx04n+bFO}J}Sqi zSKAeUc~x$BkoOwN3dE#ycmpJfvB@0B*!cYWnG3~><7zJ^(|6iSo3`%TWZ@1xm^V;R z;!_nHt!d4?V5vYgR{_OPDH6x*<iA_YG#{K3(c^g0qnkr<df=8X?IVHuvHaxtAQ1{I zc9^bj`M>e^Yw>_rTO#`8o#EHg0*(DAPHImim?)bjeOdw&n}1vy&G57-tfgD@c*FWD z;?j4N`*N1XPgDa|hzP4D*pUuq$Vg%{80iTh^4v^TP}Ug~VybKW=Mw6!Byrm})1=Yo zFX0QBz+WND(h}y-Q~$dy(+FmWPTp%rc&yNeThV}X!Fg|n^}t?Vvd*GDS?~!E>9I7k znQEV^d)W1Ua_>0ARDvoX+JQI{s2lg6r~N<Z%d@ao{Q&mm-?D*ygaNh~dUxffoUbl# zT0f>|k`;OOLFsEja~pfGzc%NU&w^)vl-@pn{=h%~l}oJ?^Ky~GKo<K%OF*n;s`Mc1 z7ye;qk3Q=|5oOfZ6e+;=Yt<lsi<FDgZ4THJX;2d+(4V_<*fGDjSkR<c<4$=WC8+RC z?B!YYOSAhKma3-8D2$8QSFnvwfy%{@$nfcULdw~4Z)>+gbH=8WWAFM4r+VE!DxA{) z<DkQS-`WD_xWa*+ufZ5Jr<Ew4g>@(uiIuDQu=rFh$B-Nk<$=ucSOfc_`7N-DS@UD% z9L9|DMO}SCv(3UZuwl9RxTGE*+fD3?--W4$2N6Gy%xeO~&g<}pupZAJ={^;)IU3@c zZ1P+5mqa@D?MztgirQtxSNP%oKBY~hYTpvc?!^j&3}RN&D~6Z{ckyH$OYIMtk{cdR z!(CJYf`yaW0NSp0;|h?Wq!;Yq8tjn=*m3w5lodWB7J##hgN%kPustFEn}1~@aE$`T zUf_+txb?Iyp-B>vUY$*X?(aJ42MYqm8+&8I#!%7}kfsH#O;GEkdv3a8$6|}N$(Pvv zy%tZGr=<Qae089et@W;WClH9u^)B3qIw=lcm_4SvzQd0aQwj+EW)<*RNH*ZyJ>r_3 z+XmEiwhWpHf6S#1guo!LU?#fHo5w7SIy9`kAUbV(d<<4gWdzCke{elhv@aD*+!5-) zn#}ZLS_jjzCppOec2O((HLb(i_D7%HpfWuqE^JeWX317x1;q<BfnC7gtKaKNZWud^ zUXRsSbSEDhS_#^IJQ7AaltZ>V6nTv_+CLnJg(Iy^rRg?nFKq_#fs&aPOUXETr`rmE zg0nCbYD^c>cydIRzLw$lZIbbXw8tr~-LToWNa9nwE`9)zzFy&(-W>iftJ>Xc59d<y zdifbPV639MfgJ4Ndtk<Jb&AD<W~6)|U1D-jv)L`)Q#Ivo0nu!!6+UfdLhHDG{y*}g z&Rx{MDNI>mQmbI6nVvc&#93`~0<g_Hc@K}=>mwP*c-bgBEM@QfvV&E3my7tAG3B6= z+#?)4L|YwqPhdSt_u#ZzZo7KPa?R4zyl-5#iR761U6bXuDJRdV7l}_zthT@L0U`E> zq@R(|K_{yY@Ic|x(V(tZnQ;wFZy$#F`QrT@YW}G<?|H<#f${NMH?l;sN<yEC=1Z_c z9#nS=qX=brnI>yN1s~m|fe3a-+NF~^kBZzHW}qtNdT;)|mZ(}4+#AF*nO;=`^e|t+ zMZ2B=ujH}t?ysi~Aa5osHL|4k7jdZ?PqY`3mKahK1i&EDMnVtlrUGZoV8_WiJ*M!Q zZ<aLwnf_m(<p}AZXS5>1e$nfR0Vz7izHsrA`Q1(2cTP#d@iO1Xu?CV%`IepVR?=ju zU}!cWH-AZrDM&C6XVDX$pwwmcC9JH5km#{q5>(GmDZ9IeUt!+=K$T_y-tB<HVw>61 z!iw^751@8Fl=<HL|IiQtZor|bV8qQ|7r)1U$U`lC#YrfohoR)T`Vkm&)jJUAECiid z8L}c$tKk1b*yQ?Ugb0q|acs~9Y76*wU>4KnIoxoV&qN1n%kN*=Fo`w)-T*O*7vY&2 zCQcAmW%PWN4x-69#Ao-1n5@XV^p)|N=TXtk=!AmPyjAqGXDzSI7KxT5xQME=TALq) z^#gPZ_6L4r=eGQ+OdE6FhXt*h*><hCB$F#AJ{a*7wR`r2)IU81oMJp5m)?F6_Q6>N z{j$hy<OdjfQ>T*Nrq4RT(&)p*0K(@jH2?Tmb)KGk2RQ{>uP}>skit@m1%|FBOpLB3 z^<Aty9>k4rhWL0~+j1Xszr`f6_2k1Rlj%y>HG<!6@A=rc0}kaVsiD#Zh;p$pfWxdT z^f&MC4793&?wfCdp@zd>s~~%;!{4`jyWtH%q~s{5x4N5<b1<o=H`y`9ZLLm1S$hjg zD0m@84Irg|i8yI!y9Le*9?57Jn|_O*+$rxQ`dn%KZxHB;4&!q{%LSf=-jkcl5qag5 zvzbRgU=*|2X_XxlP=sKE`qiP|Y@aqaJZfHWN|d*#%h{i@T>uRmU^>nGxNBWTqo6mf zdFfYJYy#)C_F+RAoMFxOpZqa&V}N41JJ~&a-tTj#FfwLI>VVQq0~D@1$c$=ex#6@u zZtqD*y^EDw-{!t`j6j)C#{#1B^VU5u9v|qW>C?BksckDASMcj3AZ%+;N4k7olP{w# z_qaAIIKd2{JSuoj0;YTK_VLCl;2M@rpnkaojMV}86UX6J-Msp53&z-rS4Y84acd~e zpGNm-RR!Iz#*sOPz&Vr!%$KHUsSr4!StUh=5#Mb7x^|dTF!MUf|AloB=(I9x$Ahkf zFx<FRpxdM`A$Z<9!QkK6v^h<JRjbXP$6inMXsjtIsSEa<C^+`0S=c%$t~$bvH;xtm zQe(X6(Kwa1mSZJYnoGK5j2<+(z0j~YjU}6M25gjVCCb=bx9qZ^s|$K)R+o{$mot~( zzAw-bnRS9Ae!SRcU~Yq!4<>eJb2~&sB}y^$=RwJ>cO?Ca?S1Iy)Ye2pxx*d?VjRzK zDGfzVNe#p3u}(+}XX-B#sGPa+m+HEWDitGYGBzkj(0}1mG5hP~YR9cZ5<CQ{U)bCx zIeDh!EHyRXJaD6(FHx~^<0Da(K#MW6T6kSuPm~n9sh8fx#w;A1X)Jqm(`pet{<SmS zv|*En$d`yU)=XHrN<NrXOnWSR5>5IsJAW{<NhZFK_Dc{0SQvsg!@uQPeSsr4cRutu zc?8<k(3tAd_wXlKn>yMJ*{kI<X~otxc^q>gf8nO_C&9g+hyvDKlQh6eE#9=<V)~`z z$aC1Ak{M)P@NMcf0V*Pf6#_YYmnP#{=f^IWd_gTf<A(KPDk`4PjxFm4dMez%>E*Ci z77DHcNORXAMY|>(kh~Dn-}1k%%YSx?)-v@|>ivS<5ic%}rv1{U(v6NibHqyp@F(tf zg^r(p8`Xgd6*Rx%u!i}1+bnqW7D5SR3+2yH=2{EmFDiY#qdXDbLE>V??+IQRSHG<j zSgeW>Eoguu3;$l4XUEb#cqjOP<libwlB6*2=@)!^vEZpTnYLX;zlY}E>eAWhY<^Lm zsmNO|)3tSKArzL71n_3$?C*tOF5`8i147w<RG;dE0@}|_@oh|q5p7a7q>C^4um{0H z%QBXHx<4gQVBo$#b1r0%O}hLzf6c7WrNgLytI#57?KQlfX3=Aq5LCFf<G0>Ml00p? z6>>EytEsC|vD~T)Jzf#=mri&$)(n@qnBIP39$odPYRBIV3`Jvq_uG*J@`&bz7A7;! zwRiSbGJee;$RY>dT_ZSe2HMJ4<f*6}8|hm7nK>Ko*HP=P-2|~9CT%+T^mjFEe$g%R z%MF6;XlNy@`nncdY?DPFpa0RI#-~7mhRa)_cQWuF3McWw$0GrOyW}+Gki}O<>h=ld zd<7*DfTFA}feZ{jCGzJ?G96u>q=*!zZlImv<<6;O&<lSJDHzQ4HKAb_Ffmoe^Ke1k zl+}3#QyE-8Eq^bKXL=+ZLB5dy-Mwtx8sq0bAKB4s)VzRTKAQ7IaRM-ce>Vd!vFb0A z(=K<Sx*i7Kv1tUfAvp(1y<U0XVp_BXKoGI^IeJ&<pRx)BZG;Ycuish4lBD*x=zAw= zpXod+Tom@gOg-EK+a!q|`tVOZw--1wNbZF~8KywoD4)-O%4I3DIU9y7+5FJx&hhr( zU!6jO4yNJp#juN<WpP}ByC3Pxmc-dObiO}7Fw7ccuqxH3{fW0`OAhIgR1`ni3%?eh zp84|8&!fUKQO+BxBG2o;*|=uE-(k6y_NX6ru=*>lkQl}+tW*#ktz7#z2f_R&wZwQl z(c!oo(XD8TFNEy_1pWe7U}WiYUtGKEl@~@dY@obvb^KuZBVXZT^To}FT$>u&8Gx;J zQ+UZ!6XeMpI?Z{s(YEhF=kGr|aZq>yvmTd=rth)>=~n2Rx*B)Wl5KjVW9>0onf~<) z%+sY$P6PoM6V+#a4WXWjrgvzw9|{?X5|llwGo3TwK;uQkKg$KR7Zlg#3kLVPbcY;A z2`RNJ{R|E4x)}APA;b4kjr->)Nq&IlAlV^LIo5{nKUV(x<8wFg!{8&a+wF_P4^@i} zr?E%V&TGVa|K4Eo^?#~Y4~>)wxuBzf1B}=XcJL()aLTFJOvH#lcMnSKTUJUTshxx& zWNz$x13r+SeT0+ay+rGyQR^Zf4%Na#JgOU>_R4@Z%_yS^CnjU1I-7`<n;$KZ#b$Je z|6H4g$K;0EAX-K6>fvnXsyUvzJKSxh3U|~HecpC-;y~8Lpo{a5nlAQ&?sL?i&Ks9N zj0ws{-Rd+LG8i5jaS0eLh_^`!@kmawSDt=rl=gsI(EFauW3SWleS%)VgX`R8^op<K zzITcSUx603s;sOYO}R`&&6N^J8yjhRM{5n^$HnMzJb7-9Rx}~b>!3#oe4tc&oUQ!b zi4cmt0+Pb2-WX8f)8RkKI@b$VR2aWr-H(_Lq_ffw#E*UZy7v`g<p-6AF~=rGk(X+o z6l9t*=F=O8<*OTKi48oalMHojNaoYtp+B3ES|ABxG5%mu)))4^QfQPNU#*@bD*ny? zB}IMWrmqUvDMAi>nDbGwQkq%G$)h_55tS*gXwvydo8E_lc8;LhDu2dvXO>4eFGM(c zs`cru#aW8hFp)gy44|;L?cX}A))igkO_#I<!bb>?l&9jh)coIL(>n^}BCmIexDCH- z9KaZ<EmEl|O0)Be-U68n)^!Qfom<wuOw#pjYgJ2q*c;}Qs+T+u-H5H0mX1LrvsM{Z z?<(S-hjk%Nd&fCwm$1R>R2Fk{@3;1xDK3Tx$!7>x4!V2dJ0TKUseeoYs$yfXd#dI% zBV5!@FJ#5K!XDvrZ~w?tKxd5hU{~jNtBOI5MyjkWVaoGxV&CmO)X8#j>{5_apU`1+ z{>+M9)xeV{f|RnC#d=i%b9R4KP?I){AqbpHqwLUDf|fpZk0(WO&cSWfzu)C&I?q^4 za3@29zVgBXtB7wRYu7d^{n35*iPC3%2VWVrHeN@KT+-+1<PRzgo2(Z(=jK88sh;Pp zhPI371J<))YZpLcsvf0N3@F>fFQA|D8@K1j%H16UIc*s3{0#P=wG%$5eE+MBQyR1E zMt%oTsbUh0CQ7;-4#;Qy4G&LE;HNlypc}p*@kdc`HI47^-mn&IO8q1crEJWnl#p7d z7N`KRt~*wny+u$2vS9jBDVTFiL-vLaEYt~v`$k(MpNdD9fDDxhT?q14%m2m`+rTBx ze)pL*usdoC`1%aCcJ@f}jh1z=n<uqbxbsD#%P=G8qO~t=p!9cD4D$M$`j#v%Ort@{ zxzNMExtW}b)1I1sz{t@GU@ZCfxBs3K4msTLJUQA+l>@2&$qMImHrB?wR`lIQ2}9$K zfKYYWIW}wR?QO&ovzW*SYwk;t-JO&nN3ER<#fGAkGUnTh@yYGyp7FgFLR^ht0K;?X zyU_II^6#DNUOWk76_IZ3w|J;q4eAn|vvZSwz}jjSS{zqp_jW;D?pI8UBSp4ej*Bl{ z+ypaHsXYTaWQcueD{D>vl2hs0jeARBxq~rZnk2nI!RBI4nyL4P7vJ1CSHe1OrE;k^ zLtZES25kS`TFapJp*D3hcBTapLY{m=XKJ|Szu22hH?{?k-wUhWDTI>GB|h~rTbil! zJ(wzc1r&Zu)BRV!MHAn!oV)r4=>L>)J9ulTTH49{uu<#K#0({PxFr|3!*V>(2xp35 zh6LTD-Ml(5H}_k%HM<qeexlf(yH#tk%8v{>e(|h8p7*by1){al*WQm}N_=%pMo6<9 zDI8eZNey7gdYoTD1#%8;`@@SKoDhpnEg=z)+sE;%7M0F5^4gC%>Os<O(q)-3dJve+ z;K0EE_yRm77)%<FLo&u1=T%x+^892q`cIi48qjfsuGZY(Ys*$la&~`zx6@Buj7|ME zLkmvWLn|`X>{<T+%4glu7X^_Jxi;F&`podeC98OZV=yi?70ngFZ4zsx{c0H3VrTXJ zd?k?*e$C#`_RI7_pZun}oB`-H?*6zC<6B$Ur|Ai6dRT}5z1fW?(f&b>MT^c)tGR`% zFr8yT0#j|02TUKmmJH)OQaHIxN~KLCEzU6qt;bD#D7x%2x;lXus<<eBt<Tbs#*0rZ z+!#_$B&Xtnt>YZzX}T32@sTJ|+OHeym0klpTuGvlDYNa<e4KsBVg0584r=RAh0d{9 zCRj|nqm(-$a|<f*O8DMnMquW_AnSaGt>vxi9HsKKakNXP3gK!U&A;GAj^PC-gqB3& z;@IusnyQgxQ^fHAtn^Z1_>Z#KA#>Je!~f1K;`jSCvCf_*gN*~sV`_q`vQE3gj`~<m z1875tLurtHYmIrMwS2HN+7a^MRe=8EqEKCK>oE-TG5EV-jzs<s?x(Z7TzRMJBRl+T z1X<_QLnw7j9h<LpxevoP&?09mH$as?zh+;$$oy8GE1eHRUk9aJ&r~$gw@PayaV2WO zqj?5E*1TLX%Rk+)bP8_rdbRFoL?`+^rm@4Lte%eSv`<+Ogf+fcuLplzlbPU0yyaey z_|yaiUHNFyH%+UJB1rBBdPC4cYoC@MZZ#;_|6N?$$$Wv;|5(ze8noszWnmMDq?D~D zmdURuBfe0YUaV0jm{oslSkS}z7d}pChGN>xpS@xBa)ipTZhtEo^M@P!twy*ss|wz( zx}B1}s`1$=1T;bahV^8;-X4BNIM~sWx_QP<rBa1PO>gZSJ9(aTn3yp)>5$1$J?*Vi zWf~e=0)a}n9{#qg7{4l=^+*zf$9LDh7aKz?_OkrbWU1}a$ib8fB6`f+CE{q0(8PE- zWcxd0ZKQoJ3L<><VH*EhK#@FiB(rdkk)mtiW|@|3zHf<!TYPTO21f$o<gfkkoO1?4 zV$`&$xQHr!j>4J(s}ZPX>R0*H!#ivuhFtg9WWK)TW*x@d;f_p|spB{`99sF(vS6+L z?2k&wo2Jr$9Hu|tC@EFTHyz!>NVaq+g!*ay{9ty=F{+U3UuWtQgZrtbazaNVg_O2* z#WzvK2WB9u>A`+ir{d{M*g{+BNrq~a9`j4&EXuzB5siOr6Sx=6lD`~q1V&QAJ%K1^ zH0*;$=FNT*uzSB`*RDb<0sJtY0L$j?0RiZKEC?QQ(%aH)Gv^)B&SZ6at;*ZBy)7j- z9%#WCgoM*KQM^Hi)xU|aVmS6ct6H7DjQXpR4nA1CKgZlUPk1Fy3H&VW*#BiQ`bh{5 z`JlNoAqwudCk6|V#^_Ax>`7EMt5)$|AR+czRz{FZsbnaR`q=ZK05_nIc=Vrok)Y4` z1GE`cMO8H9(J>ljgKP;H$&193v|g4HaE*6}_sDw9a`_F4CV^S%Y(HXOqeL4M@k-cv zS>Nx~qP%h<k9)1nsr>sCqdK?lH*qzd17i$>Pu!go)xqX)XYr+gF}GfYqJ}c1*RL3G zj-fU^wb@|h&<$^w_noQNu#`rC%@75A(Y4C$y>A*mU{-0o$|-@+p9FDzuwQ+0*c|wQ z0k(^gco~CZj{FfSt8+Zu(y5FwT~TF~au!ZGE1(^R@}KtKEjfEr3eM*kdNsd!Wee{} zv~=VAv|3>#E4{5X&+L})+9fZopy+ltmLsmB`nj8vE&NrHT%_c|3+Idrb#@1+I|BHA zgnMM8oWqA4=;zm!o=?Si4KMgtes7E{o7<b9>g}6XYC{VNuh4;!3(~^d8n+UdY4=22 zd(Ojos*`dN2hW_>M|gFPD?*!OEGlW%2)y$J{nQFOnF1);MOZ;(Bh|hn=cv8W*9fHi zW~`6jlR+d{(gk!YQ}c4DE6_pAUgQ~PFnAoBZQZ1hWz4WJ@ZgI9D5~UvmBtfU*!JD8 z`qW8=VJBPw=nwyPQAC%7S0{RJTVu*R2r{fcg{ponj)G8{w%ae>l+<1Bqszu!_H%z1 zJn*2->an%Qp*lPv3ex7yyL<U*I6z%lvpSMkqt%WVtCBS@Q>g|se*+|W0!6oncjqQ6 zp5f&p(RP}FZ`R~%hwGQZP)@6uWQNLt-ey%_GHz>iMihb@$U4vH39A22+E{IU`QyM^ zu||qTr1kKDMpL%P!Wa%hE>X}4jyh@%jAZSrK6ZXi?O=ib<>G*=?JQBy!AzB{KP&9} zfFk{ojn6vBD@o;*^DS@CZDIKC%CmGm@NnzEgm{!QA$^^}j>EX{>x0cvQ){6P6a-?k zo-nSmQh4s)mdPZ72pDlA0i{NA8VE@KUF<{nJ3m*xh4R??y4!B|;GCV{b7^CiNq&^l z4yy>*%{a9+N?ZaAs!~{8*_kl}dJSLwCZAk0C%Wfv{42T6Icq(;BL0RgSdxXx-+?yz z^dp5%H#>}SeOslpIv9WRYD<qfm2Yt^Ih?uCowmP2tuHdZ>#uqrE(psBoy!-m7jSRb z8S^}*k)nQozE&{Yx|r%A{BOJ~1qh$$>d{_WQ8k_wXbwC>w~*`TeHDQ>dr|(+mKPS6 z8;%b4z$o~!Utom&QerVp6AYD`2CS^V=M<(~E5?$^pED1iKY0;v-11($n12zQ3Bd!$ zW|V_#nP9u(CS|pgfu|oA-J5zytz;xZJJ$tMb^r5KVh-+DdB+$#SyJ7IZL^w<N_DTj zvAsH3G2A(e>da~6={VeLp3|3Jc!jqK=V+<--zp#d`It13WV1J^3FeT7x)0NU`OL%k zQ{zgpeLCGX9{7+|dzE=Y&OXZgpbZHv{S7LcQh9oB>-4ISo-`-K#an7^e3QcL8m`cC zSacxT7ZSwrAbvY^!piSgn<jiIT)J|jb+2e5`8$2r(7pU|<NuvzY-q;ntRhWSgHy}h z4L`*ilE=IeKX)Qs`5~QE(K=Hu9{CYf+gD%ylCfs{<|m>hJct)4fGTcVWU=eZi-YeB z0w{`aVLMvhDkpReCdOex8P3&u$&YR&7QZ*^+wai;n=5IsK>}^7S!RbFF$n1*jMQ4^ z*>E|6`K+BsW6nkoo#pR(Nw+I5OdVsGw=$vkS)0S4!l(I2hL=za-dfHN6D&XY82ah` z*_tp93l>S#{z)*zBs1RWaq5_t<l{4!yz!QS#QS&hhJ1{cHqw0vb~CIh5tnY>lwipn z&WEOy?H>M#z(1HsGuNz{&_#bvYY_F4tYLDyHz}6R05w(5G~ibpYwB2ka%euK6FQuP zmD}nIo=Sj$eu&T7m40kF%p<$%?f-GWy&_Lm*U`fg)K2F(x!v1sJWG)w$WnaWj`whX z+^qkQMR64jAC*UX;#B4Q!%IT`wlaAi$RXG1I%)o8^j$Nj)N?|X1XW%-@+-JLQOmf_ z&R}{^J>@Lrg84jMveHeKM{!KIPYD164dZDIop;EQbKHVy3Pl{fLoF|PxynzyIfaNw z<i#9q0UHC52<ZMD4Wkx?0Zwc48Ak{&D88eC(ROPo3L!-YaryeJas8368(bK4&UMzm z^TL)5yyH_PSik=rQBvoSA}@j#!fl9+3KN{RUWg&R3Eor^FGVE>TJJDOF!ajkY`p3l z|76Q56g`q53YBiN4PQnbyp@ZrG{za0<!%V<mdOFHIP^+ch)i?mb$Mv30iTo$m?16_ zy;G|Ns)4$*sW)+tyg??$Jt!->VbEpbG@c}5*K#8_ZAb(8^V`lzBicfM%4QtTce{*S zzx9w`-q|QrK;N+B)%bZXBy@M>D>kZ%<E#E*<W6CU#)|T1-Z0z%#xQ;SbZpZeK5z~w zMs9SB)iVKZzsS{_se2WO=X~sN_gqTf<tlPkbv`)l@_yzCe&qb9K*?pK>Ev9rwqOB$ z7mvD0=To-<N$#H@?`luG>Wnu`>QyFET69J6>^Dv^{TmG(OXxE5wh-~$c42=Q=L(IZ zEL6^T^1Z6)osne*|LP^F&&hJ0(&5Wvghe(Ef<Yd+BDzWr`e{lsmY&UwbHA@;en}ZY z3K<Fgqln_ZT65}kWIggL<-%Y*t2_dN|H7l9`gT>+3YBfLO%9uY4V2`Bg7FLf{R_T( z;d2p;AMWX&8lSs($HEP5?5*_kQPAXz#Jd_+`9y8J{|(ER_~iSsG`7wok?B{o$+>jp zbV$ZFU<G&sd>qgMneo-B++(F_zq|G2n`_ksVFjGy_lLo^A^L|eO!yBPG+V_(9LM~r zl({*8UI7*_sx&Bo4A6Qr3Eb;VQn0I*TeIQdNcr*Adm}AL;D$qUr4@0XbgxUmKFok$ zb+h+MSI~M5DUyLxDJvL%feIys8c<~V8fuuYQ^a<3=wXq7Z`i4aCwtazKt<7a9hJ=p ztD4|+Qv)i_ijg4j%B_CCLi!)=<X_KokM6nOL=LfC<9biuSF5bkt5wl1bdMfg4melK z_-#(nN9W?(BviekmWBJhMHR;aYJix?5NjcbPOjkx;<J@@=eJrH_sHBvrWqTUL2L|; zrRv_rq-k8j^hCdWxnsvX6Ed-1sJKr9jC<7<HX>*=WK1c$inMjnx9H>9&w3PgCC0Jw zlu)HY;HD(XdR_58yBSWdNI-mo&q5LR3%#C<uTSp|xp8FYneS8lQg^MDzElz-o2YHO zwScHxfZxv`%8Z|)I=f|!x27a!q#MA7mrmRH?@3EX3rvN0D`)O&rpnsUG!6SE0;?(( zXh!`N8D@4eDIcb$NjBdVP@(H(P@S`_WwNO+Pv++zESmfQj?xCwQlVS%6EWjIc0Yd6 zvReDebh8uadhGl<t5>HIpqc@AY`dp3`P0&)ZudqQ8eE@MbNe8;^&mLnpm}TWY(Yi_ z8Mm#PxDcJ*txRpSFUck`x&-31CC0=<r#V2+kz2NxL!(8w?nIvI+ISuQ@~)q{I_Hd8 z))k%9FGmJPwre_J^#)whOxOoO;Hp5-)ihP+SqTxI^i~UxzZf^^AWH)VooP>@>_#1Q zEA8IF4GxITyn;Qvo_Y7)!M}M3$wJV87);HXO9%CCRokyl>__b45{Y3fGHXTq-y^BB z^iI=ug=UbnN6XDZzZ@(!ul51gz4daGz;`?Id9cEWHGkKDDZ?pAFrGq<AvYzU9T>?v zEuAgDMnRl7?veCjKDw@(&p2WyuW%J9S+Wkv8A+`_|90j!Qx<>z+d-fUSwE}w^rgRc zY4QZM?VSFb;Jq2eyrJ=3!lgq>6Pz4gS%aPRWx7-F=NCvFwk3V8b>^Hqd$q2gl-l-0 zLH-<qkMQc~@v{b0QPt8d9bG?>1ry}c)60f-l0NEzoS(#>ZL^A)0{_PL%cL1oy2j&N z-Q=O*YZeYwy*EUlnHJ)F-$j7=2K3B|n$+sZ3*uX}p{r4wm5Tlw=G##Fm7s>8R7PwV z#+Cu`yK7hCQF#y%AN$?}Itoa{8gQ`80RaN0#5YA%gd6O%{-33z7lIiWiw&!Uz1q&q z0i>+^F)~y;fhb7&$%mV3!EU8e%>0?byOX8BW8a-R=>(UsuedcBuE^H9gDf9H+4;ow z@xb%b9YX@!LvNR!Y!B;$4Rar^4J?F`qfeKd?C3xHGYVfB;Q{ski;VqVWm&<T8FW$b zupN7MO3-KG86C*;xFzZQ*3sn_HgN7MAn)a4t2A=s!^1~%{nc0(Y~(yRW+jLiZyK5I z<2;<am@+P0bK`K~3so};p|aH|LTaRTLFb|z(u9v;4d44OYehSr2wR?oB@~xT(+{pf zpVC39gKo)M*JNN5VU;3^ofD)4Xw>ff?<P@k$kyS-(Ux;?11i639;kfe@=oo4B-ejs zL!ycGrF^Ka$ImgmC974LiO=Ti^TPc7Y=fBk8(*>*Z#c?Xcs%HUT~Un7H9_$hP>k>+ z>fA#$_e;L33<&&2C~hE*9vA^p1-UoiL;Mf=B3zyivUiS~gpRc3Gd-MD&%Y8fcr3r7 zke%3ppQzQo4zhc&FT&);J{0s9;OcRnO;7c4<@CCTIlb<3U*H`_sg)d0uc4Pn*zADt zO2&B$ZA!)|7|HzJn5Xl@u_RKwY<8K|{#UxJ2+@=Ko373!Z;Q?sacl?Zq21L=FeiD7 z85@!ZiF>_#Qz3FS_vob;8=ao?OAqAATgR84C$)_iU2$l)y~3whgKZY>!|OCTc!uaK zV&<(sH~F2!)yTKUqM=V!<Ld8|ekPZ+`P6Fx?DF7#w@+zwDcfjgpjT-ikZq4+k$ZYY z9Pc>ewlt(nJ9f?f1`vY-#i$9}VLN0|6KI}Q60<zPRMfcK5V^e$joV52@YNDu0P<D@ z#nRyKuZnk=<;x5y@cMCF9$~ok_${kzsruang_p1V^KEX!g-=g>|LSWgj%jnNGAjPK z6K3vgJ7^e;FpEObaCAp<@vV|P!?fZb!Ao>*<UxXkGVkh>xL`_JHe=`D_|IJXzBca| zCd>9+VJaaN;e>x0N0FI$Hpq$d-?L`Lyq1=ed30$PdQhAzx!1sm&!)f+B~Cd_4f8P) zB<F{B3N{2TVCJ0ijxs7zE_<|L%vXzBls+ijl?i3n;~LlwZ7m9p4NEb(H+&kPu`gZN zS0$w(S|a$YCcgH39Jhu}D;zIYIoESfi;l*BjpHpkZgZ<$OSw$HJ)h657tCbW%64%9 zZ&5j!>(3J!`xte;q=Dz^^hvd6yEFv+`C2u!VRRqo)hX0!{2gm%v~yn7z@)-meqN@S zk?a&Wz-Gm(>dE=b<V!FhgV1-4?HEl-9<X8>j}HhZFw6-ZuH59V*>3z#*(VoxVnJOO zKoauWI-;7a(e5~FYrXOU0F0r&7S^P$=?Wf|eI^_y^Y~y0V4!B@E&NS~_lh!7%$i}i zhpg59m2aw04{6$dM`aeRGm2JctdO)9k6Ecy`>8ix<5p+(ch{zBVJ}ty<rI)zA?d5< zX6Ix>ZiyXMXcJ7>+h-4)hI1nR-PsSqxYQTdvbxED-HywRflJCrJ*FYW8*@n8YK5Fo zJ1I>k2TgP1`noJ^_&{mb3X;Yg1L8!uF~reffC_Aq+5T}y;}E12y#rExDIsx}W!TEU z%1~f{_BBeFwV<&4K`vIr{&;iZjpv;2unL13I~a32<k_`KgWZq%x@-<1lEtzB*>}eD zcp?WO0k)CH(~0+kAsM4NvWG{E`96@B0~*iJqb52X$uEl@Z8#pkrj|8Uibyc(%PW7z zY44?nHMmi{@*mV1F?$?uSLV9OD?>;o^z_wjf~dg1jmC7(os7m-{%?I#dMGv!54y$p zT26#Hpf$j*WXIl8FtI9eI$HmG6h4T9Y%yDOmM7iDwE$@u(#P@JQqwgxU#inILmSQ! zdL@GA$M}XnP*CJ$?gWG3tvW*)gl%h9)=x$YEiPRApFpSe&X3bG-0K*@uq32T6)A2d z<_mX^3rC@{m6M!r<h{;sfT0)Xg7ZGcQNvuW*krVi*PIJyEWX!XD(^0e@aS&hJ<YxU zd9_J&@xqkv22@Y*v66M1Hud;P?Pi%g$q-t3kl;Z0*q*DU9blp%i3W;Dp(c836+VG- zSi8Sz^NyC#8>q%~MOmEpa?P*!j`+ne0pZxQL3gKGkt~SV1xOm3RcdOWJNA?F{7gvE zmRq^?g%5iFq}k!)vR~p5i!oU@J6a6yR@AmCH)>`&gMTXZ2dTnnn&Ma8(Zu`>O_h{~ ztc2TLAr)P<wtyHsA5C4~8#^*ck^Z54Exsfb^t^O#B0Ppg8oq1ml5}M}MQ8doHc;Y8 zsZH2ytyIk$0(ENivm}(|F_D|i$kypxYFhQ#E4Ph6UjNz=0Tr;-T?5ul8;I3u{{1qH zjj-Rjj!JsnGJm^KTgiLG(SF0=bBm8zF_DdxUg0X{7(-W`an7B*5wNgdQVcWPMGQNX z#5}XV<6?0z1X~K&SavT<Uj(y(AU$RmJf{nwSmhzVcSOApNKuS07!&Pw#&WEU3o`yq z$Z@gg_DbHNE+W%@edNSQQ!;dArXtUWmzo;mF<DpNSh292w2Na7`J%L?EUcn_(wy-y zSG+0c3k>D^Dntx)KKg5HF^}Nl5r+H#Q~s2XL2mlzjWvCcx#o`DFP5~%{oSWBY9|;< z4B~z0k8{@^vPVIXej{mXRWnyCRh`LI*g~8Qmis*;h2J$SaWsTmq#9RMirT|la0v$> zcRY#4^R2+a)C<-<06ixi1fb`fTzckwJ_L(id|t|;{``Z`bjW*{Sg4xZ_R`>G^>}$d z4j~meGbon(CHnFID2++nSriV;h9_qP-yu5*;X<@^js|9#U2cA~dakIA99!32It8j3 zo0k{$$uOQC+fE;ar7NBrMHteN${+{2bU`MisH$+8*}b%4VXBfzHm&0zgT1MKy_Pbu zHf}#IR@fw7{OeLZI?H!t%&fwm(Qg*pn=Jc1*lnkVq4h>?CJ%nRz_dI-T+P!F*kI}J zdTpXnk!x^Rsivqszbk+jXZt6^<a&K&rBFBKElAMgIQocs@4u#F!P4qYH?8+^fK@k+ z>!oYq_lFRk_4>orpC%NFm{RT*-^3;<xuV5CYK<R{lr5I6R9z{oxLsn^pdy#6pmy|U zf%4rB_=6jO4LCB4f!T*X6yAhI&e3vTs(e3#bFR<J|6|ZJ_<K0*M&EgvPQTgtX`H)x zXz93m;EkYY+(nA(U^H%?(N8jvH%7m+%K2@e{I*jV21-#GMRufBMfon+0&xbDYd@|2 ztLXl#G(@W~?oeIqnS>^Uj>q$d${2)Q38TnYADcjyJL*&ZBEqXtgb7JHW!nqpK-1D# zfAbWh`VU0bzSEqdTm5|-#sVf2N=Co6VOSjDwA83-;-ONIpat0Sa)G@g8>8Ygwck44 zm+KcUG%{`YvgjskhMf{dHPbEx=4oX(7Z0Emn9$>hkbo_2vOb#^g&%N!etCnE_nKvl zQ==vBU#yNaL=5%HSCO5^#IcI8E+V`?!0b1anSt5HD`WUUdDBJ^vp=Vm0f%2Bm3lCR zBf0kU-K`7p?|+|nc=0za<LFX0U45piZ$gp>OQ<+6oQvV?Np3%j9ytF~hNRra{L{`c zSvbU0rFz7_3m_Kh7+L5wx~g>Ew<~jsxsa(?1f{y(Z|NM`it;T%(KPpk#L)KC@V5rA zJtQ2pCKH!-uA-9xlBibaJpOgDessa*c;M$Wz7^3ZXQh)qo&`Z#4XuurLF@8@S=F^d zKB^t2v5F{)?q$!(?rZliL1ZN#z020qW#^1@dM08X!KE(3@#gfdpT^at@Y_k0I(trT zVaB&Z2KPVVvKVjD2@9w_ffw@ua6Rlz2M>r9Ay9r+<l9wuPsw&Mex1*pPgS{?&7l%J z!+CCa!9QV5PDE45!?y@Z%gO*M8<mv#OBx|FyOS<T@=TRL<gN%Gd|s@7vT6phJ;t!; z_3x=;MaMarf1)o8QOg!2!JxuEL$!55kZXZBSyTD}rc;=skW#5={T(6!wtO<7uzOR4 z`jfuv!)r@`4lrymAic)CZwyKhN^rUnSZZkXx0-H5NY-9g3Ak}qS<6D<PquEB5;9f# zZy4zDHQLPp%H*IBwK-6V(*?PNb;~qEH)kj-!5rei(Z;WFgDkh2j)-K2Rsaz$poJ2i z`P*^wB=F}Uql^D%J%ztiuXn$`$$*YSd;=_Y9P=R8+t|nAU&k^&Qe!uz1Ge#`n4O)i z|8(P}SGc?Ax=x=nQ@@mi;_3K?S(25VQvddZar}$D4qFiWE)jHHg_%wL?$l*9hdh3v z6=wXfqyE}S>uq~O*^`{4vJM0Gj^5^1!c_i6Oqf585QV3?C;=F<mI|y-afeQ@=fWrL zNR;rvZhEAPlJ`Ov1rTAcB;Oh%1Wny^9LS=4<)~t_-k<J&)`<c;Hu(UX@ww%K63#?x zr2l46(~#Ivt8|EL-nk(5bk|>Vk%K;bPhUQs_$Q%GUfB^~zUI{svSLAC{kI^?MP`N^ z(;gZ}*@>)8Fk7(@4scSBi~b}1?ltZK?q(ra=ix?bfmUQ-$B$P`j|5-hfg|shepRv{ z6FdJv#uigwrZ94+r=a!exV11odH76LNAvz38FE!GW#l-EIa+7Hwn<1Kx=QD7bg(za zl>y``9oV3$eUq;~_xJd;GLG4rGBLO)C<sBvnPia6E#&4Ad!TwjtA3)gih_sB;6NQY zV0Pz8e?l9OevsJqyCTNy|JKYebi#^3u~5a~*8v&383OO0xw-@fAk?M@SdSb%qKS%+ zEj;og{r%iGa}=o$JnrQ@_Q;FvAUV_h#4x>EG<c$6$Iwr>P?xRFtKUoM6%A9a(HeR% zUM6wB^K)pOoT(h`>GyZUMIU^VT%%*-*A-nm>G4ndno~e^aq1Tsa53ij_0}hF)5x&? zLGx+zJWE`TQ&eJ$Q6-A|N%jT=Vp@wF;B*ha8fjMp!GD;XWjy}1%iG{zJodW!Mu^!( z^$X!~N8)^Z7YP1ip1m<?Zqp&z!bu`C2cj;fe;9-{V^wXW5(K{~%Q_b8_3<1~CIl@s z4X8;O`c5%j?L`Y}Iq%>eQbX&BH~I_$92@UHsC4R1k4viVuXYknVGjMV_;bVmL3j>& z;LCWCGJH4qBHAIx`)Z>HEoN*amnb1QdU4P5?ET)v{S4i$tnFvoBd};~o&MY-KktBd zWL?~j=Gth6ER-aFoZehZa=~oyhw9EVfu2je9EfLEFAA0T&Hl+nDSCiKy-Hm)K|$s~ zIjFGc)cvojOV(`U01@FdFk{&)v`<h;t-E0GL9ja=O{qRF&(?`C$rEPf$$zi``e{WN z-u)+nh-4`4`X4%VbfD=KZ0enNvb;*(2~|#|YN}DH`7(a&n=SUs4`4E7;7-A$QQ(ze zr`qR`kk&p9J#-%=)cM?~e>na~%32x-?ZOr8Tzm>oIg+Z{MX%jZatrnZ?dz*5Ojw_V zjV-VL7AUJyg1i$F_0|fj-=LHrc{tVP-aHiV+x$YA_I_#tR|GVcNrM3omM7e+Uok(q z&-N+nJ^H%`92RkUw!%Wvy*kd%4IC*hvlo>w*Ge}XdQSk$)9(psL7Ttlk}mIB8I?Ub zoq094T|Px|hPlI=U?wJ}{l{^Ax`r<Qf~K+TuStm9H$dins>pu+)(>N70H7em=vUvd z471`MiTnh*ov<Ia%;EMxqpPT*H&#LQY{`E_mqh&Xf3-uK`_~6fiJXx9-FRa86oUlA zJw-*%ECURp^TRXd5pfKzymcBLh4`ON`Wl+l^H#!dby?@WBDp)l_vB`8?Fpfj#V0za z?%U|%>}?=s=;s$SgQAnx{w=|CGXk7fnxT2)%=_CMxrfWir3bhMa~`ve9RM)?MnCA! zx%`)OU>5ob9<iO<#QEu!56qDBS{b9Xu!lB#`|q0OEJKcV=KigTqKdPTI?bH}=N-4x z$w9fF|39kU`;qGZ|NlSekS$qRhi6H4va&@bB(kz+&x~Vlj&Tqw8QIy&$R>`FJtEoj z=-BgEhr{7G_UEDZ>-Bzrzdt<wg!|)uyI;rU)(f&!kyry<)E^<p%!GMrTh7zf`-kDY zUv*mmEgML!)#&Fgh(0{cc6lp?M{dkCYRwPBRxfgoPGRA#FM7AA=8YEHjTk_rLU%RO zZ?;^9AAXhW@ho88Z&`Q*(v+5~y{p6cDShcFCAHLHz9069?)J)(T&?pXf#K)GyI;KH zZ1bz;UMG+(9JcjX-sCW;-o<*6w3-Z3jR#=}S!7T3Kt|Q-n~48aW(^e>?@@EXoD0CT zz9^_m-YVx&O`(3mVu2+_vp3(lI6~Tp+hP1RS*Um!17%(ty8M;}n}`ks>Zl}NQ;YRB zr+XnIrB;y8)?<1*>WwHSPbvJG|FFYSIFZEi&x6q#bkKnwZ%wHQ1ue9=U|;TbNVdCZ zUppx|%Oe~1la!iID+)qW%p@h^Zf$5`Geu9~andcB0`D$IEOBtrIqN*K>jnaRmj9<( zmn#EIoVa2nvukR)fR1Om2gs58qM1?53X|~1qi0sp+`oeV{yBz^9fne>sv{=W3@um* z50fq2rFoB}&v&@?a;2D!@O_EVS4v7nezP~m5S?z~Lug~IEVMt*TTAVW)Lb4r8xs%G zHLqJaoBGT=m$2ekACvVm-|0>e915%?sZqHU&&<77Z%-6t#lD->j;LFENl9SCh1|WA z$_&nPTR4|h+4+cMMBAWvCuWyqo#vK7`#(%7kc*|M#c>%XE~p^^c&WbQ;^7(Ov8?ve zq<TOMg*QjRBLOlEE*4a(ijT1`QdgP@gC+iV{EZV>*Ex)|>#L1s`lNxHAw?w+qytiJ zG|Nr$pkiigoI@KB*xL4>uHoXakmZCkI)3VY;W%rzlw3zPo3q)#6O=1b<}7-O@Hi@+ z-XVs-CLzHKVFtmwsGoaP?%zEz$;LQyx<m(BNqc0rPe5^)o~)@@P1)pbRUk<>vWY){ zJ#^Am&tgc0EGIks<nlD7b*2}pE&pwU)G<0e<q$dmdhoo7;h@jzCEvoWn~>Z^9XfZ? znjpY5_%cy<wJ_RstE?^}>(c)LHFwxKgq^-6V0drw<;nS*#U?yZCc5u1{;hWY9+eYO zV3C2|ZC?xBKPgai;)4M@x!{X4koJQn14FBDrhYE}qz@&j=4mYXEhL@tReYu%j7sI8 z?{?ASQGZ|eZPo_x%d%UM^M`S{Rw*hGpOyYuCkf#q?|M?WP4s@f9$O6#RW=dSzlWF* z+i1GcLY!`b`=t~n_q>VmYn37mPE8`OZK-X}|Fmc~`A)!dv{lhR1KeHgiG*rVsmM#| zf9>2SEL@K#7AYIIhZk|cJRx=E{g@^5Dbzb$Vo%9}*$j;B^IzEXr;6ToOd!Y(YD&#A zT-oJ#$x-&WsD{?FRS_#-llJcGzAJ;eJDLr75kz&SfUD8%=&Xx*?*NwJbZsNzQ21Iq zBXKh0x{EL?6axV}XE83Foi3&0G+RTdv$fu&BB?iMrft%{&6?W!-<SZM0_!b?D-MWA zf!JP}>Ww>EV3@FXUew!e{{AQ5UF;Eqf0WVx+{!+RgfS^RRNj<b?;%mZ1tP6w4{}2W zHL`9IZhcc=5oIp=&vf@!YYnDIy!Lmp*yn8PfO%A-pLzq6#Tk_db^ls3U+UAcS7fwJ zjv)+kQi)KPyU8bp{$|g8e6@(%EKQ=`Oik)yYWc?0CPk6Ih}&JM7koCN4_uvSE{0ED z!aN#nXS*H8P;60(LR~L#!ZX7nWv`zL{ArgN?~?nt(JLnBEd(mhkA6J>r(vxeqRchn z_s048wjEBf#{5)%&p!l^Bb(&0Zu~<=``LGfy4Ny@U22ZQ${)Qm;v2rnrV0dT&dLy( z|FMC^ZX%8j{ak7W%hQ^pvVC}t$4uN3POkSLQS7pHb_+IrExbl1X#_PhfoQ9(&(x=d zEt69mQ5NqdtGo{uxfc{S=17r7oRXujWrLDG{7ihxiOGZJs#%oafiTx#Gm6YWT*iAb zGQQd{VUnI>-Q0FSI$$WS99e+-@{?(kMB{Yovsz=M{SeLl0`%#Je_Pc96-P2+?IC(T zz7{wsrtQcemrqLAgDm7rq$&H{v}d@%tMwxaE{zBM^i2`-HFE#{PJA+H$WNpL>N5_Z z6U^cI%;d1hmBlnCMD)OK(Z)ecDRDSkpKMr{;<X=0P6Ynsyc|qeXXPi-zKV>s=8+B; zPNGF+#ktI=ti$EFCc=7}nU?Y~`8GQ9JqIbMg>c?>nk^MJYY+HQP3@TP9(<VzvvJn! z2E8?}>%{4JWMFOqXpggXf+r)sA=whAq_Ke|mu0jfxgIIyw}N&?%b8JgY_go#G;~1J zZW$hiL-@3QsEMTb1^P4k3LM8=v`8FRl}<k_68R+8MZm7Fn5wZ=M*{Ty@&wu1!DQli z<IO<Ygneql3mHCk*mhM^@e^IUBzT_U5A3n)o{00#co47|jS_$mRxmeP6mN<jybC0v zqO9Pd=B8{nMu^zA4jYotjJ1C=7oc(=>5vFa3e63NmjCs8E0R$E6R{E`1*d^la#bS3 zHA&Z_bGVX7Lydv>0fmj46+i8DboiA=6DjYJgH2`1irCZ5ai&?&d{W4CEh5Nw;*}cW zwTuEJL3y>x>n-vU<AaR+RG7_@`5U{~*(ud}Jb)j%GFw3Ns@C%!2LcjvO8^IoN)}V2 zd(a6iO5xf5-X}crcX0szcR#~3r{6@QsbP^y^1$wc%<krM{B$S4I31_awqDg+KVr72 zAeXv4Sov0_5<x}IZ0>6nIrL2khi+yIs=s_=HMl)E&#SRH6TF;>f=FJHj<NqY3!qfu zRB;~j9IbbTnABV2CM`nbN5fbsr=LL#<4o$Xe$Kmb-7f|$^vLX2k9DM_l3*SlXpb*2 zrhtMXuIFAggdxfou%*b8cAYZL?sAa3bNKjB$$!J{+Kl(WTu>|R^0V(05ZEIIT6mvC zo)nLX^WfpkuuJ#@Il3~&FLuPjf~<zlq-67c{MyNIQky?o514nOw>?dJeMMzGhQpWu zgT=x5+;`H64^Hc~2TB?yGvH3_=rI)_mt)2?F#Y)TQy@UN>L)aqf45_bmE{H|hc4=_ zvpk(}EJr2LNp;)54H6k?32(|xJ&eIRH&p{LtNYIcouxf%9^yY{7#Opa(rT*x`2=IR zc9iXNIU4%7D4}Jwf8|fz2o1)s;qgz+ew;79vcp4;)V=D~bRm{1`O1$Su3KR(hY8v~ zvrZjk+?OXji4V4BwcmC|o!)_QI#n3S&Q(8gm{o&Vjxqw~#GPBM_Ac0ym>mn9OD}N1 zN1WY<#ezejkz9s={&Q}l3eEXXLdPOLlb<f06m`wZ=Aq3l%x(_(kIp=qhIW5kHQk!) z9NK1@x8LVPz1a0TZ%>7U(>m?BaBzz2019=NNbc4;<e{t*D=1ItaJoHgdO@W(s^gx{ zg2beY8exi<(LUr~1v0>lCALPu<Fef3on9)^V2zZDbwI^DK^hLAG(&C$js8yGD)051 z?>Xn{(fXDMu3Xj2QEU~`XMuL(%w__GV4P{#y>szmRUXIz?iGHPMJ+_julMNs&59|M zy|tf+so*HPtduWf6^>Tik%ex<_!p8<)k3vn2vRgp<&$Mqk%u%VFi4s*MDjMG_3f;< zw|LsfOc>O~>XgNjGR4QYK4wL3TC;+Wm@7<@C&z*&O1X8?p^AY}6WTwHrpE5t7iKa6 z(gR0yT$N2vgjVp$7)QwyUD$=R!VI>A+=^gd=j}k7RDAV%-a9Xc#}VaWB)(BAPbwkC zAMD7M%+tN8LwzzXY@(>*xob`L;3F?4s{F>V#+am}h=W%46UK#w(W_@eL%Oq(moYHe zwmm3UqEJ#;U(a1sy_A10FH6Dc!1YasPMAe7Ji;bnSv&pOHC(7Gr*9p@b@b=3!hJCU zJgjv+W&E@ur>3oEn787e^UaS^ohy6HZ-bj8<pb7LAQV^oKj6iUnr>kri$QPa<r}K( zH-KR$>xEfoSi%C4I^hS99__G-<X}Q9Lgy83%XtntS?!N;Kn{||qJl0InJ?tF6<^oA zPT<WBco47`W$`g62mm64M@!As&t|*61c>+z#N|$*5Y=c_#;X|Hi{%}FjJ)_wcW4(h z9a>vAcWGC@@>WC@!!B;8T@NfE=>&c|GZ)gamp<lAA{qFB`a#<ax2N!ieX?SE&x_-s zU`~VifVz7DRB39S1<Urhwow`g8c(CSEq?ry5@M2zOx?vc&&(VL$Q{?+383;;!)rJP z7ezkLp<!j?ig}^1Rk)Z{bsIHM%or!}xzya`e<g@jl@NN+ccy}Od>Ssl2HS&cnvd4r z^ioe1J*!l%t5JNbEGPx`38xuNU$n7-mQ?t2-5D6@s`~9Anyv0!fX0+c#hW~0V#(f& zXw~3KHGTROs%HG}*q2)<tLD#&QUwg}KKvotRdaT(3&oj{LrBK+!youEj?bJF9@BeH zc4zX=?-LO4u4GBgt!|3j1=guMWDSR#;CS_Ii+Bn4C^ZagzOkXe;-07wR@4I0U^-qi z)Mndo7=h8~a2<a`eLs8u^HbH-SZ0TVtYdB6A3hrH3bkKp6{<j|p0p_|L-e>LP)b7p zNp_F5(sy*%q?$8kMM3M&n0RbrSv<>{M@oNs4^>dzr~Wc~&PjaQInv|l-w=npr9(_s zb6AD^x`1R*LW=>$PZ+rp+^Zm9)LOOc)WDwj!Q(wm{`T6I?sxO_mJ+q2*;I4Z*8BjS zrCp9RX;+AYJF}Tsny3aQg=cVkQa*7G=YO`o`<7caKw+3HRUc%O?!~lx$o|+>|9v&6 zqN>3kh;wKgD?O*Z2F5LNr83#=D<ys_%uoAbYYIJlycz6rsJUL!#hwG9_KozwejGT3 z&m-J!R(F~l8&(I&f!+X5g}{Z8`2-L3lfm!Kx;a&(>_?&Vo|yI_V?xsD{3h%m{9lY= z3GkzaB83B|A{g7=irOn0pEn&huAJ-<21k>3+0ruJ??qsRyX4P}{FA@7RF`tFL=tz6 zl6B_<GR#Fp4l2p5Y-5Rc<y?$yvVkYE6NOUC&W|nP47Ze{SE8U>Zv~0Yy&2y+g01c0 z!^cEi)37L}N?G7}UI~Ey@0Ns_U4^W!x-F{LKTmlKFe=jUX8XS^)CP)mrofW;dkj&) z&wxAJp<#mI0D&?feiJL?XdXUg4M9+!(c&i78A}eFsz}f&!^Ky_f%)7~t;{8$Wm<0N zae=u^xYz{(sDAr$naVh;5e$h#{o2e@rOMbT7&RpD(A-YFYu1YQCGG61h^8F`?2;p5 z{epGsqdR)U-b7+o=GP*s=Km-*9n6ezyq@+$XLbS~^MWi<i?xlVT#w~L=jBi^We0GI zZ)OGkST-D1g!f*x>85*5@O7;mS2UCt<ueYanH@oe=C{^{(wY6`Jyckv-Cw6F1pauq zL1k!@iLIu58jg{g&#4*+qg7y`Q6Ma=OBhD8ptDRAeK|anqta8%{rR?QP1rvw92{gv zl0<Az7EjW&5HPr4ax|nF_T67}y$;y5R5!#;T5Z1fwSG|h7Aybbp>!#c8P_m#wAZ8& zFX*|v3xZlkX^cH9ah*L%k(@5w_~s}=IaAAnj4*vN8d{k))w?SHPTqHw|7l&hU-Pn; z=9cM~aGy(T4bou{-HFt6Ztgj#?AF@kzRAiqd@=vA4D|hu!Is4TDGXVR_lUT3-<1{p za8$RYV6a!B6Hp5E9>%k--$JFp>(oQ?CexuW$$Fz#_~wnc!LdP}Ez+%OR^TX7@gSw_ z3?V%gC|F&jDQW-qgr^(Oa*1Smc9s`%Q&6hT1nKnKJGyy1xJl6ohy@RZ-1C$J?be9y z4(NLB1Yp$Ku18gP*pg6wH2jAqnUrAoJwjLX{Vvq~?yS#<NE%MLH<A=920xg&-CHfU zmNtnvSY)kgHYb6ooTd)lIewT_%{6aVeBb>N8Zac0)aC4mNdft7pay;hlfx_C4txs- zU;Fs;nXt2ac9{HVwb^fP%rTHc!z;2_(1)3d_0FW>IiRqChFjf*$2zduxD2tyw4Z=( zF$e;p?Fh8!l(<j1*?ackaa&!g!ZYEK+{D=c!QQDV!dF{~ji3RKx7{v9657D;As$!G zYpXjgA3VSJH?bl66}A)y{2vx2m?OETgMp2xj7oz=t=AOMT%oH^E0d3zT!kfFC?42{ zFwiX7FR0c09E}+2BuO0EJ3SnGrSsUtjz^1Jdo@I=8Ou8LyeD|8tk!o_*^_4tRBOmp zYY2Ysq?k2z)a^oLJy8l4@ubDWinCzWmDl8q_%H2|LhDQ<v~DlrCG<Lc<|4Ix4{)Ko z;+G3DqormPN*!|Ir%ZKbOBGWs8%DC?5teh732!cViWi=8>FiZKODIemWM^$139`nP zcgJ$NvSYAv;^Au)Q!t(jk-awJmL2i5#ik;TQu*^pSkp{E3nSU`cQ>n-pj!o|#mXWT zhR|?vjXz;KEARtlZRX;4?V6`W%z8Iv*HUeihqC0SbM)737c*jHHoqDCD|iw5!Kl~a z6%YCe`kJML0a_@x0TJ&;o+`igF`n+h0qsS$SW%c_wLq%?_e!{S060zno)O*Y|LCuG zYDj$b80LFUc|Yf1HN@45isGw%F+}I|;V&Md&-b1cgMF+VDP95*(zi4obhiDF02<e5 zJr&yekG0xZW+3>`GS1|v8luFfeT$?4){2k3f3UzkLPz@UU@1z322e)KYy3SNUWl?0 zy_P>5*X`7QWyZ*{6{kW&Zky%z=F}Do&o9y~^!|Y#?{lGr7}|Wrf9M?E1p@7;(0_B& zoVO1{6aqU*r!hBbQOj+WN49n3>E)ARjjwru>JE+K8Ft;vrYDBlLvep2`EG;Z<LI2P zGl+*%$QC>T3y|+TSsQ(_TNXb;GB_1l#)1~-3hzd`p#yH8ZR9h5k<4=KAYUXb$L?Zp z4M48X7kWPZ#VT8m)53Mg`^>OrHJty(_9%9F?M&CX0N6lz;sE9;z@)t@_uUGQwIA%= zINQ(){_!udS!+7p1(x$@jO5x6Vui_-kEyyfT%|nTgnGTYzv&I<Of*wM-3Jdof+4Pr zT!vTV+6xSF#cxaH?l+9yDPTVRg9tMJ->IV*SY?f$$)MPT<%kba|GY0*o1%10TxXqn z7~7Ih!HEyCF>%2j&6sK&4z))XC51A|E=Jh#J%4;Sru+=$`)R!xY@|(T{8Rm=ib32@ z%8Y`5Usi)tru9#avC*+kMve4Xr6re|pY9&A7ImZQ*7?PeUocII?<I@a?}og$zch9o zNk`kbX&|$%bypH(d}J!y82J>9mM7!u<d0JvLKWf^DnZi@YphX50a{B(Q>m5Sc`)za z7N|o~Z$nPQ5%C!B`TYzNYlSg~h1u37+f2i42kUQ-X+|<E)cfn+4itF_>;c_d4>y4g zX2;Q$8@n<EHdYsL8OMbBh6XgAy-2bNqB-N{BvI*=a6W42nD+Iso$-u{xK(85cTe)+ z^Jzo;Nva}HtJ@D-*>(wRQ{i_Wc{NEmI?L=>svDQV9wf=SNF1YzAt>m3`M%@&aX%3} zhgLPx=9Jws<5VtXC_nqi5EScep)3-#ERtF!sAsY)_4YFnHRZ#ez*X>l`_)I3iLdR) z#K{PPub#x!qL-oksiQ$P_v6Xt-n#LD$ZepHglQjs#NqL|6CXD@!eYx<EnCy-+a-(C z>l-qrG8dNH!@P!r$bI9>iKy#)&#m7-qENJ+02BvYM+v7di$rBV2P*k(>?1`1C90qv zGTvuMXFkFd<uHt2acl~?<0I!|f`Zy)R!Bn!1rfWxfvQM2&ynGg<_yj%0Kf|(%kyKO z&%Mv=%{>v#=mQ1}2BnM6a3{<BLw$-EA)=@b_IsAla0l7dsBk*brq;@#KH<jFPR-LM za=ofh{x`r%WMkQHfs0doiy1fEu3J_#;~n(N)LYayW`dmWSROQO$b3s?>n(X=vQ%_% zHr!3`1~gtuUiGTa-xn#E2#L%dBQm$(+~+uFBRxC+q8iI-)%EU%P(}J>meoKA!t6hJ zjK_@cGG&GQP{6C2+koP$TFzqXb*8vJlATa$hHDw3h17QvQ5E4d>B<o92c?#R2<|tn zr2mkly8{dSm@*(3+S8$SuOj|Ym*xX)!se@1RH}sctQWEpzoYsVAvyIAd+i39F>)=( zwsOzKWr!9Mcof{3eM!(`Q49*G2Uuze%QFDq(W+0V&M{EoN@c~01qTx5k>vCB+jDn* zQQqHQ=}qX25@}%p`nfkJ&0!eqAph1|MG-rgI-CI%0p)$3N*AMWvi_~PC)?lIFz}S* z`}xRWGLDwxI{)Kc;JKn!X0Sag)g)$Xu#dykP78zX%$B^Wz(egWluUGF!tj6v;sP_t zYCLHTHYfaQ<nW4+Bq0^_XANV>avVJdYDLs!?5;|EG={BqKIN<~#VB9lHy+ZujW6tj z8LlOD<w|%H*tG>%bajqXp=bcn>tmPlddHI{S)OuQcR*q*HfA@xt>`?B@TokzpliTp zN9xsi&G{)`P09RAwz}pMtN&}@X@KQG-w)qdGN3%f%Fz+lEc2)=;%g-SW&EU{?OgpH zv2XXm66%;ED&aMp1Wh&(-@u#qtS8_GRiiUg>|t$!Tv4Pg_UexG1QwL;@MQ5V)FYSY zkqsK=@us&M?$&xQ$iJtR-;yNdhi<R38kICCD|(ueKYxBsjmb}fXsq%n_+U`WqRt_F z_0&44#eBQCK<%4h!;=qaIX!gX3acbS0glmrQFEe!i}HK{8N_9GKuk!DG+Um%P^4V| z^R&;My>1-XAZGkzBtf=_den5brqFOOh@p$vLLCuxtowirW5g?IF_fmZDogz<S%z2q zXW4;cuJmP|wLWX-B;w{(rC(S2arzOW>UE~eSlaRyuSDcsVA9F@V8O!N{VsZ*fp)b1 zxxG>#*CE8BwZUN{-lOI~)<&JOh$fkgD(RQBrr&UUb&<JFtD3bElu{n2z6?LrPTM6k z^geLeom&P9GW{S)cG$4!;%bk%qZ+-9vjAYB%4n>o{R8^WOdQ$(vl66OdK7F-=dCux zt>0`!xn{_(M*RF5Hj`=Yab}gVs2z<6A6)f3Mn8>bg8A5ZkGRc%0Ek2jjbPTJ?x21e zO@&3e5~klcy|XpwJn_iO^Nk^CYM_nDu<-WK!Ov(CvrhxT@s}E}XQLt5kh{SnUc?x2 z{j1Io&m>L7T&Y>`+-7su?>}?$jk*5icH}0DX4?Tq;jzuMf^CRUG}x(s?_799!NAGM z543-$iVYmc8(3Gi(XrVQA0%U@OL4MVc~5WL%y3oTiMd3s)OmiR>CY|Yi@Bo~<c~jc zhoQxKuD9iteo0+AIQky!p%aiXaz1v1C8Ex>ft86Up&StoqC?VetmYnBc4Va!{%-R| zOZN5=aVE!o<W4EY2h_Dnf`X;p0Shl*g-1fNM|Zxja?b+~!DeM>*#AT_rmiMy6Od%; zH?od=%%UxI*W)L(2Dy(x8k9Txesw|FMb+7%5q6PG7vXMY9`7DC8lP(zSOafF4rvy# zRkCYD5P)3d{Y&@zy*iXbNoBWx_H~a_Dk^CC{zYXs87v}89u{#``2&(&h>LU?4+Gy- zE#1y>G$*CUo5Dv%>@Gkiom}zs&k!myTOrUNj=LRPR-=CDdN>;!u5~tjdZ(baLbl$J zQen=$PiWC({=~pD!;Ds*n@awIFsyezLm9{E(J#6PfDY&e63oY2gD+jtnLf4ho(_vq zDjD?=Ix;GdSb4mmFR*?QS>)d!PNpW{DYfxYKZEbuX@CEj3Y9gqBfb&A7?qy!<k3Pw z{o*laP~A$C;aox~QPF%}$Oe(GJS}G7H1ml{#U)}`+Tr)&GK2Q~O4!uYlZyefO@jY_ z|IAk%{=Ph71JLI=ZQ=0iMM|jjB^xJ%)Sfj2#pbp#RpaozX2~T1!{b=BX;}uM#_e_V zfF9VxiJh9Bc_BdD#wONoWV>pQ9_~i3Xuem|QA<AU{c;Nl3QC!KCKAG#ckxvA4o(>1 ztf5DnU)xnIs_p>?3wQ9NC&`2p;V-TCZi8=stB`O@F_zLv{ZOm*v+}NUvl4yXql92x zn2#ehEJU%qk?d(zm+Z&v{ggY*KLG=H0KKHVEIXv4Ywf+_xQs)m8BMFSj>@H&G^Pa! zZZ-2oc-Uxgw-*F91JecW`*?7bhqACUk1gc}!Lk+bLUF2&Gcz+D7F;~2yvZz2zz&^$ z-MB`?RV3{YkYx<3+*#=7kd+c&^@0w@W*X(8Y5Pb7-1kVbH3qFUjWzhSO*rq5uZR)K z?G9Uyb{P(~@13{M<H8PQZx2{ut5>Y_tC|WYVo@cTbw0ZzD^Ys=up{k^&R?BFMId*d zB6w*z%8>}|``Dl8In6)zT#qZCP?~ldBr@N_8Mu(9f1}p9o~r!O7@oQrGodKaCLWy` zbdK6ac08N$F`kEUOBO@yK<D=23v~J37c5btEc))K_lPmOXIA1YTC6^H>C7Ep@mlEx zVM_LlT4l4QK7BJbtj{p-ehPKRi$^q}!p_Zo=YIruEm#zJoB9|x8~*T?*I#aiJt;zT zF4_pSVZbltld8+yR6PDE4=x(ApWe9!>6{Y?w2VTczi~T=4Cn`l++4|I*{+lW`Ncaa z0&U9uQ^`tt^Q;f;*WP&^hSBj~>4;5&_XTRc!ppGl2cu;kD0$DFN>Kv9MQb>P^&C?7 zD2vX(EHIMpj<-(SL@_n%OcLF9o-FnD?HO)B>ij9msf%r7Ld877bH)YWZ!Wf}MCgMW z43zK%IjEMuJ#vOuZIpkH|KlqGBQF1~N+p#-3cj=5%wASucXC&3b-Ck^K_)}ggjVD$ z%_Kz{!wKt@iHjl<t{ll@O@5M~$D40My*)M=VL%eD-mdM((zXa(Fuxn5<^w#g>aVEC zNdE4t6;fZ<Oal)m0CCg%SK1~RzIZxbX|)$Aim4qUT0|?#O4+MW5C&K^g$d_kBH<MQ zR#KV?lZY#j$yCIdeEhUB(MiUjKt-#{>5mCLk=U|Xy&b#JTg4EF|7Zh7^4b0~M@_@Y zbVkl->5pqKhg&x$Rzx)K8@ktZyU?4+LE}!S@HD>mT-Fo{PzdWIkGeOlD`|E)9w?@1 zd(H=srVFj>h=0X@D?|PdhvQ1&IW}k`=7PCw3b~0n(=0vn(}&#_#zYznYRvwcW6%uP z)~U7BSJu1(v%F53w-7R4BXv{53t?v&Z5H)rK})1*g5f6pJsM0m+q_U(F14O4;AGMh zR5o2|!Uj!<tAKXBJz<Jf^}dgW2}Ob-f3R_J^qRX=kT4N&gmW{+cibh#^>scX4h`B0 z02BtT)`a5*E1o#mRF>we9X`;3lAwWWi(|NM8v^3-OCFR>8FJ6|pf01Odl!AXx5qPJ z_aYxiM4?602GM&l*ngTtdSyj}J75h2p_hP-Nj+G^lx~AV!r{q*TJE*OT|t}1;}6l4 zeL=lPM&_b3KQ5nL#iAM-jEBhLciB_L+=iap$EAJkM)OfyI?8F`_B^Bq;&C>o`ETYY z6}VKF$a;-R)u3ur{+E()g)cC#LB)pZG>OZR!08{(8v%EYvxk1}2ae$a!`Fy3n+ot6 z1LjeIz7Up`SrvOK5;)ZnV|$5#(%q4w2tS@%#aM|mh>E_X3H+6FD|7b};97PG5&^^R zC`(r{*h4Vsjy$%2xosa5AReJUKqPP<I>{WKlamR~DWEHzMB9}xeu+VJ9^KJ4FCrsr zldHkr-i%Iz9v&Gu=P!2-(KlOH{c8jy$!vH$)EM8$ls|!`MuCVOO<!c4hlTALpEn;z z88k<j-Zg8SJCCV2rib?6u-92EWPCWOCJB38)F?}ibTkKb6B{m5UzQf`6lkXkJTbn? zBU`QJU$oO)(M0Hq|Lio5Appg}lMhTAW_Z1aSjiDA!1tCaFf`EIa@>ZNnwTQ4{QkrK zEnI`^s2XoLn@iIOJO6Ff7%GdIKuwN*b<905ci7W!b@~-LIW8L%lj|H5E+**p@LCOn z6B=$TS*{Q*t?4($xPgrV$<n{@m?+JV@z19FKDLx?q&IQOrE?S?@0I6M<TK6)NxQ(A z7W2bX+~%^RLDxugcD_?V#t73j!L|ezueAGC$`4Q0zcNo7cZKAP?o<SyZ&h})LOog7 zEZ2cz$9;#PIBu1{9T+<SF!r8f*=H_zQ)eV?tmG}gL^xHH+ox0~&iksGaf|b8;$Q1( zE$P+7s~l#nMRs+*8k)lgLXT$;vf8o&r%AAsu+AMUgx<|0qhe=hk;hwAQCG=16>(DH z5FKBOeB-)uc`ZiVgp$G#L4DwSG62xidH$H>h6%ng71(ar<svFHOgf+<n**U4)&P!e zkIU~sK(>0~bl&tIC<YMc02o`_&FxbpiM~DfnO{5MGEH1G(c}3~$ST!zej>$ufXg;e zuvxf3wuzTWrm~FjbqJm(r`<3)VP<Nl`BU>-Z|U_%TRlI?Aw5@NgE2a5ElNG(;vC@t zY}K{vd_<Un10+0<i6u5>E5x-50vyb&9-U(mFoluLoGy86Y*ENB9ckW_s8T=4wJnDF zuM?vu!-@1*xA*5!9Z{8QWg>l_-#cJWnxu?rNj|M)j-!$?ZZ+ZZp>*4oQ$Vx;^h^Lu zBy^f}r<@0wgW)p~@@Q)|SZX!A1b7#FLPV;~7&+VV0tvoA(6d)<J>r+*?L(S=v++1T zs-@klVhDn^G;Zaw+6DB3t{T-CE*Z~F?MdTVkzD?&oCxtGy*gLK<<yYM&dRjcr)IoK zbh75ezSX60u;)Kb3{|$sncW^v&*wm9bb{j7j5A}#ViDK%kqxkq^gCN3QAC@&qrP7o zyNtpCbt^RYjs}k!EgWqhbuu#b$-rU-1#5GnZ3BBJ4!7Nu`WDW!)^L8^&HiLRG_k3< z&|hd!^@p?iCXJ0o&+6V5032!=TM@0miwBirzk{2eo|j@pYL67o+GJXM23$|~e1=d) zlM6OK-qZ{?NZF2xQFb>M7|SW*43RVwN?WkX5R~8mo49w`G0l*aWvkcD@4)nnW%(Z# zUfoG&tCqK#?viuWp;Px1rkGFph<bFcHz2t<GWr5_J2+9iB+t-CvtnoKO#&VNru4cP z_UzCcZ=yRsE4JieUdh;!Hx2MVh$`kSTuKh>^vwi23xm0#hjHa<q@N<^Oy0t=h+0}Z z+xJ7Mb4+CdaP|lq4p#qhieIC@%Kk%0_*npFR4Tats@$I^n<{*&hRzB2N$7-+l5`}6 zmLnE!Bj?1(G!C_8f+!5@LbsbTYakyLzp<y0{K=sZ&9GuZLU999v{&h4P^#2g<GJ_L z{U$M9^Q7a8(_LC<MYH#)o-xLRM0k{@0$97?y7Zp+ZcnPXy$%ST)^hKRY!_fqSzrh* zyKgVgHWe0u{@Sk}SL->P<lj<Qj1tI(W%bWbR9l-+_Xj`!Hw&QX_<aYpL<U+ZcIvtN zY3z6tb~ilf6($>zdndp#Ts7TYyWC$SmP4ri_c;yl{tRnmRhTR#F>m&9+f7eCk3XGd zRg_cJz{$mA!?4QWxN7^F|6i*C;98Y7e@=?pYG_ERotV?3@P6jp_5!RzPM6Q{Z1XeO z#3O$(xrr_cKHmi?>bTp{TD=1Ya!l*dy=iQ{MVdL%cH9$lUrltafm7Nd*w&*H{FTX= zYl37jg_ux}VUcCcp+9Ncs*1N2CT%ci*wWp4Qf=UV%&eF)wS-Va_(%YE=^u-KXqqPI z%KT)LPZygLFAVKdLcgFS)O+%9lU+9;x+I_S@ua(1-0zKkys)ulFQfLH17fy!Yzq^I zYNUG_gI}_!S=ZeO2FAla5;p5)=%5JCTFC&8Z@`!88FFtF1X-5(uu7#y)idZgde(W0 zEdMjEz&`+)%2JTKL$(836Ogx+#IW&r2qRSYK}iM8?OU>aEke@i&`Ue+<H4z#79N>I zV($h;tdHgeAi0GEsXBi!Dxxiwo~0)fg{bdy4F3FT@}A7%*(CIZkJf~POt2iUb_4>r zH#aqkWmYmKCbH9sUQ?H{q(AlviIumcL2B!^Cpvsk`ouuB-&UPfYv)nwP{owDKebqv zvvG?}0^3EstFn1i;GK534dy#WAw%DCo@;E<_&Xp(5YBu7+k2S?u*%l)ld5%ev^-0~ zn#Q`;vI>QggOWNY&H$8vqE&>Rqr)pS$|2D^_0Bag4n>&VB!x>K5Jd!D!I3?1@1bQ@ zyFglu5H053qDPukX_d<_*}l!NkRK;A+2FW#xX!RG`NL%8G?0Jtx3^RtH3qr`G#`{| zW6!M2OWcv;MY`4HZ`szhZYkW^uAYum8EP>J_x4TQ6d|(FSbtuY&sb^|!`W<yuqs*0 zUv$e5s8Dtc36}ol_+G2{(@SWOhesLEY@%Y56l^5j0{_QM8b(kUf?iV`tbiUkH<MYf z(oNnaaYPwf^7GZFet)N|hpjFMea%NVNP>ImOSK>seV(IMyi|;!FJV+D2=aaNCf!V$ z4hvk8(E62ljgF*O=Ju2itpq@st$WcfZy5~?b9E^*cB5@MZf4c*#@NL|%Mb=EQqHEl zAQs_*2`vwt(}N=baZU+3>^2WP#ZwQ-;hwKgl)Q?J<s+C9bA@{;?)_<v$c@A>U+kS6 z<TdbSxGEDS>sluKK{U~i9EV@`iT1GH+^N4^j@yKmGRClr3OhEwxQy$2B=@2!)ORPB zT2tm|XH4pBzs0r#=K=_jk6!$VBbMNUi%<>V2_xmOImLmm@ec(#WX0BgO*Vm9T}Ho; zUl5(B433uG-c^w$wB*j>9x-o`XgsXh4vB4#NyRWKCBF&z|GaXOEx_WS2XhPp;ScMJ z8Diic97bPlcWc|5)DMz~0B+p6=7c5fnJ}1jU2a1YIKbblXs?VpvoaZ9t~=C^iM%zO zYisS4mQ|OfXs2HCzo}Q`;qp==O!@0Jc4^TE<EDD%d8GS|-wXZQ__tDhU5!M~x`jxQ zD<C7vilS7hapq7{^i8jBjUuZew&hyaraHk_sY3sW{(So+z1u=rgx$b2A0+QU>t~+C znjepxJ$pVUE0@tO&!dMxQ)#Q<4wWkUOW9Kv+-FJj@E&3UwSfxB=4sK$G%2<1q35bt zJJTUv5Fm2RjRE?QTlGM(mhbFiZQ+&b<yju@SJ<7DVU^a>V>k_M;dOPA_%DEAoE@Ry zA`_N3Nd4EFtKyQc#%AzJqo1{qXyDX;@6vM!Ryy{3D<pe1I}GuU)*qg<QFI!&in&1? z4sjQOq<Fu9<A-<--V~7^zkT=yfP1A#?%E4OCeZ7X9?iXh-<<ErzTSNF55eqeIq5I! z*nDaWBZhY)$`SzqLj7-REoF-$IDN+`D?$cD35Jw9errmB1su%Vg*Hc9NJ&V^Zmyz} z$w8w;kPT+IBA$hx9Bw?>qIyZsW%gcST_(<MaCs1rlp!Zd7<jhOR*%l%=N`EW;fa_v zC+MN|4#_)`0aX4e$HBkhTFn!Ud)&-%jip$x_f7dDgFle@c&NsVl0}VsCC{bKywPL~ zkBc_7;AsK+<4v~dG^vlLlC6JSjHFYf0^bYleZH;~5u*Mmn`dE)<t^3y?|I7inWORl zf~}eH<VjWK%_nFpyrMbK|DBPgQuHpwhUhEYdJ{=)OxG@fG`$J_v`Bi=Nh4e86&9%Y ztLNJdgeugQJhRAPyufd)FJ#`JH7N@+n56P6IrW|XgGu~y4S+5}5WBxW&&9v|YXqqa z<bdP&KHeev^{tunddU^AN0V6B4G_E*feNFHu<0xMw_opPzo8i?wLjG6EEHWIp@87L zb?!x<SaomLs(;goqwQ|O<Qpvx#lE;6Tz{If`10cD_Hu_Qunega5WE>_<PV1LJ?ytx zXi;op9XyR4x}zFK@%su-{+CZFVGjC4BWXLBQTa33B;rrC%qy!ozy#~aPccWeOHwFk zAl4Ez+Sf?do|9khZdp)>*$!?GUu)@!W|w}j4g5Sf=8$cr6)h(YgnrvmnTpolHLDqF z4zIyKnI*LYUROHVSN&AQ!$0yJbG#M~p|KK@Ir0X+iCeh?i}^OTYVAE)WQy>JtVr)f z!_&Fn8{?$CL4C_KxCrqf8<_nnz};e#sz8<g??m1AiI^4i=XMln_rkKJ$rg3HuT2@# zYrdD@*m*st=5`DJ+kU@Q<Jo$3oyxe%$b!ZOU%6K!^hu0+pWjp&zYA0<=;O-u`dx$F z?~A>mJYhwq>5H*U``>PU{rr8a+XWL>(vgT+9pAJBUZ*u5u5K?p-qAIF!94$2Fy>F{ zWEz6LPhNurh!z=Hb^(VQ3~QqM<_Tz7t)^L{T{p^7@Acl6b3XntdT_&pM8~()o?Yl8 z`j3xI%lMT?Rs#zjYZxciPXhT#(0%jC>^u{KnW{^1lEgRvx904*8-ASUm=pu_oG-|( zdd_pLo=u}@FAORdrG6%OOmi>#=iO*=g&_>2s#<4kDnEeC+kVJCO;Xz8nGN^q(ag!J zpQ;QpbvA!8mS?^P?=)muFC2gNNTOZ(*$@pNU`1OE%^M1$ZDcI$TkJmg>~f~11q0JH zFX*!Y{Mx(j{V!ZG;KyIM=psMy;0+hJ=S0XvC6sPK<{e^dGr>w7e!7`twD^#^>)H-P z4=aC?<meDah~aL{_U!6QF{oH1bm{II<pAsRGDiDV3_YlXEN2A|JrE(Hfh`!ZLQr1m z`{@PQ^eqCpxrk7qSszPbrgw=)xqo2aCS95tiKn;*xW(7FY4R{>c=o<CdEi8EHj4+< z_-UQyL??Q)W{)WdCWYBx7UiRM33I9tWv)LkL0BtruKRo@r+fhU3R@{C$MS4RXGp>t zbc>V^9GJWr`RW6hO-LU~#F8o23!Zq9t<dbHqS(xL-fj|02@Ud1qM1knHl+<<(TbPS z&(kG^`hhqJ6{p<8tJ0m-PsUf<>)pVg;nhM0j}nwGbL4@;%_b={Ic#;Eej);)UF(_q zaB&`Q<s0B&1&!qR8|4?D+4A~(!CSiV1E*UE#MTc~K~-QBR;$I41qaKL&xjl-EFaAS zd*>+Ha?jao5PHHtTkiE2ekKWTslnr_8s;XChk-Lbp!-CIR6X``dD?0;K`T7SrHO7X z@H85G)Oy)`m`QSR85?|dyWeP7pdTjkHb}dl(ClXg6q0D<{e$l}H7d7acf&~E4q+<w z|N7Z*qJe*#Y)-UV{(W4(-I0BF>>9k<HapOp1$a3}dKCgHSmO<N{pC^cB{FU7Dy>|( zRo)m;5{%O${G`=d7r(e-=mBVzNc@pTTWo_m5-Y#+B}&*KFa-vP+{A4}7C<1nlNDMz z(yl<+F9Yx=LFl9LsLlS0MbWWDDP0*wAQ(VY{1At{q1YnhSl3BY_ItL9IcDJ8sdpCj z;U`~N4Yn#@Jurcm6rU{bn)7<_2~2!JHapCAUNATJ-`Qbl!{x{d7^ah7d|2u-#{81A z;h|QJSLt8t;BVfajY5zH^jX=}Rjhn1zxh-0QM34rA+*f&gU8;SsX)W0re0o#1yXq) zsi&|7sf}oyX?8(+cNwc;iD444(GLoW_p=N`65kynf~wj;@rT0yV|$Ssww<P&aOE@O z3&A%4Gsf_kWo`ai+n67;q(FIoPA=*9ta0n3ppFkikl;(1G)%!5f{8_xF?+H;X~o33 zp7(OJ#r^*C_YI?aa`!3B3KxPw*M+vssw-S)rNv$R7Ow5;k}SUrR#McVUO<(@WAhuT z$BiN#Y^6^}s*5Ee?SRSNy}))re3klFW_$+Y^pU$QnObX6>T@1X7vb;a9170PI}Oym zCxL1=eN3+^%70M^7s$1bQx_wiq;{+(tSJpFZ-??vB`2Pv?lP;4MqcKv3B0%3V39Uy zpf-RQ?#rW|qL&UscnimN)8ng2<b6yatp5&3iANd^&5Q<c4@k%3%r{|AOP#=RjpT4S z&-LN^j#HIurf)%?8NV#uPiB1Fu_r<IPycfgOn&0R{p6=nKw6jK+dgp@LtAL}V*Xwx z7`}7@BeE1YyYB6JeO#w1jl?XiKG)dJM&XSjo;Q;$*3W+CWg`Gf(rMgCi_8}CUS`+! z(So4J9_B6BV8WQ?7}54;Z9@j|QHoFgZB_>y102}uyTN|$DP$-4&>kYI+8?<n6$=zg z!JbX8N!As-)DdHBkMg;nZ8r%=`pkJmt>$v%<N0@IDCG`9E=Uxf8MzQBAAz~T?DBIa zb+Gr8vp_z|+1U8DhAOdU(j6YmJdlX5oDVt;FU$`5p;AFR?_<Pr(EbV|nLEjDs@dLK zCB5e`lFUC+W8t7%@koPH**R0KyK!MJ)v?SlG-&VNXg-=7B6qI?>=Z_&m~O8?kA{~Z zyf=aNBI}ifen`F=(5$~IM!b0Q&&NFQEqkygC`^39r~Ms0wu<!Gmz(H?at7lF1-W5? z8ZuXE#P_zahqMB|7(_O-lHAIv43~?!FX_Iwl1Wx4K_=g$fF$<3FNKOAPK0d9@&@3m zv#GI_`2|R;c{#oS$JpD-6-O2KiVv=eNZ9S_p$OPc1MiG$C-umM;*XKF#a}m;J0LC` zjOB8d+xSIbyAb-m|G_$bo|!B_aNw(fNe&7}lYQ*y8~C((kT=s)=i$=3+cI)Z#pLlR z>r_dlJ<)8;Por5ULVNcR7rZ_+k(r`0$8>M?GFGi63-SRK{4oEItUA0pRdMmM(n75y zGv#-~H=<cZcfSu?Yk#CRf(kjLYF+G(0@7NC$;2<E%4Q8qzr@5x2HPE-05uXZ`{wfh zyZ<YQ&8WG?*q-N0jen6UmHH>e&V^{mj=>FpluW=}_LFCx%`u#hI|DbpwBmk)35cIX z)m^OW&eKxX;q-Q%+O8#<xddg)eCICoMWR9cYS1VlI#vxCdyq>)Z=qiP$RrQvY+RSN zXYFr~DiptVUIVA&ctw13OyWTfvUn*q+P=(4plC(~NX=YdY)P)^#hfrbB*tL%Mh}VA z<mQczvS$7&<H1pLa~bI<wo(}B8mLlZzb(WxXqYfap&eRw#jVdjLxXekfO+6#0`)>d zDfM@}ia7T38fx5>PmWX+S|#mep6qa4bg0MagUJT!d$&U1`dd$$E0=rT!}-gp;Z6pJ zY~3?tj8}^$^u-S{sbdHxeq~Va!rr)5bB`;!Z^pQ5i05J%mX)WuGxN{+;8b4tYm5=+ zeLer7g@(TP%VZ`(YImskMf3cvjSKGcET*xU{mK}67=U5Pn1-yk7XD<JWY5VkDH;6_ z_2gh_6gi@1J+C3V-sVEu2P}f1`(;8--=hjrC*cm&t-)IGKP%<pA<HfGH|AH<ab@2| zg6#Cr_O4+qUrvDTpziS$CCFq^wX*f?JixC4o79NLin50#OYEr<Jzm!($sS{YXG>ts z`-7rqJ<IH@vnm3AtO3yK7GM`OpkfY}9aCV0K}2aDE~WdG2Ekf~)<5v4htO%2yuCi| z7xuE8F!Og~l&-L4vNfAPV3+3*9zGPXOMDr|#H#2Y*yDuNQ;4Wt?<<hdqfo%FQM;Um zpRTs<R;flZCds3dFWf5`@P(vtRy6qwi<6%nxaEq$VnxXezcPUm@K=Y0xp2u0AY8f_ zuOYm#GK<cF373xJeo8)LHcyt;fhQUGOrq4hcAB503r#X3$lG1|W-J3#ljdpG0GVzu z(?HZ*hwgzHFSfcZ<|@{2#JY6&W2X>gaVc6^eLC=C(9Y;3fG-7X?{PQ)^6JoLc$(0r z$~P;de@{a%`<g4Ixeka1Q#ic!@y`i<D%Oj}+^$Xnja>FNv9b3H1@c87|F9?lab2H? zuQ4+G>2DOo?aC^VDKyt*U7BAS0=3@Gh4!cxmnNX7Lxlf&Rp^leGINW-QX*P_q3f`H z^Mp<8A_R1wdnM-mO<n+@Xet)4Dquk_PzqYjk~NU6gGUXMk=YA@>X1zD#<rZB#Xial zRp5-FWsEcr7vPm6Puaph3sgYn%<NrDltoKtdoIts4ZZM*!^>U`gToNQHAli>Gi~JO zq32Z1uS~>BTl-#ha6*i74=qEd1Pt_NzR#5od-vZRAWTN`VBI!U1h~BKxEb=JyWK0i z<~tFzfev`ZAM^I8pN=VZJ%>nxrnTi!QPI13w3ov>hk)end>e=+T<x^@I|Lk2c=Es< z8*-UA;u}T9(GRs?yU|s1>7g4lO;Ubin?GGWYS*5ixkFfTT@-wJqqk?!*v4zti6;P$ z+;EXgVCv<AH{W9O>4c9Gx?SW7zIp59C=&-iJCDX|vkcQ~rR}K$hPRAyNyEOx^;Qj# zZ<_5Lq(?Rk$AB3kX4JM&u^6q_SXJrcjQe1v$?nFbM)74q4wxDmnq~h|>J?JfE~kth zvF^&L4*Pyu`C>R@2QmCKrcSc#$H_%kecP5TTM^lhGT%B)h1A&KGRTjIh`-wpQt3Ra z+-Q2!`?4XRFucd+)O=?f*p_G?us-l4f6y_<377$Jb*oL~(L7rL`L)YlRq}?!Krq)q z^O&srT_#alckSBQln=A7)^`_Nbwi%!AelI6x~HSDMLY{$k8RDC(P?mzby)6OxZPC% zCO;s@EV2j}D}K>Ht4rJ3yBg5f)C{9FbD1Qw^=)@kn-K?E_6n5^S9m0irSjMAS3#J7 z6;?)gX~>1Ox386;LJLz@-u);h%dfaS&=UsM8iRjpK>D2UDc$p%o>BT1;~>-FNM;be zt>?bNVfw!uiJ`GVHeu&gh%Xx@<6$IiOqAE2V}iGYq_1rj>17$?k~b?nIogIs6?CLu zB%G>)rVM-;Bx>NufZ63;9SmU#fKv-PVXOYd_%1*>>rhS63YBXX(3xRGiIKqB4E=PY zxB@nV(f7=u`)6%(q_^*oeSjk&(5;7hY^QDP1tqQGk!hok0Nn8#{NrZQ+c@30vNaeL zSUZ<Psg+n;Oy+U%zU#`;{P~X0BCxMw^MKqS-XXA!+k}E%V=Z+w4aFDo0nh-?6>Ueq z6pvvQlz!365;8PivDJ1A^I0T*JJAdT;}BE7+IJ%yCePO1q#FA~(j7x`fJ^qq(~m48 z`wjz#@sD*YHiRjL(;2N^m>rWLcfJ^9_AKct5pw~EZ)fQWuF{ZPb^V1r$!Avl0`-)* zqmITo{5r`NyH1i%KBHg*X-%d?vTq}cU9U}S?X9W8viMrkYb!Quj#`1`MRV^x;y{V$ z?BJk>0IycB_lwJX%VHSh%raHh$2nEnqo7#GwCRKu8m=02r1afvV-pc~#dTqA9V3K$ zx1JpzFYX9<{+03$Fh}j!3ZJ!zy7}=fD$xG7bckKBYx_rOCmHcasa3^G4XM%6qZF~- zCtvgf+VKU<6{Z5<7rI-V^C>H5e`|XZ9=Wy8&gN>+4&BjzghzzebTxlM6`ZLQl2KNF zwtznQ9I3<q#DV@EXCDerMtBBEYTDc685Vwfk>*b+c>Vou<N901Bsy4WzrR<0*60W^ zAE4)92Bvk)gjnqWjLWvtOuGh!yT4~2k03ci4`Lkd5)iQUW9X3@AlJ*ID1Pu!-;vI{ z)|T?&HfYgOSchwn_#gk0W*AP-afgs~?M^{UgtvKI<cTiHZGQpF6i~d}f{a6!NvHWj zEShRVh2FX$2KB{Eps3l(ZTpt=f@b>ez4~vx%vEkzw|k_tI<(PyHzngd@0kAN3tk;k zGs@5zFHdrwUzHHkOC(Hq9xvbly<SYFac=9k9`V{qtq0xmH7Pb<)KMDUQD1c-;oWn_ zHF9_I1rAHQ6(E7`FOYvIp<3bXDIelOxW&axYk;vA%y|mnhwwzw(vUXvfnZ9gTY610 zq5bq+t~SX1qBl?rv8L$nop>VjCd!t!I$(Y!Xmm40c3BN?IE7vaD4obceW=Qrn1dN4 zce_$3b?>Qi2E72s@w~Tpv$#zfS5_(b;uj^t)li2->kD-a@%=Wr<P@@dOI(Qsp?nB^ zN)mri4e(wO;z}X|)pk)#OjERW2axU>SqW~*S!2WllZ+eh`$X-^i#{ungtku@?tDKP z)aexizwi&R^}`aQtoAtUo-M0xk)&<qK*}yS;Vh1IF}r4BEHL&Q<jGLNX~S*}3U~Mw z*m1+gK24!8*6?*U{&%}LIt6qoM9cObBTFY0-}I8-`4}}~;1q$+-ye)rWWU0J7#!XS z(7!i81UPO5#Ec(vE8^pL;XPP>BJR3D0hKR|VV6Lc4B&pE`21DQul6NeNn}q-Dj(-t zly+;hJ5T`lvo(5|v6+r`6lwE!3xxVBw&e9s5)O-C9?xY_%aIQ%zsMIF|E`@8ui5gN z)gq$dLRP;x!tH&R`^o}u0)nEoN`AgZz>VIiJMdi({6a&~z!gBhhx=%RxMoRn%}`=l zL1sFHsME~4>TY@$W)h)e*1VvpnqXvS&qL1a>MjFk*>FjZ2@{d}g-}<ZmM2`xPy_Vw zY89W~&6jSf00-Gyn2M1*W{nP8YfynQ{ZGM*U$TW2GJWQt^Oe(kw~e6{Ns+da6`fP4 z@8wd)ekQo#kh_o-PPujfvM{*VK(<7ClGtTclwsmNA9f6E{p|}<JvUe)B0g9omtJqO z9~$ni0KjT{0`R7(+E^Lw|I3*gfqxJKxpjc6ZVac`h|U*{bi-1WkNiKXzB;VQzwLW8 z(xOP0fS{BLNW%aLMMb1RN;)Ua7%3vsp@7st327K9osu#_x<)sQ*ci=w@%ugZbHC5= zJ&ygw9~{@NbKmnkKQ#sB6z%Ds$#2<L^n)XcV?WsJGJRI`si;yW=<A=KS1Pnt=l2x| z%Jbeftm<oJzb?WNsvf4J<4pbq3s8ZXu^Oisx$mE<;nUgj&Nq@V(|3bt8;*XyV{jjA z`FxFV?*IsQ#_*ntz@0Y^n)NChGxtC9M0vp7T#D7kq@DUK1d<|Bkd%ALmKV#F!zK59 zbO%9)Jo6UcM53u}c#3x_8#{55@@fN4fY);(_`^$6zUQE~6Y@tPE~ybOo4SU*2TtU0 zP{)_*9`j{6mZ)sEahZlpU>zIw<dL8#>ef1ggz2-UUccHLR4q7FJHwcWU7am+7!$~> zj$GR!2}s(!4}JLh6>elQ=YGtSV#qTJAQwust8=yWA`S-96hf&Z!*iNSMT?bo*sM>G z9d1uh!EP=~Cuc9hF3LLyf@Suah?`#FbxVX6MmH2)Twaw3bZhc+to@KWki1y3KCo7% zZrBPjyf)jSxgmDg(&bAvRuDM_^`3vJVKgY9<EKeJ1O=I;1=GPXkx?#6g(MD~GHkG^ zqets64j%@+aiCxOPV_qe;^MSQ!S0P|;t1>ZxV?s3@!EI1+jC#Q9}&KQ?Ppc!laZ?_ zkG^RKY!>KlM^SYny<BeX9rxHy)_zH|;7Vgh^brU%cfK=;Zh-cY&kaxf?R;4zXwPY^ zWb!_3H)E`-aJIgo7TyWD?>!;qmB@QS89>*2AH)$1d9E`icmt{!GHDV##j#AA0OfkP z<<)#P`+ihM&!cw&u)2Ho3s^U`k?|J|?pM$z_}k^I$&o8;>B99%Kd2YTB@HGcBd0_m zK1Mp0VXTP8l8;V2g>GG|H_!$T7!IpG_&F%pDJbt;_t%qqHknh|seG^TmYo;7on<K5 zvPd}`@HigCk;n<@1pURgfGS8(p`U{!Mg(;c<M@jJII*DN^xN#4YJp*~SmK*Kmsj*; z#rAl*a@HmIkRD05V|`lrJ({3f8ZwVwq)Sz)SBzXM=K=3jdY_|hMphD!Pt<}5df!=* z?9)adXSOTJ<AhhAPHC4IoebVevAdn`m5sr>LW2BYpMo0cg?%0K-!hXEr@(j+W+AE; zFyt^H``Ur_gClPLwK{-ZGI6{I=SBm@Re$BG#k1MgW1qhl^%u2J3ZMh;4m)Le{#QZb z{2gounsjsHCw<O;(%RpUl0;%61ho*Hp`ff*<DWl}^x?2y2R!zm&-M7#wg6ffIC|OI zBZ3&yY1tpOjLG!}(TLLV<vY*cdO4k2#hFY#-VLZ$&1&OJXbazPJg_DKSSh#oin19? zZ?|+gseV=_?^FY`Fp03R40p(2)2odIoCP@$IWPaFAU^o<EVhty8ytEHu+2B#R-@}{ zOc86xu00IDZNF$?Klc0|rwbmm*p8_o^LoQV174k)=!=sV<uz~EeQ?5?0b#TGSoHz_ z_%!MZeKy9xD7^o00VZgeP2K$OYr1HquDPWQoeSc5{f`rQd<7^&^?@!=ZD3dPy<I3^ zDf|p+|HeBZgP!}WbS?4$g0=Kn|94BIlT>?0CnTJt6fJO8Z8gyCN1X^x0BlC=qMdq^ zCPFP65BwVfZ1^?^w%DwbmHxI<bITzj8PELj(FT`In(xr%MYR(+r7)96pkz7B!dInN zyO$u>?M_rlcClN}t5fJ5@E924`&xLU&HPz}VB!;Bw%dHe_*Oh1`!ZE&YU;q%5ZoJJ zAvUjg0sPS2<~wZbVmnTlrfzmvx|bW28*ll*Bs)Pyz1TP{v<vqT68I>GN$BMs0;o*D zhOP+9dOY(6%V+^8oR#FqHh;^Ew@C#i5w-~301_!CQkyrXl1gev=1Q-xNeD9ACBoAI zXUqiKtFYPu1BN}Brw%xOsu$|Gi3!e%WhHWfDdnRJAh^U+yNGu7>wT?kQ|3FydiX3r zz63_8@IWy{KhVv{rqr8fsj~<FkCe?D-n^qQUO4IN1pXr?Ss(WaV9@+K>%)JM%dtFi z_DKEt`irxil=ib64om*^>D~o0J|m|6HnNV%Z*AW?$ORFTG^+pZ#IvvO$?M1z`3%n* z#^0djLsir90{_AgD41CTI}3CYVs4o<Ik~KVQrqkI;J5uFTf3(A9|-|8Zyo=HmSA_; z4l@++7%SN~Lv5_96^4ePwhI=UUYid2#oE&3zL7gN18`Pkt@n#x3%%;?(-5RU;tyT6 zhkV<6C&D2Rwg#^WAf+V!;tl9^ff)MFn~fA8yu>0(Fn)6-^YA*|0RX(1tdRmg4>J{l z+mf0vIc0#hBa|0by+N^8kaOTd^-V)Et}~<3)u~8$lOXI8c!5S<#KY9}<c?&H6=-2E zBc_0_2<487-8Dj-6QQ4-Q;p}m16+)D0|1LaM<BBSvGc=I`+wzM&V@kn8ff(-Mik_) z(z$)6GZDi3F~~eAHac=FvmP~choj3)Nw&C%(DNDO;ZX}@Dz%13ywFHAQ{1|JCTA)M z-@xn-@YJ;2?%!ZHg4ib=KO-PE2-IfNpaqeOzML#UB$fEhWpE?yU@6i(I<Emsuav6z zsi!AXUjp^-n65o>!CVBI6sEYcYcKpxx<@4gCW=vhIhX4Ln14UGx9?|%>~N9v6~I$` zcl!K*w3`1$+YrN7aNxHkg{_IVVu$no<Jw!Fsu9N{t1RebRs8;yF#%tpu-V+hyUWZQ z2D~2D3CGPHzUrX2`7Av90eZlf8OpHiswtE-cnm1c*95jr>#eoc{CeQ@(!y#-rUQ&s zb{EfZ^IugAGsOM$J`B8LdUDcr%}Oz*^<9B6ozLCI<@*^fhSJvZdpcJix1^vgeC)=J zwJ26ysUQ)OYUN(puGPBoTPU07mr;ak?}p|o`~{pO0~^L&&fF#VS-j_i0O|nCaAJp8 z6sVl2sr8vRp2~WtV(1+s0jGDbG9P{;e7eNMF=$bxI{k{AV_!0BH<)Cr0QpXy^eRZ# za9V514y4TlWBYvLLS>E1I9ujY%q7^uwt!Bn{!Pl#ndHmo#m1G-$SYmW#OT`dccxq{ zYWt>0FgYRfTK<1K6mFmr245RtlO3o*c~u|Es(c<{8ILcDYo;8#i|K7u{XRAujmcEu z`B!(C-Z*BuFb6_{6stO4+M{Aux8A^#NLW;vfh|IO!TxpQF-C!l5=Rf##{OC<b1Dve zn}`4SwDk7KbN&fKJnboxmh_vS^%H0r{+#YW0;FdE>-+S&mIwZmO_JSYPwY)@ALlqF z5tA~m;`#FvaViDe0n<;}VPZE2_zjGlvOoxr=rFN=yfW3>YfZ*4uG2<vhjw16H)Do^ zquH3AOhWV>QVeN&2+Ui(c^<rf!=O&~Xb3UXoKg#$H>NWGPMoqoWX&H*7rdWM!^oam zUC|cXkbZj|3l4c?<ks|AXFXx+`5~zj!C=1Uj`<lBjiiLUU}AS>6?xEbe$cn6O~kPc z*!8hJ0W7Tc21aXc_yNXLPe{H7ve#PQp*&$4>lDWSM!V+)c=DN_AMY>N&_|O6`1~$8 z1QuHC)sxn#Wt#;~34IUYW#JGlqS@SDL`nn6(i?We11d_<f~9)|DN1Eew+HQO0m=!R z1bF;3_hQ<8++M|Ibr$zpBn(yigGUr22X1+%WDgnCmR+NkMszq6#g0xtb-Er%|0f&q zRZu*roP=X+POXo1g)y}qHpOhw$U%kt4QQ-Xm9{85;k6EH{K*;1DHC_Qm@5`UfD8Aw zNh{gc_#=~f(S$sS+q4xS1SGT|PBAzgaqM;RJ~L;KqP_-vzwQhly;U!1e#$&WVnWS9 zfJq==vIPX2H#<j5re|`WkHfifhQ8?)t@uX?p9#os-#nCEUj>}=s9xNE1Yp0%@D8-M zps@`N`i-}!7|D(gfwb*hU$J;obJuD?f9r_#x@iI4TjoA3L&}D;B=KE4ft7dS)klI` z>A0+(V4hW@$7qA2CjH3eMO`Mn6p|<2<1PXL`bfurWSe!6_T%BCd%MRz<7t|y+i{)m zzFdjPK@4GglZ&5X1h<l_<8~Qqzs16vwQK;=r_m}?MQYavQSq<mC8qX5b8clvtwsnx zqr!|e2)BF8XqwpZWS8q)1ope6@WEk9uHXf(QZ{CmcN@Ar$Lji$DAtY9{)RuCFjj1k zxh6wNaO%P5LG}=;eXp1Dx_b!&zsV{N_KS~fIaMH7^#bYhqM@`iN|0Qc>2))FU9F^K zPGc>l4ALz4RTJjM0Ta;<fF}$S!~}F3Gk7|ug`FWU3u=VhuZeoujyrt`$kr)TZ~4e0 zT8aOa=UvPuDd3DwHe|>?LlYb$=)mTP-9!hKLeKSA=X;vRel0-t>TS$nDO-ZDq;W5x zac*FANQJ#u33D@a95|eCo)qz(vvx7UTSl5b1VU6zf88hi%|0oGF{D*gv1KsR&9kHc z@((3Gp6_C4R%LD~q6M7}jU;LUuru66xUkZvg0(<7v<ua5->n!ndYPo>qvH4Gz3ZO> z+A{H-B;CkPT)Gb+McL_^lKaW*>%5Z+uuS~#{_7i+$n)Jt$4fi#5<$1B$6h_m+SuSY z5Ju2l&-Pe!+qL_l3!Rb_%upF|10Hc-_XqyZD*Oi|$)bBdNPLJ`ZAZ<v)kWm&?b3tC z{Tbw~yo&C+I#&tYIR18q=RXki{t;*dA881BUSLjfX;fyW;Efql{YPSXPXj)4iUzJx zvcEqyAP};${ZRSyg@4w=FP}_uI|AWm<~vp6&ZW=E=vNm9@!4Fjyoh{@`em8+lBNQ1 zBTv%xpohQry7caa$~VbKo-YXthcE{8ZKVz4agN;MsLppS>VV`Q_nb)Ac#2WSmdWvb zm;!qQ;&Mijb|&S^d?mryR%+1!eSoElGQhJ(Gvz+99Rm#A`Y&x}(ZA3S0@I8}|37y) zZKCIR-BYRN`3Lao!I+=2A)t9{$Nv6&1frPc$DIfBao2$H{V6-F?YH1%VrW7A9sj>` zq(b|M<&<y4iO;PEHSWj$W)krgVj+8`Aa?tFualCfHk&4@LnGed{Uw~N?$?}41Iy>H z!|E2oIA&~UHM@^&Al{yR!Bh2}PzSA1eXUx>dS^H3BF`NcQ}u(dBgBNicwm9_AF^O1 z=Z?uFHLYasD`W4&YCDf$z8XKoP1K!&aM=$jIswOp`nmW0f!4>pD-=D9%1hFP#P_8u znfA7-&h&a%-USsj*vK_`x?KuCGHt4H7<;jq)`DxwmD4&i(VBTaa3|$>fA#Lx)WXv9 zwZv)FDyS)tfgCC(x|8g_`;uD5u9u%F<L6?{MiBo?3x`AFzmi6q@|)szsWY|8043|3 zgO$sMzor$ig=>oO)7U!&*HdY&Wz5Xol*aj<Uu@-Ftt#MTHgKXW(`Wx6N;VhX@4gXq zw|}Cf%#?4s{t`~q_uMwqsKoi7e;0R_4mF&2yAS&0H}t%C4}Rz4Xd94PR1Cfu%BS_( z7Ku)L9#6`O_t9dMa_Uv;z0%8`3t|v$126@P-T?_X_zX=Pf?vbKM|mC$wz1^1yB(^$ zd^1>nTP_#S4)#r9wLyYJ>_&J`3+k<QeOX)3w4W<_<Wr8jz6H6+?n9hnfRY|N4Ybe9 zmw9m*^pl$3_k{&<;S;0EpHl|dwY{n_lDT65q!x=^&C|ZDkt@SH<Hxq#Dn!dTgjSEr zNCcl6G4mrlY|`toY06IBEn7V#JE9qUbvr*bLC|5Z9WW1{4y}DoU~bK@Kuh|bt77*p zbe#~3|GCG|x4_gLy|*sP(1zFEw+-#qx;Y)FIMvUdrWe!wJc&F3761qK?ZCb4_!m6s ziVguU{dHiDcGw72=$3E91U&?(6;vY0kFCP3t@~!GFx0R2#3R{ceJUMZ<NdKhrjMG1 z;+Gwn%eFWGeoU0(#`LIC7NwC~9lRv`N1zkHL8?gS=9|MGG>ODdKRg1S<sdatX3j%a z;1{48wqQ(Fi~PoluiZQv8FPtf(>DQpB!A6H28z=eJ>ro58Ro?{GsdF!w%}BNh>zrG z><mRt-RaALH3~!A_oW<lb*uaNuG1pI<3dY+xWW^9N$}G3p&iBC*V8Eb^`WRUK;KtV zo8v0jkBcuDwG|%SNKSUOm865Q9e>+}(Kc%_Ct4j(7)j36SzA0JUps_#3Y|t%*|<v- zj_dvJ2I^W?p`A&19bV4hxL1j-#?3Ko5xESKz?m(-Qwviln1culQ#d-m?M|qcusV^; ztAm-IMe2){&8eA?S;qZ&8DE@D<6Lm?y#q26&GL((!f27V_}&;<9TSLrjh3LK^5~7R z%r)%$rstczJOKN?^RCc_FYafCkY&$*v22_dOO(wd{D|xtEs=K)rd#jTSv<rKcnKgo z0lWPAHG9KtT`W@Z0JI~Z+0PBEe&lU^ZDzgR*yN-+-M=z${b_}U81@-XZyXQX?<4lb zY>4}-@@wCC7u`7ix-832$}o*fS=6no7L~(XmuXm3p+UurFs3NC=WGgnW9;O~?>KjA z=YwWI|Fpw#mCg2Rc%)fW$j#xVeA00Y8XGUGRG!H5(D}=qc!p~2B;iJzy$ci>^<VT2 z+G}C+!9~K6#U%AF>Z(UCx<-(xoeV|d#Ya3xLTCen@>g$~a&ZA{2mM#1HdOMB?Kwcl zY+1nL{_-~Xs0Dl*{8925NJh#_&a)((eLPAr&u~u?*b)2;$^X?4DEsyh+VL1|;iDE< z=1nJ5sVVb$(P-3}33F&(z31C96hZU5nXh?ZazBFtA*K2W!=V=R1e?P1t*NRaM(5^E z9Ls5S2`r_)(yfajvs&%c7N1v)>GgC~?WfM>Q7=)FmH{SqZ#U`z`Qv&~YVU&1tX*IL zi@0iF@?m2ML_mto-Yy1lPPr-eh^XEO96Knx*@QNmQ9JmAOS{wOHH=lM64rDvq45HC zDBiE5@w)?ll~LZU)@A#|De=)U-z+jWF|QyplP@GT>h*Sr{DrfGO@~f;Q@&GKE3tXo z#~Y`R_bg*opgebf{l=72@Gx(c61ifR^~>+3Wke>4{!ksOp0_F6h^e>($FO68n*>jP zW|&+)XK+zaT?bMX+A$6l)j;-BJ6X%6$@foI6G5{R?o=E`Y_tn_leR|upNV%7ph33S z9`Z(?X$v*OwfPWJ^Yh#8wn3cv)Oc)ND7S~Xda6!Db3kO>A8Lg+l;ALbVLXn5%6r9T zTL$PEe@rQ{S&k>n5mCYl`SW|biy?3z<-p9*>Dh%!SCwu_lv4;3%AjxrU=3?Y{>nz# zUMfvzQ#a1n-eQx9JCx3l2kfw{P?#d-%HOnJz|tYo<v10=inqLbdC|bkZnZV?k1_2v z!a8V0=8b3R6P#aN9{p30bg?#W7G@#Pd}G2Q0(TDr5UcY+!-|oEF#z*&E-<~|ZO>X( zK1Y~S72vFFNP{H1TE5eKTds5eW+2~9akJFb+@rLN%B|0Bf$^zJvT~m{a;lO}<Ewv@ zExyTY#ixhzCqu8{m}{}sd)DZfy-fOL#5Nufy=Anzke|ZZ7x0L=p@n_Gh=n_@Qwp`^ zQf#~QY3=^t>nReXvJpZ$iQP3ri&FMlhnU4EU5z%HRx)J*SdsF5lW|l0`FBvFC6-i% zW|ej_ghEyODN_(p9Z|tE6X%R^II6L>yWBL_5tEj>E_Ypd`sGM*ud%P&t;Uq>?MyAc z9b$i0xSzdUZQ@^8qrKYy7gOD<B`k0pzWeiwdt_R%%$>?T0r^7x!;GWvocU0@ar4bT z{Igf@{D;$w)|1tlVRIM5({Q+?s6({9GgR%4(*XG(1@qXx{{UbV^SFifT!xH;;EzSn zvxe8?BsP^CxkOV@cIf=<KBqAx-p)<180!lh$QHhtVfK_lZcNeL1O%9y(G5sHInw#@ zA&-zP;<w`iWVRWcK06zfK8Aakp-=SY8Py^n96OEL(8IqwvKj?s91_d&`BE|2n;Pyh zo#9%VCqj7xhW1IQ$7Cgfy5QU<ZfBIfB|$aN%ik!mt<0y}Mu&+c4J)u5Yb0U<@5&D6 z0GZ3W49*aA%ak-(Btc6?l^P$<GM%{(jC9Jz_?2<)hr`{fBuki$*@<Gmzp=lON9+DT zvz6iWzB%=g0;lD=(T(45&nY`|An_&rhuO8x2#EBpP*x3l-t6vA%BjV!)-p>gr9~v6 zPoD8?`}1zh(sYZb2D>4j(PlqwT|!_ieM!O?yM@q+6UkQn86~m`zg$%RKkUn%2lD{8 zy8COe^2rN3Bg%%+5seZ<q3Wu((7`!!bRzzO#2?;t=|6a&Cwt?-5IFBth(7+t#1MBH zSIs=3YOVE5UhTo&L3pEWK+#<%XC1E3Kk@kNGE6{^sy?|s(k6RbrCle4*gZK_y~x=& zwcq)`T3O2Ib};H~H)>V%8e$B&?H@XEb+z3{SyrN_vvxx0{5ZWMGq#dZ#2As15~z>= z5}%2ovy_Z9=*@en1I`Kkuf`0dGaPZ|DKXZAxgG0Kx72v-Q)2jzLTD-Iz=MgyncvN6 zAGChHnG>Z&{NsjN*ZccwzqGbwy=IFlkW;YIE5j|{xeMJKSLB%-42^BX!FCT}x)`4N z<AY!VLpPn)uDB`V4e<@>F}lg-*Ub*F-cTowtpgc4NrA2LZnV^zWt76A_@Wp8ot(>z zli9vRFH<a-WV!M3LMCO+ZKks2KL_C$L=ngm1fCch5;BvKtMDY0`TOV%Uf@s~oVf?K zpB%)wOxXsfDiBcB_QI?>q5C&)GCPX-zP(XY^Db@<GCKjsgJ6AzQm0h?55PCy+_@i0 z@}hep6Mw$Kk~iQx*oyG!pSG?^snG-GMl{|84_MLS^|v2%?TxR(b!V%j8o7X(ircEr zdh4RXs*h~Ya7trqr$<lFfm_c|%tkvx0F?#nw7pVX{ItB_m-H@U88<a(Xv3X4*8J`& ze(&yS8B?LZH})rDsNxm>Cs~kzuBn|{e6WAaSmXK}{UlDNHl)L!U$^m*z_=nC#=JkK zdv;o0W}mBz>kMt(+BGNpi|EAKiI+W4k(y?(_F*`Q#@p&92Ppf&Ey>iZfPg(;Tz^{^ zTujAPuf1!?jlB+n!-UIy<h>ibqQw1)%fZk_Xc<Bv)ASA<zM3|MC&{~haaonrWPUP8 zH(oMVox=D61l1<0-gmq8hl4MILvcAF1W`T<c{WDz@qtGavR&%o)8(Kb0p_eFPcmRT zCv^LsBIg4;@ez}$)#NH@NjTLHO<BXGhKIAB?OlfckgoT$E>igDxT^=C7bzq=Ewdlm z<4Q*Bc7H&5x9(lAFtS&f68Wa32m65$O3?X`wTd&=@hC?MoB6GRiGeY}c&?QCp+>_^ z<eor5=#eqE*I?+K^+xrvW7w`ip_MFyetVbZd;KwK2(r>L=9GPJoG^Brx%mdiUn{!T zE3Z;j73;bJx|ld#3WLJ^Hq=e)08v7mOIzA~Q1a^J3({I5UL%xr!8y`RPqEvz#EfJs zL0`Aab{V#7Jy}eu*tDG#;O_~R+x@lD5>&<9z%?sYvGA}<gZ#xmXX4Hv=%6*2WOlYr zWY2|&P?<?tO47Q;*ieK3tpc9b-F{B$K^WaDX-VHp>N|t)a`W4Z-mEUD6rcLB1iofg zdd!Ls?XSb`f{P@gjdIXCbv6wOmp{-TV}8$^{`EsO3Y7!6RotM!$!MWEPe*v$p{}#? zq2O@s*g+O1=qEWOTis82SNrrOS`t&w3vu5b`ZoI3tE)3>>)@2`OlZ@~PQ<QI_a*CZ zLv1g(UguT7MR8xWzkJRBZ=j4u?_d1?o+*4ykR&;c`nONghS0_@21auX>hgd42wI*^ zJ^(#Tp$;Mwn6`9Yk5MFJoYmy$`fyRn8p~V`6(B3wbfNut_<>u~7WVZPJUUJDIFH^9 zldR`e=Xtx3%lm2HUBC671qupHyuD{9SYKN`Im*Sz^GqkPx+bfw^{Dg2-E%leUSBV2 zICML(g-*A4;n%^`SHIBJ7@gpCS$sr;gAsOsDV6B5sE!k>vZ&VgVdPA=Utx7`swl@z zeOyy^d&iBt`RD48SfcQdA^*SwcU?uRq@|mvm?L(Lok5s}sEsWPeukB_a4)u;6{t#6 zLb;=kPissYXS!V)15Nimej0`bp85D|@6I(TvF0@jb0-Q+NTs?x4%J^Bga0ikHkgJ2 z7;8kRTL1VXiQz__iY(Yg-epx&&CzCk7d71;VoqckJy%z}M8%jiSz`nZTmGJ=blKPj z)9sTx&m@zJ-c5qP+Q>RhlH^yv7-~j$yZsF<Ic_10wW4S9WOB6_?H)?Sp5*yeIm`+~ z>I?-vA-MaD$AJEJK|Uz&85O9&AS@kkuzU}mI7;5InOR<k^4Vd>()Yz0q^{*{e+a`j zFZSN=)O_Mq0df@mBzhh?)qh~@PL)DlEZ_KcAAe%%=;miF{IIxv#=ytqO*UIem{wLV z0l?-YNX@E$A=Kk(ol4)Y?-@hKeVe8RH4WIh$LIzzq*R9`Zc9J{`S?4vU>WkkZSkLJ zkBQeSs!U1*w?6UO7{dIycUNk=7`<+RP0)ctfiIHm=MOt7l0sTEFgiq4x}ZF!P9pOW za-m;lC)wK*ejRsEhqKhCa>olX-xDXj8@HKQgB%UyWvj7GJT3{%-p(Hisb?`@5(^*p zz^3sq{hI`jtFv4k6KBc5?ptUY*afl8g*Zil6NbIfvmIr25MqIRD0qF!qy=b*V#PYP z@CLJ$A4t7d;e2|d4Z5I|C`*90pMNkcsn@5_%mqsOdX-q;Wvu6MyCxICN5L2r^Xy(V zYikQ%OTjoM@V%or<J!&itUdlSPNu_R`;Y518$Jc&#ekC+O%6uv;)SrrPA@)sii-9F zi$Y#19($HYsSe4yp9cM0_^+ttXkUH&9;tGy?Ko1V;V!wN-o(Sy_BX^aD7`1mebx0| zSA7!=NnRr-Mggj*E!2SOzN~!4op1kpCTxR90|g*?Q8TyGh#(?+u1B2{e7_0>Ga7rQ z-MFCqb6HYPyW&{Z`={B=INlM=>h!?eT!}5?>dFKkWYPc#oI=g#eu?NG;#IIQOv}?m z77Y9+anH$35Ts(kLL;qD<|t6&z{kKw4I;W*?(w{rcEeT|Sl^&Bs?pz_18eS5$xxH8 zr)xgO()TQk1+&MpRJ}_?9WL`5*E8-Re%`4%viCue(_jyn5)@2gK!?Rrp{I}P1FXzB zj#QUt82i0g3A#^|H@Bj&b3WO@;=4saR;vkMpG@XE)5|r`l)r?^5}(!>h*@<k{FLec z^a%BbaTLf;5ffxJe4>anN@RhHfUO#xR_?yJ%CfO4W%u#Op<9E8=4g)yo%qG9V3rHV zWbt<mZA?2LS?E8QTAVc3K!}_-KRLV2wysO()8;JwCj)5-{O`VvXMhgM_u1)s%<C6c z$3+9*;#FBn4_bq{lMOV#iIIySDDv1CkH3vs4(>;ajLq{91{MZ(J|ix6E95WI!OOZ_ ztQ92;FDmxyY*sZB7(r)w?<14?X9Yfv_cBwH8cn*SY=?C;MJtO^!S)dJivy>xR!vl{ z_`vsroQ8kjpe+&ctArs+q(>wZ-%Z}~dThO6<D={f@3gY`wk6*Blbf8j{rbm(zz6kb z*@n^9Py0;fvG%qx?bD~*LMuw_hGywH-aA8cI-0_mrf1DC*IXUpEDzPq#kcZ)NNmTb zP@Od?#}^?Y@NzG=&g;U&HO_&K@3xdn_^cl?a1CpM0Y^3g)vbMi4|dxD{S`XEAn^;b zM>TNwW5gyO_RHX}G%wMgm2Py|Mk5Pe#sYCws&$nzAHb6)rO)e?L9R&28;-$8!1KGL z^OeD~gbX6La*3`_H4>i*e1tENEL23tDv|Xt@oQ?)b`bR>bHlwLy_?=i(!3&~>ShV_ zi}5!(JygMp<bg`#rT8w-*el5&O@o!@F->~Zl+0Xvh_?iIULNPM1F^_@ADN42@LI9` zhkwg#bPJ02R92&syaFTEAGVzFRqR1cv}trOT$q1hiHyCRs+)lON7YmTRBboYI9Ija znfPOW?;ATMcU0d<hHjLCRF_*(980&qc!50TgfLcBQ~v*y*Jn^SBS-?q6+mo)oMYg( z;*F)=$i2r!8AMh@+CTA-Y}wNJg$ji9Xnn0#W%?CCtce64LAkBuP10PwbkVQov$sow zf<I^dh&lNW7ocDN)B4LDxY$jJR)Q?s`i&E#z#O*mc-A;QWd2gOMXK)XUQ36o|K?Y9 zVM2HwK8f9II*9!Xi!IKU>F>d<_&x)U{CDX6<@dzLTRS@E$u+CP*vA@iZ|#gD{<I!E zZ-y>hnD~LsW$ugMvpbw2<ZO3(cTLylP}7I8R2?!J+$f{7z@PrQ>&%l&hMo>PevLAf z2bQ;M0M38^qp2E?SJ7rB`O-T9l}$e{ZH;B>ph$ZzsE7+M?#a~BkPM^;9rZgg>U9Rr z5u9Q0Wc!47tDclJd3H<C)ceTU;LsxtcK%wkzUc7H2uYj|WM|rT4*PCUOMcm#QGTw% ztNPGq#CfQh64R9R)G?zc<Bz_d%H7EuV7d46Et^>vXE<ScQ{Vl%z1aS)CtrGGcL9b< zC@l4lwZG>JPjE?~A@_S#4*lEN;fi3E+jx0aV%kbyDa5}YWYSnvpIH%uia@^vrlAY! zl8OG&puj0e_mu9RLD(w>dt=scMPI>ghLD&W=aHT+{a|lKTX}Nlcmi%h#vs#ThP@8+ z0r@<%K5^yP`gvI7PstykoFb2*i7zM5<+(gKPUndwse%%VgF|N^caH`}AdL{4%{zv; zYG%26P@W>@$x{B!81Zo!qTNMgs{R4ePPZ|l!|BOx@IKeT8o;#Q-CO{79jFs@8>e<7 z8#rj7Bb_=l2lnWu-Js~Yo6rw5kNs->uFg=9PyJ~2^W+uH80hILxH~VJ_G3bz5dIy# zmddF&Op4Kpb*XqFstVTpOcHX>nr+wi%px^FQ2g$$gQh1Q*H^;C<Q?F3jT8FpNJQx4 zXz=QT8EGu{N^WiA-~;2-5HFth9&36_=@L~`CV^7ZeqS8T6f86YR2D2GcacgAPoGQK zdb<;T(Z$PjW!r3fE?kPXJRcfJ*hBCJJU*H0g%N_kdT{)0)%nmz>YKL1B(ySqGJTk{ zJ*Pfv=)t;(laYQALJehI#-pX1cdT^q$Te!T(mL5A)0o?1k*1VWIg(9pR=cp<q3nj= ztg1a?FU`Mrf@i6L{Wkt@FMR#UGJOY(xBMq+XNWm&QYm?Vj(Njq=ru5v%^2hfTLiCD zb~4TaGnecsv;=X^2O(B<<}SrIcwEbc9YQ#6{0JhT4je4}7zg}?zoKJ&kp78^H`+%K z2JJGc{+iAwTaEC2MZ-<To;ClYL`$ut@*OD=v>aQPbmgc-%`$Xwi<ZQI@2*xr4y%gf z%y0rqb%|q1$<8#~3!hZvQt>cqJ9jIJgk!Tfc+5<azGtZl&)DHNuRVdEEbU?4CvV^J z>43Y`@E*DM#KpmrGUHoAXvTWG0O&cy;~DGkVjGiTo`V+R;%2Yng40LCewdxO6|7tX zIAHucO4dW%M`fo6U5t{=$P~-+TAj2%*55>9O)dZMeJbBW%upM`^p$twu@QbN{d!Kp zYK=9nU0XFnxJxxbmF6?UvnTe=d9yzPnAiP;qqdEc&+|W<8<(o~|GpO}Q(!WEe9{el zIpmJ}OA)DPu|XFSBLd<q=J!xA6CgKPEcq%LgX0;;fg?prw`G{Bn0r3A!Cal<#mL@R zfkyZQkeA&>#cT5@u6_1>>9w#GH{m~ThklC@ELee#1X^+r)BSO)X-FGUD*pMU-a7Tu zr;ECs9ELD{j=4W9n#U4nEhLQ6lMG{fVvE6!<E~>CgpJilO}1Uzx;sa^h{PET+eXlY z@K*hc-R`O1P-y@|qxCHsko9$T?*Mw-apoKNv37YM<+99j;K7*#>-^-R4r(}NQ~g9G z@e&Vh=Wgt(cr&#2pqKw;SipIg|NE1KaalX);1#A>v`eW#7hFw&un(*S+QiI{b|SzP zU1fstC%l}so2{+d;Rdq|X{@E)QUoq8!r1RpBEs;0x~d69@=TB`F`t71IIk{_Dv$K| z=Sojo{p7xY#_h}!ebk3enb}}L)TWCGhhSmoClgf~M>^lMjRy6jzU8+AmzYV?4<#VI zDpplb$SA@6J~e0AP{O1hc9YN2%x-U}vWX39>H2GW0R*cc+K6x>;Yq}5^e7K&6ytqW z;*TpA;Vq^qBnV}UIdk!^2*8--liHlxYql_UJGWupyY<mZtOU2fDjKGTr*=XAITi!> zf*^xSW;S^+LnEWEp@<Kc+<+$H$w0^)L*cjzn5>T&11EVfIP@)qbGg5VPz36XM6O5A z(QNM{)|mJsG(quedH4oDTpf%XUVn}rvi&oBO1u2y7q&qL5k8TPFLzWj9zQGHL?2)& zPnefoyIco<)oJjpg}lLg5!^|Av7O&IH4c{&i~pS}uG<)`_<q<6<Mx!Al29e6M3buK zoix+oO_&i(wsDOJQh1o?kn7dVHc7Uho<@!cDxr^&SXkh%h_-at5z&mLm_37?MD#k3 zzR;N6*s03er#rQF&20-PKHo@Zl{sseA&^;{a}*3;jjnQDo#DRNfO-d&1k7DL1`22; z&5L?><=z{#;r=vi{yAhvt-tKZ-x4);5^>zHMoz7rvad7>)##(a2C1SF8}Cw6+Vv;o zZ^R*{Oy;I~4`Ptsa}0-<PMHb9b9JNBkVYFJc-Gxd#Xhp{^mzkkA_|-?9hVz&z01$; z<L~W#q)h{SxmZYV6OkGus_|-&AN^4@FgdRMamLds9VY~zj2^W-qyVvO2YQP}i#Tje zte~=Jc-Lab(Um6Dj}y~njl%f54t={%hXXJo1u!~amV8Mm)P@%@Y7<1YlL37u(mfq{ ziVP3PKC*5xcR-o>H9Y$DcYaBY!p7-a6nmjRY~W#o*HaAJs9X$%Pk;(;>-q7wWo6f{ zVHD8Y*3B46qWi@+ZrRo6U2&~day<r?GJYRgK#wU)1K){BFolRz$-Dx)rPCP#x-QFZ zEF9NhQwun-q$I48G+%A643A%+`<i92M(<u1!)x2-WJ0Sow*1tk@2hs2%6QRzY$yop zah?z*8D^ZS7|{1oS=8$Km1Th;1Ze7!AJ-c^4b83(zaR%BZ-v>$Xv+pDMH#-8xYzOY zYKkrL#jrczKhZYOA5z^y1)}sGD6FETkCUwPi0m&qRo66nwZ19MQW3)%GP!%qWlfgy zBbRibADG}Y$XEGf-!-wUGAKXw#zovrBLb5wep&H|cO{nElp{5gL%YO{;>GThWBpFX zNlvmW6%Or<8nq;h<~hecZ@seCZp?WJVEO9U9xlzjTk*K^orkuYk*lh!$#_QM<t^}B z@88w84*&k|*PmX7fSY6UMv8d70*BKLjEbSGoDY{s|CbP{%I%@l%?k1E-u*b{X|6pN z6)z02qei5638I`PA>_K89`EyG_dCn~DW|D_?<A)W_{)q8AI9UMpL+Q(67fDR*Ui1^ zEdN_fKW7Q8@F>u!-|(Xj<hKa*&(+JGGk`Abu)Vb4Hyaa({hHrgTxUM{nEV{;`zj-9 z{-wGulaesEzo7o0Rz#7k(pV~mXpbA0n7-m#9;JZkk1BpGX{+7LCiKp19E0LPBYk^T z6~FzxX&J5x4`)u$>{bAOfRD>ozaaYbpn=VAU1>D=CEQQu1a*p3LUSz*>4$GRtfcsA z$oYP+B7xiiZA%hh#*R9z`iOl{N5>k$rSL~w?A;98qnSnQ%g{J>{e`&VDhQX@PaQ8= z8)bDB%y&*f*&!Ln^coeIYN7Stv+j0k0q?P`_HGohfS(9Vk9$r3Q)Fr7g-0XXqoP~j zBbCV~Z^d2V_v5$ynS!2`@3~Z3AaeeYF4~<uZHLQHlZpO#O<8&?DEtSAZ5oVkgkhVK zjqkJsncY4eY3eKLchhmtw3;4z%lBvWL!|6OV`chrfsgc2oU=W*LU8UBf6;k6Rx+IF zZ((Acn?L3XBq!_mEyUl{SX~Yes1^5(u^CGrdzDg)z+UA}H(+u~SZyhco;>x9-7rM@ zJjJQkLgN0!C9UC+7elLN;6C@)3P83iET2(D7fEbviicH;@@}tecT$H^lA~Z-Goa1^ ztE9K(EZze!-7J+s0tswMgA5_Zmt4Bn0{vP;oVtCtLS(E|=a-r-45(%+iE|20fCQG3 zl{B|gjU>5~<y%SOCa!Y_+lS3IlU}{I60P<`H?Nb9P9mC*BUT_+N01~{xr4Yq1Do_J znRNwfACBAyQWv^P-I~&UFE|?~kmvsHbB^NMSvF@}58bq>3Td^wiQTFGx#hf8K8c20 zUvm=o1OQ}#%a~>|kQ6f^;zKAm#*s!Z$1TaN=5ZP(A!A7NP{kwmKz>!b;q2aICQ+1f z{Y0;em;b2E-y&Ac<6q@xj+KW(Yj^~oF|gZ<UnN(tM)y!FJ~t=cjt9P=U*-Fi5ha-) zj6}r=3$iPlT|Hp$m6|8AwVJuxQbJc)nOoz4ef2Ts*_F|*$zMD~$PD^!t?5F5L9^P? zP$3aJ7syZ4MsRYuo!yexy2%s2n=1x6@bjOE^540#;h(js-s?NFmf=muDpTDrCu#re zd74_CG#EGdoOY)M-i%d58n^^IhEQ%;+X{E&K^ARhW#uB*rRyf8$moDo{!Do)P6eYY zu&M9dpA^!yd^+sO)_gc|(=l{4?jxS8nPMk#;&xs`B4v*$w{$;xC}=en;Xc(AT4(m+ zM&w^_(cK4Tx5rF7WAtn`98yTar1`ymn6XbP^hI_k5XEXEaFMRRnB<Pe^5La-YljOY zv;1;+N&HrOGBn>t%GDsrZo}r%#!v29sf>+@g}Mq$%{kQ1P4c@QeL3){`)#zvwMM@@ zH8h6=(HqnItt+~tBer2#esBzUk=H2V$HNkj>Zi-q=S;JzaW0yEswk8U>^)k>#7Y`T z;z!KT&z^rWmi!xb*}V;&JWT=`zS&P$oTHquj;~(Ch&+Z<8IHbyUo>OWd!)w>zW=N? zd=vpsTA7+_rmSp2X>ZrWxe=_@<D@?w!O##wu1_BL-C47i%<}A#kYHOT!lw#Ml=#YZ zEqC`!2dXu{J#SYZf)2VQ?<N_5%2vb)fBc<=+&qpy|J*h;j05u8`jd9fU%F-b+sGOY zrg_Ot28rU!+i*7Mk_>B7SUFHl3zKU&$a~L^$3ghuG7rl>sy>U3>vx|m{0Ophj@cDZ z;yHe`<w^y$8uaLLa5Hc_4x*kDQ#~5kPR+Y4U(IaV8hwqf<sqMYE}!XHvAM&?TV+Cr z{kgfoNCCmIU2<ZxGB)TC%u$8jwm~x@T&|W2*yi)TmGZ(;IWmqfhBYYvdT}4zvWGb+ zxyZaVD-QXKZr0%$8>KDW`>6bNjs7wZr{#xFShVV@Pb;}e*>%Kldvfy<CR9NFBtXV_ z(rYof&xQBAaHW;nf6w)Nze^a03b=AzAN;=3XL@;XuoiH+)@LJkn7lFvL(58`dU!J$ z8cs(`O@Iq`O%QM+Ya`fFGm?~AaHffM%I>P3s}U^<w8J*&fTckjp6Q_z77RqHLV0DW zS^y@Y0h(k^Jgob-?<=khl=Ip}H3!rlFGjq)sluV5;mj7%qUY0OE3SK^jdSl&vGB@0 znm%&Wkp7LfbpJOLmsZSAJz|Zu?Wmsm7YBc=X`m@Q`p%-70d+LI5%h%$E|hj>n-LFO z!C_4J3%&AvyVJha+tu+N3JNE6G)cfc_J_WLT<W!Tl}A5Rn=TC2^;J16ZVTm>$B%I} z`ODw~b(%Y@_k!CuTOEj#E81kj8X?-pE%jbCo7rAhJrfO8C?FCEbf62DSTp=Dc2NHp zpEQtVh^979cR(1j9X%5(P|vCC{HotwD51_0a)6w8=o1WFuY^B1q@kF@w^>UzZCx2& z3Wd5RF)hD!1sP842xBn5Y8g?RU}8{>rP4~m!4`s);Z{&g&VU+~FsSvh&DPf(>1OeR z)FJ{3w(kcu??P7<=0B50@{X5);vZv^tQbd|@9=R;@7-m+DMEeTy-XG}y4Rm~Cu$SU zd#O@JXDGiMtP&ear#({gV#m_R+MI*mQ?@F&Q+YtDcm{hcUszfx*`v^$MU!03*e4s_ zKPKU%UP$ciN*;EhDae$LS}|M?mcz8)qp2-sqP;HJRDb}z-i!l%88gW3^(+=?;mehE znUtzm!8x7C{?!tZL1nGX%@zb09P6Jl`Ta^7QCOpPMOJP4WnV@irPxw!23q4nH7#*F zk#mz5<^SMtN29g@wkH9W%{SYn@GQA2s*epf(&qZj_z~qC(ZsETLLa%6h!qp6l{vZD zjEKsAY}w2&34p%su=P}bVHNNYh+LFg!reLE#*@!^J4sdfIP`Taf=+l&V#@hgf=Q`v zHPiDiRJ~bs4SXRU?gF{!8#9iVCdX63>vi1WoUCx`u3M_A#du^-H<Z{6U)rAQ$jaQk zbv7rJg$4NiSq?@g;mDlU3sNxuAvLx!M~ON!sZ40?mA199_+Tho_o1Qak>xq}So3xx zIvrHr-G4jq&!Kip7Tx=MJz3I3s)4q%_b1j_&U1wQ?4UytCY3XQ+fmjnht0z-xg$j1 z84%+|_sbUpl&a6IQO&k0+TlbwhJJ$;VY~Eji618q!0xgG-E1bpdJ``@y$SxE_Zm4R zOH==y_nwmjgb>2FBL_B}2X8e2+wL?zU;gj%)O`Hdu@)0Tla-TfDpIw3D>3dUM}<(f z!0~W%S)|369Y8rcA%Y)htx#wCr)HC6v8FdI=UdLW&2Fdp$P1G3ybJ!iPwMR5=Q%i_ zk5!Q#uU=F+ZTTBgbJv-xsc(#|IG}Rt{J|IyQ$NRrFg0&?q*4W~>!xGI20o)yvCfrf zV7tSwk>*zXpE=JiXx1*MpD?yUJj<r;a#P5}<a0=)zDKY;KArx}uLyJPP>S<=(q%r* z_PBNWAhPAC-`)K^2C*T&YKSNJJhAqi7PUDM)mdF7f{ZHSRxzDb926t>nDSVIZut0H z%J!Lbdpq5dWoK$Hcczh%#y1j`_}C%SV?vbGT|04)tJ%Op<uv+0Rk2ddM!ZkMmc>$f zUwNy#GK@D=@HOeEPRV@*fs$Y@`JpwRmjI0I5skuiz~>G(yddXTGyY0q^~MdGQcA%L z<#WFApH(+ODWyL>8eMDNDJecPxzbsV9BEw&z!X_*@UN}oWn{^O@A};YA#aVt9g3#D z4{We~)7;u+Y<Ji)4DOYk!m0ZIt?H_P*dX&{p%`dk0zvWVTPx|rA;aeKIeVk0;Kgkd z$cfVFcbzLA?Cer#!67`F$P!(BWDIFk9gs?m3w3=}YXfzikireZ*icqTejkAqhDI6G zBg2z6>f^9up?J4CPt^ftHZxyBjtm?mqt16%tETv=l=t4eD<Ea{QKGEd{RNmdvwu># zZQ?k4T%X*l8zzKj=}OV#*d+S-7M|S?4hHg%ZAyY%%F>%o^u7YpL|<NUle`PKf&O7p z=pOCaY~`m}n}fER89F=l(s--`&Kx=hV0VMh;?uI?OiFZOs=#C&xf~Tup6whtZvhdo z^i!2mdoNc2yfqEm?@?P*Q@sjyT_Q;RzeLdYfko1|nF?KLS#T~!tS7K2-=k+jp4_zW zIBfG__{xx6YL7GouK)Py7S(``PQYX80WjN4Y*d2DjL$zv!R0{PR+}e3=%ghmixx@M zouR+JJy5zk+3Y1(5U9#O|0hclHp>+GqV(2)!JSR($Oc22a**8BNyZfv3dCkM(7dQf zF0BFbvHPg6_NYru?>azkAfEry^{ir>e2IKiH|CI8WYQ1~=Ov@j>&)gc8gV8d1vZyJ z-=90v;QXIiaUL(0k&G!wCUFL6Nr$D$EbPAZ5DXwFCU+BPOR#NvvS&j!pDzSK$m!Pj zl`_ymo;+5f^d<LGRS^F>0?OAg%hF9bez#A{;N^Q~suuoVmDmDjEq-c7_2PdM4Iq{q z+v9t8^<v!3iJma8a+?jSoNMjC+E8mq2LNE%MKk6&%q@$u`ZR}{_AY}f)lCilSKcK< zm$$HIwkVdJvjb_LW3+SD2Z{d!4@`{SnGgUm*2axJHY1)<ZdGpdr~4CMrC{x=9l9=2 z{=KrE<oF+K;7r3y=b&5BO*=?Hl%5Py72a(EposE>6yeKdpV%9=s@}cu@;tobb6tOY zU+<aIhMNOd*Qs_Z-{VkkXX@@XqKt;4B&hFo2f9Gkr@_~qOmuyB?JVK(8)n9e+f)i; z2;^xPluLR}VVyMxWc}ieUeI@m4l<Kd@pS)#z14-E4i%)UvF?N2;OE#5XNP89Xq+$P zcbO`h%x_2vRgrNdJ}@hNrvFGPe0peYOwxWCrz?2S7Wu{t&5(SU*335RQ}-00kw8`> zH{<*&AvN0(yI+dYZyT|&rl=|Sc*OQ}Ik7h`Qd7u#&2iK#hJ6rimxaEUd>%FeyB{-d z!d1T|5{Q$|qLYUs5;I~MkQ2p=kp>7aXw!7{dE#t9V~aO6_}+=>l~!posGPU9dWlIq zFl}RAE&LhHcC_?Zo#EYwEcjPgiu>R1W@o{PgMPf$wPf*<ky-gF;~NRKH3*Ej82k_; z`&w%Zom^#b`TUpxAzzHb-)nIBN_4-16`b`*o9Idh^lg|LQ}NRo!{H^YZGgw#CmHL* z?~u0*e<``;w?V?5i73TUrVKa)q)VUIcM%WE6AEex_}C{{B^6zWfNG-IHD(N@EZHaK z5zqn8xcxdgA2(z;%t!dHQmH@8l{7y+^#;4D`#`5(<x<^TMa&2b<?7C-Py(nTx6jLG zB$g&`l`=4LWd6GN8I!d>GzW3WR@1Yo@x%zBI@^<VRe2_l*7q~lbDpq<|G^h=6Fs~w zS8*%KHlR7oRxV++FO*{Vi;WU$ZqIiY-{#v+jp9-teBf>5*?y<?3EuJz4(dY-lmEfD zDenxfz4D_P2z;UG{VIi%jMM&`9ie=WSjAomA!7I3p5@aQDGrbo(Yal#rDlLLiSLU` zOO7~S)aL5U!C97u(HZxTLzBRmIPbq!>Qmd8FNs7uY{)}p>0j3K7PHtiOGdoN;_@g^ zf<Ogf2N#0ZBGF%E6P`n|7))9@70wziPdJk_3%5%nsYE-~lskyUlTS_=IVl~;pN4l2 zhU-8mYy#1`6wsvr{BGNif42TR;zi4nI}!&*&Zs#U$cV_1SLn>7sY`LbR;Z$U0FwFe zex29|9fe(|Y-MAIl^`5>>mNYDPvBhS<z>Y*0P^PIE?`vmRGQszO9f(&JHCN<#vPiK zK4DnSb@1HCKET_NtE;bWAt+4n{SX0oFU58)+EcAE%b;W2gz=Y1>d#MA=N|*p3>Z{; zTd?B<N{0KY=qk*rhr4W8r*CJd>&tL22yatj^Pn%6G}QbyzlI<)jH7JqEv9Ll*u8i~ zWF%(FdT5`Ux*T<M<g9_*^keSvYJ7sfqBHX61=4ns3zm_<oAnF3JZ2l6I9t^$>#%V= zFhZ@po5*I{sA6&7M@1Q#oaINb^Fn5-RJL|7f&@}jx%_#1+LdDSFp*{M(5qA~2FY*R z5arm3puytxMU23<wo|YQokFzP*xi7s(9ahQF2iThNl<-NK%C+xcbd*aD{e_2U9UM) z39?MQ8uvdHyai^XJKi~+A>y!+l8Apto4tU#nWx+)p!qp`EJ{)EPD97*hxp6T@0yGh zw3iWz)0PgyK_(_=I-s{xI02HL#r=RWG?QF3*p`mQwga8GQ}k>7L~AqRG-JGXSq^`2 z1OJ;r(pvlpTaYP~t@75-XzBb^abo^OA#%K4hNn9=3Jr5pSr#$&0`iq@<oK$%v~)uD zJ_$u>c|94?p!30qm9g(Ubpt8~9q|g|(`pyRdGZGf)ZLN_a2*M>kB8FqbbWL6c~UX+ z<o@LGBP8WDk7WA|C^@&9!}9j|SGT7NTDl_SF$dm>?I2ZjJ-SIpZ>OdSTfAb5LP~8F zt+caW{xhsiyXk?unJUMs507IMCW@N9ohHMQQA*}VS!>Of;*Kdl$r#2=ipo+WgSeig zT%!bi5kqP3|GBR+Tkub3pl3TZ?CClm)c8KG5ykjC&~EJ_jnpNI9ZJFcC~53Ib&TNy zrzZsP92zsPM4xKK93r_&>kkK+B0sl$Wy-9slkA%)95(oX#E3M;&H!&0e@w$sqQ9z@ z^Q4RPyhVJ7Uq4NuxO4d<o5>*5)+mTPTJ?O)aJ<mVW1FL;<gGMWv`tVz@s($57hSC| zF6#3Y*JH~Ci{uK7o2b7#<>7Aq0fneE2R+M!wcGz4xL||$qo8ua^>ypFn2?#zg=;{n z!`zpj_@Jzlo8NxN^I8z3Xv~f5gd{(%%M#y%0V9{iq|k6r<Z7Jw$1nYc>4HX&YFPIW z7_r%w^bp&-4+Fb2;0_Zdb+D>6VmH}lug1j2*C36~kE8vgS^tl!ul{SgZ{r<}poAib z#6~D6AWBG=#6VO;l$4<|LTb`6x>eEu=~NIHNK1@Rfx)N|8$Fa}z^Ku2zTVGspXZ!k z_7CiJeLvUr&ae_552Ho~)uhcqA4GRWBllvi;O;E!H;z!5tTi2N+z4?^pnF$!Rw$v+ zhB=8l0@J8qRJYJ2`w~WmwYh>LI6t%0Px+{SN<^^{wVktf#f!KrBC};h>VtBSL6T9> z^6_GYr_yLYY+Zt)X%1p1-JvrsI2aXB##Nj5$u1?g{51Lkf8b*YvN>)7w`JFNA4gW! zomldRJ|n?;^(K+XSA8NF)zL9EKfBDSu_@D+qmT*&wB4uoM&_1xJL|=2?3i_~YV^<{ zewVhdY4tUg$7%G);{|5(pjmFNeIEVoRyYd%ZVI}3WhQEYa%bGn3GQqLU0T46aR!Ue z9D7Ro_uM(1*D339Qudkv*lbqh8|TYLoBwJ7Ft$&v|Fb(T-2~S9n|d6y)^vt@Wp<ZF z6W@bO`>>(Of)6=S5pnN9M)ayW4L)l!K>H%!-5}d(6S18|6;>IP_1zzjqxqG4&EI>H zdQ7Mu*+Lr6(uxwaTTNaDjaCCG6X`S&zx@=Zwq82+*iYDyZlicm=!@maqjI^{c{3#e zT4OiP|K#Q|_GX1H^QTIxj0=#ueB~LqeV|J6b^!(M3_zwv;|JgE{?KmCyH&@Po_RmK zN|3)!V^gp6)ukP4|8g&xSaaOo{pb!p&A_%gvWV+XM~3Iso30l!y!hDOKk1maeyt}3 z=qd@n=YH8T=yZ*I;_}xOm<-@0VOaF;RGODWYsb*>-u9pYxv$y?0;dVT&y#aUj*rG( zyj1BuUr0)^x+AxKio(ZLN#r=v_+g{J9!6|6nNJ=F!Q?nDe<^k{T}5;MmUilYuDNnf zdn(L%tvec54QkEeHH38+S7$#h<$sFsuACXYYR*u0EQYUrR`GXvlzf~*ry=X_<msNY zVR9zejbM`sj`;GKD9y%Cy4u=pcmi+L#+!I4%^y+P%aX>K+3s51_4zQhTWXo8ggW9| z%K7bs$XvX9dFknR(a-3!^)P`J^wX@>7P9YrqT9ev{tDj<0qbEU0`Us6c#2X5QSb1{ z?9J<%e?Q9j8E-Kcp>>%aX)1W%ODCB0KYFykul6e^i|q62I*9z?r1&Y!y)AeSi-Z#O zBVCn;h3exF$Dd3fRvaIiYCO+MhUKBz$c}QHi;hkUi-g_dHDmtuL*V}v%3!q^zTswc zXuBshrolz^ZG^rmX)XBU#`CJQ;u5Qgw)aycjf-ZVAcNXHZv<p^W%d-7wdO}Z3nA69 zL|tBZh;RnfFx&CN&x`dnJBof_syn&s&x+?EC+`%qCz+unNYmRfbpmsw9JC^~*8;au zNJdhGE^E@3K^+($)g}O%%Qp)c(BQ#D=t@9hl#9=f@4IW45691d-%ntCYdvA$RRY>I z^Hbl@8)I>Yz3-nj%Wh=LYIglGYnE%4y8*4kWFaBO0qyHC8|jwHH(lCUpDdt@$jTR8 ze|qt0<f|Oot(+fa6-Or?NGr}zJmx$xFd5c*E1g~fj$Ct*c<cIn|89`#%xATlH4TN$ zsnZ^~n@I4=*WBgQ8^KTM@@}rY^_njZPBA4i0%kec#@P@x3AJnzi6x0K<~`j~esAJh zVtx=ksP{mn^7dgV(;sUKL6B45ywIiC+2k1m1)Cl&-6P#oT;HOp7?5WCjIKq8q#w;) zj_$RjjpW&HAD6WZ9pv`;=<zy+Z~|RPMyb?$t&^uNeIOaFAe9(QX3uuSl}?HVJB+ez zgv5Z$COszNU(|Wxr>pZ^xh4_Ys(m|eRJX$xGy{@5`Scn5e00kK$-;Z4<P`RCy@38w zT*g)reZnr8*!<Lgqb=mZ*3_jJ)R^XzP@7KmRHsyN<syyw>x{nDU{!ZB#=Y$(VYVJ? zo{X@sLYqBu#Kyl?dS;C*tD`?$$<3#S%~mJB!6(n_0Di$z>|~|yWMy;3P7PVN^d&9n zklY-UW9OhU!`)}fzw<3>bVl|YUx=ARf2#3&a+Z%5(PcDlFv)9FzT(fV3&gZD=FlVC zMeos|6VhCZ1#4;LJZ(as=iF4I{l9$LmUusP0<_=OAlDF>%-fBsOds^3ooP?>6ol2i z4qp2#{J9%^D_4^aR696ry~#SJVNRNUZ5<iOJ)k|U&#ftC+X>b^!YQBs_4baBI4gCm z9sliq^FC|uVirh$6@S@pLtwPsxdK3;HAXT!x$8E446wsazd5=lN<I54<-_byfCa3_ zs2C95_e_vIPH<#cGsS=Us~XBw+rfZi4_{&T2UtWi88iNWNlGDY0y{{<!ZMUw*94pj z{ppm+b3i%RygFUT|0oV~ALr&j+wSFip@9q2*@8$+tA)y0KyyMdJ<}XChTrm*b&mdy zW!{4?ymq)xA8)H8biw=QBhL1(4ReD{ckax6Y<h3q8HYJ{;<jSB8!$TDDL*imJ{LRd zj7W|F6isSjK8b!WM~UMq=4(9MjA&}zcMyLW`p>-hDO_sp3zdp16$|EdfA|o?>5G|( zC^^*FD(=b_hze_02$rrk%BoX7@LOh(il_I*q4n30nR}?hl#5c8e0e9NRV3=N^4J1J z;VyMs60x#c)X&x$+$E1qPm@{*FsQY_61FpIjgA)6P<E`n>Sbt(&GC$1?o3&&SJk{@ z+9o-3E9xY3iCcOz1n5_x(40p|q*R|Z1jCz1@0}K#fYytl8;L-f4Qt|o5JAh@pj1<B zka?d+vu)~gUsvMsRHaG$AFdS5DQw_Lgj!8{(C_$Cn%cjU`fA+cK9{=Aw30;g%8p)^ zHU!gs@Q9z@=%FaUH5iIA^NGw|Q4(~s(@;cJa`9HL^sJ}VZi)4jUjSAnA?A<)UQv(U z+fJhs7ocXR?xh3f+iyTIijlyma!$YxZNTX*TaXiAP#wdZb{Cj<bXIHMPg}pSK&%#F zxZ}9bJr|t*){S1&RFbm9dY)w0NfP2(3M$rg+DW|I^XX6I7FCA4hYtyi*~ygG9?;%1 z2aQPq>zcRMC{1ApK@Tgh1~*Rq<^j#R_gS)>v@<711mn-<cVg{<cLVy{YVGztYiX&M zx?CB|!3jvK8Aj2Q1446?|FKxs)20nI7xT5#dAVdn=DG^@chJV-@V6-CSpjUVKekr9 z&ktXjWDs;P2rW1W0q=#(0m&GLo6`(x-&N36Pk!MgMn(qESK3@c65c)|KE8LC$2hnL zI{$s8h?GoYa@?uNdjGFj3|%jy_FveOiP?1hK1ZmKFSqd9I}c@|!rv)FIL7tvI4vGj zSQM}PS>Di~J4zcCU~Z}Q=a1-|lSe4qXgoy6*;$?o7F<t{Id`Vl_O~^yf7PE0bZ<sY z?0YfQqj4X~_!}(T5Dfm^HG-2S)eCF#E!RlkV`V>|f2PtN#RTpeBgIDC>zTV0tzMXR z4veHEs}?k(=Uf`Md~Ft9Xt@*z;*gmKmE7yDkG+<{413Ky7sk{Mer;}M1@<oIxi67j z7zk=NyWQ#kU+LIA<`rk=$5fb1mIo60#v!#bL9|+lati}J$HJ_BVF<cxK_!kvevJOq z#687+tb15_xISC-wOsO|EotT4YQFpI7h8Z(mmJBN(^|oAech#B74?Us_|RViX7z`z zivCJ1Z?*l=L2G4}@ex`bzj<60!0OrdEFW9<RL!Wkn!k`}-$<*w=nb3_@aZ+&lu8RV zDd9T`pcgDFM_KZCYg&{pzg@iUTuRv)vnYEVdVorup%O;Al18O<kQZL)D{!Is0!%fk zwp_hV->e*?4!A;a`6x82Q$I;a5kl}=R_<9)$)fBy<!_ajG*nE*{qd^m%k#|t2+cbh zsE+f1E=S3p>h9w9m6#+24-p(S{GFwJY<T{OEj_$f9tFAubx<7_u0W7(WvW=u(xfP% z{SqR8vwmwxGEn^zz%BG%&m<R^XWreDcSyzrKHGoAG4p5_$KAJue)r;bg_X`}P5-;B zDd;A2j7Pz-1`oy8fZ*J<9pb@_JT6BcgC?Ht#^<Cw@VKbX1;8^>7NOUGjC7oK^6cf1 z*XCZnDw^5R6X(Zta>(K>+VzVpN%m0eS&z~=D8beWF3=xMq>Nn-+`2QteX3u#WXGYN zf6LI`Dy))yuk23X4$ZsTJV$7jg)5YtbGtAYln3O`_)KfjKf~7nlljpSE<G6~ePGho zBxh`KXe{gkt?t{BxYb}qI=JvPK-g~XWSC}A*8iTP&;sqt!8g5aDc=d@X<w1&nt_1` zIG+NNnq)Mi(%gK+86NYa`Qt-JPW$<d63t@wZv`;DZzfdbYTKG>RsJVij&h>wq=mq> zg{0)>qq*Z0+3=XLdO99#+)$o^*zxe2yu`B-VoN?+AwK$<h2rMzZyFDmW3kuEM1mXH z7Ih@+>(Wk2Z&?jecZAZ3uCemUe|vKZ)O!1j0u{UE**in7$)?$KkgULg%j!{$xzjmM zg_x{6T}8u%b6!%j;WjRxTnS)%l}brggxpLDw-*9E1x&79<sWC(j*3{9OIFm67YF?T zB^v1RYGWD&DJI9>{6zUC<PSmu2?T@77z~|^ej+X1ETVc+4kbO_OPJT7TC_(y3?u!B zM!UDi5&Ya9YXhH@Se;q@(<y4p9k6r^1N(L_3ogBSzS|>CM1fKLBv$`wa7;R7zWRUH zWJfC9br5`3#~j<ly>2{}z+0zKy-a_)*wgJ4-@PZA@IHhH-<2`wa4?9`$_`}&>D}(z zL+4t51)7bf695lg>R%6C6K}xO`!H3WkTaKKHh$=jkqv_ly2?B$APYC;u};&2n0XDC zF9jrN&#NT4I#uuMI&+B5;2#$+vMC<k_`~p$xhRX{=M)3Pbc66t08q&6I3AoGb*~oJ zMI(keWQg!{>j%z{t@`~$`ji^Zsp1t@Mh`9Q9L%sfvRMn!nVwZ=8I`X;?<{P}bg9fF z^W)Hx)GZsIrf38O%gY?wMVm)%Y-%t8yW)%|N@EV+^m;81BY#bZlr|~pc}yB^;e2Lp z(}3rG#=)AHT4o&`v^!RL-ha3{CJ-@a%1Lc!C2;70=vz?a+tAoPjI@y~BkCYo6%A-3 zV~1d~F@$kBDWdyJ7<bT-(m@M^>}{bRAC0ykvNtZMD8Ohi&Fe*SrM-<usi2C5p^E#2 zPoRpG7X6!y2VK;bhKzTy01is)(D~z@I73lX_@X__0bin$^)6F}#B{RFW{V;B;zUJt zElp8O8WS5Qr9lLyn`{_`jXBkebmHo^Kl`LLDD1|kHL1v<=EO2+mLqYCtUKk7f}CzS zic#H7*|keAr_aeFS-8jiu~d9KUr(5qX{vbZ^yTo?P7dGb#K&9VBbJ17L*$j8U1RZk z%~IaiJr}9AA2lP_TKmq_v)%Q%gi?}<T3m=emG-pwF_`gv7_?bF<(q#7@}5;pU?78D zz|PZA0|S@iupiDyiH{-hB`8=T>gJN>ESf16haH!k5gZAtt?lN!Ny3jLJ&vc+qg>2M z{rAi4xn?4JChatvZt_6!p>%8&y^iL#a>|V7YML)WJg-i_gVdgZ*oiB4tQj*e9%X?} zTqv_sAI7N@gi?)vcCa(X%;`)MokV)XkFZjCY@hf=l)n2Zn%)IlSjMqOKk2pcytM<G ztOyeui!k;okNwe+uc@^luFD0C6_ou_xs;tWUpOwV$GC9=sXLi)qSKJ+cqnA)FVO&1 zimhbH<mDOm)QSYWJ@XH~7~At;M)LX+_R1YB;mY+c#Z!+~HLcjyYX1DPP|FLf^?0lL zEqwk3k$6~VAl|$_a0ZfBE?xFNgO7ZYcmiEQv5O8AL3<Va@=_mj`M01-xrI$-#>dav zjN2m(AtHGVjod_Q_xQj>o-s;n55F0FBHMD?zU0IEpuVXIgDu@p<+4P;_^G=V>#q<B z^*AE&?o3RSJ{Oe4*muTtiOu!>=&%>DKJz|d!hYgOt0+87m#bA=8D%ZP1T}recbUNq za?L1+a-!W_>-PO4&r~|@cDz>^en)#bpm;vU!4OflI$H9%vbs52N*8@bUZlZiM?y>$ z_X#!9+>dVdOiH5sG}@vx-EW2l2fU;)I4<euLTw49>D8%N>H_8}kx~_cuBEvG`KvbS z;CCk<`o?u{Qg>*28vMxb&x7T+UV7w!Aj(>+gq0p+I(RzN-)g6X_%^D93@A7|*C@Dv zLKC7&b@Or~p$m+d#1n^cjbASzGR9SWN0d%|9OCChgd=U5vc0gGw?3~FtRn4wfVIEY zd6+E)soQ8~HyhndInEmL(QE=fZjVnTn!IR&kBXb?=>>Dy&I1+cDxsSFzH9&}Q!F}^ z`}?y!*JL#RS?a`|hkNqYyQ1<%t7SRY@|mwO#dcOBr=_VMuRh)zvF?aD?MxcI@d<Zn z?Cjnpl!8=JzOs;gAV+0hpJ&f3QHafoK5))6&yrl{VpxjfEj+GVFYjihmg-#N=ec{@ z-?x2vzp;8jYnLZ89(=O}B=<ryL(`D2g4Jn#WTGNV&Oq5($la!4_ubQv>#hb-4tw$z zauu#w+^in{fWqQnc8MC}@$$;EYdOZ8G;~H{bBXTS>c;$iitrGN;go$Mx!GYK168fa z8M6c`q_IFAqTEU3r-%fTg;o;&nboQY3Tbdz@|!Y#F+{A2{6HCbPDjUT)Mcb9Glrzg zYbGT!<Nueu(=mDuYxWk0oMuAJ*%Qi6Ar0PD8L2C`nID5NA0a}cl1#2RQ`)A#tR^Vd z`pOu(?nMCe<+le$ad|9butK9dA7~>4P6(6oDisz4pWg~8L}k|2EMW$w5S{5}Sir_& zsWnS^HLOAK#Qp*O<;024u20tO^GS6V8IE8Etmm+UpLADn^4w||O|Ftlci@H5<D%iI zUoR$2@gi{nB1}^zV>eTOvU8ie{z0jvq&p9ThX7xx@(~tbH_3_Hig<AN_wn8`qpAP# z>!=EKdUYL_{Oaw_nE2qgk0Muk>Xc1%7*d6#Rf{w|(mx~z?o6<);QHKS<FbIq%pG3; zYMEz4WjaPAUUtZCa-?)FZY0Jgu|^Z%;7eYQ`~oaKhcCfC8tq9d(T$IiO4NjT@3_Yv zLFPo(<6!)tJA`W<=@*+5<80ZjcHRDto9lFlZwWeVif7-siky?7oXG1k7b$0FUe`fJ zE~{CI5rmP*-?$8FutmMj4E-wp{_Uz|4)cbJ!{qMgzjlpTAbio3ri_prHqo*>#IE{+ z<JC@Uu^^qpy+wBtuPk7YcT60b%=0Gsui;Py`W}rjVYsFYg*-p_f}=T-z^ED*?0?Xp z=}2DLzwLlTGYA}d(Sw}#e;`lRMqfNW1Qy0$XFKlxVt@r$xanHvmYcY;=sX`0_fIdk zm{)4*O*SsrH0&wpQz1HAY-R7@w-(9Q^6DKsH^%gd-=4bfD;a~$b{p=4MlaBfd-TwQ z_G1xG!`Ow`j5tHJrkgX-xewpcooHo?pK145{j>y2D@Sth4(|{W#HmBIpE=hx?XXwe z?lYqLp+i}xwxiuMD`QHK8l*GAk|TCx)j8=dWmw*fx?<ek!uS-tv>4m^<~#OIIQ*0S z>PO$84gd?)>}|D7hwc!3v32Yr@s*bFyu)*(Y)W5C2dp7>dH1H-<8Cc?LYhdDa(Y;! z7reA<LgT3YU{UeUB(uC-o>#$!*<_`wB*{yQ6~y~@&S)<KbLM+Kaw4JP{6RkJnK3QU zfVjLk-9W;PG4u$vt8@!0VL=VOik~Yg4nl<O!vNGp@^2xu%12)`*`AikU;nfg0%!HW z`uUbpUe*}O4S6>{c;b)!9@i!4X#iTi8hk{M`vq|>Q>)ppAEok5<GYzmyv6wmv>E5x z#_{W0Fm1xp%s8T%ui`!Jn_RZ^befQ_Qa6n_GXZV25JJS?Ny8eJaF5|N9e7YS?-S-u z<Jn(0MYkKF>L`<sy^1>z_e|rU9exytD-|6N@&q4VJk{;6y=K&Yf)8WRaaF9i5;Rq8 zbvdcv7UQ0n4aOsIb#ATjtxFY*;=h2}H?*&tQ>sx3h?{+OdnfNOV=hQ*Es3>p2Y5`X z0y*|v+QiZ1k68=-d<3;1EMTmo1O}@959BPdfG|Pw)?;hjXLW1==FSsy113|z73r|V zhQ6j~9R{GB5ybwq4VB4(+}qI}QbMOsLc|$q2a9iI#^ov!r0TfPUne&&9HXB!ea{o| zPU0Un9>}obcq>!7`{Gq%8G3C745OOcR?>KhBK(C_8cQjvD&JpjzZgC9YtSvWy^91G zEOE+510?<N%5FxGDb;{pM*jpB?}*JzEX@U0OLH3JA-{Q$1h@)P!T$a2FIH6A#_tEg z@ivoNYXmg<462nIf*Et{KZcUd^BI)0g{;}EY>r`+&ztsU5_FT;V;&nyODp7&bY@$5 zjE~~};NAIodVZ^rkoerXqGg8b-ow7&*ehQpv%dEGO}<C(w!F_Y(CK$1PyF24dDh+& z9xrt0b!G3(%B13ar>*g^QycANghca_>H2(eCTqY*IIq=GSd69MM&1nfS2Y~@<z`7c zu|V+9X4ZeQNa0BJ(NDphB$DIaCBW$|!1H(Zl36WXSC>&|Ak55pawe49Q9TFUE5ZI< z*Bk*yrwM)NXHVdj6EWd%__~~{U(p$t&97t&`E?Ka>-wI&v;l|z>?KTapE9)lL>Wq} znCyz6GUd;{F%#shdB8{4n%jS3+tMK%#L=R)+|H=V9{}U_He<YLgcP%mKWYb65~AD! z@INm!<6QAgLaSh^hZ4a&!&!5ii!MuSOH&ry-t2XralY;kma`}sy?o@KI{9Zee{)-$ z0oxrXSMxnWAa9W-DrJK2{q*{|&#MqBM5qVBQ*SaaJLCu_+|9m%W|OZeqAC5%BN%n* zloAkcsk-wz$Xs<jgjM{MxLPYw5IFvO&NUwSWAl@HW{vWGrvYqdb$r3Shka6Yk0HzQ zhCk)-c8tPSw{$Y8ta(Z<`Lh)*wF}j+&0}`08v({ti^h9#C{sEClb)Y$ooij{PXUok zB~3-MdbikaDjk6N5R~g{w>zmZxV6s%Z<HCR&)ZB}k4Ai$&1NaYi~-M|CqdCisl~X& zitbP8iZ{WJ$1kVr`rfDKGlm^D%%E5h!Hs_)D-cR$9Jep3F1p2l%ekzoqi9L$dW}JK z+)ZFF&!Gb&w;o`F@sKlE3*VVZGr<f;3wI&a{)60GeNtMGAt#i58-Fu-_;c=#&W_Tx zpf*jl{)53c7lZ?ITfF<cF-<*Ni^qDIA+`)~#Kly?{r_fA0IoX=&S;?GOD<0c#Du&u z5Mbo@c#8HOY4H9R7(}mE2vL~4o-OB>E9bs-#14U$>ACxOLIKC?gcDzNNVr$xrJ}#q zA=1d+bXK4bUGr8k-ES#-F@$Krt;-|m-;q3#zdKsQEOkjQqkZIC8=sk&1vk++WFas5 z{JD_z{r4mKY=uCXJ=E_dey3F`w1k<^D=jfQb~b6|pH+~4_OoA|A(YykOm_d6p?l^^ zG>>#z$MGlZDD?T%-4SFJ%ONP^?aFJAMiQ^ks7E7QL&}3p9_D)Nl`8g*EWy)%D0J!T z9pb&-$9rWT)zIz@3n90JdE9pT@VWNK-|*VK4d}g%*+xo$4TrZBelF~OzaPqmKuj^W zJAt_f9>TF+XTcVvU3}coSyIF0EHh;yp#m9DOz+3vlu)h>IP#EzU$)9)7Ie>=sOKB= z=T>MrIm|IXlG`^Y@4BNzf|6s70`gvvySmxSYkTFk1@us<lpHbDn^d!teluj?`J2`t zdlMN-2iI=_-W++eOA%LId)=v?zQTI+(u^shPHMgQLmE7Xh+a3RYteQzZtZ~8eeMCA zc1&!>y&;RZuzBT;znowFGu!NdIv3iBS$T|yOH%o-EuUu?LlX}?%7o@Wocqz|ES37o zp6f3prK5QDS-T-=$UmZ$UrKlDy@ZgNr5Q&JwSkX+#Xo6QX$bPj`sj|8rdf(<@?eQ` zs~GcE^ll7vT(ugCNOO|%W1}m3JL_Pui%}l^F5>%L$Is0e3m*9g_KbD`r9{4D<OT`$ zR3)unw^aXjMXF$hO@BuEVR0sN!VKW&#m+S!q|S8}2YJ`+PL-t<sV*DFn@=6h%Spx~ zCuaOFt`*DOu&UcKNbWeAF8f5d5PT0(arl|?cBGh&M<~Oum};`?d&xVL1OL*~Z^b(< z$4cQ!nX$}v46`wm`4Kcc<P3y-1+k`LDFt$?LVm4!;lXl7T7d0cIMhJR^>Es{B7}CM zkuCqj>8*lN&xhKed*<FEtsS!sN~|c2Mi`H&;x`E-<oQL=b?bPz#_Si;oR3FIsX8O5 zP$}rqM0VMmJ;%0e6Hez-R5<i=45Myw4)hnot!FhfK96$uP=-3^t_AL=V80)si#w2* z7Iv|`?*Qsqh<Uz5W3cwPr30YI?JSG=;YuAA3$bS)NHOxi+Bbk9nI5TDr&aHrh*_gD zLQY|(Ot0I-1EQCTP98Y>r4Y)Wjm4feWmW1~`vxJ^m<cvz>Z;s_8d{u7(#2wrS$_;$ zdf5;r`T~|E8zQWKpgUYFGg1j=EupD%D-#0!)L)hqCznze96d{GS{DCt6FnkWQ>k7J zkTC6F)?LQDXDwM*56(khtMJyTXMeD_7gEVN8sD;tNjak+4I$KAPgf=ycI&|LcG?`Y zRhtD|wNviCGCN?`A3x($9Y0tlt!-y=P1MdqXYqj_A^8P?>poR)UjiuPYSpIF1xv>O zyUu_3gQpu|54R7{)o53RF71<vt$_PK-$(Z$Yu5eA*gdM)+Qgpk^IE%>@l8|Y12eEz z!%gT;nA36M!bO9|yF3+{@G=ZhPHOsF>G+WnwLf%aQi&|!Xc3gPKU-$i%q#!u83nfz zEG~}bQ7rikRJY$rJYkG60(PHz!%OAV@XwUm^QuV)=e34Hlg%5ToYT3!vclM=ZT0)- zU)=c<LFxSax?TJg<<<Cw0%SCF#9f5g)G&XE6JV8**a`<c<@PJhplI7K03KOyym)4m z9{maL%2lpGM|%l{{#XvJ4t0E8;9~08!JK7`nJnV7KZorS+*K@_G%tS>LnwcHhmbc$ zIIPLr%5+xk>P(tCQZ3_;5c*dOu&l%#wk`ca&GB@ZYI{_EX^H=<(%32ccxQcV9P66d z<!*t>mB_6+5k5hJI!H<)0pxjq-$P_ZJp63%PoCMtN!umOj;DXU2b+;{jnC`mQyrhq z2T_Wx&`*7SN0d6A**n?gYO>wObhwPaI+i>hkd){PSaT29!W3VE&+YOmmRe{XSkVw+ zx?a+t$zbx+4P8YYSXxdVp$S_7cn7`I<HA>YE9Jzq!BsDgHM|h^iq#{r|IIJrGDwaP zRTpvQh$xq0%cP|;^r@v*w?57TRns~Ha4x(!Jl+%fT=HJ#>K&e?@7>myZpXBj#h49% zLx`QbQeNn3>*7+#WQ<sg_B+iEFgof-xZ$|tA;8b^*n*X_1A=GE@Nz}X+X9!p%>#wK z?8S4*qOg5;g9+2gcC>2kY(J!g5O=W}TwQmixwfIPgMnYE=Ed;xRkGQIPBX_QjHc3d zW^q8KFC*4PdFy`4|5RpQ=w1O<-wH1NZ^DUkx_NT=0e@U?t@zu>_@#iyaiTFTQgCOs z2n*qa(3VS>`>SUs&Wmn-t87M(zU=4GygY65EWv62bK1#s+@#6Uu`$>58Jd~u=Q4M^ z2sVr!GW4tMSi;uP<}Qxtb-GMC4Y<-P^k9zwZL2>#2{QMj{j&ccV7)fwPt6PMQq<h$ za*riw=^&}cwj{2k?l>#{g~8AMiKX;lM*7^BER${NvOF67j9T+R+<Fl=pz_pnjcd7y zKjEOmJ<PU&7vmNNA|Kg~6*q#e{KJF6$zU5et<L+~(?^}@1f83YBATXLDus|{x29uN zfP2tdWl{m^uA5;_@LP!f@!dE>y|-aZGUbzcc1K}Tp8Pvv%Az^*Y`1z@9>qq)q~bFy z2&lMW{Z+I@khX=>q+rqHbTc)`2L7Gd&-T#(xr6nH74LwrSp}pj8`kZWhvf*YERdhz z8MmXq$@dR?@N76B5K*;I(=Ir3eW8D+d4ALT0m!#@YA=mG;R>U+VI=hx#v{VGcJ9p= z7yVZvB+a74!$p-C2c9z0e)iO%x-sx~|H=L$x`(O3mPl9^<}xE>0+Ic~0r<+i@Z>Wv z&EM`zY6sk}`D(0W5Hj*_@yvWQhWi%fvtMrk0RWD{F-qpF5%OYAmwH;6AX3s%kz45M zVnI8&6C5-hb0VziQTBcls>B8bZ$1YQq6v$`IJt$AdUg~aZsO%|IvxfI@Mz~gxj{r- zP71KsXroaI1#R5DqJVerW%K$5f=eFtkDvD<+1(7j_r|z2(@CaSiM(Maj(u$>$$U)Q zdiP9%X?S@*qz(r33rmvy7nD@a^P}gBic8Hicd82N+8Y;Yrz)Oqf~RVJj}@pLFJV8L zz?M?2e`;T3+ssK@i5osU73Ux9abKsZ1Qb-P=!l4ndvLghZ`OpA;t9Ym%1!DiM|M(g z3+r|<1OmK<q_6ib={!|J9w?*An4|q2$c(-z)??rUpN9&cw*`)!uJDS#7k1Kz@QnGg zrn(d=zA8lSY6ZleUey-JNpt6b!5g%XUE0`-!njz-PpX`ba1&GV<+J@eyPyxstR2_e zKyyu8%6y%-0EwLHRfT+(cvXp}-Rx?>+mP2}o$(c)XE{(EA+?_td;Erh748O|=Ra0{ zGj3Swx7%DuL@oBJud9sT2Pwex=Q00-U#!~zFVIQ^gLw@$7?hY6@|pLM7kx`VkH7Ry zYAitD`R3f`=Rs?jj%5hI$mg2G$qTBU<4pX|<R67LG2R>vHBMBB`#7ORaneYf-Kxx7 zw=7<gSs$UfreG}kK;q$}Ak|PQ8x~{>?Kw5bO)a_>LJ@eV2S?+yvBwyFY02%z+}uXT z<k7hcL%Y8?7vwd^Wxg=D3@)U1oR?W}_3KI5;+GsDnAWx54w`@8XC)V{)N!7+xP#(A z-^SLgykE3?t%zIHzuUKX*H)NV#rt#@&Si`($~y>651M@nXP|8<EwnV}(((#0*6>G+ z$})oy!@d&C>w0N`|7Ln1ZDY&|cB=iN!~Mo?X`i3PhH6*Yn}Zw@tzhk3n*BXU{|o@& zZZ-}Vg{Wk~o#hmN#?0^jvJuOY(OubZy1+ftX$Sb*|1x?!qxbNKD@|`f-*5p93t@&a zu}-aaCW`SfFd#5=Vf&Y!K0ULl&PkJiO}y%g(82DRfiuT0BT9A<Ri05R;;HXO8&{cy z`R%!pZsbY4lK<ZNZ$pp^exYHb-RhV%XIndpcQ){40!zf1y!9`fH<_v*d*H5^3EoVm ztyBMG%NMOl5(2&C;Qe^Jwtg3I78k~NLTBeGax_2*lZspIAC}=fhWma%D}6KIp2_ny zqU!DP*R*b)Q!_Y;RqwzpG<wz8I`HI{7tE=*+qIf4HRvu_TY5L%F7UKTcH0c;SU|<z zqyYgA_~4uIMHfcP>|g01V9Em$O-~NlPX~D~f>1G6guTrbtCrC*6sEoJ_3o}ZySkGP z*7yL4s8#nJkd#4=hVdjj;h63ZCPEpk(LswCaHOp$Rf>mk@z`~KRg&h(cy+p5cH-dM zWt17}P*;B<1m4GNT5hYnTtE23)%@L`xEqk=J4`(4A&MXFtn%D{9(04Exmt2u7BCwQ zHyH$3228$D^j}D22H_gEI=|~=AK>#@4%gb7HNX20)Y(&G%$R#OfMKN-7$oj2vX<xZ z#O=JBGD^-sP;sd1{FcR1pTE25(K@Rj)6K!HKRuig_3D!ZbGDhH59FpwTzmgp$X(i& z{9BhVYYG@I>A#FuYF-XgECxQ$2?b@Zm(xSIOit<@dvvz%9J$QJuY{92LzwnV$8Nn- z%ZCo_j0F~+tF$cs%$mRedHkaT=dz&~*#BVUqAO3tL$Twll(=@*;c-kD*h|QN0{aBP zHSg5pG{<MI20i>mo8Wz-;CWa6|AIK*COOxsF^Ky4o`(!-D3f{mrIntqfa7!NLInE= z=f&IEJWGi@;d!rUGK7{8Xb&UR){ZS%iEkO4>J1XoG1mlwNGg+G@dJre@~Lkj4~`Ep zH;De-8^!wa-WKX|eL}4+2fiLE@iRg3oxx|#xz6444L;AhCsUDfbd(WvwDyEgF!;m# zmt?x|#kB<#%ShJVuXAE)4?qW0Vn!>kZuIw6nPgDHq5SBU8Si>i8fQi=d}u_1g6T8v zF#f%JYh+#TC1YOw6!yIlYHGvi>s!<G0mhv_I?gX|Q5yp8Sta8G8ig`F(*c?^v%p=C zG|CXV@Kg|BD{7Ppre4Xuvk~zWPe#yJ5-ot_cyq!neI*XQMZ&~=ccuaC21<(mBk*Vo zb!pa}u=x9ehoud{5)NJ-eE}CSXLwx;>t1~Nb`OHwiCGz!$dK}SZqYxHS!s~JbGJ^~ z586C~CG+^}{FdgeCw9rtk;SJ19%r)WF2oi!voHDGqMEfa^W@7>qwgfF))LnguukK$ z2jDw;Y_;8b5MkO?^Q0o6inI_v!(G?J^`gf^y$2L+`0DXk4g(ko+CJ;yLS@JQ%;fME zPlF7Ysk!_tpv6=8e7xw#k>5=ngS*uA3$<-!9JtDG5b&W^s2Ol8<MVePMwc;Vf0!KP zyE%0C2zWrpBTT*CP{z{X$+Dq=J9~{!4)YH0N1PLTVbz{c0ITG?4y3^Vl%UmECPl;D zU|c{7=1q24%HAEp@E6R;4FYKK!+e*{V=1YoXJi1)JP6jM!pJ<K!(7*X_i8S{>RZFg zPVdY`zE{41s5I)5Gb?_?!>NURuW!Fo$@^<jlHjBDj;GdNEjI1c4oA-s`~>Fb1ur%^ zwthQOZJniWL_fGXYo6Q|^F8Lr#*>Z5vhi^}kpbQ_M1jUqXUl?9&=sYY`qRwBi!b8- zCE6CvL39adKsWhT=Xr_fFCD-qtfvqtYxOP6fn&b-Z|2KLvM7c(B)-!n<wn)zXdVrl zG4jMMfF0aPiu{Ke)K~q5;8%)WdB`3#P6x&^TunG@Pw;rjGfTy2PP~4s(&F2fl{k{K z|8ViWP4xT&j|ow`hn5xPR*8qBygD|EZAB(YXV*~y(Orl+C&@q{VsngYe&K&S`ye$8 zv`^Q(FD(|_>n>DV<1nu?%?J`9^i8E>-EF3yr{?{jhX>{JKysC6I)q8ITeg<}W&}c* z*`m8GKLUzDqU@XSPi(<otJQRNdKkM0-(UPFFc#Y%+M^6g^@_$v_=g6o3<UD;6<uU| zvhF%N@fy^2@gS}qE%;IE5KHr02PALs6j_>BU%iPjV!jTB1$S9eG9n0+*)%!}WP9w3 zm$awLWSPsiV}^K1N?gr$a|p>Vl}&Ghe<j`$Qo|e&il3`k#JGW9*Jd_eTyHWVCh;Z) zAZ1L1v*6-B>JuxPno@p3#0?1xz0oE2gUP(g%(pWiR${vatRv@RN>C&|MX02UNU(ni z6~-)BntIhdg`2KIX3vNUjC@`ivkG6h`;I%rAaA;`rXd`(Buz_OOxth2%J}ICNLoWt z{}4zI=5S<Bc=RJmcjX%kKoWkdi;+E05L6?DsDH}a+rh_`oGY51`O6lRIkqf*=*t?E z-?YfjEAFQJ&5BjxXM7yjFcT-6FlJF)4FjS8=etH8cJwi(BKCy`Lgb6(>J6$^dQbT0 z@;@UvTRtjkIkI-BwGkOrzfdq8i7F3BSrHw13*u-*KRGkKdgopsD__tr1aUuK#5wv+ z7y~PRB_H)5NXo2|8vYTMsJ`4p{u^`jA|pZUd0(W<>0bQlcjc3fc*L4dU(m@`vIPR| zS+_r*mPTkKd!G!t^}Y{u`|jBE`y9e_Y9r3u>HAtmZQ~2y9e0=lFA=RyTA+%gg!41{ z4()r3LE!mmH5cQzVIC|QA}PM6HE}%vh8aY;DGw}<3MN<Q7yq-A-DSq`I4s$6`(?kS zY<j6Wb#ek>0ym8=s$W%Ng+5dMN<*z-J11gg+HX3PZ0K7tdGE^JTFA6`^o`ugbP?)r zI>%F80$L3589JWA%Lo<3b}QY*I)i=Ajf^3x01)*O))z(x&SPcjS5;ShZt}|+5Z0~V zBHiB^=^7KG#IJCp##Fk@_I-cFqn>InpXt-6(DF<^fX*kfg0m-Hq5-;AYv|vknST(J zs$eelrazM=r4Y{Tk98@^NcW<~x+KNqM+?)8-S2QAD<8SR!>#VtQkot7)1MlO{YstN z7(<*mG%Ln!N#+A-&mEiSmkNJb=I%$@r~Z`AY{!jl-|4t(Y0qDAvBz~etl9auONoEa z-!qQqar56Z4%Op(buL>Wfvd0053q<rQ-#i6{VlsMCARSSsayR$CrEvdV*%Yx5goh$ z@pYzdLDlsMbgymlO&yHSdy8NKHk>kwvh#B`Uhp$(zLFp=aJ7hi>bT~-YMjU53`OD< zUBpKI4b0Cw7%R?v1?qke{zV1l>4J34AIhSbJR1!~wz9UhWRu&|54NWEa3sNpu!ttu z^q^6t;7R4C9gylLz4X^(MByIi3422MnP<@lNSCYJI%w9QVElAqM%l{%;?h2Cz%vaX zA*!i5aPB9+Ii;++#-`ycXlyH@;Dq9It1ImdgGj=WqLtvj5#Vae{(X<Xrrqz<ID6MV z-^F%iem_oa%>6oldSZ&ES880kTm_Ph0g`l7lR&d))%8=Zc0e_5s<0lf(N&1Ai%<{g zLHe27;1I1diUWPz8A2c~RKasmIR6;~WJ9T&O%22V?wl!N9i+m*lw$)V(`ACX8FjA- zD9h~T`WiiVuRznkLhrk*t~o^+PWiovanBVoh&Wfl_Wa9^v3g_5jb8(-B^?;=M^^Qm z+;(mEIDd4H+-z8=P)%e|U-@gDIMN3LoF1vB+FQ#Zo-2D7CU=0(4R`?LW$N^$#U6HR zN?u={PgH2y3T3`+>O_rUh#Lz(X$@VWYQD{U=c)$do8xvq$5U`fiTL*qBGkioY5K=b zXX+WI6gJZ5BGa0ily~d;WP7%YOwbN>i@9%;%77Mntb67bZzb~9Isak(pjUN^Hj-9A zxaLyop(WXmuvt$)yZn@uJWUaiRNL(`Vd-EE_!<DE>5pqYANf)BT@6pqwYf`{N{^Q@ zbX6xgY4|@;FvIm)CWo?AXh+?fjejyB(ggo+CPYmz10b9g(my#HbA?KZ*7n^raY^J) z#@pg)EF-PX?O9juz0pkzU|5mxkRQ4oHs^5K9(d3dYDqViZ@h`+it%iGjq8?Wc&%g9 zf>!oRmD@~GcSp&}RZ04W!6qH8_WURWM^mpyV9zg>*pFWE9+p$r*2_y(V3yZkSb6y` zu&iLFGN}lj$3UoEeXZNYuPV6X`iqs@s^)6A148H<wxGY`@ShBqzrQ=Bbd4alSt40b zPD)=Nn`VTTquJVX_(W%!HHR626oox~?=o*&80E4~xzj<*d|_K>dgq*636(Rf<%2-I z-4VjTit&R`!s2tgpaz~3{ImipcErl-+Y2=x7nh{z*R-xG);1Fk!P?ltR0C?}^VvJW zgZP8O$*M_1AFYl?+VAKJ`%B^Ni4t)Q`#i;_PUuTM`7En44CvWD2_846vRY!0Rqff< zKj&G<t<&d^zUW;#;#X<Z2qVgP$>=cY&B!s!PZl?T)gEh%8=koAw_M&;Uny&288&#M z_QtF$$glH~Q;$H75I><P06#S_CSB)+^(okGltpvVY$uq6I)80f+v+s1vRsaU*3H}V z2_4$ftGbL{q6(H)PoPVS1<9;laIReRL7>*!Xv<<(#{Et;+n3Q%TPPDvZ$<wJni(@I zW-lJ9SO27UzEN3?Pn0a4w*KoR6s3LL{%$B4EEtk0UA=N}gdfOR2&%FzQIiV@e6qVg zVTWo?0X$qGgrLM*nOh0Kq#xbvGL)Gzmofp0iE0^YTspI7@HAxP{GF|9)VX&2m!(3j zkgiDzj_ZxwufD%v*+=_SshI-6trg!~QB??q&lL!wWC|`3Xc7IT%gz<@T>ep>dJ>(a zw$fQYf%c^C!jndRs>h7}MFL%nE!v8UzRIt@;8mD{Z0c{Aj9QHa2kJnoGEpO7v`;kk zr-}-FM%Er6J@m2X>J=5K^(vbv!xP^>X=O{JmDIV#^rFyuedLpN{?Rk5cf<*5b04as zAU?~w5fiAfklCicYzBeh;)eAw8TZwIpcLPOQc&?7dZ$guz)XE!jXHQRPKC;FQ;uEr zB$r0USFZbPr(Zd~M!Nmt@C`a_E(oKUL_SIckFwdI*BmpqM|ub!Jk)Jz9tW2Ud^!)a zVN6loy_0|W&dv{PlIBIsfdyfz`Y1{cW>bG6oweV+$yW59+1cH@<m5Tg?-=7ttUWrg z=(V6JKJ(qVQ5AT<CDr!d^C(?uUaT`Po!Z+n;EVHaD~K#%SK6YM7%?4lnQd@TC%owa z#H3+QG*$;57@FnB(0^C?YwT^<41~%2bAEk0$C)5EWX{^8F`bvRXLw1B=69#8n!6nS z^(|&eeI;FpBdz47eBFul153_SA*Sbg_Dak83TxZ%k2ZDL#0Mg7SCmFB<|>j(d-#JT zNxQ};5XHyAZ=Y<$drS`3Lp5~HtO42oc?=MHHSOP|dy|7+S}&s7b(Wt*03BpO*BHjC zQ#*`XI#RxiDCA4@*Z=g(4s-%SjMWC_7XzRCR!Y_Enl7JQ32r<W)mu0FZ|6c)%Z<tw zTVQu%gCN}<+Ip&S4}fs>yfh3n=RB2L)ueA+kT0oLoPmYcp4#s0xhzUJ*}N5~ANXU= z)vrq!_!P8D(`WTna|`Yghv$p6e=s`_XI9;bL9}*cmJh@~3an3@8qd+|5bLYED2LlJ z7n&y06u?(TLgS&T_8|xP$z0)Z9@B5~13gdT$RYAlw)sbA1>-$j+IZ3A2!zM+>(Eo! z`pd`UlKNI_<sJKB3xkOPMMV=sCG(r_y`9ai?&wdbaD`x`kn>8Xdy~*z`p|jln}_>c zCnFBvX_cJ0T|BoyJ)-MiF65IAV38^w*@DLiIbuiV6m;reW@$|qCt2aDZC2cSwr7zB zQMe|voHB0&co?`U&G!f2@{#M5nWhFcx%#&dGTtTfnzOyX;;Vy@eMRDOZK7^9i*1RX zk>4DiNy_0;r4@fJQlwi}MWag4T-=%jznvIdx>Pc9Au2}Vx`Rp8yy*e*841T7tOg!- zHel(OT~uaIvjt!NQO%@c*Y&g@;N7hHLg7e*lsmk)XqI?)?|bs03Mgf}8K{|6@H3w3 z{$kz1r|Q4=tzgfoUG~4o4?G#rnV_9B&(dC6y5ywyj>_n|y79X+tHLRXDfkPJ9wybD zAC8>gv~R0FuK?l;x1(DuQcAJBYG#Ln%5E%L@$52kn@~F46e^t5WPuG@-TJ|Np!Tj+ z3Ls$T-tSgQj%$bpyxm>)tEd>23Wc#c3j!aa;DD*lqMD{NfKTM;Q1s+FDM`+l0!v%< zFW!}?BB{g@PIeBmJimhM0i(-!6GK1Zs{d0%@BZ(}3&I}cWJ+@27cOc^((wVZ**ozx zy6^WbDmad6e}h=Z+c>Y@SX@dcU?-rzta`8iYL7dSi+|6zU%_Y2O2VL~a*f?=YFjMm z`_Vk&wDuhFUgpVmfusq{cdmXHFj66bYxEo;(|0}vO99t8nX#Z$JZpzN_iDynEHV#< z3S2IoBC?J(v1N^GdaR9E!_FTip-w%OxlN?j_JMd6;uD(s5`DUflAJvj`KH$A2j^;_ zHJWlg#nWW$)K2E`eeuoV;|FRSUJF*)ry%X-Eypra(-vC#O8P$zN1qmI6%dd?Z+<)W zhbcFsLFsEipvv$QOM4bNZB;|Z6td)sqewJs2XKc3>%{{&TXmda#%H1~m!y2}$@}uR zal^jhiw~)8bnS^u-+reya4KX^DB>A75yj}%>01k0_EiHy8=#}ruD;Fg1@s^$Fm5ll zUEucn&8|}Hb`t}?^Qr=nY_|Mpf0{wYMZ!YMk)hrjP}Opu=J0iVo}4u;17u6e20#Db z+f|6!i4*YNQNz_aE@S!XfW%25xi9)`7cA5KIjIU|pWbor_x(n9;exQvq}bA_$E2fV zz%iS0p5v05A4tlxaVXf1<ElVL<+>}-ESjRk`d-<7rIT{y=??|@Wq6UhtosEOKh1;1 z$XBe7zX-6=ei;UVMt?A~T4bwAWv-U~w%$x$gG?0{j79S5=N$-W&{lAt_vVyQHt!qA zVFG|j-1m=l20LF$f%T?l871OkQO!`EN2jjW?=;jiR+Pr*&r4-5&*BF;v&uXqt1uvu zvcu_zLqEi06@`4L^i;CCRSb?|{|J(l5G)pMG4R295(&4F@*qaYOlQF$1^Hy5I&oEO z`y(ZCUUS#tWpd$Z$q~0kzKpFf$v!S~rT$BdKf8*gxhHj^MaSouGX}>AGl^?uK&+c@ zezT;hB0QUbJK#0(;2V8ZMzl~e-?BJNwj<(^{6HFjiZby~DzLE7vVML*lO|VneR2G4 z22L|x$5livKD0lv(#VUL^p`!Y$V4&32)J+eB4J3o`REY~?8jK1V@tHBDuN25eDa?} zfF1fCV;F<RU<yl7KF8Rt5h*QtVHC(r?<$9CQvF~bY%hiq@6{6Hg&Owi^LUt4VF$X( z&$If)4AA5K@usbat4Ejwe=rNakt$aKdw4Gg#NF{5u%@e|sJIpi)USqufKdN?JHidV zq^z0^Q4<ezzGuIz=PGtZuJJaBH&Gie0Z^qBp4Vz9?|d(8l&#jZ^(vz5wDRyvl@;(w zG!;Ir1}(o)l$`5Tt%C6EZl;H4`p%AVRBkI>4r>2-2RLAxZaOUn-ve!yHMbCLMijFw zGTs^{Jn6%(i}q+rhI*mhm)Dx6Tv3wGt<bLf@u$0K$+f4w1-GAWq!Jo!eLMAG6>G<T z9ah-NOp3UBO<&+nC04KFbnx<?zNBW&--BD>^k_>i=5zymx-r)x*)&ZRTz9MKv(DDB zc=){w`ayal-ziO;^h>ERqNY>=gMI@#-0S0uu$aQiHFNfb<l}{A7)v8%x;}naF<gmi z_SI~6$bZJf!Xacqc|if1H7ltEG4Xh$J+1fE<IDROoc<pz_5W4@Jd@|3wm$adrFvXD zfySR3WH!R_CJ`$Fp9RMYv?i*|`X%0|GKclm1Go(bZ0ID92#~5`zYUwUmDai_vW;M> zYld|eb@B6=Yvvi{YSaV9ZTw3-tDI%lx_FHuaGpsO?8(ro_&V_|7;zTBh3zM5Hp#ex z*hZknXXx+wy%S_9op^lL5u)KyMxJegstJ({K-}@=3NQY3cbY$IkbvxxdsYg6__@#r zr&Cc?$Q2j|3rwXMS(FPR&O=?!&hO{S?cQy$`YT2wsh$X25Vc+ZgeQ4wf9YE?_q$bC zV)G$bkG`~DaS)g+?J+e)*i`@hErEU~$AEWfF#hOAz~laUvDfYW>3+nojH(Pe5#Qk3 z*b`waqgSq@N3ZlYsXT-pA>6Xet=6SL4&$ggH!33Z?aF306}R`Mj=x_}6|C5H$yYfr zuWnoZ*L>3ImGDKn!-@<K)QI*<VF$K%fgfq=SviO0-wVaxD|U%F;-`D}@-zWb;i-eU z^AL6nr8sTZbqidR4%_0-!qyYUG;QQYji(&U*hi~R(Qff@?zV(#gxl1h9cC(G`61Q1 zZ$nSb%Fsa8(JCeDqgq~hru)8qobd`aYQ+c00v2FBm=1upr+DATio|`lHu+Zz@DC5$ zz5ks{fBYOc9uMAcN48fu0q1I8lO;~YQ&?X=E>l9s>)itC9;#+%G@5nRJGoo4rtz>w zgyCTW#&mtn0xkW??zo%Cm7k?dc{+YCh`7D%v}z`UN9rIlC@OZmfAjyOH<5pOsS|8Y z*n;obz3_9MCz1ehSl`RLPiN>8iq!{1wc4bhAtm=1tX>W>KK@9hp0|FhGo9|cJqvTy zwpUI`WUbCkgxHRl`Us;3aQ(db%=z>R<#TTeV`C=?{-W<4k!GsPRh)q-;YnxN!iM@M zp?=q5r-aS`Zo9-06i86oeevn;*wZ+rnV9O!iyL(XUfD<!ouCZO#WCX(A28@46&Tvk zRyl!-;y3jhM6w?~AzPv91!wXJ0;9;9;>81A6}u-x&NUYRM+cFcmAM=QKcnW;(_m8S zNR7#VMn6;ynRO~KVZO7|Yj#%PbnJ5(EReE!S|M=O_~AF4@6I9Mf9^@B`+eY;EZut~ z=a?m{NK5={kM1e9QrozoR(`Z8dAiwtqmR3JL$Kc61KtY^d4*4ijj+IHm#Mw-`LfWI z;&}n?_~7wmG-cXtx=77TghHE;2uUG4_^-OoDTz)v_@3kX$%hE;Jht4G7tufB-R>_> z+%01l7h6rGv-7>dK|MIb*qF<;6x{MyM(1a}nc6qj#tlfT>^la!LZJ*e<q~aX<xwA5 z05s8M<&D~pfi0S55DSU98SN6soeOYHnbZ7pf(Lf$I>*x?n1jS4RCxTA|HIUKxU(I; z?f<d0)aamA%vM`ejZ!2rD_U(;hdtVw6{9wZSuI*4Mp0YQ(h_?UbP;=$*sHdPks-F< zyPxOze4pRp58%k*&VAk2d7iIRgBTiaj(S~{S8)9FmkZ?+H(xg5$eU&%!E8H4fo$a@ zl>%?_O3Z#s=2m$%y6T0@pK_DFcN!-i&R-C9#}gA)Cqd1o%Q%(I{~O+oOf^o3uS0}3 z#Oh4tpD^IE(hJQS9J#o$Do;y@MA$dG4lk|w`rXUV`+ByJM=NK31U%jdbKPqrm7FXK z!QI{`kx}1m{DHa`kF0?+qgU>|pbNpp{KotB#mqk4DrYvq83BX<arc7nrf1Ijk;c}t zy0+zCcPN51g7Qpuc3T#G-Ii^W4f}qU&ttz^PxP~i)!?ak<WxD|cB6%M-+VbVrdCxK zj!^1p*I+m;ZbrqQtF8KSp)pcO2K0W3(w4*YIo=|fo#&Qf=Zux~%PDM7iqFB|{gbkQ z)h~cG_INd-C&F@tx(e|G(#THZk_?&ys*XLiIal2#O;Ffc*MZRi@LGqs5NF}KM8tYC zI22dRulj8d$-YM{IcRqFc|aY~%4d-Aa6!0Cm4m<XVm&_qkL%Ixb-QV*-8vy{E-*60 zf(<I`WcjeZ+B#1zv={-v$g%vtTb9`e$ft6{E=3(T=X&$^Y`*JCK9eirUW0P#<G^6e zH&T;lp#EX48{+95O)<L~ZI=1Rb}a;SGdh;krh4GNgV7^a+fuWPi#=7x1XOI+j+E#* z%u!1F%E{fP!>SGd;V#>5T~?S?FPN?V-*U8>wlz+2^8w&%wo^Qo^Yn4RT*WwEO-T{C zN}u%fXS+l??@~gIw(Xaj@}L)hONv@4^g&!GY;2Qn%e3tQQ}hM8TX(|2xHY<KV?m5S zPg$pQK(XC|-$lCTsdX-Uu2uG>R(Jh%{ncKhv0P2z(}tkN&hWEKWS>v}&iob;z)4k# zc-ha)T10BKn(itBnW!C<)93D!_Z!%XAh;7+Ib6yEV@7js-|KD$cKgaZgi|7=GZM-m z+;Uc<;30-#Dz>SA<m}Y_6lP<u=u59YSTS0NdEWIGdr0J7oS`+-Qk63@kZWIy1WrG= zHTNdZzJXXXkz4?ye*H3by!5!#bO%a-r@W=gg-jqyu|EELUDnOd5814?UN7<nZ}SS5 zG58vO@jcrhWT~f(<2FfI?zb`DG&j;Jp)wb&mqq_VXSJLLm!vI62BaL3`uaF<DV9D% z=Hw+;w-??2+k2xlL%+X%=j6&O(g||obiNMg`qiuBfsbR8gt)llq}VvSwj~UnoLuQP zE^iKhV;DJ1VxWt*+Wpwp#Zgjt=H{(+Q)Bel_1#+x-&dvK-niY=k=`E%MHwpe&m3Cq zk|h%kAWCMn`5Y9oBbZnaEi7M?xGg^4Z}ang-7r@<3ld+mGul`y03f<d9mFh`KOckB z&5^b*SYpf@nvHEB-YMRZKGJxl$RF*B21|2nq(e_>bw+oxL7L~=|MOyGFfH7Vg`CsQ z;Q^(YyBw|D1^Ihb`WtKj60Zjl^V(E8P+s{RA$<?4aS?sce?iy;=JmUp@GE!s(@ECu zFG-(9A>aWIzrJj}Yttnay!&}-t0F+!Eej5MWQKdKw;BjE$>9A;r}9g8?VUDFlpe9c zz*7sDz#j7Lob#uih};ON$*ieY&i3ZD!2cGH64WQ4M0L7?#M8|~r|A8f8VK8GbBUU7 z@V6L2ln{7U;EzsWgN>)frE{M9LT|Y-NCH)8O!Ck7u?o5#;CWPmP>yjM24xD2K@}Lv z5~VGqF16}#w&b*EMq9}?t=#UCZCZ<g%XVo!oNgE^lGU4xn|)+|>T|T$X6S+ylp+WY zx!-k3Ek&#v-gnO5QU5~RyNudvl;J5W?6K2Wv?q?x!U|e`hv>WcD8i^2x#NZ@a$qmD zJR>Z=Ely2YMjLLFHJ_Kd;!Hg19U51zp@+F<TEgXozU~czx#LVW{0An9!!7s>adX^J zmG6U|PK$VT6df%2K>28WujK4ktWnQO=C;{h)|4zJ!o}Y!nPj_kF>pjMGj+ui%5tqm zaI;7Jj7ZC0qI%1Q=2hTa`?(5~@X_O*0#dxv>l5TYfDajEJ7+H{f9kd9mIu=-m(XNZ zR-U&$SDA~wb)OYpP`kh-nOphclX9;@!B9$&F1erqRlwVX0R%ESz*7|{#KOwHx*b(; z;EQJ-+jeGPY+8~V?*j*#D@nH8R`u<=35Cs3?RqEO+s7HsH&vSv;zF;yEb0EqP;~<A zG!;)Ox2~>QnF0=tO3q49^^m9;3L;nR9kq}aJ5jtbrl4=%6y&JQc)0Nk$cc=wM4b{5 zhSW6|xiUhMx^7VP$v19f(Ai=(I;*mLd+AyC@aAroz3L|c<cUrx>%Gp!3#ER)2l`-u zr~f*y#jEe(2;qZ|n^v%`Scg)qter6ziC{%*Sp2tb58qC)@aP~_q+p$o4j$DO&p6tj z5iRX~(H8*vs3em!{lOClY?AU+F1-o(J2tux<&$cid9PB1!In?<yX<C8&(~V|#0;gX zKEefCWP(W7J(te={u_S{%r$1Z55{YSEFQoJlcE75@^_i@6S>aPd))cBMmgv1S4V$* z38nMA`8~C$0h9<ntI<dX3LH8)N^GOJTff@q`GBeP{tf0Y^HDFEt%icot+j(3k9at7 ztJRN7Bkkq_(*%eOQi}s1sk6!rWcRjXN8TB|CbOVIkcl`|@{davHPYus@9t#9^YhPo z@YBGx^;h^EP;|J}rvAIcTZUd3?ox6Tv6Ly89g2RL1$d-a=ZR1L7rSZp3~i#}3%y)j z+}lp^J%b!LStWlnz7P^k&;BUrOm2+p%1DEQYjl$5?dObH;tE#doOFdRC%uJFwoXHi zCktH79r)j$ll22$T`ONK2$zwoYV^Eg%-~2vhF8)}w6?!S%a*n-%)0(AeJ0H{BiIyJ zUdl~t{f+n|Oa_|%LSA}3y!B12zf5|2#3P@Etk(NxzKTATvo_)1o2s2<f3%BFBxY9v zn55_5+t2=@F1%ipbel|IPvz(}qTNpW!zPSyOr*mnPxA{75>SWsL~qBQVIKq?4Ok+Y z+=sr7M+wEF95Pt%4KhqOsgx4Hk81sjnXkuxtj0?Ui*Ob6K3DXqvhL@oF#Y&`1TONB zPZkH3^~U(EXh|_cj-x=Yvg&p$Y=$|6l&x}}tgQ)&+ax0-fPS%w6+ej$`LgQT;{osK zK<#YH0hW!G$@v)!RZ4g#$Aw#krrSmd$nmWAwO<{3@qt0FiMoIZK~ou0kep-Xw(&O= z`BCxv*)iGEzSd}PJo!C7@h_W-jamJJP5)+_m#8~O1{1W0G?kQ*d9utASjEkjHeGnM zc%(y7J_`)&SpW$9v1sjF+D1<&Ym-Aq*4q^S-YapRpP!f2`Re_y)zL83pGagbxyk-E z<XIAke~iJj@qz;1$5U_nug%S0(d#(YwsRXx4;&8@Rc6Sz<Sz!g+#XuXRx8;C;R?Vr zKb<nqt2SQ3!mW!o9A?eym!P!b#b!xgWj0vs0&ru{sRreOs+BiBpX|^F2u>+~7809n zwu6%_rNX_j@0QFypU7{OO)8sQ&UYn`e{g%kAnvbRIQP$hRZJZVo;q`ZA10)ki`~3_ z_8WGS4I$!jw!-3K&1Ycyme{`nP|{KgAd6SKmSHrN$Ap=aqZw;3wGfVhHU6W9>C0lh zetN^zF6u)cs5%fE#7TjR{k1X%0c*h)8vsr0%zSC{{@P*_xp$(jc|$dGJ3Y>BN^2$K ztWt6iP~2r&%_Q&8q9AFbxsRZFUM{h7SYp(0^>!L1aBTntws-&SW_6BXqJX`p*=Vu{ zH0vhFzGA@9HCAq>@@owngJ~44#+|!T<$_kM(QWo~t4ZB()xL_UevXDm5my2-m`;lm zoJ=@EmrWdi3!r&kMZ)j?Y{1fQxdpl{;1kp(a{_0Xm>Ak3)u2=<eT9+low3yQZ?iUO z0&nFi-36$YUKUcN60A`RZahH0*ur$;m;!fwwU90Ei#}~OGw-usCFNZQkfMudpT5{% zjWq(8`!mKTTWQtl!7_1IA9pQ}{@)$`GBkt9%o2U-;^)|E_N^O!#l}VZjVPuNL2P`s z_j{jLN%Mw&_a_vJwCw}V=X|-zDOlINj=cAb_jlX{_4C$4#9csFJ1|u63sGmgo(vR3 z(mgV(0&KVoWq-qG$di-BZuYuuq-uW9hba_aC!*uW_Vqb{vA3>jpQrJg)rvyzE~ADF zD0E|$oPun9VOr}dskf@@PRMLSl5c%$yxDuMa{XXEW*6hX5;)`rNf}v^54u!^n`(a< zXu#eNG2Oj|fKLT-6P?EfXXugAmI^3)UEd;C^xyEEPmP-=1&e82Gta?kj<OBAe~+Sg z?cYPn@>52pkp-TT^o*uusVuOh@IRi>pc=es#56EhE<z}ur1;|Ql%PSsN#ljhsTnVc z6Vg%b!STrDu`c;?Suy}DgwWzQs(h-ZD<|{D|LiC8yj6{_%d5^S^WL2J{2_Ba<VU;N zbxL}%!%=4hl(LpGa*{M#^fU_PK>?I<0mW(E?W|B5V3kf#@Nlha4*o>edG-W>-r1Ol zr(Wyjhp@Dp5T46r9in^X_Dfbx)f#EKy;0IB%yFfDv?N0B*ME2SD}fRqzI|3gT!hj8 zc8(?D_)47J_qKSBu=ocF{9-zy6@|Srv*Qd~cP0=QI~g2Eey7gHPvg4GSWL~kT3hn= z|I`(Nq5e=%lVG1m&i2paqStkhR~9@?cGh;D?eFK~PoQ$)7UyErFsNU_@;<A}@f(;X z`0Pw_PltUZ$6(u#O6uQUh3QT{9k;8&JKwn>Sspxrei~0XZi7LF<<=;V{A%H<o$Kfm z{|8f=yqI)VU-|i~$9^!OD6^61sbZ+?*@FQE`0c6W{Xcyaty#O}AV9jJc8oB^tv)&< z>M(ovl<1$y2-oiKX^&tDY?+VOctW7SI-PdC6ggbf?c{_1X<$B0Yiy~6EIdb>{BuXk zWFM_NC|mdrKQ+d{%(47jq|Ys+A|Oe$=k6IJ;9B=wWj<PA-N|fK*-Z2GkERzAJkn6} zee*QYRu4pW`E+SWoVSB3@DwhUa0dRAF*+4~<bSKcjZ8GM+J%otJyXzuvm#B-%v#;8 z_byoao;rT-^&nX{fh&WBe!l$dt<NeCxq4o=aYa5|<6<HK?KHNQ7-jmt>cjBytfhB8 z#G140bcG6WDy5cmEe>j}cm2-N@qJ#;2W$_aH8PC|5uXn&^7sepFJ+j;$`ik$%s9`( zN<i6p%~7$H0lZ8m;x{D`o=_OMl_Lu%-fjBQdm%r_=(kH1N@+C~HxZRSQ@5bfwd`g^ zNWMj9yF04P@_&_i&S^}=sQLciKBxcP*k%S(Oi8W@x2*;QY!>;!5A!4+14kRT_n&Cq zMRQ9uT5Y2)57hV}z14<3@f5^<1+6YT^{YqqZV3DbToKKCc>rd^*K~u<QkZVe(E{-C z`c%1ov)@HB3fHz0;*Y!!Z@W3ws@6JfxiQ|hB?66#G7qx}OoPKkOmwKwD=ylbF-J(j zF>boQ70Q1-*-iIRK($H9Q}XA{CZ!MM1$H&yxM#z}Vj-EW8MZv2@vZeHcBG=Ex>iRR z@@x^)zByJG?|%R!<oX%oU%KVCm_}QX=r}&8=Z!c9>jp?JYwa`{O(LCKHjCv4F-|jO z&e6aVCW3i0ez)@Pt0(Gt2l;knoXTgrU`9sg%<@Vh`+-W76uwFBVvi71{hF^VOPpuQ zWDDF~k^nhvPFqX$-T@S~K1s)(^4{hqYje)tv1_}d2mM(+079Az)yJ9F6z_rhi@BhT z%p7RI_={1qfaV`VO#Q<~ogD_LEUe0hQ0vdx{VbDZm}|$dgj8+y!uEr>RD9|=sH_R+ zX@N}M#br{K@%}-__b|fdr8JQIiyoP#no}bBai`y+#rsP>dott1MooU`gChqMo)@Xz z(bdd39TG+9Yc(`<Ou<*rP6F<=0ZpTQ5ZxzTQfvb&uwXy?Lz}Llkn>xs>3fD=<$V?U zL3a*xK&ZuXs?_5m6P_~<#md_~br7Y+G}GOWe`CEqr6@C3$9NyP65m}=$+HO!rgggh zK0ETrjotV^fj@^+^-A=Io%K+a!GzD$GA^CumFAbLTo~vevrGuAvL)6d1O|6Dx<E^9 zkRfV%N4yYC^Ob-X*qZ@eVqY8Xv?e;Kt)_{F-1}klqnPK7U`X)D*bN3H1nb_vLgxkd zrskdMQW*o}U}7@B;hkvO-#%Hgmf39Bm+A7&CLf;3&i*H({p)eCn`sW@47g0awVJWm ztF~#;T+7|{FBO<PbAX*8=$iuP`;O1@Vv;yl#UQ!xEQTf{V=hwp)2&DUnuu*$hF#t* z`oAy3EKd0Y@)O;{y=&d&|Fc_9ccBXeqceV1_Diisf+D>Ffu(vs*c*TszcJ%reEp&I zCF6@K5oyL3N$wYb!wrip2C9qY#^|>#+cu40;-IgL_V6<2QXgB@2y#;@N7>md9Zzc? z0e~8iAHnznF82_S%DxkgHjoWlfF^D@bGz|^xx3)MQw=Z8W_qT;x&F-bAy?fch1@6$ zg$8STnrpIXL-WvcdR%&7Lk9W3W$^ddcBNY_9pRQSftABB6Nw2rzEIz)ARq>@C}<iN zy+oSSnI`K~>|L`S19)C0KTL3h3gHymB5QAf%!>>u+dy{D${o$T7J(~M%~W-tCroh& zC(cmr5BpBh7LzRDT%Va<4F;IIpTou~?@y5$m)rJ}!9dh!uZ{5M1nTY<x(|tPfDmTh z2QM%OubzM-^9bRBBNf3#c)#y?1K_i7BgMUcCfxp6Nz!E1J9ON|ovT$nUh$RvWxL3U zrk2Qzq*ykZ%Tv~_^Q$LsfKipx19kD#gf+pj_ae4~V{4I+*Kq?Z$<aH@g_}*@`V$7@ zf~Mx}9H9{Lsc(Je_%XJ3qJel#&pv>~tMPaz;A>w5*0nOPA$?nzD4>TW(>LJ80vmy# z*dIBu{~ToA@uQna82EQQOxpzH&5R^sWar-peMzwjuEgW(H!(uU8Lbd<mJDP@K%r(- zdUw=ZplTy&ORmDf1U4(ELE@cKLmntqZ02(6&w1NTX*Z{vpcGqp%NwITy-vCyS0Ze8 zAkv5@vIFGYEHs$nGe@?rKmr6I^I&y>j<-QU|H1j@g?%g8^f^~zmhT6?6HXo~68O%_ z_=)$l@Sl&YRYlZJkSV`X?2q>N2NkIg5)3no8lVm$_E*(Tb9H=tz9M$I>Q2Z_dYd~T z=LT$sru-Nw!0Dct3%d(6kknbd`tvPfdUV@jx&F%!<_lBiz>=;ML2#Vu2zfK!HjT85 zS12~FA^YeizmK>a2bN}dM5AP#3!MgL&qfC@&)F$uyL%2hE-z6f=1rV;P;1a1kIkF> z+ebbf7e!(nr+OFI32f{Bz)UWD?>dm$ee$Ci!b_KS&}@S_M_NO*DNJo`hJPqqqR{7< zDQm<6mKAlN-S7vmk^%+QObE6*K9p^#n<M|5#_MMRvs|p2SeNq2@A>4T(t7*)opD4@ z{S4)cvWz<QvHHz#?A1YPW~HU(Vexy_mR$1m(rB4#pW9D)=WSW$^Zqi60@jv))kdWM z+SrI?A;%L6r$xlK@`q#@=I*e*The?kamtcA;9e<fH{aX6g?(S}54Acop;vi9k@Wsa zFp+~qVO<d?phod6uSyVrP)-?P1?iD>L_M!+*Lc=_uvt*hr$rxKvvo*~<ER)nV80Gu z(`BAa-qCs99X(Z9Ug~-Mg|a>~vQHpvz$~S_{=$jrNnTKK2zubb944(`d3)cr8a`QF zV~}%zy`Zev?$)%X0;fBVG*g>W{Ks_rzn^qOsMDy4zP=Fssll&EJCN=9oOpSMxi|oD zj&chhfkk=oRBSL^+WlzNAW*@GUL~{Y{Ht}m!kQ9l|M}l>rKGat7Mof3hP6!rigsTN zl@IFGTS<f{x0%oGv+m;ejOc`GLF3$`V(sx3;7rqF0MC$cGSKjxXrStIscQ3`-PM+= zT1nSCU@0FsklQw;z+(O#Pstg8dH)vhuh`qSMB^<64Ydmd51uHTOgtTtEr9J#@QnYV zbkWBfUO_CR4=g8d0(Z6wWLz6+Nk<ioTvv?%%369qf_ERFlSm1h(7oxR8F8V9&xQu8 zziq5d!e`wH(NGeWrT=#C28OEZQFb5T@@BbxPl%jM-F=xk40j&^T3FmM0rNB}-w*m5 zDA@LX(WQttS38uEN7ILfTTk1$PBuafsLhN2fTA#X)l&EFr(7S?o04ll;cbeKVBqG~ zsKqiU%^-gB{#c;e=@ut-|22O^%OcGX)v>DyuCQH1*b?7kl?8kye*UxQbd?@a%#EDx zgG_b!kdH4UEL<1?>Je5J*z;^bKiJRb%~fAns2m0;lMZYFRQiCyc0hkHK8U7vwwgE| znzizE$-V#X#Ko4Bxe&b{LcJVvV-WjjA~Fb#D_!Q({3!k`V`(0tOVV4(Z2RPHO!x4v zaO{H1A{1*-Au{jJj@Dl8ua~2zVlw`^Ru)bUa-{Z7j2-qAyDXxWkG&>EIW08E81Jlp z%eet;k(h7pY*10NxvDc$fZ(>Z(oXKIkCKCJpw9*$a6+@+{L9t5@VvQa`GOPZ3k=qM zW@L2dXBz8hMnAE-G?3`gb91DwMaGaeufCtBZWgP4?C04XNDJ`TQ$HzHmf1XNm04N_ zs@?yR-WddV_(#&dKRh*+cK2!b)lcrPo2!MPndQn1T2{kJ;&IjR@Z|W{TU+6)^kg9Q zOOM-lttk6Hwv$n2wv`sBxX6R^$V)fvcptAp)&iftF6G`$S=12)f=R7XNbZL>*P6b$ z<juj)zaVz}4Q*7^(Ity&eELA6B;s?%2O9Vrde^#@G(dGpEwLJ}C<>}Z2i(J=iEc)d zy52Zk{A%tuQGE0fxBA*>m&PpLM8|GC3FW~#Q2R70&(k*aBzaoz@UAdy>4|mE%S9=( z%>@W>N@0$}7YU*~GA#rodczpv#uRYR^ESQ~<hNI=w=g*mE&A_L{-5DA)|<^)$=P|V zcE7phN=Qb1DwN?-{C)Ww7bc$chH5xMnwft3dy3_1v%&DQavje?*BVg#mLq=xy%>~Y zXg=B18jActDqoOkPI~LCWqQfXovm%0uPO&zWx2zl>$obG<?<g8XfJbpm2YA#EoQ%L zICPZl=ADzoNlJx=pvwU+jCny2yoG!0TXhVcAk@+J8+n5d`RJ8!g30}8N!Kw0Tlk(< zhy<IxSZWis@AZ}*4-nTo&%fx_cf$Aa=eNx1x`F<y?Hpmt`D$1W!CN=p#2FsJ%aW8` zZP(+bEoz$%@Q$2N{Oo+=#z94q{hFWdCrF96m3lRPp8xGetp`kqVR{s370$wTH#Wxx z_8KE+)FPTVhzno8t8Jzk$>_|tj|W4`ZHN8)>IRO9d(Cm7RzQ@g_lEe4j>z5QJDZwW zfsMJSZ+7QaBUl3<<0+0k&P1n0nFFr9=JcbijJFynZ?E%}Mu$4R*PGzz7fs&wv|mMn zQl7sb_rV>AdPngMq%2~L0HxS|Z{W`=cKM{3J)?k9n{-OV+GlJn?z;7Ga3tk&C9UBR zNafF|c&6Cbb&AbQm7kuay>Sw|!P@}t05gRu?yA1F#I%;R`DvC(H+>%24*3KX+O1@d ze!CA)+TzvooBx3e>Cv;MnBepD^ZsUAdpM2@mMFXF!M?nksJA`owOJR*7TOiIE)Rio zs}eUFALJ0p%U6oHezHM*e!jC9$HxaB;3xr@xI>1(qbH|kWKX{ZMX<U(NjK%tQxyZF zv<|90R`cVewqc&Mo%`v5&L{tJG}scHmS2|0Qa?v)a1}3?a?$mcB=v{2*4xnI2n=jM zm9-<2U#A$(wURL>Xr2u*6{o;ek&(DmzsVL<W*U2mFViD{h_E)dB5I#v`>z*Ze=y~G z@NPu#O2_yj=kY9OG92=kd#PG+<TkHmRRASV*rk`6W_-9sN=RvPHjO>Zi$fS<%$bW9 znY^WSmYls$C%=-XTT}EU4AZAthOb&Y8Tfq*#F>6J{sg24(5}yE^oE}Zs{fCvG?gc$ zFB|?wbMDU9uyx2<=(1n&n=dQSeWB3!YpTx%8?G?6k&FS2rM*x%SHlX=GT?p5OK4by zt<1i`hajlytx;O*q-zw+cfh@?{D-Ml3+OMTG{QOgP<*?$CcY0R%sf81_E+h2tnt1K zL)l$n-3DWQC`752<OZY{Yf@s}8DW>93qiz=6ukc>RRjDE=Atv*?R5X?G|&GVlGp9n z^kzQ>Pu~p$oY#r#+(7vfxABA|1AV_mlM;O?)mFoeW2t_Zc?{>rc`wJV2C%{Gn91de z>he!C%GI-ck0<_Htttmdy5%Nw<<RV3t@?hg^lf6YLba2S{4l$tJ#w;)dH;3f<2oh{ zc}BHwn*7H}BJouiNpNwsuJRYe#Wy81`l~VrF@KG_#?~~}vEOGpO*sSjJG`#~HO|G$ z+ifjRm35EtHMKzRT3hB%D7k~>VvdGh6<LAse9?*#mh7Fl<VqC!ATX$&cPO+Ne16Fh zTK9ckmhG8nHVyQIP3Ox493Y$J!EJt<E2F4P&=wqa`k<eYbZWw&Gr9yrd(PR4{}wo% z;r9D6qi*G25f6K<97vH-w1;h#wT^AFTx$(&zwdpb;d(TTwm}`_F#`L0<Ge^i2Nm$z zEOh*CJXwTre6o%`RfW0jy=mU6`SKTn;hJNd=3nyFLIY>e!Kklco+!tGA9v)+(>v!y zN0wmDelN6#A}s~!`OiscU)n4zufO8-OQ$u0D_!Ru|J$4wxsks@NNpAe-GNJ44o1Vl zy`~2l*uJkfnZqh+``?^)o+5~wwEO-TUcgu$n;~ij+g^FrF<{=Jtd4A}z{kG+<<X~2 zi?)i<fHl?}hAi|%j^9SE{8PL4p^r`~xEvKZtA*{PMUXm9rI(6m3a2e)j}`!_yV7at zAig^*6IygvpzJ|ppYB-p@^kgWg<RoeV+i@z?@|XA+vkfy%o<i_c1Ibii<twWe^}66 zw7<O_^I8Kai+WSm(BvuuBtXl}GT$I1(cCvIt2}*+Y^3PZ%A3z45T{@pi_@&>iRz0G z>`ik?gVZo=Q<74t_t~LLQ%tp8ikI08+l`X&Gg!`h&r&-sTOBwwd=#7gV&@6H{z!JA ziO)}+?NMyx5pG(6-U>*`Ew--*VZ686u6Idzeq0NuU79a7+FT5}ONOsCeL%c9k#{d8 zC>=%PGYvU8SvBTK=v|8FTek<JhH7zdKylo&SKCQW8n`G0;unq@M(#gsOzB}<B<~T+ zT0h_w7bDO(Jim<&4p7<tb*`%!Ao~9u2l&Eu8D4{T7GJYnFYMK{={;IW?*uOnUb<rJ z!?A%W_Rb@`3}D+J_I$Mwl`jZy<15COhic&BsphJ&ph$T4clp<4fTIPdKi8kyJN_Rp zNKIA9bV0GYpyQtW4O`~RFGAimEF$M@PFYSNX`g$4a(}#bntbPC^g5rZ^#pV+;tJzE z4~GDa`BTU9uI=pfJ0Y-Bffel7*bY^Dg)OnZ3&iovp6tTx#fGy%m(2Z$wUJ}@r?M4n zOuuCH$&Bs!#y2gNPe-ueMjJ`I<xNR7dBG@f!^0?{eW+7}^Q7+FJhzG@)d31==t{dS zM;k8si;D0CbO^)j>NVbkU>V#+Orxr6UWLAJe+&0s-rjg8@2Yy>hADx@>fEL9*8=tb z+-o`Zp{%c@PmUu@c(6e?0g`+ln+VNSX>+w(nF0Rn<2_0ERpaxzvnqh<4F@?f!oJ)R zGl@DbwY~`|8xAbT2R`r#UEhmy%lUHh=l(B7!NHcjw;|46m>RyLlG>V`q^eV@HA*Z$ z6S3({2v9a}<O-b#CWA1=6}wc)<o>4t*E7ctn!U9WgW0q6eMlV6;17seD<Sp;ZSD>5 z7tFa0`Fa7}h2s{6k)z)PpxM<<=x*MISb`R3KCp(UdT<7|`8%c<8X&V5V#ONJt4h-5 zmGH78e`bCw?I#@HXw(QfuJi!&xzR~Gw!dQ23Zj&wtu=6thqce76ql=y^xSsDE8MEV zv!QFxdp^xktfX<H@P(2d$$74Kbg60p@KxTg*E|5@i%vmws_3~5S(|NKfq>~|<LY!v z%GYy_VjvJ5!Umm4b%s^F!nSyOCmHTGlf3VdLb=CYA&48}47JLWn>24>8MY7PrF&=B z6SZd9wK%O({50FN#7wAnymP$r3bQyl_6@qypDTVw^+e<#`9dG|B5<F`J}3?}T3c1I zAz3_naGXJ$A}<PNMMRrA*lmtilYI!N(=P4Y{f!Vez{S?%wR)4z_BpZoxc#@NP~1Ro z9`<qF#^v`~GNvEjExy-MVgIQkQh4Q!l9oeJmlS<Z1va<;dQ6{}WwN^>{om-dfr<-* zn#QkRrdT&KhULi_C`GnLBoifWMYS%AJ72d6u_LVjD4jKJbdXG(3W8y;HJr7{dAJzt z*JI=5ET%ND)BB_L>{jfM;9!y`^RPLA3xkFHk6<QNF5Gd#y>)F>7^IQXqyA#^FIAyh z9OWAC?@1l@k+jY<H+_~xiGh{<`o=e=z}B<cq5d^^h)2^gfGH%PUvp6($bYg$0VDa< z8?)(>x*-J>Jm#%syk*1HnRXN{T$JbV<&f0lGd^0)$NPk?Q<51EdtnFb(*+s75BdU@ z6uKC|TOlN%_NpuZz^+|c&h)kNo%F~+&Q8kgxO__2Zu$1gNIFExT&^IR?v!X=W-vm; zbL3kln=a^=xE+J40#HRtj~gq?o*$`YDrDY@Nv$gF8>wnFs|pFa!j<XnRLai2`t~HB zoj-^2<6*^tE#tAB>BVVMu{wPam`OMhY^3~<k#!h=Dlu1;yiqaadPopjwYuj0B^b4s zT!ETZk&0;IDy8jDr=5?YkhIF=eX%w-Bj5kxCwywRU*E*2%kSuAIZojqUYh#~q%DTj zN?YGe{fU|RgDW?jJFYY?BVTCjZ*?M$w#tiz;v32%(;4dw=09^joEozlx6b02o(@>e z<l7BY3>Vvu&Ek_Nd=2sXovuvY|39saI6!|mh2LAwZaEZ6fg7B^Y;W=buZ=hB!A>%W zIJh!iR<obuI-?`1d>(P)|BR=LmbwizmDe6OJ88bjHzq92@-RlcHsLpUaWU29{r-b# zuKPAn07yT3sVB7L{)icf`FjNWW4yCz7#gcQ4_9k7tw3Z0#G#S($W|eKX3=^szsGp~ zjSpSr^X1Ra(CZ>hWrC3n=JOEc1zM}Hm-TzCU--JLUbD*x4mX+?(7!s_35~ELuRL4K zDyaM{h;D5!mKu@7HW#?sdDnV?%QKk<;-rh!<ykVL0BTT+!k{08eTZHN>Ml+w2=HTG zV>t5?Ig)%le-p}8P;yc@S>5nA&{3b~%L^3<S7EVomYI|{_^I#b6B8cB$(SKETUqUY z2c3K&M)PY?2?B%F?^;c2tYPu*Wmol$Gf9}e`q#nsMAxRAxklG1R+(4VC#^KlX#R+S zjvEe&9Ow(gOs*`(6!ePj{iMPY)PZWCQDBdvcRb|PIr>|GtvW!HJl1H1na<6QU+^P% zR`~V_pAtrm)WAx`9DbWk`V#_)O`@CrKsj*Pjt{4Zeq=jC?#kUSb2eTj{M`#oh}<oA zu5i;r;kW&|Ad9MI<`6NFoLWNBka;Y%xTievt;T2R0l%-SGhs#2yBA~p@xc;|dY)n; zYpm&i%=q5`OG^UFX<>1>rTTEYt)-A~G`8H~cMF`EFFmpk^SB@yI@n$rR3$D=A0U=} z(VGZ_3_G<>zVM#rCJ4T7gYnahO8Pe;uqfb$kK#rdQ;J0vJF?94!GZ?kS*o?U^-6=m z0}3o%ade~bb^mu{JY$^kW!pjqc>}rA5X)GpJcJyUH`KH`no+@R)c5N-`O``)A`8|4 zDrhmOG{Ej3m2$w6wRxkkK-kP`N8Q@&E@SU-k=|5=(wA)`-^Iuo3B_LI_1^blN(-AZ z1pkuT81>U4_GA4Qi#tX_a{-h9ovmWkviC$b6Q#L15BALNfV{p_5FZ4x2Wm~(^^!tU z7fdRec^SKYbgge+>6ZK?yRUp2sI?TIn`Z@X?`RU8W}Ks>DW;YsE76qyEpPSFssksa zNa}Wx)Yqp^<@;s(BuDaR{M8~5Gbn1G&-caHF~e78K*&zG2#Dqrl4DP8{^ZmR={qa7 zvum|vVYKPUvcwJhz*$n)HGPZ?*5S}M7lRcJ-|73QEq_p5l`yT$wO9Wj9_-lc)Td{; z`kjuFFk1=9{>gbnZVZn7FV^5AkI4uRj=sH+#4h-UI{68|=48(B(U^=<;tW|QHcJu{ z-)bvzU8w(llYukwdVG5Q7&Aa4iSKR4j7GL5n!(SU6`w+&w&ajD?q_DT1FZiD@pZxe zc>1Z<nSrutUsl)yCc}6QZ(Ta8oqPnbykW<Fyt^N4J#aI9NjK23<G~Haw06TOnO(so zQ#@adc$zFyy<!2r^ggy|7`2D0LRD3TbqgrER{C@UJly6tN_@GcV>9nNfH~s4Rb}rD z!xM8*REF?7XuWmw`k7K`VKrjL;#DM<hg2!lJ&%Um=ks@@XEQ^EL=+qoRFEKZDH^o8 z{XNTO{Z$6}+kMgU_IdmE9=a1Ue_MuMqZP_!=d{mDBsY-59O5QRGoes_o8M^PCgtS( zX58Mzy~#pab{Yq!A>?%;2C9+e@6Ee{A79}svX5fzzS`~=I88eB=tRB-7o>z=cImB- z<kbDyK45lF)t_MGvZ>*OhLqlU#y{mNzU{QvgTnVaa2%_=@C~%OO-$a)_~ycO)SXA$ zV5PV&&N0U6U&ZqpB7`)Ot(`p{ez(roRlQ-R7Rmhn;f=e#B%Ov{GXk87^LiMq<;Xw# zey#ZV6~-ppi8~<)vnqXEX6=<qrAc(3Dv*=~VyHc96Tb2#{V%id;7fjEf8H(d0VqF< zim<C`o5}aN>OZezs^jV_l;Za8diXb5nQw2F&#*=szb>`q<ZP94X&SN8?^lx#(&=4k zTFy3nWrCJ$e7tN<5<SiBFdt!u`_UfRV~(npT#{}a*u_oX{1N04TMVwg2&OYb9O&1J zA4g@W=&sj`uRS=+KQe{i?a18nudS6Vt)x8$$**>bNYEuw#2zZpXH`rohN#kCPAa?H zaYufT@C9x8I^J9sn%w4Pr9AoSXKvok9m&imB><6cda?iDpP<z#V6`{vF`pS!^*Jz4 zX%OV!=C+PXxZNz~i5FQI-;`lPmb`_Ot>zYD7Wp+;Z1nP(&;KX>sV%1{JAPdzhen+~ zLpSl6`^zK34|-&2MekoLzem@yR&8&c4!E?Evat1|r5Ju*;*ZD&-@qCn?EpS?XT<JD zuN>aNJ3i;Vgx>PXwB&7NBvIs9;y7|h``r!faUO@L<<8fUd|XdsFQp^x9Mt{PaIqlc zmqZ}@DYwA%91b9|w}w0U+^~O;gnjfvZ*H6On(F#jmFns|5VdAY^NjpXYkS^br3PPz zk=lS#L`1+!Ct&9LE$4k&<$r-eWf!DdgH!%YQH^X~hR5>VTDn=h5kvtKN7RT_e{#v$ z0hk@w23_2$?&F5?i38Gff9bL?UAT^GTe6338+_J=;3o4ZYudMz+pmJ@yFchrb9}1n zYkSaJJGh(E7lK{8b6SKIZZ~p6ulr798B>1DN|Q7dPEA(Dxz6ExR#-C<%JZ5<vu(9S zU{c2URx2^o<aK#4%H3A;SEO*UJGQ!5^&GHyfJCl)-9HZ_V(xDOzmq?ON$cShV(;g% z_Ve?pjf%;l`=^A3S%rn_V1esZEukU{)5jKsvreZZmbC~xeSgNavDH$i6YQQ9j>A^I zJ~xc}YNU<y5%{zjsbDX|j4Z!1FE*vjt-)E(tg<YoiaYx0X*D#@irnd(Qf;CBg~twa z_N@j^zx#p5XkGDM_AF%@ss^lxMOD(sK(-~9?<<bQL(W`u@5F85dLoxy=z3%VQU6YD z?~tH=Fk?;07>3p_{%ab~*Ugrw3rN(~$YvYU&(l_0mL-}tcQ`f$_dP2dO6`U6XI4G= zul>~9t)rGP<Xe$$W+6+-5kblGKv#`8Jua1_SkRC!N6&{8-Fi>rC+qITw|yqdS0D(w z1nh%GquaPvLj*t%&6H?Ods_|xz2JOAZ8AXiQ>|F9&of9FAmuAF*wY6ExzjzRJ6SmV zyi>NLaWZzoz#zC`DDY~MBse|(eIM?$2^`m6P2e6)Qy;g6KD6HCUTAUb9dariIerx3 z!XykQS#MSr_KEF<udBo#sd^ff^$NrTwXq*b4@Z7<ADgLtcR1pOdg}S-a>iUV_0<Er z^}4DqPaR+PHO1?~egho^`u6?<<ZO#rRL9k7)~aAxQQHl9K{^9}pEKy#kUN)(uZr60 z^n%{p`*ye>sebhznZ1hzwndi>9Ug86qn!Um!XEid`!Dyj)+Vbti39!>HS%?m^W)kV zc!u)wnz-nzrvgzFiUnc&;r$w%pyK5h!u82?xBoLOXT8c8akb{@x3ohke|NDSH*mox zGEuBP+uUT+MymN%5G$r0doRylVdrJ)kYM#*tUlhT<X7r#ZBLeFHQ{}W8;#%w#^lw| zW!vf7jc!h3s}8@6Fc~x-x7elV51Kyfm(!8nxYr@TYE8g5S!l%<8uFC>#0yH_HF601 z5uxbI4BOd<2ppKs2J0LHGuh6kNi|k0YI8+>WhC}K*G;ovUEt-I9N?&_<13?fO?~24 zqH^IT;zs;P-~WFsrn=HG2i@Uhjx#nd;8(ODAOirGuZ-=)IG~V!XAps}^VIX1lqk@% zYr094tfmMF*K3YC&Hf-!b13|fJM#!-PxNc_V$5U}_^4a9FJ^$}S5$PLMx}JY1y6SF z9g)uQ{ps{vg((A+4fm}}Htkn{gkWt$8*;vYc_3GL*L;d)Z;q{jEvJMJz%S6^J}#>C zmqcJy1>Z(u8bV*Q-V~}Tj)Xcr<TzVfm4%-v3hNdINBxtcx7_bD7j4+@baw~w#R!y; zGsM|@{K?5B%PdA)f*;%*ninaz&nX41wk%Tq3DS<xg-@rDGjVbo|IwsBLZ`ff>|PWo z@g`J400hQxZveI$z9J>6UDccCaC;~i&!A~m9!B3_xHY{N5WW9{4X9)ABY)vhpY}|F z1nUWy949wZ@$d2A67~N{a8B=qy0ki2VuKJaJ9e^0$i1JN;#*&*Ynxzj+wA~XCnAf7 zjO_Y!Vxmhdo*8P*45U&Q#+28Bv1q?QzA;WShq5g2es}b*BA66;EN*=nT4djgG6LRj z^W3nM8d^!u^6wR2_Hm%qe1`^b+7v-5gQ14TgX|CU=Wd4EG`)JyNI&4(-0>hFswA6{ zae(b@j`ZkRrrrE0Dg?|Wo#%T#pxVcCN^XMf&u!|;*f*A6B}`mVF6~N$j(a;PfL#jd z(i;^0tN8-Ml{c%yrA7>jtw61)-H1~fXHOs|IDaQL8hdZq8qbHZ6kkZ*iPHL@2-i*K z1{Hn?J&xJ%MrPk`H$Bjm*uXTr+t0ScWnA!!UHPd?@v8~_{1YF77C-L#1znXT)JQVR ztJv(p9w{jA9qqsc%LNC{X<Uy|=fS04wesp_=yaDw8eT<B=Isc*KRTd-L}h#B(q~Ox z!%AD|H%E&#E%Q;N%ld%_6EoM{g2?GzD$5!36{}eQy~&FD%{WOCx_U>dJ!XYV+F__+ z0L5{1@3U@5&sqKyk{{?<Up06W=kKeYKXYdy&3ya{o2V4@Kcln3tv*DpEF19jJ|Hdc zl*B|E2mX1_oAse4fh(Wa{sq^&CGwA-XC<OrTZ(_yyhgwLcjYEJqro~~Z7#g6aGKwx zS$&jiC>cV*-=MLrU!SJyf||rNG!kU^l?Uo7*Eo+jR&(yDKE~7&WKu|d=DvPJZdiVO zlgN4v$QonzZoILkRCnGO)-;Q6YzKLi_Q(jy*NY#;F7;j6HGwI%HuequZB0B$1soDH zxXsC>u+SP>B5}y=e-~RC<{`~_UrqqO{NY@y9I=QpGZ!J;ynB$$oF)!9Lt!<owu6Mm znWK$wYJR}##NX3(Q`bs=AYOR{>9%g?FOe}A#wjBtwpD(d^|W^)483Z0Mw7HV-ZP0m zL*TB{(iR^O7#z`Bwi%0gxp_v}I>hmC&0BuheKsbLk2YOLoC-mM4Eb>{0=Z19*7McI zhr39qzNK-zOyt?s{bF*&d`Qqt7@MNvCbhAf_ZQv>+u`-BW-Xir^cA|>`e&@T#zDkg zf2ZtlhMZoN2R5)7s9xMZ`=jW0EEB?%kQV`h;?HR(PxWu}t0s|hJ^MczvHTvN0i#;Y z@xG264kDKW@sU2T-mKbU$kr~}MdN1JClP~<)r%`*L2sJ#!23C#GYn{WgPYj1a+bds z%_zrX9AQ^e7<%H~v>G@jURx5A`f&^NF`g{<sGZ990`apQO9g;|j}^>NvK~95{J_^t zEIVV?yT@Jj@+1A8t~JwL+t>)OXu0(S%RqVjgI{BT=4&y(iiU2}x5uX`lMcA|$^@UF zi~AO{exlSHlqTFiuq$R)TLZ*aIDH&<VX1M9yoO;_;+Nz*S~sseOnv(?fBMQjh0_3G z#NPW=RRHf>AOGu9Usd?EO+Q|rZ!xM}&4T4=7PG$j5?T*_c)vMuWUDW}`lYkP4#wmI z%z4qJPqKQn3O|$?HGur?8S@%#xMj8`LM#Jj;~PK<!w;+iuOiEqzT5wuFu|Z+vlgfu z9BbDnXbO6=%$XK8hWt80Z38e1-k#7O?Zxdi;b^E_0fO^LkgwE+H71mMPwhqD==tEY z8earXUu?d8Y%;0VThdOe1;u@3AO)XbOpJ}#w0gaUhPPH&R}wQM=&bhl#@XZo&vD7i zXp4{{)DP!KBf&=_i|?gM9K*Z*MX)?51Mn2AV`+^QlywZ@%cccp`h9l~`%FO-H)2(F z>B{v*V$W(lKJl0MQc@iC3Tm^#dT;Re$<wSu%Dj<d<xSTF(nmK+bw#ovG{xY>^t89I zOP9*beoZSdjMkp`D6V=pWqIRGqXZ%qL^9vtnr<XAe!}riZ&yT#YrbR+C=A>nYIMvS za(%mC!@nP^HEU?<MSCe5$;U)zYDF4JYQfRHakl0J!EJOkGK{1r?KXp+zM~)USh~`5 zb@^fv@Gs9N+E@cUie6{|aw5VgyAgtJyE!S3>V^m_l`bmbjl?3xa+XPexmyGcd-dYU zMn7aa5bhh@hX*TKOP|;Uq=aUBPc7Uk10!?Q3kGR%05S7lEA+pgGOSsM=rcCf*tL5z z-2G9{ZH|?Xvehmic5e+*KQ5JiZ#O5nMHDfKIJlL8{XJbM;eKDkc=JD*yjyt~#gfMP z?$jBsR~%K(xB`D78B6+=AL^?>c2W?YBhMLazkWyKOlk~Wh``vDXpvPX0F*UP<xfX) z%78$`H9NXEz4cE1nb+t6fkW@f!NYvx&9+^=68$mnY2;xq*+eA-Ktz~(|4dX<j9|Kf znzd1hun>4VD}96tLj2~8S&VIpl#!^D6*ha;(T`kV=vs&eba(r|Uv|pq2kBy<z2#Pt zYDYA|{n&YBG^DP9gJ0PrFky_`&sg6Fe}NxILAJg{TN}{u_%W0H^MNbxt9_AwQx>;Z zX6tLdxRp*w?el51^Yf#)K^$XU{nXD>-p#q5CR=i82WGrp_q$|M`vkMR#g}a)V&fEa z0IOI>zj=#89_y}|TBAs{_NF-wJXINbJc&n*#!+E|p?*5V3cRLMSWPGewJ<q$Fc~`9 zRKtmE>m*C&!t;0Vz3V{u#^$A9V6PU;rmFW_{Algk#cvT56*I!;LwTRQ;^>kdvhAGB zUVVXm3s1;BCBIRjAe1R)xJlTn05B}iF&oH7H{tgmIo(VW5^X$uk+?iB(AvFc=vRDc z(ku*Z_%X`s#m8%GB)>Lvh_6h`ClM3o@k{BqTaBfH<lVK&b`(e8iY9f^XcKN7V5>NJ zwimbO$OE&|d*6#1W0~?N{QD#!k_ikA^ECxl{i<VicBYGv33~ecwBP$VRQ&_GDRoL6 z&mr3RP3mEM1=o39&rkbZ`zh~^A`iv?UO$*Um1$jgGFBk2itY+jGehNr8D|L>`;q07 zS+@q<A-M=23D<l<!w|~}@r0JtBeN0O#B|N%F^p!42asvHEDOBYLP6hdhU6Rz*<W{) z?Q#;@Tt}5*oTxk6?NJJoZJ&J=TE8%fitjr;R2E?fgkULy`n(z>TFc21DJ*#J#aO+C zxohoAPv>zh*0g^4Ir@4nZ1(w)5C%V`e7f!K#bjg7m0GP=SH*jTsp;z$JyZ1vp#+(( z7{<WGL-B@-BO-(#MrZ%um;IZ)zWa71qJvJ}(?<4bT-@G^7mt`eG;KShMxcmBm7s$P z;9H?qHrRD@95;TyVUX5`I>tYLs{!EzwfS_0Snz96emGY|iP0GX(h--_`KF}p&(FVM z&Il(;R~#rFMbpQ5VeA+f+oFFA4*KwcVwdB$8fp=~U1%)7d@gy7HN;nk_tD^m?Rq=e zV@nA)%zjw6xDDDVOL+<74F#;Y;*Zo7g7T=?*`g1oc(z~D7>mf>Bqp34ace`R*noKw zy&ponbnJ*y`>z*Z9>aWTWIby;P=*7tCp%|TO2i&~K18X}f1|AJ1O)#-E$f+#sRn(! zL<)1Lp{aSCy9);a*5o*1lbqT7_)jUOKDd-8WPSDW7pkzkO?}2sl#NtJqR>oZ>dgCc zo`A_FUynbwHzO;(Jw4b!pI)vmtxWgyjE|UcEZDy?>uM0?$JW;~tJ{OFoica?8M1vh zIz?*9YCl&CWzAth<3374y+<5=MGUfJ2@nuF;;G2;cAqMdyKa((GAsuzl3_PFV$>2^ zM!dPE{&;cZ)Z2`;6jUaEWxheyW{fkb*<qJ(HRZvyz*&l{@~8&*>#?TcG=%&Rc)zv5 z&$d9$%=gzcGCL7=yG?%v7ja<=0vKo5S=oOeAp4{3DYY#W8T0%|N!R?NI+SdhnoOJ^ z^-~rD-Kf)EM5krZ)p$>IRkh>{LRG|P-YkB=_sLYuK2Y{4>K<bu?&1bq|Lki72-mmd zKT^JzgBDEs8c1dvl#F+p$qai<{iJtrmWB1No@L6xi~*R;skp-t6j=3(Z?cxij+HCL zLxenL(m8{CbsqYQ-oK+xn}(3`1?tctE5A+jh211yATU|h2qenJVCnrbopOVbpYX=x zBL6JU5*R@@@H)+{B6K7i+t;YKF76P?AJSb;It)WwSUKI?WU&M`PJyd;to1O9DQp_C zUNb|Th@9UHj4um+vniBI8(xVJ&1ZcgF+y1Z;0$`p>_AniLVq=rpYF4KK+t}Iw#uM3 z=;;TFH#le(7oHdcEN(xoIn$z7Vk$rc7~0Y~`P{UrUKVk=W3eOfE8v|MqND3n83ls{ zaDMRh`v}Jq(KtUuVhkRY^%#FfHb4+_@F-jF7+sj+VJYk7MpP?2G4<tRp=>%5<2bn@ zhrr3{&a<KDy!w*)*r1(-4Da4|xU(^$#AUtX&e15G>#uAG;Q^?gUgR-6nSb(qKwLZ* zX(Pg!IUJSggzz-a`O1_pR?&edj_siY89X?%W-n@Krf^zmq(KWFIx!p`cgNUF{6(eN zh=YN&U&-zHm*+JyI#jL_yi-zSGY@Kn_%)K3OgLx0=X_EsP3O^&&Y@et_~%K8%{1AY zX_#wd^>8*#JYOsv$-eN~7hsw4z`%6R5-f8wbr$Pbn=sG6Il?E|Suu?UsL(Q2)+^K| zD%ftoe}bXRT^c@j8=U?2mOr?3Mt<rCHSN%ok+DakpZ|CVR43l*-o+7lCcm_#$>&rK ztqp<^3Ju3tUio*##rg7&u>O(+rWc#|UV%(~p9cjhD~s>c8-mV5)~kgl#Q!o7=FU%? zhT#vGolfpeJ6gs!NcE|(7RwvF|370XVwhh1**vRbF;T98AlK(MC0<+I^4R<gNX1iJ zZcO*blWBp+F9R5J2Vi}Z60xm}M$x7>R4pSimZez}TnxkD2vB4Jo2?56o)i1kvRB)N z&r!Q|1o#wfr7=&<)29hj3ofChioF80@gJWH&|KSe$%a05i5jARwa<WZEU_j^HI?0L zl%*Xb7UL(rQo(To^B8yV?yY&>mr_zJal0b~_=^#<^niA1`#JT&e{X6bnJ~!{Y2jCA zw3gzxCFkpujDDJpvAF7JcA62j4lf1%?Ee}Kx8=YZi<kSZeJ$WYK8y%hJsLQtEbjy1 zygy6zj_}<a=b;QSj8eul+hh~S)#RXsNn2v3Uhhz^xn55SD!c&Sc@sWt>2GyuB9ANm zf+Tiy6<gXT&-%uM0X}f?!}ezAL&S2M-a1ZjCzkf>5pzw+Vg`lJdY<?#s?FklxYc#a z68vw)=ivOrm$vuxnN9mp2M?u`kt%W37rIn(J^eO4wfn3*AdhO}1-F(;dxdSA{)y71 zL?Ij!Z^FL}XwriF-8|)uLrKDlZ`VE{ZZ9;+0r1w9QkyTI!<a<$h3|ptjsHKY-aDS^ z|Ns9#MkomlvS-l?2}v9?vxGE^BS&RBM%j){Mj3~MtYemnL&mX>QL^_q$0i)>9P>Di zdHkNeU%%Jqa{2z<U*U0oJ|4IG?RLFgt7<u_Bhkma2uiHV|J4rleQ+puZa5ZHB6W2* zpP4oQ`%2+l<Jv)<;Kxj{@4EdaDm4!pgAeffuRn7x#e@wPG+k?=Ut`)RhXN~T+_HQ_ zo&GqyaSq1b!ILHd3m*+Z7_+^hZv}rRNgrfFKQc!j1IBqIMwS`WU8mvU_d<_PEpZIk z4~i%)>tJ2*|9o(pC>`%xsH18TXX&<39_){U;WCn5y6#`9?cps1hjpGez=<1++;uA@ zPHbkh0b-VI!itz*Y0Wp`iTkN310YGSr(aCzmC;N2H*4et(W$08fzvH1_~%A!?nReg zU#`KwaW48g>)TK%=at@fucOWah|ROVvHtx@Hxd$S=Wf{J7+7M}TiEFL*cDxZ3QPrv z=A-5NnLa3pdN7whtHmE4W!(3kCnbE7wA|^lzJGkWE%)}uMJE(L0f6>j<J-lKm$kaR zp-|?nHyj`wAVgI%=Dq2;)!^W@tCe@_ob=8V7n@;Ml=^EddAFWv|G~oTW}Z+%xkCaO z1J|3?wy6$?ZvG)s=SrA}D7J`M);sChkHbfMpTsv0$xr7WfN2F0*Z_$QIJx9YQ-Ck6 ztS+k0cg<dOM)Dw^71hydM6ZgSt#;YW?o15yfXe-MIS-N)47GN%w?5>5PX!-u{o*`_ zID|TR;;i>{P?PJ}nR+}$WQW=SSx$xC_H$jSUizSrqU(0ExA~Xo6T1e6e~kGguV&f2 z1Mr*iuHvp<(j_#4lBEnql2)Tjb1-8jgNcXUGumwyplc{Z>}P@#OrK|{38rvE->i{Q z0_R@H7*gqO_zS_^anG$7rw~C(?}%IFX2>nmTXyDBGhuwTyX0MTBp;{!5&=0+_N)dx z%VO63)&vv&E()8X<iQ~PHw^6@)f!Socf=bvS<Hu;B6(Kr{%kjSxW+vn`cFPugfvJ0 zE&PlCWhW9S28I%vRy&%0`_JpNUFamP&sflPswfsLuh2Lvoqj$|%ye77HsgVarK?=3 zUgLbF@J+f-t<t&eNST&U)mP*UL=R0^R(?yBrwZZp_?&PeN*x}us~f9}=3s?9ulyw= z(ICkai)FAMhTh1S2ffo<;WL_pgHFYoHJzD)TN`#?qIp*_=iGZ;?*w*$v5z6zTL&)9 z2dtuCs}htMy_r=HUAxWB|7WwTL(%WkrWUWFkM#D(>Z=N8M%|ybTP~iTQ<O8G4evBl zTx>ptnU9PfZ@7A^Z>-zQ2VPZAx%HTyAy;1hGM=XP_CwClG`&a%v7QyUGwkee4%<iQ zrS-M%C#5gN5jTw;OeY{epCHlZe?4TEk(-^9bbh)(oZ}~>@&3d18K&9oOckpwFBQQC zjtt0NF-t+ih`c;4l2Erw!!D8)1w)5puLUX-^wkC$cI6(EbQfdexv1myKDMA8_AQyo z-eE=mr+sk&D>Ui#vQ!;UE5ruHr1wvqkVo@GQsUt-INvt+z?rCR?J%Nf@DsP<cYosP zn)Lt9?wl9+nhtn%NR~SmvzK?*w=RHVs>VH@-O?HzkFyHxzM(PAP~Z=QE0gta))xFV zph^G*G)A1Kmcb`v+v;g=i6df#(ZTfW$#nTC^~i1fta*49x^8gi$>o|+%QRziHZEO_ zj**ucOL?jPJLjvbiMe)SEWa-~Zw3IuMn%i@5aM~H-+{L^&3C$d*R3hnSlKJ)NgM<I z9PXNhCZcriwsp98phGQ7i+l);mZg0%8YVswp$PjW&ZM_fjwy~u>Xqw_W2y=dbk#H6 z`7ht&I|J&P8s|7f^jOk2R=)ZP>wcK}y0+108ltSb#Km5Hk)J%KN{4Xb(IP28JQfdU z7#S*UC}9-k)Lj}`T?0F}laKxpl$Ros{+_cPZ3ueQU`hW)yNL<D%ZY)`|97HqNeN1^ zt-tP>RY5J$ZJ87iWLB-h5hAk&HX=@<A(;`Hi@wqio4wUth26#_-Q^VeIoA!J#xO2Q znuar3Ok2P}Oq0B|T=jPtCuxB%?(9IX;6Lsj9?U^nA;JVZ=d<idH$4puI9QB)fgE^N z9!Ln0x|P?+#S`&F6r^sgtO`Kf%9wn+%I|L(EnS-ruA5nFB+%sL$=J|je(e$U!EU1W zb7w_rO@}8vG=p6(^a+*nI;;I%I2Y6WaG)3gjq-qZ=>&xVdYA7O)t^}xp_V&+>9Mf6 zP*PL?ZW=g2I@fG73rM>K?cI>5dZP;V3lKXA#YzSCMYlS;T-Y2Jq0?NIOQ1}Bq)A(x zK+5Ga@_x_S)&YLw!N6C4DDcudW~1JZ<%9rj><6#@3LJz0$76%Ytq5KTm|D48x13Ct z8)W$>gt$7JpUnwbezRB%O=2`xqnp|zvVF4>Q5Au>cgnq92YhgVHN@byA#A%5)7&o0 zpki+nNY&`)Gf~c0y!_5I&=BFRnj)XNthJUBFQ6(gXf`zIvGIZR-Mw{}b#gxc$d$oa zf+m1J*}}9-JE;+*H&3zwnRMRmGernw{!^U&>RZoFV5lNrh)3dtZ(ZtTk4Vhh`se5t z?Qm=uLjF9y%u>e<#@9a~OTE?*2Z~kl$pf~}<c9w3WtzK=4;fVBy&L<~Qz!O@e4x|# z=PDM2=djTmHYI3U0gv_kQ-7iY?L6P$L8`3|_~(OJEsscRjZMWFj09bRJz%B>Kk`+w zkgo93UFQ7YhEDHoqO__a-fd9e$|wvDYC*d@csK!`TpiOmfyVfkuIJgKf$uq{+^6qv zG>a?15|nmR;H;39kM@lb+q=WYratL>Bl1-pevIlJO;mE}Ur0jj&R%sBM$nQLNy=RQ zc*#h)%z3{wCe>Ng3KQScU0Vurp$1-#P0qRbn+Eg9+{w$($22~q7yW5><%U<1phu}H zWk!HTK}^Wzk~;M;>cQFO9^eJnbyPvO1KDo<X#KxOWwRTNpDIQ5t}vaucD+@3q&X;Q z3Fu0p54y;7g+~3_Tky|#N#=7ub%bQAf-lp5Y@RYLg_H}>&UJRg|BQinA9A5nlg-8H z#oFURATNqBtJXgYs^5dJ`&95%%T>jO9lG~2-F{xwHDh+Q%=_|-zNP0<!K)O9<M<V5 z);F|>yO?ad`2Hu-opLL4^zak4asU_h_sH%9gsV$XFOBfZ{QuwJTT%%X>XqFxfv#e6 zpP)r2TlJ%B@Q3C0FNQPm>x22AMM>Vg7PigajsnZg>s!c^@`e2Acf}j+9g2Nng8p0C z^lE2^MioLt6s%!jZLs-TM(6>9{o>M+_}wlU!4^)4(a+bOEE5{(CD$^ByO}?0t+y%Z z`3}}`@q}&{NZTqrDP>hwz)}QtcH7*UfmW=63sq@Xhd^Slr1vJpnfg8mB<6onaPl#S zc^Rya3it)GFsRRCKhQ+nFk>hp-FkfLNxdK4Xcr7~>ulAqbCz;Th`RHmSM+u7p86{U zp`fI?BS_k1f3P~M;;#O7OyAq{QG#zIm-7@8qOPwTCj&LG78$?%_qM`rqbd+SF6sL1 zG;2o?Dkb2~-v(5mhCD;)PORsw7RYjuyL-cU|86=sC3B|XT^oW|tlv^^N{752I5S|+ zRK!p<-&JAiXkH+_pDPDH!A|Hkh{#9v_sau!7Pmu{a(jSm($9`ol>57RE?MWAX{hd# z>Ofj`1E$;$YElAvf+=dceu0r)FYwIV?d5j9{b}wjyA|)MXu3BPt#C^(_OJHl5|^K* zS?^4(bk4H-S66Q&P3=(o!Bd3z7`ZR8)n|e7_`2lMP|zYf--+mDZ6)}NAMCfX-a09> z!Kz3e^kL;8QdOahI$a?zfS7au7^rpJs%m%3)Exp+HRL)2N=_ZSplustL2v<ythaA= zc$4;nqre`pML64wBNfG3LHdXNQt-(zwi-QmZ88G1l5gD)qg!Y6JiRJZy+?COM4v){ zrsws06#X=hg8IUxGL*1Pp-1w1++v0*DK?~9<f!DMX15E??lE|e1-L*jgHhl$A4Rvl zT-L+l$hL$^<JQgcy`d(Iq6ER;w&9rVp3!`P+8Xdz1KypgfzCvM7V+tjPz!e3Yg76- zdm$T^bUQ)E8%>!wkwSVMpXt)$#lxeI%PO0G_!z5W;a6FB7(aC@Dz&<ijIyn^40_C= zUKx-(;LY+gF09O*w^Or9$M&0}8tpykVm*5jze-Y$+i0!G+>+KqfKQg8Qjw4KHvN;v z`EQKE*_wHUBz{;hB;KIE79W=AP$K>?`OG%cbIjuRYs<MHzIw>dr>4|bjz}gZiF%`9 z-mg+7bgDXw0%zIEr7&ANOwmc$c!2)x`lr~9W3G2CyK~xks`K$`#0@;zhE5cU8d3%p z(4{`e;#tpW@?zX`@Z-$_zWi-)p1+q}{|7~*)A%-oLa><|2FY2FLV#0>AnO_~bED?V zS=cM<hsB}ScbzNZ5H=nM|JgJ7T4~YoKjq2QI;qw$mlI1G%O~L)Dmk*HS0W5dB_8U4 zI%Zs7K{x|F99ebBoI_{yo>y3V2UI5EyJxn#b{eaZjrSx0%Qg)M{e8h_>kN*8zv}jQ zPNwL}B*FI;2QpgZA63f#v33Oe$yE`1MC*Cc^T};Q%%3I8dh<8-Gzkt!N)7|PVPa!X zbGr>)WDWZ%BIK&YmQ?R-IDPR(@NWIg-5o@qmzMn=Av)v0u_T3xx?0W;QOEgv&Y=~p zcHte*<#X8HS?B|OFOLmZHTk<`g^mzsK9Ab`aI$0g+7a8^+|C`AxTh7+ha&_^ymI?) zTI##>4i9)ydu?U-1BnaWMOMeb(rRq9<Al}p8Kl^9S-`tC$^J{Brk~HVCoq(qIhWt* z+KD3R6kYHOnP)$QkoiOT6DzL6z5hM(NKTqMr}8`{t(VW4Q+Ai)lA~rY`Wv%)z3<m1 z!R<oXoU#KFb*h_x>KLklG)8G1b-N!~gO}+*4psnJ%n=}qDReYmQ)RCeIJfRu>T>m< z*p#ICG==H>lu|f(Eo26q;5)h;sLCS@z(yrJW-8{R)CfX~x93Uk$%>cEEhZC!Hx9br zSFdp0qw6RPoa56*CIR&=PsdRK2At}B&3eT#k$K9G;(*tMx|WOP^dRdQCkOt^0QZt3 zG)MBam~ePxuCn?g`T*+_DTh1f>}+2a$MTmiRPDJKWdkSuawb->y*6^+3aUvO=>NyG z1hY5sH|v;pJjbXe&I%t0@6ur8voS5!BU;D_xewPMjeIwtV$fRtlME;lu7M+Q(+}9r zEii?s%I=v3Ay_H}9yN6kid`#a0%ho$?47?-(wVewETmE6`Vvuq?6t|?O%5aWSr1Mn zH}cU4=nLdj4R2o~yhWd`&E~$DdB4D3XFm@``j{E+7Jpe!^n3AMYzY=rv>!;@A;Gdl zo4~2q7%8+^9N?TT`t~@^?c~RC3p2Iu`L^5X#x$A(rv}uBNQq^>{yAjEvX(dJ#(EEE z&&!Iv_Ne~ltg^Zx0kKx%yt)>ea4E-axWyfcK#>bVGc!Otb|Qu##ix-=Iscv;loJic zZBR1|ZuwDG(6lP)3&WW3t!DttnT=QR#xN!o)O<cbGH`D7XYtE>mxUK{KC1JcT!<*d z(FQ!+f!Ft8KO`<|X!Jeo`*Ggb_sJ2U<sAn8AqcodZeSL4f3`y{;vtjFD!5l-vItHy zEX+PY3=GZRc3nljv)Z^COqkwI7qLz7k}kK8WoRBeeu?SasJoEkp+x~ZHaT3lU!Tj` z@dImXsWOLto3ms;gn}1m#{5@G{#24JG``v-)mo~YJblN<5<7ckWOpqzpJVEwyK{F` zWGpD*+N0=ulFp}SAXY7O2P-s@wUQr9*>)#BIei^N)R=DpKIjS@ut(b7+3$b0KfbWr z_JT#AY^r0Q9rT&riYaIQ!t2FoXw(gY-D}q7FV>N`-R4IszRaQTB!4g@YnK4xx@fSD zdaTqwM@0JVV@m?<*CaV+I-494q_+ULb!*}r_lM<DN|N9FVr5D23vX5b`!P?!mOF(o zo*P6bwdDct^NuLiflVnc$w%UsG`SLn!Vv7i_p?y>{7Q^*am`0Vk;gw6As)-6(CR5- zD1Pn4C2D!c(;4R{=FxjeK`D5pc#%)wjH6`J_5}aw_Ek4d?b=3N#(p%4_0{pCbCrF# zrT0GpP<TcMH1hecx{)01+YETG8ABw*kV_ZYS6D~QKSHYd2rZZZ%&E0t2Dl48w73Ob zv4MSyU<U*v`tk3<D7i;%9QjT-abLB^o1dqJv&jQW6EWtNzxM4Hr&2=^<TL<I^(}@z zUIh)wR@}EQaCNk>TQlRh+h6RGMHV?Eq%R*7OEqy;QXB689%GN{c`s~|kmJGcsPNhS zI9_$#OTeKrew0pJiK3N%?A?EL&yv2mG6cmmR*3@sVJC^D;+ADAl~~WQH66aD)9TES znNpR+mcrQZSy9jE3dPha(93UP7-@2|aep3vjtn##=kmU3WtWgh`KU%K+coZ)YFu<D zPcbayxPjkCZEs1nv}GY5WJ9kmz~wJW88yv#^Jp6R*XQIaZQ_4q&G56wyR^?cdlbzY zm(9NAB)T-lDZ-yA6zcDWbCF^^;YF!nHH~ce%vkn)>1&YVq+%7$I1G0H@mrSpINlHS zhXRmDC6RyQ*#2{dmPIJTj{1#i`d>Owsr}0lUMMeANmEFs)gh4!-z0Ff2IR6&uMFDx z_4qAknO?X%7xFpVHWhlvHCn_V^jFGWdF!mv@il1ZXG#eabQg5!oyChrBgfJ_E~(Er zH~UriC_Gp`^REC5`$@a=aH43M>n}aj9(3o9CxfW<%18C@fY^U)6h$#x3ri0zH@nc! zTTC}jcV?_|-9d$w=`O8mA0QwCXvNBlm)Dy+t*dP*#y5f)UiE^CMFKd{wwFh6L6m#_ z=%4)WB8U@Qv$kyycx+z1(4shP(gE;B*;-8xZKA^{7i$QQr73}*Y+2<|g&obWrz+3T z<zQPioX`Dt_5Lvp2~mX&$LtkiVHRGBSXq=e&KgqcXEgOP%B8}rgSKPvrcS|DG*5Sh z!ZUrYgp4$FKHH~e_nUEG@(6Hg1nkdO+$f-b>c${RL9?_k<+p@@j@zH@+*;bM#8Cx2 zWih3V>#Tr2$l$2ZJE@||pVg$~8v2ZS-Wm0-7oG2#wXg<%DphIo9C;q~0c!QG**7`3 zb*(%jLfdT%w^!;#L&Ph+l2+7h+f+s*(|zSl+o&$zNWgftI~$2Tjpw|faIqST)Mz59 z3n>LTNIpItqq;Y}yjpcFypN?QQqZg73%JgH6|znv1M@<-gmtk2AVrpG8Oydf)p|6w zN<`!4##sE4A;+ts1I?{B1u>tK!7DRDrNE-<M9gKtf9$^g^?o{-?6Q6e&FY<)u}AG7 zNQH1&cn`36BA4$udR_?Rt6LsEl;6KPO@i60Z~8tp%_>65IZ4MS^fOB-swkP2?_xK5 z5y-M)v`98Z*cz@n;O{6E4=2~A(up|X8~}f>E+NpRX=}-HotH077wf(_#rX|`ma8{~ zZrT%PmRU9r$Ca+T#37T4AyNV(3!^XTcW*9lwKWFlP0=SP_*MJRU3na0N=jc#MY+L; zFNVb46HiItiuR;`s4J`e^7dt?%H4SH7rZog%-u!iuC1~sXSn3FwD1>OFcT#orVi%Y z7jf|3AWyqBeI6s-dkr}MDW1Y=AWMG$(P^48Y@`;*Tx#jwc@6)iKnzp4*)ScBCq_>> zAcb*W{3%<c?9+zlo`c%>)TU!wzZFdZi1&nJVRBW%^t_`=>84+m-g$=;VYbrD@W6N2 z;4DHk#<usRg^;S7OEg!446{J@KGz@rkv+h%Lc-F;mrIwk{WB2Ar^I+4r2N<)(DVIj zbPYlWx88{E<;egBf{@DJ(TpY?Q?S-eAdA89l1cr`jZF2$_s>uletBwC>K9+rXK9Rh z-|;0zRQbV{%*j*&DJlEdMse+HzOuyA(!h~?{%8Ly*$Yx*7P8p#q_06c(}s*2<j2xx zpzdzbHjB)+J-+h_rh<qKH7o(FV-?%jL!Vok=h`xD{@v1oe}oh=yI)=$y7i$t$*z(v z-`l8?&Xpr9X={XO|I^${pp`^=B&9s*8zaPy=fPxd79yW72^N$;>)+zYofL7`oL@S+ zMyek10KojhzG+@}&1<w)X{h355%72Ko#1HLK}1jX&~XY6IJ<yjRrl4g2HzDE_erX^ z)CmcleCZNHXU4)@R6r$Ag~xJa-rb6J>d^8E|7Q=Cp@UCL1HtyXO3N;cy0zba{s#c( zu2e^LOUHCsMy%RToBP6wD&*_<EskyzYB~6?exOP4S16(!r>&PHMMU9!7wzm&Mw@oj zbn5NCFrbw92e!bAZQm?B-GNiDAV$pH+Kok3$IZ3Rb4!$GwAliV5E@CBG+8!2%%Vg( zF^6b@L0x6W`(ta@yWUK6)98C|2*$<Rg1e{GT^w54iPDuE<}zC4IXe|o4m~?$t|k&M z*M5y{Jv1w&7MM*BuGZRXrniUDJepnI-j(A9KW4;oqJmRcZFzAv13yymYOMfp_HsdG z0HU>F58PQw;!IXMCqTMed$aUD+~xRZdv`{5VAp%L5|vg>TI(sG&8o@bNPs&S9?w5z zhcBo+BkUtpDzeNE+XVUu-F<T!*bmDAN`1u2M7Ng!*@`|yb4|!S&ekWb^y`H~CDym! z@w;6p>#<8}?#1@gJ>lmd{x^2~a`BfC7gBgJWPg(Xu`5d8ph1CF-MRB(1j4^6y3J8) z29LxY7UM|ZIxCA>%_b7kAFcc?^`Daeya1T1FaRaOQH&l)FYJ$*!Q8C3+H)WEu%jS7 z&>QU87TI!`J3X`m5LP!LfrJN6xb`%p=>UCBu<2+T|0DbYgf>B>RD9qbEG}eqQF+(x z@=`gLvL;=hbMbt5LzR5ML)iyRqXZvj-X8S?XF0^I10aQMXe6oJeY`eVkA43xmWZxd zU)5S@F2VeDZgd0OZH$zt6+?m?EAK#suzPp>H<U7|$F_C)b1D?V)Yu4Mf>J*JO&^*; zk0x*_(>}9-t3K{=##=JnN8HrjR{oY4|8;Z$Yh7b8AAO+ko}GSI=5}nwju^9SnBR*O z5^-X1B6-~2Jn?*`6oD~3n_bNgjL=tHnPrcjU>5M0(hafV(i)5^Faydj{D3{y`M3~- ze876LB^G+<Q)mY$(YEAsbRD<j=%T0c>>dnw-~#o9)RUm*Oy+%wp#`6t&m$f%nR~6C z{B@+c13#tyDegsSf!!R&<oX$MNla8~z+Yo2kAmya<kf=~)}SA}@Ri(**+Dkh&Wa<U z&jY6xr73*E%~BeGvRBE~v4qZVOe)i=a~`Yy2$OCFtLRU|WL9ZHbbrU~-EYOB6;~*6 zhJe3y#<>Pf<$GA|(8K(fQ1a!Bw;;2iWddgF^X=@nR+w8){|*PMr&P(~;uaZ(3M5?D zDx}k|qydYIP-GU+rExx6tP57Pe-$OkOQR;$Uwj5MkO77SyY*<#bwXzR=d@jdOIgib zH!d2I{JgB<#&Q*nuMQVgT&Wmjyczoz@xOf+Wyx7$LwNgACvYp0S!E|<Dns8!s~kuK z#<zh>B|K8wbOOYid<y);LWp2kPtkuBer@LJoA+U#yUVZxb-E~q5C~3IZ#w)dYP07) z2mKws<>bSM#{XMS-(SF_nZejX9d>yGGyy8`({wR-_^s{}z~V6V^9F`A$pnZrajufr z)hb<Y>C9QT1`SzpztWn+bKiX*n@lZN5}*KC4Cy~-7vT~Mc`}+Fj=mX64qcq-h`J2% z{y|di=)3qf;ahJbn+v8^de=qf7PzCw>t$xRaTTd}o)IkC$psI(c*+HO)`sk}U|7C! zhs|G4QCQvJ@+;IR>~;|9IKQUQwA7+5zzUmyHS0b#o<?sDy3P*SmCuw(tpMFoWa#GL z<aZvEsfJ@NF%BjB?e4?L2Ku&vsOwt+=$lwfUCV;5XW7RfE%!zf%|&QrS9ZiVGmh1M ze)V<0qXx5U8ueJast)po8Evc&V<@&zonx_n7R-eblVeJAH#Uua5Ot;6Sa6z0f2jIv z9K;>BClDA3Yf0pP4N&<~nK1{t2YpXYE3(=Bm%o}&w<b(b>*INpAZe5Rz${pz-{;vu z_)Gk))0-6(U(%23jCs0J$iaZVF{yeh2y6Y?daBYFS=kjxD3Vlg{;)*`ASJY0wnY)c zgQ+_TsRFO%Xci6lAq0gc6Pm?}zV~Wm*>jZ%y|+g$hW>9hu<CJ!7TVl-Fm3$FPP9Yh z@4~CxpTM+#cfOiEOE4e4>pbdjwdN1(tOYAM!@5CpDFG^JUPw=Oe4kIF2atS%RozVr zTn5kH09yu<FLv}umu{F`K<{v@mt<A8AcA)D-TVD#hLxi>07M|3pHicHfU=nWt~}RF z*F7D4RNW2Om;HJumG|h?OTGB_#)6+NTSYbx_RY%Liw89fc%U|PLIxgE*4XDTyDI+# z+hyBMYQHK;F4AjUb?yq8RW{|w1N5pym69*u6MnM4)W#|W*^o-k#(ptVx4r6T*Gc3u zZ2G(B27Lk9q(7j|W)U2kQkPFy4b?)OwXL>@d?}Yu`CPLLesV>BYyEm)X%qGEp7%#q z@xghOtu4ClA|v?y@zYb^-S^ccHS_P?dt}-_0_X)_=C~Chu|gz#t!q}F?|_f~814LP zA^(LW$SxzO^kitQgcx@ty&g**%U}LlGgucNk`Y?c_je-f0}~-uLU-x+zqkL%+B0y= z1Q9m23`yn+?RxtMsoMB{;BNfoz4yXLQUY{W?wq$HUxT3K2*btwcyH<6wmv|kBl5k= z9t6GMP}Nz`Uh0j$`Q>&tBXs2RouS{Ucsr0{p1+MHEF-%0wM{G(JW=cOaYoi{Wfr+V znuTx-sC4ziqW8`pT8aiaWcCPXQ8@yLow+N&?i5gg6z7=WxlGc8jT~0~u@8B640mpz zGD|?H5dNQ8<qGq_EXRt+-kWwl&L$k}3QE7cEY_W?2nJn;^h04@Z}(!Yw4X?Ri3fG0 z@w{{F?TO2EMx0lE=wYMul<c}xdJya8bTZ}`{6y7zLQ&!-;rCngb%VK8pg-UNFdWjG z7_7>+R^omuv!e8^JE*(&fU?gzi>zDzimz6s@Tc44(?(DLtbE1r;ndj?k0-vj?K~TF z!2*O&_8}jFW=Vg53l=s3Rt{wv$Z-*C&wHffa-np?p{G37yosa?{6~BF(!^9S2auwq zv>ra-Pj2*>e^(UWJ2uE2gH_84)?2ogxSkIxtK$1miXJ14IIr?>x631_*@-l9V86Bj z*jml%bgJOVU=bNEJ;?S+%+71Wy%8>GxE?IJY;lxJnM$HbD?ZldzbizN|L*n$y88;W zo&AgsUKUQY?T}BI_L>n=>X4&52wkLL{h>IC>OGBJxu{sW#o|Q2M;cO_k8*yT(aoDH zxE5aW+2lgiQa_+?UnIyiEf-IJ57y&FyXV_8+fY*i;G2DmMw_|XDvF*gaoAUwC?P*H z@KNqEqXcp=J@p{yRcp{Ufu{@T)G7P@3IK%&vG0xU$8&}6F?+60)XEEMwnd3D9?gky zp8RN-PZPVT1xQTSSyAK+t0=k-E^#L<F{ASOOApL%=CJ#knOG_QMKg7Thb6;xho&P{ z1kEl_Bv+;UaG>wuJ}Z#Vc4hL>*jk}N?frv;{$Qvpd^jTo^YxOmfv)4}|HOKe9zm+l z;u?#>)T6&|WvM=m>IS@2_g~CWt^Xa%KQvTwZ1mJ^hSid|p3$<Far<mb=<z%!r7t*! zLcxrpx=vVCYluGW;FLhtyE(+_*&uv^=7k#ywVehz+qoAOMU<HIx2BnCU_2s0iHh1@ z?xFxF4B(tEH*x+P@i(w>NeNxoQ4?V;86Xc?kQ}nmAOWz`Bl&lEM;PPNR05{>r3v3V z;u|!iy-D}@=5ohIn)_@{aDJJ>Vb8^BIohC9kgUpY$Z71Awe{tH0*52KJakPb(ey&n zYZGl(LSA3dTmR?F%><AH2K`Ncb6fyZ?5A^gEsgbt*fGLCm8G8O?2S4BZWWI<JhG3R zg;=ssHygHoYF95_kuIW>p<SzL9u>bj397&>Sw458!H!%>6NGJc(@v<c$$}2y+K}hc zE^(eeN~DHBd;Y;I%$=Av#!)4+V)Et-Y>j!L@4rXWVw!fx`6kSlp7zzB3PDS6<xY0s z>O1qGz0<W_<d5~`@54&I8g&kwLl3L%B)pZkroWC2lXC79zR9O)cF7I?6644Lx1ZE~ zG3HS1npU0Yg$m;XR=L;|KYhhbzx%$EV3QINFG}lX$Hbq`@pR&#V#C3&pj5&tqnTh( z&;HE!tX;6@Y;<e=$o-AP=ejCon$~s{YtaWw^8ZbaVcO3lN?gOUg}V~N<WI}rrx|es z&wBYl{FwxQHf^IlVq7i&>C79HB?DXc)sUa`SYFBMji9M;1WNv|-lyi6#h#N3%m>$% z-nonFIT`P`RCU^5Y(nRnw;^>&GOK_J=hXnGxs$HOtcg}S%{>r5*$hS84|0?n*Qi@= z;%|+fGApspwnl_rE;lf`;1Ud64$%Iv&W8TOtYskJy%@bF<@MF8$Bqpr!DxkObGZ{N z>5Eh{5+)<ob3)G@1{%1*+|1p!%6k09Pwqne$K%DL^FkF}+`4@ZkJuWQ#R1!<75tS+ zyJ5jXb$p;G4r0%_S=$cZSJz@NkAln3dbk+ijJ-@JvdKDvRgla|rwa(jaU4&ud~p67 zw%#muzbBKr{s3a7a{>{Y)JSf|il|Cg&Yyc%n|{7yIr;o+!a>6_>h!jP(igF)9y~Vl z?{>Wx0%Y*``M-^$Jubbn4~$b_h1DX~UjFvz_raP^^7y;61U0k#Q=ux|(XV(rb@>{F zt6$Nlwb3!KZx=KoV$%^u*=aBWJgB-lrDp}*-v2}gkqD1Kip-bp`%0a&VaWLyQ6#+e zqxg2Bm(-85XZ9B-xFhd1>+|>HbU=6huGUj7?AFIh&XnDo*_?f?>9_w&pm7SHsx-yF zo6%&x;XOH-a-_Q7f_G&NbQu5f@>kc%XPn}t@wmJ@`Lz{WGEA0_m#x=Aimez00$v}i z@XXeTGGeL#lv8ssQwQsa1e{iSw4;YWr8EWm0X#HJGz#bem_+{bcvB~Alb4bIbhSqN zSQsYd0|BJ2p)l^B05w{axl=>>GX0r(#`io`JeIK?0%vs5w?x1tx=v$EN7{Rq5<%ir zz+V3?rhZCOXX6c;M7u^wKNa>eZDZRM(IdvGR%qD(X1sQ!8qH#|#%+2)K#=?KCGwt1 zf&;1<^4hNImn`bqpFx(R6e~*)4Zjuk!qjdm-u=l3g@!*Y_-K*%vGdx)&ln^h;*P@u zn0|j5U-@&BUN}P<8wHT|7PU(x)HB+3>!~Kq#PW8PoORZrxSRKsR0y5LW;<=T=$dbx zS4Q=Z9v%b%5Mh=^Y*Y9<u}<E9gVsL+A?n!%t9En1c8{a$r{4JXV+QF~Q6cs8b}+I{ z@6%eB)yI35AtJXjhy{c+Yu!i$;Hr@gQV!x_0qMMw&H$%vkJluPjxxI=d>ul1ED^Kh z61KP&=VfO{x&lIpt=nOiPjRN$IRSem4CydU1qQ?dvouwU!<IUlff7({5Ai2*#gL-) z(9AmEaJN1nrT^a+Z<^Mko**;szoPQGXhmH*By&}{_~nqEsl=|%e0pd3f_(G)wPpo0 z^#rVDHABG;n?~d463FQ)`=73-D)7UAquo-c6aJNchYaOby}qC9%BJJmgWbLd%e7ve zNT1o|pB}+bA~Q7k3RdACd@P#VhZwMNQynZzLl=ta2qUE24XEzIz|CmUjLM4GE3K#U zK;=>%pR*jBD+4dznb%r>7E;#Xs`hG~+|8^kTZfOjqyytyQ9UoAIJ$_DIf(-5Hr!;x z7pSFFLmV4l6}dzzs1BH;8D|5cY!n)O270YTw$wAqxPZdb63yW3;L#&PXvs5Vo$Wdw z+-QB>AIm#ap)2K+fQC`k*KQ(`?Nt^vSW})qj^d|Rx@UEE72bDFY8+#!hXEquKd#p5 ztUZd6|10Vvr^*(!V1ajnc6|@FVrP(?qt&S~btw?Vft|BWxbCcUUT3-zV09+~^LL88 z5*{$4qbji)NH;DAh=j}0=~8l4TO+k84G;#m8A63ct51Gk308GJ)KO%X(^&#|c%3~6 zm?`qw4H5;aqK?gZ(!}7{;&sKz`3l)iVQZWRwgey7u<Ql=r4?6V-GW917{+Qh3?0lr zInX~8vwf|8W7RML=qAXo`)AEhS@2UO=eg-OA+huoq9M8@UEA&WD{NSD6WId)mrYg+ zWIZzMMc1BI1NrKY)83OPV(qXY9%pK08HAp6wFRoQZq!qEQf+Y=)5S!9;SiJk&4cgc zR7auCxOO|vtzqTs!&773=iEl8)BE;n5(-qrsd)WKo7+d8&8Fbm@m^OG@7X|~#mp+t zxJXr%ITT%8!%mk(H-@E2d1dQYH&^hku70`i+%uGraP?=m-p48l0z^Uo#YEkdzr~+5 zYD%W6TcNCyO(byB6aKv)QBnaIUu}NcTNeWug?6ueA6+wKef4qS>#rIHRZL3CK#Bsj zKyoA^?Ny#wQ%Ue~g^vr2-}3@C?If~G(uId+28N!kSYA;Sp7U?>R%*1#8j!bM<QK#B zB9m?SVzHL1!MijdldO9JN-usxpYFA4GMa!oQlCF|4Nk3>)UQfGdT&UD4-$Yt-E6Rv zB9UqoYaj+D<=v7jEdX!9g^cy**|!{FGmpE()K2r=N0?`ACkXPTeg=Mr!`Y1o7@fiu zg~shTyp)??HwP)xXT}d7LU-Zok!Ih0(_HV;afNdqN*UftT8Ay>;T)K_)=QLy`O%#D zRany1=^_9HFs=tX0e><p%gX;(nW(a2UQr>|EW}Gx!$)H7ZK2+7mV<0<l>Khd&C48X zk+iU>Y*Y7A1Os^2m%h_DN$cIfi;b^FfZmDhV$1UZB(<HLkRg{E|Ec|}cznM%E3j$C z9?sNswq@=9fkz_lBlOP0y|DF(v3zbbHf;|S^R^{ICoMMli_+K~OuLZ1e3;Fg0}%le z4YE!f>smWI)k0#c%$9X=5lF87KZ9oc-$65Rh|CO{r8B6G$W{iGGfoZF*|?qmeDKIO z&#s;5%OaYm(BAE3q(V08ZZ`zmo3}zAuE^Jhza}hAn20H~GGOJowbp6)fGSBSx><$w zQGwf3wd)Vp&`Rz|;lM{*DOoX|*)V(ntCRZ9AT7+nfc{|uHQ$#sWmek4^G&S$j-_;{ z4}U4xyxRQ-!yjRVFPke{WwAVGQ82a2`DznVVofp`iBmtmjvw6CEDIY^Lu&r~ar`4J z{XB>G<%rjEbr!I^gPXWyzLHvNrSxhb=R%+V$5_gz+XtIQh}cwVRNf0hKx&4c)C1~d zUX+)~b}L_8zxQ^MpWwpPP!N8`Kvitg&!fNS7gJ!x5z1B$h;Go#v8i*`>STi}`v<P} zG}k~3E3p<w(W>D>_J&AZ?=oUs_sZnNX>mn8!!5Z17|bP*`X$?I-3i5a#)>@CsV0)0 zyxSOxoj~%zmmP|r-HHHy!gJqclo#HgL{Zukk+*1PsawVc%ynvZFYSV#vi^N)6jCl2 z5m)!4_nqlMQ-@QtU5V_?epQk216J)fAA7);3(gAi7(ICo$PX8266D#!!k+RPi@Z>h zqB~cnGxyR`s3A*KYD|`P$3gvE#NGQrXEpq2suEwMM3&{1Cm7C@(cV3p=WH@V?tCoV z?bEfu2d;he@48X=F=O_4X`NjBX*IN3V`ax;_Rz$yLzBNeLRnXNMFY>$?+-18p)*lp z)v@Eyz*YQ*{BR-(Nogp5v(WZz`$&X;J9?+MUHQw@@||k)=CxbCYI)EHpAv|F{9b78 z<}$*0=SeAed^j^)CU?1W>{k8kY9Jy=2XCLH8&c1oUC{$aY!el+K}JCx9U2}wZNqoX z#cnnQTnph}mz*uSEb>ZhlB{@9GoP54oJGv=U1=tUG{~y|J?Z4#Yal-@3gmL6ZK4yK z?74T4+G+^f&UN426*Veh=Usn*M@+H+=~Mg1VMDNs{yWiIlDl~@D@q;C<;{t%>Q16N zrG6G;`Rw-F*omSuqdIle?;8X731)pEb8j`9)zie(kK2b5DM~K>3-7V=f|ev7DK3<v zuu`W?zR=?Dp#8C(&&*Ax;?;PVe{l1?b-VJv^5mVjOZ)$R#{p2<Ur8fRlfB81y67Xc z^>i}q516%~)59T}oyLlM=zihy+`Qt|p!sYJ0TJ@r@o%P7w>}{r#B|^7Y_NY|;kwR( zNGVgtr*+T+ce{E<dz!A|3qwBQ0geallfUnrGh9RKdX}chtdwbZug7jy1eK1u8c}r! zS-UR_U?tb;+uhtxqWMr}6xtfD*^1*q-uK@1SVDO(rnaB-aQ|-kaIegN&k6ARGftKW zskvbfsaI+T6!peN#&ls)Nt?VpC=L4o9<Y|Jwh-o9P<+T>x(U22IN>61dUe>Dbo;Jg z6rte(MzzCTj55{KpvU2M_e;`QEEH%(?l@1k8W7-a423eDm%x}uOg*&kbbhv9EUB2g z?C4YQr2LA9`PJ`?27)gTSD3%DdJ&Bqauu+Mm7MQ?_~J!&IP<`krQj$J-BIYtU|qgU z{~NaT4>R8t9-q#rz8LSz*OYMWi0pWdPDhLg4)uq%`v)a?m(5A1UqWfPH@(=eG4U}5 zXe^LC^so08BUbJRTX0v&CIBf@9H>DbOZ^cOb=?ir70dE-R|FymFS2ocTpbfYHN~H2 zNNf7oZV&A~73D|)?;$0(N~gp^I@I#SL^_s^5sn^$TOIL`gMr<}p6nv`H^%Xo^uY6Y zV8USorf6JG?lXci4OO^uWETDAq#Z#o?yd~He`(&0H?IZ=t^g)aXAi^9W^+5fyy~iP zeX-nIg-tv)VyU>wd{N^Gniq(f9TMYPjSBYG(t#O5Ft=U}NZ&S>GEG#GeA2$yxQap_ z_Gqc)+dC5)!jYU(869Kf!yn#;>7bo>CPDkq^z%$@iCHVr)#od6K%e@)=!xAg7=Bgy z!9`-<_j0t>p)FX$?#1;p-;h9+2p%sT?AO0$u^zZqd7K)2kV+;?GELFVJf@gel5B6_ zfy%%Kem^fEe?%zv2mZX&b+$in<Ki)2AbByRY^eK50Z$Rb7WdpC7+CMQ?{tdk$^#t! zKi<_6x&j^#53)q!SA3fI2Q}_8<w}1XJV1<1?g#*LfGk@p?e%)8<z4|jsJ8gA7M1Dl zcNcM-w!ly>#hf8XWUGw#*Y;xI>$g*z`CD%YQ!?&|q`iJ*@-C8VP^E4;Ub!NHpMBi% z`BE<38Y_91(UwJ2^Nc^Z&Vd<pfN4+twNXX5c<aHmU4E%}@r}@jkkdMeF4+$5<iayT zV!I81zS@<w8DKwr&n@_9vO8bk#9<Q7eYJ7bx;rs=e`jaaC>S0?m_6w8N%=b-0;N|J zITiQD0Oo&Kf{10XSE_wjg0Z<1BtA0Xv$E0*Jg0JmwajhSL*G;@#7(7o0I$z!Wr{o| zJ~4SNLt`_-^<PbcV<`Fsjh{~7noN6g(&?i1&9ZXK7Zo5U2Gu^=oP40u;Ji+J*S(yJ z{9+b)Y`Mh!^!5FRa4o@R$=3UXu6>O^B$u<GuEieDfw<-*?1b^`_H(E}{90Wm%0jY< z-q9Eo)H;zE51?|fo5em~JPw$E63g{91F3lhMcvkTfZF@yG_Mh-*shij)2+88cmYqv z-oig$(C6;dUU)*9|L@h|7)rzO)k{Tnqh{e8u)S(j#tG+{lTdh-2|B@HgWkGu&R0o* z0C_u@J}=Qdj=SF@PhoW}BTQu)Ar%q|HpaS^BoiEc3w)9=l@(`#29H(@)*pKVg4P16 zuX@Go!;_zvCgN2D)=E(41@csv59DrrtjdloV7DsgAEHWHur7|=$@k44%UASQ@UEWc zK<AFCj5CuQLCLuurN_t`k)ySeL5Fr@^HTUNyP_q?mW<`>aK>__Is+U5&K+t3p86h- zO5$2wq=FwE*zzsZIylp-E~U56!3`q0eZ@}Rwl(7YwS2OCcsIW`6*qh?YgiUkt#*?7 zUR{z3G|JkHvN9J|%nY9%i~G(xg)V;OS8DU0mP0*HAxl42^EKqsk3QYy3IS(}Y!{NZ zn(dJ=7@8GuC}ciEHw~m;0SK?PiLe4_;)Nb1sBb?>{VT+<bP9dK+6#uA?s%&10iY@~ z$R^MqoCgRM**x%|f#ox${T7WYEiIVCsfypx6W7SgN_?FHTt12EQtkfzm<dMq1HTKK zlrva}N)tqW_);mvWRvR&PJ*g<@d$t!9r<0WxU<R|IA$m{BMcxcdjG}QFptEBAa1<R zSN}v`Ck*;b6n-C%o8vpT)hSukmNTs-$CoX2OM*5x@uo4``*k*&{arJf3M!CUvGiMe zlKnnlW84Y%$aHD)L+s&|b#*DQ7kJJZLIsqz=YDN+$g&7=F+M*VKdc0lz549M(++vr zFOY=tOY<XqzLMIMDUW{l;f$HeV<Pb4pueVVb-RC1SHLn}ap@|L7BCEaftRMSL#<uM z!;S8fu=&<BMhQ@dxE5EUnuWAiFZo{6kn8!I{zC4|$MGFPLL(i%0Ovf=ynltia6tiX zShm-oN9-iMFC-jv$T1V<BWAsh_xQD5j>fwM<WEuA5oSU5v>mpC*OFChwh6@*eg8yF zd*J}i8<eX<_8iPf{Ntjax}1nn5uNL6C~YJ{c}LG96+U;e0iE$U9Jj?;`k;MeUS<$V zj-J&{hR;3*NOYVQ&-GbzbE5{;r5zRcLp+&XTjtrB%#SuFV&=1A%vp=-W%DxbWRAYM zODCjj^Hovq6%1CA?ILNVbb7qAm_OSzKYdnE)BLt{I>)~>Ml%tPf!P+xzOQRbi|Jo4 z>fLL3rkF}p<>fN+&=%@BzpGD(Ujt<(YEH05bVwdOS#}y?%(A9eze=-TKpY=is1+8p z<^b{14BCOZB-OLXe3LVPr6Zs)SXZfJxW>2)NPFo5i8Md+CY+N3M4@th)xO`1Pa&~P zHZpHpWIehMXa`N)XfmFySkFU?(E+=OrwZdx_Gv%lC99)CJl&?USJdG#MkhYzN$h`P zLEf3h2uQ%8zjkp1JhaB`_$-Qt@Q$|l%av7maqy=o)Y!RoT^}Fdc`F6lJzy+7u7~!1 zfw8MzpJqR@{j<QW{*LzP#jAp8tM)-MA5H6PXV!i$tuM90WGka5;%tdtO<JzRRE62@ z4OFzG6k8eRk*sMg=^xbn(~UUW6RGILlZX{p09L;7yA<v17F%ozSLUv8zrd`n{gO>T z$f(z;rJeuk;96ePVc(7bH~99QOHI5*&*em0Es`|o9&q_)Vn?jiJOA?n-~?}83`T#_ z;J-Rh!Sz_KSW)?ny^3T=i-pnDK>8t#i*&%|HXgx$th~Zh2yTbts&d3wTW?qk^d6+Y zO7t+|xc<sq`4Mz4z%<5+9zIevOw|;sLZ27NS6U&!=RTu1{RA#Pj<Ga4HggxO%_pPl zbTqX!E#Ee`?^I|k{*jfcg%c?AEek&S@p=lQiIzT}9|B8=HIGuN2uVcdsej1*-Kp?Q zuFB^keI<wIH~-R|MHXPH_ekc!P%@Ec{AB2`Vb$DfBXCFKN%VBFM-}CNu3<}y>?`~B zrRU+NloEhE;<y3y);fUP8NLD-NL}!}t&+y$N`TPE!j|{q^HgZD3_sh$dtW)GCMF`} z7kSTAl+VE#@=C^z>j)(Pk!~od>o=!WY^G*amwYx*Pu+F;tz}O<fJpC#4ih+8(d{m4 z=dX{L*}O|0+4(A_Dr3hgN=<~>0C=R^jI9T5r+aSr{RM<Hz)hYxH_ac=A%ve{4S1nw z({$41Hgmj$x1%)JkH02mNL!J*t*GSzq>NcftCKa;ZU883b;osz-cV7}bE}0h)dw&Z z*fyQ)O)5WFP8VDDOL1Pdf+o#rAem`sY^T*v%qvoNU()~VSqzQOX42wXogqX|I(^d4 zu~$rsuhu)Zb?z@Oz8>%pW(Ln?r8wYgK7ub+D={?8p?koG3Ydib(Y<Kmf%&*?U}3Nr z(dS4c-fVA(K0yK`+rr23b#?4liu5So8%hnMY7FsQJ)!ZljfM$x<u#RXQX|1~^Pf_` zUfB0T<%Jsv$x>?Yj5&_N$(dqh1qhg{MXC^0-}u0d<1W+2$F6T;EavVc9d3!!f6x}R zajSM%Rr?T;<E@pFUL~8N$1x<Yls)=pY&haByMlqe4mhgle$(y&YqFTyNzIj}xS3CH z8E307kJX-~eHJo{%RwLf-yjL*NKlx~BP%lt*|jGn=if46IJDiAedtA))_>;>>gz~E zTY0*ijl1|UlGayNLZ^%W49@o0cPY9pb3~0fUboq_wvh;r^IpeqyRS!Yl9iE)<$Pw2 z>&9%Z^rHM_-V9Sc`+-`=I<WX>V1OCi>IT=POc(>=B}r<&z2oYQ`pD?PshE4+COrvM zQ>t*IJ<?+%wTl^fvUugDd3V+o+^pm42g?6Ne)*;5R&9v2SZijMrUA45@LpwDT%ET( z&0<e4rogHoJiCf{_J!*F7w6q26GGb*B5O|pZ(W#GAqib4esQBuWqWSGSj*qp0pgqP znXG?@q%Ri~lwy3PrWm=HQm)k|2e9(4sQ_03pVuxeH@M#l?q3Zxx9M^GbRmulnLk>_ zvVE;r#2QLFrUJH`GhkM=+vh^RWXaK9Fl#F>p;J+djRd?zC8KUdM+)vwsCxUK>NP&Z zPHn5wR_KIOKBSs$6IPvXP<ZH7t9HspJFBbhbv@l$+A*JS{*p%)s+FL3Ky^j}8WoU? z&SUI$^#5=I-R`;6trmQc(d~v=r`r@qU^esNU#!Y}63#oIk}ttlI+;o27oSQl&WAyl zjZ{oIzRVvetCx*%etVNbRhb*ImRpym=dYeGIP<CNlN&x-feI1iM~zpMj$|$B3N+;5 z$Xzb@TWSrh!zYuMe0E~BytkJerM@JXIgj3U!xRDj257sf8`ZcmpFDWj1pHv~^t(4P z+Zf7V-7nIXDo>he8=sTU4Xecps5Ik%T(H3fEd)e4{~)u>!n<i?xhfXZVFCa*yf*Ij z`AsYzdVDcu44jGUNZ8KZ)sULG9?~E}(_y@oDl5YL^NqK*ZGw6%=VC5e1W3H>kA(#* z3tqK~Q4g;Ri=(Z;3I_Cir*0KlN$H+$3#67F53A+ce|T2E;p$UwaWD|2`t*+T*bl!r z@ql8m6~nPn*1qB0KX0XWuW@a@VP$;UDkmuT&$)Ub4@JSg?wj?_z@7j=V43A^wlMOW z^#OgR<u6k$Q=CALz$HMN@y<7+K&L<5{lQey>xM@Pi*kK;^QVAPbBMXLqk)zc=5f_h z*KuL(sy#jZ$Ayx{iP<-1Yl&w<_+)-W<m;$^tlJX;eeJVsU4raO1}MMU<faj@CnpRj zZ!VN`K>Rc9gHg6VcU--uKwkC{;zPk7!*2`7Mm@7<YTIjl;oXE=mOA9><SBBU+N;9v z<z*1*6L>(f<8CO84ti;SL1PI4*t9Dh3Hi5W0*EUac^Cx~^EWK0xT2Zod%%)bs^n<2 zH0T+0TdZGd^R&q5lLxWslv*P9>-~K*0qQkiv8hC(f6JSq`9KJC)h`Zsg2bK|@^5iS znOwU|j0-omjQK2;pY`iEFGnnx#zyqT^8f7$VV$&aHNv!H=9$%U&_Clh&soR4{Z`{~ z>zw=jZ2K1tlJwk`{-~CAV<s7HwZwRgQa=z{;D4QtPavp~;~W3^7zc(*(T1*lid#XV zEVvFFovS>&E0+c*(&ER_b@<MvMD(u;SA#vhV?FRt)1czmUZxwnTKrcP-eA#s`~D9L z!d0n<GjPObv)y88Qhd%gqb4_RwY_H>e&qi=lY378-Z7y5_Y+dvOQIOx7IOflQ!*dd zQ$wR1;~z%-Lx272Z>G|2HlFFJ2cVsAg|4bi=CktV3fDx~O3q};HfWSrx}L9c1!_F) zhW3Pi{8~`slcsMTV8+}qpRFlviG(tm`l+C92)R~r*u46}St|ZX&>HEY=RpAg12%<k zn4(2(t-<;s6^v?wO7Orc#RJ+ic}^a?p4EbVGgrAT0fvX?hmirdsrhhHor!{85TYfp zmEWXyv&A=`=y1@?Y`ao7-NhpzfId}L*SQ^edrOj%2~|*mH6^mfsRuymfX)BFdyA4Q zDrSZ_8ziB(U)A<ZNUXA%w-#d;k#~q=1&IR(3sRHN3PajuDNUGkwR^J;eP-Uk6+^HJ zJN_-@s6#dWx%If^5zn9o60Gcxis$(*Ka+{V-#TgJ1PqhK8hRUg^#cbCfi*e=-e}qg zsjqy1tS&1^u&+DW>uXNz_jixAFV(Vj?CpeEY!7v-1*u6?Kntr&J}b}~R}tmQBfDyq z4~qIvR~l`9dvuqgG3P81?fPoD*@=~`i?vc0b?dP12AV|GpUdTudyUpV=H1?&KMd9A zNP7~z<e0U+iU9~-a5cPP29IsY66ObZ`t>gsh23+rPMFf{v1s1!rurN<(1ZVvskd-z zDqzFLH(H5-3P^)kv>@StkrIjsigXMF1Q|WLLqcgusSzSbH;itO?oR0(IcmV~_`cu! zeZPNz>#*xO_j#WCx#OYY_t<2`qBU{@06-r1``PAyxF`QDeA)4ls=t)D;9Gz3r&_<6 za>V%Eq+SkGxalJ-@AY;&`I#`dus-b}re^X+4@j(*-5jf|(l(ySp@+h(%RxSyjrzQe zb(UwVd_95q5IvlCd|n}E?d+PcX}sqtj~J#7IR#uyZaX%5UTwo0<tFrWwg7E~*OW~J ztosqI{)Yyq^VQYAMV<wTssHAVo}~HXU-pZ>#MTWR-{{)1*+CueyHQtDC?1CqK0k|F zF?RJo-_*30^_QihqV(+jvTfXFW&kEN`D5;Jrvj6@!9L(XyyH{-JMZx3GW9g33gxJQ z-pZB?cq8vk;`%oQQ%6T1DpPm3gl__gz=`$`h8Np2uBVyle@=f#`MCXV-&hD28Vjgz z){JY18l&oErz5U+qo$^d9#pb`S;US8-h{rp6rQpzs;?wiDhNWdo>auQvwu|R2%88R zIvNj#c>}#H4{h>i=VHVT>%_n-6h3u&vchXMeZ%5E-geO+_R64yil?C%02^!U0sZ(1 zr_>a3V6G<w?%@Ib8_eEq@WGvi#otTA09zU#i|n7m{d9y4eZ+YGU4!(^1soF&uN^Z3 zh?I6jlQw(j?R0Em=G(4d?gFTgq4LE1amkDOeMnsi=Z<E3OEbss7mxoHSK0`#>D*9s zq)Z^2%0cEp!oj^8AhNG`o+r(Q(fW#tZG_NCx21C>^W8J`e_ry8hv#LK=0LMMXtx^? zFZ##Y%EwVqYRh-?!C-#jt*J>CD09FQZDOV}g9Y8AGy!1g{e1Oi?&Eo;L|0)7SXH5| zun?9y8^z?axo~{gp+Ybndro_zYeiwRb9h1p`?+IKTY@`m#s{eP>vur^m5VlDWq~y* zZ(muySVjGBTT7zZcM!npZI6v7y4UXv<bhcgtmP+!2bx+1H9vU}G?|_Yfx1O#xA!K; zPv2bpG9i$#ZoWq{XgF8Ry((qhKQ3~2B&><d$)$7TgskL&@t?g4hVxAr6i)v3pru>G zvuj>;&JUtmtkB;qY{F1(_)^J<KB%ely)b?29BI^~so3i@|IU)xr!Os<LxcAmlz$<t z#xc`G0+o?xMVq_6^uU=+mGZomi?}GP1Gv79vuH;b0WwWU4BNI-(X|%Z;vDR|Bv7N< zBual|A7LWa9VE|+Wg%x#a%Xdo?`Qh{qlSg$a#T|9ZRY&KW(6ONA5f;7)AmrUKkA`@ zfl{`G@L$fJ(;w8&<7ZLSSD)(XiiJev){dWp!OjS~;cvPRD<cs<l`YnZ?tKbvarno0 z{5TsjW{1@LX~7E7nedEF{6}!1!@E!S>trx~9A}=-iVjW{JBGI6Ln?QCq>BH=az~nt zh4!*znOvR1Mt6GG^?-p%7L-p<1Fh|<|Ftf*ogdbbSJ9Y(QnqkRQ|Hp*H}H>ezqDS< z=UDwC(PRXq!5Pzni?S%{%#II#5wS*7Ff{HT$Uy4SH8IIkhffaY2_mVutZF*QS<R$b z13#1eJOD69^F!dqi_a=Hv`ad)(upG@e^l@rnP1!nOCb1-3zoHKN|~;B@kajeCJNb( z4QCSn#va@G!~n-Z-EM}3vkso0XM41b)eq)3>2670A4K%5#={L7ZmuwV*DAK%^%ze~ z>>~fb&-8)Rw}RG`9&gmHif9{(>rUBOreun2JeC<uPv@LFxhNip^aaWk)^C@SgM0=q z*keEJ`-L&|NI0WO3(InKZ-6`RYAeug@s}g7rl(YB^Twm&dG}B!3;Z<=Fv8LlH<vPs ziVPA%y6gz_(*e_iMxRsy{mRMcJk>sT!Mdk;P|Vawg~qfBUK%rM+cT~cyY+LmmP*Kc zr9t^JFuXm=J-NI&c=DhlZ1slTp?=!k>#^CZO714^-Nsvbsi4Ex*@&FXi$yjYGcB%4 z0j5|5*{+Z?fhG%LkBg`XkDqn>0elWA_#3T!u$;AjWHE9!WzR+Ge4}$fUwrcMNhzzz zPk+0fg($6J)|K{kACO=#krA}|uDydqUTqmW1<l`wzD6URUp;;Re$0@<c!u8ht`fv5 zq)$or=ij>b1CmG)=@mM*?RmUI+*^L-3jM|hAnE~rJk~xRmrTQ4Ky?<aT>nhxp2V*T zlG(@gn{r++!NNVKQG4-%Dl~t8LLVH6(Hn$rpkmp?4E!z)7R(O4Uv?f2mB=2sf@a_J zedkZ=JI`4tCu5#_<dIy!6jk@9i}$oa@p?n&?r)gDb>?*l9*!!7C+XGMi@U8)Byd=+ zaP~aUvep{OLFNC*&;OlzE0suHMCsfM!|wN7y!C*2iip!o^*fn7U@=)VqK}vl);n@8 z7mFg|IA8nZiHV?Qx$`)hReJl^N75Io3>N=UX|z_uf;sSuUc-kPU)k^xWO)`e2hMxY zUq66S?rU_s#RhIMkNPyZXmCe{)-&W6^t;XyQlp{p`7ULr=#m*--SynDlBvwvexL=F z-0p#~OG^F%;)vBN6TuCNS$}EM6))to7A+yQI_R`mLA9_+-{c4iaJNWdiMq(%<zvxU z+%yCz6mv!(5J%8s$J3l(>aG9fvDm{sZudW)yjx;Aa2d9pg=tyS0wf-Nna0kb!6wXo z<0`vqp|U?`z9I|q=Q`?bps2Sy<!JB`+wd+mtY@MhQje?n`;)e+e)3SJ?w=luwFt0f zl=G$r!ORy-?3<3X*)naoQXP1(PGfRc>e1;{UWn_R>5ICsotc{JQ;R^6_|1#yWB?q_ zpRD$GiGN&RV}g~~ObuS9{Z3>N5P}W2kC(F0cUF{f-Z2)C5JyGnps3mHOj4%YFJm41 z<vT_#Dm;Ar6@oTgHG9|)sb-!(M7q{>YYY|^21NgF>0N+%-jxU%h)|%b*`%lSNhb7t zDD~R*716f`=P*iyhLg?!siAi9qucwhShgHt_KNkeCZsa_pGXT=YyL!*9y69n_#5k^ zMeOk=qyD_l#=wCut_XHIndz|``uyX;+q>HQ{(e<4#_xQhV$Yf(-A8~5vowm;;$Rv% z0Q^eyDam~TZT3~Th&4rwD0#8(NyZrXhlAU~<-8`)K}Gx`H$R8zf!99?_2Cl4yPC56 z8bFpJ=z_M5yDoG`qw26zmB=r2oC#J)n+H?+#M(>Nl|;xYcg>BZaAR{%9jh4E<pY?K z@llE3i-t;@lI!_t*vEx9N55;H(@21|GINYbu9BZ(k1kt5S?;M@iU0zwuM~js<BeY; zLAX5p4&zC089fkHhHssn$paHy+i!(L=%hpUYz9PgXEzHYI87tmf3Ds9Q>hAjZCI_A zp<q>dKqN!A(P^EhD^_h!U3hlI;0movSx7rLb|6bB2qddGH7da45~+VOPQa-!qgIa2 z*sX-kL;Vpx5;j0sUM4*;qWtN%3Ccg4P*_`PL9uEYUh}q9h18Oqbi-3wwo5xQ96Zde zNJA%?i+tAnAzrUyC5(jfvGm&rX8N{c;L8tvbf#3}4tg0r!`)N^m}jE~#b;Vb!_7>) z{Jv=0!aIGQc$G5seBQO3%YbG`bU79EfpWAP%N7JG)W&gL{-laT;b1^ifqQ&96U>Cz zFUek0nu9^rqEoyg`lAof<zm}Cc8haWTH!ZW4U#-4uZ7=S;nHKo7gk}d!+QS>fie8& zTw~B`x}d1RKC2tsaqG?9?yub56)PD62~2DMq7mHyqOOGu{F}PSPacNEB(X^ZmN+nO z9VIn<Mb&myj#v7qK|E$_z9t<@TfeQee*CNCX;Yq!1g5!VdR@pRxKP&c6M2J+iJ5je z`Cw9F!2wO(gny#LP6-7NL+mq#pHX5-#^heBS+;8~USEA#@+2HV$@@1b6eI5#F(Sa~ zwLDn@QgwsWvjO>>`MXUBb~`TPct;uB<87Zx@M@$PB#OW5B>S;H;Zsu_ZI~p2vD9qT zLgObx4xtpgo!M&METXg_6K!xEdY3fi?zyQa{9}QCM}8%H4<;&Qo2g*)&-pyMGg|VF z;c}2d+B#EQD?ZA#(8{wsSb~)ZCVAX1p^<zHSjsdYs&R)ChJR8SHFn&@TrKn|=iKZV z{(vDq9J_a*>uiS;4(8_sCz@7QK*JEewJy|rcTVwY!kt#&*ArqLCHKl-xb5Am)T7u( zA~Sm935wTUWmQ$5qV?v9mDx_JHAPMsqj)O;znhbW+~1JGr~|5lso=?omQHqwA`?W8 zI3ikn-o;4L^rL3C&srER9HVxF`>~1<0W6M~fsKmkg-c}+UEh1T(2+j9(b#AM-P!V_ z03$&X;uasS4^c>$4yA<l=WltL!8W(*qPOMNx_k`fiKSi$%KIvor(x@9fAwq(`G3&0 z{aFs4Y(MFJKETcAecq#)a2|KF+ZY=naeeUUZ1A!aH#ow9uWxZK+~?TFW$^BLX&Vkd z*h|@S$UdwB939p2Ftlm@aoVeg@g~G?6B)j+3;=a?hk%ZwI(LFt@~qi@BR%M^oAcjz zsTIkaBS}7>zS`cgAOCc&%46+#MCricdd(4iRFaLA+YV<AI;%7N)7xO|TUTHGQM(lM za9}wW?#%P+Ag6aYe8~DhgW>^N<2M88R3=2~e!UF>3Me|?-%yn-hcQ@rV<o3hYclrU zF|daoq4|haZz4l$hwJ%Nkcrnvap_`>CofF=?ZI7wOXeg$){Y&T$F?IPJMolxSjV1i z8~{_xM#N)wIL2*TsfT^9*64FL1%bZqX=x8H7smVMvd4Y?D3yoT6_-iDY2(^=_u0O4 z1=|XO@0)CE{?t?8501>G)E~&%J(qd1D`YmY`EQ?CB#2bd$c9e175g~Kn(c1>-Ihji zESH0H)0{r_$#_g1+-W=VGq1t_=`3EPygwd0(IE<o_I%X!O45C$3$O0oK6FIrhw<yJ zUBW^0aLb)8s46JJ)U4LE^WZFM(dA_zi8x1|kyBkbID&$`X`LJN<}d=NV6am`C@k}F zmd)Q<6*q9vSmWtiKBx~*DzK6?ki2ta-w1SdB`tR%8h2!AGb+St;oa3)Fy7lNWlTJ0 zE?#@dX5;@mtT6TAT|5`vwAl^Hn>$I^ZGd^O2;Y&yg1{O(SZhj5yZg%8duVop^!?2> z#buuRTJu^(Y>wwCzX^*^STn}Mnt}nDg<Z5n{mCZ%cHpNAPawC&&f&R$sK$(*Q>uyg z>!;&eoE*E%;C*feR;Rr7B*KrZB52(bpR-~jmT1UEJ@fsH<#AFLrL%x$m2mvcfjCAn z0S`-X+OU^-K}^YjoqtQdP!9OFw8r%y#?9`@N$Dcj_ge+*HAEZ43g!$JQnr9E0Ks<I zXVKDWepUJr67Wj!8FN5{P%o$FeC>_75D*j68}F$_+6iPNjEI#TZxJ_QhS=OCehRra zJ>ski2bX`61HCY@HYtj)(JmG!4F{stmX>qoxEoD;96o3k6s?C&hdkOQ`#F&K|BT%j z@chYzo0Z7|Ggw-FImax-;M#RdkH=aE@=SUs{`YCc0MkY)e1t=7q+Z{Wheb!qS$X_T zrx_i~hS|@=KG?bgf&#Oyp5jb}s%k)PjJ?wKKS#R0B!a561(tqtJyiDcJ~E4MFgw4b za+L%ayr`#>uO}3P)WIucpx16DQZFj~1GS_RY(w6<0sF%ZiBDE@Z5N~4n<Cw!7&Lgv zK^Ac@_@U=Oh?`LjS#Y~Cq6tYFJ}L~Vt@Z-0tDFO_{YZ@jmm7eSK+&q4Z%xBH>&mtH z!UYDiXx>a-UjvlH`FyM<V2uL&4YsZr_l1z(Bhk7YuleL7(GQ#Ckz#45T#Z+|_84MG z;Pvm6z?Rq#N)w+S|H>0{qw0DorMBz16hIsIPu5*!4Ww^65|xO&qGr+@c^Z8w_+*`1 zHle=yyS<{z-1Giv5zqo3B*dY>YK7<a9G^<M=f)J$<vW#-qHNuiCxdSUdyP&>QWz86 zDV2}r8W)iN0=v|%$#jK>xSHtqcxn4+uhqM4bYcluw<Y67&QlDS@1swNL1ltHZ;O~+ zv{YzPvWeJ6aPM3#DD8<KUjhC$HRiJ>;*ahEwqpqR<@8~BHq7ezhO)5~$@#z26Y2)H z-m{sFGUx~Bnl%d+=rW{x4r4!Kh0lhlzO3PTjmFBniA&TFcr3h=lvFhSnc=uq=B3T4 z39GO;^6kILVLfb7(Y6m!`GnLGBBBQ<p%Q9B%cQJqj465X{9dJT2&9&W6c!nAVlK#} zsl?@r%xP5s6ZlG(UxIxdv0n(Dr2suU2yRtJJQ;^nWcfZfq)7vcu~WLio+X>ERG*%O zb$((S&Bthk2Qw+;!!q_1S=>O=`MaOzg69N;CtRa^*A>PoBIdFvN-s;FU|hq^$=>(d zH^?I*_R>Pfxs{wQm@HaD2TrC<j*1MZXP#?^_cUB&9q!Tc!Sqw$d7jb!6XXB?LZLCB zHFVqUO^wb%KtU(wivqZdibTHS4Tk8zz#~itH6d-IS|RnC-1tx9@9z60pvw6$^kE?4 zySkqy01P1V?RoqqoqanMF9TU;{+H<r^hGu+|3sONTEmO!mBxJIKgkYt1V5R-!F)1r zix&!xU&EZM`s>|c-P<(s<uhOas0RQRm|i;T6cH`Ct!AU$j3)=!t=A^5*U|5Skq<q3 zLG*u`AYiG#3Nw<!gc_|mIS}qk*K{(vRx-Krzf$<GX>j4=4fMEylqST9!g|UMWB6oY zY*o%c0gQ{j>fJxd#fj{+0g86@^f3<!*_iA^Hi<{_uHok~_J>3!doC6Y)X~gmy+|JV zo16wWiCsR2Ms1$6`}(MpV#8PW+D2+lo0LA+o{+F8d2-CZf|tLo>i316X|Deuek0$H z0_@`as|W5p-gWD7WT5HNR}hN5F|$hbg4EC)efxpGWv&AXasBmxvvhK3ow=OUQi;Z4 zAqMGEI|&q_KLiAM+NIwM#39v7P&f4S(T~DM9-kC;3ndZmo~Z#5rq@xt$4Jc4XFe>@ zY{=r$X#x6Sx?luesldr)OW*+a7Z8dj6uRPbE1pg8oAtnp{y>tL^3<i}cdU1&_-1Yi z;Im>Ee~({I6NU2@!p~Z0<3M)~ME3y`r<ueNoe5^zL-`)wfK>c@8A9B`bufEnB<h;g z9v~QxPqxneK1mfy7xl;_-|RzH`T0CQqmN5_-prBgK^J}SFWkgf@b<UiH#M{CgR(u; zlDKr}DIC`hkGY(~cwITNgEErUTd+BBa}S5)tamgmy=_u7J~4K$Z{*pALwhY-cf5E! z?@wKtFurWE8JE3a1p+FoO}(EfTDJp`%v3uy`NwbKy-4{{d6s({v&8^ON>wS+LDbtu z1yp(x<4U0*+QZIj4~$T955?;)VXvo_nXf$m)I1Q42-Koo?+a7nGWi$b=2SD#06Y+H zk=IF{@kPix=jv9bZvIY2HQ*Se>I87LrAreFT(x>Pe5S2|DLuSm7`^V)ptuz0x!RpK zHwKJF-Wu1-lGm9Y07P=Z0U3M`A*~*$$98%<Rg3wkSd2$M;;d-w*5u5i1fbK!Z_}Bb z&BOqIsNe!_=Ken|0Cl`L^G<O6N6ydja}aI3KV^J_<NqwB8x=S}#mNY$IKe%5ccoJ8 zN?^K+qiQPpvs%4H_!uWG=+=o@0H~Xa((qmz16b`(1t0o0G1|n#gmvR7LFcwYt#&v8 z$t#+BK6z6-c$i^5ZO}8k>r#r$rh^=^p*{3NZ{a-$oGXrRB792%5A>KRgA!5x$#mye zr@U3?(aYMxh<v8h>fdAGz%r6?kXd_GN)8hM><0-uPdu4R>SH|LDd4&PD!=*)X}qFQ z;Biafo_iw(|DTX^1aGDmPz+5NS5|qcV%)AITnK!`Z(qw<Kdk(#kEcJyp!?YiGASc+ z`h9k`U43CTxAFPvUMCgsHs%!s&Gydw6-^ex=6FES%@m3FC5?4-dM(~vT1v!_-uxwF z8)B(SHKNoV$Lphu_fC}Ze<uLB0pl=KV@*_cxlSUn@lHb~bOWfzHvX<QL-AExzf531 zS6#&1>-d)&lkPCBAmq<@9(bApXa@Qwou`IY$CQEUOmE`I3p_^8BgQ2VR&CBe9*WW# z<AE!Ygthbw#rkMFR^MlN2uD(>LdOz$kEZi<Hu*T>0z%+ok}wTLok-&pkS0bC;#!x> zcFtA1`?fQ$OLra{rx8D?Jl>s3J8i%czrQJEO&y)XPX|h~SFe3j>B?CK?cOiE>4eiM zkf=@`!sw9|$(VwLB$lr|)6+P*wxl3BQ28TLJmM<ce3=~PD$a5|wcSxvpy*gk2cv7p z@Pj=uNC+A!Y7k;3h<aWUKgW0C3K`PujEQN!bQQXEmD`&eB7RYfIxF*fxM#AHSpo0N zT}OUD_Qe%qGJ38X+s)M7fZd|NOZPx@H>MRLSvMG_f2`y2>hiL&k$`BJ3m+_3V_V83 zuwU}U_klk5>U2GYA9qn;k~j!8uI+~~B%W`-7e?XJw@**_VCca%wL~129R&!r=r5)Q zyeh@FwFh*J)~-$$yoY{Jc*!j-Q-Ly<YT>hgX;$AtT)Sxg{x3F7MwjQGdc&N01>hfC zkG!JEBisO8<dXr|#Auil$`d$C9?bvqnex`|l**(~o=zG^*cZ9aA;6RYhx>rWE5c*P zxM!mAQXe@7@!I!a9h8Mw&FHQ@yBt=lHLV+Qxu64^ufD)z`2#VS*`|Lj$&NsT*plrx zEJw+9LUc~E$fBTk1mIr4@#UZD9KEHxz$tkh@69m*C_+q8_!Lhs=g<3B$Ez9~(S26d zYS?K+k=OZ9MZfkX4QXhObSBp7P;!$8C{rFkOr-MGwNq+5>W_tPZ(MXqvX|JT@YG?u z#`QTQXYNMT$2n8aY)k$fagFb9KGum4Auq+{`@cOj&|G=u^M+Jz$%x0qftmF7Zd1$L z11)@U1}3XbGTVP0ODbN6(Q}&<ErkDZ*`irI)zDL6*PqK?C^|3Dd|?1P(5^>^DLoVL zZ8&~F^26jSUhR8|#2_zuhcd=YW3~)q!Af0UJgKgK(9(3`);gt=L#-Bg^4S+i$Wm!k z{%uFPQMRoX0WD0%J>hG4Lb?Bh{9Dr-L0<erQ?P2_Z;}=Er<@tBfG_b~u%`bE6_DXm zEDx4qN}~fsuE|*an&{i&>jbhZ;@EHZ54(K648~{&k|@#<0x5bCE0*JdRP|Z7e?GOL z8h6AgP?^Ji&QJiGwxO4o7gj;9zpN47p5Fa>^X~Fd!K)$?pbw+^Q{&|$72bILy5lRh z#<g|jhu{96>2!Mz4^gtsKl8yGQki<X#tmdUE4hjEoc`6CUreemi5)+jWzCm<jmQtB z-fO!bFnH`{U|t4zOv9)yKf7qPbauIrgPuzThCYG{31-^xR{02yt1PrJ)~Ac&tAzVR zWHFgc=sfg#zW=L^Ds{JaJ5wY;cc1@{GAr|KQ!Unq8Sa9Z7hdHHLkVL-+_#H#C^=;p z6*Zq*Ej;zGYzF0EOm<53#f2um!TDm+eS<Z2wA?mdq#c-ZzqAOV#*iUdEAynEk<^)v zy)9bMX-am!iN8Jr9PkE&YbW?0H=0@8F#B!X+r1FblE;x$vY-OZgXoI3Wl3~pm2Pz* zJoPzhROGhqsZY;5-B2mrIu~Avw%u^Ze@`Gg728xy$5Ves{-_mg$Ld}x?1^<Xn2xbN zj5Dzwb|nNZvRtY8#QWw<IweCPNe0}f_UR%wPZ!wMd3tQRyvE9WQ>z@YmSgLwJZ>2G z)ll&q(-?Bc*|c?PB?Xl$phUqOH~N=u&iQ0rd4&x!z4qfuVCJOO>5~F3{sdiFFLB9w z5aFRPoTM<WcZ^)isY`ct@aIo;$LoR(Soxm<(+FVNT4p})dh}Zy#IT)ilt`i|kK|{R zy}^Pww1dBnAK_`yM*;`dl}Zpt1fxfRnKJf=!4o{gZFHSi=5|VovF6XWkoUXwC<2ZF z*y-%yj+vx;fFgm%zZ@#9Km{??2+)m|Oe}JtaVgLqpf8<C&*2xt@o~lxu*$;tFPy&v zwTe<D9bliwqPPj3czCZMQMwXW0elG$fF4meys6_~jCzk6>SEM>Gb^4K?)*>xHPL{r zShWURS??MBT@^t`3pZo%*yV+5Y%Z)3AY0@9S>miQ+#bEyWw!{4<Fqj=nvjb=&9E&{ z&vQR4TrYLEA=hF;rLds}X9&=(>Nn>o(x+FIRrOZrxm{~bCqNn+t1-0CU#xlO*r|ig zET(Kg!d(tG*6WG7oXy6~W3CKb+t``kS=&b^S-MUE86HXA)5OdRufm>Mv+vujB)4vd z2dDJ@c$DZrJ|#2ZYd{f!bzkXaH8pSR!xFKUAvU<Zug}+uHg0FneQNIF0R5neRQ*&w z!`&AR77}tu4v0jgea4r$9=O2`9G1q{TfU2!I1FJ9Jo;zP-Sn?1hmy{1GtV7X?OB96 zwZ{R^yWsI;yIrF0mStT3#^xvJ+v(s=_hrO~aK&?Oblc@Wd<i}A7@5*3e56)cf%u0n z`U`LV0MRoj_-3fm3A3lEXz?(8CeYgSrM)JMVAJI-giPd)Vb%!@p+w1pmnILEpwpb1 zOh*fVSi>UP!a`>@7^e!Cv`J#ESPts@Sul)4g|z}s0uY|!!v%aFGmE%TrlL8AS=U4T zi6isfKTSo`d+hdn7XBo4l6b%;>8LGPU8KI^$rjQe$rMQ8u)O>qWjc&syBZ?4o^BTD zI`=;bNO_Z6na>F<`oxpVlZA%`+v8Ta=3J&D9Qb-7DNK8w>>t*Y?mydH46yx)5}5nC zqC6?6vfJV2dc@h8MMN2ubjWiSX)Ut{CeANvFcQnCB{XzvqVRnF0^cO~dU&EH<bJ&Z zvfM||N)k_NFlo}X_a#UMQJ3}nwzwjCa{;$CHRPvh&OLmt)j=1wlwbp1sCm4Vymb1g zcYO>z<|*hg3uNiByj)V}x_R})jqO=n;az(PNUy~z6@Ar11SHL3H?yLI`I*cf?VXIO zrh7_!JoppL1eRY%DS&LVMn#D^w$wwO<>!fa38i(e<xhFTZ*d@ttT!8%k{#CmtkML{ zTDYRH8}J#~dZdZr(-c~u6Qa|}oC3QQK1vJzmGAicB%|`Cn7ixRrrn&++8xs)8ob9| zp(X1Oxqa}1YU;=pl`ZsQfoe{^U1c{vyzS0RPo9mE6M;nB$#^$p6duRebwRgSnN>Sq zTW_}w7+&qofrRrMb(DLBZmK#g(xYIlxPCGy=gNe_t)KpV_*hQ5pyYR8%2B<7re;OB zH;k+Y1kcIu#2?lLrN-wcb{fMfjK2)5`tyh*wvWD_*6HM3&L>2l+zAuePCkhdgV~)= z9Aa$mO38DcXf*t~#d>p%iC+$X@ezn&bdy_5eHw}1@>Ch=E<I20LDpo#wK6z}QlPlE zR(`Ka>LKKedp*YMXba^Vftcgbu1xj(zR%Bt-1}h`XV!>Y<>IbePj!zg^4vveIU#ON z9XPXaD=ar99!Hp)2HZyDb)KFpFj3F~bBncCkXQ`BhSbh(X;kSrNIlx8a-)MfAGFiP zcf$7Ubjh9!Vn3_6C&GB}ituZARObDS$|#fZ&Qh;%>+wjg!57UfK<BoaYcr{VYCm+H zQSC<3Wi+$<#$~*$dp?N8KuApbCvXoI=~lS$#`fdC&im`ONZag3UYmBV`>f4SR`uNn zGOW+buR_>6Vf9s$N~-Uul#30e+v(eb>G{n+SW@}Jq@@MZ9tY<T1gL)aa)Vyjo2z$k z<T&lGGo^LGFh=Y&h&-?AOF3^yS*UFdYHjkx_uu;`Wii}eY&N1@liIMiHeZ*p2^#wp zOC7!5pF#=@Q;&KRg-Ad3CV+RREBu9Ys|r7>#5?wp_|)|ujO&$o<v#!oZ;lvs=e*nR zTu=39W0!9lwP~=s$V0C8$8rMKJ?t0LUvCSF215iEvvQJWir|vfk3e~sc!bCyA29yn z`msHghw6ny+!OtjCsL#8bNA5JwYl(Gd&O^s@AqeCTRnYlD*YzJBf7s0q=!=ZMN|lM z(Gr=@*rDxr&4QEUsHN7{+WeVamcV2Sa%u9yK5y`~(%-}zLeqXRN`gr9u+sb_Q)=q8 zO9r#FeaTQYp<X{XlICUOa3~QlkFqogrR~{EchN_RuV7-dBClU>%F;()q`mHPktCFH zYB&An!C@r>zT<;c0-@(^!(6+pr27`^lZWwWNW0GA{os(3ZZq1)q+KE#lE;IxK0#r| zn!E%F_=l@4AHu*ax;`}-t5rTOav#$A2(^&L#KzAFC3x|Yd54|j64K8TNcoUwvv)+q zS$=BR@!0R&n^~QHrfvhZzU`s+awmfkdZ0TRLi+xAAU4G#66ifRa~c`jLPiJ?aoGIo zul;bjewMt`0ly!mbN+&e1#t?jbFW+*%cVa14&!AJz9tn^AU*T>IB~J=7jtj~y(WKs z1ck}n$Q@i_K7i1QE|YGNKRnC=Y^w*<KzbH@(ytemKGN~s?>rLLEyc4L&Vt_G_8WXr zPA4|IBU)@#(0k7>qCoWLhnVl8iV0i<EK2aqUGBy5<Jxv-@im!)TtfG1CPgzw_BDJu z+56A)Ds-mESdN6_w#M|VEfnk#T$43xzVgl<j|s22yD<%3<)uNaAC&hjxz8rQ`>Rec zK5Zd}Yd4Wh%pUYHX!bjtvk7DDKXwuJ=^`UY{f6+wc~MK4kK17?W>dWF&eL^|23S+v zCZL1vm#IE!)j&y^eU-<#+_#$wD-ZV%AveX4!Xn}I+Kx(4yIj*a)LoFH(dLHoC?K!K z_Nlc{iD>|(=k<7pG{u`I@I6D<DgOcDGxxrl_L*oF!Cm`fa!=lvG4TSQl1B^Zl}J9` z@5_8o&t3@KbQ)--UA-y%Bb$z<O3Dl)XF=>LU|AbG?PGhq!lGo=*7#0;Xn1Ri>mpN4 z8y}qVnA_}{-Ti!-v1D8@kG`}V_Mc%kCXp=rY;V)=H9_4t8$N!P?V1@Tvf$=8{J62O zRvoVk6l+2D!Szj2IqfKN=ysRZD?DlD?RGYKO9bJcVr8oR5^AqUumaWZ+voS6ZI=Ef z@Z+|(1@&rLjjt<AywCx?!*UosAbc~CMp~LQ=FFcU<Rn?)_mI|W_@N6x2M4kj)#UoV zwg63O`c4~>5^%y@SE*fGx+w2o<Bo>tzJa=}Yj=qxFQ*1TdFdbJ#&dh{?STmmX*>E( zK^+#JHZt!%(P)>k2d2mhSZi<!Z~Mqk>*a8Ey$`&{0q-(%9Uj9U7?uyoGsx}EXDUFn zOk>rzQd^zWvfl>W0CUJQ)9!oJIbW#DQ7(UP@x7F=nHNtGb`#{jX@wLY=jJ7H@@W!q z;YrBvt8NY)EO%;hnt7is1(EMKpQXCDv=cKC6zEiz%O>a%gAd(0#c;OOXMaP|$6`2y zgwwdd8EqDm6Up8Zv!^T3LkVfbak@_0)i*7q>BtCX(DP5DXUZiS5Ax?5y}eIf6cPig zCj>|g<1$>;F53c|kIs~?RgWv-znqie#ecb_uGaw?0D3<w3w@DG+7dVW3*(c^+tWIT z-*C|4aoO0ZTvKN$WV^;DG^rgdwPy4U;Xv{-jxEK}=Q}=%#O;?}-M2#G!xJYiwsAcY zI%0e2%&4uhGP8lgx`Y)VYAvwDxbv4*dZh5#`v@kdNwC|=miqb49VPCiy?Fe}<1a)q z6jLJY*wH{->Tu9#<SD<iy>h#Cm^H5wH5jeLgoH8qU>_w^<Y<1fXA+e?5?;p`+Rwo& z`FIw~K=r`{7?BAKs}d%brmMn-3)6`$f$l%+P8yM*{bgZsmf-(|gVo?I_50uMw@Kph z_7Cp&9l%barxmQnRv&np1<;<?HG8jDsvOkEG{0{ba-KgdQCkZOWx`E;^q<WgL66xW zjYUMfs;_@zpD=OT$$D|&HN8Dy>sA)cUQ3;g=lwY?S&_nJmPtMcfucJ#&#Q6CKWd}7 zX5wj{@~G;hl}jNjeIQRe24-;<-*P(zy^Q$At>nWM2_t!=e3a%GUG86&F?KnI*$~?P zhVXcV4)i2kZwIk+5xZ^<Gv>rD8|f+$O2((;2f67lvUaBzWIQo?sH7WT#B+sSG!|7Y zC~5o;9_`7f%8~^~3fTHEVA_vueWUNo)|DDPee`NSP%|k!jOS|I@0WgLd`vdbNoCIp zBTjTYN8P+N7Nh-QQSh{5H|h4!#ZKA*Qv`T;;jjOLC1XZfzMYXL{s@WJK18UYGd}gx z2aCp=&k35iw|}V+kq6(&1I};<?=AUGEkt0?NZf2Dp5Kyq8=%QC9+K0xu7fYAURA=h zkoOtc(A!AE#655^o}z`*svP`%jchmPw$`T>L+4Ld7<7d9s!zKn<1fF~H?9ftcyMYt zJ=re7gQB)S9^3KVpwpxvJAb*&haxMa1r6z}O|#cxnWs-UD^G`g+U$3S^63(x1z$56 z&#HrI5$Cw{IW9Xft&a<)jiUo-=(*%54MzKKP-ngq1sV%WoXqfKD~NjG)a~};xx}B% z=`zHc4HZgLZ@=V|C0e1x>ur=osh7%9qR6z%(W>}pdcKcGD|r*-T4-JJH4%8_gKZ5U z&L@c-B{bp1ePha!dBH!)5vPvKRkg1#Xev^I=aDOleD(&QsEJ$LDc?<17cTaN-P2Z0 zdY`|e{|M!Qr;zHX5nGiRbVE*gne#4lF(ZUG7w?5JoBS!2yu|rRbh-*kqImSZ(ikrB z^6lg%cE%>C!ShnXF@p7Ho`5}5FumXAN7a&04mey6Y&j{Jx9C0@tJwm?|7m4Cmw_dS zP_cHrt|a~I2AGMBMwCSyDFTW|N)#Bud<|HgRj*og%>ZSQ`TPB~I|Akl4W4>+qjjy* z%Adz>Gox2&&K6rCM!V!9U*<1&@EL6CN{~AfaPRu}R}-3Shg}Bsi^HUR`Cxt+t=|mb z`y=rHHq2sxO_IehIe+OkKQ1LBI+X7eJ2R{<S3zAw<%6DrfJ`w`+m7GaEu%$^;WIM_ zZ7jqTU#zWWau-M3_t4n_ry<oE`mUzc2Qg}fGjq;GbYhP%N5VwGHNH6C+060h$9McX z{31d?wrZ!MCie>&)FBZCDRKqp$1UJQ_i<X3q1Zg}HDh!N@k^OmZflUh4<$8fIuz{c zC;no2S<yR{+FW9G+rk=SKNwfZ7y~H%L*J@@WWlrNEIov5TzNv?H&>_oz`3mD%dNYM zfJRW$bO3Nqy}1AUQ>c8>#`7MY3Hotuv-)n)VBiZoLqjRvUjIkuOa^baz1hYh6YNMU zLV_mgs?UMm`8sq*s*qn5c}*cN2J2aWO&ar`H^0dWO!{S3FW2Xnlr40hSDUvgVs@)} zlzFwp$576Hg2pA<GM+(fQbi5y^TO#}L)lp$wKK<V2*}5EM;8Zgo0RLIm>zxLojWf+ z^XtRTFD!bW6D2ZU$2RkKfCFR~el6G9Nwa`NXZPYM{=QC1EHjz^iK+lvwqq-qB4!>; zgx!(gbx!*?6#aXg;%e<Qv?lDaw_<|ByYp)iLaBb(y#koxP7*14R}E5mXm{%&1=FuM ze&ljOvm@tcier28*xbizV&r6hOQY97l2x_aQqSinS~_u3jg@X$2#;ai7W4FP;e}gp znyFV5pYch>zk1qAz8}*n%p8mViGM9gqrkfuvuJF|;tvL1vGzb-*5ICkp*dn13tZ%K z?E{J<<P8$liIX4b1`4g+t=z5QmLZb5tNZFhBWL~D-rRe7NB&-Bgr80OTW6`iG2Ob2 z25I%p+HDqo)wX<c_4Zj%Wj!gVtj;AJsNI?FF7>KCjDL(>omS$D80*OIy|TX=>hkft z2VQ=yUCu9c#AI5$;(^ALZbhn<M0dnBWj%tMV<G%E?`G|61!$@gCrf353%|AUL-k4! z1uIi}i^c_o;uD9o<!v}_<GGWvPb`$*M>1r)&!A0KZ@ec&TbkGJxf&Z^>>&<Sd&kqE zI`=$w-2V7H7E!D}pPxBF1Ew+ng9^Qn_M!_t=2FLl>*JFiPW85&G(~#wRFs-WoB-Uz zOF>r~Yndoc9YQy;UXjqJzg^(i$hrCtg)ZyTzE%xRR$~^;40>*N_2Krp7S5LA%XxIl zbNsWT8_N3#Rrzb)`zAl=to*PPtuBdV<K-ghIH|;Il=D>_wmB=KUg;TPCn^ivLoz%A zRlNWD!M#Mn;B3G3wjoCmtIy7%zw8>IgvozjanE=30dIc3m|+uWV<=s=^>#5Hp4d_E zbFQZBIwEp<0$`!BJVShUmu~8OAyAYMCS`cLNxoD-@d^k%^o@smp<kPQaZ?Cqyrt#q z=1(sCw1s?LLez1KKxE#9o%PvXZNJ39OP5J1$U`$#DZHnzys3hKOSPoYWU&#oq9^bc z77-`P2_0Nvr%B}-kKX#H2U0NI)(_yLY}uNtbuQmJ(s!OC1eVBX&11mK2$3hXwKKoD z9;9ZoB4Www6vK#5{#K%O@5l3T$JEpEzBCj)s>$>Ap8VW+<yS8b{2R7FMyg5phdEq~ z+FRcd$O};1+0OGMau-I!y|%NGHP2fII%|Bt@A##K=q>-;bRp9@i9`G{9gw!)7sB(* z{++g48k8`Q3yS7Te09KZ;#-3-S@(O?q3kdD7w*QPC+YCC4m<B2n`*2+^1(@)gjwc3 z9VB|uyMdJJv2|IWaO_R5<~yH97zO;+?`aLMYoL^+kaEZ7Kezp_05#w<#5i}tb2EzG z^9%9>*e*}P(e7;G&6)=U89#kTho?+F%fc>Cqk!|qW~$V5Zh))Y1%F)YGY$8Q+n?ia z4ZKiv-$6siUV3f=2emhJHf;a=G#dP=Q5!!!TVs69zT84{1SMY;8J6SuI%{oolA=8> zxcwhiX$(8leS_*ry#F_z{M()STJfrj+A-zlk4}%#l&s))9KZmjeh>7`GfOI2FEB=e z<Wp-U$Fuu3Hc^taiSn+{;7{PQr>XDrA`V^N5T1X#U;A`H8z*10j+Y<@YWhuSO=V2G z)W`_62QMaL(NJ3Br%$}T<s_zx+HhPW^CYv^czK;45x=b{VnEldPof(0ABdP`=R14M zJzZK}D;eO*`ah%7XJ0}y-+=TjejVd?0TJ&+xO^DJ7MVg)*M^x_%YDRgs?f?uIZqs_ zf6P?2YB03?c1}^_k64o`LWq+R$nEM=|17Vey~Pqe&wm!t84*fNOG1a2z6$o((ItHw z%?Jf?VVh@aSvmXIoLSl!Lf|ZQ3hKeeL1oA*7XJt#beiJX{VS@GsdeYRWXIKp4?cG; zzf&@w6vYP2#DfGAFU=>kaDy&|oNn|B?^0t1%f^X$8CIm|2;>YKCDPvYE9^-cz_c^{ zdrs*i-(N6Y6}T?RJ#|t8zhoYnt|nFA9_;{zOs>Jdc#-Q?l|L=!;+!7V1KxuY6r*iV z1x?^b%{3NlgjK)u>a!kFGhl$ieoH_5y32R&i6$QIFt`oM-qbA{N1p2sAUc+iK_4gZ zBzpMEjcYtIedRqRDnffat@<3(6NaTc*`Pe40M-c*cV1MT>4&56kQW9TAAeLem>Hu5 z<50#&eB+H8#OK342@xN?CU)pN&?u}s2k809kLZyC%JLo!{>Jag=#tK5{8Qb;fwKl7 zdhen+KJ-OXWt{k9$BDgNPvd2bnj}}|PWeJtn0_8PvdMJxM^!e=>lQ2DOX^5c!pIN; zAO0F{skcM1Bo)uXzPzLr^ymekN>$EJd-(KsY1)F?%*FKQC1XoIBl|wyK+SpFrpgkK z=I$p8ftBa3DzJDG0F|L8juE#dS$9NDx+tr0hdn}fexRTkW-4d9&BU_XS@D6lU6f-@ z5bo1}@|}wsw-R8x!{>DnasfyDg&&b&b6k0|r)$3Ky-a3wS@hlC$}7^?>}@tc>Gx9D z3cgqleZJUsS0%qmp#jnV<yP@hxR@KpxLwZ1{dZ&WsmOu{{#=bC0z_&|hTRy-{FFMn zw$68R0Ga&WDODf{6e!^(14E5tDVuo!bya6L6N3=M@8Cs59Ka^!0+idK(xiP4Po9@| zOL|ZX+I>$uUh>Uy#b<mz*dYn`O*HDI=`utdh|#eH2%4nnGiLb|{`olVPju_-TATFC zR);Hp0z#jNZKC^xZLeZAWe}tvRP0pmd02Y-2KL@9#Y4U;F6j9yeyWA#9HoWVs!b#3 zx+qF2M&i{9whe~dVKI`AYmdP}jpTM}gsc(w-x2@i<pl<&;Zu2cYYT!%J0_{%0S9VA ztzXzy-;r_sEjEo0E$`p`;M&DkRs((b{N~p6->dtqJu@p!XnyTtn{uO^+Z9AIQ@YkZ z|I-3gPNr&_$crro3>IfWPzFDV!>P(j_%=KbOb`4^zc~%pA?gY9XY4RU%Km)P0_4Nu zEKcgpFJA!^lj}uZOzwYCNKU7TkZrlg!Tjd6l-h0a5c~I+_wMID1>IX!BurBS8!|qp z_G7<U1I7GSj}EzCOETxmqTwSlu_xDsB8^%D<wN&526#9qpR$NK_GkGG=wTTd?h!`n zENgV=TlDuG_<Gn$zn<@}U}^WV!2Mk73t*Dy*Uo!}J?oL#U*Nz#VTjwei>fUzE&4B) z6hmXaN3(ww)^(DS3wi%5eu|ew{~Y`hH$#1`Z1WDy4YKc+wy@65n9gP_l!HEe^9dTG z9MmmK?QHsHzk1WL#b9vz3v4XtyMC;hcj<>qQKpZ?NLh99cm0#ii<BO@6^-H^*1i$4 z>TRlCc*7n}Pw^z2nv3Z7BmXuzCQOO4R5Lxr=0rq7^h<R)@xSlP_P;2^s@AAje`B)Z z-IuN&Ka-oP08;;p!knQu$VKCLGV_tiy7-bOT)%XUG-#zVPV}RWh)+t}_#`m?6P|Wb zZbP2iq>$<xHOKgaR8m+rD7X-s9afvMXk7lNz)0ilzLn%kJMrSFf`Qulo$ZV5Z+2Y0 z@w(05zpR&-^cgV{4OaqX+r9(ku59Jb;~-RGebua_xhaDCOiN5+bItEj0pPFJ<=+=5 ze^S!D+v7WpTheWmkyv+Xwo_dg6`C<6y+~0CS|el8#qk2$CbGsrBjZO=iJ745(-@So zUH+!zWA^#e`lhx!Hzfn){7@5$u-#bMtJTT&+x}}!owL|-&9;G_>5vGk^JMlT^{1L` zCQ%nkk+9L!0WpvMEu=&xp@-Azc5-~miid)6Se0u3@=*mrB&`AnZO+<`w{j26awJjy zEaZ!;VAIcJxdw^yi@RTp1Dx*-BDld5a|0EU5h)Lj7yW`f?toR*FOXA+^b<`3S3(Mj zv&GysOyi9Nch#&txp&b%K3<YpiX%-Z-E;Xz@3&-`CiH2U?+cP>OqpjA>Gvi>?+V{( z%p|7#KIgVwV1zq6*dJ^Ax_@GZYA$E-^7)KhYbyUdVz@Q>dmrefkgj#wH4GCP_Zf@V zr5K0X$xUvqO9qiUPd4-Ae&St~!bqIO^<4pk)ZKZ{(>YE$s1FVRsir?qP{|hdX0=vT zb2Jy&U527U@y~bau4j?O7&%(kPSgv#k69lxj{Xj&x6b1OTi5l~rHFaWpT|i}``d*8 z?ui-`d7RI?7iVR1@@YKlc-`Tk@3xsm?iEhA`(26GH#1x`cZ;9JCzLuAQ*+QkA4vuJ zMnCXmeK|ssLJRuBX(<^qS;Ag-(7{lmz=iGD>Mz`m)3x^Xy1H<^Y=Q<3;s6CASzb|t zC^w*bcm{A@QnqyK>%e(X*Y&IPMoahZRy<?ZR>k;in4fL1NS0^Le*w%;;M5$h)pGfo zz?@6}-Yu3OgTeG6Si21rE%w}+9!GV9omskQ_NUba(JBAgrBYILi6r0?3QfKZQIdXt z85QeL0?)q_qDk}x*B8^+ydQ;XR?2KFto!f_F;O^ZnrUhnMtx1*Q&dS_ZeTi-z%h4< zt~-8778y%vPj+wo^%`D=v2ZkYt;wF#vM29?cdBsSo4@w#c*Q==<E)I3e(j1z{WrAa zTjfM-MT2ft4>dV62iU}FKYk$g{}4S*E#_dBFLi$;venam-XkI3m6H1K`cf}Arb-3; zyBc?9!}Uh%Ck|??JCjie-Ty?jTN3dhn4el=<w&?HB4crRv|1S)WIF$vFb%o%E{0w6 zI{8dFOso|C_VPg!!n!f$jAN^vki#`-;PT#-t2?6q@l57ytk)e*7NigDg!p!>T7`s& z*+F%)clM+t5euJ%PxM&h>e2t*0L^uzNbl*gu6C5i;B?0`u%1<fQXR6JE&;jM7amk% zAm$@BNlF`-UWjLp>=e!YPGbB{lQ_^Jup$#B$`0|Kjo`5a<+^0_^3ScFZP?(671wuM z^PXuv7oPgiLO_$CEs<oOCQ8lv@Z2~kz2eAs$45f8e&hCevH98kU4{KJqMfhOBRX@Q z`z69&=eSWkR$g<hyBGo5YLB#%6fR9FK}##33XiL|d)J1r^BrVUvH;>C+_azzcG;vw zk+ChtjEnPt_L96yW-)&KpXb{@GMgSt8`Dr0D4M6v=!mWVf!p_u6^h++uJ(=rl{#IK z*W+ZCiRl_S&iy+JDg?}U-#nUl)>pUcPSg{kU!5w~+&TGVLDIZ=ZViuR-OokOxu-6A zKxeuJ+wvRKAQV^K5T{2w-H7<ee@}BdA;i-l<I<j@<ASf-`9;I0>z)&Xna5bVZ|lQ@ z*H2bX6L>zJ3JKU>eGXIH6v3j|9xL^)>rGWTGGExEAJcdo<fjZ2(yq2yuIj9FyT7qO z-QKR<d33sycqPA)#YfMu($ZNt;7*THn)=DabNOr5we^(jJ@$^2!7OM6VA4Z*Ne5e{ zQ}5j;>MZlC1Blo=ele1rrBuGM8Y%kA<~}MYaVXY?^DN<#(U?ch8&bZP*Kz`oN3EQB z`R{Gy0$7T-o@u};^ngSsik-?C>*CE{NgAweDw0?CQr-8UW<u~OKELm09h1UV)1z~V z0;aDCi#aZ2B5B#+1iaor&b9i?qk9u*kS;-koRP;^7}O_pU~ZyOQtz3qVVtXm)d-Q( z#yk@RZ0qUd$B=0Y?{sE_@p{AnjCh@be#PTXv*wri6xtw)129gz?zl-4e9`WUZv@Nt z3myCPMA^?3Q9!tDQ>8!lOe^aQNXAuLJ#7snACW~=z>IpynMBKT;a@BOD(Y(MKzN4i z`gmQERwA{>3vhzK7lC|O(hJ5D$s63!Hts9yK0Jubm#sChnZewqGr-#rh^xC00Us*J zj8&Gn3G^StY8pVFjsRW>_}SI@dh49`dB_nOIBqiR?>im;RbSwzDL_!RGgh{9Wx*<! zw}rc3UY#AHsVg1nGJ>xth>I%t#M!jpFB&h*x76&duIZ1mRYfGj@bRkZ-I@`Ydd-r@ zg^u?z)t^QNTykvfYF5(8%l`m89|(f_!fT+;duGTf&Ml_AtDrHg?-W8GER=q+d%QWa z1WclVmwUB*Irg386aKq37`)oO8ZOI<F^$1eQzc!+D34K2qM6rPFNkg<vX<1nh74TV zP3)$UBH}-~Ya5AXL3r`wqi!U|to4|8AY4Ma>7TUUK|fgOEH`S~#8F2gc0XuHjT5{% zfAilf84WpUHS3B#MZxbcb(<DWOln|xMh@Cve|Jk7V_s)K1y;)Y#w49B86{~!eO9R3 zhpRr>_m<FDuG``&V9>ju2cX4KbW)yJyd<?@-`52_TXABE{-`9g_2k?GG_?crWc?ue zdl_;~a*aC~iKy)Nk5HRP`V|;~gilHzE_h+V&z9yAF~sNUEuC1{hpCMJ9q$sW2)U(7 zXx>{}hGj8^{B>@i0YS&+OIi<0ywo^BhZ0B%(@Vp}gx<s6ab*aZAHcDM3gzKt6)d8} zkF8;heKYJkL9{_|A>xcxBSVDTPf97Kyu~fiV9Nr%ucAHQ-pApO&905-x{6D=obVqK z&PV+D`<-d#^I~7*O$Er3BvF1wYbEmT<6rKaxUb7AGV6L8oA`CvP*sed>MP-%S$45M z=RTEH$FibE3<a2C<)QtOlSg=^Zr0}W)l%unoJ}0)R^n@mwcF=i>Yl@R(&r|@z9TQZ zNG8h)Z|eARz4E_mGdr_1TubAFrb?ZoV~Jle7m(?BSurehH0k($ztl}S(l^#!d|p`e zxHF(5)4fETWm@simH(%!?E9<B9e*{RR|yVOV{_kohf#=M7o6*C*~IEz_^+!dz0raT zwt0r3@64?CL(vf1UH6YAM!yX>t+MGPv!`u#WEF4sav+_ylt_L8RMWlgfsz;Da}iVD z_IkYDU-Sh$l|>Y5Kz>I!Eo^lgB@S@fV_J{_G!p$Kx>Vby)?+sC9rMHcD3)p36<hcp zae3jY70YYCdp%z29S^HA<()P2f9)~IT;>C4l?*b`z8b$hV)$Y0xLKOnYRnKCozGhI z#VLRB*~E{Y-t?}?+a|F;PM>tXib7rVlz3Ap-owxScnS^HAbp?6`hvL(FV+3(WWs2{ z^R(b_apC6L*?gwb>IyRinZ2R-0nA|6x7%{+DWEHqP2D9*JQW|*?-G0^hA@*=&ATCr zYu~1bsyF427wBrHCW*TW2nxqI?p92+Twh|~C{h_Dl=lCl>aXLP?!P~Ne3T%qQcA0! zbSNOwa4Cuc(lv4*U7Pf15T%AnNs1^)cZ?X#Xb|bz$RQm9gAMq;yxyPd_xpbT;!n8k zcH7~39_N1U$4c)E^&&+yiF)Y33B)tmcd$j+?I;sN{<E5=rF0?jXJ>+L#<f|hxMt;J z$dUi7TZ5zds)fP58W^4dAx}D~@F$__eO+2f>;v_Mt139{1us^YhSg-cFLtaPvq+e| z?^Xrcs`K4758!9v*hhxuqY1MRkCG<utp!eg_?ct4rOcWyoCp3vIa@I>?2NeM(-Pi6 zFIT+ypvLn=ZMN2f_}RYw7qN@3Mtf<lqj2V+J%#Gc?!vqrf7nc63i(){c&Xisb>I`) zJLngaS-*kf8S29FEiNZ|$=o~Xuc1Wa7^}rj&=cgE5o>E)9Cy@xN;{~99$9j{`u-Fa zf#|iLm|Co@s(sQC%T7s|;ra9`PZiJAW+~KxWZ1NO8(kf0C6lwTu%}?$ef3ABzX)mW z%nQ|`r|-N5QYAcy`)DJJbsMC|dHc~Wa8n@U@2NG6zG|1m-sD-6;dP@fY^v~-PY>$) z{fVC2v)}&kqIqu?T$fzXj!ZzxH|bV=Z_)2rfGAzq(JEH{Fo;NSB3NA5o`&b~Gxgj5 zmzV?0hw_fQUTY2q@LwIoRFY{HI8Cj}BGAIDIvCkAVo|Us@Wk7VppUPonvVHn?l$)w zl!^r1n-TZgVIkqpj~}dZaw4wIh~9IHd1t=UnAnuUPC%ZqUQ;R%Q^ZPyCm~AwQV3#p z3lPFs-!Ej)C(m%D{FN+}nq~Ox@Z(lwn}CwG^ktx>fc1Yk!Ibb9ifEC9N=;J2uNX3T ztf8X|XquX-UcM{zsq>MYP)I*|Sau}P`tk*KA=GEmEB{pPJq@TMyj5Z<^vVay&m|3q z#~io#%hh1@MqF3&gC4r-Fkq}*nxX<DtX{K{=O}azooA70vrurueuQmlOH+}A>Uvu= zKUyQ>vVX+VCeKnHHEU+*(yh4fBatvI#qh9U`GLURz2AMST^JMkwkTk8BxP)Nsc~wK zH{k&bSrhl@)miNKNE+xtUReoCYcAUzw(!=#7psuZ>cUp)8ap0cja`;tdqr(w#zbI< z_l)JRuFFTZHpO|klyied#IeP4A9yuAa9^4GS-ZcjC35lU5F=^1il2*@V0^EXo+bv_ zhfFuWNOznF##{(8)ukzz#9!iXmHp&JU+S{Px$$PNbVk^fS(3}Moi(YPfMGmL?54q| zE<GpWrOmopHJB-|MyRYgVQ1=58Ud(9X^@Ij^%Dr;XKFEO*gQE1lFe=-X{@v<H9^#T z&ut`+hN~aoIxVi9-l3Sl)u#A69UX7~k`jx_*$1AHcAWRlO9%!xgA2S@Jl#LLi*n;< zQa}ewq&&Eg4kjJhz-MgtziSjRuZREx_IW$Lr!VZD(ueTwxgx>K#;x{>hV-;dAm7}L z&XC`14o^>hdJF7HoMh)h^BG?KGsRGJ!$|ZuB|__DAwUf5yBMkS2}@?5;dvzk*diJE z`XmN>NR_f6Z3UX!M=SYf%MrG!7QcPcDxAxH=y=_wal2OVCX}lIxCgL^rRQ{N*c)60 zgI)nDS1NYb>T5ugSf#=D$C_eprSi=-_4-LTgW}ZmTc1ks=7jiGD5X`0)MWAR4%71_ zmCF=NpbRaS;+&&*MDVpfSC3xgVy~P#>XhF~3OsqVo{MN_Q6i4CYWx0RRq+b4roaN2 z4?h(*WECe1!}+_u1%Z<|KJm9nx8NyWvUt=xH6<Cnl?9s6f$6Hu23a&jdni+m)&FFJ z0-}U(W#6DW-31m(gBpiZNm%LFKqvC&Zw$B<K{th-FB84Z9Uunr9{7vp{pCe9P%8cu zw+Gx~Wbj__vsb>^#JWp%$(dG!(t?7(8^2O0S?ut}rPpS>W{<!wafzt@!Nw`>lx#su zvtt(|p?L=E)A~K-3W@FDt%)hIm64uTV-I*iNXe%nOn&DjVja%zL8N8P`|o|$9Sm&g z;igz?#jnK{>7>5bm5U;bx#4>%_-*&#bUWyr3^qo<|Id2PE}ELzEPL{WE(4PuJ6!|^ zkmDTF{j*{a_H*~ZK1T76a0+B+<y+sBR|g9TFDu0+nKE+-@#l}UzJZu^07HrS0eMmL zze<$gKw>1dvTjl1UCi-LJU@(&UaaU@UaOU?veAbFGu}r4NbnDE7Rl;o6#0_L>Tpu? zGmJ@&*O(k@>wxR9U3p*j1N-|=R#)gY?l3E^p|pE%%6g_le6ki`#OzHaH>J&{qzd=A zW-#WkDb(Ni(Cykqe}TN_spJzk#{i-%-^Fm_!4+TZ#AaqA-S*;CXNksi@qdpb4I1HB zG{zw|BHx<6Fd2g_s;8$wl@#|qPA5!B5>7tA?#od|x77l#WiolM!)0dAhhjgLqZW1C zny4-+mD4%=EA?M5MSU=Gu7Ps8IFP(sU4)IP(1<Kmp*p5~#V8bOIo9&(d-vsz+lo|r zJFuqZQT>`gE@q5{MDuDApH26=18@_(v|(jJpCa-2UW<d3A}hRv<UVpIc^`b*$Jg@I z`*Tsbdw)*QO|$C*BdXP<_&R;=q0dUFJ_O5`7@SVE^)vTL_?u}>8tHHAJ25_kaGzD- zkK-Q+aLG=rURG7$A%B+-+`@4!K9Owp3ui+$rm0kD@K%#}2O4!WOM3UYT(|N!utWUX zz2N<D@6t$US@8u|3@_`;TQRe2`tF~DOPdiHvl5(8tH%W|Udp6wn<MY@S8bsyCx?H^ z+-;dtt6x*TFKy#_;}-jF@4g*9f3rE%1ftdVkrdWfqgM60)8)&c8gC#Zz!jxEvU5#J zHfonoq?Rhs?cKBx4(4|MO7Wo##T8ee*FsZn6r9I)ZYN1v`UYUK%poU<&3S*Mh7Sdh z;W+1h9Pz7-R(G=<3OON!3Cah8C|1yCBafE_@K~KLLV1Yq^{3BoVyF)qcyvhd!aSO= zFu)!EVE7-|W89vn26G3auU+UTooo)!*p0GX6hU+giOVt|B;;oY-3p5K>?3g~B!3^| zF=Yb`(LyM>R40LSnG!g<k&n{w+{W)}W>68#{RIXxr23~7451c~I4MEqOVx2B2)k(i zxjNxEQsj81sGH^BqnJ>6cYKBa4`MFM`(LHza=zQz!G-&G0Ky&^(OnjV%u?N<46NO% z^w<R&CKkDE+amY?Y~s)F@_t!dYt)T<hI9a9@ItJ4jPYs8h&4!+L(-NC4Bzk4VqV~U z%w%ZeKb)<^Jz?gq5R)!6Tx;fKz~(>eDxR;&KIL52oWTg)U1(m+OLr^W4H&(fdYOIM z>TfJE>sw<6p=0y!<rBv>?pl={4*RI~7%ab+*g+9?G$K7GBM%vclB2%3SC5ZRTF}(V zAE#C^6sN@|q%~audodG(MtJPmBf5;tTx-gu60v^eHx?*S^(44lulYegM~bWW)|AZ` zgbPQ@`l%%UM&`P1#OWD}jXd)@^Wm<Ep09_ZN9vo>C4R|Mz0>Ykyzp!K`jzXg9W1^I ze|bj_(sP%W0+EUq569+8%w&?_NrLFDuhHFfL9&~gQ?L7HQu0B{6i?$rdZreTKD<;n zxyxHS3_MS5>4c+GuDtzz{8ppkaBVOgNXyzv>J|zszpv#0Nnmqj?Zvvf(x|)z!}RDc zs)V;#SNd{O*BhUnkq!ZwyAoYQ@o)7rhS};J(!F!rkZJdckQzLU+G>`KQS!HKakV)v zY|MKg%7C)!!cXs!x6Ot7+6j=Zf0p~fRQA8>p=lM$L?z35`(=sM^R-2TyL!rnV8*ih zr+QYEU*BloWZ%xF0YSKdiWZn+*35zy?eh@;*@+S}$g)T0o@X{9D_NQ4WcO33ML%up zE+=nhU6&d4!B^E}&t;@MaIbK^w*P6PWevoQn^N*wVBK%0X9Yj;e2-yeIae^h;S$-$ z`{Mt&2w>#cMOwP6>6zKlLa~1|L@FS5|5qRDx@eymmE%j0?1^|#Uyd=?9TreZ-6!#K zis+nL?RnxqoFN=B+(C?=0q7~BI8x>AoMXZLmGG1i{+aNu>rX4IyX?RJWukcAB~E=f zHzvkEjsT^xF{<q2NS*gAwq7;R-0mbBC->>1w{H|QF@g82dbgp|z*sU~NW&q1@$gaJ z0|lWR?m^$unjXHEyt3{H#B;4U4#giT>ETcuqSzwtf`-IsloAdfV}+%-)1$GFi*_<i zj_>QDlQ&Bd-IkI&eDLC4%hIWiAjqD8;tjIwhcJt^W-vP=JQi|!(c_Hmm~^IY>;%~& zEsQ4-x2?dAG;x<d=z@Q~pnl1BUhuw!!$GSrPRZ=g(O{quoP!h#hw-pX&eUYR*;6Qs zQ@O}kwYvr%R&D5hLlb!A)9EB>i?df!z>|J1zPpXi4AznGf2Vv<A_sP&FrY6j-{gS} z)Zq|-6Vf(TXS{~)RlRtdcnTreHLo;Q+>G4Hu6p~u^@}K>i?}f>W^ALqX10BiJR6T| z81oWh2K{|!!D6x&J)@6Df`)vyb;<(^>7Wy|Tqi7-jW+kIG5aJv9hb+fj~2fy>l0qY zxPymlQxV^^Z<rr2q*jPkr#&0!D+e1U-YCkyp1_!xZm{@J?b$c+-3}m$*NH=%<WN}O z#TQ0~)Q8hCS{Z=9^}Niix9VUzt!M$Wy;H-nc(9}6dpW{B#eoW87J`E6DBvA7Qe;$C zxJGr)4G{W_kHD1g6&ub;S>1`?s(9=KjNH5&{{e+8g9Mphn<oWT6bc#|)sXhwzhQXV z5nb&(oJR3N40-pZi6b8#t#m<hGVf#gYN#~bonDQ*%<qO{co8t{h0%3C2U&FsBH7*Q zbMz(*j4W<0G;G(V1;jT+8bHe$cOKWDuBGQHzBib29SOa&yY1V<D{kE$QRw-%C;bao zZUnpc$XNy+F>jT3&<~|f0eZwQm)F!Eju6t_QK5s<v%-Z!!5|@ShQ?)j<2vcuC<bnN zV0fr%z@6?=c@^8-XTVN|sdgYq-)<~(Z89(nwNA05-?SNb1hHMTQF?^sR{DnTcT)YZ zuXC;4Cnew?87l6K<fyG)Qr)g25)b>}mpQIxY_d*FbN-X&;{IF4Wm}UL0U}#siyJ`D zRDDA`!dk3Ze|!f(52E_TTI5}5!grfFe-b-<(L$?TbTmTn*t0y;{YctAuevR-AI&+? zfJ}zK&FEUFq*tdG8e=l);-Xf`2zRMa*$G0_?anZ-aPOzngt%xp(DZgJ2*NmQYg#*W zcgKR5lHUAvPiu^y0zQ>W59ejB-wOe|ks`HIs*^s{_<ogL4)!P)trHs_zaP31E%_Ah zWNkIYr8i=XUb19&AvsEB#+3&Pyq7Oj{a&{u8JfsewZlwEkloXRr*drOr1!H0{|8q( zhH?OLmhIMXjkYEQ+jG?1v8=&eowpo}w6poD9^aHwFx1$!>ZArU^xC%Gq8;%1!Xp{E zxaABkz!+TX@Dosm=G9#m5(KU%JKYC5gX<AF<iBu<MEGq9$5sr(v<r8pcCz)Hi$A9Y z6a!2{<J=LoU5}q*BtvO?5QNeOc6`@v_P=@IeUT9+mIRPkANCAYjWkerD$ANz<^2R- zTLH4&R<X&Z|BG7LHCkB>TEoY}x${={^tRk7-Y}XN%8zrsJ~43n=d%rE_-YR9@w-^p zny8O<t3nnk0L<~s((LS0RK-3IbSTt`3t`#<!w|MgWLI{=5fH^cp=C20U8Q8(ESH3p zJhl0>N5`QFw-|@Mw;U_`>vf<V>;=fWzC86=$TE6~1LkaK%1=)4dob=e*Uw6&eV>y3 zghLb9m{gWO#8xV-ifeKpm&EcKoilTY-08cvma?Lhwfi?qXrXiy>#?qn#6|6Di!r6h zviBx8eGa~Jo^S>RnDf3HIT1nbNz_Fe?o6T7*~y?6&X$iBZ(&H8aH_D<Cf1nA)WPCC zPXJkX?>uNa?B%IYb~6gCM7dj2q4&>RBlLq8y@OZMnd$NUieZggKg90lKcP$!uGm*Q zHw*QVML9FXNia3oh-JT<p7pULYwP|)w{7Bk3Dh(`=XDDHWIRsn;(0;mz=`IJANMHb zuE|P1s#x4XW9NH??Uu=T&q>hCxk%k-XN+B(kD_Qv<gLq}QxnZ!Uj+SuY0$5ta)(m| zoa^RbW=l#(gv`XFJ_?@PEp606O5Y2&J*TY}ToR}f`nFf>;crCw)Ewann4andREiaE zOw;}>Rh{IkA(DrZd>NF-lFCm4uFUH}oUgVk%VPast}$kS@4`6>n&g6rzEfV4=0$Uu z%P<<NmO&YJs*<N68=tv@sMxnFV|=8Q_B(Hk6zJWXcS%7NQS@G%O}4cEJ^V0EX}AG2 zle3_rGq<{V@1GL>$6Hrh&c@5E2-|g0h9#!5*8A^o+#@ygd-7x2M*$EZj4wYA8I+{F zUf80QE=?z`ZYf*3YFhX9WEXkQw2MiIk=tj^Q!63GCkDH1VxLJNVpjWScIv^d2XCgl zAJjmOJYP%UXe<rWAD#cfFQSr`^YNHK|0g3$x^(Z|g)dHrz?WH<B)NvQ9gSEBk}Iw| zn()$O48b_G%<8=Pj|Iq$D{L_K8aG({`@A^~6rnop9r#>MraIE+JPwpJb+Ja;3Fq0` zysjDNj)5e@XvDoUZjWiZJ%Ze>e!S@++r~_uCOiIhH*M-;SJP^d(iUUu#x^Pchg-0p zPxh`1k$C2mX#u#b@7%~^*8ZH*EMH=Ax#}&ruq96dX$5q8ALFZ|MBaPQWL5HKv+A~G zRk6Y3DX}9szgjyK5f~KT+^RMcvKfCO?3Z$O8;J9l=lXudH#<=&cf)5m3VGFUbgBLc zsdIV>g!Nw9+xI}cz>yKw3_v=4B-Zk7Dj?HqA{2-s7mzGbX$yY1%<{t_xy>d4A$FsK z=er(V7>cXBXe;xps|OlgKo+Ji(Jg&_Yp~?jfZabgJz`TW6n8x{%Mr*=zs_i;MXt*l zfDZ0uuiLot1{M~FG$lMPWorE2YmKIdRD$uda;^S55!TNB$O<U$(Ct<2NUS)MuJWVO zMlTnR2XzX^AILn{S{vb8b5oiX3h|i4J!t6#($ms*>x(EQ5)=A+@T06}3VYpDmW_<v z!v4M<PBE^fygNvXn6vXkW>vh1Av!vf3A0-2BftACmb=iOgG`rYB>6i>-v7VhWU>j3 zL~cmjYOQ&_Y!dP0)QJ{^{Lz6Ipv?JcM6@cZVYo`>%plV*;j>*1F`tS!4j_Uaa}^Io zI{YQpmvgbEc(&}Ms8F>e^;PiKj+HrWa315p7U@0Vk=wl=$;uCW4>@@zIiNUi9>OPh zPRG3H^l96HF|nbQHRkdZjDwmjO;K{2(XSlVj}D!#QE{8Kk=`_u40d#RR|{+VwlZr> zizKo-cbpplLKVWep}1F@{tPkYqudG&%l+NuR9{CcdZV3OaO~yel%-)tx3>T+-vMvT zUNsu8zt0(rxa{6|^X9mZ@?B4}2CW-xn*zowx$$W;_dN;fcrSYX7UqQlFqT*~!7Rv9 z_`f;gJO4f(kO%TeLrFO|3!QjoJ9f7ze%m?tPMtF>@&q`P?(mR>GOelOs0*iK=~_B6 z6_(}MI|^Qepe`rYXx(C%5*~fm?71jC#l}P#2gMI&-iESb#Clm%9hRn!kaxO#o`jU& zjagm2fvQn!4E8v^zh*RftAKMj#oe>zNkAl<$L_&r?`*zpzN1N0x$013;vDX*DfxO8 zk`R|MHFz2nRR-WgA3&0EmzX+KNDPBq+0w#w<%!q$r$IxsJgh)UYDhK^Y56l*00N|B zD=is{k0Np@@&cEk?8bz?$WYxJSlj?_dA!l?w~e}~{1EKgU^*p$i3-PZ$X;IvBDl6X z{b&w2dAl>;IB6hRk}5?y90+8?dsW%z#ZGG%$Jp>iJU~%J$w0)he}0P}Zr54ocF@CK zhXVLQovx#i*K>?S0<F-oTY1|RgL$hY6yR9k27GJ})+b8ReNvCNXN?NarGK<@jBEqZ zk{E~odXD((0XF+=0|Rpy&m&AzOvoFmqov!cjTifWevMoqyaQVjm$^NN)An+UIty=r zB3VRKp>wtHqByWf5j6)uiy0_oF|VOgw_s0FjFEg?=Nv${lK^=g!l)>?nJ8IpYM*)K z1Q<D(PXSg*NZyY*bzedOuiC9v^cZ82s}ewv8k`dWAxxMdcc4C*E?H|MK%)g@8C89F zXYMeZ@78o}!Yt@CP2FUOCO!(J(3ls0lg#{h>W(ycBCyytwgQ7Kj63JlmZji-q<jec zib)gMhEesQw2sIxc$Hb}Y~<!s>droEn?$<cyrAg9Kbr?xxbJlQ>W9?rhI?s*1o*A0 z7Wa7c1bczL*0{}vkk4?kp5IK>Bn?TAqOmlE2A7mmqQFahhrqHmVa&TXOz@UJiLFaI zHABCr9${a9bo(R<x0u}6aiO-32i)A|gBRt;tGraa2*1N?_<lV^+a2xr$O65V_oh82 zM~Y+7YF&6Rt!nZs5d+$I4Hmx;vN;F!CYy3Ms)ybrUAJ!Xy?F`kvfV!#w7W}n^D^5> zMwC1Lv#yD(1PW+3yL(r8p@aVjW()lLI%{3B28f7Q3_%!^!@dPycYF<E6iFfbuz!r- zWumH%KsYbSOnMuJAYN<I_oUG5t{G~J3VQ3gAddUTv(Z6<P3hESUx`mh0otkmy<C=0 zTKKN6?fpV0|GudwQ)2czp7S~^&Ra}J?Q|S+b{9-XqZG~nLsT9O^*cp+?oZJqn<oFu zW>|Svw6=qfQsH-7W-T}-HolUqOp)wO!M_{}L^)UNW^@#YGHSuZrzx!Vj1SJ6b-CL4 zGHay&sNA((p33nKV$1$wO&U~l?9+yR6y)Lp@Ag^;8~#~(C5_-@B@dM}%VFYfEh?B3 zULK?fa)kG8D<9u8R}(j(RgC8p#=+?9lnExpUEprGr!X#4ItT{Gr3X|H@9we+kSUFo zeO8jJU4c&wsW}E)JeOfnL+({nXV(a(GgzDqvU7WA1SJ`)tL<ZN$-Nfa8a^7ObghAA z$vl@fIi0*sHmUtfYfb(pBn^&6CsyQ}`;;@#&*rwQPQl%mXZn%0<L%1d)F7>B(~w83 z{I@xU75nG2kxT0AK2T9<8T^!ux*KI_%Vwl7iX);z<FJcVk#HO*5aiKJuH5+66!Lyp zb5s7Pcf~Y#WkGDksAw2Mp!^+fe96*6SVMr-Z7%vt6gM_4$ta_E0!m7Bm*#3->tw~p zZH}g(Go>+yLAG;we{?!;y*_W+u1fM&JWR<mK2L45U?Eu`9B<va>`kF@RJ>Dfb7edE z&}6zn9kI$8w*xH;Q!+ajI~kfeCN^f&@i*z6aCTh3#B+Hd-Wy%zK^w+Vyupz3$ne31 zN%|5s!UTBSuQ3|AadUvw@XrE}A$No#I?+z#Y|p-HynIauf4P7C2M9RTXWxDYA+O7) zds1q?cm|0N9XB$4Za{MIcf|Oey1XeKSc|RqXbnlTM89dbe~Nhfxyn9(WJPYLzQe>3 z>4c?vv_lqKaJH2wp!dZYltCb1F6mth8j5vgER!5u;`fdfjIC2oo@jJo<w~0>4#eEg z24yg<uhU)E=!S9G!3$SiWgK3~UHJ41lcgy9VVv{91h#pohXepS$mP=6%hy5}7VjBJ z{r;t`)J$p~lH)HJwoWyN;FOyXz(M;(X?Epynt^4Sacr^rFUqr{@NIhe5kH_I-i@RY zYbxJZU=O}{P^@ULHNR_#|D*Sa2vn?sTuH9BBb&j!_>*q-^0SSi$rB~V>vMr<PLKB$ zXB8T)Gf|IycEG&~NqviNB=&-G@5Fl;jy?_GeYTx>_^pZ`Q%^2Rsyb*Q^NgRHT=Ss! z-^bLb`A{>HM_P+~+e{XLR!AFSSrMqMv*#)LXaFW-pcvmP8pvZQZyj1k2C(uZ2_i5* z>i|Qpr)^5DjiO=c*XF3q@r&^m3!!47SDSlsN40N*P59IV9fVpZm6Q&-GTv-FQQ-t` z<mx}Vz%S=`ZBFdqUN>K@%5T2nqa@FDHos^5@UC0)ks%)N<#Gj<?vRuUT}T381dxec zFU=@U>NH4_<)=C3EY#1??J`zHBN-)AKDdN3LJQ~X<)<vEBQih^DIjRvAxl(;<BWir zqS1f&yA;b_Xb&Z4z~ikNy>kmgNZB0qk0*v<FG<>>NsWIR8YYObW^_xyNG5>plF>oS zY^?b{=6fd{w^CiRl$(Ra+CVdSmibx$z)rLl4Yv8*SEG+egUFQzJ8d!B1t;<`XXN`? zMIwOiZ|1-+1z3FQLncd6w)D4qVNTi=i9^(vZ?B*4XjBQ@|G&EVFO#q7LNeKaGu?!5 zLSUy{28K#*qEM;YfEe-wJ$f71d9iX>HJuWzE6L|_Zn}3^VA0oKpH|sBG=^nwnDRqH zY=v5LlY+rQCmFK9;8DTZ-FR;Hblb$Jp;c>dOTJYAH`5PESV5?e&zvfIW2HMKR4Koi z??YtSM9&e2k%iPtQxzTC@X`bCF~g!p2F3WUSMj^wwu_Nr#NCf7vnfIts}{CHiFrTN zSGL8AR>Dl5n`i>XtdkI^U$9|D?HrZyLVDyFQn0ic{d}8-!FESGPU$K)Y~3=qiXrT0 zt66E&la^nx+Xp5ZtA@9JNu5YlAh!L?Fby|J6V$|7M*0~0FO!?oyMmz?X1JFdSwcR+ z$&JGwk5~5iy8aQqY=A1eltj<_tS+bSng==+g&g3~#x*O>7?h6OWuW4_>G>3;n(g{B zJ;9U)Vh#zlnqmcB$c|`ynO0BRWljAoW0YT-6;Hg{rAAd7KYiQV@(#oSWA|C<02Y62 zrw&XSBx}bW+{-sR+Uu%f`8xbwVdpaONcWSFjYQ(706oiie#lc)G5@hFVy4mJW6~|3 z0!F0N0{9|<(49KGr+6uf>y`-Lv^Z;PdEm0OOJg>WtrU!UI@Rs_+WHKtPxrTa#cC}v z-neiH)#>RqOpsya==OZT(-S?V*eb|U4+X;&nN^pe*cK6hWtPb=aUV$4>f0#G4!)3e z0iXhlBAe9-AyE`tP{rj19j}TyP4QF%aG=*<ei~ST5sKo{5BhOL7%(ic%C)>06`1^s zZ36L=pYZh!NDt(au-n2t1Rzs|LBQ0EMC~nYs2B=QT07$;wEHL<e(?OfIaO)B_xG<H zf@ouFzFwaG(QK)CV_G<`EF*ZWsbdua{0WB}vE>du{7G+bA7A(mD|qaDQBeuD?y~qp zNm}5A3^#oQLIGq0tpzX%C{zUAWM;nQ!Ix*zHPj*d|3`{atU6;iXYNG-yFLI-Yxs~} zd)8g)5vSb&c(>`0eye8E!oVlHVeQtG@A;8)P|NX*_@8kjuVqdzri_Iu`@rCTOWT-U zIRK$fRu{#!DGVvFcE3!0#m8>=T%9`uHYBTa9>`!-itpgQ0116TtlLy51((!VHE-9# zq+3=#zQF#-jW0yy4+N;>Jb$Yn!X!6b;IG|&%Cr8}$}Hs?6pHAG>UHKX37E@P%xAVk z-1f>ZoW5UWH=IlsOo@ds<pRfvuDqhvjpdx^To8;hjz9CHaGHOee`6ad&dim3=Q)qG zA!lk`_Ql-+;u&Op0at6aYBMli>Q$)m+cXn(_S&yLXfNO=Yl1xR6a^wY%I0k-k1a=+ zO?s++r65c+B7>BXz_W6@kX4Uwg`<&~gm)R=bnxbzAk4)LJ=ah^e_~KgZD?&dl_<O_ z@?R9zE{qLNa13c+ZFur@pdiHbZb&+57&FCw&U;;D6bI-UT9_KE`9Z;l2o8~u@rGi} zO9M3Kn16B+coKe6>#WU)PM%CDqAz3(vQMXws27<sb3OurE2wH-pPkj(;eCZk1*zS< zA$C)00r%zMg1t9RTi*xFPfz?%-%_xi)@Fs3x&K9B4*V=0B>|=3zNNUdUk&RYhrL8| zKBqRnl87cvfYD3lsuDxWj$aAOM3Da$4y?CQ-p=u`r08S}8E}pCU$zTX-}kdP9}-kj zqS8!~k%bqNl&@G@Eg-tR4@Qt%!<5{gc2)u&la~Pe1SIdi@2D4}8I8OLav!5BHffu3 z(QPlxb4Tf=7HpU_;qdsnVl7BVRN)nuCh_~(kW}7l`4n#OUJ?K9vzx*wHCZ?h+3GbB z`K><oDPW^8;<m7In6rzsf%MhGj~AuH<unAqxYBff@5~lDzQ60_iVO3oM!rSUBkZ@Q zbb}P#ZOSt;lw=PR>gZO}8AG-L03(Bl?jhy0tu^57I5nwqb9CL_cu{(iMx9p57T4vv zg%aXMORu9&3xARj)*8&Pkb;=+GteGNU+t<L9qx(dAW`}CkLIM2sHWkQ=}PXB7F{&M zk$dl{1jZ4^JKQ+qB`M9ilC~0)n}Cuq>)dg+eKQEtbfYQ>Fl{_kEY3bsZP-T%9~zy# zEQ%7!s>z;(Vt?H}P^Twy0Q=4^_fC*$(xs`-&x?U%D#HnV8_K{vTWS9ZBv~T@z|3pl zkl_N5t%b_CX&2(ndu3jgBs;z)rkjsCBeo$a5q@B0_hN>o;Wl1pZqrn~J{@3-C(ypc zCUyBFwb|xaLO-~xuq=)G4FH;IQ3TBa_NpeH%Ca&IJ4{h&VU%j%c=1iV46KjIonqx) zmM<x=BghorP6fh`8(Fr|%WYwcOa13O(QQDeLJ3<N2nD=(BXUE4uu2C16A|={tjIaw z^XXDd`FI7|P*A?HRNEr3CQqrMcqD)Xlvmau9{uKUP{zonHNekicW_v#Ousnuv6s_` zd-xT)u&km|)5}=<1#^!!3XAgJKT&~z3}W%8_q7mD{NAER`_+5@JecWxf5ZAcU$Vm< zN1s1)w-<Ki^>Un$l~+-c(Xryu(WIr^G7?jPZVzqT(mh!s8}~JDXg$YekO7W{;+w-2 zuYJ**lMM8of`c`Od&|R0QQ4XBO1&L4S3ogoOfVqA^H`3PX)m|#_$s}^@MW;#z(2eX zecOJ>!j2hVU_aOZY%~=WX&+8?v^`9MeIJvr8<npHeVek>+PUWk7kBx);$sH1LQ^eL zKF{YLj%Nfq^PO7=sa<P;nY;=nPFLFZe^9vw;|rM+UpG(q%JR57i&1_d&rBx7aN9|+ zF^@F+iQ(^6<>D)EOQfD(2HCf@n!WXh;jwMu)~w)|YF4tZyWEPbLoMr0?l0!dmL50V zE<KtNVpWx%;urjw_~Zs9@5m!c)w~Mi%Ys%Z%AlWA>%a6Hna5Lf%Yco-_cPa(<A)Xw z{!haqJi#&&(&NA*UoTQNrxE{zq+USWh`OiiXxLS|_QeOzRbwVBWUPaF?*juAl7HW0 zf%+1PSzXV5Z~~oS0p3v{!T4;NgthA7NO#zF3Jo`|eHKnkM5ZqXEU;{+lDX$suU14| z)()M|v07kLXPQ_Eibb*{hA~H1iuF<dMOV~SA@19sFz=;L3egUXlQjJ7jcL!V<L`gT zy$xDP44=|{V?n{fP-iB3^@cHh${NQyJty{<q-V~sU{8YcTbJ7MMBht*i)f;T(((*N zA|N2XwgFM426}O4hKj?~k9*%Jeqg1CLlH>80h1{4rWJg~o{bxif%xOGkV3&9vU5W! zq`w(N{@DO_377NFL@E6fPoMRLoJjfIV^yUa<g8XWGlF!0@%2BNdY9)(-~7aXg8?up z=<eqZ;UtV+t8)rQ<U;lQ7MWuFDor#YwM7q(#wGH<-uL@!i8uq<V*15&ZASo5%f8qH zd`&#58bYi1NxE)l*h79(k+1sllk~6B%YTawmGY|hSa;|!#*`CE9QGQVufqN1w6b&E z@4sE4xeIVwD>q77;F5d$=CY5*TBPis-PE<x`g{qIRU@2g-cSdK2+x6&tMy7Rn!nLj zc$zPFD#~=)I>&Mp&euVw>xZielt%K-%a82jk?s{JZ2yUgPCCx~_-n0c^Xg+xf(u); z?7=5Hmkk6o>Bvs@$qMJ>0x#dmw-x!1*}*A?6A?aw*>(G-E+AXfho@cXhpgXpK&O)} zr6j3)mCASJ(_kP&%cBnCW88^sL_z3}y8}1KtOF}oLadt3zFkT6{{7-YBZ-@kD4x(A zbT@|d3Uz}7xn}p9B}UfY*Fwto=nAQpoXUV?iOlfH@9`od=6CW=WB2iYPX&$}paJ8- zwLk)C1dkiZ1@D)}pVm>bU4_bpP3=W+L7QN5@oR56MT%l3k72+;)^=d81x(U&Ok5RP zoJS}^`#gbfb;lFqNM`h}5Vz&-wrvJ^P+%0T=W*Q-kZtz^{;Xi_U-@De`zHb*e$ZAO zCdgto1AfqZA=va`D*Z2Le;f!Dg|J0$G!oZdYUXBg2-K**i$fl(Nk)$ezsoPq#EHN2 zt$)A25zdGOy1iF)75;Yv&VJ_?Ss53%2LNNt0I#-<#XC_9QBpEfJdc~DXB=_`*@7@! z3fRYE6)aPT_&Q(*=in{7m1bBb*=91VDexq6Qxy5lZJu4xi77+eN;zq(aTg^w=RNCs z)WR)8ugImu<rcikjPL)Y-2u28yb0gcqNuTR`P$sZNos47n2GLl09*gpFmY_a93|@< zKQQ~~BKD?t$XKrW*l7+i^5j0lLw@*xWo<jUnku_$=exceLrG-nY>gP*VgUV?-faBr z+x{@-<{Ia)ScS9cj|Ai{X=T5H*u~4qr2~Gi9m%T#-KxzS<@`~CS5>>UbVG(SsJnGN z=p6+^ega`6Z3EDzBHilO^zo}C_MHQIf!V+R;8<UGJh#^`;~V?or1>-Ub!5W#IOf`) z?{xg$qDkX(-;)6ZMd;_}&#vN@cCS4{3mF)RF3<J)F-q1c85q{C<3@kw0s+G$TM>D} z_=9?Q`TxpDObG>}YB$ZL)wKp+gB38S_CA#)Gj==-EE)i8Z6FyfMae&J%8e>b9s?;B z`~$ljay$LH#Z&ebyjZ^L&J+-^A}13MaP5tAK?1OQGP_v3CTF883YQe62U_}SL%E&$ zu@U+PEg|J6bcG9|G<dpu7_-e^tl#us6{{4+7QK$eL>(0U)i;;`14S>B_``?voL^Tk zM^P8?H^U#<K+S|e2N!8%@GVk?K#=5Ql3<)uj^>yd7tb1x+LlK(!&<q%SaQ_54!>)c zehRTtNd%q&`!Yj;aAJ@`^N?<nxTlDXC>r8ZXCefj9-b~mE5%&OK2si%OfmaJ0~UY% z(r1)++^19ma@dtpr%O0!TQ+qtRuwCN)?1Axa{O~#G4<2PZ<O?n=MO47s_9`0>gQ~{ zi7;E*XS0qV4pFh@H|<VxtjgA*MR1tbrxP2jUi5CgdCmSESH>6<)RkX8k~9zC1CY;x z!_Ka_*#}(~X^dvOHUV-C++?CP=RHRTjSCGn4_WFe2s0JWWw@(<Y@_RfW17XUpi>$g zCL9OP?BkUBEtJ$TujG3b{#kz_QyKt3r?&6nxHpu1wwO}&N;xYstTX#_nk}DU*vMT> zxYZ;ZZLwmZqZ|DyT7wx$=cbCAj(-5Nv|4tZ6oE43yxLP3qWvIC2s7$mFkR*uXig9< zX>oLquK0N1H(sM%*BQ5a7{+JcKOAF7_)r4f>RDFgJ6Va1J0PGa&0I)U*5G)9$3=b5 zXI+hMe9|Q0#_6}`e(ofUZ1A*AK0A74AC~Ls`0@oe_%+R5OjwW8dx=*viJ9NeYmPe< zlf)Bz@UO<vsaj9*Y<746!nz9N?X|@QN%q(N>1OC}x_tojMrH+X1;xlEFfBMPv6rTl z0T1@vFSLWH<+f{BU#5!Z6e>lq0`g&#zdEYI)p+Gg_m*?%zvSv);mKn!MPSpIq^QWS z^k?p3Az>$61);Q4Gfa0co3`a2LYLL4=Q6gyJ}Y#&tZ`^|azSnjG^y8RB1Eq4sfJCX zXuWYg(|fBz+@+H$KLd~nip^8%CshE22-CJ3bRt(zxy<WvzBs05U6zX|4JF~6L~{qz z8k@}eY*(Hii?4Uyu<rGmEFts#ZBpDm2dMFU0pZoSCqj~&Kg5@K3Y^aVb1bn}*>5#` zC;nC3Vi;<+1ogP7+iH8=H$U@Az-R+A30vlJjVnzV9f$F~-7g>REEcA_mws$fVRnha z<cl^6$7V~A?IA*i%9@~jMp*wwK&XRsT_iE8&^To=tH)fa0g^K^&Q!MqCEs!97Yl~n z?jxUN#W-@})4v{gx|SCG8YFGey@T2@nayuSgI^<!M)b9kp^z*U*}cjgg>t&#7T|%< zYgLg=jwS<&mMA41iwtP`=&9f2T)g4CMXckds9&r7hmc|tRr=<sr3A}udZUagyT`53 zPfJSrh@Cyt9tWR;sE}<9UrnW)iw>^a#<<!jJAaT)mh97*NQ-Wh4`Kx4QYvTM!==GW zVXy*0Z_$X@TD}p*8Zj0IFDM$m784m_!50FCRId>^W4)vQUn_`V6(KDDbT{uGa5j*r zzA?(QG`}g!@2UfGrS6QXbW|=)pqJf-6<JBwY(p29Ic9UTqr<55K;5@(fH%p&Po;d3 zjjT-qn6TQHJ*$`xl|;;sH6;N>4OS+A9xfqfY!c?0GMxztM0_U8MU)t6_a#9TT*vp0 z*{rCa@ZY8(Aia_`v(hQ_f9dV5G8!|B*@Tj;uzkB0<4BW$-4Hr@Rm%_&(;IUlMQLgt zWytsLX_Q}$_&fRP?ZS7zKVc|g##p_rbndU{?3f6I+BvfI>Nc;p+YQ!t_OfFy&l>4% z&>>VGJ636$SmowKijJ+>6FHYbWmAjrXjTiq@g+6INgX2-DWGz}dA_Z^f*0<Bw;wJe z*L1ZkM}t?QQZYL8H_b#-O>#ep?V7+Cr=Q-DtII7q_-4n|FCTIiVVmIJJLmd7_;*0u zjwJ4vJfe5I-{iV2u#!-yu^G**DaYk=7ZF>bk2yVyDY5kBwtLrx#%I8<_6V9!IFwtM z*rwEu4AhKR+@^?MPJ+meeVeWtd0_bE5o=n9=c(e82lh4+RwZ|z*RE~C2Tm$#UN%nj z&)qvh^e)akaInE7DfPHX%s=#eaX3QS>FCu3=${PwebbI+r4VXR(>EirVkcQS7kV~B zb@2|zF>?%K2)i<l=sTKR(bJEs?HA8+JzJH2eZ=1L46`@%nx9*yx~an(;B1FQ1AYa7 zz(K9CC2qS>*jc2GHAuQ@z!Ew6a@xIKvCQP`1F)89|BM}}*yz0Na2kIn*<GaJkZrD} zBi_sWaF_B0$YisN0uN?VUI-2b72D!Suwh@<lz#vYBGc3-s&XvvqGEZ49-5v#BgqS7 zAzvT+j|Es>+7*Go?AN9+ATn7%aJTr8i@t5HNu;}^@@yOEw{p}GT)hw6{|?oG$ebfj zGyRHK3$d~;+q0~NzGTuk_bdoRYy?1}95F*JZ!o8vH#zXnal+Jj9CM^!k|pQkwyJ!E ztHF2a{k<khn{0%_F`IDJfTY*{gT*LrD|K}P`-R@W*O4>~j4)EiU95oC0k{NW;mhlQ z`xk?io*S_UePF-v{C%EwR1m8<`Cpn;&(QvRJGn1xeI(~Xqp#*pt{9M&Z`Is1tOx=| z2~8(*?v+=!e&#d0*%-SzQ|n4o^?HyLnDmW*vLDKDjP{E?1TufFWH7Y1!P_^%9*{Q- z1vR*#WaWud5MXtHX8aM=KNtYmH5j&;s|9wG4+vOaPPz7u6m!D|S+{WkX!eIxGuB>9 z%tjG3JI$<sJ`W}M`R{e^G!5L<pt23&a>KWRFz@XdX7RU$*yY5gY=GBHI#}UKI{eLR z$!CFyxo0ExZG>J+x&v1trkLQYrxCF8pjf_y#5XV~SxourbjU12c1O0z$m1i;%5R`f zIQ?Ra$nhsa+`Fw9xsaDplGn=LagKBfx~IAuq(nS}g=#w?!f_#4)KoVusYZdUDnE^L zrtFIL#i@4rz;!*qICC>9K&`tlX*JAwA%t*&uReSZuNdh0a3@DIU0Se8#`oRqOR{Pf zJBUC*abNMi_5VhRn>}o_24)j~Qb>)2mU2jAbK~*B<oNx|<bE=bzd8N27oh3uFITH# zqAAvSoa)@>1;sRk{*^Oqv9vZri*HLFMHB2Y+<m*&z&aq2c(0MS-px>zYK{b9R-0w! zS9*dTEIMsTJe_&vfu*}?4S`x)R^I^5OKvy%T_(pl?9FhE`3C?B1L`0%Z?GQFasrFM zhsb4CW<b*Xa6CR&Y_v)&$1di-ObfufgoVG9ye<wbmA${}uOn*E>r8(ja{ngu-+mG- zwyk2DEdr&_&of;41WIRkmw@9zmXf<IwfPp_W|zCG9hJiOPgw1H6$QmqvRPA*4Sjj( zgwv|V?9|kXTuRv~2`)kSKwO$@P3wIohcDCW@>y=n%69<sI-oeZ5lvzu#V~6n>`+wK zKwoV}-2F658h)z$Y_M%lAM-t|p)})@AQ^&lgViffo{<dg7Z82z>VcEXdL>1C$>6f+ zxm4w+q0&ACd8PA83XdA?Qj&k~abia?<ddSr{AosxpHt|NCkHEFLd-Xpv6V;9AmPbm z+_(WX*5bV08aN$_m{kNKu^nwACHo)}9pZ`9u5r%KxSV0ts)wdhL{!Od&gS-J<iyEM zXM=ZZHiQ4w)7%IKQyRsO$9617+DuA%7tY&kX7k(U?HrjWp+sO$*kd&5xXlh6HBiuL z%c{w5_O?T{XLC48VxVwJ_n)|J6MX+IRWTe@@A^j-IeT-MlGNLnFoe{IPY)NEUXx3^ zxA|d^Xh{FE8sX-wj3xA(|72?>FxRVW-OOD$z{N4R-+o{!E%XG^K!7KF41l)Uqjs{Y zd&@RE1xZ$32jQitd9s82@1sw5MPm0V$k`w~#V-gPRFe7rUtdfB>-%0Gl@X*X&Q{c< z5&J2I2?DPV6SOKzD$ce)+h<(`9C*)6n}p2%x<dDsxi9M%f7!ZMKd>N`@6f0*Z+NpG zUp@!zZv(EjF*I&f(h%Q7GJoj;D2A+a%+~Z?SD#P^+A@&h&ePZsfT)tB&bUnslWuh? z@e;Qdq4fsRsMfv3j_@>_Mj!(#CvEr&1~{!^<!Leb6ApN^|C7>lpP8$r3K#gaeBp}a z^u}bxdY4@F4h6tO{3M)Gj<370gRYSEUfAxGtRbd5`q#SsWTc|XWyvDxBnejO@|3|v zojDkCMHGuF4in~ORV43=Q*J!=zsL%}_5e;o{|uu@+IV2Y`^`6};fL9z;dfpJu6$58 zMEG>yET`nUy)(ni*d74%zC~HnKpk7u4#umU!@UN~lNLVITrW!W0hSio+1<}ihv{H% zi%Cnbf6W{9TUBjz#2QVPT_g93<|FVhN|E1WKWu`{-7b`dfn*G?T2Qx5N;T#CdA=PU zu8<m&@h-#&>|Et|YWW@1`R60M5NPm(J-WGXiJHfm-r=hdcf&R#K-+u$vU+w^bqxr& zNGh*$Fel&u?F2O?m+zMcQgf$-b8FlVrxTAu6Lm;}y><Xykl#v&_;ENQ6Y^CU4~_ly zqP@jf=Mnk8m8rP6GWG<*<45~YD|E(=cCGu{3o*K_Z5V&{gDDvciEQK<6g2nu(e>lH zvR2<+yWGe*Qusv*Glm6fpS&c4q!8wjy!t{}vbiVfBAZ5KjkM`Ws8zu@ET)hrjg&jr z9XLU7EX!;aQyD1OYBGi|K)G-1r;nbn1<0PUQY1*%?2gd$)w`RGLKiKwPIO&{+X#vH z-uD5D?&7;5O24IAkJ!qlV*ae$a;fq^ee>UfBH*Y&0gjqL^{|E+T-v8RN)8X@aH$~A z2*gkBg1>?qmz~S6Gmz|w{}B$)=Aj|!!P;k&4C)i1sHUZI7Op3pvTtl;jW@$(bxCY0 z>nIfI4X&v*Mky<^wkE>9q@lm?xz%%&s*OqTkfK+;c`~18G2z)}oj?bS@M2joKsDG& zO{6NAGiQh_n+2wjDfHC{QYv~+V<b=PTs#>SuPj0pzMYGRDSP}b3w`B41cMe#yNpW3 zL{l{$_`JGcnm+~SBZsZ&>E7|8R}KDDZ)Q0Ub*%}U3jYaX*gV@Vyx8`CAD-b;DrNM^ zxq=&YCHp@EvKa0AHrLWXr>)f+GtI6Vc3P}GZl422vu|CRoo|k|+{YWAy$~|{w+yid zkM1N>4~!XZxZP|7^rw|4M<tnQ^+Jow$4MM~VlfysS^Y-0y<v`I=`{HteibTIm54zd z%}W+p$C7y9oOEcKlv3^P%U%MpO1)65M2rBP)@a;W3)_8IYZu(Vy))zExD=_hJX?h( zqFo~{yZdj+%pK!TsqKEkhdu^)4NUqvp7d5U-S{<+BO7eE#x93`Ye!+bZ|4Y8B+fX1 zB%@RtyvK25mPcxV40J`}f+2|-lYdRxDm?Td0$AkZPxaju@q^pW<o>tn-x__WZqJtZ ztaifpB^hL?4dgX$4X3a1wtY;-Ujb?oH&qd`XZc|guo}?FZI!D9rt}r&QUxvmN-RO0 zWas%?uZOZq{F9X;m_bprH%XmuyyF+tvEeg}$S5&@)r4(Tt|oi*-?VSlH8!8TF85@> zT-4TeOaA<)`d<3s5G=Z7)~%7mFs~FWd92FZpwR8>zVd9{tMi(L-l9r6@H=M%f~PcP z+8igrxl&$xvc;+Jlk<~~;ou9Hy&a@DY@i5E2*Ik+%aQ{Z`ly><0*i^0+;C7PhqA|i zi$H%?Io<2#vC)ut9Ro%F9Mr|xriPt^-_e4%S^e(`Ga;T;sY$S&Of7t;jO<FQcDd?( z1GC9e85Df;9RZ<q!N%x&VxH@Uq?l0U#5dLSwOLO9-r7z4H;zHh!mlrm_XZ0h<Sp&g zO=_k;2K;!i5M|dAz8xqga`kB0j0WQTLtio9sbVa1i6Pj%LWLebbi&@3Y6dK}i@X`= z_<At$gX4Y@NV9P5?0iMu!APggxTfcsN1{sBq`40#k;Iapw-i{X>;N+UFI};VQp~`C zYV%6-xWGLPk|!1O#WIV{bx~?%;d5e-+SK<}+1OpmLp5p&qly_T$*XnBzKU|an6jr; z?Y39_;z_N%6`lZVwJtEyvMi1MDNw-&+VPT;o_la0)6TrQur`J$(!J6*nb8vNsC6u4 zXYxiETj){sWBBq3I^aA1;4~>dkhWBccTa#nn4mSQOc-ov%PS(`;dx>OE#_PpF1avc zB4&{0<Ped}|HL=}<fx~he1=(IT|uBFQ^b^h#pKulPJiCb$=A<%*le)t1X+=tb^^7$ z_2E`y^-z|N9i4~#;{4>-ep{36M^0b=3;q*XjC(}>?}_v|m&jGLz%KCrEkm-eUKTsC zPJYN1Kza^fqcePeT3cMWTQb{iAh>_<<CAwH>5ruO;jzok3`e8ZE`ABwM4TLUD~tt3 zuMfL4UV7WVSXn2X<!2yWXmc~hnj2pj$L%y+<mic)^}as`Oa?D&8X5&3In#IHlsgMX za=}H>ZxJ=94xB2Zi3Ww;=pZTU3O-~rK~lQ5Hxg07z(u1UX!?mvH*k28y3d_u9lRl2 zwC-qoVxWmAkr}m5*2pXk%t};yuv@@=^}^|pt~N!CgtxgwoEt&1oras;H5Gz29F%pR zpg8gz`zLAX?(U3^*LXWG4UMO4t_n#gq|v}LZd2BUzBrjm<1~}eB`QOW*F;m)W`_)r z4}FFl0i?i4SoWTsXlLt7XVgc@o*M1tU?Go<ejk)PQ*;zC<g)~Ze5;9)-~O-@<`_Tm z;i?L^<Zu+X!12Zls|2av5ZBe?L|?u@*yIZgN@z_eNU7Q$F{qGbufw8Q$`Dsl<EL{B zD>28*Y0f>nS-cI0n7Q6Rtgtn8%oy>pPxk#OKWlSAJUx_RknVnP;2Jl0y62n=>M39p zq&zJ)_|x-2ASopV#0DcgD={^B2+sih85_*WsFm<2{IIzEn2I*Z8Z^Ud2`BlRFvW5y z+~NYvz%KY>K(!KjNoct&(w5V}Qx{PIL;axzK_t?z7-;zZ9fVO+=|WSrT7B<dwo!6t z$bk(AT)<by)jpf<#T|Gtd9PsR%8;DE)y>^Zv6hJD<C*?~O<?_xhDyYs4K1K->pFDn zfAMt)tf{E9ftvpDI#1oaGa9~FREG5^<jP#nQ2<2arJN6c=r%tlUr;>yA1q+>RAm~F zn(9m49{T*ZlH9u*gH!}g-?<~WEys6&lV4>$tDL*FvsRkLK`zCLv-#TDeH<yQaZ!gC zbuH&5dI&Oa3}@+YjHOY<#CW((Nya13+gRcsV0REMv30R&X*5icxyo+%SliqOW?`xV zg_`ZjZ>8%oJrHOWj{`nHmn6RL&;Xsiim~~RKHtyFw_|TgosQUqbATRuwp9Ee>$$a_ z%02YQ@cAG~uq@Sk9zK;GHW>jcJ0MRYD?KhD2?Ulm)c`3X|BP&>DPshD|GQCpLezrC z>y5)<JRe8E2HlLX#d$`ADFj=_Y+Cz_pD@wd@PDH_PUdhCL8*6Fw;tsCOCZEV|LFs_ z9oI;p+*00(g_w+ybY54MR`-~;x>lp?=K;Is<x*f+RS7b%?mRW@e8K|-I<Ufq+yE+U zEyjiY8A)$6=_@^wnY_37G=HK#6+<{LD;7MpxSDsxA^ov8w9pMgd_~CEs4F<+LJt?n z4;EeK^<c(O{hUfC-)RQz;HcO8tiB^H>T$OW>UK;l_jFXzPqvi%6#suV2&9>^*`P52 z4Er^C6u7fwdd?OIq4*0w9kTUAniWUom_vT;3P;(LP}ZMfl({(w4&j0=k5ILRuc-bX z!rn6+&VOzHogs`)61_{;N%S&=pO7E~(HV?rqf68oC0e2<(W3X>yI>GeqW3a-ucHQ| z?>lS#*WUX%_Op-m(&6PC-|N20d45h%I7`uLaHPju0n)lItSS|EH=us9-I45yJSD#M zzLf}S^~*Euhm!j+4L{~=p+aV=#qCMc6Cfc_#rr-^d`-TD$!~r|$F{KeA1oe+uwe9Z z1M|k0b+R}z{rFes{MuN9k{D=|UqLkjOfX?Myq3nywm@G}Ud(?xiY8V5s6&{|gU69d zhEOMGi*L31oaNZ%xmav>1bXH+zDd@AdC@o3Dnza_@PNM{f~$ums@D<w26yN3wZK(B zk@ZvzsUx&EPBOa2v<nyqgA+r;d98g-e58#GouiK_ld7<yPFM%KnyMjU!gILq7^y;` z)2KVHZ=0$wh*`gzBEu_aIuwIcMlkNasopkt5^GN3Ry@lgUM!)cXEtUCzcPTo1{k1s za;t`VEmc`bHkm3FcF-?ge0OtdQrQ$;cPVg~&(do!lE~?YaRyuTowirH&h~w7GQ>^R z?hSXS+V6CsGCSrBMHq7R6C1|4u|QLkU^*o4Jx?{A*|`KHxgI}khCczXwAbTR;N=}Y zA_5v4<rn4J#*a+CFC@|vZGTnJmaRT9MWvWW0qY^nizT~*d2Ra7Z7Sn6eA0Eva0w?_ za*ihm0=4WNV6#%7)h0jrH2dkHX)p_+?(e$YuWuF0w=6*^NB|WBZk@PS3T#4J#eq9K z0N{LFWi!S_#5PjLPtHWxne1k)5%Vr?$Lswz;Y70gy((i*YlE3ppsbL!KQCs>QTdK% zBpoB=(0~=01oY{M7R_k<)X;Y8CvKy24QFTTi(3Zv`JNSEqSZtu?jq1~iU(l?n*iuh z8cz^up#=;z<Fp?-^12Wa7*j$iwJ+-qEWryse{K!v5-^sY0mRU_tCL#C2~D*zX*U4L z-Uj$>vcJjtM(y<_Z59!RA2fF`B0y+^6#%&44-lUUy&F*{ewE%wOwW*EEVu%-5qEfJ zM6s6tiBTLNdtx4{%q`w3<e2vcLb1vKw2W70Y0Cn4vH5_Pa}&Gos74d*h0?9(On@oY z6uw{nq1dU9<=p5Mth>5xvoSjEG0s`Vmi&12$G!K84R3h#Y~%4|Ehn2=*$#ZJ5Ot=G zC>l&4&0wW$=LVOtT^}j?vcLAHn9lD3Q3tGVDI!|SN{BTdu=kvw+l)t=hGHY%zaILl zBdeMBzBu=~gCu{u^|r_xrsfT<nLVSA7e8t!$dz>XAtH6sawgp3y7lfD_IJOnIZvw_ zSNp}4X!dAI84zd+{$3}V;C_fM=hw?-@`=Mt6o~@VeZ#v%eP<6H^EDf6b@bWI{qD%} zEoWA>e5?lTw%tzaE{<q4$u@3$(bhfUUOXhBy2o!ZQH2OO!AgE8l<gw&E2%Fs@(!Tl zuCT1hEtu6Nrpow5%JD0jbb8rxBKo8rw3aapUCk!gC`8L63i@*<S*O#la(W3{wwf@L zt00>%<=dgs6wTe13C?qwJmG?9ntj#pmEmJS=r)O~KO@}o`+HT4h~A3%|JPb*ly*@2 z**J7!^t`Sku3d`O5<uR;!T-`}vi+i%?p%7f?r1|n4G(k{TbVO0Bww6TOv%UZ)+lgr z27R`rmek!YcwkBJX_dpXV1WUUY5oC617SSS8rHmiKgBm4^#7zKQ~qIQ&3S~2vo-OH z<Ym8~iulWi5%TU0))1xmXE@!FoNNt`>J2M-K)inIB!Lv7!)29?6CaxCSC%zLJAW{F zU*@-N6}rWDo&2)4rfF1$R&*w)B&i{75e}mm`6VB+3)R&}I8>#Q{sh#SPxqW*R#&w8 z@Z#-S#4cio&(8NqH4#zYe}}pJq;++o8#~>W6)~%|?^J*mTzFd3>9R^0V$|28XebDP z7$#9V(1^{&quLh`@s!>PHq@Wnf8n@;06(`5kTEajE``ak$!8iU(WQhI37Y1g7y-N( zgJu*z1@XVK9>~uaml$QLa;w+KIOR!tzaKGhQ!6G6r~@j5noqNleLC}&vC(ID|DSHi zl3aPn-z;M9OEvlne@J;tuEI)W^w}N#(kjoRD+<($iQNgL6M`WT$~L@SKc-(?bd6jH zS&gn+e5#(BFrb(2UW<vk@GJfB(6_M;ugkAHzx;tQM3CdsvN%IA*<f)!1N7o!iYEI% zVCfgwfBs&!nZc%o=DogtT&VObB)lW7pWqgoOEM7M0&zcEjavkqdx5~!%W$;<@P<)( zA0^JW08uAzj-)0?Z;GqV{vO1LeC7ZVc!3j&(yrq7BVV7HDIkx)^!vNme2*|4%2Pk{ zYG)rQAE+U6M0I~Dl>bpr`GkE#>p{aV3Cn~*X$~c6<D+H3+4?fygKPpX9tfTa1kA-n z1zX1w+eP^I6(a)vnpIK@HCo(4P+zmPeZulJUyEn^1>h5s`2wk=x#^?M_M>a*dWp7p zSAe3NLq6yq)Cd~og*#^Tl1(kH`M4_SR`OZiRW(_n8m~zWxGA>oz3Bi}^o?t7nTk4T z3-ja*c#+|Z1V*)UjOugm;@&1j{gM4taqgVlcl)d221BRhsi8Y?Biz80-zO8>e4tyS znag#SNtnlSlK^4$O$;J0^fHVHrim)QQ?bYR{_1$pWv?l-ZA{vv_j@O^3)x?C``atH z!iG;6J-8TpGb7?-PVp+=xaXp*A6D|2@}J|?>&b+)1H6F``4vp4N!cPGv?N7QXkTpJ zB}=<;?aVchW@o=4x)l%?%*Jd#s=dn40#ALY3)t?av_7Y>0?0Evy(Hz%V5+>ND{XS! zmt%%5X|wmhQyBCMDH)0g>20<CqJv>&e!nWUWa+%G_7PB!K-$gf{)OK$l@#4dcykI# zVd)Q~{+QL4l3cN1nsDcQ`3yI=UZ$<S$KYIX18Cu9^Oavzy|mSi3Y*9$;%L=Ooi6Rh zSE5k5$}hR48#57S2#yArwybd%vv$a|6)o_|eDTa~%E=lWLy#xoO#Gm*_bpUQKW$1s zt=?~&^qCaKHrbG5+?H1zud{DlINRhu_qH-$9`;s4X6>K#q=+&2)AY-0k_jK=mz5=? z9>`ourL_2()2_Rfnj45iI5^Q}aHe>l%e)Zl^9=W|m>JLF;LzW5@7d#R8G$+^5!Sc3 ztm|d%fJ9s2;?KX8FW|=)K?>dkGgAq>e$wZr^oe-?VoFYM2k&&><Bvb#dHXdna<Qdh zJowI!D9BMYRlYK1wCY_SzIkhktaaJQr<U0rk)_Thv~n?HVh^bUPK8CXTpPDa3HYMU z5giYh&bid|efc<jp12yOMJ2UOvZB)Td^Fyl_8Am6t3!3>fMR6smP*DLuUHe~`s#os zRTHqcA^0$Al5SQ~f9CyIwg=RP8I$h$*MvK;<r{HlQ35OC@_F6e-r3aO(*7cVq#y@% z_KEw2kvcP*DD8y!^)f&G(3b4-2VG}WJQT}BfAu@DZ1_go_ieKSKJYc;H95%po5ZBm za&+UK(+Er}zLxu)K9aYMy_~3-8`n!sD*Ka7e!lwl^>l~R3kJQZq6rK2g%~+KaH&I3 z1Ryhx<M@N%%c4M$m~Q@I9(yJe`>ZN8QP>H9T?rD!@7)#JP7+z)M^%l}N=4=rF@+!C zK}O{(*B4#Cc)bocGCC;gOXghu-IR0uPzH+9ynJ*#y2X2n<lMICpG3~qcz1-YQuUk) zJ>4xn*qW9^xKKOrU%>!+(S6?}U|-+zgu0ybExmYS$koDtcG^RTuWDfAOKl39by>YT zpb=RW^HXM`o@~f~tY9(@Wg9GGAp3~ik?HrRXC0BjmaIO^F93AR-CPxLQEW&Id{fSG zA3$vSk7*keqaX9R-NT&%_xYAjx9=~+Sf&Q*yol6(d%uHH8t}>7-vI`^eV?1huhgw< z7xz1-9h!|ixVHhtQkt>k-`Nb;_Z-rCUWritY@!@GzwbeDH8L<>za3(Vl%(-@EWh{j z^WzjEqNSxk6E5w!E2r)8<Ac~cUQ#Cq^K!0L+;UyI)=V5DmZ>P(as5y~(P6rXSLNRx ztCjd}IW!&9{!OEyVJRyu_CCD3fd;sl#cFJmd8$lS2UC{MFa9RI(BO{0D6;wip29uf zn8w>(#fIZ?(ETUGvT3?@$lPRCyiu^7cRP(v3!p<)mk!<#{pnC(VqT9Jubuz-+A<0J ztr_=IoE1~zb-MdjK!Odj{v2TVr748cE-vDWyYwfC`#Pv30_ZOlz;yjLIgq=!>W(cy zNFKaf$P>C9eLphbnb&c}K<doQOToyitX|l}%<|P=jnt81!A=5lGxTFLU21@F*+StR zEMw|)4AN3>FJG;@Ili`dy96j=wJs-I)fMxOXRD4VyD_*_mETcpNnk!+1toM`wo^(- zD=H&Eqa2@<niH(1_fj{e48g{ACo1{)(dCC<{*He(<&*FH+rwDR-(<D#Ta<P2FNh4r zR%3b*FbzPO1cD_?u61BU7g&krH45+yt-%3sZ)ZG828PJ4=sU`MR`DItF*V|8%wG$; z;(s}5ENn#~34Ybv*c%`xdQ*9Ulge(IMk9d*c*5`rw;#X2Q5=2@t)sqg%kzSomojm( zrVIlAcfk@f@NIrVc{5vVP+RfiV2PP3d-p&81C{K(cMO*TIxVK6FQhJGaOcX)-5-~` z-7xU?PJ)iMK1#9z|5}Y4)qYcN53#pi^vtWh;)BV~pV&H?feF2Z7{a1snFHyBlH9ts zrd)OQvMyX(^5u(fBVmKfFq^ce@`B=}sCX$?5sUDOm{kE$hS|S5s8~P;RV>8^OG5%A z^l6jcPFc=;NyuC^G`cJpxHqg-kOut2AOv-+9%EK=s~!nk@3zcG1unq*X5vmxkJk&6 zRDqa1$VE#imm^{&>fB&w15kZB-CxZ;yW{V^Iuqqo819oI|B^Pe$-N}|M1Q(|ErD>( zc`B^4%AzR@%$|?#srGAkczTok_dwO_xMj31yNC6Xvz_ab_8)gcOMh`KPCG$f8yJgm zbk^dvBbz2ByRsc#6DZHYxYSoehID5-cGq!{@?IG4x$h!`yFYI1agHmxxJ#LDi`xwI zEvJ>};JtMu#ND1TG^Q_6bb!wOLo>`KKij1J?R3jwVKHb{be<xLDUW-f+79b|`(uXQ z>h0o^!>Q<c)<n1W^VL2?zpaO!SNcRP^XHWdBjON!BRjQ12UOF_h^2>Bd*Q9W85vMd ze>wXX?#)Pbb6czzN#~~;t!C9pO8rL*a2_m!IH4?YAMfX=pQr+`e%tRNitC!JO7zU0 zTBc$d4G=D@4M=Zm8Muve$|L)g6mp-}pS|oA?fOZv;<%=N^N^W|#&oXK8}2N+a4u^y z;Y~ak%wtIP7t$pd#1mKtP2Umg&cEiuI)Klfa^D_5>l3z6$u;!lgRo~#IzD#>Ze*g* zEPuKIUC&?oN!1JW7#KlI=ZD#`lfW^P8)*bBa;D;;$(Zrwp#M#^?bID@UDLpB9~O!C zARx1-NlZOR{d?oR>v%CIGXgZLs6BI-*VTvkX9!xJhV!km*B2*cbDrC83yAJ1VE~qv zl`ulP&oI3QFU^#R;f5A8kV|(xhyOeibvr;|`rfS>o9<{NPFK*)(eM^*y2={=Z^A{> z%F|u5H0XqQu6{$6(W|*^C^d~0z`H@nQq_aqd5=&n4z1LDod78w_ruQM)V-Qz_WzKW z%cgm2XYTyXneU$NONwi=*jWC{uTKG_5hPZ-KXw6nYB%!wuj<v!>o0~O&b|&D3v*Eg z^`>LEYdwqq_<gwknuM;NrvuJ!&kKFJdtRB(j|LyN6LZE)%X=EEL67~PK1Sm*%m{~B zt|CzSwo#SWt2=MId;sAqxj&@sm{P{@tnViodyhOMFH-7L#`ilmN^j)yCq2l_SMhMZ zY6TC9ZpWr@Qbn;`9iqzSIq1X`*`z&%iawu>eR%nVHuWViYyv@K6}LTDP=AIThZN|* zE7}{`5r4~&k)KG329Izo(v=9@onH5q9Z5kSxtz7N`c3ezoc4fS&kUktAQt5O%=I-V z2Cv!|!N;S*udA2V!3E*w9aQ0=!uukgw0d`9%)6V%m)`<hRL;{9;HIb`9Ns!uelY~@ zhWuyk#QqyE;e)Jek+pvrf3wx>WOS2E+-}dcAp)<-ye<0yoAP~wuj$<{{!_2i^-VZx zTb9(1;|(8xZS-b6INaV<;Ls#phYo(?F&UfeZr!aKT48M6HO#ML?RX)Y!jN^N7oGlP z0AYj<clBjrboPb2*PMlN0r${>{4b*GVVcTRQul0<x=F<%>2Ue$SNTs?C6L8|-|JId z0Mnj3_3^s5TYKn<V?8CbLl6n6L=EzFNOkz-WTb7&N=Ce-se{M#c0q8=Z-Jz-OuwhV z`dip3aoI&Y(&c<fP{s8Kd|GnJQTXA<c_{ef(;(uxu@2wS`LCiL{SWEs03G@ZrFhUP z<!^SlMoq02%JmFNs8M$)Tq?nm=wkJZ?iL&jH%yZ_Db(YK1JmDL)i$m4<Kr=H{q>_y zJ)T49>u-(A{3remXnsp)_{Bxw!|W-Rpvolzb95Q%WS8t!!KV7RIa`b`w<*Pt7laAh z?PP--%hA;c>4`o9$Fq+7q{^GqW>bJV`!}NnFz#LvLsS1S^rk|}mJXHMy1DuAd#a5| z^!bMHyyHdL&==Pb02acGbH=E|%djno249n*Q$;DSuh*Iv=D+2%-du}9%GBno0X&m7 ziRR@JU9+#M=xmj91eCfqRys4OEuroRs{`LQ;E5GdhjpEn*Of&$_(cLDtC*V&xAJP` z5@}8-<<xz`mA~K_(PDnR{ok8k{mf)5y)~NaJ*qb7#InwZ&pjj`p8gm|V!Z<|hK@7% zRC_N@n8#=Co;Nam^WU+U<vt%zjBael=3a#{7(6TTUU4eb6J5bQo$uV)MeH#FSJ`fu zepj=k<e!!MX?uEc?Xf{C7rbl1+9xaaAIMkf+WXG>!oSj<qFOh{q#$w#8(e(LcWPfj z2y|^fkpQRgSZU!2q+3JQ*SPtph7+VumP7(x6t8(4n0G@xW#rv5-@f#OE@7?z30_CW zwU{~(SzGPv-CH{I8s|`Wh1xIv!1`NkQ0~0~PW`-J!%ryNv*6D&3<>z3X!*igm0RK- zY!?~sKYa|$yMF5#LZHxV=$ik_Zzj7yex7lhAs!rxWB+k`@~{0chDpM8he+o7AQEBx zkAU@^%l$o{wsp<ipPeJQZPS{$+P?52SC$4s3T7fN`*H&0<yzwYq~somr%8zUM>}m9 zSAS|e&`bi{`*5sb_u1v-<bFBcW#m}~*cSnAHbUH5or|JG)dTn7m*4MQ_D@Iy<is;c zMZk#|QeA^K-56BBlYKO?6n7#H5$lV(SDF<n_e*x1@NE4zqtZ7_*Q=kHt?Eh3=e|3C zuND9zFvbNVgIn+~11LK|ns~ux2S6DLS3d?swxmFS8GKCaKjL`dW(P7Nz5BV@&dx#} zArK8S&RfK}lU)#wOgnwI{DbV{S3^CbF9-u{Q>L}06Q_;<Cr&T3M#xAOZ5?pL1UHYg z6-xb@$|`vR;pZ)z)MuyBr2yON!*e2vu`poC{YC!1L;w0q)tm4$70qrwaDT^e<tL9J z-3nf=E0G-j#HTOlXl|lC|D7%Puh=eyDZEh4G5DX%fJmp(X#7#WSLH*Kz_l%1*B<6A zHs(yve4!*gm`uX)rQ`<4eyPKi9c_)aY=yGaU<M6ehO{W`JR!VyW<s$g?^PDZ7E+^a z_d@)kZyV+6G79$a9kGBDNSj;oBo#lhJKl-Kd|c~O2JU(NOG^?$7}=5kj#fGC^gh(g z)zMEBc{$l?>rFpPz=l|>pbQB6owD3un0kc#HWC+Rx+|%=`?V~*u$i8H(Lb`9>OG0? zcFzRPYg@`SyfmNw^+Efz0W1yR8?1>!9|QC~-&_p~X3d{<g3Ze6$<B&JECW*|A_?MZ z*KU2NzQ#D0RE(8#F=AJv5!do%@S2zIUM9N!#>^uQv+`4`f-M}&dwo2m+MhU?ba31@ zp&DjTY|9Bs%5gwk)|<SX7i06raTo#~uzf&P_Y=sj)8l+med(;+>fK;RCf(rXdva4t z!c+O4Tesq^vM!+2ix>6rCXl;6X{g^nb-TfjoBJbmxuICtChwLC0?52{7MBhby-CNs zDrZPWmGkvrf$PkH#WOdkKjmYT(K~w{sVcyGFQ5B5IStnA4KPobMy26>k|k%kpBwH0 z=H#WWYRFe{ffX-R!+=9YTfm`A?^536{pCn~%{F{b?!4gy4-Ey<ebDyVYfN8{Z>A*g z3P^@>?WIm5en?&r5{D86y?y>P?Ev3N_Vysr{d)}bFJ^S1$+v$#2@&85i|EcHy0?_p zb&~$6`%{{KMq1jE9O%9Qk%?{vq{7Y3vwq%XSM_18+Uuqdug9ieJ~lcX3f}dauiYBG zNs@S5tJw_9_^gLtguKrOANAoYx-;cdWmf^AdBY_B9V^DawR^l3^!ecnde-yV;dTHg zo^|k_Q%QiLj3u3}f|mU{6~*sM`LY%LRbFG9sfy#zeo~Z-u!|8sJLQR99T|JDRQ<ux z1CTg@95?=x&wih~LG%wnw8x_#P10>Dq4O_WzR;4zV0cLJ0Ko%?V*%01>S;$<s&>oh zdDzzDDK<t;5@b87b5XkRdRRAPN%WR>!-*W@!Urv{%UQDqhh2S;wuCE!X5?whsg|Qu z%||DO1)<Yt*Q}Y+;b791Pgm`0JiqsLo7BnfV2uqM>E}KG+9s=Zl2sCeX0k$>S}m@% z3&Y2-2Ar9g_sx~)c5(8znOB}{*S5znn@b6=JT@aT>QzeeJf=e>65IfMp0QTwwBFt2 zgt|R-U?9Oz`8drhViH0*2tBauNqPM4iW^?4<lL`wbLz8L;o#F?aeZP47N_3D3~aiA zj!h{k6wE9$Ew@ZmDUkP+X2dPo<BNn-=0Z<(O{|6?5ne+X3I=~XG1j@Q>(JBsGU4|s z?$x*D(Ay;q3%$<<W-?NB_L?8hoK<>>G^gsPPgMVNmFpK(JP}MY?X0rP5-R<*87|~} zT8HMGDG5hFEue}*%ZHn^#M73bdcF;HZ=$4k1uKY3JUzc6S#5UiD4H%3bApQh@1HD+ zV2b6XF5IH{t>}a2E|*|Me^i%!nbw+5Wxr3uY|$SRysm_{WCf_2cv6CMnk1oIM(`%n z^)SA>)mRgiEc8k>cf93e1|5N;*&<wD_|65%TgvQ{ef%kkfN*f$AjO0+(Jde{w2J#g zR+#16DwQ{p>jX1juG0d@-cpU|SIhQ{-}_5(LTC;IV|~M+Brg%RyOS<!uB1jBFbWdB zbIGtpiVba2O9&P+HrhehM(vd0BIX93>vN~EKkwn(cY0!vJZocY=F>^Px?HXFE9JjK zluqE6Ek+V<l(4-aQ#^<4wNKGy^KC`#R!#UmP-yw`;cCqV_H6HL#_6E+cQty_WqZuv zwI{dKajvHmUEiEA%{`ZsUs`Kf`UXqE12mxnJQL?9yc`P^j&1VL6@iTL8hX&{5m(#G zq2rIIBU4kvxU>O8Nj>2{l2)!0hk3UpCU(q5;pZc^CM>)`e|}-|9&ta+t~2#Qfw=t6 zA@)>?SdsMuyUHw_L{`3SZTtKp-D6WOj}JyzJ0r=)hh7{Q#LN)&=zM5B-}Ox04HP)+ z{r+sRsIvbf%5sOYxzojk>6A6$auG9R_39#fFqh`f>(A}!Kn52)VvC9tJe*WKyKS{r zwI-=G<Su`ZWj$|}Yp^YU^~!FTIq#}u#h_vTY7pHf%D0Vft#!3pZ_av4#1%KLWasnM zV*Dz0wAnTF=kd}(VK_^{?V8Wc2Fn(CEuB_AwiGYVE+R(4y<uf*{B}haAdm4Pun!q) zfsbsq+W7kgzaMzcBuEugYT&0_`}f25VA7`QC6WHY!v$6xM_a$hp-Q8X%iPx`np-ZE zB}(EtPtx%TXOLm39dl(4)pC*$delfM+$P2*%vF3?gy`9{Ic7PAB+Em2h2U)>!V}po zq3?Q)tatRU>gM6V`;Us~XvqFo1sIUWHE2u1dKWKZA)u@k9!vb<lW-_``13(~QyseA zANz9dyY=fr^WVRgZdy468^|0nh%M~5ZJ%_kNLPI<krw0I{$ZefY)vd1cgS_Br-F`M zqMZ-5R{qA1nv3`iLnrBH#Q+A%wQP|*-gvJ;!qA{{(0c~K&cheQc!%%e#$J2}oBKWi zF|=n6KEnHo8^&w7^MDaQ;)iZFjpcn=wJ?7`V5OSW1VH*AjcdZC#OqErK1b1>?Pf?$ z(75H_1Ekbp7w`GThLv9@YhBS?kXQD`KVA#Iwq5<j#2kqC%Y$V?B6d>4?yjh1A9$<# zF1L+=(t3u{GwsZNkh4*5`nhPlqsDKY|MhwQkH;^WfCtTszhTkDi$36)*qN#X-!wEe z;oODTyH7+qsJ$Shvg52B$5ScJVlnc9vG}vpa#dOt&XA7>?1H5N*T;#aLEnQ}=-z*m z><Wcg-%fHVy?1fOp+6UN;wU^ZeOx*gabZBprVP3pm{$#r0?A<~->j=QS}#)N%3zA6 znX&jpLwvAyD@GOOr0f<^V>*d=Rn&Tw8x}5wSg#PG2&XjT&doV*RFV(3-}UAUszzs) zulq+HFp{USw|44FKH!yiy1%(TSBW?2o_ebk*!FnKB>Qk+&6XqAG4%975mVtYHB28t znwQ#-kUV`?QM>5I!oKR^OBY5nrc}=X0vnJx&Am*Q)+HI}P4&@RjEdEf5LWYBs<4-k zGCCTQNySwfz`WH=biR6OBNP(ed`}WUrRi`J9RN&zIXAb{N0nSLB)wwq)!aQ+J#`yO zm^{5MY*MR5L|el)y;>|S*hueUxYehKxI^`YgynBG%0a4qi83YH>DFg=tc=mOyq}#a z;KuF`{7j_8-0jD8H<dXmtkd?rfB(LPJ9m%35~yTl1G`>rbp+TlU~hev#kP;8U2ztD zgMxF#(Y8a4eB5_=Lb$i->&Q7kVxwr`HTN4qDw8+YI^uIKUw|i1f-L&p#(066*E2hn z#_YXHM`>F1ord?d%c{JlMBN;;OYSKj3HS{f5}!q!^I9~mTp><4(?9&W^KDekDpc6s z9rnk8k^|d4<6vS?E*{WfpwgK02P-uEi_4fB++}t1E)GTG?Ku{S=p#Rwr*R=LS1?z^ z@1iCl1{=;$P(<r;JhA1^!Q%;b1p@g7TrDeUZiQ~%wTs#fwF|8Q7SmU8c;KLx{K5FF ztx{SdE`kKN@|M<-s6KZX11Nr5M!|=1fDw4^N9~VOfu6&TtWhWW#?UKvM*hk^`BbC7 zqF=JWZ^UqfaDtC8$yFZ+pd9L-O~p&g)lxk<W*aV*Z2$m6tw4$i3839eHp>ogHXo2a z@pTwTA@?#%X}dwJXE|Ii#<R%U_%)X|JuOhXFl(WUD_;gK=<&jlrQc(P6}ux71Zj+x zSuOlr1B^L6;s%()Sd@1c5wvrWCjo28(GF%;kBZ{{upJyk1$j9){y29~I>PQ8S%DZj zUvdoQa2|Iz>u<OSaF|#1fjsBjm&5<AJU*vLZU~Po#2WbE+1c5d4roYYatb-jkw4aq zy|52!Zhr0c2iLqQ1+|<|mE`<DRG;Yeb8iO;Rm%04wu`_x-i}d(N6xj`@#94cd)A>f z{q~}57QZG}HTgmrWJ^fn!R9^x^iEj(Zy)#n@%sI<eJQy*noWO9L}(vRhI4PuXAt&q z$mh+rf}(hXogQ)m@*8WWJ7js%gDj*R^%pbG%_sn2cCPcM(QwZ9V&7RavT2x&2qp%v zCMjjeZ0qCZ2EeZ_d;NTeIpC?Kz(why@8B^@fka;?e=7vzo}(T}1C1y~^1L0%0C^GR zEWnp#AJjPbrJW(inRg@eleo^LPS_>wdi0&*J00^dq7$8v*1EbeJQ67%UC@FqXd+x8 zp>sT#NU6Suu3;Ft!X8i)V!Jb0-=y7~={jmb(x0K4p!?b{GIV^Bif294^xV7m2Ry%O z_xx)C^hf#ZpH_rOayys4e?Bc^Luie|cz06108Og@c-_#-`XDR03}x{%#=e=h7wT<; zVp;U$?RFb3*8F!d1<LiGPAb*bS*xEl-5kSi2(~9m=QHYuGBGB5I@nvKQ$C{cP8)>_ zE%>g1$&Vs~QplV3cU*c*2c#dh#lN`^@)VKHc9GT+d5^CZSkM(im>0o_KQ?|F6!WSn z6%DmQ9XJd&I13woxM*|i{|srHbzuvRL>{Z{pG@o#H(zZ(b-%2S8H5$vR}|WrGa(OT znu!2&BY#y@Rf`V#kcM8|IZKC7;zp=kOS@6i-<=}PEVt`4q3%ftKDoNytp@AUI4F}D zd&^T$i@NcCyFAbY=m$V4`*c$H&)jI0S(VRHx|`jiW8}+O`lPZX+QZ~SEP4N7yIcBo zeqXxJ?Jn!xVR=}U)BWPxvGFSxFO(Z!q7cITnlf{-!melNF~-tK*ekN33*ifC3iilU zXg#f)Ow-m08N0!weiIlI0XyKY6k{h5T-?(p;;Olqa;wzlIsG~j$+Z>xy)D`8<K4N* zkOuTZy6jh)dn$e~z^F61Rr(qN+^Z5CnWe^XLe*P&)bh9#Vb@};RY#Cr&}GO(maO9p zg{qH&cUFy+i<-^OL8Ij{%NU5;DpNY4zNb1AoSPpj<c6sJw#J6G^b^x`jxD{3q7=*8 z9okI~X8j15M(_Mh8BjkFKzWb8QEWX>WFzm~sbQ*8OVENMZHL%+)qY>4)9<%ui$G{N znHpeeb~!s9W~?ddLiRn(If7yE!4<NShkdp2UOXVdzAFl7&1BU1AXu#zXoJ$eeWOI} zch{9LIultaJPTo8`ShwI5#u6H1}<VMZF_8l#@aa?S=_m7JEVa%Q5(DL$G#1zt@|w3 zl;>9w!WXPy)DQ{1RHmQ0DLgOL8li6)7nM_fXGAf|C9KWsD@gTp3p)~zzeuq_!La=2 z38~y#n&?StMq&ly2g0cB2;lqwWZ`K2zxDV<Ehbz7hJ~z>s^ad+Hx>==s$$65HK|@3 zG{x%ExX@!}VYGnqKf3!O3-WpI5<9RUpPSNTt;+$@!~4CzHn?c+iJSDx&{FOb6Kh%J zVhcV$2~WMCoDTJi6t2l6r)G6psCpsEX#vFOc@oXGMsi3#XwKBO3a0t#tUuUY%E5N% ze{`u{OMCCQy|g1(Fh&#A;?euubu?V(LEEX2-OFV@yj_oz<JL|-`zl9*;7DEpTOMD7 zXqRfIWkjacoKhM6fVg|Jwj@O}5H?$b_0O(GnK2DZD9)3%r}Dd5jnAZm01TN7FhjWL z;QhY7O64b_^;ep52v~Ema)8l>Sx>pVnH4_MWArDiBMYCc^%>NDxH4hQKCC`X8x_`X zn)1w0=7M($f9~e`v<-j?;=RQ?TWVs#@;Y|B4F37DbB9vXl@*ftA_8P(k7s3zXJzAO z<8ovXHTgNv?$5E@J1h`M*9(gw(UN@eyc+dPV5ZEQrHU_}lD`xiRL)WivP*Ux_!jxI zF2%?+ju^fWa`T;3XGX+LRyjO8d=P^nVG^<R1w1Z{00G&&RFW5DsCUpGo-RHEDWN-? zJ8xl{lbLN(y2{nMAn(hJzpNHsfWzz=LiWoax#65qR*`#0u<znpo>4EK*k?8AY@}1} zvT~+W*FO`|3E6Q?&|@vp6w(p)Y8qrLWvQ)ls5;rZK&ROS&pFIy(zQ2>*q4#Bo(g(J zjb6SxV8Ez=SCs8>QtZBA@atutc}Sd*tDb7Y;Jka)(U6-_iEokmAQ^kEub34jf!w|u z_pSD6qUYwX{hxw<MvItnm1JP$iJ`9v+>^cU)Y0ShnH4rm+~i^+u)loE{;*y$z_cAJ zdj$vMf2N|29<NYZej_9V>bJ*5!ga_YYqdh6mj;|)#L}r*5@yX!ANS@U+2*B7t+phm zYv|JJLzc+Y_%nETD-kxu``o(j1cUgSGNMt`XkdJpBick9A+|&$1~B?fFjdxrqoY-4 zrYvF>*rXLzhix2rdyC1>&Pg|hw~0nfn5!U%T?wxc_L+M!4Ff_)rA5`j5z+)u*SEF8 zHOfHxLSyGygQF?xCV)T3jW7!G=(9P9Au-eUp<D`u+_-G40+n;0&ofb->oM~QBH5Ze zJVESXoEwVLxO$K2r?%sh8n-t_cOzK~wTnOEQ%b4Zo;vtkFlp_36IX;2*k-2$kpwW4 zZl80}xUd_NavYs#_&G4%eeKhCxv!9jC-=9bZ$!rPCRH5L{P$<Q@I)Y~kTo*{dFhXm zb#!bL!xhI8vl|qq9}K-5BdaFKV%~@l{}t)_5~6p|B)NOTV0evI7X1!|cZ6Y@>6G}{ zR2U?*cv_jOt_H7m^&u7u#}r9pbIOzf@EB{h-d<A)ih+9k-f{m{F+yUKF$~*b!#Jxa zOJHOug~bI(ju)xYxm$3e!4^aj7`&kK$C79=!^O9=rUUYoFadw_>wxg-%On08n>rx} z&Px_NCGkH${ZuYzGdNP4_32{~`I{q~8;P9>b&Ga`Ap;5``!$*i%kkx!lOxjmzWp*u zqUmwbZW~_dx+kj?+(@3)PxL@m0SmF56XjspFSb&VUKd6J#Vld5=o@+OtW?AV3F|4R z08Pz1aERuK&if+91<g#B2^G@HR}+D|<S6^&R7(9{-m%9;Ae@V6(nV`zO9&0O$V<P# z&u&xJWXwJ*(EUYC80EK4H|lOJJA1H|HG!4AA6Lq}omCMGRxI~!mND;rN#Rv5Z0t*B zI%d3_*u);S%V4Ejo8zsg$=@<c2hxrN>x!40j3)yPOe$`i&7}2f_(T-p!Op$|pcvCX zUVEuq_o>mbXn@C?))m(DtkHg6{%^YA+<_9{2{$|y?sch)mJCI*g@mdzA6=uoW}}0r zakihp2RO`aeT3X{+zB4EnYOQ->XyocQJ)0ea2Uo)@5sGB{PSY;UIEgO6A}N1Dk`ek z$VbS&s{UK2u~npsUHXAwxpmSx_}MUH_+C3MJn|WJB^nJsKSjYtBK)Uq!p}0wbuX1p zDfFj%wwR`eI=HakcJB|I^`8k~2T?YZd52e69-&|}=m4lfPWYLxN@4xIFt~>kkqT+X zAk?ftphKH7`$}UU@|=<MlYFP0pH-O;1yhHd)uc8_R~*Sc?OQvta88;u<?3Xcb&wD_ zkutXCrw-D1NKltc>Cjb(l4>xlubSzjsb*p1;Yld_i8|kK*lcb?&xcr#Jxs`Ny<KCd zD9@N$PacjhyfvX#(=07axUJgn=!F4+$n%EPp13wZ<4;Mt)*qOg>W-IUTt{_%-uo=^ zUSVb?e$MM;lD{Hi=J$io_c0S_+8ni13ZM(e&*WeAqtm#olcFUZTDmH)?}YeXI5&qf z6c!_|b9$v&FSHG^q!_zlj5uAmq8=i$a-xq#b^hHi<E_TrL2*pmi4lV=tJs@Y;3%VS zRn9!<iL<nl;lf*yv~O$3i%)M;J-yEC1Ty7F44`{E_ZI+oA%WFRLlnXBz-ArEpE^M6 zyO_y#wLxq5I$i+0*89Ek3}yVCPUbUEA#SVB$~p2t0_7*Rsi&Qbc3D&}phOJQQB|Z< zztWeV`tE(2{FA`o8}-cBP=^n{D_wRV)$+eK|L?dL3O#$lhHPn4UMIFKKDCo`us?gF z4A2v=N)^x)3tzbSD60#7LIl@|34$m3K8yK>y(S?`PUMy}o$b?7BH1+>v4fn!Rzu>o zH2r{Gw+Rj0uxDOFieZ%8f3yHZX;}gfA_=^4-Uh9m>Fy(}T6~9rdnPrvs4~>{J?(^{ z6#W`eynw25{w0+^Vedh>{9T{nri2R(UzZdDxLu2DR(CxGbYew@0`yC3DO;T)xfxxs zjJy;Cl_2Et%B(MKgvz!C;9om=F*yXK9`otq{&pkvyIk#((n96R(`g=ac(>13b<_Q1 z<ysOf`Wp`PQR|^hVc)26byimk>lGXFvSt$~>&Mwf+;xPpHm4fRw7B>+9=oDfs2r`b zENna8+dCk-km)ENxm0Bd6ZD#RTXp41%KD<FC5#C0^s_i_%k)W6S)^Y3nACE@)4<l| zSkh8ED_<KIZmnraP(lL2P5XyLdWFSmEY(wOIaX=fg5u+6E#U}lxgJtdk<*%Kq^}Kr zONI{UkMKDC+T#=$@na(-U6F1Rg>l18$$mp{O6lpF{FcAzrWZGWH3h3`CpB~|WO%q4 zx|p7N%^)#R^f%&#`L)>MEvJ8mSvemPqQsT-T|$sd%Wj-OvtEri#bdP=*d+wzb%|c4 zFh{}fY>whkUnLCMBKc^lmn}YEBG5V5VRz)%AoF%p&JINE*W^9vTfd}lb%>^9>@}vJ z^`4TBPiA`mOsy8=Uk!;W9>!&tBRGtqu814-9s=WvyAl+1_+5|2rF%h&v9U;|gy3}M zm4;OOuH5F$QT<DD1CU26Ud4B!d^C-WD7fq0&raH?_RZV)U!Bt)-%@#n$pw{Bqbomp zF^iD-{ZMdf<B$#I&EH3SndO)iR`43VKjw-d44<MRO0W?<en%qK5C7TTO!<K9%)f?i z7*v|g9#3Puf<>Gk_4(Nmu;u=O{X;XS+QyG>toplQl(L-03N={p4z=Q;cGv%%6<wwF zOBW~sGWXz_O*A&0;p?6-<|X<7yupDJt7(2}r>5s3xVuSjXn?M+eR8V56E)7-<2i6# z>3eZG7mrg-K4ngz?MMNy_}%>g@<n!WFU0Pl2DF{x2fT=Xv=>wY&V`NDLIHzYF*X>N ziCZ9Ta^J=aKKLR!6{!n(+u)5)<y@^FxEvu#!Rm8)!29B8OQ)G}7ZBC??ZuHJE-r)j zg41BeW1{by-d%Md#ETN|C{7xa1caS7(091b0Hv?{{mwhNuEuk*@0P%%9}3mFA%rJW zoSpxVvHpMS5=<`CW$|ec;8iUAmp3rz=O4}6TuH<L;~BG!!uE`*@ph|Oqv)y$cQL3a z-*_{61w<mfGQA(i+9F@LyNVwLXD_+8Kyxv|x^vtbs!uNVMES!%k`JiVKdzgBtLU*w zGIcD>HiOzVT)+eOl^$ccaN=)d6>d9~3qH!m$G&#CbQrn<3g^d0cftd|s03W-vC$4n z<3!n78P5$R$ba1D@{WwCH=PZ_vV4)i%!t1*T<&Osk%$y0W2#8(e9%092qO4>%2D>Y z{{tkUp}Ui@>5U|&H1YAUmx34nrXNi@(hnQxoI=nawT8Bs8~HF-{5CI{=1GP)b?0tn z@?P*JRKRt19lcm%b<+3eIq!J6vWt3VQ;%Q*0tHtJ>@sYafng>?!3%PBfZ?X#bJaw< zH&wTh=N{?)!6}N`)6-LAwqh^k+rs538^xZOGn;}tyKUp`*RD|&h0Zhcu!iQ4w_m0) zVD1NSnw9P!erYb)j3B}dHp$y*`NUkavjR&FM<9_oO3hS4D-5X2=3Yxd-`j9DpWSzL z2k5i?Acl)pak_cJhCPYftmoTuEsQ;?s;c`yfKwZA!ldvq6oT}7+%*5pY)&@VE~u|( zC;?)s+&d-b%G+^yXnL<aqgP~y^v=x&++@K^_g3<#v~)+kReIlexzDFpD|g?#!?jFN zcW0*`1feB!q$azko=j0~)mrW47#(>qrH4$V*L_=(>4+Pp=)2~4bgI8*GgK#z44beq zL-Q1Cm<9XHS9Yj!A;TEeugq9aey*#(=^xu+F<TOjc91`M^l)Aibha%DBY#FISu)Oo z@+l)7Cd()3l0T*_umO2I0T#6aoY|gvi7cG#)?|O49EVax5|G7MxH3?sX}e*3^!5eu z-qZWjGKt|VilO1lQ9G1=Wt@U-s5rccqZ%t=^;BAYSs4DWp=%!R%pa2ODkHF=5Q{$8 zJDdOkw4?l&_F{HTgU8*Y7RVOlQamSolBA0B?KdOl!KBZ7i+kzJ>LN(naIX}XnLR;l z9j;Y8V7cNPSWdAiA=HMD6mN+su!bHpOM4v6%61Y=@Tyv<R2C1ej%*{TlYPr%NT5Ba zViB&^YYFl&X>kCv-<(>T?h6S~KHQ+)doET%G|6mBR1Y$dKAD6o5XDm-KNf9bqnPmt zQN1rw@^||;$Ygg0fK6}d{+9Jj24q@522Pya*%!+02(WE-1aC|fVtbqA_*E~C(UxfH zj^c2&psZvd(h1qO(I$Q@*cl;8Si+a!HVR={*8}GJkK|*`tpUSU*HZug_q+d73ntOj zgL3VRvK|+V_4E3dq7<X=6!SNvbiFz8WnTvxSPAP$k=c1TQS~|d6wNZDSAzpK#g&*t zZ?P*=8E!aM+;UMbu7Y?K@3Wmkg2DL_?!~1f)?8H^JD-LkBK$Cmg@d*Sf&OepY*3j& z`N67>T*b!Bhw1OBrmZ7x$Xjm-xn|dAisv<DwscakE6mFMxG)=7lC*7u%m<nU4?I6_ zA^ZcuEShoMvit?AtMZ{3(gi#!mjPTsl58)>E~?mbg-!;O(xI7Ws)eF+ZGKM3LH|F- zLzvDFwZH7%^pEs<Sn7}<d)cVJ;;}Ut2HzZ?%|Y06cSQ4i@@Uh=*LO1oqE7toYN+U* z@X_9!FO~A(#Ax$EfakqT;rB)1$4JuR`KDJRX}|OtYynsg7tCU2Mv2(mC*I>GMl4#g zoLJatrAyw506o9GANd$bFUst8FA6Crb=*s$TVuvsWHg~yC@1A<4pB)E%h8thz}RCQ z7mVSiKjU5VYO2A1PM&8Yu4vcH8pg`+rVUTy7yBf_E*!AcM@Zk;*S;yYL7^}L^<w-1 z-MS2yf0%lW#>sEI24IWwd(wEoWlFE^PhxYB2R~n68>I|0`qH0!q3U~X%OzyJHK6>y z=~)lL1Q^5p{>pCzNCtu{b@a-l?X>YTY0jY3bPD3}C5vcb$x(ETbAh>HuhfdTLBj;` z=7MwSv=5HeSW9RwN?S0h%v`jVW7J`2h7u)8zr*|r&fXC>4smT?G0yd~D4}g<z)nl! z9#+AQRzv8E%Zeu_zo7T5Pl@fxs`f^Yxd@;+5(@*gyqe3x!anUAFW?)zkDF*x&PToh z-%YC|An=!@oTl^*xa)B$vySxb&Mzd<N_JKz_aLoepoh1@UnS(qebU|-nrc}r(gJ=O zycKk%_8YS2nF?jxY?ZWO+1^<TiDRy|io?=EIn=mUA+5TONwo^7%@H|CML9+#m+=f4 z2iOSgO@CFt{8n4OCU(Kgwm$>@iNN$BPK+>Z1zz^EF#j~}_GuMH^jwG<`0~e*U}H`3 zZPehb(^Va^HoI=+JjK$CII)$&j5FgPY!!K*cbll&8#mSK!&T#L%$^zRI`>K9Q-^~& zVZ_#U+BloW?kFbeht5h&B<Xn{=&*(O0bZB;OCH_Lgxo>9=|2)#1=ex%>%Q;Ul9<o? z|5<#~NINa>MNf1UEOQBd-LWU>HxV|`#E3o(zq{p=Hm<d@>M6$LN&E9_-1K7C^37c! z(f;v?CjbNuQ7b4iG88H5<9(7>#LcMeryt74(YmR-S)Gns85EG_xeKIO&NJ`7CP4QD zMOk}gTK?oVJ`MtJ?QXF1$*8OlHfKf}JPSM&{@>bw|8e$jS>Q_Z3#s-!T+mN2U<)D1 z!s`-*)aO(DaV%Ih5v<_=N9%{rHjTHf>_Tn0WghTfC4#R`@wk20J5~z#8eTcSFTyxE z_6^b^ttnqzfV!zZalZ9XB^cbmRhQqDK$a4)T8)WP+*M{L3>U4M-eetCP2jFN<O)oc z5B|A1S!Q&}0^@V7auKHu2n(cpXeA`14P#C7PJ<3=UB{Y`>~unAT$qq9o%%21@S-WM zhki1Ajm6Y2n{{jkH5)8Vv^RxH_Fb+fycGCYc+KpI`$vWOjk}+=-uRZPi)b=J8z<J` zI^p)FS{|DehsjtbE^g468(Y(#sJT@^-D-Eom$_m0=-|zb^|~6DO8bYwm6oR+i6md% zU}so!@|-_BaTsn&T{C4v&#*H2RH^0m*-J~d)jz%_1y1j|7S|~a;r?pKeIl{zJXHD@ zZ6SNx02}x-S!GhKL<3VhH~!wlX6EZd+^hJD6&reK8y=M;q1!KnN`p&2^qBM|#FSYb zJk|YSE8O_!5QFiF<AdQhHKEF;`R*<Z;^(xOj<^M1ohA`^VI?R!BJ8KhBy_k2WjvX+ z7@4*~&I3U|s=lq`hw0)EPLP$EGs-$_Tl`jx4Ct?M)vZMet03FW^G@(C_s<_Mg!ME- z)UAE3gLm-NZqA6sZsKPbR@ypR!a~{nU2d>&Kp{nE`vp&fJMNXZ!rf2>@xFvTWH~^w zKcM?KTO_D>5lL_$=QXFkJHC^RL^uG419X{`dAb$ZOptnWdfkY(P6pO$NqZT>h>zJr zLM3^&Ax$FG%O%Z|?lI%y+uo$pN@FC`p8(;&mr8{OwFNI;x76cS0-^qLRqQD>Ck5O^ zse9fkiRI6tctZT~YHz$p=<@Zb7hf0X>GhkQrl76G33guqq$_YSyBC44*PUm^fPdG? zN)aD)z@P$Ni{EMhO2{$)6eIrp;mN)mFHyimj`rLtNNl2($aVF4<YReNr0zySP3wn= zUvGeWVr6=mpb3D?J2R7gCW6s^!_(fY2fIXI53#pquV_Kv&ZRAOv%^wt!Uoy!eD$NE z5HT7rweLPtXmIcZV!3Q9!|Fs^K2@7WgRgVeRDKgy^^vp;yE4n#SZ9WPNW2(om(`2$ zmOq+B0f65uVNKGx^FtjgmMn<_bkTmf@Y+Li>zz6e)1j!6F<hnVu7d)OK6urCH0v>p zF1wy(wGs?E3}dLyePIXI@1Tfn`z0N^HRi*+|D-sf7u569rhC+af@9fIPI&F_yoWxF zDZQoWwcFhLaca+JY;9XECV4i7Y)ks+zD|qL>1gWs!zNabmD}`^u0`{#sjTI`ynCDk zawk^I)Dv#t;135s)R`BBp-N)3R_mA?o8mClYWsh3byRL<e5Zr9=I@QdNe$z{D;3eb z*#9mf&?l%;FDFzVKlxrdeL$@Qj)GG-YZCMT7A@7DMGcHYFX|6f(fbxC&{}{jf!0Dx ze8@Uq`V-%C+sPMs4ijp(WnV5zQCuqR3mV_`N=q1!!={@Pn`GJnG69koRts=qvtQo4 z`;W%vj$ane1qr2{O72gG&uX-4hZxyi9*|5XTx7+M%{+d$^>h(lmHuSS<AwvLo7je< z=W)%FEwW0zHMDR+TLyRpvyKDXi1Y;>(t58`GM4q~(c!=EwQowNDT3f9t%*;sG9@Ge zUf31MzH@P1Kt}{SYTA1!ie&9jOEvuV0`4|sKS0tMLx8V`6d7HBZ-VL5KPZ2km1DF5 z)vNCW9E$b@iv@AZoWHHx{jqIRi)~o{r|zV3FTlDqUfqGPkU7#Sc<YmhZ%_wj59hhL z`r7^SJIRjQLb-tRC_TNf4VjRTknBK$jI6^tQub?Ix;60NU&yIB5R8|?AcZ=7{trWM z2%FJ2g1FSwe!s_Syam<56-@-L34>ro^mWX>pW7A7PW3bnbMg1K$dZ}%p>;%kYS5_G z@=(h6b|L7T2^@>XADk_Bs-y9Jb07R75>L09(Bx+w=3<~vk8}_Bt9pOuD^2x0CFb2d z<X%zo;?TQf$L_I#R>0o{veW(iy|Mcv);f_ft^8l-#e}2uYk4dnGMc`FztsNGUqNDU zk{=VA?YodJ&RQ{m+MRX_JdB|3!KIuueBJX#&JWk}U&BvY()VNxHDOW+bZy|;t1h2+ zMRBpxvF2*8c9bq&^vOp69^O8dcN?8{<sE!CXsXKfj5F>>YsEi?#2inJfnD`8-$r>* z7aI=L3q75YP%=YhYnO)+{CMI{hcA$$EgQahw>oAw)XyO~RfGK}^?YDUeiZcU4G+8S zcTncGsQELS6#Wm;{iRPkWZ@(}bF};A1jfsZ2qFINmxB9($quA+m;#>+-g5EK0lm?_ zGqp@p2AcU~^7mx=zES5K1kl&N^)%+VL%d$Pj7o^C=V%$8wl2puc1}<sj`cach07oJ zJ*SAi2a=xL|0_kbyRYCS>Ld6lWKI1|MCC7227q@%34O0{v4FpRGdc6}NC;!nC$1$A z974bgbCCRGU@<8GY~!|HyEQZ$--<$<El_|0?h8j*ej%YQ`$t`rMvkw*mZ>7s<uyuW zx4mP0dYaTS$%*h1*dvgfAERzAUF$YrlmR(8=WKbZ62)kI`}*Tt$Loh~o)>TM4-8?( zf)ZmsLbUtqRJMK37YC1@Xa6R+Q&h;1N_i)O03+U*M8e?`UVQLNc*bg==$XSu$aSO1 zNG^kH%`F48$bTI<x=5(G*{{6rUfZ$V6cgWs^7E!8RkO~K{KD(Xhx{rIPS20SwQBdI z@9S>pF_x;~TYE3b$2-}-qB8072DXTv0lS?D58*uE0KB^|b(%^k7oDac^~p|}ey5PJ zSh|-jMAS~IVO!mW+28AIXCdY_r&QRf&4Oxvf;JCQrMmN)<?<7p*qXDQuO?kwmldD? z)m1yQT<$^SX~TTAos;A~R434IB#6>Ht;T9iz%E#RLl8|ms~PQbV2mJep~Ej(dRD`h zAgTqU&?FZ@_F}7B=3YzMR34SzdT`;&G0gmbarM>#P4{j4|7fI!O1FBUq=?cn>OvGn zx*HTnBaLh@M3~Zw0wUcV!f3{5BuC?DX=#BmYJ5Ll&vReD=l=cczxe*_JkR4eUI+HH zRA4R4M%GQczah-^krdRcv`VFjrPMK$tY;0j221i?hv+i0kK3EKcPFGQc{cS)Cd|4C z{e9c<;pEqU9R(cm69LP`q;kZ1h9p_9+ai_?Ouor7*W6BifA_%5$*ZC2DKb!{3?#-v zUsnnwRO~m)u#?qd)jnGa82)iI3R}CV4uz)C(QBCqWCdBH+_?{Z6v^P6ETo5iYA75T zyg7Am>N@QLvi6dF=WI59Z~45l!d6YJT0*+55~&?}oU&(@6ofG#9M%Sb&%6&?+1;nX z4N`}#b<L|})n$|gQXr9O-V84Z18Kaz3ZX8WyQt4d^@UK|DYHp!1?%9&(DY)$JcL|} zKH6ipFc5Q<3WZ_6bZ76ken80s6giUE3tSJ%rq>4<N2Im_gt{1=OZ7lAEwoVbS8+b^ zf#%G5hwrr2mh*Gv(54%rs_D+IN863n2XnPSj%RDftlNGPt|NreEmruM^frMXYjV~s z3dojS<`0P#Ta|yA1J{UMPSk68dH;E{f!wpwz{4zfIzMu+IaJaL*f>wpzI@X@m_6IQ zgsE)r77{#zyIa#=4kYuV%fIwZI17zCp`4lvOEx!@D+e-PuS=Vd$g=cgfO44yR9fw9 zZ&GEO6eIPz^9_`vLwGj)3yoMn@2T8q$dhPVSPZS`RdT)i6MmJuoxdQBVnEaWh=OY< z>&#i`cd->{5f1Q_yhuQ*$jakzQc*4REWum7)TXHuGsKK958f9-KRMAJ_0yee9TGXs zT^4a9vUV!A%nig&rF{MP01z?;z2eShX&CdP+SkZf$<R(elg~MlcL@Q#GhW;Jag!<? z$!L2X9z7OV3eDc{nXn~*y6-KLFtv21;5_)g{|6vFu#H)P*u^4Cul23bzsY7Gs@pJG zn_OYHzM(u$rP@t8vIBW}i_E04p<yAE_nhyadb%IM*pRb!=5!@?eA0-1ixF8nDdtam zB%$LDM0bh^w4s250W&jiwe2XyN1hiav0ZPvUUD%R{mSBT!65DAA&;i0Jdja~d*8@5 zuY$UhVgg?I%d6L>h27iay%Z<wQ6sqD6Tk6s_xpc4jSGYq_+q4+7U>b->wV?_|Jx;I z!U&O0(M~)Gb@LiuD&79PUE9J2l08EBQ&WoSJ)nqqzdd>M7<`S@ae*wZ^7LkDQsfdH z_p<~Bdwn=?u?SGVtu<f9%i#NHuhLt_lvJ9#F4EP>RYTtY$RhIMMvL>(7pt`ejq2=m zn;LLKH5XlCk)tjMRq!og_g>?uwlk8P&yJIW*Q{u+A<64P*MwQAibu6O5XM-SCwc{z zzZXzI5An;PB7sVLvKw-|-g~2RJh)yAI4<o~E^E`m`q%E^vO<pc<#>W`H1F}GH{&3V zoZe~JS8vbno@mjAFs&%i=RBTdbKF6?tS$NZI(vz3r>`pY`{1Ph0!j=QyF+rX-I$Mr zo~?S_$@8okqQa$Q4-BC%jGlHH7K)aA$$N7<wbr{SGf=I<DB==2qa%;Tqsm*=SuG8O zxn6CrJ$q{(BU{J7ZI802;kh+9eUCI*gO@T|RG&mB`j;5Pcu<Me3>AoTw-B$2Y9Kk4 z770%~(#G@&v$QQ8u(T$k@ec^wWZL66?qMAH8QSw^Q*{F;!>lt`iqfV{-HEYwYcMQ} zJtM%9Ub*b9Jje##Y<Ay3SchGy!NJd`)2U#q#w$8_$J8(kxnoMM5sY_|@%SZE_q#)U zuc$fMcIY5Ie6o4bgYaED=$j+g3ry%o0R`j}uZ*7K%sOh>n+ET_?kz51i74n`{gA5c zEr}pUu?2Ioh*cBlcIBXU`fO9GPRk8?$w11vhrND=a|wrh@oedi$6!4LXZZDGVRc`9 zB`p_@{JEu*3C#hZ9;M41_+ET2dDSjC;dFOf+M~J6M|0CFoneAY@8OIWcWF@duiKrm zqXSeNFF~eXLuy8Rt|3yka_%dCV`CIzd6Q|@ZM5GmwO&WYOrWeYej}=%QNna#?|r<{ zUF?AD`t+;iKE~25oAPb++l&suw4d@gK8i*?Ak;)z38qv2nu~{5WZf3aYwLEq>#pS= z-<|s6ypVZ5_at)HbiwwQ#$?(`(60wg^B<KggPWq`E_+F9^$!@1d;~O3e=14(*)<l{ zr<aFFgu`rVZ`VRsVeB=DS9p=<CWis*Rz=8ad`CaGw68_g@SV*1rK;9sdCRTFAbZbn zl-YS#Xm8!nO1T9IyOZ`sSN=nK|4yU#pI7gOJ^z|NRxGk4Om22+vozIPMS?(8Rb8Ju zn>%)UZD%AvZ+W<X+*Y`oL=Ko;!t4V8{42?2<_b_y5MUeM&F#l!0Y4hgbgMmwIji>E zNY`DrShC-Q%9a$91PjK^N^xThBg?jJZwdt+IL%1pso6BUH}R|XO!u{8OUjNdcLVD> zbl}t$rb~nWy;iWr&|t}2ejZOP3TX0p{dYJ4W2DPxtT-ohAKdt?1oUm?UfExH1_Rie zuSJ;RN5qsmIOS%92F8os&>W{=7oK}#fH?`6CFuWAB&k{E)kLe0e;zT_WXiFGBtZx= zIjKyv*Dyt>bet~vV<hh2q>YBRaZ`=&^RoP!G?fyPA>oXn%6{6di9;T6FKx`lO<ePT zFJ%tSl&7C%6YMW>?4#nn^|pv&y&gT25XT}Do~6Lp(BeXv>P(?n|LJw_Q?UOuAaP~N zsvYI+XN5Ie_CLlZG5@s$w!}_7+s;2oSvAZrr__x6(lSGN&V8o&{X!h{fGVyf=wSTb zLB*2DbTrAN*|r#2p690I>C>-dvK8D97MR{e;kzYASLrtzu+5o6`sZsx(o0kRN*Y|R z$Q6K|hkLdbR5Kn&Up#sajUzrvS8vgYW`m1Wdd7)II@|Lgo$U}O6gUIda@EKUp_!Y> zz+s%6q31|iAfc9<S&x^$)(THO$ZT<!l#qxH)g1MQLb<L06NQTPBy4@xlHPCr0V|p~ ze@PZhIDoBht3%l{no(CE$7uGwqv^G_(Ar(n;|oV-u2<`91$W`}%4-A;n1%2gg;ye& z271tD1ITy&%6rH4-7O?5v4X?u?7dj4>4UEJLZmn5tswK%?i&h^KNUORU_!eYZ)|Bg z)j=RD27L=(bm10YqNW@5`y##C7Ytw&y!eK`s=s;>EnB7ry%gF=pxHfvWu7n?O3Y#X zH2lha0Yy{rN5+q}5xj)GXoM+xSCc*7XMTYbX^vFe49Q8Cic$08AFDmA;O`?-)2q%p zV`(U#=wjdvgu3s1-Q0ncu9l@vp=!3cuIZOr;vUo;>-o@iri@9ama{A0&P~%T$$U*f zd4uZd;zTHKN0jnt-jb#GD;JbUk=06+O?g2QM|irzS3vPllVU?JqsRbJM&}{T40&j6 z!)bIW&zm4l=@Qw6D9_4aIUlu3)PsL&uc<VIo=l+_EuFSxTkPctBlykpG(D2ZA8C5~ zmJuyUY%$R!Xorr&ZJY<6eZj|HIzVDo7(JTQEgsA9qTH%)NjfLh9dN`@hqzl}R}-*q z)lp6CjjLS>3V*#w6>)C4iK;>^+BXcT1&SO%#9yw_^pM>#w553NCm429ZNU*b=m|BU z4f^vX5NaX8Ry8(qLM}G6CySV!t-HDE!CXFsVw?F3Jkk=MMj>oef&kDQHF&CB>&w&0 zJ;G9Kaq{Vf;w8QQvBE>2Z5GzYa-BvepG7LVNW+@-?UB>%cRj27|KkOKze`7AtQqKQ z-Aj6+kMl^kWgxUBy9XYR89f1AgA@$t8Xa1DFysGz{2(Fa;|6x+LGcg&omQ$qF~tdS ztQB>eKe^y;<fTPC1sggq@RwQq%G~)BT3V?S0}dCKV-mgy-21b^SCD=SbYvKp+xubo zChN7+PWR>?U4xxlDQHz$U~4|4f{Xe?hGY}vg?pHuz$e7<(v!d>C2uyXKujYwh~6+H zUWgE`EVxFtp)4P?TExo=EO~*Cs1783AEE!|l-Oq_6Vq!<Z9!c;{$loKQgTncrjta+ znOVO~thO`jM%RZvvb%W_2Yf7co%>Jr?sui!c7us?{eE#3_EfS0s<Cvl+HJW0nM&}O zYPR%HJ=g6k(epnzIBPQ{eec274gy?u@;N#{{eDFJ%e_qfp}A$^r#Fm7GBK;-Q6yZE zQbGPaUS1~gwJ8qCCyUcR!Y8!0W0K0t#Wzbshg4s2I>{f_OWeMq5WiJN_!+s#IZvRh zuBmA&h<P{_yqmi4DSi&5!F%Kcb@@AqEu+wtL!zdc=2R{{1f9kYVuFMRjVS72sL7<2 zb182~j|uU+lYZdTAX8R4bT=V4VP>~o)f_c}a#d>3Kib(fWdJ@SHsihgdz<qR5>{e2 z*k=EuAqsz0bq9x~YB`-?BDA4|7dg}C=o1FNQl9M`M!ZhPg6=ueVMfM5<3gCS>m(eU znJvXMO2i!tnI-euwodm{YWL&`>e+2(nP0TDix@KSTWjL)#J%_0)*!#j)5MC=!T7}@ zbF|~!nxnmL#AYc4C_5V6TsD4pfQv(OPka`9;`xO7-reN(!g<luw2$<eGrAJT_6N`( zU=QpLC(Iei)HqVt_cm5?lNl9KW=Y`gZm?pYgq1{d6BQ*V@Fs+$?&b+MK}tO#w*2$x z@ZD5<JBUMSBEIx6P?a(cc?)52`%(DSJc^Uv@YqUs7yao=;E@iy(U5cjM}UWPhy+qY z{<go#gtWV-cFqTY$l2jWl9ROMV?E3(^vz+mIFL6j0q`-L<?($nyu7~{$Z?6QHxCEv zV=nG5*mDWL$uG$Dol$Vm>%!P$?9?<@T%;*~sZoU>d75+_#%=q|)P2OrAim<U2X4Q{ z>evLx7Eo>75)LAp2D&q5Ua?F{Wyxi$<@5%DT%qpVnXb7d@o3+YGv-2W(YoEL))n4T zaE)Q-N;LkDyoo3VJ#XryNn;tf#qipxb%^n;#dP~1m&p$!dt~Fof~>$%fchN9YE>{m z^L^6N>*UMcg!D5S<FJXSNYa++Yd~#acA<fbuI`&S$qcU_hA8{i*YhTYEqfdTNhap) z^u0aIaiZ7nSbzP$Pub?o$fj=Fr`Fy5#b(w2%_6#7(k#sLh&68A$6PkBLa1Njw|*cl zI}^R>(dne-E|<M4SE0-mpdLY_*>D<@%<UDAS1v7;py)$)O7}DDmAx7;x3yQ~s*e0P zPNiiqKlyCm><XlYD{6HqpRpDU!5c<v0Y+01B%f+i+H@<p9cgoAGT)$c1nv+9p!ck( zA$R3w$^knw6$<|Ooh0zYd9%y(P<C3udgw@0$f}Us+TKf1<=sB+1X+>JGz!QnWATff zMDbYui%1uNkrFzB40*>V5+M3i;j{hGoA$?`mz5^UPcYbNQq--nn9G^M!%<$`b>YHq z=JG+^IXux#lfE+0P8jy|fUV3BM-!pPP_<<#A*)y|_URm1`5fN4lWGU5!`1-=1@7$X zN&2IuotBgC`7rm{LsyftTJ^o)`CxLcz(d=6wm%KlH-mUUMMi@0W!826{IJuH<_l{r zL0Oc%OU{?G9M0-7!5$|(9v51-7Q8OcR~t;JpZHail<brk|2^kJs;owK=x4lmU`?ce zU9cXvCptVskolFi{nVtG^EtT&oGO`8f{7=4yXmKXP)ElQ_nAE{wpRqU2MPtIEv0eg z6V{NBTt&9hF0+3;2*9Q;9ym7z*LUF&>h;RpjY&X?FlkY{sTIs;Rhr?S=<QI$wW+mS zHqjxq8ppxO7iv#;eBj}$5xymCRmoH$`*=On$JJt~VC<$H#&P~wvhO_iRXJ*UE#Bl^ zod~Z9`qpFF2M%+;FK*8sJO>5Tk@RpWM}$P!DGM;{?K>4^ibSLXVk1LdCBm=Cx=Ff* zCY#g#=?c_|idaJNuAofa7ilwT;NL{IZuw2zAS`k2y1CDoqe<7y2Rn>3m74vMEAzlg zCONP9nk_p-gmy;VX5#*({#vv|@V=rmLwlgz!87(f@uM5KH!aACwELL(BvfzW151Wy z!*iv)6X%u>iHDuax(d!-I>V=y;oD{O#ctmHcjqibPR~uMS;BX6<vx9=B@oqUcMc@` zHXMHVy=pQ2iOy*{#ta2BR1M15N3KU)gw{HImHkw%OV%Uvt`!l-9amKP93Y2B3Qe=f zD(kBI?ab8gRmQ@fTDHne%p1cu$oBBrzH_iZnv%^?eRCj5NnJe0rRw&zjH*+urURJV zGiGTnJO@xV@L>X!!qHxVPFmm=U>tn^QEBM^uId#_0{3R{4e~K-zaRZqQrIP!=8<9S zRXOxZg1HoZmmfXFr`nbeFC%O7NX<YeDZ44wp>mQ`U--{N`RH4eR&1Q<6y9DOs5P-< zWKb(laK+?GjS5vj6Y)CVn286DJEz&ic4vM@{WmP7hW4r4A1?@Bq=?WJ<FYRkiRycm zL*Q#2>*uSZ1Nn+!T8_+Hag-yr-dpXRX?TQOd0KI_k~No(f4+bQwrCjV4|Xg-r4GU; z^CZLJd)%&KITTh;GP$Tc8eoj1A9H9LG6M_33&+#pdksUB)@UCo+kWR8MuL8CSEH&D z&v2yekM>>_+<Y}`#>y!MRDeY*M$%t<`?dypaeF1`5Q(CH*{vV^+7pE6KORbk8$EsT zjHcQ4ykqC9NZ@VPXJQ-=j;*1C1Y86~(Z8?tNfh@^(B%BoFX;N$Djwpl?92AD>v9Vb zVris-TaZ_X=bw*?Gc&#DmEHX=F2Wxox*Q5>q7Y8bRDhT$dBY5*g`$CLE7HwhoLI<Q zI)hcMFC9Rq=lXtcsCXR1(=EL9nn^v?$dvVODH~fi^W8U6E(0C<7%i62wotXe4t48( zbyQYN6N3CRTdq1-);CS~+3VjrJi@o8jpE-9o&Ti8PR)=fpdNA{D4`<1;R1J6+rm%5 zsPy015Nx{_tcG<vKy69%)FK`kp*;V`Ip8r&99}8aX;F8DqVpYpN=YH}U~4OZCIdWj z{A<8XUjU!8`;=w2Bv9l#55yW(sdgNVMtX*IX`jWImam_;o}s{xg(8B_pI(<9NfY(m z!JfZxz4cIfJ}5p|x)1c*12CHj?>4-P0FL^EYh;q!0(-ASsK480=&b0ZkpPBp;IWE$ zB)02wFpkYGVU%e9?v#w8kb$Ey$8bD!?cKvrCQw{$`9?X^n9)n=G|ngnrTVT?wlDe? z%i9%upKX)1ySS@grE{8GQ^~vsD&`_Dx)Dh)Kv6GNpRR=_?T->n>nh5S{FHjDVyTgO zM4{wZaj(xq7xz7tT-&oaaW_|V6SXZQpSx9oN9YxELTj#HKhT@YftXZ1q}mzvwL7+a z(b_FzErNZ3y69xhq~r@;ePAvSPx8C~KxOeHUrn?W_BX5c)%Rc$W<3@mU58Z!Y3TAV z-87M<tS|(XG&eVU0LbHZ{zQvwr=03zYyNf1sk^W><sMG+`>Oe5a%-n{=BT3s2GC(P zbH(R}4vvwV0KuDE@3S#o*QVssLFY%7cWb^Uc4qdG?ol81@E)P}?<2U9|2jGNuREK3 z3~9S-dE=7*-PphW?_D2*&jvOWaU!Q?+3ThY!mGea12R5sd2e22&<{_(&oN3Yq+L?$ ziw>Ka2CvI0-o-KSnj_xa1j#-}f^F1AtCx;j8#B?VLom<_>W<}cXZr>59WM?I9D&x* zW8`$iIPA4Ern~GKTj_a(*;|PWsimXp*HWj}ffZJyCTei|;A{Rx@69KzzwC*Z=H*pd zMl|d6A1Lg8l~3sH9CT-}@CN0NO8;XS^5&X3HlnW(y<!D6r1OJ(1c|VAXL8++R$LC; zP$Bb=WfsU3-zrL#Z<O@n>+)Tb{`H{jgYV!ISI0?i-nTgeDA;yp{b01{a$s7pRdN** z<S30+Dy-(0$hEK7e`0~3-RS}TWZqW!U$c^%Pd_M%kF(i>WN3m<c-C#+b0=Rq4p#K@ zLhw|qN~Du%0sJCCME1JXh~rk3vu7b)<4-bdI}da=eo*tAb-1{^aoT)T5W{xm!ZkYs zr$CxfNKq6$@aQ7E#SX{{js9M9f-i2e#+D<zh?^uP)mhG?&4EvO$YL$<A<-buT&t5h z-l2DrBwG@BzO&&+%^tWDY0NF!)pq*JDU$75LKOu#&@S3@1;iO6L)={~afkhV5yv(U zl2Bd(5-zZdN(kmQMFIWM_NYkp8a-7cEbMk|opa8SqJ$rCecxV{>Zs;CgI8{GCI3!M z1Y{AO#04!;EIViUh}_Fr_x&SC^$_g<w{YFvZBDg7FE_Fw=7|eE;13inADK|NACbJb z%az>BGd|Lw9lu)Q-VYq2pMmaE#Ho(o$&I>3>)lBT8w@u{y2Fz5^Vk%W6Boo1C(5i$ z`-7YGt=)~=V;NDu3a1#TzDq!IKl|>#7OvBT*GVqg4@_aD27Cl7ADHY_c7h8bIr}d+ zCQl&vRw~$_V~D4qJ67D}Hzh9_(XF{9s?N_fgD8^_N3XD>dLo#^9^_`Z+Y8D#;ff}C z)YqGS4TDB4F2&8m=j~E1^<nSePd2G{tDtFSKd~GK>f%uc4r2wc(X~*v#$BQ7keGo$ zPFMehVGSVGVAl@NWM?&OhnSRErk9kx4nE28MV8-$R#7uX7b-lN_VT@QA!gOYhCV8F zZv}b=miq4I9fKhR8$uf^8gw>-WYsa(;e~a!H5gdq9!5Pxe=xPNWVn%!sJDxmrZxtm zM%!n~uJdO%zz0cPn^#M;zX)Ib&x@t$KR@x%1A#?|#N!sK6jl>QD&B7&t^YOkOnqQt za|@j1{_Ynb^6x+XA$rtMQFx9oZMQy=Pl!I;oo2%BH@kB;uYwxO2Y}7^(5f%f=897# zDaA+&J6}gM_^LAF!12c-!C_}fq$m8d28x4F#Hz<=Bv&A;5=l*A3N{!eTB3}{lmq$X z$nBu;$HnM+p#pVZoduF|uVa%1EjeEovfx_fYOaP1#sadNlH-+sG3t@&WDQ*N=ZKj8 zL?l>borF6T6w35(W5lzGIGVB>066=KW60zq#%!`AwF5p1yIi=urRe_V&W8mbYocIp ztsC00l}Ye8?63_&u+V0Ikb1}a?%VUUpS4-2K^nWqz9z1I$<AJ-k(^f$mDQ~m)&V(G zkaO*q?wVJPKwrAT_=7t-zytOy>A9+2cQr<+456!0?%?ZWe?P7;e?&&l_wqe=nqO** zny(Hw%$~Q!iKz*2n7lC*6MJhe7I14zya8O)ya|#jp<K758Yl7~g!Ec^NPrbMlRX1} z$%0t@0Mc^RDU=u&Vlw7TcG?p+(kQEmDQf*bzA6PONuI%ye<lVY=I%mGj_*s9n9+US zIr@s*DK}a?_F8LuoTxx+sd%K>GoVF(@3I%%J{5D-hnk(_`8jl%8d158a-Jyv0jh%d ziZ)gJq#)qGYhlOUf;_LMoF9#2N5-fX;TJlkMvJa5xxv<#ZId*|b#nu|U0?}i`wzR4 zuXLY>=3Qnfx?YXq<dC0Na$K{84i#@0H#|I;ErkZXU46FhRXU4?CmJcCcZY+FKYB{d zN;^)_K_2o1XQC44-NPQi57ZBuwH_Epzy#ngzY(H#W~=A5>l9(p6&*RFJ#dXk)NH4D z@9VIo?z2*<M}cCI39uHn2Y&0Jfir4}5yWTLerHO$FC3{iyZ~l#pkjns!d{rYixeg? zqF*%CKrG!`1oq=T5H<o+$jJAX%2O7`5;}x9<=fojoVM;lukOxodEE2{QQMAX!q@+z zRoTk?{2rH8YA0kEgX-W(m&XOww{SH<NkAJZqlDgkYg3S94ewX7eJfhRqi#D2@p_}o z^W{;|T&DvYOt;&hnYYNT<53G0`wS=~zoU5h?EcU0`@8THaf!05A<<V!r&#iQTlO=a zG5S_N*PX00{-~=@2H%<H=bk*7h@eUf=5a{%n`1E^&YwUgq;YQ?2!MWESty#H_>Jk8 zvtd8``Pp0Yvk%-u6vs$5<yp~?p}7Y{$QHsm-3mf_HYZ9SL^lA5qZlP30eAW5Xr^Xw zzm#zN#PEy0M8aXzaF4Ve?c4rhW@kRz$TkakdctP$R;u3q*I$u%=b}K?=uv8S@IQlj z!sCzSv%}08&hjxnt#2sgPvZ^<|2vP{Bnkc*(cAl*1M2?!JMe`RUDQqRkcRbeq9rC> zsz{b&YRxGquPZ+bvCk1o!l@M=T5X0{C>T5iFPMQT?D?QPfALY!DPf<M5A$q&2I)jn z>K(N)!bC#8Ldy;1c@;`gO5Oykz&Z<3<zZHcd`{@$&TGGW7G4F@ojh}2X?VQ^)SWeG z=v+v&EnV^tyVn^1B>AVPtelkT*2HLpMooMH-MvCAA>wEGqbJ)ltx|QA{^e3`b}n%X ztUR?-Wt7R?oBlhaQK{|VOp7}!u8yhMOVlTdEGesyr3({V1xJSeefOjBL#$hqILhFK zHeeXPw&z~A0~}v#C1E9|Vg(L2l$e3n>6R?J<Q#e0_fgtEX1v2yK+N1;qru9aYA6sw z0Ls?W!=-tBn9kG5x>{?scf<1NBQu_zo?0>gR-wpvMXuZjEvSwnf$~s%&XYoPNg*OU z2A!8uo8g8gm9NQ~s$Y|pZ2t<~DSHOBI{QnZx&nOnB<PN}TfuSg#BeaxWk9bkc1*|J zdE?g+R>GDkD&3$Q(d4To-L#oKiD@WHT!^?X#YaVd*_d(l?u98U?6m`}g-<XAOUoF= zofQ1Dy8WWfNL5h+d$jsIm%u+GIVHA9^vD{Z<#%z_@>Bk`8pzX6_M!>eh@Sa&tF2r7 zL>njl87LF;dcA%Y%ft$(VJ)?et3_fgj&7+_63?2uv$uzv&7aO35Kzm3X!mNs%`d^* zI1!F7C)7*r=n)?R5VhmLq8QxF%M(sr;H*Uk6dUtKW(l*r6(WPwkh*wtRRZmaFFTEF z{|+o5vSd*0YLRU05pQ}#ywPL$mUG%^q~q;MM?sh}<VLmSyU_s;dE^;&9I!mMMpb$x zbP2o8>s`dJT`S6D`76h)!ysnGOJlk`N8E_F(_~_Bb}Zo5fQsw<;T)5{EdNUYp1d2j zF#QeG^{J`dE7)Zwu@axVO3XraOP7sN(bGBXOzt3Zfhb<7dOC9FGoFiW)1Mb@2W;PX z#R;lAI`d0IEYkrqCKB<7?Z*#e!>NqQxb~24evXuJBqG6w^M8ql#p$c)F?2YAOo6_5 z5}jED9pnAEk?Co3c@OWkRKK>SfP;k}FViXB#?LzUeiE!WSk~R^wKQM!q1vUS;;o~7 zD!<;0m9d-jz<=OdW{R$5d28|2nTD^qe`?XGOSxw%Tx?ZBxoKG!msZo@r~BLn<~vwu z%+PJD^1qAr;-`=0Y&<lNylm@sUWFsO4+Q?XePQ+t{v5-Q`H2Sw{~p5_^D(DKo6f=M zotG_#BTp(|C!SDO*RY)(H-J!kGm2L!Xf$|wdeboMx3DmRhG3&k9=kvPta|dfWEH76 z4IK-jL>6Q9TnzcX8n{b8R<Y_++56=5$Xi4l^lh}SHKJN^JH+M+2*tJTBZYOUhayAY z^YgV+@pfa_&UK`LE~3i42F7=psBS9Wv#mIWk!(dHe|<UZvtj|(&PqTrA7{9t>`ep@ zC~eH?Y@OK6nH|x)enMEotVa{b5qAB+4}OFN6YMuCe>_gr<t`YC$_cpT4Q@6{?)Fy- zD^lBxTpqf~DI?NF!m*`I%X=`B&)zs><(jUrTc_Yk5h5eQ9e+UZ7rjT%i<DaRPmym# z)hBetkMh;fUzuCFv?^4Le2z-Da({0MNr%ore;aK<ka$iqMa=+&a~_2X|CFe=J9S*u zsaP(JK&pIZ@8qOaplFKUT)^IV6*yd<VHQNcx%yg9C@ePYiO^>=pJgX|P7w<whBUy< z;yW84A|WY(X=Ap$w;P7Kw^sOhZ*R1CVMS80;x$~%+xJ*S+nrZJ+Q{~~Gli8ExFWL_ zAYiyThnWw3rFekHmEzCprO)Y}eUT-z3dW?N<|j*b-uR_nYC)kU^p^qs$sAJfjdGIJ zCY`Y9LAN(cE$=hD#F}<|%0vIaD660PE3L^|B$KJ5!JX<o6^yVH(wXqnnlg61!HMOl zIBvA$Kw@!@k5o0@WG#Pal?gG1V$YUG-9gcKnAV%TMria*j=+XbA3iXSJk>7caj_JZ z+~+j&%SmcImI&0s1p^~|keE06-Iq&2##g^WpAh_F47k<ycObQxZvR+vK6_3A1hPzN z9SUbC(<?QK<T1|T4jl6_0J+!{7;%3fCT(PQQ&H<uDzlTRekQTWHCW2tOJ#;EdnDwZ z-iU2uqP|CwqbC1H2fO@y<(`~}E}7(Kk}|n}WUwdK$&xF>V86D2us=VjvV;I8DX{Pp zR@PXv>Yd3%R{)d1L6<b8a7U8f#g{Hcoc;FKF2OgW$(luKS0ORcXZP*+_=9rK!*=U^ zDxem1_)g3YK<1e8f*#zkc1O~{xX!TTa(**Y)dBJ4;Dj0V1GITn^;upkB362pn>R-a zK$c{opSpy7g?|3~lqpz$k_rEm<ay;N8+AR4wcl9g)4AZQ8C#rxAr<!J$U35QJgu#* zw{52L5-VSC=bZsAB#8u&`IDocIyTr>PJGMv0kMn4TpdSs!x`cKn77D%`fvdBian@< zsIdLNm$mO-lz8vb{JBs1%f&wUXXB=GrwE_6^79XrkA?iZf_Ku0nAgz#X@kbuP6MP> zRv;IUpHT8=iU7)J{rk-4NdJhwPm}Jcz}?Xv-MY&9!zNw>G1-rJzSrp3+ygC-^}+;o zqEMchqY6hl_l{DTmGL3zQ+aqDH#(4}YXYCpq!MWW2Z60TfA8CbZ<F?${C)U9rW3?B zC>F2NtW72i+bWynp-Ws?T4@%WT$p5xVtW(2L2`664i2Pv*={jZ`|e*S^$AEj1cF^2 z)<r3iaHxs_gJ*|<#zAM=ZK_X_$EjrsL6UN2u6v4Gt<jTNE#%rrp`W_Vmgn_ICFnr{ zxtM9l?X+XngVb_kG@%0HZ!~Y44rt~Ep-D&W`2lo@5=-F}-rj=co};n;+9`NU_3Ba} zt}Y&5VDZ3p>#-SfhyLWwVg2(K^0ct6IDB+wiexYm_Qy=Ob>B|#d?Bkn!>1pjvjJn_ zkQM>0The1&7=$iTK^L5$jFyE>jIwgTei6<2UhBiUU@N5(6I$#nEy6eoO1TcDnDS92 zYqEnWMLAQMpjj5Qhe#C&wKOFbE%lMoEw9DuR;*vP&L@<=4JP~=@HkE+GUq*IGjFsx z>f2n*p}pk9nNMKy=YC%+*>XaF=O;+rXIrj90=8bu9F91cpvqW%f4pGz2YL){tln@| zpnB|uvAhM(J4)0Z16*>in6G;k9^8BGuCm7zloN8Gig<OkhrPkrxz-t3Hg1OawB|Oz z)A!O?li3F6(CcS~W-&)U(7mXBmBJAyer7)p8!q`g<0rn!Fmyc8oK<bL;P;?(6@8}v z+lodZ{aU#U3y&I3u~wVPd9GqB;IDAzwTqSAMsEU14=$eo<<?vE(FJ+3^Xbr0uOL1# zA%JY%!$o^x7B3lK2=E-t->acmjvP*<oQa|2br$kUAR)@qQ;|WsYI51IQKfj-7pSsp z?)_wBu<b<VOS5-nbi8@Xiq;es^iheVcMm$>#Hfr)x1@I=R)T<&(GKwQPs<CY#1uqJ z7jX_;(6Uafef2r_7NJIDr5Ty`_9${G`z;A14^?7N)jJD;m3&?<gCBm|3%tw&1gs;L z`Plu&M)0nWSNAn}!ISP4TI2d3BVuryCy(6c^QPkc?jsa*D)d0lUA{ey8>6~`<XgB& z{lhRCrJP=^Di$L?7Q9Z&5tg}c7AaZ@s(L<Zqt@e{k}N(_nd{Ww&H3jc>vgO4SX{5J zI#4mEV~W<o2Gpsrvnw<3X8=pNE+1J>d*^6C%0J__jlb|e^Y`?0z~R61_mo~p7%gjC zY|!B?!U=H4j|JYY<27!e!NQ|oyc_eU1w($jTHkSdH9WkGwEovV_GrMPs>B6k>V;^2 zWG{?SpY}X9v*qCJa8zW9tg$gqw8R?{HVl>K?2^!J;{Hj7MX-`>B}%?{?`*@seXsM` zkc#t++Od~(3ab*?j73#iUUdrp=+|GZkQe%aFsf5Q@~-kf&ziw*lo)A4YCR)2@3?4R zXeZ{XCT~6KlUbMR+X(m%?)+#Yv1@vGUEZIYN?FZNB;XQ4cvegi9o$cH{c9KfyXLqA zruEkJ{TJVZM-;k}M!Uq)TR84z3%C_hW>$J_&k)o=F-*aOZl%5>axW|!VX_kB3&;tR z$lbQ3)DQph0_4<}52p^vjsZi{2G>u%$UZm+r~lS|U`}wkh|rwFtP-~VWhu+x!MjoY z_*y_$9ZtZe>;scX;2lAunLr4=>OS+S1xKiaarFTbkMv^M1)BdZE-v;4mtv7E{U?g& z%PDSCfs;n-?z5byxOwXFiP_8%YOKx%Spgn&=tpwsGp@2soNL%HA+5xLP<rn!2tgN@ z<8wTSw7cuOL016B{gq}oaFn-|twxE2TphT;PTliZR)mARSPT|E6@RrEKb7mB(J92d zoWC&gci4Dx&+q$v!u-%Bt%SiZY4mxUP|{JDdutNj$D{Q_-$Gfyw#UhZA>nAtElHw= zW>jgC5+DH;hIH4?y%uiTk9HTq-FMmlLitemw`<gV;Z?!Fp8LSDJWvm>r5-3~O{6F3 zQ1?QXNmiavx+b__2TkQx$7p`TcG=`_{|~hDxLLhkx_T#kRTKJiRMO}G!9O1)FSXOh z)(8}(cQG~^HJk;^<gccgypxf%>Wc``DPdWWb}j3al0nP_+jxmmy#UEdOxL!qF+nf` zL4ZMohR@XInp_!MTr8=8e*Q1S<1&5<$Gw-ZPM`$nRia_oXf#r_<DX03laL}k5*>zX zrsu2PON{Z9PC{hoOBapvqUfc{5)zewPFKNNs08K=+HXpN`L-B1ndZxraI!!;raCP) z{`Itt^!abuM7_sSw|7UxJ0MLMs#3FwhNKxS$=ERMz7e`LIivDuiborvVp#BmY*O-B zYIzQl5jqj}$-?;IPVA9VjCpUidOb=Y-)!rV<%85A3whv230GX7^76ao4vua8%W<}8 zAM8x=81j9ZU5`^>0t7lKxn+Z|Un8x)OyyT|NBiF6drp<(B+SlAk_Az&hgR?KRFk=4 z67#;5=V>)PBwKH&p(Iny!;~A@YkB7=_q>2UbzNSlHS(Khv%B$pE3u1@Xm$H-|HAen z<%{K+vM)&|$%`4vz!DXjc)<3*WBi}7zN-7*nLduWK)3=ZF{}f)(ThdJWxqA=yW%9K zTx(}-MsgnC3;4Z^yVQu`x&kmJ!g8M^P7q@a<I{tQ0sg-V^P9H+^sMc*f`b?zE~<qb zI3xA{X9i%n^OOfGrJvyf(3WE!%T76kAyXC#k#7o@EX)BUo3A87L<I^A0v@H_Eh`?S z<imADIIz?ynzd?`DW!vw_mM&}j$r5$Ly=TP2wMaxnDC&PXL{6N=re}XrMtCX{a*6! zJ-NBv^K$MM@;#@+L_0C>bnQ(n98LcH24mlhASYx3VF2R-w7Rn4e*3jXHUZT)+NI#D zaX5i*(~0gI=PE7GpCP}tyLd@u953C<Oi}S5n@t@5Kx*9fv+SE5n(}LvrGVjLh|{_M zQGGh(T=n|#)Wb(!v<R-M;0$Kz(s5}=o{Ft6B19l8X7H{b#R+IX$~Kv=keEl0X(w-| z`eg>e?DL896}DvFK0d8bh2kEowG=Og8IKDekAn^Ox3{bhSfNlW^3R$|htgZ*@Bu1i z>rn|r-eZuKJ&5BWnFLCe;`_3hmOc3!VVy7<14f3jpMm4AE1&0l+z)PU#?Jq^GLCZ2 z#5J>UoTb0%%G1a!JaNZj>+EiIT1fxWuSK;@2LGu3uKG5PlSAm<<}NrS8$8Fl+FVBm zA@0<+jc6od3g?@y-j1~*dx>_VNWnMjdTfz!F8r)54;9ZQi9y3e6!0)<10m)*pvYk1 za8&y^Y>s8ud7}SjI`kqpl@>ND=%jRewSXJ8#etBlUNfW>vncqkw!`s#B2LVX&(={v zVX~|GN%7v#^ICL{<h{d%xGb*cKWa6dk$7U#jBx2Bj+o=$ExU?dg4)g1T4Sh0Osz&> z^qnHR@)N#uwUT}vWvJw){2$IwAlRMJ0(x&Q0)*)Ho&@k$*4u5wr;*hyhYz(AHsYP< zwPVtJm-AL5d0gz@>&~M-JE8D1A6$Rq047!v=(*YOpyACL0WMl^T|RyajOy2`<W)wQ z)d)3%SRfW4%J*Cdkr&br!{-?2{4J_zNGED*+?b&D^hQ$AQ$i9xRw*A8z`-aXA$2D$ zFUwDJ>l6L|DDY7OrqAq)LMT*o7bFHnOSN;~e~=5GI+KODd^dID8e7B7XrbDC^(N?= z=4sAHnpP&h1OK7Q;Zrp$By=xUv;}u_N@$bGttgwy;5~1i$4ab4tlH{Yv&i@Mm@7~Y z8z)RH;>~`8V`clA)-Fs=eIRtpsdR<(ba+wFa@8C()b3<Puqs)U%C1sO3lQy;B`A4( zj&-PikTN_u;WxSi-1ZIV5)e!GZ#?I!r8vENy&^$kjky#k7y{x(q#@pYkzF>TdAv`H z%e3GRf7%9~n!i<t%%fy$6Ep4H>@{?N$wK}O>jB$;wDuf~$Mk0SVbA}E^Rdw|27wvp z|8PVBkU(pL&t?tV&h$Ld*B>Y#v?#`hVk+^d=ijdp2Hb<lS_qwD3jitt_p>G@hJirn zR$|Cc@uUIBk<hOLlk~4Ba#~V>1`aX@fQL%3hB*q|0`J>89Mn9Z=p~Pnq?&^NeruC; z({!WR>>cg27t(Z=Wad+QDH9!*v}c+FvbvN@RD=KZZh{IJ2tl5_c`@m4s?Hfvorz5A z*`K4}R8<?9K31)h`Cr|)fPr@09mxF(zE!awqT%f(Z8F;XG$?JgAuVAvKa<aR1s5vj zYOcOln-|UasMwKg`!nA|={f9;0LODcuTS7ohz)p$ATK|2SpWjR%!5kwQvm<>#sEKo zdv{+E+REz31xi>=_igr^40G|DWG<~VOX@=LpD2+ZJ;DY7VQ+hlKnb+QS^XvdyQj2# z`atX#95#2)p_0CDuKfh_vx@DJT=lBcQcUf2;FALKxDhSl4F9l`8I&_4Xc)P>u@M45 z*vlw?{@&e7C!TkD{8($DyL{C@?14LM&UEas$I9U$1rl{F;u6ZJ5e6Mcq)@dQ>i*SJ z+dBnPTXYu{4-80npBWXlc7N{5<H(r7-Y3nd_gOk4vt9mmdW5aDlU3e*f}J^Ms`j3t z2e)9acv(XceZj+|v<Z{Ke4~>EHCH7WPxSNq--5Z|Yzmjt`HP3<!Jz98%flXTJ1<gs zk=#|PXN{R@yxtW}A+!FovVySk`t$1bXEK?1zv~0e1?}CLb(#U|AnUX0`fNZE7@qNq zzKxGmp2uGNApr=x-x+h)<;0;JDm)7u=tp;MNdzd|KK>1JoGpKJGSM*KCY6IkoBRZj zcnKM%+c1E);ZM85>}Vk|+0mik`ao*OZa02Hi^|j7yTjMc>+Wo=*;&pSXz&j(>|k#R zbo@QV2i1{%grnmVX4WU`ndr@Kx-^L!o<(AB;}BxB2z3>XCcnV2So<58F`AEp%-ko< z6Pn?IeAlB}*b3fv1&S!FyI_^ulTV7?lZfUNlQint<+~89<nN}>>)$Y-{d>*jQK6k? zG508!fI@EZJ#Ptd{W0a<i^5mDWOfO_mt&H%3DUVc#wxyt@@@$f17t)!Iu}=hj0178 zuGE=f=!Zd-4ueo{p_#m$omFy6;aS=g%}+sRUv_6oWNa9c>iHvA8w?#NZb*^UA>HS1 z?spxDEZAl79+i<#5v_X6X7po4A#tL6cU{Z<belOmOCZn6O;Z1&u6KRSZHGd8a9tZK z9X6Rxq&Qt2kI<K7=*+8uJJOATC)n~qUZbA|7(GjPU#0Y?ndT2r_WKdqLcMBdJR>LU zr&oMrYuyvw`W}Kmk{-rZPIcNSoxF+*|L}F-KQ0}FaR?pX+QPpmf6SilLTvE)trVdD zv%-$Thz@ryx1JHY05FC^|0Hg(xCpACQLEI{R1csquRXEl{DuGdem9kb)QeswLYFj} zgo+pDVRq+a$F&S}`Sl9N+wroikS=4u4KQFS;X*G!tNcus>8od*3=8BTQhz~kRi}cM zgW<Zu#dgaCk3}nhi_X1<c|Yk4?mXM6<tob^@0ZvU$8~3RC-r`wdL{&W3T6U`u--F3 z16%jvP9^3m6D<FDG(A#TnZ{Noz0O39Pg9=Ne$igq&mYFx8VHMV?*8?p-#coe#(!%u zMeQ<^CmHBgwNP$VK7Tq&nfgTLC3nYq7iV(!H5gX|(&>xY#TT6R^wAH-H?jMW>MPba zs117I93bHEz()yL1yuk9VpQttQ$cjK*W5|Onf3K5y?&dtZ>l%EiLxz#ngj`k?`KiO zrC#7p)KI?8h}%T}yHy0h&u2nPRKwJZU5+$|`f`agCOKHMa0ZXP#EI`UMdKfXom#0r zp=xW}fy|WkS$Sad*^K^Tne#5<fwi9i?5CE-SP`qSP%2j8w=L8q9ZWw^oH;UOx(Cmp zKegpDD2~r)kz+qSyQ6}_F^vCuR8al&t>&~qL3OoiZehLA)p{fD`WEo(MA1;<_@A}p z>S0XXb>h-s;zA=B<ULuxC~6ho9rZ?|jKCyOx*Zx8t3b8ecp(LK`SpfEcv89hT@Zmt z)nGHzBP6op^tN4T(x7-KYBri%o+S;KB}vaVc^&O4d+x9VKkj;a3hqqhx84Un?Z{QG zAf9-4)xg3mNxn|jLbRvvfv+7->jeF1(&nXp39p{*Ge>&SWN$qMq+QA;kG#XLd^Q5x z*$Nnb<u;oyiDh{KuNpC%qn_&Tu${%4{iE*<R>Kxc=I+(6JG!{%y&R7<$L?;pQzcwz zxD0Bg9yF2u-f)*gNtMv?IcK{GTji6+sV!*r%$dZAGh<X6q2mt1k2vIa?kjT?If{_? z_{5T6VGEz~vTijSlQoTqC+aFPPe<+pxLA9kjE2gN)0iPaEqR>(xJBYM`6jql3DXMW zWZHk+1c_&gd8BSG<Il@(aI4W_3g647&+9ENCCv;h4JLb442=~vhLZ3$Dmd)h(ol1l z1?ur=Ud0L54fVZCXS;mj9KNe4mym@P@x>M$ii5R0r0S{I>XK?oE-FuutB!z*I2UD_ z4?kfz>m=^;E0o+;b0Q0yCROeVm1256Mw&ds`yw=l>)p6=sk))l#!Itx{UQs_;@DB) znqOiRNK11gYF|v+S&V4rFIS2ilfNe(2JWpHL<hKZ8GT5wdq}#B+T<#K6YaVv+!>oQ zVES^79DE+2Qu=+7h7XK<<=if6E#1U~X&&4SzC6QanwmAg^e>Fl_>b{c{u?Q~^4y2! zuiXj%T5$(xV3%j}$XNK6J=tGBfS&Zl`1JF?xB|B-bqZZf#RUvo&}c~A1%7%l()#?S zh*>pM*~^6a7_cV2bYeXrx@IT?UkLiYhb)ts1jv+QWT|(es!1)pJp>o-EA=}mGCkd1 z8252I#~s~7M1h@s#s_nIGp8J=(Owst$L0(a^686;hdK{x#7HCw5GCa;1jT0Sf&rm{ z8uCych5p26)MTxHGPSW>CN`oKVLHZOT@Y4GX*gEv+|F7{kUf4eyy$iXG^zYff?>T* zwek*06^`hynthLFT*oWP@=ffpuWLVGfB&3k$?v?)m4FDA9*j%)xGSR5F%cfij}05L z6-~NEyQf*-2XJZ_1qRx0v<%NI!%C0LuOgf*gL4U2O6AYf*gV*}MM&-?9EF}K<;W^V z0**$i+g=$8)d4t`J#lWY9&N~(eWpfXGaH`}C1j#xC1EOaUpi?|9Pjlf-;yHh<@7mj zem}-<?D^m3L+O&7#NV69Mc1<}dSf))!Ez4mL_b$*(3m+=&B9@ppgh`>pILdKj-UkJ zh?+Fs1ERS`=Aa5Q&r7!T?OEQ_GX@aUHMFbp`Y*owsy*XEhoz=u@Fxi%Ods~%sV|{b zt#2V-X$!fNCfW>}+t+B%*M{&+D4@s_lv+riwJb^b|IB@Xot>t#4lt%|@@a8nQ({>7 zD9^b#cjs$HV?H>|5@&pN)-zM@MRMuzrd-l_0;2)>ef3X>K)SixUKdBj)cV29;~VrY zQs@e6=V!@zPkGfd<L^5r9y(u#BJiT@kh0=+b22rKGkp0D-1%8;TQ^#v=*K}`8RrX7 z(}W~WdVa>L^`la6xP$(LaDTji@UP?B+sAZ%sXjx@z1D<!-}y~kM07|}ecN{a8VLQr z0V$`%=b#_SCpB$HDxiSc*<XzL6hT?O-EX2^AS@ViUMWYspcBX5iuu}Z`l2;)s-}~8 zA7sTVlgl1YE^f*vKxJpzE!vqjIzT%2Q_)`$Ey7$2uXG0*hmScJ4IW6j7<*aLGzp-z zi%-nFI8f@FR8DFoV9Rq;J~`={Pcth!o%P=ww9IyL!PXvp)keWu{U<oh`bm`I!0ZbQ zKoi!Sd%WK$=3X!B{GIp3a!|4`l;LjgMY4~|_23s>vFg&+L3AtqSwm{adW!4GUr2$K z<{5jXG*E_93qIJA2CkMK*CIgot*)7;CQ1$6_+b$DNv)fM`U`0m*Otg>KOpz?q6(KZ zLs0f44vsK&!yx_cNA{ZL0S$DFkY$bI`ia}T9x0^Gqk(+|WzD*^bbMg&o>^6C@8o|m zJf?=3dO*KpR)-bR{<Q?s8GvpqJP~VVDnp<C(&GKoKLI34$?v4Y?n_8Y(l<P}w6yeC zik5DBsdV;(1P)(U<9!2MH?y%*Q!?vP%M87`pw?Gb6q3e@cQvMGGz0#ygw$<5b!on4 zn}5wHUn6u)Nyi=d3Uqfa+ADC048WVZNqVk<!qbs9LX%=t)cf$h5UVR$s*RBd1>QRL zY)cv#H>r3v{*hsQg!d;2uSs);@B^U|CJXu#EGqzj)>~L&BYfuJ@f(lmllglW<LYYu zal87o!htwUP@UcPgUKp6;r2Q|I`DL_Yae_jA2#{PI{giYQDJ4UvHglh^6W{zU5fz- zV|Y2<FwayVfFs`MquG0*?wm#*WS$v-erRUH@tAg?L7xTIufp;K<y-xLGF*r>u9VOA z2f*2O9~O7^j1(i<Wprl+APxpUN)sAN-cMuA0I2Y0{3=~E_cN-<%UJmx8N#-EFm3#8 ztvB5~fpk1uXTtq~r>_V<zh9Dwq3fs_0#?fQF8l`GIB8$rR-OuJLtWY}{v^`xxvY)u zwS}|V<fpQ<5_I7en{OddC-Q=^xECkm^U^%88)6W|x!KHD=YK=FWI0}~aJ-#48|lN= zdJ9TOh`kA~R0@f7+kDFzB+cmQ=IZw5(r@rf#aZ1ekI5lkMD1=RYl?5T>)uqvi@jf# zW{az%#R&COX#>z;Qzp&4OeraE3kn=`nutzT;k_huFSgN&l|-NY86lCpL2N<iOT6uy zfSm=mCcy&t%Ak%JL&W10M2?iW0EU~JV6uH?y?tsrBz^zSut0GY0Kw?n4q}gaAFBzb zsnw68)Q-)Qk9upZy#jp(g0yhuU58r7^3~Q{@p;Vv3{2AP<U%k&jDJTJ>@3iW8aWP| zVtdyNcCDCuGI<V8eJuslGTi}aM93cCXt{jix(8$=LI&HE+Lm}_rDee+&taw{QcZVi zlAp8lu~ONI@LfZh(<+Ya%R$ANdm>8P%qS5?<j;nMqGgPTnn%{`<6`zgLL-5U#J{;k zu@d8%bnh6Zv$qL0CZfxE^>_JwbMT_$N&#HD=G8$0?MMlt$U*w4aNh4%znU^nb*ZnS zs+#N$Vx+9pYD`Falj`sOi2LXVElZbU=40U#y+x_V=ae5Qw#ziI_KAxA(S_NBi{P=@ zZX!Siq8Fou+`Z*Ke3JMZ%RJ^fCIF{<QsaDZdr~Z?R>D0X*UzO2bBw+-yF~PpK{tW! zN9eN5m^nzA^S-qpj|(pvGh5WXqB=oZqOQVHtZy2W1<A)$QXWn`<8fYbwBl#G@%x*V zg|v&YIou@T3sdTP^9$u3MOxblC8B*9Lc#vwHdy!M^S?j;|9l84>;>aUmbv@V10?=c zGBRjN%xlYxa@PP!a_$gc?KtW*9_c&lGg*3f;*U#0aW`NH;nd>}-!yl&;Zz+yk@RN9 z1k+}{OvCDxyp{{ZZ)uHd^DBR_e&DT<Bs%@!4;~0eTa|ifIjM8`@ky@UYdmP#)d{?c zPyo=<QYw)WG1W`8ZNUmg@V#63-Qu!kGHJ(^=k$+|HIMcr(B<q(wW00)X4WH{7AuC; zH}Zaq$CwA+vyj8?AZD;6j`Oy@PPJVO-xH_8DT%S~6_an>vzmD_nl9$M4|ndgE%YqQ zwUfX<hA%A0Z*t)WTCxaBCtaka`e${QTyHZe?iWy2&t^@nWY7+MnTlb7tR85Peb8y% zuo%UE@80_Ti5dFakr(#_dzQl``~TQ_>!_v!_idPvPDMH`K)O4Ih>Fr74blP|FvjQ@ zB7!v1T}pS14(UcDMh_U>9i!g)i|_M1=e(T5AKU)oaCU#L`?{}4YK^@~uW~%LADcRR zhM#kWxuey78I@&fdtuYyUpg6X$y)X5dJ-!$TVnN!-)12fb9^s;bw}j8Kc2oGP6>Fq zrdE0%C5thbEgbmiIjnr)H+fJlXDxMe+_wL9r~H!q^z5Ru*e(6Rh4GE~rl+f@PSe9z zWv?P$V#<RWD=jK<e~6!+|2`?!`)$H6M}9ITkDK4z${gtdr4`Y!*B4x@`E^}X`e*J; zYIA3ng5gc%m%Q(>*`azDdfDz<TATh`)}I-!2utkIR7^)#qKC)mVNkfELtuH^y~yRt z{X7xt;>9NkkNouI1^Fv%o*hG6GpS14MUhh0o+t+x_rYH9ZJtQ`*tPDmWt}4|)vqqw zhTu!j<wfYf-ZL70Q_|eGQQypNxAiJ8RV7Z9_p#^LO3puN(oVY}giDyhB8MUBJ+oKW zvKj942PwW)HhkkUKW8qEZ+$=S|FnOxW>WKHqFlsn$Iv9M`bc=VwR0ozaZ)T=#UXH& zV2^19Ia%U;UWH}4r|)|b(NpKP>(iE(Ufx<?J3`>sGPPfY;<FijQJQ+XET7a_!(Z&E zwOE6WtOH+N^9^HF1$rD&Oe$U-I#2>K1Qi**am&MkqF%zjP^w_DKa9vdd~Es&FXk1j zRQ(}=t>k4-ZEg_wbJI)8Ph6O>p&JDIs{@OC1rwaK)$F8W6P$G1mL?M^N!IltR|fyY zga6xgA2Uk-V9wTm`4^^WokGACx2(wEZ@4)V@eD&+!i|>Xl`*dc8#v%-2tD{<))~v1 zlb~w|s$7lftoSE%3J>fRyi+^lPZ@m)SvWJYxzR`XnECM-b_R3#?4H(qrcCw!JY><9 z5bX92-MyRqxwYdC7d3(jBz&Mx$cFm$a?;aI&-%+xIl_j7Y8xCjQl6GjfF-xgd+6`G z!wZf36koFvwLt7A+6A*k>SOw)@%e)VJ$u!`z2c_%rRT<O-JgkVn;P<KmFSQd>wt?R z;(3yf>i=WPWmXJWh9edPn-kvtA6IS#jz07<^KL71Y1IGjBrk06n{TG?xj{Se;Qr*| zBDwZab_ZTLHTTE%WZMkM_A*6s$p?<DS2h?W6Bz}MS>+`4VL<nObHW3T+?X1n<B#G} z?h+-`A$XW5u`k-$il202v^gh|T#`i3ow;W}x)!t$0$AjIUC8Ql`S-AA=&*GDro9rm z^wfM|P^ivbzXd$}U7m7u;w>4HefEn2>x|hY9~|lSH?)p?w#VVsrx4eyh2NvuxaU4V ztzA3vg6mU?nGw{&mGG)K(Os9f=<U%}riC>8h3-iAMJ*fGg~Iz6riQW;Mb`fw{f?pQ zPMV}&SH#}<GtfR^`*yCr!7zf**nMxwM}j$rE5cEt+1qu3t(})(P_W1nWMu6nsO(qk z`$DgXy{<0;k6^&s%(8#S)ItdUsQDBB;~mVTSedg8nxY@h80-!gJI6SO{wr}D6%E@+ z68u~w=kD&_X|#Au<FOj^);^EPb;4ER_Vyqks&zwV^PoIY+;1)JyE9UELjkHvaViJL zN9&zWEG-2b3MMF&1@=!eBpC-?)5sB+d0k@DNjI{NcuTiCxI2A=C!j-i1cdH1g1;9} z(rlWXuopNc5l2vQD0XfNLfaeiX}vPNS+d+;h^V)cw7MN=U9PAjKtIUl?0~?pf$S>Z zoZ7CKD?DxuJvP#j!ItEo14o-%h<YMR0T6Axg!00UpPE=Z;yTN>NCT2K&#L@I-(OnQ zh#3*j{Pd-A|66XJf@X<N2i$wO|CLsbG;Y7({df4B8tMF#QTMr(#y2k@N)dSw%ECtZ zPX#fdw)q=RNm9IK@oIO)TXuMsx0br;o4L^qwiE*+CIPNTFgU8z+imMQ^8AUegsE4X zO^NYAPGxY<%04ei%B1~%&(fTDZ;umjUKuG|3rR#%?zvq%h4Gg*f^QEm^oKX6g<&!d zQ+Zq+oiu<*yzF^j!WbY4<`_{Pz_P%zg7uLrn%XEx^rO3YB#4L=;w*!ovC3A0-^`0^ z3`N^znGY9b;a!ThC%nO}c`4@h{8E1|U!7z}##O9Khh@Dos?BKGvN;(23Pp=d`%w7& z!-mEX-9PgnB(aC|F1$R4iBcHx%Oe+Vl%<OJSb}WeJ^f89NF6um;ei`rS=5FPkN7Tb zw;z-RF&oJ`Z2YxdckqN<$)WxYg(^>c?fKzV{?=TgKaV#pZS~Zxp#eon#m5f>1$)Cs z6+*sbXTFU#gOuK?lz_#Wx&}*OtFVJ^SPWAcD0M$LND()ug8fymVL>h7&)Qe;<|>zs zKxi>0Ne>l)=Q#K^`h{%6`V&@aUzVcx!w0+&;181;bWL~g5B4dEob#VpV?blvtZ+qB z?7Evn@s})&#~17Ka;qkBWJCYUQT}Ti)kwi`4t%Nu^BR~J9G&ov|J4HgcMa*ca2$_! z=l>8UrUKz@DiD1bEKhoW?sWfA8fMh9(<^@mE^QR@Ux19tR%~RP*&DXfvRK`%a>x_O zU;d^LGjaK=%z+87IsG;J1bM^2Kz!M5@lOUBa2Q|ALP2I`&5h4RiOXiBz)<En@cW~e z#54}8bgDL&L?ig??ge(=|2l@{0pxqw21M->i($;~DXCiz*bHxG^fZ$cqdZD!=Ho<i zaX$A@N)46C7VM4#7^D&59liTri1Kql6Qejmdl<QOFq@jpL^*chPvZ8^>fv`O*UvAt zA(?TR_wwEw*m=6(v!i<>c=hR1PNj+`S=jM((hQ6YBt4EI-#&-)R(5qIjV8U$9iy`l z7wplevJ4Mxv2=P&{rJI8lD~shJWbaL2a@i|)^17O8$<I=UxhA|S=^D{w(CRwBWcO* z$w*mqc&aNcv`_#~JnYwY6PFK1R6bEzG%8{HL!KZZu3f{CI<0g8&#*bU4F4z|+Mo+p z?=op%4whDRn*$o39!&Y}VNDXZ3E<V<jWXQnl8xP4X(-sOOOxc^))eM&!2+g_YNW07 z7#a}h3eTN&euOlDr~!VSvpWGu$P>T<31_{vu2}_ksEMoC6$|2^W4m%ex&g~M&o6a< zXNqFMczJ}iL6+P{Wlud#;$bfng!FF&_gE5lj-Ymch*m*ip`Do1rvuQ@1au>(geTBK z5&J6WtWVu6V#V+!qf0iC3{7%$w}0Zqr1`{83IVTuTDreYIQ}xfWWC(p?KF}<E{Znt zqYLGHt6VPkZVV|J3h+C6v`WkWCJ&{rVjxM~dEZRu`6V4;!aT@zco~E2b_30>Tv#G| zLaUN5yO-lP{|W~E_;IG{+foI~yo0=uyR2eqfUbQIHei}+;s55oA0chyM+!txU$h-L zlU(Z8xxJ1k9jp)TW<&dE2S-iHM%hJWvHINRr%61<L#M1q@;+y>X)=r|G%=9{3-|Tn zfuY_SiIs7COI_f^>EeBN(0N^BgH03iZ|&$-v0Y2f_59@%c@ERZGz2MY@Z$3Yw^FiH z#WHUW=;-imS@yo<T1)G&e&by_N7;f^TjPQM{>T`{i>>oV^vXKsc+mypj~9C~jh@lb zZ2sHX`1uAWbzC9jV?S({Sg2@*AH>GMAV%F$iI;4gtz4T%)}9LyNAQ$@6&fWpbd96m zGfV3wseZ$joC%2Slj4fXPE(k!rO;u`XNk-NXE#f<=FTolMSYDdI}{wgcwUt^*TY<8 zH1+Au$B$iO@~xyL1N?P&P?X6a#Qdt@m+m<|dc|qr=Pbiv<(}6=+j(8yRd9ahR+T4q zpo|;et;Pn+E#Hb>8Z@9f{r$=>GM9UhYpi9dF<;TB<_@v#@WYfY6X8j(t^+X?E61Hg zdt1PF<(7(9z6NB~NER&z$(J)&E&^^uihVk^O!|7?w3OCS1w6euJ4}QIBX<s=mM*Cc zc&Y-fX?~jCW$nDRCgOgwJS@HB%+J%Ff91J;+&l8K3LG4fU%c)jITlqfRS2KHL^a-l z)cnH)R0T(TS?46Vu56oeOtH&hdhOgx)Q7}CdSW2Rdx6(-!iWDqA5fOI@6{nAQym)% zud@HObv=6<siFe68_&Z_j&?=Fs>^-<9Z0}9mt*-B(>}?J$@R`yw=Hize*aH@+*5TI zFs0XSE9>YR;Y(7Q4<mu?x=<RQwx0ob;u8mFnQVSl9h<w&DgLfPGhH`e30<%?lH*qg zWvvQQ8ThOXy)McS@=%f=w<dAi?lGSFG44+lVlJGpihCUJ+5AHiKT&3kxhj`sEViNC zJ5^uJxu|1HQQWkBXVJQQdm3@U40h*_1otUE>ID8SbK1|+$unvSKcq{$oSR497oxYf zh3z;KpCl=KQ&uq?kOSP($aUf2xT0)PZ8@qBg}W|a&w&1_G@rIlxQnib!BtT-36)bc zoSMro)OIP1xK%3OoLzIi3zy!QtVF?3F7#L3RB$;e_#))V$n$d&onT@%q}-ONm`}+* z;|dJQw5h&V-9MV^7G$4RwZQ+qv2%(o7Q}~@)?coA0rje%emLbW_>G-yZ_)dJ%oON0 zu*h7IBNnwGZ_*H87fd=$FP`cCbQaF5psd{ZQut^V$7TFuZbd!z=e}SY#k`U}%pm~1 zi#soVt6fv5vP{9@)}4LJ{Y9j2$4qtV!DHGf$Ea7s`+YS#Au1bus`T-Ukk6Mc_<w(y zncEyOGo9yq_N}F6jOLABwmrFCu3^TvSWi*9yg(c*&71$i_F+04?0&k=H;P)=g4QoL z=<358U!H(31~7k$iT_On0?SI<uB30$vMV?jyh+g(rdTe~Xb^ahv8upX*Yv>ES4Ag( zR;3%yrbG}qE3o0lrx_7wKiiZ*a4X{gyy&ccf_uREoL;jMqX{QuB$)G6oK70sU@{A# zIx)tK4_S&CyC$TTZ|1n3F86|u=v9g10yVuiXlmGqwE@?1kty)b*ZueCNDJxJwnYVE z1D&~JICvyoMy8|ph_bxd9qwz8>@cL;#_iJftRKC#9etmH!d`8;z;7{tXtz8fh&EAp zLoH|1@qD<kh%y%BsrOlncf&Ah^PTZ+0n1#gb~2~)zywDFFp5Q0ATNRpTqQn0^&v=H z(%`d+u*5cd4Yx=6RCi&o2z#Dj{9HILKRX#K1Wlcn2%;IOhp6$Y??ILfX8dN}Dj`kU zp@{mJulPbd2RKf19lci7dtvylC!&!^ZC^gOw=>Ht@>R?^jz(B|apNtQIha%<Po6DX zT4W-^Z|+!X3HdlMXNr0x<yr5#1pVBaR~OxENfVguPQn3R6r+4lP@B**_h=fSV1G;- zuL><+G;}k-?z8&J{qv<en{u=Wm$uX62?o57?RJ}StUgL1*$*?(k{TeL7O<;UwJKHC zo>!rUxcHn%oh-QsAL2#eAz1VyC>({fmp8t72j^h_+FYwZ&jMIileka&pON8#hZz(B z!)5`c7@PG~RMJ7fe?G{e_PD#+Fak7VqFdi!q1mC6Q&TILLgAipdf^&f7i=r8w81s! z5y_3ET7;=XSs1xw@b6*ev)>4+GJyV+<rUJ`XUg(GA+-?)j#WkWF8ACdQS(Q!dRocg z@iMfi_Z(1C5#!w+kIM$3&<nQqUxSf$ID)hEN8ETid!>xJur25ftAlzAe3@lbGa(eO z2bW67By!O4q|XSmQZk67Kk;3M_dVv$*g!}79GWooBvP!uy@GiW8lB;$0)Jwr$2I}t z-gQYx+)}vq*~Q$0?QM&I?O-wZ+uyxR_hC5g4Q>VC50~V&<ePw5&di9=x|NK<4-bGo zfFM~{e##}AtjTWjNaXesfl=0LlMP@tsmc8MG+(cZk%qdszw1m}?s_))B`sSpnM!Vt z-xw`p2;w88_Fm`i4|8zs<t4|3NkBEZ46nYwN}?lK1@@1Lvqa&$oYAz)$2R`PAu45v zwQgR_@F0>BT0qz>TfEgrHejuG1d&uC{<KtBTw%Iow??|CEiViB#RLxC-r`%X9(8K` z&Bck=rPoOyuyVp;rqvUDD&9mMgy#@Anj*wnKKM-~TwttMDXUd5*ny)Z?Cplio77`c zKB*eUk%hMO2W;gIaE)&r$4*b<w?ckov;abcbW_a^MUS~@%^ziSSJst{E=^Y#@v_KU zJ>Y8Od84|<RI^i06CeNbr&fE7Qbhe;?HZl916ix#)SvnC&bk?v#DjWTuLM2uI(hJ< zR+`W`{SA@|k%Nq8j(Pv9<~ouS5KThyZ5VF>{j66$)@-JWdmKF&!R#`$-bJd-q>YLm z9#mXyRJAm3baKf5Gx%#!^`V^G_bvFL%Ok9^X1$u3>OquM6Ea5(b>j=*<KG!NORq?r za=w;F*V;Od&CrT8dGx9=gJ&%DvYjsW0uPeX2CYlicCmpsBWpoKMP%FVpgFqrM-V77 zN;0K*v=0e=eKWI-ZFL2>Uo@h^AdcS{FwoC!O*n$zkzK`y6<(I+AZzilds2M@g_9K& zpq(bCX^<^R+y3_~Sq_3t2etg&|9mG&L6ecwq*tvpT${pH#fp=EfK?}YsK?!qq_yq{ z8^MzX-bgcC#Csm+8`p0AcRcT)1K+d6WB}sB2Xt=_Sj%;zaoOFlDR=W615C)|dc&8B zcCVk_b&V}{`>)n>pl#=(3Op+Wmr(V0RJ|7_OQ-^9Q1yEBqR{2wndbu2kDY7`Livs? zJK`YsN$VovUWFPBVUx67V7Yv-t-_BktS1jIp~KNg__!Z$mAClGv~+N!#3l<G6b+s6 zSM`k*<{qzB2K2cgH*Fc0J~DH}yqKM!Snw5faQ#c^qVsW^(;Fzdd@4)+HLXUih}TLU z!c-^5*@@f7v~fQAU$f$W(Z;mU<LTM+Mx&mBE7Sjc04Q&dd#UYwxg0Ex{3$F-A$v;s z{{8z$vAZi_{5{(0vebr|WH6QZiHV7&B+E!UY!RyH`Es+iF5o%!(PM#tZYGIj#1H=g z?mvyh(^!4;oy}5VC>Fkt)tWB4F(5bW&ueb0orRi|c|-uXBxOn9eLWlQOquj!$1jwV zxLqBlbAi7xr8hI(*af)&QYwk3QK_U<#B2%s!OY<aqPZJZAs=rXrF3}b-db<EOXe07 zHj~O|dP?G!s)}jpg=p8j?3_KN7q^6L0E4FwD|drmDaJZl%uo6?T~?42Qd;(<ce>Dk z=;g%hlrMCn6SUcROo{wA(B%bh3Ce@7U_>c({h=GghnR^mX6raFqA&rk<mQ?x*wOzX zkRj#9lsUPzQRTS3WX{+1vfStuYvg)WLWe&elSEZem%EIMc;w>6JJh*C9lo<=lySpw zhVs-99D`%1oH2R)r?9~=BV%ld@x>0;OyMYCPsgu=!35sUkcZC%{BAr*w{;^pMv=)H zW~6t+RwX4y<3J0k_ztsHFdcsF9DZTIUi6;k_s}99E-s-y4Yb9V@Gzyv31Uyhb7{Kc zBRI9(R<K?>eu={S+6)&A9!L5!Q0ei$fqK%or;QO@@99(5)XGr_DGWC~ot&nc+4I6g zI7hyE*eweEoF+-&pL+J7%->NK=5yufy?p9B%cj>tNv->mr*zPA;EUA!^*&eltqUHp zRcB+dNc!)G3)d^|O}k;ou6b7{IH8w|z5EvTVCdnDs-<H$%iJzG^=8;??{{y$6SZC1 zRbTCiFgMqeVRqBPyA3G{Ss$6)Zjv0}dWqvfugM8i(8y<f+h4gE(7BE$*Vxc?oALPK z7j@%-G<>$#vw=P$zVVIoHD4mNx6YiP{C!W@(JnLZ{i64h9RVs}=fIM;q4B^w?KY*% z1<KQa%Ie{9lw6pPF*EK7sxM?oOiKN_;4*hwEpx{L*k5S!3>O#2N+t(~IGpdEr81#P zT!W?`yhf$m#*oH7VMj0Eys`Th1<JObag6btq-J}bYI3G+&X(F5ra*?9FOgMj>~?p< zULWcZm{L27w-j4TwI<1?|L9NCpHb-UgmQOburT{^S_nhfza8dbyuuuv%=7!Y<X~{G z0C{r(I{h+zr&AfbMENXWcVjjq4%q}tDnf$S8@K)K=a+l9YmsGtoVCunuobLTUBMjo za=qz0%2!cL8^*qRe74(qaphoC6+>#FT51wUlu??my76Taki7*6RJRy@$L8pT)nE;9 z#fdH?W~Q^l?IjleF=VrY^CT5uIJes@MVQ>aD?uNk+XC8lP$f*>t(xSpm#2%5H`L?n z=NYxX1y@n*pdY^Fpq}C|qKAMS0)=E?LswO6nMcI`nGEgR71o?=2hg=4H={QP|JQm_ z0euOWi9~wslxB^$c%5}@j^&PJ2TDGp(ffswcu}^YC(EC**5O4LRvA>=m|q?)XYCHS z{<EmmAIL3Yi<j-)H8|r*e5xc;J4buriM_xBVp&Nsz5kRw9H2FZkO&Sl8DuA79sC5_ z`c%e*>o)ITt8!iN5|`yxg`Td#Su<^@b|QnshY8=Yk%@Hb_qbJ}viyu2+CVl01gS8d z`egYVgYo!CxM|rnYCrm_!%FfLkHJfpb5)I`7LAptp4R8fmiDphX%Q4VSR&@qw4~7} z%oFo=RD?i>GHxKX$7t$f1an>@`DTyO2Ck-CF;)n%_Ka1xwk5$^h<Q2bcM{5s;&kKS zKI8nsP@1*gGJ&Wv&{!mI$S?aR>wvfgsmr7(uT;<69>=k=h}$x(h<R~A8!p%l28Mm1 zBObYI7@}`5O_dv@jCd*bdWP?s3$c;A-a0(Uw+dC?1xrvC_P42A=w3q`=fZ<{ebfx1 zoEp|>nvZ%IlTK%zTWP(VhfI}CI|ai7?4CI$2Y)Bo5J{aW%weL~13&gksO>EJ#-wF3 z7{mOBd?u8%h)al*e=w__Z%~kdn3Kohwd1V)<DKK-C2*yru**ZL1pY3H-nNj*2K>`q z{YYAV`jD!Ua8}&Un_n@qLu^<Y7q7zrd_GChaUIBXf^0MY+e{tco9xO^p6Nw6;LY>< zG#+|6XJlr1{6&h#c#?8j=SM%LahD{6ci0NtJ%Yq|jNHi4JZ1-_#v#@6KKOBB4dZSm zhfBft9j&_FaF1`etdJGIX&-^#&6<AJDE`KCRZ!~GoWw18Dut#4AdS|nYmy5X0V&io zl{Q`}*qLSiC26}C!8ICz>e!f=b~t`Py(A)Aeqv(iUXZru_hDiU&1}ZY-!s21Z86=- z;S$TX@bj1gdBv8|u=kxt=^Ly2<Nd}pHA_)@rbmV@!W@HST>)U!KC|n!Q>tA3kh|xo zf;?m+REkscXco8c5v;k8^{WO!m>(@IBVZjCYd&shHTvsX?69bSpZs<I@T20{vM=w& z4r9~^^lpD({pkR1oGQyGVj=TzSm&mh3A_+jnY<4;uD{46m}}7HJLzx`420wQ@9b6# z`!G+rJL0R);G>iI91X_c6*JcJD==s9K+(}NHP)5(!7{-Qa5H%Nws1t?w)^}QTOEDx zL1D(v9zI7|@U?lUndXVXh=<dBS+Gp>*ZaO?XFSr8N{%6?95mCKgP0#@uMK+Ug=4>< z=_vv~Cvoo=>5x^h-F|;-<^vy<vl+DJykxan@dc|t6Nj0X9f|tP#Kjg<ottkDJs=N1 z_Dgi^15BlJ4O78n+}^aPG`~WQt0ueEmv5lz*TsR`#8kcgn~XMrqzb(O4!P*Q{?nZz zBHNBWzb6?R(?3VZut|C+`(2o1Vc|=x0$Z#?w`&-l<Nui|FPE@ac^zbYo&JaM__U+D zgdse7FyRS{Jndoptsy3xNU2ASa(9=O|Ab;av&L9fMXp0hf)YC=)hRn0Oa<Z$C{_Wk zPdH(c&Q+BU?_qkMo^gM|Wot@dXFaTyjDEk@!17*qaOM?{iy4Tb_APeb<MOR6_*NAj zU=r<08L<WPpmXMg>80Y{N_nIJSD7e3?r8*oB#VJ@303$we&JOL`N<xO>?8$VSqtnZ z;3VYiY&PQuKK-Q?Zy^91z{t_typ<cRat7Y{YdumMw7DusiBf}yfLDhCIWMP02v$0J zr9ygo{MYKaqg3M-G%+a)eT6;bBhx#W?S$?P0oL3Xt1_(EGEjC4W*aF>w%|+g5gT%x zgK87BJDVi|;7vB<@wYk|-og74B}yKGzGhWH>YY<Z1xFL{%Vl<&*Z4+eqBCkIQ|yif ztsO^3<98#I_p3?2D-}QgroT7lqirSyZch}}@4@E|G3-XRM%HPUDk7lCwTCx<H+0~y z2*WmP{9cGPlP6`?$A5Hj4}4%6ob6*RI<-!$zYvX!865O~Shn0=aB`1k6$u5Bt_heX z9@S545LP{*Z7n0X<*9l;?!R*^d2@+MJY1~0QeZ^NXVLn3zAmSskJjvaS`DPIs@wan zLywJ-1mW={OED|%qlIy@=-Xi<>CQWabEpybwhk1E=0T?QxZSSS0#W`M3(SAyj+9D| z?+v2ry|L&gGp4aZ1is^hKIak#m04$TSjr=Ezl$&0hZj5djlUsW0x#u1r5+iU8fTC+ zo;@RXW)vmZ${3Up+4EEdrK}4rP#EFn)CdeSkKe4#mQJCx%1g5CqJC9#^8sFNhvNol zF}ui1c(cxItO~Aosd@Tr**bbx`GK)b6Dt$u5b+zQCX*?Cb!UuYBXRyd;dA#MMkICU z^SZVOjC)$#h4`y4Qg{#1!37NpF|qiFk&+zD&eN9mqn3)=2B(RxRl$L{Y;7`o(0tkx zFv_xs^W$g7?+-G=pjq+?(h|De_|YiXYn+~k01|>PG1Aloa(P}4oUL#V`ql_NY67BM zaaV}3*|D>NW*i3|DBp&V;P%fjf~q*Td(1A$Y<X5~?d|8Y+|3o4kQ65<0bU8qoX8K4 z<2ZaW2d0Ol(|eefBzfri{f5$mmBX`{yn*)$b1xv}+rmE^VE4wi^55mABinW@`K*J; z;g#}4mr1h;sK^K;$HSA?NYo|8WX$W`D2CpGD<<1A4tjCPN>f@$$3TOPIK1-N_xfLv zse4QlzkAazpIw(6DkOUFPCYl04{bt5uZ&zDv;@bNH{=DmdtfyvlXEb;MzcSY-oYOi z#*iVzuOh#|a<Me^2n}GpI^rhAFj3&B6;U?4fV~hnUPYE84@ZAJi;_#+*|^kVQq9@@ zY+9X(-8b6ZUHLsDUYIsa2n?;-EGCGSg$;q7?!*7Xd;Aw6qKQqx<`kK`4m$teX#@Cx zcYGxiDYeb1k|85m4_;5UUHMl+VIjn8*2?sj#q&80&u6K6MrZ?0pUws*mLuX{h(m$j z#e#b;X7ve*`w`GL(b_hu2UsKs)jDo-NLsS;`*5dR-rl2|Rg2k5o%4qSbhLictUMej z3|?;tj)4xOlIL#^^c}%WKfY%e@Ad^3l3;PoWe>h#q?Ln9(v?(a?xH5L6LcvoAiY`y zkCbLUd1a7vZ35)@q8ahc``jSAgyWgQ_r^ZzEL3O%W4_=`-ODdN$$XJ}r)epY2pYfm zUH}HPq95x9W6sNzr0$D^*Aqv%avG^3e8%-%Qv@o0AK+t50%s<XS1vNbS26RgFxCe= zBX-&#y&7A_Yv?S*wBCHcZ!q~}`!bAHSYv^m%vCT<h_wl(z=dTge6HNyYtK}N9SwDW zDN8S)`7Z*g>a0GcxVkqOXVn(mMsA8DE?iqJxEzl8$9>ap?NixdT4kLgcM%a0zr!|I z{cpfN=QFnYHJ>#}4wJUG$66mzgwK=AK{+XkGDKen3uk78DAk6gq&BYCD@fG(^M5iH z&q}iY61@ZAw9)PZ8hzhA2SxY){F5gXR`vIBsHaFk-_=uMZb+YRB}4aY9#7Yb*XR9> z%Y``iKr?S>QJ4Ja0&e<T+*9i49#uV?8Me67X+INut0JmP1ULV(t$O+ByyyAioF)42 zq=zl~(c@61-9Keieq&DRq>e~3|4SDhmQ-PSlL0R9#y+wq3!Wb6#YpT)>1rI$OXTNo zd*zcRzWhRD_2lzh1M}kjDlSlOAvdT+^IBq`g&&xawEim2tl=_kBzRNUaaUl^(_{rZ zJ_e<c#G7~ZP&8Q8y1}j3;IGMctx2)&+>bt-5_?A!ZU@T@`P^-aOwYXJ&(_`U8iW?~ z1F_utxh}{|+^oS@?_w4%)(2`kzexNS*jOjvwD~97W4E&RRepfIG-AJk*yPDdJ>+ej z<LqUO!w_`ONxzw#(VraVbN$nM2C3Nq*7O^}?NgbYkpCPi->M0=#JAG(NZ`_2HrJz5 zIUR>LJDk6!Ps@gHxs$h3iLu~3s4eZJW}|ikavciJSf(~44Afg?s}WoLn9I)u^{SoY z@x31H7Z?g!Hh!YQ{9qU~zIs1@#SILJZ84nQm2!Lvv5jJJJ29J#63#Pk?A{CHZsxfz zF7|WT{e^iz0*Md7#-H`WbB&`)??a@FH_CTEm^-*@CHtxRF>*LIqW$!d#@9fq>mP>R zE4!&W-a)L%Q^=pomW&IlEf({MVkHh2w+#^SJ{ZtzoE;+FVj!0=O%iKWughspGH2Kx z|E6+ss8@)S*;?;0+dzLF4<oz0`VwT}lASZ2q~aS?E2tH;T^-~tDfS!>v0alpDsdZY zZstgzOQlTFbu=u%mNCws9-(4{*?;8US)qvU4E}SQ`7g{8aNmN%0=Pp)1=Rib#pd6; zU}{X?F2@&Ii_o3iD7jaNG85EC#<+aacbG#B5I<1q7smU*(G$tsq`(Nqvcf5dl2{2@ zmpW*@-pBGe*Q+A#vrT?I{NSOfOaBXOmCb4vj^es<5Q_Ux3Raa*#pF7GtB@^m4IoqH za-8jn=qxQFGfsjH1X%;tvHIYLnIo1~S;`|Ydi*Cv1{DQrM$X#P+0|wfwqJ@n01~y1 z<NAF7+<0n+uko?IuZKN8_zlX^l~tSJKh6F0dQu%4+#%_<DTEb^pLVse;$5}il3=Ii zU3Bp6ioA}GI<Ml)97^gcAdCpsg+&Pyte&gN6{lWE-Ez9Q+_2l<y&10KY!hucWdq)r z2!CtBfE=u8ngPP`jOv`X)@@K`X$E2PQ)c|-`jS*G%BL3PFHK1zR&ZWOQLc9cJToFy zm~P<lB&)$YpA>}m+wA64zjU14Pwbe|D?4ZaeEN6JeY@wqde-qkVF%4;>$}+G;~c$p ziK)@;D6Ly$0Q8#cA^9fjQ8g`p@*%p1eY3Mq7k!=uXEtBFg2b@wNujzk_XB?bu?y)Z z+x6mOkR#9kR||km*qrwyN;KjMi&o8@N6{_V{^!771@_g@DpIYWrvT-IFhHeT)cnLG zCyx`b%w27PiwS++7Kj}DP<6Ru;BE0a^PE?O*PsFSedEE`#{nHmSx*u>8!qnf%KhYC zdho@nSvr^V(Mda)W6`sYwXpK-NI!V7CBl*{PwjuR;=4Qdi}tR5=dNHNj!lo})BJ$a zrOWX9w0U0+%sw|%Sp0kJ;Fk96ib?2&!TjL{<6-VW`dmk3K_!&ly}?D4yx2Q9Vp5FO z27_xPciv75cU+h(qYADLaENwxCngo$AZNK!l3BINZ?og4z3Y245TrF7at$)Q`01E! zM#K10qU>P6n_?u)uwzPgrtQB5jM&{9w}L%mcLXLGj?BGQpwj>T&SN>~tkR4&|LdM{ zQ|HaJTU$|?_o=jdCi`5L_e`>?Y}u{bp3{WkLouWT*6}4hD}i8fEguD&aC`(F8wo6u zgn~GA1t6mHR%^{<FOsB#kl8AF=mB;=ik=n_h67X4uriJgC7DmT(F0_~1;yZsyuhO* zZnDCfx7l6wD@<1D6ngJcrQ{-#1vl}{7sc%BiCE7Czg68ggs1Y#a7We2)Ja$}h!OnQ z4kWtKozuIWRFa=j-?DAY*#>)#!PLYJgy-tj&%RgYN-}KxqB^c`nuvY9?$ID$MN$8y zK2M)_pJPj&y^ZL^WOBpgdT96^|AF4?b2Iz#ZUWS<cgK>Y8jYpsIv9juS2Xs-_S97b zA#<RISeTmO`4wdfHu3(WaT4{O7xGOp(z)V3YWR*yJ^Ds{LgdE9hCgxIT{Cg9Ax~Bd zfCC*g)IRGqqs-Wd@E&D_KCAo`Q7Ol7W(@~bZpQNOtDMa5p37~_z8)<;Y~&p(J0Vgu zlZEYKTRCzbje7n&?EHtUoqzmpSIIEdsJEr%|GfF?F&n@GBY(3SQRA_#P|XRe!^AN- zHg4t7=wg2G0LS8OSHol}A-HVX)`@0*o)EVP(-yDgVR7WCL3Au#9~637(n}P&RXl4& z{h1fbK#pwyk1*&RcwiuSeao4LpyMx@S{iPd{9R|a$NotE7kl^rVqZ4HmTakDC4tk% z6mN<tS+^)ZCQ2OgR}Ig8No@qjimoxe6SHnxWzm^p4q-RyK$qcAa$&WD^zg2%$O>-@ zB(vbzC$eSCTp|wkM&YsK#$s0)<d?Z<2~_EbWcmRFl)r&lm&^^ugVNLF3IRc68ZRCC zgQl#>py-&<vF7|%utmiPftZ*N&pH<eq};JAtlsw6#6X&QqAKE6YC*zl(64wrnFOXK zYrp<F($9YAQ3}{nHR)c{>_Jz5viSDRTEbW@zv98^N#X6=8bZ-Vh=FoPuh>jDMo!YL zT80do%*D~nLmOjBXt$|e<eocJwd7rC^U#1XK1NHW+xMr^P(9ZHH`0}#Ap{HTb={eF zryM>Q!d$<~N_myVueWT^`9#?}*DO(Ncyu@|saKJ;yuB?iib#%o*ygZ!xpz6*L@LX7 zlK3Yh?DVJoCsnC>GlV_>q{s)0){Zx@axZz)C=?c2B{HjH_2qdVC7X*Sc7r68lc$Jc zfJURMu16-EI>I3kdGB*<DgkUqmvqJvHCREN;-E<SKC3U`1CRus--5Z09e*c96_Q=K zsBio)BIlM;($9O;beT00N0&1^GtKQ3nOnzyQQ*V(z8t%<U*6V@%e7;tdt1D&^LvL~ zFuZYyQa9@%_^?iJcDotMigHi9$6s4hoypn?WiUzvM;YRRmMld{oK`<MI1V`AyYC6^ zd4fREnHyc%5(WjTkT8cP^cSo|FV(Et{)BqAvI3+{TYj}NpHpa)Ipn)^@WgG-UJ4`J zUBIri2OFJ^Ygl~NSR&m>_xya8sio6`ZpNi|!F{Da)yPJAf3d~;@W-pzy?H^B@u?T* zeRqoT5X%IhqVeVQ*$aI5H6-1!=>BblJOMzA9kHes9=(Bict8Gw>5>d02$BV}F%&5V zCDjXr&-%@~fU#axDNP07ViR4PU~6g05vK{Az2w7UaNpG>dneu~!Inun*o5<`QLutu z8mi_}r4MPh7!fWSok6z|`+?_3!xZ_}P?6a?X_(p>n8p@_?<@CcX-Z1om6J3+c;y@4 zoeK!jZ9?0Y*qbJj<?%>2ilG>or9N}83a$$(Q}W4%+cg*PB}%P&6^x#2q+XYgq}ShS zpb$M@qY6Qii`_@8MT78Wa^r|l1$HykmkO}ibavU?ihJtxLxPn=;a_2rY=G?oSsP5Y zb>1xzq;#z1B@(w!bww795U#kmdj=bj&3Pk^&LGWNr>VESfd)C~?zkpL>3;r*$mImY zrK%cC6w4v&u?iS_2X`G9i0EJOZeeS<*^aCk^1bI4rK*aMH5VI9j%a8<$RmHEZC1$d z8pHa>I+fRWlUgv61t7-yd_{%&|J}5(%iHtvVFc0QErLfh|7T{s3HVOMfjKs(-bE^Z zI>Zpcu<dR8RIXQ<7|WkK7!xnz=zJgZmgP4F1d({MK4A$Z!E`Q(^A{KykIG&!;gixu zzw?K$zxSC!m3p4mwYJvb!<aap>84dlvHd;&B<cyk%q7>5o8NQ$PWaJEd9Fp`!~i#J zfUeV|Lda7pcG+U-%~D)((6XSe@C@I`ZySti1p+Yo_V6{$OhxNkg?v}GWRi*<a+Lds z8{yKZMRv#!_2VR|WfD>b5yF4Y<S=t>&>v>@jdL^R5B4FbdnqKzxC$N8@UT+#+-Q>K zsEHKdauYSMRh2h*1{P)|Spa5Xr6*t>mFxeG&^jKn1Nf61lhe<7Vo?4?ADKKpvItAg zl_#AT7bxM?8e6n|$OBf%6F)+YcrNE+@cr2!CAs)|zk<r7$_U$FKk&1#o~yJ-t?!jJ z<F-5hLIV%IS8k?iB|k9nv#^cWXv%E$ckRZ;jw!bBd1yep%Ac4pK0dc-^0=`}MBAMx zJ}aEV@460Y*|yzsxIS<p=++_8EVb>~EW6H2`IDzihq!=eFvks478KrezWG%?vzgVa z#%aJuMTusVcPWfW@&-I>bXjF%9;?1rDeULDxu{?7V|2PNMe%C}|7Su1+li8$wwvV_ z4}|rryiDz1c2@4dFdb`rP;nI}x3akGTS}qZD7IC82h9?@s*THw%=Qh@4}qEGQuc&u zjI-s1S~-C)g<VYe<J^$hg}}-Xhs+!PJE`^f_13sw)e`y0{rZAY@f3Ln9vY1w_BcPc z3}Z)^Z`V%1y+i8GZ!)nZT0bC{PEvfHm8ITcXC$5A3C)nU-p9t!o$biO|GcyZn)H~I z8Jqa)AR~^YW7XlCU)O1CLHn~*;zo{1CQQ7!05fe*+IRZ4#fDKIT5`gN{rY>HR4n=T zn|2vj8$rv~h@`g|#wfIK6myA6gPKlo`y(r`{dG%Cgjjd73>X=87L<IQ7EieDd<UQL z{>*I?#?+K-@XA3<98@9c`*TCsh@@X92A=mIbjD9KEMmCmU^ZcLSQkB1GdhMLFj`M! zM&$RJv)+$~t*M-RhhV<LOz~5)xz_}=jnwbIV{kNh*<2_T7xe9H0nh7a<BAo<#LNou ztAG4YfHr&mTl`S3(<OyA7i8?0C?#33!%FVgzZje9vFZzAwJCbdBuD$?ROIJy;)<M{ z=Tux+dYBavk{swyB%|9t^8KWk=GB~80`vm)5=Ngkn^(nz;d^3k)ovVjn(XH*1Ii|+ z)u9ca^QSuZ{B>?>_TI)(giMz)BI+R~v<E=nsO{(ra`;`o(cB7FPhPFospLM1^aMsk z2fs~3_@o|%SKIWO2<u3LcBh6JUpj7*m)4v!#&jffs4hc1?^&_>yw4*kdIk<1;Z4%O z)W^?@1N9}7F%Jqs0;>c91HTM8Co)3GX%sUV<&8t*>fP8$TKI@Lw8)D+W?3(vnb!WJ zFNYd(LT(T`F3U%!I$G4K|F`93Sj}*K@~(k6=F+z);qNS@D(sK^FG1aZh-NlVJQ-M8 zb6)G4|15Ms6}-)gBMcZ~67gL0+*lsqE?4<^g~?O1n65A{C@lqPBCQimzF`PxiOUjE z4CS0yB_V&*MWw)~XAH*Ut6%!^8eLQS@hPha6+>Tg74%~3aOPi~0@JILM7>WCZ8keB zNpgV7ELg~WOJqvHCok|<qt}GW4>5w`AhNnNUrWf8B~HdBQl*Rf{Gg+8sMa_-VEZFQ z(75fyeF3Su)aU4dKXlVf`(%OnTM#wY)z;#@luv}7q}Fr+vow0s*e3#;6Zmoo6mel< z)|=NsI~g6q_e={wlB<pg@kP3uGL|=b(C2&gw|c!=;qn`RdUE4+8(^N>gHw9<*8WBe zRee>F*8KzJ-&nF`(JtXu00@Ff6q1k2{O#uD>J5783K_+do4Nudxv{rVqdspp$q=l1 z4viV7@8s#*8VY~TsE1WF(n>h58)MGnYGIs=t4CCpY*UhR)8*qT!}j#G94=$VW$Gs7 zN~AO=y${$1AxL(k`dJRgw2uh;)LSL}HjqV&BaiD&mL2(amTDxvX6rJ}m&Pw1sNSgF z!{yFOn_-4s=@l#)*MmThWt84wqQgPg#07_?>Rt+s76@5OEI;4&%oN>B<D_Cb<>U0O z+_8TD^0pFmeU==R$j`GhNF+IAXc8u`>Wf#*=pbm7$P7kAsR68qEmSfV80TE|!_Re# zzj(cG<x=ocqta2)BOz?yi5McE?2EG=s{|3@Jg(O#H3hyBaH-^rny;C~dM~$^n=k07 zM78U)W$wMS>pS&A$H*3xx)Ryvw#!f>3?5+$uivvN0Re&ggAsw`9hwX9w;ypLtsmx1 zmbF8*<=gcf90cr_6zKF9B<oG|`j(6O50+-G#siTA`+|h!GS@;}+}x>Q9P|e|tRE48 z^n*87>8i~_wZkTLpn&lLIxb{v8^PD^1PROE$tWhHNiXB=g~!X5BN-gPy`>&7Vw$^L zwgX=~jpD>T@iw7=Tdl|51lety)>^<{uoCysEqKq{ZV1`rpmO+%<m^Y=L2!IsrGhx3 z+jf3Bx95YSWCQ5xK;)RjOahXb0rs2$%(pzP9W>yckSoAC)}&<$vt9IKIdx}g!~iVj zvxw6)Y8VfI(ugt~{j{@_Z%y)#O_EKS8wXwoVUtxsFaGmTlE0eQ=1Y$W_G1QJgvbX` zlmu3PC!e{Ztg_>J(_sR%ZDG=GMd~g$Ny6;xA!@ebzBai0MQ;B+v(7Rl4xgSqQ==J| zu(l?P-Nv1?BZ0Zxh}xIBi~%BajW>n4_NULK$z54!>b}Z%f4_y_{^`cnd^_P2DUFB? zO$GbN^F;ogeF&#{eU);9L(0Vp{eGrwrQ&Yly?SM96zs5mCLyRdIXV?J(ZXQMI}s5$ zPhdBEf>GgA8e;WD$%VGKb1UPAgUASzRW>?mr7|@F%=x|=i_6Kx3g<`CI9v(u?Bt9R zy;^@&t9isu{+Qb~H8QuHm^l^8aqgfVu2DTNS8I(+oC?(5fio*kIswhkIqyvoh}hBA zhmQ|7XX*Ss1eK>|FFJ*C{``LG|9_Cve+z*Ph&`}82N~3njQ=yQrqPcvw9&JFnQd99 zMjm~>&{dZPEIvkQH(=Nb0gDG4#yBZwC6v5$GKCYJ>4&y6#%%BkjQ=8j-$a>GNHv_= zI`As`qZlz#?h~YF5P64s$Nr%6@l;Xz!Nqzh;4fAcH$a7}F+r7Bj-9a!+j=f=_4xOF z@o4j3VgMWupc_RJ_-RX0im60;DeXc1%iTZLYA;eH2vXNtdjq1NSYk7@Hi~qv_P9>l z06EFPgB--havklhS0L6=-?OMsNZ$)k3{>x2*1QhOK$T!3_FGi$ED5fSs@QhUPhyI{ z!l(3gWdf{l$$6G3)~X;X+zDE~5X?i15f3nU_e{v<?Ny1IqRIMPfoFNK;)c8?L=PW6 zfqUC=52~ed0}m9Q&kh-kMH0(wE%}cRlvQ_PauLoZX}Yvf8OrkMqcMJPgD_>r+WXZV zK8X6}kOBXypyo!#zBMBzi&X0xKUFXmwBF40WQZVkAfw2LK=LSYHqkdCH=tj9LaO0p zRE)~T*H@Mt)3~9MdMwFsg0~A<y4!4VV3l#X=;T~I86Jig^ZYK8on3lMaNgc}%XKvK zMO5PYK)bRTPBgpV+XOnyk0n>!5TpKm(2Y}J0mdHh5zmPxxNCA2GQHWxJ>6TRzA$!U z@xuHwYfo&=*XKtHc*+&8ix}eKMV_XA$XzAD;!YK!*T!GdJ(Cn!P5t{czo@G3+%7OQ zlyp|dCRLa%4A>W><Pc*v0egnLrLdVOBOJIvOJ6U4eIi|&SV34*&hS;ofPP9$WaWn7 z?pq><DzMdG|AN_hryV!Ys=rGtmhbd~9eYFrRMf-c?y-!}jxf_~3;*38|28&mp`*Xm zh>{Neyw(V!Fj^7B*`V3oqko}aDgIG7gC<#!Nq!Z`lYs*jIlE&8?YX=&xDkk>`L=K8 za<_UE=hJl-W)@<UPSH88a|wv42dsk*QG1QqMokieqaq1;wFsRIhd|;Xh=tQ5Uvh6S z(OIf@gI_Q0S+Q^3cUCyR{X@{-4_xadNx_xS%p7&Z;&i!J`BDOt^nSw@e?2BJODtLe zKN@;%Lz|iXW(Cm2C-`AMY1heO)Vewm{Jmk@bliHRsl9i?i5_sbsD$hl_pgyph%kGA zmZmUaL@||&!iFLQp4>0J>K6C7&gAa;5Z@i}xuBUecKVfsP`eebS<zBBH{r9Mr!Oz3 z@c<;%>Ej&qoGJ2cFBxATzDcmf?sgWXGqGv%DPmOe!gr15Tf&SU8tK7ItV`iUwA)(M zk&s^13`kDfyjZ72)Q6i8lIY$5T3xcPgZU$OY$xdKi%9}n<l3dfgdyq&-jsD&u~^rU zH*{X9$6_Dj`rtLX*F?xB9toGqK8@U7dAfl|k>~ry_&x@M!qCCpn&h9D#2ITo2W~%+ z243*ILP%V*H{4H7s(K4Vegj*SsX-X!-`kJA81=XQH{1r5o3LRjGuXiF*$Rm)o9SOj z6QRAH00-T<4xzJ@r3p?|;MtE%j%&hKdEDeTAJ$M<byYp=yg4ha#uR74W%cxaYv&Va z$>a>{b}2Pv&2=>t&`@tH{LS7h2jD(#$i4pon&Gpr=;6P+yEM{s*-3)vn)1S&A4-3f zlq3FcME3u;G{TD8Z>~;`N}YU*ua36H^K*ax{Awg~+OSjJhA|-dx`$n-OF=sZ`P)d* z<@JUn0Mk|<$yQ9NsYu<)*jv_YOo(OWb*-o_Wx}L>^QS+yvWqFr<~utMKd@ZVI=kl; zC$<UuF-r5Qi-EI#fs$cty%{$gKadbxZ@}B2J)HT6GpworQ#$9(AJ|$!`FOBCNflOT z!Swp4hN8AVFNVHxhD_miX;ah;9tzgpE*<Q5;!?;|C}?CS#>O2i@JYPpyHObq;n)?} zqU-es4sNeAKu69fTV^)|^i65wJAIN!K-%fxWy>PPk%BjEVZcA95FJvYz$jG|Gfnv> zWW2d<OpugIa~T^-nk!URytiZ8{0WENwY?Np*D$*U;H4i}>bg@UlB1!|S2m0FGpEM_ zRl~L|>IUbj<bIDCpRqRZcVJ)xA#U_N?O`PozcHE{ovCxKqenJD*KSa(WRynx#Th0x zmdER(!q9rPo+@$llj^9`*L(Rp7;-pyT+8|BtZAQ%zkj7(w$bu3t8BVjkNUlk>V;a1 z?bDtSAdU~g>E<`aHfbi=SzYnO)O&04ll--qQEaodZRhb-|6GLI!eQ_MX=hnrFkAd^ zj^hZaPi+?Qmd-<6YKK30&3rIFZpIG*>h>B#qP4o5G?h7g!7?-PI6@2D(R(`I&Oz@f zo2SBAxHZ_Vvg*a6=66Nujt6e_W1UEsCN()&A_LtPS>B0GCMwa-LgytpFFNvBQ8~f; z4kLcX$uB$CWG{NN#F%4L^8vd%aL`~3*UKe1JuB*0kg(>D;T44Reng8DFtExMU+MAD zpPRz^jX@&)dbrqU?ava>Z5?3su-I_p*KV3f<mOaKg{Z)`lC)GC<aXqtEz1BHG91ou zvp!L*VPrIylZxpk7AtF5XV69%&2L~eXx@t2nKzD#ek|?%WvXVDy9+jHnv|OKQgLw7 z@!VhResQkV)JWU?)w+*-Ty5BEu-0H=Lujh8fp64$zx4&FTwdb4ku*sv?L3v=n9u_O zhbbN&{R*tpAEBNn>*ia3e=+t6x^8OhdTlz=9i}@iMVDRUA^rWX=a4HHAqVJ)=a$Ul z=JM#b?RGtQkDQT2FGOO0QK8PZVrH=Z62LTVIHf){v@TtIBfhk}!5W774dlzLB{G$J zkLZyg*ul2iC&qn%`>g7De>Vp9@Ev9qs03+%p8S!(9RJz_bfI4TOQZzE5r5~YpX$_B zW53hI#0!Uvb6nA5uYf@sPXKy^gLv=4E$gHrl>x3g9KBHu(fkD4p}6MTKAlh&O{9q_ z4kf`?Dof5l*y_DlB8d$5Hm!A`D)xN?+W(O+Ta(0Lc8R)ioBNphnY3ECR<F*PH^Ps^ z{|O^-CwimF0V8k9Hu7Kddp*1%0a=(R8)kAnyA^2re$0XJHs8Us<DF8OlcjKPAsp8t zz^KQfj3fQ>!W_KkMHNl1>mAG|ZV*F86l`^mF=343&)^!MB>rtSE3Ya1iMYLodtKPd zoorUxG)ClKfkleb{0QB@zOp|V^Ua2@DTqRA3;ACX8n8gd{2tN!>F#a1JKq1;P8}IV zVI3EJ)RD*Dn)g$h#<*A_P6#+sc_aL%xGMyyvu??6Glj6O2LChbj8ZTM!^OXECN6sa z=RG((-rZTb|MI`G`#)=zk&^gz3vNcc`)!!WBZ-T7h<L$9KSyTQ((gbcpG89~AyF61 zb#bK=dhd3hFMmtKk>>nrw-rQ2jp6Q%IytZ9TpmZGVkL4pSQ}TZ&~B3YT)=T`bEc*( zdT>?(xqq+Z#Ew^i$$lLyiv_(bJG{ZVrG&qd#Wlm78Bl`<fTk2sW)i(p!I{_TRjmz! z2$&F55FPg6tBh@_pWl=(TMm5nI_joH73OEmewudl-6j8dohUv>htHZyuDsLpSBKqx zH>al*Q>B(#f=SBvVoR6HRXpUw`YB^pz+z6aq0vHWMfk~h(;)@-=|@dz9rd_Tmvz}% zr)wPixy#(1%jZj${~uLv6&H2f^=l8^CDN$~BGMgFQU=}KFwzX&9ZD-oH`3kR-5@y( z-Q5iX@1N(n_r3S~nG;UtkTt)x*7dy>**u>GYaN@I9l~<bMU=J`kkrud`^=ix`GuDw z`tEgEqOp9!1y-QFHP^pg&Vipe3nb|EdvC{(8rYOTjksb*{C};gN_bt0&c|Jb*P1h= zNa`8@LPd0=VdLx{&Z-q9j;yDI##7ZuJF`>$Nn!YDC(=`|RXnh6hF{uvJx*7ys0*fc zvI+~XRCc+DkKvl_KwG&%!ANsgU1j!%J7a;{cP_gTFa<JyoB0|XA3JMC<D=h{#A7yt znz|a+-Fl7&V%H*6r^(s4Qui+vr|o3$p-tv9@tW23FC!zhRMm>5lEVc5s-XPNon$ol zCE;_o$52lyCQ<75=Tc7VrwWw2(i@iH?Tqn-bC+Q*J}KL&r{Q(30-QXw?eW`*wUos> zThT*)?A-=x#8SrNpu$4dgc^H-sK@h2O|80ID9hEiCgIWDN})k6i|R6F@y}+|Wo*ge zu)Jsq=iHZQE{DMe2jwJ%RXO?ZH`p7@$!t>|2^g5~$*Bh4+lw9hB-DQsF=sXi8BG;3 z0b)TT&N7iUDey*0q+Kw~`ivocYJ^PhA#==nmRe~nJ*uHJSlt1R42l2qCvh14$g#Ca z@^Y3tblozimo+o~Ym0+HL*vkE286Z3q{-vl8V#yX4Kxe<rIX=Yp&O?}Yz39UIeJxS zp^?Hv6*-|!kf0{peoy*6-#j9l)6E1ncJpD<W<cXX(G2plf<+;@beiBjQBVmwwX139 z1&305<V#IFudjgRujOeL-JO1ajnFWFP815F@^TbO$yz25SS@dGF6&hh7tI&=zO6X* zB@%1BI$JiJmoabyB?>9=UO7sG1U}y+V3lDKtaT0n$Q`yJf#e2e<_{lq!zonLuZUuf z2o<V_dI*9O;KFDm(Ot@Adb+Fya3R;|TtmRRr9r*VFhY2PeMXA#Wbg4>fT`s&U8Pb; zuOQh<4a%xyPN`%1U{iaQ1Na+d@R(ySx9oR*?+ih`wWtKHkFFHUN<10MQpB1n!T?v} zkS+z|U2#{gpeucPLvWwn%kgfGci3*dbHDPi)+%%;DgHP0=a!p~dTiELa=`nlQg>+m z?S1ig{{<K4J80_%vB+2l>>B7x3ga|PTDu6o*U*HvigX;5m;rnQb%)G(#&t6gPhG}4 zoiY|vvbSAjmdW#cx7T5><PwvdHGVTJ2uijvjW1}&l#blcW9${eElM6CaK?ew;M>@n z6sy=#lAWr0d`>^nWhWHr#ia{8DjkWG$rjxJM_MbBKHcRs4MP*g40B}5Q{|3GBR{*Y z|5m(`VqhJhqh)B&@?RoLTgl(5tc4gTZVKa_EOiS3>rq0i>Q=w$O#mL8f>O5$@RM^m zZ1m&o*~)<u@1)?_`=O!_uO5$|9@*5FYePCk_D^Tn2V0|cQPvRqA)qxz#v#)-#;>7m zz;iC_P7kkU2(77r_2<OZ!{jIz;xiT@k5z*!*l#<C$^=Hp5|nFq0Fb;|ihT;js%q2^ z0%4s0hXtTFqt8E@ibF9;0C687*xZV7LElOIa2nTXaT@<K-o9m6Kk9>!KE&Rk1x_WR zbkHa>5Oo#4f<f#QlLeQn{4s%nu{df&F~(MmP2~tN1uyPJ19pZ3UU1)~k$`6|CT^@8 zWrAcdh7IShg2W9|8t$GCjb+K#6ci%raP?u)Seo5~?F$1EE1Sxa@Ec}ByH%^&ie(~0 zPI2QD0h(HUNc%FoUq1?_lJ}zdK`iLY6i4`8+uh=R9iA>8VvOFbFb66?EkcH*z$y#X zBZdWc(FsMm%afPlJ_xc?)WPQ;@fVXyid1T5krfA>6=`q}>exy}E%6T8Bs3^}x5E=^ z{xZ02v_HaD9|B#()wEEB=i@S8!&Duv&a_i5uU#wb&8X(pR9B)@6uMYpPFP1JT58?J zpCj}2v?NttY`*R_H8))GfvxvH3egGMv2qd-0Xcr8FdUrx)LZ&ER+f(WkY5a-1}~O> zyT&mn4M=DG3I*auY9@+&({&CubmSPEs4=_R{<W~*+0Abzonk>7#wx@ceSHeuW1=1` zZniMo!_(_3d5P`9!WGv+`GwL&UQL){d-UTeU$d~ym#R01Utnu@B3<K6HUhmfEZzIz z;FT)1uSnxJ_bq5^`K7RIJ2Rn|irqtBw^vu8>8tBRSQ7v~nz1Bgs5vIj$R@X%RM(eP z+ll~Ab7!z^gI{GlGNx}IojR?T=}1fokWRZSG<<6^-#IvJt7XH5I%;ZOvm13HI~~k% zT(q*{29Iz|K73Dl%B_?}nibA?W=g}4D!o7Z)vFpT@W`^-HG$*%5`;p#Z|0kIWg-*T znuxqdq3gSYusA~Jovrt-xRZ{A8uA^LpTpUY;1vmXqc6P#Amp<)kz=WlUb6b5#{Fme z_NwU9twQ)V8ZPB`C545VwH{{OCIQHp^caABUF*SeypuU0+lk|f!O;TC$EI*+w)NPt zVA}}CauK9T2veyq?F%uNk&iOdMfi~zVk$OKNUQyaO||=&ONDTJPz=teKgfPth{hPO z^#C`{hcC_!;tXm+WFOZ+a*z(u25)z4>)z4nXaPfxZ8FPcB9XWb1L)Jb*g`G7H6_0M zb{j9KkdbXntFn(Fq~C<w*N1TUVC@N_|3G{17kOy8;Bx6nmXhm}u=g2L1o(T;^;d<& z&4)-KsOerC>(P%f>pq3wSf1`5mCJ4Br~#bxT^XF$vCnKI{9l*(6QS#fs!9Bfu>4cQ zK1vO}SDe>0gE>6{O_9s|9LxQy^<W%oF<Su$9fGL`_h)o1RNDeujecdnV%1QVop-JV zhsoIdGG^c6#MLhaY^ge<*zpDB+ol?Oju(n28?k$hPM0p+HEVbO-uIO#jHnM!cTXv} zmwpE~@a}r~i)2H}{riY0Hbr}_BSf3+-&l@-Klm*q?cFu<@u4``z0BiZVf(+kgm=&L zy02uPm+}73ypEixekUuO-Q!{>W1p?<fzxH);w_kLZ*R{Y@F1I<Z49vN@t^h6L4eFY zN~rgA5f8#fw8$O$GW^IjkRf*P=zw{YL7$y@zrrmuWA~9^N!Tt;qy4i#KgfadMpMMl z(vwkD1pobt0!;8zF+ZAPPlqiJR-V`6Pfprsa-Kqz7-mlP!9X)dc8waHDU(4|%VZXT z;lv>Gh7d|yDtAJ2*#znj+DdBvi)e>Y;(@3}07lX=o+&YrE0;_>{RV|*h~3|>wBQx) z&Krq5M9>QL)^=x~vGC(YtO9S$X`4xUFSVY_<Q?@aE>UNWw6z$&k*Y|%%(k`6fkunQ zw}Tj)7a5|$l)$0^$6-Hgzl-T6D=<-xHii7$n)BCycB3g!{uCmFX6<&|^-y>7Z?ZoU zR^p_@Q}?d}Of2F3x0H?AHPaE|)~y%!YQC$(0$O%@26}x_Scdz4&i~p`LX^Byd*hJA z_8+3Sjw}_Gy`!rm3uWL(m$CPy==v$PSO-Ne{ba)ylv%7Kso8gj<er<}f#VVTo=PVW z%>&*`n3F+X!Kgl~7<_kd3<5v4YoC+R?-3mncoA+nv$BAu|4~j43H53(WZ4VM_|GDv zkvZJf=F877ME<G-t`Uh5LI%k}g_RBut2eQyBZv2>WZ&c2m<+e_1U4v=L942~!DjHY zn_dDWc0-s=#YO71qV;g5!@WM^;S@o`bdl2irYy6be%M_vGt<!iL~b*6qLa$)o+{E8 zjS{JK5}pw)fPY0PexVifo-6-IC9?2e9LS+nv3pG46GLI-=5_vdviJS!m5Mw@hIu0= zl$q&0Y_)egt+&Olb?M^_!eO1bmIZ2T9IK=|LpZhBAJUm@HlOiq(ZNYAFSR!ESasHp ze6saZwZkvJ!CxneY=k1_+`$(XIw4Mtene(g_C2vQw5jsz&*mg~!QH?8b8DPON><<Z zd?~O*YJ+8@Xuchdcv~^}oCMMYg+%`EoaPN@>A2jj(5JapdP|>#Em2q7MC?EN5dUa7 ztNa4*#<JF+z^3ir`|^lkzpwFA;1ZM|zJlSN%HyP~zQ%^Z)b~dFY0bSdBgbAMG^Rx9 z+!imE?l!1^n}fJqqFU)cQ`g_}jyWAK3E$X0Z(7#@HrqzYX<WZ)RpuK0*WLG*=|a_n z$|4uUK$p{Q>BGZ$`cwA<{rS(u%olxW9+Zu7F2={#L8PVd+FN}hUh~{GKKouE($nI} zw_hz_x;)Wh^jnbKbFG`fEvz(CLcpZ$F|i(0qP>LL`RYA?d10xHL&KgWpBs?*MiBvw zjADXJG^p|rr=PD>Gjni|Nk}M3?ix%p<BVqc$C;tDVEfuj-wCv%4=WK^+how|U2rmF z@)o;pq<p6x)XB?O*jdEu9dmDo<z49Wmh*akDwaFw+hi^C-~IZ6$Gx>{2N>ne9Q}~H z>Ot+Lr;>(pg+!DN>prlfDci@}HbX2G$&Qj*sNcn;2`zDNjAzO})#4lD<-6e@{ovre z`tgNT2qs1NY2D&E$1FL}`lggigA%Eu5p54!$%@OuS7aM?UztJ~0=Tm)iDjVqh+Y12 z>my`I*|d6%eFYM8H5{)@DHzrLmN;nVGQvxsq9mGv1G4jlb_IS_kxP5Jgw8^H#|Wl| zjyDt$Ht3gQo+$j7`A-?g+lVqe{{=&=<)5peBRPv*(f@q{GTHy9jSb-j60c5sQv+IA z?@Q|uk^c;6l<Btwbb)Plm%X?(tPuO1=4|K6b7ci}y08%R&{&!GPoMXyJLZkY^QCOt zL~~O__9!vypzgmI+b7NPP4!SW$y8QQ5+j9kxe%sU>-o<nrMVJ%O+&|ogn7Kc8+2r> zqOY#_@q25XCW0f6p6l6ehyL-e7d=l_Islj&iHV6Rrl8*#;`0vOLk|3JM{@=tbx06( z<qL0ePgo~YF=Oe+*|X6deJBn<)#aC#;?7K;ot>>bnVFe&_%07bN_@7!Ky^8YzTFH* zqyPpdRB7>C0Z>0TPPZ?uY)C(+S~0l~+MXU0xWJ>Rcf(0MSd-*nUa&Q>Nw08z$;bM} zTIay5PQvtNL2A6mGyUQcy>J3L=@pW;8<a+8tHFc(sJavm?l1d{;PH+?U6`|v0xw2F zicsiAM5<r{6j>u5U{bJu_G!d(z^dhUwpwhiu7OPBD%t|+8YWE<AWRWq5Sosd=3US2 z^*R+oGvovq0+8$#2CO1>J0i&<g~H0ILM)<bf8pzEAM_^KbNzUP+Z}im_Ctp%ueC^F zYxfU#*76$Lk(wXzg?dakeV4){piyM9-c!+d<_yy@_xRN4S!yqg`^!aQzEn3x{ReA7 zpUVx>i`@lk4PUTtbrG`!rqqSC!H<8km<6f`Lo>xaCpu}EmG@B#*Z2_&*M@(?-VLWF zb^U5U**HhWBl*el*(lU#apG8LS?n`!WX`di7<?^<@o*e>TIgEpRlLl4m|A`+2bKX_ z3FZq@jX1t6Id=)r=|44pkLq`&;y%(WnD1xI#8Jzw<>)YbUMuHV6d!1|kLn4bYDJ4D z{tTC5)=~*QZaG2DWicS1a=f<yn|wPK8R<$dVeA6zjzMj0w;qbo^@COPa|L=}Yzr8Q zXZ2wK^&gvwx-1rT?wR$raPyU}9!HK67nHc^*~~X80hz*g#>Yo`<9T{Sl?BEb3hDYe zH&84Z<Fy5P@!q9b17E607_r3lAJ%nh;y|bWtV*gL9ZY0M%RMfqHP#>Rk7s#5-!F}c zJ}sDzqFN=DicL}SMWXPDBw(xULkeauo3a{R91Z4;!r-M=3C-IZ2Q<3Ic5{tJO{lxA z$M&mgm9GkJiBV|yxVa4v<|;BsIHD-{aP68;Lk6e_TNHf1=C-{9e&V5Shc&s_7i;8h z?Y5-O$1EAgFLN8rIR}YetFPmsxslX9-+Xcd20qDh_Z`tP#{DSwnup1{-i9jv(VwrC zvL7S1kGz=1#eKBs;i;jek*BV%=?GA6iNE(ozQev3X`Z0L6==4M3aU)<dsaz$3E5JR zaf$=b?UvH2=^lPd3G(3->qjVDGe?L_zSIU9ajyeRrQHW@Y9t-}FH>~_&tcu2s*}We zv8=DR^(Kt>1^_x`ico{#2P_ONWOcpR<hM)(j(RPUjqT)c*kTJ)Hd6?*ozB&$U6VOf z)+0$>VKov{boZdTCkNlPD~VYea+$`gYUjkF5(2)>M_81<?FgZjQG!B$Yu;68axmXf zl0}pLj#D%eO`lek+DNwHF{3Ym$>CloF^2+5Oq~m3C<^W5Tn%cFtf5)0Ct=V5qE>IP zaInH2tvWES0{zS1u6K^1VnSc6Z`UJnF6}|EHFKLIIs;A(w6A6PGI$Ft3z@CJdZT|0 z{!RV6l*Ov$x<{iI2Tl4P%y;zSbj&*Oe>T088q4qh^K#TI1N!F{zW<^thUaQxsQ&14 zBc`14sqR4X!~S+N-cIo<m_%Rf=@BlmZ!HIM-vq8+0B5}dYB7R*_Dkwg?rL+0In8)@ z7nhZtPvK{n=qS1frpe5Hh4Z}l(YorC-~RZ2f(1^irek1-pW&*gr!s;PIoXfuXGce- z9mP?V#LzMe>=P}UEZKyQfrW@|laHJ*D}~1+1ILfrMGGNauk{>$vl9%8IYLyui9n}> zv?E8*ms^m4utI!4Kg*snml4psls;25Mub$FPqX(=kdSD-SMr=6N1LUgMZSU?GA!0~ zC!4?t^LCCtN<yzvB)C)?O1==#Ug>)50>-22NqHfo{+L$?!^KAx0FiyPFg+rm15%!P z72SmIXA){1s2y6+EdMpRfrj_URJ6}mN>pt}a1hn`XlL#FJEW3tv(t8J=pjzTC%h!N z-*A8!=2GEO<F-xaOP>460beJMA%?Ai4)N5s+WpW2SB56%@GqJV?6-bEcO4~jBn?@_ z5|cH{K3IupJ80rviL-4pA<T1f2^}$05#4~teZQq_BO)7ehHw4GgLT<+84Ye~lxPQ= z$V9c*lZXOmxH=FdAQS$c>+{?CeT3_8SI_A1I>$v|sL&{(-|-oaY6252j4!cnrYG&z zq0xDUWlKV<V{cQ#jah>QR2(z~MQQx{a)>NcT|Z@xtyo_0o3e8fZRRJpp%3+XM>+8Z zUNs#DseZ=5#KGy<w%TUvJ&>AaWE;uO;tnl5>)pcc&*|}~W@DCtn9(^@StVD|&Q5KY z^J;)9-+gE@Czi&2L`|H<ru8j16lr}rQ~AccF_HI`?}OmkudSn59^RE7V<O#$Y{H>l znHWyFR!xJoH^<{9G4eST#SoS_zTzIw_arL=C<eweg=W9U$cUiIPK;&kex)stPi55k zxx+(@uY*7AC@{jD2C|AxsAV2*Z!<=pRjj^l7hCGU&I=0WBle=KU7j2f5-`VF&N+Yr zxn8t5RvX0L>F{$`%2CQNk6zU!Cj+bcEeZEA`YS7LdCD7gJqf)ToWiaP??_mDz2aZ% z(050_<Vq{}eu(q|9+(h*XqEHhTao?vqu|{#`3_)F7%`D8;}}||mPQCV118vEz!tGi ziUJLnCIs#1PA2PwgQtD5F8}nO`KXc*IMtaL7=ca#8|D@luF;)qz1lcd5knybr`|Ul z+fr0xivqx#@#Dq_f~}%IcBL>eJvG|vtlo7{eJLnen-$x9P2No7h93hYhb9kkCV(@1 zi`=X5zg_c_tt{mX>PG1>1O2$O1U4nP5V*!vpZ|_PI}y-Pb448onPpZ7gQS)lc=vnt zyv$*>`ze!9YpiHn%M*^v0pj*3ti`7Gy29m#Zato7=A~ecU=|H1ZkHxJs38_ZdK9u& z1`GA?+R8gzjzT*+mGCR}g|$Neu9y_YR_O;t`VRqs9#5c8-Gtu%;4VC!W+RIC?Fsil z@4PE%4N43fXt&d~R~<2wjmg3Jz|{EjWsvM;L3?>VY-;wozA4@_q06Ncq{zk7dlljk zmA2a`pSWv2>GtZ)ipNKce7>u;j4iJFN~5-%dneb*2wg!i#iLEGwRwDuD*{Sc-9p8s z@cI9yF#Q<-4DUd=<G;hg<T6BKDw5D1T@Oz8GgC@PHvz@V7C>+Cit|;_G%N|kNo9UG z3HGkdhE{=_ob1AxO$IHk8{4G9)`T{R6kpmS#ONUyTMCY@x(E=);kdnZ)e=dt^2|?l zET2Xds}+7({P7!SOkUAfG@SWPlaE5Tphpdr?bSPiwk6Z87adi{E!WiZHCD6MDlf$> zkk$q+6*phsbHL+Ssbn)shdT51P_O!epR}J|5K;)~zLK}pK~ZR<<Ri)+y$f`CNb}fx zr+PGBOZFx!)O1JN?^}^ZSuX+_WjVqpN8tAEdM(UWBAM4F;I3;$95*O4E^oOfrDou7 z9+IEbwt$OdM_(@Kiy2<mqfoO}qV>)SgKeY>t(U{JT}Xlyon8jch2`-K2^q3_#Y86@ z8bZz`O5PKufum2>m-2`r>Q!RNz4AwO<HMIZRD1j}V{?A=jc2@Ap?8(|1mXN58AE;x zkWU)#LGp8!RdzpZwh$!*<(xp-kuH1lHdRYIe)0fs2Z7Sl8NZA$PYSQbrEz%3B;_+_ z4%-JUw57v^50<ipqFT?XFOHXRwy927{wj+xq;}TYz>uNQ7}aoxUCxyA_I1V))s;9x zHp#j{)&!0Uf}(gH|K`QJxXeEJO~!$ZamWcS<NgBf)Y2NCsJN}1@cdCT)nJqJ$4rjU zk0x~L-Xeqzji|DTFyd5LiImo71hq|pzbbNCKQP3O67I=+YTP%>rx6=+$DX_4!m&(T z_k{{W#7F)E*b6`G6uvF@X3p!crvEH-y)Sn8t>Zhy8~#{XLDq+!IuhQc*Y<m*e9ozG z-CoN7lvAGX!_!(=))RIoxDk$``lq%7m&(IXaXV^9f0}r)kV)|;y)Y(1hFoKH0pi^S zlSCda*}~o3n}^HDu|Sfj2(l-JEhqi&xCJhncY9u5X9I8f53483yKsfN=g-51Np97q zlBhOac~1wceIRR5jZ1?*cR-K&EIY}ZbOX8!H{dH0$3ynt=I(0Vb@XK&r_5!#XfM?@ zb}Nlu^T>agKBQofPa!q^p5uK4h^v1L4t`lrfF31ow9sp`)qBr)FR$J;+qNhf44eMb zsgwRbW|>#kEn3=)c5yxTmlg354Tt<?vt(so6wDa7no2qj0UyjjG^RAT*8=aNClC(H zbs%6bVC+=CS6IXj9!<%(O@DgW=6`xPhb%z){Pla^7+kper5pH~oh%MrP`xT<gV3(k zw?DX!eZZh;ITgp6^f^s_f4Y90-uG*1>pn4m+|0L>JQn!_+y3KbODwzPRDMQci3rje zkbfj^Z%#h@!{|pz@-a+j3B@qG<0scE2QHcroI)!az>DF8AC2uIg&>%<jQmPTm>2&z z7ev!ER|2Fe<_a$DX+3c5d)!0jqt=BiDQao85<(~@eNAbWH4{Ch+SUW|veNiLxqC>p z<n{mxNZ<&dfK{vzjZ06MPZ3Zoe<E<$(guHh+Whid9;Iq%l;#^t6OFE5V#S|td^95> z!cy1LMh)sY;;o~r%m?p!5`E9+Fq8RB@WCT97H<w#hSO*Kq7$EqY?kbJB0pKFSY%!` z+G(O%xfdVR0RT{p8C*=Sso7fZ%-z=oPBJtNko(%m=Re^@kMOE!rOwi7PB49=ekg;l zW;+Q%2V8{3d|3E-D(%xkxQ;6}IuxlNkoVQ+`<f4_wj=fHY}McC{|EeQ_joRhE&oqp z%tJ&Tz$Rs<W%pTW_jdzQ)Fnd7Dk_AOq(q^kk|=M}++IHdhZf2At$>XQdEU7g7BMsW zK?0{4CeY4C@Y?L+_2Xyk6nVKt0I7gBSpV!+%!1KZ8ouSW$TB|`zhPnYlKsyql}tfq z0ScYJfA6n?zGYD#b*QuY9eCvH%$Wp4JFn2xd<Ml^5pzzssClD|F0EHb9v2g^EIACZ zbcowS2GBC3a!ne+SZf92P1YY1@RYYq2K|cln`k(Zw-+45PbzS*iP1!6{=iq*?ACy! z#%v+91&sG_(AL(OaRm>&2i!O!PuFcvk?HUQ@VsCME>#%dbWcxEN|<~N;}RYoN%}gr zXZ;qxXUpF6rPnY5k}61fCdelW(S(=iMl;Vu_2K7bj>zCie1HAuMATp*n;d>C4N7kQ z)xWCLDyrg2;UcR@ay$XC*!RPPC_kX3{(Frk{}i?zU*=hJh3s#oAKqo|ft$7l9_b!> z%ny~@JmDLy@9i&qWIi@a`ZxJOBN7k5tZ=niwhD9TJY@~8!<9T!^}vLoN<g$mv8I2v zG-c!u6Hyy#^8lyi)-|RLEO$zup^^I4rngz|>FFg9{?KbVmDv)u_g~zbtc&*IjI~A3 z9Jx};({NI9iCBn@R3dWHi>3Q-&e=wf0#7^aO<1+erMz&=sHx)lCgRkn_V+W^!*Q3{ zG)CSnrT)!{`by%W)`ZaMOS0j|Mnqx35~<I*-v`eg0Qr+*XjG)5k~Jf(C>Upa4q1?o zI80M%_Isk0##-qu;Ys^yAFr7V>&w&IfAsuUw=#hS_s)yNb?Fa;s_|-uy}wZE&vrde zr~d`B=r(`Q32`y|LyImD`z15$?~By;LK)yD`MCmbPp6V%F+n}CFixsGOn;k1S%gGc z0-{X|>&X)2R6+BYFPlm_%YHwvqdWeV#jV|}NB3dWrTMEFFn5)ZEJd*8%8F|VV*OR& zUmcExw3~%yy5^;Ir1fobb~q{dfU7jdp<8D!{1{`R^haM@x@Oz?-oxsFZEy{zjoe8^ zapqS_XT2u1)vn$%Kt+ezQXE<ezh0&R?v5=hX=^uj)DLV5Ks5BvyQzZ-JvT4sU7{H% zSRVAIjEaP3)l&mQTQUNywQEf2ecZ=$SXZ;9$UL;BS3_xVJ(T|W+#iPGQVAkoJKOB* z>&6Uivb!8;{{HjFU!PcPR!N2`_;V~n%{TBy|4&k{0TF<-(LaCAAa~)3Qks(Hl9>Fv zR=;D-c{~KdX^Nh?ZWb?o_MZIuDcbRuZ?-H|bqhcsdwFJcO_*#i;j*f^qZ*@+AFN9U zeJLOXGCT6E3d^y=e$Dc`BM4#)D$|i|DySRvC*})htRM>KB_7WVTkWQ+R~R4`?3A^g z*YP>&yK8=u7rK-&pJx<n(vf-x%&e<s3fH#<U6d|^#Oo>h<FPsiR+ylj$Q6iH&NlW= zcsl<~Ghz=`bU)ujmnR}E74Txf^NYuWbfP~*346?eapG-0XW`YZmyH_Owsmc>Xtccn zNlqWz*eEc$Scd^58B*37I@pGNWcAHj?umDDJaJG@S^n7if{ODeOp&PU^bfTcp#qgZ zvNI|3ET|!a-yZ=#XG)C>%a|4x;ewSrs39j|2E<9E@IpH0ZezFxdFh|o&s`q3w87~} zySuWl?pkrcnt3{>vs1~ha<om2V(MTxx1Q43AXvM>u&TJ?@FxGNA8sYrpJbI+#E=5{ zlw~WB%sJWiKMi!0=N-6K7G)~1<#GHdPGGcE<A1`jhF0p&<@J2$zSZ-8Q?%_{#eUed z&yjmlVUKDxSH|oK6cXKXg!*P+6TJqQhRi!!OK@oMD0s2C%!kqMq@d1{TjE7gwy*F) z=|$@|etkt$gaJh1N7Q*&Iu1ZlwgHIA1VXpF1bc#g<+AF{rz$PdpGBxy;g37Glanp^ z%;QWqV65oi8Ws^>_!h~@jdyh7o8@oUWi40QUdNuxSo7fl(Kccqh&)I_&k7>*Idtpo zU!ixZl_4pJBEQvl*H%=YGdc?f)CZQw^ECQ_d`oV0P5`qzfM<UyMZy=DCph;T*KE>* z+y%aRm8|Hj*tZ3nDwUf#WB1c<-(XnGUj@F(VV<R;WG}4=qim9bTW(qExarS`Q3GOc zWZ^>mAkaSnw0pJt0aU{AEF@bSZ<)H8emFZcV=_|6_RLF-r&3R}s~SB}$?Ur{N-aPe zHWmw1Mg!Y70LV5j8{wyHTrVeH-36NE-ido^iy8*#k<J=obm1}NzGz>SG{QC@aazQl zlO)*-HuI#*#g0W4C2}g{wx7cnSlqp~H`{_nEwKMFBApNGy7>~5RoIEYGLajz2hnnW zc8x-~)@tx7{B_jRJ{99<wY6G8pWw2*zZI5R)Vs2nLPtveM-q)0M(!+UMYCC5M=dcW zj8d(yLO0VD&4*vl<Lw1IX%Y8tDt#NyHs5;9&8!m%4gWeZp762mssXfD!enI`UV7Ha z<Yb)?$;GwP^j{XBNS*FljY<$a7&zv*587zj_KIleAz$D+$j|s>!Wy04djCQ{33jpg zbhpmdDFVamZaI$b-HBfA4&&ir(an0T6Loy}Xme1Zr%+uom{s^8u(I+c=Pu&EU4|8Y zP&5*#VV5i5XH_ENmoNBpU&mrZ-ff&N4VOWd0!<Y4mHd}KJb22IiXx0HbC3_kv3BtZ zx%6Fo;&ktzOrCaTY0cRz+pUyrcV1H{5&lD-l)6nFcq`}MKIJl%lI^+ITF}}`y?i%Q zSMId?Z#-8)aQDrE)1(~5e;CD))p~7=z~{us<^_-Cfpu#cWRGC1f4>2nb`n0s+Jt?f zHAmhbxz1V<Kd*W{v_R6`f7R<f5I>~3KOcV$?#y+MR_McNYhB_ttGV7iu6fqw;0o^0 zjxYOHK@U*J3ie}11m)Rq5_oO2!v&+CPF*4WC^EL?%k(=p#Z{y88F1jWjO}|nWDp~t z7wE}a5jr^F7`pE$m=52je`*(EYn$$rM?9qQ*vahYus`*Ga}{=$LgHyKDq?ULF|POR z?bFB{SZoQKlXlXt7d6I=bQsl~VCf}VXQp6%dG20I7NtKIa{0gGTH56NNuI#nTx1&9 z$7Uwv9nP!2>6ot|?fhhvH2^P!x8pc_8V`^0iTc-xd^>+rl^;I80W#c6iM=115VjUr zH=mCi8d|r->ipqhimxcvy<ii)AcEjkdu}j#XF)#6nEY!eF#NbTvWfN1=JQF#*C#X4 zPl&A=>$rf?=Ld&TE5)pDzW(Zct(8myTnX;bLQ<=**!-jFs(V7=5R30IpCkUx{JxUF zeWd&}`>(zmfdt>>_|VqbFWRGD=JzRtezK9(#6hy{-)uYXu-j|xj8wTs)v*!2c+3q} zw)!F(ZX$MDs)M0EfYz~;mOHNwQYj5Hcfzd#s|3cx2F*>2yhV7W6P5TsQX~wN$3zwt z1oyJ_3*`<L72^L+80P|Bo`BAXxf5F7|3@ZJKK+h%<`l4G+QnSXyg@bNxdrZeS9O?_ zwLJ>nT^(96Q7_op024wM<zA^q;EeZ3rNvjTexM+}keR+Ns3BTG>}N&$8hm)R`VF_9 z;ktnbHx@=I`v>nRRxcMj!h!Pc5NUS31NP`|LE}x}G(eA(jfEbET>t56Z$;xS1$Wyv zeecJ!+CKWj*(a-}{0g5z%S#}Smmw2PojG2>uQ-g7Io0a8YQlN-XROa~<0l0*r|4GR z+m$RVR#PzDw0Ql3#HP~9S<EYm*RM$Tx<+W0epC)tig_-kv^<;*oz9u8g6_ogd?}V& z&ZaV&ze@8GWz+@RIkV>;d`jVe18$q?sHepLeLzeW)L<gxQs|h#{n8n-`iqffT#kvS zw+O%c*Mft|1K}m0e>Z+#^KOiX<m5N4g%+Jof&|~S08}Wu{T(4FH!8IG2*W1p;Vq^J zAY|Nt;PXO9g%^{|0`#XJpC?pk&$3vlnp$v`b)RgoMg&&;7J`=0(FeW}v*dW&d=9^C ze+#>B`@T~&@z}$vQLR|%kfZC9y~#4<O4UDV8Lbrv4IioVyc|h=9L8Nw>aGo>O-c!5 z4g^U*5uG0Vh47n<!N)5qIY9x-$vi}&o(HcU9uv|tJtcgZQ&J@uur3x$UM}9%Nmw*t zPI)zu{((7*_3%!A0{_c5s3b4SWs}~lJye=5q6@qIc=GfxFI}8ei|$<G?se--d{~H? zU&@&AMn_U=*Mq;&X8nUIA9H(~XnK$yd0~Z>b6&d-VvMM6CBEpxg@7*mH#S{vL$(af z?(UrMQ#fB*Z(&Fczx{TL{lA#5(>#sizWDGh+ZEhYSar`Bt}t$w%h4<{0a`gElZ+8J z!x(`C?P&z<y2p{_C@=gQ;|64^L}7J*lZXH0dtmwExQN?L-IJJwV4y_PRqrv{LFk`I zoSQKXl?tX=KeTPH5ETVcDnNwc$z|A=NRiu6{;sQUO7zkb_3#gH6k)KxG~IDj;AXna zXL^lHUi~uP-7=}`OuI{KyHk*Mo?!p?J66damPT-u!3$@4gf(S721aF+7z~G;W7<YI z9$!>bx4!f88=tz{sG|L{k}FmdmgqLy$e4Yv)g!R|`3Rq5r}PQ7e{Wu@3pp<Tir<Q- z(NqpFHi>V+yCWtZ){W2qrYF(&uz{@Wv>L2<Ei9<1H7C7Xz}mR;8vLFA-AzL8W5|$i zv_`r|L$LMl5_f>@&QzY?>rDiwAVkmKI}8>%s$Y5I(Ow@d7H1x!&PzYMNmqN~Xum7Z z!WMThBA!4XW2gv@$;YJtNfWc!ZhdhE>2fCYj;xVQ`{GWK$^-&|mEu%P8pmEP)|))5 zbP>QcmD|{9Xh^tW6-%HzWd^|0%yEUsLq5a78L@t|0<ABI7^j(YdDq$6nhgO1?)Nm< zbveH&mZFMqihJoKEo%3yZYt$D|B1^BV4TWY#>Z=t+xzvj$%aam)fW59>#HG`d=6Rw zZM1%@Z?%`$zE?Ws9CV$AI38BXQT|L{R9!KdOnS}z+gtqxm(P5F?IEVB$8pPhP9)gM zHDJF6APMCe!Jx##pGG<XUhiUUQ@)}q15U~TYFlF6X%`~Ft&ja33m(#VN<&EyPP{sX zAZD0xk;*^(%9He&!3*nkq|aTKon_1j#)qnv`G5zTS}ck%${WfefiQnQzI^fUUmc}C zi`=s^&Vp|5e*Hg+N}X853t{Ju02J|f@JLdyDpSwiV?AV@-MITZPFjvc_44hgtETg~ zL<t71igaqK%ly0Y9+Yx(S7Sn-=lDqwT3>SQ!25QDnCQk8&7y-G%RCg>H4jE@)Aqj} z0xKEwJ}H3>oh}HSiFdT=-fvcJ8idj<QDbbl*!RTvd8*7*4R+I5J}~TIA=yuNT^u)` z;}WxIzcl^B^zI$T<GnKkqm`BQBU7w2L$Y{!SlV1`G1%Tt4Z<U0lMryVYdP%ph_wLN z7AS^)KDR0WKg%eR_JARBb0IN&?}fW(xnJ%0V@7z;M;5JV6KUL`RvVI$fc0?~B#2;; zguA5z-Cpt}$R2GrLh>xD-~FO%|IeMGU9B-{cf~{}M!ZzxcS%7(N4q~#BxH9%hqc=_ z6dQL$)3X!f?eqRI^YWnai8jS|Rsje}j4r+bc?fKwgEUD^lDaNZFbN%AsS{l(y)w@} zA-~!51$eF#5~z*GZ2-BDb%r-nQLnaAy2a>^Z6~(I@@HvoYys6#I9@(m;gxCZXC&yD zDJYrZ+_X)#N`F$=_Ua5^$VuGv>TG*ZV)S~4x#gA44G#JE^g|92HsWpVXK<;+OwVWt zolG0cCcAjB7^j_~$EEGdGK+y;QMB_fS%nX{7p_MqHPxFi`J(vo?yo3<4217pQ}{PP z)huO_a&Yno&jr$l#~rR&Gdk9_0GZ>Ddfu3#Pg`si*3uZ5C>b(sFRW+VD2ZUz;;p3w zmqYuCi)Dmgm?}TzsAvk<U^rNjj~gjo)2eXN>jN;PQIGc2s9scAJi=FXM40ot*Qtn1 zU;M2@m~8`M>a>Iqur$0P9n(B9eW}^8;f3>P>1>5%UD0owBX+Ig*3*!pI+yh7q+S`c zUoW|l)381Q2^(^6LA&rWb!KcatINIV0<2Qut(ON2Hp$L05dA^Z=2t-r$kh7?8>+m4 zvRdyRPq+H&PZR4Z^p5rUPxn+#>%JaKdXJFnF4(dAjC<l+Vn&j-IS}aIdJvZ`x!XP5 z#A=*C7Q<+}wk!ImhqMs+Kw>tP15Gz6*1zAy?F$+rs#8!<h?7b7EN68}Dst@(?klNU z{|Hl4a^xQ!%G?mizo#a=u>LnrA2r{~%w&ztZ<T;atD=|g`zWwu_Ng*ArT%@{twWn! zg~puSJr$32OZlQ*fmP{HLhz+tYS*(2#0R(oOJUMyw2j@D^Hw+F5q?jRK3!u&A+&R= z5}F!Be@<NY?Kq%_t(S<7&FWOELQ^tb#QXJO<B^@OSW}A-+SuGq!4a8_!5i8!D$8q< z*{2t5i{}fdwU{(`=dU!9ZSLRCYpm0`rVrK_pwaG&m<S}VOBB3zAAQ;hbIy43HFabL zX@$@>u!rw}l@nua5GwgsN^B!^Y1`j`s@XOFAgT6Di&4A&>vlN^j&dN@ids}<#pN%I zO7|nvTX9v2Pa5Es#Zq~)UNO<|Q=JVNU{%p|@IT%h&SO2X)lHthvN8T{dsKhptL$A~ zk=o07^p(cRah%)tS3tC0re~!9D>w5P5)W$P9SyPTjw9@JvLxMsN>EcZQmxb`%ww%r z4vJ>8CI3I8O{MD~s+804CiA@+EcvF&<L*S6QY#lA3q!KKLBbz{x&T~P__^ovloM}% zCa}%zTgd}{MN44pYhVFL!Lf@ewo&nQ{E~i_zLn_vE!O}MkCk&u1W@Zwv#8aPNFN=y z{*(u&IU_h@P`iw9%h?nvx-7a~W6xRo>KPf&Ixw`Xi!Eo_@~AkXKLCIbumD<^Sib&0 z?#<@|N97Gnfa!O{d82vpzX~f*6mYs&Hi7+B%-W=as8o~7VMHafUL!XsW2f!x5`OG! zkW3&@*g1n7vptp`{e1mZ{QTLbM-7-IO_po_qHVOvg!#ZP@<e-{5h2qiY97leRIRce zILl|WWs)+#B1BzKB1je{QDPbkOa*56WgnjoQqdA#@=)|W)iurfXJ0S<rThLG5%YU6 z!}qzHgE<r>as2XEj7*rsV%8bTl3|$Wy!bs!iue>Q*B&j$*UeS&bjc&bZ@93&%Ts&T z!%{ylJ)Gwb>-xAm^Y)$9?n|z|q@$hWm&E6!Pp#-<zAKqV)q(c9GikW0lwx;gkE!xb zhF^yzohR?T<K*6@&+Q2($=04E4>?v$MAmz%^ku<476|$auV`;Y9FNmQw9u)sG?Sbr z<cm5_J{z!8<BO}>zsYYgP5%Oiw%oVP{g`kSrc^2e)ao6xf=U=0FX=!k=1gKq!*(~G zzE5yhfSOx$spoG}@RUtthmIS>C2?CAOD1!F#TdYF+qxg<F(?vibAoVtC0uh5h^Uhx z&B2G4)%@QOY(Y4$23EfQi@Gak0$KCABuHfv-G`ua%~5%@-}@1|sq@Y>{zw|}lB-#C zyYKPI`?q~CsD$!kj=#%G-Fayh{%1bz(si06x~vU-et($8#iZ>pgKqc9Df&zL9}8FK z^%JDOPKfop)ro|uQY;4`X`;iM)5+aT3PDf`>FHFleBogqY;25#iI%?vZOgkJRHrN4 zu1V32w#HrwE6G=?eSz9`{^p+}=joLnBwxP~vDcTE5U%aayRtkjG)pmiZ8Onv<9$7W zRPTL-eDQ6LqGX)VT|oRd<9?R~xSOt3_cbC0!Up94DrVrPQYFk*&2$6`n%i~I+E914 z<j6m<@|Z_1&_;xwML+-V$V;K~pDFl(4)oq3vh**E+ebfsW#kX~TYb#&4XS*=pAtr9 z)0pk0`<fH;?Br&7*{EaG`CEcAv0bVw#+N<@!>wFVjw`@Oq4#=*MyxMn<m6lQdwiM< z&IZQ552g{c#>SYPZQI$j_W>saZYjT!4Ro2WX!yS;=Z*rFr%?OblSj>)1+NX$?Ui3l zmnWvNQJ>g``&yfshnKG2B8nSsSlCn;Kk+U+S}_VTQ-L%xKcYMz{;J>2O^$#Eyv+PP zW6un|M0wxl2qQ`L^I}?GN)d2m_qwr%**?sfM6cfMm$9|Vo8cJos6!)7<ncmZyzjhL zdV5mA^v#Ao!_7W&FDZ6RC5L6#!y^t*9bb12XI|S~P)8X^>=e^D_3Epa^Iz{r)R^zv zR5@qKdOo@j4BR7V`SR>{UFaV#x-S1|6#FBn2DzW=*qL5=Jqadmy8Y~WU^px4wxmcY zK&tIp@yrA2LVAb@YR`_dN4(tU7D6WqZ+8cebtKc(P6ZOjs<x@l3UZEFIwF3S$s~ND zb(_qU>k^rrj<i)j1iK*R>O5`V`yIeT&}h>G2x59fu^DB=TrdeZ*dnDjO!+`~&`k1< zv10HpNY;zmHp?yS?o=sHEknAFHjTn9cGfT2@6*H^wfTVA<h@nYzw7?Yi~Wnm)UUm) z&7;3Gv8R_;^}O?v<qbE>pEKhr;`cvJ4Y1GJYV<a!8!^3%;tkrV#cg~%z8XjSCrQ#u zgdj~do@R$sw^D7}ZO+xU-N$m5_)9PSPo(EXqV4^_7Cr}FVcu}r!j-dnEetZbdt<iA z_aIJe+>{tuB<Mq0;&Yt)^y6x;IxpE>B|^1WrBugFYsOADBC%K7^=M&DR~XcwUO{|3 zobtU&NLu=5xo18$(x+WxA@*5T3F>+rRS51Y8KkQ>I#ObBB6Qz0a!S>eVpGNXi)B$i z%Fvy0Zw?^U3A#&TO>Jgs2V}ex2p*Gqa=;uAn(iKLOMHsa`|XrrCt^oXNphD8{21E2 zR<__hCRafC|9$*@Ll8^x{@sC9yQ(@>+W+%e5iMwczEg)M`Y@h<A4c?EvCdZSE7jTg zxp>AYU>_uN(E~1?aQHrYb@%jG?7DqG*Z>yqSwM#$B7AWST%D=57E)v54GbqUq(zFg zz<Xh1_}8~(<K4<Td}T2dv0R_^Hv-Uv!NFC&rG3O|W;`MG;X-*PrE=5jbI{pusLE-F z28|$Mj1>MTt~BAH*-9zlVK;gL^Ih3sh|i-wM4}8`qCaakHaFet3<G!O6I+N6nJT!p zLH~k_%Wulou@zgy{DdBAR#TB&w;I~8Y&ey#L~wLE_{*Zdgs2*2PckNeQ+Xn-G}eFU zp@qi#g21-vkQa88@;fDnr)ZW`siep0Vn?a^+e9aCG>Li}X8LlTbyM+J#itpj6Ctwh zI;3+Q^L)Mg^Xsas88QjCC-e65!bM}~u({ri<`$CrC9%;fEsFf8SDbFzIgjRMg$wVK z+0Zj4EciI^^gX#i>3Sw;@=ydr<F~&e@6`>X`4uWF0uGdI_@G_|6wX5AVn0-#jIP3e z(xys_srx@lj;Qt$S0Qkb&nqn%8>n9jQ=!XVcvgw*nI67vlBvOc@AE(jXWNgCpxX`k z0iVC<K0LfP87UQIUxS~b-@p@m4rB9mE18Mmt%xH<yddKM&GJ1$-T5r8J&k-@Q2(-x zeqboELeC_Xfy~v?i|>));(;@#pT3;Z@zUcyYSK91nst%v%NTFIx$)v)>}Qd@n^AqE z@l{&4pk7gSy)%QLfZusi>0mBBJUcNs|Mv?qFKcYXN76UFJ)|yoG0~XNqLWTnEDrC( zvK{421_s{afQtv+4|lVCH}+!;bZy|#y|Ai5jg9~o-}Mn3EBiNmFsH{7-A2lz!+&Tt z``nU<9eo_Z8U4Mj{9%Ta=x_=Ou?PbH2K<hI56I_^1jAzpm<~94UKI#8goF`KCSGDj z`mB|>G#&M>@kUG6qDUNKAC+HMa&k^&NRkZ;6_HHz%q1rOSm;Dtcs%bT)h**;Et&`2 zc6_|Jxe~S>k>xS8i$fPI)o&3LblwXJrSTyJ4p2w%`$_6vYj%{#w`=HkucOe=h<K46 z!s_Zilm|mOo+&pMs9A3r(|U}WXw`mTq4&_#L|7~S`NYVqEpE;G^x459fdN*wQ)nM{ zFBPvf>5eyHI)Eny-ocZ|*V&Up4`=Y?d-b7Sa_PIH;+5M}RNwiv;gNN)LGa|{<V1-M zpEazOC5!Ah%|+D1`2bhVO`Ex|wCp>|*{p%HOWXG@rnhoQK|q{RYVH<*PfY+wA<p(w zC?EW6lI_!%FO`n7=rJ0X0zbzRj?1hUHBY=5Q1M7#Dm?-*K6A>^fnS6)-)B5zbno{* zuzkOpJg(gK6awAOi}M*Cj8D`aZfqg4Ay(#ktcBClgP`vuyRE~4O00(;9BNdkj^ppt z=zcPPMxOEwBo`FW&#K1WkMlhbb7B5AgD~U0w%NJ%#F!uf|E))XbcMPFiI(Rex6|bD zf}g2+?T*-QTiE!!ld%I5@eiqw$KBw=c}4I7{(eTbP*c3<P5Y1KbMo;_1g421caSRD z9XExQ^So?|-HtvV8VP@ar~#hOlA|(Sg-8}&LL{BMGYNe5Fq6)<myIiC&hnu8PX4kT ztq#rQguGF}DY#C<OrD?*>#AwAzGKkHBshvjiclo#U;WcUvpY3#Ajf2`_fVr01EK%3 z6exd@JnBh5VsP(ep{aY=$=l5`q~4H{NfC7M?%yf%^oW%e$kOkn_cr%mcS6;beo4XI zW8(J2dGiCtnQSbOs*}uL&2M)5cDqFL2Fc!Xt?2YcOMYKu(H#pX#c_jWjeGz>kz&*u zTU;aa{QjR$dqKV*@S_)2vIRW=3g!QP@Rj{|Yo84dy<^Yg+eVi|%Xg93`f~E}ycvx1 z%tgsjZ~dB02VN|=Z>NTQme1napV>^eZz0o_O%`v-?|TX&KC|Zk^O9d5RCyGlug?WB zsXpqp92^?Sb;&yc?km<T1OHmBzfl!v%pZy=5VVjL3m%bUwQS0XI(X*4%iiwT4sH&a zCvU6Km4#iOB}2^~Pqx#2*MPNc)DA15Xj;UaB}T}h%Si>9lT@PeZ6PgnI^H==t5;Fv zVipdbg4;_w5mDhReFg8!584`*Mh%~raJ-nCG!BHyTJM+ujdl^Y<Vcz@3<T~5s_koR z@|O()dd;?-(JJH3PA2q>VyNS>TTBydS`E$76W}SZrZP}&DcqJbKO7vz+pNv*dUD?U z8Z^Buf1f_Ba)BU3;pS~-Bi34?T#HV`?s<?^G0>}>iiDR>$(10PvpDQDSyFZ2u_&N8 z<Ew@zY}i`LwKcG%dzE<fV8nA(gw4zOly`sdAY(1+GwOgyyAQl)Z=|PQYqR5SzTY2m z@nJ3s3_~q$F+0q5Xv+`8^=kIIbw~k1EInEqo12-iKf)(nmB$fd<mwiF?=(Ij`C^wl zbk(_8@4EeF(;{ZmyL>aqkh3!8z5kPx!p^TYn$;aQ`nnm9=Zgj>XhTbsrISYJD%16c z5o15vpaxb(uLb0!?{-Dutb^p_)%7>l2Z+y`-{E|;O93f}rbnY69ctd(5I+2l{HdWY zO&xVK+#Tnw)xDo(yz6%3&4C(336`WsHFm69U!>9RJuhk%PZ^8m8{G*BJN@i-5?yw( zm^n^eHTmU_)a8qulg+e4)CH?J<riNwY!-V-9UjdX|HLRFQ^#_kd_P55Rfv$H<)D7- zt{;E;jDh~9y9?~sdwSB-M+d0BIuER0ZkcA$zDEky+U+0uBs1?%r`d4lik&LVR_taV zi;X&M6D+is7A5SS_=&B*Zx+qP_$(xyqe%`ANj|rl?mbS9$@=O31YENkOs7r*jb4QA z_P;B{qjKV0tN59}c+B1XmOf~Y%7D-K(6E-;9!Ao0Dnqof-X~nu$`GhFG9QEB!{ptM z<5m2tZM43G0+KqW^MU<Sj&4!IknJ$nkU<U$wB`Bs-4bxss=S!BwOPnR<V8TAnzg;) zmGYx)n$oThWT`^ut@emcp<{LNh+C@aUAe65pzB%+$wb{E{g+Zgxtj#PZS@kJ)$tAq zBQu3$UKZ=9wYI;EqZimNCoAGAp5WmGf8fmcY^_N1SYeb_W$W8g3Rmwc7Uma1P2h!k z$j)+`FN9cd41B<#*1ueM_Q7j7jU?qJQxDlq2-I+&+5!O{<BY=V^SrqODAaz%1u5vX zD)k;iDlZqsBu$fbzW?y&Zv;{K8gQ?gPzfzapKe^Pi@#Q*9gw>)t~+5o|N53TkHTK{ z0M8a4znS-<M;m3%0g-0EGd#;T|1KP=xwhd`_t=HBY}>AfnyI{~dj5krXvgKi5oxVK z;6j|zRdv;5qORoi^!zRPy?)I|_h|^c6v^<6SAeX}^m-E92%jCsEeyBT-4x9`Rx{Em zstE+1yCZOOlfTparkAOE2}4#gNy=&R@IJYx^~0RjJj>8z4vpOOD4HMp+;1JFxo5%` znpY9}SE)W&uU}ml@i&<jJC4U!n#a6_AH2LWV*skGcPUcznOdYew+@G8h2={V0>iCW zdQ|9%vkBt;g|yX!j+yAyd)1kR#4au;9^*AHF@Q6+;by0g1xSQ4lVu|h$}(0Wc(I?B zsR8MBzl+|;{nCG*v(L}k71Q@(zTsI<yZE}Pfd2zzs=hQ(+N9C?-wNFBtA;TD=2P+v zqn_gkj~hVP$>prjVV;x7rHfGMH9A4wtezsr0CWjpbNNUS@a}HVc~)<`zgK0Ifo<=! zVEjv3WL^?tAClM>f^q8iFMm^Ek<+riwH2ynbo|Tawu5<|#|OTiq<&uCrbUL`PI0o) zVcszemtgxuD5?&DuF3yL)mukJ8Ma;DLw9!$p@K+*bSj~MbhmUPjW9Gymq<5AcXxM7 zcQ*`OL%)~L`#tyj-RoM5e^_h&nmMoI+{bV4eMTBrSp8N*-ED?3SsfHa1{I6hkQaTr z3TEYUz2hp-sI}vO*wpc_GI<oU1~W*Fs{%_XcW(lo1#b7AR@g-v5#IGZ>~#DuV^;bB z?pNfjB8h9|R=1vDJems4*DLN@5$;zL()6sY*Rwk3=l$R%v-1;$Fb655hfjlMXO;4= zTDQ=^OPyq9rp54ZobGdI5avs`;^*-r*@(~WVV_+dYr<MsoQSoWr(B9qsLw8us$<8e z%2pD;$oE6MM{xJ}^*X|WDpcthFYdTN-67)xySgg{OPh~5TzeHa@ptJ~FL+C-cM3;M z8%`Kh$|l#?n)#zJkzyT#Hnx2+dv$~|C<rmG?H#-X6S-!@FcWT)CXQB;nJcBR2VstM z6tZNfsbxp74u{W>pp^4UQEssj^tG{KwKG-^P4JG7&rb)iXvBTo;bxL-hS(tN6;v({ zW$y3)vH(jzTKt8!iMQANZo%Et?=qxve-VWrp|FzvV9iCzl<|HCdlZYU3kjBeb;gI> zrU`~ratsU)@OnqGQHh1KA$R)IaNh3HR2m7vpCb~Zyscn&jNINB$YQox%82aY6CFxG z_p1$YfJ2aoWDyF#TN4hjv~H7f^%k)!n4wf7_{HIMqzxUymULAA%)!C2zmQGu!Of3y zJ**~<izHU@*Ehr%^5cvELJ8WG?W1Q+GTh=r{I$8R@X0sIoArK<1eNrbXxAi%Mvy>D zdMf%mBT?f);V0n+g8DB$6goJR2*-~c9EAI13h$4C)8mf5THB#AdheEl6**Y(qIeHL z@V_J8Po_s$C|6M)zE0rrRu~=bO`d*oH{V;+th0ZI9$;dPnOtWdNs-<_lt%6r44@&8 zW9Y&4Q;MlRYW-AkZ4X;fqS*%R-mY&5eMH?(bbni_b>4Jj`Ora1I9-E3oRJHVNvDmi zsJj-FP@vM}c0*vQ#adL$qI=@6Ve|V0mc!voTCHl=VNz+m*U4o!uIqVm-6^AZOzg#E zzAc8G@IX{jUtLBB`O%oh=5?QT&fUVm7;4lfQS&j%8tb{S>GYBkW_tgd=|YuC&)bvW zBh}TtqHs}l|ABO!`Q(kuanS_$fvd43Dv{hvNFVf@ry|x4eYK9u$nNzzty#6XjUY&C zCoQs6Wl?t>ZUCEb+rUPf2OkGc0M21q?2{@R)wkd|zxF8aygvxom*4q+ph}W{BKbp4 zVS*dQG{1$s>4LyHSVO<|R!f-${k?%Gpa2jYyFcxlGdvF;c6FPm-_6766xsIQ3hKEl zF@Hn~tt0=KDpq0l6ja_G>pg6pi9UWoF0pmyZ!sfY|CF<FE5EKYahe*wlD)QH*P6mH zZnJFUaTpGsAbq%VCG?`vt|--NcN4Uic6v18Nk@ex-Z2M6rCkYEzVKgvMUKm0258Bb z&^duLr_{1q6JxhpD|XO!2tkDX!48kG>T`lh&rjrHi`JqKk`%Ms^mA38`UK7xtCuVT z4JudZ*h>&_Q}l*KG>~k2A=yYT%6eaX)pK{}N`tiWAR=&~JXDgX3zpko^KTc>NLcvv zD^43V+8z}mTsFn6UOj}kZ<x%rtlp->|M#<tKEN|yyuU*z)^;)U4r@JWa<)`QE%c)7 zobc>FPjD4Hz|Vi!8%vAO9gce!1jJgRi5M{6M<<FrK)G<8ovJ1Q6BKD>;lxP%<0AwZ z2jR)c$ogKt(<lqXbQRHZ2{CKbpWd=4I@sBH`%TWz0%4$u!sD(EsA97k8n={Sp-#>7 zD6o+*zp3`jXg@zN0)4-L7S5k-t@ZkFRX!~Fl{~vnh@~ylU;2<?7LY$pqX9b5-#1XH z8aMGzD(Y>KT~%2)rHf*K1xmOkB>5!gRE>t2?6F_2s@Y&U@Jq_R0hJJ0aY8wIvD%Lz zqO+-6(Ijl)B2cBw5(kz^N|EQ)KAeDs<m<s%*@J@vi^ZD4p<~TzGc09gW#;m@&ibY^ zK2gA4^cyV<QZgCz{rLOZ*Pq2wS-D4dpK6%Ni{n831R`~dzsK!2&AlXRrp9g;uB!%G zU2li_v}aviZ$ph)oAsxak%}Ay_B8!eifCdqapHncoukMgBa2GCd%ab6=VNwbmC4m< zzBKadE=f;vqPsYom(ep1nkzg`2!@0A5k=!V5id=kCO)@93qzNP*jjKi@Euhnd#7IV zFRPCO@N9exb!*oN6ue*cFS%mI4fcHENBCQRxAQ3+tFtZR@eZ$#jUq_>z4lkR^2*@b zej|-V`E8Qjug52~iRVu?a*amtLktPh?3%vM_RBvh-ibyL`k1N~P>Wsvbf6tHHWVA+ zcqS#>#aqVciN}N+;1HoX5@X$zfo&)-h5Axr(%o8!C`XxnedtvE^Qt#TRmsEvKHACg z=IicUlyT=;Ry)xv6eA^CRx};XD5lX;)w$n2zbrH>dF*W6$cl1*2d#kv$4MlB4@S+d zsGzy>sQ>}ss|d&kk_t!L9~Umo{FIXFD`rFqad1GzS4>TaS{`BE{*#x*{tm0Hq<cVN zDIn(5D{%Z}*<wC^J4$l+H@byJ9r_y^7}-aPn$C#Y<gZTh0@!<I#<0Wj^XG-um=8*M zVUx>gJ~=kJbWQ37Wa^@!<1LUEp74hULFkAKufO!1wJsRk(o8zjkSVP$vVj=574Z!} zMn&jK%=NFbogM2xRN${5<IL4$mxhue+m-i_&GUVIM}Nyt`CY7hr}}wfCfm2GT!5x3 zm(2Y(TT4{0{li=GqS(rWjE4o?e6G3SU8;LI4NszPQzL7myQniFLKfSG%*of@5;xAc zTUtDr2_-f}`$<G^nlqqnO%7|Qp2V!WjKaFIEoa3w;IkbP1EZ;Ois-s?R2gI$-|SA& z$FuGN-Nvee3R;os0!rmxb2B9Hl#3bYLeFglDRlp&2PKjO0yR1QQ#DIVB?Rb~rAF<Q zf}PSXzXb8-&M7#U4O%#G6;RrYh`wQP6m}(N8`^Dr6cNn_)#*e>+5{s*M#Pwg6o@v3 zsc|!yAR_^?CR|Tt7~dp^Uo=I8AT~&U7T|OQo=2p0W((U(Fa;SGtvVdw!Ixe?FI+iY z8_3+2@1xz_{Zvpn$X#fjoEI~|ys1`TESP5hT+&P9b;=fp9fz<Bgpb@<bS+Zgtdd}5 zbXOt&`Qz>RR>EP6G0&cs*9nlI=YnvsU$fTiXK{C<lE$?lG>EO8wdP{uRy`Jd@bA;| z+s}^&#%F+(pc~&!;0v{csP@<;N}g$2Lu*3)DdJ5al=~2R9vB_*yvRj(J;&5oDZZ06 zX-Of4V8rksA^HkcOO2xWp8wh*|Mp-$2R&=}VjJpdw}>p4N+@91X~z|j_C3GrmKsp- zYkryM{AUFHCkKc%z{`spG2Dh6N#cmzv`qYGh+W@)`?u0q8jylU+1CSbM8qz4M|1lU zZJPv?;*UTBqA!nJ_cvI!kYb&BRtw{sc9tYbVY5@}(T;6E>7V1ATxkEoP%e@VFHaQw z#&-n4uqh>3(tTtwPkFKfI6W&NX%Q=rX0sUNf!NKN65~EqSS3|;j_~-yCs{3OImBFo zCTH)w@+XniaeY_btku+Ta|7{)A0j^k)EN^jV*EH;<g$XPS_`Fz3!z-?kZm^VZngTw zlQKmk=p%#b_-ontPt6A*{!y0~7mDrPUZq(^6WNj$3st7)!dHS}zNQH=RKAp0Ko9bI zP8t^|QAFdQy9H}x?K{n9qD|8$`CfbX6$zL!3EiI-ljj^OUO$W$tR}-Sg0;BXX4I&~ zJ7OGmE|_fX%1EDZ+`0U8%3RX@+%E%>K6}9zF-I$*&B;ebt*Tc{YDKD#KE0ttc$uGa zBtIio9zXVDQcjRvYA#gqYnVR=BZGDk_3M-CVLYlx(WY%CJ3jL36da-qe^X|f#NC#0 z_gIPC=j}x1{2B{bt2mE`5`{f^_A@JhZoyH~Tf5eKvyyvm=L+fD-GnTSuwlhg6PKO` zG;b`UgVWQEyYY+<%>q<siuD|*u*dnNx)o6LUU_=J%#!+6qc>YN#E1;~Wi0~lGQ9(9 zCS6+D<}J5Hns3jxaZntdvs{HkG@};CPa*^EmCB!u$C3~9XAYT&r8@q7K6gn!KFN;v zMIxXm3Gf4}Kw28NhfXlZQ!~kVf13-VVbTJYyX4w~mv>FU1S$#dVX#!3b&mRij8x!O zV&VtI`JF^k|0s+Ii?QG=pO5V*0Y6at5aEjCJ8*iCabt1eql`C=B0eq{MdNInBJf*^ zxe7q;k{AbuyDl@lek83uS!pW1&qAcOoW`aa*6818v2lLzfMa_7ilXEEW*HOWWk}Mt zSdXQmqlc@W!!Y;cJDdA%vn#YV_x(ynY>2~=PtUIv+JU51%a*5`B@5uLJnx4=XJNj_ z=2|5XLdO*8X=bgu@PH=h`E%Th^nLO)=Z_Di_l0_3*Wx(7ld8(u&utTcy=Cs5`WCxa zzM}=2o^*_Le*^`|L~5~mDJQl-jZDdj&2eTC*oR7IuV~Sl$GoWQ`^e(zSVr%usZ(G< zkXE|$N@5Uo_Yi-~slY!p2=EVRCez24q6=5cnckibE(dvzHwb6zCg&W^6z6ZFILF8f zo_R^{iD*$Imr|61wnc8QoUS4_#!Ik12}Xw+vIkoqUUOk%PJA}AHsC$0U1{BXNI5!b zgToQIM2(l%vPkr%yXj1?K^@rq`O}BWcSIUtTV7DfQ6_(#b$I#hryay}uHxt^0SN@L z*ck&Nr~`uGrRm@C-_9V;yj+W7CdA>7DMc}0)8oCG<DM2(`lQ9X;+0!SqmV={LdMWg zML>3`0GENA8u%MV%r9ZPKB}g*2%N?P!73hS^vQF)M6@@mU+i$a)*o2lqM5uDuLP+K z_Ktc#7B@AWn@)~Tkk+dnjn9cwvN<2oj`<h~Yd#EIxD*N6`*8P-T0|O{q|CIKtC>>? zXV09jT!{iIBE_Xe>ff|#(?x-qh*CVQhvG3Z2*uR+#6Fkg_cI(uYF`_8n%Lb55KB%B ziYW0pE~l3~O%;Lf{FgujE>BcaSA4W2!>OLxenaEcIv;r8Xce%pSxlb`s4+TgjCCcM zlase~E(%#m{&FQ7cayNAWwPeP<`WXaR?c0<hsUN5EplsVzXfe?l~yFQ|MyXPMYN97 zHS=A>4W6dhjwi~SrIk_`N$5Y11d>G#jV<XPZj%Are{_{{jQnfR-GX^WF9utS;j}T} z)oOToet1#CvPU9Okz|lhMfDKv&wuF>9Q0A6_l)B|K$jp8n_c-ktf3#cZi3ef?QyIw zOp6=msVTOD67%*7z(t$Vkj%`WzjAeqk`pBp`!rW*V^atJSCwZ53|wqkv}=Odk0x;X zvmg<ndAL7q=sE4HnNVLZmXrJa9>PEB4`9w|7V&-Zya}2K!xm5CcQTXEGrcdi7+>lz zw?bNBeZKpM38iS%S}ih|&<>fE`;A)h=cBSJ{XOtovS&1{td|SKaiJBc8f5aQ)dR<O z_P2Z~!es-CYt|mSFC{(y##vF|Dd?Co6EQRmh+DgOw$D$Ct<n<i<(c{sHA*)zu~$|3 zPE-`URBzu8R5!&WC53RB4$+j$;Cja55H+_Ev?h&F@Yt}Yv&A#(`HzIl4O=qt^OHS2 z+)4x34(=R#A)irQD6%>Tc}g{RWuU_c9zuCssc_{<1X{peB%n|+)t}PJFSt*eSss<P z)fhZ=@93AG_1+XJ%|8Cu6>Z=CL>$$|L43+;5bU5KYz6ufr&9D)_tbLW?SNPu{tVt8 zjwGx8{ew^otappA{u#-Ay{msC3=J4ec@V_XTO5UMa*w-vWb?Cal(it0mSnr-x<8+1 z(!~pZyH8ukp!)cVD@#su%;9o)caX?t#bEW3AT^33;_u&Q!o5Yg^jl69g7Q^>E8#?q zAPX?7Df|(wYkbEGr{iTrZv)WzGvX^`e5E&q+ycCIK_?O8l7&5gHZA&T=NmFndGaza zA~HU*b3U}vAim03CoQl64PAZTzyI4V*-o;OVxZ~UYdvcO{k1~~B0nBh>&6%rq<@R6 zV<RZZ=54itlel(+Z-RuuVf_G}Pl)qlNO5Iu%VQ_)F%@O*GmaP+zf|7&a`a_R_+pON z-G0tvn}|5ud_3KK|GK||)+fwaxM-72olX$$u7_jTCgOiJ`*>oXVU~KswKlIz#zLbe zpRs_3`x5WIN47W3l|xo5)^6pcT*P`byIX&-QuoJ}`j<L(A#vZ?goUYU-mJ9yv{%ND zGR6c<{~{|;tw+cIMNmTrHzkfrFBKjf9f1x1%6*lYpUm^_F8K~=MrSXYjhw!H^+wox z_2wO?3Rg_@Z(pvX8mVS-^R81IsvDDp)SCAh`8JV?qp^CogAKitr~}S4y+|a1U=KsJ zI;%!zoa*2iwW^`5DlSjyPq+UZVhN~ykTs>GBgyU_kCX^9v(v8fwkID8(TcFi2RQW` zK?CLvLHqH}aqzRE4R&dOCFP%PIyHQIUeDFZ@3Tgx`*nitD8FxTbT1cKGzCm!P-u#| z7YAGVT)B_~6rJe~&WRieo>WkLH-^LCK;-TO;AnH;42Lo^39!H_Pb(7OT|Z+r)Ofvq zF{XSP{553xr-Z|D^j}N=PG3%N-njE2+a-?LiYWqKc@{QeDLKoc&Dqq1OneXp_*Vp2 zz{UJVq8MA?nC-s!q_qUl2BytbH<m1b-{ji<;t9`Yt<7q-tGV{yai+KbrdRB%>U|&X zvY|fidXfIa{yy9Mt-LAMnyNME8G629)qMO5H!Shdz#O{!KH)Ar?lg4q5T;av-4cv^ zWe4xQROY01f8D-MzVP$s=Ha>6AC+~k#R{_;8WUsPrw-x2LrXarv^odFHYyV<Bgnil z&fmd%OVNNXV4McuHhFEXq!UAZ*PPb9XS6DB8&1q@qozSs`ua|Y*E^8}?k?T)w?Xn# z*iV;orvFv~^26cfbMOV>qVGI~NOylb;EgN(f3$(0Pa*sxd8XNDvJ_(O{Je3Bg;uQs z5h#gA0sn7}=dE4$6%qz<=9d(nm(bbEZ0eVH$rHsrZ?nZ}Kqe~@X)Qu@!39-Gy1(au z^-+9zQ)S8;s)2}NpYI?t4~3#lvJ9e$*0yPU0>Ecc&%46g{feooIsob`N7eC))xM%| zf$p+;jUDWAZlWa_e%VUoHL;gM#R{NMn@3<@3Uk@Bhgpvcm{5L}hA6|!7f*tot188` zL>GDaD8Bq6PrNalnESqFjjMw~k~aH#Q{Uh6jY2x*O(SN9%7p%`E3<KR+eceYvmpZa z)1*didV)?uL^Z{1(5LR7SSoEk$BVU6h?-iVz}4q!t5ZuoUIx5N&k$Q)=+9x60_%O2 z>A>h@<>%+8L9J8Dk?NhWH?>vZIqK-I`qTwnU{JZ(+m$EA-U=~=xewnO5bbdyklaDg zF)%6}cVy%MWxA#;;kAF{;`MPbwZ=`)&gjEDG0SBXi%wmp{aPEJ`&s7k(<YLwM~!4V zN|{vY#rufL`J!VJCoGD0&_1E0uXejQmB%Z-G%dD8cLn$9=)NT$nn()=W)a<oZx>HY zO6eD~0TfFeMewDvxPIkUeM^+WT6GQ+h(QvbwO{h}v&ZaAOvf?Wo28PvnM?BmdvM=@ zvCWR9cJK+|_&Y&<qE8QcLiI+3W2HVEKi#|*gVkH!dA+;9a5}%rr+45!LQQXne)+9X z<g4MG7<^1o?eMvGqCioYc;+yOjyQ+`TOu!bKw@{YT~(V_yp<5Ikx|Bx-NLY45XZK= zYi+gTP%p#yc7mnPyo&HBO!DqeQGtFHanYMOu_PELBasGrE`f41X>iaQ3L?CC4>BY; zEKGb5zmhQxKMen~AFj=aUq3B-QfF^J$Q1ca(31`(7##2m1093?c<J<Ph-A&TW1Yh4 zeRrCRg0u_`qHDFpd%4kJNf{fV3p7WeDhd_eNB{b9s1jt<z7XYX<Q{x3Q;dOkTK$vi z(*FnP(_DRG{gyrbhuzl!_nTQk;80Brfph*<yP3vSDE0laO-puTo7?GX=5<seY74#Z zb5;=bFZcnKRZlWu?cq+-VsDu>+l6|f=i0mZY6srQ$w|dr86rRiMZp@k3ODd?r^w@@ zvey)boh#RlH_hlxD+)0WS+#z!JqJ9HMvFC;Fh3lzHzpoZ4?OOVS(Q>fwsG8^8$hj* zr#oQfAYqHAE|MS_0tmn7jkzLq6qOKUJk*!n6npZCo>Ap`yOnP%E^~e$+0Z~%*@;7Y zNQTmAY`fhfy~0N90$4H;_INo?5!5vBi%h7bc{N({k=20)4*0HzG0q+?^uLuDDZf?A zAQ*f(APdA^U2t$Fsmq5-t}wY^y02IXad?t=Z?LR5yi`2oj_<ahUPXEUS3&oiUNp^@ z2LY8eohIkE$vn1J3p=nqp{kU7f2$``<EXLL$2Dy7)bHfLy831fDf-~YY~e3v`tHUb zQE9%-a(rrk)d?3rZ_5|=IclTuDhVYC(-HO{2&gL|typ!;B|$pSAfH{<HwyKdr^R+H zB(Zmf-OwkLsPpk5ayLzIgC<WMpAq%(0;fWJ2d&F6GC3bsV2U-^Y=7ZkCQX8^qU3*t z*LIoCO8L|UYgfKZYt|c6uOV;3@&&ymjG^`LFQb{N#RUTb82Hqy8b8YJ9@FY_W>W2u zHk17Z|1S)NoK198jsU*2e!qh7vBxp>u#XOWv}`|ns;MvFJ^YXc_zXb&x}Ri0NvuTl ze=jCH)RubQc38!?CxZ3z;SYx{!jNfLq*%t5uuPsPL#^a0%Jet`A^;#vl{jwD#;`q> z#qL)@W6|n_?|jk<B+&9coLjxNyXYi4s<<Y}B9;PNb156-f=O|NtRLce+PJ6r$CkJ4 z45xK2%=Zz;mUNu?zsN8+3Z&((o|^OcXFmJ5o?zhazji2ua`7KQ7sksZmXf7iXA=@0 zj+x^TCHl+<R4F6}7d>Ck86>{9R#$!P4y0GnT{HvHsfvZlxxwSWtQZVHC$n{rWL#$J zohV@LGC<x<t*qq4MV(*0RcP-B^LSLSryXzeNr9gLMXT}R57!X*Q4+zvjP#?)bzFQy zP#NU;aU>;1OfOp8(J#~^ononW5@X5s!J?)z?HBqVL*hzp6d87eXaoI60am+-_gyAj zXR)@7br&BOaYX=LZ6r;w-kr*DHUSMki@(WfZ)|@u7enSZqY|W$2qH%@{mwKl79RCq zDO_I^*dR3ulUi+sl%+X=ljlefdOvrP8g8IbHc8#J-$5+U@S|qF&Nk)Z$$GX#Tt+6; zQm~56ncjWUYZ|*fzx3nQ_Pu6C{p%7;q6*)rF4?BSNLr(ZY6y%I109{+a`M$bRC<I} zyBH4>-$&cgVOz%)Zb9rx;eA!w2+Cp4G<(N)G6^H%#^|_l8oRi24P@UUSeZNcoyv~B zSw5NEhqY=R8XvFi80i|wKYLvdfr8N2rpWj^6e^JOL2d#u81c_CM*{|$f14Tf3*XCT znbb?%Yd|jUvjm!UQ|`J)A(lLQ7q;g4Sz`^BC^Soc^(`#&&csJ_i)E8@(8o^fS5?IU zX~?dZjRbl<p3tH7qrYWa{PWd5IAvg0f^_f&vqj05&rCrfUI}y@JZM2xt`xZT$~IMa z*KWP{w;UVQwEK$X6hB?kEIK-yr|N0i*-?z+u|H^!;kC@$i!j9(_%p<>g*DLUXQ?EK z7t9I4y&f(SQ3;~&CDgmKxDluPv=;KCy*>Pt$1C*FUnqyXEs>O7!0Cexr%X?R*v@un zfleD<vW8{EV(t;?wj&^>R#iDg$EH|I@J3?wWC9|z;a9kQLnQwI42;V##=Q@(-rOq| zexMtQ50Tcj3CE+l)4bZE!(9lr(p97+OIo)Padc!v=kx+g*om{QK3r=ET@E_ZH(ZX^ zrfjv{TcM<y5)rTat!Zqgdz30MyzTWZ-Gjm~NHmHfI4co<EajK@Lv-(-=xoOL{9%Fa zkn=kdi^)Hq&m-g^^0PSmTr-XRhaFUvQivZzokW{|t}ay!WB?X+yFgNaKa3&tuRYAq zfZkJH=CuYW=<8WAosg}Jb-6s1NuFMMk{Q;=_AH6r>QKdhc=`p26Wx!RQ#6fuQypVa zzd*Gp9u%+)xg6DO_ilA`0f@-Yx?a~ab#tP%&UAUp=ej*ESC9$7wH7>{M)}ymIZ?>f zY`)bO(I?nt-*(qm#|nk_kd{HI`zxR}YoPiNs+n1GQskwn`w;$ppQ9v|Kma#H!MHy% zu1)a0r+)Q?sE2_Z$VXWjzwXYJ?8uP}A{Rvb1kC0QXI&Wd(1jYySnjGCzQ_V0>*|p^ zbGx|O?k1O`Z+mHOQZP}fO@|qTIV0mv^{J19BcSco`s-c{dWPitX@G6A5;C8f?^~_c z;a}fLFo*;~y!)>39c&Hj^*M%v=kz26<D*b%6q1BzK+z2Hn3mt@u*sv)lpil*qf^sY zz47HD7?m;gzNGMl_jFS>foOf&SzCIUxdTF>62>vKYV>Z}J9~s3uH;WK^^(Zo!mdon zv}I8M_Q1g#M`rtWg3MtoKBhW)V5oo8fFfKqCq=rhXJAL~5U$cc0I*t>_c~&m22$PV zeOjb?mC!QqUZ9x;7JB(>p}<t`04&Rgqh{oBA}f4Rj(<Y7JSXfJ7SFxXPZ&}d5B_Jh zS19L=x%X9$Yt6B!nbEFh*Q@w^vP3*h!i6Wm+2!7_=Dt;}xsZ=zEy@|{#<d%=vk!|I zk)mogJ)+9riZAf{pEtuYoGqM5Qz~e*MC~LSNf1eM^gawVV)7JQ{EGxaG_kl&l@7{> z-3{$$938WaVP`zs;8Mqul~k+l1bV5bnZel3=_18Ct>(*#bOcpVoDV#)Og1IW$GXd& zFDd|X!ujGc=Z^m0PpTr4g%j<Qa45cCPp?l{RmBGq?VDAE)W2`8(J%k=B2ZLb2TCg0 zGZAGFNuPkIkiPbQ!H|^o!j{`LA8@I5t)nWp?V-Ts2$$_wE9s<+o)p-7a;^Z|lPdVw zW9*;&A!-+)H-s>tQH5AcedQA~!KUiZsPAXdJdD+&3)$)F<3FkxD_kc^O+bk&!kovF zs1apKKft}7-@nP52b@*4zT;Rg89?9_fvqEO1A)OKHe6>@*dlLGzm6nwNO8Ym2$@AS zV}IN8zbpXI&yrHvD4d87gdGmhX|dcpyhXllrq5;+I19uU`6>`AZ=}2u&KgE%U7M9& zwiernbDQqaT!^TuP>r=kFC8giWR#6ptH7YNZ4#O}!jSY*Byk2Xgj|4~&qShJzug7F z)pjie8g4^>^vhJ1wowl4H&cfHAyIL*R$3migmP`sW3-LF=;SiCNZ#?90puKMRp&zF zX7^aST78rQdiYR*B#6o9hr-3UIp2D_YCX>+`+4wlgU{;60>x^h^SNnPlnb19x1vqw zE21RE3!4g+Tjj@m6<%QMQ=vdWf}vM{|1=cO5{6F1MEWw7sG0?oMwRQ2`|6I084@vm z3h$C?SE;(xt9Fpx(SP$B{ZWAiU0h5apGx2ifcoe}M1(F1M?@Jn9ooI0#4<l~<Cppz z+)3QX(T%-VrLLv7EEl{P-qoEcSfUf!Lj2^2Ud|Qq9ku_=#fMDg+Y-9itX6%6b=q{o zLMZ~f_Fb$BjWqIIk%b{qmdcKIC99p7&`_~@z~U#t)X?28DO&(Z?l->IpE;(Gqc$@Z zeBHnC?MDCi=Yrs04yIU%H%q19%>#wgj->a^Mld85+~9L#xBEBSWS|QczSLT9EUQ5% zQ?Awe29CYNPRZpr`NpFOXsXFP;t^Icxp@B(c1hRxQiAcYUwh^CnE_vP9mbE&53hO| za`b!qEYu6%AI4ex{to=bn5K&zC!;O_v0)}DY=}k(#K0OzG~9lC6x!b1={&`t<L4*- z{=IQvcv#v@oo8~YIyyZmqRSutFPX{b5bxj2WMLPkn=z@-pr~v*t_Hq5RDn0JdEXd% z+U=2LW}#-N3bTGIGWY3q@hmPqMV+K0jr#XQ+0k)H@ru8o$hRLH)y3g{Jr=RM6(k;O zSvDN@<9wrGzyQJxc`bDF{~#0(W5$;*s4~5wp8a^1#jI4L=dW)XvU>M-nUI7ei1y=8 z(#-V4cs7H``|G11FPoY1bnhoEId3RZ924nWG!~%EM1QDKPhzJaZRsz+l-*|VoAode zq*DACbUL{~$vm;~{_T#kFCkc&DPDZ<W^W@;){Z@*&e{w`6nm+&-@xmd$gcHT{WYpG zvpo>6Ov-H$T}Y{V%O+~@7MQ<5-GEjexH1fhH!vs^VK|_Mi61HW2W$B#xrS$BShmW$ z7+RaRL7LyEtNCmll}!cperyvVt0xUJYXvJ!18<EeKLix*8;Ult+Hf%<yfd17KJabx zpLN{yD1%(1TP)Xbq<^;&QXV#wDyjE=zB9i%UgGVdu0|+up*4!<xEsz}NjZP&vX5@& zCvYjA6CY0({*r?-^<WDhF`9lL@KpXzN?2fs`&T8IF9e9@{|a;gf4%;qpnxJ=JtgQG z+E5zHxXDM=WEc;-4vM`-9!|KWuddWF3kWwBp71%k1({GqF*T{9)96SVsJ`uH>G?x& z7gf)oea5`=opZyOtS6@^gaT{XR}LcB;1`7^M)%@`PIqoAEh-qXI&up))Q>H~Q$2_F z*HkTC>xICp?=5QIg=o@Q<y4*9QM+A-S6<4L!^f62G_fP%radnyqBf%b*9<7q^Bn<o zDv|#~hRDdOgw3<2_@rJLXaShB=Cx&>A2u@!`fz<ftOHka5JSOLkVWVG#Qv>Gc`Szt zrTu5O3*n-aA<}iT5hn!j@b3G3r^qD^m9J&Hykn`sQ_(j4C9x1ceBTip6Wq4|<CjtO zkb_05($?}t{y6&~+mr%an+P(zPs5j-sSW!u(p?K7)c?oe1=hyMzcMBAaXv(8>0|Op zT%eOGbw$VsGd**es*I^0{Jf@ESZvnDaNipidPF&H-GVyMucF6h|6ex>zpZ@V*Z_zN zoJr~4>XwEYoB99$-|~K`sg(cRPQ4WLmI#ctHo~iK7eSn0y&xV}<r-d*p^@`4g__HR zR%7~tL>g)tLQ8)tU~6;pJcfU)es=Lt&a{p!=8CYpZ&wmu8<i5QZ8x;X+ZC!5pay5c zfSXCylaC=4g8Ne*(e#~#u8WFr?dzTD`}I>&4_8>8!@ayvW;c_^JDli?lw~e1E)Jkb z?k|L_blQAiXW#MZ$9|gQSHzw{sQYCaS>oM#O)#B(!v=CVweTWLH*g;q?+WDP*NrFH zPE)v-J&%Ny-zLAeHtUEYj9^L*!S>j~L5;Z+npr}3%wb$Wv@510XINy7AAgLz)}r40 zb2<XjT_Df>(qH|pTehA|^1={Ve9^x3IzBhNHhHEqT8n<;@*$k_ZK404KmF?yw_khV zy)vI?>J2^~^IUgjvsE+pFr8JFolYjFV1r<;TT%02PoD}^`#|L>`P`GpWykAVj+SpT zTloYguuUVTfgN%SP|<_@P2n?Q*`PV<#lcE5FJqAhH$my1fzOi*x6OP7pN_YjX%O-L zJ?1dfq`SXrzt`xca#DroMk^x43{-_Z(6t0b>@d1kLLxnW97K!WA{41h9(Um;d>Tg! zmi`_sobdke2jEBP<$Pa0i;b`->OXl-gm)Cxy8-oLB-21P#b+X$d#pV54A=KykuRPn z&tXlDboZgVF|+w7FaD;Fxe@MR(#>#kXIQD)86o_B9etX)0awq;Ancyy<C8%=BfNvh z$=5%7e~d0W*D4`fOPbYBf`}LxF_Z@QgATchjn3~=)tU*$iYAbVWK_AcIaE}Q6ki#I za@%xt5XLnyesGYZg>Vq=MR#`gi<rlRIypJUXhcW-8brMOd6Mgc$IMQ%O)t&)jr%>t z_q*?m%*<jRT5wEd`?H*rG<*8{I`)iB3F-PsxYtcG$_rWy;Q&lAIO}w*JFnGKoWpUP zSqI5JZDJI5C&8d>^w<PGzn@(T@8)LAKEnH>EwxfxJz;fRBygfR8VlTrJMQ)}*hWLK z<1^jf$Gy|wX=5-{c2z{Ca#jW$w3|XB_hg+L-5I2QZu;Ms$`&V2o1G8*|M^l}4yP+y z4Ql+rB<eor7^w5P2>i~<`SNLJfP}-emVMd6W4l<hSzG4_*Y$&t-xbKO#H;5Le1=)E z<SkQ^p~VmFdUaSqOMfe2w^1QU{n1Un?Cq9zK9u`8*|ZI0Yjt=1z(`YHYABb9W9st& z%E6SSd~*KWL%K+qFZHNkn#6bC`w|Z(udjXK5HNkrJ~MlcPKcfzL=8K@=AJYkX2`eB z(9@<5M`lUcpUQ#}6ihN%sX#SWGeyDByX&w+j}*?YE=lfBD)X?*rV7a3Xi88uB2NH~ zm4ECp9WFtE{FX0sA!q=&S#vy^?2aS}C=(kjWr`yR0j2GDH`N$0hrYqDf<__&$<>|- zkSE#`C|IB&j7r}-`?k{qRE2Q+%kEzBho0$<B!-c)pFQm2d+_UIP%-x3-nkVPiNQI8 zTH$OsdX6tH*XFm4eo<*sRB?B3Ub{8e=wd1kG3vnZ4;e`)JQAGv31~|uc95u^?W6Lk zJ&ATF^a{a{$9IuFb~yUqvIVRCf}&PBa{g)gIurUc4nrmFlY32d&DVmwTk%@L&JX3x zV8DM_<~l*iyE5V>pG~TBO+<w%qJ7px0ZW|OG3n~x#wdEa{kQqLZ?l1NC0)AkG6`zR zWGL5L*!Q@iXK<=zBi&yxNxxIz#q&13!DT-r{E+wPzCMj9&vVIxr#<^4@wi1Z(D%FH z;^C@=Uz3rJQj%THyI}}mnIXz?WT>kP8asB*JEgL6%{FK$P~CCYr(bfXS(^SPWP5gT z^Pe&P|Dt-|@<2X{i9_V_dyUWSb*!$l@T+?NPvd8hhJ#N*#-Y~L<&z94&^*_C)wK0h zjT>&;>*0oMB1#Jg`%d`V>Uq2M^#*NlD$E1&!Q=-0oRD3u<E<|gc_c)gTP$D6)|uD! za`T)b#Z*aF4P?Rvn@4{YZ~3NcoXPG45n^_y4M!;;hTqJ?&w$69J|Ws=^L)3AF2WNA zFO!lFc3=0aV6<UcEYNk5TH+zSglgI_wN}F%S0fcN?m-ylEhI5Sl8Si+CQYEqkz3e! z36I*PE$8mZgy!AG-XxQ9;QkBXMDJVtK=P_`5(*H9>|$SJvQPh-BVK8V+xa|Qlb|l| z<{T8(;5F|~%Cf!3W)BHLqg8Fe`a==cBDbb>E$m<YwkaVo{mu@*Nfm0iEE|$dts^<M zb=nS*@LBelV(F7EI^#HS7`>h|d`W*2;3PEg^ghM-YYvU;^{8M|<=rj5XO#p)?ho=S zJ+7?^)4c5WgYurVH<qWyI3haCcy^VqIL|>p^1fdKAQJC=a#C=V@J%9~uso#`nXgPo zZ-G({Wv$=J7ev~LPPDyrPdf+UJJf}L5JZk$nC&TIpNrF(zs)0n0hoZQ#)Kqodu{68 z4HxO9`AZw6%llC}gsV+-9FoU9NG_S)?r(ROwO-d)N5@cyUdbQSMH)MJTt|LFeMLV8 zwg(L$l4;14nL>QnjY&>C%Tb12X>`m}Un<hEA_Z05&<LNR_A$4@d8*WJ#n{<zgp=<X zUrEnp8{I7IJKeJsC=0Tdpd)p{+X%tlwJqJFxl%6srWI0D^14v?cYc^Ip2lP;N(~#z zwaSEqgcfLKlW!taqT*ZFJQ^UP(`yn&BzATq!g1k}Wm*n_2?E#lK6adb)SYODGmumI zZQ^Rnr~se%aU$Yi4IaTUj}uK|WO$e>`_&%FHid7MD6WJq=<MOESU;T4Us(7v7BU9= zUcxE+(Gpu_#*GC+fkrmBzQ_~&^e1LZUK6VN)3@jsNCxA)bQ}uF+-TPi@89>CFnB3Z z4y#?_?vXbs!~<!qK>RokJ$wao>&>Ed!@564XnT|Kyq@O0yb?R!$xlGRAfGAL;1#Jn z1iN8_gI_xlH&3STlKZQnA#%%RV?l54{j}X*8t?jacesJ=%t)0@eEH#}_Sdgj4w_h& zR1sGIlb?fkoW?5CLbXt=!g1x3_3;A6Z8}8QjiZT=GnXZ_s#Ax1DK#Fd*}vktWGm&o zAV#COd;|3*ZT^^7aDpf|oaxG-?c7#@!J2X=x}`pIO=}RDbXwIoVE1G=yK9E>h<mk^ z_k}{WKoPD}nPY%8DaesAqQ+Lb@}|5NcAT)5I)Ljs{-?G)h>AzTfK{ss5wZ9(<euQJ zP7X&;I0?HDCDO<w|AFq1#{XhrVPStNKWb7tkz1$H@im~I;>DDc`N{>k;~TyJ=)t?V zPII{}EGoyDMkjN{?Jb$O4?%$gzou#<6BEYBKf0cx7<dKm*+Uj#LkQ9Jb}lO+@~X&q zpbOr$0_in^h{yWQegUzBCC2nWxtK-DKk!#T^L8gYUVwl<(k=I!<f$yDxdd`Q0$90k zIekZ2!PJ!W`_T$bVl3}eNc5cvdX;$w3hq<1^p3f?DCRYv(rScHKVEUYrV8dKe?H~0 zYQZkCdw(Ku?%6h~@Cw50|J-EzV&E553hDmr6H)gPe7yhX^Z+!c?xOy~qW3QTVw%b` zDbjASV=#4a_jA!ZS%D}0T?K)r5a0T#%Hd;=n8Pt}i{XT~07aa_^x}g_d=3^(Y$dke zz@~>32I=k(f=g$FoTeWo-y>3`XyYYWW+P}IbhIW#)mA-*_0I%#yN@86vxL`Hw-&zb zb~esV<qL)#-J_w3S3X!ZXQ1s^{#bWY&;OEU8dQ9P^yb0%qmG1gUalRSFC#~idroum zt1JKSte#NM%*^bVryLMNTC9+wDBRra;-!$TUbcW6sL6sQTbI3!L|(2km8DnyGrkv= z*U0dSP1_(OCIYYxGDvuo<0|;EBblIZ6}Ifng}A<xp^mWdg*d<BVicR5rVBQ#uS^*a zSR0Ya814`UEKL1+t=)9dle>0A(VEk8$f8S)u7Fj^Tcv62jluKcTIBPOhs9;QuRM}c zIjEMuP4K`6PSiA}wx7<JCCOYV;6~*81Gf<|J!~1R-j@^Lb-G%#d;VE}J(GW=w{i~b z8aK~Yy9Z=OCGI!4d;TEBc=2Hb;edI7651Nk&|lIi)dvQl12RXyzx-J0di-fACYHy1 zF3_L%Fi|+um^X935d;pu0OWjoUy%tJM9Y6hFgzND9ed=c>Z(zmj-y4_G5$S6z$x>| zLN!}cbhmk=9@^)tBfa7mWzHJG=+WoN<zROd*|&#Ctp_>ptyAjPd;@4QnrBlVY?$Zs z5ZxtYqkh^_<cxbgDOOuD4rMdx(jkcTdRppz0sRFuj>2iVYO~P~fOUwuoT#7J;qfG} z4q3<^S@1GR!&;LH3F*gtb<D#<8xW$w<$Zs^z&r>mf$1QZ<mTs7;4PJ854#-Aj{1tB z>dSoo%t1$4HoO>^QTO7{(r(+Q;UwCBBU*BM(f%|eM9G|HRf9uubjP7kyHWtpnJ8`k zjJsj1<{p)xPNZK?J`6imJ!BK{^lI|{!3k9`<w#K*;&d`P|BFz4zF5{>P$F#dU4lYn zHpFlc>Ok(7v@$;T^yonqofH$&^7!FZ&l{s%mv&>HS=XM2SWMJA#aTPtGM!e{k6Q*G z98p;Kz=c{U?3Jhej0vhpWR21e?4G>ok1%2=UklpP;&RB5c2pN^$Bo9lf$n7gg8kPp zNV@9Acn%noox7PK?FR%VyCyX2QIa9B$pXzHasggwSf1M0Pqx1C&tU&Rtw5tHUz)G< zRStOUl>R|v;c}JUAJ4uq-aFux=u)!qTS&Z%yhXw#L8TGF6-!-<!u?Ij9BBn%ZZ}rs z3`?a%W^#&&#R#~YcXbD+Kbz31+pd^QcajhUnB@)`dmM!+(tUee4V%9ceDEGvjDs28 zBC8hbP$~G_`P0J^FB~sMve3aQ&xr8&;f%7-GqTkT=_qpB5d@=qfFFo(0UvQyRhsYS zJFy?`4(zuSC11=<=t?W89?cY!I5d+%bc-{Y-e7rRBknhSMtrsE&`(8kHVdVqtnS3_ zdA;uqq!CL=%v$6EVhDLmQ7`}R#aEY6(?Cl!>|(!F9iVk0qIxRbY!40GT_3Aq&{74t zFVMx>cPAM__(zvK6evs4;Q3Mow7(4OZh))2P7O$qc~dZ|Mh6nD+D3D>n3q3@CT%mU z=KpY==BJjD0yj`c<o3OD!D|sHgdXe-!m5N0tf7+c5eS#FXjWvx=&Rp%#L!Ehoiv}4 zPprH>yc^|RqjcrN^)dJ{J=t)PI!tFJju-y+7l$m+acD7r>N=Cx9cR~ckQY3hecd8X zrXrfb{?#APR%=JE894DZ(b8`5&2GzH&h2r;%r<R=e}GJ?_@SnNXgaq3h5Ey!-`7M7 zd0J|Qvs5j6H*FMP0gH4nkgh@{vP=8t*XtQtS7ffr{eJnHt?QhOG#L02O)~TIqJE3P z0Z0#$WVa!lwx3}nx-PDJB0)AhsKlUMqa`JvJ$SLcVXNn>$K$<U16R$U4XQ0V*=#tN zK1~BfZ))y1L)DR<Ol_&LEK)CA1ndT{^2bwQkqa=k`}o9qm6h+pX=C4+ad>eXlb+=L ztkWC<*#yY}n%sTi^BRGdzxN!APAa|g<EK21oqX#6opypf$o-1ZG2k!S2q>TjB~{`2 z8K}X26M?@N!rZqHxVzZc%27eXnMTEuam-d6IN3!OcF45Na5bhbCK+$LD$8hHldx17 zkm-vpj2kmu&=$css0*6;gm81{$N5zpgYy0=2c9@$B5~To)ieJg!vEPCX>{SJ3N-h$ zy}!u4PPg*OxqR&T&;QrkGdqCr#IWMBrlN)-#C4^bx~;fuBpMY7Y5!cUWKkb)P44Y2 zQ=ghkqB+^$qJlM&H&h6*2L^24ftQVh1{?#ZDkUOqA{cqVUU3%ajN+%3s92gGeK(2# zch#sht^DqPQUWrMZD|c-epwO9I3QH3-I=Y@`g^uuxhnovL*%3~STBDKvNO2x#<X+M z2^{?A`eto8>#D$dG!aFpH3j<BSSQ=OJWDMa5%Jv)SCRH=Pb|d1o;OgqQXIx#rFz+( z3}YT}wmPY%^MjBO->=n-?yldHy%Z{}QuJ*d%*ve<^G#9t>Mryuzx-@}#@!XViRbq8 zt`?G<Zw0o_H6wq5HO*5UepYts${`?QcKi#^o83!X)45}}U2Q*Y8?Z3Y{)t9{llToq zTW>y(v|*>8w%EePdF7FJ*Qis)i7qrl*t2fK$<orYFWC{G8?@hGr;A_;3p!UL2}LGQ zYg{*V)!7t9Q$Zm6e`&YGlImA~JtW340(6B`7%|JAgaI*P??(~gdD7cu_j))Co!s;Q zR>}OAw5UM*kSqj*x^0T}ep$9w6OSc~;k+&l{yO(mGvjUGdLO)G-@bAsg1mA`GH@mM zcr4N)u={$dax)8unwL)E@+nXf>dI+<zSTPRKi+kLhe6+{L7wx4FgnsAt9-8Vq=QUA zfww#$9N-AJb=k$9VCBfp55*bC_AQd~A~{;k{VJ_0k7>9v*4@yyf~q_F$O@HP3)VCc zlIe--^BV!e1ma??>Blk*A6w3%5;&fK>e7>{D5Yi_0z$%FWAl4t0P11qf$D`d5m9V4 ziVUCKyX92cT}mx`brNX8<}fsCw=kfxxsU#gNt)!xht3~cLs=AAa<OF?)Y%IK{Q3|H zF7ZOqc=E<E!WsLs3VO4wG5PpE;YEa<7z2O4?f}O&4oeW->;4Wf0fImP^VsRflC>yJ zpKv&ash76R{Q;wz?GHAhRXEjc27W|7FF&VCjPY_`G;%mjN<0OI`d=4uA3L<0S(`Dj zKAXWA5daeePS`|N{Ie&BVVrWa!Ms)~Gk_b_J1!U4PZ}Le=6;gHJQ%!46Lfu}KBk4| zFbuS}g(q@8SG<4iGx6LK&~Wuj))uGYuO%tG@_`rr!q6{NF|OufGmV~oI2<M_8116i z&R<$_qwM#LrBtY@zeiJ;Xvz*Z_`eA7^v{s!`_0d5ZfSi5gbd}B0c`EzDPV5>jjfY~ zVBT{DZ0<=>vV)CJ#~<bl2Yfr{TLcjAb#i5c^!)_wn{#06c+Ci+z`<@zah*grKIzFr ztmg@Ih4ndM3tPuvS8+ck#WY-A4D4|UJ^(=bAq3b}XIOw5Wg?0!Jj7*}9WO`unzd7c zi&ScCCo>eT<a%bi%+5H(x11HR1TkYz7^3gWnorOv81_lDt7mbhrP2gk{&)p-jsceR z^oc%!5bUWe1k@hF(vR2!r~4qr!w062TLOk7J@ua{ZW52pzdjb<Y-}7!kuG@PMJs#- z-KA+5^e&L-7i<P>%$?D^h07H93^0|Krnne!`Q(jLL#zxg-w+HrbWA$pir(0PY_zVv z=7Z8u@ryy9G0n2A2Z{67zi}JBu?mZY0}DpLAMk|c&f}zZ^k<vNFDlH8kKVJanK?&b zcbw#%p53rhq)umCIG{&dwr%~fVE~!btQ)tjINQGDcl8Uj^*=<Zt#Jidt=v$kx5e@+ z>^@v2$qQ;P*w2Df{RhpA)0SKUrZ{4u8%GK#u8M-iBT1Wyvh7DB3)!Lvuo$Q|)xM7I z>Hd9q$#}&$Gelvz0BS2jj7>G^rSCv7m>6WK_0WhUZE9FO72IG)Yh6Sz8~~TYSulU7 zCDGE}a7yS`z|>Km7{uCRbBoDPmlOKscfNZ@N*#%x!;sDlHqp}o`O0G?NubRx!3SX> z5Lq8u%BtC?W^Aa}>3;%1{>6ZtjVp@PE$Evisel&3ydTFc%*WCHpK_iA|F4l4AlYTj zlxVWE^tnOXmQ#6TF;rS~1wrfSDsKm~sbim*jUq_#lG-k3Rq~ZFJnPa!(Ff8)O44Ih z{~<QP$kE+<Uy+aohDgL2XCA)emO(PZO$jC6{ZBT*#M788=9K&^h}_QkI`x#)>#7`7 zR>26!i1nwPsnZ`nel#@qvfzD@RB~5nF0iZ|11R>X2t%>i_!6wSQk@_S)&{Xg4EzB> zyCL_QV(P}i)a;bWyc6?`6kmE2oRr<qhG)@kQdyr{`=m@nWd8Z*vGyHTel$1?fKqj| zu{VRCE|0=9zA4-aR@x8iTMGBoB8n3|=D7=si{5P$(0g!2kQrrL*dge*Oj%g3?(Lg% zfg*Uiai|2QtLA|X>_)hqdDe%~(1uKT2GvsS8CXWP^i-cuQEVbAsn1?95?`xLHG5_c z`iY_T$%_$#)EZF|gP%08w(bJ#{pJ{*Lrl|KeX{I36C^B0McF8}C+$FiKscinU?mdQ z|3RZXY?|$o8xMO<EC1v1>J7tYKG^$12>xs4;e^g=p$^!?X?p;j&`<}R;9AycVfpRh z%)3ox@8z#vso?g)m7gmDgg}rM7fYV_)X#?JULB<-BeE@Wyp~snEkKyo<ijcCfEu^t zRb-4C4psP%%hiO{kL!aJ_APK=JU_`n^wqK9VFXD}26uz-g2&wptpe@pq6vk>m8P?N z8))63r%`S;9AN4{QWx3W)gRJ;+;}{UyM^^z#yP6(N>@XiK~icu&c2ij0uhh>K!OH& z=dbdpSkw`tH4bbn#$=$5*_*&lp!&@lmKTb0{cEbBS86@0;Vj5wI&b{ESIu;Fc(SSh zKN1rr@%kbe%ir3;xQ5Cv7L;qTZAffBg7~8L&t1WWLeG<uiDwHml=7m%k6o18#Afs& zJa|O1>fJ~Y<sXU<)4H-%l&J{=_}=uHLrwy4;0!3?+cXg&a6R{rv|{tVfd|Pn5Dv+S z$gms=aF{x!1f2hX^k3f?3`NorF7JHC-cCai^zhRQC0tA|b<Iy}Q?-(~PD;}|SIf?S z%yP*y9*lf_qoC;G;|B{#nm&euXtR<3%L1UNm+1std%6ZT>h;22VcLFs<Dn4}9$3s7 zd8F^e$3~ID8EK?r#srv|a=aZae}~C(`&x;Zk`QN(yJ1ucGf;7_;n(EHj)27CMEc7r zwdoW63ZSe1ioYW$czL{c`nTc~y!;!v=yI-sKq+#c9$W4*q6~%o>cTLC?sowUZ)L@f zV$+2^H;>BO`^+=E%sjxI{b0zM9~@j5iVr(hu}NG%J#S={utly)zI?$<6)XC?-b}fj zDhs`w*NZbk1ZZezc*dMu!8DT3hBynC{#t^6$*<yd=rZVGL$iW8wD+PORIIj(T+E@i z^77cIzh15MN*G?fb28vcf6sVsytg}=B7RLsP%12_zr=6j)ogc)y!>tSdcKjhWnAn1 z=KObs9Oo^^I=psM5OW5baL$OkH8Nm)`xlP3^_P|`P)?~e2xl&=h9!kwb&c$7ZURni z8^I_AtN^;Za?;6>nL6D6duGF3I)&Jo3<nKgqNqgPc7Xt`&-B;^bixMTcj7AcFdbHN zgIB5wwrbvtei=%**99Enm6j@e9r8~Pz=jwlf{XC~uyvL}c?8;)#ogWAg9LYXcLKp( zf(C~VC&As_-JRehxD(vn-5sX$X6}1aQ&ah!suWcEbf3M>+On&%Uz=<+@yHa;F4yaS zq<$ZCR;p(|=Pj@!jPyNJ1*i`_Uesx2mxcC9GT+Rmpa;pFM{nyjQ7|^|W^=ImV<CvW z@D)8gOe(!MszP;QM@M4f31ojuXzZmX>WGdKOs54#Wm?xzsoO90(vf7P2++e|Cd6<P zio4nd(gyr<R`CJ?vYtmh-%JSGejL&XPm{0{pDjxb#;k%BQWPFQL7DPh`<_Bs6o53g zc)QL{8yinP>_J0zySf$`D||e5+}vH_&HQEE05@9IPPnob?OE?MIt&gsP4H4hrp4~_ z6vXX<`tq5uC2MyFi=yS4u;SeahUVnaxDkce5t20Gti%~C;8=i~kbjMc1dZu2waR^U z>?wVx3-^ugFFuj9h=mR@Dx41PEmj4#V(pUM+Ypx!YM||n8>EhK1!UFs8x%XCa|<N! zK~Zprw3gGv00rJY*Y5GAZHD?*oj_X=a_(c&9$~k~C7c%XNrEK$&>Zx(Dl!imun_9i z-vJ*24!D3l@&7xMl-2w{{*b?z=8p*xgDu@{w7$zicsBnJKlB=boqHw#AV7ZFyCMH@ z&sFEiD6gtDh@pSF;Og%8XOs45_rCp7M6e#lde+`S;JAM$uQjEl#6KN^5BRM~ndl4y z>5D6Oa=1wB#G9=;r=3hw#A*oy*^NjP_AZqTz$j!i@t~fUNnq~WutwljoggClnz-h& zm7e2r91<Dd*IrSoH0Zu<M{SJ36ermxv@6}N?`M5AId$G~^hzr(&I&$$Sefpwb=o5J zF#X*KClb51SgMZ+5;*v?4La-$)RACqBC+<H>rZj{BR{DO4@rE)jL%c=v4_#|c@uv| z0bapab^4GZ{8;{Ts9ZkX9G0lv%l#RENr?9`H61Suyj=)uy#C7CSm|vBf?UxY@|1#g zeB{Ws(dW<LWKgan7`<5f4qZHq@y$B74(r{1hV@?wbRtpeJ>TvYt{_(|Dd`(c&g6<O zxqE+OIc0!x>D12@n2#n)?e?=z3SjznZw!gu2Ixw!N5@za-zawx{S6|+Y@mu3MxwGb zn!Vl|b(&G+E^7@x;)HU%1*+X2y}xFte=KQyhee@mYK@YNwVkm%VGM}4KWG{(QhtC9 zIA9DDS*X@;pQ_g9FSCOpGhA+Rmk2(zs7j|iLj(7X3wzB~E0XIs9Lk><1)Bn_9B4yt z<82pL2lw47oWIzZyPp^;Cs%BDCEfS~g!}7sE#G;TF6$Kd^~tnB_iFHHdHglQuRY^m zgC-}fIYe}b0o4|cU1m%Fx$h_9MV_hxCI{3e{v^_r<H00Fmj2TWmY%zrFGgZH$|?m4 zkVGTP5tXkR8gkW9YeyMq5@^G9^mHlOMT?aW3rhW~nk<`l6@DTnw(vyby#DU<3VpEM z9maYIScVvByw(GgU3jDu)!yh6m223pBV3X_q~XCz^k|q7k0KP?uVJBg{lGHFf8&6g z%mN}_JS^s`o(k4mYVQEK70*x6Aq+fTEOMRJYB-MQtq;y>a>uk0I7zj2Uo%`G!w<o( z)kk)&i<Pk7WIjsaL+XJnVBuaM&%2+cS&3?TkO`yuehSH;4tK<2TZ5?1ANZx}La6*G z-~k;}dS4pg?aSvFD<cDq(8PUTh)B;?OgcWb0+pFbc8(CFy2P!NIGJf_)<j0+VgaXo z;gND73AKGSriR{*HdkBUKwUGwnPFpnL`Ta%pL>QXelkc-7YWE#z><Q{0z`V~0@quB zukeQ176-#AQAdHt!CRT3&_B+Q+4aFOM_B}d$|Op%3DtG?XDgPKnr2}=2N&HRB-CNQ zwow#JCO=tBe*5+7my`%x1*FQdp)W}_-V&*z97km8WEo?+cf0!qiLq9JMHH)}(&1+! zr*$q}{#A}$i^i@x{i}vi!}j^D<%$|seaQ4CXu&TXkMm|_ou}DZ{D4)<d4kn$5+Ui6 z(fNw=9}3+FTuVz!!V{LIJn=TLdcsF-Y@Qeze+AfwDRNntL$x1VnzhsxmH>h`q3IG! zEeLxV_B<UlDZuz6bjh`G|C<D3YV5ktgZ`<F$7v6Z?>K{ZGgWQoruV1ER1{l=p<!u@ zfjXH7P~Ggou!UUZx^y?czbf}h?}3Y7)6Y3p$3COmi|;?m3`e@q?%e9vW1|`@CfzLt z!Y&9NJHY@fl2BD)6guCp?vt7x?0Z~VHRQ(SgY+ja=Djk=sB5lE7LH7aMtcg3aMci3 z+t6fRrTjP>2#I8p$pvvXRk}MI&C*en<b@+ddol%XA=5cn*G<>;cXebT8svN@j{qg_ zQ5SbFKMo3CetiDNy5+Jn``7Fy!ne^}W8$2oaZ(P8a+36(^B5dJjPMTNSW3@ChJM^& zOnW<x#930T(Na`av{Rjx8+?Mb0oNEOHUBjyirD54Zw;54_#QIu_-Uv3Ye?TJYxl3f zc4a~wk;&APr~O?#WfHDu!L`kONfZIyOR(WOGwX3JbFjk0sK0Sm=tu{$f+2CJjyXD$ z6oo}?!&>KbI1b}C_m(b+=9?$JfY<^}VFPJGpaC&f5NxS>#dG7M9oy*Q_Pq1K;HWs$ z&;6`(`QHZV-=EBIgwnwN@&(VykCJ>`53c{j7_TYVxkI$Tp(tvE9ygG2&4gMhHGPL9 zW6^KJa$j|lcA9QnIt2{0QhIus>ds1sJ{GDm^!2O(q`;KGTqZIw&W1ooD1{g*C;V=X zAx%k;i<#=bv<*{i@=zaj>4uj0_@Ml*Q{S6bvQ^hIAPNnY5P;@(*^qVL23%AzH_&yv z)y7-Fj)o*DZ?F@a-|Qrz>IY@i5E|Kb9b6P$kH0z&MiX0YbamRHUOe>7weAh*PSN!W z{3Wd*ZMZ7&>E7-lyy%^8A}3Q(m?=?~PGQ!MjL~heKjR^&th!o-hRP1ln0SIqQ7HKW zt2}VLS5c^w<2tWn98x<a*~t2yd4I48h~%1qWdQqN>L5IX^;B-a?K7^Q_c)lXtb7Gr zHX!@vhO^?5KYMoT!!CHH<PvJUXN6RQgUhEgsh61Elw$El2%}D5EJzO%;<CA>Onq{J z)0mN6J8y+ryJ&^Fi1vg1WC3F6Ov--%iY%<rH<pe>ckWGN1T7@6cM}dFx$nBbS3VXQ zm@;xn33BA7vsuW8!Bh;~uZ)QJ^%>?$gF9C!3mh?f6w%<QZCZeOs6OokOrF&;Z(OAA z8{ge8sFGvF622NtHSy@yt4Mqlb`$tcLaza6iA*=Rfdc@EXqi5^VmSVXmvaK4Vm@4a zWAmrh7x@aglUrC4kK9Lo_BRQ*^tGOtvR88C-Gke#^To-SH3jZDfz`2;N+A=HF&Jzz z3GCJ8YhH#N;nm;AJ+jjl6?DCBzRb(j&QtI?Ej)weQjjhXr0T!4p7?cM!X6GQ@y%RI zxO8q81*X`a3*T{#pIg4EGHltdu3SNz9atI}?z?FyRYyVU@W&aZhVb}7Fwo3#7@5fF zcpiCp9BEqVH(*xVa6<ipUU~%0fD{azDkm`j0273%x8lY#Lh}O!BNuM)9Zwhok~$Vk z7vbt%f*+Uec^!)F9$k;<dBQ(T)ks1bMRX8~9vAD)`_voKiWw$Gyw=HyL=uc$KjBMi zP-Tb0&PTzLG^_jE;NxwJ1Lf&me_yVxEyQwiMIbCeCUm$jC|^zL$V{((n>f?Q0Jh_G zmki6-gM{s7+wsr0o&&%AonJUvPEJL9#`^-l&S4cc+He4n{|{hhDKaZhfQ)h3G-WdZ zUO&)JXf0NOT0<MK8OEy+ylFfhqaH9u-DbvoW=vl%_;GatN&ga;kFHEeJkWEM++9i2 zEO1p8fl6b{4P2zd1;HS*jhimcs+KP0**g__Fr+-Mug0-ou6tj|UQos+kSPR4^uy)9 zdN{OLkokK8SMtW+d`Bja)4rXr{u(e2v?$wqs&cyN<DaIh%cdUlzMK29li|<>KUzF* z-Q@TYsM6=+v@Yawh(`BZnQAo-pso)D=4Wf|xPmo@MaZNL0!MF#GsXKMROlXZC(=4V zH74a)wh-dpXbSfn4J9Qc_XfyWqH_tE??N2dcTIHk2IXj-U~z-7Rk_oLe-><*^75xZ zn`sEMDv4ezkc#>Z{5Si<(FLUT({7J-hl}-`U;!KTt{**?=aK~x;vt_eAEtW+0DZOA z@yn}Z5fEKRI2Y&2i9+9=ujzWl;z-Km2zY+4Lz&%h_LD*6fBaj$c|G?DFGO5_{f&)7 zuGGQZbMe9c78em?P={E&&mGyN2A0gw)4c8{1MTA>oNq_Xo}v9;Qevx_e6{b0tVccE z9ir1v+G~LvW+@O*lyeD5<79lqx~pJ)(|;#rIb_kb-O7pKM;63szt^Z6)(7snE>F;R z0DhaJV)g1V_^eyeX+M`H_iqilAFea*sXwR7gD^i1C{y?Y&a)&K58i?~ayMMbwOgjV zIF=Pp#mz9OiAmiEo>DIS1o{}nX0^KdN9OCkoIcnODsu0nB+w05cpcvM0(Z!T)D=OP zk68}FS#=BJskFyt?y)~@eRl`xiPy|MkD>_~QGrr{vB4F8BHpWzH}LmK_kcq7#N=0W z+%U9yOd0lNG&X$vae)H#vJ_a~VWIHg%7P>$p80@SPer`c3h(Z$kkGDNf;;Z?SbL6@ zthq128JZ{k=P?#<Sl2GE0uBpZA5#~SXg|gZ)g_Mhic$oi-@hS*o)Fitye<gzRMh;J z+M%bc!e0){OcN~4Z+`CiWxniFj^(famOk{**8n9pup${XkD)&v*4%UIL=z}ZB`K$~ zQvxBvHCz}~@wGCBf#caLfOR+d9Sq;kt=iZDxM}>kyt5I88R}Uf6P~=L_&NWMc(=GQ zG*(;gbW4H*SIRv)&FER8rF?e>l&pN<-z&MAR+hG4iZQlKu|LZ~#r|&q&9B{<=7`7q zUV}O{5_U*?*F5RkQ&0j<JkrRfC`$)Rx{GPdh%XJ9?J>1F0eom-)s>ipl#9kzyb&8T zHc~LYXEA+vn0}Jefxn=4FD#EHiX)lI)n{2`ZgO13VzlFMLTLtMkz{6aZD4ExiFPZ^ zob!-aZ)cZVe->MvDA&{PaH2{0Y?dV&)3WmqiVF+z7B0~X@M=9A55NV!$9ZB^0M(~? zUuq1D&8IrmaXP~n&eLp{nvUnIA|{1iDtWwWbUH~5S;1XDEpU-xG_X<$3vVnEIl!Q} zca{-8uP)RqlCYEYGgA<mPnjB#uCja7mZ1)D{SoFow`R<siJ`Ei`P<)eCow6SLSP7s z801m@V0F6KA-MgW#K6*U`N-+ztVO;+BSPr%Y@hV%^2a_pYFP(>uWte=V{~l<X~R;A zf_wP8^wei?My;b&`>d(^Hk8u!R+{I;UvN;WQ|T{2lEt2z@)MbmAZ~$9jB<}UC>+@e z^fV*G7^jUyp*m~kYsa`ewOd&%Q>~4Fyuu~c;H<luPmqNi8jewQ_fPC%Ju_C2UXrNT zn{GHCY%ah2UU#J&dBiLC)jY8H6M)6MX2F5Y;95B6FfM=6cfe&`Rq$TQlr~Krc_m-3 z_#R^wf+RFQEWPi|;Edc5lq6ic<Zm)_3P9o_^CJhID`zO5>PADXiwTfbv|zDQM;QE) zUdESmYPuOj#wX29b}y>!+VGrrh_hBGd6{uo7(ZYf3~oNfo5-73F2hg_CW(m}gR|Yt zAYG>V@A_XX8teay$KQ`-59c5GG!)As&ZHNNH~I5|#2Jz+e;P13h}|%pMTHJR5o0<k zY_j%_VQp#iK7r+sUJl|=43GxL<nD+ZZkS#=swVE@!QiS#Zd2f;f$dRjE`}!C_OYOR zMCOxR@YWOhoj(y~xH$~0|9Nqw+xdK7MBLX^02`Z)&&!n(8TIGaUir@I_1rW=5rqv^ zGF0ynEyLOHs8rTgi|Np=r78+Ep06S<RLhu%*eVXu5KrWN3@{)gsSj=?*>GK-lbF6; zo<8S9LXsSAVz=)?3GKf{M|En2P%~=Rg58oxClhenBZZr#Ct^Xgo2#q>wJ(7HL?ylI zyOvAn*f06^PZdQ)^V7&(E9bFFX!Wl+6T~ghN#GCgL};C<2Ht0!DlQf}Hm<6K0)p0{ z#Vui8L*o9}l|9!Y*F$MipJ0H({`z}RV?Gp}^0W87?aY2bhXsfgwjJXh+t8Hzr`qkh zq3@Gqx@G@z(r4QVaVwqbvc2>i`{JfbJpGsSI_|Agp|heJuBtCU(+Y;FeND!*(O=3* zQy_ggbbX>;?DR~@YESp|d)iCc0jl2Pe3id_0AcIMt-;qE%*|y0uGujn4YbptSb?XZ zwF0d)+ZB7FeKsgjdNc;BkN7vdKnx@*=5zaPVAp`aJ47jq4?iy7x|6<Z!PQv>#+sZz z*dMf+{~|W2Y4J_Si&nvos@NdtWjkNjOG4xUSP)ABA!8#bjDj>K?S_h3uqwofv{Sga zbl${fz7l`^D+l}Olfpb{CBlW*PCuT|_f|V$%$v{N>&+(*<-Vm1c*2&KcKhH%ADm!I zOrTR76JwGSs7=!czN#QXFnPumco_m@yx}TefNT!|i*s?@=5&MzR4wM?``T;5*L6zc zRiz!R^otEWOX3}eb7sx});D3Pa>inC$9LTU3%nJMeYKGgwVN+MB?|~|IzKEg6gFMT z*LmJEd<MEM#B{{&V`boElSlrR%TUxOQB%_=q7Bc}F!<3^ljp}aQT!O{Vlm~8Ofw$4 z{t$4_1U@0W&pQuRUq;*}4|YuVeJ*$pc)M>h_k0dOlRH#4KDD`b6|4?d#CS-LsUr>P z%GxgW=*$b?TH|gm_6LgG-*SLa*XtkS*{Rd_t>guU+o$IsDrQpKPvg{){UGe7YT|V? zyAuaUk5D-aaw>={wBEp63Iql}M4wJ#a(?T(<2aM}XhSK>Qt8Xw$oUuahE9#~t^WIX zG7f|$zu9DKkjJ*b$=msaOkN>Hj`FjszAJx0!>*Qa)n3G<B`(ptd6+^M@)FDpRixS3 z{Nj!93;0!S>Kf^tgC2E9@_)XU^c)aWW(D2YJC~lRA%^NB{5nSmt6Q=kPVfGk(`~Kn z0bD#$K_LRI5-M!OucojfAM%|Tn3z%Gb}rhH)O@8<x#0<LcppV)8Y#GUkKJ0U6_e^c zzaSXEuCJ~SA`x83$SlxNP4r_fS%dsES0??WpKqdzQd_=*54hnO4~c?(pQL|dr1YA0 zW^>er-NYEp-M|N{{0j?&hDiko;={JN3w=TTE}@1{_(z*>HxBN1l7iIT>fm`}xhUl+ z6r?R)#ZfHv8B3vCqV%dEHdlL*hdL${j)A!&wY8sjAduh1Faa=nGlOCi9WVCZ@RcrI zM=p8qpY0+wp84*F7Vnp;??n?7bMz}|2gh_k`uk>(twFc1AO$50<k!TIrjH$xojRv5 zqSM?1pk^K+{I*{WP<=%XHJKdqmEk^fpV=L8W0l3^6OO3Oy;KzZ0c^7H<6uRVD8+tP z{kUWzy$tA6&nw3pv6Rv%W#^z<&iRiS7~h{8mNoT^DSi!3CB;_L4M`W&^`6U*ELTs9 zlMo|#yx*yoo9#{>+LA~bxr;`v$3zf>OwO-sk~cGqKN=b{Uj2=X3{dd}MqOwf1D}_X zo{f``Gp}hzHZ*(v<2Ck)Y!dKoZ|3vSOEvS`z`4<K4S#(%-|NbSEsJt-LJgSE!o{i; z%Q;>|`2&M|H`n`0wlz6(xX~5UXb(YI29ncy9%j37Mj?fd&DM<iIyS-y{BV5Q?{p)< z%G>u<h}+_FBn4f1?9!4!FRuSJkH((*<i$1e1xH8X#}61pq8DM3IA@OWlV0=R>IL|` zVcvDFeYpHG`90=abIF*QQ8?w2B_6y_@<tn3rmOc)th~HoDX*RW_CuLPjD)e{DiK)h zg^q;Ss{VRaP*9Zb%?a@ny$@X-ilb-onzRXg<+T@0S$3dQH^T}35lrbU!3F%aDkx%g zfAeQt9Q?rAL$t!5`aM5pq4Z>UV-ge1G6mf61}BLJ>K-9w<dj6jzIT9QVeV#9GqBh9 z?}|(){EDTOH`pCYBsMlCheJdbwX%Zy{YMNPhafz3N~N&4gpB-%ah`P1Kj-5K`otZ$ zpg=rOJm{^BMKgnsk!Ze@(U?dhwL?snn#+7b^QEU~oQEPP3@IVqi2CZ&W_-*&6R>!5 z=wnC>DHI*O8d8|3Un*?E9nTzWEOiH~%5}0AF>ld--si74stUB;jAd|7{w1jBtk9Io zs%tkLMA?Br(aCT6NwH9WVe@#U!jKOI99RDYs=rM8LL(=kw8HM=1*N+p6L=>HUb5{F zxe3t&&oCsLnpR*1?`GAt>r8$$mIC>ZiF2Ql>8Yj+eN=H%MydhVHNd7k&YS8$scM1i zmy3ZFtrbx#wl_=6R5L}|x;ba#LyaASeAYkDGC8-fVafI|y;WT(^zjyAqx9Qp%BL-h zoNeV9XjuXa=qN-a-DY1IuPr3+=vT|BTv6@1H|~!6%-49bYwBaf=9*3ZQQL02Jwlf- za-Wj2dRU6k?AOI}3aJK8F;~I4PO>oskqKQgmS|GmXrf=v`3Dh#kv<IVxx+R!J+BI| z=@xs_dx9J=j}q_lzP1P5^^;vqB)?!CI$yN>C*uwOzLuoJ3He$tg9+%q!yDo>T*ZHK zXYOdSExaOHsi@I!$EgCG-?zin2G4qT2g|}yoy|6FI9sN5lLBB*d@!+3;skOb*j4%5 z<sMDyU@@>ihha*<$d-Vo`l9t)Z>Y?Wixzn5VuQU1O_!nhWnDNiQe{C`!7fzp?n|`q zJhYJxzXxgYLgi`8^%s3ZH-b*CMd12ktuQXyqTW*k*0*8gqVJ)f^}^-dk<TI2RK>g_ zlmH5m#2l@71P|-258j{tIqE#MGyWzyn#q1sw~M8!$FBiyP-&nd>7(^dc&%v?UdA?A z+MNVmYuA<*muqN1Fe5_ax#ukN)9=&r6xbmSx8tDKR;RZ8_Cs-05fnXzHxv#9!l3q% ztRasf`bv7KFjqCL&-wHbik)g`Y;y&OI_e}Ok3}2w81+gzAaCtAC2}(urjz}Yv+>6T z2}6}T`V#a+^Lssb!|j*JFu#bWtj8D^LjhE>ah4-cgCyvNf@}p@=He|w7Tnja#0_wC zy0gRYHvNYZ+f3IKp>C^V_CJ*r?y_iKVITW%Jgi3}puuvIAOz+Jj6xm?9+{9!fzZv7 zl3kP-i`oK6rMpb%+O~mRGO*a_m`?yE$=3_=Ng-u$Kj>4i-Igv7*k9%Q>VZQNc29r$ z-9ntkpMSvCyZDo!2mdY05!Udb*lT2ykha_WkYe+lF+0_6A4e4_h8xm4L*q2PTj@3J z@MVjQG*Aa_5kz=UVb3;NasMeywW{^3dwifiFjPiS$Qr(nQIN8Go5wj{bd<m>D<9hm z{Dw?&y9W?P-s=M>(MTp|j%K2F(95Cda~8FQ^g~Iz@rlC+OrrQ#m0+g8!pi(`biL(5 zSxm!SqYBo9<n*64A)qkL4d#?(Mcf2Xqhmd)8dX)=c>iC7t-7yOiUFv6Jj&A!PkrJ% zPJ>FO+BXYviLqEJ+;^<RDhH?;Gwx(qnu-egUY-8l{M_|K;!vh&vy{cZiy}7nj%vu+ z_#oq(kNqIT;3oDSlq|B=%G7C*h<K(h>>B=D-cBv7m6-Ia@m5mKI~l&bTsh{9im*nQ zyo_$hU)is;KqwpomOU^HjfXDpI){UGe@l1-@~Hj@r^lyoHuP5Mt{eLdZYQf3(G}{L z1#zN}{Bt)WF_(R-+w^792hHjdq{;MNfw9XvVddZAMHvL$7`2LV?KK({c&7c=*j*9e z9(L}p9g+m1(YARJkz!(Eugoz<TwkB6kXn%}c1`E*<Y7=M{C7aYsl9{cdqW^)PL?I| z{V&k5u3$Rs-G~DQAtP=Gq!D&10U<%aist>`O}`(dxXhNs)O`Af-lV_{QhY)}9Fw}m z>+N>M;p-eN+#r)j)fJYWLZ^SifoUAQz^8b%3GytmjaWFWF|SWzj=Uo*g4|AfpS0WF zbj`v?!rL9bh12Y1i@A8vn1zjnF`_TM<<rQ&@I%4Hpon@Xz==`%Hw@F(T0EH8uD75J zZljRjj|pW0YDak4gV^L`8!S<0ac!$sV_7RJJnc>6a68)$R6M8XKYxCs<C6rG4p~V^ zO5isSN=WU9jX)FeIl<w^vykUs`Td6lNLBJt%`>LU5kx(o(19ze-<POMwWZ6ZCp7uZ z#w^vke6*J3XQJGDYBkr04}(hj!=2O%(;Oj~lEZ~|faL{;eg2dc>?o#!L##^g2WbCX zE>wJGt|8JjW?!cvVyo{-+Vy;MuzIZs@#^Yoz+eGsn~JpPO#$#|xb>a&uFt&0dh8N0 z6UlHepYeFIK}LU3Ec%w*A8d^oHHlRIEl2a#bVGZki#UpaGd`Vlx1c4i!OhpSyUFqh zMd0K0=I_DbU$no0IW32lOP1K-@vFa_2eQ|hk@eXSI3u(gphJ!zvXYr}<s!W86}|{8 zS4P?4n^<Id>UG5CjO0&>thhO(oNpZZEVC<T@=S-0j2?gGa3k3VVYIpz9{^UtD8QTh znDz^3yXhYFJH2(e78-B4B`(!u%UID|P`_f&UpwHh??)1`e={STC7gm<Qg6t0n?D{f zS7~czZ+8{?=6>`Su)8r{R+M|jgiIHBqv~!HOa^QpBq7XwGFfkNhEyoAI<l=?y0cW+ z$Ggqog=2E?>uL3fRBl`_Y_w7pK_|6+gmSIddJyyj(1Y120d-{&LH)|2bh8wnvPb`= z)TWGfbGM}^cT|xsAhwJNd20kWgQ+|0(H*PE6k}WvyVWBpoQAF)Tdsjr{{?;(2<>ol z|Ce+{qk!i+PFxm|woCje5FGqiM87vU#xqnd(%z<tpcQ=He2<Sf$aYmdjo~s=y9LuD zi6yAU(Eatwt~iI@nt9b1tl8dR-QCp=_icT6F;1Vr=!;Tqt~<Byj>W`|PlyBJjQU_m z5bMLu8(qZ;d|^dfgbSu;)-l?F0`Z4CU{=n*2zM)8M5}bYu<03`HQ+Xt&UA3yH$>N% zUn;p2I=ZhXa5|sS_eLQWt82fw*KrX%pVSO#op7#)L>A&nMo{}9^(0pV6|At=4$WCZ z6?`f|OClCc2d84DpXC-mA?x=fS^`Le)KO502Q|h^4Qac>8+gO;PO%J<moOuTR^9g6 zj{Hh?#zu-$ZRoV_(Kj%;Azgry7KWf{?|lo^9>`dS<m~(9qyJ*j98DRF+?`Sp{6P)+ z1J0&ifeQ@(2SE$S&<77BqeIvz$?-j=?FY=%RuWV+iwAPld}DIc8;_-f8o0!ZrwcVX z91tT(_5Ierw%s8$P<*6i&?yH2s!>8}e`fshEl-`P6O;|{kp$re2K1Ww+39hVW}Kq9 zl_w6OK=Jb?j-a%t3GzL8qOmG_D4B;wH-&uYJ<cWT40H@ii@fix^!({&WI<Jl9WTGc ztJzBmjCuOAe`5Io4?){6f!-|id`Q`sIDQO(X)<yaAwlNoLGS{^pHCDEHt;kmzPGkI z-MrM<?aYI<69~-U6|Cc-JK|;d#X69pUbB7VeUvf%A@LxtgGGUz6-`M2Y*7%L$Fn`t zkkUoxikoJDh+9$L5M~ownI%#wia%9Kvp8tSoIQyHrJX&8_J#Z7f`wA9P6uFEu*0$F zx4m^E1?0>t-<+AGS`Z@@UID!RpBK?r((kr^(BtLK7<^8%yM-n@?Xu$tPrzv#s$8>z zhV$m`mv~&agy@c7JL@ZOGE9iRr!n)lNZ!Yr4~ROC&)yu}ZI>H|o2>$W7`#4G2mk!> zY-{1G&ennVRr4D@Sg?9u2e3Gw`EpMhvXEz2HR@-%_esCrOjcx8kiA=<%Lm?~%Z~Uh zl_ld=hCY642ZZ5DD&X<CSRsvKE>OfT{L<Guj19)oe!;RbcIn%Go@Ivp{*K9un}np~ z)^>kHsYY469hT%ubiIfFYv?AK9EH)%+!w>Gq%Q_>`^9jsZkNreS6>5ShMApSD}H-} zGF&KyZ#-R_Fq;zfGp9>6d^Sui4jNp5VbcOXU?6?V&E-%UNnVG{U{~)oAWJ2qLxRnp zTC8=+D##98=HH$!VICdf51s!ex!9C>)e#W3;}~duhJLEY>Yonol$zb1z11izvh?rv zWXBd&@W3%#U|QxP#{AM@;@=rHDjW?v+9EAsh4p<9OTtXMac@?%Hwc~&Wi&p#;XrwN zL)_2gxg53jt%oL<3~L%a4Jnpa7wER8cBjW0)x`NxzZMSCL7-p|Kp-c#2K$Q^$U#OW zxa0RbI%;7C9V0&c_LhT6k0*#pn%iW*?PE(}8^263FNK20GA2}_|MclnC16E`{fj7C zFsQ1S1=YTne`O&TcmS8GG!xPIaAC6!FV*c$NoC6LSxY>=-lg<EHp-b6FoC>G)<Bi~ zu29MXS5Su-xH{kK4{#5VKRtft<iy6(Po3ZTVBoa!B`uEp7%Pq5w0MsfNJ9o}3bep2 zH$U|apaX;qOzNL?+f+ZiRggASewx>HFuvX$2A`ZdkizLtTTp4XSJ~BUOO?j%>U!9M zdEEASBKhOcsnbaSsNGr{t4OQT18F1)=#I!HL~q;IIx>=zElwX?F?t7Qxn1Irgg2Nr zfa;U<P8z20t`Fc~$<M(IChmCA<jxjgN~XKJ1sK>zEt{oEC!T=fnCqwLV=l;8LN}|x z=(lWc=E{0`wiV4h>pI)O8(e0+!-wmFN#SVp8srxCKQ00ntG_VO(b0XMtXF3qzsRl$ zp9SGO2+=|3cE4Va`j4iRj@5ke3u#B#IQo=5uY=RCU!>&q^EG4hQNZ(h(IQU&zUUYd zwB_a;lSIz&!UYToQ82UbP|vXz>8t)c7AkRPpIf1}(ORX$MpOf0%=Y-8O(cNk(e;M% z4De7sl^DWUx!{&zu9D}v&~QM>Y1NMMICyf1lW%HX7FY?W7Z?<dIS>jDDfxsbZ$tqh zHfk*YkudS2q6N2MclmxLk?(p~%w(!CppN6lWx4FC;J;rL18QJajCu#{Uum=fA&5L? z>WALNC2LdAQ*}b!2~A+r9LN+WYQZLYCQW*^GVPUU*J5(V?6j_JZOqetpr*7ggL5!y zBM!xW-(NiB@+Du=!{yS*(7_AVhOKZDmE+O+w|IuqrW{Fh5nQ{TnO2fr*A)`#pqj6o zlI_+fB${9%BpEFzf5}{JNk$U0Vip&Vc8yz%99q2PFk-Fd>$Zl_Q3g9_2$qbb(G+YX z!)>2gYft;Zw92ZmlgpQI9go{5YwkJF0Wd2dNB;k_0v2zjlSS&e;kA7}iazhZhXuHD z0HIs}fP=}PZcJ~n@TGpq6$!{~;^yMQ!K6_DBSyfSIh-z-2A+7ibFM`Q@jlX0A#XtI zk6n*D6zNS^=K+*Whnn5x$Q@Rz4t6UBSp7(^5`X6t<OoAZ_|vrqn%slRCsL<0BZ2my zLlgc=;M+)FjR^uv>DV11Ku0nX2Q_I$^2w)GSKj4#5nh&c7$#I^80nNwx0(?H6XVTn zEd3Lq#oJX7YPsf-c>UF{kevQ5MQ;ebiiZl8f-eUo#muzLF$=LFxe51~;v4&dMSH*@ z7UH*f?%V@Nmd!;2UVjDEO*o-gYJC5}<+tT;HO9-0qki5yH_9)rFz58Dqp)ad*%Y+b z1U(@-`E+)GtS#)qPG{|wrb*EL4IYh(keE0Mh{$`u{6|t)NL6lE*ATBv-g;y_vCto0 zgb(uciq+Jd>AdhH(J^O6KfIBt8O-&bO%^_+qlw_b^3V2h4IqUuQas?nM0x7z)=Zve zCl(G+#P318Ag%~pJQv_b1<YXkEZ6tZYh3}f>}}%zR4?NJ?DZsTNEUNrvY>n~Hr8rU zvOT(kHomc~RW%;6<-wk}w#2LcJnoS75?(r|x7DGB?HK*i0eN%qZk1>aZH&fo_wc(( zs?W0|jF}XuXE7&3)~~(<Zh++@K0F82bUiXA=^&Y99sWk`lk(>rjK+_z-F6*6lUL)H z`wXsZx~%1~t^OXYm*=*c?5XLmZ~B6rTz^fw?v^ElIob&6IBtu!k_&+fODIIM^2nQb zp^j~+X8jUNE*=S2JB>a)L`3qV3soSBkD6g8i5D~0g_VEZN&wmEAckD%iQ~8UA1Btc zd~zLa87KrZW(x^YAuoI3=P~=A>|wO3RUSS9nw>e|B$g2)=KiC_^iZK=M1i#mN7}!> zG5fw*l5mk~2hCj3stP#hXfU{-UYjw6cIY5bD({Ph!bRMEV(1IC%S71uKw}5vg%mJo z-Sd9!^ka%19DP9Jr7W1H@`GPZqZGCu&4i?8GZ(+*#MoN2Ul&XQkr01;4Ydn%D%Chg z(Wp>?Ph&F|lN*uLDAR_2F-0}qJyd;cwm?iA%dCs8Z~U}{MDlLRMPXtRDf9d3=HS8G zaJ6B&%`T<SbMXA{kF7r;rhsP9tsmy>t>jrSh?^hvT3>{=y;8a|GJifCcPS13c3>rl z^{zl<Dg~DAWTA!>?d%uOFh8JO*rondDUG{z`J#v&0Fb~TfH6#4NG|1Y%2y-V?Q}6< zNan+-lrN@YOb0-66V8LikI8s$0G?eM_y@mknz{|o9)^S%KcIY!8kRD)fyGhKGyqO2 zOTFI|^1!7O`!XZsfF?(Y<>45!{w*3ox>Hy*BQ&DWKFg9?6yK`5kud0HZvotb^Y{@= zh?uH7RRpv+{*^je(o41bIxOU-%H?9Zgw{8emwyv|$-obQWd<Zu-O8*Na0c9KEN*gi zW-oys$IBDD{=GWz8e{E>eXRLgxZ6IqtK_d6`hD@cW4(L4k5<D>Nay51Q~3&tNcQJ9 zagK^Nf19T0Y(!bhg=(IWbAg1)s|op+3+HgZ%+H;Q)ASuf>3y4;S%)==+}hL5SrC=> zV2tx_?6HaKEsz*%5Pe7x6pl0Wpy3C|Ptwp9&X8zf;=N2`RL}9)LpIR*M-Vz4Fm0O^ za7RaA9{Iiku_b_79KI)h@{p$GvK!ONpm93tT!}@Hj4-fF@YAIIYWVw^%euR1JI}$f z*a7I{>IP=XY{6Hk=Fr>Cxhc(|LeGM1xrl!<b?yu%w}>rEYOBenVa|iiTiFun+O_?c zOHbLwjcLkr67wn9i{{&6)CS-(F+ZD|&Uz4S@jZYc>rQ-lOJipRlrSJIPq6Plv682Y z0>Mwhr%F&jaxmBa%x~vVW;h3bjQsxqx0K{M2A1L1r%>;EuK8?qtnC_PW=l71PiObe z`NCyVjVLsQgh)9!EBHt+sq<|+e<YkCMSRO26uCF9{Q3$9_@m&A4@TBsvLP1QASnD# z&pfN|NN!I5BMRKau6=1wB=)!U8cwEd`0vc#zof^tq9;#01j#WO#V4!`(9QfSU5KKy z1_EDxU{jH}B0Ct`Y`<pXkA=k<gz`5$jG}D{TqK;vvx*idUTj(y&-TfNm`8vj6mfH= zOVk-5wfQ2L$~Y=tbF^y-YkDM7t^}oQVEldo4OQfG)qC*xeN|%zz2ZV%GIm(@&0d6= z#CIQccf^SL%7KCnbu^I|lR)SJ>{!Tv6#05CPKva+{)sLL#0%Uc_Ff!(a6R{;AQ^K- zdxCh}-Wn)-G6CKoF#qRpJOe0`or<5vg&+z@Fk!L8rv_6m2Jw4HCEEe3&Czx}aMQ;_ z)(tmR2b+2!^nVt?v%Y*hB&rMpu)_19mBH1@F1-L7ZOwOSu4>R9;z=qaD=Vt4oqA*t z5|xq1WH=b7A26K<1li^Ud#OEGI)(dj-7(S%$#^S4M}YT7Y^8*xKYDznsl=<zmZnp! zFel~;7j25%cTUEAP1qCP+$M?L_A{icIT=a-eFnF~Kv^RXq=5eQ6U7(-KAg4+)PAMQ z{DB$<Iodqzfl|(&;E4jKO~VKyWg5A=$4VBth1Ru)YTNThh*)@t>KAPv^G6<fdicp( zKODbTG@jyAbS$UGe?ApoA0Q6B0@ytmZNu|yghL>cQBzl|v!2*U<+g6N?sL?ciAbai zSElvZZYSzNs?C!R0E&P=X*#^pe9*;dsRRK|(VI-T!A0ICq%Q1U_X@T?PI2fij8W%? z^VVT`x1oKd{T3~-@3mW|^^%q~z28b9Hp?sZZzGAvOkwHx!=UK*r>C`=&H$9@vk*7# zuqqU?eN{lj7X6{xfh^<3FBpe+9vty>pQ!IoHe~m!B!z{RJ%fo_XW1cb)FfOKR1aA< zWNxG%xM}(ICfYanbcuIdeG+=v>gEI?!I0KzlUo~<-#%Ly2xki7hEz`Ts#femwmvk6 zpktGma!au16FQ@@)%o1=`?#W!5ZJtS?+qEIHg4<p(LPBF6w8e3P*OPQ?w@+(Ntn-$ z>uaCz!$R-8&qz-cQ566o!lL6o&qPTvUb)Ih7CWvziw;6Nm)-A4_ZF!_v|(OOL1aO0 z=KgI=*h|2jya`yo@0Y3|dY8v_+`JV6k*q)74GRad((_$Iq@Db>ynN@%PTT1L2|Zyf zWr%POQ^QM9*xS`wJ3GWWMd?G9`#P&Z%AD*8Yt<-2+l(iD_;y8x&1@<8+j|s?<yxDo z2cQ-%lsw6t`F@?|diM@AqGofAjOSlqlqtTxj@HcF3`{1UGui*z!|Cqq#t)Tyk$G-6 zZ?;{Eq-$(cu6srzBT#yRXQ16&VjiA{o8!%K5S}XlUi({0X8b3#Io?XZFjw5q{kaW0 ziO@zWk?rRhbkdZY{184RKl^cgTdd8)fLgacMAW_=e)Tso&-p@s%ukqLVfRV16G`aN zB=E>wIdKxnsALrT(P~?_*;X`xMzK&6+3-rp;MX9CnM&AZrMZfuX6eAL{ppKV>Z8v1 zgu~LycJ{Q*h6OCy$ryY2eb}TYSiSh(DT)_fe;c?HUAKBDbt+heI@tidfB!?sqA#%^ zo`~$;5ky}l39*&-zBC0=A;gtFq7Ur}<b67}!XXu3Ns|3-cDP0FHU2bC$>##XP!7DC zpSq7cc<+8q`$8qSG__ss0DAS{Q)1`7!)?!^7aK)Rdt>;SIlGn+Qfue$(GOzKOV&N< zr$5$E-zwb7dKR}Ok@?QxxImQ%$Tw{_XDk{+#Gp!N>;5L7&i6LOc5Sy{UR@_qW6RIJ z=K5V(_-XC@!w`h#)QbF}+;BCWr{M7FhNvBm#@3{QEmUOb*~*ofe^c#D3cs`{J~;}O zPG2V0X3@E$MM)CwhDd(%96lX))*%lv;O@40I_1?|>xUa*)dUZ8%_{r^PJZ)tfJNbU zBA$t3GTV{?Ep!WJMhp3S)R~e&B&6P&#OTN(4br}^>vnnff3Jud7>rz1lYv5`3;52- zH|`2p&$IvNrCL?7bH%k?K1ZMkjj92dplHZA@VXHUe4jnOPaAgy2Sa5Fcw!93kaEA% zmOcT_3|8M{>hNG`SWfw*cxi-=WRClM5<~gUt)*@HO}zjk$e~k}(u7d1S%i0$_VYZA zwNQ%=n_jTGQi1n|tuMM3Tcb`nrtf{Ek>z7uEUt*gVQqD8WxvfV&<n;<k}4#E&AHG* zSKs>2VLr!G%j4wiYTWNXo*im7HOzVnKb<-24W;KJktTF^{@pjXhB6D^d;umX%q(TI zGK@y7FlKO@@##fz$jFMD7m=$s7sp5r`#t|r-gmDUiWGDadMS&YEe8v#8=pW~xGpw% z!>Y`%1l|b(2lZ^(3x(Q9E@=_SGfOzS<R{{!0-)QMtqY$IdDM~-`7;0^Rs&Vh-5oiK zV7r#mu2OR}oQy9y_gYmX-=ci@ANbP25K|b#4Q8Hqy5#)@2w`7a^Z3p?Zj?r^Ne}-L zp>{pSUf_=?e{L(?`C{l}?$2)m=-Kh#hdXCHmXJh9gnb3lxgE|&M}mHioj1@<Yz9Ra z6r#0;i$L3d3QqZWC<ph=81LkW9yVyW-M?*_?z&HCo;%y<CRKL#gz@Hrm3lKx^DO;o zz#h@@J~GCwQh3TF#T5t-mzu-x&J5R>qJJ$usoHlOj$FR^KCgIXyH)~1|EL60tbj`` zSWuEMQ%I~?vrq{_YBe)W0ZHKwp&y)|zzZe!{we>4gZVX+-?Mvf!Nm7W^G8u-rgo#g zd(H2WeY?CTG0x|vN@Rx1TI|j0c*EH~TV%~6Y2V5v$@glYsSrJ^Nh~k?aN`gg)L}zp z;GIjV_E+udcjP)<<{Qeb^=EWkH4LZcSJgs8F<ncCN3@Wg`|aLq;>-8H>oT(Kp%bnf zVnHj*6np{=dji-vS<yfEyZe4eu{#Pd<2<a+6H(u2{$}Epr)1>#?G{}C`&;aHROk?T z-WD7xGW+|FA3w<$Ik2|K^LRjR+%z9t7~dY4AC|@tx|rlq1hgLAX2&$qir5ZBN(am> z<p%VNLNL@=Om3DE7Bf6xu&{xqUn2B483n~&JDu<OT1roQ-)^E$2Sz_$!_U{+O#xf! z&pQUR%0&g8K97pf;ra{y>O>cJ?y4YKDHk|xzoUATwo4^y^u!l{%+-;qE2?|nioMe2 z@{tvZn-qK?G5}<bJFbBG6tM7{{0#&|!P_J|_ziXM{kVpYBdT?}AdA^jRjZi+qU)jv zU7&7(fhj$PrxJMMKldI!NVe1Yk;eqXz!^cY5I?DA^1|hmRU8rJ0hY+(0Z_+7*@Ep? z@j6hOE$d#0K>ib%Be>h*(mM%MHd%m3>E!{cw?)+fz;DqoP5sgW0oPqq&J)s^yT&}c z1^1lb(YVv!NxWINzda1lghIt_B(@7g8OaW}6<X8?yi05+5e}?3Sx4YB+*VUy>cx-J zG=4i#+@D=-1|%)&C0>^35xe`=B)=^_n=O4ek4ud`e9$|YuZrRgqQ2Hyg3f%3*dIJW zTpu_MFx*L5icmP&X8C1;ns$brd75RGGiOWadj#$~Y5S!G$SfAy>Mr8zKDIADfNPKC zhWLXqAt=0<wjZ+EzQnOVYf_df5FM}2To6e4p=-@t8}5N-7W;f*SVS;4wde!^i}z9S z!tjiC|9lq_t%`SmA2n}o5BR+1Z<fy_AXA3m6t*)7AB<b+`RC}-9b!CU`IlYLjAzKH zBjMGG?5^SZ!!XRVzWe&#+j`hW=v3=@TkrFfB9tb&(EObLk`e5CLv#3!nq}1I4DP3U z{xt5a2}AoWQ`K)-tcO@_+=LIeZ4;TpBU=mbCEs9@igsQE8s!O&4{5x$XoYKXW<d3< z;x$KlZ4{3k9xoqVKF{0}ZHrol4LV7HfyDqyL?=;yF8K<r4)gE}lt2(VJTKna_6jMI zOmYmLg89ks`N9Z1Q5{?guAtbgz^42d9%%op{e&m0&!gL;2YGJ9L)l5_RR69(*rDE7 zx*XNAAUY5JLA%g!V#O@xz|sb2XzY{!A0gF30)$p<-JJKf(BjLgpWt1aWySw|1++o9 z_&Weg%2aNLY%iK0v`4X2;A2p(i=xa3{cp4}p)aK6i6{>X<(i|du}SwzplfFO7v;A8 z3cz*}2hj@^`vmvfrB<C=-|jJ0Qb<cMl({*k=Vy6~)_tdfD(iuArYSrbEadNP*BQGY z{6;%y>oO^mox8lIYT#^R;43bn>na$>&O-KYe-xi0`|Ib`yi#v(JM92+fgTBAKS47B zEf6tWj~X=8yH<_C+LshKf^dTPk9gq=fkwoLpX2bM-}@2T?(SEn-u!&f(UZWbrj2h= z+eZot|Gu~^=1}PT&<hNE{4-Ul;6~>M5E}Si{R48GBXRt+KbZ;5JET2vd~Ye?mTTo0 z5gHaR#vJHs*79t%y0ixZa@@AogdAWsSfreGNIysT>*fEgO1&CiphL@2Ud{w`q1#15 zEL&Emsx$jtLifm6QS^}qIezQ6_{ZHJifKQcJtySzj3Km1y$;#*7;z+~&iyZ7C>aID zJR+Wu-x$S8Ey&bc&;?bSttBZxw#yK&-bx)HPOH?4!oi^=BqZ>Bheb4>a|Zr4Z@l-X zKnjbmPR|tEW$`#s{QmQY>5fjX4NeMlO>&Ht@?jTp-&@mppI#KB3dPY@nCo%1E#<?g z-|E=wzd?U}8&4gunV?N4DR4er5OVe1#)bh7grVNG{lscz@N;tbHQC4v0=LC(1tzfd zH-hxG*cOM`c-Ht3<$WA>Ft5kuCfMiTrJI2m2TF&!*J8IR!Y>#?#n<?c)ev`3JHL^+ zaV>}D1=n1^Wb*gFr_{l^q5CrtXLu*T0>No=(x>Hog14{pFhF(ZF+QzIbE@<g-L6hI zI~KJ<xQT4Rh~<{`h=ufyFjZkxiI!&;N?i?<S8o@@XWtJDsiw>j-S5f0SHW+sv2N?J zpyBzn)TVcF6)NMv_=tmF#Nz1_BKL<K#rj$4+2Ng*Tw8BNg0U{-@QNeCKaTqe@XbTR z3<{bApz`35;l<6gu_*rJqvI1qg<e{D-KpV(dt^$n<NI?^Q4Jd3tv}&?oEQ}r&3wbB zXQYDiq%fT-laLhn+NvNzWu*Dh5tl8mB+u!>lam}+F5e}QP^>7L;yHA0%WyW2-wp_8 z1{x-XD%2{E*j-eK^@Y^CK%Tp8f~4_?g(gddhPBTrN#LGx3ycx?z>9&N!S9kkA*h5K zb*078#DLj|WySLPNv5SK<b5DiOB~Ie&9PggmJvKOaqRwCpXR=g*!7bfxEeu8BPuoA zG$ZyWbtD$0v!ED5yb$7(mJA`_DNUxLYj(8ZyfS^CtraSq<OCj*ddNOlsnfvNYv+rs z8fG*JVzwLgytSGcZ{~-)EE2@jfBfSoRQxO6kQ-nZRhw;ftUdo#VXzR`CD4-eC2M9t zWa|P2+Sdn@i+-W>f`V(UU{1ru=8xQNgx|48q6S9=P4qHl+{$Q{2=dqubelFNV6-0c z-S<r<J}h7b*8iF9*xGcyJFEWGNt?}XG4#Ovcx~&wKqt5WET1V=Onxk{m<fj2R6h0> z>z%%)!n`#*|6u_Z&bM>UFFJ#Ts=Swnt;W*yNlqg5PhJQ0dIO<-Hr9Ydr9F2Np<Y5Y znO5lo?J42=4xj<!ezTSHJQ7oUK9$VT2`uYQ=tTI2D7>zDZ%5(3>MSRW<mJ^f4o>!o zbic*5z2d?DNb#mOFx3UT+6bajDFCko3}++CWR1@#jnJP>`<XZ3QNjZl9YY*0Cw>~y zC?MOtPkcTHG+uU}w@huiqu27!Z`%{zo%#07?q<GyF@1;AB6P7kco_@3H}Soe#3p>a zw-_(SZQ|7S-P!*Qz0@%ZbPXNjMMlt`ZkiUi=rkG<KWeP<T{>rD`p=IqTf%7SM$tAr zxD4iE9<Iyr)B5yGwuGQoKsMSh*p|eN)=)pDKL55Rl^e)-?2mLu<NdSLgkshHgUb^T z<Zw|w*V;hov1!4O!$O9;B7V8K`B@rA2$$5DSBOZiPSO$=>1mX?-1Bix^Za=x9Jc6) zQ&V`;M}#)#l(I=%>oX{9VSSOEdnRU<F=&Xh8v}42BJ~P@>C8BF4F2DJodVF+<j`Xj z_`2Xj$+*V`;&wL*|98VYTr|Y+@UV8f8&fi~zD>`Ng;A`g9n_A>Y%*|>Rsf{016!3R z<kQ(QFnBW$#;1Z^#4fyP?y^gFQw`Zs+zk-D5V31Le+q?Bsmh3u_FfdY^YL?lA_K(p z5zx<`pSTyTAno@9VtHh+w0%Xn`cb^c(CJweeK>iTfU;y`n7k!ohJ`SH#$yTexY1eq zklI8Q+b5p}YqM&1VL#NExo>v79Fk|-&6#~YTyKS*a8i$_4U?c0i|d{QB(t<`Uv!J{ zv>!RNEVYeWcb<P@yFI`+*m>iKeF$7zRtZ%anFI0Ni%&T>RfXa%4njvwL=a5}(}Ttm zOC)QwtX}ohA<c%o*aq&!RF=J`*3I3QeMxOcLDBCeyI4XrIT3tV;nD;n#`+LS4((^n zU`)dA@9Ib=oD$`%k{#5@ha$q%;Zh2)FJ+|t5lVd!d>ymsLEb2A23$dzIwTOkatIf& z=gKwMmnTb>8Y~Mem2ZOH1Zf_?Ma2_KNtFsSxuQ(ujg)>MU_&f_EwK|6;400!@;YVl zX1Y6MJ*Fla2aVJ^1IsJz4mbK_rI#glvh^_?U<yXy8iG`1QU7`aDJ0pUj>7EE_KPYz zgtyR1b&gisrFvsP=l}J^oS+73uD0~3aUogXBZKn`7t+g}U}6-#wWkjF5&(TbG%TTP z6MZ%UD^^C0)RumNS0Lm0V|x}a>1{3~{&6A&nK%yEC05w{V)cb`y^TXNTtjT*^*eGQ z{KI_ueN&E@U#jO(TsR4ZUr*Z5B6QL^nB2MsQ^=b|p@-_cq^P?WVOlO(Hw==7HaxF) z!F}HC^_qDNY%UqP-cAr#4N5T>uu8*Sf{k{!dlx-Hu<PDd;9}xpPK;Mz(OLYVcGB}( z6-}nG$x4?jQx)%cvn+E`K*aSjI6>UA;MW<mBMYnAB;%9Fq(aP!{vH?qN1PI=hlfD` zW{W%y0AIN&h-W^ufM*|cMoiW3a(%>F`g>F&o~9W~+`Ug|S<w1&8WKC8{#gFMLIh>u zH3k=xR3-n5tGA4aEAG3sfx#(OiffVL?ml>1DDGa|in~iG?heI^yA&wyP~6?!2A9Fl z?m73J=Xu}k59|OTA({MT{nolx!Dc4JFUR(jvN*eu&Wa@?+U%Q7{`{klw-@c~b0y!| zg3lQAAGA=oMSch{c@lA3z-!;p894w2U6D6227bkBGkW;miY1ya014oU<uvEcQuIN5 zSYDuz0Na7ykpVMps!7l3+-~H67x%UNCT3*_b^i{v9%&iq2d&p{-YAf`GLO8pw3(73 z2imo`Sq|AEnq(2;q=gG_B*x(WMEdj|G}vuJ<F*UGd2NN!vH{(@qF)xC>I|nyp7=JA zWc7-L;n^dweO)MXl@XqZvM$ZxmVOT})TPlZ^-TC?bQ<D9lMP3M(eM%9$v}!^BZ?zW zqXf2lyTBEI!^un+YLvX<E8>Pm-ty=Cd)4Fcdlhip?><Q{wWYG*_~qY2rE+kgDAag3 zdhR%+q@?EpRXz9U=9Bw>*OW6XJ4lFzSgQPM?zkV@?;ETNE=G9`s1)2;Wc{wiR*d(* zj?u0o3CU)cAWJqpc&}einx%3X1;)*vI)3o{vDcx)xb{O-{RQ?Hj2fewSfiU4CC}b> z^=oT0rdOY+4B@iXmGPtPakz<>tlRr1H=)<QKd)tYu=moS*b<c}3@0M~B#&Sgq25ab z#s})8{TW1Fm}IR-7aIo<)t|q^3T{9MV`v<PQau?5Ci+b9VG#xSL$X+hPC#l`V2lU4 zYNtZl9#eMo_+T<OEw+2#MbWaK>qf!ur5F7c&Ejah3VLRDk?kIOqf_=lUGtKPZ~ORP zKh+JL$#LhrhV;?pW@Da(E4&Ce+1tZ|e*ubYkcj|1T2oRKjlRsP2-dSgpPM`J2#X+; zb;Fq5vEj58JCZxZj+RlO52MkGOl!w6sXbwIR(_tO@k&J0t>%x8l;DuLGVkjyj}PCk zAU^%Tix4q^na+3!5?3P(9cI6BL9=}YawSJkrhir|2%gH5opGFAm$8nP0E+`;Emgc= z3v-@c9XkJRGbLW9O>0%2>g6)-huWrT-R-3*LfJR@WtuD4BhfDhI<5AbW??pz4a=Zd z2;!f+_W!w30B@bY%lsF2o6NGm{$`nPcFllj?fqY=>K|1Lc*FyOkT8i9N5XJuy<ZRR znwt0k6eq~zY7c?*r`an&&<N?ib_+RX0ti~1oeKcYJkzDnys`+U2+Rg!(ioCah3CyA zJwc=8OhLC6w*c#OVb37y7VgLCfEM^S?_+60*Acjh^p{kerq=g*M()%fnB=ySXgso8 z0lv2@O|;TR%hIQjDT-iVEmZooRxzkUFM9e{f7&0FCv)}3KYhmeg#PFAHay;lj{gEE z<Xgs^!!=s*6jZ)r|3L9N<`w8V%dYzj--gGa0q=pjFXo&~@LlEy)%3l^$%9vjL-jI? z(#|~WdRv%aZ4!Iu@XhgjP@caQO}LzwL76n-V5f|bnP?f=P;yrYm#sR@Q<!;~XxJ4B z;Uw)nm;DCV`Dh1oLXkgX%(l<JxvjuM(^ue~!bTbHO!xea^W81M)c%)_2YM@3u~l`; zlN|^5;;(2OGRUrM$HS#}8UuA_F6>#f5W2;HUzj3L3Z8G5F@^(qP-Rh7bY?pk7*`kt zac?+a<_=lub<buA#1<>Ht#Te{t=iMvqm3;V3qtfAk3D+=#>mi(#kkKdf<B?0^dgtk zU0<9nW0=|mUEYzEt@qb52*OAsbW_J3X4t{ytEluTbX<`v{q~)D7nm&NrFKJD^A#9w z3!69krbX#x`a0H~DhhKzHE*^+Lw55i<;;}+x^+%^)5EZ;%j5GE0tQ1v$x`eZ^a}7v zYEQSHdj`C-ORuTwLbB8XfT;S(VqFlWM?>ifMyAzW_gv30n@`Wa`}Ru=daC<XT*vuC z>;7cnQiO$G4N^QyR!>t>8x|g3*zcAU3Vs?Ch;>_d@-F9|HG9D?z1}%vzpd@rs$7rY zeZ>!cYU@<*-{&Hup-9MS;+aH`*h=y>$d*{*$NfzIentO@kH`-TtwEvhi1f-~(`aLf z0GH^8fJgX7rlU)&Fy&C?LB{EG#~6u3<4wm#7^Nxu*KyC*ms^TNkLGv<q!#`d;bKl# zdQVoVb#7Vslqs?WXglpQ6J@y6_8NnW!#d>db6u!$toPy@f=p4rc5LRfMz7QG`{+5{ z+PbejKl)AG9=9-!Mwdk4D1j#LD(&B>?;L)<)swsg>_b`)B#?1c)@>>%v~T>8!fjJX zI*n^T_*6jxh@~;z3C$pe+si^W=d(Enw$8PKQoaqq>y=J5eAAh!(BYMch?NlW*E3x! zK!tJx^vvA;J|6Wj?_&Kn&-R<4(|E?f7!+|loWsYy%flaS^%}`i-i5M``kse0vg^j4 z)B6aSCJnyD`sumN`olJFk56B$6ab}mNY+b|J+E%WwPn1~b?F?E#(6-@l#YuWh4Y44 ze}igF^efU7=ytx`_F|=8i`voiCI3&t<S!!$*}8l7ga;W)PuCN&b0I0=!e<e^pN7f3 z;||%r4<!!#{_@Soxl05bj>V8*F{89r-%=+L(U98VcGUsFU0hg`@Ryb1nceaC4g1S3 z-`PFpK(TtEbT1d(e4QCoA)ePFHMT_n{;&sB&VUuA)#BklXH<=7vJcBrbkh=Mbd1>R zgo++P$(AF7zlG)suTndVA}Mq4?5r9PqxO}Kc7oN;u6o0`$T*hVuX(AZtsYGBTAI%J zivHuZ1>x}L5z9k)<*m0bDO9_D{!_0_&ARiqdBQUFW0Lh`sQdE`BLTibtdBa8{wHWE z@CSS-9<ooD%;#9jE=+E-P6<$dyf#6+iq0LX_k_|JU8qtb@X-&B#`=xe_I!$bER^p? ziO^Mdjs4x-bqE)nK>p3au_a&2eFHgBpana4lIYUcM*7Z;Sfz5bt8{I7=qBW1>xrrq zMp|6@g>5=*Y3lM+ih`rG#00kaUdu^`s*Fe?)X|n4Up#cn<cQ|aIm!IJ(cPlW)Biu~ zwtpd<Sx(JAv#D-pK7Gku)BpNur%R<j5JnI<1CM(v{3|Ji8x#n67?t~ql;(o*qqcTM zlIk7RL1`yvEe{*9{IkoOpe!;&p>CD}z;9+sk!t%q;=A@CiKCoSx{QQ@);%z9;-%S@ zf2@FJ<GL$hp>9z$zG^vOzRk-`Wl*b-bXErGX}m&+5T-eOQG7CWI@S!OW9U-3eZSvD z;*jVC>37;KJw1It&5GX(ApzlV1xuo+%*)RM&S&nK)`IjitUJJ2F<Eb^GQEXjl=^&3 zQr^}yIT7|Y!c$JPELxb*doajE_Qy-87N;4)f&);KxYcsje$_U1x7q5jzFic<q9+Ma zrT`|QQbayeFAh(CjG|1@pC4{GpJ+Karrd3hW=f>riD`K{ZMGnO%m>A%2QXHRk_GOU z34K%Lw3?xdB<39}cuTklkajI!?Yc3%7a$-TtzArsh@?x;b06ur)qREfXrY5bSooR! zk+jXCMC3shppvzI5OAb^?fxEFY*TB3)L9JvsmcOt?tCA}Bn;JblJj<Jy%yH0*C5T_ zL`(YJ$8f}xJ#TYTdk-$?7}+pRatRr7YdKxKp;<?KvAg?EiYalQUr<yzleJ=K@&<NN zLG1}h=CJ!&qD6p<Yt0P>&Tl8IUGaIUy_v#K5@roL6WrSUBTDust@rW|)^}9ijj6x2 zv^-30@`QO=>L8)g8Iu~5yFY&kKZSaoIZ{(+O8~HwoHHMCN(LsjT6^(J!lx!vRLY)w zC7##J(6A9BRoS>`P8ky!AYUN)U79#9X;gWt&z&T4ITGZsTzxiUOTzDkMT>*6F}Mf{ z6(x^x{v}(77d{sFQed5<)NTYU&q{=WuYWl$4-Z2XKDC?qALyYt9JF0%oXCFZJuf(; zvYYh&fg!&~+5#x`X!GuqD1n*W*2%C(?0yXlDm-Q^`mLeBtm*S(JWKd{&E#y>|HzG? zELMQb0nAO`a%JamN8he7Ws3L^0m>f6=X-<*Jn{XQmVCJjj{dcTd7GP~*<^f-IM{bz zlIHFq!5Gk!T>k4jLdI{bPuRqYx%xDTEkIAOYH8Ht(5kinsQRa$+e;!>bT7~ww`r#O z5NgeYJItC<qrI#B@eF+tPUj0c<8{%?n%sp(O9gNnyyib}IE7<F#&DbaE6%*&EWbg~ zvr)du@oMb+diN!vY(aB`Qi!Q_cIm1aDWmT?yW7egA_41%3pi3O(cXap;<%|k=V9hX z!@h7_UsfQbGfHL`fYJjIWvo6!wP=46fhkt-Ifd2U5O-3f8AvZ%tTfe>7^p~|>Db*M zEyzR?eEsv9jQb(t(w6(h{aJP2TRp4Ed?p}83g$k=4nN=w4aidsP1*0FqQJbBj_KEv ziqMvc<mMbe?;HS?H^aQM8=nR2`|c^QttK(3a)<A_a~mR*-en#Q+>tYC0A?NImh!U$ zZro85-(nmk<rCI4=2BO_8$pK=K7_7Ri2PMc)XR`R53XO_O-9Cp2l}D;XMs4od-fgY z6F!JvX$HBd{Pp(ZPv1flz~=un_N4>CdM$QTr*vC=#(*~aU_c!o3pg_W&ba!yz<L{$ zc~(4<;OYfih$|`&xeWb=*<S$Cz!AadU-<f5;pE{DEuZENdXOm6PdgKUNT6-3fnqB} zS<lHQ>cLXHoh;;0ysd_a?~dP9(ll|>@PH&;Q9U`rdXdK&Vk?tcF<_KCwgPh#kg;We zBGo4Pk_LtE(H?vPD%{pcDjxGCto)<2b%)x)u^!o~=XQ(um#=;JtPHE(1=#=xim{?v z<NtR|_AB#0sVgP)XOf=D>B-dppOWW)r9e$e@6Pkc*6D-~fA^?#>caBE)Gqgonj{fJ z7to)8^REl)y9*v5--T{07|GV#02dns8~7)T@^8?vcZ&;#`Bg{@3T0k#qr+=f=l$q~ zbm9bneQF_c1xjW6Brt{FDgFS8myr@sqJ8($%&Tn{IW%rEOven^M6+I-1TRzQ4(>t^ zM1fBWH)TVqG(?S?M<Zb6;6}<XkwJUP-%j(hygUyWyY6AlW1rOITNW%N7AS3Nj*FqM zLT-z|THTy|M1M97^FL|zxO$MP&fkyU!nv3Ws`^CLQgS3d?&5bwb8We*k{<kI*7J8X zb;0+ItKF)p7E>3=yV;05+%o`66C-qqET1KGm{Qa=9d+@sL+1Ae5QezF+_eCV&EV%u zFW0UzRzOn)PiE0OW@#2h<6Sd}jl9!}Rd|B(`>~)HXOTrqEMqPmO}b1(4En7zx9>Yr z5*J3kyy@_60M9cm?|*U&@BdI(yE~LF?NfNc5<zF_vIv({1?Rvac<?9!H*ONPQ4D|z zXf~MJy%AB51B=QPdCzMW7iOb=(6wBPyFdVqqtxTR-N<tRIE?H7@o-C-)dWjBBsOjS zCKVW7AOiU&yq+{w{GH<J#dbYJ6%!Pj-O@r%Rx$$%7Erg%2xczA$dQPehy6|=EQ~%r zRvc~gXL#f0hk@b!+8O&^iasDl^fd8AC{cTr_<zWf3+=02UC%vYzQHDXYt~90&Snxb z6vd>(A9XOqR8UYvxWOr<@}byc{IAP`w+YP)e=DAs0)#ya9TN?(-Sbg)cSmKTnGVN{ z9BRe!a<JZJ3;N&C@|w+HzYUDna@>lr`V!08Es|IJc^&?W5H7YuupU$qk+}B`*$my& zw4ksk|4vl)LZRVZ<P$dtT1Wh9c2$V$&*Oe>D3`|3H^a9?><!RH5Wf$Mo2#1+QC7ZO zjf+xjC?pHGo>)j4>}`(ja9>RM;8t7!t|^W-YAG4hhY@ilul)Sy=Zoza(f;bMJ<j2% z?hTF=rS9H4oPhCv5(b(o`~=`ggII2V%c`AKMGqkUmYCF)^7b;EZ8bB+xz?&NlG-Nk zN_!U|z=)t1e}Y|X;OTY}_GOr}zm{ndfnk6%`&O^IbIw-~>Hrke=A5nd&X^i0tSB{7 z?zKHI`=(Q^J1MmTCb5N-S;)TAzbNiFwXZaF&&kqwB>u5k$GC^r2fQz@pGxoNDs-@M z5*)g|kGQI}j6SH|`hfI4N=&q;LeWy7zF2n{&Lz>%v<m4cB47Mf#kR9{PVwgX4+lbi zFHgx5(zm2&J`I2X)B(Dj<lFKI_4D^BdT*9Dz5=g1y^@>0U@kVhzerLluwUocCtSqR z`m|k&aPT0BpU*eviwRbT{I=fUzhIN8YWOypfkz0Z<D5XCroi2S3_k1*Dkm$Ngi2bd zA$7hzHiOB>nAygo*rS?{7nZ*<-mRu9We|WTrlfw%1Un}xAc++2k3OjtofVVlgY+OC z#Llon4&H}Lqn9ng=Rc)=lV{K1VO+Nw^IYZkSsI_u)V<orT_|5KA#Fsy`GV@Bb8me| zCXn9FqLBygdKvMXIi-wrtVFx1H)dSyn?dR>i3tsvtoo{0o3;56G(1F6_ZEz={EwFU z<|_pE!<vu5okse4tELYnCpaeC^OodGl8<|@77DXgVhyEUbH`AAeR9Z0CEJ`Ygu1c< zrQ$W$Hb7=~_+jkJck>#d#yC%@P%YX(c005xT(QAVeurYql+NA-?@bUE<}9*^$?K08 zJr!mV%Py)=urlxpiW>nUl=?)*an*M4U(VJ4N#Fu+Fl>~#lk3HFHKJ@oFeVO||Bk%d z{9*t5z#CQNuQg-*Kqg8}4etz&SEx6qWD9z^i~lEZ+!Xi|!z`i;$uX2jVsfe^h-#Li z!T$xb=wZp$XJZhSNrHWwWxPPmdkM=Dzg3fuy0po3_gtp=e~yx$7B=?{>X($1KyT&^ z#5Uj_Z7Mh!=t`G?^dJzn*i89y!izW&4iR-dNxk^vdw04aZR!j3yV(zHtHlvGiIuz_ z2*L$^0-VCR)85nOc5Tl*LeIXmJM{4yp0Te93L>yl3%EF9oW^(xy^H5gA_u9$`P*KU z*DN4<R^29lAXE5J>;SL{-~;t$nlQa`4JS{gjB97yQ*rWIPNqHn3W%G31(!cwUA1RE zJDUkjJz$fp*rBf~ivg2<i7X+H$QSF$+F<f00q?#?y5xPl9hC7BlD>pd!e{-Hhz|S* znSl=$j}?t;my4o)FWx}u1X}cU&dUda9Pwb(<n53(Atxq`gryjzsqIWt<5+Gt9)q-* zS1-R+ORz*2JVYw?$KhO~HF^2#qTi`WKNgq^;KwK~-}i#*wgiBAlTzBM0VIt7HG>&~ z6HX+v9izap5pP4WYQc0^uEP<Vexw{l!ARWFY@Ky7g*JOn1uVhgu3gP+Q=|GlW8|vW zIM1pGWDWlW=8?aBXrG<?V*rc933FUs>R06fZP2-WUo`k5k@B?4`OB7wd;6HzAW0fK zEA#S=un6>q#<|Iwf+*R8WL&Q7y8w+&j#uU#TzaBAI~i>PMo_2GgN-bVW5b}X^vrhk zg#GqC8a>K_#}liKXgk@XJf!==>CygaMeL_AdB{qdQ$D!CdoY_>6xC!jLP*PzN~X|e zFpXZ&(QN1k3Hze`o*d<z-BS2y5s&f9OLpwfWShk0TE$PT%B6^oQu<!!?29sriwk;w zo0Yo3_8Kg&!|Ei=pIJC*3JP-$?dm4>p79m)wU`IaYi30}{>IZBOayGWdgO7N0fh#1 zx=q#O-ZpB5iv=Rt^4XE^K`d2w-cfgJUab9cgvXCR!4rExw(J5Jlpqq^0xeEk0Z3$Q zx5KRM&^+TL%WR;<v<gvjq5$so%S!aVwrh1;ac}wMz_Mt7cH1$Pok{<a-1eee-9I2V zU`uKECV13n{k=p&^u{Zi1t!-Oo^i5(vY>1=nq>DX$97l1LbH<0wFtsHx03c;_;u0S zgIYhFVl{7o3iiBa$egRWDHKMPUHICO_|4HRk?+&G?y$vpJyPB?FHO<JhhwTmGAWm_ z6d!3-%Z_r}_VOE*R~Vn@SQ<RFyU(2iW>^=1RbQ!0ow-D~oE17e29?dH!;vC+0sgC} zOB|RH3TJh@$_Wyv$Bt0PF!tcM$vLM%>Y^JLVAPeKF8l&Id9l2qthb!g_9T3#$E03o z_Qg_XdTNTz^V(v2iYlod(AxdS@67Ih`i`5p-N$BT?i^pKvmiIAjsd!8VoU$pzTik? zUL&^_Qs5?<6OK<D>j2{Cn)+-c_sniC`2Z1M$U!a@M|3#jtBw&AqXsSo!bv8)(+UOp zbz?h^lNd)FR~mW*(_gN&Cw~}%tD7GXaJrowEdz+LVxBs~*27=6olo-|S4ITA^;doK zn>`FnJIWG!8dGvi^(<1VGJMrbrfn(<=OO*2vpb_bjm;4IW3LgG7XF|%*0E8pwblB} zJHJ&C!-gxTLU}?UQ)R=<F$|dJ?>P?4KR4rxHj#SScDgyN&eW`a*>Oy}1(asN?8arb zp$A3Nvplx-a3%J(p23&Q_kuIu=JkBfT42Fad<k8@1vZM_h|fh8GFewIAM^nhcdDC9 z0j1>wH{%MMWNdzI?u!ZDU0f>S<s_;-e80wFySsLR0PR`KpL?8jw5c!xWkE@Y{k$Hc z9EV;G?M_dGYn4@%U3YDNUWiWKjZ^<m3B$iPeALitc0D$UFp-{G$?P*dz0%!o5!V0Z zarw7iC#iNa9kG*E0|%6KI71G|uvFEr3;o4^E}11%K7?2`Ov(&G=lre#c*$Vf)`Q0t z3q06AX8X<ku-?h4Fx2`Iyc8*^DN^}U5ZM<)LDK#r6W1p1mj|g-e(MjuudW4_*UIfa z6*`SP;1Pg(`1l9ojJ4I*%q5%O2@&`h^?XOc(d~+Z=K_!=JRs_^*7d5hV%wG%DmEve z9~KHw@#+UY41>X4RIK$<(Jhyh)Lgx)$AGyYPpwQH@S_D)u}sz6eSe{|H{XPTY9eeI zBmB+r@l-$rV-T>x04!8Y3p-#&esLOjhmMu5FoHp=bV7DA_x5IqyW8jfhya3<@J1Ty zI4|dYe)5rI*=wd=;bS8JLwWH38S|1J9@7%Tn^iP8fJ>};Ar;?n1*?uyw>xsdNicPy zMrFk(>s*5-rM;JcBa9qRs~9GAhI5QZEZ1Nzn1u2#qDWIsp2<%{r5+<;p;@d1zi#&6 z8<{6f#%B+6U59J0ps3J5x#+-7jZ2&YFKw7#dnm|%i92~0eJ1*YH^WRaESZLg7U(-q zVnxm3bbI3cuNPp^6fIE$znl8ip!G%LjonpYy={6ObZ{M5?m4D}4)H`jhyogW5G*;8 z(<=7!K)nQO+xs#dns_5%M1XCi6kOcP!;fHZYRy^E;p)`Lbi`2i_ei*|-O#oA_V|P_ z^@lvAUtOyL-dHN}vg&Dgk@YPnB}E7Yspu0Yp_F5?L{j%n<ekd2@`Rtf>`ytNKx}5; z&T0dJ9E0Z=B|0yZj>1fZPn{RPB&26dK4a%94Q7;t!jffoxMLt~yBcjKk5?PbD}0m@ zFrqY4q;x}>p;HIpMyU3F^RSiDq#F$;bfutPg)ZI+PC2mC;1ctm^{g2SvU@BK@MU|F zeSQDl{g=Jv8)lv5F-X!{M{F``pwoC!!G`Lk$2Pf?#uzDK&`)foPz*Ggy=vrq{Y**h z*c5WXP&c=VuKC%F_d1e^^0zr689$SMYzoe8H0u5(xZFlw7RE2rMFPtL*#<zzs_}dE zErB@GV?Z{6am&7r;DR^HvfG50neq3NBT*X|-z=sW6P=SeqXR+n*sn;$c4+u4y})sv z+;Dm4Z^C5tSBY(xrMpQt4gvf#MQZav0-!!Y%(6v``0du%W<*xD;Z^gOAs<;KLxY25 z^iR9f-d>zQS{m2JH4ekzVi@$8BU5w=+i(Kh@~C8@dx1M*B1F}v+P4nTg8S)H&+A*B zY*?u{T%7yA0|tKK=(##kJjdnJ<xJrp1c4ZGN1l$0MbKoBOu#MFc`;0je7gN+fZR{g z=0&A2u25}>5w!2J=MC@?j>yt%aOP>(lAnq1KD%^mM+-)}s{C`Nr16Y=SZtQf{zI2n zl?r%u!v=ncZa<FDE!UOn{&}k)a{tRfJ=}K;?5Q7tLtivuZZG<DvrV+3cV!ELjwaG4 zSh_9U%mZ+{$ngQ-l7Z&4*B+f_r!Wv+^}OjBJ~t+6txm58cWxAe4{%IOtnf$qWmASW zeb^K9h({PE(;iJttDgktY_DKU-X3_q)EYz*6rtPj1uIn|_$MK`yTCB{sy4wsNcw9c zNuvU6>md|%c$KNiFYpR;n61&SirHx$M-Mf`?(<*X$vQ~867@|*J(~no>gTQ6vXa|x zHJb78kv__I-IpPN1%i`3R#zUP#x#E~MYRNs_ub2yuO(T2-4pGCu^r{I7kv3y>7#xX zmGq@+BCsmMId@&HQ%zr@{R}dP$_cr5r~<XOWM2K$ccz@yM|zmM@Muw(Xk`Xga@>76 z%+GbFFfWRY#+q?3ig-#XrPBdS9`1=bpH~N4RkNAWITo>0)G#M-3s66NXR`W*I{z@L zcEuvYEFU%-VK_2`y)0RU1Ff)%LGIFz933Qo)0P$`4{KO-Zz9qAZO5vNW7WcRGl4Qk z>;oU!wCmL^H?*C1-XJJ*EA9W;-v2X8q~B16y}%N|rG*=O5{acSW!=h_odl5Qz{39D zM?(!32rQ{3t(UE+Y?RVd06Q~fE1&SYA)>z^DFzB`)@Wp^Ll`lRP-R+F>WDqr3De5b z`^eCNYf&1jkdw2lrvi-sXsO<#ll+Na(gINe_+i5a5k=B*Kmnr<{&xvF5_VmY1?xtW z@xgqJXH2)Yt3e603J>-FB2{ANI}qnN@sC1GgGVNlF%T{xcNV<tvFyOMj<fE$*$ixA zVr#)U;9PFfNeAyb(d8N*eIV^sYCkrX&32VQs3{@=+2GzSQ7uB%hrEPu8o#-2us-`2 z+)^|Jww8W3f;%^Ftgv7f#QH@H!#W}-SBkU~MPbIXO_6W^h&RtCoBJ@~k=(vd?pWT| z$p<-4dUx~EvqjdFvSa0i;D$FGrY{++BDC#97X_BW@KHF4!;eCCn3iClynHJ1sFbIT zjNIu+N}YNOSh(1p9?m@5TFD06wF}mlhZEkB>`YjV!MBHElOx6JeaM|B9__ZbL}#+J z@;}+~%rg#Opd02BTa(E{in0JvoPAE6R3~Dl{IIFc0#|idXOh?N9O?0%C(J$&`RzzH z6to8zMoST-Vtyfec-4~MwlMi;=c{fQhug@o;Yg{7dJEt!Z@J})w|z$Bx_vxR^S!~j zt~2UG&AxR#XE&YGx-)^PvDz_iv?g0TGCRe2j2WuS=y7*2y;qxiBG+IB&(k(T!yO+o znMG|?kCOO5oY@IXR#oh=(kf}(|B)+V-ttv2f8DQ#Jdu1fWrMjN%e!IExu6Y29)>ZB z$y$xb6?N-9-xG!AQQ&y5?1vyJ(Jfc-d2Pb~5IRS9?A^I>8SR@v<I8f!qPHYGHrfld z8N^P)M8V&xHXO#+th2+$=!Hn8dG9LLxs$Hw=P-ru&{;`x@hA_{96;aNGX=%sB(NBc zw{P=#9?`Wgz0lwOTs1ZMrhm@!rv2F#Agr-v_NcB?tlS?2!=LcE5ebg>Z5{+mp2!MB zknV5U)6tW~0URBo1YFa)J@I_+J|$KKB}t-*dLK#{ex6N|A2zR~eRU=Sj0K?l+NG4O zCzQbGPxEbI<`;k0{LXctTCGAiYFy>2m<(R7dNALBJ%YkeG5Kh=%v|zzMB3Z~mugr$ z3OilJ_tlrCuuVit+u&Ark=^FrqQk=xIC>mR?OuBKbNB)!GZc@Sl(Fe~TsS`m&#Laj zE!s-;T6mUzTZag^98t{l(n3A5_bT4Lmtwb`V=}HX`W^Gn*Eb0i7Wc4-c$OTcn_36~ zv4-BH-?^&yQ_zQD!a^_7p;w^Mgg7p7{v5XYXG=coev=(-uGCHBV2$(QU&gA|Ew7kc z{ptPD&gk+-(w+fT0zGYKx96o$LC^cVQrawcxU<%)XeGeLTn6sQD9}Z4%<u8iQv^)C z+K+G4$&TwT|I#>ko_!+WcZAo8wj1bz_B0(w6S<fB63f!`jfV;xw;^Ljbg2B&gRAb7 zwIvAkquDU_eWlUi{X1{M=DDk_cCkXc`le!JPA8m80UuH6+9mm*b^y5XBKXgYU?jEW z4-sXiriLP-6A>7viAV_X7Yn1VtBstuT#U!`$LSG@g&jDZ|N4}GMM!8V7+AcPB<MTP z`gQ&5YAKj51646N?Ot}EcW~16$GK18wqLn!!&&~BJECx=!y&fdoz>lQYUa1*vyfj2 zDZ0VEejYAVu!Gv<h$~=4`Fz%JAILjH7gWQGn-<uI6-ibMMDI@!d+liX`mbd*e}RC0 zpwglMmK!kWlp^@N<cukTaV@A9I#IHo9;HV+nUOgr*gB2~@>WNn@aPy$=QdtBh!)g$ zTWxJNRx@2Q96vaq>mF!sA2_Plc?{&9A9S`Cv}@LS9Ld`g2msc)cgv^mo=kV)K|=WX zwq?;EART+1hojCaDqV@^eZu8fzvam&jnV{=17>5ul=a0vn^*>M(;qv8e#%lX(U8(5 zXQhY@UN<vnT7S@FdvCHPU(|4cot-H@XnTW4r2(k$p@njvcp03kUGTfU@?tfs_%#rD zqbfxv^;c4R{$^qrx_Q$39*?mi0i+DdQtw*-m%j7Qrcwpthny`kes{XM3{cY64<`_4 z&aPg5uV-fk|9AKJ_jL}{T`5Kw^Wt5T2krzy>?#Q=P13ytL;&l}t_x0-;&)A#PO+E3 zAIh`CIs1ve6#mg~pU0MRK@`Vi;VvM@G%*W?ki`4A_dzkEOYgf?I6@DzIdzoIy~$Ju zm3SzO7!6p-eLnMga*>OL+i1F(fQ;P8q~#o`KL@9D?)5@K6$C>{`5AK*Zv!b{;sZX6 z02ZaJgR2YL@vY8|$Jkau?14t|z7mxpEL_~+?qIZ_mpRo|cZZWPALKuPJEg4k%X5E3 z<@mX=8jV%G4c@zD`R#s;ynMQE@D4a1-^)s*M2bgy6HbPY?)m@vMNKudT<fP{bXVdN z`j;r^i3T-CqL0>|Zmalh?C693cgTH>!Unb&H+<}m;NXA_!gQSB6jr+h+V1QTGtbO; z)Hf82MXHhxl6^mN!2xjXp>3_|l)aygcNem#LRVdo!^{&DDfgL+RSNMwCo>UZG3(Cg z&LMIdY~3_rd1BL8CyG~uoOSsm9b1Ct%{C|(8@<NVBYj;={Jj)+c*9`^;*sSw(CK5; z(a9UyKIQ-(%;NP*Y|1=JIDZW-=Z={fRnSrqvMs0iSYomQSfOg%psT3u!$!6Ns|+DI zpFIVz0-2<ajC(OXyl<2D%>Ky#t+lG|G|Z>S^CFP%jEz`zLpY86vg)ur)9CeZ$E9vB zN}r<(|4+2dyb%Tq_Z)LC<pbyjs5AP?K2<Yf*9U9zyYRU3t`)Q9UL0X9rDBy!;ix_{ zoAXoS@?|1e+lt81bSUH^W<=`tVC5qw3KbE7imxBqrjpi$<ZL^#nDno88bPzIAOnMb z2tw&*6jI^)4?=Yp0#ZUE=}nk_OYkGtIfKom06__AysnU1y@d<<%#zVTnN_xmQ*b$D zN(W~{0z(dZ|DI4OF@Wn=Vd3V?q5FxT`&4NEliLaE$QmbU$zcgy3NvPV#iXtvkwYhS z$$58L1pc|+35MKy2kt-uj)Mp{LQ?I1UyIvHgpN_|(h?ZvGOib%L_S>RY;<^{`J<&X zvl1b8I80q&SYZtHFa6#DFCL?z&5FTvD#lW-KRA$eU;X1+%+;Dl;CitAXFm(A+Hx@V zz57u~k#{T$=1lr%f%<mjZWW;#A*;}u77;(Ub)`+}%>brv&W(cTvkAD(C|$&-85U=F zFJS%d8Rg`Y*x(;$l2OOE2u`??b;?n3!d|Y+r<?ow4pm@260s8afGHYMcs3k1t!8IS zV{%#i+w*M=^U*EzeBq6zL!26ui?_X}Lf_#U@p_*zxy0sW1gQiBmKxBnVY+g0+F!6b zSaqYID&|R-n!?vWoEI8!RHjkY-|TX%`0FD!;E=&zHoARav3!s2a(5!vxUOE+k3=eS zFSp6sn~UG#a%@Jt`$y_H*}d%*Ol8$*rQPg<;H%pVL%d(n(-5Uoun25`+5Y1nPsDif zvx@bd9RP5eGh}I&K+xF%_gVLk<F0?`_DD$PbwIhW`XdjVb4iTmzd%a*jW2<lu<G(0 zlEvAlb<&Alr2+u9oy4F9x71y)v?zGwEhB-CfF0V(D`Vro-cJ$*qi3O5Z!zgV9A^^! z{&qdGfRo*~mv8%uMV4!z9l-M>$7LL9t;MK{8rKLw{}`Be&<|(3IpM8q!C-=cf&~|{ zKZRL1uY<WyU%~ifTRIR#|E*yMtDt4NER&!&BI?0$D=>++mqYc&RVCuyUEzjDczN58 z{=d6WKfL*Xb-;C5VG*SAY6>xRJpd=#Qu#MScpE3{f;L}mT`&>7%LOz-%YqXII{@(W zYIlZF$04-bW{vWB@Pi&{ki$jrDgj%>+mpF7D8EAEs+<G>i6(5anyoan-0VH;tLeFG z!+G6))SJIsfo9rN0Zt>8w$>`X8!^Fbjm233ghAYW9S2q~&MtR`%R2KwBKFVUd0b!& z$x5lWo!{>mN~$EIxG_r%$~zl<+xconyBmsC4j(>y(c^EPDgEAcJ17ski%E7S(|4Yb z+^j(_obMENu<z-eqf1&K<G>Baf2(r-Td4*w3wzPc^dP73sIPS?a)u=9QbqE=1gL<r zTo_F9+6PNN38asU!Hg#^uIy;fXEeaC1Q1KWlES2kCRTdUCC=0e%v9!mz~nq6W7xYY zCHb5r72FsY^Ve4mMytSA2@`uq`I{fG3F{%JgcS9HCIu)G-*|rbrn)o^(%JE13TNA; zeO0TGn6u4%nVyYmOgk~jDf{f@k&V4+j2JM%$q=$|$e#vDOtGn*_Rl-NIa@anC?^)3 ze+M7B^!t<Rrhn-Nv$%<qlVr=mjy-tl*<cPyXzHC#Ee*l9YLJpgMu`ZI-|+*+v2nYQ zC*+Zxligvf7v|M4?nF~`n1w{M$vzhwvdFSaACEt}%q6oLen${DdtdCBG-WIE)Ww+- zx(uH^o-MlmF4g=-(Id!@D0;a&{Oe=hD1~N5p3o&W-D2wig%6L$<f{NSh3R%o)Yqh! z6g`q0VyQ963N2{e<rZCjUZ^x>ZmWh=8mcRq#^01EajnsEVYC<VBqaG<g_<K6w&Pv_ zbTKWDJ&Ht88ILk(<NW=E|E3fA8}TexzD62YVA&~ldUqt>y#B`boFk4oVfTpKA1|Wb zsE~qS;l`tYh!muE3PS&);E!*qDl<*RMIW5YAsXJ?pY>F`zOvFKbi5_2SVB<?p8;!H z@QI7loo?*<y}#zXOTQ4D0)x%G#Q|}Hdc(#WE`E%bidh)_;*$)!=`g`hKEHQTMzveh zrBO<h3Bpg7yj6#-**}xRT^t6gHx6u{uv*Z7sH%IVJAyNd5_WEi;WsUhnOb8Ml#d_( zgmOF75VY;H3dMH7KO~ouB_50#WvZ@^eV^8}cV<=3442BXa?4q)V~%T+CX?)!w5;%G z2DJZLSBQ4K?dp;_o?ec)A1DvaQqvKQJYqY6MKjkTXG@Ct%Mw?gO#X-aPrw>MxlV)s zM7-{Og%0595AVt~V_2YPw@&FcklN+rzut;&>(q=vH-u4>c>|YU_t%#%05OH&_4aJN ziZd}XbXYcpg&U!%Z=At)S(l9+Gv1AcQXxwSab@&hmMLGoLi_VAPLqDFI6@+$##qx1 zosiJVV9L**ksu@tLm*1f@5z5wM2_u6L3LE2W`yn8XTEZs!r(n}^y}r9ACqTes=-~} z8^KqVL)m8cFp8}<dL#!PgZK;K($6BCq<w95pS1+e*~{j?bcC6P@=Qy~e$v^BI{3TO z1I)@?Pj`>v1!|ZD9=~wNMY?{$ADwhFN&Lq%Mu_bLk$r9)9L_shY(6bOT7gmYCbbnK zVvxn%!d~7(Ja7UjHveUuj&3U~4bt?BA}Vb=eI+rFXB*0ys_^IFL>@bFDO0EKR)m#; zCl&!DmUyI1p!{Xo4v2xWci<pTAGUT0%zW`}1Oh{BeSen1)|;Z%h)z6)>AguJBUn}U z-l$`W0tIzZ08r*CbMdL~F6eZ@!81UDtseRR`u^IVuQ_RE@29`4+HM#tc@>Ea#`w+h zM2H-ioaS+G&FQdH7M|iSkMVHM8?Y>PA!=0EgT4(>?8y2T#DB5Pe@s`Slztv{rdO_u zSBek$+5NkmPOI0PmaVLugtW~gG>r?}&#g3OLH@SI>bohg2;I`iop9T}q<PJVp%_G$ zzpoSnP4UqzkgXMY&=x5pW&MARMmMY&!F~}a>ZoQ~tzWK@PG1EszU6i(r2QVn==`Ic z9_qQUcwul{q@8_~i!ysznaDC<)-LaIb@^YE{t6T|bl;)4uYPTENzRMa0n>@oXWec4 zu6<ryAz&Ew96z$wVa#uT)S!u8`rYho+^z=yN5*6bJcFTWC`BTnByCW%hhV>5y3Lb- z-xdiMWQlUazJ|g7WKWT4wyHE6*<K*Z=m0Tqx6457AjOiUOyl>nIu`AXXx;E<*}4a{ z$MGoJ8F^r=f;oI7otq#<4p{IO&5B?H^s3q)O0hk-HaH(7ezTkGD0U$GZE>2lmvMxo zc6q6Wuz-h``waO=V^#6t+osNIRNU2Ye^R%Nu#jvF*aB5!)rMVxFzZ0)OFk@gK@9s} z15~k63Ke{5xl|g5xwuI(9R)(Dpocb1#5yNDh2QUMVwH~F=V<a4)zIkib0QAV;2lAz zE$^+#RkR+N(whr7&~3XFKVXY?IjQ-hz`G=z9n*<j1XGJdZqUy3!+_icEWIdH*lZWK z3sI{;1EAnPNC4^yoi%o(<pT<qIE5bKcOk#!#s!i_+*RusF^iAIo0VmE<2~J@2AkX< z@K|W7C)ZbUmR}#<Gf|Vo<+2l8ZyWPDnvox+*oFaT)_~&kV%ZILVITfl_UB~m!_<jg z=U+tyl=nuc@>wE5+q7d_TYc}g+h(o-n6O~X_WrHmVlMxJuUQ@J7fgBs;%}(;65p3_ zVu#-lkCVv6M9Df_iwDBubf~<LNd_ywv-A*1OZ;B<7wFZv`;gfeOWJ{B8+FGp(?hy= zskz6^$jQ!)P!ykVZoZ;)q7XiCv2(>2|1)$Hr6>-MEF5s;5T;}UoXpO~*W6mwebS4q zH<_~dbeGS72O}!9hE&k?I^w|+;jXShTP()xo<7|shD;dD#!+DP`>Z$dQ#Jy)@;}K; zVv1uR45?VLwCfaPB+Q^R-6!r5t1cs|9=*C$vDI((D4NvDA!uT);O*V&0Q(o1&zu8o zjDdNCDyrsMrRrDmkz^NS;rOg9J_fq1YPVWu)+}w6@qW8QN$!__X<ew<vzRrjb1wXB zY|35rhX84DwcY?<glBs6P@4v)>yWIIx%EsMt=)FEO`WaZifz-5_aW??(kz(AfXyJ@ zr^xvBIW@oQS1dhsrs88+tcd}JJ_4t%2K{4D2u)KioMD>Q?!SYXGwKw4{!1AGITEOy z8a*?aO59MM@}u&zH-8sh&`>?7n9zB+M@9Zg6ZGSf7|uvf{y7Erh)Wc+`_O9qG0Ilo ztPz+O4sC2&>iNgt2@CqwWOV%8aXY^AhF8u!DD#F_r3HMdn}zFvurc92MQHxvSJ4SC zIQ#OPSSma`nb8_+n5)sEyeTpZT?1yual705XnM?no1(T)8rN-5Xe76)3&sv){zi}x zIQgB0Egc_?J(6X(sVy^&Wy^0Gqi2{tYO+!mdcWTmZK2cS`*lg*gAOw!QI3rO8yHVt z(p}WQ0)W-+6AhI<H%}&xE)k5-bqFn#f}S~+-Q835jUlta`F-t1yJbf{fY+l4rh3}j z)konB2_%kzH{*N3wnn~4ZY-irW_Um9d>DvCQ6>l6eg5bE&KnIUZ>XFpBrm3uDPa3^ z?Oc0{-X6l3gT`_Uw^+JcCgKKRJkmP&PeKLj4MJ=2K$qeqV~@&4zakhH?CKbssqvA< zGc_$w9uuVE;Tob0$uTxl0qQ!y*s&-!d9S8yQSO>9Ti2NE_A|t(q8CW{sfBtvMi$w; zQ9Zk*e@9`za#{ejF1<1nY5}f<O`TvNVC_0|EZ8ts3!N$Vos}4^eNtBpAmqz~qdaf# zO>qJs?y%a&K!r{gfluO61x{DO;`LCX7}V-~-}h9+bq`)hH9u_^D^^NSBTZri<6Onm z_CbEu&cvrnzwTI85NKj?fHS(_S=_~cR-bnRB-Gj>AXNZcjVYD+lhZt!*?VM(6A*+W zN%)mr7+jupdqRkeFBcSk?N+;9ZTE(a56&@V7B8FOBOR3U3w<1r=(O#RvC_BdKxRD- zEK~iV0<#GCDQw6J=ha}HX<_1VNlM`4!!eja^e(r4T|Z(<UIo|JvHuxHX5rM(k^e}g zg*eExe+GY~o5BunLUVAxFT$TF)n=VxUF^4*y4^m$IJ665g9aQ9zKcV^S7D{#`5JSv z7Q90pX)uQwGIr9xJ?oig8hi`z|HP4lqFI50*!N%yO#uK-e)ab+A5Mxt6bitPf65ps zzm@;czcq6&`^wV0H4?1C7w3#cJ5Da8?OCcc|9l^@{fv5p{W(}~*ZE<~LvE!Fb{f;B zYq~K}=5=cJ3-f3BFMXTe-Ps-vH61B^g*~u_<;f&-Q%Br$Ix8#^s|ad?WtJL_ysc0R z$ydVrXfbmPqZpd19H@R37JR)uSwsXF+D>MmQzqReHje$VKwY<8nP4wQ;@#3QQec}e zLkzV}U#zP)kE^(FHI7|-eT074i5QD|_+u>_VNh#=A^?>0os%7E09U^^ehO<4-tmQ5 z#<I|CFUT!w?UOB7Rb3yTZ)M&Xug0Mdx=?V&N7DxS%6EIrv^<k>J&Jmn!<lI)F}*a` z(Il9B6c>kseWUZ%)g(?$RZ$WpVguwO6o)jZg567()4neOpG`*xn}i%n$wHCihYvSS z8yZ?GOX2?mrq=xZBhTYV+iTV7uYvT>cl>&4=J9{!vd@A>+U}g)T-01#vb`SXZz_!g zS;TKhygnjV$sM<7BlEj;BHDIFZUEf|mC+iutbgCrSC-G3`-rflmDPF^h(_FyYAB4r zetN&E<6Kj+X^ekYo>uhxq0U=D+_-3bkAzPirk2OkrrYr;Bu0*|D`k>R`j&FTz$PTn z9l5txpQLrx<T9V)@Yg3{LwzPJlhKzWuy{EVYt)mbJjekLHsI^BDv~n6@S+w&1ZLoL z=YV!czJN>yN)AJnwe_$L>_Q#|*GXdk?K{d|fnRJt`}byG@E&zss)A$a?ISm+=cs1e z5Z3DZg^K*F1o364ZyUimGf0>H{&qIn&tt#OmXeoZ#?mTTqE6jgJf0cgdXxJ-4La+g zth1!Ie#YGc$-fcj+g!&wU#u}#C&uPmb$@kClONZFClXS$vfXBq;I>4LwT{F2J0kya zi6*HoaLLI*es2vnSo?(fLix-3k43*(IW69U%QweRLIdlEkGq|hP;E-g;M5w-%!u%5 z-l47=YFK|#uIL*JX9<Xo>YkCdIjo{!j9cYHuJeI7Tr6_{UD6k#kN<fF=fH1NiJcUf zW{6cZYPF`oz|(0s3nu$T0+KPGjnjBFoV7mE_<!FAPx)ik%WB0S#RtbTeF701fY<@6 zu`0uA<gsV|BJ%j~mbK+$?w1l4JflqZbr1z#Bw@^IFkx&F&j~W8*~o!0+e^m2jIB^h zdNB4^y4K3JT>-w^>N^?Y#o7{{`dqzQ{)`@Yx+eI%0)T?U)R2kvb<5{HR6`lh2jR>O zrv8fji5C;`O3Yi61@HSsQp&-XjDToc^l&K6f4u-NWCpwqyvC)@Cs3G9f;NW%*PC+v zT2y_{ODPL$+hD!MG(>WrvywF9PNcDcgYN@`UwD|a&n&^p2Etk|38sQY<(jW(QQ57R zxsEjsDH2$=BUeq6tE=e9wdZJe({isfD3l$2sgHlqM3L-TZ>61eXII>Z?u)v4MXp2@ zRi+d4lJrG3-VU@72h}`W^@$vLdE8$0k$zgPRA8|kAQRY}M(|Of1l@I|rk+hNhx5yB zY{mH2Xhfz;O#jBxj1=uX0D~wjYb*EeMMZNYYkiL~q3%{a^7eprx(k4iziEliKwanC zDmy>kKMRaG@nA@_(#6{$f-X;W^d!aTAsA%(B1L}t2{W9I-;Hmc4Zkk*l_!!+giO7W zwd_R$#C={?R2kxx-GUyyer#;t-(?V}J(8~5Efj<cmTywk2zbDTQPlV`Yn{IN+m9)j zn=vCdzYJl`#)B|R*OPTb=bo+Jtr7gKq#@nqk1B)OSG|$g6}{6+xIvv6%%E&i`!{l= zxPAw6Sw7*=r#arNpjVGmY-qMHM#y&KBB8j&b?GgS=HlF3(Wj9CXzvlt`J-u7x6>%& zhTe8Vh;hRj_AOx>?whiT;D%G%gYv}OiWPKL?yA6bRv%p7`~Is5rPCkSRMMTyjVo}> z%*^(He%1rv)q_^~q0pWUow~}vwuS}f#$h4boxY(J)cJKRH1d$SSr6lop-hCQHK4NB z{^795j)I45ApptkoG*61vg%b8FX|;6#zBe26Dnu(ggGk4?vOG@e)^*MJo&g1Aga8o z<&6*V-A8*x5^aIXoImAX8Leu&tWV02qig87;6@U0mo^IB2zuif4BW0C)wTa&owI$9 zV39?rtH(Z0=K--2)2DA5&W3Prk9aDg%u9{Hs(3h{iN=s#RaUmT7{J9M!Jc5DQ#=}u zrP^!3KSe-x#1!6Zxj(*ziJ1EY%LVUr9$pa<*b%4r#co<1I;GZo92Sro@dH^2&+;|% zNtuEfUVk^>`$~*O3%fx6E<JqsnQYEYEQ#{77VHS+d#;j`Vvz0K0{i~8jE|<3=vsSy ztJ>X6$c0_2kx5RCHX6B(2`huGq2H=|Uk@4eqWK>PSNnjmd2w{BCXR94#FHtZAD*B| zKv{eFrI5Ue-lu(AW@iN#iwlGHH4CeF*r$5Pg2d;`IU;tE-ftgfTkKA!qKU~Gjc;;b z3h#+{U<Vh`oE|pj9zp&=TE=tNT6Y$(idkA3Agc!olSDPthszfw<UbA_WYx2dN~E7Q zv~3c4{y?+v>I)u@w5fyIJ){j1d&JhI4vL9_=FJ=WJ7_$&w6d6Qi(D*kNG@qVWzWrT z4&O$T<0g-;Y|Rtb*ucf|=GCgGd;<Ny2P6*icCwu_TO~N>HLE-GKY&u;<$a{99&9ci zGVUCBVVeXTSe(4SwmEIqW(@>3A-0vq^dIDR4*8C+@~)Ltg_GOg%+=<y$8(?7M4G*9 zx->WN&he3!^T<7a2A0nSA+8O<k63;!o#@j|d%5`iQt-#h-YY^j8NcL+sx2rG!TkW6 zMa@zt8NJKdxY$&MekgJm`}2@jKQK}}>A5k(^OnPOu^;~dC1%%8r6H{Q@Cvkuoar_8 z1LL?3Z7j6mt6B`3*v^4=rc|QPxkY9kKtA{E7wNN!-D~f!J1`RVIA>q$se2W86~kiK zyG+9THiRi=z7S0f(o;HTzavHWKECSGBz@@jegofoQw)nDzR~oKde$Vjpjsq&FSFxQ zF2-q*#|0<+>V8r_WI|-ay?~-@I5hf>Rl9z$rM>m3kTLi0o6fB|A?Kdb>0^p0Jm!ig z`O~IMN62r+Ia-Bq!eair$TU7Y453pj_d5rfu8?)l_&Zy2`$KQx4|5Zr|DgZE;*1e| zyUfo|C%IiqJ}|cpS}-1dFp`dZme{q*I)&%O-xl_8Vx4cJzEvi+f?ZL38k!EM!%xx3 z2#c-4ratEVw!Zo>MXI<J(KxT?Myl&PDOEiidU4)!r-y!}-U`zawTzqo;~Ojtx1yLo z!RK?xKvnSk@_ZNmnvcZI{cVrv5^kmNuJ1y#B?ptG2l`U8_C^xTc=zm`0Sr>xE;95) zu<HBQn=k$|4khh)M>2C@$T^Y4*~xc=DwdF4yUO6?WLM3pr?(?oj=Zh)d!H;0brBz< zGm-7S5uxEUEZ`kAERb5sv>{j>r|0~xBb?)64f4`&!r5c-Bv}*w%As2~kF9vd-<0Ez zRZs~ByGg8{g(m9t%-`hkde@Z*LlISdaoND5xjcFLt*lJ>hl^h3g!*Y<-0vH<(n?r; z8!ZI82JW8pN(=&Z4YYjXA6-t?8r^ny9mv*#f^$Jj$X=kC6swI1hvjaO`~e9SJ&E6y zf2@<cvQyeHR+CUkyiRJ=4Ec|1VSd)!n2mPiy11-$eTgqGz!W-%`3ba+6Nh>MdB$#l zO}8pIZ$uIssil#-s}&Jra3|n8DlXWE*%8<O^^2&u^?B=`Vf?dSK6_Wuzl4}uOfBCq zb=bWhN#{n}8)d|h?3Wt=jD;b9X$oWQ<+jnz9tJH~n&y~PMD){s*^7V7B&LM*Tq{`i zS^lW5QcqxhK!nv;i@xdDt6_d95@OwMu<;P-(SNn*ACuk6PHZ*bb~X)KA2^*$f(lQT z>r|HENiE;ZtCemrwW6@BrvznyBgNCNzgetp<bq*?uTf>XH0Xs67;3=ZrRo*Lrh?H- zDM-GF;{6|<-omZP25cW5AR(>N2m=e0maZ{C1w{k_>5|diIXab;?hp}?9G#<Nqd{VH zN%!cn5#OHo{k`9D{{uUYZRdSmXIy8O%k>6Fs&=4$)HvY>&BI!A^dH|J97eq+C2W5g zeA1r!@ud(D)E7_#Rs4YtkHd%k&yxu^YpHj<#$Z-M7}aZ~ME{gjH>qTWFm3PgOdr%t z$K>iV<gHv8Eq5AcsY6-;yMI_(5|-*rZlIMXHUrdt5j#`Hw3DPm-Z_T*4`7!K@)psu zUq;Z96ga2gc+qORDV7gT73~^rjO$(^jx8>%thjC8$he|xAE{1jABqfko~CIFIsc8j zCCy$5;87c&U4d$@L%L$0(85}k;5mhW-I6!TgeS~?sA6~7P?ZGoTj{p|6xeC@rNaB% zw%Cc{bT>)OR*{-h1^eeZ#_EFqyHo$*Y{4E<aep+N?gSF@xtYs`<9{Ag4!aHlL%Owp zU{k_Fl4@c<^Etzn1XW@Nq&^)|q4(CPzGJ$1%wJKHNTWllXD)b!%N@EZ?AV-0^4}w7 zMTMzmhXm3Jo|k(>%zF7{)i@;Wy(M~W&A|Xh)fO#ieRWv3DC$d$B)Xdp!yM?f_w5nN zdUswsZimLh*kh6~yeb4-H|&JHX*ZZ-TZT7j1d=o%u7nJCfh|<~8u*c(Sh)6W#?+Ab zD`dzBC)xe8#0L3HC*@5&cO$Vi-TY{&Xmd?rQtql;TCENOdRwo%Q9%xv4tc0tWqWPj z^<5yccZKyth!y7#=UYgQ_azJ7WT{g4C+7~uEJMEB+N}6ENKTE6reDM^AldyWKY%XR zsQw-4+@e*%TRQ(5v<@ob_;qo}BJ}F6V+5nIx4evn>b&KnJ>Ng<g8TUkpEs|IJ9>p= zGj7cOu`fmjgfPxh+{PXGym&Hl#Z9o}uQvdpdA}_7kSrq}Vg_1V4XqdQbX{D=x2+7F zlK%Fy*;_xUKOH3j{)#$YyHrE9z92}49v2=eqbT7-f+8*y9*1CaU+Yf1|Ckzhi$Myt z3C@GATLf*VxxKABA9`P`d1<Yzyu)AjpyIU+cre_y`AEV#ZFPAsw4I#m+LRP?+5#t5 zI&gi-Yj|<r-My}hzWw%zyJua;;I>>apVE84uae-|k7t7R@A;B>M61mIvUndjspOZI z60)s(dYc~!tn`UHMH*!yI=AOk!@X3e3Eokye!iUc>S)FnmW^b`T)8nzNgJ5<{GM9o z36uaD8=T#rckOVW4%~a?!aK`EOlyaY=Q<4DR}$&$X<<*Cc|zFBGdr9A@QVWHmF#ag zJ3`!yWjoi-M~TPqc>XaOeN3#bqe+BxXcHcswg_bU7_C=^Ggg^3Se`&0hP=)xpB$Vb z17rDlf*5w(c^^Jxc`O7jdSbVY53t+?Vme60XfLMPZ$xW9@LX8t_IN(sr7u$_+SiaT zgXAHB7y2xbqLIVaE34nJgEVgw2WM)_3<{Ui5r^j$?Ki$1Wiw-Te|qm#%}Jk!Jzn&= zlkJT;s@kug)25+B8=YRVpA~$$_~5t=?V(vr82rSUGb*aJc=D{g`*=R$4DBOx?voL> zpu3>iUXC!AN7&9<4ah$=ycHVp7J|t+wf)(R3csxZti{iJ*Y_S4t=>CS1604U%Y6wI zcFMDV$3WIkUM;CtNBjBSzMVWcd4aH#p1{PF8STMKt0VI27jdV$<z2ENdEQM$4a_uQ z!m&_NijasCr5!SF@}h3P`9>GIwAsLN`}~Dp*-r-wceNGyiP!cUOW#*kb)6HiRg(w! zgR9FBDNj-VWl*+7;3kAFu`Z5n=aywD3Gv>xo!42ECnBOBKh#m@Acu8c_pp^IQNZ1{ z40xnx4PCc_YQ)bx_OlDag8Yo4u%Me*9ZGP;ou85k;u9}}H~4_iypx!}lRyHIDktsL zS>9SsGQC9q)b6fww>8u9N+%*Jg`dp|$_xTj4+k8=&y<4}9#!igv59|ZBqVt8Umm-| zn+^3Px|=_FlSY%UENhV?Gmg}^fMam&yTs@J52k|bf)o(c5h@7=$4Nhe|L^gp;`kNG z7%<h)#Ezw1nsc}MpOy7*&>=wHt5>1lS4Oui8~aVfbQ~N=(93|CPK-@R8w!no?){#| z-Wd%Gh0#8OZ^ZEYM#BW$T4}%t%3j?!gdCAUznOS&0FCF$Vf)q``VE&lEd{{2_0gJf zf66GZc$FK5$%D40b7AUmI(C6uZb}Z$_+o^Bp5ji|3Zz)PFTx&p%1^+N6Nq&W%3F8X z8VQYu2rFyercCpj`#o2FK`g)Rt%s_sCciUU8U<RD=hL!`Jv8q`f?5__kZjJK#Ehrg zo9J)dEB#9FIf|A)F_#Gy_qv)czR7hQ<~7lvTI<R6O$<m!<Zhv&C)F+WGnQE|3~TW4 z)qyYBCe}R9BB@l<t(H%Zm6QQcNMfc5l7&EOtS4tr8J<w|T&GJuwwp&_a}SrJrmG7Y znIX4jO&Ww8xDARg2WVrK5!fLp3Z+v_qZLDtP#sj7er(7v(5kw5Knu<;Qd=URgu=bA zj*u&u)%eJb4a-JMzFODcB$~*yDc&GN^P6-D*V0_xUNX+Kcj#hq_HdpXU!rV}>8H*o zkXxee-8Y>VH|qqt6IbM|t*17a;cLdu=YHAbx6K#K>Y6$<R1nsD&sS{8Ej;L})~8k0 zownR*1gW)c)B=^w52%?J?vlnYkk_+<$fY8jNO_ci<U_HV`7M^ySjSKKT3rvr>9-CX zBN-X5>G_*+BGWEw*6IFe()?!t^jrTjxr0EL{dbs&@xUOvbI^TH2dve(kBLFf`pa5R z4+Rw+u39e-zt84(qLn5lyL+*|_(4mjr)&FDFnR0WX6iSSlikzUl?<civ)Ik$L?4>Y z<mZ}zeUsI`M-BuMH7xPLp*0B8MPE%{aDNCf#qy{k%>sBWL++M(;!yH?^Kgu;d8D?P zo>#Lk2CI1=r+5-yOD)Ctr@~K>6tG@rL<hCc@Uos6HT)7;g<nKSjRP(%yz%+H-;veJ z?FY8At`yUrWA#;=6@DEc+#ZqiL7Q;dbOCd{Et8I{$n)LmGD+SJzJ9GIS34)|>2;Bn zhPg|aQ=9E*lIEW-R8wh*@Z+Nj*?C^Nz#r9UQMX0mXAU~9&acV92*nMwH2d3@Oi3An zUY2s7erJvXy!SLnOB*oO_ncVCsOizH06FMQ#uGt}z)mDg(BLC&K~G@g^^phV`}CAp zk^0-$T}b`go|_j1WgzB42=4uA$fV=LKhXrE(EXMRvDZLe0E`%l0FBrGWe~b978|#m zuMcT?jiidbE{7vG<9Qcwh%X*{rZ|)oZ=_P#zfZaEL9_-exL@;Ql1V#5GwLi}RoM)n zpR1@x(&6zCglx*pna*#orOda5()*ov-3Q6%)>(Zu>VvK#32z>Q1zMGBb{_TGw$;+U zBv$$uH~bn<t_HAP(bSKmS1791+f}&M7v?k&X3}KU>|j_*JB{zI9kxuAs=iXq`qa`K zm%BieuYy}kLfQ%#7XuW2!l?89KJQ5OO04qWrl~!?3ud=wczgVW>c5B5fvQn*rt{7< zF*B~C^lAC+#C6!4CN@JKRXuyYQYb!Xx;UWq=>OqY#}Fq|L!Ho4&vn`csJf4C2LMmz z^hwI#5X}}z5J@NqR!2b?Locxs*KyzGpWB#j+2+du%pKmaS`j;lDa1Iex9$3If=H>{ zb~U!9{oe$ZuHhFxw}V_O;+#L%6+Tm;w-lS%*e^T&jglTHzNyZ@yz`#LB{ef@r}YeF zh<HPU*u*>W=PDffdt5^k*1_pe05=L)^E|`xpC|KjRT<GevmjoZ_9kKFfpJP>2C^_K z&O+CoE$YLIP9*jBB>Ueq7FEMZp$+jDn$ie``C0)O?bpg%OzwbLbRaUWQ}r%T<$C1u zE@?X(fNgGbG~G+;v~xtq(1qC0Z@<nzVssd%7S1dl2*(=dwqxuO{!eHCtE)P$m>?Rk z>$L~1&wF4tWvc^eFfvAVX?iF$(#$N){0c3pqM^OpFdfMYc*+2-2b*fee<0NWq~S`; zD~>Of-d`qM8~KlkDpG<U>dRbGo1$ZLDk}J={Vx_iAN}h==kC*l3{Ye5XuT61{d5q~ zoQd68QtvTRQGcGYEUk7;=a?7BNa<u9hKggr?6@>^Vt&3wcTlv4n8=>VHm?4V&W9D; zWkFDdHh(<(w2YbIV(b_bpVHpdA8%snXKN3wX~@!&r7O_*9WGk9NIrmMC$RWM%sp8_ zn(KGdGXitClW==D1MKvOHzrTp^W?tT^=zTuilhkpDsY1ft*Y{8yRyE$khdW+m*5TX zDk5C?C9&soH@{v>Q^MZ=nn$gBknx86*8ix7^`ZX9iD{}KV!fj&S3r<Ev;exlzc1*q zGeH$|^u<n&3J5*?@&coqBqZ^Jr!PB2{PxeFpPuWV!N3BdAhTJ1mc}@nr1PocR`Aq^ z+J*F(1-K{#B{jk**?6e3$zyE%$ZPT8kNxP{hf{@?i$x>;LN_0#R9?q?iXAk(O~rIi z1tgw{nMiNMw(|m*+sEbo&o@&5Yaz7*K=P{DIry!=g!f>&#8%S4M?7ccVJi=RG>iYu zz`(SmQQ|rnCI^l?A-+bPnQy<%0&nAj<NcREu}f^SD(bhd&ZO0GF!lckkB5~J=|bEY zSXwfxwH$RS|6KU5Ly!;mBM}?5t*9H_$jL)dz>^Dl)${T1)eW}$2mN|9Ci%3b6Bmt? zS@%JxIJtrO;_!H*EUl$6A@jNxi($U53*J%jSRMX|KM2_hzx`~*vtnj(eN6iB$yZOw zD25Y=tNg&mP&3Q#29RE1bVPy=!|XI=+r!iHrJtlv$}5QD_k+Jd_@{t(J|8893>Ju4 zw@DU{t$%%sdl6h_)Ptf#Xn~E>Ua>n07a!gPB@_j85Nsf>@d16-Dr+*y`?fR|a_nET z=pO`Svwj+*QdB0wu(98yRd-Qw78X}Hx>BhxH!An05+bOpTyxI4kbHm(_@W!qubwnr zP@y8zVB;cL(j&8VFY#>uP?XxV^62<@gqO(M(kFhNyZzL4y?j5ie{+UmF0iJk`0xRJ zb3<zTMIC#r%`7rzeuV$<c|3NiBdX#mclAf=Zj;<k8OQ$Ow5(m){8zW*{ki>fr&dR7 z#eTJRf!GhdJdjzrwFJeG5BmtdI3Lyd?(|>}^OTYP<GCdl0Vk1@bEE7(>Hqi}#D+?v z0}rTY$mM<E*!wZOldjEjcx20$6Q%9<jd3{kD5B}wRh!XR_>XQQQX_gz)a$NU;;zYU zt3*zZsb+xSqtFs`C%>o-wOJcV=x~ZPOedRN_-;9^c8VUV&YIif_DR2^F5iRep2Og0 zMt!)U>FN?x+T9Q2Qp%qNff&~Qu<Z3qxaZMLYiy@C&HiJ!XNK~TQy4cytPwYRs1_C1 zzXk=>w`;}M9#$+Rmi^UDqhU!^aR2a(`Vr!^j}(a9`a{ZYEw&yBH6~9y_C1D*4>&<i zH%^GK#xb|p<xdlK_VXLqIAX}K#42UuAsmMd3-h{}Ygt(n$bG?bQbAa;n=Yirn%05T zjqCn7Y|MYNHUk>)Or)H^zW^(YlKBu`hCpa@QrE5)5Y{}9#RCDX=>`rN)O`%>mUt)~ zAu>Z#v*ErC9obZq;RTbsH;D;ZDr4C*TWa=hJ81%F51KaFnLaI%)fAiDjjiiODR%7E z04qm=41-y8XPfVXaGN|Uz*4_}GG8P=S4TUWH|IQddR@?CLK|J#63DdEU>$^ob(fy* z*;SSWF>VT!g!G<?E>vlD7go*n^8eyV$u|_H#M`2!MdH)R>G7A*V*T<rGhbB2I4G2m zNHt5XG?NNM$hDpFG_E_0e~y2)PDAhWL1%Px{T*?xKI{&j69Hk42o58S-o!oswPE(# zW*eUy1z{+reRqb@%b_&q+WHgI>l)2M8S9ZUZjRQf8`4lk;WtLtWXli|XWtx;mB5ak z%^RJ=Y&upyQRdL5iXV3Dc}jF2NpmR7rua{nN#70@-UJ`<d&0?wF2|0w+i5O=vb4|b z*2YAEJF3;%%ML2Yki!69K<cx9)@L@sR{kdyuNYs;Y${*#+m+{+L{}6Rt(OahBBGgY zsR(-g&ui?m1*!+15hZM=>4bhhsiXmeyyIn;>s9!_5&cTtgH53Iqecl$o6e&FA%;dU zj(dwEoZz>5c?UmU(h<0P&eA4JLaksENEH}UVztPJKemnU4d~x`oJmG$kz0HWq91t2 zulTHg(K34;6M+cCGm2q0lPx_r+U>>be;Hud!WpYNz<1(#J}F1#)zp@yF;aG14KyAo z0$%-8HUwQAle*z${-wD3%t?m+o>z24yr@`Juc@A!x-+A28pH^8?6$Bu^FoA?4rV!S z?x5sDd4y_!S^csgOnHBJYK)}}J-oQQmunf5(gElEOj~82b27qPn<({sE1I^tREe-o z$~0M_m%H>8waJoon>JZf^9@&qZ&9_&DC=(0R@C_jAt(Mn>8}Fb;4R*`PCos1p`wvG zU>=U}KeNDy#d|b%A}sVCQZC6Wuj&#e6tSpW&}dVkrxO|q9IJLX&v<O|+eMd6U)(7> z&Yxh?duVu;J8r2(fD*`Yb$$;6T}WUUmnH}s$pj#>EZ!OQPQhs(15|xTJM{)=v@nt* zoZ0(j^-B}$C(x0G4W9msvigS9shxm)f?q!HcX9%c9QAM}TJr-Z3kk-&ks!)bMOLUk zLUO#bul;)UF-KqaAkBI&QPvSNb7eBK-pd!`ZU4azo^!I0<yVL<iTe>;P#pZ7z`Oxw z&|E?%3T$DXW;`@RnOVt`Qf;!;cC?RQ9n{GE(q0KF<Hazw-~JhJcw;PNb=UCfuYmTF zTbAZ_jXBB*0iqQatBIqTO|`e)X!eqd83IJQey8;21aR8Skp`K`3f&|TyNvNR3ti_w zmb28XZjrlFg<XE|+lnj6vvm|Mtt*Cbtj*Iy!)h$-UnMJKyU#l}LEO89BQ>P|C~$Mm z>8-5F-1Z&M=`M{Nw=>OME`K_WOm|VbP6tZRKE4gW?UN3qRmy#F_E)+&OB`Pr48(bn z#b&7oHdQ=@DLuG=&KQ5R@_rlq1gN}1eZQBe4+>}9y)aA@ne&P{G`}=1qa15|yQ)5D zm_f>ctPnl^mnT4t#LJ2hy$&T4Ip}iE6SqczxVNj3=6f>OtwQICk4m3fTo<G*JK(ER zR$)eY*G$6qbZR&t2s7@^t_=%kY+G7K^DqOG_7Yi|YCHXb_nw{eUjnP{KYY)+&<=0p zOX^9aq<PSzdAX6p739h9=q0?PH$P`ZOip^zf{ew^7e<ZcVB}0XbGkGSTgnE~uNyT= zz|8NeTo=q5*J$!04&uaDkhpf>GfK_u`&par#DVbIBIe;-Kbq{O?p)_9u@*YR7I<^< z6mPgbcW0oChrs)J3*_RV+xGNOmfh2G<RF_~m94Q4m4xzo9>`xx2Dew^-|UXRT-^2C zM4AsfN~Du+c@#U8zZ2b=NF?0ISH{7a*ADr|pL&-KKqtM)FlkkCsJ~g35Qt!(R`3tx z=s?0;Bh7+{AI15+xm!1|&q1&p?LGV@r_g_OAtj}EZ(MOuo(ArUO79%ljP*$l4v2Ky zju%*3mS*pJSlHwR2K0tn)Bs8Ek8)IkPTs3+==E9j!FOh#6aM(y5j_njCaVVgX)XH{ zP^|KN@ChH%zdQ5oz1<|j*w<j^+6CR5zi>^o+rB=tGQ#6mxN^zjagf8}vopEpr+Yiq z<^(-b);_VfcB>Jh2hMquTWS>GG0I;QjOTv@Y;dmu$*+5;YNgNRZ39(+jn^l$8_YlX zX8MihP)BLYtL&<_X5yr`Cwpp;;>p9><pXd?yr|w>`3WBY4J?U#6vd^NIqrAq+dfkA zaki|-r%Sy<n=tU$zB_ZTo9=7TO!8A#3|2?vKl)k4Bj(r^w>j{eH~2e_0(+1naJP(a z8}g0zWxoO*Xt+eQ3E@NHPyX6FD;|x8-{IIKo;$r%{|hi7btrql5&1PhQ!cWeM1dpt z6%Tc;t2$kH2`}w}dL+jBR+N?_JdJN%oE~f&9?u7lym8_a__pMQt|XEWtyjE^2B?dR zLR(AbrZ6|^@dmFi*-?d(wVm50yV9s*_+X!Hcmdb{c>$>34--W%wC-+b%$r9sjwF28 zv}zR(?oFguX%@#%G2qhR<TKsUCN2K|MjB;+UP}pGIc-go)ai8;4G1x`z%T96+b`pp z8#w-Ya<IoS^m3slfWm78J4DiRdD!kKT+INEBlxZ0kM3j|8|u_xigC>UiFV;V-NpX1 zW0Z@Y;m9F&K`5veoI~VEfE`)H^yt`4eX=K&hd#8$<8qM^ADuKXsqscSUb$3F2lLlL zLQj$<>E`0<jCwa=)OVG4h<s{-*rmjfQ?dI=Flp4q2|_KG8XCZ3lch+jHk?Nh7_cG# ztGqlMLLCTvD0o!f`K7MBoa(1)R(<EDaNrG_#CF1}Fqe7W5H~o1H0Y?G&t)X`!NbGh zI3JMO+(j@Q3^Kyyl{@6KR08~Gg=vmDdho#ShwsRER^<>cpou%^BgL|-T<nnn1%`=5 z>!DaGz-Sgee)-5V_vbM_BRO2<zO5nQHv19Xw-8s}g&<>l%ryV01Z<>gbZB?AS24vN z;WT*pyw=OBYtGACTtr)HrHg1d<fE$UsNiR<x~nUn=#^L<ZLQnf=_(fLhu1GA@X_Dr zDW;#s<wvEU$8t67ke?9yN-d(Cg`Cm9lYdW4ytv#wy*kD|Rd!9P857NkXf2cC+GrxC zCa5uyF`xDw(6<b&fDsXa$PB+WyN=$u`Gvdf#+XM?hnqt5w{bc0CA>C;li|wG7Xs+L zk2G6ZR{uHOD)b%_r@c`#EJbn9cCiRrn6bpYMVs9E=}lzLG#rfgDtJ64{+L5qaAp!y z4&BY0ZW6C08Of5fX_{74s7A385rc+L3}#8w9Hz?|kKGabYxYlUl13OEE7Ar(f%2+5 zkh7ch>LzA1*ZD71JPzGyr9VLJ`1QraAoXe&FqoKA8~=W^(-2#YagiVRNcxdXGIeEa z-7OzMZ=xROrZF&x_vSB(5G-}cOUpTw`+`^y=1&2QS?8NEc@F*JVY4>XZ%R;HVxq?> zRc5Ezjl7Q;+^ngfVWFiSuoYI&?{so#F(k$4k9#?KmEijl{dF@XXMg+gKx42dtkFk6 z5$ZtXK*}UBPTmgKNT~-`3U>06ASF%26O3Z3O#+th5Ff1J_}<xQiPKK0L*0QBNfUCo z!Sx4<gJa5hpZfk|<fB#i6B1o><E6Yi1vZ4LYRor^0QeNC`JMoo9>uils1DIh^Gkl# zN)q$)lI*EcaTT?<<#C2BuNCSU@-r?9>o!F^Md^mlZ2u-PM2u7ie)NaHb5gFoLtX67 z+J=5Wzg}m3DM(Det_U9NeuZs2o4|XMmdAtUw02SbiR_l0h*ox<j%~nusyk8Dx^roO zA0RgUegn=oc0P6_8a#ij{<77zF(}kx<7!(6a=+gG>ug8eFcJU@*+tPclU`z*WlJa* zomcaTSoT;WBKZIwZ>PT2nC7bF_*g%DGO+BoEQ#_@L08oEzopuz4+8%OHcWf%IvUat zvJXt{zw1bec=WcYXy88;2@5wlIMwmR&ydmB0;s0VVuAfbP~1XtKZL823(sU_efy{l zmlPEsdbvc$+5%%bLk>u#b9gEWqgF!iL0*!$nNG;3{fbW?I^)EMBLXI-xB;=(U&1pL zXf4Zb6xS8ON@P-s8Bb}I1G|vi(Rbiym-#jFiwMWf!qWgMXxaZs`$734Zka@ShP*M% z;s?eQuikqc?nE<t$r!)a!{PakKx6X2o7FtO?#te9k5|&#ub%GJ&p9s6oTum*2YbO! za#*2aUI&Iu!j|uyBd9HQJ@x(-oy3oL!Xil<T{NCR_jU%O<;J}CHLb});?D>uz`U}x zoV6c%qlGyE8TA6pPKNRN039MAWKJI~B$6-D@syys*1z~jQQ8xZ4j1Ua8?2Xu$~}Ya zsBf4kOC7fQd}DN9dr6C+s6(4I7O?Moo^HKJTRD{rATrjyesh+a<QYI?ShduyxWBcc zE?0j~N_qgj)h!@zU}tr9qKIgvd=OEmy=y3LrlTuv_5xf&`ltd-mCQqLy>A~{S8kVI z6JtU9<{&sf%Rr~A*&@DsaMP-ry%5>$b>ZVBm`nA8Mqf($^FjYpbogapupD~;`9#^& zCO84sHcwS7BgP8HmCl(C8Xzwus~!tzBl>|9f5H*Ui}Qs9r2f?=n4;+A^St@`P9^dA zGersC5ebt*m_cFtBe3Bfz$n!EpEF7r(tGe4KTkx4IWuEgi}g+*ISfWem$D;HHbBhh z*D!cD&2U0(-zLRH%=7Z@O&a|u2{G-p@|LZWmpj~V^2W*rUO%K;-mNq*r_~o<!DR~N z4}mJe<3A236ENpD-wUx6j><74o%IKC5Lm<<<k5F^BY4%xiM@-M!8t;;VA|9M^T2g9 z>{CFp65)}axmR<cU^)r7=-kU=V{&ys=JydNg@oFin}2VTv<ODD+O5q7TC7Z_I_~w{ zefUhtUyabD*_Zdc0ox{qv%MFfLkIriJ2AbmoyR^ObokcZdY`DC(a$-=32IxDq6w^z zksFxq3yoKFQ~v0ZHf!|K#3jq-I{uspPbjgUF_-LRHpO8nh!^YJNQ`Q%k&gFBxf;74 zgi-uUzSjLEII@3*yrDv2Ux$>{`twQXKM6B5dJLo(Z6X77>b_+#%G-$^4lAAfmJ~ZT zEI-B40K*Uv&wNB0MFY^XrEvf2WYk-n#b26!vpU8Gir`tU!?Th_HJ9#3L#HBI;)2fr zdcp~s8J)j^?{Lt(Dg6m@Ag5Y$C_C$W6%6TfB48k+UUvo*(Q{0u3XlUPyT%zXv*h>m z2{i0BHon`(2mI*Ew>>km^C>-~pS<}Q6kiU9oEfS3yl4?S(4AEZY?xkT5ii>xZ>&6O zQ#H!nQDgC<RW=?YcOE#?vErwqeTlH|mtc&ne#AxVqghs=k@c1+REG#IE5eCpS{fO+ zBjqic;+y&(BV0s4%>i`n<qTRnS)tEb>8a+FxI>G8%7x5)6?z=oVg`6&(gs5d-7X|L z*``>6`R{`~S<|0pg%$~>x5y=zi!nf{JClU6cDExulZkLD`|*LB^0f@!<8?jaYpZx? z=9-}segxt1bBsBZp7GCuK~52?Q4FtsG_UIYHRDt#l~561XCwx062Rc;Dg<`;ca4V~ zr&-?8J1DZQMP<P}-Q}N;pwnV2RkM`iyhJ1Z9K&aPcj4l44e<8B3cUop9RVG#Ub}4! z-u<BIGg|I`bC)fE*3CPJl?x-V!zX}3e>j3Gg9l|~|5UhhLqk29;cILxK^HbinbZ{a zSupM~u`(3!(n*d4d$g_f-~6%etlb+>ASqi9<;GvZQbEw<42Zaq$OXyY_p55dcinX3 zJ9eqN^GO{2H?2(Pz8PqDnzjm39QWfF7Cf(VRupKdkb8|?MW=8D=(lagEX0MzAG$o6 zaFo2i)4)Vss?*S!6zEUfEHxnby`DI3-d0|;#8qd(T6fYNkgU@EJL6jm554YGJ+;1+ zetyk7vHJ5*OGvS!_BfA9@I@wLL6N_t;Z6T48va(gGw_3?pr^9q7*49WWu-{8bUBfc zJz_*v60j)d+1=Pya>sCChTza|i_v-33PeDfdmCpg^dsOwVjd!emxEvAjTs`&J@$U$ zS$oCx%d69`KnL^@F{2>>Z_zQX_#!gOU!II$WgHO|O0JZD!tp3C)a)AUhYl4(xj{zH zGGAGth=di~Ud1dmgOoRob>bTMaw?%0Dr>`8YLHxUMLXJ(5i{AVOWm=w^zIC3b+hP; zfM+R9*4K<j^Sh|hM7Z_!)OOMVLC;~(kEQr^l8tOu4HENG5<MSy38P#bwz@`{Y>@oK z`ZEg!)PB32;onMq@tLx_^|zdU)8MTt_E>iBjgM3@^b|lWLjNA^uTq`I(o<OoyiD!d zbW~U5uaoyP?t3-TcG+R7{*5Cw7!hZXNicaCRO`Rn*pTXVUjGoTo=hopv&zbaf+->^ zCjxX+tpKT*N}5MMJuEN3*JKd&i2CMx#-I6Z3cof7A)2!nVE$#A-{r{FRDyhL-KHvj z+|qfQwA5QIPGG2IG2qN2BWV+Z>zqN!BpTV;kr){*d6|L7V7f;0V6~dCal$N_xvFP4 z!AP@@!{0g0pHuFRr)Vox@H<LFfdzJ94n=8`G}iUo-%s~<i|L+hT#xOLTC4<!ngy1m z;ue>VUwIIn4ekPe!Q)5%HQwv9i0Hb&n)5Z()$65v032IdR6Mblv*oa5KSZ8uPQE3| zsv$8Gn;g55{AF56GaQt2E;)V2*_Km=)bzWdMgqkEU5flC3K|0raPrb@AVsVFBwLC} zq1>z&RhaVnRBiin0`<~^Ux#*BM~5YH8Uq4dJKTiV54h4C3c*Lwp&z!-=TaDTE-vcr z6g!tXI5dBbd6=EX#?0CNu1F&XE*Y~20QVJeW}QfJ_2uGWW>k(vSuY-^R9S+;lP13i zc!GFmUJaZLi0e$rEa+Vi1zR~?%n(m$0Gp2X7URvtzT>DJ(H80;+x~m3A{nGW2#$>7 zWT>QovX_DBYucH`W6uov8t5$PV@rp<9;yk)aj}M$JJ9paBv@AG_A|%a*HadX!=)+| zPEtniIEK|p1j|&29;uuD?Ms?6*&%w}iPZ33=mXDf!k>i47iArPj7Zpj!cjtUXMLs@ z)0m72`E#pK@0Ib#?BBBsKi=^pD8>6ckSvq2Akncvz!3$LsIAta?5~ECb_t>z=o}E2 zERF6UT8WcBAVc-wR4h;md@$?<fwC?tm@N?&x$eF94Lqpl8)ej@y9@!DtnTXG-t9yR z1ym9!Zy6p=6aH>b_w<*VBy0r9#o}N5weUo&Y<&CGawL=VnApM!P-q=`M6_D&zGo_1 zUS3{>Gq?y^k527GdP?2_)!|%ZXX1`tOoIKIC>XrTg3DPEU`17bQE8VZLNt8Ih>&M? zs!RTXS0oS3C0$ImAc<{OM0d5t?(()|B|(5k!$8#2s@<&(+fTc1_NX1Y>q{)f4*1hP zZ#Nf>&MZu3={0ffx)Sd+qZH=878le&9O~vXE8<=RL9x_+U3y6uF>I+>r=YM>6N4I- zfa{HD!FCUNs8im>sUP*)uq*STy;HEzS$BSmM#Sb+dDuEzx$~N~Q*cCPZp^sD!h+!H zj9gct3M1G~l^EnQRJ>C4VCf#4+i~(ch=#UIRgC-)OAnVg>ICy3zWWof2^t#jHEYB5 zA8($oHS}k{y$mgs)AGcPA?;1p+o?%K2L@KZTC`BC*Xl0fD-G+4%Sbt^88z0~qHg)6 zfLh+}lM(z#Ff~<g39kmU2(5xCV74e~JVcoh*-$>;$q#yTRv9Y?7vlaa#}^-{GWC#K z{2Ihf(CbsPV~K7$RHCDk7N)2cC9?MLX#B|)Drl^qPC8%<e#l!KY?32ew_I6gw4<ia zT)W;VV1SR7sUw8rZG&D!A8G_Sjs0N1O$jmw;oe?PJ(4?<vuSya0Rb;&kL>!uc9VHM zhkP?S_X1v-*x8N|M7_3jn#8sW+`I*-+X1s_kh7!P^PQP2Td7D@J2twIl(^3U-b8Hi zdSV_gr44Mb9h(N**-m)2a{s_s1sdNqp6EKUDT)@F6pi!$Pm>auY_nMSo>3?Z`H5Y< z+HbhHH2)Rior)x1qPblh6x7CuJozDJCm0^bdV#ZIfP|?XdGMO<hME-tcn%Hz(!MBH zd(6srXg;Y>uPv9)T=tP63@++_e3|q#3sz@h$bOIqyXoCIvhYC))UXA~B-0ov1FD}M z7gvYACBh5Y?@{mlXLMN-f{-!;Frv;MwCv|Y<bU-DCcbmL;*)V8)-AtoFsG^Cq26~2 zCbL-*1?+=^@#9Y6w)D;-xw?Ko<BBu`05uCQl8KBLb$PIFMWD-9U$~0eX+a%l6&@O_ z8t3jGWk@9NGY$N^&>!I~?P!ym5%hEIk7$Ky8<yn!2<|Mx^Df@bGs0)jY)m<<I|w;6 ze^HBvzY}PHorfbgy9=k}XsOs&rGB@1`HL1lG8SU}d+GF0cbF8oB)i+-Pi=hm^vzv| z{*BQG{mlKTkl+G5-^e=qlr<CI5=;p{4X72~$sx>2toPqHyB5m&ipL-Or5T_&y)?%Q zr%)KfqC0%ZA#XDf?~K^1SO|-CkjN^`wMK2W-i!(LS)`eptV8BVT{a!IHuv3f!k`_S zCinB)s;xSa_hsM7tH0Q-u1IMM*j@X2<X?Vkk&<+pv5RZ=3jm}A?u`v5#KTS^SRI4@ z7Aol+;?@&emIssq2~ZWf<HhrRTzQmBR~zDO9`tvC<1VjQ2L=p>3j!UqB^Md*-r+x! z%zIe`i$)a$9~fQ6QUM_?P=3(u&*)dX#CbPRfa4z1KS8y@U^YD={+}|3p$vcA&d`$F z^@qB-_50QBn?sD8K={s1hVy`qQv_nxw{bU0tur-twP16rU4vf25$}+p&*2kn4!9HZ z+aZjFyYpgWiAa3-R#8;?SoN?O;5qx*>X#i0bu?pTgw-aTK$)Z6Vc@l30gQ4z^NsF8 zt$<d3edfOROf&r_Oe3ijzoO!WzI6x#lhHUyN(F=N*es=rL1rm!4N<~KPBk$5rH!p6 z^r12k-fBr-w!0HHB6neh83rkJ4L?SnmU_$iLD->?#lor+ZG(<KE@z{EjUrz2%zjqz zdhoEyLXBht?_$PVUgCqWvr_3k7c%L!1D@#J$N-8K>ZKyu8MBuUtYSOyrDP&@$z_dg z5DV#a_|)G;_>%j$s`J`6C11IkGU?0Nw=b>2|Hb>6R3CJ{yaL3eTMdsAfPF?{@b4Rj z1%upsA5Gwr3WS6P%aBFSMcoy+8G)x_XYjewKR*W6<0dtHWzQGRgZ!#T`98&5u@B*A zCX0;}m%$sOM8GIMJ-0?sUdyIlo7q!9Q$mT?xb{Ds5b@Lg!4UgCz)mx3Qt()D=A)OU z48*`f=>2r$82@BA!^R785DH+eg<seQ=O`^s23bGA&;^z3J2m>Wl)H6XBQMX(1a<EL zei7%tTF~3G!V{pLh1DuGm1CenN)|vmVRB;n>+8wbyEZ;L!z{Pf=lZH7I$v(XsHGDW z+To4m<)O3c`buQ87xI02FYOS3h8DWzb><q;8>=H(eeD-5>!Sn}yD|ph8=&Yb_z`f| zFICY}cd!u~CyfR4Dl-2WF!;CTe<1DN>$j}_H{s`l_Y`|$u4EJc!ofnu_d8IeJ@0mx z-=WT*59m^-qIx5aYlpY{MtAugA}o#5YT2pDr8Az0QEY$dMvY08#1&otd#P8WLj(x7 zmvcLaG({AH?j(6hmoyX=K>tgjojmfxLVsBYbu-$?ifHlXVM`{#TUDyYz<`VWmrt8V zcsa6&|8<|2174s$2Q!t=1V-~#5*+cOt3`3VJB$CMwde4lUV7MktPL2~H(nLjo}<S@ zlXDJ&TJ@Ii*bSpQ4*zuXomzGjn|n5TS7p8mdjRGJ?&2`4=iM}N55vBxU1KP{Z4SDP ze?1HCot@NrktQN|DzY59IWPZO#=zS58PNCUn{T#A{$vmXIMv?#yzh<pnXwy!K8V(J z>)%rBXtfY+j6lF}^Lq1(KYL;mtL@yR@x@mSG~`AnF;Xx<wFN|yue8rBO6ok=8iu#q zv@(D;n%&OhW+*?1gQc0#CRVph90P>Q;isu}1)2T$x3NUoLfFs>q)hmYmzL+|Atu*S zvtc(fv>oEm>!BupUKW7&3kpG&`ZjB1*>FOF(ReDkelRrtq<=h}X-6@4PsUhkuE!B^ z5ghFWa%P0$c;!UiTOjsg{<Gwi%{4uM2Mk-F=eq-!I0ggHYx2VJcbEc0FHQhVd=m<+ z{bTJkSR{_^6}$8~6sAb%>LROrm5Y;OyG``P1QVrJ^PjI-gbdaMqB0|gdo&ltsA$1) z4&qvs3?1ukN&k6stls-9lJY>`&$*wZKMSx%#0f;?wTvK5*Q@E#+s{e6^)%x{-l;BN z=V<Tqck=KJ6qJt{5yx?f*c069o;-v8D`5TXVa1bO+g*TOIs`q~skaV$9j)EHlp8=~ zo6ky}SM~4>Dy_f+V{YvPCab@rGF2Xg8a}tJU&S8O3D)}w*^9Ok*iKnx8O<z`l;RD? zO*KT?4GoGaUX0NF6y&mI%O2_NOPhD8(bT_2j7?fdn)@vpy=Lr2^<H#KO1IT5tr`6N zOQLjjDK>3uPBeesv!DM{i=ybhrEEobAn!p))`1a0itF9sfZHe2F1ydBUH6<NF8d|X zYgbR5@x>^X>;PsS_xIW#2M7QRrX%8;kcS=V(d}Hg+?4YfD<EdyDX>NU0xyF5JpG`P zD9J+?IFC#8i04sU?=?7SeCZy`0lRxfgKd4VYXTikJD<A9F1E>PU~RGdH)4DLvMIOd zgkHeux}wvJViyA&Y?&Z|-5CGTT|ak~O27clCeuUNp>T)&XbVcs7EJ(iV9VU}7oT@2 z<hc&jM*_`0&xNJY2*#oL@7xsCB;bGdZH3eI2^5k;LzVBR$A{M3XL?JtfaNU>Z>k-A znN~oCQ{kKl1^_klJTh9Io_qP$ZdGz)w29CKR}+qzQg~+;TMG>E@M1yru9I8TOOzM4 zNSx5C`<0C&E4$O4{uZL^if76W)&#$bAo1}Z2h!Ia?Dir(bP*qvZ(4mCqZ&(dDzg`B zS0m!Czb`{dM#j>g%en{cf=09pK5^`~ya|9ie#2CrCoqqA=Rmg#{Cy>o<z;q5<dQoH z^6bPKu@nl32x~F3zLNb~!)vkII}T3)&qs83O^x2NffTKAbT5C4L264Ym2rA6sZYv= zsTJyXSaE#>HqXNh0TT0eAVnjn8#O`#{`_vcJM(WH!i31*>zeINB$$V}V$xi|w=8Hp z$Ge@fv>h6^AgfKw%kbm)trM{_Z20>u{)l7IvB7{H$D8OIUV!C2xaJdRSvWV}j8!?h zn%+==JOG8s@tFU6tUsrRXq&t!heuyG{4wIWvXiXsi1i<shMg|_mex1k_0ztrgYSwx z!MuDtPDSBfp-StjFZm2_Ff_LT<TVJ@J9AU#u|f}zh++~wll~`lQ7w{Bkg&Zms7uz< z7vw@A?(WpI6t-i)iD7~~yz%PgjMTdINTbJZ#<ZZHdgd;Qa{ZkGH`mWb)_OV1m;f35 z0tD0QV*2&U7zsOw&;mI2`gL1VfQr~DZu#w==HSl_TD-9E_ts3T(a9c@zKdwS%jzF) z>5qUTu`~-EYBdoM|N99ExO`RtA6E=mVN*p9lt2uWo<qL6NZ2nD%%+PP{7=-r=#ib> zlvyyS9?|eS5i;RVTk!+t+&#&irR9!;-EAY?%zvOJ15<;^Y>8P8j+6V4tjJuud3P=d zoG~w@e+xPUEu{=MP0$#f>-9BWZ1ZcAlq@RVunViH`{4lQHf*5Vf=KZH#AGc9<Ps&P z`bO~5sZp(J$t}(-A3+UWxY#<Q>3EuENZ2AC8wzaOCla*By!xfZiDC!%yjBvPKi*k& z-a=`9j>f*R=AP`l$ZcUf2=M2hn40K{kGq!Hs~NATvi|!5Gbii>HY#DI&Z=Tz*(Jzi zEtdCEeO&8a-&wtplCH!2#)Cij?iril_u=4S+1b|^CeAf-t^A^z2PhBEj<nk&vh23; zMYKTHN0POkq22ttQVNu(M~9BF_Z^VjK3KxpUhUbxN0o7|%<!uV(v{pEfo@5p!`)lg zX)<fu-{A;^@2|?zPz_Z!trvCb!BW3$n25SBKr}B(b3><cv#FeJ=M@2>ROq!NwXQ#3 z9!#3`bRhL0e=g0e4<>^`;lmhILSWN%0Oqnt?&$n_So$ywFK)G52&*DYjbZLXNs}}e z8J1V$K2M$<m>Iao1~o?!f4Q6&$l&omNvmy#Z_0eBiQI^q7S&vIb4=xJ-A{+T**)oF zr%;Vts|Np>-4ts+a63;MY{_IYBlawRi94;76zOQNOpW7==5K6&D7qfCSn(8~cmE3s zy6vxCLOHY6M^c8Uu}p$-!V|q^$gI<q0TJP6$#TkB*Lkbx+Q2e)C_4MugCE3?eppOZ z?88Fo)_$BPfenXA7#1{fc553qr6t|uUHF+OR+?b}uPy+p$=YkS+S;jA6A*Ib1-8)m zS&qv6NBMK#SAX_I=wqg57PnQBTo!laW?dA|oJzXCd$9{2d6XTx8riL^KBBfSi2R_u z!8afkA3J4xzM3KISDtCgI@kTmx(z6!TcRA55?w*;=_9Es`|x1j@XjdqP7wW9kLn4v zoxVZ)HcEB7y6s2%NX|9HHJ~aF##pC0^_|^&`nS_m{r{|CwYPLpaeD@2+<Qs)E`{?- zEG-}Y|1hJ-($Od{PW(*uDgMuXYRz$nO8weA80noyA2Jc5)#|`^$A+Mj7wvfOc>Ej& z*DUUnnS0Ux)J|l-o#bB)VYUoWxt14qdykim507`n^Z0#foDJ4k>)dd+aT0^~usYxv zmKv~*-;V8|#blW2m!4&CD?XEnPJ+xvov?k_t&WN8gQ{zTigqd6v0ekxSDz9c|I4y6 zxwDY5nXZM#k2^gG9lk12#HagT6R`j!Js;QX0<J%-rMrCyf(%Q-v48)c46d3NU|IeJ zV~LT$#BoV>HXmKQ$sPJ;*;vqpCMx!kgx>?vfZ=;wX*%Mat0`R4;viDzs#dx;i3*KG zY<Pfjhj7^b*1W5=94b-MZy>%REm;H5#-}JY8|a{GGcAyi*onZZVcL$w2G~-AULJ~G zIsi+77h8cZZZXZUw$;ULJu^8IpqW_|&6Caxv$>!9W@W3U?=Op}CGfTuh#}WW?G#Vd z|9Js&BiVh)>U(5Ob}MDJJRv=@<kyA6=&#}F3jQv7$fFaPg#O}4-S&$fTGDt#ZH`5> zE^{p!MjZ6rjyn7HsEe}KdrR3Ev7V4*OJGO=-HR5`Xh+_lC0+#QTVJ6tfq0kvRv-lf zZ_t0ECBEgFq-GJzE@AdhYpaZv2p>%yV$gljsj{2e+2eK?=H?Yh6T>EUzh6DrjlGpw zyH>pWoAf@#Qf2^kneG_n2<TdBBCUCO^vkZPt|Cz_<<i~$A}M?|yQI6jS65VAc$Ggy zmELe;YBr8q7d1P!xlgev3qZb~WgdnOuy0H^!d~e9Md=SM$4mj!+NtGySDnz~L_fO? z-iXjYQ}8OHtTFZ_cf=xv1wC;0(A^`kp$-it;VkBeQXN%MikDzEa0#_3Mr;&4oTpgi z&s=-6@*cC*u+iD(mfA@rJ`X6)cK!OUVp@nUg#D!7u0!5LNx`nyzrUpm5A+-RIMFFX zqW@A^zH;MTfimYR)Pu_*^i+b#Vc80!G|5!<?BsO`t-}L7KBwG`Grb@kI3jUakMlNJ zrsiZbeEP<2L;AOL-roHtf6uyU1#&gy`BSw#yx;Fn#ZfeVXo^^|^6@|KD~OdwrFZA4 zZnKZx;VAvN5Uj^7PVq9_MPx<|HpdV(O2BuYEaLZUVjs=!^G8HaQJTqDJ{)ZOEh4K` zBd?>Y<R(AIutP_)ND$R{$(T&*yI-*iRmWz({#r4-^jlV(!Id<__d?Jse=fwq{4K9b zH;dP=Jrul<7bZ46frg20xU_p!LU1GgCcA$3aX7@UjjGVm*1`;NK3aQrXto8#UIS}s z=CtU<3WNf)w;^%L=&;8_2udBEf)2hzHdo33e(WK8<b*LOoTFw?L){}z?ZqGxc7#;+ zd+Ou0O2`D_^G9CIJ9wv!;h+WZ7p?1A#)d9)*!z90WOxpo*Z0LN(k6BPA?*Sw%Q)5B z42UpdgtSvuOX)2zv?ITS;(x>M#CG{)Xr|ZIgk@RxZ;FoXz*XO|ajw=J%J{^qyh7;} zLr?-PjxMMO@B`ub5QN&o6VEC>rTgl)75*hlx(;}5c`K*E>6^(Md^`tn@ii=7bf3}2 zul(lgSKL@*uz!Q)HA0FP-E`JL%_FlvF3WDbH4wFD;Q7P8mskLy;eqG8OGq?Ot3wmc zZ})x#52+Jr0Q9P#ry;9pp%2|?DIycA@w~FxFX<n_iXRIo2d~sgjfbZ|qzOWPHPU}F z&yAWT)17`537WE#Z5wKk#~+t{any9=J<Xcr4`3)HdO?)x0uJIiTukah>hgOk*3yGe zgwOG-J^a1;7;ow?fs$KO<_c-6;#Q`Ap=;Jg8f4)S78J^xJ71XH+#g7v;U^U@hwzt7 z$QU3Wpx;3WDU%ks*y=M$^!vzUsBSaYugl15VbVZ^rg?~{qmfdh{jBq+wG-(#<d!TQ zuF8IIdSF~Jrb3an)%{9>_SL@@nS(#6e1srG{Z=yx!J1#-5soRv!8xAc`(&5y2PbGi zBDQUpjP&`_nJ0DstA{H<qOb?}S}z)WzLYZtC`K6IKYCZN!iCp;GA@L?t8}vT&TR|Y z$$7F>u455@A0xD3mN~VdGW2jH5U^vnYq}wn7i9C;GpIPvm)^F3f_kBuK0=f-61CzK zJnTxmXxKt%+nn{erP)L9yQ3n8$-sK|AVfSXtH7|tY-Dn`c`7&Js9UKcT$5V;a{edx z#O&nV=&Y<#RfyQ80poCF@f3gfh?*?zIcwpF*dDgJtdJglc4e}1%K*O<xvx*E1GQyM zul>X~!Qs~08UD(ZCd$yOqaok2RI%WsGrn7=q6G-Q&U%fmtMsQDmc1~9WtvTyac&7Y zQQD%o;UCavG)@=Q1N@<^POUDX-|-R`yCFUTH~HUsWE1}IPpsaFbFjj$+?N#*Fn<c$ z`WvI=l!lFqp=+31(QeYy-P)g(oLL{8CGk2so6txT2xqe?t&^tq=F2>1UcVoTEG0UY zUT8Hf4k;5j_J{UG=~_^M%o40)i6x3<{RmGTxfqzwA782vCJ_AQx}d5rdz_({(0+T( z4qz0Z3X5E?<)Gq=4z%Oo=Pe#ui?oOiiRU}EvlvY+{PrV?vYCY}B}8|E`aZC>qh-P> z2WXrBBKoEbVE-|L8+Mxsx7+*RKVl}wA}z99)8i=P`1VvOc;x5L-0A1|yFrHQ)hctI z=REfnCi^*#@*e%8G9hm9QMzAa$d(Fwkm&^3_*_A%3gkVBBVNSk8OqQVuLE=#t}XWU z7p9Bdik=-DiN>2R++Q^*;8KO=SuDb_T@;@;@Fa4oGBZhaGUKhDbz5E&K~h)qwAM%F z09eJk_$e7ML#V!?_%*m!H-qMD9o_`ojeT!ZUa8!{;7`!$;$^eHQO(7ead+l7m&!8> z#|ARm93V+h@%2d=&+`L<Cjdzhyjc2wSCEKI5`|s;;>dKWdT)Vn2~iIHkF=Q#T{S_b zD7e7@o^<&gsBc^-M<FxduSA}p5%gep3>-zx@#k1$;WKmeq5c5gd}FXyyszS5wJ*)* zRF{J|gsJWXYAn@+oV4DKotIoOv`_6U+6LNfe70YT^^ZMdD3MderAqR=xqxxFh9pGJ z{uVkDx5=;k4_eml{fBg{nnd>jl9r0jTL9v)pyBrI@3vEh8>`cYud67BK-t8>m+)wp zc8K#Yo;yH@p9oG!fYDZt1e^-6=Z>UlMO;9HeJ^)9E{@m0{?c1Pj;~T{9fG-PUiKms zet@EA)<@><d_V$6d~-hKD$dpV??1D(j}M)K>HUQUuu_0E*f4wPn{u3(C(k0Bl|Ll) z@ZO12k=LLi+X)rO0iB#8AnhZ!W;+QX4F#SsKJp{X?7`Mn5by>aiN^B*teI1a_As|; zye-BQd%8V2cXGvx@Um_JvHh{+FJ2wC;EVg)Vz;z;3xf!#(FiqD^^8n|bgz)4a3sJU z9-#8?+2nN^{L-S)59RdR$%ck{im*|iNreIwaTPqpMew!O->bUuEL3FCa!4BP!&<{J zk|WLd!EfLIZ8A<x!ab>=v(NM^**9qAUCEsqMi~l|04)C&Z`$X9&yNe5O4(*xT1>SR zA7AZF3v}$B=zxn_9^hey3LmIez*W5+MNLwvWa`Lk`aFHlVjQygr0xAo$~$6)y0Xyo zii1Tpc7r-5d~f$%!=_DXvxuFb#^O@Lh}T2AfZHC5;2j|RNW9UmPX*m%L>;7ApG86? zS|=}ykqTD!^lSS6Xgcq3w*L43#|TP|YD>*fZE3CAO2ul?YEf<Ny(_lb1Vw4>wnnQ~ z?Jf3<S=6TXioGJlN<xg^>HG8hUjE4+a$Se(ocr9b`}w>dPqtm-1M0Jl%D`ZAkjwC3 zH5L<F`k`c@LPBAM@Zb(7^0`)J$2+as*{hqr2Q5p)Nfdc{i}lN!70p0C)>Gd|FtxJ# zi~n3Y${ZPJh-qW;IWa#gX2z_vc`G+%v4<yi$6YVVssz1ypSAP1ujR7*WH?37+Ue(L zAEbBQNGj7=Zb0?s=b=3xcuKX&XJh_Vl0zS|aj(U=C%a82nuVGeceYd_*16X66DD%~ zQFur9DDu}&=!u0jZp%c>sh*v{n(lYka+taH4!T>bi@;68^*4rI1;mmn@D1$Kbw97w zRs*U}*H(YcJXpIJbgx&zoX=k{I9ZSjLK!d4z?x)Ax(t|gYAThI^<pWqv$&X)-mu0^ z-1Y->^}m_-U}Au<?j^7iQ!BhL3*DWE{i*8^{n20ao+%g3>&S+7U9`RykTj+^JKoy; zl=K^8{RwrC+otk0ggPnY5MjX~EAC|2Rcf;n+V%29x{|BzC}ku@04sDXzwM_D@wArs zS99zk%z-%-hsPi8?Y^57{Ib99XRr(>&bdE4*~}u&JqwH)^h?%Aw}o+4Eer)9GlOz( zj(XhI&UkCUJ0Lz=2T|Dgl_t=Y=(}CBRX`^E?xSg;C;!#Hmukm%!r%xYIy!Za&NW19 zgAr5l?1R{^Lxo<j6Ez4`a2GPFM_DfIF32a}PhmUoKjv7FDH*pNXfM&h|2}obxIz#+ za$LciDdO)8@SWRBmQ{^6LMyETJxmG)xO$f?3wEZE5@aRdh2J>AH9<J~z4N;kKGb+? z*&8oHE#N=^Y4p{(dJcCA@&~~X=j*$E?{{F!A3b)MBs|5Dya{p?Pc(ozjP*U*t8%oV z8FK0O?hG#s|Fzn71+FFN2^ss3-x5?8D%~wDJ+pB?m^+Zf9r{9#(&Wl)>XzqU3Wi_K z>!<t*K>e;m-F&3A#$}gYy4h9)jSwRbS|{ldv+<{}SbD{qMBkKXCDQ3D-I-kb?7P!U z_sfyGYg1dsqc!wt{jk~Nl-@<r<Rx1K)|CnlEFs{6-)A0Vz9jHfX1|0`Exl)?AzwU) zw*1By2wJX?*IGugS4CC&TqR#6Q?Zj=n%ZAE^tpd^<1q8Fp&M7wwzP`#5!{i&lIHPe z%wGXB02q|uwbUN=A3V$QJ><>DuNWZD9vf+TM!LQ2E1GC?qQ7v~k+Q!9+Pgw>CDV1I z;_|t}M&B|U+7B?Yy-A`=^F)Z*oavpZ7JAuBIbAj(nA`-5!w2_%gp5^#gwiSfT@o~% z1PQlBNy;=u;r`f$&dU>$t6wU(1F6PcFVFEE;qqtZx29>am1Hr&e|Ty@L9Cmlj-hS? zF8OarCK(@>uR^fZEhsV?10DRU>7}o@Q2H3S?3j+dS-PO1ng3rkJLf6ZhdkKP^@MWA zfd_Vl0FmOsY(s;YYoCnEv$#nQulDLPD0mWiET?7IOX^^0yLiz;Z@y$*B3NFUl~s!n zeY1#2h+NWjMpD5VWEWFcYQxED=U8#KB<N~5p`-ja67{DV=+;a1soiyig2QzMLZ6nv zmf|?nUCxjgY|OjVa!jX2e#w23t86q0*9=L<Bq(I{YG*exY{Fv|0Cr9I?5;mK)aSL7 z|BfXjq^Pvvfx&%*LZ<aqd$E!O4Xcvot!B-~`{e%sJk1OTw(`X3?YW|`OQZSsJ1c$! zgX`e|=oY&W#4*g_jE@l-<0w{T@^v$6=!DU3bh^ijE8)(%dYb#Ybjk<*?1#ni67(y! zjBE`Jgu}>AiqQ>rgId|_LMl{zPrpZgaDF{~cWJ`A*_q=XMLe(=HP8XqKu}cIP%o#* zN^#|)So>FvLb{t|a$z>gG(D@^W9{N^JkJcNN%a-(6w;T_0m`|c_@c8K++Us|;sGf| zRckk2-c3>unc?j+zKA#5wMf^jf)Lx51W*)T$T)l=gr>@Kr9$13GK`=_$vV#r*0Kk^ z*u@hJ3^cDHB2P6<`tRRfz}CI{jxV<Co=(Tfs&K+%*B8kL%&%wG8>|rZnB`oaZ2MI? zq@i3=3`OC95zTNV&9m7`ZYOxKJ|7RJ@vR8T-RaI8N1U7w`^r;KWCYz)%A&exaHui( zBONMG@HO$NS+Y{k7hh@bZ7wU?9rTe|*vEj=_lspZLUi%!?}4h*-k`yB1Q~nlG2l`> zpI+?d3Q_%D#XP`?T(IFLB)6o~8E?MwC5@{dOzQnNpaB-XO=tnxW%Q+RqCf@{Ra{3c zYg|9|hezHzWSUuC`XCct*_!CUmAfP??@);18_VxcCvmAb(;3O|*qpU@vq06MpSUmt zX%(R?(xt44Pq#EyYOD4LwfCI#Ky$a3nR-jW_>!7hn-_T)SXXNG9`MV3hTWyV@l_3) z3<rzGG`VSaY3;-p{XVwtweS16!Jnny7eB}r{nz>^s{;H*H#_AAiN7y7;RJK%CO3o5 z{_bQF{mKGaPvrmr0@k|&C&oGT4wA-6AZ91=Q&A=wcy^0^vLNG>x|1Ir-Ct^|^&h_4 z6##${@%rcGRwkalUzNO}F68Jhgvay!v1@$ZskTzAh`Jp;oyvlgnf7rA4`p>TaM}uz z<3=f;cg-l1eQ+a8wH4(z-fFg<epo{XxCLkE6dxlznx>Igk!}MyE|b-A)3(Z!|2)TO ze!Ibrt}QoN4tL@9y!RQn<@{UcHR+{XPP>LIh`Uz!;4#FaJbqp;Fvk~Xy%<xXx`Kaf z158_nEEG5w_L4=RD7RxhkQGf(AN^(oV9!}aA+Vy8A>e<Guzou-$W)?4>7yHWIb@L@ z>S=pvzS{*4fn4CJfXIDpA6ab3kK<wI9O~P#q0ebaJMY>FdbLII!fELR{c?URO&UX{ zw#~fpiNw~JtQoV7eV&XgIH|aoNBie$b@RNgQoaap4?o#rI~b<qP?hGAj`_tS(2s3a z(UnDoQwFgoEeOwO-E8I*-8PRhdD=rjp4z3huOT<2g^(CZNw(!T_}XplhFW6;%QLJw zxI&|J>y19<HW<{MWot;Ag%57~&1&w1E_GZK`1zNO4wDbp-BgfM8lamh1--9U>=4n^ zL7~r1ZV{$=Xt7(6$<}GxbXwHuMD4FP%qiceL@6X-H`1}%5l+Asd=S@xQzM2#7E|?= z1)6J#;xLMr!c|z56JN0QNiguFcI?MRPf3K_o%oCFhUU2i?4Rfq^4p71r+48Xed_7% z0WewzLow6+lUKLB_&~!KYDWe8vg!17l6M2*yW(!sQDdjoZ&zwwYDci5%Yp5IlgEJo zxrK`&ZiJ*Dxc>EqjmipGfMx$*Z}Thcowxq|PAii5e7b6`G%#rv#*php$wlM`62Xeb z3Og@B^5?`dlC94@Nd99(TjK35PB|Y<YDgA)X;ZGlD{MuV)4!r*5ptVOu>8LTrv>^e zJ^G=0&vy29J!)b|JPtW)1wHhZEFt3TJepL{#3+);ODtTy>!sb`w+ZtwudS)Co6$BG zj`rL7wQ9YM7qCeh<N?9NVJ9J~c6cfi9vQxLc94Y4R2SN6HayI7U3mC1UHLfF14Y3v z5^9u+rVdi^(J^|LdZWJtMFBEZw_a&=0mi)#b!dnd#B&nvzx_k3(ut%s+8q0BuW#(2 zxjc)Iss1|6uU)1Jc?4QaGy$;G)8v!+S;+`zZa#phPtL^)`9J*XW>Ijlgn-SpFIEJE zRa_2N!VS`@_DT&NridX@6H8)MQ_Uq6_N4>4w@v-k?5gb{(iqu!7~ub#ur2lT7bL0O zhr=!fK|<B3Op9aN$OzZh-*&Zt5c<eo_9%>c8~(O`(rH;$z;<I|#UOq53Pqp(@!oQ- z3h<@KTSTUl60E3T8DgHfMe@2+{l)g;mj~8L8bt$uNv9-Ot4j;w|FeRV-pN$0S)Fz2 z4Qm8Roa^iR^ATxDn5kp%%7n<qPmtlRl;Txaa}GpG!^s}=k=GMs{g04mdWj~xQl?0e z$2L6hc#zvQEP`I8e<m&!te_@&k2v#`quYTZ4<mp!(4YSq#r)v~UGG51kE-vMsSv=D zSKJZYAp!-6Lab0vh72$i7u;{xQ5>0+$t9Z@`9}d=QT5z^@Xw73InoRtxH9qXz-fP* z>_#PP&|8{ous&ck2Uk~|c-+`1^K;=sWCL7xUL+p|9%@<gNCKt}*J5P0^S5pF!|&*2 zFwMBw37js1BQgE~pTMZdpx9J;Eh=eAx?@Uz_P?Ij&ezdknxH6X@u!rtpqOMb*$NAz zt4;lsfXJ51^-bL7)FMNiC(CMn<8)zUgybIrPZbhyL+Hn;r1XlQA5N*qTQlbk{hD^b zF)GoG<$|yt>4Ju7w#~hzDdKXY9cz7do$jM$7lt(42}Q+mY^dn2%UtXTzkW}u%lcuL zuTE8U+e{3uP*1a1pENuwoqSd@X6B8$-`hsMO8^>}&!CW^Qw7Xr(u`}_(+rx)FPyjd zAG(e45mtHSuVkslmfBR9(iJU^zb)%Hue~7AX|B1^MN3w<-!FMfs_<$D+k^7+2{|q| z!flWn=3&0vj!k~?h3vNt6KY+QEU6IDL(s^7|Mx#G(UW6giipbpng66qc-P}nhwHlW zmKLR&6mXZYTu#&}J@LBVJ1#AxN<{Uin?|eXd(pdFb9p1R&N1=D-^GXOOHoyhwJ~3d z5%oLlw_}f4d`&rDuok{v@j`m0(%zMfIM$ssc?It~7ETC3YaccRc?{p;@YB+K)evzK zbDyMV^$m0&gMHJwLgQo12Sy{x`)@+HB9AIS$y147Xs$x^4U6`gYA(j{c&2;HTuI~? zht*{ynN_F4)c-n`rwSQWm#pXdt4UUH+URHM6!W*wTJkSo?;&I`o8O|2xVLF1de^-K z0QbMYhYX_AxVVEApW`9zS2EO4_)*^HQUYnqAqz!C@7qFkyT2`2W>9U;aS(z$VC#aB zd-`2;>3i8VdEtT?@&?_=ssNO~U_pnBjU1LZE>BQQ-|1fNaLz^#aQg=G(TEoE*dONH zyoK9JoxBeY<hUBS#-QYsnnf1zPL7Y)Mme#!ov2!QZl7Up0s#l+luJ(h=C)G5EC%-G z1^CjRa^~@;=^JKw@teKe@jOmo_F1xQ3WK+O0Xi1+&V8~2hSY5B+W*aIk|`=CN`KV< z{qku<dxCmMrdm(oA<g3`jNxhQ+I{J0EP}kN^8aed+-;BsjKY9l;awOg74s^IHA1Th zsKU|XUUhrViOyhY!Ic^+R1QK}biHIHrdz;YP)@w<&*GWx>jlQjl7<zT@*ZdKhD1$m zm*9^4_T>cHw!K;YigM3ZV|Ufj3)WjdS+R1;*~Q%<e<uob0ZfEGnf`F`tY0{riK(|g zbP+0tYHO<sJSrz|sDDlUYq&x`<;RJl%aQCYppqy*%h4{F;q^7gS(<266wI8P1o^u7 zcAVZtKSJf|ZLKf1Q9yV=EM!)WfMh*;m%@AhH`$Doqoh`Y$b9hZ%nMKG<sY^VAK)aV zfIpE0-?v<vWC-F93(8T*rvH%#vb*ZN=ifd_c_vUsXWHM6KXB-pU1KS-obmK<X~t+v zv6$4^avJ%pC17V@WpAB3`nxM_20nN-qm9QWec()D>s#JmVgl1JH_>jE6qhmM>1D!$ zloqYGODX^S>~K_DOB(wM{$N{T?tig&v`KfJDj_MRT1(Eq89PNAxSKkAx24aH#Ap)9 zmM4OBWpcUsIt|@U4fzYLQx>;4M{SV#i1gX^-Qw!e-4zZzq4CGA1_qzT2oVA)Q$e#* zl0L>cq}5ub?}CK#!%;0KxBX!k>5Bt39&f2m`W+WY(tp;FjY(*PCYFj;)d~iG>ac&( zhfi=U37U>$1{9~&TtTzOAogF+r49!A3=Pc(eB)%sjsVmsVFz_dz9yJbsP8ROnuWPo zI3xAPzt(MAsf2udrA28duCusaX=px%rFMAKz}6MLtAk%(Hl+>G$J2(C1)ihdZa&)k z{O7?BF!g#g;1|Bzw;|#LTeReVyp_^uv0kio=DG*l9#)6~{4PEk%Lo&mGD(xnm}0DJ zei;IGug~3SvtCuqbdgn&bmwDNvo(8X=jWzs%HwoBHaiQoYn0&7Mdv@h@!QIO>18P9 ze!QK5j#tV`s~k2ZOW>{tJ?<f~df%y~a^*~b`gyrh924kTSXGdkUYd<_!Urt_mt++N zb&iSt$~S?GRG)gw20+mjvV`YkZTi;2q@;q+DN1$)zlXf?{Bf@D?h<qpIXN?bk`Dfu zg_}GwrZLwB-MC!10kVj9?U=qT8B*AY+V&d@wCtikmgs+66egc~rh}@E;t=$Iu^}rM z>KHb;6-=p@H0E6Fj<iOu0U!&ZYPJhO!J*mMeZPyN+|;t1l>z)%&$(wCX@1(T03xQ@ zOC4E7CzqUy?_hmchDU6Zk_;4P>pV<s@9@?ZABg(t?)Dz?azvoRgG!7U#BG>q?(`fM zpkw^n$F9q)-$o!Tz^BW+xRHvA1&wG&fe8c(7E-$8tF`A=_gW*)rM!2$Xr;GK^C%dD zel~Ice?nEzP?iSI%vEb9h9PPaZP05n3v(1lPcEQ92XxnztO5LJ^#VG+$AXxan_O$> zz=q|GpTFVi)*Fs1u!7xUBrFV6ve*bS&61UI%JyApm~2e}w4^cfKi*C(wNE>`S&90s zhrd>9L7hw|1()yqnSrwnQsS`o&^Q|N3Ld%l7qQo6Jh3#(Xm__{@E=-~0!iVI^2;mk zCPO(rYX=&U{NSeI=5Tg5H6iaE<fX#-BFcLtdRm(y%;^rDquMdYGC=um^tgHM1hMv9 z+v;-J`lx5H@vZh{(T7>@-=%B*i`7=I<tCdCr;BN8s^D}AYBT>fq}Yh>WckEwl-!t4 z>5(ce5q}n6biaEsy9-Z0L-=;QJeqmA^IohD&5QAKpm<cvl42ttGk|>VvHn+i;nun} z--{5B@S|if#PZSo;a11JGc`law98P@P5T`HNrVtgL$ILwD!&iyz0&+Kg{f=_(Ug^y zUGi!n8cV3(8{|DcK4$B`1`SvvG4qEe#+VwkkO_?}2p#$sU-*k8#k4)<x=r66qwhrk zY5kd7BdfLSdco(k5zUr9Xwge}-*y~+dhy50iHWQS8W^_?3V_-S_LG)c^t#L0RR$g0 z0wQM8>Fpf042hit=t9W(Wp8m=`@m%Y&yI|nz+CyBpLu)vwMV~U8TE5udZ9qFQlI@Y zto@+@VU>>9&hi>XXRJkOjNv!yOVhd;@b;nY6J1^bN5^1~HH|6DBf@3{*;*I%8?%?d zImq6*%q9Y7%(3QFcNC^-Hj*#%FUxUpeDuRMtK;uqYc5N(Lm~0Tccr%k8`~}UO~Qs! z%Wveq-1$(_?bVil&HAlWV;{)&(vG}8Z5AZ8;LYQY4T1C*^Lh$0k%AA`IR#yqe^kzS ze*!Zb%ZUs<_n{v8(RkQ<I>A6l1I#Or_yF^Y?3hM=<kZ!|nUH(=P=VGWuu-l%@1%Pg zO8eyGC#1+7KoHMd{UdZto3va1e_jCM@LArz+<-qiMQYwxN(b`emNvP6KM@wzA~8fW zN1r|P&VQ$I!=`|~o1jjm0A2rieKFZRCKj#|9i4NA)Xrf{4y%?>NzCGa?68HLFn>N% z{cc1x9sGtVPK|6iLs0M`ZDJln@*6mRPv=_Fb}MOly{lQ+!|KyKXr-qk{SzSob^uxg z*b6zHUN!uTb4efY!1Ux&5SL}CyDeqp2}2>E&vslE<zhIRvN49oCRTGUNWLxk@s<gE z4sHXfLN6a-bp5D)fK^IR5@>Kr1!mOQyzVy%arlklvsXS2>9eA;4inyZa!O?*`<5&R zrfPf3Qp*4@zwcp!X;~~_M4z*@qs1N9+?%^${LvYdalBEApC15&egvFagDvkrp8Gez zz0IMp)7KyAA~^Bs(!TU!EP-NbZLutut}y&prX6knPGw|B+&*9^vitE;Kc5G=g!nPO zNX&a%>27-7!5pF=FR?3etM3lT=b?eb@&CtpnviAocMGX&`M;F&a=Z1CG~R$)h}z6v zQIzHhESRBwSNxz6hY+xVCbrwX3FM0<U#`NX;!Xe7prt160>@93{gHDRL0~Ns8!(A{ z<Mk#`JDtCM4?%}+BYWCf#ur|7$^LVaz%iLQ24})H^AVD}N}EIdW{a_in@y_c%xm&r z>NeJGWN<k(wY?evu|y}P(4WZgZ>7bf=l{{G?xXixE1!MgduNsp<DyX*9E~Q6-uIHh z*EX6qa*rqEJxrx1?fb_sHlS2$o&4$g^sNSO+1|rPMzH;t(~MNKvuiEPbx|WCAkiVU zr{=9wwCCDXvChy5o9mT;JBxvLTb@DyTP;k2t=k<*)L&Ch!QOSWVAEgpC>}s?B1Mcz z6-Hf!jv+IYy7b}is<5s*Q#5}Bz|VZ-L~62G%I=HDHpER0Z3H4b2#X_AHSZeV7yqtF zwZ1Gq;Tamr)>1PGFiK=7mqj``h<C20?EPG@2&PV-L%@e-^RRsI++Y!-R1wjuPY6>d za;YfGFV6^Pia?+p@Lz&C@f4sAC9Y5Kf<FV?M(5p_Xt_)u-g~<T`#a0nOXCYe!k=oX zj#q{+KNjJ9na3tOtsk09O<c{Bca>^ds9+kLb2TM&j`4tGblR9aPI<{gy<pGp27puQ z?kIHuYXO;LTc(~KI=wE_G_kD*t~^T%EepEFfs4l}&X<hv^J7EvUZ$1BPE1}KdZ5UB z^jkeP_*~X}_KNYa^b1O^T93p<4%98$va<XzVBM4+2jI#xJT1DlsBZN0I|m?LKODNU z(g=;lz(gsCU8$bU`Kg!6hayPGb+31gx10fwPi#Se<;FXqJ%!NN!!yYn&ebwW{a(oG z&o`z9sKXJIbfIS&VjG0@>3<ytl_|f$y8g==oAir<7mPktXu)*Ivo2_?u864Z|0ei9 zA^6Gnsi4)1QGueQONKt~)x{+56fyr7+z$m6!hOGeP}9KQp!hOjPBLXg)QO-&kF-YJ z=xhh95{Ng>Q=9_|XcKkJ-s(h;MxXs^ZOzNR6;vY9(uX$A2C)0D0eSa0<C#GIpm$(~ z$Oty32m}@C>uJl^$&&)>!R7sqACC-WJU<7;B3?Sqjd%J%QFW7}$k+2QG~aE@2AH9n z3#gjy18Sy!NZhZg3txrHa#bF6Mowf(2JVRbGx}@eR-&uA@&-aP=)5%hV{zkt_j?|Y zi#N~LDD|-4`mG|aALV0X>K~m%Vd25SVF&PB-e$j;Xz*XGeTsrM!_X@(eiRWgmNe;k z^AxvkgtTkxn)<x@5X)m>LK?^O?VHoH|59acb+TeP60cVxe&;d5m_~nrUfd_&Jhce6 z6SPL0|LWx-*C}zX=j7@-U}PA5AX}3cHgvy^U<1>X(~c;kq8cORvbYpniyUr&nHI|K zHfn>kVP33(#n|en_{ZzdPEDUoIF$xQpBv33Byy->MYa`x`EAIL!G9d2`0f}~rWyLD zpPJYKsmVPvWEvFv+5#Se@60NJ<Qh*Pxkifr`aScn6cjlW2k?-U)Le!#P42|#X+22T zmOtA_0OH%Fh8j8pplpUGmC>CE*`DTUOmQ7LIpFm3B8M}3MJWGZk=;SHV-(N`5$BK+ z*!8x%j*=oYk3SvoJP!5s#HOQP3+S1J#im}JOIHS&^3C2!j19ZME&x7fRM&f2J?=2+ zcaZ!6iwZ@e<hJ`QNt>N8D-|D$;2Wm$7D4260k44ri`U$wKTDM5_V83x%}WydJh3yW zT#NwG_GO}vXtxr3X>`+Roq+ygai^`)c`O)}<3@bXx*d6A`OB$O{(Q3QYjU-ag9D%N z(_(UkiS~zcnwIA}fF#^?qfGP5OE&;-G#7(FqK_1pDiK8~i~V9lp+L5=*w5+?6PJrY zqN3v+@}`DL4kU5(waYBqP+R7TS160@9?f4B3G1*5#Y>iTc_;LCp^b;r-tNaiAgQJF zjhUk^rIGX=i8rx;x0NU?htQ?JbU#X3Ot4+=DF0y~?b9u1<^TxE?@rvY2)s}Teh)?3 z!FI-yaCqZn=73ciJNioN{y1srO03K_pL6__$gKw9vL>5o#_bebIpm*$AY%g%bSq89 z1}`&#6=r&S-k<IdDb&_EU*>u}s0H*|chjgQeHb$}bp1rIL{;|EenS?ik#$)0Pp}pC z9OzvenKFOf)(y}U6pphW9ktf(CT=cNtS`Fu$yIjNl>NB{0Y?7Bz=L&}3!i`BGqE{M zG0FO`H@EI>(`Jy<klO_Z8bhjqH+yZ6H+$<}&?7#wfjjpLI&L@q<-mkwJ8qk@o)Z@> z?}x$?Qk9Gh82U+_4b6r6>Q<1>U>6C7%%@&&-~}jYAicI0$ASKOls;o@kk}{mZZZKN z#o3)|J(){7ZK;&%dh~<j*14+t$Z7VPpkxLmuQWkihxsYzsabVn?&CznhrPt8{U*m} zI~kxRFx2w^wvuLp6tY0Nz&IY@KY-=rRZwuhJ*pW!`kSNNvU+<%y%!$>Vm`+0+rTJ{ z3a$7<jiPT>`Yqap{@Nc^=&3T*)13SjIPu6LpjmsPT&bZ+PWj`)LSg=kiT;j>i;)Xy zl1Pf#5}*j&jJq$a3_=BcwJ`Ap0mfcdqkr$re?K9rWJj{^CiyWx|0q7b()!uaP=QZ< z!g|t~rt?{!{o=;2-q!@ZUx74gb|Y*Y&sYgR0lOBn(&QJoZXa+*KYBS~SG4EGQZ;Rg z;CzRw--GoUihiUg2bkS*lWvP2=zmUv%5^yGS4!{N47)my0IKKDs|JK0u^h0u+88-? z0J{5Z?l~Zn=!@m1-JuPoBgrP}7Y|*3*7v`wW2Gf5e=SRUyqTPssgNQr_QFJx<DxN4 zQwUjqIwv<odSM)0mevP%^MMegJ(c+}vNZu?-*}pGd&uH^Pzc6QQVS_n;P5(mZ%9;N zdN|h2e(9M2+XrbutjGjEiKst|Rb{WN8S7rA=hrG0Qi7J}r@a%i4|<?4GKF1%N+2^6 zNm0~#z(5-~vv_<|pa#kyK9q<Q)ktR~j)*8=uDuSwX_n*ZW@)k<*mRz(m#|^0&GkDk zWU7BnsB5`}ZYN@-vGVKCrDeBC8mGnjq1HNK3MxHckL;d-15dI4!cssCIIjBPi%o{l zkLO`+-Es0RL29Ho>Y<&3)kYMo^4W`UyQ^!z=LJ|ceGfB2=sx%yuD3-Q<HBS2>gd%D z8071HulO(j@yo1lyl#UzbdvTztI=W{XAtw1LisUp!<{hH(kEq=OS;0B!o{WfhM21} zyCq-oA4<mu9yZm`b<x2>UwclG{b-a~oY36>GO$KTyLV@)B^WJ13OT^Gb5VB(KMmSk zA1yNlD{EeCqlC;U1dFF-3S{<)QV0%7z2U@|%a_t&8otKL(nO?qfSwe4ZO9#-Y~Lq^ zdb$UiG~uc>)0xF7>FQ<fS|g%<I|sn8%c52SW=>*=Y{tNfv9CUop%Rz9R)Bhj2|6AQ zX{>PQu%39lf%W)NRmOcHa5{Mu(wE5+GIPv&Y`#TU@l%+`<9{`8AxK8tf#@aL-mgq+ zDR}|Bzj3TkCjWyg<^u|!J?za{1z^Fg<&VMPcY&cIIKfmq{H^D=OsaF(Y6kT0*qEg< z?F(xXwfmi6zQaL3p$rc_%k@2+S-hSSkauBHj;5CQ+r7+JxS#*$zb54x#Z@<;#sDCS zybk_`v<d-8ib_0=ey5JJDx!V3D-t1AD1Rf!L+0DI!u7q>i_h5eU;dhC;>^j9Qf93U zgUiYfL<6F}Sn2D=dJl~cv6i|t=Wx@7Ol2TWE#K*OL_;z5zGF5O)%3U`w-v2fABi!g zB#hgNdfI+<sf&+Qsq3^lPACyW@ALU%Tr~iB^xk|Q$gb>tmmvM%^U=aLkkxnO==}eG zl%JCM92QiiCkkYj7T;V}`ktfz7!d{jRo&jf6cOKxybnxq&NlsDkSPK=9a*R)>w&Z= zByY2wyreBbk6MIjNp=FE+NZu`+80do)C%JX&n3YNt_%XqvC7@iPDf5J!PBJEwCc;! zrr&`A?lB&N%ZDGuceNl9)d5sWOaYYlXiK*){xSZQBo_Uguo7IsMCzLl-~QvGJX{rL zpy9}}$PO=F3l7)s9h{9Aqz~?>+Y)7M^nDcNIl!XZdepH%zgtV+G{l?~`)Gq)dj02$ z-|xQ;&%GRZuse+vAgcwh`=BU{E0whKSGNfjlV=o9Ax$ZhlC{j+v({!ELl(jL4CD`j zld~D}=-FLGyGik6v{2lM=EAK-?59efg<MaFKaf^qp-SbS)anA5A+&?Oc2Qyo@uz~Z z&!7>i%iYi*Ns84)$x5pZ8fz2!Kk?u}3uAz_=COHrsfTT*A|%J?v&vf%Gmg{aP#KSX za2t+YgV&KK-(uMoJg){7hv|D?rTJY|ERr{1@(jI-&>@3l(OR5OFxQGNLQBHCLiA9_ zB@n_Kv7JV^^`DDE@9dxmnD&ifysmS1I@i>;a?q@Ilrc3^IkQzt_&bC*n<d3tew;h} zP}bo%Z;ah@hf~g`zrW?bS@E~9R(jU4aX?Z_f09BzVD>%X<iiPe<0j40CdLy8?r<WB zEinRR-4h53$t?}G#BB$CGW_O#JUv3|a$?N0RFLNtir1V0BQI*h%^YoIiw&v^HK+A5 zhDlxj**ocESO{(MqsLiRNZf$&3?kMeY$C9p;;z)H+@bZ>;EAC_|KF}tt*hPmyF)A! z0(+T{?m0JOQiSLH5c{4P9o-_Z*x{L%7RtJm_(#5RsuQ;+6dK2ni169YV%a<J6}Ipn zA-9Sx<N99ElSY1$eP6<jkew%!0SLB>!2Y;yiaN+GYC@{Rj}Ti<c*+OW8isZ0+qT&* z;GuFoYe2pO`C903ySUzhHdCKrt8MxjBcYD!Rs_!65{)8x|MlJ$VA@tNAg9uOgR_p_ zE%_c&a$y-3ThhQ!-Trrzvzu1VTR~g(8eA0}XCqKMGw@qbeWw!Ohs`4x-d)oV-0HLi z_T?o3ZmmUdWTW?H2t`nJ|97A?cq{<3LG+K{nP+~BPhYvx6K{Vqd|G<bs1Z#RJz4!q zEWR%_LH66G?#`ZWd;p7~PslIYWcCTiIeK8wHvs4Ww7?#<izpX+WYUdtOl*IXkb0#G zXFBxVge@)m05L!8eX4_w&*z}w`n1m<R`pT-DKMiKhmMcGI6WM_nm*9{0*FTQqPchS zKYd`8byJgYD<{6y>Zx^Wr>6uLEBK0~`GY)H{>M?r(m`50^5MD&p=Xf%U4-zL3Z~{) zvE!{9z%8!<-#P45F~nIF3^iUbuMrs=U(Gu|CrpzLT0n~Rhf1d1t0{%FQefwU(^_P^ z?*C#&WSp?hC3}Af+;ZZ-)VzVSO{u)UW4x)DXXJ+@2}jGof{VNE0h`3~qw{Qk+N9@q zXdPbA9`P>LiV)*x{##68XN~K;^8>4VKn${0hApVPr-Kkz58Ap-n-{fPFNP!G>9Kq1 zo(nA<iV*oGeB#v0=_-vGuXhwyrl`B1hrMm)#gEJRVLUE7AK?y@l!&!z%+Y$Uw)b`U z$go&tY-T4MIHiB!(b&_lx&Qcs+h}nmAh_-3@1|Y|JiLNWzs2DecY{R1{;w8=Qo1v& zHI1Q;pX8YY7a!h`#=n?DKC^#FUmv2pPkpYF73g%COna{H`dlV3v@1|8Mytk5P+^Nr z46*3P6w1-KjJ1x5WvSJZA5YD`^!HZ}iUIs<CP_o<KA?&rh{bXDMo;pI%zu6cJdH^@ zU@ZUs%4-UOp1aQGXD7dT>+^WC^ErckqmfX@Six)aKVJZBSpj8xi{^sz&p+NCwzB@# z7d$SgFS<*=?o+mh1D?w2yXMrWDaB@%x$x=OvXgH1I!6o)n}hPdc~4`ZVW^GPZ>L7K z*xT}&Q3cVwT=%cHH_}qV2zzvnjd>>9?#H`JC6UAUxO)2~0mF7^ewOS+z_wXSyQWu{ zpC``C1xa6&k!qEK`r$Trj9g%3><@kYl9BUo{MnHE_;9mlhX#5fM7>E<@iglJ<wOxa z4pZ*eOyH$wlc{ewsp-20^_%2<C>2)gD=4~F5u!N#-;C%0^gVxJo92~6&60V4_oz-@ z!Rf-1>4th>ALOtrevHYxg(7Y3Feat!&SCBQ)R$;awB#q$NVQ4)_q?qcG0%m(0LA%d zlkAy@bg#|m$`tF8c#svSM@CUtc*;2@orrvYq}0P~DzKLZ=K4J$@bKHs50zCIaqHku z0xVj_#FXckRmNi^CfaeMU(Rdo7&J&@m^1zOn_$NQJ0W*z_yC$(>B89etZJ-a0-FaH z806DmNl3NnKUq$!qJK(ABd>e<J+kLi-BwC{X9;Q+?Z;@0joL$CYtW70pF$7T4c3kO z{9(q<H%0>0QITIvWfpw;Gu9w}SE29!+{N~wE+%9f?#S2lG5;)hl16cQQ>m*<kgJGZ zDN$+Q@k%+9QWx?D$<x;%H^7}2fw%?m3_dIW5aWIz8XOuL`h<gu9<0i>_@s^f{c5kZ zUi-0x@HHC-2FMF7uFv_fCFx(4bt_MA(|1Ullu~UQJW*7<`q@(9(>fRHT?R0z-IJ>% zUlf!U{yg#;hmL8aK0hu)8;<*3UJ`ds%MubQ@fTiR$F+q;#1^kWc=h)(6g;2Qm3(3y zYBO`ENNXtQm)PMCxt5g6LQKo5Af^Cyi4GSO(dTY8FihA}Vc8lUqEWzYz1e#Jejcp7 zsP-yDw@u*G&zqL?_QQ!{>MyG)d}{Yd%*b<pkU&(qReqF_lRR_Z?JH^t)s%O^WNxag z4>dgHmSh!PzGMMyI1r|a>ifAX7cIw}9h51cgQOYPHs?lTTEe%}!O~f1PnEfMmro~5 z$MC0_WbvPu90Dm6hN=>LcpGf3R^%HtHh+HK4-j|OlC6OldyU~rnD?BA0&%Y!)3;*3 zx$6vM-H_J4kzd%Y`k5ETnbR7d74*?*t^bb8i_3*tXVWZFuA_sC=vFdL+?dga*yh06 zVTIAB%uOY7hx?i>l>QED8<YN6kCfa@hN}`|?zL4lg%Ym~(TqsEwj7(IVw8H4n|1ZQ zZDmp_#_}ARK6PQ3k#Eb!hpMI^A%4i;<b}RJ&D*tnR011ytAXz-$;0E5N0Rc_*cm`C zaLSjt%%qIHkFu~b)#Hv<cUy9N^bj1u*L)GGHQW~pidy_mSq-)m^adECQ^SoSsuv6- z6D22Zr3zehX_4Rz;S&?GB)0)B*wE*|I~2FkFE`AuRJdHs_0}c%-2`MD4km$Ju<FsA z2mA55;0Xzu{dySoJ>B8Xr20N4ILVI{J2%2!xI+0Uke^b;zyAF$TWVAM;k~y?Z~!DQ zU)r?nJS#Tj|3ntgZTdH6)^E%%!td;?H>K2h@9gL;2I7oF$enc(Yk1T57d=m1d0WqS zRw>9qmpJ*r?DH~MnS*BFZ~B@_&PxgWl(XsCGIOn|ALi0N7P=ON`%5pIm+G?|FmqGN z7`1TpsRg!2Jd?nf^QO%VHtexrE_i>Al{RD6ahkPrQ>Bf^>z=d0va_$c=f-1X3aiN| zaaFTUj+jnMWVYN<HnB!n?qUJ+y4F9ldE@Wp)B<rU&>r6~**pWpvo&+)Fhh4s6o(c< zO3OxUIZm)Fcbsh}q86m$<%()Av3NOK__QiHG#4kT>l*-24Mdj3-bwjF$Dv<k{FZg> zInRr^@lsGC7cp!B@~W|<des;poZKL-85|tFoI}~sKj}`rSzEc}{5?d)y=FOfCc~DK zLu{eiIY38YQs_8S|IMyfBPUhU#$sDY72XN0et971yZ_EGI!!t`P5zQmlalNWOY}~k zf>DLsxh(1LhoDbNdSYFJxcHLgy65LPC>E%SV(*Kceo&iQ%M0*W)Va7P#X>@j>O{1I z=A}wN5{EoZxlQ$uLQsXho8;b;_d}$~(@hD-a@Db|c(;<#7M_{Iv>(wB7P5T$<0&ft z;-Wnj@rd2;tPL?ysnN~-4XKYjkD5gP!ruhm87!{M^9O-2k4(o7fd~E^4AFdCG_?I$ z2-*{Vb>w^c#a~nwJ-l?X-@c*Z^Zr$DJsstk))7{vdy2!Kd$l{iw+bKgAKz}gUG`ma zpoX_dsyb<(B_i0wn(TzKcKR(lkJH>AM{0$IKMVY`>cQF>H(GC&3w=?!T+CRqB;V|P z!!;1afAcFBH}z}l2h3k-Uq!@YHukTusst-tQPK|NQ{!_~&9E*A=)7c{2^cE~7mrf# zW;DA)VEttHLz&4ex<9qL4|F*f{&6y1;7gdWZAbSkMwW%Bw&Mu)e*srM)S>WCqUXps z$(V8fY5hRTcBZ$QtvdCz2Q>U$&UXB5%Np|Lcx)AsK*ZOYo*+87tq6PbYNLhtMA95k zF-Y5djOMsa<xL21-F)Dy;G@0rEy4FOW`yM3u@Qb!E}n-Gk}q{FzKy?c!tSe^T>K4x z`w+QXq3PV(J)wT7u>o#5UB(S}`;T`S#;cFd)p~qqC5bz^UHY-1iTN}s1G%7-AtE@; zTtAi)%)2I?-h6mU8x(2^6c0ZC{Oa-@e0%rEXM|NqleeQ{6VGPKMjJG{>{$!nix2H+ z@iZ`K6wea6^Skt>Y5bh|v_O%9AHQ2BL&@7{0nySNZsK_5ZpjqB>hFp#zYe-OWP2i| ztcnH9IDCvE^L%7;E$&wm@v&Km6Wg@A0BB51k|;u|fCn*byNlO=A$}_!-vBr+FbvKD zl@6M75|R&*YcOZOf{+@L>Do1eJD6cYEp9hLSd1&j0nL#z^Z48l*%+`YEI*d!eM&al z2RR{lp1o2Hm*@B2>_X3=OHUPuNx*}CaY%-2;Aq6>E06odE>5OW(pMeId+HO9H3PDn zv6sYsT!gc9k=)J{Wo<YIKCZ@gD6Ph8Zy|d6`mM7v&Wj$;_O@n>E$2lvW<L+r!lC@e z6^6MElnaxiczIY-;QF@b&MVbg=hF(x>h7QZDza;J*Jm<+jTSGrML+1N<)|wPAbcFE zxjZijw3C|m@oQXZ4fgztsjcRP?tj7vI{Wmgq%YX#`rC|dgp9C8{d`sYFb)<DF$-C} z<g0?M(OoR?c?l2j{8e4g<UKH#;w0t-ZlNO9A9qLLdO~Cpk0$Er9*J$qq5Ky9bQLe8 z*mQJrUJ7GU)f#GW)30_HFL)l^ISsn;h}RL~tD`3q=z}-eR-RF`rTa{l9kAL)X5crZ zUtN1K7;x%7o3Ka3%E8&2=|el`yGr#yq7SBj;!_w7_)SK{6_ziu9GI4S+oiv6w@oAd z-2D!<Q3`zpdOH);`eJkskYK6q^(%R+MGWODsIOQ*P_?y`CxhBLP6F<2r!In6m8ykK zm#+A3wNw1QZSbU3pJ$oh#KRTYZ656tUkdL&+>k-z7^J~dhZbV{Ag!-P+^cHa-E+uj zZRR^-!)lPmcRr)uwyQ&f%7}SoH*iCC_N0Xn35M|4CON{(D3$Jgor7FZ9W<>~D3|lL zkd|+#{PhTllQ4TJsj_izBMB4kkw$xNv!1)b3<$7<)av_G+=gZ%##~>RBkcr7Z0+EG z(wz@077!gbgPJUjbw4JN<|y0;@jpDw<Y1waCUe>D?RdrTfDB|i^X0GTVIHnxW?urx z$Wo}%XAmVDc5PKzFU!=wV#Y(GNE(&RMjAgVtq;<zQC%8gK?oBgT31+tq_y`AP9?v( z+Yp4vf_e)PuUw!hvZ7eTksk$A1(L{hp8iR$fTLkuknJ{6!v-}Blm9AvyGH%|7iJB4 z%+}&hP(H~I#P1>FLQVGs!Ouzbf(w1Gs_W1KZIM~Z&BYe7VH9XpeF33NV&x+;>sO#3 zGv65DwDtzwAJW?0FqHR`$bq@P0VjTZ6tBKDrlB)#<vm8j1<YI+DUM7ERk#f*bWOYR z_A)JPL63_HuXmFftKp0^V)_lceu6unE?SD)WaF$tiqE2qx)Nb3?XwGY=7P7e4PZ#0 z{curGtQZ#uY-vOTDxjc*vSYWJ(CoB*tZFQw(Ul?PWq1L3%MK~g74Agbs7*WOKcv~P zzY$svvWN-0M&JxSKfcyRst`|zoQeyMUE{oe*o0T4al7DankS39dv*1gWBM96&~HDw zOKv)@ggluf!rHJR3@5C)u=Cqxb?#<?{~u433x*csHP}*qzF&5(C`*Lq)gB@bV6)S_ z=?l0!!%HydmYnmyf~I!75O*Wl!_e!m0~`;#N$@UjeD)890&|VA`z`MR52gd{9i3s_ zC?dhLyTq#|gp#hxGFGl8)o1-X=8q=a%`fF}5cPjvfSJE(6P!mGl-0J6&*iJtFE*EP zb$>6O5+Sy7$a!?KrDS*31=JNCjTtIyYzSD+Wt-`Cs?QAA#-+@1U=(K6Q}^CsTH8V} z`Eq`<y6j$ixSs2pjzo>~16<sSrv2qofU`5CamyW(Q08y#w=R}m?~}&y+x$Ik@6VNa z9zpX*PSYCXoC#=Xh0PI?CBW`;e{fIzGxW9K*lBuMTIk?oyS5p3J$0+SVPY=6^5eqZ zaPk1CQT@-Ak~_sKf#3Lx5heBB<_o}^T7KoJ!-U9}u$C+YyK$pV@u4kK8Rx4MkN&2E zX?au7fm2aXzoHq*m7_kjI>v)ydtWO(^6Wm4gnry7hry$GaJ=1_z!T=MhqeG-{K;=c z0mVAC9joJ%oPY^jK75+cHz-e5uJT}`_qCSzu64gl;5E?Kq9GOk^y8E?<vZ{SpL7QW z{=lK0M>~}cJ2sr`BG;k0CmH)v9i9`{z-1mgPnYE@Jw=JtQTnG|R`XWfNtls+p4OI& z)qKumMXZQiol1)*y54t7`Jyl|cj!#cZ;qaGM*0#NAQbvyaVu39vMolMX>c=&=X`k1 zMhU?<6*I%banCVQsfBSgr*~xJbFV?s=9$g*fPt$&okPU~GA(8!{ok)|`i!Ifc%y;8 zU*kY$x`BgQD`te5rPgweHS5NdJ=mtb`>OJ_h)|`U*9dRydHCs76)U1>St6p=xw~Vg zgt>*bW05LB<u8s=FV;+O=H!dxSj1S6_z}`d_Ef5wflA7vu}089&V$0gfB%C{)A}PT z(6GZ!F`IuBC_`rVHrix^&T8QG>{V|l8#Jb0o=mgvM71l8<B|mxFKFjye|<pqG5)Hw z&|yqfGhWKY8Cr%<AkAqt0vB%Y8vzX`&^=7S!M{Lm<P`ChUqPi!H%bC8E#$-#9N`0$ zrnS>+o7#nEa>&rw0<ygvhd%iH1ZkYdxSx**Q}Y{5$*2q{|7Tz9HMfGv4_@zz<8cnL zrQ4mTq1JAyAk`HF?PHuakrNvhQ%?@@tNsd4Vz@#&vj3BZ3rz*x9kARwfKIP~{d`4+ z-c~ikZh=|;ba5U<ugCPCek%6)PPsahwHNz92hGkn;%FsCI{M_mgnmj6v-#RAX!LWS zGkChk7rmv4esL;X30Hxs$eWRaW!|qny)*ge#1L&G?o6W)Lt6KWpw}+N*nn$Sm!Y^r z{*m;l+lv@TnS3#2xc6-DqaK1HUaQ{Aue)Yq$J+TM64$gLSB7aun-S~71C@b3p7@#k zL(}=jF@|%$Vkn?NZlglpK?>#3y*})v>z5nIk!|stP547R>U`I$cGkJ2S#B#OeK_V( zdEWUfCESBpQ0D6}vy*h$fvPgCKdA6bC(h*s9A!ZqbulN|{NZ?9@(61`9JzD_?eTfn zM|775mh0ujF89!~>e6r0%ozZia|Y-n5YudT>`EsO2T9AuOgo*M3GBZeHJwk9s!}Y% zXK-;FLADWSacKPU(@y#cDEc_~;^&?Z_H(suH0MF(d!5x9K7|F@Kp#uB?rXm5<;&%| zzZX6y?C2}k2!raYYf=<HEkMi!wc0`%D;Ufuax=%{Nec4iLSFFobnuIjT^|UgvYm>> zwY_C{Ml(tR{-Y=R-e>|!7Pk%6nzi1s#>7UISVF)I9>fURO3+}>lpDr|L~Sfkbhjs_ zhrojMRar<*@Op(={X{Vy<e2OINyl&!4Yd#KqLkUo<f<KPv^%qQNWVs$o>E>>N&h8= zJ&2Oy^;=)Wi3^MG*{%!WqlDLYOcFj6-mwUHZy39$Z|m0(#l_!Hk>kYR!zA5%ZpFPC zEw*L#@W<0n?bUN|m1B!}<^YbcxEvjFZb+af?C7$GU(bWUsr<*gcUy47_Gr8j%O|{f zbnp`ix*qRPTXP@eQ|pUdcf8MaS#G+K_c@;DB?+4J-Ky6fBgNh5O_*@!cap$&0#`0a zCl_3EgvBJSmdt5i-Dcn!P*gT64%ZF8s+K$weH;sFVui+haH*L8m^fMUaXUSXV0FA! zsnH=CMZU)a`eh`)*<`&l(lUb6C+KxdUW@p499cEcI>ZP?p3NwqFpkF_3Me1uUy8l0 z^%#8Gz0NoEre&~deCBD@<o@13?Zh0<sr4lM-ku(jF0p>M-uVyiG<ennr!roGPa<{y zH%)MVUu^R4C0|Ybgk*Ko+U$qpo<%8iWf&#7`IE=gVUGrW{2P-Qdr}w;i5B|Z>n}$J zHE9TX3~X(6KE?5jldEc_-+U=Ypkk3F(;SsnPw<oT)H_rW{s!ZU=`tprUd_SuSi7;H zS=o;|OOA5DL4zM+Rnd7~_f3>aChq#cQlGS@nT&5^0CIhuWs5gAE4<|+s#%0V|4`J^ zs&u`2ox^Ygw(CV66!Re3kDHE@e!R8U=dtY_l)+)%@p81>n3MBGF5YKF*doLv?`Ox> zWm=Og4K#QEsDWjT-#+V7!v|HViJG%*MrOg^A8R|$4T5@#CHfb+w$S^^&h!U%`G-up zDXKGW5E+QKWzuZc#>*=)TLA~gd%n7UCw-<1LiMX(aG`%WhvC?^Sf^Oz(0OrAZBYxp zUe*bQI!mLGTC`M(ZaG-UtGZmNqO4N=`QF}d@d|Hs7JkPuy#bSpYa0rzIlqvAI780c zH)j5M+~9k7cv-;a-bLy8YHU;fJp8X9Miv4LBc9Tq%y`Z%WSPlbak`i<lAnPuhXl@O zJX-sPTbF97Pd&;fHVpQUNbp%oFrp_Ka?VFnWQ%cX$lAf^CeJO~tY$;mclU05>d(U0 zTa#N2evEDQsqxbJv|VCJ|0z9x=;&5YvTvEWd}ct@paRdO;XdS(pAXIq;=93k($wQ< z^X7gM;%jhAR{g^U&tull3RoFj7CKg5_-srJKHB7kG!6`zb;m9$!h0y2f>qcypR2~J zYmFc#5vkxiS|B6et(<6h8=om2#5hcyXaM^7={A=h6Etz${Z(A}#x=07=%Q$BD%e|$ zD><~ahaR|>t0V2?$;kUaRr*+{nB<8(8<B<TRSaaG7=;5qoW?jPE^F3C_d~!M9)nTq z?5R=&nAL?0Zn);LO=qB(8(nnzK+1GA&mku(jay5$Pcg8R+6moCCvmCtai8!BSr4|? zbgZzYo95(@FG{sH)mo5ATMv*tE^Ia^+3MG1X8M@GQqtRTjZX)s%;iwUDEiWd>6T!d z!L}65@4<YN{=pyPlf`gJ+>~%TiY3)XxrfvV6?nqI)<Vb_LY}SxO4K$9GC_CG0!*z? zs>cVWyKnZEcZ8W-y-3?E@4>^$Q$*-@s^ufnx4=7mJ2lL!pzNx`&v!-Lxn<1_2nyDw zwX$Y~)MBpZNCky{7Ocns4{5L)+lLF_-JgH)eY-^*f0dk%TdUJ~;#6^?#B!3&S;mP+ zMAPw3kOnYtqYkq1mm-7S8<2W`Rbfv%B>yAO@_!Zct(UznZ)aG;Z6GrL*?jEd4w8Ek zZiTTqh%q7>e)kg3sGQv+-x+hanQnH!ISeb8`s%xh8a#H@H=*8P0*eQ|I9v}u!wKp6 z3NIP3_V#7^*N^$u6(-1^;Kdkka~ji`0+uS5HoIMQP(_;TR!FN0PrmarXJf^v_sU%N zXcmsQCqty0-wya<;v;|SK-3xDl-Y}rY8-Q^9emZ5^)0JO70&-V#6T;$pC$Y|HmH0- zhDEVY=A&TBAEv6$C2N=Qp6U%e7g>+=(YD_j1yHJ%;X7**u}8tBY&ev5&i*U9aS^lW z@@!-0dDp8v$iMsHt=;j*6On-|hx(U3>q8e`Tlu5eKIsp6Hy`TUuu4q-YjXNANrE)4 znMgbPs|ez|B1sL!i4UK1k6*1Ek30@hzK!f}?>3niY#EuFfch-92II2iPqQ!{Wq$Kr zL+P&T9ka1=M<U#6IJNU)>>tCH;WTU-&1@3<PxH*sSzmW`H)`*~Q8UlvsKjf2I*llG z*=X!LFJJ3EBv*Sr6?nOi8`W*DGA5XsqIRd;%jeN`D>G&C>-RDIu`Tp>CTHaj@tCLo zkEw5uXZri&pSw^flKZWTLW)XmLuC3Wy19qS{T^awW+YMW-4N!Ua?L%Ljc&+oE}8pA zE_0i?Y%`n9@7?$D`#m1NfBeDr$Lze%c|EU(jD$|rd0N4#<t{TA_GiMEAP>34f<7sk zT>rLXUk1{;883~JhOac!R+yB_!BGtZ@(C=zL~2%GaREhs+y0aw*e4qjbVaw^l<LeA zDKkTkV3s81nNPbRL}BCUlu}-3rBAnwjOTprc6NiOqVEeul2GK$`QI#p<K-EL)`Wx5 z4Ko^&WzXS+JpN>cnR`&hkyd$zBjwqa`97@r@5QY-scRqi0;9VL+v@QT&x<{O#NqkP zNL1vU;FDn)C6Qbd{AbvzvS7!5On^EN4i)d~ma)@wdw{b{m5))VyI{lH1}QMfeNtPK z9Y^q-kcTaUyf5}by{1Q)?$jakY0rk<g`2r>r5$Xi^Phi@H5Q4B6;@(rz*5Vuj<mI{ zDL5joa#>=B4Ue@?G`#W78+GnPxXrjvd4_!SAuwleXT*l#O1aji3E45_Qnfsa5QY1P z_g~*hkNT9`_%h#vvq)LvFa*LJ3cF7146*C#`BD3UddNlk4)Y1YeV_JgXxC-mg0$}v zac}A%!#0a{n?2bSNZ`%%v*y;=yt-soZq3oXQ(u7!YveqhZ!hw5faYzFY~o~Wdr?BJ z)j!+tqhIBG?42eObE0muu4?wxBq25x&FZF&P1mBW%XI`!nG;8ypT&+%NAk~72C}*u z&dEcV`CxQo@BT5K($A@(&r-Eo3j8a-7LeURgEt+eq_ch`eI1c9RTxx>K1V7Il}KjY z+yLn5*ya6Dwh_yUjNOo)Xvq7m=hjY{u~{-;8G*;hEvwPC0&h9JdD?`3R1nZ5B~?fM zhuy`DINTOAKcTJ(gFm*9^o*&|GZ_m6`@0QpWgq%97Otr^ZS~DCvbFk&aMVFXa&H^u znz4$Ud?hz9eJhIG6GNSBt=kN$-qsAR8Lxfx`@kSt^rht5(Xf-WukwdvciKe{D_tlK z_;I1rOO|qe71Du{qo+hOMH9kKM<#LB%bXF-`>j8jXdEa5>2fK%1b!up_N3sD!g5W) zsPz#&s47uu7JgsP+c)q$UdPmbaPSp{s6t$}+nw~0g=!*xBQj|S#)B8&dZUeWmB_ao z-4VW-`AT9cuZ1Ixf<HDuA&9eBbuW1@j6&<>OK7n=GJ*`(-Wb}SlV?BD+xZG_BJXck z`w{lsU79&&HfUqt!oDEZKVqA-QVnZlTW_N?&z(Y2K4Ke2QuPX0Q>;M$U0v#1+N)mY zRz<?1gbNPOens%M-KUC^GK*4t2v6Jj>dE`V)m~OD1g_TYsEvWCsXXW1-!%Ba{t&G+ zZ_gCoZ6Y@u_4iCvVJ?d-*XlYuuf{JOa{>bAYoSFD4c(kfmlQ+PC)N?S@7jd0bW$;L zB&bc*5?Rsz^R2*C<KTR1hgjgRhs^{<SDGJ$c_Np?Oju}`v}kgFdHqG3+LS?FmU?vP zr7D@VI;eAA#O>$@^Y*O8W^6f{n#C1E*V+s^O>-9-X)j6PQOFCiEdNhA{(X?}wT~{_ zS%%CivJa&$d9`U-m5ZyNQ|0#(KgfclN%Nen9gqaQXC&C%&EY6?fA}d&AyZ<1q$W3} zv~JY4uEr|l_x-{CXXIgIIjE~BNMlSN;P~@A=4L9wvO=x=J%2j8kpP6<_Cm{7n`^U& zKF87o93op8TQf;zgo*Lq&L|I3f965O_{$u7k6dj_-Or?hY2pWV(G`tKmHW|hvTLjm zaU%scsfkdXH>$I>pYL>p+3>j5x*pfB@PBn5=HbLIgITP7@Xm8WV#Tuh6aG)w@jkW7 z*zZ4nulCuT0=-XWN1F{^WVLt9H#fU~ZsJU9vRae!MvJR}U})Ao+%NG=@u?Q6)133y zRi6x3Xo;KKUp@>{R(O0|$?*(5+{7whU`+^gn>kpH1|X|2LyG5ur&0c0u4+<nLr<s) zDc$x8t(evw;V!Mk#?~)x6$$HqS%)Ape(P!dwe&S|DGEvW-h|^49ITmOdu?E&dnXUl z$4kMQSNeLG`VwNGfd!!9FM&L?MJ0b*<}fN&T-=ZNho>X|;pxa*<y`=twnKCBN9QyT z`Z)D9L=hje5bt+1?(Sv2eaC8Fij6-vGbD&ku<~yUX`%_VwC-g=A|HkL6&FfORTO}+ zCC-N;|D*HAWEO4~|GD|SuELN-;4&R52`oqN-bL_&Q|>1<Jig9pC)Zwsg>Rl>$;3(D z4$tkJy>c}CboTpmA_LJX5f`lpkn(Xeg`0;FT5U#%g61N=S@^z${Z2)04p*{xOuO>M zI&L^T<kPfx!{MUwv5t~;L29lwV(d-b!sPFD9bKd{-1g+gXCzwN9GUX?3YXB@n=~p2 z6vzW!wXik9H<4w=qKZOND?kQ4Gw5kz9H87inJ!#|bYZ)rbTIF@L|K(GDOHBKKkC7_ zv7uM}NH!6PN73p_8`i4ZF{GKgQUhX)X9Lk-!FO^CB1@7C8$%QG0$JDK%u+BOLyqzE zU#7069)!ZTSA>0#^sI6q8p$w%vkcq3ciNbPu5#1P%*nCFjJAbSB%pZK&v1LN+Sibt z4{2!sIyXYHwspc3T>REMHWTH`93&c!fT?lPlQFS;+LMNJhpfEXx1`&Lz_7l{(rX#c z78hiyFU$I(E+<q~qeki!{J1Aa8{OpeI}q2gmAJjpR$4<|Tb}iMxyI*emTv8DSRQi_ zm`B<c>7}E0ynda+9@w$66&kR~nAuMM^Ykfgkfcm>+W67Kg}?fCZD2<-)NI<};_H6* z^iUC}R$Z5H`KOmdSikK?);{hs4@l=QHAIDX<1<Sc>|YN#x}kdcyO$iE9XVdEcr0NZ z;r^)b=Hi?hziCJx$Y7#*gf5w2*;tBH2GN8>E^sOYKkyb2s&U#CjZ4*{y*X_EM4wC8 zf&f|4pklpJJ7t;Y%Ml(VfQV+<lQe;Q7y+J%!$z_-y-_o;kW^<~3S(!rWpRX4r}uox zQaR~REtttQ<|lH+xJgJ?*_h=Oc5Sor2)I0VTnohzY=3COQv%ect{d!Ek(o*gn#S3b zyuA-R!r0aLq#Pb$izk_keo*^xm7k%sXL1vk<AW#l$A@FjBljmNhYplAR$5~$AE3YT ziH&R$BWJj#Fd+lp8SF*tnKm11TcwokhO?lF(tGH&Zrg{)i(w0@)>_jy1nk<aQX*}- z@Ber!^5}R+QnZ_-h#Hq-aL$Ejh)WTzj&!uGgx#hP!2A4d3#_XSXU1(hI(t&nv(b2f zm2@l^q(#k&+5ChMQ&`w@>x9gwbSBZ0{rjV{Cz1P?b>r=6BXsk9Ph5_(VU~_*IbopQ zP{0VAB!4Nu<L46(?2xa<Yq;BVJ}jOnPt9$VmvnJX2>+`!Fhxf%*_V=vkJ(N*2voFl zS_pH$#f^~(I6{W|qqDaKrtfiRMCYIf%QTi8+AWIpQntxYH4pxyXL`&vr}eeBmI@@? z=g4wS7aZz7^<=Gf)q=2D8}<*`MxRN4<-D#gxGHUZ@-_Q)$ERJT%gHmbXCSWk96w94 zvRx8*0o@*zIAg#>zf1KNqG&dwnz#*DJo!ZwsPj?wCYN3)E0hPAgh+nta?7(mrS@7c zq9(rTkyv3v8M}u4X+vwywf0-x#?$G0+5Ab5zBLtvHgPkRb>7ZTI~-WR2%KS(b`YFg za$u#TMzoxLV1(yDN0$K;0$Mp5ssD<lc0r9OKS8dKvf3B6^0CE`e@GLSSjbC!Xu>uV z4gdI#sRSG+PsG3*qyv=Jmw?0t6iOaF>F9|v$D8aPia3ykPS<d>I)~52k%lca{C6kd zH{ts%xj{Dd^Fd*52tmWQR1V$-!8q-p2#f{DBr}kHli6I&enI;@72t#1UqSliHM5pz z3~XdEt6iRv{9|B}JIs()lj^VS0!P4mF=Rhs`fGTTD&~nlb2=AVZ#5lh!dz@`Z*xNI z946I|SWeM}o&Y`hO=k@Eev?E~!+$Q>zkcr2bCAUcGJOc`9a;kc-jGNg`L+G{`{~K| zpS@$g*KbM!uk}Y?U7Rp@t*2>?Uu;`^FoZVQ$nJ*4PyN;czpkdT6DIvDi-aO&_S%M~ zn3d?S8M8wnw4|>yKJ*hWxyql>yhye0ar*Q**P;+rYPvnrV)G)^2^4H?bY(2ENopIp zp9;*Z`@d+3Kuu?ASXGagecE|0c~3>n4_m+`;&ET>NPYVyEK1;?7xoG^Vvl>DJ*g|( z^&mk)^A_PatRQ$*9K~@JfTWSR_E6o8I|uF{Bise9=z$Z4Q(YBpu|67Ok(UBW4u-Zo z8CEu6+T_I|Mu5kR=ltnbVt3oz=*^|dbyQyNL0K1eAy<PDaG*z+Krskx$K4F1Xny{b zJ?mkKz~C=#2G2#O$41UGbq8wUnb8EjssPgac3n4{kA+n);<dz+I8?$um2!vvK2233 z3?l3CFRCXtpBztmZ~d;xJP>o=Wb#6^{-4h~FIwm)tOp2|oZSua^w4ss985qgI&xvJ z-s5R_i42Yl(c=A4$^-D+TzjDJ(inp=dsr_y-&-S^knYzo>uN6^eqU0P>uA+cFz(;8 zPJ>kobHC{A$a8~y4Bi&noOvZ5XxRtK&uaYll-C9{(7YT!OIMo&PWMIQ)ejHqRB%Pg zG{h{)SALjwJivd=Aos32WyM>%{o~g92Cb@)l!8Nyn6pyTFX=UtZ-JwIM!+sFR2{Jj zy(WGc(E7gI6W`?Q?smUj8P_Iam2o39pn;vDNKy*VR8We7op0Da;Wg^kn|j>t&rVaQ z{kt-aowz3(y;^)yuAd9Nu)I#_G(1Ftk)Ajq8UDI+KD+>F0m>?v9^%<ki>xrdTkE=_ z8Co52ESSS_x8O2`k>l5}WcVRgTv+tNAgA3Y@*SRQf`3dNr;wT+YjD0cQ*bbV*8kle zmT6-rR>0(JW2NQ19&{zk5$bpWeb<7e{!3?~C+OS(DeeSZM|)n+%})n|>g5M{Iw^tb zUw9U?0q$rJ!zZ=1pc*0kT3ZLo9wYDl{sD5+bq)6$-s=&R-^Iu0$Wo+I?sKmHO4a6F z+&6?1snT%mvUX>aBKlB}NDNj<ryLqps1$~zrnd(Wm}tTjdyY#!5bogHMKUNe*rM<? zkdcG-q;ktPa_4RsbFGmJ+HkO+A<;mHSZ``rBY&+#jJfPPGrO__y<B#l*G(O;vfH1G zN3?FVtWCPg?PP|n5y6BgC;{5Yl!uakEGYBuy+qQcNV4Q`2z+{8Zj83x?D&Wr`!M|Q zs-1l0BS}4csODs>7isuwQApML839rc*rOs=Em93z-94ER^*1E36dh&9SJ)gHu$^@F zaz;dfzkAgm*zpOy!o16FnAD#Yp%}>e#^P<UrBUKWlDwNEnJB^1JA%buDEbdZF1VP! z=Sd?iPrBzm?V|)Ym-vRUZiG+t9PlVeL{B?m(XYIS21Fgw?8`G@igD+Y(&AF<ME-K0 z@Z>TA=$>>BqNVFV1t@VcFAnKBDaSEn-YyC9d%AAFgH4f-yIKeG`DkDgd1p#Xq=1{h zXdauX_bU*$I|TOlb-paGYM(jY6)}L?jkLnClMWWfbAE^#4)nBsq6N1XAR&`$;51mB zgw|wlj=ZGa8*Q(3wa6U*ygw-&t`XyXY}Kmes@9g1Viq^n@UI5glvmN+e>UXO*41v? z(%!_R{WdSXrJNah@T<MPU{uEix>WFA_ro~Ck8|c0+1#JXHR=X`jn5@Ad%pjew&ImJ z?RtuyOImx;)Wl+H%nUO3$J)*wmI<Vjb3&vLVI&)hoPcwnk(dfd@=iG(ep2-|Xghs< zW(LT*kEUvg_%;d@J^y+;Ji`F<AKfn2O(m)1n<Mw~8<9oHt|nTH??t+U4>?YQdM**v z7Ll@lxlp!gF7r3VION)#mCu`)Cv_ktW(*>Mp~000pq`mwB;X35DF=1A`Z8C11J_r^ zlkbyoQl@{cy4s<;y$cNdBm@Crxplbs$mh1_#7=7uQjg7q6qEb}KfpC>s$GQvemd8S z8OWxUHm0^C4+;n?e^?;$b|oN~B?{ahod?Qh0kKS7Ae_UkMI_~Z$79x0_lt<E5#82q zf|d_`9A5K2DzLizG-URfXj}e#F`vT7Vld~tNd6g4B^pR+;aWQnc~~DkxjSqBBK33i zDVO}sf(DzRo+caWX)(97jE_Y|p#JM7U-S*D*%7202(t@WJYR0ZU+BU~>JN>o5muH? zeDq9G()X?`>YHT5q03(gX;)J~Ys%R}GP<i)UFq+3!mF=st5bdZ^L8<!9a6u?@Xbk` z&JH_82rH8-<<C-w;?3;#p8F#g>yS~$jG|VgopGo=da3Y;l~w#5JjwV!uj#s&?ME8U zp#PY8er6nhCB-GFQrCio8<LGOwel?_Wrkl^MJ*g79V0WCv^f@X>K~^bwx-@x{||*5 ztb%uag$=j<(BAEWOAx=%mI0nJ&}(}C8P)xGsK<$aLLvdRbgk>TfC9glLaWIJk71aA zdfg<8E=5z4HGB^iZxEp4LV@ip8f}^gZeq~_55nNHXsZC1V9G^Or(R%SJ2RAk+X(5a zA+0(F`dEEk_)goGXv;@aN;GEt)_6qWzW(*R?(0bdW)|m1g!7CQ`|gNaT7B)_c@M>| zk_?BmO2O@g%;Ssd;HISQDw$s9L}6)>_P|s4o;_cGsHW{|bIM<vNl{is4dA?cyzA$Z z4=;JQQzDFJT?Z4FvEe8?@-I1JrQhO8H@<vKVH?hivf?cqs4|?uWE?<1!4#+~D#wyM z5t1hNvbJAi&*Cd%5mH?C|GWTP^LG0Imlq^3U#JT1tJDi3_TMk5?EjbRJ~w|dsZIwx zakK_`d|!s+b8!?UuwMAk<LgHC)||yM_^eg5eRPo1<eTH730l6$Fj7CLOF0OjkU=d1 zB#*`9<^4KLY8B$JJS4cP5AHQE0#nZ3wbb7llzTCr&ht=97~Q?JzmC!xfhH`i?^e(> zp{AXsw5*0ayC!_%HEUV7$T09xe9hBNmk;uY#(%U~r{gk#sK1)os(Jz&{g~X=a=Ga4 zg{hO@9<jBpq2TpU;;8?Di*o}m;IV?uKs*>I5!eNNAny~y2DI6&I7|8@UH_0A@3+;B zhuoJ&1i!=+2OjS83+!6f=et(l-67b5w*TSf+|SW3F0Xw%uQ!tFPMr~OD_{uy{80H_ zMe+OxQGu8{(%xJG-`y274Xz-PzuVQ#&O>q|_?JzVG>99(*3gm&`f5NwF=T1Wx}NWk zGWi*%b8wSszZq2Qa|s*s&HbduSCU%z+ephneIJwK9dBuG<k2;y1!GBU$pk&dAfLA( zQHVTEoy0D%)%4XG64xblp+$Qdb;SQo17C!d_wF+C=E2AG0@2|}<~w=#sz(zQKA=1L zo>K<^{6UuOK288rr)R~nrx(C*<T{v9f`3etDB5+7g1~>2J0r(U>?#TJt1>s=r5?KW zF=O}jd8%>8!o2|&?UtwM?6dBYrvL(Qmp{n9MIYDW6Z>BDvPZCc|8hhDpyLl`zMr8W zG@@m*nw*2AqUB;ZfPG>$gnQ$65FDWp)LNXU9WefpU&uS=!2@xBZ#mp`#6*T|7m9o* zE#SDRMTp$5ly{`d`y0tIe(SxF*IiT2+4xRXZb2~8#ziV=&wLbItp>8@f=-gFk45VD zB8&Fc8WOLm=#HgyI_k<62q)XUayg^`AGlaI=c>{6XR6g4BDPe6<E#zr`_!#6cKb9m zg7n@KrSkMMLSa6QboI%sk+RUTFnd;b63br4HVPw_O_DK9r6h0oZhhE3zGzCB9O3|9 zxKY{w>=mDLVc&0%-nypvo{>fP>CG*>PU0`ezWKcYmkq7=9s?-&Q7{a!!~_O@ZDdVQ zZJUG<bBwUzi(QRpa>G2}!z;pmrSi~gOp3pDlRR@auYMQF)&hKYc@yU`OJ!jYy}>AY z1E9LmT9K<?1Uf4;R;ZoYm>Trl;Xtq5Rsw7CmxRkSZDzN3$``}Po@{Sl-<^uDI=BfR zEq5M8Q7b(CFKty7;F)Hmp$Ej@PW3K@a5<{<=lpXddkOm0kPQ>Va`S^iPt&#ajeZ@R zq~7bb!XV$h+;(DateiB@lKPZ#5Mk;EdS?)mK>x-Gg*5+o#X+9%Tyk*Ftp)U<E=O<t zE5oNGA6B@W^9<E<F@b=7_a+16)(EBOczcsiuqr?c>DPR>cYw`8!zQUPk%jj0mR8uF zmxFqp7EQbP-AI!E0%v-9zkRr_;KS1o3lH~eT^JP-Nj5G^B74c*NmDICisdha>h(g0 z#`}#DG{XufJ&Ac7U3KE7p578vr_%wU_*Zld<{;l^!xoy3ryCA&g;u+h0a^h9fF*=N zIIW^V;^mYdzGu>M-g7MABjQ2;dN#-y?i;K^@X#u>PA|!lzP(+PIzz|lScN-(ove6~ zc*;&+&~nBDW~-TFP0A?PdUxU^uY2AAg~-c)aQy=2oNP7j2aiIv&4CmGEtV#1`{}>+ zwZ-gOUR$Fjs88}Ws*sYV&~n@>O_0<?R;7#|HKxPsXvcL9b6>yefb~a*V=me)ks#kw zGhE=Gq^hB8ty$L(d;&F9f{`*cPSrP?0|s>a+XL#}Az{Tr+?iX*k3JO-w8e;H)4?hM zqM%D$8^iQF^l)Cv0On0dJ{dNqj<#(~UN&n!zumzpIVTCy7qIHxf{_$=2m(wkOi|-9 zM(Y^ZtP7P6r7#X~)ri@Tj;o4`Y=dIFy?{B-YM_9)@IOG{C_XL#vUfH;L#I!d8tz_z zH3@(}8nb&3!r6~Dw@|%gF9PGR)r;xG2hZQhI%ZfOEcZbDJEHLxqbkq4<{d$zEFsCM zGUuEc;3{ip%f_W-XEig$9=VQX^i9j1HR`tz<akYdSxd6>d(t0|7re^yt2HYuz}NE3 zz*$y3?eg7UxM$`NYr4mZC8ida+J#h38<W-xuDl~%&3?~|YJ5=nZhfrg)za-tWP5SP z30}s8%2LTkn|*k}=|2Iz858<qHQ%;{$%?4VTp)5OoIEJiv-gIu(iXJH1h;K9ibhw= z$^K7lU(i3<@vHggJYR$c_FL<i56k1e+aI02H-;5^@G+82N0+n8N>lY7i&n79_SaiH zcBX!<y0r?i(4#s9Ihb=Ly~$&^wNg7BCv%$!6ToO+6_(bqcxT&r3!93sgBC?-O~ATh zT3^tP-n>PgX@v&rv7gURNxE#glhO&4Wg3-0`%0^?X6{N8Ukt4QH!ph`ZgLq8;(_~4 zn~G@b{!m%{K{>`?eB&-=7fo6&O2r4$GuIPAP+VI@JT~0bE>CjsauGHUZ{T|tIgAxy zHfv<XqIbT~46x%vVQ3h9r{c2hLQcsP;1S!E%i3wSN)6C;UamVNaK!P7QrK>=Nxg70 z+0^K^0aBwc`z18W=!g{b{ejz&t*X`%7Lo${{pu0BBB2k$l-_CwlM8of8#|ZA<f4<< z&YtyWXi?6f;QP(u4UU}YzfVTIz9nHAH4x&Uw^E#(=(S;coPGHF)ohykqhf25C*?0v zZ4afrKBmR+1of<s13r9do~xluA$q$2|C1L{TIUj8%Sw_8H@8|ADnGG`9qWTbPkRRa z9jJgy!iV-k@I|z@khIHhbCF@#*VgUIH5!Q2K1b&HG6Zf{rfFiUj-))vLsKK4QVbRr z5dDgUQ9ETzK)-}I3MmI!(VxV=ahf*yRLGu+l3cqRJs)3wUE<05YyT<yeb9E}4JR&u z?gkdUzkpl0yhVce=X^bBFzs5?-EpCgiYX<K0${y->}{Rz<`V9g9@V<EWMb`xzTnI& z_m)wluqp&p!DMVIVr7%_{CEf|`>KjI45%1q-~9`@|BT)J4t1BvS}BHNO`pi)2Y;$P zD=ClPW5}}b(}2inhPVKT<n4;hW`Q1({8w57%}H&T5t`?=iw8FSqUjYy;iOp|Q~8Jv zQqk5Ak`@fIk-e_Rf^H05pYqqU8#OBFIC3YyOpYu9d6)I?6Pyz0<4s$Pn@v0)?%2Bg ztVs?y+js2|zjpxhYu9Phb+p~B*nM^$x=1$s&&BNdA$m1y%htF>QQ}c(SA^VsbXM6_ zI8lU0R6Uv>E!GA=m>;GUfXQ7ZP6a-gH)iWrv~I%LN~tq9)J3HU&J4><V7rf?VliTd zWgxuc8PP8gTf<Y1mU;{^epJ)G;VXGTJCDrc=sy9;WU9+YhOY)MB#L<A_mYgHZ-w>; z6XnSjzViwfH3;QIs}`7X)u%8VU2|!Lh^J=*&f5LdD(S!qd+8vznVzUikv9!ZKyLP| zA!#57;_7=C9um*<SD8J?Q1*xG{FH8;#+SEJTERQ{uB&!`g8smtB=D>wXaj4Lc{7v@ z(jRnbeMriI?I7%)&u01*J8Hfcu)EKCBwJk>hEuR`!md>CN`HpG(K=-k`hDVjLil_e zE6y;$AB#GCLNZyJvmUtsi7Ivq{(XwX-fchNeX}#mXBh4{J3lnhS?+lJd9qOw8Xb$V z0-KI~OQOnWLGHJDwc;aom+W)P>1&_xlE5NqJlBx=_X^|7;ap;gNBXouuf$!xKf-@- zJ&P!xK|K1qZfW<<;&RUD$c@e;0%;l~xbxu#ticCreLiA95OGgl*9GRkv?WT<oTPvj z{-vche|$4r8)3jWx<yBArakT-N<JT{5!3Akb`g;O-hC$_nfJHNg%YrbQIcp{Lg#=D zp+DNIMZkkMI(fjb2u(qT?ejm^9)V_U-H115s)6*%5r3dZK_z?2)mb1fGkC2|?M5w3 z6?74Af?tm1tDVGlXAi$fVAmQ<#Qyy92Q2W#V|z)6WqTV9)neDe&b-lHKpmqx;i8DJ zm&TU5y|Da&vWO%<g;p|J=;TNrTv%ZeQT8XP$${g=tLCzc$+r1hHs#>b1I{!+QAGo^ z6tI9T1-$4W*pu|<FTWoB%(vVNdQj|i9N$k%kLmXEb3%*p%&8?Lwi9D?-I<5gK?z!G zdMOC8lW(n&-YHVcl&Z6bobO3<e;{5g+$=@u2?#ra3cCmn+5b*X!_^S~rW~Xgcz}NX z9_+(6tDaOzmX5~_!H9xpWljt*P11TVn_f)96pHwrrcoxZht_Q$pM-=FhTk;}HOOzF zY5pIB>j&n%rYXUg7xDbps%!yfTun7vf3UuT{purtqp#i-PxIv0>H61Qf5{z_<nygd zH^TvTRY3ktP4!oLhLmwTrEp6=Y&}++yahY`27R*<S!&w&U^Dw*<oOLXxROF4k4XMz z5I4w;S=mU@3cPG4__v7G{p+#V_ugySZZ6frC_RV0Opb?=65Tx1e`7M?(h~pRO68k7 zda6tCS-rc>Q`-kBQEHoSJ561!I-jBPx{wiU{_bBpAIA&A{zyn6vvQPO-&4&(ys8=0 zyQi)kmj7=?l=Ue$$B+{P!DXkUI*WUfWt6Viv5i5sk{y%7_=^rrcTKz#DKo+b+dA2S zj*R`|pLM5W-3PypNqxcI$v2Os<Hu{Zj}O~06ghva>?teEmI03iCjMuDMhYC9W?i*7 zp(F*>2Mf5lPpRkrSd40XdjJoidCkFu8TK*w)EryNWMW0@dMsZzFB+-fJJ>UahnY|9 zP5+gIUx~aKvY9g!!F&hBOh@7HqE2<`@&3#DhSK1H`xjCo@yn!E3IdB)B~ckJot3+D zS9JN*bEn$EC=64umY1IX2YYxf&z)4<W7@hTM^^?gfTB?dHwua~4I)svg_YE`D+`Z2 zYi-{8_th_Zk))d$o#db(eJ)LR)iZB?Wo^|d+&RwK*U=TcK_Ls7zU+W%{^t?%aAHe2 zq!f06|2W49cjh`B;LYY%{MJ2HGd()7RoN9}o7|>aU=LpsB6<D_xIzv(Z!+f6$k<3S zNc4k{miB;GZNLC!nug`ZD#HwY4~*bqx*i$0Qw5j7&#P;^WdiS?&2rf{5T;;8?@fKq zFZ**Omy5wZCErJRgsnm_X)`~bXN8Qfo+xKC0h;;2&>aA;0g21fXiWGTPnvqdD3__b zVy#-6g<KycJ+9j;^4VAZ>BReAoSvl`7)`f)>ypb>-@M70i?{y#?YFu3n3}2T6#OAV zM;oQpl>*VLyu};UUomsj5$)lP=f8xNGN+#ywOloG9qC;&ET6op3&tuIxr!O(dL#e5 z=xb{Ty91;&V!-+et$|+j6S*dhkM+DyP-PqX7Yt}F8bn)hH}VlXvMOLeo2`tv+5ST{ zzrFAyknc)y<MH6^j+C_i(S`d_uo9^CwD}$e2!*u0&5cPlkeI6s&rkfV0o${Zx7nIt z>YZA(icOHyJNd<UK16!q?h{_{#<w5N^vtFHjKAucLv<Us>2@Zd^jB!K#kgB_SD=2i z^P(q1hg0jik*Us=<gU@3<pYZg{b~u%I@f>6CC{Pn=Vj?JV<J1H!n+n54M2o>!<pkR zWhZc4$&S8fISn-`oA{?oIF|3MujQN}#xVvyv`d^!^eRPkgp}G%90`x)mQSVh0*WQu z3y+(m$|)Fy(6xnGOZ`o|8!v9Cr}2f^jj{p#Xvaki{2p}tm;XuRCML1|9VX%XlVEoD zxR&c}vav;&pLpat&!oymJz@yxFsmAxBv(2W&kVyX$)4z4+JHRj+z00l*wc-^XJ14` zET@VUBmW2kAR%x)1sL)ec^nURBvW@Cu#&g(M)!R02kzNJo+YBbk}3^S1Ey`FPRrzO zWemI;;_J6V^w5nXACYp4F8g|`AeMiZq=)B;HrXVFuTz&C1J!)>ZcA(KbZ;<+a;CV_ zG?;XP-+GrL>l_GHE{^}mrpo(LzfdghV&kZ{PV8|AU*+b$Zv5${xVa{n%h<#CoRp!1 z@!L3TpP{3UZc*+Zu;X=2DLs2;9j4bP5&&vD*p88jajZ{X9RJvLK#*-b@mMC{jO2m4 zyslBFj%vq;o$l|1te&p&0l3yIIf*>>8=4*u?n}xQYOB~b6Vp8nRY1<lF2zNdoV{?n zV`tAOtj5v!{$4?@>cOx>rR~`-ZvaXbb_Q1N2gHK_zi>q~k7D>awyC1cp(h0(@&40X zN9v@qz_lWh6AFsvj7ctk->FmkFF+-C+K1>`pbm4x<B>9-JIX@;R*SOgq}*l~)uL4z zAalPH2*3sAtEb0qiD3rD6<p(0(eQZ>vtRzWV4Tc=u78{MrXJvk91{3CfiUR|bR#@c zKM`G0jo#c#La4Tq`=XKQ2s4Q7X|jFLr}6;bgwp&Q2aGD{2AC%z<7NF(s?o@dg!JY4 zyAQR`T@W-{5=0%3tdL)L$YL*+1cuwUk|#<7zk-_WZjm!GEMSpXa>zJg`P36tOMa1| zkX<%Hms81Mvy5BRK%xb{S=>~cYK=brS!uF7YOPXECdTSi_p#6UC9c=8-^yiwrSZ6o z>>6O+Ts-mlu`h?(AL7yO#?j4-*xztPXP<=nci<@j`nYR*1Ffy5=-c6KNlyQ@oEt6> z|D~{W8K267lJTb5c@-mXgUxT{j}{(@MqmdwKaKHC3;X;?7m^PVdZN~&l}xp_^nGOf z3nv$7Fs^)bH<(q^uhLgE#u@DK+BIkt8n}_(Vp<@Vi)?fCR6I$#H{*g$K?UB<TiJ4# zbJ@-lhfdTKVG<i18Gdom2{;F8g|Wu0K0_${WJlMfuD<4XlaPlub*dtlUwE>1nq@y{ zw$<bHiMe=5Y5#JfIyPs4FK~mlRPKy2krO(_9Ng=w!<(&{`#f9C`i-z4=BLXp|0ojx zWAp@oO@{hKl1%2O*#Bt7jc?-#LTLmI815R`K$R~I-Tt%ZX0Ud{DUmGplf>`b@6Cw% ze?|QXZ&3AH%FE9X$CVzOGQa-D=~RcqX3&vHY^P--z%$g<Xmf%3pAqX{D2Y6>VfryE z$hDh6IQC7Z+ErA4wid_TS{|P6x*e>S70pA=&X`m;@9By(!@pcg_FH$G5+v;+{~nVj zp3;pUQVl**hx&xh9{6&3?C!%{@h_+Vmkigj30L>*5+jaoaS32jx-B4MFJM{*XuCK` zk*w!XWsy@bt7~rjN$=x20DZ*N$2xk!r-$40Ji3@LaCI(C-TCn1G}>RyQ3+In3gLdf z_O0vAfilT1Ug&v7vUKe?&Svz4+Fxy7yy(nA%Un;Fm(akMyv~mjcVYHjPQ^Y=C9Y4| zpV*EZ@*|NV;kHYDf1)a7%d#KH2a~tH6N6`hTs|G1K%994t&b9AY559Ey11l`S0#$# zWgLK=-=;E}$~5o&$dFwuGq9cq@`W8?=Pqy=Em@%diSDF#SoYqnoyyk4<B^fy0@>KG z**jf+^(Gn#Jr)!C;0VvVHib#|8|DNj$*jfQdTk7Ynz%EDNq&|PshD@3IGlK%qWrE> zEj)v{<uw?aA*ps^Z^*~61yt2w^3AbaV7z+(pUHh|Sw~^^3vV<`=02C0X>pL!W{uw& zqGSvQ;NvwKHWzmge7s|WepfT-Xef7JzXN#;c_!j^%%}aI?)V<(%WZBv)}B;{#lu); zRr48LQ7f&^8?P7xT(|A#N5X}oBRZGBw-3~fwf_)-e?LgmLAz)x^oX3n*T;v9a1M4) zN<-PJi#i%@e%>8HiA%TlqQn&5`@ZultWKYH{iU#K$bFUcOJDaR?Q_Zsir0}xehrb? z>GDd?+vOyIwk=;+)Tie)aQV#cufPZ_0o|bM5IekKUQtK7yUWin9T%2CJO2|oojQ;w zdP_-|Ttj<;9nboYe77~BKR7pn)>Aq8PXqMe*wGv?Tl4p>TsvVpFEOJEC!XiEL$iKk zZ=xT}=#?(>6^B!Xbo<NnW&4eXiO&5C`k{T{%t7X*7V>gvENWC81k*dPybaQ<bqE`T zOGX#CBn(8}iF1nxjag9Rf9Ida^VvoR^L+MpbV|;5g;AmCOLuvKarE@&5rD{d3}yPQ z4?N4z!kjU5o1OPtuhI<*D`ZW?%cWs~G3h7rs6z5jmoWAkC#w1!trmm*K33N$yij?# zgAPOyat|EaM`<oW*H=GT&j$U6@p&NNm(7Xe+bvV8_1_*dI`(h-f0i(M9e6O_a{3lV zhjN5HePn$t$x|lvTeuM6y29z#e!Se$JegY7Zzg8!)B4I7Yn9x6v^p9;|5;+n_Yh5> zh_wYpNCyGdz`SH>8pmJ2vvK<X`$D*xae?!ThhGBN&v;3d)&#9|n2Jq_>E18GC^Lun zjxX!DDcBt%q{`KPDVEhmPS~D)@$?sP#Hk<)L%PQ+ch#Tz`N;p?Lyt=?-a2fHBirE? z^6fn;;yA6T;p)@3&Hl}9Qs>fm()ON}@_}*9nQRnbrf}L=ln6Gy?{c1100DazNiQg> zIfFhEbz$kW{T*bD)6;K`+5}zQ#OubUf(n@&@^Fn<4UR}T1%WtWb<!RBpOyqGmeQ67 zN1tCd;A}_oM!$8iV<jyWO?8)f{ecSe+n0S$cp%W^=PVv6!@~t^h@-&lb7&~7{rx!` z$2HzK+t0$i9xEYYz_K??wbH#o&twr}GVKI?*?-#4yeQh6bf^nHkE(&wQiD%SsdX7C zitXj|{j^bKQ%sc2Xlvl7)Zkv|_Dy9hr&Pq&wF~Io>)0cbnP&vHYEt%RUyr1Kd7_@g zh{t;A=ht{H%Aa&`xVX%-3eT_B4tSr!)J#e5=>@DkKcV(%<)v{F0?Ae1%-hs|3HiR* zfDIoph5C{caxx;A5I6Pf5BpU3RYTNrLkop(Zgx%pM!4^#4uO?gwb1aLca2UymKS61 zJHF4mS4>KnL1FvMPfgw2sGIBwS$r*vlk_>;G=i;&*H;XAhuGY^qJ?Ze7->(JIf)nu zQ-qmx32CPGP23~fcN4jB<)D8G*Tp}&0t^Q1CI>@#qa?$*WhlHn+HIAY6K~j-8?ya7 zoO%O=E8`fnPSH?o;eQU1((G;!h_0WVS1t+;0ezpc!$>APv7t^_b7Zb@=6mdz)g!J1 z4se>EJu*bkwPszBhL;Dl-$Udj%X-4A+|;ml(^oqg+z@2(#{(K^s%h)sf5uFP>TT<^ zF#^PWu9lNW!S?z8p!1Uj6~Js~Az^bWa{tr48i5v}N689>s1~UXlXJI&^X?>A{pb=E zQ58&gn5~rP9DNsfN93+elG&`zN-zJ<Hu23G-Wr2)9l;^@$-30DOG+RmtT0ulX{08V zjy%0@4vU?0bpwh394;IK)2ggh<r;*Qb|4||t(zrDKYCid*x2t~clAuQSK@EVto1A= z93JMII*fP3nBc$-*aZUv%$WcR*}idSWgUem#@`dDxCSJ++XO~^`~K<AHw4^3KydrR zIr(GHZ)&W?1Td7&v_qbq=;pbDIGf!iZ*bF+aa!=T7tl5)W6u$B$jD4lzY%^&f$A(E zXPx%o72#C)`?-!&hGs<nz2tW7mYXxNsp;WsRO2h*p{2V4ry}(~Sc!ofaW5Yrm6x+v ze_{g-1{tpp3>etq-1RRC|D_(br;oq-c4u8<#mrFpmk>6i;LJuUB8Bs8I<>mshR&H( z7JB++^aIh>P&v|~2$^Bbu`IW&sOpuPe*$eA?%zNY@g=+jj7X`tEB2sT-1pX9$hR)M zoR7iJ?{D-L@QU(MbN&`2Poq1Uyy??FBJvd;Hgf(|Ig&wfb##w}&3D3ITvOF3+5<$| zxQ7kGX_s4zW3`V*e1J#BEa;|IUII4xHI4y;i*;{z&GPgrcL*v6yR-{#B*>gAYww(Z zJpAv^Vav6+4Mf&gQf?M{FG_WG{!@w;I7E4{Y|(<%0SZKF^0vWJrxb|2C3Q-bcdF4S z88h-$=eEty^oHsbn)go+n9biP^qSe!9Z)bwcM&3#ft4&QIKW1Gsc}h`735vCtK%PE zf0__DDuS;7s&+cM2J$!zJr&!mji1^R$P}7oLUnfc)i9+RZNbRYP87GD^6Y`-E~|`0 z99KR->}52W9YCD59;bHp(!tX3AzC?YMk=ZOAeXE!rd1EEZ-mMH<!kg?+)I#ayEk~T z)@}`5(!o%9hzKved@I9r)mosU6<F-mt?@k=V9BWP><cUlHWdk$A5~&^G_ZY=>=XP{ zXI;6u6K!sJSa~J<;ktK2!(-NU6=H+AZfvdtHPw=h^AhsXyVL3&S$^0%Ci<=D69=q; z!=B@$)mpN%3itTPx9n3VBU9+e7XgwWyMb)tE3b^t6tHqw39?u?ZCW%guz__aJa;KA z=C(T?66xt3|Fh~Hd<C9Bv)NZ9RgOb|zkGzThv;*(iTIRdKF{#r_iTgSPur+cY=qT6 zlhqeFO}G9M+;2<jlZk&KtCm;-XSV^}j(C5~Si~NHvg(RPFH+OvtKBs2IcCi7wjHF< z+72kwx0)!l43)k{?z9Mv@hTuD9W8XSqGV>+eN~P^9JNUKT+TCsRbnlNCQ^n*9zNB< zw}@HI-_=*^cLgwN=VG(NrurVrncuFEMShu=wOP{;rGI5mKh*Ij**<)9m@waZ%1f-j zAib@$;KuO9yUmk5x%(Z@MUzBp9ILlb2m0XU%Doc$3WusW^*@8a>PlSm$?^g}jdF-1 zu-b!*t-to|{68<i>ZVZiO`k`<qP3ysNV77Dwf)oul=BhD{5Qdz|N1>RH+A{ov#vT_ zk|LQv8#aja%ZV3^^Yo}DHv-A04cV;ACthl+Lfz%0@?8hY;TnPAa@+I2+taCeF&Bs8 zPun$Xy4Go9Yr`hc8poQ@`^4P#fqPyJ0li`52q#9a=|^FWdR>S1{j47&E?4kFe5$|| zW^6OWxE^wN7p-H?>@Bj_2;2g9AyWLhD-72vQo&v^SiqFQYR(qM%{MEydpbA>qw+{P z;XsZMEA5GausPhH8v87L9nGlh*dyi<CX=-TI<%S9UmshzuX1Vw%^zOV$E=c)T}aTX zsHt_mvMaFhM5C(_2+?b?oN1u@wRwM!$H{x=`{a$JA|yr)=i%Fgb6v#o2~th#JtFet zPm}{xXrm=~`3e!y=v|)wcPM9N{ode~c?kn{_KTL-vbCap+10OF9yT@`yg-l-f7rQ7 z2Pn(w2)JY9^3~~?2tbTW!SXL0A{;e}!#t&0asT~Wqd6LXD%ygjC~^YrVlxf62>e@f zwD111{SKsjrllrkN1yQiG}|#!^f+u}W^_pq?4hS0)XVTJH!PZMSll&VB$Uv%s*L{x zPLp*zy0Xf?0i&cY;d%#bMSg{g-0}{_*KcnSF4Yf@U@0`-bIgJlx_WYqOODpr^SQAl z=}t^JN!v~*UYQIXdDwXDfJF{m^Z+1%0>i3zWq%6{>gKmf6}7wg5qS!g?a3b;fr)M1 z4@m24x|pE>$)W+!F4j6@3s<0Lx7`i+pOLj9jFM2bNB^7KT0!GmUuPu_Ua}+y?}ug} zT84f+q`R|nRULRnTkczAe7+$_UStSydraKgp*-V3)0F8}ozd<G!`D6J-uSsyj(d>f z+*W)iAocQ2H*kXDUyg{%`_Pm8625o2LAcjftPg**U^pMHXiojxRhEB)uv3+FXMYF_ zxze5UjRhmBP5HDZpWoWCs|!v^f?bL?^ZT-&J7m>JeFhv1RE#@Gf`Y2tcB$5F{u3fr zSEq19l~&aJW-2+dlw*P4Xgn6Uu67LOxN+V5$m@3x(sQ!<4bU=wlbw9o%lKYIzAFtk zr@~f~50r@1*W0k;f_a!deM?F+u<#8e6&0~HMLeJ8Mg;G6Z{V{yhMcNfyMH)Dc$)o; zDi<c}7Yg69(*{U7Rp6l*-<r(d;bRPPsvh*&S!`?Z{}SH5M!e8-8O2s=&r8!y9#BE= zIXb}CsS8s>n5JTBpV-U7fims`sH{fZ{@z^Jw&&E|ZtOcy_x8(9yi1v)U<*FmHoE*| zh?S$gx=x@!_1}d3KLq-3a$Oal<*O8Va<jVZX<J0zTb_~*Kb|syZNKZ~?WDW;pas(1 zmd>;Pqhx%=2PGb7u+tt|Kc*TP1G}AAg3W;&RdX|_al4lLcefm4vM9UJ$;1tR4wEHN z3-r{9om{t8KYgoh_6v_%NP7bN4k&{Kgi1eRM7$n&Bn$0J@wiYwP#G1TP&al~{PFHa zx7G?da8I}Hh9)Dhk69NVX{8}{GWq@t%BfjL0OYx3(37hyzl{wQQVhzO2myE-NxJr` zo@7(`2|~o6qrpPIG?j``HU>tM$dlnUOaDIT)Mq1ZBX)z~UjIKsd!R~WpD;Y}!G`-S zw<+t6E{lv+s!pG7thdMryyijU8-+rN$F&c^fsVr%zO-SizR}xJyrAab3E*v4c4SU< z#BttUS0-_huQTK#la~o255){Moq<TthuI&ZrqiuzU?0!LD(BIMweD{14I6uK6nzL> zbaCY}=LM>M#D$73-kIZDt$r4+LBD*kU-je77Z*<r&IlegKTKr+n@j<0B17BbD&Ag; z@uBnjxo)$a%TvC<s?-7)ebA*THCNLvQ7(U6L@rN|X~_Hf8<T&jx%wmfL+{9w(|P~+ zN^w5rAdSm|%mi(|m2<NItxthW(r_A+Ky_QTWUNE{DJFy|_ZkWhYP-bl1K3K-pHRm` z@~PWM-k&Kyr|!y8z;2!^ebLJSm3ls89naJObx}Q~yfL8GXjkt0qrK)P%XGx#Bzcj@ z^%Q7kK>^Jy+}F=KoAaIYT-lGE^k0#t#x+1w_C5FRcD8wf)59}ZuMcN6yqat+z7ckw zVn0zdAv9A*d}r^;D_PB>(7qWbZ9TWz5F$4CMAN=1uSYN9Kk`79zrDl%w?LySK~A*N zf77$=mmr=SXTMm8eLC0O`f~^}dr8m;%$e(O?_ay36JXgAixA#@dt2)6OY3WP{B5Ul z(_<7q=Lzw@_wEfz6HyRIr<JjLUUhPZQ=Qj%<>8>7lCodUPr?s@Y2zIc9*W&w`<@)8 z{)U=F$q?+dsqF%uR9)(As-Ku!pMJejRdsLmFF7fNM~mj80nP8VMxn}^%Rn@fvH5bn z<qgd6C{q_YGKQa4?)sQnaw4Jg{6xxz<fSM|0Ur({JtT~necpg{??Cu4_sPHypgt`y ztp5wPN`HB@7@GVKNsfJOIsNZ*zY#YAx%D(Wt+Dyug(Kgt;6o3|nlYGxK$Oy|nNNNM zYasmKw@r<-Q<*#<8;YbYou2Z}$8lYB2ikiTuIDE?!!`J2Vm2!h>HLvH<tNS9C%$P# zyxcB{vS*zjDHq1@$RA7UczDe!RcOD6h)*=&1`1kc9JDD0V%_;y4Cv#>tR(4n1CHOj zP(c&I={Z``VBD3Y$H?QrF=Yp;kvL*ZYHaj4#HGcbOc&<Kulz5x8>J`sceye<om}!F zVg|0L5BmuF*V4iPSKi!u{UU)}%sYMl-i@@s$A_${m5VDWCcLKawia~C?Sf)`F!g-u z-e0UYYm(+mn){OU*7}l`6BdNXkfYo}DbL2!!|T?uoN-!|Uj6WBtS)0t+Bxmr<*m*t z+->lu&g>?$yy~`~kJwul!c!+m+IXzWbgFpMGSL#v2jBVFCx7YVZcY*Qm5^G|tT)yW z$8|>^)0Y(3K&42$+?5RS{gu##KP9WIW1jkQ@?rlE>4U5~&yW5Ze7ErmAp5!x)*KUc z5o-o|mK=irVJO2E@jIIxeW+a_zB29qyML~G@!-&t>n&3mp<v^$zRZ@n7hx-GR%LE@ zp?6_u%0uJ|Om-#efusEUM*u}|_8;i$mUMsR7{i%1b9YqwfAxNLzC0A2pDp@&Twt?V z^|N-zO-IDlL`D2~Bh`#A)bhkNSDfo|zF>hNCmp+xpjEEN6d`qe$cADx1D^hxG?)KG ziYS06wgL7+K=bJ{5R~_AO<(=79@}Rkcrr4wzyAcNOkPN&X2wB}Vz9t*BrBXKBe_zP z(JDISK47*}f;&HnV!Z53dtUWf{)4b4@=(gL$fR}q$pE(py--nBAuJF_moe0KYHnvu z@1R;HNMX*e?bb|OB%wdg-L_6mwHw3z{72VIi+usPt*sj&cT0M@taNs#v{mID+YN`> zUDFZQ&;P9Dw%80J8wWdp*MhGqZSnKl0Y!NVg#l+c?pnF43gU}(;2I6Ppu2(ML#Hx_ z$_)hO>^J#2T~-7~?kGerd43zCAX3S8d1}$B-wakm4)IU;&@+h(aRr4T8DLisPYkNE zO?iIQwOyQv4~4k?E}QvZHDLIpRsc2(D`n?Ut&(IEW(On>z|3q{8=s9D7Yu<j0WD>= z({LG+Bls66qHhG5aeu0bb`k4v=&|p{6#37^Yll-t#M8Ww;L(IvK^jIyyIM2n1>|?8 z)9Fn((bw<_nZHBMJ_=)L(DrPeqtxJsR$QMmlJ?98)^hcRdU^pWoJjI&E!oL=fGxWE zpvP4%TE@V|yR~vh`6yG1B8|Xp=HybFVa0F_<j`*V9%R-PhM$YCRM@L(A}cX7{s?Ai zq{=2wIS-ir4Y%prZYzSLQWT$JM__t?<4Mm`sXDT8n9nT0;-pE47`sCbsUG7)XKv-7 zHJO-D8&moE{ILsp)|IWjQ+rJf27JPD`pY+HS_I`YB-Y|&-&T3`%x4hiS$Kz+#dS9! zpHvJI8d2rpB;c3KS^kW_2|jIYTrg#Tt$g`<wE;kDz?Yma_O+DLny0z&@tJdgYu=LL zF!A8uM%`f`KA7PjIw~UwrWxJPQao8J75C*+GPmT=xwl#XQ7CWo&Q(L~6Cpl2O_F}! z?8GFtqU@K0cD0A(6t(9X@%&U>_1B0pPxQ)`EswnClKPk$D|_}vU<uhClYK}Y;zxun zPsiwYC>|y2I%{s3gmf<;?Ju$`MmC)sIXj9S7sQiY*XB49$Q3<P{H8fWPF>D!*yL4@ zO2SS2{JtVqdkS_#>1fF4N3n!|OSY)Ty7hnfbHk>COJO8dZuznLx32$1$E+XWDIc}{ z?_XNVc=u*;<~sNDVRm}PJy>K^>Tb?4G=Kkmcyu)IEdur$`iV-v5=Cx+mEe7>-WJ^< z1pH$5LtLtNip{BuI?%xDmwnj@K$i&J7aJF>>in7YNouCzToStZ&0)MCX;?xZYUQ}5 z3Fj3W-0OCY<*|g1<pLAF_k&4E4lng=9h{QUe}L%NarNj<FdN{}Rwd$^7Ia_T2b;0R zV#QDW1V5`O_CU$hJx%FZ*ZWgFim%)FRH;8QNC<C(9{>l3%FS~Gt7gHM#?3aLWf92> zax`Dw;<wE|^=LOiD5AP*2W>rbZ}2es^a6M~w$KB(i4ZzWoRH~gRH(Vo<Es2`p?V>I z+vP`^uuYzJM-J3PO5~>E588Zhug0v)PKE*R#p1n58NeO&)TJ5K3au-D1AXzqdDM;! zoUZXc(E9bB8TgY=`PTb^!xXLm!`6Gpv;BAB!?D%YrDl|Bsgc&K5@HmswnkgEX{kMm zT8R+)HCnp^rFK<~*js|Oidr#ai&6<SVnuAvNALUo{qE;^?!U+@@rvYgectDs>s;qL z83wJKCNgBGrO2D2<naKo$KaBk!@P6-5mxP8IiE5ag7n)`eIf3RPxc#VPV_OxaeA+u zLmhnMsoCsR@vL%KA_3VdNWAQ7nKFw9-Gic98Q}Lm`9JSyAO4@K`4^w2)wD%N!p(nn zM|M;UfJ4?l1J^nG_hMJknpDf564p<7Jr0-L=`FVXsVK^IzP4XDpND?%(OXgBcL3|X z^I;kHkQTotOsm;huM3g~<_<&AfFX(Sv-3G!RX7v%;c{|ppvM(=-NVceO7WRAsQCt9 zx@9Gw($qgvee`-Bj@1#ibzSMBCLEjz^+YR#sg}XWk{@m)oSPF}E9+KCgle^O<v^?7 zugRZQze(~rxVWU0+h~2fF#hw8!;j$B5TK_09MYZxE;G*m2AI46G-14d15?)#@EJ^b z7<53g^Aoi;-bRS8rA-0cYM+4wxhE`M;oV002(m0a<Y+fdE0CosYb|~|mWOc=aVEXl zkU74+##UgDm)6*P*dNQYi2zBwWXSFM1$3aRuD<c*&Xm`hi53H#r3GW_KM|Pe->pZQ zXjHc)aY-Ix{4XLGX?Hsa0}igMiB~hkgFJc!3qBT(AQ{87i~KD1fcM6_wxmMZS!f5g zW<#oiU(9P4yyD3*eNAJ)rn~8j3$d&pms6yMs+Hx`Hs6NoD(l9-T0oE7b0B0l$ur+W zWItAilgBuBQ}4Y!0`#Z#+uex~9YP``*h*hygr3EX;DmL`O+K{Bl2j^#k?2}Zx4Lx* zBt7?8i2hy=e4%aN<^nn&9TCqh=tu+=ge#qwb?sAWTkuJqB=tcd#=cclGf{8rKPzCp z)fu$}tX91KF5^#&W&et|Y6N?53W}(LwaG@<T=qcwY7#bkp8)g|nNB4MZ@eYvPaibQ z*ZGmOg4@iBnl`5dU#{62FxT~6LON`fAf1^qZ4$7e4UrC@Su;-a{!HHA<CUw@h+vZZ zxFbfrl%>Nu?+@}nCDuu=@BbEDFjw?r1~c{B&8=4%@V5Yvj{G?DO!x0gyco9T4R>|v zfphem;TTI2A80#H@_m9ZKZUfS6`xL1&TK8p9Ba*dclagG?Z@%Z4Wu};MD_|Zs6!35 z-`^tYq7>^|{p-fsAHbsMC*xU6%~M4_rAZ#zk6r7&%Ls+H<AcpcPJ<QuscF|irV|Nf zK8_!sf7z&TM4Nf>ys}zkvS$5~1gO*6`uZH?gB^Y}KlaCN{@kGu5mUMjhf7c<GbULF z#iuMNf+M<_4U_&8VX~}ya0NWw^1hPt?A6}Xodb-i^oRk?l59fFfCabDp~K_z{}N_m zS)^z>4piQrh~D^hu+(J2VDRcKb)S?_Jl(}DhMtajYawmTOY%qjHivsX>rftzN*?$# zh0ol~dDXT%w^@i-E-8{A4YwSq?9r+B#2K_p5^q}dh9==4*lhlFQ~M)PEc`*5P)oND zSR)8JvAb{m#kL@`KS;`HBDeiC^Q{%EVZ=F=>AJ)v`fyy{@e9UVzWLj{QL?o6T{Cs@ zK?X04y0?g3D~7^i6-T}qvA^%S0-CFH<reL!@3puR=oqpP`)Na!j{~cd9aYx{RPyW< zhGv$WaKnC>*;yNx(x7deXK-4tOfH2`1PJw@oB(h9)c*T`8qr;H#EbZq)eP`(fjC@* z2{KhQiyh|7JcJyf6^1PgDqC(j>z@r9-uJuAq=&$iC-i?)Y3p#5Juf0$9HUGQ{CRUz zyH_7;8TXrGO4pw78W1XM#sTAj8H4CeT0Uy-;Yd-lk_}}xJm?lu8`+lnOdX<s59^cc zGmFE10so%9GEq^Yzht!PKkb!BV2$a$Q~Z`E$-2|dTgLj3A{3Q@C{9^#<K9ohNdk}6 zc+hHS9~9nggsy|4+uC2A9c=yI!2J)5%RE>fp8M^)sW(>n>7<(t{tNWwAcn0mEo<vo z6V4i8;fE5dFUK!(@x+T>l&#`a_M`vDjw^b(iI}7XyGr|jGII-}msTJwNs4R9A8&S2 z=L_h5D4nz(7WR%8lq}x?3m70L*7cXim!z?6saI~eeA>I#V^zA_ow^xz<ZX#pVFl~@ zWPVhy0DQ!Lj)Rdq+C#x7EEaV;HY%>lvCmu+PXJU>9pgWV(8cENvFs5O_1|dQyIb-m zWE))uI45MN25<GjcguS0A@B*83SpeE%Loaw+WaKh=_GrdS}X7U3e-W?ou8u%Dk?6H z>7I*7ww~56!>UP%dn}4nGA8g&5|8>>zYfWGUsq=hrCe=>7p9tlu6j#k5P@O1p$NCn zUV4z%{P<p0(GJsJT?bzXQWK)ctvz0=-Wr)7nSNUmY1jN9NARUfv~&xj7o``&A67^2 zgq%vHwt@CL&DEzb#GwY^mVD8NM8RYQ;;8PdhlEYmHBC;K%|@t#+S3r6=U<4D`GL}m z%!{7LtFsuF(^-3a#ZPv9Jgxbl$saeU%(*VuuBP$wg+n4p7D{g~u?_Dt*jl#zfqG2m z%vu`aY2k{4jaT3*gk7|i1Yb^522^STpQv@=SHfE0b6LA4t~2O8%#h4!r6dUEBsEAB z7>$mdrphfP?G0r<+L@fI)`|m&0?4A^m<k%&!D^&i9bL1uqLOiVzozGJJ7esZ6+$*- z%~8ny$mD3fWF#*Ia9s^dK)qLj)<0}R`4h_Vh5qOT-^+)wCYCmU#q-<=&HLe8qc>)n z&Z-K#;jbkQEB^Y79F#0yiH|4IAf^StfrL&;;Sum75sxCp|9Q+7J<KW;E;T;n<A0O| zvVT|(KTa$G%|G9Z2lWCSIF(y8$o?cq=^f@c;w-}(SrE5tQaG<8U{SzHo7KNBb_3D2 zzJ4`{&w5ic6riz6o6$Ft_Yiv@%&JZQ5Qp#fYM5kDMoUQXCP@5saJVNrxQ_OV63;DC zhE=J3=GM8#;5g7n)yJ*=H}7*fS=Gk2e%vys+ViN|AiM3Dnt*f)^K+HIzg}FPmZNKN z+_@0={0BhOesW}-j@z4h4OmYhHmqqnrUkR6Yag6gD60H;@XYaqWR+)-0-5!Of_@(_ z@n=ppQTQ{0Mu)b3++BKdWy9NZD@=y{PO?6vVa9#sU=v5Edh+n_h7WftV`8!s7KXzA z{y&1G(PgZ#q{3U_r=OtFz<O>n`B5;%#rA7g6j}O$*O0Y!YVWsp`l7`@gVreD^Yo4M z|7>qW$M~Ba27c1@OxLyrN&5A5ns6V#y=*dh5yrSQ{_=_Lb-fiiSTy6sJ{Lb=yu-Ko ztZSk`;o-H4?0ls+a)@z+kI6%o`t#B$uY*^^%?&|tdiDO%+?1ZyqF${ea3Uq23Y~rl z-MhU~(zrah(j1I4@JLmH>%Dq(@%m}1ETaTb^qfTC%C#V<xr)ibt?eL>N{%&PF{vx$ zo!38MMH7J7JV<oJNq*HdQk;H?p(YWhI9+cq1ZuGp4s?)k5aHbKN5o&Tapte!6za%; z7FWjm3mHuS*SlXKZE7E$kf*dN-II7Dsq89Ci3iY6r-UC&^8P$dA_}PoP+w0gb1K_x z@6F(n#CIBB9BsIKQP^-&Ox5-D<WJ|pDt4Lv!X@cV<jKADwUz<x-t%<X>cXD1dH+9g z|L^gsayzsz4D%LK-RKU8G5q(N9YX48y`c|%t6#^Z=b$JvXBtvYqc5d~ll5w)x>G{q zb2pQYlDH<BN`4gD0{j)+w+H2BHf2Uvl?fzmKra{>pimj4ONfR&cN%s?k6ryW=QM0} z%3Y){Z^&w&r9kcOq0{rT>sE}ycz4l9jnaZUh>q`^B5O_#t9YUI!)z}UF($8^o6rTs zd#=Ya)V*F>GWVfFvHHM8I#1orX=4spuVF^rV6#mDQ;I0d2x8>cN#9o{LtF*OxM%bp z+0qMJWR<Gf#Q?d}Ge7)j!-bdw88rZQZV4;9t~h}z88%@Aq{D@o744>QhSttT*f4@2 z3;trRT#DKA1ZzarU)ES>OXQ+?FAWXt7eMpr1Zk04N!)di2TNM>KFtk=ujq2kIaxZ- zoeo-FX@5;fJpRe(>b7^p+%lw$W%>?x>vK;WP4t2s5_ywJSqf`TK>4BLo&e%?Mzvys zO7oz`8t1Dm+G81>NS~yVp{R*mU;V@Qr@AC_1I$)04C^~*6#^${1zjO&xUTLsAH7ha zF5;%-eNVw^vcPA*^*bhs2YoDHObK5s76@z8qGwEhMjfc{e?68FU&~6H#_PW@nY^t! zM9K}CS;U21fw~&D#VO%po`weB=k#13O&_)ze-pTd@^#Q&t|eX{SE;DMMa1`4S}T(6 zvv#NJ8I&QN2O8^lwzY%Wmf9+(;jUA!8D2mg+6eb|Of^Y~mKR3~Epx|^x*@@9LDQ30 zcShxsyZ5?2zO^rggKGxp5}HZ7xbaEJJ{M|6bt}1mr6bCmGX1yg`CAqJyLkWmV+I%+ zKlhs@JRLAsUUbgyE?Aag5MkL8?$xsa3gprI3|nCC1>==7nO@eS&y%d_HKBiow2r}8 zk(T^bIo6!pT(4TB=zn*jx>PyM7~Zt$KDiOWEQTQtE6`Ou^Cj)(9(IEw9k(>Lb!jYJ zSFN9?jyq&|)dyp|o}%X#CLLSi$30F$`Lpvk?hNN}vrCVTnm6ZSoPTg4IbUTnt2R7Z zyY8ZrDklZyZl)YuED8|~oG@>;CM;ffst*4(&%0zm60UM-*vj;u#v;LErD)&EPZq#Y zdG{gN;xR+9zWO*G4#GFUOP5b5;q;uA)2L#wwPrKjr>m`7A;D&-3XJV*%b3q<(mSSP zCVkFFKY_-P)>-C0$5NhyKhYDl!E<lZNSb9$T>8m%iJd0rG0~PTBU~3KfDfepwOmK( ztaGqu#d<1<%1fsGe1^+#8z_ipQM54oT&X2PJyULXnD|!eFg)-{3Ug4Cyx(PVhjPF& z;5%*7luf?4t=neP$-^zI-G1yO*tyfy0u1ap4evzx-E5=m<z^ygBla~wfGXjYrg;BV zGJMrU;NrULA`XFbA`GovcRqTPeEi~MB*-Imj4mN}=1nztp(z15(fLGgxg$Urs<Tv5 zoB6D8M#r{~?VqaObbDiO7_mqk-F@kF2F#MD!(=%W|6$C><5i8tAAXwtTwy=rW_T5p z^;>TrgmuVlAM8zBMiMt#4O;!`kNTLToYO>H{n}nC<@8~y2&}L_%3Ue34`+r`Z-4Qt zBFOyEZspmN6Sx^HpR9vNQxbUXGe6gxBMg2QPpO2!sYign!|rm*7*%xlGf(C8W+|mM z%JE>Wa(v`?rkNwNTDAT?;Q>fC#{^sZA8zTtQJQs@i=|KO{GKhu#y7i6(|+ni95$tD zyj^^Uh80%w$E9dPCpKC<>o{ty)e%#6n2EdLnafJgxbaX}U`*`EReI1|@MCuH(t5wL z=}ca24lu`ozJ7x=e7DW+d{|6P?LkBvdylBBBi3WV^@ZChcX;+~eSe{33JlgJexp!9 z>!Jxef9}i+$dRFXH?SOFU29wC;*eW!vYqJu`|0Q$P%!Zq4R$BGO>Mx#nLTcM@k8JP zqRbvJ5S36jNZQ)Nvdk{R$yd*nft0#3_9pA^K!!(wXz={giNNQD!woWKlkYIo22VcL zS@QzIcI;1gbYy#?JNEIGh&(m9{&!*ZCzcLe(7~Wpwg{y4Gz?`@Q&C8UOB$KYNYZZH z`f4V@M*GW1g;i+Xx`pT^rk6!Zpn(;H3?(t6?#-F5|0$q{(&Z!smxWC>9_9Mf&fX_- zjC?y|iwbUTKN7!z-eixRUr(#K&eHcM=<7Nkjfj+bjCXvTMMko&&v)Sp_5WxAf;~BN zwrPuvR*Et61vfl?h|+pMpq5&ieWD3h9?V<eG}^d-Bs@m^X%8b|R`SR<{C*|@z26Bg zSk2R*+Y@=JITP8Wcs7`u#zC&HV~-fEy_Y)og%^zU;-n`(G;xk+Md6-tzcJG{O*?Ij zY&|57a+<W{H&=6+g9L`ZvBKlsdTGOrrVw5#bCY2i%Dx;r+tW_dvxi~5Bkv?d6}x;O zPvxoy_s_AoTI@nA{1<SDjH`u}s6f2bQ|bUO<8JpQbrtQW!VQtSn@s!ZxIm{#ujQZh zy`%oW&3Y@TK;3upoQ_XK-23jACibgqzb#o|!JYMl{tWNcTBZ2uT%^<1=1CWo$BfdR zlo<WzgoXNlRq}ryEfu4D*YnhtN-Ep^GTj-3$!eJ+G43apy&a<kEDN3TgyUmO0rYo2 z3AW~S-r8=(o?;tTkdRyzI^I8Go#L!BZ+9xkh~jiMV(Dvq#_N;@Y2$DOmR=V^mhi8b z!XMZ=Euz6@hYOcGWF`OV+;3NTj~n=0_uQJny3^w^E%9bRC697S7PR=@CXSIB!F99r zW_)xWdtJh-eE)JpfaTdB(z>JSj^;QxHd`*vGj>Bw7MMXor@MNzO*oEZb|X75)!SPN zvYvvu#!MC8_cf^>0Xy4H{5~GYRN$EV)8ZCD5V2Jfw>=cpdHKV{qvpuu40f{X=4PH- zAV5M)6*=i4PZ%f9&o?<KdDUA-Q2p<0PR}HZ_DZO-JEvQ3b8Ri;P_6<dN!8IWNG{DC z;kLYzLtv`LVIW{)y}hO7mK_X-Ef{lD%)YmCTr@s5p4XsfT<A%66Z#SRk%uMIq9J!p zG~QFAIO=d5Sdp!9HCBI0>JU0d-C!W~xv|B#EdZ5Xoh+g9vL2gub>$YxE3t0z$8*jo zbJ|;%1mwcvDVToM>MD=^T?@{hwcqTn9K~M|A-m9(*wWy`&mZ<MTnx$Zz)^TqxFg5F z@o8&afHT%V3qFDDMLJ_09Kwv*uvd#j7Guvj=dYVdXRo$|CX9LIZ$>H+U#Q5FwyP3? zFZ_6}!}?o;RX?~3Rn}i;Gv3I3?4=F|gBLv(x%=f<PIpq;IZa=W3NR>D!YOUHB&YRN zFu|GZ;4Yj1H~6{KEo7)_F{~Z!dwwIV3QxX}aIlJmy<UAlh8j)5m0K&R7JO#c*53&A zLm&3?9AYw@i%4M^{&3I+4!hu}D3nn)Tnd;{n*aLK{J)OVnJ5#W8UpvPLPahE4j0_N z+EQET0fc4<KlLp?4Y)ENRQOvneAhNY>D@Qgw6%VoGGVb+&J)$$5~A8fzgJQ}&vmdY zn7Oy`H9EtkjPH|nVOb7n#2Dqx764OZ6(EuWHEM){V!CZy434))OBABMB^x7N`*S)? z52`mD<;;kUC5D432cn?TDjG^Uy+=QjxPbH8^#%uf0S9lrPu@PBZZB|s^=Zx9+oE=t zX~FtEqHA9A<nD1F#~8?FloELfn%Sf?{%5oIbXDaWIx}{2mLMYlotD{uZ^xEx$t>I{ zw@~^73d7x^o3gdhmJXyE6EX0oJ+Ey|&+}@Z1D!x)g<NQCF{E_>2c3aX-_oFxXY?zc z(C6E?t&WlF6CiUj5IKa5?b3CzouBHPG0mT#hiv8H)A0{AtOjE39DjP^Z@y{)nJiz6 zQhv=}&*;$7&P^~2WLmlMbl;fG!47xEAG($_$F*)nKWi0+=jLh)`{K+Y#4hSb-KZ(H zOdl>$Iu@(r9HVYwpa6mN-gFO9g)32U<v0MH;>)}$JUDea_$L9mB`kJUc+f{!Fppi7 z`Qx~Lb2g_yB7%{23X%>@814B;bub}ZR>8d2HuL1&>=<7})8VEiz$qejqI~|c!3%^a zGM8<mlH(zAj3l0KFMR(5k~R8KaYy$c_T&?n=M3?d#upxZ|Ar-<!)7|bCjB0p4&)s! z;;CKGDm4Tdlh17_Z0s7=I`$YgqIX^q4KCT3&Y1@e%7Jk&Mc@eg-<7ex_p^d?8qAdc zX(B=Wonuu~&6IUZ)45xva8^U$jk0%UfuaqL=Aa4^_2x9zXRGOCT5lNlx8M2yj`Jsv zLQk4Xn)d1E64n>d4rRTY5)*eqzW`c6-S<U)MJBLJx#^?FkE<*1Z(6($9SJylO77x} zOfAr&ImM=hy2Qv`0Qn>s?;h&ZA)vFi-f{VqaSqoNT6V~-f9N%IK?I;^2-j1<6wWz) z`f$#F3!AuN{*vvVD<eJgBeQ)CamY)=FX)oJ*lwttXV<C}>NZkjb3~c7;HYTiLId#y z+pUX#^yY1f?%|J|80PL)BL)nKpjN7J2$nG+_|jIq2{<{`v-nn-_eKmQx!<#GN0upY zvwF))epTM|VW!ogBNk5D1{%NdCyucNCpRaJam$`n#={7RDPWuOwf#j>nRQ~<@?<>b zoiwBt4hE%S`+!#PXSJgqt>)cp&PiK2N`S50=XL3*ZbW|rQ`YuXkoom9{Z?(f-`l5I zM~!`4w0xd{!h_E*>f%@#_@%%xwij4!m4S^UibXf5Ze{pW#`T81!-b=Fkr%H$Y&{-( z{ll9Bpgx~A_WFN1x6l3xs!wNtV8zV)iN|o2KRKnQgO6QpPpi~Bj+%nUS{Q0y225A0 zyo<V@Wt7YPi&(mHRD3A3KE<qdx$aKL=XanAhR!UnQW95zwM^*8j=juZ`eTt-f?u#{ z_ac&xWgpRF-)p0h5QM`JsdjpAMT6X5H>k`;N#_~WQIiO?yGZBNyOzAgyBl4!CW45k zWL?rO5^s@gPpLmj+<Em4RUz0z$KZti*a(#TxGsKm?wtU<^+hU)*HvOLvREyS7$WHj z_b#xk{nD(bQsv?atWkZRn(Te&Qe2fz|L8CYlL*K=b+&N7*|Cq~dnb(dX%h%rpLLo$ z3Qy*L|BIOa&m%zR2`EfdhJ6~puL3=!$V}HHJD|9Teq`oeY=O0-uym-ZHJmL(_k435 zvN`Y2+0t&bWAFzid^WT)@oEY{^**?DCNv^B=Pddk*RL5)6dP8!uAR~2CY>3r0j(Np z*kG|;o96(}!pC}dht)C49mS5_&5@`G-scb7IW|Dy=zrpGGePKu3eJ0m9`^%ob}>8m zNkQa~z8sw<2Kej$z{+GD76{yg{s4f!caj#~>oE<n_pEY(Unci-SjmP6H>{8D4QSJJ z6sf;VV(>rdAxZN6Ed+s`Iy!6D&04AKi`WD6!o67ruHv!IuBwfve@^RN+W%?!&*xyN zlry1D1)NRx`&#iqy-ywp9G2WR*~p9N_;DbX(s$+&b~_^)4@?muV9c!3Q7BKdoq+T7 zuDgI{b>j61@7lVPCGP{i$!Ads({f??^<Y*JN-?m{=|M=wXus>Xf$0eYM$nt`@3wTO zzsqx?L-P-fn=WY3n>RAIy2j!T;mhP7?6i^E%d0<POEOli)SaeRcXf!N{SEz_I$PG} zO3bQmyPWIk)x=_GrN2kni<k_-3RwCiH4$o)@mRz)a>Bl7M(4%g(Pl~VwAp1kFfO7+ zm6J1JUnJGDno;%`LAChvz32n!po6@gaVRIO_+}}|EuOXtUb5wTDY$YDXMW5L&3$yY zvfh}?zcBq?x#%{QQs=}G1Isn`;mgFQQe&>FCwD4-nZm<gw{oUJlG<)P`7=$vP$T=; zv|J8BSTA8%@sVeQlc`&0piAAI{9_-TX*vKoUR5C~=A+IZ?Eg$5^I*CFD`c5nUII)5 zV@fh48cniM#ZeV6E_$7-8BWPV<)NCMN4L74w-!ZQ>i!30tD4NFom2qonsc0SWMzJr z8Z%RP_gs(L4`=<5_+zQjGc+*gNi+)B{nbTz8@eVN{8mwWgi!d)&RxkEnL^Ao+vGF9 zB|k)7LEDh;p9%jW`rZ}&`d7y!TKSf+(~A1py^%m3tJ+EHQkqZTf(eCM+xK17#D$kU z4qw}O1JXt$oq5CC1BG&-9uAXUHYYArh@;BPhA{<i;(Eyn+(v@y#L`0d;lHWb-)GCb zAsQ8@<!f|IZ>k5#WN#N+H@sUZ2>!^M<~AVA;a#6by0qU1lEvmz6Lc7rL>?I$McrE> zxE_zGloI;v=^twsZr|Y-X|Py}pwiZCDeT_>49mErp0bXeS--fXV^>~!sdc0>yY@b2 zEnzGDnbOOs+`3=E#lSrCqtd5T=9aBib!@wx7Nb)+um_Ih#IEX9dqPjXrtO2;X3A4X z3DNxZ9IEA!<LJ4zLL#85k8xj7%H54`;UtfGPe)DbFENHjN*{a8YvF?J(8_-X!~Ew7 z;(^EW+XDuPb;*G$fRFZg7c!F=w<G!%QM~zDDYzI=O{01JsM}d$hj6u;t4~C)xI*sU zkXs#X(i1a4PQ#vh+~%=oH%mo~>!YFjKDcSAJ|ZR1^T18aVyXMWy4^#AZLl#g&r#oN zrq}x+`!VMQN-H{sF7QZ$D2yXjja+nh4w|M0PG$L7lz$9$lAE(|haTR-uf)_<yA2+} zwgr&EK%de*QdiEkG#0eeU)dXDIi^Am@SWRk6oQpp6<v@$S<&-9;8>gsgyTR#tg&L^ z$!v<iJK9J|*e;yVgiL#6d?)i{oUtziw)X9YSwO79lay05Ep_gU+A-fk29iIl7i0<r zEYirzaV-_j*hxC{-Kc1(XJ)=yq^SFp6TMe#m(=!YgsH?`T2`$vEym&$mxvt>1t{@8 z^Jw=JD?=4?vMJWMn=A823#!0$U$;WBoUFeqN54SDx$FO|mI6%ti^JZSL*_cy`7B_E z(NHuIJKgk(w!6RXt<M$E?&IKP4^ZI4SDX*NOk1(Ep6yHij<&9;Y=F6)A1?fx7pLc; zj-brxrvQ6yj7~Gof;IuVb{&2_^_@k-1u6HHbSqh&>@gocy=V}}#+w}j=-d@@b{?<^ zq5jO-@P{>>ChN;zqj&+@btZ8&=`eR2aC%njD@+{Hu?8qwPQJ$T^wHKX*0h=kvmb@9 zE0Ua4w+oMiwB2f#9r!VarpTCs(#`OpjzN<q0$*nE>f#mJ-mY>=c#!<Ih;U~0G*GMQ zWYuefoLEQmn=seycEL2JxqkWt%8Z>o2Qz4DGj=Qgih~@dc^T1wpjA!Y^#sfn<~o|k z2FHFm5_T#O%RB%YeS|aLGIlgcPvj~O4sgCwkz~qVGMFF(zGs&lrPGVCurVjXX0c!w z2YCmmx!Wdv!PaSs2J#(zuT8)sz^Y1>sC4+0PH(*2R<sH2e0{H}t`SsJ_}AU2YqTDJ ze1IdbmR9Yu4P4!>Q$g!kXS{QhgQRSa-Ny64x668`zJRy(kfY_CV+(k>%#Mb1?%E{x zu|lf6qczhsj=kug_}pEWh$8`iIq#N}?XN32tkdw(H>17HM#UIfU~FZ<D|g!Ff5zmJ zzdc7v_i7C=$2GNboQed(H6T0R`eA_sb5^YJNhU4oniyWnVxy@5biwj6H`RKYA5%si zum_wEifPHe*=dyUoH1xJnuk_{`?!=YD5&jJ<Wo>NqOIXouOOxN{W&?%HHs3efY`$6 zpV0F&t7iGw^ouOuE=*{#mAfnbXtK|%HqEMZ1x4AA6{dQ|1d5U<xnfH1U~yE}ZFPPn zD!700qI(qW){w4Z@FyZ+ubOjG9{5+&kr7ji_%ky=YC2H#^hG|Rei;JfyuJ@%>N^_5 z@w`Y)sl$?gASrOj_d8ZE#0;Ar-X*WAU6b*auo}<OXkbprk?b%xcVVaGIaO7>j~L}T zoF8S=xOgr_bf(yvlY7dSqF~=E--J21SFW47*~LoVx9`}kJ|4+vs=9W4%nf}8H;-JY z6{^!GZ)!4d?_ly(kYKAyptb4p;Z?fcys17NnQ!5dWF|MAP_xCH==H%g#HZ7u`#nYJ zmUQsLY#&(5iWosp<#&^Fy2^_ts%CBj#Of<|f5u)#3(^2cZJzRmHg9-tb0WJRSD$dG zoLwhpqEFVk#A@!^b=M^`m4oT!mz;H(Y%xNiJe8on0U>$w@A#(V@Rp=r^Z-6Ao^0~6 z8demKs1`y_chb6ZN0s{%V^srMIy3cFZ7p&DUxD^SO9@JJ@fh!}EK3hH2rwLZcC@B; zp|evRLv&Crj^2yk(aa#|C?gixMD48A5(@npU$-H)HE5FXMpMPUFv4b#!!Ux2j}A*k z>?JnyZ+zS=_EFrkRh0DmO}^7NIu~gD;_CzpOvt^0viZMxbK5K8fO^wEEZHwJc0)6f z$#Me8G92?S0jCYaR=5d{4r(+rm<naF{i$YeQ79Y@GUMHecTbqJ@#(sF?R3HK(X>Y0 zGrK}(pZ3sN!jHD9nd~R->|9{bsN~b0aAnVN6{m~WaZm0OuE4`x&oiJ3!0cdAKc|&? zh{5u}==kFd-|?=U4+nOSj)hjFOw!EWr(yS;k1wtiU(gIHr6h7mA9knJ%B_`KS0?Tx zKI*~_f1^x_&gu9BLu5p+-$mZ?Nv_GX)BS!e!w2Vt21+rShcpJmb9q<O76K?I@}iSM zOp9gVpvVXPKC}L&>>&aEvkRNiRn`zM7IlYK8;iUM$aP-c6YtR5EEhhbudmF+{wOQC z3vn&`3SN502Urlv^r;U--*fJj^5PX2NnrMqZ1mi8uWds^n+_7B%9=BJ(zu{bk7m?} zt^gKM!6GxN@#?JNiJR-iH2Ny910&V|;lgg5`<mrK=C8)-)O8b$TDLW)E7djLeJ0Me z^c{bgtfl6WP|p{=_=7-fKy^OV;%KuNSjTpG)M@dn=$^)7pmnb@9!SELU3_##`kumz zml^8v33FyD1008aiBXVTSkm^z3dcK*i4@tJK5<)V)#QsL6Uh<Bx%<>bGo=Bd8K-dB zp1^LFzs|NA2o%_-3E?cE3!3b*%aCh)uMTZkYRbLlKeX*ena->J-7ifiWPEILw1!F^ zuhHvG_ci$)#9ux!a6uWNy?G1b@vIv`JQGW0b|XxP2E=a@RmcGEc<cfc%dy0->W;B+ z$AoS(-L?w=4&9rNFcBFaZbV{Elrhsge@4LgL%(STG)qTDj1Bqk_{G02(SM`!BXA_z z=NnRL+#djQ8t!i!7*Dz$#<mT#;!!VYGn2luow)^*Wj&(Mntfm?RE_W68PWxYwS@U; zPXqEc8dwE5+z7p!16t6ND?Ju#O?m|#py@Z909$DpBOo^l=FV3OQm6y)c$1JkH8Iqy zo!-}i^hfxnPS{7ei!L}F+cCaaD0Vw>C;3Qg!fZXdt!bDE$=SmXtO)Ik<XtQO=QCDu zX(y$elvAJUNXJJ9gzt(-J0>VK0B?cKKf87nFp~+I4kT3zEf58ZrcbsDosi_Uv?fQ@ zDC;e<%!YR<Nw#?V73O!5q%2(Md@{C=ANvqOx(FHTRNk`nXehe_IrRvzTjIR@scS4F znE~RXfaeyFr)!E`$VlHjkwG$_4v&|fx{zZHY^J&o_i8+=4n-x5$IR+%M?XZ(`xyoF zr^ZL@aP0#ln;1tLu_%vzg+7bQT$YL?zXssOqt4hFOo$zeKDv}UPVt8R5IwE5Wv$&P zW=Kzi0z?)QrHlJf+mi;K1hc~F$w4>zqd%M9<$`z2R!G)ca{QmQg$1BY=G1qH0*EI^ z!cyj@BJFp3p`ey+UDyp*c9B5+@z~-($>Rm2?d|3~BxgBTxz+`3XzSs89J|NS-|Z9p zuD*K?SpJY;S!>_O_yp?bI5$16vV%z6Nn2kFW$qSKE*(_OBRZ8-RcR@7g+*d<$@-j4 z7t%jqe9F{HfmNauiHR&fNDo8#txP0lKTjajZN$}&zNHfrSq!XSbxl9xw7g8dS`{I5 z==jifHN7%G4QgYa>5Fj`l=KXk_><C;5i)<IbuOU*_9o~YY8iTcX#fA$QT!J!6&{@E zEk0;<e*9htD0U&~clQ1bRSfM$m~<#d5bw|*Wjcaa2yc|}H_IU~a)qG+Akn-JPDtrO zi0HLp=1+nYu~0kuOCK4)Z;5U%nP{tFc2!}XCUS8qR)OckFXVDfvVjsws2$4^0$YIe zSDS}&*17gd?W`=B55NsBX#XBZnn_?d(o$uazceEe<68Y{OLQ*E?#a9St!ZvF+;xT5 zn&8Tx^FCjW6PciP#U{7jpk1wSKL)ClN`>#``gEv)G{zxzx_}WCu5`E3o-*?Q)}Aq3 zSSYk#RqX$_qmL4w35{QbZYlBXH3v(lpWD&&`K?)wxljm6c1c4Y0%xQWyukWTCB2sn zO~;3>d1%V>Q?nIrNrcUB_D0YBoP%(;T#y$(?rm1hdF4&BS#1DQpH3_Cs_N#Me3#ZO z<7#K$f2HJY>#xN-S}OCfPToS6y8zlTjXyU!TR8xIu!|=eYA`(oZ0=a*52XRdH3m(3 z2^(oj+YUtOU_-jt*kZ7Lq}U3)jH`mY7${nQpLET+*}wV1uBUzcd+O~<ZXjj+gG=9F z<7O!TUFS6ae4C+%7dgY+bA@$aK`mkQnR+jZM|YQYOeJ~4h7oR6@no^vapk>t6TsOt z@uSo7kl!mEV}eks0UaAJsWc$H5{n9Di+gVm&zs2&_<vLqJ?swHfzcD=zNLJ?-N2I= zE)gAncUdn7Zf8uB1jzyw<7PFveilm-4MK@QPhwriRzDa3_JGPdLZ?OS;tvIfrTKN? z&FfA?GeqvL=~}rz;b)cA%}Thsq*M=K&!WxmNE&@rIR5_Se+TLsqfk}|rVLAhlASr> zeJ{s-GB!W@ZT!6AIk%wj@IQ>szi479hT{D72jOS;w2ZNLLRU<(t_pkB&MH{{ebwgK z1jpB63l~{S%YU$y1Lx+cmh81sdI=-u*Y3%}f;WQ=)FJOKKj{FbfM#4hmH>W7L_aJ` zG@wEWvT8S!u?$4F3$a-7e`-B|UhN-1gsy^&b<uI(I>468*j-}<ccnP0!-+X|khEim zUb}aPKpGuKOmI?BZtu>xxXDai>^a_h!n;5R+S*TiO&Rwn?J;rFrgW*+!N$4KO~qio zu-`HtLKX<->g-!?q`X)Cd+1E?2No#}cU3z$v*9DCn(U!X<w*b-zB*HXO}X$sK}Awq zrm~Xn^yhM!&vzO}1l2w(oB04^8gMe$YPTe|^VJLG57q~soG{FLAb7uyLRES*XlHpY zY1UCi)MKT;0=qjVi&~;-9|6c*@X!!eRoV-|MTjxG7NozghJ(Je2fI!UQZd2Zu<!co zk)vF%tb{Du;%W!#wnMEELMurd3@qPoffn*!k68<cz9_@9QC41I$AGum8)>RMD3d=n z8dk<(sy#9S7Sm0nc1B(4D7?;|U)w)@;5iuHhh*9rGV78`fLbQB>%?wie6u=OulLWq zTC4x0?r)3|U22j~LZGcgMlGM3&`P?IZAtA>&U;lr3hbOl{i+4+8?B@UXpl1;@C~t` za9LtsWrk(r#@BOB>T-A^hr^s2M;h}kKq0O>=0kL{d_hhK-dt@S!)gXkKk3nFS#V-s zyrC1M;V#Nq(zhR{3JD~hF{_oEdF-Cnj!!&FyfIMzdb}TT{3Y}5L2oC&T<=}HDzw>v ze^+*M)14`S0%0m`_){2jI9&5LwG!0{^>5yvTKRwrA`*k{Ln%YsasHUxbgwpE-EIL_ zk^e5lfkm0cS3BgXdu$jB5nEuf4`>6vn_K)h3ji4bLko_-RBD(Q>HzE84jtRmwDz}m zAS}Ne+H7bcP{Y)rOnHqN5-(_`EhQ5wFN0MD<tsp1kD)z%rrjn@y|94O2aQCBZco|m zeq|pEE6VVk64nHJRO21Dtuz}V=(APr-(yk!(&-C<pC(`Wc%6#bj6$UWj9=FcP2oN# z6-hEeVbJRWI_U0UN5HKSa`$L_uT?M2iYV)`xia%BK^Mqhthz|9)Dl+<#G^ju0<2fd z8(_q(ZZ`MtcQGuH<x@r}V=~C70sZcH=4V118eb4A6!L<`@6%39VM501`%rhAwIYbe zZ`QNg<_bXqG>&-?k%q3W?jauJ3bq4hH#qOlSaPYAcTu<fJgt$j1uGCc`svDeKQC6- z!srbq&4ImNLXS6csqiO!MVtonPWqM1Fh?}lQyuW}lz;PnROGrq0iSgHvrO;qT;bT^ z5pfT+qFZD>y4t3X+IRJh(aP3Ng*T8>1ODeIA;1YjfhW{N#}&-b|E0O3VQF6V4{pDO z;}C?nsVfjLbe$g&HR5l=Cal;2jXDwx+<_@l*P3p7{eXD())2|zIEPW`x$AUbgygDB zx}YuzFNPKYtRh5k(otkgs&tRVF`M#!S0xX1u3yt7w)^o!Dgz5vjlAfX4umLO@~<eI z$Vu91#`UDt!L?Zfw`;U!yf3=f2RgL`r`{Br==2%Gbs0Y(2aa`40IJ`*V_+Q$&8nl( z@^Rj&#2lJSVVw*Td|cex7<q<Ihl_^>+;xdUpXdvWV=tHsE75CS*?La%K8qe=DIv1) zn5*OtH&@+Rjf5QW-;bLORsN#|5Yaor9n1=mx<4QZ5{NwvZcbF0G&MCw?(JPA`)^mi z?7FjO_n}tdAZh$^d9}OHU{~~|rP7hOH&c_2cifH*U&a?SGob`P-V;Hwgh<-#mTN|~ z8^3=pE?44K@fM&r?eSpsyJw?3B51Q=;#@5Mocy>}bv8<>L(5f_$;XoxCe)qFQ*-i{ z&rM*ozXE<W7MC)DKH1HNhCo3Q)q*q6`m|WDCs!QZLEYsHvZMpivsdw=>Vuuznt%J4 z+iZPn7h=ZIL7pCuEJSn}O(=je{V??x?jT`rkturu2No=b_=xyy_n%@kdA_YIEd$Gq zZ+r`Df7umrI(H7Q9a=^@9wCUON(jH$-AGcxi`928D&xR-8w;i;d2LtqjLLh@#F9(( zh*p?0Ebq=n6}SK5|NB9<N@?Zy!Jtym%uJ-p-BV-@wswPiAXD#@8s|)7x=ZV@FImx_ zIM=KtD&;{P7A5;3t-|Nr<>F(BF#l85Ne=(ab-Kb>Sbp8+tgsusg}4BI@vV&Y9h1sT zzf<JW*G<jA5OK%;gqN^u2NR6o2kFuuI&L&pau~B0uXz=?OTV)Z74`v3&!}#{H3t>@ zO()B+mBF}^)ZWQDxDth!t`pf>PZKsZ69f!t=7gP9;*&4?`0^iIa&H7A#UnRkx8;c% zQk|}5Z-WKozx3Q>^L4{~W1QN)UaKCM@6yu~j<q$;U{dx#9NN$d|B4&j$y>KljTlf! zvsjWx68(COLp(a`ipn#`h21mLnv()7%+=-Xu2g>*Y}}G#4|40Fbq{4q3)10NS-{jT z^Q>nG9^6NaaXG#m<|xj0AZSDGCGKP>nRp{)VtW)u5JAdu&p0e(*t1;26&?D!XU{h- zT7FxKbdXxRY5VFJZ|d?2A@n?H;z|pj)m$UX7k-@9&+vO#2C$vQonYIkrJ>jSH&%&W z{k_?4tdHaNdy64wL~b$K_jrg+vomOh8fNjFOObtcC-sKc)7tYj6CRE9thZGEFBDG@ z`O+Y?BTRLt&Fzc10BFNdaBSf*fK)bLhJ{c=naX(jH|~Q!S&THXCa8GHqMnwhB~XHd zBM3kAV89^fHlR3T4f(Y`^zFHVaL4Jj0tO9N4Y#aF9j)T8UpRYzeku&d7m(2v0TWXD zBr<d+&kv(q<U*Tb*vtrPHOtrV`Kj84oSCMys=F>%pwa{a&<xv)rshbR<iT!!nN_DY zq2zYth&7*)=&D1b!jbNXlzz9<#xrO7uOA}Ods-ItLhgbl&0U9GJ~ikmP(q#%H`(lL zg8iWCa@qHRI<GmYVcidg!e^O*uY!@Gj#aoW#e(+TA}(9v0Q!mw^Wg{G9^r!5y%j3M z-=vF}tzvrMV`0pkOD|ly7H%MhN>GTbP~hSp{`4;N0n1yn7Bd&R8k<5DHD)$sLQ<DI zY2&#Qf2ZT<xH;$QTZiQ=f&A=G94xkWJ9Py?GLf9G^$*WeE9pVsRZa`%4f)?5GoE-; z@kUMW2}?BD7~Flly_eDIcqdgWM0$OKmh!IczKi+@)5lQLS-SI$^>yJivOCz%oP5t4 z<rT+^=j#aOA>R-4*RCJj_c=vBtllmo5fn2qSjE0_w3c7{`KMj)g5pP#T2O(@av3Kc zb)mXx4!eU|eO$FNs4O0vT)j+u>NGJqw;lN=rt#jso<yGm1JR}G-crJN@tC?Rp5wmH z?ZCx(HFrNmXbs<9pzK%@UVUYrt`7W0;cXjRPu{5ub^ipisM%VDRQU0wxwZ<*1nV6n z?TK1b^N?%7)7d_~PnA|8{m5Te8YiAADkY=Khi-aeecr!}#S$K&WJcqGQ<F!{qx1P> zO&kaP&INh8sdbY5D~~azi36l5UB{$!ggdL(f!g0om0j2pNJ>xcy^*`m!_MzsWz=vo zrtFfOYxCd8Wea2wVckdo&ua941tl>3xE7xHucOH@iyeml_TXb{#!YWCec}_AV>8$d zHy{!b?bQL;{%O>H%f}@Uqs#x#u)K7Yuqpda9et3Ch9M-#P1-=APjfI(^^%$~pGxV9 zP;&tT*n+4^;buoI^?bbB{_RZTC(tvG${~XyvIcg%-r%t2GpAXShbmJRM&83GnMu#? z%Lo<9J5sK5$gQ$)!5LYXbo|>-Y)$d#fY0%iRHansa~5s<+*<TsRPH^d;Rtx&(!L`I z=#ja~sIW?Ulv~54ycLjJ+MYs-)NOEupcsfZSpBe^><ha8^FgQmRS8z|bF=Ch((}=p zmFIQmKK`gN+81NmsP|J~dZi2!o3JlnqUPU|yD4MBzF2qL-KkUcOL;ooNG1CKyzXZ4 z9H+Z}=hO5%$OOC=a%5Ho|9wrMMdB6j$L1%@HaGpUav5e<*mq3RZVeO)^3;Cn3z@)p z4q%-Dr<SVRp9F=YJ1gz3z=1{w_TeQ@;Q85Xf5l{HHJ^3p^6mZz_eoXu)p?veGZ0@e zVx1rl%H;V{*)6Ebm=VqC4ZRWUOoXQI4+NZ=FPj=yOd}dvSm95=C@#U(6>2ekSaI%i zQ8`XJR%_ydZSz<1o$9{UhJm`k$`9tgp5`i-8h&j|kN*15jIX&aFBLm7dBH`^Zhm7D zE;=UZWfx(WKIExycLmscc)v;fcBS6~6<0^X`)$!B)QC5*FC|rf&n~^z<ayGbP{ay= zNK+y(Rh%mCk;W8&QAiFCq?+bIU61M=2m_2@t#lgU<Mt5wmE>9fzkp)nXOO9z348IU zAm49^oa$Qg!K3b1#FJc&7xI7M`tJiK`!md5L&!z+=8gR`pp6-U+OnG`Pt(AK&pM8* z!|s}yvHhX}eGXIwff?#Qt?|}yY9VuVx_<^sG4y8%aRx0r^HZ(tAcFkvP|FY>L1Puw zv7GY53mNZFPE}-UloItZ>pe#m`58C&iEOpSfc_d+x|G!Nxhg!UZp+HxVAGyZjY_jr zM>jFq6{hbGdAjZ-I4tY?2x~1HhY)2r6UH^VTu9Gj6<AAEj2p}S@O2~SnLaHA_tar0 z^2aUMLG@O-0b2PmdnkVKoF|Zwz{CwLXk<7Z?3mJu*V1&@)-uliJakiONhNfJEM?b_ zHd5*GOzq%btbOV%CYUB=7z=5_;2THnD$Q+Xd1*n}=cCdl`cOf6-9J;;^SAZeO6d*5 zR^Bk}^I<YMx4wtykLS<DPE-lCj^K^LZe$BdcZjDW9$KXOsvKJvh7O0^=_{+Un&d3u zGzORL$WR2?O7BdB-z)}6la!Dju+_h7fzNI$!uF}=2iUWl{!n?xW{|08N1?|+Iwt(k zYR>b;>fDiQ6^`<J^Zx1DwY_(tLe7^p*eVZZa}cMO#WucMwGPxI&G_Xhz9Tj(p7}zW zrSmu(*pYdc99ApV9GdC6e|KpBRW&J4QG%G<>;D91@(7%$`SyaZXZ#>@FR?XJbuKH` zS8N|Tb{E}XXW_J*4CG%UC};HMh>HrN%&+EOi$yrozVlX|Rqj5S5V1bj2i<StR5IjB zo2WWhwq5WZkO1>kUHonoQcs0LtY-pmvxNCE{*|QNm3qs)h#jnd5Eb3%p88&m+<)EG zqU%e<06^=QyLt|0RIgO<_maetZRml-v*HEeEIAxVAGsdS%tk}*=YRCRQ)?Y31ig8< zpsUU|3yog|p8c=f^V|$aTMWOF%vW&I>?wvl%}`LnzUI7>=wB%BvGl16*@Aep6j@u} zp<7|^&x6{o@t~U5^iU!J@nM$6Kp~TsttM{6s+OebT<_k;o^@^UhUr(0-pc34kIgV+ z)YKWff)ycKehXbW8~ww}BfokMQV)CR%FIa_BYu4?H*H*KQf455Li%%z`PGaqd1QDS zha1l43SC)7aMdkjNv6;~^t$#Lxw_zAHPxF*Y0Z2gpHrO=BBPi1b_ADO*Gh+MfUklQ zJ~+*%;$(%_*L7)gF9;4eKEs<ZA#(4a>2_!bT;PcNZ>D`L*&_J*6RL>bFd{>qrURcr zq$?cwjxDQcIl83o&SX2Wyn*`1n!hXqA#mWiSSXR~U-)jh`9?yo_cg&Ky2uZCAEXwu zgMVr`O(roolxY@C-uQ~U7F1l`Oz&OuHNNf`-}HP<WQZZ2es)fl^A5?nCX@F0E{Tub zS#Y4jJfK|uxL!@q{RF&U*}7dhzxxD>exQnlbZx@M{R0SqCi6zgI<zE?%}al|)@9^u z?Kn}8qRt*PNw4foBWzGJVSKExbnC%#0_{g$zla9VKm9GL7lg+G!r2?xqqn|sjx(kB zmqj>bU6V`zQZZR-T9Bcvr&Hjmm_Z`de5crX#jUW54y(8#n70XOPIj+CMc>;rXdq>k z<8sw&ld7M|Qes~E#~YC19C|vQq5DewaTckFHJ_gGS4#!gL-M@7v-TO=ysfd2wDxXt z{P6ueGrq`*eeGRzD1cdovSJH?f5vN=S~gAOY}Tl!4I@8G)f%8SOr})VGdG77dlpiX zr#+W|4LR$kt}E^n2va*fi~7?jwrwr5MKR}l`*EKMJqn2J<uUxXB<~%(=Feo4B*mP& ze`}<df7!2V_foE9r!DHs)Z2&E@f!T6diq;Lwah$1HDbZ=+{5n!GmnI&$es+euvId5 z)V5YAa2%IO7}7qMUpqHa!SwXI@F)=ZK|hSMnB58ze_S8Q?#Vi}qNHJaTAbYx#;|6_ zvZgs<4Z0RDwi;qV{9_Cpy{8k(YG>?X2`~bhD?@1$lA(pKckB^*O)6{~DeIZsxg8`p zVsvJF7*w}AwJe!hVYyt{#-ZO>{mV{3y}HTv=VT0J)6z6?DVwY4?C1PGDaxzQ!|$U* z#+5DYdNx`n@S_^OUnzRC!OoL=S3Q_eZWBs61*<IYnxemZn#tY|@BR1f>J4PzdN)Lw zc`Bcx9@C1G&b!ieWVq?sKA4~xRm^kw);><x*|$A9#j(%{$Lx-}4|EsEHQa6EnduJ> zpfN4MFm{%G%dH2WM{#$;vmArC_&MlEcijv<C%!%{ZaydQ+0JQ^6?!&)$!(+g@=p6~ z?S;u+MJM^Hy?7?RHuFGY{IGvEl10numz@WPo(4xR!FE~WK`f`K+%@l+y3bZ6<kXb; zSa1OL>xUu*<jg_IDSD}A$iWiC`JGo5Dqc|YmGa;^57wY!h7hkdhn=9Cf!(}Q4I@?f zB)oN*MOxzNQVnGxJrX286F|<kOOR4BaPr<;$3ImnF-ZM6@fW%4Hz(<*y=UHCd7H}= z_TB|w<Jj9g0!!JoTHSbCgM9gLNwE<NdxFQwI@VQxH#UgvCMgg-CsO)N>B!$jv6EjU zh<8E}GS+uih|BFOE~K(&)3dwpA=7=MxF2eej#mo9qngyF@e0F?N~}BhH_%gY4f^Rn z?&cnCGnARfXKXV(ZUk?ZKAQ6$ua9r%8xO5}G9?$FR_c71zAxjxXGeHoWhlD-azKSf z7%>!G@lq%BN=AUll;4EA^D8l6cFwKxUMk>Tz!bakWMR$7w)35Y^K9#@shgOYFxJny zVC0LPC@$#VGVNa_ilJ@P)fU0HSNio43$~ms9t<c|5R=$lmXksbmIY#(F@@b2e|PSm zAJ=$>#=vXk=bm=)y{j0xlO->+S$Vc?gBCRw%&v`R1nawzfx?Uh?0txDNq$I&J4oDN zRzgF^w65<=FsAvqkI-MMZc)##)(P>%I}}DL_&K?KK6-sPY0SAw$M{4VRNyc@WG+{? z%c}miH-vnUmxc9XD^3<NJalR0!0r4B2C!QLemR)xHP>c!lUOgspsNt+%`j0E@a%$k z3z8|av4`gQcUaQG)4w+@vJe#eW0IhG2_>u8a2d7S6Y&YEr~e~gIDHESDQRahB-J?H z$JVpnjL=uM<xF}j`c4K%X?bg_@>q>}JBLQPltH4HDGWXDO<2=#W1G|4`?w_j%+|f@ zNEaC&cIh;`KCgZD>gSC~4X+|XQjqS`5PZwcIO}FeHK%&=9%^`dQjzhZ^qQCSa}^+k z8i1*<O(w=6B<yM!ug(x-F3Y7qQu(7~0_42Ndg-N)5BI14i3gGbe0oh_GLrq^lFfze z@UexDKU`%-^OQwrLtWM;f2w_25Aj^iw#(vG;cFgmLs#hzw2lur@Z#n!AHw<78276_ zBtt=2=RVV^xIR-SZELuzb+4$8&nS4Hx9#1wzqOmHM+=VfY|cxM!x#33KdUH(NCDRG z-?%-Cf8(us)F${L`k4@EpHbRUpoBGl-sp@geKj={shUh<TH)~k+-GfT`=K^qvYIF6 z-7+QkF%89B*zZq~xk~CZ&ezAQn-1p#7$-Yoy=;njn1nO;*Yaj@$$8NDEfS5e{`=Qd zMl`LQL$_X}fVW=qVXkiceZVi$&fF<GoY6h=+gf%(i<x8k`(9<zN1^oS@8`69Fu9Na zD}}(E&{yAGF-+#3bTA<&(bL<1X)WOgpyPEA$4if@_Bwhq|7u}Qj~=>jPoJB){y$v3 zXIRr|x4s>SfPj>-(3_(ONE2z&0*GTFQWT|E>BP`N4}l;9=txtPUIzgg>AeR82}my{ zm;eDHO-e!!g!2F4v*+E<en0pvavXuXtaV-IweEWyiR`YQ&)m>wcX}^&gY{iBX~ZS0 zkYCd|pR^?_duL&w-Ey=*>a`NSKAl@|Ts}&^IBTDjynkimw=&AbkZ<kbrZX3bvrIQ& zF6VG=J*`$vnmRXgTBw!RaNc#uP_BcbEV@OKKqa=?#h$3xUN&-`Ufvy^Ya?({GY>1V znBas@Zgg>KN_ZcBn=n@Ngb@bnSkN-Bo^NoTR=b>-Q+2>N_v8}VT+@B@gIwYznYo*T zP>?er!xD58@(`wWU!I9iKmQRhGHmy<Xm>UpPYI|J>r9eg1jTJ(7;bkbsM0-IbKk!R z6*}_|gH%Zq{spV|!o%UG$cpcboF<|*9%muoVLr{^mYI`TcL8vSu~2>a25}mlP{1vO zud9*4k-!R8DAjs9cJ#o`15~}A1DfeWrIWqPejIl1X1ml*gyja<wF{OJnB8E*DC(_! z)rS`&|Lrd;MR%>yo!{}!X&AobzTs?M2s4M_<krOYkE4V@|0JTIvU_fA#NdfQT7Cfn z)5{`$-1)`OfH)m|539>I8j_1I6mIcbTjikf+Z_T+`FXX=YPw%LFZhSngJl%wlO*fV z%io&TE~M~EZWAI<3X~}&nCQD=g@h8S<gUhYdW>?+O-ppLm1+sZ>-%DTDJDJmPx-7j zjN{X<Jz6ZhGH}{`kk5>5CZg1K?CAaigDZ5`<1+5C=Gg7kR#wqPh)+|9QpqJ%@BTBG zO6vprS1Cg1P<m3}H3Qsk-A>8!-hWP2ux$kC<ap10troxTP_(T#iKd9%q`o(^eNYcX zN<$XXTt`ozn6LT5WAy>}_xy&x!vrye;9gX)HKdJ+;baiwLgLRd0Dt2D?~s+`wn&WW z(c%2;c?PX{mPPg<=jjR%E4ah(<55O}iC(?=eOYu=pQY}dkKo8B0<;n*jrJkxecdO| z!AyBq)4i#c7e9VR;U}OHXdazaU})0nSAXaMIcmZ!n4|sm>>sj9HjdLl&#~0X!lB8` zfqC(^!5xj0hbvn&6bHKRyZe9`l+1N#Mc~p5J8vidg+GDvh+a@P=4&cCdT=hzs)0h) zpw&NtYA(oQvYw=Pox4bz_$NmqU|6A@H>RYjAruznnAlCxguE`z@WGxyV`9_5nDz_d zSCSH@yu*?-{33S%p|%rvUz-B#+F>&`Z(iNuYKzA@Y1NH9=+)k6;HJy7dz@IH<SBv{ z5S5FmOx0Y7Epq|g8U)>}XUV^TUJ%n<Q$4f-Z~AbjZQBQ~=ZbH6v*x}Vlv+1NtT$gg zJ5v*x<DPpqi1d$xRHV$fR4-#1snTiW`uD7z6c2l0?D_WVa8=@RL_!|tYf3T!H)th| z?%Lt^`@|p$T?lhuRXSQcC}UtNA<S2jK~;6Z_Z*X$AzqO=AK=KFh0o~j{eV1G({Hz$ zQ+rcXFQ=WNV(Sp;*$IZph2DjSwAi1Ewx~%PYULHx!MF*(ZG62`(SA#Lb>VprdE(5D zHyYN~u2;Q~LP#lTw6rhlJ^5gf<1XCFAPTd;?V0taL<@(~FFv8oosFX)cedJtYIiw< zP70&$$y%hqw$r+0R#G24W7<g@U4*jV_lUr^(l5yeef@G7s{QfAV>cAg!^yze`#*+& z4|yZO&O=cRx#m{YTxx{tpPJU<sdxRt8(`u1ashE|uRgKuP{T3@+|$rm@v2aD;X$Ux zG%M2wwn^VUS@+e6N-bM$YCCvL0V{N+)TJG>w~*l#fF>_yBg-;MGd!Ez7dx*Xo?s!n zl?<eWS<e~T26+q^?@lmaM)!z@Qafhl47RoZTZFircPc`h@4O|c@f&C}SE%mBFA*YQ zu)}CVyFE!!Ea6}#vmZmiCb0rk*Tv^045A3`4Cas0Ks|ix9A<1e;dgK?kAPfUvC0Jr zy!-<@1l@oFXk%Pm3lY?hSpseX36XbItp~xjcq(u_x(r%MGjV5Bo#@Mn&gjfr7r6|Y z*x_7AU&s>h`XAD!r2GvNb>^DP@i2k#lD3?c4j<(BOSMW95B6QVk6g#H8={kYB`G|h z@=6z<fcyC8Myt7S^J+8XTmZ+BuQ1kgQ{KkrXJ6z6z?Ix%j%O6=xmzmu7jF_*=5J@A zzjkzLJ}8j(CNe=`hHYL+f$&y2Ek1$Sex^IQ?_=QxgM8>YxCMj9y7xXSQi!<IdLfh4 z_5G8AP(j4REiFwk(G<Wr&gEV0*oEqzEwdSjn6U(kDxq$&eQD%pSi@kU-Vsn#ye9cN zCc+aB(Df9cDqLIeE$Amb<{Ox6ZiEv8izz8zp}O&J7jmOiu3RpK{vFP$`JTXawD>HU z8NK6S<ft{+B|YF#_=+gr(8>72!QS?!ZGrMd?76sIV>_vU`wWAV*(xjE?rnC;^M5HJ ze#Uy*!0FkY?AL<PRhHc4u0xS)_>$Iam121h^iAn)LQTxOL2XXk=ioZRZkZ{~DBmK3 z!%y~Ymv<Qn#VU%=tbco~f-<di(vmL?AN7wRYP&6zI}c#fBVBGx6M*3*&E2-&MpXiX zBLLH3q13gs{rK(Cw`Zz5jzi8?wCdM><x>}7S5p!!$l0nZO0@>hK3!e+IC>f1h>I`W zBObDb*MpJH%ZHsixA{L}1rUu|&4qFan6WJ^YT(DNG`eK}_eXQI4MRztOQ{f)I=g^q z;G~Z`@WQy@%@;Po^Y54cykO|II^^-GH2KtvsOi0y@j*Pr+P7yLCFy_WaP#M<@^xGR zu)x))5JOse#Tg?h|DM{Xhj(->;~S=6bFOfVKLqq6ED0hYf@Yfmzae+WtqA$WN?hdo z$Tt2LDO;vRgqHf^$!LFypQbnSJr2;hop)mDtna!R{2rgJV7|k)>&PJvN6wP3+0^L# z9zF4aDkehLT&F5NAR#O~ig?oHzU9BVU`wn;b=Qp?!>}B7rl4an;pVrL`#XzYVQ{Sa z_}4b+Mrs}hB>6>s=8>h{aQC^vaxGrd9|xNa$jbp7?E`V(!9S-}Vqm2fBAA_ajI~hW zpq;?p42e)>t_`;!K;Ds%Q{jv~Z~JxNWI>E*`l%|Mr&c!d3HZW%p={n8sb#?E?8eeX zTTtT~2xdh;kIpORT$y#UF1nOkDCQ2ye61V>{zSELkM7!HFqOZLeIEnqaIj*{To`DH zC8mhzGt=Fg&s%Mo&qsFg+4f<!(0s-*-JC%=kqY85e?z$?#*3p8O2e$aw#ig#)6EB; zrSGIHnZ4jGSbMD9jxZ-j`kiZOBTLEGW{erK`?$2q7(a}wBtOH=ZN*>g9GgyMJt{KR zC${9n67Dt^);?l)bkBP~n*NGjKGLQ{AwHA9fu;wzU-P)+uiRZH{ZMfIAVCRl`P2;W zzhm(7(Tc_dkeDw(<ir=SoT)o_v-ml>P;H!tUOEgL%YMWrN<Pw$-<Uos^T&8hc|F)2 zggc`5G?Hb8aVWN0E!P%Yw``ML(ss(&0K1Y0E`GGZVzImy9;$YcWm9FJ_c-et%Vuh4 zX4JiE{O1iGiV;vq-}Hj!a22EncZ@yOAhfd7dyU8IvA%%!J@Xn7PSEK0Uz^u##7vMj z((~3dL|EYA9yFZ?j)%eN&1X7(VkQ>OiCXQc2g%5$K-_R4o^94Xd$r45S_x|hPeDDS zOP47AQa1oCV;d0Q3L6)*HDoKh&bwy^FR=O_=e0-i+NNH6m^!SN@alp@VO9Xay=Q5g zbaF&;&w$tcw&xnBj9Y5qN_%rJPC#6aSAMsQkwoiO9gS0+o+y0-a21NIH-oVk{$m~g z>mV!ra*!Q`Sghnd7@kL#k{rAw+oh=TC!0p83}K==dbD00)7-m<yXXoL8lT@6?n0*a zNc9qK!FN0D*oiQ-o$?HV1H?_h<4mOKGA!}#5$ZoRENPQ$v%#}OA+D?F5UGG5cua_L zyK-IRt{suf`gq;~0SHWiP~fKHH8Fm&f-sK}_#Rl`IE)K)*ZOx4#IXDxDzl+GX!n7t z5m!(3dXQmoV9`A#pz%={km}qfac(oc=0;?>LqS}u05UhC1<Oth1(v>JPS~UWTvX|7 z#uc~pN~kCMZ$MwqXJ~?|=EYR&icNz*j)k;;3%Sv@>@9bUd<v=f3!@<%K40uU@-1aa zMKfJ*z?Np7)RDAxS-O20p28^kU4`{9omBgxGm|uz^B|W;I<#N-V*NoWFC-dbHS5p$ z`@5yPsKjz~m+HSPz)Gaw@vy=$jgw_I<SXnMPm$Wv-%-*nVsCMaTl6OyOU=wzw>-E4 zf)?AD(Vkz`i?W(6-3n^f$;!}tR9Qks;3)J$Rr%+W&dd>~(dU0z2$omq*00wAx?^nE zMEK3K`DqyA>|H&9YJFION`R$9W;*p#p`2&<(v;%d&L;!y5ViPv_=;v><<Oj}(lxh( zIer#+iS!pau0z-ky->?8gEAa8-UAFP{IP7ab=kHPq8HNm28;K$5Rg|!$_kas4r|tR z&S?h?q64UI3vfWH%Ygomyj;*~Oq_Sb1-Uc1s*(Co@ao!&^_nnM-oV-brB>3JaB`g; ziV`<CnL?AV<+rmJ8~J!qV{@q12mg~2jA16tw~MRpvx2_g{HwF2-@Pa+c~JZ0Q0|hc zjsLfrQGID~@4{M~CCnUL=$akuqP{nYKnC^HxQ~rS3wqvi(p)^+BN@TUTLVIt3sS~W z1h@RSQ3!o?Y$K&{AsuQ1VH-EGKdpr9$;x<Z#VPMjjvawl>Hk}=pPu{&Hh@PxNzuEp zdhTT>vD33Bzmtin0V>9*!x_lm!xltVjeb>g<DD|Q2krt;vA<+Zh+#o{GRlkdD{IXt z4o<XuoeSXVQ@P(=b_V8A6`nl(?M1H0OOElX12~+<_k85Vm@P5Spf?C%J3nyj5tY!v zGZ?58rl<SJz`~GSGf*jo#f=E~VtfaYWUelw<@{It)>(YNudOt!rC}x;+_9tSwL&W& znU0|=S<1}Av|@feugP88Tc10&rnM6Nrn``*CVL2AvdhSv_8NJg&}YsrfShS^gPZpW z<ey(+LT3}pc1J+m25X&^8ZqOtA&<x(-fGP84$s${DQ#!S%tLH;v?uC*ghw`0qe4q^ zx_*zjN7$|y4{Txz>X5{O8FNoNgjnOdKa%^7pKQ9qgiV32B$wq&DWcpd@~)orow6ui z=mBD9`#qOs$I2wHrLAwS=vUwL&Y=PXq~0qk_ZDWPYjQMmWP;M0$$3P9()beig&k$N z9iqIK4)bOtVqdvV>m30y3AkGMOG%oEmtoR1gP*u>t~sc>hD#kcI1g1dN7Y_lzE1Os zJ3$;v#91^b+S*k(wS72%?#wYWU?WtHG=bCI_KEt52N<1Q*?GFo3(UpRPA2CiMp4!N zIghLG84T3}i~3>Ysb=34FMw#X5HpuszuzsC61k^{cH7V@E;nA&YFDe(rcg5EnBJ`J z6%b|va&2WB2HY2VAzAcmai}NkpC}^%l_xb3HR-i4tgavy90vq)R-|-mcutD^>((-s zY=yBUDg*k6m(|V1a-5Ll?W{rh(pI>Sogk+4IDU^(u1(W-((xo1f1#vbYFMiyj<(U) ziL_BR>5@dC5>F@q9^BBD1sp;5w*3VtrH$?>;ds*t8XZ0a<gc<$oI#hu>e?CEKm2y( z?g!xTza?AM^Ltj6D6O4EeW#rQXaM2V!MWEOhL3mV#B#hpvo3#?E?*t_Y@j;z&eRHC zYx}&B@~Q3Qm_By;?z(5t_R@&^$<dyX)$__fTr<hbNT%*s{-pH=a=a%8Sdyzf^#8|m z=-oTzIsCePE4Ht4dT}McVbefb0WJ^HYW5?hjxn3I*8ArJMSfp?pNs>Ks2e_67tuII z3-MIbx4(^nAxMVyk_cuU8C|cpZKJ!A70D~fKPe-e&npk&S!_SFm5jt~S<up42F>aH zwQ<-7V_b5M6@LAV0ZLQ}6dvsJ;u@|GtfZnkAPCh8lG&K&5CFB^UG4QT0hg(KfiG)P z`$+;+pSb?cpsDd#!8qjOfd=J?rgbD4DEM!k?5sw3P_TcuMF3rlmIxo2+w_;R1Y+}6 zNb6g0Wr~Jg2a>Kak+M2}?HGrw>v2eSv)#MTl`@9no#52&;{x@7eur}IF5DFADT=t+ zBE}>uU)yTh+*oeGCB+S*Samq4d&zKL2Q(M10-0Ds=fn}KC~|gav?V5I@7Kn7-mIJ< zsv2m&ng@yF+U5;CcT|MJcgd8sxJ>H>mjhYJ3|#BzL7~lF%M6#QR}H~xyA$R#SaWX2 z&7tmb|JN`h`DgB5!=6WZPzelc3*k^Q=c&A{8f|%kMcg;RHBG1NrU$J(PuCWb{FtSY z$uX*B2L}x+Citw@&^W&7{==|CKi991WuV8;E8cb=z|-W%-=GYuO`i=^*e1VbSvqj8 z?PMGa#)l3Ihz$I3S$s{Y-M@5*ecVijyZn#6P=`e4(wS#Dyg#~Ni2=t!&v<uHLd}F{ z_c>@2Yy&1l!gJYDf7+rY2dpIAAOIR(ltv$SDfes$jXO8@)@Kg#3H9SmR1?Yz@Y0m^ zyPMr(gaht#X;O7C(v=g>UKb=8s!PGE2ULzc*;Qr|^*A2@8sd_}*i;}XE7`&&jlmhl z;V2SDoAsl1uoM;Lgb}QTW5spWnpdl!0&2d4F{42TcOz^9r<9LYUA0$m6UkPdX{V9J zHU%+D)?W(Dzn$!oB7k=I<~$JQI3l-m3!RIUsA`Rdo^u6E9+AO(?OSxi>fv@#uB!=` zZog<!IAl5DNMJ7fht-7fC6kI<M9KLeDF(0<V1ZdZ#yA~TaJ*eDs;wS!5v&#a$os5w zUaJYV*6sLrM1YSPA#gc!qyJl>&*;JLgM`vvKF#470FCq5VO(o$gGHbOxGIhW&YBA< zHr)Eblv)`LMBQs62h9e|y%9DhBF{@IUEUw+UZ89>k51^W)ODX<$?RR*;|V>t0|<;8 zxwBh_z%qt73jY%MBg#R0Qvy-$S1INp9=(N`T)tN5vy_e}As?rKQQ6Gx7H2Xz7swkg zC|QNJNL^R?-BT4WnsvsJmUfLUmvVIJ=UbucMKK4psJ+kUzN?DYJ=0#`{2fY<UD;s# zV9d+%SihZ(P1d!U(4FhYC-N1Rn=Jrzvuv{j&lOc(tR{Eyx4>#)TpO|vf#6w4;DIn} zQ~;)>eXp020{%jE`?@88+{dnTjI|LN9zRi`nUB@+W|Ozax((M_I>q#NzfE2{1V9gR zsOZn`O0E+WOPKAioN2bk_ZX#P%8s3Sw7^N;#yxdPUZ$$cC|?<ayObnQ+dShSZj%X{ zAhc;}e0;OZs(uc*hTp!Fj7HfinthzIrSZ};T`*k(g6wCoYy7ey=3?e6hhL7%U*QWX zx7MH5hS~jfDOoM{$f#-vONKWtT`5<-b$7>kl|09M-12F^<S#O>h~9(n54=z9k&RHJ zmbNREwHIL<L$&l<u*vr&dw}pm);iy1QSLc?l<qw3vbF?aYAf&H#WDeZB3e)pG~&If zrBv$<&I)|@%}O~2g4%d(Cv;<XwH9x?T-Ln)(}4@!l=wYZ5|TI-a<IMZ{&Tz-C}eDX z{eRmgdSQ-6vD(KzW?GNT3)_L7HbW$2DbG{BlMx`GU3;#-tpC4t40GIR2RnT3TjC8R z;B0>75Nr6vSPb-o?f1KvQ2(GW>FlDNl#m~pUVz_x_-<01%O@F0>cA-Ln6g5WbljC^ z964_Fsa=TfxO!B^fqs_ZJ9S>e4flMFhGeiUK}ym6i9g=c?h{ixD9VUyM30>e0g%f; zY=;0XV~og#j$Fs>!y*FJh{`Chtt$hEl@mt9y%K(NWCa~4?&s>9M2rV&1V$UnWbFW{ zJ5~@Un-FuS`ms=TY@Vto9-WYJs0!(<E4Az-@4eXx3W`N+-vl-+a{}XJ6gP@VG*aIZ z^xZd6BO#gxJ#+UrQNuc+1+4d<Jo-JT6d}1HQp)Xa$DnHo`-^Jg0|kX5FW*Hj)u8G@ zu{BdFw3v9+KyLo&5fPbkK#<bOJfknkkP5d{>c9$M<elw2VWJ69wG=P|Bs$DIRCSNU z&TZL;AarEzIBi5i9$;I4s0xszbw4~7EFp~SM0Z~@SRHfH^omUA;B+SjmA>iX_bmND z%~y#maz36rVmJ(p$5T&$3TaQ4p5sTFWBREC3v)mEs*NwRGL9eAD?S;fiDx0KGkBVz zEBCBy{K739ha{)Ae75lHChoI}o>T5)4>#Re=x?q@TVZuG)~cN}a8ryYA(!1&1A`6k z`@smR?tOYTrYD_r!{y1nZvDYPWmH-1^w_Z_q*~vRS;T9|l3~YKqtwxEQ+JKvz(6YI z0({niKQ1-osI6?Dw%8QKTUk}D$~9QW>r?b#lnxu8N$Mon+c6_gWEpTEhuDCgHd%Sr zu_7WYL9qn%qiwy!>5AgSG`}Q&Vzp;t$4<?dq2P_1p1{IhHp3h|<w9nocE2+?I(qSM zP9rG=fczp0oW=gEHzCOEPA=^h@b3wH3ZG33-oYQ)zm~1L55kgNDXn{kfTj7&jraey zpJy(e0?gjVEN&?!AR=hZF!d5!>ZP-w?=m(QS0$KaHUvBWe7oA%0OPy%A_r#w(Nc_; z`DOvP)p{fgq)?1!T#CVKR!dgN+E)U;Yf^?0cv-h>BYQR-%W;4+;}&5N^jP!U%Bm|i z>Tn-8J)WQQu9dqS=5d~ry^XV!I&0%`k1hQ4lx<t=Z6m5o2ep}K<3aQqVc22GZ^#;m z9vr*n4akfKP)E=XZ}Eve=?lWpWcr_A_tg)mLIsHvD=QJAL&WfMx<nQ|^N3JoEo{Ad z4rFouP4uot!E3s2`D|t{0FRzO4F9*ZYApw`h?+<~-HgXRwhWhgEmkwD^$r%X=uM)| zI<ddYi@M=WOzzwWqST)H@FFaf_oqHgnM}{<Ru<5JALv|G=eP)(FKsfRnIb^h8r&Rp z_Z|My3`q9L-+DD>_qT)lWNw%N&r`1PPG>uun_E-@6!fW^#k;@5@$WP`4|h>A#T-@s z&Yzcax07<U4ZEYOxSmOJ^E;K%djT0ej7)o@xL7x!p-!pg<#jsC4DRIN9(){41l!Xp z<$=d4RJXP1*J<Z8A4?WoR9%{QP$i0tP*32pR7@Q8*|u(Y7Ufg2qO}-l-dRWMSd(u> z-d(ny({ZJQ;n~LAm6Fzvs^j0*cB*q`quAE(m9ZSof7_e(E)9l5%tpCf!`^qxQ5K^{ z+f1rG$Bvq6N?3G$baRFkj%1hNA2<6D4OYs#dEEOys!iKcYh-uz7qsv|>cV?N?>w~B zXNWy)Y(l>B>G{Cgusg(Wgputq;vXWoAhQdQLDJKpS;sX<PWMA0X-1vngp)*CNQ;Rv z2sDP;n+j|PQbpoU)54NNAuFXR*~s-q&xO#<q@0I=K>AWgq0(qIXyXg-SZV!qx=Vj* z2jlniSCXxLd$&uyP}1<G<-)gkvT!8hDP9@^1go0gYtm*S$9}sf<nU4UuU}>7|A;LB z3+;DGn(Bz=<vHds*$b+-CS?D~ICUvEMf+R~^yg<5=5j^?4IYtLrGD|lc)Ky5Q`L+6 z_A$yAq)*irW07{{3SkirS}eUR!l>BH!=V9im}_-@XU5l(DZTB>QC`IqSSW<fI*}-9 z5L}XW(XR-iN6)E!M~J{S0?*|{VPH=1Zp)c4^D4@1n^6VH0S<Rk<F~Uz49gR<sVpb( zI4we>;o4q1_K9L#^kj4T=B;TDN7LRtQw!@hJhiK}uvYIUHmG%A$NejA*r;j&xcK&p z^Jv!s9Yoh%FL<k5+<6K4#fS})*u41<m&Cc>2DTg}GWHNUE5_xZJlCybcD#Md^2=u( z-BONAe@@288=Uq?EVl=ZtazL_LEqY)EMLaQXFLt`sNV%?4z|7T_+!l>NLtfsv-uG5 zz^;U<pe4B;5u*P+MSNG!4xpkya{}KHd8jfG>HDMjHgW`SqZ)gNo37;Z$ikCMlDR)i zbFCP^aA4e8aQtIQaiXpw<~`16jW)kaJ*?=rk*#@Lm9v6zT2EdOKC}*Tza7miE5^#w zHdaZNbIuEYz0K%P*eD{;^fnoy%vYL~0-)He<gxzXK`vz^i)Z-v6y#m?@K(CbiYCtO z+7umjGcBZBcg(Oiz-wU^fS)7odabr^88`1*>`eih=`oBrKz^lrciGYk3|V*<1t{l- zSPft;1NurWgyMnYT&)jUbBnaoS~0|N9$q4hlv$tvqwD$K5V4VS2}mN{fH!z}h%nli z0FS<z-N1m_=(0(`sM<8##&j;PU{d;jaZsaCZQaNqgly~O^1ID?jhsE+$nBAPTrcHp zeS5{-$#JS8+;!;Rq)Tfa1IxX>={wkX^P8ESmi_Zpm>wW6Cd#!?ZzHO6Nxc#|naOrx z!3g0;iDveZ94hyXw2%Di?L_>?A1&uba<cXQ-;yNy<7r9ql5-WblFVZ-XulAnmZ#&- zbj=`rT+N)9zxp4j;3ZR2oeLN`J0w~T-}A!~j4q^#u&Dd$ISm|IvqNx<FNKsO*QB>G ztm@;8ubyadoO(qE00tpkYLQc$R;HFyK>olV{u3<VhG#!;oT@5!LvgM=x`2kWld{|F z!jE>Af!+I{Pp0}I0ZC~ci_t!uHAD~^bKo_}ov7^+{>E%g7T*PTDIJ5_ahgtPAFr>d zlEUjTBZ8|n4v-cTEFE9@4|G;uAwG*BiBlEEMM1t+PXy922_;xPg!M0W;YP!y+;Ggy z;QoW@;=C7Ubn1kDbk}{m&gHMqx#inrK6`d()>l#^pdQ?GQzQYiIbMTL30$d@fT@#v zBZAa2ihkHPU;~DSefe^dt79!#r{BM*2D&%BZ@<LF!*W)Lr?e5*Ush<&vXe%x#~f-j zGfNzMg^QI`>_nl>*u;YcD8RH#nM^y@t3*CukGyneE));LTzg$$Mt@5|(udEOyvM1o zP1bf>HBCE6vV^gu>o3Z+4C%H%rty7UWWSOd%d9VDld6L?e0wyJK2qL&FyGshjU7#j znqVW_pH!Em6wrt&dw{~?&Q%g|f}nrL(qWjt;JZJK@DX(i?O_R|_qS(oN6Nw&%PrGU zQsEG*Y3@k#V&TTUYDz`feo!nxtd7(`9g|IwY$6`G>m|H$0Q8rhs=H{m%e&UH^P(t| z%j%$GO|X`1{KB4CHDR@<brFK|;6in`<($Bau}!=0=R7l93L1I`h=qLdNAeWnRYcA~ zDbCgd!vg6>n!;8i)ElZgId!`2G?~%X1lynC#}keT1weuEw~6=h{-V)VzoOA%`03*~ zi{bm2jguj*y-C(%7U#=-*HNbC4pDwH5iEs^IpJ+d%DmG^pa?f7^H_U#3^rS99ZZi) zZJgFM^T}3a_nd*Q5}^+KxD}<FJsuH}a`!}nom4;Z0uo9M-WFK^IIG1hhlKG~K$^rT zP6JU}s+ftxOnRU5s)%-2sB0)d#{Ty%;REa$o*#fc11>fj#}aN{I(L=Wnf%L|if(3n z4Ehd!8{BTkmm`p7t)-$Ak;1>}!Pu_}Uu?4iDXkLu=3sq)TQKfiMMbH=K4YI~$i9q5 z-OJBovF&Pt<=I%{U>f5E-=yn@;vDMkF`CS63g-kFxV&a*AE?7KVi71)kf}zX77DXc z*4%``aD^u{0?o?U$4B}7DXMPInCs)`8l^Y*%Q#E&Bq9UsK4Z5>!4#nyC!@M<PIczT zFYAki4hPR85m`J(O4JrVE~Ux6zkm|gI$ClP)7JJST+&?-_3P`l{|yQ)%9N+}&AZDa zw>Y{tKPlwU8I1X>mPG@BOpctHFg>-OVlKr~g}euyeY(k^*-6K6206qT?OZPAju~Sm z&Bv+z?6=JwKBTWb(^86=rC)d^<9rg;Rx@xp#CUS0emsXjz*4McNtRTF(tD20m%4gS z1s0mfxbC9tyA;jI$q3)<eylODWOyM9au3Zzr{7$sFE>EpMJpUQB3T3O3|+|xSP#j~ z8x2XzlZ=Zd?7!Y)`EG_N0b|~O)+~Suems7=iUkyRdtlBZc77}MzP*Tmpe9n}ljx%( z*#HEh0)FgGhf=bC!_L(0LP7RcjgK|!<Avk0M2=*UUWSO}TQh1`Y*ua^X>zB3qd+T7 zs_gQCM^N21@&o`8_&L_Gq*3z2#p)VV8IveEpdifZphbKQ%<MUfDS+9yht)Y(zF)np zTv@)*6iNJ2`Du+23~Q-*QQa^Ez69|ci;d5?c%v|7$$Sx#WoL<mhPg)3YLV~(h%=CZ z5gV@_f=I_{Y}m>J{Hnjvt=T_oh9Go5m_M$>Lip_d7hbznbZ_|4ik(Ixg#ath;xV_6 zu&o+4UHeqtmJT`D&$f#f0@}}IYA09yQ}rHbmOsv7ITJCsJn|YV8s4H9Mm-rTGBO$~ zG`O&Hk+aV_bmL3O;NNU>1)gjrZ3jP>fsS`&R}nqWke(p%uN3nOj<R36eu~VjPjLR3 zy|c$_1lEnLxcuuA9shu$z4V8pSgvlh*XI^1(|;G?I%fn<{Sc9tn|(g$f$qIQ7R@8P z4#MELh~YA{z2rl=z1{`yjBtr>q<mo!RhpN3#>Ap8|H>`<vxe;UmJGM0Z}bfCmZbR| z7KdepW3G*1{CXqqh98vS7MC<Xr{R?=^&AmnEpKp&!J9w)cE7s@DvHVp9gVh;H4x`T z;<!AXG4moloIJ58Q5mX&+oLO6-lgo?GE`p?BaW`VFA0Z=(AC;^UsPrk+_vaIY-nZu zsh28C`K%=f%{ak&teyQohI88^bhFa?$b?yn0Bw1#@As>qo9H_{740G1ntP;>Tl_1o zwGkq6G3o`zXSF7pSC-bAAAaRcV|>X;Sd6jt*4Mqk@zC@FBTnQWEVW=;Ok0>Ti9bXb zXl3`eHfoh|7IRqd8h@-W4P7>_Hw~BS!C%}dp_cR7&o5{lKFp86@0Fs4>c%z+sSzlV zV=H!1tILv}t4d9gr59U7R;fN#Cs5dH=Yh)G7bh)a!f`x-E%WZ(qe3e(-?!Z!&OP_) z3tMG)sfM<#rsN^+gTwHIEw^e<y=H)s!41~!cQOPfH|ibTRZr2U`eaNM)Q(l#eOZbA z;`Px$Kv;{6rpx?V|0@A;{Uh1CdD=LCUYJAF$I?1S(aKfj22`_zEpR~y)qAT+cS*<b zD-F&LT^Bf6l(VI}ykR2N9L>zt1qgXJr^}2_cquF3*#>=)S(9e+cc6~wJ9=_=VQX%G zZH<!V8|5tm-LlgC3xPLvZ1)+@3}{x>$=iIQ(}k9-t6hh(r?wqhcPUF9KS7Bw>5_5^ zpfeiu$zwtjUXb<&14mDrT2vJH!<qJ)jOSIoi7P4hrrG5h#Iizu7Hj>3Ud4VYt+T`r zk%Gu^oG(Lr#Z$IH&l-0VG}aq9OE~#!$(7RW3_Sp7<4FK;gw^+Vd;gm>xE;Yzt|P_n zyR5R$K+3z}ts^x|veV>{RQR0w>toDmjBrpD3#c$nOj+!F&yu?a7nu3kET$GvYFgf* zBSg@qk1<`}YIeLW;tTxm1DS|pk+qMM`^2vKs&ZV9eaXPgE4pW;@IhV{?0ZT>Wpcds zxAHBo%c0hz-w7&C`@6Bi>+G=+*8ng#IzT_bV>RdB5NBdV#{#jyfHMPuwj)Pc(i?BR zBfl)PvF56q1w<UBx8r}n@!L^IGxiEAdCqp!UOQ3Av#z7w0QOQGTqog){5lslqmW2@ zNEEqWHQ22IwmbEus8~svbHI%^TLj|Gkn;Z8I!2@_#NH7^KNpr==@_20HTbCG%a^)) zfin_T&8a6$1BD*%eWP1v&QMsXKV*8SBwC5Yu^JQG=H<W#+AZAgB`%0YyIrk&HlQ{m z7*<}2&CtC>C`mgq8;u#w^Bl+vT}6(kC<}x*@J!YfPFcrqp`^y=qR=HM`R{US!$Wa` zvCTZw11je7pj=4#PU~nC>toT6rL9iWJ{v42B^-bDz|QO%oom-W!aa^2^!X>J?j(6r zY0TTxB<VpCjQ8j4VYw__AwFLKl%AnMt*+a^IvRuL-rrUz&`M*1Y;`~MB)tpA724IQ zViu<qZ#R#sH|%KwxUL9Oh8F$WAnykOV+=kXylb{>iFfik&hi`Rp^#;oMpOF+Y+%fW zO|Ii*S$;xFNA-gw0<}`r*5klBebJOQQFR=0Qxy^5(J>~l1;Nk0vMH!^d##Hb+B4T! z5lJJM>LG=|SGP!rW$;l;$AGYGs7zWo&?&%hZ&XC3*OHXzC54DwV=hglYQJKez9Fzq z-A9!(eI<a$x(sx&KXHn;E>?~iYH$#yrvphS_Y|9eWe07YT0yUtjoh^qEcH`a({<tS z!ws!}8dj_NM$Otb--~F6)7R?9PUDoI7jUEs;Jz9L%ro&?SFC-bp>!EIb>!j<Uh@54 zd8g*$OqG}(7kRWSPkHcYBx71V^_wW_mCCPd1)nFIQU}nCu=#B}y(oFpt4sueE^7{p zawI-|fxl{;!BY5kTz<m@&SSt?hnx7^(f57uF^_whsK{C_MEXzcPHnM~fusF^7!06K zzU%vl-kN!;c-(OK@wYI*Tb%I?@Cc2M7cb)y^9Z_Hsh4l&S^dM2ijYEchTEx;V-=hH z9k(%_?{<a4i`nkhPZ-r!AB2^$5Qo!-ZokYTtnY}ngn@bpyW{ADjV7rG*GdDWb@x7U zllOwx1Cr0=H%59eJX&Ro+^WOy^9}AdkZ*=7*E;DQj@+RY=J4cP@9TeA0Muy&+J*_} zT$$b>p?TA^Y+?wv(4Io4*xSyQOqFE4`9=}0E`7z@pF5`UqM-#~!9|G#S*=O7ltoWS z0MH1U9NgzTtH+-n^@QDNW;407-9FN}pBfZ-(tc$P;XYY2pC<xio;-ga?*bn9b0H$6 zVYLsw=BQnLJJ*^d3jwC_0HVMFhcj>w90Y$IQ)91-6!lQ9dGUOFG0zCsn6s!&tMxml zNzg@Fcl(wG_^sFM<a|z=h*5%*+v*uI3bfD?jv9;fw-c}NW<uq8B}y<=*TAuZk#|RY zost5ertb7p#h&>;iv6Om+Da5tro2ZL&1GOyv-<O$NdQVeb*9+HMttz`HgPEVD9Q)& zOcYNs$#T|i6_HGPMTa%kpoxl>s}<HF_ETHiz!1VZTik@}f_lShhdF!Paq`fs?4g(m zyPEcJ_oMoCttA2bSK8JsiCx?+K(Jfd?%**HBAXz5(5Mej+>Wsg$g9m?J}RWxl^}9( z#kDCOZncqCs!AlEnN~|8gn@7!co5|n8p<}k90+PGZ+rQU?WQ#eh*%7aIwol0O6lH; zbexr!mYeu~2Ul2P+wo5MUPU<IlrAH3=##&;hoEP}WnrJL-?aV#c?SXPjg;xpxQWp( zTk8}ziyJr{ZL<8AblLqY_u{ux)t>kH@-kQh8mW6}El!{={SNlm2KZGG0g#s5X-_bF za}-4yn!c6#ZLjqu&wn;ZHru-yLHW>F2N|h5Dfe=jAF}#82+oiM#QK0)88@|Id9!cv z&-ez|3_a|-9v8Z4a-KiZBIsI~o-Jp!e?Jk2)y)?)lRJ4x@QmA^15gB)%pIOXU3Cid zyKe!=XP~kZat4$yK`(RqLuw$SaKGytvV)2DgNcz=EwMZqcuQ`1Qrh!=B8TPX_)9HQ z($KEOtm9yK!LbSBdu2!CmCxs-Ti|iXOgr<#cD7YLy3zuW#H*cBX9ki1e|OG=N>qt( z4f?Qu0=|m<lC%z4anLsdIn)XkyWT?N<$`0ngY}HAsV!AAJIyLp30)uBF|N=@NM^Wv z`Rok+9xnl?2EDhla`cLX-<#4t7d81qJ-}Lg@@i+}1gjWhFSd`?^)Zx;LdJahPH@RF zzNVn$YSEubLf+j7CVW1F`ch`4n|;)hM|)1@E>$s(*_1R-z#YUt)$p9m9UA5hQrbF8 z%&pAXvWZ)t%w)((q=j*;<#BSh;N6N^{c>5GMq)Eu+ZNTPbEpb5i)a+k({6mm76g;F z_k!d@S1q<m`R{KuL05)pa@l&LYAM9su7^#OQQFNcZDqNVj<x#b#VCv$sgyscRH1ph zQRmYYVHT+;%Emw#Y{)0wp><}D>NP;Xmf8_E8#o7=4E8ujtCVl{xRh=}{vfEtJxxb3 z+ECm|s+}rpX$h|7G1wASIosh?VCm=DXy}JYzq$-u1_n9hP=()4CIu*jpSXiJJEJkr z?`3osoGyHfO6c=ifVWzywxsb9$e<sxoP@hALvTP0C+(@BAA0Q<Z$#N!Ns#x}*nkNK zoXZg`B1^k-0FRKPLJHm5{S4mb067?GZgh#@rT{5rnJjuj1JyPK4lt(fEafMxkM#V( zYx7QJ%5SW@8z`}|Tn01K41qpln*ckxFWdf(_CrEDt<WeZ8*(s`u<Pj>ZNH`jxy>7+ zctiV4%&n%2|9wy71G<r#&<E9vXF-jsPvX>nE&e4zCVi>_E89DB=YhHOl|eXEk<V8p zxN0e6#<Ai{`o2X{WwpDIH%XjZ8gIvOf8xwf_n&zLcU`5GR6N>iE69xw?VAsWc9$-M z=2-;Qw!K8@0GozkGTo6S1b4=s_{_PO@|`)3>$G=SerIC1jQdd~9d8&ob)2uFsE)zg znh#$@l=?Im?g|8@_UV+SVW)TGU>59<x}QczIzPjMIByG6z1%Il`WNSl>P{HMUf=G@ z4UqyC{Xes8j;D)0RZ8bXuA>9{HZa7hj&O>f-CEB-)j##Xe;dQKV)}?*eZR+ZX{=Vn zvrce<Ak1ggv`0ISYEZIg9ikj0HRs(`_r4=4_<kmOL}PQn3Phc2l6Ef=W&`2#2#eu! ze*OHknq34CwEH3U*xgi$^LYFK#PZ1FXN8c=;#}01Kuyr^s;(U@HDd4l*L}nCH?#)5 zl#JipYt{^#UL61;hQ%b@3)<4*SZvPxCWuoc!aRTAeh*MFNfn6(j^Ka}lIRm*dE<3J z>i`AcBNn@oc)%5BmmXw<myH|prav+`oSiAo{PwtML1~Rh*xveVR|AJJ+s3)P9RdZ( z)w`py+6k@9wskHmUyj<>I>Mp5(1N4%Z{b@leSmt~q`byoJKyM&7^|FxNe8YW0XJsy zN2v^Wb(xJ{Yh}$q$9uTWwW8vHHg6=Vl(fSP`V?gcJtpp26kJ49GQahyunl>$GKhHk z2?c+dOg|K>>|FuFeVUf(DVl-phRc5_3>#@K5(p_>@}Go*FpC!QO3urBDj5k=*Wi!< z*jV7X^YA4gZfEsG@P%_703~b#y>8&jcl#l<rq5j08B{LtV(C^!P0#PI&viU{P8kJq zBHtNaE+c^SNFmS-7qz~4H)?lsfR8WpMuab007%-{DR%?pMid@pZ9LYev}_q#`h2;B zyzhe*4*US9Npe>O?yw#14}}{hKn~nOcD|X70WNX&@n#C<1F)HD!1D<L%KvO(zVPxE z*7Myt;xzfS5q{kg?d1TCzYF72-hP4>;f(COotXCfzYQjU1F(?HK@F28Y!UiZlxOct zxOvJt+5WyQEKYsQ<}_E*XUY8)j9NT~DVia2vuetRT`qhqilF6|Sz1)*$;Itn)EOrj zpzOv}b{nk|3PW)Z*<lEToL*%52A0e`AAXcBr1Ko?Ev`N;prO8ZFYf-bYI*O`%SiK# zzcG7n!Ipdfa;a<B(}?Y2z|XpS0OxQ}f{5;xr8<`GjwdM9c|$%*0a<Tc&8wX~mst7I zjJJ&`eRJ%3O2CIgY4rgZG(PuvLR~xv2w!*29H0Q)!J2#t2TVI^rveSM(2KA@nc$-G zi=TiusyW3}cly|!%{kw@A_<^p#Zy$<WwDzB0kKJaGpO|4#823O3VePuOZ)YX=JZbL zblceRqEB-BRiSnX*@Q*5cg3utdZ@=7fP2svGCNdqDN0hDfY>r+uL+9a<I?HdhYwyO z@*b6Qh`N5%B_n}woa16zJuY>q)XS8QU5`&X%-(R<el0($$o(j}e}oUxr2qxjyqj_K zocb{s_~YnFcqi60l`SE1@$)6=(1pf|h=W8IO8bB)B}!R<qlIc_LZbifqWT30;fV*l z6Z3wsTE-x%ry)W!BO}BIJe*AicAztzZ%^Ftkd?!NN-Lt=RJl9-A=bKSsDI73^7BEO zN%L`tw(EGj*7OiOz2U);N_lh8p}74&{?m_X6F~g-@7jY{*R@WvbK^F{Q0wE9j05K^ zy5C^1JbPY%+q0KDbYb;CahMY>1Dv>bE2w<%4>i=r8T2IYb*>jGw?O;A$SK1_{@@D8 zJz@VRFXVNX0sV961ArqVD+2nQ_W)!ey*ojal+OuGwgKTzO|osEV5(Fpt?U+HZC?N` z5t1{)WOi>FDX(>MxV1c=VrK~O4FLeV<HSe{zVTux{9r6+^q)^<g#2dhll>tU1C?dq zr^~f&bPFz<<QzP)w2xp4flroRicCPqftVvHptI+WtAMbp<)>BCb%3SN2c#z`^#so4 z#71&Xu^9z$xA)1G9>sNd^4@<13}b%<4BaZ|A3uRFYy2VVublo%ptSD+bg{&Hs2k$w z4ka$8aAgbbye#W!u*vD=5&>o{&?#iOJLa_dT;PJi?$Y_~Yf(~w-YP=c&<PR=`Yj>y zl2{jn<)oD1#AA?m(i3-wanr{FCcXvWyjHMPoj^@&E9kjZf5}K7f!TV2EoFX=r7a<v zT_!^WAVAx&`)v}Q<UU?a{u<@2iff_gLhdiHtMUkBxbK%_s00j$0P#W^LmP-8wD++J z@%LAP1_1!d>92#SKRm?vJrzuGtNo)qf=&n8ZEx%~8TM0wIhQrL_viiTWvDm|sBZOI zah<*9(rbB});L6iHS(6gi4BA6m$tce*w`zi2=SXRI&nz*Ak);0I>GrK>9_Yz_H?Io zh=6L8cBzPB4q&H90Xs$LT8!RzsvD={SY>DY@3o|);0eHrKx%OpEZVEp7-5Ily4<kw z8P&!y%yU1w?L4$q*9A!Ag)E-zLb6<KtNXME5XQ?83U#nRc<6-k(0yWo62E;Q7=HJ5 zS$yGs<ViG`GE2FknYP_>U~##9l+P`c_vc3jvbm1jbBxQ;B1l<26!0KmV03z$<3PoL z?a{Kp^^MD(IdZ}*E%+Ufxd;va53<**7N%OiuJ2bQ7(B@}KyaPf`r&)SvtkIO{cin2 zV`{;g;;-2mVX&kAVw>>R9L0}WLhZx@-Gr9hS15zFA|Phw*6&<cS9Zx-t&d4n3G!kn z;)sA!QB@(_da}tdATWHfC}V-~*rrYC?s<e;)~NsAKl%t1eK##aaNidX_+%alFRZ@y z(T^6*;$HM=|0s+lk(>~{Q2EI+5u6B^X4{-%c)=k$1(PF!T&>&&Y^lqwn_s1c#k>Ku z|AFX!&%LH&7WDMX->&ozB=9R<)7>>cyw4esDkyY@smL%Z-QIdx5wXx;1N@&)NNU#w zqYv=<sk*b5flal`s<Zoxehw@K(FTpeHk0lSU+Cr!4aD078c3H^+<srzaW5PIkv>S| zgjOYu@Kjxt0XVwTM#OjDU1j}L3LgJM6x7OV8{~ixJ>CmHQPtY)xsnyU3*bnV;IrY> zNYRH-naVq^WgRE-)%`c!f&H#rzoC7yr7hfQ2i$vQ>ZfI(Qt+lzk_sTV8FE2r_KcXr zDZukDzHkWiLwip;C#qcr^i@fr2&|phf;X2=55OWLfoNq`KlNa(o|Fy9ucArO_k?`z zsL!CmqoG>^uGdAQ{?|FNcat*y`XZJq=v=ADJ+F{|Db)WY)}|k!*I6}5`AE>?B^!>^ zQSjd{IK<DIGyVW}GZ#8qOD{>Q>C988?$Wgk#1s&?TG22)o3Q(oEq&~}n`{h<^VsSi z-fVd)$#O}KI24!XcUZ|`AGk>R9NVD`vbKx~<eY81vUbm;U-FMt0o3Bp<AVpT1)|o| zTQ`nRi=>7iH#wP^>z@IA5+Ek~_-n-dgG14REOA^TwcQ#lvQ(8pzVcVX1+;=VT@l&@ z^3mCepEeZ4zN0EDs4*}g6=eonrk6gCaP=Yvw!TdM^_>Bm$UAM>C)#SM_D5EmX%RE} zxz^ci`afFzeeO<bnQS{joGIka2`cGrXP2L>sH&&+i&%C0lm24WqBIj%KYy+;78r7Q zV}jE^1lZhBb!tIoh+7jGt;xIEFd41H^4_^u*aVRofSD2}&H{5rN~vD5v<WbSrEaDs zkF8jD($PB?4wj4><1;%aZG3@-8maP@^j62k9qrd7A8M5Iz((2o$;p%wja$q<2etb~ z4-ix<ce=H(y_wX4FTLO}D~IJ-Nu#Ge-4O3Rbl@H})Us9fLl2k$#^|BE5aG90!Vn&O z5Z^a9K~h<~n!}`|9wO!~eS`R8&j69L()^8*@hqn+u<<A(0ApUIO9tciU!(2-zc|-= zl?4l&Rhf;ZsQ||7(<dMq^XnS?ZtA}Jsd0T~2+n5@2b+|_Y&bAq{D1Ui*BnEY$Ga~N znh}-q;W%`HG9rhNc~bhytROgLEm3E9TQTOEEt1SxV}L1FqY&6flUEK!dq=J3vp|WT zdT!CAXF)%_!s$feneqj2;{0iW2lO=4y>|$^C79!bWof&8{nHr2xW#Q%KtT{`zwFe_ zRN>PAIYrtJ^U#nR*8uT2{Q(c?&gBc6Ky;xyS^Jnq9s@9N*gqLOjJby&k_^x98_KcA ztVujfwP$|LFwshzdwnXinXPsLw`$g}cv}avV5-9kV7+L<t9q|#5rPhui!{93b{V|) zy9K=-;_bpwzi|)wGgff4-<ZJ6Mhg~3@mSuUoIK4tpE-)zU8)8s@S@X#cCKmjHIS2g z@Pei%TCoM3L=G?dSP3FM-sgrDXrWS2RWodHQFb~od7{4VXY<i9;DkO+$>WF>(Q0y( z{$Gsg*B$$=N^DF|C+~@**dz3Gi_necJGDXov3riqa)EqsROOA>s|-W>UmdugvirTC zQDjYg;BehQRB6~M3F|iF_)!8@`~9A+Q3?PVEc$i?F7De~&Q-4zUBN!jxhrelw)zNo z>TYN6g&W`Jq&ik0=?hAC`-u+G1Qi1JFOa~xcTUbD^i5x=OoSK|?k47G35<?WjiCzR zok-u7o^GnG>WPz(i&=r{DesThyuzV?7_;ibFEQb>z+Y6*d_q1HnSwDU)+$7r#-m6- zV}wFwiu=f5AcYay!`|j1thJJ@yzX=C6)~*kLp0$bdbI|<=w1Vc9EHpOtjyhR<r!xs zJpg?;b0YfhNEwo}7{lvj4X}8CNi#uiC-0{v*NerFl9;4QVIJ;tUy+<Cj;UAh;#!%| zq0}s3mV(Vx8jzR%6|z5SUZ$8`bh6$6uepZ#e#w3qI;}vSu^H%iwgfKg^v#`2VlrhP z^=V^^DOb2t)GKNJRit#p`$i{Lnc+5{V4NovA`)H-JSf2bbL=zLxLbX%m=dxYQ}udC z<FPY_myP-vo198Iljs&IfzSUYgyo{pEVlD`;Xl+IdkgRFaS70F+0{h(zq38;C`D2g z*UZgNuVarx3B1F1N|2mPu4CbphH=o>X4kHj@Qg4eqMKA?(Y8sXD<#qq<5xc6lwx{M z`b<21wUQ`EuRbEE^`USTCG=?&%Ta&c$S6T&_qP?>$n{{QncTpBHBtP5mheekYR4L0 z7RJO2Biv+y&LxT;n~=ubbnwEf^>9arDQU;G8stOI7FY*1h$1^u0my<Me8xgtvgqP8 z^HICOa37SwS%RWhtB*vz!O(9zJHd0O_wLN6t5^4qHGl~KbFU-{Tlc1$^~v669f6T4 z%b(fhmGCUzA6$kmZT88NELqxP`ef^^6ZXW$mH|Lu%rfj_8_4@q(;m)8PPc;-SApDw z`JaiG!D~RR>XE!N6n>0QpMPhfexr1rp@HYn3t$b*E@9^Wl0M%LUIugTRboXl+Gt<o z^<Vq7?)BLn-_-4PJ8(O3WBmS-Vfv{2<Uqcb!x$uH2a4GPfQ6c+zhBqk7%UQ?F=y(4 z9QR%Cj884akRUt8nn>+a{(U(wM#Q5w2g%UQ{VeWBgo+q@P^s%7Z6oWosD}$jxwF;y zf2H|QzzhoAb&b*(Jt#1#Am&j#<g|G*eu7?SDIw+K86&RFShx#m>Ljy3cH8lm=*sax zxR%8y=i|K%;H5?4PJ_8uf2zXrk0c|P3>ZurFB?(ODkdOT&JY7MN;(L`#8tW_X@|gB zcOPvx=;F1DCr}k8B%?6UnOozRH`M-NDWd?s5!$!cylwzYFuX9Mz9WG#I-!q}-6QZ} z88gT|0Jwk*q$_{g3gljYVbgHyg51iVlzoyl)lNOFC_Jcx=118(4!$o+@M@AYgK00` zvqd)+iaw|s*Apc?Ywe0D;bpPs&`LQw+#pri2%oCyGLi&`!KE_RD!Xq`_+}c8qDI~@ z1m9Tqy0WriFH~Eh1{llHNsL0~=Uk(xdh%~7!gxmnn?gWN^8I-OSG#sFYaqWQmy4gr zg-Z5ZjK4rCKl4J~b7loR!9KY)sRbZB>S=+SompDdBCCV`&(M#?yd)_Mx-uXkVLd>U zr6;Z=i<IrQV;f~+721)OLGgkbrAO4Z9q)j+%B>nM=Tb34TrVN8tcB;#zY#b#Q}b$h ze6*Rvpxcal`EE88e2Wy3Q~Rau8b}|oEhun!GOmO`W?8#GClY88!F&~fbc{z6=&Pr! z*j$^746}6TL%J;gRv{|M1!ztcXLs~QxhgKv(^x>;t7cj=zHzAXON@Kl{KU1mqg%}G z3o<tK9lRc^%}SM`EA|7{utx7Lw*N=fTZc8>_W#2hBa|9~blXUGhoYh&f`D{M!$@gH z3{fzs%K&Lm38kdFC6sO$F}lZw)Tr@0@w)Hl{yu+s92^J2#y;oye!r4OC#1zXJ2b1# zx+lMGq&K-rsJqf%iF@u-(#eyg-MFNENg?0??|uh&DAq;@e^1~3`K*a11vfse5F$Zb zA5PnTg576y!Nre=uTK`%nVVp5L|G5ke;u(`m~0c=pGF@jT~juv)wWU&&hCXSgdW>w z>epuMC~>?Y3>O9u=WDQ?EBx9D24T!EKWe-KpK5`=8sr}+rJ2n?ci-Hd73qj?I9}vx zh=B2ctunj(7w@*x3c0pmPyxWxcQS>8ehq7N{46?HN-FV&5Q2bu?Yr3e(wjy%9#@5f zh4(^dxP9HDYxBS_RXQRLe6(KRp3L6v#U&)9m$|E_2sEef7IzBq``2yv0Lk*lJ)nJX z#UaFez#KinSs4iOmgl4<%wKG@9&MCpS(7hOaQX3I0X*vbB=&#}^|afwg+}`8<!G7D zh70T%!acc#Sk*TTg=D0v?0x!!+u(fhD%MAc322l;PD4GM-*~(Wm`nK|8_Y6!2J{1h z2=?m`2nGfmUI_lZM>`mEpgg0bp%9y5g^L~UATO#?=wq*u?T7@LeKB)6ElJTiN4>uu zn<91J{TJ6hz+@Q|g^Zm$>A#2i)fwH%r!!{=%w<*X>!P$Yp0&<PV4R$9(_UNevo|G{ zyzTB+nL}#2+&{^>CPN!E3o(2Rci8!yc<ThOC>*F2at2LJ0G(36cGPI;VEs1n6<p}k zUV@*hvD3lFK}BZHc)Mvo?}`9*=0zv$a%9P&60@(o^;RNv&OSlKBriU)&bO|G2szD9 zX-_SSj{ZXSd<hlA2Z<WJ`RX#dUKjK0G_E`ilU@H-xqV5sHtRg=%fp9Ip)Yo>f$26q z8{(p$GyY}4(@9+HcW30!b@;q}IuKe;ZUVJgzvZqFInqL?iY{-d(iED9zS?@~)7SJY zJt<iEuQ<BWBO)`o_0u7f5@GFSjA8;o!4?6@ljQl4I3Ci}er3Lw^JjQ58n!jP(XW5w zMc6q>5sH+O0}MSDuTb7xp&e0%yE-Asb8)$%k2yblgy`wnYaK#f^>u!kto<pq$P2R5 zJ%{TFND1W4JzmX;N*wyY{Es4`%Ko_2gR0knCa^2YWMZtpET*}gHE`5v#%LQsDD<5w z-}KEB4VzGQROv&Dy!RnKp6P`_0#qi6;W7F1GppizDT(U^SAz$ta$iqXaZ3u-71<S( zby?K<+~SsCehIRBt4Bu15FTm5`DwC(@gaW7_J*OSS-p8oujEFvt0bPI8}sE|5EoKB z6YD6$^90ihzqzDVS|n1vBN~eLXf$AM>~_lSuO2(G&tJ26iSaKnG(qndNP5+V5C%;K zfqCBR#BU7R(dX$B-YeDq3bW2Q1F`Ag0a~AkHU1;D*V(B7H#t4KL|2&`2rsLmb(%n^ zf>;VrL^}d=xur#dtf!qEh#5g1yH?t(D1kk3T%Z}UL~?&M$h8JrPTI$}zfrs0vkLGc zJxORn%q;8(aB5jM4FXt;J9RqZk<2qkU<w`?zGg$8Dr$Yt_~}c=JG=jlFqxWuS^Vx{ zoBIvQB{my)!AWBVgrGX8qaoO;e{zBgQErOAG3g1M1X(d5$MZMerwiqj?3j9XOFB+@ zkJ^UN&?%T|Yg3&ImnjB&GP8^9N_yA;JcOY7<n*g>i|u)LBgoG@zyGsR`u&w6Emw|i zdJoccl})?>5te^HjcYe&$YMnsZ9-qy(y`~vLW|L;(sHGzESe;Ri~R_gf998s&=_T@ z{dLE(VSRonGG@GB^~z_n2DcI;n4}I!IsIX!nr}T#%3Tp9$*4Yvxi*oBHwC>7Rpt*) zzrpX9^hg~<>!}ILOi2W%$jm`3CzjTSW9s9H_%64>2=iLulbvj3Y-4*_>fLx#InA1b z0<Ym!wM;<H7nzi$9dy~#A1k7^m9fe5Yq13CQ7zizxS_>_hO68k4!O+`SN7uCWO~_M zIM4X%__<gw?A(mCi;>320$Y*tXcJbJM5TZ+DxiUj*03d?R>CiTpYb)*ze}ef<^9IF zrazp;z&*sZR%Z9?cj}C6Bv#hcd>zF<B<?uS@OQhE-3OM2>Yno+L|B9cvsut<+J~{+ zAlp&_sUSivluLH2c3dL^p6c->`2DHi%g3cg3bNhM8PxTdG<{X!M(mauWyo#i()#6> z1Oc8$j+2riG()#sMk6`1<q4>v{YoW=jpe{m>jvN}1J@nL7RP6Nj=p?nChRJTbYoz} zo~-xTY2CbiRNau9AnW-mDuzTWwN)Kv>MssZ2Uje<i(AvkmAb6{Ep{|+H*eo-G`}kF zMYF2DU#yARRknq_ZH2PNho;B>l$s$LT7%a5H<9^x(Nb)b$z=6vzZQe%YHgTLi6T0{ zfn#VBsbFlpr(lkcd25vT$pA)J600#T9y1m?#c#a#$m>b;w}zWFloHRL%Qg<Gz{nG` z8iPMh7VMuF_om+wX36@hI^BFfU-JI17NFAmWsgShWV>fUTR@cXr$KX6y4iAN**%65 z_2ARPu)?nl`wHdjqb>9*o9mA}+0$l;;vJTh2@khm?o*G|I5XpxfOH9KW0fM~@wqy$ zyz#%A8dodCi%cZ}30XM?drnd+hzbNBp%cC{JT>JM$f8H(Phl&JGmjxc@rG8fdrl!< z;nYp|?WW@os0W14!Qj!&9VlaXB*S}_t>SFvo<mMMdOsE1#lnujAA$QPA&Fl;U}t3q zY=dQX55gmxXO;o3<J(Uv#8C9pe3R1wE_NtXG5IY2i3-QpZJOiF8qA4nk$_DEaipSt z7@C6IhBi>}903B|G{%lc7dLL7HZ{Ktx#JlRkr+P!*3GYs2Bt*>$}tb%PcGTUZ?$vd z2tP=b!ePk(J~Bn<?}0pK(K8idp5obWHMlq7hPff%xEafM%#cR7W3cCj*X`&F#suwW zb_&(x^@q=<7S%}y$*uhx$OQ&L^0#N|^dFH!jb-mURW3$^P;C0WwEcVkT9Scq2xg@b zltt#_ZHSRMPxoL5U*-5her3@4pX$5Bt(cJH`n|G3#v}p&wc<gtQhfnNdOVL)BU#dz z>b7xv?X|}5SU0AA%15gH{H3obWLkX3Ezs%D);MHzw+bzi-qTdksDOZ^d7}M9zBR<# z2x|DvT)3WK6+oDckUu@zIz7VeEEICz$8`ndjxuqh5@+2EFG`t%vrRrkdgfS*UR)+j zv04|a7W6|lY!9f6W6xlS1J@&l&cY8$r9GRgU#PkVw`7e+Z@!IR+-(b(WIC>pvh(oN zrh=Qyn-8)y<G%V>(x^qn!j8O>Av+Q6b0Ow5>dZa>^+#lUb87u(xAJ@PW23(*i^Ghu zsMfI6BWo(K!pCg!T4SF&v1oVB4sD!Eh4rKsk6rLk6wjW{dmhWH`)A3zVpH$BEU;1; zAZ13W#QdSnY-mpvPKO)*G7tmGK@}rv+$|*6NlD$n-d1wahee*-`8fYsYXzUgVAYU1 z7gbKbW67XpbPuHZ+MfRs_pVtK&O_wF@k4R)>5YFl{0doN8XGsDk?2Yj*xZ-->{a<) zGeU*ehI<VY;S+nswG(x`X!z4Vfg=u<tRc1640%+lvkd1cqNaiv#dA&)397T-*hJpj z^{#}vR@I>xX9gXvjfXjS@2qIU61$47TNpx5HxZ|;2^P1MH795GH2>w}H6}(rw5Iif zV3{qhBOdd8Nf*!cD7&r<h5dtpf9aP|%HCxDR*3WakyED+t@FaKsYZ6`LkdPWMN52G zK7Sm$`ML%zV@{<c1T^fyPw!iZ?-zCZ(;Cm+8)4{`$J%P3y`zsJmy0-K_SLwIa^BB~ zD+hJY_5Ag3&|lBrrS(Ns7s{LE3aftO>R?kBXT$-h@w}xD*7N5DH7|JuUcQFatfVaA zz*(yED0v9#Ov{@MK9^ODbT6u*QsEob1jpx=SNA?lk6XPJ95^#bO96oD-mBrkdv0-O zjQ)6JHHNrXF$fG3MwzP(K;`22B}&SwWa})F+!0;sn2&{kd?W1}sZ696?BaG0nI{j< zkY`*F4!L@~-78!UjR=OETTI=@gebd!i!cRtzl;kN%i97a_zZdindB7~isQh8P79&t z$&k|{%%I$Q-k0A2)DU5UFzP2a2$de>CgLM#&rx~iV?&GGZ@jT14Nua%<$0)LH}gc_ zC^H}J+Ta$}pe46@We>kNsHHr$YQPXkZ8XSRBH1+y5Hjf|!_(tsWvw5?4`M>9tb{Kd z`zXOPFsUdc{nxTG=9mIfnFnp$1mBuc#u>QWM8U1+$RhkVk^JBA-##6lVjg(@)<(=U zbr#I#Btxr|^-P)2sG{FU$%C9R>1&}Xnw2ZA*NMH^&<icosZls-r@M}Fj-#+|ae=95 z1aKd8GYsyInl?$nG5v~%D__hS5XDB{jS8skqr%f3Z1mdZL(nRy7{?L&sD_b_i#<`i zWbcv!FT{r&xX7XmNk%1uRb56uWCj&K|KN8lqHsRs@I@5~CZxqTk4_i*3EK&HvCRi~ z#XsNx?9Vqk_N8~}n%NMBCJ)p*i96sbFu|*5NPNwAi;(!R0mkg!sxHHMSgN7cXd)36 zqgWIc9c7Cx?qSU{Ghe?F&m?h>>Rsnn{o^vkW|ev~ZU54`$LR`J{@+rW^v9?p^G~VW z?`yv}mQ-MdNzRC*R9Jud%d20WjP!W|f|)6X*P!<-0S!w{v&XlIGlKBOu=^T<P|5D) zOm(bNC<#0gBIVALNUoj(jSX<_v)lSvJs38Im(M3AV-sN`rsR6xh~@-wAOohCIsFHn zI+4Ybc~P$LsXVqq8kuW1*%Yj?WU<j8;aS9#x`&$29LorG7x1Vdz2LZ9;G4hDrZor_ z>Y<PvS9IR!RYK*(Bk=t1WZb(}w=swwTYqh0dfXSC=H0N6;+tBhTJL_j-H2bI<4m8B zO9L_9ZATP%^+;lOJ%Zlt#;m1>R*Rg=G2^eQ(;^y8jQIyf4+<TnNcntJ5_Z_g_I8+$ z_9g<?%tkDMY8c*b+PzF&jp=i0%TE~b)-&ljlKtYdk#WQw19xl>earlg<?~DLD*w+@ z)&t&5hm_?`C7aJg?@VxOO=hMURQgXW4C=4Mj~F59$U}yk+x;4@N0~^96~yncX-%K{ zK|1~k%63GINH<J)kD3kk2Je-)MqvS(bf!#s%>Dyp<Ge18z1`|=xdvJT9>o6hK`EAR z2L!ro+8DUNZUc@84W-)lob+hV1PXp*(6P`6yj`QeBITThZaDq}-QrTIOXmxUR`}Dg zvs)Namlzcbts*UKiSL2<NGV>`mWvM+_#JN^;7IX#kqghf0VkyJ7gS!#2N<`rtc(OK z!u~6qAc!gil&)Kh>YYkjR1CDk<#&3wEj_u&>F2jXR^8CC0AMYPOGGDFYct#IWWiB2 zv5u@n%R#SmR2vb&mY)(kQ)zdVcZYO2`wX^KMr=Bg8$qHD2;9^fpishxx;6xFcT3C9 zkK0I_{=lc;0<U+2Y#`^U*w-v#4?KG>$0!VtRnB8h*Bgk43&ik;Be5=~hbSxn+SLur zyq{ED0<ts4$o>CszR$=%%P&OFPfXO!A78HzTXh-!Li~5UzhMcD6{XG$y;0K2=AJjC zAspM<^hu{i0AjES0%k9+{Z>gKonD+VL0wa>K`##`x#80n1~i0IAWko`uLW-Wf^4vQ zqA47&nXHBAaL*ny*$0huv1=u1dOm_Rn~0Ogqx*JV*XQz0NYIM-HY)LHt1ojv+LeJ0 zY5*RL4^sjmSq+v2TtPeiio`wg5M3D{6_G1MSdnL&xDX8ILXg-pgXf)hwqVR04Pw1H zqzPVEw_hRD#7`APsU}=@O|&Mz%sZIXc<gz9+w3c+Fl{SGN;X<|<R`GGgg3N9Ch2<{ zwRwOJDbnK{|KGR6K?l|5WT-J~<xe<0zCX?Ca1#Ns!G^k`4fwnMS>I}GPi<K&qpg%` za^G>Iz3A*lQi={#N4`oNv443XolCfJjkRJD9z%0A_R+KSH2&O|=Vbg?4grcygEJP- z8TrAD-d716B7eO(7Q#pOr9bMh&}U<%HtZ0vN1P#1CrpI$ubew(R9eXPp7oFsqBUxO zO}-9sHANcl7IV7j@v(~=P$4t?tq*&GH%Ww!6<<<ZDvegIa5d!B$vs8UjGWM7E3Vl| zV8^cz)n=~>cYW<omx8RY%GWLh?DWouikmzamQv)n@k%^Q>SJrr-WV48xQ=<+{pita zoRwK^5o{toiW4wMCl)pOp6m6auDMI-$eLtj#Ha;sITBqI(H4>$7Z;>@Uc&V)?}(pK zi<<{$T#(eVDq7}FkP3<Opik0Brz!U`l|Ie4?&)2$liJ9wiQs&ko%(iE>g52yT(XZ& zqZ>B1sXHdQGw0wDWf4y-xGA!3>p$%f$_x4m5mSkT{8Mfl{!-Nd9M?#SJHZ=8?S`I^ z$QNN(xj_}IT!R}4YZ;i<;XW9zK}|E4k?$4K>!Hp~JLN$RxJDR!!$E%?n*Iz3hU{}; zecKo!7V}24X*T`Lb4Ea-G@&wX&;2ccj;$1gXBByaf&$>lcRAbX?}(w|0BaP;Y(r)7 z9p+~oP>u;cqO~y&_>a%C*|ov-=Jo`JC<RRy@aMU_3zC`#b6AJO5hHm$ywT#IILGfb zq;a2Rb6Lgo6Z;t`_p`LXccwY??1*ah$$psdu6#ORoy$8l@XxO(qcXVcLaY}UYO?_+ zo$JsmSTXsoEt<2Fm<o%1nn4_hum4#OcUQ3akO~66^iCq;pH8bphDG*(eV%&|Y%Hsn z?iUs){2y&#WBQY`8ycO#P8U9}Ix)@@z~i98!glWkNl+f-r#^uja<KJ+1oi|<g-U*U zg|9N=r~ulKDA8i%b(M~-;4Q4_tB(;>$N@u{{+P3rPCR~bub5l;UHxuZ_gngwt6}&Q zulL<L-OnzMY_wL@GR`bQe(LB?UrXDDe9{kbL2LtGEY`G^vKP!w@6VEHxXfnSwCCRf z@mv%}TVB$(s_17u<dg>9V`s`1qOcs7xL)Q3Xa^smzP#N=vVU^(iq)St_&52Mfu+RR z7=WB|rzU>0F9!j?9X}m%da~0<RWhK(?Jv|_yCT2(i(=W74!<ZIB5M|Wuz0?HK0Cw{ z1v|iV)=Js+1%E77MjXa0Cl8$hiB1kfeagLpitKY7!nb+`4z!vKe!YYzUvt?utR68M zF*U8Q#kSSM@I^!60HrUb@>ZW%9!VK6=Y`mGKio|6y}a&$S>ee4YfhheqCyg)KD~wR zXJ6d%Z^`luN5iIu{yOwx7+>si2?1>UrBW&9hee_t845{EWu#p)|8K_Vz?`qlDc)|b z1@cLXJ(Zc~1<`Nj$qii5E!e=vW5pIE9w~SSVtYe@TKT=oi*YXga|``6h|u+Z5<#yX zhr{luy6jq?4ToA75`=KK1q-X(#g39_mIg(EcYUab8*%C>i^An0^o6oP(Q<A7Ufsq0 zE7pzqx_EI%3klKp)|~je7{IC5EFtIgxzut`7yiVZL`&Pn1!~iu2Edx%JE=SsraE3+ zl^1$DYWgb~F)@nB5TK+~@3aot9dn7Qli1V%mwwNn60^LZk0rYVPlfkX9+{=_Xn}F! zK1JZhN%CX)J-_8p->KQTyC&2{<*&<3fRk`dQ7pA^Mm{dcXJXmz*YR>aw<D}vBaKx9 zJNwq7vP>QMKpVS@24<^z>tgeQyJf;nOHiKdp(<NZS0UGltldqVtO%o62g%B}phU5- zrRtf!?$zwH)O(REyahyD$SIP7Pz$i>aWODcMAid#x31E4{Vk-0q)*N;6-Fz=GX-vU z#jdhwZ`}so_aZhIv*0#g04Q+D<f7OX$&B%DJF1X#Q0)r75~0Q<;8U_LqOE+L)@e!u z*#K$5*V%q+N{wGJb8jM31X1f+K(8@l#d#*EI@7fuA%zgWx=eq79)vf)q(?4)FV$<{ z!TBX$29yT>!C)Hk!;H>zi<312=aVXKcUg$9pDU0=3fYMXu~J(qjQvuC?A~sB1H@uA z6NMq$Nf<9N?f6vSH4(h!v!AsL*ldlX%^Kk&%AOlx7KBx;kVGG_wBK_vrnqMQA3glv ze^k!mhxK1E7W(LdR2%nK+UeOOiOzf(7B~9RC>@{+)R!H3(<jyV2*Y2`lN*)!_yOV8 zdHd(4wjKu^4?=6o*~)f5h$*It+p1S%pAl<xe?W!U6gGA=*h?av+6Jh&tx*QzOUdT% zx^lfZ0HF*5&rQ^0H^A^U`YVtMu96376)GFjF<aVa4xC)b)@hhsi;Y`c0x|M{<FdSW zUi(dFe(lPUz{-S_TNS1S6KI**MD}6QJTWfA$n=!ZbvdO$v(a|}?@LXX=meoCllfx+ zJ4q>LM_!VTnSHKaS?Rmfm5?HamxfdhF&@FACg|%b>*r+m|4ckcuJdazjDE0>@l>n0 zIXdn05bd<_D}ToC7lq_DCSVmR#S)W5>eFuPvElFrMt|Dk{Q&!e<j60-Yv#w^XSy|H z6)-QV3sX03!?`y<uCU>T48SxnIx69GX3Ww~m%f-DqLz-h;hXT&(%Akg<eBag=j$c( z!`r4Q2awVQ5fMRLs9_zeUWp+6t1437!YWJvCyMnjxYJ*pK9V&!k;g{sni79x<1TA( z7N>k(-9=POwflG&XgUysYy6&9-Nj?@h8x-S*~5L5Fa5k)4v{?diI5C^X+I35USF}u zrv49-UNaiHd+lcHNOM25yiYnW8P%0-p(FDNTX?1<6^t&OwM#i4H&ZSN7|UsgiELv0 z`o}f3Q9hc$yw?en%a(z^DkOazM<%3{I~qp2>P*`hl+$U~gDGOtyt-sLOOfROde?_h zBX_R{ZgBd_F&W?JObj)apB||CvU{udrL^hBV7Y}gw%OakDq{B!3LKFuIe@s%{Yy+o z)g&s$?0b@vX_n3KNymR>7XC2z(iVfV7UBv4zd%1HZrd`|NqD7coAdXwV$ultqmb^y z&I6|;JG6|&_g=YuMB%7DsB9emiCAAWPTZn}9B@1Bdao@HkXu(0IUov<E&TfuJ_pHA zlcl8e$->~>Tt~p^W>IIjCn*et3_md_vIF(Ex+$uh#t>GBnoNf%fZimYEp+Sv)GHh8 zg9R|fffnA<*`^PQl*LUwpZZVY9<0D^7{Y^f8g=FO2q(DU{hE}nk{!oCxxe0D_Z2gX z=R!>Y)yJ=%Sx$&Qv&C^*_W3jA)+7DbXK2}{yUesmMG#uzybBcnchJD-c-KtmR+l#6 zorchZGn8OFAW;Pec&|%mp_)^_Z(UnC(~{gxk{YuNdsIF-0JC`ETruVIS*bsM8r*H9 zpsp|W77`VEwx0VIGYz(rEWtj}RzHJtjLnM~OMeUv;OYE%U1{qz>0VyVX7!Yi0>qD1 zOndU|CkLss=4gx;@NSEvhS<g?BxD}n_@9M1Jmn0<8>8|6NE?4_ClVB%f9Z}MJtZC4 zjfWRKFPYk;-%N+bDjIQQoU-9_zZ!r@PV}!>Bla<y?n~cahEE&{kyWYVS5<}U-uIwA z^LMgGE%>V!+a@-Wr4&}W^qe%r&ytWt3@Lm4XH5j6htzV?^+Pg$@HP5QiU%F;cy)=k zd!0~PG_PZog_PIAJeLYdiLQTqzhQ<*+Dk-61bT*OQ9eKkB(5uCVY6W6nCV?{|J3&_ z%|MpLq1Ce6)d7klRmVppUB`fB_sM-0Uk87bOT=6b=mLukaX+QuapT>5LTJP5#@1qd zMeecTJeMGR>j?}**T*Q?H0*TLO$n!&J>A_MC>u4M+PdqKH}unLy%jokZB=3E@YcND z#Np6_29tbVqmuc09bU8^Pikqkil&gVBvcB55*UG~J9Ve#t2zn}{vx*!b%lDR|0dI% z@-+s*yPDeR2Dik==KR<J?kDR6ChwLN5h1V}j5ym2AtdR<DN5axe3DArn~>h=5I-5r z!n+|qv8~y9gyoA#U$i~Ys^ql53hmex=q|_m8=@mjne3HEnqLj=r)v%DeG5)32C-XN z+xmlUT9^`yUm$#aIW$Hw{@Q&Z`5XCpr8p)(G}WyDYl@VbcU2L><Bj+ta=UL;Ba2&G zhxK-wx_&xIyT<h5Rl79K3p4W^%97S9))dDkVnic1e$rRV@0o9xZTc#RhPA<J!xw|p zOu9Pifp%h0d+zk=ioW>fBtZjRAiJq&ksr;K(?zKecUbsZeloRR%%|#fq!Q}mH!8#* z=<G5<Mf5;&bq8apxIx)kS+Le}fn6PbV;67yUUKkxw&}y+#=^>4-_xhirqzIN2^ECm z(7>jVH{74zw;KMjL?U5IafHV-r!L$31Gs#kHk%v?o-mv_6i8zhz#^3|&nT63OCL*@ z)-Sm)C(`f<9O3z<V%Z~34&CWKWzov5r1uCO1LRIJDNhl26bAsv)m~o=kUk%=_5c-w z4m~PM9*TR-`FQ@q+GlcWIbV!RoK#OC_(|YA9-ElYfQcwV-lRx4>J)V%@`~<}jQNNt z#ClhM7-lPGJkwkkjL`H&PTQUT7%!IEa3;0}h<x(*AQ@`X`Ez~#46l;-r$7wr!1?RR zjpfiE2crNM9N9}+>v##BtOEznSat9j&Ik`$QB@3XZ<=)p!H<JU<gL*g-tDbX0FjRn zCF<}XySjKrc!d@sEMN$^HAvBj0l%%z<Q@#J5bHDclwvPPCY_9xdu8y$%gP~<F`Ds0 z-ta~;m`g9vq9*fzOgYHAh)R{TcWxDB3=QzJJL^Av8!Df{Oc0lI5obxJj%nE0W!4hC z37|S%6iJ6mS{+5=$<too6_Ee>tJR&V@x_xNl*QFuzSyf|k2C72FF!{8g56QC{YmAr zELD0a*_7STaD?{S^o@3r2&v=T*Hq(12F?{LAe@`%SR}^CpFjSRN0-W6-|AxseaIge zDTxT9<wH^Xw6uN-;Vv?STUJU?$f%_sJ`jgXAA}FN6{n0De6^i8BxyM9Mu^4!gDCOZ zdgLeu?W4lX3ZEPler<?6_CouuR<Si16>vE+8En$`Yc}}k$q~}S0=ld#$1V!VET)1u zjzZ6jYsZ2=zaZ7t4NkvR+&!8gp8W7y?!x(sZ@0V^iVNPQ3pwC-VQ3fDw7AmfYjV6l z^fU^oNrX4NC#fCr2$OQJeNmMab>P+Ac^ZdK#k7y-PV7-ac`E!pai{fs<;|EDo-p43 zDfv!7g0PZn=geq>Vr<1UInQ!SLSk5J<!*UPic6d^ZT6Df>a3q8cP+)bhSYudzNdZ5 z3aj>JN;1tjj1Kwxc0(O#FKbYw_@I~&A?t1^w{$fPR32L1h#{8Z-l-D2Xh)yRm7av} z*RfJ7s*sgx@v3O)OKY4;#&h<0%$E+LaW8qk;hau-6B%g=J=qAwOU4EhzLG@MW#?3l zfw9)shYFzc^iwLu?(G+;Wi7eNW=Ey<Z#xlliY^8wacxjdCS&-HG5W}HFL1{U4q1T` zeqj7}D+r<X9JR<DG*Zd2tBGjc9njnAMo6&ToZGcnhsAjhf`|M_GipjwBBG25fvv); zC;5++=(~IBF;t+f(ppdRj^cD`|Jk8{9VPm&A4ERiKQOwE9jjP{H@F?urz_7|0}{`8 z_N}i3-gl(eI>I)kMO4Z{BIC7DX76{+Z5Kmm$3LOj7ulSZLuOwx)2mc~#o7K4tl1+~ zi$Oqgwqf&nSm7+2ee7X+ufldD8<Q(ZEQtTzEbS8Oba-fuA$!GSX?*zblL#*se03Eh z^lBJb1bG*6c63qUw@9c2c#6ry25*O>2Rs2`mVGtO@V1^oNWB+e`h1I3J>fgJnJ{>P z)wC*H@k9BPk4brf4%3WyuY&j)mH!02d=sjD0J?4-oHaF#Nqxmi+bVCOccg4|!>PM1 z4t^U1jJY>LDZGkVX0lx#szb!OLLgqVBJcM3nug9GjOz?A>7LjOHrnwAso<qu57p`u zpsNzFxoGZMwHTmBhJBv4I|7TpYU<U0*LgQn0Ox)#k8g7)F8{IH|H}8M;b&QBc$VeF zI%-H&R^sDPZ5=S`XGkQf)(F!>{CS$&T`aG0NzyvJz-WX@@ZX7SYLbF{|JC``XGtz` zW}=@vK$^1>dzJ-m?&P&m$VG_SNwKC~-HIMzLNz{PdMNnn9<Kr)N}mC=EyYLQB35`E zXZ~<3h=;|V*>iWAbA-KNr|O<4Pm)7dn)?=CF278Xv~>*0<FR!-izVMalNGp_U7WF+ z0N8sGtV{8*O~Gm<K|gPcg3$BHJiO|ta?7##DRTFRFVGKZxIcDYWtI1FMhD`)bo&ev zoVwG4)@_fk{seBX8%ejS7x>w@|Dalow7M%l21=cpvh7zEmF~34*{^(p1yaXI>^(io z2Mm2^iGjx7=QX_4JczfLYyW%gWB-47xr>08d*}O|b*Ph3657G_3D&juoaP;@@2bB- zL}y*Zml92L_G_Al`~S9obfG+mPx)Sx*=SA&Zj9V?XI$G+k<p&cO;Ia(x0wk0&7mS7 zs5}NNy~J|xoD%)TUj?WXut72ybJFYKN-%r@)0M3qjJ&VIl8uYLTnyH91Mfhw>$ogf zn^r=+?N6gUZ?>MU=(eYcX!CU&0gJ6l$g_<bCz>hdmXV{#Px6(tzoc@qPBnrkChmGW zcXB*DdNGU8G{V)k_*l{vcc1d17DFjk(Siv_Tq|+g*J=EsI!A}N+i!h?2YBH(ib}}M z>4tN;_v8s~E|ckqn4kNJ$7*t}lfh4PrP`67vS(_TY7~!xs;3?H-|TE>UJrIq&J8$Y z<=zL{l{u|$1#KYTRi+#9SNa)U)|FlP%87hUbU|<r-J1C5(1vAJ-!A{Alw4%)`GBrm zcE5>U`k6Am-oTn7x+-x`k@@Y%Jrft?2_#lm{U4V*e=dAIl8;f`Go+AN;v&q^((2w% z(`mEp5IdfQhZq`bF4xhc6wc`JFWe{hpBStz(pyr~tpZki^4Bwy-;LGnT*$Awk7Vz< z;Qf~4{;LI`XsiN$2yv55B<9Hc5X{Mi1to=45}{2R!gXGvh{Cp>2=3rYadJrZGkE+h z!dYesNM&#;ip)A0s4+10xN8h}{kKoR?)^7h*2!7@JvJaa#YY{&KS)u`vH>=5#o!)J zp)<M8Su2ok={Hgy9I#!i66RwHc}?a(6=Q2Uqs_TxF`mM5v=bOIT<Oa~{!@3A7UcUM zG3e%24amH@;sO%NKOYJT-^#Lqq@7{e#jRD&!}vLn%>OA%Gs(_T?DzkTI^;Pc*52B@ zi6>^L{Ia+Ndsf-uZUCO{*-dCz_UU8>Ge?W$xM7t!qOtIulf>HsrjFkt^yC=;t(&!8 z(IDr$3ehvcWa5P=66ezD0&w1|z;o>C-M2O0ex1O@pbQ{8R_f5EUI{rvXlWJ^adK}0 zGwG(aH?SeDW5PSZaNJF%EWZ(=XKWo2R=wKY9r}*J5@PuM)*x~~u91sU9`SpCXlX&% zu<~4s>-thIi}5<RIp*FiIqG%mH`h@S7k!9f(C+XcqVOc_@Hm3y0B5L|bk@1Sr<*0w z7U}{|iQOCb^p&CtFo9&8#M$*_ZSS8c9-eCiuYzN*u=daIsbJIR8<Kd%)Eat=S+8>h znap_mOj0=qE2XHkk+=N#>dOhzeXfuW7x8M}U~Z!rq}phX<ljAKS{oH?-)Nhije!qR z2iH9{a2t7j6xqw-1RvC4&lk}wXxE(vTpZ|BJ=$)Nw1k$=`o<q$Yhsy=xZR<Nq&aj= z5}pm~Mn~Q^r^_T)QE%7stW1EdeqH>AO)s(Qnhk;<Q!gqWj-s-g!i^z?dcGUIaRSfF zep{d9S-zzB`KL_~T@L(PC!>%LS23e1G9$!!?;c8S<jD7p5Xm3|cBK0RI`5h3Bx>*J zabe=KyU#uMc4qgUuqbsFJtTn+8@grC77`_`(}r<^q6{%KTpF+UOt0OIp-J@DV~6~l zFYL#_+@B#uqsV1km+HdpFa<8TrM_R84IHc*?7Qc!?4uQDA_gMbM7jxsg!%Dp$KN{m zUNM|Py~Zt-2emU+G(z@d4<wa(vF|0Tk;|sPu9p>!GE`I*a}f-`aC7lvx!(8q;yH6v zk&hqgPJY}>sl|&<3%hS+{mKT@_(;Nf(NFnW`(o4b7eIa;w9bW*pKhU4Aun3FrA!#N zJpHU&&6;z-1qpXY1g}D6E)-9Z7^3$+pyW{0AR}eGd~Pb(f+QBv<PTJ6=Egvtf#g6# zE<*xH2}K_=E>817(_K<X36&ps{eYCTHL}5FcW)5~r8IZU{P;$bgoif0auLeHSUlIv zd~%h*vQ`J~`~jeE8_qwE7Y}e3YJkjpgYn=B;S%}~Z0mo+8eIiwMRNpOI_#CU6QTv* zWs{D?rCar3k1(gf72jv!w?)ztmt;)oRk&Z#KzNa9*V)8m&q%iO#mw=?-#%a=6JAr^ zi(wxjPz)gb9Tz%MxyuvNZBYrcJa!X?aiyVt{s)WkB}Va{3G{jOXEjQ1h3m)oksr<m z@6+DtghrfR>FMbuL3HVj47IB_uYA4n`2OCd%<K<eKlXjUiyo|3q<`S}QD1%hW`-W+ zg}V<N51#0JkV4}_f}&3YlaQMUo_0%#xMts_-2ii~O`ya@D0y~!_XKVF_cAiRqP#=) zndC9_3yCPzGcQ_ZGB2FDwlYbF(9Jf4^j)$`s9L_BwJg~z0mwMxs}E0plN>lRGD^(^ zn55q?&F`Fcq9S{?!F*DU&ULb)+I$^Qz20h%^*MP>E59L<+ro;gUKf%B^#+t(MZ*t= z%~WiJc#Q88SoU5Cbp2>5uZ%V*?F?NE3M9w*Z`B(JdBbrk@F``1P?yKf;w-tNLKd3( ztnPyltu5&!?{ue4c?y;Yr}tR7;%?*VY%Oj!5bu{N4~;u>xW4a|-RtyxO4YmuSpe^H ze6-Txih}%>QoZMLPePH(F|A#VtJi$_u5t=@REoq$p^ahjLgcRRCkgJ-6r|)P!Df?j zt8Ll5^)=^r6(zdTyr-Fed=M5p{6_)j2rOH-OF=2L5^I`h_&J`MJiN$gqL@;~u3u~* zOO!SL<Io%xJ)RFFQkF#l-C1;<ix{<^T&klqQ)?XVP2`V)dae!AK?F+A*@gl2ps&bx zN@ffHcF7$=tlFPA(xu%#Re1N2Qoead_uQ{vub6Hh%{`CF_~N$rSd>BzHO_c*i!8Kq zri2AXF*ko(<>sALl3U*pBV;*!!;Cz+R3BQPezIHl$}kkEOFbz)r-5ZHex^%s1i5vA zp=&#-okza#ge#TKr_K0+ptnA^TkEv(9^+&ogVy^BpQmD+1MKLFI)}+OPvUC!yo-8U zeTRu=Yv1>&El#)lRmgq>&fZFa+efK(e_&hlXDqK=5ytGB;fAYmf(5LWvT`q8nXrUW zezxmQQP_;d7imrQwbG6r#80?a;Rt!M-udV7@U)thf;It*nySRklfyRnXj$(`>V?A@ zSTwi$`r2HSCiNe)ieew7-i|fnmIkwu&9%9Lb-&vLmBfbZ?$8wz8^sE<qick*>(k%1 zc~X;HtYd4*U5=K6+h-?z)j3fdCHR%*7f1GW0lS|cyuW6vzcG@}el}n4q@8~9{j*YB z`1xsCSN%&G9@t*%J$^HzAC}N3IoiCkSGlrV*VUPZ9O=DQ`ZDgZ=*9bA^0a!h`dmNX zxc}W%jnX<;D(Rh79SE0Jl2p_t$Pu``H(xqX;f5i7W6%kf<cDXPxas8%!^o~yR8Lya zo>B8;4Jvau*C6|8OSsXk;jHe1?~lukn$)5>5>P*!nn0=ZQqs1PTDgVI)JLi%O6;dM ze6VNdApTf%nb&W=>^=}bpE6oUZ3R;qfiBAoLAIT4W(l013$QuY6fRnpXHpRz``Wmh zEp)x0a*k4NIoWdsv@)oBb^d0`D34O5|ItJudL`fT{c^Ib=N}=*!7Q36kN|Fb@DuTc z<lhhIO*-(b%ka~c@W^ZE-{ikzSf}?0$x)g~k~ByDurs5*Q%a0%0Aj<<qeGnr!~Lc| zv=P^HiJet5xe@#RV7xp=z@Uh1|Bih1`eN^9<(|&%Hq)d_uwWt9JN@OtDv{M3=d>u; zs-133xKRaDD$1509a7U4{d5z1_Ra8{Se-PYg*QnE``h^Cix?L#J*?NR>~QvCk$|CN zTGX%<qUw?F7|*n%CpLaGRuJ_K7Rw#5!Yc=~&1D+F)3yFD_KzGJ%4X6hi6=)BFCrJ% zyeEZ-*##D*l3p8L0XCMd*L-%uFpBu2h#S{B#(YjZi{sWe4WyKWN9TXidOuqzcgEto znF`uIEnxVk;FOPp)F6HJ6b54)emv_;K<0k6r=AW6kKKCUOIIqi&QQ}b1|I0Y@7Iru z9RY=zNa0$(#&$YQvRF62W4n<obD^=B64@sYm3%#`gd6$fx$VC=>)#VoFv%H@?Uz<r z*gOfiB699`&3<p+^Nvc|yNY8}X&&irRrP(%KGsE*sj$Me!*+WisT|Lksvr5i37(?C zjFvuRiGkt+AMX&x^nEkT?n6cx+a8bR!3~>=(%8542+v^*l?Ae<z1<u=M#yJfVVpOF zwuSexYbA0R@70NgBf=yT>1<w*_jFvB`9p??sLRI<1z&?s8H+k89H>@L3q+-Va@b96 zYr@OdDuYh6>U_}zONURtb`!LEYBwl(z6U%T`QbKVGnIU3^WtK}b<0hc##<>3;eXgi z$8hWW8L4I7y^&J&ZhyE($9RuYG^Xh2uD9{Do5r^o%j}yNd5%WbR7=j6?8#!pWf~j! zqHCH9ubu91naM`~?zM6xt{@8YQe-w-CHFeEX^^Uo?FAn9cV*)%bK5KilfP|$&Rzba zc$rire_MVGXBBa1HtIvH5{;;_Tfp=!2?Z%T(jvazP*2!Rr+8cWlV<QKDMiNt5V#vk zlYgS?E2h1(i7w*@wkG~U!KAJtr>hJo(-J$)+u>8s7JnN}<e7hpd&#1`0H*poww-aA zrIFPgd555<rw^!iTwU7NhI1KK`ag^_$08zVIrAp?d#EDd+y!KoPX~b|O*KW%yI9Pv z^$nwh?e7m)sC=3Hl1a-_d><{@f-tsTM1xU@-3}nB7lV{M=I(sg&3@Fh4Eaa29rdp0 zf|`Ly1LW2P`&d49V(b$T{HY4Q#bU(v7pBH*^<CqL5jL^dJ<We9!3$6hi@TYy*b*_J zB*+5b>Y_lxf?V#_iV#U)AQ1Ji&D#I`2MQy<@?>^D>p;!!Jcp}xec(Z|Yw?V=Y2?8a zuyOzCGN}*r7b|pXs%-#P)3XlUi^r?L@RbFY9|76>Pge(Mu0y_)1^YLjzupJry;_n^ zLq7((B<yS~)b?H*f+67VMfUe|e=&>Xf`H&dZ|RhB&4jmi>|SZwfAzTX80t$lB*Rwn zyavI7xEMup4Khx~p)0u9So6%Jew?~LyeBwbbo@DDZOrWOe$vxXVoBRk`*O+qeVuZ$ z1_}w+Ng4Q!uJ7FRO64z?Ep?5v5Ztf2Gu<*?3k&paB>;5gBc97p@GX<z=S)O-Q8?6= z{oSth`3qUL@nyTS&EVj;<uo4QJkttBk?ZT_pH%&6stB9)Wln?KxgotCFVBn<zt(JT z%DiA`sN>NUqrLM)qs9k&u3>xZB5k0voWj$#^#BN~w#1CXAX<6~<~4tg?DUPOyLE(C zkiY3Oj^bTzf6fWJNPS74*X&vL<dIX7N5#$*#0j;Ab37Ay?nFt^by16!vD<&oh8s7@ zoExpoOfh;h)+>1=9oY(gaSSi6_A*i|(P3Y8SCl8|E)s2D4SK6eNr&Csm}%CQVtobW z{j#F6u(#}^u3`hd_z7h@L=hQ@k;E$tN%TzdbcK?LFmc{ro>RVA`&P<j-LpNqUgl)I z%?%;XOuUHbWmZ2|WT$uH($3ZFQ_5>aJ4f?||6&u_d0A0mQibkSixI7^2#v|5pC+_{ zxrh00yR(imwb&;sv+X#2ptITrjctxIqHS+mZh55jGd?<A=xSh-Hf!U0M!QHT>SbcL zdZjr#Egm#);qd7}RE7~-N5bXSx)PZsp`RrM3cUrrbx!1(45e9z@A%#A|J>aSGKf|b zg5>qMZ6zR%7k+XMlN+4s&d05XUa0S7lfyb6jVRh5I@OCdw?9jMZl^Tk*iqKuzr^IB zt7yZ=PbHbHbx@PPE?&HWG{qP!H!c!QF5-(EX#?S0=YkrRZqG>JWW0I}=G@sv4f!h; zJ#o623)4rl;WqI?9)nB%JzknckPeT_0`|X&`AE>tawUn*l<E-=ZmLH5Nv^)&w?jbq z_JREWjF$+vx?O!aVAx8jhVZ<3SW*3K+%|qO<=Aad24d-4uEJ!c>klwHM{uD(Bt}x_ z?5>YWo11|DgM{?FgJ|wCu&KW82IWX1=5MGQJIZTBFETWqpTISo20KlZ=ligrbk2Y( zQoVMLb+P%xHjAofi&w&mXoJ;i<gj>FFG<G64wO2xu=zvCE7AQN-6E^I%Z>!VBI-&X z=g?STp_hbovm(od^Y~Qyss-w@Hb|>?sbzdNC#%}Fco;|^?q(pYJ`AojrtvluMzLgT z%|!DQ8_6ymedbnOxV!_Ge@JExiJU63ZYLE})3<^&Ig%Mw{u5<l0J-M7IVp?NQViJX zvrA@Tz7!7hLDwJb;ku<U3R{fFZopseJ(6?xEt7G&ETTSH{te^*?+9SID8jk+GK!^| zVfIUGrWSM%8L#QsVzinxEHl?fIg^>^sCsLq-U|5`%1!1~6nKoZFowOVRw8S(f&EEv zp8hmz%?(i>X=1$J{XAxWideY7`FV9lVT$7wjNOG-r?e?F=CK8VhB$GzC2woAEHH{c z>{Ypu{YxKH!kBICpTINY#84e_zm|{V&ydZbOSZ913AydbO_w_8QhCw!Zs|KoMv$+r zo2G$}I4gzkV-v9__1V|@1mhR&Qo{vhx_lcR1u?iQBQ~nNm)ZBV3$0re|0MoA+AYNt zE>K>+*861~S9^CoPVmy<a_>T&Lhw8SUHWY}|1P<KnU!H8-1(%{OYU^zn7RXyjt`18 zlMX#rEu!V6)~_O_O_$B0IUOdzUA9MPY_A^&_2G$+lv$_S&>jgCyY=&5EJ$PX<0|*> zimn$`cokXwdtNY+1<uY(e9AH;Z{E-whO(Lm7S~kT6%8g}t7V_KhQStcM4sc8bZ8@& z<@QRQJNoVtHTh}ow3R{PC(@9Sl3x7lJ!G<h;!Wq-E9vsl%IFd@{-L5H=jf%|FE~zy z^E%7trMermf_f5XBRQ>S4pWowYTI~w_M-rchN6p1qtf!U5;`<!LNa4*j<zsDh!}Qp zJ#2CMgQ>b`dEd#(3~);H<N8O{2Ip0|MD66;{07Cis1z&L<2@bjzd|bc(wgB8ZcA@g z4^GLujF^K%A{tt7(qfSj_7)Rf?dGv;7DZt+lNM07V!=^9tAin{6b~vpSLUkP#pztS z#py3F8AjC?r`6{4H0-#`8fi(%d_34{I#EAte>HmPPjc?o&%N9{_%ZvMk(PrU)p1hW z0!(#nsc(3HvFgLJGJm-Hj&7QysSEAmQC20wI;O{<L_)mPMK>f1covC$tTjPc?KBhT zowySTiNK9#n0{ynI4PvE5(@a4AhLGObB`v%564VX$^K;S4i(#WUJ>Vmw1a8DQFwPe z`4|ZF`at}iLl+Y@%Y`4AnoyQE&d@Ll_(*f|XBw5L4s!l@fuPv@iwR!*dO7KmWZL{v z&n-*R9&qd3*_(d+<=!}T_D|C=hCOIlE}5}GzVSy0UkGsR{5lN*+Qc)<jx4nOSq)@r ztL5Iun@>vTBg(638c?zzwNDddR~lukm%Gp2cEZ%x=_*6a3xRM?;RuulBAf5Ao89y9 zLUliqQ{j%47n8^#L+yf`rOYy#akyRsGc!p+2#^@&N$aY@P>bLPT70Ly=^V2={q@27 zq21Lh0*^`flG<#4KacL3>X<RRNJeyea~_hsDA0BHsn*8jzc>Bg?LTxN8?`Dg`QcT9 zf<~J1%01`jLqYZPYIpy$JEZ$v6oGd9Xk(iEAm~eW;PoXg^b$Lrj;pH5SFy$`=n}`y z(cp1A@VG}dktM5gvJ<2P*VBKk<B9kh$|IAo55;YtVC?zc!IzavUJ{(!+-izRUpY)v z$l3o&93Ky!sw8AOQuk`6sWI<&DBs!ofVkK9y1n$5Y0LNE@^LMfnRx-hp~xy0Ymdfu zC4}qFHiN@XdLZQBYe`QwtlE5mjJp#O80S;0+(`8AB!116+kC%#4a{MgAbCLE^E{zV zvHypHn-!OfmCDGW*}<$=Gw!le6YdiBQG*^*``$?4TsoSOU!Sg6IfSt9y3>+K6VXuZ zP_PygIeQ~{Y*rUz6M;1h<hAn09@+><DT<HTnKSPWZwwb4QduZ|S#7`vA9E$K3MC7@ zGKxuAc*CrAhNpyD8K^K<Jo#Z**G8Vhw-@Tq#>1^McVu#o*%|-RL&S!b?sN0S`Xmk2 zMr|Q>>sGxpNZ>qm7XKI~;*#Zp03EARF!;;LadxoN^Xx(!Wa$Jcl)?*~Fa>+-Jhvyh z$G>CdF+$!|_-u4T4DOkVoY0*sqx(f>U7Tx&jqTr0E4$3@V}r2EKTQj8pX=8egb-T! z>vE^bNJi)7LoVj@xLfo#m+w3n)Z!?9xsJy;w(iODUHo~@#B@1^is+dpyKtE7jWae# zk&d@7?-O}n8}8DU(QH(Jk5|~6dvQ7>>mGHLgVXeAAb&e{edMYgl}Y7^C$7fn*yp64 z2ZMB8|BdL~Hwt4Y8S-0YcLU@8pX2dEI7RAktJ9;63&9)oNYz%M!Jz9@r2E}toO7~T zdqI=eWg;GR$o=lc8L^IeH`e6BZ5*8s<?XSZv9%8EOXj(6k2vH5Q+*eRc#KhR&Nq1G zZ(r|;O5Rl8BH@dfOHD@uKl)pKEjMNe96byX_JOd%cmAlJX8DkO!TlT?I@%f3TJ5+T zQ(852#0J{x6W&MAu-k1^fMLN4NcOMdH$mJ=ylF&ac?S#C3L}1rur+^v{87p81pCwk zn=el~)wqLfx|aA0$i5n7KjH+Y^*V>MMn%*Hn6;C%nUhqUe}k{xI~%F1O7ED72NK+? z!NyNDe66t@)t!<1ji-e|?DyJl1}%lBLGP69dBp8|yn1$`h%?Ig#c&#Sda;e31o7+Q z{0G24fA=h!DYc5(B2bhUQDkiHH%mq$;C2(@4ZW1~8;tsKd0}C+ayNb*zx@N|dwLZ| z2(g2M7SA&Nx`cAZhBM~`ywWQoJ~<>T=nn}5@Dd*IxSoL@p<61U+T|IX96!=KU85R! zb*npsxQlzo^#8N%{)+@Y2`$x~w`N*c5&m;ez=#H;Y4o`N4B-OD#f<<mucXMJc|RtC zt#~yZzKFVhl}GJ8XZ>C$-3K`2>V^kanQ9uQlDfB9z3O5AxM~<PxbtBC&y}j9E#}o2 z_1(O51xAt*D14E2m^!YU4IPPTTXv5UMy<{l0s@b{PIMU-5VQp^-NQ9}3&L`O*Xr{* zQ4(<t7wSUg#eK3wUwxFP@BbvgJeb{V0@(t8UQTqSI?Xz8jDr_iG$_fsUpQs2z}usr zt-IvdzdzN^RR8GEua)F)Z@y2)SF-V>jG>-5EYGk~pSumU$@|t>SSiMEELHZl*Jbvx zDgOq>bNh*x&f5Tg-4n1I4gf>-(gH;}@p^&oC@qgq5{qR?aJJSctmAs}bOc+npJS?i zHSETV6c7R?r3lgwUWBRL9WOqX8rmV0tTa;zd8LXQg^c>^t{)Cbw`!Z5ZkdU%o%Bdq zSN9a!utT^f2vH7db-1x;lk5r9$ORQ2vwfq{_Pll@%GQQ|50Q7-C>p~&t~;+4{l<JN zzA+utRrC^X3ysB=$x3Vb_ALFO6M;EXJq=sW3NFL=;?kay{^OBPeR<yZ$47)grn#`C z%}{H|I=!VkWPol)n&LAJ%_+M1N^`vGKCZaP6q~e;WO?72<vQ$Z=B#EW6c{OGgP9!@ zD)y|3=3PK<m4#TpjFBx0TR(ahWH3cFJ(?FRy!ABAJ3{I<M>VRx)mX<mp2}UI&4!Ar z6M>N~Ns)^v*lv`S2xG4q+Olub`cndDK8&AuSE7R+<7s)Ayezp(v}cf(?aSXB?`I}b zeQVgDkYZaY@J*m#p6S`C?XJW9KCDs_Ufn!W+IsJaVXVGbcG__=+j!0-<opGBJ-XO0 zRi8YW8=eI9Gap~XR!;NZZs$4}A5_zrrBd?tuAb<6snxbgx9@*oL{ZPcrc{0~{W&bg zMdJO56u-6aQ*a8&T~Jz{7q9-b%ob*fQ!F;mvez3_RNt@a&47RAl=XP*QA;7f1Yr$} zx1jt@SKLc=JN_4XsDSJWmC6PDCP<N6NXZ=eO4X=ViZg~>kaimC_|7JU<7vq~zoLOy z0Ehy(lH@x59x$DAv9<htcZ?-;qws1b<OW&o>!{m!ienrw>S>{L9cn?=#q}A!Vzv;V z(|>8_Q{g)GF1{J4@DO&~KMH-YX!N?A&&GY#2-<GvFk}<3o!^?O@$E%q0eJG4U88np z$SHBC&_db8^UMw)6(-y`edf(`Xa*{uch%$zZ>+v>Nci&QNCU;<0_kHjXy8<-qi&&O zUioN&nLG)H+s&+Q%}OGm8$Z9-QX02)Fdxl=c62mHhOt|{{+AE<7Yfk6NO`BD^oHIv z;V@P@>sv(q!E*$A8fE|7-}B(d7C#-OiVqkFet%Q>2XXDGBogtsrCkZ~p|9zQ^3>Q+ zFJ6cn7vqJqhfXWXIbKur-+v<EYN~{qbQ>de8uSa{zoE54>qjjmF&01;L~If;F19&; za=2co2bQP>%SW}_y6{-NS!)v-?|`bVij5H78SB#crNU2d&#;2Ca?S;9J>8~T-(eKA zS89cOZuXtTGh+I?r`VRw-h}CSq<{aGV?2JC29jcYG5?RMvyO_g?cVm#3KELskOn9c zN{5tyScC%7j7oP7A)TV6qS6c@AT2R;cX!u-Gy@DBGsJiCJn#Fh_xGP`=^9+zbKTe8 z`#jEbUk1M7`^0sg_e>5v&rEnRqr>_H)PFsUUJWSY_{+9}qYRHhUqH1h?yTY+exkb# z_<v?}Js14DrJHL_M~6L5w({QWsU4|RYn!Aa*;@}SN^}|W8OCx1GoN38cJt1soLd~Z zliB>9x*2aMWO9pJ_QPvWK70PYxEsCP!b#)R%3iOQ@hSzV6CwvL*+9$WAh}X?Q_Ojy zl}Bhc#Jg%VdUL<9iB9Ti-7eEYUGWD*7={~GJU{#1{g`m_<`MI0S%g;+F*EcjWTNG$ zI6q7~qWI<)?}&e%*IgA~?mt>Ce+Vf!Xtqr{ps(peL?I5(eIrr=x|VkveImq!%kddF z=!xwfor?0P-92T95B@r^(pG^Q>W$x&an0}G;{P66iKZBj%x!=h3eXwJA`(u>e#ok? zE|CV<;Z{~N2@v|DVahnw+V~~!zZkvX;9_=tNh^rsy!ZtZu3|!Wq<o%?`W>%cTizb? z*m#5XkvPg_N1s>b(@vEiVm)q-sgez+473N%BS`bb-C}9pRgMQug;o({PTcGE#pVuk zj@sU^izSl+=Ro@a8;j{e-yo%(;e+i!P=Aosr<>R~W#vHYFxUknrO}Cbaf*YF(b!t4 z%*h_SBf0vi^#-#1KAs{})vU{~Ddm_u0&~2C(F`7wa(494tU;Ae^DVhPST}BnG4J!? zm5OkFBKhF+uenps|FQrT&YRNKGgV%i>KOg}S_~O0PAaGK>;#+Vfokx^nPv3)q6HF2 zt8vGWzdy}qGLV`1o>MVtrS~W84;b{j9TQ)Uxkq3<0==tRncR9k&ooPRR?%r|>uMx- zF1V2pegnTRbn-3iew!h_Ve{fv^jalmufK`WOZQfJ%nycPKm=W?u8ln^#<w&Qj{#!$ z{s+j<QH>Y2eiQ$R1#+W8d{x}-bf@id84zFYyjYopQ9D)TLbyZ%IxrjZ3}}riO8i$b zfHzR?!Iu%1Eo|>cPbR6Y$Qw;QBv1lNB-+>}Jg-mvi6vtz&wmVpbqHm&dD`_Jn{?AJ zL`Ej2Dn_t8=U8aGHy#}PY8ZS!T`cy3#S$f4OyVIuUq>d?2l?x6L$C5@#tWK2k1n9k z2qKjjCgcv7r~c2&g6&dUQ1tG<|87!?2~{TV{qNsGqw#^~MKiMRSLC&u6+X|*RESVU zF=Av!%j0}jhj_^W6jEnX^`oHM$^dV`mk*8@=nS4~!tCA_nR{%E7rKpIbF<)Qb~F<v zeBh)=kmG&w`5g0uWcBOyWq+0(sjbVcW3)Uv{huCm-C?j*C>-f0WXy?6o<UlU4OYU@ zX28*Ch5t4Llo;QTCqP(z_U;{I0*oVn%!yE|U9V)V9I_ogP4=+k-}LDO=S0cTkKl!U z`Gcaq$)6t&)(61Bes#Hk%!CFBO9|PGQzX$IBfY&yFW?@>BD2+>BDd@Q<K^|u?0!Kn z$3|NQ(9JPegDeLzrR)WRYS%|dn;G$fsq~l890zjPTE?>GN*c>2VKWGi-RJn0fIHr) zlQi2_(o_R4yc@^rB;lFHGPP%MFCByK_Vtw^s#Pv!rW;dM6wiT7lrv;&Ak|-x9d9}P zGG6Yqhq$s0URy7<-pR<PwMIdX_TVfuPlw8Lqg<>o&>}C3>FD{?L5NyKGv}D4hL2J{ zD`k+2hJXR<Kawvg3=jH8tJmokHy6**%aht=eiKAb`L31Yx}(e;iu$Z#n8$L_>rwda zk}On-Qbz`=a+rp*{)1itOKlud9PgH)Ba(N=FHVTzf+9td<VmxuC>3j&?S5a!yH330 zatMc&h%9p{PVqHs{P>bk)RN&Y>NTBpay@45B6WQH&tC+>VO^8EjGa1H={1^E!U*cL z*_|LJuwwXHCA1;K);>&6MBZ~uShSSv;%nOj{+&sScLH8&H|QrY&E?iBs8(Ai?Y~R* z9kmGJ$hgVpA(G;#XlZIGv*M?(9N9N4JbLFl=4tX}LgJLD&&o?0!fI-%l5;;XRe>@G z>Tm)2rv759I<(`b)6gJfXU~&0(mNg7v@34c8IK#=es^^|&d%o?v;W<n@zQhbL^#ir zq8j>P#$Hxyf|xZ})1a=DExCWt<&>*=jge2`qLv|E&YX9j>&3B_LGiKem>8Mowl7Hg zt$om0NOrNQGe#$Mc%bW+)FQyzX0o%fG#PY%YZ5C5coZH(m&uRYlCJfHtT+o!RKfd# zmmB*PN9{*uh^E9>ed76OSgzP(KFTdO-9an+4I-Y>YwhbVWlsVsW4^W>qAd>L?Li?! zxSh0GOFUK5n1YW7-Y~!N65lX-khsXA6!WklU1u4oU+MH*_}%4HbWcamC+ym#$C&>o z6;b88;um|%J#vC{zsQZm^I>&fk)?W2JT`ZrMygJgTJP~2kCy^LlP^xt{M(^Jw?L8I zl$BM{rb^8g0G@_S7@l$%7%h4N_V2eXs<yUgh`}^L_L7P!GH+9lteM<*xVoSy?4Tz5 z*0NdMN{>{V7bNsFJHv1P$f7j*)*A6xYNA^qUVDNyvIi<2i>)`|m+8(R*JIz8ZH+3Z z#lDyt238<N&;aNJ+kAljk2WDslza1`v21w!v%JNrJAB_fpUeg<6tP=I-Q=L)`e&wL zWkk9PF^a}jUHdTM=OO;tD7l?RYE<@D#;XJi6cf&~=x6p>mCs;mzKs=^>u0N~jBDlY zW9BkZFxMcELr+!-;y;m?{T-I#r(uj)v)q;<j?-!TfBSRtYP{<z$yc|Bi2DuTB1RZw zVEe1&TdfzH+0y1+=|g6{@dbA=!`rw%Fkm9JYQ({+aV;01u@srCv7w%<^^y=z{-B-n zsxOsYNX;d#4)9#@ms`(tWh1j)IfAtgk7`P>O7Rsmb<yu^;0nY@YrKtIwE8i$ZUS<P zMbIn~snSmOx9k=ZNM?BPV;JjDS88dHBt4~-RvLy%bz+E&2Tv+)Nt?MFhdPr}Tj5Dr zg7KuhsPldSDaFeLi8L!>QUnNqv3W+h{0)PO@V1|Geb@r;d&OhPmpmCL8BYpHDr-Ji zZ9<ULAyW^mB*tRv^^cFvo6pc(BLh!<T_y`t2nos7OS&Ka0^P1KLj*-Kz#uOtq+TJ^ zL<_Nd-gN^g*3ittG*)(_Y3KXxqEaA}IESm~V!V|{ZNn~)5Bj^6Jsww38Lm``4v0j9 ziOSEF7m&8&ZW^hd7>Uw;<YN|n?x>^d<&Q74#9rO6ct(=>Ifbq*<NKRi?o1~CG@<YA zzayYU&(eI)II5L`6p<08#LH+>DcJbOPkf2$P#V8Z>f<~^kB;iOW(k4BDg87Y{qP{5 zQ=Gn!_{l4QKsAJ#=6JKU+%Sya-Jbm(OMK`*1r+oq#BBO}TKTH$;6kts5mhE-e<R-H z3w)36js#v*nLaWKp{a-N^x7aLq@nF5jJJYVG0fiBu0`*C+j8sCkfSKiZM}|IF@ar~ zg4ZfNiXgyXpM-MMwXEdtwYk03JZl@d@Mz=1T_vgCKbTS7c{}=my!BNT3Y*KIysSpC zZCe~9dXCo7B&nEOT|<}A*Ilb#=%>?+mpiFlOZ4qwd@EEm@G23<?M)}1b@T+~Px53u zH>9e+ONjVn+WiHiM<{Y{VH)$&l-TGbJ^3S-{we#T)3GMNHu3t3rsR?Tx-x!Uo8DvI z@DaY18D6!R)2eFbEBtJHA1^$=5013L%;mI3pA`-#fN8nMQtdicjzh~TSro+N*2wB= zR#_KZ0G9c;VOw}<baB4i&xuPLlCKOND+c89<sPas#JluoWYx2RTdm{C(Aj0yyl_I6 z@nZYe;cShZ#f6i-6;9}9tI-HE5;O^P5YjUF9n>!2z<n3xVh(~6Rxk54tp8q0m0*^7 z*RqGh-6!jjRh;;I0QHS+SAcyLK9X381IuN9M7988mBlWfEHB0Ftt__$kY2)~8xC^{ zN>N&|BM2;ra&c>pp@GJ(pocs9Z2nFvR!jZQnDe(!9)3pD{DR3FT_TY@^sth}b(z_c zWsz`>%i)GTsU6@SB2_YNf~~&sw;Pk)>O7nD3vN8;W*ce^@z^i+%>zN=YX5Kv99GB! z;eBjL)>WSnfQd!-1GGEJWfN?ZPzL--ragTlMKz`ONs8{;K_;e{!C_ViCuH-S&cI6u zDQU--gRSIl`?EChS<QTU=FEH~;>`==5Lz_LU_z_(+3%Jg7Y4UX*Odvb*uQ%~aeeWo zOg{J>Xv<QspDvtPfl6SnQWO9n-~72oZWWN9l3(Ok^jJ);XZC$_skyA)W~(N-VVYx5 zubc+osD;+Bg=>;0iG+0mu@q0f;q+@KDU7ZmmtR5ny@q1=jlbAW_b)F*NOL@XsgiU~ z+mrNYL_6($0e33G9Qw`hC&h3J?x@0XWf_TaT=kuPhU?WgpRd*y+e~sCD|Xty9_Aq9 zbz19-`c~v6Cwfu($-E59<Jfweu?#P=tyfD%Me;zfM#AfR@z38rVpRZe1)ENv#vT}N z>dkA?&^zZ9Fmkqm_d)c1-Et*;G2(g(O*5@=-YM0rYIg~XWjE-m&S;O5j{;$&fWGvB zU(7*FCM=ivp91EK94KI7_L7oCTh-}3v)##eX71$K&Al&q&?G9%_QisXgl*aC6`ix4 z$O_Knj@(7UD~~_!r}0tWl#mj(rE<IaVNutMA9_40W%%dgQt=gXd_-6AF8IEk&ea(| zYb9yCo;GKcs7c9Gh!<>hAPPy}O2Uw;pD;0|4)q{mQN+U7z@P|yJgW-tPy)92u$fAY z1p>Li7?%;+e_jLe3yvIh4J*7-^o_(@#m@GPZMCsoa(FSj<NCFxfrF3lcx=SGjc}R- z3z77Q9zE8-W{><tfZv6pYdM`VN?G3T2z(~j>X}k!$wZ2`YjP5HRrydK$8I&+;>mhE zu3@|16(+`xI9fCv(WkYEzmqS2Km4qr<tLu@?9B(YOu^9#*JXI8`AmH@>@7rv)!!BS zUQ@^4YPq_^qf?U1Z`?^JU^G(+zjQNF^Z%*^o|gdbOl_iA%6BBm^2b;BrR~UsH6Vk8 zY_Z6OC;LRfH)F3}0(HaJO7*5h5AYByz?+c3vAh1^_BG+>^Mu}3hxHtsr?%j-CY_8h z>n2nVt|L&r{MaaR<pjT1O^~FoLiCkq06O?=J1_5zgy=*I=o{W11dFbI)SqNrtM*x< z1i;|>Y>kAF_2l=?pFEb<Th(rRzX84HHdbY1rd4c0&K^Zt<od3rV?zE_&~T1s!MnUo zFnpCAl`0ZtgA)Jk2JafuQ&SMSpRQ|Dp^$^R47IQC(<Xx6v!BV;u7GAQejM-q^lELX zE4t63_xK2O2prlxwdYt%?EELzRghZryu=(;3vy}PJHFi&gX1mUp7LlPU$geerT@=& z<gyCMdc$TS%d0VzD6Zv3%}Am8O@JtdKTqM{KjLY!5m)_$7B}RcGHu&82y4}3Uk3}9 zyRRj&G4qvdub`tBqQ9sZeBetU3F%i_xOn#5W%hQ3u;GK*yi5DZr0}Ac`?)uP*=X|s z_e!g~a(%X|NWxpEcPBVhLAr!(UD&LvNqHGL<Y_UMHmobQQpw97b<d`H=c(T)J1GTn z8q6Y}iCnZNPi5`Qy3MemHx9;Nh`kEEy1`L-m0rCU3tQE}Qo5piE8^9xZ4yq^Wg>X) zseTsJtrYKcNRT+Qfea-m<D(Dq1{yH7K_QdaIZJAvYjQgh)Lv6QUdP_296krfU@iF_ znLnEt1&$gldAy7@BL+c$Ti<xp(Eo%0x~-0UKcFeCT4h4A6g!Gssc>pTUij@E^AOfa zj~m^Zb^esNHXl8k{B)$W$8wC2tx3_oy|F1R=~th$eV22!@rm_scxJwnM-^=cdoxhW ziky`gu<$_G(s)B)g!SLZ$M@jOWOb!Gcqn8BwZ@@jXoCU?jZ|r&K2mlyqgNkDuhBBS zf#a6G?tv8j_rt^Pd8Gv-|9v=XYW2&wx{pfcVQ5}R5v4qUbsq$$e&i)#qrkgm|Nh?l zI`mVg)w%edYw)3EC8KAp@ar#lJq!86MZxtjUUHMF+b7QYEK-g_i2@PHZaCELIj@M6 z<kf2wv%AwQi6AiQII<pdmnwrjqRih62#VE1oGhN>u{2WZL*yqe^xAB)(fDBXV-)$Z z>Bmt8RAYn{eD1O7`YAD@!lGV`ThYse=Fo1Ga?&mjaWI|W+;#<zHO&*YJxf=)(rWR@ zMjOpdKlCe&Ql>{6<B)r(?O17Vw=qAg(dJkFp4wmZ=w&|UCb7rH#|sV5k*5Mfd*SzI zV@CZk$8x3{he!hP?{@%rTh3I?hWk>E;)$&(cfxO{l9|23V;jno2lBGf4{jxQvU;C> z1?$HgvsF|4s2{Fb?g2W=eCKb=RS0Q2r7(o+(&lIe8r(E++Gvg3w>NL+v0zDp$)SHH z{(|tclpf(J?g@_~u$cR<_MJHd*ev6HLrFNu0i@jaN3zT1D$y9WxZ=4R=K8qrNE=O# zgD`}=DN=j(dTq#xc0IT73-+eGJkM=D2xk1xA=rXrG4+WUqBOz}yd8ubqclF<K9+@U ze?GK)`HYZJ%&(gU#q8DO2c*m9^*(sRNJ~w=SiBbm-&>Y%<qE9&`jH1*#;|-_@&Me` zk;PQzjZravLbR|GkJg`h9qC-q<xFM7K{koW8NVB;lm+_De*!fwhY`QmP758<`LB#0 z7>|lg<le<S;1{^9>%#qix{xnAzA;1<ziId?&N`rfZu#grtlZb3{pU?#b@R`JPp)Mf zP0|SRI-^u?=P;Mml>xp<2cmu>a=n9M--rPz{$>29!#@^O)|Fjm`R}c$fFb59ib3@* zp#9`{+VVDy2^b)hBd%?{UoeRsn>%Bi&HDSUQxNtw1vLRH$gt$C@u96fa4^A{)6Jb3 zV`givl8|RGD_anp&DY%Hr!+Ngb9ht^8nfZjmAh`*iY<EA%xV-397WZye|nfL#kXH& zb1bmm-xE#WQ(uK^YgJAatJ^XQAk;uK<Hmu@<qHcK8+FfbzDZ~NQSe8=^WZh_X>QJy zD>Rr9xM727>4cVCuGsq~PZgzPU0w+h$AlEV@hk_SkE?-M!|%#^9OHuO=Idb<b5vcA z3|kjHGKL^h5?-gS9|5)hO-^{;w({uMM+<?tAeeFb00yl^#xE~}7|F1W?76#j#ZKkR zah9t<v+JwsCS)^3X5u937xJU0721%t&q>}oO9yBfT;y-(=+Tu{j7GODI9}{_&?e>P z(Ol&7MI$qYV5N8$VISNnHRcV+<m46$S4$!wMYu16k`iCu{+-8ceQag?=TRpAzS-~F zB5I~DBlbtG#|jpAG@SJ@mSSengg0ml|Ak#3gKBg!yY39rX5}M|V_uS=zcTE#GQQdI zo8=?^nqcAHbHpj~{RN$HamcmK^|+s7F+S6f*Yz$IEY$_q?{s!HU)z*#$e8miJw9@G z)X@WTA~u%R<OkZ$H#M$>oL$Oopz2_%>Mssv$k1ZXRYP~Y&3Q!aO>IA5;`HbJ4Bka8 zwPS^C=R4B#DO!W?7zQ6R)h9l7W^FY1i%g=aEVLioF<psu=^f*qR*K9hcc~R^#ZnZ_ z6;g763X9}MZP!_cohemEx)<ID6$7IraPefGcrY_H9qes8T7p{|7BiAnJiL9B%qB*R zJx25$FCjGxr^97G<iD)ez|ir+Th87{a@W&IIZKwvB=NC&Yhc`_{V9zkJvQcBSrfOk zQ8k_^ZW`{0ouFM$q4091Nn^hV>xKMvIh<AD{KRuQ%@O2Xe_Vc_?gxYGrf?F-BE#2R zJQtP#p3*{p=*rNRHg8|&t|RHW?yv?m-lyq{cLIE%gkfk+w+;=H4w!fatLEU^F2ILy z;*Kx~s~6W5WLg29rBd`rJ?l+6gn!v6h=+UTtI%KYp6In=N#+x_#51B<xbedaw<*#X zaSh>xz26h_=r-Q-;vzb*M3?t@kS>z#wmc|_1vamGHlVyO5@mMUlde6SrcENZR(xOb zDHILq8S%UJuqlY#aeWs!tNwIq4T74L@X6Dswni$E?Dmk!_TU<`fm?CnaEsl#T)&OB z_Mf|;?B~#ygQP8Zm4YSjMjB!N8O#6YeW7Yb+RWV^$g%l!<j)4l_-|rS=5#^Z7gN7E z9x_7zd12fk@_-oW_KPT)u?D{=$?WR1u#Ea0QmV(!F|kYqiM75cUGM~{MmLT%G!Msf z#A_a#f54~jT$+BFpL(_L@lQ>nhqox?zK7(_U=Pr%=U$3K(FhNEE6I80JrEKb&17_s zZf4pGJJUD3Fub*Y@b{$m?DoRCZO+@}5rRJ_V1#;?ge$vc6oz*OSe2SsVIkYu#~pKD znsn^9+1Tu=rH6mCD#g8Cu~c00sV`DV7|U>;bzci-mKX#B(5PjjGYd@l&))K5#<)Wd z;X8Wh$hX61l_YD6XPxv1k)Gqr%v=<8R#OZ3Sg(+osDrX8EbF%>23FTOyanHs;MV|8 zwVKt6Y++za>k%&sqx_I;KDJ<+hCs-qk<c*v=y?Gx*LmpBY292hYwtaLcbQ<KG^I{2 z@-F>(f)EFM7U;tnlHVZ{mW5jdW>Ixn`ay{Z?{l|CJOy4N%!ob2SBJ&RUR-}F!n<9z zX;eRJGnQbrzs9AvgUQ-5t!JJfqx6Z<mc|OzMd^;ap61}(!cvZiOi$@iKkzg>F9zsm zegOHz#fMQt_nN;y{(WuSxgSYH+XW>JoN>))az_p7Sm7<^l6Pxk?dz}f23f(Qa?q{n zk9aAr-L=~GWf0vi?0f1p^BC(#m{O04+_S9`(;1bWUCbNeB-d+Yx)MwdS8x`RnJHBO zhyWaZHuYqT19~geT0#Qa2RHwdYjd-dig!J3(xAdw1OI8TKUvZ68%kcPu@HEa2CTG+ zNapL*L6{5s)SQeH*DK;GD_I$3;>R<`EvEVQ(NSU7Jv%=I9#8Ly9Wm`=&~npWfwTGB zkJdjQ;KO#1utU6PQz@jh7=g<uJH+DEHba%!?eFL7j%j(64o?3mPV7hXHe=*k$>Ez- zlM&d26_XQpzpi6rSH5On%KU|&3+a7FK*qBBi+Jo^YB)FhK-98#fBFuEQV}x&r;Vwk zi&4v)-uKK7-S(2J^5e{nM<Z@_p99w$pu08w&g7;Ibs4z(B-kG-8si(q&P#4Go(KXe zJp@wjSn0R>&$XHGQ=44TLx0)_KDmQqRJzqS>&eu{SeuG#%P&>(7ykI746Kj`!q0y^ z(6mW|@)wETjxT=Muq=>pegR^e7(@cnk*4^OZQ;x>$)UndkM0`8r$B@?*%yLzo`y3$ z&(lFV3}X7EFMb)+eAJe0a%gIVMb?OuDUUJSff#WbF0}lgJD*hh^3IPWn!OUI341u- zW{bxbeHdmAZP-_!{uh=?!q0~kCE(FPzN*a2)UQ&qF@0%IG9#nwJlm=SD>OSx?W}3f z#wWR^0(UwO*C&sGtMa$?RB7w{tE9(bVh>e#%}o1|ngbGiO9btED}9;F&g1MWA5$c( zVx^XhKb}r*5U?l^Tuv)-Lg;5OKG%@tI%}yI%I!xKwKu8$*yQHC%}aG|OMHd$ildMw z*lm#*1C<&FiHVo;k2Q?7a~eU8l>#$#*_1pekAV2vvsr>{al9D0=X>#*25y19ARtyf z8sE$fSl||oKepl&>nr;8dEAG^f<Mkka&YSYt|q2<yj@@IGU9si`6#)_3+a6PsaG6x zBToE%Zr+sIh40wwx?+`~b@nC?amd^Jk^|wX{)Gt1r@C8ik-~;M;1~VMda`8TNqhUc z=Ou_6s!-o1j2?&gtoeAu&(0qgZ1b}bEVsxOhz8vFbbgAIgJUbZ^Rw#L<{JFh02ALC z%egA{p*eW5Sdd&9#L|8Qm7U{yHi<|V71~>(gpKL^el#fO`awXmOt3)voh>!si+U*< z0MM_Bj%sc+oKid>eAVr)MxOQ<<G6-(nWTq|$THP-?z;PN84u{m8?+1vNH(hRK9p)( znlsifH`8fOPVu2!^jq^nQ08$8kO`ZX*4#E{@s^x_n~LXZGA^zG*mL2=HmtT9eBiD~ z$7FrN5+zeiAVGi~8?W0dCg@^iDOJpB{1D%yUC*;$@rtMlHDuXLd){RI0GoW!-F%E= zk%=4`>zSO@*0RjiqM2h`%_{`(A|G$4jE)Rj>2~(E<X0qAyE~cS{L@8itH+d|u(P<` z*rLPMvF8FYr+;1qDt0s<eLMn3Q%iU(*w}~bjz&<67||2fTIRr>2SlWN&bcsc3C|Vk z#?x9FUm+#1--55lc=)m%e-RaIg3qic#ad0O;{8!Q(u=P(j<49Wi6cN9e8Po=P}J`9 z3VI=MKdt;XV}`IU`qILE<l2@nFB@lO&|8eSZmyU8iCw62s9tQ=O`%(1S$8n=6}MrX zl51)!qO8R7agr6TRr&Sk9kYyH|LY#~80Bi>Ji@&xkpr&yfC!~hVDu9AnnD(o#=@Vy zeLKC-%!V>0UtJ>03)!(l?fjB_@Dy@Vcfqp_t)_3L{Bo~F*3eYU?$jNOf>i}c*`P#4 zZ*FRs5B_>SYr?W0)+09li~6M~5!f0e8Yf_?tn5EteteF@41uMv^V8Rb@jet!eZQ}k zfX2Qp-z=RX-O{MFZvOaxkGS0B5r3#Y<2v~r_V9fZqp(nL7&({|x-LJ`Vf^Q|4#2nI zxZ{yOr4`cm65oT{-wu+hH}`}g8SWeL$ocgc?dhOfOoZ3EE8K1ov`<-p<8V_={)5M$ z48~!@Oi=+mLqYm43aYoAk}AmPc>mcXIOixva8JtP@3-Fl`BS&IWNR@|AKxsf75plS zw-#-7n21*yk{p9s<X&oti+0pA)A;I=*khZl*l50r#czv=j1>Mi6~-e@a!R(`#X*h~ z@@(O39<&lK0p+c~>%zeHieTIB-CZwBe&a_^b;0Z>4b&`BV*P5G$!@sA&AoH=^wx91 zt@Nu`A;GCwLjF-Nkaa0`usvaX0o8!uHZ|vM*S+4d#))kj=ZI0$$wo{Ro6<pm_KyQu z>{l=~O6eccJNGx#($<bPbD;B0V8@!jxXacn$l8dXOOYf_|0jVgDS0zHVMeX~SOU4n zHCF8U%+-#nzpCkQ^k@ej{^ejQ(Rnf}Sz8xpTe-$j2|O&0-OqIucLOVrE{8K>2U{}| zZ7BH-%kfnzok9<7WtdL}9xAh;%_n5+bcZDG{LCfKN^9?rwt<T`?_rluXcKMaL5*MM z$m7T65oCR-<YVJ7t?j~1t;ORt{EZw-8OiOYmK9Gtv;EiV+d7S~;{UP$kFZ}L4%l39 z%6Sb~3&Q>dEM9_f<M@G0(Kn&SMkEKBjPndtzK&%v-lX6Y8<PhV&3CQ-j?a<ZNw(z; zZB7)KARbeipH^yk;!dG{9bansn76f_Xx;-GjSlYa%7-8<svZP45#2AZ>^Q8NWWjG- z%%7pMzfI_|QMt7wnol5MwPlntZy0?N?aQcXre6cID_J<MBaNB}hb8eHQ%vqK_%q`4 zr{;R4T{i6vqYEhQaX-d>@=*o2wRxQM)>IN(z2_2h&+e0q5ICgjW2`J+7N-z0s2pCI z=5v~y7nrITlH{x!rLhj^Y(yK@{b@L=m=M_t@DY`h;+&f0du2J(Vk_sNxgTgcAY$4R zn`EqKSYUouAytfaDvKop$5UG(CA|O>0C^3!Z*OHeh$Mmb3|B;BVC$&CB!ABoRDXXp zHgRAnw&!4$dL5Uwl{Ehm)v<f_hMZ0A-2sU_9@~O#2DyeXZ1=AaID7T2c@Qtr`V2vM z)Z5k92N)FM5asPHzF9IIdVO^TxGjGHB|dPgXt?wUXbXe$pKQnV?nJXle!LTZ>d|#6 z^6#$>2-?lbpg3~Z-y5>d9slIf)zN3=`|^KSrs2yXTp`c3V_k)~e~DoB-QA=X!~YTt zVK4D?A=*K{5DY2(AiRX>iWbou{W(K$rjf89P4rtQ=A9t4F+C4i+)c^V)=d9~?b#i$ zLScl_86?Tx@M8KFGQD-dD<S7THw$3$SC=!!LS{z_>EHVEtn0(Bwh!1T%xB}mYdu7N z@Kk9#)c724OO*i4wN-eF<^18<>l$MgQCA%dnb~CX3i1FVir=U{|3RZ|Z77p^mO74f zmgZ=4O2o9+&N>eD80-S>kT6k`6`q`LN|BpTpK+Qnv)s}R$*ATRGvzg+zcCRW&8_`y z$fkyXGL_nvA!>E3dRN+MeZq4s4~#r}Vm}T5X;pJuS$D-7qVn}jTk|kWEh5QdgI(t- zP1yDhFt;d!aM+B)Ipq?!y^EmhBU`F-WL!ShjNkj>1R8#FeHVL=PbovpMq!2n?&Wmw z1m09QK<7Lv+8>ELFq!i_03DOVx(jAD@!Y*3xjVF29K!JD6!=aOT&~pn(V*8>gWSb& zua`Q1M9~W^r%9WQyCtsn37t?mfHKV-IC8~2^G_do=4A(|=e!NOG`wazs6us)7lM!` zWE<JmKfZUjaRtXOk<ou*+|y(GaH2DX%y7Iy;SqYc=w8~#l)=oAJPE}!+WQs`knAS? z+)L}LJkg6#{7U+K7N#r%e}W&p)!U7W@6xq|?<Log&`km?{fE%f@QTU1o=H6mfuj~8 zV4J`X4N0r&Z=w73<cw<4QSjc??LxlD#sO)<o5{V19u6-!&((X)#<#1)ZBD1)aut(7 zmS<E{9BsBsNVLFt<aSBl^wl$Cay^+iD)if}sTP-|T79e08O*kJu&~})bQ<euZj7U{ zuq1`N*7WwHWghhnhaJfZSUBeQPE#w%Qp|N=lnY}`y`YMXmaF$Jb<UKOu9Wh&mwC2z z_B7Ogue;TA^ff|k+ji`r2}q1WbItnEl_~EYqheYrV5_yJ$1LK1afu3t+M3?iA8F>g zj0xrFUgST_m?k#m7N4<hZPG3)>Pzv~Ji8PbPr6K)gTQsXAR_s)168131j*U65a<&= z;$porJ%W<+lqR?DE2=;#TwcaK9qqSGS3he<S{typTBhR1=i|5E*adCm0N9`VrdYk4 z%O^}|e9;AD^-(F$cmx096{z>Ffne?a4Q8Mm<oC3p$w^~`6#_a1&%l(P14_&aqPUYC zAu`P84*LU&YCzuhD`kd;{O|dtx*WHOm83!y-PenSh12n#DSn=a5D6Cxef#vE1OtuB z6D;#Qe>;B~0r}$bE>(5A(3bWkYkbSNab(Y1X=aaH|Ay93201!?r>~=hJ5mk+FDf}0 zy^x0I6TfP5?*cmI%1E2!IiRP1NxS6AB2H`Z<e&O_>7E~7Wrh#Gej8$5A60CIOBAVI zd4u%cZXmfj5erY>nrB}1ix0ZnUvTesl6$O8^|96FfoQ)i$e$pDyyg{WCb>o*$;5BQ zWrl9a5-=a_qao$(hy(aQqKFFOdL?+b?Sp9ZC8O`c2a@o0^>uKw_98Il3l!^s`FS)| zW<%kUV-V8IDiy0dj$;+i>~j+|-X+XA8PpGOxg7TkE1hLn;U{<L0D>k;8Kg%VPSN9% zgbver&ebB~kXA~Wf@gGkB(A4mWdvux>*Rd0h@@Cz=SDRhUUiA2Q@v?Vdc)6PZ-;Ak zM3G-xDo0cDe!FC5bjbeFwJKCt_9$=Gn_(xMQIY;+fpAse@VVqe`a>B695(QnJX<0; z9p{YA#L-=dH7u`9o#EU&NcY~iBKJGza%7zSK4g2viVHKxO?}Gk9Zq?=l;(}>1bYLf zu&?7hFGeRvkf!k`dYU5g%LWzt^CZXJCfw_c_P)VOmeB<n@`~|N8q6NWd?C(8$MlE^ znXzng-hMb?<esFMiDd7!fgQhpC{)!{nm-*DxwRz0?#ugXTo=T*W<5Vmj$0I8*D|Hy zET%CvME+5jNw6i@3$E0=Cm%YaD6v)j{#q>+>-wbC2J$L*?ZfLL1e{h27VQVpuw&gr zsFC#YF{RUX7GGaoSKL>4n>WtyT|xyl7LQ}Q1HZH9L+1|E&C$EXLE>BAspDu(b%rtX zqX=OI{W7WA6{w3`sDwN;;bf)Wh(DO1ZU^tn@OXrGRj&8esbdNCIRievoNoR@+Dw?6 zbp^hxMXtIpcRV}?jS_b5I!j|l#H70$IPOS{*y^LR8BeCYMH!OC`n~`^<qGd`asEft zk+yV#DO`hK3>R)Y{q@YKMOvEZa|Lg07dY}sUxCwtw{|6`iV-cx<dKa1V|nTlHs#QL zpW=92#Ie98&)x%;>A!jOlZwsuZLY|Sisw9<N*KL07VMV4H&=XK-d|}|b=tjnLqxCE z^e6DjO8@+v+xB<ysTi3<_uv9Hn5iIw)aoPYDbSPz!8Dgq*h<S_^h*4|2Z6Wf(Y!T) z1Ec|DW*}T*xmK`|L(uA}+RJ+nM_XH!&f)-m!a3lz^uj#)`s<<-Du`<=zp>x*?Mvl3 zo@SnJ{NakupDa@CaPUW0j-t-;QflzC{xv84`<R(7ukV(Wk2^>A>+~8k9*CuVAdC$0 z_DUuCr+lwh#DhXA%K0H7J2yo<u1R=6?hWsO5f2&bZ8YFQ*GbtX0dhN-<bEEXSHgDN z=;WXTx8AZ3oE!sSllRXx;Sn_RlZzUwhs>oIn}BQmKIUK9MRNo<9XJlESF_~_sO$6m zSrw|tXt|R~X@q6!F&^h+iIZDY*30Urp3~*_>UCacptKKnK)JdE`?D3#E+N;w*qJBJ z{C?GNn=xCD<m*q}Yb!Ayy26%k?Y7vmwm;hj8#JKlQY)@>zsry~RIfY3m|yM={|F*u zofei>I5Aa9=_WmKf@b|}SmT_T)kYTm)iR9=jJiS%yNXf<$~pT-W+1P-&8X-O1v5rA znpZy#et**-jW$uj=@zp_9`{u?eoLuEp=fyC{nqxNHAt@@gaMO6r0QvPRBVR1T{YCJ zacGAxrf;|fLN23o@ha-Q=i9Uk$@;b-0ZVTR-xyaO@3@dI>mI{egkXc(i4=VaBKCN5 zhD?6j=u&su`tFe~^jKH8#OAALDIpQgx7|sS@!`_vuITg9Srcfb)4tUtFhe|Nrg$8J z%_mX$y<KSO2~Xw^wp{39-}=$t?VPv9@wDgh20nh8B*Xb(q8!~Sqo+JX%!R{n)DRs> z&G#>wbN7vJZf<L(9LHY?{XO;0zN`3yttPL>dX7^`U_Nh#S0G|iP(uyl;hvLxXs4KJ zFqnxvvV+r!A438hE{<(sC|Uh*;)H7nyYwEL(VlH^LqY6O4@*}d{*!K2l3(|fgP$FW zfyGb)bjhF4Wz*#?ftl~?FqP2@nUu}#fhr^AOO?)IJ8@HGAN!i?^cRRpILL?^?YCm5 zPge4og$^rbF-ypWb$52Fqt7#3<;%r3OK{9b6c_jLIDAX=h<nH}ErhbYQcjNUitaFc zjSsCoc97H-mY6V}j+H#$1fDEYsV!f+7?GRNV$myoV=fhO=wwZ`49yYL;A!9JQTZda zVZLkN!+0FpQrDA+TCmd(UP-O&N%9Tv1+6Rc+TSB=k54Zq_t?Nc%eLgr@t%Tfs&Cr| zV=^mEjmKgzyOL^Nd~caBlwG-?-1Hokr#(b14PC`!YGb7#)7=^OQh&Fg4w<(A=>#RN zJAaLC-#kA8`fmRv!bThr_6C51>EiHpRnRg<g}vJilA39+J6VQ4IfNkD)H-BIm`m)x zXf);u8!O1r278@hX}%zVa-@ha(xg&*{Fe+1X7gtAje$f^hQfJjhA?ov=dO&??}d?T z75>lZ2J&hQO@9LgeIGC~D2+!_cV%SyoEi}AF;O~C?JtXl-@62KJV}l4+DjP}hraK1 zJms2PfM__>32*;=OdQ^m3YL8SvZ3%gJJ5WxtV`WIk){|ysB8eR0pPW;4M4;6w}tXf zmmRTQ9o5xMxT42(vkgkCEh|Z{c&os_SOatKp6n;LaQ2J=pIq`YtU4(r_f@|IGAwvw z)1%jREDt6kK3!w@W|k5!jkGm-{`9AMJpa4>Dx!Y)<N>IcrQqws&)qw==bN(htDJ>r z?vAZ4i(8U^$j?+kBn>I1&u|T~he4NQvvw80oXz-~^+}q>(hbr5h=`0q_zJ+rXvfMV z(4^}_=mbI;IwFi*l?%p3QW!yo`&VDDla2o|@zC|R5T})pZzu<ZVf#_hRRX6AJEDP9 zMAu!Vie^^LWBSdb^^p(`rMU06?a@Qnp|I$Kmu!<I4ns;JceQ_y?4EY3tk!es(lVv` zI6I%%d(G+yAbBWcBj*k0rA-szv^8L9qnn0p!ahUd8+U(`dtK3dDil!oeP(xkrB86N zrss|WwWM_>4O7|r?EFXTR)ssPG^agZphcUz$WqH5K~8?V4O09Z>{O^OvdMTlH6P1d zCRo??ux8=2_9OjN)oS5?P^_B>d~NNprC-dAoqxw4Yk81X9OYVcKDTP@2q5YbgBDe6 zb7D2$-_uO?B?xRkoU&=9EAGj9;2v6V#$KYig3=&h&{NdSKd69+XcC~fv-Oe#s<z4x zDlC<^P&AO&=t>#9O1OWYo?#GQ`yFibdlKNAz~J)TwFzMNZpn{2!Em1Fz~I=*@lA1; z1SBgd{?fS5j>9q{9-aWLGQ#N_QUVpNY->9<kDR}JxymMxJpYKNLc0D?YGN5~+2*{# zzMiczA;su0ics<9hzEq<5l^q|qxDM19euTFCnN$l&1SkUj}lfm_fo%<)8-86>Q-0| zlW|@v?ykO;(^szN$hETMOHLmbtlR9J1@18ix+5r;aO>iZZb1)7kH>Gui|w!!K4p5< z6n6H3#e+g-dq+DT6_W(ltFz2<Hk#qP<&!JcLZ8~D{k70d_~*aZqZaAu$qJ$b$WMzx ze8NLCZ|l;tX28a>cmuG=qM99?*?5qObebvn{CeBJQ>_02EB!S6);doz?Hw&rUc$JU ze@{eoeZMc)9uRgzB!{N!d$aPtgKesR(t#osg}xB%#`$o?=YC6>yR`V5&5_>7UQm#z z6w|75=1Bx;CISA(U$P;8YCv4^w=Jjo91F#;N|FH|T<e|f4{vD)^YyDZKFMY;0d+08 zv<>m1-+tYHY$&HMw7ZJ<JjcXd*f{ZN#$RyrX*Z4Kg|EH1{b~B^{T+~xq{Nx%^zE<` z&}hKB^E=*q#?(U5T>F<&>q5n)4WMVIU{WGqiDE*pf?t}<!Cx+JfJ`N#)4Z4Ps|l%z zo9rHSU;25p-qRnY8O<bCqp9e)mm&7%-U4{-NQ;qMGFwt?CFV6#?dk5EUZ>_YYrSaR zzFesf%~vWx62V@+JBX3}E$dHwhaep?Q+RIOxjK+$j`O4(@&hHzxhS5GX4fKv<L|df zeNw4b*+T74wb|BexSe)7RI&-~SxI$ZgT>+B-12nctj4zdgQY`{E!ghkis>r3#x`(l z;uI>%L5>Iup3$wLD&|#c$`~L#?R@+9(RP!L8wrgXmq%b6i#e4?!W?0XMg#j3(_Bwi zzD;l_T^IAu?`ribSu47(gOn|mwlvJu88f|Ir%V6?QRjUFGJz4ELQXwWTu$l@6q?5j zo8?A3{3kUzeW}d7l?$UQ_>Wc*j;nYir2Rd<IjD}ktp)d@v)XMzI<rX6`jLx8I9k9< z8Py_Yqc&}~!wTOlU!`lrj36Vt9hXr^o8XwPFi-ZaSRe#pW~4sR*%fWACqea+PKLEM z<Hdfsqvm_7Y$!a&5Dx;Z2^dWz52_lq%=>br`o#6_@iaRUC@R8^b;9Y``B|%4DNew^ zx$et}h-^W=^KmoJR@}_@Im3gAmSu(*N3BoQZ~Md19yzE8Pfu8w9Pbf~wda+6pP<)} zz+IBspJNIA1E!nVs2_TiGTYkT)1rLnKJ%^>FU|`uaQ$6>DG%6K62U@t@$)}12d*?1 zEMO)Opq>oec0{rE3RVA|FdB)Z7v%vvxN1|-3FkcpNQQc)9Fx=k7-SWG$#nba$v=tq zu^Fa&Iqq}!g?+Ud7v48{>Qww#(qMWw@I~QB_5!TPx8FWn1{B`bBWF)?LetC*cYS{^ z9zxWwj*k;J--IuDS-&%`P=VR0jx^8&$a}t4G@22VH3BO<$u1FXg=J+xg!}^6VPfZx zsmR!ym`O0z`Z?+SgAqqvG&U*Ynz1wIX^u1$fqbXjxA+>pSpfju=fcZ*W#=L-Iaj5g zLj8%!Q^ES|`sZaJv{MaS;A-g5xMN!KjTt5P9*}Zo7Q$27zA*$GXHD(rkG;=NyQP|m zZW)&3_W`*3CyUqde06DLk2AQv0|?Aif%j11V0~oh@N;f^UG9`$tK?V8&?!bYqa(6a zzrDW9nbQ05xn%M1jml*J8izK3s8fLFb)X2ZSr91Y*mtFe{|<>c13l<UdyfAe92>zb z@pC&Nl{Oc*zS-L=CO-$pY3U%~%T(ELRI>TIkjLNV&+32<_B(_bU_}yVQiDpRja(6c zSUt2YHKhBY0cs+OC-ueItaGSR=VQM*_IX2PUG{xP-+YbC`<pmICz-hNu{QrHfhPUx zf785fR^bb4hrtVz2nK0!$szp^4H#_3L*V`U-%|0qf%5G9zf*aXIi6B<*E2|L9O7we z9PI|~JOM|j-qf*B+>7E0%@cy?u>D478q)2F@4db^%F#comIEki9qGUb_c-sd7ouVc z<oUv99Atzw&pZ|qnMBI1?G}RgUeW%0WU8J#-#xABMKDU})msaBVaItR5CS%1l|>i! zY*L7gPnC*q#&AEGa>V*VqzuO-`5g8Y-G%31lf^%QiBZ_|Z@N|SpDqmow;61171j2` zm1H&z0!`+novPs<DaNqisJ-eGmcfM&RZN)X1jjA!e3bSR$xKq%k*CSE>vE!nFI`Yy zr(VXZx%HIi>h{WVJ}IE8wf&MN+(%)NaQJyj8$LMdzk%W1Mw+R0(~a4BZi0iv^ln^m zOzMamCh-M#RD1)EP0mi^iV?curQG)`S!*mI3#OO-*E1q<r&SqNujBX1QSHL)wzHHI zFmnq_z<^&SVy}>E8|~jy(rGUIXH@gAxfB$1Ttpve7gOI<y5a_m-RoPuUe|)Woxeg4 z4_RpL`H}{7e|v-fiB4%U;ATHwdtN0>M$x&hpA;+b9FTy<$)uj+PlCDH@4PdJzfVTQ zit|xT4Tm@@Swr}_vkA!K5aXZnMKP}?%hr{}c01|{f30B`OHPJ=VJTt<DJxCyNZOg6 zOVr#BUwdken@y1ECK;Q&UBC?K6dixf%dt%jZQ-lgAAF@dOvWf=A@9RGbn0;$dSrG? zq+b#}ZlkI>?>N6F%<v1!IvQGw!#>C<l=YYL#NHX}WKdIB0<pqT%2Fpr*1Evv|BFi_ z`;NDO?{CNt*0^m7o-<2#tZ4^SaBIFn(6pE7!Ia(Wlf1GNL%zzyO6Dow^|#%hc8mXp zk^c@A9;6Wbqg`hX*$r&syt6sgnD$3;-`uh;Tdprja*scipuYvToBN^D!o#i@ETGQ* z<k<S?KfwcP@y(cA7wK(6CD_|uSIS3t8=|2yM%14;3VP3jiH|IDHYq1sC?9Xjr16L* z%mq*@w=;V6>eNf?Eh{BS41BfpoRC2@36vkb7=KU{io?P{3Z|#w%<=e~gUsh%_Al-P zfj^>X8x<<8VpHQ?>$TXg{MSeknCzqU*6@JwanAfU!>Kp-1nKWO&*SOa`m#4CxOy`_ zmMl~i>Xfn*f?2JH$U$2EgDZaGOfxOVaz2D+UB+tezfX>4^m60fVYrI(l{D`w$6ERl z`L$5eU)u+s6Bqg`yo3QAVY;<!)UC{eUWoFhFlKfN^6e271Bq=Y;zlZYuVkCZf&?cu zBA1%(*0rmwxMSPj<HiKiu}+S8IoK?8jkJ7s|4f<nLA0--j8+W6SUC^Yg@|53a;rMW zaqin016<)kitP^FsRnYd0^<OPr*d}Gp65pNO8rXghzrG&#|8Et(e8B`h}WJ;+JP4p ztvn}UfaA`dySFk&Fyd`xh~Pax4pqpk>Tz9MAsUMbp(oL88P~7l3^)$YXw@HE;iH*` zhfNhxuD2peSv=+~kxKDy%I-f8kaA%Y!h5U^y<*lUqvY^=h&~sx*O|?b_T3<^9S0$s zxTROtxtF+5`8_)jd9|3>4a?IJihE=j^ip6)Ck!ZPi>6N+fRBLZmX%k?-G6HB|KDOI z+uaP-jwDkY`!<~YxNY1laKlBkpS;C%LE|~2ZpmFPRV~urg^yS~w6tEdIaiRzKo+BV zD<0xWdy>*E{&=sq4;-;mMyMZcD{-<KnPHUAe&a`Q#B@4shx9QSk7SLV(pT>EvR@_S ziWagQ?Fal-w1(t~I~Yl<UT&5Ky(D8n+qiA@xka4V)<CsooU>RIFRQD@2Y-r>`2f7Y z*Z-1RopIt0w3OBCocy99ZI=O%_$~{2sUBQZOs1{0{xi8=>jG<5%pJV2%Lwh(b7LQ> z?Zj<2=rKK(d8|bEtYZ17dk6gVP_WkV^`upC4{4Egj-?T}vVe%dW3AgB3Mj*_McaPr z%Ix$p<FO$Y#lv=A(fP9W$K*aIN2mj5Ve`R&NmiE|c~dL*vMUWw?i{W<T`gQ5e5bHe z#vMFN$fh}yCVt@gE{<CMzFa0R2}ipQllKdiI7vf`@W;2Hjz10Ais87!j~C1$Xn|~) zx!khjcgy^WAQrd9>xNO=M<A7Rz%qyH?%1y|(>fibdC$6@_oc}uDu!LqZFBZqaH-{# z0Md-PFWFZUnW0Q?SweCS(l7VauLdCIQ*+%kp`>3cG)uBdjU6xEV3;fAhz3VP=?koD za!d7OO@#JJC_S#lmmj)V9b@q5N<Td{%FFq7epHg%cKyYJwPL!>f5++2n_yhZgnGGC zqi{yCg|&$&W=Jf~Vz~|03qBE}P@lilFt^6VDsv2vCa$qQaStc1jSpF(X&Q;l@m0Ct zS7kqKr{Pi^3s><h{ETXI5w%qAMo_ljDv%i02i?x%x3Apt-7?UG^6U6_oQOo7To_Qo z<qvNvN>5q&n!9LBctPpq#%MWh?DnG?pdd!mxq^zl%?rMzkN5DP7F3$Or(JtnXbFdn zQY0*#Az#~NXtXx(oc#l>$X1%vN>cjz9BBZMKzbd&S-Jc!H<Z9|^lVq;Vb+tzwi(8V z$4psJra!5+AGNtu^mTxW0I{M^@%RBc$Q3<Q*YU~oG@9DmsdTEDtvBun&`3uK-P*+& zAl4&DBg{(B@$-Rs>{rh*gLWZKX@LgQ_XQ!CN|rw~T(eny7Wp6*cXJ>DyHzN9W9U0Q zL|f6YOFMF>N$0gO)ic}&+KWr_*D4Ok8m5<hbQW|#SV^ccY#nTf1VxPal29-=NxL)E z%m1k)FYnFy`@l3!msEN;#Tf3j^X7Sw4*-d8nB1RKy+IQUm3;X(S3ch9@;H9==iR@- zebZwj?#EX$zpGbz=zRRNq}Wn;r?M$BK7tJTOI-FSj{3DZFn?wMQQ<WRuKaB?Q%M+% z0o=^3>S*<sHU9DaA$^juM<-x~#LG`CAAcj6MMJM+<cRKYYX%fNmcAgVEOUWkmok62 zStq$Zm99KkKkrz{$nC=aGO~8u#aBl)_g*GAqi9SDxjL+yl=6ur=>^PFu+Mb{Bw#Ah zhukYdjAd_&Bo5R*%F~OLCf1|8=Lkd=9XnmqG9_Yzk_T4w{o>LRCE`@$KCj9y11~tn z-G?9PuSJ~{@0Q5?oVj!ro}2<ddpcmZBr$P#D!r+U7g@g~tKXer5B7nC*#EaBWDSR@ z%LG`{pn?154<ilYn>@IhJmTHBVm#sZf38|Nccfb2N0xEdk(0Z_zaGq!7qL!E7JQ)~ z=UsCo_T2CNo*@0c=Jrdww{AAG4wdDOw>OQcIh9}i4ZN4(M(54SKLB00@?RFf=!*5s zTuigbP%A+^?r`-5%p(SE3Q)5QlGZvo*4#UBxW&Bvji#m&XS~yLht(lLpoExjPgt+6 z&~$}1$&X$x4UY#v{a&@@3sR3<zjEQ)<oxTp?_dXxQy;~B0evv=Q)}-QuCS+EL*mIe zUt7NtvkXuzi*J4Shj9UeG0~gtoff$vM>}VE7t@_61$~IoR?rp|wCIrqjJ|=6O9sId zOm*N{eB9xQ3)i)2G8QP>ei&|(N=~8P6gc<9tbL^3W%GAzu9fh7&R1V_RS|i<w$W0p zr$>Z@3xD|r0juNb+c0Q8T11|Bii*mH(dnx*9ciJ>lCYi!sx0Yo5|y>xUt*GM)w5Zb z%?{OXme3Y*)~+Lc&8dnb58h{)v+mZL(ZyRf8{37L6q5dM$G$g)aeec2sT4BLG_Qp3 z<<kF+4Rh~{I-ol&$3IRezmMII73r1K^=j4K=~J22uJiPacC71{G(C|j#(JTy93*^M zAw2%mx0h6X=(!?$;2pg}J3Jz3JJC&K+7qzL%(^<Ve{F4@62w9;RY)kALO~yJdiSc7 zbKuy>iu;u5%^45J$-mfq?798c;1OAo53bJGF-Q9Y#3+Wjn5|05pjsSUjaM=uMUvMT z1@9F!fSyAQ4DTD^>YY0LzBBRIps#O)d+f{qkrn=J9KOFJdZit?zz)0XI>@aQ1>?$^ zh!~K!PjE0b|5(k<rJ74B52?^vib#~Wmt6J9QJB-Q=t|easP<pMDMrRb8uL8Lv|YKi zuLiL`k>c%dNYvQ<^U*0;G+b{puu3|r@p6YM^<w~I9R#CFgy}iRnvw@d;0Og8e0kCw zmd<;;zJv@GrN@=54Wuc46QhaD(%V9uNJ_dmvhr+vLGxBD-dLY1vwcG};a-Pbe_%}) zjS>$n%^CZ-zFEYuS#<_0aawzGxuxZbUD5poUdHSaUE_7P6|L+)FKMvWBQHK~S5uUn zN(i)%=jn=6&IdtnC=rTsdRTAIHC*jZeEX0^rW*haD!gRC@~LVfbRhKtQK(hA^qDg$ z87!W#z<n?9SGvvO6>3l|B5f!*r&@pQo>H7}n(aBCx0H?7I(*%EHHqT7SyOruiVvoA z)h7-a?m4Q*n0xQsgqi>t5nGGQ22u&d%kDQuc+|iAe{{WhJk<aDE<Ok$OC`!Kl`W(& z*0Dw<REq3mH<s)MV`-!8l-&rStXalxP_o8k8~coX>|-B>;k@*Ie}CumIlpuM@<$$z z9%k<6y<GQoUst8X3*1N=I(Ky7#+*0ZNI+q6)$LSDMx4blv)O2n+nTg>TVxiXfAw|B z4J4i@Hc%ROx^sF)R8r?HxtSpE`vQyQgfl1+JK!*~IfTURROe>9+(%Lo7{t}C+#lR8 zGKsH~m7PLuE97@Y?HdB1xIZ8gu~gc0{DA3&>`FoXQbd#Qz2#aS#q%PbrMHatDULJd ziFG4Z46B73WiCn?H7HP}gZ8_o%@j1pS*sI*Ha1HYer$59Rp)^CC3$z`hDbZG6h950 zPD8#GvxN1}k9vuR@ul~sO};w0Zi`RYQF*1#r}s>xM-BbB!F0Bh`nFo6(%^#maL)#R zWjojY4?ccA%@Tr#^ak~}r+^%$Kw~eVc2^EHyV8_i%RphphR>qA9lk&{@CM{Ct@VS6 zJ^W^C-F&MV&QAcNb9PPE{WaCIH#F0q->;D7dzG~mB*W6rwWc31TPd_n(ObSpiY#<p z3VfD{GbNam<FdqnrDMHYZ7a9M<U@1`gve{;YseDMSv|Pvi77^JzZz3MZj@94?_)h} zWY1f-HJP=GDGSywx7Avl4VVu2VG+p8{TPq)S@W>A>lpbN#Sd{efv?977wepxS!D0` zodOG0ZM%M`FIl<ucT=$xBr^O{3GH*V%+bB9ywVd}$_<FUL>+om*$n&TM&CU~P@6fE zwSqKNm4*+wSHrKwej5E_ONg`JXk+X*(w00{0-o>2vnfE`FGt+0ZthvBd852ehWxWM z#@6c6{wHpL5cJ@8tDg;wZ%0^(r%J1%z;#Ikt9$x?JZKH-4>}b3Y&Mr|DunY#&c$%s zi}2aQn%_Cc-KDxTOH=jmV1_#Pd1#SNR_^mu*_{h{D)&IOw*VzVA>nI*=|f45%Qh_i zG(74Gb<TzI$@HTEK-=&XBtK*M6BvQ{VYF4+NRA!PU;a<+GnWmFp0gJ{*V6(IQsp@9 z>5ts<TzL3bkhbx%PnFY<2rR7Hf7dBh(VD68#K5)90NGcK^^Z99WSyIh6VU2EJ>VOY zz*479-9uh2SqmEXthCN$T(ib@m6AS2%Pk~7b+}@^=&_`<bH^otie4PJ-v?RRU9!nl z0`8|5m3Nu*k<U1vyzVl2GWmhiskYgR`bzzY+XG!YJF}CJ8VY55q{J~W!V<m(q`gL} zpZ9!HeH=PgGz6SuL(Bm8D5KkRM`k+_yY_%4j*YP%XA_idND6dtDuNFhF8c94Dcgmj z1O=XX;lEys->M7z7bi8=w^2F{@N+4~6-qXLe9y&wD&s<~(l34*Vh|l*oAmIy&5(gM zVI{2ZK(KPq?i~BDk1YOylmCPfHIz8X&hwCo_N6eRiEr-gAujG|S!nNkm96E-C36E8 zubv-8BzH_5W;ANEZcyK4;ni>IP3Q_EQEmJ%&zZJ8=rOxz^%|IPEi5sjT$r*0P+`@! z2_OCkd5AAO9Zx5%E~d>i7Js$B54Vrcb^IDtb~%5o?MW6LKo%%r%w#M`kMt^GiCJaf zFHg!=vT71#Ij<J5gQoaCuhR?h1cKTHY!{ZSF3t+#tl4I#{K8^fQxY&5#5<|({1Cri zEUJ=)rBZ7i;J1{q2=9|lOI_z#gev>?bW3)(69<uaj%rz`GHYPb*i>E6e9c+`KiKsf zsUAz53$Sw^S%)??7*MDksKt7mRG~zFKE2evcKIB{CqtoENjpjv;{Jw8jkIH5ekF6f zH)DbZkH{y99rvUtXA&4D@H9Ak_~2_S7mm1Vwr<6I6${)byWkO=1UaZKabjA5nIBO* z^wZq=1*Mj9Z^gSmx^J~y@RK{(pSz>AveVX9qKJ#86^MxE5%!{|L#bXXY<+u^_uv34 z*t@bWH*?42WEwQ=apLsDqJdd-`APO|KI@HgA+=Z?vQ}F*VrRqxXm4Xnn}Pd_`+&?) zi~)zuyaVw3KK4AOa9-}WjcKz3JNy6tW>bgL*)&2EpS^k)Tk|f`9_UPkCgH?M>Kht0 z4MO$;d&n~3&uy(|SKXZNJojH^q~`Oq<fFJl2V_o~xe!ap)c55pz|20z;={0^lb<PU zlJ%Y95n-gv+EYadIVq)wjV156)s@2;*KAL{(<r;U=A*#a&+C)*c(W#xUE`g}{+L%{ zTq;9bhp5w|p-xMV>{I4hZzG(w80Q}IMea(_Ppfyp`r(NaxPJX{LP;%L$lJNWt=Ajy zq`GY8{VZr<s<I6v1$M)xcNrFHHyM1rPrHWFp@?-Z5G;*x*Hcshz}VY96fA6zt-b<} zZ$@vQov7z62<vZAZ1StW`-5SO&Vu&JzC)QSDwkS-Fco$^UES#}fN}G#B7^3Rk*DKk zr=U+RMF+6f%=^itaC0b3t+~CNS1|~e;RT2U&F#MbJqs8@_+#<Q$qN4>aaeDV8E7Tq z1!e3wZ!o>I!eQIP*r(BYz=c5nt1)f`pD)|J3Ngmjt}cTc*<pEo;`N5;B?4d%os*>= zCjT~^QxL9MOuU(7qf#a2v|w)NXSCkF%hO5`3@?vNlZwOld6PS=4Q5N%xN7!<t3}(H zx%|I-wd67K0TPB)=9fzU88@$+P^1;LIsI-AFujItK574<uQH^(M&=Q`$X0X?nKDkB zr*g8q5NH_N#r|qGIwK+U@g9@K;XG`6$Q;w0&j?GX&?Cp9x|ew6M_daG$TTNDZ>I)j zKIEXz@zq6|0XKOQc;`05L*ERA$2fMIC&Q6$1`;@L-m5+u`+km;mf1CJj7$&N=v$v; zT^ru*P6>t`m5eQ6O7vE@%@iPfG3)nx^SK|>dd{bM8zNiqQx^|nFUlnptRy&n#ox_@ zCAsWU{E%IV@oXA6P=aj3F^pyFfe4ctcP*4J>Gvs3Aj_qQI}-)RG$i}3dzSlmgwwZ- zl~@_Rd`b>#>tpnOENe7ei|$F6)4(k&jsSBfv65xO{T|D+U3<=8Az;}j#hk7Vo;yS) z2jFTre=sHaasO}|tZf@Hdto0=@cRK6&Oom=9L2Lpm{IU2oyl@`)7-2B)<TxE&zjHJ zS*l+K#I`0_W#>QYI{?%+*e75jk}i0x-W1wj9Omc!-@|4|6yWhw9`%58So*e2NY69C z&p;RyEAX(^y~nn)YVp#`(a+C!Fu!<mC{ILKC{>s!sNUa<7Y9T>U)9=Wc~LicL1wAc zvbnt@<;f+Dd&WPk-sVyMxp+md&=9xrv#exNN}0W?^q`4(k#tAiNrL)Kft>{c0+-)i z>=Pi6RDGGUkicy^h^LmiX_#+&8_+TSG%yGd1{d2n-E5?BU+!OA20L7aZ=$2`E69)R z0^3YMMZC?WkofQRfYho`&0tl@H<#u<TCSHWTgnZhCHnbH0CUHOI;jxlK|bO69Os)x zF(-)ddnK|uECuNO*v@L^%>o1Pd<x7o#@y>qC)MJ7JvwJ8q0KRjVt@K{V;C<XNCnL; z2A}RNZceq|d}MT)g#)5l?+Cgr<p$`1yB`4}pBw|HGn|c@VK=~b&;Xjx&V%Q}!NUl+ z17>g&wtFfGG6py<Ge8L5zj~yW-cQ4Z8T~xixU2MXpZb}zk1GH)ctB{W8_K-WF{ge? zGqRsN_QH;t0iEZCU={^uhTCPs1yvDi^VDS~PeIMgloH+CokTv)CJEsUNm?5wI^J=I zJl&#oL@}h)-d3v8C-DX9mYtIL0$upzKRTfy^ovXhk8qs@xTRmm)gg8tm-7bk)Cnkl z3SGl+;WOc1ewdcXj5ve9au;a7sHUNAy>X5yC7U{!j5PPDOwO&+54?nf-<%xSAehQ( z$7#>>9WX*FLP3Ml%@izsZ1(GFf6H2f4?Ki&G>D?Cj)WrAF`rkERf*&Po5D+F+r^ym zZMwKf0@vAJm3y1D1f>D6Psgn9ZUCTT(Hsa5=k)YGwq7~m+IU7hn(@cQx5yoVKcFYv zrRI7Sz<cfsDpiwO-~)tkTZ>UOMCN6tNey~=YYac(YqsLi-V<eHy8oPpU()B&D<nC4 zwcbaaYw(q<u(NMQ(<5}_#QrQ*zXw8kunto9b^|`(OPdyu8Dr-ZY=o2UN6-S60*}Ub zsE>a=pOc=WoH|r23`_UKRjCEGCjjCa&noVQny?NZ|GAJvapgk?r=vBS&A5qGgE-J+ zPKn%d8WiG?efha61yIcr9<^kP$e;vdXpoXQt$;Xh9O%M-b6CRZSu;*;wE7&)MwEW7 zw!bZY>D(kG#CQ9hK!D$Gd3_GMz*#N<c9n%6UX*rfZ1*kpFL%|4SfZxB%m0KJJ$%;a zMFJYxPyTzWj$T441;Dik>oed$ZHi+U#=?T18AfDlxi&r=9|nvKR6CYM*}3-fyT8f< zh`-Zt3HS7*9UxxO$WHYcFIrNgmZnMssxZGw6EA!&$?;06&pQ0NnDI05G6_b@M&P#M z)NHVeD~zdMbHOr@r(m$iD1iyGZ#kB0N0r^>{cS!}`dd)02VL{wR8G#~+khQH3a6BV z3RrPpO(gIf=QM-JZ`$PTuQ}1#*R>Pp&ROGLrwwHM<jsX$9H>xg+W5in=2++MuAv;$ zm!*Ze1$rvTJ2-Iq&Evh)e-|opTNms+^5_t36~nA3%rO|X1T6QjD?%4PD6-ep?XPCF zXMQO=saMRGzg;9@T={kCjziNCaaSsM(0gZR_Jk+;Pq~S*{K3TW{c|#x^qnngUh*T| zaFEh502fXRL;<ER4NVzv{2i}{+ik`pFktcN1*n3YzwJHFdg;4v;YCWs-)qlZYPpr~ z>wiFL$ahf4^Aw&#XDO9c)-akHo-=%7)y0D2v$tI*Wxc}Koq=<Z-~P@Oq}kJJh11;o zNQHf1{^xBkT3y!Rcg<1ikAp?$8{YzUD3>(iil<#W62X}IAFMx^bUA+nt+zR+MGY?B zB&j`;C&Nj2yVAGZT;ckY$*XrxxN!5%tA=p&-4m8aAQHPZV3=bOs}Y%2zixYIhBq7O z?R?jcRMwaXA&VX#2vd}qNelIkWR;jObu28`b4@<Od9Rs7*QbwkhIR(>vkoY6X4bcg zOE1)8?!tS?@){@U$e-_PUoZu}kf045Qc|kp3A{j0vvoy53p@bSRj%pZ6nME#R$tR$ zo?)8Z+F!`v0|eos?;MhFu^vwG(jr8Bka!KKY8(V$qYFDWT`;GS-Atc-^7Q3L26qp{ zp{%zp?$X#OHcU+k+6-W?NEQ_O09)AGq&JaBHUJ-2!eS4%dnW49Y(tHq!Mg!@ZZuYX z(LG0hAD}GJcnJ4~d3W*z*U0p17geS~*)eh^JgG_uC{o>P!se%|j+d<a06{!{^-|q! zzYWVDohM}@!XZ5xz7y_R*k5(4Og>_5|DNIff2B_Llqbry`#{&s`!%QejrT;s@8vb- z;-9$QW?%p3>l=8ig1slIY0HT>%QAc>edU44_xQ6f;~4K1y|R??o<mW;v0(^WvNIW3 z?|)a2#FR&AGoF2yfwWv$TB&OQbWdmKvB3QV!|}R2bNN~fW9?h>U=qZ|qN79+IBiUC znuL&S>wKF>VZ$+8a`DDUr%!<pV7WeL1UT^IVi<Cld4?RPY{{o*Vc)!iNK1mtURx<U z^FQP;F0)G0r|MZIYu+y@aakkHZoq~P#K>XOD<0_+%Wq7}IcwPpyp@c$7&KZ0>QXcP z!#Y~-T`$V+-t9!CZn6e@#zgcfOTbflsxM9tqL=B0bQ76MrXlO4O1%d#;ZFmu$Yp8A zwn||sLVeTH?^)x452d@Fk;ie@{Yo46x;Wo%1W3|W(K6TksCWDhFk~LnO8Smkc&+kF zBk$~--qA%XxORLOV>dkXj>;c}1Jdbz>2e1v*?`>sX&ZWN^I6Yrv9WPzMCMZKLTy;~ z3ngo_oDTAO)3*O9u5{MYbABx9>&kWCKF8fEnL2N_GePZZ)qOkp@7M<0DHy^PIw(&7 zb5wj~K=XO-rM&ugd3MH{)LfJ=u`qlez$b96)Mm#$WM^j!j68N;w^-(gvzn{)y9y=5 z+fK9*y=d8OmckCNAN`p}36FhbW-9A@i2UW{7fNdMwMK_H%d_<E)kX&df7tyWQ?zu- zWu*si&Akaz)(y=0O=I!?F=W8;#Z!FRSrgxU?ba)XtKVE|oIhxpS@2PK*IHLk1|k-& zq|4NdLZ6Xk$8yAmNm~>3Kho6nOHK}!32Ee~d(@*oX*%;BlaEj<3T^;g6GtJ#dheBc z4GF<C=eY~FIIF%#VS$_f@hH7U^su-P9Z?qcC;FnoGqP#hblQ7#<sKYU>a5I8>9u<p zOElvhm*dM&q%9d$QU4+9v5Fj0wmB)W#xL6W`*d}^`&KA89O}A3ZNfW)b@N7xR(a;P zDP>AajKyY}m>%LGg@sVub=LwSS5iO~K!EP86<(;r;IVE)hYbWc4p_ph?-rn~P;$6u zwL~TuV|{N{(d*~nDoLTaigyvXLeKX1oauPoE3(x?52J97LDp2f5qr+oz<#7Xg*J0! zn&I5adJ4+rZ6G$pY$NM>1^d7hqWV&@ktu;o`&!7kXLF1O?B;$}lviixvxGPgW1{lH zTJ~YUOMTQpZ<){w*xDr9YXH#e*Wi=mU(gB7{=+dOG-0)UnMNU5LdL-5n)-XY{~`>G zd||!i;xBDWJMx95>HRrPgnGk05&gV#?;n<2&aUfpx_HHhOY?6g5c)*FPuXD_qF2%s z*h3@KNkc#X>hGX<8Q7UH21W*fBxV{DOQcSP4pwUc1}OW{s5j43GAdx@3P<4N^|50g zz%V>j!PiySLJxK8J)(x6{lPElQzPjX$YanG0qr~Gam(e&Gj&@gZt4L;%8|PAwF(*i zDB(|HCNiFkFOCz*TVVJ}?{ojX(0U|+`mTUZs#KO|ubg0+HcvIfk&M@lw$v&H3Q)Qa zQl10C;3sEj!gTX)yI~uyC2SQLRorf0@Mx)3Z+rZcX7{4sCLl!no2oiYY3mx<H21#9 zqM+3M*9X0TLD`vWY`H+m+iBkSy|53Pw0G-k#!g1s&5j+e*GmXqEC=S1eFCs7S^QL+ z+1G$Z<=}#k<Mwbti~u7&Emg)PRV0P3f6nglC7C9f8IIJg2?05o9pKb5bW%r>D)X>8 z;cHUvr9d6u@4}dtsW`nG8<O<pJu74S6+u+ZZj*J$G7HX^iVTo<yh9TJG^~KKEl;H2 z#re%fc>#hhf+y?P#sF2eb>pzgYz9|U%$Nc0htszymXK4aa=YKjS&clSgJDc$_COiw zu10*v8i@;$<>Zi4?;7NiWt+i-lKCtKYn*Pf<*iKZI0=q}?s;ac*NrrN19jxRp4wp2 zT!p{vQ}|mePIukE3PVV)X7K!y?nqNW%rwIATlR`s#zhz}zx^F~&F0YiH7tB2d?o*| z_hd4+(`prGsyFrr#gc3tor&zDC_<Z!5iNkb%ItNc@bY0EO%Cnt#2QpL#gf`Sll}VK z?~I5O)8%s|rwbFg5>NIEh`mtDR?_B>MNi0*d{9YHx-))J;aYR#p6M`(nI*e7r6UBd zaKaTHOQq*Y7+yZAbt>JncnZq#C;0Rz{-&@PKyW(mGSE@gMw?Y-dZOdUH=`&#R^3Bm zYSAB-N%%JIaTxeO3_^Ub#E;+%oNXd($FCmxVN~3sH~|{bavNNiiB63ah}#%4+ee&> zRtqjN%e0SXAu>$|lz4tUEU{$h_rN?9!kzb=<^+4;MxCWU;(6Dma3#<`G`BEOQ8~PU zcO(0VMkee;vlmS<Wd7R|o|%T~5Vy0LEj8H2U>YUJ91hnU5s@!e1}Z1zaGm*)XB-%P z0eo+TC;u<;2V@xPhB|l`Ex-J=L4W_0|Kl`Te!cqO384JTe#$Rdd#;n#_~7j=_t^i2 zUyVX5fpd=CC)vr@N59q?g3Bmm5~nL`vRrafdmWtrzTu9i&9o|e6fbSnKQKTNrzOPC zgl~>iWl{0y+!`q?MLhs+6z#AL)3<&C&feE)`+F!a3~74tnXPdT_tOccf9^>aq3Xz9 z{{a>p__~7btZ)OaL_1YOomxiPf0Gk2f!mVLa9ivJ&&#d?o}i&Y<5e8+V?%1uMbpY} zlnyJyrTM4wnG~!PoRpR&PjczkfGO}Pz!aSWU1F8<ym2fB`d##kf0<0ibCC9W>=$!< z&2#i12XCAPzp4TnjRH35lYmxMdFp*oFP&R&PAQg4o$xTj;Y|4*&UkTI_huz8XlbzO zF5oKgF;6!mb%g4c@6y7M{+$owq}~sgM8Do2__>6Z=DgimwKVcRHPcs*Vq^kflmEVn z23QfffSI9G=a|?bJe4sS4eUxYfayQMW)Sm<8M4nu+m=*%|Fx8QNHXY3n&EL+N$mm$ zz+z(M@Aiq|Q93*N%^RxH+*Bp`b5N5_k$LJc(Zp5wYqYeYWix1OGqyzT*s?zeVEyd? zdVK=`9;6MlHGq$s%yEKG6vsu1lvBXnsA;)zChF~p7W9NSfI#VL^mqaezLov+bEoWU zfvX$x9aEMyLf2%^e)Uro=aAx1me7dG%+%m|dDeU*{_-EayH`KUI!UQEhtE<|H(XV{ zAklir$oeMo%2Uga6mWMH+c(5-N^W|~gG<P|g5yRs_Aq@6{qv`sed&71v7xtjsWjPy z9H!$q9iK5E$((6_<;H7B6FyraAB{TcS!!D;9ea}Kv5Grh!{x12tw3s1#9irlyA^F< z)0!xQ9EIv}|1LlG^vQS{39eWfhZiY_a`+cIa*kRI`!%9OG!3ZsOxEMzS=8i5@tK)N ze1Io-n;M0GUN5oW(-usa(7wGlU9Wc>poP^6ca6MjdBMio+u+={*KU=&&JJ#UzH2vD zpS*DZe^2VVS~pK(39tBd!(PhgF=NA;0O&<TO}>0=LP9MWyT9wy?t*5?qy2DsUi+fo zxRBx+Mb<raw{-_e*SLm=Ia90~X*6N`8&^D7Q6-#fTPQHH!HeQ8`utis)7dv4T6yUE zKONnZ{CdDQRak^e4KQiAMX?m%*t@-;bVWUpt65c6^i0@-m=u8zii|%-z?$Yqs`qsk z;NAYA?Lj5&V^yYZMqN4E#p(EBfF{ATuDrId4u{-i>ATnzh^^fJ4sHCUx4V`~#X^on zzT>*HQ?FG~QmbW>6bZnD!*EGlHB;}r;!3fzVAMsM-<u6?$e(En``wO}LCXPcDhG+{ zo|WE6p6v$IFt6tpTVkR!&k(<~H3MIQ(2>jA1D9rZ`f|Gzo?*bU9+cf&joK}4F*yRe zPBetBv|iMED470|^2vO<$YC1p=!VkKG_%BBitdBYIw_lUfGhUj@8s`~%Tbi>^ji*I zZhZxBqGoqy*$Ondvs8pw=p#!h4L~rN=5~&a!Z}Ha=7-G!w6up)9d3cMoJ&{Ger<^h zNlJm4JJ<^JL+vE-j2H228r^S6Ds`u_kn)UD$~nJn9h7{h5)w!MhYK*~nC}w0@s-)o z)W=c&gi!)<1FZ$wit}owpnZ54K)1J(t^!?ilhIHMHr|g)+HPYQ<y*VqK-$I~5O&^{ z2Hqhrw_9i{AFmJhGlC~9AFzy{Y-W4);hU0qNX^P3iG>-lPJQvqqyv{G9p*ana^(h= z_yt-WY%>Prk!qC*FxJo7xa`H-u46%E{B_4^^}Ppr!8-`Y@n7|MyvI&hkkiC$P!o_( zmh=Yng0>P`l<k6!w=)eLJ?1|d>QWhKr+UshjqP}UfgYVqGY8)rcdpPWbBK_g@phOx z0Js>UecgI=9XQ|?Y@mtkTqwLz=5yrz>-~)&r{RwD=NHflKHA49JfeP}?c~_pf2Vhx z$^6mF(tycAj*4<J$B|j#ExcNtZi4=ochWkxCDm%(aL5Mc)8WH;xmMGG55c8tzAnA% z{ekfma~Hm7%C~-NPdpm|RX_bMFY{7Akl6Vskr2CWFn5%)Z9%63**yX?%e*s|ne&W@ zi$Zp^Ejv>^p{2Lk=;<^L7aCtvg2zB{^k23}2xryL9CK%MojBhsnInSs9Hw@a>8UF# zGnThru|74Z&oZ@>YxNb|fl}Y{pLtiUV?ClDMm*EOc0e0y=y_<q&-p%WLMFcAPgz*2 zI6=XKUXZi}LR1&mQl*sUo-F!`oiY4y_;{puwmn|q{08#w`dy1;zF}x0DhcCnfV<E5 zD10qZr#eN~$|bJ)PuXlw?RBz}HFJ+bM=G<?N_9AV?icl<e2RL>JuZJzNSia1<-8FQ zMv}mDBZyLRXZx6qpdH>E<ruAXBP}{!&s!-$RdWr_;2!;_Rln{ero4(dAYt$5CA~l# z=P4o6afSnp=SN?U@LPuycsb^+P}Yom!@8<B!lFM+Bu3QkD*>6xE{W3X0c~gpaJ-XQ z75CC-*Gsl>6c7Q+cZvRz*b8`7dUf!Joy+h0Dh>Sd|9(lI3sllynSOG(DG@#p)7~#; z^%QtRZz$EDHB<6czlr&#Z_M|;TKiG*r8BcrpJ|Rn=+=wXDPuU2mm@tPsi*@ow7)!q zH`YAhD5AdsrmsgHq<mA$%Rye08U@ZHAm<|=W&+*5Y~zX<;Q@GJSBpwHZeeh}2S={< z!ib*ur)r$e31e<e-}0HgS`e6v4skWzGO;bj`^hEqQxoP9aP?b<@7ql~PfqtY*%u>$ zVRdZhC-3ns_->&Eclxp^>E<N<@zR_=HWg7xI1vf5h$tN+Havm3Bq=u?%}eCtA;em{ zI5vq<$w}zSM^`{##M4OR5oPQl$$qI`D*WfyL|HXm*B+V5kUZEqr#4<n-r=o;DrEyp zLsDc6oTN+Twk_{>h>^=jv-NPzy^YD;6MF$6P8u6xl+oO&LE4i=E{R+0<l<5n*olI0 zK27_#_7X<{`nQxoGP+eo@1QH4C~?R&ztB{6$)n?4r@f$H`gWJsplVpY?YJ)7D7o#N zM96(_0Tq=h#?f>gZ$1?ow{d#a&8qXUZ4CTXAl0wTXFI~wBG2Aat;#8zRZ^;=W(^sO z4q;XFS+WPL{7|!UVS!Vl$&jvwp&ut5HQluXX$pVbBjy;dxq37`UW+ZmD@8@)MJaC= zZ3tv`_f5p?H|?eqQWP3JAu~jAGIDRbMk^3K+Yt{2%7|)j?HH=sc;!=|C2W&whROyd zyo;MrNFgfP<`NctTZ6wcvNdJFn(nR&)Ka~rkemAu5{rO*K0rwaMivDqk@0e2neV!X zRp7H{Qwrlwi^qZQN^%mKrAXp;{E8BqRBDjt#+}~OJylwD(x4*}0jlg+T0yY0cN|`H z$*d35A*0a&Qr?}lPdwH9a1|o3OmX62^cJqCjYH+eW3=pSqz1FkQUWigD<N*U$Y>xx zIFKC(*3*yrC%qz|IjRTv5d5k%kZ3yEsyx*+|1RBU`Zu2cmWI=kW5}ngifHfyf?W`8 zZ{_~?AHPx>gRX?CbZRnPU<|^2R-lfjdbV>1ev>bSook`ae`Q0(hGXgh6^q?rF^NnX zZsJL2s+;r~XAH4^)p3?DhnXCdiA>6W){c5<JObZJ$b1;%*D6qT)L6)CynoBE6<NKX z(d7y1qik}28EvVL&Oc*?X?}?96%78)Z?>Dmn{U=n@)};_>?bKy-}jvicJwDDFx@v( z@e=d7xn^QL%)lQ|nS0yo>V7_C_gkf9%PXhfokz12T}mn>*&pqxo5Pm`k^5b8SMibk zBYlIRk&|5K009Y=*>X5((wHYjxyZWo<4UJ_F~-a~7JUa(=ZDQBi5(a&1Ti1E$iV{p zL36UZBMMNB+J8D8!%&Vu(=Kw(#6B4HfC+c|z>S+g`V-LqI0iB?af|-hMo%MV;C1J7 zRLUro6dxZ)d0d9G)>v=!7AAB#aa`*t#<>WOA_9MM&aF^yfyTj`C+ifB#;V*luk+zd ztqw9DyIW_dt7~x`t^~U8Ib8K((~dC&oAvJ`W^VAkb@i<%u3+=EKNCw7YBnenMYpU& zDJ8n3=N3F6&FzECUcJ0SBX>Prg-@XE=Gte`2>~W^H|6k7hvxk~3O;dMA^RciMcxgz zX?+c^$N0A24beJa<2;*oZSP}|x5wPPir<$cn1se+kMQtO^=>)l&c@=c7{#R;nqO5l zP4iGjH2jGu%cs&4vCTh}kdhu>xZtAz3q`&)O_1>!$X-Lu=UgD8wrK}D>YgoCs);AX z3^b)hDTO?o+3Gn(2suJJc332X7u|Mlfg%6K+-z1ez?n!lC^41$l?;%ZCE@d;7WaW7 z+~aroioxILJ<YhML9BHtTt8N@d-I6@g3{p#Q2l8npxv1x6Ea4}=l-dfe@=7PyyFA5 zmt}+E5=}NJW=Nj@Hy(zwcA>F#J6o9P&<-hiC`b*PbS`dQUkFM1G-WMv__Nd^Zt-o% zmO6S7u${<k$&4G&hc$A&7#Pdw4#;w65;qRDcMSl%PqH?PE0xp(IQI2Cqu5M|t>UA{ z1DVHCja#2exzeW;f1|C!xD&C*d)4^>Ivb>Y=coKZGRH<pD@X94l-jb;ry-2thl?&p zd-H<8ZAF<0$KeiG^hjjd2)e%1WyKL|uv|2hG4?x1AYd4mm1(c--a$@o4`~ZRoOm`G zkZrj!iofyBQ9ZMjaoLF%X9j0e)i&XOUuZ&p$nUi+NW+e^5&|9Fi<S;{ya}bIgvn?o zz>7o9iQZFn$zeg*ZNqwK?{NoO*Ihc(<-@PSK>ut|=F@Pcx^>txF`Z|8M&q``*(Y1$ zLN7AM-QJeP_Uz{#2fWBsr^6)Os01Cvx=ToZC-HJuEBgA6@Ab0Z8#e+=x$$j?(=MKE zs9qMti;*TdP8J*558b#Aa}d5_Q%{GFX8V{aEONOKNp>51jS_*eE7H)_Y)_uymIUSV zyna%IZBBXDVrz3!bdc5&zKpkLG7yXWJvw`aH-QF|ZuBZfY1glvx7;8e>O6PG$ZLIL zZVi>5)1!I>=~)hd4EqOSukP9MR5YAX!Ol8LC%EjjG{IjMR-cTnrwcp|!7!)>91+$$ z9UEuM0xoa3uU`a|;kQcxVXYyaE9ce@--BWgggd^R$?yvVT0vL2D{?1MJV$MU|8xMB zJM>3;)xdRSW}pXY0o;_b(36(52i%}}0d)L4rM(gBowm2s|C2dSd+Rrpuao@y?1T^E zI1AJY`b7A6`TDk5j@7Dw5H&v0So!<A=k!B4LVjEEh@SnpwvZp@qzB0>$9j~TF}``) z?{{O`8jBOpK!PmR8I(JdNG~W_>MBoSPWEHG0MVF-2`lm1;`dLz4DReFL*1xBjUWL^ zMNicSEO~5PNe~PuJhf({s`OhL@zkf0C;edq0B-N(@RGk8EoUc0$0%-f?0n2uyw2_Q zN7ZqqxElM!H&fH#Ft-J<RT~K=#4;j$D%Q>hPFxcF$m~Q;iP|W;ZtwF;_qg8f_~S<P zUM_`c04pSMGY1*k?h-V8SIo|7yF{9}4f_~`aS+cmjGTqLrTb4fi@g8v&k!O?J;!w| z^V6`dCQo7S@AK9VRBCK5-k^WRx8dY-zFcmc;o75dY|s9vD?!tVC-olp;u6w^mNK(i z)~b{0#Dn}tUsjBB309A|_|4{l+UID4GHn?n{x1{EHtLXrDA1rHQ}L~5qOq-1Emf!C z8MBa?xKZ{@*$Xu-*FEMlynpuv+UYgzLM%)jbp$HI-PwziQ}30#cZevz*V_}cF=x&+ z$Cq#3EuDZND&i)}m#SLEv`iA)GxG_#2V=7eUXyS3L2K3Ncb0q$j~8p8oBgaiZ^0e@ zlO~Bt0*s1fVf(yhg55~Uy7lssgNC~9WWmX$U$hvx3aRyC_~o{S9J8PNM91P6g3!hH z@Y>@)duixrqj*etYCsf9{yYm9m-7g*Hzu|w$bD1p9#E;|a_%CMh?f)h9jViy3jwvh z9b2o~ZPKJe4N%A6{o~JI<vB4j<E3j|VW-*f-=z2-KPt>ZWvO9DN~ZF=_YpygU9=Bo zOD&&YK24s@HllxM`2<)T41O`R_;+{uEcpWQwslomDOp9RM-ZCWBF(7sU>5^=EE+4M zw^<qD1s@B>E8y+}oZB5KVaP$JULH(A`RXT-XW^RpMp~p31+PhEyxK>S@wQ7z&eJNI zAg9V9Q+1h+=R=92Q4wiiYyy_}O>|&bJkX8Q5^SDzMP9$8mUu+H^j#1F;r8g@cl`C? z(y5Q>gG$J!Ow?vBa82I^BcW%}X}*eUd%t3<P$jZMt!@^aV|%qeJTEVG!fyNiA&70M z>_yOg`6LhZ!uupNYFCXbCi(3oc&{OAAQGEpKU)Gg0?au9N`41TcUY!nX<zy>k1t40 zSE))v#gy}(k3#5?f7zQr3eBZ2lcy>UvO?B7kVGi0|NHzliH<z~WodeiN8H6S_9l9t zva8|}So#<{O_kAkFIk&4$YHLLc+{|Sfp}E8LH<kQU1IvZ>uXvrZs+T55|D&iUh|h} zhm&<zl&h71j&o~nfsHSEC|IaLWc!^o;i|i~i$K0>_ed_TxK?XHN&Dpd<vi9y#Ad_~ z{9-!JY1O}tsd{n-oKmkOIWaQ`bz8sNJkADB@(BoRU)v~Yticrt#t3Yi2@{@g`$X*W zgoPEabtCKY8_o>e^wq^S{2c$1*;s~$_c5k!oBU`;BuV&rqfsuqGgP(f-`jXIeHA77 z$e(*uJ8JG)=U)#u-olr=BpHO=3Y+ibfntj%1(_Kju2$7RM^X}H7IR`k;&REvV9Yb2 z_yClz!(bSL&bg4AV6B9gJGYWDkJOt6)Tj3Uvp)YvC}ycNo#GGsn&E*oYH0aufr`=j zDp94`R!0BUD;3_5bX7jF4uJr|+wjJDk3%ut@_D`lwrd}obJ#+D@A)hhQSUz>=QkOt z)N!;Dw=+)^x&5X>2Pnzp6Vm2RH}_s9CeeV(+zNwz0scg!j5K#J`Vf+?xI&R6J!bqO zEYqvtTfM}%`%BUtizab1pZV$mgX-gJlR*PxUX$L#xd58sDvSc?)s=wdPTqB9*mL9W zD-cI~*jgHbIHEN0dR?J3_0gfOJu-b_Y+jo|WArNF`O$S3@q>pZ7Q@~gXRAsXUASQF zt`X^io?+7eWXT&b_cI!Upy#(o&5IdLl||~0dwV4AY!q=e)LM`p<-qpPvQ!<pcp(`2 z6^&QHjXyP~5E_svE!!>$HGb8H_!ehGD$Z(Zz1C;lAkl`3#6}H``q_9YJV_h_;4`$Q z&#12+w^*f?%6GQ3bR_3~{X2Qe68#M9o=M1X@eALlWwS%kD+qGIWMpQk){H@w+grVF zij-xKVP}5C*HZOGFK_MezBjvu_gHJF(mpZSRc}MlE(py-&Rlh-<26Ah9B+Tr0=>BA zTFa{_`Fq>8@S=(sDN9b2h#MfXt#FqFWBTQZ{6Gkj#f!bgP*6$NegCAOs;Za;ol#oR zqnOCazz@Pb^*d!%V#tb*TEm@UdKY%*bTxcy+cb5}+$3olK(Hlq)%xSR>jTC`?6F8S z$F(dVze+e{q_oFLyFo#-Pax_$vo7(IJPvgq0d<LCsB)9U-rPZdpbN}hUHfIdPt#NR z=65xtkKs3HRBnX!o{l_}3(R!Bz>@XhLE4_qyPOM(V~qbX!2kOZ{{<^mPxB*J17P*+ z<wO-sVba__yHfxC=HD;DfZmIuLjnSi;=Fow+=&mGM8<m5qS$VIW54!$<0(y|^sVQN zTLo|lZ;hRSkK{>0P!h0Emg#|`m3J-r`C|f(K=QMhV|1wUs}dn?dOCY*IcyHa+P>yt z7+?6518=KggoMlCAUl)F0!~ccL&?u<g>dhee%axS!AclJ)_|(l$s=}mp6Y2nCTJAJ zh7lVz6Dce%71fp6)UVHD%NYdJ=9hd$Yd#<vM8}Dw(m{3|vnM}O^E^trT`I$2%z?jW zZ$d41`O&HzJ;rk}rtPwn9cE~~8EnH)7kK$EHfSkD|4|o-59pN=6o_fta`+_rNdC{8 zqY<L&XL<mst)c^v+Epz+`4u~d^2T3u>G!z9U5x@*XEeZ7CgAdJSxxDUsu)nX*bHKM zBjnokCZIK}CX7z3Pd-JHW9G!9*3XCQEHLR|XGj4@VL#~ZVRxhD)&qbALxW;GIJOL( zb)Vrk|2EHz^3EJoqke)JO_!cKVau%`=1g2b8K8(j9Nm<2gKaV0w%?STg2R9TTVi8E zgz!j7Y@26U8Qq`<pR(i_{Y>WbltCvABBb3qb`z0`b-KOtbpDzPdd&AC7sWj<k)u`8 zzltd`-2};5RLr~Q>{8vvf1Z-(b=#@1|Lr^J>DaD3{C-T3_xak!kM<|Z+Vu(Ncm*U4 zQ%gIBrCgVCu7K~*@h0d{@ImeSqF1iHdC1JT<hG*{2g!jv;aD|iW&Xd%mfgeC*lPP? z0KUe{)#n)#5;L^*>+Rn%{hNdO0cfiq&hyFquteICa98;J8waw33g$_M&^-)b#~kvg zcI}+GqvdxF2ZKX%r|?C4;d($LBoDBn({()Vl<@cVC*a6EADbMdrl^a^^R{;u?*$M1 zaN|;}+&5m129lnb$p9uhx541mPU5BoWeRl6zs<$s<R+Z3H7_)Ck7~u|Ap5C-KJM1^ zta0QM3*(k{BX&_KiSg>hq~j^rS2p6)s!M?rR~I~S#oqG}{ovs5Ewoc6<2QRacfe5# z{r0mod;o5fctx<}@5{3EVExnFew1=^_olg`B<lDR`VomqqR%y3e3xzVoUV5^r|o$@ zD(VLw;|7=cp`(;DIdzqsE!q%`9d$n0^?*{c!>S({M~QH;RajbluRO;pJNPdU`JyzJ z%!7+NHV4IYWzQxqZ!9p{pgIp2DA0X!v=pyu(6RcmeG@HKvd|-B@^)NVw>8>Hp}-^> z2#%InEale61!weSr?`s!k+KleCtyPQFXWU;p3im2(uR9?Ll_Bt$8D>&U7shz^DFve zlcEa%(`2}rv;<-ye%?td8YJ$}7}z?oRf721dp&kVY>m`i2<vqes_7jkI^6Iq^j;Tj zX@8ne^WxLCCT^+C%sO?&-m#im%FI~exw*UmUepEaH1AmwIzQ-vDD~*ovM$~0r$2Ce z3TVI$h%>zNnAFG!NxvjS{+XkACFOq)8vl`^{skWEP6yp~QTYdM#p8d+T=VYC$IACO zB!2x!{fk1_l~S4s0vPA!(vvf9WA3zS>_YW;bD>HsIjgU$nV2;Us7fmHlNKNwD@*nf zK@)709Zt5V-M`n+Geey?!bUzstP!7K?|NWT9Beig0BSxb|0IKFe`LV!J~5C~kxRJH zKlrgYG$V0j=~WC;-a~n-F44LA;OTs76+#kxb3d8%!>X&1ZDk8Z9CS4g=*fpCe(Z8u zR-Xa_%yA}})Y@KhiRZ3|IVPdnqB{WT@O|&s9k-3zUQgq))z>>Y7`G?N9Qdb6Ltzcd zNYlst7A4@D(03Dij}O=6bC(~tn7c2#yS({D{QG_$mQZe8DN<a_J|h_81W7ghWRLM! zV|b{@$3UPfm&{mJ1=|ap!h&^I{sJu=7?~lTyNqGYu~Cipcpu@i01_#@Fz9SZGI_Mi zr-1prnHx;>P4n#K(B>~e2DtgHdsrumE&f^Ey_V%*slC{Vu-2%ZMz*$CIo360$}{eZ znOosQzGC=?1>FT!4ExC|ZlSSr>2mbrxyQzbjeL5J$Hd=hVjhXrgyIFolDGDg*YMm0 z=w)R>;OL8q>{|V{kSc;08UNnQYXZQ%ZF*I#R7$QmPE-3e!c{OLT>WMMhBNMSg$m)F zS$&clfNY>C7!AedyIa;_3~;7#O%>5FqkOiyb>l+VmXAdmGJR2LSFt)}_%b5N%!aW+ z@KyjJ;g6!kT3$^>q!~h@A>XXKW}r1{0xY!*&$&p)m#}d(&98JW?}SoKJa>5C<o_}_ z03Y+8VgTn><mt-_jj_%d_uCgK`lmCw!RkQAR~-^FjH|t3Qx1v++O*w}*g0Nijl9qX zsr;m^{f=RKmWPGBw12)rqV^>G`y%>nlgwnUEBXR5_B*)}aK{a>&WWE@Nh2*^_n%7Y zpjp9B#^ZBU+J7o&?DjqlE#}o~aw;H3*1GPS@p)!eA>0)n?vGGi2<W=U7$^VWbJ-2e zx*4gHMz@<oFH8p@j+WFuwXTSUq1=GudKvq~HW2z0IB|M9AXS~x4bu}%tlBosLh=vC zk%uWEdGf`unxUmpL1o7skO!S&PUG2A6l(Q;xZXb>L-U--BdTXmK><Biw0-sb)-V1G zBKg>183Kk?O(t)9Z^>Ttp@wbj-sAnt6vE{t;bg8?+pZ!2i08Uc?P`-$Hc|jD7%t$Y z<a-hz)Ec(O9R+Xo1qRxQlN`#+w!K>m#fh~)>Az=}y`D2$4_Ha=KGs62xAS<!=d!<B z`T%FWr?z0KH|}xt+?xHpl1M~@mr5HixdhpC{S7{)qLEJ@mtMFQP(lTuVr-0&gSHC4 z8s@KQ$yt~jnk_fkuYUGKyy-6@tEfJJ>?y(8(}?YKki8;lla_YuCVr^VD`u8DU%Pg= zE=XlqDfgn8^s&bJz@u1J?(N%QTbKLinCUkCw|o%Q+Hw^>ysskRTPCgpP+tt%lkyJ1 z1o2u*64!;tN=N)=KU?Esw4JmU-`}JkwRNWq<gqkurP7geF4sZfzjh`6)6D`G+WN6t z1nDIz%BcCyb_(7Yte%zX-w*uGHx*IPC<CwOqGRXL$Fnz(Az!{+3Uk{Qu4H4hzW~9g zn5PbC!%iCbk_``sn~ryv8n%1WZfxcznWqLdIWQ+M*X|jB{;Yt}?>lq-y4F1Fh1tTW z`J9?{T~<E}W_l<o<_mfHJ<4H6f*<Zb&nb?ipnGlXp^L%{xLO$Lz@_(V7GfaHW&;r7 zb+U60@ism{7yD`ZlQi+;jPgk_O@HFfaD&w6`Q+;+wk3ZS#Fvyynd2rTrmdmpHp>pP zUu)9x4KyAAe!g*wCr1~Fvj}Gn+kb!#3F-%O<_pCB&W_Ek_nTYRnbKQwUBKmPRfFu8 zDp)l}oB8a|l^xe`v^O8;tk$Rx!=YS9-soBLt?M(&sLoNUfzNzbmzDbk+#;)%!$9Gq z%xBL6(L^m-<CDs`d3#SWZ4<^Vq&RtXU3j6B88M=@Wq@fpoT2oZcybs{#!3o$=4tuL z?J0WmS<{AE*_><>Fq>f`dM*n!+Yas4VL^!dXUpEM^#^q~w6u#-PQx(HT$DLgOV&}G zBCmTDI?@fhP7?QpTTIzbkl!0Gg*%iftlY35S@r3Xu$;;r4bvGFCUd6O)*oMvg}zI_ zw!HS*kk~K()+5FjtKvbpwZOe5PgAQ0y`(V8j*+yevme|`!$+=q67~tNf}zCNjuE`e z6X1VYgDpk+qSUq(iE<Vzj~i3>;~<(XF5TpJff#9~69efo&}}*i@M8aK9AFhHsH-M! z+xGuVS6Qd&s_u^v#Z;x-hroE{+>4x=_U@HjrxQMa6knvT1jU}eEcDv&3=3?^nkfO( zTpEu+=s*9)BXQ1vs;3g<xE{fNl(%KRrYE-zU&=)L%}U9|z?SYiqH_!jLe)z5246Dl zkI)xyG9{^K7ZV=XLfx|{6r_jeATLM$Jkgb(4ERvKn@a(oLeVS}x^of7+xy#d+&n)K zvNuPI*0c=MF)w1B*CX6}mO48|dLTQfmsxjvS3jH_5ji?cKRnaHPSwWG=&Y=i)DRn_ zRF;$UASDzlVt6w|(-s=DDbRvfHiK}S0vCw)ms~5)|3xkwScQ~=KXLBOZEyPVC~#Qt z7eC<b=TY18blG+-)<M+{Q&l`7pB)T!jdbg)vQu|WipKVgJmsa2>y$4#R@a-)-s_9% z91S-AZ1oW&NWX~;H`@vY2c5^kWM;P|-?TyE!N!P*)geTT8I)U0@B2=q0^u}e1tzUb zZHCP$K>@tt#<=(_n*M4Phk{KEa*b`<pipi=TmanqKU{!iV*GJpS^KO3!<JVu=S|S_ z5`oOQdrkU>JUAGIrri3?x=&IYZJ^9M#ka1cR`2__-|_k5-aR~qg=PL6K<salqVSB+ zUKdFV0iv+bxek5}4lrw&0iz$&=S;Erz5vA(;!T(CvY1GUFV2pQxn?I@KE>8FW)h#v z47(aJipiz<2N;Fp!^-tEhV$f_;xDhgQHymJWpn0)z@eG>=Ygm$3FH3X<NfXCnMpf^ zNDF^NEpQ1_VmlZ~hG$dQJb8~*dw9Z^tW3d=e!i-q{<r`5KOv3^jZl<+?6zixQU6fc z%DXea@45M7;*`rqT~<xmXnHPC3laMEKj;bFDme|pJ2jf{%4QWY)pCygXZ<)&dT2@5 zPk&=jIZdij)@@*E;-u#iw5*MLK!6<#<lnE`Z_yqwk*HBrvPvp0n1%qCPz++-y_N4` zOj_Sw_wfr@A#8-6e0k6Q0|=YX%+W(36=oEUePY9*=@TL=-{-nf<zqo!zl_CbzOA8T zD&=CKcnZTtSmB()O5o5Z+AW+k>SNjRCMQx$-fu_rKyv(Ziuyrk%bX`*m^;rNAXcKw zt}i}ypq?WqVaic|hpb5imR}2m8u`DO{=WX_omX^0w@Vgm=I*a-n<A^B)_Fb`z3+p8 z0kVWS^j2TgCMIgo^fem`SGgqn<U>8U>QR{5%R$J6fQANkhmB<%Wdl#x$aO|2XI=eP zuA#ye-<|qPT`K&t*Va3+27=&#*+i(ru0@1<M637(@SvM(leDYHO@U*)ZSk?X<z)!d z`Liq;V3xHnCq-r5v)BT_;r`Q3^$4zuh=mP!ey|cI#`yQVnS*FhNx5ZhhV{Y}T^qI` z9v?(`4;M#}DA^Faw%+)naYD6Q=M@<67t{U$qPyH_YK<zDa8RSFYnC+IULZi~za)2z zB#L(gj#KhzTI=;vV2*GzXQojT<9J$)Cijq-#_dZNJag^24%|{DPisv={rsICcUf}s zPMRMmMV_C3t^pRyBfg;LoV@ODb0PTuF*f!C;`BkhJ<?agz+Zc0ct6Qh-v@eYt}(O> zS}V9sm4ZncINY4IG#O!`x1oLnR8A`Xjhmxb${X`jmGi9nOe=9>9G8n|V2-Mlt6&Xf zx6yM~u)%4#+$6;yBX^bzfL6N%I1v;x-I54dc-v;u$`GT#gmQ%A<&XwcK2GCFdlI9% z?Db~7bGLFaG|Ua-wCzsKqvD5h5ASc}E(~0aR$S|HN6+eA6fi3rC;FCX^)!t#!(mSs zxV3vf$z=v<P3h~wN@&sQV|6w~7^$AMP7krr06-vA7lW%kpgzOlU=&?A!CQD@bME;x zzNAbc0pyYHU({LduJaELSQzqqyD}*fNP0@PI}g)?+idUP<_aLj-ShG@AGfjT>n9zw z$rLKfceVM{D$1UU40~7~0ib0n*J}iaP82`LGlH=#{KQx-GF7MNIzF=N<ifRWmDplu zVkBoP0jQ#pLzJyh@2!e<-^I=zG0!ldX=1r@q5j^)+{kwNQP;K>!=Wdf?d-Wws~U1g zr%y58&GPn(0Z&VYMvx9@yh*E%2$cH!oT(lJSUho2-8-yhDkUhobj!UCswS041r@_p z7PxJS4VIgR_SigSfczQB8}vb<r#_K+tDnvbl?bnH6E9Gpy-7E<i!fbJA^xeE-6o5r zxq6OAw^<azBw|c}(%Gn|K5ZqnSGQh^JSe1YU3nXGVEj`aU;Vuvz`%SW83NPwAb<?< zI{<PgB-w4@x^8-o6oF#D{p2_+C&Oy2Ch*T#{(p{<$=dX-!X0<;Yw?)MO&!XqO)l$& zzzbqM+|&=!^c;(R39?=WQ?r0z@zd6>_0-w%&51NLg{tOAg6(g<{6|-9VHWk~LIAn* z@JoaQss{YK5j6I7diKKNx802VBAS~+ReDUDt*%J|+AKkgLOad%aB`t48Lrx4eV^gS z)qEd&6UDWKax-H-xqBCtkx(g?nlnCPa!Kvts`-cnU!~@khDUzO@f;YbwXIss74iB; znMZI%_IX?}<ckM(0&UGl)KhS^HA3V+Aa?x(wE*i*cawv6N`F&wrJ)Lzv#-MN_$c4! zwnsojc3a9L6%w4NJSkwdSEg;IBfxiFx&v%)0^Su+p>#8;_*Qi-dT*yt+^RUu38u7| z=TpEy&G08`ZH)2Aj0Cg%5dUs0YNz%lrR1mO?^Q9;oFbX+#o)d3IOHGZ(5XK)ZdSpf zA()E_i#2z*fV2c#<wGS|w-@kk^-~*R6!spSxPR`H$5$o$vYhajg+&7@|FR7q00drg z@T&c0nKd`qV*+MQ;=}6}GKSN1$Rmf+V<I`F2uLCHFgMRh06|)lTz*-~zKYVXamBWD z{fOuFLfx-zwi~bVi)UqGc%^~`wh?FWNV&_%D;X<&BMC(Fv7jUmX>!kOFe9+7(>GP_ z_K!GBTh8;pIbXj%v_u%&C-xc^2ljdoBa9({O18Y)M}$dqH-$_E0GsB2<eM95Sn-eM zc>6Ej@ZUccKA&Ru8yQ&d9~Xjy`1NKlqn`Dt3ElE#{pzBUq3}RMuicD7B4jm`?mokr zkoox~JjcdJu98!Lt0C3vA(e_TVmUL{nTl6_;d|w!yhl^C15k^@vFz3)Q;+*&!16Kz zXLQ&tx|#mG2kxRIsqdrm3XzC<WQ0ZKIWyiI0(I3gmn-S4V%nIasMdtL5R&<!F^zJ3 z(89Mu0Al+kA33@DTHFG9G?JThg9bW0h(hmemqB;0#8*F9tL~y2lj~qOuHve=9aI^V z1KBFqUdZ2h&4{W^I(!^3Q}f&6z;~^txcG`4rAoAZOgsD%*WO<;Qiv#DCb=WY2oT_d zDwarp#9Tlm1y-H8YIortU(;t5x-sAK76$%oLD>BNqU+t`neM~?@i~heiYTW|B|4yk zLvl``2&G%jD&#OM$C;T^iEf9HBBvpQoN}1soKhxdb3RTDbDCk!jNe=L{rP-9r{7;5 zdl(*@Yp>UJJx^EFoyHIoVG=gEWXANiqM3yvfxKyv@^rM#r}XCyx9vRhL1Xrv;nnS; z>;P*B=q*x~zmX)^DcXARo#WCf&VEBZ?5e7JSPQ0O&twlloOggCWREx7vTcAp*5o;i zAFB6^_dV|_-!05)$mUkFo_E+aZKAroa+v&sYpqUO$?gx!Q!QLI<TTRZ^NNO(fGpPc zuy7RHnvCOVP5N<tIcvh25}Pu1AHJZ7QUOW+R`)9WHe_x+_A|h1+36Y;z(hNn=1}JO z!{T%m+xp8}&UnbsPvKe=bRr3C*Hzh3?VlRlh@adOtv9K_ePxcSCfw3OI#&x465KBY zR?b3pmOM_&E)<T8Z_o=EWAfro-j?wmr@idG4A|fPZ*=s+-6M3g3PbpYcNbcGN$16O zyRtB9V8#y8vL=xrJ05-Aq?dDVAs2Q{4X(K`FjRa+760fnjFaBMu_DYs1j|6yowa-E z=DZPOr2wa5lxVW|NIig-Ow0;X-~aW)NPV%OjL@qCR;4I2YtT|mqcAmX$`H-&Bv@~$ z(ARQSDF@*ikgn^}O1+n6EUE+uByN6<mLF}K>NL0bHS$QrbaC#Myb`pZti!BUG$Tnl zJ!LyIy7g$Xv_%?5yhU!kIpyu7@gncxgTB#^*ae-9p9(bUXpKi2EhSU>NgIHY<{$bV zq_4$qLaQByf%!Nax^}1Ip1JIs!hbtR<5N8y=T<EpH+8rcNTajFq!6i?=M(SSD4Jv1 zdj=Jd9Lda_p|wqH^3U3Jtu|!aJ5hGtZ|z8hnanA9jz{GS%JWkU!Mxi8wPzamkzzz_ zB=So;UxVZF>5aVkNnFHmDjn?{7A<PGq<6aI#-{lw)d|e7Wj|Tbs4;RlIjn`Ri&Td$ z|LT#CVnT0Ps_RY0u9OV`Y|`$;g0fdvyMixSOlk9qQ2AHULM9YIw8ZqDL}nw#j3<+~ zPR!x>w@k)PfF4ByPv>0mBXbomU$^P=&Aj$>0FCFlzLma(tafcf3OtBOa4r8kwID;@ zp?wdo*O2GZg_!nSYhe5B3b0GFmH{vW+b(6`d!iIvGj<T8l0pwnS+g=HI~qzS)US07 zIF3I=R6mgd+2uKti1NH^fH~)+(%Jve3H`5Ld`11}ertZqdJNom!C|*G4$Y({7Jcv# zDiw1|Ls9(AyNpJCsh*A7F=`s0S=YTrI>n`G-UL*J=!(>1G3mOl6<<?F2ZG>OXgNe^ z$T;MwIh~iX2WqLr&G)a*a1zx!XD&Y}(g(sV2=5y_DcC8`qDE3BATSU-dwx%1>B}lo z+UNd;b4#j%Pwm25O6$Vv#MNK6Q5B_!{^J0N(XA@rdwuL7<7avLT#Yv0E8qsCoKt$@ z9B-49-igQ_*9<I+Czn7pht0(<J7po9G=?~tR7$iqX$sH)N(AcCVoC7give&jdPk&( zB?8)bt=qf+>Uv66dMo$Lzo3I5i=3w}#cq0a$;{OLFOU+;NX;3u(b);!v_;KP7|Sb2 z^1MaR<E&7)vC=0n@Q=3j$lp}m6$d?!wqnk5JXLEKVXqZXAsVXN%l8zprk4;eE#Kqm z-qsmSShd{%-*U@EeGj1i#7cXp4fJ{}nWEIWf&Hj~%%6V)dR|rCr3_F`Iq@CRcZ~ft znWE12t{n83=+~J9&y}(3(jtq=GRKeI9dy1<^<9U_lV=W0pI6?UyhG-KjaT}P9J}t2 zu9UWwEVup*KdF|kRK~I|jfdO{D4;1|yPrKKJX&O%{%$YlW4DC#P)K7cC~2gw>Ush! z7dAmDKb|jUnB)yLJfQe({;no!j_N!&=qi&NEMF-Rk$fDlwKwQs+?v$jRn-Y>;KE&y zlT}<a(CFgWNF9A&e2l#sp>p>hV&;Dic6>z|M7meXaXYxo{5{$Gr1Wb8x6N7oU50Ml zf6bW7$N<UY(B4TjYUF?*pMD$Xq5iC^R#8WkvIcx8hdxur<}*;`ViH*IkmxHRrsMGv zG~-WkIappZuBVo3ducfzy8WJyR=Jb-e|9U8Q@rLf!T|LBGUx$RQ9flO8<=kGmVCu; zf597@@Yy6mr>pu}+@4IXYQzeGj4Fl{edI`AWMSrB?HiCOTtPEmnDus*;pGR*^U4zD zi);X;Ys1y@P8{~Zf9Mj=bPW{ZdT~%r5{7Isv*~CD4yz=ker2*(r^VoQf^6*9_X+m3 zyZOQR0%!^0MU-#Fv%jtt{TWc6;!a7;_#M1^`>aEjRH$nznRe*3$!4y2PVYmV@NZ95 zY$sm6`0Ul=qY5K$`kNF@kcuD99grx<ry%~PcWuMgw?bm18g*PVuNp@TODL;ASSyKq zNKEIp|CP)V^mi2J$75%<%-E=T^Wy>CRd;CuLR;a~QlHkm)z)F1r>B(wE34R{_@*Zj zIXXK7=y#VjTDUo0p53fmQN>bGPHf%PkI|>pW?Ye|uxOOrmN-b^B%I*ZJ*-Jq;lCrY zfeW7NrZ1ZTzK%|l8Vm78lPz<{E|(8m+ev()>X_3jx2IaGTuTP0OQ?x3ga~35lR|T< zUj+=Jh2BslhPSH44SL-oDxR_gwVIZ=MLQu`lVLS{(w17bSA&CS<(d-tW@>;wJsAtS z?rp#%j#s}rDE6@BACBUGmXha#BaDFb8iJC0uh8(}K6<SAEQ*KJX7qH8KfKTFLbrow z?{m(e$VR)hX=_g1I{}&?6SMlo^WS9-J%gZh(ObMltlLf+a&A0~Au|=ao}Ax@*r1ON zOQVUqgO!A9c#$%e+iI)d`T77u*ViBZ=iPAjN(F9F*muh~?n0^x=o4$qfMwj?)Y}Qd z&DN_ngPKsLutxdDbJz)apYk5)(9OZoartHEMyGZkPb-OJEPamfkgov?-QF(VNUpD( zwVcjA_NaCr8KDxh_~pm#ize~`aWiuEyMxlD{+-3y!;-{UD-&=VV(3c)bGS66y%cqD zX+jH0O{leI73h|m{k?E&_XvwCGh<MK_lR8$c`$#t?xa-N@j}$Wf_T6CuU&%(O~udW zl6UVR`R@ROeYdvD8Ch&+A8K{r`KxQ6z#L#>Vo%gh_MmoP(U-GZFZ>SjyN{aNoCB2T zseiUU&M4^AYv>2XT-m{jyyJ`|*j5W1cRnx(+Oad+VU_Wsy97oxAKpFHBRupzJ<Vy) zetD%)e-1a`+_gSzKqs#{<Yk6Z?f2rxCB`a;yN1MDQzEvb$EH$|=b2yhriW5+s0eBn zv96jg^d$n3Pj5{L?Mj?$H|SBy{ID#*{?&^*g_wV%NSxW+xs|Z6)?rpE(_K?IgVUlw zpXrxd$@%y$A!*uxzM(T!<w@KNex}h<0+n!JZlFEo%6ven>Fy33T2E8A`I@3KzHneL zt5T}Nr%sTZtWJW?)jh$yCVwmw(&#eCihI*~1_kaM<3?T%m;GZ3`LFJTe?~sDPXHJi zsZ<Kh^vpA%zHsjNCMC+zVZ~w>5OM7$-1b=tvWO){smGS(Ew68UIqi0?8NB;qLG=~3 zs(6=#lMI&FgCU6Q<;QR4$Se(ePrDx-y4CN@-hr&G&9wIsvh^xo`{!5XId$Pr3&~0l zsaRi{jpu|ssFx|)`S)#4u9LOdmNTJem6)yjE?15z`dpE$Yzr{;irej_#&2K#@rebq z6rnEtdLlbhJ8V=QyMMv60!t-vgQ3qgzY*mh+`BPvQCM6%DA;<h4H&?{=c?0ozth<0 zMW|o~t9BcIBifVv&QtCSt2c|ink)X0pQ9HcHrFM2(sNTI$GhI;I*FVb266{-J`QZ0 zJn3b7OFjhDD79}nhO?wSm^{7Aa4pv@3*}|33`+~B*wl1es)*M?b+;pfSz${VD_6d# z3MotBA}WZCub~W057p%$K^;s|9dZkTf-m<}>094Gj^lRfzJ@*RGqDIJ#43ofdFAvS zgBy47YSacvA9F1A=T0tVnyHvxIAvSf#0PZZTV09%OMqsJM67kYJu7;i{9!HHd&!p+ zz4#_1yl|F_>-z;V@B!nK9)wm?F~hFe$Tw(J1<0bkU7-Y7rMpf$V)fOe?1!h1qrg2r zy~CPgRP#M&n&8lpqoz~Jzrz*-xM4(@4?ZLzW<|5;CTisRb1%nFQq7=LORFTRYIWxN zl`?9mAJ##wc{{)W)Oy}hqqUhmE%(O%N!L_d;?02o@h_?yrkIvrQ0e=?JQVbV!Ox`X z`H#P)C4c)azV*mxyvpCh*m`CzuStcV&5X&eF>lNg(I$k%;9M6x-=Lvat+?d!QN(N; z)}|}FNGz3!SPpVq4pVf)tq4&)JD;FPKanAMk;2Dh;5&wVf=N)1wEI)ICp_H7ZE-QR za)3o_Vo059ebhdJZQ3Oke)Ic11b*$JQw9YOG*zA32|3;7jSmUo<m2scDP}e!^%4#h z9~-g`M%^uh944=gF{)jd&<)%*MTO(mUya7ClmNZ&tQh~%h$o#W{jY4H^i)iD(G2IT zGeGzGP@Dat809rohkh|U^W9_b2>I46WfX#0cvVgW4j_QiYkGJ*T{!LpJ!VtFC}SeL zn<U2atEc{)#<5SLvdJ4?H5jV(^KrkN#+dkC63;>YDq=gMEOqD2G?$!P-eEY^e%}l) z<L4pg_fDLuPFLyMto#U%8GjX-Dgeq+JTMO?ADu4%U-wzf*V-gx7Qic~Ls<xJyAwC4 z5R&$+s^Q*T5({EpJ3>ivuj#tm{ra*NnWP&#DQ{E6mZF|{)n`{pyo^=D#KH8`zV7mN z(*5i1qQoD_U44=7Q>VlfrMgqlD8wn@IF^L08LUZ?zsd-sIS4;S4=g_#H{Wli)oo@` zR9afC^(+Y8{`KiMhh|0R`J&MGUEve5W_Z<Is;&H(IZE9DG=0UC>rQlBj76wPKi@y( z%72sjNnQPU)P*Zhf8^m6>V7Z$`GDRIc4LtPd9eBwKoSIlItTs4A5U5UN6uqwM$5H3 z?e9Lx_Y`c_6&Qzj37qCqYC6eqT5r^X^k$*e54oG-`t`QwxKH=7&yOdWWT1_YI^{<K z43~WD*~K`$xaiijw``LXVc|jdb9<l8)Z+9baLtQf8{@La<6QevKoi*?w3~rbhFX?7 zDX2EReU@^4-#h|q-nBlBA?#}f_Vv#ID-x8T9Uh{*+aJk^DcuFD?R3WtF#2#@GHTf) z(vg05NL=^|fHs+0{gK&TRpghNkk#8Z<-~Mx{g`!=)hjt`s9E8r9q;@p+4#iv8J1s0 zV>(=qGK5JF!%qzGggyh<;&9$jMjYgxSYYeDfX8f~tT=gjoYIzbG-UYIK5tsoRW&l* zNiPWtddeOYZ!5T=IvO~g%{iAh;=k0}Eq3UzC%^JOrMOq=6}}$tbAGeOUDh*a;6R{$ zYj#pVX=toMq!Pf)&;E9w6Fp@2k8H7ua36Cnq;v{9PTm=;)dlbXn{&_yfb_x>fkPUi zKdd^O37lNLK(jxXAow_qBT0g0w3*@T6ly5?1kpIHcWrRz%ldS)5NGBabZiNpR=7D$ zRO?)Wo2x@8nOks=qHaXa=I_vF&3JRlb`mM_Mwfn?8LF3mC~|BbZ#SO86qDB{&z@Lx zjJNu|$#2;In$wQz#q(Zp{hmX$nS;#yw>@VDG^w9dEhN!P{ZlS39Q`0Wezh*iEpyE* z0p4`B*XK9RFXbtru&m^PdRau2R+U|xK<<pInc-I#F0cjN;MPmW@lkR?CR<XweI2j( zhlHv0+3!!eTym>hglnzNXtflu1ZCgCPE8#h9>X(qSS;LE#FX?^kMn_g3xiq(My(x7 z_T-w8mV^7Yc_MG>7QVI5YXJ1^R-_Ig<?K~hBD|_jZx*ITH|>KZs^eto>3%T@@xEjv z0DDP0SY-Sd`1fUInxwOW6EJ=EZzB3p;hPR73_Dceb$`r()ho5Z&Q^j4KS(?1IRDh0 zJp*&diWdM^x4!Y`*Vf$73gVUXiRHuQy7K>q4!}#ZD>88JU5$v?kV;hz){UIMCev#Q zgn##LB9{Di(a{!|N)?ojhuAGQ1fn1Qc#`9h_*H$eu-rr6<+aEoRf%ooK+61Ei@XLb z_c2Ysa-HAc&gxuhk5V!Az~m77Hxf3fIn?GjsnJeDbSEjs0uz@ndR?@0r#T2MU4ky! z;GIh=Ry_e;*<LM3+}1PJt@ST0xDa-3%V{p9$(GnhE!y*T-GAwryil4ll{~iW_bloH z`0Va#m1J$**8R!-W*Z~Bw~Ey*1K%AfVupJJruoZqaJyQZr!3Yc1dZQFoK0blDT|x2 zkk!*X1M#@`;s0iK8!aC`1v06meLlbj6EB;u1eiD4UQ#ULc{BTo`l<WRdmdTKhytxG zPGVbF#NA`w27E>fobU9r5C6#-i+Fb4Ijym5yohz`i<R+SJ^wk8-=C=1t3og+JvI56 zv*Adcz#;lClf9k!_Wp{QX8H@PawREv*28Ri-RRg`XV4uk(=XK*#$ZtsGDP2W-R_UV zlG`@{n`46_)yRyhCtUOU?u~<*qTcUMDMAnS`eEopJyKUv<#bAHU}cw*|CN>f{I{X+ zw38B8Av@a9R`d8I8+*jDAmo~3#Oal3_sOCb8ywX-=Hp><Aa1+;dOIHb=OyV!GiED= zG(gH&B@GP>n+by)t%1aJRFDpCJ+FxC$B05F>+kviNU}w5-%?D4XBsd2$<WqT)+`i1 ztHuU&px(h)ZB)UDI$`v{{$UJfXhJWa%R^t}dqgIS{;L+XPk|A`ch^`S#XoQO;PW^^ z;L|Tp-cMl0=2oS$Q2LEy3>8VV*lkT3>&ZaEmKY@#z%p@>3t=6I>j4FTxh@!Mnvcbl zscM*AW*<z3Z~OJ}?deYHG$GHE+pEfUqq{dE$kJX!7-BCddPiU}TY$tC9JJ$?x{q%% zstN2F&m_nXRwb0!N_SH>$H}b$X85Q1!=0oAv}mW$=|oS24R*!#iHPqS4?)c@5RVuB z5v4%YO>^|mLKw_g(d<R7Hzj4020LzI0`#@JoIr`p;e9dkT4#&?yWRcYrubw2(Pd<; z3S&X%d8G)bKG88;|D#c1NMp46SA)4<A>VT>d}rNOS~m$f6>Nl{o?!pG8M8>u1@`Z~ z!Z{NYaI>l`rCuS~CP?O|ON2=dkl%inVF57D_M}ACT890AOXyblJpanN)NdI_sRYLK zmKBXyG0;~-8@-0TmKX^!6-iioMV$|tL+!%eo`xCaw1$Vw8sRk|MKiE(w(ikVYl3?J zn(0q6P$#}EX2dJqw5UPYb@;T>7;w|8eT^r}7DC3Pc}r4cp$G4T{21)~+%(gG-CO4w zV@UsA!dGkgV`#7ZM_$7ZK3ZM63#$R70k%rNQl5Oy<i4dMJ?4{VcC-H8V9muep)$8O z@vMJ_cIQCg{LuyDt^!d@HP_Ct0Fw=LDFY?Rg%kc#79jl`7-T?PkBgX@9r-|Ch*C6G zj*<+L^)6AUcCJd3qT|608{_Otr|Ucy41QKhjNPmbST8#@<&tqIg_q$Vh|^&#zfN!U zl>dho0D<S>NNXEj?GT<z37hLCUol)ldXqhWOOajH`5(){qCF-xY<xmCQ=7vu?pia_ zyic7(XHykpyh}UIVB96Cf;jJV8aG??{CP{lU6S6q8dsF4*q#t_Z*T=8(&|}Hp0*7r z{!SShS3kR=K)pONg-soUy|v0hQB4!-x28n{Y;sp)XED9CZmqZcUu^uCzmYGSMCiW> z_P9Z2m|n^Q#mYCI`~NAVh@(Qfh-f6a2SDTmNjLvYvl)$9Qh<%7fn+myKHy~0XTy`M zr6Etn*guR7aJ<gE75FuSyL^q$AzRIa-KA$%I(e)U%-Q-u^Y*3864lCHwg=7ztM4~@ zsNqB`O6-EaKB{MPf(hcR@mHt31lZZOQVyePl(qg!0JJ2ejBz(^V;H^pj?^Voo8<nu zJvH03Z@U3NeD()5M(X!Xgti1atCt^l30LZNy$>kqvXKBg?a!WZKwUdeH>E*`Qtky- z?ixjdp$M^Nz^hxX^nIcD-5Z9Ofl_0C@KGPvIBp!2=3!VTx&6_4^P@U8E|!dW-b;GN zcs<(Zuuqj>53cRMGMH-!-gI-$j;<IsT^#91y{oN=HC6>C91WL#HO8->G}vLpnu`rU zTXY9mD*ARpVAH}NFzzh_%P;d8F4@Mvzv<^vs}BoHB5petuDwxoNWa=Ig*V`QDcXsg z8`fd4_L{ui!ab@0oUkK|2WUQj7py9oYJwP2-t6@jd{+sW-FL-viYvmcxUZW(R~$n( z_GP9h_N3s4{>BX7{T<!?q})2=+FS`kqGBnr5Xz-y?RgyG0x<Z-Y_|fow<}X#N3@6u zDLXdoj84QcN%$|Bc_xewOa>wiMB{0ZK$}y<2Em-xn6kAipsB6w+)sv2qr^SXvT5F? zc-0})I|KF40c=56w2gyWXUX{w|0luvk3us3^3?=GP3{%ncs96~?4Jqi+T(Oijd<Zg z`Ole<(~LZEC_s4R*>ICrAy<z9JOlw8_>YY5G>pe@Ft;BM{E&Vvf*wxA+3(!THkpfp z+JxN1qP9TcVj3l|eZ=$1$7%hVwvq#JnooAkIH{ZS`3w%V>pi0#?MPDb<1k5XJ#)w1 zUN$iFccsq*%HG52@pvV^!u+fTJS^$@=&GUY@!stRUC14{;r>}t7*toGhdL?}(vg5W z#2;>y5k{$WGXGoB59_2;HkN<Rmc}JO<e_iUbu2qBRw+eg@9|qP#3?CNhYZsU@|97; z%O8J~WhA<+yL_HSX(<k#r63jL^co-#GtTf645oRqQpZWI4R?trD80k3B}#RF0j4Gy zXLYc$7<>LH-rw^`v1WGvXpNUDn<|^<+~fA58;u^b5d!6?e$PIR!{xTpH#2S7LNj8v zHODT1eTS|{H+oLidoSJF9$`B9$y4SQ`5wKnIvF}O&}|-FTpd&L3Q%}ZkGke_2tKPa zNXe^htFB71Lx9AxL!bH<NBfNBX*GXEKE69LssL8k-Os3QU<O0?dk9-e0O7tGQ}xr^ zuLUBjrPM(RWLDE2T#z<!oket?JnF2w`XmCOzij6(Q&oMOCd*s6!N@qD-D?}_AiKoQ z11go_ES#M$ko*6lygtTpbYue}Ay!2XIeG#8dy8Y?BX_HRG^ZMAjD<kI%X%AfVF9cj z`T=4Y^06Gu)0o)q)$~Y!J#q=q$^Rm~vlftw1}vV#0J7)rvQZhyiD=8Oppsa@vLkLc z{O5X`Il)NCO8pVLJf2|Q@S&}TSWuF@_><!2CA;wi$>^0}A|&cO)W4S8QN3Qf*>8@a zwrot3%&$aV_P3-#FK*KGE~~6Y0Q0iTPg!9|oxChP##ZLgn)bv=I>t4Sc$raSu;|0@ znt$}>WsKR#=R4zApX+NbgOa<m#-OM(%#gc=O?NadJmQ=%jVLEFYOlInAU}MEa+u&x zS=89=-q><!(`nO->EO_OIL03S9T0-+w1~opnRU#+RCdKRA_XDu-Srxf>pC=nFY0X9 zRYuPtm@X!7==h_fMdhexuSPDZ3jqhk1}HK&KzBdgHSR!n+3^<=NCIL}KWEFcYUMQX zEJ~gE_QpaNMVmp_t4o_yJb?T4Q#t%VdQez*se;+)B1UC*b)(38(h+O0Rg^GY)9&10 z2}7>azNOEl^j-H+q0jE<AxB?cB+nSv&MPk_mGP(*4TQ}}HaQgsU=U$YiW1_CXqiWc zBQ4scGFx=eT^4oteFCgV5g!8B1%ICU7rBo?t*0`ukn4YG@!<aBe*W)6GA~DmHp@2s zTI%t}Cd)-922w6s@K#qh6R<l{K6-*9`CtD1f>E#l=z5s$J;uw&r$ddZ$BzVyD(VcT zoG%BDPL1pOSpfpP1uRupnl$t2B4;!CZeqd3MEf+X<cs*Fcf2_jRI%+UYWciNvVdCl zVl~(3FlCKvD7dIMsDm8{EjVFL(L;Tmi*L`ARh`;-6;J`HA_V%6I~oLGj%`=P<Li`$ zXy3xjy9!1|6XH~q`jZu_V5-;J_r&oL_CW8CSz7rnyGB5#*yctb*MOqJixjtqGt-zf z2D^s;E$*@>42Un+Hr_Ymg>E7?zwoJQor;k)%XZjJstW)xmLQFO8)m7gncfFK2OnVj z6>9QQ#|+giG{r96xPRKd|A2(ratsU6bzC}2;*LTc`-WK8q5xAfk3NKU+0Re5G!Go5 zdY*cg*KCk?9vOiP19G2&pLrbi;v&8WL?Ckh+A|4%yRJ3bwToN?>SU4)I(f%%LYt<R z1h?C_*hAmWE%Z>RnY=xe++Ux=-5)6C8{3_1O}8jejH%BgS>KMq?6KI@O?uP=y&l4% zYi#gV52BqDoaxD3GkC`qjaKj8o~fBQ@(H8z6=AfLY^zsi=`i_a6&m+Y((VJOT6>ia zm1hP>4e4Ka_6<LXFpf?wKI}_4N%=98cJJo)1(4lL-#xBy{lA>R2A7hf+W))m;y)ji zJPd;M{NlY%*XLc!LOC`I(cyqUJ;~r!7d_=Qn|j=A+*ieap**hH^S2Up0<O>s+3rkJ z?4{ZnC)d_+sLPRUh7AKjf%2{So2ZMkH(PaxP)+S)v1aR9pq6)==Dr1w2cN_B0CC~U zhsGu$ebYU)#ZPZFYfJ{iQbew_JG-m^dLG;68*cZN?kM}<R+M3JL(e*ut>^>WCHV#a zKrEhzbxLV?%4CFBI#Sxf&%>~&kE!fF*a=_Zkm6nV5Lc}?8@*pEACT=fxMs%5uU63^ zYNnT1JLx-K_IV;1kAfFO{{gf=DsuqaY!8072cgLZSB~fBWY+~^6Y@U^kHz+d?aV%2 zeBndj#fq}58i9;!gKIBa^FlI112WgarFp5llaG++eWZm)5hCd&=#ZC=d#%H<SA$P| zhKR=ELZee3;)?{&wy%cmigjAB??*UyqYTy0m5|TuNqzreOJvgcgy1`~rE~x^nvg^n z;sSDg+Cq++UJuO!<n_LE8JYln`7J#j!Q({C6t$uGu5l+Rnindnpvdmjd1bZ}SfFL? zTu$9GZ%Nf@AZ(Y+ssSC%6e&fKml((Di0J-EcAu`cnJG#33>t#t4o9n*6FtLNz|g&J zaxyfqa&S6ueez@XM!a#pM0)frw_V+3YG3{4q--{jjTP*a=RR(ze&H39#<CyB!0MKM zV&5xjJrJd*OZ$8$^O!s0-|GbU5}Coc&)<4Y4)$CBQ`E+PM1h@FXSqP<pH{IUBaz+v z4?OuNXn0HcfYmbs|HSWD@M|#G@r1WxNsyD8`h&m(#`dkg;n|J;5w=GIQ@GKs?0WYY zbp23>C9V+cGHh8UP=5#1Vp31rp74C+^h8A*WOJ-%27=6cz|{}z!1J8Isp%;>CHHF@ zCpdI)@u>|CckD%B#xJ3a7@xxGHU_<bh3nZsEOyq9h#WCaYvDkQNY+nPEYz7u4nXc3 z+Dqi-pW(x%%rh8yrefmA+1lpY!#BEWJQ7@t7ZvAU?brHr{o3?3Z1{Z39C!28!s>SY z`fmn5h6tP?4``(3WW?e><)qJY-v3Hk)8N$f&n0hDRSuSEzf7t<Y;N{3GkdSw(`BHT zQWx%OsMr}Jm*ckhbuOVpymAAqKc3wZbplhAIkd|Ewjz9El0_P5l1S_<ziyt^NZu&~ zf8UZee6-Fe9uC;o-VN$b-fdgMy>iy^Ml5KoaRbvd?rsBAQ$S6vr{sq7q4<>Ab+;ab z*0-w|;A{$va7VXrmriPI=+G2|l;={=&gfmRboWp#Amv|&VJBC*&WHqTuBWzHKpF{t zt;#TLq;he<T0rv+s(5u+F<h|Q;v+ljK#`~s?|Xc74pDlUZ-)NqN3=^|v`wL#${SBn z#F8U55D@P~)>+0cZD2<AQ7c`_{;68(7Rv+WyXzB*h^AE+MW?9gC6{FQ<oiYGH4+~@ zaJ*c`u&p7KCCH_&-C+p&%+AlW8UXz@e}djIG-WyhmDN$jC<c7oVl$nGY?V{v6`j)V z5#WS!E~Hkl`=h5RO;tDrG0D0t>1G1VbR)Vf{@K`)@zZw(n%agpyC&O(C+D_K<I7|} zXX^Ze-6S)z2Q9Jmma^h-Y$3NtyY2I=xcn`dI<N4W<{hppQtn}Jg&Xj}CMPjZE1$t# zNm9I)oyp6Ty-tn?4$9@`A6kSN!3j@?gg_R17?<+zJjF1&=*1I3%e~D^cRh$s42|`} zrR%lR?FFGgNZH#(JDn%Ba3HVNuM=t=Cxcu0COvEYvWaEBk%>oS-})7LZx@II8r?2J z*|eDnbSiD%v$ei)VYkKYz#v)uzPEe0317PeU7o%YSytp7vK}`GtF9k+SW8ue0aX~k z3PZWS*|Ha{rds7-*n(8t@(Ex6i4IA8n9lkdsK3OL>*k5l9PcsJJ)Al-0cu{Jm(0aR zVxryJByG+?1Z(o@?DsHSGk=N?eJzVF0O9_Pq`q-E66P?ToL&Qx4Hv?;qj!5Q!HleQ zmym6blEV)im&)6OC+joob@tEz3-sn9;MIQ<PS2(80Fu_}@Cu;P-YK6yPWo93%xTW( zKS<7jY6&_eD5C3(U0}q<g#p{(xi_(jDx<WL#kZ(CfnIgN|JGl^R?!Gf$kzZjvsEI& z{&J*3ud}kode2kKarqa31%Wf`x8B3t)1>6fhns#m{4HU)ELHdN&faes=E?Y}@@%3? zqbiNN8!oneCJ|an!5lc$-n+b3X<zXEU5MgV8&=`R#3EV=@yX`9FsKwUDka$St+m=q zO*~OS20uHaZ4mC<VjcWU3*B6l#w>Qq<rW`Gcn`$E^#5qjbmQP>6UN6m1;B6Wnsaa5 zF4{XtP(byDd6A(5M=MUJ#S+GXQrtR_B0cVrLH}|Q4=+x!#Hd2fQb{)(@yyS90srOz zM3(2y&pks0$7gD8?eBKH$MljCM%!K{u+C}ulP_L2yhQ0+h*n<K1rkJ2HJhW*zV3j{ zvT{nr823~+uwYBswX9R257d(cs}7JDj(N4je60<AK|yY4+HVJ&1bEpli?NR>&(<1* zRXEx#gx#&Q#Arog62y+nHDn{ltpavt=0j!ufAM<?N}Ip02hxMG78Sd|(?Z8;op*cT z4=C%d*FXD=zN8>0VS4rNYM?gTJ!I;B>QLZg>u&{E+z^M_;C2m6x{+4yy}x2i&MWZg zx;Ud@c)Z-#UAy9kGm@A`W`LlbH_lMr{ISCbvCK^-U}n}FYS+rqLMbXj?U@7jo}{yi zpVMG)dKIsukaZ69bo26ab&lAK-HfZCFKKXNk3q4c<8UPCyO_DyKr~?Wjsc2^@l+w? z7NLpB7(&8~-U){A>uW6Ow5`=68SZ~!npYR9Wk25RsUU=WEOw^Wn%)bf2l6B-#we9B zH$!+a^3;`jK)W$33Iq=h>bD-NavcBU2{dFR;7VzjNoo?RZLNaF-q2BX0A2{gb(nZg zXJs4GaX*7J#MyhcrZU`04z}XJi$TT<tSGUWUxb@2tMjf97d>~zTW(NSs^2Y?-Ns4J znj0P>YNs!MX}&>x$)&ou@n~|VV<CdA`}R)QW)2}>?A{J@kP5S**)zNA;;?wK@IH<B z{2ZM(<@*;AfnlXczc8)_?&jHFezMO0Q?>t+1-E6%o=$P?v*3Q><jiH21L^w<aQ8wd z#@|11^di2aMwt_vnZ3rxy&pdN=Xws(V&DlHXQMglPeC$Io&U5OyCxP?Mt!e$nfU?* zILU>44Zkii5#@4xUo`lEem^pEO_T7~Uzt7FqW`rEB>BZG*Y7QRHFGIu0nxs<P*fyQ zj|c=z^#O_C5#m)q+S&R?Ns)MgE6P%;yD|9BIh49YwV|Mdafk8cqqeR-f3oP>^xKI? zR$|+xyPBMVK0o3e##ibc%RiuWo<C^;RM^(Q2E_RvvH`1dBZlgGsQDs$Lysmr%G{~N zjgmCF<=Exw{=9N4lnb?*){y_h0}vLik^!KJma`tG0MGE`e%GK!dhrzftKX-8>pJ}2 zunNeo^NaL9xY|=6DWe+d%t{HX0Y>!x9a-&c5QWk%Yu>5Q^QL>^(2K{>^HNQdfON4c zDo+poho&ESU5@?+>JI>ni0^FLHw$^iIM&ifjS)QTF{H1dKI}0eEi#qF1?8eTxUq_} z`t{{fz@yik!AR)d)dWhhVov#%j}Z~6!M_alTKcq^$t%bKI=A|2mP-zvUavFDb&TY@ zcHs;7cdZI*%&nMV9g<j-sMk1cBOX3U(|D=^$F8K}eU{;dI}f|hNP`T{2k<tyQ|26a zd<|xJ_j*h4dm5{@N==m?0FiA>isVoz1{p0-V(~YyyNQi(C{RHgF57-7GG%Lcm|Yx$ z+!qPNRP}ws_qXJ7@0Co#XBL?>HZSE|!7FOFvn8}$s<1Q(d9MeU0tptdNio)|W24&n z;NQFdo?QR+%Y}yw)Vk-#%4cD=aXFA;V12?U>dH{Qm*J2%_2$x^dTHuGJ8Ioo?(HhC z!oT@IK-8-kC^1dUOj+sCrbuvDo(%P30j^+k+V)arFna!0Ave4JjA~OU&(r=OuaU4L z4C;oR&eUpWCG38~_P8>^%psM9Czq^BMa(}7tAn_f*=@-k2R+QppB2ihlkm|y`_r^% zcq$a_IU#|PmUM&9@JfG|p}ol#fhlk4V4ijq?~Iex60FzWCFsW;T<hLVUY{1M^Sh7v z>S*KK$H)!esP5LO=4<7YERE6~KdV#*JIJ6qml(CBiZwX3txL$8S(El2O%ASC+l46x z8e3U5O8|xqaDvB)z!9_MT9spbn3tbI{(X~TFKZxkrk%hi)jlUYS~2WkI|m;tkV{`* ziz_SMb=dlOQ;L`1Y+T1SEzWk1l9@c(yP2oLk?E)y1LtDx{%mJ0NC;T26?}0fVR+*k zr11GrKKLXOIO_hEKnJwujVT4h4!fQLl($heRL4mznhayJ9kKFnl;P5+m6!P?<Py;h zQMBmp$SxsLpk8N9#&1F47C93Uwn+V{3S8OapDHqe66S8OuuVwVimWq12yWEos4OU- zU$*Pg91_8->2}3#B5m2C0=*{Jf<8*mF2<L<a-@nj01+Gw)$CxJ_u6j+=Y%8}NlR6v zwr;3;q@AUD1KjnHvRT;38r23k?x#*T1;iK#|9n%I=*K(}ap6C&O|u-0M%}Wm?A{Ee zB8`G3vM=b)7P)|tLAcv&NC*E<(|nVq0Ce5sZz;AB=cy<6beUh%G>qgOI~ao>2Hwxv zImw&71L3tEx|fJOaIfVhBHBHA=tB*-Yp8=rJ_5Xc$2-b$0eY}j-ECNm`0=Iu<*#qg z&veh{2w6!>f@skBsTYK7o)F5S3YsM<^|9dAE1&Vr0asB)oVpOZ=_WW>($fkTU7x_^ zzuT8)Ad8yU%c*`!)H+nDqgqi<Z%2_slEASg7Bm=TZ~Uzb6QB+1WwdHv1n;}u+)${L z_WZ1HaNo8uNed5bx7v-`gDn2)MK78FA@{WolhRR2yU7DSu1g*9T&EZA$;uo*!t0)L zUorwhWbc9HZz8zw3s{MjGM>EsnJ-3mZ#a9_M2Yuz@I|B3p9ensFMb}q!q1rTP}K2I z2N7~<gvg<`Ifo8WDSXOKo%MLEd>)|7M50E~DI$@K;*OKQZHM@3Gow}4<>bz9nok2@ z1>pUjRpoD-*+g2`^nRf~oKa=I?fMJJJPC7OyZ#Q|i_<DWuY(7`EO|W#=SFEb9d%@j zOpWy&S_Ql-KIiG#Esm^Itbcw=!Vn#!l-A})mhOkCEY~4~Q@45$dbi8t10W=UDA972 zhKpO@o6a}mv->;~Tx<LHQ@$xvdwjER(i-v7)N2I@NxbsRJ+H1i%ii05l4Q!rI_?I@ zj%MUy6U6lMF%oowA|EZJ6`c6!X@%C`Wz~V%ar76L++KW19a(BfBwQW+Cqm{V|HrJo zw?)P<i+L4&={IITFk1HyKj0S{C}1K^@S^^Jp}?X4lp*HC_Q(F;InS1{!$n}WU(Pv$ zZKF!-xePa<F$rv+rZ+7#WVo$PKDNI&X75lKXsIr%k+RrIuK?Mp47%YC7S<2qFp0}J z+e}qQehGmqrZEPp8~y8ofvbYXMKVs0nZlq%W2Pi%m3WrTCPFag%iQeyJ{--r;Kw8A z_V7t~si6ut7UBZLHB?%(D+4>J+|8>!X5`>1o5#YkJr8%%D)y~NI(AEgUPY)hu9<`4 zjZ}CIi@*NHYKqfOFoD4Y_`y`$2l;2V3cU;9BdL2u*x&`t*(jH={s+p-cFQ@#8K#6? zUaS=-I4%{)ig|&qy6^{)`T3WITSX?h5{MbNzmw?3Wq=vSgw2o&H0IA7{~?iD<GSM; z%<OF2f0c!i5JPYM>@yj-JHqie@>Kw)b48^hW=;Cpn};D90^y%N_^1HlFIvdW)?p`x zzVzfZoinxUK|fK{w@3^eR}W)h@N0<{ymK+?NGpWvWqh%A`=y;S#{h(MVXv`v7Qnx< z4y8wR*0pG?K&}PsoaY-$7{UYm3Dv$?hhW2k6K^Pmzi~9Y<<~||`PEx=ig_Mo#=D^C zH9mujh-`>o^e)B!)>&8Mdn;OcDiPKz-w<gP;Z{wtWV4|<c{0>kUrfb75o+SJBKyuZ z5`C0dDE~ubt5(wHUhO=*kfLH^q*Q8UQIS;$-(T_9ao4z&a@gf*JbM>m0rX2{HQrZ> zctD6N<3<W+VU7vOYc%4X0uIf1UHERY?NYsU-z?+Yu%yQKwk&P8VbTAlJGyGjah&g^ z+p&PSRI3VZ+ljQr_gjyf9igB7MJY$TR>$Q=#qu>{pBIzw9Ix@u1<2@2uC(%W!uiW~ zUWNFdvCC&bTLr9zynGC1OPlBKO5*y1k>&A>2kcsN(!?Pq38^Mh+zfbCdqly+ejJpb zUg)>mGiAG+ZwsjO>K6VgzHb4m^Oii{@$BimLt&ACZ>r56oh$j5z3bA;z0%Ls1*}0E zkvsJzzs@M)ct&2h9>((ZdDaSSe{A#r&6g$>!h}Y2*b;y^am8E{(xFVDoX}uV9tu6{ zZ+-ZVvM=d_oZ5P{ywQj!=wM1xQC7*Ynpu^3$wFeTnwu<LAADKmig#ah?KpT5cNs4- zKU*@v2sTb;fZ!Ky{;9x`{2&#3lSyLylwH7~GV;S)Ow=pJ@)Pf$#{^#CE%o8^$SY=V z)!N8NUbDbbM|vJ?W?tpxSk2f)C`yn`HAZzbi{eL-3JK^(%7lhf$EPRyqqhS@4BWJr zbZis$$t+hbrXOTcaj0FyXMoA00MM(3<@na*QOL@7xb+xfxpc5s=v&!lgZ+}(9)>(F z07+5YN=FEHDoe2sId@&K@!bz&Oqpx<;idNu>kL7yR}|670y*(WQqBjWhfM)OuV_eh zOIaE(Heg?pG2s8;map~6&=xVjhVQmRpDf%e`2$Dy0_4b<73~|Jvk(1o=C(yYZjEt9 zqdEOM@KzWaur2-fS$~zEd;N?WVqubr&$Rv@PWE4A+E;@)TKgCrNv5ptq=mgMd|iCz z0X$JZm$4+n`?jbezGE;YC=wIYu6f}y^WEi@pqP;;kx}zN2hpj5;1>ey`lB&{05WDW zh5#~SN7#DXR&*NOxeble<iX2}HU+gZb>!FT@FskOfb{*Xqz1^uewnabr(5+bup3Uu zh52x96v=haLc+2}JEu>Yi(09PM84uW4Ol&a0N<%Io+w-V4Yw)*lbAZ-RH9gn(YNvH z7z35n?@re$0ZZojTcT!}OpIW106bOBq8*l<lC6r3CofoP-?(s}={-J8a4OqRBJxWf zKm>jUm!`Z(@s^OBmwYfjUE@1a5bV<x>pe}Bz6K#dx@$+P)uE{jb`ccd&~xDejmX_? z`SFh|<u#*Mpx#T#z_${qSM<80X*qyC<&I8s>)R)doWSVF_;!}XEcXhaF<9sU!k?6e zIr&y<Vh%E|asXx%V|Bvy7DJi;&;qD&l&uWYf;6O#zj6+1QTqZG+6$*vQ-QvkH>3jn zUPn^|5IB`Y3>&y+&gdxl7@dBzl@_oWWH~pi`RiGKU?%299X;VHS`r3<vdWPwP9v=W zEgH+Jy82JT7j>aZ$H}{R=>-7E7;^3m>QLg>!K@YflRHNW#dQ~Pv166dX9CjR<G*4l zmfTT+(?8g(dCO+}DL~_2h+gQn9jdkNs&8L)NMr({%(p*)@+E!hplWLWM1s;-_4elc zC8(a|Q|n)+Y@z!WQ?@&nUIejRmLL@;F-`yp`>%cae}eee9tFX8NucYAH-ExhgwMTg zeh)m&F*k@gCT{^qpp|zK|H8rY63|JOp87^IagXU>JUMeq1(J0|Z{<C!+Gh_`dLj$s z<q^Zcz7R|HJA)26EPrb?x*M8tUpbuq%r!51NaX7~tap8NtM%BefM(&z7+_7|l-E$5 zAoo=7-7&ZONSBSW^&iwj`0ff@_k$sS%9=BZO?0)x>-#a7p5eHCev{M4dftw8CbTe` zP?h0(ro2HDtZcQhgdO(Tl^MunQ1C=rJ*~|qN{|6{j=Jg-PPMiQRZ770Iq<z=-yfI% zzP8ybOTTIrNDh%!dfNPkYhJKn`>r{#wrc7SKH>eO#x;NMvyIEBn(7h*t#3_Y>s~FT z!lH&qUmil7RfOfT3J%rlc)DE%J;C4>H^ANGQB#>8UmRSZZC(;*nbl@dn;;w@0bTW( z5uxCc+i8N<F`}b8@5t%<WeRYXAj{N~q<?z<3-^Nwb<#w;Cwl>(ZV9(^aj+EdUI&KC z;Aci3Lqy&H(<DTCOect&^7+M?DaXST9~nX)8d386FMs@2nvuGv!N?HL+|Tj)wk@8o zavZ*1rlT$g6EBHq;r^fl%#)ACj%rIL3C?@^4HVrX_M!6I&0MOERXt01=yzGa-^!oT z1Ver!L_3wkeMY$LI$zJBH$^*Z9hW{Kc%-|@or#zmy&J4G)>`D_)?!vP-&rosB{Orj z3YF>+WtcdEDHz@B{;EYr2sx5Axj)1>1;{nu>Hs&l3mHK?SxW+ekxlM3idiaH-^p)Y z6zX7&^)}>4b2fX@Uw@Q2$OXZ8V(1UZ^FJBKqq~rwWfxe307-;Q{}a@CRfYqAK4dWL z%Deoz46W=c;f>99Hy&ZeQ~_ne%S}yNG12q7IiYzsgDULmyEd0dYRWXY5U<G5ZCE`) zY?xX0nk3fgvT%|E8kX#3_3)x_%TXXl{`*z7pVb)703Ullk}&aOQO&}a$?q%UwubtL zhd(8c7v=5LZ`OV0c8iqs(Myf>F<a``aQf9<RFWBLU7M0oVipe|1Q!P7N1YsE`-q3X z*1L>t{=)4u)GlMh4p!9f3Bz^}HNc+1zLqHAzXk~O&e2e`kmVQ|J$kWDNNO<IbEln2 z*Aq>&>&hY49b}fOaLajra2Z^0zR|TSl18~;V*x0UR@TPvsYA&Hg#FA^xc$g}E!K4z zChf29$`Vv?U(5aBpId@2Lt}nNuMyApt40ZyC2(8&nLs(tp3m!Np6*`j=LHwNe6Ag_ z@G^-zbHQdkn<ust)^+?1XHF0wBKiZ)Z$3;9VG?E67-5`V{jRovlsgjdFKykvq7ry^ zN?B;{s3wq>qg&rqy8%|4R*uH_Ma{bQZ#nKE#(lA{4{+7`c2dLL<&2sFC~G-&E@k4Z zPplF+R0;u@3Nqb^1RuktijI*imeSdbwLvwAZS-DOUxmmPrX(U-)mm-Lc9r4!vUyAN z0M);9#;Otv@&#1r%c?O-mCRpd07cl&P9!;75ZB$Ge(LvK>WS$pfMr+)?~c??#weK1 zMiDrf>JJQU?ytMt3%Ed^CXJ_oVmh^M{I}-ozZTh_T>;U)M=F?m4<VrGd#nB96=ggc z0TZ=@xs4O=LiYLZUe>*{fzZ5pX_3KKeyS)NY`y()sT(4W&0)wXJ3xI+V$A{!ZKKGx zwmyvdYEz&=nqT+Dh2VS8<~{>u*Y=H*<?khFZlmQF>z2KX^#m<nYQlu`pA;L^ioCQd z=8>y$E{YwkRNFDNeWkMVm@{y5!0OfWJ%9-`i{JWg^F&l2Ja%DwxigtyLG*XtU652y zkGARz<q4Qu^BQ01-uPA}UsbiK+MT=(1m4ZPt;@z~1(r=)<dL$P@p7AUTJ)Y>#zfE3 z087REgVqQ>qp`w++0~Pv(hHnF<o}^F8W^)3WY<p$`k6_foQdc_p5|{OMH*bU-LMxY zYcC?-Qz}81c=s&I(-dbc6i(4T>L_evUVS+Elrmu&<!C6BKBIFiJ^Bc)s${&4E73a8 z0a?oV#H{6R_QN^-A0?fO9oDf0w060&nQlZ*aeMXhOVre|SS2Rle3{f3+tM)d-%kf! zSwNN_<{r%XvHVO{AtP<UKana_zcHO%^fxalU*b0(LQkfZUQnMc69KOFLe;e$@hV_z zTs9pQlhbJKqR}FHi<mi6b};E}!5LI}%HKs4dGaYX9{9tmny0NKku>ftLCwDPlEz&t zD0lvWN5{dCx{ZAPFI$2%XnN$}vE1&zJ0|{XZ5+i@A2&NPeYY~|YyfXokA5&HCi&i( zQ(O;&)j+@|3NzB1#+!F)hn`!o-Hgl%E%xO!3997rD#vKt^U;mzWAqT0H1%o12*mI) z+^p+S+8QD^_;~w~_0UFa9Y@BbP(kaenQ7aRA+?pYrLt}V7;<dPZ|v4Q(;-HzcIfba z_UXYnK+7vU#X#0;K?WJYIyN^9nd*Gr${O|$Ff*U5tv)cX^>SVXR%OS#0pw&;;2*4i zHVI0AHw6AwvpfoR>X5T3JIdp6Xju8d5N^KP|GWTT(;b`)>o>of+jPuQuP7N0I)s|l zNrEdEY9s5$w^9V<Wf{M2R!=igk9~Xm1eia6C(B{GcG&&(E7*cu41f>HpC-M;SVIji zKIc2|<*(G0lKp_CTl}n@W-PmqX`E()!Yd2)xi{+%6TlL>cg@cwFCY-@+5yX?hZX3M z$AxVcigph43#vX30o%zLK%ogab<4;62SyO1U7k~K=&|&vM2oJorp9s3eNR>6-sPZy z<gFfDFyK6?*Le<RA?l>UdrHs?>TSbLC6l+GR{|7N$zmv&=9e?j{=7$thoZcJFSFKO zQ+wyR{V_XDm7e1fP9JA2213~`B$f5-)ZazqHAozogK(K4XhfIR^C4yG+D?n#ntMbQ zErc{~Q|VIUdeukzS~GPu)?JozIErj|-xWw+Pk7!}X6Sc~<Z=YRTd16&|8FPTpGHbM z_bA)z_G;!l@X+Y85u@@@p+$}iSnv(wjSL_+dWnvIdz={oxqj@nvu0Jnhf`CoT;Mp- zGa{L3(pOJ0#(Fnlgto6j$t>ZX2p%sP`bN_e)f-my)Z(o-2O%t+1e=2yo6o@7NR^`< z4Nj9d#H&T=6?eC(PZ2Wr@!&vz2V||xUfg=9q1DG_0B<*}*3Qg7Wle6(n3*<qQeo?r z_i6h$5It63gQdj3CX+Oj4}h?3J3d%~L5CW!Z=O(1#whb$M3@Vs1KVy88dB)!9~IDB zL-h%WxPT#0u}6n-;s$)!z(saObT?Vyz_}zdFJDeuK8Z_Mt)g+S6&EZ2-rEr``hje| zRM!f3zOl2<ChS6!=-#OYV}_4`5a*qwh5qEQrYAD)?_B-slH%|{J?w?<V}1^~N9oMs z;IcKA`S1ia@2}MZyPD4;O_JB@gx4H6kKg!maBeq)*E)YqO8?h@SQOplYRu5VpXA*X zOMOJq2fA1ge`j(yr5`T0sqLiBq4B9cXRUC`p2sNU8(yjkU=|rQD^^sW2S#khmP9xv z4Eba(sS^~sEiy4>KLyTHkz|#}>eJ^9WkFJ;u6)t!ZoXZg(^PlQ@!I64EMimJ#5XHN zkV;}#JZ<tf-YCM-OC&cc*X!;st>H9<i~{E}OEUi&hm<U)!s}LG5!H`&#|ei8N;4OC zeH)z)uHb8}KkTCZL&m%L(XpZl-`@m^DKumv{a=9jpGN%SXP}luG>_ub;y!`cgsl(O z5_k-LzXj5{3^>YJW1y1ZgS%b7tT@?qHq12(gdATe1z+FN!?fhGR9CqT8bfM#pD{dr z2n155>H&gM^%@m=^D)^au2^G3Re4piQuhd>C^;gVRh~HO(NC>JP4Ug2gP{-(wLomZ zqa(;Eo{FvUN!7J-4rdBG(&3ggFOoKTi1LsdXfx@?qa;;)hR;3_uvt4wr@#V}0n)L` z3rNl!c=V(U*#AOA%$Jta4cdW(>43+#mX!=s1*j@S?;U+*)~Z8n5_$OS1mI&D(o%<# ziqBWb9~z5#L^|Vr1|wT?EA*cP`FaYX8L0D3*@(WAOr_MUoXu=u+?A17+q}grSY8^Q zC;?^;8o%Ow_wJwjtRt7vi$^d00$n|Dn-)L1b;H?3PJrX|DJjj?$1HV_XoYvq>c9Z^ zjlWP~1NoGhz&^oG{G#G~C<W}Y;pF&*B}{V4rI!%f0*a|e0l-3zOX84I;9AT<W#q)Y z*TdRpkkZfDM3K5n#!HB2F)Fg>DCqW-2dLggLw{+)x2@KsCQi*&we%9jHskJx^xpbR z1khfB2a*_m3g>QSS{KjM_nEGD?D%*7x`@VI35Ak@0dDfwfzIF+B|&qK!1jt0V<#xp z2l4f^Dq4$78coVadgYqX);Qdnj-$!g<h)}YbdTr~94DI)GjbGY@?nZp8qRY<u}Cxs zzaF@gScvQeM2~96A{YO2iu)g<UHTK?PE%q$4;<nbGy#3~NA=4l1y@}KQeGc~M#U9t z=z-3JqU#_ch$fDe!SrkG$KES(oafDEslFMco%8?k_2%(V{@wq$rP8iaO0q<fY(=sU zMJYlN*_AcPIvCr`&_c+*C2QG|E$bM|Xt8e-Vk|SpE`}KfGsc*i-=+8czQ6bVx!;f9 zKOTd5Sgz|_=k+|#^E}U^IiPbWn{p$Jmu(mvSh0-QE11NzXk01q8H>u!C<8XlP3MSQ zTAvYO`CFiZY1O1ESuovVHQ{m>gSocX<IX|`*q~zNRupf3Ih;YT6uuu_ma;WPDQg}) zI_*ol!O0moi7TrB&qOO0bfciq?Qci;2cU4c9`GxyUBU(Y?f{`>Z&B%RH*k_u=p8W# zPxrg1&%FoT{q&oTi>7VEu02oSXt&MvYHc`)c4$EKsJ~AjHj*&_G$7Oc#@!G;>!zxQ z4PLJB8`hjjwf|VmDabOmw)mN8H@!_LH{`$m{0;)^tyfPdE4u2{TfkK?j+i*dld^N3 zJ`?(2?o4U#0i?bgUg5^eukC8?Mbd^lK|V8#Y2?p@^}1ob`EkR(+bi?R>z^2Yne(4V z>+fpuB>wFZmZQ1IH2L1X(en;2f-rpH{Nr|!<^k=3)U!rvuAu#U-h^m6SW3oqHAn>h z0$Mw{I_83V(>hrD-zYpgvk%B}nqj_Lu)YINmmTl{?Sf+<BiOH9rh(~dlC^YpZ_mr` z4Y{uh@5;*_PG|>O-JR*5{GLwSHVoF8U*??=6bSl6>JQ-J3wnwN`qH^<GQeS*OzolB zu*KoEs^19s^nD-$=8ugl=2gPfyNzxhtp0$)p^@&j$LyY#wQeqt%6yNn9N+DPe{a7t z!6y!QF}e366Uu8!f$or#v}E?U@Va}SrFEg8v1Ju))S0d0ePPJI{_#H#<$rQk#|}Q+ z^;RuD(Yp8F-!lm{Ws~E2Q9UWG*ax@^%8=Oo>B0kNv5!s+9nwC(>XMzwd**Gn_B~Wp zu-eanPFpH+`}+jX^a0QWoW+PL@cKDCty<PkS&RX8!f+kY>>OBlaY*J4m!SXT$<<0i zo<Y=+9gAm(eOZ3A+qIqa>m&rCAe*(b@DkOVt|+@wJ!r7_jT`1!KaZiFt=17b)Y#F! zyL~xv_rpeiEbN-K?~uvR&+|vJ*WNnk{Wz*B<<`ug7?&ZkK1#FSldhJHUUsGS$fM6? zlwp>%#8dja@61cX?$kRMLI><AV+t!x{>9Qmu6Bj{x`zK8p@^@|1Gm%kO0QOoqqqv% z1!-@6rojd6uWm!Q-&hWqC(64*F(c{a@vwm{)Weje53vf&Ca`I3D4~WK(~HzorLHT7 zF9YR4?GK|Ax;T|JD-Ot?BF5{k{{dzXB=1|KuO0ELA?dimtA!4lom#!w@w*^!R{P45 z=m6-Trx9IRQ|2?*=t<IRi8%c^2l>h(<lc_gFU;Nm!;Bob5s-{<k|O2)n`05`*ZgR8 zI&_i*KnO&uU!(Llj$igGxuTn8{D1-Q2dx>4Low)4AQ`F)t_oPo3$r;!HNI>+ssfy? z;kD88)_@Svl^4)7Z)(|btps&@#g-&@0`~fV5GRp-Y$afPB53bJxKH20oH|I8EOU_b zaprnCe<M}$6JvmiVY;t;y|AKmlgN{guW=(UoW4W+Q1@X?*VljeaEHa+7)!kgzSCPu z<6gsu3?;Jy??E$qSFdi$vN;e2Tk1f6F2(;zeY?O_WjFdnmHy6-?Dux{%3NQo*-jwF zf)0ugeLG*a6Vw|9`Ux}+=Rp29$_6Cj6viFX#kRTzxZG7az5sp`>=<;w5|4_Z25eW9 zwc8v%$=S5$n@dP_!fg_*!+9{>*6VqNfQrk~8agQ8)UkrV@<wKg54FC$5RVJCt_OCK zbBVKQ%%11w3n^S*K;w;AfUi9PUJr8HS_&U**YI%{f7sMb*2&XR-=r-X9#v6Ohm+`^ zPDGqj^$6&;9B9BZ+)d13$1|TYfb~x5>cDOr!JW~ALhEuHResnzTCOX#d*Yf`U(4n> zcBi;0ms+<K@t!NoeBK*62@wAZ)<G*Zi>b{!jsVpKtp~P;419lmXi$J^li}Wg;T&M_ zYFrVhTzENX6Xh$j%!!ka!(YC5jW-%FH)iX$h62MTeDo~9oy|O-7S41E@VW`q2zJre zsvDI)xb_}rt8~iyySNPJ$oCxuJcLg)O}+-zklJ%kKe=`3@uM6<nm)#5Hzs|+<q$1J zi9tJKWjlEB5r*KIn-n~!cMZA8(fE4o?+1w3uid}6FZkUT{&1?}T2Y1K(g@&{1J$Vr zM{Tf2Kz9}P_W)P?Hs&X?P1J~}fV%K;41N%P&F^Pp)K^3!DK}DJc6Nc6k{iHfPO7o+ zh(8KU-HZbgMmb`@cov|T7Gyu>*r&5XeQW)Y&Ktm8!_FS^V5@GZj=Cin-4u+TUg4xR ziBq;RbO0NL_6cQ;_GpA!LeN&QKCHUT{O8q|TQP+BO;r`=M5hHNJ$%5~)b`=4*u}I< z>ik{k+bf0{7juElM~PE~U*!HdAm8oZTYr5rZfzs(_4SW{pW(j?iuzMe*@c}a^*Ld> zdH_CJ>T+`yF4`wT`G(~6RHSmU1K5r?q?r3*^+xEp&0s0oW31Bp>&h^p+(74?m-f{= ztYtJ)3cB?}$Y6I5T{S5H3M?8rT=8r-MmG>1cz@?@pnYHRwEd7NN@Zm&sd-VoXuX)_ zYZ*w=Xp*&Fn}YZrt=nBkxaeXSWHGiL(w^T9uO0e2A-3+uCRA=knd{2lQOWe$c3E`I z-p)~3aYH=5vHi1|=^EG~D%fv%yLPh=f|AdKZ7(kQ>7Eu0n2lcyfYS>$@aeqo4Q@zh zzGKqyESTr_^<hho(#pH?rZSCRSPt$cS}f{E*D~Zq>t-*Qc*!W_a*%uBzNP`HA|55- z#u}6@)RW!5v%OKMVXfemYVCK|FmNDoz+Q0u-8Tt*-jhYq&C83J%NHj$3KWlME{=t_ zvs+sR9lwbgDW9{|$Gg^&Nyu^qua536kn6ffnbytm;Q;;sFAh0dvfp~=iUK?}ScERb zRw?E^Gxk?0R~YgtYHnY{7&W&_&En|ZF^T&sN7C9jntn$bG<+~D?nP$lPd%keKR!kZ z^R5&CdQCUqJZ1G$#FhOz;BQC)0fexvlnO21^D9A$t+`i@oK;N}Y$z25#gc&D(fZM| zs?y^^t$O@FM|8SHCzVy4b5S0nsCH^@pX%?%ADh$-p@6A<ORAS++hjN%lPSQKRc(!p z{3E{uf$8eCr|n=^ltPpO1;ClI=`L6O?z6z}29~rQHb029)MlkrbQ`h{moa`SBW^7R z+Q%YRkeC}Y1~;iGjpyAENppswKO0%~gDg-ctB?DnkW??lI5wb=Q`4RL{OYrV|1=Y{ z8=Xl;KsoZ{-%QEYDNXj-u>&8WDIZoVHZQaeuU_TwHg{W4@e7UQjXTmm0iO0<tX?>b zYT7LI&<B_=nXoQ^WpAzaSsZY|*M!B&hxq@3HSn$|QF;yyKHwr@0*sc^A7!UctPeLE zb8D+Ww?^ADGmhGLg)DB|p9tC0k4A1q`fL*n?={ng_SFdN+~jC-Sr1$kwI={h>w3Gu z(t4%i=I#dO4MmoxU*Kpz*XJcsOVk!4JC6iio336fd{k(Q3)ZTi3gaID_*kyFT@51< z&Y~9z_>DYS#REY0eHLTcxsXf!_YKP>G^2=}K*fR$x9QC+x(^Y#U{Dop;00Rxl)G<r zN1b~_l%ci~$NUYq&9<SG{D@@K{pNQa<n|aZ*l&UPYWMldICgKw0QIyr%yE0C!8Y67 zEGSb~FW~sM4oSeB4Oma+GX`MO!o}qF=MtFew+F~mzy?QkwvP=U(_;{;$(ezol^Ug< zv%+k$LsQCE-xKg0AWDW!PrEBI{8VEfqhnp1VX$V^MFYFcgh@Wo{_rPmoy^#o*wpta zvKb{%jUV&3!5_Zf->rgv31|9)TlBJ3l<s!cr>XJ3btm5H&P^ZC!GSn(<G7EVasE|! z>EoDTU;r@NKjJbM6nO(Mu{sZ`-uYzDwI<*~e68#~al2czBI^5<Qk!i}i>K^@pc}bY z?i7==yf7kuz&zV_%(Bjyjy-oK_yHaaxQRx!p%f{{@PJo0;1_zCv}Rqb%>5eDH1}YI zoH64j#78f!2IiXlC)O!KLmH{njVA-`>MEq_NZeteL6!v9Aj?(Ar*GGQ&;Aid<8T01 zv?>U1xp41fMhe>OH$?L5y&%YM8aKlhHs4j$-iT)x##G+=YeMv&R?J=8dqL?Iw-x#2 zJEgs&7aXnr_FLyV^_wG3fh}gQfAWC$G2KJ9YbqtermjFkaY{HbpBw0sHaBSAijoDA zF?HPA+X?T2<`N2CWfM*odciGQH~GQ^o(k>t#q1{7bJOZi!|}Td7IWY21+8Uc@^y$? z8k$b1r)-~4Yo&`=YV&xqx}e|J4~ykCOvoaV3)BMWV%t3e(6mLI2Q0HFW}tPqSW?;e zsZ|&(=G(9_pf?z0yJJ@;ZT@YoT}Wd@2@iR3HFGt1+j1faMMEmgQ-F23@@%I5JFW)z z#e}lqOxj{e)uY0_Q4n28({t~bJ8O~yOgc#;#3vsV8$2sI7<Odv{HmgDeVPAF#9k2b zEI8E`YOgX=>Y<BgQ4Q04B~A&mZ2<*+P~ft?kZLJB-|O1o)hE3lY_`l|<n^+o7nh)3 z97B#P;36onrDnYFOm&8mY<F2RSbl#gU(mx^(%h-x$FiC4^kpx_3H5HbkNH{s8Qay| zMF7>KF$=%Ps1)8@?Mqjb5Bh|5w3h_m2%NdgdQ9A3k320M>;6f|mp;RBKJf9GlX=ff zChE_tb14HnvD(>&(ypN3SdJr?cJ6_+ztT-BoL@_{iBWFz0WGVzush4S;^Mg#QGi8~ z;u3j(odF8yfw8ry0euMmO~xmB?9<pW!8w+5U)9a0R;t^45*nYCV}F!qDI8LvfG5aL zF?M~fHBL7$-6u7MRV+q9pGUO7hQ%Xut5>7hSGdx^fPt$GD|gi^ipIMp;BDbgDTOqR zaS0mZz?=9op!Rl(u9NwH$N1kk^REW$uicUCXGPQB{|2UyBROo?`1V%F?RR8{R85&4 z)4S<l6@nsd-+cY@UJA&}ZQ;6)i~AxcSF<#mSM3k*w1RA+;n@k-wfVD(ZevNiCGE#w zg;{~mwu_s$izR@q<=0j(n*7W^uA#rJil_BW&OV|c@!xMPa^V?%{cVYcAZ6%pEuj~c z2Q%}6`fr`i2X=v4TXQ4mD=c8!{jf&kP3v0HO%B86?V<7lIv{;~*_}7>XZ!#=BbW;8 z)-JfX(y4n?V`uf7W#$4pTcebz$6SBEF*aip2ZKhHuN11UxGa|2LT=YT`(gUB-t}w! zMZ=`&r4G^9dyjxJ>KgrkX3Ed}i?I}It(gVsVzfSq-fkQ=9b34gAq%)4VILLhfSxiZ zH%JN6@&Y`==Pbzax2b6%v7Ihwk733ZjINj&z9psyU1B|5P%rZhi><Nu0JfrnLY2{Q zc42NYU+w%qRpr-%NyG<$2pRo?=3M6f{bg&+!HcHs^&TJ5Y0g>7rQ92K;%9l@8-Bgy zD!x7YbWh$i|Bna4Ka3|DKB%{ETE$xLnY?i1cur`SXhnF6$Ek+g7f&v#UrH$A_*)C$ zXW1Ha>P44*v&nLhu#)V(1UC=ggN2V&uaQ2EiB{k_IJ8%EY~E{Lz0$=Lbpwy<WGd?5 zP>|*G;EM<GABBa05v(y%zfgZx`6D&A^$INwQNkSWiY4Jp>d0D55b#4PB+py9YQ!lQ zw`N4Vg`S)QqyZeZqK<Qcu`o=TqU+?EmD=Vz4fH`a^K~EDf1dvb&T95Vzy-PTZdu)5 zvKk=hMm~mcY4&Nx^J*q|gMlO`vfHoc4Oc))^LnPHCSHHaCU+U_n_DZry~z%?W+c|f za)9Rjc!H(q!Rx(>g`7u~a+E~=Y+cU;qM{qJ|Ckxa{DsDy-Sw`E0wn+TYnCapwpqYN zcC<d7e(I4`d=^k>D|TMo!-N4#TvuLI1c+jZX!!QWjrzox@-ffs72Z5CZu6{RYew%< zO<^`HFmPTz+x%56&}4@cA{ytUGkurqW?~jc(J;Tjof)BQ;DipZ26l7jH;_=O$Md!U zvPsF>Z`5%S3r4ThgkDphODfV_;gUg#Fc`X41RO>EiaL#@t}KAEBeCBvrftkc42<M5 zd0M13Y?;K^!M;PJra3gQ4UzO-i1Axzu`V?`iAi|B_@%1AcFa|9=jYsFLM-f(TRob= z)0!3R6?*aFQKjiomj#AkcCBrlR$z<4cz!seLiVD?EZl8kVdgn7&(oPI{&aH&P^A<C zIZ`%#T+8NM{FG&PqvU4gNDN0yK=6ZX%ip#GSJ)i%F8s?urROcj9%H93a)>&o-uK$+ zE5D8DFsG8YrZ+>^9^??NIaGexCz30C(gIfd^~!MXj&h*ICVmifu>XP3v16`QA^T$e zenEhxva+$juBKSFBMxxMOJD7QX@*iew0)2s-tv5kDR6WOTqAv00kUI~kneJIcgBi< z>ZPrT%am&2cq<8!m0V%If$N+0zbaH0!7?8;mV$hm!ceQZdppA4Lbd6<1a+U@fiBom zlsHwym0`MC25EX(hP+Qlk~(=^HSXg@XTn@{G%UXbz{p?3iGb5s0GCAf0<?RiD75{d ztU7o8!bdQThmCASfo-V&bB;Xoao4!}SZnm>e`!*jN;yo~(l{SmtfoY@cUuZy`uu}) zmGaHV!V}RA+is}kKGBk2X~EbQ);@H2c>0&ip`8L{8hNSsPaV`!y>_dRgS8s2kwK<n z9}Vbpddz<pi`JdXUVSB=M<{D>e=qHGP(HJ*M`O<rXlk|a+G~viH<=^tpp+CD>#;PC zI-CZ*&3Uk*U2r@-oq~t6HcDn}60fgR<0tE1TjJA*+x@5a9a~jwHV=z;(T9$?6drkO z%&|WpV_>n}6;mc%dzZo*Z)R9p)OkKB)PIP-^$7P~hJ~pbc%(rKo%S_^Zx0q~iZ58L zu5E-p${xAK#dFOvvcX8yI^d)0a=amkK8iLdj;sy<a<<A=!$p{Uqxo6zEGPN(9GE2l zxf_0NJ8z>-W89T34Db2l2q!%Gv5Ps07$jxL3U303VthWIX4&Pn<vYw@i@)a_>CI<< z;fTPKH^#3XCc$NnBNcz#CyJ-$zB!-jE&HH=KkRp%iBfoy_T|oRmr~;&M5Ot2u-1bh zuGI7z%(|K@W4MY6qr#fe3p4UT&7>-K{H7%#fJ&f&nq#xBwd)7tTvdXyK=19QOFVG1 zf&su2@tG3qoE6ZP>D9(Zl(a31iBONpz{THZ#8;kfoxkLW$IRua)6^G2kH3#qaBcYY zLFnxHX3Ooz@wHHvp5XQ*@~!}H=m`48y9y=o`5!gBpgp40bwl^@3~u)Qnrs}jJ@03y zR~a^Fwgkg>Cvwv?6W>VLb61Jb`R&0TmnrA;x>CC?xJ5ifD!!ywAbjns=L()LYPrz! z74uW?_z$o$(kp-XT3)_+Tvb$+0J*z-M=H6+-DNo`MHZ(xep$ncZn}uVq%=MkV0pIL zETSv{TdfLP6-a)CJa$g6&9)0A9a_1up3?GVZvC0d@=+=lu^G@?=~1Hq%#fr|HWkK= zP^HMKHh0UNl9klRB)4T%x$u<m>q(V^a7pv6<jO&e6-^nzolj56ZovBeXa-BKKk6#; z!KHwzzmbrpe#&&y&;C-}$?Mz>=X&wZ3eiW{krBs8T5KH0LfG~Qv;DaUe_lQoxu3e% z{Bnk(M^=%|!5e?Q;kJC}Id+}y`$f~GPKCEVoea2_TJy%tEusM>WE!uhXJwSSlhsx3 zepuo}E~y~}c{B8j1WKB(+!vq6{JfkjvJ<(*FZdy3nUIKDa1hl`jxE-Hx`+XMe`jhZ z0tv14{u2mRccy>Gg#Pbsysd9^k$X4QZH9<kZ^jwAHz6xCI@&Q-;ONA)@NlQ?X%f+I z+@l%lMi<dHgyPGh?xx#Zge~T$VgY8Jz$5^)fu?|Q!96doOm&E6M&y_JwKo1owGw<8 ziX)F-j)4~67L@yM!yw-eh}Z2(S1aUzPC%|I(5qA_znfX*)cd*AxMcxogbz0@>U(%6 zga!c9=ENPubfjdxo(!@@0Niu6f2)4IPZj6gE&a$6lI&oGpd3}-xKHi%fPpoeM4%o3 zxG^VtC3^+}y(^?>jFN~la{}4hj&+Vv87@T(3~+Y!WNYB9{ij|Nfup9h)FT;T;H7Az z&>k(ITFYuIc(=8)7w-+Xt&zQGG&@Qs#kRKu)sFY;<C|qK&fY=7T7W&Zp+}w|Pd-5& zv?}^a7&n-ERO3ABIq3VlV!_HIFXlw(Nb2Uax9)PwyfEe1VTR`G475gPR5Z`|-CO<j zp76!fgu_DH-x3G*^J(^cl2Gb8@GGK2&IcK45_5zyQJ2`?=WSe<<e`<95GW;gw<yFS z&=|59-T%WHrF|(*S#kvq#5g^l;~D>Sqk><97^OsUD76pjLK_&7mcXa9I6LB{=~*=_ z+zC2wL}@xjy<4Gb<9(^4h#o!Pwa^EDW^qoBA`R`W?y&}*5~5}P2|JhN<P_QHrU9g^ zTeCJyCYjB5*_^U6P?1Uu%Fli@9;v?A?zt)%jRSPJx*%{hUjr{w42sEqRFaL+rC2w) zORc1qSvC=Mrzf@{HcJb)TI|P9d++bK?w<7GzaxD23J8a4zT0aa*^#Q`-O;NsWz6LH zJbQ9C5gZHo`srev=;Tw0r{pa)DKT<I_3SvM^tvNAk(Xw+EqNk!|3jWO)2O<u?>{sa zpgZHg=~kL@7X(Yq9H&+2T0kd5xv-M9w64}y`Vz)s^5KW=-+r!pee-1*0f<g;w`_AT zOV6ksCNw^wzD;OI<o$Mg9FBzsw>@yrpG#fZU_^}1*U1V(%p6CraC$oT7_z`v*OJwb z;||4cSd!3=ucLrSl7Sf{s&#heexSzo?_AXv?*1jH;gi(0Q83q?EZ?*@)G=3C$gc-F zL*!3m8oOkf!Te&l5Nrf!L4e%>=minEq1f5(9doA+09mTocUSE{x<v&`O?xbF0aVxu z1$el+K#y8=2gi!10|8)<?BsXPO43(Mfm%Q|I%HwJ)`8E|J>X2<2Ek>3)O(3lnMwVW zDP$hCkO|3Zn0#^QGMWdu9m09^H=G3&4l$}*lm327+{=}z<BwC}lnNL0aP-o<cdYgW ztR!Xx8^I>66Z!Ui6@@?<+Kp$^qTr06-r_OV4nbm|woEHlec=OlU^96%%6Ps9QMw@0 zW0p}p#hl4Y3V%Es%<ObY2mc9b;cU%Jt8)#?hA8~z2}hV~Qi!I<v17;0kH*uBW2}#2 zanZI*HKQza-p2f-QE2sS!A3&*=hya@H$#A2?$VsZxFc&TNBv%&x8vD<v_i2lEnUy9 z@ERpAxD#c5nf)l@@SrK3moJ~6f~&Pd%P|L_<<}_V@98#8^GBVRllq|P^sKojs&3X` zX+l{#MZlG%xA#{9x;P(<nMXNKJsN`*X<B|q@!KIb>K}+q5TJ&zzD(8t62sxyi4un* zl4N@Yt;lhgdO@iTF_wgpbUQq}Tw;M1^vE9HziREg3Gv<E(X^(1G56K~g!AD;p{p0| zXbHFR4qCjkPtthLzTSOLsEqg(ujv+c^}{S3okX8UiY-Por5-hGl6qZf$v4=&CzQE! z@x{?naUbeZ<xju2=0SKK1+q4j@5u$7M5)H7{UX#_M3eMckRsOeWzdt>Av)ns5&BV* zsIU#3v1zbpY@`AJ&q!caGK%4Gw@D+L^-*;R>S16p<mz6XPlvY$kLuG>uSQk8Q-zeN zu72f|B-Cw9N8vUW2kB=n)67~}EH#285o;V1os!(snIHfuQZ%r>2O7RgCwI#>pyAAQ zqMq%xQFV+92|pyTDb%jYh+|K~pfmj6H!Kdwe{>|)6J!)&!JEbhyzNZSZUOmsrWWhn ztqT$<ItRge5lzYiO;*(I;h(uuo2mFmow`gv|IK1s^+{-Pu+*uQppJOrn1`)OPT#04 zYJ}lCuyH%I_HJM@%9`9OiB;6l$n{$;Ki5z)zF8NAtv*udD_2%{Zh(R&G4dN84~OGE zzDDar?aYd6c$Ej?2r8sZzcCl$Tlwj$Ub7huSnu+cs0eJ6j$%goUM1%hg5P&<{q>(M zRng1eP|;Sf?Qh2`{lI1B15W(p3bc*XW`5T~#oJb-eJZ7T$C^6Hy=S)I(mZrGDZMnz zxhoU1GK7s&+*-eRC^uHq*wlfqB6VA=p>F4qZ_-4P&E*RK)`;JFb+<FRKmWwzqWce2 zAWb$fCHwMHWa^n(hi}W9NLc`V=!J;UZ7k`gF3X#rH7(8t<<IHa>C87?py<iR)z)}e zXkX+DvRsM%H6q*ORBJLJ(qFR~;E~s7y@<LBAQezC0f{JM46LOtH#ErzrNW#faF7k( z?h_hfVB5<=sck4{@l7)dE&+XLz$wDtrrzC$9yno~Z!dwSX7u8}oflS=pFaFop8c;V zKDa+_FOp60)}JRiNBqFG106_mPS_{0VXFhWO9$p=3RJ4u_?=v8W3&!=>aP$gQtv!A zAzSvH(QP4}pqW)#m&unI+UZS+DJp6JM(~`&6NNGAtxZg8RllWoU+&pLCDN_|nYi+< ztX17lx0)eSBCsDNx{Og=R^L?$Xbm!{!2+^<@UH?*-9ZVI$REvKg+bl?;yztrQWpl7 z0hUat)R|C<@B@gnK0A@jH~t8X9cRRNkfM*QRAl!DqtUi>x2cq1+%xxR7t1=uva2<s z8r>HDyQqX4w#o?sh~9ctQe#ym>pNpo-Op+n`UE}<fIpm~_0=4#d&A0bd}#5}rlUsM zBfxl8zZ-JXFL`<uL|>lSB}i^F74g(}^sQn3p)5$p`S7}zM^&CyfOTi>C4!cxs<74= zxT|IZ7mUb(AP16)lD*Q05peii>Oupqb}YyHdAWs?IfCkf_c9)~5j6)5yERWQ(s2}_ zk@J_&>lL`lE_bueG*3r{Uagq&Ye2eDNl_6=>N(GZpYTpz<g1aU)wu}1e^GuU6ocr$ zHvdI-%2(VG{ni@cO|^zLrBtq;-dss5VqkhyeWmZn^2@Dap1S~-G&BTzva@j)UO)Cc zaILeIta&&4+Yt3tI(Z+&;1hI%=T2wjvP1`dk>9=UiOy}xP4C4cn*hGpC?F5;=jyp% zCi%t}VeAVpP;{$`lG|)tXj57A_z_fJ2_^ls;L@8ok?#>F;{o|h=XuBc7;J7SS)|JB zD4v->W!n0!7259Dhi3}G-S?szJ^7m-f*NdOWh*4Zle1+#B7$5XO5Hp`P*=1HwW7wD z>Y3^3Mu%4U;H#D`<=lmW+)@k$Sa%xd7ip5T#tWNC%+mW^<<*P?vM6KSVsQaCfRs#* z)tfi4LzMRP@_N-jgt<pGZGPa+Usi`dq4uD&mRN#SGJ_RK@lbO?j?{XM1=vvmw*^t* z*jheEwf~E_fjlj$$vZ2PeypAoxLRx+)`mF-nlh&<8uuqj-Z)TgMKkOAU6OdV(f6E1 zNH^chd=jeJ1X`N#)Slf`L`V+`!zz<=oMbf$$zy<q0jGeml7D=LV1Vxv@W-B3<**!T z414`~&xRST8PHS-#FK#~K~H?5VE_{g7SHmxTTt(X^`p!ExXyWM&eSsz0-z}nY_&hL z(nKmRdbh=Efzj7MVQ{ns%tmUma;AEN${{b4aG=?-V6*-nQ>VeAq20f|*c;7AK@Uil zRvWPbK)f=n9xj}^27NJvyo;!N=1XR-C>&aXXHXm=YkcCj=*Zsd<8|_AwLxj*Y(mXU zoFBSj>x3e;B-a(4__m+5uEClyHGA^9F91MbvGK(Q${jebA10h*jFGDg^@h+5yz06+ z!l`&TgE*d7fo=f)=_>E%^lW8JDS24pBF)&r*M*Rd!A{C3Q+PPXjy+)k4IM%w6A)+o zWNZ|2+E1%DpuGnnAW!X%_9@h}Ces!Q$>5&!ozyRSW&Behv6%!>U)<}p?De1FUsmc% ze&|<ZcT1HRcH^lszsDPj&j<@E+8kA4-^GhL$KQ0eg6De{o7c(~88--LPHE+Y@6U6% zHLNt-HgXD+{xkkYEe%0+!e1I6j&4m!|LAxwe+40S%}nm*T~@fUjo{+Vn~T`gq||2R z#(NH@H1%$MDrf#CWad9Vg&a#-bq5q8%htn(a(HPbXRDGt0Gmy&8-D_2H<Iq{@t#%d zR<MO!u4z=ulTf=ope1T|kHM)rb`ozau6A)^p|61mCpxz!ptrY4_h_SJ#5167<#4-8 z&!6_aJAR%{m2JNDtYb6#(Ey4Sbc5M6U+aqVfRoePDDg4;$4#<ALY_P8-2I27ed0je zUghH_3;$IlW`IpN$h-|bJqd+3-`VWZJ@vphAnMJ~?cvGKpxL({E^s<#?xaXgb(wkZ zvD5I23O<*JGLp9)cH}_KrC4G;QZwEq-L2C78PodOVKli1fu~PUKL&-oDMH}<(k;DO zPpOjVD%uXag&GzZ#UHAJWCYG-5<R-_eL7Vk<Uh$y&7!tcNx+w8qp$=Fu2D3;&-OP~ zfLq#AJlMaf%}>uD?p|9$|A>%^AoupF6Y)d%cBSF(nZaykHGp^-`x?tIp3kK2q^bLj zd{<W>Avl*T?V}Y`M*Z4j1T0mxi)OM4ZG4=G&Cx;R54MWa#|~ImZl^@DV9*J<AR%DH z2Xo{p^-E%(H=VS@d<uV`hgoGTWS8^CVS>=%*l=$3nmZs~7r)+9Ha_EStMlxP<eIf& zTgK=0ZKw2Bd%I$F(8^IPyxCyhtvkMf)jka4F<!(x?rKMJco5%dl?6({o)-}e`nUs3 ztPP9j2iu#iz)}M>NrVkit&yYjg!Ef{?4y+`61y2BCqpyQ>KqDvByg^$h@C@!kGGs? zxGydJ$7b}Fifd(tt+&1G94jqazq?!cMV=U^%exyU^B-2E5{l%WeLie-Y5^78r9Y+b z`Wo0N^Y1tV;AH<@VVz*go*n#Ia!~Tlxy0c)U=;?DZcfe-Ey~>>Z6x$`=W`>UNfLZ@ z=N(vx3f9Dh1rIOO;B)74O-=W3(V9Q?)qkD)!uPduBo0;feP+M&SAc1j^Sn9Gp$hj7 zOW*mH)@lA+D=ej_5Oq}6h=w=5FqY7hUqRz`{PHl~^~>zcL%w@edQ+wf9?FjP1CI=7 z{2q<{D}dm*-?47WRLFB&&^s{U9v@G&z1JF0W+MT)Rf!QOB_<=7+-V^AR8*(KW)JRj zNhj=>Ygq$od~TNH19R)r+@9iwkKllbRL-_$TxfekE@ix?nY+1;B1t3^3aLIF9~v_* zs4qs*-4MTw^nAzmkKS#wK5qL%Yt}m6fAdM_S7}Fg_^JzJ+Q1_=W^F#*!_Nb@u6f=x zdwGUMDMl^LXF>1D6?@oHIjQ4U6F<_eBe%-!XWzY?;UK#dW)63r!B)=;LO1$jCS}M? z4&}_P58`#rrPq8b!Vmm9(9r;+a3E%`lkO(Ke<%;1AiTF1gjDd0<^g6UqiB^MpCTjF zodxG&(klG#<r#$CYB{oSNx`x)OEG?DJi#{H0>oQu%C1y9Y+SAgGuZrb`xSJFS-v2l z=<1DAS$Zq+z@aRVt-B{p-dHPT{SPf9{8ICtObY2K<Acy8iSo|qz-`9c0;N+N$1Vxy zc$D-ak+q*YhbGpB`z*b|9z(sDC!+=FpN;9s0AG-K*iz(c0j273{iTEd5l4T$Jad9$ z+ryUyf$9F(ZV16Q{-k~(B!yeGws=a&!VC2Rq+x}3X32{6eeX+IPTb9>I~YL|2w4_f zB41VltJJQ%$Km7ul2>e@=g$N5RT}?Dn8U%Rd;XAHxA%v>3#n?Ak;l$A^gK@zx7Vt6 z01a*W$LreY9Z%DhPYtvG($#KJkdQt1G{F!$j(YyQQ6fzZs-9(S(B@=&y)e^^FF@L4 zd!Q%Ve>!4}b(~scZ|KEEs<2;q7I^?%v9>dPy7{*Q;Z$eU(#X@?Xe_XveW~p&A5U+j zWVUEN+PA%B2;)c3*lfDswGGauXV3>yQP*<nxmQr-bfDlKJf7nSawb*P08I$`z6=E5 zuLM|cE-#_{u}|g&7Vy{8h^S!7X&g`z;fq8asJo?+B0x>`Z^x=r%n<E@K^x7^@srRr zx_eWfwcieia|}uDlEnQgGUjg=Ut=u~8rCQJ!JKE1+piQ9#|8wv$+S$`fKKF_MRQvC z$(;o_vrrYF2Dqx>8~r*BhQL^MRFfc&qXwlS773jHHLOOAzEy9g&uDbiW~qTDXfvT0 z8-Cp!;kDySu6R3F*)0PGbV4m5f!A?9#?+y5*w9>yw!Jgc>%}clOz(rTYcjobU*0xN zUA%&xtdJfuiGe47js9Y2%5GNf3-9e{I{JzmcL(XD*Pued@L>d5eeX`p*(;xBR~GxO z%mv&-P1QY1vv$Z2hOcy>_V~Aa^5?i;Jq=f{(acHG9KY%90>AVZiMx9ohP}{?QM(}W z(1g-mfJ#wY&MP7&QMnozDlvy_wvBZ}c>HGCO5z4^zm<(*^MhUJx%oRwdaU9I1}8kI z&bZHFV!bL!)=`#`!nCyas2gWZz-=*4JaCY_=R{Ppr2YFBkvET>Ilrg+xs-Jut>?<k zun`5EwkGj?QI6>U;<|qQ5UGybAQ+#(p4%tbM`LF-kfAA@9Jo6<mpRFIb!MH}c0=)A z;~lZ`{5#bzrG+dVE-yB8m=u{O5-NS?J08CWazg>qHA2edt7LC*$#{7e$1*y?adeY( z>()%+>C~S#{!@D)-}avNk>ke*h^*THURnaj0VkH{O)X11ZQMjJe*I@Cm-+99WPO0? zL~O`qKfwPDL4%3=+AWsKYE1i$JWG6vF=Q+yXC6ixs&D7erxg3VA$Xr#!`IiRVo3`i z$glIsX|eg0fYKU?Xj*aW56mYeoEmFyYWZ-k$W^Cr^ZRc%yfPk2!t(5^7M-q1%`M(1 zz5?25ktg!&kfxbbX6Q#ho|9=1#vbB1npx~-f}V_*2!D8cD3{=GpgmPH(*5BQ9azh} ziwK`<jsn%dw^G#h5~Cb*paZ<n=L=A|MFQ^6(Q7lwO*4yy*?uG#wIP?UoI2_(1lgJj z7jM_*p?V6?@y65QwYR|w(u$T|N?CD^y7c;^jIYs2t~MjEh~RS+^@TuD#{RjP3??1c ztrm49!`M`QKV3=<ymV93!1_Z57m$~}8UTg%(dOpll(^WHMu`KVA4DcSKp4IkJJf8g zA+E7d;P5(j;!LL^le?DfQpxSAJ?|RMZa_kehMfm2p^JCjcZ|w=OUuB_L0M+M)i9#D zU(Q3Z#@gaHx2rlPenfW+eQMMvyPVk_ohcRe9Uu7i7bPE`5K-Z?BZj@e{4SL+yzSlH zxs}gI80mI1&SrhZZ&!L+*co;O`lXNu%6|2Wf1M|(Hcg;P_#?6b-1b^|k1jRZ2B#Qq zhyR?IITt9oX6xOXl5?O6H@_#LgWx;8^SE>DrHM!1NYIKEi)PN;%ofri<IjnDF^|RR zOqcXK{|~PFx?c>aMvkj^`S5e(zWEAV@rlk^QLa&pPJc^G<%epqNi(_ZXGs<|2FKql zwcwHtzni+2;_&gX<L8qBeTeeb<}<NL^}3Y?sa@?hlX{EJLj%fljpYqh25OCdj&8`g z6ynxQEM!+(MMvw!$9kY>PU7Z)gi1lZ*FZJ?xdGtWv0H01l5;aE#}$`%j;bzC0&XV{ z5wi(;!!d++_A!eNNk?L(A=|IUpQCeCMd*gG)h2`a`PC86s#0uics7PrnbELXq=Tg` zWRn2>l?npz1$#PPY+%t_S~aJG(*adCjr-SlF%}+{7CC1_hxN16lr2(uT{l(Use#;b ze@*Ya#AK3WGSu&Df`C=3SU_CJ>^dlRl(~R_d!`DmJ)r*9YSwpEQgQ`|1VRryGdQaf z)a<*t=|0RJbmaF-hwHn`=;*ew5uX8}LbB@boR|BOn*L_KY|DA#cH7<!R1~!-`S4PE z{E4G~YXN5MBeL^Z6mbN`CI|`e1@ac{w@zEnUdIQ6Q_u29BtGd3&Zwl=yS2YEQI@5C zIAT=r5Yz{qysad7NYOeRWneL9KT2JYU9ykwy5G)hIz;XoF8%h|)y3qnZZ^K6t%cn~ zv?;&DqHi~j<8A}7K8#P@I=j%5rZpUQtubS7XxT;Kn<2~Wdf0WuGVCZuf5;85{k*p& z^+?Ibubv1YKglG<*jeY2tP=-3uh315AO^uH9oNL58;;=(jO%Sz&iVQu;6CtECMTuG zGGdR}K$)y6c7^*jn{B&G9T2A;&CzFwttz)G)o^XWN2n+T><V=o)lZo5us`CfC*a0p z;4b)f<3D-lIh3FmAN3&zU|Wb}rZo%qeqGTF(X;S$nTvz`G{dBA&;Q_w-$on`H;4lf zA1sgkKF(2*?zYgIjuFjuekU5Cc~7$zyS3tYG3w3P$cGXl#>oyD>7fJY<UZ{*`LKk; zNhZ)<fXAP;m$ojva@zERPinfCoE5(jo+`ARt{Q|d#U{p5)V)wdrKwju8EOvmif9>J zXGYON7uBt&su>am3&?HQJ#qmB4DMk*zoL5v4{bf#BiKGaSSneKxiu5z*y;A!AFWPZ zcKBg!7a{hskO_cHPfNENlZPj>Q0t+@Ni8Sk;{mrWt1spy?)z1-{$Qc5lQdDk9w_B> zOA`ik6*8=XZ0KW{`n!o2k<C(y{W#Y00B2u0&$gH-aU3s*{(AmnjoMSK@|RetF%%18 z<%LS59f>V6F0A)}5-f?Q0F#0c>IG(b-PCnRYf%+e9`66qTelBr!WpD2049zN<P3#j zvmXL}OSg!f`KD#PlUAt#jE@A8pY8T93g_5}U6gc&LqkgL=iqza<JZfZC>Dre3)f8y z4FDvNR}P);&Iz9AR-Ew9-L2JfXAV>KajB=_5Qt7lS;D5+U=4a0$|7a?Su!l4_pX(4 zH)U^ujRisdDZ$U1Ot(<aQjp}Px%Y(uDGov@fg2+WvQ}#L!Gn#orkj)u12tLCOfInw ztb#xwv9T`0!W3iuzR7$kfu<U)2hgO-Gk<7OQx1J^uPqezmm+xn-mJZ(XV;CKsQuOP z(WOygT95v-$@#Cc=1l0G#*DZ3)HuXRLe{r61(9GK^naB+f1*q@fz3QlXSTZEW&hiQ z99P*#MSNopo;m!n_(1h};f1b8XYX9H%kQ{jGX2iv(qP&p@azl_Qobj0H^g;Xc(Rlk zl$|Ec)bCcQYtSn4@H&aUIwnr?q%YSS4_KH#Y4svzj>EsZAv?o}0<8LvLGx+O*q7X( z-}4i+oS@%X<2Vu=Y-ptJJpfD<k+cmmuRQtAhEyq;1vK~hz|hh}sud#R(m;X3V?lYR z9=o}$z$FO;i%Eav^d9HS(!#BU3;dJ-&&KQ2>&Z6xXf~+!>@)_o2WcdzQGFA<a8|<+ z82;y9LRIJW@Jn~zc;`hf2d-?5Gy(PkICid$Xl1qQ6aMy)Ha=$`2QOvq3DmTX1{hyA zD?<IL4*AvB`5|0z%(mPiJmzj6>cyPtnFUZ#+iRj$x<T}?In|RbVs39N8f-hlz;Sk! zRS)Q99)4^x#{xjFZ{D=o@Uw^ppz?RGP~ldc3szhx39YQnxFAyFaM=cExt^Bf>8se7 zTUfXCai0(%_7Udmkl+PV?<AF5zu`m0HbX$<S_(n3StDWn>tYwR&loCiyc9B;E|jJz zEu81B@Jia>nBtlR{4L~)U+IBO@!(BCxhK!F>Pph|tfx3H7)`fk8z(f4#+tXi2+7dO z{L@6r<9V78wF#*)D#q@)girPa3RcnV5Qpyx%k2r@z?=Eu;!$%Qg&q6<A(jB_c`PpC zf6~$}BuxV1C;X&hG{BQYvT?{wT`}69bg)BAB(?9WWTVlAvCote#!or3$PL1RZ(esh zuS=tLeN-tIjw`+&23hbWX5>=%D)X!8#0J1jP33c<Il`cCQ($-W131gb@ykr<hUH2F zxk+8#sn>^2*Jm>~3Nxi@eB0qHD!jqO)I1*?>F|<Hgfy#8ZbZ12lKN1ys+m%q65$H2 zbq+13*p>X=9A++d!-oSTO&rQCz`)ULiwh)_E@T19&#tLA9dux%d^juj(4huCd8du$ z3vNR2<#=K{=J3gSUu2Xls!ntdLG;$-glFplwt&Erx2%w1)ezuIFcQ=8D~L*MW;G4! zbbc2Q-0ipP#9_kUPQ`@t*71NEZN$VQkWeZP!2{VHS;y)(+kWJk8E8&rk(20CO|f+( zYbuo5ZCU0O#cR5rvQK*(aQke=b>>KimyFX{)CZ6*_qBI%>u^dH>fI;RKM~M=SqKp? ztvE}XGY?SL$#c^KrMTv!g)=TQAlus;(~QzFp3n2r*ec_)(I*Yap6{q6oA7Hd-6jn7 z%@+7}(RX3I_kosKd)d?7kPB7LXrfwDs}Py*4yfD<h=<(pkUJeSZH;~#hu_5i1EB$= zmcz>IduGq;7`*(SihvzxbEKWwjft0R9AwY?h5Cm=Pq3Ri+nuCcq5d3|HP$Y0PmH}~ z#*6F~k!f#BfB(QNYFq;QJ)ojv4E45Wedl$+fVGs2kB|=qYw3<`T>*+O<cB0*R7V;x zVw*4yG!nME^S_iYjbHXK|G{8>vvuD5K%6zNazb))t9@Z50jkZ>OekfvUu3tZX9?XL z;vX+NYpW}zM(z-GPt?C&^n7M8b%J=^#(wU?@%p=4tna5$lYMIIJ#A}w8OIW1#fFE2 zd!vZeKPa9W+x@Z6ij|kY{J`1-t<0qW&Z&I?CY4SdS85A&X;h-`t8sX?p-kIdIYB)s zQ^&)o9db7=jGrk#_+_3tQ!Yp<Yg~SD@+2|YKGj!N>htMn`%|U0W6y}YtXL3W;qx<K z9ML?vrh`50R~~a9e)GrSo_eMgP3a=Mu*F@myB!&X5G+rPHO_~tLr}ORbwjlIy8z`H zmYWl3*jL}|qHGZO?TvS##(g(l>5UmF9oVfTlQiqfm(&#~jQv3}q5lgafIIWANBb(r z{*IQte}DELH&V+0sOhv~GWG!n#e#}CbatN=^v-;?%V5DbE)MHkUoQ*09+wg4%oS?n z{b{6K@GU7e->^vJ(WNJI9qkf)57W1=W{@j-63;2~Y1n_8xG(?^ht>iIkTN{LWTYG5 z2}Jyz8|$~*PTVfOb7~C+Ox@H2O;$I}LGg^hf}lmDGZs*gxvrSPm65mKtkOO|YWmEo z@d%#bC(38j?p3=Z``+YOL6XWccU%5$fDk2F|6IUQfIErXZAgi7WhOvsSgfI<TFlP9 z^Zbdbo*Ipt`v6>$LH`*JnuRe4fnHXXG(q^z8Znvh92YR6IkIrZvXfWZJlDP*fZ{)o zM*>n^5)4pJHtcc(-tYL`s^b%p-0i~6-4<4kJwiBu@UrRo{FzgZ+d0!W#S*AU>B9_- z%7repB#8BeKFjslei|U-!Io~PVh`w1w(c_xI?ZGlQoSj~qe*Y7pDY(zt=j=^Cr;l- zcGQ*L8Z$Hx6jd#Lv7zd|dkwwZJ8>@P@j(vn!Jc){Zj1XGhkjc0kCF}}NwU{j%f5e* zdi;we`WXe}=OC|PKKcK-sX2#syN`!1hRSH>wC``(GaIlzzj<bFnCglBW_ui@;}i~5 zUENn79QnpXEIr@h>+@%}6NUmWiQ!362O@mIGAFF>nf397pI-BBamE*B_}MuIw{<2d z7f;7s&Wp&QA<@u2D769a=EzsNdjxLZjK3b<sjrBerHt6evNLGp)aK<0fb6aZERDwm zJioHfGl6zt7V(W}jHjM-%`ZFOHy7w67L$~`3D|CD`vN>s&<4YZW_B4$?i9!EYO}j2 zf8(*C7}Z-K%Bee0;It@@$ld=##M3ZK4cQz##5{kMLK=s5sIO~V@eT_pdG}en|3JL8 zl?w7YW!c$?y^Y;HcHNhL6=k|VRQI10*Pz|i$#Y+is}UK^zBNgyI71v|kv~dblp=dA zv$LCweo_eU7ZD<Cxfz2DaP7Vsgy)<3R}1M)eRVs}TkDygrTL(1-!UIBv9LckOp5&H zhB@utFnjV<ZYS6IF24VGu>Q}>z%(`vZ&@_RKK$R$<lpZ&EE>8Rri2{TcfL3`k9d-C z?tR$TOBZum_sQ+NosJNk(uz8u<cql_P0ovKIQyV1Sjw^Kd0y*>b6yV-wh0kN-OWfq zn{_2w{%EFxiXy?sKsD=;)J)#3V9KVsc0F))$1~xh5Nxb_-LN{fu(71BJiYFHIx~w} zFMvlBrOP>Xu~jIhc1qZ?WFf;-LEL6*r)~T{j05;sr5q1Nt7pD-=LXbZ<i0Dg<h#T; zEhl6qWJJZ{_BL%aS7Qd|y`oVy6($~dK&5v0X8#ok)@WfjeU>#dx7it%fch8*pmEg@ zH-9_#wspy}x{p7uE+}nHzEpithaQo1Os|a5*!rDFB<R3?^#;`iqHL7?%&EOXIKpt| zs^Qhfm?w!>vg9sy`mcyB7nCf-s*&HQvUb*S04w?4$ahZJwY<r1fs_l7L0%Pyv4QAe zf`;PmEqx1)4F23r;}SQcmKnugSELk|J*tcWSD_y{f@(Y3*H6!>`)KJ+OMLe?d7Kh+ zV`+0GeCD>M%$VGz!vN^{s?gawMIS8C2a|y|RVwEn=JlUZlJNaoZ0u<x|1Z`4eOomj zkL>TsQf~l*GtPNerSBBNPmf0K{v^%uJr1N%K$ME?tabbH3v)KqiXEvtdw21Ea);@o z;ne<0aOT62U)*@OvaHGX1aOn~#S`h(*~YceQq^MSh0=}R<s=4dFVoM(_r(USoMI7~ z%#kKy?YKLbn?XE4CFo>(M=_P~O&dExIN<PkMh#PJ{8n1ob=m#<s{=$<ifyO#T38Vg z=i1~|@wTQOjyzG8!HSkF2d4@&qeGtPwpnMI(^^m4`lQ;zkgC8CIQGR}FBgjz+z=)x zHGLapjTJ{Q6sg;DDQ~CO8Np2SA*@7$`?8JMMzEL@&8h(aat&fBW*==EkIYm{@G`eC z&Go=M&`)6I$DSAq-^rzS+cKwj+o1+w_MPN&f?NVVajLU%TF(k6E<e`Q=u;s`iOYEp zRV;ljuBSGUpzfq5s$T*_N$qosjsRLuCU`8<^ZSA1m2>{T7O*8vpfOut-Q|a!NPSoB z2|RHA;Jn%eFlp_cq<H9;CI6+OLSU$f1%<5iB9PfQFH5(QEtL!IisR4a5w!XA21S(D z<GCqL{*9O1#AX%@F=T2f)~anu_Bu!(dC8Lp90qQ2a&BGiyzKR}^*m*pC;bBDq`BX& z&TVejLvCc%Ss@G;Vs4aYk0|i7D?92;pWCINymo81y7n%}mtd0awSRRK|0#ep4+7xo zq;AGvt@3}z&L8Do&hd9jUUqPt!wtb}Jr8%|62R|oG-E=R^FN-W4d-5pkM#}G=WrAV zEZ*-Dzwq5F)!{>pL-a5ZZiXq5t`*8(-uReQroAoHo!5(qwTkG?W_6FFNHXfgM9whk zW;q<!M6f1BhE2@GY7n)5?aa-f>osEdNEe&e6x_FGO>ns^>if3m1y8UHB3EiXFmqMu z2Qde<QaEP~x;ca-IUFMw+4|znfQTPDQx${rpNa0@mTg#;gP>nrCC<JC4X{xAn*rO1 zC;;5eU8X-g+hDY22ygCStY(E{cLKQUYEq%KiBr2mr>dZC%ZX|2*)kRHsu*A}LIbip z=71|jOf1>>*kk!Gddz|<2MyfmkkTQ%wr+4|Cx*Fk13}_aMAx6V_x@U^fbXvd0(>Jv zfw*6t?aUn}Ga5QtN`x(3P$j=aH~to64(holJKA_B+Nc)%Fk}K;#SnHtPu6-|hw(Mc ztc?ydO>xj!(aoz*I#q-$b0rAZ)e1Mh9f0k~`4#EKJd7Xp?j210P>x8TDd{|Rx97%i z^LKVOmI$4dgFk@{&$CxFz6F2RaAf{??0)%WNJh;qFS*?V9me77y#{2bsoC?-$>kbO zJ@5KEgL~Njsj&JxJnozi3CXw_0si~*|41!ot^ur&;I-<%gCR-ta|reV`PY$VnJ6a{ zqzdg}SBpbkJ;>}<$G5mxW3k(Ou{HRt*SEe7I_mC`!*d<_arw6XOi^NDzJ~I$7PEOf zn}EYG?aSGpP=I?d*tDKWDmknXb@)u;b&T26Eud9Bj@&(Yy%1^jtw-I{d$0SDGAeKD z{1Q<0d#;lcOh%?+XCJ98W~H;<Gyrc@{a&#%?sG11AkDoM;X&Y4W4;Mm%#u_sBmv02 zzvM?n079)=cs9FgzL*l51T9(0_%*UNM@n1Y3!%MszW!mw;rIm5sl>JMdn^Mzr$IQ% zMp42(c2m}|(L~@E_w!G#wigy8%ip$ZpJ|sk;fCmF7c&q3p)PA;aTF7?dLm2$yP@{X z%fcqLV)oaQfXA<F$}1;Sn`J%w>M5H)P8eaboF*4?+_C#RqQ`l}cdMN<n!;Gs8Y`)R zipFy8S^F3J|9>*Q_W@zyg%dNu|3fje?XeICIOgGaqw{~j*xSbdt>O%LPenz4(zZ*C zd|&xAY^(A4$w)nYvdhplGx-M=E#~-e6{jM;2)#M`?#3f2!5NYRt!ET^;e&J)UgKAq z(qHc01JTygv3%hxSA}~nxB{dYlhS3a87t;%u<^d1FbTkBu+YvV7mmkJExv}(xLq@Y zJz<~X2ZyPYXL!tt>SQby0A5{{TOQWn!282nvhlG5?e?UtgX1WGlFoReI8`zH>j;*p z3Ln{JvE5&vGCi{jD+@jswerg>YHva^)GRe-<|O;|T593jA0jl2v@mXd=N9yYXXG9k z-!M${FYgLh9rtk~oil|+nDj5uX6M3#E$_|4wL*ij!)Nk@e1jN&$o7FEZf#_V-XU>7 zov*I_NBLI-$V<F{)0J>u_b||U{f_|u@0Vl3AtBg(x0e3<tAC!0oHv@*ybeW5NoA2| zbFch`fpfZdg?19=dkA6g-rq9DJP^+==(>*r74PTkDwH5+cvL;KOA7Rv|B1b-M?2{v zo1yss@pawtRPX;kQc;SCC|TJvdykUH&fa^(G2$56lt^SdHc7}jRveos$0p?1%H|k3 z_Wr$3_uk+Aes6v6f9G*f&dX~&U(eU`_4*iDH}m?^r(~~2)eq9~CR4FaJ1K41-%uSm z?00`9eWNGy)M(zneQI(=@?;~D2;ZaqMAU70aFMqJf5NN#)ASlp(yd!2c!NP>h^2?; z+_2TeWUgPQ!*)-_`0PfvGR)eaGxQV)`bLXl1T@TZ?Iun^)^#J2+<Q~IZ`0`J|J2hH zG50`YqP5`owyZr~uI$O>!-O*GeCz&xze28CNQ__XOM{q?BM8Pr#rENr`tnWhV0p1| zjI;1gT-8^oP5s*C9G_IN;8X}RuFIxh&rbjvBn>cP{|1<PM@JIS{Ki}VN-TG-N=plx zWfA{jz5g0NJCNd+(}u-y2o1K=3AKObjjx!vu>AZF=<0elP!y+H<(=|Ut*m8vTdtOG z>8mKOaZEWdU*Wo3{HH{=R@*=}psiTVt<0)^VP#1OtMpA1jkQ%FSDhl_WEXs$yX3p1 z0sl0=&}V}ae{!_k^JiCB)eY*1auTmq6WWWTfk;_N!(s=FJv^8*e98_xbQ)sHu5~Yj zn^?sF3G1baE!NnWS&Lh>{f}c^Z=iJ}q;;>im~jY}7Qc#)+l#4O9#XQq1uG3X<{Ck8 zdb`_S;iPx0a>|JS&MaiaH=9Y|GTVkUTUn5-%ha5{Y{4NgZ;rYQyx4GeOT35=?o{lc z4`Dr;GrLf2@dK@PqWqnPG|J;-me8aBDqpuZ(>W})8ZP*=VQXZRm4tO6ss6Vs1Dmts z;90VZbo>Sk^sxorxl|^PbQwsMYI`=$1}X^F)PY<~YRt6WbY1jU_3u|E+FM?Y!t)5l zr;JsVeXk&*!1%zn1!Wdp#ZjkFZrxh-lyaa~R)J7&UdpPDbzzZ5)5@_Pt3=`g1=+Mp z$Hce70CceA-bRR}@`pm*xw4~mAY)HF%uSo~^XYue`aQ{~M@%q%?Zhq1VM2d>w4?s` zawkJOQoY(l3<G_2Mwd*idahtzSj)VKE#rom+cR%F8UVBl{f9y>=<cixAH8r+CAq-k zr$yw_=Kxcvbr{kjIn176iH~#(SD_eVHqKoc@#{XU=d|j{7tHMES%laqbhYnWN~<(5 zJfX*<?aRVz_EonLgoOw%hlpIH_e{+4Y3%sdO34B$rQyBEvfr5fg96|+il*Q3{`7x} zOuT%&+41mFuRZeUZu*QNIz$H$+=Azjz!bv!gB_)`&TRTBlHJ<Yb<HtPbLK@j80C_` z&px?E$<fNSVjz5h)#Y)Ah*l|kF}fja1&eTr@L#YO*Yzu>iB_Dth4nc^3ufQ#U0nCy zGucI>xm*&Ay=)cUUlFze9F#M5!aGqrxy`kkEL1q}xA#ds4XQrhqmyGHlxwK9B2Sn@ z<>s70e;)=nFln&sez$eE_*m1u)X!jdwZ9PUm(JmAe=9^5Q~WA7F(5^h=zBw&hDyg3 zgb#=B4<0t(-u8pSnbpa2zK79sqhEr3R?v^V`kqGsSQ&Q<20%ohF{4_-i0-V#F|nLk zkk5EQbaU0OQ-Qxs_(AmyPPK2&ApWliB_J9QzpuZsod5ZytXI-XP1W;oPd$>D^XBc- zl%&eg&yOZ9{Dh)Dwb>+DgIQy}@m2i0X;aDMZ!@d(n#E_o`Kv~6EYHrcC2`1B_S{k( zIFbTQfn(Nd!5e3GvC9<(09}d|6M;d~>I)b2fU4ALgU6=B8qA~%bfO!LX{T$Tp*@iV zAhXO7>@m5=GAX-Ym!rVKA{-Dn+vCHf7KgoGD4R4GP0`5%cQ|p)!UK!>WLr6n5XwBT z{tHV^jziFb4+a%{ix&jnt<jhnDWjQ(Dz`t9PB74*opV<_O3~RXP|3|FDz_Y7*1|44 zSgcpsdAWaS^V8gL&F+k&zJ{AK!}{DOZd{{Ln{!x(#w${qg$7cc7$US~EPluQe<xfS z<uf>*??VTuf0J-4@Y<d!1^gsnfo2Tfi9UnhDtQbYC!&|6Dj*Y9$71#2vE9`uADTkm zMA`I9V}zeeuRW~%Fx61EkbavB;#m>oX-|mJe<iQ;xlPeoi&sxWdt`j$UDxR)?z(T< zCcfJ^2CaY2UN><t&I&Xigws2^Jl-fcI+$x97Hp%1wPwrVA`?VLa(1=q4X^IL+7lbb zICmU%Qn(-;<J}A9>lx5q?W2f@+v9xd^EF)aMI-(#)t&8i>{i>tLzJlNog!{QR4#BK z#?2fgr?=0-?m62-UtuDLNV<gsX5Z@Nd9m@kG*h;vj?o{>QBq`le)wC0fjJlWKF!XE zK<BFf<G3v-{NQit%U@OM@8VxX0(5s|%<;d;plS+0cgHK#A5W4a{?odyVn8M(zJL+A zb(L+mt$UhvI|P#``xbNj;Z+nX#+EEhcgI>p$cAdj?@-b+n6=qSH;3QbD!jf9)}O$% zU!}kVA-Mt_Mpt9NHO0eZ2z&`UJow!F9k7z&hu<6zxtNG^+Hx5&&RWOH_RB*bWbc11 zO}VYQ?6)ZDg4dj1I$uI`n0iA@*@3^F#^4VOTTwJTZZLwb0pT1UnQg>5obai(;iN73 zdZ-|Cu%d;D6#u<eI=OtGshlcy3)q1=6272v0KT=QTFJ;;%UiBPfSH3G7qcHiPrKkZ zgvFtzt|V>ok+P!cB2XsG#v<W`2j_kb+dugxQ(XXGiP-&a{7OJc(>{2^b*XGzjhTe^ z!XLhA`hf)hoF%$~TPQ5|2O%SWSitPUXSR(>IvV7sqCt)iLw+GMJ<-Mdy9AxHHj{*? zXR^|9qUIRWSf!DzhQXAM$;g%r?#B9(e#XDF074I+Vi{JfDvVs_M<9_;lS9e+QX1+$ zt<q8!wZB4b%}3C(aMIgVxvS5$59`hfKnPW-PkV1@4C}1&<sGUHc(VHT9hxi;U(k^3 zF)IcW)k9^pTn8*%W}R~LI;<T$(b?tP_>;Axn%@2JH%RC8XOc{;xL#E@`vM8I3riGa zfx>QtppXB4k|B2fmUs0A8oALYf!`+L+*u7|&x?$GaU)N2)7)L?Sq@tccBdB^+uDBh z!N2mP^c!VBsXO$)<N8g!>(haDK>Ln5!B1m`tKmA}w=rVgl83mQV~8A24(+G!U9yFD zG$bX*m*0OA<M-}WL`|<5<&;jCINP>9<rUBlbWHMn(eu}sZ3k;^1K7?wtZ2ty&r{3e zhmPYaeo&tJJ4YQh#W@p!1(@#3q72Pw?$li>&wd2t^{M*#Y_#@HUcT<bv<scXEyp~- z@v`%_K3*8wzCCx(FK_0thTml%91_sIaVTnWd?dS%ai(aj_D&~nI;Ku9^K+|qYmJwv zdRqieWohgd<{vwzAXE+QL%R}rvSPo){-m&qiC-RbF7n+BbLE&yvfEGeEz_ZY?tJgR zFy#N63fpH%0&BG((_z1<Jn!k=lzX<lb_WN0y-zQ@Tkg=h{<OO(KPd}Y6ep(ideR5s zIQB>TA_Q2atVV|cqZ%t3dFz!y9&g&<_ZQ9AHUz>yrn>u#lqh|raG9QHzB9C&v%z0q zyuc@$GXPygrjUq%TS6adf#&8st>|G2sy5WguOQa`t@1s?L=oYNMy(_djbY91JlXpd z)^$U^CHPfdJ@;Dk2o>gLYJAx$hyaRQ6|^#g8??YgBEQ<`9S%7Y2K#XE%$A?#!0*;O zb@nPl>K&)nK0%xPGST?^*{!MY788%7Jq_fW&(7$Ud*)y?Nmz~Q(OpqZeHoAn?Org% zhVRw)o%VnBo-<zrPQ%at&t^y(0g~U^P*SGJf134YZVaSmCZ5z4vgt|v?D!#eO7{Nz zpD7c7Q%2k}Ez3)Mrj?8|3-Xqk;t3j+{AUY3ML9~z`h0cHE%#7p)QC_!sFXJ`6C;my z2O{M04c!B-87VBsO2C|Y=gdCWVYds!(cjd{+R%j7UP$XC7Q?f|?J-^|9E0LGZsW`y z8`3jnj#Lf(wj4f{ht71J7%J_#`V0!jIw-aP{n>@#72P3K`wUe|q-i^naU_HjtcpkE zUyz?IG?KbNzox+P4)zu5vHt9+u>1ST!dS(M$q*zrLXVRx1ar&9Xw<Ju4}@nM69`Y? zqeHa^ox5*ud-p6!{E?`B<8gt}_d%Uy=qC65xi<J;8F7mOr;K)YIEW9JAN=SdF+NRm zG~1M}Y^5Nt@Il@HeT*#a&bDrcK3|DKihfXstwD#~n#gn+tbwy%r^=86*Sv6o$+>*V zx0_EZhdnxA9c=6&fDoH-SfaI<x83*Trcf=Xw$WP`8&+E(CMQ^z#nM6i-5ku7z{do< zI&X;h(sMq^8HK;&wKtGd=^noUK^-)m|E?C}HqmQg0Fkny@;V}gEqOnYVXM8f=32N{ z&u?>DA^GHXcw^6cQc~ntRAkX%BsX*6Ja^Z+H-!+`CfOdPw01zDdeo#Jl+3j7;L(Do zdGT8bK9Hndz1p<*zZ00N6hMe;8MnQDMS6>RSnbhBp3OI`uX!a0Xi4wT8$<*R2j8D# zR_sK=B=@{Htj1y@p`)-9yHGZ{)Pc7qq0;k7diN{Y9(`avo4@WIZ=c2%1=})mBId69 z@eDhe&GK#qr1DCRY6i7$Z!^bAc|?NdhTst8EjbZho6rEt87H#E?Zo71xt?2HT}XBR zSMmji(H(UT`@{5pAsZH<7HDDD@M1o=#QxLfVV}@ShMiiv%~Gq$=8A!R35`_vXp76P z!^HJ{k%fcn9Y`}hyxH^<{H||weNa4emr}ny%_?S<6v3j{{ltHz^Y6rg!CFX+pu=xa zMif>6JJQAi8oxne;!^0&%19bvIiF3Bs|%F-0$$DQ0NJ0Gk{-+Pv5)^)NCNB9i~q4? zqH1Qgiik`iSsJrR=4_{BJ1*f34}GgVozqjt+Hd`IR>3~=C5+dmRFG)Yf*C7CAzbW@ zW7QZRuZCWzGwye^4+no}*Ph-NN2zQ$XirtL#|86>#=TeZ3#Q+(`nlHW)7<F0X2I%< zk5cX!wpbP1&R!ce@1`n1e<(bTabA?TEz4hgvvE=k$~YS1TdVojai(So9p%P;{LrW| zd-_90<dep-Po-{G)OU>WW*rYT75qWl@DW^27}522`aXsNg?&5J>o=+UZ*2BI2h2uT zn;A(+xL(1%9nthDl=M2f+>$7U;~o|9A`@;kZka6gMMfjVTj?XFujJh}Nl3rB+*gA% z`euwqHKddg2z|DC)kRito5@x%G)0*wre$*XVHkpZe;Bb}faZ2HIc$!8WcYV;tBOcr zJAHUgi!gckYj;);RXTn1vfrMUMz+-i5#%VL4XMS`f6bs_3iJZ{s_-KN`qkEXNyKrE z=<IRawg&GNiFL3LMQWolBe2rgbHuUzZcD<wzFn_YT{1kwFjRCFbuF^gbHHlWMnrvR z9i%fDkM3^7AX<xMc0;dzDfSHZ1jX%W=_*va2MH#Ce_!axK&>^zN1VhaN%A+>DS#vW zWFaMh&&l-~@{;@lCYs)3<*SdxDhLU2WfW-LZ*!}+M5*AsW+Z958s)|$oAF2}o08Kw zTfw{3tB5{kP>Vi8`(aexy(IEXmsz3SrshI6XpYu&Y{O$u0XHggf>PT$@=)bn2_1&5 z&2J0dV?yE!K=xMjcrgQ9B5^!iMXdX)BG|ce+WcLNwX9c3_J_;-nhO!Iz0wctg!$8N zSS6eH`@5j#xsJZMJ9Hka$b*DY?J@71Vs5S3!PSKDEY5u`WYoIBazi%i&5`{GpX<>q ze2k)T5dAv+f=R25l!{(swqA<G>Y+fQSn&8ZDa63IAjJbTtE$e}AxO9-u33&pbi|xD zfrz<YbabQ~#(u!#7qk5T5H$;{JXuVXd+;k!K=Bpt-c<;oVO4B+WNE3s4)lJb?YjZ8 z-jJI57g|zzvZx|VDL3=<z|o7IWAnSl=M|)UIC<tfOc^n@w&ePpYI8dVt&Y(|1+z0$ z`&@)#ms$Wo%{HkfF?hK0^ol-hobbtbD6)2}!TV#tS5Kg6tO3FlH)BLcZFZ8kg+Z_^ z?v>92*F1Ub$JmHr9Y~(<d~%X-(gzx-cUCKA_PLc`hf<NtrQ&rTMI#*JB40dQety>c zP}cFh%`KsZ3By3W1GDa<!UZmm;;$Q|J0c#0ux(OI!c+>p_PY)3$q>yfQJN;`mI%S3 z)Rc8^`b7Ja6(Y{-$XSb>Cb|^r8~<B6a3=5u0AoRaBK`j=ptqlaI}-bI2_|cn=c8q8 zE>T{6o*4_OZ^_j@BJL=aDCNh)InRP`;xFpH{%ouCQXw$ks*Ip>i{OzmuUg?GNvKeq zH1vKMnB}*=<x`utd<`ojUlNPsP{ey6b)7sAMNgB9{}$M<{U~JSdSERd_odX79|+`l zpjbUp0w9jC7b}032uW$lp6JG73b#U7SC*{gP$|3A!j{R|jg1VA22Dvg+xlTYgF33T zC+UJzWS#|)g_xCu=4uNgCI>!%`G$iQt^s`92HIKZv!d{Y?=2TyXlfp@tt~@B)>9bI zLqFSg(-|^XAG)_!@Cl*Caa}rcMX(y;2R8zTTl$t~E%ws^L7kMgE++E7>nDiY7HmIX zWfQRTo6IOr55kr~DHVuv_=%;ZneIQhh;yr;v`Z&%I-T0d0EqbV_@3zFnNJl{MHgy5 zixX625afJ4AFgLp*cHuKVE&c<hyJXEM6cFN_pL~Qv(8#nooIcgI>2*a3B7)9I&fg| z<C;dsfo8PN%3+~}003ksosc{dbBbD#IoC1KvyglrlbSUJP#QQvAMf?Pwn6vPdWj@h zRKzI8zu)#GZRKj{aVgFMr{vY0)?~Q1s-PB)ns2Yj8g-5m?bZ+S#O+M<Y7E<SQfyJp z8lK+U7oJOw3`nhQ7N!Bf$sM<}t5sieh{!Om^heZZ$Bah@*o}k#@}~bss&97jX=!&S zQ~7=+N19&#l;?}Nxl-X8$pG#NlJhKrpZGm#6tw)Qf<~rmPWJG4dx=6FBPMU%pT%+R zrDo0XH39|bmk~v1hn=VLa=CTtj&JFmaDot45_65<E~=e=|AL_01v=R_Zy+d1d)Wpl zEDcWKw^e^stL<ECr)+y(gd~S6fsH%Zw*(Xf=NQr5=ZxIQoewp>N|#Dlr`-ytl+l@R z-%E!j4VBd5Q8l}ro+Mf5aP~38)+DP%N9f^`skYb0amVzGjy|<{&NO4?N02-VmR0Yk z-7J(c2B@RK(%Ya#XT!GWZ&I+@yVEDEBkLeYS7%N9AM;eFAZ4fi#k1ZZu;v=_;nV-c zpiM8CfL~pe@O7oeHM&dN#l`P5SqERdo)#c2|3*Q&5?@Unas4F&)GSoqCjP17oCbG` zt>>K(+0drEb2XZQ3W*`d15ne@J+)ayvP_F+v)G{FPnAo@PkhX#puz_=t|_Fjhq3gE zqpx4{$d1F44JHz(2nN_V(SX)aFHg{ZEfD_^EOGKy=1^RjdElo?d1(4@K?F+O8a4Ov zEF(#UU)u{y3dD{P^3_qQ=!Mei&Ivw+bCXdfYw65jD-D(H8@ZYkizKRJ$~FusMD_GT z{3c{{)epKwu&z=Q2zWF_>zO1z8+Tr+kSQkx|LDkCv!TVa{}bxt;C}+9BEA0qp7yu@ zg|p!5Q^%$3fF-58UYfR`+jgQ$9C*Pm0(yC8olb7SRQg!(Zs32Ky%~0fv3t*jzHqYk zjp{3fcpW#PB6{^uJhoM2&=M6n;D<^LcvW=TA>?ZfG{#kNQyNxv6tzq^^`Q;}hoya$ zTSllXKGWO|q`EO8<@TdWPMQ$khgZ$EN8>Y~2PPGc&}nXR<~)qE?fh(xd$Rd(q||O8 z!E;6$MSt}LaYHGGc4Qt}JA)bH{BX)`-Y}1u`9xvIp|j<pIV6Vx@}z2BLu9!gUVec? zerkP;`nlT`g*gk#3%I;Av2<qwqg7~qekA#LuqH*$z3+S_-j7s1{Y@_V3#9?NNLrdY zZL{P3ug;eRlTKv3@Az$#H<M65=wADc=06K6MLXWTed`f~5?`pigN^#NoIFgyf(d<w zou@vYl1TFGeTjALzz)<GFo)_?j+6=W$=lYo>|lCWx^z68Tn?;5+T&q2Z!qB)QD;DA zlD8_zBOiZ9CbXBnM1Q7H;fJ!7T5@R}JWX?7SO{8eRX=EC;WD?NYeXK+N8X6FjF5ly zu^AaJjAUQfV=aFt%&j%sN+HsS??^HQTAw#sKDRySwvLJbR|Jkrdh<4?S}O(+jNb6A z+uL4bt*6#wYqQif3lTk?nB9h#-aiL44{&kzd({2qSbc|*UIT#9%+W9ROQ8Kfz#zs4 zycsF!?>tNx*Nof}KotT1sB@m5lv!^D|D4(c1T+kJ?mw`k=Kh{@27P|N>b@jr_dNsp zQi3TG+X8-QPP$3heY<GZl>4<gyB0jQOWwMXutyqm8j3@@Q*bAlOO1J}trWUL(uo4% zTE-l&Q);SI|FqJeKvM0F;=75Due>4iduo#^RzpME_mELGv9~jqofpJ~rAIF=@CaD@ zO37%$B#JGMY8o-nwY_6uS3=!Z(+Nb&%E-`3=dkKH4UWhhlzIW=>j_y}AJey6DxF9- zz35s9!`A=iAi$?95e7pndfZ=WiomOK0O#L!>{$F`7;8xa_MVfpgfkvGH|yGTq;ss! zz4EDv)Hm-Si_U*nJxEyCPvq|i#>d(=&e!(&<{sZhTMkFa;+3}{lXbLmCa89jGmq?( z&a&pdsPn%5&dSn3MsldxDZhy_>}u!HQgu`YbXIxl0=lN?1|+|YQjA+JpwJ?Xk4Pk` za|{+)wBmN8J2w;M&sOdj9l4#HkO3|CGoaQchEvw-%&mETZ%8Qj8|#oPvS`}u8u2ee ztyUhaqeXl=Lyr}o5w@{rNDG5aO$tYBkpo+yeoj?~DnFIfzXr}9juFTNAeC~^Q&1ZJ zinhN}Q6QWFq-=O9x#%C{EGJO)lqxlBki@){qc7KEctH6|Yx*){-yb-@4{T_OI4J5( z?5389LLDPq^~iY1uXZ5{7Q|%l*^?-xOcEj0cuyO+!md$PrjYJxJBwMJxEz>)eU+V$ z#bc@N1=9?Sep@wP9Oet3e~*gXrd~jKsbqk&e`Y2E{ur-iDlb3dbk#H=iOHA{tsrk3 zm8s)Bx0MC!<00=spvNYfc-if_mafBlGHje6b~1dUi|l>TNG_+hoBitI@lBlSY@%3A zfYS*H0uy~!Xwv)Q$UNmm<$v84{SPn@cn=uo#TnKs|MA|RFPnZ4(_X^-c#^C2GXg1Y z0LIECG$>ClmvMfOqe;Di;jBaaQNr!#OFK|TF35=5Nheuj^ZT-wrabOEbRzZz4>h~Y zOhiu`oX9ha7}n%seeCrR&*^A74RhenD`zcCRP9<(y7sv{BJPZB?RAE3jHURCbd6M@ z`j-Y8b)&e%byjbmY->7;8g620-KJoIV%}==!i7m$0%svZcAdu8!_w}YciCMnIn9H< zK^cZN!FSge{JlfO%udq|LLQ!<&D<Y<$W9y(@EX)E)fB<*?o^S1_C{sBXa`R9i@!R< z-xj#Dh2<I2ML2&$n)Z<d=5Vx)7mkxW&d2Vyfjw|$Q?e*U1|<o3$ocPBK|y4@zw)!f zHH4#-PQU=Jw#tb520OW7`t!Pk<QIB2rZ5)(hDgV9c3T<@80&_q=G%f8SFeB^&DPwp zr)ZAa9B_Nrs;rgzuE+PQXCAI)8bikDg>LS6&%(7qN#u|{{K!xzq)vx7LKAB#`qiAu zk)0au&|~#L#N~ZmliETkD)hS&Fx&1)ja8_HZ}-Fa$gz&}TRo@B3F&Ltc7(?INLgx@ z=B_O<F0nkro5t8Udiq<P>5Otl+OwaQMK1rFZTbX&ZECIA7V&>m`K$4g%V^Olr!MGu zfUmGo*7Ss=1iwxHd`w0WZ<4^JC!ly2TJ*<(kwgvlHb<{0>Ei5$Z8^;jcW$E==i_}b zExgUu<;@g^PdN>O^)%>4*f0mwG@hVnwRP5@vr6KU;=CPe0cu>Wc#LGJQGr^8j26wX ze=Tql+B;ExUQra$lO}Lpfx(VbDZ!9iVfXlZT;$kj<90eHG>QHc!MT2LJXGedSLTR5 z0d&>+fy2nAg|{`!DjjXEFZFj0!22%Zw|PWk-r(({Xuy`!qNOA(i;Er8w;cofU-{*y z->d??hGQh7z;BqTJA`0<C{EaCOY+mM=hFhnqqZ-vx?P5Zwy`y1Q?HaZf#xy`87}BE zY)x!%Z>j7LH&4;i3AU%nFqSUum0t8@bf)P*C>t&hnXs7W$iEcb@tw$<b2Ta4FrhzA zuvQsDb5TKi#2>G*Hv26ZlePv}J0;cLIj_FEMv<ktVZfKz_@3Lq>te0$Q9_{<WIPxZ z4m!it?|CccD5}o%>jsT^RNIDgpr??ziCY2Mvz_g;Vud5+EfNFeGSbvrk%|0(4Y>vg z*&r6JrN*zzW%Sqi6(9XeVFqm}7SCuO@&6{VNV5W>H-Y5cpVPqGKx3;HX2?q&h7Sf6 zdW=j<kUZk|j7*B>AA~(6@9>9-HdmGZK$I19M%Yt^Wv_c3mD(q>@|ryyM_P906n=5@ zenI|2gYgp^4!4bJSy^N^G-40Ezf{k}?7nO{GU(Z6bU3uFE<gjjaDK4J+moBoU4hrb zBMfMBJd#tWWx#>LJQU58e-fm?XS`LQt;S=q!W#^z0w*v@S<9J>nRN*V#IRJ~*q>`# zy7)mv@=>Bk)L-MsM*69({?#ca?t!Ms!1|frZ+b6N&@5Q;iC+AfVx$wWl#vVppRDPn z5C^j8cj<y1cFyc_=5agqpW^5XwO@-N3Sr#Pcis6<%I|#Wh;y6Hwc>y^gisw(&EsRe z!y+MBLQpV-97ifHQe#ebqCsogUQu$$Te+vENKD9v+J!eELB6(B1W|@^)3Xq^jO&=( z)iW<D_W%Ms5|8f<8MXefm9g*QIUURC-fbOa3om98rZhw<83VxJy~kf`C40xH1DpzM zEa493$PiiHQ99n28wKYG`ib&gFX2i&pk<=zP2xi;gFGu>CA-F>NqGHV(#Qik!1X*5 z&LhOG%KVMzurK={AY8^Fd_7OGa5>x$HxJk4CZ(q0eNoQ7IfchO8|*sgH%T-c$4Kth ztc#4?l-cLj3AmJn0!_J9hv*~D1mCtk-V5p>^Q=;xU-)S<tCDW_Re*a3^Qx`7@yC2_ z9_8?{d<yy376p52NC7HL>HMmNVrNw>GFSa{!82V)Wo1db5VuB4a);0|4TxjXeK@w~ zFTNl06&ufTv2?39*%v8a@l=m=TXS9UIM85Ozv#h@v+ya?t@qWoRPMwRq92VRhZ49h zCw}({ejX$X!j#G*0TI2@hIBK>eA{_7`Y(OJ8C)4Tz+5cq4v_pNzdn}-K<+tyLjKQx z!L*Iw-NiPb<{<21>wiJ=IF|p$J;E1ALrNt?278lUXE4SRe&#xFiNZU&TCh8>yzuVg zdq2YLP3~>ovOv%b2|4E(9BD`Q#_&|z9nZd;(y~x~Jp;Cgp!0MKhTq8}xIKyqMiwlM z;jd^qCvQcFg*=D(Ssjr(YZmKC!DH%CUfm@x5v3x%6PrhgiVN_TFNwaQus~rOt|Fq# zlpOoQD`I;0Lyl&cE(0IGjJM_S+9h8e8MCi@W!9n@du-#s|L%YCt>9g+7hoTwxQl=0 z*YlI%E?AN69BJ1NY~X>35hpGB=7hlcf}{87l2@+TNeLFs77>KjWG}5&w3iU~PYN!Y zC^D&qDUC3&<$zC5_4CuuT0rI>Nv!J+J4~x1R&`Ms6^8Z`^MG&YcUT@r>dw8ab6c42 zdM+j8wo+OH-ATT4zR=s+{$#d7)03Y{?uvJJp~38op>wrn-Q@*wfwh8p6JyoacO`Tf zM{?Z=m2`7d3sqO2PeP#w)N5bUre0%Q6{uP*8^A)NEAML;Y%ZWsz{T8}Ss5u$J3<rH z(|iqWml;tV{TEsgfVy)aVlv+--TaN+2nj$!B_goC@h2dm?PH)t(DBT_^6?wa#@H(1 zcY&RAq+|QuIGEUCp(+{Yz3iSd$?PNwRC?vM`uiJ`m;TZM@J1d~0iilve|d=FgEh_9 z<36`p%c%H`kTUc9J+y^<XPL4ySy@;GO>6V)#&!y1ZCG(PxkM<hZux;(yPt#JZUa%~ zR?_XlW#_oIQ?K;71<fT<s^@X&!FblDVAOTA`0`$?-#fTeJ#v0WqV<iYBurFecU65! zWXRu4?K2UMC%&X<KORT}@^4sgdb7}Mz6pBpF7<Ec_%-3)P6aeGeui|-Z}cfmQs8h{ z(&L}57Wm~JICfXr5t;m23;eGfxM@zcn)n>IYGo@g%PJPsk<V}!XDiqbDZ$-Fj5#e> zll&=*wIqjD=M~}{2wv8lVxTHgqp(PgbNA}4-I4~Yi%DMl8fI^0E#sWc^Nr$pj3b!c ztveyI`T`XbBf&R*I=ib$AOtoDt)C3d6vawvz*878{_PD86Wm9Mqxy%LKB*+nzxy#m zqsnbQ!BdPSd#c@aX>O3f69%g;&9K67YF?SJY(F%5>C4I<ht-Nz;txvTXFgPNEFc>E z%}T=G(X=QCF#Uw!?XBMgSs*=VjAwzUN0`&OeI4k0+3lV;|KU=BADM`2)At2ADSh1h zq2({N<CV!4efHJq)D$(V9TaH_aWi3Xb6!(Nj>jkDq1`s2%I!CdJR)RUIj$1C&``Rs zn56hKtQykByXw9CWTLrssCV76d%SN#xXI6oK-W9br8%FBtJc7l(H0m_B&-R1Pv#oo zlevQNR_F>r%kH&DcNk-_iqopzv<f0=z(Z<nygKZ2oxr9u#RU-_or)b7iXhylc|IK8 z{>w+mJ%fX_!_+bOe6CbEzg57HKjKi7I0`Cd^#rVcnKFVE(qoU8mX8tY*;K#DrGLc{ z80_P>Wtovv{*KBz#xvK80w>$tnT0NO&|yl)rP~1avc-in!hGb{S9^`Z69Lq?XL?eE ziU+I|&zneShaD|<CbZ@9I1A~VXTMrpgmGpudPu`Tq8DzKPe?HnNB{imMmM<n!=2<0 zu2%ywiSn-Jo!OZ|Ki%?3?V>SFPXWP`d{q8Ui*D-N%3UpI7<=Ht2g-smlJ)e5j`MRA ziuLX;JCR_MQEyiZ!YN?fa*_b|MmH2KblDtto1tJnRPd?6ZB+0X>Q)zEBXb>N#Z~WN z$M%CS5X3{t2rKe<zuUAPhs@ds7XrWBrM(uu?GXaX(jtXK*8fb%*q9q61x)$9U!#wh z!VL>=mviE9?!FPg{odVTylh{D!}d&C5zibb9Numlv0Ysnjq3X#4rAx{3}uy*-khma zK$+1%=Zf!K<4BS&;<ZAwBb(`&FIn5KfgBsC(({_<2jG7b{Z62V-^aQ*uNxq;XL|KK zvh{*OWF*P0Ge4y8%?l3M>&oT%tXK%xP{At5YhB~l4n?M`9HpoVdhTVogsuXqSow9% zwq8jiAh1}$fz{jG0Y^{FLeGgTm$f4Cq*%c!2+1~fuoXv)f4Y_*8#z~x<u|Dvv47*= zK(;@V&>|Udn0nrUaME9DSf30QyU4_LtI<V0G@bqw8WtZFzBn$PN)SB|5+uzsP35*j zg;=p0=o<0h#1zoH*Lt%#C*v$6_byiS!=4^9_h+UYa8v-%TiT$@8yGxoq#$)(L$VV8 zo7WZDbW%$0Y-$M)b-9SA*GUnIcrFD#_M8hd$QnoV?03M)cq**C@P1(5<6_yLD(IH# zX)7c&;8NvSgzJDUU4zF3_uqUaf<$dB6yB6LQQsdvXbSHyY^1A=p(rUdf#mGo+1V0p zH@fiMjgR^KmduTL?v=4(Xa{oVj3=jTGn^!9zfh8aM$8ruSK^^`gIKDBcV(<UIMwE* z#>}+KaOeExf3cZ_A^^fRx#@ww&RHJ_Vf*Qr()*4av)I}nCe1<{t5t-U_h)!5=#$uL zoD0?w1XJhl7nW0jtCXW(aNixJv-Ryu!d|+nhiK7WIZVH;7QZ7RC`BIqh29a>?oNJH zwb%bu)s36A<X5vB;=G!xI+LW9o&(NYUq}1ufWFFZOU(XQ`9PP;d;|Zcvyp)a$y0dK zXwFkMsHtAL*IF}$N_UM>*=d~oVa<9rQCi;|w3xEY{g^tMl-sL!cMdu&X?UuAUevJ= z1E)sF;#R&K5S>3>Ak1U7YJ{6Jd#f#ornB&byv-=Sjx&PEi{u4sg`|N$oH#|M{Dt}$ zn1sG||2tpx<pDr|AG%p)ewOq9HtU<8M+{T>oF>H|gI-etd}}q5$p?=}s4k-Y^2X_U zET0&!+S()fe%okP;9bWBBunZqa?}F{=#4^}lNmQHg{5Gkg(Tp&Q#%;owTvKMr0hs^ zSup8Xbbc8G(~oo%(vhruIi>~axbb|*rG^Hm8;mVPpfs@w&=~|<i*tmMJ{!#?WU5l9 zL*YotmCn&pxOe@KkCY4QHMh<|UtmnggSeLno{`}jXSMf*eTJV|U2!FJofmjCdAYb2 z<E?UNwh~{~2}G8S2Yd(bSO>;scDJk;WTktK2a)?fEv4J))`%JmZCBp@xyJk7pZ_oB z4!jF=kf6mZ;_IyXO;&mWt|_>?SY%RRS6RmE3i;Th&n1CV{Oh1kU#l0P$-8~U$!`}` zI=(U6@x10laBgN?DJ~l~SF2{IUyDGjmq+JaoC|;g4TehBdZML#rxfR<sE}aCX5@ng zgY|@CejznxXWFQpO5dW7jeEJmCSKSAw<dClX{S^mxwDw9$7T5ouDwQv1zOg){FAp9 z;oMqvjqkpQ7VhP_^;8MIYbH?CYZ?m!%5~tP*cumCZh3t9g&u@(v4Z(n(brVwCbSQ5 zLZoNEA8=SP^GKN`HMk|-I6dHtY(o~`9NU%V5h0G^tR)5=XdoM|8(5T<$RMM9`&U)_ zUHS?%W&%4qo#nm5zcwHe($by5E}Lak^mK>MIoc$M@d&T}c|s%xfAMJr&|>zRw0&kX zQIKGyPvZh7fIAdxPRW;aIv`6Or|GwywRpPnvNR<lHp<#888^A93hC3dPA-k5iu=83 zODo?gB5{6r>U#<%B%FTpUA`>b3-~2GC*Pl{54pg7#)J7MS|_g-+_pKrh>UJAUPnlF zKF#O92`5k-vNj1xuZ^pY;3>E4zP#%=;sl*;uv?qm)(ou~o3&W-wj&HeQK1)t`w2Vr z4@gIPrS2=WHcz2Nbhvwf#B3Qq=vnE>VZMq|Fx$Ui0O>a<FiF_;!n@=5OX6>0!7h+J zlKe<WgqwutO76<4@E}VmAXV}mUh~uL=qH6FaKaKzNwUyxMTEYBBH7zIyae{r7rv?v zstKD8!6W=oW@A33RXG+%#T3@#_oS^Hyn0JV5Kot8-G%$Fl#)8tCY9q)%{Of*5g8T* z_o=KSyO;b6SFc%{i5M|4U#g-hUx!AVxjAEkDz2%B8?b4V{P4|$cO6#GMiu6^8q{px zQIk-)Extd+>C9}U$d_)g>ws>#m~Y;71GtW|k$+8ns?xfs`>6Rhw9@Oq%GsGGjrhB< zacHoWbdeD|T35gi^4GE5uUe~*5oEpeKf?{%ws2y#Fr7-gSAMb589X*D(*6cgxOgCV z%E@z%#h{j#bi|W{QZ_wQ%tD<)w^~?fbV6P8?!|Fu#0m^=mRLtzX5@hEC{0C<bHLTP zn0#`!ZZmO)h^xzrvf7Zbw3K!p#kV0ve7V}GZ^E-lLUnDhmf`RNH7Tf*ih$1@PNUdS zQ)I`UOVmuOBY6cS=jiRV#t9J&zl9zpu<DL;TOS)b%nqkxc62>0@+CTq9&1Q1Cpc|0 zOfBC<F!pr_bD$BIaSYeLGZG#dzAMzLM;(fk!!y|Sg_+6UMR*kQlu9wol}GrTY{>Pr z=p;+hYye!@=Z!bJ?(cag6LfCJ#JsaJlVJo8{@DNheeCT6n{6gN`t4li!Ox4dKD^Dr z4?I9jp2Zm<6&!!kqSAgXX`l~JmbULAPFR3_dmUP=fU%Csvn_%9>e9Q2m<9vz_}z1Y zy>xPuTUK@Qi{*^D9j2mVAFk94VoG^2$yO)XsXfo^x_wf|DtpLNQmTav-|OE~BHMp= zB~3yQajHI9vvXcXvgfL4!PAA-jBOt&x$3zV&))cq7L3CxSU=r<&J+xAM;WTLyr-%` zfZq^P>UO+b^>AX{b>+k7CR9?*?gzTdNVv!5^&Fcz6je{-{1uPpldwz30~~Ywwj!T* z??~{s7y^HMx4``frdR9R3>^Oo5QyJ{bs?pvzsXvCq*zWl*E)WMK$q|j2DRpy^izwv zo*^={sV%uGx5>P`X#)+whD9S`*}esJtX36G5VEB$L*4tWYU3)aa`A<U`o_DpL7BzK znAZYb!l`V1FC4tCex>&hacf@}IWFnQO>QJd0G;fZAAnmbw;ExM1v}4jzRb2lp8Dmv zIs#OnG_=p{xfkfjCuBUf=q-$bL4qf}Eu1y43o))VdnqocwmOnEZTm4gvxNBL%MR4A zj);wUJ(hWTIp2%+!GK{pc%blFYjO8lDiauv(cP5<H%l%;Ug1pw#eY72+ax^^^+3q! zB=?T-z?5Fp&spXF3dAj5Km-`4`?mhdtS-`F#i5z)*6*>d_~0te@tN05S934#_tVE{ z0h*H}3+z%2*hdRx7xsvzvfpjJ*99Qdy8^q;{tJ|fp*~!5GIYEG5w5M@2cK@NC}w*| zgbWeB&n_*Pxf7oh7y1w(0yP1aG_ZNq6CdbCZjzU~^w<Ch{>jlPFlp#~Z_Szogwtap z2po(+3}#wLX(ewvK%BS1{?kTkvU6ScO1&2$Y=3j6^v3{U9hgZs_4_H^fO!=H892N+ zR~aoF{&ACNqFutZ_f}eh>={FFzV0CVZcvkEGWKNVW87+R%Y%Wz?>FWC@784-gj*=_ zZ}O8@Fj$fDw$7(y0fiv&k&JZN83iUwk!8Hu7>;xxeFg+0rAA(p?JdOGZjjZ1rDfRt z`Lo^hy!s<r74hu;2Mv&YL7jU*vd$Nn133GML0+vo&ZCx^+1*r%l8!JhXqiH)S?lol zMglCdBH^aL{-knft!|2zUXupb8F2ea6tse&(giOZ@|Zm)_ITo~#hqr7>B?6BP`Dk` zTpF2cHT*i-`JJNYZcqBvvBr7s4$_kBE9mDpo43u4q=p~~^TifFesD7HGztLtm;3oH zQ07g9V|US&Y2zZdr&LRKt$R{O_HTy$tBUqapdDyn?*C3>CLsg(MG3yIgx5f8)AXK> z7EqUMCoioQ%M_~AZ9?U>>H8m!IXrwKf=Q$<e`x_Yiz1l|iL#a-E*d&?A{jS}^}P0V zjXR?s&K()<XuFpxe`ISyaMI*jOUCmA^_#+jC}F*?r}8&t@~Rz4hk&MfrIbg1uKfv7 zqWGkHb;+y4=60kH;Mcm!9Jg4C^k!Oo#jJp^prc3JvJL6d5K(FvXtaCVGO5gFsdPco zKJ!(etRz?W(AtZo)Zv2KNj>z&Ow|%3xAtbEVC_Iq0^as>oi+u#u?&`TVAJl+E>V4P zTQ5jPY}~E?)~yiBGlDIDg;$%uaWo1tY`4pn6MYw(s}?D7l|hetay3AEzrSS{H@eLL z<`*abWT>muOlq5g<E|AYLLkg0?Q!yL{{v?r^bhNU-R@s<&*l~(vB=wkFATF?HJJOH zUkT-XAG`r$r%C7-)7nU9Kf55wpUH1gZVjFUP|5w6fS=h<fH~t>NULers{SaMtvmP_ zwJkpdB?67VMK1KX9Zb18CU3&0QujA8Ra+*@?P<>|DdZ3HoU@9zC+KM)SA>kbYGOlz z+P|0ILN}rqZ~t5-0+VsfZp;cJIDT-HuD)xmQUCKH8vlW@|M60q1}ibZK8oOPw&~}~ zz;vvpE<M@tJQhHhY7+PHM)VT1oGouFrj|Dt36Y(|&>Ka_>(PJI`#hp1-WAW(Vp{ep zyp$uIl~*s`(w-7@@^B=V;q30U=;Hg5Fn9jvF?)y(WDOAp+L72&Ra#aY6A5-cua;>_ z?m(&Go8x<+hfw4bgq$$yL475Lj^#FcyX(MHV0@;E^io<dBHq4Fi<Xbl0p^epZZ111 zqzrNs=^W*bk1X=mUaBFQne46KKDfTa>?54$(|T8$nvBMX5)UspTzapB$yT=$%Tv>w z)YGA9FaKtw_ct90w5?!}aVM1UcSaEh>9}X+^T<zg-KoQyZS_q1qR5L2YfIq00U=jp z9m|oAr&qB%(Du}b97!$}3CC3&GJ}x!vbFQ+oD6O<OD((#QhyL;L)x0t8Duv9WxUd2 z!~%-$JKo()B7Lsy`2}G`c4~p1K$^S?(n}T@Db7%f>x~bE?iN+fKJ1+;GaeoGZ+4zt zxs~Z-AG-f!BKMP0)zn;%5yd8nn8@J^?kQ_R-C~+-sObcs%=v0>ZG#nasFR*z<_9W; z=a`WPjX5+`ymcAPk8c52*T&7wIO*kBZUFOvF6T1;Pm?l|VCm)EuS+j~SFXNQ?6BB3 zc#-Be;Jb*F_L2*D%o7+w10OmcEzPp@7d@hcdGE@nb7N2j0`|$`zZGI~sLKT`pu+X! z`yI)j%N35w@dFtNCC+jd=UVe}0!JuAE>~YVjQ+zWPF6IX{K=&L4sFQN9haHnt)p>_ z7NqiWpiu4=bWOrZ!mHx+KsYmaI3q@6X6W!m8xkFkuHAWO>0NAxU^rihMz~(V61@xD z<3fd$VT!vn^;cR#m>!GtDF&9J&Tq_+X#pPNcA_*3MVPBNu!h3^a9I28r){plPXCOI zzm0Ln9vCC!vc%<INf6pQsd%n}5B+1`*3A<+s!;WeWIFsTVSRY_a6J3tY}eY#7lf}y zyAW|}%mK9XT5!&yppD75ozq}+J*rgpTc!2e6~X=CNTX@FPag)WS=y<_VLlw2?+>Hd zFb_ov>fT!*likF-U@4^jJf3%$58>3c^YQG@=GVj=<Oh;{9sOGboYKHGgepy-FLmgg z%v+dp>F%4ZIpUDBYjsAaQ`R!UdNPXn+!_n?ck_tsUv{U}!OCsgMNv>#?OJyI@QIm` zVyO!$v&xFxi-6Qa+lWoNzA?P5UC(>>Y?LBH+cC$~zbiQZVJ>%2l-%9pex3P9o8pn; zTfxDlG!b9YtcE~mJfc|mPa8I|(Dk(nu<76qE}H96L4)`U&deW1-8NeGL$DSrV4a1n zzk-!UajHVxkiOWvA&=L%eLRGagT#nJ2IFlvwVk>E8*qMXMPp6f^IFPeQZYj$DYjr* z0zp)rq%#(kQ?L`zKC{V<?_dS*nVYw$XIN7hGNw4pn=WBq8H3f&17O1o$(O#_D$P|p zeAEdMK@|$(Jwcznd2gqhhZrO&xApxOlwn1i#O&8EKEoiYsPq-)15Ni2$rx$-E&=V{ zuvC+Dt1;luaBsH$y6f{Fu0+8Btn*wJ_Pa%oMK7$@t=c&{CB4>fbdeSdZ_*ivC53P6 zbbX=Mhs!fHwJ5=~quJ$-wn!bwaxE}+Dvd4D<!w5~=9@D;dAGYvS&!RUiw~X1Oj|^L zH0JhXMbsMZWClHsU*s3s7pYh`s~$f=4$l=zfM;-wamEu{l)?bMnO2HdRm`@%wOf6B z8C3^HDqbHrGRBC~^2Tm>F#0B(7|e5pd+I^JIj?qPdK!%fmzYF|7hyG3#hG*i!ErHw zn?$!7Y$wW(yvi}`zI%5$$hXd(`4F%&_i^}avlr$p&2xZ*-{Lh(+*8RPmD=+D_8Zsu zkNWfmjDcK4WbFRKYJa^n4I+6tF)Qx9W6vyfzw<8k<ajAKISwEW3*#G_n9Ootfy;>d z46!v8O<Q5-Jn7@(5Z2s74xPOs80!>_T^lWPSU&6}^nQt2Gsn_%*oZo#+AR%jN-mBB zKL38d&?ddFr&Qo^cLB*`r+56I!{8jbLjt1JB=K&AZE?RIRCbsnx(CX~jXbcPsC8{z z;C5Bn=^h+rkKWm)5I-DrZ`oMrN#icIC?*x6v_PRQWn>%a&bQU_oUcX^aRw~FzCvUV zIGqy*PO(otK~I)nnvq->B^&d$m_cRO`0UIi#*Kfl`GM_CZ{kHodRwGZGX5sr{uOGj z+@NBurQP)U<5s|5Z~tt)nuxKIFJkE&Fu4Y%ef%Pz!Fo=dA?yDAfsqiXEE}ck!}$pk zXYEUM+MXS%$pUA~weE#)qu4ZHcj7ieNSyQKufsZ{o!rLds|^D+xCJL74X<k0NO!OL zw{hjJs)v?*@ZlKZfUxLHa;_rMJ<wiX+`tVu5*&o<#NNq584<VI5QfnL1<3(ICdE%8 z*hH)VPnw{m2|@SOz_xUoLxeO#TMzhDdStZq_XA`Z0bz(D>GKX=pG_5kiV_hPdKW`3 z+(-Tfxc%h|AaO@w=Y^^KHNWZFrM<BPKDDCV&sl8XM@C}f@}uruw<3L6-aNi*d9xL% z93|S$I-_4O>JfE$hemA0%Qb}-2@R@ya!jaeo*rOJ(=hluRogo2ab$d_L#?Ly7CMt5 zwouDBydsUyk7l<)3BETL{HcIX6}_*N=3noqj+}hZxGhN$LFZvpxn{&zYQWYIT$;dV zlC0|3wO^*Md3?GuaVgwE%U9}$8*63ru~4fqy6A@Y@tD+cm{#=Ow}{?X_usLxJP5sC z^6Aa?3lH<s>8!8n*;hNmd#4LDd6IdGv^b8RYqQzax0*kCH>y+eJnz|4TYg!4#qEr# z>IszFu7u5eVf{{5D$<4-FL4C2ULgrt^_@>mT|ZeBe_MxfKCBaf4sU-vyDCEAaeSqm zC^821`XM2W+-Yr3i@v}2?AOT6Hyb7SZP_UJF#&NRXP|qN@I7Q$SeQ%;a(~@g@#ABB zVp{2dGdQ?-|NWwA=n_7y(=pW|&Og3X@);McbvJ@rxHTcm;pY#T7MxSar}elT^O<6^ z(qsbRKWJ1S5Bm_dxph@C;@hw()uB4u<!AwD>TVX;<~1_;KC`Hi@WbN+n)k$=0?yEt zsB?u&!b3TLuTc=vj5KNqYZ+2r+G~NAa$Wvd$sX@seX^N9@g<V$tEyt}m|Wv#wNqr~ zR(e|RskPy*@~Is>Wu77%lH;Hu>fw24y}EN%5}wooM>#p{37a6h(Yoyi?O+n(GWFO* z$_PHP135XnUg>X*wa@n%_rxb|Zf;NDkF=r{i%kwmY43>P?t7<GXyJv=d{-<j+O8eB zdWYnnw)xXqcSLY;+hX~yd)@no{p_QgO4!<3{3>N9p!z?5_=&XXI4U=pR?i_S9}-YZ zX^G13_I;s~MdKSBIr9|#B1G;*YYkqL9qxEH)vQ^Y^TAYl?cQk&vC+yD>Zqb+06nsR zNDzE<C+9CM0Q~#sBZ}8XOM9Eu5i_4pJqY}ePiqZNR|TtQs)CJrQgn1z`~8NG;zxy$ zi(cy;>^ceJu5TjIIhE_HJ4P!QZ+G#nOv{PNg}P1&r^LJM!gl#Y3&q!Y*5*%ZgCiAJ zrVUpwel}twCjyV~GY&&n+$7`gX$kVwvh+W>N_!^*cQwEnYVvaTuA-tM>&b-u8s%p7 zgYe{Ey$Jh`ppNtzPiTGG>wkPXQ7zC+D=OsJ{$6dd0UrJ}e125r^8C5S?rAK6N87Ay zfwHnCJ_SY3M(~=FFTQ8In?L#kf@)a6nY^&dLgZA*T&0LFN<{bITzd?yQMnWK7Kd%d zZX@JOrN{&0P(DAwwIfuJ$6;!txVVJlltrq*vC~+5Fn+Cew_}D->rDgOr=GfxOI(lL zs@+orjSeMN8gxj+Zw)nAb_=a;c6`mo>608XLVa@`IOT$(9B-~=f1i;oFhL0qX=gZ& zdU$qs9U6H7m&(dU+51g(7N>0A`;>J;tM7EHX!}gOQLXzeVS5cS&$lN(`YC}4WZ(E^ zyo{ayKJ%0)&Z!xh!po`hPpA1S3}eoe$^H_M^0sPs8BcjFd^SAC*<#OtNTv>~!{~2$ z={|*eJ;=)4WQ%H)l-&~^axQ<J|4t09MIHP~kP)Y6xRL3o#`vji5fA^u>l0Wo_ZYH> zfJO+XEQVH>y4jxXvHOxtcow9#^$i-BYZ2q7A?EM-w8nCEDCqi_dp2FO`!KgjXSbIw zWEB;haEMQ{`-roezbYlg^|DgsddkLWHqQK!exu{m$@0<+ynYnTbYK9E2m_NDZz}Hz z`$;-htQ^&>>sQqF?uS{#8`thjAP8s@3NzrlW5Q0}-5%BFgOBU`=7c@{hYo7Zbhl>~ zTP^@h`2O(?gF~le=TrAm8XVjK?h@_2s$*)&2kbuL$52PSf8?sa;!y&`;{czIm%=~e zu_S=4?Iz*M4?HGm+QuI6wB#$KcUWus9?}+*592LCo>_hgb*IQ@kSAM(G#3kyiEo~9 z+M=tBAvTo6F>2BAJU0_#K{$qZcU;ooLGa~r%4zq-NAoy6$@`rxNJfZIqvJsdyPo%I zH5UPV0ZKkI;W>K(DX}~zx<m`xq3h8hP39^hpKLI5uQ$fbxcWZ1=eL&Pv`#I_l~lg7 zSG!X$qOvqDuxi(u-e~;EcYdB`r8wQtWkT@yeYGFt>^Fhpr1Be*)B+)^rr(!(r(&u7 zTc>HI{U=HN)Q@%Flni79-ME_Q6=;StGk@Hz-<Wa`c=I2D_W$`kkpu^4mNkiGg5e(y zCTRTT`14}UTgzHrOzZ9^N_^sB7VxdFW8B8O$zR$9ZoD|MHZU@hAU;DaB%Hik%_L8T z%p<@7sWzfd!}d&xgpCn0>cShvXRbHFTRic2rZ=C%H#0&`Yf@_kKM5C}X4ADY1&m~$ z?vF;Ea#imf5Yl*D-l#0N;^FyH+BU_yn}2g~yXMmR=&N<-@ct77$}rrb<jO_d53A@L z%Zb_DBDk+p#FvWW5_IKkP3Ks`XvO+i_YE%Jr_#!2s(dGqryi94d@wEtNO6Yyk2-V( zzvH<zap|Ycl!q<IQs3qNxvhkRwPHUigCkCh+tQ{Dc%wKoPo;;ar>DQ&HEZ!h`KSn) zC_GU7hui!Wk(JsYfkBSb(*9xB#5;lFsl%ENZtav4pexsiY479&oWUfjxu;wVth<Qu zjT}kqX?T*&g7=lM46^rOqJjSm`6}`<_vMr*Ng4a@QUzpR1yM`0vC;EH0i3lbMM<aP zbp|)yuGpcBFL|CSKJ#%-t+R8nJsoprTyX65Um8b@dF1JjJ8F!E_#81>L<$TT+8FgD z_4<oR8+pyFd7X?_q?iaXy6^AR?9}^CaY^_$?6^}uBCm5SPWpIy9S<?KnPSq&NMt<R z@xxin$9Q@weG?9O<Gbo)y(c{X<C*FT5hc?1TXm%Ran)9Xl|ODdL&=agbbZXmHs|$= ze|-2qohru&*k;s5?UdjzYyANu0*%kL<~^e#)1gnb2NplT^NIF6m{*FFyY!)ews9^d z-nC=Hs#sOP?ERu1^(-4ffIybrv!{%7g^yEo9^Kl?qW?7HMkqk{$^A(gB&tx*$B^&T zbFbL-#BK${{+7wda6r)H`NmZCjMK_=@)GZE1>L0Nvv8w>v<ozf`7elSFS!#K5gCYO zLv?E1A@G!(Q=^qb3U8_`J@1O|Gih>b{%8J=uPcv-a(m;5LfMiiYa4}7$iAhhY;|o( zmQtj#WM2nULS<<ugpwuODEmH9rp*#D_KCqXF?M4Z%<sJJt;_hi|MYnqGw*xObDrh< ze4po>SCgPE+_ia<dVqr?vkYHkGkpfR6z@@@T}wPOfLqR$b0T_HJ+04Gf36{U02#FS zoLZ7NFdk=byyV+sQ7DY)<h)iRjRlqL6oA2G%lMq9D?0hg&aD-~ALN7}aC2%VRV%gH z(Lf2?TTCWeeYUpYV)?fjQmcPf-nMeT&S1*mmoFw1&BgKQVCVJP8-8(F`{~+ekV-y| zbTVD|cSF^+=mU0{EPIYQCro>>akrbqu-C=QTr}`4DZY8VAn5wS+16rb^YWnS`SI<w zxondzrsUG$ct>pO^)W%WoOJ25eoR-9WyRR_g}!Q+GZ^PW*Pgn=l4qhHG*QiN$4q0Q z$>R@~F@3VIrso=kFJT`B)a%eVvt(;)b;T<^CK~J%F^v<mxQ8{EQ9WBT#&%Y|t-?*8 zCHc4fneUp5ukpYtavCxxX@%3qkR@WytxSiw!*k6Bn{~)w)P&Fx$3Q#_H(|f$rPgT1 zA2xEKN^zlGp02~ia`(+IR^^+0VaFqgbmuW57Llp-8m_`rtL^WztrYxx3t_&h$rUY} zj?PrrC``K)UL5?Rx7^fb%$?|Rf?Rl|VHg7vUj-tp-9>mLFMWKzs^Feg&{zw_g`$qh zqtj>{F)c8PvTN9+&^7n9g<{xH*3EObg6wQU_N^+C%UN(Mod3SXL_^AAj&wjqf&s=9 z+07vtaRhzq$pF3HF`8xl<49~_OpNzS+OeIq=E1^qmdWq9m`(}vj7WZAp`7C9+S;4a za&tcjoR$(YqxTkZffhBQ?|Js+z1ECRrp!<+hcWE$o`j*|3%uBb@pdt+_5Cs~!oZPY z$auR5s<pJ{)26e;^4Ba!*b^$;hzZ3-OHIV`B$Y}#2{s!{SWJRE_Ul}}=77JsT`X#G ze^`uNqSd%sKd*=RQR;lTgkx-zZcvjrF+d>WOHUCpA$`h?z&C|*QqiyIiys+THVz=- z=?7XSF<*lz%EW;Z5RV6!HA-FKKGX$ae32c9+{1x>GM#lHXYv>*+vBVeMGW;s?8%>s zLRtF*Rja0R@f!qfOAxdR<db!@{|N2xi<$i?tnNnl*;hX2m*RY}KpqGx_z*k0FI4rq z1Pdpdicaj0%N~h~JzPJcL@=e6L47TBwiE%YO_f%ZX7=FrP!`x?-3u6BUiz;aL4A39 z>h0O5MQl%V78Fy7N8T?8`ed_lV2WdcbeIcS481M1%WOUsB_x;V4YiZ1Qsrbi3&*>t z>g_JFV~S><;u|pL9+BHAmxy{zHUQwL$^`UQFKPXhvdi?lb^;U7mygNK1wpOMCA^5> zo=AeB{V|s)RWmFOqo<qvmj`Uey0z-s+S=S)(@4M_o?!w;gXf^$>(8B2w?D(;nehL; zE3<wPupJYl<VNm4LHzFv{zECOQ#!wtk3pt3bEtWFD#(QGUe--wE>IfuuiY2#7R%!s zH&rTN`_$g=T>2>yD1hFra0<h2?fv@lGfHgOk9e=M-0!nl&b5c}B`o9nac;zJXG*>V zi`(!M2jRlhoa^2}y(emXgw9zjDcMY4@}B1SEI}@gIWl%@Q~W$DG7nZA<0Xv#x%-E+ zhNnoh=~1dYthmY>vlSlgZ$P-N;WT;xV<r4<-d?|9@yYJNxyqKZFBt3A7%7}<*VCK? z63jUk(|5@fSs{=?lxY+g<cZ!FxJZ`WxdmgO5}LW-mK@BYyc6+h2Op12^cJ4?+K&gY z4Z}$?&pa~C|Gy$>YSMM?rjZgOb0p_krp^}VV}v8uwHo`AHL0xbCR|@i^7iB@>Im&U z{Cc@7v1Yi(pgY~)<utRTaIiC3{HA#t{ZDT)^4PWx-H_rLKKVVLyXYKbgXP-_D$ESV za6Nt#@(bs#+jOhXVg0W7(J5~$4<%7pliN+Ux(*}tsM7(~u_NlhIjI?51E=8`U;@kn zh~7%U##PGErWeXek@Mr_%kk>y9tBInh>l9-#g^)HXBEXBFSbJAz;CmPX&iai!Xu^0 z?|&g?leau#V#^qk)?eD+Y+#!kGMPVjc#Netv1l(1hWnkqC+raksoeFsQH#c&^m73S zHdgUcHUzRL<Bkx&%(eKbL^ZTWK9m=o%|_pzSmD?m8dJeYxV+73(|5mX*>ZtB?15fh zA2-<^FqKOvI%~`%=W2O9c?%)l%Vhfie>=5B{X1jqv==wk%70ANsgH4w-mtWdv`w$M zkoDNUz@7tz9S-C|*w6r}>(KKG5OF+uzmuwBNW>!Z0StZ-KSzDI#byv`(SOb#OW3E` z^HuwfRpBRzaG$c}DLXj;XcAj6(cWamEaCCaf;b`H676hZWL{olFV=ChzO6MBcuK~( zFYOIfm6oVer17x-gdp1szyW7?4bRY8SjP@})B6gyJ5NJ$OA$jXzT4&YF*8J9&tP7E zD$tBUaW=P;6o3fojMB97!ir#IMUFjVs(aIz5z*b@q|yrInO!MtF%Ok;*v=K#;UI)& zwMZ=`!3&*{OL_Rb+$4UYUGE38?bbp&)$^2e7@5(X$MyzZxQ<PtN)e~;5)WYDK}{oX z?I}MgW_>=ev1i2Dw%KYy^r!5k*v8<0ltSIr4OuzA<tkDswcSt{sfMcrG<?i$Hgs7+ zS2)U(HHSw=h*Yt7>2#~o<tot)G@?LSBt(_s;S@?^IGSk?P2u<ClW32r-|mJL0csYZ z^6%QssoDD|Q)-_KOxU+R%QFo~g&TzAj8C}DWXREV7qMRBJ`TGHv)W|q7ni%A5%bl` zfTy`@!cBb+kTgq3g0?|-y)?&*Xns@!1}@x>Ysu<%TUKS<4Xf7U>6e0e@2kzQYBvaA zKQk=V)ox;itg?quth=j8)t1~njW%>Jp~|xXL<PHu<;Qt7@js?@OeDwM1@UwD!Dc#- zYg8iFlbGb$#Siab;8gosbX)K*B?v*Bm$2<we%|y=)O)vA<ObeUV6RAs(|U~7&c$cO zq*N_g2E*iSI&A9vCKA#K+rJp6!mr4|m6zu45pJ4~9U8lgnfNR5AP3@!iAz**k?H;f z1c~c%1?2&SQeaIdfsUfHEW9^i8g&Un=hPajddhrzbspP9TNqzAEaiQ5v9ldU-Z#0) zhg(RBK+Pu|@Zeo?9ahV$uopOh{<-iYWbxu8wn6@^Kkjvz{ZxR_9ZG(H&SK6CUlbhk zrlby+<}FWzA-hSN_{(?CxM9bR`g}_l)x6m19bbp;f2xr$u}r;bl*4XKbtxU&dJ8L` zfVxEBaSmo^-6tbyTUlBfH&E`1-Pf}<-NAbH@dok<4WWDMwN8onv^9AKkK7+!AuPD} zH7~ym_LMwMch2udPp&9q+?EKvn=uC1sUJ=lPD_IWE%74OLbX6=55c``{34DBlyA1> zugkks+LL<^9%a=!7hiVI?)&DFioB&RR#XuYxDQOP7ljm#*UNj?Q$6^OzRV(LUJ53* zfNiU;@-9!T@g}ybdcJ^<OT1|sC-{*AdjjC2<{$=pf0&k9o?k=<$A;BV0&mpXCsn+0 zZM2wf4jJaNLZ(WMqzEdj0wo5SMn$VLHSh*Vay-4fimq66#nK@Tae>I<qVZKaER}WP zv3{1z_I0si1+OCXL0oZP_qWJBApY{5^4<eQmOEU0q}$Y=J*ANINZXtuO4r3s@69sM z#&{mU)iTTTo@~OH1q|b4{DfsJ-KE%xuKauH>35wBs9QWpy-U;WR~Lz_`5;GLEGNA6 zUdTzhxc$NyuCj>qts3U5tl?821om*HUw8FrlzP{?xbWK1W*SVNth!M6WR56>acNMK zb?qMNxr7CjYB(_qBaalGSLVa-z7`&1q-bZ<Rp4+0{%g}0M@L6ON;l>6=g%qIv!dt_ zom>%f{BF7EnuJ^y2*Nk2^0TZ#?*9UIPR-)YzXQx4FJi}<c~e#Naqo(?>}4zuyANZC zJ2Dafxr#X9dYbvD?)dHym*krL`o|;#L5L#+8~92_e>1<Irx%hhS9xso_-htUOkQt1 zYq9*h*~RiZ^4y48x7y$`Y(D1=!jbZe6@Ihr39iz}(w@i{qN+V0dkapRi}JovJyV`r zHoQ<hriZVDH~my5DY`Z+>S~ntl~8iJtbdvFt2b^!8+@30s2<5G9NTCY#BrA@1UeDX z+>;qehMXNoSFkBX+p|TRx!5TyF>V8MFaZoWl%LSZL`IqcXQCxk@^-K?ZkHJJN2gp( zlLPlSH3Rr({Z089a#)&3JnCcHUv5i|=e#Ub+XMI@PVZQ{^U3sGZVXrE%QJI-bd{oy zR79$E&GOKVZBNSE1z<-hl5_;W2@S_FF^4w<6UtxbcJ<7>fccg>V$<B6C_Pr~E(S%j z!;A8B4U6R<ppxWysW!X&T(ahl%{8}>i1Q;y!xrCdNy8&qI7^V-EtJOmaI?qJH5PMj zW8QpagUFZ`%3S`$B(CYaK6XBMi@AFAT{M?+As}=u8R{774Tf-}@d0(taTOEI<Xm}M zxjzfN=0e5>0QPBl>^i!{gmqi?%^r495TebJPI>^jaEdf1Yo2B<6Mb*ao#Y%ItfzfG z%FN(CQZD34(^>A?faYidhp^8ZB+FzGUTV2U^u+nDV}LH(OO#OUQGC3Be^K{b*U5#f zSTkH<;_Om?(F1$OZpNz@{pO7JE30f0$*n0%Fo38nJQc3y$x>B<ukj@6QWmSN{D~M7 zLzf!+CrK4e6K;tWZ+=cqgtuU-7nde}1i|Z*cJ$+r%LMm@A1blzm2#}Yb8|k4`?%uv zBZ{*4OU60IHa(K_SlEc4EeBfRQ_}j7MlIsB5$8<rPfK6Ox~UqzweCKMrZW8usmDM1 z$)te!8<!pXImP<a>i9J`z>|9#9QfkREWPm8?#nr^s}o|>mMgwXYxp>JlgcV0tKgA& zTGdi^F~{Wlaq(H5b))5EjG4%oe=lY;GQrXwGd=gM8cF98+w7nYUbIkE3<n|*y|fG) zvm#E$_7zMwyAT#<xzrnD-b{ZS<ERF$7JW-9eOu7j?UUp7_a-K(>h^m8DVFA0XC5@= z@uH4}ntFDQ*>eY<X|EzXrdmUe=YdMn8QGNuTo(;MV7$r5cwg+=QBPV*D2^RKstmiN znHAZv?>sLN8d$sQo+IU4#l5<0w=mU@rn`lrNb-B$+J}O=hhF{taMVkNYfcA<l!=k- zyAF;Kd1^jZ_0}p>`Tl-;;voIhmcHE4ya!wGdfJ&q{X;(;TjvygVp-o?ArnqBClA9M zB-&kw+PqVr+YFexAGVoyR}a77fCHwi$T$G|5&2wdE;k;NQ!VjSp&!Sm>ljhKKLi`Z zj6o42)efaO6Qre=PPgI6WzCm9J#n&uB4t+jWIp8<1vT?EJ87eM;09)oKKGV`#w%!w zaDz-^B~Tz7dH8saM^9G%O><S;3peBbk;CFHFPQsU=;C(N^}4}+T!JfBnC>u}F#5<e zYjGNVM<qO#)25-PXj9>e1d6=naZMC!J7JHGG`KT!p(yGCccqI>iB)YKP@vDBz~AQ* z?H8jiIbJJ_A;C??ofpoj1Q)?-zRbghkxc<+s)a8Lsj1?^8rRUvl#ivBpI|l>1CEKf zYV;`8au3p9fGX{&IJT`L1~tRHEjy_1;z->X(EL`}l6Ca??=Vo$!>K7`{yAukJAmS! zzHOrTM+pfD$GNd75#DPnij)6<0e1U7FE8&F`j>7c)yG59_T;1o!<jw}_#U9|60O{| zl*b~BVYE$c@?+|05zn&9#vUWd^O)fHirVhMhzl1g(qLjF*Kq0?u=ey_KeYj>a>h<i z`mlh7&(NblVW7{LU$X(({V>2V$jGHVD3&C@ge>Cs#!6XHisqu?CozQpfaa7j3&-YR z7hlkGDMovp^^d$XmwayI+WDU{UAp8_LDlHF%E`G~bJe)z*((|ht$Xs#yl|KL?UE4& z=*nLQXjl24r0O6Lg%nSL2JN3k(&P+z2FVQGf%L69A*d@Xa)`UPZ{L2IGIS;CxGVE_ zmKm<&c`P?{#uMnS#L}4-B3WLxZ6-{8uCU5FuS10`P03tzj6kM4&s3!YCeIy~#fu5c zfOW;1unT%WApr(iGoi`GF{G$?eBR44px4LQhC2~ihL?lA0Iyj1Qi<7ojgk{6D3mrL z9o*Y7I*IqdUL_9AB`TW5+Q}{V7k}%=U3ynLA-q%Fj954}ek7MgKMQrCEvma_n)BxA z=O>poN-CF_nMLMsKM&F@JxYLeus>JSNmV>fHC#1J-*zw4#Z+#k!^%Y`q18{-gPCgQ zv3+YZ8H=iJ+e{B}cOn)BoOycPe8DzPk}DYLVA_%2sPVr(0E;iSa=cL?cu{&bn{A?f zGk$t9Hm^!OOA5avUF7QW$d6g}c;W#|Vppz3i9D%+4>ndY-%$~21M4ms?@q%Q24ORa z!Gx~yk)ChkrJFBHSW)4Oap*E?WR}0`s4};rWErI+7Ujv3W7<nDY&GOpy^riv^>r;D zHgo;|nPZ41FDQXXUGMs>{(tb#vaT&A&uWY0%PzOT`J>e1>C$!=#2H2PY|j^oKKU#L zi=MHc1_aMu^vL;&3%=ZYN4GQF`_^WLjk5c0(v<tUuUnV7K=echNrG*%jWM<xKel}U zK;lqE7{-uDhTR;2hlax>n%i@#;@>Fv6xomIVVs4DusXE7_k2<7Z24=YV4Jpp1iyo9 zO*$2o99F|M!|yf~apYu{xeg*@qqGU+ZlsEpbpV=0I0l+L^5HOie3<%SAQB*zLR99j zdbd7ujs(_36Z^CCM&Jf+US<~qC~hC&@L$IxyoPuLWw&y*5{9~3p6u*wkC9717<VHc zUjJyo`9_a9SM+1sOc9a}=E#{3!fz~{MPyAoQnb{H|HoGf_Ud90eSH@RAU?hT=MV*r zVBVT|IvBCTZc!fNT<v`Zq&Qy2t=0DeoUxQ^$xSxS<f1~j2X<_vuXV^P1L@(jE5Hs5 zcv!W1`_LFPn^3mzsH+Y`fi&>lZYHWb)svRy6^?H-r!xS{cdf(nCIH7?WXV)Q80_HG z>^tF<5q}0Td-G$4M(`1}2|~bMjuGNA3(WcLp?t7LtnO@=x`eI|;LSSP%+5C8YhV#z zG5PRFN1Fogfu17zzQskW#p|``BD*Gh^_TKoXKbw1OsXEah*XYr2EhF$36A^%*RTPh zyC$7FUk1t(iiXZu9A8?HGqvl;AUs0mQe_{y2Gczj@O8}dodmBj<9*O*@iiTlA8_2b z*SCY>VA|}$c5(1`FMxxGARiB{C;G=x@f`puuk$(;76)aAJjU;dC{b3><*~Qe*dQ0B zxM%RWZu*XQp$~IK>2~8gKXYDn4k;{NaOtZSWyYjAci>G`_nj~=AKFtZwWl$D!n2Pm zg+sFaAaz#j(Y*lys(Y?$q_`OFe50CF9<_`yi3IZnG0C)oQb{^fS25oOTFVC%foIM` ze_X1Fd|{=gw9U2DW%0+GyXZ8eCrM1^IRng8!KKE>+a|72sU5b#GM<$X3AredsYW|x zk+fH=Z?9_FDuW0O!4UVOjO|<Z>Pmwkgl64~*?bCgm%wlP*i~BHy!I`R<)IhLw;^Ih zVokkNqZPkUUaoDIIj9|!H^QCi3I#<Z*<A6knsn#b3A<|V@{YN0>cKZO31k?u;u<!w zX2i>;wYsUOgl8IqhgW>*!BpB8A-ko5;OaAb+XqO&OUD7aL}&X-+A8h}zC~a{G;s2f z`bu9126j1S2$~#F8)G{+-=Rap(fwJcugQS-U2G`hTF3i}fV^{ait>I08|O81-|N-3 zRrDF|rx;}S$QUv!Z7w(CEJM;cVmPmGFB4nTIUagB3Llf#D()f~P(h{nnCeAKtzu+R z>jAp+mTD!(=;D(#+O=XiVFn>Z$0{w2{gmo0yQ@LMZC&!{t#?JvQdbfGw7JUiaJ5x8 zRSIKZ1Lp|uTJ~|oUQ{_pBpb&{rBw(&U}xoGr{G?(2%^UaY9K|eM&(bqamz&4P;qdN z0>36z%E&wB4V*0M0&p|!>ncYnVgO18-1OvQ;Fr8_?lE=s7hAnI`59M)$Z;nwv-7Zf zBJNIZ&L7P92ea6kj{$|G?@<RPheNVxwVTv5fNY{dF9L}K-lyyf1zQAf=z7g2txHXy zvp1*4Pq;bzmUt7}ijl-%qE|&5k-|Fe%~C^h!;X(sW74Sbs^L)WhWU!(7}O6x*Yjr- z0*A!jy*r2OJdT!~7K=x49^45s>uHKsrFo;mc<ent`E@SkpM{ha1I(L;@pVnwxM{|c zd9%`Vs!=cHq6ouo@yaI^j#?sl&#MCF^7v0gkH9Wn+T%>ssAvUHeNF^*p?ubqz6eHF zl<w{x>Gjv~h@$>}sXgU<!NPk!=6RU&@67=?z?!|Ha^Xhoe6~dZ+@vO#MFWpR?t;oW zUAb=(LPx4aD*<2_#(@{VR(!Q`rcM_HFHjG|KP-Gw0}RWP!Whipu$L0jOgU@rIC^M0 ziwR-D{ZY7{$L=2GxQV)u8J7jwx7{A=`zH#nz0gzu!P3R$0vD|)!l0~BRWdFzW+h?L zuh3jxDDFGo_2dDjy9W8I%q>Z3PrWML1@03gyI+n-&~?u)0xTc{Eu>iuaGvI~JvTBf zRID&A1|i2vM#{af53Ad#SmkwhEn}>G;A(O&vf-fQcWb%`kF=<O_xY6PRCWDW_J@_Q z@G4xw@r{=va2+LnY@~_jv&#z;0SpBaoO6-nZnYX8IeGx}aLflP9KMeE%N*Uv+W^w= z;>)Wxw0}k^Go;qm*WLUiv!dE4N#@6l59}%pyavg0nYP;8A-QZfov%`IS-D)UMo15r zkQU`Vgh80VI*@&LW2~JgD!s>G-I$r0W2M&o>Um(M<PyM@pC$xjC@uy!f~qDa6p_>O zT~);78kuX`k-2Z8xaC|Lzg4EQ3dE$k2s2FY<D7et8JOsrT8lPF^X$jXTL|mWw!ZxK zDD9vCZEf^%e4QEI21Qujo1H7Y4fe-VUVr7IO0nr5mXs8b=~xc-&LHX%$4tt6vn9lg z0+^A<=q_|ui!$<U%}j?oM}Y9|9?N+(cd4)h`HT`s^kj(!sVPf}%hk3~KHb=9Ojrqp z)d&>DE`J4?i5xkpM*SEv0b8b)#$Wwb!}ntO$x{=dvO8NrLmri07%HL;GSp@9gKUlY z0DkUOY50jDJAPU(|HoyD@B&s!XssFo9a94x248aW^4J?e{jjN-T^ITR_u*y}*T^5* zm|PZPdHSK*T2rp$k<NH7N6z})#Zk&SAkZBP>@kmU%|7m1w5K}P#b)>3uNX6D-!df0 zDjQ=MK<5em^;phjm-bN#CTjB!C}+M_whq81J}%<N^idZu+MC`Rpi0YZ9;+tW<zkS& zg7+{VX05p^s!7*%Lr_l4+qQzXD;!xxtwv1##o&MR3eYpaK}?=|O*l{EH0Tgiu8$I* zB&DR%$wwUm-xaW2dM#=98d_mgB}gnz_30P5-L&!>8fLEjuv4q>xvo%G>ufY{KY^m2 znKxBH_v`16`3ta&V<45Zjt*)w8_#{6WXVsL>udwl7$;ILs94p0neQUDfGm2e!L>ZU zpC=N7zaJBy37F>i>#lmwjov=bxqa<HUa#3e-wyG*wQOiq#ZO1=5KWapfw_ldC-Gu( zr8qV;CYP7iR(^9OwukBIuZ-OK{omh9nNJTfYA!cF@!#+d|GPJdTUsXP=%86>rKr&L zDE4FYMdLly!5Op1J^gHzHJrGUT+E}arR91B?9Maa=qa8S63G<|=GG)`r;D1;er?@d zIKDe@T%xjPcCm<5(6U?{yioh`?qsTqh8xm_p)ysd{C-Jqh0l1iHd}>HL+glUP_om_ zdV2Y>C69QUnwjTxh$!IvA}~G1FQ+Cm?|vLp;#c5}*8e?T8dAg_Y&i4VY5$FnMi&ux zc7VcypNWj$-YqG{E30XHk1?L^g3yJLeUIDGM~Wk^<QZCS54Dg8R1?4P+)gLLGm-9U z0K>}|F4W?$>c`$f-*TtiZHPvGJ!1oOj*(&vC=5}Lpru0Mf9z7!j~`yF%fdu^<VG7W zdFLLHUxt}BHFV?Q;w;$cU)lvE7xL$c8)N>{50Fc4mF+4?Sj^UK6+T1n28OpCm@8ed zR$`!MnBL2YWP-XlgAJ}_@yBFEj0VX`XKz$Z3(@oeZhp-B!a8!K9TaMh3GLL^IsLbg zoT4;C>3oFBG(yU(BV0{*nkw-lf_7C{*QlEX;K#N>j5<tXVM~GEZ*2=D4Lelv9YQ#A zUJG8{0e7iIr;6`2sw<s&K&vOe<=){BMv<GmNsTtx*q;Zbid{(;wfC)jm4CmC;i<A1 z8PC=wx5g4JZLjk{F$MPLo8_167ZKRPv%R>KnKNz+!rKeSHw|cU+n?acO}FkHSvS4k zE4x7+Oow;@+^8kr-xiC?VUFZC5fT%4o)8((bIthecupz*7DU5NKC#k@cJ<V(%?da^ zJUKzl9uLNek^@yd_2?}FLzc!|e!J1Hn`rI+w+BSEvjDLEyh@f96ngBH0jxu&A?_a8 z<E_ZHM}Nz$!xJI(Zm30MHebt2Y5amBT-dc$>@vGy2O2w*hCVMnkx&pT6u_STl@kO7 znI{ay3AY|Bt|0Bwi!1%O*NK;(Z&Xj@1{G#iuGSQ8nt`Guw?-N>mCe8WDXGHu$8V!% zSo8yaTKw={XipS^Mvm%Yn7=i4xv8iYE#&JUzYa0{mbE+s!$*1lu61iszXP<+=pU?D z|3iVjoE%IR7cQKcuhWYERbMB*S6WJnz4p_F?KvTu=0KD3+`HD@i<QwtOA7T?;<xPo zs3M9ULBgMzyeAr{nzi?+`bM5EL^C&p&OJAUL?a_Q#1W8s32}dRq5T6r!zM@($(=wE zfA|1JO#NGl*MAQ*e1H|`34dw#JG%cC8Dl+23bOA=-y7?zQW<(TNTv?!y|NQ%kzxe& zZp6ruJcU2Y^7n+|Ob}ohCjI`ZmgjjO<|Vmsaswa7tPf=*GY!vHmhg`>z=g4-jN}Rd z^r7y%7YMZXgAo;eoTw8%=kb4r1zp&VF?RLihF^H49{(itR^CVNeaJd6^7mX(VbInC z#W&pRrWdp~h4pQlCg`ftfg=K70Yjr}rSn6KVSdx-Kn6fWlhYqQe0Vqh_YQ^sHz`Cl z=nSCj^iri=LICW5E$?M4HncsTmI<16a!2mdm>h`25nC{a4<Ak=`xFp63miC)3h|r? zTY(yDNA%B=f49?KT=y_$kmwv{W3U97LZug}aczuAW$H@i{gA|^Zhihd->x&m|2Wrm zasmCir+^|yefM(O@B#b41LVgV*8;>;rm!k=D+`NVauH4Ur*dCMeEr9H(0W#|g8APV z7(VhJ9wpPx5V40LHA_Sql17W869}t@4Y3A$wu>~!029!Nj$T_ySz<24X4K=gz#f!C z>MD1e?o1hwONBjjp-SpHYlA+i2hdfh>67)k$M!LWfsH7SM!SkeP-S~V1Z-4`xWGd- zRThPHt*oqw_#ZTnW@H6#GKj$yq@3e<0M{`Iftk}v>-Pv!E@;w~#W&Nh(O_tNy}&Pw z4X5)UE+A9~k^UyR>1!>#`J_m&j8&m)UvhL-gk)gPNeliBpP>Vup<`r8`*+PWK(IXt ziRb>g(iwP-hlBq7#fv6aD{s9JT_b~L1ChRa^^_K%gP>C6eNgE*`W#p1hMWRt2P+gd z&-xhBJ`qJ_&b;Y#Xq{ROYo{zJosCqkMQ`1@b)WX9={<Ij*NkA*lA-nk)M_-4^OKuq z5oOFFa2G9}J9*oL#`{3QV1`9fTAJf{{d~KLLUdCYq=TyP)bF&7vVU~L_9<X@92}3W zYe<ee0~O%9HEgV`o`0Ss$k7Z#(>93-_#c4!GJHEO29jhY9}Co!tn>SS7Q7f}fM2!) zYJQ!QI+O+#`jWG6I=x;ev2;BVs*{+%Q2#(?psM&|$v{=B>FkmXWM=ymcpwKq@2GV< zV|GJ@H2W;ib&VP_PXXAjbZAP_8p54>G5Ci4x)|+Z{#M}-lqdujwimXp8)Z8vRr#sy zsS#VXek(uIuhyh!d3?h$3rkB%zDi3=+fn0MEc#e=KD9-AqvzQ6fd#=k*GI+vdGg;E zapw_k*&@iCoL4~TMx)?0bTnTOkwJngK@6_nV;)3PWcK^gb+Z0jawtzwelS9-5@^dY zGGZ&bHQ^QtF{HHfe6%mWvQjnkMxyKFF2m4=-5R{=Nr`@~+#AlM6!gD5q2;?d<3zw} z{J(l9==X<GACCK$7cUCzR*ydGzSy_1^~GHQ)^|Ed5KbG`hyMbgwJg#57A@XNHxZ#q ziiL6Z4>0eFVLh-_*obgqWToMgcBWwNakAj{N(ndW4$v&Bs13ONr{sNVw3|MdLs#q4 zmFw#T_|GSxa}so~KYi9>L?b!dJm3wJvI)l^T)lcswx1chIPTvY(<S=?CTk^5!V5s! zdx-z#*g8D-`<EV409N9+q9$mx0P%^ZqX>pUH0J)TN*vXdyZZ<2VL|F;sm<%E)F>ww z0H+%@O75HkEiLUm{A{$2;2&2KCk6!O7w>FGE3ou4B3VO_Q(6!JL<B$jSeEjX37eq< z&9DISImnsZ_TJ&x{LbXg69@e|nsBm$>pk3WrMKM;bt1a8&piA6$?t*i1O39=XR*;* zd)y5K=zgsI03v00XiP#fPyll1FQE<QBa)vE)R$>(u!R)E7E)hdch7ZRh9A*tsN&2i z=Tgkfs|);zMlwI&CR%CyUL8hzXcvrqMmy@yCx2fcEVzI-F|%(`6g@4HW%i*Mc&F39 zo>s5chY%DaYoT`iq_sOxhYz=KQ@~VI#rJm8+-lX#3PgZ1Y%RF9ZLMO3p!By!f)0yc zc-g2+HyHgnKh`MrndJQH&u3q2x3Z$ZY<xMMxf+mtJDP6CI5^Ssv9G%S>f6S7K&E3u z7iRP)%KUwy9}YE0ewzl8i9Sa-fN0?5BMnVHdGch3))ktCzQ(9?JEp|B`xr#XiGz1n zuYSO4Folu?ak6P3XZN}|=>cG8Q_thw%{w8#%~jdD+EQ%qGCqxs<zU%`t^=naTvZ7i zePL!S_ojR;oLlwK9pEOsZk@EsCX(+39s?zjP0;|*kA6*iuzI0v_dy^X3X%XlYuBP1 znsC^Sv!{*c@(7yCVBzlXq7alnTH3lg(1N)bst^mcnoB$PUKHSo)?Es`8<w8EYVfOv z1o{gugY~(5lx>~1Sc=JHvAQpx?8y8KcH$)Ej=YlP2I@{f$dV*rwc5?1cCx$=_~1KX zDWo3VWVUnX&ah=w^?*042eo$T?V)W=?G7Denr9-OZRay(Dih4Q2vEjrH~wu1b*3qi zU;)f*OGcjE>igN2Glr>UX0ZsPH({p^2TA=W64;nmDL!FqQe<a(zvPN?q9-eOR*iIw zQ9t9aqpNk<fBSiBYatWG#uDG;Qs?es?V#$nZy$P(_w}|u18-Ma{ngd;k(p-zC0-3( z|5Ixx&&;WK>hGr>-obE{!L5Nj<@@Ye)oT9-cEHrtqPdPn&)xMqsMM#R(`Gj=>~M8B zY8|)tI~wspyMQkqqO6wD%40=Of%wD{zs-s^0Z2cSBB<_iyq-%nn$Er8L({&;U?&;v zNPnBwS8zn~lxk!75n67<SX#yfbYwitVO2BM=-o_vgPHyJHNGEaR}%jRRs9FwY-<f1 zINobGinf&;?7no%63PB9JKO3+N2A4>91N2(CEmAj%)1~+MUFfG2+{yLA7+|}`Db3X zzo0M?#LD*PlYcJa%%L#xe;$93>soNwcISOpD#6lg9aI7j(7Qt1Xfq@8*SA1Yl1+K* zv|m;llor1ly`f+2cc6#!Fnsy)MO(WbtoYweKD|5+t+c`F_d)<FNd>4?fo!Y}Z7BNs z0wgvbiiEJWNiNu=aX>~gP+$*^ta<<$DDVRJ;ls&(+#3_(30fKae#qB*U=EdE3*Bgk zDk}la!7vv=t7d`__!qq*@D9f;qF}kGdi*~~Av(k@um-v7mI(EDD`<Ef;?AVdE;5o~ zzOHKk1!#u=(&#P|^<vt`ulj+2IH<^*uJv%hLhUsn>N4(x_U}3V{zew#z#eBJD=;=d z0-T!cJYCHqY<>>5N`_UJHgo*$1z2nBUlBk~0{U-3GQL*v3#6O#ZdSg1do*)Q_7}cD z`ZDd_0F)|Ap~j{=Z*)KUb{9L}I_q97pPcaZQOq0KJN-MeGeBs1A4X||C@0wGU&-no z8iV^%_$F#$K9pU%WVIJs8Q4ik#lk})!$CSQ{Sabqw7Z@Dz4f~9!$4TE+(Y{X#4ZSm z=!mw+(EI@Lh6zgiUpSqcRaBR7EMC0iY$x0kP5b*ljuN4?6ChsPCk0O$Gqoi#>X@oT zOZV3Ey4y|P?(R(R*+<heHGQy|T^Q{k1X5<l@xH1pw_{RwyZ^UJL_!-hs-LK>_OL5r z0aXwX;5-ANcB2L1Cm{oRW$U6b;ExA6F5O#3;{|{}PR1upOv*LpsnhzwKXZKqAuE=G z6plZi{Bwa20?gzXSL+K};oaFD5sm~PUhuE}6=hg?#tDPxjlWft<_7>7i-Kod<xa0y z_l#MPG5nu8B*JmWwcuecyNLUfphFqzP#1qoi&s~va7qGTmJ}3n*FBi~FtGm<Mlp@f zZ#1nB!`$55Ca!v|g=+f6sHv%`W>z%dyF;UKRY?<*SDG^VHniAB(BjN{yb518m&>4E zSOnjHrT@$23vBk9D@|ch$4zr~#hI+OV%zi;<*VvxY5D7JlV<$k<LF6`wPRW%NX;rR zxvCfY)_e%rwdh~ACOm`Qihkh8lD!n9G2<~1AvL2}KXTQtB{EQhEQNLdIE;X~F|~t% zqgB0w4bO!(oMh`^Fv;HOjJb7_d;pQ50*nwPYq@LnU*p&ji~ZLjt^mF1)c_t!;RUB( z$FBa$suc?Xt#=@Y#{?1%|FN=vE>a)>uvbc@=?-vKiTZ_?R=F6NB8JJcMSzIV78h^@ zj%Z>!&=RnfjF0|Ly8q|NOHloEgmCT82mf3k?raC{;bpqEgXjqWMvjXfS6Hn;GAB}& zEObV<^F(*|<I;D(op^I~!vMtD!3SVwl$)0cje`h5y=RWJWU}~>a_8Am!A=N_vTgSE zUHvW39RRJVzp8Z$joJ5DfGTGRw9sSQe<%d19!dxjbNA8OG{58oPME3)r&6#+C4K&| zt`b-5kL2;7xe9_qFv+1cm^v4nA*kZLZMm$wckF4<7(Pg<mtg4A^+U*DxF4N+bM#Ba zBc)*fwi->L&oQZ>ai0{tU3U#nfY^%wJrckn7y69%-2#yBRCMxxq-~7Hpe)f2E=YS; z0w@3_Q|FUg+stjAt54oB5||lkbu;vc3Fz6U*3V*e3N0KQs61ARYvIxrIAU1vtV5S) z@L8PBt)J&?!M2qf9CxI1b;&gBilrFjX(@%iH9M0X27%veZ+JjM)UVRDX~3|n279=? zU#k|jzZk2ho*-Yv1TW$UpCyp`3VptIu1gyIq1-KZ#FFBYrXFW}4dk=n8SHx+wTae% z)PV0bxL2pM!gGAMl?rj$x}M=QAz{@MBca;a=7(0R6%V+2L@rlpe;9vDLf=ffCwP;1 z%qZ#zTHsMoKX1?;Vo#ZDFVV$EPlGzGksLJa5G2~`KaxwRz%IEgH2PFOaw!{ny5q^C znkAwuhgxH_ph}A${&{Q?sW2n0jzc35wz-1E!Ia65`~LHrYvBPWpPq=&)13f#`4pU7 z3G{$xFgG*X{EjxgU1Q0^kUAz6$<hkLj`hSfa;_^J<_{Dy_|!e-t;wD(rwL?SjBwzj zCxt|`6OvC01{=2Y&28Xb<pXymTI4YoJx*1mMk|X~fkhAy2-vA$df&m#Eb7kMk?JRF zQe^M)y#tM9fuw@c(u0AjaN`{GD#5PtmZ)ujFl<8Y-wi6<tG^R>0KqI+Gc1o(x^gNb z(fy1vWwf9bx%e6Rs*YVH$NwW3qh9bNZ}0=D(jPZHyPZn}JuAuXBbI$rPqj>kEV=>P zRkTZ9Xw5xEWUN;3v4lWL_8v4|_P^TtKoJG1YFgsPa#jxuNLN$uN3_b=DI}Z-YfX!_ z_HT9-TKcJ*fnCVOp5X}mu}R^*WG#B!uPLDAB(Ifc^JHp^`^*7`rh>xPH&yit93|J@ z4oW@&?x>yYs|qj<wEyOGj!h}ZW#jCLvqKp;D)7oV`?XWk{1MDpJy$iJ##{CCB%vUA zX_WpESgh-ur-Htx3KbnP#LhSRY&MPQrvG_Qx?1r$Snh<D@CM~O;vW+Z5LMR>DnxT1 zuvc9u6++{8fOz55ux8QLthlH)@FPcuwWQY?3aW=S5XX-_<&flOUp`sS&%u`%IQ5AT z<r-+GJP@((SntqXiM&%y!OPPe%Hv<GNpEK!##ZhJc@cQ)bip~X9*9TOhA3%ri>p&= z?I->=4m2&}Wp5)wt=yx-pR6tUHwk`gb8@iG5sJ9Zx;*)hS;RR3`F%<fuA-5#?BV?K za^*~^KuDMP92`65o?5z`PNwwWTO8D<pJ_!XiSH5)BKH%96I=~de8%*BYAC&}Z_q+2 z<jW&<!pjq$gAa7`U<+-?AbB;sQqyUmp(^>Vp$~25Kbz-udtdO5kS6O4z38YFt^UE^ z&09Tj`<6Jz{J?nO*UX?*aTIbfpSUBMb`8izK*oL>=lc<mAud^a6gW;LR6h90wboGn zObpa*mmz-NwK0ljo1A_qYOaR5<oWqf#w&i8(tgcgscQ0omWW>>F#)@1Up`U0wH2*S zdgJCTJX!#!khHvTnWcf4iaxIfZ-rh@WB)Oa<b=GM<%La@5A#X$Umc?1)u*v;0O6eL zA3QA_0}dKn!N7l%=+VCBppO<azz!y-RqrQi@HxBtBgOvwdHn@wEM22hK$Y?d3_d+> zXptTY!6<kERi?4<Erwm3C#Il^(mQTgGZ#>!(3qz^*ZIay*MGzk8!t5fQ8&*oQh4#l z7DA+K<0;^-8W2m%gGS5y)kZIIj0BFIzFun9>}#Uv_izCzO*0166;{9_3!C&S`j>$n zxTLY(fK&2=<g))jwzgA1+G4JTGZAZt{<RVXv!Qf(emW2<z~>N(FjAi%^g=JC3NIb3 zTAoE3L9U^_*C$c1QG>duB|ivuh(IvllTncS0;!}%t>bXkK^woJCxpJa>N&-XI5Hgg zGV-Aot+fU(V+xuY(UT&Df%mR|tVJ8k7b$N{Bs_!gnn7tljh^N1Bu|8EBsQwe^=#@@ z#3y>RK!*6Hd^Sv>a!=Lt^YM#5*vuP?K)Ypny}O-9##9W+^CLD?fBymQWZxg>bp^dl z7N$Y4^TexQ{d#dm3+!1UMhn{r;3Tls0tQE*d6RwT#(EtuUFyFN4YyS`W5dl$%wLgU zxQmTd57we6gaI#kd9EgT;9iD-4Ea5`kv^|Ypquh&eqB593y=B52T9aLd`yq@@dgg~ zPwm!ky28)L++!=Ykt$}}En9#6`jv9ooiR+;8TdwrR_Wyd>AcQx(!9;mFY}CoCiQ^c zAfq~H_;r$meKf;&g6Q|d@cykfip`(S_2U4q{0^_ZF2DO%4-lWAtaNn#o5~tFh{zZ- z1W#-Uz7L{Q$+r3(2rTog3$_~Gdqn3xxx+3-upFy4<`$!cBGYsDqT<o>2{rQxGgGlO zs=YVr*uqEWUWut4s=^75pDz)11>Uu_8@o(;7xZl$bR!Eft0235Z~C0i20ZIfKfr-I zc<!!C%n>>wAgAd(4za5o#m?Q{XNPdif@p`WwdnSZNO#~^e)s9qY^Bt%LK!IZJ_g~i zbQ3ND<;ob0?(AC}R>-&~RsQq7RgBVb9ApNs0I-eph?|k7PDvBq#K1?J;Tm&gO=$sG zN*_!}Ok6<&7^uqGck@irzFPhtB&?6Lmx;kMK<yUyrH<>kOL68wXBuzK2uflUk6UC5 zKvv+1*FXMuD3{sbxViKTB3T>?HgZwdKL}IF=+;Z&g2iKTT`#be@2<vLJ>466@vW6V zA&mfh`s^M9$X-2%pE=|OWI}PCV2bo_LoB^yol*lP?bU6tk@8C-hK7c2i3{kCzpQl& zG;7HV6VO<j1^%lzkmBjb0`>Q7i0z}X*4H~CvdUuozS^<REZq`bwDlj{C#=+dKCk>& zmS?+#gW%io(_+sbL$3Wz^n>vk$l;u%Q@+_S?>%rwEZwJS#A-xnyk>c%rezW>oa|CL zS^uuODTq|i3M_g7jQ_Xx+u1M}$mc<o1u}YVkrcNTB&(Cv*C8v`bT`r7sV;<VOImH| zYXSk{^mbqcuR5D)x=dwK8OGt`0j=Z9W8>^Pa~){)V^!3pap8bhuD*5rEh+%2Xp^Xe zVj9na=3|5XMsb5G>bPA^r)(AncsZ`~KY5(`a!Rp#DAp#)nq14v?%i7r4=5iiA8F|i z7gjfHz0(IWk~>L4&qvOsES;RsY?KP^*_W@qDoDw+GT_2usxa|{tF|{b8a_$@LZ<Kb z^Mj-MSAVDD`ieI=S@f+#s%-F~7E`}liI^JUG+|)`sc~(N0^BwMx%>-Rul9Zqr3yl@ z`x&e3(9ts`!ITm^LD{T>8Z$3A3;;afAEjmlFJ#=JCf`1soSk`g$CJC4@6F!>CqhVb zHPt#bjq^=i$G%9}i&@NL+PA6$AVpt<+LPj^Q3TM^Epd?4Ck@ClarlcD6e+A7>Z;X% zUI%(#%von9e66y5jYHKeb=RbA(ICIuqBA>-3|15L8+>3FGX-32a5=HKo4tz)`Py-R zOhvuAP5|1xa)1Y%$hOd|r6I*4qFZNPv(QFp(UTSwz@3!#Kj%OH#ip-J<I(qgK{;!1 z5MguGqF(*vfdS?{-OTzB^)J^Lznb#Of5I^7_pYU-1+nGpNt=O7hLh97FFYn?lX;zn zB$YOvP>Kz}aY1T=v=RCm%SG?btAqp6exH71oSRn^7m2CpbwdjhVuJ{=?43&=q=Vd) z=XyNQYG3zI)Iff#&lTEOvPJa=Jz?sLv@j*k+{(*Tc_gP`GX-OIMTqh$p)eI-WJaQ# zC7{YGug(~*;&V|uSPO;Cc4~u$TyQQE^M2j`>Vk)Q%YjDKtL|o6yG)?{!lO}13_fZn z%s(@JbF0*;jO3t&4)_vKrC6bvlr4g9gH3vrX1^Pw)eWn0#1d@M7(kzbEyZ#IkS7ok zgwK52674bDW}X4?s`~Q<Imajf8VYxPOa4EtbM~gTetv#u=M}Mguk<WGo0!bWqr+{O z2M=2Sc%S5e?O02BqC}Yo+rl)c^Gz86y(88q9jZuOu-3yRJ?>`{!c_<NIHIEkPof>Y z8zp%0TgVOA#fD8!;?Zh1Of<nExFj%q6^KmWD<Qle&tRQ;Z}foO3<$^khvTX;DXNDV z_?x;cp)Fw(lRc6w8&|xC^}!$;MSE`04AOpU4|RFYsHE51vI{tgg)U5Wac73c&%kdn zXpQAb0u(>@ER`-!eQDh!G!gQ0xr*rQs>f^X+n-|K=!JjD?fkA0)L`k$ja}k3h4Nz_ zExcV!|E97Uf*bE`4#I+C3c%iBZE;~JVG)f}J)qrcu*PYuvP6LIj{yE19r~7*NIEMu z0WmLx%+%=Ja*@H*A)Do^^IHB+Zp-udjJZm@TDZU@$njhkzGtXEVOJT=e^~%0*9jZq zlLE$48X)NA*{FoyQItIM+*B)RW>-s19)llBz51ZAP2Admg9<`i;&AQ)06+^IbSqs1 zrRh%JHqP-tw>F($m>8y+CdV_|8VIoWZWHt!;<uO}&jr*L_57<=<-_r?me%CunHTsP z^1!`T5SC1OO0x6H#h-h%Tio)7Qo_3-(~{1h^4#LI`-=?XJWkxpf`5&dm2O@Z5D9`> z6y7!AlI^EO#3cTsJ}S%rPP#bK^t-41_e_dU{3svE(=+n>Y`ht-4h)DR>v6#|!G#eJ z20a%D860XuAcIR#pL!6p!(1c!792P)LqIot{!uEH<a-&oyuPAmbzP4=Uf--d^wp52 zOjc}Sd!AhCXKM`56bOg^Sxfp_4P4)7(;F_DpnjN}Jd>gEJZU-+zpatVUo**1sGM%> zo~aH50o*_uPS6Wa@&U1e@F;olN;#%#tnIZ87C@}lRH4~St#0O{=rVg}V<x|L`|mUx zzEJO0DmAKdQpfATwgGSybD-1EZ^5ixM<^r=>wzV%9`3$IV?ybygY<(dq{5JN@O=7n zs>U)-qlJn0;(aLkJ-}DawCJ_k2~Ec}!prA}6I!8ESc_s8tt4s%{P6P_`A=*>83#MR zDXcY!W-CxL7BZVvjD{-U@mMd7gr}<&V2#CqgN644HqM|R4Y~F^pnWM19DF_VI79Vn z^9b{s>d7eAp2u7(%!72$<h|BejE%-rAeX-SG8vb>Fg*u|RsslT0s^>DZo^X$49SjG zcpcRZ9S9<*EB}i8*4=6H<RE$d8Q6h{MTA`gWdRsE6m$cYGQ&XA;QaLKb>)UX-hHFS z<sTs^2#2NXst$17qEg#zmfB_@!>;pE2U%`<Z1qgqzNP|z8MMwqX+iGT7d-H^GdT`w zup5c3MjG(5ZhmGZi+Z1kDU*3t73|wAPrkCSqOV`u5?-($kT~ExHn9(zJ`Mm2N_k@u z?`B6^yZ=`c9)rR`Y=Yz3?i^6^J#>RYEXe>_chIe>81;ppo*#J3BRk&)mO!Mn5z?;# z)}jk6=s^^~1mWpEl{i61@G6EOF}{|8w_l|6Kp@uJTysbIm(Ix)i(U|x+MdZvt<i-z zF$9SM4t+K5RpbzETEGiT%i-%+#eubmh4UvU%l4QW(xg_Jwvvc+<uMYmyA@MSDtgoP zc+8Vngh_`YYp*XZYF|P%`Gd3&V<?Bl;zUbbOb<0?`DaYai1l3gyYZ%fpgqZZ^t9@I z@sE!bB^GEZ3iaH4pjqt}+HHS7Zc>%FR@AGNGR_MC6N}_aIGU8jQgWgcttJI!a<{Al zAx8s2@&MvD3YNMIHHmgZd~kOksdy$rSM?=ky8c}dr6gFT3MeVRz}&z+X+yE+o_~$w z<6dn7eO0FmUeJ&jwt-NhQCL{$dG=OjAkfX~7whYke?~eD75-dhT`K>4vT|Yjh$#t# znluSyrTdNclIq!9!r&vLF@O&j`#D-6wi<Glk*a;KGm_%1DP5TKik=~<szh0hj}R|E zb3a^lKe4;Sbpj`mSGpGhCRND}E$pRcJcElq=bV+rMLYl1xD{`N$mIchbocoRSb~1p z6<6Tw9JB5O=T+{&b~^-RFYTN`tW}J!`KWL(O3$$}JiCrBM0S-_4jpbwt%B13PKV^@ zd1b@#572j~-_IYrPFcXzBqfpGi?Oo{n?9|XNs~^4xX14V(cw~<sg?`X1(fbCFXZl{ z5swhlRQT-=<bAf4`S~}_0*c5&^^`u@SI9hKT|djeztn+Ba)ST<e?<ahbpEa1y#S~% z)*jDR141b-nm^GFa`#Z!3KtIY#e39GCQU{<yyADQbo+TE8l==(ALrHQ%8c+Hk*yOP z)CrE(aq99ohxgK~4Rv+~gb0^Jd)YTJ;vZ#?L~UMc%(7B{djyBGvx+P28<~Os(WnE? zgaRx^u_*0&vGMx0JW2v5{w*)O^Bd61wUYSN<NoC^;aO9leK{zNgpuaUn&2+V<CW7b z&&ND7*o0e+lNY{2*2v!IK}d^DN2?AW;SXv&t%Lxkc9eTz?I;=9d9KyC#;C<NU7nh7 zV|<u8^@bMYuhNDPlng!CsYqKd>9~PVsF`mdyu2s1J5el)w}xQgDz1(f^yzt!G@Ij) z1_cN0?&@Er$TLj=ebaYiy3@26Q-tSA(Sp|&AgK~Sy~ZO}+rXJ0G<bXe0^9xmxOH%0 zf8N#v95X3|o&XJtzFd5}SO0O1z`*`^`oScTb<JIThbMf3=X?m5AR>l+^ak*$uTo>w z%Ogf>w@T}~26T6a6<b(a8yW?4wcd?3FPPqHL#~DDO&si#ATw5~0wpA*QjbaT;nz1W z8qfNRyLr;Q)WHd%8L0jic!TC=sZ-z(@lp7~PW@=(wfJN8u+z^11i&h05KDsq<X98o zq)q~NGWmVV2vq$mAAG1+FH~<1s&SrEpTszNsUU5H`;!Wi)ktry`&2@i0JP$p!po*U zOSCQlwBJ={c$ETr%+ayWabFr}nKN5EsF%@KS_ta;OFj&?%?|tUw{q*!^8Kp>8^;5A z*=MJ5t5pFk5x3NY5p~9h0HV8M^OuL$RaWW`3%{~CcJ&i7l9Qlvk%8d6EU>8=C>6ZX zNFG<HA{Zq2FP~O^2GmiGf1%~1NPE6?pU;EVRfrRi86Sl?cP{PbQ77HVKOHX*-#C)$ zLtt;P>4DH%VI!V#`2u$9fjY?l+3G#M9znm-pcH86Y0MgH6b+L>)R#^(Hd33t4+kV* zz^kUfVOau15GXX@v$FHfH~{pFm?rjCDD+Mz&Gi&c8x7n6;q<-5w+uJN_@ZtT>h=>t zx(({9{EYrXO;r=<Fv0uc=x^xCsmUa@fXz>0{;MR<cm{Y^y~0siGTDUXw}(=>5y@Nm zgXz%mPcY}2X_cBs%j6g;`M#B*($M2uwJ1|Ty+gp29#$j2;}G(TbA^vs-ePS8DOk5{ zvfF}ncZTCNB<qyi*N5uGOrTMAi%pAbF?nxL;2!L1vT5ZEVBwe`aBBC|54T0&VlUX6 zXsp%D)fV{(su}xvywqJadk|hgN5>yR62-TVnOvBm=KM1bL@MEol{6KwuL+mmg38Z~ zDwlH50?1mZofUH;WuKLU7mZ4H>X;qa?{xlSe*Rgl+i?f-*mgd!l{<C+<lZY5$2sZC zSUf%0cBBN+pOYf&<m}|(+E&!Om$!iN#J&@@m>cJw@<{BscIEA<Gp8<|-5kY9Zu20; zzJAvD)pJhY$S1iKnISHJUkP7Usg95FG_7e8%Q^pM@BC7!8}9gM*dnl*ibIs^b0<Bu zT;^CiZPGNc4#$?If=<_5l)AM#S=SV26t~He>qx&HV|>mqVNFZGYU^e$gtj?$KdrA{ zWktYvG><X$1x`2IsAUlgP1B1d2~CVujI$eO820`IAaxd5w@eww%0(a;K8|3JM&*Lo zVCs_TTz6qqap&=JkrE3nr;IFcW7EC7t8$CxbcRj0W48PX6tH=_gW5-9qidE(<{~iU z{?BE9A3E{xJ@|@IFl5zEcBR4l6FnYH2Rz<K*Xli$WP#p}r-0YyP}>o${44Vk(s(>- z6VrWU<<#dJ4f2>7LwOVQus`NY%WS!b!|V7?jeB(NB-rSxrn$EV&+bz1<CKY(-rOxS zfaPIj>c>f{(>2oIy;V}W2Kq4FAH6$FyJ>JS8I@(4T}!`3fYaaK{~<L7oE7I%en>w_ zwJi20RP$E^Me78ec7+fD6=C$mq!-2tQxP;8T50?4W;#*KU@PHfV*2&YY`V7&tJ)_K zg)G}<lthzW91R3NTay&i%5gVnQRrbl>G7}B{{BYYcHhxA=vqK$w~zM$;BaImnOt@u zw4_~{9QaTdU(uIvtn<nvP8F69Tq<&aY$AMG;I9WOEey(oVflpggmx7#^hM-Wcl2ZB z*)2SphHw_af5)haW_};y_U3%g!D|Ba3t=(S^MNh%9kiocB|(v9=p`jOr{ihZM^3({ zsJB<Y)Ph<~?|9#IlZHkjP{g`;*H;Q3q&}jB0-SsSn)A<ymxq~6UB$iuDUs)bJ{F8C z{Y%2@I#c!R>dfTVQ_A4Qq5miq;w!UBkoTP}UxwV5ZlWj01Jgh0e-aE`?JCx&U$Yrt z4xCNd6KK5o^Fy3<)lj9V4lJ=hJ-a&j2j{JiK>3_lKU=z1@jR8Z*bddrnp<02E9^%@ z1!_L>tL*1*V+#9qQ$V|izg!<Jtsnbk=ofeSP1Hdar=KD_RYE`DzpWOMrV7jZrlv^` zy&^6RONzd+!Kd|ye*8PwuzkS83CPdUE;bu4d1*XY*if19Dyp0exQj!l6$YWyg3ZRG zE;L4U>Qw3I+b^{sZR~Z80w#4VaOzRAK#^C0vYWZN`S>^S8fLZc<8OcUU;Ou3(9)#{ zArQ6bC(mPW)PNU%sb%H;M>Ng^E#qa@M=)y(cJvIXq7(J}uAvav>Ida?LP>qc8@or+ zZ9;%z?iU>Q)vap}{`ESe7F6%Jgg|jU$RnCxL8=G%qf0Srq-z})p38kZ`#jo9YwVTS zJBu~mKgAjW-i>l~La~vTDpfu4Wwl^wS6e0?EOP2=n9FZ1`A1!B&ogLHI-6=^YZiwl z04w0+N&WgvwhpR=%fJ_A@|Q^682R9Bis&-oz=?M^#;fMSOtZ~qD>ODg2VwgA1VFr^ zPDO2IUJb;7WrlK*fW51A9{5%`g9pBm?MtlmxR3EwI;G?$F=<EHwclAg*2rRj9IkCm zNM8FfWGCIOHG)APZiVYfSIOmEEOSRGd1K*o6W|=pgOS1->iTgqd)_yGm}~-q-enC6 z$ocjS@%i4wt_i?vheMpvrB4qJ+!`AKT|OLQ$${=2_6g2mMK%hvyv<CV+%y>cI(Nhi z99gN{mF%0+hz18X`IX<L8Ph6>ryKzSj&hUR`o|9czK}V?;95OzUpkUkQTyDdZ`U#? z78kt8exE;;p+5Wi20%ceg5Di3UU(!(hQ25lnIrz%p`!3X`W%hdOKD}i{8KSVC+}Ej zOg78>y5q!Fd8SlAt@0nfPP^OR_B*3Be70@k<Nr8&?s%yG|G6mH$|{70N<zsN*Pf(8 z_6TVh+2o8Xqfdl(S!oiHy;oFZ7a3UzsmR{*_k81)yYzkh{<{ZvukqabwIRTB#X2W} zM#mP{ClL`5J;^UsLe~PII7R@z;?4u@;^#^qW^`WOXMx&hzbAI;s=cG5qs5)Ymp9oT zt|q)GG$8t>O*-${LneQ1rT`$JCE!fKuPnANqP}n=RJ?=@V@#}8qR?x-OsVlTvq&(_ zs42tVDA*!pWm{nYOMHp-R5~PKydmhXIPUVy8K_LU;_FcoRt=ov)Ey?g26f3s?`Fl= z>GKPJZH*2kwLXbavvZSxN60H1BDuNn-|%c2FgV^^sUz*hOVSVDiP2r7;4kmz&h-za zqz!8~3Ojd^?Jl<;&DGAlHOZLdKWKGqx^3r3mkqB~B%?s*;P~^w_(ilx!Z6|YfYf5D zQ^~q3!<W(g{&kf%Q~te}{v!GjIEDfqWa%FSU>Y%2k86&}98-7XT!arID#`$iRp}{Z z5+&W{|NDGzc#UXLe@X2QPjga1jD1b_uF|?2qif-`^^7Eij$Ojn&%XqFLLxbaEQa(` zm0B-QKB>=NnQ#^=EHl@Zdq<CSwmmtBS##!zDqVlXlJL<3`@P}Csu8j;6{VrON7yqJ z-?QavhE{aCoM;?%1S4r`&t;Oq3U=4gF|BQN|Ip`_fQ3IAY?F)BCJmj5ngG05Q|VOj zv!|zLy0^IHy_iAQ)ZI@j-o_K-v632<47q#X^zRHAjDJN0NSr@gfSml*<H@?G(FbYm z_LQGD;`1euOIK(o+(!8P4vf_;n|qIjgj4WkN|imx`Sk@S7X|6y{D*-`89Ay>5}y_w z{un(~sw2X=Z%VeBRdM7S0hvTnaDhbjps8e;)DsRZz7^UVc&l7>$7#%6Zn`>F`s)TW zq*cwUu*F=KOj2jkb(zz3EjJ!ZU)-lnycha3;5j(ooNe(oR9GfN6>)yu6AlwU9eKR= z3epLt?o;n3gv-!!v`1f@`aJFw$+oYsKhdYpTJDm0EoDd{^=K+EZL{ZHl)MD73_;)% z{vNhUO{3z=!I;VjET=llwUa~=4%1h!RUCQrOka!4G$7h72^rz=g(FUpZ24t?;-0*x zsh@7Gpr^BQbWHukY(X^38oP_GdX-{%=ire}W0HM|7Dq6J{tB?F(%3OcX2O{4H;fc5 zdVhAuJ29KWC-N2t>6;H0vZ5`YorMt$ILKn}(hp4^m#<^<3^=W=+pxI#n}|>j!#=y| z7VQq?xt)zWiovE)6*0JSiaTsp?U$n4wHI#9cH;*Dsg;jV{fXw73mQOVeX=PMk69xA zzL&`nAhTm%?8u}Tx){hzMeFU(1U>3@KlaZ%2<O|S6+Uo4d)an=gD~|i>{<JxKPJcX z;;%{e0o{fuQ>%*yae%;o1G|=1I5$hLUhpVa`&vOgjR)us+Xoe$cRW0Zr4IHj&GIX^ zPIe^k!0;;7t|bQx;V%gSppd+T<(A>MSok3D9_5-TaK=q5Q&YryY*Vic=9c{X1C|9b zR!%A3ZIhqu)8etLx=*+w>-l9^?fgQ0^XlbZ3AAJf$1jidhSme=yHzHHGvF2(rdR8_ z+9(~)n_2xq%zlw7315SPUR%|a<@lAzKA`1c=~QF&?nRZ+5KlFlGb$LjfKB1}(7%mT z5pJWIaMR#3cMT2DIl1N%$^S?s9NjLcxJ_SETtlE7oLu8Rh2}D0p(8m`k}rD7PkUZ; zZFx{4_@E@YB<0VN?6MERd+`PGIg8j<`kz-7bY(}0ifMqoo<RsAP_i;>hn?;gcFzDF zFUv)R*SPZ`C&s3LG_fp6fM<Ocp(lJ0D7PmoNPY+Lf4&}>H?@4$0vY|o_DF!i1R=xp z@{8&Z1<(QqP{Ve6{0+wZ4W^`Uh@MA%-eKB{EBXHi!(%ltxBQdha<bau!~dao>jm0} z9zAl>f@CO<g{T0g@ds_f?nM_R*6#PA>><^&=_xrKUR(L|YIi*elCkGj4ox%4)dh9f z$i91ENYdp)@<jlk90-W?J5hfY*RNAy+39{>4Y|6$o}{qplqjo`3#adUR(EEpvp>^R zx6tuNWYwgQ^#~HS5m!JWH|go5`WFGNAv?(Jbq91$6Z?bV1+>Bj7)jPcjQoY=g-h@c z=o^^X-cwyHPgeaCeiIh9Z2MkE5^p~xqAGmvc6s5xn;6B%c8ZJeOUgTTNGxab8M0^o zs8JRo;S4rBT!l5n<Po@+y0wj+v+O>J?@5F|0JJt_(i(SOVIaQj{5Kdkg)<~{lCs*` zO+XV{jtnkgLD=1ZGe)R=l_lGBD|;oatk;1=Q(}TjdW3GNxPQ$eMpx8FSG4w=d@3LD zzZY*514EiDL$mLZElv{!bkWU5V$lJ^zpJ>5=*P#`ZgR_sBCbWj0S7YvB0vOv4upq< zj`j+2%W5B3R_4{8wh{xl@;Deh+Du6f|D+0P6c41)T!e{#Jug?jYKLocwbzwJflFrA z*U_IM`l5_jK!tBb^-G+cdM#63^iNZP3X6sF@20?67GmORT<vM&v9Ynu67PD{cU(7T z<vtp2CGb|FO^X1uKt7<D?&{BTuayzNLPCYwFc!N}+dZ0`VyXpz2$>FxzB(3ebzZNt zJL9J=q1-Io0+CcM)F&J6S1gL+k}UuAIB<R${aJZ(nX-%lQ2g=y-8}RYPq5DqDq9gt zYHfkaQHeHmNg)>t2fY`o`;sb~P7S8I##*-1lMx$MitT4Za!SG_^>|Ij9hz%>{8KDW z#Yr`JQ`rP_en=r8ka3^k_&|rC+d4m`6+DU?n3vUTCqFX&FMQD6+1MXIMYVWC^c9*Y zFTcs<ca6=WK>~yo)L(#rfw-W<p)PshWq*_#{JaQzvO-`PJ)80-M8r+h?=cJ0;>AcH z`5Sm8wxZ+<{tWmJ+c^=7uH!*PW@e^NmQG13ls4MPYrGF5>{PvZK`b$~woM$@BlhWt z^&9eoaHx1s=p-2ExI+l2qrPMa6qcv#FkI^+TJ%cTqDpF;N8_%G3<Uayf%2_wCygYx zE?$&~_^@s$RM9%JHhC&+HRza-jN8dg-jj;^GM*;uIhzyO8S=tWt_E_Fbq7G(wA;wu zz+axy2R}4q4{4k%$xLk4^EW+2i(;%RK1cPhxv{}vpMPkA*Xjf{Ju^Jh6h)zv`>dxN zv)x<Fk|)Z^{9=O6sYJSW&iU#r&K3X&1rZHjGaV(^E`(ngeuzFs^WFE_*)i~SpmsU& zF|mP41~IfT=y;yC^JGRf6;_C$x{9`*>r=2UZwUeTu<stkk^OfB935^AIJlX(Yw<K0 zIO3hOi@7tnS^z}D?M&++&=J^;`?059v41a#@Lmf8q(!d3ApR$@Z3P#SI||JItoTpd zVkBYB5nRVyks}H1NFg4S=$u)&hfHD<s}4hU0L>xlFBHbgm94I%O?R%*AW2I}(EKP5 z<+(3rp!oCDoL#t+U(s!PxVGSH@})aU(2mqw08#yXBv@G!qr;T~`N6m{sfM8$Ord=9 zC%?~{q-vAo#^KixQ!KL1|0ofS&mptuDbSkM>nP%m$I4d-hKxJgMEoBZxXCI^^%WQ) znUMMfQzQiV!x-vrq)H#|*iafGD{=g>;%+}4>DV7GdSrGUfeg`x0@Tzd6^rV45o-Gl z;fFoD9o`mq4rZI(&`bQ@EA2OSV7P8OX@DBuBaEEtmZ~iOZk9dvNbFeKdoBY;$7W}< zu~rs$?yxJrUwQLc{H}gwRP$P^wQ?T|6Lh8N75tDSt~_0YV00%~l8!1dp=9$u0=2Nk zVo~67jLKhX@vvd!H%r#hILFX!NVI~9cKGUo!Z$e+7t>}Yx(XmwC4Jt)vB|zRuFYTG zyT7f_XXR?4jqc6RMd)3t@3I|<RDOJ1Cpqpu)9IdXTd`aEr^1djSLoRCib-vjtE#wF zK+GkJ@U#nhkJIDB&XH5x`CEX%?N?Jph<Wp%07%lk@tV9g-XivjGlP$3c{#Uh-wjER zh%kF*TM@Qf9!YR9P4;A3t&)kgN0O%uEW3wP$Dk;cy(f2e=B6c&yo0$6lzkUK*D_m5 z-C1Al2zx%_JxB=51HA%l4adv<WZlsMftw&km#C&pkX|4IR?~oDhHOt@NypB&{ZO*! zwfcm@4hZU|v^c*!{%BRXOq*ZaXtRqCQXu`#Y#P$c>_`se@)z|pUL4yrhJvlWzx%J1 zr)BKmLpl53*K2oBFLAaMuuu~93?j&&XLlEX$ER-TLnMR|2U;nY$<A-idV92z-+eT* zg~{x(t3&QHm7rZ)L)>S^TRoOB(;0v8yWk{Q`^JEuQ%s*X2O1f@zeJxI4Oa5bRT_SE zLK36AdI_J@<Pd}-<df}IE!L;#LI!ZBbrJvbwmoluNq796`=m0sj3aUrCgfhlr5!XF zC~)!VdXt;O<=kBt-wLJhbVw0Yoq1x}LB3gN7PZw(I)5?^>Q5$O?4k6;C$U^^n|a2s zm(OxmFc3?tMfCRk1f0%QGLd{%C039I$=F$SL4R1byxWuA%WsjI1l+FJ;s*zrlFlkV zJ|V|%W;8T3<cV~HiRfh{jmkjn>7_JCm*zt`cID6pMM(G>%}k9?X9yxC1}F5ZIre>8 z($&cUr4Y7aG0@<lE<{)_iqNlNyDnIJHeROvLIS&@zn%}|_KJ|S)lisoTwE`-^>XXG zPY<7bH3YqSTq(_0x|-8r90?B;i&YFa)Fxf1%12r~ybu~x8!<dZwke+Euyzo^h)A?_ zAh%h>J%Pm5u@L_@DFD5_wGHwhHtD*P=2vLsouOW|V2xP_TMv{nH)|nvA(x?V${Tvc z?4CX9EPYfZ#;S1jg>;6bw{I_$yt%pOS~g`|hAMX-q#!rjeHL0i<#Zv1WdvgRrf&11 z%p?hNw?d7YigmyF;xPu`g&G2<l3McTS+E!3fjPvF0ZuqHjHE+#t>1HG)297=>YI;r z^Pb=H^zH92(T83bLJb5Hq=`lk+7<>5L8tfHobfWNks0Jt3{V~ZJ+iytIgtciOn$$> zc2qGa>t581i&%Thu~)#V6U&64!`gH9q%V0^3s;WX)}u$?eY5DsguJkZ0}M0tmj_FW zt(*YQoYg5J)M2gU)wif3>$lP^7ztkYss2{%igiErW_4O7G#<*@{)qfjxM*|&3B-QE zj7XIeG2LW(0v*nbEr#sIu|NGKe|Qdp@gBdr*85XalU`xBD3M5k7|vUo@15~ukG9S5 zn!kI1e*a*P=>>(sQVB#|Wgoinc#jnfI*5aEaF#2jFhM}FJH0dP*#~hoDp}D9XuR0L z3+Ys&j^caSF4wW`NS%1~h{;)Enj!@z7MUs<@l3_x2+2hd;EOY2bDC0?|9-6=b8JBT zz8y&hQcgR3z*KdakqJbOD%NE3LFI43$6K3~gV+%f*9=uujN=OLzV-U8k(khvaOke< zd&}Y$Cs^^RWxA=wi$N^wa#4C1$j8gyuq(X7!ou1!rbk<pFp~`}mPHdJRwmA1UpU`k zu;*>ATr%q<rw?_SC?v0#4P3O9_{NL|xG<L%6Ti|x(S}yYRd(;5fF1dTPK4=l3__6v z6Cw?ByUtaF@gZgJ@Sf<vRnwCDUP$GjV==Jcci*ffvMJbdB(T!>RbCd)>I)yjHz1+6 z>pSH)68&d8CnA`?sCG>ZckN?jdIR-IukxqH2MfSh(LH?tDS=Rd=4Q9rbx_OJ>T57{ zR(Vu<;AvXVG8Tov<Fcm3cl)3+K?$Jy^4773<Y6K_G9fyLD%vW1KZabvREvQ_JbkW* z$oBF;??!#eFO0uq`9!tI{7@4BV%xj&a-I)u0ZQwHjz3gg-C8Ey!V4{^Pu*%oHRA+A zVAz3VQ%jNWS&s?ObU#A}pjD3r${9DWP4`SERs1A~JOxb$Ay3cN3ljll9;11EbIHbB zZvIRg(pZ72UU+Y33^ulW!t6CndfC<uy~ReuzrTF!H)fUfj=97KHmLI$R3A6@4xEkW zX5KwisU30tr#)2nX{15H!5vwIKG$9ODX167BWz9{gT<W%yk1CZoZco+!kKgb<$j6I z)(bSIl65;mT6+b-jqjbWx$oWte|y;5y6XHiU~N6GUFDvSHPEodvkvOD<Dgu+7IQZT z+rlKhT{~$DSj5>sWY;>k0za@SLKXO^iZfCP0mCL-Z|deRDVq^VbL5FRNl<hSfQBZJ zI0`7OXd-5iIsw)3+Ywk3IYD4o$kw0nL3)M+W%}Q(E`ZAOGqu-ty3b7YK_|@fmfg^= z$dV7GYWlSbH?w4FdVn8}zEpr7_WZ38A)kwWl`Y{#2Cf1x@-BO^6nPBnpinr&K+#kq zvF5{W!NnKnmE0i@zRP*kM(=NMeAM!>?Rg|&<_#@av5&tX(=5a-8`JKOPfzB~x{OVA zDm{L!1bh(Z)c^hdpZEF1w<V@%pt`?@<${pFBx(5>`Xxj>0*sL`t=;jlF~oUr>CJYu zy6HX~ZQupepA{RF3kYJ`y?gg`k5;;kX&sc8cLhb<;+lnOajP+C)T)&4SYVjSqD`es zK??=YVrT!zNe->G8{lZhyk$g<QRvQ<;8FVXdB|UA1UsY^YWI}=LENwKxfGlJGPVuT z?eupxpXh->(-6-W808bo9y*0okOsgU3NcT_MT26#ZLLTT?QPMIh~qO>>7&RFTH--Q zO{_ydl(liLbLoz6Pw-;k@&<jUmid0E!l|7?z{T-b<;eTn^D2={>0V+9p!yhssVA2; zJ%Vgjd10s&(&W@vX5}U!JA$;8TrLvysXovLeS=a6V^8-ylC!^<29C#_+{yj`_YlvZ z$1vq2YEUIaW(O~v(Wp7g*(f?DCF+W6GQ;=u@8jQPV02Qx7ni=~A5QYA-UPK*^bj=2 z2m0SK<Rr5XiQ%mziE&t<`Ug*gzV7cde0I8^@Kam$=@U%8Tw*Hc-M}&O0^d4Ds%~~R zYIfqyEImMawyrk}-a_zU5KR*Zfex8g=$wwdl;dRGeS<es{RD7)kIQ6pA5;qY!0?T# zj|NN;nS<9k9mx{FcoFV_)<F^(2iBM`uEYm>=MjKe`gDUf(OsUO3?JxFUKaY~4szVM z0%is%_|n`v2)>=d%Gnx;`=WY5cH4X8Un&Q4x(x=mc)~oG8i_9lYd`e7hp`jYah;k# z)em<VAGC27eZbIR<6LK({?}({AORE#D(2VgU_J7fD@3li+S$IP-y_I=Y$?M#=8X8k ze$@@jPZ^6R-zm(Q?BUUf(Z0v}F0bchu%$LvA-K6~IYl&8Yj+RpW?gR07dI;pxnlV% zdjtk&F}d`ln+?p~5mH{OA^pQ>d$$m=swZF>kpb8gkdl_cr|L-4cF32YatMM!FH@3< zDD<!|5-Vfm<D&QSXW=ir;$UyT$@FGufboj<D}}H%A2JU>(8>h{JMsG?4g%85U^F$> zZv-CIk)PwxuC4AgQg2ppQ@UBkB()tGm|`^A<oL$%QY#c0F~hju)4i@uO-=R{t;?4< zwH-Lf5t(id<Ow42<JgOOfo(}VImg${>NvhJ+M7w#zk#hrB2NB~Z~C$7yZlM+gT6Ph zC)WjAHyyiQU1sx^mdo>iFDH|e0pnT&aAu4qhJNiUpM<_1!KvKY(b;^Y4(glz>6gb> z0(r-}SoM2zOBrNevA5}!s8Cp)^7Uf0QCZtk(Eht?Q%QxK+xPw1>jrrN8{GQ8a{b}) zup|y|e?0}d{TWv3;hb>zFuQVyNq|MV-DRr3&<ENceRysm&cP9JXCFR?$VZb<Ps@S~ z5L%u!lhR<xfJo>vTZ^>IAN<xJ#_z}385r%s*o*0u?+PnmB4Q3w0-;9z0<%rU>Q6Um z4+=sQhMtHXWv%=}^y}yP0&b!=#0+kLlFu((R$>PflKW7``g@?;we)fbmMBaQRq;$u zG`sZv`4g*Ku}n(yP={9HIWPC=TK5>UsP?ZgcqSV{W1Gi&AxDtW>qwTBCt~}*mP)a5 zKLJ+}>`Hl@#RB~GiX#%UwF<pr>s-X~#faat^!}d-SB8I^{MY%|T@f)AV>`;hjqZbq z0>hb|#{83-52da2^#;w;oXUbEjoxIBtnF)tffcnziDgpJ&dv5+HHduMp6-@L-4(=T zD@e2(5m4sXNe4;Hn`)VpS2BicAtLJ7S2?1eB;7&3QadWg&mwg;Ed|MkvYbrX$1J;1 zVZ8m$M~>O)mRY{$;aJDytalI)vq*y>X;NPcp(J?q2n2wFMv;y^@9fcL`E<9hNf%mS zNJ`|mJ{P%#i$}Hzr1fQa?CpptZl3Y6Y;^C$nhM2Q3fXc^_~kLfe6H^NvpJcWE;P}^ zVhgMYB$lE(o*tPjZ&~r>ghKzj%iut63U&bnTu?6G+c<C&Iyj%T_7$+SoablY5@odg zQToc+{e;rA-NW534mPe+&Jgtbf_cM0uI=$sC;{ruLvG5+k;gyt=x8%CGOn0iH%JQD z)1G78)0i&kaLj*$19eH~;0E1K#B|DfK4&iM|9Z93Aomwq#9qHVSSG!h%ME(U=_Sm{ zU%ORpG5C~fYu)lfG^;QFG6WaDWKG=ObS(el=Eotrvq<?Z=GQTUsQ*7)kx~!dg2El1 zSw>LUu=jE*k$R}K!{&%-tUYfH>6@bip}m;R?+mSY2Y-|?z|^-3uB-j>zd%a_f86T9 z3lX85<>3DYol$_n5r~Wm({=BE&*d{d*jfAk1toeJxMY@8Pl?l|=jE|ud*SV#7h|?@ zf*4)*dW~4!HBhj6t5F8bP9zrOsVr6heCEHkn8FsIP#@5mBf-N!k4*8#w%gmlGGs(l z)6ZYOj)=~}$f;Eh@W`#Ur9DYb0FURn4^XxS30<}06rTOzeuQxNpe=Ep{?BICeT9hT zd^Vqsb_+i7)_<hQu_f-}mnf462*I9!<LuVX=vv@hi)XtL2hHg1Ra(hEf6SC7wPc1W z_uaV!(eTc`Z`W9A>>m4)o`|5Qf!7m|Qr5*;jk)gwEkYH7V0Z%&C7<VkOv+t5#dYRT ztocNhRdL;(_oA8I36~)=u~{O-`NyMHcyc`YLJHD0jD+}FlKEk<nD^^5+gBt0FE1GW zJbG7fxZQam?RQjkJ~Ap(cXGVKT@mS&lWgi-zl7p6hnb9K(P1j&)rsPJu~#w$y4a;i z=}=sFu2iyfjj;*JRsQ%12-+(2<t;M;UU?&Tfsv*KI*NJpM!_EShR|NZ@9ArLh~Mp= z$2x@g!AN0q)L;&b;1hJP*GE+4ALxQ)1H(#VSdqACXHg=F&IAdXx=7lSUk1xY56LiD z2gbn{(`;8Wi9&BsR;9+UP)xJK2FqO+y3_P1LTy5skT597avqFQ!F0u>debs+n%gy5 zq&9JmfF6n(WwCn;QHFvLS=;<OClu2^z2S$rmbg*AXVjo5IH9_i-_m}xZ~Jg_&P+t& z!BNefs3oA>jX?R$OBM329`U^p7Cc1v(}C#M+Ct)vJ&mlH1I$oi&SMaSwDDkI+}B~D zb&xl>Zqzbn8`+O@m3j1R-mx9>e1qxOR`9jmF=sIJa;q1YrCl$mt<`70rGLFH_?yM8 z@0z{OtLU;WvN^p(`so)`LaCTFbua4NMD+Sygmz60tBB>1DwbQ$U+nZ+VT`WmgMFe8 zF8$09VnE8pTQ~5JAJzoa=366iRtQhW2$z8_%`d*UO*%ugrXMm$F<0DM7+|o8S07}U zobp^QhW5gkI&ICg%SJm3pC7rC4?!DNsTRy&#z^Z@p}1}^^UWgeFP?<~99M}3!17$b zAWQ_<gX#j_{*Zcmdl0c$q0{Q*>wJ&}d@4%QL(Lp#x!NJC#F+ja7q6jy^WUIys9f?( zzQT#Z@){mgrK{Cf*U_c|M6F6Sm?sK$!aS}Aw|feCK1jo?2P808la#ZheAo)|p)5m= z{MmI%0+w4HFCY4#nA_YhHV+}o?)t=n8ywqidm-+KB{Y=Bz30?n%;<yODmKOOk0(Z& zPdm55kRdJ=(757mQ)ASfK=dn-*>d|Bcd|$!W?>vy2~5soX#viIKZYGD!hkWZQ~v~U zZDRMSRE1zskw`ZLtTia;P|{l$OqT9pJr7-hqGy-6UdT--(VRNvo>fj9R6tA21qseH zLM``s|2)Kzo9m6u_I)2)4Q}>$pzkagps&AP7z@9~Ob=4<SRLG<0BJM{13AXp>jJ$w zdl>~;=Dgs`)h7?B@1D9!%Wykx$h#k+KW_}A+oM*)Q%po8O=Ld-84KKjvp*7vCLX@E zAZB(Q6>u|BSbnqZx<O-r!M9%di5Y#f_=B&4iCV<47ophli>LPBq&o}(TWJ9HvJ2O2 zK35K_BDC!=r$55QKA-T!N@CpFZc2M26H~G;8Q6U~&<bkC;-~&`;;zy~2Mtt#$c9}_ zSuBx;Sb$*AM%DU`G$`(9DF(yv73+N>e>?n4c&~5Z^0TibF*lO>QStlwVY&dHW$yGy z^GB}j+UzgEfo?zBpJ@8J0m&p}y@JSx5EAlM?|j9mSo=9X{r7WGVoZ?$P`k>^1Xl|{ z$6DkEae#A2e+iHR7J>OQs4g2g|1pTDN*sHI=Ir5Qq*B&4?&O?|h&I!%mC`gsgak5M zr~vYkpX3K2y~_nb<lXH@`*Wb1O;Xowyko8%M$Z){JJLZk%W|v%l0{LFv+T`oeSSyf zceW*1u^+$cn8zRZ;-a@$SKy6Upw~5k^U#xK5#+6?+*rh3$g?GWtCI;5;ku_}FD%Fh z@ACne+me@RJg+g!MH(DxlPH(j5CZ9NqdUq{YizJEv?-t!TFMx~V>Qr>Kl?tg7D-}u zLyoug`UZbG2rR{B^doWCcg!EbyzS0*fW#`&_+-?WX06@}kKz(ZCsMQG?+J9YA<bMJ zj;P6tQ(LtJ-TIXr_(<BgD(SFeS{vMcaGQ@Yex~dAP|-g%SmA>V!GT7t1jhmHqd5@L zD$vleP_MeWLwpLFLq%U}CksPKyASl9iM7HA!4m^!g?x_3XR76pEZ$|9=(owJjmfw( z)R7scRHYWcLp*9-yJE?u0G^V5vI04iHQ3A%m)~)uL{7#9DL`Mp!-IDp6-id|;Cj{V zn2<TI_Mi(r#CY8gXRH&`NqOyft=D~avH;0`DfVGmc3D)xI8zfD80|1&ujU_m2MPv$ zBLQP+3w}=G@(&V<rei%EfaF6innsXQ@peuq9(hk2r+`NcmU%U+V)?<hpo_{bL(tC) zX4XDw!2-R8+HLgLyME}>yPlHxHk+2%LhBf4^A|)SSco^s6Z4{Maoai(VFubDQD3gg z5enSC8RD3OuVV<-LU<MJL=M#j0pIzMZ-m5)-&4pKF{Fk-`YV!);X`sCeYM~;Gs#x~ zpBL2T&J4$$|Mc|c4_#2I4<*c03Tk#PRj?t2<ngu6M34iH8!0k4w6Z3N2wode{qGM! zCFZznCA8-4eIkGf`P|u&nmcWAss%*0>$BCzL**n(gCw_gr?$8m4YYqmvZs=d2au>v zg~H%*%@lL3wAq=-f;j^#`w*}7ei?`a6ukRy^`)>{FQTo8fHueIp-8=Ada|Pp8Sh|? z=&C={rfVN|<}S$~?#y6jjhxCl>D<n(=pPzzbBF<HZU=+O;0UT<*-h)V4_-*r90}%_ zXZ}gacK-m$2k4>BhvJ}u60zKnDZeK_pc`Fic6a62SfSE7m}?di2VI0O`O8<5axDp^ z1wp8nv^oyQ93}_Cwe!;M2ij<egb-C0VL#B_J^JFgM33PHXwHO_ny;LKd0b48B0rV} zVsP!1IBqq6>L1|9`CO=aDQRbUXNj^FJ8vL!v+DY-<P^3Ry3WP@`HIeNPRMzteK;N4 zu!Gnl*~-PLsNzh`6E{644q!kh3yb2I;i`j(!QPaJ$T(i7Z&)KFbOa#~`e3UZ=O)`U z@`^!%GR|904JAs8#;POW0GtgmaTh`mQE-$mLx5%37ow_5IaTyn+A?4Lf?EcNt%$Ua zf;1kQ`EPa>M(%qeL$Bw8i4d0mGc5u3d7rfVptNLZH(B8aF=CRz8n6A)PHY+Cus97~ z?g*NQ;>1sZ+by;BeC*iMXJh#rh>(Ss5JVr!ApLKdk~WN6jzGmbH>g^1nY3a{?>6rt z{MYZWKu%@koQ4dC<|nVcd&Zph7hC?l&jwCrxDhnkhEKL)MElO~gFeJi-9RhrO?Q@# zxKKabmR4BlklN9)#>(^x?>j^`Jel4+kBo<Li1N-Rg22L;o~uAyy82$JH-Vaxj?X8F zSMZEu6n|kBwID7&UicR{cTB9uFku1QE~mDGz>6v|8zjIBcn3q7cW^%fC{|LzahD4O zQ3&!Ug2&R+Jt0r%&IR)g^0afsqv;o`#t8pde9$?fU4nPj0T8b>YKT&`e!Ahk48Fo< zzVnh$!RDL<(J9HGEKrF&@3H(^kUue1&fPs6iLv*TX2?t8PX*Nt!Uk$;&U*htO*+lj zlTgaW7k%~w0g^asdj)lovi6%2jR<1%nGjylk|<!@PY8>ARfl!;wiz1`{)GDyg>{*b z9?>V7BFu7c`m75+Dlb*L=#jSte7FEq6a_IQcN#!l{yOtCfoAIj1TfuNI5|oY$^zSf z(6nAXCmTdTLzm<QCZEO0<-vpnkht1o+U43=^8unESFJ!YtPsQ){Hp*GB<uWxTyXa% zf(YY!=&(3MkP(o5WaLovpQnmOXQ;y?T!e4}9DN*426;`&1-L!_Y*CGYjy+q%V5+g- zd#O<PE1_@~`x|vnOdu+2gLgn%?kH)!7}+k*l_88`!I5Y{pn`=9U{kmvUohA}-VV%4 zSzpz?4G*f96L{5p<Nr!4pP@%K<U>?Fd3p}r0Lj9}kxdDE2)h?>UKd+^LyTjsIfS%> zYi<&OlN`HA{{e`C+4Cth)g*xbVUzOv*Dk^WQgjggnXiNNSVB+2HUfxjdL{a}j2u5n zi9j-cbopEermku(y2ZWiOf%oS#7=Y5a3nt}(O)XPVRZ{lb{%rls>mCujc9uXu3e(F zQv?BBfjf}euz&PDLrzWu!l8+$j&u@!i6f`;$FSNk#2{P9;W$LQEu1-n3ENlx2PDH8 zwd?w<TfnS8N#8H-=Y$2J#+15UM7>H2M~wssLkbG%>NSYprY-uQE488+eGT|a>mXsP zpJ?a@U>3<BdxMCu2rL^2BEW@}e0@5<1oS*XYHfl(Zn|_Eh>I5=5GM$4tQ_t4+}5_- zjukv|{Ix58lN*J2K?)yEA>r95n|hLZ-nnhj+8}sUE_HO{A88C<O$${GZO6R6mv}8v z#e^(x0@6`7`kp=MAV~y**pyY^P{b4)Nf7S4oa3RqqormJX51q;U|u$Kj}tlcB+_Q? zX99yjK##=Ymyzddpq~;5$X@-@gW~!XxT~Tg5Xvuff3~Pybn(hRX#le-kqHtM*7cez z+HsVm_DR_is*I;*6I}rZ1^-qEq0LpG{%ZqvWP?{pwv;v?7scB^wi^)$kDL_Y-S-aw ziZJq|6g5g2y3x}AtGiF2a@7N1pLmT~4ZS@$*Tn9>=p}{4JISell50EKlSjZ6jwE9c zzmTLU9LwrW&tEtnwjT+{!-vv`%6K?4pU#yPWCI*n&))+m#N195#i5Q0JG&ZB9=!kV zF{NYuiM$W45}T7qL3SMn{dxbB4!-C!{2DEcF3Dj%5APxUPx!M#)^<mQES>-ZWfYqr zM@iBu$|o|PYcbq6mUfMw`4IVSi2!c6ftuQcg*;_Pg@q?~`wG0vFU6N=Gb#-6-J||* z-s$yqxTFQVwgBK?9P?u+-g@_0U%7ONLTMxZt1vVcd9wItS`bl91rFXY@T1tn8k!Lf zjt((xopVEoX}Vezv|jfCrDUCqMXybeA{tu`RzXjQwmf;?6PgFQlpDPd#Bai#9;QMe zW`Wq)R(5*9nO(Rm5gnL#@$g*e#@bA7;(DMNXfOVnt7T_YdH00DhQZhL0)7j%l|!!e zCptb%XUeI+NFo9pg<Z4-z)wZl9V2H_)}PxD!y_vNr2Oc8oAlSmf)5dOWC@`9H!1aR zQta<U&pWLDFg{qhb<tGyRip*?a|gnWZn%#pnj$}RM{dkE79mW?qf_$a)hNV9+A|Rt zMpc~m68~cYp{e=;gC%cCC(KUF`-TDo=NCqyZodCqkUTO-_&FlN*Mf?@gtj3qjGn^~ z_7jK&=OnK(eA!8$DfUTz;X>zm;uMw4;;HJSGK~qE7A<97tt6OPc+4}N2yYfG+Ohvu zMW;{d!%PB^z_1_fF&T%Y8;fC?a7EzVi7Z{>+Q9{?hxtPY<&;Q1OW8wYlAf0UU$_5N zqc%qC1m;2j1vHT;kYT8~skQpSjH5XmO95uUu{g%nt{uB3nJEY_d_2%RbnmzZ+2^{t zcX<Xhgda{J1;--VL`OkL>a?C>Fgad}g3zWu3swHP#=ozIBuB}!;0^#4x%Xz*#b%uu zOIxR0OM$A1@C#-6)Fvk=ct2RF)ju0rWpGwU<GBNg_(o>z%egqsV85YpIM>)NQ)A90 z)wc)w1Cq4XP%R|?+TmQF&tGqzti~gLm%d>nyjEv@;-ABH5NvD!3R;)`WDS8n2n*>V z6m-r0sO8v);{+EhoOZ5WdFi=E$kI1g1967DctmOG<AyR&())97VXF7Ny#08*S*))d z?NGR<1N~<I7=nVZ?^+~~r+Ffgf^B~OA%KPH*)Dv*5=~G0{h0F&D2Gy70d0p{MWVKo z>pT3_!J=FX)L(tK_#ioJ$ti<YREbU-#8ZYjga=bEzc@Z;B`|a2=m<F@h0FWh25qwC zZP0!4|D%z@bxY5uviP5U-Z)>ayA0Q_enB`g8^1Bpf$^LWcoa^qcE9W;g_9pZPEKr2 z?f;*=1S;$&giKtGw^7)T8|iV|@ZA>guAz_8fNAsZ5fliGlp)oE{)y`N`w~7d>cC-* zst9F}0-K<LH{*1=X=~eE_D>Y5oe95XExHSi-pk@&;^YX>;(#~?k+F?7V|e%A4_>Os zPmW){LJ@DcyK8R-vXgV!MuC1JwcyM_uXOR;5(ra}+}kuPGv}fM4{lf02~j2r+~R*P z06}|$hf#jn8vjO2rO|>S!bQ*<jVe4zr?4!huu6Rt42n``$s`vf0^*T?$8~sVj1`&l z0hXR^f!rkt>3&R}(I6MH0;&(t+kwqDr*r?lzQG}2^|wghy7zdDJli4;Q>W(w)&J7_ z=aHw0yDk~7!2g)=;UXszb_jWmIFjYuX#YJ#<Ll7dAn$QJ!iGP;Fy^&iGC4;Ok&HlG zJQzw5lnkYX8ARr|++<}d7%p;ib0n5AiIx89Tk)<RzxIsMNY6Tr`-535W9(Za_j+CM zK-Kel&@!zNXR7q#JiLeA`c{4{qY2x{lV#G)UOZgOd$M0#rOG_@>|2Ie+@9y_SAAn= zK1|K*q<fu~>D_e(RKzU-F1yyfqPjE{!nV83$}#SLFjM+yneU;GM%zYnT*`ONx)NUl zCkP(C05Kzy`u;hEe<BacciZ7;j!xu&z-&)Y=%{7tG5<p2RZ5ae(mx&RL*z}btdRn9 z3QO;Tw~o`<?gZoiWx3_D;^Q~>0(`YS{bBg|5((_NlMg}_NmyqNOvENT1okNQrFvlE zo{mUSM;YECf)wP|3m3-l_CD{ojG~|mh_tmw9tKQ;&P?fZ6pj&oD*OmXJ4SW9wG~U~ zk-?s%^$a-F@rA6(h{Tqm5duLL=2=f6u}$-V4KnX8U-~<2jyPmC-NCBuSy!k}c=_W5 zZ9;F@c(gFT{0;(9<Dp0NK@;&pY(|QY<4KM1PX#uaP?aNU4}Ce|;U>Y(VBsQWe1kw< z9jd>4P4CP8#K?QDTIF?vpOwgAOY#>x(A+&gevou@Ef7m0^3uJOj?wysvtpI$+jIMb zd{sv_$p(8SoF_vX(Zao!63Mt)02(@x!!TlDPq1keg<IdCQQPAnioc%arUeoT>g$hj zc%Me$7x?djLyJt-Xi;UM@GVQx$`twp@uITSQv82?xgzQL`I&v%ofJgfZgS$-hnQ*z zViX$#MYmS=oiX?@51tGR)lqs;0tTWFHeM|v4D`XKK+%WA`$oKlPjP*$DWsIdF6EX| zvI$;!05<WlnLKvX<g{XXZmt+_@q@<W@yk9?J}D0Fd1EM4Q5z2rh2&b^MtEcWM7}xe z0@Cu%(XO}S?F~85+SX+GfH$CUquWJ1m?(dU!+^2T(ffo0FG~m?oHm8iCluiCEAEX( z1c3EP-%4vWMUuHh`0o%t7$T1xcGVx-MnO|ujydcIB4w57o<NM##{BidUt|^V$1_Fs zyRGScbl}fr8QpFCm+*h5{2@*h&Ts!G;tS!}cyJyD8l=P1mG>$ruRMB6iSjzQgKORg z9!(f2F{i+z)%1~9b&s{<+YqqJfi#!`8;RPD2!w`19~Jf<P~00k<L%@fhlULcq>#HF zkkx)!MuGZcb7I4v+a(<d7Q3SH3fXB>pzE2^#CI;nSxWyzs6GH>xF_Zwxga&y6o7Zw z*B)soC@5Hcb87e1jXa9t_i?peW3#A!e<!(6eb-0IoF5+4$OtVBsX9cOIgETgN)9P0 zp>X<v4^oXbxPR@Y9WYS6^G9VYZF6iP?vIq0kAwF!+Py3u{vv1xJVMV4htu1f-bxtv z1~np=I@+E($nf57>!%|daEC+c!U9qo;l;csWhrW8G7kG^<*rv(dH39*3)vrh%rukk zM4|33+_$j(pg;<xr6}aC@koXb_aWJzmBc4Oo>+l_i!?!Md;Fn$p+|rC3JlX+JEOcz z{G_Q^)SnA@D739q^9rz<)|Y9H8YT=`!b>NT5B9)Y_IqO~7_gdYd7q;M%sRIXCG-`T zP0X*J_Zn1`Uj2VP4w)Ns7=`{V6qEr67Co8j<QteDHoTx7NE;|^C$oJanwIt-I*UI{ zC1Ru74;@ecwqu5P$Pt4F&xdrZb;wldl+l*o<dh(_KJK}2QGNP4W}i#LyYYQGhTPbx z2UR96!#$FI@%ltd;Yow=Ar1-++9kW6N>Y#rk!(Lt(aK)UW!w8G!*p4ZaG!FF)xxw{ z!!i_jJlf-Fkt_M%ICsY2_a*^69$SCF3~2~<reJL5t)~TKTdq}Gw_1S5j}5)<w06q^ z8t&702Bao6)R4`IA)Fpa7CQsjon{(dM7_wZ4cSo21@D!w78{Go@cTPEIdR*{xZ;qc zAr|PGTOu}MlfJS4g&0AD{RIm}AeQ7Q=~$_aflQC@s&gS%@ZGMIqgtx@d19ffH$^~| zQ&H4~tMTf8k`x$}_#SA)<j=k(PXSx64~Abz8@;a{7+h{G8GKf70jgLwnt@)mdO<MP zsA@O;<=6{bXdv8{z?963<5Hrp+oNpXTn%9<p)0EX6WN7Zybz_{+E&mS$Qy8o!jw%s zV&^`DpMo$YH3iKTw*&i<!`n)Wb{u&5ODvcY-5%NtvHQwn-!O`sBdT7-v9z}7C#_qu zh{GfAC4_{elbbjxl!$#hgR?~P^G+w3Q^|gCadLWiR?`?q(wy5>ITXVN^ejJJ7h2jd zp+=pdrZ!nY$#_^!n*r~&B&@Pl^biqGe7o=B$9W*2w_&@vN=^9v9MjtRW$L*;CIux0 zld~!bUJX-#W)qv%t|lZ&@#HHWn->x-(79|Em@#3r*$d=a!U<Meq03zgc04J^eMP#Y zIBY4Hyx4nmrRnrGiu-M4cw_HP;>8ZuovkRh!qg&5Np>qKUP)Jm8&txanDl@!GYZ!e z(KU>PXZWbw!IX+@PVFl=hm)q^p!ToX{Pv@u?1`w5#s)y*Pec5dd5~e57)q1r2gWff zDiKxDp={{pKEY(Wc{={~A9_Y_tIM_GLT17v668xDe_<4an78LsMG`#BUr<ET&|f{! zS9oH#f`W)<eKqcO1$3Zd#2y@5WzhJP+&sV$*TN~0Q82EwyD5;DVJph^^;KPVG-ssN z*d7Vj`J;!YqDOp<6G9LClxe<)#|7oH!=Ma^?+mEko}k2IhS+rVJ)CI$KBJxX^J^Qj zBSYmh?Uxpu)Tm8xW`7E?Tx!%u-e`fArl#FrAI*z^Ok9U~-Cgu2uEgh5%>h#O>>?D* zcVaBx0a`p!Y#}~>DiUfM<anq9&gZ>cAQ$L|7)J%IS-3kym|NShd+Zm#*Fp+!2)BYt z;W<qRml2lF%b*R?*rOP#@^KuKbJlMSJpgCaIZ<u_@Y({+so%4sBi}zrc<1u)pA7W@ zp21#sOe2>9YjAF<d`@p8(uD}O-yi7n+LsmW_y*P@N?rNU0vsw;!bLE=Pukl0w_qCm zt4Jm$5Q~4@*J1p4`m&%y6sYvIx}geC=~Zb|(WTwaH46xNjyhnFzp8dUywx-HZzA0a zLNr2EiUP{9v@1(rfN)^s^E6i1(XSkNxDXPEJB4)`l?7*esx-Br^>Pg6^*>qR6|&21 zDZE2YAPKA&9qf`oHL#c2@4|vY%%;83?DAQ~E5K#&Hvxx)-w?Be9MnBHq6aBtx3I|! zu<`(OK5w{49g_T5p=(~WHmrt-TWxXSfwIhsk@9OCIPFgXz=**h3LIfuzjBZgYEB8{ z3x8VwaHn1HF^+a~R=JwAIoD(7kj*fTR>8JYACqZz{@!$mf@H^w__+_MtgPr9uZ$fy zX+uO`L+t0*vi3+ESNnc!U%C3*7W2bnP72IxuXCb%60e0V*cLxTz_u9i+Gg7<X}k1h zc@BV()Wlj1a(NK99*d!py)R`qCOwe_QJWVU8eU>b4}5%l-aZFe{?n;wW<_wMKMq>k zX>K%Ie}|Sz&g&FLMUnsw;z%%YkYXGO+fS3gaii*_H?v?%2EENm)4T+MA*at*a25dL z23CW{RlR5WC0dw$3Wga>|1BO@fDtR(eN}d`kN}|m+#-HpQ@f7F9#QSbR5pbL-*?Q* zfp0HS2YMxcd3pv|Wk@c^JC7unz|$TUrf8THQZirjX+OX<5)h)!XNmZ=KwoFDFjF!H zqg#zhzp-y=hnE^<i-Z8%VjmJp)Cl`BBQEcMd6r+1K;@Sq@AIjzuZHl+*F4?(ja5Es zVUm}jFgBsDAEKGxUAO<!TR|HluRqu}unkgBCYAdH`1kswkEwm>Y5;R*-No+sn(Eqj zxI$B`95$-2pSAbAeOo5Z5vB|<xIqR;f5&@+6h_ozAJLl><o{yd@a=PIhW-QLqJ4hb zJQS*~om-eOfqsFW;S<Up|DQq{Xv1>kKFxfTxS;Z+p&#o&Oq4)NOUq`qZy!^Dvi*0i z!|{vXHr8Iz{$_TiLFF+gT^F3OMZ}6E1eJFgb}G>NO~G@{drfm=Z_!B$v8!Tw4cnsx z7z7-fwpT2O%F_(&*{M)psTg)e+7Bly{suQ!VAjc>&J<K@UMVfYGEww4Qe(XB4@x%G z|8bD0m0p-nZg>pmkvTioe?AhYF;C`y6REGEU)`B+@;nEH%O0KE=PA&1!m<T4muew; z%;4dwR^xxTdJyS^nn~vi^5=3WEG<XJ?iHB1{A^Ee@11Ek_s|3C{MIKLbsP=ubeK%u zm}K$UXQQw#Sm8HruyI}_SdY&Jev=PRAibC+1S>EboC+v-Sr4_tw|Mf}UXQ}CZM3XP zz<#4c9TJdfN!Zg7+xFa*ULdueZ*<{jr&qSOocwI<s#UBeZ%D~kBEAv#F}S9P)-<Ip zpP&`lLsa5L_Uop2Vw6}|SiDl1)oEpw^lN{YNi?oHy-;+;xrMb?sJ}xLB%NmfSK5=8 z|B7Us0OH`IRL#Y+np@aY!IZ;<y8R~KsgijeA;G!D<-v`uNQ^dT%K;qGlaMsAo;?Oh zPyGqzAWFW2KdPR}`s;z<qeZ9B>_KK1Bz;bZD&dn=z}Zic9P^5HobySWQ@|?4*iCjz z44#<VR%AZ_P>{zoh#0C1Z1I703%%f>%R*73k{3>E>?yBMr%M0GoW9Ur2$Q>oF^xk_ z7b4St5_KHn#U{xIzwKc8r^L-rP_(7G0#RcJ!i8GaA3lBN{hJFM?cLlKsPcN*j%7v4 zKcsE1R4o9Fb3G_qMP6pGB>ovG&<qA!{i{Ru6`a;2x}F{D76!pX;X+p>)ng#};xq%M zqq?+Se!=*+6Au*SDxDVIx~oe=&m2=d#oXF<lb^9<Y2}nx44U>#&fEJGBun@!bS>+E z8aQTcbkn<<v~aONJ(}-L8=3>Yo8>sO{96Be&C;@hRTD|!-*za`r2s=ekxEHVg#!rX z-I6v~Xqe91hMjUbbpHngS2COYIZ5zAf&Mh1lzSj5we~v&kR@>;AgdHYNguF?5-x&4 zxNp4Q1f7^vcfqOSzHRf`HpHrVcL*-HPfXZE2w<&4u5#a@L7T!=cCo`&g>9zlvo9k- z%kHC#Y(1B;AnwkKK!e4@;-%}YSq0*dHnn?JaUB$_<e$Q-9D5bl1$=D1>{LC>4yirr zcJ8Ux)+b#93OL+3Z3IH|cMNkH{X_v4B%sT?kC=6o<7443(28myu31|=8<X~0S6%!# z{_DrJJ*cAX($xz?JkU>o!c;y7dj{}v{{#DVQ2;Ux$L0)*AUnT$Z+VMLy6){J%Tbf( zFTdO#WRd!l+xoe2K{A-@3N&w5i8#GVX)vaH_qjHc`xJa+5@F5*xy9z9gE|yS2e3IP z(|gn#40)+TlWuC!YEh;4ie3qu$Ms>_;vADr%@$xOwLXZf{HM6nLAEJY(b-H+omgs` z;=M(m?6o(=T{xU}iK89r&ChH*`RKBT@|XD<%)0PAa4t0z%TuSgqRj(8+@=5<QfxuZ zbNwDlN<aJ_TAfZdiWY*nCZ!O~%`tw>eHwTAT^w5&_+8$d7yh~UQ9Kfm0fpCH1uSto z)UTy*5=IeLj%6FNPFJsI<^8g3!<LYggrn~vdne5#&5~u(v_PxdSXmYI-wJ8}BXUo7 zd_4%%m5D@sS@@g2J`pbJIRy>Xo7lwtIkD|@f?Ubw;W9Ddx-afAF{=+}Dq3$_(0vnh zCM~iF(~+3QLthe!Ig|p7;xvG(l6zj;Q{bwH=x||A-qvrsJa|uE(56lA&asoNkNSuz zT5X;nILAYI`Ly4g1e>)IdN=*L{kvJ7GsmuKX^{#96$(OCzRH<Wa|@O=l3viv3t|kh z3>;?5iayO26kNNBJ2pFgdkfF!;NQ*Cmnm2VxNprt3>yO_fmYKrnv^syHBFFrpik6Q z3>4d5tU(nmGY_B75a-d;!lPNkzNZGjX!Q!VIPwqRi9?V7z_;ge6zVqPl(9mB?N1;B zvaHDT$q$R3tGFD7;Jm*3*ze>?XscaJa=6P>gIg}Yp3*TGg)Y0hKkgo#ES~X>yvl3* zPV*B@;ED!CzBxo<)#>Q?-<;*yj+5l19x`D;ODI^yv@yY(HDsa+o4m5xE6^wMs)$Ec z%4y@>b3cLvNVojnltTuZJN@4LBLRXolt-b88tN}?TX3w2<*xWEv=-GCA!~(ocM4kj zOcclaY_v4D=+ui({5P}Ul^SNE`kAfeLT`^fvgm)mxj&4RQawrP_GBeoEx@8Xgr`H~ z-*%fLC2FZW6@H9PcJjDZa=STIdel>!l3y7M16wr1(K8IkKW}n6r1|Y!&`xs5ko=|e zAlO<&bqvJ+tl3&A<=F_4tdVuCPwm5aqUSkOnI^&D=<V#;|IG{YKuZIUT3UuJus1Oj z(cxZszwC~#7f`FdfqG{T)`(ySIAWZM)(f))APLpl_Ln3Sj>r&>$<qa{?97n(feS|x z{nOACWwEyi7N-q7;~*n4Zx$x8vnpDy{6Rb|hU8#4imrqVM*60Q#KVU0OaBxFwe7Eu z1cEtkJ^P|QXZ}jw>b7k%>pU`#OT(3@dPmQ0Og4*%(P-%sgAC{PyASd+>u%`I6N?A| z`R6va^K19UMkCaGdzsPvaN6)E=q#RhF8ilSQ^p2v)BQ8a4{-Xsqe_MK1{KOsRaiyu ztqvwgE%Thq#pB;2wQ!dUsBtl&Bb{QHtfbIgl+v>VzNBN-yaB@HltZxs1+DY?aJb0+ zyLxBW?iG!;)8$anXS?@eUhb7K%%`F;gR~oimDW;G#0fzd814tsVzaiVP@3(5PDks3 zA-!{N(PP&UtyOQIov#=6K^3WR%~7~!u18=0#AqEoh2!Cs7JM)aL!pWW8@UoGP;cB7 ztdM%(vewpC!L3nOLzp(2Ui%pBlZc)JR~#FJUaT>Dj!gZj{l_gJRoM>~TVdiCeCa5$ zkt&IbsQRUD-<F`W!H|8n<jkJ(6%RK=;-alxtaZJkN2X#bQk?i@)&=AjcGaDY#*--J z57<YbUMeuxj+4K);*s~2HsQ<7lA5*HXXbQUh4~NqUOx4eCa~7+)a$u#rwFjR_qH({ zy#bHWw#;sq(fmg*5FKk8V#CKc=8X}P$HKv+L<3{4hP-jmJ$Oe0&5mv!YmLVs;M2SZ zH8P3r4OQ>?AcNiU@6ZD4L+Ckjmb<ipf*~NFx*m4n{lcUKbNt&eif698cdZXRHRVWi zXQ9B)&>7Y5sFZVeBY4{nQXicAm$%&oAzJ-HMQuuLHwPHnqXiB<l$o?T;OiS_t#<Ty zFx9}*0FRGI1TfDVY9OSu#_`o6lo3RYN`%>vVSj2KQmCQETC7BN$V(IT4B=RXZmw~T z*Per!Po7(@`;gcAUr|$f28A<}PDd===|tdpZ|8IuvmX=?qp?X<jaHVkZkMM>WKvX{ z;ao{x95wbS&@}atDdqoxrW+v4*HY8+lfvqBCNwvq1PVLA#+Xkdh0T8n0$E7>hmOB9 zxUaudx_sy#oMU(zxHI^xr7tB7ZG}w`F51zR!3q)BtkIh8xwX)AcfA6p`5Y;w`-u78 zBQFbBvNM_g9Qi;uwBaDi6d3=dND_OknsVdRC+azoB*BG5I>gvNO1HK>vI_Y{i4+O4 z%Og(+be44bA5IF=%P<E8GT6ArR1m!9@t#+Y=FK7q=62YmyZv_^7hL%p`OwgnB0(tq zNh2vr;NN)U6a<}UzxIySirWBp^UZf;h@ZetwRAr;ooWJSG{11LaVI6QM|1@cQ~yyV z>?8fl0xG*{en$yJtS(<3L=$M1dQ)iLTD%QH@!v&nAQ8ubdMX^PN-1QA^Ip;>U|&QA zI?3?T?_U3!_g<j-$;Kh;cqfDl9zH@(S@`F6jB~nIv{|M64}j7EKs8SqJ5T_W)V4qt zQAqxp*W~BVDT=(?P<rwpN#_T7#$*u?ZUAyaDYm<w+g6!GKVZ&fcy?|_Lt_Zf%0eRT zgof*!vz)qvX(3EU;Y2uwC;|DP-_!hvLNzY-3lueHO>9q@Q$(7#K+)Zhsyc=H<zE&B zng&)<LQzt=r8)vSv|7XOoO|!pBZQdeK%3zH=CoI&6R&&bziOmGhdRU3B@Gf1xI7LK zqs48-nsdwxfkNe&mb3FvZlneEGIOrNwYHRkiXv8yG*FxmJ_fGzg+#uU>VLD7+r+lt zA4u+W_5A&@4Gbo^8#l`Hw6-<$-dsa;l12Di`1;?o_c#j?4G7FH<Y~D4FQj41ptjY@ zvSd4<k&E;u$YIcq5V8shQHj{h2ib3v)CT+Bx;uYC8U6$wdFCLj%n~_-mdYp@Hlv`R zm5z@;;81IuvsLIED#=ek_zPA8@D`KSxRf?D3|=U`bWa+i<3IGjXlhNQXg*`4>f%xq z$tcYFq^;`L&_)afDM|_WAE?X~PTSRR8W%>OxVrdtv}XX+-sY_2XUUIVjrs><#dW~G zthc=R-%_GHEfg2=De1V~?U31^Tw3&xb%BL`!fBta<{{va0CyY_=OrM*CfVQ|h0-R{ z-+9BhoORl2?n`aUDWVF`3*9I@Ij)I|A^n~DQv|^ix#^m6bSVfW<4|wc(t7zYMA!Z# zTK-)j;Wn$;0858uCoMTyD3oTs9Y!wHGeG{AGoeqG97J<}c?}ayon2in9$4!7YZ>;B z$q0jCQCxU)WfQ(Hi6Ykn0yLnvDJx@pvcOURD5HUROEOTC-W;n+cD>VE>IicbnfDu5 z^-Ssu&ZVL!;riiQaK$0I6`lBABphWd21MLNpi_0LH}99CQzv00g8V<^DFo0EkzBJB zCgTLL({7I&TUwN~`&%!kY5cQ_9U)gBscv2z6%cc0oE-M?bXCh{tL%{v<~74r?qk7S zx^A7dZkMENciu9kFa2XaJ7(toC~G{~N~WPiLFHWb9xNT-W0lRVZA?Acn-)^*e+50X z9>CZTimeBR2T_mRCN2zofA71d?EJdZ<Mm?h)}ci@29qB|gzlD`sn`b5`W;E)yqq%Y z)qRacF>R(W(X2vS{)$H)CoR<0+zS}|GndM+^xasql$@OBS2ZLawvNI<IfeWFSh>rC zF$D$-c00p<)^Ql;R=*RQyc(ZB6T;{o>%<q$|CJQ9fE3Csv%ScnBBCV_wNA->2J`Wl zxFgUtqYx`~d(!#=!_eQ?^N!Mh%01aHlmG8L7PLek2MQq`jfE3OacJ0`)-q`8Y60;e z5cB7g<n+_fC&$Jv%WN!cvu@pnt8&BrfCB#+B9DQ_dks^_1{l&{t=uOYK1MutliOR7 z0fEcB0h*`lrvNASrL84|#Gir-bOKxm{+j|Ah4pC^CV&zX9{ju*;l`9F_sOmNI3_rT zMR%h5DuGJ4Zu+A;nJI#EN@*h|>*k-OMI(Y@8~`X-J>p>yJscmM9b>uem@{0~?&$U> z^|rKui*X;!spFp!8Xs4ho{n}a`S7uY%l1E)$rS$CDfivKzmGOgN#D%ylPwHMh$#ul z@Hg1?aVq5ZkE)TGgW`{ZfBOaXxHecv_lO0ZzA~$6mSwIuo_(S>x*_$6LUFnNtxuIw z78T8vk7v!@l_q*vZs$&R<+|nzdni;NsjB!)O3uGZB;1!0eRDKyxEDaQGM<T2&}|DA z+^)x%DYIHCn<P3xX0{3CPF}%0^XsXxvs2T+>IVm0tfno^uBNngu9&D-nsyy%dpzCo z%sp0T-^dRqsfoinuQ%0HM{umk9-YWMF;P~L{l>ID#5klPcjEgexhje3bG?%f+aH(l z+_X;V`k}MSr(43J{A82nf!eO}=7$IMDw<Uv&tjC6W|~><Ae~@THV^J}JW5_UA7C=~ zt?ci2Vpr5}IH!Z}cV8@4am4u$1fny_WFH0L#jPEF1nP=;g}Lh`ry6z{>U37wPhoOR zsT(rFM^xLmViLC~1hGhWE{o*Po;Kpsdeb%DpAdSJ!%Z~!veuYmWTM^*^_0<lwC0)q z26icvG~FdXK6Q3W$~LHXiP0;~4pq!Lt}<%FtVYMmABlaVhtyj9FUF7unhQ`PdNGVb zDbHnN1T}^3)<<UgL*3nWW#;TDi{((QIx@s0=+8D&<|7>K_kml<9TO#7SEHWG*R|z7 zt$QJ}l3DF(wPZP$?B4p=#OQ{}9J9>cu>%#UvJLiaEY>3vYGyEPwlLr&w^{FdUoPmY z;OU(Bfe00!f>!PS#pa;11D~cxxKow_-AO|NR=^#ehWxmK9}~#PvA3H?QRTl3Z`LkU zxNA!I5!!|wg%uUqTg-XBc(=7wNjZFzQi{M(?TYP;mKd$S7G+ja-mGetY1yni>h_w| zrmEk!=2K^VLWTL`#M+KQ15P(i0C0M&m9k-O%jVJ1(SZBgF<Hr{4YMi8Y7z+o2}kO7 zYu#ZA{-B{6HiOEBv3aARj`T+iUEGP`hNm7D>XFuweYRiL`ua7yOgC-3n!C?arVHa6 z+gVoSGC_mREPeJGWiupsfG+)a_f5XNmM$Y1H0BK<C891Zor6jr>a7OHD*_(xwn7!@ zDVBw4J%c#eiHNE@NMHczs$BcPT=`#T_Q-n&3UT!{9`dny9(k-NWmM4)74^In%XOw4 zhr7a}Gu8>d1Eza=QoRK;+@#pr86|tq>kvS>W)s!d!mU>P*q?MpDqn7tE2pfoEj2!V zN675WiM*8Z_>%fdtK(uO{oYOZ$hhc5Nd&MdmdB=wD(Fk+P}wy7vUg=bu^|S@xDY1G zt10y$1-z47MLZNVie1e?3ZYo>hv+JRb=o`Dvv;U|8$R50@WN~=^Q_A5fCQ?as=$n{ z(G20|9{1X$bbaV`8TpM$QEy9)$<j@|pHShh<piego%oWTtqcLuYYd=}J<F_CV7}8b zal7!kLu3Dg#Z1Qt6Uh~xzsw;ak9jqBd}E~$Bq4>P2YB|p!(3X0;ywFFdEfPu<I;Qc zj8lOq{A4vcI-U-*)2a^M^C=uUpD^l*O0jpTGnvX5Tw9x|VOAc{@+G50?rgK3uvqOQ zsp+WzHn-Ryolm{byQfWEws<s}6gDMlKy(hwF|fqjv>%xN52h#+#3-z;TT$v&)ua8o zJkU5qh{HT{`R2^nXP4;WyPWJRItr4l-D|}!A!61S6={**ruX6a<ByL6qU-~#RVIdH zJx@OF_iZdejhMPrrw-UOHP^b%6w)i5DU0qNG_W6`3G8zCI-E=o**$x+81hcOMP@-M z5~0j-F+N8^2lHb2C~XEjlviFXT*u6pXP)7zxA(g8q@}HaI`@?utTyUY_SZcu?j3mW zJJn(tAR8nUAl*2?pxXUY4u`t>y0g>D6<0oW&UWTby&q<v|6G$iU|S_?(tC+#f(zaA zwENKu2FOmf&&<qx+Nq|fKM_l*`iayCdk6!D2Bt=f7%Aiw6xe>QM*ElF*n6Gr>hT7h zH`%xL(^*H=NO{T(W!*r9@A#l5G~VwE!VK`PuNU1S#Z;$pvt`qi?iT~CyqxYw@ddf| zt^M(ZV%m=Cb#BFd)}`CsM?R0=_nM&%>{2X?eRD)$Zz1Ffj*N}X+oQ_Ik~#myP-sJB zpq@k{JFe`En08e+S4w;EPCxKsjbP^IqP<i#eBz{YnlP2zeh>{}4a-#<mYa8dcoUVS zJ8E7pdoj_fp6a)%*_#XYKc?z0N%RUy{n~BX+Fxx|JQ(J$&gEyBm=intqe8#i9i+r3 zH>d#pGSc{&X3mCA+`hN4u<hiS(}my*lxhQskA*gT0%vAulBpuMtLjqnz@lZ%vZsrW zoUka37WI^=?vY+O>wlrzWa??MgR5vut?IO0OnIa9l&(GV@4ZF)&gjR=-%;N)>E!Ub z&$d3RILE^<ioVFy?2V|Qn1`<8XI+D93*LRo{}|p75Wfdxp!WF#1=TAao~KIPj;Rw1 z{jO$IWIo)$J^3tEDNDsD($Jj4Ihg7<w_tnDrPmB~HGHW#ZJm==JbTwkU_#1cqORK4 z*GJoSxn%#8kXGyHd}+Ilf&awt_#1f_bu%5R^k1TxR<Dro&i_`44|MwysVzR$ph<zV zmapPo?}6!JJ*!8RY?YVdFUNznSVd*Q6Bn~KSJXpGdUWWDdY;3Le%}v~(S|$~jL9=X zDhCe6x6}(aoK~M!?hIy8l9qK>uKC&?wtiANcjV^Ak?jp1CRSa2>0&T6?%q4>ct>fr zeDqFd#bYI{i&G6?y$7+T`Q({HZp188Qs|#CTPYB=SsyJ@$psho1p_?ZB_`>~Jt?(c zGf5QCtQbUN9vx<N+ByBBtc6Z!`UR?_k9DS_YF>pA+)7Gsp7#A8WnUf-_5S@oM3%C& zXtArbiX>#6x!h7o6h&D}rL1Ml*bODRw=B21Dcd9!vZoowHlmCQqwEHgOGd`bSTcjz zey?fyeD2MBzPDfh=rN;5ulMWydY$t;&+|OzgoF{Dej)8nd-imAg+*etA4fU=ct%YH zbKcS2RQkM}Upta<qm^mv&O1ImZ9+y_Y>GDR%6rFf_|AUp>Hq3_9qe}rsciVCiBN}q z_yaW8SR$k)lt+xHS?~Fp?^=LJfS@$$YDGxKt(8+V#B!OQt_1Ryh;-&ajxoG;fo2te zax(nbN43e<lT!h{!0C^p{pdnd)1W$xxA{I1(zYFPg#O>normuDSprn4oSI%!YR-2h zy782`bK%g#QJ)9YOiw<91k#G744WOkSdW8qAy3$z-?Y~M3k(B3_bs@A&V!LYi?uRN z1u%g3+b09i(S8cv?Nc+6d%(i|E_6@bSfHKR^XnYWH^JuSGm(yM_2v79rh<6(pf6PO zGu&bQq-SM@LBP^QJU0q=ZAA`v6%%{kX8MVY7+>s8*(5(SoEwqu2jbK1Kau5z#5e;| zm>ec?Q%DT?;3_Ny5=fGMWEn5djzgt^)3KGE=-v^0G~{`~723MtwJmuCSNzf--tikx zE*v;H8GQW>(S~v|=x*7c_Xx6{-Lyz@nJdN4Fbe!I<k@-YEQHRZc{ik+5Fd(kEI{p| z`wlfP(pNvcVKm^DkQ??7gBD%40Q_(_o^WC@D^+3<qb)MxEB;>nBDjG7O`Y*qiYgk< z*Ya?lJ)8nL-|pG|=nCx~SaaFa=}Z0YYi{{hj$&CKiEgIQ;~$5>o^}AVexZ!z+R97@ z21T*{0_cflTn^>5Ll3z{QP{S3xnT6~2$&j)P%c_^?;1!JXklM$`sV_L9EBLr7V50k zH&N=}siHbj0dio+LIfKt8NXaDXZU$uYyW0Q-|SY0<0TnqI1fWpt|jM4^Pa%pplc>v z2$|Lcy-FKAoDt_MC%DNPBih51?skB6U(U_XSBFe(_^kxcwd%oK+DBX-un;+G3{`); zR6oE5P>%>ib-4U=ije2%w|ig%3B4}w0U&N-mhBwwDdVC1ba*j7t$MaksaO>N#jci` zmf3aGnc~_5e|hzt(eCA|np{dl2OOOXelt$k=fx0!hO?V(THU}g@0=flGy4RA+pd=s zb6V9~3aV+U%x5*sf5!QB4SW?y&%fq!x!D)Ll7uaOPe&J%4c{Zp|Lo)ekdzZP2x%Fp z0D`IR7IRu%Qn;+cKq{I!RGja@o_?F8hTy*=sWI0Vy(>GXfZBLdd5D~M>ZJI{I=nwQ z?w)7ompwKl7<d(10|(Bj^(v<OIs%gdAVxs@c(WyM@HNp2f}@Y%%&oq#YdIiI@GV<l z<bTiT|DqPZF8hC0$cl=C93Vkjq*WvK-oUeG(%AV!RrV)%JEb*jS416g{av)S_fYd? z@;!}_ub~5dBMJCx^VCdQ<rFCvoxd=CEF-jF@^WwQ8+3LN6Y=B|LS*F0F?!mKuWUYc zpMcu^w-y&*k7JAk!iUh{W+9~tu-{;=kkAQv4_9$gDS=qF`glpppx*f&?1dX|KzEQ8 zSBj(`$SRZ+E_h&j*}4`@evLY(2!&`OG0#uZ2v6`7BfmB0d&Z+1_CtEnuQb2&3?@IP z4F^`04y-DitT7{mowrlBQ-DQzvC<C4%C|GSd;&JDh)}6ixF$~)#E{$N2$<(K>%o5V zju=H1;Ityr!)~frd<n;~#%eGXz1V)u=sp8*#S+<Z@$I!#0MFs;yg3p4eIo3Ka5IOG zA2;K7{Lb9T?_W3MB{H(PYB<vfDDa4<*ES2Oc>cj0jBNyMW~_+9q?-Ofv<&0T>1e-( z*3J-%&+DI<+p7YEpnDN>!A-k-QVMrUKiCiLg&{dCCvBT0xvl`JZ+5LGTtwM?0qrV0 z;ITu;K(u~JK^1+5!%2S%_E?-Io?OK_Ki%9RKs;ZQE_pd;zBM3NWDEXww_si%T5k<f z6{2{&WFlZiP`n}5tztg27@yj+PT{~zzV<SWM3pFi<D`m!-*FD&uz1Oyt_&<zqyr0r z%Bh*59&N8;AH=92Dn{tbJTJ!-m-*`DNS0Yu9q*iaSY|d#v4P--pQa|hcdJ3|xidMn zdir0Q)d;ru|48a=2Gs)U{MQ83-Rhab05`8)C&b@B{AjZR_y@3~wSXr#%`-K<d$1ki z-5u>W@Q1f?R4w|6V8v-C+={4Mih@tReX9eF>OiCb)Z0IPvk`=7CqY!CnqP@jLRSPA zgx(QPM)8yQ>ho$8{@}=UTzuVD(G|v^b`}T&ewHTo_zfIs>wbN>5U5@)xNQZK?&~AW zqf-YBUzt7?eSjExry+ZcROd@#Q%8Hs@wNLSLBQi)OMLo!rDYuYZ2^cYR|F<awHuuO zNapk}(47Zx-hnmSB~^$?Q}b(K{?GL2_fLW5EJx?}lD^8<ax#CP65<4YWb&?p)U4e8 zyp5uZoprmEgLpEZi{}P+e;_*)Zq-tZX?dQo!lJoory8uq5wD_`+lyK7O9P(3TvmZc zJM|(A43+D0dLA$@-8CQH`1Ddns}8Db!=-dHswLQcHfIg(q^Ta0vNOZ6cTf(2ecR=} z1{THRgieEbWX1;FQp>f!aSuSod+EH2F!|@-&NMhGhJuPwkxC6Aapz}+d)DG3&e!Da z@lQGfszAub{PZzJ4O;ow27)<QhxO)gQsX2uN6qpVltH9XRJGmI{MFn(-nruf%8x?1 z1r6_|TBct!_bT2R%Ne9HHm~TRlRzPwbXe2@pv~0KpB2LX{%{AI1$AHkCTGkS@2;4l zWQTZi_Ki}@>j;rH0o;wNoVt-20HBT%{vvSB=%UX1Rbbnk?Og(ny};f8M9mHQHt8hF z4!NYru9=zbPh*uuM)FR*GfFgjjV(Q#nkm4!qD}T0yN5fZR)49}e@3E>t0Mk3&+vcY zhZ0F(ANKj)J}6|*L$um9MJwChRIu<>gwYgB3Q3G7!)s^TDZS6}5o(c7$w=LC5M;C( z6ggf?PFrf1cC-C+=0o|wg$V%@?|)7qd6;KTd62gcc;ek2?gnV!iQo1X1zv|ws_{Q3 z-BOEWN!a>_e>(?HZqhflS$|_Aix;kcWvl6r$4?76qW|31|Fet}_CPC|1Lmx5a_7_B z@k<Z18wA1h)Rq=fQDgFdtrPhxH<$GM3Qc~PQeLsu!+F&B*N-2UT1_03G|H69UI+E_ z>rcITGMEV339l6|^JF|Lq|exuH$J5TD0Ex8%azjhaRLJf;hEnjXOi$NOMu?Dc3Z6i ztN#=f2F64GRPxco4_FN!(oyQ-0J<j@b{sMf@^HBY9wdoepIK5CV0}dI*>pRuzFCy- zVKOqeZ>9kJWX{bS9Y4-a2&Y!3()ZnF3p}{O740&$ao_Is&wJV55MS)^?ymD3&brf| zL9n-}w**OgfL!PWEOh17OzMtrLB~(?*9Bl^NN7Tpw-5q0e{JqEbqQEXw4aKLgL(@# zX^>Y!|HG=A3W&%IgbdViw)crK91S_I^a#Ks`vf%F2Lv7hnr9n|`E2H<haWlG4_#&m zxR(|qzm<b3(=*VHUh2X;F;OSFf3*C!@c5Nq%j|kLN0{CziuzZZ7lRWBtPoz!LP$8+ zHP~%Ppy}Xd4`+!?IXR@vgS+B@>me?+i!TTxJ-Xt@%iJu|Ryph7TAXic3icW&6Pj3A zu{xs$&7WS|J(J<*N<2G|4z6O62<hn*JH1AO=?CJD%0?_2B!56;o6|X)CD8bMtyQ&@ zdkp>8V;v1DX@STEqq12De133W!2f>k5^(L~g_hSXuc6;|ojkW^|HqHUj0(3k7MYfF zdo%c};d#d4mSVsNKBV>f2nY?-^|EddYka1dyZQhWyl>bFG7qLg+4lTJ&NK@@QJzt> z>VjqonQ@!VW?K;@)yg*n2^H!7;|i7t#?pufWacf-!TLIa=)>sFU1bYjB?t|^{rBxX zqxln2pPBKIaTjKNqs*r!S7(ufZtW9ktGKJb{+S{|HRm7%OYAKo-7Es^ngF`{e*9Qb zPVDso(C9iyB`l2Je8c2@&ACVrW`7Zz2ElTO@=w&`f~qtHu+qYb=ZM7yui(%FKZucQ z1W7=tUr_6*lorsZdsxrjG+bsdYG$hriuvbw&qKV8>8mwH4E~I77^&$dt8_e}4VIgc z0C|n%M4%7?%uy8wAiSUF9kX!HYs~;8m&W-vnS~Dm;_^r$HB$WHUzcN(%0QbskRdJ& z=+0csD*PuZ`u-QKL*S+zC7pZrk8S_z6r(iT^rMxDqn4<1L-n;H%}>IQpLk)A;nPQ4 zoka>NsPqVG$IRk%)Yc?KO6PAgO?AN2z(IKSDGUU0zJzs9vR5~}1HrXvb-qpHuN$H~ z!zo1a%2=Z)sFb>4?>QF8QknmcEY&|(<B=e+WBtc{7UwIRYG9z{z=ze><jWL%)5PrJ ze@&b+=a1j85+V8W0VQ%H;yv#0aS@yr#MlA6amMtYDbO%>L%mhyQr~<*0z<%QHL4kN zoY#~j>A7~qsp4O+yKEbV%~`JIMY*eHSaTN|4z3y6t-DGZBs@ogjLpJaJ`NY9W(tWe z{s|_;y3_%?7U%kWvD5%4*Z{jG;dD}*RI%oC-Sa<>A5dLVvo&^a3b@;Yr4;+MFu`7u z%D>`caP#Am+G|1jqNuF6EWdJald1VAFcu2yZBCx6YTB+Ap|dcsF}k0!It+Ul2woz- z$R8zy#AZK49O@<im&n26IwY_Fe-O~?Pw~3whr6hEM8+OM#?pYMBO8Wnh70n4UvcsM z0OP2mP#ZwG50?vUa5vU?#%_pW0DqI2ZO%j#o$J1F^L;qMc-W6}i_Z~M?)wq{l=_wy z==}IKIQF#qZ%h0i*ZF;Cd#V8?JemZ7EV6Pjn$pV^V2PNu*P`riZv!LC&)1528z)I> z)Pk({K|n=zirayh<3vl19|SQbA4mew@;%`~lYqoA&=}$Lvd3t0Rz}|Y6bgoY$xp}U z3N86DE@xNHyup4B{sXkvCIpscthoj>Z>9fAs{)^qya`lD2pOgsBQCvsJuLB~RSyHO zA{D&Dq&t7cdp19u^cGuH=2rj<as}qqQox4jQWbfCs8aC+0sM{nN2LM4nW^P$^++qG zo9v&%wS<+ij&N~1@~@a47aw_6=8nXoI5ldU0Tqm@SK>1P1QzER7=Cn?{Poqa<QE?8 zC*?(AQJO$twCP7=5ALMXgSs}6k;m8MH!2&Q^;rd@tt=@_^=Y4kpHu_y{`hc;I1J1! zZf|@E1v7tD2b3afK~QDBP~KB!xNN%<id&B2o)R(i^AvY~3-VnJ<b=zI{I%H42@212 zQun{>O`Eos9H00*Mf?mP(%LP7Cd6fG0ba$0cmCJ0_-@YR$AH?bpGa)~rvdqbqWw4Q zIt#&VwKHMK|COxKX)rV#RCc32+x2R;4xTgjZv6sy|0WrE9OGJEFn|2$2SXVt%GFnX zR~j=mcTHu1X@eg+Z};-m_;WlqBgHLWv4~zE7h066WrAl-HQ#}61^!S%Ko2iv%&_)} zk9<L)l?pKbz`y$8=IGl%f+_jemyFVX`{@23j{^f$7`=b|=ehp-ml9Y^mS_`5D911F zIWg6B9M)p)eBuR`$i%FNLXZ4$lr(t(grbV;w53-Q^$+)ZCH0x7#r@fizUXAcWjH&s zk96A`BvlMDl`;Civ-4XAd(iwRL&Fb=cp&Q-U>*i!g^949*gwFa7$h*swCa9giu*O9 zmLNN65$zc-Uloh0GPTE^w*1_=CDz3t%D^LCX7#eQax?F4s8184=5LG`BJrWL$&7%= znFV~2LP1sY(+dm~?=8TPZ-zbNlGD<;=3Z5;o7AHGPtBT)e*Wv#GKFMnv@2@EHz?uA zS5^4gJ87p4&h>wLRR8yYgL4hiU0~!h=fcN*8g-=>*Aak**!i0Hsjz#nBF>8Rd^2!= z+>hNWiJjN1JURRYD4&G5E|C4$@EBbF!NG+-!<8DtPsWG)*A9(5KNfg%65PKR5>LUQ zNT*dwl^d}IRQ#Ol(<m=Crvb{%9{Ctt%RhS$WNwn4xg)5TBYC3p-*e8KE&g#r4)Bk! zKuJK&hb&nxjGDgzBZIMI@*mnXUYZXLcRaO4j2ehE8-(9BY>7*5_qgm3kg)ke|9&Qu zb^hvW2AB?<ogA3ft8Wx8*z=occiRXzC~%{{Jk<*Z#|k~Sd|%IAGH5WF{?e~sNg-(? zNFme{NIl{CLI|Mmw))$djRQsCs*Oi=ggnt^g|Stz)_6#E*d>P*MLx#?#_=lap5d-O z?&kmAJq2*7fU_LETj<NlX7D`T9Zx9xGn&fG9vgg^JS#XoTU_^vk7;6kx8e>y9syCH zZC!2ivw*pBKr1umK7p)u%G|~KQ9xQMs0v70y2d;+O6}(2+`tbDmedjyGW4>&n2VRJ zq%fukqbrF{up`pmytTn%{UqlFEWkL{XU>T9+9hAf%ksqv4mV*Sz%7T{T3#E<U9XTz zj-Vy9c1EO4*fKL8*1T~eN(_~WjG{RsTax15x~19ce|o+<;HWwdK<RL7J~P|qTe!~V z4K}qNgy3jf0>W8qzenKM_|yMwdB1px-#&>^1vk`e(o~3^=EDh5SJ5WRYY>XOMRO&k z;m^xE>oHS!U!~m~<*aT=i~VWFkTjV!1U`z-Wx4MxEAXC5KHa75RXYae4oxw7PdpFs zYqVx<P;yE{-GDH6Zr_;`GeVx1;~W15PsKhtPDsn7ZO;Z%XFjdlhvu`~Q032mU$Dam zzn>CMCPi$(1_zjb*W^~$WPAMk!_+~*vn){`7gEGr*CFb+q{%WKaznK6c-EUNg3ffS z#LcqPU7R_aB@aIB?cGkW7fWnczii;a;rCRf$=fEG*iPEMP}XvCKU<QV=zS_7BKa4y z2g>Ik1lCC?JrcLRu`1<$`}=eK9^1;!T~ZS}xm3p~0dxPU_}e0L?TpPz4&li#nJVk^ zF%rjit#OK9ll>Txwnni;TeVGD(fc<P(VjbUdxh<<raJsLDYnrEwxLt5&%I>4COP6A zD)u^tm&*IFn^(Cnc(2mBt8D27wW>2HXYx4iygBy_>5cI-E`<BFlW(l^AM>WY@CV`$ zp5apa)gpL9c@VgbT@~N8`Q2>QXExe|qZ}cY3|BxQiUa)o^tWu;A_uXU+Z*(+GkCtZ z!bU*SCBbJ_FJ^CI;<wwag3;q4KJ_;&)3j($t3b_1RjX5Q7eOs_!XaniJ-2vcP=DO- zph<|M^E##7s>V4cXyx=uWhR+sstj}hD!w_Z!w%AHK(?~u0)3+4e)+&{xj_?`0-wOW zlu6rgx@M!L2)knjj^xsbN2j?gpMb>4GDzS-j7wp-#aKu{0K;J}Bs@I4bH3-lUhl^% z`d{0<4)kU>9c;ZV<RYl1!Y)EWf0KPQU_&wPDbkDRFWRx@rf=~_(cGb0c?%g00<P*W z^V$CP7PN0fp%he&|K2gp5zlRSMctwR+ez~<ITwS$7;g43d0v}`0s5N(Qm~^vmvS4A zHoFr+?JDf7vaW9`jgEYJb+&_>lFn^o;T`5c$BdD)MXhEU5z5AD!SRz*okl1VA!T(x zJkni39a2}fu=;4N4R(q!w}(qEqAc^|&y0a>zD|?OK-!;vjr-$JWf_u^ZW5$iN#YH! zMeFNSPrvt0l{cm*-P-kebxQ3oC30SEiKnvrFgNpRQln{YVcreIykHq|UrTv|n|3Df zYn_(sjMo0DEtOQfF`{3e8lPa9Azp?2>{E=rTgh6cZu_pvg$RVe#a|!hqp_#3Q6E1% zjA{{p=?$}nCPGfwQBh5RA&rxttm#J=hm(yP2&V^~F74sesx4&f2vHU#oQ|EmFvoI$ z1QkX56<s*ABo`;!g$q68leIbe)XI{ULZ8EPp<@}E1NTX==@;Sgx2rZ$DmTcj|1&eP zwS*Zq4lc(9pg0NNoif97nwy)98%XW^t_7GZ>Fw#MNLRw{SQI}jT4|jnAcLbM$DRmr zal4MIE5Kxj2#ly1lh=@mj!-y}aOIDo`>7Cb|HS@Thq;PKZLtmf$ttp;M$_KMa6d`K z@|a<IYM@A)ljMA_722pXV{bVji;A0P6PFgtne!(d2z}eK-u2CUhTmUl;kG@*eWeAx zYp|-EVUUFSl;4ukO->C=w|TlFusN3;8uG>{@nlfJs_A}LqIfbPz4>F<c^{vABO{~G zl013i!QS2#=Cv*aUqd0c;-4_0&ImlExkS8&l#pNb7C$C3;wb0lV^Hjl-g)>`RnaAs zR_tCcV6&b?u}JdKzDuxbc>d9u*0g}6aXS(tP}$iE3WU@gQ%!Jb_}hlax1#-8quy}1 z!GxQEF?~EexC9xlQedBvV%f$WtO?KGHFfWg@$06mvF^d$Qyu&W^@-qCH~<Xxzoby^ zJN>jp8wHKEGRYlELWp}U?Om&3QRBEAwPy4(-`1Ko1mN(gM-X@!kv0bZBid0S$=ZT2 zfC&Nz4Kj8QT#?WgAl?=CP00kf$f?fYcsqOoKN5c68iDQl$Ucj-?evI);Yid#d}M#b z^#0e#^SCW*s^#$1L>zC8lsYl+>{_~J*6?LXwO|61$$a|nW32=`Uvd-NnB7H#Lfo~- z(U@D1Ky8Q#OVo3y^RdPCQcR9$ZeHDAZ9L1#kl^6*i%Xg`PvlL`w3X?LkWyKPjo7*a z_fur~qm_C(W+-yh)(dL%oie1jzUA3iy+G)7<z?zAMOlxqgpR>}5jIjLk1>DE(ak$* zzRZ137Bk_kW(gioo}ke(`ap*5=PrC}4p{!Y>Dq6NE&g$qc5Ss=36s^ixANCQqbpyp zwg1k%dHW-!tZOBgl}uY(y7A_lBl;t)v^~<CZOdl@D}udQq*VF90=C~6&xCa#W2&-& zK&w5bF;e=1-1i_t$s+Mu5=p;kn637-Mza<rGJ?Yw%y0qHoryF*@x9U0|Mng({(!7| z8(E87{HvD^ipPQS&F*5ufz}3%*tv;pqE^@X=bIqL`W_}hH%sTgUh3SCh=}yU=$y>( zN~r;R{$C`h@;QfR{D}>zPuBbF(dVR}a=s`*@`oTk)lpqC;;mb=M>3CP-wMXCI>i1U z^~pdRB!?)w>MJWZiq)r#Eq6H;?&)JPdUqN(KdiO~ztF4WojVZTDt-p&&>9-fdwLG1 z7xz#9^X3W-9#cTi!TG@SY}fxf$nV?Wq8pP136y*vweZTzM@pkB2%n)B+oKu`=G;UC zZVWQvwWslmE;ghM@%rc(__ZW^_G|GlN+v$wF4e<X5!RBiuVvOj(#Y$$p4ICb%hO|h z#=o~XuKG1<_5)_Lip*)vg9JijO$n)$)Plkq9@mq&<Kuf|thSK}D;*V6SXekZGBWbQ z!g4Ic%gbtRj-!^Lu$U_San&gYz~ih`c~bb}7h;kwDlra_`42S>$K`czYF3spuT>2M zG+@$f2iKX#;w-M8zU4GGk6M^>*Vh~xHYm8atVh=Gi9z(qQyHc38;BBDZ(fhF=g^dV zAOqRw_Vlg4uNgI$mNoNx=iqRXDRTQ~sDTFwUB2gfL{LyitX-tZlMZCj2}0MPG8sh5 z^dN2KEU>dAlF2G_*^H!rn>HnQ2QjlCPf*-V^8Lr<`06;%uQ(u{+`ZmHnZ(8J@h}nl zbG$l&)`$6pEZ^1-dPFO9ESKv*dZ{YaX_;|c>X4u2J*h60!vT3V6V-iQ6#8`9DSN{Z zvWy~eMM*+SXhcMR><7QO`-)lE1{PVxIyO0{Yp{a>^w^`;I42gEPdXOSuR<b`^8bAn zmtf<+%BlkVh3U#~iyvCjMEeyZLh2Vyk<{plwelRIFd}_?c0K4tnjN7P2HECK!dnZN zVW5T7;Hn}_c4Hw@hRcS(7|Ubmghp+ZRLOMj+qkwGtyrS6dyePM3x!i=%Wsvxv}hU{ z#ntlP2W$#X^rUVz1(gtt>Urk<ZGq%qf14txOAHVL9j6S_<;|liyw3b<0~%N~&<iSl zzYTvoP?to+y_YndzLBCCyKY%DZ<ZRMuMvGu@$t~hTe8>zdim*EaHyOLuAEWvt-6h< zVrXNpx8ATHB2&<F`&V^6`+Pv$gYuNtzW^T^X3m#_jo@T42Wuh?73=)awBTQC3-hqZ zQQ5(6q?f;qJzIy{2yZMZc8AS{a7WJz!4^M2Dakva)GcM}_r8pOUw@s^s&%Vivd&wj zz}ZF=$>!3OJuR=(X8g)YcdROk4essjc!Av$Z5)jIOp##}%OoFhZEv8Prsvn>VKQc3 zw~$e_SS)-d*vDgQY&w`&xxhn5e!lM;5<#E5%wqFD%*drp@_2>=vtxbB)G6CPo^d7j z^?4rwe|9cReuYv_N?%)M{C!!A4O5*a(8$)t#(P^YKKm)|9$=n8({gd&_z~|V_w;k@ z-oG%==i5e2-#*J4?gP=s=)I$ooY5<{^uvp4%E_Uj86qQ25PdPn1{}glNwRprna+FP z=e#}E#TbQBdr`Q_!{j_)<9gcoLdN(NQzqNpI0X6`gc-IFBgdU<DPhnEWeb9diAg<S zEk*JtD8-5q5PYU^qWu;Z(0>S&J}jj!@rP3#a;s^jgG2}Y)ADPk@kKJ?$qX_cS_z3W z=4mTDx7lgXC1TWHRU{t><wVT9R?2_eI`GjD%kS$`QYx|SO_EUyCOfVyQPbmNe7YYd zfq(9U=RXE6Z|Sk2sVG|H0*Z%v_hGh&-&;2Zl~+~XDQ~PAKY2MgR&bMj|8|srJUjpS zDqRhDyczhK6GH0RA2@k<LyQhPfUZa~(Q5)W1bS-4&dqI1D(vb@jG8l4fMsq>Ubn2L zXvb#~-SGT-+Dm16IPbTBg8C-wo3|k2cK~8aC3Lx6^rKXdN~+0ac<wFF`XdcG8|(px zmq}@iD3UG(@JVl4&@=fp=BpP1sGMSVlr;pmhN(P&|AsV9z598nbg34kFNxiuPBM!y zJ0xa9jFD*5>4)IvaxYy3*7QQwMt?<VjmCOB{N&5Ksi)pn8ms4sK7fn+!U^`PU^a-7 zmKNQh$V0S0JD6?#o()7x#?AF+>@mG}lOexxr=~8i8plHq0j`t9e?d_z;>kYGE~MR> z&q?<i@9j4%AK_o}_>CJv!2{kje{r5?(E{sUfnN8uPLy+)#Wz9hgeW}T-(JPnx{|DY zc@qfrv^eR{B&pJ_9+vCi&DX{4s(xvCY@$ICePF)?YjnGxQ%dm>#p32Exiublq<cn3 z@iVVq<T;N@4-&kq$S*24ag^%`8aI`qm{u1dxX7Xa6>pO@hg$EqAZqZLlY<89c1D|~ zOox#-MpU~62%CBO<E<VAu|lThJE0R)PI}NGflfR-%Fv}qS67ttn=JV=DWIT65GS7p zAax<=m`IyuC8u$XqarV3Wr>Q#wha7P+E8Xxok6g={b*8+i9(5Op=aVGJRyL^-Y%sU z&!!ZyT_{8SV8M}N47)j{&vaUq&0G=NbE4e5yRv4>&Jql!GadY?9QtaH>V76Qb14z1 zgo#r_PD1+3#@wk}DKfIFd!Lj>ocbA)V=6~UR2G9=to<kof&s_r(N(|M*bE5}lnxkq zFTq+;(>8LH^tQ{C5Oq<$sK-N7k4G8B+d5w?wCk9gbVly&3N><^@ocClZA1BmL>!_K zCfKU@nwhlGR%$@T{qqx<IR)Ki4Wb%B)#c>^B=*fR{r;~lcjeVDiGXXMQ>Srq!=h6L ziK(pvk=rifK1mH^Lji$6$oDVFJE{cJ{uE`R>5-N{`h~O6Tmws}$p(014+^^4Sy3^u zMo*!H7Dr$ccdGDOt`Sl)p`)wWgMI8{Lu3~S4m(PkU%;d|TNT~)DUuUsR8dwMtK(-z zNnUO{qg}FVI5l4>4=YKv_@4w8uLOFlb(FJz3b_qOq_h-ZM-VXC+qP2KcU8%DkRXu! znwFgD5?kdXn(HZwHHyClf*zAI`-$cYhi6`<)TX9|bs`L#$%&JPCF_o;JRZ33Gm%rU zO~FDo1KEH<7Gw{SaA!a_#(B`bOn-I|p)75o!{w%QxP|)zozqj4*>|Gj|08g_bXo8k zT2G!_JZBveBOz)nmh7ofH(5n@+I{EbN|=qfFVuXV{X)A*GpqU5JX>>Jg!vq&;dn^7 zif6*YOim?v)cmST3b|v33-P{4`Pd&;KkFii+Z26jx9ul0%U{@&iKSjn#wdPmPuMY$ zNf*}ety9Dn@={_|%x6v4Lq0ltuW_lM7xVkOs3UVBp`LpG@c{q(FI^zrSFL?<M#yj6 zJL>TpWIDrE_DJC_F+0j~fHUww1}h!?L=;PAcb$NPC@B5>)_X=Ee9PFnwxmUc+cD2( zB(0aZe}q<@PI9j>?|5N+B3zS^9%x%|6%j%X1z|C(Q|6%@VnLSJyuzH$KzaL7T63|? zT7C*@us@Vrs8_5~#_I2LDPGm{4Q`k{nz@{jRV5_2|8dQfgCJlq&4|PN_yss%TUXyR ziB4Z8q786%JJ(=M373{Z{`Kl_k?*7IE*DKQ?COwXPLxzU)Jn@kX<4_nKFXJvcLYBm z``*(8x;Dx<$A5>u;tc(k*BRyzZjw!(n<ns|;-(4<Cw3$yCZ@Kuv`j?OWFC9Dhq+I^ z<PSIMGHejhpd7Xn&H6$_eL+n3f;R;GyK(@eTEDco&b1NJ+`_{=#%M_sSKb4ZkQsQS zk!E}7<=n|J0aXM5GSa`YbJn0>?`Dqao?o7t%m}#V&!xH|%@1Xez!N4Fd-8Z;p8IKL z)a7dNMr=%Xk*A7la=dr3{=ytL>4d&Y)ZLqG79*t}TbaY0pzwG;lmNrc(Jm1$eQ>%# zrl4OoT{3GpBSU3q7?U1-@F!x@w}MHOUep#LcYMvNdpn|Cltc?;dPX6E87XjLJe6i* zabPAv3A!_M<b@faGmp)1-s%+bOk!g#A;|Hq`r{IIDa&gbEt*cF2aZ%(w~`HQ5EhvY z_|Kd(n_ZJ|;~JPu|2ZQA$L$wPp5xy`rhEE!6w><Fckl*2)dCEdQ&md^5~P;F!NEK! zpk?|=%3Qk^5FD?dmL@;?M9`M+iYOeY^5Y;xOUY8yh{oZt2!<O1I9yxF4Sc}jJ6%hr ztkcc*uM4KPzSZ<m9XG9RoxJSi<Gqu5%IYao5IclEy21ihx}_$ZbwZ_*0ASU0N8j5{ zr2l{?pZ=Vo0Be;hri5ulRM*thjQ<?sbCDHr9G4T3iz}oqy%Kw}G&v^@qK1^zMfT%) zRd_%D2p^wgnz0`!v()Z1aFh=+q~usvGlzox>aZlP0f#qw+i)|{ar(+kTWH8u$pQWC zy~KOrffy(T3-<vKDJYURH^NAhFOU%#*_I6|yR2w3oy@j{oH%qtm3VTFLz5{{T8%h7 zwo}M4_~8)6%z#8~#Gq}{!Vg~v9Nn*CPe9^Br1{unsm4p(Xj0)!AI9hLTeWH(vE&9Q z=puca+|58LRcc98A!8xqE*UA-Ye;er;=Ed&DMOFD?7uKA6f&AGhi6C506=m4#jt?J znFpu8^{~SD?AX|aOHz5QZ*d4hQIhU>ms{?<9UEf_!JY6OWS3*Hskd+6w$~B#GBYqR z7|m!R30dSK>!Vr;j*lf&blKwLqi(0O%K+?`#Cx7MJF78k@&YKVa~naU58#xmgDa%{ z;$#V!?8x(n)){rq2IoES^G@QHjSrT8J{5@RHdIuyv!T^snT=w;@!t6v(2gOKfLaxq z6%_B8(Iiw`#5~pg;$X#8yI%RIXcLA_EPPz!w!kGSfI59`v-_cpWm&^DHD;iV)ybNl zkPBTZ5<sWfDJNkgM=I)iAc1%`tz}C<vxUn_|7LMJiqnBNA3Lvhr?ybL9W&xB3BB?V z6}>01e|Tmj6M~{u%IK4awYDm@7Mk2$t^xe8R<4;<{Ma^(qhg6o##?~CXFj3$#rc!r z1@OzSL$L!D4MYKDYZ&Z?qBeUDar2TRUb9X<OYAViP0qOt4e!E3rtjQVWchx9AOkBv znDmoK<ReJG)K+`>cr8AjDNY`TA)&4E9yblvT1+gpqtN|HMP6H@IUR6Ys1q}Q!Fdw; z*B^xah+qBPSJpomLQ}hoi=$){93)02o0UaKE_rg|b~#IZ;qZ8>K4xuMs^ZPeQ{lKi zx>9W06u9xJlX@IIWYARt&E%L!f1Ap@Bb5E>e%@VKwenSsH#0#<J_^vJPvtMHJg;rQ zx2HCmm4mox9B?@GgkdeSf0vERS6l!w>{{B(g75Rj{}w31+V|u=pn*h6OE7*uLvO#_ zj@Wf;R*GT;S;wdBOD1=v2cFdb_T3Ph-@}vUjb;z~Wnl*XV8@I&?JDzGF?wI|jq)j< z*S+0KeSLfnjWY1X<kFDekAggQqi*b&Wt{=X*RnC|?h~;@(!A%3VP>CQLcDBK*4;kc z#@bO|HcM8VmXm%uSQXfgEe<T-FNSVlVq&7BrfFdD3t9ZhTF_j0-a<8P-(sDp3nC`# z2wo#PBGdA^^|d5Rn@9D(WwEF7C5p|qra65GE>aIPIY`o&m$T*Cw<F#CO`yc}B`OPR zyhU>Xapp71V0g-A<LU{tzCLaQhi7FoM}eX_+;NJs=hmS9NR;+jau<>ukXJO!t;w-y zf?t~uPp0HzIVRX|rovEKjWW=`*b4vc6?|n(Dh>;XhEIEF*A_7-AKpkV7f-hA!N{;} zq@s_!a50X(cglIpT*-GKTMZ`LZ6DJte`$F(?Skp6EWU5K^|ZZ`yN?A7YceW*F)ldK zh~0xrkhz(3A9{X3Zbgb>i#{qhFPud;8}l`_iXWz%TZ6-=hYP?L()?3s2m!a=IG+2t zKadQXy&So)Fe}KTeT6T)zyKj8Z1J<~5*)bETrDAztvbBqomhFc8(bV>O+5Wt{*1Q_ z(6?V@$vP%oLm&tMhP~buSQRIq$h)0TT*%YogyXLw@`72l+XezCd&_b4cv_}+m0r?) zyv&+%FZ1BGig0sG)-b&+FMv-84ls1*EjT{>xZB`Yb1NExtV<xPi;U>z9Ah3H3jy&k zS0ln@l%fqh$s!~D&rj;kftlsCzFWMZi+|P8;(<YIh^p}etc0GG<mIvCV%MrQH4%Is zx069w>f60yKV922)}aAPtBB$*<ndK^R|rD$w}$BIqqDA`moyIzy@#i*j8Z*yRx6eR zatsB=CxZ!ga}_cQI=%1qV&j?0Ai;ivxa*v*1KHE3>gnFyhAbbQblY+z_(he+PgcM( zXP)yux_$>TDe;K}ji1B?f>FBpp>*V8>DHwef_z_gcJ?UVY}PhM6m>e^-V&wnQVO{c zQcDUAc?}EFj#lp4t?8kZDfdg4!OP0CIsJFSl`haE?l@EYxAd0|1YCV5fkrtgsJepb zF~%=Nv8PdLg4=S}md{4HcJ>k&s16Q=KUK9wHPjd3Mpey9n=N!MPbJ_#cfJPiu+L{n z6P3dwa|wKkL(OxHBk4uhSOMrCTYPqCc;2Sw-x_~|S9oc?ARuel*0*>#PTilYa)Ipj z0U|azUNjdTCvPvIIy%>#WWK6KxxROxj@}_37<|hhIbw2fXgen?bAK>#043+M-1Se= zJA4<fR&Be=(`;T1CL^UGeW2u$WN3LdmkW)W>K%8av>1sc^3M+Y*vzfxog5Fh2Bl8z z6tzt0d6v(myxHe7%MqEftVi|3T#(ETu!0zEaR@eBV-8TLKN5)?cN_8l_Y(<;u?F(S zh);<fasNE30((((1-3n!GgDQb=FcM4zIw0mQV`>(K2w8b0E2EaY+%P}OhE5%$m2o+ z*@-|2VCRQmt*I|X%?&Wpuxxt6SY1^|uzWq{bZpJ6m6EJKid;k4z5e-W8kqRkc>>)x zcWTvmF0>JeK9b?OYdJZ}mou)$8+}nsVnh`6wRRrUj_5a6WAl9Yo2Aq#N3uB!T_8RT z3=A9x#Z7@wp}qHiFdN!c03@A*o@Or&kSgwo=0T=`(Jrhkjw_|hpMg%?lXh=iVKJfR zU7mIs=<BO}yn{z?LC%vjjhff;xVyyzbv{}#p$46C2@S+H-AeK{<F11m=<v|*T7W95 zu9u_{NaK?u=9pFF$f~$oed&Qw7zTQQ3uF&Qg{UZl*hChS*&3S!#!ICoBqV%V6|(qH zEi%^8b&mzK#%jWaRYEL>jXLOx60DG+Skf}z6({eR)zWt)cR(Kjkj-E%saim%+!b(U zPiRg`B-`dOAQrfoIrrXCqMS^&u7JiKvEC28qnx?ou=XEm{LHAfz0GZMN`&(Xi1o6V z4i$UvDt4v|#OQdqbtE4H9x7!U$X882?*kKn+vdSPPs+iCr7m9bkDpvx4}hM|a9-cy ziFtL8*RGRRN3qh#ClOG35=eL`kKU303tao2*j;B#8pZV4B5mZsc^g3<r&Y@&_8^Fk zmzJ-hwx&`&!YQ|}+Jh){BtiM)yIF&DPLg{BQ~eHSimj$#MtxEN(>kk6*OF2s4Bi`G zFasijAY|SoZCP=3hbe#hMvEOSA&-`YIV?3;&Pcdc?un|Zqyn)3AHn!4P_g=A;)M*~ z6h@JObWo}gUk1`azzPC)lqL^5l7O?k?%6;Xm`#w<z!C|g{02r8Sq#nFDy1Ipb!LB_ zN6>bVUpYA2y_z}qa$}OdAeM^Ub$rfsonz_MX?vns&9vlc+AiJ~!{7=eAHdGN6B*8e zYvpmbVN}upLCSfH=58PrRy_e$-`r|0WK2m_T4R}d5}Udq0_4W+7oVXYKZ(f~4Da~U zmOQwaA<-@XW<lAQGg^sjRj|;K^0+HXsML#ZZLr*^kCOaw1U!e4wQ&!VLPQ+W#vB+} zYf?_H$)lGKz@`B-JyKH}POBD6Vb79yED#y06__~*LW<ByvL8%v?K6(s(7OB6(t;Bw zY0CS#9<v=qWPLJUd8rZQEfQn_1(FmaOd*X%JMZO1dFI#o@AXvRRh<{KR9VrUzu3x2 z7Fc^6f*2K)f-KMT6y8jp><fKW{ek=O78xM_?DUq)X8GLhcx2qux_*1%NxSW_)j@N( zs>Tm&dJaDzus3V?Qo&U}?RD<s$@w)#qF76x^>8K-O=RIH(&Quayp-_;*99)0Na9-y zG%e`Ul?dvIvZ^ZQQ~Mo3Y4Y$6c!z-yg~kt6n)3Htfq)cwOh-s|ucW%J0}@zO?@))l zAi<sdOF+-urk-%werzRpJJD`c$eqUL1m^5fz-p$_@;+Uwxyp_$*v!c)M3KfC829rH zV(KMr+P$*Pl?IP7hJqWiMiY|~R{IDGN9^hjhBDdcXZ-#_KpTdbb_{0V7al_bMb3YP zK9@s6dVGC-S69FIi6|Tl>IRQ7|I{&>D+E;^h%vf)9iuTKa5B5Az<oX`6K`?-u}B9< z%8!lZu5bU%uro9y;{`c7{ghP&HWBlg!a%xZZuCgo@hGZr)VmmqjUUx1tjP)<0Vxcx zfC!)bFA;dZ0k+6f|4->89TjtbL-4cQCz8c<6&Y!cfA%d`RJkKIF&K%AO@gD&y8J|q zlY9uM!zoxN<S@Cw1Wej#Huuu$?Xl~qQA+fT@JANc2ehkBw>m%&{^(tfD8^9m0EFJZ zl*@W0`XFm&va(V)Ma(Wo%oiTz=~H}m0EcUs#a&@}R_c`#Blt<RYhB0hKmuR0ped(+ zgI8=h-(e00Zx-B>{#i~OB4`LQqPX+>z{I~__PtvGpX>`)@$FWzSl&_E77#9GIYh=q z#)<wRmpyYSxD}CZ-J<*@Y)OiI_!!76DP{(o^e}fDNnCiq_Gke>eT}+*jEs`R1z+V; zb6H5eQlJYkitH$zocj3;9PGgf&=urFGxzU{`({d8W9s@vNJ#6v$qN;{5{v6NkD!UY zAeoEaQ|>=t4By7_7bJtk*aHD<9l(kA%H=m!+S|(X$j021;&*|ki`PBR>oFm89Cc<5 z^r6eB2&jirezretCn%etVHo$uZ)Tb_ly-Z1flhGU{ofd3*VX2#&Y?e1lO-uH2_lp6 zA8#yJe4Y4eCtxbKp2@;%zq@ss)N}M@*X?B%t>C`}S~2Q!ynDR0VYKmc2Jm=U(+r8< ze2BHc3hm(o-T|4|M{kA`-H(w6^yuuJDll29HG<t535WC7+Z0+t=AQ>isxW9YDK(;N z^x$IE(GNff993{)mkK+vzs6`TX?n3yftN4y;0UJWT9*C5g`}jUG>;Cow6wIAckhl) zJqr}+$OAG2i!(zMc`=)WK9Hd0XaaJTD%VXwdbReZG^<qQJU5v+HbdxIt>=wzVDg|G zI;$ngEh03~CZ*!l%`WiJUxYB&r$DTUGI}8|P`R*&zXpL@jl5a^5<Gpy69=Tnq6_v4 zdkk32<0VZokM`u^R=aCK6i21;j6|ocVTq8yix7lwodM;9lDR+Gup5Uk(TNWx1D&1! z>(^Awe)u)C{W58WK%3yGKbYvR@@FRZLUPkp%hT)<EN~hd9o(OdcAV;(g{mLubaQeX z^XCcdcyFU1$N)h07R~kda3+i8n2}Vq@tYaZc_l>>TMQ6>{N%$BsJDP}^h$(vlj#(t zN~*IgqiGwuS6C=ZN8c6mwHFyFi3d!&SB=k-+%NhVZweqrUS7B$^EJ(U?UcPrK^3O_ zEM>q+f?5*q-xR4bJ6QSZW*;~HgP&wClhFTy<g_#RYM*tcRgZHKv+H9rd%xN!keB`} zYc1IfayzCekA%H0r|i1-KuNKyOzDlw?pUYyT<j)~a3DGw`)x$D$+H13aEr&*o1i9u zU}||xAEJef2G&e&ZWKgLsPmQ?@hJr}J?3%7a`<pI{V$Ob2U^Q<60U4V9yebVHu&vc z9`z3h0Loz{A%<e{u^1J&ef##R#otI49Q1e%3EU2;ERsKipP2>}<EG>BqF4|UfD>Pa z!>y0>ItWz3*R4gd`b6Tp$nqy9@Pm*b@89IE5xRjVVcnAE$#xDp#Peo!Q|M;@O~E3I zWSekh?h9;?Mf2xumq@;rMB$BBFV=c2wiq5joOUu&$VI4#76suST%!)$Lgm?e5by0C zCXKmDQL2IjvI)Fd!wX%eRH>>L#(d|MT0%f-8KF>qFUGDCp}zG*WGp6#AD%dgNH__d zQ_YM7N{jT}Pea{lHKl#jD7ma9(}3WO9L0c?gEcWKkAV;3mCWa_gUPBSS<Y2JgUpM# zZ~BFl1T9*>-9~kgwf*QOB<`gA>T$Tjp#1ZD`42tOG?y8|;HqUkwF`f}Z2|Fn&?ply zo4P}kj?6RQh`?wat#6G;mm+ho`h5bX#U5@;u8u?w3=M<dD#=6?oXchQVQMZ9=NzAP z36lkSdKSywIoxq)boDoXZq)i!<Cy*V-oVe8M%aT>!26^&FA;L6RBM6v8R?V%&>*YP zc;o>--oZn!OBV_9Ex~?V)|2`IF1-qnFr^3ncx>DgM=tmDrBF(j(Ul6rc}^iUq~DAt zd|t~39;98<ADrvz+5p;TV;oPBED=YQqGTDBhroTA+xyLI5O^=9gwWtWnvG~!DHuMd z`=>Bza=MxTe_)1&i|Um%UT&LV**;<y1qYD*2193PdhuOy>O><{fyEh-ZlE=5yy|QX zuHFeZEz`kT{^@vB6?^&q(`MF*S!L*Kkc27Dt1-gQBVFx%iqCT+Z|DvUPbTKCgdG8# zC&lyXj{)9dj{`iQF<?%*vwVeh$=6%ftgv4h6h8THSzDsgw{QpIIbK7{G>_tEZw~r} zo&-`xqabrKOVtD^nH}dp15RrX8ZaZ?GBl@W?fxiN?&QUa+QhbiszfK+tcjcsF7sBI z`|enm-6&Rb5n#`D&htOd$LGtoV6u_C4mKf*53Ho4Ms)QziN^-=;Gi4g=jvNPlmGy@ z>-*8gjW5)9W!Jfge1WZ}I)NFvdFmXvogFWbKzHI@Iy8WfVwB9kyP<=|J(f`?D%5F- zQ;FRu{|t~@2nG}5TJ>Lg>zsf7GS%BVz_$*WtKHb)256Uj&}koxPu?7BdY+fcjSY{l z9|lq5P)&+aG>~+FVwQTsW0b>Bo$iucK!QB1c001@!oob(3&c(K7SW}yuj;dbKF9_V zRPaHdQA%IfD3*+hVlgd0nWjr?bT2(XNPSL}xt3*l!8Dz8%QKg`fLYO{n?t<Um!C&V zC-s1AHw4#sVS<BUA1b^!&Pf54{`G{cnailLi1d8Z?!|%RVxxQ)NSWLYb%hK0U9GLj z(&}o}!95XxCV1UysxNjg^}+M2e$Z%R>`b74Hm>b_NR&zpiEeD25E!l$ZiZwIxel=g zbis?5=%<+u;EoOb6_wDeV4><94o@fDn}qMOzV-<nM*lPl(n+eyR;?HE7`3jj)(eUA zI-ZbF6bjV|=ndJgJk2(AnwOAOEjHCR1>-(!D8C4?fpWHj5|7g_LFP)rq^EU-sB&Lu z(N$W1MoN6wv6V2l?Hpxi$L1$BI%`!kviss%&DW~BhVgoUW~nm|W7s*?x<w)fd-7Dc z{y?>T%tGl9r4Mc!$saP)d@<7>k+L&dd6`;#+f-{SsIMwo_fuhvlY^jFgPV?!kf-Ek zogKs|XrIi+t`|r;jM7F2V$}vNJXYTNIQ$4zl8g085QjBjCfP_W!Nn7ZZH-fI&3h7A zXeCl<*aU~G|522=?drB6io+NZ==O|^jZa0|TdEXvE11*iHFz=|vkeh-U$iOjr!|N2 zAw2+U%dy3pg}5V+SeUjGsrH$YMdps^lV$y&tl_cG*2}3Y<Fs6Xb~WG~5}&aH#Q1kX zNzKJNgP@Nx8^tUDWINLU81=Ar9vkGxfrgK=3jy}rQ-p12U8;-V-D;!Rg_s@>H!9ET z_ViIyQV-}pc19O5Kn;od#zp9N{(poyA54Mv79DS2B23x$r)ZuB4Ul&JwVc`F^e|`1 zedfF-&wPll$&)yU&)urCjsyM1$Bz6baTXHsNj|WMP3r&^F++mW;eQdR4T+CMu^J)D z7NA8)=$MK|v@$Q01LaJO^eH)8**D-V**m0?fVS-7n86xVIdWL}TV#P=SW(my7e@DB z2j1-Q8vCy5kur^-ubGYNN0q^ggLNV2UQm;pJt}M{x!g&0g#t5UWADtqxxT!T%cwm2 zCD9n?Byp<-ldcu9%F;V{pU_;A{fp+u7EK`D7YLq1N9w_ou)04}`Lo40t`NAvU#jW= zU8@hg(Q^~H2`Ud1XEo+l5pbi$QUi@Q`(|~H4{t+f$&PvA`HDBTQ7T`S)f239!-R~A zUgS~hj)8M!{W?r%(KnD7aY6Y;*0Aroyh-<iEE4Xn`Rcgka4nVVoLMs04?gVdRh6(b zTfQrEhkEMt(@1{5#t4ty7;D~Oa%C6*s{uiZ@AjdM_sVCDnEM`sFC*>2TcRdpIOAs~ z@0_xYWPmW%8yOs&ZB0e-t?5u~v=c3!j3BJLeM1p8NG1n>^VMpnqN-}3q@?5tl_36m zyGMXyoDPG#t$Rvm^J2eTk`4od5t^d}%;^oMV98P_N>Ujx-&%jjh;;C|pcXyV(4X_U znFd%RmoW)V%I>E!YUMYyfTU)5V=EcO_#$HUE7G-+G%9$kRzizE$;OliO`3OpGATeD zZo^Vt7i!9hkGgg{`cb?!j)%iLnERHi6M%FysJg5SlzQ*j{8RowRWPT-GTTsB$ZMGr z)r@%=uVOJF@hgd51?&<$kfxmF836K0g9>Ge?K>-6yFs5#P}|VtkvLCc<#_K5UTWDb zP*H!Jk{~{*2UK+0x%V+CkBp;iWuPe$jJ!T-@EKV?R{%7IC@VKQhW`gacZfaLIRy#* zcH{SfEP2+?6=A_Xn1UD&O=&OvHmUU=K3FQoMO}AI^TbkKy^nzZk!VLl4Z80{`}y-g z%7wvy$9h=KvMec;5473!EaQ=yMaFvs@^J8|3uKR1cTJ{rcvtmCWI*|n*^XctgsNL* ztEUem*iDvoGcTK71rmrS92&qwns}g%%ToK4*j#tOBYj~FGNjZ3{y)N~RzVsi6skd8 zoJLWXUm@V`0fT!I@!8KxDJp%3N191;s{EOj8eAxe%JuNP2r)8+S_O|rK@eYyuuxoU zad&ItnXy`s8Qd)P2W4HJ{B#GmyQt7Te1MWu@OV$OsYAFmitp)N`~>U#_J%yDjQo~t z@WF3o{fs>_Jq$Rn_`CpNNwAFqIv!1~8Hj}hnk-}y9s@>R^p&T@^=;T54m51)<LKPn zT=T{}$iy9>ua7o%O!)-V-*BXxtdlhY>MwVuRl8{@qjZvSr!9b%vp&COGLg<-?%``V z0jlaip>iHgBMB%@j*$j%#PI(hiP)r{Ub|q*PJE>3s1U!r%L0IiFBMXGluWl8T?M4e z_Sn5ZEa_ik5W8-pbqlm(XISo%0BGC&Wo1kXdPjum*$&rs05a_TH@?xIZvuR_SE%dL zC4nKK^!`z&Iy)LclnLan&EssX^%|yXtq-A<BJ{q#d?0Z%&gpV%VWBdY03SUyG<3SG zwDb?ppr2^g2arDwU^#O~!s##5Nq?|GZ^QycMs6L+2l-NV)~Ie4aU0;<MIUU-cP{tW z2ZF1sAAry{C${+Q5e}Mh=o$ga{j01KgKP#JX%Pd3Cj;n%4r|Bz;RDoCdR5`^aPqw{ zEmupR6%q4g&j3-9<_v+kyPM_yJ{0%z*=IJi`0qiMwRD}CkT7o%7^Q%ADf9g*zljU6 zwoS_bp#ePAd%%PJenzBi7yq6vpa-<0jr2?IQ9aVqP&X}G*&zBhRoF`(gj)3&D1>E% z>u73Brtg!i!)7a0qX81xgKFtbSqhd6sxtu6My?tpCH9qpTw(VF{t@y}EkBP8e{Jl| z=>4u)UGx88b{>M#F!zqzWsA>$%#NL)e3f@2fL|i@OY?hO{n!wJBd{L;l-Q?^cWevE zQneBr@g3InM4aq1%^RSgK_(+UM+{~?dSswot|yM|y(Bj;IL9K>zDp?_2?{+N!|i>- zFW@yX-6+6cl2DrwYyN{#Dk3L(lW47`!E+T=jI7DIP-*q}Hc-jR`u`9_!#->PM$~<z zy=-wwR?Lnae+vxqkn5FzM-aC*(Mi!N0n&QI=ssE$DUuJm{!G^1I~tr>D$=&{MvqzP z?4#x0lnP0eWNV)QUq4FoA#a`8#9Mv-=$%sPwuLo-e6@~CB1Ptm?$r?iLV^CZe;F@! zLW@4B+Xt$KYsa=+5W-_S9s@Vm9Kaw0Lqn_kF0t}w3<Xt?89Snt?K}atW>BPgcU4`q zmz(KJMh$`>UZe+S*#t5&L74zGxIjXS`gU-6J~V3|W2Ae*)!6{xrwy{;NYr846W6j< zP`Y*=bUJ9>Xsp(~wNi*GYf-%Zp<3XYSgW3y3kg<M#At%5msC)!+~qoZI$oZH!S2D; z(FD~*h#)}vk1uT3e;RdpXbE@T5;8kwrRt2hKu=76-4;YK0*W%`p(R+H=RHvW-@0O1 zPaK?@*ip`_-=U*V#B<KTLBSGEbFDkE6IsAqpzh2pm~5<l_f2igyr0tzRQlJxXUOYa zaX%p=1O{gwY`r<(>+Cflx4TsmDF0<wT}vJUN#%-#-b8=NP+v&U8(mohCqeuX8{V=W zMwhVj+;7iJ4QB!<E$9E~OSnL)H9dFN*}Y}94>I>PKJY;sqb9gGdUIX?SD-`0&oSj3 z2g)g`dGyQhZS~)^0L{ShhVyvr%KdVB|6v?XxdHDxF(Ij@<0>{NB!u|~I1oHW2@-e$ z%<Z(~MK3S9E=a)J)5gzeyu4IMx>&x6V-i`;=fpYLl_}M!q<U!=I}!T)?W%&zJZ%C8 zf`MAFihAgNf@}B$E)_DJK13Ge3j^S-WVF{-(@f)dL3&!yd;qRW$P51H9eF68*0(R1 zx^P-78C2K?1n4OKY+0&^XbR}IIlnveso7%N=@J%$6U7+iX0<k0$NRB;tP3A+5;Kc; z+#U^b`DuFEHJQsm>6+eV50X1U05kjVDjK3VS4V+@ao;;C`^C7CckyOB7kI$CxDAFN zUf?gh^LkiZAQeEwJ((;*#3eOE479$b%Bq8EHc*qQ_VYEVkeIjILDKuAZI_TaLE4x) z<(R%>54M5IYf96A;p$jf;;2QFeGSNrJhp7Y&3i_9W`4qZz1D8RFT~?tXF59(SAr}K zx>Osl;X+b@?;Rbls!^qS0CmrT+vh0+kQTq>?h2n9Fb2Z=c~m&XP|%RU*fa23anqg3 zvD&ZIs62Ud?Z4oQRTDyvgt)-^e2};>zxWC`+Ix6Ez@8{1#XOHhx@x?1fMA}yWe92_ zH;b)iXK#A$q$ns|lNlP@mL0_FQPu=REM{k?_?J5VOw|0V&JNsUhz*6J+iE#9r3oS! z9~yoB^wQ+`D|{SW3BHEwbq1qmte(WZ>(jL<;(yS<Kdvk0?N+em>uozT7x`Z??V38U zH>I+<J131Tu5%}60Jmnkm;653Z~K8N9IjVcMMabP*dSVw>S6Ng!lAi}fsQ-$=AmBs z!6-^gxfl0|n^{O<sGv(jwcfesL31HZTMM>L;7uwqhC2?7oIA}Ru;}^${c{(Cp5B_| z+|h5WEwm#K=spI-r*$Xq4*x#PzZKLNM?`#0ao?$JjQnx=i|oJPooGJ?VELN4`HjWs zt>URDRazZ7A%Er|1OqaCpa|msvG(oZQ09HyD1~z9T!f|`=^!ehabBKkOCghxW2uyJ zD1%9ZQ8~12Xl0QcvK7fWJA@deP+^pDXbvWujML00nZX#|@7?acr<rSC?{)1z&mZ^h zbM5V(-{JfHoW5%2+^$nc@B_ybDa(MLGJ`TgaHl;al&kXfZzRC9XBOe~S5*7vbv--X zW6-j0Ng|$d)?J-kgGCtbB<cPOo^tIxF1I+c4edZO!q-P0=IE*aPs(J+j=unAw$5&z zc;cnFVr8!Y+F|zWbsb6Pj>{ETym2z6IA~bdCsx!$u0<1sP7?|_k*?*lb+3}gaBmEJ zs1dpqd=)DtcRX7awMWgWnW@|0FBf)$nLGiz)+R1nhz-E%K`waq-EuW3ns-cz*lm%Z zxcJ_m>}!~RgY_89AfBWZm7Zd^*cBU8X|NP}jm*;PT7sLeG{(7Z^}rlp-+g9~iE}MH z8)LoH(H&hsRMS5jO(c9bupLv2w5_d#!a+AG9XPj3-+1!RKd?O`zZ{dRB2uaw$W_Lw z<&<=vYLZh&*yZ8?<bDR-fld%<O_uw{zY$Nr|Bv3P_%~QCj;0|5@kagWn2gP$Ti6$5 zn`jk~4Quw>*Ms@m^vtaD$yYK(sOa_8qQ*wIK6cJVyO7cy@=+B$#UI++U5sZsG*L^{ z8o6BVu+O7z>RJU<Hk^hv+zONNB%<jw$zg?MK)1m|wIF}atGYQK&4|q|bn8&~OYr|& zvyxgE%oBel@wM_4VZYb(B^OE%Qd-)1!7qAVuCyitp^Q(qt(uLO^&@`!<L$fBQJ~p6 z7M%rm6uIerakWjWrha0~frf~jMYN+GVO5G7D^r+gocnDIsy_tAo$*g9_X#+1kk<j| zkY-uW3;<hxzD;QX&ED9+V#u3o&Gijz148fb{YrgOlP!Kkia)7;i1=rZHJ#TN-`!68 z;Qhaw)t03!vbO<uCR-m|P~Yli(-8;6z$bx)@%WEwmPm|3QB@9{KI%3UO;?{$22B`S zy(g7hMex*(o*90QZ3W?0#6ahW97?&f*9|gNPWTHlrrT^pHZ>?HH}hG<Bka#rFuKRp zwe|Ju*Xs`TeR@9}q$%-$`9F`!`ZLI(KdS8~VZwB(1DNfCj>zqdO^2`w^_L8|k%|L$ z?uJy;5SIJ4Z|S=gMmV7~B)MYUVa3F*aIY#$e>Lzn#@1w}rGDFzU#k@#`+{uEngZq4 zjp|M8YLp!yL2Qr&m1Gj3^8Z`SXx;QlWu8Qe$|MYWIQ{GY)e$0p(?=PF2UmZxAQNiB zCg>K0y+<t;?yV4VdO(^v<3<asI(?th3n>VdVQ{=tOoPUoEY8h-8z2+LgesAWLeHRY z0(vI)b*aCfUA}I!O_BGO==H`MG}SeY`j$?QoEUaHgjD*<okw+NGK89n{ndx??1T{O zV#T?X@D&l19Aj1eIRqa;WHpGSJLm~l0@)+>-cNmD{X219a>PsSj}CJ12l+4lkS_Hl zO~p&8Y;*Kp5|9-;`Y!i;Dw*R3-ffqyvalL7cuMka1K%kvW%RM@srS{vCVGNw*NxHP zNbJV+k@)s90T5v!=+#z{!0#TH{*2!5&xxrYgIad4EB?FlmbRbFM>sF$0QC2oGv;e! z>;rV%Sj4AlR@}I3w&_R`k2uQ8ER*j4tElMtQzksHMVlwSrzJeOkdJ7fJ1^3fiml0d z$7Rj3x2{<qym!G*uJmQqgKM0maY1*QLS5*BW`pOP<=1iur_@8NOGDcb-N-H>1i$%p zMfv;Af)7<KIbN2;-kS$_2_$iK9ymA&*3|U_`SPeZ;Wgf;xxPeAor^zl-&?chRtk3^ zt(coZwFD+htaP^gSO{{=$CCaOX<4w}pLDfG^>0PhJ3K*%f^j_WEx0V?NL_^mEK!AU z^gVV6?kH(0BI#SLb>#zMnf1y((XgK6xx7>_(;}qzA=shq2)UrKb)_o)uQF*lDE=~s zzcM-1?l2BYaf6_t31Hg-v$*0W4!JwLlKcM-ZiCHm!4W=p$(w_8ar7DvEQoed?ltk7 z9iv)g(y%&@J}BR_mB$Tud81yc7%#=1Yf2i&cc&@@*NuD5@-|0Xb{m|MW(91~FZ4cm zJ8}}e+7fZg2aR@l=<3iRv(9*b;;wXP)wqFxExw<^T$%cld);zyz3)hyr|8MvXhc%_ zn>_22!=|&;%mTw1H<+0Cy+uo<J(i>^6aVZ_kHJ1k9p39N9?AUivD^chW>nf*nHv!H zhhqk&?p&{cN|660!QK0BDpU@qj_L;m9VEE5<$9`FGgF$tpEi)#*{jA9veH#5j>HGZ zp&9xQO>pNYjeL4)J-m<U&w+i++0}%iW)w+7VE7N5--$*~j1r!qCPaxt-`rRHT_PPX z0NK1ph?(L4dA5q*7Jok30zM91T+kY^tsr#;r*&(|hPYR`#yu&)WcO3$Wl{cVc@+zW zlSvO5e;1@qsD8L~&z2>WYQeNMX?n&H;%t)Cxg+(4M#RT_2lo5Azyy`ANza98M=hK^ zgZS>`W9^3B(p3=$OgxL<1{3BQ<WPxI*~-*~w?IsOoGvt)8({5$RaW(Ne&C$cl?2!u zLE=9WL;(D?H!|ZICLoF<!xUWOmjr1lf20}2NDY%&jGLj+5Rb_rq+ZCEzo)y*s`w6> z=L%al^I-9KkZv?Xxj9#cpC1y84g4vf8`wT!_aWc9&FAjYAhNDX^IKcy`JoB4$vsM+ zLv?^aygw9YmQ<d;K~C7IZya&p@#71H<VCj{7Ge6xNh<vrmhiFyrcKF1#i%-vhrE;e z+NwEsZwg|lxYSU#TSwYhpg&l$3maQlM3#NMp_8`OxTdD2QBHB(e%<KxS@AJVxIQ*x z8e81-;?v1F&K5-r%BtTv#d#0Zz5=b&^P!TM>U-pc2v<eT$d|-Yi@&|wKIq^JE)Da= zC(vK$ll6sjs2qUg43PDl(~+7UT9aM$;vM{V&khXSa1X3u<;tX?f~R~eW}m;wHyPMD zycmz=L-`NMKf0i8OzR5uDZWF56c_=cYn^$3dYtiWFTi-q1nYmGJD+>@ANV;gui3FT zsG}VsVR|TiyF)GJrD)u6;aBARd4Hb1U2KilKV0@meuHuhiY?bOV2`!B-sg^J(Po%Z z!-y9rTDMtL2JKmN`<b=hpZiau8BQypw^1`8zsD8G&kgY3fn&7yy|9{>r4JEKbj7OF z>Q3he+)P!ClBq@vcCgz+?ms2IGkoSirV&enF#sl-*B=shEw{|NZ(KVs;fbY(r4%}_ zUy|;GZF_G|#@Z#)v26|hD!csrUfT~ipkBUTFhrS)V~Og%OEtotsaYzmVJo00oe}uM za$$$qpO>@T7Sof;?OOFWN3S2@1E!c1qCr2fZPHV<6cU&)vH3JCF?$J0K>jHy{Twep zzm`S=t3FzOe#hfKhmIcHdvVLcy-xL^a`zyzQq4-|N@A;~O!ISHLoSO<9PX|H%199s z*63k~fXd##Qs{rF3Hs&!6t@Vo&gX{Dwv3Y1y^SsMuI4iL^;U-ze1Hrv61R@<^pXb{ z5v^>xaEMMXEID{{TPI{!<R^UOy!5tiCXt*lbeZPP?EJ;^IvH<5Ev@lt03{<{ODj9w z`%e@@QM&Pa){M1`0c2a58K7Qf%(l<ud-+(vrT?R>-!(QvNa5tydMwwV$lBYmw^iwN z4pl~k9)|qO(8F@*ima{JExQJLR}|EKkB{A8!hmnyO+6nl3(D%`*jqL-6ed2gsCG<5 z9Gi;!z#RpT@~WXo2`?o6il1Q3H&un*5I^?otye*bV`ClV4L!hJ&EaQmSC7LLdf5)9 zn9^M;Dok(zv&*kB%&c*R5QZ>)s_zRw3kmbOy~IzMQwbuT-xCmr*eG@Z-y2@Y2ckHO zhpxJT6q14ap8i&HZAAQiBs>`anmw!OovVajqtMX{tRJkb={$`4e{s_7G*}b&0SoKv zg|M+^v!H?Lo(<jyC(&zq6LwTk&y7|WT-nIH15Vmtnj-exf#SE^v@i?>40gDfNE_?s z#JM;NbJDZYx#9y)(}Wt0>B&@&OL$!`H7?rRJiscmaepBO|1{8#L!#6YgaL@T;Q(7_ z=&EVxY!a`?^#CY7w^MNv{si$?ABOKfA~E^>qnq3VtKX59VCri(`Kwp_chbc_VEiG7 zk9qRlLhPFu7n=q5;G>>2s{)R0(N3H2Q~R0hLzKGUVn#%nssg2jl#rx?7-yKzE|1ZC z^ZnZ|71K0m#PcBiZte`-7L_r)-;~6sO~6dGx;4y!zjVF-c}|`3)=T)F+|=qxo;|*z zt3$KOU|_gAWL}GDp)H{61#%Mp-8J?bR{f^f_g3Y_())&!cb4=Fq;)N`T8jM<ns`ME zJI*~owA751<5-oO)}*w#bh#zU`gNm;&R88C)#392i8*cc?&W{OwaSqOke9$ddV;_d zJdmmA8-*){1WHPe7lb-NQBfzOY!{yWIpcg`@o{p^UnxOh5WVwX>bzL6g>S&M^|pv{ zP(pppkOLMBq=wu^ICRG9a?1WY`82Zxu#+QFdyt%+)Oe0bC#SMj06`C3$G4YV@J|ar zq)62<58+!QsiFej4n>q;LH_e0V9MM=o&+?l-)}(bX4w=bSk>17I*Fvj)P^47D1OjT zS2yTW8%n5pK`|h@{;wu*W0dLVg0DJaca-=Jmby|dBh&m?Qr@@hN?@?#+DfYGF9f_T z8#^brzp*h1I_0_?r3%t`cW|nD(4m-H81;1AfOl~o9G5i%Fllh@H?|E#9;dRlZpt_* zmy}mMQNc@yacEjNioq5Y)h6+@85r7U{GmxSDUZrZiIfV1e3Kr59=!3}FY|ljASfMz zwoCFk?lqFhzx5FT2t7o6Wgi!xgK}XBQfG%wd7q@KoG#*=GjRe-Q;nvSBH7hPs>*^m z7-`}Im5(39OlheX+F<VyJTC>xp_;1u_^fW_jj7bofFroBNY<18oHk+4Fl%#-qsb5& z-l|9B|D3Cs*yO9{V>od+Z5+A*hsFK*x8^emNM(aPm=4^c`d3TX1neXCezzH-hA1P8 z{+_M7_1u*6pb-Pp!4V(`-5DmCNWN5+uhdM!Zp|vif3g?Qp%tqNVNUgbr0zh?WT--I zi5q`ejAx}PjPSGR42IteB&?&rX4ieFt_OYqdc)l&hDg_s_|#~3H<2|r*s`3qVZvcN zP97sILIOI%AZXOFe)c<0JLBmyI4dMhu@M;l5bb8ZHbuOcll0^Q7*|{zCHxc6Yy;T4 zIQ`gSI*8<=iyxt)pBBGUW_7h$fHZPRNvO%L#c}DD!9}hIr9uut0CHB&7;gB(#}4J< zgk3jeSwvy?T$y|yBk3I{)^ug5U2Vb3y3^|UgsYqay}I>^fNZE6Ajb6LtkWiqYn6%s zOyydb2xibG()}5?{*CJ-6Z{5c&Wq{X*tlv|o8s*gHgc%c(9+C5QWfsCttxE|>s;ss zT^KvVf?phYsmidPMyzus<m2LtHgz7^%eZ@@qq;gWkN*WM#7@%vp;X%EwhB<K49+1o zU+VUhbPH0OJq4NUl8)~U7q;wRdW8*{&r8#48D$BQZI=+q-yI|KyLY7;@Q(PHwkFg} zP4PuwxS!`A>eBBj(X@=0-3e@1oZ<JP^!BiqdY6YuV~gh9O<kFQU4$NFe^#;I!HHep zwVEp`hyFpH$4%(@z{dvWe~dUN^|WFH-|EsdKm=ytE+kpMRqX_YdZ4R500EFZw16#1 z{w{tEZh~|5z>Bb(;wM#o!^*X4_cE&o&<%y&UCWo;L*jLhI6L9{jms0h6hPjZp_j?& z-C8qhBdVGD!zE|l%#=lRLL-n#JghQhH#f>Eu8yd@k39xq0GN(4ACOvTH9!<l(~(K@ z=}oIGl?2qTcWdu+KY$~2**9@GNBoVXu3Uzb=(4_@gs**RP{v#4cgz~~B{wU0Zo5@J zw7ttF(}$iyjFduNY`|04kw+g#N*Q#ED&Vz)%?$vq7Tn~txv9<WC?YxBFB|WUz12|d zdFk`RfvHr_@Y0wHOt4YE$NMui)D5AYx>3(g4;rljOZD`hA-d##0n35_sUE+6c9C$b zh=1tMaO{6R&Xr)Cyb!RkOWiU-)^fpqqvUl~*OAP(6_IUbhh!e19ta_uakEKJXm_%L zOF%$?;ei7Od{Qhbg#I8MWrrYwiAqGG$SZ&=tbu}y^TfH6Y}(A(PHO*ELiO*g{-HlK zp-RB=B<M;#`)2HL+<#|HQEinkVQ};m336xqh%4JQt>?z0+#;UN3(bnEhq+p(9VLW* z0Bl;QA7#2y%Kd;`nIRHrUnIYwyN#un=LQ{a<acKf2T2eNErtjpvs`26IKXiQkfeTL z0Oh6@*>yAEK>yb9)HWAYOC_G`DYJa?j^}-;kQkFc_q?7;)7X8mvp0?V`5~VxLH;F% zhXRi43H6+#1Y7%+C0k01K_Mj-@>=DOp0Pk(ME{^+LL@(Mkfy6_P*)XAWfT8UD-TJ7 z*#VRfVO9P5BXLDFlm}MtwRoIf3^a-n6^a}6mQC+)@uc_Ri_;HRxr4p+@(?-vi_7o) z#;#B<!-1q7m2n#<junVkFj4iUusFu|US1+zKwKdy6mCcQ6(w0qNFnC4T-0V-Wc@Hd zyafYRNn6lQ_k0fG2P$r*q>()zAdDz*N#Xw~Gl*dan1|i(|Dq_}^~c{@fIkSi3xBcR z0esAAiX+bZV7=j*(vmwRgg{^JUqIUwt%^LCL)-pkT@7)5^3DxodhqU#C5(VZT!#mw z!VXK3V9dlz4-i51_rzK?Bh=c7NJOUxBU8Y9qH)6YVCmETtRzt?V}-uL$?*$|Y(~ka zkBGRHxDbnkRx`uv>y4Mk?ZT^6?eV(1q$hPRp0_UOT#X*P7u_VtE)Y3nGfudRbDR=D zB}*@7IICkMoE~sT{u#nxK>WhdG(?r6&)k&PtQ>EJ7>336dBNl(AyXpoP8h-Q<&&oL z7v^a;DO%pw4b@qd)Ic)sh?&}m7rocSXR`V<2MUJ14~Vs;pJrQh(oY2qAzr~Hp5)$W z+vk}5d*g`Yst*Vf=;N_si8Rz6#V!XXm0qxtspo30rar?Rg1`sb8cSWYtR#P%_@9;r zv0|pob~W*yMy}8uX~!VY$p~Y0Ukg>wh|P;u$&1d$jNFj*V*z_~u3bO+>9vz<ZK5F@ zC-=pDn-Gt-OqZv<m7YO!WfL!lbHO`gst<JZqLXv@+2Pby5EEzC7=+dS?s{QV7Ii86 z@!R%0CaW(bbcV(s)eSd2Yf**^2Yz5t!>5NtJlrm+xr^szSvUp+s?)`)JK_g_{geu{ zy$#y{<lz(_vX0VH05}sY3mIVZ>{0BggT8Oix#_=TJ%0>;VO+LZAIn`g%omn6kU7=e zqy*{*Fyf|U9VN~p`3>U`Xi!wGmbi|CgL`3LVkOB1a27pHZ1TwTdKn68()27fe9nMX zEA=(4W9(nJn!svW+6=M(N90OZZs`1MR9Ik<l4eoS7MgVo%ncSw?zd7Q6ANn-b1M*b zZF1Px2$2+4P8|<!B6X{_Oi@pi|JK6+vu^fAMs~V48H$bHd&CGPkSw^4j3KYg=~?UU zPo;TdxO2%Z=x-1mrU`0L&cj%#po*mo31_4DA>_3h?pBwzwb2qj|7Ap|+FtbSt>@gK z6<3LzE%Ff6n}!K>T5jbQG}}XK2>+h6XI1%OR_@QdqdrbZe1H^bfH81_1?`i@v&V{R z-NB`qPZHA0c_zO?neT5fZl}00juc(|;;&pWP>I4c1s*s_)ig?6h;m0#LjyH-x5YGh zlZ?KZOZU6&tC`mo<)s0CN!a>Y@vy0=`V)wVzRdoU{bD*z>4CSQ@k*;g`KW|DCKHEY z<QN#^?!0}jx3H?)z(M_1!we!V5Dvy&uF1@{dGV=ApzN-RAwbUvBWH?gRjw-D=lHpq z1PW%?c!Zsw$VN@hPNvq;W(HrS-Y+Q_+RtEiBS0h)d~#&c#dtsg4wM+&EEs(4LOPX5 zA^=%4r`->BUzF~Dj|rX+$0~K@JRG(iGp6DRN0uLRkT&?_(T9-buKr^K4)Fr^WFg7! z%=Eac?`?F3k$$GTE7~ltaQo^ge@bx#FKM$i^p~>$C~g&My21pfujPs69SaNI@7g@G zo`9EosiCP^GV9`-9Qy~|fKh`F?0i+TIe2}S#(=zgVgp5Me_;pS#O~naUdHyQXD?r0 z5+{ko7t=pBz)1Lr|7YMVAd+?9V91R7?sIB%Yv(3%<hMJ6{8@+{Dy~-6L&sM0+CF?X zBE_Y1$@xkZn47r}(p_&(nhTf=OQ0?3W3@X9A%=K|gCW!Y!lWSJV_3r2IKw^BW<;0a z)G_2L752;5uMRTeMVj~%AE+T#$x67Fmu!L2MM|!)W$9QBA8Qqbd7T@3Ag_%TN=6#J z`|hVaLH9GoH6oWg17|l=^Wf;~nO&d254{V-1HelrFX_n}KJ7ah_GtD+y48<{H=*|+ z_~z@*&b)^=ZPQ`gU-0#h12E`X{H4U7IKsMcbPJFml(=BN5JC4}`F}JdO(VKA)oJPW zQ1^$Zm-%_^p=5Q?0l}Oo$4nQSZmPx4IouiqTEyOgBx_n`*zk;~_LVQ;QC8|wIG#qh zUqESaB8EY6U5lm)5KaSO$aQuwUke^qXLpCs8E>#NU!(`6*PIV;*4wOR_;MfbhxlXf z&(VJ5m?{`YqxR{)@>q4a9i6^y-x3+cTS|*&Ef*Dr(W1>a9%{Xl^Y!gYYxb^+wud)1 z;$E3%E<X25!3BN&H+CfJ6N*1x*=7EdVW;!8=;RH?c3o8vVC6%h5sF9x8VhI3Cau7j zu}IyC-P5B}JkC5qrf+F&h{||%T1CQ7%h7_LHUO`=WVYBi988D0wb{kbYY!wGxRyhy z8LA$jsd@hX+;FvuZqbdTv5O}7^Yx*AcBB_%*guY`qx8}fucpfkf~xhqjfriwO~a;> z>qQeLcPSBH_20G8(c^9EE3P)>)TJu)D0B4AYBt7_@!3=akg1e;HXkvj<pw(-6)3N_ zNLU&4Ly&kMwq@lyW95=Reu`1iBM&}_PiSUpud>Rh?w*~7I69TKXHz{p*eUYQ-(>*# z?TM-3#j1QAI-t=rc$#Vca6je+#hd+8=X+SGj?u&>tL6e{GS6Ujg|cAHu%4;5T^3_9 z`(YM|L>eHk59jY6d^$&63V#W)5VW>@a4$Lj6gIEx-VYp#$CrsO<%Oa5>OudYGuW+B zd%QmMW{{552?$rIZtR)%tef*p_xeDz@*1!Bbf9gd&3JX`ks=ENh-fy-d&_G#Psl=7 z4CP%TC_Sjim@!*j8ZUbv8<%Y}tV0cjTo4mZN^IqAl}!syv6%LQ_`w=o7Yk~@rLOL7 zqgl0Co-^~Ci83XhL!Axs)vK(Ky@jDg&a1ApKLD=ng!?X%s;^`B_5jEd#kaoGCuI|( zf~%nn#H!eofdkszCr+kW+cu*|u84mv9rL1~icLW~5sNdFDh!~bBiO(wmouOy^nN^k z;c0`TnKj|3&Uc`#_>FE%;E(1iz0R}E^yTwd!~j1v<Ahpps*rq5K07-Q&@2$!9AfY! zd|L5090ng?_r*PuI*(rwPE&}g2kpx?kS2Vmh<D8#dvsCf+qwaW`^K!vK4CEtIV$L> zRfI@&XN!tF5Z2psj-c_cO{|}@R5P=2f@KAt?d>=n%BVcwU3g}w+b}nEb`~?jriNO0 z!7Q^AvW@d`(Ws<th61;>)m6gZRB;Rlos3~iV<WG-SNC2X=*dN)VHnwk@y9lN1Jq6D zul>>4IO}XowZ5Ia*Fct@cJ#CU>9PoZ{ELW?1(h}-JsP`5x;@q!J-ime%?xf*?Cf=} zH?Lfc<KHF-AZn~?q^WK9aUh@2m4WlC1gGFE9#?FYP<45ImZ_5NXQ+6|p|T?TuN*n~ zPW6Y@W0PS^b+x@r_>s1gd!J=v*EOa@-@<X&D=7eZITmWtg)n(Qb>i5mPI3Xa#)hZm zNZ57%XU&<$fqI<h<=$o8U0vxbv~82u*WL*dXpu5$Ey)u?oKhzrPAsE1C3WM#C4GS- zBP01mQ}b#@rLG{BrNdhq#21U4pZ1lEc5ov>x)v%{>iqCf_L)cHK?!H@MP*7AoTta6 z3`l|`7Ptt65Tc>P;0x52%oRToUM%EmUjEiMEG^6-yveVf)zF-w`rNJ5oFt9s>VO$N zzczzr+c}U-#K#WY?Bquru%NAvpsTZMH%4VtuOrWEg?!gSjPI6^09Rk3@?*?dcC-}U zN^aMxM>~d|g-?$4{gLoEpWAYPvYviS+q#dP{%Q~Pi&6A1){}Tgiaq2zL=N}*j>3zx z!`D<1w&&M!BdvT`gaESe*_lJvNH}a@o4`VQ#<ZWFU7mvBMySp!-{GSb62@yE*jpuj zM27xe?<_U-$96q`tLvo^xzXkrY5J~b53RDQn0IWtH5E4G8B-c>UJW64aJQMvHNv?p zXHhw!vZMrl2*DkP#41f}8<Qg6lgt05Jd&7T=;gY}*>!oO?o@0;&oOm92&CthnYk+F zQO&03xeg$V^SG>p9R~-rz0_1sTrWL5jMq)cJZCGQZv_#Pbj$Kt>P$YwdS;r2J(156 z!K_K8PfgXLSQaLyHL09^Gaj@}RLiz>PFjwX?;P*s%yLwJQcbo4ee2Ij6W+Mm=?SAf z2Uwv^VHTf5Ex9EYmHb#4ejoRg9hsjZR~Kg%GZ!aqV7xDn+cCe>O*a^p@{(Ga3~>pj z*8v3xo((h)wJ8_eamq01fNgmYEJu6XH=b5iG6cK6uT-$D6na#9b*UoJb9lyDRt+ox zO>K$~a?O+P`y}|QzAvd|D0StBoFd!h*Cx?yIp=*jA#C*#UyT=I-gfa5IJdar>f`Ls z$+*dg0DBW3MQ;gPN&*_uF72Wy;c?F|GHB`k6IVLmhOh>4(u*S@8oG*eTpDS6b!WK` z<*a(n*)#*}^WJ+fL};lzk!Su;w~Fvu@tzXMNxFZEzl$ND;dpQ@8soJaMbN)&YHih< zu~=Mr6f4S+z4{KR0OV+uc9jV!9%OP!yEHmDhHLS?;90<%Bsz3|#!DzTBrK2JvcZ;Z z(Yqx<%Ats43>%YJ;{z6iyTg$+(M)@}Fu|wnp6KyZQz%a0Alvhh=~FYJyK0hCylb^o z(X!GbXnwa{rw2#nD}#2|dyn=E1F!!0IcxS0f?;f<o71}4_=vfap3GaqZNu(T=!t#v zTi%xHur!Pl&Pvc$mO5N1llnT*{p^AK5A|AE&L6+z9MJ5gsvE4bEUGFwkr@|f3K2*! zHYJ1?yE|{;Z@Bm%h%bT6o{0WJfbT;CF~gf-*5B~@K0#TOs9lI_pfJPW|FN?|Z7jaC z*MSvLLn&+fQ~(ms2qgU2sefW(BBQ`iT#+W;ro*(grSRfQ|6?s-E%_N;$*CVLQ?9L) z2e3VxqhjSp=~j-fAnGYvLmpyc2)Srgsd`P*d$5c~`M1I~(W;mTvryP~w+ywp>Cc3J z?SH+Q>NR@aWCI>6{Q9#lUG8utB?%><2~u#%-rkT_t8iI-D@k}u%Pfe7$+~@1!t$v2 zwLZDtuXRBS2z`5OETmMe(oicI=2~!o<TM@=(6!<lp#rS{<4021p~9MRTTa_EV04*I zErZC7o>LdwczCFn5dvgsNl}pjZ>pVew<FQCG`~WK$6cUYQO!QqwZ6<`iC&vQ^9i@b z{cf8-N%&Oa`>O@&(X1m7CocZZ1#5Z|r2FsfG`N0Z+uoIykZ`KLBr0p~#Xp{LK7&0A zBHhD+8z>K%fv^a?M+B5Tg0_ZmQ52^q12OO65<^NoPKk1)!sRwpP8d-+1jpyBu3XJ$ zyskHl6F+uFJo>f!*=+1U7M*7vIGAG{mk!H;x^jNCr{x9=Gy^N%?3D1@^n+g8!MSqb zZ1>qAl~XWNLo3=1VsnaECAVp)e#Y3ib8P~cCVKDE!q9B`Q1<rY2Ojd#BTDf<t!o{6 zVpfsLP-2u7vmmMFv3BTz=jT_kh^_>I1{%na$4e@e#5*UkC-t&Bz*qBwC%HlV4j(;s zrAT2zlguM0_a`Ot&tG!OVFsg7)Cu^wK!fXNni2SncR}5WosPCJsWj052yfM0_o=lL zH6X>pZ6JOAjRopBo7n6)Lr|hsdSi}jEvCRCmz{HZCmUAC_oaruDJU|N?oUo;iQl#o zmIM_vEybVCNn(k&uj+X$6=W2pTKzn<rbJiWN(a>1i~R5$+N#<H!zj1vNWUlG9j1r* z7P43v0OZsA*ky@lwwU%RR>2tPp~S!;$R%Zs|MlZ;K5o><t%i-&pFwl|D&)06WK=Ne z?!0zZ{TyV89hWrI7P*$az984K8?n`9HTtNnRERCKrX83X7NoW{Suxjmh7IkP`#oYH zm>j<@M)y7Hn8~;Pg_e09mv<VAnim`~YNuJSl17^NW?dYko1VM<=q!r;ntrrFyQeqI z5nD8x*LC1EQ}2sNz+P#Yb@_JhyoeZksNLNL&g>Ce9a2~PQ9D;UBkiaf^rZ=fQ~iKJ zPvk<v9DxHacW-4b`qt&o3?5pVzVng}RrI;m)56Fzo?Bz>9_bC0gVZ_0L#ICimgiW+ zV!el(<kvm`K_fXK0+;)%UxZsTE9d&&#oaAyqi9JBw~}^!=Gyq0kn(z_!L(s)Ey%8L z4feI2UwOIV76ephuun^xEbgzhbG2o!uw}a;=uoVhU1531TO_)_2b)X;d#tbFo!6)F zfa>BZe&ZVdr3I_@D&jr^6ne@c&mNN*hwXA~GsoQw8n|P;`8Yw@Y<es=pywpFDrYc9 z>T4kJiy6DZ{5T<>mK8LMCOLiTjqTE$wuP}Hj3;;rYgqhPTZ+hkB<T-LK>lN?bl~x_ z)y!f5gPc~6w5@+}zYiVuwSWJb{@E2W%$)MzO9P9J2$2JfGk08n#&p1FrB(4!J1v3X z>R69g9*w@_{q+^3wU2AY2u4vEk89B^-^aRn)ZX>?%gSN4WB9(t|Mw(RIBx%>FWT;k zpS%kz4g-A=jub1eoIEZEcw(+z<GeMHiT9SKuk?tY@W#Hp`U{FN^-<eGU)9o9))K>B zCp7DNZa7*lT+hIHwJqPr;|o@Jczm4^wZ7t--yBNUS!dM-fnw5hZWwkE3ql99WG{Sc zDceRe`Yg1Q<$=5|;Jvei(^UNBI)FRRqleuV2RZ9g<l$ns%h+4C%#ypvz*=q!xw`tu zRd%*g`uJj%EvCqJn+y~_cyZsLJCUQmTBCOId7z(?so>!|Z&BwaWxwELkFP()MA9dY zLxB^sAIB?-wL8e}LqiW$n?mh{`<uvwh`PTQ)++pNO@$tI<ebR3@F)Q$w~rDI#Ux(K zlnRP3X<0A+?h>B7myu~cziE?%Rpn=b6ht%{ZNF3%>jxvFAJbOIWXL3e{u4$vd^jMq zZ5iye3v&s616637%QhkNgnLN&nwv}4FR_a2%O9J}AmMxiZT<83h9rIv)}&(ASt=|d zBEn%T!dLx>8S#z5OjbuGT6;z(D&xNYztCg@$Jo<ORr%wcev@G)`(xIE&bCwq)WAaf z0{`6ehSj^{+P#I(MD-;G%8|q6>>bODZQ{HH@9jq0UZhv`eW?<vDMYo&_oR$&t>ou+ z@q0W5V94wk*Wn87J+}oT@C+h*vtS6>-a=Bc(`)q2`57rt0JM0@RgT^i|MH3GD;ZKj zdxOu*?J`=pcX2S}4W^Z=E$S6bY6mO~bo%l;yZ4RMSD2w-HkSl?c-4F<-(f+O;21=3 zSvTWgnC1tJG-&j>m8P$Z1VYd0fEq#~({-||!2{>(E){l2>(V^%P*Ac|LCscm$4Jrw zsIokNZdcwlPUsR;*XXm1>Qci;VZ2{HwVGA3B6`QH<CF{#gyH<=IgN7V+c5xWg1NE^ zZvhket|iq(5{nstG%AaD-nJY?*-`6eL;fgqEz^Q@_sS;7m*8*hD1*zKpRc+Gf=+qv z^V)p~%rbR+zk~y($4~}LF{NKJS6VI2OpKGcYvc9;SqixkW0-7zD>&TBD=NC8EeDrV zoCVAi()6;3)bF;st>y10yvIK;9F5~2g_<ZNYm5MQIh?~E%6)JtZ!}gcN%RbPUAJSz zSfhZGQOuTU{@@g_LHu45e{#hI-nM-5nYjAi(hIu~<~}vpf@$WhUbP=4k9lnI`*iIr zRjb0PIGD5D2~mwOwe}`bTONntEGUe%x)NK_4^76Y<7}#@?WGyc9B+;}<EdA@%h;VG z6*Ma3>1M2$ZAIg`X*s&h`9vbjm5PbqROs5i`)@6P1awNQjNb`3b68e(rG)Cq?<XYe zyJysXOm8MdVXo7Bl&W&7DB`8eBZ~{K>ju9OCz=<inHywBmQKR<yVhYR-;=;GFoMZ# z2au9Pr1gTVwcn<YWx-23>O4ml0j&_$#K{Q^fV!<KJZ<~Hdw1_q7779{D^E&_g%ICF z7p1|?Cqb6twOFAASfx*mR@Ysb(guZ^SAXJ^2d}-nC8`?RU3_jIBYZSvU>fV;umemA znw~Yn;mvDrn!!gt=Avna^Xs^^6*7k*EsJ2lGd@m_pgTq<);tx}1YkwIo0{%^2ZRcY zg9q))>*}(qU%Vy@c;<(XjVn2*{oLsh;J;);Ua)9htmQqh!UU-wC>OuMRh;wd8L<Bs zzjusLfq0rKm=4zCElowyRlZjpj~5!S7ka_+*qRL?cV%edczowOM%&x`@)#D3!^;}z zVHcY1*<6uVB)1_jWbwB$+S`18>3+21;J>iZaNn)*oW2w#RG!QuH0mXK%`Xb~ls=gA za!1m!f#L5O-{WpNjIcBOb-eCgCeH=fvOiNZ`UH%zOOR;va6G6jG8R(%BPuA5Ya#`d zS^ffYm+53aB@3VN(0D@=cZ*Kl<F-#|QtI?91kXNZt)17|Fb+2}TP<CZD&>vr01U2F z|IyCkYyBI<&yUlIG6wiHbe+iFSLes?E$}2iyIr(5_=f=apq;8fk&1Gn{P%XWV{oD# z)v~A&V0o}t9WVLfJ!~T`J%8j$XW~;7ESmVMYD(a+_-Tc9I?yB3FE=7JYlKNkUHg6x zU6e}g{VuwlSoCaONZ~z*O{;2odrrciOSeKe9-W10^yi^Uke~*l%}~EUnCr%=LFgU1 zTF&Gz@wQ|RX1SFvAl|A?U>#)MLGXDBXjPC3b8`*a{Q)k5c2QKlgjmu!h4#@aqb(s4 zCNLhWBSfPg%F!KX6JJ*ewa9ZB6T-N4)3ZFYpl9;xzX>T0?)S@oQIw!!6F*SK)`C4P z@}~QvPvUFZZLJIjAazu|W2_2J<{sQ*=*(35@COps>GxY!J7FFwJ#;F*Hcc%u>qIS3 zylA{d2&V^tZ|+k^;4rJpM9I-jY#KzCQeVzpe0Szrh#|g}FG`}_b)Z8aUr}wOKMViu ze(SqR=EdG8knMPP%tZWD7JqVOH-Ia<_3leL@|IkA527)-3)+x9xu8%BM&-yVe=qXh zMcX#-x?g4%vfvQ}ia|mu*uV0b)T@axO$}}ZC&*0vwxyXXqO*Nfdet7+mf6yC+I-C( zlAOkcC5{B9S&UM%fDtJOGllM9TcfG?(I(-^Y#tcr4v7naHk2+vjB{FjpHhB<@wfZ; zo)PtgV`}v6Tr8T5DiKd&m@T^T<_mMWPd+$(v#?$7R;Kna<HBCBYA6msOw&xlCoin9 zDi^WCjNH2mOG*M4lGx^ub=X+@Yp_Qf+NVZax_|wS`5~DtU<uC(JSQn*a{HQQ`R3Ro zt@6?Xi~O3DEVO1^cYDfFJ>YuXJY&Q^>`vUWp2mBkdeav6X)I0{Y%*i37}d7*QCR|c zv0#0ge#pBTYtU%?r(h7Q9s&b6{caxP0KQyO`(>Y+f%?^VnC#i<hp3S%77?(xz1H!x zB|ZuPKu{Mc@aARrYlfF^tK6L=p}MbF)~vVzZykk^iO-MXto?CY6nYhu@bNL**@Z*| zLt87zMhMcEKdoSkYBl!+)O3fMaEdnu(*tzQ?)f~M*L(C*i6<e|Gx$R;<j-)m*<AJ_ zKX%t_yAfE$<6w+Y@}jr~0-8;!r91oQRqN}mdq$iX2|Slj$4MBkV9gPgZV}Aq2WzQZ zM;gGT63!850JWps^cOm-T4i(q0%Yu<?#Bk>3e1h5cxT2rtuxo4zkST#b&LQ-afa$z zRhTgO5CM<_IAGG+ZrthLl(+|h6mnUIxI6G7D!Q7S+Tyai;<PfShr#W~4DjcJ5yAJJ za|K<Vf%qw*(sKoLc|omX7vFX$uTFq2G?c8!(`4Z8Rl{{iZ$EF8L5|1`rKu}i&oU!~ zj_JVEnvm$`Xzw2ecRu@1i}4w9KFyVO?hvO_p*1BsyJ;Bv_-GOCCD)7%Hhod39YhY- z9pXv&?}~4%EwCDNjm9Mm;JLD6&04V9a`RDgJ>~`r^2${HK<)>t=E?{MX)5iKXrdaF zPB4+y0mbV<LySQ;x9;-oqyE>x*dcX}qYFu>6mT9W_T+-W&;zh8Ql!xg^wP)bBer;H z*0D2a1yfArPNzwNE4>Pzz5B38H!5TK*ZCc1#!z;KtZ%gvtL-ek9?Dqv_0SVE&s)Jj zSB(PHOLIOTD7Isj2w{CQR8aNOJ(z7LYl#Ku=!PjNa*hd+H^U3IyIUA~>a5Llwbtrm z-`2JZSO`uz8tnw`kC#6kA-2gUrUrZ1qH`k?*Ts~E^fV;Flvf;7K>^M4V{HDBk6s$% z{WoFbH+trE{mAh+74IG<x4chqY%z@*qqt7fIQWr^r>6yPVs;w!RQ1Jurcb^JP$1CG zC4{wt=M%6cyn&BsmH~WlYII`5<%cNRu}jZ932><V_)F__gpQwupvK1EubGP+fQ+dw z)`^38U#J@MtKvSEfz@Qj`RP6%@g?;1<3f<pJ-k}%J_u`MTNjV3wOZAmNa1+V^?pOU z`_^E)<a!cTT?zpEk~eavi&XnqXk1Gee$Y*8Kc-7haYI~$Qp_!z1`27mZ2k0Xs*=T$ z4c*m0NSbdg{-0k9EBH7{!d(KmA@t+^bF-TldO1yeTX4$;Z^_BtzPe>;Uo%%<OC?hc zhV<SWxOwNYCr0KGRK{fyzs6=a4IB!A8@lEJR>zB8azB40HMBc`V!rx!W$#?-6t#K@ zZvq>*iU26R=FG$Fz(`>c*?_&bcV5l1<-JWXC1kyXMs(B?K_dLSS=CQgD2Dvt?x+&9 z1(AoCth7{m>O;w5?$nis41%6q-5^L}%GB5Px_e@7@Q(N0*cMKtT1{n+QkOvk=v&qg zd$|V*L{o8fEQE5xVzKu}A&0>+0>>FR1&+{4t+{NK!$TUGkG<!K4_<&V$`WH9;6{#> zx!Gj?truxCo+Pb+X**0f^<z~Uq%qR%s?v1Oj_=!HZT}6pUuKWB5D-}y_K<ILI1|(r zLLvCWCuTFo#(Nm(zAuj8t&Qm7nwlPS@-H<zlkJ_H?%l>KqhZOfLwm<WNO!t<rgC6Y zlpChQO~P+2{z+q@RCZh}p%PNxQmQV~45iFqq^myE$2%Y-0LDI)&fXe_gWqrAExNXL zI3uBGnMTnna}641o2dsuH0z!Rn0!ojlW)M5=!_>fTG853rmJIh^BeN#M&p^HZFNNH z{(K|0nS}dD{B_U3>2=0qzc_cG(z9B|`gzaYMCOxs(&&ZDQp3T{3pGG~?VmlHN~8uM zchT9V41xLhjd-oRh4w(EYqcfq=BO!VVjrLJ70vRj_d_Xsp@Z{#1czHtfd8OQ!^TH( zUf<R>y2=sx`1=>hXOfz-oX5;>@-8Eo;rDWQCmv*?I3bh0Oa0H2MsJ7yCke%(gga#| zjLti*D*i`;-pO*?TA{NN%9ay(SFMzj(qLcN31ewa^0p%COkmbCqy{sBY-RmYJuE5& zvk~|v9T&D1Ve5w`s#2Y|J2{UUdOkCmAxZ`cI?sI$`clv?InR$jvi|}444L1yiZ2Yg z!V?HUEWEQKLEV@V3RB`Bu{$yr4<3oxs<`UT3K)z6j?U`A#eQ*ZdsbVvx@hBD;8yMm zd>p}@F1u(G+S$r<@>6EwaoCH#QdOSXBHGU}w@Ru{d;ZF8$6!HXg~7L*!5r2@Z~u+E z0myul$Ol!EZ)}V>pJ_}dCwWu~^y+HP8>80_P!z-Ydbv;%z&U}NZMd)Obn!b1Z-(fF z?o;I~lvnw~{51UN@;-i<@|IAalgcDmPKidxg;1rr;HH0BobJ_Wjy6Dl<7oW(%O?hC zRgQr6&pd?77obUiyz#n!lM3P%t(#Ya=LHPuS5u2XNsD{=l^m5%&_)8EfV$oj5p?wz z@5Ogi%>&pi@;$m$kwmUDq)DIaE_Q!cBs&6yrmw&PAaCH#{PmBNFDv9M_G_4-qrw?x zK1ETYTkenf5E=DwBKZWF%u9-ju&t#&LXw^e$G=9*O?V%X!wRy?r0{L%cnMs%cw1-b zRSqZJ4e<*G={*|ZjaNbp#}Rdl)iO(SDDQoI<;wO%-Ki;YyPq7wy^Q`8Yb*+_8P!#P z%Hfr>M4<s`W=%QjV54sjdm36FdCsnil<G?NRwq%QqU$vb>gJKtT<Q5WX+W{{0>vim zW4BKTfmDH>?&tCO6+sGcjI2^QA_*MoltJh8oIt$<XIQjUo;j#uXV<03k)hO76q=1O zQVkJ=_}wPGcOeF6sk(J9r<%A=0y`4i9oO{d&<Y&s-adLoAYRgyr*z59rG)x$FH=D+ z>H0o140KUH^-+m=^K18`EIXh|m&84PuhJPBSX((HdLd57DB8IIP!kUjS<!84P~kIx zm>3!#k=qD~;$(Sd9@exhiu2ft%$fDx%(QJe$w*?ReMjFDnH^M`+7eVE%(h(o0UJxq zTifYI=4FhF5{p2I_sMfDPQs}r{xkmw{X|~;P4R!Qp_H=_de4*NWjoKyB_^1O@V>D} za4>=K+HIV%9^37sGrJFB8HTSiGED&RoP*|)t%Kfs_MBe9Fyb3)9r|iA@^NnDGj#E= zf;bm#Ij?SVO{U)s4AQx4-S~h@up{rPf}$y<>BT6*teYi7%;itOVb?h_KW$0~;_%;_ zF3A~iNcIpkKDa+MyjZKXWpfX#q70POtgh#2hTR(q?et$zfo29`M^VUrubpFYH>ahV zii#_$c#SXaF!8AXfN{<9uK9Caygvcp^-Mp{H8`Gz<?W2_;#G;b@5U~ZZNKYNuam9{ z;3|*g>~P0Mj9dsTBPv&KkZH!%>GC9i+2Y^%9~;v&uu$Bk$fIoKquQ?AS#Fh4?lH`a zl-hGs`G*!l;`%@<=ik3=mEWiwLe6~uygE3$Y<jK2y@B9Tk022g7|`8eif9&LUr(>N z`lOGXpGOxx-9H)TS6enVxg4UqeKe<y@<jtyq6tkv0UZAY6d(;m^lIwBDkylRTz$!3 zIpAEq0P;($h35pzt(sBC(pTtkD9qT@?(XizVU5)+q(o-EO_uVt*lEJ9^QHD?a9a$z z^yD?-Y6fFFgd+%mdk+8gec<-I#)jIMi5j~i7~&-4bRWIsb1nj2`sly$(ktL2s}FlY zXjPm1ZChy&TBj<UY~-(5a?kwRV1wFmPDgsQ%IQnF!T$|l(aA(2@C~5>+zf7|P3*0j zJ~lIvcxuHcE=%<}GtvymNVwvwe6}cRR2sC$zGD|><LsEb%6Md<a2vU%G@x+g(j>tT zVpbkj`kRUuYT{4+bcryY?UT^;*~ktox<zkovLg3YCQ11!UN38FD3G%T_bJ9bGEU}~ zDqYmenZ!hM#!Tupin-v8vlJdY;iOcg09`LK`;|&b24s1BsqE-A?A)Cdvzg0H5#~Fc z%?x&5F{|cl63wXGHmNC`Wk-`Sg5|=9LDs!cKcWS++R3%n;x8or<enQ;B#ag{36)ZX z+r8sb==J|uEoJ(YUH_(S?UH+;HA->uI_qw<K!*Yr@X&oq?4{4CZ#Cy$4pg$HUGL9A zze3%|rx+THnUPFzWTAg6%_gP^3~<(ilhl07D`tsx<;XSixAE3gL1e_(p%j??_FI_= z3@6F+-9AAm<kN_^t9haB3vXc_&$FcZCpoy+kkr?0*4LR*zBK`@pb&b6NFD^Z1r-KY zn>pz@CP9}#(#?vN!p<E8h4ns{pjoP0p8$AAoQa;@2aHjRYC&yJhJW6tG1p0!P1XfO z*~{kY9jBkH<EWSiP)Ij;pu}%P%Iu_J{S<NVO8iMZB9a^m?UexPmY$IzfnTcroWLGa z(>L(q0qEQoRCo*1V5T0xiz`!FCWaD!mQBiwy`EQ>N+T4%a{%?onh`Jc#t;*K0XD*R zaBmA!SpzTl6`TAdYnsqv-m+@8?gr07!1ZR$A)bmFySTtj3v1_>0wVX18l6YWEZY=O zx}A3nR+$I<fNQw%a7{aOeZ6K+*_K@^F+jD~Y!n!+yPA5^j*uPps&Ot_8{EiSJ3<5V zmAY(7G;5m6N+aYnEhb-6_JFvaHh95}_W5Z|uk+Yb_3n~dKF=ls%M``S@}S`EI?%E1 z*kys}3o0s{!og=(7D@#T9a%ds|ArT=aEUflfV+TC@kZ5Ws#qO{pWi7R{ITiUpxJY^ zi2mi;MV<fNq2aka)b*L)eavd*uXKU`FEIF%fA)K+bKLZ@iy*>2=~fB)V5>?|xdpH& z0W<lv*Z6Z;xUP(k7t<`xzMnvfDSZ&Dm5~Ohp}w4aQg?)%6#C)lzi@v7uY$9RqMJqc zoQi8Bq5kXgwLilGU+0{*XQ6LX7S{xIcwJ*0x2WKf)sK9n)pdT%NeQP<RO(iLgUaAx zLzBPur;756dj~VPjk_K(K3=?phv{BKWrK)LE#VEUNysl%&xK>6Y*5Zaj~g-2+vyBk zQe90b5D6z5=T*C;>4A-a)xK21*>YTWr$LmPs!RstDK(_2&1TXw!=Q~Rhk0bEwdw@u zFLU&RE?HStg#KulXGzz6%^_N|`^SQ7wAn4Uy;nB2aK6zE6r?|O6$@GU!Xvfa=}_PJ z2p{;u-o(i8tqBxG2ox`w65(O)r-2eEn@*YY1YRraZaWLcRQ^yF^v$pO{%4*Lj5Fs! zCt$yA?;XO}!vXmbcCb$oDRT4~8rA=P!n(eK!>Czxr9X$jS^R6wCt7>ZD3hQ$t;rUl z8T8M;D0TVY+xx8DLU`Qe_soQbtV7lxfW$ZQvIzF68nG+lP|4EZ4zk99t+delsZ`ML z>Y4d*oBv2=9g+&OmD>pv+#-DgAc%d4bvpxeqD{4<?hyK*%EcD479}-Ao|0~1M5w0f z?bHrt6gylN01|C{WN5l<Y9QFBxGWM6U~ayeV15n3Kcb=_9Y6yD<b|`kj<^H*5l!hf z4-xK_RWk{oBg8cvIkVz!X&nFcP*=v!9g13Ua}fpA3)oz|Ng^FlfmH~e-7TgMz6lrM zJZaH2cgGJy4p=YftJB<#3>wIoy1Dd`N&&3Iyp6Rs;uoO!lMSAr@npUEQ37Z}-w*B= z0P)u@LFLMjz&PYR<_$AT7Fn|0f}x=`@qFXkR?Vql+J6OEF#Zz->1PabBU?shQfmL! z0<<hMR*Tb~!NWKOwK*h`b3y)50wOdrqt0;Y1~36ct4mo3TwY6ftdayMS?q;NTLDz6 zoRa;ZA)z=)vkZ_5`$@%12kwtT_fw*@CVL2E{8-o-hIafFPO!dA$0E(H;k)5ev&8cn z)58{-OEwX?x3j}q7T=~9H`)=pI_{2?guUMH^T{wg19RYH3$^`H(MOSUedHr0EO>p` z3F~%DF#&U)o%r%EtOwBCYgtFZ{?Y1AJlRxT9@^v%HRf3mtv`K2Fb4b}40}YeJS6}W z1;$;^E9hu+;hI?<muMp>z43*)^CG=<Z=qUS*q2{Zv*=bo5kLmjc6@s{qZmWWtnwt- zEM&zY{7$5j1}xy=d3Ln_xPT$a;4y?Hqh5k4@_?oS3Wj=P-A=O}n)t~55V!N2gB>V= zwY})ENe#VrnZ_Z#%U<y2CI=ht&ejQkd;27T7NHlcqr|*H#|xmPJfD!H+lb*sKB~aP zxjg3MAd3&dPvbKUW{M(G9)$o1B>OK3?t76t4fT@mvT<RLncw5_F{;$yv33|-fMM%| z>8*aVlqa}$2gJgGza+GXq=q4h3E;?GKZAd2lnMHx*XwcE66!ga;zt-b=C4;9*%CP_ z^!#o73Ct`f@BcT1h&oX-A8*fI?5InTljbX%ue5SAp2?_5+!+wC)`0z_=}!dGEIE|C z1IU~%3s@S3G|1*^>dIabsd2ZGoR^dC;W&wZbS-4BwNZcFjmv?|0fy`2+-dTX4XHQQ z&98PqMv?6?3Y0)$_BdPt^}*kyTvaWyO!rlfHY%}{%VT#HxLK^0iWmertV)kxwwCoB zv@})gS~_~9iMs=wbK55010Ng?=JpHtDjwj9fa78~m|SmiQ(dJaHOy9VN0l-n8u|1u zh(<uB0MPCG(TIUnYC^X&lSYL`E3dol{zms`b{5bPLDpNnKVf`@lGCCdxBMvop=kr( z5*RT|99`~C28tcwPWN*v?Koq&gUpz+g2~=$g6eQ&ZvsHT(*4x#!GEEk0k(#gL#FmM zR+q)%Q+p|hhRnZA(=t=7RNaJVVV^<hN`ex88j;{JfMoAJVG-JxQH2fhu<g7wX|}3D z(K`i@Mb3`ZXIXEgf**+@H`MUK=$nDb{};*?7&jEB*g0r*7gw5d>s%2cu&aWf)T-GV z8g-!(nPTupesEy)zB)$Ls<<}Nuf$uu(5WnvXjj%AXmZw(<>nmdr&%yLi{az2ghv6O zd#yX?JufKxFhc7Gu#ZT%j$Ak2LV9CYW0fjL)XUX%<jCP?51|8-%`?`^>s2O0gkV!| zHd57^toDl!39)5Hw0*!ez2xo)o>nCIXEWM5s<63@6ALN|MoriAyGm*=%zmUsr0jkT z;q)Z4v+(gAAD$m0^Jo*u&YRxpNUHU=!Q`nrkjV@nb?EP@ki7l<(_Qb2su~gKZHE0& z-Hs@z!cO_f6K!Jh;aX|p8^Y-x`1DwOh+xeUR3*9OlqjRNa9i}B##_g7Es~Q^xW>|! zneZ>z4S+Vw-h1)455mGcBh<Z~2;aqfY#PWA3J4>87o7ksv=|alhId14<ar-<Me7(Y zfBf?r_lo?w<EyF_g*RpWoT?&wtaa?^oR`nOMLNJ(L`aY2bWYDUeDD~bR!jdbTzg{n zmvC)*o+7m<%Un>YZB9NZ;iFdE{!SzM`+FB4Qbixm;`O)%sR&&-fst-WBBbx`PR0ib zlD(;#`r6Hz<t2bT^ohW$-G3N!=CF>N+U8>QQcc=uJF{D%C->GN-@u=EL|e{hYUuDx z&iGg872!(1H_w9eD!iube=!o80%wvX;_A!)PiYz<Rz=1yn*kR7$N+HuSr#objSu-< zF{ihuJlcnt3AIRpvjv;MNqg8p5!D7`OXghTPplO#g3%_xU9ecmHG26RtuFW<>a_p8 z6H;icd0OQD5rzx`7!qS(eO*-i80c2ex$k<J4=JO%n?Zh?EOoApX6-%Qg&I{~O7j`2 z_a}OaMA0m#syJtk83H{`fQA3cBt!Upzhu&yGH5?X%zby*9Iv8)m<zk`KA5HKkG*9A zZi)+R-78@ei~qRefSa<X!tP1{4^rfk0nA+lKY9K&s+=aidytZ<P|p9~OghFr1B>i` zUfr^GJsopt$C+Pug}l3!6H(}^=S_0b_p*p~a3Ek{!K5C&&mVx)orWeTkY}f5n)gBK zg7~XQc(NG~X}jF<jS`B@q7_?!HO~Z_6Gq!*TXdR9AP26wusGwM;@{qK+p|t(-m+#7 zIhvJ41y-)W^)w}28O%_<vjRMawneq?!MtZ)-N#0&XXS;@q4`_MWh_kJ7sNgiKG>c! zo~uZ7_4LdxSDznlyP+KnM#q&T`~iO^9_$_P){0&$Q(|KVgEsL+Dzh*}#nIH8;PT46 zQRWfQQQiq!(mh*Anbi%5Sr*3Y=1=5-{b@j5p{J=XPZf@Rw%WAORw~~*(frvLo}=)0 z$b^eiUb2B*uR1U2%?oz-*Ta@2k~k@aEBT<Q?oLwqgCTXcxD|-L9F<etXSl|RhU}BY zj?s}d1a2rfKJ(1$wgxCIs$ky>?>G>^kp4J+>qyhU&2{cTPM^MB8wwNnDl^6|k@-jy zyz%Y~0rbkYT^vyVo8I~GvxLth&Q0Ec_4xDCr4r&!O3fAm4nbe>!0I|lCavmJzRDj- zC+R&J-ol|pC>+Hg#7!qkbwB~ku{Kq~3m$p1v%qhLyMrO1qQimYGrY>@V0`!Zh!`hS zmy~w{-<ns(Bff?;Q>z>?C!(#e%$nA_ArOBf2~UzC&nh)+orJ6&FIPFL+=jSoOG8Rq zhazhh{ueqm=e#|Z#`r+*lY=VPTZX($e3U`r3*rE3FJz@c#^^Q2Axr_^0lKn?9WcIT z-K*it`4doxN6aCmP$PK>r~FNz9$y5dWaJO?3sv~X8`2);0U81@A6nBCy9>lI&L5_N zuBF@fZ&@wnpIMCshAgwZk|8w=L!W&EA0jceCj?!bXQtmR+t!OnFtu?!?jkcPkju>D zmi7zEN=4fmnod$gG%V#EGft^ls}idtzGKCoTp0rTu7mSTLiDfK+l9UmyN?2Gw~eg7 z_a#IS*y9+Df>IpWF8eU5mPjeCP!TMe4KEXJud2+4fw~<#4Uc!&#3j}cZpJjxd8O)3 zf234*Q$a<+#cSBM?h**7bDN^OtYOmO?}B+1ME&qfv(f7$;q9*SRG0zR;2)=QN+)d{ zzth|%b|daj6m{PogBg&dY|Z(S4;d4CU}6L~atUvCo!B_z7Q|wf*>r{Z*;&si%N<{J zRij$yyk)9xZn|usJ;$1@zG=oE3^cAH6MqTgf+TFA&?m67ZR^Yk;(;bly=0FA&tG4l zfP#!ca3K`ve&~+Ab+(m_=w@<F06YzzR<Sbw%dyk~Wu--8rh&GX>(^Pvcr@t{jM4hr z<r*@1$<MB@5~xuZ9>>kwy~*i|D34n=Cq%L+zKdK}LPE^3IP`oztL|yQrI$?cGsWWy zyqXl?vnq#J`d{%~=+IPZV}1Q(@m3%(_&c?b<VkfCA#$w?fG1B7WhDLwCX5TZA{_7* zp!}*SEA^Lq(!KO{Dz{QIyu_W3@hv&-pQjVVt22Sh;NkGU&^f@Wi~}eX4Nz#e+<tcK z2JbZ)aH*`?&kBRA1bC~KN5`RUy$MhayC=`BDUx@$yD%>)B3^b53$QHQjFsP*$&d%@ zA%zYp_!npggg03wQ@E^D{n}PVG+0+LsSZ#@)<G5NE7}S-2QHR{YxpFB3Dz$tT7v(< z-gkyYwJqBsC}IG!XcH9`1Ck^NO;QvI3W`b&Dj+#SLz5B31OlR<L`fo&b7+#Hl5@@= zp-D~7@zvUApLfr@F3#2OogeSV`{UkU1Kq3VtXVZ`)To;DKipl6lMF*^o@aG42oU3L z7teYs<T;JCkKi7dcs{D%FDHxKlm=rDdS(KeyK6cSNsJHuKZFsQkII5jOz>9^$u5a( z@N^r)uR}ZTMehO+2}7>qVMr0_-AvG=sYxL&W^QFhmMg$=J!ttxW$1x_NKTY8yW?p# zGIW(1Mcl_ff`(woL_WRN(Nx+v+TejKi9y66YsdY*brzQ(`QM}m|E&fb1D*d=W^pUV zq`ePd+}Lh=S(xsHP?cN}HF=OC;GWNEt$@6Y@EHy>j94ss7bVD1eEmUs^Mnw+AlFmQ z>HTtGlOWX*0Ar*-nV$D;xl;gY*kAtx+k6kGHEHM`I`$12P%#~V5d&=SN?pPe0@$hm zgAv@GOFY;NY&++~O-Z||0IdlaC){ZNtWl$~E#a>HZ3C>euB8D-bcVlLL9pt7t}c)% z`1s<<RUvmo7*RTDZsP^_>kwP2&m@KIo&H=@3_-8P{Gt`3#xz`F4KXh{4=gL-SU)D6 zzxdOS<u8h4_)?B<9lvE%>zSZVG5ED=j;K>U!Yy`B$ew~$ybR=_W|&9OmZ2Rq8EcMm zq}-s??swLUsw;8F&DuMO3O0<d@Q>Hx%T7ekBbrfOuJn0ngvEac(~Pll!wZ_D-N8v9 zg})Q^dsB7({kVxUTlxcz6`HlwElC&qKl%tvGnRumW*XNw@w%6|-u2*18?NGP4zpLM zqbH<&?9zsbFbh@1*@P4Y{mknKFF_5IEewiTfz+f6N|l+U|9}^RQHdaobX!?0hVd7( z?c*=m1M`UMLHECg;h~!Z=MrYEkPdtTR|HpdtN;G@_w@FVYB1@X853_Dr5J8ZNyUfW zzGgXYRJJT&UzV4tf((6fmXnEd1cm&JEes}VfC`D+nLWQJz=jLn0X-i^v4OwF(crf; ze)of@!U<AxJYU$3U%d^=D@Jflh{9gfw}fy>lgp!eO6(pNIH~V?H&8nFe&M~%T#?(Z zKhRFJ{Q90)<iT{iLpu*J-&BMOcS3Y4$`k#Xu!1b;S4^~p9DB2;ztr~z)iHbIT|Ven zg$TbyX?OZHVIy;W0eI>V*9|nZ@|OJR9ww8^&2oF-Ga>$VdU5o(+u(j9M|6z>Nbdid zm*dxmbv?A>u}KLdyouT~axf^`pUJ@}LtcM<)WV<eq%RMs{<6*}qO^baJB3wXI$Z)b zI?=yw%b&e+7?>}OpqAfQk6)MJUthsNv)^=8_BrhD$)Pv@a!Fb6MnAtDM)7xlv`BD^ z>Q--?D)rVg|N5A}WuX73$+~`r(hn_b#;?2iXRo*h1AUikIbJ6FJ5cf8Jt!MHywSzv zZU63f{?G4#z64|R;0S^(6>-v=Z^v(~`8{Y=?f?G#{BQm8zxk|tL})3W2qL81{=c=y zmq4I!$Qq%4@84OQzkde{$`?Gb<8AP7>Dc~_%_@OH0&o6c*B^W~>OpQ101tvCCXBuL zGwb_hCH}uF`ejM}`-=X5SM+br>OX$;|EMeCbZgW5rxpO+EfqdtpZw0@l)N-rus%BY z;#`7W$b4F~`Vf8l^6JJ?=tgUYO}AsE_#O}pOzzn*n;~_tliKYH_<l<X4<pS^BH-aB zFZ%rGb^k)5$U^oYtMfE#H9mJfJ<-@CXZVyz`<<RW8<TqiHJ76_1v3fVD|y|9O-)T1 z!zRWr|GRN+;%m#ngL3p4D4;q}BdQMbVuni9?&YQ%m<i{Nl|WsgzN3#b!V1eZ*QW<T zJQ#?Zf5PDqw)D|-`J-``s_O6DX%1emkQ=;Gc~-z%)Tg8)zvjhg+%@}=-?%PoG=UHh zo0uO^!*|g3`#Fu;AR)X(ebsou#`<7;KB+yU*Tnca>bv~v4*!SmnmZFbVPujw2jBJD zD|&sWL0u<z<WmREX|_7_voN)$rOG+{JWx5_Nyz*Ue(I?DgaI;uS3h^D($qv_KE;1u zn>%@;U1My3=vuvKXIknW=f$GRt~~qy;HR#xPi#2N#1<^a2J$!!C^|*q=VCQ)=+W)W z9pp{a7nu#)W8Bud(zTF3xKgmO0w2tTRb3uYEXb)nv)aeBVLRiYD-?lWD%Y%9O}5OF z{Vhoh1>ho^o~TxCG)IcP){{UOH|SdY!Ee$w>B-;K(%~>ui6!o(t&!DSDpTZRS}M85 z7rZDuftQ;Z(-mDImseW5rFCh|Cby5Fa+l`f*GAJHFMi!~v{!h@OJ1ORs$Dxf@WkLs z)vjs38|PL#s@FR3>OQm`k79Ohy-e)99W85Y*EdufQE3@66ZA-V_xVFv%OkOY)z0fP zff_45OiMn4d)9mQNJn#r{FX_&Zr+yxF5#Ff%e7ZLrp_zv{A;T(6mx&(CX{T=d|D`@ zxm-8s8@k&6vGT9uDYw=jXY55SLTNLVp6D2lAGsZB*Hs!a`Iwa_bg4p@(Tx>%0`++R zcWluPA}jn#w+{mvcu8{Gc6ssQV{bxr^~P#7f1>l+sLf;yBSF>q%Px%?hvoXvE~~=s zwa#v1r`2NT)nxmj^CGR_1iNW)?`R6tt!l5T=8Bi2)Kz{3L0OaF3^{VKHV@DqTjBJi zUuJGp_DmI)KYp>TIX@$N_+y|#)-IWWovX8Q8&)&?&NKXh?ga~Z6UHHPak>Mt_tnA* zw~H=*8FZ&Jd6qwyUG12F2Q${Z7}1MnAOC$_wo+BV(Ato5LyjoVvul^bu#Bcb{^S!5 z9)C?Ie5(Dz_!Gqoj>fJq!*Ec2!mjzFzG?P=-{9RuySt>Q3dKu3#ZF_z0T*<ui~@!! zm=?t>2i__wyz4y3<=SpqkhkR7z36$m%59HGivxU<3%^&M+^WC^82^D;If^Q#<j^9~ zd7U_joUie$zE_rsLc>yW28gG7Xl#7m2Gf_J)MK4!uYN#q2p7s<0C!H8gBv>E)vBet z<h-%qY?|Lommh}j=wswxxfxbJ1yL?dKjfcYNw?bgQC1;xVskgvCcM^<<lNG9nEF2S zd)bCkGNf-WzMMyilg!t85LaWlHrKp|#*v8aZe9H*w|b@O&JUwxF*dcOS2np3_}*Qb zS>Lhk_;P2Gp=LYPQRhB>r@rh;XRt>Tjmbx}JaV>Q4+L~C1$XOTcCoFRnT;0B19y_2 zqW{9naZIby#))WC^}dO~LjKn&UYRHm75oXjLgj)(^}^(DP4^SK_Z_0yIm%4>Umcq( zm+N*WhotPa`Sw8ji9DNwBLE`IE!&MVzjSON-@b>@XuV%<UAtb^i&3~uh$v9qx`oia ze7+071YK|;av>Yv#x%!9ioUp_n)%fkxs6%5!1eXkYNw1PKhq_@Qaa~p-%y7xLbP0{ z347jbU8$v=xl+%l=7!zOn`j{&GqE8?5&k<>zcp8p%=NEp4lkJgYX5sbu}_+A%$w4E z>8okINW_E1x2-NGZrD}b)B@uf;BdxUjx@yFAlNPtx;s~gyw-;fPZ-&AY#@*c-$&Qo z@|Y&;WOdMZ4|FIS=dTc8i1fgqdw|TmbC@S}Mzfk_Wl!>xIgYd!X1{Z3n=9!!Si8E( zSCL<hj9t=oudPG(N>_7kFSV&&{`A{KZMs!hLkr;Pb5Gglx~j77{rWS^-gF^*Wt<Sm zwCKvzl5@kj?`;wPRA9kUpfA=z(|K*~FVrK&N<Z>r#m0HTrm~N5Axp~APG7ADzMNNp z=IVD%dioPCagTm|1e8~dJi*MHvQ$w3kJtf0=Prx#^Oc%wjhcrH9cCk#jPpkc`LT<_ z9B~>wIim^KiB}toUK?3UT@!0vjiwI+6Bmafy!aCBJ4XZdIP_4Y;+FlLm;5Uagf4#j zDUiS1n8+t*-&S8`KQXbfKJk(`eL18sKC?@(ZE5YF;S+1h9f9(gXghxjL`0r-S5O?~ z^w#x0(e;RI1fuch4d1|X%t~}3@zXMNr9*1MPJixyA1deiuQ2Fe59wfAt~Ba9B)-tA zSg_n!c}-;6r<LG<0K42NPgAxKR;y#k+07r$li|oq6L4I%no`p!Ik5OdbMeVwm;D3- z{py!JPDxAo6D#?-r5?YBH$Q-6(1KU`0E%;z+_egb-q5m$nxBVhW3jm2&7Q?+=&92u ze0zJ>sVo(?%ybU0PF}js1VVMR$Xv`IHop&>fzxU?nyaB+?VNC07)l6PZiwa+U33#| z`70J&=CmGLN)=s69W0e|vZy^%l)ExdYM-t04VjG`M55=*L9UCMJ%}fmp>>|3b>je7 z8LEey=n57M^Fdi?l%2W*gkHi|z5OT=N#E6Z@u{<()4Z;8hJfYZbvrZND?^3L?)<r7 z_}Z5m4KR2vc1YbVQ=?}jHqk_6W2xG-t%PEaP>sWArqgJXsY&1%v(=naxoQ`8R^YwG zLH>g5?6H*cD$k|Kr=lIJ8)KdLAkP7&jVUijxm&Hr5Ar@6EAnUCcg*m$b=l#aPdHvj z8I|Uj`;955K8;K+e|=>zP^a$MrH@7jl%2h**H$W52ZJoTi5<j>_iNU|jEvo;mI<Sl z>diCx)vF63a>cJ0&NmM&=zM*=L!*=P`0>kSZ_e~3onmGd<1_6VyUTE@o%WMT`sc?_ zo+z^~&NXYE)cWBuzl;08maTim$bRde)#GAhtsEW)FQNT<VP0Wv#I*rZX7^$0i*}ex zmo9~e6|;sq$iFyr?i36Lol1LLeIVIInM|3YbhtiR?pyS$f(U(9Ez(XC-n7w$K+}Bv z<tgyqtfcnLOtsixYb|#_;m$HPm$=|!#W<y2{WGB}r$4xw#&*9{PSN&((F==8?{%n{ zUC79P>z}=JTc91OR_i?Kuk&qfidSXrB-q$sZ;St!$Xf{EF=&uX(J6lzarXA>7h;|m zUbP&JjrElYsewgvyDFIfCKV<9_5?R~omHrm?<uLiAD*2dP>TxVwi#Ee7kEj_;dl1# z=a&4UC>j4N6vbTE=0+NI)>oG@3^{ADTh>^1DO87!*LNv%)BV=cBU|LWbFbKZ^r>{z z?Rs`>vgZrMd(Qihu7?P<#+PF!a#@%hCSA45-s~vQKHbyN`!*qAW8>H!CK0uV5K5LC z=2A0^+wB}!UeOCYI4aM`XY?-3u(?Xh^V1FpHE#_XKMPct)_ici9>`aTqdsNFIyqY} zr^;S4-mB1SF@d|!hq{iaTVZ<+>_(LfMoXaV)je@BtewFp+j772eyNs&6=whgJ;`1C z`I7G}_sN)@7q_#ex6iGwt@w^uuj0mQYHrq!*F21XjDp*&M@-cowoFdik4?Yrs$x+w z=5hx1^5x5sCY?Dn^{N?0XCfp#Q&b<}HK>Ff7H{r6%%+ze4m(!|ZIv8nxexvJ%5Pmg z3Ex2Tbpxv9_iXG?%eB4LWjA`hmNTO9^sVs6`$VeVo})D>RT3lIqdj}4q}P4RmMtmx znTG|PDim4Ht5Z*h>=&v<H%j`vMA^6PJm`Im$+5xI{@c~Ce8-hCFgrDA#YBzHA-h06 zW0m&ye2BLx_lqd~m)a_AzdYKOF-kAupcKGoJi^%>$eBj+e5XgX3J6R-P4AYBC%7(2 zn6gxR8SR*&zUI1hFH9&V8!~#1vO(Ss+f2H0&m<PC=d@2wolDF;?tALS>x%amSf!uA zVwQe!+dfbiB{NM7buROy<39m6%t$CLKW7cuOB!0KxKRDAwAG6NdrG$|h-<#lIp%gW z-+Y5}P61_Rs&?sqR{P?sw@8oG<CM6b*15X6e*B_px*%%N|EwaAU#=-Z-7x14)~EnL z;PekV(-S!#Rsdm~gd@c#yigU4^|Yx7Og3mdJKPxSU%sFELUpKUH||U)!MHtxW(LxU z^TKwsN$M<v-yW~BFIn~2YFMo;TjP75(6ZSyivILga+dpX;ZXZAG_yo|ZSMs6R`0&x zAnJW64)b-a3oe{g6Xv#c`*u~0WUb;?-1iE}`eu%ayO+R0>xakv;cVF4rl{$7w=L6@ zW4JErrOij^!lV1r(ip2~-+`425emOFVmCMPoYFw+LV`x#*>j2sZ@#~zmC@7tR4jit zR)IGsoX?=)bgj)&<6fI5I}g#1N1DLg+<9BM*c}E9iJGDUW<4$>Oit){pptutE?iZ@ zUR*nw*jg<dtXecOQ1;gHy+X7s!+TyszvzM5P}7dATRGOFgPfWL(aXWJ55Ey@q`H&g z4PQ`J%?y;YzUS7N9{jXJaG~t&Ir$7D<@yV0`t=eL9!GpW5Vb6SJ8a0OkijJN)!A$; zipy}(+$Q0PIeFn>MN*uAQ46Pjb3*Xc()2+2K*7dZ;&8|Dt1phsl=*XTEN|3*yfNBr zDM(vC86I+@s3w0mNjsL|L3(=}jx7VrlN3*bx!5TCCEo)sI>PkUK<KISA283CDEi84 z&|uaQtzZ9<`~KIhq4P~9`EJ|y%<Ge2##$8kZjt!8h2zvdPoF%=e$&{K;NI?d6tiyQ z^pv+jBTD*E`v8wicc!W4(<O+PTse~MN2k`*>uA1I2Jl)gt`9AkK}lMod3uq4-@bj} zTgKy6Gs_^nofH!4Xwh(fI?QzDlV(BwozHhldnIYv)y$TZ8@8H;sylSO%|rF$D3-NX zatGMm?XR~7U!l2Ib9S?lhBGmDWBmq1!0O%(Rt8>kP6$&asGITTDrQ^gL^z3wxhNjf zw4p7q9`(B(D$+@HZdvP2tilbRxcBE2*}VoJ1RST>PV=U(*vmf1Q1|QNRG%mF$HYFk z_RnIn=Mp#+k14sQIN$SkKflpa;=2*RBl1?nZuS$8QA^Ez<C{0VcSA*f)DEN4wAb9o zUBs07(W(BL4+daAYOi{XGanGip&!T}rt%-Dz7-ms6KH)EXJeCERk3_+TUx!%gH`s% zcvVH7$74bd4)ARNw$h#*s+DkA%3J74oy$|nGzqqv=<4h<jOtyH6J5K*{jfr>f_K-@ zXf=3{<I9~=&txYVIlgTDeS>@k26=G$L-r{YgB0u7iC8&~)TWbDTH?bJWj)$N6AJ%S z4jmpTKdqLPbdXKyW{A_Werc#^fzVvTZTSykTk!L(y2jh*c|Ns5Y^i5Y%MUtZ;7PeX z`uh9h90jW>`tJkzO)<v?;QzV#+TSG@LtxdT9WiQ6X-#lknKxEW)AP}DY5lw_71z(e zqvtL6Mcvw@@$=gvJWqpu<67HD4;cT2%VEh3cK#6Yit&oZHpJaDG>570WA~T}QWrOU zG00OcvG4u*WV%j<L)qm0&F-ifVl|!fyFw$qtO5v^8W+?dNS#%P8%mNbsIEG;{AtJX zwcR31{RfnDtYS!cDj7!l4szCdwkqNcRpJ{;ZtJy_lql<i5+M*@c_Ikb^l_@LF3K*r zY9IgLDob=RZ)oIsC-mSOm46^^sOA!Q#X%d_orlgo!yRvujlj<gTE}bvh+(nKHEQKf zoNwy+jANnRwJSB<1XASbHp4`L3So!Eq<t?MXtKIIz%!IQ*+t7n@s-Q)ExURf)j{3L zfJV|`KsawI1NrHCOD8AIREw9!Qmgwb`McA2`3>s+#+Noq_24XP$#Ocm0Ihj3_^PJq z9eaEB?rW^gcu4@Sq4q>qp5FkV&ST3VCmL#AwFA1dY+WyOrO3$9Egs08^V(JsBv60< zB{*)G))busjW$wyW&rw10|>t%;7(0<3Igy<fHYQn)Sl6Dh!8&EF_Y!AmNR89K`5}t z&Jv<KJDk^-bp5*zsKf$iUk4XtnwE1j{Q)954T!|+-5azZFZrH)B7Y^ps3qykz%(YP zk-hjRx3<}mQHl34^SW&p2PLdW8un2i9|X6)T&&<K8p|3zI@+Ab0q30Oqh4`60~U8= z%OK!zf@4qW!^-`_;ti9htm4D2kp$J^l#+)`@_;q*yAjA&3Ne42&m+!|SRKmo{(>%_ zaQEADijfd`O~i8)Xjj%2y7TNx=a#TfJ>fPngSIU#aY=#;r1oZu>XF7+x?={UXAG7> z!uATYc|uOBS><iLMNehgGmP_co{F-LVtI9|=%2oB`_lf|ElYG^{*Fv{_O@MzFJ7;= zNwA&z$u4tXnh^Q)y|8`8P%00drsLe(eUb~fTmTCnyA4wp)`1D!ORc-Y9iH=b)Mlw( zPSpC$C9+o9yG_5HGxTUe&agfgi9tCBVml8kjwTjdA&dujp1J)|vdEqKhG;!|A!aPS z-LYA*iA_1>NU%JFQQbFGG&C6uPU_hfQExajyDUZ{u6dOqE;c|x96KdJia(t51FljA zs2y~%b|}k;Z+(f#q?~J;eEl8g7w%{D2M=nu@hTD04HlaeZA>Xj+5|G>LPcnrj|khP z=@mba>jU&<f;}|K0IV22Lk%t=)1*_7MUN!Z<-FloJE?b>+R<d$LF<)#l=Slu{;;Pa z=ZGc+-Od|%2ietH2K06;sn-<^P51+3$qAAv>VPxw=1z<}{yTo6OBg@+UiW(Q#exNe z35A9SYk4!vqlt7I_^F>)YuTrKe0*xt(+SDzm&n)F_RkP+9hkk~n0XW}NTI*xZ3cnu zKvhUEDxh+8p&SM0PqH{bI^xh%xTW0e6Fb$eUB2mt&1DGa1U0h>VflyqXSHk#iQ=E& zwaZ}eQzYOdlXCqdR^T(jFG*ngo-#S@B?TEAK4DZBDP`4?rkwilhj1q~=~es38+1~^ zg6YNa)&*Xo>q>qcnkIraV1D81pCTkAC7%r)t*9C{Pu<`@cJ^7!{mPRvekHm1nG??I ztJ4sqq#?Ob%VFnG%eqNPLqBx0OPy1A@Ec?*<**+&yyknKGD-WhtFw8nftg4{dUMrn zysBt?9d{!)Shp#gKiNvG$)vF@U2w1^(=@cdGO&3d;2(Rj@GZ{JDp4aZ5)!d4k;5>H zJqR}B`>ahXJm!%#OX`Eoamw}6_L1Ebj>J%h>Cz!yB)&pQj5x%8SR08X4FapGwOP}X zF88&(e9m3JjWxsy<H@#YYwSgF<6E{qIQHS#9cnj0FF3vBt4lL^pW_%z)3qj>HJ#66 zabuGP@zJFel)Jrcy70wR)Q8V0R9M(Ge!i<SSn)ob;d<WD@vnKew#btFg!Jo9l&3(l z=K~1CJ~e*d!v@(H%&avdriPyD!Q-huI(`$$Ji9rMn#{%;EL@e-NOj+!c@RpM?NpZx zyjOu{u1nLut>2NAoUG>Ni;46UNINhOj+@@;;o*#QOS!o?)y#we<vC@Mm9bR$2nlNZ z&*!Y+M3|OXNu^B~rHK#2Q{P~1NLsEM3Wj<@(9L4*`9|h7Ae$JfCc#sbYIz9{S|mW) zFnm8w@Wm(-ukF-#+B}@YLZ^)moOJflv~yM!vTXocy>CTGyl>h7UNO>=SjD)I4{m-d zC1uW<7FpZ<oSR{n84W=Oh`O9;oAOfv7`Kird9>t|`3aTY2L-qED)FMv81gwCJsWk> z=UnFvr}_aEMXa~`)2HQ-UiE`hK1{vDDC*Q<fh8H_f-~r|q-GAZ-qIw{c1Ga#$Lsm@ zs&P4&z=x=hfY;x3n9ci=8|C|^rkfFzj~$P;sn0>U<azX!>sm!ZjzVwQr=_+pwiWNV zUY}Rx^SYKaaxiw6xVZR9K!LTqr{qG|kl^?!+ji>h9jWGjY5^Y9%cgR%DqCmrVuQMF zQ=}jPh|4S_^oSs}q3#F>IJ4BQu65h?QG2bM3xr*Ld6>PM07VdIzwYOcm)T=nouVu9 zXWD#(Y{ok@xRf1j(w{jcS`3sOQm;O{7}r_eo^GgXsA<!g7qJtd=bcB79vL-i?MLy_ zo1m3NO;Gx7+=zLG@^kM;ThnI7A=ay#(6@iky>jL8zQGH-ju;ecrjijM;1Br9iTJ>O z14Uc`ZkMun2|YL}mT|yiZdR|7{5`}@o=3$BTeMTFzJ$!1uXlWo9rt0m)(@yWJUBSs z-QBVAym4DvMh5Sw6Mgz5q|r3iV5Apn$#4~6$4p7<2E1t$iMXGhFT*;0&MCwlp^Y+b zP8*?*>wXUbw{@p$dsjvI^M}1EL|sg3<Xtj>gaR2mca?O5qs}gc_@U~$XgSlBL0C2) zLF<tty?ElSNXdP*g|-b@X1$M)G(;!FsbJRT{^vU}LgS<CWAv8EO=^HXi|t_0#_yV` zYt}wy7kd7LJ6BQuMDiH9LZ{l?G9X?*7Z<Y7lW~s%+78_zxCMC1__Iqv+In5IJ<B}e z&FNcTxm_H7e%n#{`sB-MVk21^?iP*_vV=Dpd3MT_-re>S!b>2Y7@f(_K@R0xAWS-< z+;NGG_xhjhW4S(RGts4Pw=iye?qbX~Mj>0~zmT@l1<0e>J1x^y=ODjhm|y?-lEVX| zARcOJW@Z*ARB_m5wCTq3+$eD&C;#A`8Ne$!Z08jdf~FEyiJ}`*QATmiJcdoz!9@FS zUc-7IgJIJ${$A%Jv9jL)+`M&Inr_Xg8SjG#z#W`etpz@=HdNoGbByr0U^{Qz7&s<# zaf9XrV=L3l3r@?!tPuo$ir-z_En9zrby}<jrm~<Iv>5mP8rknUWpB9hZXuN7ct>^y zc2tn|pFtuD2jhpX6WFiaa9AFWzERzmK#`(s+tgmHeMq)}rf_bwIdeC&p;#v5y<H}> z!kXBu{MkAg_OKtH*c;k<ZDBCv$2pjvXoz))+ZHEyepDAj70j*kqDTZv>Tdwz39cHg zE>4ACQcS%+C!)`}TFz1MW(x)7J6zt}<Arg;1m=qdK|Msc@g<3P|Neawg8WK^TM~yZ z;*TFceteOwGutv+$uYWuyku%hy^dn!LTA!_w{l+I*)DMCs?E)W5jZ64ms}y?y6P-s zKW|h^)5G;NKM<2O&tuY^Z)`Q+<dW9OWjCv*UmO0UbGSkr#RNC8WKo1bWjPgv=6B^k ziV>@Vbs$WC^q*)+j@XUiDKR`k-DptL5_e!cQpz{dyzj?H-t<(xd-pz+>q4CMn<&`m zTEwe8KE<Jtdz#m<=^&|XOsFYQlLLT4+HPhrLuCxFL4W$KhDX&RE7y7Y$Ao}Y)!AAo zx;(^!WPJK-f9DUv<7<<>;#zyCuc&w~%@AL@S82UM5D7R9;$*5T#db_}lMx{A%geLF zebYX;+@`7$F#fmjq0a*JpEcLlaE@h}_wA31NFvN+kLJdXHiW@hd(EycDAzpOPDBWh zVUs)@Nd*sK?8buRr-|2$5`@FCVKX(lz&PScg9K7lY1b&WFXz?Q5E{*BZW|UljkP4x z)X&*ZtiHlF6Kp0ztVnHlGfX-Sa^5yIOoUrrRCe}1ubNR|TJ3)s9p0gFX!j7$>v-<r zanv84{qQAwkH})7d^BW((+Gm93K9_Bxjs`ZDsfYZ@GeBqy3A@lv)kz$gbd!bF+f&+ z3*FA$1F4X&mnPjW@(2xla%zqUwCU@g-(T7bMa(>gd|3vH#Mj+CfB?Hd{<sal0J5mp zPHW5A4gVy#M;&>`R!dn!l_(EJkbkk`pcW0FCUt!d)3jL-0Wxx*NXAY%gHG^fOM`)? zYB^^N&6LDzxSkSdnjVJ93->|VauVWdtqxC3wsAWj!otxD+uZ>lqV4d}^;VwD_ZNpe zX^N_a7lI;TIGADkq6(+&<h@#=HgCh-Ze8l^j}VNjyrROai4(yhqHEc$$PR^EbsGY2 z$Rq5JTWga29wQ&cy{6X9YMp-IL9-2;VuG3k19ht5qVQmiN%`_hnE}E8Bh)s_JtMwZ zLQJUY+Bgu62(sj_3KgdN?X1VeB?2H|7ZoaId2q&Txb9@k&9F!9XEDbcy)7VZ_Lwva z;L-P3LO>{x4&^Wi*^_J(vdhk<TCv2c<ei84fGDDh6!e12kjjBUez4l|nI#D)rIGKD zEj0D*K*kkd?2&qu`HmKe^G#SoB|*)6haX<-YFW(i{@n4x+9iASyYutn9j)sC-bR5F z4rd6h{DD+C0B~}O)X^k@Pc&CmHRj1$S)Hkt0ZQvi=&)(Q+QY}k0j&s!$Y2W%B%AP` zszH3m;1pLai7gHAgpV!#;Yr6Jl2X4-p__&Jq;84#mp<hK^;?BAMXnLC*w9<uQ3~-Y zQji9FBxSbRPWHG?#c^onv+<jBa0-%*65@K53CE05C#s`Eoeai|`Sy<xB6Vv;6+RT~ zsmOUXN^Emf-{2I3@@<4y0w3nEMvC@xQnMJ=qW~=FheIk&3C3;%+~$=~pP9u0e@zeX zv__`eHQj-5e0kT!T)V_6I)RO`b;Esq`1>0uwhEr^m8K~U%{fYbBoE19F3!!MoCEGf z`>~h!<;%JITfq(IPJZ3>(a;Ha2|OvinN9TN&f__O(f7Z?IAM&dL2AuM_g4#Jty+(x z=0fi--rB=u*u(~S^HmWZE-|`iHO@=dc2tQX3q!_7huPZ1{tYMve_#|<b6^Y0sPv1+ z-eJLQhvu-g<1ZRSslTcG5hP$4j+MLSm~hFIWMRyQ;WuGxGjGKYIi^D4jd)SYqzXug zs}y}-zI@53(g?q@BL>3Am2Izh3lk^Y2_LfZ6(4qH2J`dSi*`G$7%wgMdF8EFeV+p| zMtj~#fGJJV=Xm%;5Tzev5IW@-2TzbX64a}BYx00S7`o?NA%J<^%$!Xb4k-RK1U_GI z>KNw_g$8=+YYh7v)s=~JKvhIVWLMR*S77A-X#ID<`imOkP}!M5^0x1H-Cmh<@uaPO zr`3E$xfbV+Q)*&PoB5=Xms0^XjNan&Drw$O8&GYktI<q1X!M{-0Z7M{Lu*)5uRh8D zkc*m&U%&2KRQ`C3J%P@ym>6PTMJ1$>ZSlQDk+4tI8v9CFrl=4y8><~$hBo&)7a&)5 zau!nP^s;6leOQGuCSfe**yOul{fJXg^^?Vowe)g}KBQGC&}80suz_<b{OU{Xnh%%E zamfAf?{LGO*<s3L3KZ4jf;_AZHd58`6V|{;-XMh7?}}mYo%7oDZtd3N`$g{0^8=yo z<^#nJ!uj|whH~9^*Bs{?6*Z**<1rZMFKUP{R%t;LlP_>`G)?^Gfa+t=;`82TFnPCM zsVDg$Dvr{14aV9?L+-w|GvW)uOJrFRN(j2j3n0A>mC5Y=RC?glb<WE#xQc=0h=gp% zNjUm&{k>k$o#|WM%)nskaI{0Sb2sgZA)AwO-+lDDjNUUSjWn5H2kZ!$T??RHM51YZ zR9hqe>C+_x7gEP-1f?jlAbB!-pLnbVepL#5V6O0k*8v;ghv_=qGflh4n=MBz>ukDw zL-R$!$jn+*-9|A#`00=|e?~8}mt8%l1mvlXe4|jnnjLGcr<(Ah7pRMph2oG76c~PJ zRE6C)<!=-<`Ow=3TN1$D_^Q6DTGYVVbm4&r3OYndXOK9%$PQ$o4OU^ZpxbGj?)zDp z^_FP~kJV?_J2vWG>0Mxb1E5Vy8XJ$@QhJRM|4qj*c09K$h-*-dS{gjP=FWqsdor$Q zYexcHW@{3;6{z>&R$lr6W5_Hz#vNw{Zwxmlh9Vqfv!P&IebN#P-ac>mPY!w-pKURq z4Y-+IR5QygY&ZSInlueSu56++<vROs`5<K!aMWvct%`|_(Q-_~U3r9JIu&cKX7;AG z`yB=+8f?yt2q72<xfj>Hxl>(*aL6!o`cpR-6i#^s{hzwK=djhAWxZk)wkrXaL0Bc= z1{D<*XHGJ)>}|Ql8srMc0Y6(VtwN?}(!6BFeE7HPfG#dk2uDJLYP>(idSFXa?B)!X zAoIxG*=7*k<ab^b!|)O`0Vbo`@FibEy7s=R(ImX>R+B-TW`V}38v!R&ikidh3l_TU z!_}opufYA60?VOO8-5k%pH)#m8HuB9at8c8|G5f)%>c+H_EFz;2Q5cJBXJ5S?lLCl z)nibCVxJM&Shd8@H7a(Fcjn?PYbPUTZrT$~QotPjfVPpbwawD6gcvFVjBIX>p?pYO zXB5y>XEBm|GYeLkP>g%-w0#T!_bU#CxOXzjj(ZS+K&!ecEwbi=_xX+6v>(UMF^)?> zvB7ge(z*JgvrXFVEZjA(;rInn^Pi7zP0tS3*CpK-_p-k@p$YZIqoPi^+Q7}%Yxx4j zTmmqS9$}Z|rUxrW&CkP4U%w0C3q^r1T<eoOgu3!<+tLoC;EpvVjRMT@eDxj5Pu!1e zfu6sxu3vK3%?qlEGb$E9?Mvvj8bOs-ChYQ)jwS~P>0@1as!(Ic!ryrnbEdfO)5H%P z!Y>W@N5M{g?0ieo_zCVr!tD0K24GSlcMPwj3Xn=HX7L&!f{g6~Per_-RC^>$lUjXb z&4O5EyjuQ_E8^B=w}Q6pM*J$orErFKot`jIh3=9Z5ve)U_hgAY1-ePquYsl%^BGe6 z_~6m(irRz<R=JRa)UKKY+zmz%2fd)`dSQpQ3b~iF9+=nN1<nOias@KCs?Qnhax}rT zaX>xpPJ&wNkWjqn3=Sw7aoE0rl<P$`-~si**VwNzjDfD@2a{+`%{&zFF$@nCef>4; zq(GXYeFRi|Y6VW}O>xRv4Ye-okW$k>nlJWZ@`nOy4)w&f;A<N<3%!qt3sKH#eI1!$ z<h<tsr1rPPWH64O3v<B>^9=X?zzIFuD>ir@b*q`5@6x-O1erE_=4r^p<5beiM%(ol zZx)4Mv>RlrFgIm^rQP?Y%7^7C)ZS`1zGxENhD9(nP4~tC+<Q9Ch2r!YljeaUFb+C) zOaj2aHiUB;)$LoimOu(+%y12R(iyVhT1(#Dg2rt;MfFB{Hf<mhP;O$Ek9<~pOra<; z#9c(3x^SW+yPhh_z_tNz^*2ipb0>U7ZQ|px5wPhMQfEy-TdgWob!zse;#M1pMB^Ue zuzZ=RsSl8cv+SZ}e*@$+M%^bOhy9v0wKI_C9-?<{5^%<$Zbi$*hj?s(7|`llp!pLr z0A~AHtSYA-Ky~-#$CK`TAwLQwLm6Z0>lvVhke$D-f0crQA_7|~(b?%=GY&bXr&v1W z!%k}I2ZlEKa^lr@^BwRJGp68MJok=^ZmeoqeQd<Er7OCn-Riv=YX_yP3*?m^8yp{k zljrcl^DSBuWTCwpJz%as>^%>e?nHeOv=p!uMh96;+rH``M+w@@+jc2!^G>(aOHj*} zPx5e1Ncd8Q;FzcuaB!flI=kNbEwT*w^eD*n%oRi*-UrU$YsD$F58C84vDd&;1!*%L z+Kxt4i505#AoK+&G%h+QL7^N9p=Y$6_Fi;cPRm`LE@yvx`SN8ClON^+JXgA+WN4p1 zd-g2V*a9R<GlP}BP)+&X%6L%XcYPP~dy*>PzkHi!ex{*5a2C}&Yq2IVeTe+E({5sB zW=UJOZ4<b&bNB8yKuj!!<q-EHFm)e`#J5r>K!vZlw0=l}cNSuL{0BuB00d_clKBO` zK@AN|F=y|*A|WA7z>)d^HU2on%Z9ZYs_D1uv@P&gE+qUzKeb^ijHps-W0b1<MoWBI zy=p6!vD%8CZe9wp{kcO^CYc6h$2VOn=A%%cko~d8KNc+pN7B2lEKHQ|5nW{l4aZwh zM@a+JlCK)*Tl|O_48htY6tM$JeFIVS+f>!y0L$^?+5k7Fp&pA!F|JY!DMXm{ex2^H ztd>Hu)tcn>KVjpL@FWNOCcI-x0NwH<^|p{sGtkqI-Agj>lQ?rL{7Sf4t(b+9(mS<$ zht`X7;9*Zf(dae5sRoO86{-u|bZH7zp#<6Wf@_n~u>2b_2y<7&^SQ1;!Acw0lB)t{ z%BbOSlQ8T0h6N@B4+t5Wyg)~f8avhU?C!#O97y^s=1QJo*v!sxj*H<>_Ls~x68Vr} zp>wjEe2CNDEMS=EvAF9_D<uBXK>kFip{OV{;;At_W{bH%)^LNOSv2VJe$<n*X=;2z z7MAb%Ot#rs^fh@#;Tr0^u533yb|x$=thNzi$62Un_yYDMjAiMRA0F<=PA#gopBvF> zxVP!2p#H#HZ5tIcYcp&4CPJUWOFa;r<ijp+d+y<3<ueT8;#rvAKR`s}byI!C*j3;$ zMEyZ8c1Rul;Wx9S*HFUi*Iv=fK7wwq-ww7wPh6TwXWM=q(1_J#7^^<59&L(0ckI|P z8d22(r@S|8%KO#+;~$!$^U`X(qxF0lfQm?vz5ED*e@X$$4s&lnb}$V%$?YY&5pJ^U zVkw8f!5cB1r$u8r#K>IVkr%8^zifzAtY(o0w;BaCq;LkG${&FAPTu`|hjEni4{CF8 zf?Oldf+5F<2vm8ZKd8no$YpLGW>*`U%lWhol!|^p(k|wPJlW(MpuY>%iEUuFQoU!) zCD3<a@?{|)4Mb^?qE$>2Ix;t0U#wKUnL(f*eP#i~>sHD66VSXavy02iLO`rKkc)D| z#jaqav(kFq`HqMiOKjpi`iBnS6tZr@TveNFu+zR8lM={blH)Y;+7$9Wd|*BbBcbB6 zucJ{Yn}1K{Gst%;lk-YN6&t&V#ccyR?<CZgEmWf+D5U7upDKNG=62jd#3nm|dbS~k zK39hr`oQ^e&>r-K_^HOs!*d4;O|08#=s7z_p+)U2r0_qM1X?zA_NYH_{l*MS$0f9_ z3Q6Tr9_RUG=8G4V5EUhy-ye9$%bAh7^!}65Ctl2$qtm$P76G6C6M<|Fr&G71eiYS- zqpqMbDFJ@xB<OtQKi@e63JUAj0$6l}h5Ch%^(tgSGPy8GYIO77e1kW6$Jfxx+^;<i zY##b^qbwWRQvTO#>!64p+jG_6kN@z$-r`^X&-?mT2nN2}OL;e4`9J%?<Wry^oKMUD zK_>C<J^&<oU}W!wBokcGjqG2)%qDjDufKBbDChJ)wE!+IqVVNc#C<UT`WwITCY$}r zi38vqOZzWbpj*#tLLumT`=R<f4f1+vAUqIx{_*3-`+;qJli6h_W6@u7|78ypj{Ko= zQ;t+`z?ivlV@hXKGveaY|J3K4qynF_L+(8mm0$CIybWL(3kcWKk@T60UeyzL9O=#+ zYvHam05%p-PaVPoG4ckXFaawHC&1;Ef3#F*F2SLdwk_+}hvti1Pn<FvZ8{G`$V)a5 zD<M^>(OIPF0OT<EAXOkTc2HAWzvDOmc}2f2@_BLkA1DEk&tv`{zW>&(z+{{jU7Pi- zCD}p^ZvWAv!;LYBb{9I^CbK?5QFQ;|!-JfOkdY&i#>?>iW;>658pWiM_J#c++%{Lx z+ynEzU20iom!O;AT)97o&G7ZH%4DqrVhNy_t_a392a?`x5|cLBgZ}|^Fq`ih>&U+S zu;Se@2+woNA257P`%-15|5lqb8g8ScM2xaXi1Z>?ulD$V{_F0b7qGa3!~;m17MAY- zx%8ILxJ?oQ`2fiN-hu|89BHEf_SM(Lg-mr08pT+EI;62o?*a{#QM~RSC~1%#DMdn| zh4a@A)KKVp<@nzH`;S8<$RAi;JI8QfL(7oD9VkJj?c!(1Q^_nsgzyeKuM{uCY<099 z-1If(eNgC<eZ1!h3o3;1R@?dtT2JWsj5seB1HRW#KL=v3>G^ikm+oEwHp~;7asolp z%7q~C_eh3uMkVc2kf^f;%tSi}dH`RL5NCsMMh)mVZrv(2FQMsYz+~R#)+svymh1}> zW@+HNS8;qhQ6FGaJN7<;!~^%}!lS5OS@I$I9!4Q=L`(?jZf?FZ6n`T@Z6u#*qAnFG zlGi-idERz$GVCCSMhB0}Sh)m~(_)y_aNT$Rs5sOfrOj8c7FoeGFLyW8FW_yp&5up_ zIHqduA<_^BM7plk;rV7HS%XHxQbcYBG&RyB(3-6Ew4h7|szOKyqEqa(V}BOf_q#y^ z|2BE>A_c1NE4_ZNr_eRUtoO24=1wXqZ}4K0fIRwrt}#ws?CyLFhOPb))b^mNjs`vG z)vrZGC&A?_@Cu(dssHHdafIt@J&(w>P1gbK2h37X#*>xzdRc_-*m<@J+KjV7z02Ib z1yrS_NN*Hu)Qh<&MDwE$`h6f$e&qNClOu8waC5MAjwcM8;&6VP+tEDrrmwl4K=S8O zp|doo`LkHvz9T}mNl59C<O=V926n+0q-Dj+4?qq3HYq7dso9*}`=rd_7^UR!H|JFq zkfuz~QojKar%aC3@M^xI8tv*|a^@==pxTzu9ol`ATN0SL@C~S|pGoi%bNrL-bUlH< ziRVJ6U!irBxroW{Z^^}{=FU`M8NJk?u6q<Ft-b*bf1g_H4t~?FH^2a{<cbV_g>+2{ zX#0Wpx*<Mr<|Ki!KLO%(r<mZIR}p_M<HzO$g(`yZBaxSf&#OqTEzg~V6TZW`LpyT< z(du<%TS^Q;)eYXnH#)>ly6)qRz_IbNC#a_m)%P$jA<<o=`gI^`@bT9BE4o1<`*xt* zzg_S)@N$u$=nvm=&$JlY;jR>=CNm%<QBY-yWatPb6xc5$&<ok#tO^z7fVvB};C@5e zA=VJvuR{_+SQ`_Tn<Ounkw@EhA>~)b>VBd7{Utti-~>)E<<BqwbW$;c7Pci&Ve0^> z@C(j1Z2Wu@dTpmeH&(BKzPT?t_>9{hAh<;sACclrAM*(PZ)C|A#yfohq?Cf1!?<0K zRhtw9zR<XRiyAey0EA<Hm#RNiBB30?6W-o?>t}*4nqRh(Rr7?3{0;E*r!YHTfpaBz zhKN&qUY>wbFs~rd4$DKf0!3(h0-K!^=&3;HehDcC-S$Kp3Sk|ucDq5tJKe_5y5?x{ zWqd8{u})Vmo*Np(+Cj0GV6^qEFTvAz?<v%SiRyL5!V!>gnvvou`ZnWt3fWGIff4mv z({>5rrzJr}-w$ol{5cf%7$g(`S%VbMitdo&R-qR~3g`sH)_&+ZF3<WLI;W8JMhN<( zNHwX^Gv8L#usm`BJW|kUOR7v6$~Krnq;ZFVs!O>mP5`Z5M{ARsjW5qZP)#evC?<v! zt6(lRy=sQD>w`!ps+w00p1mWPgCiMiE%q(1vAIdn4fVR|Wob-qsHk)PzPdk9BS5Zt z3CUG$cLt%79j@B!W+1Iy0=g-Wrq_T>0ed_X(qcg|qpML{S`e%XbZaE^D#<6P36uu# z#+J__Qh3nS4^3M-Z}(2Wkm_!pPG@rE%9Xg#B%CP(^AtU$#?FiO^Kp8UZJ`n3Pic@| z$a0r?C?R;>%gkJ87;k}HULf8AO~Aqcw4Rj!QS0g4h6jh*F^!Zn!I%EbOejKHWsxqk z(UoE$BWPoTYH3%rNwYbn<o)0GN0P~qY~CJ=ApS%}bl(bGDDR}Rq)<NL-91uCwR7id zcuW~+v&~Z;<hB7<DzTtWG+&&&k6)cC*1C%|NOpIDoc=U0<kK>F@2;+|bZ=lf!;m3I zDymo}!|mEO5tpCroNm$bLZk%!F!oB~;sP*=Ce3=Z!$&VXl>FgwlwokV8Y$M*3q!32 zak10=zH-ZHT?Rhx1z>pwLFagQ8mt1`+p*RoC!md$^YUmu4JoHHQtE*Jczb>W9n@5z z@LutU0e=<ftJcJzgXnbG7A!eNq-i`87|7`y!?6#S+&q|5?vgdvuY>;24=Ra->+RsT z^=R8#UIDcw4Y$7KosGB;zI?`R_m*pi8ad~o?GlNB7A-`;1@ho3%Jb6cQr&ex(KE<R zgeq#dZu8!_MreuggEczJWN34rbRy4T$s%#wfDiu<HnLr*x%MyKZa1Tl5|XaOYCy%s z^MgfOn!a(Rm%uOxf#2|V*%op4$63BSc;-0Lo4hY*W2%_RlPv%4Vm^e89~b_JuH6BZ zNrA9XvK3U?$otW@w3Tsa>p}>2$k)u<_aku?6roZOB8?KC2iW$P>_gfS%5#u@U8pkT zCeT{Cx?4bi(kBp7-A&T9%%S_=lXD7O*;Oo8qH3lrLK4oiwdE-i*ox_)>-I4945$v5 zjaZL3g2e7978j?Vq=Y>O1-coDMJ4QZXxFVJ9pzPucO3w+nM#P=$eF%wNMbqKGfi!j zqt%mv!W~U52+@a#flMJtG+f*-QUL7;JjQJe#(EA=AN~>auJ@rMkyktJk{vNj1IfRZ zSJ_Y<A{roq*H&SnTD!v=HZh=31aLomtU=4BbkEQcJb^ZdEF_5cOp$&X{Tgd<;_|5v zZ<tDFSLESmJ|g&4!4nE_M}mXmCBVBPHSEFaP*Z_xqKP2nm?eV3^i7`qLQVw-wAIHc zr%FI`jqa$CpcHmTBKQKT7I$}d#6Nl5vjSOBr@I$2LZB^dDZN?4_v@2ggt2J##L67D z(GEy1kFMBad3LYDFqBew%3O7WitaRhUlQ?fx~qFIFOM-_gBV5sc-J=STBH}4ZR77D z86ur-S5X}YDj;xiDL~i`nb(5of73(jT2dkzkva}!U_Z!;Obid7zA2^vy@uJA#EiVm zyJX64`2bxq)r*aOf=Xf_n;2rLDo*Gy@b7{mOl#XHU^WSV?Npjt%-5zGWAHE@DxfdD zK*}HU8n3$b)S5Pz8Y+h1P+wA-#k=E7x^k>zhl{~I@jfc=+D+|fc+HSndPVR7rMH`# zTLeWN5XpYXX^N_!2U`3u$wRxow@OiQ8>!U@f@0y2ym3>EayQaGvt&ozgqkRz##}OT zrvpHv@k?)#9k~>2Lyq@jQz+T7$<UsWzmmT;uL?!FxACO(dJ8v7YPG@T2#S=$#!YeW zhq5c(DJ>O&y@kXv#k%Q}6jBU@3@m^$1?jV;rsIRo3yc_!5sJs$g0w4Ie-BA8Q27k1 zE(tSGuB7>?0`DRd;=GP8ujb8-t*%q|eD{(GgcepJU7&z$AJx~@qvnfU0V<{+6xf_> z(5=WiQV;E!NT*n`4fJ<H$hwC}ZEdKIK`TKeRcW9nv|2pF1vR<gA;o4Gw}4)?_k6|y zdL^w!X?nGnCA=6(Ue~UolU3+Ilo`fKdTV_zet;h2o*8bTW-=^+uIM~J)p!sPArml0 zPs8v<F2O-TFUUonyI<47#(j;!XgAh;&g|{`G_$%y43%hBr^Xc1LxJBHw0g)GmI2V5 z;so+3-C3`{+uLq<^ox2t6L2x|vEZB~ibp=lRwxAO6SYV#V<ym@cnc2NgVEd^B8;oh z7Eq+9#ISKFC4DG{CXCJlBwKn6W`x@Y@?v=q81xZ=D1HN-VVzpS!5^xHOuKSl3)|zC zAj!CHSj=|t(5cM3K~m^v1l5?t=n+w?xSBMoi*|8w-pWHS2r2wgev=^mFQ8#T!jrM_ ze9<1!5<1h|UTzx*yYh(|!m7L-H&S*ogQl-&D2*<#HqYM1-{eQz7CVp+K4=V9nh*r; zrUfpKP2bsNph$#PT3JT>F)lBAsKzVfHau10=0MNO0$|QURgS=9=-&%0K|Q{$brDe$ z0QX`m<S)yKwD*`n+999(;A*XKdVMNRZhiq7z{94LR5<jl;!m9JeiguPmqoYCZ<4>Z zlZ(6W=DHkJC61j;P)h!)5Dbt?5@M$MVB2smaGT0YfG)IQn9^%jImY);F-IJ4nBQG8 z{NEkpt%4Dakx;BChr%O+2>ahE9IAPP!e)G$L8G+D+HAcNDfVSL7Mdfp)JKN8>?q~5 zpj)CY+frAq-lH1c`yA4A+@7rv9B1u=d(kA8jbqNhQc`2Ix4OW|4G)EQWu6(`=~yv1 zEHD>b21m@OEa~v3Iq$DOn1oBjv<q&@Q~mlQWHn(7)Ol#F^@BD+)fb1)pH<JX@&UlH zLOlq`1({!gusok)I!&76Rdo$RMb{U_*JnaEvML(QEaCAxn}!-<e2qGDc<p2X+avh~ zjbR*l$*p4ws!w^|&q4u53WAxs2r{B#Hyz(~Bmr&HX~pZE#>3p7d=t971dwT@*i2|V z74&75;}aye#~>~Wnm{C>i!3^(Hq`K@5a{;(X3|;abweE=E@$H&H&xZ(klKAiylS&m zj=P&&(8c=>d_aRDjm_mkq{205EnGf03tIT3>%oGJCreBg#{Q)1W$Vw&U_EM|<WVO* zxgI<XWw32AcVb?bR{|?&&2k#%%CsrPj)6e0%VFk*zaZu(A{aJqECA9+Xr|%<yCrXK zJ9xx$&}(-fWu`2Lb+CL%pspjFO%91Qr4pf;wdrUu>+pM|u^ti^3+2RMd&qT8gT-7B z+>Z5vk0=4bnL``@;Bo39gPZ(=HG7kAIyinFj|5$_@s6=!F*{A_R8?x~AP{+W&h`Nn z_R{?kX*Y>JzPdu7fiYcSR8N=z(fseY1p~2eBl8-pRB5QG|LPL6J&XhW=iCy$KBiSx zeGolRqo=&dP*J8=z0|acmpL{G&5=hKyecIiZ*`dbvORobbhKu?45&>$b&VV=eQ>^> zq`HTfpsL=EVK9mR0p*1#(5<jD0gkH%7dSr@YUr&@*oWN{A8&7q1*Q=$)~ozlj8TIt zbS6$f+fWKfx>zbkger1)?CmLoHDeXd3s{f5DUz2<=TOhl7P>&RzIB!7VnUdDY}giU z>f?db@#-KWC{IT?)`yh=%1(@j51&G~5lA+&1}@r8eh>l&VDb0x8?`~F+1hK!!F?KP z=x<CdX?Rh+drzm#Q``u<a!TZ#c$LOQ_HPcR4){}lL?-@}M(nIqo)+zRx*k0B;Zh4E zR)oBC+};Fe8>p4ygV|(=XGKbgvl!B>Jya<mN;&rf*&tzi+pR!!byD2wS-p|l-=Ny) zCBIS2O_21CaB7N&0ppVr#|157{w^t$E^uyolYXe{YEVlVDz#N1>}ai2re=YYUp-{+ zhFjgN-v!G#Yd|+-Yld-P@cGUvu2Gk`U8iJ&PD3ZFxsp97mX)C&#e)Icl#-ehBIDM! zY=pEGl;`o*#P`m<n~-YIPQ7VWaTy2*zRN|BgXAL9KP=`(?jp^1y~%L!y-smKKv>Y_ zHDk1^297N!eh#|Cq=8|(KG8F^_m2uI`^8{6$v>o?MCU_q^?)O;gA~oS0JNx{+AxMj zgVY`-zTN+3g8TDyEkuW{W}vUv$kr3L5DajnXch7Vx-FxCKI<=d9=#yJLl(@XCE;HU zS2c4W)fvd}zYl$&V`MbJCjveev=VBv)GNN1It5_Ien5*)xmwtI<W9XHMy#nSACCTB z>Jd;q)=#gih+&x$p*xKc81;xQfTubzi_A~AK{xLHAjl@M6+0l?`|#m|Qg>yXOa2~+ zM_iYu1!rX+GQ@$f<|G^m58n$n(X?orBq4hxPeTQ8$I0q^RQzy4d<#jN0F<09K^bJ` zR|rgjPw<@#KLH860p=p>dMmV_>iG}8>x1x6_xbKQ<aCugP**9y%&h?;<{2YE2-WSR zt-+$sPArpg_D~V%qMA_I=C5_wel$MQW6~Tl(FkWDow9>0k%oK_c#Ovm1>QgObw1;v zEBQ8UfIzQdu+GycM9=_}tg2IjL`K0eNJ09@FzA8dl?H^-OX3o$nt&-#L-zZ5LB)3* z*g~9gG!f|@D@vVI3WA&AP=OGd-x51c3+z@2)81m##EkjOj#bEJ!h2gFK>k=t=3R`v z)C~-qeO~k99S5|3etXV73I@ai!vvn|NzTKjNo?<c3_qX=kr>CR6PnVrrlD>W1ymqX ztD9>TI)4Q88T!qdHetGXvA8p@NJH(Ji9m)|5V>KFmxR(qK%?xuX)BfhQU?I^WyB?_ zKL^U>EW~|3AeWEKjCbXo!yN8{(^yU9R?ULxXgjE=o~;i-hKH!4>V;u!01G51^P}sV z@fE1>c)vV?mPS(f(GSke9)JVWfO}uy`qS$xn`s}3I4&b>-wE&o`E356*O7z{SVtAQ zbV)3SA`D(g`2ihmYZ&Rqs*1KbCGp&cMe5u8|E}R`S!=4tZL!Qt<*4R8eOi)o593(I z?McX3J%~9VAIsEjRKa|Eg@J0$-n<B$_<V5<*)m{LdTD*J<tMZnoOuMw2uMs$02x19 zJm2s~s89{Q=Oa;a8dNJ0n?AS*<bVz!n-ri|`kb|}h;$X20sSH8DuE{Uqp=ot8IV#+ z0XUJfPAgBvNze179himDBquLkY^lFfUJXeY^ecG0DuY5#Mv%>5#nBk$)Q`hNxa!BR zn8~?=c;TBGf_fzZIvckif&`9o_mO*eKBD+=YVFE+OLEb4?y}|w<*j67hbpD6T)MM_ z^wo<XLiT-EM8InXK|`lS9;G`l5-cZ9eA5YOoLPg}8j;|)KU$}UP`1wr>-_#HAS0JT zN|Ex;kAhJ3lCCG1&A`ns0Z}vn-%`TtE_QfiW269zMyxr$$+-8G-pAaut3%Wo@%{Vo z5G13nzvo5({22gWnx@O3A86UR4{O@?;S$Ar9{tztviVhpJpDOfx~-t7gG=m6`h-6# zCbA=)=nbFm%KX*g;MA~{dH~>C&3z{bblL!uG7MYZLnZuB#7B3>cz4QZUbX~{F5KrP zn+M;X?Ms20u>Uk>5}6khzt9Wu!w)(~%HTA5OyYo1IRO;uYZ%24k$^it4x+(wM&OS8 z6gBYQ2Z+l~eEM})c$7@|(qz)}>7ea6jzPj-plypv0SnCN;m>Dj(^oMZwlDk>JNtPX z*3`iul@|3pQJuhSA*HFttMKmFvnSK~6m=r_vN{~AW^6b-s1E?)GH@JMDWaF<ijv{b z@I6Y#T)g4W9vpu`K(0;&8MWy!Q^{Y;1b3t^WPHuZH~O>!BYaFHGJ9bP$Prg_;SCV> zlkSSO8%zK`y_>(jY$yPT1_oP}+G+q$_l1mW(L&9i7lw-Z!#Az~56;;{c;^}Fv@C@* zVB<Dm9?7*Fn+Zk+D;&SF_#y-#AA}Ct;ISGLLfRc!-u#ht?xzg0AX2G&HdzqCEhNc~ z9Bac&Vx4Rj960_*#Q*?rSlO6~)F6sFAQARsFlS*|{Gn8Na`^&l!xkie0&r~EqJiZH za<GczQz3W*%TWTO5qdYi?&6RH@*9C05;bZBu22}6glI}IG?^nG=~KI)L)`Y&j=9%K z9OzGka)Qp>f$y->aZ+uQ`J?NJE|k`g!LxzOesO9C5K17g{;T4eT+HZp_We#h{sG|i z1v(P`9$%HopAVHSq-_Fv#4#M)Fe&lL`y1%=23lBWc1L=uwxfFVy<%cl#sQ`Hb7=CB z7XzsMc48aV!E*}NkF&5am;kzxk5S7`DQ-8U!WX)1LEN2}rPhUG(5OdM!un*qJvp1z z5N*T|`qMF+BV}K;YPWinb^Aw5{^nK)(K_<V^Zo2tm|C*o3x1+i<O{&tDCp$gzi;1t zd(YtK$edb4q7PL`(sf;$-`*Q4<{m}BH%<eQn+SNBp@O=XYR3*AXi*fTze`aHCuMed zv5guJvuU_ta&c`A6bo`ew{49SeD}TjIGUXJIt~>YpzQ^jYYW1)#50d>x@xb33Zc4( zjJ%;Cn!qFm4;aHvBp_o}5pD#@JwUD?1rm6j^CUC+y?dD<H9kbm2s@}TtGj!XT}FEr zW-g@ux?9fm!sfdHqc_hn3RpxIg#ZuY1G}(7qDPz$(u@wQYCi=0*Qah^N`tYP{neqG zDZ16;nU1F)qXDT+UR#8l<WdT*#ylk>--yn`ZhcPh>?2zb+-;?o)W$}uu$%VE6~gJ$ zlwVF#Q@UN{RNB2uBm0T+)^g6%lsY<B@9z$5+q&=SmwUAqQb%^VQPVnf+s`qWROv1R zg}$_@-f-$B&JdOdP=;0S{e*WN9v-&cSa-Zz)!5iLG&~%$G)9hg&4ifCFE?lOByT(D z&7|70fPQ&|Smln{xw-GPwYTmjbar;?85+iB$C0Cq`u_Ez31%_obQf|91M?ZQhh`S* zxwTbRR`%!O;zf;I+kJU?dH-t{uUsj4J}3YFKG*8%>Z|Ez&z>!;sE`0Qv#O`$>?8Ju zElW-fh38vzBe7rW_F$UB(xA?nGiUw)Heb)Cw>l}BnVBuxy7lY1&qtFszWRIa+_{MO z_~pQ|8}qgSY$AdS8{VX23008(Nba8A-lN~XeOqfCGrcQ3ENs!njT>KWo;!E0gskk^ zs{EQ7o4EaTbGJ5sAns6dfgiz4Tqo+iE`7&xkUTK>9f}%(XWJeJE?@g!{_eqp1RWh6 z7iZ_gXJ-bj6!G@<4n2S7?AfD7j<|gK!$xG<SRe*W8`*{0M%d%~z+M)usa_)8N9WIz zm6GZz+WF`4{0aK{`iuAPw_i8^_xIPkCfvDxy?p&lKJ5B2!g#?wpnb1v55L25Uj^{W z7AvtIpPqV`l$YPv68ZoA6fp9DX8?(wK7G2duCC7e39#0@2u!EPgVq6eE@6u-^k8oM z&8?wu|NbdYEXTY9)vj=CeEsRel`B`8C&|jmxp{k^Zr*qJFuEZ)ELtJX$oiz}=8|fx zarJ<cdy3c6h^VMvx{a^v1LC5if3Hoqbs^H(RU!cn>yEG81zh2Z#WnL59JqHc?##J! z$8O%dseiYwu+Wf=jjisbWIOgOfnjODd0;a8x+k_A>-`)DgoS_n`kK9``n%q<=g;4l zes)?YFnjjwkH8*%_1tpcp{dy1gi!$aJ327_)Xcdli`7jZj<9_D{yn*<$Vg95udeEI zzr6jiQqG&i-nzx|PcXpYTQ;^537Cp@iR{?5>(@tr(A~4;_ud1WUWP_SLDS32HHmUL z3x|yh(Cvy<^Hj0s`^H(~7Phv%MLT7pqod6ufqNmO4<9~kIzQHdNaO$T0#nZZg?Vb& zLsCWD!os4%YpGCdY%H+AoKmx|5lbS#2ymd96Rr-7meYMtV!7xZBMo5n8q`4wjlfc! zDMcBZ9oWMhyHB1dEohi?df5{de3lX6vI!1Ae=%+nBr+IqI<W|tYR;tw7ZF)F;&hpZ z0?=O!H#vxmZ>$9b$nc#Uz?y86CQ*0U<BJOwfdB`;h8z~`v4GQh^x$laVG)^7d~*tM zcdz4dkPy&c>JDa2L>3k3?tpp2K@V6=?acA=qcog3f&LO!kOnS)f*DN~LjhFc$C#;N zzhn?kc%Vr&vH<<XIiVM4Iz>~D%h40Mf!S1V_N2*V`34af4opCQv7B%tI-%fj&}i|4 vBVeErJzD%A(j1l~J6inU&wzuk_&HD>_aMu}mGid^0}yz+`njxgN@xNA<70in literal 0 HcmV?d00001 diff --git a/website/www/site/static/images/blog/apache-beam-flink-and-kubernetes/watermark-metrics.png b/website/www/site/static/images/blog/apache-beam-flink-and-kubernetes/watermark-metrics.png new file mode 100644 index 0000000000000000000000000000000000000000..968b0abe0c219e5f6209b4905cadcf1a8aba23ae GIT binary patch literal 46059 zcmce-Wl$Ym7^NFDSkT}O!GpWo!CeykpuyeU<p9Bgdyrtk-CcvbySuyH{$}p{otn8n zZdY|x_i3bI_wN0!wVpRzSyAc}5&_bO4<A0sNP|>Ae1J*@b^!!<;1iU#7qSl@{6ENm zMAhBFCv9OG--hPSUf+b*8`p{N+xG>lTE-8jhq!8a^Nhq)R6b+F(Et9OA^}nvDZ*oo zgk=we1VT~`#;TkIT=wnGm~_w9YALMia<W+GKFoo0zR#BKr#j6ojl4e{9akvmEzhre zZ*w&?joR2c>9jSxU*UeHVX}2fUtZa(54Ba)UD^8|7o*yX8wH-P<gOov;y)~_UGEeF zU+_P!1T++mXThOJz8w!&v`@kKzFbOOPb&$xRldGlkN?jf#MLm{<n#V|Pw@KzdBm^k z+G_DJ`~78~J3Z*Xt!LOj8oh|e!NDHb%sKwof%E_7U`7Y{tJ<t&6*GOQayC;)eIL4B zu7;W4NZUSVT)*B6Kb-s8OeCHEVXT-s>+i2I(@%@1D!jREZB};P3T}VX%f7p`d?=p& zv$xq3cFQ+Tdc(PW!=<d|G^^X}-S={HJ^n@t2U1*vI{a(oqacKyCFElojf<1|d{27d z^!k->em%WyJws6Evc-5~V`IX-sG;#>rET8%99w&T=PuAKBX2;74VPK3ocr<H_wQ+j zJyLCKhpi~wcB^bE#sle6F(i+EpPdtuGFdZQZa*8g>zfIC-)LE3{GPtWH0m$XF!ysh zxWK?pn}^#WB+ND-kzSvfZ8soyxm0E6Gz?w(_t(H@kGjTe{mq|3R5(o9@8z)KJoS~g zi9Utj?Uc1_*>$SQZnZv7tI9I!@ow2}H0UE5;fxt7Dr!fJXXG$yYN?*r-CI<nZky+5 zem)bcgBq5?TYxw{5_*^4`z|T9;|!jf*0-|#>&2V=Mbq7*x;l{GYyYPj`$<fyRAac1 zCCE^f+UWbc-@DP?h}>MgCJ~>v`VbtZkoQiCR*P*={h!>e46QcTOicJM>`r3>?8bXU zP6aQezF92B1&_k-*KSzcHqC3efs6G%VKUP4Y|CzEZ0p|FWv>0*34NyHYu4<-ynVXG zrEFGn<=li^wp{FH<LqX;B<T&NslEAHbz&mRT!T*i3UVCyQ%QwSyVKbtxqTKCy<YN; zvL(EZHz|h`O)ZdN!NV~68tEs*s%y;04k25^qzYY|U-_fQ?A%8#WBQFn&t$66tIako zH{+MuUAt?7UiX|I(TL~v_uS7~q!FIC`sC`wBy`M1Q|;L1%q%;uiM%IeD#(3^36l-p zp4K|#yKhqWR+_G<U541M|J=Y0jtU#xosLfmjjl9GJg2kL(R-2z0uR*USFJSLy)k|B ze}ePA*!lSdsjQP&E-(8u%u^08hcxStyO<VVW8ZXWUtM-(|A4u@{K}W4e@j~_^n2nD zuNLSP+EUYS>&M`;Gf8Hd6-S@)dAvaeX|EqHjn(w@h%X3$ag7R+#K4`>z@qL;quY8d zsL-@|yBv>ENQ5Bf-wXSBt&ix$8lA0n*xg0Y)>^D<4LM1J^21XxE3gD=-^tj0w@hld zjDrwKxx~V8f4!`~zv5DPRwOrDjSEc|TPa<ix~-??U@m$;4DIjrN2SwK$M1N$ov%(G zGwRkZ$YX7qKT3S0l7-ybo%<C9-X{uNe8q@_3JN`VD;Io9aUtYNvzoU{@$<fj!&IKT z%iKGtg1&a+rLmaPwVCe_vNSB{RbY=(G2YEs1E!;t>Gob3^;ho{E#`f3%UEU$mZ#r- z3+UK+f!iVUOjKf5#4R^07IMjQ&-yp5yzO%LVAI8`#Dc#s!JU8qT8#+rW+A|h&z2i= z+o+YXxcBZ>$1-O<W;_njV`jF59N=Yi*mu>M&(4ixwTSdRkNdrMS`RO&@cZ7_G_PFe zDe0t*DClrx-gq3l?m-alYs|*wv)>F&uamxdMBD7O`T6Af$ooxH{4pC#S#CH#a39G^ zFSR*as);EsZ+iNNfwy(luIqNU@_3{?aVCr50$W*g)AOs=*~aZXVABY<fwNoeYdePc z?Z^6F&HLCvKALeFCVr5$*hduAxVdbG@5Pjt1(X~g!}daf;k?stan<XCK}$l|?Ck2< zA{SEWHf{?r4h8-EEQ(i5LahS##r1s6hkVn1+O5;zmajsuqkJd4@Z~Km`=!5S=b@wR zKF71sMxtn`_Sgg!p-F6Qam&!^Gvylhd<3rQlm7euxF_n)b875K*>VG?<>SHhcWQ_J zuQqZ!MOCI78-}u$x-HF5CH0$)5V7ZIJ(tWwXNy!1CD;$5<OEyo+_q~|2-_PulmvCP z@y6uqG)h@W7=G3fN?B8oKquk1(!|HwU0l<X#&CMuw}*}O>@>ZW`=jeUnc4}#FY5EE z@F7y>prmv+k?{0`0n9uNT^`e-8|Iy-Sq=U4HJ=_(%6;*I)K7?Nm$3jkOYk(mZ)uLh zX*##h<mXA6I06i`SJ@eng!y(gH*C@+JfW|uxy>&0(l-3gta8&Dv;o%{O=2f^PdA*o zLF}yV=E3D*?$as|WTUr(N#nx?ngWgNHv*Rp-HDIGNfm67(@Cl0`R7Pf+G#wzioz}A zlV>RoST|`slv49Z1U`X{6Yxt-GQAx?(ovVVvwhbjLxu<u`FVI?KSoh7ea+qaCM7As zpe5ez@e!4NNUZz2u2?V+Ixehvm3#i7n56qaL&!yPL4~@~`!(0;x1!R<Sfi`JV=@qk z4@6=yoRn@z>5Vbs6COR5-9bxnB~cdk?DzhHcKakVOwQ!K6ua<oi*Z0Hz1)U83KXoO zr<b>NbusU}df6>Dt&nv)xjjGt<$ZCkG;`eMd{~WcX-3|tW9qn~P9=JVO)Z8_xijF@ z@xD3GJa-dW+fZ9N@-nW!uHhJ3O0Mena-BCW{}#|Aq=E&##{&_NlWtq@>}ocxG%!3u zR>{!NCEKXlHclA9CtRxb80MlnpPa;+nSOsdwv$)F4=O!v0pkgSkv4ekWY#A-`e@`h z2*?wqg^kJbQ9TrU6O0?FeX^N$Vj3s7o_^iTHC^Lc<}m~Z)9yT6Qm82_Yqm_-iau*5 zi@%r~3*#>X3zDUC@2fO}qh7T54NYJF-^PiszCJ2X1)K-1njAf_L(}_4z#&;2g(6y@ zSL;Qv5U!Q{!s=g14dCr}J75)#jGT`kxtQvByCq}1Sxn03s6H?H_RM@<d>@4y#T~OD zLX=<Pc}q-|?<J$g4q`6#+XxVYkA<~D(@j^O$bP>IKzDkybA0&v@6GT1S!pY=Vyb9E zNo)lM{uAfA&wVEPjp(iVIC57RxJHo@w->G}u#?<q${j?Hh}oozlWSN8xmozeE7`YO z74zb=HMK9{fN>R?p|0CIJ+LjMIrcq+dsXZyB|P2;CPMC6ag_ra;c!}sBIwYQm{~fK z{1?P>CdGlC`ezqRIQXIwV@VCu@a5&&N$e&<E(e|Fka;fu?#F&gl78)cI|HACsy>b; zWgPbD%k}!&d>vs$V`cs1Q6y?JS8#*Uvin(wtiFla{!tYK*+PN>Dz^xSaoMuhG_Ua$ z(}qkI%cvlQAu*8|y`e5s-}mzM270oS6<E1wCnpuBoAsnlgECeGu%YKPqKOz;;*AR} z)a8y)9J-WY^rjh@K#;a))ENg;#_(&;yR$+o?}d^`10tfpFwaP2*qLG-$l+98gRT5` zX<{@iHasE6T1hgR-oK#FY}_`p>L|TEoEV?y7@kn^^5+ACrJI}&Nphbry)h`HqdBn# zp#pDnOPaOw$foq&e^W|EvFg<SIDCc3=F-Sd)n8&=Np!Y9pLyxcoJ}*!gEkAwr$Ld0 zO3~R|lz0$y6^wb<-CgYFB#__5zpqvdx*v18oy>^o&O6De*q3;<VFUW?rENAgay*HA z`s>2!Ng^6X&ta&xa)n#PSj`9R*##|<?y$#uhGC!R%LK5DdiXFa=)Jm{u`C+Z3T3~2 zsF%L!bI7^+34u~U-Dwe~8JEu>`a21SfsyUe7vdlzpH>#Pk;(6N+n!&I4oaurCRz&p zaMcP*)0gG&cfXRH6Txr`b`1xy^I%sr=XQCbTJW5WreKO1DS-q^jak7EE(@uj@i&Cy zL#zQ8rw!{CH}1I)`9hQ`pSM^3=rAjcR=xH4Mmxnt28mxN+c_ADvGKMM`or>uo&TyF zSJb@|4_xCw+}flRT4#OvlG)>HmykQ)`@Gzz@wP|e+K~i(fex1)2VZsS<-0QON&5UX zLn<AEtz}%U`3~>7<#pt&;kApClURb^*QVpXeQo>H<t04Ir{2!mt8+4Nn|`SkvSE`B zw@*gqFXZohsIyFpK*vn}nTYup#I!@w1q6AUB%9yVJ(JrJ`NHGYB}QfF-&px}eX(h* z@X2$0U#mUdJXWaa{O>kG>Du%Bb=!Phy?D!sudk;BFS@tS7JvVppbhHGIWUjAMBDoM zHb*zZNPTAxA|XML6PKrK>B!qM@*av&gYf7%(^9`fH`xLOr77wD@~#iP;Tj>Y(;mQg zq6TdA?h+5UVTLRm9J+{t`rMzSh+F~&2M9f&mJB-Vwt+JJg6)4JDE|Mv#G$EMMB*N< zeHo>`c|OeCF*Q}2z*0}>*HPM;D83z7RWR53uBmL4MxV{=@o{9ja2$H6mz3D;Y>o4H z&TfWnMg4nSQE{=#vvtI{_2otv4CB4tdPMFm-0b%Xa9cs5M$lyT45JZIaoFdd$Na5C zpKy|E%os+@%9ct7Ip7tOlTz1mU~21Mv)`ZE4@st^j*F6qeV|v@$$5VBEWGS|K70<? zn@+MBk;<-{s!HLYT2r6b<uo1`NGT~fl60c~8coD+I%l)6V19SQ2O(fzLm|L4L?sr= z{bD}ZaxUWZFj(ckTL4wx;&`ileX_{@9s`b;wPKg=x)fs^yyL1eS#+#1*rWi;SFHCK z0_=Ial^NBgLdE2ESJu{QcC-BTxl}+OwN<{Kt$JzmavDB;t??PKWxRc5tJ=}#QG&#Y zZ~yMvQgtm@dD+e;eGNrnNx`C#lkabqcdFh|KMgjiCJ}I6C>>oe0MsFu*ZrxVA@sI` zGVA5bW8|Ka-(wg7(%Pqf$J|EHawqF@wQGt_gKheD2#xc5{VD%wbY{z_I4g%2N?<Lu zaodovyR=mm{|p*Bu`#RDodCOoV}^SjT3wbPJaAJ?z%x!syo^PonD+JX{Uk%z>3WZ` z#_1l6pPgA3H#)EHN2wwb@Sh;!RrCGMU|iiH39{VYW2AQ4#v$GZv}8(LUY^20x!~wo z+oEShx)@4&gEdyy5Jw_~>CfXwWZY3!@-0Efkug251OZ4s@DgRcKU>xC;v+Lgb+AeB z;ca@vbnt#8eD3=W$d1|ixf$JQD^ljKY@|L*s!Ob635m%aBHo$JHyn>2Q7u|3l|c5x zb6=g5mPTv(rAg}m9gt;KBk2~CEx_6|RsQMfes@^I1ob8VUL^P+(r5AcRzNz+Dh7{t zFQf@tF^lgH*H{gpW)TmCxjstV%$eEF6itzMX|&z|h24<ALJkJa{i9a!`^w|BCzDFb zX;3a!UqR)d+i2Qfjk@#0b3Ha&wJj|9sZw6Cg&3=4bQIkd*VUVYSq<Lq<$;7<rdU=H z4##V>nfk}OQ+~b<58GU`CoXc<q1xkx+0ymOApxH|fr$P`d{)QW$Ln<=7Co;UE*how zg~EVJ6N_f8cB90*3h`|+^Q~$y^%^d5GcBvj$&?-q1^8^M_6HwSA<ZaPW=mMgU1oE5 z;r%Z1J~U)s$<L$WuBL3cKD`|V{xHAo3)by=c0JpFGJV^)AZ1*;p3`9(Q{lbRp8bna zz7R;!HtTSzEjg8)St}~#`R!)<+>04CfYnYk;uk6f+QGf8pjUciGE<~miV5G~gJO8P zsgT{O;+>o`7XNH*EUOF_Gaj?=Ua5!a#l)tVwv&PmISk}e^#K)z<;e^Pi&16bCe?`p z6A3wn1(5LS-BPU(GBO(qs(#Wctt!VT3x@i>H&^23G)2r^MRb1i^``XjY_8Q6jA|4@ z0r}i@u&HG?N>rfPcxFYtF_c9J8nL0Iq(u597EOAr&?~5x3F1z~asGacg~dwo5AYrk zX_<H!Uq60gMPxTE8C6KyZ*x6O!{;{qbtCHKVvfF5RJA=?#JYWRP0}Vd%8L=#olBwb z)ywqp6MlKHmZrM3_CF7pzIZ9=*0Jw^5SIOoWn?sqADQ~uO!6mzaRFkLYrHfjYqRfb zGnhg946XPMjnJrksNEW4EbB`525Ow!?M)1$rjXiHHsdkZ2iFF7e#djON7DRR__4Gu zJSzJRzHisWhO~$I=HoV-&z|*kxtfN9P`#yRBv*7gkIaDgRYI|mSF*QL$2XlC$64)b zP;dVg7V5~w;N%njmKdzI1_&bU@GcQKJ)&Brkwqxxo#?m$7$CnP$OCm2^Ft>~HA-!* zr$#KRKOEyKY{_BT&GhRfF#&TR^}7v!>#xQQ!#eX>MMN7F%QSo@Fv;2=APPng4iIH= z8iH@$9w*H#bz|{E2Vp5k>}CtZ8F~3;(B8Jh@+U+gFM%NRs_N`d<o><rP!N~$ShJBG zcc;xQDi^Y;q_T-B5dqWDR5K?1V)X$3n;0SnmT#P#d_&|&2&pn-4`ROVs|d4U+1};7 zS@q6M_*~uzAiDbzM~pNmv76d(1ad$dzmY8Y#jasp@pzszV$K5c&B`X_1}8XDkKA8; z*OLddfML|6k&E<hJ|BT>)ALLNEv3Hfc2vG5tTGb|k;Y!1s&;fKXPdK&JrG!S#>k<} zC0}~V6$f4h0@ql5G`p?OPZ_bBN1XVce#~1S??a99@tSQG^74_K3134~ttZ(gsRDd1 zql7zh8-DwXN)l3D%Tkiz1$*Yfm#ahu)6x<A;4I@5hE;K~Z_-Wo2US=zhVD9BfzzT% zMt+J}hm^)V-#`hFsLT9{>n~|d?uQW(eP@DRw@sT!%-o_zvs1Wo)5%sh|D0a0hV7b= zqkHIbn5$%Z;r5L(cnEnaqcX2-Y!M<Fe?R$oy^C||Dt5J4u9oK!@IK2R+cQ$rgOC0~ zkl`WsznF5Y_{snJJUcmGVBn$zd2L<#tmM|oKT#f4-4Nn2VT?Lc_4ne}^g6x%J7+?z z2uuz~-Wq*imImi%aY^>_usWo!yuaoDL4r3HRTt-je-4Dj_^qht6~Rqhu`S-5!1`mw z0Y}qsTleg`B1TxlOI%la+OQC?sOVUQJ+9Py83s_$D>A)>OlpnynW)I8zGOF8*p-x6 zS=yv<kgPRRsVs%u%Q}mDNe{VGA{g|Alqs#VM13~5VyH5Nq<v;Kk+O_77P7hx!Tjdk z?QCJ!ioPwX&_WiNzlyF$^!168&9YRBhGO&l136Q%ToRoqg*juNOa!J`M0@_YbjYOb z)6-kf$0u*l!A~d*l-@%*=Cvt#(Zf{sqn~BEb&{EFfq`Nei%^({D-%v;(!HH`T)iQ@ zKUX)%$i9&KJRM(8X9J3Erk>{`2^m#8GrtsN9U<Eqn5EkuhO560<`Tz1n2sk`H@9cv z&5-X-r{2a*cM`$;v%}-E7ime=N{iq6O7bU4!{5J5$7gkE!I*~pSZ(M2T}P`+6Lq_| zr}{FwAHbxM!=x|I(@PR+)NTVzf*j*N8?M)cg}LT`<gygIbjx@RdNFHP-~92=2<voD zZTwQ0|3=SLOut3c;QdEQ*RmWut&SfE{m7`GU)&q{F^aYd2Wryq$%9ajdx6YE&RIO6 zs~HhiS4*qRSZ)WJmx2kM!L|8^PLgC5Jri#eGoX&>q@BWv#)lEmNQ6p#3L+w$2u7>> z#R$pPRxvG%^O&)4r{vIYzEwSbCU>1`g&-F%Oh5&VU|-e(Mw5Kx-08^MiuYlK4mFia zI31&)@l`K7xQ?$!x6M~iO~-hkuV|W6dnv7zWmW+5@^XxQ(8WQ!IRd$uX)2E2?OX*- z*v6uHd^q{p^>j@K;UR&SVO-+I;d1K-l{vWcH=6Y9r6c_H7f-RDwT5MwwBsV`4SK<N zcA2!)wwB$57|DRMHrTlLuV!WLmJ3WlMnSby5;sUG7R<*bq%t3+`r~aYfsF~mJDwV! zTmk)WY5G_aAA-iLiY~GJVC1K8%Q(m#1s{nF^BjXF#FkxaZY=cb(&Dop-K6^M53l1L z`zEXF*#50JYyh_ei!VH9Y-6m{8wg?GSk&MG9R+L5lVA65gFJ!bAV~D<+Z8@mTA!ig z#tNkd%B9Pv$|e}cEKaxf$G!Q3&qh5mjgEg*r?n4<HHzKjBTT!*vL`D0P4oKl29RbJ z$*7j}b7cl`S;@4`=`i4$F8>@d+z6<Wkz<yLtd(}5VCHP%&6$y1B9W(Y;ve_6WaZ6M z-^xB=nFWQ|<`PY*LXb)9|1$GI&|Tk0b-+9#AauRv5L&HEq6J*8f1dv~^E!MOO7%4x zg^-K+R4sw|SGW`EE*9-R&7)%d((ba1iZ#`eEuhbj%?Emv{qYB!eM^^-+u9mLZ@x&8 z1=<<bDEHm0dtP+HHE;fD*5}KqH^VF+!XTEx4tD&#o-~B#e6l1?Nk~Eb6vW%v4Et>U zar_1dAIwXJBx7!1m<11>3UAap0)qNdHH)n9x?p`oJ1(zai6(PhbeuLO4C@GcGut<u z=B&e;_oxa}>e&XWurt1?RxFT1_Yy?;S(Xa@MjI?AW3!(;1`ljX0I~rOuiHw1p@kKZ z<7;B%Dz05zcGAMVENx_i%os5Rg$o?(W6*Z0@?Dm^$P_e)k*f?I9e2)vg$rRA6FOTw z)O}c<<Gj06{PhF5M+bA~0(#~}{Hr%^@c6?J&m})tc>CXZE4*xTcvWXPPt5DO?{lVP z5_)PIe70fi0+}8EWdS}0t7{*p_0u#s{c?}blN8?=l5~xMOQDAVae|2w-Zvu5FFCpW zFphKJ-m-3XtzTk83F}%OJO3`b6uCd_^icl(2P7Y3`8>SDSxh2SbH7Owd0M;9KYpN} zHGj6v^AvENx6lqtbgg-kJsKTQ5z~=4c#yL;-9FJ>^LEh_Q#YTVsgysulKob{{}JmS z+SJ>A>t$lQgmeoY8rRauvN4fdx;yH*W|hOIYRXS~{__pTCdhYA+%&*{4h;X!et~)s z?am$wfmgctRz93A;6A*Q+-B6fYye_#!Jq%DySzC{HlYc-v>&6v+mDQp=!|Y>_$Wmh z6|P~U@jDNEyV5){as3W4j~;)KlAP%4BXtL(kqXYu!12zBkQkAPVD99->eKaJ=TD-d z;w;eFg|{K1t#oyfv#d2X`p>t0|IH5G=R;K>9b+OQT4P#^Xyo~C<FEn3ENSB^mV^Hl z!s&@hwe^NHaJjRG*)g@0S=h8XY<G;?*joLJYFJ))xd%S7zKfFZn~~F9nRDhoXXpl! zJ7e2c>e0V@ttF@LsmpH<I(k4R#lxYnaOrAqZvAqx9mDKh(Vyed86Fwb9UduOxzYQ_ z1337<KFL+-4Ov1_N6$YZlkjPEVGo3Q<EMWo$<!dSX}d@c*w4|m)El2zQAuu`7?>Gr zw7=X!ejPb~X>pTfx>zj~+2|m@KEEx-+Ir2l&lrBY#$|$kiCF;K>%XC8`Xpy%e||-B zW_lNV%v~UJ9rQD(-@njzE7lS7YVA~ftQe{=5=O0vna9TMKfjpFGo9Wmd-6kgAKb_; zUTbq>E_#=~)e{fu3kjEz5Jr+z5e|e_HXBn7xM~>S!lCsB$!H%5C{M=t&A+lKA0@s2 zekpmUt!W~UZM^8?wro(N4DmBmvcY(B5y;O<;aCPm5f@4<Qc&cnU$pIuqa^JALBL6b zDWsO4_bZEOlTUV%iVEjaG44NpjBR~ygL;q7*8b|WPxIH%k*PyU#*22AP*6znxM;x% zw~%rAY7UF0&(g*#!r@X$mF{A;$_Wjdn1LsQJv1U(4GU%t%h6n*wY*XJ#rk1z(v%^Q zyu%l~y%&QJK|iJh&Uf)mLnipj<nj2iZiKZrh)qrFY%G(XfC`A+K!5jC{6hS&I+cP0 zJlA3cX5k-kU*z47bd>#i=KlGkD?jqPouw)0)TjQuC9Ipkv-!>S6!FcfwtEf86fFO$ zJ_VT#jtdv<8q?7b3Imy3^1lT2Zruj*X(DC;m16~ap9tFBoxNjD7(qKV2{MPAdO>Ir z!$KhYNIL^m!5_I~7i&7pK2T0}5up;V@lzZE6m!(mojOq;)bBGy6D5bLuow~x#G@ku z{WiCM%*qz%UAaV6*nTG;9DaKm0a|cLs6?<o0?(dlmJlNh$Y`y`Mcs>Nlx#>#v_n5d zvlJ>NTZG6UEir<tOY772Bs`*(v}?KYB&DogvBNh08#;b`G+89Ogu&pWsx^dPgHBH) zi`u%jI&-LHr(z+K#4WDJ?pM4R?1?<2;R+arYUt?gov5354}D$=8HDsfrl$*9R#w(O zc6XQH8~r`GYJX)9AR*QLbnR=pu6s93|5s8A<=YLvSsvlEAG|F4pFL1_Q|*$(6IP9; zyF4Hqo@TMBMSI!yE$et{ceo?m)IRCa`?UF!cJEKSE|`<FT*uFY{NL>X%jVnT(Sx+Q zl$Yztl5t6eP`0=o;U^s2f4y(d@#BdJbUWg3qUz0!jiA3G1^ETV1C4;afCZG*i<J(M zAig+lyNsDZAZ)3vDHgfc8m?b@l!_)|$Lf6!&X<%i`IS+vSt0J@IQZ3O(J`hG<r6KF zGfTe={@Se7yv3|Ay*^v4I$y7`<N<A61i8`gpiofCH!HT89?kfw%}o4!?f{W3F@1xL zdLFDlA!sCgrSYLvIRcEcq|Z0RMO5x|Tp@UgNtr5aUC{}k9d^*pQJs}%894@J9X#3? z>=;z9SVR;8!a%SSDydKj=Mnz?;}G4F5H(x>KP76G?GP3%=v2xrXncFLC28#8Tr8T= z@>Q?V%1ufIJBxh<3-<ZKYVtcC@JVO5oIQrLuKib|?LrHmg8aqnk)6ikBLi(^wgCAa zNStvk2gjiXg=$GI^+*cU<oQ=8g+>bBB@{I>3V&i}jKUw{d!}{V(NvjMHbG7Du0Q57 zJpM}t9Z%uXFjw2$vQvrpzhzRLKy7;7oiOYYWf10W<``A{__0gR_L)XL(Z^w{NcD_B z%Ib2;nO*$^8QdL8N3O8h&DqVbuA?BQpHt5VRNPBXSJ;1RWv!kfX191o**Q7qEh#OR z(yM%fzH95XJ4W&Spej`QtH#CY5XyH+7nx+9I9AuKqHE_lMa1qk=6yK)hg$VaMdi|4 zu~h+BmB><2^_u_U%6*({0?~$S!385a#A=J`(Tz5LAAE_8g-@gm8+Q0F7)Lyu)|XqJ zqDu(-gUSalayXmq*U!*s5E1;M;wL-z%d@8wOwHuZf{wd}VUrKf9f!OnRuTfW{g%+s zx7=5GS;QVJ&ua39def?a9nkyZEbw!got@Q?n&aFvcNT8E=>mk7*bjxk`2M?EaHBDG zpT(+sF1)l~1_Zutk1mLq8oIMd_c{%N_Ql_xWfH@uRaCXCc0ACEB`MNGzKu?05W;~J zEa9Q=OLNp6mvfrG?|6lJ@Jd_r31icn#{rQEIvjcX=d_l4t|oCxxuB=TUgs7rZxTX~ zWgRHzKTDH!oe>pr&1=7$e6Kqs(li8}<r??nFLk<YnH!N{FL4TD<Pu%Q^ehuo>Cf`D z;=u}ua;l|r5}&WG&`Q+svsuJz2oj{3le6=;(XmUynE*L(PZ3B>PvOWUtdp}}_E}Wa z9P(58+0lo>vt-|>MW?3z?*xl5E#+tp5fBqm(w=*M@d@_%g-F&Tw$pAk4vtbs|3YC~ z&Mgs~D%7<o_c`ZB8OD_<DtcG!tH-i=Wuw8DnX`fu`ilr;`k0LknR?rKnU8Mm=zT;g z`mG;@dfYDvnw`&o0A5yFc)T&^XH$$6E)nof-z8bmpXlGM5sm0<Y>oBt;TeO^Iu3@h z9$B+ibBC|2zG4%p6xI-;ICZBOUADO&5y^GUuA4+?=Yx~br{c2cSmp6=M#6Cn@x7h~ zoef;<a)JU=c)Fxg0qUPo;o_aQc6c%c+TR6t`MLE3_${Z|fpBYEn!{uoPZ#Zc_8smM zHO_ocf^-ALR|N3g8xeNV*Q$Ua%r?xpNhX94nX_K1Muaa+BdZQr)W;87D7B{x-@@l2 zGcg^m@DFMF(aZuK5yN5={X1p6y3sbprOfHWX<@u{vzt^XEtt`nLH;hC!H4XeqoGd2 z%@lDbUoYP<_`tGhWMhaJ=$)bHEn3|=&v*OuGHf5wfB*V5P0y77k8BDP1Fk&EzMB#P z#RDK4NVu&*Eaa-7z1lj-tmP#ksy>-@N594@yLZuc=j95nktTnyE&m=XWU61}NH7AV zbUa85?KZIAbO)QJEi!P)*BGs6P<Z}CX+;`#$_7v9)LQ<9cht#^ay2q}1?fsB#&I3b z)D<$FBxUki3E4b`NK#Z#Fe&W_m11m!h<Li=yK{cmh4`D}=1{@@b0KFxOZ3~QJfxD` z*T1c%ql|4t=Vif>_k0SQF>-N<Dj>wunKD8cJUF;C1uWGl&5IrDGOuhJ9=&<JO04p6 zP6xacl$Cf6`mJufq)hV6yP#ihTeZ$EA}O7KxGvuBcmFD<UrdKaEoA%{*f^rZb<A!1 zz3Rd@v@sTQlSuV1<FfO_?H9)rMcu#{kuB(t$`Mb1v9g!K1D*K=(~xl#8HoJGK1fwc zcD2%|ZvH@>Qp!*FygB-dN_RI`uEy=c)K~~t9~1>{>|74URp_X+ulr_=$ktC%75)PA zF{JbSAd+0FZ_i%QK}1VK6dfH;^{<J#^y2j#1s#WKh5a1ercf`O4y7vf?XVw_exrF+ zyMMRuT<Sjt`h@k+PozJPBC{%<f4vcKJN%OTV*i<0&Stq?d|B2ttf@<I2(~MLCPMCu zQ6y{O@V|tSCaHRa4Q@8vbuI!4U3e6NTq_*^I!wOYof!)soxv<|sVxr_B956GVwQ!3 ze`FW<JD?fIhe6g;1HLe6XvaWv>1+&w&|w2{Tlc)uL~Hb*4eVLkOEJxflG0LjyDnS2 zFLBC6C}Ps)e%QEIi7PPOy(x{tO@5#dtJrBGG9w8gC!M)5HwwGQ0(5CzJVJ_o67rk^ zrv~(Ry53QtsqkX9L$7;cvUomq8bJ({=$^qMa~CFQM&HSt$j4R%e*AUx=`tz1DD4z~ zbV0l9kejoL9Yt92FXC%y4gm-$L@Qwl-;cKbc-rJno(YPN?w4ij$Ik>PO#fk&o6l(K zQ1h3P<LGU3pmu+bDhe})&?0!bA&>xXL*{po5`^_N7#ZFB1o>ls<ZF|M?C;WDv8y|q z&R8KBj!{ef<}M8KBN6L6({tRgAGt`exFbIVUUS6u1pBne1tkgHhl0%;IxbW-A6V3s zHBH*Fp2>aC^f7oljZ5K-gPd@`&fJRgnB1TF97&gH*3Iqg>>d+c{TWi_sB_WI#hYpw z!OF1RLwt59-K+z5%bkeZTd#L22@qQ7%<24>>^k}H7b+mI5rrSb&i#n<YXNZyXjDu< zQETNAzeC<^a=NylA(+;hQZ!^d&Y1Cwf#di@t$H}u0Q=|h57DRr48Ok(rDGt%4FaJv z|3snObmH>ecN#Mjjf8uqt7`vbhww|(<iI+b((fPE1bEKrgUL|cgWJZ=xz=C9&KJaz zVqn63n3Ey%qt4Y^Ff$gOq_<MLA>1DBC*=JpxE+Zr`F?zNOemNU88<oL&D@t2Ig0-8 z8JW)l`mE1`DCm=W1tP|R>#@~TR)<x^WzV(MMJPrE2@rf}%{%dP00R(t6)-ZS({R?x z($J<%vrV{H!2H~WM?rRPQnQWtkm<0)GZxVCQUQnO<rA-Ljc>c<9-T*AdhBlVdD&ls z2ghrZr%ta4%{B7Xfmy$$z3i2ka2GFH!z=Nz^JS9;gnV8(*VDE@T2yv1N&NV;sV`$= zX#{_*clQS!Y@Cans<SI`{M`hzeFit@E7v6LmMUrM7$?{jHFHSZ0M@l;?W_CZBY0+i zp&)pja<Tc2pQjZGxMhjoJ{@6CB%}xd<rwCqMOJ=3%JZB@OK2~TI9=q%J(2X@Qa{a0 z_KvRW(%dlYbajc=6kf6biOB)mK?As(@K4da|2)y<VA@ZH?ec2e5EXTaJ_CCo91T-c zm@p<9zk`D^kb)xMWHfRp0SEslCx8NVKaFy0zD2gRwNfQKziFv*(E<F1&~tX)XUjAl zy$bPpIo$>K*HIe<JL9YCKg{p1fL0afZs!=pwa@h794#w{)UIu9O>k^Z8KHIs%0+Vi zpZ@OugIgq^V9L1ms%YVBZ2^{T<<`Q=BAyL1JA&G-#8Mx@87e0Ak0UF!T5{^_b9Z<V zw5JPoQo?!vXkj<dBFI&!?1t;VI&JrlE-~w<z=k((q;uJECMeg3EVmXld5&7IrTwV? zamvzQyILZc4E;krJ%5h(cBt+C4>3`PO1az*k$=68Bq0d^w_#4sch0Vgb3Zs4XsUsC zrm*eO7t*cqA9-|*^HqKMnR2_UCrtYw(qm3{!oY#or+K7TKAc<K(R=yfvM{@rgRcKX zSr`@*x7V<Dkkxbus80c00iRG%<?o$}oROG(X;P+MdevI+r*IbTkioWnwbCfZ?u=m^ zLBn6)FY@SjfK>`W6jd1o1q2*xn_QmnkoVUb9eREyIBArD|K9%wan8yWlkhk=mj3$m zr;;wlOAEFeNolWu(e&g251T}G^vdI#OO)bKlD;#`v(t5*p8s=HI?EzTI9~BLXyU-I zdF*7s&65W*4mIVCzx2;il3I)Sbvm*&+_*rE8xbj28!d=M1T8J6Z`eN|&!y9MTnqjY z<z6_`SP((5MhiCiZRH@qSxF<Grg~ha9OJWmM_l(uG5O!Em8E5^y!lmbpW~=zjnlj> zqg*pB=jeX~6Z>=l&EbA5*=^ZCbR_FBxAl}Aq8gX;?hy)k0|&c<Y6dTT)^YVN!)Rl8 ze4V5V8CokZS!|<nwvel3Zj!GrZPvVhCHbCpA20EK>kABl#Kq_vz!<n>v?spp67@$p zR{z9SL87vLM#cLY*ZqUhV)>63bCf)7_)5dOfz#Ow9M!|gYtvM;ow_wDF>lHKYCJ+X z-yVnu(mOagX{n@6m?`KMAgTV7;kT+jK%}l^TPa0kT2?zbDolnOU0<4&%_!2<eWL$c zqOcsTCoi4-_9TGXB~IpGD1^n?uaL<!W|8r8QijpO#^H8}lO50<7Sr|dTC~?0tn@r! zb4;@lw(zZk4BYe_XI_<~+hgw9q}MH~;V)@z`MW<GOl`Q6D|9E{kqlI{v=|i!D6q0w zR*8f><zsr*xNH_n>JA+=D>arpp6`fVe5oTpohBfxCiRC>Vw#Sq@52p*9`BP1<h!P@ z_!L~N>@p-)znj@ymvG`Ad5>oZj?1SJnUzOwgFeSNVdG~1{uB9{hhGin9i5j@B~&x_ zXqYvkADin_DYoPZ5G-%c*h_K0nn7pTCa9#m1@ET&dLnt@$Rvg@wTFD2=%2obEWz42 z^%hMYeZxu4U|aRP<8s<(MtOKQFPAF}yE!vc{O6F~U_!SjA|_};8WeeSpRkUB+P!-y zQ={M^5dw}$n@)Ir+2~PnvA0m#y3yL=Waj?!;|B#ve68Soxjy$PzuWYBgYB{{5W|j{ z*shf3avMy<nx=n8{=x(i-|J_r0~@cXQ;TQ4jed75QKx>xWVgRyiE@-S{xX_o#iZMs zZ#2H(G|IkfrjSC^^T5hf<(L`V`{h5Jxhjg4&DE{0d5vSyO&>NAao7PMhbIyF4OL4O zD*;z4%^rho$|QF7b83dXDFlv;DbhsMXhObOnw*AOD4vCa?cWX3X~h-<CMRAeIG)vQ z3sV=E;Tb=H)ZgOimj|98xg_E$^s_bqQKzQyh!4D)3|ZixTWrk`ap(X9-4;ssC$&CP zR3ZU@xJ*g`QmGW@O*qRDr#56<=D91BI}YPL>-Z`rHOky?TLwlo3{-rfDdEu@@t;!D zM}op)MiLq=o)*cgx@g05H+doalCXhX6<AX-Snc8nIE;$yliM_@FEC-C2UDxsD0VJ0 zaJ6qJBszXPe5WEDox&brzAA9r6AgBt9&35!t5{zvKiFL??|eH%!gB=FbYC#Kd?U1Q zb;Nm`aBQE;h13BM8Jc_3nl=pA=Qeqhy}NI(&;KF5uF!|Yf!h-FEQyKV3j8!1f5D(- zG`kmLiUBi0x#b?|+f;xigpGv_vYFz}nBHKk2GyhNdbXAarh(V=y(<Mj^Yh-V13~F+ zLAHR0WphNBfZ8CFPoFGR8l8jMC$L=y0NrZ3Z^y_V*As`7+NOJLol;ouNWuu3sY#JJ zXjQU&qhTxl%HnjVd%I<AiMUPUdBBYEDKL<B;vC%5P7m7&+ofXgHJth*DoF_j23D2Z zI@FP&qk?rWAur}8+mKWg3C-FX<Lw|iGi*2qAqKjQ*vyBmsWP)7#hnX7$^-=4)h1)b z#H1tGUM~|>N|RdrU%zdDl0H&oD=;d9PtTGIVw_Bt<(}@E>1^v60E2M3ZT~gd!59^* zZv(G3Y1`qmW&`DQr^C<l-W1K%fpfPr0bGWY2>AgZQB@aV(reBYzz}i{Uea8u-c6(K zi{mBFz+qJB0Zh5syG_JKxSWM%n=r|l1^QME&%K`!ea)W;ZyPBwF)$VZJL<4bZg{N7 z=6pFjPZ3*X4nTAg2mzbxdjHn1LTf?_0p~a(KOW%nZL3{RR7DNTF}m1eHl&}TJwhPg zH0BR{pF(+`nRtex0NVjdUULvXAt}*HC_R?3_Wk$Lgq?%7XXX|(FUFZ|9Bts3)MpkX z91FxBm)a^0V^sHkHwzA&r9!XpC4tjT?{Bq#-gY`(T#uI2P%LU<XMBmt1mNIp0YoB= zg#3%`a!zD{vgXMtHlK)^U_e&Qf2HAuC}V8+brc@lfB4a7oXm-KN>BfjM`CdrZ!>65 zQ4pK!3aAE5U85&&cEuWQLu@bhSjN`AmcfUGD<;YzT@e+XWpIT`G=u0=(Au5B=b~M& znyQYRKud<`-$<qnqUYgs9}tc#W9bTays*`Z@2UABOV980{oVC{&eGgCRwF{7Md1F7 z9bn379XzW5=7-8zdRa2f9!fFHrz`j%*o8;6P(q@Q+6a^ofj%H=4c<`RS~Z?F&ZQ~& zu<>Ao=9T}c)x`1Q0z=K=thGLB5J4njVZRlQ&U8Mjjessbf-0BSrCGi_xkM57UYsSo z-XR4(yyjE(<qBv5KvKTxZs0XWng#M7A|I=F$*`M_aRMcO0@shfK8`jN^;%AQ7P3*t zn<ia05cE@{QLN6ee5CadmtILMTxuTViXCVM`Z44H`KcjAKC6R^xTQD%lhUB-{|i_E zt!?a_<TBGn2<XRT&V0o4IdX1TOU&|Z@Jpi7jY3X&zlbTsz{D`-lO<(usN}zDdU>?u zDRIYNVIqgsXqcdS;qnGp4Tw=F&nEv?NP1|{GyIU3D(3DpAz(2t+KZM6+Qf@}Gg?XQ zboUQ4>8=I`ig)(*;ghlb$Ot!*wju)s-(`fs1khsO6Jv+{j;RfJuDlbUsn#P30hZ}4 zfT*l7^RZz1ZNZ9fceY)}%)k_j0j4jWfMHwoXaTHqR$zAh6xmY!g8IQYm0^3Z88%MN z#YN$CzmtI}znbjj<kbT}-XjDJ8hA*3KBBOFo1HycZ8)D<O-RV_HEatx@Gl)Y6I`S) zU9=2ZprE^5+{)i@uipDr=Tpe{Pd7gKTcyezkMt>~Pi&f)zw~aG5)W&BsN1QeGe}o< zSY21T1FIO_r&9`GY>V)!gl!HT>ox&t$LD4rI(YZNeW<*Sg}FH__$b3@tg9HLnSU9+ zq*1D!H%Un6N5s*H&e|Q|g$wovO4W-AhTdlbw)o>Opr481BSGRztrDT;7w~9w;aW~O z=6C}-0>KsNNQ5~=y?sXfV5&IT$3>(z8cFEXO+(EfLx;%YO4QC~q#!XuF6))t(6D`- zZ)$nDT#SaH^^qY)0*1LXy&r>m?A^cj#w8<YezA@Q2A*Inva<33i8r1<l=h&Z?b(q- zO7L5M_<UUW+kqN23sy9o7I?!TH#R*wnY9s6*0n)WlC|RPD_N0^ERyJg&_RMMGBT_u zi^*kc6cpWJCn!zK*%+e>MQ|I5rpQnaQp&TloM~d6jaodR{$%LA@ae&y0dN>_o<Ad{ z#_uW0D5#BX+`2MDI2;fhUhvE)a55C<4Dr0I!>xW%4Cv5ch8B@K<LtHh6Qaj@4AFTi z&1v$ID3S?=WR7NLmvYJeD;JX{)X7TXnJ&em!CC861dR5wdbt_-r=JL+Toht2;BVi` zFodm`Ud9MGT*vC{Ax)^`iLJEXmRQB7E$kCd3Bj;rvYUV4+<(rnaz&ng&uAeVuv2A{ zP<A&BPBsIRjj`)WJ`XNEjPEuC0MU7b6%9>1Y2IW)zeanX?%Ia{4!jS1;krClG!W#7 z5s^w8tQe@6_-nimss#0KFks|ka`&n6k6p>@`(Oe8)tD|(<%}><a-3L%QwoW0q4Zos z@r09eB0}0+OH^Pm{1L@Vuqf-T7+r;5N4|)uTRy8j#SVq*Rhsq><GOBpM|=35<t;h+ z)NebXKcqX<?mM*F5YA6H{SaPDZ5So`i1;Y{vEC><=K3^Ucm-?OqTKlHKLg|;vuW%F zz*bmZ)@OqG@s#+5gjzgybYLzQU<VAOhz1@US4E6V4jr3f#?KndD?U&R7!NdqAO2oP zj7~6FXWx{zk=##ltL*aKr7lS6Jg%BqwbyG3?GP+v@JV;n$9Z)*E&TD*aac2t@!9@1 zPrvI6p@x<-VKbKw9>NXyLwcZJK5!Xu^8TNHaK&b?7#D2i-V_|ZquC@BG$@#5*A8|0 z5k^invLRtDgqtjl{s<HIV^}CfY$MMM8f0#hQfjoc3A2f6Wa1X5_fYR_`gbI9<ywI! z)UCWI8BbCDeH2nye;5{Sm_b*#+_t|nv(*>~?83W!j=2VD<Y|4j-`Ya|%Hv>>1sweT zzdb2RZoF^}6olOOuFg4gt%+@Z`uzWD*@vA&cAmaP4YidcPpdNUmJwV(pmw*h9iIEt z8k)*jx!)csD<)MxkG-=uyk0gm0|d8&d78ZiD{!BG`qF4J8hL1e9Tr(p_y5}ofr4)P z>s`zGjIsU0HW2WNS5BZi<h&tr%K4rrYJ|CA0YTGUnO4i%&8e{Fu(hga)UVorXq&G@ zg4zZ^SED5>eMz;UH#5vEOR@h0HC!yB=#ajS3DADVXA{)$a?}zivemsFBGIRvoE3GH zUH@K$$p<v`m61sUxp$rGzwxYDOV4`eIv?ATm#XZvtpwIk_txL*(b%8Uf_qc79j<`3 zp)5JCqVH*~=2c12b@w~SzfbbIH*QjEMhh6Vv}=EE7t@W^wca<noo9Xj(&}(sKv~A^ z52(uL`}1_5Wnwg+?sD&JwdIZUJ;(m_$tAu`SmQ5<>pyW>bhMDCQ8Y9(I&Zg$^BKot zS8CATJoH%eDhws%#=98F{^u*oR4xmiLHD%{_-x>uBUaWnX@NFJ-p6XtZ~>yS6e_ae zO<&`{L}gFTA!m=?UY2&?_mZst)UJ$B#3Dt-Q{Gi25JUAPdXuo^Wk(>Z&lexVKze3T z$nC6kE!lH^mXp&{=swqzWcILiK5u-%+s56g7gt)hNQOj+L-j92H6qRP{{CEe<|tqi zi#k=Xx?{sVXAY`E_1kTCa!KRb?+Zh1UbJNhgs4nxm_2Oh4d{a@rVR;^N&%=@eV+TP zL|Jy%7Z838rFOmhV5PLv{+XcD<<t*$SKXh7i!Isp150C3gxm`5y$EGxzO}zahS@4K z>%K2f6rPKe`O3nHA^+=Xf8V4jXni>$bm2;40H##tHJg4)$J=)2`oikYLh)8g#@wUq zXBepkZEDx{>h&?2f3Vm5^_%Ct9{WTCS}YY7K@HW#ZSdC@1EksdVXlT6YY{OSh1e`V zgy-*tkh2{=$r6rd@vpD6di<?l72ub^deyq+wG*5^_FV40CAV0TY*{RP-gRHvU5~N* z?@@%jb-Z6?;d7d1G`XIJx^7UBLm?y*9ATLZ4+|G%%BsWgZd6^}heRfiyui^d$v6wm z|L!5@wEQvMbv$p!kvE{oLMbJc?sCOHP{62@bKmCr8j1<lb2(Sa;IT03J;vv^&BZ4c z6qTk?`BYwAJtdD7<Nos`n>>rjXt0Ku&wEBbI=$psqf|E!rt!zID?1~r%+OhGZE{2! zph7$ac9WC;RB&hnqZk+T9^U5B^rqC_?D9cz*(_NKp<_nkVNm%v=^ZzUrwG9Ku+ed? z{mkZ(2>x|&>~+|{U@E)9VWIOYoQQMI1StSM0wEAcR;#V<!NeayzZ~%Syz?dh$oI0F zjg@RFBh>@FRs=)#BU3ln1<lQp!{w%C`z5s3UfI5ed)1(rm$fDx$P2ibVpeA`yLq?4 zZk-jt{Epfn(p99v&uQ-7Z990rs^=L8;g>)V<-`AlVoJBeLHr)0*g)=c(eYYMjKyL> z8(kvMTnFRxd=Fq=X8(rQi-#7}fshri5Ty)GSZm*PE;_SrjLN{4J@k~{CH}Epmm=Bg zKRlh?<DdF&ET8~uA2xvgMHjw@OoYOshkU*$x!nw;Dsr9r6faUNDOw@JP)YGrHry%+ zck`}pb#}#!KgHqfGm@JKxx4vVP6_<~iAlG!6*Wg)N<|yMA=<_M_h}o!6fn*JR^r&3 zgi=7fHl}EzJhf&Y(XIEOU$Am=x^TF5?G@Hy05Bbx)f0BiR2-`6sbsVK`bT)TK<iUn zoB3Ut0Bl@W!tB@MV~>O(q?-;X-N^7ZWT13fX>%K|wOt;{5cE>~=Gt4tT2OoPlkOMl z8PE}|XmBJVo^s5_oho~2Kb^#s45gR9e|Ivie=IScnrtnEi@EVI3ZN3HVwSpgqf0KH zZGbwZ20CP&l*D$a9N&g};lzna@c-<_VYCbuRi$+Y^y2Rjx7<N!)eko+l3hh@WA_1J zY?QDsKXuYXfX^teRT|_9MnFF|xNC)@a7+czHEIRcmiv=$Mz273<W@N8uWcrCQlex8 z)R^Y=PJ0{v{(7LvIu+D%(;O7`lLbyJ5F2~{wjIwY+}%90IW#4{P%(0Xm{qU&eyYm@ z3n&!m*oyIxbGc*5j2m?*P_y8S*Qv%)SwWH#^K+au^czh!>Jed0++H0#2-}>aq8*zi zC?sqvD8Hhv&sV*Eb5+{*ND|A91>9)J-msb<*eke@s|pVZ-Hl^~nZ&OQFfiS)Z+Gq} z=vB+p5Y8U~75!Uu8@X@3EqN`fo~OCzSh(E9f04a1dV<=YKh!kL`mlaKF?~*X9R=?E z+1(^EL=h4&JMuMB^LP-x0o7mBLJRW%QSA88@2|SGnmjwD;W%s^-n+6k94N^f%4}}K zV+jnb+umP>U-pC+x-l{GcNsL~f_-#&jH(x_oEpH9R^5nz>QV#qkV7Xi*ie_0l$Lyt z_Fp$&-uiQBDk@vDaO1O^{;8XB7aQA?1Ue|5!|Mz>zTK$qis01!qH})*dMVPJ7)L7E z>nbBGhG@E0=ze%;6l<VdcTX``i*W@qsuV6akwDlHgS))vy?3xI8Po@ddALv)=86tM z7Yn0BMpmL7GSZ9{@xn;EFpMi{hUpG1qGj1AVh0gXQ2f`lUW!9CV4MTy!1BAE`^qi> zA3nIPPujpTFQE*U(6zKIPwNtskHwkqQX{AF+dtErF3_V1r<SxZfqRq>&!#?QRE4T0 z=gUF|rF;dCHh)TOzfdpN`3evEB6brvuTb(aV1v(HJ|pseEYys{`GJ<VUi%pp%_7~M zSL(|axlz?&LrAfl^C`py_9+G`XIvY|K&i6UVS&Wyi$bu$Cr+8xqP9g|zJd=@0f;>I z6Uw)9)}CDjl%=>h_?!i<QNag6tUWzTX?3eNxGFq)o|4bFP*o-$Qoa&Nsf#68h4`D# z>+5RQofdHgmTjCC&p&M|6-b@8JD$fA=;_6=2En2~0F4w@&O5zS55Ti&LRD|O%;e-4 zI(9#<)4)B;X4bYq7ssvw<CIcVuW|4Ji%JnGHDp}74r{AI`D55E7aO_>N+nbyIes9n zSwD2k(wAt=432LU`n^bA$fu;H|D}f(VCz8XkU`v!!MxFoSTfrQ)rw(lP9$Rt(HL<8 zt#uC9IGKO#wYq<%64V(qnDvFqbj4Lu9r<^-o{xSn@A{4b_b`0-NqA^$B(p=AG93W@ zQ%ZrZA72dYyDpFahqSj0sw4cqba8jrU_pbs>%oIVf;+*3y9MXq65N8jdxE>W2X}YZ z>HPjPQ!^iK-I;qoQitMD)!lFJz4ltqu1EBzjNg8AXiR}Vp&h2$-c*3@U;+lr%2sfR zCNjZ0vhWG>bhzvk72UVoc0*X(;a^zpB2i$1#>m&(oupeQ)35F!2&=I@$6T=}XIg;i zG6)_TF1XC-hep{#X|tgBgC<n~l{9jYE^~2{|JNV*{uYb-d&};>k>l|Z5zmULDFzcr zg!OT94x7a7b&H%9vKp*KKy=pZ+$lv$*_P_YGZtTTG3X*Q>e0!qsKkH#VEZng#$*b$ zbO%rD-x~}DA%&B0Wn<dLzSDd*$HE+ngQoH6BqG3`M9yvAjLQQE4}o$tKV%g?%FGV8 zxEH23VUY@n;43BG(vOQ(%W@eC-1UV4?bwbgkgRx9{5`gphnakIRK~xV8nlSKMiU*1 zoy-#mG}n%$v!Ah)GF*6iJ9K2S*D;f&iBv?lfSK=4i4B-NA7DI=1U+sE#&{i^GBJw` zwG2|QS`P5~Th9<Q`a(4p1wrn}%Z>m`h_sv&7q^J`BSPjKt7`+JE5;sridO~W+7l>o ze#gi}%A~s1WDUA(iq3xkn_}RT=w=iE5i6I<ksz;f2{`k`dKMLI@6HCWej&~1WFrt2 z@Y~S_69r>FVzfEviTZVY9ZheF`W23xQ=;dt!)&xsJheS*h;Tk1E<J3XuPppC?4!@4 z+^Hrd1PCnYC$qJUSHU$GVD%1U%IG}+DOM)ncnNi@>zTZ1=n)X6r!L}f(2vD;KD!P5 z+JvpKe|Wf5<Kpc$70dwIl61ghSK17K1hnME1(EOH07ZJt2vKAyrny=)l1D6TRV@pW zYEPtVYgII4N5f}fnPTEr`I(-YOF<F=k{XE)b82(R03%u2PDxjW4KtCo#GTvV<Acol z;+7;*>&Gmfd?gW5yM8;s(R9?n=I_p&s}W`t=&ps}>!J`<*>Qn~uYx!Zs=<&No3p0m ztgFukO7etbX-nVq8?klBKvQ=8m(@WtnO+%cbh{mszTgQ8j_FD#$w3DwciB$xNV*ug zga6MSIBAPQD0UjTs>;feR}%L=*mxQ%BPN``Vkt6a+dpHo01Y=N`q~(5Z*R)CCeXNu zbU4>%IkJ-`Kj+`4=wnUi@cYAX8oj;<w@i>cOe>%i@$=}S8|9%ca=4-`aY)<KXl37k z<x_>xO($A6ndJ9)qX`d9c#+NHCo~26>BVEqn??Ig)i?emc5@gwALK^U3W5*Fm0Km3 zYUI)DR7t+&I?P+{v*_mgZI8x(X|h)MPc2EqQSzx(x!TSzI{r$5BEIz{uwnn#%d^wf zt&8qpTE@pt2+iUFR&S(iYem_8`w!W1F~Os5XUk;+VunRQbjn_{9t&qa4B`<p(}@sZ zx<v#g4`f6o=qi37IE)K@)&(W(u?)C#Qb{RUF?gTBtmJ;G>KGpq0$vzCL?_p4N_45! zvOtkD6MXuBv47Q1LfLPgKg!<Q^Giul#?<k0nwsk)9-4=iA>d5Yv+kPNOA)-hs>r!) zO%e@^0-m(@<ix}>gmS$m{lDYF%hEew3o7P;0ucf~5-5btXm`&@mhDC>myo^8)K*X~ zh%CPg!+!qvf&93c-uLvN_s_~%sdUlTDJMl{<!y1OuQJ(QZ9?-dybENy*aDAF#0Eqm z$t>=0(b7Aw*-JT>z<WAOP8PT3d*|`vBBrG&%OG8>)`n))sy*YD>H3**bS)aR$7LY} z3DmlK*Nc$fTkg2s9+h>sDVmHKrh=1#4J=>_w-35Ixo(KS5kf<BmG!FjiVkK;X*x0S z3FT%F^_U8sCA8!t`MSp@0^V3sB5pcgskjcu;ah0*kq`hcrxLHg8yStI;KM%*$xG{! z#VrBqbYr$_b;##>Csp^DmKSCq*9q0yYiclN_QQsuHvlb{;{_b~QgF*&PNB8&njo@6 z5}kK_TITnH#>RF)O(_#J>{_SK7!;V}5KV1$W{Vne-3C2!iIM3eA93AkXJho!oyDEX z^%OF<CqX*DIe8WR0G{%buV+=Cb$|QUau3pF>G&J}dtB*d)vwt++%E&Y-Hq!~p=+<7 zk-}DeLxZkmB_I0V&0kltj=dDu9UAt9uiLB;>(Bj24LWPuud}K)HG$9hv>s2$?WZWu zt;^cK0uOhs>k)=GaDa4rn^rz{#M}9!YC_id^(ZVxEznd`cxA-6LHQlVU*5HCK5Ttz zuMhMYh4lI$LLqSnZv}nYaA&g5HBcRM>|q;i%~~a!Q<rU0?ecD|Bbe=Ualzwh>mGbV z9QXC<N+Yd?f@{oa@p!39{=wkfHb<p**EY~q3%X9HIxJAoLjO$+b<WmL(ECQFE#7eT zTHzNnz*n-M1Ny(vukZmqBaooat%CMDgKW6J+**{<gPL1q7ZC>k;(LPueK-aMuncqs zU?=xO5~8iKN~%Li3Z4`F{}w8asG09p?YP@%HyR7;F7+tahVIwTxtqA01C@TNW%zIn zF-^LKi?;RlAw~X8TkFT?e$0w~_XRYR9Abl@z4!6|@)ST?0x9tv%kF|cz`T%*XZ`<4 z&iUUYARxE238ik8mxU;q>we3cveAL{*RSlI#yX4B3~&jwS`9X<KKDQKb*PDKU2YF_ zlr$N4vCWqM!V7L9@n4;Avp!-_>-0<8a~dx|)mN79!83vO9`_6Pd3hMJI~u<*Fl}oP z_)&E_3xnuw7uw%<9F9D&-s&M=7TaEu>(LcZVYh2HqCg0=#{AafE3n_X^EH`H<vqr0 zuj(LUY#Eg$S!sEcK8Z2QD9&l0p~4oVmQdo4x^|2JB^J(yT-CjuMayrWZ$K5$<H9t< zKv;NmwwF*FGe)&H;~aKj87Q2S0y0T>G(H6^4@vr5KLj0WXC;>Y2n)vdrVU{5`I)x+ zuLg1<yDxBD2jY#|Y6EygJ!o5n3Otv!4jn*amYSAU#=t(4@a^66&B+lkUQJ)kYiMwd zS=Dwh{WGoiWW{&<r1rge82kB*hF7Wn#mY>!bnid<3_~9TK%cRgFW0mCAn0v+h5kV` ze$V?z_^XD7Wrg(*TDh7E_zqXrIfTp4&CI=OKvt_m!f>u59=apD!(K(+Z+STB@Y9!6 zz^z<w^Zj8G^9jShv!QrbQQb^_W9^Td5|6ONPmA|F>6p=G#Ds<uw%fW(S3B8s`Zn*w zt6lssMDe_Q`u!#TISd@?M0&(X+KGk|a<}7oriucERMYp;jB_HOHueFyRTfN=pFK3J zmeyl0DftW5gc2`!X}lK^To?&hh*dC7l@~Hh=%59FlG{n+DIqKe7*-}(3F&?L28Rls ztgUKy^5J4&HfT4F|28(~xFFfr$KD@jCI9nF7N1w8esX-JtL}lx9WDvWY>lE`?JOhE z7j7@sxf#p|<;O&q^-Y_78Z~&kRBF-p$c#&n*v?^g$T9v{P?x$sDtW&D(u6<~6Q?po z!}I=YoQ$0^)){bP=%brM|EW%-b)e&oS+`sLUGcKz@`2#E>}!=}i8V_HX5tLo$?|d! zOUH|}M`VlqPunU9flA$yX%E9sWm_wP?hndUBeyjsoei5rl|C1fM*n8owk!)<v}PG? z)X_{Q`@0>E1-96?x>f*^XO2tTGTHT-@U@eq7^m{~%e>D&8|+5r+iReo^{Xcwv|ZA8 ze!mOAI67Lw!_CG~DPaYI5kn5wiPhu<Kp;B3xdbeSrS2HidZGMalji;l<ztq`=nn5? zR#93iFso9j<fMQ<sxwGHD_Efo!bO;^$Rf`|2XT3xz&PEX38^tab>5O^C6+Cuxe&A8 z`ey#VW6jI=L7_86&{9alsybCl5mp-sV~4~FR=3o-36gd64T+~;6Iyfbj?{XJBuJ01 z)HTUni7?M#(C6anhIZZ*_@~R5{dKLQ<5GAIN93?_uIF>u$|qv&9ucN7jnVfp!_uF} zna^Kzr*=mH3zKgdety0|#x^cijXR$f0tG|W{4-?t6#%>v7~gna==&Q28hmlzw=GGb ze#OG$g{tF^w?Oii5Q|N<F9ZC=Id?JgXCtLO5QsIi@d{5N{mO2*4&;>gdX$70YJjQY zzxM?~y2sE80br>dJ>2DQ$6M?&1#8!&D{Nh=&G@bb1oOZ5M`jmu?VX2j^8$f${&JpO zr+(ZI{s%+9)H4JO(rbgOo6Js?edRLPW`Lw*=_zpQeW6+d?VT-N^5548G*f+9?th`Z zs;sP>I-cGGYzz$=M}O5EJzhf;zP<Dg5mBQPV;1Zize~s9S?rLqcw>IRELrIA-3}RA z%Pxx(7??;+FITKRB|dQgU{LHag>pIxK~4Q-b-_QjlxMzF%4*#Lc8d~|R^J|}&7qIB z*73XsH;M1MzQn+-g0bf_l~c#ZlJyA=2a~XXzOXwNdyDIi!yP*odncVbGo+~2^OXNo zmy(c1Bu?R_9yF8V<R*l^+4&rV^3UTi4vr&<^FNxei;<#~Ph$2fX%=<HWQPEHcrJWq z-RbD{S|<6JJ`21s2a0d*M6tVo<0(zpf3X=qsMqHbhf+F<t&dj8Qinz{e9r!Lr3KmQ z216I{D5<GBYRhr~)iu}T))v-GrODSNH^p-{KwH(n*Q2fj?h(UH%FI~qSD2FR&x>4< zF2_7+06W99cEF;<y#Lsu%Yn<CrY=elu-n)c4uOmmf;zySzehAF*{SmSO9@PZB5~1z z8yfho@di`wld^}bid{v5jGsS{GgGuZr?0d^?`ewV@S<O0wJ=WAfF4wn8E0e>bIkU4 zj?MwP0p1ng{dlyr+CMkMNtjhC2T<022K=74?M&6!Fdr@FNiyyJ+?}X7&<ktj5)WY0 zj{;GlYuAOdw!&=Qx_r<k#$Xo0z#4XXu#?Q@)<e>8M&BL=?zBcdw16Jq>E-9YfV^&d zI?`Zb8f{?j%jsGYuTMGu7vdMA;ZhRVfC<YogW|OX0AAnj*L;Tk!)WP+d99Op7wLgS zeaVd3aN^oQaJKy+Kv7&SM8GUt0en<uRK2E{%^$A6+4g&$BA*cc2`!UB=RkkRHuz84 zMKCaW>-Mku*NSGGtVtP&_^#M?i}j^QjQHqOELsuivMZ-E9Ew*VuR0ChdDC)iad|F@ z(3sh-9`ZffqOg0iGL_ze>;_e6Z(N6x%1X9u-?v~-WDvt^Pe_UeR*5LhK!GhlzK}y5 z3Tbpl3PmHEx}$^weRW@jU_oFBUSXl<TYkNs=t!aMVCI%^W}6lk49BbZTcWbH31nVc zxk(#&ZBqeQhs5dY8W-?9W+l!Bp<ZJb(xwWsIcNEsfMGlo`?&xR-@Q}00>#TmpEiwN z@NF~Tx3OF|h#wk!U^gO9M!4jbjS~~*X8P7-ja5Lx>pqB*SR6F;t!?rVvbVrCwDMaT zC?#?#Br2WjYSUN9^PUTc9$C@qn!NPVd)l2q?gNvn|43?#_(cWBtrWlQ0JNS2!%9RZ zB|$_`5Sg%dzUBDjTi~dsgDK7DG-FS&nuL0UPwZ4@e=$=GQ#T<%KRfWiTOr=3=JriP zLKnYOro_xO-lJEfS_wvt)88e?5FZaD-=wVNu*Y?R)rpD>|B_V;6)GcoXUZ>A6vs>x z(ZeP4i3;js0lbw!lo<QkIn^)L<P#`_^@wrWDa*T}>y-@>xZY{`zV_Re*D=c;Yf51J zgW`q)cp1_OPlZP4y%AI0p$nrN{x1N7LESDbv%aaLl8!#Sz0ek?4j|tZU(WZMP>!Nw zMS;VgxGr~30YO(RyQ))q;5xLpttVIVxt=t8lDd}zt={AvJ1d9^AaGTX!hoWaVJtpv z1(?h8!{>z)ZC@K-Y+Zhk8Jm*!zzRrUI9{%6B4R$~An<vs+6pk5WQherCsVVx-{$^G zkvnujQ*PBBEBic!q=tuIFK4@3e5R2J{7LO6aYtg^7kZK1z$il+014v05yB2f9B%pB zcs_X$=d*~cPn|$uwD%z&Vo*!R2<;;?v>D^24A@3i!=7~cNAt#%m?{Pwc2+$ky*w?m z9ULISJo@)&YO0ryOPYg?%dD-kvQ71I@DVuJEE|`xdfU7v^c5W9nW;Zb*C~nI-pJMj z2i*b1xQA62Zx2Pj6~z+;X54Q@0(wjcOWR6t9rPtY092UK(M3-eP|+>7?m@sSaXuTt z`cXKwo3axe!R1J)Doni++Ki!R9|P4wLlTg(sY%65T_o2)8m0+Q{I2c~we{U!c1j$l zTrr~nYh7sqvR(X9CNI6BZN@0X{kP0?b8vc6aS6w=oU4DbE%|Lz6aXGnm2vHwO~;FP z)6j+JmHXg5{$h%Y6&7*4ntUBbiJ(A1(29MP>x@&vR5_*%7xGXQqXUD3Pl>QR61W&; z&7{|^+6H~bM`;yFgnd{6sH8`-N{Otg$|>f9ICgFqv71(gC<Njm(p=QvODpqYp=oBL zK8VQnSkF6Nv|=P<8bfRcHM{>Qkt}PZdGDmi6m&On*UH~<zdK%7kJ7au2cFDn4GeZg zsMDBiZV5H{NUmH_kW9G4;)Ti)1gn0lmeeu?OT9$$2ZrjKv7gN_r|*Bo;aWZVA8Rhb zL@m{1#)Py!avw61sd?ck%*7qw%h)$7m|(H+bN`7rtrsX?bRS*<+++Wd1V?&lSRJYH zflCVmT-rXHeiXZ`n`v`)JK(0a5`ouu=*rUcqizd7WsJ4x;%7fGj|RjfD1eYQ`FN@~ zy5C%1a7ct{s6!Pr+d_jzZK*yl;vUHA<(I`@b^$eoa$Lq6QlDI$?m|9Dmj2NtrVYsb zy-Mk7>MYfprC*`Des<2cX71K1`(keKgyc&|<1I@xx=eA%2ovA|FVb*6tFT`GQC>7% z^W9&`-gv>Z{43gLt$I)%2|H~)p^hK^$?vYMZV2&3ZWaT5{hW^yAp5hI$K`EzukZ%M zT&Py)KkfsM12Om|p%b@S8G#lLnG?!KfHAOm!GC&tYzrLE{<x>J?g@`<yQ6Jlyv;TG zjmo%%c1M#kYdkJ*K>dP%7a`0AW1Sdpp&@Do9(wO7Uky!OJ=okHm_Zu&?4shrMW_?l znUaf=$kPqC>;%b{P!zixBkai@Vh7<>MVj~XuFn<2{ySzXfZ^BQiGE^sZY;BR2ah#? zOM&_d7I*hTn~1M53z$A<%>V1#wVj84(9?}gTaSL)veKd1PtetwG-$ScJGJoiEsHi& z#xLQ+#vfLQ^v>@#Pvgj~;==nJ^IaFb9&pjLMAv@bLi+*3pOGK|G^Otvl4-dELK`_! zFmg5gNk~I1Wet^0<?r8x^EB1)VcFK~?cTWEeq;V#5+{qDY~ZZZ%SF;?Olv&a9S)cX zkMRj$JmoAhj-~kVep`WU`pACKb`xtkSZ3;rZ|HN%F=OYoKx@ydra=j>o7nv|4Y7Cs zil}C*UHr6S2E};S9Wu4?rZC88l<6t_j?yu|H^rQ{jaFXq__CG6WajZdZ^_P2j(E=1 zLi5?5ABnGk_;a--j#uQu_K!gVjOdzy$FR5kWv^;7sDRiLxVTHcpaCIUos*Uy&x?)r zj8dcKD2!_%V-=o%d(KblB9>jr?jPP3WINYmi98JXlfTlnEx6+YQqT_{{j9NPzUM<| z;ku9L)vy9@7)S$)V|QK1`)?<Ii%fQFpx1p21SbO6uFmRQF^XRsg*th-C;ZTzTP3#t zg)>;l+y~aW`Puc;FP6q4xHm<Lw2rSk@T#3wtlfS%&GB8;cbUhCyn3Eo(}|v{R&EN- zTgRMSH>R5J5WVck|2etW``gri(AFq<nvt*{)bT+4K~_JdbP7?6gfeG(nxnyW^cHF? zu?Z}Nnd}Dnc4{M|qh;OV{U{d*hg20T_+6s%k_L#F1L+1F7jQ&R88l`s3-m=lfn|r= z>mMftDP1CLL38q$Tw#ky>sHzS5o<s@5J^XUpyI9gU$PHl*#Cz=+~VQuU)NQ=wNxD1 zcw=WR5WU|28M8%QeOQBCpRvE2<81n!5#kAsxPP^7;;J|unlYDydV5~|2hp}iOs}gd zY?-^=)Bd00Mn~mFgxP=p-1{(=1xw&Rfr<Z0?KgA%GqF^({2vD~{-47ToZ-#DWd5(s za;t)Msq)J4rHKJWwW0s<KSYf+f%4wXy?*F5{D5jrG)v$-tW}qCP*LUnZGZ_K>z=`% zy_k^Sy|Ajy?uYS7zxI&T@Lm-8G?Az-{{{I8hTDKpGm`M*`F>h!E)l})<3{`2cG)wF zev)v<rMMRa><-Xya{g-`8Ha_@gEwpt%~<HVgGe9G-7Itg-GG((=lyD%SN0dtSE_9f z#0wE_rCE4D1VQ-$$O%*hYY=9pCCmh$ZI*to+N{^emyIvw>Q^B#7^{|C-NUe&tisj# z`JLGfe!_6~mCIU|ORGXYYv8gDP2#9HcVS(LL?u3+Lw^G_l9O~U^Y$c{LeM9!Uiy`g z6y>eN_&SdGZ`Mm1PcvS^uko!G;3VH~k$8oSU%K}A)PO47vF>M`Vj9=ivLw%mHoy0F zUtzB^Z>B_rKA|(&J{7s;X&RgFnSyl{mR*nkDpdYs-GtiFpVLyaU#f;C9qV{e@d$)= zD*@-Zz7F#e1+TlFLyjsn>^<G}$bI$O;IFM6AVqBIyMFs#Je1qDB~G~E=X;a-`N2c` z2z$E7m-BXk-Yz{My~<4d@`T>MRg#8dmmkbeJ`E%%<v_7cZ(R~{!kl4Oxfwl%Cs-5I z*3XJ9l~r&)%eA=Q`bxR9MLYr0rJT0^MWP^d0jSRPgcBGd&uU)&c)CHJxpQD@HMeq^ z`68DZ-~ljM+iV33p#ePHCQEd|7|t<;Tpnj##m9vTb25WR(-z;p7N>uf)V;~gyaqF8 zdLg+MH5n|s)Hfl=la!@879IJttCxZ+kA)9y8u^F#?e<cV3Qq}+-AND>@=q~u#l;u5 z$8Rfw|5_{RzYDl2fRm_BDvGe`RSHDJ@G2#%l?xG6_;RB4>NOd=`E+uFZX&Kj7LK6| zpW)=dO+Y(5E9`sK8+3D6l*rGb_uKdh6=eY-*r`(PWAC*x3b@ou`{)7i)BM>NWC&7y zC-Gv4ZwwAbpJ{|xUITcj%X_6qL_luj12s*5D%K*I?@LRPRb!&`Vcq@PEm(`8amPS^ zA;$qY4j(eZ{Lb57+28LhhU}|2?hmG@IPAK_|8bQ!BN=h=<#g1pg?Zk8EaP(rk_sgq zBpj=KdCuacSyZ-MB6Pa{aDDK|l|3jZ?7UMvgf|f9v0sMvn6Ny<ya^!(tE1D2Sn%=c zOz3$Q&30Go#0x-RC9w=>8J%5u0ItndKR|uCj(v4xd=Y(azAGHl`PcvM_E)t(Y0Gn+ zkFC9@W355F^_)3yd!Py2gF82IGYYp7XkFSbRz{=p*m(|!bMM`#Bc=@<a<EgH0eqPy ztn{!wyXI{6e=NA{_(8fUqiVf9|DfbD=HYzgfe>s_;H+Y3BrVs&wUL!}A03I!k}ces z68Cobpo1lvzJ6<ynItBl$)4=O;L`uz|MZ!nIL^47{>VtjAtw_=3St0rd`s2X+B60^ zN$PqcL00OveeqSAM~xwc+<GLxoZ1>9OE4c`EpKMTQS$i*w76Y%`PR6?!$+-A)g<-= z#LZQX#MYZVW=1oUZT0{R`tf$^f^J)()a<`m4=<BmU%`VZI2F(=pK&Mr(g}#}bKfnF zwJom!0$buustuEN69+(rw^Db=S)Wd>nm_m<izDP4uV@HLk^RhMAN1?|hggXA{zR6K zC6;OHI5@fgJ)ZJb^69crkb|<@$q`TzH@}6(`h9;BBkV9Ll%heTZ(Mw^Bx=AInuK3T z@Z9a8%H#A24rFz_(WFMmO58UWQFgHWD<&Jhj=6@wcRmjQ3h@vO0qULAMJ(w)Wm1_b zN=7pbzJD)lXw+j^LifZ!WRR&<mtdxlo&R`|V2#~$6))naFO^e|sX!j)<UDEI4zzK# zJMyjbSI&`)aWW#ibQoTFv$C=KZfOHME&g$elYtFEXp%K65AA?rq8F(;g~@ly@Bb)D z2>}XfnRvv;WT(M@H{ue@I6lTl<0@)f_sfIYEJwIAkwJa!+*RCEdAV~c61o61a>}{~ zJAyjA!=cB#4kCRZN0)pY*Bb}0Um@W`iU0NZz8o!Nm-Kv5ZVIKUI+L&Wi7Uchws%~o z|M8zqGazn*YdcZ=;Il>RjW|Ux(0X#3N8@8Kuk8a=MWUh6pItz8s{}+t#l^YgYG2{d zxF50TCu^Jq7pMXNdZy-3y}}>G<y85kc3s^I?e5ShR&z5$)*pPv#j%`gA-Ksv-UXav zMi-YES8Vl*fEEgTsfd@`qgGm3siKs=|D)LP>Po>?vC?x!5dEEHC>%K;EXR>?3TZ;( z=zB?+5#X{}c2x}p6oBF;31YVPF`c^EPdUuoSv&zapKLP$3YDi;grsH*nF>h>Nh=;r z|7ADj?OXHr`&i}M0DbC2d<3AdcMzRD@kBhN$z^%wu_YGp&23C~VeCb(sf+utL1s)d zmS&Q~D8I1oKp-Wpq%1(lW)&Gtw(sb7(*FleBA;K5W}*Frq|F={dlQoTgr44@BQL!9 zG9(b#cO=>JAD9NyVW87cCU{3<K<4A%0f`0|t^VIDpkY(m)lieo|BBteC^2Nn?0ErT zmZLEJM2l`HwjcSaO{0W+2s<%6F$#AFxTOWY4xAi3-{);jXEhz9%+p$-;pY%{JnypX zWw#<zk5YC=_5st&uspUTFZn5;po~SPIyp<oRFmy3*rs;y#@Y9#McyvY50^4T^$&#C zxty)Qy5LZ_-`Qc)rh15{Qxyn>;jUDF8IMXODSF-{1^ti-eca0QdTI4;Q6;6zEqq^C zOZAiVFgt!<|1EN(eZO`-)^?~^l)z2IHO@gdj!j7(@B<nRftMU<D=3U(9D6*RcS8`0 z1`TX7otZDND*NPpZ=M_q5J^FQ+_BXfvKz*+)?3!<CVaEnCQ6n8FaG=iT1Tfjm9@WL zD!WVC;+bcZCb=Nx$6883kpXPd*X6Ku-Glmn@Ca>WPy(`tXG+S&MH@3&X-?-I-BaII zibD|ZLIc-vAK%^Afy87HHLb_1xw7TQ;@!twn=k5L{<y6@iwf*(48l@J&Gbw`MkZKG zCybb~k*hKZQYoHGfHTOU2Mfe!5woKvR+JDhc0r}opJ=}(LSy2Cx;>_LR6lL9>~C6x z=0{E<euIP98RSMurt6=;{bh_7UXzogd;sLl7WN5QKWgmbH{)Xxz%4B=@4?B#h%$uj zcW*kfkkEa6eHfbQ6!ydgG6}b(4W)PF%CQML<VNo*uK@F5PwiC@^fSxZ9_fw$MgT4A z9@IR}w+M$+sWM9whu}nufcCB>NUnslo#xU&lxaVvVLe~EMK#xNNG#-<i{|(~QhweF z%pIUC5wftSjmcd1Go_<=Daal-)#v4$zwKZ#2MofBfkas6t1m^|j_FPAzRzkY&de<e zcSn>%xaJM(c_VahRD!xOaK>HA5pbUf-UG4qskd+%&b!)VwXpSe_VBINsqmiBiAsHQ zbh9Ai$+gLaxT!gGW)y#{OGPZIfcd}oBmGTT5b!z+$da!2cempAQp|?GOkp?viA7GH z+2?a%@PHMD<R=CL;*=mlWkvf^p<mRfUDGN@<=c-30tHE%qV0|FcDmV=t>@62uF+~z zs9<p`-n?5=sXK}pZ7Ew&fgHBW?VkV#*v!T4^Z9#$CD*hV9fH31eU{Hnus8vMQqzYv zu2$-R#}2o(8QsIOSY$M?^B4u#)6f$IvH?SZxga8`8Ozdpes52n$PzQ!<f8a!qK|iI zJ)tsC5Wz!*oDTUdgurBoT7Xt*g39r5s?6^B1?G?`gB>k(JM~yh&on%Zf*N5-lsUi0 zy{N@FtdH8vG4pR)_?EY0KS+WWq!Oo3*^nO>;QjXDDss*LT>ya+Usc~Sub*#Pr}a1n z#3Y_Hn)qv+AR~iEmNG6%d@Uc1Lb_3moq+#5<0en3LJX<I(u7k8f!>kXNgd^L^fu`f zJSbIugo++oCfuA#bte;`;C@!K-Px8UbaylwhpiP;77cs57G?ILUjA5txOsoZGpL7| zZrpBG5t|$*-wb^JQve$bA33uy2H6k0Fe%O!8(hCGaS{-s793@mG}a@>z4%eF(tFp| zpq)i;x4RIfNyxz<bWxi`WoWmw@)l2Z6TMp9oFb+yXwa({yd5eP{h_KIalKoRZ5yLV zi8IEc#rL&v?Y>kOI#+y>9Ag4}9;hn=iQN*r+esEXZ{8luhJ+YDEU>zme{VGUtYJ!$ zo|<{wh?Jv?cPUxl=5fsJw%IL-+Y?3;7V#(*gkx#qO>610mjI6b39$~5eg@<Cs3;Kb zj5V0SttUQSwhS((x!PxS$fjmtqIC-?lzb3*PPyqPU}eofAe9OO2ksIMwipNTo&qRd z>w&EeBlGDmAZa?BJv(KXb|dGY4q%bYoQ5o&DG6Raw8O@)1B(o@%q(GQ`KSoR5EPk# zU;6z@8MzGj5(NobaX-13lxEa-_&mO4P=}zkh*o|!1ri<e3%0XjeJg!Q6}L~R_CZ|7 z$2VD=I7m}ie`R5#vACjh^4yjAm-sk1L9{+Uws$!$wcN>OIB#M;2PAlIt=1G%7H9jS ziR}bjODD#OZ?zCik_-s+4k;zMg>+qKAV5a%+o{6lA9&sp7yHP!6Ji0FiQ?c^kEVQa z@c07?r!ck?v)A2;n;vKg4X1XWW6G|TDg%Dt+P8*6)zG(quwvpz*VQ9g%c9amrxc$t zEl2``D%(hJ3{Iiufo>WDEnqO%nYlZ>{o@sN#fW<(oUs33s<7|PLCq7as>veBFq!#q zGev1Cy$Xiqw+wdicsjZ#-$+cnE!MSEmS6~s%u!TqF}6YqBD^E=6&cyx!M4J0Lj;L6 z!9LietRH}+jMHjAla(|`;`xqjy2d{$0lcB9s<Y&X`h|f#gpk;V7FvASOI50H+11)w z=MP<PHtRS6aV#7ZVeJO#35Ke1#x{qlbmV-CLBC`Li1kev#B&)oIu2Ud>23tFi1kIB z_E>ac>U^77@X?pi#kUc;^(x8wOpd~3!7sxM<FwRU_c`Cfg6tbPaS)!t2%P9FQZRod z5Q4(nbKo?iTpk8hR1g*9*cr2TNpdl>lF>GN1+f?QIG{;57i`<zwdMkqMldL<8A`Q? zVD6NE3M_0mT|OcVfLX^QhDC87&xbLn#lF40I*<a3p-8)R?GQ+u9<MZ9rUecDuKw{v zIT|!TmUfpv8{@d<dy&Jws=)877I<14oS+EPeGq|8Itbe*Ke~49e2Fpe51rojz$5~3 z)A6bT@AAD?_fSq@=<GzuF5$HTkz`niR<)<l``wA0DIO4|_m``Ka{e9f=!g#>&$ozs zOw_Ybsx3{M=>$IRAO>G-IAcnxFx`XeL4rT|av8$WukAS7M5^G(IlseZQ^t+XysrUl zWKb$EYSC1`LlItDf5G3=Slge%ZJ)_KlTZv`rkbcR3Q`8_Y1*VX^oPsTQj>&cilQxk z++OgLMlu>&|D3zM;Eoq<Ue5l3fa*cUCi4`*L73w&MZkgi?HiPc=pTynV418-SsWN6 z%u|X=*LImHiZpk>NG@+xXV1q4;ObrdesF_h-+!5H^GJ0j=|3VS6+f!@Wgnf+^jWDn zOake*;w1MEv4OMgcTyiX9QH$jNWr^Sau`zVKNWcsA8t^@2HF?`2K(~z+RrVX%gEQ9 zP&-Ir;v%G_?^ZFlYKSMN9>q*ZTaOyE#V0RGg+irIy!?wb>pQ!D1!y@r<1&@-D^-iQ zwHZKn@V30fizeo>r=_zTkw}-P9oqZ~@&0+O)1wp@(S?ET8?K8|1sC-&gcm0Iz1>-M z-#K^x=NjlEXA1;>pVC7})cTPCloWGFwc*aR2l?%t{9~Pa+{KQuI~PF<L3co$ykqxZ z-!E22wjfxl<ei|n2r;7_V+3R=<{?BX{6OF{47^C)c<d@k=FidBJ}~Wx-<fKNIv7VB z2NqA~%lrO5j1w#NpL}A|nOvakw2nh&x-#Rl2G1W{FWSzRyGZpwQplM!>BvXUtAL)C zcZ_8|Yif^T;4v~KWdPqv7paS4)HO)W2YjdU|JDD4D9`-TtRsC~g(?yf7eT?;D*<pb zbRKi0dPab<i_Tt>tqOpT;L(Z6aq$kPs&1z_7D#c5L`4touD!x*e@`0rR>~5#6$uHD zdPXOiF7=$9K=kZTPt3pLNxY#0pTy+IEC9cFn<l1l-8_()gxBWUvq`1h2{x&RSg91! zR!x%H{Ad77jc3fYTVmsf)K70<H|a!OWGcmS=&W4bmF>@zdyHPuLMU`!{_KcvG9C4* zMa7G|beJg{gd^1%_+<)eC~ec=mm-d5=;1V<FF+_-++{p|AFAE0Y-pIv6tUa6VgTNR zB#w&u{$C#R(R{i_DJuxV>#{lA{L3^|3aB$?IEpM?1`e)EuB2s+j}pF?YnE_}A0Eo| zjW2nQl_r+e+b|`*<W2;a)4j_PE_fiz>1y0s`Lvr*;&I!q9UXChcMWzF8WvJp#Ui(- zGmw|HiZ04vbLpd~^@R>?PP--6r^L$oV}AbjM2Nb%wN8!S3sTf?4@(K`@F@DMS5mNx ztIfu(g~8L3!RPelcyYUxUS2MBJ=de}`+Zj1DO#rI5^q9FtxvLzh~tg>MgxSISjmA5 zlSKyfK#n1IxEg(0$E5ArZ@oD>MXNfSzrQQ=&zl@(zG&3^gz`B)bz8ltXVV@B&UozK zQRa^z_1Yu}W@e3Dc~g^oY%z?MNu^o74yPhXO7IsRPP3pSHOXMxzxpgYtA+XOHV<@e z0F&5dZ6G>ZIZ>AWt5tdvLYG>a`{blL(`|Uv#w@*OkRA#_M(#rX&2=2rzzwZ9GlGh; zOU#z_Vy9sX%R;_7o2l3=o9{GEMc(-3#!ktM_EX}1zO(aTE_J2qV8q}-%(>sj6FLdQ zzL0}AXjmqOfNTDB;QUQqeut=A5~GFnj3`)8ZWwRGopH)Hnt4x}j1oim;4LnkK<+V1 z$W329ktHsK30P8P>ah3_)^Kd47jYxv+WfU&(5xG*FGOb<eL;u$*Ftb+hb}Yf)gIMb z+;){F>W7Eo@TY}&Tj##5P{3i3Zx#dgre|xl7-Ijb)gUK1*|hGnubBCL*1Fn9OR^x( zhP^}(34SZNnL8VsLNe7h%WZ7=_Pf!c@ukwhU-@<~WjT+r-$XorEZYAvY)95Weqa5a z$KB>`8H~gz2fEgxhn>*Wtk2gK!S4vUp!qd32y5O^Z5ro$5z-^w$Lh#w+wJ1$c)fbW z#dWrk1IGg*k+fWj7cGjO=@zqDtJTYVADr60-7NiD<>C~qiN-~uGB*-YAmtU#+d!QQ zr_3D~8a0_5MRoGmuR2z@f(U8AIK#WJ+|tAN$1#^ikCXlP`b%mK;LQ&S`q)WsYT&Av zg;Ebk80)tX=1;V71K6|asG%xrl;NhBFcAoR^1r;^r=byWd*{a^{^j_%WmwHbZ*V3R zmHKO+{nwr>nVQf~>+R3gm?z{^*P#foz#501C;}lx_@*YRL23y~IvE7ozRrSm3VcfZ zLDbsbDI@8hZEq9r9y-=F#Q!enhWp5;FU8loOwrsdetYshUFUI3iLV%+XO^7ggh3;> z0<OPZu%_@vGx7??fZcmVQ1P!mPpGP1wzjrTOEcH?<83U^k$&PQ8yRlrOGPSd;%0#l z@GqTNZO?P5c9=EcWNEZP12y^I>zuw7+jTkMk1}8xHT3+0p36dG1u54C^U4h24g20X zpaz{7Q;<w|gO(l|V|G))J0r;zv0i6tSRNM_InQr>e>Oc@Xnn)Y)5s~0%Dvx0(M9Ga z<a*mo2EOxfaw)7sZ=pJjU1E?J$GTn8JTw4np{_1VsHDn}`Uo$#3I`1bAW=x-OIa|a zGCA!0+at*xyTbYvQ_9f-ZbksrV>S7T#Bxh!G1P>gp27b;Pq>Ad6zw`eYN^#Mogt2G zo3;g**EXN;d}wu6XACcV|7E1j@N=6jz?h<kCkJ~C2IwNeVGxx4cKUR3M{XSNTevg~ z(awauzSr}%q@6czHx1-CC|SpoJh3pZ(tdFadFYrhCwVb14Seq>OV{65dYu;*7iZU$ zP=raQo={!n96!X$1J;pQvB@TETBSKLDL-@tS*JPnjiqs2m895@&~KR{E-JxR2>4<< z-I{=7u9){;;d|wxax%J?)uBE*S|6Ch^^oHyUpu6<K68b9Q`Xth+%XX(M%2dOb#I~l zl_@|e$odrhZuBfvL$+sh`21TzXI4bi5D4_C>*%&vRfMNxtUxK$D5k}iY=OgeLmoH> zW4$(3D8za(d}B;5sPg@48rq)GEruH#8Wc{vTcx;G%o84HvQWi!^;=ZWHK?Ze-mu|M zvave`_bE9bWPKL%n#P61f5nbOr8jXh2??>pb4#7nkak2&WOP)ki-^2kE=U!{A+zAo zLJvW}-G!-atM_e*Hh&PUa0M~9cHWdMomvIj$BvV7IyV=7Y4)?WPLGFSi5UzA>oj6x zb?;yhONC7v{-MRsFj$9_Se~JyGmjh_1WXQ&C0Yjicue}sZ{sxPrdAAbH#j*$N0EaN z<`X}t^$1ZjppX*$2Ug$~Lg|$4zgFoxx%H)fPAGOAs9a?!0738_(HkQR1jFRcz>SS1 zuMDznYd}gVO-e8?B9`9tg4H~INC@tDIbHJqc)tE7=J*~jyblPYq$NT3e_VMSA7L>- z%uSNYhQzV`{Y*M~fHbcTm@FuInrYJXYU{;Gto-9KF`>AFQQ7yADe}XrZ82)aHMb;W z93%xa&!SjL@#i`$4%wqbWfC^)hVhc+VH$eozAt{fYaGz~?JYUfmq2<a-R7x!yFy&s zIT4#CL<@px5e?$KuM^kyW<HawSDYPhL+2)OZDCg=$DeiyBc^P6>0V0Px_-N`Cba&H z9Q(GNmT%WIcdm@pWf%Wt+p{hL@8IF%u$Cwc^^^#?_BwfX=FN3g6!1VPX0Xyrh+Nzf zH|hFzF`~X_iTG8@xy+?C{N^umdlw^Gg99#~&qgec+slJ2Md%1~m0@A6^LO8zPxy28 zdfhAW(|zBgTQ&LPWx!7E%+n4#x&oBRz^1~MO{#%|_j^)a!1@r`_n%AFLU+z;)=Ni9 zkk9#yN>ZvwN-O$CkPGZ63vXSh;itod`qSIO5!1Df!fq3ALucA$D1FHGD+U=7K&4>Z zZSN4$Wh_ij7b-&Cg0Cbh#B?!~_1_pXW58pdB$y8gwLD2F=>P$zg;hx+JlCtrm)-sA zLESIkDBW&p;0>o0BSW4jG5qpR&$5pMdh}{2j2eG68vf246LUb5fpl`im@nsZS`8I? zmQi2Lcir$mjb-2V&b5GHi1uqN_Qi`L;qx4=DA&)k)U0LH#iosl+ByKEG@dWgo3hH| zBT#q_j*3zin91J?+ThoAol!9N%OvV1NJ=t(PradJ>fgeO4izLoR#4bqe#HpztX*WI zZ-ItKZxP-q((LU<S{${UNsI)WKA{qnPIv+Ry27QnXL;oAhw^G~>@Q;d3d}rqYpx?f zOtNBZn^O7-x*z!@ld`db4P|1HA3fBWuW8BTKdQ2f!$9Xw9ZqBk&2&)2!cv>1q3;$X z2=uYp>+z>?BC%BEJA;V`Ci+IGlcUAr%Bb;&TqENZO$F#VvjWkK--26P<K$t0#@RRz z9Rs>9e6Qp>v{mK1jrk9W$$aVq_-N@UOog>Seg^xZ>~%%f#rljYlt`$5jdOnv)->;g zI*B9<;={^v_HIbxj?<-7xQg;a#c%LFq1@tqQmh2$g_SdSc{t)=ZSrB#hhb9|j9|?f zqUTG<xDouP2w8_5nlMl<eK)2x70DG_#`O~=%k9+~RiiNtB>VA`GT~&sj4A%k6ccd? z-3DtN+ehSO($`@=V&v1$&lix3vlTi`T(3bBT{M^?e)trDnwu@vq^jRW$s7fPdK@@2 zcugS9(Np6JgEwT{gyJUP*_fDO7!|i3cDHCBS4LkC&66i;#ASjWaqK2qKY0^AUlcsZ z_e0-#9o-Nrj7+6Oh?!DEWB82{vvDs<Ei=0^z7qj;yG-iE79=#@w{~W^83CWjaEv50 zUT<Ek#;Yp_(R{#vMRZNwDvA<<E7J6R-POPwlknjm@;}6fPSY{-4LSbhbPloiriEle zh-~=UeXrl+LXYGp%g3<5Ub<-z8%OGan*C5Txg+Ea<WVG^Ew)YQ4q*lg|5ISdHf{X? zvvbEyo=A54uI>}~yrz00JE9h3KbZ~sgQDv9p|QIa-DeoaDwh1mVl^Kg1}O0Dmb*tt zMt(!k9qkO?nMq=w66j~0Uf{N7i^6$nh#sK(8{V;~O$evL6b${=C^)3$mlL<`?{s<i ztu8k_%XhlZ!a{hn`Y#vREVU_(&w@K1cGi5_@Ksn`yIs$M{kA+DlHzP9I1{?B?S&5w z&y*xg9Z*m3o=u6j!v;gV9Bs-`vD#Q&Ef(n}6d2MFV!}ehXl!RSU-j{~BlkBvBMA}i z6fP|t667&*W43~^g;+Zv13C7)<?k{@2c`^fvc2f5T<;ApNPv}I;ew&XMmVqrS6*`y zKMxMVbWuXwF+t2B3()5=8kG?re=md?P-el4Yi1d)gM!4FYeW?~yFnj>b04{WIEJj+ zy$w%X(){;Eksp4P#d1fg_ZrFgWuc!%oH>Mt?7y1N`2XvE54=E$-w~bO?}k=Rt+*|3 z<&DVb`iIN<lA8o&>|etvJ(6P(p=<Hd%W3em84)#)RdgxIbo|xYh!Zzoobj6hc!@t; z#vol61&k;kLD(Ho&=-#_#S9nxuU#Z)`A7@&o=8;q5}K@zvWh_OiSz&Vf9B2%2H%d` zJykgq@&Bji{P#z;Xz&|tfpNdW*znY__3p^025*ZL3x2QhOp*0+*uY{RQk_4qF7|gX z9u*^NQ~3Vzus|*H?lEB&k!<zH<G2xT+gLz@H2G`g?5X-GGQ&6sR>jIw?L7m@FD^iS ze6`tOgvp>rReq5>Q|LL*sYO^rqJ<mxj&6J>7<mqfJ`JRR%y6OdFcKK<)BeUwZQCMp zpRb8M_nd283LW0Ba+P+J7IJ}s*P`5SqNh-S2klL2Xs2$ylZvuQ@!fHA#0@$$|H!aV zY8c>~*FO;jYM12?xrA9kWXdISF+c?jvc>smODaag@_K&gT~F{*9_bWF0ZOEm-?zl| z{JYKI!`~u!8PYD|N<<hK_uE8*_|F!+najTwWXW7$<B)NoW;kkZU)8@<&Hvs~$`)FX ziwmv`*Ig|f)6Vi-339I38u)W}o^JJXw!TRJv5@KS%B*FJ_GQdlp%m{z{>x485TO;z z%HM2xKGUNDyOq|y#INeVGgiDSNDvT5S(qKl^4~FW)#?iW$R}4G;I}}0{KDCxd7Qpb zUZvL6W6o<|EoNO$1`NkC2M?Yp=q)`@HK*B}nfWR4`$=#~2s^~KV;sV(I>l7LKQBiv zAFwNfCoL>^bN8Fbh#Sbf!r$Op?jhCXndo(LHCP*l3-So?_un+$qkI&9O)yM|DHI== z?>ZTY=%$-r++R%$-~BqKh6e5ik^#S$vktwt(WSG^qHL?+0h!b-f$wK8!AB@SbEYmK zs=@v+g2y!XuJ%x9wQEIsL8n84hc0TVgu_BwSK(WfuE#ln3sjwc=l1VE6?-<-D|HK~ z6GICN=`VYs^0s_`!oz~T;-{tGlBl!X7T%xyty;*<YIIW`3MY5meBHqxSDg}VnEZ*E zSD)=(Bpv->X1cciEij}7SPWyeUQxAr08mM)D05wVXey``Gwq`NIeG7Uvf8+Q`v4up zv4CM1h@Iy)1?Wj{1pz?h{z`&son_uY3y{(n;^SE{rQkjDxm)p|=C<iWzw<D*4nvb` zQ{Cm{vaJ`GasDFT^LF{#6vkKs0l`ry3ldYWWOkD>`cq*a#k}gt(BeVN78FSCXk=rz z`laf1w7JNXC5R05saEv``gK@SpGiD+`AWV1Roxr4AjD^<)^U#LE3ni3xx{OI+4ZZu z1(k6%lTDE7aWb=k!<-|dv5C|hW5dKBV~I~?3xjoxvvMszKSqz6Zpd@ntrkuI!6d<o z@-njq>L;-SoUu<y`#PI=ELt@8p7|!m6@vKfc5$F79N9=FvkVmA)l(dPcSpAF2Ke*m z62KpsISrhnq7=43Pt9;k(b!`2q6kIA69?-feHg{I!iM(Y{Lrc&&)-_o`AKn@GQ%K& zHI~O{Bv$sf0&Zn>qFvK*BGSxsD0fDhf;@1o0H56@fb910-1U652u`!cxF9Y9nFb&3 z<_VSf?@YAEV=$nZ_@sp!6I+(Nq&lsnI3)c1K@fz(kPC}Lu9z;iYX4A7EaDI@#SC!u z(Rpm2P?9NBB$GfKc>@YKDp#pqjL`AzA-WjU7*O{c-qBRSuuDF(ZofXodX1+F7}y3C z4+ucPiMT?x=wY%Gkq6CtAIATsXoRLmB}yv=_18I!5yIB#e!Fv~N{$82ecsHd^v2@6 zwQ7etsTOoHUm};JnRa^YP=zDM*%U-M1`-kDSzd*{E%WMNc*nJr%Y@lbL+gwtp8-YO zW>8^;tu2G$J`y5{u;VDHkgIMmmKNR#s=w^sE}-QYe??wNvi-|-bkQm7J}+T>er$cL z+qlTW4M#yySfkL(`nIPiAMu3ULF%>Bf+05)(mVt!b+(+B%mBe^bpiuy->VaFDOHIy zJc%IVdG5b`D6T5*Eh$Cg?LY5n$*|<#heMUskSHlx{dI_PB)P3=|NR<2DLf$SbnWT& z&vHqXICD@l#)3J^Anbr<hf^|x9m;WPaprl)d$NwcmhH|~z))}tGMsP_L>M_3&%5$z zIMKS4hT;|IYC(=ma_n*H5UB96aa;;&YO#IhST(>p3w>}|sb>Q*u<#B`cwba_5gXcj zADH+Wgi-PM6{4I_5%t|2ipaq8al?r@=w*Kk)^$(v5LYbLd&1j<qNqh@r_gh~+@fyy zGlu1YJx0(H6hZDN_K2N>W%fB)_#pv=WlMRNEB18rBNh=R78vbh__jsg3f6?ait~=P zP5Xt-Vp0uQa1OgTJk6CY6nFlx(AL(T<2E19D%%nNW@hCYjRol=;=YK0DRRi5c|rS- zW5F(391pdpg(*dLOZaPn#>ER17Vb=di79*voa@Y)r&o+N=2?zb-=eYfP9Ef(gXEYA z;K%(<eh;w`_o>9)ulZfX3VbXdZ=XB1(gj2rSF42B&!xO?ViKX&{6Uid4#h#7^SKX9 zq6+=;u<ca^VixpcQE4f*kO{vDrA@BErfCY@kYdJ!i9;TDbDi=g)Q$6rg~C3)K{oB? zo5=R73~ZCa(X;VQh2+wJ<2ghh%<}pCTj_Il=^-PD$hgt*-Ba>-fe0QG^TIQw=xpIQ zPuXcCk$?)kaHGUdt<iSj9iI>a?4>x-T`h{MstHMxbgugJuTZm2lLS5wSfZLO3wKwl zWP9K)M?lKpEnyhUW`s`-vY7lD1psa?M0K}#QtCcoU5XJ!{WRxn$*}uAPd2~$F+bS8 z#r5@fIxBt8Dv*IB6`Eq0a+SAEN79rDZC6T=K}Zk87_=aF<y#BHIT_`1q9$6V#=KSJ z(R{%s#5*By>~g*QL?snH7tH*SrdBjTmt(Xai$TA+c$s)`faB=T3&>o`$z5X}qjfuA z3PT*^k60A9w{14kk}v0EYcrE^U}N<zm_rhj@C+Jyp6=Q3;lwkM_x<~2EwLRae-KYr zgmS7gLBie63fDe^6zlllt~3H=vVfbmdQ`<v|KZ{4P4*vP!9m%)mqGAMOpmJcP^Vxe zc6zH44&9yTZyU`c83n%BLUMMoAyi^v)ggpwSwgmDQv7a;HVyq6UE2-7l)!9z<KCqG z6ntR$l27kxP&xF22DT>Wdx8{wZ03Zy1T#q6!0)vwSr~aoRkcwwk0Cb`C@8E!F28Gr z26ypAF<LI(LW`nQh#7F#J0EOpZLQbQ$&)!$b^<yhW91#lTkIA3g|9nfk&%5hgP;gC z7^@tGyzUnw5iA#Lu&4C^>jA$=KoqWQeS{x5wfxUWRFBvIDyFYg6WXoSpCvzC)5Q|p z8!+Fc2;_C%0@LJGJh47O3>B<6ylXbIt-l!j|2asmIv?!?7wgT+Tp7wj2h*ROSRt;G zc%Qhe7p!Tk(ev`0hJsB9=%#w4W2ElDD)D$!_#I;1s?@(GMDTCDmNW5+u3oNLoZn}P zl;*T5ic7!SC0FXIz}DJ$6foG+ydY=|9kaKbtI-z`O<45e2>B;R0`mPDCUlhlumCmX z!@fwz93Q@_+Gy2eHXs-NnJ8efhvI#G7RNz=-kW!SCke$iN<<rnr0o)&b8;JQG4RfD zvLp#rCdXRx+lVdX!NANHQTD%?YH`B95rlyZPu_lY;y530bkAZV0`?J4dh;V)KY-Ov z@5M4UY`k|&5a7jj#7iS&9s6A~A7E*wO_Fh_WQ5;#uyN625xZdo$e%h}X8cBb94h!R zaUV!+g6N>8{1rF`B7mc%Y<~8D>);$3p7<wB%8i}%v!+@lBb^cPVeJ8;d#;l=!o)BX zr1y{sIIHEqRrb|UQT=PbA`%i3A|1of-HpUBbeDu62q+*(%P6UYbayw>Avqu*4I&_j zG}7JOaKHS{d+vJgUF%)<T>b<08fI_S?EQT5nQgwt&@ru~s+8gH5a+TsZXt}rlyF6Q z@<QyKP=pb?sg5_KS{AU=kc4m<T3$<7fSj1~h_Cm#aPpciok~d*bdi$$C6{u1%f*Hb z1DzHdyF0^jr5=t$xr<<_YES<lifOetwp^zmW!5db$H2HQYi}UHZ^9POP=>QHFqLIK z)J4`r5Ltb*xk%5yG)YxHREE<zbKOk$L_1$Z-yE~G2*+Pjc4JB7ox~8E)4-ox|0LyI z4;>$Dt?5arN^`D2+lsLCvxqj<_X`8#lf428#`V^ObC{C3p3Xe>)v#%{nPh@|a9oS$ z?aRmbA2YB+J<y;B0uO5Ew<gr3#WTD*UmH~Z5J+_I6q+GH`Ds*A*J@?Tce-RyX-__= z1V?3oDKvbyI1z7R{7zg6)XZPU(I{y!&K5z1^8LBUaQof@q6KOBB{S|0aN}2efFERE zKAWd0T-=kmdKhAY^AO;tvw!5cASF(?99$31ocm7oS2NDPl{vyVBOFQGe}8wU)BPs^ z=RJqh%Xb~KH#dm4C46=9!eP2x9vJlh!dA`Xdt+QJyq8M9Mu$#2UI2XM@uA0XL7+LY zOK1uRc~!*r+R_(7&EKyUJx9GROLGE`YI-{(6Dz6{Y1#D++5i8Qy#M}=Mg5KRyS=tA za7&kAZ;9V9`aLXDH}^bN7m?jyuCD$24k39nuKBxtEG?^6Uf!lE<_T4k{NKkl+y#~d zO&Ki)%_?D`;TyURIJNN7(B0<CdC%L5>ySr-%V(A#CvIj+EzTw&W#;W4FR_vwHThIB zR&_<#mj6VQlID5M@UL^s5Tk-lSQF73ak9VS9jctRIS+VhM(zJL<UPSIpRFfz*Z<-( zxjLfV-KtR<DR3k$?3~{VHjOoc0=#uT-BP2{&Ti(3zLw84fXyUY&rY^-R7@Hz{sN(x z)%}S7i2BX1ADOXu`4{wPQ`xOG1&l~5M`UVC&-O&4Nt(!1p`AhP`@-k?4GP6)oy0cn z1aE<ga%!#zs6RU<!Xg|kq>Gnh5Lg7XlGkPrx^Kq_Cto?KE3Hxsk7v$SX0oy{M6!@S zxhAs_vK{%_Gju$Rn-)>glb2Vc$~FY!E=RrU1kF@`R&eitAM$daX~!2&(n<LTTeaYE z?RG3<)0aoSx7vs4(zqH~W973WqU#?q+w|<(_Z`?zzei~VVlaL7Z3K#2U-E@hHCxd$ z>!jyJhWTOCUudAj1<PQ7obKq=o|$l7tubN^C$xm)BB1O${^nUYEQ85Ehg9&95aRdV zV2S9E+bLal=-4T|vnTt!6F4hoKsILv%!bt-?7I=oi=uG6MaEnk7Bi(0wM72T89<JO zg}E(>x$Y4Ah;GUCBEf*U(@3I=DssBe*~j^m?)hMz%2vmb%4y+r!W%!ds^Dr1j@1>C zf#I)R9#Pm*E|CiD^Eq~arndVweW<2fOgQ!2{&r}4vPtg`r&OYb_ES3N<jFi*aThoz zJRugXnPX8*OZO;mW`kny38jqpJlC2!Yt#ZoL5|PIR?f2&=8J(eg`}6qR8YSHi;ZFf z-W8-=)%X}cK^l-7N1|u8=r&>N@3M~|k{@@ou6PE0O!qU;B%L=;-cmgl_pRgFO0^6t zD@(S$LmV%!8*kl||3K4%l|X=fWwK_|&PgtwwfB%DS->d~#PH+MI@cHL4RfoA16fBW zSD4h14`cX_EEOPxI&4%87S{f3$$DEW114+n9Oc92eQc~XwW9Qb`~n=mM5IW8LD6xM zG2aH)btfRX^Zf-L`3kk9S9X`6URjg=PLzr8^Bme#BVIsT)!LOK4vAx}GAd}agxycR z$$7%@b2~}Dd0yZRS2IsJ?j1?`CMKVx%G2hd_y!az%%xH{U0{7OazgBSm3Vf{<;Dt} znsIBnd#rz57Ikps1h7PhP-NC)NHsk=4_?>g^(7j8Ii`v4qdtUgr{A;C`tSI7ZuUZZ zx~CsAU-d1@l*9(#cU7f~qEjBf0H8Uo>y#7_-fFD8R?;!5YF1N(l7}KrxY&(L79VG5 ze6<SdOHXI=St=9Vjr&vljxyng>sA0yx>$}zF$}7(_gL8Tfr9GWG%>`Dk6g)22X(z* zuK!yCMQ2Wljr_9=E7<pzSiO>8mycV=-cDvu?eQ?YQXW~^*F@u6t;fKlc>`b0Sv4U1 zAe3&6<ykbR3Y;LruBT=0t4(;*>s@^0Xmgm580|0Q#-}f?r<3xlcJ^bKIKiy{T+^WR z-4|L2&U_^^KIP*_!D^esrC)5KnaxXuNwjzhKK4JaYeaX{5{`Z0iBa^%pux6tFdF;E zhaV%>W|?e97vjI`gS4l$c^K6>ZN|nL+LC0ISm7`-0}(|ER*WcYRJ$A;dXt74jcjyi zxWdf-u0Xb8M)Eeu*XOqRpi~3!_3jKu4V^y<^pOKeih(Ru1x)7@bgpe5;4&|=?4p{x zNXa5|d9OVc-bR0>BVQkFs(a)_`FP5adwU5g2rf_|Ox!mBmt?qc$8~3t?veFpis<MS zPE2WO3AmOpNjgv*jKI*IgbVEj0`T<3WSHGqPbK+;9O_hdhI-)OcQpw9)lZ=LP7j%P z$1xvtl#9RL)8He`ZQK_cM{^BP=ueGj8)(+u#;1|dao=#sq4t)t#ghEwP)ZmLosn>M zjmm#MvC}#pD88xrL}IZsqPoU&OUYlKwfNck(~0mVf@>*u6}$?nN<Pb-`waCKGSf>w zsOtb476dR|cn?#i8#z$pc%(k+;4vmbL+4dRcawV^?ka2@bd7;m=)>S6RT-XOwde=u z-743AkvHInxweh?>zsj^^$iR8F&o*rpzpfLH>cC3ra*#mii%Vr`XdT_3%YW`13-8A zW}(yT2XQiBBv@!j#@#N$epQQ@{dB^Q>WkPsp-T4h$z10uThyhwC7c+ZRVCavWn&(A z7{hx<lkqnRxL#RM`P~?miM7;^yp8GU;F|2|?Lz=+M%!#t(xE^uyFi8#W9sTqn$x^X zRg1#R!5+9tqX^dpwlLxpNBU~kF5N_g)NP0V^&eOS&Z6;lWXq{Eq9)8L@?6+!A(pe@ zJK|TwRloiWVlHzMR}*WA7uK}{`phbF-a*Or$_y9FjF2n&uT{r%h^Mgqg#+^|(+Ve| zw&BaD7+>mD&LzD*yRhF9O1tCFlo&Rt(Zz~DJ={(yn8%unl8Af4Z615dd<2@Gd*#yu z_S{oi#o?~XMYbUVR-`GVFh=<LZ~I8KX7hW1`&j?J;+8A478_;GSYJF=W^-Wygs#&h z=6OIfGZWw=w`RP5UC+-*Zhlj-jKUrwVrhsuDEW!aEwZ-yE*e*Q>8ty$L;}N)oAoNo zhDogDK@ZH`v}(fDNFmG2Xs0>*mf#SZEKbd+Bbv*5QEZ;qNpLpy`*TLa_QJSOii<ak zE2YWl5dTiBmXT_n4d#LL#?0WuR$q*biu^%le4ok5zm~b33Mr8*knEET>j;lWl&A!_ zaIlabzP$+~-GT!Fr-wi)c9Qn!sGcShwGZI%{MD3TzQaL&7ID{rx*bU@27$&R$+`p< z1>%iJ7ct4<*io4491l28KH~m9#dsuGKRPN%8^xZs**f+z!a^6qo4rbBRwoR@gHml- z)QNA#b4>k_G>!{QEJ#3O8E4V#uNaBi=?Y7MS&8g?|I}0U)#3YNqdVa3f`xS19D073 zo|$HpD`RS#@f9M#`?O9<5k-df!Ll1^*X_-Xu`W-63Y0XKbA!}UrYtq^^B?Kk-P^wp z=}A&iU!LfhT>o-#kR=S4i@`qR8d`GFN+*zk)G4q1_(9vk$}$0llvBUyFA6l#YLGX* z$;7ne!}|DqQLu^ZJW)q)HA+cWnWBera>3>wncH?~j{iI{ftTb~Bw7aj?2NNkOGfd| zLK@HwXMe!19#k|p6+3$9OYZ*ia&>sn2z!B+koKb9>G+L%u3uQ>qo604=|&ZrWu0&d zY-RFJ$K{}U?tiXy8zmLXky#Mo+A8Aj=H&Vk%#FD5Rp{XJ3JqwuH2FUF34y@$X=}T> z@!J`>8pjn~;hT|lKCb?#0`4c#3kE^S3WIzBeMR!N@jC3Q$5g`bhas578aF1jxkg=z zS<kCnn>a}o1n&kpnfVKpFvC~jXc&G<$E)9W3eZ!ZzCrW7M~eC*?xC^5z2TQ%6A#C` zMW$q_fFlJG9do;xu%J;U%NV)b+B>oEzNep8z5PB4`9OFo`>f-<p>X^n6SP>Sp7BtI ze^_KmTe1A?iTWKhIX{`&)nSf&VKxxb!}}F$HRHk#(fr<?SpxH#g5beYCCyW}TzdF} zud)N?cApWUs&f{Bkcv;BkJT!}!G4%NcRDb#UZRjohqhn6;=qHQc+L&}%`1H{2YDV{ zwiaZZ)~+ON8@vgVE8%bdmBh3l;pHDeO9}4FyBU>gBW=N4!^M@4FV65<4nq9tOCHJo zG>gei8Ar%n;9bredZ95dJlOxJ-loot*YmW33yyc)e+)qxHZuy)VR6k1fm6$^;5<#t zp}Y9YIoP0gg?|$sXASR$v!6ZQGSZW%twnO(9FVAP@{M66&2O5$<s}2|uX`E37&xYs z{2YQ<rq+-FR3;+ao82_gRXE~77ko9F{>szAtvUSO)ObM5cg=Ix+g9I%vgHgWrvbdr z16~p?7AzKftIFu$4Ca~$o-B0a1SPbPeY1_&;gWn<e%X?hh>w;H<!Io73P^YfC%9KR zKpITy%Wv&MF}yKmNK7ABPTiZja(bm6ZY2M9!Jmq(O$Hx%?_$1>t7X`$8{@(Dsi6FA z=MB}2_fWUog&0)sTbk4DxH;+J)ojKtn%0){$2Ch`iUf0V1}CVS$v2CnGSlbuw--qA zbb`yUld#^;YP1PLC@pcT9@;|$NK%~rY&xg&QeWMI+^)dC2*r`#lau-PC-J4*B132; z?);0e>}c$dDx_k)lvyqh{*B8BlZ<ev+W#Y$r79=;d{zAWuwA_Kyl=oi`8tH1>b7x0 z&!;vV-VWTAHf;tN13xfrg8rPvH{5bY3D~4fT$NiUdNRD;Byy_kVQ{2~Qr;4@85PZu zZC1GqdlnixW4T>7D;jnugQYkFmuHIee<pYRPqxq?a81-f#sBtXm3r|*tyHEh(IcX> zEC`+eF|*VYW6Pyk*`w7(6EC$Cf$3hK@E(7JLt|d2Cd02WILejkK4Xz`*E6r9%u#ef z>0`wW)AtUXQ^@>Ln;_9{-(5I}_Q|tF$jy>GUliZStm0~js6acZ>AkLaX-^W{n*VVE z!Q%?s4znE0Km=ctipuFP!Ime1&H9^;z=7`2#Gu(3571!H8JGR~h!YxXYa8wQC&>}Y zsj$S%VA?nq7QKcFmr5^1wik2Eg|7opeX5m6xc*={NoR-<08((Iro6;3Z#VIvEVk&x z#P7Mu^Dmxi4>x3$e;g|(4t+5vu^BBzge)cc=Y>>{u*uHqlbC?{be1bG0tk2>jyrfv z&PE?(^LD;?Y>mD>R)W2;ISfAb(xEnfuDzN5)cvp5&eehzhvm0wGk<FNW9fk@@43vB zbiK?c7}u5f#LLkvZ>fy?{{=+x3Bd9o_ie(qagsPD$=j=t1l!SH$UD?<?|gL1+eP+} z1XKynRA?%GmR1)THfK^Ry8nvXq{UHNIqU_}N%!^^RQw}=)w_uk>tJLZ>=W}yN#Jdu zME#@@8IU}IM<bW|H2jOxGV5t&!E9Ecb&-m%l=U*5e1y%qRxIzT7U4NqnQW-I)41C_ z7TpMk={hx44=oxmCNaJf1c8rgO3p{Y<dX$^a5;P=f8Q!KkSF5b&v;iz|2<aVpa3Ka z-@Cm?p6kIA8!!fVYoZVZ+!&$njFhaq;Rr^Gnz_(tM>dml3(L-KDz5Lgwtp`6lroIX zTqggbCeUyc%FdCb%CZz?&SX_+_G8ZJ3K-?8XE{Tt_N$=OK(e&Dc6n1g_Rw0TH#MKZ zW_{}RDr2^o>P%rZI`z`K<u-<m$ftFJ^zL4;Cv?1786m?v5GYDOJyh>@oP2>3en;lY z6$f+@a~XXr5d7SvQ4pfio7bl|jVluWwH>rG>AduN8>!fu;UEd;y6>ws4anPf0?^2_ z18zY_)qz*vw10>(6SkG%_0=G6Ea9zO%~CxA$iDzzkEqW{O{f*V3U6{*{~A0yUKw{M zEro-wMAXx_dQGp>MpZKeEYry!2=oD=DTiLF-~G)W@Vxs~T0D`-ns~8F+IK{TwqxVC z(#Swgmde}q)a^=N@!@h1il(T`#DP+s%LLX_8krAIoLUu2qb&`!nx}Pj%UzLtSjWO; z#ztQiPR-%kyj3&Tmv}nHryF7#VI)qin*E^azBFFJ_8^x18Yd`K;vqEQ7)nIUWt{Kp zYp)lP!7pp$yBMk{IbZL1AoDrw*U^KiK6)t96QF&A!?K|JB6h$!`!ryirCq+L=WgC4 z)CRsXco5#SkN>MTGH^a1J`@?xLAq2`^x7azqm8qWAhr1VIIy3|C|mor@8?`rYwmi* zq&z7rs7poMof`RbYn@g2EcGx%*@eROUs*5z*wKR@Z?-qJ)(4$}1T#XuhoUc5*XRBO zT&t_WZ>K*Jq<t}byq(?C4#2tNw7fuqp9#;xCXR22D;}-iTIbWDhe?d*kZ_EnOx3+D zAFEV53VAAZUpaw8FNH1DIJZqubO!5#uFkV{XM8eest)JHq^dU|x@>wjBcX{<R40%w zY_k8uI?<d7e@oS<em6EmWQaTNdhTVhJ_xmnEl^3t+K74djTT*M(!6a6XZz!f`w8-! zj{zs>R}urw_*q3tRn7-Svp2qReh@-TwC-F6xc~d2j-)H34I(Uj{MLEN$5&+BVSJlt zSy^;-=CrX8RordMkJK{$1qI>yP-6yb)CE=JG*Hy|h49wr3N%_v7gFDkCQssWiF^g2 z7-TG`A+T-#Eox!Hcs-C*yxWWSjV_z{5bJfrOBnv?t9yrk$w7lu>rQ!RCiLCs90+c! z`X?!^b+QRj(t!Nj-Z}5g%SV*9zgDF(x4Z~mNv-{sC9S}_Z>aTRenarBrc-Y(n?uHQ z<Cxk7=KW!Z=e2eS=T(x%Z&yh>LF1OX$E>CABvcf(!Or-Ndms<LyArHVyKMPSOwYfv z4snY@H5Izrj2f`^J{*0K$Pr=SM`ix_KJ+W85Uuex&P!h_dQVk)Sn|<I>6DqI^Sf^^ zNHY8~t*ov8>_0-0rtc@=Dm49MmDu>oL&gl0899(8iCl<VXo))n-xvgq6L4S_OvoM@ z_g~<;sSn6n?4PD0vElGB)^ot*jP>Jtlk}!u?M+0ZIL2T<s<VlBAsQ&nkDP-ab#u0y z2jI)#=>gt=92<FoHk^cikIO4YxIzNPzL}e|qEOB46l%2SQN^k!>X5{YL)0tgy^)=E z#JU9258YsNnop<kIE#|gj_k{2N!nfltJ3V##u+^2$o;p46>%^Z={;)=p^#A4C~b4# z?C`)IqVc=ubV#R0Z7JPCs2JH`c8~AuJ%Znxs^`;TkPZRak<h@)g{4APUWGc<_-4p$ zf8~u)?AG1-x#m>gJ@DzumD9kZme2{<+xxBlXPz=aA^7n7_f{4dE4FcIU{DxLDpzT| zPHTko#P>kf^FW2NI~5=6_dnRM&oY8~^Nii38FrZq%&e?S*(k_VB1za$um<G5>Cb_5 zzS(Ut2aTKwnx0po`BwY#HSZ_{+GwvO1lZ(B(o9W$&c1ZEP|0tGP{I36Y0)r0^}K3^ zOP+-c=BzmG^c@o@JYi#vo5Sz0;Dr*rTS`ax+2XS+d3~Lc#6D39Cdm$>G6@RT=tW!n z{%{_m5eBDQO-Xr2XjcVC+p+<^ilSFqZu$B009US|<5HE;^3HEz%(z<*c-^+j>;N!6 z=F$z)kA-D`Vp=!CElfRa@F)mgF7Yct4ahH&&i3%S-N_KRfXec@wprkg(I(7i`z2z} z@yRomC%kFeg5SBckGhpRdd0CtEdMgG4AwRx1k(1`EhG3X`{yP!NRmib)1>boiNq}9 zqXKC~-aCW1Ch*5MCISrlPhp^~L;E!60)je1?Uz?PWbw;POvrwARKj;}?fJ9AyfE@F z>$)_Wgh)!lS6KTV=gWHA#qAIhO1a;_mwZ$ol>CSe_LrUD5v==PAbkL#b)i!ykLi0O ze-$MdhF<Hq^3&jYs1KM*p|+1$hKTw(Mjfe?Ie&gIMOaWmNO+%|(OqHq@tTe3Gcc^& zF#c$w#q7F#K+u}oQ=#hj(3vX~1J9O0MghyI9l>L%kdHp(r$Kw%VBPmj^%|NDf!T%- zab;Ug9+9~X-l>Y<nX~p2I9^11u^q4yWO<?yskG?xgIVoZmc)gI($cA-o^q#Aja!pB zb>yT2L9G#O^h#2?xk2Qn5`M9ULL!g4VF85U*~|)cQ7_t2%%oIuht^bZd|WVBV~V>I z6(;6zc<yO2N0JSOW6`iFy)P{=NL`FAoOrV(PFEbqDr?N}+%PoI?%9y|dO0+Vd!2;w zPZAMBwnej#S8tkTW=x^5^<ROQ)ZF4k52hvs{syUVb1^^oWXL(6(-LwRFB0ooi8Ta` z<mV8#Ly|2MgB&XkinJu39<Oq34!=l7Y`(W$2g(wO7|+jXf+(C-bN9!Luht&$hJWt8 zv644i^8hw;W#yEV_>|ohMQ=jclW0c;H*xZDN<ZlpbLn1aSoF0xds~2KsQm@6aTlAj z!_oBsa6b)_ZJwo+P_Ec0#ToTzK#VGlS9F5dA*@@8e}D}Jn$71nfR+M9>#H(5Ews)N zAQ$zX_nBl|eL08<ua4wwuj>a^%f&p?QA)iQ=Q`1eJA$fB@J0SEjwq$pM}7t6GiUDL z-(jlcpMJHL{5n{TDQb(_7g}}yqDI#+-iT-^!oUf3A))tiTd{`{#6!!Tn-ss96v`$V zHU(TiB4Am3m`Ua5OOHqzwg7!yrv<Qy+?AY_^w`IZ@!}xH;e~VzR9Oe0s9Q^Xo-+k^ z9wFF~v^hS{mrjYlU%Ugx?>~^vpcEg;yfh$_^NoeF+1hPlN!fnV6-*8Ap>NAoN8WJI zQpTq^<5;u{rejRqS*Oaw%t9Fnp^p`qPn;o?(Ak7@wcn40X|KaazGSSgDq9TgF+%Pf zkBkGAA&;5V3e`<3)%f>l@KK6I9rylvt;xXAE(0MqjLrW0<OYUUTwz}c5~gD+Ln;{f zS}C!yKUbhaaaP_|kWfNu+l#oMxGih#zAETmfehOKj#9ybtIxyJ8eLu_oKru0CfEG? zIP8L^454*}J*I3iJ4CR+m@p<=QsnW(VL*L`LZ*pc4U55XNA@tdHpkhw_LBwweL!ze z|E=uyc{}TUySQEMqgVMQqrv`{YkNu%#A$!c%EOgWw83lZBz&u%?!<r#>k~JI3|%&@ z;XyXp+VSLVg2!2BZ|C0Yg)cpwrhyoevYaiiA|JklWiY4y#5(+V8~$ER=ijEm$bzN& zfol3|?Jb`Z{vBk4L1RL)%>R9a>C*pfe_W{P@mFN?iuQdsJxW+y=@)Ge>$a<V&GEPE zwA1Pz7o*|NJ;Nnf$d%Vf-6&znzBYS7-+yKA!larhZBF76|4$hXK5TZ&eW}FzzyG@P z{}tx@?~>d9<bqt|cM7@|dDCtWqJVed!RS<QR~eVn<a`r5YiB)_k&(Y;m6*BrIPMY4 zhaXLg948%?bH0e9SD6-4@=LI|7Em%|wY|RvRnpnbttYuw71_u{toD7=!VHF~Q_6Jy zqU#Q~$zlY2I<q3Br~O7GjfB%y=gp#_Lg@9C^28JnpQ~>^91W&u$E0OEU8M|$X>KH5 z?jxD2C@;^MiR3;Nt_Aykb^z8!aknFd5XUCFq&HhWMRn4GHfwj<ola0+a(uBs0gHBn z<KeThNmrqh3boD^;4wV8NHgRW@?N3z9DV<tM8@XL=EzHQ&n^q8l0!6|m|UIBugh_E z(uzs(q07Ro=<&dE_(b=Qp;PkLPB0ICSogqeaoN|>LpsGhLpKJD!KG1|eBn9>h8%fv z8P~Sug;B5W%zBS|ar#L=oXqQsZv+jjJ(VdnMJ)6cuV>uqRCXF&jZ5Pr@+LD8`1sF2 zrcMIyG}ndmvhZ0F=pBeAy2h#3d-aF4bV%(OCQ}bINqhmYedkXDYq>91^I1soY-MrN zMJX*MILSuW-FGdQ<#pA>q5|8F_hKRW{M%|10lvcC%?93#AERr8$(3+&C<7fdl7%sq zV(bWqzq4dTp6-Zc4t(Uzznj|Wawh4!7seeRG~P~wxT>nZweqCvJuk!W$TJCFt~8Zu zki~ks)2gm@N8L7Eu9g`_T<*tw$?j1v=38i{5VChvPp5At0HKCpZ-6ht9t{eaxX5V9 zt%>D%uUY;;tE;YU0Pt>eb`}#w<57I8nlJr05P07Sy<c;9O5An<OwUiCV6Hg6z&XgT zT8)JVn#qNwW#(Di&mwaOK%;)s$mrYQ&Dpq~chi2o<TG>i-vtRMCVpG4H3fRyndZ+Y zLK4dB>t;X9=2=>7mgC<GbO31O6u7Xe?Q!RUx<%TZz-g>02HlkIHC4Z%j+L|*_RxuF zr4H0OOYvd|&hy&`-8dupftmXZ#bcvZ{cg1cL--=*A=43UFEGP#-=!E2N?gxeEer?t z2W@MAO<B#1<rxb{ADm%78LT1Tid9%KV3`@<bE0nc>uRbPi&>BnCcag8KqFTBuW59x zW%j~1{w!C9k0H1<D^(29dQ8TxPqcwv>fNLonptlVMcDIm!r9;Sq_CZaP3=BiLFv?5 z!|___fZq{1!Narw!~B?Mfy1ec1E|oflC{Ao>~K!577a149Oh-*g+xf1DxpDE*Mwk> zL%bNv;GpH1b^_k=c+CaHbLRb{!ih4w#n*LKipV0XbcQIvaYVB1CJhcWhyJ9b+PaFx zP&@y8V$(z+;S@Uhh$(yCxTd<gm&c?LxQC9?W(lM3B6WnpTy&}(kx?S5HU)g7B#IWk zk(tS}nY4vvrYp7#?&wUXtD@6YH8uU)#8b2jNQZqbcO@dHBoWK9Nr<G7&v@%7L{6a+ z(IQPQKtT3qcY2voWx!!Za$_}J0ub&LARd8QUi6~$k-Optf2=6|>_M*7(g)vpz*9Z& zDf@`wF`(v0$dC0p27i)J(7qD`!)iE(rZ=DL7S{YHRo-~TP8j;Dka*S!DG4B0y8hw) zTS<Z$97sy@4B{1_!vP&Zeg2A4%JwX0VOTbKWv1@-BO8nH*61Hwjf^&D+7dkKHx57o z)EQYg3>+~LYEXIrK3Kw^@fy5yxl*nR)SM>nC<%aBS#7)z`z2jI4gxT_Ub2-cPByK5 z)CKUegOKdQOt>Q{_>^kPv@1AJrqtOYp4YwAb2@L&Xi%s=l={sA?4aT#Y7ql@RqE-% zOEcZ!5zH@AXYcgGcxK0ugI}@y{fpX<CHQO_NR<m;oQ{@GEhM5hYtU386al@fT!<q# zoQf}ysLQYZ`=M9*0-(Vs^<-Xr6T3L~VHQU(X(4u|86hbrj+Upre<DpTf;`_DS`3^? zjaegJ_1Pxiu$y_d`$ou)uu;LD3GJf;%X6c$TK_cJ80^}Q#`qIzaS}`V*@=qUyvFk- z(3#V-{l^Kk;h6K>rW(PKeSA-6@u@)-3oY!ySKW@@g}f)QBhNS-kSK9F9NiX`6y)(N z3RdapjxGG{@;A&%2qBmCT1u`*U;aC<nlcH8WAU;+O{-Z-BzAUurT23<o}iRPXoc(? z42G!&?*_pT_-|_O>&=8aiCzh2<o<n*DCmV<gSA=9)FuK6h`!Bt1R9vuTjcKatg!mm zK`rJ*5LXWd(L0(C0fJ&3JmN#Q@-f)H$qq#AP&7?Y&O+5)^k-A2rlPz)S49_f;C?{+ zmFkZ@&0$3$ej(8u(|A^Nr=z>-8Ky%MV90!iQ=t?t^A>;G^u3I>d@X`sEye{~DRp~k zbs{|}t#~j<#$1vhj%n}?VPuj9!QW$S(K*o?2YT*Fkgf^ufy}DiWq6t-STgSr$MJ%d z)vhvM<@yR!DU$!mx?;xYL8f}36)h{}eO98@)_d|<Fe#ilB<u}5?}*Bj8;?fhnYJZn z;_ooaBc$TXxm<k@ZV)?uCpFIS17Y<;fiA)19zd1Cuhq&7Jy2Xn^Pa0yB@AWGl2fGc zJEhTT;%IVuiUL72i2PBJa42JfGYMybq!0ZbS75XY61~u9_yBqC-ynSF0_nuamC5(1 zY|(&5;zEVv^`5Z>T`hA-=63B=S1VXF>GXzeE)>_F01w&7K*jGAC=3XcyJZiv!ZH%e zI=ZXw>i08HqR7RX38}K8vAaIWWbONqC<*r}QTK-g^DVe7NrR^mgPc|-l{Dv_9sF}Y zY5$bBhe;Z~(n;xT%jJ(HAE*K7OOu%fypSK{q9A+VM{n3q>>%33z=5=uOj_;@ip5S$ zO|e{C2Q$CsNGiF+9$;ex&jf~%j5)Pb(b8+hhTu=<LEjKiu@y<X>H55(x!MaV@}x*# zx#O1#xkTNeDY?$J%uicbaR{V?f6H@Jw-2jlJp90&fl5my<~0nWfSqBa0?&5wlCsEY zAEwH_xKXKtQ39SIQt5hYh>k(z*1+y>a3uBy!tKQ;3g#l`xN%@W$*6x>m=kK`g#yOw zfdmyaSmtZbG<TB@rfewr5k)a-5($H>>l|e10HMcUCmd-Mm`vc8D?^LYC47Q0F^7SU zOHsBO<;Dbk9gd#mYg{qsCrit={H<MH;o58FhH{)=aG+|T@dB4JyZI9)c?#U2axD)e zXO(z!dwQxN-|!3J{p=F|s8}wH)|?sI^n{=CUb;FJ-cV8!q@l265`sDzja*>47zyvJ zBvD!mhRrny)2+zHD~?Eyv^y5OibbOp2)8(yJ}-)N6!rSRhTCb#kn2O(>JZ<1E}SN( zNiNLAcw~U`4p>y0NKQML9bj-yzy3(l3`<a=D4A@l!VgY-BkU9;axXu}lDU*zn7u^5 zRvuC0q!?!U(+X)<gb*c`bPEQOi=?*>xGy!ymH)kOt9?Iu6{5tzPzTS>cS2Hw`m=7= zrZ2WK1(}2`o>82132G95W5O`t^|>BMa*@Ii#r32OQhhQ7r9t%riaX}E;$uKyd~>UW zf_x<=Gx?Fg<Ll7pE4O=O@amth_=A`k8c~8EuM;!qI{WpzYVo-Pdna*IsxvBE)<5_w zy*U%Q!XN7nNR<FM_je*2ZBbZwTT<1_OEZylB5G@{EiB1^egf&o^K{J$$ob1Xu5p&; zm9_oHqIijS#+>o4zN2s`Z@_+@+~bl*2bsUuahxKVwE{bQZ!dRwxgV-rw9jJa{bVK3 zxffGK#MbU8ecuR`aQ+2cP&k8}N1QyS9j`;5hj9cCE6(NuKi(nU*>ivzW0YAb(B{?Z z&tE~_J0V&7T7*=<zMGgMMN9aSu<3)CY3CD7nkeR?(3@+ZA@UF6vJ7Piqn;eTuwKd+ z2{-p?p7IH%|FLY6ZE_L5#weO5=lu(Tug@>S!!!6X;${aM66QlSJcvoQBa_@YC7m7e zSqKF<KqaH>L_v>{XSVs}E*Doa?OT7Up<Chi*@idy85toQl@TAv13xx*?80pi3okG( zQk^UEY{u-?-k%DhN4;zbB*P4`s@J@F(2Rrf<g#R|duM#`&`i&WMDo*_-%`Fxzg?X) z;|UqxS=RU_12({9LMWs|cii*S%Pz|^gfuxW(+U{2GwGaoNEg8E^NN_@@~Y%nu_1C& zw<HiQ?tV(6F$5G)+Mc|s)@XxahTek@pYGyW^jOG_HhC2|zlDj%l11S6|I*I<;S410 z>i?0F_TM26a53s9_aT&_+ccwu1^W#8|AfR+e<YZAmd<L1{3DQ^J?(;gBS!wx@mE;v zv%JfaEo;U<>-rZEtNVSv{Gh75!<I@mw_KI;bH@W~Y+qWOy|8!5$cQDMyUrA28*Wn3 zJNKn;tLd5xKwdcLzFEHk8ssL$`K%TIs#;9J0aX9b1E?bV5)AFd<Mjb91FyB`JcnRL z?e)hs@z^Iht+`^Vz23I>u6o^{EC4h<Tl|O8cV}_Ti>yEy%?kIhH3tE?(Gr8IQO8T3 z%IX+pd5#m)e}3?KTQ@l><ey(|9=%>N`_~IJU3*W@^PQE;k6+vaUy8D7@G_XG&;J6H CvXt-u literal 0 HcmV?d00001 From 5284773c364ab5f54f75fa086313a946411641b8 Mon Sep 17 00:00:00 2001 From: Danny McCormick <dannymccormick@google.com> Date: Thu, 9 Nov 2023 09:38:42 -0500 Subject: [PATCH 410/435] Update beam_PreCommit_PythonDocker.yml to allow manual triggering (#29367) --- .github/workflows/beam_PreCommit_PythonDocker.yml | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/.github/workflows/beam_PreCommit_PythonDocker.yml b/.github/workflows/beam_PreCommit_PythonDocker.yml index 0a0ee5de32a8..3379e22d64c1 100644 --- a/.github/workflows/beam_PreCommit_PythonDocker.yml +++ b/.github/workflows/beam_PreCommit_PythonDocker.yml @@ -67,7 +67,8 @@ jobs: python_version: ['3.8','3.9','3.10','3.11'] if: | github.event_name == 'push' || - github.event_name == 'pull_request_target' || + github.event_name == 'pull_request_target' || + github.event_name == 'workflow_dispatch' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || startsWith(github.event.comment.body, 'Run PythonDocker PreCommit') steps: @@ -101,4 +102,4 @@ jobs: gradle-command: :sdks:python:container:py${{steps.set_py_ver_clean.outputs.py_ver_clean}}:docker arguments: | -Pposargs=apache_beam/dataframe/ \ - -PpythonVersion=${{ matrix.python_version }} \ No newline at end of file + -PpythonVersion=${{ matrix.python_version }} From 938a2ba1343cf419fb9250f3f72226f23d712a46 Mon Sep 17 00:00:00 2001 From: Danny McCormick <dannymccormick@google.com> Date: Thu, 9 Nov 2023 10:28:32 -0500 Subject: [PATCH 411/435] Updates to release guide RC creation (#29315) * Updates from release to guide * More changes * Update link * Add mention of automated key * Unneeded info that requires inspecting automation --- contributor-docs/release-guide.md | 43 ++++++++++--------------------- 1 file changed, 13 insertions(+), 30 deletions(-) diff --git a/contributor-docs/release-guide.md b/contributor-docs/release-guide.md index 712e77a25b17..39944ebb15c6 100644 --- a/contributor-docs/release-guide.md +++ b/contributor-docs/release-guide.md @@ -555,19 +555,16 @@ The following should be confirmed: ### Run build_release_candidate GitHub Action to create a release candidate -Note: This step is partially automated (in progress), so part of the RC -creation is done by GitHub Actions and the rest is done by a script. You don't -need to wait for the action to complete to start running the script. - **Action** [build_release_candidate](https://github.com/apache/beam/actions/workflows/build_release_candidate.yml) (click `run workflow`) **The action will:** 1. Clone the repo at the selected RC tag. 2. Run gradle publish to push java artifacts into Maven staging repo. -3. Stage SDK docker images to [docker hub Apache +3. Build and push java and python source distribution into [dist.apache.org](https://dist.apache.org/repos/dist/dev/beam). +4. Stage SDK docker images to [docker hub Apache organization](https://hub.docker.com/search?q=apache%2Fbeam&type=image). -4. Build javadoc, pydoc, typedocs for a PR to update beam-site. +5. Build javadoc, pydoc, typedocs for a PR to update beam-site. - **NOTE**: Do not merge this PR until after an RC has been approved (see "Finalize the Release"). @@ -585,22 +582,23 @@ with tags: `${RELEASE_VERSION}_rc{RC_NUM}` Verify that third party licenses are included in Docker. You can do this with a simple script: + RC_TAG=${RELEASE_VERSION}_rc{RC_NUM} for pyver in 3.8 3.9 3.10 3.11; do docker run --rm --entrypoint sh \ - apache/beam_python${pyver}_sdk:2.51.0rc1 \ + apache/beam_python${pyver}_sdk:${RC_TAG} \ -c 'ls -al /opt/apache/beam/third_party_licenses/ | wc -l' done for javaver in 8 11 17; do docker run --rm --entrypoint sh \ - apache/beam_java${pyver}_sdk:2.51.0rc1 \ + apache/beam_java${pyver}_sdk:${RC_TAG} \ -c 'ls -al /opt/apache/beam/third_party_licenses/ | wc -l' done And you may choose to log in to the containers and inspect: docker run --rm -it --entrypoint=/bin/bash \ - apache/beam_java${ver}_sdk:${RELEASE_VERSION}rc${RC_NUM} + apache/beam_java${ver}_sdk:${RC_TAG} ls -al /opt/apache/beam/third_party_licenses/ ### Publish Java staging artifacts (manual) @@ -623,17 +621,9 @@ This step uploads artifacts such as `apache-beam-${RELEASE_VERSION}rc${RC_NUM}` to PyPI, so the RC artifacts can be depended upon directly by consumers, for ease of RC verification. -**Script:** [deploy_release_candidate_pypi.sh](https://github.com/apache/beam/blob/master/release/src/main/scripts/deploy_release_candidate_pypi.sh) - -**Usage** - - ./release/src/main/scripts/deploy_release_candidate_pypi.sh \ - --release "${RELEASE_VERSION}" \ - --rc "${RC_NUM}" \ - --user "${GITHUB_USER}" \ - --deploy +**Action** [deploy_release_candidate_pypi](https://github.com/apache/beam/actions/workflows/deploy_release_candidate_pypi.yml) (click `run workflow`) -**The script will:** +**The Action will:** Download previously build python binary artifacts Deploy release candidate to PyPI with an `rc` suffix. @@ -645,19 +635,13 @@ __Attention:__ Verify that: - [ ] Release source's zip published - [ ] Signatures and hashes do not need to be uploaded -You can do a dry run by omitting the `--deploy` flag. Then it will only -download the release candidate binaries. If it looks good, rerun it with -`--deploy`. - -See the source of the script for more details or to run commands manually in -case of a problem. - ### Propose pull requests for website updates Beam publishes API reference manuals for each release on the website. For Java and Python SDKs, that’s Javadoc and PyDoc, respectively. The final step of building the candidate is to propose website pull requests that update these -manuals. +manuals. The first pr will get created by the build_release_candidate action, +you will need to create the second one manually Merge the pull requests only after finalizing the release. To avoid invalid redirects for the 'current' version, merge these PRs in the order listed. Once @@ -837,11 +821,10 @@ template; please adjust as you see fit. The complete staging area is available for your review, which includes: * GitHub Release notes [1], - * the official Apache source release to be deployed to dist.apache.org [2], which is signed with the key with fingerprint FFFFFFFF [3], + * the official Apache source release to be deployed to dist.apache.org [2], which is signed with the key with fingerprint FFFFFFFF (D20316F712213422 if automated) [3], * all artifacts to be deployed to the Maven Central Repository [4], * source code tag "v1.2.3-RC3" [5], * website pull request listing the release [6], the blog post [6], and publishing the API reference manual [7]. - * Java artifacts were built with Gradle GRADLE_VERSION and OpenJDK/Oracle JDK JDK_VERSION. * Python artifacts are deployed along with the source release to the dist.apache.org [2] and PyPI[8]. * Go artifacts and documentation are available at pkg.go.dev [9] * Validation sheet with a tab for 1.2.3 release to help with validation [10]. @@ -850,7 +833,7 @@ template; please adjust as you see fit. The vote will be open for at least 72 hours. It is adopted by majority approval, with at least 3 PMC affirmative votes. - For guidelines on how to try the release in your projects, check out our blog post at /blog/validate-beam-release/. + For guidelines on how to try the release in your projects, check out our blog post at https://beam.apache.org/blog/validate-beam-release/. Thanks, Release Manager From 0f06037324d322857a72e59d0d0cb32cd60bef6f Mon Sep 17 00:00:00 2001 From: Danny McCormick <dannymccormick@google.com> Date: Thu, 9 Nov 2023 12:45:33 -0500 Subject: [PATCH 412/435] Add job for validating release snapshots (#29370) --- .../beam_PostRelease_NightlySnapshot.yml | 68 +++++++++++++++++++ 1 file changed, 68 insertions(+) create mode 100644 .github/workflows/beam_PostRelease_NightlySnapshot.yml diff --git a/.github/workflows/beam_PostRelease_NightlySnapshot.yml b/.github/workflows/beam_PostRelease_NightlySnapshot.yml new file mode 100644 index 000000000000..6e8e5bda7f6e --- /dev/null +++ b/.github/workflows/beam_PostRelease_NightlySnapshot.yml @@ -0,0 +1,68 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: Release Nightly Snapshot + +on: + workflow_dispatch: + inputs: + RELEASE: + description: Beam version of current release (e.g. 2.XX.0) + required: true + default: '2.XX.0' + SNAPSHOT_URL: + description: Location of the staged artifacts in Maven central (https://repository.apache.org/content/repositories/orgapachebeam-NNNN/). + required: true + schedule: + - cron: '15 16 * * *' + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_Release_NightlySnapshot: + name: beam_Release_NightlySnapshot + runs-on: [self-hosted, ubuntu-20.04, main] + steps: + - uses: actions/checkout@v4 + - name: Setup environment + uses: ./.github/actions/setup-environment-action + with: + java-version: 8 + - name: run PostRelease validation script + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :release:runJavaExamplesValidationTask + arguments: | + -Pver='${{ github.event.inputs.RELEASE }}' \ + -Prepourl='${{ github.event.inputs.SNAPSHOT_URL }}' \ From 20e6e87436c03c705096b65757baaec122ab7f5f Mon Sep 17 00:00:00 2001 From: Yi Hu <yathu@google.com> Date: Thu, 9 Nov 2023 12:49:46 -0500 Subject: [PATCH 413/435] Fix cleanup script missing scenario (#29371) --- .test-infra/dataproc/cleanup.sh | 4 ++-- .test-infra/tools/stale_bq_datasets_cleaner.sh | 7 ++++++- 2 files changed, 8 insertions(+), 3 deletions(-) diff --git a/.test-infra/dataproc/cleanup.sh b/.test-infra/dataproc/cleanup.sh index 3efb80d8613f..ac12a0bff05c 100755 --- a/.test-infra/dataproc/cleanup.sh +++ b/.test-infra/dataproc/cleanup.sh @@ -21,7 +21,7 @@ toDeleteList=( ) generatedResources=("beam-loadtests-go-*-flink" "beam-loadtests-python-*-flink" \ -"beam-postcommit-python-chicago" ) +"beam-loadtests-py-*-flink" "beam-postcommit-python-chicago" ) function deleteFilteredClusters(){ for cluster in ${toDeleteList[@]};do @@ -48,7 +48,7 @@ function filterClusters(){ if [[ $elapsedHours -ge 2 ]]; then for name in ${generatedResources[@]}; do # Only resources generated by the groovy jobs set are queued for deletion - if [[ "$cluster" == *${name}* ]]; then + if [[ "$cluster" == *${name}* && ! ("$cluster" =~ nokill) ]]; then toDeleteList+=( "$cluster" ) break fi diff --git a/.test-infra/tools/stale_bq_datasets_cleaner.sh b/.test-infra/tools/stale_bq_datasets_cleaner.sh index ee3f27ef671e..47e8ea95498e 100755 --- a/.test-infra/tools/stale_bq_datasets_cleaner.sh +++ b/.test-infra/tools/stale_bq_datasets_cleaner.sh @@ -41,7 +41,12 @@ for dataset in ${BQ_DATASETS[@]}; do LAST_MODIFIED=$(($LAST_MODIFIED_MS / 1000)) if [[ $GRACE_PERIOD -gt $LAST_MODIFIED ]]; then if bq --project_id=$PROJECT rm -r -f $dataset; then - echo "Deleted $dataset (modified `date -d @$LAST_MODIFIED`)" + if [[ $OSTYPE == "linux-gnu"* ]]; then + # date command usage depending on OS + echo "Deleted $dataset (modified `date -d @$LAST_MODIFIED`)" + elif [[ $OSTYPE == "darwin"* ]]; then + echo "Deleted $dataset (modified `date -r @$LAST_MODIFIED`)" + fi else failed_calls+=1 fi From 334693351e34b5ba72cfc3abd9400dfbebc6fc11 Mon Sep 17 00:00:00 2001 From: Danny McCormick <dannymccormick@google.com> Date: Thu, 9 Nov 2023 12:58:44 -0500 Subject: [PATCH 414/435] Fix beam_PostRelease_NightlySnapshot.yml naming (#29373) --- .github/workflows/beam_PostRelease_NightlySnapshot.yml | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/.github/workflows/beam_PostRelease_NightlySnapshot.yml b/.github/workflows/beam_PostRelease_NightlySnapshot.yml index 6e8e5bda7f6e..73b3d46f5b92 100644 --- a/.github/workflows/beam_PostRelease_NightlySnapshot.yml +++ b/.github/workflows/beam_PostRelease_NightlySnapshot.yml @@ -13,7 +13,7 @@ # See the License for the specific language governing permissions and # limitations under the License. -name: Release Nightly Snapshot +name: PostRelease Nightly Snapshot on: workflow_dispatch: @@ -50,8 +50,8 @@ env: GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} jobs: - beam_Release_NightlySnapshot: - name: beam_Release_NightlySnapshot + beam_PostRelease_NightlySnapshot: + name: beam_PostRelease_NightlySnapshot runs-on: [self-hosted, ubuntu-20.04, main] steps: - uses: actions/checkout@v4 From 79a38858e38c8fe90cfc14e4ae304d0af0ae53ac Mon Sep 17 00:00:00 2001 From: Robert Bradshaw <robertwb@gmail.com> Date: Thu, 9 Nov 2023 10:32:58 -0800 Subject: [PATCH 415/435] Add some missing finally blocks on context managers. This could should exit regardless of whether the operations under the context threw exceptions. --- sdks/python/apache_beam/dataframe/expressions.py | 6 ++++-- sdks/python/apache_beam/dataframe/pandas_docs_test.py | 6 ++++-- .../runners/portability/flink_uber_jar_job_server_test.py | 8 +++++--- .../runners/portability/spark_uber_jar_job_server_test.py | 8 +++++--- sdks/python/apache_beam/runners/worker/logger.py | 6 ++++-- sdks/python/apache_beam/transforms/external.py | 8 +++++--- .../transforms/fully_qualified_named_transform.py | 6 ++++-- 7 files changed, 31 insertions(+), 17 deletions(-) diff --git a/sdks/python/apache_beam/dataframe/expressions.py b/sdks/python/apache_beam/dataframe/expressions.py index ac7e1f828aca..ae08cdaf54cd 100644 --- a/sdks/python/apache_beam/dataframe/expressions.py +++ b/sdks/python/apache_beam/dataframe/expressions.py @@ -404,8 +404,10 @@ def allow_non_parallel_operations(allow=True): yield else: old_value, _ALLOW_NON_PARALLEL.value = _ALLOW_NON_PARALLEL.value, allow - yield - _ALLOW_NON_PARALLEL.value = old_value + try: + yield + finally: + _ALLOW_NON_PARALLEL.value = old_value class NonParallelOperation(Exception): diff --git a/sdks/python/apache_beam/dataframe/pandas_docs_test.py b/sdks/python/apache_beam/dataframe/pandas_docs_test.py index d52773c955f1..8302c3cb53b6 100644 --- a/sdks/python/apache_beam/dataframe/pandas_docs_test.py +++ b/sdks/python/apache_beam/dataframe/pandas_docs_test.py @@ -145,8 +145,10 @@ def run_tests(path): def deferred_stdout(): captured = io.StringIO() old_stdout, sys.stdout = sys.stdout, captured - yield captured.getvalue - sys.stdout = old_stdout + try: + yield captured.getvalue + finally: + sys.stdout = old_stdout if __name__ == '__main__': diff --git a/sdks/python/apache_beam/runners/portability/flink_uber_jar_job_server_test.py b/sdks/python/apache_beam/runners/portability/flink_uber_jar_job_server_test.py index 1294f4653b2a..12ba3940d396 100644 --- a/sdks/python/apache_beam/runners/portability/flink_uber_jar_job_server_test.py +++ b/sdks/python/apache_beam/runners/portability/flink_uber_jar_job_server_test.py @@ -37,9 +37,11 @@ def temp_name(*args, **kwargs): with tempfile.NamedTemporaryFile(*args, **kwargs) as t: name = t.name - yield name - if os.path.exists(name): - os.unlink(name) + try: + yield name + finally: + if os.path.exists(name): + os.unlink(name) class FlinkUberJarJobServerTest(unittest.TestCase): diff --git a/sdks/python/apache_beam/runners/portability/spark_uber_jar_job_server_test.py b/sdks/python/apache_beam/runners/portability/spark_uber_jar_job_server_test.py index 6bb27b5746da..a99bec840bee 100644 --- a/sdks/python/apache_beam/runners/portability/spark_uber_jar_job_server_test.py +++ b/sdks/python/apache_beam/runners/portability/spark_uber_jar_job_server_test.py @@ -40,9 +40,11 @@ def temp_name(*args, **kwargs): with tempfile.NamedTemporaryFile(*args, **kwargs) as t: name = t.name - yield name - if os.path.exists(name): - os.unlink(name) + try: + yield name + finally: + if os.path.exists(name): + os.unlink(name) def spark_job(): diff --git a/sdks/python/apache_beam/runners/worker/logger.py b/sdks/python/apache_beam/runners/worker/logger.py index e01e3863349b..e1c84bc6ded2 100644 --- a/sdks/python/apache_beam/runners/worker/logger.py +++ b/sdks/python/apache_beam/runners/worker/logger.py @@ -64,8 +64,10 @@ def PerThreadLoggingContext(**kwargs): """A context manager to add per thread attributes.""" stack = per_thread_worker_data.stack stack.append(kwargs) - yield - stack.pop() + try: + yield + finally: + stack.pop() class JsonLogFormatter(logging.Formatter): diff --git a/sdks/python/apache_beam/transforms/external.py b/sdks/python/apache_beam/transforms/external.py index 44bf2398a6dd..0d0b6f1e7be2 100644 --- a/sdks/python/apache_beam/transforms/external.py +++ b/sdks/python/apache_beam/transforms/external.py @@ -663,9 +663,11 @@ def get_local_namespace(cls): @contextlib.contextmanager def outer_namespace(cls, namespace): prev = cls.get_local_namespace() - cls._external_namespace.value = namespace - yield - cls._external_namespace.value = prev + try: + cls._external_namespace.value = namespace + yield + finally: + cls._external_namespace.value = prev @classmethod def _fresh_namespace(cls): diff --git a/sdks/python/apache_beam/transforms/fully_qualified_named_transform.py b/sdks/python/apache_beam/transforms/fully_qualified_named_transform.py index f9b1c12d5133..ab2cadd166a9 100644 --- a/sdks/python/apache_beam/transforms/fully_qualified_named_transform.py +++ b/sdks/python/apache_beam/transforms/fully_qualified_named_transform.py @@ -43,8 +43,10 @@ class FullyQualifiedNamedTransform(ptransform.PTransform): @contextlib.contextmanager def with_filter(cls, filter): old_filter, cls._FILTER_GLOB = cls._FILTER_GLOB, filter - yield - cls._FILTER_GLOB = old_filter + try: + yield + finally: + cls._FILTER_GLOB = old_filter def __init__(self, constructor, args, kwargs): self._constructor = constructor From e84eab2c1984745c7a9e280b25cf7dbbbe60815d Mon Sep 17 00:00:00 2001 From: Danny McCormick <dannymccormick@google.com> Date: Thu, 9 Nov 2023 14:14:47 -0500 Subject: [PATCH 416/435] Upgrade go version in python docker build (#29369) * Upgrade go version in python docker build * Down to 1.17 * Try 1.18 * Try 1.19 * Update beam_PreCommit_PythonDocker.yml --- .github/workflows/beam_PreCommit_PythonDocker.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/beam_PreCommit_PythonDocker.yml b/.github/workflows/beam_PreCommit_PythonDocker.yml index 3379e22d64c1..26ecf97c9b52 100644 --- a/.github/workflows/beam_PreCommit_PythonDocker.yml +++ b/.github/workflows/beam_PreCommit_PythonDocker.yml @@ -84,7 +84,7 @@ jobs: with: java-version: 8 python-version: ${{ matrix.python_version }} - go-version: 1.16 + go-version: 1.21 - name: Setup Buildx uses: docker/setup-buildx-action@v2 with: From 8957177858056c009f6c2ec6f59b1697a843d333 Mon Sep 17 00:00:00 2001 From: Yi Hu <yathu@google.com> Date: Thu, 9 Nov 2023 17:37:19 -0500 Subject: [PATCH 417/435] Add Java21 Example Dataflow Legacy worker PostCommit and ARM PostCommit (#29353) * Add Java21 Example Dataflow Legacy worker PostCommit and ARM PostCommit * Update workflow readme --- .github/workflows/README.md | 4 +-- ..._PostCommit_Java_Examples_Dataflow_ARM.yml | 2 +- ...PostCommit_Java_Examples_Dataflow_Java.yml | 2 +- .../beam/gradle/BeamModulePlugin.groovy | 2 +- .../arm/build.gradle | 36 +++++++++++++++++-- .../examples/build.gradle | 13 ++++++- 6 files changed, 50 insertions(+), 9 deletions(-) diff --git a/.github/workflows/README.md b/.github/workflows/README.md index 5afbae8677d3..bd4dcc7d067c 100644 --- a/.github/workflows/README.md +++ b/.github/workflows/README.md @@ -269,11 +269,11 @@ Please note that jobs with matrix need to have matrix element in the comment. Ex | [ PostCommit Java Avro Versions ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Avro_Versions.yml) | N/A |`Run Java Avro Versions PostCommit`| [![.github/workflows/beam_PostCommit_Java_Avro_Versions.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Avro_Versions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Avro_Versions.yml?query=event%3Aschedule) | | [ PostCommit Java Dataflow V1 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_DataflowV1.yml) | N/A |`Run PostCommit_Java_Dataflow`| [![.github/workflows/beam_PostCommit_Java_DataflowV1.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_DataflowV1.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_DataflowV1.yml?query=event%3Aschedule) | | [ PostCommit Java Dataflow V2 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_DataflowV2.yml) | N/A |`Run PostCommit_Java_DataflowV2`| [![.github/workflows/beam_PostCommit_Java_DataflowV2.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_DataflowV2.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_DataflowV2.yml?query=event%3Aschedule) | -| [ PostCommit Java Examples Dataflow ARM ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml) | ['8','11','17'] |`Run Java_Examples_Dataflow_ARM PostCommit (matrix_element)`| [![.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml?query=event%3Aschedule) | +| [ PostCommit Java Examples Dataflow ARM ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml) | ['8','11','17','21'] |`Run Java_Examples_Dataflow_ARM PostCommit (matrix_element)`| [![.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml?query=event%3Aschedule) | | [ PostCommit Java Examples Dataflow](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow.yml) | N/A |`Run Java examples on Dataflow`| [![.github/workflows/beam_PostCommit_Java_Examples_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow.yml?query=event%3Aschedule) | | [ PostCommit Java Examples Dataflow Java ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml) | ['11','17','21'] |`Run Java examples on Dataflow Java (matrix_element)`| [![.github/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml?query=event%3Aschedule) | | [ PostCommit Java Examples Dataflow V2 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml) | N/A |`Run Java Examples on Dataflow Runner V2`| [![.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml?query=event%3Aschedule) | -| [ PostCommit Java Examples Dataflow V2 Java ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml) | ['11','17'] |`Run Java (matrix_element) Examples on Dataflow Runner V2`| [![.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml?query=event%3Aschedule) | +| [ PostCommit Java Examples Dataflow V2 Java ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml) | ['11','17','21'] |`Run Java (matrix_element) Examples on Dataflow Runner V2`| [![.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml?query=event%3Aschedule) | | [ PostCommit Java Examples Direct ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Direct.yml) | N/A |`Run Java Examples_Direct`| [![.github/workflows/beam_PostCommit_Java_Examples_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Direct.yml?query=event%3Aschedule) | | [ PostCommit Java Examples Flink ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Flink.yml) | N/A |`Run Java Examples_Flink`| [![.github/workflows/beam_PostCommit_Java_Examples_Flink.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Flink.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Flink.yml?query=event%3Aschedule) | | [ PostCommit Java Examples Spark ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Spark.yml) | N/A |`Run Java Examples_Spark`| [![.github/workflows/beam_PostCommit_Java_Examples_Spark.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Spark.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Spark.yml?query=event%3Aschedule) | diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml index 16e7275c724a..26ea487e0dcf 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml @@ -62,7 +62,7 @@ jobs: matrix: job_name: [beam_PostCommit_Java_Examples__Dataflow_ARM] job_phrase: [Run Java_Examples_Dataflow_ARM PostCommit] - java_version: ['8','11','17'] + java_version: ['8','11','17','21'] if: | github.event_name == 'push' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml index 54644e27160a..747c9f0983fc 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml @@ -60,7 +60,7 @@ jobs: matrix: job_name: [beam_PostCommit_Java_Examples_Dataflow_Java] job_phrase: [Run Java examples on Dataflow Java] - java_version: ['11','17'] + java_version: ['11','17','21'] if: | github.event_name == 'workflow_dispatch' || github.event_name == 'pull_request_target' || diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index a16b21f4d633..e7eee0e2198f 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -970,7 +970,7 @@ class BeamModulePlugin implements Plugin<Project> { def java21Home = project.findProperty("java21Home") options.fork = true options.forkOptions.javaHome = java21Home as File - options.compilerArgs += ['-Xlint:-path'] + options.compilerArgs += ['-Xlint:-path', '-Xlint:-this-escape'] // Error prone requires some packages to be exported/opened for Java 17+ // Disabling checks since this property is only used for tests options.errorprone.errorproneArgs.add("-XepDisableAllChecks") diff --git a/runners/google-cloud-dataflow-java/arm/build.gradle b/runners/google-cloud-dataflow-java/arm/build.gradle index 71cbc7c58e86..90971db35d77 100644 --- a/runners/google-cloud-dataflow-java/arm/build.gradle +++ b/runners/google-cloud-dataflow-java/arm/build.gradle @@ -71,7 +71,12 @@ configurations { examplesJavaIntegrationTest } dependencies { examplesJavaIntegrationTest project(project.path) - examplesJavaIntegrationTest project(path: ":runners:google-cloud-dataflow-java", configuration: "testRuntimeMigration") + // TODO(yathu) Include full test classpath once gradle shadow plugin does not support Java21 + if (project.findProperty('testJavaVersion') == '21' || JavaVersion.current().equals(JavaVersion.VERSION_21)) { + examplesJavaIntegrationTest project(path: ":runners:google-cloud-dataflow-java") + } else { + examplesJavaIntegrationTest project(path: ":runners:google-cloud-dataflow-java", configuration: "testRuntimeMigration") + } examplesJavaIntegrationTest project(path: ":examples:java", configuration: "testRuntimeMigration") } @@ -86,14 +91,14 @@ def firestoreDb = project.findProperty('firestoreDb') ?: 'firestoredb' def dockerImageRoot = project.findProperty('docker-repository-root') ?: "us.gcr.io/${dataflowProject}/java-postcommit-it" def DockerJavaMultiarchImageContainer = "${dockerImageRoot}/${project.docker_image_default_repo_prefix}${javaVer}_sdk" def dockerTag = project.findProperty('docker-tag') ?: new Date().format('yyyyMMddHHmmss') -ext.DockerJavaMultiarchImageName = "${DockerJavaMultiarchImageContainer}:${dockerTag}" +ext.DockerJavaMultiarchImageName = "${DockerJavaMultiarchImageContainer}:${dockerTag}" as String def runnerV2PipelineOptionsARM = [ "--runner=TestDataflowRunner", "--project=${dataflowProject}", "--region=${dataflowRegion}", "--tempRoot=${dataflowValidatesTempRoot}", - "--sdkContainerImage=${DockerJavaMultiarchImageContainer}:${dockerTag}", + "--sdkContainerImage=${project.ext.DockerJavaMultiarchImageName}", "--experiments=use_unified_worker,use_runner_v2", "--firestoreDb=${firestoreDb}", "--workerMachineType=t2a-standard-1", @@ -135,3 +140,28 @@ task examplesJavaRunnerV2IntegrationTestARM(type: Test) { testClassesDirs = files(project(":examples:java").sourceSets.test.output.classesDirs) useJUnit { } } + +// Clean up built Java images +def cleanUpDockerJavaImages = tasks.register("cleanUpDockerJavaImages") { + doLast { + exec { + commandLine "docker", "rmi", "--force", "${DockerJavaMultiarchImageName}" + } + exec { + ignoreExitValue true + commandLine "gcloud", "--quiet", "container", "images", "untag", "${DockerJavaMultiarchImageName}" + } + exec { + commandLine "./../scripts/cleanup_untagged_gcr_images.sh", "${DockerJavaMultiarchImageContainer}" + } + } +} + +afterEvaluate { + // Ensure all tasks which use published docker images run before they are cleaned up + tasks.each { t -> + if (t.dependsOn.contains(buildAndPushDockerJavaMultiarchContainer) && !t.name.equalsIgnoreCase('printrunnerV2PipelineOptionsARM')) { + t.finalizedBy cleanUpDockerJavaImages + } + } +} diff --git a/runners/google-cloud-dataflow-java/examples/build.gradle b/runners/google-cloud-dataflow-java/examples/build.gradle index 96f8c07992dd..fa1b1b82e301 100644 --- a/runners/google-cloud-dataflow-java/examples/build.gradle +++ b/runners/google-cloud-dataflow-java/examples/build.gradle @@ -84,7 +84,7 @@ def commonConfig = { Map args -> include "**/WordCountIT.class" include "**/WindowedWordCountIT.class" } else { - include "**/IT.class" + include "**/*IT.class" if (runWordCount == 'exclude') { exclude "**/WordCountIT.class" exclude "**/WindowedWordCountIT.class" @@ -171,6 +171,17 @@ task java17PostCommit() { dependsOn postCommitLegacyWorkerJava17 } +task postCommitLegacyWorkerJava21(type: Test) { + dependsOn ":runners:google-cloud-dataflow-java:worker:shadowJar" + def dataflowWorkerJar = project.findProperty('dataflowWorkerJar') ?: project(":runners:google-cloud-dataflow-java:worker").shadowJar.archivePath + systemProperty "java.specification.version", "21" + with commonConfig(dataflowWorkerJar: dataflowWorkerJar, runWordCount: 'exclude') +} + +task java21PostCommit() { + dependsOn postCommitLegacyWorkerJava21 +} + task preCommit() { dependsOn preCommitLegacyWorker dependsOn preCommitLegacyWorkerImpersonate From f47d47350e099e54ae09a4684d24ab3c4e3768bd Mon Sep 17 00:00:00 2001 From: Danny McCormick <dannymccormick@google.com> Date: Thu, 9 Nov 2023 17:37:59 -0500 Subject: [PATCH 418/435] Sickbay test_bigtable_write on Dataflow runner (#29197) * Sickbay test_bigtable_write on Dataflow runner * More sickbay * Sickbay correctly --- .../apache_beam/examples/cookbook/bigtableio_it_test.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/examples/cookbook/bigtableio_it_test.py b/sdks/python/apache_beam/examples/cookbook/bigtableio_it_test.py index 98023fbc624c..8cc8b3c73a10 100644 --- a/sdks/python/apache_beam/examples/cookbook/bigtableio_it_test.py +++ b/sdks/python/apache_beam/examples/cookbook/bigtableio_it_test.py @@ -174,7 +174,9 @@ def tearDown(self): if self.instance.exists(): self.instance.delete() - @pytest.mark.it_postcommit + # TODO(https://github.com/apache/beam/issues/29076): Reenable this test + # once BigTable issues are fixed. + @pytest.mark.it_postcommit_sickbay def test_bigtable_write(self): number = self.number pipeline_args = self.test_pipeline.options_list From 3823df25c27789b5006d4b0b2ec1525aa760a49c Mon Sep 17 00:00:00 2001 From: darshan-sj <darshan-sj@users.noreply.github.com> Date: Fri, 10 Nov 2023 19:36:38 +0530 Subject: [PATCH 419/435] Adding support for pg Jsonb datatype in SpannerIO (#29313) * Adding support for pg Jsonb datatype in SpannerIO * spotless apply --- .../io/gcp/spanner/MutationSizeEstimator.java | 11 +++++ .../sdk/io/gcp/spanner/SpannerSchema.java | 3 ++ .../spanner/MutationSizeEstimatorTest.java | 41 +++++++++++++++++++ .../sdk/io/gcp/spanner/SpannerSchemaTest.java | 6 ++- 4 files changed, 59 insertions(+), 2 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/MutationSizeEstimator.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/MutationSizeEstimator.java index 7ff480ea7a76..b0a1da5fb15e 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/MutationSizeEstimator.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/MutationSizeEstimator.java @@ -126,6 +126,8 @@ private static long estimatePrimitiveValue(Value v) { return v.isNull() ? 0 : v.getNumeric().toString().length(); case JSON: return v.isNull() ? 0 : v.getJson().length(); + case PG_JSONB: + return v.isNull() ? 0 : v.getPgJsonb().length(); default: throw new IllegalArgumentException("Unsupported type " + v.getType()); } @@ -188,6 +190,15 @@ private static long estimateArrayValue(Value v) { totalLength += s.length(); } return totalLength; + case PG_JSONB: + totalLength = 0; + for (String s : v.getPgJsonbArray()) { + if (s == null) { + continue; + } + totalLength += s.length(); + } + return totalLength; default: throw new IllegalArgumentException("Unsupported type " + v.getType()); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerSchema.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerSchema.java index fbeaac8ecd24..e7587365fe12 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerSchema.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerSchema.java @@ -233,6 +233,9 @@ private static Type parseSpannerType(String spannerType, Dialect dialect) { if ("SPANNER.COMMIT_TIMESTAMP".equals(spannerType)) { return Type.timestamp(); } + if (spannerType.startsWith("JSONB")) { + return Type.pgJsonb(); + } throw new IllegalArgumentException("Unknown spanner type " + spannerType); default: throw new IllegalArgumentException("Unrecognized dialect: " + dialect.name()); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/MutationSizeEstimatorTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/MutationSizeEstimatorTest.java index f05159cbbe35..ebabfa8b575f 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/MutationSizeEstimatorTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/MutationSizeEstimatorTest.java @@ -65,6 +65,11 @@ public void primitives() throws Exception { .to(Value.json("{\"key1\":\"value1\", \"key2\":\"value2\"}")) .build(); Mutation deleteDouble = Mutation.delete("test", Key.of(1223.)); + Mutation jsonb = + Mutation.newInsertOrUpdateBuilder("test") + .set("one") + .to(Value.pgJsonb("{\"key123\":\"value123\", \"key321\":\"value321\"}")) + .build(); assertThat(MutationSizeEstimator.sizeOf(int64), is(8L)); assertThat(MutationSizeEstimator.sizeOf(float64), is(8L)); @@ -74,6 +79,7 @@ public void primitives() throws Exception { assertThat(MutationSizeEstimator.sizeOf(pgNumericNaN), is(3L)); assertThat(MutationSizeEstimator.sizeOf(json), is(34L)); assertThat(MutationSizeEstimator.sizeOf(deleteDouble), is(8L)); + assertThat(MutationSizeEstimator.sizeOf(jsonb), is(42L)); } @Test @@ -131,6 +137,14 @@ public void primitiveArrays() throws Exception { ByteArray.copyFrom("some_bytes".getBytes(UTF_8)), ByteArray.copyFrom("some_bytes".getBytes(UTF_8)))) .build(); + Mutation jsonb = + Mutation.newInsertOrUpdateBuilder("test") + .set("one") + .toPgJsonbArray( + ImmutableList.of( + "{\"key123\":\"value123\", \"key321\":\"value321\"}", + "{\"key456\":\"value456\", \"key789\":600}")) + .build(); assertThat(MutationSizeEstimator.sizeOf(int64), is(24L)); assertThat(MutationSizeEstimator.sizeOf(float64), is(16L)); assertThat(MutationSizeEstimator.sizeOf(bool), is(4L)); @@ -138,6 +152,7 @@ public void primitiveArrays() throws Exception { assertThat(MutationSizeEstimator.sizeOf(pgNumeric), is(156L)); assertThat(MutationSizeEstimator.sizeOf(json), is(62L)); assertThat(MutationSizeEstimator.sizeOf(bytes), is(20L)); + assertThat(MutationSizeEstimator.sizeOf(jsonb), is(77L)); } @Test @@ -162,6 +177,8 @@ public void nullPrimitiveArrays() throws Exception { .toPgNumericArray((Iterable<String>) null) .build(); Mutation json = Mutation.newInsertOrUpdateBuilder("test").set("one").toJsonArray(null).build(); + Mutation jsonb = + Mutation.newInsertOrUpdateBuilder("test").set("one").toPgJsonbArray(null).build(); assertThat(MutationSizeEstimator.sizeOf(int64), is(0L)); assertThat(MutationSizeEstimator.sizeOf(float64), is(0L)); @@ -169,6 +186,7 @@ public void nullPrimitiveArrays() throws Exception { assertThat(MutationSizeEstimator.sizeOf(numeric), is(0L)); assertThat(MutationSizeEstimator.sizeOf(pgNumeric), is(0L)); assertThat(MutationSizeEstimator.sizeOf(json), is(0L)); + assertThat(MutationSizeEstimator.sizeOf(jsonb), is(0L)); } @Test @@ -237,6 +255,29 @@ public void jsons() throws Exception { assertThat(MutationSizeEstimator.sizeOf(nullArray), is(0L)); } + @Test + public void pgJsonb() throws Exception { + Mutation empty = + Mutation.newInsertOrUpdateBuilder("test").set("one").to(Value.pgJsonb("{}")).build(); + Mutation nullValue = + Mutation.newInsertOrUpdateBuilder("test") + .set("one") + .to(Value.pgJsonb((String) null)) + .build(); + Mutation sample = + Mutation.newInsertOrUpdateBuilder("test") + .set("one") + .to(Value.pgJsonb("{\"type_name\":\"number\",\"value\":12345.123}")) + .build(); + Mutation nullArray = + Mutation.newInsertOrUpdateBuilder("test").set("one").toPgJsonbArray(null).build(); + + assertThat(MutationSizeEstimator.sizeOf(empty), is(2L)); + assertThat(MutationSizeEstimator.sizeOf(nullValue), is(0L)); + assertThat(MutationSizeEstimator.sizeOf(sample), is(40L)); + assertThat(MutationSizeEstimator.sizeOf(nullArray), is(0L)); + } + @Test public void dates() throws Exception { Mutation timestamp = diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerSchemaTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerSchemaTest.java index d65486b3938a..7ba345a24885 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerSchemaTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerSchemaTest.java @@ -75,10 +75,11 @@ public void testSinglePgTable() throws Exception { .addColumn("test", "maxKey", "character varying") .addColumn("test", "numericVal", "numeric") .addColumn("test", "commitTime", "spanner.commit_timestamp") + .addColumn("test", "jsonbCol", "jsonb") .build(); assertEquals(1, schema.getTables().size()); - assertEquals(4, schema.getColumns("test").size()); + assertEquals(5, schema.getColumns("test").size()); assertEquals(1, schema.getKeyParts("test").size()); assertEquals(Type.timestamp(), schema.getColumns("test").get(3).getType()); } @@ -90,6 +91,7 @@ public void testTwoPgTables() throws Exception { .addColumn("test", "pk", "character varying(48)") .addKeyPart("test", "pk", false) .addColumn("test", "maxKey", "character varying") + .addColumn("test", "jsonbCol", "jsonb") .addColumn("other", "pk", "bigint") .addKeyPart("other", "pk", true) .addColumn("other", "maxKey", "character varying") @@ -97,7 +99,7 @@ public void testTwoPgTables() throws Exception { .build(); assertEquals(2, schema.getTables().size()); - assertEquals(2, schema.getColumns("test").size()); + assertEquals(3, schema.getColumns("test").size()); assertEquals(1, schema.getKeyParts("test").size()); assertEquals(3, schema.getColumns("other").size()); From 8dde4508076ae3563207753fea9cc454750880dd Mon Sep 17 00:00:00 2001 From: Yi Hu <yathu@google.com> Date: Fri, 10 Nov 2023 11:17:38 -0500 Subject: [PATCH 420/435] Fix clean up resource workflow; fix XVR Flink (#29358) * Fix clean up resource workflow; gix XVR Flink * Move spanner cleaner to correct workflow * Fix k8s naming missing cases * Fix env --- .../workflows/beam_PostCommit_XVR_Flink.yml | 2 +- .test-infra/tools/build.gradle | 5 ++++ .../tools/stale_dataflow_jobs_cleaner.sh | 7 ----- .../tools/stale_k8s_workload_cleaner.sh | 2 +- .test-infra/tools/stale_spanner_cleaner.sh | 29 +++++++++++++++++++ 5 files changed, 36 insertions(+), 9 deletions(-) create mode 100755 .test-infra/tools/stale_spanner_cleaner.sh diff --git a/.github/workflows/beam_PostCommit_XVR_Flink.yml b/.github/workflows/beam_PostCommit_XVR_Flink.yml index 75d3353e8a15..00c756a714f3 100644 --- a/.github/workflows/beam_PostCommit_XVR_Flink.yml +++ b/.github/workflows/beam_PostCommit_XVR_Flink.yml @@ -56,7 +56,7 @@ jobs: github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run XVR_Flink PostCommit' - runs-on: [self-hosted, ubuntu-20.04, main] + runs-on: [self-hosted, ubuntu-20.04, highmem] timeout-minutes: 100 name: ${{ matrix.job_name }} (${{ matrix.job_phrase }} ${{ matrix.python_version }}) strategy: diff --git a/.test-infra/tools/build.gradle b/.test-infra/tools/build.gradle index 048fc0a8d023..274cd3022911 100644 --- a/.test-infra/tools/build.gradle +++ b/.test-infra/tools/build.gradle @@ -36,9 +36,14 @@ task removeStaleK8sWorkload(type: Exec) { commandLine './stale_k8s_workload_cleaner.sh' } +task removeStaleSpannerResources(type: Exec) { + commandLine './stale_spanner_cleaner.sh' +} + task cleanupOtherStaleResources { // declared as finalizedBy dependency so that other task continue even if one dep task fails finalizedBy tasks.removeStaleBqDatasets finalizedBy tasks.removeStaleCbtInstances finalizedBy tasks.removeStaleK8sWorkload + finalizedBy tasks.removeStaleSpannerResources } diff --git a/.test-infra/tools/stale_dataflow_jobs_cleaner.sh b/.test-infra/tools/stale_dataflow_jobs_cleaner.sh index e6df7058427b..e56304807475 100755 --- a/.test-infra/tools/stale_dataflow_jobs_cleaner.sh +++ b/.test-infra/tools/stale_dataflow_jobs_cleaner.sh @@ -27,10 +27,3 @@ if [[ ${STALE_JOBS} ]]; then else echo "No stale jobs found." fi - -# Delete spanner databases older than 1 day. -gcloud spanner databases list \ ---instance beam-test \ ---filter="createTime < $(date --iso-8601=s -d '1 day ago')" \ ---format="value(name)" | \ -xargs -I{} gcloud spanner databases delete {} --instance beam-test --quiet diff --git a/.test-infra/tools/stale_k8s_workload_cleaner.sh b/.test-infra/tools/stale_k8s_workload_cleaner.sh index f4d6f9b88324..9ddaf17f2ce8 100755 --- a/.test-infra/tools/stale_k8s_workload_cleaner.sh +++ b/.test-infra/tools/stale_k8s_workload_cleaner.sh @@ -43,7 +43,7 @@ function should_teardown() { gcloud container clusters get-credentials io-datastores --zone us-central1-a --project apache-beam-testing while read NAME STATUS AGE; do - if [[ $NAME =~ ^beam-.+test ]] && should_teardown $AGE; then + if [[ $NAME =~ ^beam-.+(test|-it) ]] && should_teardown $AGE; then kubectl delete namespace $NAME fi done < <( kubectl get namespaces --context=gke_${PROJECT}_${LOCATION}_${CLUSTER} ) diff --git a/.test-infra/tools/stale_spanner_cleaner.sh b/.test-infra/tools/stale_spanner_cleaner.sh new file mode 100755 index 000000000000..28ecc6e47c98 --- /dev/null +++ b/.test-infra/tools/stale_spanner_cleaner.sh @@ -0,0 +1,29 @@ +#!/usr/bin/env bash +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +# Deletes stale and old BQ datasets that are left after tests. +set -euo pipefail + +PROJECT=apache-beam-testing + +# Delete spanner databases older than 1 day. +gcloud spanner databases list \ +--instance beam-test \ +--project $PROJECT \ +--filter="createTime < $(date --iso-8601=s -d '1 day ago')" \ +--format="value(name)" | \ +xargs -I{} gcloud spanner databases delete {} --instance beam-test --quiet From 1b4a27c291b9e67a6b088a9094f5f671f4041ee0 Mon Sep 17 00:00:00 2001 From: Danny McCormick <dannymccormick@google.com> Date: Fri, 10 Nov 2023 11:26:19 -0500 Subject: [PATCH 421/435] Create workflow for finalizing release (#29389) --- .github/workflows/finalize_release.yml | 141 +++++++++++++++++++++++++ 1 file changed, 141 insertions(+) create mode 100644 .github/workflows/finalize_release.yml diff --git a/.github/workflows/finalize_release.yml b/.github/workflows/finalize_release.yml new file mode 100644 index 000000000000..cb1d9571a34a --- /dev/null +++ b/.github/workflows/finalize_release.yml @@ -0,0 +1,141 @@ +name: finalize_release + +# Workflow added after https://github.com/apache/beam/commit/4183e747becebd18becee5fff547af365910fc9c +# If help is needed debugging issues, you can view the release guide at that commit for guidance on how to do this manually. +# (https://github.com/apache/beam/blob/4183e747becebd18becee5fff547af365910fc9c/website/www/site/content/en/contribute/release-guide.md) +on: + workflow_dispatch: + inputs: + RELEASE: + description: Beam version of current release (e.g. 2.XX.0) + required: true + default: '2.XX.0' + RC: + description: Integer RC version for the release that we'd like to finalize (e.g. 3 for RC3) + required: true + PYPI_USER: + description: PyPi username to perform the PyPi upload with + required: false + PYPI_PASSWORD: + description: PyPi password to perform the PyPi upload with + required: false + PUSH_DOCKER_ARTIFACTS: + description: Whether to push SDK docker images to docker hub Apache organization. Should be yes unless you've already completed this step. + required: true + default: 'no' + PUBLISH_PYTHON_ARTIFACTS: + description: Whether to publish the python artifacts into PyPi. Should be yes unless you've already completed this step. + required: true + default: 'no' + TAG_RELEASE: + description: Whether to tag the release on GitHub. Should be yes unless you've already completed this step. + required: true + default: 'no' +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + push_docker_artifacts: + if: ${{github.event.inputs.PUSH_DOCKER_ARTIFACTS == 'yes'}} + runs-on: [self-hosted, ubuntu-20.04, main] + steps: + - name: Publish to Docker + env: + RELEASE: "${{ github.event.inputs.RELEASE }}" + RC_NUM: "${{ github.event.inputs.RC }}" + RC_VERSION: "rc_${{ github.event.inputs.RC }}" + run: | + + echo "Publish SDK docker images to Docker Hub." + + echo "================Pull RC Containers from DockerHub===========" + IMAGES=$(docker search apache/beam_ --format "{{.Name}}" --limit 100) + KNOWN_IMAGES=() + echo "We are using ${RC_VERSION} to push docker images for ${RELEASE}." + while read IMAGE; do + # Try pull verified RC from dockerhub. + if docker pull "${IMAGE}:${RELEASE}${RC_VERSION}" 2>/dev/null ; then + KNOWN_IMAGES+=( $IMAGE ) + fi + done < <(echo "${IMAGES}") + + echo "================Confirming Release and RC version===========" + echo "Publishing the following images:" + # Sort by name for easy examination + IFS=$'\n' KNOWN_IMAGES=($(sort <<<"${KNOWN_IMAGES[*]}")) + unset IFS + printf "%s\n" ${KNOWN_IMAGES[@]} + + for IMAGE in "${KNOWN_IMAGES[@]}"; do + # Perform a carbon copy of ${RC_VERSION} to dockerhub with a new tag as ${RELEASE}. + docker buildx imagetools create --tag "${IMAGE}:${RELEASE}" "${IMAGE}:${RELEASE}${RC_VERSION}" + + # Perform a carbon copy of ${RC_VERSION} to dockerhub with a new tag as latest. + docker buildx imagetools create --tag "${IMAGE}:latest" "${IMAGE}:${RELEASE}" + done + + publish_python_artifacts: + if: ${{github.event.inputs.PUBLISH_PYTHON_ARTIFACTS == 'yes'}} + runs-on: [self-hosted, ubuntu-20.04, main] + steps: + - name: Checkout + uses: actions/checkout@v4 + - name: Mask and validate PyPi id/password + run: | + echo "::add-mask::${{ github.event.inputs.PYPI_USER }}" + echo "::add-mask::${{ github.event.inputs.PYPI_PASSWORD }}" + if [ "${{ github.event.inputs.PYPI_USER }}" == "" ] + then + echo "Must provide a PyPi username to publish artifacts to PyPi" + exit 1 + fi + if [ "${{ github.event.inputs.PYPI_PASSWORD }}" == "" ] + then + echo "Must provide a PyPi password to publish artifacts to PyPi" + exit 1 + fi + - name: Setup environment + uses: ./.github/actions/setup-environment-action + with: + python-version: 3.11 + - name: Install dependencies + run: | + pip install python-dateutil + pip install requests + pip install twine + - name: Deploy to Pypi + env: + RELEASE: "${{ github.event.inputs.RELEASE }}" + run: | + wget -r --no-parent -A zip,whl "https://dist.apache.org/repos/dist/dev/beam/${RELEASE}/python" + cd "dist.apache.org/repos/dist/dev/beam/${RELEASE}/python/" + echo "Will upload the following files to PyPI:" + ls + twine upload * -u ${{ github.event.inputs.PYPI_USER }} -p ${{ github.event.inputs.PYPI_PASSWORD }} + + push_git_tags: + if: ${{github.event.inputs.TAG_RELEASE == 'yes'}} + runs-on: [self-hosted, ubuntu-20.04, main] + steps: + - name: Checkout + uses: actions/checkout@v4 + - name: Push tags + env: + VERSION_TAG: "v${{ github.event.inputs.RELEASE }}" + RC_TAG: "${{ github.event.inputs.RELEASE }}-RC${{ github.event.inputs.RC }}" + run: | + # Ensure local tags are in sync. If there's a mismatch, it will tell you. + git fetch --all --tags + + # If the tag exists, a commit number is produced, otherwise there's an error. + git rev-list $RC_TAG -n 1 + + # Tag for Go SDK + git tag -s "sdks/$VERSION_TAG" "$RC_TAG" + git push https://github.com/apache/beam "sdks/$VERSION_TAG" + + # Tag for repo root. + git tag -s "$VERSION_TAG" "$RC_TAG" + git push https://github.com/apache/beam "$VERSION_TAG" From 56739a0fd69d9f8ca9af44ec59ce5808f257974c Mon Sep 17 00:00:00 2001 From: Yi Hu <yathu@google.com> Date: Fri, 10 Nov 2023 11:39:14 -0500 Subject: [PATCH 422/435] Followup #29353 fix several test failures (#29384) * Fix spotless on buildSrc * Ignore exit value for cleanup container jobs * Fix #29386 --- .github/workflows/beam_PreCommit_Spotless.yml | 2 +- .../groovy/org/apache/beam/gradle/BeamModulePlugin.groovy | 5 ++++- runners/google-cloud-dataflow-java/arm/build.gradle | 1 + runners/google-cloud-dataflow-java/build.gradle | 3 +++ runners/google-cloud-dataflow-java/examples/build.gradle | 2 +- 5 files changed, 10 insertions(+), 3 deletions(-) diff --git a/.github/workflows/beam_PreCommit_Spotless.yml b/.github/workflows/beam_PreCommit_Spotless.yml index c3a2f08d65f1..6aa590e7556e 100644 --- a/.github/workflows/beam_PreCommit_Spotless.yml +++ b/.github/workflows/beam_PreCommit_Spotless.yml @@ -97,7 +97,7 @@ jobs: - name: run Spotless PreCommit script uses: ./.github/actions/gradle-command-self-hosted-action with: - gradle-command: spotlessCheck checkStyleMain checkStyleTest + gradle-command: spotlessCheck checkStyleMain checkStyleTest :buildSrc:spotlessCheck - name: Upload test report uses: actions/upload-artifact@v3 with: diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index e7eee0e2198f..2a11abb7e6df 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -970,7 +970,10 @@ class BeamModulePlugin implements Plugin<Project> { def java21Home = project.findProperty("java21Home") options.fork = true options.forkOptions.javaHome = java21Home as File - options.compilerArgs += ['-Xlint:-path', '-Xlint:-this-escape'] + options.compilerArgs += [ + '-Xlint:-path', + '-Xlint:-this-escape' + ] // Error prone requires some packages to be exported/opened for Java 17+ // Disabling checks since this property is only used for tests options.errorprone.errorproneArgs.add("-XepDisableAllChecks") diff --git a/runners/google-cloud-dataflow-java/arm/build.gradle b/runners/google-cloud-dataflow-java/arm/build.gradle index 90971db35d77..a3d191671d1b 100644 --- a/runners/google-cloud-dataflow-java/arm/build.gradle +++ b/runners/google-cloud-dataflow-java/arm/build.gradle @@ -152,6 +152,7 @@ def cleanUpDockerJavaImages = tasks.register("cleanUpDockerJavaImages") { commandLine "gcloud", "--quiet", "container", "images", "untag", "${DockerJavaMultiarchImageName}" } exec { + ignoreExitValue true commandLine "./../scripts/cleanup_untagged_gcr_images.sh", "${DockerJavaMultiarchImageContainer}" } } diff --git a/runners/google-cloud-dataflow-java/build.gradle b/runners/google-cloud-dataflow-java/build.gradle index 69759fc76a6f..5d967d90f2b8 100644 --- a/runners/google-cloud-dataflow-java/build.gradle +++ b/runners/google-cloud-dataflow-java/build.gradle @@ -308,6 +308,7 @@ def cleanUpDockerJavaImages = tasks.register("cleanUpDockerJavaImages") { commandLine "gcloud", "--quiet", "container", "images", "untag", "${dockerJavaImageName}" } exec { + ignoreExitValue true commandLine "./scripts/cleanup_untagged_gcr_images.sh", "${dockerJavaImageContainer}" } } @@ -346,9 +347,11 @@ def cleanUpDockerPythonImages = tasks.register("cleanUpDockerPythonImages") { commandLine "docker", "rmi", "--force", "${dockerPythonImageName}" } exec { + ignoreExitValue true commandLine "gcloud", "--quiet", "container", "images", "untag", "${dockerPythonImageName}" } exec { + ignoreExitValue true commandLine "./scripts/cleanup_untagged_gcr_images.sh", "${dockerPythonImageContainer}" } } diff --git a/runners/google-cloud-dataflow-java/examples/build.gradle b/runners/google-cloud-dataflow-java/examples/build.gradle index fa1b1b82e301..34addac695b1 100644 --- a/runners/google-cloud-dataflow-java/examples/build.gradle +++ b/runners/google-cloud-dataflow-java/examples/build.gradle @@ -36,7 +36,7 @@ dependencies { def gcpProject = project.findProperty('gcpProject') ?: 'apache-beam-testing' def gcpRegion = project.findProperty('gcpRegion') ?: 'us-central1' -def gcsTempRoot = project.findProperty('gcsTempRoot') ?: 'gs://temp-storage-for-end-to-end-tests/' +def gcsTempRoot = project.findProperty('gcsTempRoot') ?: 'gs://temp-storage-for-end-to-end-tests' def dockerJavaImageName = project(':runners:google-cloud-dataflow-java').ext.dockerJavaImageName // If -PuseExecutableStage is set, the use_executable_stage_bundle_execution wil be enabled. def fnapiExperiments = project.hasProperty('useExecutableStage') ? 'beam_fn_api_use_deprecated_read,use_executable_stage_bundle_execution' : "beam_fn_api,beam_fn_api_use_deprecated_read" From c713425e1ac2cdc3ec2ec264c9bf61f7356856bd Mon Sep 17 00:00:00 2001 From: "gabry.wu" <gabrywu@apache.org> Date: Sat, 11 Nov 2023 00:52:10 +0800 Subject: [PATCH 423/435] Involve CheckStopReadingFn class adding a chance to release resources gracefully (#29100) * involve CheckStopReadingFn class adding a chance to release resources gracefully * involve CheckStopReadingFnWrapper to avoid break changes --- .../beam/sdk/io/kafka/CheckStopReadingFn.java | 27 ++++++++++++ .../io/kafka/CheckStopReadingFnWrapper.java | 43 ++++++++++++++++++ .../org/apache/beam/sdk/io/kafka/KafkaIO.java | 44 ++++++++++++++++--- .../beam/sdk/io/kafka/ReadFromKafkaDoFn.java | 8 +++- .../kafka/WatchForKafkaTopicPartitions.java | 22 ++++++++-- .../apache/beam/sdk/io/kafka/KafkaIOIT.java | 11 ++--- 6 files changed, 136 insertions(+), 19 deletions(-) create mode 100644 sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/CheckStopReadingFn.java create mode 100644 sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/CheckStopReadingFnWrapper.java diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/CheckStopReadingFn.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/CheckStopReadingFn.java new file mode 100644 index 000000000000..20192508491d --- /dev/null +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/CheckStopReadingFn.java @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.kafka; + +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.kafka.common.TopicPartition; + +public interface CheckStopReadingFn extends SerializableFunction<TopicPartition, Boolean> { + default void setup() {} + + default void teardown() {} +} diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/CheckStopReadingFnWrapper.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/CheckStopReadingFnWrapper.java new file mode 100644 index 000000000000..e4012e965097 --- /dev/null +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/CheckStopReadingFnWrapper.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.kafka; + +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.kafka.common.TopicPartition; +import org.checkerframework.checker.nullness.qual.Nullable; + +public class CheckStopReadingFnWrapper implements CheckStopReadingFn { + private final SerializableFunction<TopicPartition, Boolean> serializableFunction; + + private CheckStopReadingFnWrapper( + SerializableFunction<TopicPartition, Boolean> serializableFunction) { + this.serializableFunction = serializableFunction; + } + + public static @Nullable CheckStopReadingFnWrapper of( + @Nullable SerializableFunction<TopicPartition, Boolean> serializableFunction) { + return serializableFunction != null + ? new CheckStopReadingFnWrapper(serializableFunction) + : null; + } + + @Override + public Boolean apply(TopicPartition input) { + return serializableFunction.apply(input); + } +} diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java index ea0c34e576b9..26f6c3448801 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java @@ -685,7 +685,7 @@ public abstract static class Read<K, V> abstract @Nullable DeserializerProvider<V> getValueDeserializerProvider(); @Pure - abstract @Nullable SerializableFunction<TopicPartition, Boolean> getCheckStopReadingFn(); + abstract @Nullable CheckStopReadingFn getCheckStopReadingFn(); abstract Builder<K, V> toBuilder(); @@ -733,8 +733,12 @@ abstract Builder<K, V> setKeyDeserializerProvider( abstract Builder<K, V> setValueDeserializerProvider( DeserializerProvider<V> deserializerProvider); - abstract Builder<K, V> setCheckStopReadingFn( - SerializableFunction<TopicPartition, Boolean> checkStopReadingFn); + abstract Builder<K, V> setCheckStopReadingFn(@Nullable CheckStopReadingFn checkStopReadingFn); + + Builder<K, V> setCheckStopReadingFn( + @Nullable SerializableFunction<TopicPartition, Boolean> checkStopReadingFn) { + return setCheckStopReadingFn(CheckStopReadingFnWrapper.of(checkStopReadingFn)); + } abstract Read<K, V> build(); @@ -1269,13 +1273,23 @@ public Read<K, V> withConsumerConfigUpdates(Map<String, Object> configUpdates) { return toBuilder().setConsumerConfig(config).build(); } + /** + * A custom {@link CheckStopReadingFn} that determines whether the {@link ReadFromKafkaDoFn} + * should stop reading from the given {@link TopicPartition}. + */ + public Read<K, V> withCheckStopReadingFn(CheckStopReadingFn checkStopReadingFn) { + return toBuilder().setCheckStopReadingFn(checkStopReadingFn).build(); + } + /** * A custom {@link SerializableFunction} that determines whether the {@link ReadFromKafkaDoFn} * should stop reading from the given {@link TopicPartition}. */ public Read<K, V> withCheckStopReadingFn( SerializableFunction<TopicPartition, Boolean> checkStopReadingFn) { - return toBuilder().setCheckStopReadingFn(checkStopReadingFn).build(); + return toBuilder() + .setCheckStopReadingFn(CheckStopReadingFnWrapper.of(checkStopReadingFn)) + .build(); } /** Returns a {@link PTransform} for PCollection of {@link KV}, dropping Kafka metatdata. */ @@ -1947,7 +1961,7 @@ public abstract static class ReadSourceDescriptors<K, V> getConsumerFactoryFn(); @Pure - abstract @Nullable SerializableFunction<TopicPartition, Boolean> getCheckStopReadingFn(); + abstract @Nullable CheckStopReadingFn getCheckStopReadingFn(); @Pure abstract @Nullable SerializableFunction<KafkaRecord<K, V>, Instant> @@ -1978,7 +1992,12 @@ abstract ReadSourceDescriptors.Builder<K, V> setConsumerFactoryFn( SerializableFunction<Map<String, Object>, Consumer<byte[], byte[]>> consumerFactoryFn); abstract ReadSourceDescriptors.Builder<K, V> setCheckStopReadingFn( - @Nullable SerializableFunction<TopicPartition, Boolean> checkStopReadingFn); + @Nullable CheckStopReadingFn checkStopReadingFn); + + ReadSourceDescriptors.Builder<K, V> setCheckStopReadingFn( + @Nullable SerializableFunction<TopicPartition, Boolean> checkStopReadingFn) { + return setCheckStopReadingFn(CheckStopReadingFnWrapper.of(checkStopReadingFn)); + } abstract ReadSourceDescriptors.Builder<K, V> setKeyDeserializerProvider( @Nullable DeserializerProvider<K> deserializerProvider); @@ -2096,13 +2115,24 @@ public ReadSourceDescriptors<K, V> withConsumerFactoryFn( return toBuilder().setConsumerFactoryFn(consumerFactoryFn).build(); } + /** + * A custom {@link CheckStopReadingFn} that determines whether the {@link ReadFromKafkaDoFn} + * should stop reading from the given {@link TopicPartition}. + */ + public ReadSourceDescriptors<K, V> withCheckStopReadingFn( + @Nullable CheckStopReadingFn checkStopReadingFn) { + return toBuilder().setCheckStopReadingFn(checkStopReadingFn).build(); + } + /** * A custom {@link SerializableFunction} that determines whether the {@link ReadFromKafkaDoFn} * should stop reading from the given {@link TopicPartition}. */ public ReadSourceDescriptors<K, V> withCheckStopReadingFn( @Nullable SerializableFunction<TopicPartition, Boolean> checkStopReadingFn) { - return toBuilder().setCheckStopReadingFn(checkStopReadingFn).build(); + return toBuilder() + .setCheckStopReadingFn(CheckStopReadingFnWrapper.of(checkStopReadingFn)) + .build(); } /** diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/ReadFromKafkaDoFn.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/ReadFromKafkaDoFn.java index 31620549ab22..4b0035aa3564 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/ReadFromKafkaDoFn.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/ReadFromKafkaDoFn.java @@ -182,7 +182,7 @@ private ReadFromKafkaDoFn(ReadSourceDescriptors<K, V> transform) { private final @Nullable Map<String, Object> offsetConsumerConfig; - private final @Nullable SerializableFunction<TopicPartition, Boolean> checkStopReadingFn; + private final @Nullable CheckStopReadingFn checkStopReadingFn; private final SerializableFunction<Map<String, Object>, Consumer<byte[], byte[]>> consumerFactoryFn; @@ -514,6 +514,9 @@ public AverageRecordSize load(TopicPartition topicPartition) throws Exception { keyDeserializerInstance = keyDeserializerProvider.getDeserializer(consumerConfig, true); valueDeserializerInstance = valueDeserializerProvider.getDeserializer(consumerConfig, false); offsetEstimatorCache = new HashMap<>(); + if (checkStopReadingFn != null) { + checkStopReadingFn.setup(); + } } @Teardown @@ -532,6 +535,9 @@ public void teardown() throws Exception { if (offsetEstimatorCache != null) { offsetEstimatorCache.clear(); } + if (checkStopReadingFn != null) { + checkStopReadingFn.teardown(); + } } private Map<String, Object> overrideBootstrapServersConfig( diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/WatchForKafkaTopicPartitions.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/WatchForKafkaTopicPartitions.java index ed67257ca454..0d60640316e8 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/WatchForKafkaTopicPartitions.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/WatchForKafkaTopicPartitions.java @@ -63,7 +63,7 @@ class WatchForKafkaTopicPartitions extends PTransform<PBegin, PCollection<KafkaS private final SerializableFunction<Map<String, Object>, Consumer<byte[], byte[]>> kafkaConsumerFactoryFn; private final Map<String, Object> kafkaConsumerConfig; - private final @Nullable SerializableFunction<TopicPartition, Boolean> checkStopReadingFn; + private final @Nullable CheckStopReadingFn checkStopReadingFn; private final Set<String> topics; private final @Nullable Pattern topicPattern; private final @Nullable Instant startReadTime; @@ -73,7 +73,7 @@ public WatchForKafkaTopicPartitions( @Nullable Duration checkDuration, SerializableFunction<Map<String, Object>, Consumer<byte[], byte[]>> kafkaConsumerFactoryFn, Map<String, Object> kafkaConsumerConfig, - @Nullable SerializableFunction<TopicPartition, Boolean> checkStopReadingFn, + @Nullable CheckStopReadingFn checkStopReadingFn, Set<String> topics, @Nullable Pattern topicPattern, @Nullable Instant startReadTime, @@ -104,12 +104,12 @@ public PCollection<KafkaSourceDescriptor> expand(PBegin input) { private static class ConvertToDescriptor extends DoFn<KV<byte[], TopicPartition>, KafkaSourceDescriptor> { - private final @Nullable SerializableFunction<TopicPartition, Boolean> checkStopReadingFn; + private final @Nullable CheckStopReadingFn checkStopReadingFn; private final @Nullable Instant startReadTime; private final @Nullable Instant stopReadTime; private ConvertToDescriptor( - @Nullable SerializableFunction<TopicPartition, Boolean> checkStopReadingFn, + @Nullable CheckStopReadingFn checkStopReadingFn, @Nullable Instant startReadTime, @Nullable Instant stopReadTime) { this.checkStopReadingFn = checkStopReadingFn; @@ -131,6 +131,20 @@ public void processElement( topicPartition, null, startReadTime, null, stopReadTime, null)); } } + + @Setup + public void setup() throws Exception { + if (checkStopReadingFn != null) { + checkStopReadingFn.setup(); + } + } + + @Teardown + public void teardown() throws Exception { + if (checkStopReadingFn != null) { + checkStopReadingFn.teardown(); + } + } } private static class WatchPartitionFn extends PollFn<byte[], TopicPartition> { diff --git a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java index f600e14d30f6..2c8ace9c66c1 100644 --- a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java +++ b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java @@ -71,7 +71,6 @@ import org.apache.beam.sdk.transforms.Keys; import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.Values; import org.apache.beam.sdk.transforms.windowing.CalendarWindows; import org.apache.beam.sdk.transforms.windowing.FixedWindows; @@ -483,14 +482,14 @@ public void testKafkaWithDynamicPartitions() throws IOException { @Test public void testKafkaWithStopReadingFunction() { - CheckStopReadingFn checkStopReadingFn = new CheckStopReadingFn(); + AlwaysStopCheckStopReadingFn checkStopReadingFn = new AlwaysStopCheckStopReadingFn(); PipelineResult readResult = runWithStopReadingFn(checkStopReadingFn, "stop-reading"); assertEquals(-1, readElementMetric(readResult, NAMESPACE, READ_ELEMENT_METRIC_NAME)); } - private static class CheckStopReadingFn implements SerializableFunction<TopicPartition, Boolean> { + private static class AlwaysStopCheckStopReadingFn implements CheckStopReadingFn { @Override public Boolean apply(TopicPartition input) { return true; @@ -640,8 +639,7 @@ public void runReadWriteKafkaViaSchemaTransforms( assertEquals(PipelineResult.State.DONE, readResult.getState()); } - private static class DelayedCheckStopReadingFn - implements SerializableFunction<TopicPartition, Boolean> { + private static class DelayedCheckStopReadingFn implements CheckStopReadingFn { int checkCount = 0; @Override @@ -654,8 +652,7 @@ public Boolean apply(TopicPartition input) { } } - private PipelineResult runWithStopReadingFn( - SerializableFunction<TopicPartition, Boolean> function, String topicSuffix) { + private PipelineResult runWithStopReadingFn(CheckStopReadingFn function, String topicSuffix) { writePipeline .apply("Generate records", Read.from(new SyntheticBoundedSource(sourceOptions))) .apply("Measure write time", ParDo.of(new TimeMonitor<>(NAMESPACE, WRITE_TIME_METRIC_NAME))) From aefc8880d8005aa77a427f3d6908e0e509cd128a Mon Sep 17 00:00:00 2001 From: Robert Burke <lostluck@users.noreply.github.com> Date: Fri, 10 Nov 2023 11:32:31 -0800 Subject: [PATCH 424/435] Update default Go version to 1.21.4 (#29385) --- .../main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index 2a11abb7e6df..70da837b3b34 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -2200,7 +2200,7 @@ class BeamModulePlugin implements Plugin<Project> { def goRootDir = "${project.rootDir}/sdks/go" // This sets the whole project Go version. - project.ext.goVersion = "go1.21.3" + project.ext.goVersion = "go1.21.4" // Minor TODO: Figure out if we can pull out the GOCMD env variable after goPrepare script // completion, and avoid this GOBIN substitution. From 281a4fa6db477af3d85415a775a43ebf98aef6c2 Mon Sep 17 00:00:00 2001 From: Yi Hu <yathu@google.com> Date: Fri, 10 Nov 2023 21:42:59 -0500 Subject: [PATCH 425/435] Fix MongoDBIO ignore SSL KeyStore not initialized (#29393) --- .../apache/beam/sdk/io/mongodb/SSLUtils.java | 2 +- .../beam/sdk/io/mongodb/SSLUtilsTest.java | 32 +++++++++++++++++++ 2 files changed, 33 insertions(+), 1 deletion(-) create mode 100644 sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/SSLUtilsTest.java diff --git a/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/SSLUtils.java b/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/SSLUtils.java index 1c4697289311..f68aaaea95dd 100644 --- a/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/SSLUtils.java +++ b/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/SSLUtils.java @@ -68,12 +68,12 @@ static SSLContext ignoreSSLCertificate() { InputStream inputStream = classLoader.getResourceAsStream("resources/.keystore"); if (inputStream != null) { LOG.info("Found keystore in classpath 'resources/.keystore'. Loading..."); - ks.load(inputStream, "changeit".toCharArray()); } else { LOG.info( "Unable to find keystore under 'resources/.keystore' in the classpath. " + "Continuing with an empty keystore."); } + ks.load(inputStream, "changeit".toCharArray()); KeyManagerFactory kmf = KeyManagerFactory.getInstance(KeyManagerFactory.getDefaultAlgorithm()); kmf.init(ks, "changeit".toCharArray()); diff --git a/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/SSLUtilsTest.java b/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/SSLUtilsTest.java new file mode 100644 index 000000000000..978163129f87 --- /dev/null +++ b/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/SSLUtilsTest.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.mongodb; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Test on the MongoDbIO SSLUtils. */ +@RunWith(JUnit4.class) +public class SSLUtilsTest { + @Test + public void testIgnoreSSLCertificate() { + // smoke test + SSLUtils.ignoreSSLCertificate(); + } +} From 1e1c2a319f6e873070a2e848341555700706cc2a Mon Sep 17 00:00:00 2001 From: tvalentyn <tvalentyn@users.noreply.github.com> Date: Fri, 10 Nov 2023 18:57:15 -0800 Subject: [PATCH 426/435] Invoke pyarrow_hotfix to alleviate concerns due to CVE-2023-47248. (#29396) --- sdks/python/apache_beam/__init__.py | 8 ++++++++ .../container/py310/base_image_requirements.txt | 1 + .../container/py311/base_image_requirements.txt | 1 + .../container/py38/base_image_requirements.txt | 1 + .../container/py39/base_image_requirements.txt | 1 + sdks/python/setup.py | 12 +++++++++--- 6 files changed, 21 insertions(+), 3 deletions(-) diff --git a/sdks/python/apache_beam/__init__.py b/sdks/python/apache_beam/__init__.py index a4a13eab9738..85b518718528 100644 --- a/sdks/python/apache_beam/__init__.py +++ b/sdks/python/apache_beam/__init__.py @@ -94,6 +94,14 @@ from apache_beam.pvalue import PCollection from apache_beam.pvalue import Row from apache_beam.pvalue import TaggedOutput + +try: + # Add mitigation for CVE-2023-47248 while Beam allows affected versions + # of pyarrow. (https://github.com/apache/beam/issues/29392) + import pyarrow_hotfix +except ImportError: + pass + # pylint: enable=wrong-import-position __version__ = version.__version__ diff --git a/sdks/python/container/py310/base_image_requirements.txt b/sdks/python/container/py310/base_image_requirements.txt index 035d5697b0d7..a9f94104374e 100644 --- a/sdks/python/container/py310/base_image_requirements.txt +++ b/sdks/python/container/py310/base_image_requirements.txt @@ -101,6 +101,7 @@ proto-plus==1.22.3 protobuf==4.25.0 psycopg2-binary==2.9.9 pyarrow==11.0.0 +pyarrow-hotfix==0.4 pyasn1==0.5.0 pyasn1-modules==0.3.0 pycparser==2.21 diff --git a/sdks/python/container/py311/base_image_requirements.txt b/sdks/python/container/py311/base_image_requirements.txt index f6fea89d0188..865b856683a4 100644 --- a/sdks/python/container/py311/base_image_requirements.txt +++ b/sdks/python/container/py311/base_image_requirements.txt @@ -98,6 +98,7 @@ proto-plus==1.22.3 protobuf==4.25.0 psycopg2-binary==2.9.9 pyarrow==11.0.0 +pyarrow-hotfix==0.4 pyasn1==0.5.0 pyasn1-modules==0.3.0 pycparser==2.21 diff --git a/sdks/python/container/py38/base_image_requirements.txt b/sdks/python/container/py38/base_image_requirements.txt index d2d48f19d351..5dffff5f80d9 100644 --- a/sdks/python/container/py38/base_image_requirements.txt +++ b/sdks/python/container/py38/base_image_requirements.txt @@ -105,6 +105,7 @@ proto-plus==1.22.3 protobuf==4.25.0 psycopg2-binary==2.9.9 pyarrow==11.0.0 +pyarrow-hotfix==0.4 pyasn1==0.5.0 pyasn1-modules==0.3.0 pycparser==2.21 diff --git a/sdks/python/container/py39/base_image_requirements.txt b/sdks/python/container/py39/base_image_requirements.txt index 4fb867387475..1b8ad7a2e748 100644 --- a/sdks/python/container/py39/base_image_requirements.txt +++ b/sdks/python/container/py39/base_image_requirements.txt @@ -102,6 +102,7 @@ proto-plus==1.22.3 protobuf==4.25.0 psycopg2-binary==2.9.9 pyarrow==11.0.0 +pyarrow-hotfix==0.4 pyasn1==0.5.0 pyasn1-modules==0.3.0 pycparser==2.21 diff --git a/sdks/python/setup.py b/sdks/python/setup.py index b9fd89f0707e..4310daf6fea8 100644 --- a/sdks/python/setup.py +++ b/sdks/python/setup.py @@ -140,9 +140,15 @@ def cythonize(*args, **kwargs): # [BEAM-8181] pyarrow cannot be installed on 32-bit Windows platforms. if sys.platform == 'win32' and sys.maxsize <= 2**32: - pyarrow_dependency = '' + pyarrow_dependency = [''] else: - pyarrow_dependency = 'pyarrow>=3.0.0,<12.0.0' + pyarrow_dependency = [ + 'pyarrow>=3.0.0,<12.0.0', + # NOTE(https://github.com/apache/beam/issues/29392): We can remove this + # once Beam increases the pyarrow lower bound to a version that fixes CVE. + 'pyarrow-hotfix<1' + ] + # Exclude pandas<=1.4.2 since it doesn't work with numpy 1.24.x. # Exclude 1.5.0 and 1.5.1 because of @@ -308,7 +314,7 @@ def get_portability_package_data(): # Dynamic dependencies must be specified in a separate list, otherwise # Dependabot won't be able to parse the main list. Any dynamic # dependencies will not receive updates from Dependabot. - ] + [pyarrow_dependency], + ] + pyarrow_dependency, python_requires=python_requires, # BEAM-8840: Do NOT use tests_require or setup_requires. extras_require={ From 39986e60ea367566da1f4a49d34a53f7ba75cf98 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Johanna=20=C3=96jeling?= <51084516+johannaojeling@users.noreply.github.com> Date: Sat, 11 Nov 2023 13:25:10 +0100 Subject: [PATCH 427/435] Update image tag pattern for RCs in release docs (#29406) --- contributor-docs/release-guide.md | 6 +++--- website/www/site/content/en/blog/validate-beam-release.md | 6 +++--- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/contributor-docs/release-guide.md b/contributor-docs/release-guide.md index 39944ebb15c6..7855c59ebbda 100644 --- a/contributor-docs/release-guide.md +++ b/contributor-docs/release-guide.md @@ -578,11 +578,11 @@ The following should be confirmed: At [https://hub.docker.com/u/apache](https://hub.docker.com/search?q=apache%2Fbeam&type=image), visit each repository and navigate to "tags" tab. Verify images are pushed -with tags: `${RELEASE_VERSION}_rc{RC_NUM}` +with tags: `${RELEASE_VERSION}rc{RC_NUM}` Verify that third party licenses are included in Docker. You can do this with a simple script: - RC_TAG=${RELEASE_VERSION}_rc{RC_NUM} + RC_TAG=${RELEASE_VERSION}rc{RC_NUM} for pyver in 3.8 3.9 3.10 3.11; do docker run --rm --entrypoint sh \ apache/beam_python${pyver}_sdk:${RC_TAG} \ @@ -775,7 +775,7 @@ as an example. API reference manual](https://beam.apache.org/releases/pydoc/). - [ ] Docker images are published to [DockerHub](https://hub.docker.com/search?q=apache%2Fbeam&type=image) with - tags: `{RELEASE_VERSION}_rc{RC_NUM}`. + tags: `{RELEASE_VERSION}rc{RC_NUM}`. You can (optionally) also do additional verification by: diff --git a/website/www/site/content/en/blog/validate-beam-release.md b/website/www/site/content/en/blog/validate-beam-release.md index 60d5ddb0a791..e4335530cfe3 100644 --- a/website/www/site/content/en/blog/validate-beam-release.md +++ b/website/www/site/content/en/blog/validate-beam-release.md @@ -112,10 +112,10 @@ works well. For Go SDK releases, you can fetch the Go SDK RC using [`go get`](https://golang.org/ref/mod#go-get), by requesting the specific pre-release version. -For example, to request the first release candidate for 2.34.0: +For example, to request the first release candidate for 2.44.0: ``` -go get -d github.com/apache/beam/sdks/v2@v2.34.0-RC1 +go get -d github.com/apache/beam/sdks/v2@v2.44.0-RC1 ``` With that, the Beam version in your `go.mod` will be the specified release candidate. @@ -123,4 +123,4 @@ You can go ahead and run your tests to verify that everything works well. You may need to also specify the RC's matching container when running a job. Use the `--environment_config` flag to specify the release candidate container: -eg. `--environment_config=apache/beam_go_sdk:2.34.0_rc1` +eg. `--environment_config=apache/beam_go_sdk:2.44.0rc1` From 2ad459113134dfffd3caeeb15e821bb644e08663 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ferran=20Fern=C3=A1ndez=20Garrido?= <ffernandez.upc@gmail.com> Date: Sun, 12 Nov 2023 01:27:06 +0100 Subject: [PATCH 428/435] [YAML] - Kafka Read and Write proto (#29368) * [YAML] - Kafka Read and Write proto * [YAML] - Delete packages * [YAML] - Fix discrepancies --- sdks/java/extensions/protobuf/build.gradle | 2 + .../extensions/protobuf/ProtoByteUtils.java | 233 ++++++++++++++++++ .../protobuf/ProtoByteUtilsTest.java | 98 ++++++++ .../protobuf/src/test/resources/README.md | 6 + .../file_descriptor/proto_byte_utils.pb | 13 + .../proto_byte/proto_byte_utils.proto | 35 +++ sdks/java/io/kafka/README.md | 10 + sdks/java/io/kafka/build.gradle | 1 + ...KafkaReadSchemaTransformConfiguration.java | 18 +- .../KafkaReadSchemaTransformProvider.java | 34 ++- .../KafkaWriteSchemaTransformProvider.java | 87 +++---- .../KafkaReadSchemaTransformProviderTest.java | 58 ++++- ...KafkaWriteSchemaTransformProviderTest.java | 77 ++++-- .../file_descriptor/proto_byte_utils.pb | 13 + .../proto_byte/proto_byte_utils.proto | 35 +++ sdks/python/apache_beam/yaml/standard_io.yaml | 4 + 16 files changed, 657 insertions(+), 67 deletions(-) create mode 100644 sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoByteUtils.java create mode 100644 sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/ProtoByteUtilsTest.java create mode 100644 sdks/java/extensions/protobuf/src/test/resources/proto_byte/file_descriptor/proto_byte_utils.pb create mode 100644 sdks/java/extensions/protobuf/src/test/resources/proto_byte/proto_byte_utils.proto create mode 100644 sdks/java/io/kafka/src/test/resources/proto_byte/file_descriptor/proto_byte_utils.pb create mode 100644 sdks/java/io/kafka/src/test/resources/proto_byte/proto_byte_utils.proto diff --git a/sdks/java/extensions/protobuf/build.gradle b/sdks/java/extensions/protobuf/build.gradle index 2696f8886ddd..568d4f220867 100644 --- a/sdks/java/extensions/protobuf/build.gradle +++ b/sdks/java/extensions/protobuf/build.gradle @@ -35,6 +35,8 @@ ext.summary = "Add support to Apache Beam for Google Protobuf." dependencies { implementation library.java.byte_buddy implementation library.java.vendored_guava_32_1_2_jre + implementation library.java.commons_compress + implementation library.java.slf4j_api implementation project(path: ":sdks:java:core", configuration: "shadow") implementation library.java.protobuf_java testImplementation project(path: ":sdks:java:core", configuration: "shadowTest") diff --git a/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoByteUtils.java b/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoByteUtils.java new file mode 100644 index 000000000000..f156fed0f38c --- /dev/null +++ b/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoByteUtils.java @@ -0,0 +1,233 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.extensions.protobuf; + +import static java.util.stream.Collectors.toList; +import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; + +import com.google.protobuf.DescriptorProtos; +import com.google.protobuf.Descriptors; +import com.google.protobuf.DynamicMessage; +import com.google.protobuf.InvalidProtocolBufferException; +import java.io.IOException; +import java.io.InputStream; +import java.io.Serializable; +import java.nio.channels.Channels; +import java.nio.channels.ReadableByteChannel; +import java.util.List; +import org.apache.beam.sdk.io.FileSystems; +import org.apache.beam.sdk.io.fs.MatchResult; +import org.apache.beam.sdk.io.fs.ResourceId; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.transforms.SimpleFunction; +import org.apache.beam.sdk.values.Row; +import org.apache.commons.compress.utils.IOUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Utility class for working with Protocol Buffer (Proto) data in the context of Apache Beam. This + * class provides methods to retrieve Beam Schemas from Proto messages, convert Proto bytes to Beam + * Rows, and vice versa. It also includes utilities for handling Protocol Buffer schemas and related + * file operations. + * + * <p>Users can utilize the methods in this class to facilitate the integration of Proto data + * processing within Apache Beam pipelines, allowing for the seamless transformation of Proto + * messages to Beam Rows and vice versa. + */ +public class ProtoByteUtils { + + private static final Logger LOG = LoggerFactory.getLogger(ProtoByteUtils.class); + + /** + * Retrieves a Beam Schema from a Protocol Buffer message. + * + * @param fileDescriptorPath The path to the File Descriptor Set file. + * @param messageName The name of the Protocol Buffer message. + * @return The Beam Schema representing the Protocol Buffer message. + */ + public static Schema getBeamSchemaFromProto(String fileDescriptorPath, String messageName) { + ProtoSchemaInfo dpd = getProtoDomain(fileDescriptorPath); + ProtoDomain protoDomain = dpd.getProtoDomain(); + return ProtoDynamicMessageSchema.forDescriptor(protoDomain, messageName).getSchema(); + } + + public static SerializableFunction<byte[], Row> getProtoBytesToRowFunction( + String fileDescriptorPath, String messageName) { + + ProtoSchemaInfo dynamicProtoDomain = getProtoDomain(fileDescriptorPath); + ProtoDomain protoDomain = dynamicProtoDomain.getProtoDomain(); + @SuppressWarnings("unchecked") + ProtoDynamicMessageSchema<DynamicMessage> protoDynamicMessageSchema = + ProtoDynamicMessageSchema.forDescriptor(protoDomain, messageName); + return new SimpleFunction<byte[], Row>() { + @Override + public Row apply(byte[] input) { + try { + final Descriptors.Descriptor descriptor = + protoDomain + .getFileDescriptor(dynamicProtoDomain.getFileName()) + .findMessageTypeByName(messageName); + DynamicMessage dynamicMessage = DynamicMessage.parseFrom(descriptor, input); + SerializableFunction<DynamicMessage, Row> res = + protoDynamicMessageSchema.getToRowFunction(); + return res.apply(dynamicMessage); + } catch (InvalidProtocolBufferException e) { + LOG.error("Error parsing to DynamicMessage", e); + throw new RuntimeException(e); + } + } + }; + } + + public static SerializableFunction<Row, byte[]> getRowToProtoBytes( + String fileDescriptorPath, String messageName) { + ProtoSchemaInfo dynamicProtoDomain = getProtoDomain(fileDescriptorPath); + ProtoDomain protoDomain = dynamicProtoDomain.getProtoDomain(); + @SuppressWarnings("unchecked") + ProtoDynamicMessageSchema<DynamicMessage> protoDynamicMessageSchema = + ProtoDynamicMessageSchema.forDescriptor(protoDomain, messageName); + + return new SimpleFunction<Row, byte[]>() { + @Override + public byte[] apply(Row input) { + SerializableFunction<Row, DynamicMessage> res = + protoDynamicMessageSchema.getFromRowFunction(); + return res.apply(input).toByteArray(); + } + }; + } + + /** + * Retrieves a ProtoSchemaInfo containing schema information for the specified Protocol Buffer + * file. + * + * @param fileDescriptorPath The path to the File Descriptor Set file. + * @return ProtoSchemaInfo containing the associated ProtoDomain and File Name. + * @throws RuntimeException if an error occurs during schema retrieval. + */ + private static ProtoSchemaInfo getProtoDomain(String fileDescriptorPath) { + byte[] from = getFileAsBytes(fileDescriptorPath); + try { + DescriptorProtos.FileDescriptorSet descriptorSet = + DescriptorProtos.FileDescriptorSet.parseFrom(from); + return new ProtoSchemaInfo( + descriptorSet.getFile(0).getName(), ProtoDomain.buildFrom(descriptorSet)); + } catch (InvalidProtocolBufferException e) { + throw new RuntimeException(e); + } + } + + /** + * Reads the contents of a file specified by its path and returns them as a byte array. + * + * @param fileDescriptorPath The path to the file to read. + * @return Byte array containing the file contents. + * @throws RuntimeException if an error occurs during file reading. + */ + private static byte[] getFileAsBytes(String fileDescriptorPath) { + ReadableByteChannel channel = getFileByteChannel(fileDescriptorPath); + try (InputStream inputStream = Channels.newInputStream(channel)) { + return IOUtils.toByteArray(inputStream); + } catch (IOException e) { + throw new RuntimeException("Error when reading: " + fileDescriptorPath, e); + } + } + + /** + * Retrieves a ReadableByteChannel for a file specified by its path. + * + * @param filePath The path to the file to obtain a ReadableByteChannel for. + * @return ReadableByteChannel for the specified file. + * @throws RuntimeException if an error occurs while finding or opening the file. + */ + private static ReadableByteChannel getFileByteChannel(String filePath) { + try { + MatchResult result = FileSystems.match(filePath); + checkArgument( + result.status() == MatchResult.Status.OK && !result.metadata().isEmpty(), + "Failed to match any files with the pattern: " + filePath); + + List<ResourceId> rId = + result.metadata().stream().map(MatchResult.Metadata::resourceId).collect(toList()); + + checkArgument(rId.size() == 1, "Expected exactly 1 file, but got " + rId.size() + " files."); + return FileSystems.open(rId.get(0)); + } catch (IOException e) { + throw new RuntimeException("Error when finding: " + filePath, e); + } + } + + /** + * Represents metadata associated with a Protocol Buffer schema, including the File Name and + * ProtoDomain. + */ + static class ProtoSchemaInfo implements Serializable { + private String fileName; + private ProtoDomain protoDomain; + + /** + * Constructs a ProtoSchemaInfo with the specified File Name and ProtoDomain. + * + * @param fileName The name of the associated Protocol Buffer file. + * @param protoDomain The ProtoDomain containing schema information. + */ + public ProtoSchemaInfo(String fileName, ProtoDomain protoDomain) { + this.fileName = fileName; + this.protoDomain = protoDomain; + } + + /** + * Sets the ProtoDomain associated with this ProtoSchemaInfo. + * + * @param protoDomain The ProtoDomain to set. + */ + @SuppressWarnings("unused") + public void setProtoDomain(ProtoDomain protoDomain) { + this.protoDomain = protoDomain; + } + + /** + * Gets the ProtoDomain associated with this ProtoSchemaInfo. + * + * @return The ProtoDomain containing schema information. + */ + public ProtoDomain getProtoDomain() { + return protoDomain; + } + + /** + * Gets the File Name associated with this ProtoSchemaInfo. + * + * @return The name of the associated Protocol Buffer file. + */ + public String getFileName() { + return fileName; + } + + /** + * Sets the File Name associated with this ProtoSchemaInfo. + * + * @param fileName The name of the Protocol Buffer file to set. + */ + public void setFileName(String fileName) { + this.fileName = fileName; + } + } +} diff --git a/sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/ProtoByteUtilsTest.java b/sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/ProtoByteUtilsTest.java new file mode 100644 index 000000000000..2a4cb4b5d5fb --- /dev/null +++ b/sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/ProtoByteUtilsTest.java @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.extensions.protobuf; + +import java.util.Objects; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.values.Row; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class ProtoByteUtilsTest { + + private static final String DESCRIPTOR_PATH = + Objects.requireNonNull( + ProtoByteUtilsTest.class.getResource( + "/proto_byte/file_descriptor/proto_byte_utils.pb")) + .getPath(); + + private static final String MESSAGE_NAME = "MyMessage"; + + private static final Schema SCHEMA = + Schema.builder() + .addField("id", Schema.FieldType.INT32) + .addField("name", Schema.FieldType.STRING) + .addField("active", Schema.FieldType.BOOLEAN) + .addField( + "address", + Schema.FieldType.row( + Schema.builder() + .addField("city", Schema.FieldType.STRING) + .addField("street", Schema.FieldType.STRING) + .addField("state", Schema.FieldType.STRING) + .addField("zip_code", Schema.FieldType.STRING) + .build())) + .build(); + + @Test + public void testProtoSchemaToBeamSchema() { + Schema schema = ProtoByteUtils.getBeamSchemaFromProto(DESCRIPTOR_PATH, MESSAGE_NAME); + Assert.assertEquals(schema.getFieldNames(), SCHEMA.getFieldNames()); + } + + @Test + public void testProtoBytesToRowFunctionGenerateSerializableFunction() { + SerializableFunction<byte[], Row> protoBytesToRowFunction = + ProtoByteUtils.getProtoBytesToRowFunction(DESCRIPTOR_PATH, MESSAGE_NAME); + Assert.assertNotNull(protoBytesToRowFunction); + } + + @Test(expected = java.lang.RuntimeException.class) + public void testProtoBytesToRowFunctionReturnsRowFailure() { + // Create a proto bytes to row function + SerializableFunction<byte[], Row> protoBytesToRowFunction = + ProtoByteUtils.getProtoBytesToRowFunction(DESCRIPTOR_PATH, MESSAGE_NAME); + + // Create some test input bytes that are not matching + byte[] inputBytes = new byte[] {1, 2, 3, 4, 5}; + + // Call the proto bytes to row function that should fail because the input does not match + protoBytesToRowFunction.apply(inputBytes); + } + + @Test + public void testRowToProtoFunction() { + Row row = + Row.withSchema(SCHEMA) + .withFieldValue("id", 1234) + .withFieldValue("name", "Doe") + .withFieldValue("active", false) + .withFieldValue("address.city", "seattle") + .withFieldValue("address.street", "fake street") + .withFieldValue("address.zip_code", "TO-1234") + .withFieldValue("address.state", "wa") + .build(); + + Assert.assertNotNull( + ProtoByteUtils.getRowToProtoBytes(DESCRIPTOR_PATH, MESSAGE_NAME).apply(row)); + } +} diff --git a/sdks/java/extensions/protobuf/src/test/resources/README.md b/sdks/java/extensions/protobuf/src/test/resources/README.md index 79083f5142b0..de9cb742788b 100644 --- a/sdks/java/extensions/protobuf/src/test/resources/README.md +++ b/sdks/java/extensions/protobuf/src/test/resources/README.md @@ -32,3 +32,9 @@ protoc \ --include_imports \ sdks/java/extensions/protobuf/src/test/resources/test/option/v1/simple.proto ``` +```bash +protoc \ + -Isdks/java/extensions/protobuf/src/test/resources/ \ + --descriptor_set_out=sdks/java/extensions/protobuf/src/test/resources/proto_byte/file_descriptor/proto_byte_utils.pb \ + sdks/java/extensions/protobuf/src/test/resources/proto_byte/proto_byte_utils.proto +``` diff --git a/sdks/java/extensions/protobuf/src/test/resources/proto_byte/file_descriptor/proto_byte_utils.pb b/sdks/java/extensions/protobuf/src/test/resources/proto_byte/file_descriptor/proto_byte_utils.pb new file mode 100644 index 000000000000..67e93cc177cc --- /dev/null +++ b/sdks/java/extensions/protobuf/src/test/resources/proto_byte/file_descriptor/proto_byte_utils.pb @@ -0,0 +1,13 @@ + +� +test_proto.proto"� + MyMessage +id (Rid +name ( Rname +active (Ractive, +address ( 2.MyMessage.AddressRaddressf +Address +street ( Rstreet +city ( Rcity +state ( Rstate +zip_code ( RzipCodebproto3 \ No newline at end of file diff --git a/sdks/java/extensions/protobuf/src/test/resources/proto_byte/proto_byte_utils.proto b/sdks/java/extensions/protobuf/src/test/resources/proto_byte/proto_byte_utils.proto new file mode 100644 index 000000000000..aead141f4b9a --- /dev/null +++ b/sdks/java/extensions/protobuf/src/test/resources/proto_byte/proto_byte_utils.proto @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +syntax = "proto3"; + +message MyMessage { + int32 id = 1; + string name = 2; + bool active = 3; + + // Nested field + message Address { + string street = 1; + string city = 2; + string state = 3; + string zip_code = 4; + } + + Address address = 4; +} diff --git a/sdks/java/io/kafka/README.md b/sdks/java/io/kafka/README.md index 4ecf095bec5b..b137e0b240a9 100644 --- a/sdks/java/io/kafka/README.md +++ b/sdks/java/io/kafka/README.md @@ -47,3 +47,13 @@ complete list. The documentation is maintained in JavaDoc for KafkaIO class. It includes usage examples and primary concepts. - [KafkaIO.java](src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java) + +### Protobuf tests +This recreates the proto descriptor set included in this resource directory. + +```bash +protoc \ + -Isdks/java/io/kafka/src/test/resources/ \ + --descriptor_set_out=sdks/java/io/kafka/src/test/resources/proto_byte/file_descriptor/proto_byte_utils.pb \ + sdks/java/io/kafka/src/test/resources/proto_byte/proto_byte_utils.proto +``` \ No newline at end of file diff --git a/sdks/java/io/kafka/build.gradle b/sdks/java/io/kafka/build.gradle index 61209aa50928..dc190ef9d8fd 100644 --- a/sdks/java/io/kafka/build.gradle +++ b/sdks/java/io/kafka/build.gradle @@ -51,6 +51,7 @@ dependencies { permitUnusedDeclared library.java.jackson_dataformat_csv implementation project(path: ":sdks:java:core", configuration: "shadow") implementation project(":sdks:java:extensions:avro") + implementation project(":sdks:java:extensions:protobuf") implementation project(":runners:core-construction-java") implementation project(":sdks:java:expansion-service") permitUnusedDeclared project(":sdks:java:expansion-service") // BEAM-11761 diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaReadSchemaTransformConfiguration.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaReadSchemaTransformConfiguration.java index ae03c49b9b04..2fa365b1c7f3 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaReadSchemaTransformConfiguration.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaReadSchemaTransformConfiguration.java @@ -40,7 +40,7 @@ public abstract class KafkaReadSchemaTransformConfiguration { public static final Set<String> VALID_START_OFFSET_VALUES = Sets.newHashSet("earliest", "latest"); - public static final String VALID_FORMATS_STR = "RAW,AVRO,JSON"; + public static final String VALID_FORMATS_STR = "RAW,AVRO,JSON,PROTO"; public static final Set<String> VALID_DATA_FORMATS = Sets.newHashSet(VALID_FORMATS_STR.split(",")); @@ -87,6 +87,18 @@ public static Builder builder() { @Nullable public abstract String getSchema(); + @SchemaFieldDescription( + "The path to the Protocol Buffer File Descriptor Set file. This file is used for schema" + + " definition and message serialization.") + @Nullable + public abstract String getFileDescriptorPath(); + + @SchemaFieldDescription( + "The name of the Protocol Buffer message to be used for schema" + + " extraction and data conversion.") + @Nullable + public abstract String getMessageName(); + @SchemaFieldDescription( "What to do when there is no initial offset in Kafka or if the current offset" + " does not exist any more on the server. (1) earliest: automatically reset the offset to the earliest" @@ -123,6 +135,10 @@ public abstract static class Builder { public abstract Builder setSchema(String schema); + public abstract Builder setFileDescriptorPath(String fileDescriptorPath); + + public abstract Builder setMessageName(String messageName); + public abstract Builder setFormat(String format); public abstract Builder setAutoOffsetResetConfig(String startOffset); diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaReadSchemaTransformProvider.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaReadSchemaTransformProvider.java index 2f5278cd7129..996976ee9a75 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaReadSchemaTransformProvider.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaReadSchemaTransformProvider.java @@ -17,6 +17,8 @@ */ package org.apache.beam.sdk.io.kafka; +import static org.apache.beam.sdk.io.kafka.KafkaReadSchemaTransformConfiguration.VALID_DATA_FORMATS; + import com.google.auto.service.AutoService; import java.io.FileOutputStream; import java.io.IOException; @@ -35,6 +37,7 @@ import org.apache.avro.generic.GenericRecord; import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; import org.apache.beam.sdk.extensions.avro.schemas.utils.AvroUtils; +import org.apache.beam.sdk.extensions.protobuf.ProtoByteUtils; import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.metrics.Counter; import org.apache.beam.sdk.metrics.Metrics; @@ -95,9 +98,6 @@ protected Class<KafkaReadSchemaTransformConfiguration> configurationClass() { return KafkaReadSchemaTransformConfiguration.class; } - @SuppressWarnings({ - "nullness" // TODO(https://github.com/apache/beam/issues/20497) - }) @Override protected SchemaTransform from(KafkaReadSchemaTransformConfiguration configuration) { final String inputSchema = configuration.getSchema(); @@ -115,8 +115,14 @@ protected SchemaTransform from(KafkaReadSchemaTransformConfiguration configurati String format = configuration.getFormat(); boolean handleErrors = ErrorHandling.hasOutput(configuration.getErrorHandling()); - - if ((format != null && format.equals("RAW")) || (!Strings.isNullOrEmpty(inputSchema))) { + String descriptorPath = configuration.getFileDescriptorPath(); + String messageName = configuration.getMessageName(); + + if ((format != null && VALID_DATA_FORMATS.contains(format)) + || (!Strings.isNullOrEmpty(inputSchema) && !Objects.equals(format, "RAW")) + || (Objects.equals(format, "PROTO") + && !Strings.isNullOrEmpty(descriptorPath) + && !Strings.isNullOrEmpty(messageName))) { SerializableFunction<byte[], Row> valueMapper; Schema beamSchema; if (format != null && format.equals("RAW")) { @@ -126,11 +132,21 @@ protected SchemaTransform from(KafkaReadSchemaTransformConfiguration configurati } beamSchema = Schema.builder().addField("payload", Schema.FieldType.BYTES).build(); valueMapper = getRawBytesToRowFunction(beamSchema); + } else if (format != null && format.equals("PROTO")) { + if (descriptorPath == null || messageName == null) { + throw new IllegalArgumentException( + "Expecting both descriptorPath and messageName to be non-null."); + } + valueMapper = ProtoByteUtils.getProtoBytesToRowFunction(descriptorPath, messageName); + beamSchema = ProtoByteUtils.getBeamSchemaFromProto(descriptorPath, messageName); } else { assert Strings.isNullOrEmpty(configuration.getConfluentSchemaRegistryUrl()) : "To read from Kafka, a schema must be provided directly or though Confluent " + "Schema Registry, but not both."; - + if (inputSchema == null) { + throw new IllegalArgumentException( + "To read from Kafka in JSON or AVRO format, you must provide a schema."); + } beamSchema = Objects.equals(format, "JSON") ? JsonUtils.beamSchemaFromJsonSchema(inputSchema) @@ -170,7 +186,11 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) { PCollection<Row> errorOutput = outputTuple.get(ERROR_TAG).setRowSchema(errorSchema); if (handleErrors) { - outputRows = outputRows.and(configuration.getErrorHandling().getOutput(), errorOutput); + ErrorHandling errorHandling = configuration.getErrorHandling(); + if (errorHandling == null) { + throw new IllegalArgumentException("You must specify an error handling option."); + } + outputRows = outputRows.and(errorHandling.getOutput(), errorOutput); } return outputRows; } diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaWriteSchemaTransformProvider.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaWriteSchemaTransformProvider.java index 93d6d73493d2..694c3e9f2c14 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaWriteSchemaTransformProvider.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaWriteSchemaTransformProvider.java @@ -27,6 +27,7 @@ import java.util.Set; import javax.annotation.Nullable; import org.apache.beam.sdk.extensions.avro.schemas.utils.AvroUtils; +import org.apache.beam.sdk.extensions.protobuf.ProtoByteUtils; import org.apache.beam.sdk.metrics.Counter; import org.apache.beam.sdk.metrics.Metrics; import org.apache.beam.sdk.schemas.AutoValueSchema; @@ -36,14 +37,13 @@ import org.apache.beam.sdk.schemas.transforms.SchemaTransform; import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider; import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider; -import org.apache.beam.sdk.schemas.transforms.providers.ErrorHandling; import org.apache.beam.sdk.schemas.utils.JsonUtils; import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.DoFn.ProcessElement; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.SimpleFunction; import org.apache.beam.sdk.values.KV; -import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionRowTuple; import org.apache.beam.sdk.values.PCollectionTuple; import org.apache.beam.sdk.values.Row; @@ -62,12 +62,14 @@ public class KafkaWriteSchemaTransformProvider extends TypedSchemaTransformProvider< KafkaWriteSchemaTransformProvider.KafkaWriteSchemaTransformConfiguration> { - public static final String SUPPORTED_FORMATS_STR = "RAW,JSON,AVRO"; + public static final String SUPPORTED_FORMATS_STR = "RAW,JSON,AVRO,PROTO"; public static final Set<String> SUPPORTED_FORMATS = Sets.newHashSet(SUPPORTED_FORMATS_STR.split(",")); public static final TupleTag<Row> ERROR_TAG = new TupleTag<Row>() {}; public static final TupleTag<KV<byte[], byte[]>> OUTPUT_TAG = new TupleTag<KV<byte[], byte[]>>() {}; + public static final Schema ERROR_SCHEMA = + Schema.builder().addStringField("error").addNullableByteArrayField("row").build(); private static final Logger LOG = LoggerFactory.getLogger(KafkaWriteSchemaTransformProvider.class); @@ -99,38 +101,25 @@ static final class KafkaWriteSchemaTransform extends SchemaTransform implements } public static class ErrorCounterFn extends DoFn<Row, KV<byte[], byte[]>> { - private final SerializableFunction<Row, byte[]> toBytesFn; - private final Counter errorCounter; + private SerializableFunction<Row, byte[]> toBytesFn; + private Counter errorCounter; private Long errorsInBundle = 0L; - private final boolean handleErrors; - private final Schema errorSchema; - - public ErrorCounterFn( - String name, - SerializableFunction<Row, byte[]> toBytesFn, - Schema errorSchema, - boolean handleErrors) { + + public ErrorCounterFn(String name, SerializableFunction<Row, byte[]> toBytesFn) { this.toBytesFn = toBytesFn; - this.errorCounter = Metrics.counter(KafkaWriteSchemaTransformProvider.class, name); - this.handleErrors = handleErrors; - this.errorSchema = errorSchema; + errorCounter = Metrics.counter(KafkaWriteSchemaTransformProvider.class, name); } @ProcessElement public void process(@DoFn.Element Row row, MultiOutputReceiver receiver) { - KV<byte[], byte[]> output = null; try { - output = KV.of(new byte[1], toBytesFn.apply(row)); + receiver.get(OUTPUT_TAG).output(KV.of(new byte[1], toBytesFn.apply(row))); } catch (Exception e) { - if (!handleErrors) { - throw new RuntimeException(e); - } errorsInBundle += 1; LOG.warn("Error while processing the element", e); - receiver.get(ERROR_TAG).output(ErrorHandling.errorRecord(errorSchema, row, e)); - } - if (output != null) { - receiver.get(OUTPUT_TAG).output(output); + receiver + .get(ERROR_TAG) + .output(Row.withSchema(ERROR_SCHEMA).addValues(e.toString(), row.toString()).build()); } } @@ -141,9 +130,6 @@ public void finish() { } } - @SuppressWarnings({ - "nullness" // TODO(https://github.com/apache/beam/issues/20497) - }) @Override public PCollectionRowTuple expand(PCollectionRowTuple input) { Schema inputSchema = input.get("input").getSchema(); @@ -153,24 +139,32 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) { if (numFields != 1) { throw new IllegalArgumentException("Expecting exactly one field, found " + numFields); } + if (inputSchema.getField(0).getType().equals(Schema.FieldType.BYTES)) { + throw new IllegalArgumentException( + "The input schema must have exactly one field of type byte."); + } toBytesFn = getRowToRawBytesFunction(inputSchema.getField(0).getName()); } else if (configuration.getFormat().equals("JSON")) { toBytesFn = JsonUtils.getRowToJsonBytesFunction(inputSchema); + } else if (configuration.getFormat().equals("PROTO")) { + String descriptorPath = configuration.getFileDescriptorPath(); + String messageName = configuration.getMessageName(); + if (descriptorPath == null || messageName == null) { + throw new IllegalArgumentException( + "Expecting both descriptorPath and messageName to be non-null."); + } + toBytesFn = ProtoByteUtils.getRowToProtoBytes(descriptorPath, messageName); } else { toBytesFn = AvroUtils.getRowToAvroBytesFunction(inputSchema); } - boolean handleErrors = ErrorHandling.hasOutput(configuration.getErrorHandling()); final Map<String, String> configOverrides = configuration.getProducerConfigUpdates(); - Schema errorSchema = ErrorHandling.errorSchema(inputSchema); PCollectionTuple outputTuple = input .get("input") .apply( "Map rows to Kafka messages", - ParDo.of( - new ErrorCounterFn( - "Kafka-write-error-counter", toBytesFn, errorSchema, handleErrors)) + ParDo.of(new ErrorCounterFn("Kafka-write-error-counter", toBytesFn)) .withOutputTags(OUTPUT_TAG, TupleTagList.of(ERROR_TAG))); outputTuple @@ -186,11 +180,8 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) { .withKeySerializer(ByteArraySerializer.class) .withValueSerializer(ByteArraySerializer.class)); - // TODO: include output from KafkaIO Write once updated from PDone - PCollection<Row> errorOutput = - outputTuple.get(ERROR_TAG).setRowSchema(ErrorHandling.errorSchema(errorSchema)); return PCollectionRowTuple.of( - handleErrors ? configuration.getErrorHandling().getOutput() : "errors", errorOutput); + "errors", outputTuple.get(ERROR_TAG).setRowSchema(ERROR_SCHEMA)); } } @@ -241,6 +232,18 @@ public abstract static class KafkaWriteSchemaTransformConfiguration implements S + " of servers. | Format: host1:port1,host2:port2,...") public abstract String getBootstrapServers(); + @SchemaFieldDescription( + "The path to the Protocol Buffer File Descriptor Set file. This file is used for schema" + + " definition and message serialization.") + @Nullable + public abstract String getFileDescriptorPath(); + + @SchemaFieldDescription( + "The name of the Protocol Buffer message to be used for schema" + + " extraction and data conversion.") + @Nullable + public abstract String getMessageName(); + @SchemaFieldDescription( "A list of key-value pairs that act as configuration parameters for Kafka producers." + " Most of these configurations will not be needed, but if you need to customize your Kafka producer," @@ -249,10 +252,6 @@ public abstract static class KafkaWriteSchemaTransformConfiguration implements S @Nullable public abstract Map<String, String> getProducerConfigUpdates(); - @SchemaFieldDescription("This option specifies whether and where to output unwritable rows.") - @Nullable - public abstract ErrorHandling getErrorHandling(); - public static Builder builder() { return new AutoValue_KafkaWriteSchemaTransformProvider_KafkaWriteSchemaTransformConfiguration .Builder(); @@ -266,9 +265,11 @@ public abstract static class Builder { public abstract Builder setBootstrapServers(String bootstrapServers); - public abstract Builder setProducerConfigUpdates(Map<String, String> producerConfigUpdates); + public abstract Builder setFileDescriptorPath(String fileDescriptorPath); + + public abstract Builder setMessageName(String messageName); - public abstract Builder setErrorHandling(ErrorHandling errorHandling); + public abstract Builder setProducerConfigUpdates(Map<String, String> producerConfigUpdates); public abstract KafkaWriteSchemaTransformConfiguration build(); } diff --git a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaReadSchemaTransformProviderTest.java b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaReadSchemaTransformProviderTest.java index 1570a33c2580..27fa18715c32 100644 --- a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaReadSchemaTransformProviderTest.java +++ b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaReadSchemaTransformProviderTest.java @@ -101,7 +101,9 @@ public void testFindTransformAndMakeItWork() { "format", "confluentSchemaRegistrySubject", "confluentSchemaRegistryUrl", - "errorHandling"), + "errorHandling", + "fileDescriptorPath", + "messageName"), kafkaProvider.configurationSchema().getFields().stream() .map(field -> field.getName()) .collect(Collectors.toSet())); @@ -150,7 +152,7 @@ public void testBuildTransformWithJsonSchema() throws IOException { } @Test - public void testBuildTransformWithRawFormat() throws IOException { + public void testBuildTransformWithRawFormat() { ServiceLoader<SchemaTransformProvider> serviceLoader = ServiceLoader.load(SchemaTransformProvider.class); List<SchemaTransformProvider> providers = @@ -166,4 +168,56 @@ public void testBuildTransformWithRawFormat() throws IOException { .setFormat("RAW") .build()); } + + @Test + public void testBuildTransformWithProtoFormat() { + ServiceLoader<SchemaTransformProvider> serviceLoader = + ServiceLoader.load(SchemaTransformProvider.class); + List<SchemaTransformProvider> providers = + StreamSupport.stream(serviceLoader.spliterator(), false) + .filter(provider -> provider.getClass() == KafkaReadSchemaTransformProvider.class) + .collect(Collectors.toList()); + KafkaReadSchemaTransformProvider kafkaProvider = + (KafkaReadSchemaTransformProvider) providers.get(0); + + kafkaProvider.from( + KafkaReadSchemaTransformConfiguration.builder() + .setTopic("anytopic") + .setBootstrapServers("anybootstrap") + .setFormat("PROTO") + .setMessageName("MyMessage") + .setFileDescriptorPath( + Objects.requireNonNull( + getClass().getResource("/proto_byte/file_descriptor/proto_byte_utils.pb")) + .getPath()) + .build()); + } + + @Test + public void testBuildTransformWithProtoFormatWrongMessageName() { + ServiceLoader<SchemaTransformProvider> serviceLoader = + ServiceLoader.load(SchemaTransformProvider.class); + List<SchemaTransformProvider> providers = + StreamSupport.stream(serviceLoader.spliterator(), false) + .filter(provider -> provider.getClass() == KafkaReadSchemaTransformProvider.class) + .collect(Collectors.toList()); + KafkaReadSchemaTransformProvider kafkaProvider = + (KafkaReadSchemaTransformProvider) providers.get(0); + + assertThrows( + NullPointerException.class, + () -> + kafkaProvider.from( + KafkaReadSchemaTransformConfiguration.builder() + .setTopic("anytopic") + .setBootstrapServers("anybootstrap") + .setFormat("PROTO") + .setMessageName("MyOtherMessage") + .setFileDescriptorPath( + Objects.requireNonNull( + getClass() + .getResource("/proto_byte/file_descriptor/proto_byte_utils.pb")) + .getPath()) + .build())); + } } diff --git a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaWriteSchemaTransformProviderTest.java b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaWriteSchemaTransformProviderTest.java index b8649727f36d..20f474790cc7 100644 --- a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaWriteSchemaTransformProviderTest.java +++ b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaWriteSchemaTransformProviderTest.java @@ -21,10 +21,12 @@ import java.io.UnsupportedEncodingException; import java.util.Arrays; +import java.util.Collections; import java.util.List; +import java.util.Objects; +import org.apache.beam.sdk.extensions.protobuf.ProtoByteUtils; import org.apache.beam.sdk.io.kafka.KafkaWriteSchemaTransformProvider.KafkaWriteSchemaTransform.ErrorCounterFn; import org.apache.beam.sdk.schemas.Schema; -import org.apache.beam.sdk.schemas.transforms.providers.ErrorHandling; import org.apache.beam.sdk.schemas.utils.JsonUtils; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; @@ -52,8 +54,37 @@ public class KafkaWriteSchemaTransformProviderTest { private static final Schema BEAMSCHEMA = Schema.of(Schema.Field.of("name", Schema.FieldType.STRING)); - private static final Schema BEAMRAWSCHEMA = + private static final Schema BEAM_RAW_SCHEMA = Schema.of(Schema.Field.of("payload", Schema.FieldType.BYTES)); + private static final Schema ERRORSCHEMA = KafkaWriteSchemaTransformProvider.ERROR_SCHEMA; + + private static final Schema BEAM_PROTO_SCHEMA = + Schema.builder() + .addField("id", Schema.FieldType.INT32) + .addField("name", Schema.FieldType.STRING) + .addField("active", Schema.FieldType.BOOLEAN) + .addField( + "address", + Schema.FieldType.row( + Schema.builder() + .addField("city", Schema.FieldType.STRING) + .addField("street", Schema.FieldType.STRING) + .addField("state", Schema.FieldType.STRING) + .addField("zip_code", Schema.FieldType.STRING) + .build())) + .build(); + + private static final List<Row> PROTO_ROWS = + Collections.singletonList( + Row.withSchema(BEAM_PROTO_SCHEMA) + .withFieldValue("id", 1234) + .withFieldValue("name", "Doe") + .withFieldValue("active", false) + .withFieldValue("address.city", "seattle") + .withFieldValue("address.street", "fake street") + .withFieldValue("address.zip_code", "TO-1234") + .withFieldValue("address.state", "wa") + .build()); private static final List<Row> ROWS = Arrays.asList( @@ -67,9 +98,13 @@ public class KafkaWriteSchemaTransformProviderTest { try { RAW_ROWS = Arrays.asList( - Row.withSchema(BEAMRAWSCHEMA).withFieldValue("payload", "a".getBytes("UTF8")).build(), - Row.withSchema(BEAMRAWSCHEMA).withFieldValue("payload", "b".getBytes("UTF8")).build(), - Row.withSchema(BEAMRAWSCHEMA) + Row.withSchema(BEAM_RAW_SCHEMA) + .withFieldValue("payload", "a".getBytes("UTF8")) + .build(), + Row.withSchema(BEAM_RAW_SCHEMA) + .withFieldValue("payload", "b".getBytes("UTF8")) + .build(), + Row.withSchema(BEAM_RAW_SCHEMA) .withFieldValue("payload", "c".getBytes("UTF8")) .build()); } catch (UnsupportedEncodingException e) { @@ -82,6 +117,13 @@ public class KafkaWriteSchemaTransformProviderTest { final SerializableFunction<Row, byte[]> valueRawMapper = getRowToRawBytesFunction("payload"); + final SerializableFunction<Row, byte[]> protoValueRawMapper = + ProtoByteUtils.getRowToProtoBytes( + Objects.requireNonNull( + getClass().getResource("/proto_byte/file_descriptor/proto_byte_utils.pb")) + .getPath(), + "MyMessage"); + @Rule public transient TestPipeline p = TestPipeline.create(); @Test @@ -93,14 +135,12 @@ public void testKafkaErrorFnSuccess() throws Exception { KV.of(new byte[1], "{\"name\":\"c\"}".getBytes("UTF8"))); PCollection<Row> input = p.apply(Create.of(ROWS)); - Schema errorSchema = ErrorHandling.errorSchema(BEAMSCHEMA); PCollectionTuple output = input.apply( - ParDo.of( - new ErrorCounterFn("Kafka-write-error-counter", valueMapper, errorSchema, true)) + ParDo.of(new ErrorCounterFn("Kafka-write-error-counter", valueMapper)) .withOutputTags(OUTPUT_TAG, TupleTagList.of(ERROR_TAG))); - output.get(ERROR_TAG).setRowSchema(errorSchema); + output.get(ERROR_TAG).setRowSchema(ERRORSCHEMA); PAssert.that(output.get(OUTPUT_TAG)).containsInAnyOrder(msg); p.run().waitUntilFinish(); @@ -115,17 +155,26 @@ public void testKafkaErrorFnRawSuccess() throws Exception { KV.of(new byte[1], "c".getBytes("UTF8"))); PCollection<Row> input = p.apply(Create.of(RAW_ROWS)); - Schema errorSchema = ErrorHandling.errorSchema(BEAMRAWSCHEMA); PCollectionTuple output = input.apply( - ParDo.of( - new ErrorCounterFn( - "Kafka-write-error-counter", valueRawMapper, errorSchema, true)) + ParDo.of(new ErrorCounterFn("Kafka-write-error-counter", valueRawMapper)) .withOutputTags(OUTPUT_TAG, TupleTagList.of(ERROR_TAG))); - output.get(ERROR_TAG).setRowSchema(errorSchema); + output.get(ERROR_TAG).setRowSchema(ERRORSCHEMA); PAssert.that(output.get(OUTPUT_TAG)).containsInAnyOrder(msg); p.run().waitUntilFinish(); } + + @Test + public void testKafkaErrorFnProtoSuccess() { + PCollection<Row> input = p.apply(Create.of(PROTO_ROWS)); + PCollectionTuple output = + input.apply( + ParDo.of(new ErrorCounterFn("Kafka-write-error-counter", protoValueRawMapper)) + .withOutputTags(OUTPUT_TAG, TupleTagList.of(ERROR_TAG))); + + PAssert.that(output.get(ERROR_TAG).setRowSchema(ERRORSCHEMA)).empty(); + p.run().waitUntilFinish(); + } } diff --git a/sdks/java/io/kafka/src/test/resources/proto_byte/file_descriptor/proto_byte_utils.pb b/sdks/java/io/kafka/src/test/resources/proto_byte/file_descriptor/proto_byte_utils.pb new file mode 100644 index 000000000000..67e93cc177cc --- /dev/null +++ b/sdks/java/io/kafka/src/test/resources/proto_byte/file_descriptor/proto_byte_utils.pb @@ -0,0 +1,13 @@ + +� +test_proto.proto"� + MyMessage +id (Rid +name ( Rname +active (Ractive, +address ( 2.MyMessage.AddressRaddressf +Address +street ( Rstreet +city ( Rcity +state ( Rstate +zip_code ( RzipCodebproto3 \ No newline at end of file diff --git a/sdks/java/io/kafka/src/test/resources/proto_byte/proto_byte_utils.proto b/sdks/java/io/kafka/src/test/resources/proto_byte/proto_byte_utils.proto new file mode 100644 index 000000000000..aead141f4b9a --- /dev/null +++ b/sdks/java/io/kafka/src/test/resources/proto_byte/proto_byte_utils.proto @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +syntax = "proto3"; + +message MyMessage { + int32 id = 1; + string name = 2; + bool active = 3; + + // Nested field + message Address { + string street = 1; + string city = 2; + string state = 3; + string zip_code = 4; + } + + Address address = 4; +} diff --git a/sdks/python/apache_beam/yaml/standard_io.yaml b/sdks/python/apache_beam/yaml/standard_io.yaml index 116d200a86a7..b19c1e5b063e 100644 --- a/sdks/python/apache_beam/yaml/standard_io.yaml +++ b/sdks/python/apache_beam/yaml/standard_io.yaml @@ -62,12 +62,16 @@ 'confluent_schema_registry_subject': 'confluentSchemaRegistrySubject' 'auto_offset_reset_config': 'autoOffsetResetConfig' 'error_handling': 'errorHandling' + 'file_descriptor_path': 'fileDescriptorPath' + 'message_name': 'messageName' 'WriteToKafka': 'format': 'format' 'topic': 'topic' 'bootstrap_servers': 'bootstrapServers' 'producer_config_updates': 'ProducerConfigUpdates' 'error_handling': 'errorHandling' + 'file_descriptor_path': 'fileDescriptorPath' + 'message_name': 'messageName' underlying_provider: type: beamJar transforms: From 8fa5b19f5468c716e16c688994b538cc2abd3811 Mon Sep 17 00:00:00 2001 From: Bruno Volpato <bvolpato@google.com> Date: Mon, 13 Nov 2023 01:21:34 -0500 Subject: [PATCH 429/435] Add getters to CheckResult in IT framework (#29411) --- .../org/apache/beam/it/conditions/ConditionCheck.java | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/it/conditions/src/main/java/org/apache/beam/it/conditions/ConditionCheck.java b/it/conditions/src/main/java/org/apache/beam/it/conditions/ConditionCheck.java index b562b4a068c8..de1b2b0cf2b7 100644 --- a/it/conditions/src/main/java/org/apache/beam/it/conditions/ConditionCheck.java +++ b/it/conditions/src/main/java/org/apache/beam/it/conditions/ConditionCheck.java @@ -77,6 +77,14 @@ public CheckResult(boolean success, String message) { this.message = message; } + public boolean isSuccess() { + return success; + } + + public String getMessage() { + return message; + } + @Override public String toString() { return "CheckResult{" + "success=" + success + ", message='" + message + '\'' + '}'; From 570f41ffca949d33a7a231457020e85e39d8eb8a Mon Sep 17 00:00:00 2001 From: Alexey Romanenko <aromanenko.dev@gmail.com> Date: Tue, 7 Nov 2023 15:40:55 +0100 Subject: [PATCH 430/435] TPC-DS query processing should fail if output is empty --- .../beam/sdk/tpcds/SqlTransformRunner.java | 18 ++++++++++ .../org/apache/beam/sdk/tpcds/TpcdsRun.java | 36 +++++++++++++++++++ 2 files changed, 54 insertions(+) diff --git a/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/SqlTransformRunner.java b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/SqlTransformRunner.java index 6570b7fe81b2..1550a25b7c8f 100644 --- a/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/SqlTransformRunner.java +++ b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/SqlTransformRunner.java @@ -39,12 +39,16 @@ import org.apache.beam.sdk.extensions.sql.meta.provider.text.TextTable; import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.io.parquet.ParquetIO; +import org.apache.beam.sdk.metrics.Counter; +import org.apache.beam.sdk.metrics.Metrics; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.SchemaCoder; import org.apache.beam.sdk.testutils.publishing.InfluxDBPublisher; import org.apache.beam.sdk.testutils.publishing.InfluxDBSettings; +import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionTuple; import org.apache.beam.sdk.values.Row; @@ -81,6 +85,9 @@ public class SqlTransformRunner { private static final Logger LOG = LoggerFactory.getLogger(SqlTransformRunner.class); + static final String METRICS_NAMESPACE = "tpcds"; + static final String OUTPUT_COUNTER = "output_rows"; + /** This class is used to extract all SQL query identifiers. */ static class SqlIdentifierVisitor extends SqlBasicVisitor<Void> { private final Set<String> identifiers = new HashSet<>(); @@ -305,6 +312,7 @@ public static void runUsingSqlTransform(String[] args) throws Exception { tables .apply(SqlTransform.query(queryString)) .apply(MapElements.into(TypeDescriptors.strings()).via(Row::toString)) + .apply(ParDo.of(new CounterDoFn())) .apply( TextIO.write() .to( @@ -396,4 +404,14 @@ private static InfluxDBSettings getInfluxSettings(final TpcdsOptions options) { .withRetentionPolicy(options.getInfluxRetentionPolicy()) .get(); } + + private static class CounterDoFn extends DoFn<String, String> { + private final Counter counter = Metrics.counter(METRICS_NAMESPACE, OUTPUT_COUNTER); + + @ProcessElement + public void processElement(ProcessContext context) { + counter.inc(); + context.output(context.element()); + } + } } diff --git a/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/TpcdsRun.java b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/TpcdsRun.java index b6235db1c123..700de369b609 100644 --- a/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/TpcdsRun.java +++ b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/TpcdsRun.java @@ -17,10 +17,17 @@ */ package org.apache.beam.sdk.tpcds; +import static org.apache.beam.sdk.tpcds.SqlTransformRunner.METRICS_NAMESPACE; +import static org.apache.beam.sdk.tpcds.SqlTransformRunner.OUTPUT_COUNTER; + import java.util.concurrent.Callable; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.PipelineResult.State; +import org.apache.beam.sdk.metrics.MetricNameFilter; +import org.apache.beam.sdk.metrics.MetricQueryResults; +import org.apache.beam.sdk.metrics.MetricResult; +import org.apache.beam.sdk.metrics.MetricsFilter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -48,6 +55,35 @@ public TpcdsRunResult call() { // Make sure to set the job status to be successful only when pipelineResult's final state is // DONE. boolean isSuccessful = state == State.DONE; + + // Check a number of output records - it MUST be greater than 0. + if (isSuccessful) { + long outputRecords = 0; + MetricQueryResults metrics = + pipelineResult + .metrics() + .queryMetrics( + MetricsFilter.builder() + .addNameFilter(MetricNameFilter.named(METRICS_NAMESPACE, OUTPUT_COUNTER)) + .build()); + if (metrics.getCounters().iterator().hasNext()) { + // Despite it's iterable, it should contain only one entry + MetricResult<Long> metricResult = metrics.getCounters().iterator().next(); + if (metricResult.getAttempted() != null && metricResult.getAttempted() > 0) { + outputRecords = metricResult.getAttempted(); + } + } + + // It's expected a "greater than zero" number of output records for successful jobs. + if (outputRecords <= 0) { + LOG.warn( + "Output records counter for job \"{}\" is {}", + pipeline.getOptions().getJobName(), + outputRecords); + isSuccessful = false; + } + } + tpcdsRunResult = new TpcdsRunResult( isSuccessful, startTimeStamp, endTimeStamp, pipeline.getOptions(), pipelineResult); From abc0a057935d66dadd6841595850329fc84dcba5 Mon Sep 17 00:00:00 2001 From: Bruno Volpato <bvolpato@google.com> Date: Mon, 13 Nov 2023 11:24:36 -0500 Subject: [PATCH 431/435] Mark Avro as provided in harness JAR (#29412) * Mark Avro as provided in harness JAR * Add permitUnusedDeclared for Avro --- sdks/java/harness/build.gradle | 2 ++ 1 file changed, 2 insertions(+) diff --git a/sdks/java/harness/build.gradle b/sdks/java/harness/build.gradle index 50eed2f2573b..937e870a22f4 100644 --- a/sdks/java/harness/build.gradle +++ b/sdks/java/harness/build.gradle @@ -30,6 +30,7 @@ dependencies { provided project(path: ":model:pipeline", configuration: "shadow") provided project(path: ":sdks:java:core", configuration: "shadow") provided project(path: ":sdks:java:transform-service:launcher", configuration: "shadow") + provided library.java.avro provided library.java.joda_time provided library.java.slf4j_api provided library.java.vendored_grpc_1_54_0 @@ -93,4 +94,5 @@ dependencies { shadowTestRuntimeClasspath project(path: ":sdks:java:core", configuration: "shadowTest") testImplementation project(path: ":sdks:java:fn-execution", configuration: "testRuntimeMigration") shadowTestRuntimeClasspath library.java.slf4j_jdk14 + permitUnusedDeclared library.java.avro } From 7057ee634c9739c81621f130c8932684675effd7 Mon Sep 17 00:00:00 2001 From: Jeffrey Kinard <jeff@thekinards.com> Date: Mon, 13 Nov 2023 11:35:51 -0500 Subject: [PATCH 432/435] [YAML] Add yaml metadata to provider schema Signed-off-by: Jeffrey Kinard <jeff@thekinards.com> --- sdks/python/apache_beam/yaml/pipeline.schema.yaml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/sdks/python/apache_beam/yaml/pipeline.schema.yaml b/sdks/python/apache_beam/yaml/pipeline.schema.yaml index e6b10f06e9a6..40f576c1618b 100644 --- a/sdks/python/apache_beam/yaml/pipeline.schema.yaml +++ b/sdks/python/apache_beam/yaml/pipeline.schema.yaml @@ -146,6 +146,8 @@ $defs: additionalProperties: type: string config: { type: object } + __line__: {} + __uuid__: {} additionalProperties: false required: - type From 618d7a8f2c6520bfae1ca84dced7fd39d7af45f4 Mon Sep 17 00:00:00 2001 From: Jack McCluskey <34928439+jrmccluskey@users.noreply.github.com> Date: Mon, 13 Nov 2023 13:01:20 -0500 Subject: [PATCH 433/435] Establish list to collection compatibility (#29417) --- sdks/python/apache_beam/typehints/typehints.py | 12 ++++++++---- sdks/python/apache_beam/typehints/typehints_test.py | 2 ++ 2 files changed, 10 insertions(+), 4 deletions(-) diff --git a/sdks/python/apache_beam/typehints/typehints.py b/sdks/python/apache_beam/typehints/typehints.py index 5726a8a8ca92..4fd4b97e82cd 100644 --- a/sdks/python/apache_beam/typehints/typehints.py +++ b/sdks/python/apache_beam/typehints/typehints.py @@ -1038,13 +1038,13 @@ def __repr__(self): @staticmethod def _is_subclass_constraint(sub): return isinstance( - sub, ( + sub, + ( CollectionTypeConstraint, FrozenSetTypeConstraint, - SetTypeConstraint)) + SetTypeConstraint, + ListConstraint)) - # TODO(https://github.com/apache/beam/issues/29135): allow for consistency - # with Mapping types def _consistent_with_check_(self, sub): if self._is_subclass_constraint(sub): return is_consistent_with(sub.inner_type, self.inner_type) @@ -1058,6 +1058,10 @@ def _consistent_with_check_(self, sub): return all( is_consistent_with(elem, self.inner_type) for elem in sub.tuple_types) + # TODO(https://github.com/apache/beam/issues/29135): allow for + # consistency checks with Mapping types + elif isinstance(sub, DictConstraint): + return True elif not isinstance(sub, TypeConstraint): if getattr(sub, '__origin__', None) is not None and getattr( sub, '__args__', None) is not None: diff --git a/sdks/python/apache_beam/typehints/typehints_test.py b/sdks/python/apache_beam/typehints/typehints_test.py index 1d938edcc24b..c395893a23ba 100644 --- a/sdks/python/apache_beam/typehints/typehints_test.py +++ b/sdks/python/apache_beam/typehints/typehints_test.py @@ -874,6 +874,7 @@ def test_type_constraint_compatibility(self): typehints.Collection[typehints.Any], typehints.Collection[int]) self.assertCompatible(typehints.Collection[int], typehints.Tuple[int]) self.assertCompatible(typehints.Any, typehints.Collection[str]) + self.assertCompatible(typehints.Collection[str], typehints.List[str]) def test_one_way_compatibility(self): self.assertNotCompatible(typehints.Set[int], typehints.Collection[int]) @@ -881,6 +882,7 @@ def test_one_way_compatibility(self): typehints.FrozenSet[int], typehints.Collection[int]) self.assertNotCompatible(typehints.Tuple[int], typehints.Collection[int]) self.assertNotCompatible(typehints.Collection[int], typehints.Iterable[int]) + self.assertNotCompatible(typehints.List[str], typehints.Collection[str]) def test_getitem_invalid_composite_type_param(self): with self.assertRaises(TypeError) as e: From b6b1902620a22bea23b56e9b873f5025f79e5edb Mon Sep 17 00:00:00 2001 From: Ahmet Altay <aaltay@gmail.com> Date: Mon, 13 Nov 2023 12:42:39 -0800 Subject: [PATCH 434/435] Create contributor-spotlight-johanna-ojeling.md (#29408) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * Create contributor-spotlight-johanna-ojeling.md Add Contributor Spotlight: Johanna Öjeling blog post. * Update contributor-spotlight-johanna-ojeling.md remove trailing space --- .../contributor-spotlight-johanna-ojeling.md | 63 +++++++++++++++++++ 1 file changed, 63 insertions(+) create mode 100644 website/www/site/content/en/blog/contributor-spotlight-johanna-ojeling.md diff --git a/website/www/site/content/en/blog/contributor-spotlight-johanna-ojeling.md b/website/www/site/content/en/blog/contributor-spotlight-johanna-ojeling.md new file mode 100644 index 000000000000..717f591eca92 --- /dev/null +++ b/website/www/site/content/en/blog/contributor-spotlight-johanna-ojeling.md @@ -0,0 +1,63 @@ +--- +title: "Contributor Spotlight: Johanna Öjeling" +date: 2023-11-11 15:00:00 -0800 +categories: + - blog +authors: + - altay +--- +<!-- +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at +http://www.apache.org/licenses/LICENSE-2.0 +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +--> + +Johanna Öjeling is a Senior Software Engineer at [Normative](https://normative.io/). She started using Apache Beam in 2020 at her previous company [Datatonic](http://datatonic.com) and began contributing in 2022 at a personal capacity. We interviewed Johanna to learn more about her interests and we hope that this will inspire new, future, diverse set of contributors to participate in OSS projects. + +**What areas of interest are you passionate about in your career?** + +My core interest lies in distributed and data-intensive systems, and I enjoy working on challenges related to performance, scalability and maintainability. I also feel strongly about developer experience, and like to build tools and frameworks that make developers happier and more productive. Aside from that, I take pleasure in mentoring and coaching other software engineers to grow their skills and pursue a fulfilling career. + +**What motivated you to make your first contribution?** + +I was already a user of the Apache Beam Java and Python SDKs and Google Cloud Dataflow in my previous job, and had started to play around with the Go SDK to learn Go. When I noticed that a feature I wanted was missing, it seemed like a great opportunity to implement it. I had been curious about developing open source software for some time, but did not have a good idea until then of what to contribute with. + +**In which way have you contributed to Apache Beam?** + +I have primarily worked on the Go SDK with implementation of new features, bug fixes, tests, documentation and code reviews. Some examples include a MongoDB I/O connector with dynamically scalable reads and writes, a file I/O connector supporting continuous file discovery, and an Amazon S3 file system implementation. + +**How has your open source engagement impacted your personal or professional growth?** + +Contributing to open source is one of the best decisions I have taken professionally. The Beam community has been incredibly welcoming and appreciative, and it has been rewarding to collaborate with talented people around the world to create software that is free for anyone to benefit from. Open source has opened up new opportunities to challenge myself, dive deeper into technologies I like, and learn from highly skilled professionals. To me, it has served as an outlet for creativity, problem solving and purposeful work. + +**How have you noticed contributing to open source is different from contributing to closed source/proprietary software?** + +My observation has been that there are higher requirements for software quality in open source, and it is more important to get things right the first time. My closed source software experience is from startups/scale-ups where speed is prioritized. When not working on public facing APIs or libraries, one can also more easily change things, whereas we need to be mindful about breaking changes in Beam. I care for software quality and value the high standards the Beam committers hold. + +**What do you like to do with your spare time when you're not contributing to Beam?** + +Coding is a passion of mine so I tend to spend a lot of my free time on hobby projects, reading books and articles, listening to talks and attending events. When I was younger I loved learning foreign languages and studied English, French, German and Spanish. Later I discovered an interest in computer science and switched focus to programming languages. I decided to change careers to software engineering and have tried to learn as much as possible ever since. I love that it never ends. + +**What future features/improvements are you most excited about, or would you like to see on Beam?** + +The multi-language pipeline support is an impressive feature of Beam, and I like that new SDKs such as TypeScript and Swift are emerging, which enables developers to write pipelines in their preferred language. Naturally, I am also excited to see where the Go SDK is headed and how we can make use of newer features of the Go language. + +**What types of contributions or support do you think the Beam community needs more of?** + +Many data and machine learning engineers feel more comfortable with Python than Java and wish the Python SDK were as feature rich as the Java SDK. This presents great opportunities for Python developers to start contributing to Beam. As an SDK author, one can take advantage of Beam's multiple SDKs. When I have developed in Go I have often studied the Java and Python implementations to get ideas for how to solve specific problems and make sure the Go SDK follows a similar pattern. + +**What advice would you give to someone who wants to contribute but does not know where to begin?** + +Start with asking yourself what prior knowledge you have and what you would like to learn, then look for opportunities that match that. The contribution guidelines will tell you where to find open issues and what the process looks like. There are tasks labeled as "good first issue" which can be a good starting point. I was quite nervous about making my first contribution and had my mentor pre-review my PR. There was no need to worry though, as people will be grateful for your effort to improve the project. The pride I felt when a committer approved my PR and welcomed me to Beam is something I still remember. + +**What advice would you give to the Beam community? What could we improve?** + +We can make it easier for new community members to get involved by providing more examples of tasks that we need help with, both in the form of code and non-code contributions. I will take it as an action point myself to label more issues accordingly and tailor the descriptions for newcomers. However, this is contingent on community members visiting the GitHub project. To address this, we could also proactively promote opportunities through social channels and the user mailing list. + +*We thank Johanna for the interview and for her contributions! If you would like to learn more about contributing to Beam you can learn more about it here: https://beam.apache.org/contribute/.* From 7fabc1267a0b174cb117320dd375447402797e0c Mon Sep 17 00:00:00 2001 From: Shunping Huang <shunping@google.com> Date: Mon, 13 Nov 2023 18:58:38 -0500 Subject: [PATCH 435/435] Fix the test 'test_user_agent_passed' without skipping it (#29422) --- sdks/python/apache_beam/io/gcp/bigquery_tools_test.py | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/io/gcp/bigquery_tools_test.py b/sdks/python/apache_beam/io/gcp/bigquery_tools_test.py index b4c84d589c07..0383db7925f2 100644 --- a/sdks/python/apache_beam/io/gcp/bigquery_tools_test.py +++ b/sdks/python/apache_beam/io/gcp/bigquery_tools_test.py @@ -263,8 +263,14 @@ def test_temporary_dataset_is_unique(self, patched_time_sleep): wrapper.create_temporary_dataset('project-id', 'location') self.assertTrue(client.datasets.Get.called) + @mock.patch( + 'apache_beam.io.gcp.bigquery_tools.gcp_bigquery', + return_value=mock.Mock()) + @mock.patch( + 'apitools.base.py.base_api._SkipGetCredentials', return_value=True) @mock.patch('time.sleep', return_value=None) - def test_user_agent_passed(self, sleep_mock): + def test_user_agent_passed( + self, sleep_mock, skip_get_credentials_mock, gcp_bigquery_mock): try: wrapper = beam.io.gcp.bigquery_tools.BigQueryWrapper() except: # pylint: disable=bare-except